diff --git a/.github/PULL_REQUEST_TEMPLATE b/.github/PULL_REQUEST_TEMPLATE new file mode 100644 index 000000000000..5af45d6fa798 --- /dev/null +++ b/.github/PULL_REQUEST_TEMPLATE @@ -0,0 +1,10 @@ +## What changes were proposed in this pull request? + +(Please fill in changes proposed in this fix) + +## How was this patch tested? + +(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) +(If this patch involves UI changes, please attach a screenshot; otherwise, remove this) + +Please review http://spark.apache.org/contributing.html before opening a pull request. diff --git a/.gitignore b/.gitignore index debad77ec2ad..903297db9690 100644 --- a/.gitignore +++ b/.gitignore @@ -1,76 +1,93 @@ -*~ -*.#* *#*# -*.swp -*.ipr +*.#* *.iml +*.ipr *.iws *.pyc *.pyo +*.swp +*~ +.DS_Store +.cache +.classpath +.ensime +.ensime_cache/ +.ensime_lucene +.generated-mima* .idea/ .idea_modules/ -build/*.jar +.project +.pydevproject +.scala_dependencies .settings -.cache -cache -.generated-mima* -work/ -out/ -.DS_Store -third_party/libmesos.so -third_party/libmesos.dylib +/lib/ +R-unit-tests.log +R/unit-tests.out +R/cran-check.out +R/pkg/vignettes/sparkr-vignettes.html +R/pkg/tests/fulltests/Rplots.pdf +build/*.jar build/apache-maven* -build/zinc* build/scala* -conf/java-opts -conf/*.sh +build/zinc* +cache +checkpoint conf/*.cmd -conf/*.properties conf/*.conf +conf/*.properties +conf/*.sh conf/*.xml +conf/java-opts conf/slaves +dependency-reduced-pom.xml +derby.log +dev/create-release/*final +dev/create-release/*txt +dev/pr-deps/ +dist/ docs/_site docs/api -target/ -reports/ -.project -.classpath -.scala_dependencies +sql/docs +sql/site lib_managed/ -src_managed/ +lint-r-report.log +log/ +logs/ +out/ project/boot/ -project/plugins/project/build.properties project/build/target/ -project/plugins/target/ project/plugins/lib_managed/ +project/plugins/project/build.properties project/plugins/src_managed/ -logs/ -log/ +project/plugins/target/ +python/lib/pyspark.zip +python/deps +python/pyspark/python +reports/ +scalastyle-on-compile.generated.xml +scalastyle-output.xml +scalastyle.txt +spark-*-bin-*.tgz spark-tests.log +src_managed/ streaming-tests.log -dependency-reduced-pom.xml -.ensime -.ensime_lucene -checkpoint -derby.log -dist/ -dev/create-release/*txt -dev/create-release/*final -spark-*-bin-*.tgz +target/ unit-tests.log -/lib/ -ec2/lib/ -rat-results.txt -scalastyle.txt -scalastyle-output.xml -R-unit-tests.log -R/unit-tests.out -python/lib/pyspark.zip -lint-r-report.log +work/ # For Hive -metastore_db/ -metastore/ -warehouse/ TempStatsStore/ +metastore/ +metastore_db/ sql/hive-thriftserver/test_warehouses +warehouse/ +spark-warehouse/ + +# For R session data +.RData +.RHistory +.Rhistory +*.Rproj +*.Rproj.* + +.Rproj.user diff --git a/.rat-excludes b/.rat-excludes deleted file mode 100644 index 08fba6d351d6..000000000000 --- a/.rat-excludes +++ /dev/null @@ -1,85 +0,0 @@ -target -cache -.gitignore -.gitattributes -.project -.classpath -.mima-excludes -.generated-mima-excludes -.generated-mima-class-excludes -.generated-mima-member-excludes -.rat-excludes -.*md -derby.log -TAGS -RELEASE -control -docs -slaves -spark-env.cmd -bootstrap-tooltip.js -jquery-1.11.1.min.js -d3.min.js -dagre-d3.min.js -graphlib-dot.min.js -sorttable.js -vis.min.js -vis.min.css -.*avsc -.*txt -.*json -.*data -.*log -cloudpickle.py -heapq3.py -join.py -SparkExprTyper.scala -SparkILoop.scala -SparkILoopInit.scala -SparkIMain.scala -SparkImports.scala -SparkJLineCompletion.scala -SparkJLineReader.scala -SparkMemberHandlers.scala -SparkReplReporter.scala -sbt -sbt-launch-lib.bash -plugins.sbt -work -.*\.q -.*\.qv -golden -test.out/* -.*iml -service.properties -db.lck -build/* -dist/* -.*out -.*ipr -.*iws -logs -.*scalastyle-output.xml -.*dependency-reduced-pom.xml -known_translations -json_expectation -local-1422981759269/* -local-1422981780767/* -local-1425081759269/* -local-1426533911241/* -local-1426633911242/* -local-1430917381534/* -local-1430917381535_1 -local-1430917381535_2 -DESCRIPTION -NAMESPACE -test_support/* -.*Rd -help/* -html/* -INDEX -.lintr -gen-java.* -.*avpr -org.apache.spark.sql.sources.DataSourceRegister -.*parquet diff --git a/.travis.yml b/.travis.yml new file mode 100644 index 000000000000..d7e9f8c0290e --- /dev/null +++ b/.travis.yml @@ -0,0 +1,50 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Spark provides this Travis CI configuration file to help contributors +# check Scala/Java style conformance and JDK7/8 compilation easily +# during their preparing pull requests. +# - Scalastyle is executed during `maven install` implicitly. +# - Java Checkstyle is executed by `lint-java`. +# See the related discussion here. +# https://github.com/apache/spark/pull/12980 + +# 1. Choose OS (Ubuntu 14.04.3 LTS Server Edition 64bit, ~2 CORE, 7.5GB RAM) +sudo: required +dist: trusty + +# 2. Choose language and target JDKs for parallel builds. +language: java +jdk: + - oraclejdk8 + +# 3. Setup cache directory for SBT and Maven. +cache: + directories: + - $HOME/.sbt + - $HOME/.m2 + +# 4. Turn off notifications. +notifications: + email: false + +# 5. Run maven install before running lint-java. +install: + - export MAVEN_SKIP_RC=1 + - build/mvn -T 4 -q -DskipTests -Pmesos -Pyarn -Pkinesis-asl -Phive -Phive-thriftserver install + +# 6. Run lint-java. +script: + - dev/lint-java diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index f10d7e277eea..8fdd5aa9e7df 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -1,12 +1,12 @@ ## Contributing to Spark *Before opening a pull request*, review the -[Contributing to Spark wiki](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark). +[Contributing to Spark guide](http://spark.apache.org/contributing.html). It lists steps that are required before creating a PR. In particular, consider: - Is the change important and ready enough to ask the community to spend time reviewing? - Have you searched for existing, related JIRAs and pull requests? -- Is this a new feature that can stand alone as a package on http://spark-packages.org ? +- Is this a new feature that can stand alone as a [third party project](http://spark.apache.org/third-party-projects.html) ? - Is the change being proposed clearly explained and motivated? When you contribute code, you affirm that the contribution is your original work and that you diff --git a/LICENSE b/LICENSE index 0db2d14465bd..39fe0dc46238 100644 --- a/LICENSE +++ b/LICENSE @@ -1,4 +1,3 @@ - Apache License Version 2.0, January 2004 http://www.apache.org/licenses/ @@ -237,9 +236,9 @@ The following components are provided under a BSD-style license. See project lin The text of each license is also included at licenses/LICENSE-[project].txt. (BSD 3 Clause) netlib core (com.github.fommil.netlib:core:1.1.2 - https://github.com/fommil/netlib-java/core) - (BSD 3 Clause) JPMML-Model (org.jpmml:pmml-model:1.1.15 - https://github.com/jpmml/jpmml-model) - (BSD 3-clause style license) jblas (org.jblas:jblas:1.2.4 - http://jblas.org/) + (BSD 3 Clause) JPMML-Model (org.jpmml:pmml-model:1.2.7 - https://github.com/jpmml/jpmml-model) (BSD License) AntLR Parser Generator (antlr:antlr:2.7.7 - http://www.antlr.org/) + (BSD License) ANTLR 4.5.2-1 (org.antlr:antlr4:4.5.2-1 - http://wwww.antlr.org/) (BSD licence) ANTLR ST4 4.0.4 (org.antlr:ST4:4.0.4 - http://www.stringtemplate.org) (BSD licence) ANTLR StringTemplate (org.antlr:stringtemplate:3.2.1 - http://www.stringtemplate.org) (BSD License) Javolution (javolution:javolution:5.5.1 - http://javolution.org) @@ -250,22 +249,21 @@ The text of each license is also included at licenses/LICENSE-[project].txt. (Interpreter classes (all .scala files in repl/src/main/scala except for Main.Scala, SparkHelper.scala and ExecutorClassLoader.scala), and for SerializableMapWrapper in JavaUtils.scala) - (BSD-like) Scala Actors library (org.scala-lang:scala-actors:2.10.5 - http://www.scala-lang.org/) - (BSD-like) Scala Compiler (org.scala-lang:scala-compiler:2.10.5 - http://www.scala-lang.org/) - (BSD-like) Scala Compiler (org.scala-lang:scala-reflect:2.10.5 - http://www.scala-lang.org/) - (BSD-like) Scala Library (org.scala-lang:scala-library:2.10.5 - http://www.scala-lang.org/) - (BSD-like) Scalap (org.scala-lang:scalap:2.10.5 - http://www.scala-lang.org/) - (BSD-style) scalacheck (org.scalacheck:scalacheck_2.10:1.10.0 - http://www.scalacheck.org) - (BSD-style) spire (org.spire-math:spire_2.10:0.7.1 - http://spire-math.org) - (BSD-style) spire-macros (org.spire-math:spire-macros_2.10:0.7.1 - http://spire-math.org) - (New BSD License) Kryo (com.esotericsoftware.kryo:kryo:2.21 - http://code.google.com/p/kryo/) - (New BSD License) MinLog (com.esotericsoftware.minlog:minlog:1.2 - http://code.google.com/p/minlog/) - (New BSD License) ReflectASM (com.esotericsoftware.reflectasm:reflectasm:1.07 - http://code.google.com/p/reflectasm/) + (BSD-like) Scala Actors library (org.scala-lang:scala-actors:2.11.8 - http://www.scala-lang.org/) + (BSD-like) Scala Compiler (org.scala-lang:scala-compiler:2.11.8 - http://www.scala-lang.org/) + (BSD-like) Scala Compiler (org.scala-lang:scala-reflect:2.11.8 - http://www.scala-lang.org/) + (BSD-like) Scala Library (org.scala-lang:scala-library:2.11.8 - http://www.scala-lang.org/) + (BSD-like) Scalap (org.scala-lang:scalap:2.11.8 - http://www.scala-lang.org/) + (BSD-style) scalacheck (org.scalacheck:scalacheck_2.11:1.10.0 - http://www.scalacheck.org) + (BSD-style) spire (org.spire-math:spire_2.11:0.7.1 - http://spire-math.org) + (BSD-style) spire-macros (org.spire-math:spire-macros_2.11:0.7.1 - http://spire-math.org) + (New BSD License) Kryo (com.esotericsoftware:kryo:3.0.3 - https://github.com/EsotericSoftware/kryo) + (New BSD License) MinLog (com.esotericsoftware:minlog:1.3.0 - https://github.com/EsotericSoftware/minlog) (New BSD license) Protocol Buffer Java API (com.google.protobuf:protobuf-java:2.5.0 - http://code.google.com/p/protobuf) (New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf) (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net) (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net) - (The New BSD License) Py4J (net.sf.py4j:py4j:0.9 - http://py4j.sourceforge.net/) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.10.6 - http://py4j.sourceforge.net/) (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/) (BSD licence) sbt and sbt-launch-lib.bash (BSD 3 Clause) d3.min.js (https://github.com/mbostock/d3/blob/master/LICENSE) @@ -284,7 +282,7 @@ The text of each license is also included at licenses/LICENSE-[project].txt. (MIT License) SLF4J API Module (org.slf4j:slf4j-api:1.7.5 - http://www.slf4j.org) (MIT License) SLF4J LOG4J-12 Binding (org.slf4j:slf4j-log4j12:1.7.5 - http://www.slf4j.org) (MIT License) pyrolite (org.spark-project:pyrolite:2.0.1 - http://pythonhosted.org/Pyro4/) - (MIT License) scopt (com.github.scopt:scopt_2.10:3.2.0 - https://github.com/scopt/scopt) + (MIT License) scopt (com.github.scopt:scopt_2.11:3.2.0 - https://github.com/scopt/scopt) (The MIT License) Mockito (org.mockito:mockito-core:1.9.5 - http://www.mockito.org) (MIT License) jquery (https://jquery.org/license/) (MIT License) AnchorJS (https://github.com/bryanbraun/anchorjs) @@ -292,3 +290,11 @@ The text of each license is also included at licenses/LICENSE-[project].txt. (MIT License) dagre-d3 (https://github.com/cpettitt/dagre-d3) (MIT License) sorttable (https://github.com/stuartlangridge/sorttable) (MIT License) boto (https://github.com/boto/boto/blob/develop/LICENSE) + (MIT License) datatables (http://datatables.net/license) + (MIT License) mustache (https://github.com/mustache/mustache/blob/master/LICENSE) + (MIT License) cookies (http://code.google.com/p/cookies/wiki/License) + (MIT License) blockUI (http://jquery.malsup.com/block/) + (MIT License) RowsGroup (http://datatables.net/license/mit) + (MIT License) jsonFormatter (http://www.jqueryscript.net/other/jQuery-Plugin-For-Pretty-JSON-Formatting-jsonFormatter.html) + (MIT License) modernizr (https://github.com/Modernizr/Modernizr/blob/master/LICENSE) + (MIT License) machinist (https://github.com/typelevel/machinist) diff --git a/NOTICE b/NOTICE index 7f7769f73047..f4b64b5c3f47 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache Spark -Copyright 2014 The Apache Software Foundation. +Copyright 2014 and onwards The Apache Software Foundation. This product includes software developed at The Apache Software Foundation (http://www.apache.org/). @@ -12,7 +12,9 @@ Common Development and Distribution License 1.0 The following components are provided under the Common Development and Distribution License 1.0. See project link for details. (CDDL 1.0) Glassfish Jasper (org.mortbay.jetty:jsp-2.1:6.1.14 - http://jetty.mortbay.org/project/modules/jsp-2.1) + (CDDL 1.0) JAX-RS (https://jax-rs-spec.java.net/) (CDDL 1.0) Servlet Specification 2.5 API (org.mortbay.jetty:servlet-api-2.5:6.1.14 - http://jetty.mortbay.org/project/modules/servlet-api-2.5) + (CDDL 1.0) (GPL2 w/ CPE) javax.annotation API (https://glassfish.java.net/nonav/public/CDDL+GPL.html) (COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0) (GNU General Public Library) Streaming API for XML (javax.xml.stream:stax-api:1.0-2 - no url defined) (Common Development and Distribution License (CDDL) v1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp) @@ -22,15 +24,10 @@ Common Development and Distribution License 1.1 The following components are provided under the Common Development and Distribution License 1.1. See project link for details. + (CDDL 1.1) (GPL2 w/ CPE) org.glassfish.hk2 (https://hk2.java.net) (CDDL 1.1) (GPL2 w/ CPE) JAXB API bundle for GlassFish V3 (javax.xml.bind:jaxb-api:2.2.2 - https://jaxb.dev.java.net/) (CDDL 1.1) (GPL2 w/ CPE) JAXB RI (com.sun.xml.bind:jaxb-impl:2.2.3-1 - http://jaxb.java.net/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:1.8 - https://jersey.dev.java.net/jersey-core/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:1.9 - https://jersey.java.net/jersey-core/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-guice (com.sun.jersey.contribs:jersey-guice:1.9 - https://jersey.java.net/jersey-contribs/jersey-guice/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:1.8 - https://jersey.dev.java.net/jersey-json/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:1.9 - https://jersey.java.net/jersey-json/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:1.8 - https://jersey.dev.java.net/jersey-server/) - (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:1.9 - https://jersey.java.net/jersey-server/) + (CDDL 1.1) (GPL2 w/ CPE) Jersey 2 (https://jersey.java.net) ======================================================================== Common Public License 1.0 @@ -48,7 +45,6 @@ Eclipse Public License 1.0 The following components are provided under the Eclipse Public License 1.0. See project link for details. - (Eclipse Public License - Version 1.0) mqtt-client (org.eclipse.paho:mqtt-client:0.4.0 - http://www.eclipse.org/paho/mqtt-client) (Eclipse Public License v1.0) Eclipse JDT Core (org.eclipse.jdt:core:3.1.1 - http://www.eclipse.org/jdt/) ======================================================================== @@ -425,9 +421,6 @@ Copyright (c) 2011, Terrence Parr. This product includes/uses ASM (http://asm.ow2.org/), Copyright (c) 2000-2007 INRIA, France Telecom. -This product includes/uses org.json (http://www.json.org/java/index.html), -Copyright (c) 2002 JSON.org - This product includes/uses JLine (http://jline.sourceforge.net/), Copyright (c) 2002-2006, Marc Prud'hommeaux . @@ -606,4 +599,63 @@ Vis.js uses and redistributes the following third-party libraries: - keycharm https://github.com/AlexDM0/keycharm - The MIT License \ No newline at end of file + The MIT License + +=============================================================================== + +The CSS style for the navigation sidebar of the documentation was originally +submitted by Óscar Nájera for the scikit-learn project. The scikit-learn project +is distributed under the 3-Clause BSD license. +=============================================================================== + +For CSV functionality: + +/* + * Copyright 2014 Databricks + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Copyright 2015 Ayasdi Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +=============================================================================== +For dev/sparktestsupport/toposort.py: + +Copyright 2014 True Blade Systems, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. diff --git a/R/.gitignore b/R/.gitignore index 9a5889ba28b2..c98504ab0778 100644 --- a/R/.gitignore +++ b/R/.gitignore @@ -4,3 +4,5 @@ lib pkg/man pkg/html +SparkR.Rcheck/ +SparkR_*.tar.gz diff --git a/R/CRAN_RELEASE.md b/R/CRAN_RELEASE.md new file mode 100644 index 000000000000..d6084c7a7cc9 --- /dev/null +++ b/R/CRAN_RELEASE.md @@ -0,0 +1,91 @@ +# SparkR CRAN Release + +To release SparkR as a package to CRAN, we would use the `devtools` package. Please work with the +`dev@spark.apache.org` community and R package maintainer on this. + +### Release + +First, check that the `Version:` field in the `pkg/DESCRIPTION` file is updated. Also, check for stale files not under source control. + +Note that while `run-tests.sh` runs `check-cran.sh` (which runs `R CMD check`), it is doing so with `--no-manual --no-vignettes`, which skips a few vignettes or PDF checks - therefore it will be preferred to run `R CMD check` on the source package built manually before uploading a release. Also note that for CRAN checks for pdf vignettes to success, `qpdf` tool must be there (to install it, eg. `yum -q -y install qpdf`). + +To upload a release, we would need to update the `cran-comments.md`. This should generally contain the results from running the `check-cran.sh` script along with comments on status of all `WARNING` (should not be any) or `NOTE`. As a part of `check-cran.sh` and the release process, the vignettes is build - make sure `SPARK_HOME` is set and Spark jars are accessible. + +Once everything is in place, run in R under the `SPARK_HOME/R` directory: + +```R +paths <- .libPaths(); .libPaths(c("lib", paths)); Sys.setenv(SPARK_HOME=tools::file_path_as_absolute("..")); devtools::release(); .libPaths(paths) +``` + +For more information please refer to http://r-pkgs.had.co.nz/release.html#release-check + +### Testing: build package manually + +To build package manually such as to inspect the resulting `.tar.gz` file content, we would also use the `devtools` package. + +Source package is what get released to CRAN. CRAN would then build platform-specific binary packages from the source package. + +#### Build source package + +To build source package locally without releasing to CRAN, run in R under the `SPARK_HOME/R` directory: + +```R +paths <- .libPaths(); .libPaths(c("lib", paths)); Sys.setenv(SPARK_HOME=tools::file_path_as_absolute("..")); devtools::build("pkg"); .libPaths(paths) +``` + +(http://r-pkgs.had.co.nz/vignettes.html#vignette-workflow-2) + +Similarly, the source package is also created by `check-cran.sh` with `R CMD build pkg`. + +For example, this should be the content of the source package: + +```sh +DESCRIPTION R inst tests +NAMESPACE build man vignettes + +inst/doc/ +sparkr-vignettes.html +sparkr-vignettes.Rmd +sparkr-vignettes.Rman + +build/ +vignette.rds + +man/ + *.Rd files... + +vignettes/ +sparkr-vignettes.Rmd +``` + +#### Test source package + +To install, run this: + +```sh +R CMD INSTALL SparkR_2.1.0.tar.gz +``` + +With "2.1.0" replaced with the version of SparkR. + +This command installs SparkR to the default libPaths. Once that is done, you should be able to start R and run: + +```R +library(SparkR) +vignette("sparkr-vignettes", package="SparkR") +``` + +#### Build binary package + +To build binary package locally, run in R under the `SPARK_HOME/R` directory: + +```R +paths <- .libPaths(); .libPaths(c("lib", paths)); Sys.setenv(SPARK_HOME=tools::file_path_as_absolute("..")); devtools::build("pkg", binary = TRUE); .libPaths(paths) +``` + +For example, this should be the content of the binary package: + +```sh +DESCRIPTION Meta R html tests +INDEX NAMESPACE help profile worker +``` diff --git a/R/DOCUMENTATION.md b/R/DOCUMENTATION.md index 931d01549b26..7314a1fcccda 100644 --- a/R/DOCUMENTATION.md +++ b/R/DOCUMENTATION.md @@ -1,12 +1,12 @@ # SparkR Documentation -SparkR documentation is generated using in-source comments annotated using using -`roxygen2`. After making changes to the documentation, to generate man pages, +SparkR documentation is generated by using in-source comments and annotated by using +[`roxygen2`](https://cran.r-project.org/web/packages/roxygen2/index.html). After making changes to the documentation and generating man pages, you can run the following from an R console in the SparkR home directory - - library(devtools) - devtools::document(pkg="./pkg", roclets=c("rd")) - +```R +library(devtools) +devtools::document(pkg="./pkg", roclets=c("rd")) +``` You can verify if your changes are good by running R CMD check pkg/ diff --git a/R/README.md b/R/README.md index 005f56da1670..1152b1e8e5f9 100644 --- a/R/README.md +++ b/R/README.md @@ -2,13 +2,25 @@ SparkR is an R package that provides a light-weight frontend to use Spark from R. +### Installing sparkR + +Libraries of sparkR need to be created in `$SPARK_HOME/R/lib`. This can be done by running the script `$SPARK_HOME/R/install-dev.sh`. +By default the above script uses the system wide installation of R. However, this can be changed to any user installed location of R by setting the environment variable `R_HOME` the full path of the base directory where R is installed, before running install-dev.sh script. +Example: +```bash +# where /home/username/R is where R is installed and /home/username/R/bin contains the files R and RScript +export R_HOME=/home/username/R +./install-dev.sh +``` + ### SparkR development #### Build Spark Build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn) and include the `-Psparkr` profile to build the R package. For example to use the default Hadoop versions you can run -``` - build/mvn -DskipTests -Psparkr package + +```bash +build/mvn -DskipTests -Psparkr package ``` #### Running sparkR @@ -27,41 +39,39 @@ To set other options like driver memory, executor memory etc. you can pass in th #### Using SparkR from RStudio -If you wish to use SparkR from RStudio or other R frontends you will need to set some environment variables which point SparkR to your Spark installation. For example -``` +If you wish to use SparkR from RStudio or other R frontends you will need to set some environment variables which point SparkR to your Spark installation. For example +```R # Set this to where Spark is installed -Sys.setenv(SPARK_HOME="/Users/shivaram/spark") +Sys.setenv(SPARK_HOME="/Users/username/spark") # This line loads SparkR from the installed directory .libPaths(c(file.path(Sys.getenv("SPARK_HOME"), "R", "lib"), .libPaths())) library(SparkR) -sc <- sparkR.init(master="local") +sparkR.session() ``` #### Making changes to SparkR -The [instructions](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) for making contributions to Spark also apply to SparkR. +The [instructions](http://spark.apache.org/contributing.html) for making contributions to Spark also apply to SparkR. If you only make R file changes (i.e. no Scala changes) then you can just re-install the R package using `R/install-dev.sh` and test your changes. -Once you have made your changes, please include unit tests for them and run existing unit tests using the `run-tests.sh` script as described below. - +Once you have made your changes, please include unit tests for them and run existing unit tests using the `R/run-tests.sh` script as described below. + #### Generating documentation -The SparkR documentation (Rd files and HTML files) are not a part of the source repository. To generate them you can run the script `R/create-docs.sh`. This script uses `devtools` and `knitr` to generate the docs and these packages need to be installed on the machine before using the script. - +The SparkR documentation (Rd files and HTML files) are not a part of the source repository. To generate them you can run the script `R/create-docs.sh`. This script uses `devtools` and `knitr` to generate the docs and these packages need to be installed on the machine before using the script. Also, you may need to install these [prerequisites](https://github.com/apache/spark/tree/master/docs#prerequisites). See also, `R/DOCUMENTATION.md` + ### Examples, Unit tests SparkR comes with several sample programs in the `examples/src/main/r` directory. -To run one of them, use `./bin/sparkR `. For example: - - ./bin/sparkR examples/src/main/r/dataframe.R - -You can also run the unit-tests for SparkR by running (you need to install the [testthat](http://cran.r-project.org/web/packages/testthat/index.html) package first): - - R -e 'install.packages("testthat", repos="http://cran.us.r-project.org")' - ./R/run-tests.sh +To run one of them, use `./bin/spark-submit `. For example: +```bash +./bin/spark-submit examples/src/main/r/dataframe.R +``` +You can run R unit tests by following the instructions under [Running R Tests](http://spark.apache.org/docs/latest/building-spark.html#running-r-tests). ### Running on YARN -The `./bin/spark-submit` and `./bin/sparkR` can also be used to submit jobs to YARN clusters. You will need to set YARN conf dir before doing so. For example on CDH you can run -``` + +The `./bin/spark-submit` can also be used to submit jobs to YARN clusters. You will need to set YARN conf dir before doing so. For example on CDH you can run +```bash export YARN_CONF_DIR=/etc/hadoop/conf ./bin/spark-submit --master yarn examples/src/main/r/dataframe.R ``` diff --git a/R/WINDOWS.md b/R/WINDOWS.md index 3f889c0ca3d1..124bc631be9c 100644 --- a/R/WINDOWS.md +++ b/R/WINDOWS.md @@ -4,10 +4,39 @@ To build SparkR on Windows, the following steps are required 1. Install R (>= 3.1) and [Rtools](http://cran.r-project.org/bin/windows/Rtools/). Make sure to include Rtools and R in `PATH`. + 2. Install -[JDK7](http://www.oracle.com/technetwork/java/javase/downloads/jdk7-downloads-1880260.html) and set +[JDK8](http://www.oracle.com/technetwork/java/javase/downloads/jdk8-downloads-2133151.html) and set `JAVA_HOME` in the system environment variables. + 3. Download and install [Maven](http://maven.apache.org/download.html). Also include the `bin` directory in Maven in `PATH`. + 4. Set `MAVEN_OPTS` as described in [Building Spark](http://spark.apache.org/docs/latest/building-spark.html). -5. Open a command shell (`cmd`) in the Spark directory and run `mvn -DskipTests -Psparkr package` + +5. Open a command shell (`cmd`) in the Spark directory and build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn) and include the `-Psparkr` profile to build the R package. For example to use the default Hadoop versions you can run + + ```bash + mvn.cmd -DskipTests -Psparkr package + ``` + + `.\build\mvn` is a shell script so `mvn.cmd` should be used directly on Windows. + +## Unit tests + +To run the SparkR unit tests on Windows, the following steps are required —assuming you are in the Spark root directory and do not have Apache Hadoop installed already: + +1. Create a folder to download Hadoop related files for Windows. For example, `cd ..` and `mkdir hadoop`. + +2. Download the relevant Hadoop bin package from [steveloughran/winutils](https://github.com/steveloughran/winutils). While these are not official ASF artifacts, they are built from the ASF release git hashes by a Hadoop PMC member on a dedicated Windows VM. For further reading, consult [Windows Problems on the Hadoop wiki](https://wiki.apache.org/hadoop/WindowsProblems). + +3. Install the files into `hadoop\bin`; make sure that `winutils.exe` and `hadoop.dll` are present. + +4. Set the environment variable `HADOOP_HOME` to the full path to the newly created `hadoop` directory. + +5. Run unit tests for SparkR by running the command below. You need to install the needed packages following the instructions under [Running R Tests](http://spark.apache.org/docs/latest/building-spark.html#running-r-tests) first: + + ``` + .\bin\spark-submit2.cmd --conf spark.hadoop.fs.defaultFS="file:///" R\pkg\tests\run-all.R + ``` + diff --git a/R/check-cran.sh b/R/check-cran.sh new file mode 100755 index 000000000000..22cc9c6b601f --- /dev/null +++ b/R/check-cran.sh @@ -0,0 +1,76 @@ +#!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +set -o pipefail +set -e + +FWDIR="$(cd "`dirname "${BASH_SOURCE[0]}"`"; pwd)" +pushd "$FWDIR" > /dev/null + +. "$FWDIR/find-r.sh" + +# Install the package (this is required for code in vignettes to run when building it later) +# Build the latest docs, but not vignettes, which is built with the package next +. "$FWDIR/install-dev.sh" + +# Build source package with vignettes +SPARK_HOME="$(cd "${FWDIR}"/..; pwd)" +. "${SPARK_HOME}/bin/load-spark-env.sh" +if [ -f "${SPARK_HOME}/RELEASE" ]; then + SPARK_JARS_DIR="${SPARK_HOME}/jars" +else + SPARK_JARS_DIR="${SPARK_HOME}/assembly/target/scala-$SPARK_SCALA_VERSION/jars" +fi + +if [ -d "$SPARK_JARS_DIR" ]; then + # Build a zip file containing the source package with vignettes + SPARK_HOME="${SPARK_HOME}" "$R_SCRIPT_PATH/R" CMD build "$FWDIR/pkg" + + find pkg/vignettes/. -not -name '.' -not -name '*.Rmd' -not -name '*.md' -not -name '*.pdf' -not -name '*.html' -delete +else + echo "Error Spark JARs not found in '$SPARK_HOME'" + exit 1 +fi + +# Run check as-cran. +VERSION=`grep Version "$FWDIR/pkg/DESCRIPTION" | awk '{print $NF}'` + +CRAN_CHECK_OPTIONS="--as-cran" + +if [ -n "$NO_TESTS" ] +then + CRAN_CHECK_OPTIONS=$CRAN_CHECK_OPTIONS" --no-tests" +fi + +if [ -n "$NO_MANUAL" ] +then + CRAN_CHECK_OPTIONS=$CRAN_CHECK_OPTIONS" --no-manual --no-vignettes" +fi + +echo "Running CRAN check with $CRAN_CHECK_OPTIONS options" + +if [ -n "$NO_TESTS" ] && [ -n "$NO_MANUAL" ] +then + "$R_SCRIPT_PATH/R" CMD check $CRAN_CHECK_OPTIONS "SparkR_$VERSION.tar.gz" +else + # This will run tests and/or build vignettes, and require SPARK_HOME + SPARK_HOME="${SPARK_HOME}" "$R_SCRIPT_PATH/R" CMD check $CRAN_CHECK_OPTIONS "SparkR_$VERSION.tar.gz" +fi + +popd > /dev/null diff --git a/R/create-docs.sh b/R/create-docs.sh index d2ae160b5002..310dbc5fb50a 100755 --- a/R/create-docs.sh +++ b/R/create-docs.sh @@ -17,21 +17,31 @@ # limitations under the License. # -# Script to create API docs for SparkR -# This requires `devtools` and `knitr` to be installed on the machine. +# Script to create API docs and vignettes for SparkR +# This requires `devtools`, `knitr` and `rmarkdown` to be installed on the machine. -# After running this script the html docs can be found in +# After running this script the html docs can be found in # $SPARK_HOME/R/pkg/html +# The vignettes can be found in +# $SPARK_HOME/R/pkg/vignettes/sparkr_vignettes.html set -o pipefail set -e # Figure out where the script is -export FWDIR="$(cd "`dirname "$0"`"; pwd)" -pushd $FWDIR +export FWDIR="$(cd "`dirname "${BASH_SOURCE[0]}"`"; pwd)" +export SPARK_HOME="$(cd "`dirname "${BASH_SOURCE[0]}"`"/..; pwd)" + +# Required for setting SPARK_SCALA_VERSION +. "${SPARK_HOME}/bin/load-spark-env.sh" + +echo "Using Scala $SPARK_SCALA_VERSION" + +pushd "$FWDIR" > /dev/null +. "$FWDIR/find-r.sh" # Install the package (this will also generate the Rd files) -./install-dev.sh +. "$FWDIR/install-dev.sh" # Now create HTML files @@ -39,7 +49,7 @@ pushd $FWDIR mkdir -p pkg/html pushd pkg/html -Rscript -e 'libDir <- "../../lib"; library(SparkR, lib.loc=libDir); library(knitr); knit_rd("SparkR", links = tools::findHTMLlinks(paste(libDir, "SparkR", sep="/")))' +"$R_SCRIPT_PATH/Rscript" -e 'libDir <- "../../lib"; library(SparkR, lib.loc=libDir); library(knitr); knit_rd("SparkR", links = tools::findHTMLlinks(paste(libDir, "SparkR", sep="/")))' popd diff --git a/R/create-rd.sh b/R/create-rd.sh new file mode 100755 index 000000000000..ff622a41a46c --- /dev/null +++ b/R/create-rd.sh @@ -0,0 +1,37 @@ +#!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# This scripts packages the SparkR source files (R and C files) and +# creates a package that can be loaded in R. The package is by default installed to +# $FWDIR/lib and the package can be loaded by using the following command in R: +# +# library(SparkR, lib.loc="$FWDIR/lib") +# +# NOTE(shivaram): Right now we use $SPARK_HOME/R/lib to be the installation directory +# to load the SparkR package on the worker nodes. + +set -o pipefail +set -e + +FWDIR="$(cd "`dirname "${BASH_SOURCE[0]}"`"; pwd)" +pushd "$FWDIR" > /dev/null +. "$FWDIR/find-r.sh" + +# Generate Rd files if devtools is installed +"$R_SCRIPT_PATH/Rscript" -e ' if("devtools" %in% rownames(installed.packages())) { library(devtools); devtools::document(pkg="./pkg", roclets=c("rd")) }' diff --git a/R/find-r.sh b/R/find-r.sh new file mode 100755 index 000000000000..690acc083af9 --- /dev/null +++ b/R/find-r.sh @@ -0,0 +1,34 @@ +#!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +if [ -z "$R_SCRIPT_PATH" ] +then + if [ ! -z "$R_HOME" ] + then + R_SCRIPT_PATH="$R_HOME/bin" + else + # if system wide R_HOME is not found, then exit + if [ ! `command -v R` ]; then + echo "Cannot find 'R_HOME'. Please specify 'R_HOME' or make sure R is properly installed." + exit 1 + fi + R_SCRIPT_PATH="$(dirname $(which R))" + fi + echo "Using R_SCRIPT_PATH = ${R_SCRIPT_PATH}" +fi diff --git a/R/install-dev.bat b/R/install-dev.bat index 008a5c668bc4..ed1c91ae3a0f 100644 --- a/R/install-dev.bat +++ b/R/install-dev.bat @@ -25,3 +25,9 @@ set SPARK_HOME=%~dp0.. MKDIR %SPARK_HOME%\R\lib R.exe CMD INSTALL --library="%SPARK_HOME%\R\lib" %SPARK_HOME%\R\pkg\ + +rem Zip the SparkR package so that it can be distributed to worker nodes on YARN +pushd %SPARK_HOME%\R\lib +%JAVA_HOME%\bin\jar.exe cfM "%SPARK_HOME%\R\lib\sparkr.zip" SparkR +popd + diff --git a/R/install-dev.sh b/R/install-dev.sh index 59d98c9c7a64..d61355271830 100755 --- a/R/install-dev.sh +++ b/R/install-dev.sh @@ -29,17 +29,21 @@ set -o pipefail set -e -FWDIR="$(cd `dirname $0`; pwd)" +FWDIR="$(cd "`dirname "${BASH_SOURCE[0]}"`"; pwd)" LIB_DIR="$FWDIR/lib" -mkdir -p $LIB_DIR +mkdir -p "$LIB_DIR" -pushd $FWDIR > /dev/null +pushd "$FWDIR" > /dev/null +. "$FWDIR/find-r.sh" -# Generate Rd files if devtools is installed -Rscript -e ' if("devtools" %in% rownames(installed.packages())) { library(devtools); devtools::document(pkg="./pkg", roclets=c("rd")) }' +. "$FWDIR/create-rd.sh" # Install SparkR to $LIB_DIR -R CMD INSTALL --library=$LIB_DIR $FWDIR/pkg/ +"$R_SCRIPT_PATH/R" CMD INSTALL --library="$LIB_DIR" "$FWDIR/pkg/" + +# Zip the SparkR package so that it can be distributed to worker nodes on YARN +cd "$LIB_DIR" +jar cfM "$LIB_DIR/sparkr.zip" SparkR popd > /dev/null diff --git a/R/install-source-package.sh b/R/install-source-package.sh new file mode 100755 index 000000000000..8de3569d1d48 --- /dev/null +++ b/R/install-source-package.sh @@ -0,0 +1,57 @@ +#!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# This scripts packages the SparkR source files (R and C files) and +# creates a package that can be loaded in R. The package is by default installed to +# $FWDIR/lib and the package can be loaded by using the following command in R: +# +# library(SparkR, lib.loc="$FWDIR/lib") +# +# NOTE(shivaram): Right now we use $SPARK_HOME/R/lib to be the installation directory +# to load the SparkR package on the worker nodes. + +set -o pipefail +set -e + +FWDIR="$(cd "`dirname "${BASH_SOURCE[0]}"`"; pwd)" +pushd "$FWDIR" > /dev/null +. "$FWDIR/find-r.sh" + +if [ -z "$VERSION" ]; then + VERSION=`grep Version "$FWDIR/pkg/DESCRIPTION" | awk '{print $NF}'` +fi + +if [ ! -f "$FWDIR/SparkR_$VERSION.tar.gz" ]; then + echo -e "R source package file '$FWDIR/SparkR_$VERSION.tar.gz' is not found." + echo -e "Please build R source package with check-cran.sh" + exit -1; +fi + +echo "Removing lib path and installing from source package" +LIB_DIR="$FWDIR/lib" +rm -rf "$LIB_DIR" +mkdir -p "$LIB_DIR" +"$R_SCRIPT_PATH/R" CMD INSTALL "SparkR_$VERSION.tar.gz" --library="$LIB_DIR" + +# Zip the SparkR package so that it can be distributed to worker nodes on YARN +pushd "$LIB_DIR" > /dev/null +jar cfM "$LIB_DIR/sparkr.zip" SparkR +popd > /dev/null + +popd diff --git a/R/pkg/.Rbuildignore b/R/pkg/.Rbuildignore new file mode 100644 index 000000000000..18b2db69db8f --- /dev/null +++ b/R/pkg/.Rbuildignore @@ -0,0 +1,9 @@ +^.*\.Rproj$ +^\.Rproj\.user$ +^\.lintr$ +^cran-comments\.md$ +^NEWS\.md$ +^README\.Rmd$ +^src-native$ +^html$ +^tests/fulltests/* diff --git a/R/pkg/.lintr b/R/pkg/.lintr index 038236fc149e..ae50b28ec616 100644 --- a/R/pkg/.lintr +++ b/R/pkg/.lintr @@ -1,2 +1,2 @@ -linters: with_defaults(line_length_linter(100), camel_case_linter = NULL, open_curly_linter(allow_single_line = TRUE), closed_curly_linter(allow_single_line = TRUE)) +linters: with_defaults(line_length_linter(100), multiple_dots_linter = NULL, camel_case_linter = NULL, open_curly_linter(allow_single_line = TRUE), closed_curly_linter(allow_single_line = TRUE)) exclusions: list("inst/profile/general.R" = 1, "inst/profile/shell.R") diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 3d6edb70ec98..d1c846c04827 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,37 +1,61 @@ Package: SparkR Type: Package -Title: R frontend for Spark -Version: 1.6.0 -Date: 2013-09-09 -Author: The Apache Software Foundation -Maintainer: Shivaram Venkataraman -Imports: - methods +Version: 2.3.0 +Title: R Frontend for Apache Spark +Description: Provides an R Frontend for Apache Spark. +Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), + email = "shivaram@cs.berkeley.edu"), + person("Xiangrui", "Meng", role = "aut", + email = "meng@databricks.com"), + person("Felix", "Cheung", role = "aut", + email = "felixcheung@apache.org"), + person(family = "The Apache Software Foundation", role = c("aut", "cph"))) +License: Apache License (== 2.0) +URL: http://www.apache.org/ http://spark.apache.org/ +BugReports: http://spark.apache.org/contributing.html Depends: R (>= 3.0), - methods, + methods Suggests: - testthat -Description: R frontend for Spark -License: Apache License (== 2.0) + knitr, + rmarkdown, + testthat, + e1071, + survival Collate: 'schema.R' 'generics.R' 'jobj.R' - 'RDD.R' - 'pairRDD.R' 'column.R' 'group.R' + 'RDD.R' + 'pairRDD.R' 'DataFrame.R' 'SQLContext.R' + 'WindowSpec.R' 'backend.R' 'broadcast.R' + 'catalog.R' 'client.R' 'context.R' 'deserialize.R' 'functions.R' - 'mllib.R' + 'install.R' + 'jvm.R' + 'mllib_classification.R' + 'mllib_clustering.R' + 'mllib_fpm.R' + 'mllib_recommendation.R' + 'mllib_regression.R' + 'mllib_stat.R' + 'mllib_tree.R' + 'mllib_utils.R' 'serialize.R' 'sparkR.R' 'stats.R' + 'streaming.R' + 'types.R' 'utils.R' + 'window.R' +RoxygenNote: 5.0.1 +VignetteBuilder: knitr diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index cd9537a2655f..3fc756b9ef40 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -1,39 +1,117 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Imports from base R -importFrom(methods, setGeneric, setMethod, setOldClass) +# Do not include stats:: "rpois", "runif" - causes error at runtime +importFrom("methods", "setGeneric", "setMethod", "setOldClass") +importFrom("methods", "is", "new", "signature", "show") +importFrom("stats", "gaussian", "setNames") +importFrom("utils", "download.file", "object.size", "packageVersion", "tail", "untar") # Disable native libraries till we figure out how to package it # See SPARKR-7839 #useDynLib(SparkR, stringHashCode) # S3 methods exported +export("sparkR.session") export("sparkR.init") export("sparkR.stop") +export("sparkR.session.stop") +export("sparkR.conf") +export("sparkR.version") +export("sparkR.uiWebUrl") export("print.jobj") +export("sparkR.newJObject") +export("sparkR.callJMethod") +export("sparkR.callJStatic") + +export("install.spark") + +export("sparkRSQL.init", + "sparkRHive.init") + # MLlib integration exportMethods("glm", + "spark.glm", "predict", - "summary") + "summary", + "spark.kmeans", + "fitted", + "spark.mlp", + "spark.naiveBayes", + "spark.survreg", + "spark.lda", + "spark.posterior", + "spark.perplexity", + "spark.isoreg", + "spark.gaussianMixture", + "spark.als", + "spark.kstest", + "spark.logit", + "spark.decisionTree", + "spark.randomForest", + "spark.gbt", + "spark.bisectingKmeans", + "spark.svmLinear", + "spark.fpGrowth", + "spark.freqItemsets", + "spark.associationRules") # Job group lifecycle management methods export("setJobGroup", "clearJobGroup", - "cancelJobGroup") + "cancelJobGroup", + "setJobDescription") -exportClasses("DataFrame") +# Export Utility methods +export("setLogLevel") + +exportClasses("SparkDataFrame") exportMethods("arrange", + "as.data.frame", "attach", + "broadcast", "cache", + "checkpoint", + "coalesce", "collect", + "colnames", + "colnames<-", + "coltypes", + "coltypes<-", "columns", "count", "cov", "corr", + "covar_samp", + "covar_pop", + "createOrReplaceTempView", + "crossJoin", "crosstab", + "cube", + "dapply", + "dapplyCollect", "describe", "dim", "distinct", + "drop", + "dropDuplicates", "dropna", "dtypes", "except", @@ -42,27 +120,35 @@ exportMethods("arrange", "filter", "first", "freqItems", + "gapply", + "gapplyCollect", + "getNumPartitions", "group_by", "groupBy", "head", + "hint", "insertInto", "intersect", "isLocal", + "isStreaming", "join", "limit", "merge", "mutate", "na.omit", "names", + "names<-", "ncol", "nrow", "orderBy", "persist", "printSchema", + "randomSplit", "rbind", "registerTempTable", "rename", "repartition", + "rollup", "sample", "sample_frac", "sampleBy", @@ -74,27 +160,42 @@ exportMethods("arrange", "selectExpr", "show", "showDF", + "storageLevel", "subset", "summarize", "summary", "take", + "toJSON", "transform", + "union", "unionAll", + "unionByName", "unique", "unpersist", "where", + "with", "withColumn", "withColumnRenamed", - "write.df") + "write.df", + "write.jdbc", + "write.json", + "write.orc", + "write.parquet", + "write.stream", + "write.text", + "write.ml") exportClasses("Column") -exportMethods("%in%", +exportMethods("%<=>%", + "%in%", "abs", "acos", "add_months", "alias", "approxCountDistinct", + "approxQuantile", + "array_contains", "asc", "ascii", "asin", @@ -105,10 +206,13 @@ exportMethods("%in%", "between", "bin", "bitwiseNOT", + "bround", "cast", "cbrt", "ceil", "ceiling", + "collect_list", + "collect_set", "column", "concat", "concat_ws", @@ -119,18 +223,24 @@ exportMethods("%in%", "count", "countDistinct", "crc32", - "cumeDist", + "create_array", + "create_map", + "hash", + "cume_dist", "date_add", "date_format", "date_sub", "datediff", "dayofmonth", "dayofyear", - "denseRank", + "decode", + "dense_rank", "desc", + "encode", "endsWith", "exp", "explode", + "explode_outer", "expm1", "expr", "factorial", @@ -138,20 +248,28 @@ exportMethods("%in%", "floor", "format_number", "format_string", + "from_json", "from_unixtime", "from_utc_timestamp", "getField", "getItem", "greatest", + "grouping_bit", + "grouping_id", "hex", + "histogram", "hour", "hypot", "ifelse", "initcap", + "input_file_name", "instr", "isNaN", "isNotNull", "isNull", + "is.nan", + "isnan", + "kurtosis", "lag", "last", "last_day", @@ -169,11 +287,14 @@ exportMethods("%in%", "lower", "lpad", "ltrim", + "map_keys", + "map_values", "max", "md5", "mean", "min", "minute", + "monotonically_increasing_id", "month", "months_between", "n", @@ -181,21 +302,26 @@ exportMethods("%in%", "nanvl", "negate", "next_day", + "not", "ntile", "otherwise", - "percentRank", + "over", + "percent_rank", "pmod", + "posexplode", + "posexplode_outer", "quarter", "rand", "randn", "rank", "regexp_extract", "regexp_replace", + "repeat_string", "reverse", "rint", "rlike", "round", - "rowNumber", + "row_number", "rpad", "rtrim", "second", @@ -204,12 +330,21 @@ exportMethods("%in%", "shiftLeft", "shiftRight", "shiftRightUnsigned", + "sd", "sign", "signum", "sin", "sinh", "size", + "skewness", + "sort_array", "soundex", + "spark_partition_id", + "split_string", + "stddev", + "stddev_pop", + "stddev_samp", + "struct", "sqrt", "startsWith", "substr", @@ -221,46 +356,119 @@ exportMethods("%in%", "toDegrees", "toRadians", "to_date", + "to_json", + "to_timestamp", "to_utc_timestamp", "translate", "trim", + "trunc", "unbase64", "unhex", "unix_timestamp", "upper", + "var", + "variance", + "var_pop", + "var_samp", "weekofyear", "when", + "window", "year") exportClasses("GroupedData") exportMethods("agg") - -export("sparkRSQL.init", - "sparkRHive.init") +exportMethods("pivot") export("as.DataFrame", "cacheTable", "clearCache", "createDataFrame", "createExternalTable", + "createTable", + "currentDatabase", "dropTempTable", + "dropTempView", "jsonFile", + "listColumns", + "listDatabases", + "listFunctions", + "listTables", "loadDF", "parquetFile", "read.df", + "read.jdbc", + "read.json", + "read.orc", + "read.parquet", + "read.stream", + "read.text", + "recoverPartitions", + "refreshByPath", + "refreshTable", + "setCheckpointDir", + "setCurrentDatabase", + "spark.lapply", + "spark.addFile", + "spark.getSparkFilesRootDirectory", + "spark.getSparkFiles", "sql", - "table", + "str", + "tableToDF", "tableNames", "tables", - "uncacheTable") + "uncacheTable", + "print.summary.GeneralizedLinearRegressionModel", + "read.ml", + "print.summary.KSTest", + "print.summary.DecisionTreeRegressionModel", + "print.summary.DecisionTreeClassificationModel", + "print.summary.RandomForestRegressionModel", + "print.summary.RandomForestClassificationModel", + "print.summary.GBTRegressionModel", + "print.summary.GBTClassificationModel") export("structField", "structField.jobj", "structField.character", "print.structField", "structType", + "structType.character", "structType.jobj", "structType.structField", "print.structType") -export("as.data.frame") +exportClasses("WindowSpec") + +export("partitionBy", + "rowsBetween", + "rangeBetween") + +export("windowPartitionBy", + "windowOrderBy") + +exportClasses("StreamingQuery") + +export("awaitTermination", + "isActive", + "lastProgress", + "queryName", + "status", + "stopQuery") + + +S3method(print, jobj) +S3method(print, structField) +S3method(print, structType) +S3method(print, summary.GeneralizedLinearRegressionModel) +S3method(print, summary.KSTest) +S3method(print, summary.DecisionTreeRegressionModel) +S3method(print, summary.DecisionTreeClassificationModel) +S3method(print, summary.RandomForestRegressionModel) +S3method(print, summary.RandomForestClassificationModel) +S3method(print, summary.GBTRegressionModel) +S3method(print, summary.GBTClassificationModel) +S3method(structField, character) +S3method(structField, jobj) +S3method(structType, character) +S3method(structType, jobj) +S3method(structType, structField) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index df5bc8137187..0728141fa483 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -15,36 +15,39 @@ # limitations under the License. # -# DataFrame.R - DataFrame class and methods implemented in S4 OO classes +# DataFrame.R - SparkDataFrame class and methods implemented in S4 OO classes #' @include generics.R jobj.R schema.R RDD.R pairRDD.R column.R group.R NULL setOldClass("jobj") +setOldClass("structType") -#' @title S4 class that represents a DataFrame -#' @description DataFrames can be created using functions like \link{createDataFrame}, -#' \link{jsonFile}, \link{table} etc. -#' @family dataframe_funcs -#' @rdname DataFrame +#' S4 class that represents a SparkDataFrame +#' +#' SparkDataFrames can be created using functions like \link{createDataFrame}, +#' \link{read.json}, \link{table} etc. +#' +#' @family SparkDataFrame functions +#' @rdname SparkDataFrame #' @docType class #' -#' @slot env An R environment that stores bookkeeping states of the DataFrame +#' @slot env An R environment that stores bookkeeping states of the SparkDataFrame #' @slot sdf A Java object reference to the backing Scala DataFrame -#' @seealso \link{createDataFrame}, \link{jsonFile}, \link{table} +#' @seealso \link{createDataFrame}, \link{read.json}, \link{table} #' @seealso \url{https://spark.apache.org/docs/latest/sparkr.html#sparkr-dataframes} #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' df <- createDataFrame(sqlContext, faithful) +#' sparkR.session() +#' df <- createDataFrame(faithful) #'} -setClass("DataFrame", +#' @note SparkDataFrame since 2.0.0 +setClass("SparkDataFrame", slots = list(env = "environment", sdf = "jobj")) -setMethod("initialize", "DataFrame", function(.Object, sdf, isCached) { +setMethod("initialize", "SparkDataFrame", function(.Object, sdf, isCached) { .Object@env <- new.env() .Object@env$isCached <- isCached @@ -52,36 +55,50 @@ setMethod("initialize", "DataFrame", function(.Object, sdf, isCached) { .Object }) -#' @rdname DataFrame +#' Set options/mode and then return the write object +#' @noRd +setWriteOptions <- function(write, path = NULL, mode = "error", ...) { + options <- varargsToStrEnv(...) + if (!is.null(path)) { + options[["path"]] <- path + } + jmode <- convertToJSaveMode(mode) + write <- callJMethod(write, "mode", jmode) + write <- callJMethod(write, "options", options) + write +} + #' @export #' @param sdf A Java object reference to the backing Scala DataFrame -#' @param isCached TRUE if the dataFrame is cached +#' @param isCached TRUE if the SparkDataFrame is cached +#' @noRd dataFrame <- function(sdf, isCached = FALSE) { - new("DataFrame", sdf, isCached) + new("SparkDataFrame", sdf, isCached) } -############################ DataFrame Methods ############################################## +############################ SparkDataFrame Methods ############################################## -#' Print Schema of a DataFrame +#' Print Schema of a SparkDataFrame #' #' Prints out the schema in tree format #' -#' @param x A SparkSQL DataFrame +#' @param x A SparkDataFrame #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname printSchema #' @name printSchema +#' @aliases printSchema,SparkDataFrame-method #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' printSchema(df) #'} +#' @note printSchema since 1.4.0 setMethod("printSchema", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { schemaString <- callJMethod(schema(x)$jobj, "treeString") cat(schemaString) @@ -89,24 +106,25 @@ setMethod("printSchema", #' Get schema object #' -#' Returns the schema of this DataFrame as a structType object. +#' Returns the schema of this SparkDataFrame as a structType object. #' -#' @param x A SparkSQL DataFrame +#' @param x A SparkDataFrame #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname schema #' @name schema +#' @aliases schema,SparkDataFrame-method #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' dfSchema <- schema(df) #'} +#' @note schema since 1.4.0 setMethod("schema", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { structType(callJMethod(x@sdf, "schema")) }) @@ -115,22 +133,21 @@ setMethod("schema", #' #' Print the logical and physical Catalyst plans to the console for debugging. #' -#' @param x A SparkSQL DataFrame -#' @param extended Logical. If extended is False, explain() only prints the physical plan. -#' @family dataframe_funcs +#' @family SparkDataFrame functions +#' @aliases explain,SparkDataFrame-method #' @rdname explain #' @name explain #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' explain(df, TRUE) #'} +#' @note explain since 1.4.0 setMethod("explain", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x, extended = FALSE) { queryExec <- callJMethod(x@sdf, "queryExecution") if (extended) { @@ -143,230 +160,429 @@ setMethod("explain", #' isLocal #' -#' Returns True if the `collect` and `take` methods can be run locally +#' Returns True if the \code{collect} and \code{take} methods can be run locally #' (without any Spark executors). #' -#' @param x A SparkSQL DataFrame +#' @param x A SparkDataFrame #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname isLocal #' @name isLocal +#' @aliases isLocal,SparkDataFrame-method #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' isLocal(df) #'} +#' @note isLocal since 1.4.0 setMethod("isLocal", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { callJMethod(x@sdf, "isLocal") }) #' showDF #' -#' Print the first numRows rows of a DataFrame -#' -#' @param x A SparkSQL DataFrame -#' @param numRows The number of rows to print. Defaults to 20. -#' -#' @family dataframe_funcs +#' Print the first numRows rows of a SparkDataFrame +#' +#' @param x a SparkDataFrame. +#' @param numRows the number of rows to print. Defaults to 20. +#' @param truncate whether truncate long strings. If \code{TRUE}, strings more than +#' 20 characters will be truncated. However, if set greater than zero, +#' truncates strings longer than \code{truncate} characters and all cells +#' will be aligned right. +#' @param vertical whether print output rows vertically (one line per column value). +#' @param ... further arguments to be passed to or from other methods. +#' @family SparkDataFrame functions +#' @aliases showDF,SparkDataFrame-method #' @rdname showDF #' @name showDF #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' showDF(df) #'} +#' @note showDF since 1.4.0 setMethod("showDF", - signature(x = "DataFrame"), - function(x, numRows = 20, truncate = TRUE) { - s <- callJMethod(x@sdf, "showString", numToInt(numRows), truncate) + signature(x = "SparkDataFrame"), + function(x, numRows = 20, truncate = TRUE, vertical = FALSE) { + if (is.logical(truncate) && truncate) { + s <- callJMethod(x@sdf, "showString", numToInt(numRows), numToInt(20), vertical) + } else { + truncate2 <- as.numeric(truncate) + s <- callJMethod(x@sdf, "showString", numToInt(numRows), numToInt(truncate2), + vertical) + } cat(s) }) #' show #' -#' Print the DataFrame column names and types +#' Print class and type information of a Spark object. #' -#' @param x A SparkSQL DataFrame +#' @param object a Spark object. Can be a SparkDataFrame, Column, GroupedData, WindowSpec. #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname show +#' @aliases show,SparkDataFrame-method #' @name show #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) -#' df +#' df <- read.json(path) +#' show(df) #'} -setMethod("show", "DataFrame", +#' @note show(SparkDataFrame) since 1.4.0 +setMethod("show", "SparkDataFrame", function(object) { cols <- lapply(dtypes(object), function(l) { paste(l, collapse = ":") }) s <- paste(cols, collapse = ", ") - cat(paste("DataFrame[", s, "]\n", sep = "")) + cat(paste(class(object), "[", s, "]\n", sep = "")) }) #' DataTypes #' #' Return all column names and their data types as a list #' -#' @param x A SparkSQL DataFrame +#' @param x A SparkDataFrame #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname dtypes #' @name dtypes +#' @aliases dtypes,SparkDataFrame-method #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' dtypes(df) #'} +#' @note dtypes since 1.4.0 setMethod("dtypes", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { lapply(schema(x)$fields(), function(f) { c(f$name(), f$dataType.simpleString()) }) }) -#' Column names +#' Column Names of SparkDataFrame #' -#' Return all column names as a list +#' Return a vector of column names. #' -#' @param x A SparkSQL DataFrame +#' @param x a SparkDataFrame. #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname columns #' @name columns -#' @aliases names +#' @aliases columns,SparkDataFrame-method #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' columns(df) +#' colnames(df) #'} +#' @note columns since 1.4.0 setMethod("columns", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { sapply(schema(x)$fields(), function(f) { f$name() }) }) -#' @family dataframe_funcs #' @rdname columns #' @name names +#' @aliases names,SparkDataFrame-method +#' @note names since 1.5.0 setMethod("names", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { columns(x) }) -#' @family dataframe_funcs #' @rdname columns +#' @aliases names<-,SparkDataFrame-method #' @name names<- +#' @note names<- since 1.5.0 setMethod("names<-", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x, value) { - if (!is.null(value)) { - sdf <- callJMethod(x@sdf, "toDF", as.list(value)) - dataFrame(sdf) + colnames(x) <- value + x + }) + +#' @rdname columns +#' @aliases colnames,SparkDataFrame-method +#' @name colnames +#' @note colnames since 1.6.0 +setMethod("colnames", + signature(x = "SparkDataFrame"), + function(x) { + columns(x) + }) + +#' @param value a character vector. Must have the same length as the number +#' of columns to be renamed. +#' @rdname columns +#' @aliases colnames<-,SparkDataFrame-method +#' @name colnames<- +#' @note colnames<- since 1.6.0 +setMethod("colnames<-", + signature(x = "SparkDataFrame"), + function(x, value) { + + # Check parameter integrity + if (class(value) != "character") { + stop("Invalid column names.") + } + + if (length(value) != ncol(x)) { + stop( + "Column names must have the same length as the number of columns in the dataset.") + } + + if (any(is.na(value))) { + stop("Column names cannot be NA.") + } + + # Check if the column names have . in it + if (any(regexec(".", value, fixed = TRUE)[[1]][1] != -1)) { + stop("Column names cannot contain the '.' symbol.") + } + + sdf <- callJMethod(x@sdf, "toDF", as.list(value)) + dataFrame(sdf) + }) + +#' coltypes +#' +#' Get column types of a SparkDataFrame +#' +#' @param x A SparkDataFrame +#' @return value A character vector with the column types of the given SparkDataFrame +#' @rdname coltypes +#' @aliases coltypes,SparkDataFrame-method +#' @name coltypes +#' @family SparkDataFrame functions +#' @export +#' @examples +#'\dontrun{ +#' irisDF <- createDataFrame(iris) +#' coltypes(irisDF) # get column types +#'} +#' @note coltypes since 1.6.0 +setMethod("coltypes", + signature(x = "SparkDataFrame"), + function(x) { + # Get the data types of the SparkDataFrame by invoking dtypes() function + types <- sapply(dtypes(x), function(x) {x[[2]]}) + + # Map Spark data types into R's data types using DATA_TYPES environment + rTypes <- sapply(types, USE.NAMES = F, FUN = function(x) { + # Check for primitive types + type <- PRIMITIVE_TYPES[[x]] + + if (is.null(type)) { + # Check for complex types + for (t in names(COMPLEX_TYPES)) { + if (substring(x, 1, nchar(t)) == t) { + type <- COMPLEX_TYPES[[t]] + break + } + } + + if (is.null(type)) { + specialtype <- specialtypeshandle(x) + if (is.null(specialtype)) { + stop(paste("Unsupported data type: ", x)) + } + type <- PRIMITIVE_TYPES[[specialtype]] + } + } + type[[1]] + }) + + # Find which types don't have mapping to R + naIndices <- which(is.na(rTypes)) + + # Assign the original scala data types to the unmatched ones + rTypes[naIndices] <- types[naIndices] + + rTypes + }) + +#' coltypes +#' +#' Set the column types of a SparkDataFrame. +#' +#' @param value A character vector with the target column types for the given +#' SparkDataFrame. Column types can be one of integer, numeric/double, character, logical, or NA +#' to keep that column as-is. +#' @rdname coltypes +#' @name coltypes<- +#' @aliases coltypes<-,SparkDataFrame,character-method +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' coltypes(df) <- c("character", "integer") # set column types +#' coltypes(df) <- c(NA, "numeric") # set column types +#'} +#' @note coltypes<- since 1.6.0 +setMethod("coltypes<-", + signature(x = "SparkDataFrame", value = "character"), + function(x, value) { + cols <- columns(x) + ncols <- length(cols) + if (length(value) == 0) { + stop("Cannot set types of an empty SparkDataFrame with no Column") } + if (length(value) != ncols) { + stop("Length of type vector should match the number of columns for SparkDataFrame") + } + newCols <- lapply(seq_len(ncols), function(i) { + col <- getColumn(x, cols[i]) + if (!is.na(value[i])) { + stype <- rToSQLTypes[[value[i]]] + if (is.null(stype)) { + stop("Only atomic type is supported for column types") + } + cast(col, stype) + } else { + col + } + }) + nx <- select(x, newCols) + dataFrame(nx@sdf) }) -#' Register Temporary Table +#' Creates a temporary view using the given name. +#' +#' Creates a new temporary view using a SparkDataFrame in the Spark Session. If a +#' temporary view with the same name already exists, replaces it. #' -#' Registers a DataFrame as a Temporary Table in the SQLContext +#' @param x A SparkDataFrame +#' @param viewName A character vector containing the name of the table #' -#' @param x A SparkSQL DataFrame +#' @family SparkDataFrame functions +#' @rdname createOrReplaceTempView +#' @name createOrReplaceTempView +#' @aliases createOrReplaceTempView,SparkDataFrame,character-method +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' createOrReplaceTempView(df, "json_df") +#' new_df <- sql("SELECT * FROM json_df") +#'} +#' @note createOrReplaceTempView since 2.0.0 +setMethod("createOrReplaceTempView", + signature(x = "SparkDataFrame", viewName = "character"), + function(x, viewName) { + invisible(callJMethod(x@sdf, "createOrReplaceTempView", viewName)) + }) + +#' (Deprecated) Register Temporary Table +#' +#' Registers a SparkDataFrame as a Temporary Table in the SparkSession +#' @param x A SparkDataFrame #' @param tableName A character vector containing the name of the table #' -#' @family dataframe_funcs -#' @rdname registerTempTable +#' @family SparkDataFrame functions +#' @seealso \link{createOrReplaceTempView} +#' @rdname registerTempTable-deprecated #' @name registerTempTable +#' @aliases registerTempTable,SparkDataFrame,character-method #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' registerTempTable(df, "json_df") -#' new_df <- sql(sqlContext, "SELECT * FROM json_df") +#' new_df <- sql("SELECT * FROM json_df") #'} +#' @note registerTempTable since 1.4.0 setMethod("registerTempTable", - signature(x = "DataFrame", tableName = "character"), + signature(x = "SparkDataFrame", tableName = "character"), function(x, tableName) { - invisible(callJMethod(x@sdf, "registerTempTable", tableName)) + .Deprecated("createOrReplaceTempView") + invisible(callJMethod(x@sdf, "createOrReplaceTempView", tableName)) }) #' insertInto #' -#' Insert the contents of a DataFrame into a table registered in the current SQL Context. +#' Insert the contents of a SparkDataFrame into a table registered in the current SparkSession. #' -#' @param x A SparkSQL DataFrame -#' @param tableName A character vector containing the name of the table -#' @param overwrite A logical argument indicating whether or not to overwrite +#' @param x a SparkDataFrame. +#' @param tableName a character vector containing the name of the table. +#' @param overwrite a logical argument indicating whether or not to overwrite. +#' @param ... further arguments to be passed to or from other methods. #' the existing rows in the table. #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname insertInto #' @name insertInto +#' @aliases insertInto,SparkDataFrame,character-method #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' df <- read.df(sqlContext, path, "parquet") -#' df2 <- read.df(sqlContext, path2, "parquet") -#' registerTempTable(df, "table1") +#' sparkR.session() +#' df <- read.df(path, "parquet") +#' df2 <- read.df(path2, "parquet") +#' saveAsTable(df, "table1") #' insertInto(df2, "table1", overwrite = TRUE) #'} +#' @note insertInto since 1.4.0 setMethod("insertInto", - signature(x = "DataFrame", tableName = "character"), + signature(x = "SparkDataFrame", tableName = "character"), function(x, tableName, overwrite = FALSE) { - callJMethod(x@sdf, "insertInto", tableName, overwrite) + jmode <- convertToJSaveMode(ifelse(overwrite, "overwrite", "append")) + write <- callJMethod(x@sdf, "write") + write <- callJMethod(write, "mode", jmode) + invisible(callJMethod(write, "insertInto", tableName)) }) #' Cache #' #' Persist with the default storage level (MEMORY_ONLY). #' -#' @param x A SparkSQL DataFrame +#' @param x A SparkDataFrame #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions +#' @aliases cache,SparkDataFrame-method #' @rdname cache #' @name cache #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' cache(df) #'} +#' @note cache since 1.4.0 setMethod("cache", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { cached <- callJMethod(x@sdf, "cache") x@env$isCached <- TRUE @@ -375,26 +591,29 @@ setMethod("cache", #' Persist #' -#' Persist this DataFrame with the specified storage level. For details of the +#' Persist this SparkDataFrame with the specified storage level. For details of the #' supported storage levels, refer to -#' \url{http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence}. +#' \url{http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence}. #' -#' @param x The DataFrame to persist +#' @param x the SparkDataFrame to persist. +#' @param newLevel storage level chosen for the persistance. See available options in +#' the description. #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname persist #' @name persist +#' @aliases persist,SparkDataFrame,character-method #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' persist(df, "MEMORY_AND_DISK") #'} +#' @note persist since 1.4.0 setMethod("persist", - signature(x = "DataFrame", newLevel = "character"), + signature(x = "SparkDataFrame", newLevel = "character"), function(x, newLevel) { callJMethod(x@sdf, "persist", getStorageLevel(newLevel)) x@env$isCached <- TRUE @@ -403,299 +622,532 @@ setMethod("persist", #' Unpersist #' -#' Mark this DataFrame as non-persistent, and remove all blocks for it from memory and +#' Mark this SparkDataFrame as non-persistent, and remove all blocks for it from memory and #' disk. #' -#' @param x The DataFrame to unpersist -#' @param blocking Whether to block until all blocks are deleted +#' @param x the SparkDataFrame to unpersist. +#' @param blocking whether to block until all blocks are deleted. +#' @param ... further arguments to be passed to or from other methods. #' -#' @family dataframe_funcs -#' @rdname unpersist-methods +#' @family SparkDataFrame functions +#' @rdname unpersist +#' @aliases unpersist,SparkDataFrame-method #' @name unpersist #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' persist(df, "MEMORY_AND_DISK") #' unpersist(df) #'} +#' @note unpersist since 1.4.0 setMethod("unpersist", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x, blocking = TRUE) { callJMethod(x@sdf, "unpersist", blocking) x@env$isCached <- FALSE x }) -#' Repartition +#' StorageLevel +#' +#' Get storagelevel of this SparkDataFrame. +#' +#' @param x the SparkDataFrame to get the storageLevel. +#' +#' @family SparkDataFrame functions +#' @rdname storageLevel +#' @aliases storageLevel,SparkDataFrame-method +#' @name storageLevel +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' persist(df, "MEMORY_AND_DISK") +#' storageLevel(df) +#'} +#' @note storageLevel since 2.1.0 +setMethod("storageLevel", + signature(x = "SparkDataFrame"), + function(x) { + storageLevelToString(callJMethod(x@sdf, "storageLevel")) + }) + +#' Coalesce +#' +#' Returns a new SparkDataFrame that has exactly \code{numPartitions} partitions. +#' This operation results in a narrow dependency, e.g. if you go from 1000 partitions to 100 +#' partitions, there will not be a shuffle, instead each of the 100 new partitions will claim 10 of +#' the current partitions. If a larger number of partitions is requested, it will stay at the +#' current number of partitions. +#' +#' However, if you're doing a drastic coalesce on a SparkDataFrame, e.g. to numPartitions = 1, +#' this may result in your computation taking place on fewer nodes than +#' you like (e.g. one node in the case of numPartitions = 1). To avoid this, +#' call \code{repartition}. This will add a shuffle step, but means the +#' current upstream partitions will be executed in parallel (per whatever +#' the current partitioning is). #' -#' Return a new DataFrame that has exactly numPartitions partitions. +#' @param numPartitions the number of partitions to use. +#' +#' @family SparkDataFrame functions +#' @rdname coalesce +#' @name coalesce +#' @aliases coalesce,SparkDataFrame-method +#' @seealso \link{repartition} +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' newDF <- coalesce(df, 1L) +#'} +#' @note coalesce(SparkDataFrame) since 2.1.1 +setMethod("coalesce", + signature(x = "SparkDataFrame"), + function(x, numPartitions) { + stopifnot(is.numeric(numPartitions)) + sdf <- callJMethod(x@sdf, "coalesce", numToInt(numPartitions)) + dataFrame(sdf) + }) + +#' Repartition #' -#' @param x A SparkSQL DataFrame -#' @param numPartitions The number of partitions to use. +#' The following options for repartition are possible: +#' \itemize{ +#' \item{1.} {Return a new SparkDataFrame that has exactly \code{numPartitions}.} +#' \item{2.} {Return a new SparkDataFrame hash partitioned by +#' the given columns into \code{numPartitions}.} +#' \item{3.} {Return a new SparkDataFrame hash partitioned by the given column(s), +#' using \code{spark.sql.shuffle.partitions} as number of partitions.} +#'} +#' @param x a SparkDataFrame. +#' @param numPartitions the number of partitions to use. +#' @param col the column by which the partitioning will be performed. +#' @param ... additional column(s) to be used in the partitioning. #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname repartition #' @name repartition +#' @aliases repartition,SparkDataFrame-method +#' @seealso \link{coalesce} #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' newDF <- repartition(df, 2L) +#' newDF <- repartition(df, numPartitions = 2L) +#' newDF <- repartition(df, col = df$"col1", df$"col2") +#' newDF <- repartition(df, 3L, col = df$"col1", df$"col2") #'} +#' @note repartition since 1.4.0 setMethod("repartition", - signature(x = "DataFrame", numPartitions = "numeric"), - function(x, numPartitions) { - sdf <- callJMethod(x@sdf, "repartition", numToInt(numPartitions)) + signature(x = "SparkDataFrame"), + function(x, numPartitions = NULL, col = NULL, ...) { + if (!is.null(numPartitions) && is.numeric(numPartitions)) { + # number of partitions and columns both are specified + if (!is.null(col) && class(col) == "Column") { + cols <- list(col, ...) + jcol <- lapply(cols, function(c) { c@jc }) + sdf <- callJMethod(x@sdf, "repartition", numToInt(numPartitions), jcol) + } else { + # only number of partitions is specified + sdf <- callJMethod(x@sdf, "repartition", numToInt(numPartitions)) + } + } else if (!is.null(col) && class(col) == "Column") { + # only columns are specified + cols <- list(col, ...) + jcol <- lapply(cols, function(c) { c@jc }) + sdf <- callJMethod(x@sdf, "repartition", jcol) + } else { + stop("Please, specify the number of partitions and/or a column(s)") + } dataFrame(sdf) }) -# toJSON -# -# Convert the rows of a DataFrame into JSON objects and return an RDD where -# each element contains a JSON string. -# -# @param x A SparkSQL DataFrame -# @return A StringRRDD of JSON objects -# -# @family dataframe_funcs -# @rdname tojson -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# sqlContext <- sparkRSQL.init(sc) -# path <- "path/to/file.json" -# df <- jsonFile(sqlContext, path) -# newRDD <- toJSON(df) -#} +#' toJSON +#' +#' Converts a SparkDataFrame into a SparkDataFrame of JSON string. +#' +#' Each row is turned into a JSON document with columns as different fields. +#' The returned SparkDataFrame has a single character column with the name \code{value} +#' +#' @param x a SparkDataFrame +#' @return a SparkDataFrame +#' @family SparkDataFrame functions +#' @rdname toJSON +#' @name toJSON +#' @aliases toJSON,SparkDataFrame-method +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.parquet" +#' df <- read.parquet(path) +#' df_json <- toJSON(df) +#'} +#' @note toJSON since 2.2.0 setMethod("toJSON", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { - rdd <- callJMethod(x@sdf, "toJSON") - jrdd <- callJMethod(rdd, "toJavaRDD") - RDD(jrdd, serializedMode = "string") + jsonDS <- callJMethod(x@sdf, "toJSON") + df <- callJMethod(jsonDS, "toDF") + dataFrame(df) }) -#' saveAsParquetFile +#' Save the contents of SparkDataFrame as a JSON file #' -#' Save the contents of a DataFrame as a Parquet file, preserving the schema. Files written out -#' with this method can be read back in as a DataFrame using parquetFile(). +#' Save the contents of a SparkDataFrame as a JSON file (\href{http://jsonlines.org/}{ +#' JSON Lines text format or newline-delimited JSON}). Files written out +#' with this method can be read back in as a SparkDataFrame using read.json(). #' -#' @param x A SparkSQL DataFrame +#' @param x A SparkDataFrame #' @param path The directory where the file is saved +#' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) +#' @param ... additional argument(s) passed to the method. #' -#' @family dataframe_funcs -#' @rdname saveAsParquetFile -#' @name saveAsParquetFile +#' @family SparkDataFrame functions +#' @rdname write.json +#' @name write.json +#' @aliases write.json,SparkDataFrame,character-method +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' write.json(df, "/tmp/sparkr-tmp/") +#'} +#' @note write.json since 1.6.0 +setMethod("write.json", + signature(x = "SparkDataFrame", path = "character"), + function(x, path, mode = "error", ...) { + write <- callJMethod(x@sdf, "write") + write <- setWriteOptions(write, mode = mode, ...) + invisible(handledCallJMethod(write, "json", path)) + }) + +#' Save the contents of SparkDataFrame as an ORC file, preserving the schema. +#' +#' Save the contents of a SparkDataFrame as an ORC file, preserving the schema. Files written out +#' with this method can be read back in as a SparkDataFrame using read.orc(). +#' +#' @param x A SparkDataFrame +#' @param path The directory where the file is saved +#' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) +#' @param ... additional argument(s) passed to the method. +#' +#' @family SparkDataFrame functions +#' @aliases write.orc,SparkDataFrame,character-method +#' @rdname write.orc +#' @name write.orc +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' write.orc(df, "/tmp/sparkr-tmp1/") +#' } +#' @note write.orc since 2.0.0 +setMethod("write.orc", + signature(x = "SparkDataFrame", path = "character"), + function(x, path, mode = "error", ...) { + write <- callJMethod(x@sdf, "write") + write <- setWriteOptions(write, mode = mode, ...) + invisible(handledCallJMethod(write, "orc", path)) + }) + +#' Save the contents of SparkDataFrame as a Parquet file, preserving the schema. +#' +#' Save the contents of a SparkDataFrame as a Parquet file, preserving the schema. Files written out +#' with this method can be read back in as a SparkDataFrame using read.parquet(). +#' +#' @param x A SparkDataFrame +#' @param path The directory where the file is saved +#' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) +#' @param ... additional argument(s) passed to the method. +#' +#' @family SparkDataFrame functions +#' @rdname write.parquet +#' @name write.parquet +#' @aliases write.parquet,SparkDataFrame,character-method #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) -#' saveAsParquetFile(df, "/tmp/sparkr-tmp/") +#' df <- read.json(path) +#' write.parquet(df, "/tmp/sparkr-tmp1/") +#' saveAsParquetFile(df, "/tmp/sparkr-tmp2/") #'} +#' @note write.parquet since 1.6.0 +setMethod("write.parquet", + signature(x = "SparkDataFrame", path = "character"), + function(x, path, mode = "error", ...) { + write <- callJMethod(x@sdf, "write") + write <- setWriteOptions(write, mode = mode, ...) + invisible(handledCallJMethod(write, "parquet", path)) + }) + +#' @rdname write.parquet +#' @name saveAsParquetFile +#' @aliases saveAsParquetFile,SparkDataFrame,character-method +#' @export +#' @note saveAsParquetFile since 1.4.0 setMethod("saveAsParquetFile", - signature(x = "DataFrame", path = "character"), + signature(x = "SparkDataFrame", path = "character"), function(x, path) { - invisible(callJMethod(x@sdf, "saveAsParquetFile", path)) + .Deprecated("write.parquet") + write.parquet(x, path) + }) + +#' Save the content of SparkDataFrame in a text file at the specified path. +#' +#' Save the content of the SparkDataFrame in a text file at the specified path. +#' The SparkDataFrame must have only one column of string type with the name "value". +#' Each row becomes a new line in the output file. +#' +#' @param x A SparkDataFrame +#' @param path The directory where the file is saved +#' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) +#' @param ... additional argument(s) passed to the method. +#' +#' @family SparkDataFrame functions +#' @aliases write.text,SparkDataFrame,character-method +#' @rdname write.text +#' @name write.text +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.txt" +#' df <- read.text(path) +#' write.text(df, "/tmp/sparkr-tmp/") +#'} +#' @note write.text since 2.0.0 +setMethod("write.text", + signature(x = "SparkDataFrame", path = "character"), + function(x, path, mode = "error", ...) { + write <- callJMethod(x@sdf, "write") + write <- setWriteOptions(write, mode = mode, ...) + invisible(handledCallJMethod(write, "text", path)) }) #' Distinct #' -#' Return a new DataFrame containing the distinct rows in this DataFrame. +#' Return a new SparkDataFrame containing the distinct rows in this SparkDataFrame. #' -#' @param x A SparkSQL DataFrame +#' @param x A SparkDataFrame #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions +#' @aliases distinct,SparkDataFrame-method #' @rdname distinct #' @name distinct #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' distinctDF <- distinct(df) #'} +#' @note distinct since 1.4.0 setMethod("distinct", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { sdf <- callJMethod(x@sdf, "distinct") dataFrame(sdf) }) -#' @title Distinct rows in a DataFrame -# -#' @description Returns a new DataFrame containing distinct rows in this DataFrame -#' -#' @family dataframe_funcs -#' @rdname unique +#' @rdname distinct #' @name unique -#' @aliases distinct +#' @aliases unique,SparkDataFrame-method +#' @note unique since 1.5.0 setMethod("unique", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { distinct(x) }) #' Sample #' -#' Return a sampled subset of this DataFrame using a random seed. +#' Return a sampled subset of this SparkDataFrame using a random seed. +#' Note: this is not guaranteed to provide exactly the fraction specified +#' of the total count of of the given SparkDataFrame. #' -#' @param x A SparkSQL DataFrame +#' @param x A SparkDataFrame #' @param withReplacement Sampling with replacement or not #' @param fraction The (rough) sample target fraction +#' @param seed Randomness seed value. Default is a random seed. #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions +#' @aliases sample,SparkDataFrame-method #' @rdname sample -#' @aliases sample_frac +#' @name sample #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) +#' collect(sample(df, fraction = 0.5)) #' collect(sample(df, FALSE, 0.5)) -#' collect(sample(df, TRUE, 0.5)) +#' collect(sample(df, TRUE, 0.5, seed = 3)) #'} +#' @note sample since 1.4.0 setMethod("sample", - # TODO : Figure out how to send integer as java.lang.Long to JVM so - # we can send seed as an argument through callJMethod - signature(x = "DataFrame", withReplacement = "logical", - fraction = "numeric"), - function(x, withReplacement, fraction) { - if (fraction < 0.0) stop(cat("Negative fraction value:", fraction)) - sdf <- callJMethod(x@sdf, "sample", withReplacement, fraction) + signature(x = "SparkDataFrame"), + function(x, withReplacement = FALSE, fraction, seed) { + if (!is.numeric(fraction)) { + stop(paste("fraction must be numeric; however, got", class(fraction))) + } + if (!is.logical(withReplacement)) { + stop(paste("withReplacement must be logical; however, got", class(withReplacement))) + } + + if (!missing(seed)) { + if (is.null(seed)) { + stop("seed must not be NULL or NA; however, got NULL") + } + if (is.na(seed)) { + stop("seed must not be NULL or NA; however, got NA") + } + + # TODO : Figure out how to send integer as java.lang.Long to JVM so + # we can send seed as an argument through callJMethod + sdf <- handledCallJMethod(x@sdf, "sample", as.logical(withReplacement), + as.numeric(fraction), as.integer(seed)) + } else { + sdf <- handledCallJMethod(x@sdf, "sample", + as.logical(withReplacement), as.numeric(fraction)) + } dataFrame(sdf) }) -#' @family dataframe_funcs #' @rdname sample +#' @aliases sample_frac,SparkDataFrame-method #' @name sample_frac +#' @note sample_frac since 1.4.0 setMethod("sample_frac", - signature(x = "DataFrame", withReplacement = "logical", - fraction = "numeric"), - function(x, withReplacement, fraction) { - sample(x, withReplacement, fraction) + signature(x = "SparkDataFrame"), + function(x, withReplacement = FALSE, fraction, seed) { + sample(x, withReplacement, fraction, seed) }) -#' Count -#' -#' Returns the number of rows in a DataFrame +#' Returns the number of rows in a SparkDataFrame #' -#' @param x A SparkSQL DataFrame -#' -#' @family dataframe_funcs -#' @rdname count -#' @name count -#' @aliases nrow +#' @param x a SparkDataFrame. +#' @family SparkDataFrame functions +#' @rdname nrow +#' @name nrow +#' @aliases count,SparkDataFrame-method #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' count(df) #' } +#' @note count since 1.4.0 setMethod("count", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { callJMethod(x@sdf, "count") }) -#' @title Number of rows for a DataFrame -#' @description Returns number of rows in a DataFrames -#' #' @name nrow -#' -#' @family dataframe_funcs #' @rdname nrow -#' @aliases count +#' @aliases nrow,SparkDataFrame-method +#' @note nrow since 1.5.0 setMethod("nrow", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { count(x) }) -#' Returns the number of columns in a DataFrame +#' Returns the number of columns in a SparkDataFrame #' -#' @param x a SparkSQL DataFrame +#' @param x a SparkDataFrame #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname ncol #' @name ncol +#' @aliases ncol,SparkDataFrame-method #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' ncol(df) #' } +#' @note ncol since 1.5.0 setMethod("ncol", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { length(columns(x)) }) -#' Returns the dimentions (number of rows and columns) of a DataFrame -#' @param x a SparkSQL DataFrame +#' Returns the dimensions of SparkDataFrame +#' +#' Returns the dimensions (number of rows and columns) of a SparkDataFrame +#' @param x a SparkDataFrame #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname dim +#' @aliases dim,SparkDataFrame-method #' @name dim #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' dim(df) #' } +#' @note dim since 1.5.0 setMethod("dim", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { c(count(x), ncol(x)) }) -#' Collects all the elements of a Spark DataFrame and coerces them into an R data.frame. +#' Collects all the elements of a SparkDataFrame and coerces them into an R data.frame. #' -#' @param x A SparkSQL DataFrame -#' @param stringsAsFactors (Optional) A logical indicating whether or not string columns +#' @param x a SparkDataFrame. +#' @param stringsAsFactors (Optional) a logical indicating whether or not string columns #' should be converted to factors. FALSE by default. +#' @param ... further arguments to be passed to or from other methods. #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname collect +#' @aliases collect,SparkDataFrame-method #' @name collect #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' collected <- collect(df) -#' firstName <- collected[[1]]$name +#' class(collected) +#' firstName <- names(collected)[1] #' } +#' @note collect since 1.4.0 setMethod("collect", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x, stringsAsFactors = FALSE) { - names <- columns(x) - ncol <- length(names) + dtypes <- dtypes(x) + ncol <- length(dtypes) if (ncol <= 0) { # empty data.frame with 0 columns and 0 rows data.frame() @@ -718,69 +1170,85 @@ setMethod("collect", # data of complex type can be held. But getting a cell from a column # of list type returns a list instead of a vector. So for columns of # non-complex type, append them as vector. + # + # For columns of complex type, be careful to access them. + # Get a column of complex type returns a list. + # Get a cell from a column of complex type returns a list instead of a vector. col <- listCols[[colIndex]] if (length(col) <= 0) { - df[[names[colIndex]]] <- col + df[[colIndex]] <- col } else { - # TODO: more robust check on column of primitive types - vec <- do.call(c, col) - if (class(vec) != "list") { - df[[names[colIndex]]] <- vec + colType <- dtypes[[colIndex]][[2]] + if (is.null(PRIMITIVE_TYPES[[colType]])) { + specialtype <- specialtypeshandle(colType) + if (!is.null(specialtype)) { + colType <- specialtype + } + } + + # Note that "binary" columns behave like complex types. + if (!is.null(PRIMITIVE_TYPES[[colType]]) && colType != "binary") { + vec <- do.call(c, col) + stopifnot(class(vec) != "list") + class(vec) <- PRIMITIVE_TYPES[[colType]] + df[[colIndex]] <- vec } else { - # For columns of complex type, be careful to access them. - # Get a column of complex type returns a list. - # Get a cell from a column of complex type returns a list instead of a vector. - df[[names[colIndex]]] <- col - } + df[[colIndex]] <- col + } + } } + names(df) <- names(x) + df } - df - } - }) + }) #' Limit #' -#' Limit the resulting DataFrame to the number of rows specified. +#' Limit the resulting SparkDataFrame to the number of rows specified. #' -#' @param x A SparkSQL DataFrame +#' @param x A SparkDataFrame #' @param num The number of rows to return -#' @return A new DataFrame containing the number of rows specified. +#' @return A new SparkDataFrame containing the number of rows specified. #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname limit #' @name limit +#' @aliases limit,SparkDataFrame,numeric-method #' @export #' @examples #' \dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' limitedDF <- limit(df, 10) #' } +#' @note limit since 1.4.0 setMethod("limit", - signature(x = "DataFrame", num = "numeric"), + signature(x = "SparkDataFrame", num = "numeric"), function(x, num) { res <- callJMethod(x@sdf, "limit", as.integer(num)) dataFrame(res) }) -#' Take the first NUM rows of a DataFrame and return a the results as a data.frame +#' Take the first NUM rows of a SparkDataFrame and return the results as a R data.frame #' -#' @family dataframe_funcs +#' @param x a SparkDataFrame. +#' @param num number of rows to take. +#' @family SparkDataFrame functions #' @rdname take #' @name take +#' @aliases take,SparkDataFrame,numeric-method #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' take(df, 2) #' } +#' @note take since 1.4.0 setMethod("take", - signature(x = "DataFrame", num = "numeric"), + signature(x = "SparkDataFrame", num = "numeric"), function(x, num) { limited <- limit(x, num) collect(limited) @@ -788,74 +1256,73 @@ setMethod("take", #' Head #' -#' Return the first NUM rows of a DataFrame as a data.frame. If NUM is NULL, -#' then head() returns the first 6 rows in keeping with the current data.frame -#' convention in R. +#' Return the first \code{num} rows of a SparkDataFrame as a R data.frame. If \code{num} is not +#' specified, then head() returns the first 6 rows as with R data.frame. #' -#' @param x A SparkSQL DataFrame -#' @param num The number of rows to return. Default is 6. -#' @return A data.frame +#' @param x a SparkDataFrame. +#' @param num the number of rows to return. Default is 6. +#' @return A data.frame. #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions +#' @aliases head,SparkDataFrame-method #' @rdname head #' @name head #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' head(df) #' } +#' @note head since 1.4.0 setMethod("head", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x, num = 6L) { # Default num is 6L in keeping with R's data.frame convention take(x, num) }) -#' Return the first row of a DataFrame +#' Return the first row of a SparkDataFrame #' -#' @param x A SparkSQL DataFrame +#' @param x a SparkDataFrame or a column used in aggregation function. +#' @param ... further arguments to be passed to or from other methods. #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions +#' @aliases first,SparkDataFrame-method #' @rdname first #' @name first #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' first(df) #' } +#' @note first(SparkDataFrame) since 1.4.0 setMethod("first", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { take(x, 1) }) -# toRDD -# -# Converts a Spark DataFrame to an RDD while preserving column names. -# -# @param x A Spark DataFrame -# -# @family dataframe_funcs -# @rdname DataFrame -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# sqlContext <- sparkRSQL.init(sc) -# path <- "path/to/file.json" -# df <- jsonFile(sqlContext, path) -# rdd <- toRDD(df) -# } +#' toRDD +#' +#' Converts a SparkDataFrame to an RDD while preserving column names. +#' +#' @param x A SparkDataFrame +#' +#' @noRd +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' rdd <- toRDD(df) +#'} setMethod("toRDD", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { jrdd <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToRowRDD", x@sdf) colNames <- callJMethod(x@sdf, "columns") @@ -868,13 +1335,13 @@ setMethod("toRDD", #' GroupBy #' -#' Groups the DataFrame using the specified columns, so we can run aggregation on them. +#' Groups the SparkDataFrame using the specified columns, so we can run aggregation on them. #' -#' @param x a DataFrame -#' @return a GroupedData -#' @seealso GroupedData -#' @aliases group_by -#' @family dataframe_funcs +#' @param x a SparkDataFrame. +#' @param ... character name(s) or Column(s) to group on. +#' @return A GroupedData. +#' @family SparkDataFrame functions +#' @aliases groupBy,SparkDataFrame-method #' @rdname groupBy #' @name groupBy #' @export @@ -886,8 +1353,10 @@ setMethod("toRDD", #' # Compute the max age and average salary, grouped by department and gender. #' agg(groupBy(df, "department", "gender"), salary="avg", "age" -> "max") #' } +#' @note groupBy since 1.4.0 +#' @seealso \link{agg}, \link{cube}, \link{rollup} setMethod("groupBy", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x, ...) { cols <- list(...) if (length(cols) >= 1 && class(cols[[1]]) == "character") { @@ -899,11 +1368,12 @@ setMethod("groupBy", groupedData(sgd) }) -#' @family dataframe_funcs #' @rdname groupBy #' @name group_by +#' @aliases group_by,SparkDataFrame-method +#' @note group_by since 1.4.0 setMethod("group_by", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x, ...) { groupBy(x, ...) }) @@ -912,27 +1382,348 @@ setMethod("group_by", #' #' Compute aggregates by specifying a list of columns #' -#' @param x a DataFrame -#' @family dataframe_funcs -#' @rdname agg +#' @family SparkDataFrame functions +#' @aliases agg,SparkDataFrame-method +#' @rdname summarize #' @name agg -#' @aliases summarize #' @export +#' @note agg since 1.4.0 setMethod("agg", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x, ...) { agg(groupBy(x), ...) }) -#' @family dataframe_funcs -#' @rdname agg +#' @rdname summarize #' @name summarize +#' @aliases summarize,SparkDataFrame-method +#' @note summarize since 1.4.0 setMethod("summarize", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x, ...) { agg(x, ...) }) +dapplyInternal <- function(x, func, schema) { + if (is.character(schema)) { + schema <- structType(schema) + } + + packageNamesArr <- serialize(.sparkREnv[[".packages"]], + connection = NULL) + + broadcastArr <- lapply(ls(.broadcastNames), + function(name) { get(name, .broadcastNames) }) + + sdf <- callJStatic( + "org.apache.spark.sql.api.r.SQLUtils", + "dapply", + x@sdf, + serialize(cleanClosure(func), connection = NULL), + packageNamesArr, + broadcastArr, + if (is.null(schema)) { schema } else { schema$jobj }) + dataFrame(sdf) +} + +setClassUnion("characterOrstructType", c("character", "structType")) + +#' dapply +#' +#' Apply a function to each partition of a SparkDataFrame. +#' +#' @param x A SparkDataFrame +#' @param func A function to be applied to each partition of the SparkDataFrame. +#' func should have only one parameter, to which a R data.frame corresponds +#' to each partition will be passed. +#' The output of func should be a R data.frame. +#' @param schema The schema of the resulting SparkDataFrame after the function is applied. +#' It must match the output of func. Since Spark 2.3, the DDL-formatted string +#' is also supported for the schema. +#' @family SparkDataFrame functions +#' @rdname dapply +#' @aliases dapply,SparkDataFrame,function,characterOrstructType-method +#' @name dapply +#' @seealso \link{dapplyCollect} +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame(iris) +#' df1 <- dapply(df, function(x) { x }, schema(df)) +#' collect(df1) +#' +#' # filter and add a column +#' df <- createDataFrame( +#' list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), +#' c("a", "b", "c")) +#' schema <- structType(structField("a", "integer"), structField("b", "double"), +#' structField("c", "string"), structField("d", "integer")) +#' df1 <- dapply( +#' df, +#' function(x) { +#' y <- x[x[1] > 1, ] +#' y <- cbind(y, y[1] + 1L) +#' }, +#' schema) +#' +#' # The schema also can be specified in a DDL-formatted string. +#' schema <- "a INT, d DOUBLE, c STRING, d INT" +#' df1 <- dapply( +#' df, +#' function(x) { +#' y <- x[x[1] > 1, ] +#' y <- cbind(y, y[1] + 1L) +#' }, +#' schema) +#' +#' collect(df1) +#' # the result +#' # a b c d +#' # 1 2 2 2 3 +#' # 2 3 3 3 4 +#' } +#' @note dapply since 2.0.0 +setMethod("dapply", + signature(x = "SparkDataFrame", func = "function", schema = "characterOrstructType"), + function(x, func, schema) { + dapplyInternal(x, func, schema) + }) + +#' dapplyCollect +#' +#' Apply a function to each partition of a SparkDataFrame and collect the result back +#' to R as a data.frame. +#' +#' @param x A SparkDataFrame +#' @param func A function to be applied to each partition of the SparkDataFrame. +#' func should have only one parameter, to which a R data.frame corresponds +#' to each partition will be passed. +#' The output of func should be a R data.frame. +#' @family SparkDataFrame functions +#' @rdname dapplyCollect +#' @aliases dapplyCollect,SparkDataFrame,function-method +#' @name dapplyCollect +#' @seealso \link{dapply} +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame(iris) +#' ldf <- dapplyCollect(df, function(x) { x }) +#' +#' # filter and add a column +#' df <- createDataFrame( +#' list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), +#' c("a", "b", "c")) +#' ldf <- dapplyCollect( +#' df, +#' function(x) { +#' y <- x[x[1] > 1, ] +#' y <- cbind(y, y[1] + 1L) +#' }) +#' # the result +#' # a b c d +#' # 2 2 2 3 +#' # 3 3 3 4 +#' } +#' @note dapplyCollect since 2.0.0 +setMethod("dapplyCollect", + signature(x = "SparkDataFrame", func = "function"), + function(x, func) { + df <- dapplyInternal(x, func, NULL) + + content <- callJMethod(df@sdf, "collect") + # content is a list of items of struct type. Each item has a single field + # which is a serialized data.frame corresponds to one partition of the + # SparkDataFrame. + ldfs <- lapply(content, function(x) { unserialize(x[[1]]) }) + ldf <- do.call(rbind, ldfs) + row.names(ldf) <- NULL + ldf + }) + +#' gapply +#' +#' Groups the SparkDataFrame using the specified columns and applies the R function to each +#' group. +#' +#' @param cols grouping columns. +#' @param func a function to be applied to each group partition specified by grouping +#' column of the SparkDataFrame. The function \code{func} takes as argument +#' a key - grouping columns and a data frame - a local R data.frame. +#' The output of \code{func} is a local R data.frame. +#' @param schema the schema of the resulting SparkDataFrame after the function is applied. +#' The schema must match to output of \code{func}. It has to be defined for each +#' output column with preferred output column name and corresponding data type. +#' Since Spark 2.3, the DDL-formatted string is also supported for the schema. +#' @return A SparkDataFrame. +#' @family SparkDataFrame functions +#' @aliases gapply,SparkDataFrame-method +#' @rdname gapply +#' @name gapply +#' @seealso \link{gapplyCollect} +#' @export +#' @examples +#' +#' \dontrun{ +#' Computes the arithmetic mean of the second column by grouping +#' on the first and third columns. Output the grouping values and the average. +#' +#' df <- createDataFrame ( +#' list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), +#' c("a", "b", "c", "d")) +#' +#' Here our output contains three columns, the key which is a combination of two +#' columns with data types integer and string and the mean which is a double. +#' schema <- structType(structField("a", "integer"), structField("c", "string"), +#' structField("avg", "double")) +#' result <- gapply( +#' df, +#' c("a", "c"), +#' function(key, x) { +#' y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) +#' }, schema) +#' +#' The schema also can be specified in a DDL-formatted string. +#' schema <- "a INT, c STRING, avg DOUBLE" +#' result <- gapply( +#' df, +#' c("a", "c"), +#' function(key, x) { +#' y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) +#' }, schema) +#' +#' We can also group the data and afterwards call gapply on GroupedData. +#' For Example: +#' gdf <- group_by(df, "a", "c") +#' result <- gapply( +#' gdf, +#' function(key, x) { +#' y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) +#' }, schema) +#' collect(result) +#' +#' Result +#' ------ +#' a c avg +#' 3 3 3.0 +#' 1 1 1.5 +#' +#' Fits linear models on iris dataset by grouping on the 'Species' column and +#' using 'Sepal_Length' as a target variable, 'Sepal_Width', 'Petal_Length' +#' and 'Petal_Width' as training features. +#' +#' df <- createDataFrame (iris) +#' schema <- structType(structField("(Intercept)", "double"), +#' structField("Sepal_Width", "double"),structField("Petal_Length", "double"), +#' structField("Petal_Width", "double")) +#' df1 <- gapply( +#' df, +#' df$"Species", +#' function(key, x) { +#' m <- suppressWarnings(lm(Sepal_Length ~ +#' Sepal_Width + Petal_Length + Petal_Width, x)) +#' data.frame(t(coef(m))) +#' }, schema) +#' collect(df1) +#' +#' Result +#' --------- +#' Model (Intercept) Sepal_Width Petal_Length Petal_Width +#' 1 0.699883 0.3303370 0.9455356 -0.1697527 +#' 2 1.895540 0.3868576 0.9083370 -0.6792238 +#' 3 2.351890 0.6548350 0.2375602 0.2521257 +#' +#'} +#' @note gapply(SparkDataFrame) since 2.0.0 +setMethod("gapply", + signature(x = "SparkDataFrame"), + function(x, cols, func, schema) { + grouped <- do.call("groupBy", c(x, cols)) + gapply(grouped, func, schema) + }) + +#' gapplyCollect +#' +#' Groups the SparkDataFrame using the specified columns, applies the R function to each +#' group and collects the result back to R as data.frame. +#' +#' @param cols grouping columns. +#' @param func a function to be applied to each group partition specified by grouping +#' column of the SparkDataFrame. The function \code{func} takes as argument +#' a key - grouping columns and a data frame - a local R data.frame. +#' The output of \code{func} is a local R data.frame. +#' @return A data.frame. +#' @family SparkDataFrame functions +#' @aliases gapplyCollect,SparkDataFrame-method +#' @rdname gapplyCollect +#' @name gapplyCollect +#' @seealso \link{gapply} +#' @export +#' @examples +#' +#' \dontrun{ +#' Computes the arithmetic mean of the second column by grouping +#' on the first and third columns. Output the grouping values and the average. +#' +#' df <- createDataFrame ( +#' list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), +#' c("a", "b", "c", "d")) +#' +#' result <- gapplyCollect( +#' df, +#' c("a", "c"), +#' function(key, x) { +#' y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) +#' colnames(y) <- c("key_a", "key_c", "mean_b") +#' y +#' }) +#' +#' We can also group the data and afterwards call gapply on GroupedData. +#' For Example: +#' gdf <- group_by(df, "a", "c") +#' result <- gapplyCollect( +#' gdf, +#' function(key, x) { +#' y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) +#' colnames(y) <- c("key_a", "key_c", "mean_b") +#' y +#' }) +#' +#' Result +#' ------ +#' key_a key_c mean_b +#' 3 3 3.0 +#' 1 1 1.5 +#' +#' Fits linear models on iris dataset by grouping on the 'Species' column and +#' using 'Sepal_Length' as a target variable, 'Sepal_Width', 'Petal_Length' +#' and 'Petal_Width' as training features. +#' +#' df <- createDataFrame (iris) +#' result <- gapplyCollect( +#' df, +#' df$"Species", +#' function(key, x) { +#' m <- suppressWarnings(lm(Sepal_Length ~ +#' Sepal_Width + Petal_Length + Petal_Width, x)) +#' data.frame(t(coef(m))) +#' }) +#' +#' Result +#'--------- +#' Model X.Intercept. Sepal_Width Petal_Length Petal_Width +#' 1 0.699883 0.3303370 0.9455356 -0.1697527 +#' 2 1.895540 0.3868576 0.9083370 -0.6792238 +#' 3 2.351890 0.6548350 0.2375602 0.2521257 +#' +#'} +#' @note gapplyCollect(SparkDataFrame) since 2.0.0 +setMethod("gapplyCollect", + signature(x = "SparkDataFrame"), + function(x, cols, func) { + grouped <- do.call("groupBy", c(x, cols)) + gapplyCollect(grouped, func) + }) ############################## RDD Map Functions ################################## # All of the following functions mirror the existing RDD map functions, # @@ -940,61 +1731,62 @@ setMethod("summarize", # the requested map function. # ################################################################################### -# @family dataframe_funcs -# @rdname lapply +#' @rdname lapply +#' @noRd setMethod("lapply", - signature(X = "DataFrame", FUN = "function"), + signature(X = "SparkDataFrame", FUN = "function"), function(X, FUN) { rdd <- toRDD(X) lapply(rdd, FUN) }) -# @family dataframe_funcs -# @rdname lapply +#' @rdname lapply +#' @noRd setMethod("map", - signature(X = "DataFrame", FUN = "function"), + signature(X = "SparkDataFrame", FUN = "function"), function(X, FUN) { lapply(X, FUN) }) -# @family dataframe_funcs -# @rdname flatMap +#' @rdname flatMap +#' @noRd setMethod("flatMap", - signature(X = "DataFrame", FUN = "function"), + signature(X = "SparkDataFrame", FUN = "function"), function(X, FUN) { rdd <- toRDD(X) flatMap(rdd, FUN) }) -# @family dataframe_funcs -# @rdname lapplyPartition + +#' @rdname lapplyPartition +#' @noRd setMethod("lapplyPartition", - signature(X = "DataFrame", FUN = "function"), + signature(X = "SparkDataFrame", FUN = "function"), function(X, FUN) { rdd <- toRDD(X) lapplyPartition(rdd, FUN) }) -# @family dataframe_funcs -# @rdname lapplyPartition +#' @rdname lapplyPartition +#' @noRd setMethod("mapPartitions", - signature(X = "DataFrame", FUN = "function"), + signature(X = "SparkDataFrame", FUN = "function"), function(X, FUN) { lapplyPartition(X, FUN) }) -# @family dataframe_funcs -# @rdname foreach +#' @rdname foreach +#' @noRd setMethod("foreach", - signature(x = "DataFrame", func = "function"), + signature(x = "SparkDataFrame", func = "function"), function(x, func) { rdd <- toRDD(x) foreach(rdd, func) }) -# @family dataframe_funcs -# @rdname foreach +#' @rdname foreach +#' @noRd setMethod("foreachPartition", - signature(x = "DataFrame", func = "function"), + signature(x = "SparkDataFrame", func = "function"), function(x, func) { rdd <- toRDD(x) foreachPartition(rdd, func) @@ -1007,37 +1799,42 @@ getColumn <- function(x, c) { column(callJMethod(x@sdf, "col", c)) } +setColumn <- function(x, c, value) { + if (class(value) != "Column" && !is.null(value)) { + if (isAtomicLengthOne(value)) { + value <- lit(value) + } else { + stop("value must be a Column, literal value as atomic in length of 1, or NULL") + } + } + + if (is.null(value)) { + nx <- drop(x, c) + } else { + nx <- withColumn(x, c, value) + } + nx +} + +#' @param name name of a Column (without being wrapped by \code{""}). #' @rdname select #' @name $ -setMethod("$", signature(x = "DataFrame"), +#' @aliases $,SparkDataFrame-method +#' @note $ since 1.4.0 +setMethod("$", signature(x = "SparkDataFrame"), function(x, name) { getColumn(x, name) }) +#' @param value a Column or an atomic vector in the length of 1 as literal value, or \code{NULL}. +#' If \code{NULL}, the specified Column is dropped. #' @rdname select #' @name $<- -setMethod("$<-", signature(x = "DataFrame"), +#' @aliases $<-,SparkDataFrame-method +#' @note $<- since 1.4.0 +setMethod("$<-", signature(x = "SparkDataFrame"), function(x, name, value) { - stopifnot(class(value) == "Column" || is.null(value)) - cols <- columns(x) - if (name %in% cols) { - if (is.null(value)) { - cols <- Filter(function(c) { c != name }, cols) - } - cols <- lapply(cols, function(c) { - if (c == name) { - alias(value, name) - } else { - col(c) - } - }) - nx <- select(x, cols) - } else { - if (is.null(value)) { - return(x) - } - nx <- withColumn(x, name, value) - } + nx <- setColumn(x, name, value) x@sdf <- nx@sdf x }) @@ -1046,8 +1843,14 @@ setClassUnion("numericOrcharacter", c("numeric", "character")) #' @rdname subset #' @name [[ -setMethod("[[", signature(x = "DataFrame", i = "numericOrcharacter"), +#' @aliases [[,SparkDataFrame,numericOrcharacter-method +#' @note [[ since 1.4.0 +setMethod("[[", signature(x = "SparkDataFrame", i = "numericOrcharacter"), function(x, i) { + if (length(i) > 1) { + warning("Subset index has length > 1. Only the first index is used.") + i <- i[1] + } if (is.numeric(i)) { cols <- columns(x) i <- cols[[i]] @@ -1056,74 +1859,126 @@ setMethod("[[", signature(x = "DataFrame", i = "numericOrcharacter"), }) #' @rdname subset -#' @name [ -setMethod("[", signature(x = "DataFrame", i = "missing"), - function(x, i, j, ...) { - if (is.numeric(j)) { - cols <- columns(x) - j <- cols[j] +#' @name [[<- +#' @aliases [[<-,SparkDataFrame,numericOrcharacter-method +#' @note [[<- since 2.1.1 +setMethod("[[<-", signature(x = "SparkDataFrame", i = "numericOrcharacter"), + function(x, i, value) { + if (length(i) > 1) { + warning("Subset index has length > 1. Only the first index is used.") + i <- i[1] } - if (length(j) > 1) { - j <- as.list(j) + if (is.numeric(i)) { + cols <- columns(x) + i <- cols[[i]] } - select(x, j) + nx <- setColumn(x, i, value) + x@sdf <- nx@sdf + x }) #' @rdname subset #' @name [ -setMethod("[", signature(x = "DataFrame", i = "Column"), - function(x, i, j, ...) { - # It could handle i as "character" but it seems confusing and not required - # https://stat.ethz.ch/R-manual/R-devel/library/base/html/Extract.data.frame.html - filtered <- filter(x, i) - if (!missing(j)) { - filtered[, j, ...] +#' @aliases [,SparkDataFrame-method +#' @note [ since 1.4.0 +setMethod("[", signature(x = "SparkDataFrame"), + function(x, i, j, ..., drop = F) { + # Perform filtering first if needed + filtered <- if (missing(i)) { + x } else { + if (class(i) != "Column") { + stop(paste0("Expressions other than filtering predicates are not supported ", + "in the first parameter of extract operator [ or subset() method.")) + } + filter(x, i) + } + + # If something is to be projected, then do so on the filtered SparkDataFrame + if (missing(j)) { filtered + } else { + if (is.numeric(j)) { + cols <- columns(filtered) + j <- cols[j] + } + if (length(j) > 1) { + j <- as.list(j) + } + selected <- select(filtered, j) + + # Acknowledge parameter drop. Return a Column or SparkDataFrame accordingly + if (ncol(selected) == 1 & drop == T) { + getColumn(selected, names(selected)) + } else { + selected + } } }) #' Subset #' -#' Return subsets of DataFrame according to given conditions -#' @param x A DataFrame -#' @param subset A logical expression to filter on rows -#' @param select expression for the single Column or a list of columns to select from the DataFrame -#' @return A new DataFrame containing only the rows that meet the condition with selected columns +#' Return subsets of SparkDataFrame according to given conditions +#' @param x a SparkDataFrame. +#' @param i,subset (Optional) a logical expression to filter on rows. +#' For extract operator [[ and replacement operator [[<-, the indexing parameter for +#' a single Column. +#' @param j,select expression for the single Column or a list of columns to select from the SparkDataFrame. +#' @param drop if TRUE, a Column will be returned if the resulting dataset has only one column. +#' Otherwise, a SparkDataFrame will always be returned. +#' @param value a Column or an atomic vector in the length of 1 as literal value, or \code{NULL}. +#' If \code{NULL}, the specified Column is dropped. +#' @param ... currently not used. +#' @return A new SparkDataFrame containing only the rows that meet the condition with selected columns. #' @export -#' @family dataframe_funcs +#' @family SparkDataFrame functions +#' @aliases subset,SparkDataFrame-method +#' @seealso \link{withColumn} #' @rdname subset #' @name subset -#' @aliases [ #' @family subsetting functions #' @examples #' \dontrun{ -#' # Columns can be selected using `[[` and `[` +#' # Columns can be selected using [[ and [ #' df[[2]] == df[["age"]] #' df[,2] == df[,"age"] #' df[,c("name", "age")] #' # Or to filter rows #' df[df$age > 20,] -#' # DataFrame can be subset on both rows and Columns +#' # SparkDataFrame can be subset on both rows and Columns #' df[df$name == "Smith", c(1,2)] #' df[df$age %in% c(19, 30), 1:2] #' subset(df, df$age %in% c(19, 30), 1:2) #' subset(df, df$age %in% c(19), select = c(1,2)) +#' subset(df, select = c(1,2)) +#' # Columns can be selected and set +#' df[["age"]] <- 23 +#' df[[1]] <- df$age +#' df[[2]] <- NULL # drop column #' } -setMethod("subset", signature(x = "DataFrame"), - function(x, subset, select, ...) { - x[subset, select, ...] +#' @note subset since 1.5.0 +setMethod("subset", signature(x = "SparkDataFrame"), + function(x, subset, select, drop = F, ...) { + if (missing(subset)) { + x[, select, drop = drop, ...] + } else { + x[subset, select, drop = drop, ...] + } }) #' Select #' #' Selects a set of columns with names or Column expressions. -#' @param x A DataFrame -#' @param col A list of columns or single Column or name -#' @return A new DataFrame with selected columns +#' @param x a SparkDataFrame. +#' @param col a list of columns or single Column or name. +#' @param ... additional column(s) if only one column is specified in \code{col}. +#' If more than one column is assigned in \code{col}, \code{...} +#' should be left empty. +#' @return A new SparkDataFrame with selected columns. #' @export -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname select +#' @aliases select,SparkDataFrame,character-method #' @name select #' @family subsetting functions #' @examples @@ -1133,10 +1988,11 @@ setMethod("subset", signature(x = "DataFrame"), #' select(df, df$name, df$age + 1) #' select(df, c("col1", "col2")) #' select(df, list(df$name, df$age + 1)) -#' # Similar to R data frames columns can also be selected using `$` +#' # Similar to R data frames columns can also be selected using $ #' df[,df$age] #' } -setMethod("select", signature(x = "DataFrame", col = "character"), +#' @note select(SparkDataFrame, character) since 1.4.0 +setMethod("select", signature(x = "SparkDataFrame", col = "character"), function(x, col, ...) { if (length(col) > 1) { if (length(list(...)) > 0) { @@ -1150,10 +2006,11 @@ setMethod("select", signature(x = "DataFrame", col = "character"), } }) -#' @family dataframe_funcs #' @rdname select #' @export -setMethod("select", signature(x = "DataFrame", col = "Column"), +#' @aliases select,SparkDataFrame,Column-method +#' @note select(SparkDataFrame, Column) since 1.4.0 +setMethod("select", signature(x = "SparkDataFrame", col = "Column"), function(x, col, ...) { jcols <- lapply(list(col, ...), function(c) { c@jc @@ -1162,11 +2019,12 @@ setMethod("select", signature(x = "DataFrame", col = "Column"), dataFrame(sdf) }) -#' @family dataframe_funcs #' @rdname select #' @export +#' @aliases select,SparkDataFrame,list-method +#' @note select(SparkDataFrame, list) since 1.4.0 setMethod("select", - signature(x = "DataFrame", col = "list"), + signature(x = "SparkDataFrame", col = "list"), function(x, col) { cols <- lapply(col, function(c) { if (class(c) == "Column") { @@ -1181,26 +2039,27 @@ setMethod("select", #' SelectExpr #' -#' Select from a DataFrame using a set of SQL expressions. +#' Select from a SparkDataFrame using a set of SQL expressions. #' -#' @param x A DataFrame to be selected from. +#' @param x A SparkDataFrame to be selected from. #' @param expr A string containing a SQL expression #' @param ... Additional expressions -#' @return A DataFrame -#' @family dataframe_funcs +#' @return A SparkDataFrame +#' @family SparkDataFrame functions +#' @aliases selectExpr,SparkDataFrame,character-method #' @rdname selectExpr #' @name selectExpr #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' selectExpr(df, "col1", "(col2 * 5) as newCol") #' } +#' @note selectExpr since 1.4.0 setMethod("selectExpr", - signature(x = "DataFrame", expr = "character"), + signature(x = "SparkDataFrame", expr = "character"), function(x, expr, ...) { exprList <- list(expr, ...) sdf <- callJMethod(x@sdf, "selectExpr", exprList) @@ -1209,104 +2068,163 @@ setMethod("selectExpr", #' WithColumn #' -#' Return a new DataFrame with the specified column added. +#' Return a new SparkDataFrame by adding a column or replacing the existing column +#' that has the same name. #' -#' @param x A DataFrame -#' @param colName A string containing the name of the new column. -#' @param col A Column expression. -#' @return A DataFrame with the new column added. -#' @family dataframe_funcs +#' @param x a SparkDataFrame. +#' @param colName a column name. +#' @param col a Column expression, or an atomic vector in the length of 1 as literal value. +#' @return A SparkDataFrame with the new column added or the existing column replaced. +#' @family SparkDataFrame functions +#' @aliases withColumn,SparkDataFrame,character-method #' @rdname withColumn #' @name withColumn -#' @aliases mutate transform +#' @seealso \link{rename} \link{mutate} \link{subset} #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' newDF <- withColumn(df, "newCol", df$col1 * 5) +#' # Replace an existing column +#' newDF2 <- withColumn(newDF, "newCol", newDF$col1) +#' newDF3 <- withColumn(newDF, "newCol", 42) +#' # Use extract operator to set an existing or new column +#' df[["age"]] <- 23 +#' df[[2]] <- df$col1 +#' df[[2]] <- NULL # drop column #' } +#' @note withColumn since 1.4.0 setMethod("withColumn", - signature(x = "DataFrame", colName = "character", col = "Column"), + signature(x = "SparkDataFrame", colName = "character"), function(x, colName, col) { - select(x, x$"*", alias(col, colName)) + if (class(col) != "Column") { + if (!isAtomicLengthOne(col)) stop("Literal value must be atomic in length of 1") + col <- lit(col) + } + sdf <- callJMethod(x@sdf, "withColumn", colName, col@jc) + dataFrame(sdf) }) #' Mutate #' -#' Return a new DataFrame with the specified columns added. +#' Return a new SparkDataFrame with the specified columns added or replaced. #' -#' @param .data A DataFrame -#' @param col a named argument of the form name = col -#' @return A new DataFrame with the new columns added. -#' @family dataframe_funcs -#' @rdname withColumn +#' @param .data a SparkDataFrame. +#' @param ... additional column argument(s) each in the form name = col. +#' @return A new SparkDataFrame with the new columns added or replaced. +#' @family SparkDataFrame functions +#' @aliases mutate,SparkDataFrame-method +#' @rdname mutate #' @name mutate -#' @aliases withColumn transform +#' @seealso \link{rename} \link{withColumn} #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' newDF <- mutate(df, newCol = df$col1 * 5, newCol2 = df$col1 * 2) #' names(newDF) # Will contain newCol, newCol2 #' newDF2 <- transform(df, newCol = df$col1 / 5, newCol2 = df$col1 * 2) +#' +#' df <- createDataFrame(list(list("Andy", 30L), list("Justin", 19L)), c("name", "age")) +#' # Replace the "age" column +#' df1 <- mutate(df, age = df$age + 1L) #' } +#' @note mutate since 1.4.0 setMethod("mutate", - signature(.data = "DataFrame"), + signature(.data = "SparkDataFrame"), function(.data, ...) { x <- .data cols <- list(...) - stopifnot(length(cols) > 0) - stopifnot(class(cols[[1]]) == "Column") + if (length(cols) <= 0) { + return(x) + } + + lapply(cols, function(col) { + stopifnot(class(col) == "Column") + }) + + # Check if there is any duplicated column name in the DataFrame + dfCols <- columns(x) + if (length(unique(dfCols)) != length(dfCols)) { + stop("Error: found duplicated column name in the DataFrame") + } + + # TODO: simplify the implementation of this method after SPARK-12225 is resolved. + + # For named arguments, use the names for arguments as the column names + # For unnamed arguments, use the argument symbols as the column names + args <- sapply(substitute(list(...))[-1], deparse) ns <- names(cols) if (!is.null(ns)) { - for (n in ns) { - if (n != "") { - cols[[n]] <- alias(cols[[n]], n) + lapply(seq_along(args), function(i) { + if (ns[[i]] != "") { + args[[i]] <<- ns[[i]] } - } + }) } - do.call(select, c(x, x$"*", cols)) - }) + ns <- args -#' @export -#' @family dataframe_funcs -#' @rdname withColumn + # The last column of the same name in the specific columns takes effect + deDupCols <- list() + for (i in 1:length(cols)) { + deDupCols[[ns[[i]]]] <- alias(cols[[i]], ns[[i]]) + } + + # Construct the column list for projection + colList <- lapply(dfCols, function(col) { + if (!is.null(deDupCols[[col]])) { + # Replace existing column + tmpCol <- deDupCols[[col]] + deDupCols[[col]] <<- NULL + tmpCol + } else { + col(col) + } + }) + + do.call(select, c(x, colList, deDupCols)) + }) + +#' @param _data a SparkDataFrame. +#' @export +#' @rdname mutate +#' @aliases transform,SparkDataFrame-method #' @name transform -#' @aliases withColumn mutate +#' @note transform since 1.5.0 setMethod("transform", - signature(`_data` = "DataFrame"), + signature(`_data` = "SparkDataFrame"), function(`_data`, ...) { mutate(`_data`, ...) }) -#' WithColumnRenamed +#' rename #' -#' Rename an existing column in a DataFrame. +#' Rename an existing column in a SparkDataFrame. #' -#' @param x A DataFrame +#' @param x A SparkDataFrame #' @param existingCol The name of the column you want to change. #' @param newCol The new column name. -#' @return A DataFrame with the column name changed. -#' @family dataframe_funcs -#' @rdname withColumnRenamed +#' @return A SparkDataFrame with the column name changed. +#' @family SparkDataFrame functions +#' @rdname rename #' @name withColumnRenamed +#' @aliases withColumnRenamed,SparkDataFrame,character,character-method +#' @seealso \link{mutate} #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' newDF <- withColumnRenamed(df, "col1", "newCol1") #' } +#' @note withColumnRenamed since 1.4.0 setMethod("withColumnRenamed", - signature(x = "DataFrame", existingCol = "character", newCol = "character"), + signature(x = "SparkDataFrame", existingCol = "character", newCol = "character"), function(x, existingCol, newCol) { cols <- lapply(columns(x), function(c) { if (c == existingCol) { @@ -1318,28 +2236,21 @@ setMethod("withColumnRenamed", select(x, cols) }) -#' Rename -#' -#' Rename an existing column in a DataFrame. -#' -#' @param x A DataFrame -#' @param newCol A named pair of the form new_column_name = existing_column -#' @return A DataFrame with the column name changed. -#' @family dataframe_funcs -#' @rdname withColumnRenamed +#' @param ... A named pair of the form new_column_name = existing_column +#' @rdname rename #' @name rename -#' @aliases withColumnRenamed +#' @aliases rename,SparkDataFrame-method #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' newDF <- rename(df, col1 = df$newCol1) #' } +#' @note rename since 1.4.0 setMethod("rename", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x, ...) { renameCols <- list(...) stopifnot(length(renameCols) > 0) @@ -1360,34 +2271,34 @@ setMethod("rename", setClassUnion("characterOrColumn", c("character", "Column")) -#' Arrange +#' Arrange Rows by Variables #' -#' Sort a DataFrame by the specified column(s). +#' Sort a SparkDataFrame by the specified column(s). #' -#' @param x A DataFrame to be sorted. -#' @param col A character or Column object vector indicating the fields to sort on -#' @param ... Additional sorting fields -#' @param decreasing A logical argument indicating sorting order for columns when +#' @param x a SparkDataFrame to be sorted. +#' @param col a character or Column object indicating the fields to sort on +#' @param ... additional sorting fields +#' @param decreasing a logical argument indicating sorting order for columns when #' a character vector is specified for col -#' @return A DataFrame where all elements are sorted. -#' @family dataframe_funcs +#' @return A SparkDataFrame where all elements are sorted. +#' @family SparkDataFrame functions +#' @aliases arrange,SparkDataFrame,Column-method #' @rdname arrange #' @name arrange -#' @aliases orderby #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' arrange(df, df$col1) #' arrange(df, asc(df$col1), desc(abs(df$col2))) #' arrange(df, "col1", decreasing = TRUE) #' arrange(df, "col1", "col2", decreasing = c(TRUE, FALSE)) #' } +#' @note arrange(SparkDataFrame, Column) since 1.4.0 setMethod("arrange", - signature(x = "DataFrame", col = "Column"), + signature(x = "SparkDataFrame", col = "Column"), function(x, col, ...) { jcols <- lapply(list(col, ...), function(c) { c@jc @@ -1397,11 +2308,13 @@ setMethod("arrange", dataFrame(sdf) }) -#' @family dataframe_funcs #' @rdname arrange +#' @name arrange +#' @aliases arrange,SparkDataFrame,character-method #' @export +#' @note arrange(SparkDataFrame, character) since 1.4.0 setMethod("arrange", - signature(x = "DataFrame", col = "character"), + signature(x = "SparkDataFrame", col = "character"), function(x, col, ..., decreasing = FALSE) { # all sorting columns @@ -1429,39 +2342,41 @@ setMethod("arrange", do.call("arrange", c(x, jcols)) }) -#' @family dataframe_funcs #' @rdname arrange -#' @name orderby +#' @aliases orderBy,SparkDataFrame,characterOrColumn-method +#' @export +#' @note orderBy(SparkDataFrame, characterOrColumn) since 1.4.0 setMethod("orderBy", - signature(x = "DataFrame", col = "characterOrColumn"), - function(x, col) { - arrange(x, col) + signature(x = "SparkDataFrame", col = "characterOrColumn"), + function(x, col, ...) { + arrange(x, col, ...) }) #' Filter #' -#' Filter the rows of a DataFrame according to a given condition. +#' Filter the rows of a SparkDataFrame according to a given condition. #' -#' @param x A DataFrame to be sorted. +#' @param x A SparkDataFrame to be sorted. #' @param condition The condition to filter on. This may either be a Column expression #' or a string containing a SQL statement -#' @return A DataFrame containing only the rows that meet the condition. -#' @family dataframe_funcs +#' @return A SparkDataFrame containing only the rows that meet the condition. +#' @family SparkDataFrame functions +#' @aliases filter,SparkDataFrame,characterOrColumn-method #' @rdname filter #' @name filter #' @family subsetting functions #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' filter(df, "col1 > 0") #' filter(df, df$col2 != "abcdefg") #' } +#' @note filter since 1.4.0 setMethod("filter", - signature(x = "DataFrame", condition = "characterOrColumn"), + signature(x = "SparkDataFrame", condition = "characterOrColumn"), function(x, condition) { if (class(condition) == "Column") { condition <- condition@jc @@ -1470,108 +2385,199 @@ setMethod("filter", dataFrame(sdf) }) -#' @family dataframe_funcs #' @rdname filter #' @name where +#' @aliases where,SparkDataFrame,characterOrColumn-method +#' @note where since 1.4.0 setMethod("where", - signature(x = "DataFrame", condition = "characterOrColumn"), + signature(x = "SparkDataFrame", condition = "characterOrColumn"), function(x, condition) { filter(x, condition) }) +#' dropDuplicates +#' +#' Returns a new SparkDataFrame with duplicate rows removed, considering only +#' the subset of columns. +#' +#' @param x A SparkDataFrame. +#' @param ... A character vector of column names or string column names. +#' If the first argument contains a character vector, the followings are ignored. +#' @return A SparkDataFrame with duplicate rows removed. +#' @family SparkDataFrame functions +#' @aliases dropDuplicates,SparkDataFrame-method +#' @rdname dropDuplicates +#' @name dropDuplicates +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' dropDuplicates(df) +#' dropDuplicates(df, "col1", "col2") +#' dropDuplicates(df, c("col1", "col2")) +#' } +#' @note dropDuplicates since 2.0.0 +setMethod("dropDuplicates", + signature(x = "SparkDataFrame"), + function(x, ...) { + cols <- list(...) + if (length(cols) == 0) { + sdf <- callJMethod(x@sdf, "dropDuplicates", as.list(columns(x))) + } else { + if (!all(sapply(cols, function(c) { is.character(c) }))) { + stop("all columns names should be characters") + } + col <- cols[[1]] + if (length(col) > 1) { + sdf <- callJMethod(x@sdf, "dropDuplicates", as.list(col)) + } else { + sdf <- callJMethod(x@sdf, "dropDuplicates", cols) + } + } + dataFrame(sdf) + }) + #' Join #' -#' Join two DataFrames based on the given join expression. +#' Joins two SparkDataFrames based on the given join expression. #' -#' @param x A Spark DataFrame -#' @param y A Spark DataFrame +#' @param x A SparkDataFrame +#' @param y A SparkDataFrame #' @param joinExpr (Optional) The expression used to perform the join. joinExpr must be a -#' Column expression. If joinExpr is omitted, join() will perform a Cartesian join -#' @param joinType The type of join to perform. The following join types are available: -#' 'inner', 'outer', 'full', 'fullouter', leftouter', 'left_outer', 'left', -#' 'right_outer', 'rightouter', 'right', and 'leftsemi'. The default joinType is "inner". -#' @return A DataFrame containing the result of the join operation. -#' @family dataframe_funcs +#' Column expression. If joinExpr is omitted, the default, inner join is attempted and an error is +#' thrown if it would be a Cartesian Product. For Cartesian join, use crossJoin instead. +#' @param joinType The type of join to perform, default 'inner'. +#' Must be one of: 'inner', 'cross', 'outer', 'full', 'full_outer', +#' 'left', 'left_outer', 'right', 'right_outer', 'left_semi', or 'left_anti'. +#' @return A SparkDataFrame containing the result of the join operation. +#' @family SparkDataFrame functions +#' @aliases join,SparkDataFrame,SparkDataFrame-method #' @rdname join #' @name join +#' @seealso \link{merge} \link{crossJoin} #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' df1 <- jsonFile(sqlContext, path) -#' df2 <- jsonFile(sqlContext, path2) -#' join(df1, df2) # Performs a Cartesian +#' sparkR.session() +#' df1 <- read.json(path) +#' df2 <- read.json(path2) #' join(df1, df2, df1$col1 == df2$col2) # Performs an inner join based on expression #' join(df1, df2, df1$col1 == df2$col2, "right_outer") +#' join(df1, df2) # Attempts an inner join #' } +#' @note join since 1.4.0 setMethod("join", - signature(x = "DataFrame", y = "DataFrame"), + signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y, joinExpr = NULL, joinType = NULL) { if (is.null(joinExpr)) { + # this may not fail until the planner checks for Cartesian join later on. sdf <- callJMethod(x@sdf, "join", y@sdf) } else { if (class(joinExpr) != "Column") stop("joinExpr must be a Column") if (is.null(joinType)) { sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc) } else { - if (joinType %in% c("inner", "outer", "full", "fullouter", - "leftouter", "left_outer", "left", - "rightouter", "right_outer", "right", "leftsemi")) { + if (joinType %in% c("inner", "cross", + "outer", "full", "fullouter", "full_outer", + "left", "leftouter", "left_outer", + "right", "rightouter", "right_outer", + "left_semi", "leftsemi", "left_anti", "leftanti")) { joinType <- gsub("_", "", joinType) sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc, joinType) } else { stop("joinType must be one of the following types: ", - "'inner', 'outer', 'full', 'fullouter', 'leftouter', 'left_outer', 'left', - 'rightouter', 'right_outer', 'right', 'leftsemi'") + "'inner', 'cross', 'outer', 'full', 'full_outer',", + "'left', 'left_outer', 'right', 'right_outer',", + "'left_semi', or 'left_anti'.") } } } dataFrame(sdf) }) +#' CrossJoin +#' +#' Returns Cartesian Product on two SparkDataFrames. +#' +#' @param x A SparkDataFrame +#' @param y A SparkDataFrame +#' @return A SparkDataFrame containing the result of the join operation. +#' @family SparkDataFrame functions +#' @aliases crossJoin,SparkDataFrame,SparkDataFrame-method +#' @rdname crossJoin +#' @name crossJoin +#' @seealso \link{merge} \link{join} +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df1 <- read.json(path) +#' df2 <- read.json(path2) +#' crossJoin(df1, df2) # Performs a Cartesian +#' } +#' @note crossJoin since 2.1.0 +setMethod("crossJoin", + signature(x = "SparkDataFrame", y = "SparkDataFrame"), + function(x, y) { + sdf <- callJMethod(x@sdf, "crossJoin", y@sdf) + dataFrame(sdf) + }) + +#' Merges two data frames #' #' @name merge -#' @aliases join -#' @title Merges two data frames -#' @param x the first data frame to be joined -#' @param y the second data frame to be joined +#' @param x the first data frame to be joined. +#' @param y the second data frame to be joined. #' @param by a character vector specifying the join columns. If by is not #' specified, the common column names in \code{x} and \code{y} will be used. +#' If by or both by.x and by.y are explicitly set to NULL or of length 0, the Cartesian +#' Product of x and y will be returned. #' @param by.x a character vector specifying the joining columns for x. #' @param by.y a character vector specifying the joining columns for y. +#' @param all a boolean value setting \code{all.x} and \code{all.y} +#' if any of them are unset. #' @param all.x a boolean value indicating whether all the rows in x should -#' be including in the join +#' be including in the join. #' @param all.y a boolean value indicating whether all the rows in y should -#' be including in the join -#' @param sort a logical argument indicating whether the resulting columns should be sorted +#' be including in the join. +#' @param sort a logical argument indicating whether the resulting columns should be sorted. +#' @param suffixes a string vector of length 2 used to make colnames of +#' \code{x} and \code{y} unique. +#' The first element is appended to each colname of \code{x}. +#' The second element is appended to each colname of \code{y}. +#' @param ... additional argument(s) passed to the method. #' @details If all.x and all.y are set to FALSE, a natural join will be returned. If #' all.x is set to TRUE and all.y is set to FALSE, a left outer join will #' be returned. If all.x is set to FALSE and all.y is set to TRUE, a right #' outer join will be returned. If all.x and all.y are set to TRUE, a full #' outer join will be returned. -#' @family dataframe_funcs +#' @family SparkDataFrame functions +#' @aliases merge,SparkDataFrame,SparkDataFrame-method #' @rdname merge +#' @seealso \link{join} \link{crossJoin} #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' df1 <- jsonFile(sqlContext, path) -#' df2 <- jsonFile(sqlContext, path2) -#' merge(df1, df2) # Performs a Cartesian +#' sparkR.session() +#' df1 <- read.json(path) +#' df2 <- read.json(path2) +#' merge(df1, df2) # Performs an inner join by common columns #' merge(df1, df2, by = "col1") # Performs an inner join based on expression #' merge(df1, df2, by.x = "col1", by.y = "col2", all.y = TRUE) #' merge(df1, df2, by.x = "col1", by.y = "col2", all.x = TRUE) #' merge(df1, df2, by.x = "col1", by.y = "col2", all.x = TRUE, all.y = TRUE) #' merge(df1, df2, by.x = "col1", by.y = "col2", all = TRUE, sort = FALSE) #' merge(df1, df2, by = "col1", all = TRUE, suffixes = c("-X", "-Y")) +#' merge(df1, df2, by = NULL) # Performs a Cartesian join #' } +#' @note merge since 1.5.0 setMethod("merge", - signature(x = "DataFrame", y = "DataFrame"), + signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y, by = intersect(names(x), names(y)), by.x = by, by.y = by, all = FALSE, all.x = all, all.y = all, - sort = TRUE, suffixes = c("_x","_y"), ... ) { + sort = TRUE, suffixes = c("_x", "_y"), ...) { if (length(suffixes) != 2) { stop("suffixes must have length 2") @@ -1601,7 +2607,7 @@ setMethod("merge", joinY <- by } else { # if by or both by.x and by.y have length 0, use Cartesian Product - joinRes <- join(x, y) + joinRes <- crossJoin(x, y) return (joinRes) } @@ -1646,15 +2652,17 @@ setMethod("merge", joinRes }) +#' Creates a list of columns by replacing the intersected ones with aliases #' #' Creates a list of columns by replacing the intersected ones with aliases. #' The name of the alias column is formed by concatanating the original column name and a suffix. #' -#' @param x a DataFrame on which the -#' @param intersectedColNames a list of intersected column names +#' @param x a SparkDataFrame +#' @param intersectedColNames a list of intersected column names of the SparkDataFrame #' @param suffix a suffix for the column name #' @return list of columns #' +#' @note generateAliasesForIntersectedCols since 1.6.0 generateAliasesForIntersectedCols <- function (x, intersectedColNames, suffix) { allColNames <- names(x) # sets alias for making colnames unique in dataframe 'x' @@ -1673,74 +2681,148 @@ generateAliasesForIntersectedCols <- function (x, intersectedColNames, suffix) { cols } -#' UnionAll +#' Return a new SparkDataFrame containing the union of rows #' -#' Return a new DataFrame containing the union of rows in this DataFrame -#' and another DataFrame. This is equivalent to `UNION ALL` in SQL. -#' Note that this does not remove duplicate rows across the two DataFrames. +#' Return a new SparkDataFrame containing the union of rows in this SparkDataFrame +#' and another SparkDataFrame. This is equivalent to \code{UNION ALL} in SQL. +#' Input SparkDataFrames can have different schemas (names and data types). #' -#' @param x A Spark DataFrame -#' @param y A Spark DataFrame -#' @return A DataFrame containing the result of the union. -#' @family dataframe_funcs -#' @rdname unionAll -#' @name unionAll +#' Note: This does not remove duplicate rows across the two SparkDataFrames. +#' Also as standard in SQL, this function resolves columns by position (not by name). +#' +#' @param x A SparkDataFrame +#' @param y A SparkDataFrame +#' @return A SparkDataFrame containing the result of the union. +#' @family SparkDataFrame functions +#' @rdname union +#' @name union +#' @aliases union,SparkDataFrame,SparkDataFrame-method +#' @seealso \link{rbind} \link{unionByName} #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' df1 <- jsonFile(sqlContext, path) -#' df2 <- jsonFile(sqlContext, path2) -#' unioned <- unionAll(df, df2) +#' sparkR.session() +#' df1 <- read.json(path) +#' df2 <- read.json(path2) +#' unioned <- union(df, df2) +#' unions <- rbind(df, df2, df3, df4) #' } +#' @note union since 2.0.0 +setMethod("union", + signature(x = "SparkDataFrame", y = "SparkDataFrame"), + function(x, y) { + unioned <- callJMethod(x@sdf, "union", y@sdf) + dataFrame(unioned) + }) + +#' unionAll is deprecated - use union instead +#' @rdname union +#' @name unionAll +#' @aliases unionAll,SparkDataFrame,SparkDataFrame-method +#' @export +#' @note unionAll since 1.4.0 setMethod("unionAll", - signature(x = "DataFrame", y = "DataFrame"), + signature(x = "SparkDataFrame", y = "SparkDataFrame"), + function(x, y) { + .Deprecated("union") + union(x, y) + }) + +#' Return a new SparkDataFrame containing the union of rows, matched by column names +#' +#' Return a new SparkDataFrame containing the union of rows in this SparkDataFrame +#' and another SparkDataFrame. This is different from \code{union} function, and both +#' \code{UNION ALL} and \code{UNION DISTINCT} in SQL as column positions are not taken +#' into account. Input SparkDataFrames can have different data types in the schema. +#' +#' Note: This does not remove duplicate rows across the two SparkDataFrames. +#' This function resolves columns by name (not by position). +#' +#' @param x A SparkDataFrame +#' @param y A SparkDataFrame +#' @return A SparkDataFrame containing the result of the union. +#' @family SparkDataFrame functions +#' @rdname unionByName +#' @name unionByName +#' @aliases unionByName,SparkDataFrame,SparkDataFrame-method +#' @seealso \link{rbind} \link{union} +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df1 <- select(createDataFrame(mtcars), "carb", "am", "gear") +#' df2 <- select(createDataFrame(mtcars), "am", "gear", "carb") +#' head(unionByName(df1, df2)) +#' } +#' @note unionByName since 2.3.0 +setMethod("unionByName", + signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y) { - unioned <- callJMethod(x@sdf, "unionAll", y@sdf) + unioned <- callJMethod(x@sdf, "unionByName", y@sdf) dataFrame(unioned) }) -#' @title Union two or more DataFrames +#' Union two or more SparkDataFrames +#' +#' Union two or more SparkDataFrames by row. As in R's \code{rbind}, this method +#' requires that the input SparkDataFrames have the same column names. #' -#' @description Returns a new DataFrame containing rows of all parameters. +#' Note: This does not remove duplicate rows across the two SparkDataFrames. #' -#' @family dataframe_funcs +#' @param x a SparkDataFrame. +#' @param ... additional SparkDataFrame(s). +#' @param deparse.level currently not used (put here to match the signature of +#' the base implementation). +#' @return A SparkDataFrame containing the result of the union. +#' @family SparkDataFrame functions +#' @aliases rbind,SparkDataFrame-method #' @rdname rbind #' @name rbind -#' @aliases unionAll +#' @seealso \link{union} \link{unionByName} +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' unions <- rbind(df, df2, df3, df4) +#' } +#' @note rbind since 1.5.0 setMethod("rbind", - signature(... = "DataFrame"), + signature(... = "SparkDataFrame"), function(x, ..., deparse.level = 1) { + nm <- lapply(list(x, ...), names) + if (length(unique(nm)) != 1) { + stop("Names of input data frames are different.") + } if (nargs() == 3) { - unionAll(x, ...) + union(x, ...) } else { - unionAll(x, Recall(..., deparse.level = 1)) + union(x, Recall(..., deparse.level = 1)) } }) #' Intersect #' -#' Return a new DataFrame containing rows only in both this DataFrame -#' and another DataFrame. This is equivalent to `INTERSECT` in SQL. +#' Return a new SparkDataFrame containing rows only in both this SparkDataFrame +#' and another SparkDataFrame. This is equivalent to \code{INTERSECT} in SQL. #' -#' @param x A Spark DataFrame -#' @param y A Spark DataFrame -#' @return A DataFrame containing the result of the intersect. -#' @family dataframe_funcs +#' @param x A SparkDataFrame +#' @param y A SparkDataFrame +#' @return A SparkDataFrame containing the result of the intersect. +#' @family SparkDataFrame functions +#' @aliases intersect,SparkDataFrame,SparkDataFrame-method #' @rdname intersect #' @name intersect #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' df1 <- jsonFile(sqlContext, path) -#' df2 <- jsonFile(sqlContext, path2) +#' sparkR.session() +#' df1 <- read.json(path) +#' df2 <- read.json(path2) #' intersectDF <- intersect(df, df2) #' } +#' @note intersect since 1.4.0 setMethod("intersect", - signature(x = "DataFrame", y = "DataFrame"), + signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y) { intersected <- callJMethod(x@sdf, "intersect", y@sdf) dataFrame(intersected) @@ -1748,238 +2830,279 @@ setMethod("intersect", #' except #' -#' Return a new DataFrame containing rows in this DataFrame -#' but not in another DataFrame. This is equivalent to `EXCEPT` in SQL. +#' Return a new SparkDataFrame containing rows in this SparkDataFrame +#' but not in another SparkDataFrame. This is equivalent to \code{EXCEPT} in SQL. #' -#' @param x A Spark DataFrame -#' @param y A Spark DataFrame -#' @return A DataFrame containing the result of the except operation. -#' @family dataframe_funcs +#' @param x a SparkDataFrame. +#' @param y a SparkDataFrame. +#' @return A SparkDataFrame containing the result of the except operation. +#' @family SparkDataFrame functions +#' @aliases except,SparkDataFrame,SparkDataFrame-method #' @rdname except #' @name except #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' df1 <- jsonFile(sqlContext, path) -#' df2 <- jsonFile(sqlContext, path2) +#' sparkR.session() +#' df1 <- read.json(path) +#' df2 <- read.json(path2) #' exceptDF <- except(df, df2) #' } #' @rdname except #' @export +#' @note except since 1.4.0 setMethod("except", - signature(x = "DataFrame", y = "DataFrame"), + signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y) { excepted <- callJMethod(x@sdf, "except", y@sdf) dataFrame(excepted) }) -#' Save the contents of the DataFrame to a data source +#' Save the contents of SparkDataFrame 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 +#' The data source is specified by the \code{source} and a set of options (...). +#' If \code{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: \cr -#' append: Contents of this DataFrame are expected to be appended to existing data. \cr -#' overwrite: Existing data is expected to be overwritten by the contents of this DataFrame. \cr -#' error: An exception is expected to be thrown. \cr -#' ignore: The save operation is expected to not save the contents of the DataFrame -#' and to not change the existing data. \cr +#' Additionally, mode is used to specify the behavior of the save operation when data already +#' exists in the data source. There are four modes: +#' \itemize{ +#' \item append: Contents of this SparkDataFrame are expected to be appended to existing data. +#' \item overwrite: Existing data is expected to be overwritten by the contents of this +#' SparkDataFrame. +#' \item error: An exception is expected to be thrown. +#' \item ignore: The save operation is expected to not save the contents of the SparkDataFrame +#' and to not change the existing data. +#' } #' -#' @param df A SparkSQL DataFrame -#' @param path A name for the table -#' @param source A name for external data source -#' @param mode One of 'append', 'overwrite', 'error', 'ignore' save mode +#' @param df a SparkDataFrame. +#' @param path a name for the table. +#' @param source a name for external data source. +#' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) +#' @param ... additional argument(s) passed to the method. #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions +#' @aliases write.df,SparkDataFrame-method #' @rdname write.df #' @name write.df -#' @aliases saveDF #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' write.df(df, "myfile", "parquet", "overwrite") -#' saveDF(df, parquetPath2, "parquet", mode = saveMode, mergeSchema = mergeSchema) +#' saveDF(df, parquetPath2, "parquet", mode = "append", mergeSchema = TRUE) #' } +#' @note write.df since 1.4.0 setMethod("write.df", - signature(df = "DataFrame", path = "character"), - function(df, path, source = NULL, mode = "append", ...){ - if (is.null(source)) { - sqlContext <- get(".sparkRSQLsc", envir = .sparkREnv) - source <- callJMethod(sqlContext, "getConf", "spark.sql.sources.default", - "org.apache.spark.sql.parquet") + signature(df = "SparkDataFrame"), + function(df, path = NULL, source = NULL, mode = "error", ...) { + if (!is.null(path) && !is.character(path)) { + stop("path should be character, NULL or omitted.") + } + if (!is.null(source) && !is.character(source)) { + stop("source should be character, NULL or omitted. It is the datasource specified ", + "in 'spark.sql.sources.default' configuration by default.") } - allModes <- c("append", "overwrite", "error", "ignore") - # nolint start - if (!(mode %in% allModes)) { - stop('mode should be one of "append", "overwrite", "error", "ignore"') + if (!is.character(mode)) { + stop("mode should be character or omitted. It is 'error' by default.") } - # nolint end - jmode <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "saveMode", mode) - options <- varargsToEnv(...) - if (!is.null(path)) { - options[["path"]] <- path + if (is.null(source)) { + source <- getDefaultSqlSource() } - callJMethod(df@sdf, "save", source, jmode, options) + write <- callJMethod(df@sdf, "write") + write <- callJMethod(write, "format", source) + write <- setWriteOptions(write, path = path, mode = mode, ...) + write <- handledCallJMethod(write, "save") }) -#' @family dataframe_funcs #' @rdname write.df #' @name saveDF +#' @aliases saveDF,SparkDataFrame,character-method #' @export +#' @note saveDF since 1.4.0 setMethod("saveDF", - signature(df = "DataFrame", path = "character"), - function(df, path, source = NULL, mode = "append", ...){ + signature(df = "SparkDataFrame", path = "character"), + function(df, path, source = NULL, mode = "error", ...) { write.df(df, path, source, mode, ...) }) -#' saveAsTable -#' -#' Save the contents of the DataFrame to a data source as a table +#' Save the contents of the SparkDataFrame 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 +#' The data source is specified by the \code{source} and a set of options (...). +#' If \code{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: \cr -#' append: Contents of this DataFrame are expected to be appended to existing data. \cr -#' overwrite: Existing data is expected to be overwritten by the contents of this DataFrame. \cr +#' append: Contents of this SparkDataFrame are expected to be appended to existing data. \cr +#' overwrite: Existing data is expected to be overwritten by the contents of this +#' SparkDataFrame. \cr #' error: An exception is expected to be thrown. \cr -#' ignore: The save operation is expected to not save the contents of the DataFrame +#' ignore: The save operation is expected to not save the contents of the SparkDataFrame #' and to not change the existing data. \cr #' -#' @param df A SparkSQL DataFrame -#' @param tableName A name for the table -#' @param source A name for external data source -#' @param mode One of 'append', 'overwrite', 'error', 'ignore' save mode +#' @param df a SparkDataFrame. +#' @param tableName a name for the table. +#' @param source a name for external data source. +#' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default). +#' @param ... additional option(s) passed to the method. #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions +#' @aliases saveAsTable,SparkDataFrame,character-method #' @rdname saveAsTable #' @name saveAsTable #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' saveAsTable(df, "myfile") #' } +#' @note saveAsTable since 1.4.0 setMethod("saveAsTable", - signature(df = "DataFrame", tableName = "character", source = "character", - mode = "character"), - function(df, tableName, source = NULL, mode="append", ...){ + signature(df = "SparkDataFrame", tableName = "character"), + function(df, tableName, source = NULL, mode="error", ...) { if (is.null(source)) { - sqlContext <- get(".sparkRSQLsc", envir = .sparkREnv) - source <- callJMethod(sqlContext, "getConf", "spark.sql.sources.default", - "org.apache.spark.sql.parquet") + source <- getDefaultSqlSource() } - allModes <- c("append", "overwrite", "error", "ignore") - # nolint start - if (!(mode %in% allModes)) { - stop('mode should be one of "append", "overwrite", "error", "ignore"') - } - # nolint end - jmode <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "saveMode", mode) - options <- varargsToEnv(...) - callJMethod(df@sdf, "saveAsTable", tableName, source, jmode, options) + jmode <- convertToJSaveMode(mode) + options <- varargsToStrEnv(...) + + write <- callJMethod(df@sdf, "write") + write <- callJMethod(write, "format", source) + write <- callJMethod(write, "mode", jmode) + write <- callJMethod(write, "options", options) + invisible(callJMethod(write, "saveAsTable", tableName)) }) #' describe #' -#' Computes statistics for numeric columns. -#' If no columns are given, this function computes statistics for all numerical columns. +#' Computes statistics for numeric and string columns. +#' If no columns are given, this function computes statistics for all numerical or string columns. #' -#' @param x A DataFrame to be computed. -#' @param col A string of name -#' @param ... Additional expressions -#' @return A DataFrame -#' @family dataframe_funcs +#' @param x a SparkDataFrame to be computed. +#' @param col a string of name. +#' @param ... additional expressions. +#' @return A SparkDataFrame. +#' @family SparkDataFrame functions +#' @aliases describe,SparkDataFrame,character-method describe,SparkDataFrame,ANY-method #' @rdname describe #' @name describe -#' @aliases summary #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) #' describe(df) #' describe(df, "col1") #' describe(df, "col1", "col2") #' } +#' @seealso See \link{summary} for expanded statistics and control over which statistics to compute. +#' @note describe(SparkDataFrame, character) since 1.4.0 setMethod("describe", - signature(x = "DataFrame", col = "character"), + signature(x = "SparkDataFrame", col = "character"), function(x, col, ...) { colList <- list(col, ...) sdf <- callJMethod(x@sdf, "describe", colList) dataFrame(sdf) }) -#' @family dataframe_funcs #' @rdname describe #' @name describe +#' @aliases describe,SparkDataFrame-method +#' @note describe(SparkDataFrame) since 1.4.0 setMethod("describe", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x) { - colList <- as.list(c(columns(x))) - sdf <- callJMethod(x@sdf, "describe", colList) + sdf <- callJMethod(x@sdf, "describe", list()) dataFrame(sdf) }) -#' @title Summary +#' summary +#' +#' Computes specified statistics for numeric and string columns. Available statistics are: +#' \itemize{ +#' \item count +#' \item mean +#' \item stddev +#' \item min +#' \item max +#' \item arbitrary approximate percentiles specified as a percentage (eg, "75%") +#' } +#' If no statistics are given, this function computes count, mean, stddev, min, +#' approximate quartiles (percentiles at 25%, 50%, and 75%), and max. +#' This function is meant for exploratory data analysis, as we make no guarantee about the +#' backward compatibility of the schema of the resulting Dataset. If you want to +#' programmatically compute summary statistics, use the \code{agg} function instead. #' -#' @description Computes statistics for numeric columns of the DataFrame #' -#' @family dataframe_funcs +#' @param object a SparkDataFrame to be summarized. +#' @param ... (optional) statistics to be computed for all columns. +#' @return A SparkDataFrame. +#' @family SparkDataFrame functions #' @rdname summary #' @name summary +#' @aliases summary,SparkDataFrame-method +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' summary(df) +#' summary(df, "min", "25%", "75%", "max") +#' summary(select(df, "age", "height")) +#' } +#' @note summary(SparkDataFrame) since 1.5.0 +#' @note The statistics provided by \code{summary} were change in 2.3.0 use \link{describe} for previous defaults. +#' @seealso \link{describe} setMethod("summary", - signature(x = "DataFrame"), - function(x) { - describe(x) + signature(object = "SparkDataFrame"), + function(object, ...) { + statisticsList <- list(...) + sdf <- callJMethod(object@sdf, "summary", statisticsList) + dataFrame(sdf) }) -#' dropna +#' A set of SparkDataFrame functions working with NA values #' -#' Returns a new DataFrame omitting rows with null values. +#' dropna, na.omit - Returns a new SparkDataFrame omitting rows with null values. #' -#' @param x A SparkSQL DataFrame. +#' @param x a SparkDataFrame. #' @param how "any" or "all". #' if "any", drop a row if it contains any nulls. #' if "all", drop a row only if all its values are null. -#' if minNonNulls is specified, how is ignored. -#' @param minNonNulls If specified, drop rows that have less than -#' minNonNulls non-null values. +#' if \code{minNonNulls} is specified, how is ignored. +#' @param minNonNulls if specified, drop rows that have less than +#' \code{minNonNulls} non-null values. #' This overwrites the how parameter. -#' @param cols Optional list of column names to consider. -#' @return A DataFrame +#' @param cols optional list of column names to consider. In \code{fillna}, +#' columns specified in cols that do not have matching data +#' type are ignored. For example, if value is a character, and +#' subset contains a non-character column, then the non-character +#' column is simply ignored. +#' @return A SparkDataFrame. #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname nafunctions +#' @aliases dropna,SparkDataFrame-method #' @name dropna -#' @aliases na.omit #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlCtx, path) +#' df <- read.json(path) #' dropna(df) #' } +#' @note dropna since 1.4.0 setMethod("dropna", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x, how = c("any", "all"), minNonNulls = NULL, cols = NULL) { how <- match.arg(how) if (is.null(cols)) { @@ -1995,52 +3118,46 @@ setMethod("dropna", dataFrame(sdf) }) -#' @family dataframe_funcs +#' @param object a SparkDataFrame. +#' @param ... further arguments to be passed to or from other methods. #' @rdname nafunctions #' @name na.omit +#' @aliases na.omit,SparkDataFrame-method #' @export +#' @note na.omit since 1.5.0 setMethod("na.omit", - signature(object = "DataFrame"), + signature(object = "SparkDataFrame"), function(object, how = c("any", "all"), minNonNulls = NULL, cols = NULL) { dropna(object, how, minNonNulls, cols) }) -#' fillna -#' -#' Replace null values. +#' fillna - Replace null values. #' -#' @param x A SparkSQL DataFrame. -#' @param value Value to replace null values with. +#' @param value value to replace null values with. #' Should be an integer, numeric, character or named list. #' If the value is a named list, then cols is ignored and #' value must be a mapping from column name (character) to #' replacement value. The replacement value must be an #' integer, numeric or character. -#' @param cols optional list of column names to consider. -#' Columns specified in cols that do not have matching data -#' type are ignored. For example, if value is a character, and -#' subset contains a non-character column, then the non-character -#' column is simply ignored. -#' @return A DataFrame #' -#' @family dataframe_funcs #' @rdname nafunctions #' @name fillna +#' @aliases fillna,SparkDataFrame-method #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlCtx, path) +#' df <- read.json(path) #' fillna(df, 1) #' fillna(df, list("age" = 20, "name" = "unknown")) #' } +#' @note fillna since 1.4.0 setMethod("fillna", - signature(x = "DataFrame"), + signature(x = "SparkDataFrame"), function(x, value, cols = NULL) { if (!(class(value) %in% c("integer", "numeric", "character", "list"))) { - stop("value should be an integer, numeric, charactor or named list.") + stop("value should be an integer, numeric, character or named list.") } if (class(value) == "list") { @@ -2052,7 +3169,7 @@ setMethod("fillna", # Check each item in the named list is of valid type lapply(value, function(v) { if (!(class(v) %in% c("integer", "numeric", "character"))) { - stop("Each item in value should be an integer, numeric or charactor.") + stop("Each item in value should be an integer, numeric or character.") } }) @@ -2080,57 +3197,719 @@ setMethod("fillna", dataFrame(sdf) }) -#' This function downloads the contents of a DataFrame into an R's data.frame. +#' Download data from a SparkDataFrame into a R data.frame +#' +#' This function downloads the contents of a SparkDataFrame into an R's data.frame. #' Since data.frames are held in memory, ensure that you have enough memory #' in your system to accommodate the contents. #' -#' @title Download data from a DataFrame into a data.frame -#' @param x a DataFrame -#' @return a data.frame -#' @family dataframe_funcs +#' @param x a SparkDataFrame. +#' @param row.names \code{NULL} or a character vector giving the row names for the data frame. +#' @param optional If \code{TRUE}, converting column names is optional. +#' @param ... additional arguments to pass to base::as.data.frame. +#' @return A data.frame. +#' @family SparkDataFrame functions +#' @aliases as.data.frame,SparkDataFrame-method #' @rdname as.data.frame -#' @examples \dontrun{ -#' -#' irisDF <- createDataFrame(sqlContext, iris) +#' @examples +#' \dontrun{ +#' irisDF <- createDataFrame(iris) #' df <- as.data.frame(irisDF[irisDF$Species == "setosa", ]) #' } +#' @note as.data.frame since 1.6.0 setMethod("as.data.frame", - signature(x = "DataFrame"), - function(x, ...) { - # Check if additional parameters have been passed - if (length(list(...)) > 0) { - stop(paste("Unused argument(s): ", paste(list(...), collapse=", "))) - } - collect(x) + signature(x = "SparkDataFrame"), + function(x, row.names = NULL, optional = FALSE, ...) { + as.data.frame(collect(x), row.names, optional, ...) }) -#' The specified DataFrame is attached to the R search path. This means that -#' the DataFrame is searched by R when evaluating a variable, so columns in -#' the DataFrame can be accessed by simply giving their names. +#' Attach SparkDataFrame to R search path +#' +#' The specified SparkDataFrame is attached to the R search path. This means that +#' the SparkDataFrame is searched by R when evaluating a variable, so columns in +#' the SparkDataFrame can be accessed by simply giving their names. #' -#' @family dataframe_funcs +#' @family SparkDataFrame functions #' @rdname attach -#' @title Attach DataFrame to R search path -#' @param what (DataFrame) The DataFrame to attach +#' @aliases attach,SparkDataFrame-method +#' @param what (SparkDataFrame) The SparkDataFrame to attach #' @param pos (integer) Specify position in search() where to attach. -#' @param name (character) Name to use for the attached DataFrame. Names +#' @param name (character) Name to use for the attached SparkDataFrame. Names #' starting with package: are reserved for library. #' @param warn.conflicts (logical) If TRUE, warnings are printed about conflicts -#' from attaching the database, unless that DataFrame contains an object +#' from attaching the database, unless that SparkDataFrame contains an object #' @examples #' \dontrun{ #' attach(irisDf) #' summary(Sepal_Width) #' } #' @seealso \link{detach} +#' @note attach since 1.6.0 setMethod("attach", - signature(what = "DataFrame"), + signature(what = "SparkDataFrame"), function(what, pos = 2, name = deparse(substitute(what)), warn.conflicts = TRUE) { - cols <- columns(what) - stopifnot(length(cols) > 0) - newEnv <- new.env() - for (i in 1:length(cols)) { - assign(x = cols[i], value = what[, cols[i]], envir = newEnv) - } + newEnv <- assignNewEnv(what) attach(newEnv, pos = pos, name = name, warn.conflicts = warn.conflicts) }) + +#' Evaluate a R expression in an environment constructed from a SparkDataFrame +#' +#' Evaluate a R expression in an environment constructed from a SparkDataFrame +#' with() allows access to columns of a SparkDataFrame by simply referring to +#' their name. It appends every column of a SparkDataFrame into a new +#' environment. Then, the given expression is evaluated in this new +#' environment. +#' +#' @rdname with +#' @family SparkDataFrame functions +#' @aliases with,SparkDataFrame-method +#' @param data (SparkDataFrame) SparkDataFrame to use for constructing an environment. +#' @param expr (expression) Expression to evaluate. +#' @param ... arguments to be passed to future methods. +#' @examples +#' \dontrun{ +#' with(irisDf, nrow(Sepal_Width)) +#' } +#' @seealso \link{attach} +#' @note with since 1.6.0 +setMethod("with", + signature(data = "SparkDataFrame"), + function(data, expr, ...) { + newEnv <- assignNewEnv(data) + eval(substitute(expr), envir = newEnv, enclos = newEnv) + }) + +#' Compactly display the structure of a dataset +#' +#' Display the structure of a SparkDataFrame, including column names, column types, as well as a +#' a small sample of rows. +#' +#' @name str +#' @rdname str +#' @aliases str,SparkDataFrame-method +#' @family SparkDataFrame functions +#' @param object a SparkDataFrame +#' @examples +#' \dontrun{ +#' # Create a SparkDataFrame from the Iris dataset +#' irisDF <- createDataFrame(iris) +#' +#' # Show the structure of the SparkDataFrame +#' str(irisDF) +#' } +#' @note str since 1.6.1 +setMethod("str", + signature(object = "SparkDataFrame"), + function(object) { + + # TODO: These could be made global parameters, though in R it's not the case + MAX_CHAR_PER_ROW <- 120 + MAX_COLS <- 100 + + # Get the column names and types of the DataFrame + names <- names(object) + types <- coltypes(object) + + # Get the first elements of the dataset. Limit number of columns accordingly + localDF <- if (ncol(object) > MAX_COLS) { + head(object[, c(1:MAX_COLS)]) + } else { + head(object) + } + + # The number of observations will not be displayed as computing the + # number of rows is a very expensive operation + cat(paste0("'", class(object), "': ", length(names), " variables:\n")) + + if (nrow(localDF) > 0) { + for (i in 1 : ncol(localDF)) { + # Get the first elements for each column + + firstElements <- if (types[i] == "character") { + paste(paste0("\"", localDF[, i], "\""), collapse = " ") + } else { + paste(localDF[, i], collapse = " ") + } + + # Add the corresponding number of spaces for alignment + spaces <- paste(rep(" ", max(nchar(names) - nchar(names[i]))), collapse = "") + + # Get the short type. For 'character', it would be 'chr'; + # 'for numeric', it's 'num', etc. + dataType <- SHORT_TYPES[[types[i]]] + if (is.null(dataType)) { + dataType <- substring(types[i], 1, 3) + } + + # Concatenate the colnames, coltypes, and first + # elements of each column + line <- paste0(" $ ", names[i], spaces, ": ", + dataType, " ", firstElements) + + # Chop off extra characters if this is too long + cat(substr(line, 1, MAX_CHAR_PER_ROW)) + cat("\n") + } + + if (ncol(localDF) < ncol(object)) { + cat(paste0("\nDisplaying first ", ncol(localDF), " columns only.")) + } + } + }) + +#' drop +#' +#' Returns a new SparkDataFrame with columns dropped. +#' This is a no-op if schema doesn't contain column name(s). +#' +#' @param x a SparkDataFrame. +#' @param col a character vector of column names or a Column. +#' @param ... further arguments to be passed to or from other methods. +#' @return A SparkDataFrame. +#' +#' @family SparkDataFrame functions +#' @rdname drop +#' @name drop +#' @aliases drop,SparkDataFrame-method +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' drop(df, "col1") +#' drop(df, c("col1", "col2")) +#' drop(df, df$col1) +#' } +#' @note drop since 2.0.0 +setMethod("drop", + signature(x = "SparkDataFrame"), + function(x, col) { + stopifnot(class(col) == "character" || class(col) == "Column") + + if (class(col) == "Column") { + sdf <- callJMethod(x@sdf, "drop", col@jc) + } else { + sdf <- callJMethod(x@sdf, "drop", as.list(col)) + } + dataFrame(sdf) + }) + +# Expose base::drop +#' @name drop +#' @rdname drop +#' @aliases drop,ANY-method +#' @export +setMethod("drop", + signature(x = "ANY"), + function(x) { + base::drop(x) + }) + +#' Compute histogram statistics for given column +#' +#' This function computes a histogram for a given SparkR Column. +#' +#' @name histogram +#' @param nbins the number of bins (optional). Default value is 10. +#' @param col the column as Character string or a Column to build the histogram from. +#' @param df the SparkDataFrame containing the Column to build the histogram from. +#' @return a data.frame with the histogram statistics, i.e., counts and centroids. +#' @rdname histogram +#' @aliases histogram,SparkDataFrame,characterOrColumn-method +#' @family SparkDataFrame functions +#' @export +#' @examples +#' \dontrun{ +#' +#' # Create a SparkDataFrame from the Iris dataset +#' irisDF <- createDataFrame(iris) +#' +#' # Compute histogram statistics +#' histStats <- histogram(irisDF, irisDF$Sepal_Length, nbins = 12) +#' +#' # Once SparkR has computed the histogram statistics, the histogram can be +#' # rendered using the ggplot2 library: +#' +#' require(ggplot2) +#' plot <- ggplot(histStats, aes(x = centroids, y = counts)) + +#' geom_bar(stat = "identity") + +#' xlab("Sepal_Length") + ylab("Frequency") +#' } +#' @note histogram since 2.0.0 +setMethod("histogram", + signature(df = "SparkDataFrame", col = "characterOrColumn"), + function(df, col, nbins = 10) { + # Validate nbins + if (nbins < 2) { + stop("The number of bins must be a positive integer number greater than 1.") + } + + # Round nbins to the smallest integer + nbins <- floor(nbins) + + # Validate col + if (is.null(col)) { + stop("col must be specified.") + } + + colname <- col + x <- if (class(col) == "character") { + if (!colname %in% names(df)) { + stop("Specified colname does not belong to the given SparkDataFrame.") + } + + # Filter NA values in the target column and remove all other columns + df <- na.omit(df[, colname, drop = F]) + getColumn(df, colname) + + } else if (class(col) == "Column") { + + # The given column needs to be appended to the SparkDataFrame so that we can + # use method describe() to compute statistics in one single pass. The new + # column must have a name that doesn't exist in the dataset. + # To do so, we generate a random column name with more characters than the + # longest colname in the dataset, but no more than 100 (think of a UUID). + # This column name will never be visible to the user, so the name is irrelevant. + # Limiting the colname length to 100 makes debugging easier and it does + # introduce a negligible probability of collision: assuming the user has 1 million + # columns AND all of them have names 100 characters long (which is very unlikely), + # AND they run 1 billion histograms, the probability of collision will roughly be + # 1 in 4.4 x 10 ^ 96 + colname <- paste(base::sample(c(letters, LETTERS), + size = min(max(nchar(colnames(df))) + 1, 100), + replace = TRUE), + collapse = "") + + # Append the given column to the dataset. This is to support Columns that + # don't belong to the SparkDataFrame but are rather expressions + df <- withColumn(df, colname, col) + + # Filter NA values in the target column. Cannot remove all other columns + # since given Column may be an expression on one or more existing columns + df <- na.omit(df) + + col + } + + stats <- collect(describe(df[, colname, drop = F])) + min <- as.numeric(stats[4, 2]) + max <- as.numeric(stats[5, 2]) + + # Normalize the data + xnorm <- (x - min) / (max - min) + + # Round the data to 4 significant digits. This is to avoid rounding issues. + xnorm <- cast(xnorm * 10000, "integer") / 10000.0 + + # Since min = 0, max = 1 (data is already normalized) + normBinSize <- 1 / nbins + binsize <- (max - min) / nbins + approxBins <- xnorm / normBinSize + + # Adjust values that are equal to the upper bound of each bin + bins <- cast(approxBins - + ifelse(approxBins == cast(approxBins, "integer") & x != min, 1, 0), + "integer") + + df$bins <- bins + histStats <- collect(count(groupBy(df, "bins"))) + names(histStats) <- c("bins", "counts") + + # Fill bins with zero counts + y <- data.frame("bins" = seq(0, nbins - 1)) + histStats <- merge(histStats, y, all.x = T, all.y = T) + histStats[is.na(histStats$count), 2] <- 0 + + # Compute centroids + histStats$centroids <- histStats$bins * binsize + min + binsize / 2 + + # Return the statistics + return(histStats) + }) + +#' Save the content of SparkDataFrame to an external database table via JDBC. +#' +#' Save the content of the SparkDataFrame to an external database table via JDBC. Additional JDBC +#' database connection properties can be set (...) +#' +#' Also, mode is used to specify the behavior of the save operation when +#' data already exists in the data source. There are four modes: +#' \itemize{ +#' \item append: Contents of this SparkDataFrame are expected to be appended to existing data. +#' \item overwrite: Existing data is expected to be overwritten by the contents of this +#' SparkDataFrame. +#' \item error: An exception is expected to be thrown. +#' \item ignore: The save operation is expected to not save the contents of the SparkDataFrame +#' and to not change the existing data. +#' } +#' +#' @param x a SparkDataFrame. +#' @param url JDBC database url of the form \code{jdbc:subprotocol:subname}. +#' @param tableName yhe name of the table in the external database. +#' @param mode one of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default). +#' @param ... additional JDBC database connection properties. +#' @family SparkDataFrame functions +#' @rdname write.jdbc +#' @name write.jdbc +#' @aliases write.jdbc,SparkDataFrame,character,character-method +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' jdbcUrl <- "jdbc:mysql://localhost:3306/databasename" +#' write.jdbc(df, jdbcUrl, "table", user = "username", password = "password") +#' } +#' @note write.jdbc since 2.0.0 +setMethod("write.jdbc", + signature(x = "SparkDataFrame", url = "character", tableName = "character"), + function(x, url, tableName, mode = "error", ...) { + jmode <- convertToJSaveMode(mode) + jprops <- varargsToJProperties(...) + write <- callJMethod(x@sdf, "write") + write <- callJMethod(write, "mode", jmode) + invisible(handledCallJMethod(write, "jdbc", url, tableName, jprops)) + }) + +#' randomSplit +#' +#' Return a list of randomly split dataframes with the provided weights. +#' +#' @param x A SparkDataFrame +#' @param weights A vector of weights for splits, will be normalized if they don't sum to 1 +#' @param seed A seed to use for random split +#' +#' @family SparkDataFrame functions +#' @aliases randomSplit,SparkDataFrame,numeric-method +#' @rdname randomSplit +#' @name randomSplit +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df <- createDataFrame(data.frame(id = 1:1000)) +#' df_list <- randomSplit(df, c(2, 3, 5), 0) +#' # df_list contains 3 SparkDataFrames with each having about 200, 300 and 500 rows respectively +#' sapply(df_list, count) +#' } +#' @note randomSplit since 2.0.0 +setMethod("randomSplit", + signature(x = "SparkDataFrame", weights = "numeric"), + function(x, weights, seed) { + if (!all(sapply(weights, function(c) { c >= 0 }))) { + stop("all weight values should not be negative") + } + normalized_list <- as.list(weights / sum(weights)) + if (!missing(seed)) { + sdfs <- callJMethod(x@sdf, "randomSplit", normalized_list, as.integer(seed)) + } else { + sdfs <- callJMethod(x@sdf, "randomSplit", normalized_list) + } + sapply(sdfs, dataFrame) + }) + +#' getNumPartitions +#' +#' Return the number of partitions +#' +#' @param x A SparkDataFrame +#' @family SparkDataFrame functions +#' @aliases getNumPartitions,SparkDataFrame-method +#' @rdname getNumPartitions +#' @name getNumPartitions +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df <- createDataFrame(cars, numPartitions = 2) +#' getNumPartitions(df) +#' } +#' @note getNumPartitions since 2.1.1 +setMethod("getNumPartitions", + signature(x = "SparkDataFrame"), + function(x) { + callJMethod(callJMethod(x@sdf, "rdd"), "getNumPartitions") + }) + +#' isStreaming +#' +#' Returns TRUE if this SparkDataFrame contains one or more sources that continuously return data +#' as it arrives. +#' +#' @param x A SparkDataFrame +#' @return TRUE if this SparkDataFrame is from a streaming source +#' @family SparkDataFrame functions +#' @aliases isStreaming,SparkDataFrame-method +#' @rdname isStreaming +#' @name isStreaming +#' @seealso \link{read.stream} \link{write.stream} +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df <- read.stream("socket", host = "localhost", port = 9999) +#' isStreaming(df) +#' } +#' @note isStreaming since 2.2.0 +#' @note experimental +setMethod("isStreaming", + signature(x = "SparkDataFrame"), + function(x) { + callJMethod(x@sdf, "isStreaming") + }) + +#' Write the streaming SparkDataFrame to a data source. +#' +#' The data source is specified by the \code{source} and a set of options (...). +#' If \code{source} is not specified, the default data source configured by +#' spark.sql.sources.default will be used. +#' +#' Additionally, \code{outputMode} specifies how data of a streaming SparkDataFrame is written to a +#' output data source. There are three modes: +#' \itemize{ +#' \item append: Only the new rows in the streaming SparkDataFrame will be written out. This +#' output mode can be only be used in queries that do not contain any aggregation. +#' \item complete: All the rows in the streaming SparkDataFrame will be written out every time +#' there are some updates. This output mode can only be used in queries that +#' contain aggregations. +#' \item update: Only the rows that were updated in the streaming SparkDataFrame will be written +#' out every time there are some updates. If the query doesn't contain aggregations, +#' it will be equivalent to \code{append} mode. +#' } +#' +#' @param df a streaming SparkDataFrame. +#' @param source a name for external data source. +#' @param outputMode one of 'append', 'complete', 'update'. +#' @param ... additional argument(s) passed to the method. +#' +#' @family SparkDataFrame functions +#' @seealso \link{read.stream} +#' @aliases write.stream,SparkDataFrame-method +#' @rdname write.stream +#' @name write.stream +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df <- read.stream("socket", host = "localhost", port = 9999) +#' isStreaming(df) +#' wordCounts <- count(group_by(df, "value")) +#' +#' # console +#' q <- write.stream(wordCounts, "console", outputMode = "complete") +#' # text stream +#' q <- write.stream(df, "text", path = "/home/user/out", checkpointLocation = "/home/user/cp") +#' # memory stream +#' q <- write.stream(wordCounts, "memory", queryName = "outs", outputMode = "complete") +#' head(sql("SELECT * from outs")) +#' queryName(q) +#' +#' stopQuery(q) +#' } +#' @note write.stream since 2.2.0 +#' @note experimental +setMethod("write.stream", + signature(df = "SparkDataFrame"), + function(df, source = NULL, outputMode = NULL, ...) { + if (!is.null(source) && !is.character(source)) { + stop("source should be character, NULL or omitted. It is the data source specified ", + "in 'spark.sql.sources.default' configuration by default.") + } + if (!is.null(outputMode) && !is.character(outputMode)) { + stop("outputMode should be character or omitted.") + } + if (is.null(source)) { + source <- getDefaultSqlSource() + } + options <- varargsToStrEnv(...) + write <- handledCallJMethod(df@sdf, "writeStream") + write <- callJMethod(write, "format", source) + if (!is.null(outputMode)) { + write <- callJMethod(write, "outputMode", outputMode) + } + write <- callJMethod(write, "options", options) + ssq <- handledCallJMethod(write, "start") + streamingQuery(ssq) + }) + +#' checkpoint +#' +#' Returns a checkpointed version of this SparkDataFrame. Checkpointing can be used to truncate the +#' logical plan, which is especially useful in iterative algorithms where the plan may grow +#' exponentially. It will be saved to files inside the checkpoint directory set with +#' \code{setCheckpointDir} +#' +#' @param x A SparkDataFrame +#' @param eager whether to checkpoint this SparkDataFrame immediately +#' @return a new checkpointed SparkDataFrame +#' @family SparkDataFrame functions +#' @aliases checkpoint,SparkDataFrame-method +#' @rdname checkpoint +#' @name checkpoint +#' @seealso \link{setCheckpointDir} +#' @export +#' @examples +#'\dontrun{ +#' setCheckpointDir("/checkpoint") +#' df <- checkpoint(df) +#' } +#' @note checkpoint since 2.2.0 +setMethod("checkpoint", + signature(x = "SparkDataFrame"), + function(x, eager = TRUE) { + df <- callJMethod(x@sdf, "checkpoint", as.logical(eager)) + dataFrame(df) + }) + +#' cube +#' +#' Create a multi-dimensional cube for the SparkDataFrame using the specified columns. +#' +#' If grouping expression is missing \code{cube} creates a single global aggregate and is equivalent to +#' direct application of \link{agg}. +#' +#' @param x a SparkDataFrame. +#' @param ... character name(s) or Column(s) to group on. +#' @return A GroupedData. +#' @family SparkDataFrame functions +#' @aliases cube,SparkDataFrame-method +#' @rdname cube +#' @name cube +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame(mtcars) +#' mean(cube(df, "cyl", "gear", "am"), "mpg") +#' +#' # Following calls are equivalent +#' agg(cube(df), mean(df$mpg)) +#' agg(df, mean(df$mpg)) +#' } +#' @note cube since 2.3.0 +#' @seealso \link{agg}, \link{groupBy}, \link{rollup} +setMethod("cube", + signature(x = "SparkDataFrame"), + function(x, ...) { + cols <- list(...) + jcol <- lapply(cols, function(x) if (class(x) == "Column") x@jc else column(x)@jc) + sgd <- callJMethod(x@sdf, "cube", jcol) + groupedData(sgd) + }) + +#' rollup +#' +#' Create a multi-dimensional rollup for the SparkDataFrame using the specified columns. +#' +#' If grouping expression is missing \code{rollup} creates a single global aggregate and is equivalent to +#' direct application of \link{agg}. +#' +#' @param x a SparkDataFrame. +#' @param ... character name(s) or Column(s) to group on. +#' @return A GroupedData. +#' @family SparkDataFrame functions +#' @aliases rollup,SparkDataFrame-method +#' @rdname rollup +#' @name rollup +#' @export +#' @examples +#'\dontrun{ +#' df <- createDataFrame(mtcars) +#' mean(rollup(df, "cyl", "gear", "am"), "mpg") +#' +#' # Following calls are equivalent +#' agg(rollup(df), mean(df$mpg)) +#' agg(df, mean(df$mpg)) +#' } +#' @note rollup since 2.3.0 +#' @seealso \link{agg}, \link{cube}, \link{groupBy} +setMethod("rollup", + signature(x = "SparkDataFrame"), + function(x, ...) { + cols <- list(...) + jcol <- lapply(cols, function(x) if (class(x) == "Column") x@jc else column(x)@jc) + sgd <- callJMethod(x@sdf, "rollup", jcol) + groupedData(sgd) + }) + +#' hint +#' +#' Specifies execution plan hint and return a new SparkDataFrame. +#' +#' @param x a SparkDataFrame. +#' @param name a name of the hint. +#' @param ... optional parameters for the hint. +#' @return A SparkDataFrame. +#' @family SparkDataFrame functions +#' @aliases hint,SparkDataFrame,character-method +#' @rdname hint +#' @name hint +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame(mtcars) +#' avg_mpg <- mean(groupBy(createDataFrame(mtcars), "cyl"), "mpg") +#' +#' head(join(df, hint(avg_mpg, "broadcast"), df$cyl == avg_mpg$cyl)) +#' } +#' @note hint since 2.2.0 +setMethod("hint", + signature(x = "SparkDataFrame", name = "character"), + function(x, name, ...) { + parameters <- list(...) + stopifnot(all(sapply(parameters, is.character))) + jdf <- callJMethod(x@sdf, "hint", name, parameters) + dataFrame(jdf) + }) + +#' alias +#' +#' @aliases alias,SparkDataFrame-method +#' @family SparkDataFrame functions +#' @rdname alias +#' @name alias +#' @export +#' @examples +#' \dontrun{ +#' df <- alias(createDataFrame(mtcars), "mtcars") +#' avg_mpg <- alias(agg(groupBy(df, df$cyl), avg(df$mpg)), "avg_mpg") +#' +#' head(select(df, column("mtcars.mpg"))) +#' head(join(df, avg_mpg, column("mtcars.cyl") == column("avg_mpg.cyl"))) +#' } +#' @note alias(SparkDataFrame) since 2.3.0 +setMethod("alias", + signature(object = "SparkDataFrame"), + function(object, data) { + stopifnot(is.character(data)) + sdf <- callJMethod(object@sdf, "alias", data) + dataFrame(sdf) + }) + +#' broadcast +#' +#' Return a new SparkDataFrame marked as small enough for use in broadcast joins. +#' +#' Equivalent to \code{hint(x, "broadcast")}. +#' +#' @param x a SparkDataFrame. +#' @return a SparkDataFrame. +#' +#' @aliases broadcast,SparkDataFrame-method +#' @family SparkDataFrame functions +#' @rdname broadcast +#' @name broadcast +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame(mtcars) +#' avg_mpg <- mean(groupBy(createDataFrame(mtcars), "cyl"), "mpg") +#' +#' head(join(df, broadcast(avg_mpg), df$cyl == avg_mpg$cyl)) +#' } +#' @note broadcast since 2.3.0 +setMethod("broadcast", + signature(x = "SparkDataFrame"), + function(x) { + sdf <- callJStatic("org.apache.spark.sql.functions", "broadcast", x@sdf) + dataFrame(sdf) + }) diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 051e441d4e06..15ca212acf87 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -19,16 +19,17 @@ setOldClass("jobj") -# @title S4 class that represents an RDD -# @description RDD can be created using functions like -# \code{parallelize}, \code{textFile} etc. -# @rdname RDD -# @seealso parallelize, textFile -# -# @slot env An R environment that stores bookkeeping states of the RDD -# @slot jrdd Java object reference to the backing JavaRDD -# to an RDD -# @export +#' S4 class that represents an RDD +#' +#' RDD can be created using functions like +#' \code{parallelize}, \code{textFile} etc. +#' +#' @rdname RDD +#' @seealso parallelize, textFile +#' @slot env An R environment that stores bookkeeping states of the RDD +#' @slot jrdd Java object reference to the backing JavaRDD +#' to an RDD +#' @noRd setClass("RDD", slots = list(env = "environment", jrdd = "jobj")) @@ -47,7 +48,7 @@ setMethod("initialize", "RDD", function(.Object, jrdd, serializedMode, # RDD has three serialization types: # byte: The RDD stores data serialized in R. # string: The RDD stores data as strings. - # row: The RDD stores the serialized rows of a DataFrame. + # row: The RDD stores the serialized rows of a SparkDataFrame. # We use an environment to store mutable states inside an RDD object. # Note that R's call-by-value semantics makes modifying slots inside an @@ -66,9 +67,9 @@ setMethod("initialize", "RDD", function(.Object, jrdd, serializedMode, .Object }) -setMethod("show", "RDD", +setMethod("showRDD", "RDD", function(object) { - cat(paste(callJMethod(getJRDD(object), "toString"), "\n", sep="")) + cat(paste(callJMethod(getJRDD(object), "toString"), "\n", sep = "")) }) setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) { @@ -111,14 +112,13 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) .Object }) -# @rdname RDD -# @export -# -# @param jrdd Java object reference to the backing JavaRDD -# @param serializedMode Use "byte" if the RDD stores data serialized in R, "string" if the RDD -# stores strings, and "row" if the RDD stores the rows of a DataFrame -# @param isCached TRUE if the RDD is cached -# @param isCheckpointed TRUE if the RDD has been checkpointed +#' @rdname RDD +#' @noRd +#' @param jrdd Java object reference to the backing JavaRDD +#' @param serializedMode Use "byte" if the RDD stores data serialized in R, "string" if the RDD +#' stores strings, and "row" if the RDD stores the rows of a SparkDataFrame +#' @param isCached TRUE if the RDD is cached +#' @param isCheckpointed TRUE if the RDD has been checkpointed RDD <- function(jrdd, serializedMode = "byte", isCached = FALSE, isCheckpointed = FALSE) { new("RDD", jrdd, serializedMode, isCached, isCheckpointed) @@ -182,7 +182,7 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), } # Save the serialization flag after we create a RRDD rdd@env$serializedMode <- serializedMode - rdd@env$jrdd_val <- callJMethod(rddRef, "asJavaRDD") # rddRef$asJavaRDD() + rdd@env$jrdd_val <- callJMethod(rddRef, "asJavaRDD") rdd@env$jrdd_val }) @@ -201,20 +201,21 @@ setValidity("RDD", ############ Actions and Transformations ############ -# Persist an RDD -# -# Persist this RDD with the default storage level (MEMORY_ONLY). -# -# @param x The RDD to cache -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10, 2L) -# cache(rdd) -#} -# @rdname cache-methods -# @aliases cache,RDD-method -setMethod("cache", +#' Persist an RDD +#' +#' Persist this RDD with the default storage level (MEMORY_ONLY). +#' +#' @param x The RDD to cache +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' cache(rdd) +#'} +#' @rdname cache-methods +#' @aliases cache,RDD-method +#' @noRd +setMethod("cacheRDD", signature(x = "RDD"), function(x) { callJMethod(getJRDD(x), "cache") @@ -222,23 +223,24 @@ setMethod("cache", x }) -# Persist an RDD -# -# Persist this RDD with the specified storage level. For details of the -# supported storage levels, refer to -# http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. -# -# @param x The RDD to persist -# @param newLevel The new storage level to be assigned -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10, 2L) -# persist(rdd, "MEMORY_AND_DISK") -#} -# @rdname persist -# @aliases persist,RDD-method -setMethod("persist", +#' Persist an RDD +#' +#' Persist this RDD with the specified storage level. For details of the +#' supported storage levels, refer to +#'\url{http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence}. +#' +#' @param x The RDD to persist +#' @param newLevel The new storage level to be assigned +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' persistRDD(rdd, "MEMORY_AND_DISK") +#'} +#' @rdname persist +#' @aliases persist,RDD-method +#' @noRd +setMethod("persistRDD", signature(x = "RDD", newLevel = "character"), function(x, newLevel = "MEMORY_ONLY") { callJMethod(getJRDD(x), "persist", getStorageLevel(newLevel)) @@ -246,22 +248,23 @@ setMethod("persist", x }) -# Unpersist an RDD -# -# Mark the RDD as non-persistent, and remove all blocks for it from memory and -# disk. -# -# @param x The RDD to unpersist -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10, 2L) -# cache(rdd) # rdd@@env$isCached == TRUE -# unpersist(rdd) # rdd@@env$isCached == FALSE -#} -# @rdname unpersist-methods -# @aliases unpersist,RDD-method -setMethod("unpersist", +#' Unpersist an RDD +#' +#' Mark the RDD as non-persistent, and remove all blocks for it from memory and +#' disk. +#' +#' @param x The RDD to unpersist +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' cache(rdd) # rdd@@env$isCached == TRUE +#' unpersistRDD(rdd) # rdd@@env$isCached == FALSE +#'} +#' @rdname unpersist +#' @aliases unpersist,RDD-method +#' @noRd +setMethod("unpersistRDD", signature(x = "RDD"), function(x) { callJMethod(getJRDD(x), "unpersist") @@ -269,25 +272,26 @@ setMethod("unpersist", x }) -# Checkpoint an RDD -# -# Mark this RDD for checkpointing. It will be saved to a file inside the -# checkpoint directory set with setCheckpointDir() and all references to its -# parent RDDs will be removed. This function must be called before any job has -# been executed on this RDD. It is strongly recommended that this RDD is -# persisted in memory, otherwise saving it on a file will require recomputation. -# -# @param x The RDD to checkpoint -# @examples -#\dontrun{ -# sc <- sparkR.init() -# setCheckpointDir(sc, "checkpoint") -# rdd <- parallelize(sc, 1:10, 2L) -# checkpoint(rdd) -#} -# @rdname checkpoint-methods -# @aliases checkpoint,RDD-method -setMethod("checkpoint", +#' Checkpoint an RDD +#' +#' Mark this RDD for checkpointing. It will be saved to a file inside the +#' checkpoint directory set with setCheckpointDir() and all references to its +#' parent RDDs will be removed. This function must be called before any job has +#' been executed on this RDD. It is strongly recommended that this RDD is +#' persisted in memory, otherwise saving it on a file will require recomputation. +#' +#' @param x The RDD to checkpoint +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' setCheckpointDir(sc, "checkpoint") +#' rdd <- parallelize(sc, 1:10, 2L) +#' checkpoint(rdd) +#'} +#' @rdname checkpoint-methods +#' @aliases checkpoint,RDD-method +#' @noRd +setMethod("checkpointRDD", signature(x = "RDD"), function(x) { jrdd <- getJRDD(x) @@ -296,45 +300,58 @@ setMethod("checkpoint", x }) -# Gets the number of partitions of an RDD -# -# @param x A RDD. -# @return the number of partitions of rdd as an integer. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10, 2L) -# numPartitions(rdd) # 2L -#} -# @rdname numPartitions -# @aliases numPartitions,RDD-method +#' Gets the number of partitions of an RDD +#' +#' @param x A RDD. +#' @return the number of partitions of rdd as an integer. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' getNumPartitions(rdd) # 2L +#'} +#' @rdname getNumPartitions +#' @aliases getNumPartitions,RDD-method +#' @noRd +setMethod("getNumPartitionsRDD", + signature(x = "RDD"), + function(x) { + callJMethod(getJRDD(x), "getNumPartitions") + }) + +#' Gets the number of partitions of an RDD, the same as getNumPartitions. +#' But this function has been deprecated, please use getNumPartitions. +#' +#' @rdname getNumPartitions +#' @aliases numPartitions,RDD-method +#' @noRd setMethod("numPartitions", signature(x = "RDD"), function(x) { - jrdd <- getJRDD(x) - partitions <- callJMethod(jrdd, "partitions") - callJMethod(partitions, "size") + .Deprecated("getNumPartitions") + getNumPartitionsRDD(x) }) -# Collect elements of an RDD -# -# @description -# \code{collect} returns a list that contains all of the elements in this RDD. -# -# @param x The RDD to collect -# @param ... Other optional arguments to collect -# @param flatten FALSE if the list should not flattened -# @return a list containing elements in the RDD -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10, 2L) -# collect(rdd) # list from 1 to 10 -# collectPartition(rdd, 0L) # list from 1 to 5 -#} -# @rdname collect-methods -# @aliases collect,RDD-method -setMethod("collect", +#' Collect elements of an RDD +#' +#' @description +#' \code{collect} returns a list that contains all of the elements in this RDD. +#' +#' @param x The RDD to collect +#' @param ... Other optional arguments to collect +#' @param flatten FALSE if the list should not flattened +#' @return a list containing elements in the RDD +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' collectRDD(rdd) # list from 1 to 10 +#' collectPartition(rdd, 0L) # list from 1 to 5 +#'} +#' @rdname collect-methods +#' @aliases collect,RDD-method +#' @noRd +setMethod("collectRDD", signature(x = "RDD"), function(x, flatten = TRUE) { # Assumes a pairwise RDD is backed by a JavaPairRDD. @@ -344,12 +361,13 @@ setMethod("collect", }) -# @description -# \code{collectPartition} returns a list that contains all of the elements -# in the specified partition of the RDD. -# @param partitionId the partition to collect (starts from 0) -# @rdname collect-methods -# @aliases collectPartition,integer,RDD-method +#' @description +#' \code{collectPartition} returns a list that contains all of the elements +#' in the specified partition of the RDD. +#' @param partitionId the partition to collect (starts from 0) +#' @rdname collect-methods +#' @aliases collectPartition,integer,RDD-method +#' @noRd setMethod("collectPartition", signature(x = "RDD", partitionId = "integer"), function(x, partitionId) { @@ -362,99 +380,107 @@ setMethod("collectPartition", serializedMode = getSerializedMode(x)) }) -# @description -# \code{collectAsMap} returns a named list as a map that contains all of the elements -# in a key-value pair RDD. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) -# collectAsMap(rdd) # list(`1` = 2, `3` = 4) -#} -# @rdname collect-methods -# @aliases collectAsMap,RDD-method +#' @description +#' \code{collectAsMap} returns a named list as a map that contains all of the elements +#' in a key-value pair RDD. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) +#' collectAsMap(rdd) # list(`1` = 2, `3` = 4) +#'} +# nolint end +#' @rdname collect-methods +#' @aliases collectAsMap,RDD-method +#' @noRd setMethod("collectAsMap", signature(x = "RDD"), function(x) { - pairList <- collect(x) + pairList <- collectRDD(x) map <- new.env() lapply(pairList, function(i) { assign(as.character(i[[1]]), i[[2]], envir = map) }) as.list(map) }) -# Return the number of elements in the RDD. -# -# @param x The RDD to count -# @return number of elements in the RDD. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# count(rdd) # 10 -# length(rdd) # Same as count -#} -# @rdname count -# @aliases count,RDD-method -setMethod("count", +#' Return the number of elements in the RDD. +#' +#' @param x The RDD to count +#' @return number of elements in the RDD. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' countRDD(rdd) # 10 +#' length(rdd) # Same as count +#'} +#' @rdname count +#' @aliases count,RDD-method +#' @noRd +setMethod("countRDD", signature(x = "RDD"), function(x) { countPartition <- function(part) { as.integer(length(part)) } valsRDD <- lapplyPartition(x, countPartition) - vals <- collect(valsRDD) + vals <- collectRDD(valsRDD) sum(as.integer(vals)) }) -# Return the number of elements in the RDD -# @export -# @rdname count -setMethod("length", +#' Return the number of elements in the RDD +#' @rdname count +#' @noRd +setMethod("lengthRDD", signature(x = "RDD"), function(x) { - count(x) + countRDD(x) }) -# Return the count of each unique value in this RDD as a list of -# (value, count) pairs. -# -# Same as countByValue in Spark. -# -# @param x The RDD to count -# @return list of (value, count) pairs, where count is number of each unique -# value in rdd. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, c(1,2,3,2,1)) -# countByValue(rdd) # (1,2L), (2,2L), (3,1L) -#} -# @rdname countByValue -# @aliases countByValue,RDD-method +#' Return the count of each unique value in this RDD as a list of +#' (value, count) pairs. +#' +#' Same as countByValue in Spark. +#' +#' @param x The RDD to count +#' @return list of (value, count) pairs, where count is number of each unique +#' value in rdd. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, c(1,2,3,2,1)) +#' countByValue(rdd) # (1,2L), (2,2L), (3,1L) +#'} +# nolint end +#' @rdname countByValue +#' @aliases countByValue,RDD-method +#' @noRd setMethod("countByValue", signature(x = "RDD"), function(x) { ones <- lapply(x, function(item) { list(item, 1L) }) - collect(reduceByKey(ones, `+`, numPartitions(x))) + collectRDD(reduceByKey(ones, `+`, getNumPartitionsRDD(x))) }) -# Apply a function to all elements -# -# This function creates a new RDD by applying the given transformation to all -# elements of the given RDD -# -# @param X The RDD to apply the transformation. -# @param FUN the transformation to apply on each element -# @return a new RDD created by the transformation. -# @rdname lapply -# @aliases lapply -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# multiplyByTwo <- lapply(rdd, function(x) { x * 2 }) -# collect(multiplyByTwo) # 2,4,6... -#} +#' Apply a function to all elements +#' +#' This function creates a new RDD by applying the given transformation to all +#' elements of the given RDD +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each element +#' @return a new RDD created by the transformation. +#' @rdname lapply +#' @noRd +#' @aliases lapply +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' multiplyByTwo <- lapply(rdd, function(x) { x * 2 }) +#' collectRDD(multiplyByTwo) # 2,4,6... +#'} setMethod("lapply", signature(X = "RDD", FUN = "function"), function(X, FUN) { @@ -464,31 +490,33 @@ setMethod("lapply", lapplyPartitionsWithIndex(X, func) }) -# @rdname lapply -# @aliases map,RDD,function-method +#' @rdname lapply +#' @aliases map,RDD,function-method +#' @noRd setMethod("map", signature(X = "RDD", FUN = "function"), function(X, FUN) { lapply(X, FUN) }) -# Flatten results after apply a function to all elements -# -# This function return a new RDD by first applying a function to all -# elements of this RDD, and then flattening the results. -# -# @param X The RDD to apply the transformation. -# @param FUN the transformation to apply on each element -# @return a new RDD created by the transformation. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# multiplyByTwo <- flatMap(rdd, function(x) { list(x*2, x*10) }) -# collect(multiplyByTwo) # 2,20,4,40,6,60... -#} -# @rdname flatMap -# @aliases flatMap,RDD,function-method +#' Flatten results after applying a function to all elements +#' +#' This function returns a new RDD by first applying a function to all +#' elements of this RDD, and then flattening the results. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each element +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' multiplyByTwo <- flatMap(rdd, function(x) { list(x*2, x*10) }) +#' collectRDD(multiplyByTwo) # 2,20,4,40,6,60... +#'} +#' @rdname flatMap +#' @aliases flatMap,RDD,function-method +#' @noRd setMethod("flatMap", signature(X = "RDD", FUN = "function"), function(X, FUN) { @@ -501,83 +529,90 @@ setMethod("flatMap", lapplyPartition(X, partitionFunc) }) -# Apply a function to each partition of an RDD -# -# Return a new RDD by applying a function to each partition of this RDD. -# -# @param X The RDD to apply the transformation. -# @param FUN the transformation to apply on each partition. -# @return a new RDD created by the transformation. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) -# collect(partitionSum) # 15, 40 -#} -# @rdname lapplyPartition -# @aliases lapplyPartition,RDD,function-method +#' Apply a function to each partition of an RDD +#' +#' Return a new RDD by applying a function to each partition of this RDD. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each partition. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) +#' collectRDD(partitionSum) # 15, 40 +#'} +#' @rdname lapplyPartition +#' @aliases lapplyPartition,RDD,function-method +#' @noRd setMethod("lapplyPartition", signature(X = "RDD", FUN = "function"), function(X, FUN) { lapplyPartitionsWithIndex(X, function(s, part) { FUN(part) }) }) -# mapPartitions is the same as lapplyPartition. -# -# @rdname lapplyPartition -# @aliases mapPartitions,RDD,function-method +#' mapPartitions is the same as lapplyPartition. +#' +#' @rdname lapplyPartition +#' @aliases mapPartitions,RDD,function-method +#' @noRd setMethod("mapPartitions", signature(X = "RDD", FUN = "function"), function(X, FUN) { lapplyPartition(X, FUN) }) -# Return a new RDD by applying a function to each partition of this RDD, while -# tracking the index of the original partition. -# -# @param X The RDD to apply the transformation. -# @param FUN the transformation to apply on each partition; takes the partition -# index and a list of elements in the particular partition. -# @return a new RDD created by the transformation. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10, 5L) -# prod <- lapplyPartitionsWithIndex(rdd, function(partIndex, part) { -# partIndex * Reduce("+", part) }) -# collect(prod, flatten = FALSE) # 0, 7, 22, 45, 76 -#} -# @rdname lapplyPartitionsWithIndex -# @aliases lapplyPartitionsWithIndex,RDD,function-method +#' Return a new RDD by applying a function to each partition of this RDD, while +#' tracking the index of the original partition. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each partition; takes the partition +#' index and a list of elements in the particular partition. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 5L) +#' prod <- lapplyPartitionsWithIndex(rdd, function(partIndex, part) { +#' partIndex * Reduce("+", part) }) +#' collectRDD(prod, flatten = FALSE) # 0, 7, 22, 45, 76 +#'} +#' @rdname lapplyPartitionsWithIndex +#' @aliases lapplyPartitionsWithIndex,RDD,function-method +#' @noRd setMethod("lapplyPartitionsWithIndex", signature(X = "RDD", FUN = "function"), function(X, FUN) { PipelinedRDD(X, FUN) }) -# @rdname lapplyPartitionsWithIndex -# @aliases mapPartitionsWithIndex,RDD,function-method +#' @rdname lapplyPartitionsWithIndex +#' @aliases mapPartitionsWithIndex,RDD,function-method +#' @noRd setMethod("mapPartitionsWithIndex", signature(X = "RDD", FUN = "function"), function(X, FUN) { lapplyPartitionsWithIndex(X, FUN) }) -# This function returns a new RDD containing only the elements that satisfy -# a predicate (i.e. returning TRUE in a given logical function). -# The same as `filter()' in Spark. -# -# @param x The RDD to be filtered. -# @param f A unary predicate function. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) -#} -# @rdname filterRDD -# @aliases filterRDD,RDD,function-method +#' This function returns a new RDD containing only the elements that satisfy +#' a predicate (i.e. returning TRUE in a given logical function). +#' The same as `filter()' in Spark. +#' +#' @param x The RDD to be filtered. +#' @param f A unary predicate function. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' unlist(collectRDD(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) +#'} +# nolint end +#' @rdname filterRDD +#' @aliases filterRDD,RDD,function-method +#' @noRd setMethod("filterRDD", signature(x = "RDD", f = "function"), function(x, f) { @@ -587,30 +622,32 @@ setMethod("filterRDD", lapplyPartition(x, filter.func) }) -# @rdname filterRDD -# @aliases Filter +#' @rdname filterRDD +#' @aliases Filter +#' @noRd setMethod("Filter", signature(f = "function", x = "RDD"), function(f, x) { filterRDD(x, f) }) -# Reduce across elements of an RDD. -# -# This function reduces the elements of this RDD using the -# specified commutative and associative binary operator. -# -# @param x The RDD to reduce -# @param func Commutative and associative function to apply on elements -# of the RDD. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# reduce(rdd, "+") # 55 -#} -# @rdname reduce -# @aliases reduce,RDD,ANY-method +#' Reduce across elements of an RDD. +#' +#' This function reduces the elements of this RDD using the +#' specified commutative and associative binary operator. +#' +#' @param x The RDD to reduce +#' @param func Commutative and associative function to apply on elements +#' of the RDD. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' reduce(rdd, "+") # 55 +#'} +#' @rdname reduce +#' @aliases reduce,RDD,ANY-method +#' @noRd setMethod("reduce", signature(x = "RDD", func = "ANY"), function(x, func) { @@ -619,75 +656,79 @@ setMethod("reduce", Reduce(func, part) } - partitionList <- collect(lapplyPartition(x, reducePartition), + partitionList <- collectRDD(lapplyPartition(x, reducePartition), flatten = FALSE) Reduce(func, partitionList) }) -# Get the maximum element of an RDD. -# -# @param x The RDD to get the maximum element from -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# maximum(rdd) # 10 -#} -# @rdname maximum -# @aliases maximum,RDD +#' Get the maximum element of an RDD. +#' +#' @param x The RDD to get the maximum element from +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' maximum(rdd) # 10 +#'} +#' @rdname maximum +#' @aliases maximum,RDD +#' @noRd setMethod("maximum", signature(x = "RDD"), function(x) { reduce(x, max) }) -# Get the minimum element of an RDD. -# -# @param x The RDD to get the minimum element from -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# minimum(rdd) # 1 -#} -# @rdname minimum -# @aliases minimum,RDD +#' Get the minimum element of an RDD. +#' +#' @param x The RDD to get the minimum element from +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' minimum(rdd) # 1 +#'} +#' @rdname minimum +#' @aliases minimum,RDD +#' @noRd setMethod("minimum", signature(x = "RDD"), function(x) { reduce(x, min) }) -# Add up the elements in an RDD. -# -# @param x The RDD to add up the elements in -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# sumRDD(rdd) # 55 -#} -# @rdname sumRDD -# @aliases sumRDD,RDD +#' Add up the elements in an RDD. +#' +#' @param x The RDD to add up the elements in +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' sumRDD(rdd) # 55 +#'} +#' @rdname sumRDD +#' @aliases sumRDD,RDD +#' @noRd setMethod("sumRDD", signature(x = "RDD"), function(x) { reduce(x, "+") }) -# Applies a function to all elements in an RDD, and force evaluation. -# -# @param x The RDD to apply the function -# @param func The function to be applied. -# @return invisible NULL. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# foreach(rdd, function(x) { save(x, file=...) }) -#} -# @rdname foreach -# @aliases foreach,RDD,function-method +#' Applies a function to all elements in an RDD, and forces evaluation. +#' +#' @param x The RDD to apply the function +#' @param func The function to be applied. +#' @return invisible NULL. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' foreach(rdd, function(x) { save(x, file=...) }) +#'} +#' @rdname foreach +#' @aliases foreach,RDD,function-method +#' @noRd setMethod("foreach", signature(x = "RDD", func = "function"), function(x, func) { @@ -695,47 +736,51 @@ setMethod("foreach", lapply(x, func) NULL } - invisible(collect(mapPartitions(x, partition.func))) + invisible(collectRDD(mapPartitions(x, partition.func))) }) -# Applies a function to each partition in an RDD, and force evaluation. -# -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# foreachPartition(rdd, function(part) { save(part, file=...); NULL }) -#} -# @rdname foreach -# @aliases foreachPartition,RDD,function-method +#' Applies a function to each partition in an RDD, and forces evaluation. +#' +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' foreachPartition(rdd, function(part) { save(part, file=...); NULL }) +#'} +#' @rdname foreach +#' @aliases foreachPartition,RDD,function-method +#' @noRd setMethod("foreachPartition", signature(x = "RDD", func = "function"), function(x, func) { - invisible(collect(mapPartitions(x, func))) + invisible(collectRDD(mapPartitions(x, func))) }) -# Take elements from an RDD. -# -# This function takes the first NUM elements in the RDD and -# returns them in a list. -# -# @param x The RDD to take elements from -# @param num Number of elements to take -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# take(rdd, 2L) # list(1, 2) -#} -# @rdname take -# @aliases take,RDD,numeric-method -setMethod("take", +#' Take elements from an RDD. +#' +#' This function takes the first NUM elements in the RDD and +#' returns them in a list. +#' +#' @param x The RDD to take elements from +#' @param num Number of elements to take +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' takeRDD(rdd, 2L) # list(1, 2) +#'} +# nolint end +#' @rdname take +#' @aliases take,RDD,numeric-method +#' @noRd +setMethod("takeRDD", signature(x = "RDD", num = "numeric"), function(x, num) { resList <- list() index <- -1 jrdd <- getJRDD(x) - numPartitions <- numPartitions(x) + numPartitions <- getNumPartitionsRDD(x) serializedModeRDD <- getSerializedMode(x) # TODO(shivaram): Collect more than one partition based on size @@ -763,42 +808,45 @@ setMethod("take", }) -# First -# -# Return the first element of an RDD -# -# @rdname first -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# first(rdd) -# } -setMethod("first", +#' First +#' +#' Return the first element of an RDD +#' +#' @rdname first +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' firstRDD(rdd) +#' } +#' @noRd +setMethod("firstRDD", signature(x = "RDD"), function(x) { - take(x, 1)[[1]] + takeRDD(x, 1)[[1]] }) -# Removes the duplicates from RDD. -# -# This function returns a new RDD containing the distinct elements in the -# given RDD. The same as `distinct()' in Spark. -# -# @param x The RDD to remove duplicates from. -# @param numPartitions Number of partitions to create. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, c(1,2,2,3,3,3)) -# sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) -#} -# @rdname distinct -# @aliases distinct,RDD-method -setMethod("distinct", +#' Removes the duplicates from RDD. +#' +#' This function returns a new RDD containing the distinct elements in the +#' given RDD. The same as `distinct()' in Spark. +#' +#' @param x The RDD to remove duplicates from. +#' @param numPartitions Number of partitions to create. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, c(1,2,2,3,3,3)) +#' sort(unlist(collectRDD(distinctRDD(rdd)))) # c(1, 2, 3) +#'} +# nolint end +#' @rdname distinct +#' @aliases distinct,RDD-method +#' @noRd +setMethod("distinctRDD", signature(x = "RDD"), - function(x, numPartitions = SparkR:::numPartitions(x)) { + function(x, numPartitions = SparkR:::getNumPartitionsRDD(x)) { identical.mapped <- lapply(x, function(x) { list(x, NULL) }) reduced <- reduceByKey(identical.mapped, function(x, y) { x }, @@ -807,24 +855,25 @@ setMethod("distinct", resRDD }) -# Return an RDD that is a sampled subset of the given RDD. -# -# The same as `sample()' in Spark. (We rename it due to signature -# inconsistencies with the `sample()' function in R's base package.) -# -# @param x The RDD to sample elements from -# @param withReplacement Sampling with replacement or not -# @param fraction The (rough) sample target fraction -# @param seed Randomness seed value -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# collect(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements -# collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates -#} -# @rdname sampleRDD -# @aliases sampleRDD,RDD +#' Return an RDD that is a sampled subset of the given RDD. +#' +#' The same as `sample()' in Spark. (We rename it due to signature +#' inconsistencies with the `sample()' function in R's base package.) +#' +#' @param x The RDD to sample elements from +#' @param withReplacement Sampling with replacement or not +#' @param fraction The (rough) sample target fraction +#' @param seed Randomness seed value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' collectRDD(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements +#' collectRDD(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates +#'} +#' @rdname sampleRDD +#' @aliases sampleRDD,RDD +#' @noRd setMethod("sampleRDD", signature(x = "RDD", withReplacement = "logical", fraction = "numeric", seed = "integer"), @@ -838,17 +887,17 @@ setMethod("sampleRDD", # Discards some random values to ensure each partition has a # different random seed. - runif(partIndex) + stats::runif(partIndex) for (elem in part) { if (withReplacement) { - count <- rpois(1, fraction) + count <- stats::rpois(1, fraction) if (count > 0) { res[ (len + 1) : (len + count) ] <- rep(list(elem), count) len <- len + count } } else { - if (runif(1) < fraction) { + if (stats::runif(1) < fraction) { len <- len + 1 res[[len]] <- elem } @@ -868,23 +917,24 @@ setMethod("sampleRDD", lapplyPartitionsWithIndex(x, samplingFunc) }) -# Return a list of the elements that are a sampled subset of the given RDD. -# -# @param x The RDD to sample elements from -# @param withReplacement Sampling with replacement or not -# @param num Number of elements to return -# @param seed Randomness seed value -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:100) -# # exactly 5 elements sampled, which may not be distinct -# takeSample(rdd, TRUE, 5L, 1618L) -# # exactly 5 distinct elements sampled -# takeSample(rdd, FALSE, 5L, 16181618L) -#} -# @rdname takeSample -# @aliases takeSample,RDD +#' Return a list of the elements that are a sampled subset of the given RDD. +#' +#' @param x The RDD to sample elements from +#' @param withReplacement Sampling with replacement or not +#' @param num Number of elements to return +#' @param seed Randomness seed value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:100) +#' # exactly 5 elements sampled, which may not be distinct +#' takeSample(rdd, TRUE, 5L, 1618L) +#' # exactly 5 distinct elements sampled +#' takeSample(rdd, FALSE, 5L, 16181618L) +#'} +#' @rdname takeSample +#' @aliases takeSample,RDD +#' @noRd setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", num = "integer", seed = "integer"), function(x, withReplacement, num, seed) { @@ -892,7 +942,7 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", fraction <- 0.0 total <- 0 multiplier <- 3.0 - initialCount <- count(x) + initialCount <- countRDD(x) maxSelected <- 0 MAXINT <- .Machine$integer.max @@ -914,16 +964,16 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", } set.seed(seed) - samples <- collect(sampleRDD(x, withReplacement, fraction, - as.integer(ceiling(runif(1, + samples <- collectRDD(sampleRDD(x, withReplacement, fraction, + as.integer(ceiling(stats::runif(1, -MAXINT, MAXINT))))) # If the first sample didn't turn out large enough, keep trying to # take samples; this shouldn't happen often because we use a big # multiplier for thei initial size while (length(samples) < total) - samples <- collect(sampleRDD(x, withReplacement, fraction, - as.integer(ceiling(runif(1, + samples <- collectRDD(sampleRDD(x, withReplacement, fraction, + as.integer(ceiling(stats::runif(1, -MAXINT, MAXINT))))) @@ -931,18 +981,21 @@ setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", base::sample(samples)[1:total] }) -# Creates tuples of the elements in this RDD by applying a function. -# -# @param x The RDD. -# @param func The function to be applied. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(1, 2, 3)) -# collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) -#} -# @rdname keyBy -# @aliases keyBy,RDD +#' Creates tuples of the elements in this RDD by applying a function. +#' +#' @param x The RDD. +#' @param func The function to be applied. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3)) +#' collectRDD(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) +#'} +# nolint end +#' @rdname keyBy +#' @aliases keyBy,RDD +#' @noRd setMethod("keyBy", signature(x = "RDD", func = "function"), function(x, func) { @@ -952,49 +1005,55 @@ setMethod("keyBy", lapply(x, apply.func) }) -# Return a new RDD that has exactly numPartitions partitions. -# Can increase or decrease the level of parallelism in this RDD. Internally, -# this uses a shuffle to redistribute data. -# If you are decreasing the number of partitions in this RDD, consider using -# coalesce, which can avoid performing a shuffle. -# -# @param x The RDD. -# @param numPartitions Number of partitions to create. -# @seealso coalesce -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) -# numPartitions(rdd) # 4 -# numPartitions(repartition(rdd, 2L)) # 2 -#} -# @rdname repartition -# @aliases repartition,RDD -setMethod("repartition", - signature(x = "RDD", numPartitions = "numeric"), +#' Return a new RDD that has exactly numPartitions partitions. +#' Can increase or decrease the level of parallelism in this RDD. Internally, +#' this uses a shuffle to redistribute data. +#' If you are decreasing the number of partitions in this RDD, consider using +#' coalesce, which can avoid performing a shuffle. +#' +#' @param x The RDD. +#' @param numPartitions Number of partitions to create. +#' @seealso coalesce +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) +#' getNumPartitions(rdd) # 4 +#' getNumPartitions(repartitionRDD(rdd, 2L)) # 2 +#'} +#' @rdname repartition +#' @aliases repartition,RDD +#' @noRd +setMethod("repartitionRDD", + signature(x = "RDD"), function(x, numPartitions) { - coalesce(x, numPartitions, TRUE) + if (!is.null(numPartitions) && is.numeric(numPartitions)) { + coalesceRDD(x, numPartitions, TRUE) + } else { + stop("Please, specify the number of partitions") + } }) -# Return a new RDD that is reduced into numPartitions partitions. -# -# @param x The RDD. -# @param numPartitions Number of partitions to create. -# @seealso repartition -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) -# numPartitions(rdd) # 3 -# numPartitions(coalesce(rdd, 1L)) # 1 -#} -# @rdname coalesce -# @aliases coalesce,RDD -setMethod("coalesce", +#' Return a new RDD that is reduced into numPartitions partitions. +#' +#' @param x The RDD. +#' @param numPartitions Number of partitions to create. +#' @seealso repartition +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) +#' getNumPartitions(rdd) # 3 +#' getNumPartitions(coalesce(rdd, 1L)) # 1 +#'} +#' @rdname coalesce +#' @aliases coalesce,RDD +#' @noRd +setMethod("coalesceRDD", signature(x = "RDD", numPartitions = "numeric"), function(x, numPartitions, shuffle = FALSE) { numPartitions <- numToInt(numPartitions) - if (shuffle || numPartitions > SparkR:::numPartitions(x)) { + if (shuffle || numPartitions > SparkR:::getNumPartitionsRDD(x)) { func <- function(partIndex, part) { set.seed(partIndex) # partIndex as seed start <- as.integer(base::sample(numPartitions, 1) - 1) @@ -1005,7 +1064,7 @@ setMethod("coalesce", }) } shuffled <- lapplyPartitionsWithIndex(x, func) - repartitioned <- partitionBy(shuffled, numPartitions) + repartitioned <- partitionByRDD(shuffled, numPartitions) values(repartitioned) } else { jrdd <- callJMethod(getJRDD(x), "coalesce", numPartitions, shuffle) @@ -1013,19 +1072,20 @@ setMethod("coalesce", } }) -# Save this RDD as a SequenceFile of serialized objects. -# -# @param x The RDD to save -# @param path The directory where the file is saved -# @seealso objectFile -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:3) -# saveAsObjectFile(rdd, "/tmp/sparkR-tmp") -#} -# @rdname saveAsObjectFile -# @aliases saveAsObjectFile,RDD +#' Save this RDD as a SequenceFile of serialized objects. +#' +#' @param x The RDD to save +#' @param path The directory where the file is saved +#' @seealso objectFile +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' saveAsObjectFile(rdd, "/tmp/sparkR-tmp") +#'} +#' @rdname saveAsObjectFile +#' @aliases saveAsObjectFile,RDD +#' @noRd setMethod("saveAsObjectFile", signature(x = "RDD", path = "character"), function(x, path) { @@ -1038,18 +1098,19 @@ setMethod("saveAsObjectFile", invisible(callJMethod(getJRDD(x), "saveAsObjectFile", path)) }) -# Save this RDD as a text file, using string representations of elements. -# -# @param x The RDD to save -# @param path The directory where the partitions of the text file are saved -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:3) -# saveAsTextFile(rdd, "/tmp/sparkR-tmp") -#} -# @rdname saveAsTextFile -# @aliases saveAsTextFile,RDD +#' Save this RDD as a text file, using string representations of elements. +#' +#' @param x The RDD to save +#' @param path The directory where the partitions of the text file are saved +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' saveAsTextFile(rdd, "/tmp/sparkR-tmp") +#'} +#' @rdname saveAsTextFile +#' @aliases saveAsTextFile,RDD +#' @noRd setMethod("saveAsTextFile", signature(x = "RDD", path = "character"), function(x, path) { @@ -1062,24 +1123,27 @@ setMethod("saveAsTextFile", callJMethod(getJRDD(stringRdd, serializedMode = "string"), "saveAsTextFile", path)) }) -# Sort an RDD by the given key function. -# -# @param x An RDD to be sorted. -# @param func A function used to compute the sort key for each element. -# @param ascending A flag to indicate whether the sorting is ascending or descending. -# @param numPartitions Number of partitions to create. -# @return An RDD where all elements are sorted. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(3, 2, 1)) -# collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) -#} -# @rdname sortBy -# @aliases sortBy,RDD,RDD-method +#' Sort an RDD by the given key function. +#' +#' @param x An RDD to be sorted. +#' @param func A function used to compute the sort key for each element. +#' @param ascending A flag to indicate whether the sorting is ascending or descending. +#' @param numPartitions Number of partitions to create. +#' @return An RDD where all elements are sorted. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(3, 2, 1)) +#' collectRDD(sortBy(rdd, function(x) { x })) # list (1, 2, 3) +#'} +# nolint end +#' @rdname sortBy +#' @aliases sortBy,RDD,RDD-method +#' @noRd setMethod("sortBy", signature(x = "RDD", func = "function"), - function(x, func, ascending = TRUE, numPartitions = SparkR:::numPartitions(x)) { + function(x, func, ascending = TRUE, numPartitions = SparkR:::getNumPartitionsRDD(x)) { values(sortByKey(keyBy(x, func), ascending, numPartitions)) }) @@ -1111,7 +1175,7 @@ takeOrderedElem <- function(x, num, ascending = TRUE) { resList <- list() index <- -1 jrdd <- getJRDD(newRdd) - numPartitions <- numPartitions(newRdd) + numPartitions <- getNumPartitionsRDD(newRdd) serializedModeRDD <- getSerializedMode(newRdd) while (TRUE) { @@ -1138,97 +1202,101 @@ takeOrderedElem <- function(x, num, ascending = TRUE) { resList } -# Returns the first N elements from an RDD in ascending order. -# -# @param x An RDD. -# @param num Number of elements to return. -# @return The first N elements from the RDD in ascending order. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) -# takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) -#} -# @rdname takeOrdered -# @aliases takeOrdered,RDD,RDD-method +#' Returns the first N elements from an RDD in ascending order. +#' +#' @param x An RDD. +#' @param num Number of elements to return. +#' @return The first N elements from the RDD in ascending order. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +#' takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) +#'} +# nolint end +#' @rdname takeOrdered +#' @aliases takeOrdered,RDD,RDD-method +#' @noRd setMethod("takeOrdered", signature(x = "RDD", num = "integer"), function(x, num) { takeOrderedElem(x, num) }) -# Returns the top N elements from an RDD. -# -# @param x An RDD. -# @param num Number of elements to return. -# @return The top N elements from the RDD. -# @rdname top -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) -# top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) -#} -# @rdname top -# @aliases top,RDD,RDD-method +#' Returns the top N elements from an RDD. +#' +#' @param x An RDD. +#' @param num Number of elements to return. +#' @return The top N elements from the RDD. +#' @rdname top +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +#' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) +#'} +# nolint end +#' @aliases top,RDD,RDD-method +#' @noRd setMethod("top", signature(x = "RDD", num = "integer"), function(x, num) { takeOrderedElem(x, num, FALSE) }) -# Fold an RDD using a given associative function and a neutral "zero value". -# -# Aggregate the elements of each partition, and then the results for all the -# partitions, using a given associative function and a neutral "zero value". -# -# @param x An RDD. -# @param zeroValue A neutral "zero value". -# @param op An associative function for the folding operation. -# @return The folding result. -# @rdname fold -# @seealso reduce -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) -# fold(rdd, 0, "+") # 15 -#} -# @rdname fold -# @aliases fold,RDD,RDD-method +#' Fold an RDD using a given associative function and a neutral "zero value". +#' +#' Aggregate the elements of each partition, and then the results for all the +#' partitions, using a given associative function and a neutral "zero value". +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param op An associative function for the folding operation. +#' @return The folding result. +#' @rdname fold +#' @seealso reduce +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) +#' fold(rdd, 0, "+") # 15 +#'} +#' @aliases fold,RDD,RDD-method +#' @noRd setMethod("fold", signature(x = "RDD", zeroValue = "ANY", op = "ANY"), function(x, zeroValue, op) { aggregateRDD(x, zeroValue, op, op) }) -# Aggregate an RDD using the given combine functions and a neutral "zero value". -# -# Aggregate the elements of each partition, and then the results for all the -# partitions, using given combine functions and a neutral "zero value". -# -# @param x An RDD. -# @param zeroValue A neutral "zero value". -# @param seqOp A function to aggregate the RDD elements. It may return a different -# result type from the type of the RDD elements. -# @param combOp A function to aggregate results of seqOp. -# @return The aggregation result. -# @rdname aggregateRDD -# @seealso reduce -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(1, 2, 3, 4)) -# zeroValue <- list(0, 0) -# seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } -# combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } -# aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) -#} -# @rdname aggregateRDD -# @aliases aggregateRDD,RDD,RDD-method +#' Aggregate an RDD using the given combine functions and a neutral "zero value". +#' +#' Aggregate the elements of each partition, and then the results for all the +#' partitions, using given combine functions and a neutral "zero value". +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param seqOp A function to aggregate the RDD elements. It may return a different +#' result type from the type of the RDD elements. +#' @param combOp A function to aggregate results of seqOp. +#' @return The aggregation result. +#' @rdname aggregateRDD +#' @seealso reduce +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4)) +#' zeroValue <- list(0, 0) +#' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } +#' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } +#' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) +#'} +# nolint end +#' @aliases aggregateRDD,RDD,RDD-method +#' @noRd setMethod("aggregateRDD", signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY"), function(x, zeroValue, seqOp, combOp) { @@ -1236,30 +1304,29 @@ setMethod("aggregateRDD", Reduce(seqOp, part, zeroValue) } - partitionList <- collect(lapplyPartition(x, partitionFunc), + partitionList <- collectRDD(lapplyPartition(x, partitionFunc), flatten = FALSE) Reduce(combOp, partitionList, zeroValue) }) -# Pipes elements to a forked external process. -# -# The same as 'pipe()' in Spark. -# -# @param x The RDD whose elements are piped to the forked external process. -# @param command The command to fork an external process. -# @param env A named list to set environment variables of the external process. -# @return A new RDD created by piping all elements to a forked external process. -# @rdname pipeRDD -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# collect(pipeRDD(rdd, "more") -# Output: c("1", "2", ..., "10") -#} -# @rdname pipeRDD -# @aliases pipeRDD,RDD,character-method +#' Pipes elements to a forked external process. +#' +#' The same as 'pipe()' in Spark. +#' +#' @param x The RDD whose elements are piped to the forked external process. +#' @param command The command to fork an external process. +#' @param env A named list to set environment variables of the external process. +#' @return A new RDD created by piping all elements to a forked external process. +#' @rdname pipeRDD +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' pipeRDD(rdd, "more") +#' Output: c("1", "2", ..., "10") +#'} +#' @aliases pipeRDD,RDD,character-method +#' @noRd setMethod("pipeRDD", signature(x = "RDD", command = "character"), function(x, command, env = list()) { @@ -1274,42 +1341,40 @@ setMethod("pipeRDD", lapplyPartition(x, func) }) -# TODO: Consider caching the name in the RDD's environment -# Return an RDD's name. -# -# @param x The RDD whose name is returned. -# @rdname name -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(1,2,3)) -# name(rdd) # NULL (if not set before) -#} -# @rdname name -# @aliases name,RDD +#' TODO: Consider caching the name in the RDD's environment +#' Return an RDD's name. +#' +#' @param x The RDD whose name is returned. +#' @rdname name +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1,2,3)) +#' name(rdd) # NULL (if not set before) +#'} +#' @aliases name,RDD +#' @noRd setMethod("name", signature(x = "RDD"), function(x) { callJMethod(getJRDD(x), "name") }) -# Set an RDD's name. -# -# @param x The RDD whose name is to be set. -# @param name The RDD name to be set. -# @return a new RDD renamed. -# @rdname setName -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(1,2,3)) -# setName(rdd, "myRDD") -# name(rdd) # "myRDD" -#} -# @rdname setName -# @aliases setName,RDD +#' Set an RDD's name. +#' +#' @param x The RDD whose name is to be set. +#' @param name The RDD name to be set. +#' @return a new RDD renamed. +#' @rdname setName +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1,2,3)) +#' setName(rdd, "myRDD") +#' name(rdd) # "myRDD" +#'} +#' @aliases setName,RDD +#' @noRd setMethod("setName", signature(x = "RDD", name = "character"), function(x, name) { @@ -1317,29 +1382,32 @@ setMethod("setName", x }) -# Zip an RDD with generated unique Long IDs. -# -# Items in the kth partition will get ids k, n+k, 2*n+k, ..., where -# n is the number of partitions. So there may exist gaps, but this -# method won't trigger a spark job, which is different from -# zipWithIndex. -# -# @param x An RDD to be zipped. -# @return An RDD with zipped items. -# @seealso zipWithIndex -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) -# collect(zipWithUniqueId(rdd)) -# # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) -#} -# @rdname zipWithUniqueId -# @aliases zipWithUniqueId,RDD +#' Zip an RDD with generated unique Long IDs. +#' +#' Items in the kth partition will get ids k, n+k, 2*n+k, ..., where +#' n is the number of partitions. So there may exist gaps, but this +#' method won't trigger a spark job, which is different from +#' zipWithIndex. +#' +#' @param x An RDD to be zipped. +#' @return An RDD with zipped items. +#' @seealso zipWithIndex +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +#' collectRDD(zipWithUniqueId(rdd)) +#' # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) +#'} +# nolint end +#' @rdname zipWithUniqueId +#' @aliases zipWithUniqueId,RDD +#' @noRd setMethod("zipWithUniqueId", signature(x = "RDD"), function(x) { - n <- numPartitions(x) + n <- getNumPartitionsRDD(x) partitionFunc <- function(partIndex, part) { mapply( @@ -1354,34 +1422,37 @@ setMethod("zipWithUniqueId", lapplyPartitionsWithIndex(x, partitionFunc) }) -# Zip an RDD with its element indices. -# -# The ordering is first based on the partition index and then the -# ordering of items within each partition. So the first item in -# the first partition gets index 0, and the last item in the last -# partition receives the largest index. -# -# This method needs to trigger a Spark job when this RDD contains -# more than one partition. -# -# @param x An RDD to be zipped. -# @return An RDD with zipped items. -# @seealso zipWithUniqueId -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) -# collect(zipWithIndex(rdd)) -# # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) -#} -# @rdname zipWithIndex -# @aliases zipWithIndex,RDD +#' Zip an RDD with its element indices. +#' +#' The ordering is first based on the partition index and then the +#' ordering of items within each partition. So the first item in +#' the first partition gets index 0, and the last item in the last +#' partition receives the largest index. +#' +#' This method needs to trigger a Spark job when this RDD contains +#' more than one partition. +#' +#' @param x An RDD to be zipped. +#' @return An RDD with zipped items. +#' @seealso zipWithUniqueId +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +#' collectRDD(zipWithIndex(rdd)) +#' # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) +#'} +# nolint end +#' @rdname zipWithIndex +#' @aliases zipWithIndex,RDD +#' @noRd setMethod("zipWithIndex", signature(x = "RDD"), function(x) { - n <- numPartitions(x) + n <- getNumPartitionsRDD(x) if (n > 1) { - nums <- collect(lapplyPartition(x, + nums <- collectRDD(lapplyPartition(x, function(part) { list(length(part)) })) @@ -1407,20 +1478,23 @@ setMethod("zipWithIndex", lapplyPartitionsWithIndex(x, partitionFunc) }) -# Coalesce all elements within each partition of an RDD into a list. -# -# @param x An RDD. -# @return An RDD created by coalescing all elements within -# each partition into a list. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, as.list(1:4), 2L) -# collect(glom(rdd)) -# # list(list(1, 2), list(3, 4)) -#} -# @rdname glom -# @aliases glom,RDD +#' Coalesce all elements within each partition of an RDD into a list. +#' +#' @param x An RDD. +#' @return An RDD created by coalescing all elements within +#' each partition into a list. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, as.list(1:4), 2L) +#' collectRDD(glom(rdd)) +#' # list(list(1, 2), list(3, 4)) +#'} +# nolint end +#' @rdname glom +#' @aliases glom,RDD +#' @noRd setMethod("glom", signature(x = "RDD"), function(x) { @@ -1433,21 +1507,22 @@ setMethod("glom", ############ Binary Functions ############# -# Return the union RDD of two RDDs. -# The same as union() in Spark. -# -# @param x An RDD. -# @param y An RDD. -# @return a new RDD created by performing the simple union (witout removing -# duplicates) of two input RDDs. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:3) -# unionRDD(rdd, rdd) # 1, 2, 3, 1, 2, 3 -#} -# @rdname unionRDD -# @aliases unionRDD,RDD,RDD-method +#' Return the union RDD of two RDDs. +#' The same as union() in Spark. +#' +#' @param x An RDD. +#' @param y An RDD. +#' @return a new RDD created by performing the simple union (witout removing +#' duplicates) of two input RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' unionRDD(rdd, rdd) # 1, 2, 3, 1, 2, 3 +#'} +#' @rdname unionRDD +#' @aliases unionRDD,RDD,RDD-method +#' @noRd setMethod("unionRDD", signature(x = "RDD", y = "RDD"), function(x, y) { @@ -1464,32 +1539,35 @@ setMethod("unionRDD", union.rdd }) -# Zip an RDD with another RDD. -# -# Zips this RDD with another one, returning key-value pairs with the -# first element in each RDD second element in each RDD, etc. Assumes -# that the two RDDs have the same number of partitions and the same -# number of elements in each partition (e.g. one was made through -# a map on the other). -# -# @param x An RDD to be zipped. -# @param other Another RDD to be zipped. -# @return An RDD zipped from the two RDDs. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd1 <- parallelize(sc, 0:4) -# rdd2 <- parallelize(sc, 1000:1004) -# collect(zipRDD(rdd1, rdd2)) -# # list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004)) -#} -# @rdname zipRDD -# @aliases zipRDD,RDD +#' Zip an RDD with another RDD. +#' +#' Zips this RDD with another one, returning key-value pairs with the +#' first element in each RDD second element in each RDD, etc. Assumes +#' that the two RDDs have the same number of partitions and the same +#' number of elements in each partition (e.g. one was made through +#' a map on the other). +#' +#' @param x An RDD to be zipped. +#' @param other Another RDD to be zipped. +#' @return An RDD zipped from the two RDDs. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, 0:4) +#' rdd2 <- parallelize(sc, 1000:1004) +#' collectRDD(zipRDD(rdd1, rdd2)) +#' # list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004)) +#'} +# nolint end +#' @rdname zipRDD +#' @aliases zipRDD,RDD +#' @noRd setMethod("zipRDD", signature(x = "RDD", other = "RDD"), function(x, other) { - n1 <- numPartitions(x) - n2 <- numPartitions(other) + n1 <- getNumPartitionsRDD(x) + n2 <- getNumPartitionsRDD(other) if (n1 != n2) { stop("Can only zip RDDs which have the same number of partitions.") } @@ -1503,24 +1581,27 @@ setMethod("zipRDD", mergePartitions(rdd, TRUE) }) -# Cartesian product of this RDD and another one. -# -# Return the Cartesian product of this RDD and another one, -# that is, the RDD of all pairs of elements (a, b) where a -# is in this and b is in other. -# -# @param x An RDD. -# @param other An RDD. -# @return A new RDD which is the Cartesian product of these two RDDs. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:2) -# sortByKey(cartesian(rdd, rdd)) -# # list(list(1, 1), list(1, 2), list(2, 1), list(2, 2)) -#} -# @rdname cartesian -# @aliases cartesian,RDD,RDD-method +#' Cartesian product of this RDD and another one. +#' +#' Return the Cartesian product of this RDD and another one, +#' that is, the RDD of all pairs of elements (a, b) where a +#' is in this and b is in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @return A new RDD which is the Cartesian product of these two RDDs. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:2) +#' sortByKey(cartesian(rdd, rdd)) +#' # list(list(1, 1), list(1, 2), list(2, 1), list(2, 2)) +#'} +# nolint end +#' @rdname cartesian +#' @aliases cartesian,RDD,RDD-method +#' @noRd setMethod("cartesian", signature(x = "RDD", other = "RDD"), function(x, other) { @@ -1533,58 +1614,64 @@ setMethod("cartesian", mergePartitions(rdd, FALSE) }) -# Subtract an RDD with another RDD. -# -# Return an RDD with the elements from this that are not in other. -# -# @param x An RDD. -# @param other An RDD. -# @param numPartitions Number of the partitions in the result RDD. -# @return An RDD with the elements from this that are not in other. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd1 <- parallelize(sc, list(1, 1, 2, 2, 3, 4)) -# rdd2 <- parallelize(sc, list(2, 4)) -# collect(subtract(rdd1, rdd2)) -# # list(1, 1, 3) -#} -# @rdname subtract -# @aliases subtract,RDD +#' Subtract an RDD with another RDD. +#' +#' Return an RDD with the elements from this that are not in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions Number of the partitions in the result RDD. +#' @return An RDD with the elements from this that are not in other. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(1, 1, 2, 2, 3, 4)) +#' rdd2 <- parallelize(sc, list(2, 4)) +#' collectRDD(subtract(rdd1, rdd2)) +#' # list(1, 1, 3) +#'} +# nolint end +#' @rdname subtract +#' @aliases subtract,RDD +#' @noRd setMethod("subtract", signature(x = "RDD", other = "RDD"), - function(x, other, numPartitions = SparkR:::numPartitions(x)) { + function(x, other, numPartitions = SparkR:::getNumPartitionsRDD(x)) { mapFunction <- function(e) { list(e, NA) } rdd1 <- map(x, mapFunction) rdd2 <- map(other, mapFunction) keys(subtractByKey(rdd1, rdd2, numPartitions)) }) -# Intersection of this RDD and another one. -# -# Return the intersection of this RDD and another one. -# The output will not contain any duplicate elements, -# even if the input RDDs did. Performs a hash partition -# across the cluster. -# Note that this method performs a shuffle internally. -# -# @param x An RDD. -# @param other An RDD. -# @param numPartitions The number of partitions in the result RDD. -# @return An RDD which is the intersection of these two RDDs. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) -# rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) -# collect(sortBy(intersection(rdd1, rdd2), function(x) { x })) -# # list(1, 2, 3) -#} -# @rdname intersection -# @aliases intersection,RDD +#' Intersection of this RDD and another one. +#' +#' Return the intersection of this RDD and another one. +#' The output will not contain any duplicate elements, +#' even if the input RDDs did. Performs a hash partition +#' across the cluster. +#' Note that this method performs a shuffle internally. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions The number of partitions in the result RDD. +#' @return An RDD which is the intersection of these two RDDs. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) +#' rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) +#' collectRDD(sortBy(intersection(rdd1, rdd2), function(x) { x })) +#' # list(1, 2, 3) +#'} +# nolint end +#' @rdname intersection +#' @aliases intersection,RDD +#' @noRd setMethod("intersection", signature(x = "RDD", other = "RDD"), - function(x, other, numPartitions = SparkR:::numPartitions(x)) { + function(x, other, numPartitions = SparkR:::getNumPartitionsRDD(x)) { rdd1 <- map(x, function(v) { list(v, NA) }) rdd2 <- map(other, function(v) { list(v, NA) }) @@ -1597,26 +1684,29 @@ setMethod("intersection", keys(filterRDD(cogroup(rdd1, rdd2, numPartitions = numPartitions), filterFunction)) }) -# Zips an RDD's partitions with one (or more) RDD(s). -# Same as zipPartitions in Spark. -# -# @param ... RDDs to be zipped. -# @param func A function to transform zipped partitions. -# @return A new RDD by applying a function to the zipped partitions. -# Assumes that all the RDDs have the *same number of partitions*, but -# does *not* require them to have the same number of elements in each partition. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd1 <- parallelize(sc, 1:2, 2L) # 1, 2 -# rdd2 <- parallelize(sc, 1:4, 2L) # 1:2, 3:4 -# rdd3 <- parallelize(sc, 1:6, 2L) # 1:3, 4:6 -# collect(zipPartitions(rdd1, rdd2, rdd3, -# func = function(x, y, z) { list(list(x, y, z))} )) -# # list(list(1, c(1,2), c(1,2,3)), list(2, c(3,4), c(4,5,6))) -#} -# @rdname zipRDD -# @aliases zipPartitions,RDD +#' Zips an RDD's partitions with one (or more) RDD(s). +#' Same as zipPartitions in Spark. +#' +#' @param ... RDDs to be zipped. +#' @param func A function to transform zipped partitions. +#' @return A new RDD by applying a function to the zipped partitions. +#' Assumes that all the RDDs have the *same number of partitions*, but +#' does *not* require them to have the same number of elements in each partition. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, 1:2, 2L) # 1, 2 +#' rdd2 <- parallelize(sc, 1:4, 2L) # 1:2, 3:4 +#' rdd3 <- parallelize(sc, 1:6, 2L) # 1:3, 4:6 +#' collectRDD(zipPartitions(rdd1, rdd2, rdd3, +#' func = function(x, y, z) { list(list(x, y, z))} )) +#' # list(list(1, c(1,2), c(1,2,3)), list(2, c(3,4), c(4,5,6))) +#'} +# nolint end +#' @rdname zipRDD +#' @aliases zipPartitions,RDD +#' @noRd setMethod("zipPartitions", "RDD", function(..., func) { @@ -1624,7 +1714,7 @@ setMethod("zipPartitions", if (length(rrdds) == 1) { return(rrdds[[1]]) } - nPart <- sapply(rrdds, numPartitions) + nPart <- sapply(rrdds, getNumPartitionsRDD) if (length(unique(nPart)) != 1) { stop("Can only zipPartitions RDDs which have the same number of partitions.") } diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 1bf025cce437..3b7f71bbbffb 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -17,27 +17,75 @@ # SQLcontext.R: SQLContext-driven functions + +# Map top level R type to SQL type +getInternalType <- function(x) { + # class of POSIXlt is c("POSIXlt" "POSIXt") + switch(class(x)[[1]], + integer = "integer", + character = "string", + logical = "boolean", + double = "double", + numeric = "double", + raw = "binary", + list = "array", + struct = "struct", + environment = "map", + Date = "date", + POSIXlt = "timestamp", + POSIXct = "timestamp", + stop(paste("Unsupported type for SparkDataFrame:", class(x)))) +} + +#' Temporary function to reroute old S3 Method call to new +#' This function is specifically implemented to remove SQLContext from the parameter list. +#' It determines the target to route the call by checking the parent of this callsite (say 'func'). +#' The target should be called 'func.default'. +#' We need to check the class of x to ensure it is SQLContext/HiveContext before dispatching. +#' @param newFuncSig name of the function the user should call instead in the deprecation message +#' @param x the first parameter of the original call +#' @param ... the rest of parameter to pass along +#' @return whatever the target returns +#' @noRd +dispatchFunc <- function(newFuncSig, x, ...) { + # When called with SparkR::createDataFrame, sys.call()[[1]] returns c(::, SparkR, createDataFrame) + callsite <- as.character(sys.call(sys.parent())[[1]]) + funcName <- callsite[[length(callsite)]] + f <- get(paste0(funcName, ".default")) + # Strip sqlContext from list of parameters and then pass the rest along. + contextNames <- c("org.apache.spark.sql.SQLContext", + "org.apache.spark.sql.hive.HiveContext", + "org.apache.spark.sql.hive.test.TestHiveContext", + "org.apache.spark.sql.SparkSession") + if (missing(x) && length(list(...)) == 0) { + f() + } else if (class(x) == "jobj" && + any(grepl(paste(contextNames, collapse = "|"), getClassName.jobj(x)))) { + .Deprecated(newFuncSig, old = paste0(funcName, "(sqlContext...)")) + f(...) + } else { + f(x, ...) + } +} + +#' return the SparkSession +#' @noRd +getSparkSession <- function() { + if (exists(".sparkRsession", envir = .sparkREnv)) { + get(".sparkRsession", envir = .sparkREnv) + } else { + stop("SparkSession not initialized") + } +} + #' infer the SQL type +#' @noRd infer_type <- function(x) { if (is.null(x)) { stop("can not infer type from NULL") } - # class of POSIXlt is c("POSIXlt" "POSIXt") - type <- switch(class(x)[[1]], - integer = "integer", - character = "string", - logical = "boolean", - double = "double", - numeric = "double", - raw = "binary", - list = "array", - struct = "struct", - environment = "map", - Date = "date", - POSIXlt = "timestamp", - POSIXct = "timestamp", - stop(paste("Unsupported type for DataFrame:", class(x)))) + type <- getInternalType(x) if (type == "map") { stopifnot(length(x) > 0) @@ -57,56 +105,144 @@ infer_type <- function(x) { }) type <- Reduce(paste0, type) type <- paste0("struct<", substr(type, 1, nchar(type) - 1), ">") - } else if (length(x) > 1) { + } else if (length(x) > 1 && type != "binary") { paste0("array<", infer_type(x[[1]]), ">") } else { type } } -#' Create a DataFrame +#' Get Runtime Config from the current active SparkSession #' -#' Converts R data.frame or list into DataFrame. +#' Get Runtime Config from the current active SparkSession. +#' To change SparkSession Runtime Config, please see \code{sparkR.session()}. #' -#' @param sqlContext A SQLContext -#' @param data An RDD or list or data.frame -#' @param schema a list of column names or named list (StructType), optional -#' @return an DataFrame -#' @rdname createDataFrame +#' @param key (optional) The key of the config to get, if omitted, all config is returned +#' @param defaultValue (optional) The default value of the config to return if they config is not +#' set, if omitted, the call fails if the config key is not set +#' @return a list of config values with keys as their names +#' @rdname sparkR.conf +#' @name sparkR.conf #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' df1 <- as.DataFrame(sqlContext, iris) -#' df2 <- as.DataFrame(sqlContext, list(3,4,5,6)) -#' df3 <- createDataFrame(sqlContext, iris) +#' sparkR.session() +#' allConfigs <- sparkR.conf() +#' masterValue <- unlist(sparkR.conf("spark.master")) +#' namedConfig <- sparkR.conf("spark.executor.memory", "0g") #' } +#' @note sparkR.conf since 2.0.0 +sparkR.conf <- function(key, defaultValue) { + sparkSession <- getSparkSession() + if (missing(key)) { + m <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getSessionConf", sparkSession) + as.list(m, all.names = TRUE, sorted = TRUE) + } else { + conf <- callJMethod(sparkSession, "conf") + value <- if (missing(defaultValue)) { + tryCatch(callJMethod(conf, "get", key), + error = function(e) { + if (any(grep("java.util.NoSuchElementException", as.character(e)))) { + stop(paste0("Config '", key, "' is not set")) + } else { + stop(paste0("Unknown error: ", as.character(e))) + } + }) + } else { + callJMethod(conf, "get", key, defaultValue) + } + l <- setNames(list(value), key) + l + } +} +#' Get version of Spark on which this application is running +#' +#' Get version of Spark on which this application is running. +#' +#' @return a character string of the Spark version +#' @rdname sparkR.version +#' @name sparkR.version +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' version <- sparkR.version() +#' } +#' @note sparkR.version since 2.0.1 +sparkR.version <- function() { + sparkSession <- getSparkSession() + callJMethod(sparkSession, "version") +} + +getDefaultSqlSource <- function() { + l <- sparkR.conf("spark.sql.sources.default", "org.apache.spark.sql.parquet") + l[["spark.sql.sources.default"]] +} + +#' Create a SparkDataFrame +#' +#' Converts R data.frame or list into SparkDataFrame. +#' +#' @param data a list or data.frame. +#' @param schema a list of column names or named list (StructType), optional. +#' @param samplingRatio Currently not used. +#' @param numPartitions the number of partitions of the SparkDataFrame. Defaults to 1, this is +#' limited by length of the list or number of rows of the data.frame +#' @return A SparkDataFrame. +#' @rdname createDataFrame +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df1 <- as.DataFrame(iris) +#' df2 <- as.DataFrame(list(3,4,5,6)) +#' df3 <- createDataFrame(iris) +#' df4 <- createDataFrame(cars, numPartitions = 2) +#' } +#' @name createDataFrame +#' @method createDataFrame default +#' @note createDataFrame since 1.4.0 # TODO(davies): support sampling and infer type from NA -createDataFrame <- function(sqlContext, data, schema = NULL, samplingRatio = 1.0) { +createDataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0, + numPartitions = NULL) { + sparkSession <- getSparkSession() + if (is.data.frame(data)) { + # Convert data into a list of rows. Each row is a list. + # get the names of columns, they will be put into RDD if (is.null(schema)) { schema <- names(data) } - n <- nrow(data) - m <- ncol(data) + # get rid of factor type - dropFactor <- function(x) { + cleanCols <- function(x) { if (is.factor(x)) { as.character(x) } else { x } } - data <- lapply(1:n, function(i) { - lapply(1:m, function(j) { dropFactor(data[i,j]) }) - }) + + # drop factors and wrap lists + data <- setNames(lapply(data, cleanCols), NULL) + + # check if all columns have supported type + lapply(data, getInternalType) + + # convert to rows + args <- list(FUN = list, SIMPLIFY = FALSE, USE.NAMES = FALSE) + data <- do.call(mapply, append(args, data)) } + if (is.list(data)) { - sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sqlContext) - rdd <- parallelize(sc, data) + sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) + if (!is.null(numPartitions)) { + rdd <- parallelize(sc, data, numSlices = numToInt(numPartitions)) + } else { + rdd <- parallelize(sc, data, numSlices = 1) + } } else if (inherits(data, "RDD")) { rdd <- data } else { @@ -114,7 +250,7 @@ createDataFrame <- function(sqlContext, data, schema = NULL, samplingRatio = 1.0 } if (is.null(schema) || (!inherits(schema, "structType") && is.null(names(schema)))) { - row <- first(rdd) + row <- firstRDD(rdd) names <- if (is.null(schema)) { names(row) } else { @@ -144,384 +280,494 @@ createDataFrame <- function(sqlContext, data, schema = NULL, samplingRatio = 1.0 } stopifnot(class(schema) == "structType") - # schemaString <- tojson(schema) jrdd <- getJRDD(lapply(rdd, function(x) x), "row") srdd <- callJMethod(jrdd, "rdd") sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createDF", - srdd, schema$jobj, sqlContext) + srdd, schema$jobj, sparkSession) dataFrame(sdf) } +createDataFrame <- function(x, ...) { + dispatchFunc("createDataFrame(data, schema = NULL)", x, ...) +} + #' @rdname createDataFrame #' @aliases createDataFrame #' @export -as.DataFrame <- function(sqlContext, data, schema = NULL, samplingRatio = 1.0) { - createDataFrame(sqlContext, data, schema, samplingRatio) +#' @method as.DataFrame default +#' @note as.DataFrame since 1.6.0 +as.DataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0, numPartitions = NULL) { + createDataFrame(data, schema, samplingRatio, numPartitions) } -# toDF -# -# Converts an RDD to a DataFrame by infer the types. -# -# @param x An RDD -# -# @rdname DataFrame -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# sqlContext <- sparkRSQL.init(sc) -# rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x))) -# df <- toDF(rdd) -# } +#' @param ... additional argument(s). +#' @rdname createDataFrame +#' @aliases as.DataFrame +#' @export +as.DataFrame <- function(data, ...) { + dispatchFunc("as.DataFrame(data, schema = NULL)", data, ...) +} +#' toDF +#' +#' Converts an RDD to a SparkDataFrame by infer the types. +#' +#' @param x An RDD +#' +#' @rdname SparkDataFrame +#' @noRd +#' @examples +#'\dontrun{ +#' sparkR.session() +#' rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x))) +#' df <- toDF(rdd) +#'} setGeneric("toDF", function(x, ...) { standardGeneric("toDF") }) setMethod("toDF", signature(x = "RDD"), function(x, ...) { - sqlContext <- if (exists(".sparkRHivesc", envir = .sparkREnv)) { - get(".sparkRHivesc", envir = .sparkREnv) - } else if (exists(".sparkRSQLsc", envir = .sparkREnv)) { - get(".sparkRSQLsc", envir = .sparkREnv) - } else { - stop("no SQL context available") - } - createDataFrame(sqlContext, x, ...) + createDataFrame(x, ...) }) -#' Create a DataFrame from a JSON file. +#' Create a SparkDataFrame from a JSON file. #' -#' Loads a JSON file (one object per line), returning the result as a DataFrame +#' Loads a JSON file, returning the result as a SparkDataFrame +#' By default, (\href{http://jsonlines.org/}{JSON Lines text format or newline-delimited JSON} +#' ) is supported. For JSON (one record per file), set a named property \code{multiLine} to +#' \code{TRUE}. #' It goes through the entire dataset once to determine the schema. #' -#' @param sqlContext SQLContext to use #' @param path Path of file to read. A vector of multiple paths is allowed. -#' @return DataFrame +#' @param ... additional external data source specific named properties. +#' @return SparkDataFrame +#' @rdname read.json #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) +#' df <- read.json(path) +#' df <- read.json(path, multiLine = TRUE) +#' df <- jsonFile(path) #' } - -jsonFile <- function(sqlContext, path) { +#' @name read.json +#' @method read.json default +#' @note read.json since 1.6.0 +read.json.default <- function(path, ...) { + sparkSession <- getSparkSession() + options <- varargsToStrEnv(...) # Allow the user to have a more flexible definiton of the text file path - path <- suppressWarnings(normalizePath(path)) - # Convert a string vector of paths to a string containing comma separated paths - path <- paste(path, collapse = ",") - sdf <- callJMethod(sqlContext, "jsonFile", path) + paths <- as.list(suppressWarnings(normalizePath(path))) + read <- callJMethod(sparkSession, "read") + read <- callJMethod(read, "options", options) + sdf <- handledCallJMethod(read, "json", paths) dataFrame(sdf) } +read.json <- function(x, ...) { + dispatchFunc("read.json(path)", x, ...) +} -# JSON RDD -# -# Loads an RDD storing one JSON object per string as a DataFrame. -# -# @param sqlContext SQLContext to use -# @param rdd An RDD of JSON string -# @param schema A StructType object to use as schema -# @param samplingRatio The ratio of simpling used to infer the schema -# @return A DataFrame -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# sqlContext <- sparkRSQL.init(sc) -# rdd <- texFile(sc, "path/to/json") -# df <- jsonRDD(sqlContext, rdd) -# } +#' @rdname read.json +#' @name jsonFile +#' @export +#' @method jsonFile default +#' @note jsonFile since 1.4.0 +jsonFile.default <- function(path) { + .Deprecated("read.json") + read.json(path) +} + +jsonFile <- function(x, ...) { + dispatchFunc("jsonFile(path)", x, ...) +} + +#' JSON RDD +#' +#' Loads an RDD storing one JSON object per string as a SparkDataFrame. +#' +#' @param sqlContext SQLContext to use +#' @param rdd An RDD of JSON string +#' @param schema A StructType object to use as schema +#' @param samplingRatio The ratio of simpling used to infer the schema +#' @return A SparkDataFrame +#' @noRd +#' @examples +#'\dontrun{ +#' sparkR.session() +#' rdd <- texFile(sc, "path/to/json") +#' df <- jsonRDD(sqlContext, rdd) +#'} +# TODO: remove - this method is no longer exported # TODO: support schema jsonRDD <- function(sqlContext, rdd, schema = NULL, samplingRatio = 1.0) { + .Deprecated("read.json") rdd <- serializeToString(rdd) if (is.null(schema)) { - sdf <- callJMethod(sqlContext, "jsonRDD", callJMethod(getJRDD(rdd), "rdd"), samplingRatio) + read <- callJMethod(sqlContext, "read") + # samplingRatio is deprecated + sdf <- callJMethod(read, "json", callJMethod(getJRDD(rdd), "rdd")) dataFrame(sdf) } else { stop("not implemented") } } - -#' Create a DataFrame from a Parquet file. +#' Create a SparkDataFrame from an ORC file. #' -#' Loads a Parquet file, returning the result as a DataFrame. +#' Loads an ORC file, returning the result as a SparkDataFrame. #' -#' @param sqlContext SQLContext to use -#' @param ... Path(s) of parquet file(s) to read. -#' @return DataFrame +#' @param path Path of file to read. +#' @param ... additional external data source specific named properties. +#' @return SparkDataFrame +#' @rdname read.orc #' @export - -# TODO: Implement saveasParquetFile and write examples for both -parquetFile <- function(sqlContext, ...) { - # Allow the user to have a more flexible definiton of the text file path - paths <- lapply(list(...), function(x) suppressWarnings(normalizePath(x))) - sdf <- callJMethod(sqlContext, "parquetFile", paths) +#' @name read.orc +#' @note read.orc since 2.0.0 +read.orc <- function(path, ...) { + sparkSession <- getSparkSession() + options <- varargsToStrEnv(...) + # Allow the user to have a more flexible definiton of the ORC file path + path <- suppressWarnings(normalizePath(path)) + read <- callJMethod(sparkSession, "read") + read <- callJMethod(read, "options", options) + sdf <- handledCallJMethod(read, "orc", path) dataFrame(sdf) } -#' SQL Query -#' -#' Executes a SQL query using Spark, returning the result as a DataFrame. -#' -#' @param sqlContext SQLContext to use -#' @param sqlQuery A character vector containing the SQL query -#' @return DataFrame -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) -#' registerTempTable(df, "table") -#' new_df <- sql(sqlContext, "SELECT * FROM table") -#' } - -sql <- function(sqlContext, sqlQuery) { - sdf <- callJMethod(sqlContext, "sql", sqlQuery) - dataFrame(sdf) -} - -#' Create a DataFrame from a SparkSQL Table +#' Create a SparkDataFrame from a Parquet file. #' -#' Returns the specified Table as a DataFrame. The Table must have already been registered -#' in the SQLContext. +#' Loads a Parquet file, returning the result as a SparkDataFrame. #' -#' @param sqlContext SQLContext to use -#' @param tableName The SparkSQL Table to convert to a DataFrame. -#' @return DataFrame +#' @param path path of file to read. A vector of multiple paths is allowed. +#' @return SparkDataFrame +#' @rdname read.parquet #' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) -#' registerTempTable(df, "table") -#' new_df <- table(sqlContext, "table") -#' } - -table <- function(sqlContext, tableName) { - sdf <- callJMethod(sqlContext, "table", tableName) +#' @name read.parquet +#' @method read.parquet default +#' @note read.parquet since 1.6.0 +read.parquet.default <- function(path, ...) { + sparkSession <- getSparkSession() + options <- varargsToStrEnv(...) + # Allow the user to have a more flexible definiton of the Parquet file path + paths <- as.list(suppressWarnings(normalizePath(path))) + read <- callJMethod(sparkSession, "read") + read <- callJMethod(read, "options", options) + sdf <- handledCallJMethod(read, "parquet", paths) dataFrame(sdf) } +read.parquet <- function(x, ...) { + dispatchFunc("read.parquet(...)", x, ...) +} -#' Tables -#' -#' Returns a DataFrame containing names of tables in the given database. -#' -#' @param sqlContext SQLContext to use -#' @param databaseName name of the database -#' @return a DataFrame +#' @param ... argument(s) passed to the method. +#' @rdname read.parquet +#' @name parquetFile #' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' tables(sqlContext, "hive") -#' } - -tables <- function(sqlContext, databaseName = NULL) { - jdf <- if (is.null(databaseName)) { - callJMethod(sqlContext, "tables") - } else { - callJMethod(sqlContext, "tables", databaseName) - } - dataFrame(jdf) +#' @method parquetFile default +#' @note parquetFile since 1.4.0 +parquetFile.default <- function(...) { + .Deprecated("read.parquet") + read.parquet(unlist(list(...))) } +parquetFile <- function(x, ...) { + dispatchFunc("parquetFile(...)", x, ...) +} -#' Table Names +#' Create a SparkDataFrame from a text file. #' -#' Returns the names of tables in the given database as an array. +#' Loads text files and returns a SparkDataFrame whose schema starts with +#' a string column named "value", and followed by partitioned columns if +#' there are any. #' -#' @param sqlContext SQLContext to use -#' @param databaseName name of the database -#' @return a list of table names +#' Each line in the text file is a new row in the resulting SparkDataFrame. +#' +#' @param path Path of file to read. A vector of multiple paths is allowed. +#' @param ... additional external data source specific named properties. +#' @return SparkDataFrame +#' @rdname read.text #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' tableNames(sqlContext, "hive") +#' sparkR.session() +#' path <- "path/to/file.txt" +#' df <- read.text(path) #' } - -tableNames <- function(sqlContext, databaseName = NULL) { - if (is.null(databaseName)) { - callJMethod(sqlContext, "tableNames") - } else { - callJMethod(sqlContext, "tableNames", databaseName) - } +#' @name read.text +#' @method read.text default +#' @note read.text since 1.6.1 +read.text.default <- function(path, ...) { + sparkSession <- getSparkSession() + options <- varargsToStrEnv(...) + # Allow the user to have a more flexible definiton of the text file path + paths <- as.list(suppressWarnings(normalizePath(path))) + read <- callJMethod(sparkSession, "read") + read <- callJMethod(read, "options", options) + sdf <- handledCallJMethod(read, "text", paths) + dataFrame(sdf) } +read.text <- function(x, ...) { + dispatchFunc("read.text(path)", x, ...) +} -#' Cache Table +#' SQL Query #' -#' Caches the specified table in-memory. +#' Executes a SQL query using Spark, returning the result as a SparkDataFrame. #' -#' @param sqlContext SQLContext to use -#' @param tableName The name of the table being cached -#' @return DataFrame +#' @param sqlQuery A character vector containing the SQL query +#' @return SparkDataFrame +#' @rdname sql #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) -#' registerTempTable(df, "table") -#' cacheTable(sqlContext, "table") +#' df <- read.json(path) +#' createOrReplaceTempView(df, "table") +#' new_df <- sql("SELECT * FROM table") #' } +#' @name sql +#' @method sql default +#' @note sql since 1.4.0 +sql.default <- function(sqlQuery) { + sparkSession <- getSparkSession() + sdf <- callJMethod(sparkSession, "sql", sqlQuery) + dataFrame(sdf) +} -cacheTable <- function(sqlContext, tableName) { - callJMethod(sqlContext, "cacheTable", tableName) +sql <- function(x, ...) { + dispatchFunc("sql(sqlQuery)", x, ...) } -#' Uncache Table +#' Create a SparkDataFrame from a SparkSQL table or view #' -#' Removes the specified table from the in-memory cache. +#' Returns the specified table or view as a SparkDataFrame. The table or view must already exist or +#' have already been registered in the SparkSession. #' -#' @param sqlContext SQLContext to use -#' @param tableName The name of the table being uncached -#' @return DataFrame +#' @param tableName the qualified or unqualified name that designates a table or view. If a database +#' is specified, it identifies the table/view from the database. +#' Otherwise, it first attempts to find a temporary view with the given name +#' and then match the table/view from the current database. +#' @return SparkDataFrame +#' @rdname tableToDF +#' @name tableToDF #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) +#' sparkR.session() #' path <- "path/to/file.json" -#' df <- jsonFile(sqlContext, path) -#' registerTempTable(df, "table") -#' uncacheTable(sqlContext, "table") +#' df <- read.json(path) +#' createOrReplaceTempView(df, "table") +#' new_df <- tableToDF("table") #' } - -uncacheTable <- function(sqlContext, tableName) { - callJMethod(sqlContext, "uncacheTable", tableName) -} - -#' Clear Cache -#' -#' Removes all cached tables from the in-memory cache. -#' -#' @param sqlContext SQLContext to use -#' @examples -#' \dontrun{ -#' clearCache(sqlContext) -#' } - -clearCache <- function(sqlContext) { - callJMethod(sqlContext, "clearCache") -} - -#' Drop Temporary Table -#' -#' Drops the temporary table with the given table name in the catalog. -#' If the table has been cached/persisted before, it's also unpersisted. -#' -#' @param sqlContext SQLContext to use -#' @param tableName The name of the SparkSQL table to be dropped. -#' @examples -#' \dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' df <- read.df(sqlContext, path, "parquet") -#' registerTempTable(df, "table") -#' dropTempTable(sqlContext, "table") -#' } - -dropTempTable <- function(sqlContext, tableName) { - if (class(tableName) != "character") { - stop("tableName must be a string.") - } - callJMethod(sqlContext, "dropTempTable", tableName) +#' @note tableToDF since 2.0.0 +tableToDF <- function(tableName) { + sparkSession <- getSparkSession() + sdf <- callJMethod(sparkSession, "table", tableName) + dataFrame(sdf) } -#' Load an DataFrame +#' Load a SparkDataFrame #' -#' Returns the dataset in a data source as a DataFrame +#' Returns the dataset in a data source as a SparkDataFrame #' -#' The data source is specified by the `source` and a set of options(...). -#' If `source` is not specified, the default data source configured by -#' "spark.sql.sources.default" will be used. +#' The data source is specified by the \code{source} and a set of options(...). +#' If \code{source} is not specified, the default data source configured by +#' "spark.sql.sources.default" will be used. \cr +#' Similar to R read.csv, when \code{source} is "csv", by default, a value of "NA" will be +#' interpreted as NA. #' -#' @param sqlContext SQLContext to use #' @param path The path of files to load #' @param source The name of external data source -#' @param schema The data schema defined in structType -#' @return DataFrame +#' @param schema The data schema defined in structType or a DDL-formatted string. +#' @param na.strings Default string value for NA when source is "csv" +#' @param ... additional external data source specific named properties. +#' @return SparkDataFrame #' @rdname read.df #' @name read.df +#' @seealso \link{read.json} #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' df1 <- read.df(sqlContext, "path/to/file.json", source = "json") +#' sparkR.session() +#' df1 <- read.df("path/to/file.json", source = "json") #' schema <- structType(structField("name", "string"), #' structField("info", "map")) -#' df2 <- read.df(sqlContext, mapTypeJsonPath, "json", schema) -#' df3 <- loadDF(sqlContext, "data/test_table", "parquet", mergeSchema = "true") +#' df2 <- read.df(mapTypeJsonPath, "json", schema, multiLine = TRUE) +#' df3 <- loadDF("data/test_table", "parquet", mergeSchema = "true") +#' stringSchema <- "name STRING, info MAP" +#' df4 <- read.df(mapTypeJsonPath, "json", stringSchema, multiLine = TRUE) #' } - -read.df <- function(sqlContext, path = NULL, source = NULL, schema = NULL, ...) { - options <- varargsToEnv(...) +#' @name read.df +#' @method read.df default +#' @note read.df since 1.4.0 +read.df.default <- function(path = NULL, source = NULL, schema = NULL, na.strings = "NA", ...) { + if (!is.null(path) && !is.character(path)) { + stop("path should be character, NULL or omitted.") + } + if (!is.null(source) && !is.character(source)) { + stop("source should be character, NULL or omitted. It is the datasource specified ", + "in 'spark.sql.sources.default' configuration by default.") + } + sparkSession <- getSparkSession() + options <- varargsToStrEnv(...) if (!is.null(path)) { options[["path"]] <- path } if (is.null(source)) { - sqlContext <- get(".sparkRSQLsc", envir = .sparkREnv) - source <- callJMethod(sqlContext, "getConf", "spark.sql.sources.default", - "org.apache.spark.sql.parquet") + source <- getDefaultSqlSource() + } + if (source == "csv" && is.null(options[["nullValue"]])) { + options[["nullValue"]] <- na.strings } + read <- callJMethod(sparkSession, "read") + read <- callJMethod(read, "format", source) if (!is.null(schema)) { - stopifnot(class(schema) == "structType") - sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "loadDF", sqlContext, source, - schema$jobj, options) - } else { - sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "loadDF", sqlContext, source, options) + if (class(schema) == "structType") { + read <- callJMethod(read, "schema", schema$jobj) + } else if (is.character(schema)) { + read <- callJMethod(read, "schema", schema) + } else { + stop("schema should be structType or character.") + } } + read <- callJMethod(read, "options", options) + sdf <- handledCallJMethod(read, "load") dataFrame(sdf) } +read.df <- function(x = NULL, ...) { + dispatchFunc("read.df(path = NULL, source = NULL, schema = NULL, ...)", x, ...) +} + #' @rdname read.df #' @name loadDF -loadDF <- function(sqlContext, path = NULL, source = NULL, schema = NULL, ...) { - read.df(sqlContext, path, source, schema, ...) +#' @method loadDF default +#' @note loadDF since 1.6.0 +loadDF.default <- function(path = NULL, source = NULL, schema = NULL, ...) { + read.df(path, source, schema, ...) +} + +loadDF <- function(x = NULL, ...) { + dispatchFunc("loadDF(path = NULL, source = NULL, schema = NULL, ...)", x, ...) } -#' Create an external table +#' Create a SparkDataFrame representing the database table accessible via JDBC URL #' -#' Creates an external table based on the dataset in a data source, -#' Returns the DataFrame associated with the external table. +#' Additional JDBC database connection properties can be set (...) #' -#' 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. +#' Only one of partitionColumn or predicates should be set. Partitions of the table will be +#' retrieved in parallel based on the \code{numPartitions} or by the predicates. #' -#' @param sqlContext SQLContext to use -#' @param tableName A name of the table -#' @param path The path of files to load -#' @param source the name of external data source -#' @return DataFrame +#' Don't create too many partitions in parallel on a large cluster; otherwise Spark might crash +#' your external database systems. +#' +#' @param url JDBC database url of the form \code{jdbc:subprotocol:subname} +#' @param tableName the name of the table in the external database +#' @param partitionColumn the name of a column of integral type that will be used for partitioning +#' @param lowerBound the minimum value of \code{partitionColumn} used to decide partition stride +#' @param upperBound the maximum value of \code{partitionColumn} used to decide partition stride +#' @param numPartitions the number of partitions, This, along with \code{lowerBound} (inclusive), +#' \code{upperBound} (exclusive), form partition strides for generated WHERE +#' clause expressions used to split the column \code{partitionColumn} evenly. +#' This defaults to SparkContext.defaultParallelism when unset. +#' @param predicates a list of conditions in the where clause; each one defines one partition +#' @param ... additional JDBC database connection named properties. +#' @return SparkDataFrame +#' @rdname read.jdbc +#' @name read.jdbc #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' df <- sparkRSQL.createExternalTable(sqlContext, "myjson", path="path/to/json", source="json") +#' sparkR.session() +#' jdbcUrl <- "jdbc:mysql://localhost:3306/databasename" +#' df <- read.jdbc(jdbcUrl, "table", predicates = list("field<=123"), user = "username") +#' df2 <- read.jdbc(jdbcUrl, "table2", partitionColumn = "index", lowerBound = 0, +#' upperBound = 10000, user = "username", password = "password") #' } +#' @note read.jdbc since 2.0.0 +read.jdbc <- function(url, tableName, + partitionColumn = NULL, lowerBound = NULL, upperBound = NULL, + numPartitions = 0L, predicates = list(), ...) { + jprops <- varargsToJProperties(...) + sparkSession <- getSparkSession() + read <- callJMethod(sparkSession, "read") + if (!is.null(partitionColumn)) { + if (is.null(numPartitions) || numPartitions == 0) { + sc <- callJMethod(sparkSession, "sparkContext") + numPartitions <- callJMethod(sc, "defaultParallelism") + } else { + numPartitions <- numToInt(numPartitions) + } + sdf <- handledCallJMethod(read, "jdbc", url, tableName, as.character(partitionColumn), + numToInt(lowerBound), numToInt(upperBound), numPartitions, jprops) + } else if (length(predicates) > 0) { + sdf <- handledCallJMethod(read, "jdbc", url, tableName, as.list(as.character(predicates)), + jprops) + } else { + sdf <- handledCallJMethod(read, "jdbc", url, tableName, jprops) + } + dataFrame(sdf) +} -createExternalTable <- function(sqlContext, tableName, path = NULL, source = NULL, ...) { - options <- varargsToEnv(...) - if (!is.null(path)) { - options[["path"]] <- path +#' Load a streaming SparkDataFrame +#' +#' Returns the dataset in a data source as a SparkDataFrame +#' +#' The data source is specified by the \code{source} and a set of options(...). +#' If \code{source} is not specified, the default data source configured by +#' "spark.sql.sources.default" will be used. +#' +#' @param source The name of external data source +#' @param schema The data schema defined in structType or a DDL-formatted string, this is +#' required for file-based streaming data source +#' @param ... additional external data source specific named options, for instance \code{path} for +#' file-based streaming data source +#' @return SparkDataFrame +#' @rdname read.stream +#' @name read.stream +#' @seealso \link{write.stream} +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df <- read.stream("socket", host = "localhost", port = 9999) +#' q <- write.stream(df, "text", path = "/home/user/out", checkpointLocation = "/home/user/cp") +#' +#' df <- read.stream("json", path = jsonDir, schema = schema, maxFilesPerTrigger = 1) +#' stringSchema <- "name STRING, info MAP" +#' df1 <- read.stream("json", path = jsonDir, schema = stringSchema, maxFilesPerTrigger = 1) +#' } +#' @name read.stream +#' @note read.stream since 2.2.0 +#' @note experimental +read.stream <- function(source = NULL, schema = NULL, ...) { + sparkSession <- getSparkSession() + if (!is.null(source) && !is.character(source)) { + stop("source should be character, NULL or omitted. It is the data source specified ", + "in 'spark.sql.sources.default' configuration by default.") + } + if (is.null(source)) { + source <- getDefaultSqlSource() + } + options <- varargsToStrEnv(...) + read <- callJMethod(sparkSession, "readStream") + read <- callJMethod(read, "format", source) + if (!is.null(schema)) { + if (class(schema) == "structType") { + read <- callJMethod(read, "schema", schema$jobj) + } else if (is.character(schema)) { + read <- callJMethod(read, "schema", schema) + } else { + stop("schema should be structType or character.") + } } - sdf <- callJMethod(sqlContext, "createExternalTable", tableName, source, options) + read <- callJMethod(read, "options", options) + sdf <- handledCallJMethod(read, "load") dataFrame(sdf) } diff --git a/R/pkg/R/WindowSpec.R b/R/pkg/R/WindowSpec.R new file mode 100644 index 000000000000..81beac9ea992 --- /dev/null +++ b/R/pkg/R/WindowSpec.R @@ -0,0 +1,224 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# WindowSpec.R - WindowSpec class and methods implemented in S4 OO classes + +#' @include generics.R jobj.R column.R +NULL + +#' S4 class that represents a WindowSpec +#' +#' WindowSpec can be created by using windowPartitionBy() or windowOrderBy() +#' +#' @rdname WindowSpec +#' @seealso \link{windowPartitionBy}, \link{windowOrderBy} +#' +#' @param sws A Java object reference to the backing Scala WindowSpec +#' @export +#' @note WindowSpec since 2.0.0 +setClass("WindowSpec", + slots = list(sws = "jobj")) + +setMethod("initialize", "WindowSpec", function(.Object, sws) { + .Object@sws <- sws + .Object +}) + +windowSpec <- function(sws) { + stopifnot(class(sws) == "jobj") + new("WindowSpec", sws) +} + +#' @rdname show +#' @export +#' @note show(WindowSpec) since 2.0.0 +setMethod("show", "WindowSpec", + function(object) { + cat("WindowSpec", callJMethod(object@sws, "toString"), "\n") + }) + +#' partitionBy +#' +#' Defines the partitioning columns in a WindowSpec. +#' +#' @param x a WindowSpec. +#' @param col a column to partition on (desribed by the name or Column). +#' @param ... additional column(s) to partition on. +#' @return A WindowSpec. +#' @rdname partitionBy +#' @name partitionBy +#' @aliases partitionBy,WindowSpec-method +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' partitionBy(ws, "col1", "col2") +#' partitionBy(ws, df$col1, df$col2) +#' } +#' @note partitionBy(WindowSpec) since 2.0.0 +setMethod("partitionBy", + signature(x = "WindowSpec"), + function(x, col, ...) { + stopifnot (class(col) %in% c("character", "Column")) + + if (class(col) == "character") { + windowSpec(callJMethod(x@sws, "partitionBy", col, list(...))) + } else { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec(callJMethod(x@sws, "partitionBy", jcols)) + } + }) + +#' Ordering Columns in a WindowSpec +#' +#' Defines the ordering columns in a WindowSpec. +#' @param x a WindowSpec +#' @param col a character or Column indicating an ordering column +#' @param ... additional sorting fields +#' @return A WindowSpec. +#' @name orderBy +#' @rdname orderBy +#' @aliases orderBy,WindowSpec,character-method +#' @family windowspec_method +#' @seealso See \link{arrange} for use in sorting a SparkDataFrame +#' @export +#' @examples +#' \dontrun{ +#' orderBy(ws, "col1", "col2") +#' orderBy(ws, df$col1, df$col2) +#' } +#' @note orderBy(WindowSpec, character) since 2.0.0 +setMethod("orderBy", + signature(x = "WindowSpec", col = "character"), + function(x, col, ...) { + windowSpec(callJMethod(x@sws, "orderBy", col, list(...))) + }) + +#' @rdname orderBy +#' @name orderBy +#' @aliases orderBy,WindowSpec,Column-method +#' @export +#' @note orderBy(WindowSpec, Column) since 2.0.0 +setMethod("orderBy", + signature(x = "WindowSpec", col = "Column"), + function(x, col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec(callJMethod(x@sws, "orderBy", jcols)) + }) + +#' rowsBetween +#' +#' Defines the frame boundaries, from \code{start} (inclusive) to \code{end} (inclusive). +#' +#' Both \code{start} and \code{end} are relative positions from the current row. For example, +#' "0" means "current row", while "-1" means the row before the current row, and "5" means the +#' fifth row after the current row. +#' +#' @param x a WindowSpec +#' @param start boundary start, inclusive. +#' The frame is unbounded if this is the minimum long value. +#' @param end boundary end, inclusive. +#' The frame is unbounded if this is the maximum long value. +#' @return a WindowSpec +#' @rdname rowsBetween +#' @aliases rowsBetween,WindowSpec,numeric,numeric-method +#' @name rowsBetween +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' rowsBetween(ws, 0, 3) +#' } +#' @note rowsBetween since 2.0.0 +setMethod("rowsBetween", + signature(x = "WindowSpec", start = "numeric", end = "numeric"), + function(x, start, end) { + # "start" and "end" should be long, due to serde limitation, + # limit "start" and "end" as integer now + windowSpec(callJMethod(x@sws, "rowsBetween", as.integer(start), as.integer(end))) + }) + +#' rangeBetween +#' +#' Defines the frame boundaries, from \code{start} (inclusive) to \code{end} (inclusive). +#' +#' Both \code{start} and \code{end} are relative from the current row. For example, "0" means +#' "current row", while "-1" means one off before the current row, and "5" means the five off +#' after the current row. +#' +#' @param x a WindowSpec +#' @param start boundary start, inclusive. +#' The frame is unbounded if this is the minimum long value. +#' @param end boundary end, inclusive. +#' The frame is unbounded if this is the maximum long value. +#' @return a WindowSpec +#' @rdname rangeBetween +#' @aliases rangeBetween,WindowSpec,numeric,numeric-method +#' @name rangeBetween +#' @family windowspec_method +#' @export +#' @examples +#' \dontrun{ +#' rangeBetween(ws, 0, 3) +#' } +#' @note rangeBetween since 2.0.0 +setMethod("rangeBetween", + signature(x = "WindowSpec", start = "numeric", end = "numeric"), + function(x, start, end) { + # "start" and "end" should be long, due to serde limitation, + # limit "start" and "end" as integer now + windowSpec(callJMethod(x@sws, "rangeBetween", as.integer(start), as.integer(end))) + }) + +# Note that over is a method of Column class, but it is placed here to +# avoid Roxygen circular-dependency between class Column and WindowSpec. + +#' over +#' +#' Define a windowing column. +#' +#' @param x a Column, usually one returned by window function(s). +#' @param window a WindowSpec object. Can be created by \code{windowPartitionBy} or +#' \code{windowOrderBy} and configured by other WindowSpec methods. +#' @rdname over +#' @name over +#' @aliases over,Column,WindowSpec-method +#' @family colum_func +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame(mtcars) +#' +#' # Partition by am (transmission) and order by hp (horsepower) +#' ws <- orderBy(windowPartitionBy("am"), "hp") +#' +#' # Rank on hp within each partition +#' out <- select(df, over(rank(), ws), df$hp, df$am) +#' +#' # Lag mpg values by 1 row on the partition-and-ordered table +#' out <- select(df, over(lead(df$mpg), ws), df$mpg, df$hp, df$am) +#' } +#' @note over since 2.0.0 +setMethod("over", + signature(x = "Column", window = "WindowSpec"), + function(x, window) { + column(callJMethod(x@jc, "over", window@sws)) + }) diff --git a/R/pkg/R/backend.R b/R/pkg/R/backend.R index 49162838b8d1..0a789e6c379d 100644 --- a/R/pkg/R/backend.R +++ b/R/pkg/R/backend.R @@ -68,7 +68,7 @@ isRemoveMethod <- function(isStatic, objId, methodName) { # methodName - name of method to be invoked invokeJava <- function(isStatic, objId, methodName, ...) { if (!exists(".sparkRCon", .sparkREnv)) { - stop("No connection to backend found. Please re-run sparkR.init") + stop("No connection to backend found. Please re-run sparkR.session()") } # If this isn't a removeJObject call @@ -108,10 +108,27 @@ invokeJava <- function(isStatic, objId, methodName, ...) { conn <- get(".sparkRCon", .sparkREnv) writeBin(requestMessage, conn) - # TODO: check the status code to output error information returnStatus <- readInt(conn) - if (returnStatus != 0) { - stop(readString(conn)) + handleErrors(returnStatus, conn) + + # Backend will send +1 as keep alive value to prevent various connection timeouts + # on very long running jobs. See spark.r.heartBeatInterval + while (returnStatus == 1) { + returnStatus <- readInt(conn) + handleErrors(returnStatus, conn) } + readObject(conn) } + +# Helper function to check for returned errors and print appropriate error message to user +handleErrors <- function(returnStatus, conn) { + if (length(returnStatus) == 0) { + stop("No status is returned. Java SparkR backend might have failed.") + } + + # 0 is success and +1 is reserved for heartbeats. Other negative values indicate errors. + if (returnStatus < 0) { + stop(readString(conn)) + } +} diff --git a/R/pkg/R/broadcast.R b/R/pkg/R/broadcast.R index 2403925b267c..398dffc4ab1b 100644 --- a/R/pkg/R/broadcast.R +++ b/R/pkg/R/broadcast.R @@ -23,9 +23,11 @@ .broadcastValues <- new.env() .broadcastIdToName <- new.env() -# @title S4 class that represents a Broadcast variable -# @description Broadcast variables can be created using the broadcast -# function from a \code{SparkContext}. +# S4 class that represents a Broadcast variable +# +# Broadcast variables can be created using the broadcast +# function from a \code{SparkContext}. +# # @rdname broadcast-class # @seealso broadcast # @@ -51,7 +53,6 @@ Broadcast <- function(id, value, jBroadcastRef, objName) { # # @param bcast The broadcast variable to get # @rdname broadcast -# @aliases value,Broadcast-method setMethod("value", signature(bcast = "Broadcast"), function(bcast) { diff --git a/R/pkg/R/catalog.R b/R/pkg/R/catalog.R new file mode 100644 index 000000000000..e59a7024333a --- /dev/null +++ b/R/pkg/R/catalog.R @@ -0,0 +1,526 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# catalog.R: SparkSession catalog functions + +#' (Deprecated) Create an external table +#' +#' Creates an external table based on the dataset in a data source, +#' Returns a SparkDataFrame associated with the external table. +#' +#' The data source is specified by the \code{source} and a set of options(...). +#' If \code{source} is not specified, the default data source configured by +#' "spark.sql.sources.default" will be used. +#' +#' @param tableName a name of the table. +#' @param path the path of files to load. +#' @param source the name of external data source. +#' @param schema the schema of the data required for some data sources. +#' @param ... additional argument(s) passed to the method. +#' @return A SparkDataFrame. +#' @rdname createExternalTable-deprecated +#' @seealso \link{createTable} +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df <- createExternalTable("myjson", path="path/to/json", source="json", schema) +#' } +#' @name createExternalTable +#' @method createExternalTable default +#' @note createExternalTable since 1.4.0 +createExternalTable.default <- function(tableName, path = NULL, source = NULL, schema = NULL, ...) { + .Deprecated("createTable", old = "createExternalTable") + createTable(tableName, path, source, schema, ...) +} + +createExternalTable <- function(x, ...) { + dispatchFunc("createExternalTable(tableName, path = NULL, source = NULL, ...)", x, ...) +} + +#' Creates a table based on the dataset in a data source +#' +#' Creates a table based on the dataset in a data source. Returns a SparkDataFrame associated with +#' the table. +#' +#' The data source is specified by the \code{source} and a set of options(...). +#' If \code{source} is not specified, the default data source configured by +#' "spark.sql.sources.default" will be used. When a \code{path} is specified, an external table is +#' created from the data at the given path. Otherwise a managed table is created. +#' +#' @param tableName the qualified or unqualified name that designates a table. If no database +#' identifier is provided, it refers to a table in the current database. +#' @param path (optional) the path of files to load. +#' @param source (optional) the name of the data source. +#' @param schema (optional) the schema of the data required for some data sources. +#' @param ... additional named parameters as options for the data source. +#' @return A SparkDataFrame. +#' @rdname createTable +#' @seealso \link{createExternalTable} +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df <- createTable("myjson", path="path/to/json", source="json", schema) +#' +#' createTable("people", source = "json", schema = schema) +#' insertInto(df, "people") +#' } +#' @name createTable +#' @note createTable since 2.2.0 +createTable <- function(tableName, path = NULL, source = NULL, schema = NULL, ...) { + sparkSession <- getSparkSession() + options <- varargsToStrEnv(...) + if (!is.null(path)) { + options[["path"]] <- path + } + if (is.null(source)) { + source <- getDefaultSqlSource() + } + catalog <- callJMethod(sparkSession, "catalog") + if (is.null(schema)) { + sdf <- callJMethod(catalog, "createTable", tableName, source, options) + } else if (class(schema) == "structType") { + sdf <- callJMethod(catalog, "createTable", tableName, source, schema$jobj, options) + } else { + stop("schema must be a structType.") + } + dataFrame(sdf) +} + +#' Cache Table +#' +#' Caches the specified table in-memory. +#' +#' @param tableName the qualified or unqualified name that designates a table. If no database +#' identifier is provided, it refers to a table in the current database. +#' @return SparkDataFrame +#' @rdname cacheTable +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' createOrReplaceTempView(df, "table") +#' cacheTable("table") +#' } +#' @name cacheTable +#' @method cacheTable default +#' @note cacheTable since 1.4.0 +cacheTable.default <- function(tableName) { + sparkSession <- getSparkSession() + catalog <- callJMethod(sparkSession, "catalog") + invisible(handledCallJMethod(catalog, "cacheTable", tableName)) +} + +cacheTable <- function(x, ...) { + dispatchFunc("cacheTable(tableName)", x, ...) +} + +#' Uncache Table +#' +#' Removes the specified table from the in-memory cache. +#' +#' @param tableName the qualified or unqualified name that designates a table. If no database +#' identifier is provided, it refers to a table in the current database. +#' @return SparkDataFrame +#' @rdname uncacheTable +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' createOrReplaceTempView(df, "table") +#' uncacheTable("table") +#' } +#' @name uncacheTable +#' @method uncacheTable default +#' @note uncacheTable since 1.4.0 +uncacheTable.default <- function(tableName) { + sparkSession <- getSparkSession() + catalog <- callJMethod(sparkSession, "catalog") + invisible(handledCallJMethod(catalog, "uncacheTable", tableName)) +} + +uncacheTable <- function(x, ...) { + dispatchFunc("uncacheTable(tableName)", x, ...) +} + +#' Clear Cache +#' +#' Removes all cached tables from the in-memory cache. +#' +#' @rdname clearCache +#' @export +#' @examples +#' \dontrun{ +#' clearCache() +#' } +#' @name clearCache +#' @method clearCache default +#' @note clearCache since 1.4.0 +clearCache.default <- function() { + sparkSession <- getSparkSession() + catalog <- callJMethod(sparkSession, "catalog") + invisible(callJMethod(catalog, "clearCache")) +} + +clearCache <- function() { + dispatchFunc("clearCache()") +} + +#' (Deprecated) Drop Temporary Table +#' +#' Drops the temporary table with the given table name in the catalog. +#' If the table has been cached/persisted before, it's also unpersisted. +#' +#' @param tableName The name of the SparkSQL table to be dropped. +#' @seealso \link{dropTempView} +#' @rdname dropTempTable-deprecated +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' df <- read.df(path, "parquet") +#' createOrReplaceTempView(df, "table") +#' dropTempTable("table") +#' } +#' @name dropTempTable +#' @method dropTempTable default +#' @note dropTempTable since 1.4.0 +dropTempTable.default <- function(tableName) { + .Deprecated("dropTempView", old = "dropTempTable") + if (class(tableName) != "character") { + stop("tableName must be a string.") + } + dropTempView(tableName) +} + +dropTempTable <- function(x, ...) { + dispatchFunc("dropTempView(viewName)", x, ...) +} + +#' Drops the temporary view with the given view name in the catalog. +#' +#' Drops the temporary view with the given view name in the catalog. +#' If the view has been cached before, then it will also be uncached. +#' +#' @param viewName the name of the temporary view to be dropped. +#' @return TRUE if the view is dropped successfully, FALSE otherwise. +#' @rdname dropTempView +#' @name dropTempView +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' df <- read.df(path, "parquet") +#' createOrReplaceTempView(df, "table") +#' dropTempView("table") +#' } +#' @note since 2.0.0 +dropTempView <- function(viewName) { + sparkSession <- getSparkSession() + if (class(viewName) != "character") { + stop("viewName must be a string.") + } + catalog <- callJMethod(sparkSession, "catalog") + callJMethod(catalog, "dropTempView", viewName) +} + +#' Tables +#' +#' Returns a SparkDataFrame containing names of tables in the given database. +#' +#' @param databaseName (optional) name of the database +#' @return a SparkDataFrame +#' @rdname tables +#' @seealso \link{listTables} +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' tables("hive") +#' } +#' @name tables +#' @method tables default +#' @note tables since 1.4.0 +tables.default <- function(databaseName = NULL) { + # rename column to match previous output schema + withColumnRenamed(listTables(databaseName), "name", "tableName") +} + +tables <- function(x, ...) { + dispatchFunc("tables(databaseName = NULL)", x, ...) +} + +#' Table Names +#' +#' Returns the names of tables in the given database as an array. +#' +#' @param databaseName (optional) name of the database +#' @return a list of table names +#' @rdname tableNames +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' tableNames("hive") +#' } +#' @name tableNames +#' @method tableNames default +#' @note tableNames since 1.4.0 +tableNames.default <- function(databaseName = NULL) { + sparkSession <- getSparkSession() + callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "getTableNames", + sparkSession, + databaseName) +} + +tableNames <- function(x, ...) { + dispatchFunc("tableNames(databaseName = NULL)", x, ...) +} + +#' Returns the current default database +#' +#' Returns the current default database. +#' +#' @return name of the current default database. +#' @rdname currentDatabase +#' @name currentDatabase +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' currentDatabase() +#' } +#' @note since 2.2.0 +currentDatabase <- function() { + sparkSession <- getSparkSession() + catalog <- callJMethod(sparkSession, "catalog") + callJMethod(catalog, "currentDatabase") +} + +#' Sets the current default database +#' +#' Sets the current default database. +#' +#' @param databaseName name of the database +#' @rdname setCurrentDatabase +#' @name setCurrentDatabase +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' setCurrentDatabase("default") +#' } +#' @note since 2.2.0 +setCurrentDatabase <- function(databaseName) { + sparkSession <- getSparkSession() + if (class(databaseName) != "character") { + stop("databaseName must be a string.") + } + catalog <- callJMethod(sparkSession, "catalog") + invisible(handledCallJMethod(catalog, "setCurrentDatabase", databaseName)) +} + +#' Returns a list of databases available +#' +#' Returns a list of databases available. +#' +#' @return a SparkDataFrame of the list of databases. +#' @rdname listDatabases +#' @name listDatabases +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' listDatabases() +#' } +#' @note since 2.2.0 +listDatabases <- function() { + sparkSession <- getSparkSession() + catalog <- callJMethod(sparkSession, "catalog") + dataFrame(callJMethod(callJMethod(catalog, "listDatabases"), "toDF")) +} + +#' Returns a list of tables or views in the specified database +#' +#' Returns a list of tables or views in the specified database. +#' This includes all temporary views. +#' +#' @param databaseName (optional) name of the database +#' @return a SparkDataFrame of the list of tables. +#' @rdname listTables +#' @name listTables +#' @seealso \link{tables} +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' listTables() +#' listTables("default") +#' } +#' @note since 2.2.0 +listTables <- function(databaseName = NULL) { + sparkSession <- getSparkSession() + if (!is.null(databaseName) && class(databaseName) != "character") { + stop("databaseName must be a string.") + } + catalog <- callJMethod(sparkSession, "catalog") + jdst <- if (is.null(databaseName)) { + callJMethod(catalog, "listTables") + } else { + handledCallJMethod(catalog, "listTables", databaseName) + } + dataFrame(callJMethod(jdst, "toDF")) +} + +#' Returns a list of columns for the given table/view in the specified database +#' +#' Returns a list of columns for the given table/view in the specified database. +#' +#' @param tableName the qualified or unqualified name that designates a table/view. If no database +#' identifier is provided, it refers to a table/view in the current database. +#' If \code{databaseName} parameter is specified, this must be an unqualified name. +#' @param databaseName (optional) name of the database +#' @return a SparkDataFrame of the list of column descriptions. +#' @rdname listColumns +#' @name listColumns +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' listColumns("mytable") +#' } +#' @note since 2.2.0 +listColumns <- function(tableName, databaseName = NULL) { + sparkSession <- getSparkSession() + if (!is.null(databaseName) && class(databaseName) != "character") { + stop("databaseName must be a string.") + } + catalog <- callJMethod(sparkSession, "catalog") + jdst <- if (is.null(databaseName)) { + handledCallJMethod(catalog, "listColumns", tableName) + } else { + handledCallJMethod(catalog, "listColumns", databaseName, tableName) + } + dataFrame(callJMethod(jdst, "toDF")) +} + +#' Returns a list of functions registered in the specified database +#' +#' Returns a list of functions registered in the specified database. +#' This includes all temporary functions. +#' +#' @param databaseName (optional) name of the database +#' @return a SparkDataFrame of the list of function descriptions. +#' @rdname listFunctions +#' @name listFunctions +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' listFunctions() +#' } +#' @note since 2.2.0 +listFunctions <- function(databaseName = NULL) { + sparkSession <- getSparkSession() + if (!is.null(databaseName) && class(databaseName) != "character") { + stop("databaseName must be a string.") + } + catalog <- callJMethod(sparkSession, "catalog") + jdst <- if (is.null(databaseName)) { + callJMethod(catalog, "listFunctions") + } else { + handledCallJMethod(catalog, "listFunctions", databaseName) + } + dataFrame(callJMethod(jdst, "toDF")) +} + +#' Recovers all the partitions in the directory of a table and update the catalog +#' +#' Recovers all the partitions in the directory of a table and update the catalog. The name should +#' reference a partitioned table, and not a view. +#' +#' @param tableName the qualified or unqualified name that designates a table. If no database +#' identifier is provided, it refers to a table in the current database. +#' @rdname recoverPartitions +#' @name recoverPartitions +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' recoverPartitions("myTable") +#' } +#' @note since 2.2.0 +recoverPartitions <- function(tableName) { + sparkSession <- getSparkSession() + catalog <- callJMethod(sparkSession, "catalog") + invisible(handledCallJMethod(catalog, "recoverPartitions", tableName)) +} + +#' Invalidates and refreshes all the cached data and metadata of the given table +#' +#' Invalidates and refreshes all the cached data and metadata of the given table. For performance +#' reasons, Spark SQL or the external data source library it uses might cache certain metadata about +#' a table, such as the location of blocks. When those change outside of Spark SQL, users should +#' call this function to invalidate the cache. +#' +#' If this table is cached as an InMemoryRelation, drop the original cached version and make the +#' new version cached lazily. +#' +#' @param tableName the qualified or unqualified name that designates a table. If no database +#' identifier is provided, it refers to a table in the current database. +#' @rdname refreshTable +#' @name refreshTable +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' refreshTable("myTable") +#' } +#' @note since 2.2.0 +refreshTable <- function(tableName) { + sparkSession <- getSparkSession() + catalog <- callJMethod(sparkSession, "catalog") + invisible(handledCallJMethod(catalog, "refreshTable", tableName)) +} + +#' Invalidates and refreshes all the cached data and metadata for SparkDataFrame containing path +#' +#' Invalidates and refreshes all the cached data (and the associated metadata) for any +#' SparkDataFrame that contains the given data source path. Path matching is by prefix, i.e. "/" +#' would invalidate everything that is cached. +#' +#' @param path the path of the data source. +#' @rdname refreshByPath +#' @name refreshByPath +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' refreshByPath("/path") +#' } +#' @note since 2.2.0 +refreshByPath <- function(path) { + sparkSession <- getSparkSession() + catalog <- callJMethod(sparkSession, "catalog") + invisible(handledCallJMethod(catalog, "refreshByPath", path)) +} diff --git a/R/pkg/R/client.R b/R/pkg/R/client.R index c811d1dac3bd..9d82814211bc 100644 --- a/R/pkg/R/client.R +++ b/R/pkg/R/client.R @@ -19,7 +19,7 @@ # Creates a SparkR client connection object # if one doesn't already exist -connectBackend <- function(hostname, port, timeout = 6000) { +connectBackend <- function(hostname, port, timeout) { if (exists(".sparkRcon", envir = .sparkREnv)) { if (isOpen(.sparkREnv[[".sparkRCon"]])) { cat("SparkRBackend client connection already exists\n") @@ -38,18 +38,22 @@ determineSparkSubmitBin <- function() { if (.Platform$OS.type == "unix") { sparkSubmitBinName <- "spark-submit" } else { - sparkSubmitBinName <- "spark-submit.cmd" + sparkSubmitBinName <- "spark-submit2.cmd" } sparkSubmitBinName } generateSparkSubmitArgs <- function(args, sparkHome, jars, sparkSubmitOpts, packages) { + jars <- paste0(jars, collapse = ",") if (jars != "") { - jars <- paste("--jars", jars) + # construct the jars argument with a space between --jars and comma-separated values + jars <- paste0("--jars ", jars) } - if (!identical(packages, "")) { - packages <- paste("--packages", packages) + packages <- paste0(packages, collapse = ",") + if (packages != "") { + # construct the packages argument with a space between --packages and comma-separated values + packages <- paste0("--packages ", packages) } combinedArgs <- paste(jars, packages, sparkSubmitOpts, args, sep = " ") @@ -65,5 +69,5 @@ launchBackend <- function(args, sparkHome, jars, sparkSubmitOpts, packages) { } combinedArgs <- generateSparkSubmitArgs(args, sparkHome, jars, sparkSubmitOpts, packages) cat("Launching java with spark-submit command", sparkSubmitBin, combinedArgs, "\n") - invisible(system2(sparkSubmitBin, combinedArgs, wait = F)) + invisible(launchScript(sparkSubmitBin, combinedArgs)) } diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index 20de3907b7dd..a5c2ea81f249 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -22,20 +22,31 @@ NULL setOldClass("jobj") -#' @title S4 class that represents a DataFrame column -#' @description The column class supports unary, binary operations on DataFrame columns +#' S4 class that represents a SparkDataFrame column +#' +#' The column class supports unary, binary operations on SparkDataFrame columns +#' #' @rdname column #' -#' @slot jc reference to JVM DataFrame column +#' @slot jc reference to JVM SparkDataFrame column #' @export +#' @note Column since 1.4.0 setClass("Column", slots = list(jc = "jobj")) +#' A set of operations working with SparkDataFrame columns +#' @rdname columnfunctions +#' @name columnfunctions +NULL + setMethod("initialize", "Column", function(.Object, jc) { .Object@jc <- jc .Object }) +#' @rdname column +#' @name column +#' @aliases column,jobj-method setMethod("column", signature(x = "jobj"), function(x) { @@ -44,6 +55,9 @@ setMethod("column", #' @rdname show #' @name show +#' @aliases show,Column-method +#' @export +#' @note show(Column) since 1.4.0 setMethod("show", "Column", function(object) { cat("Column", callJMethod(object@jc, "toString"), "\n") @@ -53,11 +67,10 @@ operators <- list( "+" = "plus", "-" = "minus", "*" = "multiply", "/" = "divide", "%%" = "mod", "==" = "equalTo", ">" = "gt", "<" = "lt", "!=" = "notEqual", "<=" = "leq", ">=" = "geq", # we can not override `&&` and `||`, so use `&` and `|` instead - "&" = "and", "|" = "or", #, "!" = "unary_$bang" - "^" = "pow" + "&" = "and", "|" = "or", "^" = "pow" ) -column_functions1 <- c("asc", "desc", "isNull", "isNotNull") -column_functions2 <- c("like", "rlike", "startsWith", "endsWith", "getField", "getItem", "contains") +column_functions1 <- c("asc", "desc", "isNaN", "isNull", "isNotNull") +column_functions2 <- c("like", "rlike", "getField", "getItem", "contains") createOperator <- function(op) { setMethod(op, @@ -117,14 +130,20 @@ createMethods <- function() { createMethods() -#' alias -#' -#' Set a new name for a column -#' #' @rdname alias #' @name alias +#' @aliases alias,Column-method #' @family colum_func #' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame(iris) +#' +#' head(select( +#' df, alias(df$Sepal_Length, "slength"), alias(df$Petal_Length, "plength") +#' )) +#' } +#' @note alias(Column) since 1.4.0 setMethod("alias", signature(object = "Column"), function(object, data) { @@ -142,15 +161,56 @@ setMethod("alias", #' @rdname substr #' @name substr #' @family colum_func +#' @aliases substr,Column-method #' -#' @param start starting position -#' @param stop ending position +#' @param x a Column. +#' @param start starting position. +#' @param stop ending position. +#' @note substr since 1.4.0 setMethod("substr", signature(x = "Column"), function(x, start, stop) { jc <- callJMethod(x@jc, "substr", as.integer(start - 1), as.integer(stop - start + 1)) column(jc) }) +#' startsWith +#' +#' Determines if entries of x start with string (entries of) prefix respectively, +#' where strings are recycled to common lengths. +#' +#' @rdname startsWith +#' @name startsWith +#' @family colum_func +#' @aliases startsWith,Column-method +#' +#' @param x vector of character string whose "starts" are considered +#' @param prefix character vector (often of length one) +#' @note startsWith since 1.4.0 +setMethod("startsWith", signature(x = "Column"), + function(x, prefix) { + jc <- callJMethod(x@jc, "startsWith", as.vector(prefix)) + column(jc) + }) + +#' endsWith +#' +#' Determines if entries of x end with string (entries of) suffix respectively, +#' where strings are recycled to common lengths. +#' +#' @rdname endsWith +#' @name endsWith +#' @family colum_func +#' @aliases endsWith,Column-method +#' +#' @param x vector of character string whose "ends" are considered +#' @param suffix character vector (often of length one) +#' @note endsWith since 1.4.0 +setMethod("endsWith", signature(x = "Column"), + function(x, suffix) { + jc <- callJMethod(x@jc, "endsWith", as.vector(suffix)) + column(jc) + }) + #' between #' #' Test if the column is between the lower bound and upper bound, inclusive. @@ -158,8 +218,11 @@ setMethod("substr", signature(x = "Column"), #' @rdname between #' @name between #' @family colum_func +#' @aliases between,Column-method #' +#' @param x a Column #' @param bounds lower and upper bounds +#' @note between since 1.5.0 setMethod("between", signature(x = "Column"), function(x, bounds) { if (is.vector(bounds) && length(bounds) == 2) { @@ -172,60 +235,123 @@ setMethod("between", signature(x = "Column"), #' Casts the column to a different data type. #' +#' @param x a Column. +#' @param dataType a character object describing the target data type. +#' See +#' \href{https://spark.apache.org/docs/latest/sparkr.html#data-type-mapping-between-r-and-spark}{ +#' Spark Data Types} for available data types. #' @rdname cast #' @name cast #' @family colum_func +#' @aliases cast,Column-method #' -#' @examples \dontrun{ +#' @examples +#' \dontrun{ #' cast(df$age, "string") -#' cast(df$name, list(type="array", elementType="byte", containsNull = TRUE)) #' } +#' @note cast since 1.4.0 setMethod("cast", signature(x = "Column"), function(x, dataType) { if (is.character(dataType)) { column(callJMethod(x@jc, "cast", dataType)) - } else if (is.list(dataType)) { - json <- tojson(dataType) - jdataType <- callJStatic("org.apache.spark.sql.types.DataType", "fromJson", json) - column(callJMethod(x@jc, "cast", jdataType)) } else { - stop("dataType should be character or list") + stop("dataType should be character") } }) #' Match a column with given values. #' +#' @param x a Column. +#' @param table a collection of values (coercible to list) to compare with. #' @rdname match #' @name %in% -#' @aliases %in% -#' @return a matched values as a result of comparing with given values. +#' @aliases %in%,Column-method +#' @return A matched values as a result of comparing with given values. #' @export #' @examples #' \dontrun{ #' filter(df, "age in (10, 30)") #' where(df, df$age %in% c(10, 30)) #' } +#' @note \%in\% since 1.5.0 setMethod("%in%", signature(x = "Column"), function(x, table) { - jc <- callJMethod(x@jc, "in", as.list(table)) + jc <- callJMethod(x@jc, "isin", as.list(table)) return(column(jc)) }) #' otherwise #' -#' If values in the specified column are null, returns the value. -#' Can be used in conjunction with `when` to specify a default value for expressions. +#' If values in the specified column are null, returns the value. +#' Can be used in conjunction with \code{when} to specify a default value for expressions. #' +#' @param x a Column. +#' @param value value to replace when the corresponding entry in \code{x} is NA. +#' Can be a single value or a Column. #' @rdname otherwise #' @name otherwise #' @family colum_func +#' @aliases otherwise,Column-method #' @export +#' @note otherwise since 1.5.0 setMethod("otherwise", signature(x = "Column", value = "ANY"), function(x, value) { - value <- ifelse(class(value) == "Column", value@jc, value) + value <- if (class(value) == "Column") { value@jc } else { value } jc <- callJMethod(x@jc, "otherwise", value) column(jc) }) + +#' \%<=>\% +#' +#' Equality test that is safe for null values. +#' +#' Can be used, unlike standard equality operator, to perform null-safe joins. +#' Equivalent to Scala \code{Column.<=>} and \code{Column.eqNullSafe}. +#' +#' @param x a Column +#' @param value a value to compare +#' @rdname eq_null_safe +#' @name %<=>% +#' @aliases %<=>%,Column-method +#' @export +#' @examples +#' \dontrun{ +#' df1 <- createDataFrame(data.frame( +#' x = c(1, NA, 3, NA), y = c(2, 6, 3, NA) +#' )) +#' +#' head(select(df1, df1$x == df1$y, df1$x %<=>% df1$y)) +#' +#' df2 <- createDataFrame(data.frame(y = c(3, NA))) +#' count(join(df1, df2, df1$y == df2$y)) +#' +#' count(join(df1, df2, df1$y %<=>% df2$y)) +#' } +#' @note \%<=>\% since 2.3.0 +setMethod("%<=>%", + signature(x = "Column", value = "ANY"), + function(x, value) { + value <- if (class(value) == "Column") { value@jc } else { value } + jc <- callJMethod(x@jc, "eqNullSafe", value) + column(jc) + }) + +#' ! +#' +#' Inversion of boolean expression. +#' +#' @rdname not +#' @name not +#' @aliases !,Column-method +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame(data.frame(x = c(-1, 0, 1))) +#' +#' head(select(df, !column("x") > 0)) +#' } +#' @note ! since 2.3.0 +setMethod("!", signature(x = "Column"), function(x) not(x)) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 720990e1c608..8349b57a30a9 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -25,23 +25,23 @@ getMinPartitions <- function(sc, minPartitions) { as.integer(minPartitions) } -# Create an RDD from a text file. -# -# This function reads a text file from HDFS, a local file system (available on all -# nodes), or any Hadoop-supported file system URI, and creates an -# RDD of strings from it. -# -# @param sc SparkContext to use -# @param path Path of file to read. A vector of multiple paths is allowed. -# @param minPartitions Minimum number of partitions to be created. If NULL, the default -# value is chosen based on available parallelism. -# @return RDD where each item is of type \code{character} -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# lines <- textFile(sc, "myfile.txt") -#} +#' Create an RDD from a text file. +#' +#' This function reads a text file from HDFS, a local file system (available on all +#' nodes), or any Hadoop-supported file system URI, and creates an +#' RDD of strings from it. +#' +#' @param sc SparkContext to use +#' @param path Path of file to read. A vector of multiple paths is allowed. +#' @param minPartitions Minimum number of partitions to be created. If NULL, the default +#' value is chosen based on available parallelism. +#' @return RDD where each item is of type \code{character} +#' @noRd +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' lines <- textFile(sc, "myfile.txt") +#'} textFile <- function(sc, path, minPartitions = NULL) { # Allow the user to have a more flexible definiton of the text file path path <- suppressWarnings(normalizePath(path)) @@ -53,23 +53,23 @@ textFile <- function(sc, path, minPartitions = NULL) { RDD(jrdd, "string") } -# Load an RDD saved as a SequenceFile containing serialized objects. -# -# The file to be loaded should be one that was previously generated by calling -# saveAsObjectFile() of the RDD class. -# -# @param sc SparkContext to use -# @param path Path of file to read. A vector of multiple paths is allowed. -# @param minPartitions Minimum number of partitions to be created. If NULL, the default -# value is chosen based on available parallelism. -# @return RDD containing serialized R objects. -# @seealso saveAsObjectFile -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- objectFile(sc, "myfile") -#} +#' Load an RDD saved as a SequenceFile containing serialized objects. +#' +#' The file to be loaded should be one that was previously generated by calling +#' saveAsObjectFile() of the RDD class. +#' +#' @param sc SparkContext to use +#' @param path Path of file to read. A vector of multiple paths is allowed. +#' @param minPartitions Minimum number of partitions to be created. If NULL, the default +#' value is chosen based on available parallelism. +#' @return RDD containing serialized R objects. +#' @seealso saveAsObjectFile +#' @noRd +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- objectFile(sc, "myfile") +#'} objectFile <- function(sc, path, minPartitions = NULL) { # Allow the user to have a more flexible definiton of the text file path path <- suppressWarnings(normalizePath(path)) @@ -81,29 +81,48 @@ objectFile <- function(sc, path, minPartitions = NULL) { RDD(jrdd, "byte") } -# Create an RDD from a homogeneous list or vector. -# -# This function creates an RDD from a local homogeneous list in R. The elements -# in the list are split into \code{numSlices} slices and distributed to nodes -# in the cluster. -# -# @param sc SparkContext to use -# @param coll collection to parallelize -# @param numSlices number of partitions to create in the RDD -# @return an RDD created from this collection -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10, 2) -# # The RDD should contain 10 elements -# length(rdd) -#} +#' Create an RDD from a homogeneous list or vector. +#' +#' This function creates an RDD from a local homogeneous list in R. The elements +#' in the list are split into \code{numSlices} slices and distributed to nodes +#' in the cluster. +#' +#' If size of serialized slices is larger than spark.r.maxAllocationLimit or (200MB), the function +#' will write it to disk and send the file name to JVM. Also to make sure each slice is not +#' larger than that limit, number of slices may be increased. +#' +#' In 2.2.0 we are changing how the numSlices are used/computed to handle +#' 1 < (length(coll) / numSlices) << length(coll) better, and to get the exact number of slices. +#' This change affects both createDataFrame and spark.lapply. +#' In the specific one case that it is used to convert R native object into SparkDataFrame, it has +#' always been kept at the default of 1. In the case the object is large, we are explicitly setting +#' the parallism to numSlices (which is still 1). +#' +#' Specifically, we are changing to split positions to match the calculation in positions() of +#' ParallelCollectionRDD in Spark. +#' +#' @param sc SparkContext to use +#' @param coll collection to parallelize +#' @param numSlices number of partitions to create in the RDD +#' @return an RDD created from this collection +#' @noRd +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2) +#' # The RDD should contain 10 elements +#' length(rdd) +#'} parallelize <- function(sc, coll, numSlices = 1) { # TODO: bound/safeguard numSlices # TODO: unit tests for if the split works for all primitives # TODO: support matrix, data frame, etc + + # Note, for data.frame, createDataFrame turns it into a list before it calls here. + # nolint start + # suppress lintr warning: Place a space before left parenthesis, except in a function call. if ((!is.list(coll) && !is.vector(coll)) || is.data.frame(coll)) { + # nolint end if (is.data.frame(coll)) { message(paste("context.R: A data frame is parallelized by columns.")) } else { @@ -117,49 +136,102 @@ parallelize <- function(sc, coll, numSlices = 1) { coll <- as.list(coll) } - if (numSlices > length(coll)) - numSlices <- length(coll) + sizeLimit <- getMaxAllocationLimit(sc) + objectSize <- object.size(coll) + + # For large objects we make sure the size of each slice is also smaller than sizeLimit + numSerializedSlices <- max(numSlices, ceiling(objectSize / sizeLimit)) + if (numSerializedSlices > length(coll)) + numSerializedSlices <- length(coll) + + # Generate the slice ids to put each row + # For instance, for numSerializedSlices of 22, length of 50 + # [1] 0 0 2 2 4 4 6 6 6 9 9 11 11 13 13 15 15 15 18 18 20 20 22 22 22 + # [26] 25 25 27 27 29 29 31 31 31 34 34 36 36 38 38 40 40 40 43 43 45 45 47 47 47 + # Notice the slice group with 3 slices (ie. 6, 15, 22) are roughly evenly spaced. + # We are trying to reimplement the calculation in the positions method in ParallelCollectionRDD + splits <- if (numSerializedSlices > 0) { + unlist(lapply(0: (numSerializedSlices - 1), function(x) { + # nolint start + start <- trunc((x * length(coll)) / numSerializedSlices) + end <- trunc(((x + 1) * length(coll)) / numSerializedSlices) + # nolint end + rep(start, end - start) + })) + } else { + 1 + } - sliceLen <- ceiling(length(coll) / numSlices) - slices <- split(coll, rep(1: (numSlices + 1), each = sliceLen)[1:length(coll)]) + slices <- split(coll, splits) # Serialize each slice: obtain a list of raws, or a list of lists (slices) of # 2-tuples of raws serializedSlices <- lapply(slices, serialize, connection = NULL) - jrdd <- callJStatic("org.apache.spark.api.r.RRDD", - "createRDDFromArray", sc, serializedSlices) + # The PRC backend cannot handle arguments larger than 2GB (INT_MAX) + # If serialized data is safely less than that threshold we send it over the PRC channel. + # Otherwise, we write it to a file and send the file name + if (objectSize < sizeLimit) { + jrdd <- callJStatic("org.apache.spark.api.r.RRDD", "createRDDFromArray", sc, serializedSlices) + } else { + fileName <- writeToTempFile(serializedSlices) + jrdd <- tryCatch(callJStatic( + "org.apache.spark.api.r.RRDD", "createRDDFromFile", sc, fileName, as.integer(numSlices)), + finally = { + file.remove(fileName) + }) + } RDD(jrdd, "byte") } -# Include this specified package on all workers -# -# This function can be used to include a package on all workers before the -# user's code is executed. This is useful in scenarios where other R package -# functions are used in a function passed to functions like \code{lapply}. -# NOTE: The package is assumed to be installed on every node in the Spark -# cluster. -# -# @param sc SparkContext to use -# @param pkg Package name -# -# @export -# @examples -#\dontrun{ -# library(Matrix) -# -# sc <- sparkR.init() -# # Include the matrix library we will be using -# includePackage(sc, Matrix) -# -# generateSparse <- function(x) { -# sparseMatrix(i=c(1, 2, 3), j=c(1, 2, 3), x=c(1, 2, 3)) -# } -# -# rdd <- lapplyPartition(parallelize(sc, 1:2, 2L), generateSparse) -# collect(rdd) -#} +getMaxAllocationLimit <- function(sc) { + conf <- callJMethod(sc, "getConf") + as.numeric( + callJMethod(conf, + "get", + "spark.r.maxAllocationLimit", + toString(.Machine$integer.max / 10) # Default to a safe value: 200MB + )) +} + +writeToTempFile <- function(serializedSlices) { + fileName <- tempfile() + conn <- file(fileName, "wb") + for (slice in serializedSlices) { + writeBin(as.integer(length(slice)), conn, endian = "big") + writeBin(slice, conn, endian = "big") + } + close(conn) + fileName +} + +#' Include this specified package on all workers +#' +#' This function can be used to include a package on all workers before the +#' user's code is executed. This is useful in scenarios where other R package +#' functions are used in a function passed to functions like \code{lapply}. +#' NOTE: The package is assumed to be installed on every node in the Spark +#' cluster. +#' +#' @param sc SparkContext to use +#' @param pkg Package name +#' @noRd +#' @examples +#'\dontrun{ +#' library(Matrix) +#' +#' sc <- sparkR.init() +#' # Include the matrix library we will be using +#' includePackage(sc, Matrix) +#' +#' generateSparse <- function(x) { +#' sparseMatrix(i=c(1, 2, 3), j=c(1, 2, 3), x=c(1, 2, 3)) +#' } +#' +#' rdd <- lapplyPartition(parallelize(sc, 1:2, 2L), generateSparse) +#' collect(rdd) +#'} includePackage <- function(sc, pkg) { pkg <- as.character(substitute(pkg)) if (exists(".packages", .sparkREnv)) { @@ -171,31 +243,30 @@ includePackage <- function(sc, pkg) { .sparkREnv$.packages <- packages } -# @title Broadcast a variable to all workers -# -# @description -# Broadcast a read-only variable to the cluster, returning a \code{Broadcast} -# object for reading it in distributed functions. -# -# @param sc Spark Context to use -# @param object Object to be broadcast -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:2, 2L) -# -# # Large Matrix object that we want to broadcast -# randomMat <- matrix(nrow=100, ncol=10, data=rnorm(1000)) -# randomMatBr <- broadcast(sc, randomMat) -# -# # Use the broadcast variable inside the function -# useBroadcast <- function(x) { -# sum(value(randomMatBr) * x) -# } -# sumRDD <- lapply(rdd, useBroadcast) -#} -broadcast <- function(sc, object) { +#' Broadcast a variable to all workers +#' +#' Broadcast a read-only variable to the cluster, returning a \code{Broadcast} +#' object for reading it in distributed functions. +#' +#' @param sc Spark Context to use +#' @param object Object to be broadcast +#' @noRd +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:2, 2L) +#' +#' # Large Matrix object that we want to broadcast +#' randomMat <- matrix(nrow=100, ncol=10, data=rnorm(1000)) +#' randomMatBr <- broadcastRDD(sc, randomMat) +#' +#' # Use the broadcast variable inside the function +#' useBroadcast <- function(x) { +#' sum(value(randomMatBr) * x) +#' } +#' sumRDD <- lapply(rdd, useBroadcast) +#'} +broadcastRDD <- function(sc, object) { objName <- as.character(substitute(object)) serializedObj <- serialize(object, connection = NULL) @@ -205,21 +276,168 @@ broadcast <- function(sc, object) { Broadcast(id, object, jBroadcast, objName) } -# @title Set the checkpoint directory -# -# Set the directory under which RDDs are going to be checkpointed. The -# directory must be a HDFS path if running on a cluster. -# -# @param sc Spark Context to use -# @param dirName Directory path -# @export -# @examples -#\dontrun{ -# sc <- sparkR.init() -# setCheckpointDir(sc, "~/checkpoint") -# rdd <- parallelize(sc, 1:2, 2L) -# checkpoint(rdd) -#} -setCheckpointDir <- function(sc, dirName) { +#' Set the checkpoint directory +#' +#' Set the directory under which RDDs are going to be checkpointed. The +#' directory must be a HDFS path if running on a cluster. +#' +#' @param sc Spark Context to use +#' @param dirName Directory path +#' @noRd +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' setCheckpointDir(sc, "~/checkpoint") +#' rdd <- parallelize(sc, 1:2, 2L) +#' checkpoint(rdd) +#'} +setCheckpointDirSC <- function(sc, dirName) { invisible(callJMethod(sc, "setCheckpointDir", suppressWarnings(normalizePath(dirName)))) } + +#' Add a file or directory to be downloaded with this Spark job on every node. +#' +#' The path passed can be either a local file, a file in HDFS (or other Hadoop-supported +#' filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs, +#' use spark.getSparkFiles(fileName) to find its download location. +#' +#' A directory can be given if the recursive option is set to true. +#' Currently directories are only supported for Hadoop-supported filesystems. +#' Refer Hadoop-supported filesystems at \url{https://wiki.apache.org/hadoop/HCFS}. +#' +#' @rdname spark.addFile +#' @param path The path of the file to be added +#' @param recursive Whether to add files recursively from the path. Default is FALSE. +#' @export +#' @examples +#'\dontrun{ +#' spark.addFile("~/myfile") +#'} +#' @note spark.addFile since 2.1.0 +spark.addFile <- function(path, recursive = FALSE) { + sc <- getSparkContext() + invisible(callJMethod(sc, "addFile", suppressWarnings(normalizePath(path)), recursive)) +} + +#' Get the root directory that contains files added through spark.addFile. +#' +#' @rdname spark.getSparkFilesRootDirectory +#' @return the root directory that contains files added through spark.addFile +#' @export +#' @examples +#'\dontrun{ +#' spark.getSparkFilesRootDirectory() +#'} +#' @note spark.getSparkFilesRootDirectory since 2.1.0 +spark.getSparkFilesRootDirectory <- function() { + if (Sys.getenv("SPARKR_IS_RUNNING_ON_WORKER") == "") { + # Running on driver. + callJStatic("org.apache.spark.SparkFiles", "getRootDirectory") + } else { + # Running on worker. + Sys.getenv("SPARKR_SPARKFILES_ROOT_DIR") + } +} + +#' Get the absolute path of a file added through spark.addFile. +#' +#' @rdname spark.getSparkFiles +#' @param fileName The name of the file added through spark.addFile +#' @return the absolute path of a file added through spark.addFile. +#' @export +#' @examples +#'\dontrun{ +#' spark.getSparkFiles("myfile") +#'} +#' @note spark.getSparkFiles since 2.1.0 +spark.getSparkFiles <- function(fileName) { + if (Sys.getenv("SPARKR_IS_RUNNING_ON_WORKER") == "") { + # Running on driver. + callJStatic("org.apache.spark.SparkFiles", "get", as.character(fileName)) + } else { + # Running on worker. + file.path(spark.getSparkFilesRootDirectory(), as.character(fileName)) + } +} + +#' Run a function over a list of elements, distributing the computations with Spark +#' +#' Run a function over a list of elements, distributing the computations with Spark. Applies a +#' function in a manner that is similar to doParallel or lapply to elements of a list. +#' The computations are distributed using Spark. It is conceptually the same as the following code: +#' lapply(list, func) +#' +#' Known limitations: +#' \itemize{ +#' \item variable scoping and capture: compared to R's rich support for variable resolutions, +#' the distributed nature of SparkR limits how variables are resolved at runtime. All the +#' variables that are available through lexical scoping are embedded in the closure of the +#' function and available as read-only variables within the function. The environment variables +#' should be stored into temporary variables outside the function, and not directly accessed +#' within the function. +#' +#' \item loading external packages: In order to use a package, you need to load it inside the +#' closure. For example, if you rely on the MASS module, here is how you would use it: +#' \preformatted{ +#' train <- function(hyperparam) { +#' library(MASS) +#' lm.ridge("y ~ x+z", data, lambda=hyperparam) +#' model +#' } +#' } +#' } +#' +#' @rdname spark.lapply +#' @param list the list of elements +#' @param func a function that takes one argument. +#' @return a list of results (the exact type being determined by the function) +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' doubled <- spark.lapply(1:10, function(x){2 * x}) +#'} +#' @note spark.lapply since 2.0.0 +spark.lapply <- function(list, func) { + sc <- getSparkContext() + rdd <- parallelize(sc, list, length(list)) + results <- map(rdd, func) + local <- collectRDD(results) + local +} + +#' Set new log level +#' +#' Set new log level: "ALL", "DEBUG", "ERROR", "FATAL", "INFO", "OFF", "TRACE", "WARN" +#' +#' @rdname setLogLevel +#' @param level New log level +#' @export +#' @examples +#'\dontrun{ +#' setLogLevel("ERROR") +#'} +#' @note setLogLevel since 2.0.0 +setLogLevel <- function(level) { + sc <- getSparkContext() + invisible(callJMethod(sc, "setLogLevel", level)) +} + +#' Set checkpoint directory +#' +#' Set the directory under which SparkDataFrame are going to be checkpointed. The directory must be +#' a HDFS path if running on a cluster. +#' +#' @rdname setCheckpointDir +#' @param directory Directory path to checkpoint to +#' @seealso \link{checkpoint} +#' @export +#' @examples +#'\dontrun{ +#' setCheckpointDir("/checkpoint") +#'} +#' @note setCheckpointDir since 2.2.0 +setCheckpointDir <- function(directory) { + sc <- getSparkContext() + invisible(callJMethod(sc, "setCheckpointDir", suppressWarnings(normalizePath(directory)))) +} diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index f7e56e43016e..0e99b171cabe 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -17,6 +17,7 @@ # Utility functions to deserialize objects from Java. +# nolint start # Type mapping from Java to R # # void -> NULL @@ -32,6 +33,8 @@ # # Array[T] -> list() # Object -> jobj +# +# nolint end readObject <- function(con) { # Read type first @@ -136,7 +139,7 @@ readEnv <- function(con) { env } -# Read a field of StructType from DataFrame +# Read a field of StructType from SparkDataFrame # into a named list in R whose class is "struct" readStruct <- function(con) { names <- readObject(con) @@ -183,7 +186,7 @@ readMultipleObjects <- function(inputCon) { # of the objects, so the number of objects varies, we try to read # all objects in a loop until the end of the stream. data <- list() - while(TRUE) { + while (TRUE) { # If reaching the end of the stream, type returned should be "". type <- readType(inputCon) if (type == "") { @@ -194,6 +197,36 @@ readMultipleObjects <- function(inputCon) { data # this is a list of named lists now } +readMultipleObjectsWithKeys <- function(inputCon) { + # readMultipleObjectsWithKeys will read multiple continuous objects from + # a DataOutputStream. There is no preceding field telling the count + # of the objects, so the number of objects varies, we try to read + # all objects in a loop until the end of the stream. This function + # is for use by gapply. Each group of rows is followed by the grouping + # key for this group which is then followed by next group. + keys <- list() + data <- list() + subData <- list() + while (TRUE) { + # If reaching the end of the stream, type returned should be "". + type <- readType(inputCon) + if (type == "") { + break + } else if (type == "r") { + type <- readType(inputCon) + # A grouping boundary detected + key <- readTypedObject(inputCon, type) + index <- length(data) + 1L + data[[index]] <- subData + keys[[index]] <- key + subData <- list() + } else { + subData[[length(subData) + 1L]] <- readTypedObject(inputCon, type) + } + } + list(keys = keys, data = data) # this is a list of keys and corresponding data +} + readRowList <- function(obj) { # readRowList is meant for use inside an lapply. As a result, it is # necessary to open a standalone connection for the row and consume diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index d7fd27927913..9f286263c216 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -18,38 +18,249 @@ #' @include generics.R column.R NULL -#' lit +#' Aggregate functions for Column operations #' -#' A new \linkS4class{Column} is created to represent the literal value. -#' If the parameter is a \linkS4class{Column}, it is returned unchanged. +#' Aggregate functions defined for \code{Column}. #' -#' @family normal_funcs -#' @rdname lit -#' @name lit +#' @param x Column to compute on. +#' @param y,na.rm,use currently not used. +#' @param ... additional argument(s). For example, it could be used to pass additional Columns. +#' @name column_aggregate_functions +#' @rdname column_aggregate_functions +#' @family aggregate functions +#' @examples +#' \dontrun{ +#' # Dataframe used throughout this doc +#' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars))} +NULL + +#' Date time functions for Column operations +#' +#' Date time functions defined for \code{Column}. +#' +#' @param x Column to compute on. In \code{window}, it must be a time Column of \code{TimestampType}. +#' @param format For \code{to_date} and \code{to_timestamp}, it is the string to use to parse +#' Column \code{x} to DateType or TimestampType. For \code{trunc}, it is the string +#' to use to specify the truncation method. For example, "year", "yyyy", "yy" for +#' truncate by year, or "month", "mon", "mm" for truncate by month. +#' @param ... additional argument(s). +#' @name column_datetime_functions +#' @rdname column_datetime_functions +#' @family data time functions +#' @examples +#' \dontrun{ +#' dts <- c("2005-01-02 18:47:22", +#' "2005-12-24 16:30:58", +#' "2005-10-28 07:30:05", +#' "2005-12-28 07:01:05", +#' "2006-01-24 00:01:10") +#' y <- c(2.0, 2.2, 3.4, 2.5, 1.8) +#' df <- createDataFrame(data.frame(time = as.POSIXct(dts), y = y))} +NULL + +#' Date time arithmetic functions for Column operations +#' +#' Date time arithmetic functions defined for \code{Column}. +#' +#' @param y Column to compute on. +#' @param x For class \code{Column}, it is the column used to perform arithmetic operations +#' with column \code{y}. For class \code{numeric}, it is the number of months or +#' days to be added to or subtracted from \code{y}. For class \code{character}, it is +#' \itemize{ +#' \item \code{date_format}: date format specification. +#' \item \code{from_utc_timestamp}, \code{to_utc_timestamp}: time zone to use. +#' \item \code{next_day}: day of the week string. +#' } +#' +#' @name column_datetime_diff_functions +#' @rdname column_datetime_diff_functions +#' @family data time functions +#' @examples +#' \dontrun{ +#' dts <- c("2005-01-02 18:47:22", +#' "2005-12-24 16:30:58", +#' "2005-10-28 07:30:05", +#' "2005-12-28 07:01:05", +#' "2006-01-24 00:01:10") +#' y <- c(2.0, 2.2, 3.4, 2.5, 1.8) +#' df <- createDataFrame(data.frame(time = as.POSIXct(dts), y = y))} +NULL + +#' Math functions for Column operations +#' +#' Math functions defined for \code{Column}. +#' +#' @param x Column to compute on. In \code{shiftLeft}, \code{shiftRight} and \code{shiftRightUnsigned}, +#' this is the number of bits to shift. +#' @param y Column to compute on. +#' @param ... additional argument(s). +#' @name column_math_functions +#' @rdname column_math_functions +#' @family math functions +#' @examples +#' \dontrun{ +#' # Dataframe used throughout this doc +#' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars)) +#' tmp <- mutate(df, v1 = log(df$mpg), v2 = cbrt(df$disp), +#' v3 = bround(df$wt, 1), v4 = bin(df$cyl), +#' v5 = hex(df$wt), v6 = toDegrees(df$gear), +#' v7 = atan2(df$cyl, df$am), v8 = hypot(df$cyl, df$am), +#' v9 = pmod(df$hp, df$cyl), v10 = shiftLeft(df$disp, 1), +#' v11 = conv(df$hp, 10, 16), v12 = sign(df$vs - 0.5), +#' v13 = sqrt(df$disp), v14 = ceil(df$wt)) +#' head(tmp)} +NULL + +#' String functions for Column operations +#' +#' String functions defined for \code{Column}. +#' +#' @param x Column to compute on except in the following methods: +#' \itemize{ +#' \item \code{instr}: \code{character}, the substring to check. See 'Details'. +#' \item \code{format_number}: \code{numeric}, the number of decimal place to +#' format to. See 'Details'. +#' } +#' @param y Column to compute on. +#' @param ... additional Columns. +#' @name column_string_functions +#' @rdname column_string_functions +#' @family string functions +#' @examples +#' \dontrun{ +#' # Dataframe used throughout this doc +#' df <- createDataFrame(as.data.frame(Titanic, stringsAsFactors = FALSE))} +NULL + +#' Non-aggregate functions for Column operations +#' +#' Non-aggregate functions defined for \code{Column}. +#' +#' @param x Column to compute on. In \code{lit}, it is a literal value or a Column. +#' In \code{expr}, it contains an expression character object to be parsed. +#' @param y Column to compute on. +#' @param ... additional Columns. +#' @name column_nonaggregate_functions +#' @rdname column_nonaggregate_functions +#' @seealso coalesce,SparkDataFrame-method +#' @family non-aggregate functions +#' @examples +#' \dontrun{ +#' # Dataframe used throughout this doc +#' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars))} +NULL + +#' Miscellaneous functions for Column operations +#' +#' Miscellaneous functions defined for \code{Column}. +#' +#' @param x Column to compute on. In \code{sha2}, it is one of 224, 256, 384, or 512. +#' @param y Column to compute on. +#' @param ... additional Columns. +#' @name column_misc_functions +#' @rdname column_misc_functions +#' @family misc functions +#' @examples +#' \dontrun{ +#' # Dataframe used throughout this doc +#' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars)[, 1:2]) +#' tmp <- mutate(df, v1 = crc32(df$model), v2 = hash(df$model), +#' v3 = hash(df$model, df$mpg), v4 = md5(df$model), +#' v5 = sha1(df$model), v6 = sha2(df$model, 256)) +#' head(tmp)} +NULL + +#' Collection functions for Column operations +#' +#' Collection functions defined for \code{Column}. +#' +#' @param x Column to compute on. Note the difference in the following methods: +#' \itemize{ +#' \item \code{to_json}: it is the column containing the struct, array of the structs, +#' the map or array of maps. +#' \item \code{from_json}: it is the column containing the JSON string. +#' } +#' @param ... additional argument(s). In \code{to_json} and \code{from_json}, this contains +#' additional named properties to control how it is converted, accepts the same +#' options as the JSON data source. +#' @name column_collection_functions +#' @rdname column_collection_functions +#' @family collection functions +#' @examples +#' \dontrun{ +#' # Dataframe used throughout this doc +#' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars)) +#' tmp <- mutate(df, v1 = create_array(df$mpg, df$cyl, df$hp)) +#' head(select(tmp, array_contains(tmp$v1, 21), size(tmp$v1))) +#' tmp2 <- mutate(tmp, v2 = explode(tmp$v1)) +#' head(tmp2) +#' head(select(tmp, posexplode(tmp$v1))) +#' head(select(tmp, sort_array(tmp$v1))) +#' head(select(tmp, sort_array(tmp$v1, asc = FALSE))) +#' tmp3 <- mutate(df, v3 = create_map(df$model, df$cyl)) +#' head(select(tmp3, map_keys(tmp3$v3))) +#' head(select(tmp3, map_values(tmp3$v3)))} +NULL + +#' Window functions for Column operations +#' +#' Window functions defined for \code{Column}. +#' +#' @param x In \code{lag} and \code{lead}, it is the column as a character string or a Column +#' to compute on. In \code{ntile}, it is the number of ntile groups. +#' @param offset In \code{lag}, the number of rows back from the current row from which to obtain +#' a value. In \code{lead}, the number of rows after the current row from which to +#' obtain a value. If not specified, the default is 1. +#' @param defaultValue (optional) default to use when the offset row does not exist. +#' @param ... additional argument(s). +#' @name column_window_functions +#' @rdname column_window_functions +#' @family window functions +#' @examples +#' \dontrun{ +#' # Dataframe used throughout this doc +#' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars)) +#' ws <- orderBy(windowPartitionBy("am"), "hp") +#' tmp <- mutate(df, dist = over(cume_dist(), ws), dense_rank = over(dense_rank(), ws), +#' lag = over(lag(df$mpg), ws), lead = over(lead(df$mpg, 1), ws), +#' percent_rank = over(percent_rank(), ws), +#' rank = over(rank(), ws), row_number = over(row_number(), ws)) +#' # Get ntile group id (1-4) for hp +#' tmp <- mutate(tmp, ntile = over(ntile(4), ws)) +#' head(tmp)} +NULL + +#' @details +#' \code{lit}: A new Column is created to represent the literal value. +#' If the parameter is a Column, it is returned unchanged. +#' +#' @rdname column_nonaggregate_functions #' @export +#' @aliases lit lit,ANY-method #' @examples +#' #' \dontrun{ -#' lit(df$name) -#' select(df, lit("x")) -#' select(df, lit("2015-01-01")) -#'} +#' tmp <- mutate(df, v1 = lit(df$mpg), v2 = lit("x"), v3 = lit("2015-01-01"), +#' v4 = negate(df$mpg), v5 = expr('length(model)'), +#' v6 = greatest(df$vs, df$am), v7 = least(df$vs, df$am), +#' v8 = column("mpg")) +#' head(tmp)} +#' @note lit since 1.5.0 setMethod("lit", signature("ANY"), function(x) { jc <- callJStatic("org.apache.spark.sql.functions", "lit", - ifelse(class(x) == "Column", x@jc, x)) + if (class(x) == "Column") { x@jc } else { x }) column(jc) }) -#' abs -#' -#' Computes the absolute value. +#' @details +#' \code{abs}: Computes the absolute value. #' -#' @rdname abs -#' @name abs -#' @family normal_funcs +#' @rdname column_math_functions #' @export -#' @examples \dontrun{abs(df$c)} +#' @aliases abs abs,Column-method +#' @note abs since 1.5.0 setMethod("abs", signature(x = "Column"), function(x) { @@ -57,16 +268,14 @@ setMethod("abs", column(jc) }) -#' acos +#' @details +#' \code{acos}: Computes the cosine inverse of the given value; the returned angle is in +#' the range 0.0 through pi. #' -#' Computes the cosine inverse of the given value; the returned angle is in the range -#' 0.0 through pi. -#' -#' @rdname acos -#' @name acos -#' @family math_funcs +#' @rdname column_math_functions #' @export -#' @examples \dontrun{acos(df$c)} +#' @aliases acos acos,Column-method +#' @note acos since 1.5.0 setMethod("acos", signature(x = "Column"), function(x) { @@ -74,15 +283,21 @@ setMethod("acos", column(jc) }) -#' approxCountDistinct -#' -#' Aggregate function: returns the approximate number of distinct items in a group. +#' @details +#' \code{approxCountDistinct}: Returns the approximate number of distinct items in a group. #' -#' @rdname approxCountDistinct -#' @name approxCountDistinct -#' @family agg_funcs +#' @rdname column_aggregate_functions #' @export -#' @examples \dontrun{approxCountDistinct(df$c)} +#' @aliases approxCountDistinct approxCountDistinct,Column-method +#' @examples +#' +#' \dontrun{ +#' head(select(df, approxCountDistinct(df$gear))) +#' head(select(df, approxCountDistinct(df$gear, 0.02))) +#' head(select(df, countDistinct(df$gear, df$cyl))) +#' head(select(df, n_distinct(df$gear))) +#' head(distinct(select(df, "gear")))} +#' @note approxCountDistinct(Column) since 1.4.0 setMethod("approxCountDistinct", signature(x = "Column"), function(x) { @@ -90,16 +305,18 @@ setMethod("approxCountDistinct", column(jc) }) -#' ascii -#' -#' Computes the numeric value of the first character of the string column, and returns the -#' result as a int column. +#' @details +#' \code{ascii}: Computes the numeric value of the first character of the string column, +#' and returns the result as an int column. #' -#' @rdname ascii -#' @name ascii -#' @family string_funcs +#' @rdname column_string_functions #' @export -#' @examples \dontrun{\dontrun{ascii(df$c)}} +#' @aliases ascii ascii,Column-method +#' @examples +#' +#' \dontrun{ +#' head(select(df, ascii(df$Class), ascii(df$Sex)))} +#' @note ascii since 1.5.0 setMethod("ascii", signature(x = "Column"), function(x) { @@ -107,16 +324,14 @@ setMethod("ascii", column(jc) }) -#' asin -#' -#' Computes the sine inverse of the given value; the returned angle is in the range -#' -pi/2 through pi/2. +#' @details +#' \code{asin}: Computes the sine inverse of the given value; the returned angle is in +#' the range -pi/2 through pi/2. #' -#' @rdname asin -#' @name asin -#' @family math_funcs +#' @rdname column_math_functions #' @export -#' @examples \dontrun{asin(df$c)} +#' @aliases asin asin,Column-method +#' @note asin since 1.5.0 setMethod("asin", signature(x = "Column"), function(x) { @@ -124,15 +339,14 @@ setMethod("asin", column(jc) }) -#' atan -#' -#' Computes the tangent inverse of the given value. +#' @details +#' \code{atan}: Computes the tangent inverse of the given value; the returned angle is in the range +#' -pi/2 through pi/2. #' -#' @rdname atan -#' @name atan -#' @family math_funcs +#' @rdname column_math_functions #' @export -#' @examples \dontrun{atan(df$c)} +#' @aliases atan atan,Column-method +#' @note atan since 1.5.0 setMethod("atan", signature(x = "Column"), function(x) { @@ -146,9 +360,11 @@ setMethod("atan", #' #' @rdname avg #' @name avg -#' @family agg_funcs +#' @family aggregate functions #' @export +#' @aliases avg,Column-method #' @examples \dontrun{avg(df$c)} +#' @note avg since 1.4.0 setMethod("avg", signature(x = "Column"), function(x) { @@ -156,16 +372,23 @@ setMethod("avg", column(jc) }) -#' base64 -#' -#' Computes the BASE64 encoding of a binary column and returns it as a string column. -#' This is the reverse of unbase64. +#' @details +#' \code{base64}: Computes the BASE64 encoding of a binary column and returns it as +#' a string column. This is the reverse of unbase64. #' -#' @rdname base64 -#' @name base64 -#' @family string_funcs +#' @rdname column_string_functions #' @export -#' @examples \dontrun{base64(df$c)} +#' @aliases base64 base64,Column-method +#' @examples +#' +#' \dontrun{ +#' tmp <- mutate(df, s1 = encode(df$Class, "UTF-8")) +#' str(tmp) +#' tmp2 <- mutate(tmp, s2 = base64(tmp$s1), s3 = decode(tmp$s1, "UTF-8"), +#' s4 = soundex(tmp$Sex)) +#' head(tmp2) +#' head(select(tmp2, unbase64(tmp2$s2)))} +#' @note base64 since 1.5.0 setMethod("base64", signature(x = "Column"), function(x) { @@ -173,16 +396,14 @@ setMethod("base64", column(jc) }) -#' bin -#' -#' An expression that returns the string representation of the binary value of the given long -#' column. For example, bin("12") returns "1100". +#' @details +#' \code{bin}: Returns the string representation of the binary value +#' of the given long column. For example, bin("12") returns "1100". #' -#' @rdname bin -#' @name bin -#' @family math_funcs +#' @rdname column_math_functions #' @export -#' @examples \dontrun{bin(df$c)} +#' @aliases bin bin,Column-method +#' @note bin since 1.5.0 setMethod("bin", signature(x = "Column"), function(x) { @@ -190,15 +411,17 @@ setMethod("bin", column(jc) }) -#' bitwiseNOT +#' @details +#' \code{bitwiseNOT}: Computes bitwise NOT. #' -#' Computes bitwise NOT. -#' -#' @rdname bitwiseNOT -#' @name bitwiseNOT -#' @family normal_funcs +#' @rdname column_nonaggregate_functions #' @export -#' @examples \dontrun{bitwiseNOT(df$c)} +#' @aliases bitwiseNOT bitwiseNOT,Column-method +#' @examples +#' +#' \dontrun{ +#' head(select(df, bitwiseNOT(cast(df$vs, "int"))))} +#' @note bitwiseNOT since 1.5.0 setMethod("bitwiseNOT", signature(x = "Column"), function(x) { @@ -206,15 +429,13 @@ setMethod("bitwiseNOT", column(jc) }) -#' cbrt +#' @details +#' \code{cbrt}: Computes the cube-root of the given value. #' -#' Computes the cube-root of the given value. -#' -#' @rdname cbrt -#' @name cbrt -#' @family math_funcs +#' @rdname column_math_functions #' @export -#' @examples \dontrun{cbrt(df$c)} +#' @aliases cbrt cbrt,Column-method +#' @note cbrt since 1.4.0 setMethod("cbrt", signature(x = "Column"), function(x) { @@ -222,15 +443,13 @@ setMethod("cbrt", column(jc) }) -#' ceil -#' -#' Computes the ceiling of the given value. +#' @details +#' \code{ceil}: Computes the ceiling of the given value. #' -#' @rdname ceil -#' @name ceil -#' @family math_funcs +#' @rdname column_math_functions #' @export -#' @examples \dontrun{ceil(df$c)} +#' @aliases ceil ceil,Column-method +#' @note ceil since 1.5.0 setMethod("ceil", signature(x = "Column"), function(x) { @@ -238,37 +457,159 @@ setMethod("ceil", column(jc) }) +#' @details +#' \code{ceiling}: Alias for \code{ceil}. +#' +#' @rdname column_math_functions +#' @aliases ceiling ceiling,Column-method +#' @export +#' @note ceiling since 1.5.0 +setMethod("ceiling", + signature(x = "Column"), + function(x) { + ceil(x) + }) + +#' @details +#' \code{coalesce}: Returns the first column that is not NA, or NA if all inputs are. +#' +#' @rdname column_nonaggregate_functions +#' @export +#' @aliases coalesce,Column-method +#' @note coalesce(Column) since 2.1.1 +setMethod("coalesce", + signature(x = "Column"), + function(x, ...) { + jcols <- lapply(list(x, ...), function (x) { + stopifnot(class(x) == "Column") + x@jc + }) + jc <- callJStatic("org.apache.spark.sql.functions", "coalesce", jcols) + column(jc) + }) + #' Though scala functions has "col" function, we don't expose it in SparkR #' because we don't want to conflict with the "col" function in the R base #' package and we also have "column" function exported which is an alias of "col". +#' @noRd col <- function(x) { column(callJStatic("org.apache.spark.sql.functions", "col", x)) } -#' column +#' Returns a Column based on the given column name #' #' Returns a Column based on the given column name. #' -#' @rdname col +#' @param x Character column name. +#' +#' @rdname column #' @name column -#' @family normal_funcs +#' @family non-aggregate functions #' @export -#' @examples \dontrun{column(df)} +#' @aliases column,character-method +#' @examples \dontrun{column("name")} +#' @note column since 1.6.0 setMethod("column", signature(x = "character"), function(x) { col(x) }) -#' cos +#' corr +#' +#' Computes the Pearson Correlation Coefficient for two Columns. +#' +#' @param col2 a (second) Column. +#' +#' @rdname corr +#' @name corr +#' @family aggregate functions +#' @export +#' @aliases corr,Column-method +#' @examples +#' \dontrun{ +#' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars)) +#' head(select(df, corr(df$mpg, df$hp)))} +#' @note corr since 1.6.0 +setMethod("corr", signature(x = "Column"), + function(x, col2) { + stopifnot(class(col2) == "Column") + jc <- callJStatic("org.apache.spark.sql.functions", "corr", x@jc, col2@jc) + column(jc) + }) + +#' cov +#' +#' Compute the covariance between two expressions. +#' +#' @details +#' \code{cov}: Compute the sample covariance between two expressions. +#' +#' @rdname cov +#' @name cov +#' @family aggregate functions +#' @export +#' @aliases cov,characterOrColumn-method +#' @examples +#' \dontrun{ +#' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars)) +#' head(select(df, cov(df$mpg, df$hp), cov("mpg", "hp"), +#' covar_samp(df$mpg, df$hp), covar_samp("mpg", "hp"), +#' covar_pop(df$mpg, df$hp), covar_pop("mpg", "hp")))} +#' @note cov since 1.6.0 +setMethod("cov", signature(x = "characterOrColumn"), + function(x, col2) { + stopifnot(is(class(col2), "characterOrColumn")) + covar_samp(x, col2) + }) + +#' @details +#' \code{covar_sample}: Alias for \code{cov}. +#' +#' @rdname cov +#' +#' @param col1 the first Column. +#' @param col2 the second Column. +#' @name covar_samp +#' @aliases covar_samp,characterOrColumn,characterOrColumn-method +#' @note covar_samp since 2.0.0 +setMethod("covar_samp", signature(col1 = "characterOrColumn", col2 = "characterOrColumn"), + function(col1, col2) { + stopifnot(class(col1) == class(col2)) + if (class(col1) == "Column") { + col1 <- col1@jc + col2 <- col2@jc + } + jc <- callJStatic("org.apache.spark.sql.functions", "covar_samp", col1, col2) + column(jc) + }) + +#' @details +#' \code{covar_pop}: Computes the population covariance between two expressions. #' -#' Computes the cosine of the given value. +#' @rdname cov +#' @name covar_pop +#' @export +#' @aliases covar_pop,characterOrColumn,characterOrColumn-method +#' @note covar_pop since 2.0.0 +setMethod("covar_pop", signature(col1 = "characterOrColumn", col2 = "characterOrColumn"), + function(col1, col2) { + stopifnot(class(col1) == class(col2)) + if (class(col1) == "Column") { + col1 <- col1@jc + col2 <- col2@jc + } + jc <- callJStatic("org.apache.spark.sql.functions", "covar_pop", col1, col2) + column(jc) + }) + +#' @details +#' \code{cos}: Computes the cosine of the given value. Units in radians. #' -#' @rdname cos -#' @name cos -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases cos cos,Column-method #' @export -#' @examples \dontrun{cos(df$c)} +#' @note cos since 1.5.0 setMethod("cos", signature(x = "Column"), function(x) { @@ -276,15 +617,13 @@ setMethod("cos", column(jc) }) -#' cosh +#' @details +#' \code{cosh}: Computes the hyperbolic cosine of the given value. #' -#' Computes the hyperbolic cosine of the given value. -#' -#' @rdname cosh -#' @name cosh -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases cosh cosh,Column-method #' @export -#' @examples \dontrun{cosh(df$c)} +#' @note cosh since 1.5.0 setMethod("cosh", signature(x = "Column"), function(x) { @@ -292,15 +631,18 @@ setMethod("cosh", column(jc) }) -#' count +#' Returns the number of items in a group #' -#' Aggregate function: returns the number of items in a group. +#' This can be used as a column aggregate function with \code{Column} as input, +#' and returns the number of items in a group. #' #' @rdname count #' @name count -#' @family agg_funcs +#' @family aggregate functions +#' @aliases count,Column-method #' @export #' @examples \dontrun{count(df$c)} +#' @note count since 1.4.0 setMethod("count", signature(x = "Column"), function(x) { @@ -308,16 +650,14 @@ setMethod("count", column(jc) }) -#' crc32 -#' -#' Calculates the cyclic redundancy check value (CRC32) of a binary column and -#' returns the value as a bigint. +#' @details +#' \code{crc32}: Calculates the cyclic redundancy check value (CRC32) of a binary column +#' and returns the value as a bigint. #' -#' @rdname crc32 -#' @name crc32 -#' @family misc_funcs +#' @rdname column_misc_functions +#' @aliases crc32 crc32,Column-method #' @export -#' @examples \dontrun{crc32(df$c)} +#' @note crc32 since 1.5.0 setMethod("crc32", signature(x = "Column"), function(x) { @@ -325,15 +665,40 @@ setMethod("crc32", column(jc) }) -#' dayofmonth +#' @details +#' \code{hash}: Calculates the hash code of given columns, and returns the result +#' as an int column. #' -#' Extracts the day of the month as an integer from a given date/timestamp/string. +#' @rdname column_misc_functions +#' @aliases hash hash,Column-method +#' @export +#' @note hash since 2.0.0 +setMethod("hash", + signature(x = "Column"), + function(x, ...) { + jcols <- lapply(list(x, ...), function (x) { + stopifnot(class(x) == "Column") + x@jc + }) + jc <- callJStatic("org.apache.spark.sql.functions", "hash", jcols) + column(jc) + }) + +#' @details +#' \code{dayofmonth}: Extracts the day of the month as an integer from a +#' given date/timestamp/string. #' -#' @rdname dayofmonth -#' @name dayofmonth -#' @family datetime_funcs +#' @rdname column_datetime_functions +#' @aliases dayofmonth dayofmonth,Column-method #' @export -#' @examples \dontrun{dayofmonth(df$c)} +#' @examples +#' +#' \dontrun{ +#' head(select(df, df$time, year(df$time), quarter(df$time), month(df$time), +#' dayofmonth(df$time), dayofyear(df$time), weekofyear(df$time))) +#' head(agg(groupBy(df, year(df$time)), count(df$y), avg(df$y))) +#' head(agg(groupBy(df, month(df$time)), avg(df$y)))} +#' @note dayofmonth since 1.5.0 setMethod("dayofmonth", signature(x = "Column"), function(x) { @@ -341,15 +706,14 @@ setMethod("dayofmonth", column(jc) }) -#' dayofyear -#' -#' Extracts the day of the year as an integer from a given date/timestamp/string. +#' @details +#' \code{dayofyear}: Extracts the day of the year as an integer from a +#' given date/timestamp/string. #' -#' @rdname dayofyear -#' @name dayofyear -#' @family datetime_funcs +#' @rdname column_datetime_functions +#' @aliases dayofyear dayofyear,Column-method #' @export -#' @examples \dontrun{dayofyear(df$c)} +#' @note dayofyear since 1.5.0 setMethod("dayofyear", signature(x = "Column"), function(x) { @@ -357,47 +721,60 @@ setMethod("dayofyear", column(jc) }) -#' exp +#' @details +#' \code{decode}: Computes the first argument into a string from a binary using the provided +#' character set. #' -#' Computes the exponential of the given value. +#' @param charset character set to use (one of "US-ASCII", "ISO-8859-1", "UTF-8", "UTF-16BE", +#' "UTF-16LE", "UTF-16"). #' -#' @rdname exp -#' @name exp -#' @family math_funcs +#' @rdname column_string_functions +#' @aliases decode decode,Column,character-method #' @export -#' @examples \dontrun{exp(df$c)} -setMethod("exp", - signature(x = "Column"), - function(x) { - jc <- callJStatic("org.apache.spark.sql.functions", "exp", x@jc) +#' @note decode since 1.6.0 +setMethod("decode", + signature(x = "Column", charset = "character"), + function(x, charset) { + jc <- callJStatic("org.apache.spark.sql.functions", "decode", x@jc, charset) column(jc) }) -#' explode +#' @details +#' \code{encode}: Computes the first argument into a binary from a string using the provided +#' character set. #' -#' Creates a new row for each element in the given array or map column. +#' @rdname column_string_functions +#' @aliases encode encode,Column,character-method +#' @export +#' @note encode since 1.6.0 +setMethod("encode", + signature(x = "Column", charset = "character"), + function(x, charset) { + jc <- callJStatic("org.apache.spark.sql.functions", "encode", x@jc, charset) + column(jc) + }) + +#' @details +#' \code{exp}: Computes the exponential of the given value. #' -#' @rdname explode -#' @name explode -#' @family collection_funcs +#' @rdname column_math_functions +#' @aliases exp exp,Column-method #' @export -#' @examples \dontrun{explode(df$c)} -setMethod("explode", +#' @note exp since 1.5.0 +setMethod("exp", signature(x = "Column"), function(x) { - jc <- callJStatic("org.apache.spark.sql.functions", "explode", x@jc) + jc <- callJStatic("org.apache.spark.sql.functions", "exp", x@jc) column(jc) }) -#' expm1 +#' @details +#' \code{expm1}: Computes the exponential of the given value minus one. #' -#' Computes the exponential of the given value minus one. -#' -#' @rdname expm1 -#' @name expm1 -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases expm1 expm1,Column-method #' @export -#' @examples \dontrun{expm1(df$c)} +#' @note expm1 since 1.5.0 setMethod("expm1", signature(x = "Column"), function(x) { @@ -405,15 +782,13 @@ setMethod("expm1", column(jc) }) -#' factorial -#' -#' Computes the factorial of the given value. +#' @details +#' \code{factorial}: Computes the factorial of the given value. #' -#' @rdname factorial -#' @name factorial -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases factorial factorial,Column-method #' @export -#' @examples \dontrun{factorial(df$c)} +#' @note factorial since 1.5.0 setMethod("factorial", signature(x = "Column"), function(x) { @@ -425,27 +800,42 @@ setMethod("factorial", #' #' Aggregate function: returns the first value in a group. #' +#' The function by default returns the first values it sees. It will return the first non-missing +#' value it sees when na.rm is set to true. If all values are missing, then NA is returned. +#' +#' @param na.rm a logical value indicating whether NA values should be stripped +#' before the computation proceeds. +#' #' @rdname first #' @name first -#' @family agg_funcs +#' @aliases first,characterOrColumn-method +#' @family aggregate functions #' @export -#' @examples \dontrun{first(df$c)} +#' @examples +#' \dontrun{ +#' first(df$c) +#' first(df$c, TRUE) +#' } +#' @note first(characterOrColumn) since 1.4.0 setMethod("first", - signature(x = "Column"), - function(x) { - jc <- callJStatic("org.apache.spark.sql.functions", "first", x@jc) + signature(x = "characterOrColumn"), + function(x, na.rm = FALSE) { + col <- if (class(x) == "Column") { + x@jc + } else { + x + } + jc <- callJStatic("org.apache.spark.sql.functions", "first", col, na.rm) column(jc) }) -#' floor -#' -#' Computes the floor of the given value. +#' @details +#' \code{floor}: Computes the floor of the given value. #' -#' @rdname floor -#' @name floor -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases floor floor,Column-method #' @export -#' @examples \dontrun{floor(df$c)} +#' @note floor since 1.5.0 setMethod("floor", signature(x = "Column"), function(x) { @@ -453,15 +843,13 @@ setMethod("floor", column(jc) }) -#' hex +#' @details +#' \code{hex}: Computes hex value of the given column. #' -#' Computes hex value of the given column. -#' -#' @rdname hex -#' @name hex -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases hex hex,Column-method #' @export -#' @examples \dontrun{hex(df$c)} +#' @note hex since 1.5.0 setMethod("hex", signature(x = "Column"), function(x) { @@ -469,15 +857,20 @@ setMethod("hex", column(jc) }) -#' hour -#' -#' Extracts the hours as an integer from a given date/timestamp/string. +#' @details +#' \code{hour}: Extracts the hour as an integer from a given date/timestamp/string. #' -#' @rdname hour -#' @name hour -#' @family datetime_funcs +#' @rdname column_datetime_functions +#' @aliases hour hour,Column-method #' @export -#' @examples \dontrun{hour(df$c)} +#' @examples +#' +#' \dontrun{ +#' head(select(df, hour(df$time), minute(df$time), second(df$time))) +#' head(agg(groupBy(df, dayofmonth(df$time)), avg(df$y))) +#' head(agg(groupBy(df, hour(df$time)), avg(df$y))) +#' head(agg(groupBy(df, minute(df$time)), avg(df$y)))} +#' @note hour since 1.5.0 setMethod("hour", signature(x = "Column"), function(x) { @@ -485,18 +878,24 @@ setMethod("hour", column(jc) }) -#' initcap -#' -#' Returns a new string column by converting the first letter of each word to uppercase. -#' Words are delimited by whitespace. +#' @details +#' \code{initcap}: Returns a new string column by converting the first letter of +#' each word to uppercase. Words are delimited by whitespace. For example, "hello world" +#' will become "Hello World". #' -#' For example, "hello world" will become "Hello World". -#' -#' @rdname initcap -#' @name initcap -#' @family string_funcs +#' @rdname column_string_functions +#' @aliases initcap initcap,Column-method #' @export -#' @examples \dontrun{initcap(df$c)} +#' @examples +#' +#' \dontrun{ +#' tmp <- mutate(df, sex_lower = lower(df$Sex), age_upper = upper(df$age), +#' sex_age = concat_ws(" ", lower(df$sex), lower(df$age))) +#' head(tmp) +#' tmp2 <- mutate(tmp, s1 = initcap(tmp$sex_lower), s2 = initcap(tmp$sex_age), +#' s3 = reverse(df$Sex)) +#' head(tmp2)} +#' @note initcap since 1.5.0 setMethod("initcap", signature(x = "Column"), function(x) { @@ -504,19 +903,46 @@ setMethod("initcap", column(jc) }) -#' isNaN +#' @details +#' \code{isnan}: Returns true if the column is NaN. +#' @rdname column_nonaggregate_functions +#' @aliases isnan isnan,Column-method +#' @note isnan since 2.0.0 +setMethod("isnan", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "isnan", x@jc) + column(jc) + }) + +#' @details +#' \code{is.nan}: Alias for \link{isnan}. #' -#' Return true iff the column is NaN. +#' @rdname column_nonaggregate_functions +#' @aliases is.nan is.nan,Column-method +#' @export +#' @note is.nan since 2.0.0 +setMethod("is.nan", + signature(x = "Column"), + function(x) { + isnan(x) + }) + +#' @details +#' \code{kurtosis}: Returns the kurtosis of the values in a group. #' -#' @rdname isNaN -#' @name isNaN -#' @family normal_funcs +#' @rdname column_aggregate_functions +#' @aliases kurtosis kurtosis,Column-method #' @export -#' @examples \dontrun{isNaN(df$c)} -setMethod("isNaN", +#' @examples +#' +#' \dontrun{ +#' head(select(df, mean(df$mpg), sd(df$mpg), skewness(df$mpg), kurtosis(df$mpg)))} +#' @note kurtosis since 1.6.0 +setMethod("kurtosis", signature(x = "Column"), function(x) { - jc <- callJStatic("org.apache.spark.sql.functions", "isNaN", x@jc) + jc <- callJStatic("org.apache.spark.sql.functions", "kurtosis", x@jc) column(jc) }) @@ -524,29 +950,50 @@ setMethod("isNaN", #' #' Aggregate function: returns the last value in a group. #' +#' The function by default returns the last values it sees. It will return the last non-missing +#' value it sees when na.rm is set to true. If all values are missing, then NA is returned. +#' +#' @param x column to compute on. +#' @param na.rm a logical value indicating whether NA values should be stripped +#' before the computation proceeds. +#' @param ... further arguments to be passed to or from other methods. +#' #' @rdname last #' @name last -#' @family agg_funcs +#' @aliases last,characterOrColumn-method +#' @family aggregate functions #' @export -#' @examples \dontrun{last(df$c)} +#' @examples +#' \dontrun{ +#' last(df$c) +#' last(df$c, TRUE) +#' } +#' @note last since 1.4.0 setMethod("last", - signature(x = "Column"), - function(x) { - jc <- callJStatic("org.apache.spark.sql.functions", "last", x@jc) + signature(x = "characterOrColumn"), + function(x, na.rm = FALSE) { + col <- if (class(x) == "Column") { + x@jc + } else { + x + } + jc <- callJStatic("org.apache.spark.sql.functions", "last", col, na.rm) column(jc) }) -#' last_day +#' @details +#' \code{last_day}: Given a date column, returns the last day of the month which the +#' given date belongs to. For example, input "2015-07-27" returns "2015-07-31" since +#' July 31 is the last day of the month in July 2015. #' -#' Given a date column, returns the last day of the month which the given date belongs to. -#' For example, input "2015-07-27" returns "2015-07-31" since July 31 is the last day of the -#' month in July 2015. -#' -#' @rdname last_day -#' @name last_day -#' @family datetime_funcs +#' @rdname column_datetime_functions +#' @aliases last_day last_day,Column-method #' @export -#' @examples \dontrun{last_day(df$c)} +#' @examples +#' +#' \dontrun{ +#' head(select(df, df$time, last_day(df$time), month(df$time)))} +#' @note last_day since 1.5.0 setMethod("last_day", signature(x = "Column"), function(x) { @@ -554,15 +1001,13 @@ setMethod("last_day", column(jc) }) -#' length +#' @details +#' \code{length}: Computes the length of a given string or binary column. #' -#' Computes the length of a given string or binary column. -#' -#' @rdname length -#' @name length -#' @family string_funcs +#' @rdname column_string_functions +#' @aliases length length,Column-method #' @export -#' @examples \dontrun{length(df$c)} +#' @note length since 1.5.0 setMethod("length", signature(x = "Column"), function(x) { @@ -570,15 +1015,13 @@ setMethod("length", column(jc) }) -#' log -#' -#' Computes the natural logarithm of the given value. +#' @details +#' \code{log}: Computes the natural logarithm of the given value. #' -#' @rdname log -#' @name log -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases log log,Column-method #' @export -#' @examples \dontrun{log(df$c)} +#' @note log since 1.5.0 setMethod("log", signature(x = "Column"), function(x) { @@ -586,15 +1029,13 @@ setMethod("log", column(jc) }) -#' log10 +#' @details +#' \code{log10}: Computes the logarithm of the given value in base 10. #' -#' Computes the logarithm of the given value in base 10. -#' -#' @rdname log10 -#' @name log10 -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases log10 log10,Column-method #' @export -#' @examples \dontrun{log10(df$c)} +#' @note log10 since 1.5.0 setMethod("log10", signature(x = "Column"), function(x) { @@ -602,15 +1043,13 @@ setMethod("log10", column(jc) }) -#' log1p -#' -#' Computes the natural logarithm of the given value plus one. +#' @details +#' \code{log1p}: Computes the natural logarithm of the given value plus one. #' -#' @rdname log1p -#' @name log1p -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases log1p log1p,Column-method #' @export -#' @examples \dontrun{log1p(df$c)} +#' @note log1p since 1.5.0 setMethod("log1p", signature(x = "Column"), function(x) { @@ -618,15 +1057,13 @@ setMethod("log1p", column(jc) }) -#' log2 +#' @details +#' \code{log2}: Computes the logarithm of the given column in base 2. #' -#' Computes the logarithm of the given column in base 2. -#' -#' @rdname log2 -#' @name log2 -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases log2 log2,Column-method #' @export -#' @examples \dontrun{log2(df$c)} +#' @note log2 since 1.5.0 setMethod("log2", signature(x = "Column"), function(x) { @@ -634,15 +1071,13 @@ setMethod("log2", column(jc) }) -#' lower -#' -#' Converts a string column to lower case. +#' @details +#' \code{lower}: Converts a string column to lower case. #' -#' @rdname lower -#' @name lower -#' @family string_funcs +#' @rdname column_string_functions +#' @aliases lower lower,Column-method #' @export -#' @examples \dontrun{lower(df$c)} +#' @note lower since 1.4.0 setMethod("lower", signature(x = "Column"), function(x) { @@ -650,15 +1085,25 @@ setMethod("lower", column(jc) }) -#' ltrim +#' @details +#' \code{ltrim}: Trims the spaces from left end for the specified string value. #' -#' Trim the spaces from left end for the specified string value. -#' -#' @rdname ltrim -#' @name ltrim -#' @family string_funcs +#' @rdname column_string_functions +#' @aliases ltrim ltrim,Column-method #' @export -#' @examples \dontrun{ltrim(df$c)} +#' @examples +#' +#' \dontrun{ +#' tmp <- mutate(df, SexLpad = lpad(df$Sex, 6, " "), SexRpad = rpad(df$Sex, 7, " ")) +#' head(select(tmp, length(tmp$Sex), length(tmp$SexLpad), length(tmp$SexRpad))) +#' tmp2 <- mutate(tmp, SexLtrim = ltrim(tmp$SexLpad), SexRtrim = rtrim(tmp$SexRpad), +#' SexTrim = trim(tmp$SexLpad)) +#' head(select(tmp2, length(tmp2$Sex), length(tmp2$SexLtrim), +#' length(tmp2$SexRtrim), length(tmp2$SexTrim))) +#' +#' tmp <- mutate(df, SexLpad = lpad(df$Sex, 6, "xx"), SexRpad = rpad(df$Sex, 7, "xx")) +#' head(tmp)} +#' @note ltrim since 1.5.0 setMethod("ltrim", signature(x = "Column"), function(x) { @@ -666,15 +1111,12 @@ setMethod("ltrim", column(jc) }) -#' max +#' @details +#' \code{max}: Returns the maximum value of the expression in a group. #' -#' Aggregate function: returns the maximum value of the expression in a group. -#' -#' @rdname max -#' @name max -#' @family agg_funcs -#' @export -#' @examples \dontrun{max(df$c)} +#' @rdname column_aggregate_functions +#' @aliases max max,Column-method +#' @note max since 1.5.0 setMethod("max", signature(x = "Column"), function(x) { @@ -682,16 +1124,14 @@ setMethod("max", column(jc) }) -#' md5 -#' -#' Calculates the MD5 digest of a binary column and returns the value +#' @details +#' \code{md5}: Calculates the MD5 digest of a binary column and returns the value #' as a 32 character hex string. #' -#' @rdname md5 -#' @name md5 -#' @family misc_funcs +#' @rdname column_misc_functions +#' @aliases md5 md5,Column-method #' @export -#' @examples \dontrun{md5(df$c)} +#' @note md5 since 1.5.0 setMethod("md5", signature(x = "Column"), function(x) { @@ -699,16 +1139,25 @@ setMethod("md5", column(jc) }) -#' mean -#' -#' Aggregate function: returns the average of the values in a group. -#' Alias for avg. +#' @details +#' \code{mean}: Returns the average of the values in a group. Alias for \code{avg}. #' -#' @rdname mean -#' @name mean -#' @family agg_funcs +#' @rdname column_aggregate_functions +#' @aliases mean mean,Column-method #' @export -#' @examples \dontrun{mean(df$c)} +#' @examples +#' +#' \dontrun{ +#' head(select(df, avg(df$mpg), mean(df$mpg), sum(df$mpg), min(df$wt), max(df$qsec))) +#' +#' # metrics by num of cylinders +#' tmp <- agg(groupBy(df, "cyl"), avg(df$mpg), avg(df$hp), avg(df$wt), avg(df$qsec)) +#' head(orderBy(tmp, "cyl")) +#' +#' # car with the max mpg +#' mpg_max <- as.numeric(collect(agg(df, max(df$mpg)))) +#' head(where(df, df$mpg == mpg_max))} +#' @note mean since 1.5.0 setMethod("mean", signature(x = "Column"), function(x) { @@ -716,15 +1165,13 @@ setMethod("mean", column(jc) }) -#' min +#' @details +#' \code{min}: Returns the minimum value of the expression in a group. #' -#' Aggregate function: returns the minimum value of the expression in a group. -#' -#' @rdname min -#' @name min -#' @family agg_funcs +#' @rdname column_aggregate_functions +#' @aliases min min,Column-method #' @export -#' @examples \dontrun{min(df$c)} +#' @note min since 1.5.0 setMethod("min", signature(x = "Column"), function(x) { @@ -732,15 +1179,13 @@ setMethod("min", column(jc) }) -#' minute -#' -#' Extracts the minutes as an integer from a given date/timestamp/string. +#' @details +#' \code{minute}: Extracts the minute as an integer from a given date/timestamp/string. #' -#' @rdname minute -#' @name minute -#' @family datetime_funcs +#' @rdname column_datetime_functions +#' @aliases minute minute,Column-method #' @export -#' @examples \dontrun{minute(df$c)} +#' @note minute since 1.5.0 setMethod("minute", signature(x = "Column"), function(x) { @@ -748,15 +1193,38 @@ setMethod("minute", column(jc) }) -#' month +#' @details +#' \code{monotonically_increasing_id}: Returns a column that generates monotonically increasing +#' 64-bit integers. The generated ID is guaranteed to be monotonically increasing and unique, +#' but not consecutive. The current implementation puts the partition ID in the upper 31 bits, +#' and the record number within each partition in the lower 33 bits. The assumption is that the +#' SparkDataFrame has less than 1 billion partitions, and each partition has less than 8 billion +#' records. As an example, consider a SparkDataFrame with two partitions, each with 3 records. +#' This expression would return the following IDs: +#' 0, 1, 2, 8589934592 (1L << 33), 8589934593, 8589934594. +#' This is equivalent to the MONOTONICALLY_INCREASING_ID function in SQL. +#' The method should be used with no argument. +#' +#' @rdname column_nonaggregate_functions +#' @aliases monotonically_increasing_id monotonically_increasing_id,missing-method +#' @export +#' @examples #' -#' Extracts the month as an integer from a given date/timestamp/string. +#' \dontrun{head(select(df, monotonically_increasing_id()))} +setMethod("monotonically_increasing_id", + signature("missing"), + function() { + jc <- callJStatic("org.apache.spark.sql.functions", "monotonically_increasing_id") + column(jc) + }) + +#' @details +#' \code{month}: Extracts the month as an integer from a given date/timestamp/string. #' -#' @rdname month -#' @name month -#' @family datetime_funcs +#' @rdname column_datetime_functions +#' @aliases month month,Column-method #' @export -#' @examples \dontrun{month(df$c)} +#' @note month since 1.5.0 setMethod("month", signature(x = "Column"), function(x) { @@ -764,15 +1232,13 @@ setMethod("month", column(jc) }) -#' negate -#' -#' Unary minus, i.e. negate the expression. +#' @details +#' \code{negate}: Unary minus, i.e. negate the expression. #' -#' @rdname negate -#' @name negate -#' @family normal_funcs +#' @rdname column_nonaggregate_functions +#' @aliases negate negate,Column-method #' @export -#' @examples \dontrun{negate(df$c)} +#' @note negate since 1.5.0 setMethod("negate", signature(x = "Column"), function(x) { @@ -780,15 +1246,13 @@ setMethod("negate", column(jc) }) -#' quarter +#' @details +#' \code{quarter}: Extracts the quarter as an integer from a given date/timestamp/string. #' -#' Extracts the quarter as an integer from a given date/timestamp/string. -#' -#' @rdname quarter -#' @name quarter -#' @family datetime_funcs +#' @rdname column_datetime_functions +#' @aliases quarter quarter,Column-method #' @export -#' @examples \dontrun{quarter(df$c)} +#' @note quarter since 1.5.0 setMethod("quarter", signature(x = "Column"), function(x) { @@ -796,15 +1260,13 @@ setMethod("quarter", column(jc) }) -#' reverse -#' -#' Reverses the string column and returns it as a new string column. +#' @details +#' \code{reverse}: Reverses the string column and returns it as a new string column. #' -#' @rdname reverse -#' @name reverse -#' @family string_funcs +#' @rdname column_string_functions +#' @aliases reverse reverse,Column-method #' @export -#' @examples \dontrun{reverse(df$c)} +#' @note reverse since 1.5.0 setMethod("reverse", signature(x = "Column"), function(x) { @@ -812,16 +1274,14 @@ setMethod("reverse", column(jc) }) -#' rint -#' -#' Returns the double value that is closest in value to the argument and +#' @details +#' \code{rint}: Returns the double value that is closest in value to the argument and #' is equal to a mathematical integer. #' -#' @rdname rint -#' @name rint -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases rint rint,Column-method #' @export -#' @examples \dontrun{rint(df$c)} +#' @note rint since 1.5.0 setMethod("rint", signature(x = "Column"), function(x) { @@ -829,15 +1289,14 @@ setMethod("rint", column(jc) }) -#' round -#' -#' Returns the value of the column `e` rounded to 0 decimal places. +#' @details +#' \code{round}: Returns the value of the column rounded to 0 decimal places +#' using HALF_UP rounding mode. #' -#' @rdname round -#' @name round -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases round round,Column-method #' @export -#' @examples \dontrun{round(df$c)} +#' @note round since 1.5.0 setMethod("round", signature(x = "Column"), function(x) { @@ -845,15 +1304,33 @@ setMethod("round", column(jc) }) -#' rtrim +#' @details +#' \code{bround}: Returns the value of the column \code{e} rounded to \code{scale} decimal places +#' using HALF_EVEN rounding mode if \code{scale} >= 0 or at integer part when \code{scale} < 0. +#' Also known as Gaussian rounding or bankers' rounding that rounds to the nearest even number. +#' bround(2.5, 0) = 2, bround(3.5, 0) = 4. #' -#' Trim the spaces from right end for the specified string value. +#' @param scale round to \code{scale} digits to the right of the decimal point when \code{scale} > 0, +#' the nearest even number when \code{scale} = 0, and \code{scale} digits to the left +#' of the decimal point when \code{scale} < 0. +#' @rdname column_math_functions +#' @aliases bround bround,Column-method +#' @export +#' @note bround since 2.0.0 +setMethod("bround", + signature(x = "Column"), + function(x, scale = 0) { + jc <- callJStatic("org.apache.spark.sql.functions", "bround", x@jc, as.integer(scale)) + column(jc) + }) + +#' @details +#' \code{rtrim}: Trims the spaces from right end for the specified string value. #' -#' @rdname rtrim -#' @name rtrim -#' @family string_funcs +#' @rdname column_string_functions +#' @aliases rtrim rtrim,Column-method #' @export -#' @examples \dontrun{rtrim(df$c)} +#' @note rtrim since 1.5.0 setMethod("rtrim", signature(x = "Column"), function(x) { @@ -861,15 +1338,31 @@ setMethod("rtrim", column(jc) }) -#' second +#' @details +#' \code{sd}: Alias for \code{stddev_samp}. +#' +#' @rdname column_aggregate_functions +#' @aliases sd sd,Column-method +#' @export +#' @examples #' -#' Extracts the seconds as an integer from a given date/timestamp/string. +#' \dontrun{ +#' head(select(df, sd(df$mpg), stddev(df$mpg), stddev_pop(df$wt), stddev_samp(df$qsec)))} +#' @note sd since 1.6.0 +setMethod("sd", + signature(x = "Column"), + function(x) { + # In R, sample standard deviation is calculated with the sd() function. + stddev_samp(x) + }) + +#' @details +#' \code{second}: Extracts the second as an integer from a given date/timestamp/string. #' -#' @rdname second -#' @name second -#' @family datetime_funcs +#' @rdname column_datetime_functions +#' @aliases second second,Column-method #' @export -#' @examples \dontrun{second(df$c)} +#' @note second since 1.5.0 setMethod("second", signature(x = "Column"), function(x) { @@ -877,16 +1370,14 @@ setMethod("second", column(jc) }) -#' sha1 -#' -#' Calculates the SHA-1 digest of a binary column and returns the value +#' @details +#' \code{sha1}: Calculates the SHA-1 digest of a binary column and returns the value #' as a 40 character hex string. #' -#' @rdname sha1 -#' @name sha1 -#' @family misc_funcs +#' @rdname column_misc_functions +#' @aliases sha1 sha1,Column-method #' @export -#' @examples \dontrun{sha1(df$c)} +#' @note sha1 since 1.5.0 setMethod("sha1", signature(x = "Column"), function(x) { @@ -894,15 +1385,13 @@ setMethod("sha1", column(jc) }) -#' signum +#' @details +#' \code{signum}: Computes the signum of the given value. #' -#' Computes the signum of the given value. -#' -#' @rdname signum -#' @name signum -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases signum signum,Column-method #' @export -#' @examples \dontrun{signum(df$c)} +#' @note signum since 1.5.0 setMethod("signum", signature(x = "Column"), function(x) { @@ -910,15 +1399,25 @@ setMethod("signum", column(jc) }) -#' sin +#' @details +#' \code{sign}: Alias for \code{signum}. #' -#' Computes the sine of the given value. +#' @rdname column_math_functions +#' @aliases sign sign,Column-method +#' @export +#' @note sign since 1.5.0 +setMethod("sign", signature(x = "Column"), + function(x) { + signum(x) + }) + +#' @details +#' \code{sin}: Computes the sine of the given value. Units in radians. #' -#' @rdname sin -#' @name sin -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases sin sin,Column-method #' @export -#' @examples \dontrun{sin(df$c)} +#' @note sin since 1.5.0 setMethod("sin", signature(x = "Column"), function(x) { @@ -926,15 +1425,13 @@ setMethod("sin", column(jc) }) -#' sinh -#' -#' Computes the hyperbolic sine of the given value. +#' @details +#' \code{sinh}: Computes the hyperbolic sine of the given value. #' -#' @rdname sinh -#' @name sinh -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases sinh sinh,Column-method #' @export -#' @examples \dontrun{sinh(df$c)} +#' @note sinh since 1.5.0 setMethod("sinh", signature(x = "Column"), function(x) { @@ -942,31 +1439,27 @@ setMethod("sinh", column(jc) }) -#' size +#' @details +#' \code{skewness}: Returns the skewness of the values in a group. #' -#' Returns length of array or map. -#' -#' @rdname size -#' @name size -#' @family collection_funcs +#' @rdname column_aggregate_functions +#' @aliases skewness skewness,Column-method #' @export -#' @examples \dontrun{size(df$c)} -setMethod("size", +#' @note skewness since 1.6.0 +setMethod("skewness", signature(x = "Column"), function(x) { - jc <- callJStatic("org.apache.spark.sql.functions", "size", x@jc) + jc <- callJStatic("org.apache.spark.sql.functions", "skewness", x@jc) column(jc) }) -#' soundex -#' -#' Return the soundex code for the specified expression. +#' @details +#' \code{soundex}: Returns the soundex code for the specified expression. #' -#' @rdname soundex -#' @name soundex -#' @family string_funcs +#' @rdname column_string_functions +#' @aliases soundex soundex,Column-method #' @export -#' @examples \dontrun{soundex(df$c)} +#' @note soundex since 1.5.0 setMethod("soundex", signature(x = "Column"), function(x) { @@ -974,15 +1467,100 @@ setMethod("soundex", column(jc) }) -#' sqrt +#' @details +#' \code{spark_partition_id}: Returns the partition ID as a SparkDataFrame column. +#' Note that this is nondeterministic because it depends on data partitioning and +#' task scheduling. +#' This is equivalent to the \code{SPARK_PARTITION_ID} function in SQL. #' -#' Computes the square root of the specified float value. +#' @rdname column_nonaggregate_functions +#' @aliases spark_partition_id spark_partition_id,missing-method +#' @export +#' @examples +#' +#' \dontrun{head(select(df, spark_partition_id()))} +#' @note spark_partition_id since 2.0.0 +setMethod("spark_partition_id", + signature("missing"), + function() { + jc <- callJStatic("org.apache.spark.sql.functions", "spark_partition_id") + column(jc) + }) + +#' @details +#' \code{stddev}: Alias for \code{std_dev}. +#' +#' @rdname column_aggregate_functions +#' @aliases stddev stddev,Column-method +#' @note stddev since 1.6.0 +setMethod("stddev", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "stddev", x@jc) + column(jc) + }) + +#' @details +#' \code{stddev_pop}: Returns the population standard deviation of the expression in a group. +#' +#' @rdname column_aggregate_functions +#' @aliases stddev_pop stddev_pop,Column-method +#' @export +#' @note stddev_pop since 1.6.0 +setMethod("stddev_pop", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "stddev_pop", x@jc) + column(jc) + }) + +#' @details +#' \code{stddev_samp}: Returns the unbiased sample standard deviation of the expression in a group. +#' +#' @rdname column_aggregate_functions +#' @aliases stddev_samp stddev_samp,Column-method +#' @export +#' @note stddev_samp since 1.6.0 +setMethod("stddev_samp", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "stddev_samp", x@jc) + column(jc) + }) + +#' @details +#' \code{struct}: Creates a new struct column that composes multiple input columns. #' -#' @rdname sqrt -#' @name sqrt -#' @family math_funcs +#' @rdname column_nonaggregate_functions +#' @aliases struct struct,characterOrColumn-method #' @export -#' @examples \dontrun{sqrt(df$c)} +#' @examples +#' +#' \dontrun{ +#' tmp <- mutate(df, v1 = struct(df$mpg, df$cyl), v2 = struct("hp", "wt", "vs"), +#' v3 = create_array(df$mpg, df$cyl, df$hp), +#' v4 = create_map(lit("x"), lit(1.0), lit("y"), lit(-1.0))) +#' head(tmp)} +#' @note struct since 1.6.0 +setMethod("struct", + signature(x = "characterOrColumn"), + function(x, ...) { + if (class(x) == "Column") { + jcols <- lapply(list(x, ...), function(x) { x@jc }) + jc <- callJStatic("org.apache.spark.sql.functions", "struct", jcols) + } else { + jc <- callJStatic("org.apache.spark.sql.functions", "struct", x, list(...)) + } + column(jc) + }) + +#' @details +#' \code{sqrt}: Computes the square root of the specified float value. +#' +#' @rdname column_math_functions +#' @aliases sqrt sqrt,Column-method +#' @export +#' @note sqrt since 1.5.0 setMethod("sqrt", signature(x = "Column"), function(x) { @@ -990,15 +1568,13 @@ setMethod("sqrt", column(jc) }) -#' sum +#' @details +#' \code{sum}: Returns the sum of all values in the expression. #' -#' Aggregate function: returns the sum of all values in the expression. -#' -#' @rdname sum -#' @name sum -#' @family agg_funcs +#' @rdname column_aggregate_functions +#' @aliases sum sum,Column-method #' @export -#' @examples \dontrun{sum(df$c)} +#' @note sum since 1.5.0 setMethod("sum", signature(x = "Column"), function(x) { @@ -1006,15 +1582,18 @@ setMethod("sum", column(jc) }) -#' sumDistinct -#' -#' Aggregate function: returns the sum of distinct values in the expression. +#' @details +#' \code{sumDistinct}: Returns the sum of distinct values in the expression. #' -#' @rdname sumDistinct -#' @name sumDistinct -#' @family agg_funcs +#' @rdname column_aggregate_functions +#' @aliases sumDistinct sumDistinct,Column-method #' @export -#' @examples \dontrun{sumDistinct(df$c)} +#' @examples +#' +#' \dontrun{ +#' head(select(df, sumDistinct(df$gear))) +#' head(distinct(select(df, "gear")))} +#' @note sumDistinct since 1.4.0 setMethod("sumDistinct", signature(x = "Column"), function(x) { @@ -1022,15 +1601,13 @@ setMethod("sumDistinct", column(jc) }) -#' tan +#' @details +#' \code{tan}: Computes the tangent of the given value. Units in radians. #' -#' Computes the tangent of the given value. -#' -#' @rdname tan -#' @name tan -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases tan tan,Column-method #' @export -#' @examples \dontrun{tan(df$c)} +#' @note tan since 1.5.0 setMethod("tan", signature(x = "Column"), function(x) { @@ -1038,15 +1615,13 @@ setMethod("tan", column(jc) }) -#' tanh -#' -#' Computes the hyperbolic tangent of the given value. +#' @details +#' \code{tanh}: Computes the hyperbolic tangent of the given value. #' -#' @rdname tanh -#' @name tanh -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases tanh tanh,Column-method #' @export -#' @examples \dontrun{tanh(df$c)} +#' @note tanh since 1.5.0 setMethod("tanh", signature(x = "Column"), function(x) { @@ -1054,15 +1629,14 @@ setMethod("tanh", column(jc) }) -#' toDegrees +#' @details +#' \code{toDegrees}: Converts an angle measured in radians to an approximately equivalent angle +#' measured in degrees. #' -#' Converts an angle measured in radians to an approximately equivalent angle measured in degrees. -#' -#' @rdname toDegrees -#' @name toDegrees -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases toDegrees toDegrees,Column-method #' @export -#' @examples \dontrun{toDegrees(df$c)} +#' @note toDegrees since 1.4.0 setMethod("toDegrees", signature(x = "Column"), function(x) { @@ -1070,15 +1644,14 @@ setMethod("toDegrees", column(jc) }) -#' toRadians -#' -#' Converts an angle measured in degrees to an approximately equivalent angle measured in radians. +#' @details +#' \code{toRadians}: Converts an angle measured in degrees to an approximately equivalent angle +#' measured in radians. #' -#' @rdname toRadians -#' @name toRadians -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases toRadians toRadians,Column-method #' @export -#' @examples \dontrun{toRadians(df$c)} +#' @note toRadians since 1.4.0 setMethod("toRadians", signature(x = "Column"), function(x) { @@ -1086,31 +1659,119 @@ setMethod("toRadians", column(jc) }) -#' to_date -#' -#' Converts the column into DateType. +#' @details +#' \code{to_date}: Converts the column into a DateType. You may optionally specify +#' a format according to the rules in: +#' \url{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}. +#' If the string cannot be parsed according to the specified format (or default), +#' the value of the column will be null. +#' By default, it follows casting rules to a DateType if the format is omitted +#' (equivalent to \code{cast(df$x, "date")}). #' -#' @rdname to_date -#' @name to_date -#' @family datetime_funcs +#' @rdname column_datetime_functions +#' @aliases to_date to_date,Column,missing-method #' @export -#' @examples \dontrun{to_date(df$c)} +#' @examples +#' +#' \dontrun{ +#' tmp <- createDataFrame(data.frame(time_string = dts)) +#' tmp2 <- mutate(tmp, date1 = to_date(tmp$time_string), +#' date2 = to_date(tmp$time_string, "yyyy-MM-dd"), +#' date3 = date_format(tmp$time_string, "MM/dd/yyy"), +#' time1 = to_timestamp(tmp$time_string), +#' time2 = to_timestamp(tmp$time_string, "yyyy-MM-dd")) +#' head(tmp2)} +#' @note to_date(Column) since 1.5.0 setMethod("to_date", - signature(x = "Column"), - function(x) { + signature(x = "Column", format = "missing"), + function(x, format) { jc <- callJStatic("org.apache.spark.sql.functions", "to_date", x@jc) column(jc) }) -#' trim +#' @rdname column_datetime_functions +#' @aliases to_date,Column,character-method +#' @export +#' @note to_date(Column, character) since 2.2.0 +setMethod("to_date", + signature(x = "Column", format = "character"), + function(x, format) { + jc <- callJStatic("org.apache.spark.sql.functions", "to_date", x@jc, format) + column(jc) + }) + +#' @details +#' \code{to_json}: Converts a column containing a \code{structType}, array of \code{structType}, +#' a \code{mapType} or array of \code{mapType} into a Column of JSON string. +#' Resolving the Column can fail if an unsupported type is encountered. +#' +#' @rdname column_collection_functions +#' @aliases to_json to_json,Column-method +#' @export +#' @examples #' -#' Trim the spaces from both ends for the specified string column. +#' \dontrun{ +#' # Converts a struct into a JSON object +#' df2 <- sql("SELECT named_struct('date', cast('2000-01-01' as date)) as d") +#' select(df2, to_json(df2$d, dateFormat = 'dd/MM/yyyy')) +#' +#' # Converts an array of structs into a JSON array +#' df2 <- sql("SELECT array(named_struct('name', 'Bob'), named_struct('name', 'Alice')) as people") +#' df2 <- mutate(df2, people_json = to_json(df2$people)) +#' +#' # Converts a map into a JSON object +#' df2 <- sql("SELECT map('name', 'Bob')) as people") +#' df2 <- mutate(df2, people_json = to_json(df2$people)) +#' +#' # Converts an array of maps into a JSON array +#' df2 <- sql("SELECT array(map('name', 'Bob'), map('name', 'Alice')) as people") +#' df2 <- mutate(df2, people_json = to_json(df2$people))} +#' @note to_json since 2.2.0 +setMethod("to_json", signature(x = "Column"), + function(x, ...) { + options <- varargsToStrEnv(...) + jc <- callJStatic("org.apache.spark.sql.functions", "to_json", x@jc, options) + column(jc) + }) + +#' @details +#' \code{to_timestamp}: Converts the column into a TimestampType. You may optionally specify +#' a format according to the rules in: +#' \url{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}. +#' If the string cannot be parsed according to the specified format (or default), +#' the value of the column will be null. +#' By default, it follows casting rules to a TimestampType if the format is omitted +#' (equivalent to \code{cast(df$x, "timestamp")}). #' -#' @rdname trim -#' @name trim -#' @family string_funcs +#' @rdname column_datetime_functions +#' @aliases to_timestamp to_timestamp,Column,missing-method #' @export -#' @examples \dontrun{trim(df$c)} +#' @note to_timestamp(Column) since 2.2.0 +setMethod("to_timestamp", + signature(x = "Column", format = "missing"), + function(x, format) { + jc <- callJStatic("org.apache.spark.sql.functions", "to_timestamp", x@jc) + column(jc) + }) + +#' @rdname column_datetime_functions +#' @aliases to_timestamp,Column,character-method +#' @export +#' @note to_timestamp(Column, character) since 2.2.0 +setMethod("to_timestamp", + signature(x = "Column", format = "character"), + function(x, format) { + jc <- callJStatic("org.apache.spark.sql.functions", "to_timestamp", x@jc, format) + column(jc) + }) + +#' @details +#' \code{trim}: Trims the spaces from both ends for the specified string column. +#' +#' @rdname column_string_functions +#' @aliases trim trim,Column-method +#' @export +#' @note trim since 1.5.0 setMethod("trim", signature(x = "Column"), function(x) { @@ -1118,16 +1779,14 @@ setMethod("trim", column(jc) }) -#' unbase64 -#' -#' Decodes a BASE64 encoded string column and returns it as a binary column. +#' @details +#' \code{unbase64}: Decodes a BASE64 encoded string column and returns it as a binary column. #' This is the reverse of base64. #' -#' @rdname unbase64 -#' @name unbase64 -#' @family string_funcs +#' @rdname column_string_functions +#' @aliases unbase64 unbase64,Column-method #' @export -#' @examples \dontrun{unbase64(df$c)} +#' @note unbase64 since 1.5.0 setMethod("unbase64", signature(x = "Column"), function(x) { @@ -1135,16 +1794,14 @@ setMethod("unbase64", column(jc) }) -#' unhex -#' -#' Inverse of hex. Interprets each pair of characters as a hexadecimal number +#' @details +#' \code{unhex}: Inverse of hex. Interprets each pair of characters as a hexadecimal number #' and converts to the byte representation of number. #' -#' @rdname unhex -#' @name unhex -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases unhex unhex,Column-method #' @export -#' @examples \dontrun{unhex(df$c)} +#' @note unhex since 1.5.0 setMethod("unhex", signature(x = "Column"), function(x) { @@ -1152,15 +1809,13 @@ setMethod("unhex", column(jc) }) -#' upper +#' @details +#' \code{upper}: Converts a string column to upper case. #' -#' Converts a string column to upper case. -#' -#' @rdname upper -#' @name upper -#' @family string_funcs +#' @rdname column_string_functions +#' @aliases upper upper,Column-method #' @export -#' @examples \dontrun{upper(df$c)} +#' @note upper since 1.4.0 setMethod("upper", signature(x = "Column"), function(x) { @@ -1168,15 +1823,70 @@ setMethod("upper", column(jc) }) -#' weekofyear +#' @details +#' \code{var}: Alias for \code{var_samp}. +#' +#' @rdname column_aggregate_functions +#' @aliases var var,Column-method +#' @export +#' @examples #' -#' Extracts the week number as an integer from a given date/timestamp/string. +#'\dontrun{ +#'head(agg(df, var(df$mpg), variance(df$mpg), var_pop(df$mpg), var_samp(df$mpg)))} +#' @note var since 1.6.0 +setMethod("var", + signature(x = "Column"), + function(x) { + # In R, sample variance is calculated with the var() function. + var_samp(x) + }) + +#' @rdname column_aggregate_functions +#' @aliases variance variance,Column-method +#' @export +#' @note variance since 1.6.0 +setMethod("variance", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "variance", x@jc) + column(jc) + }) + +#' @details +#' \code{var_pop}: Returns the population variance of the values in a group. +#' +#' @rdname column_aggregate_functions +#' @aliases var_pop var_pop,Column-method +#' @export +#' @note var_pop since 1.5.0 +setMethod("var_pop", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "var_pop", x@jc) + column(jc) + }) + +#' @details +#' \code{var_samp}: Returns the unbiased variance of the values in a group. +#' +#' @rdname column_aggregate_functions +#' @aliases var_samp var_samp,Column-method +#' @export +#' @note var_samp since 1.6.0 +setMethod("var_samp", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "var_samp", x@jc) + column(jc) + }) + +#' @details +#' \code{weekofyear}: Extracts the week number as an integer from a given date/timestamp/string. #' -#' @rdname weekofyear -#' @name weekofyear -#' @family datetime_funcs +#' @rdname column_datetime_functions +#' @aliases weekofyear weekofyear,Column-method #' @export -#' @examples \dontrun{weekofyear(df$c)} +#' @note weekofyear since 1.5.0 setMethod("weekofyear", signature(x = "Column"), function(x) { @@ -1184,15 +1894,13 @@ setMethod("weekofyear", column(jc) }) -#' year -#' -#' Extracts the year as an integer from a given date/timestamp/string. +#' @details +#' \code{year}: Extracts the year as an integer from a given date/timestamp/string. #' -#' @rdname year -#' @name year -#' @family datetime_funcs +#' @rdname column_datetime_functions +#' @aliases year year,Column-method #' @export -#' @examples \dontrun{year(df$c)} +#' @note year since 1.5.0 setMethod("year", signature(x = "Column"), function(x) { @@ -1200,16 +1908,14 @@ setMethod("year", column(jc) }) -#' atan2 +#' @details +#' \code{atan2}: Returns the angle theta from the conversion of rectangular coordinates +#' (x, y) to polar coordinates (r, theta). Units in radians. #' -#' Returns the angle theta from the conversion of rectangular coordinates (x, y) to -#' polar coordinates (r, theta). -#' -#' @rdname atan2 -#' @name atan2 -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases atan2 atan2,Column-method #' @export -#' @examples \dontrun{atan2(df$c, x)} +#' @note atan2 since 1.5.0 setMethod("atan2", signature(y = "Column"), function(y, x) { if (class(x) == "Column") { @@ -1219,15 +1925,21 @@ setMethod("atan2", signature(y = "Column"), column(jc) }) -#' datediff -#' -#' Returns the number of days from `start` to `end`. +#' @details +#' \code{datediff}: Returns the number of days from \code{y} to \code{x}. #' -#' @rdname datediff -#' @name datediff -#' @family datetime_funcs +#' @rdname column_datetime_diff_functions +#' @aliases datediff datediff,Column-method #' @export -#' @examples \dontrun{datediff(df$c, x)} +#' @examples +#' +#' \dontrun{ +#' tmp <- createDataFrame(data.frame(time_string1 = as.POSIXct(dts), +#' time_string2 = as.POSIXct(dts[order(runif(length(dts)))]))) +#' tmp2 <- mutate(tmp, datediff = datediff(tmp$time_string1, tmp$time_string2), +#' monthdiff = months_between(tmp$time_string1, tmp$time_string2)) +#' head(tmp2)} +#' @note datediff since 1.5.0 setMethod("datediff", signature(y = "Column"), function(y, x) { if (class(x) == "Column") { @@ -1237,15 +1949,13 @@ setMethod("datediff", signature(y = "Column"), column(jc) }) -#' hypot +#' @details +#' \code{hypot}: Computes "sqrt(a^2 + b^2)" without intermediate overflow or underflow. #' -#' Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. -#' -#' @rdname hypot -#' @name hypot -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases hypot hypot,Column-method #' @export -#' @examples \dontrun{hypot(df$c, x)} +#' @note hypot since 1.4.0 setMethod("hypot", signature(y = "Column"), function(y, x) { if (class(x) == "Column") { @@ -1255,15 +1965,20 @@ setMethod("hypot", signature(y = "Column"), column(jc) }) -#' levenshtein -#' -#' Computes the Levenshtein distance of the two given string columns. +#' @details +#' \code{levenshtein}: Computes the Levenshtein distance of the two given string columns. #' -#' @rdname levenshtein -#' @name levenshtein -#' @family string_funcs +#' @rdname column_string_functions +#' @aliases levenshtein levenshtein,Column-method #' @export -#' @examples \dontrun{levenshtein(df$c, x)} +#' @examples +#' +#' \dontrun{ +#' tmp <- mutate(df, d1 = levenshtein(df$Class, df$Sex), +#' d2 = levenshtein(df$Age, df$Sex), +#' d3 = levenshtein(df$Age, df$Age)) +#' head(tmp)} +#' @note levenshtein since 1.5.0 setMethod("levenshtein", signature(y = "Column"), function(y, x) { if (class(x) == "Column") { @@ -1273,15 +1988,13 @@ setMethod("levenshtein", signature(y = "Column"), column(jc) }) -#' months_between -#' -#' Returns number of months between dates `date1` and `date2`. +#' @details +#' \code{months_between}: Returns number of months between dates \code{y} and \code{x}. #' -#' @rdname months_between -#' @name months_between -#' @family datetime_funcs +#' @rdname column_datetime_diff_functions +#' @aliases months_between months_between,Column-method #' @export -#' @examples \dontrun{months_between(df$c, x)} +#' @note months_between since 1.5.0 setMethod("months_between", signature(y = "Column"), function(y, x) { if (class(x) == "Column") { @@ -1291,16 +2004,14 @@ setMethod("months_between", signature(y = "Column"), column(jc) }) -#' nanvl -#' -#' Returns col1 if it is not NaN, or col2 if col1 is NaN. -#' hhBoth inputs should be floating point columns (DoubleType or FloatType). +#' @details +#' \code{nanvl}: Returns the first column (\code{y}) if it is not NaN, or the second column (\code{x}) if +#' the first column is NaN. Both inputs should be floating point columns (DoubleType or FloatType). #' -#' @rdname nanvl -#' @name nanvl -#' @family normal_funcs +#' @rdname column_nonaggregate_functions +#' @aliases nanvl nanvl,Column-method #' @export -#' @examples \dontrun{nanvl(df$c, x)} +#' @note nanvl since 1.5.0 setMethod("nanvl", signature(y = "Column"), function(y, x) { if (class(x) == "Column") { @@ -1310,16 +2021,14 @@ setMethod("nanvl", signature(y = "Column"), column(jc) }) -#' pmod +#' @details +#' \code{pmod}: Returns the positive value of dividend mod divisor. +#' Column \code{x} is divisor column, and column \code{y} is the dividend column. #' -#' Returns the positive value of dividend mod divisor. -#' -#' @rdname pmod -#' @name pmod -#' @docType methods -#' @family math_funcs +#' @rdname column_math_functions +#' @aliases pmod pmod,Column-method #' @export -#' @examples \dontrun{pmod(df$c, x)} +#' @note pmod since 1.5.0 setMethod("pmod", signature(y = "Column"), function(y, x) { if (class(x) == "Column") { @@ -1329,292 +2038,334 @@ setMethod("pmod", signature(y = "Column"), column(jc) }) - -#' Approx Count Distinct +#' @param rsd maximum estimation error allowed (default = 0.05). #' -#' @family agg_funcs -#' @rdname approxCountDistinct -#' @name approxCountDistinct -#' @return the approximate number of distinct items in a group. +#' @rdname column_aggregate_functions +#' @aliases approxCountDistinct,Column-method #' @export +#' @note approxCountDistinct(Column, numeric) since 1.4.0 setMethod("approxCountDistinct", signature(x = "Column"), - function(x, rsd = 0.95) { + function(x, rsd = 0.05) { jc <- callJStatic("org.apache.spark.sql.functions", "approxCountDistinct", x@jc, rsd) column(jc) }) -#' Count Distinct +#' @details +#' \code{countDistinct}: Returns the number of distinct items in a group. #' -#' @family agg_funcs -#' @rdname countDistinct -#' @name countDistinct -#' @return the number of distinct items in a group. +#' @rdname column_aggregate_functions +#' @aliases countDistinct countDistinct,Column-method #' @export +#' @note countDistinct since 1.4.0 setMethod("countDistinct", signature(x = "Column"), function(x, ...) { - jcol <- lapply(list(...), function (x) { + jcols <- lapply(list(...), function (x) { + stopifnot(class(x) == "Column") x@jc }) jc <- callJStatic("org.apache.spark.sql.functions", "countDistinct", x@jc, - jcol) + jcols) column(jc) }) - -#' concat -#' -#' Concatenates multiple input string columns together into a single string column. +#' @details +#' \code{concat}: Concatenates multiple input string columns together into a single string column. #' -#' @family string_funcs -#' @rdname concat -#' @name concat +#' @rdname column_string_functions +#' @aliases concat concat,Column-method #' @export +#' @examples +#' +#' \dontrun{ +#' # concatenate strings +#' tmp <- mutate(df, s1 = concat(df$Class, df$Sex), +#' s2 = concat(df$Class, df$Sex, df$Age), +#' s3 = concat(df$Class, df$Sex, df$Age, df$Class), +#' s4 = concat_ws("_", df$Class, df$Sex), +#' s5 = concat_ws("+", df$Class, df$Sex, df$Age, df$Survived)) +#' head(tmp)} +#' @note concat since 1.5.0 setMethod("concat", signature(x = "Column"), function(x, ...) { - jcols <- lapply(list(x, ...), function(x) { x@jc }) + jcols <- lapply(list(x, ...), function (x) { + stopifnot(class(x) == "Column") + x@jc + }) jc <- callJStatic("org.apache.spark.sql.functions", "concat", jcols) column(jc) }) -#' greatest -#' -#' Returns the greatest value of the list of column names, skipping null values. +#' @details +#' \code{greatest}: Returns the greatest value of the list of column names, skipping null values. #' This function takes at least 2 parameters. It will return null if all parameters are null. #' -#' @family normal_funcs -#' @rdname greatest -#' @name greatest +#' @rdname column_nonaggregate_functions +#' @aliases greatest greatest,Column-method #' @export +#' @note greatest since 1.5.0 setMethod("greatest", signature(x = "Column"), function(x, ...) { stopifnot(length(list(...)) > 0) - jcols <- lapply(list(x, ...), function(x) { x@jc }) + jcols <- lapply(list(x, ...), function (x) { + stopifnot(class(x) == "Column") + x@jc + }) jc <- callJStatic("org.apache.spark.sql.functions", "greatest", jcols) column(jc) }) -#' least -#' -#' Returns the least value of the list of column names, skipping null values. -#' This function takes at least 2 parameters. It will return null iff all parameters are null. +#' @details +#' \code{least}: Returns the least value of the list of column names, skipping null values. +#' This function takes at least 2 parameters. It will return null if all parameters are null. #' -#' @family normal_funcs -#' @rdname least -#' @name least +#' @rdname column_nonaggregate_functions +#' @aliases least least,Column-method #' @export +#' @note least since 1.5.0 setMethod("least", signature(x = "Column"), function(x, ...) { stopifnot(length(list(...)) > 0) - jcols <- lapply(list(x, ...), function(x) { x@jc }) + jcols <- lapply(list(x, ...), function (x) { + stopifnot(class(x) == "Column") + x@jc + }) jc <- callJStatic("org.apache.spark.sql.functions", "least", jcols) column(jc) }) -#' ceiling -#' -#' Computes the ceiling of the given value. +#' @details +#' \code{n_distinct}: Returns the number of distinct items in a group. #' -#' @family math_funcs -#' @rdname ceil -#' @name ceil -#' @aliases ceil -#' @export -setMethod("ceiling", - signature(x = "Column"), - function(x) { - ceil(x) - }) - -#' sign -#' -#' Computes the signum of the given value. -#' -#' @family math_funcs -#' @rdname signum -#' @name signum -#' @aliases signum -#' @export -setMethod("sign", signature(x = "Column"), - function(x) { - signum(x) - }) - -#' n_distinct -#' -#' Aggregate function: returns the number of distinct items in a group. -#' -#' @family agg_funcs -#' @rdname countDistinct -#' @name countDistinct -#' @aliases countDistinct +#' @rdname column_aggregate_functions +#' @aliases n_distinct n_distinct,Column-method #' @export +#' @note n_distinct since 1.4.0 setMethod("n_distinct", signature(x = "Column"), function(x, ...) { countDistinct(x, ...) }) -#' n -#' -#' Aggregate function: returns the number of items in a group. -#' -#' @family agg_funcs #' @rdname count -#' @name count -#' @aliases count +#' @name n +#' @aliases n,Column-method #' @export +#' @examples \dontrun{n(df$c)} +#' @note n since 1.4.0 setMethod("n", signature(x = "Column"), function(x) { count(x) }) -#' date_format -#' -#' Converts a date/timestamp/string to a value of string in the format specified by the date -#' format given by the second argument. -#' -#' A pattern could be for instance \preformatted{dd.MM.yyyy} and could return a string like '18.03.1993'. All +#' @details +#' \code{date_format}: Converts a date/timestamp/string to a value of string in the format +#' specified by the date format given by the second argument. A pattern could be for instance +#' \code{dd.MM.yyyy} and could return a string like '18.03.1993'. All #' pattern letters of \code{java.text.SimpleDateFormat} can be used. -#' -#' NOTE: Use when ever possible specialized functions like \code{year}. These benefit from a +#' Note: Use when ever possible specialized functions like \code{year}. These benefit from a #' specialized implementation. #' -#' @family datetime_funcs -#' @rdname date_format -#' @name date_format +#' @rdname column_datetime_diff_functions +#' +#' @aliases date_format date_format,Column,character-method #' @export +#' @note date_format since 1.5.0 setMethod("date_format", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "date_format", y@jc, x) column(jc) }) -#' from_utc_timestamp +#' @details +#' \code{from_json}: Parses a column containing a JSON string into a Column of \code{structType} +#' with the specified \code{schema} or array of \code{structType} if \code{as.json.array} is set +#' to \code{TRUE}. If the string is unparseable, the Column will contain the value NA. #' -#' Assumes given timestamp is UTC and converts to given timezone. +#' @rdname column_collection_functions +#' @param schema a structType object to use as the schema to use when parsing the JSON string. +#' Since Spark 2.3, the DDL-formatted string is also supported for the schema. +#' @param as.json.array indicating if input string is JSON array of objects or a single object. +#' @aliases from_json from_json,Column,characterOrstructType-method +#' @export +#' @examples #' -#' @family datetime_funcs -#' @rdname from_utc_timestamp -#' @name from_utc_timestamp +#' \dontrun{ +#' df2 <- sql("SELECT named_struct('date', cast('2000-01-01' as date)) as d") +#' df2 <- mutate(df2, d2 = to_json(df2$d, dateFormat = 'dd/MM/yyyy')) +#' schema <- structType(structField("date", "string")) +#' head(select(df2, from_json(df2$d2, schema, dateFormat = 'dd/MM/yyyy'))) + +#' df2 <- sql("SELECT named_struct('name', 'Bob') as people") +#' df2 <- mutate(df2, people_json = to_json(df2$people)) +#' schema <- structType(structField("name", "string")) +#' head(select(df2, from_json(df2$people_json, schema))) +#' head(select(df2, from_json(df2$people_json, "name STRING")))} +#' @note from_json since 2.2.0 +setMethod("from_json", signature(x = "Column", schema = "characterOrstructType"), + function(x, schema, as.json.array = FALSE, ...) { + if (is.character(schema)) { + schema <- structType(schema) + } + + if (as.json.array) { + jschema <- callJStatic("org.apache.spark.sql.types.DataTypes", + "createArrayType", + schema$jobj) + } else { + jschema <- schema$jobj + } + options <- varargsToStrEnv(...) + jc <- callJStatic("org.apache.spark.sql.functions", + "from_json", + x@jc, jschema, options) + column(jc) + }) + +#' @details +#' \code{from_utc_timestamp}: Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a +#' time in UTC, and renders that time as a timestamp in the given time zone. For example, 'GMT+1' +#' would yield '2017-07-14 03:40:00.0'. +#' +#' @rdname column_datetime_diff_functions +#' +#' @aliases from_utc_timestamp from_utc_timestamp,Column,character-method #' @export +#' @examples +#' +#' \dontrun{ +#' tmp <- mutate(df, from_utc = from_utc_timestamp(df$time, "PST"), +#' to_utc = to_utc_timestamp(df$time, "PST")) +#' head(tmp)} +#' @note from_utc_timestamp since 1.5.0 setMethod("from_utc_timestamp", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "from_utc_timestamp", y@jc, x) column(jc) }) -#' instr -#' -#' Locate the position of the first occurrence of substr column in the given string. -#' Returns null if either of the arguments are null. +#' @details +#' \code{instr}: Locates the position of the first occurrence of a substring (\code{x}) +#' in the given string column (\code{y}). Returns null if either of the arguments are null. +#' Note: The position is not zero based, but 1 based index. Returns 0 if the substring +#' could not be found in the string column. #' -#' NOTE: The position is not zero based, but 1 based index, returns 0 if substr -#' could not be found in str. -#' -#' @family string_funcs -#' @rdname instr -#' @name instr +#' @rdname column_string_functions +#' @aliases instr instr,Column,character-method #' @export +#' @examples +#' +#' \dontrun{ +#' tmp <- mutate(df, s1 = instr(df$Sex, "m"), s2 = instr(df$Sex, "M"), +#' s3 = locate("m", df$Sex), s4 = locate("m", df$Sex, pos = 4)) +#' head(tmp)} +#' @note instr since 1.5.0 setMethod("instr", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "instr", y@jc, x) column(jc) }) -#' next_day -#' -#' Given a date column, returns the first date which is later than the value of the date column -#' that is on the specified day of the week. -#' -#' For example, \code{next_day('2015-07-27', "Sunday")} returns 2015-08-02 because that is the first -#' Sunday after 2015-07-27. +#' @details +#' \code{next_day}: Given a date column, returns the first date which is later than the value of +#' the date column that is on the specified day of the week. For example, +#' \code{next_day("2015-07-27", "Sunday")} returns 2015-08-02 because that is the first Sunday +#' after 2015-07-27. Day of the week parameter is case insensitive, and accepts first three or +#' two characters: "Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun". #' -#' Day of the week parameter is case insensitive, and accepts: -#' "Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun". -#' -#' @family datetime_funcs -#' @rdname next_day -#' @name next_day +#' @rdname column_datetime_diff_functions +#' @aliases next_day next_day,Column,character-method #' @export +#' @note next_day since 1.5.0 setMethod("next_day", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "next_day", y@jc, x) column(jc) }) -#' to_utc_timestamp -#' -#' Assumes given timestamp is in given timezone and converts to UTC. +#' @details +#' \code{to_utc_timestamp}: Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a +#' time in the given time zone, and renders that time as a timestamp in UTC. For example, 'GMT+1' +#' would yield '2017-07-14 01:40:00.0'. #' -#' @family datetime_funcs -#' @rdname to_utc_timestamp -#' @name to_utc_timestamp +#' @rdname column_datetime_diff_functions +#' @aliases to_utc_timestamp to_utc_timestamp,Column,character-method #' @export +#' @note to_utc_timestamp since 1.5.0 setMethod("to_utc_timestamp", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "to_utc_timestamp", y@jc, x) column(jc) }) -#' add_months +#' @details +#' \code{add_months}: Returns the date that is numMonths (\code{x}) after startDate (\code{y}). #' -#' Returns the date that is numMonths after startDate. -#' -#' @name add_months -#' @family datetime_funcs -#' @rdname add_months -#' @name add_months +#' @rdname column_datetime_diff_functions +#' @aliases add_months add_months,Column,numeric-method #' @export +#' @examples +#' +#' \dontrun{ +#' tmp <- mutate(df, t1 = add_months(df$time, 1), +#' t2 = date_add(df$time, 2), +#' t3 = date_sub(df$time, 3), +#' t4 = next_day(df$time, "Sun")) +#' head(tmp)} +#' @note add_months since 1.5.0 setMethod("add_months", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "add_months", y@jc, as.integer(x)) column(jc) }) -#' date_add -#' -#' Returns the date that is `days` days after `start` +#' @details +#' \code{date_add}: Returns the date that is \code{x} days after. #' -#' @family datetime_funcs -#' @rdname date_add -#' @name date_add +#' @rdname column_datetime_diff_functions +#' @aliases date_add date_add,Column,numeric-method #' @export +#' @note date_add since 1.5.0 setMethod("date_add", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "date_add", y@jc, as.integer(x)) column(jc) }) -#' date_sub +#' @details +#' \code{date_sub}: Returns the date that is \code{x} days before. #' -#' Returns the date that is `days` days before `start` +#' @rdname column_datetime_diff_functions #' -#' @family datetime_funcs -#' @rdname date_sub -#' @name date_sub +#' @aliases date_sub date_sub,Column,numeric-method #' @export +#' @note date_sub since 1.5.0 setMethod("date_sub", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "date_sub", y@jc, as.integer(x)) column(jc) }) -#' format_number +#' @details +#' \code{format_number}: Formats numeric column \code{y} to a format like '#,###,###.##', +#' rounded to \code{x} decimal places with HALF_EVEN round mode, and returns the result +#' as a string column. +#' If \code{x} is 0, the result has no decimal point or fractional part. +#' If \code{x} < 0, the result will be null. #' -#' Formats numeric column x to a format like '#,###,###.##', rounded to d decimal places, -#' and returns the result as a string column. -#' -#' If d is 0, the result has no decimal point or fractional part. -#' If d < 0, the result will be null.' -#' -#' @family string_funcs -#' @rdname format_number -#' @name format_number +#' @rdname column_string_functions +#' @aliases format_number format_number,Column,numeric-method #' @export +#' @examples +#' +#' \dontrun{ +#' tmp <- mutate(df, v1 = df$Freq/3) +#' head(select(tmp, format_number(tmp$v1, 0), format_number(tmp$v1, 2), +#' format_string("%4.2f %s", tmp$v1, tmp$Sex)), 10)} +#' @note format_number since 1.5.0 setMethod("format_number", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -1623,32 +2374,29 @@ setMethod("format_number", signature(y = "Column", x = "numeric"), column(jc) }) -#' sha2 -#' -#' Calculates the SHA-2 family of hash functions of a binary column and -#' returns the value as a hex string. +#' @details +#' \code{sha2}: Calculates the SHA-2 family of hash functions of a binary column and +#' returns the value as a hex string. The second argument \code{x} specifies the number +#' of bits, and is one of 224, 256, 384, or 512. #' -#' @param y column to compute SHA-2 on. -#' @param x one of 224, 256, 384, or 512. -#' @family misc_funcs -#' @rdname sha2 -#' @name sha2 +#' @rdname column_misc_functions +#' @aliases sha2 sha2,Column,numeric-method #' @export +#' @note sha2 since 1.5.0 setMethod("sha2", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "sha2", y@jc, as.integer(x)) column(jc) }) -#' shiftLeft -#' -#' Shift the the given value numBits left. If the given value is a long value, this function -#' will return a long value else it will return an integer value. +#' @details +#' \code{shiftLeft}: Shifts the given value numBits left. If the given value is a long value, +#' this function will return a long value else it will return an integer value. #' -#' @family math_funcs -#' @rdname shiftLeft -#' @name shiftLeft +#' @rdname column_math_functions +#' @aliases shiftLeft shiftLeft,Column,numeric-method #' @export +#' @note shiftLeft since 1.5.0 setMethod("shiftLeft", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -1657,15 +2405,14 @@ setMethod("shiftLeft", signature(y = "Column", x = "numeric"), column(jc) }) -#' shiftRight -#' -#' Shift the the given value numBits right. If the given value is a long value, it will return -#' a long value else it will return an integer value. +#' @details +#' \code{shiftRight}: (Signed) shifts the given value numBits right. If the given value is a long value, +#' it will return a long value else it will return an integer value. #' -#' @family math_funcs -#' @rdname shiftRight -#' @name shiftRight +#' @rdname column_math_functions +#' @aliases shiftRight shiftRight,Column,numeric-method #' @export +#' @note shiftRight since 1.5.0 setMethod("shiftRight", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -1674,15 +2421,14 @@ setMethod("shiftRight", signature(y = "Column", x = "numeric"), column(jc) }) -#' shiftRightUnsigned -#' -#' Unsigned shift the the given value numBits right. If the given value is a long value, -#' it will return a long value else it will return an integer value. +#' @details +#' \code{shiftRightUnsigned}: (Unigned) shifts the given value numBits right. If the given value is +#' a long value, it will return a long value else it will return an integer value. #' -#' @family math_funcs -#' @rdname shiftRightUnsigned -#' @name shiftRightUnsigned +#' @rdname column_math_functions +#' @aliases shiftRightUnsigned shiftRightUnsigned,Column,numeric-method #' @export +#' @note shiftRightUnsigned since 1.5.0 setMethod("shiftRightUnsigned", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -1691,15 +2437,15 @@ setMethod("shiftRightUnsigned", signature(y = "Column", x = "numeric"), column(jc) }) -#' concat_ws +#' @details +#' \code{concat_ws}: Concatenates multiple input string columns together into a single +#' string column, using the given separator. #' -#' Concatenates multiple input string columns together into a single string column, -#' using the given separator. -#' -#' @family string_funcs -#' @rdname concat_ws -#' @name concat_ws +#' @param sep separator to use. +#' @rdname column_string_functions +#' @aliases concat_ws concat_ws,character,Column-method #' @export +#' @note concat_ws since 1.5.0 setMethod("concat_ws", signature(sep = "character", x = "Column"), function(sep, x, ...) { jcols <- lapply(list(x, ...), function(x) { x@jc }) @@ -1707,14 +2453,15 @@ setMethod("concat_ws", signature(sep = "character", x = "Column"), column(jc) }) -#' conv -#' -#' Convert a number in a string column from one base to another. +#' @details +#' \code{conv}: Converts a number in a string column from one base to another. #' -#' @family math_funcs -#' @rdname conv -#' @name conv +#' @param fromBase base to convert from. +#' @param toBase base to convert to. +#' @rdname column_math_functions +#' @aliases conv conv,Column,numeric,numeric-method #' @export +#' @note conv since 1.5.0 setMethod("conv", signature(x = "Column", fromBase = "numeric", toBase = "numeric"), function(x, fromBase, toBase) { fromBase <- as.integer(fromBase) @@ -1725,29 +2472,29 @@ setMethod("conv", signature(x = "Column", fromBase = "numeric", toBase = "numeri column(jc) }) -#' expr -#' -#' Parses the expression string into the column that it represents, similar to -#' DataFrame.selectExpr +#' @details +#' \code{expr}: Parses the expression string into the column that it represents, similar to +#' \code{SparkDataFrame.selectExpr} #' -#' @family normal_funcs -#' @rdname expr -#' @name expr +#' @rdname column_nonaggregate_functions +#' @aliases expr expr,character-method #' @export +#' @note expr since 1.5.0 setMethod("expr", signature(x = "character"), function(x) { jc <- callJStatic("org.apache.spark.sql.functions", "expr", x) column(jc) }) -#' format_string +#' @details +#' \code{format_string}: Formats the arguments in printf-style and returns the result +#' as a string column. #' -#' Formats the arguments in printf-style and returns the result as a string column. -#' -#' @family string_funcs -#' @rdname format_string -#' @name format_string +#' @param format a character object of format strings. +#' @rdname column_string_functions +#' @aliases format_string format_string,character,Column-method #' @export +#' @note format_string since 1.5.0 setMethod("format_string", signature(format = "character", x = "Column"), function(format, x, ...) { jcols <- lapply(list(x, ...), function(arg) { arg@jc }) @@ -1757,16 +2504,25 @@ setMethod("format_string", signature(format = "character", x = "Column"), column(jc) }) -#' from_unixtime +#' @details +#' \code{from_unixtime}: Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) to a +#' string representing the timestamp of that moment in the current system time zone in the JVM in the +#' given format. See \href{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}{ +#' Customizing Formats} for available options. #' -#' Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) to a string -#' representing the timestamp of that moment in the current system time zone in the given -#' format. +#' @rdname column_datetime_functions #' -#' @family datetime_funcs -#' @rdname from_unixtime -#' @name from_unixtime +#' @aliases from_unixtime from_unixtime,Column-method #' @export +#' @examples +#' +#' \dontrun{ +#' tmp <- mutate(df, to_unix = unix_timestamp(df$time), +#' to_unix2 = unix_timestamp(df$time, 'yyyy-MM-dd HH'), +#' from_unix = from_unixtime(unix_timestamp(df$time)), +#' from_unix2 = from_unixtime(unix_timestamp(df$time), 'yyyy-MM-dd HH:mm')) +#' head(tmp)} +#' @note from_unixtime since 1.5.0 setMethod("from_unixtime", signature(x = "Column"), function(x, format = "yyyy-MM-dd HH:mm:ss") { jc <- callJStatic("org.apache.spark.sql.functions", @@ -1775,32 +2531,100 @@ setMethod("from_unixtime", signature(x = "Column"), column(jc) }) -#' locate +#' @details +#' \code{window}: Bucketizes rows into one or more time windows given a timestamp specifying column. +#' Window starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window +#' [12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in +#' the order of months are not supported. It returns an output column of struct called 'window' +#' by default with the nested columns 'start' and 'end' +#' +#' @param windowDuration a string specifying the width of the window, e.g. '1 second', +#' '1 day 12 hours', '2 minutes'. Valid interval strings are 'week', +#' 'day', 'hour', 'minute', 'second', 'millisecond', 'microsecond'. Note that +#' the duration is a fixed length of time, and does not vary over time +#' according to a calendar. For example, '1 day' always means 86,400,000 +#' milliseconds, not a calendar day. +#' @param slideDuration a string specifying the sliding interval of the window. Same format as +#' \code{windowDuration}. A new window will be generated every +#' \code{slideDuration}. Must be less than or equal to +#' the \code{windowDuration}. This duration is likewise absolute, and does not +#' vary according to a calendar. +#' @param startTime the offset with respect to 1970-01-01 00:00:00 UTC with which to start +#' window intervals. For example, in order to have hourly tumbling windows +#' that start 15 minutes past the hour, e.g. 12:15-13:15, 13:15-14:15... provide +#' \code{startTime} as \code{"15 minutes"}. +#' @rdname column_datetime_functions +#' @aliases window window,Column-method +#' @export +#' @examples #' -#' Locate the position of the first occurrence of substr. -#' NOTE: The position is not zero based, but 1 based index, returns 0 if substr +#' \dontrun{ +#' # One minute windows every 15 seconds 10 seconds after the minute, e.g. 09:00:10-09:01:10, +#' # 09:00:25-09:01:25, 09:00:40-09:01:40, ... +#' window(df$time, "1 minute", "15 seconds", "10 seconds") +#' +#' # One minute tumbling windows 15 seconds after the minute, e.g. 09:00:15-09:01:15, +#' # 09:01:15-09:02:15... +#' window(df$time, "1 minute", startTime = "15 seconds") +#' +#' # Thirty-second windows every 10 seconds, e.g. 09:00:00-09:00:30, 09:00:10-09:00:40, ... +#' window(df$time, "30 seconds", "10 seconds")} +#' @note window since 2.0.0 +setMethod("window", signature(x = "Column"), + function(x, windowDuration, slideDuration = NULL, startTime = NULL) { + stopifnot(is.character(windowDuration)) + if (!is.null(slideDuration) && !is.null(startTime)) { + stopifnot(is.character(slideDuration) && is.character(startTime)) + jc <- callJStatic("org.apache.spark.sql.functions", + "window", + x@jc, windowDuration, slideDuration, startTime) + } else if (!is.null(slideDuration)) { + stopifnot(is.character(slideDuration)) + jc <- callJStatic("org.apache.spark.sql.functions", + "window", + x@jc, windowDuration, slideDuration) + } else if (!is.null(startTime)) { + stopifnot(is.character(startTime)) + jc <- callJStatic("org.apache.spark.sql.functions", + "window", + x@jc, windowDuration, windowDuration, startTime) + } else { + jc <- callJStatic("org.apache.spark.sql.functions", + "window", + x@jc, windowDuration) + } + column(jc) + }) + +#' @details +#' \code{locate}: Locates the position of the first occurrence of substr. +#' Note: The position is not zero based, but 1 based index. Returns 0 if substr #' could not be found in str. #' -#' @family string_funcs -#' @rdname locate -#' @name locate +#' @param substr a character string to be matched. +#' @param str a Column where matches are sought for each entry. +#' @param pos start position of search. +#' @rdname column_string_functions +#' @aliases locate locate,character,Column-method #' @export +#' @note locate since 1.5.0 setMethod("locate", signature(substr = "character", str = "Column"), - function(substr, str, pos = 0) { + function(substr, str, pos = 1) { jc <- callJStatic("org.apache.spark.sql.functions", "locate", substr, str@jc, as.integer(pos)) column(jc) }) -#' lpad -#' -#' Left-pad the string column with +#' @details +#' \code{lpad}: Left-padded with pad to a length of len. #' -#' @family string_funcs -#' @rdname lpad -#' @name lpad +#' @param len maximum length of each output result. +#' @param pad a character string to be padded with. +#' @rdname column_string_functions +#' @aliases lpad lpad,Column,numeric,character-method #' @export +#' @note lpad since 1.5.0 setMethod("lpad", signature(x = "Column", len = "numeric", pad = "character"), function(x, len, pad) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -1809,60 +2633,82 @@ setMethod("lpad", signature(x = "Column", len = "numeric", pad = "character"), column(jc) }) -#' rand +#' @details +#' \code{rand}: Generates a random column with independent and identically distributed (i.i.d.) samples +#' from U[0.0, 1.0]. #' -#' Generate a random column with i.i.d. samples from U[0.0, 1.0]. -#' -#' @family normal_funcs -#' @rdname rand -#' @name rand +#' @rdname column_nonaggregate_functions +#' @param seed a random seed. Can be missing. +#' @aliases rand rand,missing-method #' @export +#' @examples +#' +#' \dontrun{ +#' tmp <- mutate(df, r1 = rand(), r2 = rand(10), r3 = randn(), r4 = randn(10)) +#' head(tmp)} +#' @note rand since 1.5.0 setMethod("rand", signature(seed = "missing"), function(seed) { jc <- callJStatic("org.apache.spark.sql.functions", "rand") column(jc) }) -#' @family normal_funcs -#' @rdname rand -#' @name rand + +#' @rdname column_nonaggregate_functions +#' @aliases rand,numeric-method #' @export +#' @note rand(numeric) since 1.5.0 setMethod("rand", signature(seed = "numeric"), function(seed) { jc <- callJStatic("org.apache.spark.sql.functions", "rand", as.integer(seed)) column(jc) }) -#' randn -#' -#' Generate a column with i.i.d. samples from the standard normal distribution. +#' @details +#' \code{randn}: Generates a column with independent and identically distributed (i.i.d.) samples from +#' the standard normal distribution. #' -#' @family normal_funcs -#' @rdname randn -#' @name randn +#' @rdname column_nonaggregate_functions +#' @aliases randn randn,missing-method #' @export +#' @note randn since 1.5.0 setMethod("randn", signature(seed = "missing"), function(seed) { jc <- callJStatic("org.apache.spark.sql.functions", "randn") column(jc) }) -#' @family normal_funcs -#' @rdname randn -#' @name randn + +#' @rdname column_nonaggregate_functions +#' @aliases randn,numeric-method #' @export +#' @note randn(numeric) since 1.5.0 setMethod("randn", signature(seed = "numeric"), function(seed) { jc <- callJStatic("org.apache.spark.sql.functions", "randn", as.integer(seed)) column(jc) }) -#' regexp_extract +#' @details +#' \code{regexp_extract}: Extracts a specific \code{idx} group identified by a Java regex, +#' from the specified string column. If the regex did not match, or the specified group did +#' not match, an empty string is returned. #' -#' Extract a specific(idx) group identified by a java regex, from the specified string column. -#' -#' @family string_funcs -#' @rdname regexp_extract -#' @name regexp_extract +#' @param pattern a regular expression. +#' @param idx a group index. +#' @rdname column_string_functions +#' @aliases regexp_extract regexp_extract,Column,character,numeric-method #' @export +#' @examples +#' +#' \dontrun{ +#' tmp <- mutate(df, s1 = regexp_extract(df$Class, "(\\d+)\\w+", 1), +#' s2 = regexp_extract(df$Sex, "^(\\w)\\w+", 1), +#' s3 = regexp_replace(df$Class, "\\D+", ""), +#' s4 = substring_index(df$Sex, "a", 1), +#' s5 = substring_index(df$Sex, "a", -1), +#' s6 = translate(df$Sex, "ale", ""), +#' s7 = translate(df$Sex, "a", "-")) +#' head(tmp)} +#' @note regexp_extract since 1.5.0 setMethod("regexp_extract", signature(x = "Column", pattern = "character", idx = "numeric"), function(x, pattern, idx) { @@ -1872,14 +2718,15 @@ setMethod("regexp_extract", column(jc) }) -#' regexp_replace +#' @details +#' \code{regexp_replace}: Replaces all substrings of the specified string value that +#' match regexp with rep. #' -#' Replace all substrings of the specified string value that match regexp with rep. -#' -#' @family string_funcs -#' @rdname regexp_replace -#' @name regexp_replace +#' @param replacement a character string that a matched \code{pattern} is replaced with. +#' @rdname column_string_functions +#' @aliases regexp_replace regexp_replace,Column,character,character-method #' @export +#' @note regexp_replace since 1.5.0 setMethod("regexp_replace", signature(x = "Column", pattern = "character", replacement = "character"), function(x, pattern, replacement) { @@ -1889,14 +2736,13 @@ setMethod("regexp_replace", column(jc) }) -#' rpad -#' -#' Right-padded with pad to a length of len. +#' @details +#' \code{rpad}: Right-padded with pad to a length of len. #' -#' @family string_funcs -#' @rdname rpad -#' @name rpad +#' @rdname column_string_functions +#' @aliases rpad rpad,Column,numeric,character-method #' @export +#' @note rpad since 1.5.0 setMethod("rpad", signature(x = "Column", len = "numeric", pad = "character"), function(x, len, pad) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -1905,17 +2751,21 @@ setMethod("rpad", signature(x = "Column", len = "numeric", pad = "character"), column(jc) }) -#' substring_index -#' -#' Returns the substring from string str before count occurrences of the delimiter delim. -#' If count is positive, everything the left of the final delimiter (counting from left) is -#' returned. If count is negative, every to the right of the final delimiter (counting from the -#' right) is returned. substring <- index performs a case-sensitive match when searching for delim. +#' @details +#' \code{substring_index}: Returns the substring from string str before count occurrences of +#' the delimiter delim. If count is positive, everything the left of the final delimiter +#' (counting from left) is returned. If count is negative, every to the right of the final +#' delimiter (counting from the right) is returned. substring_index performs a case-sensitive +#' match when searching for delim. #' -#' @family string_funcs -#' @rdname substring_index -#' @name substring_index +#' @param delim a delimiter string. +#' @param count number of occurrences of \code{delim} before the substring is returned. +#' A positive number means counting from the left, while negative means +#' counting from the right. +#' @rdname column_string_functions +#' @aliases substring_index substring_index,Column,character,numeric-method #' @export +#' @note substring_index since 1.5.0 setMethod("substring_index", signature(x = "Column", delim = "character", count = "numeric"), function(x, delim, count) { @@ -1925,17 +2775,20 @@ setMethod("substring_index", column(jc) }) -#' translate -#' -#' Translate any character in the src by a character in replaceString. +#' @details +#' \code{translate}: Translates any character in the src by a character in replaceString. #' The characters in replaceString is corresponding to the characters in matchingString. #' The translate will happen when any character in the string matching with the character #' in the matchingString. #' -#' @family string_funcs -#' @rdname translate -#' @name translate +#' @param matchingString a source string where each character will be translated. +#' @param replaceString a target string where each \code{matchingString} character will +#' be replaced by the character in \code{replaceString} +#' at the same location, if any. +#' @rdname column_string_functions +#' @aliases translate translate,Column,character,character-method #' @export +#' @note translate since 1.5.0 setMethod("translate", signature(x = "Column", matchingString = "character", replaceString = "character"), function(x, matchingString, replaceString) { @@ -1944,69 +2797,84 @@ setMethod("translate", column(jc) }) -#' unix_timestamp -#' -#' Gets current Unix timestamp in seconds. +#' @details +#' \code{unix_timestamp}: Gets current Unix timestamp in seconds. #' -#' @family datetime_funcs -#' @rdname unix_timestamp -#' @name unix_timestamp +#' @rdname column_datetime_functions +#' @aliases unix_timestamp unix_timestamp,missing,missing-method #' @export +#' @note unix_timestamp since 1.5.0 setMethod("unix_timestamp", signature(x = "missing", format = "missing"), function(x, format) { jc <- callJStatic("org.apache.spark.sql.functions", "unix_timestamp") column(jc) }) -#' @family datetime_funcs -#' @rdname unix_timestamp -#' @name unix_timestamp + +#' @rdname column_datetime_functions +#' @aliases unix_timestamp,Column,missing-method #' @export +#' @note unix_timestamp(Column) since 1.5.0 setMethod("unix_timestamp", signature(x = "Column", format = "missing"), function(x, format) { jc <- callJStatic("org.apache.spark.sql.functions", "unix_timestamp", x@jc) column(jc) }) -#' @family datetime_funcs -#' @rdname unix_timestamp -#' @name unix_timestamp + +#' @rdname column_datetime_functions +#' @aliases unix_timestamp,Column,character-method #' @export +#' @note unix_timestamp(Column, character) since 1.5.0 setMethod("unix_timestamp", signature(x = "Column", format = "character"), function(x, format = "yyyy-MM-dd HH:mm:ss") { jc <- callJStatic("org.apache.spark.sql.functions", "unix_timestamp", x@jc, format) column(jc) }) -#' when -#' -#' Evaluates a list of conditions and returns one of multiple possible result expressions. + +#' @details +#' \code{when}: Evaluates a list of conditions and returns one of multiple possible result expressions. #' For unmatched expressions null is returned. #' -#' @family normal_funcs -#' @rdname when -#' @name when +#' @rdname column_nonaggregate_functions +#' @param condition the condition to test on. Must be a Column expression. +#' @param value result expression. +#' @aliases when when,Column-method #' @export +#' @examples +#' +#' \dontrun{ +#' tmp <- mutate(df, mpg_na = otherwise(when(df$mpg > 20, df$mpg), lit(NaN)), +#' mpg2 = ifelse(df$mpg > 20 & df$am > 0, 0, 1), +#' mpg3 = ifelse(df$mpg > 20, df$mpg, 20.0)) +#' head(tmp) +#' tmp <- mutate(tmp, ind_na1 = is.nan(tmp$mpg_na), ind_na2 = isnan(tmp$mpg_na)) +#' head(select(tmp, coalesce(tmp$mpg_na, tmp$mpg))) +#' head(select(tmp, nanvl(tmp$mpg_na, tmp$hp)))} +#' @note when since 1.5.0 setMethod("when", signature(condition = "Column", value = "ANY"), function(condition, value) { condition <- condition@jc - value <- ifelse(class(value) == "Column", value@jc, value) + value <- if (class(value) == "Column") { value@jc } else { value } jc <- callJStatic("org.apache.spark.sql.functions", "when", condition, value) column(jc) }) -#' ifelse -#' -#' Evaluates a list of conditions and returns \code{yes} if the conditions are satisfied. +#' @details +#' \code{ifelse}: Evaluates a list of conditions and returns \code{yes} if the conditions are satisfied. #' Otherwise \code{no} is returned for unmatched conditions. #' -#' @family normal_funcs -#' @rdname ifelse -#' @name ifelse +#' @rdname column_nonaggregate_functions +#' @param test a Column expression that describes the condition. +#' @param yes return values for \code{TRUE} elements of test. +#' @param no return values for \code{FALSE} elements of test. +#' @aliases ifelse ifelse,Column-method #' @export +#' @note ifelse since 1.5.0 setMethod("ifelse", signature(test = "Column", yes = "ANY", no = "ANY"), function(test, yes, no) { test <- test@jc - yes <- ifelse(class(yes) == "Column", yes@jc, yes) - no <- ifelse(class(no) == "Column", no@jc, no) + yes <- if (class(yes) == "Column") { yes@jc } else { yes } + no <- if (class(no) == "Column") { no@jc } else { no } jc <- callJMethod(callJStatic("org.apache.spark.sql.functions", "when", test, yes), @@ -2016,66 +2884,58 @@ setMethod("ifelse", ###################### Window functions###################### -#' cumeDist -#' -#' Window function: returns the cumulative distribution of values within a window partition, -#' i.e. the fraction of rows that are below the current row. -#' -#' N = total number of rows in the partition -#' cumeDist(x) = number of values before (and including) x / N -#' -#' This is equivalent to the CUME_DIST function in SQL. +#' @details +#' \code{cume_dist}: Returns the cumulative distribution of values within a window partition, +#' i.e. the fraction of rows that are below the current row: +#' (number of values before and including x) / (total number of rows in the partition). +#' This is equivalent to the \code{CUME_DIST} function in SQL. +#' The method should be used with no argument. #' -#' @rdname cumeDist -#' @name cumeDist -#' @family window_funcs +#' @rdname column_window_functions +#' @aliases cume_dist cume_dist,missing-method #' @export -#' @examples \dontrun{cumeDist()} -setMethod("cumeDist", - signature(x = "missing"), +#' @note cume_dist since 1.6.0 +setMethod("cume_dist", + signature("missing"), function() { - jc <- callJStatic("org.apache.spark.sql.functions", "cumeDist") + jc <- callJStatic("org.apache.spark.sql.functions", "cume_dist") column(jc) }) -#' denseRank -#' -#' Window function: returns the rank of rows within a window partition, without any gaps. -#' The difference between rank and denseRank is that denseRank leaves no gaps in ranking -#' sequence when there are ties. That is, if you were ranking a competition using denseRank +#' @details +#' \code{dense_rank}: Returns the rank of rows within a window partition, without any gaps. +#' The difference between rank and dense_rank is that dense_rank leaves no gaps in ranking +#' sequence when there are ties. That is, if you were ranking a competition using dense_rank #' and had three people tie for second place, you would say that all three were in second -#' place and that the next person came in third. -#' -#' This is equivalent to the DENSE_RANK function in SQL. +#' place and that the next person came in third. Rank would give me sequential numbers, making +#' the person that came in third place (after the ties) would register as coming in fifth. +#' This is equivalent to the \code{DENSE_RANK} function in SQL. +#' The method should be used with no argument. #' -#' @rdname denseRank -#' @name denseRank -#' @family window_funcs +#' @rdname column_window_functions +#' @aliases dense_rank dense_rank,missing-method #' @export -#' @examples \dontrun{denseRank()} -setMethod("denseRank", - signature(x = "missing"), +#' @note dense_rank since 1.6.0 +setMethod("dense_rank", + signature("missing"), function() { - jc <- callJStatic("org.apache.spark.sql.functions", "denseRank") + jc <- callJStatic("org.apache.spark.sql.functions", "dense_rank") column(jc) }) -#' lag -#' -#' Window function: returns the value that is `offset` rows before the current row, and -#' `defaultValue` if there is less than `offset` rows before the current row. For example, -#' an `offset` of one will return the previous row at any given point in the window partition. -#' -#' This is equivalent to the LAG function in SQL. +#' @details +#' \code{lag}: Returns the value that is \code{offset} rows before the current row, and +#' \code{defaultValue} if there is less than \code{offset} rows before the current row. For example, +#' an \code{offset} of one will return the previous row at any given point in the window partition. +#' This is equivalent to the \code{LAG} function in SQL. #' -#' @rdname lag -#' @name lag -#' @family window_funcs +#' @rdname column_window_functions +#' @aliases lag lag,characterOrColumn-method #' @export -#' @examples \dontrun{lag(df$c)} +#' @note lag since 1.6.0 setMethod("lag", - signature(x = "characterOrColumn", offset = "numeric", defaultValue = "ANY"), - function(x, offset, defaultValue = NULL) { + signature(x = "characterOrColumn"), + function(x, offset = 1, defaultValue = NULL) { col <- if (class(x) == "Column") { x@jc } else { @@ -2087,22 +2947,20 @@ setMethod("lag", column(jc) }) -#' lead -#' -#' Window function: returns the value that is `offset` rows after the current row, and -#' `null` if there is less than `offset` rows after the current row. For example, -#' an `offset` of one will return the next row at any given point in the window partition. -#' -#' This is equivalent to the LEAD function in SQL. +#' @details +#' \code{lead}: Returns the value that is \code{offset} rows after the current row, and +#' \code{defaultValue} if there is less than \code{offset} rows after the current row. +#' For example, an \code{offset} of one will return the next row at any given point +#' in the window partition. +#' This is equivalent to the \code{LEAD} function in SQL. #' -#' @rdname lead -#' @name lead -#' @family window_funcs +#' @rdname column_window_functions +#' @aliases lead lead,characterOrColumn,numeric-method #' @export -#' @examples \dontrun{lead(df$c)} +#' @note lead since 1.6.0 setMethod("lead", signature(x = "characterOrColumn", offset = "numeric", defaultValue = "ANY"), - function(x, offset, defaultValue = NULL) { + function(x, offset = 1, defaultValue = NULL) { col <- if (class(x) == "Column") { x@jc } else { @@ -2114,19 +2972,16 @@ setMethod("lead", column(jc) }) -#' ntile -#' -#' Window function: returns the ntile group id (from 1 to `n` inclusive) in an ordered window -#' partition. Fow example, if `n` is 4, the first quarter of the rows will get value 1, the second +#' @details +#' \code{ntile}: Returns the ntile group id (from 1 to n inclusive) in an ordered window +#' partition. For example, if n is 4, the first quarter of the rows will get value 1, the second #' quarter will get 2, the third quarter will get 3, and the last quarter will get 4. -#' -#' This is equivalent to the NTILE function in SQL. +#' This is equivalent to the \code{NTILE} function in SQL. #' -#' @rdname ntile -#' @name ntile -#' @family window_funcs +#' @rdname column_window_functions +#' @aliases ntile ntile,numeric-method #' @export -#' @examples \dontrun{ntile(1)} +#' @note ntile since 1.6.0 setMethod("ntile", signature(x = "numeric"), function(x) { @@ -2134,44 +2989,37 @@ setMethod("ntile", column(jc) }) -#' percentRank -#' -#' Window function: returns the relative rank (i.e. percentile) of rows within a window partition. -#' -#' This is computed by: -#' -#' (rank of row in its partition - 1) / (number of rows in the partition - 1) +#' @details +#' \code{percent_rank}: Returns the relative rank (i.e. percentile) of rows within a window partition. +#' This is computed by: (rank of row in its partition - 1) / (number of rows in the partition - 1). +#' This is equivalent to the \code{PERCENT_RANK} function in SQL. +#' The method should be used with no argument. #' -#' This is equivalent to the PERCENT_RANK function in SQL. -#' -#' @rdname percentRank -#' @name percentRank -#' @family window_funcs +#' @rdname column_window_functions +#' @aliases percent_rank percent_rank,missing-method #' @export -#' @examples \dontrun{percentRank()} -setMethod("percentRank", - signature(x = "missing"), +#' @note percent_rank since 1.6.0 +setMethod("percent_rank", + signature("missing"), function() { - jc <- callJStatic("org.apache.spark.sql.functions", "percentRank") + jc <- callJStatic("org.apache.spark.sql.functions", "percent_rank") column(jc) }) -#' rank -#' -#' Window function: returns the rank of rows within a window partition. -#' -#' The difference between rank and denseRank is that denseRank leaves no gaps in ranking -#' sequence when there are ties. That is, if you were ranking a competition using denseRank +#' @details +#' \code{rank}: Returns the rank of rows within a window partition. +#' The difference between rank and dense_rank is that dense_rank leaves no gaps in ranking +#' sequence when there are ties. That is, if you were ranking a competition using dense_rank #' and had three people tie for second place, you would say that all three were in second -#' place and that the next person came in third. -#' -#' This is equivalent to the RANK function in SQL. +#' place and that the next person came in third. Rank would give me sequential numbers, making +#' the person that came in third place (after the ties) would register as coming in fifth. +#' This is equivalent to the \code{RANK} function in SQL. +#' The method should be used with no argument. #' -#' @rdname rank -#' @name rank -#' @family window_funcs +#' @rdname column_window_functions +#' @aliases rank rank,missing-method #' @export -#' @examples \dontrun{rank()} +#' @note rank since 1.6.0 setMethod("rank", signature(x = "missing"), function() { @@ -2179,27 +3027,434 @@ setMethod("rank", column(jc) }) -# Expose rank() in the R base package +#' @rdname column_window_functions +#' @aliases rank,ANY-method +#' @export setMethod("rank", signature(x = "ANY"), function(x, ...) { base::rank(x, ...) }) -#' rowNumber +#' @details +#' \code{row_number}: Returns a sequential number starting at 1 within a window partition. +#' This is equivalent to the \code{ROW_NUMBER} function in SQL. +#' The method should be used with no argument. #' -#' Window function: returns a sequential number starting at 1 within a window partition. -#' -#' This is equivalent to the ROW_NUMBER function in SQL. +#' @rdname column_window_functions +#' @aliases row_number row_number,missing-method +#' @export +#' @note row_number since 1.6.0 +setMethod("row_number", + signature("missing"), + function() { + jc <- callJStatic("org.apache.spark.sql.functions", "row_number") + column(jc) + }) + +###################### Collection functions###################### + +#' @details +#' \code{array_contains}: Returns null if the array is null, true if the array contains +#' the value, and false otherwise. #' -#' @rdname rowNumber -#' @name rowNumber -#' @family window_funcs +#' @param value a value to be checked if contained in the column +#' @rdname column_collection_functions +#' @aliases array_contains array_contains,Column-method #' @export -#' @examples \dontrun{rowNumber()} -setMethod("rowNumber", - signature(x = "missing"), +#' @note array_contains since 1.6.0 +setMethod("array_contains", + signature(x = "Column", value = "ANY"), + function(x, value) { + jc <- callJStatic("org.apache.spark.sql.functions", "array_contains", x@jc, value) + column(jc) + }) + +#' @details +#' \code{map_keys}: Returns an unordered array containing the keys of the map. +#' +#' @rdname column_collection_functions +#' @aliases map_keys map_keys,Column-method +#' @export +#' @note map_keys since 2.3.0 +setMethod("map_keys", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "map_keys", x@jc) + column(jc) + }) + +#' @details +#' \code{map_values}: Returns an unordered array containing the values of the map. +#' +#' @rdname column_collection_functions +#' @aliases map_values map_values,Column-method +#' @export +#' @note map_values since 2.3.0 +setMethod("map_values", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "map_values", x@jc) + column(jc) + }) + +#' @details +#' \code{explode}: Creates a new row for each element in the given array or map column. +#' +#' @rdname column_collection_functions +#' @aliases explode explode,Column-method +#' @export +#' @note explode since 1.5.0 +setMethod("explode", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "explode", x@jc) + column(jc) + }) + +#' @details +#' \code{size}: Returns length of array or map. +#' +#' @rdname column_collection_functions +#' @aliases size size,Column-method +#' @export +#' @note size since 1.5.0 +setMethod("size", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "size", x@jc) + column(jc) + }) + +#' @details +#' \code{sort_array}: Sorts the input array in ascending or descending order according +#' to the natural ordering of the array elements. +#' +#' @rdname column_collection_functions +#' @param asc a logical flag indicating the sorting order. +#' TRUE, sorting is in ascending order. +#' FALSE, sorting is in descending order. +#' @aliases sort_array sort_array,Column-method +#' @export +#' @note sort_array since 1.6.0 +setMethod("sort_array", + signature(x = "Column"), + function(x, asc = TRUE) { + jc <- callJStatic("org.apache.spark.sql.functions", "sort_array", x@jc, asc) + column(jc) + }) + +#' @details +#' \code{posexplode}: Creates a new row for each element with position in the given array +#' or map column. +#' +#' @rdname column_collection_functions +#' @aliases posexplode posexplode,Column-method +#' @export +#' @note posexplode since 2.1.0 +setMethod("posexplode", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "posexplode", x@jc) + column(jc) + }) + +#' @details +#' \code{create_array}: Creates a new array column. The input columns must all have the same data type. +#' +#' @rdname column_nonaggregate_functions +#' @aliases create_array create_array,Column-method +#' @export +#' @note create_array since 2.3.0 +setMethod("create_array", + signature(x = "Column"), + function(x, ...) { + jcols <- lapply(list(x, ...), function (x) { + stopifnot(class(x) == "Column") + x@jc + }) + jc <- callJStatic("org.apache.spark.sql.functions", "array", jcols) + column(jc) + }) + +#' @details +#' \code{create_map}: Creates a new map column. The input columns must be grouped as key-value pairs, +#' e.g. (key1, value1, key2, value2, ...). +#' The key columns must all have the same data type, and can't be null. +#' The value columns must all have the same data type. +#' +#' @rdname column_nonaggregate_functions +#' @aliases create_map create_map,Column-method +#' @export +#' @note create_map since 2.3.0 +setMethod("create_map", + signature(x = "Column"), + function(x, ...) { + jcols <- lapply(list(x, ...), function (x) { + stopifnot(class(x) == "Column") + x@jc + }) + jc <- callJStatic("org.apache.spark.sql.functions", "map", jcols) + column(jc) + }) + +#' @details +#' \code{collect_list}: Creates a list of objects with duplicates. +#' +#' @rdname column_aggregate_functions +#' @aliases collect_list collect_list,Column-method +#' @export +#' @examples +#' +#' \dontrun{ +#' df2 = df[df$mpg > 20, ] +#' collect(select(df2, collect_list(df2$gear))) +#' collect(select(df2, collect_set(df2$gear)))} +#' @note collect_list since 2.3.0 +setMethod("collect_list", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "collect_list", x@jc) + column(jc) + }) + +#' @details +#' \code{collect_set}: Creates a list of objects with duplicate elements eliminated. +#' +#' @rdname column_aggregate_functions +#' @aliases collect_set collect_set,Column-method +#' @export +#' @note collect_set since 2.3.0 +setMethod("collect_set", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "collect_set", x@jc) + column(jc) + }) + +#' @details +#' \code{split_string}: Splits string on regular expression. +#' Equivalent to \code{split} SQL function. +#' +#' @rdname column_string_functions +#' @aliases split_string split_string,Column-method +#' @export +#' @examples +#' +#' \dontrun{ +#' head(select(df, split_string(df$Sex, "a"))) +#' head(select(df, split_string(df$Class, "\\d"))) +#' # This is equivalent to the following SQL expression +#' head(selectExpr(df, "split(Class, '\\\\d')"))} +#' @note split_string 2.3.0 +setMethod("split_string", + signature(x = "Column", pattern = "character"), + function(x, pattern) { + jc <- callJStatic("org.apache.spark.sql.functions", "split", x@jc, pattern) + column(jc) + }) + +#' @details +#' \code{repeat_string}: Repeats string n times. +#' Equivalent to \code{repeat} SQL function. +#' +#' @param n number of repetitions. +#' @rdname column_string_functions +#' @aliases repeat_string repeat_string,Column-method +#' @export +#' @examples +#' +#' \dontrun{ +#' head(select(df, repeat_string(df$Class, 3))) +#' # This is equivalent to the following SQL expression +#' head(selectExpr(df, "repeat(Class, 3)"))} +#' @note repeat_string since 2.3.0 +setMethod("repeat_string", + signature(x = "Column", n = "numeric"), + function(x, n) { + jc <- callJStatic("org.apache.spark.sql.functions", "repeat", x@jc, numToInt(n)) + column(jc) + }) + +#' @details +#' \code{explode}: Creates a new row for each element in the given array or map column. +#' Unlike \code{explode}, if the array/map is \code{null} or empty +#' then \code{null} is produced. +#' +#' +#' @rdname column_collection_functions +#' @aliases explode_outer explode_outer,Column-method +#' @export +#' @examples +#' +#' \dontrun{ +#' df2 <- createDataFrame(data.frame( +#' id = c(1, 2, 3), text = c("a,b,c", NA, "d,e") +#' )) +#' +#' head(select(df2, df2$id, explode_outer(split_string(df2$text, ",")))) +#' head(select(df2, df2$id, posexplode_outer(split_string(df2$text, ","))))} +#' @note explode_outer since 2.3.0 +setMethod("explode_outer", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "explode_outer", x@jc) + column(jc) + }) + +#' @details +#' \code{posexplode_outer}: Creates a new row for each element with position in the given +#' array or map column. Unlike \code{posexplode}, if the array/map is \code{null} or empty +#' then the row (\code{null}, \code{null}) is produced. +#' +#' @rdname column_collection_functions +#' @aliases posexplode_outer posexplode_outer,Column-method +#' @export +#' @note posexplode_outer since 2.3.0 +setMethod("posexplode_outer", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "posexplode_outer", x@jc) + column(jc) + }) + +#' not +#' +#' Inversion of boolean expression. +#' +#' \code{not} and \code{!} cannot be applied directly to numerical column. +#' To achieve R-like truthiness column has to be casted to \code{BooleanType}. +#' +#' @param x Column to compute on +#' @rdname not +#' @name not +#' @aliases not,Column-method +#' @family non-aggregate functions +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame(data.frame( +#' is_true = c(TRUE, FALSE, NA), +#' flag = c(1, 0, 1) +#' )) +#' +#' head(select(df, not(df$is_true))) +#' +#' # Explicit cast is required when working with numeric column +#' head(select(df, not(cast(df$flag, "boolean")))) +#' } +#' @note not since 2.3.0 +setMethod("not", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "not", x@jc) + column(jc) + }) + +#' @details +#' \code{grouping_bit}: Indicates whether a specified column in a GROUP BY list is aggregated or not, +#' returns 1 for aggregated or 0 for not aggregated in the result set. Same as \code{GROUPING} in SQL +#' and \code{grouping} function in Scala. +#' +#' @rdname column_aggregate_functions +#' @aliases grouping_bit grouping_bit,Column-method +#' @export +#' @examples +#' +#' \dontrun{ +#' # With cube +#' agg( +#' cube(df, "cyl", "gear", "am"), +#' mean(df$mpg), +#' grouping_bit(df$cyl), grouping_bit(df$gear), grouping_bit(df$am) +#' ) +#' +#' # With rollup +#' agg( +#' rollup(df, "cyl", "gear", "am"), +#' mean(df$mpg), +#' grouping_bit(df$cyl), grouping_bit(df$gear), grouping_bit(df$am) +#' )} +#' @note grouping_bit since 2.3.0 +setMethod("grouping_bit", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "grouping", x@jc) + column(jc) + }) + +#' @details +#' \code{grouping_id}: Returns the level of grouping. +#' Equals to \code{ +#' grouping_bit(c1) * 2^(n - 1) + grouping_bit(c2) * 2^(n - 2) + ... + grouping_bit(cn) +#' }. +#' +#' @rdname column_aggregate_functions +#' @aliases grouping_id grouping_id,Column-method +#' @export +#' @examples +#' +#' \dontrun{ +#' # With cube +#' agg( +#' cube(df, "cyl", "gear", "am"), +#' mean(df$mpg), +#' grouping_id(df$cyl, df$gear, df$am) +#' ) +#' +#' # With rollup +#' agg( +#' rollup(df, "cyl", "gear", "am"), +#' mean(df$mpg), +#' grouping_id(df$cyl, df$gear, df$am) +#' )} +#' @note grouping_id since 2.3.0 +setMethod("grouping_id", + signature(x = "Column"), + function(x, ...) { + jcols <- lapply(list(x, ...), function (x) { + stopifnot(class(x) == "Column") + x@jc + }) + jc <- callJStatic("org.apache.spark.sql.functions", "grouping_id", jcols) + column(jc) + }) + +#' @details +#' \code{input_file_name}: Creates a string column with the input file name for a given row. +#' The method should be used with no argument. +#' +#' @rdname column_nonaggregate_functions +#' @aliases input_file_name input_file_name,missing-method +#' @export +#' @examples +#' +#' \dontrun{ +#' tmp <- read.text("README.md") +#' head(select(tmp, input_file_name()))} +#' @note input_file_name since 2.3.0 +setMethod("input_file_name", signature("missing"), function() { - jc <- callJStatic("org.apache.spark.sql.functions", "rowNumber") + jc <- callJStatic("org.apache.spark.sql.functions", "input_file_name") + column(jc) + }) + +#' @details +#' \code{trunc}: Returns date truncated to the unit specified by the format. +#' +#' @rdname column_datetime_functions +#' @aliases trunc trunc,Column-method +#' @export +#' @examples +#' +#' \dontrun{ +#' head(select(df, df$time, trunc(df$time, "year"), trunc(df$time, "yy"), +#' trunc(df$time, "month"), trunc(df$time, "mon")))} +#' @note trunc since 2.3.0 +setMethod("trunc", + signature(x = "Column"), + function(x, format) { + jc <- callJStatic("org.apache.spark.sql.functions", "trunc", + x@jc, as.character(format)) column(jc) }) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 0b35340e48e4..0fe8f0453b06 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -23,22 +23,18 @@ setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) -# @rdname cache-methods -# @export -setGeneric("cache", function(x) { standardGeneric("cache") }) +setGeneric("cacheRDD", function(x) { standardGeneric("cacheRDD") }) # @rdname coalesce # @seealso repartition # @export -setGeneric("coalesce", function(x, numPartitions, ...) { standardGeneric("coalesce") }) +setGeneric("coalesceRDD", function(x, numPartitions, ...) { standardGeneric("coalesceRDD") }) # @rdname checkpoint-methods # @export -setGeneric("checkpoint", function(x) { standardGeneric("checkpoint") }) +setGeneric("checkpointRDD", function(x) { standardGeneric("checkpointRDD") }) -# @rdname collect-methods -# @export -setGeneric("collect", function(x, ...) { standardGeneric("collect") }) +setGeneric("collectRDD", function(x, ...) { standardGeneric("collectRDD") }) # @rdname collect-methods # @export @@ -51,33 +47,36 @@ setGeneric("collectPartition", standardGeneric("collectPartition") }) -# @rdname count -# @export -setGeneric("count", function(x) { standardGeneric("count") }) +setGeneric("countRDD", function(x) { standardGeneric("countRDD") }) + +setGeneric("lengthRDD", function(x) { standardGeneric("lengthRDD") }) # @rdname countByValue # @export setGeneric("countByValue", function(x) { standardGeneric("countByValue") }) -# @rdname statfunctions +# @rdname crosstab # @export setGeneric("crosstab", function(x, col1, col2) { standardGeneric("crosstab") }) -# @rdname statfunctions +# @rdname freqItems # @export setGeneric("freqItems", function(x, cols, support = 0.01) { standardGeneric("freqItems") }) -# @rdname distinct +# @rdname approxQuantile # @export -setGeneric("distinct", function(x, numPartitions = 1) { standardGeneric("distinct") }) +setGeneric("approxQuantile", + function(x, cols, probabilities, relativeError) { + standardGeneric("approxQuantile") + }) + +setGeneric("distinctRDD", function(x, numPartitions = 1) { standardGeneric("distinctRDD") }) # @rdname filterRDD # @export setGeneric("filterRDD", function(x, f) { standardGeneric("filterRDD") }) -# @rdname first -# @export -setGeneric("first", function(x) { standardGeneric("first") }) +setGeneric("firstRDD", function(x, ...) { standardGeneric("firstRDD") }) # @rdname flatMap # @export @@ -88,12 +87,8 @@ setGeneric("flatMap", function(X, FUN) { standardGeneric("flatMap") }) # @export setGeneric("fold", function(x, zeroValue, op) { standardGeneric("fold") }) -# @rdname foreach -# @export setGeneric("foreach", function(x, func) { standardGeneric("foreach") }) -# @rdname foreach -# @export setGeneric("foreachPartition", function(x, func) { standardGeneric("foreachPartition") }) # The jrdd accessor function. @@ -103,31 +98,27 @@ setGeneric("getJRDD", function(rdd, ...) { standardGeneric("getJRDD") }) # @export setGeneric("glom", function(x) { standardGeneric("glom") }) +# @rdname histogram +# @export +setGeneric("histogram", function(df, col, nbins=10) { standardGeneric("histogram") }) + +setGeneric("joinRDD", function(x, y, ...) { standardGeneric("joinRDD") }) + # @rdname keyBy # @export setGeneric("keyBy", function(x, func) { standardGeneric("keyBy") }) -# @rdname lapplyPartition -# @export setGeneric("lapplyPartition", function(X, FUN) { standardGeneric("lapplyPartition") }) -# @rdname lapplyPartitionsWithIndex -# @export setGeneric("lapplyPartitionsWithIndex", function(X, FUN) { standardGeneric("lapplyPartitionsWithIndex") }) -# @rdname lapply -# @export setGeneric("map", function(X, FUN) { standardGeneric("map") }) -# @rdname lapplyPartition -# @export setGeneric("mapPartitions", function(X, FUN) { standardGeneric("mapPartitions") }) -# @rdname lapplyPartitionsWithIndex -# @export setGeneric("mapPartitionsWithIndex", function(X, FUN) { standardGeneric("mapPartitionsWithIndex") }) @@ -147,26 +138,29 @@ setGeneric("sumRDD", function(x) { standardGeneric("sumRDD") }) # @export setGeneric("name", function(x) { standardGeneric("name") }) -# @rdname numPartitions +# @rdname getNumPartitionsRDD # @export -setGeneric("numPartitions", function(x) { standardGeneric("numPartitions") }) +setGeneric("getNumPartitionsRDD", function(x) { standardGeneric("getNumPartitionsRDD") }) -# @rdname persist +# @rdname getNumPartitions # @export -setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) +setGeneric("numPartitions", function(x) { standardGeneric("numPartitions") }) + +setGeneric("persistRDD", function(x, newLevel) { standardGeneric("persistRDD") }) # @rdname pipeRDD # @export setGeneric("pipeRDD", function(x, command, env = list()) { standardGeneric("pipeRDD")}) +# @rdname pivot +# @export +setGeneric("pivot", function(x, colname, values = list()) { standardGeneric("pivot") }) + # @rdname reduce # @export setGeneric("reduce", function(x, func) { standardGeneric("reduce") }) -# @rdname repartition -# @seealso coalesce -# @export -setGeneric("repartition", function(x, numPartitions) { standardGeneric("repartition") }) +setGeneric("repartitionRDD", function(x, ...) { standardGeneric("repartitionRDD") }) # @rdname sampleRDD # @export @@ -188,6 +182,8 @@ setGeneric("saveAsTextFile", function(x, path) { standardGeneric("saveAsTextFile # @export setGeneric("setName", function(x, name) { standardGeneric("setName") }) +setGeneric("showRDD", function(object, ...) { standardGeneric("showRDD") }) + # @rdname sortBy # @export setGeneric("sortBy", @@ -195,9 +191,7 @@ setGeneric("sortBy", standardGeneric("sortBy") }) -# @rdname take -# @export -setGeneric("take", function(x, num) { standardGeneric("take") }) +setGeneric("takeRDD", function(x, num) { standardGeneric("takeRDD") }) # @rdname takeOrdered # @export @@ -218,9 +212,7 @@ setGeneric("top", function(x, num) { standardGeneric("top") }) # @export setGeneric("unionRDD", function(x, y) { standardGeneric("unionRDD") }) -# @rdname unpersist-methods -# @export -setGeneric("unpersist", function(x, ...) { standardGeneric("unpersist") }) +setGeneric("unpersistRDD", function(x, ...) { standardGeneric("unpersistRDD") }) # @rdname zipRDD # @export @@ -338,9 +330,7 @@ setGeneric("join", function(x, y, ...) { standardGeneric("join") }) # @export setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) -# @rdname partitionBy -# @export -setGeneric("partitionBy", function(x, numPartitions, ...) { standardGeneric("partitionBy") }) +setGeneric("partitionByRDD", function(x, ...) { standardGeneric("partitionByRDD") }) # @rdname reduceByKey # @seealso groupByKey @@ -388,33 +378,165 @@ setGeneric("subtractByKey", setGeneric("value", function(bcast) { standardGeneric("value") }) +#################### SparkDataFrame Methods ######################## -#################### DataFrame Methods ######################## - -#' @rdname agg +#' @param x a SparkDataFrame or GroupedData. +#' @param ... further arguments to be passed to or from other methods. +#' @return A SparkDataFrame. +#' @rdname summarize #' @export setGeneric("agg", function (x, ...) { standardGeneric("agg") }) +#' alias +#' +#' Returns a new SparkDataFrame or a Column with an alias set. Equivalent to SQL "AS" keyword. +#' +#' @name alias +#' @rdname alias +#' @param object x a SparkDataFrame or a Column +#' @param data new name to use +#' @return a SparkDataFrame or a Column +NULL + #' @rdname arrange #' @export setGeneric("arrange", function(x, col, ...) { standardGeneric("arrange") }) -#' @rdname schema +#' @rdname as.data.frame +#' @export +setGeneric("as.data.frame", + function(x, row.names = NULL, optional = FALSE, ...) { + standardGeneric("as.data.frame") + }) + +#' @rdname attach +#' @export +setGeneric("attach") + +#' @rdname cache +#' @export +setGeneric("cache", function(x) { standardGeneric("cache") }) + +#' @rdname checkpoint +#' @export +setGeneric("checkpoint", function(x, eager = TRUE) { standardGeneric("checkpoint") }) + +#' @rdname coalesce +#' @param x a SparkDataFrame. +#' @param ... additional argument(s). +#' @export +setGeneric("coalesce", function(x, ...) { standardGeneric("coalesce") }) + +#' @rdname collect +#' @export +setGeneric("collect", function(x, ...) { standardGeneric("collect") }) + +#' @param do.NULL currently not used. +#' @param prefix currently not used. +#' @rdname columns +#' @export +setGeneric("colnames", function(x, do.NULL = TRUE, prefix = "col") { standardGeneric("colnames") }) + +#' @rdname columns +#' @export +setGeneric("colnames<-", function(x, value) { standardGeneric("colnames<-") }) + +#' @rdname coltypes +#' @export +setGeneric("coltypes", function(x) { standardGeneric("coltypes") }) + +#' @rdname coltypes +#' @export +setGeneric("coltypes<-", function(x, value) { standardGeneric("coltypes<-") }) + +#' @rdname columns #' @export setGeneric("columns", function(x) {standardGeneric("columns") }) -#' @rdname statfunctions +#' @param x a GroupedData or Column. +#' @rdname count +#' @export +setGeneric("count", function(x) { standardGeneric("count") }) + +#' @rdname cov +#' @param x a Column or a SparkDataFrame. +#' @param ... additional argument(s). If \code{x} is a Column, a Column +#' should be provided. If \code{x} is a SparkDataFrame, two column names should +#' be provided. +#' @export +setGeneric("cov", function(x, ...) {standardGeneric("cov") }) + +#' @rdname corr +#' @param x a Column or a SparkDataFrame. +#' @param ... additional argument(s). If \code{x} is a Column, a Column +#' should be provided. If \code{x} is a SparkDataFrame, two column names should +#' be provided. +#' @export +setGeneric("corr", function(x, ...) {standardGeneric("corr") }) + +#' @rdname cov +#' @export +setGeneric("covar_samp", function(col1, col2) {standardGeneric("covar_samp") }) + +#' @rdname cov +#' @export +setGeneric("covar_pop", function(col1, col2) {standardGeneric("covar_pop") }) + +#' @rdname createOrReplaceTempView +#' @export +setGeneric("createOrReplaceTempView", + function(x, viewName) { + standardGeneric("createOrReplaceTempView") + }) + +# @rdname crossJoin +# @export +setGeneric("crossJoin", function(x, y) { standardGeneric("crossJoin") }) + +#' @rdname cube +#' @export +setGeneric("cube", function(x, ...) { standardGeneric("cube") }) + +#' @rdname dapply +#' @export +setGeneric("dapply", function(x, func, schema) { standardGeneric("dapply") }) + +#' @rdname dapplyCollect #' @export -setGeneric("cov", function(x, col1, col2) {standardGeneric("cov") }) +setGeneric("dapplyCollect", function(x, func) { standardGeneric("dapplyCollect") }) -#' @rdname statfunctions +#' @param x a SparkDataFrame or GroupedData. +#' @param ... additional argument(s) passed to the method. +#' @rdname gapply #' @export -setGeneric("corr", function(x, col1, col2, method = "pearson") {standardGeneric("corr") }) +setGeneric("gapply", function(x, ...) { standardGeneric("gapply") }) + +#' @param x a SparkDataFrame or GroupedData. +#' @param ... additional argument(s) passed to the method. +#' @rdname gapplyCollect +#' @export +setGeneric("gapplyCollect", function(x, ...) { standardGeneric("gapplyCollect") }) + +# @rdname getNumPartitions +# @export +setGeneric("getNumPartitions", function(x) { standardGeneric("getNumPartitions") }) #' @rdname describe #' @export setGeneric("describe", function(x, col, ...) { standardGeneric("describe") }) +#' @rdname distinct +#' @export +setGeneric("distinct", function(x) { standardGeneric("distinct") }) + +#' @rdname drop +#' @export +setGeneric("drop", function(x, ...) { standardGeneric("drop") }) + +#' @rdname dropDuplicates +#' @export +setGeneric("dropDuplicates", function(x, ...) { standardGeneric("dropDuplicates") }) + #' @rdname nafunctions #' @export setGeneric("dropna", @@ -429,12 +551,15 @@ setGeneric("na.omit", standardGeneric("na.omit") }) -#' @rdname schema +#' @rdname dtypes #' @export setGeneric("dtypes", function(x) { standardGeneric("dtypes") }) #' @rdname explain #' @export +#' @param x a SparkDataFrame or a StreamingQuery. +#' @param extended Logical. If extended is FALSE, prints only the physical plan. +#' @param ... further arguments to be passed to or from other methods. setGeneric("explain", function(x, ...) { standardGeneric("explain") }) #' @rdname except @@ -449,6 +574,10 @@ setGeneric("fillna", function(x, value, cols = NULL) { standardGeneric("fillna") #' @export setGeneric("filter", function(x, condition) { standardGeneric("filter") }) +#' @rdname first +#' @export +setGeneric("first", function(x, ...) { standardGeneric("first") }) + #' @rdname groupBy #' @export setGeneric("group_by", function(x, ...) { standardGeneric("group_by") }) @@ -457,6 +586,10 @@ setGeneric("group_by", function(x, ...) { standardGeneric("group_by") }) #' @export setGeneric("groupBy", function(x, ...) { standardGeneric("groupBy") }) +#' @rdname hint +#' @export +setGeneric("hint", function(x, name, ...) { standardGeneric("hint") }) + #' @rdname insertInto #' @export setGeneric("insertInto", function(x, tableName, ...) { standardGeneric("insertInto") }) @@ -469,71 +602,128 @@ setGeneric("intersect", function(x, y) { standardGeneric("intersect") }) #' @export setGeneric("isLocal", function(x) { standardGeneric("isLocal") }) +#' @rdname isStreaming +#' @export +setGeneric("isStreaming", function(x) { standardGeneric("isStreaming") }) + #' @rdname limit #' @export setGeneric("limit", function(x, num) {standardGeneric("limit") }) -#' rdname merge +#' @rdname merge #' @export setGeneric("merge") -#' @rdname withColumn +#' @rdname mutate #' @export setGeneric("mutate", function(.data, ...) {standardGeneric("mutate") }) -#' @rdname arrange +#' @rdname orderBy #' @export -setGeneric("orderBy", function(x, col) { standardGeneric("orderBy") }) +setGeneric("orderBy", function(x, col, ...) { standardGeneric("orderBy") }) -#' @rdname schema +#' @rdname persist +#' @export +setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) + +#' @rdname printSchema #' @export setGeneric("printSchema", function(x) { standardGeneric("printSchema") }) -#' @rdname withColumnRenamed +#' @rdname registerTempTable-deprecated +#' @export +setGeneric("registerTempTable", function(x, tableName) { standardGeneric("registerTempTable") }) + +#' @rdname rename #' @export setGeneric("rename", function(x, ...) { standardGeneric("rename") }) -#' @rdname registerTempTable +#' @rdname repartition #' @export -setGeneric("registerTempTable", function(x, tableName) { standardGeneric("registerTempTable") }) +setGeneric("repartition", function(x, ...) { standardGeneric("repartition") }) #' @rdname sample #' @export setGeneric("sample", - function(x, withReplacement, fraction, seed) { + function(x, withReplacement = FALSE, fraction, seed) { standardGeneric("sample") }) +#' @rdname rollup +#' @export +setGeneric("rollup", function(x, ...) { standardGeneric("rollup") }) + #' @rdname sample #' @export setGeneric("sample_frac", - function(x, withReplacement, fraction, seed) { standardGeneric("sample_frac") }) + function(x, withReplacement = FALSE, fraction, seed) { standardGeneric("sample_frac") }) -#' @rdname statfunctions +#' @rdname sampleBy #' @export setGeneric("sampleBy", function(x, col, fractions, seed) { standardGeneric("sampleBy") }) -#' @rdname saveAsParquetFile -#' @export -setGeneric("saveAsParquetFile", function(x, path) { standardGeneric("saveAsParquetFile") }) - #' @rdname saveAsTable #' @export -setGeneric("saveAsTable", function(df, tableName, source, mode, ...) { +setGeneric("saveAsTable", function(df, tableName, source = NULL, mode = "error", ...) { standardGeneric("saveAsTable") }) -#' @rdname withColumn +#' @export +setGeneric("str") + +#' @rdname take +#' @export +setGeneric("take", function(x, num) { standardGeneric("take") }) + +#' @rdname mutate #' @export setGeneric("transform", function(`_data`, ...) {standardGeneric("transform") }) #' @rdname write.df #' @export -setGeneric("write.df", function(df, path, ...) { standardGeneric("write.df") }) +setGeneric("write.df", function(df, path = NULL, source = NULL, mode = "error", ...) { + standardGeneric("write.df") +}) #' @rdname write.df #' @export -setGeneric("saveDF", function(df, path, ...) { standardGeneric("saveDF") }) +setGeneric("saveDF", function(df, path, source = NULL, mode = "error", ...) { + standardGeneric("saveDF") +}) + +#' @rdname write.jdbc +#' @export +setGeneric("write.jdbc", function(x, url, tableName, mode = "error", ...) { + standardGeneric("write.jdbc") +}) + +#' @rdname write.json +#' @export +setGeneric("write.json", function(x, path, ...) { standardGeneric("write.json") }) + +#' @rdname write.orc +#' @export +setGeneric("write.orc", function(x, path, ...) { standardGeneric("write.orc") }) + +#' @rdname write.parquet +#' @export +setGeneric("write.parquet", function(x, path, ...) { + standardGeneric("write.parquet") +}) + +#' @rdname write.parquet +#' @export +setGeneric("saveAsParquetFile", function(x, path) { standardGeneric("saveAsParquetFile") }) + +#' @rdname write.stream +#' @export +setGeneric("write.stream", function(df, source = NULL, outputMode = NULL, ...) { + standardGeneric("write.stream") +}) + +#' @rdname write.text +#' @export +setGeneric("write.text", function(x, path, ...) { standardGeneric("write.text") }) #' @rdname schema #' @export @@ -543,503 +733,978 @@ setGeneric("schema", function(x) { standardGeneric("schema") }) #' @export setGeneric("select", function(x, col, ...) { standardGeneric("select") } ) -#' @rdname select +#' @rdname selectExpr #' @export setGeneric("selectExpr", function(x, expr, ...) { standardGeneric("selectExpr") }) #' @rdname showDF #' @export -setGeneric("showDF", function(x,...) { standardGeneric("showDF") }) +setGeneric("showDF", function(x, ...) { standardGeneric("showDF") }) -# @rdname subset +# @rdname storageLevel # @export -setGeneric("subset", function(x, subset, select, ...) { standardGeneric("subset") }) +setGeneric("storageLevel", function(x) { standardGeneric("storageLevel") }) -#' @rdname agg +#' @rdname subset #' @export -setGeneric("summarize", function(x,...) { standardGeneric("summarize") }) +setGeneric("subset", function(x, ...) { standardGeneric("subset") }) + +#' @rdname summarize +#' @export +setGeneric("summarize", function(x, ...) { standardGeneric("summarize") }) #' @rdname summary #' @export -setGeneric("summary", function(x, ...) { standardGeneric("summary") }) +setGeneric("summary", function(object, ...) { standardGeneric("summary") }) -# @rdname tojson -# @export setGeneric("toJSON", function(x) { standardGeneric("toJSON") }) -#' @rdname DataFrame -#' @export setGeneric("toRDD", function(x) { standardGeneric("toRDD") }) -#' @rdname unionAll +#' @rdname union +#' @export +setGeneric("union", function(x, y) { standardGeneric("union") }) + +#' @rdname union #' @export setGeneric("unionAll", function(x, y) { standardGeneric("unionAll") }) +#' @rdname unionByName +#' @export +setGeneric("unionByName", function(x, y) { standardGeneric("unionByName") }) + +#' @rdname unpersist +#' @export +setGeneric("unpersist", function(x, ...) { standardGeneric("unpersist") }) + #' @rdname filter #' @export setGeneric("where", function(x, condition) { standardGeneric("where") }) +#' @rdname with +#' @export +setGeneric("with") + #' @rdname withColumn #' @export setGeneric("withColumn", function(x, colName, col) { standardGeneric("withColumn") }) -#' @rdname withColumnRenamed +#' @rdname rename #' @export setGeneric("withColumnRenamed", function(x, existingCol, newCol) { standardGeneric("withColumnRenamed") }) +#' @rdname write.df +#' @export +setGeneric("write.df", function(df, path = NULL, ...) { standardGeneric("write.df") }) + +#' @rdname randomSplit +#' @export +setGeneric("randomSplit", function(x, weights, seed) { standardGeneric("randomSplit") }) + +#' @rdname broadcast +#' @export +setGeneric("broadcast", function(x) { standardGeneric("broadcast") }) ###################### Column Methods ########################## -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("asc", function(x) { standardGeneric("asc") }) -#' @rdname column +#' @rdname between #' @export setGeneric("between", function(x, bounds) { standardGeneric("between") }) -#' @rdname column +#' @rdname cast #' @export setGeneric("cast", function(x, dataType) { standardGeneric("cast") }) -#' @rdname column +#' @rdname columnfunctions +#' @param x a Column object. +#' @param ... additional argument(s). #' @export setGeneric("contains", function(x, ...) { standardGeneric("contains") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("desc", function(x) { standardGeneric("desc") }) -#' @rdname column +#' @rdname endsWith #' @export -setGeneric("endsWith", function(x, ...) { standardGeneric("endsWith") }) +setGeneric("endsWith", function(x, suffix) { standardGeneric("endsWith") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("getField", function(x, ...) { standardGeneric("getField") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("getItem", function(x, ...) { standardGeneric("getItem") }) -#' @rdname column +#' @rdname columnfunctions +#' @export +setGeneric("isNaN", function(x) { standardGeneric("isNaN") }) + +#' @rdname columnfunctions #' @export setGeneric("isNull", function(x) { standardGeneric("isNull") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("isNotNull", function(x) { standardGeneric("isNotNull") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("like", function(x, ...) { standardGeneric("like") }) -#' @rdname column +#' @rdname columnfunctions #' @export setGeneric("rlike", function(x, ...) { standardGeneric("rlike") }) -#' @rdname column +#' @rdname startsWith #' @export -setGeneric("startsWith", function(x, ...) { standardGeneric("startsWith") }) +setGeneric("startsWith", function(x, prefix) { standardGeneric("startsWith") }) -#' @rdname column +#' @rdname column_nonaggregate_functions #' @export +#' @name NULL setGeneric("when", function(condition, value) { standardGeneric("when") }) -#' @rdname column +#' @rdname otherwise #' @export setGeneric("otherwise", function(x, value) { standardGeneric("otherwise") }) +#' @rdname over +#' @export +setGeneric("over", function(x, window) { standardGeneric("over") }) + +#' @rdname eq_null_safe +#' @export +setGeneric("%<=>%", function(x, value) { standardGeneric("%<=>%") }) + +###################### WindowSpec Methods ########################## + +#' @rdname partitionBy +#' @export +setGeneric("partitionBy", function(x, ...) { standardGeneric("partitionBy") }) + +#' @rdname rowsBetween +#' @export +setGeneric("rowsBetween", function(x, start, end) { standardGeneric("rowsBetween") }) + +#' @rdname rangeBetween +#' @export +setGeneric("rangeBetween", function(x, start, end) { standardGeneric("rangeBetween") }) + +#' @rdname windowPartitionBy +#' @export +setGeneric("windowPartitionBy", function(col, ...) { standardGeneric("windowPartitionBy") }) + +#' @rdname windowOrderBy +#' @export +setGeneric("windowOrderBy", function(col, ...) { standardGeneric("windowOrderBy") }) ###################### Expression Function Methods ########################## -#' @rdname add_months +#' @rdname column_datetime_diff_functions #' @export +#' @name NULL setGeneric("add_months", function(y, x) { standardGeneric("add_months") }) -#' @rdname approxCountDistinct +#' @rdname column_aggregate_functions #' @export +#' @name NULL setGeneric("approxCountDistinct", function(x, ...) { standardGeneric("approxCountDistinct") }) -#' @rdname ascii +#' @rdname column_collection_functions #' @export +#' @name NULL +setGeneric("array_contains", function(x, value) { standardGeneric("array_contains") }) + +#' @rdname column_string_functions +#' @export +#' @name NULL setGeneric("ascii", function(x) { standardGeneric("ascii") }) +#' @param x Column to compute on or a GroupedData object. +#' @param ... additional argument(s) when \code{x} is a GroupedData object. #' @rdname avg #' @export setGeneric("avg", function(x, ...) { standardGeneric("avg") }) -#' @rdname base64 +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("base64", function(x) { standardGeneric("base64") }) -#' @rdname bin +#' @rdname column_math_functions #' @export +#' @name NULL setGeneric("bin", function(x) { standardGeneric("bin") }) -#' @rdname bitwiseNOT +#' @rdname column_nonaggregate_functions #' @export +#' @name NULL setGeneric("bitwiseNOT", function(x) { standardGeneric("bitwiseNOT") }) -#' @rdname cbrt +#' @rdname column_math_functions #' @export +#' @name NULL +setGeneric("bround", function(x, ...) { standardGeneric("bround") }) + +#' @rdname column_math_functions +#' @export +#' @name NULL setGeneric("cbrt", function(x) { standardGeneric("cbrt") }) -#' @rdname ceil +#' @rdname column_math_functions #' @export +#' @name NULL setGeneric("ceil", function(x) { standardGeneric("ceil") }) -#' @rdname col +#' @rdname column_aggregate_functions +#' @export +#' @name NULL +setGeneric("collect_list", function(x) { standardGeneric("collect_list") }) + +#' @rdname column_aggregate_functions +#' @export +#' @name NULL +setGeneric("collect_set", function(x) { standardGeneric("collect_set") }) + +#' @rdname column #' @export setGeneric("column", function(x) { standardGeneric("column") }) -#' @rdname concat +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("concat", function(x, ...) { standardGeneric("concat") }) -#' @rdname concat_ws +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("concat_ws", function(sep, x, ...) { standardGeneric("concat_ws") }) -#' @rdname conv +#' @rdname column_math_functions #' @export +#' @name NULL setGeneric("conv", function(x, fromBase, toBase) { standardGeneric("conv") }) -#' @rdname countDistinct +#' @rdname column_aggregate_functions #' @export +#' @name NULL setGeneric("countDistinct", function(x, ...) { standardGeneric("countDistinct") }) -#' @rdname crc32 +#' @rdname column_misc_functions #' @export +#' @name NULL setGeneric("crc32", function(x) { standardGeneric("crc32") }) -#' @rdname cumeDist +#' @rdname column_nonaggregate_functions +#' @export +#' @name NULL +setGeneric("create_array", function(x, ...) { standardGeneric("create_array") }) + +#' @rdname column_nonaggregate_functions +#' @export +#' @name NULL +setGeneric("create_map", function(x, ...) { standardGeneric("create_map") }) + +#' @rdname column_misc_functions +#' @export +#' @name NULL +setGeneric("hash", function(x, ...) { standardGeneric("hash") }) + +#' @rdname column_window_functions #' @export -setGeneric("cumeDist", function(x) { standardGeneric("cumeDist") }) +#' @name NULL +setGeneric("cume_dist", function(x = "missing") { standardGeneric("cume_dist") }) -#' @rdname datediff +#' @rdname column_datetime_diff_functions #' @export +#' @name NULL setGeneric("datediff", function(y, x) { standardGeneric("datediff") }) -#' @rdname date_add +#' @rdname column_datetime_diff_functions #' @export +#' @name NULL setGeneric("date_add", function(y, x) { standardGeneric("date_add") }) -#' @rdname date_format +#' @rdname column_datetime_diff_functions #' @export +#' @name NULL setGeneric("date_format", function(y, x) { standardGeneric("date_format") }) -#' @rdname date_sub +#' @rdname column_datetime_diff_functions #' @export +#' @name NULL setGeneric("date_sub", function(y, x) { standardGeneric("date_sub") }) -#' @rdname dayofmonth +#' @rdname column_datetime_functions #' @export +#' @name NULL setGeneric("dayofmonth", function(x) { standardGeneric("dayofmonth") }) -#' @rdname dayofyear +#' @rdname column_datetime_functions #' @export +#' @name NULL setGeneric("dayofyear", function(x) { standardGeneric("dayofyear") }) -#' @rdname denseRank +#' @rdname column_string_functions #' @export -setGeneric("denseRank", function(x) { standardGeneric("denseRank") }) +#' @name NULL +setGeneric("decode", function(x, charset) { standardGeneric("decode") }) -#' @rdname explode +#' @rdname column_window_functions #' @export +#' @name NULL +setGeneric("dense_rank", function(x = "missing") { standardGeneric("dense_rank") }) + +#' @rdname column_string_functions +#' @export +#' @name NULL +setGeneric("encode", function(x, charset) { standardGeneric("encode") }) + +#' @rdname column_collection_functions +#' @export +#' @name NULL setGeneric("explode", function(x) { standardGeneric("explode") }) -#' @rdname expr +#' @rdname column_collection_functions +#' @export +#' @name NULL +setGeneric("explode_outer", function(x) { standardGeneric("explode_outer") }) + +#' @rdname column_nonaggregate_functions #' @export +#' @name NULL setGeneric("expr", function(x) { standardGeneric("expr") }) -#' @rdname from_utc_timestamp +#' @rdname column_datetime_diff_functions #' @export +#' @name NULL setGeneric("from_utc_timestamp", function(y, x) { standardGeneric("from_utc_timestamp") }) -#' @rdname format_number +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("format_number", function(y, x) { standardGeneric("format_number") }) -#' @rdname format_string +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("format_string", function(format, x, ...) { standardGeneric("format_string") }) -#' @rdname from_unixtime +#' @rdname column_collection_functions #' @export +#' @name NULL +setGeneric("from_json", function(x, schema, ...) { standardGeneric("from_json") }) + +#' @rdname column_datetime_functions +#' @export +#' @name NULL setGeneric("from_unixtime", function(x, ...) { standardGeneric("from_unixtime") }) -#' @rdname greatest +#' @rdname column_nonaggregate_functions #' @export +#' @name NULL setGeneric("greatest", function(x, ...) { standardGeneric("greatest") }) -#' @rdname hex +#' @rdname column_aggregate_functions #' @export +#' @name NULL +setGeneric("grouping_bit", function(x) { standardGeneric("grouping_bit") }) + +#' @rdname column_aggregate_functions +#' @export +#' @name NULL +setGeneric("grouping_id", function(x, ...) { standardGeneric("grouping_id") }) + +#' @rdname column_math_functions +#' @export +#' @name NULL setGeneric("hex", function(x) { standardGeneric("hex") }) -#' @rdname hour +#' @rdname column_datetime_functions #' @export +#' @name NULL setGeneric("hour", function(x) { standardGeneric("hour") }) -#' @rdname hypot +#' @rdname column_math_functions #' @export +#' @name NULL setGeneric("hypot", function(y, x) { standardGeneric("hypot") }) -#' @rdname initcap +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("initcap", function(x) { standardGeneric("initcap") }) -#' @rdname instr +#' @rdname column_nonaggregate_functions +#' @export +#' @name NULL +setGeneric("input_file_name", + function(x = "missing") { standardGeneric("input_file_name") }) + +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("instr", function(y, x) { standardGeneric("instr") }) -#' @rdname isNaN +#' @rdname column_nonaggregate_functions #' @export -setGeneric("isNaN", function(x) { standardGeneric("isNaN") }) +#' @name NULL +setGeneric("isnan", function(x) { standardGeneric("isnan") }) + +#' @rdname column_aggregate_functions +#' @export +#' @name NULL +setGeneric("kurtosis", function(x) { standardGeneric("kurtosis") }) -#' @rdname lag +#' @rdname column_window_functions #' @export -setGeneric("lag", function(x, offset, defaultValue = NULL) { standardGeneric("lag") }) +#' @name NULL +setGeneric("lag", function(x, ...) { standardGeneric("lag") }) #' @rdname last #' @export -setGeneric("last", function(x) { standardGeneric("last") }) +setGeneric("last", function(x, ...) { standardGeneric("last") }) -#' @rdname last_day +#' @rdname column_datetime_functions #' @export +#' @name NULL setGeneric("last_day", function(x) { standardGeneric("last_day") }) -#' @rdname lead +#' @rdname column_window_functions #' @export +#' @name NULL setGeneric("lead", function(x, offset, defaultValue = NULL) { standardGeneric("lead") }) -#' @rdname least +#' @rdname column_nonaggregate_functions #' @export +#' @name NULL setGeneric("least", function(x, ...) { standardGeneric("least") }) -#' @rdname levenshtein +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("levenshtein", function(y, x) { standardGeneric("levenshtein") }) -#' @rdname lit +#' @rdname column_nonaggregate_functions #' @export +#' @name NULL setGeneric("lit", function(x) { standardGeneric("lit") }) -#' @rdname locate +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("locate", function(substr, str, ...) { standardGeneric("locate") }) -#' @rdname lower +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("lower", function(x) { standardGeneric("lower") }) -#' @rdname lpad +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("lpad", function(x, len, pad) { standardGeneric("lpad") }) -#' @rdname ltrim +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("ltrim", function(x) { standardGeneric("ltrim") }) -#' @rdname md5 +#' @rdname column_collection_functions +#' @export +#' @name NULL +setGeneric("map_keys", function(x) { standardGeneric("map_keys") }) + +#' @rdname column_collection_functions +#' @export +#' @name NULL +setGeneric("map_values", function(x) { standardGeneric("map_values") }) + +#' @rdname column_misc_functions #' @export +#' @name NULL setGeneric("md5", function(x) { standardGeneric("md5") }) -#' @rdname minute +#' @rdname column_datetime_functions #' @export +#' @name NULL setGeneric("minute", function(x) { standardGeneric("minute") }) -#' @rdname month +#' @rdname column_nonaggregate_functions #' @export +#' @name NULL +setGeneric("monotonically_increasing_id", + function(x = "missing") { standardGeneric("monotonically_increasing_id") }) + +#' @rdname column_datetime_functions +#' @export +#' @name NULL setGeneric("month", function(x) { standardGeneric("month") }) -#' @rdname months_between +#' @rdname column_datetime_diff_functions #' @export +#' @name NULL setGeneric("months_between", function(y, x) { standardGeneric("months_between") }) #' @rdname count #' @export setGeneric("n", function(x) { standardGeneric("n") }) -#' @rdname nanvl +#' @rdname column_nonaggregate_functions #' @export +#' @name NULL setGeneric("nanvl", function(y, x) { standardGeneric("nanvl") }) -#' @rdname negate +#' @rdname column_nonaggregate_functions #' @export +#' @name NULL setGeneric("negate", function(x) { standardGeneric("negate") }) -#' @rdname next_day +#' @rdname not +#' @export +setGeneric("not", function(x) { standardGeneric("not") }) + +#' @rdname column_datetime_diff_functions #' @export +#' @name NULL setGeneric("next_day", function(y, x) { standardGeneric("next_day") }) -#' @rdname ntile +#' @rdname column_window_functions #' @export +#' @name NULL setGeneric("ntile", function(x) { standardGeneric("ntile") }) -#' @rdname countDistinct +#' @rdname column_aggregate_functions #' @export +#' @name NULL setGeneric("n_distinct", function(x, ...) { standardGeneric("n_distinct") }) -#' @rdname percentRank +#' @rdname column_window_functions #' @export -setGeneric("percentRank", function(x) { standardGeneric("percentRank") }) +#' @name NULL +setGeneric("percent_rank", function(x = "missing") { standardGeneric("percent_rank") }) -#' @rdname pmod +#' @rdname column_math_functions #' @export +#' @name NULL setGeneric("pmod", function(y, x) { standardGeneric("pmod") }) -#' @rdname quarter +#' @rdname column_collection_functions +#' @export +#' @name NULL +setGeneric("posexplode", function(x) { standardGeneric("posexplode") }) + +#' @rdname column_collection_functions #' @export +#' @name NULL +setGeneric("posexplode_outer", function(x) { standardGeneric("posexplode_outer") }) + +#' @rdname column_datetime_functions +#' @export +#' @name NULL setGeneric("quarter", function(x) { standardGeneric("quarter") }) -#' @rdname rand +#' @rdname column_nonaggregate_functions #' @export +#' @name NULL setGeneric("rand", function(seed) { standardGeneric("rand") }) -#' @rdname randn +#' @rdname column_nonaggregate_functions #' @export +#' @name NULL setGeneric("randn", function(seed) { standardGeneric("randn") }) -#' @rdname rank +#' @rdname column_window_functions #' @export +#' @name NULL setGeneric("rank", function(x, ...) { standardGeneric("rank") }) -#' @rdname regexp_extract +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("regexp_extract", function(x, pattern, idx) { standardGeneric("regexp_extract") }) -#' @rdname regexp_replace +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("regexp_replace", function(x, pattern, replacement) { standardGeneric("regexp_replace") }) -#' @rdname reverse +#' @rdname column_string_functions #' @export +#' @name NULL +setGeneric("repeat_string", function(x, n) { standardGeneric("repeat_string") }) + +#' @rdname column_string_functions +#' @export +#' @name NULL setGeneric("reverse", function(x) { standardGeneric("reverse") }) -#' @rdname rint +#' @rdname column_math_functions #' @export -setGeneric("rint", function(x, ...) { standardGeneric("rint") }) +#' @name NULL +setGeneric("rint", function(x) { standardGeneric("rint") }) -#' @rdname rowNumber +#' @rdname column_window_functions #' @export -setGeneric("rowNumber", function(x) { standardGeneric("rowNumber") }) +#' @name NULL +setGeneric("row_number", function(x = "missing") { standardGeneric("row_number") }) -#' @rdname rpad +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("rpad", function(x, len, pad) { standardGeneric("rpad") }) -#' @rdname rtrim +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("rtrim", function(x) { standardGeneric("rtrim") }) -#' @rdname second +#' @rdname column_aggregate_functions #' @export +#' @name NULL +setGeneric("sd", function(x, na.rm = FALSE) { standardGeneric("sd") }) + +#' @rdname column_datetime_functions +#' @export +#' @name NULL setGeneric("second", function(x) { standardGeneric("second") }) -#' @rdname sha1 +#' @rdname column_misc_functions #' @export +#' @name NULL setGeneric("sha1", function(x) { standardGeneric("sha1") }) -#' @rdname sha2 +#' @rdname column_misc_functions #' @export +#' @name NULL setGeneric("sha2", function(y, x) { standardGeneric("sha2") }) -#' @rdname shiftLeft +#' @rdname column_math_functions #' @export +#' @name NULL setGeneric("shiftLeft", function(y, x) { standardGeneric("shiftLeft") }) -#' @rdname shiftRight +#' @rdname column_math_functions #' @export +#' @name NULL setGeneric("shiftRight", function(y, x) { standardGeneric("shiftRight") }) -#' @rdname shiftRightUnsigned +#' @rdname column_math_functions #' @export +#' @name NULL setGeneric("shiftRightUnsigned", function(y, x) { standardGeneric("shiftRightUnsigned") }) -#' @rdname signum +#' @rdname column_math_functions #' @export +#' @name NULL setGeneric("signum", function(x) { standardGeneric("signum") }) -#' @rdname size +#' @rdname column_collection_functions #' @export +#' @name NULL setGeneric("size", function(x) { standardGeneric("size") }) -#' @rdname soundex +#' @rdname column_aggregate_functions +#' @export +#' @name NULL +setGeneric("skewness", function(x) { standardGeneric("skewness") }) + +#' @rdname column_collection_functions +#' @export +#' @name NULL +setGeneric("sort_array", function(x, asc = TRUE) { standardGeneric("sort_array") }) + +#' @rdname column_string_functions #' @export +#' @name NULL +setGeneric("split_string", function(x, pattern) { standardGeneric("split_string") }) + +#' @rdname column_string_functions +#' @export +#' @name NULL setGeneric("soundex", function(x) { standardGeneric("soundex") }) -#' @rdname substring_index +#' @rdname column_nonaggregate_functions #' @export +#' @name NULL +setGeneric("spark_partition_id", function(x = "missing") { standardGeneric("spark_partition_id") }) + +#' @rdname column_aggregate_functions +#' @export +#' @name NULL +setGeneric("stddev", function(x) { standardGeneric("stddev") }) + +#' @rdname column_aggregate_functions +#' @export +#' @name NULL +setGeneric("stddev_pop", function(x) { standardGeneric("stddev_pop") }) + +#' @rdname column_aggregate_functions +#' @export +#' @name NULL +setGeneric("stddev_samp", function(x) { standardGeneric("stddev_samp") }) + +#' @rdname column_nonaggregate_functions +#' @export +#' @name NULL +setGeneric("struct", function(x, ...) { standardGeneric("struct") }) + +#' @rdname column_string_functions +#' @export +#' @name NULL setGeneric("substring_index", function(x, delim, count) { standardGeneric("substring_index") }) -#' @rdname sumDistinct +#' @rdname column_aggregate_functions #' @export +#' @name NULL setGeneric("sumDistinct", function(x) { standardGeneric("sumDistinct") }) -#' @rdname toDegrees +#' @rdname column_math_functions #' @export +#' @name NULL setGeneric("toDegrees", function(x) { standardGeneric("toDegrees") }) -#' @rdname toRadians +#' @rdname column_math_functions #' @export +#' @name NULL setGeneric("toRadians", function(x) { standardGeneric("toRadians") }) -#' @rdname to_date +#' @rdname column_datetime_functions +#' @export +#' @name NULL +setGeneric("to_date", function(x, format) { standardGeneric("to_date") }) + +#' @rdname column_collection_functions +#' @export +#' @name NULL +setGeneric("to_json", function(x, ...) { standardGeneric("to_json") }) + +#' @rdname column_datetime_functions #' @export -setGeneric("to_date", function(x) { standardGeneric("to_date") }) +#' @name NULL +setGeneric("to_timestamp", function(x, format) { standardGeneric("to_timestamp") }) -#' @rdname to_utc_timestamp +#' @rdname column_datetime_diff_functions #' @export +#' @name NULL setGeneric("to_utc_timestamp", function(y, x) { standardGeneric("to_utc_timestamp") }) -#' @rdname translate +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("translate", function(x, matchingString, replaceString) { standardGeneric("translate") }) -#' @rdname trim +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("trim", function(x) { standardGeneric("trim") }) -#' @rdname unbase64 +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("unbase64", function(x) { standardGeneric("unbase64") }) -#' @rdname unhex +#' @rdname column_math_functions #' @export +#' @name NULL setGeneric("unhex", function(x) { standardGeneric("unhex") }) -#' @rdname unix_timestamp +#' @rdname column_datetime_functions #' @export +#' @name NULL setGeneric("unix_timestamp", function(x, format) { standardGeneric("unix_timestamp") }) -#' @rdname upper +#' @rdname column_string_functions #' @export +#' @name NULL setGeneric("upper", function(x) { standardGeneric("upper") }) -#' @rdname weekofyear +#' @rdname column_aggregate_functions #' @export +#' @name NULL +setGeneric("var", function(x, y = NULL, na.rm = FALSE, use) { standardGeneric("var") }) + +#' @rdname column_aggregate_functions +#' @export +#' @name NULL +setGeneric("variance", function(x) { standardGeneric("variance") }) + +#' @rdname column_aggregate_functions +#' @export +#' @name NULL +setGeneric("var_pop", function(x) { standardGeneric("var_pop") }) + +#' @rdname column_aggregate_functions +#' @export +#' @name NULL +setGeneric("var_samp", function(x) { standardGeneric("var_samp") }) + +#' @rdname column_datetime_functions +#' @export +#' @name NULL setGeneric("weekofyear", function(x) { standardGeneric("weekofyear") }) -#' @rdname year +#' @rdname column_datetime_functions +#' @export +#' @name NULL +setGeneric("window", function(x, ...) { standardGeneric("window") }) + +#' @rdname column_datetime_functions #' @export +#' @name NULL setGeneric("year", function(x) { standardGeneric("year") }) +###################### Spark.ML Methods ########################## + +#' @rdname fitted +#' @export +setGeneric("fitted") + +#' @param x,y For \code{glm}: logical values indicating whether the response vector +#' and model matrix used in the fitting process should be returned as +#' components of the returned value. +#' @inheritParams stats::glm #' @rdname glm #' @export setGeneric("glm") +#' @param object a fitted ML model object. +#' @param ... additional argument(s) passed to the method. +#' @rdname predict +#' @export +setGeneric("predict", function(object, ...) { standardGeneric("predict") }) + #' @rdname rbind #' @export setGeneric("rbind", signature = "...") -#' @rdname as.data.frame +#' @rdname spark.als #' @export -setGeneric("as.data.frame") +setGeneric("spark.als", function(data, ...) { standardGeneric("spark.als") }) -#' @rdname attach +#' @rdname spark.bisectingKmeans #' @export -setGeneric("attach") +setGeneric("spark.bisectingKmeans", + function(data, formula, ...) { standardGeneric("spark.bisectingKmeans") }) + +#' @rdname spark.gaussianMixture +#' @export +setGeneric("spark.gaussianMixture", + function(data, formula, ...) { standardGeneric("spark.gaussianMixture") }) + +#' @rdname spark.gbt +#' @export +setGeneric("spark.gbt", function(data, formula, ...) { standardGeneric("spark.gbt") }) + +#' @rdname spark.glm +#' @export +setGeneric("spark.glm", function(data, formula, ...) { standardGeneric("spark.glm") }) + +#' @rdname spark.isoreg +#' @export +setGeneric("spark.isoreg", function(data, formula, ...) { standardGeneric("spark.isoreg") }) + +#' @rdname spark.kmeans +#' @export +setGeneric("spark.kmeans", function(data, formula, ...) { standardGeneric("spark.kmeans") }) + +#' @rdname spark.kstest +#' @export +setGeneric("spark.kstest", function(data, ...) { standardGeneric("spark.kstest") }) + +#' @rdname spark.lda +#' @export +setGeneric("spark.lda", function(data, ...) { standardGeneric("spark.lda") }) + +#' @rdname spark.logit +#' @export +setGeneric("spark.logit", function(data, formula, ...) { standardGeneric("spark.logit") }) + +#' @rdname spark.mlp +#' @export +setGeneric("spark.mlp", function(data, formula, ...) { standardGeneric("spark.mlp") }) + +#' @rdname spark.naiveBayes +#' @export +setGeneric("spark.naiveBayes", function(data, formula, ...) { standardGeneric("spark.naiveBayes") }) + +#' @rdname spark.decisionTree +#' @export +setGeneric("spark.decisionTree", + function(data, formula, ...) { standardGeneric("spark.decisionTree") }) + +#' @rdname spark.randomForest +#' @export +setGeneric("spark.randomForest", + function(data, formula, ...) { standardGeneric("spark.randomForest") }) + +#' @rdname spark.survreg +#' @export +setGeneric("spark.survreg", function(data, formula, ...) { standardGeneric("spark.survreg") }) + +#' @rdname spark.svmLinear +#' @export +setGeneric("spark.svmLinear", function(data, formula, ...) { standardGeneric("spark.svmLinear") }) + +#' @rdname spark.lda +#' @export +setGeneric("spark.posterior", function(object, newData) { standardGeneric("spark.posterior") }) + +#' @rdname spark.lda +#' @export +setGeneric("spark.perplexity", function(object, data) { standardGeneric("spark.perplexity") }) + +#' @rdname spark.fpGrowth +#' @export +setGeneric("spark.fpGrowth", function(data, ...) { standardGeneric("spark.fpGrowth") }) + +#' @rdname spark.fpGrowth +#' @export +setGeneric("spark.freqItemsets", function(object) { standardGeneric("spark.freqItemsets") }) + +#' @rdname spark.fpGrowth +#' @export +setGeneric("spark.associationRules", function(object) { standardGeneric("spark.associationRules") }) + +#' @param object a fitted ML model object. +#' @param path the directory where the model is saved. +#' @param ... additional argument(s) passed to the method. +#' @rdname write.ml +#' @export +setGeneric("write.ml", function(object, path, ...) { standardGeneric("write.ml") }) + + +###################### Streaming Methods ########################## + +#' @rdname awaitTermination +#' @export +setGeneric("awaitTermination", function(x, timeout = NULL) { standardGeneric("awaitTermination") }) + +#' @rdname isActive +#' @export +setGeneric("isActive", function(x) { standardGeneric("isActive") }) + +#' @rdname lastProgress +#' @export +setGeneric("lastProgress", function(x) { standardGeneric("lastProgress") }) + +#' @rdname queryName +#' @export +setGeneric("queryName", function(x) { standardGeneric("queryName") }) + +#' @rdname status +#' @export +setGeneric("status", function(x) { standardGeneric("status") }) + +#' @rdname stopQuery +#' @export +setGeneric("stopQuery", function(x) { standardGeneric("stopQuery") }) diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 4cab1a69f601..0a7be0e99397 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -22,13 +22,16 @@ NULL setOldClass("jobj") -#' @title S4 class that represents a GroupedData -#' @description GroupedDatas can be created using groupBy() on a DataFrame +#' S4 class that represents a GroupedData +#' +#' GroupedDatas can be created using groupBy() on a SparkDataFrame +#' #' @rdname GroupedData #' @seealso groupBy #' #' @param sgd A Java object reference to the backing Scala GroupedData #' @export +#' @note GroupedData since 1.4.0 setClass("GroupedData", slots = list(sgd = "jobj")) @@ -37,13 +40,16 @@ setMethod("initialize", "GroupedData", function(.Object, sgd) { .Object }) -#' @rdname DataFrame +#' @rdname GroupedData groupedData <- function(sgd) { new("GroupedData", sgd) } #' @rdname show +#' @aliases show,GroupedData-method +#' @export +#' @note show(GroupedData) since 1.4.0 setMethod("show", "GroupedData", function(object) { cat("GroupedData\n") @@ -51,39 +57,44 @@ setMethod("show", "GroupedData", #' Count #' -#' Count the number of rows for each group. -#' The resulting DataFrame will also contain the grouping columns. +#' Count the number of rows for each group when we have \code{GroupedData} input. +#' The resulting SparkDataFrame will also contain the grouping columns. #' -#' @param x a GroupedData -#' @return a DataFrame -#' @rdname agg +#' @return A SparkDataFrame. +#' @rdname count +#' @aliases count,GroupedData-method #' @export #' @examples #' \dontrun{ #' count(groupBy(df, "name")) #' } +#' @note count since 1.4.0 setMethod("count", signature(x = "GroupedData"), function(x) { dataFrame(callJMethod(x@sgd, "count")) }) -#' Agg +#' summarize #' -#' Aggregates on the entire DataFrame without groups. -#' The resulting DataFrame will also contain the grouping columns. +#' Aggregates on the entire SparkDataFrame without groups. +#' The resulting SparkDataFrame will also contain the grouping columns. #' #' df2 <- agg(df, = ) #' df2 <- agg(df, newColName = aggFunction(column)) #' -#' @param x a GroupedData -#' @return a DataFrame -#' @rdname agg +#' @rdname summarize +#' @aliases agg,GroupedData-method +#' @name agg +#' @family agg_funcs +#' @export #' @examples #' \dontrun{ #' df2 <- agg(df, age = "sum") # new column name will be created as 'SUM(age#0)' -#' df2 <- agg(df, ageSum = sum(df$age)) # Creates a new column named ageSum +#' df3 <- agg(df, ageSum = sum(df$age)) # Creates a new column named ageSum +#' df4 <- summarize(df, ageSum = max(df$age)) #' } +#' @note agg since 1.4.0 setMethod("agg", signature(x = "GroupedData"), function(x, ...) { @@ -109,16 +120,65 @@ setMethod("agg", dataFrame(sdf) }) -#' @rdname agg -#' @aliases agg +#' @rdname summarize +#' @name summarize +#' @aliases summarize,GroupedData-method +#' @note summarize since 1.4.0 setMethod("summarize", signature(x = "GroupedData"), function(x, ...) { agg(x, ...) }) -# sum/mean/avg/min/max -methods <- c("sum", "mean", "avg", "min", "max") +# Aggregate Functions by name +methods <- c("avg", "max", "mean", "min", "sum") + +# These are not exposed on GroupedData: "kurtosis", "skewness", "stddev", "stddev_samp", "stddev_pop", +# "variance", "var_samp", "var_pop" + +#' Pivot a column of the GroupedData and perform the specified aggregation. +#' +#' Pivot a column of the GroupedData and perform the specified aggregation. +#' There are two versions of pivot function: one that requires the caller to specify the list +#' of distinct values to pivot on, and one that does not. The latter is more concise but less +#' efficient, because Spark needs to first compute the list of distinct values internally. +#' +#' @param x a GroupedData object +#' @param colname A column name +#' @param values A value or a list/vector of distinct values for the output columns. +#' @return GroupedData object +#' @rdname pivot +#' @aliases pivot,GroupedData,character-method +#' @name pivot +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame(data.frame( +#' earnings = c(10000, 10000, 11000, 15000, 12000, 20000, 21000, 22000), +#' course = c("R", "Python", "R", "Python", "R", "Python", "R", "Python"), +#' period = c("1H", "1H", "2H", "2H", "1H", "1H", "2H", "2H"), +#' year = c(2015, 2015, 2015, 2015, 2016, 2016, 2016, 2016) +#' )) +#' group_sum <- sum(pivot(groupBy(df, "year"), "course"), "earnings") +#' group_min <- min(pivot(groupBy(df, "year"), "course", "R"), "earnings") +#' group_max <- max(pivot(groupBy(df, "year"), "course", c("Python", "R")), "earnings") +#' group_mean <- mean(pivot(groupBy(df, "year"), "course", list("Python", "R")), "earnings") +#' } +#' @note pivot since 2.0.0 +setMethod("pivot", + signature(x = "GroupedData", colname = "character"), + function(x, colname, values = list()){ + stopifnot(length(colname) == 1) + if (length(values) == 0) { + result <- callJMethod(x@sgd, "pivot", colname) + } else { + if (length(values) > length(unique(values))) { + stop("Values are not unique") + } + result <- callJMethod(x@sgd, "pivot", colname, as.list(values)) + } + groupedData(result) + }) createMethod <- function(name) { setMethod(name, @@ -136,3 +196,57 @@ createMethods <- function() { } createMethods() + +#' gapply +#' +#' @rdname gapply +#' @aliases gapply,GroupedData-method +#' @name gapply +#' @export +#' @note gapply(GroupedData) since 2.0.0 +setMethod("gapply", + signature(x = "GroupedData"), + function(x, func, schema) { + if (is.null(schema)) stop("schema cannot be NULL") + gapplyInternal(x, func, schema) + }) + +#' gapplyCollect +#' +#' @rdname gapplyCollect +#' @aliases gapplyCollect,GroupedData-method +#' @name gapplyCollect +#' @export +#' @note gapplyCollect(GroupedData) since 2.0.0 +setMethod("gapplyCollect", + signature(x = "GroupedData"), + function(x, func) { + gdf <- gapplyInternal(x, func, NULL) + content <- callJMethod(gdf@sdf, "collect") + # content is a list of items of struct type. Each item has a single field + # which is a serialized data.frame corresponds to one group of the + # SparkDataFrame. + ldfs <- lapply(content, function(x) { unserialize(x[[1]]) }) + ldf <- do.call(rbind, ldfs) + row.names(ldf) <- NULL + ldf + }) + +gapplyInternal <- function(x, func, schema) { + if (is.character(schema)) { + schema <- structType(schema) + } + packageNamesArr <- serialize(.sparkREnv[[".packages"]], + connection = NULL) + broadcastArr <- lapply(ls(.broadcastNames), + function(name) { get(name, .broadcastNames) }) + sdf <- callJStatic( + "org.apache.spark.sql.api.r.SQLUtils", + "gapply", + x@sgd, + serialize(cleanClosure(func), connection = NULL), + packageNamesArr, + broadcastArr, + if (class(schema) == "structType") { schema$jobj } else { NULL }) + dataFrame(sdf) +} diff --git a/R/pkg/R/install.R b/R/pkg/R/install.R new file mode 100644 index 000000000000..492dee68e164 --- /dev/null +++ b/R/pkg/R/install.R @@ -0,0 +1,312 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Functions to install Spark in case the user directly downloads SparkR +# from CRAN. + +#' Download and Install Apache Spark to a Local Directory +#' +#' \code{install.spark} downloads and installs Spark to a local directory if +#' it is not found. If SPARK_HOME is set in the environment, and that directory is found, that is +#' returned. The Spark version we use is the same as the SparkR version. Users can specify a desired +#' Hadoop version, the remote mirror site, and the directory where the package is installed locally. +#' +#' The full url of remote file is inferred from \code{mirrorUrl} and \code{hadoopVersion}. +#' \code{mirrorUrl} specifies the remote path to a Spark folder. It is followed by a subfolder +#' named after the Spark version (that corresponds to SparkR), and then the tar filename. +#' The filename is composed of four parts, i.e. [Spark version]-bin-[Hadoop version].tgz. +#' For example, the full path for a Spark 2.0.0 package for Hadoop 2.7 from +#' \code{http://apache.osuosl.org} has path: +#' \code{http://apache.osuosl.org/spark/spark-2.0.0/spark-2.0.0-bin-hadoop2.7.tgz}. +#' For \code{hadoopVersion = "without"}, [Hadoop version] in the filename is then +#' \code{without-hadoop}. +#' +#' @param hadoopVersion Version of Hadoop to install. Default is \code{"2.7"}. It can take other +#' version number in the format of "x.y" where x and y are integer. +#' If \code{hadoopVersion = "without"}, "Hadoop free" build is installed. +#' See +#' \href{http://spark.apache.org/docs/latest/hadoop-provided.html}{ +#' "Hadoop Free" Build} for more information. +#' Other patched version names can also be used, e.g. \code{"cdh4"} +#' @param mirrorUrl base URL of the repositories to use. The directory layout should follow +#' \href{http://www.apache.org/dyn/closer.lua/spark/}{Apache mirrors}. +#' @param localDir a local directory where Spark is installed. The directory contains +#' version-specific folders of Spark packages. Default is path to +#' the cache directory: +#' \itemize{ +#' \item Mac OS X: \file{~/Library/Caches/spark} +#' \item Unix: \env{$XDG_CACHE_HOME} if defined, otherwise \file{~/.cache/spark} +#' \item Windows: \file{\%LOCALAPPDATA\%\\Apache\\Spark\\Cache}. +#' } +#' @param overwrite If \code{TRUE}, download and overwrite the existing tar file in localDir +#' and force re-install Spark (in case the local directory or file is corrupted) +#' @return the (invisible) local directory where Spark is found or installed +#' @rdname install.spark +#' @name install.spark +#' @aliases install.spark +#' @export +#' @examples +#'\dontrun{ +#' install.spark() +#'} +#' @note install.spark since 2.1.0 +#' @seealso See available Hadoop versions: +#' \href{http://spark.apache.org/downloads.html}{Apache Spark} +install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL, + localDir = NULL, overwrite = FALSE) { + sparkHome <- Sys.getenv("SPARK_HOME") + if (isSparkRShell()) { + stopifnot(nchar(sparkHome) > 0) + message("Spark is already running in sparkR shell.") + return(invisible(sparkHome)) + } else if (!is.na(file.info(sparkHome)$isdir)) { + message("Spark package found in SPARK_HOME: ", sparkHome) + return(invisible(sparkHome)) + } + + version <- paste0("spark-", packageVersion("SparkR")) + hadoopVersion <- tolower(hadoopVersion) + hadoopVersionName <- hadoopVersionName(hadoopVersion) + packageName <- paste(version, "bin", hadoopVersionName, sep = "-") + localDir <- ifelse(is.null(localDir), sparkCachePath(), + normalizePath(localDir, mustWork = FALSE)) + + if (is.na(file.info(localDir)$isdir)) { + dir.create(localDir, recursive = TRUE) + } + + if (overwrite) { + message(paste0("Overwrite = TRUE: download and overwrite the tar file", + "and Spark package directory if they exist.")) + } + + releaseUrl <- Sys.getenv("SPARKR_RELEASE_DOWNLOAD_URL") + if (releaseUrl != "") { + packageName <- basenameSansExtFromUrl(releaseUrl) + } + + packageLocalDir <- file.path(localDir, packageName) + + # can use dir.exists(packageLocalDir) under R 3.2.0 or later + if (!is.na(file.info(packageLocalDir)$isdir) && !overwrite) { + if (releaseUrl != "") { + message(paste(packageName, "found, setting SPARK_HOME to", packageLocalDir)) + } else { + fmt <- "%s for Hadoop %s found, setting SPARK_HOME to %s" + msg <- sprintf(fmt, version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion), + packageLocalDir) + message(msg) + } + Sys.setenv(SPARK_HOME = packageLocalDir) + return(invisible(packageLocalDir)) + } else { + message("Spark not found in the cache directory. Installation will start.") + } + + packageLocalPath <- paste0(packageLocalDir, ".tgz") + tarExists <- file.exists(packageLocalPath) + + if (tarExists && !overwrite) { + message("tar file found.") + } else { + if (releaseUrl != "") { + message("Downloading from alternate URL:\n- ", releaseUrl) + success <- downloadUrl(releaseUrl, packageLocalPath) + if (!success) { + unlink(packageLocalPath) + stop(paste0("Fetch failed from ", releaseUrl)) + } + } else { + robustDownloadTar(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) + } + } + + message(sprintf("Installing to %s", localDir)) + # There are two ways untar can fail - untar could stop() on errors like incomplete block on file + # or, tar command can return failure code + success <- tryCatch(untar(tarfile = packageLocalPath, exdir = localDir) == 0, + error = function(e) { + message(e) + message() + FALSE + }, + warning = function(w) { + # Treat warning as error, add an empty line with message() + message(w) + message() + FALSE + }) + if (!tarExists || overwrite || !success) { + unlink(packageLocalPath) + } + if (!success) stop("Extract archive failed.") + message("DONE.") + Sys.setenv(SPARK_HOME = packageLocalDir) + message(paste("SPARK_HOME set to", packageLocalDir)) + invisible(packageLocalDir) +} + +robustDownloadTar <- function(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) { + # step 1: use user-provided url + if (!is.null(mirrorUrl)) { + message("Use user-provided mirror site: ", mirrorUrl) + success <- directDownloadTar(mirrorUrl, version, hadoopVersion, + packageName, packageLocalPath) + if (success) { + return() + } else { + message(paste0("Unable to download from mirrorUrl: ", mirrorUrl)) + } + } else { + message("MirrorUrl not provided.") + } + + # step 2: use url suggested from apache website + message("Looking for preferred site from apache website...") + mirrorUrl <- getPreferredMirror(version, packageName) + if (!is.null(mirrorUrl)) { + success <- directDownloadTar(mirrorUrl, version, hadoopVersion, + packageName, packageLocalPath) + if (success) return() + } else { + message("Unable to download from preferred mirror site: ", mirrorUrl) + } + + # step 3: use backup option + message("To use backup site...") + mirrorUrl <- defaultMirrorUrl() + success <- directDownloadTar(mirrorUrl, version, hadoopVersion, + packageName, packageLocalPath) + if (success) { + return() + } else { + # remove any partially downloaded file + unlink(packageLocalPath) + message("Unable to download from default mirror site: ", mirrorUrl) + msg <- sprintf(paste("Unable to download Spark %s for Hadoop %s.", + "Please check network connection, Hadoop version,", + "or provide other mirror sites."), + version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion)) + stop(msg) + } +} + +getPreferredMirror <- function(version, packageName) { + jsonUrl <- paste0("http://www.apache.org/dyn/closer.cgi?path=", + file.path("spark", version, packageName), + ".tgz&as_json=1") + textLines <- readLines(jsonUrl, warn = FALSE) + rowNum <- grep("\"preferred\"", textLines) + linePreferred <- textLines[rowNum] + matchInfo <- regexpr("\"[A-Za-z][A-Za-z0-9+-.]*://.+\"", linePreferred) + if (matchInfo != -1) { + startPos <- matchInfo + 1 + endPos <- matchInfo + attr(matchInfo, "match.length") - 2 + mirrorPreferred <- base::substr(linePreferred, startPos, endPos) + mirrorPreferred <- paste0(mirrorPreferred, "spark") + message(sprintf("Preferred mirror site found: %s", mirrorPreferred)) + } else { + mirrorPreferred <- NULL + } + mirrorPreferred +} + +directDownloadTar <- function(mirrorUrl, version, hadoopVersion, packageName, packageLocalPath) { + packageRemotePath <- paste0(file.path(mirrorUrl, version, packageName), ".tgz") + fmt <- "Downloading %s for Hadoop %s from:\n- %s" + msg <- sprintf(fmt, version, ifelse(hadoopVersion == "without", "Free build", hadoopVersion), + packageRemotePath) + message(msg) + downloadUrl(packageRemotePath, packageLocalPath) +} + +downloadUrl <- function(remotePath, localPath) { + isFail <- tryCatch(download.file(remotePath, localPath), + error = function(e) { + message(e) + message() + TRUE + }, + warning = function(w) { + # Treat warning as error, add an empty line with message() + message(w) + message() + TRUE + }) + !isFail +} + +defaultMirrorUrl <- function() { + "http://www-us.apache.org/dist/spark" +} + +hadoopVersionName <- function(hadoopVersion) { + if (hadoopVersion == "without") { + "without-hadoop" + } else if (grepl("^[0-9]+\\.[0-9]+$", hadoopVersion, perl = TRUE)) { + paste0("hadoop", hadoopVersion) + } else { + hadoopVersion + } +} + +# The implementation refers to appdirs package: https://pypi.python.org/pypi/appdirs and +# adapt to Spark context +sparkCachePath <- function() { + if (is_windows()) { + winAppPath <- Sys.getenv("LOCALAPPDATA", unset = NA) + if (is.na(winAppPath)) { + message("%LOCALAPPDATA% not found. Falling back to %USERPROFILE%.") + winAppPath <- Sys.getenv("USERPROFILE", unset = NA) + } + if (is.na(winAppPath)) { + stop(paste("%LOCALAPPDATA% and %USERPROFILE% not found.", + "Please define the environment variable", + "or restart and enter an installation path in localDir.")) + } else { + path <- file.path(winAppPath, "Apache", "Spark", "Cache") + } + } else if (.Platform$OS.type == "unix") { + if (Sys.info()["sysname"] == "Darwin") { + path <- file.path(Sys.getenv("HOME"), "Library/Caches", "spark") + } else { + path <- file.path( + Sys.getenv("XDG_CACHE_HOME", file.path(Sys.getenv("HOME"), ".cache")), "spark") + } + } else { + stop(sprintf("Unknown OS: %s", .Platform$OS.type)) + } + normalizePath(path, mustWork = FALSE) +} + + +installInstruction <- function(mode) { + if (mode == "remote") { + paste0("Connecting to a remote Spark master. ", + "Please make sure Spark package is also installed in this machine.\n", + "- If there is one, set the path in sparkHome parameter or ", + "environment variable SPARK_HOME.\n", + "- If not, you may run install.spark function to do the job. ", + "Please make sure the Spark and the Hadoop versions ", + "match the versions on the cluster. ", + "SparkR package is compatible with Spark ", packageVersion("SparkR"), ".", + "If you need further help, ", + "contact the administrators of the cluster.") + } else { + stop(paste0("No instruction found for ", mode, " mode.")) + } +} diff --git a/R/pkg/R/jobj.R b/R/pkg/R/jobj.R index 0838a7bb35e0..4905e1fe5c61 100644 --- a/R/pkg/R/jobj.R +++ b/R/pkg/R/jobj.R @@ -71,12 +71,17 @@ jobj <- function(objId) { #' #' @param x The JVM object reference #' @param ... further arguments passed to or from other methods +#' @note print.jobj since 1.4.0 print.jobj <- function(x, ...) { - cls <- callJMethod(x, "getClass") - name <- callJMethod(cls, "getName") + name <- getClassName.jobj(x) cat("Java ref type", name, "id", x$id, "\n", sep = " ") } +getClassName.jobj <- function(x) { + cls <- callJMethod(x, "getClass") + callJMethod(cls, "getName") +} + cleanup.jobj <- function(jobj) { if (isValidJobj(jobj)) { objId <- jobj$id diff --git a/R/pkg/R/jvm.R b/R/pkg/R/jvm.R new file mode 100644 index 000000000000..bb5c77544a3d --- /dev/null +++ b/R/pkg/R/jvm.R @@ -0,0 +1,117 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Methods to directly access the JVM running the SparkR backend. + +#' Call Java Methods +#' +#' Call a Java method in the JVM running the Spark driver. The return +#' values are automatically converted to R objects for simple objects. Other +#' values are returned as "jobj" which are references to objects on JVM. +#' +#' @details +#' This is a low level function to access the JVM directly and should only be used +#' for advanced use cases. The arguments and return values that are primitive R +#' types (like integer, numeric, character, lists) are automatically translated to/from +#' Java types (like Integer, Double, String, Array). A full list can be found in +#' serialize.R and deserialize.R in the Apache Spark code base. +#' +#' @param x object to invoke the method on. Should be a "jobj" created by newJObject. +#' @param methodName method name to call. +#' @param ... parameters to pass to the Java method. +#' @return the return value of the Java method. Either returned as a R object +#' if it can be deserialized or returned as a "jobj". See details section for more. +#' @export +#' @seealso \link{sparkR.callJStatic}, \link{sparkR.newJObject} +#' @rdname sparkR.callJMethod +#' @examples +#' \dontrun{ +#' sparkR.session() # Need to have a Spark JVM running before calling newJObject +#' # Create a Java ArrayList and populate it +#' jarray <- sparkR.newJObject("java.util.ArrayList") +#' sparkR.callJMethod(jarray, "add", 42L) +#' sparkR.callJMethod(jarray, "get", 0L) # Will print 42 +#' } +#' @note sparkR.callJMethod since 2.0.1 +sparkR.callJMethod <- function(x, methodName, ...) { + callJMethod(x, methodName, ...) +} + +#' Call Static Java Methods +#' +#' Call a static method in the JVM running the Spark driver. The return +#' value is automatically converted to R objects for simple objects. Other +#' values are returned as "jobj" which are references to objects on JVM. +#' +#' @details +#' This is a low level function to access the JVM directly and should only be used +#' for advanced use cases. The arguments and return values that are primitive R +#' types (like integer, numeric, character, lists) are automatically translated to/from +#' Java types (like Integer, Double, String, Array). A full list can be found in +#' serialize.R and deserialize.R in the Apache Spark code base. +#' +#' @param x fully qualified Java class name that contains the static method to invoke. +#' @param methodName name of static method to invoke. +#' @param ... parameters to pass to the Java method. +#' @return the return value of the Java method. Either returned as a R object +#' if it can be deserialized or returned as a "jobj". See details section for more. +#' @export +#' @seealso \link{sparkR.callJMethod}, \link{sparkR.newJObject} +#' @rdname sparkR.callJStatic +#' @examples +#' \dontrun{ +#' sparkR.session() # Need to have a Spark JVM running before calling callJStatic +#' sparkR.callJStatic("java.lang.System", "currentTimeMillis") +#' sparkR.callJStatic("java.lang.System", "getProperty", "java.home") +#' } +#' @note sparkR.callJStatic since 2.0.1 +sparkR.callJStatic <- function(x, methodName, ...) { + callJStatic(x, methodName, ...) +} + +#' Create Java Objects +#' +#' Create a new Java object in the JVM running the Spark driver. The return +#' value is automatically converted to an R object for simple objects. Other +#' values are returned as a "jobj" which is a reference to an object on JVM. +#' +#' @details +#' This is a low level function to access the JVM directly and should only be used +#' for advanced use cases. The arguments and return values that are primitive R +#' types (like integer, numeric, character, lists) are automatically translated to/from +#' Java types (like Integer, Double, String, Array). A full list can be found in +#' serialize.R and deserialize.R in the Apache Spark code base. +#' +#' @param x fully qualified Java class name. +#' @param ... arguments to be passed to the constructor. +#' @return the object created. Either returned as a R object +#' if it can be deserialized or returned as a "jobj". See details section for more. +#' @export +#' @seealso \link{sparkR.callJMethod}, \link{sparkR.callJStatic} +#' @rdname sparkR.newJObject +#' @examples +#' \dontrun{ +#' sparkR.session() # Need to have a Spark JVM running before calling newJObject +#' # Create a Java ArrayList and populate it +#' jarray <- sparkR.newJObject("java.util.ArrayList") +#' sparkR.callJMethod(jarray, "add", 42L) +#' sparkR.callJMethod(jarray, "get", 0L) # Will print 42 +#' } +#' @note sparkR.newJObject since 2.0.1 +sparkR.newJObject <- function(x, ...) { + newJObject(x, ...) +} diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R deleted file mode 100644 index 60bfadb8e750..000000000000 --- a/R/pkg/R/mllib.R +++ /dev/null @@ -1,101 +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. -# - -# mllib.R: Provides methods for MLlib integration - -#' @title S4 class that represents a PipelineModel -#' @param model A Java object reference to the backing Scala PipelineModel -#' @export -setClass("PipelineModel", representation(model = "jobj")) - -#' Fits a generalized linear model -#' -#' Fits a generalized linear model, similarly to R's glm(). Also see the glmnet package. -#' -#' @param formula A symbolic description of the model to be fitted. Currently only a few formula -#' operators are supported, including '~', '.', ':', '+', and '-'. -#' @param data DataFrame for training -#' @param family Error distribution. "gaussian" -> linear regression, "binomial" -> logistic reg. -#' @param lambda Regularization parameter -#' @param alpha Elastic-net mixing parameter (see glmnet's documentation for details) -#' @return a fitted MLlib model -#' @rdname glm -#' @export -#' @examples -#'\dontrun{ -#' sc <- sparkR.init() -#' sqlContext <- sparkRSQL.init(sc) -#' data(iris) -#' df <- createDataFrame(sqlContext, iris) -#' model <- glm(Sepal_Length ~ Sepal_Width, df, family="gaussian") -#' summary(model) -#'} -setMethod("glm", signature(formula = "formula", family = "ANY", data = "DataFrame"), - function(formula, family = c("gaussian", "binomial"), data, lambda = 0, alpha = 0, - standardize = TRUE, solver = "auto") { - family <- match.arg(family) - model <- callJStatic("org.apache.spark.ml.api.r.SparkRWrappers", - "fitRModelFormula", deparse(formula), data@sdf, family, lambda, - alpha, standardize, solver) - return(new("PipelineModel", model = model)) - }) - -#' Make predictions from a model -#' -#' Makes predictions from a model produced by glm(), similarly to R's predict(). -#' -#' @param object A fitted MLlib model -#' @param newData DataFrame for testing -#' @return DataFrame containing predicted values -#' @rdname predict -#' @export -#' @examples -#'\dontrun{ -#' model <- glm(y ~ x, trainingData) -#' predicted <- predict(model, testData) -#' showDF(predicted) -#'} -setMethod("predict", signature(object = "PipelineModel"), - function(object, newData) { - return(dataFrame(callJMethod(object@model, "transform", newData@sdf))) - }) - -#' Get the summary of a model -#' -#' Returns the summary of a model produced by glm(), similarly to R's summary(). -#' -#' @param x A fitted MLlib model -#' @return a list with a 'coefficient' component, which is the matrix of coefficients. See -#' summary.glm for more information. -#' @rdname summary -#' @export -#' @examples -#'\dontrun{ -#' model <- glm(y ~ x, trainingData) -#' summary(model) -#'} -setMethod("summary", signature(x = "PipelineModel"), - function(x, ...) { - features <- callJStatic("org.apache.spark.ml.api.r.SparkRWrappers", - "getModelFeatures", x@model) - coefficients <- callJStatic("org.apache.spark.ml.api.r.SparkRWrappers", - "getModelCoefficients", x@model) - coefficients <- as.matrix(unlist(coefficients)) - colnames(coefficients) <- c("Estimate") - rownames(coefficients) <- unlist(features) - return(list(coefficients = coefficients)) - }) diff --git a/R/pkg/R/mllib_classification.R b/R/pkg/R/mllib_classification.R new file mode 100644 index 000000000000..15af8298ba48 --- /dev/null +++ b/R/pkg/R/mllib_classification.R @@ -0,0 +1,635 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# mllib_regression.R: Provides methods for MLlib classification algorithms +# (except for tree-based algorithms) integration + +#' S4 class that represents an LinearSVCModel +#' +#' @param jobj a Java object reference to the backing Scala LinearSVCModel +#' @export +#' @note LinearSVCModel since 2.2.0 +setClass("LinearSVCModel", representation(jobj = "jobj")) + +#' S4 class that represents an LogisticRegressionModel +#' +#' @param jobj a Java object reference to the backing Scala LogisticRegressionModel +#' @export +#' @note LogisticRegressionModel since 2.1.0 +setClass("LogisticRegressionModel", representation(jobj = "jobj")) + +#' S4 class that represents a MultilayerPerceptronClassificationModel +#' +#' @param jobj a Java object reference to the backing Scala MultilayerPerceptronClassifierWrapper +#' @export +#' @note MultilayerPerceptronClassificationModel since 2.1.0 +setClass("MultilayerPerceptronClassificationModel", representation(jobj = "jobj")) + +#' S4 class that represents a NaiveBayesModel +#' +#' @param jobj a Java object reference to the backing Scala NaiveBayesWrapper +#' @export +#' @note NaiveBayesModel since 2.0.0 +setClass("NaiveBayesModel", representation(jobj = "jobj")) + +#' Linear SVM Model +#' +#' Fits a linear SVM model against a SparkDataFrame, similar to svm in e1071 package. +#' Currently only supports binary classification model with linear kernel. +#' Users can print, make predictions on the produced model and save the model to the input path. +#' +#' @param data SparkDataFrame for training. +#' @param formula A symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', '.', ':', '+', and '-'. +#' @param regParam The regularization parameter. Only supports L2 regularization currently. +#' @param maxIter Maximum iteration number. +#' @param tol Convergence tolerance of iterations. +#' @param standardization Whether to standardize the training features before fitting the model. The coefficients +#' of models will be always returned on the original scale, so it will be transparent for +#' users. Note that with/without standardization, the models should be always converged +#' to the same solution when no regularization is applied. +#' @param threshold The threshold in binary classification applied to the linear model prediction. +#' This threshold can be any real number, where Inf will make all predictions 0.0 +#' and -Inf will make all predictions 1.0. +#' @param weightCol The weight column name. +#' @param aggregationDepth The depth for treeAggregate (greater than or equal to 2). If the dimensions of features +#' or the number of partitions are large, this param could be adjusted to a larger size. +#' This is an expert parameter. Default value should be good for most cases. +#' @param handleInvalid How to handle invalid data (unseen labels or NULL values) in features and label +#' column of string type. +#' Supported options: "skip" (filter out rows with invalid data), +#' "error" (throw an error), "keep" (put invalid data in a special additional +#' bucket, at index numLabels). Default is "error". +#' @param ... additional arguments passed to the method. +#' @return \code{spark.svmLinear} returns a fitted linear SVM model. +#' @rdname spark.svmLinear +#' @aliases spark.svmLinear,SparkDataFrame,formula-method +#' @name spark.svmLinear +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' t <- as.data.frame(Titanic) +#' training <- createDataFrame(t) +#' model <- spark.svmLinear(training, Survived ~ ., regParam = 0.5) +#' summary <- summary(model) +#' +#' # fitted values on training data +#' fitted <- predict(model, training) +#' +#' # save fitted model to input path +#' path <- "path/to/model" +#' write.ml(model, path) +#' +#' # can also read back the saved model and predict +#' # Note that summary deos not work on loaded model +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' } +#' @note spark.svmLinear since 2.2.0 +setMethod("spark.svmLinear", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, regParam = 0.0, maxIter = 100, tol = 1E-6, standardization = TRUE, + threshold = 0.0, weightCol = NULL, aggregationDepth = 2, + handleInvalid = c("error", "keep", "skip")) { + formula <- paste(deparse(formula), collapse = "") + + if (!is.null(weightCol) && weightCol == "") { + weightCol <- NULL + } else if (!is.null(weightCol)) { + weightCol <- as.character(weightCol) + } + + handleInvalid <- match.arg(handleInvalid) + + jobj <- callJStatic("org.apache.spark.ml.r.LinearSVCWrapper", "fit", + data@sdf, formula, as.numeric(regParam), as.integer(maxIter), + as.numeric(tol), as.logical(standardization), as.numeric(threshold), + weightCol, as.integer(aggregationDepth), handleInvalid) + new("LinearSVCModel", jobj = jobj) + }) + +# Predicted values based on a LinearSVCModel model + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns the predicted values based on a LinearSVCModel. +#' @rdname spark.svmLinear +#' @aliases predict,LinearSVCModel,SparkDataFrame-method +#' @export +#' @note predict(LinearSVCModel) since 2.2.0 +setMethod("predict", signature(object = "LinearSVCModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Get the summary of a LinearSVCModel + +#' @param object a LinearSVCModel fitted by \code{spark.svmLinear}. +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list includes \code{coefficients} (coefficients of the fitted model), +#' \code{numClasses} (number of classes), \code{numFeatures} (number of features). +#' @rdname spark.svmLinear +#' @aliases summary,LinearSVCModel-method +#' @export +#' @note summary(LinearSVCModel) since 2.2.0 +setMethod("summary", signature(object = "LinearSVCModel"), + function(object) { + jobj <- object@jobj + features <- callJMethod(jobj, "rFeatures") + coefficients <- callJMethod(jobj, "rCoefficients") + coefficients <- as.matrix(unlist(coefficients)) + colnames(coefficients) <- c("Estimate") + rownames(coefficients) <- unlist(features) + numClasses <- callJMethod(jobj, "numClasses") + numFeatures <- callJMethod(jobj, "numFeatures") + list(coefficients = coefficients, numClasses = numClasses, numFeatures = numFeatures) + }) + +# Save fitted LinearSVCModel to the input path + +#' @param path The directory where the model is saved. +#' @param overwrite Overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' +#' @rdname spark.svmLinear +#' @aliases write.ml,LinearSVCModel,character-method +#' @export +#' @note write.ml(LogisticRegression, character) since 2.2.0 +setMethod("write.ml", signature(object = "LinearSVCModel", path = "character"), +function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) +}) + +#' Logistic Regression Model +#' +#' Fits an logistic regression model against a SparkDataFrame. It supports "binomial": Binary logistic regression +#' with pivoting; "multinomial": Multinomial logistic (softmax) regression without pivoting, similar to glmnet. +#' Users can print, make predictions on the produced model and save the model to the input path. +#' +#' @param data SparkDataFrame for training. +#' @param formula A symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', '.', ':', '+', and '-'. +#' @param regParam the regularization parameter. +#' @param elasticNetParam the ElasticNet mixing parameter. For alpha = 0.0, the penalty is an L2 penalty. +#' For alpha = 1.0, it is an L1 penalty. For 0.0 < alpha < 1.0, the penalty is a combination +#' of L1 and L2. Default is 0.0 which is an L2 penalty. +#' @param maxIter maximum iteration number. +#' @param tol convergence tolerance of iterations. +#' @param family the name of family which is a description of the label distribution to be used in the model. +#' Supported options: +#' \itemize{ +#' \item{"auto": Automatically select the family based on the number of classes: +#' If number of classes == 1 || number of classes == 2, set to "binomial". +#' Else, set to "multinomial".} +#' \item{"binomial": Binary logistic regression with pivoting.} +#' \item{"multinomial": Multinomial logistic (softmax) regression without pivoting.} +#' } +#' @param standardization whether to standardize the training features before fitting the model. The coefficients +#' of models will be always returned on the original scale, so it will be transparent for +#' users. Note that with/without standardization, the models should be always converged +#' to the same solution when no regularization is applied. Default is TRUE, same as glmnet. +#' @param thresholds in binary classification, in range [0, 1]. If the estimated probability of class label 1 +#' is > threshold, then predict 1, else 0. A high threshold encourages the model to predict 0 +#' more often; a low threshold encourages the model to predict 1 more often. Note: Setting this with +#' threshold p is equivalent to setting thresholds c(1-p, p). In multiclass (or binary) classification to adjust the probability of +#' predicting each class. Array must have length equal to the number of classes, with values > 0, +#' excepting that at most one value may be 0. The class with largest value p/t is predicted, where p +#' is the original probability of that class and t is the class's threshold. +#' @param weightCol The weight column name. +#' @param aggregationDepth The depth for treeAggregate (greater than or equal to 2). If the dimensions of features +#' or the number of partitions are large, this param could be adjusted to a larger size. +#' This is an expert parameter. Default value should be good for most cases. +#' @param lowerBoundsOnCoefficients The lower bounds on coefficients if fitting under bound constrained optimization. +#' The bound matrix must be compatible with the shape (1, number of features) for binomial +#' regression, or (number of classes, number of features) for multinomial regression. +#' It is a R matrix. +#' @param upperBoundsOnCoefficients The upper bounds on coefficients if fitting under bound constrained optimization. +#' The bound matrix must be compatible with the shape (1, number of features) for binomial +#' regression, or (number of classes, number of features) for multinomial regression. +#' It is a R matrix. +#' @param lowerBoundsOnIntercepts The lower bounds on intercepts if fitting under bound constrained optimization. +#' The bounds vector size must be equal to 1 for binomial regression, or the number +#' of classes for multinomial regression. +#' @param upperBoundsOnIntercepts The upper bounds on intercepts if fitting under bound constrained optimization. +#' The bound vector size must be equal to 1 for binomial regression, or the number +#' of classes for multinomial regression. +#' @param handleInvalid How to handle invalid data (unseen labels or NULL values) in features and label +#' column of string type. +#' Supported options: "skip" (filter out rows with invalid data), +#' "error" (throw an error), "keep" (put invalid data in a special additional +#' bucket, at index numLabels). Default is "error". +#' @param ... additional arguments passed to the method. +#' @return \code{spark.logit} returns a fitted logistic regression model. +#' @rdname spark.logit +#' @aliases spark.logit,SparkDataFrame,formula-method +#' @name spark.logit +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' # binary logistic regression +#' t <- as.data.frame(Titanic) +#' training <- createDataFrame(t) +#' model <- spark.logit(training, Survived ~ ., regParam = 0.5) +#' summary <- summary(model) +#' +#' # fitted values on training data +#' fitted <- predict(model, training) +#' +#' # save fitted model to input path +#' path <- "path/to/model" +#' write.ml(model, path) +#' +#' # can also read back the saved model and predict +#' # Note that summary deos not work on loaded model +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' +#' # multinomial logistic regression +#' +#' model <- spark.logit(training, Class ~ ., regParam = 0.5) +#' summary <- summary(model) +#' +#' } +#' @note spark.logit since 2.1.0 +setMethod("spark.logit", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, regParam = 0.0, elasticNetParam = 0.0, maxIter = 100, + tol = 1E-6, family = "auto", standardization = TRUE, + thresholds = 0.5, weightCol = NULL, aggregationDepth = 2, + lowerBoundsOnCoefficients = NULL, upperBoundsOnCoefficients = NULL, + lowerBoundsOnIntercepts = NULL, upperBoundsOnIntercepts = NULL, + handleInvalid = c("error", "keep", "skip")) { + formula <- paste(deparse(formula), collapse = "") + row <- 0 + col <- 0 + + if (!is.null(weightCol) && weightCol == "") { + weightCol <- NULL + } else if (!is.null(weightCol)) { + weightCol <- as.character(weightCol) + } + + if (!is.null(lowerBoundsOnIntercepts)) { + lowerBoundsOnIntercepts <- as.array(lowerBoundsOnIntercepts) + } + + if (!is.null(upperBoundsOnIntercepts)) { + upperBoundsOnIntercepts <- as.array(upperBoundsOnIntercepts) + } + + if (!is.null(lowerBoundsOnCoefficients)) { + if (class(lowerBoundsOnCoefficients) != "matrix") { + stop("lowerBoundsOnCoefficients must be a matrix.") + } + row <- nrow(lowerBoundsOnCoefficients) + col <- ncol(lowerBoundsOnCoefficients) + lowerBoundsOnCoefficients <- as.array(as.vector(lowerBoundsOnCoefficients)) + } + + if (!is.null(upperBoundsOnCoefficients)) { + if (class(upperBoundsOnCoefficients) != "matrix") { + stop("upperBoundsOnCoefficients must be a matrix.") + } + + if (!is.null(lowerBoundsOnCoefficients) && (row != nrow(upperBoundsOnCoefficients) + || col != ncol(upperBoundsOnCoefficients))) { + stop(paste0("dimension of upperBoundsOnCoefficients ", + "is not the same as lowerBoundsOnCoefficients", sep = "")) + } + + if (is.null(lowerBoundsOnCoefficients)) { + row <- nrow(upperBoundsOnCoefficients) + col <- ncol(upperBoundsOnCoefficients) + } + + upperBoundsOnCoefficients <- as.array(as.vector(upperBoundsOnCoefficients)) + } + + handleInvalid <- match.arg(handleInvalid) + + jobj <- callJStatic("org.apache.spark.ml.r.LogisticRegressionWrapper", "fit", + data@sdf, formula, as.numeric(regParam), + as.numeric(elasticNetParam), as.integer(maxIter), + as.numeric(tol), as.character(family), + as.logical(standardization), as.array(thresholds), + weightCol, as.integer(aggregationDepth), + as.integer(row), as.integer(col), + lowerBoundsOnCoefficients, upperBoundsOnCoefficients, + lowerBoundsOnIntercepts, upperBoundsOnIntercepts, + handleInvalid) + new("LogisticRegressionModel", jobj = jobj) + }) + +# Get the summary of an LogisticRegressionModel + +#' @param object an LogisticRegressionModel fitted by \code{spark.logit}. +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list includes \code{coefficients} (coefficients matrix of the fitted model). +#' @rdname spark.logit +#' @aliases summary,LogisticRegressionModel-method +#' @export +#' @note summary(LogisticRegressionModel) since 2.1.0 +setMethod("summary", signature(object = "LogisticRegressionModel"), + function(object) { + jobj <- object@jobj + features <- callJMethod(jobj, "rFeatures") + labels <- callJMethod(jobj, "labels") + coefficients <- callJMethod(jobj, "rCoefficients") + nCol <- length(coefficients) / length(features) + coefficients <- matrix(unlist(coefficients), ncol = nCol) + # If nCol == 1, means this is a binomial logistic regression model with pivoting. + # Otherwise, it's a multinomial logistic regression model without pivoting. + if (nCol == 1) { + colnames(coefficients) <- c("Estimate") + } else { + colnames(coefficients) <- unlist(labels) + } + rownames(coefficients) <- unlist(features) + + list(coefficients = coefficients) + }) + +# Predicted values based on an LogisticRegressionModel model + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns the predicted values based on an LogisticRegressionModel. +#' @rdname spark.logit +#' @aliases predict,LogisticRegressionModel,SparkDataFrame-method +#' @export +#' @note predict(LogisticRegressionModel) since 2.1.0 +setMethod("predict", signature(object = "LogisticRegressionModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Save fitted LogisticRegressionModel to the input path + +#' @param path The directory where the model is saved. +#' @param overwrite Overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' +#' @rdname spark.logit +#' @aliases write.ml,LogisticRegressionModel,character-method +#' @export +#' @note write.ml(LogisticRegression, character) since 2.1.0 +setMethod("write.ml", signature(object = "LogisticRegressionModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +#' Multilayer Perceptron Classification Model +#' +#' \code{spark.mlp} fits a multi-layer perceptron neural network model against a SparkDataFrame. +#' Users can call \code{summary} to print a summary of the fitted model, \code{predict} to make +#' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. +#' Only categorical data is supported. +#' For more details, see +#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html}{ +#' Multilayer Perceptron} +#' +#' @param data a \code{SparkDataFrame} of observations and labels for model fitting. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', '.', ':', '+', and '-'. +#' @param blockSize blockSize parameter. +#' @param layers integer vector containing the number of nodes for each layer. +#' @param solver solver parameter, supported options: "gd" (minibatch gradient descent) or "l-bfgs". +#' @param maxIter maximum iteration number. +#' @param tol convergence tolerance of iterations. +#' @param stepSize stepSize parameter. +#' @param seed seed parameter for weights initialization. +#' @param initialWeights initialWeights parameter for weights initialization, it should be a +#' numeric vector. +#' @param handleInvalid How to handle invalid data (unseen labels or NULL values) in features and label +#' column of string type. +#' Supported options: "skip" (filter out rows with invalid data), +#' "error" (throw an error), "keep" (put invalid data in a special additional +#' bucket, at index numLabels). Default is "error". +#' @param ... additional arguments passed to the method. +#' @return \code{spark.mlp} returns a fitted Multilayer Perceptron Classification Model. +#' @rdname spark.mlp +#' @aliases spark.mlp,SparkDataFrame,formula-method +#' @name spark.mlp +#' @seealso \link{read.ml} +#' @export +#' @examples +#' \dontrun{ +#' df <- read.df("data/mllib/sample_multiclass_classification_data.txt", source = "libsvm") +#' +#' # fit a Multilayer Perceptron Classification Model +#' model <- spark.mlp(df, label ~ features, blockSize = 128, layers = c(4, 3), solver = "l-bfgs", +#' maxIter = 100, tol = 0.5, stepSize = 1, seed = 1, +#' initialWeights = c(0, 0, 0, 0, 0, 5, 5, 5, 5, 5, 9, 9, 9, 9, 9)) +#' +#' # get the summary of the model +#' summary(model) +#' +#' # make predictions +#' predictions <- predict(model, df) +#' +#' # save and load the model +#' path <- "path/to/model" +#' write.ml(model, path) +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' } +#' @note spark.mlp since 2.1.0 +setMethod("spark.mlp", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, layers, blockSize = 128, solver = "l-bfgs", maxIter = 100, + tol = 1E-6, stepSize = 0.03, seed = NULL, initialWeights = NULL, + handleInvalid = c("error", "keep", "skip")) { + formula <- paste(deparse(formula), collapse = "") + if (is.null(layers)) { + stop ("layers must be a integer vector with length > 1.") + } + layers <- as.integer(na.omit(layers)) + if (length(layers) <= 1) { + stop ("layers must be a integer vector with length > 1.") + } + if (!is.null(seed)) { + seed <- as.character(as.integer(seed)) + } + if (!is.null(initialWeights)) { + initialWeights <- as.array(as.numeric(na.omit(initialWeights))) + } + handleInvalid <- match.arg(handleInvalid) + jobj <- callJStatic("org.apache.spark.ml.r.MultilayerPerceptronClassifierWrapper", + "fit", data@sdf, formula, as.integer(blockSize), as.array(layers), + as.character(solver), as.integer(maxIter), as.numeric(tol), + as.numeric(stepSize), seed, initialWeights, handleInvalid) + new("MultilayerPerceptronClassificationModel", jobj = jobj) + }) + +# Returns the summary of a Multilayer Perceptron Classification Model produced by \code{spark.mlp} + +#' @param object a Multilayer Perceptron Classification Model fitted by \code{spark.mlp} +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list includes \code{numOfInputs} (number of inputs), \code{numOfOutputs} +#' (number of outputs), \code{layers} (array of layer sizes including input +#' and output layers), and \code{weights} (the weights of layers). +#' For \code{weights}, it is a numeric vector with length equal to the expected +#' given the architecture (i.e., for 8-10-2 network, 112 connection weights). +#' @rdname spark.mlp +#' @export +#' @aliases summary,MultilayerPerceptronClassificationModel-method +#' @note summary(MultilayerPerceptronClassificationModel) since 2.1.0 +setMethod("summary", signature(object = "MultilayerPerceptronClassificationModel"), + function(object) { + jobj <- object@jobj + layers <- unlist(callJMethod(jobj, "layers")) + numOfInputs <- head(layers, n = 1) + numOfOutputs <- tail(layers, n = 1) + weights <- callJMethod(jobj, "weights") + list(numOfInputs = numOfInputs, numOfOutputs = numOfOutputs, + layers = layers, weights = weights) + }) + +# Makes predictions from a model produced by spark.mlp(). + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns a SparkDataFrame containing predicted labeled in a column named +#' "prediction". +#' @rdname spark.mlp +#' @aliases predict,MultilayerPerceptronClassificationModel-method +#' @export +#' @note predict(MultilayerPerceptronClassificationModel) since 2.1.0 +setMethod("predict", signature(object = "MultilayerPerceptronClassificationModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Saves the Multilayer Perceptron Classification Model to the input path. + +#' @param path the directory where the model is saved. +#' @param overwrite overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' +#' @rdname spark.mlp +#' @aliases write.ml,MultilayerPerceptronClassificationModel,character-method +#' @export +#' @seealso \link{write.ml} +#' @note write.ml(MultilayerPerceptronClassificationModel, character) since 2.1.0 +setMethod("write.ml", signature(object = "MultilayerPerceptronClassificationModel", + path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +#' Naive Bayes Models +#' +#' \code{spark.naiveBayes} fits a Bernoulli naive Bayes model against a SparkDataFrame. +#' Users can call \code{summary} to print a summary of the fitted model, \code{predict} to make +#' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. +#' Only categorical data is supported. +#' +#' @param data a \code{SparkDataFrame} of observations and labels for model fitting. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', '.', ':', '+', and '-'. +#' @param smoothing smoothing parameter. +#' @param handleInvalid How to handle invalid data (unseen labels or NULL values) in features and label +#' column of string type. +#' Supported options: "skip" (filter out rows with invalid data), +#' "error" (throw an error), "keep" (put invalid data in a special additional +#' bucket, at index numLabels). Default is "error". +#' @param ... additional argument(s) passed to the method. Currently only \code{smoothing}. +#' @return \code{spark.naiveBayes} returns a fitted naive Bayes model. +#' @rdname spark.naiveBayes +#' @aliases spark.naiveBayes,SparkDataFrame,formula-method +#' @name spark.naiveBayes +#' @seealso e1071: \url{https://cran.r-project.org/package=e1071} +#' @export +#' @examples +#' \dontrun{ +#' data <- as.data.frame(UCBAdmissions) +#' df <- createDataFrame(data) +#' +#' # fit a Bernoulli naive Bayes model +#' model <- spark.naiveBayes(df, Admit ~ Gender + Dept, smoothing = 0) +#' +#' # get the summary of the model +#' summary(model) +#' +#' # make predictions +#' predictions <- predict(model, df) +#' +#' # save and load the model +#' path <- "path/to/model" +#' write.ml(model, path) +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' } +#' @note spark.naiveBayes since 2.0.0 +setMethod("spark.naiveBayes", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, smoothing = 1.0, + handleInvalid = c("error", "keep", "skip")) { + formula <- paste(deparse(formula), collapse = "") + handleInvalid <- match.arg(handleInvalid) + jobj <- callJStatic("org.apache.spark.ml.r.NaiveBayesWrapper", "fit", + formula, data@sdf, smoothing, handleInvalid) + new("NaiveBayesModel", jobj = jobj) + }) + +# Returns the summary of a naive Bayes model produced by \code{spark.naiveBayes} + +#' @param object a naive Bayes model fitted by \code{spark.naiveBayes}. +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list includes \code{apriori} (the label distribution) and +#' \code{tables} (conditional probabilities given the target label). +#' @rdname spark.naiveBayes +#' @export +#' @note summary(NaiveBayesModel) since 2.0.0 +setMethod("summary", signature(object = "NaiveBayesModel"), + function(object) { + jobj <- object@jobj + features <- callJMethod(jobj, "features") + labels <- callJMethod(jobj, "labels") + apriori <- callJMethod(jobj, "apriori") + apriori <- t(as.matrix(unlist(apriori))) + colnames(apriori) <- unlist(labels) + tables <- callJMethod(jobj, "tables") + tables <- matrix(tables, nrow = length(labels)) + rownames(tables) <- unlist(labels) + colnames(tables) <- unlist(features) + list(apriori = apriori, tables = tables) + }) + +# Makes predictions from a naive Bayes model or a model produced by spark.naiveBayes(), +# similarly to R package e1071's predict. + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns a SparkDataFrame containing predicted labeled in a column named +#' "prediction". +#' @rdname spark.naiveBayes +#' @export +#' @note predict(NaiveBayesModel) since 2.0.0 +setMethod("predict", signature(object = "NaiveBayesModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Saves the Bernoulli naive Bayes model to the input path. + +#' @param path the directory where the model is saved. +#' @param overwrite overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' +#' @rdname spark.naiveBayes +#' @export +#' @seealso \link{write.ml} +#' @note write.ml(NaiveBayesModel, character) since 2.0.0 +setMethod("write.ml", signature(object = "NaiveBayesModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) diff --git a/R/pkg/R/mllib_clustering.R b/R/pkg/R/mllib_clustering.R new file mode 100644 index 000000000000..97c9fa1b4584 --- /dev/null +++ b/R/pkg/R/mllib_clustering.R @@ -0,0 +1,634 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# mllib_clustering.R: Provides methods for MLlib clustering algorithms integration + +#' S4 class that represents a BisectingKMeansModel +#' +#' @param jobj a Java object reference to the backing Scala BisectingKMeansModel +#' @export +#' @note BisectingKMeansModel since 2.2.0 +setClass("BisectingKMeansModel", representation(jobj = "jobj")) + +#' S4 class that represents a GaussianMixtureModel +#' +#' @param jobj a Java object reference to the backing Scala GaussianMixtureModel +#' @export +#' @note GaussianMixtureModel since 2.1.0 +setClass("GaussianMixtureModel", representation(jobj = "jobj")) + +#' S4 class that represents a KMeansModel +#' +#' @param jobj a Java object reference to the backing Scala KMeansModel +#' @export +#' @note KMeansModel since 2.0.0 +setClass("KMeansModel", representation(jobj = "jobj")) + +#' S4 class that represents an LDAModel +#' +#' @param jobj a Java object reference to the backing Scala LDAWrapper +#' @export +#' @note LDAModel since 2.1.0 +setClass("LDAModel", representation(jobj = "jobj")) + +#' Bisecting K-Means Clustering Model +#' +#' Fits a bisecting k-means clustering model against a SparkDataFrame. +#' Users can call \code{summary} to print a summary of the fitted model, \code{predict} to make +#' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. +#' +#' @param data a SparkDataFrame for training. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', '.', ':', '+', and '-'. +#' Note that the response variable of formula is empty in spark.bisectingKmeans. +#' @param k the desired number of leaf clusters. Must be > 1. +#' The actual number could be smaller if there are no divisible leaf clusters. +#' @param maxIter maximum iteration number. +#' @param seed the random seed. +#' @param minDivisibleClusterSize The minimum number of points (if greater than or equal to 1.0) +#' or the minimum proportion of points (if less than 1.0) of a divisible cluster. +#' Note that it is an expert parameter. The default value should be good enough +#' for most cases. +#' @param ... additional argument(s) passed to the method. +#' @return \code{spark.bisectingKmeans} returns a fitted bisecting k-means model. +#' @rdname spark.bisectingKmeans +#' @aliases spark.bisectingKmeans,SparkDataFrame,formula-method +#' @name spark.bisectingKmeans +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' t <- as.data.frame(Titanic) +#' df <- createDataFrame(t) +#' model <- spark.bisectingKmeans(df, Class ~ Survived, k = 4) +#' summary(model) +#' +#' # get fitted result from a bisecting k-means model +#' fitted.model <- fitted(model, "centers") +#' showDF(fitted.model) +#' +#' # fitted values on training data +#' fitted <- predict(model, df) +#' head(select(fitted, "Class", "prediction")) +#' +#' # save fitted model to input path +#' path <- "path/to/model" +#' write.ml(model, path) +#' +#' # can also read back the saved model and print +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' } +#' @note spark.bisectingKmeans since 2.2.0 +#' @seealso \link{predict}, \link{read.ml}, \link{write.ml} +setMethod("spark.bisectingKmeans", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, k = 4, maxIter = 20, seed = NULL, minDivisibleClusterSize = 1.0) { + formula <- paste0(deparse(formula), collapse = "") + if (!is.null(seed)) { + seed <- as.character(as.integer(seed)) + } + jobj <- callJStatic("org.apache.spark.ml.r.BisectingKMeansWrapper", "fit", + data@sdf, formula, as.integer(k), as.integer(maxIter), + seed, as.numeric(minDivisibleClusterSize)) + new("BisectingKMeansModel", jobj = jobj) + }) + +# Get the summary of a bisecting k-means model + +#' @param object a fitted bisecting k-means model. +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list includes the model's \code{k} (number of cluster centers), +#' \code{coefficients} (model cluster centers), +#' \code{size} (number of data points in each cluster), \code{cluster} +#' (cluster centers of the transformed data; cluster is NULL if is.loaded is TRUE), +#' and \code{is.loaded} (whether the model is loaded from a saved file). +#' @rdname spark.bisectingKmeans +#' @export +#' @note summary(BisectingKMeansModel) since 2.2.0 +setMethod("summary", signature(object = "BisectingKMeansModel"), + function(object) { + jobj <- object@jobj + is.loaded <- callJMethod(jobj, "isLoaded") + features <- callJMethod(jobj, "features") + coefficients <- callJMethod(jobj, "coefficients") + k <- callJMethod(jobj, "k") + size <- callJMethod(jobj, "size") + coefficients <- t(matrix(coefficients, ncol = k)) + colnames(coefficients) <- unlist(features) + rownames(coefficients) <- 1:k + cluster <- if (is.loaded) { + NULL + } else { + dataFrame(callJMethod(jobj, "cluster")) + } + list(k = k, coefficients = coefficients, size = size, + cluster = cluster, is.loaded = is.loaded) + }) + +# Predicted values based on a bisecting k-means model + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns the predicted values based on a bisecting k-means model. +#' @rdname spark.bisectingKmeans +#' @export +#' @note predict(BisectingKMeansModel) since 2.2.0 +setMethod("predict", signature(object = "BisectingKMeansModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +#' Get fitted result from a bisecting k-means model +#' +#' Get fitted result from a bisecting k-means model. +#' Note: A saved-loaded model does not support this method. +#' +#' @param method type of fitted results, \code{"centers"} for cluster centers +#' or \code{"classes"} for assigned classes. +#' @return \code{fitted} returns a SparkDataFrame containing fitted values. +#' @rdname spark.bisectingKmeans +#' @export +#' @note fitted since 2.2.0 +setMethod("fitted", signature(object = "BisectingKMeansModel"), + function(object, method = c("centers", "classes")) { + method <- match.arg(method) + jobj <- object@jobj + is.loaded <- callJMethod(jobj, "isLoaded") + if (is.loaded) { + stop("Saved-loaded bisecting k-means model does not support 'fitted' method") + } else { + dataFrame(callJMethod(jobj, "fitted", method)) + } + }) + +# Save fitted MLlib model to the input path + +#' @param path the directory where the model is saved. +#' @param overwrite overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' +#' @rdname spark.bisectingKmeans +#' @export +#' @note write.ml(BisectingKMeansModel, character) since 2.2.0 +setMethod("write.ml", signature(object = "BisectingKMeansModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +#' Multivariate Gaussian Mixture Model (GMM) +#' +#' Fits multivariate gaussian mixture model against a SparkDataFrame, similarly to R's +#' mvnormalmixEM(). Users can call \code{summary} to print a summary of the fitted model, +#' \code{predict} to make predictions on new data, and \code{write.ml}/\code{read.ml} +#' to save/load fitted models. +#' +#' @param data a SparkDataFrame for training. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', '.', ':', '+', and '-'. +#' Note that the response variable of formula is empty in spark.gaussianMixture. +#' @param k number of independent Gaussians in the mixture model. +#' @param maxIter maximum iteration number. +#' @param tol the convergence tolerance. +#' @param ... additional arguments passed to the method. +#' @aliases spark.gaussianMixture,SparkDataFrame,formula-method +#' @return \code{spark.gaussianMixture} returns a fitted multivariate gaussian mixture model. +#' @rdname spark.gaussianMixture +#' @name spark.gaussianMixture +#' @seealso mixtools: \url{https://cran.r-project.org/package=mixtools} +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' library(mvtnorm) +#' set.seed(100) +#' a <- rmvnorm(4, c(0, 0)) +#' b <- rmvnorm(6, c(3, 4)) +#' data <- rbind(a, b) +#' df <- createDataFrame(as.data.frame(data)) +#' model <- spark.gaussianMixture(df, ~ V1 + V2, k = 2) +#' summary(model) +#' +#' # fitted values on training data +#' fitted <- predict(model, df) +#' head(select(fitted, "V1", "prediction")) +#' +#' # save fitted model to input path +#' path <- "path/to/model" +#' write.ml(model, path) +#' +#' # can also read back the saved model and print +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' } +#' @note spark.gaussianMixture since 2.1.0 +#' @seealso \link{predict}, \link{read.ml}, \link{write.ml} +setMethod("spark.gaussianMixture", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, k = 2, maxIter = 100, tol = 0.01) { + formula <- paste(deparse(formula), collapse = "") + jobj <- callJStatic("org.apache.spark.ml.r.GaussianMixtureWrapper", "fit", data@sdf, + formula, as.integer(k), as.integer(maxIter), as.numeric(tol)) + new("GaussianMixtureModel", jobj = jobj) + }) + +# Get the summary of a multivariate gaussian mixture model + +#' @param object a fitted gaussian mixture model. +#' @return \code{summary} returns summary of the fitted model, which is a list. +#' The list includes the model's \code{lambda} (lambda), \code{mu} (mu), +#' \code{sigma} (sigma), \code{loglik} (loglik), and \code{posterior} (posterior). +#' @aliases spark.gaussianMixture,SparkDataFrame,formula-method +#' @rdname spark.gaussianMixture +#' @export +#' @note summary(GaussianMixtureModel) since 2.1.0 +setMethod("summary", signature(object = "GaussianMixtureModel"), + function(object) { + jobj <- object@jobj + is.loaded <- callJMethod(jobj, "isLoaded") + lambda <- unlist(callJMethod(jobj, "lambda")) + muList <- callJMethod(jobj, "mu") + sigmaList <- callJMethod(jobj, "sigma") + k <- callJMethod(jobj, "k") + dim <- callJMethod(jobj, "dim") + loglik <- callJMethod(jobj, "logLikelihood") + mu <- c() + for (i in 1 : k) { + start <- (i - 1) * dim + 1 + end <- i * dim + mu[[i]] <- unlist(muList[start : end]) + } + sigma <- c() + for (i in 1 : k) { + start <- (i - 1) * dim * dim + 1 + end <- i * dim * dim + sigma[[i]] <- t(matrix(sigmaList[start : end], ncol = dim)) + } + posterior <- if (is.loaded) { + NULL + } else { + dataFrame(callJMethod(jobj, "posterior")) + } + list(lambda = lambda, mu = mu, sigma = sigma, loglik = loglik, + posterior = posterior, is.loaded = is.loaded) + }) + +# Predicted values based on a gaussian mixture model + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns a SparkDataFrame containing predicted labels in a column named +#' "prediction". +#' @aliases predict,GaussianMixtureModel,SparkDataFrame-method +#' @rdname spark.gaussianMixture +#' @export +#' @note predict(GaussianMixtureModel) since 2.1.0 +setMethod("predict", signature(object = "GaussianMixtureModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Save fitted MLlib model to the input path + +#' @param path the directory where the model is saved. +#' @param overwrite overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' +#' @aliases write.ml,GaussianMixtureModel,character-method +#' @rdname spark.gaussianMixture +#' @export +#' @note write.ml(GaussianMixtureModel, character) since 2.1.0 +setMethod("write.ml", signature(object = "GaussianMixtureModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +#' K-Means Clustering Model +#' +#' Fits a k-means clustering model against a SparkDataFrame, similarly to R's kmeans(). +#' Users can call \code{summary} to print a summary of the fitted model, \code{predict} to make +#' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. +#' +#' @param data a SparkDataFrame for training. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', '.', ':', '+', and '-'. +#' Note that the response variable of formula is empty in spark.kmeans. +#' @param k number of centers. +#' @param maxIter maximum iteration number. +#' @param initMode the initialization algorithm choosen to fit the model. +#' @param seed the random seed for cluster initialization. +#' @param initSteps the number of steps for the k-means|| initialization mode. +#' This is an advanced setting, the default of 2 is almost always enough. Must be > 0. +#' @param tol convergence tolerance of iterations. +#' @param ... additional argument(s) passed to the method. +#' @return \code{spark.kmeans} returns a fitted k-means model. +#' @rdname spark.kmeans +#' @aliases spark.kmeans,SparkDataFrame,formula-method +#' @name spark.kmeans +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' t <- as.data.frame(Titanic) +#' df <- createDataFrame(t) +#' model <- spark.kmeans(df, Class ~ Survived, k = 4, initMode = "random") +#' summary(model) +#' +#' # fitted values on training data +#' fitted <- predict(model, df) +#' head(select(fitted, "Class", "prediction")) +#' +#' # save fitted model to input path +#' path <- "path/to/model" +#' write.ml(model, path) +#' +#' # can also read back the saved model and print +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' } +#' @note spark.kmeans since 2.0.0 +#' @seealso \link{predict}, \link{read.ml}, \link{write.ml} +setMethod("spark.kmeans", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, k = 2, maxIter = 20, initMode = c("k-means||", "random"), + seed = NULL, initSteps = 2, tol = 1E-4) { + formula <- paste(deparse(formula), collapse = "") + initMode <- match.arg(initMode) + if (!is.null(seed)) { + seed <- as.character(as.integer(seed)) + } + jobj <- callJStatic("org.apache.spark.ml.r.KMeansWrapper", "fit", data@sdf, formula, + as.integer(k), as.integer(maxIter), initMode, seed, + as.integer(initSteps), as.numeric(tol)) + new("KMeansModel", jobj = jobj) + }) + +# Get the summary of a k-means model + +#' @param object a fitted k-means model. +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list includes the model's \code{k} (the configured number of cluster centers), +#' \code{coefficients} (model cluster centers), +#' \code{size} (number of data points in each cluster), \code{cluster} +#' (cluster centers of the transformed data), {is.loaded} (whether the model is loaded +#' from a saved file), and \code{clusterSize} +#' (the actual number of cluster centers. When using initMode = "random", +#' \code{clusterSize} may not equal to \code{k}). +#' @rdname spark.kmeans +#' @export +#' @note summary(KMeansModel) since 2.0.0 +setMethod("summary", signature(object = "KMeansModel"), + function(object) { + jobj <- object@jobj + is.loaded <- callJMethod(jobj, "isLoaded") + features <- callJMethod(jobj, "features") + coefficients <- callJMethod(jobj, "coefficients") + k <- callJMethod(jobj, "k") + size <- callJMethod(jobj, "size") + clusterSize <- callJMethod(jobj, "clusterSize") + coefficients <- t(matrix(unlist(coefficients), ncol = clusterSize)) + colnames(coefficients) <- unlist(features) + rownames(coefficients) <- 1:clusterSize + cluster <- if (is.loaded) { + NULL + } else { + dataFrame(callJMethod(jobj, "cluster")) + } + list(k = k, coefficients = coefficients, size = size, + cluster = cluster, is.loaded = is.loaded, clusterSize = clusterSize) + }) + +# Predicted values based on a k-means model + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns the predicted values based on a k-means model. +#' @rdname spark.kmeans +#' @export +#' @note predict(KMeansModel) since 2.0.0 +setMethod("predict", signature(object = "KMeansModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +#' Get fitted result from a k-means model +#' +#' Get fitted result from a k-means model, similarly to R's fitted(). +#' Note: A saved-loaded model does not support this method. +#' +#' @param object a fitted k-means model. +#' @param method type of fitted results, \code{"centers"} for cluster centers +#' or \code{"classes"} for assigned classes. +#' @param ... additional argument(s) passed to the method. +#' @return \code{fitted} returns a SparkDataFrame containing fitted values. +#' @rdname fitted +#' @export +#' @examples +#' \dontrun{ +#' model <- spark.kmeans(trainingData, ~ ., 2) +#' fitted.model <- fitted(model) +#' showDF(fitted.model) +#'} +#' @note fitted since 2.0.0 +setMethod("fitted", signature(object = "KMeansModel"), + function(object, method = c("centers", "classes")) { + method <- match.arg(method) + jobj <- object@jobj + is.loaded <- callJMethod(jobj, "isLoaded") + if (is.loaded) { + stop("Saved-loaded k-means model does not support 'fitted' method") + } else { + dataFrame(callJMethod(jobj, "fitted", method)) + } + }) + +# Save fitted MLlib model to the input path + +#' @param path the directory where the model is saved. +#' @param overwrite overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' +#' @rdname spark.kmeans +#' @export +#' @note write.ml(KMeansModel, character) since 2.0.0 +setMethod("write.ml", signature(object = "KMeansModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +#' Latent Dirichlet Allocation +#' +#' \code{spark.lda} fits a Latent Dirichlet Allocation model on a SparkDataFrame. Users can call +#' \code{summary} to get a summary of the fitted LDA model, \code{spark.posterior} to compute +#' posterior probabilities on new data, \code{spark.perplexity} to compute log perplexity on new +#' data and \code{write.ml}/\code{read.ml} to save/load fitted models. +#' +#' @param data A SparkDataFrame for training. +#' @param features Features column name. Either libSVM-format column or character-format column is +#' valid. +#' @param k Number of topics. +#' @param maxIter Maximum iterations. +#' @param optimizer Optimizer to train an LDA model, "online" or "em", default is "online". +#' @param subsamplingRate (For online optimizer) Fraction of the corpus to be sampled and used in +#' each iteration of mini-batch gradient descent, in range (0, 1]. +#' @param topicConcentration concentration parameter (commonly named \code{beta} or \code{eta}) for +#' the prior placed on topic distributions over terms, default -1 to set automatically on the +#' Spark side. Use \code{summary} to retrieve the effective topicConcentration. Only 1-size +#' numeric is accepted. +#' @param docConcentration concentration parameter (commonly named \code{alpha}) for the +#' prior placed on documents distributions over topics (\code{theta}), default -1 to set +#' automatically on the Spark side. Use \code{summary} to retrieve the effective +#' docConcentration. Only 1-size or \code{k}-size numeric is accepted. +#' @param customizedStopWords stopwords that need to be removed from the given corpus. Ignore the +#' parameter if libSVM-format column is used as the features column. +#' @param maxVocabSize maximum vocabulary size, default 1 << 18 +#' @param ... additional argument(s) passed to the method. +#' @return \code{spark.lda} returns a fitted Latent Dirichlet Allocation model. +#' @rdname spark.lda +#' @aliases spark.lda,SparkDataFrame-method +#' @seealso topicmodels: \url{https://cran.r-project.org/package=topicmodels} +#' @export +#' @examples +#' \dontrun{ +#' text <- read.df("data/mllib/sample_lda_libsvm_data.txt", source = "libsvm") +#' model <- spark.lda(data = text, optimizer = "em") +#' +#' # get a summary of the model +#' summary(model) +#' +#' # compute posterior probabilities +#' posterior <- spark.posterior(model, text) +#' showDF(posterior) +#' +#' # compute perplexity +#' perplexity <- spark.perplexity(model, text) +#' +#' # save and load the model +#' path <- "path/to/model" +#' write.ml(model, path) +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' } +#' @note spark.lda since 2.1.0 +setMethod("spark.lda", signature(data = "SparkDataFrame"), + function(data, features = "features", k = 10, maxIter = 20, optimizer = c("online", "em"), + subsamplingRate = 0.05, topicConcentration = -1, docConcentration = -1, + customizedStopWords = "", maxVocabSize = bitwShiftL(1, 18)) { + optimizer <- match.arg(optimizer) + jobj <- callJStatic("org.apache.spark.ml.r.LDAWrapper", "fit", data@sdf, features, + as.integer(k), as.integer(maxIter), optimizer, + as.numeric(subsamplingRate), topicConcentration, + as.array(docConcentration), as.array(customizedStopWords), + maxVocabSize) + new("LDAModel", jobj = jobj) + }) + +# Returns the summary of a Latent Dirichlet Allocation model produced by \code{spark.lda} + +#' @param object A Latent Dirichlet Allocation model fitted by \code{spark.lda}. +#' @param maxTermsPerTopic Maximum number of terms to collect for each topic. Default value of 10. +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list includes +#' \item{\code{docConcentration}}{concentration parameter commonly named \code{alpha} for +#' the prior placed on documents distributions over topics \code{theta}} +#' \item{\code{topicConcentration}}{concentration parameter commonly named \code{beta} or +#' \code{eta} for the prior placed on topic distributions over terms} +#' \item{\code{logLikelihood}}{log likelihood of the entire corpus} +#' \item{\code{logPerplexity}}{log perplexity} +#' \item{\code{isDistributed}}{TRUE for distributed model while FALSE for local model} +#' \item{\code{vocabSize}}{number of terms in the corpus} +#' \item{\code{topics}}{top 10 terms and their weights of all topics} +#' \item{\code{vocabulary}}{whole terms of the training corpus, NULL if libsvm format file +#' used as training set} +#' \item{\code{trainingLogLikelihood}}{Log likelihood of the observed tokens in the training set, +#' given the current parameter estimates: +#' log P(docs | topics, topic distributions for docs, Dirichlet hyperparameters) +#' It is only for distributed LDA model (i.e., optimizer = "em")} +#' \item{\code{logPrior}}{Log probability of the current parameter estimate: +#' log P(topics, topic distributions for docs | Dirichlet hyperparameters) +#' It is only for distributed LDA model (i.e., optimizer = "em")} +#' @rdname spark.lda +#' @aliases summary,LDAModel-method +#' @export +#' @note summary(LDAModel) since 2.1.0 +setMethod("summary", signature(object = "LDAModel"), + function(object, maxTermsPerTopic) { + maxTermsPerTopic <- as.integer(ifelse(missing(maxTermsPerTopic), 10, maxTermsPerTopic)) + jobj <- object@jobj + docConcentration <- callJMethod(jobj, "docConcentration") + topicConcentration <- callJMethod(jobj, "topicConcentration") + logLikelihood <- callJMethod(jobj, "logLikelihood") + logPerplexity <- callJMethod(jobj, "logPerplexity") + isDistributed <- callJMethod(jobj, "isDistributed") + vocabSize <- callJMethod(jobj, "vocabSize") + topics <- dataFrame(callJMethod(jobj, "topics", maxTermsPerTopic)) + vocabulary <- callJMethod(jobj, "vocabulary") + trainingLogLikelihood <- if (isDistributed) { + callJMethod(jobj, "trainingLogLikelihood") + } else { + NA + } + logPrior <- if (isDistributed) { + callJMethod(jobj, "logPrior") + } else { + NA + } + list(docConcentration = unlist(docConcentration), + topicConcentration = topicConcentration, + logLikelihood = logLikelihood, logPerplexity = logPerplexity, + isDistributed = isDistributed, vocabSize = vocabSize, + topics = topics, vocabulary = unlist(vocabulary), + trainingLogLikelihood = trainingLogLikelihood, logPrior = logPrior) + }) + +# Returns the log perplexity of a Latent Dirichlet Allocation model produced by \code{spark.lda} + +#' @return \code{spark.perplexity} returns the log perplexity of given SparkDataFrame, or the log +#' perplexity of the training data if missing argument "data". +#' @rdname spark.lda +#' @aliases spark.perplexity,LDAModel-method +#' @export +#' @note spark.perplexity(LDAModel) since 2.1.0 +setMethod("spark.perplexity", signature(object = "LDAModel", data = "SparkDataFrame"), + function(object, data) { + ifelse(missing(data), callJMethod(object@jobj, "logPerplexity"), + callJMethod(object@jobj, "computeLogPerplexity", data@sdf)) + }) + +# Returns posterior probabilities from a Latent Dirichlet Allocation model produced by spark.lda() + +#' @param newData A SparkDataFrame for testing. +#' @return \code{spark.posterior} returns a SparkDataFrame containing posterior probabilities +#' vectors named "topicDistribution". +#' @rdname spark.lda +#' @aliases spark.posterior,LDAModel,SparkDataFrame-method +#' @export +#' @note spark.posterior(LDAModel) since 2.1.0 +setMethod("spark.posterior", signature(object = "LDAModel", newData = "SparkDataFrame"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Saves the Latent Dirichlet Allocation model to the input path. + +#' @param path The directory where the model is saved. +#' @param overwrite Overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' +#' @rdname spark.lda +#' @aliases write.ml,LDAModel,character-method +#' @export +#' @seealso \link{read.ml} +#' @note write.ml(LDAModel, character) since 2.1.0 +setMethod("write.ml", signature(object = "LDAModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) diff --git a/R/pkg/R/mllib_fpm.R b/R/pkg/R/mllib_fpm.R new file mode 100644 index 000000000000..dfcb45a1b66c --- /dev/null +++ b/R/pkg/R/mllib_fpm.R @@ -0,0 +1,162 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# mllib_fpm.R: Provides methods for MLlib frequent pattern mining algorithms integration + +#' S4 class that represents a FPGrowthModel +#' +#' @param jobj a Java object reference to the backing Scala FPGrowthModel +#' @export +#' @note FPGrowthModel since 2.2.0 +setClass("FPGrowthModel", slots = list(jobj = "jobj")) + +#' FP-growth +#' +#' A parallel FP-growth algorithm to mine frequent itemsets. +#' \code{spark.fpGrowth} fits a FP-growth model on a SparkDataFrame. Users can +#' \code{spark.freqItemsets} to get frequent itemsets, \code{spark.associationRules} to get +#' association rules, \code{predict} to make predictions on new data based on generated association +#' rules, and \code{write.ml}/\code{read.ml} to save/load fitted models. +#' For more details, see +#' \href{https://spark.apache.org/docs/latest/mllib-frequent-pattern-mining.html#fp-growth}{ +#' FP-growth}. +#' +#' @param data A SparkDataFrame for training. +#' @param minSupport Minimal support level. +#' @param minConfidence Minimal confidence level. +#' @param itemsCol Features column name. +#' @param numPartitions Number of partitions used for fitting. +#' @param ... additional argument(s) passed to the method. +#' @return \code{spark.fpGrowth} returns a fitted FPGrowth model. +#' @rdname spark.fpGrowth +#' @name spark.fpGrowth +#' @aliases spark.fpGrowth,SparkDataFrame-method +#' @export +#' @examples +#' \dontrun{ +#' raw_data <- read.df( +#' "data/mllib/sample_fpgrowth.txt", +#' source = "csv", +#' schema = structType(structField("raw_items", "string"))) +#' +#' data <- selectExpr(raw_data, "split(raw_items, ' ') as items") +#' model <- spark.fpGrowth(data) +#' +#' # Show frequent itemsets +#' frequent_itemsets <- spark.freqItemsets(model) +#' showDF(frequent_itemsets) +#' +#' # Show association rules +#' association_rules <- spark.associationRules(model) +#' showDF(association_rules) +#' +#' # Predict on new data +#' new_itemsets <- data.frame(items = c("t", "t,s")) +#' new_data <- selectExpr(createDataFrame(new_itemsets), "split(items, ',') as items") +#' predict(model, new_data) +#' +#' # Save and load model +#' path <- "/path/to/model" +#' write.ml(model, path) +#' read.ml(path) +#' +#' # Optional arguments +#' baskets_data <- selectExpr(createDataFrame(itemsets), "split(items, ',') as baskets") +#' another_model <- spark.fpGrowth(data, minSupport = 0.1, minConfidence = 0.5, +#' itemsCol = "baskets", numPartitions = 10) +#' } +#' @note spark.fpGrowth since 2.2.0 +setMethod("spark.fpGrowth", signature(data = "SparkDataFrame"), + function(data, minSupport = 0.3, minConfidence = 0.8, + itemsCol = "items", numPartitions = NULL) { + if (!is.numeric(minSupport) || minSupport < 0 || minSupport > 1) { + stop("minSupport should be a number [0, 1].") + } + if (!is.numeric(minConfidence) || minConfidence < 0 || minConfidence > 1) { + stop("minConfidence should be a number [0, 1].") + } + if (!is.null(numPartitions)) { + numPartitions <- as.integer(numPartitions) + stopifnot(numPartitions > 0) + } + + jobj <- callJStatic("org.apache.spark.ml.r.FPGrowthWrapper", "fit", + data@sdf, as.numeric(minSupport), as.numeric(minConfidence), + itemsCol, numPartitions) + new("FPGrowthModel", jobj = jobj) + }) + +# Get frequent itemsets. + +#' @param object a fitted FPGrowth model. +#' @return A \code{SparkDataFrame} with frequent itemsets. +#' The \code{SparkDataFrame} contains two columns: +#' \code{items} (an array of the same type as the input column) +#' and \code{freq} (frequency of the itemset). +#' @rdname spark.fpGrowth +#' @aliases freqItemsets,FPGrowthModel-method +#' @export +#' @note spark.freqItemsets(FPGrowthModel) since 2.2.0 +setMethod("spark.freqItemsets", signature(object = "FPGrowthModel"), + function(object) { + dataFrame(callJMethod(object@jobj, "freqItemsets")) + }) + +# Get association rules. + +#' @return A \code{SparkDataFrame} with association rules. +#' The \code{SparkDataFrame} contains three columns: +#' \code{antecedent} (an array of the same type as the input column), +#' \code{consequent} (an array of the same type as the input column), +#' and \code{condfidence} (confidence). +#' @rdname spark.fpGrowth +#' @aliases associationRules,FPGrowthModel-method +#' @export +#' @note spark.associationRules(FPGrowthModel) since 2.2.0 +setMethod("spark.associationRules", signature(object = "FPGrowthModel"), + function(object) { + dataFrame(callJMethod(object@jobj, "associationRules")) + }) + +# Makes predictions based on generated association rules + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns a SparkDataFrame containing predicted values. +#' @rdname spark.fpGrowth +#' @aliases predict,FPGrowthModel-method +#' @export +#' @note predict(FPGrowthModel) since 2.2.0 +setMethod("predict", signature(object = "FPGrowthModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Saves the FPGrowth model to the output path. + +#' @param path the directory where the model is saved. +#' @param overwrite logical value indicating whether to overwrite if the output path +#' already exists. Default is FALSE which means throw exception +#' if the output path exists. +#' @rdname spark.fpGrowth +#' @aliases write.ml,FPGrowthModel,character-method +#' @export +#' @seealso \link{read.ml} +#' @note write.ml(FPGrowthModel, character) since 2.2.0 +setMethod("write.ml", signature(object = "FPGrowthModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) diff --git a/R/pkg/R/mllib_recommendation.R b/R/pkg/R/mllib_recommendation.R new file mode 100644 index 000000000000..fa794249085d --- /dev/null +++ b/R/pkg/R/mllib_recommendation.R @@ -0,0 +1,162 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# mllib_recommendation.R: Provides methods for MLlib recommendation algorithms integration + +#' S4 class that represents an ALSModel +#' +#' @param jobj a Java object reference to the backing Scala ALSWrapper +#' @export +#' @note ALSModel since 2.1.0 +setClass("ALSModel", representation(jobj = "jobj")) + +#' Alternating Least Squares (ALS) for Collaborative Filtering +#' +#' \code{spark.als} learns latent factors in collaborative filtering via alternating least +#' squares. Users can call \code{summary} to obtain fitted latent factors, \code{predict} +#' to make predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. +#' +#' For more details, see +#' \href{http://spark.apache.org/docs/latest/ml-collaborative-filtering.html}{MLlib: +#' Collaborative Filtering}. +#' +#' @param data a SparkDataFrame for training. +#' @param ratingCol column name for ratings. +#' @param userCol column name for user ids. Ids must be (or can be coerced into) integers. +#' @param itemCol column name for item ids. Ids must be (or can be coerced into) integers. +#' @param rank rank of the matrix factorization (> 0). +#' @param regParam regularization parameter (>= 0). +#' @param maxIter maximum number of iterations (>= 0). +#' @param nonnegative logical value indicating whether to apply nonnegativity constraints. +#' @param implicitPrefs logical value indicating whether to use implicit preference. +#' @param alpha alpha parameter in the implicit preference formulation (>= 0). +#' @param seed integer seed for random number generation. +#' @param numUserBlocks number of user blocks used to parallelize computation (> 0). +#' @param numItemBlocks number of item blocks used to parallelize computation (> 0). +#' @param checkpointInterval number of checkpoint intervals (>= 1) or disable checkpoint (-1). +#' @param ... additional argument(s) passed to the method. +#' @return \code{spark.als} returns a fitted ALS model. +#' @rdname spark.als +#' @aliases spark.als,SparkDataFrame-method +#' @name spark.als +#' @export +#' @examples +#' \dontrun{ +#' ratings <- list(list(0, 0, 4.0), list(0, 1, 2.0), list(1, 1, 3.0), list(1, 2, 4.0), +#' list(2, 1, 1.0), list(2, 2, 5.0)) +#' df <- createDataFrame(ratings, c("user", "item", "rating")) +#' model <- spark.als(df, "rating", "user", "item") +#' +#' # extract latent factors +#' stats <- summary(model) +#' userFactors <- stats$userFactors +#' itemFactors <- stats$itemFactors +#' +#' # make predictions +#' predicted <- predict(model, df) +#' showDF(predicted) +#' +#' # save and load the model +#' path <- "path/to/model" +#' write.ml(model, path) +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' +#' # set other arguments +#' modelS <- spark.als(df, "rating", "user", "item", rank = 20, +#' regParam = 0.1, nonnegative = TRUE) +#' statsS <- summary(modelS) +#' } +#' @note spark.als since 2.1.0 +setMethod("spark.als", signature(data = "SparkDataFrame"), + function(data, ratingCol = "rating", userCol = "user", itemCol = "item", + rank = 10, regParam = 0.1, maxIter = 10, nonnegative = FALSE, + implicitPrefs = FALSE, alpha = 1.0, numUserBlocks = 10, numItemBlocks = 10, + checkpointInterval = 10, seed = 0) { + + if (!is.numeric(rank) || rank <= 0) { + stop("rank should be a positive number.") + } + if (!is.numeric(regParam) || regParam < 0) { + stop("regParam should be a nonnegative number.") + } + if (!is.numeric(maxIter) || maxIter <= 0) { + stop("maxIter should be a positive number.") + } + + jobj <- callJStatic("org.apache.spark.ml.r.ALSWrapper", + "fit", data@sdf, ratingCol, userCol, itemCol, as.integer(rank), + regParam, as.integer(maxIter), implicitPrefs, alpha, nonnegative, + as.integer(numUserBlocks), as.integer(numItemBlocks), + as.integer(checkpointInterval), as.integer(seed)) + new("ALSModel", jobj = jobj) + }) + +# Returns a summary of the ALS model produced by spark.als. + +#' @param object a fitted ALS model. +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list includes \code{user} (the names of the user column), +#' \code{item} (the item column), \code{rating} (the rating column), \code{userFactors} +#' (the estimated user factors), \code{itemFactors} (the estimated item factors), +#' and \code{rank} (rank of the matrix factorization model). +#' @rdname spark.als +#' @aliases summary,ALSModel-method +#' @export +#' @note summary(ALSModel) since 2.1.0 +setMethod("summary", signature(object = "ALSModel"), + function(object) { + jobj <- object@jobj + user <- callJMethod(jobj, "userCol") + item <- callJMethod(jobj, "itemCol") + rating <- callJMethod(jobj, "ratingCol") + userFactors <- dataFrame(callJMethod(jobj, "userFactors")) + itemFactors <- dataFrame(callJMethod(jobj, "itemFactors")) + rank <- callJMethod(jobj, "rank") + list(user = user, item = item, rating = rating, userFactors = userFactors, + itemFactors = itemFactors, rank = rank) + }) + +# Makes predictions from an ALS model or a model produced by spark.als. + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns a SparkDataFrame containing predicted values. +#' @rdname spark.als +#' @aliases predict,ALSModel-method +#' @export +#' @note predict(ALSModel) since 2.1.0 +setMethod("predict", signature(object = "ALSModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Saves the ALS model to the input path. + +#' @param path the directory where the model is saved. +#' @param overwrite logical value indicating whether to overwrite if the output path +#' already exists. Default is FALSE which means throw exception +#' if the output path exists. +#' +#' @rdname spark.als +#' @aliases write.ml,ALSModel,character-method +#' @export +#' @seealso \link{read.ml} +#' @note write.ml(ALSModel, character) since 2.1.0 +setMethod("write.ml", signature(object = "ALSModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) diff --git a/R/pkg/R/mllib_regression.R b/R/pkg/R/mllib_regression.R new file mode 100644 index 000000000000..ebaeae970218 --- /dev/null +++ b/R/pkg/R/mllib_regression.R @@ -0,0 +1,552 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# mllib_regression.R: Provides methods for MLlib regression algorithms +# (except for tree-based algorithms) integration + +#' S4 class that represents a AFTSurvivalRegressionModel +#' +#' @param jobj a Java object reference to the backing Scala AFTSurvivalRegressionWrapper +#' @export +#' @note AFTSurvivalRegressionModel since 2.0.0 +setClass("AFTSurvivalRegressionModel", representation(jobj = "jobj")) + +#' S4 class that represents a generalized linear model +#' +#' @param jobj a Java object reference to the backing Scala GeneralizedLinearRegressionWrapper +#' @export +#' @note GeneralizedLinearRegressionModel since 2.0.0 +setClass("GeneralizedLinearRegressionModel", representation(jobj = "jobj")) + +#' S4 class that represents an IsotonicRegressionModel +#' +#' @param jobj a Java object reference to the backing Scala IsotonicRegressionModel +#' @export +#' @note IsotonicRegressionModel since 2.1.0 +setClass("IsotonicRegressionModel", representation(jobj = "jobj")) + +#' Generalized Linear Models +#' +#' Fits generalized linear model against a SparkDataFrame. +#' Users can call \code{summary} to print a summary of the fitted model, \code{predict} to make +#' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. +#' +#' @param data a SparkDataFrame for training. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', '.', ':', '+', and '-'. +#' @param family a description of the error distribution and link function to be used in the model. +#' This can be a character string naming a family function, a family function or +#' the result of a call to a family function. Refer R family at +#' \url{https://stat.ethz.ch/R-manual/R-devel/library/stats/html/family.html}. +#' Currently these families are supported: \code{binomial}, \code{gaussian}, +#' \code{Gamma}, \code{poisson} and \code{tweedie}. +#' +#' Note that there are two ways to specify the tweedie family. +#' \itemize{ +#' \item Set \code{family = "tweedie"} and specify the var.power and link.power; +#' \item When package \code{statmod} is loaded, the tweedie family is specified using the +#' family definition therein, i.e., \code{tweedie(var.power, link.power)}. +#' } +#' @param tol positive convergence tolerance of iterations. +#' @param maxIter integer giving the maximal number of IRLS iterations. +#' @param weightCol the weight column name. If this is not set or \code{NULL}, we treat all instance +#' weights as 1.0. +#' @param regParam regularization parameter for L2 regularization. +#' @param var.power the power in the variance function of the Tweedie distribution which provides +#' the relationship between the variance and mean of the distribution. Only +#' applicable to the Tweedie family. +#' @param link.power the index in the power link function. Only applicable to the Tweedie family. +#' @param stringIndexerOrderType how to order categories of a string feature column. This is used to +#' decide the base level of a string feature as the last category after +#' ordering is dropped when encoding strings. Supported options are +#' "frequencyDesc", "frequencyAsc", "alphabetDesc", and "alphabetAsc". +#' The default value is "frequencyDesc". When the ordering is set to +#' "alphabetDesc", this drops the same category as R when encoding strings. +#' @param offsetCol the offset column name. If this is not set or empty, we treat all instance offsets +#' as 0.0. The feature specified as offset has a constant coefficient of 1.0. +#' @param ... additional arguments passed to the method. +#' @aliases spark.glm,SparkDataFrame,formula-method +#' @return \code{spark.glm} returns a fitted generalized linear model. +#' @rdname spark.glm +#' @name spark.glm +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' t <- as.data.frame(Titanic, stringsAsFactors = FALSE) +#' df <- createDataFrame(t) +#' model <- spark.glm(df, Freq ~ Sex + Age, family = "gaussian") +#' summary(model) +#' +#' # fitted values on training data +#' fitted <- predict(model, df) +#' head(select(fitted, "Freq", "prediction")) +#' +#' # save fitted model to input path +#' path <- "path/to/model" +#' write.ml(model, path) +#' +#' # can also read back the saved model and print +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' +#' # note that the default string encoding is different from R's glm +#' model2 <- glm(Freq ~ Sex + Age, family = "gaussian", data = t) +#' summary(model2) +#' # use stringIndexerOrderType = "alphabetDesc" to force string encoding +#' # to be consistent with R +#' model3 <- spark.glm(df, Freq ~ Sex + Age, family = "gaussian", +#' stringIndexerOrderType = "alphabetDesc") +#' summary(model3) +#' +#' # fit tweedie model +#' model <- spark.glm(df, Freq ~ Sex + Age, family = "tweedie", +#' var.power = 1.2, link.power = 0) +#' summary(model) +#' +#' # use the tweedie family from statmod +#' library(statmod) +#' model <- spark.glm(df, Freq ~ Sex + Age, family = tweedie(1.2, 0)) +#' summary(model) +#' } +#' @note spark.glm since 2.0.0 +#' @seealso \link{glm}, \link{read.ml} +setMethod("spark.glm", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, family = gaussian, tol = 1e-6, maxIter = 25, weightCol = NULL, + regParam = 0.0, var.power = 0.0, link.power = 1.0 - var.power, + stringIndexerOrderType = c("frequencyDesc", "frequencyAsc", + "alphabetDesc", "alphabetAsc"), + offsetCol = NULL) { + + stringIndexerOrderType <- match.arg(stringIndexerOrderType) + if (is.character(family)) { + # Handle when family = "tweedie" + if (tolower(family) == "tweedie") { + family <- list(family = "tweedie", link = NULL) + } else { + family <- get(family, mode = "function", envir = parent.frame()) + } + } + if (is.function(family)) { + family <- family() + } + if (is.null(family$family)) { + print(family) + stop("'family' not recognized") + } + # Handle when family = statmod::tweedie() + if (tolower(family$family) == "tweedie" && !is.null(family$variance)) { + var.power <- log(family$variance(exp(1))) + link.power <- log(family$linkfun(exp(1))) + family <- list(family = "tweedie", link = NULL) + } + + formula <- paste(deparse(formula), collapse = "") + if (!is.null(weightCol) && weightCol == "") { + weightCol <- NULL + } else if (!is.null(weightCol)) { + weightCol <- as.character(weightCol) + } + + if (!is.null(offsetCol)) { + offsetCol <- as.character(offsetCol) + if (nchar(offsetCol) == 0) { + offsetCol <- NULL + } + } + + # For known families, Gamma is upper-cased + jobj <- callJStatic("org.apache.spark.ml.r.GeneralizedLinearRegressionWrapper", + "fit", formula, data@sdf, tolower(family$family), family$link, + tol, as.integer(maxIter), weightCol, regParam, + as.double(var.power), as.double(link.power), + stringIndexerOrderType, offsetCol) + new("GeneralizedLinearRegressionModel", jobj = jobj) + }) + +#' Generalized Linear Models (R-compliant) +#' +#' Fits a generalized linear model, similarly to R's glm(). +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', '.', ':', '+', and '-'. +#' @param data a SparkDataFrame or R's glm data for training. +#' @param family a description of the error distribution and link function to be used in the model. +#' This can be a character string naming a family function, a family function or +#' the result of a call to a family function. Refer R family at +#' \url{https://stat.ethz.ch/R-manual/R-devel/library/stats/html/family.html}. +#' Currently these families are supported: \code{binomial}, \code{gaussian}, +#' \code{poisson}, \code{Gamma}, and \code{tweedie}. +#' @param weightCol the weight column name. If this is not set or \code{NULL}, we treat all instance +#' weights as 1.0. +#' @param epsilon positive convergence tolerance of iterations. +#' @param maxit integer giving the maximal number of IRLS iterations. +#' @param var.power the index of the power variance function in the Tweedie family. +#' @param link.power the index of the power link function in the Tweedie family. +#' @param stringIndexerOrderType how to order categories of a string feature column. This is used to +#' decide the base level of a string feature as the last category after +#' ordering is dropped when encoding strings. Supported options are +#' "frequencyDesc", "frequencyAsc", "alphabetDesc", and "alphabetAsc". +#' The default value is "frequencyDesc". When the ordering is set to +#' "alphabetDesc", this drops the same category as R when encoding strings. +#' @param offsetCol the offset column name. If this is not set or empty, we treat all instance offsets +#' as 0.0. The feature specified as offset has a constant coefficient of 1.0. +#' @return \code{glm} returns a fitted generalized linear model. +#' @rdname glm +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' t <- as.data.frame(Titanic) +#' df <- createDataFrame(t) +#' model <- glm(Freq ~ Sex + Age, df, family = "gaussian") +#' summary(model) +#' } +#' @note glm since 1.5.0 +#' @seealso \link{spark.glm} +setMethod("glm", signature(formula = "formula", family = "ANY", data = "SparkDataFrame"), + function(formula, family = gaussian, data, epsilon = 1e-6, maxit = 25, weightCol = NULL, + var.power = 0.0, link.power = 1.0 - var.power, + stringIndexerOrderType = c("frequencyDesc", "frequencyAsc", + "alphabetDesc", "alphabetAsc"), + offsetCol = NULL) { + spark.glm(data, formula, family, tol = epsilon, maxIter = maxit, weightCol = weightCol, + var.power = var.power, link.power = link.power, + stringIndexerOrderType = stringIndexerOrderType, + offsetCol = offsetCol) + }) + +# Returns the summary of a model produced by glm() or spark.glm(), similarly to R's summary(). + +#' @param object a fitted generalized linear model. +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list of components includes at least the \code{coefficients} (coefficients matrix, which includes +#' coefficients, standard error of coefficients, t value and p value), +#' \code{null.deviance} (null/residual degrees of freedom), \code{aic} (AIC) +#' and \code{iter} (number of iterations IRLS takes). If there are collinear columns in the data, +#' the coefficients matrix only provides coefficients. +#' @rdname spark.glm +#' @export +#' @note summary(GeneralizedLinearRegressionModel) since 2.0.0 +setMethod("summary", signature(object = "GeneralizedLinearRegressionModel"), + function(object) { + jobj <- object@jobj + is.loaded <- callJMethod(jobj, "isLoaded") + features <- callJMethod(jobj, "rFeatures") + coefficients <- callJMethod(jobj, "rCoefficients") + dispersion <- callJMethod(jobj, "rDispersion") + null.deviance <- callJMethod(jobj, "rNullDeviance") + deviance <- callJMethod(jobj, "rDeviance") + df.null <- callJMethod(jobj, "rResidualDegreeOfFreedomNull") + df.residual <- callJMethod(jobj, "rResidualDegreeOfFreedom") + iter <- callJMethod(jobj, "rNumIterations") + family <- callJMethod(jobj, "rFamily") + aic <- callJMethod(jobj, "rAic") + if (family == "tweedie" && aic == 0) aic <- NA + deviance.resid <- if (is.loaded) { + NULL + } else { + dataFrame(callJMethod(jobj, "rDevianceResiduals")) + } + # If the underlying WeightedLeastSquares using "normal" solver, we can provide + # coefficients, standard error of coefficients, t value and p value. Otherwise, + # it will be fitted by local "l-bfgs", we can only provide coefficients. + if (length(features) == length(coefficients)) { + coefficients <- matrix(unlist(coefficients), ncol = 1) + colnames(coefficients) <- c("Estimate") + rownames(coefficients) <- unlist(features) + } else { + coefficients <- matrix(unlist(coefficients), ncol = 4) + colnames(coefficients) <- c("Estimate", "Std. Error", "t value", "Pr(>|t|)") + rownames(coefficients) <- unlist(features) + } + ans <- list(deviance.resid = deviance.resid, coefficients = coefficients, + dispersion = dispersion, null.deviance = null.deviance, + deviance = deviance, df.null = df.null, df.residual = df.residual, + aic = aic, iter = iter, family = family, is.loaded = is.loaded) + class(ans) <- "summary.GeneralizedLinearRegressionModel" + ans + }) + +# Prints the summary of GeneralizedLinearRegressionModel + +#' @rdname spark.glm +#' @param x summary object of fitted generalized linear model returned by \code{summary} function. +#' @export +#' @note print.summary.GeneralizedLinearRegressionModel since 2.0.0 +print.summary.GeneralizedLinearRegressionModel <- function(x, ...) { + if (x$is.loaded) { + cat("\nSaved-loaded model does not support output 'Deviance Residuals'.\n") + } else { + x$deviance.resid <- setNames(unlist(approxQuantile(x$deviance.resid, "devianceResiduals", + c(0.0, 0.25, 0.5, 0.75, 1.0), 0.01)), c("Min", "1Q", "Median", "3Q", "Max")) + x$deviance.resid <- zapsmall(x$deviance.resid, 5L) + cat("\nDeviance Residuals: \n") + cat("(Note: These are approximate quantiles with relative error <= 0.01)\n") + print.default(x$deviance.resid, digits = 5L, na.print = "", print.gap = 2L) + } + + cat("\nCoefficients:\n") + print.default(x$coefficients, digits = 5L, na.print = "", print.gap = 2L) + + cat("\n(Dispersion parameter for ", x$family, " family taken to be ", format(x$dispersion), + ")\n\n", apply(cbind(paste(format(c("Null", "Residual"), justify = "right"), "deviance:"), + format(unlist(x[c("null.deviance", "deviance")]), digits = 5L), + " on", format(unlist(x[c("df.null", "df.residual")])), " degrees of freedom\n"), + 1L, paste, collapse = " "), sep = "") + cat("AIC: ", format(x$aic, digits = 4L), "\n\n", + "Number of Fisher Scoring iterations: ", x$iter, "\n\n", sep = "") + invisible(x) + } + +# Makes predictions from a generalized linear model produced by glm() or spark.glm(), +# similarly to R's predict(). + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns a SparkDataFrame containing predicted labels in a column named +#' "prediction". +#' @rdname spark.glm +#' @export +#' @note predict(GeneralizedLinearRegressionModel) since 1.5.0 +setMethod("predict", signature(object = "GeneralizedLinearRegressionModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Saves the generalized linear model to the input path. + +#' @param path the directory where the model is saved. +#' @param overwrite overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' +#' @rdname spark.glm +#' @export +#' @note write.ml(GeneralizedLinearRegressionModel, character) since 2.0.0 +setMethod("write.ml", signature(object = "GeneralizedLinearRegressionModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +#' Isotonic Regression Model +#' +#' Fits an Isotonic Regression model against a SparkDataFrame, similarly to R's isoreg(). +#' Users can print, make predictions on the produced model and save the model to the input path. +#' +#' @param data SparkDataFrame for training. +#' @param formula A symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', '.', ':', '+', and '-'. +#' @param isotonic Whether the output sequence should be isotonic/increasing (TRUE) or +#' antitonic/decreasing (FALSE). +#' @param featureIndex The index of the feature if \code{featuresCol} is a vector column +#' (default: 0), no effect otherwise. +#' @param weightCol The weight column name. +#' @param ... additional arguments passed to the method. +#' @return \code{spark.isoreg} returns a fitted Isotonic Regression model. +#' @rdname spark.isoreg +#' @aliases spark.isoreg,SparkDataFrame,formula-method +#' @name spark.isoreg +#' @export +#' @examples +#' \dontrun{ +#' sparkR.session() +#' data <- list(list(7.0, 0.0), list(5.0, 1.0), list(3.0, 2.0), +#' list(5.0, 3.0), list(1.0, 4.0)) +#' df <- createDataFrame(data, c("label", "feature")) +#' model <- spark.isoreg(df, label ~ feature, isotonic = FALSE) +#' # return model boundaries and prediction as lists +#' result <- summary(model, df) +#' # prediction based on fitted model +#' predict_data <- list(list(-2.0), list(-1.0), list(0.5), +#' list(0.75), list(1.0), list(2.0), list(9.0)) +#' predict_df <- createDataFrame(predict_data, c("feature")) +#' # get prediction column +#' predict_result <- collect(select(predict(model, predict_df), "prediction")) +#' +#' # save fitted model to input path +#' path <- "path/to/model" +#' write.ml(model, path) +#' +#' # can also read back the saved model and print +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' } +#' @note spark.isoreg since 2.1.0 +setMethod("spark.isoreg", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, isotonic = TRUE, featureIndex = 0, weightCol = NULL) { + formula <- paste(deparse(formula), collapse = "") + + if (!is.null(weightCol) && weightCol == "") { + weightCol <- NULL + } else if (!is.null(weightCol)) { + weightCol <- as.character(weightCol) + } + + jobj <- callJStatic("org.apache.spark.ml.r.IsotonicRegressionWrapper", "fit", + data@sdf, formula, as.logical(isotonic), as.integer(featureIndex), + weightCol) + new("IsotonicRegressionModel", jobj = jobj) + }) + +# Get the summary of an IsotonicRegressionModel model + +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list includes model's \code{boundaries} (boundaries in increasing order) +#' and \code{predictions} (predictions associated with the boundaries at the same index). +#' @rdname spark.isoreg +#' @aliases summary,IsotonicRegressionModel-method +#' @export +#' @note summary(IsotonicRegressionModel) since 2.1.0 +setMethod("summary", signature(object = "IsotonicRegressionModel"), + function(object) { + jobj <- object@jobj + boundaries <- callJMethod(jobj, "boundaries") + predictions <- callJMethod(jobj, "predictions") + list(boundaries = boundaries, predictions = predictions) + }) + +# Predicted values based on an isotonicRegression model + +#' @param object a fitted IsotonicRegressionModel. +#' @param newData SparkDataFrame for testing. +#' @return \code{predict} returns a SparkDataFrame containing predicted values. +#' @rdname spark.isoreg +#' @aliases predict,IsotonicRegressionModel,SparkDataFrame-method +#' @export +#' @note predict(IsotonicRegressionModel) since 2.1.0 +setMethod("predict", signature(object = "IsotonicRegressionModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Save fitted IsotonicRegressionModel to the input path + +#' @param path The directory where the model is saved. +#' @param overwrite Overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' +#' @rdname spark.isoreg +#' @aliases write.ml,IsotonicRegressionModel,character-method +#' @export +#' @note write.ml(IsotonicRegression, character) since 2.1.0 +setMethod("write.ml", signature(object = "IsotonicRegressionModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +#' Accelerated Failure Time (AFT) Survival Regression Model +#' +#' \code{spark.survreg} fits an accelerated failure time (AFT) survival regression model on +#' a SparkDataFrame. Users can call \code{summary} to get a summary of the fitted AFT model, +#' \code{predict} to make predictions on new data, and \code{write.ml}/\code{read.ml} to +#' save/load fitted models. +#' +#' @param data a SparkDataFrame for training. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', ':', '+', and '-'. +#' Note that operator '.' is not supported currently. +#' @param aggregationDepth The depth for treeAggregate (greater than or equal to 2). If the dimensions of features +#' or the number of partitions are large, this param could be adjusted to a larger size. +#' This is an expert parameter. Default value should be good for most cases. +#' @param stringIndexerOrderType how to order categories of a string feature column. This is used to +#' decide the base level of a string feature as the last category after +#' ordering is dropped when encoding strings. Supported options are +#' "frequencyDesc", "frequencyAsc", "alphabetDesc", and "alphabetAsc". +#' The default value is "frequencyDesc". When the ordering is set to +#' "alphabetDesc", this drops the same category as R when encoding strings. +#' @param ... additional arguments passed to the method. +#' @return \code{spark.survreg} returns a fitted AFT survival regression model. +#' @rdname spark.survreg +#' @seealso survival: \url{https://cran.r-project.org/package=survival} +#' @export +#' @examples +#' \dontrun{ +#' df <- createDataFrame(ovarian) +#' model <- spark.survreg(df, Surv(futime, fustat) ~ ecog_ps + rx) +#' +#' # get a summary of the model +#' summary(model) +#' +#' # make predictions +#' predicted <- predict(model, df) +#' showDF(predicted) +#' +#' # save and load the model +#' path <- "path/to/model" +#' write.ml(model, path) +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' } +#' @note spark.survreg since 2.0.0 +setMethod("spark.survreg", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, aggregationDepth = 2, + stringIndexerOrderType = c("frequencyDesc", "frequencyAsc", + "alphabetDesc", "alphabetAsc")) { + stringIndexerOrderType <- match.arg(stringIndexerOrderType) + formula <- paste(deparse(formula), collapse = "") + jobj <- callJStatic("org.apache.spark.ml.r.AFTSurvivalRegressionWrapper", + "fit", formula, data@sdf, as.integer(aggregationDepth), + stringIndexerOrderType) + new("AFTSurvivalRegressionModel", jobj = jobj) + }) + +# Returns a summary of the AFT survival regression model produced by spark.survreg, +# similarly to R's summary(). + +#' @param object a fitted AFT survival regression model. +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list includes the model's \code{coefficients} (features, coefficients, +#' intercept and log(scale)). +#' @rdname spark.survreg +#' @export +#' @note summary(AFTSurvivalRegressionModel) since 2.0.0 +setMethod("summary", signature(object = "AFTSurvivalRegressionModel"), + function(object) { + jobj <- object@jobj + features <- callJMethod(jobj, "rFeatures") + coefficients <- callJMethod(jobj, "rCoefficients") + coefficients <- as.matrix(unlist(coefficients)) + colnames(coefficients) <- c("Value") + rownames(coefficients) <- unlist(features) + list(coefficients = coefficients) + }) + +# Makes predictions from an AFT survival regression model or a model produced by +# spark.survreg, similarly to R package survival's predict. + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns a SparkDataFrame containing predicted values +#' on the original scale of the data (mean predicted value at scale = 1.0). +#' @rdname spark.survreg +#' @export +#' @note predict(AFTSurvivalRegressionModel) since 2.0.0 +setMethod("predict", signature(object = "AFTSurvivalRegressionModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Saves the AFT survival regression model to the input path. + +#' @param path the directory where the model is saved. +#' @param overwrite overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' @rdname spark.survreg +#' @export +#' @note write.ml(AFTSurvivalRegressionModel, character) since 2.0.0 +#' @seealso \link{write.ml} +setMethod("write.ml", signature(object = "AFTSurvivalRegressionModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) diff --git a/R/pkg/R/mllib_stat.R b/R/pkg/R/mllib_stat.R new file mode 100644 index 000000000000..3e013f1d45e3 --- /dev/null +++ b/R/pkg/R/mllib_stat.R @@ -0,0 +1,127 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# mllib_stat.R: Provides methods for MLlib statistics algorithms integration + +#' S4 class that represents an KSTest +#' +#' @param jobj a Java object reference to the backing Scala KSTestWrapper +#' @export +#' @note KSTest since 2.1.0 +setClass("KSTest", representation(jobj = "jobj")) + +#' (One-Sample) Kolmogorov-Smirnov Test +#' +#' @description +#' \code{spark.kstest} Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a +#' continuous distribution. +#' +#' By comparing the largest difference between the empirical cumulative +#' distribution of the sample data and the theoretical distribution we can provide a test for the +#' the null hypothesis that the sample data comes from that theoretical distribution. +#' +#' Users can call \code{summary} to obtain a summary of the test, and \code{print.summary.KSTest} +#' to print out a summary result. +#' +#' @param data a SparkDataFrame of user data. +#' @param testCol column name where the test data is from. It should be a column of double type. +#' @param nullHypothesis name of the theoretical distribution tested against. Currently only +#' \code{"norm"} for normal distribution is supported. +#' @param distParams parameters(s) of the distribution. For \code{nullHypothesis = "norm"}, +#' we can provide as a vector the mean and standard deviation of +#' the distribution. If none is provided, then standard normal will be used. +#' If only one is provided, then the standard deviation will be set to be one. +#' @param ... additional argument(s) passed to the method. +#' @return \code{spark.kstest} returns a test result object. +#' @rdname spark.kstest +#' @aliases spark.kstest,SparkDataFrame-method +#' @name spark.kstest +#' @seealso \href{http://spark.apache.org/docs/latest/mllib-statistics.html#hypothesis-testing}{ +#' MLlib: Hypothesis Testing} +#' @export +#' @examples +#' \dontrun{ +#' data <- data.frame(test = c(0.1, 0.15, 0.2, 0.3, 0.25)) +#' df <- createDataFrame(data) +#' test <- spark.kstest(df, "test", "norm", c(0, 1)) +#' +#' # get a summary of the test result +#' testSummary <- summary(test) +#' testSummary +#' +#' # print out the summary in an organized way +#' print.summary.KSTest(testSummary) +#' } +#' @note spark.kstest since 2.1.0 +setMethod("spark.kstest", signature(data = "SparkDataFrame"), + function(data, testCol = "test", nullHypothesis = c("norm"), distParams = c(0, 1)) { + tryCatch(match.arg(nullHypothesis), + error = function(e) { + msg <- paste("Distribution", nullHypothesis, "is not supported.") + stop(msg) + }) + if (nullHypothesis == "norm") { + distParams <- as.numeric(distParams) + mu <- ifelse(length(distParams) < 1, 0, distParams[1]) + sigma <- ifelse(length(distParams) < 2, 1, distParams[2]) + jobj <- callJStatic("org.apache.spark.ml.r.KSTestWrapper", + "test", data@sdf, testCol, nullHypothesis, + as.array(c(mu, sigma))) + new("KSTest", jobj = jobj) + } +}) + +# Get the summary of Kolmogorov-Smirnov (KS) Test. + +#' @param object test result object of KSTest by \code{spark.kstest}. +#' @return \code{summary} returns summary information of KSTest object, which is a list. +#' The list includes the \code{p.value} (p-value), \code{statistic} (test statistic +#' computed for the test), \code{nullHypothesis} (the null hypothesis with its +#' parameters tested against) and \code{degreesOfFreedom} (degrees of freedom of the test). +#' @rdname spark.kstest +#' @aliases summary,KSTest-method +#' @export +#' @note summary(KSTest) since 2.1.0 +setMethod("summary", signature(object = "KSTest"), + function(object) { + jobj <- object@jobj + pValue <- callJMethod(jobj, "pValue") + statistic <- callJMethod(jobj, "statistic") + nullHypothesis <- callJMethod(jobj, "nullHypothesis") + distName <- callJMethod(jobj, "distName") + distParams <- unlist(callJMethod(jobj, "distParams")) + degreesOfFreedom <- callJMethod(jobj, "degreesOfFreedom") + + ans <- list(p.value = pValue, statistic = statistic, nullHypothesis = nullHypothesis, + nullHypothesis.name = distName, nullHypothesis.parameters = distParams, + degreesOfFreedom = degreesOfFreedom, jobj = jobj) + class(ans) <- "summary.KSTest" + ans + }) + +# Prints the summary of KSTest + +#' @rdname spark.kstest +#' @param x summary object of KSTest returned by \code{summary}. +#' @export +#' @note print.summary.KSTest since 2.1.0 +print.summary.KSTest <- function(x, ...) { + jobj <- x$jobj + summaryStr <- callJMethod(jobj, "summary") + cat(summaryStr, "\n") + invisible(x) +} diff --git a/R/pkg/R/mllib_tree.R b/R/pkg/R/mllib_tree.R new file mode 100644 index 000000000000..33c4653f4c18 --- /dev/null +++ b/R/pkg/R/mllib_tree.R @@ -0,0 +1,765 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# mllib_tree.R: Provides methods for MLlib tree-based algorithms integration + +#' S4 class that represents a GBTRegressionModel +#' +#' @param jobj a Java object reference to the backing Scala GBTRegressionModel +#' @export +#' @note GBTRegressionModel since 2.1.0 +setClass("GBTRegressionModel", representation(jobj = "jobj")) + +#' S4 class that represents a GBTClassificationModel +#' +#' @param jobj a Java object reference to the backing Scala GBTClassificationModel +#' @export +#' @note GBTClassificationModel since 2.1.0 +setClass("GBTClassificationModel", representation(jobj = "jobj")) + +#' S4 class that represents a RandomForestRegressionModel +#' +#' @param jobj a Java object reference to the backing Scala RandomForestRegressionModel +#' @export +#' @note RandomForestRegressionModel since 2.1.0 +setClass("RandomForestRegressionModel", representation(jobj = "jobj")) + +#' S4 class that represents a RandomForestClassificationModel +#' +#' @param jobj a Java object reference to the backing Scala RandomForestClassificationModel +#' @export +#' @note RandomForestClassificationModel since 2.1.0 +setClass("RandomForestClassificationModel", representation(jobj = "jobj")) + +#' S4 class that represents a DecisionTreeRegressionModel +#' +#' @param jobj a Java object reference to the backing Scala DecisionTreeRegressionModel +#' @export +#' @note DecisionTreeRegressionModel since 2.3.0 +setClass("DecisionTreeRegressionModel", representation(jobj = "jobj")) + +#' S4 class that represents a DecisionTreeClassificationModel +#' +#' @param jobj a Java object reference to the backing Scala DecisionTreeClassificationModel +#' @export +#' @note DecisionTreeClassificationModel since 2.3.0 +setClass("DecisionTreeClassificationModel", representation(jobj = "jobj")) + +# Create the summary of a tree ensemble model (eg. Random Forest, GBT) +summary.treeEnsemble <- function(model) { + jobj <- model@jobj + formula <- callJMethod(jobj, "formula") + numFeatures <- callJMethod(jobj, "numFeatures") + features <- callJMethod(jobj, "features") + featureImportances <- callJMethod(callJMethod(jobj, "featureImportances"), "toString") + maxDepth <- callJMethod(jobj, "maxDepth") + numTrees <- callJMethod(jobj, "numTrees") + treeWeights <- callJMethod(jobj, "treeWeights") + list(formula = formula, + numFeatures = numFeatures, + features = features, + featureImportances = featureImportances, + maxDepth = maxDepth, + numTrees = numTrees, + treeWeights = treeWeights, + jobj = jobj) +} + +# Prints the summary of tree ensemble models (eg. Random Forest, GBT) +print.summary.treeEnsemble <- function(x) { + jobj <- x$jobj + cat("Formula: ", x$formula) + cat("\nNumber of features: ", x$numFeatures) + cat("\nFeatures: ", unlist(x$features)) + cat("\nFeature importances: ", x$featureImportances) + cat("\nMax Depth: ", x$maxDepth) + cat("\nNumber of trees: ", x$numTrees) + cat("\nTree weights: ", unlist(x$treeWeights)) + + summaryStr <- callJMethod(jobj, "summary") + cat("\n", summaryStr, "\n") + invisible(x) +} + +# Create the summary of a decision tree model +summary.decisionTree <- function(model) { + jobj <- model@jobj + formula <- callJMethod(jobj, "formula") + numFeatures <- callJMethod(jobj, "numFeatures") + features <- callJMethod(jobj, "features") + featureImportances <- callJMethod(callJMethod(jobj, "featureImportances"), "toString") + maxDepth <- callJMethod(jobj, "maxDepth") + list(formula = formula, + numFeatures = numFeatures, + features = features, + featureImportances = featureImportances, + maxDepth = maxDepth, + jobj = jobj) +} + +# Prints the summary of decision tree models +print.summary.decisionTree <- function(x) { + jobj <- x$jobj + cat("Formula: ", x$formula) + cat("\nNumber of features: ", x$numFeatures) + cat("\nFeatures: ", unlist(x$features)) + cat("\nFeature importances: ", x$featureImportances) + cat("\nMax Depth: ", x$maxDepth) + + summaryStr <- callJMethod(jobj, "summary") + cat("\n", summaryStr, "\n") + invisible(x) +} + +#' Gradient Boosted Tree Model for Regression and Classification +#' +#' \code{spark.gbt} fits a Gradient Boosted Tree Regression model or Classification model on a +#' SparkDataFrame. Users can call \code{summary} to get a summary of the fitted +#' Gradient Boosted Tree model, \code{predict} to make predictions on new data, and +#' \code{write.ml}/\code{read.ml} to save/load fitted models. +#' For more details, see +#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html#gradient-boosted-tree-regression}{ +#' GBT Regression} and +#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html#gradient-boosted-tree-classifier}{ +#' GBT Classification} +#' +#' @param data a SparkDataFrame for training. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', ':', '+', and '-'. +#' @param type type of model, one of "regression" or "classification", to fit +#' @param maxDepth Maximum depth of the tree (>= 0). +#' @param maxBins Maximum number of bins used for discretizing continuous features and for choosing +#' how to split on features at each node. More bins give higher granularity. Must be +#' >= 2 and >= number of categories in any categorical feature. +#' @param maxIter Param for maximum number of iterations (>= 0). +#' @param stepSize Param for Step size to be used for each iteration of optimization. +#' @param lossType Loss function which GBT tries to minimize. +#' For classification, must be "logistic". For regression, must be one of +#' "squared" (L2) and "absolute" (L1), default is "squared". +#' @param seed integer seed for random number generation. +#' @param subsamplingRate Fraction of the training data used for learning each decision tree, in +#' range (0, 1]. +#' @param minInstancesPerNode Minimum number of instances each child must have after split. If a +#' split causes the left or right child to have fewer than +#' minInstancesPerNode, the split will be discarded as invalid. Should be +#' >= 1. +#' @param minInfoGain Minimum information gain for a split to be considered at a tree node. +#' @param checkpointInterval Param for set checkpoint interval (>= 1) or disable checkpoint (-1). +#' @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. +#' @param cacheNodeIds If FALSE, the algorithm will pass trees to executors to match instances with +#' nodes. If TRUE, the algorithm will cache node IDs for each instance. Caching +#' can speed up training of deeper trees. Users can set how often should the +#' cache be checkpointed or disable it by setting checkpointInterval. +#' @param handleInvalid How to handle invalid data (unseen labels or NULL values) in features and label +#' column of string type in classification model. +#' Supported options: "skip" (filter out rows with invalid data), +#' "error" (throw an error), "keep" (put invalid data in a special additional +#' bucket, at index numLabels). Default is "error". +#' @param ... additional arguments passed to the method. +#' @aliases spark.gbt,SparkDataFrame,formula-method +#' @return \code{spark.gbt} returns a fitted Gradient Boosted Tree model. +#' @rdname spark.gbt +#' @name spark.gbt +#' @export +#' @examples +#' \dontrun{ +#' # fit a Gradient Boosted Tree Regression Model +#' df <- createDataFrame(longley) +#' model <- spark.gbt(df, Employed ~ ., type = "regression", maxDepth = 5, maxBins = 16) +#' +#' # get the summary of the model +#' summary(model) +#' +#' # make predictions +#' predictions <- predict(model, df) +#' +#' # save and load the model +#' path <- "path/to/model" +#' write.ml(model, path) +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' +#' # fit a Gradient Boosted Tree Classification Model +#' # label must be binary - Only binary classification is supported for GBT. +#' t <- as.data.frame(Titanic) +#' df <- createDataFrame(t) +#' model <- spark.gbt(df, Survived ~ Age + Freq, "classification") +#' +#' # numeric label is also supported +#' t2 <- as.data.frame(Titanic) +#' t2$NumericGender <- ifelse(t2$Sex == "Male", 0, 1) +#' df <- createDataFrame(t2) +#' model <- spark.gbt(df, NumericGender ~ ., type = "classification") +#' } +#' @note spark.gbt since 2.1.0 +setMethod("spark.gbt", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, type = c("regression", "classification"), + maxDepth = 5, maxBins = 32, maxIter = 20, stepSize = 0.1, lossType = NULL, + seed = NULL, subsamplingRate = 1.0, minInstancesPerNode = 1, minInfoGain = 0.0, + checkpointInterval = 10, maxMemoryInMB = 256, cacheNodeIds = FALSE, + handleInvalid = c("error", "keep", "skip")) { + type <- match.arg(type) + formula <- paste(deparse(formula), collapse = "") + if (!is.null(seed)) { + seed <- as.character(as.integer(seed)) + } + switch(type, + regression = { + if (is.null(lossType)) lossType <- "squared" + lossType <- match.arg(lossType, c("squared", "absolute")) + jobj <- callJStatic("org.apache.spark.ml.r.GBTRegressorWrapper", + "fit", data@sdf, formula, as.integer(maxDepth), + as.integer(maxBins), as.integer(maxIter), + as.numeric(stepSize), as.integer(minInstancesPerNode), + as.numeric(minInfoGain), as.integer(checkpointInterval), + lossType, seed, as.numeric(subsamplingRate), + as.integer(maxMemoryInMB), as.logical(cacheNodeIds)) + new("GBTRegressionModel", jobj = jobj) + }, + classification = { + handleInvalid <- match.arg(handleInvalid) + if (is.null(lossType)) lossType <- "logistic" + lossType <- match.arg(lossType, "logistic") + jobj <- callJStatic("org.apache.spark.ml.r.GBTClassifierWrapper", + "fit", data@sdf, formula, as.integer(maxDepth), + as.integer(maxBins), as.integer(maxIter), + as.numeric(stepSize), as.integer(minInstancesPerNode), + as.numeric(minInfoGain), as.integer(checkpointInterval), + lossType, seed, as.numeric(subsamplingRate), + as.integer(maxMemoryInMB), as.logical(cacheNodeIds), + handleInvalid) + new("GBTClassificationModel", jobj = jobj) + } + ) + }) + +# Get the summary of a Gradient Boosted Tree Regression Model + +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list of components includes \code{formula} (formula), +#' \code{numFeatures} (number of features), \code{features} (list of features), +#' \code{featureImportances} (feature importances), \code{maxDepth} (max depth of trees), +#' \code{numTrees} (number of trees), and \code{treeWeights} (tree weights). +#' @rdname spark.gbt +#' @aliases summary,GBTRegressionModel-method +#' @export +#' @note summary(GBTRegressionModel) since 2.1.0 +setMethod("summary", signature(object = "GBTRegressionModel"), + function(object) { + ans <- summary.treeEnsemble(object) + class(ans) <- "summary.GBTRegressionModel" + ans + }) + +# Prints the summary of Gradient Boosted Tree Regression Model + +#' @param x summary object of Gradient Boosted Tree regression model or classification model +#' returned by \code{summary}. +#' @rdname spark.gbt +#' @export +#' @note print.summary.GBTRegressionModel since 2.1.0 +print.summary.GBTRegressionModel <- function(x, ...) { + print.summary.treeEnsemble(x) +} + +# Get the summary of a Gradient Boosted Tree Classification Model + +#' @rdname spark.gbt +#' @aliases summary,GBTClassificationModel-method +#' @export +#' @note summary(GBTClassificationModel) since 2.1.0 +setMethod("summary", signature(object = "GBTClassificationModel"), + function(object) { + ans <- summary.treeEnsemble(object) + class(ans) <- "summary.GBTClassificationModel" + ans + }) + +# Prints the summary of Gradient Boosted Tree Classification Model + +#' @rdname spark.gbt +#' @export +#' @note print.summary.GBTClassificationModel since 2.1.0 +print.summary.GBTClassificationModel <- function(x, ...) { + print.summary.treeEnsemble(x) +} + +# Makes predictions from a Gradient Boosted Tree Regression model or Classification model + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns a SparkDataFrame containing predicted labeled in a column named +#' "prediction". +#' @rdname spark.gbt +#' @aliases predict,GBTRegressionModel-method +#' @export +#' @note predict(GBTRegressionModel) since 2.1.0 +setMethod("predict", signature(object = "GBTRegressionModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +#' @rdname spark.gbt +#' @aliases predict,GBTClassificationModel-method +#' @export +#' @note predict(GBTClassificationModel) since 2.1.0 +setMethod("predict", signature(object = "GBTClassificationModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Save the Gradient Boosted Tree Regression or Classification model to the input path. + +#' @param object A fitted Gradient Boosted Tree regression model or classification model. +#' @param path The directory where the model is saved. +#' @param overwrite Overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' @aliases write.ml,GBTRegressionModel,character-method +#' @rdname spark.gbt +#' @export +#' @note write.ml(GBTRegressionModel, character) since 2.1.0 +setMethod("write.ml", signature(object = "GBTRegressionModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +#' @aliases write.ml,GBTClassificationModel,character-method +#' @rdname spark.gbt +#' @export +#' @note write.ml(GBTClassificationModel, character) since 2.1.0 +setMethod("write.ml", signature(object = "GBTClassificationModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +#' Random Forest Model for Regression and Classification +#' +#' \code{spark.randomForest} fits a Random Forest Regression model or Classification model on +#' a SparkDataFrame. Users can call \code{summary} to get a summary of the fitted Random Forest +#' model, \code{predict} to make predictions on new data, and \code{write.ml}/\code{read.ml} to +#' save/load fitted models. +#' For more details, see +#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html#random-forest-regression}{ +#' Random Forest Regression} and +#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html#random-forest-classifier}{ +#' Random Forest Classification} +#' +#' @param data a SparkDataFrame for training. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', ':', '+', and '-'. +#' @param type type of model, one of "regression" or "classification", to fit +#' @param maxDepth Maximum depth of the tree (>= 0). +#' @param maxBins Maximum number of bins used for discretizing continuous features and for choosing +#' how to split on features at each node. More bins give higher granularity. Must be +#' >= 2 and >= number of categories in any categorical feature. +#' @param numTrees Number of trees to train (>= 1). +#' @param impurity Criterion used for information gain calculation. +#' For regression, must be "variance". For classification, must be one of +#' "entropy" and "gini", default is "gini". +#' @param featureSubsetStrategy The number of features to consider for splits at each tree node. +#' Supported options: "auto", "all", "onethird", "sqrt", "log2", (0.0-1.0], [1-n]. +#' @param seed integer seed for random number generation. +#' @param subsamplingRate Fraction of the training data used for learning each decision tree, in +#' range (0, 1]. +#' @param minInstancesPerNode Minimum number of instances each child must have after split. +#' @param minInfoGain Minimum information gain for a split to be considered at a tree node. +#' @param checkpointInterval Param for set checkpoint interval (>= 1) or disable checkpoint (-1). +#' @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. +#' @param cacheNodeIds If FALSE, the algorithm will pass trees to executors to match instances with +#' nodes. If TRUE, the algorithm will cache node IDs for each instance. Caching +#' can speed up training of deeper trees. Users can set how often should the +#' cache be checkpointed or disable it by setting checkpointInterval. +#' @param handleInvalid How to handle invalid data (unseen labels or NULL values) in features and label +#' column of string type in classification model. +#' Supported options: "skip" (filter out rows with invalid data), +#' "error" (throw an error), "keep" (put invalid data in a special additional +#' bucket, at index numLabels). Default is "error". +#' @param ... additional arguments passed to the method. +#' @aliases spark.randomForest,SparkDataFrame,formula-method +#' @return \code{spark.randomForest} returns a fitted Random Forest model. +#' @rdname spark.randomForest +#' @name spark.randomForest +#' @export +#' @examples +#' \dontrun{ +#' # fit a Random Forest Regression Model +#' df <- createDataFrame(longley) +#' model <- spark.randomForest(df, Employed ~ ., type = "regression", maxDepth = 5, maxBins = 16) +#' +#' # get the summary of the model +#' summary(model) +#' +#' # make predictions +#' predictions <- predict(model, df) +#' +#' # save and load the model +#' path <- "path/to/model" +#' write.ml(model, path) +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' +#' # fit a Random Forest Classification Model +#' t <- as.data.frame(Titanic) +#' df <- createDataFrame(t) +#' model <- spark.randomForest(df, Survived ~ Freq + Age, "classification") +#' } +#' @note spark.randomForest since 2.1.0 +setMethod("spark.randomForest", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, type = c("regression", "classification"), + maxDepth = 5, maxBins = 32, numTrees = 20, impurity = NULL, + featureSubsetStrategy = "auto", seed = NULL, subsamplingRate = 1.0, + minInstancesPerNode = 1, minInfoGain = 0.0, checkpointInterval = 10, + maxMemoryInMB = 256, cacheNodeIds = FALSE, + handleInvalid = c("error", "keep", "skip")) { + type <- match.arg(type) + formula <- paste(deparse(formula), collapse = "") + if (!is.null(seed)) { + seed <- as.character(as.integer(seed)) + } + switch(type, + regression = { + if (is.null(impurity)) impurity <- "variance" + impurity <- match.arg(impurity, "variance") + jobj <- callJStatic("org.apache.spark.ml.r.RandomForestRegressorWrapper", + "fit", data@sdf, formula, as.integer(maxDepth), + as.integer(maxBins), as.integer(numTrees), + impurity, as.integer(minInstancesPerNode), + as.numeric(minInfoGain), as.integer(checkpointInterval), + as.character(featureSubsetStrategy), seed, + as.numeric(subsamplingRate), + as.integer(maxMemoryInMB), as.logical(cacheNodeIds)) + new("RandomForestRegressionModel", jobj = jobj) + }, + classification = { + handleInvalid <- match.arg(handleInvalid) + if (is.null(impurity)) impurity <- "gini" + impurity <- match.arg(impurity, c("gini", "entropy")) + jobj <- callJStatic("org.apache.spark.ml.r.RandomForestClassifierWrapper", + "fit", data@sdf, formula, as.integer(maxDepth), + as.integer(maxBins), as.integer(numTrees), + impurity, as.integer(minInstancesPerNode), + as.numeric(minInfoGain), as.integer(checkpointInterval), + as.character(featureSubsetStrategy), seed, + as.numeric(subsamplingRate), + as.integer(maxMemoryInMB), as.logical(cacheNodeIds), + handleInvalid) + new("RandomForestClassificationModel", jobj = jobj) + } + ) + }) + +# Get the summary of a Random Forest Regression Model + +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list of components includes \code{formula} (formula), +#' \code{numFeatures} (number of features), \code{features} (list of features), +#' \code{featureImportances} (feature importances), \code{maxDepth} (max depth of trees), +#' \code{numTrees} (number of trees), and \code{treeWeights} (tree weights). +#' @rdname spark.randomForest +#' @aliases summary,RandomForestRegressionModel-method +#' @export +#' @note summary(RandomForestRegressionModel) since 2.1.0 +setMethod("summary", signature(object = "RandomForestRegressionModel"), + function(object) { + ans <- summary.treeEnsemble(object) + class(ans) <- "summary.RandomForestRegressionModel" + ans + }) + +# Prints the summary of Random Forest Regression Model + +#' @param x summary object of Random Forest regression model or classification model +#' returned by \code{summary}. +#' @rdname spark.randomForest +#' @export +#' @note print.summary.RandomForestRegressionModel since 2.1.0 +print.summary.RandomForestRegressionModel <- function(x, ...) { + print.summary.treeEnsemble(x) +} + +# Get the summary of a Random Forest Classification Model + +#' @rdname spark.randomForest +#' @aliases summary,RandomForestClassificationModel-method +#' @export +#' @note summary(RandomForestClassificationModel) since 2.1.0 +setMethod("summary", signature(object = "RandomForestClassificationModel"), + function(object) { + ans <- summary.treeEnsemble(object) + class(ans) <- "summary.RandomForestClassificationModel" + ans + }) + +# Prints the summary of Random Forest Classification Model + +#' @rdname spark.randomForest +#' @export +#' @note print.summary.RandomForestClassificationModel since 2.1.0 +print.summary.RandomForestClassificationModel <- function(x, ...) { + print.summary.treeEnsemble(x) +} + +# Makes predictions from a Random Forest Regression model or Classification model + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns a SparkDataFrame containing predicted labeled in a column named +#' "prediction". +#' @rdname spark.randomForest +#' @aliases predict,RandomForestRegressionModel-method +#' @export +#' @note predict(RandomForestRegressionModel) since 2.1.0 +setMethod("predict", signature(object = "RandomForestRegressionModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +#' @rdname spark.randomForest +#' @aliases predict,RandomForestClassificationModel-method +#' @export +#' @note predict(RandomForestClassificationModel) since 2.1.0 +setMethod("predict", signature(object = "RandomForestClassificationModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Save the Random Forest Regression or Classification model to the input path. + +#' @param object A fitted Random Forest regression model or classification model. +#' @param path The directory where the model is saved. +#' @param overwrite Overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' +#' @aliases write.ml,RandomForestRegressionModel,character-method +#' @rdname spark.randomForest +#' @export +#' @note write.ml(RandomForestRegressionModel, character) since 2.1.0 +setMethod("write.ml", signature(object = "RandomForestRegressionModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +#' @aliases write.ml,RandomForestClassificationModel,character-method +#' @rdname spark.randomForest +#' @export +#' @note write.ml(RandomForestClassificationModel, character) since 2.1.0 +setMethod("write.ml", signature(object = "RandomForestClassificationModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +#' Decision Tree Model for Regression and Classification +#' +#' \code{spark.decisionTree} fits a Decision Tree Regression model or Classification model on +#' a SparkDataFrame. Users can call \code{summary} to get a summary of the fitted Decision Tree +#' model, \code{predict} to make predictions on new data, and \code{write.ml}/\code{read.ml} to +#' save/load fitted models. +#' For more details, see +#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html#decision-tree-regression}{ +#' Decision Tree Regression} and +#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html#decision-tree-classifier}{ +#' Decision Tree Classification} +#' +#' @param data a SparkDataFrame for training. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', ':', '+', and '-'. +#' @param type type of model, one of "regression" or "classification", to fit +#' @param maxDepth Maximum depth of the tree (>= 0). +#' @param maxBins Maximum number of bins used for discretizing continuous features and for choosing +#' how to split on features at each node. More bins give higher granularity. Must be +#' >= 2 and >= number of categories in any categorical feature. +#' @param impurity Criterion used for information gain calculation. +#' For regression, must be "variance". For classification, must be one of +#' "entropy" and "gini", default is "gini". +#' @param seed integer seed for random number generation. +#' @param minInstancesPerNode Minimum number of instances each child must have after split. +#' @param minInfoGain Minimum information gain for a split to be considered at a tree node. +#' @param checkpointInterval Param for set checkpoint interval (>= 1) or disable checkpoint (-1). +#' @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. +#' @param cacheNodeIds If FALSE, the algorithm will pass trees to executors to match instances with +#' nodes. If TRUE, the algorithm will cache node IDs for each instance. Caching +#' can speed up training of deeper trees. Users can set how often should the +#' cache be checkpointed or disable it by setting checkpointInterval. +#' @param handleInvalid How to handle invalid data (unseen labels or NULL values) in features and label +#' column of string type in classification model. +#' Supported options: "skip" (filter out rows with invalid data), +#' "error" (throw an error), "keep" (put invalid data in a special additional +#' bucket, at index numLabels). Default is "error". +#' @param ... additional arguments passed to the method. +#' @aliases spark.decisionTree,SparkDataFrame,formula-method +#' @return \code{spark.decisionTree} returns a fitted Decision Tree model. +#' @rdname spark.decisionTree +#' @name spark.decisionTree +#' @export +#' @examples +#' \dontrun{ +#' # fit a Decision Tree Regression Model +#' df <- createDataFrame(longley) +#' model <- spark.decisionTree(df, Employed ~ ., type = "regression", maxDepth = 5, maxBins = 16) +#' +#' # get the summary of the model +#' summary(model) +#' +#' # make predictions +#' predictions <- predict(model, df) +#' +#' # save and load the model +#' path <- "path/to/model" +#' write.ml(model, path) +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' +#' # fit a Decision Tree Classification Model +#' t <- as.data.frame(Titanic) +#' df <- createDataFrame(t) +#' model <- spark.decisionTree(df, Survived ~ Freq + Age, "classification") +#' } +#' @note spark.decisionTree since 2.3.0 +setMethod("spark.decisionTree", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, type = c("regression", "classification"), + maxDepth = 5, maxBins = 32, impurity = NULL, seed = NULL, + minInstancesPerNode = 1, minInfoGain = 0.0, checkpointInterval = 10, + maxMemoryInMB = 256, cacheNodeIds = FALSE, + handleInvalid = c("error", "keep", "skip")) { + type <- match.arg(type) + formula <- paste(deparse(formula), collapse = "") + if (!is.null(seed)) { + seed <- as.character(as.integer(seed)) + } + switch(type, + regression = { + if (is.null(impurity)) impurity <- "variance" + impurity <- match.arg(impurity, "variance") + jobj <- callJStatic("org.apache.spark.ml.r.DecisionTreeRegressorWrapper", + "fit", data@sdf, formula, as.integer(maxDepth), + as.integer(maxBins), impurity, + as.integer(minInstancesPerNode), as.numeric(minInfoGain), + as.integer(checkpointInterval), seed, + as.integer(maxMemoryInMB), as.logical(cacheNodeIds)) + new("DecisionTreeRegressionModel", jobj = jobj) + }, + classification = { + handleInvalid <- match.arg(handleInvalid) + if (is.null(impurity)) impurity <- "gini" + impurity <- match.arg(impurity, c("gini", "entropy")) + jobj <- callJStatic("org.apache.spark.ml.r.DecisionTreeClassifierWrapper", + "fit", data@sdf, formula, as.integer(maxDepth), + as.integer(maxBins), impurity, + as.integer(minInstancesPerNode), as.numeric(minInfoGain), + as.integer(checkpointInterval), seed, + as.integer(maxMemoryInMB), as.logical(cacheNodeIds), + handleInvalid) + new("DecisionTreeClassificationModel", jobj = jobj) + } + ) + }) + +# Get the summary of a Decision Tree Regression Model + +#' @return \code{summary} returns summary information of the fitted model, which is a list. +#' The list of components includes \code{formula} (formula), +#' \code{numFeatures} (number of features), \code{features} (list of features), +#' \code{featureImportances} (feature importances), and \code{maxDepth} (max depth of trees). +#' @rdname spark.decisionTree +#' @aliases summary,DecisionTreeRegressionModel-method +#' @export +#' @note summary(DecisionTreeRegressionModel) since 2.3.0 +setMethod("summary", signature(object = "DecisionTreeRegressionModel"), + function(object) { + ans <- summary.decisionTree(object) + class(ans) <- "summary.DecisionTreeRegressionModel" + ans + }) + +# Prints the summary of Decision Tree Regression Model + +#' @param x summary object of Decision Tree regression model or classification model +#' returned by \code{summary}. +#' @rdname spark.decisionTree +#' @export +#' @note print.summary.DecisionTreeRegressionModel since 2.3.0 +print.summary.DecisionTreeRegressionModel <- function(x, ...) { + print.summary.decisionTree(x) +} + +# Get the summary of a Decision Tree Classification Model + +#' @rdname spark.decisionTree +#' @aliases summary,DecisionTreeClassificationModel-method +#' @export +#' @note summary(DecisionTreeClassificationModel) since 2.3.0 +setMethod("summary", signature(object = "DecisionTreeClassificationModel"), + function(object) { + ans <- summary.decisionTree(object) + class(ans) <- "summary.DecisionTreeClassificationModel" + ans + }) + +# Prints the summary of Decision Tree Classification Model + +#' @rdname spark.decisionTree +#' @export +#' @note print.summary.DecisionTreeClassificationModel since 2.3.0 +print.summary.DecisionTreeClassificationModel <- function(x, ...) { + print.summary.decisionTree(x) +} + +# Makes predictions from a Decision Tree Regression model or Classification model + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns a SparkDataFrame containing predicted labeled in a column named +#' "prediction". +#' @rdname spark.decisionTree +#' @aliases predict,DecisionTreeRegressionModel-method +#' @export +#' @note predict(DecisionTreeRegressionModel) since 2.3.0 +setMethod("predict", signature(object = "DecisionTreeRegressionModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +#' @rdname spark.decisionTree +#' @aliases predict,DecisionTreeClassificationModel-method +#' @export +#' @note predict(DecisionTreeClassificationModel) since 2.3.0 +setMethod("predict", signature(object = "DecisionTreeClassificationModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Save the Decision Tree Regression or Classification model to the input path. + +#' @param object A fitted Decision Tree regression model or classification model. +#' @param path The directory where the model is saved. +#' @param overwrite Overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' +#' @aliases write.ml,DecisionTreeRegressionModel,character-method +#' @rdname spark.decisionTree +#' @export +#' @note write.ml(DecisionTreeRegressionModel, character) since 2.3.0 +setMethod("write.ml", signature(object = "DecisionTreeRegressionModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +#' @aliases write.ml,DecisionTreeClassificationModel,character-method +#' @rdname spark.decisionTree +#' @export +#' @note write.ml(DecisionTreeClassificationModel, character) since 2.3.0 +setMethod("write.ml", signature(object = "DecisionTreeClassificationModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) diff --git a/R/pkg/R/mllib_utils.R b/R/pkg/R/mllib_utils.R new file mode 100644 index 000000000000..a53c92c2c481 --- /dev/null +++ b/R/pkg/R/mllib_utils.R @@ -0,0 +1,132 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# mllib_utils.R: Utilities for MLlib integration + +# Integration with R's standard functions. +# Most of MLlib's argorithms are provided in two flavours: +# - a specialization of the default R methods (glm). These methods try to respect +# the inputs and the outputs of R's method to the largest extent, but some small differences +# may exist. +# - a set of methods that reflect the arguments of the other languages supported by Spark. These +# methods are prefixed with the `spark.` prefix: spark.glm, spark.kmeans, etc. + +#' Saves the MLlib model to the input path +#' +#' Saves the MLlib model to the input path. For more information, see the specific +#' MLlib model below. +#' @rdname write.ml +#' @name write.ml +#' @export +#' @seealso \link{spark.als}, \link{spark.bisectingKmeans}, \link{spark.decisionTree}, +#' @seealso \link{spark.gaussianMixture}, \link{spark.gbt}, +#' @seealso \link{spark.glm}, \link{glm}, \link{spark.isoreg}, +#' @seealso \link{spark.kmeans}, +#' @seealso \link{spark.lda}, \link{spark.logit}, +#' @seealso \link{spark.mlp}, \link{spark.naiveBayes}, +#' @seealso \link{spark.randomForest}, \link{spark.survreg}, \link{spark.svmLinear}, +#' @seealso \link{read.ml} +NULL + +#' Makes predictions from a MLlib model +#' +#' Makes predictions from a MLlib model. For more information, see the specific +#' MLlib model below. +#' @rdname predict +#' @name predict +#' @export +#' @seealso \link{spark.als}, \link{spark.bisectingKmeans}, \link{spark.decisionTree}, +#' @seealso \link{spark.gaussianMixture}, \link{spark.gbt}, +#' @seealso \link{spark.glm}, \link{glm}, \link{spark.isoreg}, +#' @seealso \link{spark.kmeans}, +#' @seealso \link{spark.logit}, \link{spark.mlp}, \link{spark.naiveBayes}, +#' @seealso \link{spark.randomForest}, \link{spark.survreg}, \link{spark.svmLinear} +NULL + +write_internal <- function(object, path, overwrite = FALSE) { + writer <- callJMethod(object@jobj, "write") + if (overwrite) { + writer <- callJMethod(writer, "overwrite") + } + invisible(callJMethod(writer, "save", path)) +} + +predict_internal <- function(object, newData) { + dataFrame(callJMethod(object@jobj, "transform", newData@sdf)) +} + +#' Load a fitted MLlib model from the input path. +#' +#' @param path path of the model to read. +#' @return A fitted MLlib model. +#' @rdname read.ml +#' @name read.ml +#' @export +#' @seealso \link{write.ml} +#' @examples +#' \dontrun{ +#' path <- "path/to/model" +#' model <- read.ml(path) +#' } +#' @note read.ml since 2.0.0 +read.ml <- function(path) { + path <- suppressWarnings(normalizePath(path)) + sparkSession <- getSparkSession() + callJStatic("org.apache.spark.ml.r.RWrappers", "session", sparkSession) + jobj <- callJStatic("org.apache.spark.ml.r.RWrappers", "load", path) + if (isInstanceOf(jobj, "org.apache.spark.ml.r.NaiveBayesWrapper")) { + new("NaiveBayesModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.AFTSurvivalRegressionWrapper")) { + new("AFTSurvivalRegressionModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.GeneralizedLinearRegressionWrapper")) { + new("GeneralizedLinearRegressionModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.KMeansWrapper")) { + new("KMeansModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.LDAWrapper")) { + new("LDAModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.MultilayerPerceptronClassifierWrapper")) { + new("MultilayerPerceptronClassificationModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.IsotonicRegressionWrapper")) { + new("IsotonicRegressionModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.GaussianMixtureWrapper")) { + new("GaussianMixtureModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.ALSWrapper")) { + new("ALSModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.LogisticRegressionWrapper")) { + new("LogisticRegressionModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.RandomForestRegressorWrapper")) { + new("RandomForestRegressionModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.RandomForestClassifierWrapper")) { + new("RandomForestClassificationModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.DecisionTreeRegressorWrapper")) { + new("DecisionTreeRegressionModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.DecisionTreeClassifierWrapper")) { + new("DecisionTreeClassificationModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.GBTRegressorWrapper")) { + new("GBTRegressionModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.GBTClassifierWrapper")) { + new("GBTClassificationModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.BisectingKMeansWrapper")) { + new("BisectingKMeansModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.LinearSVCWrapper")) { + new("LinearSVCModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.FPGrowthWrapper")) { + new("FPGrowthModel", jobj = jobj) + } else { + stop("Unsupported model: ", jobj) + } +} diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 199c3fd6ab1b..8fa21be3076b 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -21,23 +21,26 @@ NULL ############ Actions and Transformations ############ -# Look up elements of a key in an RDD -# -# @description -# \code{lookup} returns a list of values in this RDD for key key. -# -# @param x The RDD to collect -# @param key The key to look up for -# @return a list of values in this RDD for key key -# @examples -#\dontrun{ -# sc <- sparkR.init() -# pairs <- list(c(1, 1), c(2, 2), c(1, 3)) -# rdd <- parallelize(sc, pairs) -# lookup(rdd, 1) # list(1, 3) -#} -# @rdname lookup -# @aliases lookup,RDD-method +#' Look up elements of a key in an RDD +#' +#' @description +#' \code{lookup} returns a list of values in this RDD for key key. +#' +#' @param x The RDD to collect +#' @param key The key to look up for +#' @return a list of values in this RDD for key key +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(c(1, 1), c(2, 2), c(1, 3)) +#' rdd <- parallelize(sc, pairs) +#' lookup(rdd, 1) # list(1, 3) +#'} +# nolint end +#' @rdname lookup +#' @aliases lookup,RDD-method +#' @noRd setMethod("lookup", signature(x = "RDD", key = "ANY"), function(x, key) { @@ -46,24 +49,27 @@ setMethod("lookup", lapply(filtered, function(i) { i[[2]] }) } valsRDD <- lapplyPartition(x, partitionFunc) - collect(valsRDD) + collectRDD(valsRDD) }) -# Count the number of elements for each key, and return the result to the -# master as lists of (key, count) pairs. -# -# Same as countByKey in Spark. -# -# @param x The RDD to count keys. -# @return list of (key, count) pairs, where count is number of each key in rdd. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) -# countByKey(rdd) # ("a", 2L), ("b", 1L) -#} -# @rdname countByKey -# @aliases countByKey,RDD-method +#' Count the number of elements for each key, and return the result to the +#' master as lists of (key, count) pairs. +#' +#' Same as countByKey in Spark. +#' +#' @param x The RDD to count keys. +#' @return list of (key, count) pairs, where count is number of each key in rdd. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) +#' countByKey(rdd) # ("a", 2L), ("b", 1L) +#'} +# nolint end +#' @rdname countByKey +#' @aliases countByKey,RDD-method +#' @noRd setMethod("countByKey", signature(x = "RDD"), function(x) { @@ -71,17 +77,20 @@ setMethod("countByKey", countByValue(keys) }) -# Return an RDD with the keys of each tuple. -# -# @param x The RDD from which the keys of each tuple is returned. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) -# collect(keys(rdd)) # list(1, 3) -#} -# @rdname keys -# @aliases keys,RDD +#' Return an RDD with the keys of each tuple. +#' +#' @param x The RDD from which the keys of each tuple is returned. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +#' collectRDD(keys(rdd)) # list(1, 3) +#'} +# nolint end +#' @rdname keys +#' @aliases keys,RDD +#' @noRd setMethod("keys", signature(x = "RDD"), function(x) { @@ -91,17 +100,20 @@ setMethod("keys", lapply(x, func) }) -# Return an RDD with the values of each tuple. -# -# @param x The RDD from which the values of each tuple is returned. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) -# collect(values(rdd)) # list(2, 4) -#} -# @rdname values -# @aliases values,RDD +#' Return an RDD with the values of each tuple. +#' +#' @param x The RDD from which the values of each tuple is returned. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +#' collectRDD(values(rdd)) # list(2, 4) +#'} +# nolint end +#' @rdname values +#' @aliases values,RDD +#' @noRd setMethod("values", signature(x = "RDD"), function(x) { @@ -111,23 +123,24 @@ setMethod("values", lapply(x, func) }) -# Applies a function to all values of the elements, without modifying the keys. -# -# The same as `mapValues()' in Spark. -# -# @param X The RDD to apply the transformation. -# @param FUN the transformation to apply on the value of each element. -# @return a new RDD created by the transformation. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:10) -# makePairs <- lapply(rdd, function(x) { list(x, x) }) -# collect(mapValues(makePairs, function(x) { x * 2) }) -# Output: list(list(1,2), list(2,4), list(3,6), ...) -#} -# @rdname mapValues -# @aliases mapValues,RDD,function-method +#' Applies a function to all values of the elements, without modifying the keys. +#' +#' The same as `mapValues()' in Spark. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on the value of each element. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' makePairs <- lapply(rdd, function(x) { list(x, x) }) +#' collectRDD(mapValues(makePairs, function(x) { x * 2) }) +#' Output: list(list(1,2), list(2,4), list(3,6), ...) +#'} +#' @rdname mapValues +#' @aliases mapValues,RDD,function-method +#' @noRd setMethod("mapValues", signature(X = "RDD", FUN = "function"), function(X, FUN) { @@ -137,23 +150,24 @@ setMethod("mapValues", lapply(X, func) }) -# Pass each value in the key-value pair RDD through a flatMap function without -# changing the keys; this also retains the original RDD's partitioning. -# -# The same as 'flatMapValues()' in Spark. -# -# @param X The RDD to apply the transformation. -# @param FUN the transformation to apply on the value of each element. -# @return a new RDD created by the transformation. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) -# collect(flatMapValues(rdd, function(x) { x })) -# Output: list(list(1,1), list(1,2), list(2,3), list(2,4)) -#} -# @rdname flatMapValues -# @aliases flatMapValues,RDD,function-method +#' Pass each value in the key-value pair RDD through a flatMap function without +#' changing the keys; this also retains the original RDD's partitioning. +#' +#' The same as 'flatMapValues()' in Spark. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on the value of each element. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) +#' collectRDD(flatMapValues(rdd, function(x) { x })) +#' Output: list(list(1,1), list(1,2), list(2,3), list(2,4)) +#'} +#' @rdname flatMapValues +#' @aliases flatMapValues,RDD,function-method +#' @noRd setMethod("flatMapValues", signature(X = "RDD", FUN = "function"), function(X, FUN) { @@ -165,37 +179,35 @@ setMethod("flatMapValues", ############ Shuffle Functions ############ -# Partition an RDD by key -# -# This function operates on RDDs where every element is of the form list(K, V) or c(K, V). -# For each element of this RDD, the partitioner is used to compute a hash -# function and the RDD is partitioned using this hash value. -# -# @param x The RDD to partition. Should be an RDD where each element is -# list(K, V) or c(K, V). -# @param numPartitions Number of partitions to create. -# @param ... Other optional arguments to partitionBy. -# -# @param partitionFunc The partition function to use. Uses a default hashCode -# function if not provided -# @return An RDD partitioned using the specified partitioner. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -# rdd <- parallelize(sc, pairs) -# parts <- partitionBy(rdd, 2L) -# collectPartition(parts, 0L) # First partition should contain list(1, 2) and list(1, 4) -#} -# @rdname partitionBy -# @aliases partitionBy,RDD,integer-method -setMethod("partitionBy", - signature(x = "RDD", numPartitions = "numeric"), +#' Partition an RDD by key +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' For each element of this RDD, the partitioner is used to compute a hash +#' function and the RDD is partitioned using this hash value. +#' +#' @param x The RDD to partition. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param numPartitions Number of partitions to create. +#' @param ... Other optional arguments to partitionBy. +#' +#' @param partitionFunc The partition function to use. Uses a default hashCode +#' function if not provided +#' @return An RDD partitioned using the specified partitioner. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- partitionByRDD(rdd, 2L) +#' collectPartition(parts, 0L) # First partition should contain list(1, 2) and list(1, 4) +#'} +#' @rdname partitionBy +#' @aliases partitionBy,RDD,integer-method +#' @noRd +setMethod("partitionByRDD", + signature(x = "RDD"), function(x, numPartitions, partitionFunc = hashCode) { - - #if (missing(partitionFunc)) { - # partitionFunc <- hashCode - #} + stopifnot(is.numeric(numPartitions)) partitionFunc <- cleanClosure(partitionFunc) serializedHashFuncBytes <- serialize(partitionFunc, connection = NULL) @@ -233,31 +245,32 @@ setMethod("partitionBy", RDD(r, serializedMode = "byte") }) -# Group values by key -# -# This function operates on RDDs where every element is of the form list(K, V) or c(K, V). -# and group values for each key in the RDD into a single sequence. -# -# @param x The RDD to group. Should be an RDD where each element is -# list(K, V) or c(K, V). -# @param numPartitions Number of partitions to create. -# @return An RDD where each element is list(K, list(V)) -# @seealso reduceByKey -# @examples -#\dontrun{ -# sc <- sparkR.init() -# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -# rdd <- parallelize(sc, pairs) -# parts <- groupByKey(rdd, 2L) -# grouped <- collect(parts) -# grouped[[1]] # Should be a list(1, list(2, 4)) -#} -# @rdname groupByKey -# @aliases groupByKey,RDD,integer-method +#' Group values by key +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' and group values for each key in the RDD into a single sequence. +#' +#' @param x The RDD to group. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param numPartitions Number of partitions to create. +#' @return An RDD where each element is list(K, list(V)) +#' @seealso reduceByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- groupByKey(rdd, 2L) +#' grouped <- collectRDD(parts) +#' grouped[[1]] # Should be a list(1, list(2, 4)) +#'} +#' @rdname groupByKey +#' @aliases groupByKey,RDD,integer-method +#' @noRd setMethod("groupByKey", signature(x = "RDD", numPartitions = "numeric"), function(x, numPartitions) { - shuffled <- partitionBy(x, numPartitions) + shuffled <- partitionByRDD(x, numPartitions) groupVals <- function(part) { vals <- new.env() keys <- new.env() @@ -291,28 +304,29 @@ setMethod("groupByKey", lapplyPartition(shuffled, groupVals) }) -# Merge values by key -# -# This function operates on RDDs where every element is of the form list(K, V) or c(K, V). -# and merges the values for each key using an associative reduce function. -# -# @param x The RDD to reduce by key. Should be an RDD where each element is -# list(K, V) or c(K, V). -# @param combineFunc The associative reduce function to use. -# @param numPartitions Number of partitions to create. -# @return An RDD where each element is list(K, V') where V' is the merged -# value -# @examples -#\dontrun{ -# sc <- sparkR.init() -# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -# rdd <- parallelize(sc, pairs) -# parts <- reduceByKey(rdd, "+", 2L) -# reduced <- collect(parts) -# reduced[[1]] # Should be a list(1, 6) -#} -# @rdname reduceByKey -# @aliases reduceByKey,RDD,integer-method +#' Merge values by key +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' and merges the values for each key using an associative and commutative reduce function. +#' +#' @param x The RDD to reduce by key. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param combineFunc The associative and commutative reduce function to use. +#' @param numPartitions Number of partitions to create. +#' @return An RDD where each element is list(K, V') where V' is the merged +#' value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- reduceByKey(rdd, "+", 2L) +#' reduced <- collectRDD(parts) +#' reduced[[1]] # Should be a list(1, 6) +#'} +#' @rdname reduceByKey +#' @aliases reduceByKey,RDD,integer-method +#' @noRd setMethod("reduceByKey", signature(x = "RDD", combineFunc = "ANY", numPartitions = "numeric"), function(x, combineFunc, numPartitions) { @@ -328,31 +342,34 @@ setMethod("reduceByKey", convertEnvsToList(keys, vals) } locallyReduced <- lapplyPartition(x, reduceVals) - shuffled <- partitionBy(locallyReduced, numToInt(numPartitions)) + shuffled <- partitionByRDD(locallyReduced, numToInt(numPartitions)) lapplyPartition(shuffled, reduceVals) }) -# Merge values by key locally -# -# This function operates on RDDs where every element is of the form list(K, V) or c(K, V). -# and merges the values for each key using an associative reduce function, but return the -# results immediately to the driver as an R list. -# -# @param x The RDD to reduce by key. Should be an RDD where each element is -# list(K, V) or c(K, V). -# @param combineFunc The associative reduce function to use. -# @return A list of elements of type list(K, V') where V' is the merged value for each key -# @seealso reduceByKey -# @examples -#\dontrun{ -# sc <- sparkR.init() -# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -# rdd <- parallelize(sc, pairs) -# reduced <- reduceByKeyLocally(rdd, "+") -# reduced # list(list(1, 6), list(1.1, 3)) -#} -# @rdname reduceByKeyLocally -# @aliases reduceByKeyLocally,RDD,integer-method +#' Merge values by key locally +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' and merges the values for each key using an associative and commutative reduce function, but +#' return the results immediately to the driver as an R list. +#' +#' @param x The RDD to reduce by key. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param combineFunc The associative and commutative reduce function to use. +#' @return A list of elements of type list(K, V') where V' is the merged value for each key +#' @seealso reduceByKey +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' reduced <- reduceByKeyLocally(rdd, "+") +#' reduced # list(list(1, 6), list(1.1, 3)) +#'} +# nolint end +#' @rdname reduceByKeyLocally +#' @aliases reduceByKeyLocally,RDD,integer-method +#' @noRd setMethod("reduceByKeyLocally", signature(x = "RDD", combineFunc = "ANY"), function(x, combineFunc) { @@ -384,41 +401,42 @@ setMethod("reduceByKeyLocally", convertEnvsToList(merged[[1]], merged[[2]]) }) -# Combine values by key -# -# Generic function to combine the elements for each key using a custom set of -# aggregation functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], -# for a "combined type" C. Note that V and C can be different -- for example, one -# might group an RDD of type (Int, Int) into an RDD of type (Int, Seq[Int]). - -# Users provide three functions: -# \itemize{ -# \item createCombiner, which turns a V into a C (e.g., creates a one-element list) -# \item mergeValue, to merge a V into a C (e.g., adds it to the end of a list) - -# \item mergeCombiners, to combine two C's into a single one (e.g., concatentates -# two lists). -# } -# -# @param x The RDD to combine. Should be an RDD where each element is -# list(K, V) or c(K, V). -# @param createCombiner Create a combiner (C) given a value (V) -# @param mergeValue Merge the given value (V) with an existing combiner (C) -# @param mergeCombiners Merge two combiners and return a new combiner -# @param numPartitions Number of partitions to create. -# @return An RDD where each element is list(K, C) where C is the combined type -# -# @seealso groupByKey, reduceByKey -# @examples -#\dontrun{ -# sc <- sparkR.init() -# pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) -# rdd <- parallelize(sc, pairs) -# parts <- combineByKey(rdd, function(x) { x }, "+", "+", 2L) -# combined <- collect(parts) -# combined[[1]] # Should be a list(1, 6) -#} -# @rdname combineByKey -# @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method +#' Combine values by key +#' +#' Generic function to combine the elements for each key using a custom set of +#' aggregation functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], +#' for a "combined type" C. Note that V and C can be different -- for example, one +#' might group an RDD of type (Int, Int) into an RDD of type (Int, Seq[Int]). +#' Users provide three functions: +#' \itemize{ +#' \item createCombiner, which turns a V into a C (e.g., creates a one-element list) +#' \item mergeValue, to merge a V into a C (e.g., adds it to the end of a list) - +#' \item mergeCombiners, to combine two C's into a single one (e.g., concatentates +#' two lists). +#' } +#' +#' @param x The RDD to combine. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param createCombiner Create a combiner (C) given a value (V) +#' @param mergeValue Merge the given value (V) with an existing combiner (C) +#' @param mergeCombiners Merge two combiners and return a new combiner +#' @param numPartitions Number of partitions to create. +#' @return An RDD where each element is list(K, C) where C is the combined type +#' @seealso groupByKey, reduceByKey +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- combineByKey(rdd, function(x) { x }, "+", "+", 2L) +#' combined <- collectRDD(parts) +#' combined[[1]] # Should be a list(1, 6) +#'} +# nolint end +#' @rdname combineByKey +#' @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method +#' @noRd setMethod("combineByKey", signature(x = "RDD", createCombiner = "ANY", mergeValue = "ANY", mergeCombiners = "ANY", numPartitions = "numeric"), @@ -435,7 +453,7 @@ setMethod("combineByKey", convertEnvsToList(keys, combiners) } locallyCombined <- lapplyPartition(x, combineLocally) - shuffled <- partitionBy(locallyCombined, numToInt(numPartitions)) + shuffled <- partitionByRDD(locallyCombined, numToInt(numPartitions)) mergeAfterShuffle <- function(part) { combiners <- new.env() keys <- new.env() @@ -450,36 +468,39 @@ setMethod("combineByKey", lapplyPartition(shuffled, mergeAfterShuffle) }) -# Aggregate a pair RDD by each key. -# -# Aggregate the values of each key in an RDD, using given combine functions -# and a neutral "zero value". This function can return a different result type, -# U, than the type of the values in this RDD, V. Thus, we need one operation -# for merging a V into a U and one operation for merging two U's, The former -# operation is used for merging values within a partition, and the latter is -# used for merging values between partitions. To avoid memory allocation, both -# of these functions are allowed to modify and return their first argument -# instead of creating a new U. -# -# @param x An RDD. -# @param zeroValue A neutral "zero value". -# @param seqOp A function to aggregate the values of each key. It may return -# a different result type from the type of the values. -# @param combOp A function to aggregate results of seqOp. -# @return An RDD containing the aggregation result. -# @seealso foldByKey, combineByKey -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) -# zeroValue <- list(0, 0) -# seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } -# combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } -# aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) -# # list(list(1, list(3, 2)), list(2, list(7, 2))) -#} -# @rdname aggregateByKey -# @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method +#' Aggregate a pair RDD by each key. +#' +#' Aggregate the values of each key in an RDD, using given combine functions +#' and a neutral "zero value". This function can return a different result type, +#' U, than the type of the values in this RDD, V. Thus, we need one operation +#' for merging a V into a U and one operation for merging two U's, The former +#' operation is used for merging values within a partition, and the latter is +#' used for merging values between partitions. To avoid memory allocation, both +#' of these functions are allowed to modify and return their first argument +#' instead of creating a new U. +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param seqOp A function to aggregate the values of each key. It may return +#' a different result type from the type of the values. +#' @param combOp A function to aggregate results of seqOp. +#' @return An RDD containing the aggregation result. +#' @seealso foldByKey, combineByKey +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) +#' zeroValue <- list(0, 0) +#' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } +#' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } +#' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) +#' # list(list(1, list(3, 2)), list(2, list(7, 2))) +#'} +# nolint end +#' @rdname aggregateByKey +#' @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method +#' @noRd setMethod("aggregateByKey", signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY", numPartitions = "numeric"), @@ -491,26 +512,29 @@ setMethod("aggregateByKey", combineByKey(x, createCombiner, seqOp, combOp, numPartitions) }) -# Fold a pair RDD by each key. -# -# Aggregate the values of each key in an RDD, using an associative function "func" -# and a neutral "zero value" which may be added to the result an arbitrary -# number of times, and must not change the result (e.g., 0 for addition, or -# 1 for multiplication.). -# -# @param x An RDD. -# @param zeroValue A neutral "zero value". -# @param func An associative function for folding values of each key. -# @return An RDD containing the aggregation result. -# @seealso aggregateByKey, combineByKey -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) -# foldByKey(rdd, 0, "+", 2L) # list(list(1, 3), list(2, 7)) -#} -# @rdname foldByKey -# @aliases foldByKey,RDD,ANY,ANY,integer-method +#' Fold a pair RDD by each key. +#' +#' Aggregate the values of each key in an RDD, using an associative function "func" +#' and a neutral "zero value" which may be added to the result an arbitrary +#' number of times, and must not change the result (e.g., 0 for addition, or +#' 1 for multiplication.). +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param func An associative function for folding values of each key. +#' @return An RDD containing the aggregation result. +#' @seealso aggregateByKey, combineByKey +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) +#' foldByKey(rdd, 0, "+", 2L) # list(list(1, 3), list(2, 7)) +#'} +# nolint end +#' @rdname foldByKey +#' @aliases foldByKey,RDD,ANY,ANY,integer-method +#' @noRd setMethod("foldByKey", signature(x = "RDD", zeroValue = "ANY", func = "ANY", numPartitions = "numeric"), @@ -520,29 +544,32 @@ setMethod("foldByKey", ############ Binary Functions ############# -# Join two RDDs -# -# @description -# \code{join} This function joins two RDDs where every element is of the form list(K, V). -# The key types of the two RDDs should be the same. -# -# @param x An RDD to be joined. Should be an RDD where each element is -# list(K, V). -# @param y An RDD to be joined. Should be an RDD where each element is -# list(K, V). -# @param numPartitions Number of partitions to create. -# @return a new RDD containing all pairs of elements with matching keys in -# two input RDDs. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) -# rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) -# join(rdd1, rdd2, 2L) # list(list(1, list(1, 2)), list(1, list(1, 3)) -#} -# @rdname join-methods -# @aliases join,RDD,RDD-method -setMethod("join", +#' Join two RDDs +#' +#' @description +#' \code{join} This function joins two RDDs where every element is of the form list(K, V). +#' The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return a new RDD containing all pairs of elements with matching keys in +#' two input RDDs. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' joinRDD(rdd1, rdd2, 2L) # list(list(1, list(1, 2)), list(1, list(1, 3)) +#'} +# nolint end +#' @rdname join-methods +#' @aliases join,RDD,RDD-method +#' @noRd +setMethod("joinRDD", signature(x = "RDD", y = "RDD"), function(x, y, numPartitions) { xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) @@ -556,30 +583,33 @@ setMethod("join", doJoin) }) -# Left outer join two RDDs -# -# @description -# \code{leftouterjoin} This function left-outer-joins two RDDs where every element is of -# the form list(K, V). The key types of the two RDDs should be the same. -# -# @param x An RDD to be joined. Should be an RDD where each element is -# list(K, V). -# @param y An RDD to be joined. Should be an RDD where each element is -# list(K, V). -# @param numPartitions Number of partitions to create. -# @return For each element (k, v) in x, the resulting RDD will either contain -# all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) -# if no elements in rdd2 have key k. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) -# rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) -# leftOuterJoin(rdd1, rdd2, 2L) -# # list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) -#} -# @rdname join-methods -# @aliases leftOuterJoin,RDD,RDD-method +#' Left outer join two RDDs +#' +#' @description +#' \code{leftouterjoin} This function left-outer-joins two RDDs where every element is of +#' the form list(K, V). The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return For each element (k, v) in x, the resulting RDD will either contain +#' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) +#' if no elements in rdd2 have key k. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' leftOuterJoin(rdd1, rdd2, 2L) +#' # list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) +#'} +# nolint end +#' @rdname join-methods +#' @aliases leftOuterJoin,RDD,RDD-method +#' @noRd setMethod("leftOuterJoin", signature(x = "RDD", y = "RDD", numPartitions = "numeric"), function(x, y, numPartitions) { @@ -593,30 +623,33 @@ setMethod("leftOuterJoin", joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) -# Right outer join two RDDs -# -# @description -# \code{rightouterjoin} This function right-outer-joins two RDDs where every element is of -# the form list(K, V). The key types of the two RDDs should be the same. -# -# @param x An RDD to be joined. Should be an RDD where each element is -# list(K, V). -# @param y An RDD to be joined. Should be an RDD where each element is -# list(K, V). -# @param numPartitions Number of partitions to create. -# @return For each element (k, w) in y, the resulting RDD will either contain -# all pairs (k, (v, w)) for (k, v) in x, or the pair (k, (NULL, w)) -# if no elements in x have key k. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3))) -# rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) -# rightOuterJoin(rdd1, rdd2, 2L) -# # list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) -#} -# @rdname join-methods -# @aliases rightOuterJoin,RDD,RDD-method +#' Right outer join two RDDs +#' +#' @description +#' \code{rightouterjoin} This function right-outer-joins two RDDs where every element is of +#' the form list(K, V). The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return For each element (k, w) in y, the resulting RDD will either contain +#' all pairs (k, (v, w)) for (k, v) in x, or the pair (k, (NULL, w)) +#' if no elements in x have key k. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rightOuterJoin(rdd1, rdd2, 2L) +#' # list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) +#'} +# nolint end +#' @rdname join-methods +#' @aliases rightOuterJoin,RDD,RDD-method +#' @noRd setMethod("rightOuterJoin", signature(x = "RDD", y = "RDD", numPartitions = "numeric"), function(x, y, numPartitions) { @@ -630,33 +663,36 @@ setMethod("rightOuterJoin", joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) -# Full outer join two RDDs -# -# @description -# \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of -# the form list(K, V). The key types of the two RDDs should be the same. -# -# @param x An RDD to be joined. Should be an RDD where each element is -# list(K, V). -# @param y An RDD to be joined. Should be an RDD where each element is -# list(K, V). -# @param numPartitions Number of partitions to create. -# @return For each element (k, v) in x and (k, w) in y, the resulting RDD -# will contain all pairs (k, (v, w)) for both (k, v) in x and -# (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements -# in x/y have key k. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3), list(3, 3))) -# rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) -# fullOuterJoin(rdd1, rdd2, 2L) # list(list(1, list(2, 1)), -# # list(1, list(3, 1)), -# # list(2, list(NULL, 4))) -# # list(3, list(3, NULL)), -#} -# @rdname join-methods -# @aliases fullOuterJoin,RDD,RDD-method +#' Full outer join two RDDs +#' +#' @description +#' \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of +#' the form list(K, V). The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return For each element (k, v) in x and (k, w) in y, the resulting RDD +#' will contain all pairs (k, (v, w)) for both (k, v) in x and +#' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements +#' in x/y have key k. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3), list(3, 3))) +#' rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' fullOuterJoin(rdd1, rdd2, 2L) # list(list(1, list(2, 1)), +#' # list(1, list(3, 1)), +#' # list(2, list(NULL, 4))) +#' # list(3, list(3, NULL)), +#'} +# nolint end +#' @rdname join-methods +#' @aliases fullOuterJoin,RDD,RDD-method +#' @noRd setMethod("fullOuterJoin", signature(x = "RDD", y = "RDD", numPartitions = "numeric"), function(x, y, numPartitions) { @@ -670,23 +706,26 @@ setMethod("fullOuterJoin", joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) }) -# For each key k in several RDDs, return a resulting RDD that -# whose values are a list of values for the key in all RDDs. -# -# @param ... Several RDDs. -# @param numPartitions Number of partitions to create. -# @return a new RDD containing all pairs of elements with values in a list -# in all RDDs. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) -# rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) -# cogroup(rdd1, rdd2, numPartitions = 2L) -# # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list())) -#} -# @rdname cogroup -# @aliases cogroup,RDD-method +#' For each key k in several RDDs, return a resulting RDD that +#' whose values are a list of values for the key in all RDDs. +#' +#' @param ... Several RDDs. +#' @param numPartitions Number of partitions to create. +#' @return a new RDD containing all pairs of elements with values in a list +#' in all RDDs. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' cogroup(rdd1, rdd2, numPartitions = 2L) +#' # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list())) +#'} +# nolint end +#' @rdname cogroup +#' @aliases cogroup,RDD-method +#' @noRd setMethod("cogroup", "RDD", function(..., numPartitions) { @@ -722,32 +761,35 @@ setMethod("cogroup", group.func) }) -# Sort a (k, v) pair RDD by k. -# -# @param x A (k, v) pair RDD to be sorted. -# @param ascending A flag to indicate whether the sorting is ascending or descending. -# @param numPartitions Number of partitions to create. -# @return An RDD where all (k, v) pair elements are sorted. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, list(list(3, 1), list(2, 2), list(1, 3))) -# collect(sortByKey(rdd)) # list (list(1, 3), list(2, 2), list(3, 1)) -#} -# @rdname sortByKey -# @aliases sortByKey,RDD,RDD-method +#' Sort a (k, v) pair RDD by k. +#' +#' @param x A (k, v) pair RDD to be sorted. +#' @param ascending A flag to indicate whether the sorting is ascending or descending. +#' @param numPartitions Number of partitions to create. +#' @return An RDD where all (k, v) pair elements are sorted. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(3, 1), list(2, 2), list(1, 3))) +#' collectRDD(sortByKey(rdd)) # list (list(1, 3), list(2, 2), list(3, 1)) +#'} +# nolint end +#' @rdname sortByKey +#' @aliases sortByKey,RDD,RDD-method +#' @noRd setMethod("sortByKey", signature(x = "RDD"), - function(x, ascending = TRUE, numPartitions = SparkR:::numPartitions(x)) { + function(x, ascending = TRUE, numPartitions = SparkR:::getNumPartitionsRDD(x)) { rangeBounds <- list() if (numPartitions > 1) { - rddSize <- count(x) + rddSize <- countRDD(x) # constant from Spark's RangePartitioner maxSampleSize <- numPartitions * 20 fraction <- min(maxSampleSize / max(rddSize, 1), 1.0) - samples <- collect(keys(sampleRDD(x, FALSE, fraction, 1L))) + samples <- collectRDD(keys(sampleRDD(x, FALSE, fraction, 1L))) # Note: the built-in R sort() function only works on atomic vectors samples <- sort(unlist(samples, recursive = FALSE), decreasing = !ascending) @@ -780,32 +822,35 @@ setMethod("sortByKey", sortKeyValueList(part, decreasing = !ascending) } - newRDD <- partitionBy(x, numPartitions, rangePartitionFunc) + newRDD <- partitionByRDD(x, numPartitions, rangePartitionFunc) lapplyPartition(newRDD, partitionFunc) }) -# Subtract a pair RDD with another pair RDD. -# -# Return an RDD with the pairs from x whose keys are not in other. -# -# @param x An RDD. -# @param other An RDD. -# @param numPartitions Number of the partitions in the result RDD. -# @return An RDD with the pairs from x whose keys are not in other. -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4), -# list("b", 5), list("a", 2))) -# rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) -# collect(subtractByKey(rdd1, rdd2)) -# # list(list("b", 4), list("b", 5)) -#} -# @rdname subtractByKey -# @aliases subtractByKey,RDD +#' Subtract a pair RDD with another pair RDD. +#' +#' Return an RDD with the pairs from x whose keys are not in other. +#' +#' @param x An RDD. +#' @param other An RDD. +#' @param numPartitions Number of the partitions in the result RDD. +#' @return An RDD with the pairs from x whose keys are not in other. +#' @examples +# nolint start +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4), +#' list("b", 5), list("a", 2))) +#' rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) +#' collectRDD(subtractByKey(rdd1, rdd2)) +#' # list(list("b", 4), list("b", 5)) +#'} +# nolint end +#' @rdname subtractByKey +#' @aliases subtractByKey,RDD +#' @noRd setMethod("subtractByKey", signature(x = "RDD", other = "RDD"), - function(x, other, numPartitions = SparkR:::numPartitions(x)) { + function(x, other, numPartitions = SparkR:::getNumPartitionsRDD(x)) { filterFunction <- function(elem) { iters <- elem[[2]] (length(iters[[1]]) > 0) && (length(iters[[2]]) == 0) @@ -818,41 +863,42 @@ setMethod("subtractByKey", function (v) { v[[1]] }) }) -# Return a subset of this RDD sampled by key. -# -# @description -# \code{sampleByKey} Create a sample of this RDD using variable sampling rates -# for different keys as specified by fractions, a key to sampling rate map. -# -# @param x The RDD to sample elements by key, where each element is -# list(K, V) or c(K, V). -# @param withReplacement Sampling with replacement or not -# @param fraction The (rough) sample target fraction -# @param seed Randomness seed value -# @examples -#\dontrun{ -# sc <- sparkR.init() -# rdd <- parallelize(sc, 1:3000) -# pairs <- lapply(rdd, function(x) { if (x %% 3 == 0) list("a", x) -# else { if (x %% 3 == 1) list("b", x) else list("c", x) }}) -# fractions <- list(a = 0.2, b = 0.1, c = 0.3) -# sample <- sampleByKey(pairs, FALSE, fractions, 1618L) -# 100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")) # TRUE -# 50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")) # TRUE -# 200 < length(lookup(sample, "c")) && 400 > length(lookup(sample, "c")) # TRUE -# lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0 # TRUE -# lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000 # TRUE -# lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0 # TRUE -# lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000 # TRUE -# lookup(sample, "c")[which.min(lookup(sample, "c"))] >= 0 # TRUE -# lookup(sample, "c")[which.max(lookup(sample, "c"))] <= 2000 # TRUE -# fractions <- list(a = 0.2, b = 0.1, c = 0.3, d = 0.4) -# sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # Key "d" will be ignored -# fractions <- list(a = 0.2, b = 0.1) -# sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # KeyError: "c" -#} -# @rdname sampleByKey -# @aliases sampleByKey,RDD-method +#' Return a subset of this RDD sampled by key. +#' +#' @description +#' \code{sampleByKey} Create a sample of this RDD using variable sampling rates +#' for different keys as specified by fractions, a key to sampling rate map. +#' +#' @param x The RDD to sample elements by key, where each element is +#' list(K, V) or c(K, V). +#' @param withReplacement Sampling with replacement or not +#' @param fraction The (rough) sample target fraction +#' @param seed Randomness seed value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3000) +#' pairs <- lapply(rdd, function(x) { if (x %% 3 == 0) list("a", x) +#' else { if (x %% 3 == 1) list("b", x) else list("c", x) }}) +#' fractions <- list(a = 0.2, b = 0.1, c = 0.3) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) +#' 100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")) # TRUE +#' 50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")) # TRUE +#' 200 < length(lookup(sample, "c")) && 400 > length(lookup(sample, "c")) # TRUE +#' lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0 # TRUE +#' lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000 # TRUE +#' lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0 # TRUE +#' lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000 # TRUE +#' lookup(sample, "c")[which.min(lookup(sample, "c"))] >= 0 # TRUE +#' lookup(sample, "c")[which.max(lookup(sample, "c"))] <= 2000 # TRUE +#' fractions <- list(a = 0.2, b = 0.1, c = 0.3, d = 0.4) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # Key "d" will be ignored +#' fractions <- list(a = 0.2, b = 0.1) +#' sample <- sampleByKey(pairs, FALSE, fractions, 1618L) # KeyError: "c" +#'} +#' @rdname sampleByKey +#' @aliases sampleByKey,RDD-method +#' @noRd setMethod("sampleByKey", signature(x = "RDD", withReplacement = "logical", fractions = "vector", seed = "integer"), @@ -871,19 +917,19 @@ setMethod("sampleByKey", len <- 0 # mixing because the initial seeds are close to each other - runif(10) + stats::runif(10) for (elem in part) { if (elem[[1]] %in% names(fractions)) { frac <- as.numeric(fractions[which(elem[[1]] == names(fractions))]) if (withReplacement) { - count <- rpois(1, frac) + count <- stats::rpois(1, frac) if (count > 0) { res[ (len + 1) : (len + count) ] <- rep(list(elem), count) len <- len + count } } else { - if (runif(1) < frac) { + if (stats::runif(1) < frac) { len <- len + 1 res[[len]] <- elem } diff --git a/R/pkg/R/schema.R b/R/pkg/R/schema.R index 6f0e9a94e9bf..d1ed6833d5d0 100644 --- a/R/pkg/R/schema.R +++ b/R/pkg/R/schema.R @@ -16,36 +16,50 @@ # # A set of S3 classes and methods that support the SparkSQL `StructType` and `StructField -# datatypes. These are used to create and interact with DataFrame schemas. +# datatypes. These are used to create and interact with SparkDataFrame schemas. #' structType #' -#' Create a structType object that contains the metadata for a DataFrame. Intended for +#' Create a structType object that contains the metadata for a SparkDataFrame. Intended for #' use with createDataFrame and toDF. #' -#' @param x a structField object (created with the field() function) +#' @param x a structField object (created with the \code{structField} method). Since Spark 2.3, +#' this can be a DDL-formatted string, which is a comma separated list of field +#' definitions, e.g., "a INT, b STRING". #' @param ... additional structField objects #' @return a structType object +#' @rdname structType #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) -#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) -#' schema <- structType(structField("a", "integer"), structField("b", "string")) -#' df <- createDataFrame(sqlCtx, rdd, schema) +#' schema <- structType(structField("a", "integer"), structField("c", "string"), +#' structField("avg", "double")) +#' df1 <- gapply(df, list("a", "c"), +#' function(key, x) { y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) }, +#' schema) +#' schema <- structType("a INT, c STRING, avg DOUBLE") +#' df1 <- gapply(df, list("a", "c"), +#' function(key, x) { y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) }, +#' schema) #' } +#' @note structType since 1.4.0 structType <- function(x, ...) { UseMethod("structType", x) } -structType.jobj <- function(x) { +#' @rdname structType +#' @method structType jobj +#' @export +structType.jobj <- function(x, ...) { obj <- structure(list(), class = "structType") obj$jobj <- x obj$fields <- function() { lapply(callJMethod(obj$jobj, "fields"), structField) } obj } +#' @rdname structType +#' @method structType structField +#' @export structType.structField <- function(x, ...) { fields <- list(x, ...) if (!all(sapply(fields, inherits, "structField"))) { @@ -60,6 +74,23 @@ structType.structField <- function(x, ...) { structType(stObj) } +#' @rdname structType +#' @method structType character +#' @export +structType.character <- function(x, ...) { + if (!is.character(x)) { + stop("schema must be a DDL-formatted string.") + } + if (length(list(...)) > 0) { + stop("multiple DDL-formatted strings are not supported") + } + + stObj <- handledCallJStatic("org.apache.spark.sql.types.StructType", + "fromDDL", + x) + structType(stObj) +} + #' Print a Spark StructType. #' #' This function prints the contents of a StructType returned from the @@ -67,6 +98,7 @@ structType.structField <- function(x, ...) { #' #' @param x A StructType object #' @param ... further arguments passed to or from other methods +#' @note print.structType since 1.4.0 print.structType <- function(x, ...) { cat("StructType\n", sapply(x$fields(), @@ -83,27 +115,30 @@ print.structType <- function(x, ...) { #' #' Create a structField object that contains the metadata for a single field in a schema. #' -#' @param x The name of the field -#' @param type The data type of the field -#' @param nullable A logical vector indicating whether or not the field is nullable -#' @return a structField object +#' @param x the name of the field. +#' @param ... additional argument(s) passed to the method. +#' @return A structField object. +#' @rdname structField #' @export #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' sqlCtx <- sparkRSQL.init(sc) -#' rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) -#' field1 <- structField("a", "integer", TRUE) -#' field2 <- structField("b", "string", TRUE) -#' schema <- structType(field1, field2) -#' df <- createDataFrame(sqlCtx, rdd, schema) +#' field1 <- structField("a", "integer") +#' field2 <- structField("c", "string") +#' field3 <- structField("avg", "double") +#' schema <- structType(field1, field2, field3) +#' df1 <- gapply(df, list("a", "c"), +#' function(key, x) { y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) }, +#' schema) #' } - +#' @note structField since 1.4.0 structField <- function(x, ...) { UseMethod("structField", x) } -structField.jobj <- function(x) { +#' @rdname structField +#' @method structField jobj +#' @export +structField.jobj <- function(x, ...) { obj <- structure(list(), class = "structField") obj$jobj <- x obj$name <- function() { callJMethod(x, "name") } @@ -115,20 +150,7 @@ structField.jobj <- function(x) { } checkType <- function(type) { - primtiveTypes <- c("byte", - "integer", - "float", - "double", - "numeric", - "character", - "string", - "binary", - "raw", - "logical", - "boolean", - "timestamp", - "date") - if (type %in% primtiveTypes) { + if (!is.null(PRIMITIVE_TYPES[[type]])) { return() } else { # Check complex types @@ -184,10 +206,14 @@ checkType <- function(type) { }) } - stop(paste("Unsupported type for Dataframe:", type)) + stop(paste("Unsupported type for SparkDataframe:", type)) } -structField.character <- function(x, type, nullable = TRUE) { +#' @param type The data type of the field +#' @param nullable A logical vector indicating whether or not the field is nullable +#' @rdname structField +#' @export +structField.character <- function(x, type, nullable = TRUE, ...) { if (class(x) != "character") { stop("Field name must be a string.") } @@ -215,6 +241,7 @@ structField.character <- function(x, type, nullable = TRUE) { #' #' @param x A StructField object #' @param ... further arguments passed to or from other methods +#' @note print.structField since 1.4.0 print.structField <- function(x, ...) { cat("StructField(name = \"", x$name(), "\", type = \"", x$dataType.toString(), diff --git a/R/pkg/R/serialize.R b/R/pkg/R/serialize.R index 17082b4e52fc..3bbf60d9b668 100644 --- a/R/pkg/R/serialize.R +++ b/R/pkg/R/serialize.R @@ -17,6 +17,7 @@ # Utility functions to serialize R objects so they can be read in Java. +# nolint start # Type mapping from R to Java # # NULL -> Void @@ -31,6 +32,7 @@ # list[T] -> Array[T], where T is one of above mentioned types # environment -> Map[String, T], where T is a native type # jobj -> Object, where jobj is an object created in the backend +# nolint end getSerdeType <- function(object) { type <- class(object)[[1]] @@ -52,7 +54,7 @@ writeObject <- function(con, object, writeType = TRUE) { # passing in vectors as arrays and instead require arrays to be passed # as lists. type <- class(object)[[1]] # class of POSIXlt is c("POSIXlt", "POSIXt") - # Checking types is needed here, since ‘is.na’ only handles atomic vectors, + # Checking types is needed here, since 'is.na' only handles atomic vectors, # lists and pairlists if (type %in% c("integer", "character", "logical", "double", "numeric")) { if (is.na(object)) { @@ -98,7 +100,7 @@ writeJobj <- function(con, value) { writeString <- function(con, value) { utfVal <- enc2utf8(value) writeInt(con, as.integer(nchar(utfVal, type = "bytes") + 1)) - writeBin(utfVal, con, endian = "big", useBytes=TRUE) + writeBin(utfVal, con, endian = "big", useBytes = TRUE) } writeInt <- function(con, value) { diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 004d08e74e1c..81507ea7186a 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -28,25 +28,32 @@ connExists <- function(env) { }) } -#' Stop the Spark context. +#' Stop the Spark Session and Spark Context #' -#' Also terminates the backend this R session is connected to -sparkR.stop <- function() { +#' Stop the Spark Session and Spark Context. +#' +#' Also terminates the backend this R session is connected to. +#' @rdname sparkR.session.stop +#' @name sparkR.session.stop +#' @export +#' @note sparkR.session.stop since 2.0.0 +sparkR.session.stop <- function() { env <- .sparkREnv if (exists(".sparkRCon", envir = env)) { - # cat("Stopping SparkR\n") if (exists(".sparkRjsc", envir = env)) { sc <- get(".sparkRjsc", envir = env) callJMethod(sc, "stop") rm(".sparkRjsc", envir = env) - if (exists(".sparkRSQLsc", envir = env)) { - rm(".sparkRSQLsc", envir = env) + if (exists(".sparkRsession", envir = env)) { + rm(".sparkRsession", envir = env) } + } - if (exists(".sparkRHivesc", envir = env)) { - rm(".sparkRHivesc", envir = env) - } + # Remove the R package lib path from .libPaths() + if (exists(".libPath", envir = env)) { + libPath <- get(".libPath", envir = env) + .libPaths(.libPaths()[.libPaths() != libPath]) } if (exists(".backendLaunched", envir = env)) { @@ -75,19 +82,27 @@ sparkR.stop <- function() { clearJobjs() } -#' Initialize a new Spark Context. +#' @rdname sparkR.session.stop +#' @name sparkR.stop +#' @export +#' @note sparkR.stop since 1.4.0 +sparkR.stop <- function() { + sparkR.session.stop() +} + +#' (Deprecated) Initialize a new Spark Context #' -#' This function initializes a new SparkContext. For details on how to initialize -#' and use SparkR, refer to SparkR programming guide at -#' \url{http://spark.apache.org/docs/latest/sparkr.html#starting-up-sparkcontext-sqlcontext}. +#' This function initializes a new SparkContext. #' -#' @param master The Spark master URL. +#' @param master The Spark master URL #' @param appName Application name to register with cluster manager #' @param sparkHome Spark Home directory -#' @param sparkEnvir Named list of environment variables to set on worker nodes. -#' @param sparkExecutorEnv Named list of environment variables to be used when launching executors. -#' @param sparkJars Character string vector of jar files to pass to the worker nodes. -#' @param sparkPackages Character string vector of packages from spark-packages.org +#' @param sparkEnvir Named list of environment variables to set on worker nodes +#' @param sparkExecutorEnv Named list of environment variables to be used when launching executors +#' @param sparkJars Character vector of jar files to pass to the worker nodes +#' @param sparkPackages Character vector of package coordinates +#' @seealso \link{sparkR.session} +#' @rdname sparkR.init-deprecated #' @export #' @examples #'\dontrun{ @@ -97,9 +112,10 @@ sparkR.stop <- function() { #' sc <- sparkR.init("yarn-client", "SparkR", "/home/spark", #' list(spark.executor.memory="4g"), #' list(LD_LIBRARY_PATH="/directory of JVM libraries (libjvm.so) on workers/"), -#' c("jarfile1.jar","jarfile2.jar")) +#' c("one.jar", "two.jar", "three.jar"), +#' c("com.databricks:spark-avro_2.11:2.0.1")) #'} - +#' @note sparkR.init since 1.4.0 sparkR.init <- function( master = "", appName = "SparkR", @@ -108,27 +124,42 @@ sparkR.init <- function( sparkExecutorEnv = list(), sparkJars = "", sparkPackages = "") { + .Deprecated("sparkR.session") + sparkR.sparkContext(master, + appName, + sparkHome, + convertNamedListToEnv(sparkEnvir), + convertNamedListToEnv(sparkExecutorEnv), + sparkJars, + sparkPackages) +} + +# Internal function to handle creating the SparkContext. +sparkR.sparkContext <- function( + master = "", + appName = "SparkR", + sparkHome = Sys.getenv("SPARK_HOME"), + sparkEnvirMap = new.env(), + sparkExecutorEnvMap = new.env(), + sparkJars = "", + sparkPackages = "") { if (exists(".sparkRjsc", envir = .sparkREnv)) { cat(paste("Re-using existing Spark Context.", - "Please stop SparkR with sparkR.stop() or restart R to create a new Spark Context\n")) + "Call sparkR.session.stop() or restart R to create a new Spark Context\n")) return(get(".sparkRjsc", envir = .sparkREnv)) } - jars <- suppressWarnings(normalizePath(as.character(sparkJars))) - - # Classpath separator is ";" on Windows - # URI needs four /// as from http://stackoverflow.com/a/18522792 - if (.Platform$OS.type == "unix") { - uriSep <- "//" - } else { - uriSep <- "////" - } - - sparkEnvirMap <- convertNamedListToEnv(sparkEnvir) + jars <- processSparkJars(sparkJars) + packages <- processSparkPackages(sparkPackages) existingPort <- Sys.getenv("EXISTING_SPARKR_BACKEND_PORT", "") + connectionTimeout <- as.numeric(Sys.getenv("SPARKR_BACKEND_CONNECTION_TIMEOUT", "6000")) if (existingPort != "") { + if (length(packages) != 0) { + warning(paste("sparkPackages has no effect when using spark-submit or sparkR shell", + " please use the --packages commandline instead", sep = ",")) + } backendPort <- existingPort } else { path <- tempfile(pattern = "backend_port") @@ -140,7 +171,7 @@ sparkR.init <- function( sparkHome = sparkHome, jars = jars, sparkSubmitOpts = submitOps, - packages = sparkPackages) + packages = packages) # wait atmost 100 seconds for JVM to launch wait <- 0.1 for (i in 1:25) { @@ -153,22 +184,31 @@ sparkR.init <- function( if (!file.exists(path)) { stop("JVM is not ready after 10 seconds") } - f <- file(path, open="rb") + f <- file(path, open = "rb") backendPort <- readInt(f) monitorPort <- readInt(f) + rLibPath <- readString(f) + connectionTimeout <- readInt(f) close(f) file.remove(path) if (length(backendPort) == 0 || backendPort == 0 || - length(monitorPort) == 0 || monitorPort == 0) { + length(monitorPort) == 0 || monitorPort == 0 || + length(rLibPath) != 1) { stop("JVM failed to launch") } - assign(".monitorConn", socketConnection(port = monitorPort), envir = .sparkREnv) + assign(".monitorConn", + socketConnection(port = monitorPort, timeout = connectionTimeout), + envir = .sparkREnv) assign(".backendLaunched", 1, envir = .sparkREnv) + if (rLibPath != "") { + assign(".libPath", rLibPath, envir = .sparkREnv) + .libPaths(c(rLibPath, .libPaths())) + } } .sparkREnv$backendPort <- backendPort tryCatch({ - connectBackend("localhost", backendPort) + connectBackend("localhost", backendPort, timeout = connectionTimeout) }, error = function(err) { stop("Failed to connect JVM\n") @@ -178,14 +218,19 @@ sparkR.init <- function( sparkHome <- suppressWarnings(normalizePath(sparkHome)) } - sparkExecutorEnvMap <- convertNamedListToEnv(sparkExecutorEnv) - if(is.null(sparkExecutorEnvMap$LD_LIBRARY_PATH)) { + if (is.null(sparkExecutorEnvMap$LD_LIBRARY_PATH)) { sparkExecutorEnvMap[["LD_LIBRARY_PATH"]] <- - paste0("$LD_LIBRARY_PATH:",Sys.getenv("LD_LIBRARY_PATH")) + paste0("$LD_LIBRARY_PATH:", Sys.getenv("LD_LIBRARY_PATH")) } - nonEmptyJars <- Filter(function(x) { x != "" }, jars) - localJarPaths <- lapply(nonEmptyJars, + # Classpath separator is ";" on Windows + # URI needs four /// as from http://stackoverflow.com/a/18522792 + if (.Platform$OS.type == "unix") { + uriSep <- "//" + } else { + uriSep <- "////" + } + localJarPaths <- lapply(jars, function(j) { utils::URLencode(paste("file:", uriSep, j, sep = "")) }) # Set the start time to identify jobjs @@ -214,116 +259,297 @@ sparkR.init <- function( sc } -#' Initialize a new SQLContext. +#' (Deprecated) Initialize a new SQLContext #' #' This function creates a SparkContext from an existing JavaSparkContext and #' then uses it to initialize a new SQLContext #' +#' Starting SparkR 2.0, a SparkSession is initialized and returned instead. +#' This API is deprecated and kept for backward compatibility only. +#' #' @param jsc The existing JavaSparkContext created with SparkR.init() +#' @seealso \link{sparkR.session} +#' @rdname sparkRSQL.init-deprecated #' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRSQL.init(sc) #'} - +#' @note sparkRSQL.init since 1.4.0 sparkRSQL.init <- function(jsc = NULL) { - if (exists(".sparkRSQLsc", envir = .sparkREnv)) { - return(get(".sparkRSQLsc", envir = .sparkREnv)) - } + .Deprecated("sparkR.session") - # If jsc is NULL, create a Spark Context - sc <- if (is.null(jsc)) { - sparkR.init() - } else { - jsc + if (exists(".sparkRsession", envir = .sparkREnv)) { + return(get(".sparkRsession", envir = .sparkREnv)) } - sqlContext <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", - "createSQLContext", - sc) - assign(".sparkRSQLsc", sqlContext, envir = .sparkREnv) - sqlContext + # Default to without Hive support for backward compatibility. + sparkR.session(enableHiveSupport = FALSE) } -#' Initialize a new HiveContext. +#' (Deprecated) Initialize a new HiveContext #' #' This function creates a HiveContext from an existing JavaSparkContext #' +#' Starting SparkR 2.0, a SparkSession is initialized and returned instead. +#' This API is deprecated and kept for backward compatibility only. +#' #' @param jsc The existing JavaSparkContext created with SparkR.init() +#' @seealso \link{sparkR.session} +#' @rdname sparkRHive.init-deprecated #' @export #' @examples #'\dontrun{ #' sc <- sparkR.init() #' sqlContext <- sparkRHive.init(sc) #'} - +#' @note sparkRHive.init since 1.4.0 sparkRHive.init <- function(jsc = NULL) { - if (exists(".sparkRHivesc", envir = .sparkREnv)) { - return(get(".sparkRHivesc", envir = .sparkREnv)) + .Deprecated("sparkR.session") + + if (exists(".sparkRsession", envir = .sparkREnv)) { + return(get(".sparkRsession", envir = .sparkREnv)) } - # If jsc is NULL, create a Spark Context - sc <- if (is.null(jsc)) { - sparkR.init() - } else { - jsc + # Default to without Hive support for backward compatibility. + sparkR.session(enableHiveSupport = TRUE) +} + +#' Get the existing SparkSession or initialize a new SparkSession. +#' +#' SparkSession is the entry point into SparkR. \code{sparkR.session} gets the existing +#' SparkSession or initializes a new SparkSession. +#' Additional Spark properties can be set in \code{...}, and these named parameters take priority +#' over values in \code{master}, \code{appName}, named lists of \code{sparkConfig}. +#' +#' When called in an interactive session, this method checks for the Spark installation, and, if not +#' found, it will be downloaded and cached automatically. Alternatively, \code{install.spark} can +#' be called manually. +#' +#' A default warehouse is created automatically in the current directory when a managed table is +#' created via \code{sql} statement \code{CREATE TABLE}, for example. To change the location of the +#' warehouse, set the named parameter \code{spark.sql.warehouse.dir} to the SparkSession. Along with +#' the warehouse, an accompanied metastore may also be automatically created in the current +#' directory when a new SparkSession is initialized with \code{enableHiveSupport} set to +#' \code{TRUE}, which is the default. For more details, refer to Hive configuration at +#' \url{http://spark.apache.org/docs/latest/sql-programming-guide.html#hive-tables}. +#' +#' For details on how to initialize and use SparkR, refer to SparkR programming guide at +#' \url{http://spark.apache.org/docs/latest/sparkr.html#starting-up-sparksession}. +#' +#' @param master the Spark master URL. +#' @param appName application name to register with cluster manager. +#' @param sparkHome Spark Home directory. +#' @param sparkConfig named list of Spark configuration to set on worker nodes. +#' @param sparkJars character vector of jar files to pass to the worker nodes. +#' @param sparkPackages character vector of package coordinates +#' @param enableHiveSupport enable support for Hive, fallback if not built with Hive support; once +#' set, this cannot be turned off on an existing session +#' @param ... named Spark properties passed to the method. +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df <- read.json(path) +#' +#' sparkR.session("local[2]", "SparkR", "/home/spark") +#' sparkR.session("yarn-client", "SparkR", "/home/spark", +#' list(spark.executor.memory="4g"), +#' c("one.jar", "two.jar", "three.jar"), +#' c("com.databricks:spark-avro_2.11:2.0.1")) +#' sparkR.session(spark.master = "yarn-client", spark.executor.memory = "4g") +#'} +#' @note sparkR.session since 2.0.0 +sparkR.session <- function( + master = "", + appName = "SparkR", + sparkHome = Sys.getenv("SPARK_HOME"), + sparkConfig = list(), + sparkJars = "", + sparkPackages = "", + enableHiveSupport = TRUE, + ...) { + + sparkConfigMap <- convertNamedListToEnv(sparkConfig) + namedParams <- list(...) + if (length(namedParams) > 0) { + paramMap <- convertNamedListToEnv(namedParams) + # Override for certain named parameters + if (exists("spark.master", envir = paramMap)) { + master <- paramMap[["spark.master"]] + } + if (exists("spark.app.name", envir = paramMap)) { + appName <- paramMap[["spark.app.name"]] + } + overrideEnvs(sparkConfigMap, paramMap) } - ssc <- callJMethod(sc, "sc") - hiveCtx <- tryCatch({ - newJObject("org.apache.spark.sql.hive.HiveContext", ssc) - }, - error = function(err) { - stop("Spark SQL is not built with Hive support") - }) + deployMode <- "" + if (exists("spark.submit.deployMode", envir = sparkConfigMap)) { + deployMode <- sparkConfigMap[["spark.submit.deployMode"]] + } - assign(".sparkRHivesc", hiveCtx, envir = .sparkREnv) - hiveCtx + if (!exists("spark.r.sql.derby.temp.dir", envir = sparkConfigMap)) { + sparkConfigMap[["spark.r.sql.derby.temp.dir"]] <- tempdir() + } + + if (!exists(".sparkRjsc", envir = .sparkREnv)) { + retHome <- sparkCheckInstall(sparkHome, master, deployMode) + if (!is.null(retHome)) sparkHome <- retHome + sparkExecutorEnvMap <- new.env() + sparkR.sparkContext(master, appName, sparkHome, sparkConfigMap, sparkExecutorEnvMap, + sparkJars, sparkPackages) + stopifnot(exists(".sparkRjsc", envir = .sparkREnv)) + } + + if (exists(".sparkRsession", envir = .sparkREnv)) { + sparkSession <- get(".sparkRsession", envir = .sparkREnv) + # Apply config to Spark Context and Spark Session if already there + # Cannot change enableHiveSupport + callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "setSparkContextSessionConf", + sparkSession, + sparkConfigMap) + } else { + jsc <- get(".sparkRjsc", envir = .sparkREnv) + sparkSession <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "getOrCreateSparkSession", + jsc, + sparkConfigMap, + enableHiveSupport) + assign(".sparkRsession", sparkSession, envir = .sparkREnv) + } + sparkSession +} + +#' Get the URL of the SparkUI instance for the current active SparkSession +#' +#' Get the URL of the SparkUI instance for the current active SparkSession. +#' +#' @return the SparkUI URL, or NA if it is disabled, or not started. +#' @rdname sparkR.uiWebUrl +#' @name sparkR.uiWebUrl +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' url <- sparkR.uiWebUrl() +#' } +#' @note sparkR.uiWebUrl since 2.1.1 +sparkR.uiWebUrl <- function() { + sc <- sparkR.callJMethod(getSparkContext(), "sc") + u <- callJMethod(sc, "uiWebUrl") + if (callJMethod(u, "isDefined")) { + callJMethod(u, "get") + } else { + NA + } } #' Assigns a group ID to all the jobs started by this thread until the group ID is set to a #' different value or cleared. #' -#' @param sc existing spark context -#' @param groupid the ID to be assigned to job groups -#' @param description description for the the job group ID -#' @param interruptOnCancel flag to indicate if the job is interrupted on job cancellation +#' @param groupId the ID to be assigned to job groups. +#' @param description description for the job group ID. +#' @param interruptOnCancel flag to indicate if the job is interrupted on job cancellation. +#' @rdname setJobGroup +#' @name setJobGroup #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' setJobGroup(sc, "myJobGroup", "My job group description", TRUE) +#' sparkR.session() +#' setJobGroup("myJobGroup", "My job group description", TRUE) #'} +#' @note setJobGroup since 1.5.0 +#' @method setJobGroup default +setJobGroup.default <- function(groupId, description, interruptOnCancel) { + sc <- getSparkContext() + invisible(callJMethod(sc, "setJobGroup", groupId, description, interruptOnCancel)) +} setJobGroup <- function(sc, groupId, description, interruptOnCancel) { - callJMethod(sc, "setJobGroup", groupId, description, interruptOnCancel) + if (class(sc) == "jobj" && any(grepl("JavaSparkContext", getClassName.jobj(sc)))) { + .Deprecated("setJobGroup(groupId, description, interruptOnCancel)", + old = "setJobGroup(sc, groupId, description, interruptOnCancel)") + setJobGroup.default(groupId, description, interruptOnCancel) + } else { + # Parameter order is shifted + groupIdToUse <- sc + descriptionToUse <- groupId + interruptOnCancelToUse <- description + setJobGroup.default(groupIdToUse, descriptionToUse, interruptOnCancelToUse) + } } #' Clear current job group ID and its description #' -#' @param sc existing spark context +#' @rdname clearJobGroup +#' @name clearJobGroup #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' clearJobGroup(sc) +#' sparkR.session() +#' clearJobGroup() #'} +#' @note clearJobGroup since 1.5.0 +#' @method clearJobGroup default +clearJobGroup.default <- function() { + sc <- getSparkContext() + invisible(callJMethod(sc, "clearJobGroup")) +} clearJobGroup <- function(sc) { - callJMethod(sc, "clearJobGroup") + if (!missing(sc) && + class(sc) == "jobj" && + any(grepl("JavaSparkContext", getClassName.jobj(sc)))) { + .Deprecated("clearJobGroup()", old = "clearJobGroup(sc)") + } + clearJobGroup.default() } + #' Cancel active jobs for the specified group #' -#' @param sc existing spark context #' @param groupId the ID of job group to be cancelled +#' @rdname cancelJobGroup +#' @name cancelJobGroup #' @examples #'\dontrun{ -#' sc <- sparkR.init() -#' cancelJobGroup(sc, "myJobGroup") +#' sparkR.session() +#' cancelJobGroup("myJobGroup") #'} +#' @note cancelJobGroup since 1.5.0 +#' @method cancelJobGroup default +cancelJobGroup.default <- function(groupId) { + sc <- getSparkContext() + invisible(callJMethod(sc, "cancelJobGroup", groupId)) +} cancelJobGroup <- function(sc, groupId) { - callJMethod(sc, "cancelJobGroup", groupId) + if (class(sc) == "jobj" && any(grepl("JavaSparkContext", getClassName.jobj(sc)))) { + .Deprecated("cancelJobGroup(groupId)", old = "cancelJobGroup(sc, groupId)") + cancelJobGroup.default(groupId) + } else { + # Parameter order is shifted + groupIdToUse <- sc + cancelJobGroup.default(groupIdToUse) + } +} + +#' Set a human readable description of the current job. +#' +#' Set a description that is shown as a job description in UI. +#' +#' @param value The job description of the current job. +#' @rdname setJobDescription +#' @name setJobDescription +#' @examples +#'\dontrun{ +#' setJobDescription("This is an example job.") +#'} +#' @note setJobDescription since 2.3.0 +setJobDescription <- function(value) { + sc <- getSparkContext() + invisible(callJMethod(sc, "setJobDescription", value)) } sparkConfToSubmitOps <- new.env() @@ -331,6 +557,10 @@ sparkConfToSubmitOps[["spark.driver.memory"]] <- "--driver-memory" sparkConfToSubmitOps[["spark.driver.extraClassPath"]] <- "--driver-class-path" sparkConfToSubmitOps[["spark.driver.extraJavaOptions"]] <- "--driver-java-options" sparkConfToSubmitOps[["spark.driver.extraLibraryPath"]] <- "--driver-library-path" +sparkConfToSubmitOps[["spark.master"]] <- "--master" +sparkConfToSubmitOps[["spark.yarn.keytab"]] <- "--keytab" +sparkConfToSubmitOps[["spark.yarn.principal"]] <- "--principal" + # Utility function that returns Spark Submit arguments as a string # @@ -355,3 +585,55 @@ getClientModeSparkSubmitOpts <- function(submitOps, sparkEnvirMap) { # --option must be before the application class "sparkr-shell" in submitOps paste0(paste0(envirToOps, collapse = ""), submitOps) } + +# Utility function that handles sparkJars argument, and normalize paths +processSparkJars <- function(jars) { + splittedJars <- splitString(jars) + if (length(splittedJars) > length(jars)) { + warning("sparkJars as a comma-separated string is deprecated, use character vector instead") + } + normalized <- suppressWarnings(normalizePath(splittedJars)) + normalized +} + +# Utility function that handles sparkPackages argument +processSparkPackages <- function(packages) { + splittedPackages <- splitString(packages) + if (length(splittedPackages) > length(packages)) { + warning("sparkPackages as a comma-separated string is deprecated, use character vector instead") + } + splittedPackages +} + +# Utility function that checks and install Spark to local folder if not found +# +# Installation will not be triggered if it's called from sparkR shell +# or if the master url is not local +# +# @param sparkHome directory to find Spark package. +# @param master the Spark master URL, used to check local or remote mode. +# @param deployMode whether to deploy your driver on the worker nodes (cluster) +# or locally as an external client (client). +# @return NULL if no need to update sparkHome, and new sparkHome otherwise. +sparkCheckInstall <- function(sparkHome, master, deployMode) { + if (!isSparkRShell()) { + if (!is.na(file.info(sparkHome)$isdir)) { + message("Spark package found in SPARK_HOME: ", sparkHome) + NULL + } else { + if (interactive() || isMasterLocal(master)) { + message("Spark not found in SPARK_HOME: ", sparkHome) + packageLocalDir <- install.spark() + packageLocalDir + } else if (isClientMode(master) || deployMode == "client") { + msg <- paste0("Spark not found in SPARK_HOME: ", + sparkHome, "\n", installInstruction("remote")) + stop(msg) + } else { + NULL + } + } + } else { + NULL + } +} diff --git a/R/pkg/R/stats.R b/R/pkg/R/stats.R index f79329b11540..9a9fa84044ce 100644 --- a/R/pkg/R/stats.R +++ b/R/pkg/R/stats.R @@ -15,140 +15,205 @@ # limitations under the License. # -# stats.R - Statistic functions for DataFrames. +# stats.R - Statistic functions for SparkDataFrames. setOldClass("jobj") -#' crosstab +#' Computes a pair-wise frequency table of the given columns #' #' Computes a pair-wise frequency table of the given columns. Also known as a contingency #' table. The number of distinct values for each column should be less than 1e4. At most 1e6 #' non-zero pair frequencies will be returned. #' +#' @param x a SparkDataFrame #' @param col1 name of the first column. Distinct items will make the first item of each row. #' @param col2 name of the second column. Distinct items will make the column names of the output. #' @return a local R data.frame representing the contingency table. The first column of each row -#' will be the distinct values of `col1` and the column names will be the distinct values -#' of `col2`. The name of the first column will be `$col1_$col2`. Pairs that have no -#' occurrences will have zero as their counts. +#' will be the distinct values of \code{col1} and the column names will be the distinct values +#' of \code{col2}. The name of the first column will be "\code{col1}_\code{col2}". Pairs +#' that have no occurrences will have zero as their counts. #' -#' @rdname statfunctions +#' @rdname crosstab #' @name crosstab +#' @aliases crosstab,SparkDataFrame,character,character-method +#' @family stat functions #' @export #' @examples #' \dontrun{ -#' df <- jsonFile(sqlContext, "/path/to/file.json") +#' df <- read.json("/path/to/file.json") #' ct <- crosstab(df, "title", "gender") #' } +#' @note crosstab since 1.5.0 setMethod("crosstab", - signature(x = "DataFrame", col1 = "character", col2 = "character"), + signature(x = "SparkDataFrame", col1 = "character", col2 = "character"), function(x, col1, col2) { statFunctions <- callJMethod(x@sdf, "stat") sct <- callJMethod(statFunctions, "crosstab", col1, col2) collect(dataFrame(sct)) }) -#' cov +#' @details +#' \code{cov}: When applied to SparkDataFrame, this calculates the sample covariance of two numerical +#' columns of \emph{one} SparkDataFrame. #' -#' Calculate the sample covariance of two numerical columns of a DataFrame. +#' @param colName1 the name of the first column +#' @param colName2 the name of the second column +#' @return The covariance of the two columns. #' -#' @param x A SparkSQL DataFrame -#' @param col1 the name of the first column -#' @param col2 the name of the second column -#' @return the covariance of the two columns. -#' -#' @rdname statfunctions -#' @name cov +#' @rdname cov +#' @aliases cov,SparkDataFrame-method +#' @family stat functions #' @export #' @examples -#'\dontrun{ -#' df <- jsonFile(sqlContext, "/path/to/file.json") -#' cov <- cov(df, "title", "gender") -#' } +#' +#' \dontrun{ +#' cov(df, "mpg", "hp") +#' cov(df, df$mpg, df$hp)} +#' @note cov since 1.6.0 setMethod("cov", - signature(x = "DataFrame", col1 = "character", col2 = "character"), - function(x, col1, col2) { + signature(x = "SparkDataFrame"), + function(x, colName1, colName2) { + stopifnot(class(colName1) == "character" && class(colName2) == "character") statFunctions <- callJMethod(x@sdf, "stat") - callJMethod(statFunctions, "cov", col1, col2) + callJMethod(statFunctions, "cov", colName1, colName2) }) -#' corr -#' -#' Calculates the correlation of two columns of a DataFrame. +#' Calculates the correlation of two columns of a SparkDataFrame. #' Currently only supports the Pearson Correlation Coefficient. #' For Spearman Correlation, consider using RDD methods found in MLlib's Statistics. -#' -#' @param x A SparkSQL DataFrame -#' @param col1 the name of the first column -#' @param col2 the name of the second column +#' +#' @param colName1 the name of the first column +#' @param colName2 the name of the second column #' @param method Optional. A character specifying the method for calculating the correlation. #' only "pearson" is allowed now. #' @return The Pearson Correlation Coefficient as a Double. #' -#' @rdname statfunctions +#' @rdname corr #' @name corr +#' @aliases corr,SparkDataFrame-method +#' @family stat functions #' @export #' @examples -#'\dontrun{ -#' df <- jsonFile(sqlContext, "/path/to/file.json") -#' corr <- corr(df, "title", "gender") -#' corr <- corr(df, "title", "gender", method = "pearson") -#' } +#' +#' \dontrun{ +#' corr(df, "mpg", "hp") +#' corr(df, "mpg", "hp", method = "pearson")} +#' @note corr since 1.6.0 setMethod("corr", - signature(x = "DataFrame", col1 = "character", col2 = "character"), - function(x, col1, col2, method = "pearson") { + signature(x = "SparkDataFrame"), + function(x, colName1, colName2, method = "pearson") { + stopifnot(class(colName1) == "character" && class(colName2) == "character") statFunctions <- callJMethod(x@sdf, "stat") - callJMethod(statFunctions, "corr", col1, col2, method) + callJMethod(statFunctions, "corr", colName1, colName2, method) }) -#' freqItems + +#' Finding frequent items for columns, possibly with false positives #' #' Finding frequent items for columns, possibly with false positives. #' Using the frequent element count algorithm described in #' \url{http://dx.doi.org/10.1145/762471.762473}, proposed by Karp, Schenker, and Papadimitriou. #' -#' @param x A SparkSQL DataFrame. +#' @param x A SparkDataFrame. #' @param cols A vector column names to search frequent items in. -#' @param support (Optional) The minimum frequency for an item to be considered `frequent`. +#' @param support (Optional) The minimum frequency for an item to be considered \code{frequent}. #' Should be greater than 1e-4. Default support = 0.01. #' @return a local R data.frame with the frequent items in each column #' -#' @rdname statfunctions +#' @rdname freqItems #' @name freqItems +#' @aliases freqItems,SparkDataFrame,character-method +#' @family stat functions #' @export #' @examples #' \dontrun{ -#' df <- jsonFile(sqlContext, "/path/to/file.json") +#' df <- read.json("/path/to/file.json") #' fi = freqItems(df, c("title", "gender")) #' } -setMethod("freqItems", signature(x = "DataFrame", cols = "character"), +#' @note freqItems since 1.6.0 +setMethod("freqItems", signature(x = "SparkDataFrame", cols = "character"), function(x, cols, support = 0.01) { statFunctions <- callJMethod(x@sdf, "stat") sct <- callJMethod(statFunctions, "freqItems", as.list(cols), support) collect(dataFrame(sct)) }) -#' sampleBy +#' Calculates the approximate quantiles of numerical columns of a SparkDataFrame +#' +#' Calculates the approximate quantiles of numerical columns of a SparkDataFrame. +#' The result of this algorithm has the following deterministic bound: +#' If the SparkDataFrame has N elements and if we request the quantile at probability p up to +#' error err, then the algorithm will return a sample x from the SparkDataFrame so that the +#' *exact* rank of x is close to (p * N). More precisely, +#' floor((p - err) * N) <= rank(x) <= ceil((p + err) * N). +#' This method implements a variation of the Greenwald-Khanna algorithm (with some speed +#' optimizations). The algorithm was first present in [[http://dx.doi.org/10.1145/375663.375670 +#' Space-efficient Online Computation of Quantile Summaries]] by Greenwald and Khanna. +#' Note that NA values will be ignored in numerical columns before calculation. For +#' columns only containing NA values, an empty list is returned. +#' +#' @param x A SparkDataFrame. +#' @param cols A single column name, or a list of names for multiple columns. +#' @param probabilities A list of quantile probabilities. Each number must belong to [0, 1]. +#' For example 0 is the minimum, 0.5 is the median, 1 is the maximum. +#' @param relativeError The relative target precision to achieve (>= 0). If set to zero, +#' the exact quantiles are computed, which could be very expensive. +#' Note that values greater than 1 are accepted but give the same result as 1. +#' @return The approximate quantiles at the given probabilities. If the input is a single column name, +#' the output is a list of approximate quantiles in that column; If the input is +#' multiple column names, the output should be a list, and each element in it is a list of +#' numeric values which represents the approximate quantiles in corresponding column. +#' +#' @rdname approxQuantile +#' @name approxQuantile +#' @aliases approxQuantile,SparkDataFrame,character,numeric,numeric-method +#' @family stat functions +#' @export +#' @examples +#' \dontrun{ +#' df <- read.json("/path/to/file.json") +#' quantiles <- approxQuantile(df, "key", c(0.5, 0.8), 0.0) +#' } +#' @note approxQuantile since 2.0.0 +setMethod("approxQuantile", + signature(x = "SparkDataFrame", cols = "character", + probabilities = "numeric", relativeError = "numeric"), + function(x, cols, probabilities, relativeError) { + statFunctions <- callJMethod(x@sdf, "stat") + quantiles <- callJMethod(statFunctions, "approxQuantile", as.list(cols), + as.list(probabilities), relativeError) + if (length(cols) == 1) { + quantiles[[1]] + } else { + quantiles + } + }) + +#' Returns a stratified sample without replacement +#' +#' Returns a stratified sample without replacement based on the fraction given on each +#' stratum. #' -#' Returns a stratified sample without replacement based on the fraction given on each stratum. -#' -#' @param x A SparkSQL DataFrame +#' @param x A SparkDataFrame #' @param col column that defines strata #' @param fractions A named list giving sampling fraction for each stratum. If a stratum is #' not specified, we treat its fraction as zero. #' @param seed random seed -#' @return A new DataFrame that represents the stratified sample +#' @return A new SparkDataFrame that represents the stratified sample #' -#' @rdname statfunctions +#' @rdname sampleBy +#' @aliases sampleBy,SparkDataFrame,character,list,numeric-method #' @name sampleBy +#' @family stat functions #' @export #' @examples #'\dontrun{ -#' df <- jsonFile(sqlContext, "/path/to/file.json") +#' df <- read.json("/path/to/file.json") #' sample <- sampleBy(df, "key", fractions, 36) #' } +#' @note sampleBy since 1.6.0 setMethod("sampleBy", - signature(x = "DataFrame", col = "character", + signature(x = "SparkDataFrame", col = "character", fractions = "list", seed = "numeric"), function(x, col, fractions, seed) { fractionsEnv <- convertNamedListToEnv(fractions) diff --git a/R/pkg/R/streaming.R b/R/pkg/R/streaming.R new file mode 100644 index 000000000000..8390bd5e6de7 --- /dev/null +++ b/R/pkg/R/streaming.R @@ -0,0 +1,214 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# streaming.R - Structured Streaming / StreamingQuery class and methods implemented in S4 OO classes + +#' @include generics.R jobj.R +NULL + +#' S4 class that represents a StreamingQuery +#' +#' StreamingQuery can be created by using read.stream() and write.stream() +#' +#' @rdname StreamingQuery +#' @seealso \link{read.stream} +#' +#' @param ssq A Java object reference to the backing Scala StreamingQuery +#' @export +#' @note StreamingQuery since 2.2.0 +#' @note experimental +setClass("StreamingQuery", + slots = list(ssq = "jobj")) + +setMethod("initialize", "StreamingQuery", function(.Object, ssq) { + .Object@ssq <- ssq + .Object +}) + +streamingQuery <- function(ssq) { + stopifnot(class(ssq) == "jobj") + new("StreamingQuery", ssq) +} + +#' @rdname show +#' @export +#' @note show(StreamingQuery) since 2.2.0 +setMethod("show", "StreamingQuery", + function(object) { + name <- callJMethod(object@ssq, "name") + if (!is.null(name)) { + cat(paste0("StreamingQuery '", name, "'\n")) + } else { + cat("StreamingQuery", "\n") + } + }) + +#' queryName +#' +#' Returns the user-specified name of the query. This is specified in +#' \code{write.stream(df, queryName = "query")}. This name, if set, must be unique across all active +#' queries. +#' +#' @param x a StreamingQuery. +#' @return The name of the query, or NULL if not specified. +#' @rdname queryName +#' @name queryName +#' @aliases queryName,StreamingQuery-method +#' @family StreamingQuery methods +#' @seealso \link{write.stream} +#' @export +#' @examples +#' \dontrun{ queryName(sq) } +#' @note queryName(StreamingQuery) since 2.2.0 +#' @note experimental +setMethod("queryName", + signature(x = "StreamingQuery"), + function(x) { + callJMethod(x@ssq, "name") + }) + +#' @rdname explain +#' @name explain +#' @aliases explain,StreamingQuery-method +#' @family StreamingQuery methods +#' @export +#' @examples +#' \dontrun{ explain(sq) } +#' @note explain(StreamingQuery) since 2.2.0 +setMethod("explain", + signature(x = "StreamingQuery"), + function(x, extended = FALSE) { + cat(callJMethod(x@ssq, "explainInternal", extended), "\n") + }) + +#' lastProgress +#' +#' Prints the most recent progess update of this streaming query in JSON format. +#' +#' @param x a StreamingQuery. +#' @rdname lastProgress +#' @name lastProgress +#' @aliases lastProgress,StreamingQuery-method +#' @family StreamingQuery methods +#' @export +#' @examples +#' \dontrun{ lastProgress(sq) } +#' @note lastProgress(StreamingQuery) since 2.2.0 +#' @note experimental +setMethod("lastProgress", + signature(x = "StreamingQuery"), + function(x) { + p <- callJMethod(x@ssq, "lastProgress") + if (is.null(p)) { + cat("Streaming query has no progress") + } else { + cat(callJMethod(p, "toString"), "\n") + } + }) + +#' status +#' +#' Prints the current status of the query in JSON format. +#' +#' @param x a StreamingQuery. +#' @rdname status +#' @name status +#' @aliases status,StreamingQuery-method +#' @family StreamingQuery methods +#' @export +#' @examples +#' \dontrun{ status(sq) } +#' @note status(StreamingQuery) since 2.2.0 +#' @note experimental +setMethod("status", + signature(x = "StreamingQuery"), + function(x) { + cat(callJMethod(callJMethod(x@ssq, "status"), "toString"), "\n") + }) + +#' isActive +#' +#' Returns TRUE if this query is actively running. +#' +#' @param x a StreamingQuery. +#' @return TRUE if query is actively running, FALSE if stopped. +#' @rdname isActive +#' @name isActive +#' @aliases isActive,StreamingQuery-method +#' @family StreamingQuery methods +#' @export +#' @examples +#' \dontrun{ isActive(sq) } +#' @note isActive(StreamingQuery) since 2.2.0 +#' @note experimental +setMethod("isActive", + signature(x = "StreamingQuery"), + function(x) { + callJMethod(x@ssq, "isActive") + }) + +#' awaitTermination +#' +#' Waits for the termination of the query, either by \code{stopQuery} or by an error. +#' +#' If the query has terminated, then all subsequent calls to this method will return TRUE +#' immediately. +#' +#' @param x a StreamingQuery. +#' @param timeout time to wait in milliseconds, if omitted, wait indefinitely until \code{stopQuery} +#' is called or an error has occured. +#' @return TRUE if query has terminated within the timeout period; nothing if timeout is not +#' specified. +#' @rdname awaitTermination +#' @name awaitTermination +#' @aliases awaitTermination,StreamingQuery-method +#' @family StreamingQuery methods +#' @export +#' @examples +#' \dontrun{ awaitTermination(sq, 10000) } +#' @note awaitTermination(StreamingQuery) since 2.2.0 +#' @note experimental +setMethod("awaitTermination", + signature(x = "StreamingQuery"), + function(x, timeout = NULL) { + if (is.null(timeout)) { + invisible(handledCallJMethod(x@ssq, "awaitTermination")) + } else { + handledCallJMethod(x@ssq, "awaitTermination", as.integer(timeout)) + } + }) + +#' stopQuery +#' +#' Stops the execution of this query if it is running. This method blocks until the execution is +#' stopped. +#' +#' @param x a StreamingQuery. +#' @rdname stopQuery +#' @name stopQuery +#' @aliases stopQuery,StreamingQuery-method +#' @family StreamingQuery methods +#' @export +#' @examples +#' \dontrun{ stopQuery(sq) } +#' @note stopQuery(StreamingQuery) since 2.2.0 +#' @note experimental +setMethod("stopQuery", + signature(x = "StreamingQuery"), + function(x) { + invisible(callJMethod(x@ssq, "stop")) + }) diff --git a/R/pkg/R/types.R b/R/pkg/R/types.R new file mode 100644 index 000000000000..ade0f05c0254 --- /dev/null +++ b/R/pkg/R/types.R @@ -0,0 +1,85 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# types.R. This file handles the data type mapping between Spark and R + +# The primitive data types, where names(PRIMITIVE_TYPES) are Scala types whereas +# values are equivalent R types. This is stored in an environment to allow for +# more efficient look up (environments use hashmaps). +PRIMITIVE_TYPES <- as.environment(list( + "tinyint" = "integer", + "smallint" = "integer", + "int" = "integer", + "bigint" = "numeric", + "float" = "numeric", + "double" = "numeric", + "decimal" = "numeric", + "string" = "character", + "binary" = "raw", + "boolean" = "logical", + "timestamp" = c("POSIXct", "POSIXt"), + "date" = "Date", + # following types are not SQL types returned by dtypes(). They are listed here for usage + # by checkType() in schema.R. + # TODO: refactor checkType() in schema.R. + "byte" = "integer", + "integer" = "integer" + )) + +# The complex data types. These do not have any direct mapping to R's types. +COMPLEX_TYPES <- list( + "map" = NA, + "array" = NA, + "struct" = NA) + +# The full list of data types. +DATA_TYPES <- as.environment(c(as.list(PRIMITIVE_TYPES), COMPLEX_TYPES)) + +SHORT_TYPES <- as.environment(list( + "character" = "chr", + "logical" = "logi", + "POSIXct" = "POSIXct", + "integer" = "int", + "numeric" = "num", + "raw" = "raw", + "Date" = "Date", + "map" = "map", + "array" = "array", + "struct" = "struct" +)) + +# An environment for mapping R to Scala, names are R types and values are Scala types. +rToSQLTypes <- as.environment(list( + "integer" = "integer", # in R, integer is 32bit + "numeric" = "double", # in R, numeric == double which is 64bit + "double" = "double", + "character" = "string", + "logical" = "boolean")) + +# Helper function of coverting decimal type. When backend returns column type in the +# format of decimal(,) (e.g., decimal(10, 0)), this function coverts the column type +# as double type. This function converts backend returned types that are not the key +# of PRIMITIVE_TYPES, but should be treated as PRIMITIVE_TYPES. +# @param A type returned from the JVM backend. +# @return A type is the key of the PRIMITIVE_TYPES. +specialtypeshandle <- function(type) { + returntype <- NULL + m <- regexec("^decimal(.+)$", type) + matchedStrings <- regmatches(type, m) + if (length(matchedStrings[[1]]) >= 2) { + returntype <- "double" + } + returntype +} diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index 0b9e2957fe9a..91483a4d23d9 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -110,9 +110,12 @@ isRDD <- function(name, env) { #' @return the hash code as an integer #' @export #' @examples +#'\dontrun{ #' hashCode(1L) # 1 #' hashCode(1.0) # 1072693248 #' hashCode("1") # 49 +#'} +#' @note hashCode since 1.4.0 hashCode <- function(key) { if (class(key) == "integer") { as.integer(key[[1]]) @@ -123,20 +126,16 @@ hashCode <- function(key) { as.integer(bitwXor(intBits[2], intBits[1])) } else if (class(key) == "character") { # TODO: SPARK-7839 means we might not have the native library available - if (is.loaded("stringHashCode")) { - .Call("stringHashCode", key) + n <- nchar(key) + if (n == 0) { + 0L } else { - n <- nchar(key) - if (n == 0) { - 0L - } else { - asciiVals <- sapply(charToRaw(key), function(x) { strtoi(x, 16L) }) - hashC <- 0 - for (k in 1:length(asciiVals)) { - hashC <- mult31AndAdd(hashC, asciiVals[k]) - } - as.integer(hashC) + asciiVals <- sapply(charToRaw(key), function(x) { strtoi(x, 16L) }) + hashC <- 0 + for (k in 1:length(asciiVals)) { + hashC <- mult31AndAdd(hashC, asciiVals[k]) } + as.integer(hashC) } } else { warning(paste("Could not hash object, returning 0", sep = "")) @@ -157,8 +156,11 @@ wrapInt <- function(value) { # Multiply `val` by 31 and add `addVal` to the result. Ensures that # integer-overflows are handled at every step. +# +# TODO: this function does not handle integer overflow well mult31AndAdd <- function(val, addVal) { - vec <- c(bitwShiftL(val, c(4,3,2,1,0)), addVal) + vec <- c(bitwShiftL(val, c(4, 3, 2, 1, 0)), addVal) + vec[is.na(vec)] <- 0 Reduce(function(a, b) { wrapInt(as.numeric(a) + as.numeric(b)) }, @@ -202,7 +204,7 @@ serializeToString <- function(rdd) { # This function amortizes the allocation cost by doubling # the size of the list every time it fills up. addItemToAccumulator <- function(acc, item) { - if(acc$counter == acc$size) { + if (acc$counter == acc$size) { acc$size <- acc$size * 2 length(acc$data) <- acc$size } @@ -312,6 +314,15 @@ convertEnvsToList <- function(keys, vals) { }) } +# Utility function to merge 2 environments with the second overriding values in the first +# env1 is changed in place +overrideEnvs <- function(env1, env2) { + lapply(ls(env2), + function(name) { + env1[[name]] <- env2[[name]] + }) +} + # Utility function to capture the varargs into environment object varargsToEnv <- function(...) { # Based on http://stackoverflow.com/a/3057419/4577954 @@ -323,6 +334,48 @@ varargsToEnv <- function(...) { env } +# Utility function to capture the varargs into environment object but all values are converted +# into string. +varargsToStrEnv <- function(...) { + pairs <- list(...) + nameList <- names(pairs) + env <- new.env() + ignoredNames <- list() + + if (is.null(nameList)) { + # When all arguments are not named, names(..) returns NULL. + ignoredNames <- pairs + } else { + for (i in seq_along(pairs)) { + name <- nameList[i] + value <- pairs[i] + if (identical(name, "")) { + # When some of arguments are not named, name is "". + ignoredNames <- append(ignoredNames, value) + } else { + value <- pairs[[name]] + if (!(is.logical(value) || is.numeric(value) || is.character(value) || is.null(value))) { + stop(paste0("Unsupported type for ", name, " : ", class(value), + ". Supported types are logical, numeric, character and NULL."), call. = FALSE) + } + if (is.logical(value)) { + env[[name]] <- tolower(as.character(value)) + } else if (is.null(value)) { + env[[name]] <- value + } else { + env[[name]] <- as.character(value) + } + } + } + } + + if (length(ignoredNames) != 0) { + warning(paste0("Unnamed arguments ignored: ", paste(ignoredNames, collapse = ", "), "."), + call. = FALSE) + } + env +} + getStorageLevel <- function(newLevel = c("DISK_ONLY", "DISK_ONLY_2", "MEMORY_AND_DISK", @@ -352,6 +405,47 @@ getStorageLevel <- function(newLevel = c("DISK_ONLY", "OFF_HEAP" = callJStatic(storageLevelClass, "OFF_HEAP")) } +storageLevelToString <- function(levelObj) { + useDisk <- callJMethod(levelObj, "useDisk") + useMemory <- callJMethod(levelObj, "useMemory") + useOffHeap <- callJMethod(levelObj, "useOffHeap") + deserialized <- callJMethod(levelObj, "deserialized") + replication <- callJMethod(levelObj, "replication") + shortName <- if (!useDisk && !useMemory && !useOffHeap && !deserialized && replication == 1) { + "NONE" + } else if (useDisk && !useMemory && !useOffHeap && !deserialized && replication == 1) { + "DISK_ONLY" + } else if (useDisk && !useMemory && !useOffHeap && !deserialized && replication == 2) { + "DISK_ONLY_2" + } else if (!useDisk && useMemory && !useOffHeap && deserialized && replication == 1) { + "MEMORY_ONLY" + } else if (!useDisk && useMemory && !useOffHeap && deserialized && replication == 2) { + "MEMORY_ONLY_2" + } else if (!useDisk && useMemory && !useOffHeap && !deserialized && replication == 1) { + "MEMORY_ONLY_SER" + } else if (!useDisk && useMemory && !useOffHeap && !deserialized && replication == 2) { + "MEMORY_ONLY_SER_2" + } else if (useDisk && useMemory && !useOffHeap && deserialized && replication == 1) { + "MEMORY_AND_DISK" + } else if (useDisk && useMemory && !useOffHeap && deserialized && replication == 2) { + "MEMORY_AND_DISK_2" + } else if (useDisk && useMemory && !useOffHeap && !deserialized && replication == 1) { + "MEMORY_AND_DISK_SER" + } else if (useDisk && useMemory && !useOffHeap && !deserialized && replication == 2) { + "MEMORY_AND_DISK_SER_2" + } else if (useDisk && useMemory && useOffHeap && !deserialized && replication == 1) { + "OFF_HEAP" + } else { + NULL + } + fullInfo <- callJMethod(levelObj, "toString") + if (is.null(shortName)) { + fullInfo + } else { + paste(shortName, "-", fullInfo) + } +} + # Utility function for functions where an argument needs to be integer but we want to allow # the user to type (for example) `5` instead of `5L` to avoid a confusing error message. numToInt <- function(num) { @@ -486,7 +580,7 @@ processClosure <- function(node, oldEnv, defVars, checkedFuncs, newEnv) { # checkedFunc An environment of function objects examined during cleanClosure. It can be # considered as a "name"-to-"list of functions" mapping. # return value -# a new version of func that has an correct environment (closure). +# a new version of func that has a correct environment (closure). cleanClosure <- function(func, checkedFuncs = new.env()) { if (is.function(func)) { newEnv <- new.env(parent = .GlobalEnv) @@ -623,3 +717,205 @@ convertNamedListToEnv <- function(namedList) { } env } + +# Assign a new environment for attach() and with() methods +assignNewEnv <- function(data) { + stopifnot(class(data) == "SparkDataFrame") + cols <- columns(data) + stopifnot(length(cols) > 0) + + env <- new.env() + for (i in 1:length(cols)) { + assign(x = cols[i], value = data[, cols[i], drop = F], envir = env) + } + env +} + +# Utility function to split by ',' and whitespace, remove empty tokens +splitString <- function(input) { + Filter(nzchar, unlist(strsplit(input, ",|\\s"))) +} + +convertToJSaveMode <- function(mode) { + allModes <- c("append", "overwrite", "error", "ignore") + if (!(mode %in% allModes)) { + stop('mode should be one of "append", "overwrite", "error", "ignore"') # nolint + } + jmode <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "saveMode", mode) + jmode +} + +varargsToJProperties <- function(...) { + pairs <- list(...) + props <- newJObject("java.util.Properties") + if (length(pairs) > 0) { + lapply(ls(pairs), function(k) { + callJMethod(props, "setProperty", as.character(k), as.character(pairs[[k]])) + }) + } + props +} + +launchScript <- function(script, combinedArgs, wait = FALSE) { + if (.Platform$OS.type == "windows") { + scriptWithArgs <- paste(script, combinedArgs, sep = " ") + # on Windows, intern = F seems to mean output to the console. (documentation on this is missing) + shell(scriptWithArgs, translate = TRUE, wait = wait, intern = wait) # nolint + } else { + # http://stat.ethz.ch/R-manual/R-devel/library/base/html/system2.html + # stdout = F means discard output + # stdout = "" means to its console (default) + # Note that the console of this child process might not be the same as the running R process. + system2(script, combinedArgs, stdout = "", wait = wait) + } +} + +getSparkContext <- function() { + if (!exists(".sparkRjsc", envir = .sparkREnv)) { + stop("SparkR has not been initialized. Please call sparkR.session()") + } + sc <- get(".sparkRjsc", envir = .sparkREnv) + sc +} + +isMasterLocal <- function(master) { + grepl("^local(\\[([0-9]+|\\*)\\])?$", master, perl = TRUE) +} + +isClientMode <- function(master) { + grepl("([a-z]+)-client$", master, perl = TRUE) +} + +isSparkRShell <- function() { + grepl(".*shell\\.R$", Sys.getenv("R_PROFILE_USER"), perl = TRUE) +} + +# Works identically with `callJStatic(...)` but throws a pretty formatted exception. +handledCallJStatic <- function(cls, method, ...) { + result <- tryCatch(callJStatic(cls, method, ...), + error = function(e) { + captureJVMException(e, method) + }) + result +} + +# Works identically with `callJMethod(...)` but throws a pretty formatted exception. +handledCallJMethod <- function(obj, method, ...) { + result <- tryCatch(callJMethod(obj, method, ...), + error = function(e) { + captureJVMException(e, method) + }) + result +} + +captureJVMException <- function(e, method) { + rawmsg <- as.character(e) + if (any(grep("^Error in .*?: ", rawmsg))) { + # If the exception message starts with "Error in ...", this is possibly + # "Error in invokeJava(...)". Here, it replaces the characters to + # `paste("Error in", method, ":")` in order to identify which function + # was called in JVM side. + stacktrace <- strsplit(rawmsg, "Error in .*?: ")[[1]] + rmsg <- paste("Error in", method, ":") + stacktrace <- paste(rmsg[1], stacktrace[2]) + } else { + # Otherwise, do not convert the error message just in case. + stacktrace <- rawmsg + } + + # StreamingQueryException could wrap an IllegalArgumentException, so look for that first + if (any(grep("org.apache.spark.sql.streaming.StreamingQueryException: ", stacktrace))) { + msg <- strsplit(stacktrace, "org.apache.spark.sql.streaming.StreamingQueryException: ", + fixed = TRUE)[[1]] + # Extract "Error in ..." message. + rmsg <- msg[1] + # Extract the first message of JVM exception. + first <- strsplit(msg[2], "\r?\n\tat")[[1]][1] + stop(paste0(rmsg, "streaming query error - ", first), call. = FALSE) + } else if (any(grep("java.lang.IllegalArgumentException: ", stacktrace))) { + msg <- strsplit(stacktrace, "java.lang.IllegalArgumentException: ", fixed = TRUE)[[1]] + # Extract "Error in ..." message. + rmsg <- msg[1] + # Extract the first message of JVM exception. + first <- strsplit(msg[2], "\r?\n\tat")[[1]][1] + stop(paste0(rmsg, "illegal argument - ", first), call. = FALSE) + } else if (any(grep("org.apache.spark.sql.AnalysisException: ", stacktrace))) { + msg <- strsplit(stacktrace, "org.apache.spark.sql.AnalysisException: ", fixed = TRUE)[[1]] + # Extract "Error in ..." message. + rmsg <- msg[1] + # Extract the first message of JVM exception. + first <- strsplit(msg[2], "\r?\n\tat")[[1]][1] + stop(paste0(rmsg, "analysis error - ", first), call. = FALSE) + } else + if (any(grep("org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: ", stacktrace))) { + msg <- strsplit(stacktrace, "org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: ", + fixed = TRUE)[[1]] + # Extract "Error in ..." message. + rmsg <- msg[1] + # Extract the first message of JVM exception. + first <- strsplit(msg[2], "\r?\n\tat")[[1]][1] + stop(paste0(rmsg, "no such database - ", first), call. = FALSE) + } else + if (any(grep("org.apache.spark.sql.catalyst.analysis.NoSuchTableException: ", stacktrace))) { + msg <- strsplit(stacktrace, "org.apache.spark.sql.catalyst.analysis.NoSuchTableException: ", + fixed = TRUE)[[1]] + # Extract "Error in ..." message. + rmsg <- msg[1] + # Extract the first message of JVM exception. + first <- strsplit(msg[2], "\r?\n\tat")[[1]][1] + stop(paste0(rmsg, "no such table - ", first), call. = FALSE) + } else if (any(grep("org.apache.spark.sql.catalyst.parser.ParseException: ", stacktrace))) { + msg <- strsplit(stacktrace, "org.apache.spark.sql.catalyst.parser.ParseException: ", + fixed = TRUE)[[1]] + # Extract "Error in ..." message. + rmsg <- msg[1] + # Extract the first message of JVM exception. + first <- strsplit(msg[2], "\r?\n\tat")[[1]][1] + stop(paste0(rmsg, "parse error - ", first), call. = FALSE) + } else { + stop(stacktrace, call. = FALSE) + } +} + +# rbind a list of rows with raw (binary) columns +# +# @param inputData a list of rows, with each row a list +# @return data.frame with raw columns as lists +rbindRaws <- function(inputData) { + row1 <- inputData[[1]] + rawcolumns <- ("raw" == sapply(row1, class)) + + listmatrix <- do.call(rbind, inputData) + # A dataframe with all list columns + out <- as.data.frame(listmatrix) + out[!rawcolumns] <- lapply(out[!rawcolumns], unlist) + out +} + +# Get basename without extension from URL +basenameSansExtFromUrl <- function(url) { + # split by '/' + splits <- unlist(strsplit(url, "^.+/")) + last <- tail(splits, 1) + # this is from file_path_sans_ext + # first, remove any compression extension + filename <- sub("[.](gz|bz2|xz)$", "", last) + # then, strip extension by the last '.' + sub("([^.]+)\\.[[:alnum:]]+$", "\\1", filename) +} + +isAtomicLengthOne <- function(x) { + is.atomic(x) && length(x) == 1 +} + +is_windows <- function() { + .Platform$OS.type == "windows" +} + +hadoop_home_set <- function() { + !identical(Sys.getenv("HADOOP_HOME"), "") +} + +windows_with_hadoop <- function() { + !is_windows() || hadoop_home_set() +} diff --git a/R/pkg/R/window.R b/R/pkg/R/window.R new file mode 100644 index 000000000000..0799d841e5dc --- /dev/null +++ b/R/pkg/R/window.R @@ -0,0 +1,116 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# window.R - Utility functions for defining window in DataFrames + +#' windowPartitionBy +#' +#' Creates a WindowSpec with the partitioning defined. +#' +#' @param col A column name or Column by which rows are partitioned to +#' windows. +#' @param ... Optional column names or Columns in addition to col, by +#' which rows are partitioned to windows. +#' +#' @rdname windowPartitionBy +#' @name windowPartitionBy +#' @aliases windowPartitionBy,character-method +#' @export +#' @examples +#' \dontrun{ +#' ws <- orderBy(windowPartitionBy("key1", "key2"), "key3") +#' df1 <- select(df, over(lead("value", 1), ws)) +#' +#' ws <- orderBy(windowPartitionBy(df$key1, df$key2), df$key3) +#' df1 <- select(df, over(lead("value", 1), ws)) +#' } +#' @note windowPartitionBy(character) since 2.0.0 +setMethod("windowPartitionBy", + signature(col = "character"), + function(col, ...) { + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "partitionBy", + col, + list(...))) + }) + +#' @rdname windowPartitionBy +#' @name windowPartitionBy +#' @aliases windowPartitionBy,Column-method +#' @export +#' @note windowPartitionBy(Column) since 2.0.0 +setMethod("windowPartitionBy", + signature(col = "Column"), + function(col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "partitionBy", + jcols)) + }) + +#' windowOrderBy +#' +#' Creates a WindowSpec with the ordering defined. +#' +#' @param col A column name or Column by which rows are ordered within +#' windows. +#' @param ... Optional column names or Columns in addition to col, by +#' which rows are ordered within windows. +#' +#' @rdname windowOrderBy +#' @name windowOrderBy +#' @aliases windowOrderBy,character-method +#' @export +#' @examples +#' \dontrun{ +#' ws <- windowOrderBy("key1", "key2") +#' df1 <- select(df, over(lead("value", 1), ws)) +#' +#' ws <- windowOrderBy(df$key1, df$key2) +#' df1 <- select(df, over(lead("value", 1), ws)) +#' } +#' @note windowOrderBy(character) since 2.0.0 +setMethod("windowOrderBy", + signature(col = "character"), + function(col, ...) { + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "orderBy", + col, + list(...))) + }) + +#' @rdname windowOrderBy +#' @name windowOrderBy +#' @aliases windowOrderBy,Column-method +#' @export +#' @note windowOrderBy(Column) since 2.0.0 +setMethod("windowOrderBy", + signature(col = "Column"), + function(col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + windowSpec( + callJStatic("org.apache.spark.sql.expressions.Window", + "orderBy", + jcols)) + }) diff --git a/R/pkg/inst/profile/general.R b/R/pkg/inst/profile/general.R index 2a8a8213d084..8c75c19ca7ac 100644 --- a/R/pkg/inst/profile/general.R +++ b/R/pkg/inst/profile/general.R @@ -17,6 +17,7 @@ .First <- function() { packageDir <- Sys.getenv("SPARKR_PACKAGE_DIR") - .libPaths(c(packageDir, .libPaths())) - Sys.setenv(NOAWT=1) + dirs <- strsplit(packageDir, ",")[[1]] + .libPaths(c(dirs, .libPaths())) + Sys.setenv(NOAWT = 1) } diff --git a/R/pkg/inst/profile/shell.R b/R/pkg/inst/profile/shell.R index 7189f1a26093..8a8111a8c541 100644 --- a/R/pkg/inst/profile/shell.R +++ b/R/pkg/inst/profile/shell.R @@ -18,17 +18,17 @@ .First <- function() { home <- Sys.getenv("SPARK_HOME") .libPaths(c(file.path(home, "R", "lib"), .libPaths())) - Sys.setenv(NOAWT=1) + Sys.setenv(NOAWT = 1) # Make sure SparkR package is the last loaded one old <- getOption("defaultPackages") options(defaultPackages = c(old, "SparkR")) - sc <- SparkR::sparkR.init() - assign("sc", sc, envir=.GlobalEnv) - sqlContext <- SparkR::sparkRSQL.init(sc) + spark <- SparkR::sparkR.session() + assign("spark", spark, envir = .GlobalEnv) + sc <- SparkR:::callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", spark) + assign("sc", sc, envir = .GlobalEnv) sparkVer <- SparkR:::callJMethod(sc, "version") - assign("sqlContext", sqlContext, envir=.GlobalEnv) cat("\n Welcome to") cat("\n") cat(" ____ __", "\n") @@ -38,10 +38,10 @@ if (nchar(sparkVer) == 0) { cat("\n") } else { - cat(" version ", sparkVer, "\n") + cat(" version ", sparkVer, "\n") } cat(" /_/", "\n") cat("\n") - cat("\n Spark context is available as sc, SQL context is available as sqlContext\n") + cat("\n SparkSession available as 'spark'.\n") } diff --git a/R/pkg/inst/test_support/sparktestjar_2.10-1.0.jar b/R/pkg/inst/test_support/sparktestjar_2.10-1.0.jar deleted file mode 100644 index 1d5c2af631aa..000000000000 Binary files a/R/pkg/inst/test_support/sparktestjar_2.10-1.0.jar and /dev/null differ diff --git a/R/pkg/inst/tests/test_broadcast.R b/R/pkg/inst/tests/test_broadcast.R deleted file mode 100644 index bb86a5c922bd..000000000000 --- a/R/pkg/inst/tests/test_broadcast.R +++ /dev/null @@ -1,48 +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. -# - -context("broadcast variables") - -# JavaSparkContext handle -sc <- sparkR.init() - -# Partitioned data -nums <- 1:2 -rrdd <- parallelize(sc, nums, 2L) - -test_that("using broadcast variable", { - randomMat <- matrix(nrow=10, ncol=10, data=rnorm(100)) - randomMatBr <- broadcast(sc, randomMat) - - useBroadcast <- function(x) { - sum(SparkR:::value(randomMatBr) * x) - } - actual <- collect(lapply(rrdd, useBroadcast)) - expected <- list(sum(randomMat) * 1, sum(randomMat) * 2) - expect_equal(actual, expected) -}) - -test_that("without using broadcast variable", { - randomMat <- matrix(nrow=10, ncol=10, data=rnorm(100)) - - useBroadcast <- function(x) { - sum(randomMat * x) - } - actual <- collect(lapply(rrdd, useBroadcast)) - expected <- list(sum(randomMat) * 1, sum(randomMat) * 2) - expect_equal(actual, expected) -}) diff --git a/R/pkg/inst/tests/test_context.R b/R/pkg/inst/tests/test_context.R deleted file mode 100644 index 80c1b89a4c62..000000000000 --- a/R/pkg/inst/tests/test_context.R +++ /dev/null @@ -1,94 +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. -# - -context("test functions in sparkR.R") - -test_that("repeatedly starting and stopping SparkR", { - for (i in 1:4) { - sc <- sparkR.init() - rdd <- parallelize(sc, 1:20, 2L) - expect_equal(count(rdd), 20) - sparkR.stop() - } -}) - -test_that("repeatedly starting and stopping SparkR SQL", { - for (i in 1:4) { - sc <- sparkR.init() - sqlContext <- sparkRSQL.init(sc) - df <- createDataFrame(sqlContext, data.frame(a = 1:20)) - expect_equal(count(df), 20) - sparkR.stop() - } -}) - -test_that("rdd GC across sparkR.stop", { - sparkR.stop() - sc <- sparkR.init() # sc should get id 0 - rdd1 <- parallelize(sc, 1:20, 2L) # rdd1 should get id 1 - rdd2 <- parallelize(sc, 1:10, 2L) # rdd2 should get id 2 - sparkR.stop() - - sc <- sparkR.init() # sc should get id 0 again - - # GC rdd1 before creating rdd3 and rdd2 after - rm(rdd1) - gc() - - rdd3 <- parallelize(sc, 1:20, 2L) # rdd3 should get id 1 now - rdd4 <- parallelize(sc, 1:10, 2L) # rdd4 should get id 2 now - - rm(rdd2) - gc() - - count(rdd3) - count(rdd4) -}) - -test_that("job group functions can be called", { - sc <- sparkR.init() - setJobGroup(sc, "groupId", "job description", TRUE) - cancelJobGroup(sc, "groupId") - clearJobGroup(sc) -}) - -test_that("getClientModeSparkSubmitOpts() returns spark-submit args from whitelist", { - e <- new.env() - e[["spark.driver.memory"]] <- "512m" - ops <- getClientModeSparkSubmitOpts("sparkrmain", e) - expect_equal("--driver-memory \"512m\" sparkrmain", ops) - - e[["spark.driver.memory"]] <- "5g" - e[["spark.driver.extraClassPath"]] <- "/opt/class_path" # nolint - e[["spark.driver.extraJavaOptions"]] <- "-XX:+UseCompressedOops -XX:+UseCompressedStrings" - e[["spark.driver.extraLibraryPath"]] <- "/usr/local/hadoop/lib" # nolint - e[["random"]] <- "skipthis" - ops2 <- getClientModeSparkSubmitOpts("sparkr-shell", e) - # nolint start - expect_equal(ops2, paste0("--driver-class-path \"/opt/class_path\" --driver-java-options \"", - "-XX:+UseCompressedOops -XX:+UseCompressedStrings\" --driver-library-path \"", - "/usr/local/hadoop/lib\" --driver-memory \"5g\" sparkr-shell")) - # nolint end - - e[["spark.driver.extraClassPath"]] <- "/" # too short - ops3 <- getClientModeSparkSubmitOpts("--driver-memory 4g sparkr-shell2", e) - # nolint start - expect_equal(ops3, paste0("--driver-java-options \"-XX:+UseCompressedOops ", - "-XX:+UseCompressedStrings\" --driver-library-path \"/usr/local/hadoop/lib\"", - " --driver-memory 4g sparkr-shell2")) - # nolint end -}) diff --git a/R/pkg/inst/tests/test_includeJAR.R b/R/pkg/inst/tests/test_includeJAR.R deleted file mode 100644 index cc1faeabffe3..000000000000 --- a/R/pkg/inst/tests/test_includeJAR.R +++ /dev/null @@ -1,37 +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. -# -context("include an external JAR in SparkContext") - -runScript <- function() { - sparkHome <- Sys.getenv("SPARK_HOME") - sparkTestJarPath <- "R/lib/SparkR/test_support/sparktestjar_2.10-1.0.jar" - jarPath <- paste("--jars", shQuote(file.path(sparkHome, sparkTestJarPath))) - scriptPath <- file.path(sparkHome, "R/lib/SparkR/tests/jarTest.R") - submitPath <- file.path(sparkHome, "bin/spark-submit") - res <- system2(command = submitPath, - args = c(jarPath, scriptPath), - stdout = TRUE) - tail(res, 2) -} - -test_that("sparkJars tag in SparkContext", { - testOutput <- runScript() - helloTest <- testOutput[1] - expect_equal(helloTest, "Hello, Dave") - basicFunction <- testOutput[2] - expect_equal(basicFunction, "4") -}) diff --git a/R/pkg/inst/tests/test_mllib.R b/R/pkg/inst/tests/test_mllib.R deleted file mode 100644 index 032cfef061fd..000000000000 --- a/R/pkg/inst/tests/test_mllib.R +++ /dev/null @@ -1,86 +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. -# - -library(testthat) - -context("MLlib functions") - -# Tests for MLlib functions in SparkR - -sc <- sparkR.init() - -sqlContext <- sparkRSQL.init(sc) - -test_that("glm and predict", { - training <- createDataFrame(sqlContext, iris) - test <- select(training, "Sepal_Length") - model <- glm(Sepal_Width ~ Sepal_Length, training, family = "gaussian") - prediction <- predict(model, test) - expect_equal(typeof(take(select(prediction, "prediction"), 1)$prediction), "double") -}) - -test_that("predictions match with native glm", { - training <- createDataFrame(sqlContext, iris) - model <- glm(Sepal_Width ~ Sepal_Length + Species, data = training) - vals <- collect(select(predict(model, training), "prediction")) - rVals <- predict(glm(Sepal.Width ~ Sepal.Length + Species, data = iris), iris) - expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) -}) - -test_that("dot minus and intercept vs native glm", { - training <- createDataFrame(sqlContext, iris) - model <- glm(Sepal_Width ~ . - Species + 0, data = training) - vals <- collect(select(predict(model, training), "prediction")) - rVals <- predict(glm(Sepal.Width ~ . - Species + 0, data = iris), iris) - expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) -}) - -test_that("feature interaction vs native glm", { - training <- createDataFrame(sqlContext, iris) - model <- glm(Sepal_Width ~ Species:Sepal_Length, data = training) - vals <- collect(select(predict(model, training), "prediction")) - rVals <- predict(glm(Sepal.Width ~ Species:Sepal.Length, data = iris), iris) - expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) -}) - -test_that("summary coefficients match with native glm", { - training <- createDataFrame(sqlContext, iris) - stats <- summary(glm(Sepal_Width ~ Sepal_Length + Species, data = training, solver = "l-bfgs")) - coefs <- as.vector(stats$coefficients) - rCoefs <- as.vector(coef(glm(Sepal.Width ~ Sepal.Length + Species, data = iris))) - expect_true(all(abs(rCoefs - coefs) < 1e-6)) - expect_true(all( - as.character(stats$features) == - c("(Intercept)", "Sepal_Length", "Species_versicolor", "Species_virginica"))) -}) - -test_that("summary coefficients match with native glm of family 'binomial'", { - df <- createDataFrame(sqlContext, iris) - training <- filter(df, df$Species != "setosa") - stats <- summary(glm(Species ~ Sepal_Length + Sepal_Width, data = training, - family = "binomial")) - coefs <- as.vector(stats$coefficients) - - rTraining <- iris[iris$Species %in% c("versicolor","virginica"),] - rCoefs <- as.vector(coef(glm(Species ~ Sepal.Length + Sepal.Width, data = rTraining, - family = binomial(link = "logit")))) - - expect_true(all(abs(rCoefs - coefs) < 1e-4)) - expect_true(all( - as.character(stats$features) == - c("(Intercept)", "Sepal_Length", "Sepal_Width"))) -}) diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R deleted file mode 100644 index 71aed2bb9d6a..000000000000 --- a/R/pkg/inst/tests/test_rdd.R +++ /dev/null @@ -1,793 +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. -# - -context("basic RDD functions") - -# JavaSparkContext handle -sc <- sparkR.init() - -# Data -nums <- 1:10 -rdd <- parallelize(sc, nums, 2L) - -intPairs <- list(list(1L, -1), list(2L, 100), list(2L, 1), list(1L, 200)) -intRdd <- parallelize(sc, intPairs, 2L) - -test_that("get number of partitions in RDD", { - expect_equal(numPartitions(rdd), 2) - expect_equal(numPartitions(intRdd), 2) -}) - -test_that("first on RDD", { - expect_equal(first(rdd), 1) - newrdd <- lapply(rdd, function(x) x + 1) - expect_equal(first(newrdd), 2) -}) - -test_that("count and length on RDD", { - expect_equal(count(rdd), 10) - expect_equal(length(rdd), 10) -}) - -test_that("count by values and keys", { - mods <- lapply(rdd, function(x) { x %% 3 }) - actual <- countByValue(mods) - expected <- list(list(0, 3L), list(1, 4L), list(2, 3L)) - expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) - - actual <- countByKey(intRdd) - expected <- list(list(2L, 2L), list(1L, 2L)) - expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) -}) - -test_that("lapply on RDD", { - multiples <- lapply(rdd, function(x) { 2 * x }) - actual <- collect(multiples) - expect_equal(actual, as.list(nums * 2)) -}) - -test_that("lapplyPartition on RDD", { - sums <- lapplyPartition(rdd, function(part) { sum(unlist(part)) }) - actual <- collect(sums) - expect_equal(actual, list(15, 40)) -}) - -test_that("mapPartitions on RDD", { - sums <- mapPartitions(rdd, function(part) { sum(unlist(part)) }) - actual <- collect(sums) - expect_equal(actual, list(15, 40)) -}) - -test_that("flatMap() on RDDs", { - flat <- flatMap(intRdd, function(x) { list(x, x) }) - actual <- collect(flat) - expect_equal(actual, rep(intPairs, each=2)) -}) - -test_that("filterRDD on RDD", { - filtered.rdd <- filterRDD(rdd, function(x) { x %% 2 == 0 }) - actual <- collect(filtered.rdd) - expect_equal(actual, list(2, 4, 6, 8, 10)) - - filtered.rdd <- Filter(function(x) { x[[2]] < 0 }, intRdd) - actual <- collect(filtered.rdd) - expect_equal(actual, list(list(1L, -1))) - - # Filter out all elements. - filtered.rdd <- filterRDD(rdd, function(x) { x > 10 }) - actual <- collect(filtered.rdd) - expect_equal(actual, list()) -}) - -test_that("lookup on RDD", { - vals <- lookup(intRdd, 1L) - expect_equal(vals, list(-1, 200)) - - vals <- lookup(intRdd, 3L) - expect_equal(vals, list()) -}) - -test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { - rdd2 <- rdd - for (i in 1:12) - rdd2 <- lapplyPartitionsWithIndex( - rdd2, function(partIndex, part) { - part <- as.list(unlist(part) * partIndex + i) - }) - rdd2 <- lapply(rdd2, function(x) x + x) - actual <- collect(rdd2) - expected <- list(24, 24, 24, 24, 24, - 168, 170, 172, 174, 176) - expect_equal(actual, expected) -}) - -test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkpoint()", { - # RDD - rdd2 <- rdd - # PipelinedRDD - rdd2 <- lapplyPartitionsWithIndex( - rdd2, - function(partIndex, part) { - part <- as.list(unlist(part) * partIndex) - }) - - cache(rdd2) - expect_true(rdd2@env$isCached) - rdd2 <- lapply(rdd2, function(x) x) - expect_false(rdd2@env$isCached) - - unpersist(rdd2) - expect_false(rdd2@env$isCached) - - persist(rdd2, "MEMORY_AND_DISK") - expect_true(rdd2@env$isCached) - rdd2 <- lapply(rdd2, function(x) x) - expect_false(rdd2@env$isCached) - - unpersist(rdd2) - expect_false(rdd2@env$isCached) - - tempDir <- tempfile(pattern = "checkpoint") - setCheckpointDir(sc, tempDir) - checkpoint(rdd2) - expect_true(rdd2@env$isCheckpointed) - - rdd2 <- lapply(rdd2, function(x) x) - expect_false(rdd2@env$isCached) - expect_false(rdd2@env$isCheckpointed) - - # make sure the data is collectable - collect(rdd2) - - unlink(tempDir) -}) - -test_that("reduce on RDD", { - sum <- reduce(rdd, "+") - expect_equal(sum, 55) - - # Also test with an inline function - sumInline <- reduce(rdd, function(x, y) { x + y }) - expect_equal(sumInline, 55) -}) - -test_that("lapply with dependency", { - fa <- 5 - multiples <- lapply(rdd, function(x) { fa * x }) - actual <- collect(multiples) - - expect_equal(actual, as.list(nums * 5)) -}) - -test_that("lapplyPartitionsWithIndex on RDDs", { - func <- function(partIndex, part) { list(partIndex, Reduce("+", part)) } - actual <- collect(lapplyPartitionsWithIndex(rdd, func), flatten = FALSE) - expect_equal(actual, list(list(0, 15), list(1, 40))) - - pairsRDD <- parallelize(sc, list(list(1, 2), list(3, 4), list(4, 8)), 1L) - partitionByParity <- function(key) { if (key %% 2 == 1) 0 else 1 } - mkTup <- function(partIndex, part) { list(partIndex, part) } - actual <- collect(lapplyPartitionsWithIndex( - partitionBy(pairsRDD, 2L, partitionByParity), - mkTup), - FALSE) - expect_equal(actual, list(list(0, list(list(1, 2), list(3, 4))), - list(1, list(list(4, 8))))) -}) - -test_that("sampleRDD() on RDDs", { - expect_equal(unlist(collect(sampleRDD(rdd, FALSE, 1.0, 2014L))), nums) -}) - -test_that("takeSample() on RDDs", { - # ported from RDDSuite.scala, modified seeds - data <- parallelize(sc, 1:100, 2L) - for (seed in 4:5) { - s <- takeSample(data, FALSE, 20L, seed) - expect_equal(length(s), 20L) - expect_equal(length(unique(s)), 20L) - for (elem in s) { - expect_true(elem >= 1 && elem <= 100) - } - } - for (seed in 4:5) { - s <- takeSample(data, FALSE, 200L, seed) - expect_equal(length(s), 100L) - expect_equal(length(unique(s)), 100L) - for (elem in s) { - expect_true(elem >= 1 && elem <= 100) - } - } - for (seed in 4:5) { - s <- takeSample(data, TRUE, 20L, seed) - expect_equal(length(s), 20L) - for (elem in s) { - expect_true(elem >= 1 && elem <= 100) - } - } - for (seed in 4:5) { - s <- takeSample(data, TRUE, 100L, seed) - expect_equal(length(s), 100L) - # Chance of getting all distinct elements is astronomically low, so test we - # got < 100 - expect_true(length(unique(s)) < 100L) - } - for (seed in 4:5) { - s <- takeSample(data, TRUE, 200L, seed) - expect_equal(length(s), 200L) - # Chance of getting all distinct elements is still quite low, so test we - # got < 100 - expect_true(length(unique(s)) < 100L) - } -}) - -test_that("mapValues() on pairwise RDDs", { - multiples <- mapValues(intRdd, function(x) { x * 2 }) - actual <- collect(multiples) - expected <- lapply(intPairs, function(x) { - list(x[[1]], x[[2]] * 2) - }) - expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) -}) - -test_that("flatMapValues() on pairwise RDDs", { - l <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) - actual <- collect(flatMapValues(l, function(x) { x })) - expect_equal(actual, list(list(1,1), list(1,2), list(2,3), list(2,4))) - - # Generate x to x+1 for every value - actual <- collect(flatMapValues(intRdd, function(x) { x: (x + 1) })) - expect_equal(actual, - list(list(1L, -1), list(1L, 0), list(2L, 100), list(2L, 101), - list(2L, 1), list(2L, 2), list(1L, 200), list(1L, 201))) -}) - -test_that("reduceByKeyLocally() on PairwiseRDDs", { - pairs <- parallelize(sc, list(list(1, 2), list(1.1, 3), list(1, 4)), 2L) - actual <- reduceByKeyLocally(pairs, "+") - expect_equal(sortKeyValueList(actual), - sortKeyValueList(list(list(1, 6), list(1.1, 3)))) - - pairs <- parallelize(sc, list(list("abc", 1.2), list(1.1, 0), list("abc", 1.3), - list("bb", 5)), 4L) - actual <- reduceByKeyLocally(pairs, "+") - expect_equal(sortKeyValueList(actual), - sortKeyValueList(list(list("abc", 2.5), list(1.1, 0), list("bb", 5)))) -}) - -test_that("distinct() on RDDs", { - nums.rep2 <- rep(1:10, 2) - rdd.rep2 <- parallelize(sc, nums.rep2, 2L) - uniques <- distinct(rdd.rep2) - actual <- sort(unlist(collect(uniques))) - expect_equal(actual, nums) -}) - -test_that("maximum() on RDDs", { - max <- maximum(rdd) - expect_equal(max, 10) -}) - -test_that("minimum() on RDDs", { - min <- minimum(rdd) - expect_equal(min, 1) -}) - -test_that("sumRDD() on RDDs", { - sum <- sumRDD(rdd) - expect_equal(sum, 55) -}) - -test_that("keyBy on RDDs", { - func <- function(x) { x * x } - keys <- keyBy(rdd, func) - actual <- collect(keys) - expect_equal(actual, lapply(nums, function(x) { list(func(x), x) })) -}) - -test_that("repartition/coalesce on RDDs", { - rdd <- parallelize(sc, 1:20, 4L) # each partition contains 5 elements - - # repartition - r1 <- repartition(rdd, 2) - expect_equal(numPartitions(r1), 2L) - count <- length(collectPartition(r1, 0L)) - expect_true(count >= 8 && count <= 12) - - r2 <- repartition(rdd, 6) - expect_equal(numPartitions(r2), 6L) - count <- length(collectPartition(r2, 0L)) - expect_true(count >= 0 && count <= 4) - - # coalesce - r3 <- coalesce(rdd, 1) - expect_equal(numPartitions(r3), 1L) - count <- length(collectPartition(r3, 0L)) - expect_equal(count, 20) -}) - -test_that("sortBy() on RDDs", { - sortedRdd <- sortBy(rdd, function(x) { x * x }, ascending = FALSE) - actual <- collect(sortedRdd) - expect_equal(actual, as.list(sort(nums, decreasing = TRUE))) - - rdd2 <- parallelize(sc, sort(nums, decreasing = TRUE), 2L) - sortedRdd2 <- sortBy(rdd2, function(x) { x * x }) - actual <- collect(sortedRdd2) - expect_equal(actual, as.list(nums)) -}) - -test_that("takeOrdered() on RDDs", { - l <- list(10, 1, 2, 9, 3, 4, 5, 6, 7) - rdd <- parallelize(sc, l) - actual <- takeOrdered(rdd, 6L) - expect_equal(actual, as.list(sort(unlist(l)))[1:6]) - - l <- list("e", "d", "c", "d", "a") - rdd <- parallelize(sc, l) - actual <- takeOrdered(rdd, 3L) - expect_equal(actual, as.list(sort(unlist(l)))[1:3]) -}) - -test_that("top() on RDDs", { - l <- list(10, 1, 2, 9, 3, 4, 5, 6, 7) - rdd <- parallelize(sc, l) - actual <- top(rdd, 6L) - expect_equal(actual, as.list(sort(unlist(l), decreasing = TRUE))[1:6]) - - l <- list("e", "d", "c", "d", "a") - rdd <- parallelize(sc, l) - actual <- top(rdd, 3L) - expect_equal(actual, as.list(sort(unlist(l), decreasing = TRUE))[1:3]) -}) - -test_that("fold() on RDDs", { - actual <- fold(rdd, 0, "+") - expect_equal(actual, Reduce("+", nums, 0)) - - rdd <- parallelize(sc, list()) - actual <- fold(rdd, 0, "+") - expect_equal(actual, 0) -}) - -test_that("aggregateRDD() on RDDs", { - rdd <- parallelize(sc, list(1, 2, 3, 4)) - zeroValue <- list(0, 0) - seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } - combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } - actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp) - expect_equal(actual, list(10, 4)) - - rdd <- parallelize(sc, list()) - actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp) - expect_equal(actual, list(0, 0)) -}) - -test_that("zipWithUniqueId() on RDDs", { - rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) - actual <- collect(zipWithUniqueId(rdd)) - expected <- list(list("a", 0), list("b", 3), list("c", 1), - list("d", 4), list("e", 2)) - expect_equal(actual, expected) - - rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L) - actual <- collect(zipWithUniqueId(rdd)) - expected <- list(list("a", 0), list("b", 1), list("c", 2), - list("d", 3), list("e", 4)) - expect_equal(actual, expected) -}) - -test_that("zipWithIndex() on RDDs", { - rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) - actual <- collect(zipWithIndex(rdd)) - expected <- list(list("a", 0), list("b", 1), list("c", 2), - list("d", 3), list("e", 4)) - expect_equal(actual, expected) - - rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L) - actual <- collect(zipWithIndex(rdd)) - expected <- list(list("a", 0), list("b", 1), list("c", 2), - list("d", 3), list("e", 4)) - expect_equal(actual, expected) -}) - -test_that("glom() on RDD", { - rdd <- parallelize(sc, as.list(1:4), 2L) - actual <- collect(glom(rdd)) - expect_equal(actual, list(list(1, 2), list(3, 4))) -}) - -test_that("keys() on RDDs", { - keys <- keys(intRdd) - actual <- collect(keys) - expect_equal(actual, lapply(intPairs, function(x) { x[[1]] })) -}) - -test_that("values() on RDDs", { - values <- values(intRdd) - actual <- collect(values) - expect_equal(actual, lapply(intPairs, function(x) { x[[2]] })) -}) - -test_that("pipeRDD() on RDDs", { - actual <- collect(pipeRDD(rdd, "more")) - expected <- as.list(as.character(1:10)) - expect_equal(actual, expected) - - trailed.rdd <- parallelize(sc, c("1", "", "2\n", "3\n\r\n")) - actual <- collect(pipeRDD(trailed.rdd, "sort")) - expected <- list("", "1", "2", "3") - expect_equal(actual, expected) - - rev.nums <- 9:0 - rev.rdd <- parallelize(sc, rev.nums, 2L) - actual <- collect(pipeRDD(rev.rdd, "sort")) - expected <- as.list(as.character(c(5:9, 0:4))) - expect_equal(actual, expected) -}) - -test_that("zipRDD() on RDDs", { - rdd1 <- parallelize(sc, 0:4, 2) - rdd2 <- parallelize(sc, 1000:1004, 2) - actual <- collect(zipRDD(rdd1, rdd2)) - expect_equal(actual, - list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004))) - - mockFile <- c("Spark is pretty.", "Spark is awesome.") - fileName <- tempfile(pattern="spark-test", fileext=".tmp") - writeLines(mockFile, fileName) - - rdd <- textFile(sc, fileName, 1) - actual <- collect(zipRDD(rdd, rdd)) - expected <- lapply(mockFile, function(x) { list(x ,x) }) - expect_equal(actual, expected) - - rdd1 <- parallelize(sc, 0:1, 1) - actual <- collect(zipRDD(rdd1, rdd)) - expected <- lapply(0:1, function(x) { list(x, mockFile[x + 1]) }) - expect_equal(actual, expected) - - rdd1 <- map(rdd, function(x) { x }) - actual <- collect(zipRDD(rdd, rdd1)) - expected <- lapply(mockFile, function(x) { list(x, x) }) - expect_equal(actual, expected) - - unlink(fileName) -}) - -test_that("cartesian() on RDDs", { - rdd <- parallelize(sc, 1:3) - actual <- collect(cartesian(rdd, rdd)) - expect_equal(sortKeyValueList(actual), - list( - list(1, 1), list(1, 2), list(1, 3), - list(2, 1), list(2, 2), list(2, 3), - list(3, 1), list(3, 2), list(3, 3))) - - # test case where one RDD is empty - emptyRdd <- parallelize(sc, list()) - actual <- collect(cartesian(rdd, emptyRdd)) - expect_equal(actual, list()) - - mockFile <- c("Spark is pretty.", "Spark is awesome.") - fileName <- tempfile(pattern="spark-test", fileext=".tmp") - writeLines(mockFile, fileName) - - rdd <- textFile(sc, fileName) - actual <- collect(cartesian(rdd, rdd)) - expected <- list( - list("Spark is awesome.", "Spark is pretty."), - list("Spark is awesome.", "Spark is awesome."), - list("Spark is pretty.", "Spark is pretty."), - list("Spark is pretty.", "Spark is awesome.")) - expect_equal(sortKeyValueList(actual), expected) - - rdd1 <- parallelize(sc, 0:1) - actual <- collect(cartesian(rdd1, rdd)) - expect_equal(sortKeyValueList(actual), - list( - list(0, "Spark is pretty."), - list(0, "Spark is awesome."), - list(1, "Spark is pretty."), - list(1, "Spark is awesome."))) - - rdd1 <- map(rdd, function(x) { x }) - actual <- collect(cartesian(rdd, rdd1)) - expect_equal(sortKeyValueList(actual), expected) - - unlink(fileName) -}) - -test_that("subtract() on RDDs", { - l <- list(1, 1, 2, 2, 3, 4) - rdd1 <- parallelize(sc, l) - - # subtract by itself - actual <- collect(subtract(rdd1, rdd1)) - expect_equal(actual, list()) - - # subtract by an empty RDD - rdd2 <- parallelize(sc, list()) - actual <- collect(subtract(rdd1, rdd2)) - expect_equal(as.list(sort(as.vector(actual, mode="integer"))), - l) - - rdd2 <- parallelize(sc, list(2, 4)) - actual <- collect(subtract(rdd1, rdd2)) - expect_equal(as.list(sort(as.vector(actual, mode="integer"))), - list(1, 1, 3)) - - l <- list("a", "a", "b", "b", "c", "d") - rdd1 <- parallelize(sc, l) - rdd2 <- parallelize(sc, list("b", "d")) - actual <- collect(subtract(rdd1, rdd2)) - expect_equal(as.list(sort(as.vector(actual, mode="character"))), - list("a", "a", "c")) -}) - -test_that("subtractByKey() on pairwise RDDs", { - l <- list(list("a", 1), list("b", 4), - list("b", 5), list("a", 2)) - rdd1 <- parallelize(sc, l) - - # subtractByKey by itself - actual <- collect(subtractByKey(rdd1, rdd1)) - expect_equal(actual, list()) - - # subtractByKey by an empty RDD - rdd2 <- parallelize(sc, list()) - actual <- collect(subtractByKey(rdd1, rdd2)) - expect_equal(sortKeyValueList(actual), - sortKeyValueList(l)) - - rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) - actual <- collect(subtractByKey(rdd1, rdd2)) - expect_equal(actual, - list(list("b", 4), list("b", 5))) - - l <- list(list(1, 1), list(2, 4), - list(2, 5), list(1, 2)) - rdd1 <- parallelize(sc, l) - rdd2 <- parallelize(sc, list(list(1, 3), list(3, 1))) - actual <- collect(subtractByKey(rdd1, rdd2)) - expect_equal(actual, - list(list(2, 4), list(2, 5))) -}) - -test_that("intersection() on RDDs", { - # intersection with self - actual <- collect(intersection(rdd, rdd)) - expect_equal(sort(as.integer(actual)), nums) - - # intersection with an empty RDD - emptyRdd <- parallelize(sc, list()) - actual <- collect(intersection(rdd, emptyRdd)) - expect_equal(actual, list()) - - rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) - rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) - actual <- collect(intersection(rdd1, rdd2)) - expect_equal(sort(as.integer(actual)), 1:3) -}) - -test_that("join() on pairwise RDDs", { - rdd1 <- parallelize(sc, list(list(1,1), list(2,4))) - rdd2 <- parallelize(sc, list(list(1,2), list(1,3))) - actual <- collect(join(rdd1, rdd2, 2L)) - expect_equal(sortKeyValueList(actual), - sortKeyValueList(list(list(1, list(1, 2)), list(1, list(1, 3))))) - - rdd1 <- parallelize(sc, list(list("a",1), list("b",4))) - rdd2 <- parallelize(sc, list(list("a",2), list("a",3))) - actual <- collect(join(rdd1, rdd2, 2L)) - expect_equal(sortKeyValueList(actual), - sortKeyValueList(list(list("a", list(1, 2)), list("a", list(1, 3))))) - - rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) - rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) - actual <- collect(join(rdd1, rdd2, 2L)) - expect_equal(actual, list()) - - rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) - rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) - actual <- collect(join(rdd1, rdd2, 2L)) - expect_equal(actual, list()) -}) - -test_that("leftOuterJoin() on pairwise RDDs", { - rdd1 <- parallelize(sc, list(list(1,1), list(2,4))) - rdd2 <- parallelize(sc, list(list(1,2), list(1,3))) - actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) - expected <- list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) - expect_equal(sortKeyValueList(actual), - sortKeyValueList(expected)) - - rdd1 <- parallelize(sc, list(list("a",1), list("b",4))) - rdd2 <- parallelize(sc, list(list("a",2), list("a",3))) - actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) - expected <- list(list("b", list(4, NULL)), list("a", list(1, 2)), list("a", list(1, 3))) - expect_equal(sortKeyValueList(actual), - sortKeyValueList(expected)) - - rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) - rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) - actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) - expected <- list(list(1, list(1, NULL)), list(2, list(2, NULL))) - expect_equal(sortKeyValueList(actual), - sortKeyValueList(expected)) - - rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) - rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) - actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) - expected <- list(list("b", list(2, NULL)), list("a", list(1, NULL))) - expect_equal(sortKeyValueList(actual), - sortKeyValueList(expected)) -}) - -test_that("rightOuterJoin() on pairwise RDDs", { - rdd1 <- parallelize(sc, list(list(1,2), list(1,3))) - rdd2 <- parallelize(sc, list(list(1,1), list(2,4))) - actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) - expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) - expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) - - rdd1 <- parallelize(sc, list(list("a",2), list("a",3))) - rdd2 <- parallelize(sc, list(list("a",1), list("b",4))) - actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) - expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), list("a", list(3, 1))) - expect_equal(sortKeyValueList(actual), - sortKeyValueList(expected)) - - rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) - rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) - actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) - expect_equal(sortKeyValueList(actual), - sortKeyValueList(list(list(3, list(NULL, 3)), list(4, list(NULL, 4))))) - - rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) - rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) - actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) - expect_equal(sortKeyValueList(actual), - sortKeyValueList(list(list("d", list(NULL, 4)), list("c", list(NULL, 3))))) -}) - -test_that("fullOuterJoin() on pairwise RDDs", { - rdd1 <- parallelize(sc, list(list(1,2), list(1,3), list(3,3))) - rdd2 <- parallelize(sc, list(list(1,1), list(2,4))) - actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) - expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), - list(2, list(NULL, 4)), list(3, list(3, NULL))) - expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) - - rdd1 <- parallelize(sc, list(list("a",2), list("a",3), list("c", 1))) - rdd2 <- parallelize(sc, list(list("a",1), list("b",4))) - actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) - expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), - list("a", list(3, 1)), list("c", list(1, NULL))) - expect_equal(sortKeyValueList(actual), - sortKeyValueList(expected)) - - rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) - rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) - actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) - expect_equal(sortKeyValueList(actual), - sortKeyValueList(list(list(1, list(1, NULL)), list(2, list(2, NULL)), - list(3, list(NULL, 3)), list(4, list(NULL, 4))))) - - rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) - rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) - actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) - expect_equal(sortKeyValueList(actual), - sortKeyValueList(list(list("a", list(1, NULL)), list("b", list(2, NULL)), - list("d", list(NULL, 4)), list("c", list(NULL, 3))))) -}) - -test_that("sortByKey() on pairwise RDDs", { - numPairsRdd <- map(rdd, function(x) { list (x, x) }) - sortedRdd <- sortByKey(numPairsRdd, ascending = FALSE) - actual <- collect(sortedRdd) - numPairs <- lapply(nums, function(x) { list (x, x) }) - expect_equal(actual, sortKeyValueList(numPairs, decreasing = TRUE)) - - rdd2 <- parallelize(sc, sort(nums, decreasing = TRUE), 2L) - numPairsRdd2 <- map(rdd2, function(x) { list (x, x) }) - sortedRdd2 <- sortByKey(numPairsRdd2) - actual <- collect(sortedRdd2) - expect_equal(actual, numPairs) - - # sort by string keys - l <- list(list("a", 1), list("b", 2), list("1", 3), list("d", 4), list("2", 5)) - rdd3 <- parallelize(sc, l, 2L) - sortedRdd3 <- sortByKey(rdd3) - actual <- collect(sortedRdd3) - expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) - - # test on the boundary cases - - # boundary case 1: the RDD to be sorted has only 1 partition - rdd4 <- parallelize(sc, l, 1L) - sortedRdd4 <- sortByKey(rdd4) - actual <- collect(sortedRdd4) - expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) - - # boundary case 2: the sorted RDD has only 1 partition - rdd5 <- parallelize(sc, l, 2L) - sortedRdd5 <- sortByKey(rdd5, numPartitions = 1L) - actual <- collect(sortedRdd5) - expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) - - # boundary case 3: the RDD to be sorted has only 1 element - l2 <- list(list("a", 1)) - rdd6 <- parallelize(sc, l2, 2L) - sortedRdd6 <- sortByKey(rdd6) - actual <- collect(sortedRdd6) - expect_equal(actual, l2) - - # boundary case 4: the RDD to be sorted has 0 element - l3 <- list() - rdd7 <- parallelize(sc, l3, 2L) - sortedRdd7 <- sortByKey(rdd7) - actual <- collect(sortedRdd7) - expect_equal(actual, l3) -}) - -test_that("collectAsMap() on a pairwise RDD", { - rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) - vals <- collectAsMap(rdd) - expect_equal(vals, list(`1` = 2, `3` = 4)) - - rdd <- parallelize(sc, list(list("a", 1), list("b", 2))) - vals <- collectAsMap(rdd) - expect_equal(vals, list(a = 1, b = 2)) - - rdd <- parallelize(sc, list(list(1.1, 2.2), list(1.2, 2.4))) - vals <- collectAsMap(rdd) - expect_equal(vals, list(`1.1` = 2.2, `1.2` = 2.4)) - - rdd <- parallelize(sc, list(list(1, "a"), list(2, "b"))) - vals <- collectAsMap(rdd) - expect_equal(vals, list(`1` = "a", `2` = "b")) -}) - -test_that("show()", { - rdd <- parallelize(sc, list(1:10)) - expect_output(show(rdd), "ParallelCollectionRDD\\[\\d+\\] at parallelize at RRDD\\.scala:\\d+") -}) - -test_that("sampleByKey() on pairwise RDDs", { - rdd <- parallelize(sc, 1:2000) - pairsRDD <- lapply(rdd, function(x) { if (x %% 2 == 0) list("a", x) else list("b", x) }) - fractions <- list(a = 0.2, b = 0.1) - sample <- sampleByKey(pairsRDD, FALSE, fractions, 1618L) - expect_equal(100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")), TRUE) - expect_equal(50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")), TRUE) - expect_equal(lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0, TRUE) - expect_equal(lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000, TRUE) - expect_equal(lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0, TRUE) - expect_equal(lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000, TRUE) - - rdd <- parallelize(sc, 1:2000) - pairsRDD <- lapply(rdd, function(x) { if (x %% 2 == 0) list(2, x) else list(3, x) }) - fractions <- list(`2` = 0.2, `3` = 0.1) - sample <- sampleByKey(pairsRDD, TRUE, fractions, 1618L) - expect_equal(100 < length(lookup(sample, 2)) && 300 > length(lookup(sample, 2)), TRUE) - expect_equal(50 < length(lookup(sample, 3)) && 150 > length(lookup(sample, 3)), TRUE) - expect_equal(lookup(sample, 2)[which.min(lookup(sample, 2))] >= 0, TRUE) - expect_equal(lookup(sample, 2)[which.max(lookup(sample, 2))] <= 2000, TRUE) - expect_equal(lookup(sample, 3)[which.min(lookup(sample, 3))] >= 0, TRUE) - expect_equal(lookup(sample, 3)[which.max(lookup(sample, 3))] <= 2000, TRUE) -}) diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R deleted file mode 100644 index b4a4d03b2643..000000000000 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ /dev/null @@ -1,1499 +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. -# - -library(testthat) - -context("SparkSQL functions") - -# Utility function for easily checking the values of a StructField -checkStructField <- function(actual, expectedName, expectedType, expectedNullable) { - expect_equal(class(actual), "structField") - expect_equal(actual$name(), expectedName) - expect_equal(actual$dataType.toString(), expectedType) - expect_equal(actual$nullable(), expectedNullable) -} - -# Tests for SparkSQL functions in SparkR - -sc <- sparkR.init() - -sqlContext <- sparkRSQL.init(sc) - -mockLines <- c("{\"name\":\"Michael\"}", - "{\"name\":\"Andy\", \"age\":30}", - "{\"name\":\"Justin\", \"age\":19}") -jsonPath <- tempfile(pattern="sparkr-test", fileext=".tmp") -parquetPath <- tempfile(pattern="sparkr-test", fileext=".parquet") -writeLines(mockLines, jsonPath) - -# For test nafunctions, like dropna(), fillna(),... -mockLinesNa <- c("{\"name\":\"Bob\",\"age\":16,\"height\":176.5}", - "{\"name\":\"Alice\",\"age\":null,\"height\":164.3}", - "{\"name\":\"David\",\"age\":60,\"height\":null}", - "{\"name\":\"Amy\",\"age\":null,\"height\":null}", - "{\"name\":null,\"age\":null,\"height\":null}") -jsonPathNa <- tempfile(pattern="sparkr-test", fileext=".tmp") -writeLines(mockLinesNa, jsonPathNa) - -# For test complex types in DataFrame -mockLinesComplexType <- - c("{\"c1\":[1, 2, 3], \"c2\":[\"a\", \"b\", \"c\"], \"c3\":[1.0, 2.0, 3.0]}", - "{\"c1\":[4, 5, 6], \"c2\":[\"d\", \"e\", \"f\"], \"c3\":[4.0, 5.0, 6.0]}", - "{\"c1\":[7, 8, 9], \"c2\":[\"g\", \"h\", \"i\"], \"c3\":[7.0, 8.0, 9.0]}") -complexTypeJsonPath <- tempfile(pattern="sparkr-test", fileext=".tmp") -writeLines(mockLinesComplexType, complexTypeJsonPath) - -test_that("infer types and check types", { - expect_equal(infer_type(1L), "integer") - expect_equal(infer_type(1.0), "double") - expect_equal(infer_type("abc"), "string") - expect_equal(infer_type(TRUE), "boolean") - expect_equal(infer_type(as.Date("2015-03-11")), "date") - expect_equal(infer_type(as.POSIXlt("2015-03-11 12:13:04.043")), "timestamp") - expect_equal(infer_type(c(1L, 2L)), "array") - expect_equal(infer_type(list(1L, 2L)), "array") - expect_equal(infer_type(listToStruct(list(a = 1L, b = "2"))), "struct") - e <- new.env() - assign("a", 1L, envir = e) - expect_equal(infer_type(e), "map") - - expect_error(checkType("map"), "Key type in a map must be string or character") -}) - -test_that("structType and structField", { - testField <- structField("a", "string") - expect_is(testField, "structField") - expect_equal(testField$name(), "a") - expect_true(testField$nullable()) - - testSchema <- structType(testField, structField("b", "integer")) - expect_is(testSchema, "structType") - expect_is(testSchema$fields()[[2]], "structField") - expect_equal(testSchema$fields()[[1]]$dataType.toString(), "StringType") -}) - -test_that("create DataFrame from RDD", { - rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) - df <- createDataFrame(sqlContext, rdd, list("a", "b")) - dfAsDF <- as.DataFrame(sqlContext, rdd, list("a", "b")) - expect_is(df, "DataFrame") - expect_is(dfAsDF, "DataFrame") - expect_equal(count(df), 10) - expect_equal(count(dfAsDF), 10) - expect_equal(nrow(df), 10) - expect_equal(nrow(dfAsDF), 10) - expect_equal(ncol(df), 2) - expect_equal(ncol(dfAsDF), 2) - expect_equal(dim(df), c(10, 2)) - expect_equal(dim(dfAsDF), c(10, 2)) - expect_equal(columns(df), c("a", "b")) - expect_equal(columns(dfAsDF), c("a", "b")) - expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) - expect_equal(dtypes(dfAsDF), list(c("a", "int"), c("b", "string"))) - - df <- createDataFrame(sqlContext, rdd) - dfAsDF <- as.DataFrame(sqlContext, rdd) - expect_is(df, "DataFrame") - expect_is(dfAsDF, "DataFrame") - expect_equal(columns(df), c("_1", "_2")) - expect_equal(columns(dfAsDF), c("_1", "_2")) - - schema <- structType(structField(x = "a", type = "integer", nullable = TRUE), - structField(x = "b", type = "string", nullable = TRUE)) - df <- createDataFrame(sqlContext, rdd, schema) - expect_is(df, "DataFrame") - expect_equal(columns(df), c("a", "b")) - expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) - - rdd <- lapply(parallelize(sc, 1:10), function(x) { list(a = x, b = as.character(x)) }) - df <- createDataFrame(sqlContext, rdd) - expect_is(df, "DataFrame") - expect_equal(count(df), 10) - expect_equal(columns(df), c("a", "b")) - expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) - - df <- jsonFile(sqlContext, jsonPathNa) - hiveCtx <- tryCatch({ - newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc) - }, - error = function(err) { - skip("Hive is not build with SparkSQL, skipped") - }) - sql(hiveCtx, "CREATE TABLE people (name string, age double, height float)") - insertInto(df, "people") - expect_equal(sql(hiveCtx, "SELECT age from people WHERE name = 'Bob'"), c(16)) - expect_equal(sql(hiveCtx, "SELECT height from people WHERE name ='Bob'"), c(176.5)) - - schema <- structType(structField("name", "string"), structField("age", "integer"), - structField("height", "float")) - df2 <- createDataFrame(sqlContext, df.toRDD, schema) - df2AsDF <- as.DataFrame(sqlContext, df.toRDD, schema) - expect_equal(columns(df2), c("name", "age", "height")) - expect_equal(columns(df2AsDF), c("name", "age", "height")) - expect_equal(dtypes(df2), list(c("name", "string"), c("age", "int"), c("height", "float"))) - expect_equal(dtypes(df2AsDF), list(c("name", "string"), c("age", "int"), c("height", "float"))) - expect_equal(collect(where(df2, df2$name == "Bob")), c("Bob", 16, 176.5)) - expect_equal(collect(where(df2AsDF, df2$name == "Bob")), c("Bob", 16, 176.5)) - - localDF <- data.frame(name=c("John", "Smith", "Sarah"), - age=c(19, 23, 18), - height=c(164.10, 181.4, 173.7)) - df <- createDataFrame(sqlContext, localDF, schema) - expect_is(df, "DataFrame") - expect_equal(count(df), 3) - expect_equal(columns(df), c("name", "age", "height")) - expect_equal(dtypes(df), list(c("name", "string"), c("age", "int"), c("height", "float"))) - expect_equal(collect(where(df, df$name == "John")), c("John", 19, 164.10)) -}) - -test_that("convert NAs to null type in DataFrames", { - rdd <- parallelize(sc, list(list(1L, 2L), list(NA, 4L))) - df <- createDataFrame(sqlContext, rdd, list("a", "b")) - expect_true(is.na(collect(df)[2, "a"])) - expect_equal(collect(df)[2, "b"], 4L) - - l <- data.frame(x = 1L, y = c(1L, NA_integer_, 3L)) - df <- createDataFrame(sqlContext, l) - expect_equal(collect(df)[2, "x"], 1L) - expect_true(is.na(collect(df)[2, "y"])) - - rdd <- parallelize(sc, list(list(1, 2), list(NA, 4))) - df <- createDataFrame(sqlContext, rdd, list("a", "b")) - expect_true(is.na(collect(df)[2, "a"])) - expect_equal(collect(df)[2, "b"], 4) - - l <- data.frame(x = 1, y = c(1, NA_real_, 3)) - df <- createDataFrame(sqlContext, l) - expect_equal(collect(df)[2, "x"], 1) - expect_true(is.na(collect(df)[2, "y"])) - - l <- list("a", "b", NA, "d") - df <- createDataFrame(sqlContext, l) - expect_true(is.na(collect(df)[3, "_1"])) - expect_equal(collect(df)[4, "_1"], "d") - - l <- list("a", "b", NA_character_, "d") - df <- createDataFrame(sqlContext, l) - expect_true(is.na(collect(df)[3, "_1"])) - expect_equal(collect(df)[4, "_1"], "d") - - l <- list(TRUE, FALSE, NA, TRUE) - df <- createDataFrame(sqlContext, l) - expect_true(is.na(collect(df)[3, "_1"])) - expect_equal(collect(df)[4, "_1"], TRUE) -}) - -test_that("toDF", { - rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) - df <- toDF(rdd, list("a", "b")) - expect_is(df, "DataFrame") - expect_equal(count(df), 10) - expect_equal(columns(df), c("a", "b")) - expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) - - df <- toDF(rdd) - expect_is(df, "DataFrame") - expect_equal(columns(df), c("_1", "_2")) - - schema <- structType(structField(x = "a", type = "integer", nullable = TRUE), - structField(x = "b", type = "string", nullable = TRUE)) - df <- toDF(rdd, schema) - expect_is(df, "DataFrame") - expect_equal(columns(df), c("a", "b")) - expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) - - rdd <- lapply(parallelize(sc, 1:10), function(x) { list(a = x, b = as.character(x)) }) - df <- toDF(rdd) - expect_is(df, "DataFrame") - expect_equal(count(df), 10) - expect_equal(columns(df), c("a", "b")) - expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) -}) - -test_that("create DataFrame from list or data.frame", { - l <- list(list(1, 2), list(3, 4)) - df <- createDataFrame(sqlContext, l, c("a", "b")) - expect_equal(columns(df), c("a", "b")) - - l <- list(list(a=1, b=2), list(a=3, b=4)) - df <- createDataFrame(sqlContext, l) - expect_equal(columns(df), c("a", "b")) - - a <- 1:3 - b <- c("a", "b", "c") - ldf <- data.frame(a, b) - df <- createDataFrame(sqlContext, ldf) - expect_equal(columns(df), c("a", "b")) - expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) - expect_equal(count(df), 3) - ldf2 <- collect(df) - expect_equal(ldf$a, ldf2$a) -}) - -test_that("create DataFrame with different data types", { - l <- list(a = 1L, b = 2, c = TRUE, d = "ss", e = as.Date("2012-12-13"), - f = as.POSIXct("2015-03-15 12:13:14.056")) - df <- createDataFrame(sqlContext, list(l)) - expect_equal(dtypes(df), list(c("a", "int"), c("b", "double"), c("c", "boolean"), - c("d", "string"), c("e", "date"), c("f", "timestamp"))) - expect_equal(count(df), 1) - expect_equal(collect(df), data.frame(l, stringsAsFactors = FALSE)) -}) - -test_that("create DataFrame with complex types", { - e <- new.env() - assign("n", 3L, envir = e) - - s <- listToStruct(list(a = "aa", b = 3L)) - - l <- list(as.list(1:10), list("a", "b"), e, s) - df <- createDataFrame(sqlContext, list(l), c("a", "b", "c", "d")) - expect_equal(dtypes(df), list(c("a", "array"), - c("b", "array"), - c("c", "map"), - c("d", "struct"))) - expect_equal(count(df), 1) - ldf <- collect(df) - expect_equal(names(ldf), c("a", "b", "c", "d")) - expect_equal(ldf[1, 1][[1]], l[[1]]) - expect_equal(ldf[1, 2][[1]], l[[2]]) - - e <- ldf$c[[1]] - expect_equal(class(e), "environment") - expect_equal(ls(e), "n") - expect_equal(e$n, 3L) - - s <- ldf$d[[1]] - expect_equal(class(s), "struct") - expect_equal(s$a, "aa") - expect_equal(s$b, 3L) -}) - -# For test map type and struct type in DataFrame -mockLinesMapType <- c("{\"name\":\"Bob\",\"info\":{\"age\":16,\"height\":176.5}}", - "{\"name\":\"Alice\",\"info\":{\"age\":20,\"height\":164.3}}", - "{\"name\":\"David\",\"info\":{\"age\":60,\"height\":180}}") -mapTypeJsonPath <- tempfile(pattern="sparkr-test", fileext=".tmp") -writeLines(mockLinesMapType, mapTypeJsonPath) - -test_that("Collect DataFrame with complex types", { - # ArrayType - df <- jsonFile(sqlContext, complexTypeJsonPath) - - ldf <- collect(df) - expect_equal(nrow(ldf), 3) - expect_equal(ncol(ldf), 3) - expect_equal(names(ldf), c("c1", "c2", "c3")) - expect_equal(ldf$c1, list(list(1, 2, 3), list(4, 5, 6), list (7, 8, 9))) - expect_equal(ldf$c2, list(list("a", "b", "c"), list("d", "e", "f"), list ("g", "h", "i"))) - expect_equal(ldf$c3, list(list(1.0, 2.0, 3.0), list(4.0, 5.0, 6.0), list (7.0, 8.0, 9.0))) - - # MapType - schema <- structType(structField("name", "string"), - structField("info", "map")) - df <- read.df(sqlContext, mapTypeJsonPath, "json", schema) - expect_equal(dtypes(df), list(c("name", "string"), - c("info", "map"))) - ldf <- collect(df) - expect_equal(nrow(ldf), 3) - expect_equal(ncol(ldf), 2) - expect_equal(names(ldf), c("name", "info")) - expect_equal(ldf$name, c("Bob", "Alice", "David")) - bob <- ldf$info[[1]] - expect_equal(class(bob), "environment") - expect_equal(bob$age, 16) - expect_equal(bob$height, 176.5) - - # StructType - df <- jsonFile(sqlContext, mapTypeJsonPath) - expect_equal(dtypes(df), list(c("info", "struct"), - c("name", "string"))) - ldf <- collect(df) - expect_equal(nrow(ldf), 3) - expect_equal(ncol(ldf), 2) - expect_equal(names(ldf), c("info", "name")) - expect_equal(ldf$name, c("Bob", "Alice", "David")) - bob <- ldf$info[[1]] - expect_equal(class(bob), "struct") - expect_equal(bob$age, 16) - expect_equal(bob$height, 176.5) -}) - -test_that("jsonFile() on a local file returns a DataFrame", { - df <- jsonFile(sqlContext, jsonPath) - expect_is(df, "DataFrame") - expect_equal(count(df), 3) -}) - -test_that("jsonRDD() on a RDD with json string", { - rdd <- parallelize(sc, mockLines) - expect_equal(count(rdd), 3) - df <- jsonRDD(sqlContext, rdd) - expect_is(df, "DataFrame") - expect_equal(count(df), 3) - - rdd2 <- flatMap(rdd, function(x) c(x, x)) - df <- jsonRDD(sqlContext, rdd2) - expect_is(df, "DataFrame") - expect_equal(count(df), 6) -}) - -test_that("test cache, uncache and clearCache", { - df <- jsonFile(sqlContext, jsonPath) - registerTempTable(df, "table1") - cacheTable(sqlContext, "table1") - uncacheTable(sqlContext, "table1") - clearCache(sqlContext) - dropTempTable(sqlContext, "table1") -}) - -test_that("test tableNames and tables", { - df <- jsonFile(sqlContext, jsonPath) - registerTempTable(df, "table1") - expect_equal(length(tableNames(sqlContext)), 1) - df <- tables(sqlContext) - expect_equal(count(df), 1) - dropTempTable(sqlContext, "table1") -}) - -test_that("registerTempTable() results in a queryable table and sql() results in a new DataFrame", { - df <- jsonFile(sqlContext, jsonPath) - registerTempTable(df, "table1") - newdf <- sql(sqlContext, "SELECT * FROM table1 where name = 'Michael'") - expect_is(newdf, "DataFrame") - expect_equal(count(newdf), 1) - dropTempTable(sqlContext, "table1") -}) - -test_that("insertInto() on a registered table", { - df <- read.df(sqlContext, jsonPath, "json") - write.df(df, parquetPath, "parquet", "overwrite") - dfParquet <- read.df(sqlContext, parquetPath, "parquet") - - lines <- c("{\"name\":\"Bob\", \"age\":24}", - "{\"name\":\"James\", \"age\":35}") - jsonPath2 <- tempfile(pattern="jsonPath2", fileext=".tmp") - parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet") - writeLines(lines, jsonPath2) - df2 <- read.df(sqlContext, jsonPath2, "json") - write.df(df2, parquetPath2, "parquet", "overwrite") - dfParquet2 <- read.df(sqlContext, parquetPath2, "parquet") - - registerTempTable(dfParquet, "table1") - insertInto(dfParquet2, "table1") - expect_equal(count(sql(sqlContext, "select * from table1")), 5) - expect_equal(first(sql(sqlContext, "select * from table1 order by age"))$name, "Michael") - dropTempTable(sqlContext, "table1") - - registerTempTable(dfParquet, "table1") - insertInto(dfParquet2, "table1", overwrite = TRUE) - expect_equal(count(sql(sqlContext, "select * from table1")), 2) - expect_equal(first(sql(sqlContext, "select * from table1 order by age"))$name, "Bob") - dropTempTable(sqlContext, "table1") -}) - -test_that("table() returns a new DataFrame", { - df <- jsonFile(sqlContext, jsonPath) - registerTempTable(df, "table1") - tabledf <- table(sqlContext, "table1") - expect_is(tabledf, "DataFrame") - expect_equal(count(tabledf), 3) - dropTempTable(sqlContext, "table1") -}) - -test_that("toRDD() returns an RRDD", { - df <- jsonFile(sqlContext, jsonPath) - testRDD <- toRDD(df) - expect_is(testRDD, "RDD") - expect_equal(count(testRDD), 3) -}) - -test_that("union on two RDDs created from DataFrames returns an RRDD", { - df <- jsonFile(sqlContext, jsonPath) - RDD1 <- toRDD(df) - RDD2 <- toRDD(df) - unioned <- unionRDD(RDD1, RDD2) - expect_is(unioned, "RDD") - expect_equal(SparkR:::getSerializedMode(unioned), "byte") - expect_equal(collect(unioned)[[2]]$name, "Andy") -}) - -test_that("union on mixed serialization types correctly returns a byte RRDD", { - # Byte RDD - nums <- 1:10 - rdd <- parallelize(sc, nums, 2L) - - # String RDD - textLines <- c("Michael", - "Andy, 30", - "Justin, 19") - textPath <- tempfile(pattern="sparkr-textLines", fileext=".tmp") - writeLines(textLines, textPath) - textRDD <- textFile(sc, textPath) - - df <- jsonFile(sqlContext, jsonPath) - dfRDD <- toRDD(df) - - unionByte <- unionRDD(rdd, dfRDD) - expect_is(unionByte, "RDD") - expect_equal(SparkR:::getSerializedMode(unionByte), "byte") - expect_equal(collect(unionByte)[[1]], 1) - expect_equal(collect(unionByte)[[12]]$name, "Andy") - - unionString <- unionRDD(textRDD, dfRDD) - expect_is(unionString, "RDD") - expect_equal(SparkR:::getSerializedMode(unionString), "byte") - expect_equal(collect(unionString)[[1]], "Michael") - expect_equal(collect(unionString)[[5]]$name, "Andy") -}) - -test_that("objectFile() works with row serialization", { - objectPath <- tempfile(pattern="spark-test", fileext=".tmp") - df <- jsonFile(sqlContext, jsonPath) - dfRDD <- toRDD(df) - saveAsObjectFile(coalesce(dfRDD, 1L), objectPath) - objectIn <- objectFile(sc, objectPath) - - expect_is(objectIn, "RDD") - expect_equal(SparkR:::getSerializedMode(objectIn), "byte") - expect_equal(collect(objectIn)[[2]]$age, 30) -}) - -test_that("lapply() on a DataFrame returns an RDD with the correct columns", { - df <- jsonFile(sqlContext, jsonPath) - testRDD <- lapply(df, function(row) { - row$newCol <- row$age + 5 - row - }) - expect_is(testRDD, "RDD") - collected <- collect(testRDD) - expect_equal(collected[[1]]$name, "Michael") - expect_equal(collected[[2]]$newCol, 35) -}) - -test_that("collect() returns a data.frame", { - df <- jsonFile(sqlContext, jsonPath) - rdf <- collect(df) - expect_true(is.data.frame(rdf)) - expect_equal(names(rdf)[1], "age") - expect_equal(nrow(rdf), 3) - expect_equal(ncol(rdf), 2) - - # collect() returns data correctly from a DataFrame with 0 row - df0 <- limit(df, 0) - rdf <- collect(df0) - expect_true(is.data.frame(rdf)) - expect_equal(names(rdf)[1], "age") - expect_equal(nrow(rdf), 0) - expect_equal(ncol(rdf), 2) -}) - -test_that("limit() returns DataFrame with the correct number of rows", { - df <- jsonFile(sqlContext, jsonPath) - dfLimited <- limit(df, 2) - expect_is(dfLimited, "DataFrame") - expect_equal(count(dfLimited), 2) -}) - -test_that("collect() and take() on a DataFrame return the same number of rows and columns", { - df <- jsonFile(sqlContext, jsonPath) - expect_equal(nrow(collect(df)), nrow(take(df, 10))) - expect_equal(ncol(collect(df)), ncol(take(df, 10))) -}) - -test_that("collect() support Unicode characters", { - markUtf8 <- function(s) { - Encoding(s) <- "UTF-8" - s - } - - lines <- c("{\"name\":\"안녕하세요\"}", - "{\"name\":\"您好\", \"age\":30}", - "{\"name\":\"こんにちは\", \"age\":19}", - "{\"name\":\"Xin chào\"}") - - jsonPath <- tempfile(pattern="sparkr-test", fileext=".tmp") - writeLines(lines, jsonPath) - - df <- read.df(sqlContext, jsonPath, "json") - rdf <- collect(df) - expect_true(is.data.frame(rdf)) - expect_equal(rdf$name[1], markUtf8("안녕하세요")) - expect_equal(rdf$name[2], markUtf8("您好")) - expect_equal(rdf$name[3], markUtf8("こんにちは")) - expect_equal(rdf$name[4], markUtf8("Xin chào")) - - df1 <- createDataFrame(sqlContext, rdf) - expect_equal(collect(where(df1, df1$name == markUtf8("您好")))$name, markUtf8("您好")) -}) - -test_that("multiple pipeline transformations result in an RDD with the correct values", { - df <- jsonFile(sqlContext, jsonPath) - first <- lapply(df, function(row) { - row$age <- row$age + 5 - row - }) - second <- lapply(first, function(row) { - row$testCol <- if (row$age == 35 && !is.na(row$age)) TRUE else FALSE - row - }) - expect_is(second, "RDD") - expect_equal(count(second), 3) - expect_equal(collect(second)[[2]]$age, 35) - expect_true(collect(second)[[2]]$testCol) - expect_false(collect(second)[[3]]$testCol) -}) - -test_that("cache(), persist(), and unpersist() on a DataFrame", { - df <- jsonFile(sqlContext, jsonPath) - expect_false(df@env$isCached) - cache(df) - expect_true(df@env$isCached) - - unpersist(df) - expect_false(df@env$isCached) - - persist(df, "MEMORY_AND_DISK") - expect_true(df@env$isCached) - - unpersist(df) - expect_false(df@env$isCached) - - # make sure the data is collectable - expect_true(is.data.frame(collect(df))) -}) - -test_that("schema(), dtypes(), columns(), names() return the correct values/format", { - df <- jsonFile(sqlContext, jsonPath) - testSchema <- schema(df) - expect_equal(length(testSchema$fields()), 2) - expect_equal(testSchema$fields()[[1]]$dataType.toString(), "LongType") - expect_equal(testSchema$fields()[[2]]$dataType.simpleString(), "string") - expect_equal(testSchema$fields()[[1]]$name(), "age") - - testTypes <- dtypes(df) - expect_equal(length(testTypes[[1]]), 2) - expect_equal(testTypes[[1]][1], "age") - - testCols <- columns(df) - expect_equal(length(testCols), 2) - expect_equal(testCols[2], "name") - - testNames <- names(df) - expect_equal(length(testNames), 2) - expect_equal(testNames[2], "name") -}) - -test_that("head() and first() return the correct data", { - df <- jsonFile(sqlContext, jsonPath) - testHead <- head(df) - expect_equal(nrow(testHead), 3) - expect_equal(ncol(testHead), 2) - - testHead2 <- head(df, 2) - expect_equal(nrow(testHead2), 2) - expect_equal(ncol(testHead2), 2) - - testFirst <- first(df) - expect_equal(nrow(testFirst), 1) - - # head() and first() return the correct data on - # a DataFrame with 0 row - df0 <- limit(df, 0) - - testHead <- head(df0) - expect_equal(nrow(testHead), 0) - expect_equal(ncol(testHead), 2) - - testFirst <- first(df0) - expect_equal(nrow(testFirst), 0) - expect_equal(ncol(testFirst), 2) -}) - -test_that("distinct() and unique on DataFrames", { - lines <- c("{\"name\":\"Michael\"}", - "{\"name\":\"Andy\", \"age\":30}", - "{\"name\":\"Justin\", \"age\":19}", - "{\"name\":\"Justin\", \"age\":19}") - jsonPathWithDup <- tempfile(pattern="sparkr-test", fileext=".tmp") - writeLines(lines, jsonPathWithDup) - - df <- jsonFile(sqlContext, jsonPathWithDup) - uniques <- distinct(df) - expect_is(uniques, "DataFrame") - expect_equal(count(uniques), 3) - - uniques2 <- unique(df) - expect_is(uniques2, "DataFrame") - expect_equal(count(uniques2), 3) -}) - -test_that("sample on a DataFrame", { - df <- jsonFile(sqlContext, jsonPath) - sampled <- sample(df, FALSE, 1.0) - expect_equal(nrow(collect(sampled)), count(df)) - expect_is(sampled, "DataFrame") - sampled2 <- sample(df, FALSE, 0.1) - expect_true(count(sampled2) < 3) - - # Also test sample_frac - sampled3 <- sample_frac(df, FALSE, 0.1) - expect_true(count(sampled3) < 3) -}) - -test_that("select operators", { - df <- select(jsonFile(sqlContext, jsonPath), "name", "age") - expect_is(df$name, "Column") - expect_is(df[[2]], "Column") - expect_is(df[["age"]], "Column") - - expect_is(df[,1], "DataFrame") - expect_equal(columns(df[,1]), c("name")) - expect_equal(columns(df[,"age"]), c("age")) - df2 <- df[,c("age", "name")] - expect_is(df2, "DataFrame") - expect_equal(columns(df2), c("age", "name")) - - df$age2 <- df$age - expect_equal(columns(df), c("name", "age", "age2")) - expect_equal(count(where(df, df$age2 == df$age)), 2) - df$age2 <- df$age * 2 - expect_equal(columns(df), c("name", "age", "age2")) - expect_equal(count(where(df, df$age2 == df$age * 2)), 2) - - df$age2 <- NULL - expect_equal(columns(df), c("name", "age")) - df$age3 <- NULL - expect_equal(columns(df), c("name", "age")) -}) - -test_that("select with column", { - df <- jsonFile(sqlContext, jsonPath) - df1 <- select(df, "name") - expect_equal(columns(df1), c("name")) - expect_equal(count(df1), 3) - - df2 <- select(df, df$age) - expect_equal(columns(df2), c("age")) - expect_equal(count(df2), 3) - - df3 <- select(df, lit("x")) - expect_equal(columns(df3), c("x")) - expect_equal(count(df3), 3) - expect_equal(collect(select(df3, "x"))[[1, 1]], "x") - - df4 <- select(df, c("name", "age")) - expect_equal(columns(df4), c("name", "age")) - expect_equal(count(df4), 3) - - expect_error(select(df, c("name", "age"), "name"), - "To select multiple columns, use a character vector or list for col") -}) - -test_that("subsetting", { - # jsonFile returns columns in random order - df <- select(jsonFile(sqlContext, jsonPath), "name", "age") - filtered <- df[df$age > 20,] - expect_equal(count(filtered), 1) - expect_equal(columns(filtered), c("name", "age")) - expect_equal(collect(filtered)$name, "Andy") - - df2 <- df[df$age == 19, 1] - expect_is(df2, "DataFrame") - expect_equal(count(df2), 1) - expect_equal(columns(df2), c("name")) - expect_equal(collect(df2)$name, "Justin") - - df3 <- df[df$age > 20, 2] - expect_equal(count(df3), 1) - expect_equal(columns(df3), c("age")) - - df4 <- df[df$age %in% c(19, 30), 1:2] - expect_equal(count(df4), 2) - expect_equal(columns(df4), c("name", "age")) - - df5 <- df[df$age %in% c(19), c(1,2)] - expect_equal(count(df5), 1) - expect_equal(columns(df5), c("name", "age")) - - df6 <- subset(df, df$age %in% c(30), c(1,2)) - expect_equal(count(df6), 1) - expect_equal(columns(df6), c("name", "age")) -}) - -test_that("selectExpr() on a DataFrame", { - df <- jsonFile(sqlContext, jsonPath) - selected <- selectExpr(df, "age * 2") - expect_equal(names(selected), "(age * 2)") - expect_equal(collect(selected), collect(select(df, df$age * 2L))) - - selected2 <- selectExpr(df, "name as newName", "abs(age) as age") - expect_equal(names(selected2), c("newName", "age")) - expect_equal(count(selected2), 3) -}) - -test_that("expr() on a DataFrame", { - df <- jsonFile(sqlContext, jsonPath) - expect_equal(collect(select(df, expr("abs(-123)")))[1, 1], 123) -}) - -test_that("column calculation", { - df <- jsonFile(sqlContext, jsonPath) - d <- collect(select(df, alias(df$age + 1, "age2"))) - expect_equal(names(d), c("age2")) - df2 <- select(df, lower(df$name), abs(df$age)) - expect_is(df2, "DataFrame") - expect_equal(count(df2), 3) -}) - -test_that("read.df() from json file", { - df <- read.df(sqlContext, jsonPath, "json") - expect_is(df, "DataFrame") - expect_equal(count(df), 3) - - # Check if we can apply a user defined schema - schema <- structType(structField("name", type = "string"), - structField("age", type = "double")) - - df1 <- read.df(sqlContext, jsonPath, "json", schema) - expect_is(df1, "DataFrame") - expect_equal(dtypes(df1), list(c("name", "string"), c("age", "double"))) - - # Run the same with loadDF - df2 <- loadDF(sqlContext, jsonPath, "json", schema) - expect_is(df2, "DataFrame") - expect_equal(dtypes(df2), list(c("name", "string"), c("age", "double"))) -}) - -test_that("write.df() as parquet file", { - df <- read.df(sqlContext, jsonPath, "json") - write.df(df, parquetPath, "parquet", mode="overwrite") - df2 <- read.df(sqlContext, parquetPath, "parquet") - expect_is(df2, "DataFrame") - expect_equal(count(df2), 3) -}) - -test_that("test HiveContext", { - hiveCtx <- tryCatch({ - newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc) - }, - error = function(err) { - skip("Hive is not build with SparkSQL, skipped") - }) - df <- createExternalTable(hiveCtx, "json", jsonPath, "json") - expect_is(df, "DataFrame") - expect_equal(count(df), 3) - df2 <- sql(hiveCtx, "select * from json") - expect_is(df2, "DataFrame") - expect_equal(count(df2), 3) - - jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") - saveAsTable(df, "json", "json", "append", path = jsonPath2) - df3 <- sql(hiveCtx, "select * from json") - expect_is(df3, "DataFrame") - expect_equal(count(df3), 6) -}) - -test_that("column operators", { - c <- column("a") - c2 <- (- c + 1 - 2) * 3 / 4.0 - c3 <- (c + c2 - c2) * c2 %% c2 - c4 <- (c > c2) & (c2 <= c3) | (c == c2) & (c2 != c3) - c5 <- c2 ^ c3 ^ c4 -}) - -test_that("column functions", { - c <- column("a") - c1 <- abs(c) + acos(c) + approxCountDistinct(c) + ascii(c) + asin(c) + atan(c) - c2 <- avg(c) + base64(c) + bin(c) + bitwiseNOT(c) + cbrt(c) + ceil(c) + cos(c) - c3 <- cosh(c) + count(c) + crc32(c) + exp(c) - c4 <- explode(c) + expm1(c) + factorial(c) + first(c) + floor(c) + hex(c) - c5 <- hour(c) + initcap(c) + isNaN(c) + last(c) + last_day(c) + length(c) - c6 <- log(c) + (c) + log1p(c) + log2(c) + lower(c) + ltrim(c) + max(c) + md5(c) - c7 <- mean(c) + min(c) + month(c) + negate(c) + quarter(c) - c8 <- reverse(c) + rint(c) + round(c) + rtrim(c) + sha1(c) - c9 <- signum(c) + sin(c) + sinh(c) + size(c) + soundex(c) + sqrt(c) + sum(c) - c10 <- sumDistinct(c) + tan(c) + tanh(c) + toDegrees(c) + toRadians(c) - c11 <- to_date(c) + trim(c) + unbase64(c) + unhex(c) + upper(c) - c12 <- lead("col", 1) + lead(c, 1) + lag("col", 1) + lag(c, 1) - c13 <- cumeDist() + ntile(1) - c14 <- denseRank() + percentRank() + rank() + rowNumber() - - # Test if base::rank() is exposed - expect_equal(class(rank())[[1]], "Column") - expect_equal(rank(1:3), as.numeric(c(1:3))) - - df <- jsonFile(sqlContext, jsonPath) - df2 <- select(df, between(df$age, c(20, 30)), between(df$age, c(10, 20))) - expect_equal(collect(df2)[[2, 1]], TRUE) - expect_equal(collect(df2)[[2, 2]], FALSE) - expect_equal(collect(df2)[[3, 1]], FALSE) - expect_equal(collect(df2)[[3, 2]], TRUE) - - df3 <- select(df, between(df$name, c("Apache", "Spark"))) - expect_equal(collect(df3)[[1, 1]], TRUE) - expect_equal(collect(df3)[[2, 1]], FALSE) - expect_equal(collect(df3)[[3, 1]], TRUE) - - df4 <- createDataFrame(sqlContext, list(list(a = "010101"))) - expect_equal(collect(select(df4, conv(df4$a, 2, 16)))[1, 1], "15") -}) -# -test_that("column binary mathfunctions", { - lines <- c("{\"a\":1, \"b\":5}", - "{\"a\":2, \"b\":6}", - "{\"a\":3, \"b\":7}", - "{\"a\":4, \"b\":8}") - jsonPathWithDup <- tempfile(pattern="sparkr-test", fileext=".tmp") - writeLines(lines, jsonPathWithDup) - df <- jsonFile(sqlContext, jsonPathWithDup) - expect_equal(collect(select(df, atan2(df$a, df$b)))[1, "ATAN2(a, b)"], atan2(1, 5)) - expect_equal(collect(select(df, atan2(df$a, df$b)))[2, "ATAN2(a, b)"], atan2(2, 6)) - expect_equal(collect(select(df, atan2(df$a, df$b)))[3, "ATAN2(a, b)"], atan2(3, 7)) - expect_equal(collect(select(df, atan2(df$a, df$b)))[4, "ATAN2(a, b)"], atan2(4, 8)) - ## nolint start - expect_equal(collect(select(df, hypot(df$a, df$b)))[1, "HYPOT(a, b)"], sqrt(1^2 + 5^2)) - expect_equal(collect(select(df, hypot(df$a, df$b)))[2, "HYPOT(a, b)"], sqrt(2^2 + 6^2)) - expect_equal(collect(select(df, hypot(df$a, df$b)))[3, "HYPOT(a, b)"], sqrt(3^2 + 7^2)) - expect_equal(collect(select(df, hypot(df$a, df$b)))[4, "HYPOT(a, b)"], sqrt(4^2 + 8^2)) - ## nolint end - expect_equal(collect(select(df, shiftLeft(df$b, 1)))[4, 1], 16) - expect_equal(collect(select(df, shiftRight(df$b, 1)))[4, 1], 4) - expect_equal(collect(select(df, shiftRightUnsigned(df$b, 1)))[4, 1], 4) - expect_equal(class(collect(select(df, rand()))[2, 1]), "numeric") - expect_equal(collect(select(df, rand(1)))[1, 1], 0.45, tolerance = 0.01) - expect_equal(class(collect(select(df, randn()))[2, 1]), "numeric") - expect_equal(collect(select(df, randn(1)))[1, 1], -0.0111, tolerance = 0.01) -}) - -test_that("string operators", { - df <- jsonFile(sqlContext, jsonPath) - expect_equal(count(where(df, like(df$name, "A%"))), 1) - expect_equal(count(where(df, startsWith(df$name, "A"))), 1) - expect_equal(first(select(df, substr(df$name, 1, 2)))[[1]], "Mi") - expect_equal(collect(select(df, cast(df$age, "string")))[[2, 1]], "30") - expect_equal(collect(select(df, concat(df$name, lit(":"), df$age)))[[2, 1]], "Andy:30") - expect_equal(collect(select(df, concat_ws(":", df$name)))[[2, 1]], "Andy") - expect_equal(collect(select(df, concat_ws(":", df$name, df$age)))[[2, 1]], "Andy:30") - expect_equal(collect(select(df, instr(df$name, "i")))[, 1], c(2, 0, 5)) - expect_equal(collect(select(df, format_number(df$age, 2)))[2, 1], "30.00") - expect_equal(collect(select(df, sha1(df$name)))[2, 1], - "ab5a000e88b5d9d0fa2575f5c6263eb93452405d") - expect_equal(collect(select(df, sha2(df$name, 256)))[2, 1], - "80f2aed3c618c423ddf05a2891229fba44942d907173152442cf6591441ed6dc") - expect_equal(collect(select(df, format_string("Name:%s", df$name)))[2, 1], "Name:Andy") - expect_equal(collect(select(df, format_string("%s, %d", df$name, df$age)))[2, 1], "Andy, 30") - expect_equal(collect(select(df, regexp_extract(df$name, "(n.y)", 1)))[2, 1], "ndy") - expect_equal(collect(select(df, regexp_replace(df$name, "(n.y)", "ydn")))[2, 1], "Aydn") - - l2 <- list(list(a = "aaads")) - df2 <- createDataFrame(sqlContext, l2) - expect_equal(collect(select(df2, locate("aa", df2$a)))[1, 1], 1) - expect_equal(collect(select(df2, locate("aa", df2$a, 1)))[1, 1], 2) - expect_equal(collect(select(df2, lpad(df2$a, 8, "#")))[1, 1], "###aaads") - expect_equal(collect(select(df2, rpad(df2$a, 8, "#")))[1, 1], "aaads###") - - l3 <- list(list(a = "a.b.c.d")) - df3 <- createDataFrame(sqlContext, l3) - expect_equal(collect(select(df3, substring_index(df3$a, ".", 2)))[1, 1], "a.b") - expect_equal(collect(select(df3, substring_index(df3$a, ".", -3)))[1, 1], "b.c.d") - expect_equal(collect(select(df3, translate(df3$a, "bc", "12")))[1, 1], "a.1.2.d") -}) - -test_that("date functions on a DataFrame", { - .originalTimeZone <- Sys.getenv("TZ") - Sys.setenv(TZ = "UTC") - l <- list(list(a = 1L, b = as.Date("2012-12-13")), - list(a = 2L, b = as.Date("2013-12-14")), - list(a = 3L, b = as.Date("2014-12-15"))) - df <- createDataFrame(sqlContext, l) - expect_equal(collect(select(df, dayofmonth(df$b)))[, 1], c(13, 14, 15)) - expect_equal(collect(select(df, dayofyear(df$b)))[, 1], c(348, 348, 349)) - expect_equal(collect(select(df, weekofyear(df$b)))[, 1], c(50, 50, 51)) - expect_equal(collect(select(df, year(df$b)))[, 1], c(2012, 2013, 2014)) - expect_equal(collect(select(df, month(df$b)))[, 1], c(12, 12, 12)) - expect_equal(collect(select(df, last_day(df$b)))[, 1], - c(as.Date("2012-12-31"), as.Date("2013-12-31"), as.Date("2014-12-31"))) - expect_equal(collect(select(df, next_day(df$b, "MONDAY")))[, 1], - c(as.Date("2012-12-17"), as.Date("2013-12-16"), as.Date("2014-12-22"))) - expect_equal(collect(select(df, date_format(df$b, "y")))[, 1], c("2012", "2013", "2014")) - expect_equal(collect(select(df, add_months(df$b, 3)))[, 1], - c(as.Date("2013-03-13"), as.Date("2014-03-14"), as.Date("2015-03-15"))) - expect_equal(collect(select(df, date_add(df$b, 1)))[, 1], - c(as.Date("2012-12-14"), as.Date("2013-12-15"), as.Date("2014-12-16"))) - expect_equal(collect(select(df, date_sub(df$b, 1)))[, 1], - c(as.Date("2012-12-12"), as.Date("2013-12-13"), as.Date("2014-12-14"))) - - l2 <- list(list(a = 1L, b = as.POSIXlt("2012-12-13 12:34:00", tz = "UTC")), - list(a = 2L, b = as.POSIXlt("2014-12-15 01:24:34", tz = "UTC"))) - df2 <- createDataFrame(sqlContext, l2) - expect_equal(collect(select(df2, minute(df2$b)))[, 1], c(34, 24)) - expect_equal(collect(select(df2, second(df2$b)))[, 1], c(0, 34)) - expect_equal(collect(select(df2, from_utc_timestamp(df2$b, "JST")))[, 1], - c(as.POSIXlt("2012-12-13 21:34:00 UTC"), as.POSIXlt("2014-12-15 10:24:34 UTC"))) - expect_equal(collect(select(df2, to_utc_timestamp(df2$b, "JST")))[, 1], - c(as.POSIXlt("2012-12-13 03:34:00 UTC"), as.POSIXlt("2014-12-14 16:24:34 UTC"))) - expect_more_than(collect(select(df2, unix_timestamp()))[1, 1], 0) - expect_more_than(collect(select(df2, unix_timestamp(df2$b)))[1, 1], 0) - expect_more_than(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0) - - l3 <- list(list(a = 1000), list(a = -1000)) - df3 <- createDataFrame(sqlContext, l3) - result31 <- collect(select(df3, from_unixtime(df3$a))) - expect_equal(grep("\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", result31[, 1], perl = TRUE), - c(1, 2)) - result32 <- collect(select(df3, from_unixtime(df3$a, "yyyy"))) - expect_equal(grep("\\d{4}", result32[, 1]), c(1, 2)) - Sys.setenv(TZ = .originalTimeZone) -}) - -test_that("greatest() and least() on a DataFrame", { - l <- list(list(a = 1, b = 2), list(a = 3, b = 4)) - df <- createDataFrame(sqlContext, l) - expect_equal(collect(select(df, greatest(df$a, df$b)))[, 1], c(2, 4)) - expect_equal(collect(select(df, least(df$a, df$b)))[, 1], c(1, 3)) -}) - -test_that("when(), otherwise() and ifelse() on a DataFrame", { - l <- list(list(a = 1, b = 2), list(a = 3, b = 4)) - df <- createDataFrame(sqlContext, l) - expect_equal(collect(select(df, when(df$a > 1 & df$b > 2, 1)))[, 1], c(NA, 1)) - expect_equal(collect(select(df, otherwise(when(df$a > 1, 1), 0)))[, 1], c(0, 1)) - expect_equal(collect(select(df, ifelse(df$a > 1 & df$b > 2, 0, 1)))[, 1], c(1, 0)) -}) - -test_that("group by", { - df <- jsonFile(sqlContext, jsonPath) - df1 <- agg(df, name = "max", age = "sum") - expect_equal(1, count(df1)) - df1 <- agg(df, age2 = max(df$age)) - expect_equal(1, count(df1)) - expect_equal(columns(df1), c("age2")) - - gd <- groupBy(df, "name") - expect_is(gd, "GroupedData") - df2 <- count(gd) - expect_is(df2, "DataFrame") - expect_equal(3, count(df2)) - - # Also test group_by, summarize, mean - gd1 <- group_by(df, "name") - expect_is(gd1, "GroupedData") - df_summarized <- summarize(gd, mean_age = mean(df$age)) - expect_is(df_summarized, "DataFrame") - expect_equal(3, count(df_summarized)) - - df3 <- agg(gd, age = "sum") - expect_is(df3, "DataFrame") - expect_equal(3, count(df3)) - - df3 <- agg(gd, age = sum(df$age)) - expect_is(df3, "DataFrame") - expect_equal(3, count(df3)) - expect_equal(columns(df3), c("name", "age")) - - df4 <- sum(gd, "age") - expect_is(df4, "DataFrame") - expect_equal(3, count(df4)) - expect_equal(3, count(mean(gd, "age"))) - expect_equal(3, count(max(gd, "age"))) -}) - -test_that("arrange() and orderBy() on a DataFrame", { - df <- jsonFile(sqlContext, jsonPath) - sorted <- arrange(df, df$age) - expect_equal(collect(sorted)[1,2], "Michael") - - sorted2 <- arrange(df, "name", decreasing = FALSE) - expect_equal(collect(sorted2)[2,"age"], 19) - - sorted3 <- orderBy(df, asc(df$age)) - expect_true(is.na(first(sorted3)$age)) - expect_equal(collect(sorted3)[2, "age"], 19) - - sorted4 <- orderBy(df, desc(df$name)) - expect_equal(first(sorted4)$name, "Michael") - expect_equal(collect(sorted4)[3,"name"], "Andy") - - sorted5 <- arrange(df, "age", "name", decreasing = TRUE) - expect_equal(collect(sorted5)[1,2], "Andy") - - sorted6 <- arrange(df, "age","name", decreasing = c(T, F)) - expect_equal(collect(sorted6)[1,2], "Andy") - - sorted7 <- arrange(df, "name", decreasing = FALSE) - expect_equal(collect(sorted7)[2,"age"], 19) -}) - -test_that("filter() on a DataFrame", { - df <- jsonFile(sqlContext, jsonPath) - filtered <- filter(df, "age > 20") - expect_equal(count(filtered), 1) - expect_equal(collect(filtered)$name, "Andy") - filtered2 <- where(df, df$name != "Michael") - expect_equal(count(filtered2), 2) - expect_equal(collect(filtered2)$age[2], 19) - - # test suites for %in% - filtered3 <- filter(df, "age in (19)") - expect_equal(count(filtered3), 1) - filtered4 <- filter(df, "age in (19, 30)") - expect_equal(count(filtered4), 2) - filtered5 <- where(df, df$age %in% c(19)) - expect_equal(count(filtered5), 1) - filtered6 <- where(df, df$age %in% c(19, 30)) - expect_equal(count(filtered6), 2) -}) - -test_that("join() and merge() on a DataFrame", { - df <- jsonFile(sqlContext, jsonPath) - - mockLines2 <- c("{\"name\":\"Michael\", \"test\": \"yes\"}", - "{\"name\":\"Andy\", \"test\": \"no\"}", - "{\"name\":\"Justin\", \"test\": \"yes\"}", - "{\"name\":\"Bob\", \"test\": \"yes\"}") - jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") - writeLines(mockLines2, jsonPath2) - df2 <- jsonFile(sqlContext, jsonPath2) - - joined <- join(df, df2) - expect_equal(names(joined), c("age", "name", "name", "test")) - expect_equal(count(joined), 12) - - joined2 <- join(df, df2, df$name == df2$name) - expect_equal(names(joined2), c("age", "name", "name", "test")) - expect_equal(count(joined2), 3) - - joined3 <- join(df, df2, df$name == df2$name, "rightouter") - expect_equal(names(joined3), c("age", "name", "name", "test")) - expect_equal(count(joined3), 4) - expect_true(is.na(collect(orderBy(joined3, joined3$age))$age[2])) - - joined4 <- select(join(df, df2, df$name == df2$name, "outer"), - alias(df$age + 5, "newAge"), df$name, df2$test) - expect_equal(names(joined4), c("newAge", "name", "test")) - expect_equal(count(joined4), 4) - expect_equal(collect(orderBy(joined4, joined4$name))$newAge[3], 24) - - joined5 <- join(df, df2, df$name == df2$name, "leftouter") - expect_equal(names(joined5), c("age", "name", "name", "test")) - expect_equal(count(joined5), 3) - expect_true(is.na(collect(orderBy(joined5, joined5$age))$age[1])) - - joined6 <- join(df, df2, df$name == df2$name, "inner") - expect_equal(names(joined6), c("age", "name", "name", "test")) - expect_equal(count(joined6), 3) - - joined7 <- join(df, df2, df$name == df2$name, "leftsemi") - expect_equal(names(joined7), c("age", "name")) - expect_equal(count(joined7), 3) - - joined8 <- join(df, df2, df$name == df2$name, "left_outer") - expect_equal(names(joined8), c("age", "name", "name", "test")) - expect_equal(count(joined8), 3) - expect_true(is.na(collect(orderBy(joined8, joined8$age))$age[1])) - - joined9 <- join(df, df2, df$name == df2$name, "right_outer") - expect_equal(names(joined9), c("age", "name", "name", "test")) - expect_equal(count(joined9), 4) - expect_true(is.na(collect(orderBy(joined9, joined9$age))$age[2])) - - merged <- merge(df, df2, by.x = "name", by.y = "name", all.x = TRUE, all.y = TRUE) - expect_equal(count(merged), 4) - expect_equal(names(merged), c("age", "name_x", "name_y", "test")) - expect_equal(collect(orderBy(merged, merged$name_x))$age[3], 19) - - merged <- merge(df, df2, suffixes = c("-X","-Y")) - expect_equal(count(merged), 3) - expect_equal(names(merged), c("age", "name-X", "name-Y", "test")) - expect_equal(collect(orderBy(merged, merged$"name-X"))$age[1], 30) - - merged <- merge(df, df2, by = "name", suffixes = c("-X","-Y"), sort = FALSE) - expect_equal(count(merged), 3) - expect_equal(names(merged), c("age", "name-X", "name-Y", "test")) - expect_equal(collect(orderBy(merged, merged$"name-Y"))$"name-X"[3], "Michael") - - merged <- merge(df, df2, by = "name", all = T, sort = T) - expect_equal(count(merged), 4) - expect_equal(names(merged), c("age", "name_x", "name_y", "test")) - expect_equal(collect(orderBy(merged, merged$"name_y"))$"name_x"[1], "Andy") - - merged <- merge(df, df2, by = NULL) - expect_equal(count(merged), 12) - expect_equal(names(merged), c("age", "name", "name", "test")) - - mockLines3 <- c("{\"name\":\"Michael\", \"name_y\":\"Michael\", \"test\": \"yes\"}", - "{\"name\":\"Andy\", \"name_y\":\"Andy\", \"test\": \"no\"}", - "{\"name\":\"Justin\", \"name_y\":\"Justin\", \"test\": \"yes\"}", - "{\"name\":\"Bob\", \"name_y\":\"Bob\", \"test\": \"yes\"}") - jsonPath3 <- tempfile(pattern="sparkr-test", fileext=".tmp") - writeLines(mockLines3, jsonPath3) - df3 <- jsonFile(sqlContext, jsonPath3) - expect_error(merge(df, df3), - paste("The following column name: name_y occurs more than once in the 'DataFrame'.", - "Please use different suffixes for the intersected columns.", sep = "")) -}) - -test_that("toJSON() returns an RDD of the correct values", { - df <- jsonFile(sqlContext, jsonPath) - testRDD <- toJSON(df) - expect_is(testRDD, "RDD") - expect_equal(SparkR:::getSerializedMode(testRDD), "string") - expect_equal(collect(testRDD)[[1]], mockLines[1]) -}) - -test_that("showDF()", { - df <- jsonFile(sqlContext, jsonPath) - s <- capture.output(showDF(df)) - expected <- paste("+----+-------+\n", - "| age| name|\n", - "+----+-------+\n", - "|null|Michael|\n", - "| 30| Andy|\n", - "| 19| Justin|\n", - "+----+-------+\n", sep="") - expect_output(s , expected) -}) - -test_that("isLocal()", { - df <- jsonFile(sqlContext, jsonPath) - expect_false(isLocal(df)) -}) - -test_that("unionAll(), rbind(), except(), and intersect() on a DataFrame", { - df <- jsonFile(sqlContext, jsonPath) - - lines <- c("{\"name\":\"Bob\", \"age\":24}", - "{\"name\":\"Andy\", \"age\":30}", - "{\"name\":\"James\", \"age\":35}") - jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") - writeLines(lines, jsonPath2) - df2 <- read.df(sqlContext, jsonPath2, "json") - - unioned <- arrange(unionAll(df, df2), df$age) - expect_is(unioned, "DataFrame") - expect_equal(count(unioned), 6) - expect_equal(first(unioned)$name, "Michael") - - unioned2 <- arrange(rbind(unioned, df, df2), df$age) - expect_is(unioned2, "DataFrame") - expect_equal(count(unioned2), 12) - expect_equal(first(unioned2)$name, "Michael") - - excepted <- arrange(except(df, df2), desc(df$age)) - expect_is(unioned, "DataFrame") - expect_equal(count(excepted), 2) - expect_equal(first(excepted)$name, "Justin") - - intersected <- arrange(intersect(df, df2), df$age) - expect_is(unioned, "DataFrame") - expect_equal(count(intersected), 1) - expect_equal(first(intersected)$name, "Andy") -}) - -test_that("withColumn() and withColumnRenamed()", { - df <- jsonFile(sqlContext, jsonPath) - newDF <- withColumn(df, "newAge", df$age + 2) - expect_equal(length(columns(newDF)), 3) - expect_equal(columns(newDF)[3], "newAge") - expect_equal(first(filter(newDF, df$name != "Michael"))$newAge, 32) - - newDF2 <- withColumnRenamed(df, "age", "newerAge") - expect_equal(length(columns(newDF2)), 2) - expect_equal(columns(newDF2)[1], "newerAge") -}) - -test_that("mutate(), transform(), rename() and names()", { - df <- jsonFile(sqlContext, jsonPath) - newDF <- mutate(df, newAge = df$age + 2) - expect_equal(length(columns(newDF)), 3) - expect_equal(columns(newDF)[3], "newAge") - expect_equal(first(filter(newDF, df$name != "Michael"))$newAge, 32) - - newDF2 <- rename(df, newerAge = df$age) - expect_equal(length(columns(newDF2)), 2) - expect_equal(columns(newDF2)[1], "newerAge") - - names(newDF2) <- c("newerName", "evenNewerAge") - expect_equal(length(names(newDF2)), 2) - expect_equal(names(newDF2)[1], "newerName") - - transformedDF <- transform(df, newAge = -df$age, newAge2 = df$age / 2) - expect_equal(length(columns(transformedDF)), 4) - expect_equal(columns(transformedDF)[3], "newAge") - expect_equal(columns(transformedDF)[4], "newAge2") - expect_equal(first(filter(transformedDF, transformedDF$name == "Andy"))$newAge, -30) - - # test if transform on local data frames works - # ensure the proper signature is used - otherwise this will fail to run - attach(airquality) - result <- transform(Ozone, logOzone = log(Ozone)) - expect_equal(nrow(result), 153) - expect_equal(ncol(result), 2) - detach(airquality) -}) - -test_that("write.df() on DataFrame and works with parquetFile", { - df <- jsonFile(sqlContext, jsonPath) - write.df(df, parquetPath, "parquet", mode="overwrite") - parquetDF <- parquetFile(sqlContext, parquetPath) - expect_is(parquetDF, "DataFrame") - expect_equal(count(df), count(parquetDF)) -}) - -test_that("parquetFile works with multiple input paths", { - df <- jsonFile(sqlContext, jsonPath) - write.df(df, parquetPath, "parquet", mode="overwrite") - parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet") - write.df(df, parquetPath2, "parquet", mode="overwrite") - parquetDF <- parquetFile(sqlContext, parquetPath, parquetPath2) - expect_is(parquetDF, "DataFrame") - expect_equal(count(parquetDF), count(df) * 2) - - # Test if varargs works with variables - saveMode <- "overwrite" - mergeSchema <- "true" - parquetPath3 <- tempfile(pattern = "parquetPath3", fileext = ".parquet") - write.df(df, parquetPath2, "parquet", mode = saveMode, mergeSchema = mergeSchema) -}) - -test_that("describe() and summarize() on a DataFrame", { - df <- jsonFile(sqlContext, jsonPath) - stats <- describe(df, "age") - expect_equal(collect(stats)[1, "summary"], "count") - expect_equal(collect(stats)[2, "age"], "24.5") - expect_equal(collect(stats)[3, "age"], "7.7781745930520225") - stats <- describe(df) - expect_equal(collect(stats)[4, "name"], "Andy") - expect_equal(collect(stats)[5, "age"], "30") - - stats2 <- summary(df) - expect_equal(collect(stats2)[4, "name"], "Andy") - expect_equal(collect(stats2)[5, "age"], "30") -}) - -test_that("dropna() and na.omit() on a DataFrame", { - df <- jsonFile(sqlContext, jsonPathNa) - rows <- collect(df) - - # drop with columns - - expected <- rows[!is.na(rows$name),] - actual <- collect(dropna(df, cols = "name")) - expect_identical(expected, actual) - actual <- collect(na.omit(df, cols = "name")) - expect_identical(expected, actual) - - expected <- rows[!is.na(rows$age),] - actual <- collect(dropna(df, cols = "age")) - row.names(expected) <- row.names(actual) - # identical on two dataframes does not work here. Don't know why. - # use identical on all columns as a workaround. - expect_identical(expected$age, actual$age) - expect_identical(expected$height, actual$height) - expect_identical(expected$name, actual$name) - actual <- collect(na.omit(df, cols = "age")) - - expected <- rows[!is.na(rows$age) & !is.na(rows$height),] - actual <- collect(dropna(df, cols = c("age", "height"))) - expect_identical(expected, actual) - actual <- collect(na.omit(df, cols = c("age", "height"))) - expect_identical(expected, actual) - - expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name),] - actual <- collect(dropna(df)) - expect_identical(expected, actual) - actual <- collect(na.omit(df)) - expect_identical(expected, actual) - - # drop with how - - expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name),] - actual <- collect(dropna(df)) - expect_identical(expected, actual) - actual <- collect(na.omit(df)) - expect_identical(expected, actual) - - expected <- rows[!is.na(rows$age) | !is.na(rows$height) | !is.na(rows$name),] - actual <- collect(dropna(df, "all")) - expect_identical(expected, actual) - actual <- collect(na.omit(df, "all")) - expect_identical(expected, actual) - - expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name),] - actual <- collect(dropna(df, "any")) - expect_identical(expected, actual) - actual <- collect(na.omit(df, "any")) - expect_identical(expected, actual) - - expected <- rows[!is.na(rows$age) & !is.na(rows$height),] - actual <- collect(dropna(df, "any", cols = c("age", "height"))) - expect_identical(expected, actual) - actual <- collect(na.omit(df, "any", cols = c("age", "height"))) - expect_identical(expected, actual) - - expected <- rows[!is.na(rows$age) | !is.na(rows$height),] - actual <- collect(dropna(df, "all", cols = c("age", "height"))) - expect_identical(expected, actual) - actual <- collect(na.omit(df, "all", cols = c("age", "height"))) - expect_identical(expected, actual) - - # drop with threshold - - expected <- rows[as.integer(!is.na(rows$age)) + as.integer(!is.na(rows$height)) >= 2,] - actual <- collect(dropna(df, minNonNulls = 2, cols = c("age", "height"))) - expect_identical(expected, actual) - actual <- collect(na.omit(df, minNonNulls = 2, cols = c("age", "height"))) - expect_identical(expected, actual) - - expected <- rows[as.integer(!is.na(rows$age)) + - as.integer(!is.na(rows$height)) + - as.integer(!is.na(rows$name)) >= 3,] - actual <- collect(dropna(df, minNonNulls = 3, cols = c("name", "age", "height"))) - expect_identical(expected, actual) - actual <- collect(na.omit(df, minNonNulls = 3, cols = c("name", "age", "height"))) - expect_identical(expected, actual) -}) - -test_that("fillna() on a DataFrame", { - df <- jsonFile(sqlContext, jsonPathNa) - rows <- collect(df) - - # fill with value - - expected <- rows - expected$age[is.na(expected$age)] <- 50 - expected$height[is.na(expected$height)] <- 50.6 - actual <- collect(fillna(df, 50.6)) - expect_identical(expected, actual) - - expected <- rows - expected$name[is.na(expected$name)] <- "unknown" - actual <- collect(fillna(df, "unknown")) - expect_identical(expected, actual) - - expected <- rows - expected$age[is.na(expected$age)] <- 50 - actual <- collect(fillna(df, 50.6, "age")) - expect_identical(expected, actual) - - expected <- rows - expected$name[is.na(expected$name)] <- "unknown" - actual <- collect(fillna(df, "unknown", c("age", "name"))) - expect_identical(expected, actual) - - # fill with named list - - expected <- rows - expected$age[is.na(expected$age)] <- 50 - expected$height[is.na(expected$height)] <- 50.6 - expected$name[is.na(expected$name)] <- "unknown" - actual <- collect(fillna(df, list("age" = 50, "height" = 50.6, "name" = "unknown"))) - expect_identical(expected, actual) -}) - -test_that("crosstab() on a DataFrame", { - rdd <- lapply(parallelize(sc, 0:3), function(x) { - list(paste0("a", x %% 3), paste0("b", x %% 2)) - }) - df <- toDF(rdd, list("a", "b")) - ct <- crosstab(df, "a", "b") - ordered <- ct[order(ct$a_b),] - row.names(ordered) <- NULL - expected <- data.frame("a_b" = c("a0", "a1", "a2"), "b0" = c(1, 0, 1), "b1" = c(1, 1, 0), - stringsAsFactors = FALSE, row.names = NULL) - expect_identical(expected, ordered) -}) - -test_that("cov() and corr() on a DataFrame", { - l <- lapply(c(0:9), function(x) { list(x, x * 2.0) }) - df <- createDataFrame(sqlContext, l, c("singles", "doubles")) - result <- cov(df, "singles", "doubles") - expect_true(abs(result - 55.0 / 3) < 1e-12) - - result <- corr(df, "singles", "doubles") - expect_true(abs(result - 1.0) < 1e-12) - result <- corr(df, "singles", "doubles", "pearson") - expect_true(abs(result - 1.0) < 1e-12) -}) - -test_that("freqItems() on a DataFrame", { - input <- 1:1000 - rdf <- data.frame(numbers = input, letters = as.character(input), - negDoubles = input * -1.0, stringsAsFactors = F) - rdf[ input %% 3 == 0, ] <- c(1, "1", -1) - df <- createDataFrame(sqlContext, rdf) - multiColResults <- freqItems(df, c("numbers", "letters"), support=0.1) - expect_true(1 %in% multiColResults$numbers[[1]]) - expect_true("1" %in% multiColResults$letters[[1]]) - singleColResult <- freqItems(df, "negDoubles", support=0.1) - expect_true(-1 %in% head(singleColResult$negDoubles)[[1]]) - - l <- lapply(c(0:99), function(i) { - if (i %% 2 == 0) { list(1L, -1.0) } - else { list(i, i * -1.0) }}) - df <- createDataFrame(sqlContext, l, c("a", "b")) - result <- freqItems(df, c("a", "b"), 0.4) - expect_identical(result[[1]], list(list(1L, 99L))) - expect_identical(result[[2]], list(list(-1, -99))) -}) - -test_that("sampleBy() on a DataFrame", { - l <- lapply(c(0:99), function(i) { as.character(i %% 3) }) - df <- createDataFrame(sqlContext, l, "key") - fractions <- list("0" = 0.1, "1" = 0.2) - sample <- sampleBy(df, "key", fractions, 0) - result <- collect(orderBy(count(groupBy(sample, "key")), "key")) - expect_identical(as.list(result[1, ]), list(key = "0", count = 2)) - expect_identical(as.list(result[2, ]), list(key = "1", count = 10)) -}) - -test_that("SQL error message is returned from JVM", { - retError <- tryCatch(sql(sqlContext, "select * from blah"), error = function(e) e) - expect_equal(grepl("Table not found: blah", retError), TRUE) -}) - -test_that("Method as.data.frame as a synonym for collect()", { - irisDF <- createDataFrame(sqlContext, iris) - expect_equal(as.data.frame(irisDF), collect(irisDF)) - irisDF2 <- irisDF[irisDF$Species == "setosa", ] - expect_equal(as.data.frame(irisDF2), collect(irisDF2)) -}) - -test_that("attach() on a DataFrame", { - df <- jsonFile(sqlContext, jsonPath) - expect_error(age) - attach(df) - expect_is(age, "DataFrame") - expected_age <- data.frame(age = c(NA, 30, 19)) - expect_equal(head(age), expected_age) - stat <- summary(age) - expect_equal(collect(stat)[5, "age"], "30") - age <- age$age + 1 - expect_is(age, "Column") - rm(age) - stat2 <- summary(age) - expect_equal(collect(stat2)[5, "age"], "30") - detach("df") - stat3 <- summary(df[, "age"]) - expect_equal(collect(stat3)[5, "age"], "30") - expect_error(age) -}) - -unlink(parquetPath) -unlink(jsonPath) -unlink(jsonPathNa) diff --git a/R/pkg/inst/tests/test_take.R b/R/pkg/inst/tests/test_take.R deleted file mode 100644 index c2c724cdc762..000000000000 --- a/R/pkg/inst/tests/test_take.R +++ /dev/null @@ -1,66 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -context("tests RDD function take()") - -# Mock data -numVector <- c(-10:97) -numList <- list(sqrt(1), sqrt(2), sqrt(3), 4 ** 10) -strVector <- c("Dexter Morgan: I suppose I should be upset, even feel", - "violated, but I'm not. No, in fact, I think this is a friendly", - "message, like \"Hey, wanna play?\" and yes, I want to play. ", - "I really, really do.") -strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge, ", - "other times it helps me control the chaos.", - "Dexter Morgan: Harry and Dorris Morgan did a wonderful job ", - "raising me. But they're both dead now. I didn't kill them. Honest.") - -# JavaSparkContext handle -jsc <- sparkR.init() - -test_that("take() gives back the original elements in correct count and order", { - numVectorRDD <- parallelize(jsc, numVector, 10) - # case: number of elements to take is less than the size of the first partition - expect_equal(take(numVectorRDD, 1), as.list(head(numVector, n = 1))) - # case: number of elements to take is the same as the size of the first partition - expect_equal(take(numVectorRDD, 11), as.list(head(numVector, n = 11))) - # case: number of elements to take is greater than all elements - expect_equal(take(numVectorRDD, length(numVector)), as.list(numVector)) - expect_equal(take(numVectorRDD, length(numVector) + 1), as.list(numVector)) - - numListRDD <- parallelize(jsc, numList, 1) - numListRDD2 <- parallelize(jsc, numList, 4) - expect_equal(take(numListRDD, 3), take(numListRDD2, 3)) - expect_equal(take(numListRDD, 5), take(numListRDD2, 5)) - expect_equal(take(numListRDD, 1), as.list(head(numList, n = 1))) - expect_equal(take(numListRDD2, 999), numList) - - strVectorRDD <- parallelize(jsc, strVector, 2) - strVectorRDD2 <- parallelize(jsc, strVector, 3) - expect_equal(take(strVectorRDD, 4), as.list(strVector)) - expect_equal(take(strVectorRDD2, 2), as.list(head(strVector, n = 2))) - - strListRDD <- parallelize(jsc, strList, 4) - strListRDD2 <- parallelize(jsc, strList, 1) - expect_equal(take(strListRDD, 3), as.list(head(strList, n = 3))) - expect_equal(take(strListRDD2, 1), as.list(head(strList, n = 1))) - - expect_equal(length(take(strListRDD, 0)), 0) - expect_equal(length(take(strVectorRDD, 0)), 0) - expect_equal(length(take(numListRDD, 0)), 0) - expect_equal(length(take(numVectorRDD, 0)), 0) -}) diff --git a/R/pkg/inst/tests/test_textFile.R b/R/pkg/inst/tests/test_textFile.R deleted file mode 100644 index a9cf83dbdbdb..000000000000 --- a/R/pkg/inst/tests/test_textFile.R +++ /dev/null @@ -1,161 +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. -# - -context("the textFile() function") - -# JavaSparkContext handle -sc <- sparkR.init() - -mockFile <- c("Spark is pretty.", "Spark is awesome.") - -test_that("textFile() on a local file returns an RDD", { - fileName <- tempfile(pattern="spark-test", fileext=".tmp") - writeLines(mockFile, fileName) - - rdd <- textFile(sc, fileName) - expect_is(rdd, "RDD") - expect_true(count(rdd) > 0) - expect_equal(count(rdd), 2) - - unlink(fileName) -}) - -test_that("textFile() followed by a collect() returns the same content", { - fileName <- tempfile(pattern="spark-test", fileext=".tmp") - writeLines(mockFile, fileName) - - rdd <- textFile(sc, fileName) - expect_equal(collect(rdd), as.list(mockFile)) - - unlink(fileName) -}) - -test_that("textFile() word count works as expected", { - fileName <- tempfile(pattern="spark-test", fileext=".tmp") - writeLines(mockFile, fileName) - - rdd <- textFile(sc, fileName) - - words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] }) - wordCount <- lapply(words, function(word) { list(word, 1L) }) - - counts <- reduceByKey(wordCount, "+", 2L) - output <- collect(counts) - expected <- list(list("pretty.", 1), list("is", 2), list("awesome.", 1), - list("Spark", 2)) - expect_equal(sortKeyValueList(output), sortKeyValueList(expected)) - - unlink(fileName) -}) - -test_that("several transformations on RDD created by textFile()", { - fileName <- tempfile(pattern="spark-test", fileext=".tmp") - writeLines(mockFile, fileName) - - rdd <- textFile(sc, fileName) # RDD - for (i in 1:10) { - # PipelinedRDD initially created from RDD - rdd <- lapply(rdd, function(x) paste(x, x)) - } - collect(rdd) - - unlink(fileName) -}) - -test_that("textFile() followed by a saveAsTextFile() returns the same content", { - fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") - fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") - writeLines(mockFile, fileName1) - - rdd <- textFile(sc, fileName1, 1L) - saveAsTextFile(rdd, fileName2) - rdd <- textFile(sc, fileName2) - expect_equal(collect(rdd), as.list(mockFile)) - - unlink(fileName1) - unlink(fileName2) -}) - -test_that("saveAsTextFile() on a parallelized list works as expected", { - fileName <- tempfile(pattern="spark-test", fileext=".tmp") - l <- list(1, 2, 3) - rdd <- parallelize(sc, l, 1L) - saveAsTextFile(rdd, fileName) - rdd <- textFile(sc, fileName) - expect_equal(collect(rdd), lapply(l, function(x) {toString(x)})) - - unlink(fileName) -}) - -test_that("textFile() and saveAsTextFile() word count works as expected", { - fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") - fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") - writeLines(mockFile, fileName1) - - rdd <- textFile(sc, fileName1) - - words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] }) - wordCount <- lapply(words, function(word) { list(word, 1L) }) - - counts <- reduceByKey(wordCount, "+", 2L) - - saveAsTextFile(counts, fileName2) - rdd <- textFile(sc, fileName2) - - output <- collect(rdd) - expected <- list(list("awesome.", 1), list("Spark", 2), - list("pretty.", 1), list("is", 2)) - expectedStr <- lapply(expected, function(x) { toString(x) }) - expect_equal(sortKeyValueList(output), sortKeyValueList(expectedStr)) - - unlink(fileName1) - unlink(fileName2) -}) - -test_that("textFile() on multiple paths", { - fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") - fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") - writeLines("Spark is pretty.", fileName1) - writeLines("Spark is awesome.", fileName2) - - rdd <- textFile(sc, c(fileName1, fileName2)) - expect_equal(count(rdd), 2) - - unlink(fileName1) - unlink(fileName2) -}) - -test_that("Pipelined operations on RDDs created using textFile", { - fileName <- tempfile(pattern="spark-test", fileext=".tmp") - writeLines(mockFile, fileName) - - rdd <- textFile(sc, fileName) - - lengths <- lapply(rdd, function(x) { length(x) }) - expect_equal(collect(lengths), list(1, 1)) - - lengthsPipelined <- lapply(lengths, function(x) { x + 10 }) - expect_equal(collect(lengthsPipelined), list(11, 11)) - - lengths30 <- lapply(lengthsPipelined, function(x) { x + 20 }) - expect_equal(collect(lengths30), list(31, 31)) - - lengths20 <- lapply(lengths, function(x) { x + 20 }) - expect_equal(collect(lengths20), list(21, 21)) - - unlink(fileName) -}) diff --git a/R/pkg/inst/tests/test_utils.R b/R/pkg/inst/tests/test_utils.R deleted file mode 100644 index 12df4cf4f65b..000000000000 --- a/R/pkg/inst/tests/test_utils.R +++ /dev/null @@ -1,140 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -context("functions in utils.R") - -# JavaSparkContext handle -sc <- sparkR.init() - -test_that("convertJListToRList() gives back (deserializes) the original JLists - of strings and integers", { - # It's hard to manually create a Java List using rJava, since it does not - # support generics well. Instead, we rely on collect() returning a - # JList. - nums <- as.list(1:10) - rdd <- parallelize(sc, nums, 1L) - jList <- callJMethod(rdd@jrdd, "collect") - rList <- convertJListToRList(jList, flatten = TRUE) - expect_equal(rList, nums) - - strs <- as.list("hello", "spark") - rdd <- parallelize(sc, strs, 2L) - jList <- callJMethod(rdd@jrdd, "collect") - rList <- convertJListToRList(jList, flatten = TRUE) - expect_equal(rList, strs) -}) - -test_that("serializeToBytes on RDD", { - # File content - mockFile <- c("Spark is pretty.", "Spark is awesome.") - fileName <- tempfile(pattern="spark-test", fileext=".tmp") - writeLines(mockFile, fileName) - - text.rdd <- textFile(sc, fileName) - expect_equal(getSerializedMode(text.rdd), "string") - ser.rdd <- serializeToBytes(text.rdd) - expect_equal(collect(ser.rdd), as.list(mockFile)) - expect_equal(getSerializedMode(ser.rdd), "byte") - - unlink(fileName) -}) - -test_that("cleanClosure on R functions", { - y <- c(1, 2, 3) - g <- function(x) { x + 1 } - f <- function(x) { g(x) + y } - newF <- cleanClosure(f) - env <- environment(newF) - expect_equal(length(ls(env)), 2) # y, g - actual <- get("y", envir = env, inherits = FALSE) - expect_equal(actual, y) - actual <- get("g", envir = env, inherits = FALSE) - expect_equal(actual, g) - - # Test for nested enclosures and package variables. - env2 <- new.env() - funcEnv <- new.env(parent = env2) - f <- function(x) { log(g(x) + y) } - environment(f) <- funcEnv # enclosing relationship: f -> funcEnv -> env2 -> .GlobalEnv - newF <- cleanClosure(f) - env <- environment(newF) - expect_equal(length(ls(env)), 2) # "min" should not be included - actual <- get("y", envir = env, inherits = FALSE) - expect_equal(actual, y) - actual <- get("g", envir = env, inherits = FALSE) - expect_equal(actual, g) - - base <- c(1, 2, 3) - l <- list(field = matrix(1)) - field <- matrix(2) - defUse <- 3 - g <- function(x) { x + y } - f <- function(x) { - defUse <- base::as.integer(x) + 1 # Test for access operators `::`. - lapply(x, g) + 1 # Test for capturing function call "g"'s closure as a argument of lapply. - l$field[1,1] <- 3 # Test for access operators `$`. - res <- defUse + l$field[1,] # Test for def-use chain of "defUse", and "" symbol. - f(res) # Test for recursive calls. - } - newF <- cleanClosure(f) - env <- environment(newF) - # TODO(shivaram): length(ls(env)) is 4 here for some reason and `lapply` is included in `env`. - # Disabling this test till we debug this. - # - # expect_equal(length(ls(env)), 3) # Only "g", "l" and "f". No "base", "field" or "defUse". - expect_true("g" %in% ls(env)) - expect_true("l" %in% ls(env)) - expect_true("f" %in% ls(env)) - expect_equal(get("l", envir = env, inherits = FALSE), l) - # "y" should be in the environemnt of g. - newG <- get("g", envir = env, inherits = FALSE) - env <- environment(newG) - expect_equal(length(ls(env)), 1) - actual <- get("y", envir = env, inherits = FALSE) - expect_equal(actual, y) - - # Test for function (and variable) definitions. - f <- function(x) { - g <- function(y) { y * 2 } - g(x) - } - newF <- cleanClosure(f) - env <- environment(newF) - expect_equal(length(ls(env)), 0) # "y" and "g" should not be included. - - # Test for overriding variables in base namespace (Issue: SparkR-196). - nums <- as.list(1:10) - rdd <- parallelize(sc, nums, 2L) - t <- 4 # Override base::t in .GlobalEnv. - f <- function(x) { x > t } - newF <- cleanClosure(f) - env <- environment(newF) - expect_equal(ls(env), "t") - expect_equal(get("t", envir = env, inherits = FALSE), t) - actual <- collect(lapply(rdd, f)) - expected <- as.list(c(rep(FALSE, 4), rep(TRUE, 6))) - expect_equal(actual, expected) - - # Test for broadcast variables. - a <- matrix(nrow=10, ncol=10, data=rnorm(100)) - aBroadcast <- broadcast(sc, a) - normMultiply <- function(x) { norm(aBroadcast$value) * x } - newnormMultiply <- SparkR:::cleanClosure(normMultiply) - env <- environment(newnormMultiply) - expect_equal(ls(env), "aBroadcast") - expect_equal(get("aBroadcast", envir = env, inherits = FALSE), aBroadcast) -}) diff --git a/R/pkg/inst/tests/testthat/test_basic.R b/R/pkg/inst/tests/testthat/test_basic.R new file mode 100644 index 000000000000..de47162d5325 --- /dev/null +++ b/R/pkg/inst/tests/testthat/test_basic.R @@ -0,0 +1,90 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("basic tests for CRAN") + +test_that("create DataFrame from list or data.frame", { + sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + + i <- 4 + df <- createDataFrame(data.frame(dummy = 1:i)) + expect_equal(count(df), i) + + l <- list(list(a = 1, b = 2), list(a = 3, b = 4)) + df <- createDataFrame(l) + expect_equal(columns(df), c("a", "b")) + + a <- 1:3 + b <- c("a", "b", "c") + ldf <- data.frame(a, b) + df <- createDataFrame(ldf) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + expect_equal(count(df), 3) + ldf2 <- collect(df) + expect_equal(ldf$a, ldf2$a) + + mtcarsdf <- createDataFrame(mtcars) + expect_equivalent(collect(mtcarsdf), mtcars) + + bytes <- as.raw(c(1, 2, 3)) + df <- createDataFrame(list(list(bytes))) + expect_equal(collect(df)[[1]][[1]], bytes) + + sparkR.session.stop() +}) + +test_that("spark.glm and predict", { + sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + + training <- suppressWarnings(createDataFrame(iris)) + # gaussian family + model <- spark.glm(training, Sepal_Width ~ Sepal_Length + Species) + prediction <- predict(model, training) + expect_equal(typeof(take(select(prediction, "prediction"), 1)$prediction), "double") + vals <- collect(select(prediction, "prediction")) + rVals <- predict(glm(Sepal.Width ~ Sepal.Length + Species, data = iris), iris) + expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) + + # Gamma family + x <- runif(100, -1, 1) + y <- rgamma(100, rate = 10 / exp(0.5 + 1.2 * x), shape = 10) + df <- as.DataFrame(as.data.frame(list(x = x, y = y))) + model <- glm(y ~ x, family = Gamma, df) + out <- capture.output(print(summary(model))) + expect_true(any(grepl("Dispersion parameter for gamma family", out))) + + # tweedie family + model <- spark.glm(training, Sepal_Width ~ Sepal_Length + Species, + family = "tweedie", var.power = 1.2, link.power = 0.0) + prediction <- predict(model, training) + expect_equal(typeof(take(select(prediction, "prediction"), 1)$prediction), "double") + vals <- collect(select(prediction, "prediction")) + + # manual calculation of the R predicted values to avoid dependence on statmod + #' library(statmod) + #' rModel <- glm(Sepal.Width ~ Sepal.Length + Species, data = iris, + #' family = tweedie(var.power = 1.2, link.power = 0.0)) + #' print(coef(rModel)) + + rCoef <- c(0.6455409, 0.1169143, -0.3224752, -0.3282174) + rVals <- exp(as.numeric(model.matrix(Sepal.Width ~ Sepal.Length + Species, + data = iris) %*% rCoef)) + expect_true(all(abs(rVals - vals) < 1e-5), rVals - vals) + + sparkR.session.stop() +}) diff --git a/R/pkg/inst/worker/daemon.R b/R/pkg/inst/worker/daemon.R index 3584b418a71a..2e31dc5f728c 100644 --- a/R/pkg/inst/worker/daemon.R +++ b/R/pkg/inst/worker/daemon.R @@ -18,17 +18,62 @@ # Worker daemon rLibDir <- Sys.getenv("SPARKR_RLIBDIR") -script <- paste(rLibDir, "SparkR/worker/worker.R", sep = "/") +connectionTimeout <- as.integer(Sys.getenv("SPARKR_BACKEND_CONNECTION_TIMEOUT", "6000")) +dirs <- strsplit(rLibDir, ",")[[1]] +script <- file.path(dirs[[1]], "SparkR", "worker", "worker.R") # preload SparkR package, speedup worker -.libPaths(c(rLibDir, .libPaths())) +.libPaths(c(dirs, .libPaths())) suppressPackageStartupMessages(library(SparkR)) port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) -inputCon <- socketConnection(port = port, open = "rb", blocking = TRUE, timeout = 3600) +inputCon <- socketConnection( + port = port, open = "rb", blocking = TRUE, timeout = connectionTimeout) + +# Waits indefinitely for a socket connecion by default. +selectTimeout <- NULL while (TRUE) { - ready <- socketSelect(list(inputCon)) + ready <- socketSelect(list(inputCon), timeout = selectTimeout) + + # Note that the children should be terminated in the parent. If each child terminates + # itself, it appears that the resource is not released properly, that causes an unexpected + # termination of this daemon due to, for example, running out of file descriptors + # (see SPARK-21093). Therefore, the current implementation tries to retrieve children + # that are exited (but not terminated) and then sends a kill signal to terminate them properly + # in the parent. + # + # There are two paths that it attempts to send a signal to terminate the children in the parent. + # + # 1. Every second if any socket connection is not available and if there are child workers + # running. + # 2. Right after a socket connection is available. + # + # In other words, the parent attempts to send the signal to the children every second if + # any worker is running or right before launching other worker children from the following + # new socket connection. + + # The process IDs of exited children are returned below. + children <- parallel:::selectChildren(timeout = 0) + + if (is.integer(children)) { + lapply(children, function(child) { + # This should be the PIDs of exited children. Otherwise, this returns raw bytes if any data + # was sent from this child. In this case, we discard it. + pid <- parallel:::readChild(child) + if (is.integer(pid)) { + # This checks if the data from this child is the same pid of this selected child. + if (child == pid) { + # If so, we terminate this child. + tools::pskill(child, tools::SIGUSR1) + } + } + }) + } else if (is.null(children)) { + # If it is NULL, there are no children. Waits indefinitely for a socket connecion. + selectTimeout <- NULL + } + if (ready) { port <- SparkR:::readInt(inputCon) # There is a small chance that it could be interrupted by signal, retry one time @@ -41,12 +86,15 @@ while (TRUE) { } p <- parallel:::mcfork() if (inherits(p, "masterProcess")) { + # Reach here because this is a child process. close(inputCon) Sys.setenv(SPARKR_WORKER_PORT = port) - source(script) - # Set SIGUSR1 so that child can exit - tools::pskill(Sys.getpid(), tools::SIGUSR1) + try(source(script)) + # Note that this mcexit does not fully terminate this child. parallel:::mcexit(0L) + } else { + # Forking succeeded and we need to check if they finished their jobs every second. + selectTimeout <- 1 } } } diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index 0c3b0d1f4be2..03e745014786 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -27,6 +27,61 @@ elapsedSecs <- function() { proc.time()[3] } +compute <- function(mode, partition, serializer, deserializer, key, + colNames, computeFunc, inputData) { + if (mode > 0) { + if (deserializer == "row") { + # Transform the list of rows into a data.frame + # Note that the optional argument stringsAsFactors for rbind is + # available since R 3.2.4. So we set the global option here. + oldOpt <- getOption("stringsAsFactors") + options(stringsAsFactors = FALSE) + + # Handle binary data types + if ("raw" %in% sapply(inputData[[1]], class)) { + inputData <- SparkR:::rbindRaws(inputData) + } else { + inputData <- do.call(rbind.data.frame, inputData) + } + + options(stringsAsFactors = oldOpt) + + names(inputData) <- colNames + } else { + # Check to see if inputData is a valid data.frame + stopifnot(deserializer == "byte") + stopifnot(class(inputData) == "data.frame") + } + + if (mode == 2) { + output <- computeFunc(key, inputData) + } else { + output <- computeFunc(inputData) + } + if (serializer == "row") { + # Transform the result data.frame back to a list of rows + output <- split(output, seq(nrow(output))) + } else { + # Serialize the ouput to a byte array + stopifnot(serializer == "byte") + } + } else { + output <- computeFunc(partition, inputData) + } + return (output) +} + +outputResult <- function(serializer, output, outputCon) { + if (serializer == "byte") { + SparkR:::writeRawSerialize(outputCon, output) + } else if (serializer == "row") { + SparkR:::writeRowSerialize(outputCon, output) + } else { + # write lines one-by-one with flag + lapply(output, function(line) SparkR:::writeString(outputCon, line)) + } +} + # Constants specialLengths <- list(END_OF_STERAM = 0L, TIMING_DATA = -1L) @@ -35,15 +90,19 @@ bootTime <- currentTimeSecs() bootElap <- elapsedSecs() rLibDir <- Sys.getenv("SPARKR_RLIBDIR") +connectionTimeout <- as.integer(Sys.getenv("SPARKR_BACKEND_CONNECTION_TIMEOUT", "6000")) +dirs <- strsplit(rLibDir, ",")[[1]] # Set libPaths to include SparkR package as loadNamespace needs this # TODO: Figure out if we can avoid this by not loading any objects that require # SparkR namespace -.libPaths(c(rLibDir, .libPaths())) +.libPaths(c(dirs, .libPaths())) suppressPackageStartupMessages(library(SparkR)) port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) -inputCon <- socketConnection(port = port, blocking = TRUE, open = "rb") -outputCon <- socketConnection(port = port, blocking = TRUE, open = "wb") +inputCon <- socketConnection( + port = port, blocking = TRUE, open = "rb", timeout = connectionTimeout) +outputCon <- socketConnection( + port = port, blocking = TRUE, open = "wb", timeout = connectionTimeout) # read the index of the current partition inside the RDD partition <- SparkR:::readInt(inputCon) @@ -54,7 +113,7 @@ serializer <- SparkR:::readString(inputCon) # Include packages as required packageNames <- unserialize(SparkR:::readRaw(inputCon)) for (pkg in packageNames) { - suppressPackageStartupMessages(library(as.character(pkg), character.only=TRUE)) + suppressPackageStartupMessages(library(as.character(pkg), character.only = TRUE)) } # read function dependencies @@ -78,41 +137,71 @@ if (numBroadcastVars > 0) { # Timing broadcast broadcastElap <- elapsedSecs() +# Initial input timing +inputElap <- broadcastElap # If -1: read as normal RDD; if >= 0, treat as pairwise RDD and treat the int # as number of partitions to create. numPartitions <- SparkR:::readInt(inputCon) +# 0 - RDD mode, 1 - dapply mode, 2 - gapply mode +mode <- SparkR:::readInt(inputCon) + +if (mode > 0) { + colNames <- SparkR:::readObject(inputCon) +} + isEmpty <- SparkR:::readInt(inputCon) +computeInputElapsDiff <- 0 +outputComputeElapsDiff <- 0 if (isEmpty != 0) { - if (numPartitions == -1) { if (deserializer == "byte") { # Now read as many characters as described in funcLen data <- SparkR:::readDeserialize(inputCon) } else if (deserializer == "string") { data <- as.list(readLines(inputCon)) + } else if (deserializer == "row" && mode == 2) { + dataWithKeys <- SparkR:::readMultipleObjectsWithKeys(inputCon) + keys <- dataWithKeys$keys + data <- dataWithKeys$data } else if (deserializer == "row") { data <- SparkR:::readMultipleObjects(inputCon) } + # Timing reading input data for execution inputElap <- elapsedSecs() - - output <- computeFunc(partition, data) - # Timing computing - computeElap <- elapsedSecs() - - if (serializer == "byte") { - SparkR:::writeRawSerialize(outputCon, output) - } else if (serializer == "row") { - SparkR:::writeRowSerialize(outputCon, output) + if (mode > 0) { + if (mode == 1) { + output <- compute(mode, partition, serializer, deserializer, NULL, + colNames, computeFunc, data) + } else { + # gapply mode + for (i in 1:length(data)) { + # Timing reading input data for execution + inputElap <- elapsedSecs() + output <- compute(mode, partition, serializer, deserializer, keys[[i]], + colNames, computeFunc, data[[i]]) + computeElap <- elapsedSecs() + outputResult(serializer, output, outputCon) + outputElap <- elapsedSecs() + computeInputElapsDiff <- computeInputElapsDiff + (computeElap - inputElap) + outputComputeElapsDiff <- outputComputeElapsDiff + (outputElap - computeElap) + } + } } else { - # write lines one-by-one with flag - lapply(output, function(line) SparkR:::writeString(outputCon, line)) + output <- compute(mode, partition, serializer, deserializer, NULL, + colNames, computeFunc, data) + } + if (mode != 2) { + # Not a gapply mode + computeElap <- elapsedSecs() + outputResult(serializer, output, outputCon) + outputElap <- elapsedSecs() + computeInputElapsDiff <- computeElap - inputElap + outputComputeElapsDiff <- outputElap - computeElap } - # Timing output - outputElap <- elapsedSecs() } else { if (deserializer == "byte") { # Now read as many characters as described in funcLen @@ -154,11 +243,9 @@ if (isEmpty != 0) { } # Timing output outputElap <- elapsedSecs() + computeInputElapsDiff <- computeElap - inputElap + outputComputeElapsDiff <- outputElap - computeElap } -} else { - inputElap <- broadcastElap - computeElap <- broadcastElap - outputElap <- broadcastElap } # Report timing @@ -167,8 +254,8 @@ SparkR:::writeDouble(outputCon, bootTime) SparkR:::writeDouble(outputCon, initElap - bootElap) # init SparkR:::writeDouble(outputCon, broadcastElap - initElap) # broadcast SparkR:::writeDouble(outputCon, inputElap - broadcastElap) # input -SparkR:::writeDouble(outputCon, computeElap - inputElap) # compute -SparkR:::writeDouble(outputCon, outputElap - computeElap) # output +SparkR:::writeDouble(outputCon, computeInputElapsDiff) # compute +SparkR:::writeDouble(outputCon, outputComputeElapsDiff) # output # End of output SparkR:::writeInt(outputCon, specialLengths$END_OF_STERAM) diff --git a/R/pkg/inst/tests/jarTest.R b/R/pkg/tests/fulltests/jarTest.R similarity index 79% rename from R/pkg/inst/tests/jarTest.R rename to R/pkg/tests/fulltests/jarTest.R index d68bb20950b0..e2241e03b55f 100644 --- a/R/pkg/inst/tests/jarTest.R +++ b/R/pkg/tests/fulltests/jarTest.R @@ -16,17 +16,17 @@ # library(SparkR) -sc <- sparkR.init() +sc <- sparkR.session(master = "local[1]") -helloTest <- SparkR:::callJStatic("sparkR.test.hello", +helloTest <- SparkR:::callJStatic("sparkrtest.DummyClass", "helloWorld", "Dave") +stopifnot(identical(helloTest, "Hello Dave")) -basicFunction <- SparkR:::callJStatic("sparkR.test.basicFunction", +basicFunction <- SparkR:::callJStatic("sparkrtest.DummyClass", "addStuff", 2L, 2L) +stopifnot(basicFunction == 4L) -sparkR.stop() -output <- c(helloTest, basicFunction) -writeLines(output) +sparkR.session.stop() diff --git a/R/pkg/inst/tests/packageInAJarTest.R b/R/pkg/tests/fulltests/packageInAJarTest.R similarity index 85% rename from R/pkg/inst/tests/packageInAJarTest.R rename to R/pkg/tests/fulltests/packageInAJarTest.R index 207a37a0cb47..ac706261999f 100644 --- a/R/pkg/inst/tests/packageInAJarTest.R +++ b/R/pkg/tests/fulltests/packageInAJarTest.R @@ -17,14 +17,14 @@ library(SparkR) library(sparkPackageTest) -sc <- sparkR.init() +sparkR.session(master = "local[1]") run1 <- myfunc(5L) run2 <- myfunc(-4L) -sparkR.stop() +sparkR.session.stop() -if(run1 != 6) quit(save = "no", status = 1) +if (run1 != 6) quit(save = "no", status = 1) -if(run2 != -3) quit(save = "no", status = 1) +if (run2 != -3) quit(save = "no", status = 1) diff --git a/R/pkg/inst/tests/test_Serde.R b/R/pkg/tests/fulltests/test_Serde.R similarity index 95% rename from R/pkg/inst/tests/test_Serde.R rename to R/pkg/tests/fulltests/test_Serde.R index dddce54d7044..6bbd201bf1d8 100644 --- a/R/pkg/inst/tests/test_Serde.R +++ b/R/pkg/tests/fulltests/test_Serde.R @@ -17,7 +17,7 @@ context("SerDe functionality") -sc <- sparkR.init() +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) test_that("SerDe of primitive types", { x <- callJStatic("SparkRHandler", "echo", 1L) @@ -75,3 +75,5 @@ test_that("SerDe of list of lists", { y <- callJStatic("SparkRHandler", "echo", x) expect_equal(x, y) }) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_Windows.R b/R/pkg/tests/fulltests/test_Windows.R new file mode 100644 index 000000000000..b2ec6c67311d --- /dev/null +++ b/R/pkg/tests/fulltests/test_Windows.R @@ -0,0 +1,27 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +context("Windows-specific tests") + +test_that("sparkJars tag in SparkContext", { + if (!is_windows()) { + skip("This test is only for Windows, skipped") + } + + testOutput <- launchScript("ECHO", "a/b/c", wait = TRUE) + abcPath <- testOutput[1] + expect_equal(abcPath, "a\\b\\c") +}) diff --git a/R/pkg/inst/tests/test_binaryFile.R b/R/pkg/tests/fulltests/test_binaryFile.R similarity index 75% rename from R/pkg/inst/tests/test_binaryFile.R rename to R/pkg/tests/fulltests/test_binaryFile.R index f2452ed97d2e..758b174b8787 100644 --- a/R/pkg/inst/tests/test_binaryFile.R +++ b/R/pkg/tests/fulltests/test_binaryFile.R @@ -18,39 +18,40 @@ context("functions on binary files") # JavaSparkContext handle -sc <- sparkR.init() +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) mockFile <- c("Spark is pretty.", "Spark is awesome.") test_that("saveAsObjectFile()/objectFile() following textFile() works", { - fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") - fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName1 <- tempfile(pattern = "spark-test", fileext = ".tmp") + fileName2 <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName1) rdd <- textFile(sc, fileName1, 1) saveAsObjectFile(rdd, fileName2) rdd <- objectFile(sc, fileName2) - expect_equal(collect(rdd), as.list(mockFile)) + expect_equal(collectRDD(rdd), as.list(mockFile)) unlink(fileName1) unlink(fileName2, recursive = TRUE) }) test_that("saveAsObjectFile()/objectFile() works on a parallelized list", { - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") l <- list(1, 2, 3) rdd <- parallelize(sc, l, 1) saveAsObjectFile(rdd, fileName) rdd <- objectFile(sc, fileName) - expect_equal(collect(rdd), l) + expect_equal(collectRDD(rdd), l) unlink(fileName, recursive = TRUE) }) test_that("saveAsObjectFile()/objectFile() following RDD transformations works", { - fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") - fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName1 <- tempfile(pattern = "spark-test", fileext = ".tmp") + fileName2 <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName1) rdd <- textFile(sc, fileName1) @@ -63,7 +64,7 @@ test_that("saveAsObjectFile()/objectFile() following RDD transformations works", saveAsObjectFile(counts, fileName2) counts <- objectFile(sc, fileName2) - output <- collect(counts) + output <- collectRDD(counts) expected <- list(list("awesome.", 1), list("Spark", 2), list("pretty.", 1), list("is", 2)) expect_equal(sortKeyValueList(output), sortKeyValueList(expected)) @@ -73,8 +74,8 @@ test_that("saveAsObjectFile()/objectFile() following RDD transformations works", }) test_that("saveAsObjectFile()/objectFile() works with multiple paths", { - fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") - fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName1 <- tempfile(pattern = "spark-test", fileext = ".tmp") + fileName2 <- tempfile(pattern = "spark-test", fileext = ".tmp") rdd1 <- parallelize(sc, "Spark is pretty.") saveAsObjectFile(rdd1, fileName1) @@ -82,8 +83,10 @@ test_that("saveAsObjectFile()/objectFile() works with multiple paths", { saveAsObjectFile(rdd2, fileName2) rdd <- objectFile(sc, c(fileName1, fileName2)) - expect_equal(count(rdd), 2) + expect_equal(countRDD(rdd), 2) unlink(fileName1, recursive = TRUE) unlink(fileName2, recursive = TRUE) }) + +sparkR.session.stop() diff --git a/R/pkg/inst/tests/test_binary_function.R b/R/pkg/tests/fulltests/test_binary_function.R similarity index 79% rename from R/pkg/inst/tests/test_binary_function.R rename to R/pkg/tests/fulltests/test_binary_function.R index f054ac9a87d6..442bed509bb1 100644 --- a/R/pkg/inst/tests/test_binary_function.R +++ b/R/pkg/tests/fulltests/test_binary_function.R @@ -18,7 +18,8 @@ context("binary functions") # JavaSparkContext handle -sc <- sparkR.init() +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) # Data nums <- 1:10 @@ -28,21 +29,21 @@ rdd <- parallelize(sc, nums, 2L) mockFile <- c("Spark is pretty.", "Spark is awesome.") test_that("union on two RDDs", { - actual <- collect(unionRDD(rdd, rdd)) + actual <- collectRDD(unionRDD(rdd, rdd)) expect_equal(actual, as.list(rep(nums, 2))) - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName) text.rdd <- textFile(sc, fileName) union.rdd <- unionRDD(rdd, text.rdd) - actual <- collect(union.rdd) + actual <- collectRDD(union.rdd) expect_equal(actual, c(as.list(nums), mockFile)) expect_equal(getSerializedMode(union.rdd), "byte") rdd <- map(text.rdd, function(x) {x}) union.rdd <- unionRDD(rdd, text.rdd) - actual <- collect(union.rdd) + actual <- collectRDD(union.rdd) expect_equal(actual, as.list(c(mockFile, mockFile))) expect_equal(getSerializedMode(union.rdd), "byte") @@ -53,14 +54,14 @@ test_that("cogroup on two RDDs", { rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) cogroup.rdd <- cogroup(rdd1, rdd2, numPartitions = 2L) - actual <- collect(cogroup.rdd) + actual <- collectRDD(cogroup.rdd) expect_equal(actual, list(list(1, list(list(1), list(2, 3))), list(2, list(list(4), list())))) rdd1 <- parallelize(sc, list(list("a", 1), list("a", 4))) rdd2 <- parallelize(sc, list(list("b", 2), list("a", 3))) cogroup.rdd <- cogroup(rdd1, rdd2, numPartitions = 2L) - actual <- collect(cogroup.rdd) + actual <- collectRDD(cogroup.rdd) expected <- list(list("b", list(list(), list(2))), list("a", list(list(1, 4), list(3)))) expect_equal(sortKeyValueList(actual), @@ -71,31 +72,33 @@ test_that("zipPartitions() on RDDs", { rdd1 <- parallelize(sc, 1:2, 2L) # 1, 2 rdd2 <- parallelize(sc, 1:4, 2L) # 1:2, 3:4 rdd3 <- parallelize(sc, 1:6, 2L) # 1:3, 4:6 - actual <- collect(zipPartitions(rdd1, rdd2, rdd3, + actual <- collectRDD(zipPartitions(rdd1, rdd2, rdd3, func = function(x, y, z) { list(list(x, y, z))} )) expect_equal(actual, - list(list(1, c(1,2), c(1,2,3)), list(2, c(3,4), c(4,5,6)))) + list(list(1, c(1, 2), c(1, 2, 3)), list(2, c(3, 4), c(4, 5, 6)))) mockFile <- c("Spark is pretty.", "Spark is awesome.") - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName) rdd <- textFile(sc, fileName, 1) - actual <- collect(zipPartitions(rdd, rdd, + actual <- collectRDD(zipPartitions(rdd, rdd, func = function(x, y) { list(paste(x, y, sep = "\n")) })) expected <- list(paste(mockFile, mockFile, sep = "\n")) expect_equal(actual, expected) rdd1 <- parallelize(sc, 0:1, 1) - actual <- collect(zipPartitions(rdd1, rdd, + actual <- collectRDD(zipPartitions(rdd1, rdd, func = function(x, y) { list(x + nchar(y)) })) expected <- list(0:1 + nchar(mockFile)) expect_equal(actual, expected) rdd <- map(rdd, function(x) { x }) - actual <- collect(zipPartitions(rdd, rdd1, + actual <- collectRDD(zipPartitions(rdd, rdd1, func = function(x, y) { list(y + nchar(x)) })) expect_equal(actual, expected) unlink(fileName) }) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_broadcast.R b/R/pkg/tests/fulltests/test_broadcast.R new file mode 100644 index 000000000000..fc2c7c2deb82 --- /dev/null +++ b/R/pkg/tests/fulltests/test_broadcast.R @@ -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. +# + +context("broadcast variables") + +# JavaSparkContext handle +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) + +# Partitioned data +nums <- 1:2 +rrdd <- parallelize(sc, nums, 2L) + +test_that("using broadcast variable", { + randomMat <- matrix(nrow = 10, ncol = 10, data = rnorm(100)) + randomMatBr <- broadcastRDD(sc, randomMat) + + useBroadcast <- function(x) { + sum(SparkR:::value(randomMatBr) * x) + } + actual <- collectRDD(lapply(rrdd, useBroadcast)) + expected <- list(sum(randomMat) * 1, sum(randomMat) * 2) + expect_equal(actual, expected) +}) + +test_that("without using broadcast variable", { + randomMat <- matrix(nrow = 10, ncol = 10, data = rnorm(100)) + + useBroadcast <- function(x) { + sum(randomMat * x) + } + actual <- collectRDD(lapply(rrdd, useBroadcast)) + expected <- list(sum(randomMat) * 1, sum(randomMat) * 2) + expect_equal(actual, expected) +}) + +sparkR.session.stop() diff --git a/R/pkg/inst/tests/test_client.R b/R/pkg/tests/fulltests/test_client.R similarity index 76% rename from R/pkg/inst/tests/test_client.R rename to R/pkg/tests/fulltests/test_client.R index 8a20991f89af..de624b572cc2 100644 --- a/R/pkg/inst/tests/test_client.R +++ b/R/pkg/tests/fulltests/test_client.R @@ -32,5 +32,12 @@ test_that("no package specified doesn't add packages flag", { }) test_that("multiple packages don't produce a warning", { - expect_that(generateSparkSubmitArgs("", "", "", "", c("A", "B")), not(gives_warning())) + expect_warning(generateSparkSubmitArgs("", "", "", "", c("A", "B")), NA) +}) + +test_that("sparkJars sparkPackages as character vectors", { + args <- generateSparkSubmitArgs("", "", c("one.jar", "two.jar", "three.jar"), "", + c("com.databricks:spark-avro_2.11:2.0.1")) + expect_match(args, "--jars one.jar,two.jar,three.jar") + expect_match(args, "--packages com.databricks:spark-avro_2.11:2.0.1") }) diff --git a/R/pkg/tests/fulltests/test_context.R b/R/pkg/tests/fulltests/test_context.R new file mode 100644 index 000000000000..77635c5a256b --- /dev/null +++ b/R/pkg/tests/fulltests/test_context.R @@ -0,0 +1,211 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("test functions in sparkR.R") + +test_that("Check masked functions", { + # Check that we are not masking any new function from base, stats, testthat unexpectedly + # NOTE: We should avoid adding entries to *namesOfMaskedCompletely* as masked functions make it + # hard for users to use base R functions. Please check when in doubt. + namesOfMaskedCompletely <- c("cov", "filter", "sample", "not") + namesOfMasked <- c("describe", "cov", "filter", "lag", "na.omit", "predict", "sd", "var", + "colnames", "colnames<-", "intersect", "rank", "rbind", "sample", "subset", + "summary", "transform", "drop", "window", "as.data.frame", "union", "not") + if (as.numeric(R.version$major) >= 3 && as.numeric(R.version$minor) >= 3) { + namesOfMasked <- c("endsWith", "startsWith", namesOfMasked) + } + masked <- conflicts(detail = TRUE)$`package:SparkR` + expect_true("describe" %in% masked) # only when with testthat.. + func <- lapply(masked, function(x) { capture.output(showMethods(x))[[1]] }) + funcSparkROrEmpty <- grepl("\\(package SparkR\\)$|^$", func) + maskedBySparkR <- masked[funcSparkROrEmpty] + expect_equal(length(maskedBySparkR), length(namesOfMasked)) + # make the 2 lists the same length so expect_equal will print their content + l <- max(length(maskedBySparkR), length(namesOfMasked)) + length(maskedBySparkR) <- l + length(namesOfMasked) <- l + expect_equal(sort(maskedBySparkR, na.last = TRUE), sort(namesOfMasked, na.last = TRUE)) + # above are those reported as masked when `library(SparkR)` + # note that many of these methods are still callable without base:: or stats:: prefix + # there should be a test for each of these, except followings, which are currently "broken" + funcHasAny <- unlist(lapply(masked, function(x) { + any(grepl("=\"ANY\"", capture.output(showMethods(x)[-1]))) + })) + maskedCompletely <- masked[!funcHasAny] + expect_equal(length(maskedCompletely), length(namesOfMaskedCompletely)) + l <- max(length(maskedCompletely), length(namesOfMaskedCompletely)) + length(maskedCompletely) <- l + length(namesOfMaskedCompletely) <- l + expect_equal(sort(maskedCompletely, na.last = TRUE), + sort(namesOfMaskedCompletely, na.last = TRUE)) +}) + +test_that("repeatedly starting and stopping SparkR", { + for (i in 1:4) { + sc <- suppressWarnings(sparkR.init(master = sparkRTestMaster)) + rdd <- parallelize(sc, 1:20, 2L) + expect_equal(countRDD(rdd), 20) + suppressWarnings(sparkR.stop()) + } +}) + +test_that("repeatedly starting and stopping SparkSession", { + for (i in 1:4) { + sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + df <- createDataFrame(data.frame(dummy = 1:i)) + expect_equal(count(df), i) + sparkR.session.stop() + } +}) + +test_that("rdd GC across sparkR.stop", { + sc <- sparkR.sparkContext(master = sparkRTestMaster) # sc should get id 0 + rdd1 <- parallelize(sc, 1:20, 2L) # rdd1 should get id 1 + rdd2 <- parallelize(sc, 1:10, 2L) # rdd2 should get id 2 + sparkR.session.stop() + + sc <- sparkR.sparkContext(master = sparkRTestMaster) # sc should get id 0 again + + # GC rdd1 before creating rdd3 and rdd2 after + rm(rdd1) + gc() + + rdd3 <- parallelize(sc, 1:20, 2L) # rdd3 should get id 1 now + rdd4 <- parallelize(sc, 1:10, 2L) # rdd4 should get id 2 now + + rm(rdd2) + gc() + + countRDD(rdd3) + countRDD(rdd4) + sparkR.session.stop() +}) + +test_that("job group functions can be called", { + sc <- sparkR.sparkContext(master = sparkRTestMaster) + setJobGroup("groupId", "job description", TRUE) + cancelJobGroup("groupId") + clearJobGroup() + setJobDescription("job description") + + suppressWarnings(setJobGroup(sc, "groupId", "job description", TRUE)) + suppressWarnings(cancelJobGroup(sc, "groupId")) + suppressWarnings(clearJobGroup(sc)) + sparkR.session.stop() +}) + +test_that("utility function can be called", { + sparkR.sparkContext(master = sparkRTestMaster) + setLogLevel("ERROR") + sparkR.session.stop() +}) + +test_that("getClientModeSparkSubmitOpts() returns spark-submit args from whitelist", { + e <- new.env() + e[["spark.driver.memory"]] <- "512m" + ops <- getClientModeSparkSubmitOpts("sparkrmain", e) + expect_equal("--driver-memory \"512m\" sparkrmain", ops) + + e[["spark.driver.memory"]] <- "5g" + e[["spark.driver.extraClassPath"]] <- "/opt/class_path" # nolint + e[["spark.driver.extraJavaOptions"]] <- "-XX:+UseCompressedOops -XX:+UseCompressedStrings" + e[["spark.driver.extraLibraryPath"]] <- "/usr/local/hadoop/lib" # nolint + e[["random"]] <- "skipthis" + ops2 <- getClientModeSparkSubmitOpts("sparkr-shell", e) + # nolint start + expect_equal(ops2, paste0("--driver-class-path \"/opt/class_path\" --driver-java-options \"", + "-XX:+UseCompressedOops -XX:+UseCompressedStrings\" --driver-library-path \"", + "/usr/local/hadoop/lib\" --driver-memory \"5g\" sparkr-shell")) + # nolint end + + e[["spark.driver.extraClassPath"]] <- "/" # too short + ops3 <- getClientModeSparkSubmitOpts("--driver-memory 4g sparkr-shell2", e) + # nolint start + expect_equal(ops3, paste0("--driver-java-options \"-XX:+UseCompressedOops ", + "-XX:+UseCompressedStrings\" --driver-library-path \"/usr/local/hadoop/lib\"", + " --driver-memory 4g sparkr-shell2")) + # nolint end +}) + +test_that("sparkJars sparkPackages as comma-separated strings", { + expect_warning(processSparkJars(" a, b ")) + jars <- suppressWarnings(processSparkJars(" a, b ")) + expect_equal(lapply(jars, basename), list("a", "b")) + + jars <- suppressWarnings(processSparkJars(" abc ,, def ")) + expect_equal(lapply(jars, basename), list("abc", "def")) + + jars <- suppressWarnings(processSparkJars(c(" abc ,, def ", "", "xyz", " ", "a,b"))) + expect_equal(lapply(jars, basename), list("abc", "def", "xyz", "a", "b")) + + p <- processSparkPackages(c("ghi", "lmn")) + expect_equal(p, c("ghi", "lmn")) + + # check normalizePath + f <- dir()[[1]] + expect_warning(processSparkJars(f), NA) + expect_match(processSparkJars(f), f) +}) + +test_that("spark.lapply should perform simple transforms", { + sparkR.sparkContext(master = sparkRTestMaster) + doubled <- spark.lapply(1:10, function(x) { 2 * x }) + expect_equal(doubled, as.list(2 * 1:10)) + sparkR.session.stop() +}) + +test_that("add and get file to be downloaded with Spark job on every node", { + sparkR.sparkContext(master = sparkRTestMaster) + # Test add file. + path <- tempfile(pattern = "hello", fileext = ".txt") + filename <- basename(path) + words <- "Hello World!" + writeLines(words, path) + spark.addFile(path) + download_path <- spark.getSparkFiles(filename) + expect_equal(readLines(download_path), words) + + # Test spark.getSparkFiles works well on executors. + seq <- seq(from = 1, to = 10, length.out = 5) + f <- function(seq) { spark.getSparkFiles(filename) } + results <- spark.lapply(seq, f) + for (i in 1:5) { expect_equal(basename(results[[i]]), filename) } + + unlink(path) + + # Test add directory recursively. + path <- paste0(tempdir(), "/", "recursive_dir") + dir.create(path) + dir_name <- basename(path) + path1 <- paste0(path, "/", "hello.txt") + file.create(path1) + sub_path <- paste0(path, "/", "sub_hello") + dir.create(sub_path) + path2 <- paste0(sub_path, "/", "sub_hello.txt") + file.create(path2) + words <- "Hello World!" + sub_words <- "Sub Hello World!" + writeLines(words, path1) + writeLines(sub_words, path2) + spark.addFile(path, recursive = TRUE) + download_path1 <- spark.getSparkFiles(paste0(dir_name, "/", "hello.txt")) + expect_equal(readLines(download_path1), words) + download_path2 <- spark.getSparkFiles(paste0(dir_name, "/", "sub_hello/sub_hello.txt")) + expect_equal(readLines(download_path2), sub_words) + unlink(path, recursive = TRUE) + sparkR.session.stop() +}) diff --git a/R/pkg/inst/tests/test_includePackage.R b/R/pkg/tests/fulltests/test_includePackage.R similarity index 86% rename from R/pkg/inst/tests/test_includePackage.R rename to R/pkg/tests/fulltests/test_includePackage.R index 8152b448d087..f4ea0d1b5cb2 100644 --- a/R/pkg/inst/tests/test_includePackage.R +++ b/R/pkg/tests/fulltests/test_includePackage.R @@ -18,7 +18,8 @@ context("include R packages") # JavaSparkContext handle -sc <- sparkR.init() +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) # Partitioned data nums <- 1:2 @@ -36,7 +37,7 @@ test_that("include inside function", { } data <- lapplyPartition(rdd, generateData) - actual <- collect(data) + actual <- collectRDD(data) } }) @@ -52,6 +53,8 @@ test_that("use include package", { includePackage(sc, plyr) data <- lapplyPartition(rdd, generateData) - actual <- collect(data) + actual <- collectRDD(data) } }) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_jvm_api.R b/R/pkg/tests/fulltests/test_jvm_api.R new file mode 100644 index 000000000000..8b3b4f73de17 --- /dev/null +++ b/R/pkg/tests/fulltests/test_jvm_api.R @@ -0,0 +1,36 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("JVM API") + +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + +test_that("Create and call methods on object", { + jarr <- sparkR.newJObject("java.util.ArrayList") + # Add an element to the array + sparkR.callJMethod(jarr, "add", 1L) + # Check if get returns the same element + expect_equal(sparkR.callJMethod(jarr, "get", 0L), 1L) +}) + +test_that("Call static methods", { + # Convert a boolean to a string + strTrue <- sparkR.callJStatic("java.lang.String", "valueOf", TRUE) + expect_equal(strTrue, "true") +}) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_mllib_classification.R b/R/pkg/tests/fulltests/test_mllib_classification.R new file mode 100644 index 000000000000..a4d0397236d1 --- /dev/null +++ b/R/pkg/tests/fulltests/test_mllib_classification.R @@ -0,0 +1,490 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(testthat) + +context("MLlib classification algorithms, except for tree-based algorithms") + +# Tests for MLlib classification algorithms in SparkR +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + +absoluteSparkPath <- function(x) { + sparkHome <- sparkR.conf("spark.home") + file.path(sparkHome, x) +} + +test_that("spark.svmLinear", { + df <- suppressWarnings(createDataFrame(iris)) + training <- df[df$Species %in% c("versicolor", "virginica"), ] + model <- spark.svmLinear(training, Species ~ ., regParam = 0.01, maxIter = 10) + summary <- summary(model) + + # test summary coefficients return matrix type + expect_true(class(summary$coefficients) == "matrix") + expect_true(class(summary$coefficients[, 1]) == "numeric") + + coefs <- summary$coefficients[, "Estimate"] + expected_coefs <- c(-0.06004978, -0.1563083, -0.460648, 0.2276626, 1.055085) + expect_true(all(abs(coefs - expected_coefs) < 0.1)) + + # Test prediction with string label + prediction <- predict(model, training) + expect_equal(typeof(take(select(prediction, "prediction"), 1)$prediction), "character") + expected <- c("versicolor", "versicolor", "versicolor", "virginica", "virginica", + "virginica", "virginica", "virginica", "virginica", "virginica") + expect_equal(sort(as.list(take(select(prediction, "prediction"), 10))[[1]]), expected) + + # Test model save and load + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-svm-linear", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + coefs <- summary(model)$coefficients + coefs2 <- summary(model2)$coefficients + expect_equal(coefs, coefs2) + unlink(modelPath) + } + + # Test prediction with numeric label + label <- c(0.0, 0.0, 0.0, 1.0, 1.0) + feature <- c(1.1419053, 0.9194079, -0.9498666, -1.1069903, 0.2809776) + data <- as.data.frame(cbind(label, feature)) + df <- createDataFrame(data) + model <- spark.svmLinear(df, label ~ feature, regParam = 0.1) + prediction <- collect(select(predict(model, df), "prediction")) + expect_equal(sort(prediction$prediction), c("0.0", "0.0", "0.0", "1.0", "1.0")) + + # Test unseen labels + data <- data.frame(clicked = base::sample(c(0, 1), 10, replace = TRUE), + someString = base::sample(c("this", "that"), 10, replace = TRUE), + stringsAsFactors = FALSE) + trainidxs <- base::sample(nrow(data), nrow(data) * 0.7) + traindf <- as.DataFrame(data[trainidxs, ]) + testdf <- as.DataFrame(rbind(data[-trainidxs, ], c(0, "the other"))) + model <- spark.svmLinear(traindf, clicked ~ ., regParam = 0.1) + predictions <- predict(model, testdf) + expect_error(collect(predictions)) + model <- spark.svmLinear(traindf, clicked ~ ., regParam = 0.1, handleInvalid = "skip") + predictions <- predict(model, testdf) + expect_equal(class(collect(predictions)$clicked[1]), "list") + +}) + +test_that("spark.logit", { + # R code to reproduce the result. + # nolint start + #' library(glmnet) + #' iris.x = as.matrix(iris[, 1:4]) + #' iris.y = as.factor(as.character(iris[, 5])) + #' logit = glmnet(iris.x, iris.y, family="multinomial", alpha=0, lambda=0.5) + #' coef(logit) + # + # $setosa + # 5 x 1 sparse Matrix of class "dgCMatrix" + # s0 + # 1.0981324 + # Sepal.Length -0.2909860 + # Sepal.Width 0.5510907 + # Petal.Length -0.1915217 + # Petal.Width -0.4211946 + # + # $versicolor + # 5 x 1 sparse Matrix of class "dgCMatrix" + # s0 + # 1.520061e+00 + # Sepal.Length 2.524501e-02 + # Sepal.Width -5.310313e-01 + # Petal.Length 3.656543e-02 + # Petal.Width -3.144464e-05 + # + # $virginica + # 5 x 1 sparse Matrix of class "dgCMatrix" + # s0 + # -2.61819385 + # Sepal.Length 0.26574097 + # Sepal.Width -0.02005932 + # Petal.Length 0.15495629 + # Petal.Width 0.42122607 + # nolint end + + # Test multinomial logistic regression againt three classes + df <- suppressWarnings(createDataFrame(iris)) + model <- spark.logit(df, Species ~ ., regParam = 0.5) + summary <- summary(model) + + # test summary coefficients return matrix type + expect_true(class(summary$coefficients) == "matrix") + expect_true(class(summary$coefficients[, 1]) == "numeric") + + versicolorCoefsR <- c(1.52, 0.03, -0.53, 0.04, 0.00) + virginicaCoefsR <- c(-2.62, 0.27, -0.02, 0.16, 0.42) + setosaCoefsR <- c(1.10, -0.29, 0.55, -0.19, -0.42) + versicolorCoefs <- summary$coefficients[, "versicolor"] + virginicaCoefs <- summary$coefficients[, "virginica"] + setosaCoefs <- summary$coefficients[, "setosa"] + expect_true(all(abs(versicolorCoefsR - versicolorCoefs) < 0.1)) + expect_true(all(abs(virginicaCoefsR - virginicaCoefs) < 0.1)) + expect_true(all(abs(setosaCoefs - setosaCoefs) < 0.1)) + + # Test model save and load + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-logit", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + coefs <- summary(model)$coefficients + coefs2 <- summary(model2)$coefficients + expect_equal(coefs, coefs2) + unlink(modelPath) + } + + # R code to reproduce the result. + # nolint start + #' library(glmnet) + #' iris2 <- iris[iris$Species %in% c("versicolor", "virginica"), ] + #' iris.x = as.matrix(iris2[, 1:4]) + #' iris.y = as.factor(as.character(iris2[, 5])) + #' logit = glmnet(iris.x, iris.y, family="multinomial", alpha=0, lambda=0.5) + #' coef(logit) + # + # $versicolor + # 5 x 1 sparse Matrix of class "dgCMatrix" + # s0 + # 3.93844796 + # Sepal.Length -0.13538675 + # Sepal.Width -0.02386443 + # Petal.Length -0.35076451 + # Petal.Width -0.77971954 + # + # $virginica + # 5 x 1 sparse Matrix of class "dgCMatrix" + # s0 + # -3.93844796 + # Sepal.Length 0.13538675 + # Sepal.Width 0.02386443 + # Petal.Length 0.35076451 + # Petal.Width 0.77971954 + # + #' logit = glmnet(iris.x, iris.y, family="binomial", alpha=0, lambda=0.5) + #' coef(logit) + # + # 5 x 1 sparse Matrix of class "dgCMatrix" + # s0 + # (Intercept) -6.0824412 + # Sepal.Length 0.2458260 + # Sepal.Width 0.1642093 + # Petal.Length 0.4759487 + # Petal.Width 1.0383948 + # + # nolint end + + # Test multinomial logistic regression againt two classes + df <- suppressWarnings(createDataFrame(iris)) + training <- df[df$Species %in% c("versicolor", "virginica"), ] + model <- spark.logit(training, Species ~ ., regParam = 0.5, family = "multinomial") + summary <- summary(model) + versicolorCoefsR <- c(3.94, -0.16, -0.02, -0.35, -0.78) + virginicaCoefsR <- c(-3.94, 0.16, -0.02, 0.35, 0.78) + versicolorCoefs <- summary$coefficients[, "versicolor"] + virginicaCoefs <- summary$coefficients[, "virginica"] + expect_true(all(abs(versicolorCoefsR - versicolorCoefs) < 0.1)) + expect_true(all(abs(virginicaCoefsR - virginicaCoefs) < 0.1)) + + # Test binomial logistic regression againt two classes + model <- spark.logit(training, Species ~ ., regParam = 0.5) + summary <- summary(model) + coefsR <- c(-6.08, 0.25, 0.16, 0.48, 1.04) + coefs <- summary$coefficients[, "Estimate"] + expect_true(all(abs(coefsR - coefs) < 0.1)) + + # Test prediction with string label + prediction <- predict(model, training) + expect_equal(typeof(take(select(prediction, "prediction"), 1)$prediction), "character") + expected <- c("versicolor", "versicolor", "virginica", "versicolor", "versicolor", + "versicolor", "versicolor", "versicolor", "versicolor", "versicolor") + expect_equal(as.list(take(select(prediction, "prediction"), 10))[[1]], expected) + + # Test prediction with numeric label + label <- c(0.0, 0.0, 0.0, 1.0, 1.0) + feature <- c(1.1419053, 0.9194079, -0.9498666, -1.1069903, 0.2809776) + data <- as.data.frame(cbind(label, feature)) + df <- createDataFrame(data) + model <- spark.logit(df, label ~ feature) + prediction <- collect(select(predict(model, df), "prediction")) + expect_equal(sort(prediction$prediction), c("0.0", "0.0", "0.0", "1.0", "1.0")) + + # Test prediction with weightCol + weight <- c(2.0, 2.0, 2.0, 1.0, 1.0) + data2 <- as.data.frame(cbind(label, feature, weight)) + df2 <- createDataFrame(data2) + model2 <- spark.logit(df2, label ~ feature, weightCol = "weight") + prediction2 <- collect(select(predict(model2, df2), "prediction")) + expect_equal(sort(prediction2$prediction), c("0.0", "0.0", "0.0", "0.0", "0.0")) + + # Test binomial logistic regression againt two classes with upperBoundsOnCoefficients + # and upperBoundsOnIntercepts + u <- matrix(c(1.0, 0.0, 1.0, 0.0), nrow = 1, ncol = 4) + model <- spark.logit(training, Species ~ ., upperBoundsOnCoefficients = u, + upperBoundsOnIntercepts = 1.0) + summary <- summary(model) + coefsR <- c(-11.13331, 1.00000, 0.00000, 1.00000, 0.00000) + coefs <- summary$coefficients[, "Estimate"] + expect_true(all(abs(coefsR - coefs) < 0.1)) + # Test upperBoundsOnCoefficients should be matrix + expect_error(spark.logit(training, Species ~ ., upperBoundsOnCoefficients = as.array(c(1, 2)), + upperBoundsOnIntercepts = 1.0)) + + # Test binomial logistic regression againt two classes with lowerBoundsOnCoefficients + # and lowerBoundsOnIntercepts + l <- matrix(c(0.0, -1.0, 0.0, -1.0), nrow = 1, ncol = 4) + model <- spark.logit(training, Species ~ ., lowerBoundsOnCoefficients = l, + lowerBoundsOnIntercepts = 0.0) + summary <- summary(model) + coefsR <- c(0, 0, -1, 0, 1.902192) + coefs <- summary$coefficients[, "Estimate"] + expect_true(all(abs(coefsR - coefs) < 0.1)) + # Test lowerBoundsOnCoefficients should be matrix + expect_error(spark.logit(training, Species ~ ., lowerBoundsOnCoefficients = as.array(c(1, 2)), + lowerBoundsOnIntercepts = 0.0)) + + # Test multinomial logistic regression with lowerBoundsOnCoefficients + # and lowerBoundsOnIntercepts + l <- matrix(c(0.0, -1.0, 0.0, -1.0, 0.0, -1.0, 0.0, -1.0), nrow = 2, ncol = 4) + model <- spark.logit(training, Species ~ ., family = "multinomial", + lowerBoundsOnCoefficients = l, + lowerBoundsOnIntercepts = as.array(c(0.0, 0.0))) + summary <- summary(model) + versicolorCoefsR <- c(42.639465, 7.258104, 14.330814, 16.298243, 11.716429) + virginicaCoefsR <- c(0.0002970796, 4.79274, 7.65047, 25.72793, 30.0021) + versicolorCoefs <- summary$coefficients[, "versicolor"] + virginicaCoefs <- summary$coefficients[, "virginica"] + expect_true(all(abs(versicolorCoefsR - versicolorCoefs) < 0.1)) + expect_true(all(abs(virginicaCoefsR - virginicaCoefs) < 0.1)) + + # Test unseen labels + data <- data.frame(clicked = base::sample(c(0, 1), 10, replace = TRUE), + someString = base::sample(c("this", "that"), 10, replace = TRUE), + stringsAsFactors = FALSE) + trainidxs <- base::sample(nrow(data), nrow(data) * 0.7) + traindf <- as.DataFrame(data[trainidxs, ]) + testdf <- as.DataFrame(rbind(data[-trainidxs, ], c(0, "the other"))) + model <- spark.logit(traindf, clicked ~ ., regParam = 0.5) + predictions <- predict(model, testdf) + expect_error(collect(predictions)) + model <- spark.logit(traindf, clicked ~ ., regParam = 0.5, handleInvalid = "keep") + predictions <- predict(model, testdf) + expect_equal(class(collect(predictions)$clicked[1]), "character") + +}) + +test_that("spark.mlp", { + df <- read.df(absoluteSparkPath("data/mllib/sample_multiclass_classification_data.txt"), + source = "libsvm") + model <- spark.mlp(df, label ~ features, blockSize = 128, layers = c(4, 5, 4, 3), + solver = "l-bfgs", maxIter = 100, tol = 0.5, stepSize = 1, seed = 1) + + # Test summary method + summary <- summary(model) + expect_equal(summary$numOfInputs, 4) + expect_equal(summary$numOfOutputs, 3) + expect_equal(summary$layers, c(4, 5, 4, 3)) + expect_equal(length(summary$weights), 64) + expect_equal(head(summary$weights, 5), list(-0.878743, 0.2154151, -1.16304, -0.6583214, 1.009825), + tolerance = 1e-6) + + # Test predict method + mlpTestDF <- df + mlpPredictions <- collect(select(predict(model, mlpTestDF), "prediction")) + expect_equal(head(mlpPredictions$prediction, 6), c("1.0", "0.0", "0.0", "0.0", "0.0", "0.0")) + + # Test model save/load + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-mlp", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + summary2 <- summary(model2) + + expect_equal(summary2$numOfInputs, 4) + expect_equal(summary2$numOfOutputs, 3) + expect_equal(summary2$layers, c(4, 5, 4, 3)) + expect_equal(length(summary2$weights), 64) + + unlink(modelPath) + } + + # Test default parameter + model <- spark.mlp(df, label ~ features, layers = c(4, 5, 4, 3)) + mlpPredictions <- collect(select(predict(model, mlpTestDF), "prediction")) + expect_equal(head(mlpPredictions$prediction, 10), + c("1.0", "1.0", "1.0", "1.0", "0.0", "1.0", "2.0", "2.0", "1.0", "0.0")) + + # Test illegal parameter + expect_error(spark.mlp(df, label ~ features, layers = NULL), + "layers must be a integer vector with length > 1.") + expect_error(spark.mlp(df, label ~ features, layers = c()), + "layers must be a integer vector with length > 1.") + expect_error(spark.mlp(df, label ~ features, layers = c(3)), + "layers must be a integer vector with length > 1.") + + # Test random seed + # default seed + model <- spark.mlp(df, label ~ features, layers = c(4, 5, 4, 3), maxIter = 10) + mlpPredictions <- collect(select(predict(model, mlpTestDF), "prediction")) + expect_equal(head(mlpPredictions$prediction, 10), + c("1.0", "1.0", "1.0", "1.0", "0.0", "1.0", "2.0", "2.0", "1.0", "0.0")) + # seed equals 10 + model <- spark.mlp(df, label ~ features, layers = c(4, 5, 4, 3), maxIter = 10, seed = 10) + mlpPredictions <- collect(select(predict(model, mlpTestDF), "prediction")) + expect_equal(head(mlpPredictions$prediction, 10), + c("1.0", "1.0", "1.0", "1.0", "0.0", "1.0", "2.0", "2.0", "1.0", "0.0")) + + # test initialWeights + model <- spark.mlp(df, label ~ features, layers = c(4, 3), initialWeights = + c(0, 0, 0, 0, 0, 5, 5, 5, 5, 5, 9, 9, 9, 9, 9)) + mlpPredictions <- collect(select(predict(model, mlpTestDF), "prediction")) + expect_equal(head(mlpPredictions$prediction, 10), + c("1.0", "1.0", "1.0", "1.0", "0.0", "1.0", "2.0", "2.0", "1.0", "0.0")) + + # Test formula works well + df <- suppressWarnings(createDataFrame(iris)) + model <- spark.mlp(df, Species ~ Sepal_Length + Sepal_Width + Petal_Length + Petal_Width, + layers = c(4, 3)) + summary <- summary(model) + expect_equal(summary$numOfInputs, 4) + expect_equal(summary$numOfOutputs, 3) + expect_equal(summary$layers, c(4, 3)) + expect_equal(length(summary$weights), 15) + + # Test unseen labels + data <- data.frame(clicked = base::sample(c(0, 1), 10, replace = TRUE), + someString = base::sample(c("this", "that"), 10, replace = TRUE), + stringsAsFactors = FALSE) + trainidxs <- base::sample(nrow(data), nrow(data) * 0.7) + traindf <- as.DataFrame(data[trainidxs, ]) + testdf <- as.DataFrame(rbind(data[-trainidxs, ], c(0, "the other"))) + model <- spark.mlp(traindf, clicked ~ ., layers = c(1, 3)) + predictions <- predict(model, testdf) + expect_error(collect(predictions)) + model <- spark.mlp(traindf, clicked ~ ., layers = c(1, 3), handleInvalid = "skip") + predictions <- predict(model, testdf) + expect_equal(class(collect(predictions)$clicked[1]), "list") + +}) + +test_that("spark.naiveBayes", { + # R code to reproduce the result. + # We do not support instance weights yet. So we ignore the frequencies. + # + #' library(e1071) + #' t <- as.data.frame(Titanic) + #' t1 <- t[t$Freq > 0, -5] + #' m <- naiveBayes(Survived ~ ., data = t1) + #' m + #' predict(m, t1) + # + # -- output of 'm' + # + # A-priori probabilities: + # Y + # No Yes + # 0.4166667 0.5833333 + # + # Conditional probabilities: + # Class + # Y 1st 2nd 3rd Crew + # No 0.2000000 0.2000000 0.4000000 0.2000000 + # Yes 0.2857143 0.2857143 0.2857143 0.1428571 + # + # Sex + # Y Male Female + # No 0.5 0.5 + # Yes 0.5 0.5 + # + # Age + # Y Child Adult + # No 0.2000000 0.8000000 + # Yes 0.4285714 0.5714286 + # + # -- output of 'predict(m, t1)' + # + # Yes Yes Yes Yes No No Yes Yes No No Yes Yes Yes Yes Yes Yes Yes Yes No No Yes Yes No No + # + + t <- as.data.frame(Titanic) + t1 <- t[t$Freq > 0, -5] + df <- suppressWarnings(createDataFrame(t1)) + m <- spark.naiveBayes(df, Survived ~ ., smoothing = 0.0) + s <- summary(m) + expect_equal(as.double(s$apriori[1, "Yes"]), 0.5833333, tolerance = 1e-6) + expect_equal(sum(s$apriori), 1) + expect_equal(as.double(s$tables["Yes", "Age_Adult"]), 0.5714286, tolerance = 1e-6) + p <- collect(select(predict(m, df), "prediction")) + expect_equal(p$prediction, c("Yes", "Yes", "Yes", "Yes", "No", "No", "Yes", "Yes", "No", "No", + "Yes", "Yes", "Yes", "Yes", "Yes", "Yes", "Yes", "Yes", "No", "No", + "Yes", "Yes", "No", "No")) + + # Test model save/load + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-naiveBayes", fileext = ".tmp") + write.ml(m, modelPath) + expect_error(write.ml(m, modelPath)) + write.ml(m, modelPath, overwrite = TRUE) + m2 <- read.ml(modelPath) + s2 <- summary(m2) + expect_equal(s$apriori, s2$apriori) + expect_equal(s$tables, s2$tables) + + unlink(modelPath) + } + + # Test e1071::naiveBayes + if (requireNamespace("e1071", quietly = TRUE)) { + expect_error(m <- e1071::naiveBayes(Survived ~ ., data = t1), NA) + expect_equal(as.character(predict(m, t1[1, ])), "Yes") + } + + # Test numeric response variable + t1$NumericSurvived <- ifelse(t1$Survived == "No", 0, 1) + t2 <- t1[-4] + df <- suppressWarnings(createDataFrame(t2)) + m <- spark.naiveBayes(df, NumericSurvived ~ ., smoothing = 0.0) + s <- summary(m) + expect_equal(as.double(s$apriori[1, 1]), 0.5833333, tolerance = 1e-6) + expect_equal(sum(s$apriori), 1) + expect_equal(as.double(s$tables[1, "Age_Adult"]), 0.5714286, tolerance = 1e-6) + + # Test unseen labels + data <- data.frame(clicked = base::sample(c(0, 1), 10, replace = TRUE), + someString = base::sample(c("this", "that"), 10, replace = TRUE), + stringsAsFactors = FALSE) + trainidxs <- base::sample(nrow(data), nrow(data) * 0.7) + traindf <- as.DataFrame(data[trainidxs, ]) + testdf <- as.DataFrame(rbind(data[-trainidxs, ], c(0, "the other"))) + model <- spark.naiveBayes(traindf, clicked ~ ., smoothing = 0.0) + predictions <- predict(model, testdf) + expect_error(collect(predictions)) + model <- spark.naiveBayes(traindf, clicked ~ ., smoothing = 0.0, handleInvalid = "keep") + predictions <- predict(model, testdf) + expect_equal(class(collect(predictions)$clicked[1]), "character") +}) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_mllib_clustering.R b/R/pkg/tests/fulltests/test_mllib_clustering.R new file mode 100644 index 000000000000..4110e13da494 --- /dev/null +++ b/R/pkg/tests/fulltests/test_mllib_clustering.R @@ -0,0 +1,322 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(testthat) + +context("MLlib clustering algorithms") + +# Tests for MLlib clustering algorithms in SparkR +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + +absoluteSparkPath <- function(x) { + sparkHome <- sparkR.conf("spark.home") + file.path(sparkHome, x) +} + +test_that("spark.bisectingKmeans", { + newIris <- iris + newIris$Species <- NULL + training <- suppressWarnings(createDataFrame(newIris)) + + take(training, 1) + + model <- spark.bisectingKmeans(data = training, ~ .) + sample <- take(select(predict(model, training), "prediction"), 1) + expect_equal(typeof(sample$prediction), "integer") + expect_equal(sample$prediction, 1) + + # Test fitted works on Bisecting KMeans + fitted.model <- fitted(model) + expect_equal(sort(collect(distinct(select(fitted.model, "prediction")))$prediction), + c(0, 1, 2, 3)) + + # Test summary works on KMeans + summary.model <- summary(model) + cluster <- summary.model$cluster + k <- summary.model$k + expect_equal(k, 4) + expect_equal(sort(collect(distinct(select(cluster, "prediction")))$prediction), + c(0, 1, 2, 3)) + + # Test model save/load + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-bisectingkmeans", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + summary2 <- summary(model2) + expect_equal(sort(unlist(summary.model$size)), sort(unlist(summary2$size))) + expect_equal(summary.model$coefficients, summary2$coefficients) + expect_true(!summary.model$is.loaded) + expect_true(summary2$is.loaded) + + unlink(modelPath) + } +}) + +test_that("spark.gaussianMixture", { + # R code to reproduce the result. + # nolint start + #' library(mvtnorm) + #' set.seed(1) + #' a <- rmvnorm(7, c(0, 0)) + #' b <- rmvnorm(8, c(10, 10)) + #' data <- rbind(a, b) + #' model <- mvnormalmixEM(data, k = 2) + #' model$lambda + # + # [1] 0.4666667 0.5333333 + # + #' model$mu + # + # [1] 0.11731091 -0.06192351 + # [1] 10.363673 9.897081 + # + #' model$sigma + # + # [[1]] + # [,1] [,2] + # [1,] 0.62049934 0.06880802 + # [2,] 0.06880802 1.27431874 + # + # [[2]] + # [,1] [,2] + # [1,] 0.2961543 0.160783 + # [2,] 0.1607830 1.008878 + # + #' model$loglik + # + # [1] -46.89499 + # nolint end + data <- list(list(-0.6264538, 0.1836433), list(-0.8356286, 1.5952808), + list(0.3295078, -0.8204684), list(0.4874291, 0.7383247), + list(0.5757814, -0.3053884), list(1.5117812, 0.3898432), + list(-0.6212406, -2.2146999), list(11.1249309, 9.9550664), + list(9.9838097, 10.9438362), list(10.8212212, 10.5939013), + list(10.9189774, 10.7821363), list(10.0745650, 8.0106483), + list(10.6198257, 9.9438713), list(9.8442045, 8.5292476), + list(9.5218499, 10.4179416)) + df <- createDataFrame(data, c("x1", "x2")) + model <- spark.gaussianMixture(df, ~ x1 + x2, k = 2) + stats <- summary(model) + rLambda <- c(0.4666667, 0.5333333) + rMu <- c(0.11731091, -0.06192351, 10.363673, 9.897081) + rSigma <- c(0.62049934, 0.06880802, 0.06880802, 1.27431874, + 0.2961543, 0.160783, 0.1607830, 1.008878) + rLoglik <- -46.89499 + expect_equal(stats$lambda, rLambda, tolerance = 1e-3) + expect_equal(unlist(stats$mu), rMu, tolerance = 1e-3) + expect_equal(unlist(stats$sigma), rSigma, tolerance = 1e-3) + expect_equal(unlist(stats$loglik), rLoglik, tolerance = 1e-3) + p <- collect(select(predict(model, df), "prediction")) + expect_equal(p$prediction, c(0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 1, 1, 1)) + + # Test model save/load + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-gaussianMixture", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + stats2 <- summary(model2) + expect_equal(stats$lambda, stats2$lambda) + expect_equal(unlist(stats$mu), unlist(stats2$mu)) + expect_equal(unlist(stats$sigma), unlist(stats2$sigma)) + expect_equal(unlist(stats$loglik), unlist(stats2$loglik)) + + unlink(modelPath) + } +}) + +test_that("spark.kmeans", { + newIris <- iris + newIris$Species <- NULL + training <- suppressWarnings(createDataFrame(newIris)) + + take(training, 1) + + model <- spark.kmeans(data = training, ~ ., k = 2, maxIter = 10, initMode = "random") + sample <- take(select(predict(model, training), "prediction"), 1) + expect_equal(typeof(sample$prediction), "integer") + expect_equal(sample$prediction, 1) + + # Test stats::kmeans is working + statsModel <- kmeans(x = newIris, centers = 2) + expect_equal(sort(unique(statsModel$cluster)), c(1, 2)) + + # Test fitted works on KMeans + fitted.model <- fitted(model) + expect_equal(sort(collect(distinct(select(fitted.model, "prediction")))$prediction), c(0, 1)) + + # Test summary works on KMeans + summary.model <- summary(model) + cluster <- summary.model$cluster + k <- summary.model$k + expect_equal(k, 2) + expect_equal(sort(collect(distinct(select(cluster, "prediction")))$prediction), c(0, 1)) + + # test summary coefficients return matrix type + expect_true(class(summary.model$coefficients) == "matrix") + expect_true(class(summary.model$coefficients[1, ]) == "numeric") + + # Test model save/load + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-kmeans", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + summary2 <- summary(model2) + expect_equal(sort(unlist(summary.model$size)), sort(unlist(summary2$size))) + expect_equal(summary.model$coefficients, summary2$coefficients) + expect_true(!summary.model$is.loaded) + expect_true(summary2$is.loaded) + + unlink(modelPath) + } + + # Test Kmeans on dataset that is sensitive to seed value + col1 <- c(1, 2, 3, 4, 0, 1, 2, 3, 4, 0) + col2 <- c(1, 2, 3, 4, 0, 1, 2, 3, 4, 0) + col3 <- c(1, 2, 3, 4, 0, 1, 2, 3, 4, 0) + cols <- as.data.frame(cbind(col1, col2, col3)) + df <- createDataFrame(cols) + + model1 <- spark.kmeans(data = df, ~ ., k = 5, maxIter = 10, + initMode = "random", seed = 1, tol = 1E-5) + model2 <- spark.kmeans(data = df, ~ ., k = 5, maxIter = 10, + initMode = "random", seed = 22222, tol = 1E-5) + + summary.model1 <- summary(model1) + summary.model2 <- summary(model2) + cluster1 <- summary.model1$cluster + cluster2 <- summary.model2$cluster + clusterSize1 <- summary.model1$clusterSize + clusterSize2 <- summary.model2$clusterSize + + # The predicted clusters are different + expect_equal(sort(collect(distinct(select(cluster1, "prediction")))$prediction), + c(0, 1, 2, 3)) + expect_equal(sort(collect(distinct(select(cluster2, "prediction")))$prediction), + c(0, 1, 2)) + expect_equal(clusterSize1, 4) + expect_equal(clusterSize2, 3) +}) + +test_that("spark.lda with libsvm", { + text <- read.df(absoluteSparkPath("data/mllib/sample_lda_libsvm_data.txt"), source = "libsvm") + model <- spark.lda(text, optimizer = "em") + + stats <- summary(model, 10) + isDistributed <- stats$isDistributed + logLikelihood <- stats$logLikelihood + logPerplexity <- stats$logPerplexity + vocabSize <- stats$vocabSize + topics <- stats$topicTopTerms + weights <- stats$topicTopTermsWeights + vocabulary <- stats$vocabulary + trainingLogLikelihood <- stats$trainingLogLikelihood + logPrior <- stats$logPrior + + expect_true(isDistributed) + expect_true(logLikelihood <= 0 & is.finite(logLikelihood)) + expect_true(logPerplexity >= 0 & is.finite(logPerplexity)) + expect_equal(vocabSize, 11) + expect_true(is.null(vocabulary)) + expect_true(trainingLogLikelihood <= 0 & !is.na(trainingLogLikelihood)) + expect_true(logPrior <= 0 & !is.na(logPrior)) + + # Test model save/load + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-lda", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + stats2 <- summary(model2) + + expect_true(stats2$isDistributed) + expect_equal(logLikelihood, stats2$logLikelihood) + expect_equal(logPerplexity, stats2$logPerplexity) + expect_equal(vocabSize, stats2$vocabSize) + expect_equal(vocabulary, stats2$vocabulary) + expect_equal(trainingLogLikelihood, stats2$trainingLogLikelihood) + expect_equal(logPrior, stats2$logPrior) + + unlink(modelPath) + } +}) + +test_that("spark.lda with text input", { + text <- read.text(absoluteSparkPath("data/mllib/sample_lda_data.txt")) + model <- spark.lda(text, optimizer = "online", features = "value") + + stats <- summary(model) + isDistributed <- stats$isDistributed + logLikelihood <- stats$logLikelihood + logPerplexity <- stats$logPerplexity + vocabSize <- stats$vocabSize + topics <- stats$topicTopTerms + weights <- stats$topicTopTermsWeights + vocabulary <- stats$vocabulary + trainingLogLikelihood <- stats$trainingLogLikelihood + logPrior <- stats$logPrior + + expect_false(isDistributed) + expect_true(logLikelihood <= 0 & is.finite(logLikelihood)) + expect_true(logPerplexity >= 0 & is.finite(logPerplexity)) + expect_equal(vocabSize, 10) + expect_true(setequal(stats$vocabulary, c("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"))) + expect_true(is.na(trainingLogLikelihood)) + expect_true(is.na(logPrior)) + + # Test model save/load + modelPath <- tempfile(pattern = "spark-lda-text", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + stats2 <- summary(model2) + + expect_false(stats2$isDistributed) + expect_equal(logLikelihood, stats2$logLikelihood) + expect_equal(logPerplexity, stats2$logPerplexity) + expect_equal(vocabSize, stats2$vocabSize) + expect_true(all.equal(vocabulary, stats2$vocabulary)) + expect_true(is.na(stats2$trainingLogLikelihood)) + expect_true(is.na(stats2$logPrior)) + + unlink(modelPath) +}) + +test_that("spark.posterior and spark.perplexity", { + text <- read.text(absoluteSparkPath("data/mllib/sample_lda_data.txt")) + model <- spark.lda(text, features = "value", k = 3) + + # Assert perplexities are equal + stats <- summary(model) + logPerplexity <- spark.perplexity(model, text) + expect_equal(logPerplexity, stats$logPerplexity) + + # Assert the sum of every topic distribution is equal to 1 + posterior <- spark.posterior(model, text) + local.posterior <- collect(posterior)$topicDistribution + expect_equal(length(local.posterior), sum(unlist(local.posterior))) +}) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_mllib_fpm.R b/R/pkg/tests/fulltests/test_mllib_fpm.R new file mode 100644 index 000000000000..69dda52f0c27 --- /dev/null +++ b/R/pkg/tests/fulltests/test_mllib_fpm.R @@ -0,0 +1,85 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(testthat) + +context("MLlib frequent pattern mining") + +# Tests for MLlib frequent pattern mining algorithms in SparkR +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + +test_that("spark.fpGrowth", { + data <- selectExpr(createDataFrame(data.frame(items = c( + "1,2", + "1,2", + "1,2,3", + "1,3" + ))), "split(items, ',') as items") + + model <- spark.fpGrowth(data, minSupport = 0.3, minConfidence = 0.8, numPartitions = 1) + + itemsets <- collect(spark.freqItemsets(model)) + + expected_itemsets <- data.frame( + items = I(list(list("3"), list("3", "1"), list("2"), list("2", "1"), list("1"))), + freq = c(2, 2, 3, 3, 4) + ) + + expect_equivalent(expected_itemsets, itemsets) + + expected_association_rules <- data.frame( + antecedent = I(list(list("2"), list("3"))), + consequent = I(list(list("1"), list("1"))), + confidence = c(1, 1) + ) + + expect_equivalent(expected_association_rules, collect(spark.associationRules(model))) + + new_data <- selectExpr(createDataFrame(data.frame(items = c( + "1,2", + "1,3", + "2,3" + ))), "split(items, ',') as items") + + expected_predictions <- data.frame( + items = I(list(list("1", "2"), list("1", "3"), list("2", "3"))), + prediction = I(list(list(), list(), list("1"))) + ) + + expect_equivalent(expected_predictions, collect(predict(model, new_data))) + + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-fpm", fileext = ".tmp") + write.ml(model, modelPath, overwrite = TRUE) + loaded_model <- read.ml(modelPath) + + expect_equivalent( + itemsets, + collect(spark.freqItemsets(loaded_model))) + + unlink(modelPath) + } + + model_without_numpartitions <- spark.fpGrowth(data, minSupport = 0.3, minConfidence = 0.8) + expect_equal( + count(spark.freqItemsets(model_without_numpartitions)), + count(spark.freqItemsets(model)) + ) + +}) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_mllib_recommendation.R b/R/pkg/tests/fulltests/test_mllib_recommendation.R new file mode 100644 index 000000000000..4d919c9d746b --- /dev/null +++ b/R/pkg/tests/fulltests/test_mllib_recommendation.R @@ -0,0 +1,67 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(testthat) + +context("MLlib recommendation algorithms") + +# Tests for MLlib recommendation algorithms in SparkR +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + +test_that("spark.als", { + data <- list(list(0, 0, 4.0), list(0, 1, 2.0), list(1, 1, 3.0), list(1, 2, 4.0), + list(2, 1, 1.0), list(2, 2, 5.0)) + df <- createDataFrame(data, c("user", "item", "score")) + model <- spark.als(df, ratingCol = "score", userCol = "user", itemCol = "item", + rank = 10, maxIter = 5, seed = 0, regParam = 0.1) + stats <- summary(model) + expect_equal(stats$rank, 10) + test <- createDataFrame(list(list(0, 2), list(1, 0), list(2, 0)), c("user", "item")) + predictions <- collect(predict(model, test)) + + expect_equal(predictions$prediction, c(-0.1380762, 2.6258414, -1.5018409), + tolerance = 1e-4) + + # Test model save/load + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-als", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + stats2 <- summary(model2) + expect_equal(stats2$rating, "score") + userFactors <- collect(stats$userFactors) + itemFactors <- collect(stats$itemFactors) + userFactors2 <- collect(stats2$userFactors) + itemFactors2 <- collect(stats2$itemFactors) + + orderUser <- order(userFactors$id) + orderUser2 <- order(userFactors2$id) + expect_equal(userFactors$id[orderUser], userFactors2$id[orderUser2]) + expect_equal(userFactors$features[orderUser], userFactors2$features[orderUser2]) + + orderItem <- order(itemFactors$id) + orderItem2 <- order(itemFactors2$id) + expect_equal(itemFactors$id[orderItem], itemFactors2$id[orderItem2]) + expect_equal(itemFactors$features[orderItem], itemFactors2$features[orderItem2]) + + unlink(modelPath) + } +}) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_mllib_regression.R b/R/pkg/tests/fulltests/test_mllib_regression.R new file mode 100644 index 000000000000..23daca75fcc2 --- /dev/null +++ b/R/pkg/tests/fulltests/test_mllib_regression.R @@ -0,0 +1,538 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(testthat) + +context("MLlib regression algorithms, except for tree-based algorithms") + +# Tests for MLlib regression algorithms in SparkR +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + +test_that("formula of spark.glm", { + training <- suppressWarnings(createDataFrame(iris)) + # directly calling the spark API + # dot minus and intercept vs native glm + model <- spark.glm(training, Sepal_Width ~ . - Species + 0) + vals <- collect(select(predict(model, training), "prediction")) + rVals <- predict(glm(Sepal.Width ~ . - Species + 0, data = iris), iris) + expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) + + # feature interaction vs native glm + model <- spark.glm(training, Sepal_Width ~ Species:Sepal_Length) + vals <- collect(select(predict(model, training), "prediction")) + rVals <- predict(glm(Sepal.Width ~ Species:Sepal.Length, data = iris), iris) + expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) + + # glm should work with long formula + training <- suppressWarnings(createDataFrame(iris)) + training$LongLongLongLongLongName <- training$Sepal_Width + training$VeryLongLongLongLonLongName <- training$Sepal_Length + training$AnotherLongLongLongLongName <- training$Species + model <- spark.glm(training, LongLongLongLongLongName ~ VeryLongLongLongLonLongName + + AnotherLongLongLongLongName) + vals <- collect(select(predict(model, training), "prediction")) + rVals <- predict(glm(Sepal.Width ~ Sepal.Length + Species, data = iris), iris) + expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) +}) + +test_that("spark.glm and predict", { + training <- suppressWarnings(createDataFrame(iris)) + # gaussian family + model <- spark.glm(training, Sepal_Width ~ Sepal_Length + Species) + prediction <- predict(model, training) + expect_equal(typeof(take(select(prediction, "prediction"), 1)$prediction), "double") + vals <- collect(select(prediction, "prediction")) + rVals <- predict(glm(Sepal.Width ~ Sepal.Length + Species, data = iris), iris) + expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) + + # poisson family + model <- spark.glm(training, Sepal_Width ~ Sepal_Length + Species, + family = poisson(link = identity)) + prediction <- predict(model, training) + expect_equal(typeof(take(select(prediction, "prediction"), 1)$prediction), "double") + vals <- collect(select(prediction, "prediction")) + rVals <- suppressWarnings(predict(glm(Sepal.Width ~ Sepal.Length + Species, + data = iris, family = poisson(link = identity)), iris)) + expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) + + # Gamma family + x <- runif(100, -1, 1) + y <- rgamma(100, rate = 10 / exp(0.5 + 1.2 * x), shape = 10) + df <- as.DataFrame(as.data.frame(list(x = x, y = y))) + model <- glm(y ~ x, family = Gamma, df) + out <- capture.output(print(summary(model))) + expect_true(any(grepl("Dispersion parameter for gamma family", out))) + + # tweedie family + model <- spark.glm(training, Sepal_Width ~ Sepal_Length + Species, + family = "tweedie", var.power = 1.2, link.power = 0.0) + prediction <- predict(model, training) + expect_equal(typeof(take(select(prediction, "prediction"), 1)$prediction), "double") + vals <- collect(select(prediction, "prediction")) + + # manual calculation of the R predicted values to avoid dependence on statmod + #' library(statmod) + #' rModel <- glm(Sepal.Width ~ Sepal.Length + Species, data = iris, + #' family = tweedie(var.power = 1.2, link.power = 0.0)) + #' print(coef(rModel)) + + rCoef <- c(0.6455409, 0.1169143, -0.3224752, -0.3282174) + rVals <- exp(as.numeric(model.matrix(Sepal.Width ~ Sepal.Length + Species, + data = iris) %*% rCoef)) + expect_true(all(abs(rVals - vals) < 1e-5), rVals - vals) + + # Test stats::predict is working + x <- rnorm(15) + y <- x + rnorm(15) + expect_equal(length(predict(lm(y ~ x))), 15) +}) + +test_that("spark.glm summary", { + # gaussian family + training <- suppressWarnings(createDataFrame(iris)) + stats <- summary(spark.glm(training, Sepal_Width ~ Sepal_Length + Species)) + rStats <- summary(glm(Sepal.Width ~ Sepal.Length + Species, data = iris)) + + # test summary coefficients return matrix type + expect_true(class(stats$coefficients) == "matrix") + expect_true(class(stats$coefficients[, 1]) == "numeric") + + coefs <- stats$coefficients + rCoefs <- rStats$coefficients + expect_true(all(abs(rCoefs - coefs) < 1e-4)) + expect_true(all( + rownames(stats$coefficients) == + c("(Intercept)", "Sepal_Length", "Species_versicolor", "Species_virginica"))) + expect_equal(stats$dispersion, rStats$dispersion) + expect_equal(stats$null.deviance, rStats$null.deviance) + expect_equal(stats$deviance, rStats$deviance) + expect_equal(stats$df.null, rStats$df.null) + expect_equal(stats$df.residual, rStats$df.residual) + expect_equal(stats$aic, rStats$aic) + + out <- capture.output(print(stats)) + expect_match(out[2], "Deviance Residuals:") + expect_true(any(grepl("AIC: 59.22", out))) + + # binomial family + df <- suppressWarnings(createDataFrame(iris)) + training <- df[df$Species %in% c("versicolor", "virginica"), ] + stats <- summary(spark.glm(training, Species ~ Sepal_Length + Sepal_Width, + family = binomial(link = "logit"))) + + rTraining <- iris[iris$Species %in% c("versicolor", "virginica"), ] + rStats <- summary(glm(Species ~ Sepal.Length + Sepal.Width, data = rTraining, + family = binomial(link = "logit"))) + + coefs <- stats$coefficients + rCoefs <- rStats$coefficients + expect_true(all(abs(rCoefs - coefs) < 1e-4)) + expect_true(all( + rownames(stats$coefficients) == + c("(Intercept)", "Sepal_Length", "Sepal_Width"))) + expect_equal(stats$dispersion, rStats$dispersion) + expect_equal(stats$null.deviance, rStats$null.deviance) + expect_equal(stats$deviance, rStats$deviance) + expect_equal(stats$df.null, rStats$df.null) + expect_equal(stats$df.residual, rStats$df.residual) + expect_equal(stats$aic, rStats$aic) + + # Test spark.glm works with weighted dataset + a1 <- c(0, 1, 2, 3) + a2 <- c(5, 2, 1, 3) + w <- c(1, 2, 3, 4) + b <- c(1, 0, 1, 0) + data <- as.data.frame(cbind(a1, a2, w, b)) + df <- createDataFrame(data) + + stats <- summary(spark.glm(df, b ~ a1 + a2, family = "binomial", weightCol = "w")) + rStats <- summary(glm(b ~ a1 + a2, family = "binomial", data = data, weights = w)) + + coefs <- stats$coefficients + rCoefs <- rStats$coefficients + expect_true(all(abs(rCoefs - coefs) < 1e-3)) + expect_true(all(rownames(stats$coefficients) == c("(Intercept)", "a1", "a2"))) + expect_equal(stats$dispersion, rStats$dispersion) + expect_equal(stats$null.deviance, rStats$null.deviance) + expect_equal(stats$deviance, rStats$deviance) + expect_equal(stats$df.null, rStats$df.null) + expect_equal(stats$df.residual, rStats$df.residual) + expect_equal(stats$aic, rStats$aic) + + # Test spark.glm works with offset + training <- suppressWarnings(createDataFrame(iris)) + stats <- summary(spark.glm(training, Sepal_Width ~ Sepal_Length + Species, + family = poisson(), offsetCol = "Petal_Length")) + rStats <- suppressWarnings(summary(glm(Sepal.Width ~ Sepal.Length + Species, + data = iris, family = poisson(), offset = iris$Petal.Length))) + expect_true(all(abs(rStats$coefficients - stats$coefficients) < 1e-3)) + + # Test summary works on base GLM models + baseModel <- stats::glm(Sepal.Width ~ Sepal.Length + Species, data = iris) + baseSummary <- summary(baseModel) + expect_true(abs(baseSummary$deviance - 12.19313) < 1e-4) + + # Test spark.glm works with regularization parameter + data <- as.data.frame(cbind(a1, a2, b)) + df <- suppressWarnings(createDataFrame(data)) + regStats <- summary(spark.glm(df, b ~ a1 + a2, regParam = 1.0)) + expect_equal(regStats$aic, 13.32836, tolerance = 1e-4) # 13.32836 is from summary() result + + # Test spark.glm works on collinear data + A <- matrix(c(1, 2, 3, 4, 2, 4, 6, 8), 4, 2) + b <- c(1, 2, 3, 4) + data <- as.data.frame(cbind(A, b)) + df <- createDataFrame(data) + stats <- summary(spark.glm(df, b ~ . - 1)) + coefs <- stats$coefficients + expect_true(all(abs(c(0.5, 0.25) - coefs) < 1e-4)) +}) + +test_that("spark.glm save/load", { + training <- suppressWarnings(createDataFrame(iris)) + m <- spark.glm(training, Sepal_Width ~ Sepal_Length + Species) + s <- summary(m) + + modelPath <- tempfile(pattern = "spark-glm", fileext = ".tmp") + write.ml(m, modelPath) + expect_error(write.ml(m, modelPath)) + write.ml(m, modelPath, overwrite = TRUE) + m2 <- read.ml(modelPath) + s2 <- summary(m2) + + expect_equal(s$coefficients, s2$coefficients) + expect_equal(rownames(s$coefficients), rownames(s2$coefficients)) + expect_equal(s$dispersion, s2$dispersion) + expect_equal(s$null.deviance, s2$null.deviance) + expect_equal(s$deviance, s2$deviance) + expect_equal(s$df.null, s2$df.null) + expect_equal(s$df.residual, s2$df.residual) + expect_equal(s$aic, s2$aic) + expect_equal(s$iter, s2$iter) + expect_true(!s$is.loaded) + expect_true(s2$is.loaded) + + unlink(modelPath) +}) + +test_that("formula of glm", { + training <- suppressWarnings(createDataFrame(iris)) + # dot minus and intercept vs native glm + model <- glm(Sepal_Width ~ . - Species + 0, data = training) + vals <- collect(select(predict(model, training), "prediction")) + rVals <- predict(glm(Sepal.Width ~ . - Species + 0, data = iris), iris) + expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) + + # feature interaction vs native glm + model <- glm(Sepal_Width ~ Species:Sepal_Length, data = training) + vals <- collect(select(predict(model, training), "prediction")) + rVals <- predict(glm(Sepal.Width ~ Species:Sepal.Length, data = iris), iris) + expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) + + # glm should work with long formula + training <- suppressWarnings(createDataFrame(iris)) + training$LongLongLongLongLongName <- training$Sepal_Width + training$VeryLongLongLongLonLongName <- training$Sepal_Length + training$AnotherLongLongLongLongName <- training$Species + model <- glm(LongLongLongLongLongName ~ VeryLongLongLongLonLongName + AnotherLongLongLongLongName, + data = training) + vals <- collect(select(predict(model, training), "prediction")) + rVals <- predict(glm(Sepal.Width ~ Sepal.Length + Species, data = iris), iris) + expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) +}) + +test_that("glm and predict", { + training <- suppressWarnings(createDataFrame(iris)) + # gaussian family + model <- glm(Sepal_Width ~ Sepal_Length + Species, data = training) + prediction <- predict(model, training) + expect_equal(typeof(take(select(prediction, "prediction"), 1)$prediction), "double") + vals <- collect(select(prediction, "prediction")) + rVals <- predict(glm(Sepal.Width ~ Sepal.Length + Species, data = iris), iris) + expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) + + # poisson family + model <- glm(Sepal_Width ~ Sepal_Length + Species, data = training, + family = poisson(link = identity)) + prediction <- predict(model, training) + expect_equal(typeof(take(select(prediction, "prediction"), 1)$prediction), "double") + vals <- collect(select(prediction, "prediction")) + rVals <- suppressWarnings(predict(glm(Sepal.Width ~ Sepal.Length + Species, + data = iris, family = poisson(link = identity)), iris)) + expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) + + # tweedie family + model <- glm(Sepal_Width ~ Sepal_Length + Species, data = training, + family = "tweedie", var.power = 1.2, link.power = 0.0) + prediction <- predict(model, training) + expect_equal(typeof(take(select(prediction, "prediction"), 1)$prediction), "double") + vals <- collect(select(prediction, "prediction")) + + # manual calculation of the R predicted values to avoid dependence on statmod + #' library(statmod) + #' rModel <- glm(Sepal.Width ~ Sepal.Length + Species, data = iris, + #' family = tweedie(var.power = 1.2, link.power = 0.0)) + #' print(coef(rModel)) + + rCoef <- c(0.6455409, 0.1169143, -0.3224752, -0.3282174) + rVals <- exp(as.numeric(model.matrix(Sepal.Width ~ Sepal.Length + Species, + data = iris) %*% rCoef)) + expect_true(all(abs(rVals - vals) < 1e-5), rVals - vals) + + # Test stats::predict is working + x <- rnorm(15) + y <- x + rnorm(15) + expect_equal(length(predict(lm(y ~ x))), 15) +}) + +test_that("glm summary", { + # gaussian family + training <- suppressWarnings(createDataFrame(iris)) + stats <- summary(glm(Sepal_Width ~ Sepal_Length + Species, data = training)) + + rStats <- summary(glm(Sepal.Width ~ Sepal.Length + Species, data = iris)) + + coefs <- stats$coefficients + rCoefs <- rStats$coefficients + expect_true(all(abs(rCoefs - coefs) < 1e-4)) + expect_true(all( + rownames(stats$coefficients) == + c("(Intercept)", "Sepal_Length", "Species_versicolor", "Species_virginica"))) + expect_equal(stats$dispersion, rStats$dispersion) + expect_equal(stats$null.deviance, rStats$null.deviance) + expect_equal(stats$deviance, rStats$deviance) + expect_equal(stats$df.null, rStats$df.null) + expect_equal(stats$df.residual, rStats$df.residual) + expect_equal(stats$aic, rStats$aic) + + # binomial family + df <- suppressWarnings(createDataFrame(iris)) + training <- df[df$Species %in% c("versicolor", "virginica"), ] + stats <- summary(glm(Species ~ Sepal_Length + Sepal_Width, data = training, + family = binomial(link = "logit"))) + + rTraining <- iris[iris$Species %in% c("versicolor", "virginica"), ] + rStats <- summary(glm(Species ~ Sepal.Length + Sepal.Width, data = rTraining, + family = binomial(link = "logit"))) + + coefs <- stats$coefficients + rCoefs <- rStats$coefficients + expect_true(all(abs(rCoefs - coefs) < 1e-4)) + expect_true(all( + rownames(stats$coefficients) == + c("(Intercept)", "Sepal_Length", "Sepal_Width"))) + expect_equal(stats$dispersion, rStats$dispersion) + expect_equal(stats$null.deviance, rStats$null.deviance) + expect_equal(stats$deviance, rStats$deviance) + expect_equal(stats$df.null, rStats$df.null) + expect_equal(stats$df.residual, rStats$df.residual) + expect_equal(stats$aic, rStats$aic) + + # Test summary works on base GLM models + baseModel <- stats::glm(Sepal.Width ~ Sepal.Length + Species, data = iris) + baseSummary <- summary(baseModel) + expect_true(abs(baseSummary$deviance - 12.19313) < 1e-4) +}) + +test_that("glm save/load", { + training <- suppressWarnings(createDataFrame(iris)) + m <- glm(Sepal_Width ~ Sepal_Length + Species, data = training) + s <- summary(m) + + modelPath <- tempfile(pattern = "glm", fileext = ".tmp") + write.ml(m, modelPath) + expect_error(write.ml(m, modelPath)) + write.ml(m, modelPath, overwrite = TRUE) + m2 <- read.ml(modelPath) + s2 <- summary(m2) + + expect_equal(s$coefficients, s2$coefficients) + expect_equal(rownames(s$coefficients), rownames(s2$coefficients)) + expect_equal(s$dispersion, s2$dispersion) + expect_equal(s$null.deviance, s2$null.deviance) + expect_equal(s$deviance, s2$deviance) + expect_equal(s$df.null, s2$df.null) + expect_equal(s$df.residual, s2$df.residual) + expect_equal(s$aic, s2$aic) + expect_equal(s$iter, s2$iter) + expect_true(!s$is.loaded) + expect_true(s2$is.loaded) + + unlink(modelPath) +}) + +test_that("spark.glm and glm with string encoding", { + t <- as.data.frame(Titanic, stringsAsFactors = FALSE) + df <- createDataFrame(t) + + # base R + rm <- stats::glm(Freq ~ Sex + Age, family = "gaussian", data = t) + # spark.glm with default stringIndexerOrderType = "frequencyDesc" + sm0 <- spark.glm(df, Freq ~ Sex + Age, family = "gaussian") + # spark.glm with stringIndexerOrderType = "alphabetDesc" + sm1 <- spark.glm(df, Freq ~ Sex + Age, family = "gaussian", + stringIndexerOrderType = "alphabetDesc") + # glm with stringIndexerOrderType = "alphabetDesc" + sm2 <- glm(Freq ~ Sex + Age, family = "gaussian", data = df, + stringIndexerOrderType = "alphabetDesc") + + rStats <- summary(rm) + rCoefs <- rStats$coefficients + sStats <- lapply(list(sm0, sm1, sm2), summary) + # order by coefficient size since column rendering may be different + o <- order(rCoefs[, 1]) + + # default encoding does not produce same results as R + expect_false(all(abs(rCoefs[o, ] - sStats[[1]]$coefficients[o, ]) < 1e-4)) + + # all estimates should be the same as R with stringIndexerOrderType = "alphabetDesc" + test <- lapply(sStats[2:3], function(stats) { + expect_true(all(abs(rCoefs[o, ] - stats$coefficients[o, ]) < 1e-4)) + expect_equal(stats$dispersion, rStats$dispersion) + expect_equal(stats$null.deviance, rStats$null.deviance) + expect_equal(stats$deviance, rStats$deviance) + expect_equal(stats$df.null, rStats$df.null) + expect_equal(stats$df.residual, rStats$df.residual) + expect_equal(stats$aic, rStats$aic) + }) + + # fitted values should be equal regardless of string encoding + rVals <- predict(rm, t) + test <- lapply(list(sm0, sm1, sm2), function(sm) { + vals <- collect(select(predict(sm, df), "prediction")) + expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals) + }) +}) + +test_that("spark.isoreg", { + label <- c(7.0, 5.0, 3.0, 5.0, 1.0) + feature <- c(0.0, 1.0, 2.0, 3.0, 4.0) + weight <- c(1.0, 1.0, 1.0, 1.0, 1.0) + data <- as.data.frame(cbind(label, feature, weight)) + df <- createDataFrame(data) + + model <- spark.isoreg(df, label ~ feature, isotonic = FALSE, + weightCol = "weight") + # only allow one variable on the right hand side of the formula + expect_error(model2 <- spark.isoreg(df, ~., isotonic = FALSE)) + result <- summary(model) + expect_equal(result$predictions, list(7, 5, 4, 4, 1)) + + # Test model prediction + predict_data <- list(list(-2.0), list(-1.0), list(0.5), + list(0.75), list(1.0), list(2.0), list(9.0)) + predict_df <- createDataFrame(predict_data, c("feature")) + predict_result <- collect(select(predict(model, predict_df), "prediction")) + expect_equal(predict_result$prediction, c(7.0, 7.0, 6.0, 5.5, 5.0, 4.0, 1.0)) + + # Test model save/load + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-isoreg", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + expect_equal(result, summary(model2)) + + unlink(modelPath) + } +}) + +test_that("spark.survreg", { + # R code to reproduce the result. + # + #' rData <- list(time = c(4, 3, 1, 1, 2, 2, 3), status = c(1, 1, 1, 0, 1, 1, 0), + #' x = c(0, 2, 1, 1, 1, 0, 0), sex = c(0, 0, 0, 0, 1, 1, 1)) + #' library(survival) + #' model <- survreg(Surv(time, status) ~ x + sex, rData) + #' summary(model) + #' predict(model, data) + # + # -- output of 'summary(model)' + # + # Value Std. Error z p + # (Intercept) 1.315 0.270 4.88 1.07e-06 + # x -0.190 0.173 -1.10 2.72e-01 + # sex -0.253 0.329 -0.77 4.42e-01 + # Log(scale) -1.160 0.396 -2.93 3.41e-03 + # + # -- output of 'predict(model, data)' + # + # 1 2 3 4 5 6 7 + # 3.724591 2.545368 3.079035 3.079035 2.390146 2.891269 2.891269 + # + data <- list(list(4, 1, 0, 0), list(3, 1, 2, 0), list(1, 1, 1, 0), + list(1, 0, 1, 0), list(2, 1, 1, 1), list(2, 1, 0, 1), list(3, 0, 0, 1)) + df <- createDataFrame(data, c("time", "status", "x", "sex")) + model <- spark.survreg(df, Surv(time, status) ~ x + sex) + stats <- summary(model) + coefs <- as.vector(stats$coefficients[, 1]) + rCoefs <- c(1.3149571, -0.1903409, -0.2532618, -1.1599800) + expect_equal(coefs, rCoefs, tolerance = 1e-4) + expect_true(all( + rownames(stats$coefficients) == + c("(Intercept)", "x", "sex", "Log(scale)"))) + p <- collect(select(predict(model, df), "prediction")) + expect_equal(p$prediction, c(3.724591, 2.545368, 3.079035, 3.079035, + 2.390146, 2.891269, 2.891269), tolerance = 1e-4) + + # Test model save/load + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-survreg", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + stats2 <- summary(model2) + coefs2 <- as.vector(stats2$coefficients[, 1]) + expect_equal(coefs, coefs2) + expect_equal(rownames(stats$coefficients), rownames(stats2$coefficients)) + + unlink(modelPath) + } + + # Test survival::survreg + if (requireNamespace("survival", quietly = TRUE)) { + rData <- list(time = c(4, 3, 1, 1, 2, 2, 3), status = c(1, 1, 1, 0, 1, 1, 0), + x = c(0, 2, 1, 1, 1, 0, 0), sex = c(0, 0, 0, 0, 1, 1, 1)) + expect_error( + model <- survival::survreg(formula = survival::Surv(time, status) ~ x + sex, data = rData), + NA) + expect_equal(predict(model, rData)[[1]], 3.724591, tolerance = 1e-4) + + # Test stringIndexerOrderType + rData <- as.data.frame(rData) + rData$sex2 <- c("female", "male")[rData$sex + 1] + df <- createDataFrame(rData) + expect_error( + rModel <- survival::survreg(survival::Surv(time, status) ~ x + sex2, rData), NA) + rCoefs <- as.numeric(summary(rModel)$table[, 1]) + model <- spark.survreg(df, Surv(time, status) ~ x + sex2) + coefs <- as.vector(summary(model)$coefficients[, 1]) + o <- order(rCoefs) + # stringIndexerOrderType = "frequencyDesc" produces different estimates from R + expect_false(all(abs(rCoefs[o] - coefs[o]) < 1e-4)) + + # stringIndexerOrderType = "alphabetDesc" produces the same estimates as R + model <- spark.survreg(df, Surv(time, status) ~ x + sex2, + stringIndexerOrderType = "alphabetDesc") + coefs <- as.vector(summary(model)$coefficients[, 1]) + expect_true(all(abs(rCoefs[o] - coefs[o]) < 1e-4)) + } +}) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_mllib_stat.R b/R/pkg/tests/fulltests/test_mllib_stat.R new file mode 100644 index 000000000000..1600833a5d03 --- /dev/null +++ b/R/pkg/tests/fulltests/test_mllib_stat.R @@ -0,0 +1,53 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(testthat) + +context("MLlib statistics algorithms") + +# Tests for MLlib statistics algorithms in SparkR +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + +test_that("spark.kstest", { + data <- data.frame(test = c(0.1, 0.15, 0.2, 0.3, 0.25, -1, -0.5)) + df <- createDataFrame(data) + testResult <- spark.kstest(df, "test", "norm") + stats <- summary(testResult) + + rStats <- ks.test(data$test, "pnorm", alternative = "two.sided") + + expect_equal(stats$p.value, rStats$p.value, tolerance = 1e-4) + expect_equal(stats$statistic, unname(rStats$statistic), tolerance = 1e-4) + expect_match(capture.output(stats)[1], "Kolmogorov-Smirnov test summary:") + + testResult <- spark.kstest(df, "test", "norm", -0.5) + stats <- summary(testResult) + + rStats <- ks.test(data$test, "pnorm", -0.5, 1, alternative = "two.sided") + + expect_equal(stats$p.value, rStats$p.value, tolerance = 1e-4) + expect_equal(stats$statistic, unname(rStats$statistic), tolerance = 1e-4) + expect_match(capture.output(stats)[1], "Kolmogorov-Smirnov test summary:") + + # Test print.summary.KSTest + printStats <- capture.output(print.summary.KSTest(stats)) + expect_match(printStats[1], "Kolmogorov-Smirnov test summary:") + expect_match(printStats[5], + "Low presumption against null hypothesis: Sample follows theoretical distribution. ") +}) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_mllib_tree.R b/R/pkg/tests/fulltests/test_mllib_tree.R new file mode 100644 index 000000000000..facd3a941cf1 --- /dev/null +++ b/R/pkg/tests/fulltests/test_mllib_tree.R @@ -0,0 +1,365 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(testthat) + +context("MLlib tree-based algorithms") + +# Tests for MLlib tree-based algorithms in SparkR +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + +absoluteSparkPath <- function(x) { + sparkHome <- sparkR.conf("spark.home") + file.path(sparkHome, x) +} + +test_that("spark.gbt", { + # regression + data <- suppressWarnings(createDataFrame(longley)) + model <- spark.gbt(data, Employed ~ ., "regression", maxDepth = 5, maxBins = 16, seed = 123) + predictions <- collect(predict(model, data)) + expect_equal(predictions$prediction, c(60.323, 61.122, 60.171, 61.187, + 63.221, 63.639, 64.989, 63.761, + 66.019, 67.857, 68.169, 66.513, + 68.655, 69.564, 69.331, 70.551), + tolerance = 1e-4) + stats <- summary(model) + expect_equal(stats$numTrees, 20) + expect_equal(stats$maxDepth, 5) + expect_equal(stats$formula, "Employed ~ .") + expect_equal(stats$numFeatures, 6) + expect_equal(length(stats$treeWeights), 20) + + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-gbtRegression", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + stats2 <- summary(model2) + expect_equal(stats$formula, stats2$formula) + expect_equal(stats$numFeatures, stats2$numFeatures) + expect_equal(stats$features, stats2$features) + expect_equal(stats$featureImportances, stats2$featureImportances) + expect_equal(stats$maxDepth, stats2$maxDepth) + expect_equal(stats$numTrees, stats2$numTrees) + expect_equal(stats$treeWeights, stats2$treeWeights) + + unlink(modelPath) + } + + # classification + # label must be binary - GBTClassifier currently only supports binary classification. + iris2 <- iris[iris$Species != "virginica", ] + data <- suppressWarnings(createDataFrame(iris2)) + model <- spark.gbt(data, Species ~ Petal_Length + Petal_Width, "classification", seed = 12) + stats <- summary(model) + expect_equal(stats$numFeatures, 2) + expect_equal(stats$numTrees, 20) + expect_equal(stats$maxDepth, 5) + expect_error(capture.output(stats), NA) + expect_true(length(capture.output(stats)) > 6) + predictions <- collect(predict(model, data))$prediction + # test string prediction values + expect_equal(length(grep("setosa", predictions)), 50) + expect_equal(length(grep("versicolor", predictions)), 50) + + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-gbtClassification", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + stats2 <- summary(model2) + expect_equal(stats$depth, stats2$depth) + expect_equal(stats$numNodes, stats2$numNodes) + expect_equal(stats$numClasses, stats2$numClasses) + + unlink(modelPath) + } + + iris2$NumericSpecies <- ifelse(iris2$Species == "setosa", 0, 1) + df <- suppressWarnings(createDataFrame(iris2)) + m <- spark.gbt(df, NumericSpecies ~ ., type = "classification", seed = 12) + s <- summary(m) + # test numeric prediction values + expect_equal(iris2$NumericSpecies, as.double(collect(predict(m, df))$prediction)) + expect_equal(s$numFeatures, 5) + expect_equal(s$numTrees, 20) + expect_equal(stats$maxDepth, 5) + + # spark.gbt classification can work on libsvm data + if (windows_with_hadoop()) { + data <- read.df(absoluteSparkPath("data/mllib/sample_binary_classification_data.txt"), + source = "libsvm") + model <- spark.gbt(data, label ~ features, "classification", seed = 12) + expect_equal(summary(model)$numFeatures, 692) + } + + # Test unseen labels + data <- data.frame(clicked = base::sample(c(0, 1), 10, replace = TRUE), + someString = base::sample(c("this", "that"), 10, replace = TRUE), + stringsAsFactors = FALSE) + trainidxs <- base::sample(nrow(data), nrow(data) * 0.7) + traindf <- as.DataFrame(data[trainidxs, ]) + testdf <- as.DataFrame(rbind(data[-trainidxs, ], c(0, "the other"))) + model <- spark.gbt(traindf, clicked ~ ., type = "classification", seed = 23) + predictions <- predict(model, testdf) + expect_error(collect(predictions)) + model <- spark.gbt(traindf, clicked ~ ., type = "classification", handleInvalid = "keep", + seed = 23) + predictions <- predict(model, testdf) + expect_equal(class(collect(predictions)$clicked[1]), "character") +}) + +test_that("spark.randomForest", { + # regression + data <- suppressWarnings(createDataFrame(longley)) + model <- spark.randomForest(data, Employed ~ ., "regression", maxDepth = 5, maxBins = 16, + numTrees = 1, seed = 1) + + predictions <- collect(predict(model, data)) + expect_equal(predictions$prediction, c(60.323, 61.122, 60.171, 61.187, + 63.221, 63.639, 64.989, 63.761, + 66.019, 67.857, 68.169, 66.513, + 68.655, 69.564, 69.331, 70.551), + tolerance = 1e-4) + + stats <- summary(model) + expect_equal(stats$numTrees, 1) + expect_equal(stats$maxDepth, 5) + expect_error(capture.output(stats), NA) + expect_true(length(capture.output(stats)) > 6) + + model <- spark.randomForest(data, Employed ~ ., "regression", maxDepth = 5, maxBins = 16, + numTrees = 20, seed = 123) + predictions <- collect(predict(model, data)) + expect_equal(predictions$prediction, c(60.32820, 61.22315, 60.69025, 62.11070, + 63.53160, 64.05470, 65.12710, 64.30450, + 66.70910, 67.86125, 68.08700, 67.21865, + 68.89275, 69.53180, 69.39640, 69.68250), + tolerance = 1e-4) + stats <- summary(model) + expect_equal(stats$numTrees, 20) + expect_equal(stats$maxDepth, 5) + + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-randomForestRegression", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + stats2 <- summary(model2) + expect_equal(stats$formula, stats2$formula) + expect_equal(stats$numFeatures, stats2$numFeatures) + expect_equal(stats$features, stats2$features) + expect_equal(stats$featureImportances, stats2$featureImportances) + expect_equal(stats$numTrees, stats2$numTrees) + expect_equal(stats$maxDepth, stats2$maxDepth) + expect_equal(stats$treeWeights, stats2$treeWeights) + + unlink(modelPath) + } + + # classification + data <- suppressWarnings(createDataFrame(iris)) + model <- spark.randomForest(data, Species ~ Petal_Length + Petal_Width, "classification", + maxDepth = 5, maxBins = 16, seed = 123) + + stats <- summary(model) + expect_equal(stats$numFeatures, 2) + expect_equal(stats$numTrees, 20) + expect_equal(stats$maxDepth, 5) + expect_error(capture.output(stats), NA) + expect_true(length(capture.output(stats)) > 6) + # Test string prediction values + predictions <- collect(predict(model, data))$prediction + expect_equal(length(grep("setosa", predictions)), 50) + expect_equal(length(grep("versicolor", predictions)), 50) + + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-randomForestClassification", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + stats2 <- summary(model2) + expect_equal(stats$depth, stats2$depth) + expect_equal(stats$numNodes, stats2$numNodes) + expect_equal(stats$numClasses, stats2$numClasses) + + unlink(modelPath) + } + + # Test numeric response variable + labelToIndex <- function(species) { + switch(as.character(species), + setosa = 0.0, + versicolor = 1.0, + virginica = 2.0 + ) + } + iris$NumericSpecies <- lapply(iris$Species, labelToIndex) + data <- suppressWarnings(createDataFrame(iris[-5])) + model <- spark.randomForest(data, NumericSpecies ~ Petal_Length + Petal_Width, "classification", + maxDepth = 5, maxBins = 16, seed = 123) + stats <- summary(model) + expect_equal(stats$numFeatures, 2) + expect_equal(stats$numTrees, 20) + expect_equal(stats$maxDepth, 5) + + # Test numeric prediction values + predictions <- collect(predict(model, data))$prediction + expect_equal(length(grep("1.0", predictions)), 50) + expect_equal(length(grep("2.0", predictions)), 50) + + # Test unseen labels + data <- data.frame(clicked = base::sample(c(0, 1), 10, replace = TRUE), + someString = base::sample(c("this", "that"), 10, replace = TRUE), + stringsAsFactors = FALSE) + trainidxs <- base::sample(nrow(data), nrow(data) * 0.7) + traindf <- as.DataFrame(data[trainidxs, ]) + testdf <- as.DataFrame(rbind(data[-trainidxs, ], c(0, "the other"))) + model <- spark.randomForest(traindf, clicked ~ ., type = "classification", + maxDepth = 10, maxBins = 10, numTrees = 10, seed = 123) + predictions <- predict(model, testdf) + expect_error(collect(predictions)) + model <- spark.randomForest(traindf, clicked ~ ., type = "classification", + maxDepth = 10, maxBins = 10, numTrees = 10, + handleInvalid = "keep", seed = 123) + predictions <- predict(model, testdf) + expect_equal(class(collect(predictions)$clicked[1]), "character") + + # spark.randomForest classification can work on libsvm data + if (windows_with_hadoop()) { + data <- read.df(absoluteSparkPath("data/mllib/sample_multiclass_classification_data.txt"), + source = "libsvm") + model <- spark.randomForest(data, label ~ features, "classification", seed = 123) + expect_equal(summary(model)$numFeatures, 4) + } +}) + +test_that("spark.decisionTree", { + # regression + data <- suppressWarnings(createDataFrame(longley)) + model <- spark.decisionTree(data, Employed ~ ., "regression", maxDepth = 5, maxBins = 16, + seed = 42) + + predictions <- collect(predict(model, data)) + expect_equal(predictions$prediction, c(60.323, 61.122, 60.171, 61.187, + 63.221, 63.639, 64.989, 63.761, + 66.019, 67.857, 68.169, 66.513, + 68.655, 69.564, 69.331, 70.551), + tolerance = 1e-4) + + stats <- summary(model) + expect_equal(stats$maxDepth, 5) + expect_error(capture.output(stats), NA) + expect_true(length(capture.output(stats)) > 6) + + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-decisionTreeRegression", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + stats2 <- summary(model2) + expect_equal(stats$formula, stats2$formula) + expect_equal(stats$numFeatures, stats2$numFeatures) + expect_equal(stats$features, stats2$features) + expect_equal(stats$featureImportances, stats2$featureImportances) + expect_equal(stats$maxDepth, stats2$maxDepth) + + unlink(modelPath) + } + + # classification + data <- suppressWarnings(createDataFrame(iris)) + model <- spark.decisionTree(data, Species ~ Petal_Length + Petal_Width, "classification", + maxDepth = 5, maxBins = 16, seed = 43) + + stats <- summary(model) + expect_equal(stats$numFeatures, 2) + expect_equal(stats$maxDepth, 5) + expect_error(capture.output(stats), NA) + expect_true(length(capture.output(stats)) > 6) + # Test string prediction values + predictions <- collect(predict(model, data))$prediction + expect_equal(length(grep("setosa", predictions)), 50) + expect_equal(length(grep("versicolor", predictions)), 50) + + if (windows_with_hadoop()) { + modelPath <- tempfile(pattern = "spark-decisionTreeClassification", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + stats2 <- summary(model2) + expect_equal(stats$depth, stats2$depth) + expect_equal(stats$numNodes, stats2$numNodes) + expect_equal(stats$numClasses, stats2$numClasses) + + unlink(modelPath) + } + + # Test numeric response variable + labelToIndex <- function(species) { + switch(as.character(species), + setosa = 0.0, + versicolor = 1.0, + virginica = 2.0 + ) + } + iris$NumericSpecies <- lapply(iris$Species, labelToIndex) + data <- suppressWarnings(createDataFrame(iris[-5])) + model <- spark.decisionTree(data, NumericSpecies ~ Petal_Length + Petal_Width, "classification", + maxDepth = 5, maxBins = 16, seed = 44) + stats <- summary(model) + expect_equal(stats$numFeatures, 2) + expect_equal(stats$maxDepth, 5) + + # Test numeric prediction values + predictions <- collect(predict(model, data))$prediction + expect_equal(length(grep("1.0", predictions)), 50) + expect_equal(length(grep("2.0", predictions)), 50) + + # spark.decisionTree classification can work on libsvm data + if (windows_with_hadoop()) { + data <- read.df(absoluteSparkPath("data/mllib/sample_multiclass_classification_data.txt"), + source = "libsvm") + model <- spark.decisionTree(data, label ~ features, "classification", seed = 45) + expect_equal(summary(model)$numFeatures, 4) + } + + # Test unseen labels + data <- data.frame(clicked = base::sample(c(0, 1), 10, replace = TRUE), + someString = base::sample(c("this", "that"), 10, replace = TRUE), + stringsAsFactors = FALSE) + trainidxs <- base::sample(nrow(data), nrow(data) * 0.7) + traindf <- as.DataFrame(data[trainidxs, ]) + testdf <- as.DataFrame(rbind(data[-trainidxs, ], c(0, "the other"))) + model <- spark.decisionTree(traindf, clicked ~ ., type = "classification", + maxDepth = 5, maxBins = 16, seed = 46) + predictions <- predict(model, testdf) + expect_error(collect(predictions)) + model <- spark.decisionTree(traindf, clicked ~ ., type = "classification", + maxDepth = 5, maxBins = 16, handleInvalid = "keep", seed = 46) + predictions <- predict(model, testdf) + expect_equal(class(collect(predictions)$clicked[1]), "character") +}) + +sparkR.session.stop() diff --git a/R/pkg/inst/tests/test_parallelize_collect.R b/R/pkg/tests/fulltests/test_parallelize_collect.R similarity index 79% rename from R/pkg/inst/tests/test_parallelize_collect.R rename to R/pkg/tests/fulltests/test_parallelize_collect.R index 2552127cc547..3d122ccaf448 100644 --- a/R/pkg/inst/tests/test_parallelize_collect.R +++ b/R/pkg/tests/fulltests/test_parallelize_collect.R @@ -33,7 +33,8 @@ numPairs <- list(list(1, 1), list(1, 2), list(2, 2), list(2, 3)) strPairs <- list(list(strList, strList), list(strList, strList)) # JavaSparkContext handle -jsc <- sparkR.init() +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) +jsc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) # Tests @@ -66,22 +67,22 @@ test_that("parallelize() on simple vectors and lists returns an RDD", { test_that("collect(), following a parallelize(), gives back the original collections", { numVectorRDD <- parallelize(jsc, numVector, 10) - expect_equal(collect(numVectorRDD), as.list(numVector)) + expect_equal(collectRDD(numVectorRDD), as.list(numVector)) numListRDD <- parallelize(jsc, numList, 1) numListRDD2 <- parallelize(jsc, numList, 4) - expect_equal(collect(numListRDD), as.list(numList)) - expect_equal(collect(numListRDD2), as.list(numList)) + expect_equal(collectRDD(numListRDD), as.list(numList)) + expect_equal(collectRDD(numListRDD2), as.list(numList)) strVectorRDD <- parallelize(jsc, strVector, 2) strVectorRDD2 <- parallelize(jsc, strVector, 3) - expect_equal(collect(strVectorRDD), as.list(strVector)) - expect_equal(collect(strVectorRDD2), as.list(strVector)) + expect_equal(collectRDD(strVectorRDD), as.list(strVector)) + expect_equal(collectRDD(strVectorRDD2), as.list(strVector)) strListRDD <- parallelize(jsc, strList, 4) strListRDD2 <- parallelize(jsc, strList, 1) - expect_equal(collect(strListRDD), as.list(strList)) - expect_equal(collect(strListRDD2), as.list(strList)) + expect_equal(collectRDD(strListRDD), as.list(strList)) + expect_equal(collectRDD(strListRDD2), as.list(strList)) }) test_that("regression: collect() following a parallelize() does not drop elements", { @@ -89,7 +90,7 @@ test_that("regression: collect() following a parallelize() does not drop element collLen <- 10 numPart <- 6 expected <- runif(collLen) - actual <- collect(parallelize(jsc, expected, numPart)) + actual <- collectRDD(parallelize(jsc, expected, numPart)) expect_equal(actual, as.list(expected)) }) @@ -98,12 +99,14 @@ test_that("parallelize() and collect() work for lists of pairs (pairwise data)", numPairsRDDD1 <- parallelize(jsc, numPairs, 1) numPairsRDDD2 <- parallelize(jsc, numPairs, 2) numPairsRDDD3 <- parallelize(jsc, numPairs, 3) - expect_equal(collect(numPairsRDDD1), numPairs) - expect_equal(collect(numPairsRDDD2), numPairs) - expect_equal(collect(numPairsRDDD3), numPairs) + expect_equal(collectRDD(numPairsRDDD1), numPairs) + expect_equal(collectRDD(numPairsRDDD2), numPairs) + expect_equal(collectRDD(numPairsRDDD3), numPairs) # can also leave out the parameter name, if the params are supplied in order strPairsRDDD1 <- parallelize(jsc, strPairs, 1) strPairsRDDD2 <- parallelize(jsc, strPairs, 2) - expect_equal(collect(strPairsRDDD1), strPairs) - expect_equal(collect(strPairsRDDD2), strPairs) + expect_equal(collectRDD(strPairsRDDD1), strPairs) + expect_equal(collectRDD(strPairsRDDD2), strPairs) }) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_rdd.R b/R/pkg/tests/fulltests/test_rdd.R new file mode 100644 index 000000000000..6ee1fceffd82 --- /dev/null +++ b/R/pkg/tests/fulltests/test_rdd.R @@ -0,0 +1,804 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("basic RDD functions") + +# JavaSparkContext handle +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) + +# Data +nums <- 1:10 +rdd <- parallelize(sc, nums, 2L) + +intPairs <- list(list(1L, -1), list(2L, 100), list(2L, 1), list(1L, 200)) +intRdd <- parallelize(sc, intPairs, 2L) + +test_that("get number of partitions in RDD", { + expect_equal(getNumPartitionsRDD(rdd), 2) + expect_equal(getNumPartitionsRDD(intRdd), 2) +}) + +test_that("first on RDD", { + expect_equal(firstRDD(rdd), 1) + newrdd <- lapply(rdd, function(x) x + 1) + expect_equal(firstRDD(newrdd), 2) +}) + +test_that("count and length on RDD", { + expect_equal(countRDD(rdd), 10) + expect_equal(lengthRDD(rdd), 10) +}) + +test_that("count by values and keys", { + mods <- lapply(rdd, function(x) { x %% 3 }) + actual <- countByValue(mods) + expected <- list(list(0, 3L), list(1, 4L), list(2, 3L)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + actual <- countByKey(intRdd) + expected <- list(list(2L, 2L), list(1L, 2L)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("lapply on RDD", { + multiples <- lapply(rdd, function(x) { 2 * x }) + actual <- collectRDD(multiples) + expect_equal(actual, as.list(nums * 2)) +}) + +test_that("lapplyPartition on RDD", { + sums <- lapplyPartition(rdd, function(part) { sum(unlist(part)) }) + actual <- collectRDD(sums) + expect_equal(actual, list(15, 40)) +}) + +test_that("mapPartitions on RDD", { + sums <- mapPartitions(rdd, function(part) { sum(unlist(part)) }) + actual <- collectRDD(sums) + expect_equal(actual, list(15, 40)) +}) + +test_that("flatMap() on RDDs", { + flat <- flatMap(intRdd, function(x) { list(x, x) }) + actual <- collectRDD(flat) + expect_equal(actual, rep(intPairs, each = 2)) +}) + +test_that("filterRDD on RDD", { + filtered.rdd <- filterRDD(rdd, function(x) { x %% 2 == 0 }) + actual <- collectRDD(filtered.rdd) + expect_equal(actual, list(2, 4, 6, 8, 10)) + + filtered.rdd <- Filter(function(x) { x[[2]] < 0 }, intRdd) + actual <- collectRDD(filtered.rdd) + expect_equal(actual, list(list(1L, -1))) + + # Filter out all elements. + filtered.rdd <- filterRDD(rdd, function(x) { x > 10 }) + actual <- collectRDD(filtered.rdd) + expect_equal(actual, list()) +}) + +test_that("lookup on RDD", { + vals <- lookup(intRdd, 1L) + expect_equal(vals, list(-1, 200)) + + vals <- lookup(intRdd, 3L) + expect_equal(vals, list()) +}) + +test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { + rdd2 <- rdd + for (i in 1:12) + rdd2 <- lapplyPartitionsWithIndex( + rdd2, function(partIndex, part) { + part <- as.list(unlist(part) * partIndex + i) + }) + rdd2 <- lapply(rdd2, function(x) x + x) + actual <- collectRDD(rdd2) + expected <- list(24, 24, 24, 24, 24, + 168, 170, 172, 174, 176) + expect_equal(actual, expected) +}) + +test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkpoint()", { + # RDD + rdd2 <- rdd + # PipelinedRDD + rdd2 <- lapplyPartitionsWithIndex( + rdd2, + function(partIndex, part) { + part <- as.list(unlist(part) * partIndex) + }) + + cacheRDD(rdd2) + expect_true(rdd2@env$isCached) + rdd2 <- lapply(rdd2, function(x) x) + expect_false(rdd2@env$isCached) + + unpersistRDD(rdd2) + expect_false(rdd2@env$isCached) + + persistRDD(rdd2, "MEMORY_AND_DISK") + expect_true(rdd2@env$isCached) + rdd2 <- lapply(rdd2, function(x) x) + expect_false(rdd2@env$isCached) + + unpersistRDD(rdd2) + expect_false(rdd2@env$isCached) + + tempDir <- tempfile(pattern = "checkpoint") + setCheckpointDirSC(sc, tempDir) + checkpointRDD(rdd2) + expect_true(rdd2@env$isCheckpointed) + + rdd2 <- lapply(rdd2, function(x) x) + expect_false(rdd2@env$isCached) + expect_false(rdd2@env$isCheckpointed) + + # make sure the data is collectable + collectRDD(rdd2) + + unlink(tempDir) +}) + +test_that("reduce on RDD", { + sum <- reduce(rdd, "+") + expect_equal(sum, 55) + + # Also test with an inline function + sumInline <- reduce(rdd, function(x, y) { x + y }) + expect_equal(sumInline, 55) +}) + +test_that("lapply with dependency", { + fa <- 5 + multiples <- lapply(rdd, function(x) { fa * x }) + actual <- collectRDD(multiples) + + expect_equal(actual, as.list(nums * 5)) +}) + +test_that("lapplyPartitionsWithIndex on RDDs", { + func <- function(partIndex, part) { list(partIndex, Reduce("+", part)) } + actual <- collectRDD(lapplyPartitionsWithIndex(rdd, func), flatten = FALSE) + expect_equal(actual, list(list(0, 15), list(1, 40))) + + pairsRDD <- parallelize(sc, list(list(1, 2), list(3, 4), list(4, 8)), 1L) + partitionByParity <- function(key) { if (key %% 2 == 1) 0 else 1 } + mkTup <- function(partIndex, part) { list(partIndex, part) } + actual <- collectRDD(lapplyPartitionsWithIndex( + partitionByRDD(pairsRDD, 2L, partitionByParity), + mkTup), + FALSE) + expect_equal(actual, list(list(0, list(list(1, 2), list(3, 4))), + list(1, list(list(4, 8))))) +}) + +test_that("sampleRDD() on RDDs", { + expect_equal(unlist(collectRDD(sampleRDD(rdd, FALSE, 1.0, 2014L))), nums) +}) + +test_that("takeSample() on RDDs", { + # ported from RDDSuite.scala, modified seeds + data <- parallelize(sc, 1:100, 2L) + for (seed in 4:5) { + s <- takeSample(data, FALSE, 20L, seed) + expect_equal(length(s), 20L) + expect_equal(length(unique(s)), 20L) + for (elem in s) { + expect_true(elem >= 1 && elem <= 100) + } + } + for (seed in 4:5) { + s <- takeSample(data, FALSE, 200L, seed) + expect_equal(length(s), 100L) + expect_equal(length(unique(s)), 100L) + for (elem in s) { + expect_true(elem >= 1 && elem <= 100) + } + } + for (seed in 4:5) { + s <- takeSample(data, TRUE, 20L, seed) + expect_equal(length(s), 20L) + for (elem in s) { + expect_true(elem >= 1 && elem <= 100) + } + } + for (seed in 4:5) { + s <- takeSample(data, TRUE, 100L, seed) + expect_equal(length(s), 100L) + # Chance of getting all distinct elements is astronomically low, so test we + # got less than 100 + expect_true(length(unique(s)) < 100L) + } + for (seed in 4:5) { + s <- takeSample(data, TRUE, 200L, seed) + expect_equal(length(s), 200L) + # Chance of getting all distinct elements is still quite low, so test we + # got less than 100 + expect_true(length(unique(s)) < 100L) + } +}) + +test_that("mapValues() on pairwise RDDs", { + multiples <- mapValues(intRdd, function(x) { x * 2 }) + actual <- collectRDD(multiples) + expected <- lapply(intPairs, function(x) { + list(x[[1]], x[[2]] * 2) + }) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("flatMapValues() on pairwise RDDs", { + l <- parallelize(sc, list(list(1, c(1, 2)), list(2, c(3, 4)))) + actual <- collectRDD(flatMapValues(l, function(x) { x })) + expect_equal(actual, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) + + # Generate x to x+1 for every value + actual <- collectRDD(flatMapValues(intRdd, function(x) { x: (x + 1) })) + expect_equal(actual, + list(list(1L, -1), list(1L, 0), list(2L, 100), list(2L, 101), + list(2L, 1), list(2L, 2), list(1L, 200), list(1L, 201))) +}) + +test_that("reduceByKeyLocally() on PairwiseRDDs", { + pairs <- parallelize(sc, list(list(1, 2), list(1.1, 3), list(1, 4)), 2L) + actual <- reduceByKeyLocally(pairs, "+") + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(1, 6), list(1.1, 3)))) + + pairs <- parallelize(sc, list(list("abc", 1.2), list(1.1, 0), list("abc", 1.3), + list("bb", 5)), 4L) + actual <- reduceByKeyLocally(pairs, "+") + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("abc", 2.5), list(1.1, 0), list("bb", 5)))) +}) + +test_that("distinct() on RDDs", { + nums.rep2 <- rep(1:10, 2) + rdd.rep2 <- parallelize(sc, nums.rep2, 2L) + uniques <- distinctRDD(rdd.rep2) + actual <- sort(unlist(collectRDD(uniques))) + expect_equal(actual, nums) +}) + +test_that("maximum() on RDDs", { + max <- maximum(rdd) + expect_equal(max, 10) +}) + +test_that("minimum() on RDDs", { + min <- minimum(rdd) + expect_equal(min, 1) +}) + +test_that("sumRDD() on RDDs", { + sum <- sumRDD(rdd) + expect_equal(sum, 55) +}) + +test_that("keyBy on RDDs", { + func <- function(x) { x * x } + keys <- keyBy(rdd, func) + actual <- collectRDD(keys) + expect_equal(actual, lapply(nums, function(x) { list(func(x), x) })) +}) + +test_that("repartition/coalesce on RDDs", { + rdd <- parallelize(sc, 1:20, 4L) # each partition contains 5 elements + + # repartition + r1 <- repartitionRDD(rdd, 2) + expect_equal(getNumPartitionsRDD(r1), 2L) + count <- length(collectPartition(r1, 0L)) + expect_true(count >= 8 && count <= 12) + + r2 <- repartitionRDD(rdd, 6) + expect_equal(getNumPartitionsRDD(r2), 6L) + count <- length(collectPartition(r2, 0L)) + expect_true(count >= 0 && count <= 4) + + # coalesce + r3 <- coalesceRDD(rdd, 1) + expect_equal(getNumPartitionsRDD(r3), 1L) + count <- length(collectPartition(r3, 0L)) + expect_equal(count, 20) +}) + +test_that("sortBy() on RDDs", { + sortedRdd <- sortBy(rdd, function(x) { x * x }, ascending = FALSE) + actual <- collectRDD(sortedRdd) + expect_equal(actual, as.list(sort(nums, decreasing = TRUE))) + + rdd2 <- parallelize(sc, sort(nums, decreasing = TRUE), 2L) + sortedRdd2 <- sortBy(rdd2, function(x) { x * x }) + actual <- collectRDD(sortedRdd2) + expect_equal(actual, as.list(nums)) +}) + +test_that("takeOrdered() on RDDs", { + l <- list(10, 1, 2, 9, 3, 4, 5, 6, 7) + rdd <- parallelize(sc, l) + actual <- takeOrdered(rdd, 6L) + expect_equal(actual, as.list(sort(unlist(l)))[1:6]) + + l <- list("e", "d", "c", "d", "a") + rdd <- parallelize(sc, l) + actual <- takeOrdered(rdd, 3L) + expect_equal(actual, as.list(sort(unlist(l)))[1:3]) +}) + +test_that("top() on RDDs", { + l <- list(10, 1, 2, 9, 3, 4, 5, 6, 7) + rdd <- parallelize(sc, l) + actual <- top(rdd, 6L) + expect_equal(actual, as.list(sort(unlist(l), decreasing = TRUE))[1:6]) + + l <- list("e", "d", "c", "d", "a") + rdd <- parallelize(sc, l) + actual <- top(rdd, 3L) + expect_equal(actual, as.list(sort(unlist(l), decreasing = TRUE))[1:3]) +}) + +test_that("fold() on RDDs", { + actual <- fold(rdd, 0, "+") + expect_equal(actual, Reduce("+", nums, 0)) + + rdd <- parallelize(sc, list()) + actual <- fold(rdd, 0, "+") + expect_equal(actual, 0) +}) + +test_that("aggregateRDD() on RDDs", { + rdd <- parallelize(sc, list(1, 2, 3, 4)) + zeroValue <- list(0, 0) + seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } + combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } + actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp) + expect_equal(actual, list(10, 4)) + + rdd <- parallelize(sc, list()) + actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp) + expect_equal(actual, list(0, 0)) +}) + +test_that("zipWithUniqueId() on RDDs", { + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) + actual <- collectRDD(zipWithUniqueId(rdd)) + expected <- list(list("a", 0), list("b", 1), list("c", 4), + list("d", 2), list("e", 5)) + expect_equal(actual, expected) + + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L) + actual <- collectRDD(zipWithUniqueId(rdd)) + expected <- list(list("a", 0), list("b", 1), list("c", 2), + list("d", 3), list("e", 4)) + expect_equal(actual, expected) +}) + +test_that("zipWithIndex() on RDDs", { + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) + actual <- collectRDD(zipWithIndex(rdd)) + expected <- list(list("a", 0), list("b", 1), list("c", 2), + list("d", 3), list("e", 4)) + expect_equal(actual, expected) + + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L) + actual <- collectRDD(zipWithIndex(rdd)) + expected <- list(list("a", 0), list("b", 1), list("c", 2), + list("d", 3), list("e", 4)) + expect_equal(actual, expected) +}) + +test_that("glom() on RDD", { + rdd <- parallelize(sc, as.list(1:4), 2L) + actual <- collectRDD(glom(rdd)) + expect_equal(actual, list(list(1, 2), list(3, 4))) +}) + +test_that("keys() on RDDs", { + keys <- keys(intRdd) + actual <- collectRDD(keys) + expect_equal(actual, lapply(intPairs, function(x) { x[[1]] })) +}) + +test_that("values() on RDDs", { + values <- values(intRdd) + actual <- collectRDD(values) + expect_equal(actual, lapply(intPairs, function(x) { x[[2]] })) +}) + +test_that("pipeRDD() on RDDs", { + actual <- collectRDD(pipeRDD(rdd, "more")) + expected <- as.list(as.character(1:10)) + expect_equal(actual, expected) + + trailed.rdd <- parallelize(sc, c("1", "", "2\n", "3\n\r\n")) + actual <- collectRDD(pipeRDD(trailed.rdd, "sort")) + expected <- list("", "1", "2", "3") + expect_equal(actual, expected) + + rev.nums <- 9:0 + rev.rdd <- parallelize(sc, rev.nums, 2L) + actual <- collectRDD(pipeRDD(rev.rdd, "sort")) + expected <- as.list(as.character(c(5:9, 0:4))) + expect_equal(actual, expected) +}) + +test_that("zipRDD() on RDDs", { + rdd1 <- parallelize(sc, 0:4, 2) + rdd2 <- parallelize(sc, 1000:1004, 2) + actual <- collectRDD(zipRDD(rdd1, rdd2)) + expect_equal(actual, + list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004))) + + mockFile <- c("Spark is pretty.", "Spark is awesome.") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName, 1) + actual <- collectRDD(zipRDD(rdd, rdd)) + expected <- lapply(mockFile, function(x) { list(x, x) }) + expect_equal(actual, expected) + + rdd1 <- parallelize(sc, 0:1, 1) + actual <- collectRDD(zipRDD(rdd1, rdd)) + expected <- lapply(0:1, function(x) { list(x, mockFile[x + 1]) }) + expect_equal(actual, expected) + + rdd1 <- map(rdd, function(x) { x }) + actual <- collectRDD(zipRDD(rdd, rdd1)) + expected <- lapply(mockFile, function(x) { list(x, x) }) + expect_equal(actual, expected) + + unlink(fileName) +}) + +test_that("cartesian() on RDDs", { + rdd <- parallelize(sc, 1:3) + actual <- collectRDD(cartesian(rdd, rdd)) + expect_equal(sortKeyValueList(actual), + list( + list(1, 1), list(1, 2), list(1, 3), + list(2, 1), list(2, 2), list(2, 3), + list(3, 1), list(3, 2), list(3, 3))) + + # test case where one RDD is empty + emptyRdd <- parallelize(sc, list()) + actual <- collectRDD(cartesian(rdd, emptyRdd)) + expect_equal(actual, list()) + + mockFile <- c("Spark is pretty.", "Spark is awesome.") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + actual <- collectRDD(cartesian(rdd, rdd)) + expected <- list( + list("Spark is awesome.", "Spark is pretty."), + list("Spark is awesome.", "Spark is awesome."), + list("Spark is pretty.", "Spark is pretty."), + list("Spark is pretty.", "Spark is awesome.")) + expect_equal(sortKeyValueList(actual), expected) + + rdd1 <- parallelize(sc, 0:1) + actual <- collectRDD(cartesian(rdd1, rdd)) + expect_equal(sortKeyValueList(actual), + list( + list(0, "Spark is pretty."), + list(0, "Spark is awesome."), + list(1, "Spark is pretty."), + list(1, "Spark is awesome."))) + + rdd1 <- map(rdd, function(x) { x }) + actual <- collectRDD(cartesian(rdd, rdd1)) + expect_equal(sortKeyValueList(actual), expected) + + unlink(fileName) +}) + +test_that("subtract() on RDDs", { + l <- list(1, 1, 2, 2, 3, 4) + rdd1 <- parallelize(sc, l) + + # subtract by itself + actual <- collectRDD(subtract(rdd1, rdd1)) + expect_equal(actual, list()) + + # subtract by an empty RDD + rdd2 <- parallelize(sc, list()) + actual <- collectRDD(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode = "integer"))), + l) + + rdd2 <- parallelize(sc, list(2, 4)) + actual <- collectRDD(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode = "integer"))), + list(1, 1, 3)) + + l <- list("a", "a", "b", "b", "c", "d") + rdd1 <- parallelize(sc, l) + rdd2 <- parallelize(sc, list("b", "d")) + actual <- collectRDD(subtract(rdd1, rdd2)) + expect_equal(as.list(sort(as.vector(actual, mode = "character"))), + list("a", "a", "c")) +}) + +test_that("subtractByKey() on pairwise RDDs", { + l <- list(list("a", 1), list("b", 4), + list("b", 5), list("a", 2)) + rdd1 <- parallelize(sc, l) + + # subtractByKey by itself + actual <- collectRDD(subtractByKey(rdd1, rdd1)) + expect_equal(actual, list()) + + # subtractByKey by an empty RDD + rdd2 <- parallelize(sc, list()) + actual <- collectRDD(subtractByKey(rdd1, rdd2)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(l)) + + rdd2 <- parallelize(sc, list(list("a", 3), list("c", 1))) + actual <- collectRDD(subtractByKey(rdd1, rdd2)) + expect_equal(actual, + list(list("b", 4), list("b", 5))) + + l <- list(list(1, 1), list(2, 4), + list(2, 5), list(1, 2)) + rdd1 <- parallelize(sc, l) + rdd2 <- parallelize(sc, list(list(1, 3), list(3, 1))) + actual <- collectRDD(subtractByKey(rdd1, rdd2)) + expect_equal(actual, + list(list(2, 4), list(2, 5))) +}) + +test_that("intersection() on RDDs", { + # intersection with self + actual <- collectRDD(intersection(rdd, rdd)) + expect_equal(sort(as.integer(actual)), nums) + + # intersection with an empty RDD + emptyRdd <- parallelize(sc, list()) + actual <- collectRDD(intersection(rdd, emptyRdd)) + expect_equal(actual, list()) + + rdd1 <- parallelize(sc, list(1, 10, 2, 3, 4, 5)) + rdd2 <- parallelize(sc, list(1, 6, 2, 3, 7, 8)) + actual <- collectRDD(intersection(rdd1, rdd2)) + expect_equal(sort(as.integer(actual)), 1:3) +}) + +test_that("join() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) + rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) + actual <- collectRDD(joinRDD(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(1, list(1, 2)), list(1, list(1, 3))))) + + rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4))) + rdd2 <- parallelize(sc, list(list("a", 2), list("a", 3))) + actual <- collectRDD(joinRDD(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("a", list(1, 2)), list("a", list(1, 3))))) + + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 2))) + rdd2 <- parallelize(sc, list(list(3, 3), list(4, 4))) + actual <- collectRDD(joinRDD(rdd1, rdd2, 2L)) + expect_equal(actual, list()) + + rdd1 <- parallelize(sc, list(list("a", 1), list("b", 2))) + rdd2 <- parallelize(sc, list(list("c", 3), list("d", 4))) + actual <- collectRDD(joinRDD(rdd1, rdd2, 2L)) + expect_equal(actual, list()) +}) + +test_that("leftOuterJoin() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) + rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) + actual <- collectRDD(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4))) + rdd2 <- parallelize(sc, list(list("a", 2), list("a", 3))) + actual <- collectRDD(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(4, NULL)), list("a", list(1, 2)), list("a", list(1, 3))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 2))) + rdd2 <- parallelize(sc, list(list(3, 3), list(4, 4))) + actual <- collectRDD(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(1, NULL)), list(2, list(2, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a", 1), list("b", 2))) + rdd2 <- parallelize(sc, list(list("c", 3), list("d", 4))) + actual <- collectRDD(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(2, NULL)), list("a", list(1, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) +}) + +test_that("rightOuterJoin() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3))) + rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) + actual <- collectRDD(rightOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a", 2), list("a", 3))) + rdd2 <- parallelize(sc, list(list("a", 1), list("b", 4))) + actual <- collectRDD(rightOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), list("a", list(3, 1))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 2))) + rdd2 <- parallelize(sc, list(list(3, 3), list(4, 4))) + actual <- collectRDD(rightOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(3, list(NULL, 3)), list(4, list(NULL, 4))))) + + rdd1 <- parallelize(sc, list(list("a", 1), list("b", 2))) + rdd2 <- parallelize(sc, list(list("c", 3), list("d", 4))) + actual <- collectRDD(rightOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("d", list(NULL, 4)), list("c", list(NULL, 3))))) +}) + +test_that("fullOuterJoin() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3), list(3, 3))) + rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) + actual <- collectRDD(fullOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), + list(2, list(NULL, 4)), list(3, list(3, NULL))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a", 2), list("a", 3), list("c", 1))) + rdd2 <- parallelize(sc, list(list("a", 1), list("b", 4))) + actual <- collectRDD(fullOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), + list("a", list(3, 1)), list("c", list(1, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 2))) + rdd2 <- parallelize(sc, list(list(3, 3), list(4, 4))) + actual <- collectRDD(fullOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(1, list(1, NULL)), list(2, list(2, NULL)), + list(3, list(NULL, 3)), list(4, list(NULL, 4))))) + + rdd1 <- parallelize(sc, list(list("a", 1), list("b", 2))) + rdd2 <- parallelize(sc, list(list("c", 3), list("d", 4))) + actual <- collectRDD(fullOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("a", list(1, NULL)), list("b", list(2, NULL)), + list("d", list(NULL, 4)), list("c", list(NULL, 3))))) +}) + +test_that("sortByKey() on pairwise RDDs", { + numPairsRdd <- map(rdd, function(x) { list (x, x) }) + sortedRdd <- sortByKey(numPairsRdd, ascending = FALSE) + actual <- collectRDD(sortedRdd) + numPairs <- lapply(nums, function(x) { list (x, x) }) + expect_equal(actual, sortKeyValueList(numPairs, decreasing = TRUE)) + + rdd2 <- parallelize(sc, sort(nums, decreasing = TRUE), 2L) + numPairsRdd2 <- map(rdd2, function(x) { list (x, x) }) + sortedRdd2 <- sortByKey(numPairsRdd2) + actual <- collectRDD(sortedRdd2) + expect_equal(actual, numPairs) + + # sort by string keys + l <- list(list("a", 1), list("b", 2), list("1", 3), list("d", 4), list("2", 5)) + rdd3 <- parallelize(sc, l, 2L) + sortedRdd3 <- sortByKey(rdd3) + actual <- collectRDD(sortedRdd3) + expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) + + # test on the boundary cases + + # boundary case 1: the RDD to be sorted has only 1 partition + rdd4 <- parallelize(sc, l, 1L) + sortedRdd4 <- sortByKey(rdd4) + actual <- collectRDD(sortedRdd4) + expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) + + # boundary case 2: the sorted RDD has only 1 partition + rdd5 <- parallelize(sc, l, 2L) + sortedRdd5 <- sortByKey(rdd5, numPartitions = 1L) + actual <- collectRDD(sortedRdd5) + expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) + + # boundary case 3: the RDD to be sorted has only 1 element + l2 <- list(list("a", 1)) + rdd6 <- parallelize(sc, l2, 2L) + sortedRdd6 <- sortByKey(rdd6) + actual <- collectRDD(sortedRdd6) + expect_equal(actual, l2) + + # boundary case 4: the RDD to be sorted has 0 element + l3 <- list() + rdd7 <- parallelize(sc, l3, 2L) + sortedRdd7 <- sortByKey(rdd7) + actual <- collectRDD(sortedRdd7) + expect_equal(actual, l3) +}) + +test_that("collectAsMap() on a pairwise RDD", { + rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(`1` = 2, `3` = 4)) + + rdd <- parallelize(sc, list(list("a", 1), list("b", 2))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(a = 1, b = 2)) + + rdd <- parallelize(sc, list(list(1.1, 2.2), list(1.2, 2.4))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(`1.1` = 2.2, `1.2` = 2.4)) + + rdd <- parallelize(sc, list(list(1, "a"), list(2, "b"))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(`1` = "a", `2` = "b")) +}) + +test_that("show()", { + rdd <- parallelize(sc, list(1:10)) + expect_output(showRDD(rdd), "ParallelCollectionRDD\\[\\d+\\] at parallelize at RRDD\\.scala:\\d+") +}) + +test_that("sampleByKey() on pairwise RDDs", { + rdd <- parallelize(sc, 1:2000) + pairsRDD <- lapply(rdd, function(x) { if (x %% 2 == 0) list("a", x) else list("b", x) }) + fractions <- list(a = 0.2, b = 0.1) + sample <- sampleByKey(pairsRDD, FALSE, fractions, 1618L) + expect_equal(100 < length(lookup(sample, "a")) && 300 > length(lookup(sample, "a")), TRUE) + expect_equal(50 < length(lookup(sample, "b")) && 150 > length(lookup(sample, "b")), TRUE) + expect_equal(lookup(sample, "a")[which.min(lookup(sample, "a"))] >= 0, TRUE) + expect_equal(lookup(sample, "a")[which.max(lookup(sample, "a"))] <= 2000, TRUE) + expect_equal(lookup(sample, "b")[which.min(lookup(sample, "b"))] >= 0, TRUE) + expect_equal(lookup(sample, "b")[which.max(lookup(sample, "b"))] <= 2000, TRUE) + + rdd <- parallelize(sc, 1:2000) + pairsRDD <- lapply(rdd, function(x) { if (x %% 2 == 0) list(2, x) else list(3, x) }) + fractions <- list(`2` = 0.2, `3` = 0.1) + sample <- sampleByKey(pairsRDD, TRUE, fractions, 1618L) + expect_equal(100 < length(lookup(sample, 2)) && 300 > length(lookup(sample, 2)), TRUE) + expect_equal(50 < length(lookup(sample, 3)) && 150 > length(lookup(sample, 3)), TRUE) + expect_equal(lookup(sample, 2)[which.min(lookup(sample, 2))] >= 0, TRUE) + expect_equal(lookup(sample, 2)[which.max(lookup(sample, 2))] <= 2000, TRUE) + expect_equal(lookup(sample, 3)[which.min(lookup(sample, 3))] >= 0, TRUE) + expect_equal(lookup(sample, 3)[which.max(lookup(sample, 3))] <= 2000, TRUE) +}) + +test_that("Test correct concurrency of RRDD.compute()", { + rdd <- parallelize(sc, 1:1000, 100) + jrdd <- getJRDD(lapply(rdd, function(x) { x }), "row") + zrdd <- callJMethod(jrdd, "zip", jrdd) + count <- callJMethod(zrdd, "count") + expect_equal(count, 1000) +}) + +sparkR.session.stop() diff --git a/R/pkg/inst/tests/test_shuffle.R b/R/pkg/tests/fulltests/test_shuffle.R similarity index 88% rename from R/pkg/inst/tests/test_shuffle.R rename to R/pkg/tests/fulltests/test_shuffle.R index adf0b91d25fe..98300c67c415 100644 --- a/R/pkg/inst/tests/test_shuffle.R +++ b/R/pkg/tests/fulltests/test_shuffle.R @@ -18,7 +18,8 @@ context("partitionBy, groupByKey, reduceByKey etc.") # JavaSparkContext handle -sc <- sparkR.init() +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) # Data intPairs <- list(list(1L, -1), list(2L, 100), list(2L, 1), list(1L, 200)) @@ -38,7 +39,7 @@ strListRDD <- parallelize(sc, strList, 4) test_that("groupByKey for integers", { grouped <- groupByKey(intRdd, 2L) - actual <- collect(grouped) + actual <- collectRDD(grouped) expected <- list(list(2L, list(100, 1)), list(1L, list(-1, 200))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -47,7 +48,7 @@ test_that("groupByKey for integers", { test_that("groupByKey for doubles", { grouped <- groupByKey(doubleRdd, 2L) - actual <- collect(grouped) + actual <- collectRDD(grouped) expected <- list(list(1.5, list(-1, 200)), list(2.5, list(100, 1))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -56,7 +57,7 @@ test_that("groupByKey for doubles", { test_that("reduceByKey for ints", { reduced <- reduceByKey(intRdd, "+", 2L) - actual <- collect(reduced) + actual <- collectRDD(reduced) expected <- list(list(2L, 101), list(1L, 199)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -64,7 +65,7 @@ test_that("reduceByKey for ints", { test_that("reduceByKey for doubles", { reduced <- reduceByKey(doubleRdd, "+", 2L) - actual <- collect(reduced) + actual <- collectRDD(reduced) expected <- list(list(1.5, 199), list(2.5, 101)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -73,7 +74,7 @@ test_that("reduceByKey for doubles", { test_that("combineByKey for ints", { reduced <- combineByKey(intRdd, function(x) { x }, "+", "+", 2L) - actual <- collect(reduced) + actual <- collectRDD(reduced) expected <- list(list(2L, 101), list(1L, 199)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -81,7 +82,7 @@ test_that("combineByKey for ints", { test_that("combineByKey for doubles", { reduced <- combineByKey(doubleRdd, function(x) { x }, "+", "+", 2L) - actual <- collect(reduced) + actual <- collectRDD(reduced) expected <- list(list(1.5, 199), list(2.5, 101)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -93,7 +94,7 @@ test_that("combineByKey for characters", { list("other", 3L), list("max", 4L)), 2L) reduced <- combineByKey(stringKeyRDD, function(x) { x }, "+", "+", 2L) - actual <- collect(reduced) + actual <- collectRDD(reduced) expected <- list(list("max", 5L), list("min", 2L), list("other", 3L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -108,7 +109,7 @@ test_that("aggregateByKey", { combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } aggregatedRDD <- aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) - actual <- collect(aggregatedRDD) + actual <- collectRDD(aggregatedRDD) expected <- list(list(1, list(3, 2)), list(2, list(7, 2))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -121,7 +122,7 @@ test_that("aggregateByKey", { combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } aggregatedRDD <- aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) - actual <- collect(aggregatedRDD) + actual <- collectRDD(aggregatedRDD) expected <- list(list("a", list(3, 2)), list("b", list(7, 2))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -131,7 +132,7 @@ test_that("foldByKey", { # test foldByKey for int keys folded <- foldByKey(intRdd, 0, "+", 2L) - actual <- collect(folded) + actual <- collectRDD(folded) expected <- list(list(2L, 101), list(1L, 199)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -139,7 +140,7 @@ test_that("foldByKey", { # test foldByKey for double keys folded <- foldByKey(doubleRdd, 0, "+", 2L) - actual <- collect(folded) + actual <- collectRDD(folded) expected <- list(list(1.5, 199), list(2.5, 101)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -150,7 +151,7 @@ test_that("foldByKey", { stringKeyRDD <- parallelize(sc, stringKeyPairs) folded <- foldByKey(stringKeyRDD, 0, "+", 2L) - actual <- collect(folded) + actual <- collectRDD(folded) expected <- list(list("b", 101), list("a", 199)) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -158,14 +159,14 @@ test_that("foldByKey", { # test foldByKey for empty pair RDD rdd <- parallelize(sc, list()) folded <- foldByKey(rdd, 0, "+", 2L) - actual <- collect(folded) + actual <- collectRDD(folded) expected <- list() expect_equal(actual, expected) # test foldByKey for RDD with only 1 pair rdd <- parallelize(sc, list(list(1, 1))) folded <- foldByKey(rdd, 0, "+", 2L) - actual <- collect(folded) + actual <- collectRDD(folded) expected <- list(list(1, 1)) expect_equal(actual, expected) }) @@ -174,10 +175,10 @@ test_that("partitionBy() partitions data correctly", { # Partition by magnitude partitionByMagnitude <- function(key) { if (key >= 3) 1 else 0 } - resultRDD <- partitionBy(numPairsRdd, 2L, partitionByMagnitude) + resultRDD <- partitionByRDD(numPairsRdd, 2L, partitionByMagnitude) - expected_first <- list(list(1, 100), list(2, 200)) # key < 3 - expected_second <- list(list(4, -1), list(3, 1), list(3, 0)) # key >= 3 + expected_first <- list(list(1, 100), list(2, 200)) # key less than 3 + expected_second <- list(list(4, -1), list(3, 1), list(3, 0)) # key greater than or equal 3 actual_first <- collectPartition(resultRDD, 0L) actual_second <- collectPartition(resultRDD, 1L) @@ -190,7 +191,7 @@ test_that("partitionBy works with dependencies", { partitionByParity <- function(key) { if (key %% 2 == kOne) 7 else 4 } # Partition by parity - resultRDD <- partitionBy(numPairsRdd, numPartitions = 2L, partitionByParity) + resultRDD <- partitionByRDD(numPairsRdd, numPartitions = 2L, partitionByParity) # keys even; 100 %% 2 == 0 expected_first <- list(list(2, 200), list(4, -1)) @@ -207,7 +208,7 @@ test_that("test partitionBy with string keys", { words <- flatMap(strListRDD, function(line) { strsplit(line, " ")[[1]] }) wordCount <- lapply(words, function(word) { list(word, 1L) }) - resultRDD <- partitionBy(wordCount, 2L) + resultRDD <- partitionByRDD(wordCount, 2L) expected_first <- list(list("Dexter", 1), list("Dexter", 1)) expected_second <- list(list("and", 1), list("and", 1)) @@ -219,3 +220,5 @@ test_that("test partitionBy with string keys", { expect_equal(sortKeyValueList(actual_first), sortKeyValueList(expected_first)) expect_equal(sortKeyValueList(actual_second), sortKeyValueList(expected_second)) }) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_sparkR.R b/R/pkg/tests/fulltests/test_sparkR.R new file mode 100644 index 000000000000..f73fc6baecce --- /dev/null +++ b/R/pkg/tests/fulltests/test_sparkR.R @@ -0,0 +1,46 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("functions in sparkR.R") + +test_that("sparkCheckInstall", { + # "local, yarn-client, mesos-client" mode, SPARK_HOME was set correctly, + # and the SparkR job was submitted by "spark-submit" + sparkHome <- paste0(tempdir(), "/", "sparkHome") + dir.create(sparkHome) + master <- "" + deployMode <- "" + expect_true(is.null(sparkCheckInstall(sparkHome, master, deployMode))) + unlink(sparkHome, recursive = TRUE) + + # "yarn-cluster, mesos-cluster" mode, SPARK_HOME was not set, + # and the SparkR job was submitted by "spark-submit" + sparkHome <- "" + master <- "" + deployMode <- "" + expect_true(is.null(sparkCheckInstall(sparkHome, master, deployMode))) + + # "yarn-client, mesos-client" mode, SPARK_HOME was not set + sparkHome <- "" + master <- "yarn-client" + deployMode <- "" + expect_error(sparkCheckInstall(sparkHome, master, deployMode)) + sparkHome <- "" + master <- "" + deployMode <- "client" + expect_error(sparkCheckInstall(sparkHome, master, deployMode)) +}) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R new file mode 100644 index 000000000000..4e62be9b4d61 --- /dev/null +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -0,0 +1,3476 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(testthat) + +context("SparkSQL functions") + +# Utility function for easily checking the values of a StructField +checkStructField <- function(actual, expectedName, expectedType, expectedNullable) { + expect_equal(class(actual), "structField") + expect_equal(actual$name(), expectedName) + expect_equal(actual$dataType.toString(), expectedType) + expect_equal(actual$nullable(), expectedNullable) +} + +markUtf8 <- function(s) { + Encoding(s) <- "UTF-8" + s +} + +setHiveContext <- function(sc) { + if (exists(".testHiveSession", envir = .sparkREnv)) { + hiveSession <- get(".testHiveSession", envir = .sparkREnv) + } else { + # initialize once and reuse + ssc <- callJMethod(sc, "sc") + hiveCtx <- tryCatch({ + newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc, FALSE) + }, + error = function(err) { + skip("Hive is not build with SparkSQL, skipped") + }) + hiveSession <- callJMethod(hiveCtx, "sparkSession") + } + previousSession <- get(".sparkRsession", envir = .sparkREnv) + assign(".sparkRsession", hiveSession, envir = .sparkREnv) + assign(".prevSparkRsession", previousSession, envir = .sparkREnv) + hiveSession +} + +unsetHiveContext <- function() { + previousSession <- get(".prevSparkRsession", envir = .sparkREnv) + assign(".sparkRsession", previousSession, envir = .sparkREnv) + remove(".prevSparkRsession", envir = .sparkREnv) +} + +# Tests for SparkSQL functions in SparkR + +filesBefore <- list.files(path = sparkRDir, all.files = TRUE) +sparkSession <- if (windows_with_hadoop()) { + sparkR.session(master = sparkRTestMaster) + } else { + sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + } +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) + +mockLines <- c("{\"name\":\"Michael\"}", + "{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"Justin\", \"age\":19}") +jsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") +parquetPath <- tempfile(pattern = "sparkr-test", fileext = ".parquet") +orcPath <- tempfile(pattern = "sparkr-test", fileext = ".orc") +writeLines(mockLines, jsonPath) + +# For test nafunctions, like dropna(), fillna(),... +mockLinesNa <- c("{\"name\":\"Bob\",\"age\":16,\"height\":176.5}", + "{\"name\":\"Alice\",\"age\":null,\"height\":164.3}", + "{\"name\":\"David\",\"age\":60,\"height\":null}", + "{\"name\":\"Amy\",\"age\":null,\"height\":null}", + "{\"name\":null,\"age\":null,\"height\":null}") +jsonPathNa <- tempfile(pattern = "sparkr-test", fileext = ".tmp") +writeLines(mockLinesNa, jsonPathNa) + +# For test complex types in DataFrame +mockLinesComplexType <- + c("{\"c1\":[1, 2, 3], \"c2\":[\"a\", \"b\", \"c\"], \"c3\":[1.0, 2.0, 3.0]}", + "{\"c1\":[4, 5, 6], \"c2\":[\"d\", \"e\", \"f\"], \"c3\":[4.0, 5.0, 6.0]}", + "{\"c1\":[7, 8, 9], \"c2\":[\"g\", \"h\", \"i\"], \"c3\":[7.0, 8.0, 9.0]}") +complexTypeJsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") +writeLines(mockLinesComplexType, complexTypeJsonPath) + +# For test map type and struct type in DataFrame +mockLinesMapType <- c("{\"name\":\"Bob\",\"info\":{\"age\":16,\"height\":176.5}}", + "{\"name\":\"Alice\",\"info\":{\"age\":20,\"height\":164.3}}", + "{\"name\":\"David\",\"info\":{\"age\":60,\"height\":180}}") +mapTypeJsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") +writeLines(mockLinesMapType, mapTypeJsonPath) + +if (is_windows()) { + Sys.setenv(TZ = "GMT") +} + +test_that("calling sparkRSQL.init returns existing SQL context", { + sqlContext <- suppressWarnings(sparkRSQL.init(sc)) + expect_equal(suppressWarnings(sparkRSQL.init(sc)), sqlContext) +}) + +test_that("calling sparkRSQL.init returns existing SparkSession", { + expect_equal(suppressWarnings(sparkRSQL.init(sc)), sparkSession) +}) + +test_that("calling sparkR.session returns existing SparkSession", { + expect_equal(sparkR.session(), sparkSession) +}) + +test_that("infer types and check types", { + expect_equal(infer_type(1L), "integer") + expect_equal(infer_type(1.0), "double") + expect_equal(infer_type("abc"), "string") + expect_equal(infer_type(TRUE), "boolean") + expect_equal(infer_type(as.Date("2015-03-11")), "date") + expect_equal(infer_type(as.POSIXlt("2015-03-11 12:13:04.043")), "timestamp") + expect_equal(infer_type(c(1L, 2L)), "array") + expect_equal(infer_type(list(1L, 2L)), "array") + expect_equal(infer_type(listToStruct(list(a = 1L, b = "2"))), "struct") + e <- new.env() + assign("a", 1L, envir = e) + expect_equal(infer_type(e), "map") + + expect_error(checkType("map"), "Key type in a map must be string or character") + + expect_equal(infer_type(as.raw(c(1, 2, 3))), "binary") +}) + +test_that("structType and structField", { + testField <- structField("a", "string") + expect_is(testField, "structField") + expect_equal(testField$name(), "a") + expect_true(testField$nullable()) + + testSchema <- structType(testField, structField("b", "integer")) + expect_is(testSchema, "structType") + expect_is(testSchema$fields()[[2]], "structField") + expect_equal(testSchema$fields()[[1]]$dataType.toString(), "StringType") + + testSchema <- structType("a STRING, b INT") + expect_is(testSchema, "structType") + expect_is(testSchema$fields()[[2]], "structField") + expect_equal(testSchema$fields()[[1]]$dataType.toString(), "StringType") + + expect_error(structType("A stri"), "DataType stri is not supported.") +}) + +test_that("structField type strings", { + # positive cases + primitiveTypes <- list(byte = "ByteType", + integer = "IntegerType", + float = "FloatType", + double = "DoubleType", + string = "StringType", + binary = "BinaryType", + boolean = "BooleanType", + timestamp = "TimestampType", + date = "DateType", + tinyint = "ByteType", + smallint = "ShortType", + int = "IntegerType", + bigint = "LongType", + decimal = "DecimalType(10,0)") + + complexTypes <- list("map" = "MapType(StringType,IntegerType,true)", + "array" = "ArrayType(StringType,true)", + "struct" = "StructType(StructField(a,StringType,true))") + + typeList <- c(primitiveTypes, complexTypes) + typeStrings <- names(typeList) + + for (i in seq_along(typeStrings)){ + typeString <- typeStrings[i] + expected <- typeList[[i]] + testField <- structField("_col", typeString) + expect_is(testField, "structField") + expect_true(testField$nullable()) + expect_equal(testField$dataType.toString(), expected) + } + + # negative cases + primitiveErrors <- list(Byte = "Byte", + INTEGER = "INTEGER", + numeric = "numeric", + character = "character", + raw = "raw", + logical = "logical", + short = "short", + varchar = "varchar", + long = "long", + char = "char") + + complexErrors <- list("map" = " integer", + "array" = "String", + "struct" = "string ", + "map " = "map ", + "array< string>" = " string", + "struct" = " string") + + errorList <- c(primitiveErrors, complexErrors) + typeStrings <- names(errorList) + + for (i in seq_along(typeStrings)){ + typeString <- typeStrings[i] + expected <- paste0("Unsupported type for SparkDataframe: ", errorList[[i]]) + expect_error(structField("_col", typeString), expected) + } +}) + +test_that("create DataFrame from RDD", { + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) + df <- createDataFrame(rdd, list("a", "b")) + dfAsDF <- as.DataFrame(rdd, list("a", "b")) + expect_is(df, "SparkDataFrame") + expect_is(dfAsDF, "SparkDataFrame") + expect_equal(count(df), 10) + expect_equal(count(dfAsDF), 10) + expect_equal(nrow(df), 10) + expect_equal(nrow(dfAsDF), 10) + expect_equal(ncol(df), 2) + expect_equal(ncol(dfAsDF), 2) + expect_equal(dim(df), c(10, 2)) + expect_equal(dim(dfAsDF), c(10, 2)) + expect_equal(columns(df), c("a", "b")) + expect_equal(columns(dfAsDF), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + expect_equal(dtypes(dfAsDF), list(c("a", "int"), c("b", "string"))) + + df <- createDataFrame(rdd) + dfAsDF <- as.DataFrame(rdd) + expect_is(df, "SparkDataFrame") + expect_is(dfAsDF, "SparkDataFrame") + expect_equal(columns(df), c("_1", "_2")) + expect_equal(columns(dfAsDF), c("_1", "_2")) + + schema <- structType(structField(x = "a", type = "integer", nullable = TRUE), + structField(x = "b", type = "string", nullable = TRUE)) + df <- createDataFrame(rdd, schema) + expect_is(df, "SparkDataFrame") + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(a = x, b = as.character(x)) }) + df <- createDataFrame(rdd) + expect_is(df, "SparkDataFrame") + expect_equal(count(df), 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + schema <- structType(structField("name", "string"), structField("age", "integer"), + structField("height", "float")) + df <- read.df(jsonPathNa, "json", schema) + df2 <- createDataFrame(toRDD(df), schema) + df2AsDF <- as.DataFrame(toRDD(df), schema) + expect_equal(columns(df2), c("name", "age", "height")) + expect_equal(columns(df2AsDF), c("name", "age", "height")) + expect_equal(dtypes(df2), list(c("name", "string"), c("age", "int"), c("height", "float"))) + expect_equal(dtypes(df2AsDF), list(c("name", "string"), c("age", "int"), c("height", "float"))) + expect_equal(as.list(collect(where(df2, df2$name == "Bob"))), + list(name = "Bob", age = 16, height = 176.5)) + expect_equal(as.list(collect(where(df2AsDF, df2AsDF$name == "Bob"))), + list(name = "Bob", age = 16, height = 176.5)) + + localDF <- data.frame(name = c("John", "Smith", "Sarah"), + age = c(19L, 23L, 18L), + height = c(176.5, 181.4, 173.7)) + df <- createDataFrame(localDF, schema) + expect_is(df, "SparkDataFrame") + expect_equal(count(df), 3) + expect_equal(columns(df), c("name", "age", "height")) + expect_equal(dtypes(df), list(c("name", "string"), c("age", "int"), c("height", "float"))) + expect_equal(as.list(collect(where(df, df$name == "John"))), + list(name = "John", age = 19L, height = 176.5)) + expect_equal(getNumPartitions(df), 1) + + df <- as.DataFrame(cars, numPartitions = 2) + expect_equal(getNumPartitions(df), 2) + df <- createDataFrame(cars, numPartitions = 3) + expect_equal(getNumPartitions(df), 3) + # validate limit by num of rows + df <- createDataFrame(cars, numPartitions = 60) + expect_equal(getNumPartitions(df), 50) + # validate when 1 < (length(coll) / numSlices) << length(coll) + df <- createDataFrame(cars, numPartitions = 20) + expect_equal(getNumPartitions(df), 20) + + df <- as.DataFrame(data.frame(0)) + expect_is(df, "SparkDataFrame") + df <- createDataFrame(list(list(1))) + expect_is(df, "SparkDataFrame") + df <- as.DataFrame(data.frame(0), numPartitions = 2) + # no data to partition, goes to 1 + expect_equal(getNumPartitions(df), 1) + + setHiveContext(sc) + sql("CREATE TABLE people (name string, age double, height float)") + df <- read.df(jsonPathNa, "json", schema) + insertInto(df, "people") + expect_equal(collect(sql("SELECT age from people WHERE name = 'Bob'"))$age, + c(16)) + expect_equal(collect(sql("SELECT height from people WHERE name ='Bob'"))$height, + c(176.5)) + sql("DROP TABLE people") + unsetHiveContext() +}) + +test_that("createDataFrame uses files for large objects", { + # To simulate a large file scenario, we set spark.r.maxAllocationLimit to a smaller value + conf <- callJMethod(sparkSession, "conf") + callJMethod(conf, "set", "spark.r.maxAllocationLimit", "100") + df <- suppressWarnings(createDataFrame(iris, numPartitions = 3)) + expect_equal(getNumPartitions(df), 3) + + # Resetting the conf back to default value + callJMethod(conf, "set", "spark.r.maxAllocationLimit", toString(.Machine$integer.max / 10)) + expect_equal(dim(df), dim(iris)) +}) + +test_that("read/write csv as DataFrame", { + if (windows_with_hadoop()) { + csvPath <- tempfile(pattern = "sparkr-test", fileext = ".csv") + mockLinesCsv <- c("year,make,model,comment,blank", + "\"2012\",\"Tesla\",\"S\",\"No comment\",", + "1997,Ford,E350,\"Go get one now they are going fast\",", + "2015,Chevy,Volt", + "NA,Dummy,Placeholder") + writeLines(mockLinesCsv, csvPath) + + # default "header" is false, inferSchema to handle "year" as "int" + df <- read.df(csvPath, "csv", header = "true", inferSchema = "true") + expect_equal(count(df), 4) + expect_equal(columns(df), c("year", "make", "model", "comment", "blank")) + expect_equal(sort(unlist(collect(where(df, df$year == 2015)))), + sort(unlist(list(year = 2015, make = "Chevy", model = "Volt")))) + + # since "year" is "int", let's skip the NA values + withoutna <- na.omit(df, how = "any", cols = "year") + expect_equal(count(withoutna), 3) + + unlink(csvPath) + csvPath <- tempfile(pattern = "sparkr-test", fileext = ".csv") + mockLinesCsv <- c("year,make,model,comment,blank", + "\"2012\",\"Tesla\",\"S\",\"No comment\",", + "1997,Ford,E350,\"Go get one now they are going fast\",", + "2015,Chevy,Volt", + "Empty,Dummy,Placeholder") + writeLines(mockLinesCsv, csvPath) + + df2 <- read.df(csvPath, "csv", header = "true", inferSchema = "true", na.strings = "Empty") + expect_equal(count(df2), 4) + withoutna2 <- na.omit(df2, how = "any", cols = "year") + expect_equal(count(withoutna2), 3) + expect_equal(count(where(withoutna2, withoutna2$make == "Dummy")), 0) + + # writing csv file + csvPath2 <- tempfile(pattern = "csvtest2", fileext = ".csv") + write.df(df2, path = csvPath2, "csv", header = "true") + df3 <- read.df(csvPath2, "csv", header = "true") + expect_equal(nrow(df3), nrow(df2)) + expect_equal(colnames(df3), colnames(df2)) + csv <- read.csv(file = list.files(csvPath2, pattern = "^part", full.names = T)[[1]]) + expect_equal(colnames(df3), colnames(csv)) + + unlink(csvPath) + unlink(csvPath2) + } +}) + +test_that("Support other types for options", { + csvPath <- tempfile(pattern = "sparkr-test", fileext = ".csv") + mockLinesCsv <- c("year,make,model,comment,blank", + "\"2012\",\"Tesla\",\"S\",\"No comment\",", + "1997,Ford,E350,\"Go get one now they are going fast\",", + "2015,Chevy,Volt", + "NA,Dummy,Placeholder") + writeLines(mockLinesCsv, csvPath) + + csvDf <- read.df(csvPath, "csv", header = "true", inferSchema = "true") + expected <- read.df(csvPath, "csv", header = TRUE, inferSchema = TRUE) + expect_equal(collect(csvDf), collect(expected)) + + expect_error(read.df(csvPath, "csv", header = TRUE, maxColumns = 3)) + unlink(csvPath) +}) + +test_that("convert NAs to null type in DataFrames", { + rdd <- parallelize(sc, list(list(1L, 2L), list(NA, 4L))) + df <- createDataFrame(rdd, list("a", "b")) + expect_true(is.na(collect(df)[2, "a"])) + expect_equal(collect(df)[2, "b"], 4L) + + l <- data.frame(x = 1L, y = c(1L, NA_integer_, 3L)) + df <- createDataFrame(l) + expect_equal(collect(df)[2, "x"], 1L) + expect_true(is.na(collect(df)[2, "y"])) + + rdd <- parallelize(sc, list(list(1, 2), list(NA, 4))) + df <- createDataFrame(rdd, list("a", "b")) + expect_true(is.na(collect(df)[2, "a"])) + expect_equal(collect(df)[2, "b"], 4) + + l <- data.frame(x = 1, y = c(1, NA_real_, 3)) + df <- createDataFrame(l) + expect_equal(collect(df)[2, "x"], 1) + expect_true(is.na(collect(df)[2, "y"])) + + l <- list("a", "b", NA, "d") + df <- createDataFrame(l) + expect_true(is.na(collect(df)[3, "_1"])) + expect_equal(collect(df)[4, "_1"], "d") + + l <- list("a", "b", NA_character_, "d") + df <- createDataFrame(l) + expect_true(is.na(collect(df)[3, "_1"])) + expect_equal(collect(df)[4, "_1"], "d") + + l <- list(TRUE, FALSE, NA, TRUE) + df <- createDataFrame(l) + expect_true(is.na(collect(df)[3, "_1"])) + expect_equal(collect(df)[4, "_1"], TRUE) +}) + +test_that("toDF", { + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) + df <- toDF(rdd, list("a", "b")) + expect_is(df, "SparkDataFrame") + expect_equal(count(df), 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + df <- toDF(rdd) + expect_is(df, "SparkDataFrame") + expect_equal(columns(df), c("_1", "_2")) + + schema <- structType(structField(x = "a", type = "integer", nullable = TRUE), + structField(x = "b", type = "string", nullable = TRUE)) + df <- toDF(rdd, schema) + expect_is(df, "SparkDataFrame") + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(a = x, b = as.character(x)) }) + df <- toDF(rdd) + expect_is(df, "SparkDataFrame") + expect_equal(count(df), 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) +}) + +test_that("create DataFrame from list or data.frame", { + l <- list(list(1, 2), list(3, 4)) + df <- createDataFrame(l, c("a", "b")) + expect_equal(columns(df), c("a", "b")) + + l <- list(list(a = 1, b = 2), list(a = 3, b = 4)) + df <- createDataFrame(l) + expect_equal(columns(df), c("a", "b")) + + a <- 1:3 + b <- c("a", "b", "c") + ldf <- data.frame(a, b) + df <- createDataFrame(ldf) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + expect_equal(count(df), 3) + ldf2 <- collect(df) + expect_equal(ldf$a, ldf2$a) + + irisdf <- suppressWarnings(createDataFrame(iris)) + iris_collected <- collect(irisdf) + expect_equivalent(iris_collected[, -5], iris[, -5]) + expect_equal(iris_collected$Species, as.character(iris$Species)) + + mtcarsdf <- createDataFrame(mtcars) + expect_equivalent(collect(mtcarsdf), mtcars) + + bytes <- as.raw(c(1, 2, 3)) + df <- createDataFrame(list(list(bytes))) + expect_equal(collect(df)[[1]][[1]], bytes) +}) + +test_that("create DataFrame with different data types", { + l <- list(a = 1L, b = 2, c = TRUE, d = "ss", e = as.Date("2012-12-13"), + f = as.POSIXct("2015-03-15 12:13:14.056")) + df <- createDataFrame(list(l)) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "double"), c("c", "boolean"), + c("d", "string"), c("e", "date"), c("f", "timestamp"))) + expect_equal(count(df), 1) + expect_equal(collect(df), data.frame(l, stringsAsFactors = FALSE)) +}) + +test_that("SPARK-17811: can create DataFrame containing NA as date and time", { + df <- data.frame( + id = 1:2, + time = c(as.POSIXlt("2016-01-10"), NA), + date = c(as.Date("2016-10-01"), NA)) + + DF <- collect(createDataFrame(df)) + expect_true(is.na(DF$date[2])) + expect_equal(DF$date[1], as.Date("2016-10-01")) + expect_true(is.na(DF$time[2])) + expect_equal(DF$time[1], as.POSIXlt("2016-01-10")) +}) + +test_that("create DataFrame with complex types", { + e <- new.env() + assign("n", 3L, envir = e) + + s <- listToStruct(list(a = "aa", b = 3L)) + + l <- list(as.list(1:10), list("a", "b"), e, s) + df <- createDataFrame(list(l), c("a", "b", "c", "d")) + expect_equal(dtypes(df), list(c("a", "array"), + c("b", "array"), + c("c", "map"), + c("d", "struct"))) + expect_equal(count(df), 1) + ldf <- collect(df) + expect_equal(names(ldf), c("a", "b", "c", "d")) + expect_equal(ldf[1, 1][[1]], l[[1]]) + expect_equal(ldf[1, 2][[1]], l[[2]]) + + e <- ldf$c[[1]] + expect_equal(class(e), "environment") + expect_equal(ls(e), "n") + expect_equal(e$n, 3L) + + s <- ldf$d[[1]] + expect_equal(class(s), "struct") + expect_equal(s$a, "aa") + expect_equal(s$b, 3L) +}) + +test_that("create DataFrame from a data.frame with complex types", { + ldf <- data.frame(row.names = 1:2) + ldf$a_list <- list(list(1, 2), list(3, 4)) + ldf$an_envir <- c(as.environment(list(a = 1, b = 2)), as.environment(list(c = 3))) + + sdf <- createDataFrame(ldf) + collected <- collect(sdf) + + expect_identical(ldf[, 1, FALSE], collected[, 1, FALSE]) + expect_equal(ldf$an_envir, collected$an_envir) +}) + +test_that("Collect DataFrame with complex types", { + # ArrayType + df <- read.json(complexTypeJsonPath) + ldf <- collect(df) + expect_equal(nrow(ldf), 3) + expect_equal(ncol(ldf), 3) + expect_equal(names(ldf), c("c1", "c2", "c3")) + expect_equal(ldf$c1, list(list(1, 2, 3), list(4, 5, 6), list (7, 8, 9))) + expect_equal(ldf$c2, list(list("a", "b", "c"), list("d", "e", "f"), list ("g", "h", "i"))) + expect_equal(ldf$c3, list(list(1.0, 2.0, 3.0), list(4.0, 5.0, 6.0), list (7.0, 8.0, 9.0))) + + # MapType + schema <- structType(structField("name", "string"), + structField("info", "map")) + df <- read.df(mapTypeJsonPath, "json", schema) + expect_equal(dtypes(df), list(c("name", "string"), + c("info", "map"))) + ldf <- collect(df) + expect_equal(nrow(ldf), 3) + expect_equal(ncol(ldf), 2) + expect_equal(names(ldf), c("name", "info")) + expect_equal(ldf$name, c("Bob", "Alice", "David")) + bob <- ldf$info[[1]] + expect_equal(class(bob), "environment") + expect_equal(bob$age, 16) + expect_equal(bob$height, 176.5) + + # StructType + df <- read.json(mapTypeJsonPath) + expect_equal(dtypes(df), list(c("info", "struct"), + c("name", "string"))) + ldf <- collect(df) + expect_equal(nrow(ldf), 3) + expect_equal(ncol(ldf), 2) + expect_equal(names(ldf), c("info", "name")) + expect_equal(ldf$name, c("Bob", "Alice", "David")) + bob <- ldf$info[[1]] + expect_equal(class(bob), "struct") + expect_equal(bob$age, 16) + expect_equal(bob$height, 176.5) +}) + +test_that("read/write json files", { + if (windows_with_hadoop()) { + # Test read.df + df <- read.df(jsonPath, "json") + expect_is(df, "SparkDataFrame") + expect_equal(count(df), 3) + + # Test read.df with a user defined schema + schema <- structType(structField("name", type = "string"), + structField("age", type = "double")) + + df1 <- read.df(jsonPath, "json", schema) + expect_is(df1, "SparkDataFrame") + expect_equal(dtypes(df1), list(c("name", "string"), c("age", "double"))) + + # Test loadDF + df2 <- loadDF(jsonPath, "json", schema) + expect_is(df2, "SparkDataFrame") + expect_equal(dtypes(df2), list(c("name", "string"), c("age", "double"))) + + # Test read.json + df <- read.json(jsonPath) + expect_is(df, "SparkDataFrame") + expect_equal(count(df), 3) + + # Test write.df + jsonPath2 <- tempfile(pattern = "jsonPath2", fileext = ".json") + write.df(df, jsonPath2, "json", mode = "overwrite") + + # Test write.json + jsonPath3 <- tempfile(pattern = "jsonPath3", fileext = ".json") + write.json(df, jsonPath3) + + # Test read.json()/jsonFile() works with multiple input paths + jsonDF1 <- read.json(c(jsonPath2, jsonPath3)) + expect_is(jsonDF1, "SparkDataFrame") + expect_equal(count(jsonDF1), 6) + # Suppress warnings because jsonFile is deprecated + jsonDF2 <- suppressWarnings(jsonFile(c(jsonPath2, jsonPath3))) + expect_is(jsonDF2, "SparkDataFrame") + expect_equal(count(jsonDF2), 6) + + unlink(jsonPath2) + unlink(jsonPath3) + } +}) + +test_that("read/write json files - compression option", { + df <- read.df(jsonPath, "json") + + jsonPath <- tempfile(pattern = "jsonPath", fileext = ".json") + write.json(df, jsonPath, compression = "gzip") + jsonDF <- read.json(jsonPath) + expect_is(jsonDF, "SparkDataFrame") + expect_equal(count(jsonDF), count(df)) + expect_true(length(list.files(jsonPath, pattern = ".gz")) > 0) + + unlink(jsonPath) +}) + +test_that("jsonRDD() on a RDD with json string", { + sqlContext <- suppressWarnings(sparkRSQL.init(sc)) + rdd <- parallelize(sc, mockLines) + expect_equal(countRDD(rdd), 3) + df <- suppressWarnings(jsonRDD(sqlContext, rdd)) + expect_is(df, "SparkDataFrame") + expect_equal(count(df), 3) + + rdd2 <- flatMap(rdd, function(x) c(x, x)) + df <- suppressWarnings(jsonRDD(sqlContext, rdd2)) + expect_is(df, "SparkDataFrame") + expect_equal(count(df), 6) +}) + +test_that("test tableNames and tables", { + count <- count(listTables()) + + df <- read.json(jsonPath) + createOrReplaceTempView(df, "table1") + expect_equal(length(tableNames()), count + 1) + expect_equal(length(tableNames("default")), count + 1) + + tables <- listTables() + expect_equal(count(tables), count + 1) + expect_equal(count(tables()), count(tables)) + expect_true("tableName" %in% colnames(tables())) + expect_true(all(c("tableName", "database", "isTemporary") %in% colnames(tables()))) + + suppressWarnings(registerTempTable(df, "table2")) + tables <- listTables() + expect_equal(count(tables), count + 2) + suppressWarnings(dropTempTable("table1")) + expect_true(dropTempView("table2")) + + tables <- listTables() + expect_equal(count(tables), count + 0) +}) + +test_that( + "createOrReplaceTempView() results in a queryable table and sql() results in a new DataFrame", { + df <- read.json(jsonPath) + createOrReplaceTempView(df, "table1") + newdf <- sql("SELECT * FROM table1 where name = 'Michael'") + expect_is(newdf, "SparkDataFrame") + expect_equal(count(newdf), 1) + expect_true(dropTempView("table1")) + + createOrReplaceTempView(df, "dfView") + sqlCast <- collect(sql("select cast('2' as decimal) as x from dfView limit 1")) + out <- capture.output(sqlCast) + expect_true(is.data.frame(sqlCast)) + expect_equal(names(sqlCast)[1], "x") + expect_equal(nrow(sqlCast), 1) + expect_equal(ncol(sqlCast), 1) + expect_equal(out[1], " x") + expect_equal(out[2], "1 2") + expect_true(dropTempView("dfView")) +}) + +test_that("test cache, uncache and clearCache", { + df <- read.json(jsonPath) + createOrReplaceTempView(df, "table1") + cacheTable("table1") + uncacheTable("table1") + clearCache() + expect_true(dropTempView("table1")) + + expect_error(uncacheTable("foo"), + "Error in uncacheTable : no such table - Table or view 'foo' not found in database 'default'") +}) + +test_that("insertInto() on a registered table", { + if (windows_with_hadoop()) { + df <- read.df(jsonPath, "json") + write.df(df, parquetPath, "parquet", "overwrite") + dfParquet <- read.df(parquetPath, "parquet") + + lines <- c("{\"name\":\"Bob\", \"age\":24}", + "{\"name\":\"James\", \"age\":35}") + jsonPath2 <- tempfile(pattern = "jsonPath2", fileext = ".tmp") + parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet") + writeLines(lines, jsonPath2) + df2 <- read.df(jsonPath2, "json") + write.df(df2, parquetPath2, "parquet", "overwrite") + dfParquet2 <- read.df(parquetPath2, "parquet") + + createOrReplaceTempView(dfParquet, "table1") + insertInto(dfParquet2, "table1") + expect_equal(count(sql("select * from table1")), 5) + expect_equal(first(sql("select * from table1 order by age"))$name, "Michael") + expect_true(dropTempView("table1")) + + createOrReplaceTempView(dfParquet, "table1") + insertInto(dfParquet2, "table1", overwrite = TRUE) + expect_equal(count(sql("select * from table1")), 2) + expect_equal(first(sql("select * from table1 order by age"))$name, "Bob") + expect_true(dropTempView("table1")) + + unlink(jsonPath2) + unlink(parquetPath2) + } +}) + +test_that("tableToDF() returns a new DataFrame", { + df <- read.json(jsonPath) + createOrReplaceTempView(df, "table1") + tabledf <- tableToDF("table1") + expect_is(tabledf, "SparkDataFrame") + expect_equal(count(tabledf), 3) + tabledf2 <- tableToDF("table1") + expect_equal(count(tabledf2), 3) + expect_true(dropTempView("table1")) +}) + +test_that("toRDD() returns an RRDD", { + df <- read.json(jsonPath) + testRDD <- toRDD(df) + expect_is(testRDD, "RDD") + expect_equal(countRDD(testRDD), 3) +}) + +test_that("union on two RDDs created from DataFrames returns an RRDD", { + df <- read.json(jsonPath) + RDD1 <- toRDD(df) + RDD2 <- toRDD(df) + unioned <- unionRDD(RDD1, RDD2) + expect_is(unioned, "RDD") + expect_equal(getSerializedMode(unioned), "byte") + expect_equal(collectRDD(unioned)[[2]]$name, "Andy") +}) + +test_that("union on mixed serialization types correctly returns a byte RRDD", { + # Byte RDD + nums <- 1:10 + rdd <- parallelize(sc, nums, 2L) + + # String RDD + textLines <- c("Michael", + "Andy, 30", + "Justin, 19") + textPath <- tempfile(pattern = "sparkr-textLines", fileext = ".tmp") + writeLines(textLines, textPath) + textRDD <- textFile(sc, textPath) + + df <- read.json(jsonPath) + dfRDD <- toRDD(df) + + unionByte <- unionRDD(rdd, dfRDD) + expect_is(unionByte, "RDD") + expect_equal(getSerializedMode(unionByte), "byte") + expect_equal(collectRDD(unionByte)[[1]], 1) + expect_equal(collectRDD(unionByte)[[12]]$name, "Andy") + + unionString <- unionRDD(textRDD, dfRDD) + expect_is(unionString, "RDD") + expect_equal(getSerializedMode(unionString), "byte") + expect_equal(collectRDD(unionString)[[1]], "Michael") + expect_equal(collectRDD(unionString)[[5]]$name, "Andy") +}) + +test_that("objectFile() works with row serialization", { + objectPath <- tempfile(pattern = "spark-test", fileext = ".tmp") + df <- read.json(jsonPath) + dfRDD <- toRDD(df) + saveAsObjectFile(coalesceRDD(dfRDD, 1L), objectPath) + objectIn <- objectFile(sc, objectPath) + + expect_is(objectIn, "RDD") + expect_equal(getSerializedMode(objectIn), "byte") + expect_equal(collectRDD(objectIn)[[2]]$age, 30) +}) + +test_that("lapply() on a DataFrame returns an RDD with the correct columns", { + df <- read.json(jsonPath) + testRDD <- lapply(df, function(row) { + row$newCol <- row$age + 5 + row + }) + expect_is(testRDD, "RDD") + collected <- collectRDD(testRDD) + expect_equal(collected[[1]]$name, "Michael") + expect_equal(collected[[2]]$newCol, 35) +}) + +test_that("collect() returns a data.frame", { + df <- read.json(jsonPath) + rdf <- collect(df) + expect_true(is.data.frame(rdf)) + expect_equal(names(rdf)[1], "age") + expect_equal(nrow(rdf), 3) + expect_equal(ncol(rdf), 2) + + # collect() returns data correctly from a DataFrame with 0 row + df0 <- limit(df, 0) + rdf <- collect(df0) + expect_true(is.data.frame(rdf)) + expect_equal(names(rdf)[1], "age") + expect_equal(nrow(rdf), 0) + expect_equal(ncol(rdf), 2) + + # collect() correctly handles multiple columns with same name + df <- createDataFrame(list(list(1, 2)), schema = c("name", "name")) + ldf <- collect(df) + expect_equal(names(ldf), c("name", "name")) +}) + +test_that("limit() returns DataFrame with the correct number of rows", { + df <- read.json(jsonPath) + dfLimited <- limit(df, 2) + expect_is(dfLimited, "SparkDataFrame") + expect_equal(count(dfLimited), 2) +}) + +test_that("collect() and take() on a DataFrame return the same number of rows and columns", { + df <- read.json(jsonPath) + expect_equal(nrow(collect(df)), nrow(take(df, 10))) + expect_equal(ncol(collect(df)), ncol(take(df, 10))) +}) + +test_that("collect() support Unicode characters", { + lines <- c("{\"name\":\"안녕하세요\"}", + "{\"name\":\"您好\", \"age\":30}", + "{\"name\":\"こんにちは\", \"age\":19}", + "{\"name\":\"Xin chào\"}") + + jsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") + writeLines(lines, jsonPath) + + df <- read.df(jsonPath, "json") + rdf <- collect(df) + expect_true(is.data.frame(rdf)) + expect_equal(rdf$name[1], markUtf8("안녕하세요")) + expect_equal(rdf$name[2], markUtf8("您好")) + expect_equal(rdf$name[3], markUtf8("こんにちは")) + expect_equal(rdf$name[4], markUtf8("Xin chào")) + + df1 <- createDataFrame(rdf) + expect_equal(collect(where(df1, df1$name == markUtf8("您好")))$name, markUtf8("您好")) +}) + +test_that("multiple pipeline transformations result in an RDD with the correct values", { + df <- read.json(jsonPath) + first <- lapply(df, function(row) { + row$age <- row$age + 5 + row + }) + second <- lapply(first, function(row) { + row$testCol <- if (row$age == 35 && !is.na(row$age)) TRUE else FALSE + row + }) + expect_is(second, "RDD") + expect_equal(countRDD(second), 3) + expect_equal(collectRDD(second)[[2]]$age, 35) + expect_true(collectRDD(second)[[2]]$testCol) + expect_false(collectRDD(second)[[3]]$testCol) +}) + +test_that("cache(), storageLevel(), persist(), and unpersist() on a DataFrame", { + df <- read.json(jsonPath) + expect_false(df@env$isCached) + cache(df) + expect_true(df@env$isCached) + + unpersist(df) + expect_false(df@env$isCached) + + persist(df, "MEMORY_AND_DISK") + expect_true(df@env$isCached) + + expect_equal(storageLevel(df), + "MEMORY_AND_DISK - StorageLevel(disk, memory, deserialized, 1 replicas)") + + unpersist(df) + expect_false(df@env$isCached) + + # make sure the data is collectable + expect_true(is.data.frame(collect(df))) +}) + +test_that("setCheckpointDir(), checkpoint() on a DataFrame", { + if (windows_with_hadoop()) { + checkpointDir <- file.path(tempdir(), "cproot") + expect_true(length(list.files(path = checkpointDir, all.files = TRUE)) == 0) + + setCheckpointDir(checkpointDir) + df <- read.json(jsonPath) + df <- checkpoint(df) + expect_is(df, "SparkDataFrame") + expect_false(length(list.files(path = checkpointDir, all.files = TRUE)) == 0) + } +}) + +test_that("schema(), dtypes(), columns(), names() return the correct values/format", { + df <- read.json(jsonPath) + testSchema <- schema(df) + expect_equal(length(testSchema$fields()), 2) + expect_equal(testSchema$fields()[[1]]$dataType.toString(), "LongType") + expect_equal(testSchema$fields()[[2]]$dataType.simpleString(), "string") + expect_equal(testSchema$fields()[[1]]$name(), "age") + + testTypes <- dtypes(df) + expect_equal(length(testTypes[[1]]), 2) + expect_equal(testTypes[[1]][1], "age") + + testCols <- columns(df) + expect_equal(length(testCols), 2) + expect_equal(testCols[2], "name") + + testNames <- names(df) + expect_equal(length(testNames), 2) + expect_equal(testNames[2], "name") +}) + +test_that("names() colnames() set the column names", { + df <- read.json(jsonPath) + names(df) <- c("col1", "col2") + expect_equal(colnames(df)[2], "col2") + + colnames(df) <- c("col3", "col4") + expect_equal(names(df)[1], "col3") + + expect_error(names(df) <- NULL, "Invalid column names.") + expect_error(names(df) <- c("sepal.length", "sepal_width"), + "Column names cannot contain the '.' symbol.") + expect_error(names(df) <- c(1, 2), "Invalid column names.") + expect_error(names(df) <- c("a"), + "Column names must have the same length as the number of columns in the dataset.") + expect_error(names(df) <- c("1", NA), "Column names cannot be NA.") + + expect_error(colnames(df) <- c("sepal.length", "sepal_width"), + "Column names cannot contain the '.' symbol.") + expect_error(colnames(df) <- c(1, 2), "Invalid column names.") + expect_error(colnames(df) <- c("a"), + "Column names must have the same length as the number of columns in the dataset.") + expect_error(colnames(df) <- c("1", NA), "Column names cannot be NA.") + + # Note: if this test is broken, remove check for "." character on colnames<- method + irisDF <- suppressWarnings(createDataFrame(iris)) + expect_equal(names(irisDF)[1], "Sepal_Length") + + # Test base::colnames base::names + m2 <- cbind(1, 1:4) + expect_equal(colnames(m2, do.NULL = FALSE), c("col1", "col2")) + colnames(m2) <- c("x", "Y") + expect_equal(colnames(m2), c("x", "Y")) + + z <- list(a = 1, b = "c", c = 1:3) + expect_equal(names(z)[3], "c") + names(z)[3] <- "c2" + expect_equal(names(z)[3], "c2") + + # Test subset assignment + colnames(df)[1] <- "col5" + expect_equal(colnames(df)[1], "col5") + names(df)[2] <- "col6" + expect_equal(names(df)[2], "col6") +}) + +test_that("head() and first() return the correct data", { + df <- read.json(jsonPath) + testHead <- head(df) + expect_equal(nrow(testHead), 3) + expect_equal(ncol(testHead), 2) + + testHead2 <- head(df, 2) + expect_equal(nrow(testHead2), 2) + expect_equal(ncol(testHead2), 2) + + testFirst <- first(df) + expect_equal(nrow(testFirst), 1) + + # head() and first() return the correct data on + # a DataFrame with 0 row + df0 <- limit(df, 0) + + testHead <- head(df0) + expect_equal(nrow(testHead), 0) + expect_equal(ncol(testHead), 2) + + testFirst <- first(df0) + expect_equal(nrow(testFirst), 0) + expect_equal(ncol(testFirst), 2) +}) + +test_that("distinct(), unique() and dropDuplicates() on DataFrames", { + lines <- c("{\"name\":\"Michael\"}", + "{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"Justin\", \"age\":19}", + "{\"name\":\"Justin\", \"age\":19}") + jsonPathWithDup <- tempfile(pattern = "sparkr-test", fileext = ".tmp") + writeLines(lines, jsonPathWithDup) + + df <- read.json(jsonPathWithDup) + uniques <- distinct(df) + expect_is(uniques, "SparkDataFrame") + expect_equal(count(uniques), 3) + + uniques2 <- unique(df) + expect_is(uniques2, "SparkDataFrame") + expect_equal(count(uniques2), 3) + + # Test dropDuplicates() + df <- createDataFrame( + list( + list(2, 1, 2), list(1, 1, 1), + list(1, 2, 1), list(2, 1, 2), + list(2, 2, 2), list(2, 2, 1), + list(2, 1, 1), list(1, 1, 2), + list(1, 2, 2), list(1, 2, 1)), + schema = c("key", "value1", "value2")) + result <- collect(dropDuplicates(df)) + expected <- rbind.data.frame( + c(1, 1, 1), c(1, 1, 2), c(1, 2, 1), + c(1, 2, 2), c(2, 1, 1), c(2, 1, 2), + c(2, 2, 1), c(2, 2, 2)) + names(expected) <- c("key", "value1", "value2") + expect_equivalent( + result[order(result$key, result$value1, result$value2), ], + expected) + + result <- collect(dropDuplicates(df, c("key", "value1"))) + expected <- rbind.data.frame( + c(1, 1, 1), c(1, 2, 1), c(2, 1, 2), c(2, 2, 2)) + names(expected) <- c("key", "value1", "value2") + expect_equivalent( + result[order(result$key, result$value1, result$value2), ], + expected) + + result <- collect(dropDuplicates(df, "key", "value1")) + expected <- rbind.data.frame( + c(1, 1, 1), c(1, 2, 1), c(2, 1, 2), c(2, 2, 2)) + names(expected) <- c("key", "value1", "value2") + expect_equivalent( + result[order(result$key, result$value1, result$value2), ], + expected) + + result <- collect(dropDuplicates(df, "key")) + expected <- rbind.data.frame( + c(1, 1, 1), c(2, 1, 2)) + names(expected) <- c("key", "value1", "value2") + expect_equivalent( + result[order(result$key, result$value1, result$value2), ], + expected) +}) + +test_that("sample on a DataFrame", { + df <- read.json(jsonPath) + sampled <- sample(df, FALSE, 1.0) + expect_equal(nrow(collect(sampled)), count(df)) + expect_is(sampled, "SparkDataFrame") + sampled2 <- sample(df, FALSE, 0.1, 0) # set seed for predictable result + expect_true(count(sampled2) < 3) + + count1 <- count(sample(df, FALSE, 0.1, 0)) + count2 <- count(sample(df, FALSE, 0.1, 0)) + expect_equal(count1, count2) + + # Also test sample_frac + sampled3 <- sample_frac(df, FALSE, 0.1, 0) # set seed for predictable result + expect_true(count(sampled3) < 3) + + # Different arguments + df <- createDataFrame(as.list(seq(10))) + expect_equal(count(sample(df, fraction = 0.5, seed = 3)), 4) + expect_equal(count(sample(df, withReplacement = TRUE, fraction = 0.5, seed = 3)), 2) + expect_equal(count(sample(df, fraction = 1.0)), 10) + expect_equal(count(sample(df, fraction = 1L)), 10) + expect_equal(count(sample(df, FALSE, fraction = 1.0)), 10) + + expect_error(sample(df, fraction = "a"), "fraction must be numeric") + expect_error(sample(df, "a", fraction = 0.1), "however, got character") + expect_error(sample(df, fraction = 1, seed = NA), "seed must not be NULL or NA; however, got NA") + expect_error(sample(df, fraction = -1.0), + "illegal argument - requirement failed: Sampling fraction \\(-1.0\\)") + + # nolint start + # Test base::sample is working + #expect_equal(length(sample(1:12)), 12) + # nolint end +}) + +test_that("select operators", { + df <- select(read.json(jsonPath), "name", "age") + expect_is(df$name, "Column") + expect_is(df[[2]], "Column") + expect_is(df[["age"]], "Column") + + expect_warning(df[[1:2]], + "Subset index has length > 1. Only the first index is used.") + expect_is(suppressWarnings(df[[1:2]]), "Column") + expect_warning(df[[c("name", "age")]], + "Subset index has length > 1. Only the first index is used.") + expect_is(suppressWarnings(df[[c("name", "age")]]), "Column") + + expect_warning(df[[1:2]] <- df[[1]], + "Subset index has length > 1. Only the first index is used.") + expect_warning(df[[c("name", "age")]] <- df[[1]], + "Subset index has length > 1. Only the first index is used.") + + expect_is(df[, 1, drop = F], "SparkDataFrame") + expect_equal(columns(df[, 1, drop = F]), c("name")) + expect_equal(columns(df[, "age", drop = F]), c("age")) + + df2 <- df[, c("age", "name")] + expect_is(df2, "SparkDataFrame") + expect_equal(columns(df2), c("age", "name")) + + df$age2 <- df$age + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == df$age)), 2) + df$age2 <- df$age * 2 + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == df$age * 2)), 2) + df$age2 <- df[["age"]] * 3 + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == df$age * 3)), 2) + + df$age2 <- 21 + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == 21)), 3) + + df$age2 <- c(22) + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == 22)), 3) + + expect_error(df$age3 <- c(22, NA), + "value must be a Column, literal value as atomic in length of 1, or NULL") + + df[["age2"]] <- 23 + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == 23)), 3) + + df[[3]] <- 24 + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == 24)), 3) + + df[[3]] <- df$age + expect_equal(count(where(df, df$age2 == df$age)), 2) + + df[["age2"]] <- df[["name"]] + expect_equal(count(where(df, df$age2 == df$name)), 3) + + expect_error(df[["age3"]] <- c(22, 23), + "value must be a Column, literal value as atomic in length of 1, or NULL") + + # Test parameter drop + expect_equal(class(df[, 1]) == "SparkDataFrame", T) + expect_equal(class(df[, 1, drop = T]) == "Column", T) + expect_equal(class(df[, 1, drop = F]) == "SparkDataFrame", T) + expect_equal(class(df[df$age > 4, 2, drop = T]) == "Column", T) + expect_equal(class(df[df$age > 4, 2, drop = F]) == "SparkDataFrame", T) +}) + +test_that("select with column", { + df <- read.json(jsonPath) + df1 <- select(df, "name") + expect_equal(columns(df1), c("name")) + expect_equal(count(df1), 3) + + df2 <- select(df, df$age) + expect_equal(columns(df2), c("age")) + expect_equal(count(df2), 3) + + df3 <- select(df, lit("x")) + expect_equal(columns(df3), c("x")) + expect_equal(count(df3), 3) + expect_equal(collect(select(df3, "x"))[[1, 1]], "x") + + df4 <- select(df, c("name", "age")) + expect_equal(columns(df4), c("name", "age")) + expect_equal(count(df4), 3) + + # Test select with alias + df5 <- alias(df, "table") + + expect_equal(columns(select(df5, column("table.name"))), "name") + expect_equal(columns(select(df5, "table.name")), "name") + + # Test that stats::alias is not masked + expect_is(alias(aov(yield ~ block + N * P * K, npk)), "listof") + + + expect_error(select(df, c("name", "age"), "name"), + "To select multiple columns, use a character vector or list for col") +}) + +test_that("drop column", { + df <- select(read.json(jsonPath), "name", "age") + df1 <- drop(df, "name") + expect_equal(columns(df1), c("age")) + + df$age2 <- df$age + df1 <- drop(df, c("name", "age")) + expect_equal(columns(df1), c("age2")) + + df1 <- drop(df, df$age) + expect_equal(columns(df1), c("name", "age2")) + + df$age2 <- NULL + expect_equal(columns(df), c("name", "age")) + df$age3 <- NULL + expect_equal(columns(df), c("name", "age")) + + # Test to make sure base::drop is not masked + expect_equal(drop(1:3 %*% 2:4), 20) +}) + +test_that("subsetting", { + # read.json returns columns in random order + df <- select(read.json(jsonPath), "name", "age") + filtered <- df[df$age > 20, ] + expect_equal(count(filtered), 1) + expect_equal(columns(filtered), c("name", "age")) + expect_equal(collect(filtered)$name, "Andy") + + df2 <- df[df$age == 19, 1, drop = F] + expect_is(df2, "SparkDataFrame") + expect_equal(count(df2), 1) + expect_equal(columns(df2), c("name")) + expect_equal(collect(df2)$name, "Justin") + + df3 <- df[df$age > 20, 2, drop = F] + expect_equal(count(df3), 1) + expect_equal(columns(df3), c("age")) + + df4 <- df[df$age %in% c(19, 30), 1:2] + expect_equal(count(df4), 2) + expect_equal(columns(df4), c("name", "age")) + + df5 <- df[df$age %in% c(19), c(1, 2)] + expect_equal(count(df5), 1) + expect_equal(columns(df5), c("name", "age")) + + df6 <- subset(df, df$age %in% c(30), c(1, 2)) + expect_equal(count(df6), 1) + expect_equal(columns(df6), c("name", "age")) + + df7 <- subset(df, select = "name", drop = F) + expect_equal(count(df7), 3) + expect_equal(columns(df7), c("name")) + + # Test base::subset is working + expect_equal(nrow(subset(airquality, Temp > 80, select = c(Ozone, Temp))), 68) +}) + +test_that("selectExpr() on a DataFrame", { + df <- read.json(jsonPath) + selected <- selectExpr(df, "age * 2") + expect_equal(names(selected), "(age * 2)") + expect_equal(collect(selected), collect(select(df, df$age * 2L))) + + selected2 <- selectExpr(df, "name as newName", "abs(age) as age") + expect_equal(names(selected2), c("newName", "age")) + expect_equal(count(selected2), 3) +}) + +test_that("expr() on a DataFrame", { + df <- read.json(jsonPath) + expect_equal(collect(select(df, expr("abs(-123)")))[1, 1], 123) +}) + +test_that("column calculation", { + df <- read.json(jsonPath) + d <- collect(select(df, alias(df$age + 1, "age2"))) + expect_equal(names(d), c("age2")) + df2 <- select(df, lower(df$name), abs(df$age)) + expect_is(df2, "SparkDataFrame") + expect_equal(count(df2), 3) +}) + +test_that("test HiveContext", { + if (windows_with_hadoop()) { + setHiveContext(sc) + + schema <- structType(structField("name", "string"), structField("age", "integer"), + structField("height", "float")) + createTable("people", source = "json", schema = schema) + df <- read.df(jsonPathNa, "json", schema) + insertInto(df, "people") + expect_equal(collect(sql("SELECT age from people WHERE name = 'Bob'"))$age, c(16)) + sql("DROP TABLE people") + + df <- createTable("json", jsonPath, "json") + expect_is(df, "SparkDataFrame") + expect_equal(count(df), 3) + df2 <- sql("select * from json") + expect_is(df2, "SparkDataFrame") + expect_equal(count(df2), 3) + + jsonPath2 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") + saveAsTable(df, "json2", "json", "append", path = jsonPath2) + df3 <- sql("select * from json2") + expect_is(df3, "SparkDataFrame") + expect_equal(count(df3), 3) + unlink(jsonPath2) + + hivetestDataPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") + saveAsTable(df, "hivetestbl", path = hivetestDataPath) + df4 <- sql("select * from hivetestbl") + expect_is(df4, "SparkDataFrame") + expect_equal(count(df4), 3) + unlink(hivetestDataPath) + + parquetDataPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") + saveAsTable(df, "parquetest", "parquet", mode = "overwrite", path = parquetDataPath) + df5 <- sql("select * from parquetest") + expect_is(df5, "SparkDataFrame") + expect_equal(count(df5), 3) + unlink(parquetDataPath) + + unsetHiveContext() + } +}) + +test_that("column operators", { + c <- column("a") + c2 <- (- c + 1 - 2) * 3 / 4.0 + c3 <- (c + c2 - c2) * c2 %% c2 + c4 <- (c > c2) & (c2 <= c3) | (c == c2) & (c2 != c3) + c5 <- c2 ^ c3 ^ c4 + c6 <- c2 %<=>% c3 + c7 <- !c6 +}) + +test_that("column functions", { + c <- column("a") + c1 <- abs(c) + acos(c) + approxCountDistinct(c) + ascii(c) + asin(c) + atan(c) + c2 <- avg(c) + base64(c) + bin(c) + bitwiseNOT(c) + cbrt(c) + ceil(c) + cos(c) + c3 <- cosh(c) + count(c) + crc32(c) + hash(c) + exp(c) + c4 <- explode(c) + expm1(c) + factorial(c) + first(c) + floor(c) + hex(c) + c5 <- hour(c) + initcap(c) + last(c) + last_day(c) + length(c) + c6 <- log(c) + (c) + log1p(c) + log2(c) + lower(c) + ltrim(c) + max(c) + md5(c) + c7 <- mean(c) + min(c) + month(c) + negate(c) + posexplode(c) + quarter(c) + c8 <- reverse(c) + rint(c) + round(c) + rtrim(c) + sha1(c) + monotonically_increasing_id() + c9 <- signum(c) + sin(c) + sinh(c) + size(c) + stddev(c) + soundex(c) + sqrt(c) + sum(c) + c10 <- sumDistinct(c) + tan(c) + tanh(c) + toDegrees(c) + toRadians(c) + c11 <- to_date(c) + trim(c) + unbase64(c) + unhex(c) + upper(c) + c12 <- variance(c) + c13 <- lead("col", 1) + lead(c, 1) + lag("col", 1) + lag(c, 1) + c14 <- cume_dist() + ntile(1) + corr(c, c1) + c15 <- dense_rank() + percent_rank() + rank() + row_number() + c16 <- is.nan(c) + isnan(c) + isNaN(c) + c17 <- cov(c, c1) + cov("c", "c1") + covar_samp(c, c1) + covar_samp("c", "c1") + c18 <- covar_pop(c, c1) + covar_pop("c", "c1") + c19 <- spark_partition_id() + coalesce(c) + coalesce(c1, c2, c3) + c20 <- to_timestamp(c) + to_timestamp(c, "yyyy") + to_date(c, "yyyy") + c21 <- posexplode_outer(c) + explode_outer(c) + c22 <- not(c) + c23 <- trunc(c, "year") + trunc(c, "yyyy") + trunc(c, "yy") + + trunc(c, "month") + trunc(c, "mon") + trunc(c, "mm") + + # Test if base::is.nan() is exposed + expect_equal(is.nan(c("a", "b")), c(FALSE, FALSE)) + + # Test if base::rank() is exposed + expect_equal(class(rank())[[1]], "Column") + expect_equal(rank(1:3), as.numeric(c(1:3))) + + df <- read.json(jsonPath) + df2 <- select(df, between(df$age, c(20, 30)), between(df$age, c(10, 20))) + expect_equal(collect(df2)[[2, 1]], TRUE) + expect_equal(collect(df2)[[2, 2]], FALSE) + expect_equal(collect(df2)[[3, 1]], FALSE) + expect_equal(collect(df2)[[3, 2]], TRUE) + + # Test that input_file_name() + actual_names <- sort(collect(distinct(select(df, input_file_name())))) + expect_equal(length(actual_names), 1) + expect_equal(basename(actual_names[1, 1]), basename(jsonPath)) + + df3 <- select(df, between(df$name, c("Apache", "Spark"))) + expect_equal(collect(df3)[[1, 1]], TRUE) + expect_equal(collect(df3)[[2, 1]], FALSE) + expect_equal(collect(df3)[[3, 1]], TRUE) + + df4 <- select(df, countDistinct(df$age, df$name)) + expect_equal(collect(df4)[[1, 1]], 2) + + expect_equal(collect(select(df, sum(df$age)))[1, 1], 49) + expect_true(abs(collect(select(df, stddev(df$age)))[1, 1] - 7.778175) < 1e-6) + expect_equal(collect(select(df, var_pop(df$age)))[1, 1], 30.25) + + df5 <- createDataFrame(list(list(a = "010101"))) + expect_equal(collect(select(df5, conv(df5$a, 2, 16)))[1, 1], "15") + + # Test array_contains() and sort_array() + df <- createDataFrame(list(list(list(1L, 2L, 3L)), list(list(6L, 5L, 4L)))) + result <- collect(select(df, array_contains(df[[1]], 1L)))[[1]] + expect_equal(result, c(TRUE, FALSE)) + + result <- collect(select(df, sort_array(df[[1]], FALSE)))[[1]] + expect_equal(result, list(list(3L, 2L, 1L), list(6L, 5L, 4L))) + result <- collect(select(df, sort_array(df[[1]])))[[1]] + expect_equal(result, list(list(1L, 2L, 3L), list(4L, 5L, 6L))) + + # Test map_keys() and map_values() + df <- createDataFrame(list(list(map = as.environment(list(x = 1, y = 2))))) + result <- collect(select(df, map_keys(df$map)))[[1]] + expect_equal(result, list(list("x", "y"))) + + result <- collect(select(df, map_values(df$map)))[[1]] + expect_equal(result, list(list(1, 2))) + + # Test that stats::lag is working + expect_equal(length(lag(ldeaths, 12)), 72) + + # Test struct() + df <- createDataFrame(list(list(1L, 2L, 3L), list(4L, 5L, 6L)), + schema = c("a", "b", "c")) + result <- collect(select(df, alias(struct("a", "c"), "d"))) + expected <- data.frame(row.names = 1:2) + expected$"d" <- list(listToStruct(list(a = 1L, c = 3L)), + listToStruct(list(a = 4L, c = 6L))) + expect_equal(result, expected) + + result <- collect(select(df, alias(struct(df$a, df$b), "d"))) + expected <- data.frame(row.names = 1:2) + expected$"d" <- list(listToStruct(list(a = 1L, b = 2L)), + listToStruct(list(a = 4L, b = 5L))) + expect_equal(result, expected) + + # Test encode(), decode() + bytes <- as.raw(c(0xe5, 0xa4, 0xa7, 0xe5, 0x8d, 0x83, 0xe4, 0xb8, 0x96, 0xe7, 0x95, 0x8c)) + df <- createDataFrame(list(list(markUtf8("大千世界"), "utf-8", bytes)), + schema = c("a", "b", "c")) + result <- collect(select(df, encode(df$a, "utf-8"), decode(df$c, "utf-8"))) + expect_equal(result[[1]][[1]], bytes) + expect_equal(result[[2]], markUtf8("大千世界")) + + # Test first(), last() + df <- read.json(jsonPath) + expect_equal(collect(select(df, first(df$age)))[[1]], NA_real_) + expect_equal(collect(select(df, first(df$age, TRUE)))[[1]], 30) + expect_equal(collect(select(df, first("age")))[[1]], NA_real_) + expect_equal(collect(select(df, first("age", TRUE)))[[1]], 30) + expect_equal(collect(select(df, last(df$age)))[[1]], 19) + expect_equal(collect(select(df, last(df$age, TRUE)))[[1]], 19) + expect_equal(collect(select(df, last("age")))[[1]], 19) + expect_equal(collect(select(df, last("age", TRUE)))[[1]], 19) + + # Test bround() + df <- createDataFrame(data.frame(x = c(2.5, 3.5))) + expect_equal(collect(select(df, bround(df$x, 0)))[[1]][1], 2) + expect_equal(collect(select(df, bround(df$x, 0)))[[1]][2], 4) + + # Test to_json(), from_json() + df <- sql("SELECT array(named_struct('name', 'Bob'), named_struct('name', 'Alice')) as people") + j <- collect(select(df, alias(to_json(df$people), "json"))) + expect_equal(j[order(j$json), ][1], "[{\"name\":\"Bob\"},{\"name\":\"Alice\"}]") + + df <- sql("SELECT map('name', 'Bob') as people") + j <- collect(select(df, alias(to_json(df$people), "json"))) + expect_equal(j[order(j$json), ][1], "{\"name\":\"Bob\"}") + + df <- sql("SELECT array(map('name', 'Bob'), map('name', 'Alice')) as people") + j <- collect(select(df, alias(to_json(df$people), "json"))) + expect_equal(j[order(j$json), ][1], "[{\"name\":\"Bob\"},{\"name\":\"Alice\"}]") + + df <- read.json(mapTypeJsonPath) + j <- collect(select(df, alias(to_json(df$info), "json"))) + expect_equal(j[order(j$json), ][1], "{\"age\":16,\"height\":176.5}") + df <- as.DataFrame(j) + schemas <- list(structType(structField("age", "integer"), structField("height", "double")), + "age INT, height DOUBLE") + for (schema in schemas) { + s <- collect(select(df, alias(from_json(df$json, schema), "structcol"))) + expect_equal(ncol(s), 1) + expect_equal(nrow(s), 3) + expect_is(s[[1]][[1]], "struct") + expect_true(any(apply(s, 1, function(x) { x[[1]]$age == 16 } ))) + } + + # passing option + df <- as.DataFrame(list(list("col" = "{\"date\":\"21/10/2014\"}"))) + schema2 <- structType(structField("date", "date")) + s <- collect(select(df, from_json(df$col, schema2))) + expect_equal(s[[1]][[1]], NA) + s <- collect(select(df, from_json(df$col, schema2, dateFormat = "dd/MM/yyyy"))) + expect_is(s[[1]][[1]]$date, "Date") + expect_equal(as.character(s[[1]][[1]]$date), "2014-10-21") + + # check for unparseable + df <- as.DataFrame(list(list("a" = ""))) + expect_equal(collect(select(df, from_json(df$a, schema)))[[1]][[1]], NA) + + # check if array type in string is correctly supported. + jsonArr <- "[{\"name\":\"Bob\"}, {\"name\":\"Alice\"}]" + df <- as.DataFrame(list(list("people" = jsonArr))) + for (schema in list(structType(structField("name", "string")), "name STRING")) { + arr <- collect(select(df, alias(from_json(df$people, schema, as.json.array = TRUE), "arrcol"))) + expect_equal(ncol(arr), 1) + expect_equal(nrow(arr), 1) + expect_is(arr[[1]][[1]], "list") + expect_equal(length(arr$arrcol[[1]]), 2) + expect_equal(arr$arrcol[[1]][[1]]$name, "Bob") + expect_equal(arr$arrcol[[1]][[2]]$name, "Alice") + } + + # Test create_array() and create_map() + df <- as.DataFrame(data.frame( + x = c(1.0, 2.0), y = c(-1.0, 3.0), z = c(-2.0, 5.0) + )) + + arrs <- collect(select(df, create_array(df$x, df$y, df$z))) + expect_equal(arrs[, 1], list(list(1, -1, -2), list(2, 3, 5))) + + maps <- collect(select( + df, create_map(lit("x"), df$x, lit("y"), df$y, lit("z"), df$z))) + + expect_equal( + maps[, 1], + lapply( + list(list(x = 1, y = -1, z = -2), list(x = 2, y = 3, z = 5)), + as.environment)) + + df <- as.DataFrame(data.frame(is_true = c(TRUE, FALSE, NA))) + expect_equal( + collect(select(df, alias(not(df$is_true), "is_false"))), + data.frame(is_false = c(FALSE, TRUE, NA)) + ) +}) + +test_that("column binary mathfunctions", { + lines <- c("{\"a\":1, \"b\":5}", + "{\"a\":2, \"b\":6}", + "{\"a\":3, \"b\":7}", + "{\"a\":4, \"b\":8}") + jsonPathWithDup <- tempfile(pattern = "sparkr-test", fileext = ".tmp") + writeLines(lines, jsonPathWithDup) + df <- read.json(jsonPathWithDup) + expect_equal(collect(select(df, atan2(df$a, df$b)))[1, "ATAN2(a, b)"], atan2(1, 5)) + expect_equal(collect(select(df, atan2(df$a, df$b)))[2, "ATAN2(a, b)"], atan2(2, 6)) + expect_equal(collect(select(df, atan2(df$a, df$b)))[3, "ATAN2(a, b)"], atan2(3, 7)) + expect_equal(collect(select(df, atan2(df$a, df$b)))[4, "ATAN2(a, b)"], atan2(4, 8)) + ## nolint start + expect_equal(collect(select(df, hypot(df$a, df$b)))[1, "HYPOT(a, b)"], sqrt(1^2 + 5^2)) + expect_equal(collect(select(df, hypot(df$a, df$b)))[2, "HYPOT(a, b)"], sqrt(2^2 + 6^2)) + expect_equal(collect(select(df, hypot(df$a, df$b)))[3, "HYPOT(a, b)"], sqrt(3^2 + 7^2)) + expect_equal(collect(select(df, hypot(df$a, df$b)))[4, "HYPOT(a, b)"], sqrt(4^2 + 8^2)) + ## nolint end + expect_equal(collect(select(df, shiftLeft(df$b, 1)))[4, 1], 16) + expect_equal(collect(select(df, shiftRight(df$b, 1)))[4, 1], 4) + expect_equal(collect(select(df, shiftRightUnsigned(df$b, 1)))[4, 1], 4) + expect_equal(class(collect(select(df, rand()))[2, 1]), "numeric") + expect_equal(collect(select(df, rand(1)))[1, 1], 0.134, tolerance = 0.01) + expect_equal(class(collect(select(df, randn()))[2, 1]), "numeric") + expect_equal(collect(select(df, randn(1)))[1, 1], -1.03, tolerance = 0.01) +}) + +test_that("string operators", { + df <- read.json(jsonPath) + expect_equal(count(where(df, like(df$name, "A%"))), 1) + expect_equal(count(where(df, startsWith(df$name, "A"))), 1) + expect_true(first(select(df, startsWith(df$name, "M")))[[1]]) + expect_false(first(select(df, startsWith(df$name, "m")))[[1]]) + expect_true(first(select(df, endsWith(df$name, "el")))[[1]]) + expect_equal(first(select(df, substr(df$name, 1, 2)))[[1]], "Mi") + if (as.numeric(R.version$major) >= 3 && as.numeric(R.version$minor) >= 3) { + expect_true(startsWith("Hello World", "Hello")) + expect_false(endsWith("Hello World", "a")) + } + expect_equal(collect(select(df, cast(df$age, "string")))[[2, 1]], "30") + expect_equal(collect(select(df, concat(df$name, lit(":"), df$age)))[[2, 1]], "Andy:30") + expect_equal(collect(select(df, concat_ws(":", df$name)))[[2, 1]], "Andy") + expect_equal(collect(select(df, concat_ws(":", df$name, df$age)))[[2, 1]], "Andy:30") + expect_equal(collect(select(df, instr(df$name, "i")))[, 1], c(2, 0, 5)) + expect_equal(collect(select(df, format_number(df$age, 2)))[2, 1], "30.00") + expect_equal(collect(select(df, sha1(df$name)))[2, 1], + "ab5a000e88b5d9d0fa2575f5c6263eb93452405d") + expect_equal(collect(select(df, sha2(df$name, 256)))[2, 1], + "80f2aed3c618c423ddf05a2891229fba44942d907173152442cf6591441ed6dc") + expect_equal(collect(select(df, format_string("Name:%s", df$name)))[2, 1], "Name:Andy") + expect_equal(collect(select(df, format_string("%s, %d", df$name, df$age)))[2, 1], "Andy, 30") + expect_equal(collect(select(df, regexp_extract(df$name, "(n.y)", 1)))[2, 1], "ndy") + expect_equal(collect(select(df, regexp_replace(df$name, "(n.y)", "ydn")))[2, 1], "Aydn") + + l2 <- list(list(a = "aaads")) + df2 <- createDataFrame(l2) + expect_equal(collect(select(df2, locate("aa", df2$a)))[1, 1], 1) + expect_equal(collect(select(df2, locate("aa", df2$a, 2)))[1, 1], 2) + expect_equal(collect(select(df2, lpad(df2$a, 8, "#")))[1, 1], "###aaads") # nolint + expect_equal(collect(select(df2, rpad(df2$a, 8, "#")))[1, 1], "aaads###") # nolint + + l3 <- list(list(a = "a.b.c.d")) + df3 <- createDataFrame(l3) + expect_equal(collect(select(df3, substring_index(df3$a, ".", 2)))[1, 1], "a.b") + expect_equal(collect(select(df3, substring_index(df3$a, ".", -3)))[1, 1], "b.c.d") + expect_equal(collect(select(df3, translate(df3$a, "bc", "12")))[1, 1], "a.1.2.d") + + l4 <- list(list(a = "a.b@c.d 1\\b")) + df4 <- createDataFrame(l4) + expect_equal( + collect(select(df4, split_string(df4$a, "\\s+")))[1, 1], + list(list("a.b@c.d", "1\\b")) + ) + expect_equal( + collect(select(df4, split_string(df4$a, "\\.")))[1, 1], + list(list("a", "b@c", "d 1\\b")) + ) + expect_equal( + collect(select(df4, split_string(df4$a, "@")))[1, 1], + list(list("a.b", "c.d 1\\b")) + ) + expect_equal( + collect(select(df4, split_string(df4$a, "\\\\")))[1, 1], + list(list("a.b@c.d 1", "b")) + ) + + l5 <- list(list(a = "abc")) + df5 <- createDataFrame(l5) + expect_equal( + collect(select(df5, repeat_string(df5$a, 1L)))[1, 1], + "abc" + ) + expect_equal( + collect(select(df5, repeat_string(df5$a, 3)))[1, 1], + "abcabcabc" + ) + expect_equal( + collect(select(df5, repeat_string(df5$a, -1)))[1, 1], + "" + ) +}) + +test_that("date functions on a DataFrame", { + .originalTimeZone <- Sys.getenv("TZ") + Sys.setenv(TZ = "UTC") + l <- list(list(a = 1L, b = as.Date("2012-12-13")), + list(a = 2L, b = as.Date("2013-12-14")), + list(a = 3L, b = as.Date("2014-12-15"))) + df <- createDataFrame(l) + expect_equal(collect(select(df, dayofmonth(df$b)))[, 1], c(13, 14, 15)) + expect_equal(collect(select(df, dayofyear(df$b)))[, 1], c(348, 348, 349)) + expect_equal(collect(select(df, weekofyear(df$b)))[, 1], c(50, 50, 51)) + expect_equal(collect(select(df, year(df$b)))[, 1], c(2012, 2013, 2014)) + expect_equal(collect(select(df, month(df$b)))[, 1], c(12, 12, 12)) + expect_equal(collect(select(df, last_day(df$b)))[, 1], + c(as.Date("2012-12-31"), as.Date("2013-12-31"), as.Date("2014-12-31"))) + expect_equal(collect(select(df, next_day(df$b, "MONDAY")))[, 1], + c(as.Date("2012-12-17"), as.Date("2013-12-16"), as.Date("2014-12-22"))) + expect_equal(collect(select(df, date_format(df$b, "y")))[, 1], c("2012", "2013", "2014")) + expect_equal(collect(select(df, add_months(df$b, 3)))[, 1], + c(as.Date("2013-03-13"), as.Date("2014-03-14"), as.Date("2015-03-15"))) + expect_equal(collect(select(df, date_add(df$b, 1)))[, 1], + c(as.Date("2012-12-14"), as.Date("2013-12-15"), as.Date("2014-12-16"))) + expect_equal(collect(select(df, date_sub(df$b, 1)))[, 1], + c(as.Date("2012-12-12"), as.Date("2013-12-13"), as.Date("2014-12-14"))) + + l2 <- list(list(a = 1L, b = as.POSIXlt("2012-12-13 12:34:00", tz = "UTC")), + list(a = 2L, b = as.POSIXlt("2014-12-15 01:24:34", tz = "UTC"))) + df2 <- createDataFrame(l2) + expect_equal(collect(select(df2, minute(df2$b)))[, 1], c(34, 24)) + expect_equal(collect(select(df2, second(df2$b)))[, 1], c(0, 34)) + expect_equal(collect(select(df2, from_utc_timestamp(df2$b, "JST")))[, 1], + c(as.POSIXlt("2012-12-13 21:34:00 UTC"), as.POSIXlt("2014-12-15 10:24:34 UTC"))) + expect_equal(collect(select(df2, to_utc_timestamp(df2$b, "JST")))[, 1], + c(as.POSIXlt("2012-12-13 03:34:00 UTC"), as.POSIXlt("2014-12-14 16:24:34 UTC"))) + expect_gt(collect(select(df2, unix_timestamp()))[1, 1], 0) + expect_gt(collect(select(df2, unix_timestamp(df2$b)))[1, 1], 0) + expect_gt(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0) + + l3 <- list(list(a = 1000), list(a = -1000)) + df3 <- createDataFrame(l3) + result31 <- collect(select(df3, from_unixtime(df3$a))) + expect_equal(grep("\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", result31[, 1], perl = TRUE), + c(1, 2)) + result32 <- collect(select(df3, from_unixtime(df3$a, "yyyy"))) + expect_equal(grep("\\d{4}", result32[, 1]), c(1, 2)) + Sys.setenv(TZ = .originalTimeZone) +}) + +test_that("greatest() and least() on a DataFrame", { + l <- list(list(a = 1, b = 2), list(a = 3, b = 4)) + df <- createDataFrame(l) + expect_equal(collect(select(df, greatest(df$a, df$b)))[, 1], c(2, 4)) + expect_equal(collect(select(df, least(df$a, df$b)))[, 1], c(1, 3)) +}) + +test_that("time windowing (window()) with all inputs", { + df <- createDataFrame(data.frame(t = c("2016-03-11 09:00:07"), v = c(1))) + df$window <- window(df$t, "5 seconds", "5 seconds", "0 seconds") + local <- collect(df)$v + # Not checking time windows because of possible time zone issues. Just checking that the function + # works + expect_equal(local, c(1)) +}) + +test_that("time windowing (window()) with slide duration", { + df <- createDataFrame(data.frame(t = c("2016-03-11 09:00:07"), v = c(1))) + df$window <- window(df$t, "5 seconds", "2 seconds") + local <- collect(df)$v + # Not checking time windows because of possible time zone issues. Just checking that the function + # works + expect_equal(local, c(1, 1)) +}) + +test_that("time windowing (window()) with start time", { + df <- createDataFrame(data.frame(t = c("2016-03-11 09:00:07"), v = c(1))) + df$window <- window(df$t, "5 seconds", startTime = "2 seconds") + local <- collect(df)$v + # Not checking time windows because of possible time zone issues. Just checking that the function + # works + expect_equal(local, c(1)) +}) + +test_that("time windowing (window()) with just window duration", { + df <- createDataFrame(data.frame(t = c("2016-03-11 09:00:07"), v = c(1))) + df$window <- window(df$t, "5 seconds") + local <- collect(df)$v + # Not checking time windows because of possible time zone issues. Just checking that the function + # works + expect_equal(local, c(1)) +}) + +test_that("when(), otherwise() and ifelse() on a DataFrame", { + l <- list(list(a = 1, b = 2), list(a = 3, b = 4)) + df <- createDataFrame(l) + expect_equal(collect(select(df, when(df$a > 1 & df$b > 2, 1)))[, 1], c(NA, 1)) + expect_equal(collect(select(df, otherwise(when(df$a > 1, 1), 0)))[, 1], c(0, 1)) + expect_equal(collect(select(df, ifelse(df$a > 1 & df$b > 2, 0, 1)))[, 1], c(1, 0)) +}) + +test_that("when(), otherwise() and ifelse() with column on a DataFrame", { + l <- list(list(a = 1, b = 2), list(a = 3, b = 4)) + df <- createDataFrame(l) + expect_equal(collect(select(df, when(df$a > 1 & df$b > 2, lit(1))))[, 1], c(NA, 1)) + expect_equal(collect(select(df, otherwise(when(df$a > 1, lit(1)), lit(0))))[, 1], c(0, 1)) + expect_equal(collect(select(df, ifelse(df$a > 1 & df$b > 2, lit(0), lit(1))))[, 1], c(1, 0)) +}) + +test_that("group by, agg functions", { + df <- read.json(jsonPath) + df1 <- agg(df, name = "max", age = "sum") + expect_equal(1, count(df1)) + df1 <- agg(df, age2 = max(df$age)) + expect_equal(1, count(df1)) + expect_equal(columns(df1), c("age2")) + + gd <- groupBy(df, "name") + expect_is(gd, "GroupedData") + df2 <- count(gd) + expect_is(df2, "SparkDataFrame") + expect_equal(3, count(df2)) + + # Also test group_by, summarize, mean + gd1 <- group_by(df, "name") + expect_is(gd1, "GroupedData") + df_summarized <- summarize(gd, mean_age = mean(df$age)) + expect_is(df_summarized, "SparkDataFrame") + expect_equal(3, count(df_summarized)) + + df3 <- agg(gd, age = "stddev") + expect_is(df3, "SparkDataFrame") + df3_local <- collect(df3) + expect_true(is.nan(df3_local[df3_local$name == "Andy", ][1, 2])) + + df4 <- agg(gd, sumAge = sum(df$age)) + expect_is(df4, "SparkDataFrame") + expect_equal(3, count(df4)) + expect_equal(columns(df4), c("name", "sumAge")) + + df5 <- sum(gd, "age") + expect_is(df5, "SparkDataFrame") + expect_equal(3, count(df5)) + + expect_equal(3, count(mean(gd))) + expect_equal(3, count(max(gd))) + expect_equal(30, collect(max(gd))[2, 2]) + expect_equal(1, collect(count(gd))[1, 2]) + + mockLines2 <- c("{\"name\":\"ID1\", \"value\": \"10\"}", + "{\"name\":\"ID1\", \"value\": \"10\"}", + "{\"name\":\"ID1\", \"value\": \"22\"}", + "{\"name\":\"ID2\", \"value\": \"-3\"}") + jsonPath2 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") + writeLines(mockLines2, jsonPath2) + gd2 <- groupBy(read.json(jsonPath2), "name") + df6 <- agg(gd2, value = "sum") + df6_local <- collect(df6) + expect_equal(42, df6_local[df6_local$name == "ID1", ][1, 2]) + expect_equal(-3, df6_local[df6_local$name == "ID2", ][1, 2]) + + df7 <- agg(gd2, value = "stddev") + df7_local <- collect(df7) + expect_true(abs(df7_local[df7_local$name == "ID1", ][1, 2] - 6.928203) < 1e-6) + expect_true(is.nan(df7_local[df7_local$name == "ID2", ][1, 2])) + + mockLines3 <- c("{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"Justin\", \"age\":19}", + "{\"name\":\"Justin\", \"age\":1}") + jsonPath3 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") + writeLines(mockLines3, jsonPath3) + df8 <- read.json(jsonPath3) + gd3 <- groupBy(df8, "name") + gd3_local <- collect(sum(gd3)) + expect_equal(60, gd3_local[gd3_local$name == "Andy", ][1, 2]) + expect_equal(20, gd3_local[gd3_local$name == "Justin", ][1, 2]) + + expect_true(abs(collect(agg(df, sd(df$age)))[1, 1] - 7.778175) < 1e-6) + gd3_local <- collect(agg(gd3, var(df8$age))) + expect_equal(162, gd3_local[gd3_local$name == "Justin", ][1, 2]) + + # Test stats::sd, stats::var are working + expect_true(abs(sd(1:2) - 0.7071068) < 1e-6) + expect_true(abs(var(1:5, 1:5) - 2.5) < 1e-6) + + # Test collect_list and collect_set + gd3_collections_local <- collect( + agg(gd3, collect_set(df8$age), collect_list(df8$age)) + ) + + expect_equal( + unlist(gd3_collections_local[gd3_collections_local$name == "Andy", 2]), + c(30) + ) + + expect_equal( + unlist(gd3_collections_local[gd3_collections_local$name == "Andy", 3]), + c(30, 30) + ) + + expect_equal( + sort(unlist( + gd3_collections_local[gd3_collections_local$name == "Justin", 3] + )), + c(1, 19) + ) + + unlink(jsonPath2) + unlink(jsonPath3) +}) + +test_that("pivot GroupedData column", { + df <- createDataFrame(data.frame( + earnings = c(10000, 10000, 11000, 15000, 12000, 20000, 21000, 22000), + course = c("R", "Python", "R", "Python", "R", "Python", "R", "Python"), + year = c(2013, 2013, 2014, 2014, 2015, 2015, 2016, 2016) + )) + sum1 <- collect(sum(pivot(groupBy(df, "year"), "course"), "earnings")) + sum2 <- collect(sum(pivot(groupBy(df, "year"), "course", c("Python", "R")), "earnings")) + sum3 <- collect(sum(pivot(groupBy(df, "year"), "course", list("Python", "R")), "earnings")) + sum4 <- collect(sum(pivot(groupBy(df, "year"), "course", "R"), "earnings")) + + correct_answer <- data.frame( + year = c(2013, 2014, 2015, 2016), + Python = c(10000, 15000, 20000, 22000), + R = c(10000, 11000, 12000, 21000) + ) + expect_equal(sum1, correct_answer) + expect_equal(sum2, correct_answer) + expect_equal(sum3, correct_answer) + expect_equal(sum4, correct_answer[, c("year", "R")]) + + expect_error(collect(sum(pivot(groupBy(df, "year"), "course", c("R", "R")), "earnings"))) + expect_error(collect(sum(pivot(groupBy(df, "year"), "course", list("R", "R")), "earnings"))) +}) + +test_that("test multi-dimensional aggregations with cube and rollup", { + df <- createDataFrame(data.frame( + id = 1:6, + year = c(2016, 2016, 2016, 2017, 2017, 2017), + salary = c(10000, 15000, 20000, 22000, 32000, 21000), + department = c("management", "rnd", "sales", "management", "rnd", "sales") + )) + + actual_cube <- collect( + orderBy( + agg( + cube(df, "year", "department"), + expr("sum(salary) AS total_salary"), + expr("avg(salary) AS average_salary"), + alias(grouping_bit(df$year), "grouping_year"), + alias(grouping_bit(df$department), "grouping_department"), + alias(grouping_id(df$year, df$department), "grouping_id") + ), + "year", "department" + ) + ) + + expected_cube <- data.frame( + year = c(rep(NA, 4), rep(2016, 4), rep(2017, 4)), + department = rep(c(NA, "management", "rnd", "sales"), times = 3), + total_salary = c( + 120000, # Total + 10000 + 22000, 15000 + 32000, 20000 + 21000, # Department only + 20000 + 15000 + 10000, # 2016 + 10000, 15000, 20000, # 2016 each department + 21000 + 32000 + 22000, # 2017 + 22000, 32000, 21000 # 2017 each department + ), + average_salary = c( + # Total + mean(c(20000, 15000, 10000, 21000, 32000, 22000)), + # Mean by department + mean(c(10000, 22000)), mean(c(15000, 32000)), mean(c(20000, 21000)), + mean(c(10000, 15000, 20000)), # 2016 + 10000, 15000, 20000, # 2016 each department + mean(c(21000, 32000, 22000)), # 2017 + 22000, 32000, 21000 # 2017 each department + ), + grouping_year = c( + 1, # global + 1, 1, 1, # by department + 0, # 2016 + 0, 0, 0, # 2016 by department + 0, # 2017 + 0, 0, 0 # 2017 by department + ), + grouping_department = c( + 1, # global + 0, 0, 0, # by department + 1, # 2016 + 0, 0, 0, # 2016 by department + 1, # 2017 + 0, 0, 0 # 2017 by department + ), + grouping_id = c( + 3, # 11 + 2, 2, 2, # 10 + 1, # 01 + 0, 0, 0, # 00 + 1, # 01 + 0, 0, 0 # 00 + ), + stringsAsFactors = FALSE + ) + + expect_equal(actual_cube, expected_cube) + + # cube should accept column objects + expect_equal( + count(sum(cube(df, df$year, df$department), "salary")), + 12 + ) + + # cube without columns should result in a single aggregate + expect_equal( + collect(agg(cube(df), expr("sum(salary) as total_salary"))), + data.frame(total_salary = 120000) + ) + + actual_rollup <- collect( + orderBy( + agg( + rollup(df, "year", "department"), + expr("sum(salary) AS total_salary"), expr("avg(salary) AS average_salary"), + alias(grouping_bit(df$year), "grouping_year"), + alias(grouping_bit(df$department), "grouping_department"), + alias(grouping_id(df$year, df$department), "grouping_id") + ), + "year", "department" + ) + ) + + expected_rollup <- data.frame( + year = c(NA, rep(2016, 4), rep(2017, 4)), + department = c(NA, rep(c(NA, "management", "rnd", "sales"), times = 2)), + total_salary = c( + 120000, # Total + 20000 + 15000 + 10000, # 2016 + 10000, 15000, 20000, # 2016 each department + 21000 + 32000 + 22000, # 2017 + 22000, 32000, 21000 # 2017 each department + ), + average_salary = c( + # Total + mean(c(20000, 15000, 10000, 21000, 32000, 22000)), + mean(c(10000, 15000, 20000)), # 2016 + 10000, 15000, 20000, # 2016 each department + mean(c(21000, 32000, 22000)), # 2017 + 22000, 32000, 21000 # 2017 each department + ), + grouping_year = c( + 1, # global + 0, # 2016 + 0, 0, 0, # 2016 each department + 0, # 2017 + 0, 0, 0 # 2017 each department + ), + grouping_department = c( + 1, # global + 1, # 2016 + 0, 0, 0, # 2016 each department + 1, # 2017 + 0, 0, 0 # 2017 each department + ), + grouping_id = c( + 3, # 11 + 1, # 01 + 0, 0, 0, # 00 + 1, # 01 + 0, 0, 0 # 00 + ), + stringsAsFactors = FALSE + ) + + expect_equal(actual_rollup, expected_rollup) + + # cube should accept column objects + expect_equal( + count(sum(rollup(df, df$year, df$department), "salary")), + 9 + ) + + # rollup without columns should result in a single aggregate + expect_equal( + collect(agg(rollup(df), expr("sum(salary) as total_salary"))), + data.frame(total_salary = 120000) + ) +}) + +test_that("arrange() and orderBy() on a DataFrame", { + df <- read.json(jsonPath) + sorted <- arrange(df, df$age) + expect_equal(collect(sorted)[1, 2], "Michael") + + sorted2 <- arrange(df, "name", decreasing = FALSE) + expect_equal(collect(sorted2)[2, "age"], 19) + + sorted3 <- orderBy(df, asc(df$age)) + expect_true(is.na(first(sorted3)$age)) + expect_equal(collect(sorted3)[2, "age"], 19) + + sorted4 <- orderBy(df, desc(df$name)) + expect_equal(first(sorted4)$name, "Michael") + expect_equal(collect(sorted4)[3, "name"], "Andy") + + sorted5 <- arrange(df, "age", "name", decreasing = TRUE) + expect_equal(collect(sorted5)[1, 2], "Andy") + + sorted6 <- arrange(df, "age", "name", decreasing = c(T, F)) + expect_equal(collect(sorted6)[1, 2], "Andy") + + sorted7 <- arrange(df, "name", decreasing = FALSE) + expect_equal(collect(sorted7)[2, "age"], 19) +}) + +test_that("filter() on a DataFrame", { + df <- read.json(jsonPath) + filtered <- filter(df, "age > 20") + expect_equal(count(filtered), 1) + expect_equal(collect(filtered)$name, "Andy") + filtered2 <- where(df, df$name != "Michael") + expect_equal(count(filtered2), 2) + expect_equal(collect(filtered2)$age[2], 19) + + # test suites for %in% + filtered3 <- filter(df, "age in (19)") + expect_equal(count(filtered3), 1) + filtered4 <- filter(df, "age in (19, 30)") + expect_equal(count(filtered4), 2) + filtered5 <- where(df, df$age %in% c(19)) + expect_equal(count(filtered5), 1) + filtered6 <- where(df, df$age %in% c(19, 30)) + expect_equal(count(filtered6), 2) + + # test suites for %<=>% + dfNa <- read.json(jsonPathNa) + expect_equal(count(filter(dfNa, dfNa$age %<=>% 60)), 1) + expect_equal(count(filter(dfNa, !(dfNa$age %<=>% 60))), 5 - 1) + expect_equal(count(filter(dfNa, dfNa$age %<=>% NULL)), 3) + expect_equal(count(filter(dfNa, !(dfNa$age %<=>% NULL))), 5 - 3) + # match NA from two columns + expect_equal(count(filter(dfNa, dfNa$age %<=>% dfNa$height)), 2) + expect_equal(count(filter(dfNa, !(dfNa$age %<=>% dfNa$height))), 5 - 2) + + # Test stats::filter is working + #expect_true(is.ts(filter(1:100, rep(1, 3)))) # nolint +}) + +test_that("join(), crossJoin() and merge() on a DataFrame", { + df <- read.json(jsonPath) + + mockLines2 <- c("{\"name\":\"Michael\", \"test\": \"yes\"}", + "{\"name\":\"Andy\", \"test\": \"no\"}", + "{\"name\":\"Justin\", \"test\": \"yes\"}", + "{\"name\":\"Bob\", \"test\": \"yes\"}") + jsonPath2 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") + writeLines(mockLines2, jsonPath2) + df2 <- read.json(jsonPath2) + + # inner join, not cartesian join + expect_equal(count(where(join(df, df2), df$name == df2$name)), 3) + # cartesian join + expect_error(tryCatch(count(join(df, df2)), error = function(e) { stop(e) }), + paste0(".*(org.apache.spark.sql.AnalysisException: Detected cartesian product for", + " INNER join between logical plans).*")) + + joined <- crossJoin(df, df2) + expect_equal(names(joined), c("age", "name", "name", "test")) + expect_equal(count(joined), 12) + expect_equal(names(collect(joined)), c("age", "name", "name", "test")) + + joined2 <- join(df, df2, df$name == df2$name) + expect_equal(names(joined2), c("age", "name", "name", "test")) + expect_equal(count(joined2), 3) + + joined3 <- join(df, df2, df$name == df2$name, "rightouter") + expect_equal(names(joined3), c("age", "name", "name", "test")) + expect_equal(count(joined3), 4) + expect_true(is.na(collect(orderBy(joined3, joined3$age))$age[2])) + + joined4 <- select(join(df, df2, df$name == df2$name, "outer"), + alias(df$age + 5, "newAge"), df$name, df2$test) + expect_equal(names(joined4), c("newAge", "name", "test")) + expect_equal(count(joined4), 4) + expect_equal(collect(orderBy(joined4, joined4$name))$newAge[3], 24) + + joined5 <- join(df, df2, df$name == df2$name, "leftouter") + expect_equal(names(joined5), c("age", "name", "name", "test")) + expect_equal(count(joined5), 3) + expect_true(is.na(collect(orderBy(joined5, joined5$age))$age[1])) + + joined6 <- join(df, df2, df$name == df2$name, "inner") + expect_equal(names(joined6), c("age", "name", "name", "test")) + expect_equal(count(joined6), 3) + + joined7 <- join(df, df2, df$name == df2$name, "leftsemi") + expect_equal(names(joined7), c("age", "name")) + expect_equal(count(joined7), 3) + + joined8 <- join(df, df2, df$name == df2$name, "left_outer") + expect_equal(names(joined8), c("age", "name", "name", "test")) + expect_equal(count(joined8), 3) + expect_true(is.na(collect(orderBy(joined8, joined8$age))$age[1])) + + joined9 <- join(df, df2, df$name == df2$name, "right_outer") + expect_equal(names(joined9), c("age", "name", "name", "test")) + expect_equal(count(joined9), 4) + expect_true(is.na(collect(orderBy(joined9, joined9$age))$age[2])) + + merged <- merge(df, df2, by.x = "name", by.y = "name", all.x = TRUE, all.y = TRUE) + expect_equal(count(merged), 4) + expect_equal(names(merged), c("age", "name_x", "name_y", "test")) + expect_equal(collect(orderBy(merged, merged$name_x))$age[3], 19) + + merged <- merge(df, df2, suffixes = c("-X", "-Y")) + expect_equal(count(merged), 3) + expect_equal(names(merged), c("age", "name-X", "name-Y", "test")) + expect_equal(collect(orderBy(merged, merged$"name-X"))$age[1], 30) + + merged <- merge(df, df2, by = "name", suffixes = c("-X", "-Y"), sort = FALSE) + expect_equal(count(merged), 3) + expect_equal(names(merged), c("age", "name-X", "name-Y", "test")) + expect_equal(collect(orderBy(merged, merged$"name-Y"))$"name-X"[3], "Michael") + + merged <- merge(df, df2, by = "name", all = T, sort = T) + expect_equal(count(merged), 4) + expect_equal(names(merged), c("age", "name_x", "name_y", "test")) + expect_equal(collect(orderBy(merged, merged$"name_y"))$"name_x"[1], "Andy") + + merged <- merge(df, df2, by = NULL) + expect_equal(count(merged), 12) + expect_equal(names(merged), c("age", "name", "name", "test")) + + mockLines3 <- c("{\"name\":\"Michael\", \"name_y\":\"Michael\", \"test\": \"yes\"}", + "{\"name\":\"Andy\", \"name_y\":\"Andy\", \"test\": \"no\"}", + "{\"name\":\"Justin\", \"name_y\":\"Justin\", \"test\": \"yes\"}", + "{\"name\":\"Bob\", \"name_y\":\"Bob\", \"test\": \"yes\"}") + jsonPath3 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") + writeLines(mockLines3, jsonPath3) + df3 <- read.json(jsonPath3) + expect_error(merge(df, df3), + paste("The following column name: name_y occurs more than once in the 'DataFrame'.", + "Please use different suffixes for the intersected columns.", sep = "")) + + unlink(jsonPath2) + unlink(jsonPath3) + + # Join with broadcast hint + df1 <- sql("SELECT * FROM range(10e10)") + df2 <- sql("SELECT * FROM range(10e10)") + + execution_plan <- capture.output(explain(join(df1, df2, df1$id == df2$id))) + expect_false(any(grepl("BroadcastHashJoin", execution_plan))) + + execution_plan_hint <- capture.output( + explain(join(df1, hint(df2, "broadcast"), df1$id == df2$id)) + ) + expect_true(any(grepl("BroadcastHashJoin", execution_plan_hint))) + + execution_plan_broadcast <- capture.output( + explain(join(df1, broadcast(df2), df1$id == df2$id)) + ) + expect_true(any(grepl("BroadcastHashJoin", execution_plan_broadcast))) +}) + +test_that("toJSON() on DataFrame", { + df <- as.DataFrame(cars) + df_json <- toJSON(df) + expect_is(df_json, "SparkDataFrame") + expect_equal(colnames(df_json), c("value")) + expect_equal(head(df_json, 1), + data.frame(value = "{\"speed\":4.0,\"dist\":2.0}", stringsAsFactors = FALSE)) +}) + +test_that("showDF()", { + df <- read.json(jsonPath) + expected <- paste("+----+-------+\n", + "| age| name|\n", + "+----+-------+\n", + "|null|Michael|\n", + "| 30| Andy|\n", + "| 19| Justin|\n", + "+----+-------+\n", sep = "") + expected2 <- paste("+---+----+\n", + "|age|name|\n", + "+---+----+\n", + "|nul| Mic|\n", + "| 30| And|\n", + "| 19| Jus|\n", + "+---+----+\n", sep = "") + expect_output(showDF(df), expected) + expect_output(showDF(df, truncate = 3), expected2) +}) + +test_that("isLocal()", { + df <- read.json(jsonPath) + expect_false(isLocal(df)) +}) + +test_that("union(), unionByName(), rbind(), except(), and intersect() on a DataFrame", { + df <- read.json(jsonPath) + + lines <- c("{\"name\":\"Bob\", \"age\":24}", + "{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"James\", \"age\":35}") + jsonPath2 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") + writeLines(lines, jsonPath2) + df2 <- read.df(jsonPath2, "json") + + unioned <- arrange(union(df, df2), df$age) + expect_is(unioned, "SparkDataFrame") + expect_equal(count(unioned), 6) + expect_equal(first(unioned)$name, "Michael") + expect_equal(count(arrange(suppressWarnings(unionAll(df, df2)), df$age)), 6) + + df1 <- select(df2, "age", "name") + unioned1 <- arrange(unionByName(df1, df), df1$age) + expect_is(unioned, "SparkDataFrame") + expect_equal(count(unioned), 6) + # Here, we test if 'Michael' in df is correctly mapped to the same name. + expect_equal(first(unioned)$name, "Michael") + + unioned2 <- arrange(rbind(unioned, df, df2), df$age) + expect_is(unioned2, "SparkDataFrame") + expect_equal(count(unioned2), 12) + expect_equal(first(unioned2)$name, "Michael") + + df3 <- df2 + names(df3)[1] <- "newName" + expect_error(rbind(df, df3), + "Names of input data frames are different.") + expect_error(rbind(df, df2, df3), + "Names of input data frames are different.") + + excepted <- arrange(except(df, df2), desc(df$age)) + expect_is(unioned, "SparkDataFrame") + expect_equal(count(excepted), 2) + expect_equal(first(excepted)$name, "Justin") + + intersected <- arrange(intersect(df, df2), df$age) + expect_is(unioned, "SparkDataFrame") + expect_equal(count(intersected), 1) + expect_equal(first(intersected)$name, "Andy") + + # Test base::union is working + expect_equal(union(c(1:3), c(3:5)), c(1:5)) + + # Test base::rbind is working + expect_equal(length(rbind(1:4, c = 2, a = 10, 10, deparse.level = 0)), 16) + + # Test base::intersect is working + expect_equal(length(intersect(1:20, 3:23)), 18) + + unlink(jsonPath2) +}) + +test_that("withColumn() and withColumnRenamed()", { + df <- read.json(jsonPath) + newDF <- withColumn(df, "newAge", df$age + 2) + expect_equal(length(columns(newDF)), 3) + expect_equal(columns(newDF)[3], "newAge") + expect_equal(first(filter(newDF, df$name != "Michael"))$newAge, 32) + + # Replace existing column + newDF <- withColumn(df, "age", df$age + 2) + expect_equal(length(columns(newDF)), 2) + expect_equal(first(filter(newDF, df$name != "Michael"))$age, 32) + + newDF <- withColumn(df, "age", 18) + expect_equal(length(columns(newDF)), 2) + expect_equal(first(newDF)$age, 18) + + expect_error(withColumn(df, "age", list("a")), + "Literal value must be atomic in length of 1") + + newDF2 <- withColumnRenamed(df, "age", "newerAge") + expect_equal(length(columns(newDF2)), 2) + expect_equal(columns(newDF2)[1], "newerAge") +}) + +test_that("mutate(), transform(), rename() and names()", { + df <- read.json(jsonPath) + newDF <- mutate(df, newAge = df$age + 2) + expect_equal(length(columns(newDF)), 3) + expect_equal(columns(newDF)[3], "newAge") + expect_equal(first(filter(newDF, df$name != "Michael"))$newAge, 32) + + newDF <- mutate(df, age = df$age + 2, newAge = df$age + 3) + expect_equal(length(columns(newDF)), 3) + expect_equal(columns(newDF)[3], "newAge") + expect_equal(first(filter(newDF, df$name != "Michael"))$newAge, 33) + expect_equal(first(filter(newDF, df$name != "Michael"))$age, 32) + + newDF <- mutate(df, age = df$age + 2, newAge = df$age + 3, + age = df$age + 4, newAge = df$age + 5) + expect_equal(length(columns(newDF)), 3) + expect_equal(columns(newDF)[3], "newAge") + expect_equal(first(filter(newDF, df$name != "Michael"))$newAge, 35) + expect_equal(first(filter(newDF, df$name != "Michael"))$age, 34) + + newDF <- mutate(df, df$age + 3) + expect_equal(length(columns(newDF)), 3) + expect_equal(columns(newDF)[[3]], "df$age + 3") + expect_equal(first(filter(newDF, df$name != "Michael"))[[3]], 33) + + newDF2 <- rename(df, newerAge = df$age) + expect_equal(length(columns(newDF2)), 2) + expect_equal(columns(newDF2)[1], "newerAge") + + names(newDF2) <- c("newerName", "evenNewerAge") + expect_equal(length(names(newDF2)), 2) + expect_equal(names(newDF2)[1], "newerName") + + transformedDF <- transform(df, newAge = -df$age, newAge2 = df$age / 2) + expect_equal(length(columns(transformedDF)), 4) + expect_equal(columns(transformedDF)[3], "newAge") + expect_equal(columns(transformedDF)[4], "newAge2") + expect_equal(first(filter(transformedDF, transformedDF$name == "Andy"))$newAge, -30) + + # test if base::transform on local data frames works + # ensure the proper signature is used - otherwise this will fail to run + attach(airquality) + result <- transform(Ozone, logOzone = log(Ozone)) + expect_equal(nrow(result), 153) + expect_equal(ncol(result), 2) + detach(airquality) +}) + +test_that("read/write ORC files", { + setHiveContext(sc) + df <- read.df(jsonPath, "json") + + # Test write.df and read.df + write.df(df, orcPath, "orc", mode = "overwrite") + df2 <- read.df(orcPath, "orc") + expect_is(df2, "SparkDataFrame") + expect_equal(count(df), count(df2)) + + # Test write.orc and read.orc + orcPath2 <- tempfile(pattern = "orcPath2", fileext = ".orc") + write.orc(df, orcPath2) + orcDF <- read.orc(orcPath2) + expect_is(orcDF, "SparkDataFrame") + expect_equal(count(orcDF), count(df)) + + unlink(orcPath2) + unsetHiveContext() +}) + +test_that("read/write ORC files - compression option", { + setHiveContext(sc) + df <- read.df(jsonPath, "json") + + orcPath2 <- tempfile(pattern = "orcPath2", fileext = ".orc") + write.orc(df, orcPath2, compression = "ZLIB") + orcDF <- read.orc(orcPath2) + expect_is(orcDF, "SparkDataFrame") + expect_equal(count(orcDF), count(df)) + expect_true(length(list.files(orcPath2, pattern = ".zlib.orc")) > 0) + + unlink(orcPath2) + unsetHiveContext() +}) + +test_that("read/write Parquet files", { + if (windows_with_hadoop()) { + df <- read.df(jsonPath, "json") + # Test write.df and read.df + write.df(df, parquetPath, "parquet", mode = "overwrite") + df2 <- read.df(parquetPath, "parquet") + expect_is(df2, "SparkDataFrame") + expect_equal(count(df2), 3) + + # Test write.parquet/saveAsParquetFile and read.parquet/parquetFile + parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet") + write.parquet(df, parquetPath2) + parquetPath3 <- tempfile(pattern = "parquetPath3", fileext = ".parquet") + suppressWarnings(saveAsParquetFile(df, parquetPath3)) + parquetDF <- read.parquet(c(parquetPath2, parquetPath3)) + expect_is(parquetDF, "SparkDataFrame") + expect_equal(count(parquetDF), count(df) * 2) + parquetDF2 <- suppressWarnings(parquetFile(parquetPath2, parquetPath3)) + expect_is(parquetDF2, "SparkDataFrame") + expect_equal(count(parquetDF2), count(df) * 2) + + # Test if varargs works with variables + saveMode <- "overwrite" + mergeSchema <- "true" + parquetPath4 <- tempfile(pattern = "parquetPath3", fileext = ".parquet") + write.df(df, parquetPath3, "parquet", mode = saveMode, mergeSchema = mergeSchema) + + unlink(parquetPath2) + unlink(parquetPath3) + unlink(parquetPath4) + } +}) + +test_that("read/write Parquet files - compression option/mode", { + df <- read.df(jsonPath, "json") + tempPath <- tempfile(pattern = "tempPath", fileext = ".parquet") + + # Test write.df and read.df + write.parquet(df, tempPath, compression = "GZIP") + df2 <- read.parquet(tempPath) + expect_is(df2, "SparkDataFrame") + expect_equal(count(df2), 3) + expect_true(length(list.files(tempPath, pattern = ".gz.parquet")) > 0) + + write.parquet(df, tempPath, mode = "overwrite") + df3 <- read.parquet(tempPath) + expect_is(df3, "SparkDataFrame") + expect_equal(count(df3), 3) +}) + +test_that("read/write text files", { + # Test write.df and read.df + df <- read.df(jsonPath, "text") + expect_is(df, "SparkDataFrame") + expect_equal(colnames(df), c("value")) + expect_equal(count(df), 3) + textPath <- tempfile(pattern = "textPath", fileext = ".txt") + write.df(df, textPath, "text", mode = "overwrite") + + # Test write.text and read.text + textPath2 <- tempfile(pattern = "textPath2", fileext = ".txt") + write.text(df, textPath2) + df2 <- read.text(c(textPath, textPath2)) + expect_is(df2, "SparkDataFrame") + expect_equal(colnames(df2), c("value")) + expect_equal(count(df2), count(df) * 2) + + unlink(textPath) + unlink(textPath2) +}) + +test_that("read/write text files - compression option", { + df <- read.df(jsonPath, "text") + + textPath <- tempfile(pattern = "textPath", fileext = ".txt") + write.text(df, textPath, compression = "GZIP") + textDF <- read.text(textPath) + expect_is(textDF, "SparkDataFrame") + expect_equal(count(textDF), count(df)) + expect_true(length(list.files(textPath, pattern = ".gz")) > 0) + + unlink(textPath) +}) + +test_that("describe() and summary() on a DataFrame", { + df <- read.json(jsonPath) + stats <- describe(df, "age") + expect_equal(collect(stats)[1, "summary"], "count") + expect_equal(collect(stats)[2, "age"], "24.5") + expect_equal(collect(stats)[3, "age"], "7.7781745930520225") + stats <- describe(df) + expect_equal(collect(stats)[4, "summary"], "min") + expect_equal(collect(stats)[5, "age"], "30") + + stats2 <- summary(df) + expect_equal(collect(stats2)[5, "summary"], "25%") + expect_equal(collect(stats2)[5, "age"], "30") + + stats3 <- summary(df, "min", "max", "55.1%") + + expect_equal(collect(stats3)[1, "summary"], "min") + expect_equal(collect(stats3)[2, "summary"], "max") + expect_equal(collect(stats3)[3, "summary"], "55.1%") + expect_equal(collect(stats3)[3, "age"], "30") + + # SPARK-16425: SparkR summary() fails on column of type logical + df <- withColumn(df, "boolean", df$age == 30) + summary(df) + + # Test base::summary is working + expect_equal(length(summary(attenu, digits = 4)), 35) +}) + +test_that("dropna() and na.omit() on a DataFrame", { + df <- read.json(jsonPathNa) + rows <- collect(df) + + # drop with columns + + expected <- rows[!is.na(rows$name), ] + actual <- collect(dropna(df, cols = "name")) + expect_identical(expected, actual) + actual <- collect(na.omit(df, cols = "name")) + expect_identical(expected, actual) + + expected <- rows[!is.na(rows$age), ] + actual <- collect(dropna(df, cols = "age")) + row.names(expected) <- row.names(actual) + # identical on two dataframes does not work here. Don't know why. + # use identical on all columns as a workaround. + expect_identical(expected$age, actual$age) + expect_identical(expected$height, actual$height) + expect_identical(expected$name, actual$name) + actual <- collect(na.omit(df, cols = "age")) + + expected <- rows[!is.na(rows$age) & !is.na(rows$height), ] + actual <- collect(dropna(df, cols = c("age", "height"))) + expect_identical(expected, actual) + actual <- collect(na.omit(df, cols = c("age", "height"))) + expect_identical(expected, actual) + + expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name), ] + actual <- collect(dropna(df)) + expect_identical(expected, actual) + actual <- collect(na.omit(df)) + expect_identical(expected, actual) + + # drop with how + + expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name), ] + actual <- collect(dropna(df)) + expect_identical(expected, actual) + actual <- collect(na.omit(df)) + expect_identical(expected, actual) + + expected <- rows[!is.na(rows$age) | !is.na(rows$height) | !is.na(rows$name), ] + actual <- collect(dropna(df, "all")) + expect_identical(expected, actual) + actual <- collect(na.omit(df, "all")) + expect_identical(expected, actual) + + expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name), ] + actual <- collect(dropna(df, "any")) + expect_identical(expected, actual) + actual <- collect(na.omit(df, "any")) + expect_identical(expected, actual) + + expected <- rows[!is.na(rows$age) & !is.na(rows$height), ] + actual <- collect(dropna(df, "any", cols = c("age", "height"))) + expect_identical(expected, actual) + actual <- collect(na.omit(df, "any", cols = c("age", "height"))) + expect_identical(expected, actual) + + expected <- rows[!is.na(rows$age) | !is.na(rows$height), ] + actual <- collect(dropna(df, "all", cols = c("age", "height"))) + expect_identical(expected, actual) + actual <- collect(na.omit(df, "all", cols = c("age", "height"))) + expect_identical(expected, actual) + + # drop with threshold + + expected <- rows[as.integer(!is.na(rows$age)) + as.integer(!is.na(rows$height)) >= 2, ] + actual <- collect(dropna(df, minNonNulls = 2, cols = c("age", "height"))) + expect_identical(expected, actual) + actual <- collect(na.omit(df, minNonNulls = 2, cols = c("age", "height"))) + expect_identical(expected, actual) + + expected <- rows[as.integer(!is.na(rows$age)) + + as.integer(!is.na(rows$height)) + + as.integer(!is.na(rows$name)) >= 3, ] + actual <- collect(dropna(df, minNonNulls = 3, cols = c("name", "age", "height"))) + expect_identical(expected, actual) + actual <- collect(na.omit(df, minNonNulls = 3, cols = c("name", "age", "height"))) + expect_identical(expected, actual) + + # Test stats::na.omit is working + expect_equal(nrow(na.omit(data.frame(x = c(0, 10, NA)))), 2) +}) + +test_that("fillna() on a DataFrame", { + df <- read.json(jsonPathNa) + rows <- collect(df) + + # fill with value + + expected <- rows + expected$age[is.na(expected$age)] <- 50 + expected$height[is.na(expected$height)] <- 50.6 + actual <- collect(fillna(df, 50.6)) + expect_identical(expected, actual) + + expected <- rows + expected$name[is.na(expected$name)] <- "unknown" + actual <- collect(fillna(df, "unknown")) + expect_identical(expected, actual) + + expected <- rows + expected$age[is.na(expected$age)] <- 50 + actual <- collect(fillna(df, 50.6, "age")) + expect_identical(expected, actual) + + expected <- rows + expected$name[is.na(expected$name)] <- "unknown" + actual <- collect(fillna(df, "unknown", c("age", "name"))) + expect_identical(expected, actual) + + # fill with named list + + expected <- rows + expected$age[is.na(expected$age)] <- 50 + expected$height[is.na(expected$height)] <- 50.6 + expected$name[is.na(expected$name)] <- "unknown" + actual <- collect(fillna(df, list("age" = 50, "height" = 50.6, "name" = "unknown"))) + expect_identical(expected, actual) +}) + +test_that("crosstab() on a DataFrame", { + rdd <- lapply(parallelize(sc, 0:3), function(x) { + list(paste0("a", x %% 3), paste0("b", x %% 2)) + }) + df <- toDF(rdd, list("a", "b")) + ct <- crosstab(df, "a", "b") + ordered <- ct[order(ct$a_b), ] + row.names(ordered) <- NULL + expected <- data.frame("a_b" = c("a0", "a1", "a2"), "b0" = c(1, 0, 1), "b1" = c(1, 1, 0), + stringsAsFactors = FALSE, row.names = NULL) + expect_identical(expected, ordered) +}) + +test_that("cov() and corr() on a DataFrame", { + l <- lapply(c(0:9), function(x) { list(x, x * 2.0) }) + df <- createDataFrame(l, c("singles", "doubles")) + result <- cov(df, "singles", "doubles") + expect_true(abs(result - 55.0 / 3) < 1e-12) + + result <- corr(df, "singles", "doubles") + expect_true(abs(result - 1.0) < 1e-12) + result <- corr(df, "singles", "doubles", "pearson") + expect_true(abs(result - 1.0) < 1e-12) + + # Test stats::cov is working + #expect_true(abs(max(cov(swiss)) - 1739.295) < 1e-3) # nolint +}) + +test_that("freqItems() on a DataFrame", { + input <- 1:1000 + rdf <- data.frame(numbers = input, letters = as.character(input), + negDoubles = input * -1.0, stringsAsFactors = F) + rdf[ input %% 3 == 0, ] <- c(1, "1", -1) + df <- createDataFrame(rdf) + multiColResults <- freqItems(df, c("numbers", "letters"), support = 0.1) + expect_true(1 %in% multiColResults$numbers[[1]]) + expect_true("1" %in% multiColResults$letters[[1]]) + singleColResult <- freqItems(df, "negDoubles", support = 0.1) + expect_true(-1 %in% head(singleColResult$negDoubles)[[1]]) + + l <- lapply(c(0:99), function(i) { + if (i %% 2 == 0) { list(1L, -1.0) } + else { list(i, i * -1.0) }}) + df <- createDataFrame(l, c("a", "b")) + result <- freqItems(df, c("a", "b"), 0.4) + expect_identical(result[[1]], list(list(1L, 99L))) + expect_identical(result[[2]], list(list(-1, -99))) +}) + +test_that("sampleBy() on a DataFrame", { + l <- lapply(c(0:99), function(i) { as.character(i %% 3) }) + df <- createDataFrame(l, "key") + fractions <- list("0" = 0.1, "1" = 0.2) + sample <- sampleBy(df, "key", fractions, 0) + result <- collect(orderBy(count(groupBy(sample, "key")), "key")) + expect_identical(as.list(result[1, ]), list(key = "0", count = 3)) + expect_identical(as.list(result[2, ]), list(key = "1", count = 7)) +}) + +test_that("approxQuantile() on a DataFrame", { + l <- lapply(c(0:99), function(i) { list(i, 99 - i) }) + df <- createDataFrame(l, list("a", "b")) + quantiles <- approxQuantile(df, "a", c(0.5, 0.8), 0.0) + expect_equal(quantiles, list(50, 80)) + quantiles2 <- approxQuantile(df, c("a", "b"), c(0.5, 0.8), 0.0) + expect_equal(quantiles2[[1]], list(50, 80)) + expect_equal(quantiles2[[2]], list(50, 80)) + + dfWithNA <- createDataFrame(data.frame(a = c(NA, 30, 19, 11, 28, 15), + b = c(-30, -19, NA, -11, -28, -15))) + quantiles3 <- approxQuantile(dfWithNA, c("a", "b"), c(0.5), 0.0) + expect_equal(quantiles3[[1]], list(28)) + expect_equal(quantiles3[[2]], list(-15)) +}) + +test_that("SQL error message is returned from JVM", { + retError <- tryCatch(sql("select * from blah"), error = function(e) e) + expect_equal(grepl("Table or view not found", retError), TRUE) + expect_equal(grepl("blah", retError), TRUE) +}) + +irisDF <- suppressWarnings(createDataFrame(iris)) + +test_that("Method as.data.frame as a synonym for collect()", { + expect_equal(as.data.frame(irisDF), collect(irisDF)) + irisDF2 <- irisDF[irisDF$Species == "setosa", ] + expect_equal(as.data.frame(irisDF2), collect(irisDF2)) + + # Make sure as.data.frame in the R base package is not covered + expect_error(as.data.frame(c(1, 2)), NA) +}) + +test_that("attach() on a DataFrame", { + df <- read.json(jsonPath) + expect_error(age) + attach(df) + expect_is(age, "SparkDataFrame") + expected_age <- data.frame(age = c(NA, 30, 19)) + expect_equal(head(age), expected_age) + stat <- summary(age) + expect_equal(collect(stat)[8, "age"], "30") + age <- age$age + 1 + expect_is(age, "Column") + rm(age) + stat2 <- summary(age) + expect_equal(collect(stat2)[8, "age"], "30") + detach("df") + stat3 <- summary(df[, "age", drop = F]) + expect_equal(collect(stat3)[8, "age"], "30") + expect_error(age) +}) + +test_that("with() on a DataFrame", { + df <- suppressWarnings(createDataFrame(iris)) + expect_error(Sepal_Length) + sum1 <- with(df, list(summary(Sepal_Length), summary(Sepal_Width))) + expect_equal(collect(sum1[[1]])[1, "Sepal_Length"], "150") + sum2 <- with(df, distinct(Sepal_Length)) + expect_equal(nrow(sum2), 35) +}) + +test_that("Method coltypes() to get and set R's data types of a DataFrame", { + expect_equal(coltypes(irisDF), c(rep("numeric", 4), "character")) + + data <- data.frame(c1 = c(1, 2, 3), + c2 = c(T, F, T), + c3 = c("2015/01/01 10:00:00", "2015/01/02 10:00:00", "2015/01/03 10:00:00")) + + schema <- structType(structField("c1", "byte"), + structField("c3", "boolean"), + structField("c4", "timestamp")) + + # Test primitive types + DF <- createDataFrame(data, schema) + expect_equal(coltypes(DF), c("integer", "logical", "POSIXct")) + createOrReplaceTempView(DF, "DFView") + sqlCast <- sql("select cast('2' as decimal) as x from DFView limit 1") + expect_equal(coltypes(sqlCast), "numeric") + + # Test complex types + x <- createDataFrame(list(list(as.environment( + list("a" = "b", "c" = "d", "e" = "f"))))) + expect_equal(coltypes(x), "map") + + df <- selectExpr(read.json(jsonPath), "name", "(age * 1.21) as age") + expect_equal(dtypes(df), list(c("name", "string"), c("age", "decimal(24,2)"))) + + df1 <- select(df, cast(df$age, "integer")) + coltypes(df) <- c("character", "integer") + expect_equal(dtypes(df), list(c("name", "string"), c("age", "int"))) + value <- collect(df[, 2, drop = F])[[3, 1]] + expect_equal(value, collect(df1)[[3, 1]]) + expect_equal(value, 22) + + coltypes(df) <- c(NA, "numeric") + expect_equal(dtypes(df), list(c("name", "string"), c("age", "double"))) + + expect_error(coltypes(df) <- c("character"), + "Length of type vector should match the number of columns for SparkDataFrame") + expect_error(coltypes(df) <- c("environment", "list"), + "Only atomic type is supported for column types") +}) + +test_that("Method str()", { + # Structure of Iris + iris2 <- iris + colnames(iris2) <- c("Sepal_Length", "Sepal_Width", "Petal_Length", "Petal_Width", "Species") + iris2$col <- TRUE + irisDF2 <- createDataFrame(iris2) + + out <- capture.output(str(irisDF2)) + expect_equal(length(out), 7) + expect_equal(out[1], "'SparkDataFrame': 6 variables:") + expect_equal(out[2], " $ Sepal_Length: num 5.1 4.9 4.7 4.6 5 5.4") + expect_equal(out[3], " $ Sepal_Width : num 3.5 3 3.2 3.1 3.6 3.9") + expect_equal(out[4], " $ Petal_Length: num 1.4 1.4 1.3 1.5 1.4 1.7") + expect_equal(out[5], " $ Petal_Width : num 0.2 0.2 0.2 0.2 0.2 0.4") + expect_equal(out[6], paste0(" $ Species : chr \"setosa\" \"setosa\" \"", + "setosa\" \"setosa\" \"setosa\" \"setosa\"")) + expect_equal(out[7], " $ col : logi TRUE TRUE TRUE TRUE TRUE TRUE") + + createOrReplaceTempView(irisDF2, "irisView") + + sqlCast <- sql("select cast('2' as decimal) as x from irisView limit 1") + castStr <- capture.output(str(sqlCast)) + expect_equal(length(castStr), 2) + expect_equal(castStr[1], "'SparkDataFrame': 1 variables:") + expect_equal(castStr[2], " $ x: num 2") + + # A random dataset with many columns. This test is to check str limits + # the number of columns. Therefore, it will suffice to check for the + # number of returned rows + x <- runif(200, 1, 10) + df <- data.frame(t(as.matrix(data.frame(x, x, x, x, x, x, x, x, x)))) + DF <- createDataFrame(df) + out <- capture.output(str(DF)) + expect_equal(length(out), 103) + + # Test utils:::str + expect_equal(capture.output(utils:::str(iris)), capture.output(str(iris))) +}) + +test_that("Histogram", { + + # Basic histogram test with colname + expect_equal( + all(histogram(irisDF, "Petal_Width", 8) == + data.frame(bins = seq(0, 7), + counts = c(48, 2, 7, 21, 24, 19, 15, 14), + centroids = seq(0, 7) * 0.3 + 0.25)), + TRUE) + + # Basic histogram test with Column + expect_equal( + all(histogram(irisDF, irisDF$Petal_Width, 8) == + data.frame(bins = seq(0, 7), + counts = c(48, 2, 7, 21, 24, 19, 15, 14), + centroids = seq(0, 7) * 0.3 + 0.25)), + TRUE) + + # Basic histogram test with derived column + expect_equal( + all(round(histogram(irisDF, irisDF$Petal_Width + 1, 8), 2) == + data.frame(bins = seq(0, 7), + counts = c(48, 2, 7, 21, 24, 19, 15, 14), + centroids = seq(0, 7) * 0.3 + 1.25)), + TRUE) + + # Missing nbins + expect_equal(length(histogram(irisDF, "Petal_Width")$counts), 10) + + # Wrong colname + expect_error(histogram(irisDF, "xxx"), + "Specified colname does not belong to the given SparkDataFrame.") + + # Invalid nbins + expect_error(histogram(irisDF, "Petal_Width", nbins = 0), + "The number of bins must be a positive integer number greater than 1.") + + # Test against R's hist + expect_equal(all(hist(iris$Sepal.Width)$counts == + histogram(irisDF, "Sepal_Width", 12)$counts), T) + + # Test when there are zero counts + df <- as.DataFrame(data.frame(x = c(1, 2, 3, 4, 100))) + expect_equal(histogram(df, "x")$counts, c(4, 0, 0, 0, 0, 0, 0, 0, 0, 1)) +}) + +test_that("dapply() and dapplyCollect() on a DataFrame", { + df <- createDataFrame( + list(list(1L, 1, "1"), list(2L, 2, "2"), list(3L, 3, "3")), + c("a", "b", "c")) + ldf <- collect(df) + df1 <- dapply(df, function(x) { x }, schema(df)) + result <- collect(df1) + expect_identical(ldf, result) + + result <- dapplyCollect(df, function(x) { x }) + expect_identical(ldf, result) + + # Filter and add a column + schemas <- list(structType(structField("a", "integer"), structField("b", "double"), + structField("c", "string"), structField("d", "integer")), + "a INT, b DOUBLE, c STRING, d INT") + for (schema in schemas) { + df1 <- dapply( + df, + function(x) { + y <- x[x$a > 1, ] + y <- cbind(y, y$a + 1L) + }, + schema) + result <- collect(df1) + expected <- ldf[ldf$a > 1, ] + expected$d <- expected$a + 1L + rownames(expected) <- NULL + expect_identical(expected, result) + + result <- dapplyCollect( + df, + function(x) { + y <- x[x$a > 1, ] + y <- cbind(y, y$a + 1L) + }) + expected1 <- expected + names(expected1) <- names(result) + expect_identical(expected1, result) + } + + # Remove the added column + df2 <- dapply( + df1, + function(x) { + x[, c("a", "b", "c")] + }, + schema(df)) + result <- collect(df2) + expected <- expected[, c("a", "b", "c")] + expect_identical(expected, result) + + result <- dapplyCollect( + df1, + function(x) { + x[, c("a", "b", "c")] + }) + expect_identical(expected, result) +}) + +test_that("dapplyCollect() on DataFrame with a binary column", { + df <- data.frame(key = 1:3) + df$bytes <- lapply(df$key, serialize, connection = NULL) + + df_spark <- createDataFrame(df) + + result1 <- collect(df_spark) + expect_identical(df, result1) + + result2 <- dapplyCollect(df_spark, function(x) x) + expect_identical(df, result2) + + # A data.frame with a single column of bytes + scb <- subset(df, select = "bytes") + scb_spark <- createDataFrame(scb) + result <- dapplyCollect(scb_spark, function(x) x) + expect_identical(scb, result) + +}) + +test_that("repartition by columns on DataFrame", { + df <- createDataFrame( + list(list(1L, 1, "1", 0.1), list(1L, 2, "2", 0.2), list(3L, 3, "3", 0.3)), + c("a", "b", "c", "d")) + + # no column and number of partitions specified + retError <- tryCatch(repartition(df), error = function(e) e) + expect_equal(grepl + ("Please, specify the number of partitions and/or a column\\(s\\)", retError), TRUE) + + # repartition by column and number of partitions + actual <- repartition(df, 3, col = df$"a") + + # Checking that at least the dimensions are identical + expect_identical(dim(df), dim(actual)) + expect_equal(getNumPartitions(actual), 3L) + + # repartition by number of partitions + actual <- repartition(df, 13L) + expect_identical(dim(df), dim(actual)) + expect_equal(getNumPartitions(actual), 13L) + + expect_equal(getNumPartitions(coalesce(actual, 1L)), 1L) + + # a test case with a column and dapply + schema <- structType(structField("a", "integer"), structField("avg", "double")) + df <- repartition(df, col = df$"a") + df1 <- dapply( + df, + function(x) { + y <- (data.frame(x$a[1], mean(x$b))) + }, + schema) + + # Number of partitions is equal to 2 + expect_equal(nrow(df1), 2) +}) + +test_that("coalesce, repartition, numPartitions", { + df <- as.DataFrame(cars, numPartitions = 5) + expect_equal(getNumPartitions(df), 5) + expect_equal(getNumPartitions(coalesce(df, 3)), 3) + expect_equal(getNumPartitions(coalesce(df, 6)), 5) + + df1 <- coalesce(df, 3) + expect_equal(getNumPartitions(df1), 3) + expect_equal(getNumPartitions(coalesce(df1, 6)), 5) + expect_equal(getNumPartitions(coalesce(df1, 4)), 4) + expect_equal(getNumPartitions(coalesce(df1, 2)), 2) + + df2 <- repartition(df1, 10) + expect_equal(getNumPartitions(df2), 10) + expect_equal(getNumPartitions(coalesce(df2, 13)), 10) + expect_equal(getNumPartitions(coalesce(df2, 7)), 7) + expect_equal(getNumPartitions(coalesce(df2, 3)), 3) +}) + +test_that("gapply() and gapplyCollect() on a DataFrame", { + df <- createDataFrame ( + list(list(1L, 1, "1", 0.1), list(1L, 2, "1", 0.2), list(3L, 3, "3", 0.3)), + c("a", "b", "c", "d")) + expected <- collect(df) + df1 <- gapply(df, "a", function(key, x) { x }, schema(df)) + actual <- collect(df1) + expect_identical(actual, expected) + + df1Collect <- gapplyCollect(df, list("a"), function(key, x) { x }) + expect_identical(df1Collect, expected) + + # Computes the sum of second column by grouping on the first and third columns + # and checks if the sum is larger than 2 + schemas <- list(structType(structField("a", "integer"), structField("e", "boolean")), + "a INT, e BOOLEAN") + for (schema in schemas) { + df2 <- gapply( + df, + c(df$"a", df$"c"), + function(key, x) { + y <- data.frame(key[1], sum(x$b) > 2) + }, + schema) + actual <- collect(df2)$e + expected <- c(TRUE, TRUE) + expect_identical(actual, expected) + + df2Collect <- gapplyCollect( + df, + c(df$"a", df$"c"), + function(key, x) { + y <- data.frame(key[1], sum(x$b) > 2) + colnames(y) <- c("a", "e") + y + }) + actual <- df2Collect$e + expect_identical(actual, expected) + } + + # Computes the arithmetic mean of the second column by grouping + # on the first and third columns. Output the groupping value and the average. + schema <- structType(structField("a", "integer"), structField("c", "string"), + structField("avg", "double")) + df3 <- gapply( + df, + c("a", "c"), + function(key, x) { + y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) + }, + schema) + actual <- collect(df3) + actual <- actual[order(actual$a), ] + rownames(actual) <- NULL + expected <- collect(select(df, "a", "b", "c")) + expected <- data.frame(aggregate(expected$b, by = list(expected$a, expected$c), FUN = mean)) + colnames(expected) <- c("a", "c", "avg") + expected <- expected[order(expected$a), ] + rownames(expected) <- NULL + expect_identical(actual, expected) + + df3Collect <- gapplyCollect( + df, + c("a", "c"), + function(key, x) { + y <- data.frame(key, mean(x$b), stringsAsFactors = FALSE) + colnames(y) <- c("a", "c", "avg") + y + }) + actual <- df3Collect[order(df3Collect$a), ] + expect_identical(actual$avg, expected$avg) + + irisDF <- suppressWarnings(createDataFrame (iris)) + schema <- structType(structField("Sepal_Length", "double"), structField("Avg", "double")) + # Groups by `Sepal_Length` and computes the average for `Sepal_Width` + df4 <- gapply( + cols = "Sepal_Length", + irisDF, + function(key, x) { + y <- data.frame(key, mean(x$Sepal_Width), stringsAsFactors = FALSE) + }, + schema) + actual <- collect(df4) + actual <- actual[order(actual$Sepal_Length), ] + rownames(actual) <- NULL + agg_local_df <- data.frame(aggregate(iris$Sepal.Width, by = list(iris$Sepal.Length), FUN = mean), + stringsAsFactors = FALSE) + colnames(agg_local_df) <- c("Sepal_Length", "Avg") + expected <- agg_local_df[order(agg_local_df$Sepal_Length), ] + rownames(expected) <- NULL + expect_identical(actual, expected) +}) + +test_that("Window functions on a DataFrame", { + df <- createDataFrame(list(list(1L, "1"), list(2L, "2"), list(1L, "1"), list(2L, "2")), + schema = c("key", "value")) + ws <- orderBy(windowPartitionBy("key"), "value") + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expected <- data.frame(key = c(1L, NA, 2L, NA), + value = c("1", NA, "2", NA), + stringsAsFactors = FALSE) + expect_equal(result, expected) + + ws <- orderBy(windowPartitionBy(df$key), df$value) + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expect_equal(result, expected) + + ws <- partitionBy(windowOrderBy("value"), "key") + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expect_equal(result, expected) + + ws <- partitionBy(windowOrderBy(df$value), df$key) + result <- collect(select(df, over(lead("key", 1), ws), over(lead("value", 1), ws))) + names(result) <- c("key", "value") + expect_equal(result, expected) +}) + +test_that("createDataFrame sqlContext parameter backward compatibility", { + sqlContext <- suppressWarnings(sparkRSQL.init(sc)) + a <- 1:3 + b <- c("a", "b", "c") + ldf <- data.frame(a, b) + # Call function with namespace :: operator - SPARK-16538 + df <- suppressWarnings(SparkR::createDataFrame(sqlContext, ldf)) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + expect_equal(count(df), 3) + ldf2 <- collect(df) + expect_equal(ldf$a, ldf2$a) + + df2 <- suppressWarnings(createDataFrame(sqlContext, iris)) + expect_equal(count(df2), 150) + expect_equal(ncol(df2), 5) + + df3 <- suppressWarnings(read.df(sqlContext, jsonPath, "json")) + expect_is(df3, "SparkDataFrame") + expect_equal(count(df3), 3) + + before <- suppressWarnings(createDataFrame(sqlContext, iris)) + after <- suppressWarnings(createDataFrame(iris)) + expect_equal(collect(before), collect(after)) + + # more tests for SPARK-16538 + createOrReplaceTempView(df, "table") + SparkR::listTables() + SparkR::sql("SELECT 1") + suppressWarnings(SparkR::sql(sqlContext, "SELECT * FROM table")) + suppressWarnings(SparkR::dropTempTable(sqlContext, "table")) +}) + +test_that("randomSplit", { + num <- 4000 + df <- createDataFrame(data.frame(id = 1:num)) + weights <- c(2, 3, 5) + df_list <- randomSplit(df, weights) + expect_equal(length(weights), length(df_list)) + counts <- sapply(df_list, count) + expect_equal(num, sum(counts)) + expect_true(all(sapply(abs(counts / num - weights / sum(weights)), function(e) { e < 0.05 }))) + + df_list <- randomSplit(df, weights, 0) + expect_equal(length(weights), length(df_list)) + counts <- sapply(df_list, count) + expect_equal(num, sum(counts)) + expect_true(all(sapply(abs(counts / num - weights / sum(weights)), function(e) { e < 0.05 }))) +}) + +test_that("Setting and getting config on SparkSession, sparkR.conf(), sparkR.uiWebUrl()", { + # first, set it to a random but known value + conf <- callJMethod(sparkSession, "conf") + property <- paste0("spark.testing.", as.character(runif(1))) + value1 <- as.character(runif(1)) + callJMethod(conf, "set", property, value1) + + # next, change the same property to the new value + value2 <- as.character(runif(1)) + l <- list(value2) + names(l) <- property + sparkR.session(sparkConfig = l) + + newValue <- unlist(sparkR.conf(property, ""), use.names = FALSE) + expect_equal(value2, newValue) + + value <- as.character(runif(1)) + sparkR.session(spark.app.name = "sparkSession test", spark.testing.r.session.r = value) + allconf <- sparkR.conf() + appNameValue <- allconf[["spark.app.name"]] + testValue <- allconf[["spark.testing.r.session.r"]] + expect_equal(appNameValue, "sparkSession test") + expect_equal(testValue, value) + expect_error(sparkR.conf("completely.dummy"), "Config 'completely.dummy' is not set") + + url <- sparkR.uiWebUrl() + expect_equal(substr(url, 1, 7), "http://") +}) + +test_that("enableHiveSupport on SparkSession", { + setHiveContext(sc) + unsetHiveContext() + # if we are still here, it must be built with hive + conf <- callJMethod(sparkSession, "conf") + value <- callJMethod(conf, "get", "spark.sql.catalogImplementation") + expect_equal(value, "hive") +}) + +test_that("Spark version from SparkSession", { + ver <- callJMethod(sc, "version") + version <- sparkR.version() + expect_equal(ver, version) +}) + +test_that("Call DataFrameWriter.save() API in Java without path and check argument types", { + df <- read.df(jsonPath, "json") + # This tests if the exception is thrown from JVM not from SparkR side. + # It makes sure that we can omit path argument in write.df API and then it calls + # DataFrameWriter.save() without path. + expect_error(write.df(df, source = "csv"), + "Error in save : illegal argument - Expected exactly one path to be specified") + expect_error(write.json(df, jsonPath), + "Error in json : analysis error - path file:.*already exists") + expect_error(write.text(df, jsonPath), + "Error in text : analysis error - path file:.*already exists") + expect_error(write.orc(df, jsonPath), + "Error in orc : analysis error - path file:.*already exists") + expect_error(write.parquet(df, jsonPath), + "Error in parquet : analysis error - path file:.*already exists") + + # Arguments checking in R side. + expect_error(write.df(df, "data.tmp", source = c(1, 2)), + paste("source should be character, NULL or omitted. It is the datasource specified", + "in 'spark.sql.sources.default' configuration by default.")) + expect_error(write.df(df, path = c(3)), + "path should be character, NULL or omitted.") + expect_error(write.df(df, mode = TRUE), + "mode should be character or omitted. It is 'error' by default.") +}) + +test_that("Call DataFrameWriter.load() API in Java without path and check argument types", { + # This tests if the exception is thrown from JVM not from SparkR side. + # It makes sure that we can omit path argument in read.df API and then it calls + # DataFrameWriter.load() without path. + expect_error(read.df(source = "json"), + paste("Error in load : analysis error - Unable to infer schema for JSON.", + "It must be specified manually")) + expect_error(read.df("arbitrary_path"), "Error in load : analysis error - Path does not exist") + expect_error(read.json("arbitrary_path"), "Error in json : analysis error - Path does not exist") + expect_error(read.text("arbitrary_path"), "Error in text : analysis error - Path does not exist") + expect_error(read.orc("arbitrary_path"), "Error in orc : analysis error - Path does not exist") + expect_error(read.parquet("arbitrary_path"), + "Error in parquet : analysis error - Path does not exist") + + # Arguments checking in R side. + expect_error(read.df(path = c(3)), + "path should be character, NULL or omitted.") + expect_error(read.df(jsonPath, source = c(1, 2)), + paste("source should be character, NULL or omitted. It is the datasource specified", + "in 'spark.sql.sources.default' configuration by default.")) + + expect_warning(read.json(jsonPath, a = 1, 2, 3, "a"), + "Unnamed arguments ignored: 2, 3, a.") +}) + +test_that("Specify a schema by using a DDL-formatted string when reading", { + # Test read.df with a user defined schema in a DDL-formatted string. + df1 <- read.df(jsonPath, "json", "name STRING, age DOUBLE") + expect_is(df1, "SparkDataFrame") + expect_equal(dtypes(df1), list(c("name", "string"), c("age", "double"))) + + expect_error(read.df(jsonPath, "json", "name stri"), "DataType stri is not supported.") + + # Test loadDF with a user defined schema in a DDL-formatted string. + df2 <- loadDF(jsonPath, "json", "name STRING, age DOUBLE") + expect_is(df2, "SparkDataFrame") + expect_equal(dtypes(df2), list(c("name", "string"), c("age", "double"))) + + expect_error(loadDF(jsonPath, "json", "name stri"), "DataType stri is not supported.") +}) + +test_that("Collect on DataFrame when NAs exists at the top of a timestamp column", { + ldf <- data.frame(col1 = c(0, 1, 2), + col2 = c(as.POSIXct("2017-01-01 00:00:01"), + NA, + as.POSIXct("2017-01-01 12:00:01")), + col3 = c(as.POSIXlt("2016-01-01 00:59:59"), + NA, + as.POSIXlt("2016-01-01 12:01:01"))) + sdf1 <- createDataFrame(ldf) + ldf1 <- collect(sdf1) + expect_equal(dtypes(sdf1), list(c("col1", "double"), + c("col2", "timestamp"), + c("col3", "timestamp"))) + expect_equal(class(ldf1$col1), "numeric") + expect_equal(class(ldf1$col2), c("POSIXct", "POSIXt")) + expect_equal(class(ldf1$col3), c("POSIXct", "POSIXt")) + + # Columns with NAs at the top + sdf2 <- filter(sdf1, "col1 > 1") + ldf2 <- collect(sdf2) + expect_equal(dtypes(sdf2), list(c("col1", "double"), + c("col2", "timestamp"), + c("col3", "timestamp"))) + expect_equal(class(ldf2$col1), "numeric") + expect_equal(class(ldf2$col2), c("POSIXct", "POSIXt")) + expect_equal(class(ldf2$col3), c("POSIXct", "POSIXt")) + + # Columns with only NAs, the type will also be cast to PRIMITIVE_TYPE + sdf3 <- filter(sdf1, "col1 == 0") + ldf3 <- collect(sdf3) + expect_equal(dtypes(sdf3), list(c("col1", "double"), + c("col2", "timestamp"), + c("col3", "timestamp"))) + expect_equal(class(ldf3$col1), "numeric") + expect_equal(class(ldf3$col2), c("POSIXct", "POSIXt")) + expect_equal(class(ldf3$col3), c("POSIXct", "POSIXt")) +}) + +test_that("catalog APIs, currentDatabase, setCurrentDatabase, listDatabases", { + expect_equal(currentDatabase(), "default") + expect_error(setCurrentDatabase("default"), NA) + expect_error(setCurrentDatabase("foo"), + "Error in setCurrentDatabase : analysis error - Database 'foo' does not exist") + dbs <- collect(listDatabases()) + expect_equal(names(dbs), c("name", "description", "locationUri")) + expect_equal(dbs[[1]], "default") +}) + +test_that("catalog APIs, listTables, listColumns, listFunctions", { + tb <- listTables() + count <- count(tables()) + expect_equal(nrow(tb), count) + expect_equal(colnames(tb), c("name", "database", "description", "tableType", "isTemporary")) + + createOrReplaceTempView(as.DataFrame(cars), "cars") + + tb <- listTables() + expect_equal(nrow(tb), count + 1) + tbs <- collect(tb) + expect_true(nrow(tbs[tbs$name == "cars", ]) > 0) + expect_error(listTables("bar"), + "Error in listTables : no such database - Database 'bar' not found") + + c <- listColumns("cars") + expect_equal(nrow(c), 2) + expect_equal(colnames(c), + c("name", "description", "dataType", "nullable", "isPartition", "isBucket")) + expect_equal(collect(c)[[1]][[1]], "speed") + expect_error(listColumns("foo", "default"), + "Error in listColumns : analysis error - Table 'foo' does not exist in database 'default'") + + f <- listFunctions() + expect_true(nrow(f) >= 200) # 250 + expect_equal(colnames(f), + c("name", "database", "description", "className", "isTemporary")) + expect_equal(take(orderBy(f, "className"), 1)$className, + "org.apache.spark.sql.catalyst.expressions.Abs") + expect_error(listFunctions("foo_db"), + "Error in listFunctions : analysis error - Database 'foo_db' does not exist") + + # recoverPartitions does not work with tempory view + expect_error(recoverPartitions("cars"), + "no such table - Table or view 'cars' not found in database 'default'") + expect_error(refreshTable("cars"), NA) + expect_error(refreshByPath("/"), NA) + + dropTempView("cars") +}) + +compare_list <- function(list1, list2) { + # get testthat to show the diff by first making the 2 lists equal in length + expect_equal(length(list1), length(list2)) + l <- max(length(list1), length(list2)) + length(list1) <- l + length(list2) <- l + expect_equal(sort(list1, na.last = TRUE), sort(list2, na.last = TRUE)) +} + +# This should always be the **very last test** in this test file. +test_that("No extra files are created in SPARK_HOME by starting session and making calls", { + # Check that it is not creating any extra file. + # Does not check the tempdir which would be cleaned up after. + filesAfter <- list.files(path = sparkRDir, all.files = TRUE) + + expect_true(length(sparkRFilesBefore) > 0) + # first, ensure derby.log is not there + expect_false("derby.log" %in% filesAfter) + # second, ensure only spark-warehouse is created when calling SparkSession, enableHiveSupport = F + # note: currently all other test files have enableHiveSupport = F, so we capture the list of files + # before creating a SparkSession with enableHiveSupport = T at the top of this test file + # (filesBefore). The test here is to compare that (filesBefore) against the list of files before + # any test is run in run-all.R (sparkRFilesBefore). + # sparkRWhitelistSQLDirs is also defined in run-all.R, and should contain only 2 whitelisted dirs, + # here allow the first value, spark-warehouse, in the diff, everything else should be exactly the + # same as before any test is run. + compare_list(sparkRFilesBefore, setdiff(filesBefore, sparkRWhitelistSQLDirs[[1]])) + # third, ensure only spark-warehouse and metastore_db are created when enableHiveSupport = T + # note: as the note above, after running all tests in this file while enableHiveSupport = T, we + # check the list of files again. This time we allow both whitelisted dirs to be in the diff. + compare_list(sparkRFilesBefore, setdiff(filesAfter, sparkRWhitelistSQLDirs)) +}) + +unlink(parquetPath) +unlink(orcPath) +unlink(jsonPath) +unlink(jsonPathNa) +unlink(complexTypeJsonPath) +unlink(mapTypeJsonPath) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_streaming.R b/R/pkg/tests/fulltests/test_streaming.R new file mode 100644 index 000000000000..54f40bbd5f51 --- /dev/null +++ b/R/pkg/tests/fulltests/test_streaming.R @@ -0,0 +1,178 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(testthat) + +context("Structured Streaming") + +# Tests for Structured Streaming functions in SparkR + +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + +jsonSubDir <- file.path("sparkr-test", "json", "") +if (is_windows()) { + # file.path removes the empty separator on Windows, adds it back + jsonSubDir <- paste0(jsonSubDir, .Platform$file.sep) +} +jsonDir <- file.path(tempdir(), jsonSubDir) +dir.create(jsonDir, recursive = TRUE) + +mockLines <- c("{\"name\":\"Michael\"}", + "{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"Justin\", \"age\":19}") +jsonPath <- tempfile(pattern = jsonSubDir, fileext = ".tmp") +writeLines(mockLines, jsonPath) + +mockLinesNa <- c("{\"name\":\"Bob\",\"age\":16,\"height\":176.5}", + "{\"name\":\"Alice\",\"age\":null,\"height\":164.3}", + "{\"name\":\"David\",\"age\":60,\"height\":null}") +jsonPathNa <- tempfile(pattern = jsonSubDir, fileext = ".tmp") + +schema <- structType(structField("name", "string"), + structField("age", "integer"), + structField("count", "double")) + +stringSchema <- "name STRING, age INTEGER, count DOUBLE" + +test_that("read.stream, write.stream, awaitTermination, stopQuery", { + df <- read.stream("json", path = jsonDir, schema = schema, maxFilesPerTrigger = 1) + expect_true(isStreaming(df)) + counts <- count(group_by(df, "name")) + q <- write.stream(counts, "memory", queryName = "people", outputMode = "complete") + + expect_false(awaitTermination(q, 5 * 1000)) + callJMethod(q@ssq, "processAllAvailable") + expect_equal(head(sql("SELECT count(*) FROM people"))[[1]], 3) + + writeLines(mockLinesNa, jsonPathNa) + awaitTermination(q, 5 * 1000) + callJMethod(q@ssq, "processAllAvailable") + expect_equal(head(sql("SELECT count(*) FROM people"))[[1]], 6) + + stopQuery(q) + expect_true(awaitTermination(q, 1)) + expect_error(awaitTermination(q), NA) +}) + +test_that("print from explain, lastProgress, status, isActive", { + df <- read.stream("json", path = jsonDir, schema = schema) + expect_true(isStreaming(df)) + counts <- count(group_by(df, "name")) + q <- write.stream(counts, "memory", queryName = "people2", outputMode = "complete") + + awaitTermination(q, 5 * 1000) + callJMethod(q@ssq, "processAllAvailable") + + expect_equal(capture.output(explain(q))[[1]], "== Physical Plan ==") + expect_true(any(grepl("\"description\" : \"MemorySink\"", capture.output(lastProgress(q))))) + expect_true(any(grepl("\"isTriggerActive\" : ", capture.output(status(q))))) + + expect_equal(queryName(q), "people2") + expect_true(isActive(q)) + + stopQuery(q) +}) + +test_that("Stream other format", { + parquetPath <- tempfile(pattern = "sparkr-test", fileext = ".parquet") + df <- read.df(jsonPath, "json", schema) + write.df(df, parquetPath, "parquet", "overwrite") + + df <- read.stream(path = parquetPath, schema = schema) + expect_true(isStreaming(df)) + counts <- count(group_by(df, "name")) + q <- write.stream(counts, "memory", queryName = "people3", outputMode = "complete") + + expect_false(awaitTermination(q, 5 * 1000)) + callJMethod(q@ssq, "processAllAvailable") + expect_equal(head(sql("SELECT count(*) FROM people3"))[[1]], 3) + + expect_equal(queryName(q), "people3") + expect_true(any(grepl("\"description\" : \"FileStreamSource[[:print:]]+parquet", + capture.output(lastProgress(q))))) + expect_true(isActive(q)) + + stopQuery(q) + expect_true(awaitTermination(q, 1)) + expect_false(isActive(q)) + + unlink(parquetPath) +}) + +test_that("Specify a schema by using a DDL-formatted string when reading", { + # Test read.stream with a user defined schema in a DDL-formatted string. + parquetPath <- tempfile(pattern = "sparkr-test", fileext = ".parquet") + df <- read.df(jsonPath, "json", schema) + write.df(df, parquetPath, "parquet", "overwrite") + + df <- read.stream(path = parquetPath, schema = stringSchema) + expect_true(isStreaming(df)) + counts <- count(group_by(df, "name")) + q <- write.stream(counts, "memory", queryName = "people3", outputMode = "complete") + + expect_false(awaitTermination(q, 5 * 1000)) + callJMethod(q@ssq, "processAllAvailable") + expect_equal(head(sql("SELECT count(*) FROM people3"))[[1]], 3) + + expect_error(read.stream(path = parquetPath, schema = "name stri"), + "DataType stri is not supported.") + + unlink(parquetPath) +}) + +test_that("Non-streaming DataFrame", { + c <- as.DataFrame(cars) + expect_false(isStreaming(c)) + + expect_error(write.stream(c, "memory", queryName = "people", outputMode = "complete"), + paste0(".*(writeStream : analysis error - 'writeStream' can be called only on ", + "streaming Dataset/DataFrame).*")) +}) + +test_that("Unsupported operation", { + # memory sink without aggregation + df <- read.stream("json", path = jsonDir, schema = schema, maxFilesPerTrigger = 1) + expect_error(write.stream(df, "memory", queryName = "people", outputMode = "complete"), + paste0(".*(start : analysis error - Complete output mode not supported when there ", + "are no streaming aggregations on streaming DataFrames/Datasets).*")) +}) + +test_that("Terminated by error", { + df <- read.stream("json", path = jsonDir, schema = schema, maxFilesPerTrigger = -1) + counts <- count(group_by(df, "name")) + # This would not fail before returning with a StreamingQuery, + # but could dump error log at just about the same time + expect_error(q <- write.stream(counts, "memory", queryName = "people4", outputMode = "complete"), + NA) + + expect_error(awaitTermination(q, 5 * 1000), + paste0(".*(awaitTermination : streaming query error - Invalid value '-1' for option", + " 'maxFilesPerTrigger', must be a positive integer).*")) + + expect_true(any(grepl("\"message\" : \"Terminated with exception: Invalid value", + capture.output(status(q))))) + expect_true(any(grepl("Streaming query has no progress", capture.output(lastProgress(q))))) + expect_equal(queryName(q), "people4") + expect_false(isActive(q)) + + stopQuery(q) +}) + +unlink(jsonPath) +unlink(jsonPathNa) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_take.R b/R/pkg/tests/fulltests/test_take.R new file mode 100644 index 000000000000..8936cc57da22 --- /dev/null +++ b/R/pkg/tests/fulltests/test_take.R @@ -0,0 +1,69 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("tests RDD function take()") + +# Mock data +numVector <- c(-10:97) +numList <- list(sqrt(1), sqrt(2), sqrt(3), 4 ** 10) +strVector <- c("Dexter Morgan: I suppose I should be upset, even feel", + "violated, but I'm not. No, in fact, I think this is a friendly", + "message, like \"Hey, wanna play?\" and yes, I want to play. ", + "I really, really do.") +strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge, ", + "other times it helps me control the chaos.", + "Dexter Morgan: Harry and Dorris Morgan did a wonderful job ", + "raising me. But they're both dead now. I didn't kill them. Honest.") + +# JavaSparkContext handle +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) + +test_that("take() gives back the original elements in correct count and order", { + numVectorRDD <- parallelize(sc, numVector, 10) + # case: number of elements to take is less than the size of the first partition + expect_equal(takeRDD(numVectorRDD, 1), as.list(head(numVector, n = 1))) + # case: number of elements to take is the same as the size of the first partition + expect_equal(takeRDD(numVectorRDD, 11), as.list(head(numVector, n = 11))) + # case: number of elements to take is greater than all elements + expect_equal(takeRDD(numVectorRDD, length(numVector)), as.list(numVector)) + expect_equal(takeRDD(numVectorRDD, length(numVector) + 1), as.list(numVector)) + + numListRDD <- parallelize(sc, numList, 1) + numListRDD2 <- parallelize(sc, numList, 4) + expect_equal(takeRDD(numListRDD, 3), takeRDD(numListRDD2, 3)) + expect_equal(takeRDD(numListRDD, 5), takeRDD(numListRDD2, 5)) + expect_equal(takeRDD(numListRDD, 1), as.list(head(numList, n = 1))) + expect_equal(takeRDD(numListRDD2, 999), numList) + + strVectorRDD <- parallelize(sc, strVector, 2) + strVectorRDD2 <- parallelize(sc, strVector, 3) + expect_equal(takeRDD(strVectorRDD, 4), as.list(strVector)) + expect_equal(takeRDD(strVectorRDD2, 2), as.list(head(strVector, n = 2))) + + strListRDD <- parallelize(sc, strList, 4) + strListRDD2 <- parallelize(sc, strList, 1) + expect_equal(takeRDD(strListRDD, 3), as.list(head(strList, n = 3))) + expect_equal(takeRDD(strListRDD2, 1), as.list(head(strList, n = 1))) + + expect_equal(length(takeRDD(strListRDD, 0)), 0) + expect_equal(length(takeRDD(strVectorRDD, 0)), 0) + expect_equal(length(takeRDD(numListRDD, 0)), 0) + expect_equal(length(takeRDD(numVectorRDD, 0)), 0) +}) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_textFile.R b/R/pkg/tests/fulltests/test_textFile.R new file mode 100644 index 000000000000..be2d2711ff88 --- /dev/null +++ b/R/pkg/tests/fulltests/test_textFile.R @@ -0,0 +1,164 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("the textFile() function") + +# JavaSparkContext handle +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) + +mockFile <- c("Spark is pretty.", "Spark is awesome.") + +test_that("textFile() on a local file returns an RDD", { + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + expect_is(rdd, "RDD") + expect_true(countRDD(rdd) > 0) + expect_equal(countRDD(rdd), 2) + + unlink(fileName) +}) + +test_that("textFile() followed by a collect() returns the same content", { + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + expect_equal(collectRDD(rdd), as.list(mockFile)) + + unlink(fileName) +}) + +test_that("textFile() word count works as expected", { + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + + words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] }) + wordCount <- lapply(words, function(word) { list(word, 1L) }) + + counts <- reduceByKey(wordCount, "+", 2L) + output <- collectRDD(counts) + expected <- list(list("pretty.", 1), list("is", 2), list("awesome.", 1), + list("Spark", 2)) + expect_equal(sortKeyValueList(output), sortKeyValueList(expected)) + + unlink(fileName) +}) + +test_that("several transformations on RDD created by textFile()", { + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) # RDD + for (i in 1:10) { + # PipelinedRDD initially created from RDD + rdd <- lapply(rdd, function(x) paste(x, x)) + } + collectRDD(rdd) + + unlink(fileName) +}) + +test_that("textFile() followed by a saveAsTextFile() returns the same content", { + fileName1 <- tempfile(pattern = "spark-test", fileext = ".tmp") + fileName2 <- tempfile(pattern = "spark-test", fileext = ".tmp") + writeLines(mockFile, fileName1) + + rdd <- textFile(sc, fileName1, 1L) + saveAsTextFile(rdd, fileName2) + rdd <- textFile(sc, fileName2) + expect_equal(collectRDD(rdd), as.list(mockFile)) + + unlink(fileName1) + unlink(fileName2) +}) + +test_that("saveAsTextFile() on a parallelized list works as expected", { + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") + l <- list(1, 2, 3) + rdd <- parallelize(sc, l, 1L) + saveAsTextFile(rdd, fileName) + rdd <- textFile(sc, fileName) + expect_equal(collectRDD(rdd), lapply(l, function(x) {toString(x)})) + + unlink(fileName) +}) + +test_that("textFile() and saveAsTextFile() word count works as expected", { + fileName1 <- tempfile(pattern = "spark-test", fileext = ".tmp") + fileName2 <- tempfile(pattern = "spark-test", fileext = ".tmp") + writeLines(mockFile, fileName1) + + rdd <- textFile(sc, fileName1) + + words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] }) + wordCount <- lapply(words, function(word) { list(word, 1L) }) + + counts <- reduceByKey(wordCount, "+", 2L) + + saveAsTextFile(counts, fileName2) + rdd <- textFile(sc, fileName2) + + output <- collectRDD(rdd) + expected <- list(list("awesome.", 1), list("Spark", 2), + list("pretty.", 1), list("is", 2)) + expectedStr <- lapply(expected, function(x) { toString(x) }) + expect_equal(sortKeyValueList(output), sortKeyValueList(expectedStr)) + + unlink(fileName1) + unlink(fileName2) +}) + +test_that("textFile() on multiple paths", { + fileName1 <- tempfile(pattern = "spark-test", fileext = ".tmp") + fileName2 <- tempfile(pattern = "spark-test", fileext = ".tmp") + writeLines("Spark is pretty.", fileName1) + writeLines("Spark is awesome.", fileName2) + + rdd <- textFile(sc, c(fileName1, fileName2)) + expect_equal(countRDD(rdd), 2) + + unlink(fileName1) + unlink(fileName2) +}) + +test_that("Pipelined operations on RDDs created using textFile", { + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + + lengths <- lapply(rdd, function(x) { length(x) }) + expect_equal(collectRDD(lengths), list(1, 1)) + + lengthsPipelined <- lapply(lengths, function(x) { x + 10 }) + expect_equal(collectRDD(lengthsPipelined), list(11, 11)) + + lengths30 <- lapply(lengthsPipelined, function(x) { x + 20 }) + expect_equal(collectRDD(lengths30), list(31, 31)) + + lengths20 <- lapply(lengths, function(x) { x + 20 }) + expect_equal(collectRDD(lengths20), list(21, 21)) + + unlink(fileName) +}) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_utils.R b/R/pkg/tests/fulltests/test_utils.R new file mode 100644 index 000000000000..af81423aa8dd --- /dev/null +++ b/R/pkg/tests/fulltests/test_utils.R @@ -0,0 +1,239 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("functions in utils.R") + +# JavaSparkContext handle +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) +sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) + +test_that("convertJListToRList() gives back (deserializes) the original JLists + of strings and integers", { + # It's hard to manually create a Java List using rJava, since it does not + # support generics well. Instead, we rely on collectRDD() returning a + # JList. + nums <- as.list(1:10) + rdd <- parallelize(sc, nums, 1L) + jList <- callJMethod(rdd@jrdd, "collect") + rList <- convertJListToRList(jList, flatten = TRUE) + expect_equal(rList, nums) + + strs <- as.list("hello", "spark") + rdd <- parallelize(sc, strs, 2L) + jList <- callJMethod(rdd@jrdd, "collect") + rList <- convertJListToRList(jList, flatten = TRUE) + expect_equal(rList, strs) +}) + +test_that("serializeToBytes on RDD", { + # File content + mockFile <- c("Spark is pretty.", "Spark is awesome.") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") + writeLines(mockFile, fileName) + + text.rdd <- textFile(sc, fileName) + expect_equal(getSerializedMode(text.rdd), "string") + ser.rdd <- serializeToBytes(text.rdd) + expect_equal(collectRDD(ser.rdd), as.list(mockFile)) + expect_equal(getSerializedMode(ser.rdd), "byte") + + unlink(fileName) +}) + +test_that("cleanClosure on R functions", { + y <- c(1, 2, 3) + g <- function(x) { x + 1 } + f <- function(x) { g(x) + y } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 2) # y, g + actual <- get("y", envir = env, inherits = FALSE) + expect_equal(actual, y) + actual <- get("g", envir = env, inherits = FALSE) + expect_equal(actual, g) + + # Test for nested enclosures and package variables. + env2 <- new.env() + funcEnv <- new.env(parent = env2) + f <- function(x) { log(g(x) + y) } + environment(f) <- funcEnv # enclosing relationship: f -> funcEnv -> env2 -> .GlobalEnv + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 2) # "min" should not be included + actual <- get("y", envir = env, inherits = FALSE) + expect_equal(actual, y) + actual <- get("g", envir = env, inherits = FALSE) + expect_equal(actual, g) + + base <- c(1, 2, 3) + l <- list(field = matrix(1)) + field <- matrix(2) + defUse <- 3 + g <- function(x) { x + y } + f <- function(x) { + defUse <- base::as.integer(x) + 1 # Test for access operators `::`. + lapply(x, g) + 1 # Test for capturing function call "g"'s closure as a argument of lapply. + l$field[1, 1] <- 3 # Test for access operators `$`. + res <- defUse + l$field[1, ] # Test for def-use chain of "defUse", and "" symbol. + f(res) # Test for recursive calls. + } + newF <- cleanClosure(f) + env <- environment(newF) + # TODO(shivaram): length(ls(env)) is 4 here for some reason and `lapply` is included in `env`. + # Disabling this test till we debug this. + # + # nolint start + # expect_equal(length(ls(env)), 3) # Only "g", "l" and "f". No "base", "field" or "defUse". + # nolint end + expect_true("g" %in% ls(env)) + expect_true("l" %in% ls(env)) + expect_true("f" %in% ls(env)) + expect_equal(get("l", envir = env, inherits = FALSE), l) + # "y" should be in the environemnt of g. + newG <- get("g", envir = env, inherits = FALSE) + env <- environment(newG) + expect_equal(length(ls(env)), 1) + actual <- get("y", envir = env, inherits = FALSE) + expect_equal(actual, y) + + # Test for function (and variable) definitions. + f <- function(x) { + g <- function(y) { y * 2 } + g(x) + } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 0) # "y" and "g" should not be included. + + # Test for overriding variables in base namespace (Issue: SparkR-196). + nums <- as.list(1:10) + rdd <- parallelize(sc, nums, 2L) + t <- 4 # Override base::t in .GlobalEnv. + f <- function(x) { x > t } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(ls(env), "t") + expect_equal(get("t", envir = env, inherits = FALSE), t) + actual <- collectRDD(lapply(rdd, f)) + expected <- as.list(c(rep(FALSE, 4), rep(TRUE, 6))) + expect_equal(actual, expected) + + # Test for broadcast variables. + a <- matrix(nrow = 10, ncol = 10, data = rnorm(100)) + aBroadcast <- broadcastRDD(sc, a) + normMultiply <- function(x) { norm(aBroadcast$value) * x } + newnormMultiply <- SparkR:::cleanClosure(normMultiply) + env <- environment(newnormMultiply) + expect_equal(ls(env), "aBroadcast") + expect_equal(get("aBroadcast", envir = env, inherits = FALSE), aBroadcast) +}) + +test_that("varargsToJProperties", { + jprops <- newJObject("java.util.Properties") + expect_true(class(jprops) == "jobj") + + jprops <- varargsToJProperties(abc = "123") + expect_true(class(jprops) == "jobj") + expect_equal(callJMethod(jprops, "getProperty", "abc"), "123") + + jprops <- varargsToJProperties(abc = "abc", b = 1) + expect_equal(callJMethod(jprops, "getProperty", "abc"), "abc") + expect_equal(callJMethod(jprops, "getProperty", "b"), "1") + + jprops <- varargsToJProperties() + expect_equal(callJMethod(jprops, "size"), 0L) +}) + +test_that("convertToJSaveMode", { + s <- convertToJSaveMode("error") + expect_true(class(s) == "jobj") + expect_match(capture.output(print.jobj(s)), "Java ref type org.apache.spark.sql.SaveMode id ") + expect_error(convertToJSaveMode("foo"), + 'mode should be one of "append", "overwrite", "error", "ignore"') #nolint +}) + +test_that("captureJVMException", { + method <- "createStructField" + expect_error(tryCatch(callJStatic("org.apache.spark.sql.api.r.SQLUtils", method, + "col", "unknown", TRUE), + error = function(e) { + captureJVMException(e, method) + }), + "parse error - .*DataType unknown.*not supported.") +}) + +test_that("hashCode", { + expect_error(hashCode("bc53d3605e8a5b7de1e8e271c2317645"), NA) +}) + +test_that("overrideEnvs", { + config <- new.env() + config[["spark.master"]] <- "foo" + config[["config_only"]] <- "ok" + param <- new.env() + param[["spark.master"]] <- "local" + param[["param_only"]] <- "blah" + overrideEnvs(config, param) + expect_equal(config[["spark.master"]], "local") + expect_equal(config[["param_only"]], "blah") + expect_equal(config[["config_only"]], "ok") +}) + +test_that("rbindRaws", { + + # Mixed Column types + r <- serialize(1:5, connection = NULL) + r1 <- serialize(1, connection = NULL) + r2 <- serialize(letters, connection = NULL) + r3 <- serialize(1:10, connection = NULL) + inputData <- list(list(1L, r1, "a", r), list(2L, r2, "b", r), + list(3L, r3, "c", r)) + expected <- data.frame(V1 = 1:3) + expected$V2 <- list(r1, r2, r3) + expected$V3 <- c("a", "b", "c") + expected$V4 <- list(r, r, r) + result <- rbindRaws(inputData) + expect_equal(expected, result) + + # Single binary column + input <- list(list(r1), list(r2), list(r3)) + expected <- subset(expected, select = "V2") + result <- setNames(rbindRaws(input), "V2") + expect_equal(expected, result) + +}) + +test_that("varargsToStrEnv", { + strenv <- varargsToStrEnv(a = 1, b = 1.1, c = TRUE, d = "abcd") + env <- varargsToEnv(a = "1", b = "1.1", c = "true", d = "abcd") + expect_equal(strenv, env) + expect_error(varargsToStrEnv(a = list(1, "a")), + paste0("Unsupported type for a : list. Supported types are logical, ", + "numeric, character and NULL.")) + expect_warning(varargsToStrEnv(a = 1, 2, 3, 4), "Unnamed arguments ignored: 2, 3, 4.") + expect_warning(varargsToStrEnv(1, 2, 3, 4), "Unnamed arguments ignored: 1, 2, 3, 4.") +}) + +test_that("basenameSansExtFromUrl", { + x <- paste0("http://people.apache.org/~pwendell/spark-nightly/spark-branch-2.1-bin/spark-2.1.1-", + "SNAPSHOT-2016_12_09_11_08-eb2d9bf-bin/spark-2.1.1-SNAPSHOT-bin-hadoop2.7.tgz") + expect_equal(basenameSansExtFromUrl(x), "spark-2.1.1-SNAPSHOT-bin-hadoop2.7") + z <- "http://people.apache.org/~pwendell/spark-releases/spark-2.1.0--hive.tar.gz" + expect_equal(basenameSansExtFromUrl(z), "spark-2.1.0--hive") +}) + +sparkR.session.stop() diff --git a/R/pkg/tests/run-all.R b/R/pkg/tests/run-all.R index 4f8a1ed2d83e..a1834a220261 100644 --- a/R/pkg/tests/run-all.R +++ b/R/pkg/tests/run-all.R @@ -18,4 +18,36 @@ library(testthat) library(SparkR) +# Turn all warnings into errors +options("warn" = 2) + +if (.Platform$OS.type == "windows") { + Sys.setenv(TZ = "GMT") +} + +# Setup global test environment +# Install Spark first to set SPARK_HOME +install.spark() + +sparkRDir <- file.path(Sys.getenv("SPARK_HOME"), "R") +sparkRWhitelistSQLDirs <- c("spark-warehouse", "metastore_db") +invisible(lapply(sparkRWhitelistSQLDirs, + function(x) { unlink(file.path(sparkRDir, x), recursive = TRUE, force = TRUE)})) +sparkRFilesBefore <- list.files(path = sparkRDir, all.files = TRUE) + +sparkRTestMaster <- "local[1]" +if (identical(Sys.getenv("NOT_CRAN"), "true")) { + sparkRTestMaster <- "" +} + test_package("SparkR") + +if (identical(Sys.getenv("NOT_CRAN"), "true")) { + # set random seed for predictable results. mostly for base's sample() in tree and classification + set.seed(42) + # for testthat 1.0.2 later, change reporter from "summary" to default_reporter() + testthat:::run_tests("SparkR", + file.path(sparkRDir, "pkg", "tests", "fulltests"), + NULL, + "summary") +} diff --git a/R/pkg/vignettes/sparkr-vignettes.Rmd b/R/pkg/vignettes/sparkr-vignettes.Rmd new file mode 100644 index 000000000000..caeae72e37bb --- /dev/null +++ b/R/pkg/vignettes/sparkr-vignettes.Rmd @@ -0,0 +1,1179 @@ +--- +title: "SparkR - Practical Guide" +output: + rmarkdown::html_vignette: + toc: true + toc_depth: 4 +vignette: > + %\VignetteIndexEntry{SparkR - Practical Guide} + %\VignetteEngine{knitr::rmarkdown} + \usepackage[utf8]{inputenc} +--- + + + +```{r setup, include=FALSE} +library(knitr) +opts_hooks$set(eval = function(options) { + # override eval to FALSE only on windows + if (.Platform$OS.type == "windows") { + options$eval = FALSE + } + options +}) +``` + +## Overview + +SparkR is an R package that provides a light-weight frontend to use Apache Spark from R. With Spark `r packageVersion("SparkR")`, SparkR provides a distributed data frame implementation that supports data processing operations like selection, filtering, aggregation etc. and distributed machine learning using [MLlib](http://spark.apache.org/mllib/). + +## Getting Started + +We begin with an example running on the local machine and provide an overview of the use of SparkR: data ingestion, data processing and machine learning. + +First, let's load and attach the package. +```{r, message=FALSE} +library(SparkR) +``` + +`SparkSession` is the entry point into SparkR which connects your R program to a Spark cluster. You can create a `SparkSession` using `sparkR.session` and pass in options such as the application name, any Spark packages depended on, etc. + +We use default settings in which it runs in local mode. It auto downloads Spark package in the background if no previous installation is found. For more details about setup, see [Spark Session](#SetupSparkSession). + +```{r, include=FALSE} +install.spark() +sparkR.session(master = "local[1]") +``` +```{r, eval=FALSE} +sparkR.session() +``` + +The operations in SparkR are centered around an R class called `SparkDataFrame`. It is a distributed collection of data organized into named columns, which is conceptually equivalent to a table in a relational database or a data frame in R, but with richer optimizations under the hood. + +`SparkDataFrame` can be constructed from a wide array of sources such as: structured data files, tables in Hive, external databases, or existing local R data frames. For example, we create a `SparkDataFrame` from a local R data frame, + +```{r} +cars <- cbind(model = rownames(mtcars), mtcars) +carsDF <- createDataFrame(cars) +``` + +We can view the first few rows of the `SparkDataFrame` by `head` or `showDF` function. +```{r} +head(carsDF) +``` + +Common data processing operations such as `filter` and `select` are supported on the `SparkDataFrame`. +```{r} +carsSubDF <- select(carsDF, "model", "mpg", "hp") +carsSubDF <- filter(carsSubDF, carsSubDF$hp >= 200) +head(carsSubDF) +``` + +SparkR can use many common aggregation functions after grouping. + +```{r} +carsGPDF <- summarize(groupBy(carsDF, carsDF$gear), count = n(carsDF$gear)) +head(carsGPDF) +``` + +The results `carsDF` and `carsSubDF` are `SparkDataFrame` objects. To convert back to R `data.frame`, we can use `collect`. **Caution**: This can cause your interactive environment to run out of memory, though, because `collect()` fetches the entire distributed `DataFrame` to your client, which is acting as a Spark driver. +```{r} +carsGP <- collect(carsGPDF) +class(carsGP) +``` + +SparkR supports a number of commonly used machine learning algorithms. Under the hood, SparkR uses MLlib to train the model. Users can call `summary` to print a summary of the fitted model, `predict` to make predictions on new data, and `write.ml`/`read.ml` to save/load fitted models. + +SparkR supports a subset of R formula operators for model fitting, including ‘~’, ‘.’, ‘:’, ‘+’, and ‘-‘. We use linear regression as an example. +```{r} +model <- spark.glm(carsDF, mpg ~ wt + cyl) +``` + +The result matches that returned by R `glm` function applied to the corresponding `data.frame` `mtcars` of `carsDF`. In fact, for Generalized Linear Model, we specifically expose `glm` for `SparkDataFrame` as well so that the above is equivalent to `model <- glm(mpg ~ wt + cyl, data = carsDF)`. + +```{r} +summary(model) +``` + +The model can be saved by `write.ml` and loaded back using `read.ml`. +```{r, eval=FALSE} +write.ml(model, path = "/HOME/tmp/mlModel/glmModel") +``` + +In the end, we can stop Spark Session by running +```{r, eval=FALSE} +sparkR.session.stop() +``` + +## Setup + +### Installation + +Different from many other R packages, to use SparkR, you need an additional installation of Apache Spark. The Spark installation will be used to run a backend process that will compile and execute SparkR programs. + +After installing the SparkR package, you can call `sparkR.session` as explained in the previous section to start and it will check for the Spark installation. If you are working with SparkR from an interactive shell (eg. R, RStudio) then Spark is downloaded and cached automatically if it is not found. Alternatively, we provide an easy-to-use function `install.spark` for running this manually. If you don't have Spark installed on the computer, you may download it from [Apache Spark Website](http://spark.apache.org/downloads.html). + +```{r, eval=FALSE} +install.spark() +``` + +If you already have Spark installed, you don't have to install again and can pass the `sparkHome` argument to `sparkR.session` to let SparkR know where the existing Spark installation is. + +```{r, eval=FALSE} +sparkR.session(sparkHome = "/HOME/spark") +``` + +### Spark Session {#SetupSparkSession} + + +In addition to `sparkHome`, many other options can be specified in `sparkR.session`. For a complete list, see [Starting up: SparkSession](http://spark.apache.org/docs/latest/sparkr.html#starting-up-sparksession) and [SparkR API doc](http://spark.apache.org/docs/latest/api/R/sparkR.session.html). + +In particular, the following Spark driver properties can be set in `sparkConfig`. + +Property Name | Property group | spark-submit equivalent +---------------- | ------------------ | ---------------------- +`spark.driver.memory` | Application Properties | `--driver-memory` +`spark.driver.extraClassPath` | Runtime Environment | `--driver-class-path` +`spark.driver.extraJavaOptions` | Runtime Environment | `--driver-java-options` +`spark.driver.extraLibraryPath` | Runtime Environment | `--driver-library-path` +`spark.yarn.keytab` | Application Properties | `--keytab` +`spark.yarn.principal` | Application Properties | `--principal` + +**For Windows users**: Due to different file prefixes across operating systems, to avoid the issue of potential wrong prefix, a current workaround is to specify `spark.sql.warehouse.dir` when starting the `SparkSession`. + +```{r, eval=FALSE} +spark_warehouse_path <- file.path(path.expand('~'), "spark-warehouse") +sparkR.session(spark.sql.warehouse.dir = spark_warehouse_path) +``` + + +#### Cluster Mode +SparkR can connect to remote Spark clusters. [Cluster Mode Overview](http://spark.apache.org/docs/latest/cluster-overview.html) is a good introduction to different Spark cluster modes. + +When connecting SparkR to a remote Spark cluster, make sure that the Spark version and Hadoop version on the machine match the corresponding versions on the cluster. Current SparkR package is compatible with +```{r, echo=FALSE, tidy = TRUE} +paste("Spark", packageVersion("SparkR")) +``` +It should be used both on the local computer and on the remote cluster. + +To connect, pass the URL of the master node to `sparkR.session`. A complete list can be seen in [Spark Master URLs](http://spark.apache.org/docs/latest/submitting-applications.html#master-urls). +For example, to connect to a local standalone Spark master, we can call + +```{r, eval=FALSE} +sparkR.session(master = "spark://local:7077") +``` + +For YARN cluster, SparkR supports the client mode with the master set as "yarn". +```{r, eval=FALSE} +sparkR.session(master = "yarn") +``` +Yarn cluster mode is not supported in the current version. + +## Data Import + +### Local Data Frame +The simplest way is to convert a local R data frame into a `SparkDataFrame`. Specifically we can use `as.DataFrame` or `createDataFrame` and pass in the local R data frame to create a `SparkDataFrame`. As an example, the following creates a `SparkDataFrame` based using the `faithful` dataset from R. +```{r} +df <- as.DataFrame(faithful) +head(df) +``` + +### Data Sources +SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. You can check the Spark SQL Programming Guide for more [specific options](https://spark.apache.org/docs/latest/sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. + +The general method for creating `SparkDataFrame` from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active Spark Session will be used automatically. SparkR supports reading CSV, JSON and Parquet files natively and through Spark Packages you can find data source connectors for popular file formats like Avro. These packages can be added with `sparkPackages` parameter when initializing SparkSession using `sparkR.session`. + +```{r, eval=FALSE} +sparkR.session(sparkPackages = "com.databricks:spark-avro_2.11:3.0.0") +``` + +We can see how to use data sources using an example CSV input file. For more information please refer to SparkR [read.df](https://spark.apache.org/docs/latest/api/R/read.df.html) API documentation. +```{r, eval=FALSE} +df <- read.df(csvPath, "csv", header = "true", inferSchema = "true", na.strings = "NA") +``` + +The data sources API natively supports JSON formatted input files. Note that the file that is used here is not a typical JSON file. Each line in the file must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail. + +Let's take a look at the first two lines of the raw JSON file used here. + +```{r} +filePath <- paste0(sparkR.conf("spark.home"), + "/examples/src/main/resources/people.json") +readLines(filePath, n = 2L) +``` + +We use `read.df` to read that into a `SparkDataFrame`. + +```{r} +people <- read.df(filePath, "json") +count(people) +head(people) +``` + +SparkR automatically infers the schema from the JSON file. +```{r} +printSchema(people) +``` + +If we want to read multiple JSON files, `read.json` can be used. +```{r} +people <- read.json(paste0(Sys.getenv("SPARK_HOME"), + c("/examples/src/main/resources/people.json", + "/examples/src/main/resources/people.json"))) +count(people) +``` + +The data sources API can also be used to save out `SparkDataFrames` into multiple file formats. For example we can save the `SparkDataFrame` from the previous example to a Parquet file using `write.df`. +```{r, eval=FALSE} +write.df(people, path = "people.parquet", source = "parquet", mode = "overwrite") +``` + +### Hive Tables +You can also create SparkDataFrames from Hive tables. To do this we will need to create a SparkSession with Hive support which can access tables in the Hive MetaStore. Note that Spark should have been built with Hive support and more details can be found in the [SQL Programming Guide](https://spark.apache.org/docs/latest/sql-programming-guide.html). In SparkR, by default it will attempt to create a SparkSession with Hive support enabled (`enableHiveSupport = TRUE`). + +```{r, eval=FALSE} +sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + +txtPath <- paste0(sparkR.conf("spark.home"), "/examples/src/main/resources/kv1.txt") +sqlCMD <- sprintf("LOAD DATA LOCAL INPATH '%s' INTO TABLE src", txtPath) +sql(sqlCMD) + +results <- sql("FROM src SELECT key, value") + +# results is now a SparkDataFrame +head(results) +``` + + +## Data Processing + +**To dplyr users**: SparkR has similar interface as dplyr in data processing. However, some noticeable differences are worth mentioning in the first place. We use `df` to represent a `SparkDataFrame` and `col` to represent the name of column here. + +1. indicate columns. SparkR uses either a character string of the column name or a Column object constructed with `$` to indicate a column. For example, to select `col` in `df`, we can write `select(df, "col")` or `select(df, df$col)`. + +2. describe conditions. In SparkR, the Column object representation can be inserted into the condition directly, or we can use a character string to describe the condition, without referring to the `SparkDataFrame` used. For example, to select rows with value > 1, we can write `filter(df, df$col > 1)` or `filter(df, "col > 1")`. + +Here are more concrete examples. + +dplyr | SparkR +-------- | --------- +`select(mtcars, mpg, hp)` | `select(carsDF, "mpg", "hp")` +`filter(mtcars, mpg > 20, hp > 100)` | `filter(carsDF, carsDF$mpg > 20, carsDF$hp > 100)` + +Other differences will be mentioned in the specific methods. + +We use the `SparkDataFrame` `carsDF` created above. We can get basic information about the `SparkDataFrame`. +```{r} +carsDF +``` + +Print out the schema in tree format. +```{r} +printSchema(carsDF) +``` + +### SparkDataFrame Operations + +#### Selecting rows, columns + +SparkDataFrames support a number of functions to do structured data processing. Here we include some basic examples and a complete list can be found in the [API](https://spark.apache.org/docs/latest/api/R/index.html) docs: + +You can also pass in column name as strings. +```{r} +head(select(carsDF, "mpg")) +``` + +Filter the SparkDataFrame to only retain rows with mpg less than 20 miles/gallon. +```{r} +head(filter(carsDF, carsDF$mpg < 20)) +``` + +#### Grouping, Aggregation + +A common flow of grouping and aggregation is + +1. Use `groupBy` or `group_by` with respect to some grouping variables to create a `GroupedData` object + +2. Feed the `GroupedData` object to `agg` or `summarize` functions, with some provided aggregation functions to compute a number within each group. + +A number of widely used functions are supported to aggregate data after grouping, including `avg`, `countDistinct`, `count`, `first`, `kurtosis`, `last`, `max`, `mean`, `min`, `sd`, `skewness`, `stddev_pop`, `stddev_samp`, `sumDistinct`, `sum`, `var_pop`, `var_samp`, `var`. See the [API doc for `mean`](http://spark.apache.org/docs/latest/api/R/mean.html) and other `agg_funcs` linked there. + +For example we can compute a histogram of the number of cylinders in the `mtcars` dataset as shown below. + +```{r} +numCyl <- summarize(groupBy(carsDF, carsDF$cyl), count = n(carsDF$cyl)) +head(numCyl) +``` + +Use `cube` or `rollup` to compute subtotals across multiple dimensions. + +```{r} +mean(cube(carsDF, "cyl", "gear", "am"), "mpg") +``` + +generates groupings for {(`cyl`, `gear`, `am`), (`cyl`, `gear`), (`cyl`), ()}, while + +```{r} +mean(rollup(carsDF, "cyl", "gear", "am"), "mpg") +``` + +generates groupings for all possible combinations of grouping columns. + + +#### Operating on Columns + +SparkR also provides a number of functions that can directly applied to columns for data processing and during aggregation. The example below shows the use of basic arithmetic functions. + +```{r} +carsDF_km <- carsDF +carsDF_km$kmpg <- carsDF_km$mpg * 1.61 +head(select(carsDF_km, "model", "mpg", "kmpg")) +``` + + +### Window Functions +A window function is a variation of aggregation function. In simple words, + +* aggregation function: `n` to `1` mapping - returns a single value for a group of entries. Examples include `sum`, `count`, `max`. + +* window function: `n` to `n` mapping - returns one value for each entry in the group, but the value may depend on all the entries of the *group*. Examples include `rank`, `lead`, `lag`. + +Formally, the *group* mentioned above is called the *frame*. Every input row can have a unique frame associated with it and the output of the window function on that row is based on the rows confined in that frame. + +Window functions are often used in conjunction with the following functions: `windowPartitionBy`, `windowOrderBy`, `partitionBy`, `orderBy`, `over`. To illustrate this we next look at an example. + +We still use the `mtcars` dataset. The corresponding `SparkDataFrame` is `carsDF`. Suppose for each number of cylinders, we want to calculate the rank of each car in `mpg` within the group. +```{r} +carsSubDF <- select(carsDF, "model", "mpg", "cyl") +ws <- orderBy(windowPartitionBy("cyl"), "mpg") +carsRank <- withColumn(carsSubDF, "rank", over(rank(), ws)) +head(carsRank, n = 20L) +``` + +We explain in detail the above steps. + +* `windowPartitionBy` creates a window specification object `WindowSpec` that defines the partition. It controls which rows will be in the same partition as the given row. In this case, rows with the same value in `cyl` will be put in the same partition. `orderBy` further defines the ordering - the position a given row is in the partition. The resulting `WindowSpec` is returned as `ws`. + +More window specification methods include `rangeBetween`, which can define boundaries of the frame by value, and `rowsBetween`, which can define the boundaries by row indices. + +* `withColumn` appends a Column called `rank` to the `SparkDataFrame`. `over` returns a windowing column. The first argument is usually a Column returned by window function(s) such as `rank()`, `lead(carsDF$wt)`. That calculates the corresponding values according to the partitioned-and-ordered table. + +### User-Defined Function + +In SparkR, we support several kinds of user-defined functions (UDFs). + +#### Apply by Partition + +`dapply` can apply a function to each partition of a `SparkDataFrame`. The function to be applied to each partition of the `SparkDataFrame` should have only one parameter, a `data.frame` corresponding to a partition, and the output should be a `data.frame` as well. Schema specifies the row format of the resulting a `SparkDataFrame`. It must match to data types of returned value. See [here](#DataTypes) for mapping between R and Spark. + +We convert `mpg` to `kmpg` (kilometers per gallon). `carsSubDF` is a `SparkDataFrame` with a subset of `carsDF` columns. + +```{r} +carsSubDF <- select(carsDF, "model", "mpg") +schema <- structType(structField("model", "string"), structField("mpg", "double"), + structField("kmpg", "double")) +out <- dapply(carsSubDF, function(x) { x <- cbind(x, x$mpg * 1.61) }, schema) +head(collect(out)) +``` + +Like `dapply`, `dapplyCollect` can apply a function to each partition of a `SparkDataFrame` and collect the result back. The output of the function should be a `data.frame`, but no schema is required in this case. Note that `dapplyCollect` can fail if the output of the UDF on all partitions cannot be pulled into the driver's memory. + +```{r} +out <- dapplyCollect( + carsSubDF, + function(x) { + x <- cbind(x, "kmpg" = x$mpg * 1.61) + }) +head(out, 3) +``` + +#### Apply by Group +`gapply` can apply a function to each group of a `SparkDataFrame`. The function is to be applied to each group of the `SparkDataFrame` and should have only two parameters: grouping key and R `data.frame` corresponding to that key. The groups are chosen from `SparkDataFrames` column(s). The output of function should be a `data.frame`. Schema specifies the row format of the resulting `SparkDataFrame`. It must represent R function’s output schema on the basis of Spark data types. The column names of the returned `data.frame` are set by user. See [here](#DataTypes) for mapping between R and Spark. + +```{r} +schema <- structType(structField("cyl", "double"), structField("max_mpg", "double")) +result <- gapply( + carsDF, + "cyl", + function(key, x) { + y <- data.frame(key, max(x$mpg)) + }, + schema) +head(arrange(result, "max_mpg", decreasing = TRUE)) +``` + +Like `gapply`, `gapplyCollect` can apply a function to each partition of a `SparkDataFrame` and collect the result back to R `data.frame`. The output of the function should be a `data.frame` but no schema is required in this case. Note that `gapplyCollect` can fail if the output of the UDF on all partitions cannot be pulled into the driver's memory. + +```{r} +result <- gapplyCollect( + carsDF, + "cyl", + function(key, x) { + y <- data.frame(key, max(x$mpg)) + colnames(y) <- c("cyl", "max_mpg") + y + }) +head(result[order(result$max_mpg, decreasing = TRUE), ]) +``` + +#### Distribute Local Functions + +Similar to `lapply` in native R, `spark.lapply` runs a function over a list of elements and distributes the computations with Spark. `spark.lapply` works in a manner that is similar to `doParallel` or `lapply` to elements of a list. The results of all the computations should fit in a single machine. If that is not the case you can do something like `df <- createDataFrame(list)` and then use `dapply`. + +We use `svm` in package `e1071` as an example. We use all default settings except for varying costs of constraints violation. `spark.lapply` can train those different models in parallel. + +```{r} +costs <- exp(seq(from = log(1), to = log(1000), length.out = 5)) +train <- function(cost) { + stopifnot(requireNamespace("e1071", quietly = TRUE)) + model <- e1071::svm(Species ~ ., data = iris, cost = cost) + summary(model) +} +``` + +Return a list of model's summaries. +```{r} +model.summaries <- spark.lapply(costs, train) +``` + +```{r} +class(model.summaries) +``` + + +To avoid lengthy display, we only present the partial result of the second fitted model. You are free to inspect other models as well. +```{r, include=FALSE} +ops <- options() +options(max.print=40) +``` +```{r} +print(model.summaries[[2]]) +``` +```{r, include=FALSE} +options(ops) +``` + + +### SQL Queries +A `SparkDataFrame` can also be registered as a temporary view in Spark SQL so that one can run SQL queries over its data. The sql function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. + +```{r} +people <- read.df(paste0(sparkR.conf("spark.home"), + "/examples/src/main/resources/people.json"), "json") +``` + +Register this `SparkDataFrame` as a temporary view. + +```{r} +createOrReplaceTempView(people, "people") +``` + +SQL statements can be run using the sql method. +```{r} +teenagers <- sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +head(teenagers) +``` + + +## Machine Learning + +SparkR supports the following machine learning models and algorithms. + +#### Classification + +* Linear Support Vector Machine (SVM) Classifier + +* Logistic Regression + +* Multilayer Perceptron (MLP) + +* Naive Bayes + +#### Regression + +* Accelerated Failure Time (AFT) Survival Model + +* Generalized Linear Model (GLM) + +* Isotonic Regression + +#### Tree - Classification and Regression + +* Decision Tree + +* Gradient-Boosted Trees (GBT) + +* Random Forest + +#### Clustering + +* Bisecting $k$-means + +* Gaussian Mixture Model (GMM) + +* $k$-means Clustering + +* Latent Dirichlet Allocation (LDA) + +#### Collaborative Filtering + +* Alternating Least Squares (ALS) + +#### Frequent Pattern Mining + +* FP-growth + +#### Statistics + +* Kolmogorov-Smirnov Test + +### R Formula + +For most above, SparkR supports **R formula operators**, including `~`, `.`, `:`, `+` and `-` for model fitting. This makes it a similar experience as using R functions. + +### Training and Test Sets + +We can easily split `SparkDataFrame` into random training and test sets by the `randomSplit` function. It returns a list of split `SparkDataFrames` with provided `weights`. We use `carsDF` as an example and want to have about $70%$ training data and $30%$ test data. +```{r} +splitDF_list <- randomSplit(carsDF, c(0.7, 0.3), seed = 0) +carsDF_train <- splitDF_list[[1]] +carsDF_test <- splitDF_list[[2]] +``` + +```{r} +count(carsDF_train) +head(carsDF_train) +``` + +```{r} +count(carsDF_test) +head(carsDF_test) +``` + +### Models and Algorithms + +#### Linear Support Vector Machine (SVM) Classifier + +[Linear Support Vector Machine (SVM)](https://en.wikipedia.org/wiki/Support_vector_machine#Linear_SVM) classifier is an SVM classifier with linear kernels. +This is a binary classifier. We use a simple example to show how to use `spark.svmLinear` +for binary classification. + +```{r} +# load training data and create a DataFrame +t <- as.data.frame(Titanic) +training <- createDataFrame(t) +# fit a Linear SVM classifier model +model <- spark.svmLinear(training, Survived ~ ., regParam = 0.01, maxIter = 10) +summary(model) +``` + +Predict values on training data +```{r} +prediction <- predict(model, training) +``` + +#### Logistic Regression + +[Logistic regression](https://en.wikipedia.org/wiki/Logistic_regression) is a widely-used model when the response is categorical. It can be seen as a special case of the [Generalized Linear Predictive Model](https://en.wikipedia.org/wiki/Generalized_linear_model). +We provide `spark.logit` on top of `spark.glm` to support logistic regression with advanced hyper-parameters. +It supports both binary and multiclass classification with elastic-net regularization and feature standardization, similar to `glmnet`. + +We use a simple example to demonstrate `spark.logit` usage. In general, there are three steps of using `spark.logit`: +1). Create a dataframe from a proper data source; 2). Fit a logistic regression model using `spark.logit` with a proper parameter setting; +and 3). Obtain the coefficient matrix of the fitted model using `summary` and use the model for prediction with `predict`. + +Binomial logistic regression +```{r} +t <- as.data.frame(Titanic) +training <- createDataFrame(t) +model <- spark.logit(training, Survived ~ ., regParam = 0.04741301) +summary(model) +``` + +Predict values on training data +```{r} +fitted <- predict(model, training) +``` + +Multinomial logistic regression against three classes +```{r} +t <- as.data.frame(Titanic) +training <- createDataFrame(t) +# Note in this case, Spark infers it is multinomial logistic regression, so family = "multinomial" is optional. +model <- spark.logit(training, Class ~ ., regParam = 0.07815179) +summary(model) +``` + +#### Multilayer Perceptron + +Multilayer perceptron classifier (MLPC) is a classifier based on the [feedforward artificial neural network](https://en.wikipedia.org/wiki/Feedforward_neural_network). MLPC consists of multiple layers of nodes. Each layer is fully connected to the next layer in the network. Nodes in the input layer represent the input data. All other nodes map inputs to outputs by a linear combination of the inputs with the node’s weights $w$ and bias $b$ and applying an activation function. This can be written in matrix form for MLPC with $K+1$ layers as follows: +$$ +y(x)=f_K(\ldots f_2(w_2^T f_1(w_1^T x + b_1) + b_2) \ldots + b_K). +$$ + +Nodes in intermediate layers use sigmoid (logistic) function: +$$ +f(z_i) = \frac{1}{1+e^{-z_i}}. +$$ + +Nodes in the output layer use softmax function: +$$ +f(z_i) = \frac{e^{z_i}}{\sum_{k=1}^N e^{z_k}}. +$$ + +The number of nodes $N$ in the output layer corresponds to the number of classes. + +MLPC employs backpropagation for learning the model. We use the logistic loss function for optimization and L-BFGS as an optimization routine. + +`spark.mlp` requires at least two columns in `data`: one named `"label"` and the other one `"features"`. The `"features"` column should be in libSVM-format. + +We use Titanic data set to show how to use `spark.mlp` in classification. +```{r} +t <- as.data.frame(Titanic) +training <- createDataFrame(t) +# fit a Multilayer Perceptron Classification Model +model <- spark.mlp(training, Survived ~ Age + Sex, blockSize = 128, layers = c(2, 3), solver = "l-bfgs", maxIter = 100, tol = 0.5, stepSize = 1, seed = 1, initialWeights = c( 0, 0, 0, 5, 5, 5, 9, 9, 9)) +``` + +To avoid lengthy display, we only present partial results of the model summary. You can check the full result from your sparkR shell. +```{r, include=FALSE} +ops <- options() +options(max.print=5) +``` +```{r} +# check the summary of the fitted model +summary(model) +``` +```{r, include=FALSE} +options(ops) +``` +```{r} +# make predictions use the fitted model +predictions <- predict(model, training) +head(select(predictions, predictions$prediction)) +``` + +#### Naive Bayes + +Naive Bayes model assumes independence among the features. `spark.naiveBayes` fits a [Bernoulli naive Bayes model](https://en.wikipedia.org/wiki/Naive_Bayes_classifier#Bernoulli_naive_Bayes) against a SparkDataFrame. The data should be all categorical. These models are often used for document classification. + +```{r} +titanic <- as.data.frame(Titanic) +titanicDF <- createDataFrame(titanic[titanic$Freq > 0, -5]) +naiveBayesModel <- spark.naiveBayes(titanicDF, Survived ~ Class + Sex + Age) +summary(naiveBayesModel) +naiveBayesPrediction <- predict(naiveBayesModel, titanicDF) +head(select(naiveBayesPrediction, "Class", "Sex", "Age", "Survived", "prediction")) +``` + +#### Accelerated Failure Time Survival Model + +Survival analysis studies the expected duration of time until an event happens, and often the relationship with risk factors or treatment taken on the subject. In contrast to standard regression analysis, survival modeling has to deal with special characteristics in the data including non-negative survival time and censoring. + +Accelerated Failure Time (AFT) model is a parametric survival model for censored data that assumes the effect of a covariate is to accelerate or decelerate the life course of an event by some constant. For more information, refer to the Wikipedia page [AFT Model](https://en.wikipedia.org/wiki/Accelerated_failure_time_model) and the references there. Different from a [Proportional Hazards Model](https://en.wikipedia.org/wiki/Proportional_hazards_model) designed for the same purpose, the AFT model is easier to parallelize because each instance contributes to the objective function independently. + +```{r, warning=FALSE} +library(survival) +ovarianDF <- createDataFrame(ovarian) +aftModel <- spark.survreg(ovarianDF, Surv(futime, fustat) ~ ecog_ps + rx) +summary(aftModel) +aftPredictions <- predict(aftModel, ovarianDF) +head(aftPredictions) +``` + +#### Generalized Linear Model + +The main function is `spark.glm`. The following families and link functions are supported. The default is gaussian. + +Family | Link Function +------ | --------- +gaussian | identity, log, inverse +binomial | logit, probit, cloglog (complementary log-log) +poisson | log, identity, sqrt +gamma | inverse, identity, log +tweedie | power link function + +There are three ways to specify the `family` argument. + +* Family name as a character string, e.g. `family = "gaussian"`. + +* Family function, e.g. `family = binomial`. + +* Result returned by a family function, e.g. `family = poisson(link = log)`. + +* Note that there are two ways to specify the tweedie family: + a) Set `family = "tweedie"` and specify the `var.power` and `link.power` + b) When package `statmod` is loaded, the tweedie family is specified using the family definition therein, i.e., `tweedie()`. + +For more information regarding the families and their link functions, see the Wikipedia page [Generalized Linear Model](https://en.wikipedia.org/wiki/Generalized_linear_model). + +We use the `mtcars` dataset as an illustration. The corresponding `SparkDataFrame` is `carsDF`. After fitting the model, we print out a summary and see the fitted values by making predictions on the original dataset. We can also pass into a new `SparkDataFrame` of same schema to predict on new data. + +```{r} +gaussianGLM <- spark.glm(carsDF, mpg ~ wt + hp) +summary(gaussianGLM) +``` +When doing prediction, a new column called `prediction` will be appended. Let's look at only a subset of columns here. +```{r} +gaussianFitted <- predict(gaussianGLM, carsDF) +head(select(gaussianFitted, "model", "prediction", "mpg", "wt", "hp")) +``` + +The following is the same fit using the tweedie family: +```{r} +tweedieGLM1 <- spark.glm(carsDF, mpg ~ wt + hp, family = "tweedie", var.power = 0.0) +summary(tweedieGLM1) +``` +We can try other distributions in the tweedie family, for example, a compound Poisson distribution with a log link: +```{r} +tweedieGLM2 <- spark.glm(carsDF, mpg ~ wt + hp, family = "tweedie", + var.power = 1.2, link.power = 0.0) +summary(tweedieGLM2) +``` + +#### Isotonic Regression + +`spark.isoreg` fits an [Isotonic Regression](https://en.wikipedia.org/wiki/Isotonic_regression) model against a `SparkDataFrame`. It solves a weighted univariate a regression problem under a complete order constraint. Specifically, given a set of real observed responses $y_1, \ldots, y_n$, corresponding real features $x_1, \ldots, x_n$, and optionally positive weights $w_1, \ldots, w_n$, we want to find a monotone (piecewise linear) function $f$ to minimize +$$ +\ell(f) = \sum_{i=1}^n w_i (y_i - f(x_i))^2. +$$ + +There are a few more arguments that may be useful. + +* `weightCol`: a character string specifying the weight column. + +* `isotonic`: logical value indicating whether the output sequence should be isotonic/increasing (`TRUE`) or antitonic/decreasing (`FALSE`). + +* `featureIndex`: the index of the feature on the right hand side of the formula if it is a vector column (default: 0), no effect otherwise. + +We use an artificial example to show the use. + +```{r} +y <- c(3.0, 6.0, 8.0, 5.0, 7.0) +x <- c(1.0, 2.0, 3.5, 3.0, 4.0) +w <- rep(1.0, 5) +data <- data.frame(y = y, x = x, w = w) +df <- createDataFrame(data) +isoregModel <- spark.isoreg(df, y ~ x, weightCol = "w") +isoregFitted <- predict(isoregModel, df) +head(select(isoregFitted, "x", "y", "prediction")) +``` + +In the prediction stage, based on the fitted monotone piecewise function, the rules 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. + +* 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. + +For example, when the input is $3.2$, the two closest feature values are $3.0$ and $3.5$, then predicted value would be a linear interpolation between the predicted values at $3.0$ and $3.5$. + +```{r} +newDF <- createDataFrame(data.frame(x = c(1.5, 3.2))) +head(predict(isoregModel, newDF)) +``` + +#### Decision Tree + +`spark.decisionTree` fits a [decision tree](https://en.wikipedia.org/wiki/Decision_tree_learning) classification or regression model on a `SparkDataFrame`. +Users can call `summary` to get a summary of the fitted model, `predict` to make predictions, and `write.ml`/`read.ml` to save/load fitted models. + +We use the `Titanic` dataset to train a decision tree and make predictions: + +```{r} +t <- as.data.frame(Titanic) +df <- createDataFrame(t) +dtModel <- spark.decisionTree(df, Survived ~ ., type = "classification", maxDepth = 2) +summary(dtModel) +predictions <- predict(dtModel, df) +``` + +#### Gradient-Boosted Trees + +`spark.gbt` fits a [gradient-boosted tree](https://en.wikipedia.org/wiki/Gradient_boosting) classification or regression model on a `SparkDataFrame`. +Users can call `summary` to get a summary of the fitted model, `predict` to make predictions, and `write.ml`/`read.ml` to save/load fitted models. + +We use the `Titanic` dataset to train a gradient-boosted tree and make predictions: + +```{r} +t <- as.data.frame(Titanic) +df <- createDataFrame(t) +gbtModel <- spark.gbt(df, Survived ~ ., type = "classification", maxDepth = 2, maxIter = 2) +summary(gbtModel) +predictions <- predict(gbtModel, df) +``` + +#### Random Forest + +`spark.randomForest` fits a [random forest](https://en.wikipedia.org/wiki/Random_forest) classification or regression model on a `SparkDataFrame`. +Users can call `summary` to get a summary of the fitted model, `predict` to make predictions, and `write.ml`/`read.ml` to save/load fitted models. + +In the following example, we use the `Titanic` dataset to train a random forest and make predictions: + +```{r} +t <- as.data.frame(Titanic) +df <- createDataFrame(t) +rfModel <- spark.randomForest(df, Survived ~ ., type = "classification", maxDepth = 2, numTrees = 2) +summary(rfModel) +predictions <- predict(rfModel, df) +``` + +#### Bisecting k-Means + +`spark.bisectingKmeans` is a kind of [hierarchical clustering](https://en.wikipedia.org/wiki/Hierarchical_clustering) using a divisive (or "top-down") approach: all observations start in one cluster, and splits are performed recursively as one moves down the hierarchy. + +```{r} +t <- as.data.frame(Titanic) +training <- createDataFrame(t) +model <- spark.bisectingKmeans(training, Class ~ Survived, k = 4) +summary(model) +fitted <- predict(model, training) +head(select(fitted, "Class", "prediction")) +``` + +#### Gaussian Mixture Model + +`spark.gaussianMixture` fits multivariate [Gaussian Mixture Model](https://en.wikipedia.org/wiki/Mixture_model#Multivariate_Gaussian_mixture_model) (GMM) against a `SparkDataFrame`. [Expectation-Maximization](https://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm) (EM) is used to approximate the maximum likelihood estimator (MLE) of the model. + +We use a simulated example to demonstrate the usage. +```{r} +X1 <- data.frame(V1 = rnorm(4), V2 = rnorm(4)) +X2 <- data.frame(V1 = rnorm(6, 3), V2 = rnorm(6, 4)) +data <- rbind(X1, X2) +df <- createDataFrame(data) +gmmModel <- spark.gaussianMixture(df, ~ V1 + V2, k = 2) +summary(gmmModel) +gmmFitted <- predict(gmmModel, df) +head(select(gmmFitted, "V1", "V2", "prediction")) +``` + +#### k-Means Clustering + +`spark.kmeans` fits a $k$-means clustering model against a `SparkDataFrame`. As an unsupervised learning method, we don't need a response variable. Hence, the left hand side of the R formula should be left blank. The clustering is based only on the variables on the right hand side. + +```{r} +kmeansModel <- spark.kmeans(carsDF, ~ mpg + hp + wt, k = 3) +summary(kmeansModel) +kmeansPredictions <- predict(kmeansModel, carsDF) +head(select(kmeansPredictions, "model", "mpg", "hp", "wt", "prediction"), n = 20L) +``` + +#### Latent Dirichlet Allocation + +`spark.lda` fits a [Latent Dirichlet Allocation](https://en.wikipedia.org/wiki/Latent_Dirichlet_allocation) model on a `SparkDataFrame`. It is often used in topic modeling in which topics are inferred 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 (bag of words). + +* Rather than clustering using a traditional distance, LDA uses a function based on a statistical model of how text documents are generated. + +To use LDA, we need to specify a `features` column in `data` where each entry represents a document. There are two options for the column: + +* character string: This can be a string of the whole document. It will be parsed automatically. Additional stop words can be added in `customizedStopWords`. + +* libSVM: Each entry is a collection of words and will be processed directly. + +Two more functions are provided for the fitted model. + +* `spark.posterior` returns a `SparkDataFrame` containing a column of posterior probabilities vectors named "topicDistribution". + +* `spark.perplexity` returns the log perplexity of given `SparkDataFrame`, or the log perplexity of the training data if missing argument `data`. + +For more information, see the help document `?spark.lda`. + +Let's look an artificial example. +```{r} +corpus <- data.frame(features = c( + "1 2 6 0 2 3 1 1 0 0 3", + "1 3 0 1 3 0 0 2 0 0 1", + "1 4 1 0 0 4 9 0 1 2 0", + "2 1 0 3 0 0 5 0 2 3 9", + "3 1 1 9 3 0 2 0 0 1 3", + "4 2 0 3 4 5 1 1 1 4 0", + "2 1 0 3 0 0 5 0 2 2 9", + "1 1 1 9 2 1 2 0 0 1 3", + "4 4 0 3 4 2 1 3 0 0 0", + "2 8 2 0 3 0 2 0 2 7 2", + "1 1 1 9 0 2 2 0 0 3 3", + "4 1 0 0 4 5 1 3 0 1 0")) +corpusDF <- createDataFrame(corpus) +model <- spark.lda(data = corpusDF, k = 5, optimizer = "em") +summary(model) +``` + +```{r} +posterior <- spark.posterior(model, corpusDF) +head(posterior) +``` + +```{r} +perplexity <- spark.perplexity(model, corpusDF) +perplexity +``` + +#### Alternating Least Squares + +`spark.als` learns latent factors in [collaborative filtering](https://en.wikipedia.org/wiki/Recommender_system#Collaborative_filtering) via [alternating least squares](http://dl.acm.org/citation.cfm?id=1608614). + +There are multiple options that can be configured in `spark.als`, including `rank`, `reg`, and `nonnegative`. For a complete list, refer to the help file. + +```{r, eval=FALSE} +ratings <- list(list(0, 0, 4.0), list(0, 1, 2.0), list(1, 1, 3.0), list(1, 2, 4.0), + list(2, 1, 1.0), list(2, 2, 5.0)) +df <- createDataFrame(ratings, c("user", "item", "rating")) +model <- spark.als(df, "rating", "user", "item", rank = 10, reg = 0.1, nonnegative = TRUE) +``` + +Extract latent factors. +```{r, eval=FALSE} +stats <- summary(model) +userFactors <- stats$userFactors +itemFactors <- stats$itemFactors +head(userFactors) +head(itemFactors) +``` + +Make predictions. + +```{r, eval=FALSE} +predicted <- predict(model, df) +head(predicted) +``` + +#### FP-growth + +`spark.fpGrowth` executes FP-growth algorithm to mine frequent itemsets on a `SparkDataFrame`. `itemsCol` should be an array of values. + +```{r} +df <- selectExpr(createDataFrame(data.frame(rawItems = c( + "T,R,U", "T,S", "V,R", "R,U,T,V", "R,S", "V,S,U", "U,R", "S,T", "V,R", "V,U,S", + "T,V,U", "R,V", "T,S", "T,S", "S,T", "S,U", "T,R", "V,R", "S,V", "T,S,U" +))), "split(rawItems, ',') AS items") + +fpm <- spark.fpGrowth(df, minSupport = 0.2, minConfidence = 0.5) +``` + +`spark.freqItemsets` method can be used to retrieve a `SparkDataFrame` with the frequent itemsets. + +```{r} +head(spark.freqItemsets(fpm)) +``` + +`spark.associationRules` returns a `SparkDataFrame` with the association rules. + +```{r} +head(spark.associationRules(fpm)) +``` + +We can make predictions based on the `antecedent`. + +```{r} +head(predict(fpm, df)) +``` + +#### Kolmogorov-Smirnov Test + +`spark.kstest` runs a two-sided, one-sample [Kolmogorov-Smirnov (KS) test](https://en.wikipedia.org/wiki/Kolmogorov%E2%80%93Smirnov_test). +Given a `SparkDataFrame`, the test compares continuous data in a given column `testCol` with the theoretical distribution +specified by parameter `nullHypothesis`. +Users can call `summary` to get a summary of the test results. + +In the following example, we test whether the `Titanic` dataset's `Freq` column +follows a normal distribution. We set the parameters of the normal distribution using +the mean and standard deviation of the sample. + +```{r} +t <- as.data.frame(Titanic) +df <- createDataFrame(t) +freqStats <- head(select(df, mean(df$Freq), sd(df$Freq))) +freqMean <- freqStats[1] +freqStd <- freqStats[2] + +test <- spark.kstest(df, "Freq", "norm", c(freqMean, freqStd)) +testSummary <- summary(test) +testSummary +``` + + +### Model Persistence +The following example shows how to save/load an ML model in SparkR. +```{r} +t <- as.data.frame(Titanic) +training <- createDataFrame(t) +gaussianGLM <- spark.glm(training, Freq ~ Sex + Age, family = "gaussian") + +# Save and then load a fitted MLlib model +modelPath <- tempfile(pattern = "ml", fileext = ".tmp") +write.ml(gaussianGLM, modelPath) +gaussianGLM2 <- read.ml(modelPath) + +# Check model summary +summary(gaussianGLM2) + +# Check model prediction +gaussianPredictions <- predict(gaussianGLM2, training) +head(gaussianPredictions) + +unlink(modelPath) +``` + + +## Structured Streaming + +SparkR supports the Structured Streaming API (experimental). + +You can check the Structured Streaming Programming Guide for [an introduction](https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html#programming-model) to its programming model and basic concepts. + +### Simple Source and Sink + +Spark has a few built-in input sources. As an example, to test with a socket source reading text into words and displaying the computed word counts: + +```{r, eval=FALSE} +# Create DataFrame representing the stream of input lines from connection +lines <- read.stream("socket", host = hostname, port = port) + +# Split the lines into words +words <- selectExpr(lines, "explode(split(value, ' ')) as word") + +# Generate running word count +wordCounts <- count(groupBy(words, "word")) + +# Start running the query that prints the running counts to the console +query <- write.stream(wordCounts, "console", outputMode = "complete") +``` + +### Kafka Source + +It is simple to read data from Kafka. For more information, see [Input Sources](https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html#input-sources) supported by Structured Streaming. + +```{r, eval=FALSE} +topic <- read.stream("kafka", + kafka.bootstrap.servers = "host1:port1,host2:port2", + subscribe = "topic1") +keyvalue <- selectExpr(topic, "CAST(key AS STRING)", "CAST(value AS STRING)") +``` + +### Operations and Sinks + +Most of the common operations on `SparkDataFrame` are supported for streaming, including selection, projection, and aggregation. Once you have defined the final result, to start the streaming computation, you will call the `write.stream` method setting a sink and `outputMode`. + +A streaming `SparkDataFrame` can be written for debugging to the console, to a temporary in-memory table, or for further processing in a fault-tolerant manner to a File Sink in different formats. + +```{r, eval=FALSE} +noAggDF <- select(where(deviceDataStreamingDf, "signal > 10"), "device") + +# Print new data to console +write.stream(noAggDF, "console") + +# Write new data to Parquet files +write.stream(noAggDF, + "parquet", + path = "path/to/destination/dir", + checkpointLocation = "path/to/checkpoint/dir") + +# Aggregate +aggDF <- count(groupBy(noAggDF, "device")) + +# Print updated aggregations to console +write.stream(aggDF, "console", outputMode = "complete") + +# Have all the aggregates in an in memory table. The query name will be the table name +write.stream(aggDF, "memory", queryName = "aggregates", outputMode = "complete") + +head(sql("select * from aggregates")) +``` + + +## Advanced Topics + +### SparkR Object Classes + +There are three main object classes in SparkR you may be working with. + +* `SparkDataFrame`: the central component of SparkR. It is an S4 class representing distributed collection of data organized into named columns, which is conceptually equivalent to a table in a relational database or a data frame in R. It has two slots `sdf` and `env`. + + `sdf` stores a reference to the corresponding Spark Dataset in the Spark JVM backend. + + `env` saves the meta-information of the object such as `isCached`. + + It can be created by data import methods or by transforming an existing `SparkDataFrame`. We can manipulate `SparkDataFrame` by numerous data processing functions and feed that into machine learning algorithms. + +* `Column`: an S4 class representing a column of `SparkDataFrame`. The slot `jc` saves a reference to the corresponding `Column` object in the Spark JVM backend. + + It can be obtained from a `SparkDataFrame` by `$` operator, e.g., `df$col`. More often, it is used together with other functions, for example, with `select` to select particular columns, with `filter` and constructed conditions to select rows, with aggregation functions to compute aggregate statistics for each group. + +* `GroupedData`: an S4 class representing grouped data created by `groupBy` or by transforming other `GroupedData`. Its `sgd` slot saves a reference to a `RelationalGroupedDataset` object in the backend. + + This is often an intermediate object with group information and followed up by aggregation operations. + +### Architecture + +A complete description of architecture can be seen in the references, in particular the paper *SparkR: Scaling R Programs with Spark*. + +Under the hood of SparkR is Spark SQL engine. This avoids the overheads of running interpreted R code, and the optimized SQL execution engine in Spark uses structural information about data and computation flow to perform a bunch of optimizations to speed up the computation. + +The main method calls of actual computation happen in the Spark JVM of the driver. We have a socket-based SparkR API that allows us to invoke functions on the JVM from R. We use a SparkR JVM backend that listens on a Netty-based socket server. + +Two kinds of RPCs are supported in the SparkR JVM backend: method invocation and creating new objects. Method invocation can be done in two ways. + +* `sparkR.callJMethod` takes a reference to an existing Java object and a list of arguments to be passed on to the method. + +* `sparkR.callJStatic` takes a class name for static method and a list of arguments to be passed on to the method. + +The arguments are serialized using our custom wire format which is then deserialized on the JVM side. We then use Java reflection to invoke the appropriate method. + +To create objects, `sparkR.newJObject` is used and then similarly the appropriate constructor is invoked with provided arguments. + +Finally, we use a new R class `jobj` that refers to a Java object existing in the backend. These references are tracked on the Java side and are automatically garbage collected when they go out of scope on the R side. + +## Appendix + +### R and Spark Data Types {#DataTypes} + +R | Spark +----------- | ------------- +byte | byte +integer | integer +float | float +double | double +numeric | double +character | string +string | string +binary | binary +raw | binary +logical | boolean +POSIXct | timestamp +POSIXlt | timestamp +Date | date +array | array +list | array +env | map + +## References + +* [Spark Cluster Mode Overview](http://spark.apache.org/docs/latest/cluster-overview.html) + +* [Submitting Spark Applications](http://spark.apache.org/docs/latest/submitting-applications.html) + +* [Machine Learning Library Guide (MLlib)](http://spark.apache.org/docs/latest/ml-guide.html) + +* [SparkR: Scaling R Programs with Spark](https://people.csail.mit.edu/matei/papers/2016/sigmod_sparkr.pdf), Shivaram Venkataraman, Zongheng Yang, Davies Liu, Eric Liang, Hossein Falaki, Xiangrui Meng, Reynold Xin, Ali Ghodsi, Michael Franklin, Ion Stoica, and Matei Zaharia. SIGMOD 2016. June 2016. + +```{r, echo=FALSE} +sparkR.session.stop() +``` diff --git a/R/run-tests.sh b/R/run-tests.sh index e82ad0ba2cd0..29764f48bd15 100755 --- a/R/run-tests.sh +++ b/R/run-tests.sh @@ -23,17 +23,40 @@ FAILED=0 LOGFILE=$FWDIR/unit-tests.out rm -f $LOGFILE -SPARK_TESTING=1 $FWDIR/../bin/sparkR --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE +SPARK_TESTING=1 NOT_CRAN=true $FWDIR/../bin/spark-submit --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" --conf spark.hadoop.fs.defaultFS="file:///" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE FAILED=$((PIPESTATUS[0]||$FAILED)) -if [[ $FAILED != 0 ]]; then +NUM_TEST_WARNING="$(grep -c -e 'Warnings ----------------' $LOGFILE)" + +# Also run the documentation tests for CRAN +CRAN_CHECK_LOG_FILE=$FWDIR/cran-check.out +rm -f $CRAN_CHECK_LOG_FILE + +NO_TESTS=1 NO_MANUAL=1 $FWDIR/check-cran.sh 2>&1 | tee -a $CRAN_CHECK_LOG_FILE +FAILED=$((PIPESTATUS[0]||$FAILED)) + +NUM_CRAN_WARNING="$(grep -c WARNING$ $CRAN_CHECK_LOG_FILE)" +NUM_CRAN_ERROR="$(grep -c ERROR$ $CRAN_CHECK_LOG_FILE)" +NUM_CRAN_NOTES="$(grep -c NOTE$ $CRAN_CHECK_LOG_FILE)" + +if [[ $FAILED != 0 || $NUM_TEST_WARNING != 0 ]]; then cat $LOGFILE echo -en "\033[31m" # Red - echo "Had test failures; see logs." + echo "Had test warnings or failures; see logs." echo -en "\033[0m" # No color exit -1 else - echo -en "\033[32m" # Green - echo "Tests passed." - echo -en "\033[0m" # No color + # We have 2 existing NOTEs for new maintainer, attach() + # We have one more NOTE in Jenkins due to "No repository set" + if [[ $NUM_CRAN_WARNING != 0 || $NUM_CRAN_ERROR != 0 || $NUM_CRAN_NOTES -gt 3 ]]; then + cat $CRAN_CHECK_LOG_FILE + echo -en "\033[31m" # Red + echo "Had CRAN check errors; see logs." + echo -en "\033[0m" # No color + exit -1 + else + echo -en "\033[32m" # Green + echo "Tests passed." + echo -en "\033[0m" # No color + fi fi diff --git a/README.md b/README.md index c0d6a946035a..1e521a7e7b17 100644 --- a/README.md +++ b/README.md @@ -13,8 +13,7 @@ and Spark Streaming for stream processing. ## Online Documentation You can find the latest Spark documentation, including a programming -guide, on the [project web page](http://spark.apache.org/documentation.html) -and [project wiki](https://cwiki.apache.org/confluence/display/SPARK). +guide, on the [project web page](http://spark.apache.org/documentation.html). This README file only contains basic setup instructions. ## Building Spark @@ -25,9 +24,13 @@ To build Spark and its example programs, run: build/mvn -DskipTests clean package (You do not need to do this if you downloaded a pre-built package.) + +You can build Spark using more than one thread by using the -T option with Maven, see ["Parallel builds in Maven 3"](https://cwiki.apache.org/confluence/display/MAVEN/Parallel+builds+in+Maven+3). More detailed documentation is available from the project site, at ["Building Spark"](http://spark.apache.org/docs/latest/building-spark.html). +For general development tips, including info on developing Spark using an IDE, see ["Useful Developer Tools"](http://spark.apache.org/developer-tools.html). + ## Interactive Scala Shell The easiest way to start using Spark is through the Scala shell: @@ -76,7 +79,7 @@ can be run using: ./dev/run-tests Please see the guidance on how to -[run tests for a module, or individual tests](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools). +[run tests for a module, or individual tests](http://spark.apache.org/developer-tools.html#individual-tests). ## A Note About Hadoop Versions @@ -93,3 +96,8 @@ building for particular Hive and Hive Thriftserver distributions. Please refer to the [Configuration Guide](http://spark.apache.org/docs/latest/configuration.html) in the online documentation for an overview on how to configure Spark. + +## Contributing + +Please review the [Contribution to Spark guide](http://spark.apache.org/contributing.html) +for information on how to get started contributing to the project. diff --git a/appveyor.yml b/appveyor.yml new file mode 100644 index 000000000000..dc2d81fcdc09 --- /dev/null +++ b/appveyor.yml @@ -0,0 +1,60 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +version: "{build}-{branch}" + +shallow_clone: true + +platform: x64 +configuration: Debug + +branches: + only: + - master + +only_commits: + files: + - appveyor.yml + - dev/appveyor-install-dependencies.ps1 + - R/ + - sql/core/src/main/scala/org/apache/spark/sql/api/r/ + - core/src/main/scala/org/apache/spark/api/r/ + - mllib/src/main/scala/org/apache/spark/ml/r/ + - core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala + +cache: + - C:\Users\appveyor\.m2 + +install: + # Install maven and dependencies + - ps: .\dev\appveyor-install-dependencies.ps1 + # Required package for R unit tests + - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'testthat', 'e1071', 'survival'), repos='http://cran.us.r-project.org')" + - cmd: R -e "packageVersion('knitr'); packageVersion('rmarkdown'); packageVersion('testthat'); packageVersion('e1071'); packageVersion('survival')" + +build_script: + - cmd: mvn -DskipTests -Psparkr -Phive -Phive-thriftserver package + +environment: + NOT_CRAN: true + +test_script: + - cmd: .\bin\spark-submit2.cmd --driver-java-options "-Dlog4j.configuration=file:///%CD:\=/%/R/log4j.properties" --conf spark.hadoop.fs.defaultFS="file:///" R\pkg\tests\run-all.R + +notifications: + - provider: Email + on_build_success: false + on_build_failure: false + on_build_status_changed: false diff --git a/assembly/README b/assembly/README index 14a5ff8dfc78..d5dafab47741 100644 --- a/assembly/README +++ b/assembly/README @@ -9,4 +9,4 @@ This module is off by default. To activate it specify the profile in the command If you need to build an assembly for a different version of Hadoop the hadoop-version system property needs to be set as in this example: - -Dhadoop.version=2.0.6-alpha + -Dhadoop.version=2.7.3 diff --git a/assembly/pom.xml b/assembly/pom.xml index 4b60ee00ffbe..01fe354235e5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -20,33 +20,33 @@ 4.0.0 org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT + spark-parent_2.11 + 2.3.0-SNAPSHOT ../pom.xml - org.apache.spark - spark-assembly_2.10 + spark-assembly_2.11 Spark Project Assembly http://spark.apache.org/ pom assembly - scala-${scala.binary.version} - spark-assembly-${project.version}-hadoop${hadoop.version}.jar - ${project.build.directory}/${spark.jar.dir}/${spark.jar.basename} + none + package + - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} + org.spark-project.spark + unused + 1.0.0 + provided org.apache.spark - spark-bagel_${scala.binary.version} + spark-core_${scala.binary.version} ${project.version} @@ -74,6 +74,17 @@ spark-repl_${scala.binary.version} ${project.version} + + + + com.google.guava + guava + ${hadoop.deps.scope} + @@ -92,75 +103,26 @@ true - - - org.apache.maven.plugins - maven-antrun-plugin - - - package - - run - - - - - - - - - - - - - + org.apache.maven.plugins - maven-shade-plugin - - false - ${spark.jar} - - - *:* - - - - - *:* - - org/datanucleus/** - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - package - - shade - - - - - - META-INF/services/org.apache.hadoop.fs.FileSystem - - - reference.conf - - - log4j.properties - - - - - - - + maven-antrun-plugin + + + package + + run + + + + + + + + + + + @@ -176,6 +138,16 @@ + + mesos + + + org.apache.spark + spark-mesos_${scala.binary.version} + ${project.version} + + + hive @@ -215,6 +187,7 @@ org.apache.maven.plugins maven-assembly-plugin + 3.1.0 dist @@ -247,11 +220,31 @@ provided + + orc-provided + + provided + + parquet-provided provided + + + + hadoop-cloud + + + org.apache.spark + spark-hadoop-cloud_${scala.binary.version} + ${project.version} + + + diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml index 711156337b7c..009d4b92f406 100644 --- a/assembly/src/main/assembly/assembly.xml +++ b/assembly/src/main/assembly/assembly.xml @@ -32,7 +32,7 @@ ${project.parent.basedir}/core/src/main/resources/org/apache/spark/ui/static/ - /ui-resources/org/apache/spark/ui/static + ui-resources/org/apache/spark/ui/static **/* @@ -41,7 +41,7 @@ ${project.parent.basedir}/sbin/ - /sbin + sbin **/* @@ -50,7 +50,7 @@ ${project.parent.basedir}/bin/ - /bin + bin **/* @@ -59,7 +59,7 @@ ${project.parent.basedir}/assembly/target/${spark.jar.dir} - / + ${spark.jar.basename} diff --git a/bagel/pom.xml b/bagel/pom.xml deleted file mode 100644 index 672e9469aec9..000000000000 --- a/bagel/pom.xml +++ /dev/null @@ -1,64 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../pom.xml - - - org.apache.spark - spark-bagel_2.10 - - bagel - - jar - Spark Project Bagel - http://spark.apache.org/ - - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala deleted file mode 100644 index 8399033ac61e..000000000000 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ /dev/null @@ -1,318 +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.bagel - -import org.apache.spark._ -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel - -@deprecated("Uses of Bagel should migrate to GraphX", "1.6.0") -object Bagel extends Logging { - val DEFAULT_STORAGE_LEVEL = StorageLevel.MEMORY_AND_DISK - - /** - * Runs a Bagel program. - * @param sc org.apache.spark.SparkContext to use for the program. - * @param vertices vertices of the graph represented as an RDD of (Key, Vertex) pairs. Often the - * Key will be the vertex id. - * @param messages initial set of messages represented as an RDD of (Key, Message) pairs. Often - * this will be an empty array, i.e. sc.parallelize(Array[K, Message]()). - * @param combiner [[org.apache.spark.bagel.Combiner]] combines multiple individual messages to a - * given vertex into one message before sending (which often involves network - * I/O). - * @param aggregator [[org.apache.spark.bagel.Aggregator]] performs a reduce across all vertices - * after each superstep and provides the result to each vertex in the next - * superstep. - * @param partitioner org.apache.spark.Partitioner partitions values by key - * @param numPartitions number of partitions across which to split the graph. - * Default is the default parallelism of the SparkContext - * @param storageLevel org.apache.spark.storage.StorageLevel to use for caching of - * intermediate RDDs in each superstep. Defaults to caching in memory. - * @param compute function that takes a Vertex, optional set of (possibly combined) messages to - * the Vertex, optional Aggregator and the current superstep, - * and returns a set of (Vertex, outgoing Messages) pairs - * @tparam K key - * @tparam V vertex type - * @tparam M message type - * @tparam C combiner - * @tparam A aggregator - * @return an RDD of (K, V) pairs representing the graph after completion of the program - */ - def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, - C: Manifest, A: Manifest]( - sc: SparkContext, - vertices: RDD[(K, V)], - messages: RDD[(K, M)], - combiner: Combiner[M, C], - aggregator: Option[Aggregator[V, A]], - partitioner: Partitioner, - numPartitions: Int, - storageLevel: StorageLevel = DEFAULT_STORAGE_LEVEL - )( - compute: (V, Option[C], Option[A], Int) => (V, Array[M]) - ): RDD[(K, V)] = { - val splits = if (numPartitions != 0) numPartitions else sc.defaultParallelism - - var superstep = 0 - var verts = vertices - var msgs = messages - var noActivity = false - var lastRDD: RDD[(K, (V, Array[M]))] = null - do { - logInfo("Starting superstep " + superstep + ".") - val startTime = System.currentTimeMillis - - val aggregated = agg(verts, aggregator) - val combinedMsgs = msgs.combineByKeyWithClassTag( - combiner.createCombiner _, combiner.mergeMsg _, combiner.mergeCombiners _, partitioner) - val grouped = combinedMsgs.groupWith(verts) - val superstep_ = superstep // Create a read-only copy of superstep for capture in closure - val (processed, numMsgs, numActiveVerts) = - comp[K, V, M, C](sc, grouped, compute(_, _, aggregated, superstep_), storageLevel) - if (lastRDD != null) { - lastRDD.unpersist(false) - } - lastRDD = processed - - val timeTaken = System.currentTimeMillis - startTime - logInfo("Superstep %d took %d s".format(superstep, timeTaken / 1000)) - - verts = processed.mapValues { case (vert, msgs) => vert } - msgs = processed.flatMap { - case (id, (vert, msgs)) => msgs.map(m => (m.targetId, m)) - } - superstep += 1 - - noActivity = numMsgs == 0 && numActiveVerts == 0 - } while (!noActivity) - - verts - } - - /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the default - * storage level */ - def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( - sc: SparkContext, - vertices: RDD[(K, V)], - messages: RDD[(K, M)], - combiner: Combiner[M, C], - partitioner: Partitioner, - numPartitions: Int - )( - compute: (V, Option[C], Int) => (V, Array[M])): RDD[(K, V)] = run(sc, vertices, messages, - combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute) - - /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] */ - def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( - sc: SparkContext, - vertices: RDD[(K, V)], - messages: RDD[(K, M)], - combiner: Combiner[M, C], - partitioner: Partitioner, - numPartitions: Int, - storageLevel: StorageLevel - )( - compute: (V, Option[C], Int) => (V, Array[M]) - ): RDD[(K, V)] = { - run[K, V, M, C, Nothing]( - sc, vertices, messages, combiner, None, partitioner, numPartitions, storageLevel)( - addAggregatorArg[K, V, M, C](compute)) - } - - /** - * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], default - * org.apache.spark.HashPartitioner and default storage level - */ - def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( - sc: SparkContext, - vertices: RDD[(K, V)], - messages: RDD[(K, M)], - combiner: Combiner[M, C], - numPartitions: Int - )( - compute: (V, Option[C], Int) => (V, Array[M]) - ): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, - DEFAULT_STORAGE_LEVEL)(compute) - - /** - * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the - * default org.apache.spark.HashPartitioner - */ - def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( - sc: SparkContext, - vertices: RDD[(K, V)], - messages: RDD[(K, M)], - combiner: Combiner[M, C], - numPartitions: Int, - storageLevel: StorageLevel - )( - compute: (V, Option[C], Int) => (V, Array[M]) - ): RDD[(K, V)] = { - val part = new HashPartitioner(numPartitions) - run[K, V, M, C, Nothing]( - sc, vertices, messages, combiner, None, part, numPartitions, storageLevel)( - addAggregatorArg[K, V, M, C](compute)) - } - - /** - * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], - * default org.apache.spark.HashPartitioner, - * [[org.apache.spark.bagel.DefaultCombiner]] and the default storage level - */ - def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest]( - sc: SparkContext, - vertices: RDD[(K, V)], - messages: RDD[(K, M)], - numPartitions: Int - )( - compute: (V, Option[Array[M]], Int) => (V, Array[M]) - ): RDD[(K, V)] = run(sc, vertices, messages, numPartitions, DEFAULT_STORAGE_LEVEL)(compute) - - /** - * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], - * the default org.apache.spark.HashPartitioner - * and [[org.apache.spark.bagel.DefaultCombiner]] - */ - def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest]( - sc: SparkContext, - vertices: RDD[(K, V)], - messages: RDD[(K, M)], - numPartitions: Int, - storageLevel: StorageLevel - )( - compute: (V, Option[Array[M]], Int) => (V, Array[M]) - ): RDD[(K, V)] = { - val part = new HashPartitioner(numPartitions) - run[K, V, M, Array[M], Nothing]( - sc, vertices, messages, new DefaultCombiner(), None, part, numPartitions, storageLevel)( - addAggregatorArg[K, V, M, Array[M]](compute)) - } - - /** - * Aggregates the given vertices using the given aggregator, if it - * is specified. - */ - private def agg[K, V <: Vertex, A: Manifest]( - verts: RDD[(K, V)], - aggregator: Option[Aggregator[V, A]] - ): Option[A] = aggregator match { - case Some(a) => - Some(verts.map { - case (id, vert) => a.createAggregator(vert) - }.reduce(a.mergeAggregators(_, _))) - case None => None - } - - /** - * Processes the given vertex-message RDD using the compute - * function. Returns the processed RDD, the number of messages - * created, and the number of active vertices. - */ - private def comp[K: Manifest, V <: Vertex, M <: Message[K], C]( - sc: SparkContext, - grouped: RDD[(K, (Iterable[C], Iterable[V]))], - compute: (V, Option[C]) => (V, Array[M]), - storageLevel: StorageLevel - ): (RDD[(K, (V, Array[M]))], Int, Int) = { - var numMsgs = sc.accumulator(0) - var numActiveVerts = sc.accumulator(0) - val processed = grouped.mapValues(x => (x._1.iterator, x._2.iterator)) - .flatMapValues { - case (_, vs) if !vs.hasNext => None - case (c, vs) => { - val (newVert, newMsgs) = - compute(vs.next, - c.hasNext match { - case true => Some(c.next) - case false => None - } - ) - - numMsgs += newMsgs.size - if (newVert.active) { - numActiveVerts += 1 - } - - Some((newVert, newMsgs)) - } - }.persist(storageLevel) - - // Force evaluation of processed RDD for accurate performance measurements - processed.foreach(x => {}) - - (processed, numMsgs.value, numActiveVerts.value) - } - - /** - * Converts a compute function that doesn't take an aggregator to - * one that does, so it can be passed to Bagel.run. - */ - private def addAggregatorArg[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C]( - compute: (V, Option[C], Int) => (V, Array[M]) - ): (V, Option[C], Option[Nothing], Int) => (V, Array[M]) = { - (vert: V, msgs: Option[C], aggregated: Option[Nothing], superstep: Int) => - compute(vert, msgs, superstep) - } -} - -@deprecated("Uses of Bagel should migrate to GraphX", "1.6.0") -trait Combiner[M, C] { - def createCombiner(msg: M): C - def mergeMsg(combiner: C, msg: M): C - def mergeCombiners(a: C, b: C): C -} - -@deprecated("Uses of Bagel should migrate to GraphX", "1.6.0") -trait Aggregator[V, A] { - def createAggregator(vert: V): A - def mergeAggregators(a: A, b: A): A -} - -/** Default combiner that simply appends messages together (i.e. performs no aggregation) */ -@deprecated("Uses of Bagel should migrate to GraphX", "1.6.0") -class DefaultCombiner[M: Manifest] extends Combiner[M, Array[M]] with Serializable { - def createCombiner(msg: M): Array[M] = - Array(msg) - def mergeMsg(combiner: Array[M], msg: M): Array[M] = - combiner :+ msg - def mergeCombiners(a: Array[M], b: Array[M]): Array[M] = - a ++ b -} - -/** - * Represents a Bagel vertex. - * - * Subclasses may store state along with each vertex and must - * inherit from java.io.Serializable or scala.Serializable. - */ -@deprecated("Uses of Bagel should migrate to GraphX", "1.6.0") -trait Vertex { - def active: Boolean -} - -/** - * Represents a Bagel message to a target vertex. - * - * Subclasses may contain a payload to deliver to the target vertex - * and must inherit from java.io.Serializable or scala.Serializable. - */ -@deprecated("Uses of Bagel should migrate to GraphX", "1.6.0") -trait Message[K] { - def targetId: K -} diff --git a/bagel/src/main/scala/org/apache/spark/bagel/package-info.java b/bagel/src/main/scala/org/apache/spark/bagel/package-info.java deleted file mode 100644 index 81f26f276549..000000000000 --- a/bagel/src/main/scala/org/apache/spark/bagel/package-info.java +++ /dev/null @@ -1,21 +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. - */ - -/** - * Bagel: An implementation of Pregel in Spark. THIS IS DEPRECATED - use Spark's GraphX library. - */ -package org.apache.spark.bagel; \ No newline at end of file diff --git a/bagel/src/main/scala/org/apache/spark/bagel/package.scala b/bagel/src/main/scala/org/apache/spark/bagel/package.scala deleted file mode 100644 index 2fb193457978..000000000000 --- a/bagel/src/main/scala/org/apache/spark/bagel/package.scala +++ /dev/null @@ -1,23 +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 - -/** - * Bagel: An implementation of Pregel in Spark. THIS IS DEPRECATED - use Spark's GraphX library. - */ -package object bagel diff --git a/bin/beeline b/bin/beeline index 1627626941a7..058534699e44 100755 --- a/bin/beeline +++ b/bin/beeline @@ -25,7 +25,7 @@ set -o posix # Figure out if SPARK_HOME is set if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + source "$(dirname "$0")"/find-spark-home fi CLASS="org.apache.hive.beeline.BeeLine" diff --git a/bin/beeline.cmd b/bin/beeline.cmd index 8293f311029d..02464bd08879 100644 --- a/bin/beeline.cmd +++ b/bin/beeline.cmd @@ -17,5 +17,4 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -set SPARK_HOME=%~dp0.. -cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.hive.beeline.BeeLine %* +cmd /V /E /C "%~dp0spark-class.cmd" org.apache.hive.beeline.BeeLine %* diff --git a/bin/find-spark-home b/bin/find-spark-home new file mode 100755 index 000000000000..fa78407d4175 --- /dev/null +++ b/bin/find-spark-home @@ -0,0 +1,41 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Attempts to find a proper value for SPARK_HOME. Should be included using "source" directive. + +FIND_SPARK_HOME_PYTHON_SCRIPT="$(cd "$(dirname "$0")"; pwd)/find_spark_home.py" + +# Short cirtuit if the user already has this set. +if [ ! -z "${SPARK_HOME}" ]; then + exit 0 +elif [ ! -f "$FIND_SPARK_HOME_PYTHON_SCRIPT" ]; then + # If we are not in the same directory as find_spark_home.py we are not pip installed so we don't + # need to search the different Python directories for a Spark installation. + # Note only that, if the user has pip installed PySpark but is directly calling pyspark-shell or + # spark-submit in another directory we want to use that version of PySpark rather than the + # pip installed version of PySpark. + export SPARK_HOME="$(cd "$(dirname "$0")"/..; pwd)" +else + # We are pip installed, use the Python script to resolve a reasonable SPARK_HOME + # Default to standard python interpreter unless told otherwise + if [[ -z "$PYSPARK_DRIVER_PYTHON" ]]; then + PYSPARK_DRIVER_PYTHON="${PYSPARK_PYTHON:-"python"}" + fi + export SPARK_HOME=$($PYSPARK_DRIVER_PYTHON "$FIND_SPARK_HOME_PYTHON_SCRIPT") +fi diff --git a/bin/load-spark-env.cmd b/bin/load-spark-env.cmd index 36d932c453b6..f946197b02d5 100644 --- a/bin/load-spark-env.cmd +++ b/bin/load-spark-env.cmd @@ -27,7 +27,7 @@ if [%SPARK_ENV_LOADED%] == [] ( if not [%SPARK_CONF_DIR%] == [] ( set user_conf_dir=%SPARK_CONF_DIR% ) else ( - set user_conf_dir=%~dp0..\..\conf + set user_conf_dir=..\conf ) call :LoadSparkEnv @@ -35,20 +35,20 @@ if [%SPARK_ENV_LOADED%] == [] ( rem Setting SPARK_SCALA_VERSION if not already set. -set ASSEMBLY_DIR2=%SPARK_HOME%/assembly/target/scala-2.11 -set ASSEMBLY_DIR1=%SPARK_HOME%/assembly/target/scala-2.10 +set ASSEMBLY_DIR2="%SPARK_HOME%\assembly\target\scala-2.11" +set ASSEMBLY_DIR1="%SPARK_HOME%\assembly\target\scala-2.12" if [%SPARK_SCALA_VERSION%] == [] ( if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% ( - echo "Presence of build for both scala versions(SCALA 2.10 and SCALA 2.11) detected." - echo "Either clean one of them or, set SPARK_SCALA_VERSION=2.11 in spark-env.cmd." + echo "Presence of build for multiple Scala versions detected." + echo "Either clean one of them or, set SPARK_SCALA_VERSION in spark-env.cmd." exit 1 ) if exist %ASSEMBLY_DIR2% ( set SPARK_SCALA_VERSION=2.11 ) else ( - set SPARK_SCALA_VERSION=2.10 + set SPARK_SCALA_VERSION=2.12 ) ) exit /b 0 diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index eaea964ed5b3..d05d94e68c81 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -23,7 +23,7 @@ # Figure out where Spark is installed if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + source "$(dirname "$0")"/find-spark-home fi if [ -z "$SPARK_ENV_LOADED" ]; then @@ -47,17 +47,17 @@ fi if [ -z "$SPARK_SCALA_VERSION" ]; then ASSEMBLY_DIR2="${SPARK_HOME}/assembly/target/scala-2.11" - ASSEMBLY_DIR1="${SPARK_HOME}/assembly/target/scala-2.10" + ASSEMBLY_DIR1="${SPARK_HOME}/assembly/target/scala-2.12" if [[ -d "$ASSEMBLY_DIR2" && -d "$ASSEMBLY_DIR1" ]]; then - echo -e "Presence of build for both scala versions(SCALA 2.10 and SCALA 2.11) detected." 1>&2 - echo -e 'Either clean one of them or, export SPARK_SCALA_VERSION=2.11 in spark-env.sh.' 1>&2 + echo -e "Presence of build for multiple Scala versions detected." 1>&2 + echo -e 'Either clean one of them or, export SPARK_SCALA_VERSION in spark-env.sh.' 1>&2 exit 1 fi if [ -d "$ASSEMBLY_DIR2" ]; then export SPARK_SCALA_VERSION="2.11" else - export SPARK_SCALA_VERSION="2.10" + export SPARK_SCALA_VERSION="2.12" fi fi diff --git a/bin/pyspark b/bin/pyspark index 5eaa17d3c201..dd286277c1fc 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -18,56 +18,46 @@ # if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + source "$(dirname "$0")"/find-spark-home fi source "${SPARK_HOME}"/bin/load-spark-env.sh export _SPARK_CMD_USAGE="Usage: ./bin/pyspark [options]" -# In Spark <= 1.1, setting IPYTHON=1 would cause the driver to be launched using the `ipython` -# executable, while the worker would still be launched using PYSPARK_PYTHON. -# -# In Spark 1.2, we removed the documentation of the IPYTHON and IPYTHON_OPTS variables and added -# PYSPARK_DRIVER_PYTHON and PYSPARK_DRIVER_PYTHON_OPTS to allow IPython to be used for the driver. -# Now, users can simply set PYSPARK_DRIVER_PYTHON=ipython to use IPython and set -# PYSPARK_DRIVER_PYTHON_OPTS to pass options when starting the Python driver +# In Spark 2.0, IPYTHON and IPYTHON_OPTS are removed and pyspark fails to launch if either option +# is set in the user's environment. Instead, users should set PYSPARK_DRIVER_PYTHON=ipython +# to use IPython and set PYSPARK_DRIVER_PYTHON_OPTS to pass options when starting the Python driver # (e.g. PYSPARK_DRIVER_PYTHON_OPTS='notebook'). This supports full customization of the IPython # and executor Python executables. -# -# For backwards-compatibility, we retain the old IPYTHON and IPYTHON_OPTS variables. -# Determine the Python executable to use if PYSPARK_PYTHON or PYSPARK_DRIVER_PYTHON isn't set: -if hash python2.7 2>/dev/null; then - # Attempt to use Python 2.7, if installed: - DEFAULT_PYTHON="python2.7" -else - DEFAULT_PYTHON="python" +# Fail noisily if removed options are set +if [[ -n "$IPYTHON" || -n "$IPYTHON_OPTS" ]]; then + echo "Error in pyspark startup:" + echo "IPYTHON and IPYTHON_OPTS are removed in Spark 2.0+. Remove these from the environment and set PYSPARK_DRIVER_PYTHON and PYSPARK_DRIVER_PYTHON_OPTS instead." + exit 1 fi -# Determine the Python executable to use for the driver: -if [[ -n "$IPYTHON_OPTS" || "$IPYTHON" == "1" ]]; then - # If IPython options are specified, assume user wants to run IPython - # (for backwards-compatibility) - PYSPARK_DRIVER_PYTHON_OPTS="$PYSPARK_DRIVER_PYTHON_OPTS $IPYTHON_OPTS" - PYSPARK_DRIVER_PYTHON="ipython" -elif [[ -z "$PYSPARK_DRIVER_PYTHON" ]]; then - PYSPARK_DRIVER_PYTHON="${PYSPARK_PYTHON:-"$DEFAULT_PYTHON"}" +# Default to standard python interpreter unless told otherwise +if [[ -z "$PYSPARK_DRIVER_PYTHON" ]]; then + PYSPARK_DRIVER_PYTHON="${PYSPARK_PYTHON:-"python"}" fi +WORKS_WITH_IPYTHON=$(python -c 'import sys; print(sys.version_info >= (2, 7, 0))') + # Determine the Python executable to use for the executors: if [[ -z "$PYSPARK_PYTHON" ]]; then - if [[ $PYSPARK_DRIVER_PYTHON == *ipython* && $DEFAULT_PYTHON != "python2.7" ]]; then + if [[ $PYSPARK_DRIVER_PYTHON == *ipython* && ! $WORKS_WITH_IPYTHON ]]; then echo "IPython requires Python 2.7+; please install python2.7 or set PYSPARK_PYTHON" 1>&2 exit 1 else - PYSPARK_PYTHON="$DEFAULT_PYTHON" + PYSPARK_PYTHON=python fi fi export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH="${SPARK_HOME}/python/:$PYTHONPATH" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.9-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.6-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" @@ -78,7 +68,7 @@ if [[ -n "$SPARK_TESTING" ]]; then unset YARN_CONF_DIR unset HADOOP_CONF_DIR export PYTHONHASHSEED=0 - exec "$PYSPARK_DRIVER_PYTHON" -m $1 + exec "$PYSPARK_DRIVER_PYTHON" -m "$@" exit fi diff --git a/bin/pyspark.cmd b/bin/pyspark.cmd index 7c26fbbac28b..72d046a4ba2c 100644 --- a/bin/pyspark.cmd +++ b/bin/pyspark.cmd @@ -20,4 +20,4 @@ rem rem This is the entry point for running PySpark. To avoid polluting the rem environment, it just launches a new cmd to do the real work. -cmd /V /E /C %~dp0pyspark2.cmd %* +cmd /V /E /C "%~dp0pyspark2.cmd" %* diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index a97d884f0bf3..46d4d5c883cf 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -20,7 +20,7 @@ rem rem Figure out where the Spark framework is installed set SPARK_HOME=%~dp0.. -call %SPARK_HOME%\bin\load-spark-env.cmd +call "%SPARK_HOME%\bin\load-spark-env.cmd" set _SPARK_CMD_USAGE=Usage: bin\pyspark.cmd [options] rem Figure out which Python to use. @@ -30,9 +30,9 @@ if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( ) set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH% -set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.9-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.6-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py -call %SPARK_HOME%\bin\spark-submit2.cmd pyspark-shell-main --name "PySparkShell" %* +call "%SPARK_HOME%\bin\spark-submit2.cmd" pyspark-shell-main --name "PySparkShell" %* diff --git a/bin/run-example b/bin/run-example index e1b0d5789bed..4ba5399311d3 100755 --- a/bin/run-example +++ b/bin/run-example @@ -18,59 +18,8 @@ # if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + source "$(dirname "$0")"/find-spark-home fi -EXAMPLES_DIR="${SPARK_HOME}"/examples - -. "${SPARK_HOME}"/bin/load-spark-env.sh - -if [ -n "$1" ]; then - EXAMPLE_CLASS="$1" - shift -else - echo "Usage: ./bin/run-example [example-args]" 1>&2 - echo " - set MASTER=XX to use a specific master" 1>&2 - echo " - can use abbreviated example class name relative to com.apache.spark.examples" 1>&2 - echo " (e.g. SparkPi, mllib.LinearRegression, streaming.KinesisWordCountASL)" 1>&2 - exit 1 -fi - -if [ -f "${SPARK_HOME}/RELEASE" ]; then - JAR_PATH="${SPARK_HOME}/lib" -else - JAR_PATH="${EXAMPLES_DIR}/target/scala-${SPARK_SCALA_VERSION}" -fi - -JAR_COUNT=0 - -for f in "${JAR_PATH}"/spark-examples-*hadoop*.jar; do - if [[ ! -e "$f" ]]; then - echo "Failed to find Spark examples assembly in ${SPARK_HOME}/lib or ${SPARK_HOME}/examples/target" 1>&2 - echo "You need to build Spark before running this program" 1>&2 - exit 1 - fi - SPARK_EXAMPLES_JAR="$f" - JAR_COUNT=$((JAR_COUNT+1)) -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 - echo "Please remove all but one jar." 1>&2 - exit 1 -fi - -export SPARK_EXAMPLES_JAR - -EXAMPLE_MASTER=${MASTER:-"local[*]"} - -if [[ ! $EXAMPLE_CLASS == org.apache.spark.examples* ]]; then - EXAMPLE_CLASS="org.apache.spark.examples.$EXAMPLE_CLASS" -fi - -exec "${SPARK_HOME}"/bin/spark-submit \ - --master $EXAMPLE_MASTER \ - --class $EXAMPLE_CLASS \ - "$SPARK_EXAMPLES_JAR" \ - "$@" +export _SPARK_CMD_USAGE="Usage: ./bin/run-example [options] example-class [example args]" +exec "${SPARK_HOME}"/bin/spark-submit run-example "$@" diff --git a/bin/run-example.cmd b/bin/run-example.cmd index 5b2d048d6ed5..f9b786e92b82 100644 --- a/bin/run-example.cmd +++ b/bin/run-example.cmd @@ -17,7 +17,6 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -rem This is the entry point for running a Spark example. To avoid polluting -rem the environment, it just launches a new cmd to do the real work. - -cmd /V /E /C %~dp0run-example2.cmd %* +set SPARK_HOME=%~dp0.. +set _SPARK_CMD_USAGE=Usage: ./bin/run-example [options] example-class [example args] +cmd /V /E /C "%~dp0spark-submit.cmd" run-example %* diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd deleted file mode 100644 index c3e0221fb62e..000000000000 --- a/bin/run-example2.cmd +++ /dev/null @@ -1,88 +0,0 @@ -@echo off - -rem -rem Licensed to the Apache Software Foundation (ASF) under one or more -rem contributor license agreements. See the NOTICE file distributed with -rem this work for additional information regarding copyright ownership. -rem The ASF licenses this file to You under the Apache License, Version 2.0 -rem (the "License"); you may not use this file except in compliance with -rem the License. You may obtain a copy of the License at -rem -rem http://www.apache.org/licenses/LICENSE-2.0 -rem -rem Unless required by applicable law or agreed to in writing, software -rem distributed under the License is distributed on an "AS IS" BASIS, -rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -rem See the License for the specific language governing permissions and -rem limitations under the License. -rem - -set SCALA_VERSION=2.10 - -rem Figure out where the Spark framework is installed -set FWDIR=%~dp0..\ - -rem Export this as SPARK_HOME -set SPARK_HOME=%FWDIR% - -call %SPARK_HOME%\bin\load-spark-env.cmd - -rem Test that an argument was given -if not "x%1"=="x" goto arg_given - echo Usage: run-example ^ [example-args] - echo - set MASTER=XX to use a specific master - echo - can use abbreviated example class name relative to com.apache.spark.examples - echo (e.g. SparkPi, mllib.LinearRegression, streaming.KinesisWordCountASL) - goto exit -:arg_given - -set EXAMPLES_DIR=%FWDIR%examples - -rem Figure out the JAR file that our examples were packaged into. -set SPARK_EXAMPLES_JAR= -if exist "%FWDIR%RELEASE" ( - for %%d in ("%FWDIR%lib\spark-examples*.jar") do ( - set SPARK_EXAMPLES_JAR=%%d - ) -) else ( - for %%d in ("%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\spark-examples*.jar") do ( - set SPARK_EXAMPLES_JAR=%%d - ) -) -if "x%SPARK_EXAMPLES_JAR%"=="x" ( - echo Failed to find Spark examples assembly JAR. - echo You need to build Spark before running this program. - goto exit -) - -rem Set master from MASTER environment variable if given -if "x%MASTER%"=="x" ( - set EXAMPLE_MASTER=local[*] -) else ( - set EXAMPLE_MASTER=%MASTER% -) - -rem If the EXAMPLE_CLASS does not start with org.apache.spark.examples, add that -set EXAMPLE_CLASS=%1 -set PREFIX=%EXAMPLE_CLASS:~0,25% -if not %PREFIX%==org.apache.spark.examples ( - set EXAMPLE_CLASS=org.apache.spark.examples.%EXAMPLE_CLASS% -) - -rem Get the tail of the argument list, to skip the first one. This is surprisingly -rem complicated on Windows. -set "ARGS=" -:top -shift -if "%~1" neq "" ( - set ARGS=%ARGS% "%~1" - goto :top -) -if defined ARGS set ARGS=%ARGS:~1% - -call "%FWDIR%bin\spark-submit.cmd" ^ - --master %EXAMPLE_MASTER% ^ - --class %EXAMPLE_CLASS% ^ - "%SPARK_EXAMPLES_JAR%" %ARGS% - -:exit diff --git a/bin/spark-class b/bin/spark-class index 87d06693af4f..65d3b9612909 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -18,7 +18,7 @@ # if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + source "$(dirname "$0")"/find-spark-home fi . "${SPARK_HOME}"/bin/load-spark-env.sh @@ -27,7 +27,7 @@ fi if [ -n "${JAVA_HOME}" ]; then RUNNER="${JAVA_HOME}/bin/java" else - if [ `command -v java` ]; then + if [ "$(command -v java)" ]; then RUNNER="java" else echo "JAVA_HOME is not set" >&2 @@ -35,47 +35,65 @@ else fi fi -# Find assembly jar -SPARK_ASSEMBLY_JAR= -if [ -f "${SPARK_HOME}/RELEASE" ]; then - ASSEMBLY_DIR="${SPARK_HOME}/lib" +# Find Spark jars. +if [ -d "${SPARK_HOME}/jars" ]; then + SPARK_JARS_DIR="${SPARK_HOME}/jars" else - ASSEMBLY_DIR="${SPARK_HOME}/assembly/target/scala-$SPARK_SCALA_VERSION" + SPARK_JARS_DIR="${SPARK_HOME}/assembly/target/scala-$SPARK_SCALA_VERSION/jars" fi -GREP_OPTIONS= -num_jars="$(ls -1 "$ASSEMBLY_DIR" | grep "^spark-assembly.*hadoop.*\.jar$" | wc -l)" -if [ "$num_jars" -eq "0" -a -z "$SPARK_ASSEMBLY_JAR" -a "$SPARK_PREPEND_CLASSES" != "1" ]; then - echo "Failed to find Spark assembly in $ASSEMBLY_DIR." 1>&2 - echo "You need to build Spark before running this program." 1>&2 +if [ ! -d "$SPARK_JARS_DIR" ] && [ -z "$SPARK_TESTING$SPARK_SQL_TESTING" ]; then + echo "Failed to find Spark jars directory ($SPARK_JARS_DIR)." 1>&2 + echo "You need to build Spark with the target \"package\" before running this program." 1>&2 exit 1 -fi -if [ -d "$ASSEMBLY_DIR" ]; then - ASSEMBLY_JARS="$(ls -1 "$ASSEMBLY_DIR" | grep "^spark-assembly.*hadoop.*\.jar$" || true)" - if [ "$num_jars" -gt "1" ]; then - echo "Found multiple Spark assembly jars in $ASSEMBLY_DIR:" 1>&2 - echo "$ASSEMBLY_JARS" 1>&2 - echo "Please remove all but one jar." 1>&2 - exit 1 - fi +else + LAUNCH_CLASSPATH="$SPARK_JARS_DIR/*" fi -SPARK_ASSEMBLY_JAR="${ASSEMBLY_DIR}/${ASSEMBLY_JARS}" - -LAUNCH_CLASSPATH="$SPARK_ASSEMBLY_JAR" - # Add the launcher build dir to the classpath if requested. if [ -n "$SPARK_PREPEND_CLASSES" ]; then LAUNCH_CLASSPATH="${SPARK_HOME}/launcher/target/scala-$SPARK_SCALA_VERSION/classes:$LAUNCH_CLASSPATH" fi -export _SPARK_ASSEMBLY="$SPARK_ASSEMBLY_JAR" +# For tests +if [[ -n "$SPARK_TESTING" ]]; then + unset YARN_CONF_DIR + unset HADOOP_CONF_DIR +fi # The launcher library will print arguments separated by a NULL character, to allow arguments with # characters that would be otherwise interpreted by the shell. Read that in a while loop, populating # an array that will be used to exec the final command. +# +# The exit code of the launcher is appended to the output, so the parent shell removes it from the +# command array and checks the value to see if the launcher succeeded. +build_command() { + "$RUNNER" -Xmx128m -cp "$LAUNCH_CLASSPATH" org.apache.spark.launcher.Main "$@" + printf "%d\0" $? +} + +# Turn off posix mode since it does not allow process substitution +set +o posix CMD=() while IFS= read -d '' -r ARG; do CMD+=("$ARG") -done < <("$RUNNER" -cp "$LAUNCH_CLASSPATH" org.apache.spark.launcher.Main "$@") +done < <(build_command "$@") + +COUNT=${#CMD[@]} +LAST=$((COUNT - 1)) +LAUNCHER_EXIT_CODE=${CMD[$LAST]} + +# Certain JVM failures result in errors being printed to stdout (instead of stderr), which causes +# the code that parses the output of the launcher to get confused. In those cases, check if the +# exit code is an integer, and if it's not, handle it as a special error case. +if ! [[ $LAUNCHER_EXIT_CODE =~ ^[0-9]+$ ]]; then + echo "${CMD[@]}" | head -n-1 1>&2 + exit 1 +fi + +if [ $LAUNCHER_EXIT_CODE != 0 ]; then + exit $LAUNCHER_EXIT_CODE +fi + +CMD=("${CMD[@]:0:$LAST}") exec "${CMD[@]}" diff --git a/bin/spark-class.cmd b/bin/spark-class.cmd index 19850db9e1e5..3bf3d20cb57b 100644 --- a/bin/spark-class.cmd +++ b/bin/spark-class.cmd @@ -20,4 +20,4 @@ rem rem This is the entry point for running a Spark class. To avoid polluting rem the environment, it just launches a new cmd to do the real work. -cmd /V /E /C %~dp0spark-class2.cmd %* +cmd /V /E /C "%~dp0spark-class2.cmd" %* diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index db09fa27e51a..a93fd2f0e54b 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -20,7 +20,7 @@ rem rem Figure out where the Spark framework is installed set SPARK_HOME=%~dp0.. -call %SPARK_HOME%\bin\load-spark-env.cmd +call "%SPARK_HOME%\bin\load-spark-env.cmd" rem Test that an argument was given if "x%1"=="x" ( @@ -28,41 +28,43 @@ if "x%1"=="x" ( exit /b 1 ) -rem Find assembly jar -set SPARK_ASSEMBLY_JAR=0 - -if exist "%SPARK_HOME%\RELEASE" ( - set ASSEMBLY_DIR=%SPARK_HOME%\lib +rem Find Spark jars. +if exist "%SPARK_HOME%\jars" ( + set SPARK_JARS_DIR="%SPARK_HOME%\jars" ) else ( - set ASSEMBLY_DIR=%SPARK_HOME%\assembly\target\scala-%SPARK_SCALA_VERSION% + set SPARK_JARS_DIR="%SPARK_HOME%\assembly\target\scala-%SPARK_SCALA_VERSION%\jars" ) -for %%d in (%ASSEMBLY_DIR%\spark-assembly*hadoop*.jar) do ( - set SPARK_ASSEMBLY_JAR=%%d -) -if "%SPARK_ASSEMBLY_JAR%"=="0" ( - echo Failed to find Spark assembly JAR. +if not exist "%SPARK_JARS_DIR%"\ ( + echo Failed to find Spark jars directory. echo You need to build Spark before running this program. exit /b 1 ) -set LAUNCH_CLASSPATH=%SPARK_ASSEMBLY_JAR% +set LAUNCH_CLASSPATH=%SPARK_JARS_DIR%\* rem Add the launcher build dir to the classpath if requested. if not "x%SPARK_PREPEND_CLASSES%"=="x" ( - set LAUNCH_CLASSPATH=%SPARK_HOME%\launcher\target\scala-%SPARK_SCALA_VERSION%\classes;%LAUNCH_CLASSPATH% + set LAUNCH_CLASSPATH="%SPARK_HOME%\launcher\target\scala-%SPARK_SCALA_VERSION%\classes;%LAUNCH_CLASSPATH%" ) -set _SPARK_ASSEMBLY=%SPARK_ASSEMBLY_JAR% - rem Figure out where java is. set RUNNER=java -if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java +if not "x%JAVA_HOME%"=="x" ( + set RUNNER=%JAVA_HOME%\bin\java +) else ( + where /q "%RUNNER%" + if ERRORLEVEL 1 ( + echo Java not found and JAVA_HOME environment variable is not set. + echo Install Java and set JAVA_HOME to point to the Java installation directory. + exit /b 1 + ) +) rem The launcher library prints the command to be executed in a single line suitable for being rem executed by the batch interpreter. So read all the output of the launcher into a variable. set LAUNCHER_OUTPUT=%temp%\spark-class-launcher-output-%RANDOM%.txt -"%RUNNER%" -cp %LAUNCH_CLASSPATH% org.apache.spark.launcher.Main %* > %LAUNCHER_OUTPUT% +"%RUNNER%" -Xmx128m -cp "%LAUNCH_CLASSPATH%" org.apache.spark.launcher.Main %* > %LAUNCHER_OUTPUT% for /f "tokens=*" %%i in (%LAUNCHER_OUTPUT%) do ( set SPARK_CMD=%%i ) diff --git a/bin/spark-shell b/bin/spark-shell index 6583b5bd880e..421f36cac3d4 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -21,7 +21,7 @@ # Shell script for starting the Spark Shell REPL cygwin=false -case "`uname`" in +case "$(uname)" in CYGWIN*) cygwin=true;; esac @@ -29,7 +29,7 @@ esac set -o posix if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + source "$(dirname "$0")"/find-spark-home fi export _SPARK_CMD_USAGE="Usage: ./bin/spark-shell [options]" diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index 8f90ba5a0b3b..991423da6ab9 100644 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -20,4 +20,4 @@ rem rem This is the entry point for running Spark shell. To avoid polluting the rem environment, it just launches a new cmd to do the real work. -cmd /V /E /C %~dp0spark-shell2.cmd %* +cmd /V /E /C "%~dp0spark-shell2.cmd" %* diff --git a/bin/spark-shell2.cmd b/bin/spark-shell2.cmd index b9b0f510d7f5..7b5d396be888 100644 --- a/bin/spark-shell2.cmd +++ b/bin/spark-shell2.cmd @@ -32,4 +32,4 @@ if "x%SPARK_SUBMIT_OPTS%"=="x" ( set SPARK_SUBMIT_OPTS="%SPARK_SUBMIT_OPTS% -Dscala.usejavacp=true" :run_shell -%SPARK_HOME%\bin\spark-submit2.cmd --class org.apache.spark.repl.Main --name "Spark shell" %* +"%SPARK_HOME%\bin\spark-submit2.cmd" --class org.apache.spark.repl.Main --name "Spark shell" %* diff --git a/bin/spark-sql b/bin/spark-sql index 970d12cbf51d..b08b944ebd31 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -18,7 +18,7 @@ # if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + source "$(dirname "$0")"/find-spark-home fi export _SPARK_CMD_USAGE="Usage: ./bin/spark-sql [options] [cli option]" diff --git a/bin/spark-submit b/bin/spark-submit index 023f9c162f4b..4e9d3614e637 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -18,7 +18,7 @@ # if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + source "$(dirname "$0")"/find-spark-home fi # disable randomized hash for string in Python 3.3+ diff --git a/bin/spark-submit.cmd b/bin/spark-submit.cmd index 8f3b84c7b971..f301606933a9 100644 --- a/bin/spark-submit.cmd +++ b/bin/spark-submit.cmd @@ -20,4 +20,4 @@ rem rem This is the entry point for running Spark submit. To avoid polluting the rem environment, it just launches a new cmd to do the real work. -cmd /V /E /C %~dp0spark-submit2.cmd %* +cmd /V /E /C "%~dp0spark-submit2.cmd" %* diff --git a/bin/spark-submit2.cmd b/bin/spark-submit2.cmd index 651376e52692..49e350fa5c41 100644 --- a/bin/spark-submit2.cmd +++ b/bin/spark-submit2.cmd @@ -24,4 +24,4 @@ rem disable randomized hash for string in Python 3.3+ set PYTHONHASHSEED=0 set CLASS=org.apache.spark.deploy.SparkSubmit -%~dp0spark-class2.cmd %CLASS% %* +"%~dp0spark-class2.cmd" %CLASS% %* diff --git a/bin/sparkR b/bin/sparkR index 2c07a82e2173..29ab10df8ab6 100755 --- a/bin/sparkR +++ b/bin/sparkR @@ -18,7 +18,7 @@ # if [ -z "${SPARK_HOME}" ]; then - export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + source "$(dirname "$0")"/find-spark-home fi source "${SPARK_HOME}"/bin/load-spark-env.sh diff --git a/bin/sparkR.cmd b/bin/sparkR.cmd index d7b60183ca8e..1e5ea6a62321 100644 --- a/bin/sparkR.cmd +++ b/bin/sparkR.cmd @@ -20,4 +20,4 @@ rem rem This is the entry point for running SparkR. To avoid polluting the rem environment, it just launches a new cmd to do the real work. -cmd /V /E /C %~dp0sparkR2.cmd %* +cmd /V /E /C "%~dp0sparkR2.cmd" %* diff --git a/bin/sparkR2.cmd b/bin/sparkR2.cmd index e47f22c7300b..459b780e2ae3 100644 --- a/bin/sparkR2.cmd +++ b/bin/sparkR2.cmd @@ -20,7 +20,7 @@ rem rem Figure out where the Spark framework is installed set SPARK_HOME=%~dp0.. -call %SPARK_HOME%\bin\load-spark-env.cmd +call "%SPARK_HOME%\bin\load-spark-env.cmd" -call %SPARK_HOME%\bin\spark-submit2.cmd sparkr-shell-main %* +call "%SPARK_HOME%\bin\spark-submit2.cmd" sparkr-shell-main %* diff --git a/build/mvn b/build/mvn index 7603ea03deb7..efa4f9364ea5 100755 --- a/build/mvn +++ b/build/mvn @@ -22,7 +22,7 @@ _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" +_COMPILE_JVM_OPTS="-Xmx2g -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 @@ -67,25 +67,37 @@ install_app() { fi } -# Install maven under the build/ folder +# Determine the Maven version from the root pom.xml file and +# install maven under the build/ folder if needed. install_mvn() { - local MVN_VERSION="3.3.3" + local MVN_VERSION=`grep "" "${_DIR}/../pom.xml" | head -n1 | awk -F '[<>]' '{print $3}'` + MVN_BIN="$(command -v mvn)" + if [ "$MVN_BIN" ]; then + local MVN_DETECTED_VERSION="$(mvn --version | head -n1 | awk '{print $3}')" + fi + # See simple version normalization: http://stackoverflow.com/questions/16989598/bash-comparing-version-numbers + function version { echo "$@" | awk -F. '{ printf("%03d%03d%03d\n", $1,$2,$3); }'; } + if [ $(version $MVN_DETECTED_VERSION) -lt $(version $MVN_VERSION) ]; then + local APACHE_MIRROR=${APACHE_MIRROR:-'https://www.apache.org/dyn/closer.lua?action=download&filename='} - install_app \ - "http://archive.apache.org/dist/maven/maven-3/${MVN_VERSION}/binaries" \ - "apache-maven-${MVN_VERSION}-bin.tar.gz" \ - "apache-maven-${MVN_VERSION}/bin/mvn" + install_app \ + "${APACHE_MIRROR}/maven/maven-3/${MVN_VERSION}/binaries" \ + "apache-maven-${MVN_VERSION}-bin.tar.gz" \ + "apache-maven-${MVN_VERSION}/bin/mvn" - MVN_BIN="${_DIR}/apache-maven-${MVN_VERSION}/bin/mvn" + MVN_BIN="${_DIR}/apache-maven-${MVN_VERSION}/bin/mvn" + fi } # Install zinc under the build/ folder install_zinc() { - local zinc_path="zinc-0.3.5.3/bin/zinc" + local zinc_path="zinc-0.3.15/bin/zinc" [ ! -f "${_DIR}/${zinc_path}" ] && ZINC_INSTALL_FLAG=1 + local TYPESAFE_MIRROR=${TYPESAFE_MIRROR:-https://downloads.typesafe.com} + install_app \ - "http://downloads.typesafe.com/zinc/0.3.5.3" \ - "zinc-0.3.5.3.tgz" \ + "${TYPESAFE_MIRROR}/zinc/0.3.15" \ + "zinc-0.3.15.tgz" \ "${zinc_path}" ZINC_BIN="${_DIR}/${zinc_path}" } @@ -95,12 +107,12 @@ install_zinc() { # the build/ folder install_scala() { # determine the Scala version used in Spark - local scala_version=`grep "scala.version" "${_DIR}/../pom.xml" | \ - head -1 | cut -f2 -d'>' | cut -f1 -d'<'` + local scala_version=`grep "scala.version" "${_DIR}/../pom.xml" | head -n1 | awk -F '[<>]' '{print $3}'` local scala_bin="${_DIR}/scala-${scala_version}/bin/scala" + local TYPESAFE_MIRROR=${TYPESAFE_MIRROR:-https://downloads.typesafe.com} install_app \ - "http://downloads.typesafe.com/scala/${scala_version}" \ + "${TYPESAFE_MIRROR}/scala/${scala_version}" \ "scala-${scala_version}.tgz" \ "scala-${scala_version}/bin/scala" @@ -112,23 +124,16 @@ install_scala() { # the environment ZINC_PORT=${ZINC_PORT:-"3030"} -# Check for the `--force` flag dictating that `mvn` should be downloaded -# regardless of whether the system already has a `mvn` install +# Remove `--force` for backward compatibility. if [ "$1" == "--force" ]; then - FORCE_MVN=1 + echo "WARNING: '--force' is deprecated and ignored." shift fi -# Install Maven if necessary -MVN_BIN="$(command -v mvn)" - -if [ ! "$MVN_BIN" -o -n "$FORCE_MVN" ]; then - install_mvn -fi - -# Install the proper version of Scala and Zinc for the build +# Install the proper version of Scala, Zinc and Maven for the build install_zinc install_scala +install_mvn # Reset the current working directory cd "${_CALLING_DIR}" diff --git a/build/sbt-launch-lib.bash b/build/sbt-launch-lib.bash index 615f84839465..4732669ee651 100755 --- a/build/sbt-launch-lib.bash +++ b/build/sbt-launch-lib.bash @@ -117,7 +117,7 @@ get_mem_opts () { (( $perm < 4096 )) || perm=4096 local codecache=$(( $perm / 2 )) - echo "-Xms${mem}m -Xmx${mem}m -XX:MaxPermSize=${perm}m -XX:ReservedCodeCacheSize=${codecache}m" + echo "-Xms${mem}m -Xmx${mem}m -XX:ReservedCodeCacheSize=${codecache}m" } require_arg () { diff --git a/build/spark-build-info b/build/spark-build-info new file mode 100755 index 000000000000..ad0ec67f455c --- /dev/null +++ b/build/spark-build-info @@ -0,0 +1,38 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# This script generates the build info for spark and places it into the spark-version-info.properties file. +# Arguments: +# build_tgt_directory - The target directory where properties file would be created. [./core/target/extra-resources] +# spark_version - The current version of spark + +RESOURCE_DIR="$1" +mkdir -p "$RESOURCE_DIR" +SPARK_BUILD_INFO="${RESOURCE_DIR}"/spark-version-info.properties + +echo_build_properties() { + echo version=$1 + echo user=$USER + echo revision=$(git rev-parse HEAD) + echo branch=$(git rev-parse --abbrev-ref HEAD) + echo date=$(date -u +%Y-%m-%dT%H:%M:%SZ) + echo url=$(git config --get remote.origin.url) +} + +echo_build_properties $2 > "$SPARK_BUILD_INFO" diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml new file mode 100644 index 000000000000..cf93d41cd77c --- /dev/null +++ b/common/kvstore/pom.xml @@ -0,0 +1,106 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-kvstore_2.11 + jar + Spark Project Local DB + http://spark.apache.org/ + + kvstore + + + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + com.google.guava + guava + + + org.fusesource.leveldbjni + leveldbjni-all + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + + commons-io + commons-io + test + + + log4j + log4j + test + + + org.slf4j + slf4j-api + test + + + org.slf4j + slf4j-log4j12 + test + + + io.dropwizard.metrics + metrics-core + test + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/ArrayWrappers.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/ArrayWrappers.java new file mode 100644 index 000000000000..9bc8c55bd538 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/ArrayWrappers.java @@ -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.util.kvstore; + +import java.util.Arrays; + +import com.google.common.base.Preconditions; + +/** + * A factory for array wrappers so that arrays can be used as keys in a map, sorted or not. + * + * The comparator implementation makes two assumptions: + * - All elements are instances of Comparable + * - When comparing two arrays, they both contain elements of the same type in corresponding + * indices. + * + * Otherwise, ClassCastExceptions may occur. The equality method can compare any two arrays. + * + * This class is not efficient and is mostly meant to compare really small arrays, like those + * generally used as indices and keys in a KVStore. + */ +class ArrayWrappers { + + @SuppressWarnings("unchecked") + public static Comparable forArray(Object a) { + Preconditions.checkArgument(a.getClass().isArray()); + Comparable ret; + if (a instanceof int[]) { + ret = new ComparableIntArray((int[]) a); + } else if (a instanceof long[]) { + ret = new ComparableLongArray((long[]) a); + } else if (a instanceof byte[]) { + ret = new ComparableByteArray((byte[]) a); + } else { + Preconditions.checkArgument(!a.getClass().getComponentType().isPrimitive()); + ret = new ComparableObjectArray((Object[]) a); + } + return (Comparable) ret; + } + + private static class ComparableIntArray implements Comparable { + + private final int[] array; + + ComparableIntArray(int[] array) { + this.array = array; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof ComparableIntArray)) { + return false; + } + return Arrays.equals(array, ((ComparableIntArray) other).array); + } + + @Override + public int hashCode() { + int code = 0; + for (int i = 0; i < array.length; i++) { + code = (code * 31) + array[i]; + } + return code; + } + + @Override + public int compareTo(ComparableIntArray other) { + int len = Math.min(array.length, other.array.length); + for (int i = 0; i < len; i++) { + int diff = array[i] - other.array[i]; + if (diff != 0) { + return diff; + } + } + + return array.length - other.array.length; + } + } + + private static class ComparableLongArray implements Comparable { + + private final long[] array; + + ComparableLongArray(long[] array) { + this.array = array; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof ComparableLongArray)) { + return false; + } + return Arrays.equals(array, ((ComparableLongArray) other).array); + } + + @Override + public int hashCode() { + int code = 0; + for (int i = 0; i < array.length; i++) { + code = (code * 31) + (int) array[i]; + } + return code; + } + + @Override + public int compareTo(ComparableLongArray other) { + int len = Math.min(array.length, other.array.length); + for (int i = 0; i < len; i++) { + long diff = array[i] - other.array[i]; + if (diff != 0) { + return diff > 0 ? 1 : -1; + } + } + + return array.length - other.array.length; + } + } + + private static class ComparableByteArray implements Comparable { + + private final byte[] array; + + ComparableByteArray(byte[] array) { + this.array = array; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof ComparableByteArray)) { + return false; + } + return Arrays.equals(array, ((ComparableByteArray) other).array); + } + + @Override + public int hashCode() { + int code = 0; + for (int i = 0; i < array.length; i++) { + code = (code * 31) + array[i]; + } + return code; + } + + @Override + public int compareTo(ComparableByteArray other) { + int len = Math.min(array.length, other.array.length); + for (int i = 0; i < len; i++) { + int diff = array[i] - other.array[i]; + if (diff != 0) { + return diff; + } + } + + return array.length - other.array.length; + } + } + + private static class ComparableObjectArray implements Comparable { + + private final Object[] array; + + ComparableObjectArray(Object[] array) { + this.array = array; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof ComparableObjectArray)) { + return false; + } + return Arrays.equals(array, ((ComparableObjectArray) other).array); + } + + @Override + public int hashCode() { + int code = 0; + for (int i = 0; i < array.length; i++) { + code = (code * 31) + array[i].hashCode(); + } + return code; + } + + @Override + @SuppressWarnings("unchecked") + public int compareTo(ComparableObjectArray other) { + int len = Math.min(array.length, other.array.length); + for (int i = 0; i < len; i++) { + int diff = ((Comparable) array[i]).compareTo((Comparable) other.array[i]); + if (diff != 0) { + return diff; + } + } + + return array.length - other.array.length; + } + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java new file mode 100644 index 000000000000..5ca437128519 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java @@ -0,0 +1,320 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; + +import org.apache.spark.annotation.Private; + +/** + * Implementation of KVStore that keeps data deserialized in memory. This store does not index + * data; instead, whenever iterating over an indexed field, the stored data is copied and sorted + * according to the index. This saves memory but makes iteration more expensive. + */ +@Private +public class InMemoryStore implements KVStore { + + private Object metadata; + private ConcurrentMap, InstanceList> data = new ConcurrentHashMap<>(); + + @Override + public T getMetadata(Class klass) { + return klass.cast(metadata); + } + + @Override + public void setMetadata(Object value) { + this.metadata = value; + } + + @Override + public long count(Class type) { + InstanceList list = data.get(type); + return list != null ? list.size() : 0; + } + + @Override + public long count(Class type, String index, Object indexedValue) throws Exception { + InstanceList list = data.get(type); + int count = 0; + Object comparable = asKey(indexedValue); + KVTypeInfo.Accessor accessor = list.getIndexAccessor(index); + for (Object o : view(type)) { + if (Objects.equal(comparable, asKey(accessor.get(o)))) { + count++; + } + } + return count; + } + + @Override + public T read(Class klass, Object naturalKey) { + InstanceList list = data.get(klass); + Object value = list != null ? list.get(naturalKey) : null; + if (value == null) { + throw new NoSuchElementException(); + } + return klass.cast(value); + } + + @Override + public void write(Object value) throws Exception { + InstanceList list = data.computeIfAbsent(value.getClass(), key -> { + try { + return new InstanceList(key); + } catch (Exception e) { + throw Throwables.propagate(e); + } + }); + list.put(value); + } + + @Override + public void delete(Class type, Object naturalKey) { + InstanceList list = data.get(type); + if (list != null) { + list.delete(naturalKey); + } + } + + @Override + public KVStoreView view(Class type){ + InstanceList list = data.get(type); + return list != null ? list.view(type) + : new InMemoryView<>(type, Collections.emptyList(), null); + } + + @Override + public void close() { + metadata = null; + data.clear(); + } + + @SuppressWarnings("unchecked") + private static Comparable asKey(Object in) { + if (in.getClass().isArray()) { + in = ArrayWrappers.forArray(in); + } + return (Comparable) in; + } + + private static class InstanceList { + + private final KVTypeInfo ti; + private final KVTypeInfo.Accessor naturalKey; + private final ConcurrentMap, Object> data; + + private int size; + + private InstanceList(Class type) throws Exception { + this.ti = new KVTypeInfo(type); + this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME); + this.data = new ConcurrentHashMap<>(); + this.size = 0; + } + + KVTypeInfo.Accessor getIndexAccessor(String indexName) { + return ti.getAccessor(indexName); + } + + public Object get(Object key) { + return data.get(asKey(key)); + } + + public void put(Object value) throws Exception { + Preconditions.checkArgument(ti.type().equals(value.getClass()), + "Unexpected type: %s", value.getClass()); + if (data.put(asKey(naturalKey.get(value)), value) == null) { + size++; + } + } + + public void delete(Object key) { + if (data.remove(asKey(key)) != null) { + size--; + } + } + + public int size() { + return size; + } + + @SuppressWarnings("unchecked") + public InMemoryView view(Class type) { + Preconditions.checkArgument(ti.type().equals(type), "Unexpected type: %s", type); + Collection all = (Collection) data.values(); + return new InMemoryView<>(type, all, ti); + } + + } + + private static class InMemoryView extends KVStoreView { + + private final Collection elements; + private final KVTypeInfo ti; + private final KVTypeInfo.Accessor natural; + + InMemoryView(Class type, Collection elements, KVTypeInfo ti) { + super(type); + this.elements = elements; + this.ti = ti; + this.natural = ti != null ? ti.getAccessor(KVIndex.NATURAL_INDEX_NAME) : null; + } + + @Override + public Iterator iterator() { + if (elements.isEmpty()) { + return new InMemoryIterator<>(elements.iterator()); + } + + try { + KVTypeInfo.Accessor getter = index != null ? ti.getAccessor(index) : null; + int modifier = ascending ? 1 : -1; + + final List sorted = copyElements(); + Collections.sort(sorted, (e1, e2) -> modifier * compare(e1, e2, getter)); + Stream stream = sorted.stream(); + + if (first != null) { + stream = stream.filter(e -> modifier * compare(e, getter, first) >= 0); + } + + if (last != null) { + stream = stream.filter(e -> modifier * compare(e, getter, last) <= 0); + } + + if (skip > 0) { + stream = stream.skip(skip); + } + + if (max < sorted.size()) { + stream = stream.limit((int) max); + } + + return new InMemoryIterator<>(stream.iterator()); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + + /** + * Create a copy of the input elements, filtering the values for child indices if needed. + */ + private List copyElements() { + if (parent != null) { + KVTypeInfo.Accessor parentGetter = ti.getParentAccessor(index); + Preconditions.checkArgument(parentGetter != null, "Parent filter for non-child index."); + + return elements.stream() + .filter(e -> compare(e, parentGetter, parent) == 0) + .collect(Collectors.toList()); + } else { + return new ArrayList<>(elements); + } + } + + private int compare(T e1, T e2, KVTypeInfo.Accessor getter) { + try { + int diff = compare(e1, getter, getter.get(e2)); + if (diff == 0 && getter != natural) { + diff = compare(e1, natural, natural.get(e2)); + } + return diff; + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + + private int compare(T e1, KVTypeInfo.Accessor getter, Object v2) { + try { + return asKey(getter.get(e1)).compareTo(asKey(v2)); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + + } + + private static class InMemoryIterator implements KVStoreIterator { + + private final Iterator iter; + + InMemoryIterator(Iterator iter) { + this.iter = iter; + } + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public T next() { + return iter.next(); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + @Override + public List next(int max) { + List list = new ArrayList<>(max); + while (hasNext() && list.size() < max) { + list.add(next()); + } + return list; + } + + @Override + public boolean skip(long n) { + long skipped = 0; + while (skipped < n) { + if (hasNext()) { + next(); + skipped++; + } else { + return false; + } + } + + return hasNext(); + } + + @Override + public void close() { + // no op. + } + + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVIndex.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVIndex.java new file mode 100644 index 000000000000..80f492110724 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVIndex.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +import org.apache.spark.annotation.Private; + +/** + * Tags a field to be indexed when storing an object. + * + *

+ * Types are required to have a natural index that uniquely identifies instances in the store. + * The default value of the annotation identifies the natural index for the type. + *

+ * + *

+ * Indexes allow for more efficient sorting of data read from the store. By annotating a field or + * "getter" method with this annotation, an index will be created that will provide sorting based on + * the string value of that field. + *

+ * + *

+ * Note that creating indices means more space will be needed, and maintenance operations like + * updating or deleting a value will become more expensive. + *

+ * + *

+ * Indices are restricted to String, integral types (byte, short, int, long, boolean), and arrays + * of those values. + *

+ */ +@Private +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.FIELD, ElementType.METHOD}) +public @interface KVIndex { + + String NATURAL_INDEX_NAME = "__main__"; + + /** + * The name of the index to be created for the annotated entity. Must be unique within + * the class. Index names are not allowed to start with an underscore (that's reserved for + * internal use). The default value is the natural index name (which is always a copy index + * regardless of the annotation's values). + */ + String value() default NATURAL_INDEX_NAME; + + /** + * The name of the parent index of this index. By default there is no parent index, so the + * generated data can be retrieved without having to provide a parent value. + * + *

+ * If a parent index is defined, iterating over the data using the index will require providing + * a single value for the parent index. This serves as a rudimentary way to provide relationships + * between entities in the store. + *

+ */ + String parent() default ""; + + /** + * Whether to copy the instance's data to the index, instead of just storing a pointer to the + * data. The default behavior is to just store a reference; that saves disk space but is slower + * to read, since there's a level of indirection. + */ + boolean copy() default false; + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStore.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStore.java new file mode 100644 index 000000000000..72d06a8ca807 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStore.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import java.io.Closeable; + +import org.apache.spark.annotation.Private; + +/** + * Abstraction for a local key/value store for storing app data. + * + *

+ * There are two main features provided by the implementations of this interface: + *

+ * + *

Serialization

+ * + *

+ * If the underlying data store requires serialization, data will be serialized to and deserialized + * using a {@link KVStoreSerializer}, which can be customized by the application. The serializer is + * based on Jackson, so it supports all the Jackson annotations for controlling the serialization of + * app-defined types. + *

+ * + *

+ * Data is also automatically compressed to save disk space. + *

+ * + *

Automatic Key Management

+ * + *

+ * When using the built-in key management, the implementation will automatically create unique + * keys for each type written to the store. Keys are based on the type name, and always start + * with the "+" prefix character (so that it's easy to use both manual and automatic key + * management APIs without conflicts). + *

+ * + *

+ * Another feature of automatic key management is indexing; by annotating fields or methods of + * objects written to the store with {@link KVIndex}, indices are created to sort the data + * by the values of those properties. This makes it possible to provide sorting without having + * to load all instances of those types from the store. + *

+ * + *

+ * KVStore instances are thread-safe for both reads and writes. + *

+ */ +@Private +public interface KVStore extends Closeable { + + /** + * Returns app-specific metadata from the store, or null if it's not currently set. + * + *

+ * The metadata type is application-specific. This is a convenience method so that applications + * don't need to define their own keys for this information. + *

+ */ + T getMetadata(Class klass) throws Exception; + + /** + * Writes the given value in the store metadata key. + */ + void setMetadata(Object value) throws Exception; + + /** + * Read a specific instance of an object. + * + * @param naturalKey The object's "natural key", which uniquely identifies it. Null keys + * are not allowed. + * @throws java.util.NoSuchElementException If an element with the given key does not exist. + */ + T read(Class klass, Object naturalKey) throws Exception; + + /** + * Writes the given object to the store, including indexed fields. Indices are updated based + * on the annotated fields of the object's class. + * + *

+ * Writes may be slower when the object already exists in the store, since it will involve + * updating existing indices. + *

+ * + * @param value The object to write. + */ + void write(Object value) throws Exception; + + /** + * Removes an object and all data related to it, like index entries, from the store. + * + * @param type The object's type. + * @param naturalKey The object's "natural key", which uniquely identifies it. Null keys + * are not allowed. + * @throws java.util.NoSuchElementException If an element with the given key does not exist. + */ + void delete(Class type, Object naturalKey) throws Exception; + + /** + * Returns a configurable view for iterating over entities of the given type. + */ + KVStoreView view(Class type) throws Exception; + + /** + * Returns the number of items of the given type currently in the store. + */ + long count(Class type) throws Exception; + + /** + * Returns the number of items of the given type which match the given indexed value. + */ + long count(Class type, String index, Object indexedValue) throws Exception; + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreIterator.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreIterator.java new file mode 100644 index 000000000000..e6254a9368ff --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreIterator.java @@ -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.util.kvstore; + +import java.io.Closeable; +import java.util.Iterator; +import java.util.List; + +import org.apache.spark.annotation.Private; + +/** + * An iterator for KVStore. + * + *

+ * Iterators may keep references to resources that need to be closed. It's recommended that users + * explicitly close iterators after they're used. + *

+ */ +@Private +public interface KVStoreIterator extends Iterator, Closeable { + + /** + * Retrieve multiple elements from the store. + * + * @param max Maximum number of elements to retrieve. + */ + List next(int max); + + /** + * Skip in the iterator. + * + * @return Whether there are items left after skipping. + */ + boolean skip(long n); + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreSerializer.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreSerializer.java new file mode 100644 index 000000000000..bd8d9486acde --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreSerializer.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.util.zip.GZIPInputStream; +import java.util.zip.GZIPOutputStream; +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.spark.annotation.Private; + +/** + * Serializer used to translate between app-defined types and the LevelDB store. + * + *

+ * The serializer is based on Jackson, so values are written as JSON. It also allows "naked strings" + * and integers to be written as values directly, which will be written as UTF-8 strings. + *

+ */ +@Private +public class KVStoreSerializer { + + /** + * Object mapper used to process app-specific types. If an application requires a specific + * configuration of the mapper, it can subclass this serializer and add custom configuration + * to this object. + */ + protected final ObjectMapper mapper; + + public KVStoreSerializer() { + this.mapper = new ObjectMapper(); + } + + public final byte[] serialize(Object o) throws Exception { + if (o instanceof String) { + return ((String) o).getBytes(UTF_8); + } else { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + GZIPOutputStream out = new GZIPOutputStream(bytes); + try { + mapper.writeValue(out, o); + } finally { + out.close(); + } + return bytes.toByteArray(); + } + } + + @SuppressWarnings("unchecked") + public final T deserialize(byte[] data, Class klass) throws Exception { + if (klass.equals(String.class)) { + return (T) new String(data, UTF_8); + } else { + GZIPInputStream in = new GZIPInputStream(new ByteArrayInputStream(data)); + try { + return mapper.readValue(in, klass); + } finally { + in.close(); + } + } + } + + final byte[] serialize(long value) { + return String.valueOf(value).getBytes(UTF_8); + } + + final long deserializeLong(byte[] data) { + return Long.parseLong(new String(data, UTF_8)); + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreView.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreView.java new file mode 100644 index 000000000000..8ea79bbe160d --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreView.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import com.google.common.base.Preconditions; + +import org.apache.spark.annotation.Private; + +/** + * A configurable view that allows iterating over values in a {@link KVStore}. + * + *

+ * The different methods can be used to configure the behavior of the iterator. Calling the same + * method multiple times is allowed; the most recent value will be used. + *

+ * + *

+ * The iterators returned by this view are of type {@link KVStoreIterator}; they auto-close + * when used in a for loop that exhausts their contents, but when used manually, they need + * to be closed explicitly unless all elements are read. + *

+ */ +@Private +public abstract class KVStoreView implements Iterable { + + final Class type; + + boolean ascending = true; + String index = KVIndex.NATURAL_INDEX_NAME; + Object first = null; + Object last = null; + Object parent = null; + long skip = 0L; + long max = Long.MAX_VALUE; + + public KVStoreView(Class type) { + this.type = type; + } + + /** + * Reverses the order of iteration. By default, iterates in ascending order. + */ + public KVStoreView reverse() { + ascending = !ascending; + return this; + } + + /** + * Iterates according to the given index. + */ + public KVStoreView index(String name) { + this.index = Preconditions.checkNotNull(name); + return this; + } + + /** + * Defines the value of the parent index when iterating over a child index. Only elements that + * match the parent index's value will be included in the iteration. + * + *

+ * Required for iterating over child indices, will generate an error if iterating over a + * parent-less index. + *

+ */ + public KVStoreView parent(Object value) { + this.parent = value; + return this; + } + + /** + * Iterates starting at the given value of the chosen index (inclusive). + */ + public KVStoreView first(Object value) { + this.first = value; + return this; + } + + /** + * Stops iteration at the given value of the chosen index (inclusive). + */ + public KVStoreView last(Object value) { + this.last = value; + return this; + } + + /** + * Stops iteration after a number of elements has been retrieved. + */ + public KVStoreView max(long max) { + Preconditions.checkArgument(max > 0L, "max must be positive."); + this.max = max; + return this; + } + + /** + * Skips a number of elements at the start of iteration. Skipped elements are not accounted + * when using {@link #max(long)}. + */ + public KVStoreView skip(long n) { + this.skip = n; + return this; + } + + /** + * Returns an iterator for the current configuration. + */ + public KVStoreIterator closeableIterator() throws Exception { + return (KVStoreIterator) iterator(); + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java new file mode 100644 index 000000000000..a2b077e4531e --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Stream; + +import com.google.common.base.Preconditions; + +import org.apache.spark.annotation.Private; + +/** + * Wrapper around types managed in a KVStore, providing easy access to their indexed fields. + */ +@Private +public class KVTypeInfo { + + private final Class type; + private final Map indices; + private final Map accessors; + + public KVTypeInfo(Class type) throws Exception { + this.type = type; + this.accessors = new HashMap<>(); + this.indices = new HashMap<>(); + + for (Field f : type.getDeclaredFields()) { + KVIndex idx = f.getAnnotation(KVIndex.class); + if (idx != null) { + checkIndex(idx, indices); + indices.put(idx.value(), idx); + f.setAccessible(true); + accessors.put(idx.value(), new FieldAccessor(f)); + } + } + + for (Method m : type.getDeclaredMethods()) { + KVIndex idx = m.getAnnotation(KVIndex.class); + if (idx != null) { + checkIndex(idx, indices); + Preconditions.checkArgument(m.getParameterTypes().length == 0, + "Annotated method %s::%s should not have any parameters.", type.getName(), m.getName()); + indices.put(idx.value(), idx); + m.setAccessible(true); + accessors.put(idx.value(), new MethodAccessor(m)); + } + } + + Preconditions.checkArgument(indices.containsKey(KVIndex.NATURAL_INDEX_NAME), + "No natural index defined for type %s.", type.getName()); + Preconditions.checkArgument(indices.get(KVIndex.NATURAL_INDEX_NAME).parent().isEmpty(), + "Natural index of %s cannot have a parent.", type.getName()); + + for (KVIndex idx : indices.values()) { + if (!idx.parent().isEmpty()) { + KVIndex parent = indices.get(idx.parent()); + Preconditions.checkArgument(parent != null, + "Cannot find parent %s of index %s.", idx.parent(), idx.value()); + Preconditions.checkArgument(parent.parent().isEmpty(), + "Parent index %s of index %s cannot be itself a child index.", idx.parent(), idx.value()); + } + } + } + + private void checkIndex(KVIndex idx, Map indices) { + Preconditions.checkArgument(idx.value() != null && !idx.value().isEmpty(), + "No name provided for index in type %s.", type.getName()); + Preconditions.checkArgument( + !idx.value().startsWith("_") || idx.value().equals(KVIndex.NATURAL_INDEX_NAME), + "Index name %s (in type %s) is not allowed.", idx.value(), type.getName()); + Preconditions.checkArgument(idx.parent().isEmpty() || !idx.parent().equals(idx.value()), + "Index %s cannot be parent of itself.", idx.value()); + Preconditions.checkArgument(!indices.containsKey(idx.value()), + "Duplicate index %s for type %s.", idx.value(), type.getName()); + } + + public Class type() { + return type; + } + + public Object getIndexValue(String indexName, Object instance) throws Exception { + return getAccessor(indexName).get(instance); + } + + public Stream indices() { + return indices.values().stream(); + } + + Accessor getAccessor(String indexName) { + Accessor a = accessors.get(indexName); + Preconditions.checkArgument(a != null, "No index %s.", indexName); + return a; + } + + Accessor getParentAccessor(String indexName) { + KVIndex index = indices.get(indexName); + return index.parent().isEmpty() ? null : getAccessor(index.parent()); + } + + /** + * Abstracts the difference between invoking a Field and a Method. + */ + interface Accessor { + + Object get(Object instance) throws Exception; + + } + + private class FieldAccessor implements Accessor { + + private final Field field; + + FieldAccessor(Field field) { + this.field = field; + } + + @Override + public Object get(Object instance) throws Exception { + return field.get(instance); + } + + } + + private class MethodAccessor implements Accessor { + + private final Method method; + + MethodAccessor(Method method) { + this.method = method; + } + + @Override + public Object get(Object instance) throws Exception { + return method.invoke(instance); + } + + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java new file mode 100644 index 000000000000..310febc352ef --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java @@ -0,0 +1,310 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicReference; +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import org.fusesource.leveldbjni.JniDBFactory; +import org.iq80.leveldb.DB; +import org.iq80.leveldb.Options; +import org.iq80.leveldb.WriteBatch; + +import org.apache.spark.annotation.Private; + +/** + * Implementation of KVStore that uses LevelDB as the underlying data store. + */ +@Private +public class LevelDB implements KVStore { + + @VisibleForTesting + static final long STORE_VERSION = 1L; + + @VisibleForTesting + static final byte[] STORE_VERSION_KEY = "__version__".getBytes(UTF_8); + + /** DB key where app metadata is stored. */ + private static final byte[] METADATA_KEY = "__meta__".getBytes(UTF_8); + + /** DB key where type aliases are stored. */ + private static final byte[] TYPE_ALIASES_KEY = "__types__".getBytes(UTF_8); + + final AtomicReference _db; + final KVStoreSerializer serializer; + + /** + * Keep a mapping of class names to a shorter, unique ID managed by the store. This serves two + * purposes: make the keys stored on disk shorter, and spread out the keys, since class names + * will often have a long, redundant prefix (think "org.apache.spark."). + */ + private final ConcurrentMap typeAliases; + private final ConcurrentMap, LevelDBTypeInfo> types; + + public LevelDB(File path) throws Exception { + this(path, new KVStoreSerializer()); + } + + public LevelDB(File path, KVStoreSerializer serializer) throws Exception { + this.serializer = serializer; + this.types = new ConcurrentHashMap<>(); + + Options options = new Options(); + options.createIfMissing(!path.exists()); + this._db = new AtomicReference<>(JniDBFactory.factory.open(path, options)); + + byte[] versionData = db().get(STORE_VERSION_KEY); + if (versionData != null) { + long version = serializer.deserializeLong(versionData); + if (version != STORE_VERSION) { + throw new UnsupportedStoreVersionException(); + } + } else { + db().put(STORE_VERSION_KEY, serializer.serialize(STORE_VERSION)); + } + + Map aliases; + try { + aliases = get(TYPE_ALIASES_KEY, TypeAliases.class).aliases; + } catch (NoSuchElementException e) { + aliases = new HashMap<>(); + } + typeAliases = new ConcurrentHashMap<>(aliases); + } + + @Override + public T getMetadata(Class klass) throws Exception { + try { + return get(METADATA_KEY, klass); + } catch (NoSuchElementException nsee) { + return null; + } + } + + @Override + public void setMetadata(Object value) throws Exception { + if (value != null) { + put(METADATA_KEY, value); + } else { + db().delete(METADATA_KEY); + } + } + + T get(byte[] key, Class klass) throws Exception { + byte[] data = db().get(key); + if (data == null) { + throw new NoSuchElementException(new String(key, UTF_8)); + } + return serializer.deserialize(data, klass); + } + + private void put(byte[] key, Object value) throws Exception { + Preconditions.checkArgument(value != null, "Null values are not allowed."); + db().put(key, serializer.serialize(value)); + } + + @Override + public T read(Class klass, Object naturalKey) throws Exception { + Preconditions.checkArgument(naturalKey != null, "Null keys are not allowed."); + byte[] key = getTypeInfo(klass).naturalIndex().start(null, naturalKey); + return get(key, klass); + } + + @Override + public void write(Object value) throws Exception { + Preconditions.checkArgument(value != null, "Null values are not allowed."); + LevelDBTypeInfo ti = getTypeInfo(value.getClass()); + + try (WriteBatch batch = db().createWriteBatch()) { + byte[] data = serializer.serialize(value); + synchronized (ti) { + Object existing; + try { + existing = get(ti.naturalIndex().entityKey(null, value), value.getClass()); + } catch (NoSuchElementException e) { + existing = null; + } + + PrefixCache cache = new PrefixCache(value); + byte[] naturalKey = ti.naturalIndex().toKey(ti.naturalIndex().getValue(value)); + for (LevelDBTypeInfo.Index idx : ti.indices()) { + byte[] prefix = cache.getPrefix(idx); + idx.add(batch, value, existing, data, naturalKey, prefix); + } + db().write(batch); + } + } + } + + @Override + public void delete(Class type, Object naturalKey) throws Exception { + Preconditions.checkArgument(naturalKey != null, "Null keys are not allowed."); + try (WriteBatch batch = db().createWriteBatch()) { + LevelDBTypeInfo ti = getTypeInfo(type); + byte[] key = ti.naturalIndex().start(null, naturalKey); + synchronized (ti) { + byte[] data = db().get(key); + if (data != null) { + Object existing = serializer.deserialize(data, type); + PrefixCache cache = new PrefixCache(existing); + byte[] keyBytes = ti.naturalIndex().toKey(ti.naturalIndex().getValue(existing)); + for (LevelDBTypeInfo.Index idx : ti.indices()) { + idx.remove(batch, existing, keyBytes, cache.getPrefix(idx)); + } + db().write(batch); + } + } + } catch (NoSuchElementException nse) { + // Ignore. + } + } + + @Override + public KVStoreView view(Class type) throws Exception { + return new KVStoreView(type) { + @Override + public Iterator iterator() { + try { + return new LevelDBIterator<>(LevelDB.this, this); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + }; + } + + @Override + public long count(Class type) throws Exception { + LevelDBTypeInfo.Index idx = getTypeInfo(type).naturalIndex(); + return idx.getCount(idx.end(null)); + } + + @Override + public long count(Class type, String index, Object indexedValue) throws Exception { + LevelDBTypeInfo.Index idx = getTypeInfo(type).index(index); + return idx.getCount(idx.end(null, indexedValue)); + } + + @Override + public void close() throws IOException { + DB _db = this._db.getAndSet(null); + if (_db == null) { + return; + } + + try { + _db.close(); + } catch (IOException ioe) { + throw ioe; + } catch (Exception e) { + throw new IOException(e.getMessage(), e); + } + } + + /** Returns metadata about indices for the given type. */ + LevelDBTypeInfo getTypeInfo(Class type) throws Exception { + LevelDBTypeInfo ti = types.get(type); + if (ti == null) { + LevelDBTypeInfo tmp = new LevelDBTypeInfo(this, type, getTypeAlias(type)); + ti = types.putIfAbsent(type, tmp); + if (ti == null) { + ti = tmp; + } + } + return ti; + } + + /** + * Try to avoid use-after close since that has the tendency of crashing the JVM. This doesn't + * prevent methods that retrieved the instance from using it after close, but hopefully will + * catch most cases; otherwise, we'll need some kind of locking. + */ + DB db() { + DB _db = this._db.get(); + if (_db == null) { + throw new IllegalStateException("DB is closed."); + } + return _db; + } + + private byte[] getTypeAlias(Class klass) throws Exception { + byte[] alias = typeAliases.get(klass.getName()); + if (alias == null) { + synchronized (typeAliases) { + byte[] tmp = String.valueOf(typeAliases.size()).getBytes(UTF_8); + alias = typeAliases.putIfAbsent(klass.getName(), tmp); + if (alias == null) { + alias = tmp; + put(TYPE_ALIASES_KEY, new TypeAliases(typeAliases)); + } + } + } + return alias; + } + + /** Needs to be public for Jackson. */ + public static class TypeAliases { + + public Map aliases; + + TypeAliases(Map aliases) { + this.aliases = aliases; + } + + TypeAliases() { + this(null); + } + + } + + private static class PrefixCache { + + private final Object entity; + private final Map prefixes; + + PrefixCache(Object entity) { + this.entity = entity; + this.prefixes = new HashMap<>(); + } + + byte[] getPrefix(LevelDBTypeInfo.Index idx) throws Exception { + byte[] prefix = null; + if (idx.isChild()) { + prefix = prefixes.get(idx.parent()); + if (prefix == null) { + prefix = idx.parent().childPrefix(idx.parent().getValue(entity)); + prefixes.put(idx.parent(), prefix); + } + } + return prefix; + } + + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java new file mode 100644 index 000000000000..a2181f3874f8 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java @@ -0,0 +1,277 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import org.iq80.leveldb.DBIterator; + +class LevelDBIterator implements KVStoreIterator { + + private final LevelDB db; + private final boolean ascending; + private final DBIterator it; + private final Class type; + private final LevelDBTypeInfo ti; + private final LevelDBTypeInfo.Index index; + private final byte[] indexKeyPrefix; + private final byte[] end; + private final long max; + + private boolean checkedNext; + private byte[] next; + private boolean closed; + private long count; + + LevelDBIterator(LevelDB db, KVStoreView params) throws Exception { + this.db = db; + this.ascending = params.ascending; + this.it = db.db().iterator(); + this.type = params.type; + this.ti = db.getTypeInfo(type); + this.index = ti.index(params.index); + this.max = params.max; + + Preconditions.checkArgument(!index.isChild() || params.parent != null, + "Cannot iterate over child index %s without parent value.", params.index); + byte[] parent = index.isChild() ? index.parent().childPrefix(params.parent) : null; + + this.indexKeyPrefix = index.keyPrefix(parent); + + byte[] firstKey; + if (params.first != null) { + if (ascending) { + firstKey = index.start(parent, params.first); + } else { + firstKey = index.end(parent, params.first); + } + } else if (ascending) { + firstKey = index.keyPrefix(parent); + } else { + firstKey = index.end(parent); + } + it.seek(firstKey); + + byte[] end = null; + if (ascending) { + if (params.last != null) { + end = index.end(parent, params.last); + } else { + end = index.end(parent); + } + } else { + if (params.last != null) { + end = index.start(parent, params.last); + } + if (it.hasNext()) { + // When descending, the caller may have set up the start of iteration at a non-existant + // entry that is guaranteed to be after the desired entry. For example, if you have a + // compound key (a, b) where b is a, integer, you may seek to the end of the elements that + // have the same "a" value by specifying Integer.MAX_VALUE for "b", and that value may not + // exist in the database. So need to check here whether the next value actually belongs to + // the set being returned by the iterator before advancing. + byte[] nextKey = it.peekNext().getKey(); + if (compare(nextKey, indexKeyPrefix) <= 0) { + it.next(); + } + } + } + this.end = end; + + if (params.skip > 0) { + skip(params.skip); + } + } + + @Override + public boolean hasNext() { + if (!checkedNext && !closed) { + next = loadNext(); + checkedNext = true; + } + if (!closed && next == null) { + try { + close(); + } catch (IOException ioe) { + throw Throwables.propagate(ioe); + } + } + return next != null; + } + + @Override + public T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + checkedNext = false; + + try { + T ret; + if (index == null || index.isCopy()) { + ret = db.serializer.deserialize(next, type); + } else { + byte[] key = ti.buildKey(false, ti.naturalIndex().keyPrefix(null), next); + ret = db.get(key, type); + } + next = null; + return ret; + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + @Override + public List next(int max) { + List list = new ArrayList<>(max); + while (hasNext() && list.size() < max) { + list.add(next()); + } + return list; + } + + @Override + public boolean skip(long n) { + long skipped = 0; + while (skipped < n) { + if (next != null) { + checkedNext = false; + next = null; + skipped++; + continue; + } + + boolean hasNext = ascending ? it.hasNext() : it.hasPrev(); + if (!hasNext) { + checkedNext = true; + return false; + } + + Map.Entry e = ascending ? it.next() : it.prev(); + if (!isEndMarker(e.getKey())) { + skipped++; + } + } + + return hasNext(); + } + + @Override + public synchronized void close() throws IOException { + if (!closed) { + it.close(); + closed = true; + } + } + + private byte[] loadNext() { + if (count >= max) { + return null; + } + + try { + while (true) { + boolean hasNext = ascending ? it.hasNext() : it.hasPrev(); + if (!hasNext) { + return null; + } + + Map.Entry nextEntry; + try { + // Avoid races if another thread is updating the DB. + nextEntry = ascending ? it.next() : it.prev(); + } catch (NoSuchElementException e) { + return null; + } + + byte[] nextKey = nextEntry.getKey(); + // Next key is not part of the index, stop. + if (!startsWith(nextKey, indexKeyPrefix)) { + return null; + } + + // If the next key is an end marker, then skip it. + if (isEndMarker(nextKey)) { + continue; + } + + // If there's a known end key and iteration has gone past it, stop. + if (end != null) { + int comp = compare(nextKey, end) * (ascending ? 1 : -1); + if (comp > 0) { + return null; + } + } + + count++; + + // Next element is part of the iteration, return it. + return nextEntry.getValue(); + } + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + + @VisibleForTesting + static boolean startsWith(byte[] key, byte[] prefix) { + if (key.length < prefix.length) { + return false; + } + + for (int i = 0; i < prefix.length; i++) { + if (key[i] != prefix[i]) { + return false; + } + } + + return true; + } + + private boolean isEndMarker(byte[] key) { + return (key.length > 2 && + key[key.length - 2] == LevelDBTypeInfo.KEY_SEPARATOR && + key[key.length - 1] == LevelDBTypeInfo.END_MARKER[0]); + } + + static int compare(byte[] a, byte[] b) { + int diff = 0; + int minLen = Math.min(a.length, b.length); + for (int i = 0; i < minLen; i++) { + diff += (a[i] - b[i]); + if (diff != 0) { + return diff; + } + } + + return a.length - b.length; + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java new file mode 100644 index 000000000000..232ee41dd0b1 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java @@ -0,0 +1,511 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import java.lang.reflect.Array; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.google.common.base.Preconditions; +import org.iq80.leveldb.WriteBatch; + +/** + * Holds metadata about app-specific types stored in LevelDB. Serves as a cache for data collected + * via reflection, to make it cheaper to access it multiple times. + * + *

+ * The hierarchy of keys stored in LevelDB looks roughly like the following. This hierarchy ensures + * that iteration over indices is easy, and that updating values in the store is not overly + * expensive. Of note, indices choose using more disk space (one value per key) instead of keeping + * lists of pointers, which would be more expensive to update at runtime. + *

+ * + *

+ * Indentation defines when a sub-key lives under a parent key. In LevelDB, this means the full + * key would be the concatenation of everything up to that point in the hierarchy, with each + * component separated by a NULL byte. + *

+ * + *
+ * +TYPE_NAME
+ *   NATURAL_INDEX
+ *     +NATURAL_KEY
+ *     -
+ *   -NATURAL_INDEX
+ *   INDEX_NAME
+ *     +INDEX_VALUE
+ *       +NATURAL_KEY
+ *     -INDEX_VALUE
+ *     .INDEX_VALUE
+ *       CHILD_INDEX_NAME
+ *         +CHILD_INDEX_VALUE
+ *           NATURAL_KEY_OR_DATA
+ *         -
+ *   -INDEX_NAME
+ * 
+ * + *

+ * Entity data (either the entity's natural key or a copy of the data) is stored in all keys + * that end with "+". A count of all objects that match a particular top-level index + * value is kept at the end marker ("-"). A count is also kept at the natural index's end + * marker, to make it easy to retrieve the number of all elements of a particular type. + *

+ * + *

+ * To illustrate, given a type "Foo", with a natural index and a second index called "bar", you'd + * have these keys and values in the store for two instances, one with natural key "key1" and the + * other "key2", both with value "yes" for "bar": + *

+ * + *
+ * Foo __main__ +key1   [data for instance 1]
+ * Foo __main__ +key2   [data for instance 2]
+ * Foo __main__ -       [count of all Foo]
+ * Foo bar +yes +key1   [instance 1 key or data, depending on index type]
+ * Foo bar +yes +key2   [instance 2 key or data, depending on index type]
+ * Foo bar +yes -       [count of all Foo with "bar=yes" ]
+ * 
+ * + *

+ * Note that all indexed values are prepended with "+", even if the index itself does not have an + * explicit end marker. This allows for easily skipping to the end of an index by telling LevelDB + * to seek to the "phantom" end marker of the index. Throughout the code and comments, this part + * of the full LevelDB key is generally referred to as the "index value" of the entity. + *

+ * + *

+ * Child indices are stored after their parent index. In the example above, let's assume there is + * a child index "child", whose parent is "bar". If both instances have value "no" for this field, + * the data in the store would look something like the following: + *

+ * + *
+ * ...
+ * Foo bar +yes -
+ * Foo bar .yes .child +no +key1   [instance 1 key or data, depending on index type]
+ * Foo bar .yes .child +no +key2   [instance 2 key or data, depending on index type]
+ * ...
+ * 
+ */ +class LevelDBTypeInfo { + + static final byte[] END_MARKER = new byte[] { '-' }; + static final byte ENTRY_PREFIX = (byte) '+'; + static final byte KEY_SEPARATOR = 0x0; + static byte TRUE = (byte) '1'; + static byte FALSE = (byte) '0'; + + private static final byte SECONDARY_IDX_PREFIX = (byte) '.'; + private static final byte POSITIVE_MARKER = (byte) '='; + private static final byte NEGATIVE_MARKER = (byte) '*'; + private static final byte[] HEX_BYTES = new byte[] { + '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f' + }; + + private final LevelDB db; + private final Class type; + private final Map indices; + private final byte[] typePrefix; + + LevelDBTypeInfo(LevelDB db, Class type, byte[] alias) throws Exception { + this.db = db; + this.type = type; + this.indices = new HashMap<>(); + + KVTypeInfo ti = new KVTypeInfo(type); + + // First create the parent indices, then the child indices. + ti.indices().forEach(idx -> { + if (idx.parent().isEmpty()) { + indices.put(idx.value(), new Index(idx, ti.getAccessor(idx.value()), null)); + } + }); + ti.indices().forEach(idx -> { + if (!idx.parent().isEmpty()) { + indices.put(idx.value(), new Index(idx, ti.getAccessor(idx.value()), + indices.get(idx.parent()))); + } + }); + + this.typePrefix = alias; + } + + Class type() { + return type; + } + + byte[] keyPrefix() { + return typePrefix; + } + + Index naturalIndex() { + return index(KVIndex.NATURAL_INDEX_NAME); + } + + Index index(String name) { + Index i = indices.get(name); + Preconditions.checkArgument(i != null, "Index %s does not exist for type %s.", name, + type.getName()); + return i; + } + + Collection indices() { + return indices.values(); + } + + byte[] buildKey(byte[]... components) { + return buildKey(true, components); + } + + byte[] buildKey(boolean addTypePrefix, byte[]... components) { + int len = 0; + if (addTypePrefix) { + len += typePrefix.length + 1; + } + for (byte[] comp : components) { + len += comp.length; + } + len += components.length - 1; + + byte[] dest = new byte[len]; + int written = 0; + + if (addTypePrefix) { + System.arraycopy(typePrefix, 0, dest, 0, typePrefix.length); + dest[typePrefix.length] = KEY_SEPARATOR; + written += typePrefix.length + 1; + } + + for (byte[] comp : components) { + System.arraycopy(comp, 0, dest, written, comp.length); + written += comp.length; + if (written < dest.length) { + dest[written] = KEY_SEPARATOR; + written++; + } + } + + return dest; + } + + /** + * Models a single index in LevelDB. See top-level class's javadoc for a description of how the + * keys are generated. + */ + class Index { + + private final boolean copy; + private final boolean isNatural; + private final byte[] name; + private final KVTypeInfo.Accessor accessor; + private final Index parent; + + private Index(KVIndex self, KVTypeInfo.Accessor accessor, Index parent) { + byte[] name = self.value().getBytes(UTF_8); + if (parent != null) { + byte[] child = new byte[name.length + 1]; + child[0] = SECONDARY_IDX_PREFIX; + System.arraycopy(name, 0, child, 1, name.length); + } + + this.name = name; + this.isNatural = self.value().equals(KVIndex.NATURAL_INDEX_NAME); + this.copy = isNatural || self.copy(); + this.accessor = accessor; + this.parent = parent; + } + + boolean isCopy() { + return copy; + } + + boolean isChild() { + return parent != null; + } + + Index parent() { + return parent; + } + + /** + * Creates a key prefix for child indices of this index. This allows the prefix to be + * calculated only once, avoiding redundant work when multiple child indices of the + * same parent index exist. + */ + byte[] childPrefix(Object value) { + Preconditions.checkState(parent == null, "Not a parent index."); + return buildKey(name, toParentKey(value)); + } + + /** + * Gets the index value for a particular entity (which is the value of the field or method + * tagged with the index annotation). This is used as part of the LevelDB key where the + * entity (or its id) is stored. + */ + Object getValue(Object entity) throws Exception { + return accessor.get(entity); + } + + private void checkParent(byte[] prefix) { + if (prefix != null) { + Preconditions.checkState(parent != null, "Parent prefix provided for parent index."); + } else { + Preconditions.checkState(parent == null, "Parent prefix missing for child index."); + } + } + + /** The prefix for all keys that belong to this index. */ + byte[] keyPrefix(byte[] prefix) { + checkParent(prefix); + return (parent != null) ? buildKey(false, prefix, name) : buildKey(name); + } + + /** + * The key where to start ascending iteration for entities whose value for the indexed field + * match the given value. + */ + byte[] start(byte[] prefix, Object value) { + checkParent(prefix); + return (parent != null) ? buildKey(false, prefix, name, toKey(value)) + : buildKey(name, toKey(value)); + } + + /** The key for the index's end marker. */ + byte[] end(byte[] prefix) { + checkParent(prefix); + return (parent != null) ? buildKey(false, prefix, name, END_MARKER) + : buildKey(name, END_MARKER); + } + + /** The key for the end marker for entries with the given value. */ + byte[] end(byte[] prefix, Object value) { + checkParent(prefix); + return (parent != null) ? buildKey(false, prefix, name, toKey(value), END_MARKER) + : buildKey(name, toKey(value), END_MARKER); + } + + /** The full key in the index that identifies the given entity. */ + byte[] entityKey(byte[] prefix, Object entity) throws Exception { + Object indexValue = getValue(entity); + Preconditions.checkNotNull(indexValue, "Null index value for %s in type %s.", + name, type.getName()); + byte[] entityKey = start(prefix, indexValue); + if (!isNatural) { + entityKey = buildKey(false, entityKey, toKey(naturalIndex().getValue(entity))); + } + return entityKey; + } + + private void updateCount(WriteBatch batch, byte[] key, long delta) { + long updated = getCount(key) + delta; + if (updated > 0) { + batch.put(key, db.serializer.serialize(updated)); + } else { + batch.delete(key); + } + } + + private void addOrRemove( + WriteBatch batch, + Object entity, + Object existing, + byte[] data, + byte[] naturalKey, + byte[] prefix) throws Exception { + Object indexValue = getValue(entity); + Preconditions.checkNotNull(indexValue, "Null index value for %s in type %s.", + name, type.getName()); + + byte[] entityKey = start(prefix, indexValue); + if (!isNatural) { + entityKey = buildKey(false, entityKey, naturalKey); + } + + boolean needCountUpdate = (existing == null); + + // Check whether there's a need to update the index. The index needs to be updated in two + // cases: + // + // - There is no existing value for the entity, so a new index value will be added. + // - If there is a previously stored value for the entity, and the index value for the + // current index does not match the new value, the old entry needs to be deleted and + // the new one added. + // + // Natural indices don't need to be checked, because by definition both old and new entities + // will have the same key. The put() call is all that's needed in that case. + // + // Also check whether we need to update the counts. If the indexed value is changing, we + // need to decrement the count at the old index value, and the new indexed value count needs + // to be incremented. + if (existing != null && !isNatural) { + byte[] oldPrefix = null; + Object oldIndexedValue = getValue(existing); + boolean removeExisting = !indexValue.equals(oldIndexedValue); + if (!removeExisting && isChild()) { + oldPrefix = parent().childPrefix(parent().getValue(existing)); + removeExisting = LevelDBIterator.compare(prefix, oldPrefix) != 0; + } + + if (removeExisting) { + if (oldPrefix == null && isChild()) { + oldPrefix = parent().childPrefix(parent().getValue(existing)); + } + + byte[] oldKey = entityKey(oldPrefix, existing); + batch.delete(oldKey); + + // If the indexed value has changed, we need to update the counts at the old and new + // end markers for the indexed value. + if (!isChild()) { + byte[] oldCountKey = end(null, oldIndexedValue); + updateCount(batch, oldCountKey, -1L); + needCountUpdate = true; + } + } + } + + if (data != null) { + byte[] stored = copy ? data : naturalKey; + batch.put(entityKey, stored); + } else { + batch.delete(entityKey); + } + + if (needCountUpdate && !isChild()) { + long delta = data != null ? 1L : -1L; + byte[] countKey = isNatural ? end(prefix) : end(prefix, indexValue); + updateCount(batch, countKey, delta); + } + } + + /** + * Add an entry to the index. + * + * @param batch Write batch with other related changes. + * @param entity The entity being added to the index. + * @param existing The entity being replaced in the index, or null. + * @param data Serialized entity to store (when storing the entity, not a reference). + * @param naturalKey The value's natural key (to avoid re-computing it for every index). + * @param prefix The parent index prefix, if this is a child index. + */ + void add( + WriteBatch batch, + Object entity, + Object existing, + byte[] data, + byte[] naturalKey, + byte[] prefix) throws Exception { + addOrRemove(batch, entity, existing, data, naturalKey, prefix); + } + + /** + * Remove a value from the index. + * + * @param batch Write batch with other related changes. + * @param entity The entity being removed, to identify the index entry to modify. + * @param naturalKey The value's natural key (to avoid re-computing it for every index). + * @param prefix The parent index prefix, if this is a child index. + */ + void remove( + WriteBatch batch, + Object entity, + byte[] naturalKey, + byte[] prefix) throws Exception { + addOrRemove(batch, entity, null, null, naturalKey, prefix); + } + + long getCount(byte[] key) { + byte[] data = db.db().get(key); + return data != null ? db.serializer.deserializeLong(data) : 0; + } + + byte[] toParentKey(Object value) { + return toKey(value, SECONDARY_IDX_PREFIX); + } + + byte[] toKey(Object value) { + return toKey(value, ENTRY_PREFIX); + } + + /** + * Translates a value to be used as part of the store key. + * + * Integral numbers are encoded as a string in a way that preserves lexicographical + * ordering. The string is prepended with a marker telling whether the number is negative + * or positive ("*" for negative and "=" for positive are used since "-" and "+" have the + * opposite of the desired order), and then the number is encoded into a hex string (so + * it occupies twice the number of bytes as the original type). + * + * Arrays are encoded by encoding each element separately, separated by KEY_SEPARATOR. + */ + byte[] toKey(Object value, byte prefix) { + final byte[] result; + + if (value instanceof String) { + byte[] str = ((String) value).getBytes(UTF_8); + result = new byte[str.length + 1]; + result[0] = prefix; + System.arraycopy(str, 0, result, 1, str.length); + } else if (value instanceof Boolean) { + result = new byte[] { prefix, (Boolean) value ? TRUE : FALSE }; + } else if (value.getClass().isArray()) { + int length = Array.getLength(value); + byte[][] components = new byte[length][]; + for (int i = 0; i < length; i++) { + components[i] = toKey(Array.get(value, i)); + } + result = buildKey(false, components); + } else { + int bytes; + + if (value instanceof Integer) { + bytes = Integer.SIZE; + } else if (value instanceof Long) { + bytes = Long.SIZE; + } else if (value instanceof Short) { + bytes = Short.SIZE; + } else if (value instanceof Byte) { + bytes = Byte.SIZE; + } else { + throw new IllegalArgumentException(String.format("Type %s not allowed as key.", + value.getClass().getName())); + } + + bytes = bytes / Byte.SIZE; + + byte[] key = new byte[bytes * 2 + 2]; + long longValue = ((Number) value).longValue(); + key[0] = prefix; + key[1] = longValue > 0 ? POSITIVE_MARKER : NEGATIVE_MARKER; + + for (int i = 0; i < key.length - 2; i++) { + int masked = (int) ((longValue >>> (4 * i)) & 0xF); + key[key.length - i - 1] = HEX_BYTES[masked]; + } + + result = key; + } + + return result; + } + + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/UnsupportedStoreVersionException.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/UnsupportedStoreVersionException.java new file mode 100644 index 000000000000..75a33b7c75de --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/UnsupportedStoreVersionException.java @@ -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.util.kvstore; + +import java.io.IOException; + +import org.apache.spark.annotation.Private; + +/** + * Exception thrown when the store implementation is not compatible with the underlying data. + */ +@Private +public class UnsupportedStoreVersionException extends IOException { + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayKeyIndexType.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayKeyIndexType.java new file mode 100644 index 000000000000..32030fb4115c --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayKeyIndexType.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import java.util.Arrays; + +public class ArrayKeyIndexType { + + @KVIndex + public int[] key; + + @KVIndex("id") + public String[] id; + + @Override + public boolean equals(Object o) { + if (o instanceof ArrayKeyIndexType) { + ArrayKeyIndexType other = (ArrayKeyIndexType) o; + return Arrays.equals(key, other.key) && Arrays.equals(id, other.id); + } + return false; + } + + @Override + public int hashCode() { + return key.hashCode(); + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayWrappersSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayWrappersSuite.java new file mode 100644 index 000000000000..b1c8927d0761 --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayWrappersSuite.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import org.junit.Test; +import static org.junit.Assert.*; + +public class ArrayWrappersSuite { + + @Test + public void testGenericArrayKey() { + byte[] b1 = new byte[] { 0x01, 0x02, 0x03 }; + byte[] b2 = new byte[] { 0x01, 0x02 }; + int[] i1 = new int[] { 1, 2, 3 }; + int[] i2 = new int[] { 1, 2 }; + String[] s1 = new String[] { "1", "2", "3" }; + String[] s2 = new String[] { "1", "2" }; + + assertEquals(ArrayWrappers.forArray(b1), ArrayWrappers.forArray(b1)); + assertNotEquals(ArrayWrappers.forArray(b1), ArrayWrappers.forArray(b2)); + assertNotEquals(ArrayWrappers.forArray(b1), ArrayWrappers.forArray(i1)); + assertNotEquals(ArrayWrappers.forArray(b1), ArrayWrappers.forArray(s1)); + + assertEquals(ArrayWrappers.forArray(i1), ArrayWrappers.forArray(i1)); + assertNotEquals(ArrayWrappers.forArray(i1), ArrayWrappers.forArray(i2)); + assertNotEquals(ArrayWrappers.forArray(i1), ArrayWrappers.forArray(b1)); + assertNotEquals(ArrayWrappers.forArray(i1), ArrayWrappers.forArray(s1)); + + assertEquals(ArrayWrappers.forArray(s1), ArrayWrappers.forArray(s1)); + assertNotEquals(ArrayWrappers.forArray(s1), ArrayWrappers.forArray(s2)); + assertNotEquals(ArrayWrappers.forArray(s1), ArrayWrappers.forArray(b1)); + assertNotEquals(ArrayWrappers.forArray(s1), ArrayWrappers.forArray(i1)); + + assertEquals(0, ArrayWrappers.forArray(b1).compareTo(ArrayWrappers.forArray(b1))); + assertTrue(ArrayWrappers.forArray(b1).compareTo(ArrayWrappers.forArray(b2)) > 0); + + assertEquals(0, ArrayWrappers.forArray(i1).compareTo(ArrayWrappers.forArray(i1))); + assertTrue(ArrayWrappers.forArray(i1).compareTo(ArrayWrappers.forArray(i2)) > 0); + + assertEquals(0, ArrayWrappers.forArray(s1).compareTo(ArrayWrappers.forArray(s1))); + assertTrue(ArrayWrappers.forArray(s1).compareTo(ArrayWrappers.forArray(s2)) > 0); + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/CustomType1.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/CustomType1.java new file mode 100644 index 000000000000..92b643b0cb92 --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/CustomType1.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.util.kvstore; + +import com.google.common.base.Objects; + +public class CustomType1 { + + @KVIndex + public String key; + + @KVIndex("id") + public String id; + + @KVIndex(value = "name", copy = true) + public String name; + + @KVIndex("int") + public int num; + + @KVIndex(value = "child", parent = "id") + public String child; + + @Override + public boolean equals(Object o) { + if (o instanceof CustomType1) { + CustomType1 other = (CustomType1) o; + return id.equals(other.id) && name.equals(other.name); + } + return false; + } + + @Override + public int hashCode() { + return id.hashCode(); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("key", key) + .add("id", id) + .add("name", name) + .add("num", num) + .toString(); + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java new file mode 100644 index 000000000000..9a81f86812cd --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java @@ -0,0 +1,504 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import java.util.Arrays; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Random; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import static org.junit.Assert.*; + +public abstract class DBIteratorSuite { + + private static final Logger LOG = LoggerFactory.getLogger(DBIteratorSuite.class); + + private static final int MIN_ENTRIES = 42; + private static final int MAX_ENTRIES = 1024; + private static final Random RND = new Random(); + + private static List allEntries; + private static List clashingEntries; + private static KVStore db; + + private interface BaseComparator extends Comparator { + /** + * Returns a comparator that falls back to natural order if this comparator's ordering + * returns equality for two elements. Used to mimic how the index sorts things internally. + */ + default BaseComparator fallback() { + return (t1, t2) -> { + int result = BaseComparator.this.compare(t1, t2); + if (result != 0) { + return result; + } + + return t1.key.compareTo(t2.key); + }; + } + + /** Reverses the order of this comparator. */ + default BaseComparator reverse() { + return (t1, t2) -> -BaseComparator.this.compare(t1, t2); + } + } + + private static final BaseComparator NATURAL_ORDER = (t1, t2) -> t1.key.compareTo(t2.key); + private static final BaseComparator REF_INDEX_ORDER = (t1, t2) -> t1.id.compareTo(t2.id); + private static final BaseComparator COPY_INDEX_ORDER = (t1, t2) -> t1.name.compareTo(t2.name); + private static final BaseComparator NUMERIC_INDEX_ORDER = (t1, t2) -> t1.num - t2.num; + private static final BaseComparator CHILD_INDEX_ORDER = (t1, t2) -> t1.child.compareTo(t2.child); + + /** + * Implementations should override this method; it is called only once, before all tests are + * run. Any state can be safely stored in static variables and cleaned up in a @AfterClass + * handler. + */ + protected abstract KVStore createStore() throws Exception; + + @BeforeClass + public static void setupClass() { + long seed = RND.nextLong(); + LOG.info("Random seed: {}", seed); + RND.setSeed(seed); + } + + @AfterClass + public static void cleanupData() throws Exception { + allEntries = null; + db = null; + } + + @Before + public void setup() throws Exception { + if (db != null) { + return; + } + + db = createStore(); + + int count = RND.nextInt(MAX_ENTRIES) + MIN_ENTRIES; + + allEntries = new ArrayList<>(count); + for (int i = 0; i < count; i++) { + CustomType1 t = new CustomType1(); + t.key = "key" + i; + t.id = "id" + i; + t.name = "name" + RND.nextInt(MAX_ENTRIES); + t.num = RND.nextInt(MAX_ENTRIES); + t.child = "child" + (i % MIN_ENTRIES); + allEntries.add(t); + } + + // Shuffle the entries to avoid the insertion order matching the natural ordering. Just in case. + Collections.shuffle(allEntries, RND); + for (CustomType1 e : allEntries) { + db.write(e); + } + + // Pick the first generated value, and forcefully create a few entries that will clash + // with the indexed values (id and name), to make sure the index behaves correctly when + // multiple entities are indexed by the same value. + // + // This also serves as a test for the test code itself, to make sure it's sorting indices + // the same way the store is expected to. + CustomType1 first = allEntries.get(0); + clashingEntries = new ArrayList<>(); + + int clashCount = RND.nextInt(MIN_ENTRIES) + 1; + for (int i = 0; i < clashCount; i++) { + CustomType1 t = new CustomType1(); + t.key = "n-key" + (count + i); + t.id = first.id; + t.name = first.name; + t.num = first.num; + t.child = first.child; + allEntries.add(t); + clashingEntries.add(t); + db.write(t); + } + + // Create another entry that could cause problems: take the first entry, and make its indexed + // name be an extension of the existing ones, to make sure the implementation sorts these + // correctly even considering the separator character (shorter strings first). + CustomType1 t = new CustomType1(); + t.key = "extended-key-0"; + t.id = first.id; + t.name = first.name + "a"; + t.num = first.num; + t.child = first.child; + allEntries.add(t); + db.write(t); + } + + @Test + public void naturalIndex() throws Exception { + testIteration(NATURAL_ORDER, view(), null, null); + } + + @Test + public void refIndex() throws Exception { + testIteration(REF_INDEX_ORDER, view().index("id"), null, null); + } + + @Test + public void copyIndex() throws Exception { + testIteration(COPY_INDEX_ORDER, view().index("name"), null, null); + } + + @Test + public void numericIndex() throws Exception { + testIteration(NUMERIC_INDEX_ORDER, view().index("int"), null, null); + } + + @Test + public void childIndex() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, view().index("child").parent(any.id), null, null); + } + + @Test + public void naturalIndexDescending() throws Exception { + testIteration(NATURAL_ORDER, view().reverse(), null, null); + } + + @Test + public void refIndexDescending() throws Exception { + testIteration(REF_INDEX_ORDER, view().index("id").reverse(), null, null); + } + + @Test + public void copyIndexDescending() throws Exception { + testIteration(COPY_INDEX_ORDER, view().index("name").reverse(), null, null); + } + + @Test + public void numericIndexDescending() throws Exception { + testIteration(NUMERIC_INDEX_ORDER, view().index("int").reverse(), null, null); + } + + @Test + public void childIndexDescending() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, view().index("child").parent(any.id).reverse(), null, null); + } + + @Test + public void naturalIndexWithStart() throws Exception { + CustomType1 first = pickLimit(); + testIteration(NATURAL_ORDER, view().first(first.key), first, null); + } + + @Test + public void refIndexWithStart() throws Exception { + CustomType1 first = pickLimit(); + testIteration(REF_INDEX_ORDER, view().index("id").first(first.id), first, null); + } + + @Test + public void copyIndexWithStart() throws Exception { + CustomType1 first = pickLimit(); + testIteration(COPY_INDEX_ORDER, view().index("name").first(first.name), first, null); + } + + @Test + public void numericIndexWithStart() throws Exception { + CustomType1 first = pickLimit(); + testIteration(NUMERIC_INDEX_ORDER, view().index("int").first(first.num), first, null); + } + + @Test + public void childIndexWithStart() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, view().index("child").parent(any.id).first(any.child), null, + null); + } + + @Test + public void naturalIndexDescendingWithStart() throws Exception { + CustomType1 first = pickLimit(); + testIteration(NATURAL_ORDER, view().reverse().first(first.key), first, null); + } + + @Test + public void refIndexDescendingWithStart() throws Exception { + CustomType1 first = pickLimit(); + testIteration(REF_INDEX_ORDER, view().reverse().index("id").first(first.id), first, null); + } + + @Test + public void copyIndexDescendingWithStart() throws Exception { + CustomType1 first = pickLimit(); + testIteration(COPY_INDEX_ORDER, view().reverse().index("name").first(first.name), first, null); + } + + @Test + public void numericIndexDescendingWithStart() throws Exception { + CustomType1 first = pickLimit(); + testIteration(NUMERIC_INDEX_ORDER, view().reverse().index("int").first(first.num), first, null); + } + + @Test + public void childIndexDescendingWithStart() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, + view().index("child").parent(any.id).first(any.child).reverse(), null, null); + } + + @Test + public void naturalIndexWithSkip() throws Exception { + testIteration(NATURAL_ORDER, view().skip(pickCount()), null, null); + } + + @Test + public void refIndexWithSkip() throws Exception { + testIteration(REF_INDEX_ORDER, view().index("id").skip(pickCount()), null, null); + } + + @Test + public void copyIndexWithSkip() throws Exception { + testIteration(COPY_INDEX_ORDER, view().index("name").skip(pickCount()), null, null); + } + + @Test + public void childIndexWithSkip() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, view().index("child").parent(any.id).skip(pickCount()), + null, null); + } + + @Test + public void naturalIndexWithMax() throws Exception { + testIteration(NATURAL_ORDER, view().max(pickCount()), null, null); + } + + @Test + public void copyIndexWithMax() throws Exception { + testIteration(COPY_INDEX_ORDER, view().index("name").max(pickCount()), null, null); + } + + @Test + public void childIndexWithMax() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, view().index("child").parent(any.id).max(pickCount()), null, + null); + } + + @Test + public void naturalIndexWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(NATURAL_ORDER, view().last(last.key), null, last); + } + + @Test + public void refIndexWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(REF_INDEX_ORDER, view().index("id").last(last.id), null, last); + } + + @Test + public void copyIndexWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(COPY_INDEX_ORDER, view().index("name").last(last.name), null, last); + } + + @Test + public void numericIndexWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(NUMERIC_INDEX_ORDER, view().index("int").last(last.num), null, last); + } + + @Test + public void childIndexWithLast() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, view().index("child").parent(any.id).last(any.child), null, + null); + } + + @Test + public void naturalIndexDescendingWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(NATURAL_ORDER, view().reverse().last(last.key), null, last); + } + + @Test + public void refIndexDescendingWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(REF_INDEX_ORDER, view().reverse().index("id").last(last.id), null, last); + } + + @Test + public void copyIndexDescendingWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(COPY_INDEX_ORDER, view().reverse().index("name").last(last.name), + null, last); + } + + @Test + public void numericIndexDescendingWithLast() throws Exception { + CustomType1 last = pickLimit(); + testIteration(NUMERIC_INDEX_ORDER, view().reverse().index("int").last(last.num), + null, last); + } + + @Test + public void childIndexDescendingWithLast() throws Exception { + CustomType1 any = pickLimit(); + testIteration(CHILD_INDEX_ORDER, view().index("child").parent(any.id).last(any.child).reverse(), + null, null); + } + + @Test + public void testRefWithIntNaturalKey() throws Exception { + LevelDBSuite.IntKeyType i = new LevelDBSuite.IntKeyType(); + i.key = 1; + i.id = "1"; + i.values = Arrays.asList("1"); + + db.write(i); + + try(KVStoreIterator it = db.view(i.getClass()).closeableIterator()) { + Object read = it.next(); + assertEquals(i, read); + } + } + + private CustomType1 pickLimit() { + // Picks an element that has clashes with other elements in the given index. + return clashingEntries.get(RND.nextInt(clashingEntries.size())); + } + + private int pickCount() { + int count = RND.nextInt(allEntries.size() / 2); + return Math.max(count, 1); + } + + /** + * Compares the two values and falls back to comparing the natural key of CustomType1 + * if they're the same, to mimic the behavior of the indexing code. + */ + private > int compareWithFallback( + T v1, + T v2, + CustomType1 ct1, + CustomType1 ct2) { + int result = v1.compareTo(v2); + if (result != 0) { + return result; + } + + return ct1.key.compareTo(ct2.key); + } + + private void testIteration( + final BaseComparator order, + final KVStoreView params, + final CustomType1 first, + final CustomType1 last) throws Exception { + List indexOrder = sortBy(order.fallback()); + if (!params.ascending) { + indexOrder = Lists.reverse(indexOrder); + } + + Iterable expected = indexOrder; + BaseComparator expectedOrder = params.ascending ? order : order.reverse(); + + if (params.parent != null) { + expected = Iterables.filter(expected, v -> params.parent.equals(v.id)); + } + + if (first != null) { + expected = Iterables.filter(expected, v -> expectedOrder.compare(first, v) <= 0); + } + + if (last != null) { + expected = Iterables.filter(expected, v -> expectedOrder.compare(v, last) <= 0); + } + + if (params.skip > 0) { + expected = Iterables.skip(expected, (int) params.skip); + } + + if (params.max != Long.MAX_VALUE) { + expected = Iterables.limit(expected, (int) params.max); + } + + List actual = collect(params); + compareLists(expected, actual); + } + + /** Could use assertEquals(), but that creates hard to read errors for large lists. */ + private void compareLists(Iterable expected, List actual) { + Iterator expectedIt = expected.iterator(); + Iterator actualIt = actual.iterator(); + + int count = 0; + while (expectedIt.hasNext()) { + if (!actualIt.hasNext()) { + break; + } + count++; + assertEquals(expectedIt.next(), actualIt.next()); + } + + String message; + Object[] remaining; + int expectedCount = count; + int actualCount = count; + + if (expectedIt.hasNext()) { + remaining = Iterators.toArray(expectedIt, Object.class); + expectedCount += remaining.length; + message = "missing"; + } else { + remaining = Iterators.toArray(actualIt, Object.class); + actualCount += remaining.length; + message = "stray"; + } + + assertEquals(String.format("Found %s elements: %s", message, Arrays.asList(remaining)), + expectedCount, actualCount); + } + + private KVStoreView view() throws Exception { + return db.view(CustomType1.class); + } + + private List collect(KVStoreView view) throws Exception { + return Arrays.asList(Iterables.toArray(view, CustomType1.class)); + } + + private List sortBy(Comparator comp) { + List copy = new ArrayList<>(allEntries); + Collections.sort(copy, comp); + return copy; + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryIteratorSuite.java new file mode 100644 index 000000000000..27dde6a9fbea --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryIteratorSuite.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +public class InMemoryIteratorSuite extends DBIteratorSuite { + + @Override + protected KVStore createStore() { + return new InMemoryStore(); + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java new file mode 100644 index 000000000000..510b3058a4e3 --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import java.util.NoSuchElementException; + +import org.junit.Test; +import static org.junit.Assert.*; + +public class InMemoryStoreSuite { + + @Test + public void testObjectWriteReadDelete() throws Exception { + KVStore store = new InMemoryStore(); + + CustomType1 t = new CustomType1(); + t.key = "key"; + t.id = "id"; + t.name = "name"; + + try { + store.read(CustomType1.class, t.key); + fail("Expected exception for non-existant object."); + } catch (NoSuchElementException nsee) { + // Expected. + } + + store.write(t); + assertEquals(t, store.read(t.getClass(), t.key)); + assertEquals(1L, store.count(t.getClass())); + + store.delete(t.getClass(), t.key); + try { + store.read(t.getClass(), t.key); + fail("Expected exception for deleted object."); + } catch (NoSuchElementException nsee) { + // Expected. + } + } + + @Test + public void testMultipleObjectWriteReadDelete() throws Exception { + KVStore store = new InMemoryStore(); + + CustomType1 t1 = new CustomType1(); + t1.key = "key1"; + t1.id = "id"; + t1.name = "name1"; + + CustomType1 t2 = new CustomType1(); + t2.key = "key2"; + t2.id = "id"; + t2.name = "name2"; + + store.write(t1); + store.write(t2); + + assertEquals(t1, store.read(t1.getClass(), t1.key)); + assertEquals(t2, store.read(t2.getClass(), t2.key)); + assertEquals(2L, store.count(t1.getClass())); + + store.delete(t1.getClass(), t1.key); + assertEquals(t2, store.read(t2.getClass(), t2.key)); + store.delete(t2.getClass(), t2.key); + try { + store.read(t2.getClass(), t2.key); + fail("Expected exception for deleted object."); + } catch (NoSuchElementException nsee) { + // Expected. + } + } + + @Test + public void testMetadata() throws Exception { + KVStore store = new InMemoryStore(); + assertNull(store.getMetadata(CustomType1.class)); + + CustomType1 t = new CustomType1(); + t.id = "id"; + t.name = "name"; + + store.setMetadata(t); + assertEquals(t, store.getMetadata(CustomType1.class)); + + store.setMetadata(null); + assertNull(store.getMetadata(CustomType1.class)); + } + + @Test + public void testUpdate() throws Exception { + KVStore store = new InMemoryStore(); + + CustomType1 t = new CustomType1(); + t.key = "key"; + t.id = "id"; + t.name = "name"; + + store.write(t); + + t.name = "anotherName"; + + store.write(t); + assertEquals(1, store.count(t.getClass())); + assertSame(t, store.read(t.getClass(), t.key)); + } + + @Test + public void testArrayIndices() throws Exception { + KVStore store = new InMemoryStore(); + + ArrayKeyIndexType o = new ArrayKeyIndexType(); + o.key = new int[] { 1, 2 }; + o.id = new String[] { "3", "4" }; + + store.write(o); + assertEquals(o, store.read(ArrayKeyIndexType.class, o.key)); + assertEquals(o, store.view(ArrayKeyIndexType.class).index("id").first(o.id).iterator().next()); + } + + @Test + public void testBasicIteration() throws Exception { + KVStore store = new InMemoryStore(); + + CustomType1 t1 = new CustomType1(); + t1.key = "1"; + t1.id = "id1"; + t1.name = "name1"; + store.write(t1); + + CustomType1 t2 = new CustomType1(); + t2.key = "2"; + t2.id = "id2"; + t2.name = "name2"; + store.write(t2); + + assertEquals(t1.id, store.view(t1.getClass()).iterator().next().id); + assertEquals(t2.id, store.view(t1.getClass()).skip(1).iterator().next().id); + assertEquals(t2.id, store.view(t1.getClass()).skip(1).max(1).iterator().next().id); + assertEquals(t1.id, + store.view(t1.getClass()).first(t1.key).max(1).iterator().next().id); + assertEquals(t2.id, + store.view(t1.getClass()).first(t2.key).max(1).iterator().next().id); + assertFalse(store.view(t1.getClass()).first(t2.id).skip(1).iterator().hasNext()); + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBBenchmark.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBBenchmark.java new file mode 100644 index 000000000000..268d025f5f06 --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBBenchmark.java @@ -0,0 +1,280 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.Slf4jReporter; +import com.codahale.metrics.Snapshot; +import com.codahale.metrics.Timer; +import org.apache.commons.io.FileUtils; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; +import org.slf4j.LoggerFactory; +import static org.junit.Assert.*; + +/** + * A set of small benchmarks for the LevelDB implementation. + * + * The benchmarks are run over two different types (one with just a natural index, and one + * with a ref index), over a set of 2^20 elements, and the following tests are performed: + * + * - write (then update) elements in sequential natural key order + * - write (then update) elements in random natural key order + * - iterate over natural index, ascending and descending + * - iterate over ref index, ascending and descending + */ +@Ignore +public class LevelDBBenchmark { + + private static final int COUNT = 1024; + private static final AtomicInteger IDGEN = new AtomicInteger(); + private static final MetricRegistry metrics = new MetricRegistry(); + private static final Timer dbCreation = metrics.timer("dbCreation"); + private static final Timer dbClose = metrics.timer("dbClose"); + + private LevelDB db; + private File dbpath; + + @Before + public void setup() throws Exception { + dbpath = File.createTempFile("test.", ".ldb"); + dbpath.delete(); + try(Timer.Context ctx = dbCreation.time()) { + db = new LevelDB(dbpath); + } + } + + @After + public void cleanup() throws Exception { + if (db != null) { + try(Timer.Context ctx = dbClose.time()) { + db.close(); + } + } + if (dbpath != null) { + FileUtils.deleteQuietly(dbpath); + } + } + + @AfterClass + public static void report() { + if (metrics.getTimers().isEmpty()) { + return; + } + + int headingPrefix = 0; + for (Map.Entry e : metrics.getTimers().entrySet()) { + headingPrefix = Math.max(e.getKey().length(), headingPrefix); + } + headingPrefix += 4; + + StringBuilder heading = new StringBuilder(); + for (int i = 0; i < headingPrefix; i++) { + heading.append(" "); + } + heading.append("\tcount"); + heading.append("\tmean"); + heading.append("\tmin"); + heading.append("\tmax"); + heading.append("\t95th"); + System.out.println(heading); + + for (Map.Entry e : metrics.getTimers().entrySet()) { + StringBuilder row = new StringBuilder(); + row.append(e.getKey()); + for (int i = 0; i < headingPrefix - e.getKey().length(); i++) { + row.append(" "); + } + + Snapshot s = e.getValue().getSnapshot(); + row.append("\t").append(e.getValue().getCount()); + row.append("\t").append(toMs(s.getMean())); + row.append("\t").append(toMs(s.getMin())); + row.append("\t").append(toMs(s.getMax())); + row.append("\t").append(toMs(s.get95thPercentile())); + + System.out.println(row); + } + + Slf4jReporter.forRegistry(metrics).outputTo(LoggerFactory.getLogger(LevelDBBenchmark.class)) + .build().report(); + } + + private static String toMs(double nanos) { + return String.format("%.3f", nanos / 1000 / 1000); + } + + @Test + public void sequentialWritesNoIndex() throws Exception { + List entries = createSimpleType(); + writeAll(entries, "sequentialWritesNoIndex"); + writeAll(entries, "sequentialUpdatesNoIndex"); + deleteNoIndex(entries, "sequentialDeleteNoIndex"); + } + + @Test + public void randomWritesNoIndex() throws Exception { + List entries = createSimpleType(); + + Collections.shuffle(entries); + writeAll(entries, "randomWritesNoIndex"); + + Collections.shuffle(entries); + writeAll(entries, "randomUpdatesNoIndex"); + + Collections.shuffle(entries); + deleteNoIndex(entries, "randomDeletesNoIndex"); + } + + @Test + public void sequentialWritesIndexedType() throws Exception { + List entries = createIndexedType(); + writeAll(entries, "sequentialWritesIndexed"); + writeAll(entries, "sequentialUpdatesIndexed"); + deleteIndexed(entries, "sequentialDeleteIndexed"); + } + + @Test + public void randomWritesIndexedTypeAndIteration() throws Exception { + List entries = createIndexedType(); + + Collections.shuffle(entries); + writeAll(entries, "randomWritesIndexed"); + + Collections.shuffle(entries); + writeAll(entries, "randomUpdatesIndexed"); + + // Run iteration benchmarks here since we've gone through the trouble of writing all + // the data already. + KVStoreView view = db.view(IndexedType.class); + iterate(view, "naturalIndex"); + iterate(view.reverse(), "naturalIndexDescending"); + iterate(view.index("name"), "refIndex"); + iterate(view.index("name").reverse(), "refIndexDescending"); + + Collections.shuffle(entries); + deleteIndexed(entries, "randomDeleteIndexed"); + } + + private void iterate(KVStoreView view, String name) throws Exception { + Timer create = metrics.timer(name + "CreateIterator"); + Timer iter = metrics.timer(name + "Iteration"); + KVStoreIterator it = null; + { + // Create the iterator several times, just to have multiple data points. + for (int i = 0; i < 1024; i++) { + if (it != null) { + it.close(); + } + try(Timer.Context ctx = create.time()) { + it = view.closeableIterator(); + } + } + } + + for (; it.hasNext(); ) { + try(Timer.Context ctx = iter.time()) { + it.next(); + } + } + } + + private void writeAll(List entries, String timerName) throws Exception { + Timer timer = newTimer(timerName); + for (Object o : entries) { + try(Timer.Context ctx = timer.time()) { + db.write(o); + } + } + } + + private void deleteNoIndex(List entries, String timerName) throws Exception { + Timer delete = newTimer(timerName); + for (SimpleType i : entries) { + try(Timer.Context ctx = delete.time()) { + db.delete(i.getClass(), i.key); + } + } + } + + private void deleteIndexed(List entries, String timerName) throws Exception { + Timer delete = newTimer(timerName); + for (IndexedType i : entries) { + try(Timer.Context ctx = delete.time()) { + db.delete(i.getClass(), i.key); + } + } + } + + private List createSimpleType() { + List entries = new ArrayList<>(); + for (int i = 0; i < COUNT; i++) { + SimpleType t = new SimpleType(); + t.key = IDGEN.getAndIncrement(); + t.name = "name" + (t.key % 1024); + entries.add(t); + } + return entries; + } + + private List createIndexedType() { + List entries = new ArrayList<>(); + for (int i = 0; i < COUNT; i++) { + IndexedType t = new IndexedType(); + t.key = IDGEN.getAndIncrement(); + t.name = "name" + (t.key % 1024); + entries.add(t); + } + return entries; + } + + private Timer newTimer(String name) { + assertNull("Timer already exists: " + name, metrics.getTimers().get(name)); + return metrics.timer(name); + } + + public static class SimpleType { + + @KVIndex + public int key; + + public String name; + + } + + public static class IndexedType { + + @KVIndex + public int key; + + @KVIndex("name") + public String name; + + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBIteratorSuite.java new file mode 100644 index 000000000000..f8195da58cf9 --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBIteratorSuite.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import java.io.File; + +import org.apache.commons.io.FileUtils; +import org.junit.AfterClass; + +public class LevelDBIteratorSuite extends DBIteratorSuite { + + private static File dbpath; + private static LevelDB db; + + @AfterClass + public static void cleanup() throws Exception { + if (db != null) { + db.close(); + } + if (dbpath != null) { + FileUtils.deleteQuietly(dbpath); + } + } + + @Override + protected KVStore createStore() throws Exception { + dbpath = File.createTempFile("test.", ".ldb"); + dbpath.delete(); + db = new LevelDB(dbpath); + return db; + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java new file mode 100644 index 000000000000..2b07d249d202 --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java @@ -0,0 +1,286 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import java.io.File; +import java.util.Arrays; +import java.util.List; +import java.util.NoSuchElementException; + +import org.apache.commons.io.FileUtils; +import org.iq80.leveldb.DBIterator; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import static org.junit.Assert.*; + +public class LevelDBSuite { + + private LevelDB db; + private File dbpath; + + @After + public void cleanup() throws Exception { + if (db != null) { + db.close(); + } + if (dbpath != null) { + FileUtils.deleteQuietly(dbpath); + } + } + + @Before + public void setup() throws Exception { + dbpath = File.createTempFile("test.", ".ldb"); + dbpath.delete(); + db = new LevelDB(dbpath); + } + + @Test + public void testReopenAndVersionCheckDb() throws Exception { + db.close(); + db = null; + assertTrue(dbpath.exists()); + + db = new LevelDB(dbpath); + assertEquals(LevelDB.STORE_VERSION, + db.serializer.deserializeLong(db.db().get(LevelDB.STORE_VERSION_KEY))); + db.db().put(LevelDB.STORE_VERSION_KEY, db.serializer.serialize(LevelDB.STORE_VERSION + 1)); + db.close(); + db = null; + + try { + db = new LevelDB(dbpath); + fail("Should have failed version check."); + } catch (UnsupportedStoreVersionException e) { + // Expected. + } + } + + @Test + public void testObjectWriteReadDelete() throws Exception { + CustomType1 t = new CustomType1(); + t.key = "key"; + t.id = "id"; + t.name = "name"; + t.child = "child"; + + try { + db.read(CustomType1.class, t.key); + fail("Expected exception for non-existant object."); + } catch (NoSuchElementException nsee) { + // Expected. + } + + db.write(t); + assertEquals(t, db.read(t.getClass(), t.key)); + assertEquals(1L, db.count(t.getClass())); + + db.delete(t.getClass(), t.key); + try { + db.read(t.getClass(), t.key); + fail("Expected exception for deleted object."); + } catch (NoSuchElementException nsee) { + // Expected. + } + + // Look into the actual DB and make sure that all the keys related to the type have been + // removed. + assertEquals(0, countKeys(t.getClass())); + } + + @Test + public void testMultipleObjectWriteReadDelete() throws Exception { + CustomType1 t1 = new CustomType1(); + t1.key = "key1"; + t1.id = "id"; + t1.name = "name1"; + t1.child = "child1"; + + CustomType1 t2 = new CustomType1(); + t2.key = "key2"; + t2.id = "id"; + t2.name = "name2"; + t2.child = "child2"; + + db.write(t1); + db.write(t2); + + assertEquals(t1, db.read(t1.getClass(), t1.key)); + assertEquals(t2, db.read(t2.getClass(), t2.key)); + assertEquals(2L, db.count(t1.getClass())); + + // There should be one "id" index entry with two values. + assertEquals(2, db.count(t1.getClass(), "id", t1.id)); + + // Delete the first entry; now there should be 3 remaining keys, since one of the "name" + // index entries should have been removed. + db.delete(t1.getClass(), t1.key); + + // Make sure there's a single entry in the "id" index now. + assertEquals(1, db.count(t2.getClass(), "id", t2.id)); + + // Delete the remaining entry, make sure all data is gone. + db.delete(t2.getClass(), t2.key); + assertEquals(0, countKeys(t2.getClass())); + } + + @Test + public void testMultipleTypesWriteReadDelete() throws Exception { + CustomType1 t1 = new CustomType1(); + t1.key = "1"; + t1.id = "id"; + t1.name = "name1"; + t1.child = "child1"; + + IntKeyType t2 = new IntKeyType(); + t2.key = 2; + t2.id = "2"; + t2.values = Arrays.asList("value1", "value2"); + + ArrayKeyIndexType t3 = new ArrayKeyIndexType(); + t3.key = new int[] { 42, 84 }; + t3.id = new String[] { "id1", "id2" }; + + db.write(t1); + db.write(t2); + db.write(t3); + + assertEquals(t1, db.read(t1.getClass(), t1.key)); + assertEquals(t2, db.read(t2.getClass(), t2.key)); + assertEquals(t3, db.read(t3.getClass(), t3.key)); + + // There should be one "id" index with a single entry for each type. + assertEquals(1, db.count(t1.getClass(), "id", t1.id)); + assertEquals(1, db.count(t2.getClass(), "id", t2.id)); + assertEquals(1, db.count(t3.getClass(), "id", t3.id)); + + // Delete the first entry; this should not affect the entries for the second type. + db.delete(t1.getClass(), t1.key); + assertEquals(0, countKeys(t1.getClass())); + assertEquals(1, db.count(t2.getClass(), "id", t2.id)); + assertEquals(1, db.count(t3.getClass(), "id", t3.id)); + + // Delete the remaining entries, make sure all data is gone. + db.delete(t2.getClass(), t2.key); + assertEquals(0, countKeys(t2.getClass())); + + db.delete(t3.getClass(), t3.key); + assertEquals(0, countKeys(t3.getClass())); + } + + @Test + public void testMetadata() throws Exception { + assertNull(db.getMetadata(CustomType1.class)); + + CustomType1 t = new CustomType1(); + t.id = "id"; + t.name = "name"; + t.child = "child"; + + db.setMetadata(t); + assertEquals(t, db.getMetadata(CustomType1.class)); + + db.setMetadata(null); + assertNull(db.getMetadata(CustomType1.class)); + } + + @Test + public void testUpdate() throws Exception { + CustomType1 t = new CustomType1(); + t.key = "key"; + t.id = "id"; + t.name = "name"; + t.child = "child"; + + db.write(t); + + t.name = "anotherName"; + + db.write(t); + + assertEquals(1, db.count(t.getClass())); + assertEquals(1, db.count(t.getClass(), "name", "anotherName")); + assertEquals(0, db.count(t.getClass(), "name", "name")); + } + + @Test + public void testSkip() throws Exception { + for (int i = 0; i < 10; i++) { + CustomType1 t = new CustomType1(); + t.key = "key" + i; + t.id = "id" + i; + t.name = "name" + i; + t.child = "child" + i; + + db.write(t); + } + + KVStoreIterator it = db.view(CustomType1.class).closeableIterator(); + assertTrue(it.hasNext()); + assertTrue(it.skip(5)); + assertEquals("key5", it.next().key); + assertTrue(it.skip(3)); + assertEquals("key9", it.next().key); + assertFalse(it.hasNext()); + } + + private int countKeys(Class type) throws Exception { + byte[] prefix = db.getTypeInfo(type).keyPrefix(); + int count = 0; + + DBIterator it = db.db().iterator(); + it.seek(prefix); + + while (it.hasNext()) { + byte[] key = it.next().getKey(); + if (LevelDBIterator.startsWith(key, prefix)) { + count++; + } + } + + return count; + } + + public static class IntKeyType { + + @KVIndex + public int key; + + @KVIndex("id") + public String id; + + public List values; + + @Override + public boolean equals(Object o) { + if (o instanceof IntKeyType) { + IntKeyType other = (IntKeyType) o; + return key == other.key && id.equals(other.id) && values.equals(other.values); + } + return false; + } + + @Override + public int hashCode() { + return id.hashCode(); + } + + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBTypeInfoSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBTypeInfoSuite.java new file mode 100644 index 000000000000..38db3bedaef6 --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBTypeInfoSuite.java @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.kvstore; + +import static java.nio.charset.StandardCharsets.UTF_8; + +import org.junit.Test; +import static org.junit.Assert.*; + +public class LevelDBTypeInfoSuite { + + @Test + public void testIndexAnnotation() throws Exception { + KVTypeInfo ti = new KVTypeInfo(CustomType1.class); + assertEquals(5, ti.indices().count()); + + CustomType1 t1 = new CustomType1(); + t1.key = "key"; + t1.id = "id"; + t1.name = "name"; + t1.num = 42; + t1.child = "child"; + + assertEquals(t1.key, ti.getIndexValue(KVIndex.NATURAL_INDEX_NAME, t1)); + assertEquals(t1.id, ti.getIndexValue("id", t1)); + assertEquals(t1.name, ti.getIndexValue("name", t1)); + assertEquals(t1.num, ti.getIndexValue("int", t1)); + assertEquals(t1.child, ti.getIndexValue("child", t1)); + } + + @Test(expected = IllegalArgumentException.class) + public void testNoNaturalIndex() throws Exception { + newTypeInfo(NoNaturalIndex.class); + } + + @Test(expected = IllegalArgumentException.class) + public void testNoNaturalIndex2() throws Exception { + newTypeInfo(NoNaturalIndex2.class); + } + + @Test(expected = IllegalArgumentException.class) + public void testDuplicateIndex() throws Exception { + newTypeInfo(DuplicateIndex.class); + } + + @Test(expected = IllegalArgumentException.class) + public void testEmptyIndexName() throws Exception { + newTypeInfo(EmptyIndexName.class); + } + + @Test(expected = IllegalArgumentException.class) + public void testIllegalIndexName() throws Exception { + newTypeInfo(IllegalIndexName.class); + } + + @Test(expected = IllegalArgumentException.class) + public void testIllegalIndexMethod() throws Exception { + newTypeInfo(IllegalIndexMethod.class); + } + + @Test + public void testKeyClashes() throws Exception { + LevelDBTypeInfo ti = newTypeInfo(CustomType1.class); + + CustomType1 t1 = new CustomType1(); + t1.key = "key1"; + t1.name = "a"; + + CustomType1 t2 = new CustomType1(); + t2.key = "key2"; + t2.name = "aa"; + + CustomType1 t3 = new CustomType1(); + t3.key = "key3"; + t3.name = "aaa"; + + // Make sure entries with conflicting names are sorted correctly. + assertBefore(ti.index("name").entityKey(null, t1), ti.index("name").entityKey(null, t2)); + assertBefore(ti.index("name").entityKey(null, t1), ti.index("name").entityKey(null, t3)); + assertBefore(ti.index("name").entityKey(null, t2), ti.index("name").entityKey(null, t3)); + } + + @Test + public void testNumEncoding() throws Exception { + LevelDBTypeInfo.Index idx = newTypeInfo(CustomType1.class).indices().iterator().next(); + + assertEquals("+=00000001", new String(idx.toKey(1), UTF_8)); + assertEquals("+=00000010", new String(idx.toKey(16), UTF_8)); + assertEquals("+=7fffffff", new String(idx.toKey(Integer.MAX_VALUE), UTF_8)); + + assertBefore(idx.toKey(1), idx.toKey(2)); + assertBefore(idx.toKey(-1), idx.toKey(2)); + assertBefore(idx.toKey(-11), idx.toKey(2)); + assertBefore(idx.toKey(-11), idx.toKey(-1)); + assertBefore(idx.toKey(1), idx.toKey(11)); + assertBefore(idx.toKey(Integer.MIN_VALUE), idx.toKey(Integer.MAX_VALUE)); + + assertBefore(idx.toKey(1L), idx.toKey(2L)); + assertBefore(idx.toKey(-1L), idx.toKey(2L)); + assertBefore(idx.toKey(Long.MIN_VALUE), idx.toKey(Long.MAX_VALUE)); + + assertBefore(idx.toKey((short) 1), idx.toKey((short) 2)); + assertBefore(idx.toKey((short) -1), idx.toKey((short) 2)); + assertBefore(idx.toKey(Short.MIN_VALUE), idx.toKey(Short.MAX_VALUE)); + + assertBefore(idx.toKey((byte) 1), idx.toKey((byte) 2)); + assertBefore(idx.toKey((byte) -1), idx.toKey((byte) 2)); + assertBefore(idx.toKey(Byte.MIN_VALUE), idx.toKey(Byte.MAX_VALUE)); + + byte prefix = LevelDBTypeInfo.ENTRY_PREFIX; + assertSame(new byte[] { prefix, LevelDBTypeInfo.FALSE }, idx.toKey(false)); + assertSame(new byte[] { prefix, LevelDBTypeInfo.TRUE }, idx.toKey(true)); + } + + @Test + public void testArrayIndices() throws Exception { + LevelDBTypeInfo.Index idx = newTypeInfo(CustomType1.class).indices().iterator().next(); + + assertBefore(idx.toKey(new String[] { "str1" }), idx.toKey(new String[] { "str2" })); + assertBefore(idx.toKey(new String[] { "str1", "str2" }), + idx.toKey(new String[] { "str1", "str3" })); + + assertBefore(idx.toKey(new int[] { 1 }), idx.toKey(new int[] { 2 })); + assertBefore(idx.toKey(new int[] { 1, 2 }), idx.toKey(new int[] { 1, 3 })); + } + + private LevelDBTypeInfo newTypeInfo(Class type) throws Exception { + return new LevelDBTypeInfo(null, type, type.getName().getBytes(UTF_8)); + } + + private void assertBefore(byte[] key1, byte[] key2) { + assertBefore(new String(key1, UTF_8), new String(key2, UTF_8)); + } + + private void assertBefore(String str1, String str2) { + assertTrue(String.format("%s < %s failed", str1, str2), str1.compareTo(str2) < 0); + } + + private void assertSame(byte[] key1, byte[] key2) { + assertEquals(new String(key1, UTF_8), new String(key2, UTF_8)); + } + + public static class NoNaturalIndex { + + public String id; + + } + + public static class NoNaturalIndex2 { + + @KVIndex("id") + public String id; + + } + + public static class DuplicateIndex { + + @KVIndex + public String key; + + @KVIndex("id") + public String id; + + @KVIndex("id") + public String id2; + + } + + public static class EmptyIndexName { + + @KVIndex("") + public String id; + + } + + public static class IllegalIndexName { + + @KVIndex("__invalid") + public String id; + + } + + public static class IllegalIndexMethod { + + @KVIndex("id") + public String id(boolean illegalParam) { + return null; + } + + } + +} diff --git a/network/common/src/test/resources/log4j.properties b/common/kvstore/src/test/resources/log4j.properties similarity index 100% rename from network/common/src/test/resources/log4j.properties rename to common/kvstore/src/test/resources/log4j.properties diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml new file mode 100644 index 000000000000..18cbdadd224a --- /dev/null +++ b/common/network-common/pom.xml @@ -0,0 +1,144 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-network-common_2.11 + jar + Spark Project Networking + http://spark.apache.org/ + + network-common + + + + + + io.netty + netty-all + + + org.apache.commons + commons-lang3 + + + + org.fusesource.leveldbjni + leveldbjni-all + 1.8 + + + + com.fasterxml.jackson.core + jackson-databind + + + + com.fasterxml.jackson.core + jackson-annotations + + + + io.dropwizard.metrics + metrics-core + + + + + org.slf4j + slf4j-api + provided + + + com.google.code.findbugs + jsr305 + + + com.google.guava + guava + compile + + + org.apache.commons + commons-crypto + + + + + log4j + log4j + test + + + org.apache.spark + spark-tags_${scala.binary.version} + test + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + org.mockito + mockito-core + test + + + org.slf4j + slf4j-log4j12 + test + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + + org.apache.maven.plugins + maven-jar-plugin + + + prepare-test-jar + test-compile + + test-jar + + + + + + + diff --git a/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java b/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java new file mode 100644 index 000000000000..ae91bc9cfdd0 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import java.util.ArrayList; +import java.util.List; + +import io.netty.channel.Channel; +import io.netty.channel.socket.SocketChannel; +import io.netty.handler.timeout.IdleStateHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientBootstrap; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.client.TransportResponseHandler; +import org.apache.spark.network.protocol.MessageDecoder; +import org.apache.spark.network.protocol.MessageEncoder; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportChannelHandler; +import org.apache.spark.network.server.TransportRequestHandler; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.TransportServerBootstrap; +import org.apache.spark.network.util.NettyUtils; +import org.apache.spark.network.util.TransportConf; +import org.apache.spark.network.util.TransportFrameDecoder; + +/** + * Contains the context to create a {@link TransportServer}, {@link TransportClientFactory}, and to + * setup Netty Channel pipelines with a + * {@link org.apache.spark.network.server.TransportChannelHandler}. + * + * There are two communication protocols that the TransportClient provides, control-plane RPCs and + * data-plane "chunk fetching". The handling of the RPCs is performed outside of the scope of the + * TransportContext (i.e., by a user-provided handler), and it is responsible for setting up streams + * which can be streamed through the data plane in chunks using zero-copy IO. + * + * The TransportServer and TransportClientFactory both create a TransportChannelHandler for each + * channel. As each TransportChannelHandler contains a TransportClient, this enables server + * processes to send messages back to the client on an existing channel. + */ +public class TransportContext { + private static final Logger logger = LoggerFactory.getLogger(TransportContext.class); + + private final TransportConf conf; + private final RpcHandler rpcHandler; + private final boolean closeIdleConnections; + + /** + * Force to create MessageEncoder and MessageDecoder so that we can make sure they will be created + * before switching the current context class loader to ExecutorClassLoader. + * + * Netty's MessageToMessageEncoder uses Javassist to generate a matcher class and the + * implementation calls "Class.forName" to check if this calls is already generated. If the + * following two objects are created in "ExecutorClassLoader.findClass", it will cause + * "ClassCircularityError". This is because loading this Netty generated class will call + * "ExecutorClassLoader.findClass" to search this class, and "ExecutorClassLoader" will try to use + * RPC to load it and cause to load the non-exist matcher class again. JVM will report + * `ClassCircularityError` to prevent such infinite recursion. (See SPARK-17714) + */ + private static final MessageEncoder ENCODER = MessageEncoder.INSTANCE; + private static final MessageDecoder DECODER = MessageDecoder.INSTANCE; + + public TransportContext(TransportConf conf, RpcHandler rpcHandler) { + this(conf, rpcHandler, false); + } + + public TransportContext( + TransportConf conf, + RpcHandler rpcHandler, + boolean closeIdleConnections) { + this.conf = conf; + this.rpcHandler = rpcHandler; + this.closeIdleConnections = closeIdleConnections; + } + + /** + * Initializes a ClientFactory which runs the given TransportClientBootstraps prior to returning + * a new Client. Bootstraps will be executed synchronously, and must run successfully in order + * to create a Client. + */ + public TransportClientFactory createClientFactory(List bootstraps) { + return new TransportClientFactory(this, bootstraps); + } + + public TransportClientFactory createClientFactory() { + return createClientFactory(new ArrayList<>()); + } + + /** Create a server which will attempt to bind to a specific port. */ + public TransportServer createServer(int port, List bootstraps) { + return new TransportServer(this, null, port, rpcHandler, bootstraps); + } + + /** Create a server which will attempt to bind to a specific host and port. */ + public TransportServer createServer( + String host, int port, List bootstraps) { + return new TransportServer(this, host, port, rpcHandler, bootstraps); + } + + /** Creates a new server, binding to any available ephemeral port. */ + public TransportServer createServer(List bootstraps) { + return createServer(0, bootstraps); + } + + public TransportServer createServer() { + return createServer(0, new ArrayList<>()); + } + + public TransportChannelHandler initializePipeline(SocketChannel channel) { + return initializePipeline(channel, rpcHandler); + } + + /** + * Initializes a client or server Netty Channel Pipeline which encodes/decodes messages and + * has a {@link org.apache.spark.network.server.TransportChannelHandler} to handle request or + * response messages. + * + * @param channel The channel to initialize. + * @param channelRpcHandler The RPC handler to use for the channel. + * + * @return Returns the created TransportChannelHandler, which includes a TransportClient that can + * be used to communicate on this channel. The TransportClient is directly associated with a + * ChannelHandler to ensure all users of the same channel get the same TransportClient object. + */ + public TransportChannelHandler initializePipeline( + SocketChannel channel, + RpcHandler channelRpcHandler) { + try { + TransportChannelHandler channelHandler = createChannelHandler(channel, channelRpcHandler); + channel.pipeline() + .addLast("encoder", ENCODER) + .addLast(TransportFrameDecoder.HANDLER_NAME, NettyUtils.createFrameDecoder()) + .addLast("decoder", DECODER) + .addLast("idleStateHandler", new IdleStateHandler(0, 0, conf.connectionTimeoutMs() / 1000)) + // NOTE: Chunks are currently guaranteed to be returned in the order of request, but this + // would require more logic to guarantee if this were not part of the same event loop. + .addLast("handler", channelHandler); + return channelHandler; + } catch (RuntimeException e) { + logger.error("Error while initializing Netty pipeline", e); + throw e; + } + } + + /** + * Creates the server- and client-side handler which is used to handle both RequestMessages and + * ResponseMessages. The channel is expected to have been successfully created, though certain + * properties (such as the remoteAddress()) may not be available yet. + */ + private TransportChannelHandler createChannelHandler(Channel channel, RpcHandler rpcHandler) { + TransportResponseHandler responseHandler = new TransportResponseHandler(channel); + TransportClient client = new TransportClient(channel, responseHandler); + TransportRequestHandler requestHandler = new TransportRequestHandler(channel, client, + rpcHandler, conf.maxChunksBeingTransferred()); + return new TransportChannelHandler(client, responseHandler, requestHandler, + conf.connectionTimeoutMs(), closeIdleConnections); + } + + public TransportConf getConf() { return conf; } +} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java similarity index 93% rename from network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java rename to common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java index 844eff4f4c70..ea9b3ce4e352 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java @@ -18,12 +18,13 @@ package org.apache.spark.network.buffer; import java.io.File; -import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.io.RandomAccessFile; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.StandardOpenOption; import com.google.common.base.Objects; import com.google.common.io.ByteStreams; @@ -93,9 +94,9 @@ public ByteBuffer nioByteBuffer() throws IOException { @Override public InputStream createInputStream() throws IOException { - FileInputStream is = null; + InputStream is = null; try { - is = new FileInputStream(file); + is = Files.newInputStream(file.toPath()); ByteStreams.skipFully(is, offset); return new LimitedInputStream(is, length); } catch (IOException e) { @@ -130,9 +131,9 @@ public ManagedBuffer release() { @Override public Object convertToNetty() throws IOException { if (conf.lazyFileDescriptor()) { - return new LazyFileRegion(file, offset, length); + return new DefaultFileRegion(file, offset, length); } else { - FileChannel fileChannel = new FileInputStream(file).getChannel(); + FileChannel fileChannel = FileChannel.open(file.toPath(), StandardOpenOption.READ); return new DefaultFileRegion(fileChannel, offset, length); } } diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java similarity index 90% rename from network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java rename to common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java index a415db593a78..1861f8d7fd8f 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java @@ -65,7 +65,11 @@ public abstract class ManagedBuffer { public abstract ManagedBuffer release(); /** - * Convert the buffer into an Netty object, used to write the data out. + * Convert the buffer into an Netty object, used to write the data out. The return value is either + * a {@link io.netty.buffer.ByteBuf} or a {@link io.netty.channel.FileRegion}. + * + * If this method returns a ByteBuf, then that buffer's reference count will be incremented and + * the caller will be responsible for releasing this new reference. */ public abstract Object convertToNetty() throws IOException; } diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java similarity index 95% rename from network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java rename to common/network-common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java index c806bfa45bef..acc49d968c18 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java @@ -28,7 +28,7 @@ /** * A {@link ManagedBuffer} backed by a Netty {@link ByteBuf}. */ -public final class NettyManagedBuffer extends ManagedBuffer { +public class NettyManagedBuffer extends ManagedBuffer { private final ByteBuf buf; public NettyManagedBuffer(ByteBuf buf) { @@ -64,7 +64,7 @@ public ManagedBuffer release() { @Override public Object convertToNetty() throws IOException { - return buf.duplicate(); + return buf.duplicate().retain(); } @Override diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java similarity index 96% rename from network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java rename to common/network-common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java index f55b884bc45c..631d76771525 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java @@ -28,7 +28,7 @@ /** * A {@link ManagedBuffer} backed by {@link ByteBuffer}. */ -public final class NioManagedBuffer extends ManagedBuffer { +public class NioManagedBuffer extends ManagedBuffer { private final ByteBuffer buf; public NioManagedBuffer(ByteBuffer buf) { diff --git a/network/common/src/main/java/org/apache/spark/network/client/ChunkFetchFailureException.java b/common/network-common/src/main/java/org/apache/spark/network/client/ChunkFetchFailureException.java similarity index 100% rename from network/common/src/main/java/org/apache/spark/network/client/ChunkFetchFailureException.java rename to common/network-common/src/main/java/org/apache/spark/network/client/ChunkFetchFailureException.java diff --git a/network/common/src/main/java/org/apache/spark/network/client/ChunkReceivedCallback.java b/common/network-common/src/main/java/org/apache/spark/network/client/ChunkReceivedCallback.java similarity index 100% rename from network/common/src/main/java/org/apache/spark/network/client/ChunkReceivedCallback.java rename to common/network-common/src/main/java/org/apache/spark/network/client/ChunkReceivedCallback.java diff --git a/network/common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java b/common/network-common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java similarity index 77% rename from network/common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java rename to common/network-common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java index 6ec960d79542..6afc63f71bb3 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java @@ -17,13 +17,20 @@ package org.apache.spark.network.client; +import java.nio.ByteBuffer; + /** * Callback for the result of a single RPC. This will be invoked once with either success or * failure. */ public interface RpcResponseCallback { - /** Successful serialized result from server. */ - void onSuccess(byte[] response); + /** + * Successful serialized result from server. + * + * After `onSuccess` returns, `response` will be recycled and its content will become invalid. + * Please copy the content of `response` if you want to use it after `onSuccess` returns. + */ + void onSuccess(ByteBuffer response); /** Exception either propagated from server or raised on client side. */ void onFailure(Throwable e); diff --git a/network/common/src/main/java/org/apache/spark/network/client/StreamCallback.java b/common/network-common/src/main/java/org/apache/spark/network/client/StreamCallback.java similarity index 91% rename from network/common/src/main/java/org/apache/spark/network/client/StreamCallback.java rename to common/network-common/src/main/java/org/apache/spark/network/client/StreamCallback.java index 093fada320cc..d322aec28793 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/StreamCallback.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/StreamCallback.java @@ -21,9 +21,9 @@ import java.nio.ByteBuffer; /** - * Callback for streaming data. Stream data will be offered to the {@link onData(ByteBuffer)} - * method as it arrives. Once all the stream data is received, {@link onComplete()} will be - * called. + * Callback for streaming data. Stream data will be offered to the + * {@link #onData(String, ByteBuffer)} method as it arrives. Once all the stream data is received, + * {@link #onComplete(String)} will be called. *

* The network library guarantees that a single thread will call these methods at a time, but * different call may be made by different threads. diff --git a/network/common/src/main/java/org/apache/spark/network/client/StreamInterceptor.java b/common/network-common/src/main/java/org/apache/spark/network/client/StreamInterceptor.java similarity index 86% rename from network/common/src/main/java/org/apache/spark/network/client/StreamInterceptor.java rename to common/network-common/src/main/java/org/apache/spark/network/client/StreamInterceptor.java index 02230a00e69f..b0e85bae7c30 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/StreamInterceptor.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/StreamInterceptor.java @@ -30,13 +30,18 @@ */ class StreamInterceptor implements TransportFrameDecoder.Interceptor { + private final TransportResponseHandler handler; private final String streamId; private final long byteCount; private final StreamCallback callback; + private long bytesRead; - private volatile long bytesRead; - - StreamInterceptor(String streamId, long byteCount, StreamCallback callback) { + StreamInterceptor( + TransportResponseHandler handler, + String streamId, + long byteCount, + StreamCallback callback) { + this.handler = handler; this.streamId = streamId; this.byteCount = byteCount; this.callback = callback; @@ -45,11 +50,13 @@ class StreamInterceptor implements TransportFrameDecoder.Interceptor { @Override public void exceptionCaught(Throwable cause) throws Exception { + handler.deactivateStream(); callback.onFailure(streamId, cause); } @Override public void channelInactive() throws Exception { + handler.deactivateStream(); callback.onFailure(streamId, new ClosedChannelException()); } @@ -65,8 +72,10 @@ public boolean handle(ByteBuf buf) throws Exception { RuntimeException re = new IllegalStateException(String.format( "Read too many bytes? Expected %d, but read %d.", byteCount, bytesRead)); callback.onFailure(streamId, re); + handler.deactivateStream(); throw re; } else if (bytesRead == byteCount) { + handler.deactivateStream(); callback.onComplete(streamId); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java new file mode 100644 index 000000000000..8f354ad78bba --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -0,0 +1,322 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.client; + +import java.io.Closeable; +import java.io.IOException; +import java.net.SocketAddress; +import java.nio.ByteBuffer; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.util.concurrent.SettableFuture; +import io.netty.channel.Channel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.protocol.ChunkFetchRequest; +import org.apache.spark.network.protocol.OneWayMessage; +import org.apache.spark.network.protocol.RpcRequest; +import org.apache.spark.network.protocol.StreamChunkId; +import org.apache.spark.network.protocol.StreamRequest; +import static org.apache.spark.network.util.NettyUtils.getRemoteAddress; + +/** + * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow + * efficient transfer of a large amount of data, broken up into chunks with size ranging from + * hundreds of KB to a few MB. + * + * Note that while this client deals with the fetching of chunks from a stream (i.e., data plane), + * the actual setup of the streams is done outside the scope of the transport layer. The convenience + * method "sendRPC" is provided to enable control plane communication between the client and server + * to perform this setup. + * + * For example, a typical workflow might be: + * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100 + * client.fetchChunk(streamId = 100, chunkIndex = 0, callback) + * client.fetchChunk(streamId = 100, chunkIndex = 1, callback) + * ... + * client.sendRPC(new CloseStream(100)) + * + * Construct an instance of TransportClient using {@link TransportClientFactory}. A single + * TransportClient may be used for multiple streams, but any given stream must be restricted to a + * single client, in order to avoid out-of-order responses. + * + * NB: This class is used to make requests to the server, while {@link TransportResponseHandler} is + * responsible for handling responses from the server. + * + * Concurrency: thread safe and can be called from multiple threads. + */ +public class TransportClient implements Closeable { + private static final Logger logger = LoggerFactory.getLogger(TransportClient.class); + + private final Channel channel; + private final TransportResponseHandler handler; + @Nullable private String clientId; + private volatile boolean timedOut; + + public TransportClient(Channel channel, TransportResponseHandler handler) { + this.channel = Preconditions.checkNotNull(channel); + this.handler = Preconditions.checkNotNull(handler); + this.timedOut = false; + } + + public Channel getChannel() { + return channel; + } + + public boolean isActive() { + return !timedOut && (channel.isOpen() || channel.isActive()); + } + + public SocketAddress getSocketAddress() { + return channel.remoteAddress(); + } + + /** + * Returns the ID used by the client to authenticate itself when authentication is enabled. + * + * @return The client ID, or null if authentication is disabled. + */ + public String getClientId() { + return clientId; + } + + /** + * Sets the authenticated client ID. This is meant to be used by the authentication layer. + * + * Trying to set a different client ID after it's been set will result in an exception. + */ + public void setClientId(String id) { + Preconditions.checkState(clientId == null, "Client ID has already been set."); + this.clientId = id; + } + + /** + * Requests a single chunk from the remote side, from the pre-negotiated streamId. + * + * Chunk indices go from 0 onwards. It is valid to request the same chunk multiple times, though + * some streams may not support this. + * + * Multiple fetchChunk requests may be outstanding simultaneously, and the chunks are guaranteed + * to be returned in the same order that they were requested, assuming only a single + * TransportClient is used to fetch the chunks. + * + * @param streamId Identifier that refers to a stream in the remote StreamManager. This should + * be agreed upon by client and server beforehand. + * @param chunkIndex 0-based index of the chunk to fetch + * @param callback Callback invoked upon successful receipt of chunk, or upon any failure. + */ + public void fetchChunk( + long streamId, + int chunkIndex, + ChunkReceivedCallback callback) { + long startTime = System.currentTimeMillis(); + if (logger.isDebugEnabled()) { + logger.debug("Sending fetch chunk request {} to {}", chunkIndex, getRemoteAddress(channel)); + } + + StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex); + handler.addFetchRequest(streamChunkId, callback); + + channel.writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener(future -> { + if (future.isSuccess()) { + long timeTaken = System.currentTimeMillis() - startTime; + if (logger.isTraceEnabled()) { + logger.trace("Sending request {} to {} took {} ms", streamChunkId, + getRemoteAddress(channel), timeTaken); + } + } else { + String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId, + getRemoteAddress(channel), future.cause()); + logger.error(errorMsg, future.cause()); + handler.removeFetchRequest(streamChunkId); + channel.close(); + try { + callback.onFailure(chunkIndex, new IOException(errorMsg, future.cause())); + } catch (Exception e) { + logger.error("Uncaught exception in RPC response callback handler!", e); + } + } + }); + } + + /** + * Request to stream the data with the given stream ID from the remote end. + * + * @param streamId The stream to fetch. + * @param callback Object to call with the stream data. + */ + public void stream(String streamId, StreamCallback callback) { + long startTime = System.currentTimeMillis(); + if (logger.isDebugEnabled()) { + logger.debug("Sending stream request for {} to {}", streamId, getRemoteAddress(channel)); + } + + // Need to synchronize here so that the callback is added to the queue and the RPC is + // written to the socket atomically, so that callbacks are called in the right order + // when responses arrive. + synchronized (this) { + handler.addStreamCallback(streamId, callback); + channel.writeAndFlush(new StreamRequest(streamId)).addListener(future -> { + if (future.isSuccess()) { + long timeTaken = System.currentTimeMillis() - startTime; + if (logger.isTraceEnabled()) { + logger.trace("Sending request for {} to {} took {} ms", streamId, + getRemoteAddress(channel), timeTaken); + } + } else { + String errorMsg = String.format("Failed to send request for %s to %s: %s", streamId, + getRemoteAddress(channel), future.cause()); + logger.error(errorMsg, future.cause()); + channel.close(); + try { + callback.onFailure(streamId, new IOException(errorMsg, future.cause())); + } catch (Exception e) { + logger.error("Uncaught exception in RPC response callback handler!", e); + } + } + }); + } + } + + /** + * Sends an opaque message to the RpcHandler on the server-side. The callback will be invoked + * with the server's response or upon any failure. + * + * @param message The message to send. + * @param callback Callback to handle the RPC's reply. + * @return The RPC's id. + */ + public long sendRpc(ByteBuffer message, RpcResponseCallback callback) { + long startTime = System.currentTimeMillis(); + if (logger.isTraceEnabled()) { + logger.trace("Sending RPC to {}", getRemoteAddress(channel)); + } + + long requestId = Math.abs(UUID.randomUUID().getLeastSignificantBits()); + handler.addRpcRequest(requestId, callback); + + channel.writeAndFlush(new RpcRequest(requestId, new NioManagedBuffer(message))) + .addListener(future -> { + if (future.isSuccess()) { + long timeTaken = System.currentTimeMillis() - startTime; + if (logger.isTraceEnabled()) { + logger.trace("Sending request {} to {} took {} ms", requestId, + getRemoteAddress(channel), timeTaken); + } + } else { + String errorMsg = String.format("Failed to send RPC %s to %s: %s", requestId, + getRemoteAddress(channel), future.cause()); + logger.error(errorMsg, future.cause()); + handler.removeRpcRequest(requestId); + channel.close(); + try { + callback.onFailure(new IOException(errorMsg, future.cause())); + } catch (Exception e) { + logger.error("Uncaught exception in RPC response callback handler!", e); + } + } + }); + + return requestId; + } + + /** + * Synchronously sends an opaque message to the RpcHandler on the server-side, waiting for up to + * a specified timeout for a response. + */ + public ByteBuffer sendRpcSync(ByteBuffer message, long timeoutMs) { + final SettableFuture result = SettableFuture.create(); + + sendRpc(message, new RpcResponseCallback() { + @Override + public void onSuccess(ByteBuffer response) { + ByteBuffer copy = ByteBuffer.allocate(response.remaining()); + copy.put(response); + // flip "copy" to make it readable + copy.flip(); + result.set(copy); + } + + @Override + public void onFailure(Throwable e) { + result.setException(e); + } + }); + + try { + return result.get(timeoutMs, TimeUnit.MILLISECONDS); + } catch (ExecutionException e) { + throw Throwables.propagate(e.getCause()); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + + /** + * Sends an opaque message to the RpcHandler on the server-side. No reply is expected for the + * message, and no delivery guarantees are made. + * + * @param message The message to send. + */ + public void send(ByteBuffer message) { + channel.writeAndFlush(new OneWayMessage(new NioManagedBuffer(message))); + } + + /** + * Removes any state associated with the given RPC. + * + * @param requestId The RPC id returned by {@link #sendRpc(ByteBuffer, RpcResponseCallback)}. + */ + public void removeRpcRequest(long requestId) { + handler.removeRpcRequest(requestId); + } + + /** Mark this channel as having timed out. */ + public void timeOut() { + this.timedOut = true; + } + + @VisibleForTesting + public TransportResponseHandler getHandler() { + return handler; + } + + @Override + public void close() { + // close is a local operation and should finish with milliseconds; timeout just to be safe + channel.close().awaitUninterruptibly(10, TimeUnit.SECONDS); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("remoteAdress", channel.remoteAddress()) + .add("clientId", clientId) + .add("isActive", isActive()) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientBootstrap.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientBootstrap.java similarity index 100% rename from network/common/src/main/java/org/apache/spark/network/client/TransportClientBootstrap.java rename to common/network-common/src/main/java/org/apache/spark/network/client/TransportClientBootstrap.java diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java new file mode 100644 index 000000000000..16d242dbb2c4 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -0,0 +1,293 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.client; + +import java.io.Closeable; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.List; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; + +import com.codahale.metrics.MetricSet; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import io.netty.bootstrap.Bootstrap; +import io.netty.buffer.PooledByteBufAllocator; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.server.TransportChannelHandler; +import org.apache.spark.network.util.*; + +/** + * Factory for creating {@link TransportClient}s by using createClient. + * + * The factory maintains a connection pool to other hosts and should return the same + * TransportClient for the same remote host. It also shares a single worker thread pool for + * all TransportClients. + * + * TransportClients will be reused whenever possible. Prior to completing the creation of a new + * TransportClient, all given {@link TransportClientBootstrap}s will be run. + */ +public class TransportClientFactory implements Closeable { + + /** A simple data structure to track the pool of clients between two peer nodes. */ + private static class ClientPool { + TransportClient[] clients; + Object[] locks; + + ClientPool(int size) { + clients = new TransportClient[size]; + locks = new Object[size]; + for (int i = 0; i < size; i++) { + locks[i] = new Object(); + } + } + } + + private static final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class); + + private final TransportContext context; + private final TransportConf conf; + private final List clientBootstraps; + private final ConcurrentHashMap connectionPool; + + /** Random number generator for picking connections between peers. */ + private final Random rand; + private final int numConnectionsPerPeer; + + private final Class socketChannelClass; + private EventLoopGroup workerGroup; + private PooledByteBufAllocator pooledAllocator; + private final NettyMemoryMetrics metrics; + + public TransportClientFactory( + TransportContext context, + List clientBootstraps) { + this.context = Preconditions.checkNotNull(context); + this.conf = context.getConf(); + this.clientBootstraps = Lists.newArrayList(Preconditions.checkNotNull(clientBootstraps)); + this.connectionPool = new ConcurrentHashMap<>(); + this.numConnectionsPerPeer = conf.numConnectionsPerPeer(); + this.rand = new Random(); + + IOMode ioMode = IOMode.valueOf(conf.ioMode()); + this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode); + this.workerGroup = NettyUtils.createEventLoop( + ioMode, + conf.clientThreads(), + conf.getModuleName() + "-client"); + this.pooledAllocator = NettyUtils.createPooledByteBufAllocator( + conf.preferDirectBufs(), false /* allowCache */, conf.clientThreads()); + this.metrics = new NettyMemoryMetrics( + this.pooledAllocator, conf.getModuleName() + "-client", conf); + } + + public MetricSet getAllMetrics() { + return metrics; + } + + /** + * Create a {@link TransportClient} connecting to the given remote host / port. + * + * We maintains an array of clients (size determined by spark.shuffle.io.numConnectionsPerPeer) + * and randomly picks one to use. If no client was previously created in the randomly selected + * spot, this function creates a new client and places it there. + * + * Prior to the creation of a new TransportClient, we will execute all + * {@link TransportClientBootstrap}s that are registered with this factory. + * + * This blocks until a connection is successfully established and fully bootstrapped. + * + * Concurrency: This method is safe to call from multiple threads. + */ + public TransportClient createClient(String remoteHost, int remotePort) + throws IOException, InterruptedException { + // Get connection from the connection pool first. + // If it is not found or not active, create a new one. + // Use unresolved address here to avoid DNS resolution each time we creates a client. + final InetSocketAddress unresolvedAddress = + InetSocketAddress.createUnresolved(remoteHost, remotePort); + + // Create the ClientPool if we don't have it yet. + ClientPool clientPool = connectionPool.get(unresolvedAddress); + if (clientPool == null) { + connectionPool.putIfAbsent(unresolvedAddress, new ClientPool(numConnectionsPerPeer)); + clientPool = connectionPool.get(unresolvedAddress); + } + + int clientIndex = rand.nextInt(numConnectionsPerPeer); + TransportClient cachedClient = clientPool.clients[clientIndex]; + + if (cachedClient != null && cachedClient.isActive()) { + // Make sure that the channel will not timeout by updating the last use time of the + // handler. Then check that the client is still alive, in case it timed out before + // this code was able to update things. + TransportChannelHandler handler = cachedClient.getChannel().pipeline() + .get(TransportChannelHandler.class); + synchronized (handler) { + handler.getResponseHandler().updateTimeOfLastRequest(); + } + + if (cachedClient.isActive()) { + logger.trace("Returning cached connection to {}: {}", + cachedClient.getSocketAddress(), cachedClient); + return cachedClient; + } + } + + // If we reach here, we don't have an existing connection open. Let's create a new one. + // Multiple threads might race here to create new connections. Keep only one of them active. + final long preResolveHost = System.nanoTime(); + final InetSocketAddress resolvedAddress = new InetSocketAddress(remoteHost, remotePort); + final long hostResolveTimeMs = (System.nanoTime() - preResolveHost) / 1000000; + if (hostResolveTimeMs > 2000) { + logger.warn("DNS resolution for {} took {} ms", resolvedAddress, hostResolveTimeMs); + } else { + logger.trace("DNS resolution for {} took {} ms", resolvedAddress, hostResolveTimeMs); + } + + synchronized (clientPool.locks[clientIndex]) { + cachedClient = clientPool.clients[clientIndex]; + + if (cachedClient != null) { + if (cachedClient.isActive()) { + logger.trace("Returning cached connection to {}: {}", resolvedAddress, cachedClient); + return cachedClient; + } else { + logger.info("Found inactive connection to {}, creating a new one.", resolvedAddress); + } + } + clientPool.clients[clientIndex] = createClient(resolvedAddress); + return clientPool.clients[clientIndex]; + } + } + + /** + * Create a completely new {@link TransportClient} to the given remote host / port. + * This connection is not pooled. + * + * As with {@link #createClient(String, int)}, this method is blocking. + */ + public TransportClient createUnmanagedClient(String remoteHost, int remotePort) + throws IOException, InterruptedException { + final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort); + return createClient(address); + } + + /** Create a completely new {@link TransportClient} to the remote address. */ + private TransportClient createClient(InetSocketAddress address) + throws IOException, InterruptedException { + logger.debug("Creating new connection to {}", address); + + Bootstrap bootstrap = new Bootstrap(); + bootstrap.group(workerGroup) + .channel(socketChannelClass) + // Disable Nagle's Algorithm since we don't want packets to wait + .option(ChannelOption.TCP_NODELAY, true) + .option(ChannelOption.SO_KEEPALIVE, true) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs()) + .option(ChannelOption.ALLOCATOR, pooledAllocator); + + if (conf.receiveBuf() > 0) { + bootstrap.option(ChannelOption.SO_RCVBUF, conf.receiveBuf()); + } + + if (conf.sendBuf() > 0) { + bootstrap.option(ChannelOption.SO_SNDBUF, conf.sendBuf()); + } + + final AtomicReference clientRef = new AtomicReference<>(); + final AtomicReference channelRef = new AtomicReference<>(); + + bootstrap.handler(new ChannelInitializer() { + @Override + public void initChannel(SocketChannel ch) { + TransportChannelHandler clientHandler = context.initializePipeline(ch); + clientRef.set(clientHandler.getClient()); + channelRef.set(ch); + } + }); + + // Connect to the remote server + long preConnect = System.nanoTime(); + ChannelFuture cf = bootstrap.connect(address); + if (!cf.await(conf.connectionTimeoutMs())) { + throw new IOException( + String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs())); + } else if (cf.cause() != null) { + throw new IOException(String.format("Failed to connect to %s", address), cf.cause()); + } + + TransportClient client = clientRef.get(); + Channel channel = channelRef.get(); + assert client != null : "Channel future completed successfully with null client"; + + // Execute any client bootstraps synchronously before marking the Client as successful. + long preBootstrap = System.nanoTime(); + logger.debug("Connection to {} successful, running bootstraps...", address); + try { + for (TransportClientBootstrap clientBootstrap : clientBootstraps) { + clientBootstrap.doBootstrap(client, channel); + } + } catch (Exception e) { // catch non-RuntimeExceptions too as bootstrap may be written in Scala + long bootstrapTimeMs = (System.nanoTime() - preBootstrap) / 1000000; + logger.error("Exception while bootstrapping client after " + bootstrapTimeMs + " ms", e); + client.close(); + throw Throwables.propagate(e); + } + long postBootstrap = System.nanoTime(); + + logger.info("Successfully created connection to {} after {} ms ({} ms spent in bootstraps)", + address, (postBootstrap - preConnect) / 1000000, (postBootstrap - preBootstrap) / 1000000); + + return client; + } + + /** Close all connections in the connection pool, and shutdown the worker thread pool. */ + @Override + public void close() { + // Go through all clients and close them if they are active. + for (ClientPool clientPool : connectionPool.values()) { + for (int i = 0; i < clientPool.clients.length; i++) { + TransportClient client = clientPool.clients[i]; + if (client != null) { + clientPool.clients[i] = null; + JavaUtils.closeQuietly(client); + } + } + } + connectionPool.clear(); + + if (workerGroup != null) { + workerGroup.shutdownGracefully(); + workerGroup = null; + } + } +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java new file mode 100644 index 000000000000..7a3d96ceaef0 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -0,0 +1,270 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.client; + +import java.io.IOException; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicLong; + +import com.google.common.annotations.VisibleForTesting; +import io.netty.channel.Channel; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.protocol.ChunkFetchFailure; +import org.apache.spark.network.protocol.ChunkFetchSuccess; +import org.apache.spark.network.protocol.ResponseMessage; +import org.apache.spark.network.protocol.RpcFailure; +import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.protocol.StreamChunkId; +import org.apache.spark.network.protocol.StreamFailure; +import org.apache.spark.network.protocol.StreamResponse; +import org.apache.spark.network.server.MessageHandler; +import static org.apache.spark.network.util.NettyUtils.getRemoteAddress; +import org.apache.spark.network.util.TransportFrameDecoder; + +/** + * Handler that processes server responses, in response to requests issued from a + * [[TransportClient]]. It works by tracking the list of outstanding requests (and their callbacks). + * + * Concurrency: thread safe and can be called from multiple threads. + */ +public class TransportResponseHandler extends MessageHandler { + private static final Logger logger = LoggerFactory.getLogger(TransportResponseHandler.class); + + private final Channel channel; + + private final Map outstandingFetches; + + private final Map outstandingRpcs; + + private final Queue> streamCallbacks; + private volatile boolean streamActive; + + /** Records the time (in system nanoseconds) that the last fetch or RPC request was sent. */ + private final AtomicLong timeOfLastRequestNs; + + public TransportResponseHandler(Channel channel) { + this.channel = channel; + this.outstandingFetches = new ConcurrentHashMap<>(); + this.outstandingRpcs = new ConcurrentHashMap<>(); + this.streamCallbacks = new ConcurrentLinkedQueue<>(); + this.timeOfLastRequestNs = new AtomicLong(0); + } + + public void addFetchRequest(StreamChunkId streamChunkId, ChunkReceivedCallback callback) { + updateTimeOfLastRequest(); + outstandingFetches.put(streamChunkId, callback); + } + + public void removeFetchRequest(StreamChunkId streamChunkId) { + outstandingFetches.remove(streamChunkId); + } + + public void addRpcRequest(long requestId, RpcResponseCallback callback) { + updateTimeOfLastRequest(); + outstandingRpcs.put(requestId, callback); + } + + public void removeRpcRequest(long requestId) { + outstandingRpcs.remove(requestId); + } + + public void addStreamCallback(String streamId, StreamCallback callback) { + timeOfLastRequestNs.set(System.nanoTime()); + streamCallbacks.offer(ImmutablePair.of(streamId, callback)); + } + + @VisibleForTesting + public void deactivateStream() { + streamActive = false; + } + + /** + * Fire the failure callback for all outstanding requests. This is called when we have an + * uncaught exception or pre-mature connection termination. + */ + private void failOutstandingRequests(Throwable cause) { + for (Map.Entry entry : outstandingFetches.entrySet()) { + try { + entry.getValue().onFailure(entry.getKey().chunkIndex, cause); + } catch (Exception e) { + logger.warn("ChunkReceivedCallback.onFailure throws exception", e); + } + } + for (Map.Entry entry : outstandingRpcs.entrySet()) { + try { + entry.getValue().onFailure(cause); + } catch (Exception e) { + logger.warn("RpcResponseCallback.onFailure throws exception", e); + } + } + for (Pair entry : streamCallbacks) { + try { + entry.getValue().onFailure(entry.getKey(), cause); + } catch (Exception e) { + logger.warn("StreamCallback.onFailure throws exception", e); + } + } + + // It's OK if new fetches appear, as they will fail immediately. + outstandingFetches.clear(); + outstandingRpcs.clear(); + streamCallbacks.clear(); + } + + @Override + public void channelActive() { + } + + @Override + public void channelInactive() { + if (numOutstandingRequests() > 0) { + String remoteAddress = getRemoteAddress(channel); + logger.error("Still have {} requests outstanding when connection from {} is closed", + numOutstandingRequests(), remoteAddress); + failOutstandingRequests(new IOException("Connection from " + remoteAddress + " closed")); + } + } + + @Override + public void exceptionCaught(Throwable cause) { + if (numOutstandingRequests() > 0) { + String remoteAddress = getRemoteAddress(channel); + logger.error("Still have {} requests outstanding when connection from {} is closed", + numOutstandingRequests(), remoteAddress); + failOutstandingRequests(cause); + } + } + + @Override + public void handle(ResponseMessage message) throws Exception { + if (message instanceof ChunkFetchSuccess) { + ChunkFetchSuccess resp = (ChunkFetchSuccess) message; + ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId); + if (listener == null) { + logger.warn("Ignoring response for block {} from {} since it is not outstanding", + resp.streamChunkId, getRemoteAddress(channel)); + resp.body().release(); + } else { + outstandingFetches.remove(resp.streamChunkId); + listener.onSuccess(resp.streamChunkId.chunkIndex, resp.body()); + resp.body().release(); + } + } else if (message instanceof ChunkFetchFailure) { + ChunkFetchFailure resp = (ChunkFetchFailure) message; + ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId); + if (listener == null) { + logger.warn("Ignoring response for block {} from {} ({}) since it is not outstanding", + resp.streamChunkId, getRemoteAddress(channel), resp.errorString); + } else { + outstandingFetches.remove(resp.streamChunkId); + listener.onFailure(resp.streamChunkId.chunkIndex, new ChunkFetchFailureException( + "Failure while fetching " + resp.streamChunkId + ": " + resp.errorString)); + } + } else if (message instanceof RpcResponse) { + RpcResponse resp = (RpcResponse) message; + RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); + if (listener == null) { + logger.warn("Ignoring response for RPC {} from {} ({} bytes) since it is not outstanding", + resp.requestId, getRemoteAddress(channel), resp.body().size()); + } else { + outstandingRpcs.remove(resp.requestId); + try { + listener.onSuccess(resp.body().nioByteBuffer()); + } finally { + resp.body().release(); + } + } + } else if (message instanceof RpcFailure) { + RpcFailure resp = (RpcFailure) message; + RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); + if (listener == null) { + logger.warn("Ignoring response for RPC {} from {} ({}) since it is not outstanding", + resp.requestId, getRemoteAddress(channel), resp.errorString); + } else { + outstandingRpcs.remove(resp.requestId); + listener.onFailure(new RuntimeException(resp.errorString)); + } + } else if (message instanceof StreamResponse) { + StreamResponse resp = (StreamResponse) message; + Pair entry = streamCallbacks.poll(); + if (entry != null) { + StreamCallback callback = entry.getValue(); + if (resp.byteCount > 0) { + StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount, + callback); + try { + TransportFrameDecoder frameDecoder = (TransportFrameDecoder) + channel.pipeline().get(TransportFrameDecoder.HANDLER_NAME); + frameDecoder.setInterceptor(interceptor); + streamActive = true; + } catch (Exception e) { + logger.error("Error installing stream handler.", e); + deactivateStream(); + } + } else { + try { + callback.onComplete(resp.streamId); + } catch (Exception e) { + logger.warn("Error in stream handler onComplete().", e); + } + } + } else { + logger.error("Could not find callback for StreamResponse."); + } + } else if (message instanceof StreamFailure) { + StreamFailure resp = (StreamFailure) message; + Pair entry = streamCallbacks.poll(); + if (entry != null) { + StreamCallback callback = entry.getValue(); + try { + callback.onFailure(resp.streamId, new RuntimeException(resp.error)); + } catch (IOException ioe) { + logger.warn("Error in stream failure handler.", ioe); + } + } else { + logger.warn("Stream failure with unknown callback: {}", resp.error); + } + } else { + throw new IllegalStateException("Unknown response type: " + message.type()); + } + } + + /** Returns total number of outstanding requests (fetch requests + rpcs) */ + public int numOutstandingRequests() { + return outstandingFetches.size() + outstandingRpcs.size() + streamCallbacks.size() + + (streamActive ? 1 : 0); + } + + /** Returns the time in nanoseconds of when the last request was sent out. */ + public long getTimeOfLastRequestNs() { + return timeOfLastRequestNs.get(); + } + + /** Updates the time of the last request to the current system time. */ + public void updateTimeOfLastRequest() { + timeOfLastRequestNs.set(System.nanoTime()); + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java new file mode 100644 index 000000000000..3c263783a610 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.crypto; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.security.GeneralSecurityException; + +import com.google.common.base.Throwables; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientBootstrap; +import org.apache.spark.network.sasl.SaslClientBootstrap; +import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.util.TransportConf; + +/** + * Bootstraps a {@link TransportClient} by performing authentication using Spark's auth protocol. + * + * This bootstrap falls back to using the SASL bootstrap if the server throws an error during + * authentication, and the configuration allows it. This is used for backwards compatibility + * with external shuffle services that do not support the new protocol. + * + * It also automatically falls back to SASL if the new encryption backend is disabled, so that + * callers only need to install this bootstrap when authentication is enabled. + */ +public class AuthClientBootstrap implements TransportClientBootstrap { + + private static final Logger LOG = LoggerFactory.getLogger(AuthClientBootstrap.class); + + private final TransportConf conf; + private final String appId; + private final SecretKeyHolder secretKeyHolder; + + public AuthClientBootstrap( + TransportConf conf, + String appId, + SecretKeyHolder secretKeyHolder) { + this.conf = conf; + // TODO: right now this behaves like the SASL backend, because when executors start up + // they don't necessarily know the app ID. So they send a hardcoded "user" that is defined + // in the SecurityManager, which will also always return the same secret (regardless of the + // user name). All that's needed here is for this "user" to match on both sides, since that's + // required by the protocol. At some point, though, it would be better for the actual app ID + // to be provided here. + this.appId = appId; + this.secretKeyHolder = secretKeyHolder; + } + + @Override + public void doBootstrap(TransportClient client, Channel channel) { + if (!conf.encryptionEnabled()) { + LOG.debug("AES encryption disabled, using old auth protocol."); + doSaslAuth(client, channel); + return; + } + + try { + doSparkAuth(client, channel); + } catch (GeneralSecurityException | IOException e) { + throw Throwables.propagate(e); + } catch (RuntimeException e) { + // There isn't a good exception that can be caught here to know whether it's really + // OK to switch back to SASL (because the server doesn't speak the new protocol). So + // try it anyway, and in the worst case things will fail again. + if (conf.saslFallback()) { + LOG.warn("New auth protocol failed, trying SASL.", e); + doSaslAuth(client, channel); + } else { + throw e; + } + } + } + + private void doSparkAuth(TransportClient client, Channel channel) + throws GeneralSecurityException, IOException { + + String secretKey = secretKeyHolder.getSecretKey(appId); + try (AuthEngine engine = new AuthEngine(appId, secretKey, conf)) { + ClientChallenge challenge = engine.challenge(); + ByteBuf challengeData = Unpooled.buffer(challenge.encodedLength()); + challenge.encode(challengeData); + + ByteBuffer responseData = + client.sendRpcSync(challengeData.nioBuffer(), conf.authRTTimeoutMs()); + ServerResponse response = ServerResponse.decodeMessage(responseData); + + engine.validate(response); + engine.sessionCipher().addToChannel(channel); + } + } + + private void doSaslAuth(TransportClient client, Channel channel) { + SaslClientBootstrap sasl = new SaslClientBootstrap(conf, appId, secretKeyHolder); + sasl.doBootstrap(client, channel); + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java new file mode 100644 index 000000000000..056505ef5335 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java @@ -0,0 +1,284 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.crypto; + +import java.io.Closeable; +import java.io.IOException; +import java.math.BigInteger; +import java.security.GeneralSecurityException; +import java.util.Arrays; +import java.util.Properties; +import javax.crypto.Cipher; +import javax.crypto.SecretKey; +import javax.crypto.SecretKeyFactory; +import javax.crypto.ShortBufferException; +import javax.crypto.spec.IvParameterSpec; +import javax.crypto.spec.PBEKeySpec; +import javax.crypto.spec.SecretKeySpec; +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.primitives.Bytes; +import org.apache.commons.crypto.cipher.CryptoCipher; +import org.apache.commons.crypto.cipher.CryptoCipherFactory; +import org.apache.commons.crypto.random.CryptoRandom; +import org.apache.commons.crypto.random.CryptoRandomFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.util.TransportConf; + +/** + * A helper class for abstracting authentication and key negotiation details. This is used by + * both client and server sides, since the operations are basically the same. + */ +class AuthEngine implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(AuthEngine.class); + private static final BigInteger ONE = new BigInteger(new byte[] { 0x1 }); + + private final byte[] appId; + private final char[] secret; + private final TransportConf conf; + private final Properties cryptoConf; + private final CryptoRandom random; + + private byte[] authNonce; + + @VisibleForTesting + byte[] challenge; + + private TransportCipher sessionCipher; + private CryptoCipher encryptor; + private CryptoCipher decryptor; + + AuthEngine(String appId, String secret, TransportConf conf) throws GeneralSecurityException { + this.appId = appId.getBytes(UTF_8); + this.conf = conf; + this.cryptoConf = conf.cryptoConf(); + this.secret = secret.toCharArray(); + this.random = CryptoRandomFactory.getCryptoRandom(cryptoConf); + } + + /** + * Create the client challenge. + * + * @return A challenge to be sent the remote side. + */ + ClientChallenge challenge() throws GeneralSecurityException { + this.authNonce = randomBytes(conf.encryptionKeyLength() / Byte.SIZE); + SecretKeySpec authKey = generateKey(conf.keyFactoryAlgorithm(), conf.keyFactoryIterations(), + authNonce, conf.encryptionKeyLength()); + initializeForAuth(conf.cipherTransformation(), authNonce, authKey); + + this.challenge = randomBytes(conf.encryptionKeyLength() / Byte.SIZE); + return new ClientChallenge(new String(appId, UTF_8), + conf.keyFactoryAlgorithm(), + conf.keyFactoryIterations(), + conf.cipherTransformation(), + conf.encryptionKeyLength(), + authNonce, + challenge(appId, authNonce, challenge)); + } + + /** + * Validates the client challenge, and create the encryption backend for the channel from the + * parameters sent by the client. + * + * @param clientChallenge The challenge from the client. + * @return A response to be sent to the client. + */ + ServerResponse respond(ClientChallenge clientChallenge) + throws GeneralSecurityException { + + SecretKeySpec authKey = generateKey(clientChallenge.kdf, clientChallenge.iterations, + clientChallenge.nonce, clientChallenge.keyLength); + initializeForAuth(clientChallenge.cipher, clientChallenge.nonce, authKey); + + byte[] challenge = validateChallenge(clientChallenge.nonce, clientChallenge.challenge); + byte[] response = challenge(appId, clientChallenge.nonce, rawResponse(challenge)); + byte[] sessionNonce = randomBytes(conf.encryptionKeyLength() / Byte.SIZE); + byte[] inputIv = randomBytes(conf.ivLength()); + byte[] outputIv = randomBytes(conf.ivLength()); + + SecretKeySpec sessionKey = generateKey(clientChallenge.kdf, clientChallenge.iterations, + sessionNonce, clientChallenge.keyLength); + this.sessionCipher = new TransportCipher(cryptoConf, clientChallenge.cipher, sessionKey, + inputIv, outputIv); + + // Note the IVs are swapped in the response. + return new ServerResponse(response, encrypt(sessionNonce), encrypt(outputIv), encrypt(inputIv)); + } + + /** + * Validates the server response and initializes the cipher to use for the session. + * + * @param serverResponse The response from the server. + */ + void validate(ServerResponse serverResponse) throws GeneralSecurityException { + byte[] response = validateChallenge(authNonce, serverResponse.response); + + byte[] expected = rawResponse(challenge); + Preconditions.checkArgument(Arrays.equals(expected, response)); + + byte[] nonce = decrypt(serverResponse.nonce); + byte[] inputIv = decrypt(serverResponse.inputIv); + byte[] outputIv = decrypt(serverResponse.outputIv); + + SecretKeySpec sessionKey = generateKey(conf.keyFactoryAlgorithm(), conf.keyFactoryIterations(), + nonce, conf.encryptionKeyLength()); + this.sessionCipher = new TransportCipher(cryptoConf, conf.cipherTransformation(), sessionKey, + inputIv, outputIv); + } + + TransportCipher sessionCipher() { + Preconditions.checkState(sessionCipher != null); + return sessionCipher; + } + + @Override + public void close() throws IOException { + // Close ciphers (by calling "doFinal()" with dummy data) and the random instance so that + // internal state is cleaned up. Error handling here is just for paranoia, and not meant to + // accurately report the errors when they happen. + RuntimeException error = null; + byte[] dummy = new byte[8]; + try { + doCipherOp(encryptor, dummy, true); + } catch (Exception e) { + error = new RuntimeException(e); + } + try { + doCipherOp(decryptor, dummy, true); + } catch (Exception e) { + error = new RuntimeException(e); + } + random.close(); + + if (error != null) { + throw error; + } + } + + @VisibleForTesting + byte[] challenge(byte[] appId, byte[] nonce, byte[] challenge) throws GeneralSecurityException { + return encrypt(Bytes.concat(appId, nonce, challenge)); + } + + @VisibleForTesting + byte[] rawResponse(byte[] challenge) { + BigInteger orig = new BigInteger(challenge); + BigInteger response = orig.add(ONE); + return response.toByteArray(); + } + + private byte[] decrypt(byte[] in) throws GeneralSecurityException { + return doCipherOp(decryptor, in, false); + } + + private byte[] encrypt(byte[] in) throws GeneralSecurityException { + return doCipherOp(encryptor, in, false); + } + + private void initializeForAuth(String cipher, byte[] nonce, SecretKeySpec key) + throws GeneralSecurityException { + + // commons-crypto currently only supports ciphers that require an initial vector; so + // create a dummy vector so that we can initialize the ciphers. In the future, if + // different ciphers are supported, this will have to be configurable somehow. + byte[] iv = new byte[conf.ivLength()]; + System.arraycopy(nonce, 0, iv, 0, Math.min(nonce.length, iv.length)); + + encryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); + encryptor.init(Cipher.ENCRYPT_MODE, key, new IvParameterSpec(iv)); + + decryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); + decryptor.init(Cipher.DECRYPT_MODE, key, new IvParameterSpec(iv)); + } + + /** + * Validates an encrypted challenge as defined in the protocol, and returns the byte array + * that corresponds to the actual challenge data. + */ + private byte[] validateChallenge(byte[] nonce, byte[] encryptedChallenge) + throws GeneralSecurityException { + + byte[] challenge = decrypt(encryptedChallenge); + checkSubArray(appId, challenge, 0); + checkSubArray(nonce, challenge, appId.length); + return Arrays.copyOfRange(challenge, appId.length + nonce.length, challenge.length); + } + + private SecretKeySpec generateKey(String kdf, int iterations, byte[] salt, int keyLength) + throws GeneralSecurityException { + + SecretKeyFactory factory = SecretKeyFactory.getInstance(kdf); + PBEKeySpec spec = new PBEKeySpec(secret, salt, iterations, keyLength); + + long start = System.nanoTime(); + SecretKey key = factory.generateSecret(spec); + long end = System.nanoTime(); + + LOG.debug("Generated key with {} iterations in {} us.", conf.keyFactoryIterations(), + (end - start) / 1000); + + return new SecretKeySpec(key.getEncoded(), conf.keyAlgorithm()); + } + + private byte[] doCipherOp(CryptoCipher cipher, byte[] in, boolean isFinal) + throws GeneralSecurityException { + + Preconditions.checkState(cipher != null); + + int scale = 1; + while (true) { + int size = in.length * scale; + byte[] buffer = new byte[size]; + try { + int outSize = isFinal ? cipher.doFinal(in, 0, in.length, buffer, 0) + : cipher.update(in, 0, in.length, buffer, 0); + if (outSize != buffer.length) { + byte[] output = new byte[outSize]; + System.arraycopy(buffer, 0, output, 0, output.length); + return output; + } else { + return buffer; + } + } catch (ShortBufferException e) { + // Try again with a bigger buffer. + scale *= 2; + } + } + } + + private byte[] randomBytes(int count) { + byte[] bytes = new byte[count]; + random.nextBytes(bytes); + return bytes; + } + + /** Checks that the "test" array is in the data array starting at the given offset. */ + private void checkSubArray(byte[] test, byte[] data, int offset) { + Preconditions.checkArgument(data.length >= test.length + offset); + for (int i = 0; i < test.length; i++) { + Preconditions.checkArgument(test[i] == data[i + offset]); + } + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java new file mode 100644 index 000000000000..8a6e3858081b --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthRpcHandler.java @@ -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.network.crypto; + +import java.nio.ByteBuffer; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.sasl.SaslRpcHandler; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.util.TransportConf; + +/** + * RPC Handler which performs authentication using Spark's auth protocol before delegating to a + * child RPC handler. If the configuration allows, this handler will delegate messages to a SASL + * RPC handler for further authentication, to support for clients that do not support Spark's + * protocol. + * + * The delegate will only receive messages if the given connection has been successfully + * authenticated. A connection may be authenticated at most once. + */ +class AuthRpcHandler extends RpcHandler { + private static final Logger LOG = LoggerFactory.getLogger(AuthRpcHandler.class); + + /** Transport configuration. */ + private final TransportConf conf; + + /** The client channel. */ + private final Channel channel; + + /** + * RpcHandler we will delegate to for authenticated connections. When falling back to SASL + * this will be replaced with the SASL RPC handler. + */ + @VisibleForTesting + RpcHandler delegate; + + /** Class which provides secret keys which are shared by server and client on a per-app basis. */ + private final SecretKeyHolder secretKeyHolder; + + /** Whether auth is done and future calls should be delegated. */ + @VisibleForTesting + boolean doDelegate; + + AuthRpcHandler( + TransportConf conf, + Channel channel, + RpcHandler delegate, + SecretKeyHolder secretKeyHolder) { + this.conf = conf; + this.channel = channel; + this.delegate = delegate; + this.secretKeyHolder = secretKeyHolder; + } + + @Override + public void receive(TransportClient client, ByteBuffer message, RpcResponseCallback callback) { + if (doDelegate) { + delegate.receive(client, message, callback); + return; + } + + int position = message.position(); + int limit = message.limit(); + + ClientChallenge challenge; + try { + challenge = ClientChallenge.decodeMessage(message); + LOG.debug("Received new auth challenge for client {}.", channel.remoteAddress()); + } catch (RuntimeException e) { + if (conf.saslFallback()) { + LOG.warn("Failed to parse new auth challenge, reverting to SASL for client {}.", + channel.remoteAddress()); + delegate = new SaslRpcHandler(conf, channel, delegate, secretKeyHolder); + message.position(position); + message.limit(limit); + delegate.receive(client, message, callback); + doDelegate = true; + } else { + LOG.debug("Unexpected challenge message from client {}, closing channel.", + channel.remoteAddress()); + callback.onFailure(new IllegalArgumentException("Unknown challenge message.")); + channel.close(); + } + return; + } + + // Here we have the client challenge, so perform the new auth protocol and set up the channel. + AuthEngine engine = null; + try { + String secret = secretKeyHolder.getSecretKey(challenge.appId); + Preconditions.checkState(secret != null, + "Trying to authenticate non-registered app %s.", challenge.appId); + LOG.debug("Authenticating challenge for app {}.", challenge.appId); + engine = new AuthEngine(challenge.appId, secret, conf); + ServerResponse response = engine.respond(challenge); + ByteBuf responseData = Unpooled.buffer(response.encodedLength()); + response.encode(responseData); + callback.onSuccess(responseData.nioBuffer()); + engine.sessionCipher().addToChannel(channel); + } catch (Exception e) { + // This is a fatal error: authentication has failed. Close the channel explicitly. + LOG.debug("Authentication failed for client {}, closing channel.", channel.remoteAddress()); + callback.onFailure(new IllegalArgumentException("Authentication failed.")); + channel.close(); + return; + } finally { + if (engine != null) { + try { + engine.close(); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + + LOG.debug("Authorization successful for client {}.", channel.remoteAddress()); + doDelegate = true; + } + + @Override + public void receive(TransportClient client, ByteBuffer message) { + delegate.receive(client, message); + } + + @Override + public StreamManager getStreamManager() { + return delegate.getStreamManager(); + } + + @Override + public void channelActive(TransportClient client) { + delegate.channelActive(client); + } + + @Override + public void channelInactive(TransportClient client) { + delegate.channelInactive(client); + } + + @Override + public void exceptionCaught(Throwable cause, TransportClient client) { + delegate.exceptionCaught(cause, client); + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthServerBootstrap.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthServerBootstrap.java new file mode 100644 index 000000000000..77a2a6af4d13 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthServerBootstrap.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.crypto; + +import io.netty.channel.Channel; + +import org.apache.spark.network.sasl.SaslServerBootstrap; +import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportServerBootstrap; +import org.apache.spark.network.util.TransportConf; + +/** + * A bootstrap which is executed on a TransportServer's client channel once a client connects + * to the server, enabling authentication using Spark's auth protocol (and optionally SASL for + * clients that don't support the new protocol). + * + * It also automatically falls back to SASL if the new encryption backend is disabled, so that + * callers only need to install this bootstrap when authentication is enabled. + */ +public class AuthServerBootstrap implements TransportServerBootstrap { + + private final TransportConf conf; + private final SecretKeyHolder secretKeyHolder; + + public AuthServerBootstrap(TransportConf conf, SecretKeyHolder secretKeyHolder) { + this.conf = conf; + this.secretKeyHolder = secretKeyHolder; + } + + public RpcHandler doBootstrap(Channel channel, RpcHandler rpcHandler) { + if (!conf.encryptionEnabled()) { + TransportServerBootstrap sasl = new SaslServerBootstrap(conf, secretKeyHolder); + return sasl.doBootstrap(channel, rpcHandler); + } + + return new AuthRpcHandler(conf, channel, rpcHandler, secretKeyHolder); + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/ClientChallenge.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/ClientChallenge.java new file mode 100644 index 000000000000..819b8a7efbdb --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/ClientChallenge.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.crypto; + +import java.nio.ByteBuffer; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + +import org.apache.spark.network.protocol.Encodable; +import org.apache.spark.network.protocol.Encoders; + +/** + * The client challenge message, used to initiate authentication. + * + * Please see crypto/README.md for more details of implementation. + */ +public class ClientChallenge implements Encodable { + /** Serialization tag used to catch incorrect payloads. */ + private static final byte TAG_BYTE = (byte) 0xFA; + + public final String appId; + public final String kdf; + public final int iterations; + public final String cipher; + public final int keyLength; + public final byte[] nonce; + public final byte[] challenge; + + public ClientChallenge( + String appId, + String kdf, + int iterations, + String cipher, + int keyLength, + byte[] nonce, + byte[] challenge) { + this.appId = appId; + this.kdf = kdf; + this.iterations = iterations; + this.cipher = cipher; + this.keyLength = keyLength; + this.nonce = nonce; + this.challenge = challenge; + } + + @Override + public int encodedLength() { + return 1 + 4 + 4 + + Encoders.Strings.encodedLength(appId) + + Encoders.Strings.encodedLength(kdf) + + Encoders.Strings.encodedLength(cipher) + + Encoders.ByteArrays.encodedLength(nonce) + + Encoders.ByteArrays.encodedLength(challenge); + } + + @Override + public void encode(ByteBuf buf) { + buf.writeByte(TAG_BYTE); + Encoders.Strings.encode(buf, appId); + Encoders.Strings.encode(buf, kdf); + buf.writeInt(iterations); + Encoders.Strings.encode(buf, cipher); + buf.writeInt(keyLength); + Encoders.ByteArrays.encode(buf, nonce); + Encoders.ByteArrays.encode(buf, challenge); + } + + public static ClientChallenge decodeMessage(ByteBuffer buffer) { + ByteBuf buf = Unpooled.wrappedBuffer(buffer); + + if (buf.readByte() != TAG_BYTE) { + throw new IllegalArgumentException("Expected ClientChallenge, received something else."); + } + + return new ClientChallenge( + Encoders.Strings.decode(buf), + Encoders.Strings.decode(buf), + buf.readInt(), + Encoders.Strings.decode(buf), + buf.readInt(), + Encoders.ByteArrays.decode(buf), + Encoders.ByteArrays.decode(buf)); + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/README.md b/common/network-common/src/main/java/org/apache/spark/network/crypto/README.md new file mode 100644 index 000000000000..14df70327049 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/README.md @@ -0,0 +1,158 @@ +Spark Auth Protocol and AES Encryption Support +============================================== + +This file describes an auth protocol used by Spark as a more secure alternative to DIGEST-MD5. This +protocol is built on symmetric key encryption, based on the assumption that the two endpoints being +authenticated share a common secret, which is how Spark authentication currently works. The protocol +provides mutual authentication, meaning that after the negotiation both parties know that the remote +side knows the shared secret. The protocol is influenced by the ISO/IEC 9798 protocol, although it's +not an implementation of it. + +This protocol could be replaced with TLS PSK, except no PSK ciphers are available in the currently +released JREs. + +The protocol aims at solving the following shortcomings in Spark's current usage of DIGEST-MD5: + +- MD5 is an aging hash algorithm with known weaknesses, and a more secure alternative is desired. +- DIGEST-MD5 has a pre-defined set of ciphers for which it can generate keys. The only + viable, supported cipher these days is 3DES, and a more modern alternative is desired. +- Encrypting AES session keys with 3DES doesn't solve the issue, since the weakest link + in the negotiation would still be MD5 and 3DES. + +The protocol assumes that the shared secret is generated and distributed in a secure manner. + +The protocol always negotiates encryption keys. If encryption is not desired, the existing +SASL-based authentication, or no authentication at all, can be chosen instead. + +When messages are described below, it's expected that the implementation should support +arbitrary sizes for fields that don't have a fixed size. + +Client Challenge +---------------- + +The auth negotiation is started by the client. The client starts by generating an encryption +key based on the application's shared secret, and a nonce. + + KEY = KDF(SECRET, SALT, KEY_LENGTH) + +Where: +- KDF(): a key derivation function that takes a secret, a salt, a configurable number of + iterations, and a configurable key length. +- SALT: a byte sequence used to salt the key derivation function. +- KEY_LENGTH: length of the encryption key to generate. + + +The client generates a message with the following content: + + CLIENT_CHALLENGE = ( + APP_ID, + KDF, + ITERATIONS, + CIPHER, + KEY_LENGTH, + ANONCE, + ENC(APP_ID || ANONCE || CHALLENGE)) + +Where: + +- APP_ID: the application ID which the server uses to identify the shared secret. +- KDF: the key derivation function described above. +- ITERATIONS: number of iterations to run the KDF when generating keys. +- CIPHER: the cipher used to encrypt data. +- KEY_LENGTH: length of the encryption keys to generate, in bits. +- ANONCE: the nonce used as the salt when generating the auth key. +- ENC(): an encryption function that uses the cipher and the generated key. This function + will also be used in the definition of other messages below. +- CHALLENGE: a byte sequence used as a challenge to the server. +- ||: concatenation operator. + +When strings are used where byte arrays are expected, the UTF-8 representation of the string +is assumed. + +To respond to the challenge, the server should consider the byte array as representing an +arbitrary-length integer, and respond with the value of the integer plus one. + + +Server Response And Challenge +----------------------------- + +Once the client challenge is received, the server will generate the same auth key by +using the same algorithm the client has used. It will then verify the client challenge: +if the APP_ID and ANONCE fields match, the server knows that the client has the shared +secret. The server then creates a response to the client challenge, to prove that it also +has the secret key, and provides parameters to be used when creating the session key. + +The following describes the response from the server: + + SERVER_CHALLENGE = ( + ENC(APP_ID || ANONCE || RESPONSE), + ENC(SNONCE), + ENC(INIV), + ENC(OUTIV)) + +Where: + +- RESPONSE: the server's response to the client challenge. +- SNONCE: a nonce to be used as salt when generating the session key. +- INIV: initialization vector used to initialize the input channel of the client. +- OUTIV: initialization vector used to initialize the output channel of the client. + +At this point the server considers the client to be authenticated, and will try to +decrypt any data further sent by the client using the session key. + + +Default Algorithms +------------------ + +Configuration options are available for the KDF and cipher algorithms to use. + +The default KDF is "PBKDF2WithHmacSHA1". Users should be able to select any algorithm +from those supported by the `javax.crypto.SecretKeyFactory` class, as long as they support +PBEKeySpec when generating keys. The default number of iterations was chosen to take a +reasonable amount of time on modern CPUs. See the documentation in TransportConf for more +details. + +The default cipher algorithm is "AES/CTR/NoPadding". Users should be able to select any +algorithm supported by the commons-crypto library. It should allow the cipher to operate +in stream mode. + +The default key length is 128 (bits). + + +Implementation Details +---------------------- + +The commons-crypto library currently only supports AES ciphers, and requires an initialization +vector (IV). This first version of the protocol does not explicitly include the IV in the client +challenge message. Instead, the IV should be derived from the nonce, including the needed bytes, and +padding the IV with zeroes in case the nonce is not long enough. + +Future versions of the protocol might add support for new ciphers and explicitly include needed +configuration parameters in the messages. + + +Threat Assessment +----------------- + +The protocol is secure against different forms of attack: + +* Eavesdropping: the protocol is built on the assumption that it's computationally infeasible + to calculate the original secret from the encrypted messages. Neither the secret nor any + encryption keys are transmitted on the wire, encrypted or not. + +* Man-in-the-middle: because the protocol performs mutual authentication, both ends need to + know the shared secret to be able to decrypt session data. Even if an attacker is able to insert a + malicious "proxy" between endpoints, the attacker won't be able to read any of the data exchanged + between client and server, nor insert arbitrary commands for the server to execute. + +* Replay attacks: the use of nonces when generating keys prevents an attacker from being able to + just replay messages sniffed from the communication channel. + +An attacker may replay the client challenge and successfully "prove" to a server that it "knows" the +shared secret. But the attacker won't be able to decrypt the server's response, and thus won't be +able to generate a session key, which will make it hard to craft a valid, encrypted message that the +server will be able to understand. This will cause the server to close the connection as soon as the +attacker tries to send any command to the server. The attacker can just hold the channel open for +some time, which will be closed when the server times out the channel. These issues could be +separately mitigated by adding a shorter timeout for the first message after authentication, and +potentially by adding host blacklists if a possible attack is detected from a particular host. diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/ServerResponse.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/ServerResponse.java new file mode 100644 index 000000000000..caf3a0f3b38c --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/ServerResponse.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.crypto; + +import java.nio.ByteBuffer; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + +import org.apache.spark.network.protocol.Encodable; +import org.apache.spark.network.protocol.Encoders; + +/** + * Server's response to client's challenge. + * + * Please see crypto/README.md for more details. + */ +public class ServerResponse implements Encodable { + /** Serialization tag used to catch incorrect payloads. */ + private static final byte TAG_BYTE = (byte) 0xFB; + + public final byte[] response; + public final byte[] nonce; + public final byte[] inputIv; + public final byte[] outputIv; + + public ServerResponse( + byte[] response, + byte[] nonce, + byte[] inputIv, + byte[] outputIv) { + this.response = response; + this.nonce = nonce; + this.inputIv = inputIv; + this.outputIv = outputIv; + } + + @Override + public int encodedLength() { + return 1 + + Encoders.ByteArrays.encodedLength(response) + + Encoders.ByteArrays.encodedLength(nonce) + + Encoders.ByteArrays.encodedLength(inputIv) + + Encoders.ByteArrays.encodedLength(outputIv); + } + + @Override + public void encode(ByteBuf buf) { + buf.writeByte(TAG_BYTE); + Encoders.ByteArrays.encode(buf, response); + Encoders.ByteArrays.encode(buf, nonce); + Encoders.ByteArrays.encode(buf, inputIv); + Encoders.ByteArrays.encode(buf, outputIv); + } + + public static ServerResponse decodeMessage(ByteBuffer buffer) { + ByteBuf buf = Unpooled.wrappedBuffer(buffer); + + if (buf.readByte() != TAG_BYTE) { + throw new IllegalArgumentException("Expected ServerResponse, received something else."); + } + + return new ServerResponse( + Encoders.ByteArrays.decode(buf), + Encoders.ByteArrays.decode(buf), + Encoders.ByteArrays.decode(buf), + Encoders.ByteArrays.decode(buf)); + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java new file mode 100644 index 000000000000..7376d1ddc481 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.crypto; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; +import java.util.Properties; +import javax.crypto.spec.SecretKeySpec; +import javax.crypto.spec.IvParameterSpec; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.*; +import io.netty.util.AbstractReferenceCounted; +import org.apache.commons.crypto.stream.CryptoInputStream; +import org.apache.commons.crypto.stream.CryptoOutputStream; + +import org.apache.spark.network.util.ByteArrayReadableChannel; +import org.apache.spark.network.util.ByteArrayWritableChannel; + +/** + * Cipher for encryption and decryption. + */ +public class TransportCipher { + @VisibleForTesting + static final String ENCRYPTION_HANDLER_NAME = "TransportEncryption"; + private static final String DECRYPTION_HANDLER_NAME = "TransportDecryption"; + private static final int STREAM_BUFFER_SIZE = 1024 * 32; + + private final Properties conf; + private final String cipher; + private final SecretKeySpec key; + private final byte[] inIv; + private final byte[] outIv; + + public TransportCipher( + Properties conf, + String cipher, + SecretKeySpec key, + byte[] inIv, + byte[] outIv) { + this.conf = conf; + this.cipher = cipher; + this.key = key; + this.inIv = inIv; + this.outIv = outIv; + } + + public String getCipherTransformation() { + return cipher; + } + + @VisibleForTesting + SecretKeySpec getKey() { + return key; + } + + /** The IV for the input channel (i.e. output channel of the remote side). */ + public byte[] getInputIv() { + return inIv; + } + + /** The IV for the output channel (i.e. input channel of the remote side). */ + public byte[] getOutputIv() { + return outIv; + } + + private CryptoOutputStream createOutputStream(WritableByteChannel ch) throws IOException { + return new CryptoOutputStream(cipher, conf, ch, key, new IvParameterSpec(outIv)); + } + + private CryptoInputStream createInputStream(ReadableByteChannel ch) throws IOException { + return new CryptoInputStream(cipher, conf, ch, key, new IvParameterSpec(inIv)); + } + + /** + * Add handlers to channel. + * + * @param ch the channel for adding handlers + * @throws IOException + */ + public void addToChannel(Channel ch) throws IOException { + ch.pipeline() + .addFirst(ENCRYPTION_HANDLER_NAME, new EncryptionHandler(this)) + .addFirst(DECRYPTION_HANDLER_NAME, new DecryptionHandler(this)); + } + + private static class EncryptionHandler extends ChannelOutboundHandlerAdapter { + private final ByteArrayWritableChannel byteChannel; + private final CryptoOutputStream cos; + + EncryptionHandler(TransportCipher cipher) throws IOException { + byteChannel = new ByteArrayWritableChannel(STREAM_BUFFER_SIZE); + cos = cipher.createOutputStream(byteChannel); + } + + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) + throws Exception { + ctx.write(new EncryptedMessage(cos, msg, byteChannel), promise); + } + + @Override + public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { + try { + cos.close(); + } finally { + super.close(ctx, promise); + } + } + } + + private static class DecryptionHandler extends ChannelInboundHandlerAdapter { + private final CryptoInputStream cis; + private final ByteArrayReadableChannel byteChannel; + + DecryptionHandler(TransportCipher cipher) throws IOException { + byteChannel = new ByteArrayReadableChannel(); + cis = cipher.createInputStream(byteChannel); + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception { + byteChannel.feedData((ByteBuf) data); + + byte[] decryptedData = new byte[byteChannel.readableBytes()]; + int offset = 0; + while (offset < decryptedData.length) { + offset += cis.read(decryptedData, offset, decryptedData.length - offset); + } + + ctx.fireChannelRead(Unpooled.wrappedBuffer(decryptedData, 0, decryptedData.length)); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + try { + cis.close(); + } finally { + super.channelInactive(ctx); + } + } + } + + private static class EncryptedMessage extends AbstractReferenceCounted implements FileRegion { + private final boolean isByteBuf; + private final ByteBuf buf; + private final FileRegion region; + private long transferred; + private CryptoOutputStream cos; + + // Due to streaming issue CRYPTO-125: https://issues.apache.org/jira/browse/CRYPTO-125, it has + // to utilize two helper ByteArrayWritableChannel for streaming. One is used to receive raw data + // from upper handler, another is used to store encrypted data. + private ByteArrayWritableChannel byteEncChannel; + private ByteArrayWritableChannel byteRawChannel; + + private ByteBuffer currentEncrypted; + + EncryptedMessage(CryptoOutputStream cos, Object msg, ByteArrayWritableChannel ch) { + Preconditions.checkArgument(msg instanceof ByteBuf || msg instanceof FileRegion, + "Unrecognized message type: %s", msg.getClass().getName()); + this.isByteBuf = msg instanceof ByteBuf; + this.buf = isByteBuf ? (ByteBuf) msg : null; + this.region = isByteBuf ? null : (FileRegion) msg; + this.transferred = 0; + this.byteRawChannel = new ByteArrayWritableChannel(STREAM_BUFFER_SIZE); + this.cos = cos; + this.byteEncChannel = ch; + } + + @Override + public long count() { + return isByteBuf ? buf.readableBytes() : region.count(); + } + + @Override + public long position() { + return 0; + } + + @Override + public long transfered() { + return transferred; + } + + @Override + public long transferTo(WritableByteChannel target, long position) throws IOException { + Preconditions.checkArgument(position == transfered(), "Invalid position."); + + do { + if (currentEncrypted == null) { + encryptMore(); + } + + int bytesWritten = currentEncrypted.remaining(); + target.write(currentEncrypted); + bytesWritten -= currentEncrypted.remaining(); + transferred += bytesWritten; + if (!currentEncrypted.hasRemaining()) { + currentEncrypted = null; + byteEncChannel.reset(); + } + } while (transferred < count()); + + return transferred; + } + + private void encryptMore() throws IOException { + byteRawChannel.reset(); + + if (isByteBuf) { + int copied = byteRawChannel.write(buf.nioBuffer()); + buf.skipBytes(copied); + } else { + region.transferTo(byteRawChannel, region.transfered()); + } + cos.write(byteRawChannel.getData(), 0, byteRawChannel.length()); + cos.flush(); + + currentEncrypted = ByteBuffer.wrap(byteEncChannel.getData(), + 0, byteEncChannel.length()); + } + + @Override + protected void deallocate() { + byteRawChannel.reset(); + byteEncChannel.reset(); + if (region != null) { + region.release(); + } + if (buf != null) { + buf.release(); + } + } + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/AbstractMessage.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/AbstractMessage.java new file mode 100644 index 000000000000..2924218c2f08 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/AbstractMessage.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import com.google.common.base.Objects; + +import org.apache.spark.network.buffer.ManagedBuffer; + +/** + * Abstract class for messages which optionally contain a body kept in a separate buffer. + */ +public abstract class AbstractMessage implements Message { + private final ManagedBuffer body; + private final boolean isBodyInFrame; + + protected AbstractMessage() { + this(null, false); + } + + protected AbstractMessage(ManagedBuffer body, boolean isBodyInFrame) { + this.body = body; + this.isBodyInFrame = isBodyInFrame; + } + + @Override + public ManagedBuffer body() { + return body; + } + + @Override + public boolean isBodyInFrame() { + return isBodyInFrame; + } + + protected boolean equals(AbstractMessage other) { + return isBodyInFrame == other.isBodyInFrame && Objects.equal(body, other.body); + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/AbstractResponseMessage.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/AbstractResponseMessage.java new file mode 100644 index 000000000000..c362c92fc4f5 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/AbstractResponseMessage.java @@ -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.network.protocol; + +import org.apache.spark.network.buffer.ManagedBuffer; + +/** + * Abstract class for response messages. + */ +public abstract class AbstractResponseMessage extends AbstractMessage implements ResponseMessage { + + protected AbstractResponseMessage(ManagedBuffer body, boolean isBodyInFrame) { + super(body, isBodyInFrame); + } + + public abstract ResponseMessage createFailureResponse(String error); +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java similarity index 96% rename from network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java index f0363830b61a..7b28a9a96948 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java @@ -23,7 +23,7 @@ /** * Response to {@link ChunkFetchRequest} when there is an error fetching the chunk. */ -public final class ChunkFetchFailure implements ResponseMessage { +public final class ChunkFetchFailure extends AbstractMessage implements ResponseMessage { public final StreamChunkId streamChunkId; public final String errorString; diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java similarity index 95% rename from network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java index 5a173af54f61..26d063feb5fe 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java @@ -24,7 +24,7 @@ * Request to fetch a sequence of a single chunk of a stream. This will correspond to a single * {@link org.apache.spark.network.protocol.ResponseMessage} (either success or failure). */ -public final class ChunkFetchRequest implements RequestMessage { +public final class ChunkFetchRequest extends AbstractMessage implements RequestMessage { public final StreamChunkId streamChunkId; public ChunkFetchRequest(StreamChunkId streamChunkId) { diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java similarity index 92% rename from network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java index e6a7e9a8b414..94c2ac9b20e4 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java @@ -30,7 +30,7 @@ * may be written by Netty in a more efficient manner (i.e., zero-copy write). * Similarly, the client-side decoding will reuse the Netty ByteBuf as the buffer. */ -public final class ChunkFetchSuccess extends ResponseWithBody { +public final class ChunkFetchSuccess extends AbstractResponseMessage { public final StreamChunkId streamChunkId; public ChunkFetchSuccess(StreamChunkId streamChunkId, ManagedBuffer buffer) { @@ -67,14 +67,14 @@ public static ChunkFetchSuccess decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode(streamChunkId, body); + return Objects.hashCode(streamChunkId, body()); } @Override public boolean equals(Object other) { if (other instanceof ChunkFetchSuccess) { ChunkFetchSuccess o = (ChunkFetchSuccess) other; - return streamChunkId.equals(o.streamChunkId) && body.equals(o.body); + return streamChunkId.equals(o.streamChunkId) && super.equals(o); } return false; } @@ -83,7 +83,7 @@ public boolean equals(Object other) { public String toString() { return Objects.toStringHelper(this) .add("streamChunkId", streamChunkId) - .add("buffer", body) + .add("buffer", body()) .toString(); } } diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/Encodable.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encodable.java similarity index 100% rename from network/common/src/main/java/org/apache/spark/network/protocol/Encodable.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/Encodable.java diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java similarity index 92% rename from network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java index 9162d0b977f8..be217522367c 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java @@ -17,8 +17,8 @@ package org.apache.spark.network.protocol; +import java.nio.charset.StandardCharsets; -import com.google.common.base.Charsets; import io.netty.buffer.ByteBuf; /** Provides a canonical set of Encoders for simple types. */ @@ -27,11 +27,11 @@ public class Encoders { /** Strings are encoded with their length followed by UTF-8 bytes. */ public static class Strings { public static int encodedLength(String s) { - return 4 + s.getBytes(Charsets.UTF_8).length; + return 4 + s.getBytes(StandardCharsets.UTF_8).length; } public static void encode(ByteBuf buf, String s) { - byte[] bytes = s.getBytes(Charsets.UTF_8); + byte[] bytes = s.getBytes(StandardCharsets.UTF_8); buf.writeInt(bytes.length); buf.writeBytes(bytes); } @@ -40,7 +40,7 @@ public static String decode(ByteBuf buf) { int length = buf.readInt(); byte[] bytes = new byte[length]; buf.readBytes(bytes); - return new String(bytes, Charsets.UTF_8); + return new String(bytes, StandardCharsets.UTF_8); } } diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/Message.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java similarity index 80% rename from network/common/src/main/java/org/apache/spark/network/protocol/Message.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java index d01598c20f16..434935a8ef2a 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/Message.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java @@ -19,20 +19,29 @@ import io.netty.buffer.ByteBuf; +import org.apache.spark.network.buffer.ManagedBuffer; + /** An on-the-wire transmittable message. */ public interface Message extends Encodable { /** Used to identify this request type. */ Type type(); + /** An optional body for the message. */ + ManagedBuffer body(); + + /** Whether to include the body of the message in the same frame as the message. */ + boolean isBodyInFrame(); + /** Preceding every serialized Message is its type, which allows us to deserialize it. */ - public static enum Type implements Encodable { + enum Type implements Encodable { ChunkFetchRequest(0), ChunkFetchSuccess(1), ChunkFetchFailure(2), RpcRequest(3), RpcResponse(4), RpcFailure(5), - StreamRequest(6), StreamResponse(7), StreamFailure(8); + StreamRequest(6), StreamResponse(7), StreamFailure(8), + OneWayMessage(9), User(-1); private final byte id; - private Type(int id) { + Type(int id) { assert id < 128 : "Cannot have more than 128 message types"; this.id = (byte) id; } @@ -55,6 +64,8 @@ public static Type decode(ByteBuf buf) { case 6: return StreamRequest; case 7: return StreamResponse; case 8: return StreamFailure; + case 9: return OneWayMessage; + case -1: throw new IllegalArgumentException("User type messages cannot be decoded."); default: throw new IllegalArgumentException("Unknown message type: " + id); } } diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java similarity index 88% rename from network/common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java index 3c04048f3821..39a7495828a8 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java @@ -33,13 +33,18 @@ @ChannelHandler.Sharable public final class MessageDecoder extends MessageToMessageDecoder { - private final Logger logger = LoggerFactory.getLogger(MessageDecoder.class); + private static final Logger logger = LoggerFactory.getLogger(MessageDecoder.class); + + public static final MessageDecoder INSTANCE = new MessageDecoder(); + + private MessageDecoder() {} + @Override public void decode(ChannelHandlerContext ctx, ByteBuf in, List out) { Message.Type msgType = Message.Type.decode(in); Message decoded = decode(msgType, in); assert decoded.type() == msgType; - logger.trace("Received message " + msgType + ": " + decoded); + logger.trace("Received message {}: {}", msgType, decoded); out.add(decoded); } @@ -63,6 +68,9 @@ private Message decode(Message.Type msgType, ByteBuf in) { case RpcFailure: return RpcFailure.decode(in); + case OneWayMessage: + return OneWayMessage.decode(in); + case StreamRequest: return StreamRequest.decode(in); diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java new file mode 100644 index 000000000000..997f74e1a21b --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.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.network.protocol; + +import java.util.List; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.MessageToMessageEncoder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Encoder used by the server side to encode server-to-client responses. + * This encoder is stateless so it is safe to be shared by multiple threads. + */ +@ChannelHandler.Sharable +public final class MessageEncoder extends MessageToMessageEncoder { + + private static final Logger logger = LoggerFactory.getLogger(MessageEncoder.class); + + public static final MessageEncoder INSTANCE = new MessageEncoder(); + + private MessageEncoder() {} + + /*** + * Encodes a Message by invoking its encode() method. For non-data messages, we will add one + * ByteBuf to 'out' containing the total frame length, the message type, and the message itself. + * In the case of a ChunkFetchSuccess, we will also add the ManagedBuffer corresponding to the + * data to 'out', in order to enable zero-copy transfer. + */ + @Override + public void encode(ChannelHandlerContext ctx, Message in, List out) throws Exception { + Object body = null; + long bodyLength = 0; + boolean isBodyInFrame = false; + + // If the message has a body, take it out to enable zero-copy transfer for the payload. + if (in.body() != null) { + try { + bodyLength = in.body().size(); + body = in.body().convertToNetty(); + isBodyInFrame = in.isBodyInFrame(); + } catch (Exception e) { + in.body().release(); + if (in instanceof AbstractResponseMessage) { + AbstractResponseMessage resp = (AbstractResponseMessage) in; + // Re-encode this message as a failure response. + String error = e.getMessage() != null ? e.getMessage() : "null"; + logger.error(String.format("Error processing %s for client %s", + in, ctx.channel().remoteAddress()), e); + encode(ctx, resp.createFailureResponse(error), out); + } else { + throw e; + } + return; + } + } + + Message.Type msgType = in.type(); + // All messages have the frame length, message type, and message itself. The frame length + // may optionally include the length of the body data, depending on what message is being + // sent. + int headerLength = 8 + msgType.encodedLength() + in.encodedLength(); + long frameLength = headerLength + (isBodyInFrame ? bodyLength : 0); + ByteBuf header = ctx.alloc().heapBuffer(headerLength); + header.writeLong(frameLength); + msgType.encode(header); + in.encode(header); + assert header.writableBytes() == 0; + + if (body != null) { + // We transfer ownership of the reference on in.body() to MessageWithHeader. + // This reference will be freed when MessageWithHeader.deallocate() is called. + out.add(new MessageWithHeader(in.body(), header, body, bodyLength)); + } else { + out.add(header); + } + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java new file mode 100644 index 000000000000..4f8781b42a0e --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java @@ -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.network.protocol; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import javax.annotation.Nullable; + +import com.google.common.base.Preconditions; +import io.netty.buffer.ByteBuf; +import io.netty.channel.FileRegion; +import io.netty.util.AbstractReferenceCounted; +import io.netty.util.ReferenceCountUtil; + +import org.apache.spark.network.buffer.ManagedBuffer; + +/** + * A wrapper message that holds two separate pieces (a header and a body). + * + * The header must be a ByteBuf, while the body can be a ByteBuf or a FileRegion. + */ +class MessageWithHeader extends AbstractReferenceCounted implements FileRegion { + + @Nullable private final ManagedBuffer managedBuffer; + private final ByteBuf header; + private final int headerLength; + private final Object body; + private final long bodyLength; + private long totalBytesTransferred; + + /** + * When the write buffer size is larger than this limit, I/O will be done in chunks of this size. + * The size should not be too large as it will waste underlying memory copy. e.g. If network + * avaliable buffer is smaller than this limit, the data cannot be sent within one single write + * operation while it still will make memory copy with this size. + */ + private static final int NIO_BUFFER_LIMIT = 256 * 1024; + + /** + * Construct a new MessageWithHeader. + * + * @param managedBuffer the {@link ManagedBuffer} that the message body came from. This needs to + * be passed in so that the buffer can be freed when this message is + * deallocated. Ownership of the caller's reference to this buffer is + * transferred to this class, so if the caller wants to continue to use the + * ManagedBuffer in other messages then they will need to call retain() on + * it before passing it to this constructor. This may be null if and only if + * `body` is a {@link FileRegion}. + * @param header the message header. + * @param body the message body. Must be either a {@link ByteBuf} or a {@link FileRegion}. + * @param bodyLength the length of the message body, in bytes. + */ + MessageWithHeader( + @Nullable ManagedBuffer managedBuffer, + ByteBuf header, + Object body, + long bodyLength) { + Preconditions.checkArgument(body instanceof ByteBuf || body instanceof FileRegion, + "Body must be a ByteBuf or a FileRegion."); + this.managedBuffer = managedBuffer; + this.header = header; + this.headerLength = header.readableBytes(); + this.body = body; + this.bodyLength = bodyLength; + } + + @Override + public long count() { + return headerLength + bodyLength; + } + + @Override + public long position() { + return 0; + } + + @Override + public long transfered() { + return totalBytesTransferred; + } + + /** + * This code is more complicated than you would think because we might require multiple + * transferTo invocations in order to transfer a single MessageWithHeader to avoid busy waiting. + * + * The contract is that the caller will ensure position is properly set to the total number + * of bytes transferred so far (i.e. value returned by transfered()). + */ + @Override + public long transferTo(final WritableByteChannel target, final long position) throws IOException { + Preconditions.checkArgument(position == totalBytesTransferred, "Invalid position."); + // Bytes written for header in this call. + long writtenHeader = 0; + if (header.readableBytes() > 0) { + writtenHeader = copyByteBuf(header, target); + totalBytesTransferred += writtenHeader; + if (header.readableBytes() > 0) { + return writtenHeader; + } + } + + // Bytes written for body in this call. + long writtenBody = 0; + if (body instanceof FileRegion) { + writtenBody = ((FileRegion) body).transferTo(target, totalBytesTransferred - headerLength); + } else if (body instanceof ByteBuf) { + writtenBody = copyByteBuf((ByteBuf) body, target); + } + totalBytesTransferred += writtenBody; + + return writtenHeader + writtenBody; + } + + @Override + protected void deallocate() { + header.release(); + ReferenceCountUtil.release(body); + if (managedBuffer != null) { + managedBuffer.release(); + } + } + + private int copyByteBuf(ByteBuf buf, WritableByteChannel target) throws IOException { + ByteBuffer buffer = buf.nioBuffer(); + int written = (buffer.remaining() <= NIO_BUFFER_LIMIT) ? + target.write(buffer) : writeNioBuffer(target, buffer); + buf.skipBytes(written); + return written; + } + + private int writeNioBuffer( + WritableByteChannel writeCh, + ByteBuffer buf) throws IOException { + int originalLimit = buf.limit(); + int ret = 0; + + try { + int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT); + buf.limit(buf.position() + ioSize); + ret = writeCh.write(buf); + } finally { + buf.limit(originalLimit); + } + + return ret; + } +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java new file mode 100644 index 000000000000..f7ffb1bd49bb --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NettyManagedBuffer; + +/** + * A RPC that does not expect a reply, which is handled by a remote + * {@link org.apache.spark.network.server.RpcHandler}. + */ +public final class OneWayMessage extends AbstractMessage implements RequestMessage { + + public OneWayMessage(ManagedBuffer body) { + super(body, true); + } + + @Override + public Type type() { return Type.OneWayMessage; } + + @Override + public int encodedLength() { + // The integer (a.k.a. the body size) is not really used, since that information is already + // encoded in the frame length. But this maintains backwards compatibility with versions of + // RpcRequest that use Encoders.ByteArrays. + return 4; + } + + @Override + public void encode(ByteBuf buf) { + // See comment in encodedLength(). + buf.writeInt((int) body().size()); + } + + public static OneWayMessage decode(ByteBuf buf) { + // See comment in encodedLength(). + buf.readInt(); + return new OneWayMessage(new NettyManagedBuffer(buf.retain())); + } + + @Override + public int hashCode() { + return Objects.hashCode(body()); + } + + @Override + public boolean equals(Object other) { + if (other instanceof OneWayMessage) { + OneWayMessage o = (OneWayMessage) other; + return super.equals(o); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("body", body()) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java similarity index 94% rename from network/common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java index 31b15bb17a32..b85171ed6f3d 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java @@ -17,8 +17,6 @@ package org.apache.spark.network.protocol; -import org.apache.spark.network.protocol.Message; - /** Messages from the client to the server. */ public interface RequestMessage extends Message { // token interface diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java similarity index 94% rename from network/common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java index 6edffd11cf1e..194e6d9aa2bd 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java @@ -17,8 +17,6 @@ package org.apache.spark.network.protocol; -import org.apache.spark.network.protocol.Message; - /** Messages from the server to the client. */ public interface ResponseMessage extends Message { // token interface diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java similarity index 96% rename from network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java index 2dfc7876ba32..a76624ef5dc9 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java @@ -21,7 +21,7 @@ import io.netty.buffer.ByteBuf; /** Response to {@link RpcRequest} for a failed RPC. */ -public final class RpcFailure implements ResponseMessage { +public final class RpcFailure extends AbstractMessage implements ResponseMessage { public final long requestId; public final String errorString; diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java new file mode 100644 index 000000000000..2b30920f0598 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NettyManagedBuffer; + +/** + * A generic RPC which is handled by a remote {@link org.apache.spark.network.server.RpcHandler}. + * This will correspond to a single + * {@link org.apache.spark.network.protocol.ResponseMessage} (either success or failure). + */ +public final class RpcRequest extends AbstractMessage implements RequestMessage { + /** Used to link an RPC request with its response. */ + public final long requestId; + + public RpcRequest(long requestId, ManagedBuffer message) { + super(message, true); + this.requestId = requestId; + } + + @Override + public Type type() { return Type.RpcRequest; } + + @Override + public int encodedLength() { + // The integer (a.k.a. the body size) is not really used, since that information is already + // encoded in the frame length. But this maintains backwards compatibility with versions of + // RpcRequest that use Encoders.ByteArrays. + return 8 + 4; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + // See comment in encodedLength(). + buf.writeInt((int) body().size()); + } + + public static RpcRequest decode(ByteBuf buf) { + long requestId = buf.readLong(); + // See comment in encodedLength(). + buf.readInt(); + return new RpcRequest(requestId, new NettyManagedBuffer(buf.retain())); + } + + @Override + public int hashCode() { + return Objects.hashCode(requestId, body()); + } + + @Override + public boolean equals(Object other) { + if (other instanceof RpcRequest) { + RpcRequest o = (RpcRequest) other; + return requestId == o.requestId && super.equals(o); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("requestId", requestId) + .add("body", body()) + .toString(); + } +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java new file mode 100644 index 000000000000..d73014ecd850 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NettyManagedBuffer; + +/** Response to {@link RpcRequest} for a successful RPC. */ +public final class RpcResponse extends AbstractResponseMessage { + public final long requestId; + + public RpcResponse(long requestId, ManagedBuffer message) { + super(message, true); + this.requestId = requestId; + } + + @Override + public Type type() { return Type.RpcResponse; } + + @Override + public int encodedLength() { + // The integer (a.k.a. the body size) is not really used, since that information is already + // encoded in the frame length. But this maintains backwards compatibility with versions of + // RpcRequest that use Encoders.ByteArrays. + return 8 + 4; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + // See comment in encodedLength(). + buf.writeInt((int) body().size()); + } + + @Override + public ResponseMessage createFailureResponse(String error) { + return new RpcFailure(requestId, error); + } + + public static RpcResponse decode(ByteBuf buf) { + long requestId = buf.readLong(); + // See comment in encodedLength(). + buf.readInt(); + return new RpcResponse(requestId, new NettyManagedBuffer(buf.retain())); + } + + @Override + public int hashCode() { + return Objects.hashCode(requestId, body()); + } + + @Override + public boolean equals(Object other) { + if (other instanceof RpcResponse) { + RpcResponse o = (RpcResponse) other; + return requestId == o.requestId && super.equals(o); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("requestId", requestId) + .add("body", body()) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java similarity index 100% rename from network/common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java similarity index 92% rename from network/common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java index e3dade2ebf90..258ef81c6783 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java @@ -20,13 +20,10 @@ import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.buffer.NettyManagedBuffer; - /** * Message indicating an error when transferring a stream. */ -public final class StreamFailure implements ResponseMessage { +public final class StreamFailure extends AbstractMessage implements ResponseMessage { public final String streamId; public final String error; diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java similarity index 92% rename from network/common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java index 821e8f53884d..dc183c043ed9 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java @@ -20,16 +20,13 @@ import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.buffer.NettyManagedBuffer; - /** * Request to stream data from the remote end. *

* The stream ID is an arbitrary string that needs to be negotiated between the two endpoints before * the data can be streamed. */ -public final class StreamRequest implements RequestMessage { +public final class StreamRequest extends AbstractMessage implements RequestMessage { public final String streamId; public StreamRequest(String streamId) { diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java similarity index 85% rename from network/common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java rename to common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java index ac5ab9a323a1..87e212f3e157 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java @@ -21,7 +21,6 @@ import io.netty.buffer.ByteBuf; import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.buffer.NettyManagedBuffer; /** * Response to {@link StreamRequest} when the stream has been successfully opened. @@ -30,15 +29,15 @@ * sender. The receiver is expected to set a temporary channel handler that will consume the * number of bytes this message says the stream has. */ -public final class StreamResponse extends ResponseWithBody { - public final String streamId; - public final long byteCount; +public final class StreamResponse extends AbstractResponseMessage { + public final String streamId; + public final long byteCount; - public StreamResponse(String streamId, long byteCount, ManagedBuffer buffer) { - super(buffer, false); - this.streamId = streamId; - this.byteCount = byteCount; - } + public StreamResponse(String streamId, long byteCount, ManagedBuffer buffer) { + super(buffer, false); + this.streamId = streamId; + this.byteCount = byteCount; + } @Override public Type type() { return Type.StreamResponse; } @@ -68,7 +67,7 @@ public static StreamResponse decode(ByteBuf buf) { @Override public int hashCode() { - return Objects.hashCode(byteCount, streamId); + return Objects.hashCode(byteCount, streamId, body()); } @Override @@ -85,6 +84,7 @@ public String toString() { return Objects.toStringHelper(this) .add("streamId", streamId) .add("byteCount", byteCount) + .add("body", body()) .toString(); } diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java similarity index 81% rename from network/common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java rename to common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java index 69923769d44b..647813772294 100644 --- a/network/common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java +++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java @@ -17,6 +17,8 @@ package org.apache.spark.network.sasl; +import java.io.IOException; +import java.nio.ByteBuffer; import javax.security.sasl.Sasl; import javax.security.sasl.SaslException; @@ -28,6 +30,7 @@ import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientBootstrap; +import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.TransportConf; /** @@ -35,26 +38,16 @@ * server should be setup with a {@link SaslRpcHandler} with matching keys for the given appId. */ public class SaslClientBootstrap implements TransportClientBootstrap { - private final Logger logger = LoggerFactory.getLogger(SaslClientBootstrap.class); + private static final Logger logger = LoggerFactory.getLogger(SaslClientBootstrap.class); - private final boolean encrypt; private final TransportConf conf; private final String appId; private final SecretKeyHolder secretKeyHolder; public SaslClientBootstrap(TransportConf conf, String appId, SecretKeyHolder secretKeyHolder) { - this(conf, appId, secretKeyHolder, false); - } - - public SaslClientBootstrap( - TransportConf conf, - String appId, - SecretKeyHolder secretKeyHolder, - boolean encrypt) { this.conf = conf; this.appId = appId; this.secretKeyHolder = secretKeyHolder; - this.encrypt = encrypt; } /** @@ -64,30 +57,34 @@ public SaslClientBootstrap( */ @Override public void doBootstrap(TransportClient client, Channel channel) { - SparkSaslClient saslClient = new SparkSaslClient(appId, secretKeyHolder, encrypt); + SparkSaslClient saslClient = new SparkSaslClient(appId, secretKeyHolder, conf.saslEncryption()); try { byte[] payload = saslClient.firstToken(); while (!saslClient.isComplete()) { SaslMessage msg = new SaslMessage(appId, payload); - ByteBuf buf = Unpooled.buffer(msg.encodedLength()); + ByteBuf buf = Unpooled.buffer(msg.encodedLength() + (int) msg.body().size()); msg.encode(buf); + buf.writeBytes(msg.body().nioByteBuffer()); - byte[] response = client.sendRpcSync(buf.array(), conf.saslRTTimeoutMs()); - payload = saslClient.response(response); + ByteBuffer response = client.sendRpcSync(buf.nioBuffer(), conf.authRTTimeoutMs()); + payload = saslClient.response(JavaUtils.bufferToArray(response)); } client.setClientId(appId); - if (encrypt) { + if (conf.saslEncryption()) { if (!SparkSaslServer.QOP_AUTH_CONF.equals(saslClient.getNegotiatedProperty(Sasl.QOP))) { throw new RuntimeException( new SaslException("Encryption requests by negotiated non-encrypted connection.")); } + SaslEncryption.addToChannel(channel, saslClient, conf.maxSaslEncryptedBlockSize()); saslClient = null; - logger.debug("Channel {} configured for SASL encryption.", client); + logger.debug("Channel {} configured for encryption.", client); } + } catch (IOException ioe) { + throw new RuntimeException(ioe); } finally { if (saslClient != null) { try { diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java similarity index 99% rename from network/common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java rename to common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java index 127335e4d35f..3d71ebaa7ea0 100644 --- a/network/common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java +++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java @@ -33,7 +33,6 @@ import io.netty.channel.FileRegion; import io.netty.handler.codec.MessageToMessageDecoder; import io.netty.util.AbstractReferenceCounted; -import io.netty.util.ReferenceCountUtil; import org.apache.spark.network.util.ByteArrayWritableChannel; import org.apache.spark.network.util.NettyUtils; diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SaslEncryptionBackend.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryptionBackend.java similarity index 100% rename from network/common/src/main/java/org/apache/spark/network/sasl/SaslEncryptionBackend.java rename to common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryptionBackend.java diff --git a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java new file mode 100644 index 000000000000..7331c2b481fb --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.sasl; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + +import org.apache.spark.network.buffer.NettyManagedBuffer; +import org.apache.spark.network.protocol.Encoders; +import org.apache.spark.network.protocol.AbstractMessage; + +/** + * Encodes a Sasl-related message which is attempting to authenticate using some credentials tagged + * with the given appId. This appId allows a single SaslRpcHandler to multiplex different + * applications which may be using different sets of credentials. + */ +class SaslMessage extends AbstractMessage { + + /** Serialization tag used to catch incorrect payloads. */ + private static final byte TAG_BYTE = (byte) 0xEA; + + public final String appId; + + SaslMessage(String appId, byte[] message) { + this(appId, Unpooled.wrappedBuffer(message)); + } + + SaslMessage(String appId, ByteBuf message) { + super(new NettyManagedBuffer(message), true); + this.appId = appId; + } + + @Override + public Type type() { return Type.User; } + + @Override + public int encodedLength() { + // The integer (a.k.a. the body size) is not really used, since that information is already + // encoded in the frame length. But this maintains backwards compatibility with versions of + // RpcRequest that use Encoders.ByteArrays. + return 1 + Encoders.Strings.encodedLength(appId) + 4; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeByte(TAG_BYTE); + Encoders.Strings.encode(buf, appId); + // See comment in encodedLength(). + buf.writeInt((int) body().size()); + } + + public static SaslMessage decode(ByteBuf buf) { + if (buf.readByte() != TAG_BYTE) { + throw new IllegalStateException("Expected SaslMessage, received something else" + + " (maybe your client does not have SASL enabled?)"); + } + + String appId = Encoders.Strings.decode(buf); + // See comment in encodedLength(). + buf.readInt(); + return new SaslMessage(appId, buf.retain()); + } +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java new file mode 100644 index 000000000000..0231428318ad --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.sasl; + +import java.io.IOException; +import java.nio.ByteBuffer; +import javax.security.sasl.Sasl; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.TransportConf; + +/** + * RPC Handler which performs SASL authentication before delegating to a child RPC handler. + * The delegate will only receive messages if the given connection has been successfully + * authenticated. A connection may be authenticated at most once. + * + * Note that the authentication process consists of multiple challenge-response pairs, each of + * which are individual RPCs. + */ +public class SaslRpcHandler extends RpcHandler { + private static final Logger logger = LoggerFactory.getLogger(SaslRpcHandler.class); + + /** Transport configuration. */ + private final TransportConf conf; + + /** The client channel. */ + private final Channel channel; + + /** RpcHandler we will delegate to for authenticated connections. */ + private final RpcHandler delegate; + + /** Class which provides secret keys which are shared by server and client on a per-app basis. */ + private final SecretKeyHolder secretKeyHolder; + + private SparkSaslServer saslServer; + private boolean isComplete; + private boolean isAuthenticated; + + public SaslRpcHandler( + TransportConf conf, + Channel channel, + RpcHandler delegate, + SecretKeyHolder secretKeyHolder) { + this.conf = conf; + this.channel = channel; + this.delegate = delegate; + this.secretKeyHolder = secretKeyHolder; + this.saslServer = null; + this.isComplete = false; + this.isAuthenticated = false; + } + + @Override + public void receive(TransportClient client, ByteBuffer message, RpcResponseCallback callback) { + if (isComplete) { + // Authentication complete, delegate to base handler. + delegate.receive(client, message, callback); + return; + } + if (saslServer == null || !saslServer.isComplete()) { + ByteBuf nettyBuf = Unpooled.wrappedBuffer(message); + SaslMessage saslMessage; + try { + saslMessage = SaslMessage.decode(nettyBuf); + } finally { + nettyBuf.release(); + } + + if (saslServer == null) { + // First message in the handshake, setup the necessary state. + client.setClientId(saslMessage.appId); + saslServer = new SparkSaslServer(saslMessage.appId, secretKeyHolder, + conf.saslServerAlwaysEncrypt()); + } + + byte[] response; + try { + response = saslServer.response(JavaUtils.bufferToArray( + saslMessage.body().nioByteBuffer())); + } catch (IOException ioe) { + throw new RuntimeException(ioe); + } + callback.onSuccess(ByteBuffer.wrap(response)); + } + + // Setup encryption after the SASL response is sent, otherwise the client can't parse the + // response. It's ok to change the channel pipeline here since we are processing an incoming + // message, so the pipeline is busy and no new incoming messages will be fed to it before this + // method returns. This assumes that the code ensures, through other means, that no outbound + // messages are being written to the channel while negotiation is still going on. + if (saslServer.isComplete()) { + if (!SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP))) { + logger.debug("SASL authentication successful for channel {}", client); + complete(true); + return; + } + + logger.debug("Enabling encryption for channel {}", client); + SaslEncryption.addToChannel(channel, saslServer, conf.maxSaslEncryptedBlockSize()); + complete(false); + return; + } + } + + @Override + public void receive(TransportClient client, ByteBuffer message) { + delegate.receive(client, message); + } + + @Override + public StreamManager getStreamManager() { + return delegate.getStreamManager(); + } + + @Override + public void channelActive(TransportClient client) { + delegate.channelActive(client); + } + + @Override + public void channelInactive(TransportClient client) { + try { + delegate.channelInactive(client); + } finally { + if (saslServer != null) { + saslServer.dispose(); + } + } + } + + @Override + public void exceptionCaught(Throwable cause, TransportClient client) { + delegate.exceptionCaught(cause, client); + } + + private void complete(boolean dispose) { + if (dispose) { + try { + saslServer.dispose(); + } catch (RuntimeException e) { + logger.error("Error while disposing SASL server", e); + } + } + + saslServer = null; + isComplete = true; + } + +} diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SaslServerBootstrap.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslServerBootstrap.java similarity index 100% rename from network/common/src/main/java/org/apache/spark/network/sasl/SaslServerBootstrap.java rename to common/network-common/src/main/java/org/apache/spark/network/sasl/SaslServerBootstrap.java diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java similarity index 100% rename from network/common/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java rename to common/network-common/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java similarity index 96% rename from network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java rename to common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java index 94685e91b862..05a5afe195e8 100644 --- a/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java +++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java @@ -17,7 +17,6 @@ package org.apache.spark.network.sasl; -import java.io.IOException; import java.util.Map; import javax.security.auth.callback.Callback; import javax.security.auth.callback.CallbackHandler; @@ -43,7 +42,7 @@ * firstToken, which is then followed by a set of challenges and responses. */ public class SparkSaslClient implements SaslEncryptionBackend { - private final Logger logger = LoggerFactory.getLogger(SparkSaslClient.class); + private static final Logger logger = LoggerFactory.getLogger(SparkSaslClient.class); private final String secretKeyId; private final SecretKeyHolder secretKeyHolder; @@ -125,7 +124,7 @@ public synchronized void dispose() { */ private class ClientCallbackHandler implements CallbackHandler { @Override - public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + public void handle(Callback[] callbacks) throws UnsupportedCallbackException { for (Callback callback : callbacks) { if (callback instanceof NameCallback) { diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java similarity index 87% rename from network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java rename to common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java index 431cb67a2ae0..e22e09d2a22e 100644 --- a/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java @@ -27,13 +27,13 @@ import javax.security.sasl.Sasl; import javax.security.sasl.SaslException; import javax.security.sasl.SaslServer; -import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Map; -import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; +import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.handler.codec.base64.Base64; import org.slf4j.Logger; @@ -45,7 +45,7 @@ * connections on some socket.) */ public class SparkSaslServer implements SaslEncryptionBackend { - private final Logger logger = LoggerFactory.getLogger(SparkSaslServer.class); + private static final Logger logger = LoggerFactory.getLogger(SparkSaslServer.class); /** * This is passed as the server name when creating the sasl client/server. @@ -154,7 +154,7 @@ public byte[] unwrap(byte[] data, int offset, int len) throws SaslException { */ private class DigestCallbackHandler implements CallbackHandler { @Override - public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + public void handle(Callback[] callbacks) throws UnsupportedCallbackException { for (Callback callback : callbacks) { if (callback instanceof NameCallback) { logger.trace("SASL server callback: setting username"); @@ -187,14 +187,31 @@ public void handle(Callback[] callbacks) throws IOException, UnsupportedCallback /* Encode a byte[] identifier as a Base64-encoded string. */ public static String encodeIdentifier(String identifier) { Preconditions.checkNotNull(identifier, "User cannot be null if SASL is enabled"); - return Base64.encode(Unpooled.wrappedBuffer(identifier.getBytes(Charsets.UTF_8))) - .toString(Charsets.UTF_8); + return getBase64EncodedString(identifier); } /** Encode a password as a base64-encoded char[] array. */ public static char[] encodePassword(String password) { Preconditions.checkNotNull(password, "Password cannot be null if SASL is enabled"); - return Base64.encode(Unpooled.wrappedBuffer(password.getBytes(Charsets.UTF_8))) - .toString(Charsets.UTF_8).toCharArray(); + return getBase64EncodedString(password).toCharArray(); + } + + /** Return a Base64-encoded string. */ + private static String getBase64EncodedString(String str) { + ByteBuf byteBuf = null; + ByteBuf encodedByteBuf = null; + try { + byteBuf = Unpooled.wrappedBuffer(str.getBytes(StandardCharsets.UTF_8)); + encodedByteBuf = Base64.encode(byteBuf); + return encodedByteBuf.toString(StandardCharsets.UTF_8); + } finally { + // The release is called to suppress the memory leak error messages raised by netty. + if (byteBuf != null) { + byteBuf.release(); + if (encodedByteBuf != null) { + encodedByteBuf.release(); + } + } + } } } diff --git a/network/common/src/main/java/org/apache/spark/network/server/MessageHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/MessageHandler.java similarity index 82% rename from network/common/src/main/java/org/apache/spark/network/server/MessageHandler.java rename to common/network-common/src/main/java/org/apache/spark/network/server/MessageHandler.java index b80c15106ecb..4a1f28e9ffb3 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/MessageHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/MessageHandler.java @@ -26,11 +26,14 @@ */ public abstract class MessageHandler { /** Handles the receipt of a single message. */ - public abstract void handle(T message); + public abstract void handle(T message) throws Exception; + + /** Invoked when the channel this MessageHandler is on is active. */ + public abstract void channelActive(); /** Invoked when an exception was caught on the Channel. */ public abstract void exceptionCaught(Throwable cause); - /** Invoked when the channel this MessageHandler is on has been unregistered. */ - public abstract void channelUnregistered(); + /** Invoked when the channel this MessageHandler is on is inactive. */ + public abstract void channelInactive(); } diff --git a/network/common/src/main/java/org/apache/spark/network/server/NoOpRpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/NoOpRpcHandler.java similarity index 91% rename from network/common/src/main/java/org/apache/spark/network/server/NoOpRpcHandler.java rename to common/network-common/src/main/java/org/apache/spark/network/server/NoOpRpcHandler.java index 1502b7489e86..6ed61da5c7ef 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/NoOpRpcHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/NoOpRpcHandler.java @@ -1,5 +1,3 @@ -package org.apache.spark.network.server; - /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -17,6 +15,10 @@ * limitations under the License. */ +package org.apache.spark.network.server; + +import java.nio.ByteBuffer; + import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; @@ -29,7 +31,7 @@ public NoOpRpcHandler() { } @Override - public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + public void receive(TransportClient client, ByteBuffer message, RpcResponseCallback callback) { throw new UnsupportedOperationException("Cannot handle messages"); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java new file mode 100644 index 000000000000..0f6a8824d95e --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import java.util.Iterator; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +import com.google.common.base.Preconditions; +import io.netty.channel.Channel; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.TransportClient; + +/** + * StreamManager which allows registration of an Iterator<ManagedBuffer>, which are + * individually fetched as chunks by the client. Each registered buffer is one chunk. + */ +public class OneForOneStreamManager extends StreamManager { + private static final Logger logger = LoggerFactory.getLogger(OneForOneStreamManager.class); + + private final AtomicLong nextStreamId; + private final ConcurrentHashMap streams; + + /** State of a single stream. */ + private static class StreamState { + final String appId; + final Iterator buffers; + + // The channel associated to the stream + Channel associatedChannel = null; + + // Used to keep track of the index of the buffer that the user has retrieved, just to ensure + // that the caller only requests each chunk one at a time, in order. + int curChunk = 0; + + // Used to keep track of the number of chunks being transferred and not finished yet. + volatile long chunksBeingTransferred = 0L; + + StreamState(String appId, Iterator buffers) { + this.appId = appId; + this.buffers = Preconditions.checkNotNull(buffers); + } + } + + public OneForOneStreamManager() { + // For debugging purposes, start with a random stream id to help identifying different streams. + // This does not need to be globally unique, only unique to this class. + nextStreamId = new AtomicLong((long) new Random().nextInt(Integer.MAX_VALUE) * 1000); + streams = new ConcurrentHashMap<>(); + } + + @Override + public void registerChannel(Channel channel, long streamId) { + if (streams.containsKey(streamId)) { + streams.get(streamId).associatedChannel = channel; + } + } + + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + StreamState state = streams.get(streamId); + if (chunkIndex != state.curChunk) { + throw new IllegalStateException(String.format( + "Received out-of-order chunk index %s (expected %s)", chunkIndex, state.curChunk)); + } else if (!state.buffers.hasNext()) { + throw new IllegalStateException(String.format( + "Requested chunk index beyond end %s", chunkIndex)); + } + state.curChunk += 1; + ManagedBuffer nextChunk = state.buffers.next(); + + if (!state.buffers.hasNext()) { + logger.trace("Removing stream id {}", streamId); + streams.remove(streamId); + } + + return nextChunk; + } + + @Override + public ManagedBuffer openStream(String streamChunkId) { + Pair streamChunkIdPair = parseStreamChunkId(streamChunkId); + return getChunk(streamChunkIdPair.getLeft(), streamChunkIdPair.getRight()); + } + + public static String genStreamChunkId(long streamId, int chunkId) { + return String.format("%d_%d", streamId, chunkId); + } + + // Parse streamChunkId to be stream id and chunk id. This is used when fetch remote chunk as a + // stream. + public static Pair parseStreamChunkId(String streamChunkId) { + String[] array = streamChunkId.split("_"); + assert array.length == 2: + "Stream id and chunk index should be specified."; + long streamId = Long.valueOf(array[0]); + int chunkIndex = Integer.valueOf(array[1]); + return ImmutablePair.of(streamId, chunkIndex); + } + + @Override + public void connectionTerminated(Channel channel) { + // Close all streams which have been associated with the channel. + for (Map.Entry entry: streams.entrySet()) { + StreamState state = entry.getValue(); + if (state.associatedChannel == channel) { + streams.remove(entry.getKey()); + + // Release all remaining buffers. + while (state.buffers.hasNext()) { + state.buffers.next().release(); + } + } + } + } + + @Override + public void checkAuthorization(TransportClient client, long streamId) { + if (client.getClientId() != null) { + StreamState state = streams.get(streamId); + Preconditions.checkArgument(state != null, "Unknown stream ID."); + if (!client.getClientId().equals(state.appId)) { + throw new SecurityException(String.format( + "Client %s not authorized to read stream %d (app %s).", + client.getClientId(), + streamId, + state.appId)); + } + } + } + + @Override + public void chunkBeingSent(long streamId) { + StreamState streamState = streams.get(streamId); + if (streamState != null) { + streamState.chunksBeingTransferred++; + } + + } + + @Override + public void streamBeingSent(String streamId) { + chunkBeingSent(parseStreamChunkId(streamId).getLeft()); + } + + @Override + public void chunkSent(long streamId) { + StreamState streamState = streams.get(streamId); + if (streamState != null) { + streamState.chunksBeingTransferred--; + } + } + + @Override + public void streamSent(String streamId) { + chunkSent(OneForOneStreamManager.parseStreamChunkId(streamId).getLeft()); + } + + @Override + public long chunksBeingTransferred() { + long sum = 0L; + for (StreamState streamState: streams.values()) { + sum += streamState.chunksBeingTransferred; + } + return sum; + } + + /** + * Registers a stream of ManagedBuffers which are served as individual chunks one at a time to + * callers. Each ManagedBuffer will be release()'d after it is transferred on the wire. If a + * client connection is closed before the iterator is fully drained, then the remaining buffers + * will all be release()'d. + * + * If an app ID is provided, only callers who've authenticated with the given app ID will be + * allowed to fetch from this stream. + */ + public long registerStream(String appId, Iterator buffers) { + long myStreamId = nextStreamId.getAndIncrement(); + streams.put(myStreamId, new StreamState(appId, buffers)); + return myStreamId; + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/RpcHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/RpcHandler.java new file mode 100644 index 000000000000..8f7554e2e07d --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/server/RpcHandler.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import java.nio.ByteBuffer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; + +/** + * Handler for sendRPC() messages sent by {@link org.apache.spark.network.client.TransportClient}s. + */ +public abstract class RpcHandler { + + private static final RpcResponseCallback ONE_WAY_CALLBACK = new OneWayRpcCallback(); + + /** + * Receive a single RPC message. Any exception thrown while in this method will be sent back to + * the client in string form as a standard RPC failure. + * + * This method will not be called in parallel for a single TransportClient (i.e., channel). + * + * @param client A channel client which enables the handler to make requests back to the sender + * of this RPC. This will always be the exact same object for a particular channel. + * @param message The serialized bytes of the RPC. + * @param callback Callback which should be invoked exactly once upon success or failure of the + * RPC. + */ + public abstract void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback); + + /** + * Returns the StreamManager which contains the state about which streams are currently being + * fetched by a TransportClient. + */ + public abstract StreamManager getStreamManager(); + + /** + * Receives an RPC message that does not expect a reply. The default implementation will + * call "{@link #receive(TransportClient, ByteBuffer, RpcResponseCallback)}" and log a warning if + * any of the callback methods are called. + * + * @param client A channel client which enables the handler to make requests back to the sender + * of this RPC. This will always be the exact same object for a particular channel. + * @param message The serialized bytes of the RPC. + */ + public void receive(TransportClient client, ByteBuffer message) { + receive(client, message, ONE_WAY_CALLBACK); + } + + /** + * Invoked when the channel associated with the given client is active. + */ + public void channelActive(TransportClient client) { } + + /** + * Invoked when the channel associated with the given client is inactive. + * No further requests will come from this client. + */ + public void channelInactive(TransportClient client) { } + + public void exceptionCaught(Throwable cause, TransportClient client) { } + + private static class OneWayRpcCallback implements RpcResponseCallback { + + private static final Logger logger = LoggerFactory.getLogger(OneWayRpcCallback.class); + + @Override + public void onSuccess(ByteBuffer response) { + logger.warn("Response provided for one-way RPC."); + } + + @Override + public void onFailure(Throwable e) { + logger.error("Error response provided for one-way RPC.", e); + } + + } + +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/StreamManager.java b/common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java similarity index 84% rename from network/common/src/main/java/org/apache/spark/network/server/StreamManager.java rename to common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java index 3f0155957a14..c53529583160 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/StreamManager.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java @@ -54,6 +54,7 @@ public abstract class StreamManager { * {@link #getChunk(long, int)} method. * * @param streamId id of a stream that has been previously registered with the StreamManager. + * @return A managed buffer for the stream, or null if the stream was not found. */ public ManagedBuffer openStream(String streamId) { throw new UnsupportedOperationException(); @@ -82,4 +83,31 @@ public void connectionTerminated(Channel channel) { } */ public void checkAuthorization(TransportClient client, long streamId) { } + /** + * Return the number of chunks being transferred and not finished yet in this StreamManager. + */ + public long chunksBeingTransferred() { + return 0; + } + + /** + * Called when start sending a chunk. + */ + public void chunkBeingSent(long streamId) { } + + /** + * Called when start sending a stream. + */ + public void streamBeingSent(String streamId) { } + + /** + * Called when a chunk is successfully sent. + */ + public void chunkSent(long streamId) { } + + /** + * Called when a stream is successfully sent. + */ + public void streamSent(String streamId) { } + } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java new file mode 100644 index 000000000000..56782a832787 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.handler.timeout.IdleState; +import io.netty.handler.timeout.IdleStateEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportResponseHandler; +import org.apache.spark.network.protocol.RequestMessage; +import org.apache.spark.network.protocol.ResponseMessage; +import static org.apache.spark.network.util.NettyUtils.getRemoteAddress; + +/** + * The single Transport-level Channel handler which is used for delegating requests to the + * {@link TransportRequestHandler} and responses to the {@link TransportResponseHandler}. + * + * All channels created in the transport layer are bidirectional. When the Client initiates a Netty + * Channel with a RequestMessage (which gets handled by the Server's RequestHandler), the Server + * will produce a ResponseMessage (handled by the Client's ResponseHandler). However, the Server + * also gets a handle on the same Channel, so it may then begin to send RequestMessages to the + * Client. + * This means that the Client also needs a RequestHandler and the Server needs a ResponseHandler, + * for the Client's responses to the Server's requests. + * + * This class also handles timeouts from a {@link io.netty.handler.timeout.IdleStateHandler}. + * We consider a connection timed out if there are outstanding fetch or RPC requests but no traffic + * on the channel for at least `requestTimeoutMs`. Note that this is duplex traffic; we will not + * timeout if the client is continuously sending but getting no responses, for simplicity. + */ +public class TransportChannelHandler extends ChannelInboundHandlerAdapter { + private static final Logger logger = LoggerFactory.getLogger(TransportChannelHandler.class); + + private final TransportClient client; + private final TransportResponseHandler responseHandler; + private final TransportRequestHandler requestHandler; + private final long requestTimeoutNs; + private final boolean closeIdleConnections; + + public TransportChannelHandler( + TransportClient client, + TransportResponseHandler responseHandler, + TransportRequestHandler requestHandler, + long requestTimeoutMs, + boolean closeIdleConnections) { + this.client = client; + this.responseHandler = responseHandler; + this.requestHandler = requestHandler; + this.requestTimeoutNs = requestTimeoutMs * 1000L * 1000; + this.closeIdleConnections = closeIdleConnections; + } + + public TransportClient getClient() { + return client; + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + logger.warn("Exception in connection from " + getRemoteAddress(ctx.channel()), + cause); + requestHandler.exceptionCaught(cause); + responseHandler.exceptionCaught(cause); + ctx.close(); + } + + @Override + public void channelActive(ChannelHandlerContext ctx) throws Exception { + try { + requestHandler.channelActive(); + } catch (RuntimeException e) { + logger.error("Exception from request handler while channel is active", e); + } + try { + responseHandler.channelActive(); + } catch (RuntimeException e) { + logger.error("Exception from response handler while channel is active", e); + } + super.channelActive(ctx); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + try { + requestHandler.channelInactive(); + } catch (RuntimeException e) { + logger.error("Exception from request handler while channel is inactive", e); + } + try { + responseHandler.channelInactive(); + } catch (RuntimeException e) { + logger.error("Exception from response handler while channel is inactive", e); + } + super.channelInactive(ctx); + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object request) throws Exception { + if (request instanceof RequestMessage) { + requestHandler.handle((RequestMessage) request); + } else if (request instanceof ResponseMessage) { + responseHandler.handle((ResponseMessage) request); + } else { + ctx.fireChannelRead(request); + } + } + + /** Triggered based on events from an {@link io.netty.handler.timeout.IdleStateHandler}. */ + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof IdleStateEvent) { + IdleStateEvent e = (IdleStateEvent) evt; + // See class comment for timeout semantics. In addition to ensuring we only timeout while + // there are outstanding requests, we also do a secondary consistency check to ensure + // there's no race between the idle timeout and incrementing the numOutstandingRequests + // (see SPARK-7003). + // + // To avoid a race between TransportClientFactory.createClient() and this code which could + // result in an inactive client being returned, this needs to run in a synchronized block. + synchronized (this) { + boolean isActuallyOverdue = + System.nanoTime() - responseHandler.getTimeOfLastRequestNs() > requestTimeoutNs; + if (e.state() == IdleState.ALL_IDLE && isActuallyOverdue) { + if (responseHandler.numOutstandingRequests() > 0) { + String address = getRemoteAddress(ctx.channel()); + logger.error("Connection to {} has been quiet for {} ms while there are outstanding " + + "requests. Assuming connection is dead; please adjust spark.network.timeout if " + + "this is wrong.", address, requestTimeoutNs / 1000 / 1000); + client.timeOut(); + ctx.close(); + } else if (closeIdleConnections) { + // While CloseIdleConnections is enable, we also close idle connection + client.timeOut(); + ctx.close(); + } + } + } + } + ctx.fireUserEventTriggered(evt); + } + + public TransportResponseHandler getResponseHandler() { + return responseHandler; + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java new file mode 100644 index 000000000000..e94453578e6b --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -0,0 +1,232 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import java.net.SocketAddress; +import java.nio.ByteBuffer; + +import com.google.common.base.Throwables; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.protocol.ChunkFetchRequest; +import org.apache.spark.network.protocol.ChunkFetchFailure; +import org.apache.spark.network.protocol.ChunkFetchSuccess; +import org.apache.spark.network.protocol.Encodable; +import org.apache.spark.network.protocol.OneWayMessage; +import org.apache.spark.network.protocol.RequestMessage; +import org.apache.spark.network.protocol.RpcFailure; +import org.apache.spark.network.protocol.RpcRequest; +import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.protocol.StreamFailure; +import org.apache.spark.network.protocol.StreamRequest; +import org.apache.spark.network.protocol.StreamResponse; +import static org.apache.spark.network.util.NettyUtils.getRemoteAddress; + +/** + * A handler that processes requests from clients and writes chunk data back. Each handler is + * attached to a single Netty channel, and keeps track of which streams have been fetched via this + * channel, in order to clean them up if the channel is terminated (see #channelUnregistered). + * + * The messages should have been processed by the pipeline setup by {@link TransportServer}. + */ +public class TransportRequestHandler extends MessageHandler { + private static final Logger logger = LoggerFactory.getLogger(TransportRequestHandler.class); + + /** The Netty channel that this handler is associated with. */ + private final Channel channel; + + /** Client on the same channel allowing us to talk back to the requester. */ + private final TransportClient reverseClient; + + /** Handles all RPC messages. */ + private final RpcHandler rpcHandler; + + /** Returns each chunk part of a stream. */ + private final StreamManager streamManager; + + /** The max number of chunks being transferred and not finished yet. */ + private final long maxChunksBeingTransferred; + + public TransportRequestHandler( + Channel channel, + TransportClient reverseClient, + RpcHandler rpcHandler, + Long maxChunksBeingTransferred) { + this.channel = channel; + this.reverseClient = reverseClient; + this.rpcHandler = rpcHandler; + this.streamManager = rpcHandler.getStreamManager(); + this.maxChunksBeingTransferred = maxChunksBeingTransferred; + } + + @Override + public void exceptionCaught(Throwable cause) { + rpcHandler.exceptionCaught(cause, reverseClient); + } + + @Override + public void channelActive() { + rpcHandler.channelActive(reverseClient); + } + + @Override + public void channelInactive() { + if (streamManager != null) { + try { + streamManager.connectionTerminated(channel); + } catch (RuntimeException e) { + logger.error("StreamManager connectionTerminated() callback failed.", e); + } + } + rpcHandler.channelInactive(reverseClient); + } + + @Override + public void handle(RequestMessage request) { + if (request instanceof ChunkFetchRequest) { + processFetchRequest((ChunkFetchRequest) request); + } else if (request instanceof RpcRequest) { + processRpcRequest((RpcRequest) request); + } else if (request instanceof OneWayMessage) { + processOneWayMessage((OneWayMessage) request); + } else if (request instanceof StreamRequest) { + processStreamRequest((StreamRequest) request); + } else { + throw new IllegalArgumentException("Unknown request type: " + request); + } + } + + private void processFetchRequest(final ChunkFetchRequest req) { + if (logger.isTraceEnabled()) { + logger.trace("Received req from {} to fetch block {}", getRemoteAddress(channel), + req.streamChunkId); + } + long chunksBeingTransferred = streamManager.chunksBeingTransferred(); + if (chunksBeingTransferred >= maxChunksBeingTransferred) { + logger.warn("The number of chunks being transferred {} is above {}, close the connection.", + chunksBeingTransferred, maxChunksBeingTransferred); + channel.close(); + return; + } + ManagedBuffer buf; + try { + streamManager.checkAuthorization(reverseClient, req.streamChunkId.streamId); + streamManager.registerChannel(channel, req.streamChunkId.streamId); + buf = streamManager.getChunk(req.streamChunkId.streamId, req.streamChunkId.chunkIndex); + } catch (Exception e) { + logger.error(String.format("Error opening block %s for request from %s", + req.streamChunkId, getRemoteAddress(channel)), e); + respond(new ChunkFetchFailure(req.streamChunkId, Throwables.getStackTraceAsString(e))); + return; + } + + streamManager.chunkBeingSent(req.streamChunkId.streamId); + respond(new ChunkFetchSuccess(req.streamChunkId, buf)).addListener(future -> { + streamManager.chunkSent(req.streamChunkId.streamId); + }); + } + + private void processStreamRequest(final StreamRequest req) { + if (logger.isTraceEnabled()) { + logger.trace("Received req from {} to fetch stream {}", getRemoteAddress(channel), + req.streamId); + } + + long chunksBeingTransferred = streamManager.chunksBeingTransferred(); + if (chunksBeingTransferred >= maxChunksBeingTransferred) { + logger.warn("The number of chunks being transferred {} is above {}, close the connection.", + chunksBeingTransferred, maxChunksBeingTransferred); + channel.close(); + return; + } + ManagedBuffer buf; + try { + buf = streamManager.openStream(req.streamId); + } catch (Exception e) { + logger.error(String.format( + "Error opening stream %s for request from %s", req.streamId, getRemoteAddress(channel)), e); + respond(new StreamFailure(req.streamId, Throwables.getStackTraceAsString(e))); + return; + } + + if (buf != null) { + streamManager.streamBeingSent(req.streamId); + respond(new StreamResponse(req.streamId, buf.size(), buf)).addListener(future -> { + streamManager.streamSent(req.streamId); + }); + } else { + respond(new StreamFailure(req.streamId, String.format( + "Stream '%s' was not found.", req.streamId))); + } + } + + private void processRpcRequest(final RpcRequest req) { + try { + rpcHandler.receive(reverseClient, req.body().nioByteBuffer(), new RpcResponseCallback() { + @Override + public void onSuccess(ByteBuffer response) { + respond(new RpcResponse(req.requestId, new NioManagedBuffer(response))); + } + + @Override + public void onFailure(Throwable e) { + respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); + } + }); + } catch (Exception e) { + logger.error("Error while invoking RpcHandler#receive() on RPC id " + req.requestId, e); + respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); + } finally { + req.body().release(); + } + } + + private void processOneWayMessage(OneWayMessage req) { + try { + rpcHandler.receive(reverseClient, req.body().nioByteBuffer()); + } catch (Exception e) { + logger.error("Error while invoking RpcHandler#receive() for one-way message.", e); + } finally { + req.body().release(); + } + } + + /** + * Responds to a single message with some Encodable object. If a failure occurs while sending, + * it will be logged and the channel closed. + */ + private ChannelFuture respond(Encodable result) { + SocketAddress remoteAddress = channel.remoteAddress(); + return channel.writeAndFlush(result).addListener(future -> { + if (future.isSuccess()) { + logger.trace("Sent result {} to client {}", result, remoteAddress); + } else { + logger.error(String.format("Error sending result %s to %s; closing connection", + result, remoteAddress), future.cause()); + channel.close(); + } + }); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java similarity index 79% rename from network/common/src/main/java/org/apache/spark/network/server/TransportServer.java rename to common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java index f4fadb1ee3b8..0719fa7647bc 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.concurrent.TimeUnit; +import com.codahale.metrics.MetricSet; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import io.netty.bootstrap.ServerBootstrap; @@ -31,20 +32,17 @@ import io.netty.channel.ChannelOption; import io.netty.channel.EventLoopGroup; import io.netty.channel.socket.SocketChannel; -import org.apache.spark.network.util.JavaUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.spark.network.TransportContext; -import org.apache.spark.network.util.IOMode; -import org.apache.spark.network.util.NettyUtils; -import org.apache.spark.network.util.TransportConf; +import org.apache.spark.network.util.*; /** * Server for the efficient, low-level streaming service. */ public class TransportServer implements Closeable { - private final Logger logger = LoggerFactory.getLogger(TransportServer.class); + private static final Logger logger = LoggerFactory.getLogger(TransportServer.class); private final TransportContext context; private final TransportConf conf; @@ -54,10 +52,15 @@ public class TransportServer implements Closeable { private ServerBootstrap bootstrap; private ChannelFuture channelFuture; private int port = -1; + private NettyMemoryMetrics metrics; - /** Creates a TransportServer that binds to the given port, or to any available if 0. */ + /** + * Creates a TransportServer that binds to the given host and the given port, or to any available + * if 0. If you don't want to bind to any special host, set "hostToBind" to null. + * */ public TransportServer( TransportContext context, + String hostToBind, int portToBind, RpcHandler appRpcHandler, List bootstraps) { @@ -67,7 +70,7 @@ public TransportServer( this.bootstraps = Lists.newArrayList(Preconditions.checkNotNull(bootstraps)); try { - init(portToBind); + init(hostToBind, portToBind); } catch (RuntimeException e) { JavaUtils.closeQuietly(this); throw e; @@ -81,11 +84,11 @@ public int getPort() { return port; } - private void init(int portToBind) { + private void init(String hostToBind, int portToBind) { IOMode ioMode = IOMode.valueOf(conf.ioMode()); EventLoopGroup bossGroup = - NettyUtils.createEventLoop(ioMode, conf.serverThreads(), "shuffle-server"); + NettyUtils.createEventLoop(ioMode, conf.serverThreads(), conf.getModuleName() + "-server"); EventLoopGroup workerGroup = bossGroup; PooledByteBufAllocator allocator = NettyUtils.createPooledByteBufAllocator( @@ -97,6 +100,9 @@ private void init(int portToBind) { .option(ChannelOption.ALLOCATOR, allocator) .childOption(ChannelOption.ALLOCATOR, allocator); + this.metrics = new NettyMemoryMetrics( + allocator, conf.getModuleName() + "-server", conf); + if (conf.backLog() > 0) { bootstrap.option(ChannelOption.SO_BACKLOG, conf.backLog()); } @@ -111,7 +117,7 @@ private void init(int portToBind) { bootstrap.childHandler(new ChannelInitializer() { @Override - protected void initChannel(SocketChannel ch) throws Exception { + protected void initChannel(SocketChannel ch) { RpcHandler rpcHandler = appRpcHandler; for (TransportServerBootstrap bootstrap : bootstraps) { rpcHandler = bootstrap.doBootstrap(ch, rpcHandler); @@ -120,11 +126,17 @@ protected void initChannel(SocketChannel ch) throws Exception { } }); - channelFuture = bootstrap.bind(new InetSocketAddress(portToBind)); + InetSocketAddress address = hostToBind == null ? + new InetSocketAddress(portToBind): new InetSocketAddress(hostToBind, portToBind); + channelFuture = bootstrap.bind(address); channelFuture.syncUninterruptibly(); port = ((InetSocketAddress) channelFuture.channel().localAddress()).getPort(); - logger.debug("Shuffle server started on port :" + port); + logger.debug("Shuffle server started on port: {}", port); + } + + public MetricSet getAllMetrics() { + return metrics; } @Override diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportServerBootstrap.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServerBootstrap.java similarity index 100% rename from network/common/src/main/java/org/apache/spark/network/server/TransportServerBootstrap.java rename to common/network-common/src/main/java/org/apache/spark/network/server/TransportServerBootstrap.java diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java b/common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java new file mode 100644 index 000000000000..25d103d0e316 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayReadableChannel.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; + +import io.netty.buffer.ByteBuf; + +public class ByteArrayReadableChannel implements ReadableByteChannel { + private ByteBuf data; + + public int readableBytes() { + return data.readableBytes(); + } + + public void feedData(ByteBuf buf) { + data = buf; + } + + @Override + public int read(ByteBuffer dst) throws IOException { + int totalRead = 0; + while (data.readableBytes() > 0 && dst.remaining() > 0) { + int bytesToRead = Math.min(data.readableBytes(), dst.remaining()); + dst.put(data.readSlice(bytesToRead).nioBuffer()); + totalRead += bytesToRead; + } + + if (data.readableBytes() == 0) { + data.release(); + } + + return totalRead; + } + + @Override + public void close() throws IOException { + } + + @Override + public boolean isOpen() { + return true; + } + +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteArrayWritableChannel.java b/common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayWritableChannel.java similarity index 100% rename from network/common/src/main/java/org/apache/spark/network/util/ByteArrayWritableChannel.java rename to common/network-common/src/main/java/org/apache/spark/network/util/ByteArrayWritableChannel.java diff --git a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java similarity index 89% rename from network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java rename to common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java index 36d655017fb0..984575acaf51 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/ByteUnit.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java @@ -17,14 +17,14 @@ package org.apache.spark.network.util; public enum ByteUnit { - BYTE (1), - KiB (1024L), - MiB ((long) Math.pow(1024L, 2L)), - GiB ((long) Math.pow(1024L, 3L)), - TiB ((long) Math.pow(1024L, 4L)), - PiB ((long) Math.pow(1024L, 5L)); + BYTE(1), + KiB(1024L), + MiB((long) Math.pow(1024L, 2L)), + GiB((long) Math.pow(1024L, 3L)), + TiB((long) Math.pow(1024L, 4L)), + PiB((long) Math.pow(1024L, 5L)); - private ByteUnit(long multiplier) { + ByteUnit(long multiplier) { this.multiplier = multiplier; } @@ -33,8 +33,8 @@ private ByteUnit(long multiplier) { public long convertFrom(long d, ByteUnit u) { return u.convertTo(d, this); } - - // Convert the provided number (d) interpreted as this unit type to unit type (u). + + // Convert the provided number (d) interpreted as this unit type to unit type (u). public long convertTo(long d, ByteUnit u) { if (multiplier > u.multiplier) { long ratio = multiplier / u.multiplier; @@ -44,7 +44,7 @@ public long convertTo(long d, ByteUnit u) { } return d * ratio; } else { - // Perform operations in this order to avoid potential overflow + // Perform operations in this order to avoid potential overflow // when computing d * multiplier return d / (u.multiplier / multiplier); } @@ -54,14 +54,14 @@ public double toBytes(long d) { if (d < 0) { throw new IllegalArgumentException("Negative size value. Size must be positive: " + d); } - return d * multiplier; + return d * multiplier; } - + public long toKiB(long d) { return convertTo(d, KiB); } public long toMiB(long d) { return convertTo(d, MiB); } public long toGiB(long d) { return convertTo(d, GiB); } public long toTiB(long d) { return convertTo(d, TiB); } public long toPiB(long d) { return convertTo(d, PiB); } - + private final long multiplier; } diff --git a/network/common/src/main/java/org/apache/spark/network/util/ConfigProvider.java b/common/network-common/src/main/java/org/apache/spark/network/util/ConfigProvider.java similarity index 92% rename from network/common/src/main/java/org/apache/spark/network/util/ConfigProvider.java rename to common/network-common/src/main/java/org/apache/spark/network/util/ConfigProvider.java index d944d9da1c7f..f6aef499b2bf 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/ConfigProvider.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/ConfigProvider.java @@ -17,6 +17,7 @@ package org.apache.spark.network.util; +import java.util.Map; import java.util.NoSuchElementException; /** @@ -26,6 +27,9 @@ public abstract class ConfigProvider { /** Obtains the value of the given config, throws NoSuchElementException if it doesn't exist. */ public abstract String get(String name); + /** Returns all the config values in the provider. */ + public abstract Iterable> getAll(); + public String get(String name, String defaultValue) { try { return get(name); @@ -49,4 +53,5 @@ public double getDouble(String name, double defaultValue) { public boolean getBoolean(String name, boolean defaultValue) { return Boolean.parseBoolean(get(name, Boolean.toString(defaultValue))); } + } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/CryptoUtils.java b/common/network-common/src/main/java/org/apache/spark/network/util/CryptoUtils.java new file mode 100644 index 000000000000..a6d8358ee900 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/util/CryptoUtils.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.util.Map; +import java.util.Properties; + +/** + * Utility methods related to the commons-crypto library. + */ +public class CryptoUtils { + + // The prefix for the configurations passing to Apache Commons Crypto library. + public static final String COMMONS_CRYPTO_CONFIG_PREFIX = "commons.crypto."; + + /** + * Extract the commons-crypto configuration embedded in a list of config values. + * + * @param prefix Prefix in the given configuration that identifies the commons-crypto configs. + * @param conf List of configuration values. + */ + public static Properties toCryptoConf(String prefix, Iterable> conf) { + Properties props = new Properties(); + for (Map.Entry e : conf) { + String key = e.getKey(); + if (key.startsWith(prefix)) { + props.setProperty(COMMONS_CRYPTO_CONFIG_PREFIX + key.substring(prefix.length()), + e.getValue()); + } + } + return props; + } + +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/IOMode.java b/common/network-common/src/main/java/org/apache/spark/network/util/IOMode.java similarity index 100% rename from network/common/src/main/java/org/apache/spark/network/util/IOMode.java rename to common/network-common/src/main/java/org/apache/spark/network/util/IOMode.java diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java new file mode 100644 index 000000000000..afc59efaef81 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -0,0 +1,363 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.io.Closeable; +import java.io.EOFException; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.nio.charset.StandardCharsets; +import java.util.Locale; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import io.netty.buffer.Unpooled; +import org.apache.commons.lang3.SystemUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * General utilities available in the network package. Many of these are sourced from Spark's + * own Utils, just accessible within this package. + */ +public class JavaUtils { + private static final Logger logger = LoggerFactory.getLogger(JavaUtils.class); + + /** + * Define a default value for driver memory here since this value is referenced across the code + * base and nearly all files already use Utils.scala + */ + public static final long DEFAULT_DRIVER_MEM_MB = 1024; + + /** Closes the given object, ignoring IOExceptions. */ + public static void closeQuietly(Closeable closeable) { + try { + if (closeable != null) { + closeable.close(); + } + } catch (IOException e) { + logger.error("IOException should not have been thrown.", e); + } + } + + /** Returns a hash consistent with Spark's Utils.nonNegativeHash(). */ + public static int nonNegativeHash(Object obj) { + if (obj == null) { return 0; } + int hash = obj.hashCode(); + return hash != Integer.MIN_VALUE ? Math.abs(hash) : 0; + } + + /** + * Convert the given string to a byte buffer. The resulting buffer can be + * converted back to the same string through {@link #bytesToString(ByteBuffer)}. + */ + public static ByteBuffer stringToBytes(String s) { + return Unpooled.wrappedBuffer(s.getBytes(StandardCharsets.UTF_8)).nioBuffer(); + } + + /** + * Convert the given byte buffer to a string. The resulting string can be + * converted back to the same byte buffer through {@link #stringToBytes(String)}. + */ + public static String bytesToString(ByteBuffer b) { + return Unpooled.wrappedBuffer(b).toString(StandardCharsets.UTF_8); + } + + /** + * Delete a file or directory and its contents recursively. + * Don't follow directories if they are symlinks. + * + * @param file Input file / dir to be deleted + * @throws IOException if deletion is unsuccessful + */ + public static void deleteRecursively(File file) throws IOException { + if (file == null) { return; } + + // On Unix systems, use operating system command to run faster + // If that does not work out, fallback to the Java IO way + if (SystemUtils.IS_OS_UNIX) { + try { + deleteRecursivelyUsingUnixNative(file); + return; + } catch (IOException e) { + logger.warn("Attempt to delete using native Unix OS command failed for path = {}. " + + "Falling back to Java IO way", file.getAbsolutePath(), e); + } + } + + deleteRecursivelyUsingJavaIO(file); + } + + private static void deleteRecursivelyUsingJavaIO(File file) throws IOException { + if (file.isDirectory() && !isSymlink(file)) { + IOException savedIOException = null; + for (File child : listFilesSafely(file)) { + try { + deleteRecursively(child); + } catch (IOException e) { + // In case of multiple exceptions, only last one will be thrown + savedIOException = e; + } + } + if (savedIOException != null) { + throw savedIOException; + } + } + + boolean deleted = file.delete(); + // Delete can also fail if the file simply did not exist. + if (!deleted && file.exists()) { + throw new IOException("Failed to delete: " + file.getAbsolutePath()); + } + } + + private static void deleteRecursivelyUsingUnixNative(File file) throws IOException { + ProcessBuilder builder = new ProcessBuilder("rm", "-rf", file.getAbsolutePath()); + Process process = null; + int exitCode = -1; + + try { + // In order to avoid deadlocks, consume the stdout (and stderr) of the process + builder.redirectErrorStream(true); + builder.redirectOutput(new File("/dev/null")); + + process = builder.start(); + + exitCode = process.waitFor(); + } catch (Exception e) { + throw new IOException("Failed to delete: " + file.getAbsolutePath(), e); + } finally { + if (process != null) { + process.destroy(); + } + } + + if (exitCode != 0 || file.exists()) { + throw new IOException("Failed to delete: " + file.getAbsolutePath()); + } + } + + private static File[] listFilesSafely(File file) throws IOException { + if (file.exists()) { + File[] files = file.listFiles(); + if (files == null) { + throw new IOException("Failed to list files for dir: " + file); + } + return files; + } else { + return new File[0]; + } + } + + private static boolean isSymlink(File file) throws IOException { + Preconditions.checkNotNull(file); + File fileInCanonicalDir = null; + if (file.getParent() == null) { + fileInCanonicalDir = file; + } else { + fileInCanonicalDir = new File(file.getParentFile().getCanonicalFile(), file.getName()); + } + return !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()); + } + + private static final ImmutableMap timeSuffixes = + ImmutableMap.builder() + .put("us", TimeUnit.MICROSECONDS) + .put("ms", TimeUnit.MILLISECONDS) + .put("s", TimeUnit.SECONDS) + .put("m", TimeUnit.MINUTES) + .put("min", TimeUnit.MINUTES) + .put("h", TimeUnit.HOURS) + .put("d", TimeUnit.DAYS) + .build(); + + private static final ImmutableMap byteSuffixes = + ImmutableMap.builder() + .put("b", ByteUnit.BYTE) + .put("k", ByteUnit.KiB) + .put("kb", ByteUnit.KiB) + .put("m", ByteUnit.MiB) + .put("mb", ByteUnit.MiB) + .put("g", ByteUnit.GiB) + .put("gb", ByteUnit.GiB) + .put("t", ByteUnit.TiB) + .put("tb", ByteUnit.TiB) + .put("p", ByteUnit.PiB) + .put("pb", ByteUnit.PiB) + .build(); + + /** + * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a time count in the given unit. + * The unit is also considered the default if the given string does not specify a unit. + */ + public static long timeStringAs(String str, TimeUnit unit) { + String lower = str.toLowerCase(Locale.ROOT).trim(); + + try { + Matcher m = Pattern.compile("(-?[0-9]+)([a-z]+)?").matcher(lower); + if (!m.matches()) { + throw new NumberFormatException("Failed to parse time string: " + str); + } + + long val = Long.parseLong(m.group(1)); + String suffix = m.group(2); + + // Check for invalid suffixes + if (suffix != null && !timeSuffixes.containsKey(suffix)) { + throw new NumberFormatException("Invalid suffix: \"" + suffix + "\""); + } + + // If suffix is valid use that, otherwise none was provided and use the default passed + return unit.convert(val, suffix != null ? timeSuffixes.get(suffix) : unit); + } catch (NumberFormatException e) { + String timeError = "Time must be specified as seconds (s), " + + "milliseconds (ms), microseconds (us), minutes (m or min), hour (h), or day (d). " + + "E.g. 50s, 100ms, or 250us."; + + throw new NumberFormatException(timeError + "\n" + e.getMessage()); + } + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in ms. + */ + public static long timeStringAsMs(String str) { + return timeStringAs(str, TimeUnit.MILLISECONDS); + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to seconds for internal use. If + * no suffix is provided, the passed number is assumed to be in seconds. + */ + public static long timeStringAsSec(String str) { + return timeStringAs(str, TimeUnit.SECONDS); + } + + /** + * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to the given. If no suffix is + * provided, a direct conversion to the provided unit is attempted. + */ + public static long byteStringAs(String str, ByteUnit unit) { + String lower = str.toLowerCase(Locale.ROOT).trim(); + + try { + Matcher m = Pattern.compile("([0-9]+)([a-z]+)?").matcher(lower); + Matcher fractionMatcher = Pattern.compile("([0-9]+\\.[0-9]+)([a-z]+)?").matcher(lower); + + if (m.matches()) { + long val = Long.parseLong(m.group(1)); + String suffix = m.group(2); + + // Check for invalid suffixes + if (suffix != null && !byteSuffixes.containsKey(suffix)) { + throw new NumberFormatException("Invalid suffix: \"" + suffix + "\""); + } + + // If suffix is valid use that, otherwise none was provided and use the default passed + return unit.convertFrom(val, suffix != null ? byteSuffixes.get(suffix) : unit); + } else if (fractionMatcher.matches()) { + throw new NumberFormatException("Fractional values are not supported. Input was: " + + fractionMatcher.group(1)); + } else { + throw new NumberFormatException("Failed to parse byte string: " + str); + } + + } catch (NumberFormatException e) { + String byteError = "Size must be specified as bytes (b), " + + "kibibytes (k), mebibytes (m), gibibytes (g), tebibytes (t), or pebibytes(p). " + + "E.g. 50b, 100k, or 250m."; + + throw new NumberFormatException(byteError + "\n" + e.getMessage()); + } + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to bytes for + * internal use. + * + * If no suffix is provided, the passed number is assumed to be in bytes. + */ + public static long byteStringAsBytes(String str) { + return byteStringAs(str, ByteUnit.BYTE); + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to kibibytes for + * internal use. + * + * If no suffix is provided, the passed number is assumed to be in kibibytes. + */ + public static long byteStringAsKb(String str) { + return byteStringAs(str, ByteUnit.KiB); + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to mebibytes for + * internal use. + * + * If no suffix is provided, the passed number is assumed to be in mebibytes. + */ + public static long byteStringAsMb(String str) { + return byteStringAs(str, ByteUnit.MiB); + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to gibibytes for + * internal use. + * + * If no suffix is provided, the passed number is assumed to be in gibibytes. + */ + public static long byteStringAsGb(String str) { + return byteStringAs(str, ByteUnit.GiB); + } + + /** + * Returns a byte array with the buffer's contents, trying to avoid copying the data if + * possible. + */ + public static byte[] bufferToArray(ByteBuffer buffer) { + if (buffer.hasArray() && buffer.arrayOffset() == 0 && + buffer.array().length == buffer.remaining()) { + return buffer.array(); + } else { + byte[] bytes = new byte[buffer.remaining()]; + buffer.get(bytes); + return bytes; + } + } + + /** + * Fills a buffer with data read from the channel. + */ + public static void readFully(ReadableByteChannel channel, ByteBuffer dst) throws IOException { + int expected = dst.remaining(); + while (dst.hasRemaining()) { + if (channel.read(dst) < 0) { + throw new EOFException(String.format("Not enough bytes in channel (expected %d).", + expected)); + } + } + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java b/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java new file mode 100644 index 000000000000..f96d068cf3d5 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.fusesource.leveldbjni.JniDBFactory; +import org.fusesource.leveldbjni.internal.NativeDB; +import org.iq80.leveldb.DB; +import org.iq80.leveldb.Options; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * LevelDB utility class available in the network package. + */ +public class LevelDBProvider { + private static final Logger logger = LoggerFactory.getLogger(LevelDBProvider.class); + + public static DB initLevelDB(File dbFile, StoreVersion version, ObjectMapper mapper) throws + IOException { + DB tmpDb = null; + if (dbFile != null) { + Options options = new Options(); + options.createIfMissing(false); + options.logger(new LevelDBLogger()); + try { + tmpDb = JniDBFactory.factory.open(dbFile, options); + } catch (NativeDB.DBException e) { + if (e.isNotFound() || e.getMessage().contains(" does not exist ")) { + logger.info("Creating state database at " + dbFile); + options.createIfMissing(true); + try { + tmpDb = JniDBFactory.factory.open(dbFile, options); + } catch (NativeDB.DBException dbExc) { + throw new IOException("Unable to create state store", dbExc); + } + } else { + // the leveldb file seems to be corrupt somehow. Lets just blow it away and create a new + // one, so we can keep processing new apps + logger.error("error opening leveldb file {}. Creating new file, will not be able to " + + "recover state for existing applications", dbFile, e); + if (dbFile.isDirectory()) { + for (File f : dbFile.listFiles()) { + if (!f.delete()) { + logger.warn("error deleting {}", f.getPath()); + } + } + } + if (!dbFile.delete()) { + logger.warn("error deleting {}", dbFile.getPath()); + } + options.createIfMissing(true); + try { + tmpDb = JniDBFactory.factory.open(dbFile, options); + } catch (NativeDB.DBException dbExc) { + throw new IOException("Unable to create state store", dbExc); + } + + } + } + // if there is a version mismatch, we throw an exception, which means the service is unusable + checkVersion(tmpDb, version, mapper); + } + return tmpDb; + } + + private static class LevelDBLogger implements org.iq80.leveldb.Logger { + private static final Logger LOG = LoggerFactory.getLogger(LevelDBLogger.class); + + @Override + public void log(String message) { + LOG.info(message); + } + } + + /** + * Simple major.minor versioning scheme. Any incompatible changes should be across major + * versions. Minor version differences are allowed -- meaning we should be able to read + * dbs that are either earlier *or* later on the minor version. + */ + public static void checkVersion(DB db, StoreVersion newversion, ObjectMapper mapper) throws + IOException { + byte[] bytes = db.get(StoreVersion.KEY); + if (bytes == null) { + storeVersion(db, newversion, mapper); + } else { + StoreVersion version = mapper.readValue(bytes, StoreVersion.class); + if (version.major != newversion.major) { + throw new IOException("cannot read state DB with version " + version + ", incompatible " + + "with current version " + newversion); + } + storeVersion(db, newversion, mapper); + } + } + + public static void storeVersion(DB db, StoreVersion version, ObjectMapper mapper) + throws IOException { + db.put(StoreVersion.KEY, mapper.writeValueAsBytes(version)); + } + + public static class StoreVersion { + + static final byte[] KEY = "StoreVersion".getBytes(StandardCharsets.UTF_8); + + public final int major; + public final int minor; + + @JsonCreator + public StoreVersion(@JsonProperty("major") int major, @JsonProperty("minor") int minor) { + this.major = major; + this.minor = minor; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StoreVersion that = (StoreVersion) o; + + return major == that.major && minor == that.minor; + } + + @Override + public int hashCode() { + int result = major; + result = 31 * result + minor; + return result; + } + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java b/common/network-common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java similarity index 81% rename from network/common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java rename to common/network-common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java index 922c37a10efd..e79eef032589 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java @@ -48,11 +48,27 @@ * use this functionality in both a Guava 11 environment and a Guava >14 environment. */ public final class LimitedInputStream extends FilterInputStream { + private final boolean closeWrappedStream; private long left; private long mark = -1; public LimitedInputStream(InputStream in, long limit) { + this(in, limit, true); + } + + /** + * Create a LimitedInputStream that will read {@code limit} bytes from {@code in}. + *

+ * If {@code closeWrappedStream} is true, this will close {@code in} when it is closed. + * Otherwise, the stream is left open for reading its remaining content. + * + * @param in a {@link InputStream} to read from + * @param limit the number of bytes to read + * @param closeWrappedStream whether to close {@code in} when {@link #close} is called + */ + public LimitedInputStream(InputStream in, long limit, boolean closeWrappedStream) { super(in); + this.closeWrappedStream = closeWrappedStream; Preconditions.checkNotNull(in); Preconditions.checkArgument(limit >= 0, "limit must be non-negative"); left = limit; @@ -102,4 +118,11 @@ public LimitedInputStream(InputStream in, long limit) { left -= skipped; return skipped; } + + @Override + public void close() throws IOException { + if (closeWrappedStream) { + super.close(); + } + } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java b/common/network-common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java new file mode 100644 index 000000000000..a2cf87d1af7e --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.NoSuchElementException; + +/** ConfigProvider based on a Map (copied in the constructor). */ +public class MapConfigProvider extends ConfigProvider { + + public static final MapConfigProvider EMPTY = new MapConfigProvider(Collections.emptyMap()); + + private final Map config; + + public MapConfigProvider(Map config) { + this.config = new HashMap<>(config); + } + + @Override + public String get(String name) { + String value = config.get(name); + if (value == null) { + throw new NoSuchElementException(name); + } + return value; + } + + @Override + public String get(String name, String defaultValue) { + String value = config.get(name); + return value == null ? defaultValue : value; + } + + @Override + public Iterable> getAll() { + return config.entrySet(); + } + +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/NettyMemoryMetrics.java b/common/network-common/src/main/java/org/apache/spark/network/util/NettyMemoryMetrics.java new file mode 100644 index 000000000000..77f807cdb541 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/util/NettyMemoryMetrics.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.util.*; + +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Metric; +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.MetricSet; +import com.google.common.annotations.VisibleForTesting; +import io.netty.buffer.PoolArenaMetric; +import io.netty.buffer.PooledByteBufAllocator; +import io.netty.buffer.PooledByteBufAllocatorMetric; + +/** + * A Netty memory metrics class to collect metrics from Netty PooledByteBufAllocator. + */ +public class NettyMemoryMetrics implements MetricSet { + + private final PooledByteBufAllocator pooledAllocator; + + private final boolean verboseMetricsEnabled; + + private final Map allMetrics; + + private final String metricPrefix; + + @VisibleForTesting + static final Set VERBOSE_METRICS = new HashSet<>(); + static { + VERBOSE_METRICS.addAll(Arrays.asList( + "numAllocations", + "numTinyAllocations", + "numSmallAllocations", + "numNormalAllocations", + "numHugeAllocations", + "numDeallocations", + "numTinyDeallocations", + "numSmallDeallocations", + "numNormalDeallocations", + "numHugeDeallocations", + "numActiveAllocations", + "numActiveTinyAllocations", + "numActiveSmallAllocations", + "numActiveNormalAllocations", + "numActiveHugeAllocations", + "numActiveBytes")); + } + + public NettyMemoryMetrics(PooledByteBufAllocator pooledAllocator, + String metricPrefix, + TransportConf conf) { + this.pooledAllocator = pooledAllocator; + this.allMetrics = new HashMap<>(); + this.metricPrefix = metricPrefix; + this.verboseMetricsEnabled = conf.verboseMetrics(); + + registerMetrics(this.pooledAllocator); + } + + private void registerMetrics(PooledByteBufAllocator allocator) { + PooledByteBufAllocatorMetric pooledAllocatorMetric = allocator.metric(); + + // Register general metrics. + allMetrics.put(MetricRegistry.name(metricPrefix, "usedHeapMemory"), + (Gauge) () -> pooledAllocatorMetric.usedHeapMemory()); + allMetrics.put(MetricRegistry.name(metricPrefix, "usedDirectMemory"), + (Gauge) () -> pooledAllocatorMetric.usedDirectMemory()); + + if (verboseMetricsEnabled) { + int directArenaIndex = 0; + for (PoolArenaMetric metric : pooledAllocatorMetric.directArenas()) { + registerArenaMetric(metric, "directArena" + directArenaIndex); + directArenaIndex++; + } + + int heapArenaIndex = 0; + for (PoolArenaMetric metric : pooledAllocatorMetric.heapArenas()) { + registerArenaMetric(metric, "heapArena" + heapArenaIndex); + heapArenaIndex++; + } + } + } + + private void registerArenaMetric(PoolArenaMetric arenaMetric, String arenaName) { + for (String methodName : VERBOSE_METRICS) { + Method m; + try { + m = PoolArenaMetric.class.getMethod(methodName); + } catch (Exception e) { + // Failed to find metric related method, ignore this metric. + continue; + } + + if (!Modifier.isPublic(m.getModifiers())) { + // Ignore non-public methods. + continue; + } + + Class returnType = m.getReturnType(); + String metricName = MetricRegistry.name(metricPrefix, arenaName, m.getName()); + if (returnType.equals(int.class)) { + allMetrics.put(metricName, (Gauge) () -> { + try { + return (Integer) m.invoke(arenaMetric); + } catch (Exception e) { + return -1; // Swallow the exceptions. + } + }); + + } else if (returnType.equals(long.class)) { + allMetrics.put(metricName, (Gauge) () -> { + try { + return (Long) m.invoke(arenaMetric); + } catch (Exception e) { + return -1L; // Swallow the exceptions. + } + }); + } + } + } + + @Override + public Map getMetrics() { + return Collections.unmodifiableMap(allMetrics); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java b/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java similarity index 94% rename from network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java rename to common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java index caa7260bc828..5e85180bd6f9 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java @@ -20,7 +20,6 @@ import java.lang.reflect.Field; import java.util.concurrent.ThreadFactory; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.netty.buffer.PooledByteBufAllocator; import io.netty.channel.Channel; import io.netty.channel.EventLoopGroup; @@ -31,8 +30,7 @@ import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.socket.nio.NioServerSocketChannel; import io.netty.channel.socket.nio.NioSocketChannel; -import io.netty.handler.codec.ByteToMessageDecoder; -import io.netty.handler.codec.LengthFieldBasedFrameDecoder; +import io.netty.util.concurrent.DefaultThreadFactory; import io.netty.util.internal.PlatformDependent; /** @@ -41,10 +39,7 @@ public class NettyUtils { /** Creates a new ThreadFactory which prefixes each thread with the given name. */ public static ThreadFactory createThreadFactory(String threadPoolPrefix) { - return new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat(threadPoolPrefix + "-%d") - .build(); + return new DefaultThreadFactory(threadPoolPrefix, true); } /** Creates a Netty EventLoopGroup based on the IOMode. */ diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java new file mode 100644 index 000000000000..91497b949221 --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -0,0 +1,284 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.util.Locale; +import java.util.Properties; + +import com.google.common.primitives.Ints; + +/** + * A central location that tracks all the settings we expose to users. + */ +public class TransportConf { + + private final String SPARK_NETWORK_IO_MODE_KEY; + private final String SPARK_NETWORK_IO_PREFERDIRECTBUFS_KEY; + private final String SPARK_NETWORK_IO_CONNECTIONTIMEOUT_KEY; + private final String SPARK_NETWORK_IO_BACKLOG_KEY; + private final String SPARK_NETWORK_IO_NUMCONNECTIONSPERPEER_KEY; + private final String SPARK_NETWORK_IO_SERVERTHREADS_KEY; + private final String SPARK_NETWORK_IO_CLIENTTHREADS_KEY; + private final String SPARK_NETWORK_IO_RECEIVEBUFFER_KEY; + private final String SPARK_NETWORK_IO_SENDBUFFER_KEY; + private final String SPARK_NETWORK_SASL_TIMEOUT_KEY; + private final String SPARK_NETWORK_IO_MAXRETRIES_KEY; + private final String SPARK_NETWORK_IO_RETRYWAIT_KEY; + private final String SPARK_NETWORK_IO_LAZYFD_KEY; + private final String SPARK_NETWORK_VERBOSE_METRICS; + + private final ConfigProvider conf; + + private final String module; + + public TransportConf(String module, ConfigProvider conf) { + this.module = module; + this.conf = conf; + SPARK_NETWORK_IO_MODE_KEY = getConfKey("io.mode"); + SPARK_NETWORK_IO_PREFERDIRECTBUFS_KEY = getConfKey("io.preferDirectBufs"); + SPARK_NETWORK_IO_CONNECTIONTIMEOUT_KEY = getConfKey("io.connectionTimeout"); + SPARK_NETWORK_IO_BACKLOG_KEY = getConfKey("io.backLog"); + SPARK_NETWORK_IO_NUMCONNECTIONSPERPEER_KEY = getConfKey("io.numConnectionsPerPeer"); + SPARK_NETWORK_IO_SERVERTHREADS_KEY = getConfKey("io.serverThreads"); + SPARK_NETWORK_IO_CLIENTTHREADS_KEY = getConfKey("io.clientThreads"); + SPARK_NETWORK_IO_RECEIVEBUFFER_KEY = getConfKey("io.receiveBuffer"); + SPARK_NETWORK_IO_SENDBUFFER_KEY = getConfKey("io.sendBuffer"); + SPARK_NETWORK_SASL_TIMEOUT_KEY = getConfKey("sasl.timeout"); + SPARK_NETWORK_IO_MAXRETRIES_KEY = getConfKey("io.maxRetries"); + SPARK_NETWORK_IO_RETRYWAIT_KEY = getConfKey("io.retryWait"); + SPARK_NETWORK_IO_LAZYFD_KEY = getConfKey("io.lazyFD"); + SPARK_NETWORK_VERBOSE_METRICS = getConfKey("io.enableVerboseMetrics"); + } + + public int getInt(String name, int defaultValue) { + return conf.getInt(name, defaultValue); + } + + public String get(String name, String defaultValue) { + return conf.get(name, defaultValue); + } + + private String getConfKey(String suffix) { + return "spark." + module + "." + suffix; + } + + public String getModuleName() { + return module; + } + + /** IO mode: nio or epoll */ + public String ioMode() { + return conf.get(SPARK_NETWORK_IO_MODE_KEY, "NIO").toUpperCase(Locale.ROOT); + } + + /** If true, we will prefer allocating off-heap byte buffers within Netty. */ + public boolean preferDirectBufs() { + return conf.getBoolean(SPARK_NETWORK_IO_PREFERDIRECTBUFS_KEY, true); + } + + /** Connect timeout in milliseconds. Default 120 secs. */ + public int connectionTimeoutMs() { + long defaultNetworkTimeoutS = JavaUtils.timeStringAsSec( + conf.get("spark.network.timeout", "120s")); + long defaultTimeoutMs = JavaUtils.timeStringAsSec( + conf.get(SPARK_NETWORK_IO_CONNECTIONTIMEOUT_KEY, defaultNetworkTimeoutS + "s")) * 1000; + return (int) defaultTimeoutMs; + } + + /** Number of concurrent connections between two nodes for fetching data. */ + public int numConnectionsPerPeer() { + return conf.getInt(SPARK_NETWORK_IO_NUMCONNECTIONSPERPEER_KEY, 1); + } + + /** Requested maximum length of the queue of incoming connections. Default -1 for no backlog. */ + public int backLog() { return conf.getInt(SPARK_NETWORK_IO_BACKLOG_KEY, -1); } + + /** Number of threads used in the server thread pool. Default to 0, which is 2x#cores. */ + public int serverThreads() { return conf.getInt(SPARK_NETWORK_IO_SERVERTHREADS_KEY, 0); } + + /** Number of threads used in the client thread pool. Default to 0, which is 2x#cores. */ + public int clientThreads() { return conf.getInt(SPARK_NETWORK_IO_CLIENTTHREADS_KEY, 0); } + + /** + * Receive buffer size (SO_RCVBUF). + * Note: the optimal size for receive buffer and send buffer should be + * latency * network_bandwidth. + * Assuming latency = 1ms, network_bandwidth = 10Gbps + * buffer size should be ~ 1.25MB + */ + public int receiveBuf() { return conf.getInt(SPARK_NETWORK_IO_RECEIVEBUFFER_KEY, -1); } + + /** Send buffer size (SO_SNDBUF). */ + public int sendBuf() { return conf.getInt(SPARK_NETWORK_IO_SENDBUFFER_KEY, -1); } + + /** Timeout for a single round trip of auth message exchange, in milliseconds. */ + public int authRTTimeoutMs() { + return (int) JavaUtils.timeStringAsSec(conf.get("spark.network.auth.rpcTimeout", + conf.get(SPARK_NETWORK_SASL_TIMEOUT_KEY, "30s"))) * 1000; + } + + /** + * Max number of times we will try IO exceptions (such as connection timeouts) per request. + * If set to 0, we will not do any retries. + */ + public int maxIORetries() { return conf.getInt(SPARK_NETWORK_IO_MAXRETRIES_KEY, 3); } + + /** + * Time (in milliseconds) that we will wait in order to perform a retry after an IOException. + * Only relevant if maxIORetries > 0. + */ + public int ioRetryWaitTimeMs() { + return (int) JavaUtils.timeStringAsSec(conf.get(SPARK_NETWORK_IO_RETRYWAIT_KEY, "5s")) * 1000; + } + + /** + * Minimum size of a block that we should start using memory map rather than reading in through + * normal IO operations. This prevents Spark from memory mapping very small blocks. In general, + * memory mapping has high overhead for blocks close to or below the page size of the OS. + */ + public int memoryMapBytes() { + return Ints.checkedCast(JavaUtils.byteStringAsBytes( + conf.get("spark.storage.memoryMapThreshold", "2m"))); + } + + /** + * Whether to initialize FileDescriptor lazily or not. If true, file descriptors are + * created only when data is going to be transferred. This can reduce the number of open files. + */ + public boolean lazyFileDescriptor() { + return conf.getBoolean(SPARK_NETWORK_IO_LAZYFD_KEY, true); + } + + /** + * Whether to track Netty memory detailed metrics. If true, the detailed metrics of Netty + * PoolByteBufAllocator will be gotten, otherwise only general memory usage will be tracked. + */ + public boolean verboseMetrics() { + return conf.getBoolean(SPARK_NETWORK_VERBOSE_METRICS, false); + } + + /** + * Maximum number of retries when binding to a port before giving up. + */ + public int portMaxRetries() { + return conf.getInt("spark.port.maxRetries", 16); + } + + /** + * Enables strong encryption. Also enables the new auth protocol, used to negotiate keys. + */ + public boolean encryptionEnabled() { + return conf.getBoolean("spark.network.crypto.enabled", false); + } + + /** + * The cipher transformation to use for encrypting session data. + */ + public String cipherTransformation() { + return conf.get("spark.network.crypto.cipher", "AES/CTR/NoPadding"); + } + + /** + * The key generation algorithm. This should be an algorithm that accepts a "PBEKeySpec" + * as input. The default value (PBKDF2WithHmacSHA1) is available in Java 7. + */ + public String keyFactoryAlgorithm() { + return conf.get("spark.network.crypto.keyFactoryAlgorithm", "PBKDF2WithHmacSHA1"); + } + + /** + * How many iterations to run when generating keys. + * + * See some discussion about this at: http://security.stackexchange.com/q/3959 + * The default value was picked for speed, since it assumes that the secret has good entropy + * (128 bits by default), which is not generally the case with user passwords. + */ + public int keyFactoryIterations() { + return conf.getInt("spark.networy.crypto.keyFactoryIterations", 1024); + } + + /** + * Encryption key length, in bits. + */ + public int encryptionKeyLength() { + return conf.getInt("spark.network.crypto.keyLength", 128); + } + + /** + * Initial vector length, in bytes. + */ + public int ivLength() { + return conf.getInt("spark.network.crypto.ivLength", 16); + } + + /** + * The algorithm for generated secret keys. Nobody should really need to change this, + * but configurable just in case. + */ + public String keyAlgorithm() { + return conf.get("spark.network.crypto.keyAlgorithm", "AES"); + } + + /** + * Whether to fall back to SASL if the new auth protocol fails. Enabled by default for + * backwards compatibility. + */ + public boolean saslFallback() { + return conf.getBoolean("spark.network.crypto.saslFallback", true); + } + + /** + * Whether to enable SASL-based encryption when authenticating using SASL. + */ + public boolean saslEncryption() { + return conf.getBoolean("spark.authenticate.enableSaslEncryption", false); + } + + /** + * Maximum number of bytes to be encrypted at a time when SASL encryption is used. + */ + public int maxSaslEncryptedBlockSize() { + return Ints.checkedCast(JavaUtils.byteStringAsBytes( + conf.get("spark.network.sasl.maxEncryptedBlockSize", "64k"))); + } + + /** + * Whether the server should enforce encryption on SASL-authenticated connections. + */ + public boolean saslServerAlwaysEncrypt() { + return conf.getBoolean("spark.network.sasl.serverAlwaysEncrypt", false); + } + + /** + * The commons-crypto configuration for the module. + */ + public Properties cryptoConf() { + return CryptoUtils.toCryptoConf("spark.network.crypto.config.", conf.getAll()); + } + + /** + * The max number of chunks allowed to be transferred at the same time on shuffle service. + * Note that new incoming connections will be closed when the max number is hit. The client will + * retry according to the shuffle retry configs (see `spark.shuffle.io.maxRetries` and + * `spark.shuffle.io.retryWait`), if those limits are reached the task will fail with fetch + * failure. + */ + public long maxChunksBeingTransferred() { + return conf.getLong("spark.shuffle.maxChunksBeingTransferred", Long.MAX_VALUE); + } +} diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java new file mode 100644 index 000000000000..50d9651ccbbb --- /dev/null +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.util.LinkedList; + +import com.google.common.base.Preconditions; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; + +/** + * A customized frame decoder that allows intercepting raw data. + *

+ * This behaves like Netty's frame decoder (with harcoded parameters that match this library's + * needs), except it allows an interceptor to be installed to read data directly before it's + * framed. + *

+ * Unlike Netty's frame decoder, each frame is dispatched to child handlers as soon as it's + * decoded, instead of building as many frames as the current buffer allows and dispatching + * all of them. This allows a child handler to install an interceptor if needed. + *

+ * If an interceptor is installed, framing stops, and data is instead fed directly to the + * interceptor. When the interceptor indicates that it doesn't need to read any more data, + * framing resumes. Interceptors should not hold references to the data buffers provided + * to their handle() method. + */ +public class TransportFrameDecoder extends ChannelInboundHandlerAdapter { + + public static final String HANDLER_NAME = "frameDecoder"; + private static final int LENGTH_SIZE = 8; + private static final int MAX_FRAME_SIZE = Integer.MAX_VALUE; + private static final int UNKNOWN_FRAME_SIZE = -1; + + private final LinkedList buffers = new LinkedList<>(); + private final ByteBuf frameLenBuf = Unpooled.buffer(LENGTH_SIZE, LENGTH_SIZE); + + private long totalSize = 0; + private long nextFrameSize = UNKNOWN_FRAME_SIZE; + private volatile Interceptor interceptor; + + @Override + public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception { + ByteBuf in = (ByteBuf) data; + buffers.add(in); + totalSize += in.readableBytes(); + + while (!buffers.isEmpty()) { + // First, feed the interceptor, and if it's still, active, try again. + if (interceptor != null) { + ByteBuf first = buffers.getFirst(); + int available = first.readableBytes(); + if (feedInterceptor(first)) { + assert !first.isReadable() : "Interceptor still active but buffer has data."; + } + + int read = available - first.readableBytes(); + if (read == available) { + buffers.removeFirst().release(); + } + totalSize -= read; + } else { + // Interceptor is not active, so try to decode one frame. + ByteBuf frame = decodeNext(); + if (frame == null) { + break; + } + ctx.fireChannelRead(frame); + } + } + } + + private long decodeFrameSize() { + if (nextFrameSize != UNKNOWN_FRAME_SIZE || totalSize < LENGTH_SIZE) { + return nextFrameSize; + } + + // We know there's enough data. If the first buffer contains all the data, great. Otherwise, + // hold the bytes for the frame length in a composite buffer until we have enough data to read + // the frame size. Normally, it should be rare to need more than one buffer to read the frame + // size. + ByteBuf first = buffers.getFirst(); + if (first.readableBytes() >= LENGTH_SIZE) { + nextFrameSize = first.readLong() - LENGTH_SIZE; + totalSize -= LENGTH_SIZE; + if (!first.isReadable()) { + buffers.removeFirst().release(); + } + return nextFrameSize; + } + + while (frameLenBuf.readableBytes() < LENGTH_SIZE) { + ByteBuf next = buffers.getFirst(); + int toRead = Math.min(next.readableBytes(), LENGTH_SIZE - frameLenBuf.readableBytes()); + frameLenBuf.writeBytes(next, toRead); + if (!next.isReadable()) { + buffers.removeFirst().release(); + } + } + + nextFrameSize = frameLenBuf.readLong() - LENGTH_SIZE; + totalSize -= LENGTH_SIZE; + frameLenBuf.clear(); + return nextFrameSize; + } + + private ByteBuf decodeNext() { + long frameSize = decodeFrameSize(); + if (frameSize == UNKNOWN_FRAME_SIZE || totalSize < frameSize) { + return null; + } + + // Reset size for next frame. + nextFrameSize = UNKNOWN_FRAME_SIZE; + + Preconditions.checkArgument(frameSize < MAX_FRAME_SIZE, "Too large frame: %s", frameSize); + Preconditions.checkArgument(frameSize > 0, "Frame length should be positive: %s", frameSize); + + // If the first buffer holds the entire frame, return it. + int remaining = (int) frameSize; + if (buffers.getFirst().readableBytes() >= remaining) { + return nextBufferForFrame(remaining); + } + + // Otherwise, create a composite buffer. + CompositeByteBuf frame = buffers.getFirst().alloc().compositeBuffer(Integer.MAX_VALUE); + while (remaining > 0) { + ByteBuf next = nextBufferForFrame(remaining); + remaining -= next.readableBytes(); + frame.addComponent(next).writerIndex(frame.writerIndex() + next.readableBytes()); + } + assert remaining == 0; + return frame; + } + + /** + * Takes the first buffer in the internal list, and either adjust it to fit in the frame + * (by taking a slice out of it) or remove it from the internal list. + */ + private ByteBuf nextBufferForFrame(int bytesToRead) { + ByteBuf buf = buffers.getFirst(); + ByteBuf frame; + + if (buf.readableBytes() > bytesToRead) { + frame = buf.retain().readSlice(bytesToRead); + totalSize -= bytesToRead; + } else { + frame = buf; + buffers.removeFirst(); + totalSize -= frame.readableBytes(); + } + + return frame; + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + for (ByteBuf b : buffers) { + b.release(); + } + if (interceptor != null) { + interceptor.channelInactive(); + } + frameLenBuf.release(); + super.channelInactive(ctx); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + if (interceptor != null) { + interceptor.exceptionCaught(cause); + } + super.exceptionCaught(ctx, cause); + } + + public void setInterceptor(Interceptor interceptor) { + Preconditions.checkState(this.interceptor == null, "Already have an interceptor."); + this.interceptor = interceptor; + } + + /** + * @return Whether the interceptor is still active after processing the data. + */ + private boolean feedInterceptor(ByteBuf buf) throws Exception { + if (interceptor != null && !interceptor.handle(buf)) { + interceptor = null; + } + return interceptor != null; + } + + public interface Interceptor { + + /** + * Handles data received from the remote end. + * + * @param data Buffer containing data. + * @return "true" if the interceptor expects more data, "false" to uninstall the interceptor. + */ + boolean handle(ByteBuf data) throws Exception; + + /** Called if an exception is thrown in the channel pipeline. */ + void exceptionCaught(Throwable cause) throws Exception; + + /** Called if the channel is closed and the interceptor is still installed. */ + void channelInactive() throws Exception; + + } + +} diff --git a/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java similarity index 76% rename from network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java rename to common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java index dfb7740344ed..824482af08dd 100644 --- a/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java @@ -20,6 +20,7 @@ import java.io.File; import java.io.RandomAccessFile; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.LinkedList; @@ -29,8 +30,8 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; -import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import com.google.common.io.Closeables; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -47,7 +48,7 @@ import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.TransportServer; import org.apache.spark.network.server.StreamManager; -import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; public class ChunkFetchIntegrationSuite { @@ -63,8 +64,6 @@ public class ChunkFetchIntegrationSuite { static ManagedBuffer bufferChunk; static ManagedBuffer fileChunk; - private TransportConf transportConf; - @BeforeClass public static void setUp() throws Exception { int bufSize = 100000; @@ -78,12 +77,17 @@ public static void setUp() throws Exception { testFile = File.createTempFile("shuffle-test-file", "txt"); testFile.deleteOnExit(); RandomAccessFile fp = new RandomAccessFile(testFile, "rw"); - byte[] fileContent = new byte[1024]; - new Random().nextBytes(fileContent); - fp.write(fileContent); - fp.close(); + boolean shouldSuppressIOException = true; + try { + byte[] fileContent = new byte[1024]; + new Random().nextBytes(fileContent); + fp.write(fileContent); + shouldSuppressIOException = false; + } finally { + Closeables.close(fp, shouldSuppressIOException); + } - final TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + final TransportConf conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); fileChunk = new FileSegmentManagedBuffer(conf, testFile, 10, testFile.length() - 25); streamManager = new StreamManager() { @@ -101,7 +105,10 @@ public ManagedBuffer getChunk(long streamId, int chunkIndex) { }; RpcHandler handler = new RpcHandler() { @Override - public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { throw new UnsupportedOperationException(); } @@ -117,12 +124,13 @@ public StreamManager getStreamManager() { @AfterClass public static void tearDown() { + bufferChunk.release(); server.close(); clientFactory.close(); testFile.delete(); } - class FetchResult { + static class FetchResult { public Set successChunks; public Set failedChunks; public List buffers; @@ -171,49 +179,49 @@ public void onFailure(int chunkIndex, Throwable e) { @Test public void fetchBufferChunk() throws Exception { - FetchResult res = fetchChunks(Lists.newArrayList(BUFFER_CHUNK_INDEX)); - assertEquals(res.successChunks, Sets.newHashSet(BUFFER_CHUNK_INDEX)); + FetchResult res = fetchChunks(Arrays.asList(BUFFER_CHUNK_INDEX)); + assertEquals(Sets.newHashSet(BUFFER_CHUNK_INDEX), res.successChunks); assertTrue(res.failedChunks.isEmpty()); - assertBufferListsEqual(res.buffers, Lists.newArrayList(bufferChunk)); + assertBufferListsEqual(Arrays.asList(bufferChunk), res.buffers); res.releaseBuffers(); } @Test public void fetchFileChunk() throws Exception { - FetchResult res = fetchChunks(Lists.newArrayList(FILE_CHUNK_INDEX)); - assertEquals(res.successChunks, Sets.newHashSet(FILE_CHUNK_INDEX)); + FetchResult res = fetchChunks(Arrays.asList(FILE_CHUNK_INDEX)); + assertEquals(Sets.newHashSet(FILE_CHUNK_INDEX), res.successChunks); assertTrue(res.failedChunks.isEmpty()); - assertBufferListsEqual(res.buffers, Lists.newArrayList(fileChunk)); + assertBufferListsEqual(Arrays.asList(fileChunk), res.buffers); res.releaseBuffers(); } @Test public void fetchNonExistentChunk() throws Exception { - FetchResult res = fetchChunks(Lists.newArrayList(12345)); + FetchResult res = fetchChunks(Arrays.asList(12345)); assertTrue(res.successChunks.isEmpty()); - assertEquals(res.failedChunks, Sets.newHashSet(12345)); + assertEquals(Sets.newHashSet(12345), res.failedChunks); assertTrue(res.buffers.isEmpty()); } @Test public void fetchBothChunks() throws Exception { - FetchResult res = fetchChunks(Lists.newArrayList(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX)); - assertEquals(res.successChunks, Sets.newHashSet(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX)); + FetchResult res = fetchChunks(Arrays.asList(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX)); + assertEquals(Sets.newHashSet(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX), res.successChunks); assertTrue(res.failedChunks.isEmpty()); - assertBufferListsEqual(res.buffers, Lists.newArrayList(bufferChunk, fileChunk)); + assertBufferListsEqual(Arrays.asList(bufferChunk, fileChunk), res.buffers); res.releaseBuffers(); } @Test public void fetchChunkAndNonExistent() throws Exception { - FetchResult res = fetchChunks(Lists.newArrayList(BUFFER_CHUNK_INDEX, 12345)); - assertEquals(res.successChunks, Sets.newHashSet(BUFFER_CHUNK_INDEX)); - assertEquals(res.failedChunks, Sets.newHashSet(12345)); - assertBufferListsEqual(res.buffers, Lists.newArrayList(bufferChunk)); + FetchResult res = fetchChunks(Arrays.asList(BUFFER_CHUNK_INDEX, 12345)); + assertEquals(Sets.newHashSet(BUFFER_CHUNK_INDEX), res.successChunks); + assertEquals(Sets.newHashSet(12345), res.failedChunks); + assertBufferListsEqual(Arrays.asList(bufferChunk), res.buffers); res.releaseBuffers(); } - private void assertBufferListsEqual(List list0, List list1) + private static void assertBufferListsEqual(List list0, List list1) throws Exception { assertEquals(list0.size(), list1.size()); for (int i = 0; i < list0.size(); i ++) { @@ -221,7 +229,8 @@ private void assertBufferListsEqual(List list0, List configMap = new HashMap<>(); + configMap.put("spark.shuffle.io.connectionTimeout", "10s"); + conf = new TransportConf("shuffle", new MapConfigProvider(configMap)); + + defaultManager = new StreamManager() { + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + throw new UnsupportedOperationException(); + } + }; + } + + @After + public void tearDown() { + if (server != null) { + server.close(); + } + if (clientFactory != null) { + clientFactory.close(); + } + } + + // Basic suite: First request completes quickly, and second waits for longer than network timeout. + @Test + public void timeoutInactiveRequests() throws Exception { + final Semaphore semaphore = new Semaphore(1); + final int responseSize = 16; + RpcHandler handler = new RpcHandler() { + @Override + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + try { + semaphore.acquire(); + callback.onSuccess(ByteBuffer.allocate(responseSize)); + } catch (InterruptedException e) { + // do nothing + } + } + + @Override + public StreamManager getStreamManager() { + return defaultManager; + } + }; + + TransportContext context = new TransportContext(conf, handler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + + // First completes quickly (semaphore starts at 1). + TestCallback callback0 = new TestCallback(); + client.sendRpc(ByteBuffer.allocate(0), callback0); + callback0.latch.await(); + assertEquals(responseSize, callback0.successLength); + + // Second times out after 10 seconds, with slack. Must be IOException. + TestCallback callback1 = new TestCallback(); + client.sendRpc(ByteBuffer.allocate(0), callback1); + callback1.latch.await(60, TimeUnit.SECONDS); + assertNotNull(callback1.failure); + assertTrue(callback1.failure instanceof IOException); + + semaphore.release(); + } + + // A timeout will cause the connection to be closed, invalidating the current TransportClient. + // It should be the case that requesting a client from the factory produces a new, valid one. + @Test + public void timeoutCleanlyClosesClient() throws Exception { + final Semaphore semaphore = new Semaphore(0); + final int responseSize = 16; + RpcHandler handler = new RpcHandler() { + @Override + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + try { + semaphore.acquire(); + callback.onSuccess(ByteBuffer.allocate(responseSize)); + } catch (InterruptedException e) { + // do nothing + } + } + + @Override + public StreamManager getStreamManager() { + return defaultManager; + } + }; + + TransportContext context = new TransportContext(conf, handler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + + // First request should eventually fail. + TransportClient client0 = + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + TestCallback callback0 = new TestCallback(); + client0.sendRpc(ByteBuffer.allocate(0), callback0); + callback0.latch.await(); + assertTrue(callback0.failure instanceof IOException); + assertFalse(client0.isActive()); + + // Increment the semaphore and the second request should succeed quickly. + semaphore.release(2); + TransportClient client1 = + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + TestCallback callback1 = new TestCallback(); + client1.sendRpc(ByteBuffer.allocate(0), callback1); + callback1.latch.await(); + assertEquals(responseSize, callback1.successLength); + assertNull(callback1.failure); + } + + // The timeout is relative to the LAST request sent, which is kinda weird, but still. + // This test also makes sure the timeout works for Fetch requests as well as RPCs. + @Test + public void furtherRequestsDelay() throws Exception { + final byte[] response = new byte[16]; + final StreamManager manager = new StreamManager() { + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + Uninterruptibles.sleepUninterruptibly(FOREVER, TimeUnit.MILLISECONDS); + return new NioManagedBuffer(ByteBuffer.wrap(response)); + } + }; + RpcHandler handler = new RpcHandler() { + @Override + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + throw new UnsupportedOperationException(); + } + + @Override + public StreamManager getStreamManager() { + return manager; + } + }; + + TransportContext context = new TransportContext(conf, handler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + + // Send one request, which will eventually fail. + TestCallback callback0 = new TestCallback(); + client.fetchChunk(0, 0, callback0); + Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS); + + // Send a second request before the first has failed. + TestCallback callback1 = new TestCallback(); + client.fetchChunk(0, 1, callback1); + Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS); + + // not complete yet, but should complete soon + assertEquals(-1, callback0.successLength); + assertNull(callback0.failure); + callback0.latch.await(60, TimeUnit.SECONDS); + assertTrue(callback0.failure instanceof IOException); + + // make sure callback1 is called. + callback1.latch.await(60, TimeUnit.SECONDS); + // failed at same time as previous + assertTrue(callback1.failure instanceof IOException); + } + + /** + * Callback which sets 'success' or 'failure' on completion. + * Additionally notifies all waiters on this callback when invoked. + */ + static class TestCallback implements RpcResponseCallback, ChunkReceivedCallback { + + int successLength = -1; + Throwable failure; + final CountDownLatch latch = new CountDownLatch(1); + + @Override + public void onSuccess(ByteBuffer response) { + successLength = response.remaining(); + latch.countDown(); + } + + @Override + public void onFailure(Throwable e) { + failure = e; + latch.countDown(); + } + + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + try { + successLength = buffer.nioByteBuffer().remaining(); + } catch (IOException e) { + // weird + } finally { + latch.countDown(); + } + } + + @Override + public void onFailure(int chunkIndex, Throwable e) { + failure = e; + latch.countDown(); + } + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java similarity index 76% rename from network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java rename to common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java index 64b457b4b3f0..8ff737b12964 100644 --- a/network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java @@ -17,14 +17,16 @@ package org.apache.spark.network; +import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.Set; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; -import com.google.common.base.Charsets; import com.google.common.collect.Sets; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -39,24 +41,29 @@ import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.StreamManager; import org.apache.spark.network.server.TransportServer; -import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; public class RpcIntegrationSuite { static TransportServer server; static TransportClientFactory clientFactory; static RpcHandler rpcHandler; + static List oneWayMsgs; @BeforeClass public static void setUp() throws Exception { - TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + TransportConf conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); rpcHandler = new RpcHandler() { @Override - public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { - String msg = new String(message, Charsets.UTF_8); + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + String msg = JavaUtils.bytesToString(message); String[] parts = msg.split("/"); if (parts[0].equals("hello")) { - callback.onSuccess(("Hello, " + parts[1] + "!").getBytes(Charsets.UTF_8)); + callback.onSuccess(JavaUtils.stringToBytes("Hello, " + parts[1] + "!")); } else if (parts[0].equals("return error")) { callback.onFailure(new RuntimeException("Returned: " + parts[1])); } else if (parts[0].equals("throw error")) { @@ -64,12 +71,18 @@ public void receive(TransportClient client, byte[] message, RpcResponseCallback } } + @Override + public void receive(TransportClient client, ByteBuffer message) { + oneWayMsgs.add(JavaUtils.bytesToString(message)); + } + @Override public StreamManager getStreamManager() { return new OneForOneStreamManager(); } }; TransportContext context = new TransportContext(conf, rpcHandler); server = context.createServer(); clientFactory = context.createClientFactory(); + oneWayMsgs = new ArrayList<>(); } @AfterClass @@ -78,7 +91,7 @@ public static void tearDown() { clientFactory.close(); } - class RpcResult { + static class RpcResult { public Set successMessages; public Set errorMessages; } @@ -93,8 +106,9 @@ private RpcResult sendRPC(String ... commands) throws Exception { RpcResponseCallback callback = new RpcResponseCallback() { @Override - public void onSuccess(byte[] message) { - res.successMessages.add(new String(message, Charsets.UTF_8)); + public void onSuccess(ByteBuffer message) { + String response = JavaUtils.bytesToString(message); + res.successMessages.add(response); sem.release(); } @@ -106,7 +120,7 @@ public void onFailure(Throwable e) { }; for (String command : commands) { - client.sendRpc(command.getBytes(Charsets.UTF_8), callback); + client.sendRpc(JavaUtils.stringToBytes(command), callback); } if (!sem.tryAcquire(commands.length, 5, TimeUnit.SECONDS)) { @@ -158,6 +172,27 @@ public void sendSuccessAndFailure() throws Exception { assertErrorsContain(res.errorMessages, Sets.newHashSet("Thrown: the", "Returned: !")); } + @Test + public void sendOneWayMessage() throws Exception { + final String message = "no reply"; + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + try { + client.send(JavaUtils.stringToBytes(message)); + assertEquals(0, client.getHandler().numOutstandingRequests()); + + // Make sure the message arrives. + long deadline = System.nanoTime() + TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS); + while (System.nanoTime() < deadline && oneWayMsgs.size() == 0) { + TimeUnit.MILLISECONDS.sleep(10); + } + + assertEquals(1, oneWayMsgs.size()); + assertEquals(message, oneWayMsgs.get(0)); + } finally { + client.close(); + } + } + private void assertErrorsContain(Set errors, Set contains) { assertEquals(contains.size(), errors.size()); diff --git a/network/common/src/test/java/org/apache/spark/network/StreamSuite.java b/common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java similarity index 90% rename from network/common/src/test/java/org/apache/spark/network/StreamSuite.java rename to common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java index 6dcec831dec7..f253a07e64be 100644 --- a/network/common/src/test/java/org/apache/spark/network/StreamSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java @@ -47,17 +47,18 @@ import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.StreamManager; import org.apache.spark.network.server.TransportServer; -import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; public class StreamSuite { - private static final String[] STREAMS = { "largeBuffer", "smallBuffer", "file" }; + private static final String[] STREAMS = { "largeBuffer", "smallBuffer", "emptyBuffer", "file" }; private static TransportServer server; private static TransportClientFactory clientFactory; private static File testFile; private static File tempDir; + private static ByteBuffer emptyBuffer; private static ByteBuffer smallBuffer; private static ByteBuffer largeBuffer; @@ -73,6 +74,7 @@ private static ByteBuffer createBuffer(int bufSize) { @BeforeClass public static void setUp() throws Exception { tempDir = Files.createTempDir(); + emptyBuffer = createBuffer(0); smallBuffer = createBuffer(100); largeBuffer = createBuffer(100000); @@ -89,7 +91,7 @@ public static void setUp() throws Exception { fp.close(); } - final TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + final TransportConf conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); final StreamManager streamManager = new StreamManager() { @Override public ManagedBuffer getChunk(long streamId, int chunkIndex) { @@ -103,6 +105,8 @@ public ManagedBuffer openStream(String streamId) { return new NioManagedBuffer(largeBuffer); case "smallBuffer": return new NioManagedBuffer(smallBuffer); + case "emptyBuffer": + return new NioManagedBuffer(emptyBuffer); case "file": return new FileSegmentManagedBuffer(conf, testFile, 0, testFile.length()); default: @@ -112,7 +116,10 @@ public ManagedBuffer openStream(String streamId) { }; RpcHandler handler = new RpcHandler() { @Override - public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { throw new UnsupportedOperationException(); } @@ -138,6 +145,18 @@ public static void tearDown() { } } + @Test + public void testZeroLengthStream() throws Throwable { + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + try { + StreamTask task = new StreamTask(client, "emptyBuffer", TimeUnit.SECONDS.toMillis(5)); + task.run(); + task.check(); + } finally { + client.close(); + } + } + @Test public void testSingleStream() throws Throwable { TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); @@ -226,6 +245,11 @@ public void run() { outFile = File.createTempFile("data", ".tmp", tempDir); out = new FileOutputStream(outFile); break; + case "emptyBuffer": + baos = new ByteArrayOutputStream(); + out = baos; + srcBuffer = emptyBuffer; + break; default: throw new IllegalArgumentException(streamId); } diff --git a/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java b/common/network-common/src/test/java/org/apache/spark/network/TestManagedBuffer.java similarity index 100% rename from network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java rename to common/network-common/src/test/java/org/apache/spark/network/TestManagedBuffer.java diff --git a/network/common/src/test/java/org/apache/spark/network/TestUtils.java b/common/network-common/src/test/java/org/apache/spark/network/TestUtils.java similarity index 100% rename from network/common/src/test/java/org/apache/spark/network/TestUtils.java rename to common/network-common/src/test/java/org/apache/spark/network/TestUtils.java diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java new file mode 100644 index 000000000000..e95d25fe6ae9 --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -0,0 +1,218 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertTrue; + +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.server.NoOpRpcHandler; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.util.ConfigProvider; +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.TransportConf; + +public class TransportClientFactorySuite { + private TransportConf conf; + private TransportContext context; + private TransportServer server1; + private TransportServer server2; + + @Before + public void setUp() { + conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); + RpcHandler rpcHandler = new NoOpRpcHandler(); + context = new TransportContext(conf, rpcHandler); + server1 = context.createServer(); + server2 = context.createServer(); + } + + @After + public void tearDown() { + JavaUtils.closeQuietly(server1); + JavaUtils.closeQuietly(server2); + } + + /** + * Request a bunch of clients to a single server to test + * we create up to maxConnections of clients. + * + * If concurrent is true, create multiple threads to create clients in parallel. + */ + private void testClientReuse(int maxConnections, boolean concurrent) + throws IOException, InterruptedException { + + Map configMap = new HashMap<>(); + configMap.put("spark.shuffle.io.numConnectionsPerPeer", Integer.toString(maxConnections)); + TransportConf conf = new TransportConf("shuffle", new MapConfigProvider(configMap)); + + RpcHandler rpcHandler = new NoOpRpcHandler(); + TransportContext context = new TransportContext(conf, rpcHandler); + TransportClientFactory factory = context.createClientFactory(); + Set clients = Collections.synchronizedSet( + new HashSet()); + + AtomicInteger failed = new AtomicInteger(); + Thread[] attempts = new Thread[maxConnections * 10]; + + // Launch a bunch of threads to create new clients. + for (int i = 0; i < attempts.length; i++) { + attempts[i] = new Thread(() -> { + try { + TransportClient client = + factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + assertTrue(client.isActive()); + clients.add(client); + } catch (IOException e) { + failed.incrementAndGet(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + + if (concurrent) { + attempts[i].start(); + } else { + attempts[i].run(); + } + } + + // Wait until all the threads complete. + for (Thread attempt : attempts) { + attempt.join(); + } + + Assert.assertEquals(0, failed.get()); + Assert.assertEquals(clients.size(), maxConnections); + + for (TransportClient client : clients) { + client.close(); + } + + factory.close(); + } + + @Test + public void reuseClientsUpToConfigVariable() throws Exception { + testClientReuse(1, false); + testClientReuse(2, false); + testClientReuse(3, false); + testClientReuse(4, false); + } + + @Test + public void reuseClientsUpToConfigVariableConcurrent() throws Exception { + testClientReuse(1, true); + testClientReuse(2, true); + testClientReuse(3, true); + testClientReuse(4, true); + } + + @Test + public void returnDifferentClientsForDifferentServers() throws IOException, InterruptedException { + TransportClientFactory factory = context.createClientFactory(); + TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); + assertTrue(c1.isActive()); + assertTrue(c2.isActive()); + assertNotSame(c1, c2); + factory.close(); + } + + @Test + public void neverReturnInactiveClients() throws IOException, InterruptedException { + TransportClientFactory factory = context.createClientFactory(); + TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + c1.close(); + + long start = System.currentTimeMillis(); + while (c1.isActive() && (System.currentTimeMillis() - start) < 3000) { + Thread.sleep(10); + } + assertFalse(c1.isActive()); + + TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + assertNotSame(c1, c2); + assertTrue(c2.isActive()); + factory.close(); + } + + @Test + public void closeBlockClientsWithFactory() throws IOException, InterruptedException { + TransportClientFactory factory = context.createClientFactory(); + TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); + assertTrue(c1.isActive()); + assertTrue(c2.isActive()); + factory.close(); + assertFalse(c1.isActive()); + assertFalse(c2.isActive()); + } + + @Test + public void closeIdleConnectionForRequestTimeOut() throws IOException, InterruptedException { + TransportConf conf = new TransportConf("shuffle", new ConfigProvider() { + + @Override + public String get(String name) { + if ("spark.shuffle.io.connectionTimeout".equals(name)) { + // We should make sure there is enough time for us to observe the channel is active + return "1s"; + } + String value = System.getProperty(name); + if (value == null) { + throw new NoSuchElementException(name); + } + return value; + } + + @Override + public Iterable> getAll() { + throw new UnsupportedOperationException(); + } + }); + TransportContext context = new TransportContext(conf, new NoOpRpcHandler(), true); + try (TransportClientFactory factory = context.createClientFactory()) { + TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + assertTrue(c1.isActive()); + long expiredTime = System.currentTimeMillis() + 10000; // 10 seconds + while (c1.isActive() && System.currentTimeMillis() < expiredTime) { + Thread.sleep(10); + } + assertFalse(c1.isActive()); + } + } +} diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java new file mode 100644 index 000000000000..2656cbee95a2 --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportRequestHandlerSuite.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import java.util.ArrayList; +import java.util.List; + +import io.netty.channel.Channel; +import io.netty.channel.ChannelPromise; +import io.netty.channel.DefaultChannelPromise; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GenericFutureListener; +import org.junit.Test; + +import static org.mockito.Mockito.*; + +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.protocol.*; +import org.apache.spark.network.server.NoOpRpcHandler; +import org.apache.spark.network.server.OneForOneStreamManager; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportRequestHandler; + +public class TransportRequestHandlerSuite { + + @Test + public void handleFetchRequestAndStreamRequest() throws Exception { + RpcHandler rpcHandler = new NoOpRpcHandler(); + OneForOneStreamManager streamManager = (OneForOneStreamManager) (rpcHandler.getStreamManager()); + Channel channel = mock(Channel.class); + List> responseAndPromisePairs = + new ArrayList<>(); + when(channel.writeAndFlush(any())) + .thenAnswer(invocationOnMock0 -> { + Object response = invocationOnMock0.getArguments()[0]; + ExtendedChannelPromise channelFuture = new ExtendedChannelPromise(channel); + responseAndPromisePairs.add(ImmutablePair.of(response, channelFuture)); + return channelFuture; + }); + + // Prepare the stream. + List managedBuffers = new ArrayList<>(); + managedBuffers.add(new TestManagedBuffer(10)); + managedBuffers.add(new TestManagedBuffer(20)); + managedBuffers.add(new TestManagedBuffer(30)); + managedBuffers.add(new TestManagedBuffer(40)); + long streamId = streamManager.registerStream("test-app", managedBuffers.iterator()); + streamManager.registerChannel(channel, streamId); + TransportClient reverseClient = mock(TransportClient.class); + TransportRequestHandler requestHandler = new TransportRequestHandler(channel, reverseClient, + rpcHandler, 2L); + + RequestMessage request0 = new ChunkFetchRequest(new StreamChunkId(streamId, 0)); + requestHandler.handle(request0); + assert responseAndPromisePairs.size() == 1; + assert responseAndPromisePairs.get(0).getLeft() instanceof ChunkFetchSuccess; + assert ((ChunkFetchSuccess) (responseAndPromisePairs.get(0).getLeft())).body() == + managedBuffers.get(0); + + RequestMessage request1 = new ChunkFetchRequest(new StreamChunkId(streamId, 1)); + requestHandler.handle(request1); + assert responseAndPromisePairs.size() == 2; + assert responseAndPromisePairs.get(1).getLeft() instanceof ChunkFetchSuccess; + assert ((ChunkFetchSuccess) (responseAndPromisePairs.get(1).getLeft())).body() == + managedBuffers.get(1); + + // Finish flushing the response for request0. + responseAndPromisePairs.get(0).getRight().finish(true); + + RequestMessage request2 = new StreamRequest(String.format("%d_%d", streamId, 2)); + requestHandler.handle(request2); + assert responseAndPromisePairs.size() == 3; + assert responseAndPromisePairs.get(2).getLeft() instanceof StreamResponse; + assert ((StreamResponse) (responseAndPromisePairs.get(2).getLeft())).body() == + managedBuffers.get(2); + + // Request3 will trigger the close of channel, because the number of max chunks being + // transferred is 2; + RequestMessage request3 = new StreamRequest(String.format("%d_%d", streamId, 3)); + requestHandler.handle(request3); + verify(channel, times(1)).close(); + assert responseAndPromisePairs.size() == 3; + } + + private class ExtendedChannelPromise extends DefaultChannelPromise { + + private List>> listeners = new ArrayList<>(); + private boolean success; + + ExtendedChannelPromise(Channel channel) { + super(channel); + success = false; + } + + @Override + public ChannelPromise addListener( + GenericFutureListener> listener) { + @SuppressWarnings("unchecked") + GenericFutureListener> gfListener = + (GenericFutureListener>) listener; + listeners.add(gfListener); + return super.addListener(listener); + } + + @Override + public boolean isSuccess() { + return success; + } + + public void finish(boolean success) { + this.success = success; + listeners.forEach(listener -> { + try { + listener.operationComplete(this); + } catch (Exception e) { + // do nothing + } + }); + } + } +} diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java new file mode 100644 index 000000000000..b4032c4c3f03 --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java @@ -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. + */ + +package org.apache.spark.network; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import io.netty.channel.Channel; +import io.netty.channel.local.LocalChannel; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.*; + +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.StreamCallback; +import org.apache.spark.network.client.TransportResponseHandler; +import org.apache.spark.network.protocol.ChunkFetchFailure; +import org.apache.spark.network.protocol.ChunkFetchSuccess; +import org.apache.spark.network.protocol.RpcFailure; +import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.protocol.StreamChunkId; +import org.apache.spark.network.protocol.StreamFailure; +import org.apache.spark.network.protocol.StreamResponse; +import org.apache.spark.network.util.TransportFrameDecoder; + +public class TransportResponseHandlerSuite { + @Test + public void handleSuccessfulFetch() throws Exception { + StreamChunkId streamChunkId = new StreamChunkId(1, 0); + + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); + handler.addFetchRequest(streamChunkId, callback); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new ChunkFetchSuccess(streamChunkId, new TestManagedBuffer(123))); + verify(callback, times(1)).onSuccess(eq(0), any()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void handleFailedFetch() throws Exception { + StreamChunkId streamChunkId = new StreamChunkId(1, 0); + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); + handler.addFetchRequest(streamChunkId, callback); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new ChunkFetchFailure(streamChunkId, "some error msg")); + verify(callback, times(1)).onFailure(eq(0), any()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void clearAllOutstandingRequests() throws Exception { + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); + handler.addFetchRequest(new StreamChunkId(1, 0), callback); + handler.addFetchRequest(new StreamChunkId(1, 1), callback); + handler.addFetchRequest(new StreamChunkId(1, 2), callback); + assertEquals(3, handler.numOutstandingRequests()); + + handler.handle(new ChunkFetchSuccess(new StreamChunkId(1, 0), new TestManagedBuffer(12))); + handler.exceptionCaught(new Exception("duh duh duhhhh")); + + // should fail both b2 and b3 + verify(callback, times(1)).onSuccess(eq(0), any()); + verify(callback, times(1)).onFailure(eq(1), any()); + verify(callback, times(1)).onFailure(eq(2), any()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void handleSuccessfulRPC() throws Exception { + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + RpcResponseCallback callback = mock(RpcResponseCallback.class); + handler.addRpcRequest(12345, callback); + assertEquals(1, handler.numOutstandingRequests()); + + // This response should be ignored. + handler.handle(new RpcResponse(54321, new NioManagedBuffer(ByteBuffer.allocate(7)))); + assertEquals(1, handler.numOutstandingRequests()); + + ByteBuffer resp = ByteBuffer.allocate(10); + handler.handle(new RpcResponse(12345, new NioManagedBuffer(resp))); + verify(callback, times(1)).onSuccess(eq(ByteBuffer.allocate(10))); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void handleFailedRPC() throws Exception { + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + RpcResponseCallback callback = mock(RpcResponseCallback.class); + handler.addRpcRequest(12345, callback); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new RpcFailure(54321, "uh-oh!")); // should be ignored + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new RpcFailure(12345, "oh no")); + verify(callback, times(1)).onFailure(any()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void testActiveStreams() throws Exception { + Channel c = new LocalChannel(); + c.pipeline().addLast(TransportFrameDecoder.HANDLER_NAME, new TransportFrameDecoder()); + TransportResponseHandler handler = new TransportResponseHandler(c); + + StreamResponse response = new StreamResponse("stream", 1234L, null); + StreamCallback cb = mock(StreamCallback.class); + handler.addStreamCallback("stream", cb); + assertEquals(1, handler.numOutstandingRequests()); + handler.handle(response); + assertEquals(1, handler.numOutstandingRequests()); + handler.deactivateStream(); + assertEquals(0, handler.numOutstandingRequests()); + + StreamFailure failure = new StreamFailure("stream", "uh-oh"); + handler.addStreamCallback("stream", cb); + assertEquals(1, handler.numOutstandingRequests()); + handler.handle(failure); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void failOutstandingStreamCallbackOnClose() throws Exception { + Channel c = new LocalChannel(); + c.pipeline().addLast(TransportFrameDecoder.HANDLER_NAME, new TransportFrameDecoder()); + TransportResponseHandler handler = new TransportResponseHandler(c); + + StreamCallback cb = mock(StreamCallback.class); + handler.addStreamCallback("stream-1", cb); + handler.channelInactive(); + + verify(cb).onFailure(eq("stream-1"), isA(IOException.class)); + } + + @Test + public void failOutstandingStreamCallbackOnException() throws Exception { + Channel c = new LocalChannel(); + c.pipeline().addLast(TransportFrameDecoder.HANDLER_NAME, new TransportFrameDecoder()); + TransportResponseHandler handler = new TransportResponseHandler(c); + + StreamCallback cb = mock(StreamCallback.class); + handler.addStreamCallback("stream-1", cb); + handler.exceptionCaught(new IOException("Oops!")); + + verify(cb).onFailure(eq("stream-1"), isA(IOException.class)); + } +} diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java new file mode 100644 index 000000000000..a3519fe4a423 --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.crypto; + +import java.util.Arrays; +import static java.nio.charset.StandardCharsets.UTF_8; + +import org.junit.BeforeClass; +import org.junit.Test; +import static org.junit.Assert.*; + +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.TransportConf; + +public class AuthEngineSuite { + + private static TransportConf conf; + + @BeforeClass + public static void setUp() { + conf = new TransportConf("rpc", MapConfigProvider.EMPTY); + } + + @Test + public void testAuthEngine() throws Exception { + AuthEngine client = new AuthEngine("appId", "secret", conf); + AuthEngine server = new AuthEngine("appId", "secret", conf); + + try { + ClientChallenge clientChallenge = client.challenge(); + ServerResponse serverResponse = server.respond(clientChallenge); + client.validate(serverResponse); + + TransportCipher serverCipher = server.sessionCipher(); + TransportCipher clientCipher = client.sessionCipher(); + + assertTrue(Arrays.equals(serverCipher.getInputIv(), clientCipher.getOutputIv())); + assertTrue(Arrays.equals(serverCipher.getOutputIv(), clientCipher.getInputIv())); + assertEquals(serverCipher.getKey(), clientCipher.getKey()); + } finally { + client.close(); + server.close(); + } + } + + @Test + public void testMismatchedSecret() throws Exception { + AuthEngine client = new AuthEngine("appId", "secret", conf); + AuthEngine server = new AuthEngine("appId", "different_secret", conf); + + ClientChallenge clientChallenge = client.challenge(); + try { + server.respond(clientChallenge); + fail("Should have failed to validate response."); + } catch (IllegalArgumentException e) { + // Expected. + } + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongAppId() throws Exception { + AuthEngine engine = new AuthEngine("appId", "secret", conf); + ClientChallenge challenge = engine.challenge(); + + byte[] badChallenge = engine.challenge(new byte[] { 0x00 }, challenge.nonce, + engine.rawResponse(engine.challenge)); + engine.respond(new ClientChallenge(challenge.appId, challenge.kdf, challenge.iterations, + challenge.cipher, challenge.keyLength, challenge.nonce, badChallenge)); + } + + @Test(expected = IllegalArgumentException.class) + public void testWrongNonce() throws Exception { + AuthEngine engine = new AuthEngine("appId", "secret", conf); + ClientChallenge challenge = engine.challenge(); + + byte[] badChallenge = engine.challenge(challenge.appId.getBytes(UTF_8), new byte[] { 0x00 }, + engine.rawResponse(engine.challenge)); + engine.respond(new ClientChallenge(challenge.appId, challenge.kdf, challenge.iterations, + challenge.cipher, challenge.keyLength, challenge.nonce, badChallenge)); + } + + @Test(expected = IllegalArgumentException.class) + public void testBadChallenge() throws Exception { + AuthEngine engine = new AuthEngine("appId", "secret", conf); + ClientChallenge challenge = engine.challenge(); + + byte[] badChallenge = new byte[challenge.challenge.length]; + engine.respond(new ClientChallenge(challenge.appId, challenge.kdf, challenge.iterations, + challenge.cipher, challenge.keyLength, challenge.nonce, badChallenge)); + } + +} diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java new file mode 100644 index 000000000000..8751944a1c2a --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.crypto; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import com.google.common.collect.ImmutableMap; +import io.netty.channel.Channel; +import org.junit.After; +import org.junit.Test; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import org.apache.spark.network.TestUtils; +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientBootstrap; +import org.apache.spark.network.sasl.SaslRpcHandler; +import org.apache.spark.network.sasl.SaslServerBootstrap; +import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.TransportServerBootstrap; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.TransportConf; + +public class AuthIntegrationSuite { + + private AuthTestCtx ctx; + + @After + public void cleanUp() throws Exception { + if (ctx != null) { + ctx.close(); + } + ctx = null; + } + + @Test + public void testNewAuth() throws Exception { + ctx = new AuthTestCtx(); + ctx.createServer("secret"); + ctx.createClient("secret"); + + ByteBuffer reply = ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), 5000); + assertEquals("Pong", JavaUtils.bytesToString(reply)); + assertTrue(ctx.authRpcHandler.doDelegate); + assertFalse(ctx.authRpcHandler.delegate instanceof SaslRpcHandler); + } + + @Test + public void testAuthFailure() throws Exception { + ctx = new AuthTestCtx(); + ctx.createServer("server"); + + try { + ctx.createClient("client"); + fail("Should have failed to create client."); + } catch (Exception e) { + assertFalse(ctx.authRpcHandler.doDelegate); + assertFalse(ctx.serverChannel.isActive()); + } + } + + @Test + public void testSaslServerFallback() throws Exception { + ctx = new AuthTestCtx(); + ctx.createServer("secret", true); + ctx.createClient("secret", false); + + ByteBuffer reply = ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), 5000); + assertEquals("Pong", JavaUtils.bytesToString(reply)); + } + + @Test + public void testSaslClientFallback() throws Exception { + ctx = new AuthTestCtx(); + ctx.createServer("secret", false); + ctx.createClient("secret", true); + + ByteBuffer reply = ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), 5000); + assertEquals("Pong", JavaUtils.bytesToString(reply)); + } + + @Test + public void testAuthReplay() throws Exception { + // This test covers the case where an attacker replays a challenge message sniffed from the + // network, but doesn't know the actual secret. The server should close the connection as + // soon as a message is sent after authentication is performed. This is emulated by removing + // the client encryption handler after authentication. + ctx = new AuthTestCtx(); + ctx.createServer("secret"); + ctx.createClient("secret"); + + assertNotNull(ctx.client.getChannel().pipeline() + .remove(TransportCipher.ENCRYPTION_HANDLER_NAME)); + + try { + ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), 5000); + fail("Should have failed unencrypted RPC."); + } catch (Exception e) { + assertTrue(ctx.authRpcHandler.doDelegate); + } + } + + private class AuthTestCtx { + + private final String appId = "testAppId"; + private final TransportConf conf; + private final TransportContext ctx; + + TransportClient client; + TransportServer server; + volatile Channel serverChannel; + volatile AuthRpcHandler authRpcHandler; + + AuthTestCtx() throws Exception { + Map testConf = ImmutableMap.of("spark.network.crypto.enabled", "true"); + this.conf = new TransportConf("rpc", new MapConfigProvider(testConf)); + + RpcHandler rpcHandler = new RpcHandler() { + @Override + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + assertEquals("Ping", JavaUtils.bytesToString(message)); + callback.onSuccess(JavaUtils.stringToBytes("Pong")); + } + + @Override + public StreamManager getStreamManager() { + return null; + } + }; + + this.ctx = new TransportContext(conf, rpcHandler); + } + + void createServer(String secret) throws Exception { + createServer(secret, true); + } + + void createServer(String secret, boolean enableAes) throws Exception { + TransportServerBootstrap introspector = (channel, rpcHandler) -> { + this.serverChannel = channel; + if (rpcHandler instanceof AuthRpcHandler) { + this.authRpcHandler = (AuthRpcHandler) rpcHandler; + } + return rpcHandler; + }; + SecretKeyHolder keyHolder = createKeyHolder(secret); + TransportServerBootstrap auth = enableAes ? new AuthServerBootstrap(conf, keyHolder) + : new SaslServerBootstrap(conf, keyHolder); + this.server = ctx.createServer(Arrays.asList(auth, introspector)); + } + + void createClient(String secret) throws Exception { + createClient(secret, true); + } + + void createClient(String secret, boolean enableAes) throws Exception { + TransportConf clientConf = enableAes ? conf + : new TransportConf("rpc", MapConfigProvider.EMPTY); + List bootstraps = Arrays.asList( + new AuthClientBootstrap(clientConf, appId, createKeyHolder(secret))); + this.client = ctx.createClientFactory(bootstraps) + .createClient(TestUtils.getLocalHost(), server.getPort()); + } + + void close() { + if (client != null) { + client.close(); + } + if (server != null) { + server.close(); + } + } + + private SecretKeyHolder createKeyHolder(String secret) { + SecretKeyHolder keyHolder = mock(SecretKeyHolder.class); + when(keyHolder.getSaslUser(anyString())).thenReturn(appId); + when(keyHolder.getSecretKey(anyString())).thenReturn(secret); + return keyHolder; + } + + } + +} diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthMessagesSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthMessagesSuite.java new file mode 100644 index 000000000000..a90ff247da4f --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthMessagesSuite.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.crypto; + +import java.nio.ByteBuffer; +import java.util.Arrays; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.junit.Test; +import static org.junit.Assert.*; + +import org.apache.spark.network.protocol.Encodable; + +public class AuthMessagesSuite { + + private static int COUNTER = 0; + + private static String string() { + return String.valueOf(COUNTER++); + } + + private static byte[] byteArray() { + byte[] bytes = new byte[COUNTER++]; + for (int i = 0; i < bytes.length; i++) { + bytes[i] = (byte) COUNTER; + } return bytes; + } + + private static int integer() { + return COUNTER++; + } + + @Test + public void testClientChallenge() { + ClientChallenge msg = new ClientChallenge(string(), string(), integer(), string(), integer(), + byteArray(), byteArray()); + ClientChallenge decoded = ClientChallenge.decodeMessage(encode(msg)); + + assertEquals(msg.appId, decoded.appId); + assertEquals(msg.kdf, decoded.kdf); + assertEquals(msg.iterations, decoded.iterations); + assertEquals(msg.cipher, decoded.cipher); + assertEquals(msg.keyLength, decoded.keyLength); + assertTrue(Arrays.equals(msg.nonce, decoded.nonce)); + assertTrue(Arrays.equals(msg.challenge, decoded.challenge)); + } + + @Test + public void testServerResponse() { + ServerResponse msg = new ServerResponse(byteArray(), byteArray(), byteArray(), byteArray()); + ServerResponse decoded = ServerResponse.decodeMessage(encode(msg)); + assertTrue(Arrays.equals(msg.response, decoded.response)); + assertTrue(Arrays.equals(msg.nonce, decoded.nonce)); + assertTrue(Arrays.equals(msg.inputIv, decoded.inputIv)); + assertTrue(Arrays.equals(msg.outputIv, decoded.outputIv)); + } + + private ByteBuffer encode(Encodable msg) { + ByteBuf buf = Unpooled.buffer(); + msg.encode(buf); + return buf.nioBuffer(); + } + +} diff --git a/common/network-common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java b/common/network-common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java new file mode 100644 index 000000000000..b341c5681e00 --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.FileRegion; +import io.netty.util.AbstractReferenceCounted; +import org.junit.Test; +import org.mockito.Mockito; + +import static org.junit.Assert.*; + +import org.apache.spark.network.TestManagedBuffer; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NettyManagedBuffer; +import org.apache.spark.network.util.ByteArrayWritableChannel; + +public class MessageWithHeaderSuite { + + @Test + public void testSingleWrite() throws Exception { + testFileRegionBody(8, 8); + } + + @Test + public void testShortWrite() throws Exception { + testFileRegionBody(8, 1); + } + + @Test + public void testByteBufBody() throws Exception { + ByteBuf header = Unpooled.copyLong(42); + ByteBuf bodyPassedToNettyManagedBuffer = Unpooled.copyLong(84); + assertEquals(1, header.refCnt()); + assertEquals(1, bodyPassedToNettyManagedBuffer.refCnt()); + ManagedBuffer managedBuf = new NettyManagedBuffer(bodyPassedToNettyManagedBuffer); + + Object body = managedBuf.convertToNetty(); + assertEquals(2, bodyPassedToNettyManagedBuffer.refCnt()); + assertEquals(1, header.refCnt()); + + MessageWithHeader msg = new MessageWithHeader(managedBuf, header, body, managedBuf.size()); + ByteBuf result = doWrite(msg, 1); + assertEquals(msg.count(), result.readableBytes()); + assertEquals(42, result.readLong()); + assertEquals(84, result.readLong()); + + assertTrue(msg.release()); + assertEquals(0, bodyPassedToNettyManagedBuffer.refCnt()); + assertEquals(0, header.refCnt()); + } + + @Test + public void testDeallocateReleasesManagedBuffer() throws Exception { + ByteBuf header = Unpooled.copyLong(42); + ManagedBuffer managedBuf = Mockito.spy(new TestManagedBuffer(84)); + ByteBuf body = (ByteBuf) managedBuf.convertToNetty(); + assertEquals(2, body.refCnt()); + MessageWithHeader msg = new MessageWithHeader(managedBuf, header, body, body.readableBytes()); + assertTrue(msg.release()); + Mockito.verify(managedBuf, Mockito.times(1)).release(); + assertEquals(0, body.refCnt()); + } + + private void testFileRegionBody(int totalWrites, int writesPerCall) throws Exception { + ByteBuf header = Unpooled.copyLong(42); + int headerLength = header.readableBytes(); + TestFileRegion region = new TestFileRegion(totalWrites, writesPerCall); + MessageWithHeader msg = new MessageWithHeader(null, header, region, region.count()); + + ByteBuf result = doWrite(msg, totalWrites / writesPerCall); + assertEquals(headerLength + region.count(), result.readableBytes()); + assertEquals(42, result.readLong()); + for (long i = 0; i < 8; i++) { + assertEquals(i, result.readLong()); + } + assertTrue(msg.release()); + } + + private ByteBuf doWrite(MessageWithHeader msg, int minExpectedWrites) throws Exception { + int writes = 0; + ByteArrayWritableChannel channel = new ByteArrayWritableChannel((int) msg.count()); + while (msg.transfered() < msg.count()) { + msg.transferTo(channel, msg.transfered()); + writes++; + } + assertTrue("Not enough writes!", minExpectedWrites <= writes); + return Unpooled.wrappedBuffer(channel.getData()); + } + + private static class TestFileRegion extends AbstractReferenceCounted implements FileRegion { + + private final int writeCount; + private final int writesPerCall; + private int written; + + TestFileRegion(int totalWrites, int writesPerCall) { + this.writeCount = totalWrites; + this.writesPerCall = writesPerCall; + } + + @Override + public long count() { + return 8 * writeCount; + } + + @Override + public long position() { + return 0; + } + + @Override + public long transfered() { + return 8 * written; + } + + @Override + public long transferTo(WritableByteChannel target, long position) throws IOException { + for (int i = 0; i < writesPerCall; i++) { + ByteBuf buf = Unpooled.copyLong((position / 8) + i); + ByteBuffer nio = buf.nioBuffer(); + while (nio.remaining() > 0) { + target.write(nio); + } + buf.release(); + written++; + } + return 8 * writesPerCall; + } + + @Override + protected void deallocate() { + } + + } + +} diff --git a/network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java similarity index 75% rename from network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java rename to common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java index 3469e84e7f4d..6f15718bd870 100644 --- a/network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java @@ -21,16 +21,21 @@ import static org.mockito.Mockito.*; import java.io.File; -import java.nio.charset.StandardCharsets; +import java.lang.reflect.Method; +import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Random; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import javax.security.sasl.SaslException; -import com.google.common.collect.Lists; +import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; import com.google.common.io.Files; import io.netty.buffer.ByteBuf; @@ -40,8 +45,6 @@ import io.netty.channel.ChannelOutboundHandlerAdapter; import io.netty.channel.ChannelPromise; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import org.apache.spark.network.TestUtils; import org.apache.spark.network.TransportContext; @@ -56,7 +59,8 @@ import org.apache.spark.network.server.TransportServer; import org.apache.spark.network.server.TransportServerBootstrap; import org.apache.spark.network.util.ByteArrayWritableChannel; -import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; /** @@ -122,39 +126,50 @@ public void testNonMatching() { } @Test - public void testSaslAuthentication() throws Exception { + public void testSaslAuthentication() throws Throwable { testBasicSasl(false); } @Test - public void testSaslEncryption() throws Exception { + public void testSaslEncryption() throws Throwable { testBasicSasl(true); } - private void testBasicSasl(boolean encrypt) throws Exception { + private static void testBasicSasl(boolean encrypt) throws Throwable { RpcHandler rpcHandler = mock(RpcHandler.class); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) { - byte[] message = (byte[]) invocation.getArguments()[1]; - RpcResponseCallback cb = (RpcResponseCallback) invocation.getArguments()[2]; - assertEquals("Ping", new String(message, StandardCharsets.UTF_8)); - cb.onSuccess("Pong".getBytes(StandardCharsets.UTF_8)); - return null; - } - }) + doAnswer(invocation -> { + ByteBuffer message = (ByteBuffer) invocation.getArguments()[1]; + RpcResponseCallback cb = (RpcResponseCallback) invocation.getArguments()[2]; + assertEquals("Ping", JavaUtils.bytesToString(message)); + cb.onSuccess(JavaUtils.stringToBytes("Pong")); + return null; + }) .when(rpcHandler) - .receive(any(TransportClient.class), any(byte[].class), any(RpcResponseCallback.class)); + .receive(any(TransportClient.class), any(ByteBuffer.class), any(RpcResponseCallback.class)); SaslTestCtx ctx = new SaslTestCtx(rpcHandler, encrypt, false); try { - byte[] response = ctx.client.sendRpcSync("Ping".getBytes(StandardCharsets.UTF_8), - TimeUnit.SECONDS.toMillis(10)); - assertEquals("Pong", new String(response, StandardCharsets.UTF_8)); + ByteBuffer response = ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), + TimeUnit.SECONDS.toMillis(10)); + assertEquals("Pong", JavaUtils.bytesToString(response)); } finally { ctx.close(); // There should be 2 terminated events; one for the client, one for the server. - verify(rpcHandler, times(2)).connectionTerminated(any(TransportClient.class)); + Throwable error = null; + long deadline = System.nanoTime() + TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS); + while (deadline > System.nanoTime()) { + try { + verify(rpcHandler, times(2)).channelInactive(any(TransportClient.class)); + error = null; + break; + } catch (Throwable t) { + error = t; + TimeUnit.MILLISECONDS.sleep(10); + } + } + if (error != null) { + throw error; + } } } @@ -207,7 +222,7 @@ public void testEncryptedMessage() throws Exception { public void testEncryptedMessageChunking() throws Exception { File file = File.createTempFile("sasltest", ".txt"); try { - TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + TransportConf conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); byte[] data = new byte[8 * 1024]; new Random().nextBytes(data); @@ -235,21 +250,17 @@ public void testEncryptedMessageChunking() throws Exception { @Test public void testFileRegionEncryption() throws Exception { - final String blockSizeConf = "spark.network.sasl.maxEncryptedBlockSize"; - System.setProperty(blockSizeConf, "1k"); + Map testConf = ImmutableMap.of( + "spark.network.sasl.maxEncryptedBlockSize", "1k"); - final AtomicReference response = new AtomicReference<>(); - final File file = File.createTempFile("sasltest", ".txt"); + AtomicReference response = new AtomicReference<>(); + File file = File.createTempFile("sasltest", ".txt"); SaslTestCtx ctx = null; try { - final TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + TransportConf conf = new TransportConf("shuffle", new MapConfigProvider(testConf)); StreamManager sm = mock(StreamManager.class); - when(sm.getChunk(anyLong(), anyInt())).thenAnswer(new Answer() { - @Override - public ManagedBuffer answer(InvocationOnMock invocation) { - return new FileSegmentManagedBuffer(conf, file, 0, file.length()); - } - }); + when(sm.getChunk(anyLong(), anyInt())).thenAnswer(invocation -> + new FileSegmentManagedBuffer(conf, file, 0, file.length())); RpcHandler rpcHandler = mock(RpcHandler.class); when(rpcHandler.getStreamManager()).thenReturn(sm); @@ -258,27 +269,20 @@ public ManagedBuffer answer(InvocationOnMock invocation) { new Random().nextBytes(data); Files.write(data, file); - ctx = new SaslTestCtx(rpcHandler, true, false); + ctx = new SaslTestCtx(rpcHandler, true, false, testConf); - final Object lock = new Object(); + CountDownLatch lock = new CountDownLatch(1); ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) { - response.set((ManagedBuffer) invocation.getArguments()[1]); - response.get().retain(); - synchronized (lock) { - lock.notifyAll(); - } - return null; - } - }).when(callback).onSuccess(anyInt(), any(ManagedBuffer.class)); - - synchronized (lock) { - ctx.client.fetchChunk(0, 0, callback); - lock.wait(10 * 1000); - } + doAnswer(invocation -> { + response.set((ManagedBuffer) invocation.getArguments()[1]); + response.get().retain(); + lock.countDown(); + return null; + }).when(callback).onSuccess(anyInt(), any(ManagedBuffer.class)); + + ctx.client.fetchChunk(0, 0, callback); + lock.await(10, TimeUnit.SECONDS); verify(callback, times(1)).onSuccess(anyInt(), any(ManagedBuffer.class)); verify(callback, never()).onFailure(anyInt(), any(Throwable.class)); @@ -293,18 +297,15 @@ public Void answer(InvocationOnMock invocation) { if (response.get() != null) { response.get().release(); } - System.clearProperty(blockSizeConf); } } @Test public void testServerAlwaysEncrypt() throws Exception { - final String alwaysEncryptConfName = "spark.network.sasl.serverAlwaysEncrypt"; - System.setProperty(alwaysEncryptConfName, "true"); - SaslTestCtx ctx = null; try { - ctx = new SaslTestCtx(mock(RpcHandler.class), false, false); + ctx = new SaslTestCtx(mock(RpcHandler.class), false, false, + ImmutableMap.of("spark.network.sasl.serverAlwaysEncrypt", "true")); fail("Should have failed to connect without encryption."); } catch (Exception e) { assertTrue(e.getCause() instanceof SaslException); @@ -312,7 +313,6 @@ public void testServerAlwaysEncrypt() throws Exception { if (ctx != null) { ctx.close(); } - System.clearProperty(alwaysEncryptConfName); } } @@ -324,8 +324,8 @@ public void testDataEncryptionIsActuallyEnabled() throws Exception { SaslTestCtx ctx = null; try { ctx = new SaslTestCtx(mock(RpcHandler.class), true, true); - ctx.client.sendRpcSync("Ping".getBytes(StandardCharsets.UTF_8), - TimeUnit.SECONDS.toMillis(10)); + ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), + TimeUnit.SECONDS.toMillis(10)); fail("Should have failed to send RPC to server."); } catch (Exception e) { assertFalse(e.getCause() instanceof TimeoutException); @@ -346,13 +346,21 @@ public void testRpcHandlerDelegate() throws Exception { saslHandler.getStreamManager(); verify(handler).getStreamManager(); - saslHandler.connectionTerminated(null); - verify(handler).connectionTerminated(any(TransportClient.class)); + saslHandler.channelInactive(null); + verify(handler).channelInactive(any(TransportClient.class)); saslHandler.exceptionCaught(null, null); verify(handler).exceptionCaught(any(Throwable.class), any(TransportClient.class)); } + @Test + public void testDelegates() throws Exception { + Method[] rpcHandlerMethods = RpcHandler.class.getDeclaredMethods(); + for (Method m : rpcHandlerMethods) { + SaslRpcHandler.class.getDeclaredMethod(m.getName(), m.getParameterTypes()); + } + } + private static class SaslTestCtx { final TransportClient client; @@ -368,7 +376,21 @@ private static class SaslTestCtx { boolean disableClientEncryption) throws Exception { - TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + this(rpcHandler, encrypt, disableClientEncryption, Collections.emptyMap()); + } + + SaslTestCtx( + RpcHandler rpcHandler, + boolean encrypt, + boolean disableClientEncryption, + Map extraConf) + throws Exception { + + Map testConf = ImmutableMap.builder() + .putAll(extraConf) + .put("spark.authenticate.enableSaslEncryption", String.valueOf(encrypt)) + .build(); + TransportConf conf = new TransportConf("shuffle", new MapConfigProvider(testConf)); SecretKeyHolder keyHolder = mock(SecretKeyHolder.class); when(keyHolder.getSaslUser(anyString())).thenReturn("user"); @@ -376,13 +398,14 @@ private static class SaslTestCtx { TransportContext ctx = new TransportContext(conf, rpcHandler); - this.checker = new EncryptionCheckerBootstrap(); + this.checker = new EncryptionCheckerBootstrap(SaslEncryption.ENCRYPTION_HANDLER_NAME); + this.server = ctx.createServer(Arrays.asList(new SaslServerBootstrap(conf, keyHolder), checker)); try { - List clientBootstraps = Lists.newArrayList(); - clientBootstraps.add(new SaslClientBootstrap(conf, "user", keyHolder, encrypt)); + List clientBootstraps = new ArrayList<>(); + clientBootstraps.add(new SaslClientBootstrap(conf, "user", keyHolder)); if (disableClientEncryption) { clientBootstraps.add(new EncryptionDisablerBootstrap()); } @@ -416,22 +439,22 @@ private static class EncryptionCheckerBootstrap extends ChannelOutboundHandlerAd implements TransportServerBootstrap { boolean foundEncryptionHandler; + String encryptHandlerName; + + EncryptionCheckerBootstrap(String encryptHandlerName) { + this.encryptHandlerName = encryptHandlerName; + } @Override public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { if (!foundEncryptionHandler) { foundEncryptionHandler = - ctx.channel().pipeline().get(SaslEncryption.ENCRYPTION_HANDLER_NAME) != null; + ctx.channel().pipeline().get(encryptHandlerName) != null; } ctx.write(msg, promise); } - @Override - public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { - super.handlerRemoved(ctx); - } - @Override public RpcHandler doBootstrap(Channel channel, RpcHandler rpcHandler) { channel.pipeline().addFirst("encryptionChecker", this); diff --git a/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java new file mode 100644 index 000000000000..c647525d8f1b --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import java.util.ArrayList; +import java.util.List; + +import io.netty.channel.Channel; +import org.junit.Test; +import org.mockito.Mockito; + +import org.apache.spark.network.TestManagedBuffer; +import org.apache.spark.network.buffer.ManagedBuffer; + +public class OneForOneStreamManagerSuite { + + @Test + public void managedBuffersAreFeedWhenConnectionIsClosed() throws Exception { + OneForOneStreamManager manager = new OneForOneStreamManager(); + List buffers = new ArrayList<>(); + TestManagedBuffer buffer1 = Mockito.spy(new TestManagedBuffer(10)); + TestManagedBuffer buffer2 = Mockito.spy(new TestManagedBuffer(20)); + buffers.add(buffer1); + buffers.add(buffer2); + long streamId = manager.registerStream("appId", buffers.iterator()); + + Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS); + manager.registerChannel(dummyChannel, streamId); + + manager.connectionTerminated(dummyChannel); + + Mockito.verify(buffer1, Mockito.times(1)).release(); + Mockito.verify(buffer2, Mockito.times(1)).release(); + } +} diff --git a/common/network-common/src/test/java/org/apache/spark/network/util/CryptoUtilsSuite.java b/common/network-common/src/test/java/org/apache/spark/network/util/CryptoUtilsSuite.java new file mode 100644 index 000000000000..2b45d1e39713 --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/util/CryptoUtilsSuite.java @@ -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.network.util; + +import java.util.Map; +import java.util.Properties; + +import com.google.common.collect.ImmutableMap; +import org.junit.Test; +import static org.junit.Assert.*; + +public class CryptoUtilsSuite { + + @Test + public void testConfConversion() { + String prefix = "my.prefix.commons.config."; + + String confKey1 = prefix + "a.b.c"; + String confVal1 = "val1"; + String cryptoKey1 = CryptoUtils.COMMONS_CRYPTO_CONFIG_PREFIX + "a.b.c"; + + String confKey2 = prefix.substring(0, prefix.length() - 1) + "A.b.c"; + String confVal2 = "val2"; + String cryptoKey2 = CryptoUtils.COMMONS_CRYPTO_CONFIG_PREFIX + "A.b.c"; + + Map conf = ImmutableMap.of( + confKey1, confVal1, + confKey2, confVal2); + + Properties cryptoConf = CryptoUtils.toCryptoConf(prefix, conf.entrySet()); + + assertEquals(confVal1, cryptoConf.getProperty(cryptoKey1)); + assertFalse(cryptoConf.containsKey(cryptoKey2)); + } + +} diff --git a/common/network-common/src/test/java/org/apache/spark/network/util/NettyMemoryMetricsSuite.java b/common/network-common/src/test/java/org/apache/spark/network/util/NettyMemoryMetricsSuite.java new file mode 100644 index 000000000000..400b385c9703 --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/util/NettyMemoryMetricsSuite.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Metric; +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.MetricSet; +import org.apache.spark.network.TestUtils; +import org.apache.spark.network.client.TransportClient; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.server.NoOpRpcHandler; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportServer; + +public class NettyMemoryMetricsSuite { + + private TransportConf conf; + private TransportContext context; + private TransportServer server; + private TransportClientFactory clientFactory; + + private void setUp(boolean enableVerboseMetrics) { + HashMap configMap = new HashMap<>(); + configMap.put("spark.shuffle.io.enableVerboseMetrics", String.valueOf(enableVerboseMetrics)); + conf = new TransportConf("shuffle", new MapConfigProvider(configMap)); + RpcHandler rpcHandler = new NoOpRpcHandler(); + context = new TransportContext(conf, rpcHandler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + } + + @After + public void tearDown() { + if (clientFactory != null) { + JavaUtils.closeQuietly(clientFactory); + clientFactory = null; + } + + if (server != null) { + JavaUtils.closeQuietly(server); + server = null; + } + } + + @Test + @SuppressWarnings("unchecked") + public void testGeneralNettyMemoryMetrics() throws IOException, InterruptedException { + setUp(false); + + MetricSet serverMetrics = server.getAllMetrics(); + Assert.assertNotNull(serverMetrics); + Assert.assertNotNull(serverMetrics.getMetrics()); + Assert.assertNotEquals(serverMetrics.getMetrics().size(), 0); + + Map serverMetricMap = serverMetrics.getMetrics(); + serverMetricMap.forEach((name, metric) -> + Assert.assertTrue(name.startsWith("shuffle-server")) + ); + + MetricSet clientMetrics = clientFactory.getAllMetrics(); + Assert.assertNotNull(clientMetrics); + Assert.assertNotNull(clientMetrics.getMetrics()); + Assert.assertNotEquals(clientMetrics.getMetrics().size(), 0); + + Map clientMetricMap = clientMetrics.getMetrics(); + clientMetricMap.forEach((name, metrics) -> + Assert.assertTrue(name.startsWith("shuffle-client")) + ); + + // Make sure general metrics existed. + String heapMemoryMetric = "usedHeapMemory"; + String directMemoryMetric = "usedDirectMemory"; + Assert.assertNotNull(serverMetricMap.get( + MetricRegistry.name("shuffle-server", heapMemoryMetric))); + Assert.assertNotNull(serverMetricMap.get( + MetricRegistry.name("shuffle-server", directMemoryMetric))); + + Assert.assertNotNull(clientMetricMap.get( + MetricRegistry.name("shuffle-client", heapMemoryMetric))); + Assert.assertNotNull(clientMetricMap.get( + MetricRegistry.name("shuffle-client", directMemoryMetric))); + + TransportClient client = null; + try { + client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + Assert.assertTrue(client.isActive()); + + Assert.assertTrue(((Gauge)serverMetricMap.get( + MetricRegistry.name("shuffle-server", heapMemoryMetric))).getValue() >= 0L); + Assert.assertTrue(((Gauge)serverMetricMap.get( + MetricRegistry.name("shuffle-server", directMemoryMetric))).getValue() >= 0L); + + Assert.assertTrue(((Gauge)clientMetricMap.get( + MetricRegistry.name("shuffle-client", heapMemoryMetric))).getValue() >= 0L); + Assert.assertTrue(((Gauge)clientMetricMap.get( + MetricRegistry.name("shuffle-client", directMemoryMetric))).getValue() >= 0L); + + } finally { + if (client != null) { + client.close(); + } + } + } + + @Test + @SuppressWarnings("unchecked") + public void testAdditionalMetrics() throws IOException, InterruptedException { + setUp(true); + + // Make sure additional metrics are added. + Map serverMetricMap = server.getAllMetrics().getMetrics(); + serverMetricMap.forEach((name, metric) -> { + Assert.assertTrue(name.startsWith("shuffle-server")); + String metricName = name.substring(name.lastIndexOf(".") + 1); + Assert.assertTrue(metricName.equals("usedDirectMemory") + || metricName.equals("usedHeapMemory") + || NettyMemoryMetrics.VERBOSE_METRICS.contains(metricName)); + }); + + Map clientMetricMap = clientFactory.getAllMetrics().getMetrics(); + clientMetricMap.forEach((name, metric) -> { + Assert.assertTrue(name.startsWith("shuffle-client")); + String metricName = name.substring(name.lastIndexOf(".") + 1); + Assert.assertTrue(metricName.equals("usedDirectMemory") + || metricName.equals("usedHeapMemory") + || NettyMemoryMetrics.VERBOSE_METRICS.contains(metricName)); + }); + + TransportClient client = null; + try { + client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + Assert.assertTrue(client.isActive()); + + String activeBytesMetric = "numActiveBytes"; + Assert.assertTrue(((Gauge) serverMetricMap.get(MetricRegistry.name("shuffle-server", + "directArena0", activeBytesMetric))).getValue() >= 0L); + + Assert.assertTrue(((Gauge) clientMetricMap.get(MetricRegistry.name("shuffle-client", + "directArena0", activeBytesMetric))).getValue() >= 0L); + } finally { + if (client != null) { + client.close(); + } + } + } +} diff --git a/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java b/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java new file mode 100644 index 000000000000..b53e41303751 --- /dev/null +++ b/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.atomic.AtomicInteger; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import org.junit.AfterClass; +import org.junit.Test; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +public class TransportFrameDecoderSuite { + + private static Random RND = new Random(); + + @AfterClass + public static void cleanup() { + RND = null; + } + + @Test + public void testFrameDecoding() throws Exception { + TransportFrameDecoder decoder = new TransportFrameDecoder(); + ChannelHandlerContext ctx = mockChannelHandlerContext(); + ByteBuf data = createAndFeedFrames(100, decoder, ctx); + verifyAndCloseDecoder(decoder, ctx, data); + } + + @Test + public void testInterception() throws Exception { + int interceptedReads = 3; + TransportFrameDecoder decoder = new TransportFrameDecoder(); + TransportFrameDecoder.Interceptor interceptor = spy(new MockInterceptor(interceptedReads)); + ChannelHandlerContext ctx = mockChannelHandlerContext(); + + byte[] data = new byte[8]; + ByteBuf len = Unpooled.copyLong(8 + data.length); + ByteBuf dataBuf = Unpooled.wrappedBuffer(data); + + try { + decoder.setInterceptor(interceptor); + for (int i = 0; i < interceptedReads; i++) { + decoder.channelRead(ctx, dataBuf); + assertEquals(0, dataBuf.refCnt()); + dataBuf = Unpooled.wrappedBuffer(data); + } + decoder.channelRead(ctx, len); + decoder.channelRead(ctx, dataBuf); + verify(interceptor, times(interceptedReads)).handle(any(ByteBuf.class)); + verify(ctx).fireChannelRead(any(ByteBuffer.class)); + assertEquals(0, len.refCnt()); + assertEquals(0, dataBuf.refCnt()); + } finally { + release(len); + release(dataBuf); + } + } + + @Test + public void testRetainedFrames() throws Exception { + TransportFrameDecoder decoder = new TransportFrameDecoder(); + + AtomicInteger count = new AtomicInteger(); + List retained = new ArrayList<>(); + + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.fireChannelRead(any())).thenAnswer(in -> { + // Retain a few frames but not others. + ByteBuf buf = (ByteBuf) in.getArguments()[0]; + if (count.incrementAndGet() % 2 == 0) { + retained.add(buf); + } else { + buf.release(); + } + return null; + }); + + ByteBuf data = createAndFeedFrames(100, decoder, ctx); + try { + // Verify all retained buffers are readable. + for (ByteBuf b : retained) { + byte[] tmp = new byte[b.readableBytes()]; + b.readBytes(tmp); + b.release(); + } + verifyAndCloseDecoder(decoder, ctx, data); + } finally { + for (ByteBuf b : retained) { + release(b); + } + } + } + + @Test + public void testSplitLengthField() throws Exception { + byte[] frame = new byte[1024 * (RND.nextInt(31) + 1)]; + ByteBuf buf = Unpooled.buffer(frame.length + 8); + buf.writeLong(frame.length + 8); + buf.writeBytes(frame); + + TransportFrameDecoder decoder = new TransportFrameDecoder(); + ChannelHandlerContext ctx = mockChannelHandlerContext(); + try { + decoder.channelRead(ctx, buf.readSlice(RND.nextInt(7)).retain()); + verify(ctx, never()).fireChannelRead(any(ByteBuf.class)); + decoder.channelRead(ctx, buf); + verify(ctx).fireChannelRead(any(ByteBuf.class)); + assertEquals(0, buf.refCnt()); + } finally { + decoder.channelInactive(ctx); + release(buf); + } + } + + @Test(expected = IllegalArgumentException.class) + public void testNegativeFrameSize() throws Exception { + testInvalidFrame(-1); + } + + @Test(expected = IllegalArgumentException.class) + public void testEmptyFrame() throws Exception { + // 8 because frame size includes the frame length. + testInvalidFrame(8); + } + + /** + * Creates a number of randomly sized frames and feed them to the given decoder, verifying + * that the frames were read. + */ + private ByteBuf createAndFeedFrames( + int frameCount, + TransportFrameDecoder decoder, + ChannelHandlerContext ctx) throws Exception { + ByteBuf data = Unpooled.buffer(); + for (int i = 0; i < frameCount; i++) { + byte[] frame = new byte[1024 * (RND.nextInt(31) + 1)]; + data.writeLong(frame.length + 8); + data.writeBytes(frame); + } + + try { + while (data.isReadable()) { + int size = RND.nextInt(4 * 1024) + 256; + decoder.channelRead(ctx, data.readSlice(Math.min(data.readableBytes(), size)).retain()); + } + + verify(ctx, times(frameCount)).fireChannelRead(any(ByteBuf.class)); + } catch (Exception e) { + release(data); + throw e; + } + return data; + } + + private void verifyAndCloseDecoder( + TransportFrameDecoder decoder, + ChannelHandlerContext ctx, + ByteBuf data) throws Exception { + try { + decoder.channelInactive(ctx); + assertTrue("There shouldn't be dangling references to the data.", data.release()); + } finally { + release(data); + } + } + + private void testInvalidFrame(long size) throws Exception { + TransportFrameDecoder decoder = new TransportFrameDecoder(); + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + ByteBuf frame = Unpooled.copyLong(size); + try { + decoder.channelRead(ctx, frame); + } finally { + release(frame); + } + } + + private ChannelHandlerContext mockChannelHandlerContext() { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.fireChannelRead(any())).thenAnswer(in -> { + ByteBuf buf = (ByteBuf) in.getArguments()[0]; + buf.release(); + return null; + }); + return ctx; + } + + private void release(ByteBuf buf) { + if (buf.refCnt() > 0) { + buf.release(buf.refCnt()); + } + } + + private static class MockInterceptor implements TransportFrameDecoder.Interceptor { + + private int remainingReads; + + MockInterceptor(int readCount) { + this.remainingReads = readCount; + } + + @Override + public boolean handle(ByteBuf data) throws Exception { + data.readerIndex(data.readerIndex() + data.readableBytes()); + assertFalse(data.isReadable()); + remainingReads -= 1; + return remainingReads != 0; + } + + @Override + public void exceptionCaught(Throwable cause) throws Exception { + + } + + @Override + public void channelInactive() throws Exception { + + } + + } + +} diff --git a/common/network-common/src/test/resources/log4j.properties b/common/network-common/src/test/resources/log4j.properties new file mode 100644 index 000000000000..e8da774f7ca9 --- /dev/null +++ b/common/network-common/src/test/resources/log4j.properties @@ -0,0 +1,27 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=DEBUG, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Silence verbose logs from 3rd-party libraries. +log4j.logger.io.netty=INFO diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml new file mode 100644 index 000000000000..9968480ab765 --- /dev/null +++ b/common/network-shuffle/pom.xml @@ -0,0 +1,102 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-network-shuffle_2.11 + jar + Spark Project Shuffle Streaming Service + http://spark.apache.org/ + + network-shuffle + + + + + + org.apache.spark + spark-network-common_${scala.binary.version} + ${project.version} + + + + io.dropwizard.metrics + metrics-core + + + + + org.slf4j + slf4j-api + provided + + + com.google.guava + guava + + + + + org.apache.spark + spark-network-common_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-tags_${scala.binary.version} + test + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + log4j + log4j + test + + + org.mockito + mockito-core + test + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java b/common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java similarity index 90% rename from network/shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java rename to common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java index 351c7930a900..d2d008f8a3d3 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java @@ -17,21 +17,20 @@ package org.apache.spark.network.sasl; -import java.lang.Override; import java.nio.ByteBuffer; import java.util.concurrent.ConcurrentHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.spark.network.sasl.SecretKeyHolder; import org.apache.spark.network.util.JavaUtils; /** * A class that manages shuffle secret used by the external shuffle service. */ public class ShuffleSecretManager implements SecretKeyHolder { - private final Logger logger = LoggerFactory.getLogger(ShuffleSecretManager.class); + private static final Logger logger = LoggerFactory.getLogger(ShuffleSecretManager.class); + private final ConcurrentHashMap shuffleSecretMap; // Spark user used for authenticating SASL connections @@ -39,7 +38,7 @@ public class ShuffleSecretManager implements SecretKeyHolder { private static final String SPARK_SASL_USER = "sparkSaslUser"; public ShuffleSecretManager() { - shuffleSecretMap = new ConcurrentHashMap(); + shuffleSecretMap = new ConcurrentHashMap<>(); } /** @@ -48,7 +47,7 @@ public ShuffleSecretManager() { * fetching shuffle files written by other executors in this application. */ public void registerApp(String appId, String shuffleSecret) { - if (!shuffleSecretMap.contains(appId)) { + if (!shuffleSecretMap.containsKey(appId)) { shuffleSecretMap.put(appId, shuffleSecret); logger.info("Registered shuffle secret for application {}", appId); } else { @@ -68,7 +67,7 @@ public void registerApp(String appId, ByteBuffer shuffleSecret) { * This is called when the application terminates. */ public void unregisterApp(String appId) { - if (shuffleSecretMap.contains(appId)) { + if (shuffleSecretMap.containsKey(appId)) { shuffleSecretMap.remove(appId); logger.info("Unregistered shuffle secret for application {}", appId); } else { diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/BlockFetchingListener.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockFetchingListener.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/shuffle/BlockFetchingListener.java rename to common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/BlockFetchingListener.java diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java new file mode 100644 index 000000000000..fc7bba41185f --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -0,0 +1,240 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Meter; +import com.codahale.metrics.Metric; +import com.codahale.metrics.MetricSet; +import com.codahale.metrics.Timer; +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.server.OneForOneStreamManager; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; +import org.apache.spark.network.shuffle.protocol.*; +import static org.apache.spark.network.util.NettyUtils.getRemoteAddress; +import org.apache.spark.network.util.TransportConf; + +/** + * RPC Handler for a server which can serve shuffle blocks from outside of an Executor process. + * + * Handles registering executors and opening shuffle blocks from them. Shuffle blocks are registered + * with the "one-for-one" strategy, meaning each Transport-layer Chunk is equivalent to one Spark- + * level shuffle block. + */ +public class ExternalShuffleBlockHandler extends RpcHandler { + private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockHandler.class); + + @VisibleForTesting + final ExternalShuffleBlockResolver blockManager; + private final OneForOneStreamManager streamManager; + private final ShuffleMetrics metrics; + + public ExternalShuffleBlockHandler(TransportConf conf, File registeredExecutorFile) + throws IOException { + this(new OneForOneStreamManager(), + new ExternalShuffleBlockResolver(conf, registeredExecutorFile)); + } + + /** Enables mocking out the StreamManager and BlockManager. */ + @VisibleForTesting + public ExternalShuffleBlockHandler( + OneForOneStreamManager streamManager, + ExternalShuffleBlockResolver blockManager) { + this.metrics = new ShuffleMetrics(); + this.streamManager = streamManager; + this.blockManager = blockManager; + } + + @Override + public void receive(TransportClient client, ByteBuffer message, RpcResponseCallback callback) { + BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(message); + handleMessage(msgObj, client, callback); + } + + protected void handleMessage( + BlockTransferMessage msgObj, + TransportClient client, + RpcResponseCallback callback) { + if (msgObj instanceof OpenBlocks) { + final Timer.Context responseDelayContext = metrics.openBlockRequestLatencyMillis.time(); + try { + OpenBlocks msg = (OpenBlocks) msgObj; + checkAuth(client, msg.appId); + long streamId = streamManager.registerStream(client.getClientId(), + new ManagedBufferIterator(msg.appId, msg.execId, msg.blockIds)); + if (logger.isTraceEnabled()) { + logger.trace("Registered streamId {} with {} buffers for client {} from host {}", + streamId, + msg.blockIds.length, + client.getClientId(), + getRemoteAddress(client.getChannel())); + } + callback.onSuccess(new StreamHandle(streamId, msg.blockIds.length).toByteBuffer()); + } finally { + responseDelayContext.stop(); + } + + } else if (msgObj instanceof RegisterExecutor) { + final Timer.Context responseDelayContext = + metrics.registerExecutorRequestLatencyMillis.time(); + try { + RegisterExecutor msg = (RegisterExecutor) msgObj; + checkAuth(client, msg.appId); + blockManager.registerExecutor(msg.appId, msg.execId, msg.executorInfo); + callback.onSuccess(ByteBuffer.wrap(new byte[0])); + } finally { + responseDelayContext.stop(); + } + + } else { + throw new UnsupportedOperationException("Unexpected message: " + msgObj); + } + } + + public MetricSet getAllMetrics() { + return metrics; + } + + @Override + public StreamManager getStreamManager() { + return streamManager; + } + + /** + * Removes an application (once it has been terminated), and optionally will clean up any + * local directories associated with the executors of that application in a separate thread. + */ + public void applicationRemoved(String appId, boolean cleanupLocalDirs) { + blockManager.applicationRemoved(appId, cleanupLocalDirs); + } + + /** + * Register an (application, executor) with the given shuffle info. + * + * The "re-" is meant to highlight the intended use of this method -- when this service is + * restarted, this is used to restore the state of executors from before the restart. Normal + * registration will happen via a message handled in receive() + * + * @param appExecId + * @param executorInfo + */ + public void reregisterExecutor(AppExecId appExecId, ExecutorShuffleInfo executorInfo) { + blockManager.registerExecutor(appExecId.appId, appExecId.execId, executorInfo); + } + + public void close() { + blockManager.close(); + } + + private void checkAuth(TransportClient client, String appId) { + if (client.getClientId() != null && !client.getClientId().equals(appId)) { + throw new SecurityException(String.format( + "Client for %s not authorized for application %s.", client.getClientId(), appId)); + } + } + + /** + * A simple class to wrap all shuffle service wrapper metrics + */ + private class ShuffleMetrics implements MetricSet { + private final Map allMetrics; + // Time latency for open block request in ms + private final Timer openBlockRequestLatencyMillis = new Timer(); + // Time latency for executor registration latency in ms + private final Timer registerExecutorRequestLatencyMillis = new Timer(); + // Block transfer rate in byte per second + private final Meter blockTransferRateBytes = new Meter(); + + private ShuffleMetrics() { + allMetrics = new HashMap<>(); + allMetrics.put("openBlockRequestLatencyMillis", openBlockRequestLatencyMillis); + allMetrics.put("registerExecutorRequestLatencyMillis", registerExecutorRequestLatencyMillis); + allMetrics.put("blockTransferRateBytes", blockTransferRateBytes); + allMetrics.put("registeredExecutorsSize", + (Gauge) () -> blockManager.getRegisteredExecutorsSize()); + } + + @Override + public Map getMetrics() { + return allMetrics; + } + } + + private class ManagedBufferIterator implements Iterator { + + private int index = 0; + private final String appId; + private final String execId; + private final int shuffleId; + // An array containing mapId and reduceId pairs. + private final int[] mapIdAndReduceIds; + + ManagedBufferIterator(String appId, String execId, String[] blockIds) { + this.appId = appId; + this.execId = execId; + String[] blockId0Parts = blockIds[0].split("_"); + if (blockId0Parts.length != 4 || !blockId0Parts[0].equals("shuffle")) { + throw new IllegalArgumentException("Unexpected shuffle block id format: " + blockIds[0]); + } + this.shuffleId = Integer.parseInt(blockId0Parts[1]); + mapIdAndReduceIds = new int[2 * blockIds.length]; + for (int i = 0; i < blockIds.length; i++) { + String[] blockIdParts = blockIds[i].split("_"); + if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) { + throw new IllegalArgumentException("Unexpected shuffle block id format: " + blockIds[i]); + } + if (Integer.parseInt(blockIdParts[1]) != shuffleId) { + throw new IllegalArgumentException("Expected shuffleId=" + shuffleId + + ", got:" + blockIds[i]); + } + mapIdAndReduceIds[2 * i] = Integer.parseInt(blockIdParts[2]); + mapIdAndReduceIds[2 * i + 1] = Integer.parseInt(blockIdParts[3]); + } + } + + @Override + public boolean hasNext() { + return index < mapIdAndReduceIds.length; + } + + @Override + public ManagedBuffer next() { + final ManagedBuffer block = blockManager.getBlockData(appId, execId, shuffleId, + mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1]); + index += 2; + metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); + return block; + } + } + +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java new file mode 100644 index 000000000000..e6399897be9c --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.*; +import java.nio.charset.StandardCharsets; +import java.util.*; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Objects; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import com.google.common.cache.Weigher; +import com.google.common.collect.Maps; +import org.iq80.leveldb.DB; +import org.iq80.leveldb.DBIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.FileSegmentManagedBuffer; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.apache.spark.network.util.LevelDBProvider; +import org.apache.spark.network.util.LevelDBProvider.StoreVersion; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.NettyUtils; +import org.apache.spark.network.util.TransportConf; + +/** + * Manages converting shuffle BlockIds into physical segments of local files, from a process outside + * of Executors. Each Executor must register its own configuration about where it stores its files + * (local dirs) and how (shuffle manager). The logic for retrieval of individual files is replicated + * from Spark's IndexShuffleBlockResolver. + */ +public class ExternalShuffleBlockResolver { + private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockResolver.class); + + private static final ObjectMapper mapper = new ObjectMapper(); + /** + * This a common prefix to the key for each app registration we stick in leveldb, so they + * are easy to find, since leveldb lets you search based on prefix. + */ + private static final String APP_KEY_PREFIX = "AppExecShuffleInfo"; + private static final StoreVersion CURRENT_VERSION = new StoreVersion(1, 0); + + // Map containing all registered executors' metadata. + @VisibleForTesting + final ConcurrentMap executors; + + /** + * Caches index file information so that we can avoid open/close the index files + * for each block fetch. + */ + private final LoadingCache shuffleIndexCache; + + // Single-threaded Java executor used to perform expensive recursive directory deletion. + private final Executor directoryCleaner; + + private final TransportConf conf; + + @VisibleForTesting + final File registeredExecutorFile; + @VisibleForTesting + final DB db; + + private final List knownManagers = Arrays.asList( + "org.apache.spark.shuffle.sort.SortShuffleManager", + "org.apache.spark.shuffle.unsafe.UnsafeShuffleManager"); + + public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) + throws IOException { + this(conf, registeredExecutorFile, Executors.newSingleThreadExecutor( + // Add `spark` prefix because it will run in NM in Yarn mode. + NettyUtils.createThreadFactory("spark-shuffle-directory-cleaner"))); + } + + // Allows tests to have more control over when directories are cleaned up. + @VisibleForTesting + ExternalShuffleBlockResolver( + TransportConf conf, + File registeredExecutorFile, + Executor directoryCleaner) throws IOException { + this.conf = conf; + this.registeredExecutorFile = registeredExecutorFile; + String indexCacheSize = conf.get("spark.shuffle.service.index.cache.size", "100m"); + CacheLoader indexCacheLoader = + new CacheLoader() { + public ShuffleIndexInformation load(File file) throws IOException { + return new ShuffleIndexInformation(file); + } + }; + shuffleIndexCache = CacheBuilder.newBuilder() + .maximumWeight(JavaUtils.byteStringAsBytes(indexCacheSize)) + .weigher(new Weigher() { + public int weigh(File file, ShuffleIndexInformation indexInfo) { + return indexInfo.getSize(); + } + }) + .build(indexCacheLoader); + db = LevelDBProvider.initLevelDB(this.registeredExecutorFile, CURRENT_VERSION, mapper); + if (db != null) { + executors = reloadRegisteredExecutors(db); + } else { + executors = Maps.newConcurrentMap(); + } + this.directoryCleaner = directoryCleaner; + } + + public int getRegisteredExecutorsSize() { + return executors.size(); + } + + /** Registers a new Executor with all the configuration we need to find its shuffle files. */ + public void registerExecutor( + String appId, + String execId, + ExecutorShuffleInfo executorInfo) { + AppExecId fullId = new AppExecId(appId, execId); + logger.info("Registered executor {} with {}", fullId, executorInfo); + if (!knownManagers.contains(executorInfo.shuffleManager)) { + throw new UnsupportedOperationException( + "Unsupported shuffle manager of executor: " + executorInfo); + } + try { + if (db != null) { + byte[] key = dbAppExecKey(fullId); + byte[] value = mapper.writeValueAsString(executorInfo).getBytes(StandardCharsets.UTF_8); + db.put(key, value); + } + } catch (Exception e) { + logger.error("Error saving registered executors", e); + } + executors.put(fullId, executorInfo); + } + + /** + * Obtains a FileSegmentManagedBuffer from (shuffleId, mapId, reduceId). We make assumptions + * about how the hash and sort based shuffles store their data. + */ + public ManagedBuffer getBlockData( + String appId, + String execId, + int shuffleId, + int mapId, + int reduceId) { + ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId)); + if (executor == null) { + throw new RuntimeException( + String.format("Executor is not registered (appId=%s, execId=%s)", appId, execId)); + } + return getSortBasedShuffleBlockData(executor, shuffleId, mapId, reduceId); + } + + /** + * Removes our metadata of all executors registered for the given application, and optionally + * also deletes the local directories associated with the executors of that application in a + * separate thread. + * + * It is not valid to call registerExecutor() for an executor with this appId after invoking + * this method. + */ + public void applicationRemoved(String appId, boolean cleanupLocalDirs) { + logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); + Iterator> it = executors.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry entry = it.next(); + AppExecId fullId = entry.getKey(); + final ExecutorShuffleInfo executor = entry.getValue(); + + // Only touch executors associated with the appId that was removed. + if (appId.equals(fullId.appId)) { + it.remove(); + if (db != null) { + try { + db.delete(dbAppExecKey(fullId)); + } catch (IOException e) { + logger.error("Error deleting {} from executor state db", appId, e); + } + } + + if (cleanupLocalDirs) { + logger.info("Cleaning up executor {}'s {} local dirs", fullId, executor.localDirs.length); + + // Execute the actual deletion in a different thread, as it may take some time. + directoryCleaner.execute(() -> deleteExecutorDirs(executor.localDirs)); + } + } + } + } + + /** + * Synchronously deletes each directory one at a time. + * Should be executed in its own thread, as this may take a long time. + */ + private void deleteExecutorDirs(String[] dirs) { + for (String localDir : dirs) { + try { + JavaUtils.deleteRecursively(new File(localDir)); + logger.debug("Successfully cleaned up directory: {}", localDir); + } catch (Exception e) { + logger.error("Failed to delete directory: " + localDir, e); + } + } + } + + /** + * Sort-based shuffle data uses an index called "shuffle_ShuffleId_MapId_0.index" into a data file + * called "shuffle_ShuffleId_MapId_0.data". This logic is from IndexShuffleBlockResolver, + * and the block id format is from ShuffleDataBlockId and ShuffleIndexBlockId. + */ + private ManagedBuffer getSortBasedShuffleBlockData( + ExecutorShuffleInfo executor, int shuffleId, int mapId, int reduceId) { + File indexFile = getFile(executor.localDirs, executor.subDirsPerLocalDir, + "shuffle_" + shuffleId + "_" + mapId + "_0.index"); + + try { + ShuffleIndexInformation shuffleIndexInformation = shuffleIndexCache.get(indexFile); + ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(reduceId); + return new FileSegmentManagedBuffer( + conf, + getFile(executor.localDirs, executor.subDirsPerLocalDir, + "shuffle_" + shuffleId + "_" + mapId + "_0.data"), + shuffleIndexRecord.getOffset(), + shuffleIndexRecord.getLength()); + } catch (ExecutionException e) { + throw new RuntimeException("Failed to open file: " + indexFile, e); + } + } + + /** + * Hashes a filename into the corresponding local directory, in a manner consistent with + * Spark's DiskBlockManager.getFile(). + */ + @VisibleForTesting + static File getFile(String[] localDirs, int subDirsPerLocalDir, String filename) { + int hash = JavaUtils.nonNegativeHash(filename); + String localDir = localDirs[hash % localDirs.length]; + int subDirId = (hash / localDirs.length) % subDirsPerLocalDir; + return new File(new File(localDir, String.format("%02x", subDirId)), filename); + } + + void close() { + if (db != null) { + try { + db.close(); + } catch (IOException e) { + logger.error("Exception closing leveldb with registered executors", e); + } + } + } + + /** Simply encodes an executor's full ID, which is appId + execId. */ + public static class AppExecId { + public final String appId; + public final String execId; + + @JsonCreator + public AppExecId(@JsonProperty("appId") String appId, @JsonProperty("execId") String execId) { + this.appId = appId; + this.execId = execId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + AppExecId appExecId = (AppExecId) o; + return Objects.equal(appId, appExecId.appId) && Objects.equal(execId, appExecId.execId); + } + + @Override + public int hashCode() { + return Objects.hashCode(appId, execId); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("execId", execId) + .toString(); + } + } + + private static byte[] dbAppExecKey(AppExecId appExecId) throws IOException { + // we stick a common prefix on all the keys so we can find them in the DB + String appExecJson = mapper.writeValueAsString(appExecId); + String key = (APP_KEY_PREFIX + ";" + appExecJson); + return key.getBytes(StandardCharsets.UTF_8); + } + + private static AppExecId parseDbAppExecKey(String s) throws IOException { + if (!s.startsWith(APP_KEY_PREFIX)) { + throw new IllegalArgumentException("expected a string starting with " + APP_KEY_PREFIX); + } + String json = s.substring(APP_KEY_PREFIX.length() + 1); + AppExecId parsed = mapper.readValue(json, AppExecId.class); + return parsed; + } + + @VisibleForTesting + static ConcurrentMap reloadRegisteredExecutors(DB db) + throws IOException { + ConcurrentMap registeredExecutors = Maps.newConcurrentMap(); + if (db != null) { + DBIterator itr = db.iterator(); + itr.seek(APP_KEY_PREFIX.getBytes(StandardCharsets.UTF_8)); + while (itr.hasNext()) { + Map.Entry e = itr.next(); + String key = new String(e.getKey(), StandardCharsets.UTF_8); + if (!key.startsWith(APP_KEY_PREFIX)) { + break; + } + AppExecId id = parseDbAppExecKey(key); + logger.info("Reloading registered executors: " + id.toString()); + ExecutorShuffleInfo shuffleInfo = mapper.readValue(e.getValue(), ExecutorShuffleInfo.class); + registeredExecutors.put(id, shuffleInfo); + } + } + return registeredExecutors; + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java new file mode 100644 index 000000000000..77702447edb8 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; + +import com.codahale.metrics.MetricSet; +import com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientBootstrap; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.crypto.AuthClientBootstrap; +import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.server.NoOpRpcHandler; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.apache.spark.network.shuffle.protocol.RegisterExecutor; +import org.apache.spark.network.util.TransportConf; + +/** + * Client for reading shuffle blocks which points to an external (outside of executor) server. + * This is instead of reading shuffle blocks directly from other executors (via + * BlockTransferService), which has the downside of losing the shuffle data if we lose the + * executors. + */ +public class ExternalShuffleClient extends ShuffleClient { + private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleClient.class); + + private final TransportConf conf; + private final boolean authEnabled; + private final SecretKeyHolder secretKeyHolder; + private final long registrationTimeoutMs; + + protected TransportClientFactory clientFactory; + protected String appId; + + /** + * Creates an external shuffle client, with SASL optionally enabled. If SASL is not enabled, + * then secretKeyHolder may be null. + */ + public ExternalShuffleClient( + TransportConf conf, + SecretKeyHolder secretKeyHolder, + boolean authEnabled, + long registrationTimeoutMs) { + this.conf = conf; + this.secretKeyHolder = secretKeyHolder; + this.authEnabled = authEnabled; + this.registrationTimeoutMs = registrationTimeoutMs; + } + + protected void checkInit() { + assert appId != null : "Called before init()"; + } + + @Override + public void init(String appId) { + this.appId = appId; + TransportContext context = new TransportContext(conf, new NoOpRpcHandler(), true); + List bootstraps = Lists.newArrayList(); + if (authEnabled) { + bootstraps.add(new AuthClientBootstrap(conf, appId, secretKeyHolder)); + } + clientFactory = context.createClientFactory(bootstraps); + } + + @Override + public void fetchBlocks( + String host, + int port, + String execId, + String[] blockIds, + BlockFetchingListener listener, + TempShuffleFileManager tempShuffleFileManager) { + checkInit(); + logger.debug("External shuffle fetch from {}:{} (executor id {})", host, port, execId); + try { + RetryingBlockFetcher.BlockFetchStarter blockFetchStarter = + (blockIds1, listener1) -> { + TransportClient client = clientFactory.createClient(host, port); + new OneForOneBlockFetcher(client, appId, execId, + blockIds1, listener1, conf, tempShuffleFileManager).start(); + }; + + int maxRetries = conf.maxIORetries(); + if (maxRetries > 0) { + // Note this Fetcher will correctly handle maxRetries == 0; we avoid it just in case there's + // a bug in this code. We should remove the if statement once we're sure of the stability. + new RetryingBlockFetcher(conf, blockFetchStarter, blockIds, listener).start(); + } else { + blockFetchStarter.createAndStart(blockIds, listener); + } + } catch (Exception e) { + logger.error("Exception while beginning fetchBlocks", e); + for (String blockId : blockIds) { + listener.onBlockFetchFailure(blockId, e); + } + } + } + + @Override + public MetricSet shuffleMetrics() { + checkInit(); + return clientFactory.getAllMetrics(); + } + + /** + * Registers this executor with an external shuffle server. This registration is required to + * inform the shuffle server about where and how we store our shuffle files. + * + * @param host Host of shuffle server. + * @param port Port of shuffle server. + * @param execId This Executor's id. + * @param executorInfo Contains all info necessary for the service to find our shuffle files. + */ + public void registerWithShuffleServer( + String host, + int port, + String execId, + ExecutorShuffleInfo executorInfo) throws IOException, InterruptedException { + checkInit(); + try (TransportClient client = clientFactory.createUnmanagedClient(host, port)) { + ByteBuffer registerMessage = new RegisterExecutor(appId, execId, executorInfo).toByteBuffer(); + client.sendRpcSync(registerMessage, registrationTimeoutMs); + } + } + + @Override + public void close() { + checkInit(); + clientFactory.close(); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java new file mode 100644 index 000000000000..66b67e282c80 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.nio.file.Files; +import java.util.Arrays; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.FileSegmentManagedBuffer; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.StreamCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.server.OneForOneStreamManager; +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; +import org.apache.spark.network.shuffle.protocol.OpenBlocks; +import org.apache.spark.network.shuffle.protocol.StreamHandle; +import org.apache.spark.network.util.TransportConf; + +/** + * Simple wrapper on top of a TransportClient which interprets each chunk as a whole block, and + * invokes the BlockFetchingListener appropriately. This class is agnostic to the actual RPC + * handler, as long as there is a single "open blocks" message which returns a ShuffleStreamHandle, + * and Java serialization is used. + * + * Note that this typically corresponds to a + * {@link org.apache.spark.network.server.OneForOneStreamManager} on the server side. + */ +public class OneForOneBlockFetcher { + private static final Logger logger = LoggerFactory.getLogger(OneForOneBlockFetcher.class); + + private final TransportClient client; + private final OpenBlocks openMessage; + private final String[] blockIds; + private final BlockFetchingListener listener; + private final ChunkReceivedCallback chunkCallback; + private final TransportConf transportConf; + private final TempShuffleFileManager tempShuffleFileManager; + + private StreamHandle streamHandle = null; + + public OneForOneBlockFetcher( + TransportClient client, + String appId, + String execId, + String[] blockIds, + BlockFetchingListener listener, + TransportConf transportConf) { + this(client, appId, execId, blockIds, listener, transportConf, null); + } + + public OneForOneBlockFetcher( + TransportClient client, + String appId, + String execId, + String[] blockIds, + BlockFetchingListener listener, + TransportConf transportConf, + TempShuffleFileManager tempShuffleFileManager) { + this.client = client; + this.openMessage = new OpenBlocks(appId, execId, blockIds); + this.blockIds = blockIds; + this.listener = listener; + this.chunkCallback = new ChunkCallback(); + this.transportConf = transportConf; + this.tempShuffleFileManager = tempShuffleFileManager; + } + + /** Callback invoked on receipt of each chunk. We equate a single chunk to a single block. */ + private class ChunkCallback implements ChunkReceivedCallback { + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + // On receipt of a chunk, pass it upwards as a block. + listener.onBlockFetchSuccess(blockIds[chunkIndex], buffer); + } + + @Override + public void onFailure(int chunkIndex, Throwable e) { + // On receipt of a failure, fail every block from chunkIndex onwards. + String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); + failRemainingBlocks(remainingBlockIds, e); + } + } + + /** + * Begins the fetching process, calling the listener with every block fetched. + * The given message will be serialized with the Java serializer, and the RPC must return a + * {@link StreamHandle}. We will send all fetch requests immediately, without throttling. + */ + public void start() { + if (blockIds.length == 0) { + throw new IllegalArgumentException("Zero-sized blockIds array"); + } + + client.sendRpc(openMessage.toByteBuffer(), new RpcResponseCallback() { + @Override + public void onSuccess(ByteBuffer response) { + try { + streamHandle = (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response); + logger.trace("Successfully opened blocks {}, preparing to fetch chunks.", streamHandle); + + // Immediately request all chunks -- we expect that the total size of the request is + // reasonable due to higher level chunking in [[ShuffleBlockFetcherIterator]]. + for (int i = 0; i < streamHandle.numChunks; i++) { + if (tempShuffleFileManager != null) { + client.stream(OneForOneStreamManager.genStreamChunkId(streamHandle.streamId, i), + new DownloadCallback(i)); + } else { + client.fetchChunk(streamHandle.streamId, i, chunkCallback); + } + } + } catch (Exception e) { + logger.error("Failed while starting block fetches after success", e); + failRemainingBlocks(blockIds, e); + } + } + + @Override + public void onFailure(Throwable e) { + logger.error("Failed while starting block fetches", e); + failRemainingBlocks(blockIds, e); + } + }); + } + + /** Invokes the "onBlockFetchFailure" callback for every listed block id. */ + private void failRemainingBlocks(String[] failedBlockIds, Throwable e) { + for (String blockId : failedBlockIds) { + try { + listener.onBlockFetchFailure(blockId, e); + } catch (Exception e2) { + logger.error("Error in block fetch failure callback", e2); + } + } + } + + private class DownloadCallback implements StreamCallback { + + private WritableByteChannel channel = null; + private File targetFile = null; + private int chunkIndex; + + DownloadCallback(int chunkIndex) throws IOException { + this.targetFile = tempShuffleFileManager.createTempShuffleFile(); + this.channel = Channels.newChannel(Files.newOutputStream(targetFile.toPath())); + this.chunkIndex = chunkIndex; + } + + @Override + public void onData(String streamId, ByteBuffer buf) throws IOException { + channel.write(buf); + } + + @Override + public void onComplete(String streamId) throws IOException { + channel.close(); + ManagedBuffer buffer = new FileSegmentManagedBuffer(transportConf, targetFile, 0, + targetFile.length()); + listener.onBlockFetchSuccess(blockIds[chunkIndex], buffer); + if (!tempShuffleFileManager.registerTempShuffleFileToClean(targetFile)) { + targetFile.delete(); + } + } + + @Override + public void onFailure(String streamId, Throwable cause) throws IOException { + channel.close(); + // On receipt of a failure, fail every block from chunkIndex onwards. + String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); + failRemainingBlocks(remainingBlockIds, cause); + targetFile.delete(); + } + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java similarity index 95% rename from network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java rename to common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java index 4bb0498e5d5a..f309dda8afca 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java @@ -46,7 +46,7 @@ public class RetryingBlockFetcher { * Used to initiate the first fetch for all blocks, and subsequently for retrying the fetch on any * remaining blocks. */ - public static interface BlockFetchStarter { + public interface BlockFetchStarter { /** * Creates a new BlockFetcher to fetch the given block ids which may do some synchronous * bootstrapping followed by fully asynchronous block fetching. @@ -57,14 +57,15 @@ public static interface BlockFetchStarter { * {@link org.apache.spark.network.client.TransportClientFactory} in order to fix connection * issues. */ - void createAndStart(String[] blockIds, BlockFetchingListener listener) throws IOException; + void createAndStart(String[] blockIds, BlockFetchingListener listener) + throws IOException, InterruptedException; } /** Shared executor service used for waiting and retrying. */ private static final ExecutorService executorService = Executors.newCachedThreadPool( NettyUtils.createThreadFactory("Block Fetch Retry")); - private final Logger logger = LoggerFactory.getLogger(RetryingBlockFetcher.class); + private static final Logger logger = LoggerFactory.getLogger(RetryingBlockFetcher.class); /** Used to initiate new Block Fetches on our remaining blocks. */ private final BlockFetchStarter fetchStarter; @@ -163,12 +164,9 @@ private synchronized void initiateRetry() { logger.info("Retrying fetch ({}/{}) for {} outstanding blocks after {} ms", retryCount, maxRetries, outstandingBlocksIds.size(), retryWaitTime); - executorService.submit(new Runnable() { - @Override - public void run() { - Uninterruptibles.sleepUninterruptibly(retryWaitTime, TimeUnit.MILLISECONDS); - fetchAllOutstanding(); - } + executorService.submit(() -> { + Uninterruptibles.sleepUninterruptibly(retryWaitTime, TimeUnit.MILLISECONDS); + fetchAllOutstanding(); }); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java new file mode 100644 index 000000000000..5bd4412b7527 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.Closeable; +import java.util.Collections; + +import com.codahale.metrics.MetricSet; + +/** Provides an interface for reading shuffle files, either from an Executor or external service. */ +public abstract class ShuffleClient implements Closeable { + + /** + * Initializes the ShuffleClient, specifying this Executor's appId. + * Must be called before any other method on the ShuffleClient. + */ + public void init(String appId) { } + + /** + * Fetch a sequence of blocks from a remote node asynchronously, + * + * Note that this API takes a sequence so the implementation can batch requests, and does not + * return a future so the underlying implementation can invoke onBlockFetchSuccess as soon as + * the data of a block is fetched, rather than waiting for all blocks to be fetched. + * + * @param host the host of the remote node. + * @param port the port of the remote node. + * @param execId the executor id. + * @param blockIds block ids to fetch. + * @param listener the listener to receive block fetching status. + * @param tempShuffleFileManager TempShuffleFileManager to create and clean temp shuffle files. + * If it's not null, the remote blocks will be streamed + * into temp shuffle files to reduce the memory usage, otherwise, + * they will be kept in memory. + */ + public abstract void fetchBlocks( + String host, + int port, + String execId, + String[] blockIds, + BlockFetchingListener listener, + TempShuffleFileManager tempShuffleFileManager); + + /** + * Get the shuffle MetricsSet from ShuffleClient, this will be used in MetricsSystem to + * get the Shuffle related metrics. + */ + public MetricSet shuffleMetrics() { + // Return an empty MetricSet by default. + return () -> Collections.emptyMap(); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java new file mode 100644 index 000000000000..386738ece51a --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.DataInputStream; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.LongBuffer; +import java.nio.file.Files; + +/** + * Keeps the index information for a particular map output + * as an in-memory LongBuffer. + */ +public class ShuffleIndexInformation { + /** offsets as long buffer */ + private final LongBuffer offsets; + private int size; + + public ShuffleIndexInformation(File indexFile) throws IOException { + size = (int)indexFile.length(); + ByteBuffer buffer = ByteBuffer.allocate(size); + offsets = buffer.asLongBuffer(); + DataInputStream dis = null; + try { + dis = new DataInputStream(Files.newInputStream(indexFile.toPath())); + dis.readFully(buffer.array()); + } finally { + if (dis != null) { + dis.close(); + } + } + } + + /** + * Size of the index file + * @return size + */ + public int getSize() { + return size; + } + + /** + * Get index offset for a particular reducer. + */ + public ShuffleIndexRecord getIndex(int reduceId) { + long offset = offsets.get(reduceId); + long nextOffset = offsets.get(reduceId + 1); + return new ShuffleIndexRecord(offset, nextOffset - offset); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexRecord.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexRecord.java new file mode 100644 index 000000000000..6a4fac150a6b --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexRecord.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +/** + * Contains offset and length of the shuffle block data. + */ +public class ShuffleIndexRecord { + private final long offset; + private final long length; + + public ShuffleIndexRecord(long offset, long length) { + this.offset = offset; + this.length = length; + } + + public long getOffset() { + return offset; + } + + public long getLength() { + return length; + } +} + diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/TempShuffleFileManager.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/TempShuffleFileManager.java new file mode 100644 index 000000000000..84a5ed6a276b --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/TempShuffleFileManager.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.File; + +/** + * A manager to create temp shuffle block files to reduce the memory usage and also clean temp + * files when they won't be used any more. + */ +public interface TempShuffleFileManager { + + /** Create a temp shuffle block file. */ + File createTempShuffleFile(); + + /** + * Register a temp shuffle file to clean up when it won't be used any more. Return whether the + * file is registered successfully. If `false`, the caller should clean up the file by itself. + */ + boolean registerTempShuffleFileToClean(File file); +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java new file mode 100644 index 000000000000..60179f126bc4 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle.mesos; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.spark.network.shuffle.protocol.mesos.ShuffleServiceHeartbeat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.shuffle.ExternalShuffleClient; +import org.apache.spark.network.shuffle.protocol.mesos.RegisterDriver; +import org.apache.spark.network.util.TransportConf; + +/** + * A client for talking to the external shuffle service in Mesos coarse-grained mode. + * + * This is used by the Spark driver to register with each external shuffle service on the cluster. + * The reason why the driver has to talk to the service is for cleaning up shuffle files reliably + * after the application exits. Mesos does not provide a great alternative to do this, so Spark + * has to detect this itself. + */ +public class MesosExternalShuffleClient extends ExternalShuffleClient { + private static final Logger logger = LoggerFactory.getLogger(MesosExternalShuffleClient.class); + + private final ScheduledExecutorService heartbeaterThread = + Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("mesos-external-shuffle-client-heartbeater") + .build()); + + /** + * Creates an Mesos external shuffle client that wraps the {@link ExternalShuffleClient}. + * Please refer to docs on {@link ExternalShuffleClient} for more information. + */ + public MesosExternalShuffleClient( + TransportConf conf, + SecretKeyHolder secretKeyHolder, + boolean authEnabled, + long registrationTimeoutMs) { + super(conf, secretKeyHolder, authEnabled, registrationTimeoutMs); + } + + public void registerDriverWithShuffleService( + String host, + int port, + long heartbeatTimeoutMs, + long heartbeatIntervalMs) throws IOException, InterruptedException { + + checkInit(); + ByteBuffer registerDriver = new RegisterDriver(appId, heartbeatTimeoutMs).toByteBuffer(); + TransportClient client = clientFactory.createClient(host, port); + client.sendRpc(registerDriver, new RegisterDriverCallback(client, heartbeatIntervalMs)); + } + + private class RegisterDriverCallback implements RpcResponseCallback { + private final TransportClient client; + private final long heartbeatIntervalMs; + + private RegisterDriverCallback(TransportClient client, long heartbeatIntervalMs) { + this.client = client; + this.heartbeatIntervalMs = heartbeatIntervalMs; + } + + @Override + public void onSuccess(ByteBuffer response) { + heartbeaterThread.scheduleAtFixedRate( + new Heartbeater(client), 0, heartbeatIntervalMs, TimeUnit.MILLISECONDS); + logger.info("Successfully registered app " + appId + " with external shuffle service."); + } + + @Override + public void onFailure(Throwable e) { + logger.warn("Unable to register app " + appId + " with external shuffle service. " + + "Please manually remove shuffle data after driver exit. Error: " + e); + } + } + + @Override + public void close() { + heartbeaterThread.shutdownNow(); + super.close(); + } + + private class Heartbeater implements Runnable { + + private final TransportClient client; + + private Heartbeater(TransportClient client) { + this.client = client; + } + + @Override + public void run() { + // TODO: Stop sending heartbeats if the shuffle service has lost the app due to timeout + client.send(new ShuffleServiceHeartbeat(appId).toByteBuffer()); + } + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java similarity index 88% rename from network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java rename to common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java index fcb52363e632..9af6759f5d5f 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java @@ -17,11 +17,14 @@ package org.apache.spark.network.shuffle.protocol; +import java.nio.ByteBuffer; + import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import org.apache.spark.network.protocol.Encodable; import org.apache.spark.network.shuffle.protocol.mesos.RegisterDriver; +import org.apache.spark.network.shuffle.protocol.mesos.ShuffleServiceHeartbeat; /** * Messages handled by the {@link org.apache.spark.network.shuffle.ExternalShuffleBlockHandler}, or @@ -37,12 +40,13 @@ public abstract class BlockTransferMessage implements Encodable { protected abstract Type type(); /** Preceding every serialized message is its type, which allows us to deserialize it. */ - public static enum Type { - OPEN_BLOCKS(0), UPLOAD_BLOCK(1), REGISTER_EXECUTOR(2), STREAM_HANDLE(3), REGISTER_DRIVER(4); + public enum Type { + OPEN_BLOCKS(0), UPLOAD_BLOCK(1), REGISTER_EXECUTOR(2), STREAM_HANDLE(3), REGISTER_DRIVER(4), + HEARTBEAT(5); private final byte id; - private Type(int id) { + Type(int id) { assert id < 128 : "Cannot have more than 128 message types"; this.id = (byte) id; } @@ -53,7 +57,7 @@ private Type(int id) { // NB: Java does not support static methods in interfaces, so we must put this in a static class. public static class Decoder { /** Deserializes the 'type' byte followed by the message itself. */ - public static BlockTransferMessage fromByteArray(byte[] msg) { + public static BlockTransferMessage fromByteBuffer(ByteBuffer msg) { ByteBuf buf = Unpooled.wrappedBuffer(msg); byte type = buf.readByte(); switch (type) { @@ -62,18 +66,19 @@ public static BlockTransferMessage fromByteArray(byte[] msg) { case 2: return RegisterExecutor.decode(buf); case 3: return StreamHandle.decode(buf); case 4: return RegisterDriver.decode(buf); + case 5: return ShuffleServiceHeartbeat.decode(buf); default: throw new IllegalArgumentException("Unknown message type: " + type); } } } /** Serializes the 'type' byte followed by the message itself. */ - public byte[] toByteArray() { + public ByteBuffer toByteBuffer() { // Allow room for encoded message, plus the type byte ByteBuf buf = Unpooled.buffer(encodedLength() + 1); buf.writeByte(type().id); encode(buf); assert buf.writableBytes() == 0 : "Writable bytes remain: " + buf.writableBytes(); - return buf.array(); + return buf.nioBuffer(); } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java similarity index 97% rename from network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java rename to common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java index 102d4efb8bf3..93758bdc58fb 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java @@ -33,7 +33,7 @@ public class ExecutorShuffleInfo implements Encodable { public final String[] localDirs; /** Number of subdirectories created within each localDir. */ public final int subDirsPerLocalDir; - /** Shuffle manager (SortShuffleManager or HashShuffleManager) that the executor is using. */ + /** Shuffle manager (SortShuffleManager) that the executor is using. */ public final String shuffleManager; @JsonCreator diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java rename to common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java rename to common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java rename to common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java rename to common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java new file mode 100644 index 000000000000..d5f53ccb7f74 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java @@ -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.network.shuffle.protocol.mesos; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.protocol.Encoders; +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; + +// Needed by ScalaDoc. See SPARK-7726 +import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; + +/** + * A message sent from the driver to register with the MesosExternalShuffleService. + */ +public class RegisterDriver extends BlockTransferMessage { + private final String appId; + private final long heartbeatTimeoutMs; + + public RegisterDriver(String appId, long heartbeatTimeoutMs) { + this.appId = appId; + this.heartbeatTimeoutMs = heartbeatTimeoutMs; + } + + public String getAppId() { return appId; } + + public long getHeartbeatTimeoutMs() { return heartbeatTimeoutMs; } + + @Override + protected Type type() { return Type.REGISTER_DRIVER; } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(appId) + Long.SIZE / Byte.SIZE; + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + buf.writeLong(heartbeatTimeoutMs); + } + + @Override + public int hashCode() { + return Objects.hashCode(appId, heartbeatTimeoutMs); + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof RegisterDriver)) { + return false; + } + return Objects.equal(appId, ((RegisterDriver) o).appId); + } + + public static RegisterDriver decode(ByteBuf buf) { + String appId = Encoders.Strings.decode(buf); + long heartbeatTimeout = buf.readLong(); + return new RegisterDriver(appId, heartbeatTimeout); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/ShuffleServiceHeartbeat.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/ShuffleServiceHeartbeat.java new file mode 100644 index 000000000000..b30bb9aed55b --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/ShuffleServiceHeartbeat.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle.protocol.mesos; + +import io.netty.buffer.ByteBuf; +import org.apache.spark.network.protocol.Encoders; +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; + +// Needed by ScalaDoc. See SPARK-7726 +import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; + +/** + * A heartbeat sent from the driver to the MesosExternalShuffleService. + */ +public class ShuffleServiceHeartbeat extends BlockTransferMessage { + private final String appId; + + public ShuffleServiceHeartbeat(String appId) { + this.appId = appId; + } + + public String getAppId() { return appId; } + + @Override + protected Type type() { return Type.HEARTBEAT; } + + @Override + public int encodedLength() { return Encoders.Strings.encodedLength(appId); } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + } + + public static ShuffleServiceHeartbeat decode(ByteBuf buf) { + return new ShuffleServiceHeartbeat(Encoders.Strings.decode(buf)); + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java similarity index 76% rename from network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java rename to common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java index c393a5e1e681..02e6eb3a4467 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java @@ -18,10 +18,12 @@ package org.apache.spark.network.sasl; import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; -import com.google.common.collect.Lists; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -36,7 +38,6 @@ import org.apache.spark.network.client.ChunkReceivedCallback; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.client.TransportClientBootstrap; import org.apache.spark.network.client.TransportClientFactory; import org.apache.spark.network.server.OneForOneStreamManager; import org.apache.spark.network.server.RpcHandler; @@ -52,14 +53,15 @@ import org.apache.spark.network.shuffle.protocol.OpenBlocks; import org.apache.spark.network.shuffle.protocol.RegisterExecutor; import org.apache.spark.network.shuffle.protocol.StreamHandle; -import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; public class SaslIntegrationSuite { // Use a long timeout to account for slow / overloaded build machines. In the normal case, // tests should finish way before the timeout expires. - private final static long TIMEOUT_MS = 10_000; + private static final long TIMEOUT_MS = 10_000; static TransportServer server; static TransportConf conf; @@ -70,7 +72,7 @@ public class SaslIntegrationSuite { @BeforeClass public static void beforeAll() throws IOException { - conf = new TransportConf(new SystemPropertyConfigProvider()); + conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); context = new TransportContext(conf, new TestRpcHandler()); secretKeyHolder = mock(SecretKeyHolder.class); @@ -100,15 +102,14 @@ public void afterEach() { } @Test - public void testGoodClient() throws IOException { + public void testGoodClient() throws IOException, InterruptedException { clientFactory = context.createClientFactory( - Lists.newArrayList( - new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); + Arrays.asList(new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); String msg = "Hello, World!"; - byte[] resp = client.sendRpcSync(msg.getBytes(), TIMEOUT_MS); - assertEquals(msg, new String(resp)); // our rpc handler should just return the given msg + ByteBuffer resp = client.sendRpcSync(JavaUtils.stringToBytes(msg), TIMEOUT_MS); + assertEquals(msg, JavaUtils.bytesToString(resp)); } @Test @@ -117,8 +118,7 @@ public void testBadClient() { when(badKeyHolder.getSaslUser(anyString())).thenReturn("other-app"); when(badKeyHolder.getSecretKey(anyString())).thenReturn("wrong-password"); clientFactory = context.createClientFactory( - Lists.newArrayList( - new SaslClientBootstrap(conf, "unknown-app", badKeyHolder))); + Arrays.asList(new SaslClientBootstrap(conf, "unknown-app", badKeyHolder))); try { // Bootstrap should fail on startup. @@ -130,13 +130,12 @@ public void testBadClient() { } @Test - public void testNoSaslClient() throws IOException { - clientFactory = context.createClientFactory( - Lists.newArrayList()); + public void testNoSaslClient() throws IOException, InterruptedException { + clientFactory = context.createClientFactory(new ArrayList<>()); TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); try { - client.sendRpcSync(new byte[13], TIMEOUT_MS); + client.sendRpcSync(ByteBuffer.allocate(13), TIMEOUT_MS); fail("Should have failed"); } catch (Exception e) { assertTrue(e.getMessage(), e.getMessage().contains("Expected SaslMessage")); @@ -144,7 +143,7 @@ public void testNoSaslClient() throws IOException { try { // Guessing the right tag byte doesn't magically get you in... - client.sendRpcSync(new byte[] { (byte) 0xEA }, TIMEOUT_MS); + client.sendRpcSync(ByteBuffer.wrap(new byte[] { (byte) 0xEA }), TIMEOUT_MS); fail("Should have failed"); } catch (Exception e) { assertTrue(e.getMessage(), e.getMessage().contains("java.lang.IndexOutOfBoundsException")); @@ -156,15 +155,11 @@ public void testNoSaslServer() { RpcHandler handler = new TestRpcHandler(); TransportContext context = new TransportContext(conf, handler); clientFactory = context.createClientFactory( - Lists.newArrayList( - new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); - TransportServer server = context.createServer(); - try { + Arrays.asList(new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); + try (TransportServer server = context.createServer()) { clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); } catch (Exception e) { assertTrue(e.getMessage(), e.getMessage().contains("Digest-challenge format violation")); - } finally { - server.close(); } } @@ -188,75 +183,69 @@ public void testAppIsolation() throws Exception { try { // Create a client, and make a request to fetch blocks from a different app. clientFactory = blockServerContext.createClientFactory( - Lists.newArrayList( - new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); + Arrays.asList(new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); client1 = clientFactory.createClient(TestUtils.getLocalHost(), blockServer.getPort()); - final AtomicReference exception = new AtomicReference<>(); + AtomicReference exception = new AtomicReference<>(); + CountDownLatch blockFetchLatch = new CountDownLatch(1); BlockFetchingListener listener = new BlockFetchingListener() { @Override - public synchronized void onBlockFetchSuccess(String blockId, ManagedBuffer data) { - notifyAll(); + public void onBlockFetchSuccess(String blockId, ManagedBuffer data) { + blockFetchLatch.countDown(); } - @Override - public synchronized void onBlockFetchFailure(String blockId, Throwable t) { + public void onBlockFetchFailure(String blockId, Throwable t) { exception.set(t); - notifyAll(); + blockFetchLatch.countDown(); } }; - String[] blockIds = new String[] { "shuffle_2_3_4", "shuffle_6_7_8" }; - OneForOneBlockFetcher fetcher = new OneForOneBlockFetcher(client1, "app-2", "0", - blockIds, listener); - synchronized (listener) { - fetcher.start(); - listener.wait(); - } + String[] blockIds = { "shuffle_0_1_2", "shuffle_0_3_4" }; + OneForOneBlockFetcher fetcher = + new OneForOneBlockFetcher(client1, "app-2", "0", blockIds, listener, conf); + fetcher.start(); + blockFetchLatch.await(); checkSecurityException(exception.get()); // Register an executor so that the next steps work. ExecutorShuffleInfo executorInfo = new ExecutorShuffleInfo( new String[] { System.getProperty("java.io.tmpdir") }, 1, - "org.apache.spark.shuffle.sort.SortShuffleManager"); + "org.apache.spark.shuffle.sort.SortShuffleManager"); RegisterExecutor regmsg = new RegisterExecutor("app-1", "0", executorInfo); - client1.sendRpcSync(regmsg.toByteArray(), TIMEOUT_MS); + client1.sendRpcSync(regmsg.toByteBuffer(), TIMEOUT_MS); // Make a successful request to fetch blocks, which creates a new stream. But do not actually // fetch any blocks, to keep the stream open. OpenBlocks openMessage = new OpenBlocks("app-1", "0", blockIds); - byte[] response = client1.sendRpcSync(openMessage.toByteArray(), TIMEOUT_MS); - StreamHandle stream = (StreamHandle) BlockTransferMessage.Decoder.fromByteArray(response); + ByteBuffer response = client1.sendRpcSync(openMessage.toByteBuffer(), TIMEOUT_MS); + StreamHandle stream = (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response); long streamId = stream.streamId; // Create a second client, authenticated with a different app ID, and try to read from // the stream created for the previous app. clientFactory2 = blockServerContext.createClientFactory( - Lists.newArrayList( - new SaslClientBootstrap(conf, "app-2", secretKeyHolder))); + Arrays.asList(new SaslClientBootstrap(conf, "app-2", secretKeyHolder))); client2 = clientFactory2.createClient(TestUtils.getLocalHost(), blockServer.getPort()); + CountDownLatch chunkReceivedLatch = new CountDownLatch(1); ChunkReceivedCallback callback = new ChunkReceivedCallback() { @Override - public synchronized void onSuccess(int chunkIndex, ManagedBuffer buffer) { - notifyAll(); + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + chunkReceivedLatch.countDown(); } - @Override - public synchronized void onFailure(int chunkIndex, Throwable t) { + public void onFailure(int chunkIndex, Throwable t) { exception.set(t); - notifyAll(); + chunkReceivedLatch.countDown(); } }; exception.set(null); - synchronized (callback) { - client2.fetchChunk(streamId, 0, callback); - callback.wait(); - } + client2.fetchChunk(streamId, 0, callback); + chunkReceivedLatch.await(); checkSecurityException(exception.get()); } finally { if (client1 != null) { @@ -275,7 +264,7 @@ public synchronized void onFailure(int chunkIndex, Throwable t) { /** RPC handler which simply responds with the message it received. */ public static class TestRpcHandler extends RpcHandler { @Override - public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + public void receive(TransportClient client, ByteBuffer message, RpcResponseCallback callback) { callback.onSuccess(message); } @@ -285,7 +274,7 @@ public StreamManager getStreamManager() { } } - private void checkSecurityException(Throwable t) { + private static void checkSecurityException(Throwable t) { assertNotNull("No exception was caught.", t); assertTrue("Expected SecurityException.", t.getMessage().contains(SecurityException.class.getName())); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java similarity index 98% rename from network/shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java rename to common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java index d65de9ca550a..86c8609e7070 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java @@ -36,7 +36,7 @@ public void serializeOpenShuffleBlocks() { } private void checkSerializeDeserialize(BlockTransferMessage msg) { - BlockTransferMessage msg2 = BlockTransferMessage.Decoder.fromByteArray(msg.toByteArray()); + BlockTransferMessage msg2 = BlockTransferMessage.Decoder.fromByteBuffer(msg.toByteBuffer()); assertEquals(msg, msg2); assertEquals(msg.hashCode(), msg2.hashCode()); assertEquals(msg.toString(), msg2.toString()); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java new file mode 100644 index 000000000000..7846b71d5a8b --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.nio.ByteBuffer; +import java.util.Iterator; + +import com.codahale.metrics.Meter; +import com.codahale.metrics.Timer; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; + +import static org.junit.Assert.*; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.*; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.server.OneForOneStreamManager; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.apache.spark.network.shuffle.protocol.OpenBlocks; +import org.apache.spark.network.shuffle.protocol.RegisterExecutor; +import org.apache.spark.network.shuffle.protocol.StreamHandle; +import org.apache.spark.network.shuffle.protocol.UploadBlock; + +public class ExternalShuffleBlockHandlerSuite { + TransportClient client = mock(TransportClient.class); + + OneForOneStreamManager streamManager; + ExternalShuffleBlockResolver blockResolver; + RpcHandler handler; + + @Before + public void beforeEach() { + streamManager = mock(OneForOneStreamManager.class); + blockResolver = mock(ExternalShuffleBlockResolver.class); + handler = new ExternalShuffleBlockHandler(streamManager, blockResolver); + } + + @Test + public void testRegisterExecutor() { + RpcResponseCallback callback = mock(RpcResponseCallback.class); + + ExecutorShuffleInfo config = new ExecutorShuffleInfo(new String[] {"/a", "/b"}, 16, "sort"); + ByteBuffer registerMessage = new RegisterExecutor("app0", "exec1", config).toByteBuffer(); + handler.receive(client, registerMessage, callback); + verify(blockResolver, times(1)).registerExecutor("app0", "exec1", config); + + verify(callback, times(1)).onSuccess(any(ByteBuffer.class)); + verify(callback, never()).onFailure(any(Throwable.class)); + // Verify register executor request latency metrics + Timer registerExecutorRequestLatencyMillis = (Timer) ((ExternalShuffleBlockHandler) handler) + .getAllMetrics() + .getMetrics() + .get("registerExecutorRequestLatencyMillis"); + assertEquals(1, registerExecutorRequestLatencyMillis.getCount()); + } + + @SuppressWarnings("unchecked") + @Test + public void testOpenShuffleBlocks() { + RpcResponseCallback callback = mock(RpcResponseCallback.class); + + ManagedBuffer block0Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[3])); + ManagedBuffer block1Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); + when(blockResolver.getBlockData("app0", "exec1", 0, 0, 0)).thenReturn(block0Marker); + when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1)).thenReturn(block1Marker); + ByteBuffer openBlocks = new OpenBlocks("app0", "exec1", + new String[] { "shuffle_0_0_0", "shuffle_0_0_1" }) + .toByteBuffer(); + handler.receive(client, openBlocks, callback); + + ArgumentCaptor response = ArgumentCaptor.forClass(ByteBuffer.class); + verify(callback, times(1)).onSuccess(response.capture()); + verify(callback, never()).onFailure(any()); + + StreamHandle handle = + (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response.getValue()); + assertEquals(2, handle.numChunks); + + @SuppressWarnings("unchecked") + ArgumentCaptor> stream = (ArgumentCaptor>) + (ArgumentCaptor) ArgumentCaptor.forClass(Iterator.class); + verify(streamManager, times(1)).registerStream(anyString(), stream.capture()); + Iterator buffers = stream.getValue(); + assertEquals(block0Marker, buffers.next()); + assertEquals(block1Marker, buffers.next()); + assertFalse(buffers.hasNext()); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 1); + + // Verify open block request latency metrics + Timer openBlockRequestLatencyMillis = (Timer) ((ExternalShuffleBlockHandler) handler) + .getAllMetrics() + .getMetrics() + .get("openBlockRequestLatencyMillis"); + assertEquals(1, openBlockRequestLatencyMillis.getCount()); + // Verify block transfer metrics + Meter blockTransferRateBytes = (Meter) ((ExternalShuffleBlockHandler) handler) + .getAllMetrics() + .getMetrics() + .get("blockTransferRateBytes"); + assertEquals(10, blockTransferRateBytes.getCount()); + } + + @Test + public void testBadMessages() { + RpcResponseCallback callback = mock(RpcResponseCallback.class); + + ByteBuffer unserializableMsg = ByteBuffer.wrap(new byte[] { 0x12, 0x34, 0x56 }); + try { + handler.receive(client, unserializableMsg, callback); + fail("Should have thrown"); + } catch (Exception e) { + // pass + } + + ByteBuffer unexpectedMsg = new UploadBlock("a", "e", "b", new byte[1], + new byte[2]).toByteBuffer(); + try { + handler.receive(client, unexpectedMsg, callback); + fail("Should have thrown"); + } catch (UnsupportedOperationException e) { + // pass + } + + verify(callback, never()).onSuccess(any(ByteBuffer.class)); + verify(callback, never()).onFailure(any(Throwable.class)); + } +} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java new file mode 100644 index 000000000000..23438a08fa09 --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.io.CharStreams; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.TransportConf; +import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class ExternalShuffleBlockResolverSuite { + private static final String sortBlock0 = "Hello!"; + private static final String sortBlock1 = "World!"; + private static final String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"; + + private static TestShuffleDataContext dataContext; + + private static final TransportConf conf = + new TransportConf("shuffle", MapConfigProvider.EMPTY); + + @BeforeClass + public static void beforeAll() throws IOException { + dataContext = new TestShuffleDataContext(2, 5); + + dataContext.create(); + // Write some sort data. + dataContext.insertSortShuffleData(0, 0, new byte[][] { + sortBlock0.getBytes(StandardCharsets.UTF_8), + sortBlock1.getBytes(StandardCharsets.UTF_8)}); + } + + @AfterClass + public static void afterAll() { + dataContext.cleanup(); + } + + @Test + public void testBadRequests() throws IOException { + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); + // Unregistered executor + try { + resolver.getBlockData("app0", "exec1", 1, 1, 0); + fail("Should have failed"); + } catch (RuntimeException e) { + assertTrue("Bad error message: " + e, e.getMessage().contains("not registered")); + } + + // Invalid shuffle manager + try { + resolver.registerExecutor("app0", "exec2", dataContext.createExecutorInfo("foobar")); + resolver.getBlockData("app0", "exec2", 1, 1, 0); + fail("Should have failed"); + } catch (UnsupportedOperationException e) { + // pass + } + + // Nonexistent shuffle block + resolver.registerExecutor("app0", "exec3", + dataContext.createExecutorInfo(SORT_MANAGER)); + try { + resolver.getBlockData("app0", "exec3", 1, 1, 0); + fail("Should have failed"); + } catch (Exception e) { + // pass + } + } + + @Test + public void testSortShuffleBlocks() throws IOException { + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); + resolver.registerExecutor("app0", "exec0", + dataContext.createExecutorInfo(SORT_MANAGER)); + + InputStream block0Stream = + resolver.getBlockData("app0", "exec0", 0, 0, 0).createInputStream(); + String block0 = CharStreams.toString( + new InputStreamReader(block0Stream, StandardCharsets.UTF_8)); + block0Stream.close(); + assertEquals(sortBlock0, block0); + + InputStream block1Stream = + resolver.getBlockData("app0", "exec0", 0, 0, 1).createInputStream(); + String block1 = CharStreams.toString( + new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); + block1Stream.close(); + assertEquals(sortBlock1, block1); + } + + @Test + public void jsonSerializationOfExecutorRegistration() throws IOException { + ObjectMapper mapper = new ObjectMapper(); + AppExecId appId = new AppExecId("foo", "bar"); + String appIdJson = mapper.writeValueAsString(appId); + AppExecId parsedAppId = mapper.readValue(appIdJson, AppExecId.class); + assertEquals(parsedAppId, appId); + + ExecutorShuffleInfo shuffleInfo = + new ExecutorShuffleInfo(new String[]{"/bippy", "/flippy"}, 7, SORT_MANAGER); + String shuffleJson = mapper.writeValueAsString(shuffleInfo); + ExecutorShuffleInfo parsedShuffleInfo = + mapper.readValue(shuffleJson, ExecutorShuffleInfo.class); + assertEquals(parsedShuffleInfo, shuffleInfo); + + // Intentionally keep these hard-coded strings in here, to check backwards-compatability. + // its not legacy yet, but keeping this here in case anybody changes it + String legacyAppIdJson = "{\"appId\":\"foo\", \"execId\":\"bar\"}"; + assertEquals(appId, mapper.readValue(legacyAppIdJson, AppExecId.class)); + String legacyShuffleJson = "{\"localDirs\": [\"/bippy\", \"/flippy\"], " + + "\"subDirsPerLocalDir\": 7, \"shuffleManager\": " + "\"" + SORT_MANAGER + "\"}"; + assertEquals(shuffleInfo, mapper.readValue(legacyShuffleJson, ExecutorShuffleInfo.class)); + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java similarity index 81% rename from network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java rename to common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java index 2f4f1d0df478..47c087088a8a 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java @@ -19,6 +19,7 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Random; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; @@ -28,14 +29,15 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; public class ExternalShuffleCleanupSuite { // Same-thread Executor used to ensure cleanup happens synchronously in test thread. - Executor sameThreadExecutor = MoreExecutors.sameThreadExecutor(); - TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + private Executor sameThreadExecutor = MoreExecutors.sameThreadExecutor(); + private TransportConf conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); + private static final String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"; @Test public void noCleanupAndCleanup() throws IOException { @@ -43,12 +45,12 @@ public void noCleanupAndCleanup() throws IOException { ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); - resolver.registerExecutor("app", "exec0", dataContext.createExecutorInfo("shuffleMgr")); + resolver.registerExecutor("app", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)); resolver.applicationRemoved("app", false /* cleanup */); assertStillThere(dataContext); - resolver.registerExecutor("app", "exec1", dataContext.createExecutorInfo("shuffleMgr")); + resolver.registerExecutor("app", "exec1", dataContext.createExecutorInfo(SORT_MANAGER)); resolver.applicationRemoved("app", true /* cleanup */); assertCleanedUp(dataContext); @@ -58,17 +60,15 @@ public void noCleanupAndCleanup() throws IOException { public void cleanupUsesExecutor() throws IOException { TestShuffleDataContext dataContext = createSomeData(); - final AtomicBoolean cleanupCalled = new AtomicBoolean(false); + AtomicBoolean cleanupCalled = new AtomicBoolean(false); // Executor which does nothing to ensure we're actually using it. - Executor noThreadExecutor = new Executor() { - @Override public void execute(Runnable runnable) { cleanupCalled.set(true); } - }; + Executor noThreadExecutor = runnable -> cleanupCalled.set(true); ExternalShuffleBlockResolver manager = new ExternalShuffleBlockResolver(conf, null, noThreadExecutor); - manager.registerExecutor("app", "exec0", dataContext.createExecutorInfo("shuffleMgr")); + manager.registerExecutor("app", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)); manager.applicationRemoved("app", true); assertTrue(cleanupCalled.get()); @@ -86,8 +86,8 @@ public void cleanupMultipleExecutors() throws IOException { ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); - resolver.registerExecutor("app", "exec0", dataContext0.createExecutorInfo("shuffleMgr")); - resolver.registerExecutor("app", "exec1", dataContext1.createExecutorInfo("shuffleMgr")); + resolver.registerExecutor("app", "exec0", dataContext0.createExecutorInfo(SORT_MANAGER)); + resolver.registerExecutor("app", "exec1", dataContext1.createExecutorInfo(SORT_MANAGER)); resolver.applicationRemoved("app", true); assertCleanedUp(dataContext0); @@ -102,8 +102,8 @@ public void cleanupOnlyRemovedApp() throws IOException { ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); - resolver.registerExecutor("app-0", "exec0", dataContext0.createExecutorInfo("shuffleMgr")); - resolver.registerExecutor("app-1", "exec0", dataContext1.createExecutorInfo("shuffleMgr")); + resolver.registerExecutor("app-0", "exec0", dataContext0.createExecutorInfo(SORT_MANAGER)); + resolver.registerExecutor("app-1", "exec0", dataContext1.createExecutorInfo(SORT_MANAGER)); resolver.applicationRemoved("app-nonexistent", true); assertStillThere(dataContext0); @@ -123,27 +123,26 @@ public void cleanupOnlyRemovedApp() throws IOException { assertCleanedUp(dataContext1); } - private void assertStillThere(TestShuffleDataContext dataContext) { + private static void assertStillThere(TestShuffleDataContext dataContext) { for (String localDir : dataContext.localDirs) { assertTrue(localDir + " was cleaned up prematurely", new File(localDir).exists()); } } - private void assertCleanedUp(TestShuffleDataContext dataContext) { + private static void assertCleanedUp(TestShuffleDataContext dataContext) { for (String localDir : dataContext.localDirs) { assertFalse(localDir + " wasn't cleaned up", new File(localDir).exists()); } } - private TestShuffleDataContext createSomeData() throws IOException { + private static TestShuffleDataContext createSomeData() throws IOException { Random rand = new Random(123); TestShuffleDataContext dataContext = new TestShuffleDataContext(10, 5); dataContext.create(); - dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), - new byte[][] { "ABC".getBytes(), "DEF".getBytes() } ); - dataContext.insertHashShuffleData(rand.nextInt(1000), rand.nextInt(1000) + 1000, - new byte[][] { "GHI".getBytes(), "JKLMNOPQRSTUVWXYZ".getBytes() } ); + dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), new byte[][] { + "ABC".getBytes(StandardCharsets.UTF_8), + "DEF".getBytes(StandardCharsets.UTF_8)}); return dataContext; } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java new file mode 100644 index 000000000000..a6a1b8d0ac3f --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -0,0 +1,270 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Sets; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.junit.Assert.*; + +import org.apache.spark.network.TestUtils; +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.TransportConf; + +public class ExternalShuffleIntegrationSuite { + + private static final String APP_ID = "app-id"; + private static final String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"; + + // Executor 0 is sort-based + static TestShuffleDataContext dataContext0; + + static ExternalShuffleBlockHandler handler; + static TransportServer server; + static TransportConf conf; + + static byte[][] exec0Blocks = new byte[][] { + new byte[123], + new byte[12345], + new byte[1234567], + }; + + static byte[][] exec1Blocks = new byte[][] { + new byte[321], + new byte[54321], + }; + + @BeforeClass + public static void beforeAll() throws IOException { + Random rand = new Random(); + + for (byte[] block : exec0Blocks) { + rand.nextBytes(block); + } + for (byte[] block: exec1Blocks) { + rand.nextBytes(block); + } + + dataContext0 = new TestShuffleDataContext(2, 5); + dataContext0.create(); + dataContext0.insertSortShuffleData(0, 0, exec0Blocks); + + conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); + handler = new ExternalShuffleBlockHandler(conf, null); + TransportContext transportContext = new TransportContext(conf, handler); + server = transportContext.createServer(); + } + + @AfterClass + public static void afterAll() { + dataContext0.cleanup(); + server.close(); + } + + @After + public void afterEach() { + handler.applicationRemoved(APP_ID, false /* cleanupLocalDirs */); + } + + static class FetchResult { + public Set successBlocks; + public Set failedBlocks; + public List buffers; + + public void releaseBuffers() { + for (ManagedBuffer buffer : buffers) { + buffer.release(); + } + } + } + + // Fetch a set of blocks from a pre-registered executor. + private FetchResult fetchBlocks(String execId, String[] blockIds) throws Exception { + return fetchBlocks(execId, blockIds, conf, server.getPort()); + } + + // Fetch a set of blocks from a pre-registered executor. Connects to the server on the given port, + // to allow connecting to invalid servers. + private FetchResult fetchBlocks( + String execId, + String[] blockIds, + TransportConf clientConf, + int port) throws Exception { + final FetchResult res = new FetchResult(); + res.successBlocks = Collections.synchronizedSet(new HashSet()); + res.failedBlocks = Collections.synchronizedSet(new HashSet()); + res.buffers = Collections.synchronizedList(new LinkedList()); + + final Semaphore requestsRemaining = new Semaphore(0); + + ExternalShuffleClient client = new ExternalShuffleClient(clientConf, null, false, 5000); + client.init(APP_ID); + client.fetchBlocks(TestUtils.getLocalHost(), port, execId, blockIds, + new BlockFetchingListener() { + @Override + public void onBlockFetchSuccess(String blockId, ManagedBuffer data) { + synchronized (this) { + if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) { + data.retain(); + res.successBlocks.add(blockId); + res.buffers.add(data); + requestsRemaining.release(); + } + } + } + + @Override + public void onBlockFetchFailure(String blockId, Throwable exception) { + synchronized (this) { + if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) { + res.failedBlocks.add(blockId); + requestsRemaining.release(); + } + } + } + }, null); + + if (!requestsRemaining.tryAcquire(blockIds.length, 5, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server"); + } + client.close(); + return res; + } + + @Test + public void testFetchOneSort() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult exec0Fetch = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0" }); + assertEquals(Sets.newHashSet("shuffle_0_0_0"), exec0Fetch.successBlocks); + assertTrue(exec0Fetch.failedBlocks.isEmpty()); + assertBufferListsEqual(exec0Fetch.buffers, Arrays.asList(exec0Blocks[0])); + exec0Fetch.releaseBuffers(); + } + + @Test + public void testFetchThreeSort() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult exec0Fetch = fetchBlocks("exec-0", + new String[] { "shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2" }); + assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2"), + exec0Fetch.successBlocks); + assertTrue(exec0Fetch.failedBlocks.isEmpty()); + assertBufferListsEqual(exec0Fetch.buffers, Arrays.asList(exec0Blocks)); + exec0Fetch.releaseBuffers(); + } + + @Test (expected = RuntimeException.class) + public void testRegisterInvalidExecutor() throws Exception { + registerExecutor("exec-1", dataContext0.createExecutorInfo("unknown sort manager")); + } + + @Test + public void testFetchWrongBlockId() throws Exception { + registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult execFetch = fetchBlocks("exec-1", new String[] { "rdd_1_0_0" }); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet("rdd_1_0_0"), execFetch.failedBlocks); + } + + @Test + public void testFetchNonexistent() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult execFetch = fetchBlocks("exec-0", + new String[] { "shuffle_2_0_0" }); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet("shuffle_2_0_0"), execFetch.failedBlocks); + } + + @Test + public void testFetchWrongExecutor() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult execFetch0 = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0" /* right */}); + FetchResult execFetch1 = fetchBlocks("exec-0", new String[] { "shuffle_1_0_0" /* wrong */ }); + assertEquals(Sets.newHashSet("shuffle_0_0_0"), execFetch0.successBlocks); + assertEquals(Sets.newHashSet("shuffle_1_0_0"), execFetch1.failedBlocks); + } + + @Test + public void testFetchUnregisteredExecutor() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult execFetch = fetchBlocks("exec-2", + new String[] { "shuffle_0_0_0", "shuffle_1_0_0" }); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_1_0_0"), execFetch.failedBlocks); + } + + @Test + public void testFetchNoServer() throws Exception { + TransportConf clientConf = new TransportConf("shuffle", + new MapConfigProvider(ImmutableMap.of("spark.shuffle.io.maxRetries", "0"))); + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult execFetch = fetchBlocks("exec-0", + new String[]{"shuffle_1_0_0", "shuffle_1_0_1"}, clientConf, 1 /* port */); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks); + } + + private static void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo) + throws IOException, InterruptedException { + ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false, 5000); + client.init(APP_ID); + client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), + executorId, executorInfo); + } + + private static void assertBufferListsEqual(List list0, List list1) + throws Exception { + assertEquals(list0.size(), list1.size()); + for (int i = 0; i < list0.size(); i ++) { + assertBuffersEqual(list0.get(i), new NioManagedBuffer(ByteBuffer.wrap(list1.get(i)))); + } + } + + private static void assertBuffersEqual(ManagedBuffer buffer0, ManagedBuffer buffer1) + throws Exception { + ByteBuffer nio0 = buffer0.nioByteBuffer(); + ByteBuffer nio1 = buffer1.nioByteBuffer(); + + int len = nio0.remaining(); + assertEquals(nio0.remaining(), nio1.remaining()); + for (int i = 0; i < len; i ++) { + assertEquals(nio0.get(), nio1.get()); + } + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java similarity index 81% rename from network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java rename to common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java index aa99efda9494..16bad9f1b319 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Arrays; +import com.google.common.collect.ImmutableMap; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -30,16 +31,15 @@ import org.apache.spark.network.TransportContext; import org.apache.spark.network.sasl.SaslServerBootstrap; import org.apache.spark.network.sasl.SecretKeyHolder; -import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.TransportServer; import org.apache.spark.network.server.TransportServerBootstrap; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; -import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; public class ExternalShuffleSecuritySuite { - TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + TransportConf conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); TransportServer server; @Before @@ -60,7 +60,7 @@ public void afterEach() { } @Test - public void testValid() throws IOException { + public void testValid() throws IOException, InterruptedException { validate("my-app-id", "secret", false); } @@ -83,18 +83,26 @@ public void testBadSecret() { } @Test - public void testEncryption() throws IOException { + public void testEncryption() throws IOException, InterruptedException { validate("my-app-id", "secret", true); } /** Creates an ExternalShuffleClient and attempts to register with the server. */ - private void validate(String appId, String secretKey, boolean encrypt) throws IOException { + private void validate(String appId, String secretKey, boolean encrypt) + throws IOException, InterruptedException { + TransportConf testConf = conf; + if (encrypt) { + testConf = new TransportConf("shuffle", new MapConfigProvider( + ImmutableMap.of("spark.authenticate.enableSaslEncryption", "true"))); + } + ExternalShuffleClient client = - new ExternalShuffleClient(conf, new TestSecretKeyHolder(appId, secretKey), true, encrypt); + new ExternalShuffleClient(testConf, new TestSecretKeyHolder(appId, secretKey), true, 5000); client.init(appId); // Registration either succeeds or throws an exception. client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), "exec0", - new ExecutorShuffleInfo(new String[0], 0, "")); + new ExecutorShuffleInfo(new String[0], 0, + "org.apache.spark.shuffle.sort.SortShuffleManager")); client.close(); } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java new file mode 100644 index 000000000000..dc947a619bf0 --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.collect.Maps; +import io.netty.buffer.Unpooled; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NettyManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; +import org.apache.spark.network.shuffle.protocol.OpenBlocks; +import org.apache.spark.network.shuffle.protocol.StreamHandle; +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.TransportConf; + +public class OneForOneBlockFetcherSuite { + + private static final TransportConf conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); + + @Test + public void testFetchOne() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); + + BlockFetchingListener listener = fetchBlocks(blocks); + + verify(listener).onBlockFetchSuccess("shuffle_0_0_0", blocks.get("shuffle_0_0_0")); + } + + @Test + public void testFetchThree() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("b1", new NioManagedBuffer(ByteBuffer.wrap(new byte[23]))); + blocks.put("b2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23]))); + + BlockFetchingListener listener = fetchBlocks(blocks); + + for (int i = 0; i < 3; i ++) { + verify(listener, times(1)).onBlockFetchSuccess("b" + i, blocks.get("b" + i)); + } + } + + @Test + public void testFailure() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("b1", null); + blocks.put("b2", null); + + BlockFetchingListener listener = fetchBlocks(blocks); + + // Each failure will cause a failure to be invoked in all remaining block fetches. + verify(listener, times(1)).onBlockFetchSuccess("b0", blocks.get("b0")); + verify(listener, times(1)).onBlockFetchFailure(eq("b1"), any()); + verify(listener, times(2)).onBlockFetchFailure(eq("b2"), any()); + } + + @Test + public void testFailureAndSuccess() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("b1", null); + blocks.put("b2", new NioManagedBuffer(ByteBuffer.wrap(new byte[21]))); + + BlockFetchingListener listener = fetchBlocks(blocks); + + // We may call both success and failure for the same block. + verify(listener, times(1)).onBlockFetchSuccess("b0", blocks.get("b0")); + verify(listener, times(1)).onBlockFetchFailure(eq("b1"), any()); + verify(listener, times(1)).onBlockFetchSuccess("b2", blocks.get("b2")); + verify(listener, times(1)).onBlockFetchFailure(eq("b2"), any()); + } + + @Test + public void testEmptyBlockFetch() { + try { + fetchBlocks(Maps.newLinkedHashMap()); + fail(); + } catch (IllegalArgumentException e) { + assertEquals("Zero-sized blockIds array", e.getMessage()); + } + } + + /** + * Begins a fetch on the given set of blocks by mocking out the server side of the RPC which + * simply returns the given (BlockId, Block) pairs. + * As "blocks" is a LinkedHashMap, the blocks are guaranteed to be returned in the same order + * that they were inserted in. + * + * If a block's buffer is "null", an exception will be thrown instead. + */ + private static BlockFetchingListener fetchBlocks(LinkedHashMap blocks) { + TransportClient client = mock(TransportClient.class); + BlockFetchingListener listener = mock(BlockFetchingListener.class); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + OneForOneBlockFetcher fetcher = + new OneForOneBlockFetcher(client, "app-id", "exec-id", blockIds, listener, conf); + + // Respond to the "OpenBlocks" message with an appropriate ShuffleStreamHandle with streamId 123 + doAnswer(invocationOnMock -> { + BlockTransferMessage message = BlockTransferMessage.Decoder.fromByteBuffer( + (ByteBuffer) invocationOnMock.getArguments()[0]); + RpcResponseCallback callback = (RpcResponseCallback) invocationOnMock.getArguments()[1]; + callback.onSuccess(new StreamHandle(123, blocks.size()).toByteBuffer()); + assertEquals(new OpenBlocks("app-id", "exec-id", blockIds), message); + return null; + }).when(client).sendRpc(any(ByteBuffer.class), any(RpcResponseCallback.class)); + + // Respond to each chunk request with a single buffer from our blocks array. + AtomicInteger expectedChunkIndex = new AtomicInteger(0); + Iterator blockIterator = blocks.values().iterator(); + doAnswer(invocation -> { + try { + long streamId = (Long) invocation.getArguments()[0]; + int myChunkIndex = (Integer) invocation.getArguments()[1]; + assertEquals(123, streamId); + assertEquals(expectedChunkIndex.getAndIncrement(), myChunkIndex); + + ChunkReceivedCallback callback = (ChunkReceivedCallback) invocation.getArguments()[2]; + ManagedBuffer result = blockIterator.next(); + if (result != null) { + callback.onSuccess(myChunkIndex, result); + } else { + callback.onFailure(myChunkIndex, new RuntimeException("Failed " + myChunkIndex)); + } + } catch (Exception e) { + e.printStackTrace(); + fail("Unexpected failure"); + } + return null; + }).when(client).fetchChunk(anyLong(), anyInt(), any()); + + fetcher.start(); + return listener; + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java similarity index 77% rename from network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java rename to common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java index 06e46f924109..a530e16734db 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java @@ -27,10 +27,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.mockito.stubbing.Stubber; @@ -39,7 +36,7 @@ import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.buffer.NioManagedBuffer; -import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; import static org.apache.spark.network.shuffle.RetryingBlockFetcher.BlockFetchStarter; @@ -53,20 +50,8 @@ public class RetryingBlockFetcherSuite { ManagedBuffer block1 = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); ManagedBuffer block2 = new NioManagedBuffer(ByteBuffer.wrap(new byte[19])); - @Before - public void beforeEach() { - System.setProperty("spark.shuffle.io.maxRetries", "2"); - System.setProperty("spark.shuffle.io.retryWait", "0"); - } - - @After - public void afterEach() { - System.clearProperty("spark.shuffle.io.maxRetries"); - System.clearProperty("spark.shuffle.io.retryWait"); - } - @Test - public void testNoFailures() throws IOException { + public void testNoFailures() throws IOException, InterruptedException { BlockFetchingListener listener = mock(BlockFetchingListener.class); List> interactions = Arrays.asList( @@ -85,7 +70,7 @@ public void testNoFailures() throws IOException { } @Test - public void testUnrecoverableFailure() throws IOException { + public void testUnrecoverableFailure() throws IOException, InterruptedException { BlockFetchingListener listener = mock(BlockFetchingListener.class); List> interactions = Arrays.asList( @@ -98,13 +83,13 @@ public void testUnrecoverableFailure() throws IOException { performInteractions(interactions, listener); - verify(listener).onBlockFetchFailure(eq("b0"), (Throwable) any()); + verify(listener).onBlockFetchFailure(eq("b0"), any()); verify(listener).onBlockFetchSuccess("b1", block1); verifyNoMoreInteractions(listener); } @Test - public void testSingleIOExceptionOnFirst() throws IOException { + public void testSingleIOExceptionOnFirst() throws IOException, InterruptedException { BlockFetchingListener listener = mock(BlockFetchingListener.class); List> interactions = Arrays.asList( @@ -127,7 +112,7 @@ public void testSingleIOExceptionOnFirst() throws IOException { } @Test - public void testSingleIOExceptionOnSecond() throws IOException { + public void testSingleIOExceptionOnSecond() throws IOException, InterruptedException { BlockFetchingListener listener = mock(BlockFetchingListener.class); List> interactions = Arrays.asList( @@ -149,7 +134,7 @@ public void testSingleIOExceptionOnSecond() throws IOException { } @Test - public void testTwoIOExceptions() throws IOException { + public void testTwoIOExceptions() throws IOException, InterruptedException { BlockFetchingListener listener = mock(BlockFetchingListener.class); List> interactions = Arrays.asList( @@ -177,7 +162,7 @@ public void testTwoIOExceptions() throws IOException { } @Test - public void testThreeIOExceptions() throws IOException { + public void testThreeIOExceptions() throws IOException, InterruptedException { BlockFetchingListener listener = mock(BlockFetchingListener.class); List> interactions = Arrays.asList( @@ -204,12 +189,12 @@ public void testThreeIOExceptions() throws IOException { performInteractions(interactions, listener); verify(listener, timeout(5000)).onBlockFetchSuccess("b0", block0); - verify(listener, timeout(5000)).onBlockFetchFailure(eq("b1"), (Throwable) any()); + verify(listener, timeout(5000)).onBlockFetchFailure(eq("b1"), any()); verifyNoMoreInteractions(listener); } @Test - public void testRetryAndUnrecoverable() throws IOException { + public void testRetryAndUnrecoverable() throws IOException, InterruptedException { BlockFetchingListener listener = mock(BlockFetchingListener.class); List> interactions = Arrays.asList( @@ -234,7 +219,7 @@ public void testRetryAndUnrecoverable() throws IOException { performInteractions(interactions, listener); verify(listener, timeout(5000)).onBlockFetchSuccess("b0", block0); - verify(listener, timeout(5000)).onBlockFetchFailure(eq("b1"), (Throwable) any()); + verify(listener, timeout(5000)).onBlockFetchFailure(eq("b1"), any()); verify(listener, timeout(5000)).onBlockFetchSuccess("b2", block2); verifyNoMoreInteractions(listener); } @@ -252,48 +237,48 @@ public void testRetryAndUnrecoverable() throws IOException { @SuppressWarnings("unchecked") private static void performInteractions(List> interactions, BlockFetchingListener listener) - throws IOException { + throws IOException, InterruptedException { - TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + MapConfigProvider provider = new MapConfigProvider(ImmutableMap.of( + "spark.shuffle.io.maxRetries", "2", + "spark.shuffle.io.retryWait", "0")); + TransportConf conf = new TransportConf("shuffle", provider); BlockFetchStarter fetchStarter = mock(BlockFetchStarter.class); Stubber stub = null; // Contains all blockIds that are referenced across all interactions. - final LinkedHashSet blockIds = Sets.newLinkedHashSet(); + LinkedHashSet blockIds = Sets.newLinkedHashSet(); - for (final Map interaction : interactions) { + for (Map interaction : interactions) { blockIds.addAll(interaction.keySet()); - Answer answer = new Answer() { - @Override - public Void answer(InvocationOnMock invocationOnMock) throws Throwable { - try { - // Verify that the RetryingBlockFetcher requested the expected blocks. - String[] requestedBlockIds = (String[]) invocationOnMock.getArguments()[0]; - String[] desiredBlockIds = interaction.keySet().toArray(new String[interaction.size()]); - assertArrayEquals(desiredBlockIds, requestedBlockIds); - - // Now actually invoke the success/failure callbacks on each block. - BlockFetchingListener retryListener = - (BlockFetchingListener) invocationOnMock.getArguments()[1]; - for (Map.Entry block : interaction.entrySet()) { - String blockId = block.getKey(); - Object blockValue = block.getValue(); - - if (blockValue instanceof ManagedBuffer) { - retryListener.onBlockFetchSuccess(blockId, (ManagedBuffer) blockValue); - } else if (blockValue instanceof Exception) { - retryListener.onBlockFetchFailure(blockId, (Exception) blockValue); - } else { - fail("Can only handle ManagedBuffers and Exceptions, got " + blockValue); - } + Answer answer = invocationOnMock -> { + try { + // Verify that the RetryingBlockFetcher requested the expected blocks. + String[] requestedBlockIds = (String[]) invocationOnMock.getArguments()[0]; + String[] desiredBlockIds = interaction.keySet().toArray(new String[interaction.size()]); + assertArrayEquals(desiredBlockIds, requestedBlockIds); + + // Now actually invoke the success/failure callbacks on each block. + BlockFetchingListener retryListener = + (BlockFetchingListener) invocationOnMock.getArguments()[1]; + for (Map.Entry block : interaction.entrySet()) { + String blockId = block.getKey(); + Object blockValue = block.getValue(); + + if (blockValue instanceof ManagedBuffer) { + retryListener.onBlockFetchSuccess(blockId, (ManagedBuffer) blockValue); + } else if (blockValue instanceof Exception) { + retryListener.onBlockFetchFailure(blockId, (Exception) blockValue); + } else { + fail("Can only handle ManagedBuffers and Exceptions, got " + blockValue); } - return null; - } catch (Throwable e) { - e.printStackTrace(); - throw e; } + return null; + } catch (Throwable e) { + e.printStackTrace(); + throw e; } }; @@ -305,8 +290,8 @@ public Void answer(InvocationOnMock invocationOnMock) throws Throwable { } } - assert stub != null; - stub.when(fetchStarter).createAndStart((String[]) any(), (BlockFetchingListener) anyObject()); + assertNotNull(stub); + stub.when(fetchStarter).createAndStart(any(), anyObject()); String[] blockIdArray = blockIds.toArray(new String[blockIds.size()]); new RetryingBlockFetcher(conf, fetchStarter, blockIdArray, listener).start(); } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java new file mode 100644 index 000000000000..81e01949e50f --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; + +import com.google.common.io.Closeables; +import com.google.common.io.Files; + +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.apache.spark.network.util.JavaUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Manages some sort-shuffle data, including the creation + * and cleanup of directories that can be read by the {@link ExternalShuffleBlockResolver}. + */ +public class TestShuffleDataContext { + private static final Logger logger = LoggerFactory.getLogger(TestShuffleDataContext.class); + + public final String[] localDirs; + public final int subDirsPerLocalDir; + + public TestShuffleDataContext(int numLocalDirs, int subDirsPerLocalDir) { + this.localDirs = new String[numLocalDirs]; + this.subDirsPerLocalDir = subDirsPerLocalDir; + } + + public void create() { + for (int i = 0; i < localDirs.length; i ++) { + localDirs[i] = Files.createTempDir().getAbsolutePath(); + + for (int p = 0; p < subDirsPerLocalDir; p ++) { + new File(localDirs[i], String.format("%02x", p)).mkdirs(); + } + } + } + + public void cleanup() { + for (String localDir : localDirs) { + try { + JavaUtils.deleteRecursively(new File(localDir)); + } catch (IOException e) { + logger.warn("Unable to cleanup localDir = " + localDir, e); + } + } + } + + /** Creates reducer blocks in a sort-based data format within our local dirs. */ + public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) throws IOException { + String blockId = "shuffle_" + shuffleId + "_" + mapId + "_0"; + + OutputStream dataStream = null; + DataOutputStream indexStream = null; + boolean suppressExceptionsDuringClose = true; + + try { + dataStream = new FileOutputStream( + ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId + ".data")); + indexStream = new DataOutputStream(new FileOutputStream( + ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId + ".index"))); + + long offset = 0; + indexStream.writeLong(offset); + for (byte[] block : blocks) { + offset += block.length; + dataStream.write(block); + indexStream.writeLong(offset); + } + suppressExceptionsDuringClose = false; + } finally { + Closeables.close(dataStream, suppressExceptionsDuringClose); + Closeables.close(indexStream, suppressExceptionsDuringClose); + } + } + + /** + * Creates an ExecutorShuffleInfo object based on the given shuffle manager which targets this + * context's directories. + */ + public ExecutorShuffleInfo createExecutorInfo(String shuffleManager) { + return new ExecutorShuffleInfo(localDirs, subDirsPerLocalDir, shuffleManager); + } +} diff --git a/common/network-shuffle/src/test/resources/log4j.properties b/common/network-shuffle/src/test/resources/log4j.properties new file mode 100644 index 000000000000..e73978908b68 --- /dev/null +++ b/common/network-shuffle/src/test/resources/log4j.properties @@ -0,0 +1,24 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=DEBUG, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml new file mode 100644 index 000000000000..ec2db6e5bb88 --- /dev/null +++ b/common/network-yarn/pom.xml @@ -0,0 +1,169 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-network-yarn_2.11 + jar + Spark Project YARN Shuffle Service + http://spark.apache.org/ + + network-yarn + + provided + ${project.build.directory}/scala-${scala.binary.version}/spark-${project.version}-yarn-shuffle.jar + org/spark_project/ + + + + + + org.apache.spark + spark-network-shuffle_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-tags_${scala.binary.version} + test + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + + org.apache.hadoop + hadoop-client + + + org.slf4j + slf4j-api + provided + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + ${shuffle.jar} + + + *:* + + + org.scala-lang:scala-library + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + com.fasterxml.jackson + ${spark.shade.packageName}.com.fasterxml.jackson + + com.fasterxml.jackson.** + + + + io.netty + ${spark.shade.packageName}.io.netty + + io.netty.** + + + + + + + package + + shade + + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + verify + + run + + + + + + + + + + + + + + + + + + Verifying dependency shading + + + + + + + + + + + diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java new file mode 100644 index 000000000000..d8b2ed6b5dc7 --- /dev/null +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -0,0 +1,417 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.yarn; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.server.api.*; +import org.apache.spark.network.util.LevelDBProvider; +import org.iq80.leveldb.DB; +import org.iq80.leveldb.DBIterator; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.crypto.AuthServerBootstrap; +import org.apache.spark.network.sasl.ShuffleSecretManager; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.TransportServerBootstrap; +import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler; +import org.apache.spark.network.util.TransportConf; +import org.apache.spark.network.yarn.util.HadoopConfigProvider; + +/** + * An external shuffle service used by Spark on Yarn. + * + * This is intended to be a long-running auxiliary service that runs in the NodeManager process. + * A Spark application may connect to this service by setting `spark.shuffle.service.enabled`. + * The application also automatically derives the service port through `spark.shuffle.service.port` + * specified in the Yarn configuration. This is so that both the clients and the server agree on + * the same port to communicate on. + * + * The service also optionally supports authentication. This ensures that executors from one + * application cannot read the shuffle files written by those from another. This feature can be + * enabled by setting `spark.authenticate` in the Yarn configuration before starting the NM. + * Note that the Spark application must also set `spark.authenticate` manually and, unlike in + * the case of the service port, will not inherit this setting from the Yarn configuration. This + * is because an application running on the same Yarn cluster may choose to not use the external + * shuffle service, in which case its setting of `spark.authenticate` should be independent of + * the service's. + */ +public class YarnShuffleService extends AuxiliaryService { + private static final Logger logger = LoggerFactory.getLogger(YarnShuffleService.class); + + // Port on which the shuffle server listens for fetch requests + private static final String SPARK_SHUFFLE_SERVICE_PORT_KEY = "spark.shuffle.service.port"; + private static final int DEFAULT_SPARK_SHUFFLE_SERVICE_PORT = 7337; + + // Whether the shuffle server should authenticate fetch requests + private static final String SPARK_AUTHENTICATE_KEY = "spark.authenticate"; + private static final boolean DEFAULT_SPARK_AUTHENTICATE = false; + + private static final String RECOVERY_FILE_NAME = "registeredExecutors.ldb"; + private static final String SECRETS_RECOVERY_FILE_NAME = "sparkShuffleRecovery.ldb"; + + // Whether failure during service initialization should stop the NM. + @VisibleForTesting + static final String STOP_ON_FAILURE_KEY = "spark.yarn.shuffle.stopOnFailure"; + private static final boolean DEFAULT_STOP_ON_FAILURE = false; + + // just for testing when you want to find an open port + @VisibleForTesting + static int boundPort = -1; + private static final ObjectMapper mapper = new ObjectMapper(); + private static final String APP_CREDS_KEY_PREFIX = "AppCreds"; + private static final LevelDBProvider.StoreVersion CURRENT_VERSION = new LevelDBProvider + .StoreVersion(1, 0); + + // just for integration tests that want to look at this file -- in general not sensible as + // a static + @VisibleForTesting + static YarnShuffleService instance; + + // An entity that manages the shuffle secret per application + // This is used only if authentication is enabled + @VisibleForTesting + ShuffleSecretManager secretManager; + + // The actual server that serves shuffle files + private TransportServer shuffleServer = null; + + private Configuration _conf = null; + + // The recovery path used to shuffle service recovery + @VisibleForTesting + Path _recoveryPath = null; + + // Handles registering executors and opening shuffle blocks + @VisibleForTesting + ExternalShuffleBlockHandler blockHandler; + + // Where to store & reload executor info for recovering state after an NM restart + @VisibleForTesting + File registeredExecutorFile; + + // Where to store & reload application secrets for recovering state after an NM restart + @VisibleForTesting + File secretsFile; + + private DB db; + + public YarnShuffleService() { + super("spark_shuffle"); + logger.info("Initializing YARN shuffle service for Spark"); + instance = this; + } + + /** + * Return whether authentication is enabled as specified by the configuration. + * If so, fetch requests will fail unless the appropriate authentication secret + * for the application is provided. + */ + private boolean isAuthenticationEnabled() { + return secretManager != null; + } + + /** + * Start the shuffle server with the given configuration. + */ + @Override + protected void serviceInit(Configuration conf) throws Exception { + _conf = conf; + + boolean stopOnFailure = conf.getBoolean(STOP_ON_FAILURE_KEY, DEFAULT_STOP_ON_FAILURE); + + try { + // In case this NM was killed while there were running spark applications, we need to restore + // lost state for the existing executors. We look for an existing file in the NM's local dirs. + // If we don't find one, then we choose a file to use to save the state next time. Even if + // an application was stopped while the NM was down, we expect yarn to call stopApplication() + // when it comes back + if (_recoveryPath != null) { + registeredExecutorFile = initRecoveryDb(RECOVERY_FILE_NAME); + } + + TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); + blockHandler = new ExternalShuffleBlockHandler(transportConf, registeredExecutorFile); + + // If authentication is enabled, set up the shuffle server to use a + // special RPC handler that filters out unauthenticated fetch requests + List bootstraps = Lists.newArrayList(); + boolean authEnabled = conf.getBoolean(SPARK_AUTHENTICATE_KEY, DEFAULT_SPARK_AUTHENTICATE); + if (authEnabled) { + secretManager = new ShuffleSecretManager(); + if (_recoveryPath != null) { + loadSecretsFromDb(); + } + bootstraps.add(new AuthServerBootstrap(transportConf, secretManager)); + } + + int port = conf.getInt( + SPARK_SHUFFLE_SERVICE_PORT_KEY, DEFAULT_SPARK_SHUFFLE_SERVICE_PORT); + TransportContext transportContext = new TransportContext(transportConf, blockHandler); + shuffleServer = transportContext.createServer(port, bootstraps); + // the port should normally be fixed, but for tests its useful to find an open port + port = shuffleServer.getPort(); + boundPort = port; + String authEnabledString = authEnabled ? "enabled" : "not enabled"; + logger.info("Started YARN shuffle service for Spark on port {}. " + + "Authentication is {}. Registered executor file is {}", port, authEnabledString, + registeredExecutorFile); + } catch (Exception e) { + if (stopOnFailure) { + throw e; + } else { + noteFailure(e); + } + } + } + + private void loadSecretsFromDb() throws IOException { + secretsFile = initRecoveryDb(SECRETS_RECOVERY_FILE_NAME); + + // Make sure this is protected in case its not in the NM recovery dir + FileSystem fs = FileSystem.getLocal(_conf); + fs.mkdirs(new Path(secretsFile.getPath()), new FsPermission((short) 0700)); + + db = LevelDBProvider.initLevelDB(secretsFile, CURRENT_VERSION, mapper); + logger.info("Recovery location is: " + secretsFile.getPath()); + if (db != null) { + logger.info("Going to reload spark shuffle data"); + DBIterator itr = db.iterator(); + itr.seek(APP_CREDS_KEY_PREFIX.getBytes(StandardCharsets.UTF_8)); + while (itr.hasNext()) { + Map.Entry e = itr.next(); + String key = new String(e.getKey(), StandardCharsets.UTF_8); + if (!key.startsWith(APP_CREDS_KEY_PREFIX)) { + break; + } + String id = parseDbAppKey(key); + ByteBuffer secret = mapper.readValue(e.getValue(), ByteBuffer.class); + logger.info("Reloading tokens for app: " + id); + secretManager.registerApp(id, secret); + } + } + } + + private static String parseDbAppKey(String s) throws IOException { + if (!s.startsWith(APP_CREDS_KEY_PREFIX)) { + throw new IllegalArgumentException("expected a string starting with " + APP_CREDS_KEY_PREFIX); + } + String json = s.substring(APP_CREDS_KEY_PREFIX.length() + 1); + AppId parsed = mapper.readValue(json, AppId.class); + return parsed.appId; + } + + private static byte[] dbAppKey(AppId appExecId) throws IOException { + // we stick a common prefix on all the keys so we can find them in the DB + String appExecJson = mapper.writeValueAsString(appExecId); + String key = (APP_CREDS_KEY_PREFIX + ";" + appExecJson); + return key.getBytes(StandardCharsets.UTF_8); + } + + @Override + public void initializeApplication(ApplicationInitializationContext context) { + String appId = context.getApplicationId().toString(); + try { + ByteBuffer shuffleSecret = context.getApplicationDataForService(); + if (isAuthenticationEnabled()) { + AppId fullId = new AppId(appId); + if (db != null) { + byte[] key = dbAppKey(fullId); + byte[] value = mapper.writeValueAsString(shuffleSecret).getBytes(StandardCharsets.UTF_8); + db.put(key, value); + } + secretManager.registerApp(appId, shuffleSecret); + } + } catch (Exception e) { + logger.error("Exception when initializing application {}", appId, e); + } + } + + @Override + public void stopApplication(ApplicationTerminationContext context) { + String appId = context.getApplicationId().toString(); + try { + if (isAuthenticationEnabled()) { + AppId fullId = new AppId(appId); + if (db != null) { + try { + db.delete(dbAppKey(fullId)); + } catch (IOException e) { + logger.error("Error deleting {} from executor state db", appId, e); + } + } + secretManager.unregisterApp(appId); + } + blockHandler.applicationRemoved(appId, false /* clean up local dirs */); + } catch (Exception e) { + logger.error("Exception when stopping application {}", appId, e); + } + } + + @Override + public void initializeContainer(ContainerInitializationContext context) { + ContainerId containerId = context.getContainerId(); + logger.info("Initializing container {}", containerId); + } + + @Override + public void stopContainer(ContainerTerminationContext context) { + ContainerId containerId = context.getContainerId(); + logger.info("Stopping container {}", containerId); + } + + /** + * Close the shuffle server to clean up any associated state. + */ + @Override + protected void serviceStop() { + try { + if (shuffleServer != null) { + shuffleServer.close(); + } + if (blockHandler != null) { + blockHandler.close(); + } + if (db != null) { + db.close(); + } + } catch (Exception e) { + logger.error("Exception when stopping service", e); + } + } + + // Not currently used + @Override + public ByteBuffer getMetaData() { + return ByteBuffer.allocate(0); + } + + /** + * Set the recovery path for shuffle service recovery when NM is restarted. This will be call + * by NM if NM recovery is enabled. + */ + @Override + public void setRecoveryPath(Path recoveryPath) { + _recoveryPath = recoveryPath; + } + + /** + * Get the path specific to this auxiliary service to use for recovery. + */ + protected Path getRecoveryPath(String fileName) { + return _recoveryPath; + } + + /** + * Figure out the recovery path and handle moving the DB if YARN NM recovery gets enabled + * and DB exists in the local dir of NM by old version of shuffle service. + */ + protected File initRecoveryDb(String dbName) { + Preconditions.checkNotNull(_recoveryPath, + "recovery path should not be null if NM recovery is enabled"); + + File recoveryFile = new File(_recoveryPath.toUri().getPath(), dbName); + if (recoveryFile.exists()) { + return recoveryFile; + } + + // db doesn't exist in recovery path go check local dirs for it + String[] localDirs = _conf.getTrimmedStrings("yarn.nodemanager.local-dirs"); + for (String dir : localDirs) { + File f = new File(new Path(dir).toUri().getPath(), dbName); + if (f.exists()) { + // If the recovery path is set then either NM recovery is enabled or another recovery + // DB has been initialized. If NM recovery is enabled and had set the recovery path + // make sure to move all DBs to the recovery path from the old NM local dirs. + // If another DB was initialized first just make sure all the DBs are in the same + // location. + Path newLoc = new Path(_recoveryPath, dbName); + Path copyFrom = new Path(f.toURI()); + if (!newLoc.equals(copyFrom)) { + logger.info("Moving " + copyFrom + " to: " + newLoc); + try { + // The move here needs to handle moving non-empty directories across NFS mounts + FileSystem fs = FileSystem.getLocal(_conf); + fs.rename(copyFrom, newLoc); + } catch (Exception e) { + // Fail to move recovery file to new path, just continue on with new DB location + logger.error("Failed to move recovery file {} to the path {}", + dbName, _recoveryPath.toString(), e); + } + } + return new File(newLoc.toUri().getPath()); + } + } + + return new File(_recoveryPath.toUri().getPath(), dbName); + } + + /** + * Simply encodes an application ID. + */ + public static class AppId { + public final String appId; + + @JsonCreator + public AppId(@JsonProperty("appId") String appId) { + this.appId = appId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + AppId appExecId = (AppId) o; + return Objects.equal(appId, appExecId.appId); + } + + @Override + public int hashCode() { + return Objects.hashCode(appId); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .toString(); + } + } + +} diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/util/HadoopConfigProvider.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/util/HadoopConfigProvider.java similarity index 83% rename from network/yarn/src/main/java/org/apache/spark/network/yarn/util/HadoopConfigProvider.java rename to common/network-yarn/src/main/java/org/apache/spark/network/yarn/util/HadoopConfigProvider.java index 884861752e80..8beb03369947 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/util/HadoopConfigProvider.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/util/HadoopConfigProvider.java @@ -17,6 +17,7 @@ package org.apache.spark.network.yarn.util; +import java.util.Map; import java.util.NoSuchElementException; import org.apache.hadoop.conf.Configuration; @@ -39,4 +40,16 @@ public String get(String name) { } return value; } + + @Override + public String get(String name, String defaultValue) { + String value = conf.get(name); + return value == null ? defaultValue : value; + } + + @Override + public Iterable> getAll() { + return conf; + } + } diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml new file mode 100644 index 000000000000..2d59c71cc375 --- /dev/null +++ b/common/sketch/pom.xml @@ -0,0 +1,72 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-sketch_2.11 + jar + Spark Project Sketch + http://spark.apache.org/ + + sketch + + + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + net.alchim31.maven + scala-maven-plugin + + + + -XDignore.symbol.file + + + + + + diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BitArray.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BitArray.java new file mode 100644 index 000000000000..480a0a79db32 --- /dev/null +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BitArray.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.sketch; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.Arrays; + +final class BitArray { + private final long[] data; + private long bitCount; + + static int numWords(long numBits) { + if (numBits <= 0) { + throw new IllegalArgumentException("numBits must be positive, but got " + numBits); + } + long numWords = (long) Math.ceil(numBits / 64.0); + if (numWords > Integer.MAX_VALUE) { + throw new IllegalArgumentException("Can't allocate enough space for " + numBits + " bits"); + } + return (int) numWords; + } + + BitArray(long numBits) { + this(new long[numWords(numBits)]); + } + + private BitArray(long[] data) { + this.data = data; + long bitCount = 0; + for (long word : data) { + bitCount += Long.bitCount(word); + } + this.bitCount = bitCount; + } + + /** Returns true if the bit changed value. */ + boolean set(long index) { + if (!get(index)) { + data[(int) (index >>> 6)] |= (1L << index); + bitCount++; + return true; + } + return false; + } + + boolean get(long index) { + return (data[(int) (index >>> 6)] & (1L << index)) != 0; + } + + /** Number of bits */ + long bitSize() { + return (long) data.length * Long.SIZE; + } + + /** Number of set bits (1s) */ + long cardinality() { + return bitCount; + } + + /** Combines the two BitArrays using bitwise OR. */ + void putAll(BitArray array) { + assert data.length == array.data.length : "BitArrays must be of equal length when merging"; + long bitCount = 0; + for (int i = 0; i < data.length; i++) { + data[i] |= array.data[i]; + bitCount += Long.bitCount(data[i]); + } + this.bitCount = bitCount; + } + + void writeTo(DataOutputStream out) throws IOException { + out.writeInt(data.length); + for (long datum : data) { + out.writeLong(datum); + } + } + + static BitArray readFrom(DataInputStream in) throws IOException { + int numWords = in.readInt(); + long[] data = new long[numWords]; + for (int i = 0; i < numWords; i++) { + data[i] = in.readLong(); + } + return new BitArray(data); + } + + @Override + public boolean equals(Object other) { + if (this == other) return true; + if (other == null || !(other instanceof BitArray)) return false; + BitArray that = (BitArray) other; + return Arrays.equals(data, that.data); + } + + @Override + public int hashCode() { + return Arrays.hashCode(data); + } +} diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java new file mode 100644 index 000000000000..c0b425e72959 --- /dev/null +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java @@ -0,0 +1,236 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.sketch; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** + * A Bloom filter is a space-efficient probabilistic data structure that offers an approximate + * containment test with one-sided error: if it claims that an item is contained in it, this + * might be in error, but if it claims that an item is not contained in it, then this is + * definitely true. Currently supported data types include: + *

    + *
  • {@link Byte}
  • + *
  • {@link Short}
  • + *
  • {@link Integer}
  • + *
  • {@link Long}
  • + *
  • {@link String}
  • + *
+ * The false positive probability ({@code FPP}) of a Bloom filter is defined as the probability that + * {@linkplain #mightContain(Object)} will erroneously return {@code true} for an object that hasu + * not actually been put in the {@code BloomFilter}. + * + * The implementation is largely based on the {@code BloomFilter} class from Guava. + */ +public abstract class BloomFilter { + + public enum Version { + /** + * {@code BloomFilter} binary format version 1. All values written in big-endian order: + *
    + *
  • Version number, always 1 (32 bit)
  • + *
  • Number of hash functions (32 bit)
  • + *
  • Total number of words of the underlying bit array (32 bit)
  • + *
  • The words/longs (numWords * 64 bit)
  • + *
+ */ + V1(1); + + private final int versionNumber; + + Version(int versionNumber) { + this.versionNumber = versionNumber; + } + + int getVersionNumber() { + return versionNumber; + } + } + + /** + * Returns the probability that {@linkplain #mightContain(Object)} erroneously return {@code true} + * for an object that has not actually been put in the {@code BloomFilter}. + * + * Ideally, this number should be close to the {@code fpp} parameter passed in + * {@linkplain #create(long, double)}, or smaller. If it is significantly higher, it is usually + * the case that too many items (more than expected) have been put in the {@code BloomFilter}, + * degenerating it. + */ + public abstract double expectedFpp(); + + /** + * Returns the number of bits in the underlying bit array. + */ + public abstract long bitSize(); + + /** + * Puts an item into this {@code BloomFilter}. Ensures that subsequent invocations of + * {@linkplain #mightContain(Object)} with the same item will always return {@code true}. + * + * @return true if the bloom filter's bits changed as a result of this operation. If the bits + * changed, this is definitely the first time {@code object} has been added to the + * filter. If the bits haven't changed, this might be the first time {@code object} + * has been added to the filter. Note that {@code put(t)} always returns the + * opposite result to what {@code mightContain(t)} would have returned at the time + * it is called. + */ + public abstract boolean put(Object item); + + /** + * A specialized variant of {@link #put(Object)} that only supports {@code String} items. + */ + public abstract boolean putString(String item); + + /** + * A specialized variant of {@link #put(Object)} that only supports {@code long} items. + */ + public abstract boolean putLong(long item); + + /** + * A specialized variant of {@link #put(Object)} that only supports byte array items. + */ + public abstract boolean putBinary(byte[] item); + + /** + * Determines whether a given bloom filter is compatible with this bloom filter. For two + * bloom filters to be compatible, they must have the same bit size. + * + * @param other The bloom filter to check for compatibility. + */ + public abstract boolean isCompatible(BloomFilter other); + + /** + * Combines this bloom filter with another bloom filter by performing a bitwise OR of the + * underlying data. The mutations happen to this instance. Callers must ensure the + * bloom filters are appropriately sized to avoid saturating them. + * + * @param other The bloom filter to combine this bloom filter with. It is not mutated. + * @throws IncompatibleMergeException if {@code isCompatible(other) == false} + */ + public abstract BloomFilter mergeInPlace(BloomFilter other) throws IncompatibleMergeException; + + /** + * Returns {@code true} if the element might have been put in this Bloom filter, + * {@code false} if this is definitely not the case. + */ + public abstract boolean mightContain(Object item); + + /** + * A specialized variant of {@link #mightContain(Object)} that only tests {@code String} items. + */ + public abstract boolean mightContainString(String item); + + /** + * A specialized variant of {@link #mightContain(Object)} that only tests {@code long} items. + */ + public abstract boolean mightContainLong(long item); + + /** + * A specialized variant of {@link #mightContain(Object)} that only tests byte array items. + */ + public abstract boolean mightContainBinary(byte[] item); + + /** + * Writes out this {@link BloomFilter} to an output stream in binary format. It is the caller's + * responsibility to close the stream. + */ + public abstract void writeTo(OutputStream out) throws IOException; + + /** + * Reads in a {@link BloomFilter} from an input stream. It is the caller's responsibility to close + * the stream. + */ + public static BloomFilter readFrom(InputStream in) throws IOException { + return BloomFilterImpl.readFrom(in); + } + + /** + * Computes the optimal k (number of hashes per item inserted in Bloom filter), given the + * expected insertions and total number of bits in the Bloom filter. + * + * See http://en.wikipedia.org/wiki/File:Bloom_filter_fp_probability.svg for the formula. + * + * @param n expected insertions (must be positive) + * @param m total number of bits in Bloom filter (must be positive) + */ + private static int optimalNumOfHashFunctions(long n, long m) { + // (m / n) * log(2), but avoid truncation due to division! + return Math.max(1, (int) Math.round((double) m / n * Math.log(2))); + } + + /** + * Computes m (total bits of Bloom filter) which is expected to achieve, for the specified + * expected insertions, the required false positive probability. + * + * See http://en.wikipedia.org/wiki/Bloom_filter#Probability_of_false_positives for the formula. + * + * @param n expected insertions (must be positive) + * @param p false positive rate (must be 0 < p < 1) + */ + private static long optimalNumOfBits(long n, double p) { + return (long) (-n * Math.log(p) / (Math.log(2) * Math.log(2))); + } + + static final double DEFAULT_FPP = 0.03; + + /** + * Creates a {@link BloomFilter} with the expected number of insertions and a default expected + * false positive probability of 3%. + * + * Note that overflowing a {@code BloomFilter} with significantly more elements than specified, + * will result in its saturation, and a sharp deterioration of its false positive probability. + */ + public static BloomFilter create(long expectedNumItems) { + return create(expectedNumItems, DEFAULT_FPP); + } + + /** + * Creates a {@link BloomFilter} with the expected number of insertions and expected false + * positive probability. + * + * Note that overflowing a {@code BloomFilter} with significantly more elements than specified, + * will result in its saturation, and a sharp deterioration of its false positive probability. + */ + public static BloomFilter create(long expectedNumItems, double fpp) { + if (fpp <= 0D || fpp >= 1D) { + throw new IllegalArgumentException( + "False positive probability must be within range (0.0, 1.0)" + ); + } + + return create(expectedNumItems, optimalNumOfBits(expectedNumItems, fpp)); + } + + /** + * Creates a {@link BloomFilter} with given {@code expectedNumItems} and {@code numBits}, it will + * pick an optimal {@code numHashFunctions} which can minimize {@code fpp} for the bloom filter. + */ + public static BloomFilter create(long expectedNumItems, long numBits) { + if (expectedNumItems <= 0) { + throw new IllegalArgumentException("Expected insertions must be positive"); + } + + if (numBits <= 0) { + throw new IllegalArgumentException("Number of bits must be positive"); + } + + return new BloomFilterImpl(optimalNumOfHashFunctions(expectedNumItems, numBits), numBits); + } +} diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java new file mode 100644 index 000000000000..92c28bcb56a5 --- /dev/null +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.sketch; + +import java.io.*; + +class BloomFilterImpl extends BloomFilter implements Serializable { + + private int numHashFunctions; + + private BitArray bits; + + BloomFilterImpl(int numHashFunctions, long numBits) { + this(new BitArray(numBits), numHashFunctions); + } + + private BloomFilterImpl(BitArray bits, int numHashFunctions) { + this.bits = bits; + this.numHashFunctions = numHashFunctions; + } + + private BloomFilterImpl() {} + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + + if (other == null || !(other instanceof BloomFilterImpl)) { + return false; + } + + BloomFilterImpl that = (BloomFilterImpl) other; + + return this.numHashFunctions == that.numHashFunctions && this.bits.equals(that.bits); + } + + @Override + public int hashCode() { + return bits.hashCode() * 31 + numHashFunctions; + } + + @Override + public double expectedFpp() { + return Math.pow((double) bits.cardinality() / bits.bitSize(), numHashFunctions); + } + + @Override + public long bitSize() { + return bits.bitSize(); + } + + @Override + public boolean put(Object item) { + if (item instanceof String) { + return putString((String) item); + } else if (item instanceof byte[]) { + return putBinary((byte[]) item); + } else { + return putLong(Utils.integralToLong(item)); + } + } + + @Override + public boolean putString(String item) { + return putBinary(Utils.getBytesFromUTF8String(item)); + } + + @Override + public boolean putBinary(byte[] item) { + int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, 0); + int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); + + long bitSize = bits.bitSize(); + boolean bitsChanged = false; + for (int i = 1; i <= numHashFunctions; i++) { + int combinedHash = h1 + (i * h2); + // Flip all the bits if it's negative (guaranteed positive number) + if (combinedHash < 0) { + combinedHash = ~combinedHash; + } + bitsChanged |= bits.set(combinedHash % bitSize); + } + return bitsChanged; + } + + @Override + public boolean mightContainString(String item) { + return mightContainBinary(Utils.getBytesFromUTF8String(item)); + } + + @Override + public boolean mightContainBinary(byte[] item) { + int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, 0); + int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); + + long bitSize = bits.bitSize(); + for (int i = 1; i <= numHashFunctions; i++) { + int combinedHash = h1 + (i * h2); + // Flip all the bits if it's negative (guaranteed positive number) + if (combinedHash < 0) { + combinedHash = ~combinedHash; + } + if (!bits.get(combinedHash % bitSize)) { + return false; + } + } + return true; + } + + @Override + public boolean putLong(long item) { + // Here we first hash the input long element into 2 int hash values, h1 and h2, then produce n + // hash values by `h1 + i * h2` with 1 <= i <= numHashFunctions. + // Note that `CountMinSketch` use a different strategy, it hash the input long element with + // every i to produce n hash values. + // TODO: the strategy of `CountMinSketch` looks more advanced, should we follow it here? + int h1 = Murmur3_x86_32.hashLong(item, 0); + int h2 = Murmur3_x86_32.hashLong(item, h1); + + long bitSize = bits.bitSize(); + boolean bitsChanged = false; + for (int i = 1; i <= numHashFunctions; i++) { + int combinedHash = h1 + (i * h2); + // Flip all the bits if it's negative (guaranteed positive number) + if (combinedHash < 0) { + combinedHash = ~combinedHash; + } + bitsChanged |= bits.set(combinedHash % bitSize); + } + return bitsChanged; + } + + @Override + public boolean mightContainLong(long item) { + int h1 = Murmur3_x86_32.hashLong(item, 0); + int h2 = Murmur3_x86_32.hashLong(item, h1); + + long bitSize = bits.bitSize(); + for (int i = 1; i <= numHashFunctions; i++) { + int combinedHash = h1 + (i * h2); + // Flip all the bits if it's negative (guaranteed positive number) + if (combinedHash < 0) { + combinedHash = ~combinedHash; + } + if (!bits.get(combinedHash % bitSize)) { + return false; + } + } + return true; + } + + @Override + public boolean mightContain(Object item) { + if (item instanceof String) { + return mightContainString((String) item); + } else if (item instanceof byte[]) { + return mightContainBinary((byte[]) item); + } else { + return mightContainLong(Utils.integralToLong(item)); + } + } + + @Override + public boolean isCompatible(BloomFilter other) { + if (other == null) { + return false; + } + + if (!(other instanceof BloomFilterImpl)) { + return false; + } + + BloomFilterImpl that = (BloomFilterImpl) other; + return this.bitSize() == that.bitSize() && this.numHashFunctions == that.numHashFunctions; + } + + @Override + public BloomFilter mergeInPlace(BloomFilter other) throws IncompatibleMergeException { + // Duplicates the logic of `isCompatible` here to provide better error message. + if (other == null) { + throw new IncompatibleMergeException("Cannot merge null bloom filter"); + } + + if (!(other instanceof BloomFilterImpl)) { + throw new IncompatibleMergeException( + "Cannot merge bloom filter of class " + other.getClass().getName() + ); + } + + BloomFilterImpl that = (BloomFilterImpl) other; + + if (this.bitSize() != that.bitSize()) { + throw new IncompatibleMergeException("Cannot merge bloom filters with different bit size"); + } + + if (this.numHashFunctions != that.numHashFunctions) { + throw new IncompatibleMergeException( + "Cannot merge bloom filters with different number of hash functions" + ); + } + + this.bits.putAll(that.bits); + return this; + } + + @Override + public void writeTo(OutputStream out) throws IOException { + DataOutputStream dos = new DataOutputStream(out); + + dos.writeInt(Version.V1.getVersionNumber()); + dos.writeInt(numHashFunctions); + bits.writeTo(dos); + } + + private void readFrom0(InputStream in) throws IOException { + DataInputStream dis = new DataInputStream(in); + + int version = dis.readInt(); + if (version != Version.V1.getVersionNumber()) { + throw new IOException("Unexpected Bloom filter version number (" + version + ")"); + } + + this.numHashFunctions = dis.readInt(); + this.bits = BitArray.readFrom(dis); + } + + public static BloomFilterImpl readFrom(InputStream in) throws IOException { + BloomFilterImpl filter = new BloomFilterImpl(); + filter.readFrom0(in); + return filter; + } + + private void writeObject(ObjectOutputStream out) throws IOException { + writeTo(out); + } + + private void readObject(ObjectInputStream in) throws IOException { + readFrom0(in); + } +} diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java new file mode 100644 index 000000000000..f7c22dddb8cc --- /dev/null +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java @@ -0,0 +1,223 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.sketch; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** + * A Count-min sketch is a probabilistic data structure used for cardinality estimation using + * sub-linear space. Currently, supported data types include: + *
    + *
  • {@link Byte}
  • + *
  • {@link Short}
  • + *
  • {@link Integer}
  • + *
  • {@link Long}
  • + *
  • {@link String}
  • + *
+ * A {@link CountMinSketch} is initialized with a random seed, and a pair of parameters: + *
    + *
  1. relative error (or {@code eps}), and + *
  2. confidence (or {@code delta}) + *
+ * Suppose you want to estimate the number of times an element {@code x} has appeared in a data + * stream so far. With probability {@code delta}, the estimate of this frequency is within the + * range {@code true frequency <= estimate <= true frequency + eps * N}, where {@code N} is the + * total count of items have appeared the data stream so far. + * + * Under the cover, a {@link CountMinSketch} is essentially a two-dimensional {@code long} array + * with depth {@code d} and width {@code w}, where + *
    + *
  • {@code d = ceil(2 / eps)}
  • + *
  • {@code w = ceil(-log(1 - confidence) / log(2))}
  • + *
+ * + * This implementation is largely based on the {@code CountMinSketch} class from stream-lib. + */ +public abstract class CountMinSketch { + + public enum Version { + /** + * {@code CountMinSketch} binary format version 1. All values written in big-endian order: + *
    + *
  • Version number, always 1 (32 bit)
  • + *
  • Total count of added items (64 bit)
  • + *
  • Depth (32 bit)
  • + *
  • Width (32 bit)
  • + *
  • Hash functions (depth * 64 bit)
  • + *
  • + * Count table + *
      + *
    • Row 0 (width * 64 bit)
    • + *
    • Row 1 (width * 64 bit)
    • + *
    • ...
    • + *
    • Row {@code depth - 1} (width * 64 bit)
    • + *
    + *
  • + *
+ */ + V1(1); + + private final int versionNumber; + + Version(int versionNumber) { + this.versionNumber = versionNumber; + } + + int getVersionNumber() { + return versionNumber; + } + } + + /** + * Returns the relative error (or {@code eps}) of this {@link CountMinSketch}. + */ + public abstract double relativeError(); + + /** + * Returns the confidence (or {@code delta}) of this {@link CountMinSketch}. + */ + public abstract double confidence(); + + /** + * Depth of this {@link CountMinSketch}. + */ + public abstract int depth(); + + /** + * Width of this {@link CountMinSketch}. + */ + public abstract int width(); + + /** + * Total count of items added to this {@link CountMinSketch} so far. + */ + public abstract long totalCount(); + + /** + * Increments {@code item}'s count by one. + */ + public abstract void add(Object item); + + /** + * Increments {@code item}'s count by {@code count}. + */ + public abstract void add(Object item, long count); + + /** + * Increments {@code item}'s count by one. + */ + public abstract void addLong(long item); + + /** + * Increments {@code item}'s count by {@code count}. + */ + public abstract void addLong(long item, long count); + + /** + * Increments {@code item}'s count by one. + */ + public abstract void addString(String item); + + /** + * Increments {@code item}'s count by {@code count}. + */ + public abstract void addString(String item, long count); + + /** + * Increments {@code item}'s count by one. + */ + public abstract void addBinary(byte[] item); + + /** + * Increments {@code item}'s count by {@code count}. + */ + public abstract void addBinary(byte[] item, long count); + + /** + * Returns the estimated frequency of {@code item}. + */ + public abstract long estimateCount(Object item); + + /** + * Merges another {@link CountMinSketch} with this one in place. + * + * Note that only Count-Min sketches with the same {@code depth}, {@code width}, and random seed + * can be merged. + * + * @exception IncompatibleMergeException if the {@code other} {@link CountMinSketch} has + * incompatible depth, width, relative-error, confidence, or random seed. + */ + public abstract CountMinSketch mergeInPlace(CountMinSketch other) + throws IncompatibleMergeException; + + /** + * Writes out this {@link CountMinSketch} to an output stream in binary format. It is the caller's + * responsibility to close the stream. + */ + public abstract void writeTo(OutputStream out) throws IOException; + + /** + * Serializes this {@link CountMinSketch} and returns the serialized form. + */ + public abstract byte[] toByteArray() throws IOException; + + /** + * Reads in a {@link CountMinSketch} from an input stream. It is the caller's responsibility to + * close the stream. + */ + public static CountMinSketch readFrom(InputStream in) throws IOException { + return CountMinSketchImpl.readFrom(in); + } + + /** + * Reads in a {@link CountMinSketch} from a byte array. + */ + public static CountMinSketch readFrom(byte[] bytes) throws IOException { + InputStream in = new ByteArrayInputStream(bytes); + CountMinSketch cms = readFrom(in); + in.close(); + return cms; + } + + /** + * Creates a {@link CountMinSketch} with given {@code depth}, {@code width}, and random + * {@code seed}. + * + * @param depth depth of the Count-min Sketch, must be positive + * @param width width of the Count-min Sketch, must be positive + * @param seed random seed + */ + public static CountMinSketch create(int depth, int width, int seed) { + return new CountMinSketchImpl(depth, width, seed); + } + + /** + * Creates a {@link CountMinSketch} with given relative error ({@code eps}), {@code confidence}, + * and random {@code seed}. + * + * @param eps relative error, must be positive + * @param confidence confidence, must be positive and less than 1.0 + * @param seed random seed + */ + public static CountMinSketch create(double eps, double confidence, int seed) { + return new CountMinSketchImpl(eps, confidence, seed); + } +} diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketchImpl.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketchImpl.java new file mode 100644 index 000000000000..fd1906d2e5ae --- /dev/null +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketchImpl.java @@ -0,0 +1,371 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.sketch; + +import java.io.*; +import java.util.Arrays; +import java.util.Random; + +class CountMinSketchImpl extends CountMinSketch implements Serializable { + private static final long PRIME_MODULUS = (1L << 31) - 1; + + private int depth; + private int width; + private long[][] table; + private long[] hashA; + private long totalCount; + private double eps; + private double confidence; + + private CountMinSketchImpl() {} + + CountMinSketchImpl(int depth, int width, int seed) { + if (depth <= 0 || width <= 0) { + throw new IllegalArgumentException("Depth and width must be both positive"); + } + + this.depth = depth; + this.width = width; + this.eps = 2.0 / width; + this.confidence = 1 - 1 / Math.pow(2, depth); + initTablesWith(depth, width, seed); + } + + CountMinSketchImpl(double eps, double confidence, int seed) { + if (eps <= 0D) { + throw new IllegalArgumentException("Relative error must be positive"); + } + + if (confidence <= 0D || confidence >= 1D) { + throw new IllegalArgumentException("Confidence must be within range (0.0, 1.0)"); + } + + // 2/w = eps ; w = 2/eps + // 1/2^depth <= 1-confidence ; depth >= -log2 (1-confidence) + this.eps = eps; + this.confidence = confidence; + this.width = (int) Math.ceil(2 / eps); + this.depth = (int) Math.ceil(-Math.log(1 - confidence) / Math.log(2)); + initTablesWith(depth, width, seed); + } + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + + if (other == null || !(other instanceof CountMinSketchImpl)) { + return false; + } + + CountMinSketchImpl that = (CountMinSketchImpl) other; + + return + this.depth == that.depth && + this.width == that.width && + this.totalCount == that.totalCount && + Arrays.equals(this.hashA, that.hashA) && + Arrays.deepEquals(this.table, that.table); + } + + @Override + public int hashCode() { + int hash = depth; + + hash = hash * 31 + width; + hash = hash * 31 + (int) (totalCount ^ (totalCount >>> 32)); + hash = hash * 31 + Arrays.hashCode(hashA); + hash = hash * 31 + Arrays.deepHashCode(table); + + return hash; + } + + private void initTablesWith(int depth, int width, int seed) { + this.table = new long[depth][width]; + this.hashA = new long[depth]; + Random r = new Random(seed); + // We're using a linear hash functions + // of the form (a*x+b) mod p. + // a,b are chosen independently for each hash function. + // However we can set b = 0 as all it does is shift the results + // without compromising their uniformity or independence with + // the other hashes. + for (int i = 0; i < depth; ++i) { + hashA[i] = r.nextInt(Integer.MAX_VALUE); + } + } + + @Override + public double relativeError() { + return eps; + } + + @Override + public double confidence() { + return confidence; + } + + @Override + public int depth() { + return depth; + } + + @Override + public int width() { + return width; + } + + @Override + public long totalCount() { + return totalCount; + } + + @Override + public void add(Object item) { + add(item, 1); + } + + @Override + public void add(Object item, long count) { + if (item instanceof String) { + addString((String) item, count); + } else if (item instanceof byte[]) { + addBinary((byte[]) item, count); + } else { + addLong(Utils.integralToLong(item), count); + } + } + + @Override + public void addString(String item) { + addString(item, 1); + } + + @Override + public void addString(String item, long count) { + addBinary(Utils.getBytesFromUTF8String(item), count); + } + + @Override + public void addLong(long item) { + addLong(item, 1); + } + + @Override + public void addLong(long item, long count) { + if (count < 0) { + throw new IllegalArgumentException("Negative increments not implemented"); + } + + for (int i = 0; i < depth; ++i) { + table[i][hash(item, i)] += count; + } + + totalCount += count; + } + + @Override + public void addBinary(byte[] item) { + addBinary(item, 1); + } + + @Override + public void addBinary(byte[] item, long count) { + if (count < 0) { + throw new IllegalArgumentException("Negative increments not implemented"); + } + + int[] buckets = getHashBuckets(item, depth, width); + + for (int i = 0; i < depth; ++i) { + table[i][buckets[i]] += count; + } + + totalCount += count; + } + + private int hash(long item, int count) { + long hash = hashA[count] * item; + // A super fast way of computing x mod 2^p-1 + // See http://www.cs.princeton.edu/courses/archive/fall09/cos521/Handouts/universalclasses.pdf + // page 149, right after Proposition 7. + hash += hash >> 32; + hash &= PRIME_MODULUS; + // Doing "%" after (int) conversion is ~2x faster than %'ing longs. + return ((int) hash) % width; + } + + private static int[] getHashBuckets(String key, int hashCount, int max) { + return getHashBuckets(Utils.getBytesFromUTF8String(key), hashCount, max); + } + + private static int[] getHashBuckets(byte[] b, int hashCount, int max) { + int[] result = new int[hashCount]; + int hash1 = Murmur3_x86_32.hashUnsafeBytes(b, Platform.BYTE_ARRAY_OFFSET, b.length, 0); + int hash2 = Murmur3_x86_32.hashUnsafeBytes(b, Platform.BYTE_ARRAY_OFFSET, b.length, hash1); + for (int i = 0; i < hashCount; i++) { + result[i] = Math.abs((hash1 + i * hash2) % max); + } + return result; + } + + @Override + public long estimateCount(Object item) { + if (item instanceof String) { + return estimateCountForStringItem((String) item); + } else if (item instanceof byte[]) { + return estimateCountForBinaryItem((byte[]) item); + } else { + return estimateCountForLongItem(Utils.integralToLong(item)); + } + } + + private long estimateCountForLongItem(long item) { + long res = Long.MAX_VALUE; + for (int i = 0; i < depth; ++i) { + res = Math.min(res, table[i][hash(item, i)]); + } + return res; + } + + private long estimateCountForStringItem(String item) { + long res = Long.MAX_VALUE; + int[] buckets = getHashBuckets(item, depth, width); + for (int i = 0; i < depth; ++i) { + res = Math.min(res, table[i][buckets[i]]); + } + return res; + } + + private long estimateCountForBinaryItem(byte[] item) { + long res = Long.MAX_VALUE; + int[] buckets = getHashBuckets(item, depth, width); + for (int i = 0; i < depth; ++i) { + res = Math.min(res, table[i][buckets[i]]); + } + return res; + } + + @Override + public CountMinSketch mergeInPlace(CountMinSketch other) throws IncompatibleMergeException { + if (other == null) { + throw new IncompatibleMergeException("Cannot merge null estimator"); + } + + if (!(other instanceof CountMinSketchImpl)) { + throw new IncompatibleMergeException( + "Cannot merge estimator of class " + other.getClass().getName() + ); + } + + CountMinSketchImpl that = (CountMinSketchImpl) other; + + if (this.depth != that.depth) { + throw new IncompatibleMergeException("Cannot merge estimators of different depth"); + } + + if (this.width != that.width) { + throw new IncompatibleMergeException("Cannot merge estimators of different width"); + } + + if (!Arrays.equals(this.hashA, that.hashA)) { + throw new IncompatibleMergeException("Cannot merge estimators of different seed"); + } + + for (int i = 0; i < this.table.length; ++i) { + for (int j = 0; j < this.table[i].length; ++j) { + this.table[i][j] = this.table[i][j] + that.table[i][j]; + } + } + + this.totalCount += that.totalCount; + + return this; + } + + @Override + public void writeTo(OutputStream out) throws IOException { + DataOutputStream dos = new DataOutputStream(out); + + dos.writeInt(Version.V1.getVersionNumber()); + + dos.writeLong(this.totalCount); + dos.writeInt(this.depth); + dos.writeInt(this.width); + + for (int i = 0; i < this.depth; ++i) { + dos.writeLong(this.hashA[i]); + } + + for (int i = 0; i < this.depth; ++i) { + for (int j = 0; j < this.width; ++j) { + dos.writeLong(table[i][j]); + } + } + } + + @Override + public byte[] toByteArray() throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + writeTo(out); + out.close(); + return out.toByteArray(); + } + + public static CountMinSketchImpl readFrom(InputStream in) throws IOException { + CountMinSketchImpl sketch = new CountMinSketchImpl(); + sketch.readFrom0(in); + return sketch; + } + + private void readFrom0(InputStream in) throws IOException { + DataInputStream dis = new DataInputStream(in); + + int version = dis.readInt(); + if (version != Version.V1.getVersionNumber()) { + throw new IOException("Unexpected Count-Min Sketch version number (" + version + ")"); + } + + this.totalCount = dis.readLong(); + this.depth = dis.readInt(); + this.width = dis.readInt(); + this.eps = 2.0 / width; + this.confidence = 1 - 1 / Math.pow(2, depth); + + this.hashA = new long[depth]; + for (int i = 0; i < depth; ++i) { + this.hashA[i] = dis.readLong(); + } + + this.table = new long[depth][width]; + for (int i = 0; i < depth; ++i) { + for (int j = 0; j < width; ++j) { + this.table[i][j] = dis.readLong(); + } + } + } + + private void writeObject(ObjectOutputStream out) throws IOException { + this.writeTo(out); + } + + private void readObject(ObjectInputStream in) throws IOException { + this.readFrom0(in); + } +} diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/IncompatibleMergeException.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/IncompatibleMergeException.java new file mode 100644 index 000000000000..64b567caa57c --- /dev/null +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/IncompatibleMergeException.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.sketch; + +public class IncompatibleMergeException extends Exception { + public IncompatibleMergeException(String message) { + super(message); + } +} diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/Murmur3_x86_32.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/Murmur3_x86_32.java new file mode 100644 index 000000000000..a61ce4fb7241 --- /dev/null +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/Murmur3_x86_32.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.sketch; + +/** + * 32-bit Murmur3 hasher. This is based on Guava's Murmur3_32HashFunction. + */ +// This class is duplicated from `org.apache.spark.unsafe.hash.Murmur3_x86_32` to make sure +// spark-sketch has no external dependencies. +final class Murmur3_x86_32 { + private static final int C1 = 0xcc9e2d51; + private static final int C2 = 0x1b873593; + + private final int seed; + + Murmur3_x86_32(int seed) { + this.seed = seed; + } + + @Override + public String toString() { + return "Murmur3_32(seed=" + seed + ")"; + } + + public int hashInt(int input) { + return hashInt(input, seed); + } + + public static int hashInt(int input, int seed) { + int k1 = mixK1(input); + int h1 = mixH1(seed, k1); + + return fmix(h1, 4); + } + + public int hashUnsafeWords(Object base, long offset, int lengthInBytes) { + return hashUnsafeWords(base, offset, lengthInBytes, seed); + } + + public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, int seed) { + // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. + assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)"; + int h1 = hashBytesByInt(base, offset, lengthInBytes, seed); + return fmix(h1, lengthInBytes); + } + + public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, int seed) { + assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; + int lengthAligned = lengthInBytes - lengthInBytes % 4; + int h1 = hashBytesByInt(base, offset, lengthAligned, seed); + for (int i = lengthAligned; i < lengthInBytes; i++) { + int halfWord = Platform.getByte(base, offset + i); + int k1 = mixK1(halfWord); + h1 = mixH1(h1, k1); + } + return fmix(h1, lengthInBytes); + } + + private static int hashBytesByInt(Object base, long offset, int lengthInBytes, int seed) { + assert (lengthInBytes % 4 == 0); + int h1 = seed; + for (int i = 0; i < lengthInBytes; i += 4) { + int halfWord = Platform.getInt(base, offset + i); + int k1 = mixK1(halfWord); + h1 = mixH1(h1, k1); + } + return h1; + } + + public int hashLong(long input) { + return hashLong(input, seed); + } + + public static int hashLong(long input, int seed) { + int low = (int) input; + int high = (int) (input >>> 32); + + int k1 = mixK1(low); + int h1 = mixH1(seed, k1); + + k1 = mixK1(high); + h1 = mixH1(h1, k1); + + return fmix(h1, 8); + } + + private static int mixK1(int k1) { + k1 *= C1; + k1 = Integer.rotateLeft(k1, 15); + k1 *= C2; + return k1; + } + + private static int mixH1(int h1, int k1) { + h1 ^= k1; + h1 = Integer.rotateLeft(h1, 13); + h1 = h1 * 5 + 0xe6546b64; + return h1; + } + + // Finalization mix - force all bits of a hash block to avalanche + private static int fmix(int h1, int length) { + h1 ^= length; + h1 ^= h1 >>> 16; + h1 *= 0x85ebca6b; + h1 ^= h1 >>> 13; + h1 *= 0xc2b2ae35; + h1 ^= h1 >>> 16; + return h1; + } +} diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/Platform.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/Platform.java new file mode 100644 index 000000000000..75d6a6beec40 --- /dev/null +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/Platform.java @@ -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.util.sketch; + +import java.lang.reflect.Field; + +import sun.misc.Unsafe; + +// This class is duplicated from `org.apache.spark.unsafe.Platform` to make sure spark-sketch has no +// external dependencies. +final class Platform { + + private static final Unsafe _UNSAFE; + + public static final int BYTE_ARRAY_OFFSET; + + public static final int INT_ARRAY_OFFSET; + + public static final int LONG_ARRAY_OFFSET; + + public static final int DOUBLE_ARRAY_OFFSET; + + public static int getInt(Object object, long offset) { + return _UNSAFE.getInt(object, offset); + } + + public static void putInt(Object object, long offset, int value) { + _UNSAFE.putInt(object, offset, value); + } + + public static boolean getBoolean(Object object, long offset) { + return _UNSAFE.getBoolean(object, offset); + } + + public static void putBoolean(Object object, long offset, boolean value) { + _UNSAFE.putBoolean(object, offset, value); + } + + public static byte getByte(Object object, long offset) { + return _UNSAFE.getByte(object, offset); + } + + public static void putByte(Object object, long offset, byte value) { + _UNSAFE.putByte(object, offset, value); + } + + public static short getShort(Object object, long offset) { + return _UNSAFE.getShort(object, offset); + } + + public static void putShort(Object object, long offset, short value) { + _UNSAFE.putShort(object, offset, value); + } + + public static long getLong(Object object, long offset) { + return _UNSAFE.getLong(object, offset); + } + + public static void putLong(Object object, long offset, long value) { + _UNSAFE.putLong(object, offset, value); + } + + public static float getFloat(Object object, long offset) { + return _UNSAFE.getFloat(object, offset); + } + + public static void putFloat(Object object, long offset, float value) { + _UNSAFE.putFloat(object, offset, value); + } + + public static double getDouble(Object object, long offset) { + return _UNSAFE.getDouble(object, offset); + } + + public static void putDouble(Object object, long offset, double value) { + _UNSAFE.putDouble(object, offset, value); + } + + public static Object getObjectVolatile(Object object, long offset) { + return _UNSAFE.getObjectVolatile(object, offset); + } + + public static void putObjectVolatile(Object object, long offset, Object value) { + _UNSAFE.putObjectVolatile(object, offset, value); + } + + public static long allocateMemory(long size) { + return _UNSAFE.allocateMemory(size); + } + + public static void freeMemory(long address) { + _UNSAFE.freeMemory(address); + } + + public static void copyMemory( + Object src, long srcOffset, Object dst, long dstOffset, long length) { + // Check if dstOffset is before or after srcOffset to determine if we should copy + // forward or backwards. This is necessary in case src and dst overlap. + if (dstOffset < srcOffset) { + while (length > 0) { + long size = Math.min(length, UNSAFE_COPY_THRESHOLD); + _UNSAFE.copyMemory(src, srcOffset, dst, dstOffset, size); + length -= size; + srcOffset += size; + dstOffset += size; + } + } else { + srcOffset += length; + dstOffset += length; + while (length > 0) { + long size = Math.min(length, UNSAFE_COPY_THRESHOLD); + srcOffset -= size; + dstOffset -= size; + _UNSAFE.copyMemory(src, srcOffset, dst, dstOffset, size); + length -= size; + } + + } + } + + /** + * Raises an exception bypassing compiler checks for checked exceptions. + */ + public static void throwException(Throwable t) { + _UNSAFE.throwException(t); + } + + /** + * Limits the number of bytes to copy per {@link Unsafe#copyMemory(long, long, long)} to + * allow safepoint polling during a large copy. + */ + private static final long UNSAFE_COPY_THRESHOLD = 1024L * 1024L; + + static { + sun.misc.Unsafe unsafe; + try { + Field unsafeField = Unsafe.class.getDeclaredField("theUnsafe"); + unsafeField.setAccessible(true); + unsafe = (sun.misc.Unsafe) unsafeField.get(null); + } catch (Throwable cause) { + unsafe = null; + } + _UNSAFE = unsafe; + + if (_UNSAFE != null) { + BYTE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(byte[].class); + INT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(int[].class); + LONG_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(long[].class); + DOUBLE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(double[].class); + } else { + BYTE_ARRAY_OFFSET = 0; + INT_ARRAY_OFFSET = 0; + LONG_ARRAY_OFFSET = 0; + DOUBLE_ARRAY_OFFSET = 0; + } + } +} diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/Utils.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/Utils.java new file mode 100644 index 000000000000..81461f03000a --- /dev/null +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/Utils.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.sketch; + +import java.nio.charset.StandardCharsets; + +class Utils { + public static byte[] getBytesFromUTF8String(String str) { + return str.getBytes(StandardCharsets.UTF_8); + } + + public static long integralToLong(Object i) { + long longValue; + + if (i instanceof Long) { + longValue = (Long) i; + } else if (i instanceof Integer) { + longValue = ((Integer) i).longValue(); + } else if (i instanceof Short) { + longValue = ((Short) i).longValue(); + } else if (i instanceof Byte) { + longValue = ((Byte) i).longValue(); + } else { + throw new IllegalArgumentException("Unsupported data type " + i.getClass().getName()); + } + + return longValue; + } +} diff --git a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BitArraySuite.scala b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BitArraySuite.scala new file mode 100644 index 000000000000..ff728f0ebcb8 --- /dev/null +++ b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BitArraySuite.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.util.sketch + +import scala.util.Random + +import org.scalatest.FunSuite // scalastyle:ignore funsuite + +class BitArraySuite extends FunSuite { // scalastyle:ignore funsuite + + test("error case when create BitArray") { + intercept[IllegalArgumentException](new BitArray(0)) + intercept[IllegalArgumentException](new BitArray(64L * Integer.MAX_VALUE + 1)) + } + + test("bitSize") { + assert(new BitArray(64).bitSize() == 64) + // BitArray is word-aligned, so 65~128 bits need 2 long to store, which is 128 bits. + assert(new BitArray(65).bitSize() == 128) + assert(new BitArray(127).bitSize() == 128) + assert(new BitArray(128).bitSize() == 128) + } + + test("set") { + val bitArray = new BitArray(64) + assert(bitArray.set(1)) + // Only returns true if the bit changed. + assert(!bitArray.set(1)) + assert(bitArray.set(2)) + } + + test("normal operation") { + // use a fixed seed to make the test predictable. + val r = new Random(37) + + val bitArray = new BitArray(320) + val indexes = (1 to 100).map(_ => r.nextInt(320).toLong).distinct + + indexes.foreach(bitArray.set) + indexes.foreach(i => assert(bitArray.get(i))) + assert(bitArray.cardinality() == indexes.length) + } + + test("merge") { + // use a fixed seed to make the test predictable. + val r = new Random(37) + + val bitArray1 = new BitArray(64 * 6) + val bitArray2 = new BitArray(64 * 6) + + val indexes1 = (1 to 100).map(_ => r.nextInt(64 * 6).toLong).distinct + val indexes2 = (1 to 100).map(_ => r.nextInt(64 * 6).toLong).distinct + + indexes1.foreach(bitArray1.set) + indexes2.foreach(bitArray2.set) + + bitArray1.putAll(bitArray2) + indexes1.foreach(i => assert(bitArray1.get(i))) + indexes2.foreach(i => assert(bitArray1.get(i))) + assert(bitArray1.cardinality() == (indexes1 ++ indexes2).distinct.length) + } +} diff --git a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala new file mode 100644 index 000000000000..a0408d2da4df --- /dev/null +++ b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.sketch + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} + +import scala.reflect.ClassTag +import scala.util.Random + +import org.scalatest.FunSuite // scalastyle:ignore funsuite + +class BloomFilterSuite extends FunSuite { // scalastyle:ignore funsuite + private final val EPSILON = 0.01 + + // Serializes and deserializes a given `BloomFilter`, then checks whether the deserialized + // version is equivalent to the original one. + private def checkSerDe(filter: BloomFilter): Unit = { + val out = new ByteArrayOutputStream() + filter.writeTo(out) + out.close() + + val in = new ByteArrayInputStream(out.toByteArray) + val deserialized = BloomFilter.readFrom(in) + in.close() + + assert(filter == deserialized) + } + + def testAccuracy[T: ClassTag](typeName: String, numItems: Int)(itemGen: Random => T): Unit = { + test(s"accuracy - $typeName") { + // use a fixed seed to make the test predictable. + val r = new Random(37) + val fpp = 0.05 + val numInsertion = numItems / 10 + + val allItems = Array.fill(numItems)(itemGen(r)) + + val filter = BloomFilter.create(numInsertion, fpp) + + // insert first `numInsertion` items. + allItems.take(numInsertion).foreach(filter.put) + + // false negative is not allowed. + assert(allItems.take(numInsertion).forall(filter.mightContain)) + + // The number of inserted items doesn't exceed `expectedNumItems`, so the `expectedFpp` + // should not be significantly higher than the one we passed in to create this bloom filter. + assert(filter.expectedFpp() - fpp < EPSILON) + + val errorCount = allItems.drop(numInsertion).count(filter.mightContain) + + // Also check the actual fpp is not significantly higher than we expected. + val actualFpp = errorCount.toDouble / (numItems - numInsertion) + assert(actualFpp - fpp < EPSILON) + + checkSerDe(filter) + } + } + + def testMergeInPlace[T: ClassTag](typeName: String, numItems: Int)(itemGen: Random => T): Unit = { + test(s"mergeInPlace - $typeName") { + // use a fixed seed to make the test predictable. + val r = new Random(37) + + val items1 = Array.fill(numItems / 2)(itemGen(r)) + val items2 = Array.fill(numItems / 2)(itemGen(r)) + + val filter1 = BloomFilter.create(numItems) + items1.foreach(filter1.put) + + val filter2 = BloomFilter.create(numItems) + items2.foreach(filter2.put) + + filter1.mergeInPlace(filter2) + + // After merge, `filter1` has `numItems` items which doesn't exceed `expectedNumItems`, so the + // `expectedFpp` should not be significantly higher than the default one. + assert(filter1.expectedFpp() - BloomFilter.DEFAULT_FPP < EPSILON) + + items1.foreach(i => assert(filter1.mightContain(i))) + items2.foreach(i => assert(filter1.mightContain(i))) + + checkSerDe(filter1) + } + } + + def testItemType[T: ClassTag](typeName: String, numItems: Int)(itemGen: Random => T): Unit = { + testAccuracy[T](typeName, numItems)(itemGen) + testMergeInPlace[T](typeName, numItems)(itemGen) + } + + testItemType[Byte]("Byte", 160) { _.nextInt().toByte } + + testItemType[Short]("Short", 1000) { _.nextInt().toShort } + + testItemType[Int]("Int", 100000) { _.nextInt() } + + testItemType[Long]("Long", 100000) { _.nextLong() } + + testItemType[String]("String", 100000) { r => r.nextString(r.nextInt(512)) } + + test("incompatible merge") { + intercept[IncompatibleMergeException] { + BloomFilter.create(1000).mergeInPlace(null) + } + + intercept[IncompatibleMergeException] { + val filter1 = BloomFilter.create(1000, 6400) + val filter2 = BloomFilter.create(1000, 3200) + filter1.mergeInPlace(filter2) + } + + intercept[IncompatibleMergeException] { + val filter1 = BloomFilter.create(1000, 6400) + val filter2 = BloomFilter.create(2000, 6400) + filter1.mergeInPlace(filter2) + } + } +} diff --git a/common/sketch/src/test/scala/org/apache/spark/util/sketch/CountMinSketchSuite.scala b/common/sketch/src/test/scala/org/apache/spark/util/sketch/CountMinSketchSuite.scala new file mode 100644 index 000000000000..174eb01986c4 --- /dev/null +++ b/common/sketch/src/test/scala/org/apache/spark/util/sketch/CountMinSketchSuite.scala @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.sketch + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} + +import scala.reflect.ClassTag +import scala.util.Random + +import org.scalatest.FunSuite // scalastyle:ignore funsuite + +class CountMinSketchSuite extends FunSuite { // scalastyle:ignore funsuite + private val epsOfTotalCount = 0.01 + + private val confidence = 0.9 + + private val seed = 42 + + // Serializes and deserializes a given `CountMinSketch`, then checks whether the deserialized + // version is equivalent to the original one. + private def checkSerDe(sketch: CountMinSketch): Unit = { + val out = new ByteArrayOutputStream() + sketch.writeTo(out) + + val in = new ByteArrayInputStream(out.toByteArray) + val deserialized = CountMinSketch.readFrom(in) + + assert(sketch === deserialized) + } + + def testAccuracy[T: ClassTag](typeName: String)(itemGenerator: Random => T): Unit = { + test(s"accuracy - $typeName") { + // Uses fixed seed to ensure reproducible test execution + val r = new Random(31) + + val numAllItems = 1000000 + val allItems = Array.fill(numAllItems)(itemGenerator(r)) + + val numSamples = numAllItems / 10 + val sampledItemIndices = Array.fill(numSamples)(r.nextInt(numAllItems)) + + val exactFreq = { + val sampledItems = sampledItemIndices.map(allItems) + sampledItems.groupBy(identity).mapValues(_.length.toLong) + } + + val sketch = CountMinSketch.create(epsOfTotalCount, confidence, seed) + checkSerDe(sketch) + + sampledItemIndices.foreach(i => sketch.add(allItems(i))) + checkSerDe(sketch) + + val probCorrect = { + val numErrors = allItems.map { item => + val count = exactFreq.getOrElse(item, 0L) + val ratio = (sketch.estimateCount(item) - count).toDouble / numAllItems + if (ratio > epsOfTotalCount) 1 else 0 + }.sum + + 1.0 - (numErrors.toDouble / numAllItems) + } + + assert( + probCorrect > confidence, + s"Confidence not reached: required $confidence, reached $probCorrect" + ) + } + } + + def testMergeInPlace[T: ClassTag](typeName: String)(itemGenerator: Random => T): Unit = { + test(s"mergeInPlace - $typeName") { + // Uses fixed seed to ensure reproducible test execution + val r = new Random(31) + + val numToMerge = 5 + val numItemsPerSketch = 100000 + val perSketchItems = Array.fill(numToMerge, numItemsPerSketch) { itemGenerator(r) } + + val sketches = perSketchItems.map { items => + val sketch = CountMinSketch.create(epsOfTotalCount, confidence, seed) + checkSerDe(sketch) + + items.foreach(sketch.add) + checkSerDe(sketch) + + sketch + } + + val mergedSketch = sketches.reduce(_ mergeInPlace _) + checkSerDe(mergedSketch) + + val expectedSketch = CountMinSketch.create(epsOfTotalCount, confidence, seed) + perSketchItems.foreach(_.foreach(expectedSketch.add)) + + perSketchItems.foreach { + _.foreach { item => + assert(mergedSketch.estimateCount(item) === expectedSketch.estimateCount(item)) + } + } + } + } + + def testItemType[T: ClassTag](typeName: String)(itemGenerator: Random => T): Unit = { + testAccuracy[T](typeName)(itemGenerator) + testMergeInPlace[T](typeName)(itemGenerator) + } + + testItemType[Byte]("Byte") { _.nextInt().toByte } + + testItemType[Short]("Short") { _.nextInt().toShort } + + testItemType[Int]("Int") { _.nextInt() } + + testItemType[Long]("Long") { _.nextLong() } + + testItemType[String]("String") { r => r.nextString(r.nextInt(20)) } + + testItemType[Array[Byte]]("Byte array") { r => r.nextString(r.nextInt(60)).getBytes } + + test("incompatible merge") { + intercept[IncompatibleMergeException] { + CountMinSketch.create(10, 10, 1).mergeInPlace(null) + } + + intercept[IncompatibleMergeException] { + val sketch1 = CountMinSketch.create(10, 20, 1) + val sketch2 = CountMinSketch.create(10, 20, 2) + sketch1.mergeInPlace(sketch2) + } + + intercept[IncompatibleMergeException] { + val sketch1 = CountMinSketch.create(10, 10, 1) + val sketch2 = CountMinSketch.create(10, 20, 2) + sketch1.mergeInPlace(sketch2) + } + } +} diff --git a/common/tags/README.md b/common/tags/README.md new file mode 100644 index 000000000000..01e5126945eb --- /dev/null +++ b/common/tags/README.md @@ -0,0 +1 @@ +This module includes annotations in Java that are used to annotate test suites. diff --git a/common/tags/pom.xml b/common/tags/pom.xml new file mode 100644 index 000000000000..f7e586ee777e --- /dev/null +++ b/common/tags/pom.xml @@ -0,0 +1,49 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-tags_2.11 + jar + Spark Project Tags + http://spark.apache.org/ + + tags + + + + + org.scala-lang + scala-library + ${scala.version} + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/core/src/main/java/org/apache/spark/annotation/AlphaComponent.java b/common/tags/src/main/java/org/apache/spark/annotation/AlphaComponent.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/AlphaComponent.java rename to common/tags/src/main/java/org/apache/spark/annotation/AlphaComponent.java diff --git a/core/src/main/java/org/apache/spark/annotation/DeveloperApi.java b/common/tags/src/main/java/org/apache/spark/annotation/DeveloperApi.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/DeveloperApi.java rename to common/tags/src/main/java/org/apache/spark/annotation/DeveloperApi.java diff --git a/core/src/main/java/org/apache/spark/annotation/Experimental.java b/common/tags/src/main/java/org/apache/spark/annotation/Experimental.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/Experimental.java rename to common/tags/src/main/java/org/apache/spark/annotation/Experimental.java diff --git a/common/tags/src/main/java/org/apache/spark/annotation/InterfaceStability.java b/common/tags/src/main/java/org/apache/spark/annotation/InterfaceStability.java new file mode 100644 index 000000000000..323098f69c6e --- /dev/null +++ b/common/tags/src/main/java/org/apache/spark/annotation/InterfaceStability.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.annotation; + +import java.lang.annotation.Documented; + +/** + * Annotation to inform users of how much to rely on a particular package, + * class or method not changing over time. + */ +public class InterfaceStability { + + /** + * Stable APIs that retain source and binary compatibility within a major release. + * These interfaces can change from one major release to another major release + * (e.g. from 1.0 to 2.0). + */ + @Documented + public @interface Stable {}; + + /** + * APIs that are meant to evolve towards becoming stable APIs, but are not stable APIs yet. + * Evolving interfaces can change from one feature release to another release (i.e. 2.1 to 2.2). + */ + @Documented + public @interface Evolving {}; + + /** + * Unstable APIs, with no guarantee on stability. + * Classes that are unannotated are considered Unstable. + */ + @Documented + public @interface Unstable {}; +} diff --git a/core/src/main/java/org/apache/spark/annotation/Private.java b/common/tags/src/main/java/org/apache/spark/annotation/Private.java similarity index 100% rename from core/src/main/java/org/apache/spark/annotation/Private.java rename to common/tags/src/main/java/org/apache/spark/annotation/Private.java diff --git a/core/src/main/scala/org/apache/spark/annotation/Since.scala b/common/tags/src/main/scala/org/apache/spark/annotation/Since.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/annotation/Since.scala rename to common/tags/src/main/scala/org/apache/spark/annotation/Since.scala diff --git a/common/tags/src/main/scala/org/apache/spark/annotation/package-info.java b/common/tags/src/main/scala/org/apache/spark/annotation/package-info.java new file mode 100644 index 000000000000..9efdccf6b040 --- /dev/null +++ b/common/tags/src/main/scala/org/apache/spark/annotation/package-info.java @@ -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. + */ + +/** + * Spark annotations to mark an API experimental or intended only for advanced usages by developers. + * This package consist of these annotations, which are used project wide and are reflected in + * Scala and Java docs. + */ +package org.apache.spark.annotation; diff --git a/core/src/main/scala/org/apache/spark/annotation/package.scala b/common/tags/src/main/scala/org/apache/spark/annotation/package.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/annotation/package.scala rename to common/tags/src/main/scala/org/apache/spark/annotation/package.scala diff --git a/common/tags/src/test/java/org/apache/spark/tags/DockerTest.java b/common/tags/src/test/java/org/apache/spark/tags/DockerTest.java new file mode 100644 index 000000000000..0fecf3b8f979 --- /dev/null +++ b/common/tags/src/test/java/org/apache/spark/tags/DockerTest.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.tags; + +import java.lang.annotation.*; +import org.scalatest.TagAnnotation; + +@TagAnnotation +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.METHOD, ElementType.TYPE}) +public @interface DockerTest { } diff --git a/tags/src/main/java/org/apache/spark/tags/ExtendedHiveTest.java b/common/tags/src/test/java/org/apache/spark/tags/ExtendedHiveTest.java similarity index 99% rename from tags/src/main/java/org/apache/spark/tags/ExtendedHiveTest.java rename to common/tags/src/test/java/org/apache/spark/tags/ExtendedHiveTest.java index 1b0c416b0fe4..83279e5e93c0 100644 --- a/tags/src/main/java/org/apache/spark/tags/ExtendedHiveTest.java +++ b/common/tags/src/test/java/org/apache/spark/tags/ExtendedHiveTest.java @@ -18,6 +18,7 @@ package org.apache.spark.tags; import java.lang.annotation.*; + import org.scalatest.TagAnnotation; @TagAnnotation diff --git a/tags/src/main/java/org/apache/spark/tags/ExtendedYarnTest.java b/common/tags/src/test/java/org/apache/spark/tags/ExtendedYarnTest.java similarity index 99% rename from tags/src/main/java/org/apache/spark/tags/ExtendedYarnTest.java rename to common/tags/src/test/java/org/apache/spark/tags/ExtendedYarnTest.java index 2a631bfc88cf..108300168e17 100644 --- a/tags/src/main/java/org/apache/spark/tags/ExtendedYarnTest.java +++ b/common/tags/src/test/java/org/apache/spark/tags/ExtendedYarnTest.java @@ -18,6 +18,7 @@ package org.apache.spark.tags; import java.lang.annotation.*; + import org.scalatest.TagAnnotation; @TagAnnotation diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml new file mode 100644 index 000000000000..a3772a262008 --- /dev/null +++ b/common/unsafe/pom.xml @@ -0,0 +1,109 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-unsafe_2.11 + jar + Spark Project Unsafe + http://spark.apache.org/ + + unsafe + + + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + com.twitter + chill_${scala.binary.version} + + + + + com.google.code.findbugs + jsr305 + + + com.google.guava + guava + + + + + org.slf4j + slf4j-api + provided + + + + + org.mockito + mockito-core + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.apache.commons + commons-lang3 + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + net.alchim31.maven + scala-maven-plugin + + + + -XDignore.symbol.file + + + + + + diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java new file mode 100644 index 000000000000..73577437ac50 --- /dev/null +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions; + +import org.apache.spark.unsafe.Platform; + +/** + * Simulates Hive's hashing function from Hive v1.2.1 + * org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils#hashcode() + */ +public class HiveHasher { + + @Override + public String toString() { + return HiveHasher.class.getSimpleName(); + } + + public static int hashInt(int input) { + return input; + } + + public static int hashLong(long input) { + return (int) ((input >>> 32) ^ input); + } + + public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes) { + assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; + int result = 0; + for (int i = 0; i < lengthInBytes; i++) { + result = (result * 31) + (int) Platform.getByte(base, offset + i); + } + return result; + } +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/KVIterator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/KVIterator.java similarity index 100% rename from unsafe/src/main/java/org/apache/spark/unsafe/KVIterator.java rename to common/unsafe/src/main/java/org/apache/spark/unsafe/KVIterator.java diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java new file mode 100644 index 000000000000..aca6fca00c48 --- /dev/null +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.unsafe; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.nio.ByteBuffer; + +import sun.misc.Cleaner; +import sun.misc.Unsafe; + +public final class Platform { + + private static final Unsafe _UNSAFE; + + public static final int BOOLEAN_ARRAY_OFFSET; + + public static final int BYTE_ARRAY_OFFSET; + + public static final int SHORT_ARRAY_OFFSET; + + public static final int INT_ARRAY_OFFSET; + + public static final int LONG_ARRAY_OFFSET; + + public static final int FLOAT_ARRAY_OFFSET; + + public static final int DOUBLE_ARRAY_OFFSET; + + private static final boolean unaligned; + static { + boolean _unaligned; + String arch = System.getProperty("os.arch", ""); + if (arch.equals("ppc64le") || arch.equals("ppc64")) { + // Since java.nio.Bits.unaligned() doesn't return true on ppc (See JDK-8165231), but + // ppc64 and ppc64le support it + _unaligned = true; + } else { + try { + Class bitsClass = + Class.forName("java.nio.Bits", false, ClassLoader.getSystemClassLoader()); + Method unalignedMethod = bitsClass.getDeclaredMethod("unaligned"); + unalignedMethod.setAccessible(true); + _unaligned = Boolean.TRUE.equals(unalignedMethod.invoke(null)); + } catch (Throwable t) { + // We at least know x86 and x64 support unaligned access. + //noinspection DynamicRegexReplaceableByCompiledPattern + _unaligned = arch.matches("^(i[3-6]86|x86(_64)?|x64|amd64|aarch64)$"); + } + } + unaligned = _unaligned; + } + + /** + * @return true when running JVM is having sun's Unsafe package available in it and underlying + * system having unaligned-access capability. + */ + public static boolean unaligned() { + return unaligned; + } + + public static int getInt(Object object, long offset) { + return _UNSAFE.getInt(object, offset); + } + + public static void putInt(Object object, long offset, int value) { + _UNSAFE.putInt(object, offset, value); + } + + public static boolean getBoolean(Object object, long offset) { + return _UNSAFE.getBoolean(object, offset); + } + + public static void putBoolean(Object object, long offset, boolean value) { + _UNSAFE.putBoolean(object, offset, value); + } + + public static byte getByte(Object object, long offset) { + return _UNSAFE.getByte(object, offset); + } + + public static void putByte(Object object, long offset, byte value) { + _UNSAFE.putByte(object, offset, value); + } + + public static short getShort(Object object, long offset) { + return _UNSAFE.getShort(object, offset); + } + + public static void putShort(Object object, long offset, short value) { + _UNSAFE.putShort(object, offset, value); + } + + public static long getLong(Object object, long offset) { + return _UNSAFE.getLong(object, offset); + } + + public static void putLong(Object object, long offset, long value) { + _UNSAFE.putLong(object, offset, value); + } + + public static float getFloat(Object object, long offset) { + return _UNSAFE.getFloat(object, offset); + } + + public static void putFloat(Object object, long offset, float value) { + _UNSAFE.putFloat(object, offset, value); + } + + public static double getDouble(Object object, long offset) { + return _UNSAFE.getDouble(object, offset); + } + + public static void putDouble(Object object, long offset, double value) { + _UNSAFE.putDouble(object, offset, value); + } + + public static Object getObjectVolatile(Object object, long offset) { + return _UNSAFE.getObjectVolatile(object, offset); + } + + public static void putObjectVolatile(Object object, long offset, Object value) { + _UNSAFE.putObjectVolatile(object, offset, value); + } + + public static long allocateMemory(long size) { + return _UNSAFE.allocateMemory(size); + } + + public static void freeMemory(long address) { + _UNSAFE.freeMemory(address); + } + + public static long reallocateMemory(long address, long oldSize, long newSize) { + long newMemory = _UNSAFE.allocateMemory(newSize); + copyMemory(null, address, null, newMemory, oldSize); + freeMemory(address); + return newMemory; + } + + /** + * Uses internal JDK APIs to allocate a DirectByteBuffer while ignoring the JVM's + * MaxDirectMemorySize limit (the default limit is too low and we do not want to require users + * to increase it). + */ + @SuppressWarnings("unchecked") + public static ByteBuffer allocateDirectBuffer(int size) { + try { + Class cls = Class.forName("java.nio.DirectByteBuffer"); + Constructor constructor = cls.getDeclaredConstructor(Long.TYPE, Integer.TYPE); + constructor.setAccessible(true); + Field cleanerField = cls.getDeclaredField("cleaner"); + cleanerField.setAccessible(true); + long memory = allocateMemory(size); + ByteBuffer buffer = (ByteBuffer) constructor.newInstance(memory, size); + Cleaner cleaner = Cleaner.create(buffer, () -> freeMemory(memory)); + cleanerField.set(buffer, cleaner); + return buffer; + } catch (Exception e) { + throwException(e); + } + throw new IllegalStateException("unreachable"); + } + + public static void setMemory(Object object, long offset, long size, byte value) { + _UNSAFE.setMemory(object, offset, size, value); + } + + public static void setMemory(long address, byte value, long size) { + _UNSAFE.setMemory(address, size, value); + } + + public static void copyMemory( + Object src, long srcOffset, Object dst, long dstOffset, long length) { + // Check if dstOffset is before or after srcOffset to determine if we should copy + // forward or backwards. This is necessary in case src and dst overlap. + if (dstOffset < srcOffset) { + while (length > 0) { + long size = Math.min(length, UNSAFE_COPY_THRESHOLD); + _UNSAFE.copyMemory(src, srcOffset, dst, dstOffset, size); + length -= size; + srcOffset += size; + dstOffset += size; + } + } else { + srcOffset += length; + dstOffset += length; + while (length > 0) { + long size = Math.min(length, UNSAFE_COPY_THRESHOLD); + srcOffset -= size; + dstOffset -= size; + _UNSAFE.copyMemory(src, srcOffset, dst, dstOffset, size); + length -= size; + } + + } + } + + /** + * Raises an exception bypassing compiler checks for checked exceptions. + */ + public static void throwException(Throwable t) { + _UNSAFE.throwException(t); + } + + /** + * Limits the number of bytes to copy per {@link Unsafe#copyMemory(long, long, long)} to + * allow safepoint polling during a large copy. + */ + private static final long UNSAFE_COPY_THRESHOLD = 1024L * 1024L; + + static { + sun.misc.Unsafe unsafe; + try { + Field unsafeField = Unsafe.class.getDeclaredField("theUnsafe"); + unsafeField.setAccessible(true); + unsafe = (sun.misc.Unsafe) unsafeField.get(null); + } catch (Throwable cause) { + unsafe = null; + } + _UNSAFE = unsafe; + + if (_UNSAFE != null) { + BOOLEAN_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(boolean[].class); + BYTE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(byte[].class); + SHORT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(short[].class); + INT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(int[].class); + LONG_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(long[].class); + FLOAT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(float[].class); + DOUBLE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(double[].class); + } else { + BOOLEAN_ARRAY_OFFSET = 0; + BYTE_ARRAY_OFFSET = 0; + SHORT_ARRAY_OFFSET = 0; + INT_ARRAY_OFFSET = 0; + LONG_ARRAY_OFFSET = 0; + FLOAT_ARRAY_OFFSET = 0; + DOUBLE_ARRAY_OFFSET = 0; + } + } +} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/UnsafeAlignedOffset.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/UnsafeAlignedOffset.java new file mode 100644 index 000000000000..be62e40412f8 --- /dev/null +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/UnsafeAlignedOffset.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.unsafe; + +/** + * Class to make changes to record length offsets uniform through out + * various areas of Apache Spark core and unsafe. The SPARC platform + * requires this because using a 4 byte Int for record lengths causes + * the entire record of 8 byte Items to become misaligned by 4 bytes. + * Using a 8 byte long for record length keeps things 8 byte aligned. + */ +public class UnsafeAlignedOffset { + + private static final int UAO_SIZE = Platform.unaligned() ? 4 : 8; + + public static int getUaoSize() { + return UAO_SIZE; + } + + public static int getSize(Object object, long offset) { + switch (UAO_SIZE) { + case 4: + return Platform.getInt(object, offset); + case 8: + return (int)Platform.getLong(object, offset); + default: + throw new AssertionError("Illegal UAO_SIZE"); + } + } + + public static void putSize(Object object, long offset, int value) { + switch (UAO_SIZE) { + case 4: + Platform.putInt(object, offset, value); + break; + case 8: + Platform.putLong(object, offset, value); + break; + default: + throw new AssertionError("Illegal UAO_SIZE"); + } + } +} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java new file mode 100644 index 000000000000..9c551ab19e9a --- /dev/null +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.unsafe.array; + +import org.apache.spark.unsafe.Platform; + +public class ByteArrayMethods { + + private ByteArrayMethods() { + // Private constructor, since this class only contains static methods. + } + + /** Returns the next number greater or equal num that is power of 2. */ + public static long nextPowerOf2(long num) { + final long highBit = Long.highestOneBit(num); + return (highBit == num) ? num : highBit << 1; + } + + public static int roundNumberOfBytesToNearestWord(int numBytes) { + int remainder = numBytes & 0x07; // This is equivalent to `numBytes % 8` + if (remainder == 0) { + return numBytes; + } else { + return numBytes + (8 - remainder); + } + } + + private static final boolean unaligned = Platform.unaligned(); + /** + * Optimized byte array equality check for byte arrays. + * @return true if the arrays are equal, false otherwise + */ + public static boolean arrayEquals( + Object leftBase, long leftOffset, Object rightBase, long rightOffset, final long length) { + int i = 0; + + // check if stars align and we can get both offsets to be aligned + if ((leftOffset % 8) == (rightOffset % 8)) { + while ((leftOffset + i) % 8 != 0 && i < length) { + if (Platform.getByte(leftBase, leftOffset + i) != + Platform.getByte(rightBase, rightOffset + i)) { + return false; + } + i += 1; + } + } + // for architectures that suport unaligned accesses, chew it up 8 bytes at a time + if (unaligned || (((leftOffset + i) % 8 == 0) && ((rightOffset + i) % 8 == 0))) { + while (i <= length - 8) { + if (Platform.getLong(leftBase, leftOffset + i) != + Platform.getLong(rightBase, rightOffset + i)) { + return false; + } + i += 8; + } + } + // this will finish off the unaligned comparisons, or do the entire aligned + // comparison whichever is needed. + while (i < length) { + if (Platform.getByte(leftBase, leftOffset + i) != + Platform.getByte(rightBase, rightOffset + i)) { + return false; + } + i += 1; + } + return true; + } +} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java similarity index 87% rename from unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java rename to common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java index 74105050e419..2cd39bd60c2a 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java @@ -39,8 +39,7 @@ public final class LongArray { private final long length; public LongArray(MemoryBlock memory) { - assert memory.size() % WIDTH == 0 : "Memory not aligned (" + memory.size() + ")"; - assert memory.size() < (long) Integer.MAX_VALUE * 8: "Array size > 4 billion elements"; + assert memory.size() < (long) Integer.MAX_VALUE * 8: "Array size >= Integer.MAX_VALUE elements"; this.memory = memory; this.baseObj = memory.getBaseObject(); this.baseOffset = memory.getBaseOffset(); @@ -51,6 +50,14 @@ public MemoryBlock memoryBlock() { return memory; } + public Object getBaseObject() { + return baseObj; + } + + public long getBaseOffset() { + return baseOffset; + } + /** * Returns the number of elements this array can hold. */ @@ -58,6 +65,15 @@ public long size() { return length; } + /** + * Fill this all with 0L. + */ + public void zeroOut() { + for (long off = baseOffset; off < baseOffset + length * WIDTH; off += WIDTH) { + Platform.putLong(baseObj, off, 0); + } + } + /** * Sets the value at position {@code index}. */ diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java similarity index 97% rename from unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java rename to common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java index 7857bf66a72a..c8c57381f332 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java @@ -87,7 +87,8 @@ public static boolean anySet(Object baseObject, long baseOffset, long bitSetWidt * To iterate over the true bits in a BitSet, use the following loop: *
    * 
-   *  for (long i = bs.nextSetBit(0, sizeInWords); i >= 0; i = bs.nextSetBit(i + 1, sizeInWords)) {
+   *  for (long i = bs.nextSetBit(0, sizeInWords); i >= 0;
+   *    i = bs.nextSetBit(i + 1, sizeInWords)) {
    *    // operate on index i here
    *  }
    * 
diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java
new file mode 100644
index 000000000000..5e7ee480cafd
--- /dev/null
+++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.unsafe.hash;
+
+import org.apache.spark.unsafe.Platform;
+
+/**
+ * 32-bit Murmur3 hasher.  This is based on Guava's Murmur3_32HashFunction.
+ */
+public final class Murmur3_x86_32 {
+  private static final int C1 = 0xcc9e2d51;
+  private static final int C2 = 0x1b873593;
+
+  private final int seed;
+
+  public Murmur3_x86_32(int seed) {
+    this.seed = seed;
+  }
+
+  @Override
+  public String toString() {
+    return "Murmur3_32(seed=" + seed + ")";
+  }
+
+  public int hashInt(int input) {
+    return hashInt(input, seed);
+  }
+
+  public static int hashInt(int input, int seed) {
+    int k1 = mixK1(input);
+    int h1 = mixH1(seed, k1);
+
+    return fmix(h1, 4);
+  }
+
+  public int hashUnsafeWords(Object base, long offset, int lengthInBytes) {
+    return hashUnsafeWords(base, offset, lengthInBytes, seed);
+  }
+
+  public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, int seed) {
+    // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method.
+    assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)";
+    int h1 = hashBytesByInt(base, offset, lengthInBytes, seed);
+    return fmix(h1, lengthInBytes);
+  }
+
+  public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, int seed) {
+    assert (lengthInBytes >= 0): "lengthInBytes cannot be negative";
+    int lengthAligned = lengthInBytes - lengthInBytes % 4;
+    int h1 = hashBytesByInt(base, offset, lengthAligned, seed);
+    for (int i = lengthAligned; i < lengthInBytes; i++) {
+      int halfWord = Platform.getByte(base, offset + i);
+      int k1 = mixK1(halfWord);
+      h1 = mixH1(h1, k1);
+    }
+    return fmix(h1, lengthInBytes);
+  }
+
+  private static int hashBytesByInt(Object base, long offset, int lengthInBytes, int seed) {
+    assert (lengthInBytes % 4 == 0);
+    int h1 = seed;
+    for (int i = 0; i < lengthInBytes; i += 4) {
+      int halfWord = Platform.getInt(base, offset + i);
+      int k1 = mixK1(halfWord);
+      h1 = mixH1(h1, k1);
+    }
+    return h1;
+  }
+
+  public int hashLong(long input) {
+    return hashLong(input, seed);
+  }
+
+  public static int hashLong(long input, int seed) {
+    int low = (int) input;
+    int high = (int) (input >>> 32);
+
+    int k1 = mixK1(low);
+    int h1 = mixH1(seed, k1);
+
+    k1 = mixK1(high);
+    h1 = mixH1(h1, k1);
+
+    return fmix(h1, 8);
+  }
+
+  private static int mixK1(int k1) {
+    k1 *= C1;
+    k1 = Integer.rotateLeft(k1, 15);
+    k1 *= C2;
+    return k1;
+  }
+
+  private static int mixH1(int h1, int k1) {
+    h1 ^= k1;
+    h1 = Integer.rotateLeft(h1, 13);
+    h1 = h1 * 5 + 0xe6546b64;
+    return h1;
+  }
+
+  // Finalization mix - force all bits of a hash block to avalanche
+  private static int fmix(int h1, int length) {
+    h1 ^= length;
+    h1 ^= h1 >>> 16;
+    h1 *= 0x85ebca6b;
+    h1 ^= h1 >>> 13;
+    h1 *= 0xc2b2ae35;
+    h1 ^= h1 >>> 16;
+    return h1;
+  }
+}
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java
similarity index 88%
rename from unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java
rename to common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java
index 09847cec9c4c..355748238540 100644
--- a/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java
+++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java
@@ -64,12 +64,19 @@ public MemoryBlock allocate(long size) throws OutOfMemoryError {
       }
     }
     long[] array = new long[(int) ((size + 7) / 8)];
-    return new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size);
+    MemoryBlock memory = new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size);
+    if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) {
+      memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE);
+    }
+    return memory;
   }
 
   @Override
   public void free(MemoryBlock memory) {
     final long size = memory.size();
+    if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) {
+      memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_FREED_VALUE);
+    }
     if (shouldPool(size)) {
       synchronized (this) {
         LinkedList> pool = bufferPoolsBySize.get(size);
diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java
new file mode 100644
index 000000000000..7b588681d979
--- /dev/null
+++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.unsafe.memory;
+
+public interface MemoryAllocator {
+
+  /**
+   * Whether to fill newly allocated and deallocated memory with 0xa5 and 0x5a bytes respectively.
+   * This helps catch misuse of uninitialized or freed memory, but imposes some overhead.
+   */
+  boolean MEMORY_DEBUG_FILL_ENABLED = Boolean.parseBoolean(
+    System.getProperty("spark.memory.debugFill", "false"));
+
+  // Same as jemalloc's debug fill values.
+  byte MEMORY_DEBUG_FILL_CLEAN_VALUE = (byte)0xa5;
+  byte MEMORY_DEBUG_FILL_FREED_VALUE = (byte)0x5a;
+
+  /**
+   * Allocates a contiguous block of memory. Note that the allocated memory is not guaranteed
+   * to be zeroed out (call `fill(0)` on the result if this is necessary).
+   */
+  MemoryBlock allocate(long size) throws OutOfMemoryError;
+
+  void free(MemoryBlock memory);
+
+  MemoryAllocator UNSAFE = new UnsafeMemoryAllocator();
+
+  MemoryAllocator HEAP = new HeapMemoryAllocator();
+}
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java
similarity index 91%
rename from unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java
rename to common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java
index e3e79471154d..cd1d378bc147 100644
--- a/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java
+++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java
@@ -51,6 +51,13 @@ public long size() {
    * Creates a memory block pointing to the memory used by the long array.
    */
   public static MemoryBlock fromLongArray(final long[] array) {
-    return new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8);
+    return new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8L);
+  }
+
+  /**
+   * Fills the memory block with the specified byte value.
+   */
+  public void fill(byte value) {
+    Platform.setMemory(obj, offset, length, value);
   }
 }
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java
similarity index 100%
rename from unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java
rename to common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java
similarity index 80%
rename from unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java
rename to common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java
index 98ce711176e4..55bcdf1ed7b0 100644
--- a/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java
+++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java
@@ -27,13 +27,20 @@ public class UnsafeMemoryAllocator implements MemoryAllocator {
   @Override
   public MemoryBlock allocate(long size) throws OutOfMemoryError {
     long address = Platform.allocateMemory(size);
-    return new MemoryBlock(null, address, size);
+    MemoryBlock memory = new MemoryBlock(null, address, size);
+    if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) {
+      memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE);
+    }
+    return memory;
   }
 
   @Override
   public void free(MemoryBlock memory) {
     assert (memory.obj == null) :
       "baseObject not null; are you trying to use the off-heap allocator to free on-heap memory?";
+    if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) {
+      memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_FREED_VALUE);
+    }
     Platform.freeMemory(memory.offset);
   }
 }
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
similarity index 96%
rename from unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
rename to common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
index 3ced2094f5e6..7ced13d35723 100644
--- a/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
+++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/ByteArray.java
@@ -44,7 +44,7 @@ public static long getPrefix(byte[] bytes) {
       final int minLen = Math.min(bytes.length, 8);
       long p = 0;
       for (int i = 0; i < minLen; ++i) {
-        p |= (128L + Platform.getByte(bytes, Platform.BYTE_ARRAY_OFFSET + i))
+        p |= ((long) Platform.getByte(bytes, Platform.BYTE_ARRAY_OFFSET + i) & 0xff)
             << (56 - 8 * i);
       }
       return p;
diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java
similarity index 81%
rename from unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java
rename to common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java
index 30e175807636..621f2c6bf377 100644
--- a/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java
+++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java
@@ -62,7 +62,7 @@ private static long toLong(String s) {
     if (s == null) {
       return 0;
     } else {
-      return Long.valueOf(s);
+      return Long.parseLong(s);
     }
   }
 
@@ -91,7 +91,7 @@ public static long toLongWithRange(String fieldName,
       String s, long minValue, long maxValue) throws IllegalArgumentException {
     long result = 0;
     if (s != null) {
-      result = Long.valueOf(s);
+      result = Long.parseLong(s);
       if (result < minValue || result > maxValue) {
         throw new IllegalArgumentException(String.format("%s %d outside range [%d, %d]",
           fieldName, result, minValue, maxValue));
@@ -178,34 +178,52 @@ public static CalendarInterval fromSingleUnitString(String unit, String s)
         "Interval string does not match day-time format of 'd h:m:s.n': " + s);
     } else {
       try {
-        if (unit.equals("year")) {
-          int year = (int) toLongWithRange("year", m.group(1),
-            Integer.MIN_VALUE / 12, Integer.MAX_VALUE / 12);
-          result = new CalendarInterval(year * 12, 0L);
-
-        } else if (unit.equals("month")) {
-          int month = (int) toLongWithRange("month", m.group(1),
-            Integer.MIN_VALUE, Integer.MAX_VALUE);
-          result = new CalendarInterval(month, 0L);
-
-        } else if (unit.equals("day")) {
-          long day = toLongWithRange("day", m.group(1),
-            Long.MIN_VALUE / MICROS_PER_DAY, Long.MAX_VALUE / MICROS_PER_DAY);
-          result = new CalendarInterval(0, day * MICROS_PER_DAY);
-
-        } else if (unit.equals("hour")) {
-          long hour = toLongWithRange("hour", m.group(1),
-            Long.MIN_VALUE / MICROS_PER_HOUR, Long.MAX_VALUE / MICROS_PER_HOUR);
-          result = new CalendarInterval(0, hour * MICROS_PER_HOUR);
-
-        } else if (unit.equals("minute")) {
-          long minute = toLongWithRange("minute", m.group(1),
-            Long.MIN_VALUE / MICROS_PER_MINUTE, Long.MAX_VALUE / MICROS_PER_MINUTE);
-          result = new CalendarInterval(0, minute * MICROS_PER_MINUTE);
-
-        } else if (unit.equals("second")) {
-          long micros = parseSecondNano(m.group(1));
-          result = new CalendarInterval(0, micros);
+        switch (unit) {
+          case "year":
+            int year = (int) toLongWithRange("year", m.group(1),
+              Integer.MIN_VALUE / 12, Integer.MAX_VALUE / 12);
+            result = new CalendarInterval(year * 12, 0L);
+            break;
+          case "month":
+            int month = (int) toLongWithRange("month", m.group(1),
+              Integer.MIN_VALUE, Integer.MAX_VALUE);
+            result = new CalendarInterval(month, 0L);
+            break;
+          case "week":
+            long week = toLongWithRange("week", m.group(1),
+              Long.MIN_VALUE / MICROS_PER_WEEK, Long.MAX_VALUE / MICROS_PER_WEEK);
+            result = new CalendarInterval(0, week * MICROS_PER_WEEK);
+            break;
+          case "day":
+            long day = toLongWithRange("day", m.group(1),
+              Long.MIN_VALUE / MICROS_PER_DAY, Long.MAX_VALUE / MICROS_PER_DAY);
+            result = new CalendarInterval(0, day * MICROS_PER_DAY);
+            break;
+          case "hour":
+            long hour = toLongWithRange("hour", m.group(1),
+              Long.MIN_VALUE / MICROS_PER_HOUR, Long.MAX_VALUE / MICROS_PER_HOUR);
+            result = new CalendarInterval(0, hour * MICROS_PER_HOUR);
+            break;
+          case "minute":
+            long minute = toLongWithRange("minute", m.group(1),
+              Long.MIN_VALUE / MICROS_PER_MINUTE, Long.MAX_VALUE / MICROS_PER_MINUTE);
+            result = new CalendarInterval(0, minute * MICROS_PER_MINUTE);
+            break;
+          case "second": {
+            long micros = parseSecondNano(m.group(1));
+            result = new CalendarInterval(0, micros);
+            break;
+          }
+          case "millisecond":
+            long millisecond = toLongWithRange("millisecond", m.group(1),
+              Long.MIN_VALUE / MICROS_PER_MILLI, Long.MAX_VALUE / MICROS_PER_MILLI);
+            result = new CalendarInterval(0, millisecond * MICROS_PER_MILLI);
+            break;
+          case "microsecond": {
+            long micros = Long.parseLong(m.group(1));
+            result = new CalendarInterval(0, micros);
+            break;
+          }
         }
       } catch (Exception e) {
         throw new IllegalArgumentException("Error parsing interval string: " + e.getMessage(), e);
@@ -238,6 +256,10 @@ public static long parseSecondNano(String secondNano) throws IllegalArgumentExce
   public final int months;
   public final long microseconds;
 
+  public long milliseconds() {
+    return this.microseconds / MICROS_PER_MILLI;
+  }
+
   public CalendarInterval(int months, long microseconds) {
     this.months = months;
     this.microseconds = microseconds;
@@ -304,7 +326,7 @@ public String toString() {
 
   private void appendUnit(StringBuilder sb, long value, String unit) {
     if (value != 0) {
-      sb.append(" " + value + " " + unit + "s");
+      sb.append(' ').append(value).append(' ').append(unit).append('s');
     }
   }
 }
diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java
new file mode 100644
index 000000000000..ce4a06bde80c
--- /dev/null
+++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java
@@ -0,0 +1,1390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.unsafe.types;
+
+import javax.annotation.Nonnull;
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Map;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.KryoSerializable;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.array.ByteArrayMethods;
+import org.apache.spark.unsafe.hash.Murmur3_x86_32;
+
+import static org.apache.spark.unsafe.Platform.*;
+
+
+/**
+ * A UTF-8 String for internal Spark use.
+ * 

+ * A String encoded in UTF-8 as an Array[Byte], which can be used for comparison, + * search, see http://en.wikipedia.org/wiki/UTF-8 for details. + *

+ * Note: This is not designed for general use cases, should not be used outside SQL. + */ +public final class UTF8String implements Comparable, Externalizable, KryoSerializable, + Cloneable { + + // These are only updated by readExternal() or read() + @Nonnull + private Object base; + private long offset; + private int numBytes; + + public Object getBaseObject() { return base; } + public long getBaseOffset() { return offset; } + + private static int[] bytesOfCodePointInUTF8 = {2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, + 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, + 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, + 4, 4, 4, 4, 4, 4, 4, 4, + 5, 5, 5, 5, + 6, 6}; + + private static final boolean IS_LITTLE_ENDIAN = + ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN; + + private static final UTF8String COMMA_UTF8 = UTF8String.fromString(","); + public static final UTF8String EMPTY_UTF8 = UTF8String.fromString(""); + + /** + * Creates an UTF8String from byte array, which should be encoded in UTF-8. + * + * Note: `bytes` will be hold by returned UTF8String. + */ + public static UTF8String fromBytes(byte[] bytes) { + if (bytes != null) { + return new UTF8String(bytes, BYTE_ARRAY_OFFSET, bytes.length); + } else { + return null; + } + } + + /** + * Creates an UTF8String from byte array, which should be encoded in UTF-8. + * + * Note: `bytes` will be hold by returned UTF8String. + */ + public static UTF8String fromBytes(byte[] bytes, int offset, int numBytes) { + if (bytes != null) { + return new UTF8String(bytes, BYTE_ARRAY_OFFSET + offset, numBytes); + } else { + return null; + } + } + + /** + * Creates an UTF8String from given address (base and offset) and length. + */ + public static UTF8String fromAddress(Object base, long offset, int numBytes) { + return new UTF8String(base, offset, numBytes); + } + + /** + * Creates an UTF8String from String. + */ + public static UTF8String fromString(String str) { + return str == null ? null : fromBytes(str.getBytes(StandardCharsets.UTF_8)); + } + + /** + * Creates an UTF8String that contains `length` spaces. + */ + public static UTF8String blankString(int length) { + byte[] spaces = new byte[length]; + Arrays.fill(spaces, (byte) ' '); + return fromBytes(spaces); + } + + protected UTF8String(Object base, long offset, int numBytes) { + this.base = base; + this.offset = offset; + this.numBytes = numBytes; + } + + // for serialization + public UTF8String() { + this(null, 0, 0); + } + + /** + * Writes the content of this string into a memory address, identified by an object and an offset. + * The target memory address must already been allocated, and have enough space to hold all the + * bytes in this string. + */ + public void writeToMemory(Object target, long targetOffset) { + Platform.copyMemory(base, offset, target, targetOffset, numBytes); + } + + public void writeTo(ByteBuffer buffer) { + assert(buffer.hasArray()); + byte[] target = buffer.array(); + int offset = buffer.arrayOffset(); + int pos = buffer.position(); + writeToMemory(target, Platform.BYTE_ARRAY_OFFSET + offset + pos); + buffer.position(pos + numBytes); + } + + /** + * Returns a {@link ByteBuffer} wrapping the base object if it is a byte array + * or a copy of the data if the base object is not a byte array. + * + * Unlike getBytes this will not create a copy the array if this is a slice. + */ + @Nonnull + public ByteBuffer getByteBuffer() { + if (base instanceof byte[] && offset >= BYTE_ARRAY_OFFSET) { + final byte[] bytes = (byte[]) base; + + // the offset includes an object header... this is only needed for unsafe copies + final long arrayOffset = offset - BYTE_ARRAY_OFFSET; + + // verify that the offset and length points somewhere inside the byte array + // and that the offset can safely be truncated to a 32-bit integer + if ((long) bytes.length < arrayOffset + numBytes) { + throw new ArrayIndexOutOfBoundsException(); + } + + return ByteBuffer.wrap(bytes, (int) arrayOffset, numBytes); + } else { + return ByteBuffer.wrap(getBytes()); + } + } + + public void writeTo(OutputStream out) throws IOException { + final ByteBuffer bb = this.getByteBuffer(); + assert(bb.hasArray()); + + // similar to Utils.writeByteBuffer but without the spark-core dependency + out.write(bb.array(), bb.arrayOffset() + bb.position(), bb.remaining()); + } + + /** + * Returns the number of bytes for a code point with the first byte as `b` + * @param b The first byte of a code point + */ + private static int numBytesForFirstByte(final byte b) { + final int offset = (b & 0xFF) - 192; + return (offset >= 0) ? bytesOfCodePointInUTF8[offset] : 1; + } + + /** + * Returns the number of bytes + */ + public int numBytes() { + return numBytes; + } + + /** + * Returns the number of code points in it. + */ + public int numChars() { + int len = 0; + for (int i = 0; i < numBytes; i += numBytesForFirstByte(getByte(i))) { + len += 1; + } + return len; + } + + /** + * Returns a 64-bit integer that can be used as the prefix used in sorting. + */ + public long getPrefix() { + // Since JVMs are either 4-byte aligned or 8-byte aligned, we check the size of the string. + // If size is 0, just return 0. + // If size is between 0 and 4 (inclusive), assume data is 4-byte aligned under the hood and + // use a getInt to fetch the prefix. + // If size is greater than 4, assume we have at least 8 bytes of data to fetch. + // After getting the data, we use a mask to mask out data that is not part of the string. + long p; + long mask = 0; + if (IS_LITTLE_ENDIAN) { + if (numBytes >= 8) { + p = Platform.getLong(base, offset); + } else if (numBytes > 4) { + p = Platform.getLong(base, offset); + mask = (1L << (8 - numBytes) * 8) - 1; + } else if (numBytes > 0) { + p = (long) Platform.getInt(base, offset); + mask = (1L << (8 - numBytes) * 8) - 1; + } else { + p = 0; + } + p = java.lang.Long.reverseBytes(p); + } else { + // byteOrder == ByteOrder.BIG_ENDIAN + if (numBytes >= 8) { + p = Platform.getLong(base, offset); + } else if (numBytes > 4) { + p = Platform.getLong(base, offset); + mask = (1L << (8 - numBytes) * 8) - 1; + } else if (numBytes > 0) { + p = ((long) Platform.getInt(base, offset)) << 32; + mask = (1L << (8 - numBytes) * 8) - 1; + } else { + p = 0; + } + } + p &= ~mask; + return p; + } + + /** + * Returns the underline bytes, will be a copy of it if it's part of another array. + */ + public byte[] getBytes() { + // avoid copy if `base` is `byte[]` + if (offset == BYTE_ARRAY_OFFSET && base instanceof byte[] + && ((byte[]) base).length == numBytes) { + return (byte[]) base; + } else { + byte[] bytes = new byte[numBytes]; + copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); + return bytes; + } + } + + /** + * Returns a substring of this. + * @param start the position of first code point + * @param until the position after last code point, exclusive. + */ + public UTF8String substring(final int start, final int until) { + if (until <= start || start >= numBytes) { + return EMPTY_UTF8; + } + + int i = 0; + int c = 0; + while (i < numBytes && c < start) { + i += numBytesForFirstByte(getByte(i)); + c += 1; + } + + int j = i; + while (i < numBytes && c < until) { + i += numBytesForFirstByte(getByte(i)); + c += 1; + } + + if (i > j) { + byte[] bytes = new byte[i - j]; + copyMemory(base, offset + j, bytes, BYTE_ARRAY_OFFSET, i - j); + return fromBytes(bytes); + } else { + return EMPTY_UTF8; + } + } + + public UTF8String substringSQL(int pos, int length) { + // Information regarding the pos calculation: + // Hive and SQL use one-based indexing for SUBSTR arguments but also accept zero and + // negative indices for start positions. If a start index i is greater than 0, it + // refers to element i-1 in the sequence. If a start index i is less than 0, it refers + // to the -ith element before the end of the sequence. If a start index i is 0, it + // refers to the first element. + int len = numChars(); + int start = (pos > 0) ? pos -1 : ((pos < 0) ? len + pos : 0); + int end = (length == Integer.MAX_VALUE) ? len : start + length; + return substring(start, end); + } + + /** + * Returns whether this contains `substring` or not. + */ + public boolean contains(final UTF8String substring) { + if (substring.numBytes == 0) { + return true; + } + + byte first = substring.getByte(0); + for (int i = 0; i <= numBytes - substring.numBytes; i++) { + if (getByte(i) == first && matchAt(substring, i)) { + return true; + } + } + return false; + } + + /** + * Returns the byte at position `i`. + */ + private byte getByte(int i) { + return Platform.getByte(base, offset + i); + } + + private boolean matchAt(final UTF8String s, int pos) { + if (s.numBytes + pos > numBytes || pos < 0) { + return false; + } + return ByteArrayMethods.arrayEquals(base, offset + pos, s.base, s.offset, s.numBytes); + } + + public boolean startsWith(final UTF8String prefix) { + return matchAt(prefix, 0); + } + + public boolean endsWith(final UTF8String suffix) { + return matchAt(suffix, numBytes - suffix.numBytes); + } + + /** + * Returns the upper case of this string + */ + public UTF8String toUpperCase() { + if (numBytes == 0) { + return EMPTY_UTF8; + } + + byte[] bytes = new byte[numBytes]; + bytes[0] = (byte) Character.toTitleCase(getByte(0)); + for (int i = 0; i < numBytes; i++) { + byte b = getByte(i); + if (numBytesForFirstByte(b) != 1) { + // fallback + return toUpperCaseSlow(); + } + int upper = Character.toUpperCase((int) b); + if (upper > 127) { + // fallback + return toUpperCaseSlow(); + } + bytes[i] = (byte) upper; + } + return fromBytes(bytes); + } + + private UTF8String toUpperCaseSlow() { + return fromString(toString().toUpperCase()); + } + + /** + * Returns the lower case of this string + */ + public UTF8String toLowerCase() { + if (numBytes == 0) { + return EMPTY_UTF8; + } + + byte[] bytes = new byte[numBytes]; + bytes[0] = (byte) Character.toTitleCase(getByte(0)); + for (int i = 0; i < numBytes; i++) { + byte b = getByte(i); + if (numBytesForFirstByte(b) != 1) { + // fallback + return toLowerCaseSlow(); + } + int lower = Character.toLowerCase((int) b); + if (lower > 127) { + // fallback + return toLowerCaseSlow(); + } + bytes[i] = (byte) lower; + } + return fromBytes(bytes); + } + + private UTF8String toLowerCaseSlow() { + return fromString(toString().toLowerCase()); + } + + /** + * Returns the title case of this string, that could be used as title. + */ + public UTF8String toTitleCase() { + if (numBytes == 0) { + return EMPTY_UTF8; + } + + byte[] bytes = new byte[numBytes]; + for (int i = 0; i < numBytes; i++) { + byte b = getByte(i); + if (i == 0 || getByte(i - 1) == ' ') { + if (numBytesForFirstByte(b) != 1) { + // fallback + return toTitleCaseSlow(); + } + int upper = Character.toTitleCase(b); + if (upper > 127) { + // fallback + return toTitleCaseSlow(); + } + bytes[i] = (byte) upper; + } else { + bytes[i] = b; + } + } + return fromBytes(bytes); + } + + private UTF8String toTitleCaseSlow() { + StringBuffer sb = new StringBuffer(); + String s = toString(); + sb.append(s); + sb.setCharAt(0, Character.toTitleCase(sb.charAt(0))); + for (int i = 1; i < s.length(); i++) { + if (sb.charAt(i - 1) == ' ') { + sb.setCharAt(i, Character.toTitleCase(sb.charAt(i))); + } + } + return fromString(sb.toString()); + } + + /* + * Returns the index of the string `match` in this String. This string has to be a comma separated + * list. If `match` contains a comma 0 will be returned. If the `match` isn't part of this String, + * 0 will be returned, else the index of match (1-based index) + */ + public int findInSet(UTF8String match) { + if (match.contains(COMMA_UTF8)) { + return 0; + } + + int n = 1, lastComma = -1; + for (int i = 0; i < numBytes; i++) { + if (getByte(i) == (byte) ',') { + if (i - (lastComma + 1) == match.numBytes && + ByteArrayMethods.arrayEquals(base, offset + (lastComma + 1), match.base, match.offset, + match.numBytes)) { + return n; + } + lastComma = i; + n++; + } + } + if (numBytes - (lastComma + 1) == match.numBytes && + ByteArrayMethods.arrayEquals(base, offset + (lastComma + 1), match.base, match.offset, + match.numBytes)) { + return n; + } + return 0; + } + + /** + * Copy the bytes from the current UTF8String, and make a new UTF8String. + * @param start the start position of the current UTF8String in bytes. + * @param end the end position of the current UTF8String in bytes. + * @return a new UTF8String in the position of [start, end] of current UTF8String bytes. + */ + private UTF8String copyUTF8String(int start, int end) { + int len = end - start + 1; + byte[] newBytes = new byte[len]; + copyMemory(base, offset + start, newBytes, BYTE_ARRAY_OFFSET, len); + return UTF8String.fromBytes(newBytes); + } + + public UTF8String trim() { + int s = 0; + int e = this.numBytes - 1; + // skip all of the space (0x20) in the left side + while (s < this.numBytes && getByte(s) == 0x20) s++; + // skip all of the space (0x20) in the right side + while (e >= 0 && getByte(e) == 0x20) e--; + if (s > e) { + // empty string + return EMPTY_UTF8; + } else { + return copyUTF8String(s, e); + } + } + + /** + * Based on the given trim string, trim this string starting from both ends + * This method searches for each character in the source string, removes the character if it is + * found in the trim string, stops at the first not found. It calls the trimLeft first, then + * trimRight. It returns a new string in which both ends trim characters have been removed. + * @param trimString the trim character string + */ + public UTF8String trim(UTF8String trimString) { + if (trimString != null) { + return trimLeft(trimString).trimRight(trimString); + } else { + return null; + } + } + + public UTF8String trimLeft() { + int s = 0; + // skip all of the space (0x20) in the left side + while (s < this.numBytes && getByte(s) == 0x20) s++; + if (s == this.numBytes) { + // empty string + return EMPTY_UTF8; + } else { + return copyUTF8String(s, this.numBytes - 1); + } + } + + /** + * Based on the given trim string, trim this string starting from left end + * This method searches each character in the source string starting from the left end, removes + * the character if it is in the trim string, stops at the first character which is not in the + * trim string, returns the new string. + * @param trimString the trim character string + */ + public UTF8String trimLeft(UTF8String trimString) { + if (trimString == null) return null; + // the searching byte position in the source string + int srchIdx = 0; + // the first beginning byte position of a non-matching character + int trimIdx = 0; + + while (srchIdx < numBytes) { + UTF8String searchChar = copyUTF8String( + srchIdx, srchIdx + numBytesForFirstByte(this.getByte(srchIdx)) - 1); + int searchCharBytes = searchChar.numBytes; + // try to find the matching for the searchChar in the trimString set + if (trimString.find(searchChar, 0) >= 0) { + trimIdx += searchCharBytes; + } else { + // no matching, exit the search + break; + } + srchIdx += searchCharBytes; + } + + if (trimIdx >= numBytes) { + // empty string + return EMPTY_UTF8; + } else { + return copyUTF8String(trimIdx, numBytes - 1); + } + } + + public UTF8String trimRight() { + int e = numBytes - 1; + // skip all of the space (0x20) in the right side + while (e >= 0 && getByte(e) == 0x20) e--; + + if (e < 0) { + // empty string + return EMPTY_UTF8; + } else { + return copyUTF8String(0, e); + } + } + + /** + * Based on the given trim string, trim this string starting from right end + * This method searches each character in the source string starting from the right end, + * removes the character if it is in the trim string, stops at the first character which is not + * in the trim string, returns the new string. + * @param trimString the trim character string + */ + public UTF8String trimRight(UTF8String trimString) { + if (trimString == null) return null; + int charIdx = 0; + // number of characters from the source string + int numChars = 0; + // array of character length for the source string + int[] stringCharLen = new int[numBytes]; + // array of the first byte position for each character in the source string + int[] stringCharPos = new int[numBytes]; + // build the position and length array + while (charIdx < numBytes) { + stringCharPos[numChars] = charIdx; + stringCharLen[numChars] = numBytesForFirstByte(getByte(charIdx)); + charIdx += stringCharLen[numChars]; + numChars ++; + } + + // index trimEnd points to the first no matching byte position from the right side of + // the source string. + int trimEnd = numBytes - 1; + while (numChars > 0) { + UTF8String searchChar = copyUTF8String( + stringCharPos[numChars - 1], + stringCharPos[numChars - 1] + stringCharLen[numChars - 1] - 1); + if (trimString.find(searchChar, 0) >= 0) { + trimEnd -= stringCharLen[numChars - 1]; + } else { + break; + } + numChars --; + } + + if (trimEnd < 0) { + // empty string + return EMPTY_UTF8; + } else { + return copyUTF8String(0, trimEnd); + } + } + + public UTF8String reverse() { + byte[] result = new byte[this.numBytes]; + + int i = 0; // position in byte + while (i < numBytes) { + int len = numBytesForFirstByte(getByte(i)); + copyMemory(this.base, this.offset + i, result, + BYTE_ARRAY_OFFSET + result.length - i - len, len); + + i += len; + } + + return UTF8String.fromBytes(result); + } + + public UTF8String repeat(int times) { + if (times <= 0) { + return EMPTY_UTF8; + } + + byte[] newBytes = new byte[numBytes * times]; + copyMemory(this.base, this.offset, newBytes, BYTE_ARRAY_OFFSET, numBytes); + + int copied = 1; + while (copied < times) { + int toCopy = Math.min(copied, times - copied); + System.arraycopy(newBytes, 0, newBytes, copied * numBytes, numBytes * toCopy); + copied += toCopy; + } + + return UTF8String.fromBytes(newBytes); + } + + /** + * Returns the position of the first occurrence of substr in + * current string from the specified position (0-based index). + * + * @param v the string to be searched + * @param start the start position of the current string for searching + * @return the position of the first occurrence of substr, if not found, -1 returned. + */ + public int indexOf(UTF8String v, int start) { + if (v.numBytes() == 0) { + return 0; + } + + // locate to the start position. + int i = 0; // position in byte + int c = 0; // position in character + while (i < numBytes && c < start) { + i += numBytesForFirstByte(getByte(i)); + c += 1; + } + + do { + if (i + v.numBytes > numBytes) { + return -1; + } + if (ByteArrayMethods.arrayEquals(base, offset + i, v.base, v.offset, v.numBytes)) { + return c; + } + i += numBytesForFirstByte(getByte(i)); + c += 1; + } while (i < numBytes); + + return -1; + } + + /** + * Find the `str` from left to right. + */ + private int find(UTF8String str, int start) { + assert (str.numBytes > 0); + while (start <= numBytes - str.numBytes) { + if (ByteArrayMethods.arrayEquals(base, offset + start, str.base, str.offset, str.numBytes)) { + return start; + } + start += 1; + } + return -1; + } + + /** + * Find the `str` from right to left. + */ + private int rfind(UTF8String str, int start) { + assert (str.numBytes > 0); + while (start >= 0) { + if (ByteArrayMethods.arrayEquals(base, offset + start, str.base, str.offset, str.numBytes)) { + return start; + } + start -= 1; + } + return -1; + } + + /** + * Returns the substring from string str before count occurrences of the delimiter delim. + * If count is positive, everything the left of the final delimiter (counting from left) is + * returned. If count is negative, every to the right of the final delimiter (counting from the + * right) is returned. subStringIndex performs a case-sensitive match when searching for delim. + */ + public UTF8String subStringIndex(UTF8String delim, int count) { + if (delim.numBytes == 0 || count == 0) { + return EMPTY_UTF8; + } + if (count > 0) { + int idx = -1; + while (count > 0) { + idx = find(delim, idx + 1); + if (idx >= 0) { + count --; + } else { + // can not find enough delim + return this; + } + } + if (idx == 0) { + return EMPTY_UTF8; + } + byte[] bytes = new byte[idx]; + copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, idx); + return fromBytes(bytes); + + } else { + int idx = numBytes - delim.numBytes + 1; + count = -count; + while (count > 0) { + idx = rfind(delim, idx - 1); + if (idx >= 0) { + count --; + } else { + // can not find enough delim + return this; + } + } + if (idx + delim.numBytes == numBytes) { + return EMPTY_UTF8; + } + int size = numBytes - delim.numBytes - idx; + byte[] bytes = new byte[size]; + copyMemory(base, offset + idx + delim.numBytes, bytes, BYTE_ARRAY_OFFSET, size); + return fromBytes(bytes); + } + } + + /** + * Returns str, right-padded with pad to a length of len + * For example: + * ('hi', 5, '??') => 'hi???' + * ('hi', 1, '??') => 'h' + */ + public UTF8String rpad(int len, UTF8String pad) { + int spaces = len - this.numChars(); // number of char need to pad + if (spaces <= 0 || pad.numBytes() == 0) { + // no padding at all, return the substring of the current string + return substring(0, len); + } else { + int padChars = pad.numChars(); + int count = spaces / padChars; // how many padding string needed + // the partial string of the padding + UTF8String remain = pad.substring(0, spaces - padChars * count); + + byte[] data = new byte[this.numBytes + pad.numBytes * count + remain.numBytes]; + copyMemory(this.base, this.offset, data, BYTE_ARRAY_OFFSET, this.numBytes); + int offset = this.numBytes; + int idx = 0; + while (idx < count) { + copyMemory(pad.base, pad.offset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); + ++ idx; + offset += pad.numBytes; + } + copyMemory(remain.base, remain.offset, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); + + return UTF8String.fromBytes(data); + } + } + + /** + * Returns str, left-padded with pad to a length of len. + * For example: + * ('hi', 5, '??') => '???hi' + * ('hi', 1, '??') => 'h' + */ + public UTF8String lpad(int len, UTF8String pad) { + int spaces = len - this.numChars(); // number of char need to pad + if (spaces <= 0 || pad.numBytes() == 0) { + // no padding at all, return the substring of the current string + return substring(0, len); + } else { + int padChars = pad.numChars(); + int count = spaces / padChars; // how many padding string needed + // the partial string of the padding + UTF8String remain = pad.substring(0, spaces - padChars * count); + + byte[] data = new byte[this.numBytes + pad.numBytes * count + remain.numBytes]; + + int offset = 0; + int idx = 0; + while (idx < count) { + copyMemory(pad.base, pad.offset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); + ++ idx; + offset += pad.numBytes; + } + copyMemory(remain.base, remain.offset, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); + offset += remain.numBytes; + copyMemory(this.base, this.offset, data, BYTE_ARRAY_OFFSET + offset, numBytes()); + + return UTF8String.fromBytes(data); + } + } + + /** + * Concatenates input strings together into a single string. Returns null if any input is null. + */ + public static UTF8String concat(UTF8String... inputs) { + // Compute the total length of the result. + int totalLength = 0; + for (int i = 0; i < inputs.length; i++) { + if (inputs[i] != null) { + totalLength += inputs[i].numBytes; + } else { + return null; + } + } + + // Allocate a new byte array, and copy the inputs one by one into it. + final byte[] result = new byte[totalLength]; + int offset = 0; + for (int i = 0; i < inputs.length; i++) { + int len = inputs[i].numBytes; + copyMemory( + inputs[i].base, inputs[i].offset, + result, BYTE_ARRAY_OFFSET + offset, + len); + offset += len; + } + return fromBytes(result); + } + + /** + * Concatenates input strings together into a single string using the separator. + * A null input is skipped. For example, concat(",", "a", null, "c") would yield "a,c". + */ + public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { + if (separator == null) { + return null; + } + + int numInputBytes = 0; // total number of bytes from the inputs + int numInputs = 0; // number of non-null inputs + for (int i = 0; i < inputs.length; i++) { + if (inputs[i] != null) { + numInputBytes += inputs[i].numBytes; + numInputs++; + } + } + + if (numInputs == 0) { + // Return an empty string if there is no input, or all the inputs are null. + return EMPTY_UTF8; + } + + // Allocate a new byte array, and copy the inputs one by one into it. + // The size of the new array is the size of all inputs, plus the separators. + final byte[] result = new byte[numInputBytes + (numInputs - 1) * separator.numBytes]; + int offset = 0; + + for (int i = 0, j = 0; i < inputs.length; i++) { + if (inputs[i] != null) { + int len = inputs[i].numBytes; + copyMemory( + inputs[i].base, inputs[i].offset, + result, BYTE_ARRAY_OFFSET + offset, + len); + offset += len; + + j++; + // Add separator if this is not the last input. + if (j < numInputs) { + copyMemory( + separator.base, separator.offset, + result, BYTE_ARRAY_OFFSET + offset, + separator.numBytes); + offset += separator.numBytes; + } + } + } + return fromBytes(result); + } + + public UTF8String[] split(UTF8String pattern, int limit) { + String[] splits = toString().split(pattern.toString(), limit); + UTF8String[] res = new UTF8String[splits.length]; + for (int i = 0; i < res.length; i++) { + res[i] = fromString(splits[i]); + } + return res; + } + + public UTF8String replace(UTF8String search, UTF8String replace) { + if (EMPTY_UTF8.equals(search)) { + return this; + } + String replaced = toString().replace( + search.toString(), replace.toString()); + return fromString(replaced); + } + + // TODO: Need to use `Code Point` here instead of Char in case the character longer than 2 bytes + public UTF8String translate(Map dict) { + String srcStr = this.toString(); + + StringBuilder sb = new StringBuilder(); + for(int k = 0; k< srcStr.length(); k++) { + if (null == dict.get(srcStr.charAt(k))) { + sb.append(srcStr.charAt(k)); + } else if ('\0' != dict.get(srcStr.charAt(k))){ + sb.append(dict.get(srcStr.charAt(k))); + } + } + return fromString(sb.toString()); + } + + /** + * Wrapper over `long` to allow result of parsing long from string to be accessed via reference. + * This is done solely for better performance and is not expected to be used by end users. + */ + public static class LongWrapper implements Serializable { + public transient long value = 0; + } + + /** + * Wrapper over `int` to allow result of parsing integer from string to be accessed via reference. + * This is done solely for better performance and is not expected to be used by end users. + * + * {@link LongWrapper} could have been used here but using `int` directly save the extra cost of + * conversion from `long` to `int` + */ + public static class IntWrapper implements Serializable { + public transient int value = 0; + } + + /** + * Parses this UTF8String to long. + * + * Note that, in this method we accumulate the result in negative format, and convert it to + * positive format at the end, if this string is not started with '-'. This is because min value + * is bigger than max value in digits, e.g. Long.MAX_VALUE is '9223372036854775807' and + * Long.MIN_VALUE is '-9223372036854775808'. + * + * This code is mostly copied from LazyLong.parseLong in Hive. + * + * @param toLongResult If a valid `long` was parsed from this UTF8String, then its value would + * be set in `toLongResult` + * @return true if the parsing was successful else false + */ + public boolean toLong(LongWrapper toLongResult) { + if (numBytes == 0) { + return false; + } + + byte b = getByte(0); + final boolean negative = b == '-'; + int offset = 0; + if (negative || b == '+') { + offset++; + if (numBytes == 1) { + return false; + } + } + + final byte separator = '.'; + final int radix = 10; + final long stopValue = Long.MIN_VALUE / radix; + long result = 0; + + while (offset < numBytes) { + b = getByte(offset); + offset++; + if (b == separator) { + // We allow decimals and will return a truncated integral in that case. + // Therefore we won't throw an exception here (checking the fractional + // part happens below.) + break; + } + + int digit; + if (b >= '0' && b <= '9') { + digit = b - '0'; + } else { + return false; + } + + // We are going to process the new digit and accumulate the result. However, before doing + // this, if the result is already smaller than the stopValue(Long.MIN_VALUE / radix), then + // result * 10 will definitely be smaller than minValue, and we can stop. + if (result < stopValue) { + return false; + } + + result = result * radix - digit; + // Since the previous result is less than or equal to stopValue(Long.MIN_VALUE / radix), we + // can just use `result > 0` to check overflow. If result overflows, we should stop. + if (result > 0) { + return false; + } + } + + // This is the case when we've encountered a decimal separator. The fractional + // part will not change the number, but we will verify that the fractional part + // is well formed. + while (offset < numBytes) { + byte currentByte = getByte(offset); + if (currentByte < '0' || currentByte > '9') { + return false; + } + offset++; + } + + if (!negative) { + result = -result; + if (result < 0) { + return false; + } + } + + toLongResult.value = result; + return true; + } + + /** + * Parses this UTF8String to int. + * + * Note that, in this method we accumulate the result in negative format, and convert it to + * positive format at the end, if this string is not started with '-'. This is because min value + * is bigger than max value in digits, e.g. Integer.MAX_VALUE is '2147483647' and + * Integer.MIN_VALUE is '-2147483648'. + * + * This code is mostly copied from LazyInt.parseInt in Hive. + * + * Note that, this method is almost same as `toLong`, but we leave it duplicated for performance + * reasons, like Hive does. + * + * @param intWrapper If a valid `int` was parsed from this UTF8String, then its value would + * be set in `intWrapper` + * @return true if the parsing was successful else false + */ + public boolean toInt(IntWrapper intWrapper) { + if (numBytes == 0) { + return false; + } + + byte b = getByte(0); + final boolean negative = b == '-'; + int offset = 0; + if (negative || b == '+') { + offset++; + if (numBytes == 1) { + return false; + } + } + + final byte separator = '.'; + final int radix = 10; + final int stopValue = Integer.MIN_VALUE / radix; + int result = 0; + + while (offset < numBytes) { + b = getByte(offset); + offset++; + if (b == separator) { + // We allow decimals and will return a truncated integral in that case. + // Therefore we won't throw an exception here (checking the fractional + // part happens below.) + break; + } + + int digit; + if (b >= '0' && b <= '9') { + digit = b - '0'; + } else { + return false; + } + + // We are going to process the new digit and accumulate the result. However, before doing + // this, if the result is already smaller than the stopValue(Integer.MIN_VALUE / radix), then + // result * 10 will definitely be smaller than minValue, and we can stop + if (result < stopValue) { + return false; + } + + result = result * radix - digit; + // Since the previous result is less than or equal to stopValue(Integer.MIN_VALUE / radix), + // we can just use `result > 0` to check overflow. If result overflows, we should stop + if (result > 0) { + return false; + } + } + + // This is the case when we've encountered a decimal separator. The fractional + // part will not change the number, but we will verify that the fractional part + // is well formed. + while (offset < numBytes) { + byte currentByte = getByte(offset); + if (currentByte < '0' || currentByte > '9') { + return false; + } + offset++; + } + + if (!negative) { + result = -result; + if (result < 0) { + return false; + } + } + intWrapper.value = result; + return true; + } + + public boolean toShort(IntWrapper intWrapper) { + if (toInt(intWrapper)) { + int intValue = intWrapper.value; + short result = (short) intValue; + if (result == intValue) { + return true; + } + } + return false; + } + + public boolean toByte(IntWrapper intWrapper) { + if (toInt(intWrapper)) { + int intValue = intWrapper.value; + byte result = (byte) intValue; + if (result == intValue) { + return true; + } + } + return false; + } + + @Override + public String toString() { + return new String(getBytes(), StandardCharsets.UTF_8); + } + + @Override + public UTF8String clone() { + return fromBytes(getBytes()); + } + + public UTF8String copy() { + byte[] bytes = new byte[numBytes]; + copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); + return fromBytes(bytes); + } + + @Override + public int compareTo(@Nonnull final UTF8String other) { + int len = Math.min(numBytes, other.numBytes); + int wordMax = (len / 8) * 8; + long roffset = other.offset; + Object rbase = other.base; + for (int i = 0; i < wordMax; i += 8) { + long left = getLong(base, offset + i); + long right = getLong(rbase, roffset + i); + if (left != right) { + if (IS_LITTLE_ENDIAN) { + return Long.compareUnsigned(Long.reverseBytes(left), Long.reverseBytes(right)); + } else { + return Long.compareUnsigned(left, right); + } + } + } + for (int i = wordMax; i < len; i++) { + // In UTF-8, the byte should be unsigned, so we should compare them as unsigned int. + int res = (getByte(i) & 0xFF) - (Platform.getByte(rbase, roffset + i) & 0xFF); + if (res != 0) { + return res; + } + } + return numBytes - other.numBytes; + } + + public int compare(final UTF8String other) { + return compareTo(other); + } + + @Override + public boolean equals(final Object other) { + if (other instanceof UTF8String) { + UTF8String o = (UTF8String) other; + if (numBytes != o.numBytes) { + return false; + } + return ByteArrayMethods.arrayEquals(base, offset, o.base, o.offset, numBytes); + } else { + return false; + } + } + + /** + * Levenshtein distance is a metric for measuring the distance of two strings. The distance is + * defined by the minimum number of single-character edits (i.e. insertions, deletions or + * substitutions) that are required to change one of the strings into the other. + */ + public int levenshteinDistance(UTF8String other) { + // Implementation adopted from org.apache.common.lang3.StringUtils.getLevenshteinDistance + + int n = numChars(); + int m = other.numChars(); + + if (n == 0) { + return m; + } else if (m == 0) { + return n; + } + + UTF8String s, t; + + if (n <= m) { + s = this; + t = other; + } else { + s = other; + t = this; + int swap; + swap = n; + n = m; + m = swap; + } + + int[] p = new int[n + 1]; + int[] d = new int[n + 1]; + int[] swap; + + int i, i_bytes, j, j_bytes, num_bytes_j, cost; + + for (i = 0; i <= n; i++) { + p[i] = i; + } + + for (j = 0, j_bytes = 0; j < m; j_bytes += num_bytes_j, j++) { + num_bytes_j = numBytesForFirstByte(t.getByte(j_bytes)); + d[0] = j + 1; + + for (i = 0, i_bytes = 0; i < n; i_bytes += numBytesForFirstByte(s.getByte(i_bytes)), i++) { + if (s.getByte(i_bytes) != t.getByte(j_bytes) || + num_bytes_j != numBytesForFirstByte(s.getByte(i_bytes))) { + cost = 1; + } else { + cost = (ByteArrayMethods.arrayEquals(t.base, t.offset + j_bytes, s.base, + s.offset + i_bytes, num_bytes_j)) ? 0 : 1; + } + d[i + 1] = Math.min(Math.min(d[i] + 1, p[i + 1] + 1), p[i] + cost); + } + + swap = p; + p = d; + d = swap; + } + + return p[n]; + } + + @Override + public int hashCode() { + return Murmur3_x86_32.hashUnsafeBytes(base, offset, numBytes, 42); + } + + /** + * Soundex mapping table + */ + private static final byte[] US_ENGLISH_MAPPING = {'0', '1', '2', '3', '0', '1', '2', '7', + '0', '2', '2', '4', '5', '5', '0', '1', '2', '6', '2', '3', '0', '1', '7', '2', '0', '2'}; + + /** + * Encodes a string into a Soundex value. Soundex is an encoding used to relate similar names, + * but can also be used as a general purpose scheme to find word with similar phonemes. + * https://en.wikipedia.org/wiki/Soundex + */ + public UTF8String soundex() { + if (numBytes == 0) { + return EMPTY_UTF8; + } + + byte b = getByte(0); + if ('a' <= b && b <= 'z') { + b -= 32; + } else if (b < 'A' || 'Z' < b) { + // first character must be a letter + return this; + } + byte[] sx = {'0', '0', '0', '0'}; + sx[0] = b; + int sxi = 1; + int idx = b - 'A'; + byte lastCode = US_ENGLISH_MAPPING[idx]; + + for (int i = 1; i < numBytes; i++) { + b = getByte(i); + if ('a' <= b && b <= 'z') { + b -= 32; + } else if (b < 'A' || 'Z' < b) { + // not a letter, skip it + lastCode = '0'; + continue; + } + idx = b - 'A'; + byte code = US_ENGLISH_MAPPING[idx]; + if (code == '7') { + // ignore it + } else { + if (code != '0' && code != lastCode) { + sx[sxi++] = code; + if (sxi > 3) break; + } + lastCode = code; + } + } + return UTF8String.fromBytes(sx); + } + + public void writeExternal(ObjectOutput out) throws IOException { + byte[] bytes = getBytes(); + out.writeInt(bytes.length); + out.write(bytes); + } + + public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + offset = BYTE_ARRAY_OFFSET; + numBytes = in.readInt(); + base = new byte[numBytes]; + in.readFully((byte[]) base); + } + + @Override + public void write(Kryo kryo, Output out) { + byte[] bytes = getBytes(); + out.writeInt(bytes.length); + out.write(bytes); + } + + @Override + public void read(Kryo kryo, Input in) { + this.offset = BYTE_ARRAY_OFFSET; + this.numBytes = in.readInt(); + this.base = new byte[numBytes]; + in.read((byte[]) base); + } + +} diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java new file mode 100644 index 000000000000..4ae49d82efa2 --- /dev/null +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.unsafe; + +import org.apache.spark.unsafe.memory.MemoryAllocator; +import org.apache.spark.unsafe.memory.MemoryBlock; + +import org.junit.Assert; +import org.junit.Test; + +public class PlatformUtilSuite { + + @Test + public void overlappingCopyMemory() { + byte[] data = new byte[3 * 1024 * 1024]; + int size = 2 * 1024 * 1024; + for (int i = 0; i < data.length; ++i) { + data[i] = (byte)i; + } + + Platform.copyMemory(data, Platform.BYTE_ARRAY_OFFSET, data, Platform.BYTE_ARRAY_OFFSET, size); + for (int i = 0; i < data.length; ++i) { + Assert.assertEquals((byte)i, data[i]); + } + + Platform.copyMemory( + data, + Platform.BYTE_ARRAY_OFFSET + 1, + data, + Platform.BYTE_ARRAY_OFFSET, + size); + for (int i = 0; i < size; ++i) { + Assert.assertEquals((byte)(i + 1), data[i]); + } + + for (int i = 0; i < data.length; ++i) { + data[i] = (byte)i; + } + Platform.copyMemory( + data, + Platform.BYTE_ARRAY_OFFSET, + data, + Platform.BYTE_ARRAY_OFFSET + 1, + size); + for (int i = 0; i < size; ++i) { + Assert.assertEquals((byte)i, data[i + 1]); + } + } + + @Test + public void memoryDebugFillEnabledInTest() { + Assert.assertTrue(MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED); + MemoryBlock onheap = MemoryAllocator.HEAP.allocate(1); + MemoryBlock offheap = MemoryAllocator.UNSAFE.allocate(1); + Assert.assertEquals( + Platform.getByte(onheap.getBaseObject(), onheap.getBaseOffset()), + MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); + Assert.assertEquals( + Platform.getByte(offheap.getBaseObject(), offheap.getBaseOffset()), + MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); + MemoryAllocator.UNSAFE.free(offheap); + } +} diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java similarity index 92% rename from unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java rename to common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java index 5974cf91ff99..fb8e53b3348f 100644 --- a/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java @@ -34,5 +34,9 @@ public void basicTest() { Assert.assertEquals(2, arr.size()); Assert.assertEquals(1L, arr.get(0)); Assert.assertEquals(3L, arr.get(1)); + + arr.zeroOut(); + Assert.assertEquals(0L, arr.get(0)); + Assert.assertEquals(0L, arr.get(1)); } } diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java similarity index 100% rename from unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java rename to common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java similarity index 100% rename from unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java rename to common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java new file mode 100644 index 000000000000..7b03d2c650fc --- /dev/null +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -0,0 +1,791 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.unsafe.types; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.charset.StandardCharsets; +import java.util.*; + +import com.google.common.collect.ImmutableMap; +import org.apache.spark.unsafe.Platform; +import org.junit.Test; + +import static org.junit.Assert.*; + +import static org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET; +import static org.apache.spark.unsafe.types.UTF8String.*; + +public class UTF8StringSuite { + + private static void checkBasic(String str, int len) { + UTF8String s1 = fromString(str); + UTF8String s2 = fromBytes(str.getBytes(StandardCharsets.UTF_8)); + assertEquals(s1.numChars(), len); + assertEquals(s2.numChars(), len); + + assertEquals(s1.toString(), str); + assertEquals(s2.toString(), str); + assertEquals(s1, s2); + + assertEquals(s1.hashCode(), s2.hashCode()); + + assertEquals(0, s1.compareTo(s2)); + + assertTrue(s1.contains(s2)); + assertTrue(s2.contains(s1)); + assertTrue(s1.startsWith(s1)); + assertTrue(s1.endsWith(s1)); + } + + @Test + public void basicTest() { + checkBasic("", 0); + checkBasic("hello", 5); + checkBasic("大 千 世 界", 7); + } + + @Test + public void emptyStringTest() { + assertEquals(EMPTY_UTF8, fromString("")); + assertEquals(EMPTY_UTF8, fromBytes(new byte[0])); + assertEquals(0, EMPTY_UTF8.numChars()); + assertEquals(0, EMPTY_UTF8.numBytes()); + } + + @Test + public void prefix() { + assertTrue(fromString("a").getPrefix() - fromString("b").getPrefix() < 0); + assertTrue(fromString("ab").getPrefix() - fromString("b").getPrefix() < 0); + assertTrue( + fromString("abbbbbbbbbbbasdf").getPrefix() - fromString("bbbbbbbbbbbbasdf").getPrefix() < 0); + assertTrue(fromString("").getPrefix() - fromString("a").getPrefix() < 0); + assertTrue(fromString("你好").getPrefix() - fromString("世界").getPrefix() > 0); + + byte[] buf1 = {1, 2, 3, 4, 5, 6, 7, 8, 9}; + byte[] buf2 = {1, 2, 3}; + UTF8String str1 = fromBytes(buf1, 0, 3); + UTF8String str2 = fromBytes(buf1, 0, 8); + UTF8String str3 = fromBytes(buf2); + assertTrue(str1.getPrefix() - str2.getPrefix() < 0); + assertEquals(str1.getPrefix(), str3.getPrefix()); + } + + @Test + public void compareTo() { + assertTrue(fromString("").compareTo(fromString("a")) < 0); + assertTrue(fromString("abc").compareTo(fromString("ABC")) > 0); + assertTrue(fromString("abc0").compareTo(fromString("abc")) > 0); + assertTrue(fromString("abcabcabc").compareTo(fromString("abcabcabc")) == 0); + assertTrue(fromString("aBcabcabc").compareTo(fromString("Abcabcabc")) > 0); + assertTrue(fromString("Abcabcabc").compareTo(fromString("abcabcabC")) < 0); + assertTrue(fromString("abcabcabc").compareTo(fromString("abcabcabC")) > 0); + + assertTrue(fromString("abc").compareTo(fromString("世界")) < 0); + assertTrue(fromString("你好").compareTo(fromString("世界")) > 0); + assertTrue(fromString("你好123").compareTo(fromString("你好122")) > 0); + } + + protected static void testUpperandLower(String upper, String lower) { + UTF8String us = fromString(upper); + UTF8String ls = fromString(lower); + assertEquals(ls, us.toLowerCase()); + assertEquals(us, ls.toUpperCase()); + assertEquals(us, us.toUpperCase()); + assertEquals(ls, ls.toLowerCase()); + } + + @Test + public void upperAndLower() { + testUpperandLower("", ""); + testUpperandLower("0123456", "0123456"); + testUpperandLower("ABCXYZ", "abcxyz"); + testUpperandLower("ЀЁЂѺΏỀ", "ѐёђѻώề"); + testUpperandLower("大千世界 数据砖头", "大千世界 数据砖头"); + } + + @Test + public void titleCase() { + assertEquals(fromString(""), fromString("").toTitleCase()); + assertEquals(fromString("Ab Bc Cd"), fromString("ab bc cd").toTitleCase()); + assertEquals(fromString("Ѐ Ё Ђ Ѻ Ώ Ề"), fromString("ѐ ё ђ ѻ ώ ề").toTitleCase()); + assertEquals(fromString("大千世界 数据砖头"), fromString("大千世界 数据砖头").toTitleCase()); + } + + @Test + public void concatTest() { + assertEquals(EMPTY_UTF8, concat()); + assertNull(concat((UTF8String) null)); + assertEquals(EMPTY_UTF8, concat(EMPTY_UTF8)); + assertEquals(fromString("ab"), concat(fromString("ab"))); + assertEquals(fromString("ab"), concat(fromString("a"), fromString("b"))); + assertEquals(fromString("abc"), concat(fromString("a"), fromString("b"), fromString("c"))); + assertNull(concat(fromString("a"), null, fromString("c"))); + assertNull(concat(fromString("a"), null, null)); + assertNull(concat(null, null, null)); + assertEquals(fromString("数据砖头"), concat(fromString("数据"), fromString("砖头"))); + } + + @Test + public void concatWsTest() { + // Returns null if the separator is null + assertNull(concatWs(null, (UTF8String) null)); + assertNull(concatWs(null, fromString("a"))); + + // If separator is null, concatWs should skip all null inputs and never return null. + UTF8String sep = fromString("哈哈"); + assertEquals( + EMPTY_UTF8, + concatWs(sep, EMPTY_UTF8)); + assertEquals( + fromString("ab"), + concatWs(sep, fromString("ab"))); + assertEquals( + fromString("a哈哈b"), + concatWs(sep, fromString("a"), fromString("b"))); + assertEquals( + fromString("a哈哈b哈哈c"), + concatWs(sep, fromString("a"), fromString("b"), fromString("c"))); + assertEquals( + fromString("a哈哈c"), + concatWs(sep, fromString("a"), null, fromString("c"))); + assertEquals( + fromString("a"), + concatWs(sep, fromString("a"), null, null)); + assertEquals( + EMPTY_UTF8, + concatWs(sep, null, null, null)); + assertEquals( + fromString("数据哈哈砖头"), + concatWs(sep, fromString("数据"), fromString("砖头"))); + } + + @Test + public void contains() { + assertTrue(EMPTY_UTF8.contains(EMPTY_UTF8)); + assertTrue(fromString("hello").contains(fromString("ello"))); + assertFalse(fromString("hello").contains(fromString("vello"))); + assertFalse(fromString("hello").contains(fromString("hellooo"))); + assertTrue(fromString("大千世界").contains(fromString("千世界"))); + assertFalse(fromString("大千世界").contains(fromString("世千"))); + assertFalse(fromString("大千世界").contains(fromString("大千世界好"))); + } + + @Test + public void startsWith() { + assertTrue(EMPTY_UTF8.startsWith(EMPTY_UTF8)); + assertTrue(fromString("hello").startsWith(fromString("hell"))); + assertFalse(fromString("hello").startsWith(fromString("ell"))); + assertFalse(fromString("hello").startsWith(fromString("hellooo"))); + assertTrue(fromString("数据砖头").startsWith(fromString("数据"))); + assertFalse(fromString("大千世界").startsWith(fromString("千"))); + assertFalse(fromString("大千世界").startsWith(fromString("大千世界好"))); + } + + @Test + public void endsWith() { + assertTrue(EMPTY_UTF8.endsWith(EMPTY_UTF8)); + assertTrue(fromString("hello").endsWith(fromString("ello"))); + assertFalse(fromString("hello").endsWith(fromString("ellov"))); + assertFalse(fromString("hello").endsWith(fromString("hhhello"))); + assertTrue(fromString("大千世界").endsWith(fromString("世界"))); + assertFalse(fromString("大千世界").endsWith(fromString("世"))); + assertFalse(fromString("数据砖头").endsWith(fromString("我的数据砖头"))); + } + + @Test + public void substring() { + assertEquals(EMPTY_UTF8, fromString("hello").substring(0, 0)); + assertEquals(fromString("el"), fromString("hello").substring(1, 3)); + assertEquals(fromString("数"), fromString("数据砖头").substring(0, 1)); + assertEquals(fromString("据砖"), fromString("数据砖头").substring(1, 3)); + assertEquals(fromString("头"), fromString("数据砖头").substring(3, 5)); + assertEquals(fromString("ߵ梷"), fromString("ߵ梷").substring(0, 2)); + } + + @Test + public void trims() { + assertEquals(fromString("hello"), fromString(" hello ").trim()); + assertEquals(fromString("hello "), fromString(" hello ").trimLeft()); + assertEquals(fromString(" hello"), fromString(" hello ").trimRight()); + + assertEquals(EMPTY_UTF8, fromString(" ").trim()); + assertEquals(EMPTY_UTF8, fromString(" ").trimLeft()); + assertEquals(EMPTY_UTF8, fromString(" ").trimRight()); + + assertEquals(fromString("数据砖头"), fromString(" 数据砖头 ").trim()); + assertEquals(fromString("数据砖头 "), fromString(" 数据砖头 ").trimLeft()); + assertEquals(fromString(" 数据砖头"), fromString(" 数据砖头 ").trimRight()); + + assertEquals(fromString("数据砖头"), fromString("数据砖头").trim()); + assertEquals(fromString("数据砖头"), fromString("数据砖头").trimLeft()); + assertEquals(fromString("数据砖头"), fromString("数据砖头").trimRight()); + + char[] charsLessThan0x20 = new char[10]; + Arrays.fill(charsLessThan0x20, (char)(' ' - 1)); + String stringStartingWithSpace = + new String(charsLessThan0x20) + "hello" + new String(charsLessThan0x20); + assertEquals(fromString(stringStartingWithSpace), fromString(stringStartingWithSpace).trim()); + assertEquals(fromString(stringStartingWithSpace), + fromString(stringStartingWithSpace).trimLeft()); + assertEquals(fromString(stringStartingWithSpace), + fromString(stringStartingWithSpace).trimRight()); + } + + @Test + public void indexOf() { + assertEquals(0, EMPTY_UTF8.indexOf(EMPTY_UTF8, 0)); + assertEquals(-1, EMPTY_UTF8.indexOf(fromString("l"), 0)); + assertEquals(0, fromString("hello").indexOf(EMPTY_UTF8, 0)); + assertEquals(2, fromString("hello").indexOf(fromString("l"), 0)); + assertEquals(3, fromString("hello").indexOf(fromString("l"), 3)); + assertEquals(-1, fromString("hello").indexOf(fromString("a"), 0)); + assertEquals(2, fromString("hello").indexOf(fromString("ll"), 0)); + assertEquals(-1, fromString("hello").indexOf(fromString("ll"), 4)); + assertEquals(1, fromString("数据砖头").indexOf(fromString("据砖"), 0)); + assertEquals(-1, fromString("数据砖头").indexOf(fromString("数"), 3)); + assertEquals(0, fromString("数据砖头").indexOf(fromString("数"), 0)); + assertEquals(3, fromString("数据砖头").indexOf(fromString("头"), 0)); + } + + @Test + public void substring_index() { + assertEquals(fromString("www.apache.org"), + fromString("www.apache.org").subStringIndex(fromString("."), 3)); + assertEquals(fromString("www.apache"), + fromString("www.apache.org").subStringIndex(fromString("."), 2)); + assertEquals(fromString("www"), + fromString("www.apache.org").subStringIndex(fromString("."), 1)); + assertEquals(fromString(""), + fromString("www.apache.org").subStringIndex(fromString("."), 0)); + assertEquals(fromString("org"), + fromString("www.apache.org").subStringIndex(fromString("."), -1)); + assertEquals(fromString("apache.org"), + fromString("www.apache.org").subStringIndex(fromString("."), -2)); + assertEquals(fromString("www.apache.org"), + fromString("www.apache.org").subStringIndex(fromString("."), -3)); + // str is empty string + assertEquals(fromString(""), + fromString("").subStringIndex(fromString("."), 1)); + // empty string delim + assertEquals(fromString(""), + fromString("www.apache.org").subStringIndex(fromString(""), 1)); + // delim does not exist in str + assertEquals(fromString("www.apache.org"), + fromString("www.apache.org").subStringIndex(fromString("#"), 2)); + // delim is 2 chars + assertEquals(fromString("www||apache"), + fromString("www||apache||org").subStringIndex(fromString("||"), 2)); + assertEquals(fromString("apache||org"), + fromString("www||apache||org").subStringIndex(fromString("||"), -2)); + // non ascii chars + assertEquals(fromString("大千世界大"), + fromString("大千世界大千世界").subStringIndex(fromString("千"), 2)); + // overlapped delim + assertEquals(fromString("||"), fromString("||||||").subStringIndex(fromString("|||"), 3)); + assertEquals(fromString("|||"), fromString("||||||").subStringIndex(fromString("|||"), -4)); + } + + @Test + public void reverse() { + assertEquals(fromString("olleh"), fromString("hello").reverse()); + assertEquals(EMPTY_UTF8, EMPTY_UTF8.reverse()); + assertEquals(fromString("者行孙"), fromString("孙行者").reverse()); + assertEquals(fromString("者行孙 olleh"), fromString("hello 孙行者").reverse()); + } + + @Test + public void repeat() { + assertEquals(fromString("数d数d数d数d数d"), fromString("数d").repeat(5)); + assertEquals(fromString("数d"), fromString("数d").repeat(1)); + assertEquals(EMPTY_UTF8, fromString("数d").repeat(-1)); + } + + @Test + public void pad() { + assertEquals(fromString("hel"), fromString("hello").lpad(3, fromString("????"))); + assertEquals(fromString("hello"), fromString("hello").lpad(5, fromString("????"))); + assertEquals(fromString("?hello"), fromString("hello").lpad(6, fromString("????"))); + assertEquals(fromString("???????hello"), fromString("hello").lpad(12, fromString("????"))); + assertEquals(fromString("?????hello"), fromString("hello").lpad(10, fromString("?????"))); + assertEquals(fromString("???????"), EMPTY_UTF8.lpad(7, fromString("?????"))); + + assertEquals(fromString("hel"), fromString("hello").rpad(3, fromString("????"))); + assertEquals(fromString("hello"), fromString("hello").rpad(5, fromString("????"))); + assertEquals(fromString("hello?"), fromString("hello").rpad(6, fromString("????"))); + assertEquals(fromString("hello???????"), fromString("hello").rpad(12, fromString("????"))); + assertEquals(fromString("hello?????"), fromString("hello").rpad(10, fromString("?????"))); + assertEquals(fromString("???????"), EMPTY_UTF8.rpad(7, fromString("?????"))); + + assertEquals(fromString("数据砖"), fromString("数据砖头").lpad(3, fromString("????"))); + assertEquals(fromString("?数据砖头"), fromString("数据砖头").lpad(5, fromString("????"))); + assertEquals(fromString("??数据砖头"), fromString("数据砖头").lpad(6, fromString("????"))); + assertEquals(fromString("孙行数据砖头"), fromString("数据砖头").lpad(6, fromString("孙行者"))); + assertEquals(fromString("孙行者数据砖头"), fromString("数据砖头").lpad(7, fromString("孙行者"))); + assertEquals( + fromString("孙行者孙行者孙行数据砖头"), + fromString("数据砖头").lpad(12, fromString("孙行者"))); + + assertEquals(fromString("数据砖"), fromString("数据砖头").rpad(3, fromString("????"))); + assertEquals(fromString("数据砖头?"), fromString("数据砖头").rpad(5, fromString("????"))); + assertEquals(fromString("数据砖头??"), fromString("数据砖头").rpad(6, fromString("????"))); + assertEquals(fromString("数据砖头孙行"), fromString("数据砖头").rpad(6, fromString("孙行者"))); + assertEquals(fromString("数据砖头孙行者"), fromString("数据砖头").rpad(7, fromString("孙行者"))); + assertEquals( + fromString("数据砖头孙行者孙行者孙行"), + fromString("数据砖头").rpad(12, fromString("孙行者"))); + + assertEquals(EMPTY_UTF8, fromString("数据砖头").lpad(-10, fromString("孙行者"))); + assertEquals(EMPTY_UTF8, fromString("数据砖头").lpad(-10, EMPTY_UTF8)); + assertEquals(fromString("数据砖头"), fromString("数据砖头").lpad(5, EMPTY_UTF8)); + assertEquals(fromString("数据砖"), fromString("数据砖头").lpad(3, EMPTY_UTF8)); + assertEquals(EMPTY_UTF8, EMPTY_UTF8.lpad(3, EMPTY_UTF8)); + + assertEquals(EMPTY_UTF8, fromString("数据砖头").rpad(-10, fromString("孙行者"))); + assertEquals(EMPTY_UTF8, fromString("数据砖头").rpad(-10, EMPTY_UTF8)); + assertEquals(fromString("数据砖头"), fromString("数据砖头").rpad(5, EMPTY_UTF8)); + assertEquals(fromString("数据砖"), fromString("数据砖头").rpad(3, EMPTY_UTF8)); + assertEquals(EMPTY_UTF8, EMPTY_UTF8.rpad(3, EMPTY_UTF8)); + } + + @Test + public void substringSQL() { + UTF8String e = fromString("example"); + assertEquals(e.substringSQL(0, 2), fromString("ex")); + assertEquals(e.substringSQL(1, 2), fromString("ex")); + assertEquals(e.substringSQL(0, 7), fromString("example")); + assertEquals(e.substringSQL(1, 2), fromString("ex")); + assertEquals(e.substringSQL(0, 100), fromString("example")); + assertEquals(e.substringSQL(1, 100), fromString("example")); + assertEquals(e.substringSQL(2, 2), fromString("xa")); + assertEquals(e.substringSQL(1, 6), fromString("exampl")); + assertEquals(e.substringSQL(2, 100), fromString("xample")); + assertEquals(e.substringSQL(0, 0), fromString("")); + assertEquals(e.substringSQL(100, 4), EMPTY_UTF8); + assertEquals(e.substringSQL(0, Integer.MAX_VALUE), fromString("example")); + assertEquals(e.substringSQL(1, Integer.MAX_VALUE), fromString("example")); + assertEquals(e.substringSQL(2, Integer.MAX_VALUE), fromString("xample")); + } + + @Test + public void split() { + assertTrue(Arrays.equals(fromString("ab,def,ghi").split(fromString(","), -1), + new UTF8String[]{fromString("ab"), fromString("def"), fromString("ghi")})); + assertTrue(Arrays.equals(fromString("ab,def,ghi").split(fromString(","), 2), + new UTF8String[]{fromString("ab"), fromString("def,ghi")})); + assertTrue(Arrays.equals(fromString("ab,def,ghi").split(fromString(","), 2), + new UTF8String[]{fromString("ab"), fromString("def,ghi")})); + } + + @Test + public void levenshteinDistance() { + assertEquals(0, EMPTY_UTF8.levenshteinDistance(EMPTY_UTF8)); + assertEquals(1, EMPTY_UTF8.levenshteinDistance(fromString("a"))); + assertEquals(7, fromString("aaapppp").levenshteinDistance(EMPTY_UTF8)); + assertEquals(1, fromString("frog").levenshteinDistance(fromString("fog"))); + assertEquals(3, fromString("fly").levenshteinDistance(fromString("ant"))); + assertEquals(7, fromString("elephant").levenshteinDistance(fromString("hippo"))); + assertEquals(7, fromString("hippo").levenshteinDistance(fromString("elephant"))); + assertEquals(8, fromString("hippo").levenshteinDistance(fromString("zzzzzzzz"))); + assertEquals(1, fromString("hello").levenshteinDistance(fromString("hallo"))); + assertEquals(4, fromString("世界千世").levenshteinDistance(fromString("千a世b"))); + } + + @Test + public void translate() { + assertEquals( + fromString("1a2s3ae"), + fromString("translate").translate(ImmutableMap.of( + 'r', '1', + 'n', '2', + 'l', '3', + 't', '\0' + ))); + assertEquals( + fromString("translate"), + fromString("translate").translate(new HashMap())); + assertEquals( + fromString("asae"), + fromString("translate").translate(ImmutableMap.of( + 'r', '\0', + 'n', '\0', + 'l', '\0', + 't', '\0' + ))); + assertEquals( + fromString("aa世b"), + fromString("花花世界").translate(ImmutableMap.of( + '花', 'a', + '界', 'b' + ))); + } + + @Test + public void createBlankString() { + assertEquals(fromString(" "), blankString(1)); + assertEquals(fromString(" "), blankString(2)); + assertEquals(fromString(" "), blankString(3)); + assertEquals(fromString(""), blankString(0)); + } + + @Test + public void findInSet() { + assertEquals(1, fromString("ab").findInSet(fromString("ab"))); + assertEquals(2, fromString("a,b").findInSet(fromString("b"))); + assertEquals(3, fromString("abc,b,ab,c,def").findInSet(fromString("ab"))); + assertEquals(1, fromString("ab,abc,b,ab,c,def").findInSet(fromString("ab"))); + assertEquals(4, fromString(",,,ab,abc,b,ab,c,def").findInSet(fromString("ab"))); + assertEquals(1, fromString(",ab,abc,b,ab,c,def").findInSet(fromString(""))); + assertEquals(4, fromString("数据砖头,abc,b,ab,c,def").findInSet(fromString("ab"))); + assertEquals(6, fromString("数据砖头,abc,b,ab,c,def").findInSet(fromString("def"))); + } + + @Test + public void soundex() { + assertEquals(fromString("Robert").soundex(), fromString("R163")); + assertEquals(fromString("Rupert").soundex(), fromString("R163")); + assertEquals(fromString("Rubin").soundex(), fromString("R150")); + assertEquals(fromString("Ashcraft").soundex(), fromString("A261")); + assertEquals(fromString("Ashcroft").soundex(), fromString("A261")); + assertEquals(fromString("Burroughs").soundex(), fromString("B620")); + assertEquals(fromString("Burrows").soundex(), fromString("B620")); + assertEquals(fromString("Ekzampul").soundex(), fromString("E251")); + assertEquals(fromString("Example").soundex(), fromString("E251")); + assertEquals(fromString("Ellery").soundex(), fromString("E460")); + assertEquals(fromString("Euler").soundex(), fromString("E460")); + assertEquals(fromString("Ghosh").soundex(), fromString("G200")); + assertEquals(fromString("Gauss").soundex(), fromString("G200")); + assertEquals(fromString("Gutierrez").soundex(), fromString("G362")); + assertEquals(fromString("Heilbronn").soundex(), fromString("H416")); + assertEquals(fromString("Hilbert").soundex(), fromString("H416")); + assertEquals(fromString("Jackson").soundex(), fromString("J250")); + assertEquals(fromString("Kant").soundex(), fromString("K530")); + assertEquals(fromString("Knuth").soundex(), fromString("K530")); + assertEquals(fromString("Lee").soundex(), fromString("L000")); + assertEquals(fromString("Lukasiewicz").soundex(), fromString("L222")); + assertEquals(fromString("Lissajous").soundex(), fromString("L222")); + assertEquals(fromString("Ladd").soundex(), fromString("L300")); + assertEquals(fromString("Lloyd").soundex(), fromString("L300")); + assertEquals(fromString("Moses").soundex(), fromString("M220")); + assertEquals(fromString("O'Hara").soundex(), fromString("O600")); + assertEquals(fromString("Pfister").soundex(), fromString("P236")); + assertEquals(fromString("Rubin").soundex(), fromString("R150")); + assertEquals(fromString("Robert").soundex(), fromString("R163")); + assertEquals(fromString("Rupert").soundex(), fromString("R163")); + assertEquals(fromString("Soundex").soundex(), fromString("S532")); + assertEquals(fromString("Sownteks").soundex(), fromString("S532")); + assertEquals(fromString("Tymczak").soundex(), fromString("T522")); + assertEquals(fromString("VanDeusen").soundex(), fromString("V532")); + assertEquals(fromString("Washington").soundex(), fromString("W252")); + assertEquals(fromString("Wheaton").soundex(), fromString("W350")); + + assertEquals(fromString("a").soundex(), fromString("A000")); + assertEquals(fromString("ab").soundex(), fromString("A100")); + assertEquals(fromString("abc").soundex(), fromString("A120")); + assertEquals(fromString("abcd").soundex(), fromString("A123")); + assertEquals(fromString("").soundex(), fromString("")); + assertEquals(fromString("123").soundex(), fromString("123")); + assertEquals(fromString("世界千世").soundex(), fromString("世界千世")); + } + + @Test + public void writeToOutputStreamUnderflow() throws IOException { + // offset underflow is apparently supported? + final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + final byte[] test = "01234567".getBytes(StandardCharsets.UTF_8); + + for (int i = 1; i <= Platform.BYTE_ARRAY_OFFSET; ++i) { + UTF8String.fromAddress(test, Platform.BYTE_ARRAY_OFFSET - i, test.length + i) + .writeTo(outputStream); + final ByteBuffer buffer = ByteBuffer.wrap(outputStream.toByteArray(), i, test.length); + assertEquals("01234567", StandardCharsets.UTF_8.decode(buffer).toString()); + outputStream.reset(); + } + } + + @Test + public void writeToOutputStreamSlice() throws IOException { + final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + final byte[] test = "01234567".getBytes(StandardCharsets.UTF_8); + + for (int i = 0; i < test.length; ++i) { + for (int j = 0; j < test.length - i; ++j) { + UTF8String.fromAddress(test, Platform.BYTE_ARRAY_OFFSET + i, j) + .writeTo(outputStream); + + assertArrayEquals(Arrays.copyOfRange(test, i, i + j), outputStream.toByteArray()); + outputStream.reset(); + } + } + } + + @Test + public void writeToOutputStreamOverflow() throws IOException { + final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + final byte[] test = "01234567".getBytes(StandardCharsets.UTF_8); + + final HashSet offsets = new HashSet<>(); + for (int i = 0; i < 16; ++i) { + // touch more points around MAX_VALUE + offsets.add((long) Integer.MAX_VALUE - i); + // subtract off BYTE_ARRAY_OFFSET to avoid wrapping around to a negative value, + // which will hit the slower copy path instead of the optimized one + offsets.add(Long.MAX_VALUE - BYTE_ARRAY_OFFSET - i); + } + + for (long i = 1; i > 0L; i <<= 1) { + for (long j = 0; j < 32L; ++j) { + offsets.add(i + j); + } + } + + for (final long offset : offsets) { + try { + fromAddress(test, BYTE_ARRAY_OFFSET + offset, test.length) + .writeTo(outputStream); + + throw new IllegalStateException(Long.toString(offset)); + } catch (ArrayIndexOutOfBoundsException e) { + // ignore + } finally { + outputStream.reset(); + } + } + } + + @Test + public void writeToOutputStream() throws IOException { + final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + EMPTY_UTF8.writeTo(outputStream); + assertEquals("", outputStream.toString("UTF-8")); + outputStream.reset(); + + fromString("数据砖很重").writeTo(outputStream); + assertEquals( + "数据砖很重", + outputStream.toString("UTF-8")); + outputStream.reset(); + } + + @Test + public void writeToOutputStreamIntArray() throws IOException { + // verify that writes work on objects that are not byte arrays + final ByteBuffer buffer = StandardCharsets.UTF_8.encode("大千世界"); + buffer.position(0); + buffer.order(ByteOrder.nativeOrder()); + + final int length = buffer.limit(); + assertEquals(12, length); + + final int ints = length / 4; + final int[] array = new int[ints]; + + for (int i = 0; i < ints; ++i) { + array[i] = buffer.getInt(); + } + + final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + fromAddress(array, Platform.INT_ARRAY_OFFSET, length) + .writeTo(outputStream); + assertEquals("大千世界", outputStream.toString("UTF-8")); + } + + @Test + public void testToShort() throws IOException { + Map inputToExpectedOutput = new HashMap<>(); + inputToExpectedOutput.put("1", (short) 1); + inputToExpectedOutput.put("+1", (short) 1); + inputToExpectedOutput.put("-1", (short) -1); + inputToExpectedOutput.put("0", (short) 0); + inputToExpectedOutput.put("1111.12345678901234567890", (short) 1111); + inputToExpectedOutput.put(String.valueOf(Short.MAX_VALUE), Short.MAX_VALUE); + inputToExpectedOutput.put(String.valueOf(Short.MIN_VALUE), Short.MIN_VALUE); + + Random rand = new Random(); + for (int i = 0; i < 10; i++) { + short value = (short) rand.nextInt(); + inputToExpectedOutput.put(String.valueOf(value), value); + } + + IntWrapper wrapper = new IntWrapper(); + for (Map.Entry entry : inputToExpectedOutput.entrySet()) { + assertTrue(entry.getKey(), UTF8String.fromString(entry.getKey()).toShort(wrapper)); + assertEquals((short) entry.getValue(), wrapper.value); + } + + List negativeInputs = + Arrays.asList("", " ", "null", "NULL", "\n", "~1212121", "3276700"); + + for (String negativeInput : negativeInputs) { + assertFalse(negativeInput, UTF8String.fromString(negativeInput).toShort(wrapper)); + } + } + + @Test + public void testToByte() throws IOException { + Map inputToExpectedOutput = new HashMap<>(); + inputToExpectedOutput.put("1", (byte) 1); + inputToExpectedOutput.put("+1",(byte) 1); + inputToExpectedOutput.put("-1", (byte) -1); + inputToExpectedOutput.put("0", (byte) 0); + inputToExpectedOutput.put("111.12345678901234567890", (byte) 111); + inputToExpectedOutput.put(String.valueOf(Byte.MAX_VALUE), Byte.MAX_VALUE); + inputToExpectedOutput.put(String.valueOf(Byte.MIN_VALUE), Byte.MIN_VALUE); + + Random rand = new Random(); + for (int i = 0; i < 10; i++) { + byte value = (byte) rand.nextInt(); + inputToExpectedOutput.put(String.valueOf(value), value); + } + + IntWrapper intWrapper = new IntWrapper(); + for (Map.Entry entry : inputToExpectedOutput.entrySet()) { + assertTrue(entry.getKey(), UTF8String.fromString(entry.getKey()).toByte(intWrapper)); + assertEquals((byte) entry.getValue(), intWrapper.value); + } + + List negativeInputs = + Arrays.asList("", " ", "null", "NULL", "\n", "~1212121", "12345678901234567890"); + + for (String negativeInput : negativeInputs) { + assertFalse(negativeInput, UTF8String.fromString(negativeInput).toByte(intWrapper)); + } + } + + @Test + public void testToInt() throws IOException { + Map inputToExpectedOutput = new HashMap<>(); + inputToExpectedOutput.put("1", 1); + inputToExpectedOutput.put("+1", 1); + inputToExpectedOutput.put("-1", -1); + inputToExpectedOutput.put("0", 0); + inputToExpectedOutput.put("11111.1234567", 11111); + inputToExpectedOutput.put(String.valueOf(Integer.MAX_VALUE), Integer.MAX_VALUE); + inputToExpectedOutput.put(String.valueOf(Integer.MIN_VALUE), Integer.MIN_VALUE); + + Random rand = new Random(); + for (int i = 0; i < 10; i++) { + int value = rand.nextInt(); + inputToExpectedOutput.put(String.valueOf(value), value); + } + + IntWrapper intWrapper = new IntWrapper(); + for (Map.Entry entry : inputToExpectedOutput.entrySet()) { + assertTrue(entry.getKey(), UTF8String.fromString(entry.getKey()).toInt(intWrapper)); + assertEquals((int) entry.getValue(), intWrapper.value); + } + + List negativeInputs = + Arrays.asList("", " ", "null", "NULL", "\n", "~1212121", "12345678901234567890"); + + for (String negativeInput : negativeInputs) { + assertFalse(negativeInput, UTF8String.fromString(negativeInput).toInt(intWrapper)); + } + } + + @Test + public void testToLong() throws IOException { + Map inputToExpectedOutput = new HashMap<>(); + inputToExpectedOutput.put("1", 1L); + inputToExpectedOutput.put("+1", 1L); + inputToExpectedOutput.put("-1", -1L); + inputToExpectedOutput.put("0", 0L); + inputToExpectedOutput.put("1076753423.12345678901234567890", 1076753423L); + inputToExpectedOutput.put(String.valueOf(Long.MAX_VALUE), Long.MAX_VALUE); + inputToExpectedOutput.put(String.valueOf(Long.MIN_VALUE), Long.MIN_VALUE); + + Random rand = new Random(); + for (int i = 0; i < 10; i++) { + long value = rand.nextLong(); + inputToExpectedOutput.put(String.valueOf(value), value); + } + + LongWrapper wrapper = new LongWrapper(); + for (Map.Entry entry : inputToExpectedOutput.entrySet()) { + assertTrue(entry.getKey(), UTF8String.fromString(entry.getKey()).toLong(wrapper)); + assertEquals((long) entry.getValue(), wrapper.value); + } + + List negativeInputs = Arrays.asList("", " ", "null", "NULL", "\n", "~1212121", + "1234567890123456789012345678901234"); + + for (String negativeInput : negativeInputs) { + assertFalse(negativeInput, UTF8String.fromString(negativeInput).toLong(wrapper)); + } + } + + @Test + public void trimBothWithTrimString() { + assertEquals(fromString("hello"), fromString(" hello ").trim(fromString(" "))); + assertEquals(fromString("o"), fromString(" hello ").trim(fromString(" hle"))); + assertEquals(fromString("h e"), fromString("ooh e ooo").trim(fromString("o "))); + assertEquals(fromString(""), fromString("ooo...oooo").trim(fromString("o."))); + assertEquals(fromString("b"), fromString("%^b[]@").trim(fromString("][@^%"))); + + assertEquals(EMPTY_UTF8, fromString(" ").trim(fromString(" "))); + + assertEquals(fromString("数据砖头"), fromString(" 数据砖头 ").trim()); + assertEquals(fromString("数"), fromString("a数b").trim(fromString("ab"))); + assertEquals(fromString(""), fromString("a").trim(fromString("a数b"))); + assertEquals(fromString(""), fromString("数数 数数数").trim(fromString("数 "))); + assertEquals(fromString("据砖头"), fromString("数]数[数据砖头#数数").trim(fromString("[数]#"))); + assertEquals(fromString("据砖头数数 "), fromString("数数数据砖头数数 ").trim(fromString("数"))); + } + + @Test + public void trimLeftWithTrimString() { + assertEquals(fromString(" hello "), fromString(" hello ").trimLeft(fromString(""))); + assertEquals(fromString(""), fromString("a").trimLeft(fromString("a"))); + assertEquals(fromString("b"), fromString("b").trimLeft(fromString("a"))); + assertEquals(fromString("ba"), fromString("ba").trimLeft(fromString("a"))); + assertEquals(fromString(""), fromString("aaaaaaa").trimLeft(fromString("a"))); + assertEquals(fromString("trim"), fromString("oabtrim").trimLeft(fromString("bao"))); + assertEquals(fromString("rim "), fromString("ooootrim ").trimLeft(fromString("otm"))); + + assertEquals(EMPTY_UTF8, fromString(" ").trimLeft(fromString(" "))); + + assertEquals(fromString("数据砖头 "), fromString(" 数据砖头 ").trimLeft(fromString(" "))); + assertEquals(fromString("数"), fromString("数").trimLeft(fromString("a"))); + assertEquals(fromString("a"), fromString("a").trimLeft(fromString("数"))); + assertEquals(fromString("砖头数数"), fromString("数数数据砖头数数").trimLeft(fromString("据数"))); + assertEquals(fromString("据砖头数数"), fromString(" 数数数据砖头数数").trimLeft(fromString("数 "))); + assertEquals(fromString("据砖头数数"), fromString("aa数数数据砖头数数").trimLeft(fromString("a数砖"))); + assertEquals(fromString("$S,.$BR"), fromString(",,,,%$S,.$BR").trimLeft(fromString("%,"))); + } + + @Test + public void trimRightWithTrimString() { + assertEquals(fromString(" hello "), fromString(" hello ").trimRight(fromString(""))); + assertEquals(fromString(""), fromString("a").trimRight(fromString("a"))); + assertEquals(fromString("cc"), fromString("ccbaaaa").trimRight(fromString("ba"))); + assertEquals(fromString(""), fromString("aabbbbaaa").trimRight(fromString("ab"))); + assertEquals(fromString(" he"), fromString(" hello ").trimRight(fromString(" ol"))); + assertEquals(fromString("oohell"), + fromString("oohellooo../*&").trimRight(fromString("./,&%*o"))); + + assertEquals(EMPTY_UTF8, fromString(" ").trimRight(fromString(" "))); + + assertEquals(fromString(" 数据砖头"), fromString(" 数据砖头 ").trimRight(fromString(" "))); + assertEquals(fromString("数数砖头"), fromString("数数砖头数aa数").trimRight(fromString("a数"))); + assertEquals(fromString(""), fromString("数数数据砖ab").trimRight(fromString("数据砖ab"))); + assertEquals(fromString("头"), fromString("头a???/").trimRight(fromString("数?/*&^%a"))); + assertEquals(fromString("头"), fromString("头数b数数 [").trimRight(fromString(" []数b"))); + } +} diff --git a/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala similarity index 96% rename from unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala rename to common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala index 12a002befa0a..62d4176d00f9 100644 --- a/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala +++ b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.unsafe.types import org.apache.commons.lang3.StringUtils - import org.scalacheck.{Arbitrary, Gen} import org.scalatest.prop.GeneratorDrivenPropertyChecks // scalastyle:off @@ -99,7 +98,7 @@ class UTF8StringPropertyCheckSuite extends FunSuite with GeneratorDrivenProperty } } - val whitespaceChar: Gen[Char] = Gen.choose(0x00, 0x20).map(_.toChar) + val whitespaceChar: Gen[Char] = Gen.const(0x20.toChar) val whitespaceString: Gen[String] = Gen.listOf(whitespaceChar).map(_.mkString) val randomString: Gen[String] = Arbitrary.arbString.arbitrary @@ -108,7 +107,7 @@ class UTF8StringPropertyCheckSuite extends FunSuite with GeneratorDrivenProperty def lTrim(s: String): String = { var st = 0 val array: Array[Char] = s.toCharArray - while ((st < s.length) && (array(st) <= ' ')) { + while ((st < s.length) && (array(st) == ' ')) { st += 1 } if (st > 0) s.substring(st, s.length) else s @@ -116,7 +115,7 @@ class UTF8StringPropertyCheckSuite extends FunSuite with GeneratorDrivenProperty def rTrim(s: String): String = { var len = s.length val array: Array[Char] = s.toCharArray - while ((len > 0) && (array(len - 1) <= ' ')) { + while ((len > 0) && (array(len - 1) == ' ')) { len -= 1 } if (len < s.length) s.substring(0, len) else s @@ -128,7 +127,7 @@ class UTF8StringPropertyCheckSuite extends FunSuite with GeneratorDrivenProperty whitespaceString ) { (start: String, middle: String, end: String) => val s = start + middle + end - assert(toUTF8(s).trim() === toUTF8(s.trim())) + assert(toUTF8(s).trim() === toUTF8(rTrim(lTrim(s)))) assert(toUTF8(s).trimLeft() === toUTF8(lTrim(s))) assert(toUTF8(s).trimRight() === toUTF8(rTrim(s))) } @@ -194,7 +193,7 @@ class UTF8StringPropertyCheckSuite extends FunSuite with GeneratorDrivenProperty test("concat") { def concat(orgin: Seq[String]): String = - if (orgin.exists(_ == null)) null else orgin.mkString + if (orgin.contains(null)) null else orgin.mkString forAll { (inputs: Seq[String]) => assert(UTF8String.concat(inputs.map(toUTF8): _*) === toUTF8(inputs.mkString)) diff --git a/conf/docker.properties.template b/conf/docker.properties.template index 55cb094b4af4..2ecb4f1464a4 100644 --- a/conf/docker.properties.template +++ b/conf/docker.properties.template @@ -15,6 +15,6 @@ # limitations under the License. # -spark.mesos.executor.docker.image: +spark.mesos.executor.docker.image: spark.mesos.executor.docker.volumes: /usr/local/lib:/host/usr/local/lib:ro spark.mesos.executor.home: /opt/spark diff --git a/conf/log4j.properties.template b/conf/log4j.properties.template index f3046be54d7c..ec1aa187dfb3 100644 --- a/conf/log4j.properties.template +++ b/conf/log4j.properties.template @@ -22,9 +22,14 @@ log4j.appender.console.target=System.err log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n +# Set the default spark-shell log level to WARN. When running the spark-shell, the +# log level for this class is used to overwrite the root logger's log level, so that +# the user can have different defaults for the shell and regular Spark apps. +log4j.logger.org.apache.spark.repl.Main=WARN + # Settings to quiet third party logs that are too verbose -log4j.logger.org.spark-project.jetty=WARN -log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO log4j.logger.org.apache.parquet=ERROR diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index d6962e0da2f3..4c008a13607c 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -57,40 +57,43 @@ # added to Java properties using -Dspark.metrics.conf=xxx if you want to # customize metrics system. You can also put the file in ${SPARK_HOME}/conf # and it will be loaded automatically. -# 5. MetricsServlet is added by default as a sink in master, worker and client -# driver, you can send http request "/metrics/json" to get a snapshot of all the -# registered metrics in json format. For master, requests "/metrics/master/json" and -# "/metrics/applications/json" can be sent seperately to get metrics snapshot of -# instance master and applications. MetricsServlet may not be configured by self. -# +# 5. The MetricsServlet sink is added by default as a sink in the master, +# worker and driver, and you can send HTTP requests to the "/metrics/json" +# endpoint to get a snapshot of all the registered metrics in JSON format. +# For master, requests to the "/metrics/master/json" and +# "/metrics/applications/json" endpoints can be sent separately to get +# metrics snapshots of the master instance and applications. This +# MetricsServlet does not have to be configured. ## List of available common sources and their properties. # org.apache.spark.metrics.source.JvmSource -# Note: Currently, JvmSource is the only available common source -# to add additionaly to an instance, to enable this, -# set the "class" option to its fully qulified class name (see examples below) +# Note: Currently, JvmSource is the only available common source. +# It can be added to an instance by setting the "class" option to its +# fully qualified class name (see examples below). ## List of available sinks and their properties. # org.apache.spark.metrics.sink.ConsoleSink # Name: Default: Description: # period 10 Poll period -# unit seconds Units of poll period +# unit seconds Unit of the poll period # org.apache.spark.metrics.sink.CSVSink # Name: Default: Description: # period 10 Poll period -# unit seconds Units of poll period +# unit seconds Unit of the poll period # directory /tmp Where to store CSV files # org.apache.spark.metrics.sink.GangliaSink # Name: Default: Description: -# host NONE Hostname or multicast group of Ganglia server -# port NONE Port of Ganglia server(s) +# host NONE Hostname or multicast group of the Ganglia server, +# must be set +# port NONE Port of the Ganglia server(s), must be set # period 10 Poll period -# unit seconds Units of poll period +# unit seconds Unit of the poll period # ttl 1 TTL of messages sent by Ganglia +# dmax 0 Lifetime in seconds of metrics (0 never expired) # mode multicast Ganglia network mode ('unicast' or 'multicast') # org.apache.spark.metrics.sink.JmxSink @@ -98,20 +101,30 @@ # org.apache.spark.metrics.sink.MetricsServlet # Name: Default: Description: # path VARIES* Path prefix from the web server root -# sample false Whether to show entire set of samples for histograms ('false' or 'true') +# sample false Whether to show entire set of samples for histograms +# ('false' or 'true') # -# * Default path is /metrics/json for all instances except the master. The master has two paths: +# * Default path is /metrics/json for all instances except the master. The +# master has two paths: # /metrics/applications/json # App information # /metrics/master/json # Master information # org.apache.spark.metrics.sink.GraphiteSink # Name: Default: Description: -# host NONE Hostname of Graphite server -# port NONE Port of Graphite server +# host NONE Hostname of the Graphite server, must be set +# port NONE Port of the Graphite server, must be set +# period 10 Poll period +# unit seconds Unit of the poll period +# prefix EMPTY STRING Prefix to prepend to every metric's name +# protocol tcp Protocol ("tcp" or "udp") to use + +# org.apache.spark.metrics.sink.StatsdSink +# Name: Default: Description: +# host 127.0.0.1 Hostname or IP of StatsD server +# port 8125 Port of StatsD server # period 10 Poll period # unit seconds Units of poll period # prefix EMPTY STRING Prefix to prepend to metric name -# protocol tcp Protocol ("tcp" or "udp") to use ## Examples # Enable JmxSink for all instances by class name @@ -120,42 +133,46 @@ # Enable ConsoleSink for all instances by class name #*.sink.console.class=org.apache.spark.metrics.sink.ConsoleSink -# Polling period for ConsoleSink -#*.sink.console.period=10 +# Enable StatsdSink for all instances by class name +#*.sink.statsd.class=org.apache.spark.metrics.sink.StatsdSink +#*.sink.statsd.prefix=spark +# Polling period for the ConsoleSink +#*.sink.console.period=10 +# Unit of the polling period for the ConsoleSink #*.sink.console.unit=seconds -# Master instance overlap polling period +# Polling period for the ConsoleSink specific for the master instance #master.sink.console.period=15 - +# Unit of the polling period for the ConsoleSink specific for the master +# instance #master.sink.console.unit=seconds -# Enable CsvSink for all instances +# Enable CsvSink for all instances by class name #*.sink.csv.class=org.apache.spark.metrics.sink.CsvSink -# Polling period for CsvSink +# Polling period for the CsvSink #*.sink.csv.period=1 - +# Unit of the polling period for the CsvSink #*.sink.csv.unit=minutes # Polling directory for CsvSink #*.sink.csv.directory=/tmp/ -# Worker instance overlap polling period +# Polling period for the CsvSink specific for the worker instance #worker.sink.csv.period=10 - +# Unit of the polling period for the CsvSink specific for the worker instance #worker.sink.csv.unit=minutes # Enable Slf4jSink for all instances by class name #*.sink.slf4j.class=org.apache.spark.metrics.sink.Slf4jSink -# Polling period for Slf4JSink +# Polling period for the Slf4JSink #*.sink.slf4j.period=1 - +# Unit of the polling period for the Slf4jSink #*.sink.slf4j.unit=minutes - -# Enable jvm source for instance master, worker, driver and executor +# Enable JvmSource for instance master, worker, driver and executor #master.source.jvm.class=org.apache.spark.metrics.source.JvmSource #worker.source.jvm.class=org.apache.spark.metrics.source.JvmSource @@ -163,4 +180,3 @@ #driver.source.jvm.class=org.apache.spark.metrics.source.JvmSource #executor.source.jvm.class=org.apache.spark.metrics.source.JvmSource - diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 771251f90ee3..f8c895f5303b 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -25,40 +25,34 @@ # - HADOOP_CONF_DIR, to point Spark towards Hadoop configuration files # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node # - SPARK_PUBLIC_DNS, to set the public dns name of the driver program -# - SPARK_CLASSPATH, default classpath entries to append # Options read by executors and drivers running inside the cluster # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node # - SPARK_PUBLIC_DNS, to set the public DNS name of the driver program -# - SPARK_CLASSPATH, default classpath entries to append # - SPARK_LOCAL_DIRS, storage directories to use on this node for shuffle and RDD data # - MESOS_NATIVE_JAVA_LIBRARY, to point to your libmesos.so if you use Mesos # Options read in YARN client mode # - HADOOP_CONF_DIR, to point Spark towards Hadoop configuration files -# - SPARK_EXECUTOR_INSTANCES, Number of executors to start (Default: 2) +# - YARN_CONF_DIR, to point Spark towards YARN configuration files when you use YARN # - SPARK_EXECUTOR_CORES, Number of cores for the executors (Default: 1). # - SPARK_EXECUTOR_MEMORY, Memory per Executor (e.g. 1000M, 2G) (Default: 1G) # - SPARK_DRIVER_MEMORY, Memory for Driver (e.g. 1000M, 2G) (Default: 1G) -# - SPARK_YARN_APP_NAME, The name of your application (Default: Spark) -# - SPARK_YARN_QUEUE, The hadoop queue to use for allocation requests (Default: ‘default’) -# - SPARK_YARN_DIST_FILES, Comma separated list of files to be distributed with the job. -# - SPARK_YARN_DIST_ARCHIVES, Comma separated list of archives to be distributed with the job. # Options for the daemons used in the standalone deploy mode -# - SPARK_MASTER_IP, to bind the master to a different IP address or hostname +# - SPARK_MASTER_HOST, to bind the master to a different IP address or hostname # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports for the master # - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. "-Dx=y") # - SPARK_WORKER_CORES, to set the number of cores to use on this machine # - SPARK_WORKER_MEMORY, to set how much total memory workers have to give executors (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT, to use non-default ports for the worker -# - SPARK_WORKER_INSTANCES, to set the number of worker processes per node # - SPARK_WORKER_DIR, to set the working directory of worker processes # - SPARK_WORKER_OPTS, to set config properties only for the worker (e.g. "-Dx=y") # - SPARK_DAEMON_MEMORY, to allocate to the master, worker and history server themselves (default: 1g). # - SPARK_HISTORY_OPTS, to set config properties only for the history server (e.g. "-Dx=y") # - SPARK_SHUFFLE_OPTS, to set config properties only for the external shuffle service (e.g. "-Dx=y") # - SPARK_DAEMON_JAVA_OPTS, to set config properties for all daemons (e.g. "-Dx=y") +# - SPARK_DAEMON_CLASSPATH, to set the classpath for all daemons # - SPARK_PUBLIC_DNS, to set the public dns name of the master or workers # Generic options for the daemons used in the standalone deploy mode @@ -67,3 +61,8 @@ # - SPARK_PID_DIR Where the pid file is stored. (Default: /tmp) # - SPARK_IDENT_STRING A string representing this instance of spark. (Default: $USER) # - SPARK_NICENESS The scheduling priority for daemons. (Default: 0) +# - SPARK_NO_DAEMONIZE Run the proposed command in the foreground. It will not output a PID file. +# Options for native BLAS, like Intel MKL, OpenBLAS, and so on. +# You might get better performance to enable these options if using native BLAS (see SPARK-21305). +# - MKL_NUM_THREADS=1 Disable multi-threading of Intel MKL +# - OPENBLAS_NUM_THREADS=1 Disable multi-threading of OpenBLAS diff --git a/core/pom.xml b/core/pom.xml index 570a25cf325a..da68abd855c7 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -20,13 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT + spark-parent_2.11 + 2.3.0-SNAPSHOT ../pom.xml - org.apache.spark - spark-core_2.10 + spark-core_2.11 core @@ -34,6 +33,10 @@ Spark Project Core http://spark.apache.org/ + + org.apache.avro + avro + org.apache.avro avro-mapred @@ -51,6 +54,10 @@ com.twitter chill-java + + org.apache.xbean + xbean-asm5-shaded + org.apache.hadoop hadoop-client @@ -121,12 +128,25 @@ jetty-servlet compile - - org.eclipse.jetty.orbit - javax.servlet - ${orbit.version} + org.eclipse.jetty + jetty-proxy + compile + + + org.eclipse.jetty + jetty-client + compile + + + org.eclipse.jetty + jetty-servlets + compile + + + javax.servlet + javax.servlet-api + ${javaxservlet.version} @@ -170,25 +190,16 @@ snappy-java - net.jpountz.lz4 - lz4 - - - commons-net - commons-net - - - ${akka.group} - akka-remote_${scala.binary.version} + org.lz4 + lz4-java - ${akka.group} - akka-slf4j_${scala.binary.version} + org.roaringbitmap + RoaringBitmap - ${akka.group} - akka-testkit_${scala.binary.version} - test + commons-net + commons-net org.scala-lang @@ -197,25 +208,35 @@ org.json4s json4s-jackson_${scala.binary.version} - 3.2.10 - com.sun.jersey + org.glassfish.jersey.core + jersey-client + + + org.glassfish.jersey.core + jersey-common + + + org.glassfish.jersey.core jersey-server - com.sun.jersey - jersey-core + org.glassfish.jersey.containers + jersey-container-servlet - org.apache.mesos - mesos - ${mesos.classifier} + org.glassfish.jersey.containers + jersey-container-servlet-core io.netty netty-all + + io.netty + netty + com.clearspring.analytics stream @@ -260,41 +281,13 @@ ${oro.version} - org.tachyonproject - tachyon-client - 0.8.1 - - - org.apache.hadoop - hadoop-client - - - org.apache.curator - curator-client - - - org.apache.curator - curator-framework - - - org.apache.curator - curator-recipes - - - org.tachyonproject - tachyon-underfs-glusterfs - - + org.seleniumhq.selenium + selenium-java + test org.seleniumhq.selenium - selenium-java - - - com.google.guava - guava - - + selenium-htmlunit-driver test @@ -331,7 +324,7 @@ net.razorvine pyrolite - 4.9 + 4.13 net.razorvine @@ -342,17 +335,93 @@ net.sf.py4j py4j - 0.9 + 0.10.6 org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + org.apache.commons + commons-crypto + + + + + ${hive.group} + hive-exec + provided + + + ${hive.group} + hive-metastore + provided + + + org.apache.thrift + libthrift + provided + + + org.apache.thrift + libfb303 + provided + + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + ${project.basedir}/src/main/resources + + + + ${project.build.directory}/extra-resources + true + + + + org.apache.maven.plugins + maven-antrun-plugin + + + generate-resources + + + + + + + + + + + + run + + + + org.apache.maven.plugins maven-dependency-plugin @@ -360,6 +429,7 @@ + copy-dependencies package @@ -373,7 +443,7 @@ true true - guava,jetty-io,jetty-servlet,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server,jetty-security + guava,jetty-io,jetty-servlet,jetty-servlets,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server,jetty-security,jetty-proxy,jetty-client true @@ -392,7 +462,6 @@ - \ .bat @@ -404,7 +473,6 @@ - / .sh @@ -415,6 +483,7 @@ org.codehaus.mojo exec-maven-plugin + 1.6.0 sparkr-pkg @@ -425,7 +494,7 @@ - ..${path.separator}R${path.separator}install-dev${script.extension} + ..${file.separator}R${file.separator}install-dev${script.extension} diff --git a/core/src/main/java/org/apache/spark/JavaSparkListener.java b/core/src/main/java/org/apache/spark/JavaSparkListener.java deleted file mode 100644 index fa9acf0a15b8..000000000000 --- a/core/src/main/java/org/apache/spark/JavaSparkListener.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark; - -import org.apache.spark.scheduler.*; - -/** - * Java clients should extend this class instead of implementing - * SparkListener directly. This is to prevent java clients - * from breaking when new events are added to the SparkListener - * trait. - * - * This is a concrete class instead of abstract to enforce - * new events get added to both the SparkListener and this adapter - * in lockstep. - */ -public class JavaSparkListener implements SparkListener { - - @Override - public void onStageCompleted(SparkListenerStageCompleted stageCompleted) { } - - @Override - public void onStageSubmitted(SparkListenerStageSubmitted stageSubmitted) { } - - @Override - public void onTaskStart(SparkListenerTaskStart taskStart) { } - - @Override - public void onTaskGettingResult(SparkListenerTaskGettingResult taskGettingResult) { } - - @Override - public void onTaskEnd(SparkListenerTaskEnd taskEnd) { } - - @Override - public void onJobStart(SparkListenerJobStart jobStart) { } - - @Override - public void onJobEnd(SparkListenerJobEnd jobEnd) { } - - @Override - public void onEnvironmentUpdate(SparkListenerEnvironmentUpdate environmentUpdate) { } - - @Override - public void onBlockManagerAdded(SparkListenerBlockManagerAdded blockManagerAdded) { } - - @Override - public void onBlockManagerRemoved(SparkListenerBlockManagerRemoved blockManagerRemoved) { } - - @Override - public void onUnpersistRDD(SparkListenerUnpersistRDD unpersistRDD) { } - - @Override - public void onApplicationStart(SparkListenerApplicationStart applicationStart) { } - - @Override - public void onApplicationEnd(SparkListenerApplicationEnd applicationEnd) { } - - @Override - public void onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate executorMetricsUpdate) { } - - @Override - public void onExecutorAdded(SparkListenerExecutorAdded executorAdded) { } - - @Override - public void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved) { } - - @Override - public void onBlockUpdated(SparkListenerBlockUpdated blockUpdated) { } - -} diff --git a/core/src/main/java/org/apache/spark/SparkExecutorInfo.java b/core/src/main/java/org/apache/spark/SparkExecutorInfo.java new file mode 100644 index 000000000000..dc3e82647598 --- /dev/null +++ b/core/src/main/java/org/apache/spark/SparkExecutorInfo.java @@ -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; + +import java.io.Serializable; + +/** + * Exposes information about Spark Executors. + * + * This interface is not designed to be implemented outside of Spark. We may add additional methods + * which may break binary compatibility with outside implementations. + */ +public interface SparkExecutorInfo extends Serializable { + String host(); + int port(); + long cacheSize(); + int numRunningTasks(); +} diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java index 1214d05ba606..3583856d8899 100644 --- a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java +++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java @@ -28,94 +28,124 @@ * this was a concrete Scala class, default implementations of new event handlers would be inherited * from the SparkListener trait). */ -public class SparkFirehoseListener implements SparkListener { - - public void onEvent(SparkListenerEvent event) { } - - @Override - public final void onStageCompleted(SparkListenerStageCompleted stageCompleted) { - onEvent(stageCompleted); - } - - @Override - public final void onStageSubmitted(SparkListenerStageSubmitted stageSubmitted) { - onEvent(stageSubmitted); - } - - @Override - public final void onTaskStart(SparkListenerTaskStart taskStart) { - onEvent(taskStart); - } - - @Override - public final void onTaskGettingResult(SparkListenerTaskGettingResult taskGettingResult) { - onEvent(taskGettingResult); - } - - @Override - public final void onTaskEnd(SparkListenerTaskEnd taskEnd) { - onEvent(taskEnd); - } - - @Override - public final void onJobStart(SparkListenerJobStart jobStart) { - onEvent(jobStart); - } - - @Override - public final void onJobEnd(SparkListenerJobEnd jobEnd) { - onEvent(jobEnd); - } - - @Override - public final void onEnvironmentUpdate(SparkListenerEnvironmentUpdate environmentUpdate) { - onEvent(environmentUpdate); - } - - @Override - public final void onBlockManagerAdded(SparkListenerBlockManagerAdded blockManagerAdded) { - onEvent(blockManagerAdded); - } - - @Override - public final void onBlockManagerRemoved(SparkListenerBlockManagerRemoved blockManagerRemoved) { - onEvent(blockManagerRemoved); - } - - @Override - public final void onUnpersistRDD(SparkListenerUnpersistRDD unpersistRDD) { - onEvent(unpersistRDD); - } - - @Override - public final void onApplicationStart(SparkListenerApplicationStart applicationStart) { - onEvent(applicationStart); - } - - @Override - public final void onApplicationEnd(SparkListenerApplicationEnd applicationEnd) { - onEvent(applicationEnd); - } - - @Override - public final void onExecutorMetricsUpdate( - SparkListenerExecutorMetricsUpdate executorMetricsUpdate) { - onEvent(executorMetricsUpdate); - } - - @Override - public final void onExecutorAdded(SparkListenerExecutorAdded executorAdded) { - onEvent(executorAdded); - } - - @Override - public final void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved) { - onEvent(executorRemoved); - } - - @Override - public void onBlockUpdated(SparkListenerBlockUpdated blockUpdated) { - onEvent(blockUpdated); - } - +public class SparkFirehoseListener implements SparkListenerInterface { + + public void onEvent(SparkListenerEvent event) { } + + @Override + public final void onStageCompleted(SparkListenerStageCompleted stageCompleted) { + onEvent(stageCompleted); + } + + @Override + public final void onStageSubmitted(SparkListenerStageSubmitted stageSubmitted) { + onEvent(stageSubmitted); + } + + @Override + public final void onTaskStart(SparkListenerTaskStart taskStart) { + onEvent(taskStart); + } + + @Override + public final void onTaskGettingResult(SparkListenerTaskGettingResult taskGettingResult) { + onEvent(taskGettingResult); + } + + @Override + public final void onTaskEnd(SparkListenerTaskEnd taskEnd) { + onEvent(taskEnd); + } + + @Override + public final void onJobStart(SparkListenerJobStart jobStart) { + onEvent(jobStart); + } + + @Override + public final void onJobEnd(SparkListenerJobEnd jobEnd) { + onEvent(jobEnd); + } + + @Override + public final void onEnvironmentUpdate(SparkListenerEnvironmentUpdate environmentUpdate) { + onEvent(environmentUpdate); + } + + @Override + public final void onBlockManagerAdded(SparkListenerBlockManagerAdded blockManagerAdded) { + onEvent(blockManagerAdded); + } + + @Override + public final void onBlockManagerRemoved(SparkListenerBlockManagerRemoved blockManagerRemoved) { + onEvent(blockManagerRemoved); + } + + @Override + public final void onUnpersistRDD(SparkListenerUnpersistRDD unpersistRDD) { + onEvent(unpersistRDD); + } + + @Override + public final void onApplicationStart(SparkListenerApplicationStart applicationStart) { + onEvent(applicationStart); + } + + @Override + public final void onApplicationEnd(SparkListenerApplicationEnd applicationEnd) { + onEvent(applicationEnd); + } + + @Override + public final void onExecutorMetricsUpdate( + SparkListenerExecutorMetricsUpdate executorMetricsUpdate) { + onEvent(executorMetricsUpdate); + } + + @Override + public final void onExecutorAdded(SparkListenerExecutorAdded executorAdded) { + onEvent(executorAdded); + } + + @Override + public final void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved) { + onEvent(executorRemoved); + } + + @Override + public final void onExecutorBlacklisted(SparkListenerExecutorBlacklisted executorBlacklisted) { + onEvent(executorBlacklisted); + } + + @Override + public final void onExecutorUnblacklisted( + SparkListenerExecutorUnblacklisted executorUnblacklisted) { + onEvent(executorUnblacklisted); + } + + @Override + public final void onNodeBlacklisted(SparkListenerNodeBlacklisted nodeBlacklisted) { + onEvent(nodeBlacklisted); + } + + @Override + public final void onNodeUnblacklisted(SparkListenerNodeUnblacklisted nodeUnblacklisted) { + onEvent(nodeUnblacklisted); + } + + @Override + public void onBlockUpdated(SparkListenerBlockUpdated blockUpdated) { + onEvent(blockUpdated); + } + + @Override + public void onSpeculativeTaskSubmitted(SparkListenerSpeculativeTaskSubmitted speculativeTask) { + onEvent(speculativeTask); + } + + @Override + public void onOtherEvent(SparkListenerEvent event) { + onEvent(event); + } } diff --git a/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java b/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java index d4c42b38ac22..0dd8fafbf2c8 100644 --- a/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java +++ b/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java @@ -62,5 +62,6 @@ public final JavaPairRDD union(JavaPairRDD... rdds) { // These methods take separate "first" and "rest" elements to avoid having the same type erasure public abstract JavaRDD union(JavaRDD first, List> rest); public abstract JavaDoubleRDD union(JavaDoubleRDD first, List rest); - public abstract JavaPairRDD union(JavaPairRDD first, List> rest); + public abstract JavaPairRDD union(JavaPairRDD first, List> + rest); } diff --git a/core/src/main/java/org/apache/spark/api/java/Optional.java b/core/src/main/java/org/apache/spark/api/java/Optional.java new file mode 100644 index 000000000000..fd0f495ca29d --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/Optional.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.java; + +import java.io.Serializable; +import java.util.Objects; + +import com.google.common.base.Preconditions; + +/** + *

Like {@code java.util.Optional} in Java 8, {@code scala.Option} in Scala, and + * {@code com.google.common.base.Optional} in Google Guava, this class represents a + * value of a given type that may or may not exist. It is used in methods that wish + * to optionally return a value, in preference to returning {@code null}.

+ * + *

In fact, the class here is a reimplementation of the essential API of both + * {@code java.util.Optional} and {@code com.google.common.base.Optional}. From + * {@code java.util.Optional}, it implements:

+ * + *
    + *
  • {@link #empty()}
  • + *
  • {@link #of(Object)}
  • + *
  • {@link #ofNullable(Object)}
  • + *
  • {@link #get()}
  • + *
  • {@link #orElse(Object)}
  • + *
  • {@link #isPresent()}
  • + *
+ * + *

From {@code com.google.common.base.Optional} it implements:

+ * + *
    + *
  • {@link #absent()}
  • + *
  • {@link #of(Object)}
  • + *
  • {@link #fromNullable(Object)}
  • + *
  • {@link #get()}
  • + *
  • {@link #or(Object)}
  • + *
  • {@link #orNull()}
  • + *
  • {@link #isPresent()}
  • + *
+ * + *

{@code java.util.Optional} itself was not used because at the time, the + * project did not require Java 8. Using {@code com.google.common.base.Optional} + * has in the past caused serious library version conflicts with Guava that can't + * be resolved by shading. Hence this work-alike clone.

+ * + * @param type of value held inside + */ +public final class Optional implements Serializable { + + private static final Optional EMPTY = new Optional<>(); + + private final T value; + + private Optional() { + this.value = null; + } + + private Optional(T value) { + Preconditions.checkNotNull(value); + this.value = value; + } + + // java.util.Optional API (subset) + + /** + * @return an empty {@code Optional} + */ + public static Optional empty() { + @SuppressWarnings("unchecked") + Optional t = (Optional) EMPTY; + return t; + } + + /** + * @param value non-null value to wrap + * @return {@code Optional} wrapping this value + * @throws NullPointerException if value is null + */ + public static Optional of(T value) { + return new Optional<>(value); + } + + /** + * @param value value to wrap, which may be null + * @return {@code Optional} wrapping this value, which may be empty + */ + public static Optional ofNullable(T value) { + if (value == null) { + return empty(); + } else { + return of(value); + } + } + + /** + * @return the value wrapped by this {@code Optional} + * @throws NullPointerException if this is empty (contains no value) + */ + public T get() { + Preconditions.checkNotNull(value); + return value; + } + + /** + * @param other value to return if this is empty + * @return this {@code Optional}'s value if present, or else the given value + */ + public T orElse(T other) { + return value != null ? value : other; + } + + /** + * @return true iff this {@code Optional} contains a value (non-empty) + */ + public boolean isPresent() { + return value != null; + } + + // Guava API (subset) + // of(), get() and isPresent() are identically present in the Guava API + + /** + * @return an empty {@code Optional} + */ + public static Optional absent() { + return empty(); + } + + /** + * @param value value to wrap, which may be null + * @return {@code Optional} wrapping this value, which may be empty + */ + public static Optional fromNullable(T value) { + return ofNullable(value); + } + + /** + * @param other value to return if this is empty + * @return this {@code Optional}'s value if present, or else the given value + */ + public T or(T other) { + return value != null ? value : other; + } + + /** + * @return this {@code Optional}'s value if present, or else null + */ + public T orNull() { + return value; + } + + // Common methods + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof Optional)) { + return false; + } + Optional other = (Optional) obj; + return Objects.equals(value, other.value); + } + + @Override + public int hashCode() { + return value == null ? 0 : value.hashCode(); + } + + @Override + public String toString() { + return value == null ? "Optional.empty" : String.format("Optional[%s]", value); + } + +} diff --git a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java index 840a1bd93bfb..3fcb52f61583 100644 --- a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java +++ b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java @@ -34,26 +34,13 @@ public class StorageLevels { public static final StorageLevel MEMORY_AND_DISK_2 = create(true, true, false, true, 2); public static final StorageLevel MEMORY_AND_DISK_SER = create(true, true, false, false, 1); public static final StorageLevel MEMORY_AND_DISK_SER_2 = create(true, true, false, false, 2); - public static final StorageLevel OFF_HEAP = create(false, false, true, false, 1); + public static final StorageLevel OFF_HEAP = create(true, true, true, false, 1); /** * Create a new StorageLevel object. * @param useDisk saved to disk, if true - * @param useMemory saved to memory, if true - * @param deserialized saved as deserialized objects, if true - * @param replication replication factor - */ - @Deprecated - public static StorageLevel create(boolean useDisk, boolean useMemory, boolean deserialized, - int replication) { - return StorageLevel.apply(useDisk, useMemory, false, deserialized, replication); - } - - /** - * Create a new StorageLevel object. - * @param useDisk saved to disk, if true - * @param useMemory saved to memory, if true - * @param useOffHeap saved to Tachyon, if true + * @param useMemory saved to on-heap memory, if true + * @param useOffHeap saved to off-heap memory, if true * @param deserialized saved as deserialized objects, if true * @param replication replication factor */ diff --git a/core/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java b/core/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java new file mode 100644 index 000000000000..33bedf7ebcb0 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/CoGroupFunction.java @@ -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.api.java.function; + +import java.io.Serializable; +import java.util.Iterator; + +/** + * A function that returns zero or more output records from each grouping key and its values from 2 + * Datasets. + */ +@FunctionalInterface +public interface CoGroupFunction extends Serializable { + Iterator call(K key, Iterator left, Iterator right) throws Exception; +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java b/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java index 57fd0a7a8049..2f23da5bfec1 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java @@ -18,10 +18,12 @@ package org.apache.spark.api.java.function; import java.io.Serializable; +import java.util.Iterator; /** * A function that returns zero or more records of type Double from each input record. */ +@FunctionalInterface public interface DoubleFlatMapFunction extends Serializable { - public Iterable call(T t) throws Exception; + Iterator call(T t) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java b/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java index 150144e0e418..3c0291cf4624 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java @@ -22,6 +22,7 @@ /** * A function that returns Doubles, and can be used to construct DoubleRDDs. */ +@FunctionalInterface public interface DoubleFunction extends Serializable { - public double call(T t) throws Exception; + double call(T t) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java b/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java new file mode 100644 index 000000000000..a6f69f7cdca8 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/FilterFunction.java @@ -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.api.java.function; + +import java.io.Serializable; + +/** + * Base interface for a function used in Dataset's filter function. + * + * If the function returns true, the element is included in the returned Dataset. + */ +@FunctionalInterface +public interface FilterFunction extends Serializable { + boolean call(T value) throws Exception; +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java index 23f5fdd43631..91d61292f167 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java @@ -18,10 +18,12 @@ package org.apache.spark.api.java.function; import java.io.Serializable; +import java.util.Iterator; /** * A function that returns zero or more output records from each input record. */ +@FunctionalInterface public interface FlatMapFunction extends Serializable { - public Iterable call(T t) throws Exception; + Iterator call(T t) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java index c48e92f535ff..f9f2580b01f4 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java +++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java @@ -18,10 +18,12 @@ package org.apache.spark.api.java.function; import java.io.Serializable; +import java.util.Iterator; /** * A function that takes two inputs and returns zero or more output records. */ +@FunctionalInterface public interface FlatMapFunction2 extends Serializable { - public Iterable call(T1 t1, T2 t2) throws Exception; + Iterator call(T1 t1, T2 t2) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsFunction.java b/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsFunction.java new file mode 100644 index 000000000000..6423c5d0fce5 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsFunction.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.java.function; + +import java.io.Serializable; +import java.util.Iterator; + +/** + * A function that returns zero or more output records from each grouping key and its values. + */ +@FunctionalInterface +public interface FlatMapGroupsFunction extends Serializable { + Iterator call(K key, Iterator values) throws Exception; +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/ForeachFunction.java b/core/src/main/java/org/apache/spark/api/java/function/ForeachFunction.java new file mode 100644 index 000000000000..2e6e90818d58 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/ForeachFunction.java @@ -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.api.java.function; + +import java.io.Serializable; + +/** + * Base interface for a function used in Dataset's foreach function. + * + * Spark will invoke the call function on each element in the input Dataset. + */ +@FunctionalInterface +public interface ForeachFunction extends Serializable { + void call(T t) throws Exception; +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/ForeachPartitionFunction.java b/core/src/main/java/org/apache/spark/api/java/function/ForeachPartitionFunction.java new file mode 100644 index 000000000000..d8f55d0ae1dc --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/ForeachPartitionFunction.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.java.function; + +import java.io.Serializable; +import java.util.Iterator; + +/** + * Base interface for a function used in Dataset's foreachPartition function. + */ +@FunctionalInterface +public interface ForeachPartitionFunction extends Serializable { + void call(Iterator t) throws Exception; +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/Function.java b/core/src/main/java/org/apache/spark/api/java/function/Function.java index d00551bb0add..8b2bbd501c49 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/Function.java +++ b/core/src/main/java/org/apache/spark/api/java/function/Function.java @@ -24,6 +24,7 @@ * DoubleFunction are handled separately, to allow PairRDDs and DoubleRDDs to be constructed * when mapping RDDs of other types. */ +@FunctionalInterface public interface Function extends Serializable { - public R call(T1 v1) throws Exception; + R call(T1 v1) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/Function0.java b/core/src/main/java/org/apache/spark/api/java/function/Function0.java index 38e410c5debe..5c649d9de414 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/Function0.java +++ b/core/src/main/java/org/apache/spark/api/java/function/Function0.java @@ -22,6 +22,7 @@ /** * A zero-argument function that returns an R. */ +@FunctionalInterface public interface Function0 extends Serializable { - public R call() throws Exception; + R call() throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/Function2.java b/core/src/main/java/org/apache/spark/api/java/function/Function2.java index 793caaa61ac5..a7d964709515 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/Function2.java +++ b/core/src/main/java/org/apache/spark/api/java/function/Function2.java @@ -22,6 +22,7 @@ /** * A two-argument function that takes arguments of type T1 and T2 and returns an R. */ +@FunctionalInterface public interface Function2 extends Serializable { - public R call(T1 v1, T2 v2) throws Exception; + R call(T1 v1, T2 v2) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/Function3.java b/core/src/main/java/org/apache/spark/api/java/function/Function3.java index b4151c3417df..77acd21d4eff 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/Function3.java +++ b/core/src/main/java/org/apache/spark/api/java/function/Function3.java @@ -22,6 +22,7 @@ /** * A three-argument function that takes arguments of type T1, T2 and T3 and returns an R. */ +@FunctionalInterface public interface Function3 extends Serializable { - public R call(T1 v1, T2 v2, T3 v3) throws Exception; + R call(T1 v1, T2 v2, T3 v3) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/Function4.java b/core/src/main/java/org/apache/spark/api/java/function/Function4.java new file mode 100644 index 000000000000..d530ba446b3c --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/Function4.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java.function; + +import java.io.Serializable; + +/** + * A four-argument function that takes arguments of type T1, T2, T3 and T4 and returns an R. + */ +@FunctionalInterface +public interface Function4 extends Serializable { + R call(T1 v1, T2 v2, T3 v3, T4 v4) throws Exception; +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/MapFunction.java b/core/src/main/java/org/apache/spark/api/java/function/MapFunction.java new file mode 100644 index 000000000000..5efff943c8cd --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/MapFunction.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java.function; + +import java.io.Serializable; + +/** + * Base interface for a map function used in Dataset's map function. + */ +@FunctionalInterface +public interface MapFunction extends Serializable { + U call(T value) throws Exception; +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/MapGroupsFunction.java b/core/src/main/java/org/apache/spark/api/java/function/MapGroupsFunction.java new file mode 100644 index 000000000000..2c3d43afc0b3 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/MapGroupsFunction.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.java.function; + +import java.io.Serializable; +import java.util.Iterator; + +/** + * Base interface for a map function used in GroupedDataset's mapGroup function. + */ +@FunctionalInterface +public interface MapGroupsFunction extends Serializable { + R call(K key, Iterator values) throws Exception; +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/MapPartitionsFunction.java b/core/src/main/java/org/apache/spark/api/java/function/MapPartitionsFunction.java new file mode 100644 index 000000000000..68e8557c88d1 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/MapPartitionsFunction.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.java.function; + +import java.io.Serializable; +import java.util.Iterator; + +/** + * Base interface for function used in Dataset's mapPartitions. + */ +@FunctionalInterface +public interface MapPartitionsFunction extends Serializable { + Iterator call(Iterator input) throws Exception; +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java index 691ef2eceb1f..97bd2b37a059 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java @@ -18,6 +18,7 @@ package org.apache.spark.api.java.function; import java.io.Serializable; +import java.util.Iterator; import scala.Tuple2; @@ -25,6 +26,7 @@ * A function that returns zero or more key-value pair records from each input record. The * key-value pairs are represented as scala.Tuple2 objects. */ +@FunctionalInterface public interface PairFlatMapFunction extends Serializable { - public Iterable> call(T t) throws Exception; + Iterator> call(T t) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java b/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java index 99bf240a1722..34a7e4489a31 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java @@ -25,6 +25,7 @@ * A function that returns key-value pairs (Tuple2<K, V>), and can be used to * construct PairRDDs. */ +@FunctionalInterface public interface PairFunction extends Serializable { - public Tuple2 call(T t) throws Exception; + Tuple2 call(T t) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/ReduceFunction.java b/core/src/main/java/org/apache/spark/api/java/function/ReduceFunction.java new file mode 100644 index 000000000000..d9029d85387a --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/ReduceFunction.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java.function; + +import java.io.Serializable; + +/** + * Base interface for function used in Dataset's reduce. + */ +@FunctionalInterface +public interface ReduceFunction extends Serializable { + T call(T v1, T v2) throws Exception; +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/VoidFunction.java b/core/src/main/java/org/apache/spark/api/java/function/VoidFunction.java index 2a10435b7523..aff2bc6e94fb 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/VoidFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/VoidFunction.java @@ -22,6 +22,7 @@ /** * A function with no return value. */ +@FunctionalInterface public interface VoidFunction extends Serializable { - public void call(T t) throws Exception; + void call(T t) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/VoidFunction2.java b/core/src/main/java/org/apache/spark/api/java/function/VoidFunction2.java new file mode 100644 index 000000000000..ddb616241b24 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/VoidFunction2.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java.function; + +import java.io.Serializable; + +/** + * A two-argument function that takes arguments of type T1 and T2 with no return value. + */ +@FunctionalInterface +public interface VoidFunction2 extends Serializable { + void call(T1 v1, T2 v2) throws Exception; +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/package-info.java b/core/src/main/java/org/apache/spark/api/java/function/package-info.java index 463a42f23342..eefb29aca9d4 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/package-info.java +++ b/core/src/main/java/org/apache/spark/api/java/function/package-info.java @@ -20,4 +20,4 @@ * these interfaces to pass functions to various Java API methods for Spark. Please visit Spark's * Java programming guide for more details. */ -package org.apache.spark.api.java.function; \ No newline at end of file +package org.apache.spark.api.java.function; diff --git a/core/src/main/java/org/apache/spark/api/java/function/package.scala b/core/src/main/java/org/apache/spark/api/java/function/package.scala index 0f9bac716416..e19f12fdac09 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/package.scala +++ b/core/src/main/java/org/apache/spark/api/java/function/package.scala @@ -22,4 +22,4 @@ package org.apache.spark.api.java * these interfaces to pass functions to various Java API methods for Spark. Please visit Spark's * Java programming guide for more details. */ -package object function +package object function diff --git a/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java b/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java new file mode 100644 index 000000000000..ea5f1a9abf69 --- /dev/null +++ b/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java @@ -0,0 +1,139 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.io; + +import org.apache.spark.storage.StorageUtils; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; + +/** + * {@link InputStream} implementation which uses direct buffer + * to read a file to avoid extra copy of data between Java and + * native memory which happens when using {@link java.io.BufferedInputStream}. + * Unfortunately, this is not something already available in JDK, + * {@link sun.nio.ch.ChannelInputStream} supports reading a file using nio, + * but does not support buffering. + */ +public final class NioBufferedFileInputStream extends InputStream { + + private static final int DEFAULT_BUFFER_SIZE_BYTES = 8192; + + private final ByteBuffer byteBuffer; + + private final FileChannel fileChannel; + + public NioBufferedFileInputStream(File file, int bufferSizeInBytes) throws IOException { + byteBuffer = ByteBuffer.allocateDirect(bufferSizeInBytes); + fileChannel = FileChannel.open(file.toPath(), StandardOpenOption.READ); + byteBuffer.flip(); + } + + public NioBufferedFileInputStream(File file) throws IOException { + this(file, DEFAULT_BUFFER_SIZE_BYTES); + } + + /** + * Checks weather data is left to be read from the input stream. + * @return true if data is left, false otherwise + * @throws IOException + */ + private boolean refill() throws IOException { + if (!byteBuffer.hasRemaining()) { + byteBuffer.clear(); + int nRead = 0; + while (nRead == 0) { + nRead = fileChannel.read(byteBuffer); + } + if (nRead < 0) { + return false; + } + byteBuffer.flip(); + } + return true; + } + + @Override + public synchronized int read() throws IOException { + if (!refill()) { + return -1; + } + return byteBuffer.get() & 0xFF; + } + + @Override + public synchronized int read(byte[] b, int offset, int len) throws IOException { + if (offset < 0 || len < 0 || offset + len < 0 || offset + len > b.length) { + throw new IndexOutOfBoundsException(); + } + if (!refill()) { + return -1; + } + len = Math.min(len, byteBuffer.remaining()); + byteBuffer.get(b, offset, len); + return len; + } + + @Override + public synchronized int available() throws IOException { + return byteBuffer.remaining(); + } + + @Override + public synchronized long skip(long n) throws IOException { + if (n <= 0L) { + return 0L; + } + if (byteBuffer.remaining() >= n) { + // The buffered content is enough to skip + byteBuffer.position(byteBuffer.position() + (int) n); + return n; + } + long skippedFromBuffer = byteBuffer.remaining(); + long toSkipFromFileChannel = n - skippedFromBuffer; + // Discard everything we have read in the buffer. + byteBuffer.position(0); + byteBuffer.flip(); + return skippedFromBuffer + skipFromFileChannel(toSkipFromFileChannel); + } + + private long skipFromFileChannel(long n) throws IOException { + long currentFilePosition = fileChannel.position(); + long size = fileChannel.size(); + if (n > size - currentFilePosition) { + fileChannel.position(size); + return size - currentFilePosition; + } else { + fileChannel.position(currentFilePosition + n); + return n; + } + } + + @Override + public synchronized void close() throws IOException { + fileChannel.close(); + StorageUtils.dispose(byteBuffer); + } + + //checkstyle.off: NoFinalizer + @Override + protected void finalize() throws IOException { + close(); + } + //checkstyle.on: NoFinalizer +} diff --git a/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java b/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java new file mode 100644 index 000000000000..5b45d268ace8 --- /dev/null +++ b/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java @@ -0,0 +1,411 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.io; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import org.apache.spark.util.ThreadUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.GuardedBy; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.InterruptedIOException; +import java.nio.ByteBuffer; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; + +/** + * {@link InputStream} implementation which asynchronously reads ahead from the underlying input + * stream when specified amount of data has been read from the current buffer. It does it by + * maintaining two buffers - active buffer and read ahead buffer. Active buffer contains data + * which should be returned when a read() call is issued. The read ahead buffer is used to + * asynchronously read from the underlying input stream and once the current active buffer is + * exhausted, we flip the two buffers so that we can start reading from the read ahead buffer + * without being blocked in disk I/O. + */ +public class ReadAheadInputStream extends InputStream { + + private static final Logger logger = LoggerFactory.getLogger(ReadAheadInputStream.class); + + private ReentrantLock stateChangeLock = new ReentrantLock(); + + @GuardedBy("stateChangeLock") + private ByteBuffer activeBuffer; + + @GuardedBy("stateChangeLock") + private ByteBuffer readAheadBuffer; + + @GuardedBy("stateChangeLock") + private boolean endOfStream; + + @GuardedBy("stateChangeLock") + // true if async read is in progress + private boolean readInProgress; + + @GuardedBy("stateChangeLock") + // true if read is aborted due to an exception in reading from underlying input stream. + private boolean readAborted; + + @GuardedBy("stateChangeLock") + private Throwable readException; + + @GuardedBy("stateChangeLock") + // whether the close method is called. + private boolean isClosed; + + @GuardedBy("stateChangeLock") + // true when the close method will close the underlying input stream. This is valid only if + // `isClosed` is true. + private boolean isUnderlyingInputStreamBeingClosed; + + @GuardedBy("stateChangeLock") + // whether there is a read ahead task running, + private boolean isReading; + + // If the remaining data size in the current buffer is below this threshold, + // we issue an async read from the underlying input stream. + private final int readAheadThresholdInBytes; + + private final InputStream underlyingInputStream; + + private final ExecutorService executorService = + ThreadUtils.newDaemonSingleThreadExecutor("read-ahead"); + + private final Condition asyncReadComplete = stateChangeLock.newCondition(); + + private static final ThreadLocal oneByte = ThreadLocal.withInitial(() -> new byte[1]); + + /** + * Creates a ReadAheadInputStream with the specified buffer size and read-ahead + * threshold + * + * @param inputStream The underlying input stream. + * @param bufferSizeInBytes The buffer size. + * @param readAheadThresholdInBytes If the active buffer has less data than the read-ahead + * threshold, an async read is triggered. + */ + public ReadAheadInputStream( + InputStream inputStream, int bufferSizeInBytes, int readAheadThresholdInBytes) { + Preconditions.checkArgument(bufferSizeInBytes > 0, + "bufferSizeInBytes should be greater than 0, but the value is " + bufferSizeInBytes); + Preconditions.checkArgument(readAheadThresholdInBytes > 0 && + readAheadThresholdInBytes < bufferSizeInBytes, + "readAheadThresholdInBytes should be greater than 0 and less than bufferSizeInBytes, " + + "but the value is " + readAheadThresholdInBytes); + activeBuffer = ByteBuffer.allocate(bufferSizeInBytes); + readAheadBuffer = ByteBuffer.allocate(bufferSizeInBytes); + this.readAheadThresholdInBytes = readAheadThresholdInBytes; + this.underlyingInputStream = inputStream; + activeBuffer.flip(); + readAheadBuffer.flip(); + } + + private boolean isEndOfStream() { + return (!activeBuffer.hasRemaining() && !readAheadBuffer.hasRemaining() && endOfStream); + } + + private void checkReadException() throws IOException { + if (readAborted) { + Throwables.propagateIfPossible(readException, IOException.class); + throw new IOException(readException); + } + } + + /** Read data from underlyingInputStream to readAheadBuffer asynchronously. */ + private void readAsync() throws IOException { + stateChangeLock.lock(); + final byte[] arr = readAheadBuffer.array(); + try { + if (endOfStream || readInProgress) { + return; + } + checkReadException(); + readAheadBuffer.position(0); + readAheadBuffer.flip(); + readInProgress = true; + } finally { + stateChangeLock.unlock(); + } + executorService.execute(new Runnable() { + + @Override + public void run() { + stateChangeLock.lock(); + try { + if (isClosed) { + readInProgress = false; + return; + } + // Flip this so that the close method will not close the underlying input stream when we + // are reading. + isReading = true; + } finally { + stateChangeLock.unlock(); + } + + // Please note that it is safe to release the lock and read into the read ahead buffer + // because either of following two conditions will hold - 1. The active buffer has + // data available to read so the reader will not read from the read ahead buffer. + // 2. This is the first time read is called or the active buffer is exhausted, + // in that case the reader waits for this async read to complete. + // So there is no race condition in both the situations. + int read = 0; + Throwable exception = null; + try { + while (true) { + read = underlyingInputStream.read(arr); + if (0 != read) break; + } + } catch (Throwable ex) { + exception = ex; + if (ex instanceof Error) { + // `readException` may not be reported to the user. Rethrow Error to make sure at least + // The user can see Error in UncaughtExceptionHandler. + throw (Error) ex; + } + } finally { + stateChangeLock.lock(); + if (read < 0 || (exception instanceof EOFException)) { + endOfStream = true; + } else if (exception != null) { + readAborted = true; + readException = exception; + } else { + readAheadBuffer.limit(read); + } + readInProgress = false; + signalAsyncReadComplete(); + stateChangeLock.unlock(); + closeUnderlyingInputStreamIfNecessary(); + } + } + }); + } + + private void closeUnderlyingInputStreamIfNecessary() { + boolean needToCloseUnderlyingInputStream = false; + stateChangeLock.lock(); + try { + isReading = false; + if (isClosed && !isUnderlyingInputStreamBeingClosed) { + // close method cannot close underlyingInputStream because we were reading. + needToCloseUnderlyingInputStream = true; + } + } finally { + stateChangeLock.unlock(); + } + if (needToCloseUnderlyingInputStream) { + try { + underlyingInputStream.close(); + } catch (IOException e) { + logger.warn(e.getMessage(), e); + } + } + } + + private void signalAsyncReadComplete() { + stateChangeLock.lock(); + try { + asyncReadComplete.signalAll(); + } finally { + stateChangeLock.unlock(); + } + } + + private void waitForAsyncReadComplete() throws IOException { + stateChangeLock.lock(); + try { + while (readInProgress) { + asyncReadComplete.await(); + } + } catch (InterruptedException e) { + InterruptedIOException iio = new InterruptedIOException(e.getMessage()); + iio.initCause(e); + throw iio; + } finally { + stateChangeLock.unlock(); + } + checkReadException(); + } + + @Override + public int read() throws IOException { + byte[] oneByteArray = oneByte.get(); + return read(oneByteArray, 0, 1) == -1 ? -1 : oneByteArray[0] & 0xFF; + } + + @Override + public int read(byte[] b, int offset, int len) throws IOException { + if (offset < 0 || len < 0 || len > b.length - offset) { + throw new IndexOutOfBoundsException(); + } + if (len == 0) { + return 0; + } + stateChangeLock.lock(); + try { + return readInternal(b, offset, len); + } finally { + stateChangeLock.unlock(); + } + } + + /** + * flip the active and read ahead buffer + */ + private void swapBuffers() { + ByteBuffer temp = activeBuffer; + activeBuffer = readAheadBuffer; + readAheadBuffer = temp; + } + + /** + * Internal read function which should be called only from read() api. The assumption is that + * the stateChangeLock is already acquired in the caller before calling this function. + */ + private int readInternal(byte[] b, int offset, int len) throws IOException { + assert (stateChangeLock.isLocked()); + if (!activeBuffer.hasRemaining()) { + waitForAsyncReadComplete(); + if (readAheadBuffer.hasRemaining()) { + swapBuffers(); + } else { + // The first read or activeBuffer is skipped. + readAsync(); + waitForAsyncReadComplete(); + if (isEndOfStream()) { + return -1; + } + swapBuffers(); + } + } else { + checkReadException(); + } + len = Math.min(len, activeBuffer.remaining()); + activeBuffer.get(b, offset, len); + + if (activeBuffer.remaining() <= readAheadThresholdInBytes && !readAheadBuffer.hasRemaining()) { + readAsync(); + } + return len; + } + + @Override + public int available() throws IOException { + stateChangeLock.lock(); + // Make sure we have no integer overflow. + try { + return (int) Math.min((long) Integer.MAX_VALUE, + (long) activeBuffer.remaining() + readAheadBuffer.remaining()); + } finally { + stateChangeLock.unlock(); + } + } + + @Override + public long skip(long n) throws IOException { + if (n <= 0L) { + return 0L; + } + stateChangeLock.lock(); + long skipped; + try { + skipped = skipInternal(n); + } finally { + stateChangeLock.unlock(); + } + return skipped; + } + + /** + * Internal skip function which should be called only from skip() api. The assumption is that + * the stateChangeLock is already acquired in the caller before calling this function. + */ + private long skipInternal(long n) throws IOException { + assert (stateChangeLock.isLocked()); + waitForAsyncReadComplete(); + if (isEndOfStream()) { + return 0; + } + if (available() >= n) { + // we can skip from the internal buffers + int toSkip = (int) n; + if (toSkip <= activeBuffer.remaining()) { + // Only skipping from active buffer is sufficient + activeBuffer.position(toSkip + activeBuffer.position()); + if (activeBuffer.remaining() <= readAheadThresholdInBytes + && !readAheadBuffer.hasRemaining()) { + readAsync(); + } + return n; + } + // We need to skip from both active buffer and read ahead buffer + toSkip -= activeBuffer.remaining(); + activeBuffer.position(0); + activeBuffer.flip(); + readAheadBuffer.position(toSkip + readAheadBuffer.position()); + swapBuffers(); + readAsync(); + return n; + } else { + int skippedBytes = available(); + long toSkip = n - skippedBytes; + activeBuffer.position(0); + activeBuffer.flip(); + readAheadBuffer.position(0); + readAheadBuffer.flip(); + long skippedFromInputStream = underlyingInputStream.skip(toSkip); + readAsync(); + return skippedBytes + skippedFromInputStream; + } + } + + @Override + public void close() throws IOException { + boolean isSafeToCloseUnderlyingInputStream = false; + stateChangeLock.lock(); + try { + if (isClosed) { + return; + } + isClosed = true; + if (!isReading) { + // Nobody is reading, so we can close the underlying input stream in this method. + isSafeToCloseUnderlyingInputStream = true; + // Flip this to make sure the read ahead task will not close the underlying input stream. + isUnderlyingInputStreamBeingClosed = true; + } + } finally { + stateChangeLock.unlock(); + } + + try { + executorService.shutdownNow(); + executorService.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS); + } catch (InterruptedException e) { + InterruptedIOException iio = new InterruptedIOException(e.getMessage()); + iio.initCause(e); + throw iio; + } finally { + if (isSafeToCloseUnderlyingInputStream) { + underlyingInputStream.close(); + } + } + } +} diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index 008799cc7739..0efae16e9838 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -17,42 +17,49 @@ package org.apache.spark.memory; - import java.io.IOException; +import org.apache.spark.unsafe.array.LongArray; import org.apache.spark.unsafe.memory.MemoryBlock; - /** - * An memory consumer of TaskMemoryManager, which support spilling. + * A memory consumer of {@link TaskMemoryManager} that supports spilling. + * + * Note: this only supports allocation / spilling of Tungsten memory. */ public abstract class MemoryConsumer { - private final TaskMemoryManager taskMemoryManager; + protected final TaskMemoryManager taskMemoryManager; private final long pageSize; - private long used; + private final MemoryMode mode; + protected long used; - protected MemoryConsumer(TaskMemoryManager taskMemoryManager, long pageSize) { + protected MemoryConsumer(TaskMemoryManager taskMemoryManager, long pageSize, MemoryMode mode) { this.taskMemoryManager = taskMemoryManager; this.pageSize = pageSize; - this.used = 0; + this.mode = mode; } protected MemoryConsumer(TaskMemoryManager taskMemoryManager) { - this(taskMemoryManager, taskMemoryManager.pageSizeBytes()); + this(taskMemoryManager, taskMemoryManager.pageSizeBytes(), MemoryMode.ON_HEAP); + } + + /** + * Returns the memory mode, {@link MemoryMode#ON_HEAP} or {@link MemoryMode#OFF_HEAP}. + */ + public MemoryMode getMode() { + return mode; } /** * Returns the size of used memory in bytes. */ - long getUsed() { + protected long getUsed() { return used; } /** * Force spill during building. - * - * For testing. */ public void spill() throws IOException { spill(Long.MAX_VALUE, this); @@ -66,6 +73,8 @@ public void spill() throws IOException { * * Note: In order to avoid possible deadlock, should not call acquireMemory() from spill(). * + * Note: today, this only frees Tungsten-managed pages. + * * @param size the amount of memory should be released * @param trigger the MemoryConsumer that trigger this spilling * @return the amount of released memory in bytes @@ -74,45 +83,34 @@ public void spill() throws IOException { public abstract long spill(long size, MemoryConsumer trigger) throws IOException; /** - * Acquire `size` bytes memory. - * - * If there is not enough memory, throws OutOfMemoryError. + * Allocates a LongArray of `size`. */ - protected void acquireMemory(long size) { - long got = taskMemoryManager.acquireExecutionMemory(size, this); - if (got < size) { - taskMemoryManager.releaseExecutionMemory(got, this); - taskMemoryManager.showMemoryUsage(); - throw new OutOfMemoryError("Could not acquire " + size + " bytes of memory, got " + got); + public LongArray allocateArray(long size) { + long required = size * 8L; + MemoryBlock page = taskMemoryManager.allocatePage(required, this); + if (page == null || page.size() < required) { + throwOom(page, required); } - used += got; + used += required; + return new LongArray(page); } /** - * Release `size` bytes memory. + * Frees a LongArray. */ - protected void releaseMemory(long size) { - used -= size; - taskMemoryManager.releaseExecutionMemory(size, this); + public void freeArray(LongArray array) { + freePage(array.memoryBlock()); } /** * Allocate a memory block with at least `required` bytes. * - * Throws IOException if there is not enough memory. - * * @throws OutOfMemoryError */ protected MemoryBlock allocatePage(long required) { MemoryBlock page = taskMemoryManager.allocatePage(Math.max(pageSize, required), this); if (page == null || page.size() < required) { - long got = 0; - if (page != null) { - got = page.size(); - freePage(page); - } - taskMemoryManager.showMemoryUsage(); - throw new OutOfMemoryError("Unable to acquire " + required + " bytes of memory, got " + got); + throwOom(page, required); } used += page.size(); return page; @@ -125,4 +123,31 @@ protected void freePage(MemoryBlock page) { used -= page.size(); taskMemoryManager.freePage(page, this); } + + /** + * Allocates memory of `size`. + */ + public long acquireMemory(long size) { + long granted = taskMemoryManager.acquireExecutionMemory(size, this); + used += granted; + return granted; + } + + /** + * Release N bytes of memory. + */ + public void freeMemory(long size) { + taskMemoryManager.releaseExecutionMemory(size, this); + used -= size; + } + + private void throwOom(final MemoryBlock page, final long required) { + long got = 0; + if (page != null) { + got = page.size(); + taskMemoryManager.freePage(page, this); + } + taskMemoryManager.showMemoryUsage(); + throw new OutOfMemoryError("Unable to acquire " + required + " bytes of memory, got " + got); + } } diff --git a/core/src/main/java/org/apache/spark/memory/MemoryMode.java b/core/src/main/java/org/apache/spark/memory/MemoryMode.java new file mode 100644 index 000000000000..3a5e72d8aaec --- /dev/null +++ b/core/src/main/java/org/apache/spark/memory/MemoryMode.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.memory; + +import org.apache.spark.annotation.Private; + +@Private +public enum MemoryMode { + ON_HEAP, + OFF_HEAP +} diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 4230575446d3..44b60c1e4e8c 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -19,9 +19,14 @@ import javax.annotation.concurrent.GuardedBy; import java.io.IOException; +import java.nio.channels.ClosedByInterruptException; import java.util.Arrays; +import java.util.ArrayList; import java.util.BitSet; import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; @@ -48,12 +53,12 @@ * retrieve the base object. *

* This allows us to address 8192 pages. In on-heap mode, the maximum page size is limited by the - * maximum size of a long[] array, allowing us to address 8192 * 2^32 * 8 bytes, which is - * approximately 35 terabytes of memory. + * maximum size of a long[] array, allowing us to address 8192 * (2^31 - 1) * 8 bytes, which is + * approximately 140 terabytes of memory. */ public class TaskMemoryManager { - private final Logger logger = LoggerFactory.getLogger(TaskMemoryManager.class); + private static final Logger logger = LoggerFactory.getLogger(TaskMemoryManager.class); /** The number of bits used to address the page table. */ private static final int PAGE_NUMBER_BITS = 13; @@ -67,18 +72,16 @@ public class TaskMemoryManager { /** * Maximum supported data page size (in bytes). In principle, the maximum addressable page size is - * (1L << OFFSET_BITS) bytes, which is 2+ petabytes. However, the on-heap allocator's maximum page - * size is limited by the maximum amount of data that can be stored in a long[] array, which is - * (2^32 - 1) * 8 bytes (or 16 gigabytes). Therefore, we cap this at 16 gigabytes. + * (1L << OFFSET_BITS) bytes, which is 2+ petabytes. However, the on-heap allocator's + * maximum page size is limited by the maximum amount of data that can be stored in a long[] + * array, which is (2^31 - 1) * 8 bytes (or about 17 gigabytes). Therefore, we cap this at 17 + * gigabytes. */ public static final long MAXIMUM_PAGE_SIZE_BYTES = ((1L << 31) - 1) * 8L; /** Bit mask for the lower 51 bits of a long. */ private static final long MASK_LONG_LOWER_51_BITS = 0x7FFFFFFFFFFFFL; - /** Bit mask for the upper 13 bits of a long */ - private static final long MASK_LONG_UPPER_13_BITS = ~MASK_LONG_LOWER_51_BITS; - /** * Similar to an operating system's page table, this array maps page numbers into base object * pointers, allowing us to translate between the hashtable's internal 64-bit address @@ -103,19 +106,24 @@ public class TaskMemoryManager { * without doing any masking or lookups. Since this branching should be well-predicted by the JIT, * this extra layer of indirection / abstraction hopefully shouldn't be too expensive. */ - private final boolean inHeap; + final MemoryMode tungstenMemoryMode; /** - * The size of memory granted to each consumer. + * Tracks spillable memory consumers. */ @GuardedBy("this") private final HashSet consumers; + /** + * The amount of memory that is acquired but not used. + */ + private volatile long acquiredButNotUsed = 0L; + /** * Construct a new TaskMemoryManager. */ public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) { - this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap(); + this.tungstenMemoryMode = memoryManager.tungstenMemoryMode(); this.memoryManager = memoryManager; this.taskAttemptId = taskAttemptId; this.consumers = new HashSet<>(); @@ -129,43 +137,80 @@ public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) { */ public long acquireExecutionMemory(long required, MemoryConsumer consumer) { assert(required >= 0); + assert(consumer != null); + MemoryMode mode = consumer.getMode(); + // If we are allocating Tungsten pages off-heap and receive a request to allocate on-heap + // memory here, then it may not make sense to spill since that would only end up freeing + // off-heap memory. This is subject to change, though, so it may be risky to make this + // optimization now in case we forget to undo it late when making changes. synchronized (this) { - long got = memoryManager.acquireExecutionMemory(required, taskAttemptId); + long got = memoryManager.acquireExecutionMemory(required, taskAttemptId, mode); - // try to release memory from other consumers first, then we can reduce the frequency of + // Try to release memory from other consumers first, then we can reduce the frequency of // spilling, avoid to have too many spilled files. if (got < required) { // Call spill() on other consumers to release memory + // Sort the consumers according their memory usage. So we avoid spilling the same consumer + // which is just spilled in last few times and re-spilling on it will produce many small + // spill files. + TreeMap> sortedConsumers = new TreeMap<>(); for (MemoryConsumer c: consumers) { - if (c != null && c != consumer && c.getUsed() > 0) { - try { - long released = c.spill(required - got, consumer); - if (released > 0) { - logger.info("Task {} released {} from {} for {}", taskAttemptId, - Utils.bytesToString(released), c, consumer); - got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId); - if (got >= required) { - break; - } + if (c != consumer && c.getUsed() > 0 && c.getMode() == mode) { + long key = c.getUsed(); + List list = + sortedConsumers.computeIfAbsent(key, k -> new ArrayList<>(1)); + list.add(c); + } + } + while (!sortedConsumers.isEmpty()) { + // Get the consumer using the least memory more than the remaining required memory. + Map.Entry> currentEntry = + sortedConsumers.ceilingEntry(required - got); + // No consumer has used memory more than the remaining required memory. + // Get the consumer of largest used memory. + if (currentEntry == null) { + currentEntry = sortedConsumers.lastEntry(); + } + List cList = currentEntry.getValue(); + MemoryConsumer c = cList.remove(cList.size() - 1); + if (cList.isEmpty()) { + sortedConsumers.remove(currentEntry.getKey()); + } + try { + long released = c.spill(required - got, consumer); + if (released > 0) { + logger.debug("Task {} released {} from {} for {}", taskAttemptId, + Utils.bytesToString(released), c, consumer); + got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); + if (got >= required) { + break; } - } catch (IOException e) { - logger.error("error while calling spill() on " + c, e); - throw new OutOfMemoryError("error while calling spill() on " + c + " : " - + e.getMessage()); } + } catch (ClosedByInterruptException e) { + // This called by user to kill a task (e.g: speculative task). + logger.error("error while calling spill() on " + c, e); + throw new RuntimeException(e.getMessage()); + } catch (IOException e) { + logger.error("error while calling spill() on " + c, e); + throw new OutOfMemoryError("error while calling spill() on " + c + " : " + + e.getMessage()); } } } // call spill() on itself - if (got < required && consumer != null) { + if (got < required) { try { long released = consumer.spill(required - got, consumer); if (released > 0) { - logger.info("Task {} released {} from itself ({})", taskAttemptId, + logger.debug("Task {} released {} from itself ({})", taskAttemptId, Utils.bytesToString(released), consumer); - got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId); + got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); } + } catch (ClosedByInterruptException e) { + // This called by user to kill a task (e.g: speculative task). + logger.error("error while calling spill() on " + consumer, e); + throw new RuntimeException(e.getMessage()); } catch (IOException e) { logger.error("error while calling spill() on " + consumer, e); throw new OutOfMemoryError("error while calling spill() on " + consumer + " : " @@ -174,7 +219,7 @@ public long acquireExecutionMemory(long required, MemoryConsumer consumer) { } consumers.add(consumer); - logger.debug("Task {} acquire {} for {}", taskAttemptId, Utils.bytesToString(got), consumer); + logger.debug("Task {} acquired {} for {}", taskAttemptId, Utils.bytesToString(got), consumer); return got; } } @@ -184,7 +229,7 @@ public long acquireExecutionMemory(long required, MemoryConsumer consumer) { */ public void releaseExecutionMemory(long size, MemoryConsumer consumer) { logger.debug("Task {} release {} from {}", taskAttemptId, Utils.bytesToString(size), consumer); - memoryManager.releaseExecutionMemory(size, taskAttemptId); + memoryManager.releaseExecutionMemory(size, taskAttemptId, consumer.getMode()); } /** @@ -193,11 +238,22 @@ public void releaseExecutionMemory(long size, MemoryConsumer consumer) { public void showMemoryUsage() { logger.info("Memory used in task " + taskAttemptId); synchronized (this) { + long memoryAccountedForByConsumers = 0; for (MemoryConsumer c: consumers) { - if (c.getUsed() > 0) { - logger.info("Acquired by " + c + ": " + Utils.bytesToString(c.getUsed())); + long totalMemUsage = c.getUsed(); + memoryAccountedForByConsumers += totalMemUsage; + if (totalMemUsage > 0) { + logger.info("Acquired by " + c + ": " + Utils.bytesToString(totalMemUsage)); } } + long memoryNotAccountedFor = + memoryManager.getExecutionMemoryUsageForTask(taskAttemptId) - memoryAccountedForByConsumers; + logger.info( + "{} bytes of memory were used by task {} but are not associated with specific consumers", + memoryNotAccountedFor, taskAttemptId); + logger.info( + "{} bytes of memory are used for execution and {} bytes of memory are used for storage", + memoryManager.executionMemoryUsed(), memoryManager.storageMemoryUsed()); } } @@ -212,9 +268,12 @@ public long pageSizeBytes() { * Allocate a block of memory that will be tracked in the MemoryManager's page table; this is * intended for allocating large blocks of Tungsten memory that will be shared between operators. * - * Returns `null` if there was not enough memory to allocate the page. + * Returns `null` if there was not enough memory to allocate the page. May return a page that + * contains fewer bytes than requested, so callers should verify the size of returned pages. */ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { + assert(consumer != null); + assert(consumer.getMode() == tungstenMemoryMode); if (size > MAXIMUM_PAGE_SIZE_BYTES) { throw new IllegalArgumentException( "Cannot allocate a page with more than " + MAXIMUM_PAGE_SIZE_BYTES + " bytes"); @@ -235,7 +294,20 @@ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { } allocatedPages.set(pageNumber); } - final MemoryBlock page = memoryManager.tungstenMemoryAllocator().allocate(acquired); + MemoryBlock page = null; + try { + page = memoryManager.tungstenMemoryAllocator().allocate(acquired); + } catch (OutOfMemoryError e) { + logger.warn("Failed to allocate a page ({} bytes), try again.", acquired); + // there is no enough memory actually, it means the actual free memory is smaller than + // MemoryManager thought, we should keep the acquired memory. + synchronized (this) { + acquiredButNotUsed += acquired; + allocatedPages.clear(pageNumber); + } + // this could trigger spilling to free some pages. + return allocatePage(size, consumer); + } page.pageNumber = pageNumber; pageTable[pageNumber] = page; if (logger.isTraceEnabled()) { @@ -274,7 +346,7 @@ public void freePage(MemoryBlock page, MemoryConsumer consumer) { * @return an encoded page address. */ public long encodePageNumberAndOffset(MemoryBlock page, long offsetInPage) { - if (!inHeap) { + if (tungstenMemoryMode == MemoryMode.OFF_HEAP) { // In off-heap mode, an offset is an absolute address that may require a full 64 bits to // encode. Due to our page size limitation, though, we can convert this into an offset that's // relative to the page's base offset; this relative offset will fit in 51 bits. @@ -291,7 +363,7 @@ public static long encodePageNumberAndOffset(int pageNumber, long offsetInPage) @VisibleForTesting public static int decodePageNumber(long pagePlusOffsetAddress) { - return (int) ((pagePlusOffsetAddress & MASK_LONG_UPPER_13_BITS) >>> OFFSET_BITS); + return (int) (pagePlusOffsetAddress >>> OFFSET_BITS); } private static long decodeOffset(long pagePlusOffsetAddress) { @@ -303,7 +375,7 @@ private static long decodeOffset(long pagePlusOffsetAddress) { * {@link TaskMemoryManager#encodePageNumberAndOffset(MemoryBlock, long)} */ public Object getPage(long pagePlusOffsetAddress) { - if (inHeap) { + if (tungstenMemoryMode == MemoryMode.ON_HEAP) { final int pageNumber = decodePageNumber(pagePlusOffsetAddress); assert (pageNumber >= 0 && pageNumber < PAGE_TABLE_SIZE); final MemoryBlock page = pageTable[pageNumber]; @@ -321,7 +393,7 @@ public Object getPage(long pagePlusOffsetAddress) { */ public long getOffsetInPage(long pagePlusOffsetAddress) { final long offsetInPage = decodeOffset(pagePlusOffsetAddress); - if (inHeap) { + if (tungstenMemoryMode == MemoryMode.ON_HEAP) { return offsetInPage; } else { // In off-heap mode, an offset is an absolute address. In encodePageNumberAndOffset, we @@ -340,22 +412,40 @@ public long getOffsetInPage(long pagePlusOffsetAddress) { */ public long cleanUpAllAllocatedMemory() { synchronized (this) { - Arrays.fill(pageTable, null); for (MemoryConsumer c: consumers) { if (c != null && c.getUsed() > 0) { // In case of failed task, it's normal to see leaked memory - logger.warn("leak " + Utils.bytesToString(c.getUsed()) + " memory from " + c); + logger.debug("unreleased " + Utils.bytesToString(c.getUsed()) + " memory from " + c); } } consumers.clear(); + + for (MemoryBlock page : pageTable) { + if (page != null) { + logger.debug("unreleased page: " + page + " in task " + taskAttemptId); + memoryManager.tungstenMemoryAllocator().free(page); + } + } + Arrays.fill(pageTable, null); } + + // release the memory that is not used by any consumer (acquired for pages in tungsten mode). + memoryManager.releaseExecutionMemory(acquiredButNotUsed, taskAttemptId, tungstenMemoryMode); + return memoryManager.releaseAllExecutionMemoryForTask(taskAttemptId); } /** - * Returns the memory consumption, in bytes, for the current task + * Returns the memory consumption, in bytes, for the current task. */ public long getMemoryConsumptionForThisTask() { return memoryManager.getExecutionMemoryUsageForTask(taskAttemptId); } + + /** + * Returns Tungsten memory mode + */ + public MemoryMode getTungstenMemoryMode() { + return tungstenMemoryMode; + } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index ee82d679935c..a9b5236ab817 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -18,9 +18,9 @@ package org.apache.spark.shuffle.sort; import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; +import java.nio.channels.FileChannel; +import static java.nio.file.StandardOpenOption.*; import javax.annotation.Nullable; import scala.None$; @@ -52,8 +52,7 @@ * This class implements sort-based shuffle's hash-style shuffle fallback path. This write path * writes incoming records to separate files, one file per reduce partition, then concatenates these * per-partition files to form a single output file, regions of which are served to reducers. - * Records are not buffered in memory. This is essentially identical to - * {@link org.apache.spark.shuffle.hash.HashShuffleWriter}, except that it writes output in a format + * Records are not buffered in memory. It writes output in a format * that can be served / consumed via {@link org.apache.spark.shuffle.IndexShuffleBlockResolver}. *

* This write path is inefficient for shuffles with large numbers of reduce partitions because it @@ -61,7 +60,7 @@ * {@link SortShuffleManager} only selects this write path when *

    *
  • no Ordering is specified,
  • - *
  • no Aggregator is specific, and
  • + *
  • no Aggregator is specified, and
  • *
  • the number of partitions is less than * spark.shuffle.sort.bypassMergeThreshold.
  • *
@@ -73,10 +72,9 @@ */ final class BypassMergeSortShuffleWriter extends ShuffleWriter { - private final Logger logger = LoggerFactory.getLogger(BypassMergeSortShuffleWriter.class); + private static final Logger logger = LoggerFactory.getLogger(BypassMergeSortShuffleWriter.class); private final int fileBufferSize; - private final boolean transferToEnabled; private final int numPartitions; private final BlockManager blockManager; private final Partitioner partitioner; @@ -88,6 +86,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { /** Array of file writers, one for each partition */ private DiskBlockObjectWriter[] partitionWriters; + private FileSegment[] partitionWriterSegments; @Nullable private MapStatus mapStatus; private long[] partitionLengths; @@ -98,7 +97,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { */ private boolean stopping = false; - public BypassMergeSortShuffleWriter( + BypassMergeSortShuffleWriter( BlockManager blockManager, IndexShuffleBlockResolver shuffleBlockResolver, BypassMergeSortShuffleHandle handle, @@ -107,16 +106,14 @@ public BypassMergeSortShuffleWriter( SparkConf conf) { // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided this.fileBufferSize = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024; - this.transferToEnabled = conf.getBoolean("spark.file.transferTo", true); this.blockManager = blockManager; final ShuffleDependency dep = handle.dependency(); this.mapId = mapId; this.shuffleId = dep.shuffleId(); this.partitioner = dep.partitioner(); this.numPartitions = partitioner.numPartitions(); - this.writeMetrics = new ShuffleWriteMetrics(); - taskContext.taskMetrics().shuffleWriteMetrics_$eq(Option.apply(writeMetrics)); - this.serializer = Serializer.getSerializer(dep.serializer()); + this.writeMetrics = taskContext.taskMetrics().shuffleWriteMetrics(); + this.serializer = dep.serializer(); this.shuffleBlockResolver = shuffleBlockResolver; } @@ -125,25 +122,26 @@ public void write(Iterator> records) throws IOException { assert (partitionWriters == null); if (!records.hasNext()) { partitionLengths = new long[numPartitions]; - shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); + shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, null); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); return; } final SerializerInstance serInstance = serializer.newInstance(); final long openStartTime = System.nanoTime(); partitionWriters = new DiskBlockObjectWriter[numPartitions]; + partitionWriterSegments = new FileSegment[numPartitions]; for (int i = 0; i < numPartitions; i++) { final Tuple2 tempShuffleBlockIdPlusFile = blockManager.diskBlockManager().createTempShuffleBlock(); final File file = tempShuffleBlockIdPlusFile._2(); final BlockId blockId = tempShuffleBlockIdPlusFile._1(); partitionWriters[i] = - blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, writeMetrics).open(); + blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, writeMetrics); } // Creating the file to write to and creating a disk writer both involve interacting with // the disk, and can take a long time in aggregate when we open many files, so should be // included in the shuffle write time. - writeMetrics.incShuffleWriteTime(System.nanoTime() - openStartTime); + writeMetrics.incWriteTime(System.nanoTime() - openStartTime); while (records.hasNext()) { final Product2 record = records.next(); @@ -151,13 +149,22 @@ public void write(Iterator> records) throws IOException { partitionWriters[partitioner.getPartition(key)].write(key, record._2()); } - for (DiskBlockObjectWriter writer : partitionWriters) { - writer.commitAndClose(); + for (int i = 0; i < numPartitions; i++) { + final DiskBlockObjectWriter writer = partitionWriters[i]; + partitionWriterSegments[i] = writer.commitAndGet(); + writer.close(); } - partitionLengths = - writePartitionedFile(shuffleBlockResolver.getDataFile(shuffleId, mapId)); - shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); + File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); + File tmp = Utils.tempFileWith(output); + try { + partitionLengths = writePartitionedFile(tmp); + shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); + } finally { + if (tmp.exists() && !tmp.delete()) { + logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + } + } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } @@ -179,27 +186,34 @@ private long[] writePartitionedFile(File outputFile) throws IOException { return lengths; } - final FileOutputStream out = new FileOutputStream(outputFile, true); + // This file needs to opened in append mode in order to work around a Linux kernel bug that + // affects transferTo; see SPARK-3948 for more details. + final FileChannel out = FileChannel.open(outputFile.toPath(), WRITE, APPEND, CREATE); final long writeStartTime = System.nanoTime(); boolean threwException = true; try { for (int i = 0; i < numPartitions; i++) { - final FileInputStream in = new FileInputStream(partitionWriters[i].fileSegment().file()); - boolean copyThrewException = true; - try { - lengths[i] = Utils.copyStream(in, out, false, transferToEnabled); - copyThrewException = false; - } finally { - Closeables.close(in, copyThrewException); - } - if (!partitionWriters[i].fileSegment().file().delete()) { - logger.error("Unable to delete file for partition {}", i); + final File file = partitionWriterSegments[i].file(); + if (file.exists()) { + final FileChannel in = FileChannel.open(file.toPath(), READ); + boolean copyThrewException = true; + try { + long size = in.size(); + Utils.copyFileStreamNIO(in, out, 0, size); + lengths[i] = size; + copyThrewException = false; + } finally { + Closeables.close(in, copyThrewException); + } + if (!file.delete()) { + logger.error("Unable to delete file for partition {}", i); + } } } threwException = false; } finally { Closeables.close(out, threwException); - writeMetrics.incShuffleWriteTime(System.nanoTime() - writeStartTime); + writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); } partitionWriters = null; return lengths; @@ -231,7 +245,6 @@ public Option stop(boolean success) { partitionWriters = null; } } - shuffleBlockResolver.removeDataByMap(shuffleId, mapId); return None$.empty(); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/PackedRecordPointer.java b/core/src/main/java/org/apache/spark/shuffle/sort/PackedRecordPointer.java index f8f2b220e181..b36da80dbcff 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/PackedRecordPointer.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/PackedRecordPointer.java @@ -17,8 +17,6 @@ package org.apache.spark.shuffle.sort; -import org.apache.spark.memory.TaskMemoryManager; - /** * Wrapper around an 8-byte word that holds a 24-bit partition number and 40-bit record pointer. *

@@ -28,7 +26,7 @@ *

* This implies that the maximum addressable page size is 2^27 bits = 128 megabytes, assuming that * our offsets in pages are not 8-byte-word-aligned. Since we have 2^13 pages (based off the - * 13-bit page numbers assigned by {@link TaskMemoryManager}), this + * 13-bit page numbers assigned by {@link org.apache.spark.memory.TaskMemoryManager}), this * implies that we can address 2^13 * 128 megabytes = 1 terabyte of RAM per task. *

* Assuming word-alignment would allow for a 1 gigabyte maximum page size, but we leave this @@ -44,6 +42,16 @@ final class PackedRecordPointer { */ static final int MAXIMUM_PARTITION_ID = (1 << 24) - 1; // 16777215 + /** + * The index of the first byte of the partition id, counting from the least significant byte. + */ + static final int PARTITION_ID_START_BYTE_INDEX = 5; + + /** + * The index of the last byte of the partition id, counting from the least significant byte. + */ + static final int PARTITION_ID_END_BYTE_INDEX = 7; + /** Bit mask for the lower 40 bits of a long. */ private static final long MASK_LONG_LOWER_40_BITS = (1L << 40) - 1; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 400d8520019b..b4f46306f282 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -37,10 +37,13 @@ import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.storage.BlockManager; import org.apache.spark.storage.DiskBlockObjectWriter; +import org.apache.spark.storage.FileSegment; import org.apache.spark.storage.TempShuffleBlockId; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.array.LongArray; import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.util.Utils; +import org.apache.spark.internal.config.package$; /** * An external sorter that is specialized for sort-based shuffle. @@ -60,7 +63,7 @@ */ final class ShuffleExternalSorter extends MemoryConsumer { - private final Logger logger = LoggerFactory.getLogger(ShuffleExternalSorter.class); + private static final Logger logger = LoggerFactory.getLogger(ShuffleExternalSorter.class); @VisibleForTesting static final int DISK_WRITE_BUFFER_SIZE = 1024 * 1024; @@ -71,21 +74,27 @@ final class ShuffleExternalSorter extends MemoryConsumer { private final TaskContext taskContext; private final ShuffleWriteMetrics writeMetrics; - /** Force this sorter to spill when there are this many elements in memory. For testing only */ + /** + * Force this sorter to spill when there are this many elements in memory. The default value is + * 1024 * 1024 * 1024, which allows the maximum size of the pointer array to be 8G. + */ private final long numElementsForSpillThreshold; /** The buffer size to use when writing spills using DiskBlockObjectWriter */ private final int fileBufferSizeBytes; + /** The buffer size to use when writing the sorted records to an on-disk file */ + private final int diskWriteBufferSize; + /** * Memory pages that hold the records being sorted. The pages in this list are freed when * spilling, although in principle we could recycle these pages across spills (on the other hand, * this might not be necessary if we maintained a pool of re-usable pages in the TaskMemoryManager * itself). */ - private final LinkedList allocatedPages = new LinkedList(); + private final LinkedList allocatedPages = new LinkedList<>(); - private final LinkedList spills = new LinkedList(); + private final LinkedList spills = new LinkedList<>(); /** Peak memory used by this sorter so far, in bytes. **/ private long peakMemoryUsedBytes; @@ -95,7 +104,7 @@ final class ShuffleExternalSorter extends MemoryConsumer { @Nullable private MemoryBlock currentPage = null; private long pageCursor = -1; - public ShuffleExternalSorter( + ShuffleExternalSorter( TaskMemoryManager memoryManager, BlockManager blockManager, TaskContext taskContext, @@ -103,20 +112,24 @@ public ShuffleExternalSorter( int numPartitions, SparkConf conf, ShuffleWriteMetrics writeMetrics) { - super(memoryManager, (int) Math.min(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, - memoryManager.pageSizeBytes())); + super(memoryManager, + (int) Math.min(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, memoryManager.pageSizeBytes()), + memoryManager.getTungstenMemoryMode()); this.taskMemoryManager = memoryManager; this.blockManager = blockManager; this.taskContext = taskContext; this.numPartitions = numPartitions; // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided - this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024; + this.fileBufferSizeBytes = + (int) (long) conf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; this.numElementsForSpillThreshold = - conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", Long.MAX_VALUE); + conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", 1024 * 1024 * 1024); this.writeMetrics = writeMetrics; - acquireMemory(initialSize * 8L); - this.inMemSorter = new ShuffleInMemorySorter(initialSize); + this.inMemSorter = new ShuffleInMemorySorter( + this, initialSize, conf.getBoolean("spark.shuffle.sort.useRadixSort", true)); this.peakMemoryUsedBytes = getMemoryUsage(); + this.diskWriteBufferSize = + (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); } /** @@ -127,7 +140,7 @@ public ShuffleExternalSorter( * bytes written should be counted towards shuffle spill metrics rather than * shuffle write metrics. */ - private void writeSortedFile(boolean isLastFile) throws IOException { + private void writeSortedFile(boolean isLastFile) { final ShuffleWriteMetrics writeMetricsToUse; @@ -145,15 +158,11 @@ private void writeSortedFile(boolean isLastFile) throws IOException { final ShuffleInMemorySorter.ShuffleSorterIterator sortedRecords = inMemSorter.getSortedIterator(); - // Currently, we need to open a new DiskBlockObjectWriter for each partition; we can avoid this - // after SPARK-5581 is fixed. - DiskBlockObjectWriter writer; - // Small writes to DiskBlockObjectWriter will be fairly inefficient. Since there doesn't seem to // be an API to directly transfer bytes from managed memory to the disk writer, we buffer // data through a byte array. This array does not need to be large enough to hold a single // record; - final byte[] writeBuffer = new byte[DISK_WRITE_BUFFER_SIZE]; + final byte[] writeBuffer = new byte[diskWriteBufferSize]; // Because this output will be read during shuffle, its compression codec must be controlled by // spark.shuffle.compress instead of spark.shuffle.spill.compress, so we need to use @@ -170,7 +179,8 @@ private void writeSortedFile(boolean isLastFile) throws IOException { // around this, we pass a dummy no-op serializer. final SerializerInstance ser = DummySerializerInstance.INSTANCE; - writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSizeBytes, writeMetricsToUse); + final DiskBlockObjectWriter writer = + blockManager.getDiskWriter(blockId, file, ser, fileBufferSizeBytes, writeMetricsToUse); int currentPartition = -1; while (sortedRecords.hasNext()) { @@ -180,12 +190,10 @@ private void writeSortedFile(boolean isLastFile) throws IOException { if (partition != currentPartition) { // Switch to the new partition if (currentPartition != -1) { - writer.commitAndClose(); - spillInfo.partitionLengths[currentPartition] = writer.fileSegment().length(); + final FileSegment fileSegment = writer.commitAndGet(); + spillInfo.partitionLengths[currentPartition] = fileSegment.length(); } currentPartition = partition; - writer = - blockManager.getDiskWriter(blockId, file, ser, fileBufferSizeBytes, writeMetricsToUse); } final long recordPointer = sortedRecords.packedRecordPointer.getRecordPointer(); @@ -194,7 +202,7 @@ private void writeSortedFile(boolean isLastFile) throws IOException { int dataRemaining = Platform.getInt(recordPage, recordOffsetInPage); long recordReadPosition = recordOffsetInPage + 4; // skip over record length while (dataRemaining > 0) { - final int toTransfer = Math.min(DISK_WRITE_BUFFER_SIZE, dataRemaining); + final int toTransfer = Math.min(diskWriteBufferSize, dataRemaining); Platform.copyMemory( recordPage, recordReadPosition, writeBuffer, Platform.BYTE_ARRAY_OFFSET, toTransfer); writer.write(writeBuffer, 0, toTransfer); @@ -204,19 +212,16 @@ private void writeSortedFile(boolean isLastFile) throws IOException { writer.recordWritten(); } - if (writer != null) { - writer.commitAndClose(); - // If `writeSortedFile()` was called from `closeAndGetSpills()` and no records were inserted, - // then the file might be empty. Note that it might be better to avoid calling - // writeSortedFile() in that case. - if (currentPartition != -1) { - spillInfo.partitionLengths[currentPartition] = writer.fileSegment().length(); - spills.add(spillInfo); - } + final FileSegment committedSegment = writer.commitAndGet(); + writer.close(); + // If `writeSortedFile()` was called from `closeAndGetSpills()` and no records were inserted, + // then the file might be empty. Note that it might be better to avoid calling + // writeSortedFile() in that case. + if (currentPartition != -1) { + spillInfo.partitionLengths[currentPartition] = committedSegment.length(); + spills.add(spillInfo); } - inMemSorter.reset(); - if (!isLastFile) { // i.e. this is a spill file // The current semantics of `shuffleRecordsWritten` seem to be that it's updated when records // are written to disk, not when they enter the shuffle sorting code. DiskBlockObjectWriter @@ -233,8 +238,8 @@ private void writeSortedFile(boolean isLastFile) throws IOException { // Note that we intentionally ignore the value of `writeMetricsToUse.shuffleWriteTime()`. // Consistent with ExternalSorter, we do not count this IO towards shuffle write time. // This means that this IO time is not accounted for anywhere; SPARK-3577 will fix this. - writeMetrics.incShuffleRecordsWritten(writeMetricsToUse.shuffleRecordsWritten()); - taskContext.taskMetrics().incDiskBytesSpilled(writeMetricsToUse.shuffleBytesWritten()); + writeMetrics.incRecordsWritten(writeMetricsToUse.recordsWritten()); + taskContext.taskMetrics().incDiskBytesSpilled(writeMetricsToUse.bytesWritten()); } } @@ -255,6 +260,10 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { writeSortedFile(false); final long spillSize = freeMemory(); + inMemSorter.reset(); + // Reset the in-memory sorter's pointer array only after freeing up the memory pages holding the + // records. Otherwise, if the task is over allocated memory, then without freeing the memory + // pages, we might not be able to get memory for the pointer array. taskContext.taskMetrics().incMemoryBytesSpilled(spillSize); return spillSize; } @@ -301,9 +310,8 @@ private long freeMemory() { public void cleanupResources() { freeMemory(); if (inMemSorter != null) { - long sorterMemoryUsage = inMemSorter.getMemoryUsage(); + inMemSorter.free(); inMemSorter = null; - releaseMemory(sorterMemoryUsage); } for (SpillInfo spill : spills) { if (spill.file.exists() && !spill.file.delete()) { @@ -317,30 +325,27 @@ public void cleanupResources() { * array and grows the array if additional space is required. If the required space cannot be * obtained, then the in-memory data will be spilled to disk. */ - private void growPointerArrayIfNecessary() throws IOException { + private void growPointerArrayIfNecessary() { assert(inMemSorter != null); if (!inMemSorter.hasSpaceForAnotherRecord()) { long used = inMemSorter.getMemoryUsage(); - long needed = used + inMemSorter.getMemoryToExpand(); + LongArray array; try { - acquireMemory(needed); // could trigger spilling + // could trigger spilling + array = allocateArray(used / 8 * 2); } catch (OutOfMemoryError e) { // should have trigger spilling - assert(inMemSorter.hasSpaceForAnotherRecord()); + if (!inMemSorter.hasSpaceForAnotherRecord()) { + logger.error("Unable to grow the pointer array"); + throw e; + } return; } // check if spilling is triggered or not if (inMemSorter.hasSpaceForAnotherRecord()) { - releaseMemory(needed); + freeArray(array); } else { - try { - inMemSorter.expandPointerArray(); - releaseMemory(used); - } catch (OutOfMemoryError oom) { - // Just in case that JVM had run out of memory - releaseMemory(needed); - spill(); - } + inMemSorter.expandPointerArray(array); } } } @@ -372,7 +377,9 @@ public void insertRecord(Object recordBase, long recordOffset, int length, int p // for tests assert(inMemSorter != null); - if (inMemSorter.numRecords() > numElementsForSpillThreshold) { + if (inMemSorter.numRecords() >= numElementsForSpillThreshold) { + logger.info("Spilling data because number of spilledRecords crossed the threshold " + + numElementsForSpillThreshold); spill(); } @@ -399,20 +406,14 @@ public void insertRecord(Object recordBase, long recordOffset, int length, int p * @throws IOException */ public SpillInfo[] closeAndGetSpills() throws IOException { - try { - if (inMemSorter != null) { - // Do not count the final file towards the spill count. - writeSortedFile(true); - freeMemory(); - long sorterMemoryUsage = inMemSorter.getMemoryUsage(); - inMemSorter = null; - releaseMemory(sorterMemoryUsage); - } - return spills.toArray(new SpillInfo[spills.size()]); - } catch (IOException e) { - cleanupResources(); - throw e; + if (inMemSorter != null) { + // Do not count the final file towards the spill count. + writeSortedFile(true); + freeMemory(); + inMemSorter.free(); + inMemSorter = null; } + return spills.toArray(new SpillInfo[spills.size()]); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java index e630575d1ae1..dc36809d8911 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java @@ -19,69 +19,110 @@ import java.util.Comparator; +import org.apache.spark.memory.MemoryConsumer; +import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.array.LongArray; +import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.util.collection.Sorter; +import org.apache.spark.util.collection.unsafe.sort.RadixSort; final class ShuffleInMemorySorter { - private final Sorter sorter; private static final class SortComparator implements Comparator { @Override public int compare(PackedRecordPointer left, PackedRecordPointer right) { - return left.getPartitionId() - right.getPartitionId(); + int leftId = left.getPartitionId(); + int rightId = right.getPartitionId(); + return leftId < rightId ? -1 : (leftId > rightId ? 1 : 0); } } private static final SortComparator SORT_COMPARATOR = new SortComparator(); + private final MemoryConsumer consumer; + /** * An array of record pointers and partition ids that have been encoded by * {@link PackedRecordPointer}. The sort operates on this array instead of directly manipulating * records. + * + * Only part of the array will be used to store the pointers, the rest part is preserved as + * temporary buffer for sorting. + */ + private LongArray array; + + /** + * Whether to use radix sort for sorting in-memory partition ids. Radix sort is much faster + * but requires additional memory to be reserved memory as pointers are added. */ - private long[] array; + private final boolean useRadixSort; /** * The position in the pointer array where new records can be inserted. */ private int pos = 0; - public ShuffleInMemorySorter(int initialSize) { + /** + * How many records could be inserted, because part of the array should be left for sorting. + */ + private int usableCapacity = 0; + + private int initialSize; + + ShuffleInMemorySorter(MemoryConsumer consumer, int initialSize, boolean useRadixSort) { + this.consumer = consumer; assert (initialSize > 0); - this.array = new long[initialSize]; - this.sorter = new Sorter<>(ShuffleSortDataFormat.INSTANCE); + this.initialSize = initialSize; + this.useRadixSort = useRadixSort; + this.array = consumer.allocateArray(initialSize); + this.usableCapacity = getUsableCapacity(); } - public int numRecords() { - return pos; + private int getUsableCapacity() { + // Radix sort requires same amount of used memory as buffer, Tim sort requires + // half of the used memory as buffer. + return (int) (array.size() / (useRadixSort ? 2 : 1.5)); } - public void reset() { - pos = 0; + public void free() { + if (array != null) { + consumer.freeArray(array); + array = null; + } } - private int newLength() { - // Guard against overflow: - return array.length <= Integer.MAX_VALUE / 2 ? (array.length * 2) : Integer.MAX_VALUE; + public int numRecords() { + return pos; } - /** - * Returns the memory needed to expand - */ - public long getMemoryToExpand() { - return ((long) (newLength() - array.length)) * 8; + public void reset() { + if (consumer != null) { + consumer.freeArray(array); + array = consumer.allocateArray(initialSize); + usableCapacity = getUsableCapacity(); + } + pos = 0; } - public void expandPointerArray() { - final long[] oldArray = array; - array = new long[newLength()]; - System.arraycopy(oldArray, 0, array, 0, oldArray.length); + public void expandPointerArray(LongArray newArray) { + assert(newArray.size() > array.size()); + Platform.copyMemory( + array.getBaseObject(), + array.getBaseOffset(), + newArray.getBaseObject(), + newArray.getBaseOffset(), + pos * 8L + ); + consumer.freeArray(array); + array = newArray; + usableCapacity = getUsableCapacity(); } public boolean hasSpaceForAnotherRecord() { - return pos < array.length; + return pos < usableCapacity; } public long getMemoryUsage() { - return array.length * 8L; + return array.size() * 8; } /** @@ -96,14 +137,9 @@ public long getMemoryUsage() { */ public void insertRecord(long recordPointer, int partitionId) { if (!hasSpaceForAnotherRecord()) { - if (array.length == Integer.MAX_VALUE) { - throw new IllegalStateException("Sort pointer array has reached maximum size"); - } else { - expandPointerArray(); - } + throw new IllegalStateException("There is no space for new record"); } - array[pos] = - PackedRecordPointer.packPointer(recordPointer, partitionId); + array.set(pos, PackedRecordPointer.packPointer(recordPointer, partitionId)); pos++; } @@ -112,22 +148,23 @@ public void insertRecord(long recordPointer, int partitionId) { */ public static final class ShuffleSorterIterator { - private final long[] pointerArray; - private final int numRecords; + private final LongArray pointerArray; + private final int limit; final PackedRecordPointer packedRecordPointer = new PackedRecordPointer(); private int position = 0; - public ShuffleSorterIterator(int numRecords, long[] pointerArray) { - this.numRecords = numRecords; + ShuffleSorterIterator(int numRecords, LongArray pointerArray, int startingPosition) { + this.limit = numRecords + startingPosition; this.pointerArray = pointerArray; + this.position = startingPosition; } public boolean hasNext() { - return position < numRecords; + return position < limit; } public void loadNext() { - packedRecordPointer.set(pointerArray[position]); + packedRecordPointer.set(pointerArray.get(position)); position++; } } @@ -136,7 +173,23 @@ public void loadNext() { * Return an iterator over record pointers in sorted order. */ public ShuffleSorterIterator getSortedIterator() { - sorter.sort(array, 0, pos, SORT_COMPARATOR); - return new ShuffleSorterIterator(pos, array); + int offset = 0; + if (useRadixSort) { + offset = RadixSort.sort( + array, pos, + PackedRecordPointer.PARTITION_ID_START_BYTE_INDEX, + PackedRecordPointer.PARTITION_ID_END_BYTE_INDEX, false, false); + } else { + MemoryBlock unused = new MemoryBlock( + array.getBaseObject(), + array.getBaseOffset() + pos * 8L, + (array.size() - pos) * 8L); + LongArray buffer = new LongArray(unused); + Sorter sorter = + new Sorter<>(new ShuffleSortDataFormat(buffer)); + + sorter.sort(array, 0, pos, SORT_COMPARATOR); + } + return new ShuffleSorterIterator(pos, array, offset); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java index 8a1e5aec6ff0..717bdd79d47e 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java @@ -17,16 +17,20 @@ package org.apache.spark.shuffle.sort; +import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.array.LongArray; import org.apache.spark.util.collection.SortDataFormat; -final class ShuffleSortDataFormat extends SortDataFormat { +final class ShuffleSortDataFormat extends SortDataFormat { - public static final ShuffleSortDataFormat INSTANCE = new ShuffleSortDataFormat(); + private final LongArray buffer; - private ShuffleSortDataFormat() { } + ShuffleSortDataFormat(LongArray buffer) { + this.buffer = buffer; + } @Override - public PackedRecordPointer getKey(long[] data, int pos) { + public PackedRecordPointer getKey(LongArray data, int pos) { // Since we re-use keys, this method shouldn't be called. throw new UnsupportedOperationException(); } @@ -37,31 +41,38 @@ public PackedRecordPointer newKey() { } @Override - public PackedRecordPointer getKey(long[] data, int pos, PackedRecordPointer reuse) { - reuse.set(data[pos]); + public PackedRecordPointer getKey(LongArray data, int pos, PackedRecordPointer reuse) { + reuse.set(data.get(pos)); return reuse; } @Override - public void swap(long[] data, int pos0, int pos1) { - final long temp = data[pos0]; - data[pos0] = data[pos1]; - data[pos1] = temp; + public void swap(LongArray data, int pos0, int pos1) { + final long temp = data.get(pos0); + data.set(pos0, data.get(pos1)); + data.set(pos1, temp); } @Override - public void copyElement(long[] src, int srcPos, long[] dst, int dstPos) { - dst[dstPos] = src[srcPos]; + public void copyElement(LongArray src, int srcPos, LongArray dst, int dstPos) { + dst.set(dstPos, src.get(srcPos)); } @Override - public void copyRange(long[] src, int srcPos, long[] dst, int dstPos, int length) { - System.arraycopy(src, srcPos, dst, dstPos, length); + public void copyRange(LongArray src, int srcPos, LongArray dst, int dstPos, int length) { + Platform.copyMemory( + src.getBaseObject(), + src.getBaseOffset() + srcPos * 8L, + dst.getBaseObject(), + dst.getBaseOffset() + dstPos * 8L, + length * 8L + ); } @Override - public long[] allocate(int length) { - return new long[length]; + public LongArray allocate(int length) { + assert (length <= buffer.size()) : + "the buffer is smaller than required: " + buffer.size() + " < " + length; + return buffer; } - } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/SpillInfo.java b/core/src/main/java/org/apache/spark/shuffle/sort/SpillInfo.java index df9f7b7abe02..865def6b83c5 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/SpillInfo.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/SpillInfo.java @@ -29,7 +29,7 @@ final class SpillInfo { final File file; final TempShuffleBlockId blockId; - public SpillInfo(int numPartitions, File file, TempShuffleBlockId blockId) { + SpillInfo(int numPartitions, File file, TempShuffleBlockId blockId) { this.partitionLengths = new long[numPartitions]; this.file = file; this.blockId = blockId; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 6a0a89e81c32..e9c2a69c47cb 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -20,12 +20,12 @@ import javax.annotation.Nullable; import java.io.*; import java.nio.channels.FileChannel; +import static java.nio.file.StandardOpenOption.*; import java.util.Iterator; import scala.Option; import scala.Product2; import scala.collection.JavaConverters; -import scala.collection.immutable.Map; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; @@ -41,29 +41,33 @@ import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.io.CompressionCodec; import org.apache.spark.io.CompressionCodec$; -import org.apache.spark.io.LZFCompressionCodec; +import org.apache.spark.io.NioBufferedFileInputStream; +import org.apache.commons.io.output.CloseShieldOutputStream; +import org.apache.commons.io.output.CountingOutputStream; +import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.network.util.LimitedInputStream; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; import org.apache.spark.serializer.SerializationStream; -import org.apache.spark.serializer.Serializer; import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.storage.BlockManager; import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; -import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.util.Utils; +import org.apache.spark.internal.config.package$; @Private public class UnsafeShuffleWriter extends ShuffleWriter { - private final Logger logger = LoggerFactory.getLogger(UnsafeShuffleWriter.class); + private static final Logger logger = LoggerFactory.getLogger(UnsafeShuffleWriter.class); private static final ClassTag OBJECT_CLASS_TAG = ClassTag$.MODULE$.Object(); @VisibleForTesting - static final int INITIAL_SORT_BUFFER_SIZE = 4096; + static final int DEFAULT_INITIAL_SORT_BUFFER_SIZE = 4096; + static final int DEFAULT_INITIAL_SER_BUFFER_SIZE = 1024 * 1024; private final BlockManager blockManager; private final IndexShuffleBlockResolver shuffleBlockResolver; @@ -76,6 +80,9 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final TaskContext taskContext; private final SparkConf sparkConf; private final boolean transferToEnabled; + private final int initialSortBufferSize; + private final int inputBufferSizeInBytes; + private final int outputBufferSizeInBytes; @Nullable private MapStatus mapStatus; @Nullable private ShuffleExternalSorter sorter; @@ -83,7 +90,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { /** Subclass of ByteArrayOutputStream that exposes `buf` directly. */ private static final class MyByteArrayOutputStream extends ByteArrayOutputStream { - public MyByteArrayOutputStream(int size) { super(size); } + MyByteArrayOutputStream(int size) { super(size); } public byte[] getBuf() { return buf; } } @@ -97,6 +104,18 @@ private static final class MyByteArrayOutputStream extends ByteArrayOutputStream */ private boolean stopping = false; + private class CloseAndFlushShieldOutputStream extends CloseShieldOutputStream { + + CloseAndFlushShieldOutputStream(OutputStream outputStream) { + super(outputStream); + } + + @Override + public void flush() { + // do nothing + } + } + public UnsafeShuffleWriter( BlockManager blockManager, IndexShuffleBlockResolver shuffleBlockResolver, @@ -109,7 +128,8 @@ public UnsafeShuffleWriter( if (numPartitions > SortShuffleManager.MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE()) { throw new IllegalArgumentException( "UnsafeShuffleWriter can only be used for shuffles with at most " + - SortShuffleManager.MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE() + " reduce partitions"); + SortShuffleManager.MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE() + + " reduce partitions"); } this.blockManager = blockManager; this.shuffleBlockResolver = shuffleBlockResolver; @@ -117,13 +137,18 @@ public UnsafeShuffleWriter( this.mapId = mapId; final ShuffleDependency dep = handle.dependency(); this.shuffleId = dep.shuffleId(); - this.serializer = Serializer.getSerializer(dep.serializer()).newInstance(); + this.serializer = dep.serializer().newInstance(); this.partitioner = dep.partitioner(); - this.writeMetrics = new ShuffleWriteMetrics(); - taskContext.taskMetrics().shuffleWriteMetrics_$eq(Option.apply(writeMetrics)); + this.writeMetrics = taskContext.taskMetrics().shuffleWriteMetrics(); this.taskContext = taskContext; this.sparkConf = sparkConf; this.transferToEnabled = sparkConf.getBoolean("spark.file.transferTo", true); + this.initialSortBufferSize = sparkConf.getInt("spark.shuffle.sort.initialBufferSize", + DEFAULT_INITIAL_SORT_BUFFER_SIZE); + this.inputBufferSizeInBytes = + (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; + this.outputBufferSizeInBytes = + (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE()) * 1024; open(); } @@ -183,17 +208,17 @@ public void write(scala.collection.Iterator> records) throws IOEx } } - private void open() throws IOException { + private void open() { assert (sorter == null); sorter = new ShuffleExternalSorter( memoryManager, blockManager, taskContext, - INITIAL_SORT_BUFFER_SIZE, + initialSortBufferSize, partitioner.numPartitions(), sparkConf, writeMetrics); - serBuffer = new MyByteArrayOutputStream(1024 * 1024); + serBuffer = new MyByteArrayOutputStream(DEFAULT_INITIAL_SER_BUFFER_SIZE); serOutputStream = serializer.serializeStream(serBuffer); } @@ -206,16 +231,24 @@ void closeAndWriteOutput() throws IOException { final SpillInfo[] spills = sorter.closeAndGetSpills(); sorter = null; final long[] partitionLengths; + final File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); + final File tmp = Utils.tempFileWith(output); try { - partitionLengths = mergeSpills(spills); - } finally { - for (SpillInfo spill : spills) { - if (spill.file.exists() && ! spill.file.delete()) { - logger.error("Error while deleting spill file {}", spill.file.getPath()); + try { + partitionLengths = mergeSpills(spills, tmp); + } finally { + for (SpillInfo spill : spills) { + if (spill.file.exists() && ! spill.file.delete()) { + logger.error("Error while deleting spill file {}", spill.file.getPath()); + } } } + shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); + } finally { + if (tmp.exists() && !tmp.delete()) { + logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + } } - shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } @@ -248,17 +281,17 @@ void forceSorterToSpill() throws IOException { * * @return the partition lengths in the merged file. */ - private long[] mergeSpills(SpillInfo[] spills) throws IOException { - final File outputFile = shuffleBlockResolver.getDataFile(shuffleId, mapId); + private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOException { final boolean compressionEnabled = sparkConf.getBoolean("spark.shuffle.compress", true); final CompressionCodec compressionCodec = CompressionCodec$.MODULE$.createCodec(sparkConf); final boolean fastMergeEnabled = sparkConf.getBoolean("spark.shuffle.unsafe.fastMergeEnabled", true); final boolean fastMergeIsSupported = !compressionEnabled || CompressionCodec$.MODULE$.supportsConcatenationOfSerializedStreams(compressionCodec); + final boolean encryptionEnabled = blockManager.serializerManager().encryptionEnabled(); try { if (spills.length == 0) { - new FileOutputStream(outputFile).close(); // Create an empty file + java.nio.file.Files.newOutputStream(outputFile.toPath()).close(); // Create an empty file return new long[partitioner.numPartitions()]; } else if (spills.length == 1) { // Here, we don't need to perform any metrics updates because the bytes written to this @@ -281,7 +314,7 @@ private long[] mergeSpills(SpillInfo[] spills) throws IOException { // Compression is disabled or we are using an IO compression codec that supports // decompression of concatenated compressed streams, so we can perform a fast spill merge // that doesn't need to interpret the spilled bytes. - if (transferToEnabled) { + if (transferToEnabled && !encryptionEnabled) { logger.debug("Using transferTo-based fast merge"); partitionLengths = mergeSpillsWithTransferTo(spills, outputFile); } else { @@ -297,8 +330,8 @@ private long[] mergeSpills(SpillInfo[] spills) throws IOException { // final write as bytes spilled (instead, it's accounted as shuffle write). The merge needs // to be counted as shuffle write, but this will lead to double-counting of the final // SpillInfo's bytes. - writeMetrics.decShuffleBytesWritten(spills[spills.length - 1].file.length()); - writeMetrics.incShuffleBytesWritten(outputFile.length()); + writeMetrics.decBytesWritten(spills[spills.length - 1].file.length()); + writeMetrics.incBytesWritten(outputFile.length()); return partitionLengths; } } catch (IOException e) { @@ -310,11 +343,15 @@ private long[] mergeSpills(SpillInfo[] spills) throws IOException { } /** - * Merges spill files using Java FileStreams. This code path is slower than the NIO-based merge, - * {@link UnsafeShuffleWriter#mergeSpillsWithTransferTo(SpillInfo[], File)}, so it's only used in - * cases where the IO compression codec does not support concatenation of compressed data, or in - * cases where users have explicitly disabled use of {@code transferTo} in order to work around - * kernel bugs. + * Merges spill files using Java FileStreams. This code path is typically slower than + * the NIO-based merge, {@link UnsafeShuffleWriter#mergeSpillsWithTransferTo(SpillInfo[], + * File)}, and it's mostly used in cases where the IO compression codec does not support + * concatenation of compressed data, when encryption is enabled, or when users have + * explicitly disabled use of {@code transferTo} in order to work around kernel bugs. + * This code path might also be faster in cases where individual partition size in a spill + * is small and UnsafeShuffleWriter#mergeSpillsWithTransferTo method performs many small + * disk ios which is inefficient. In those case, Using large buffers for input and output + * files helps reducing the number of disk ios, making the file merging faster. * * @param spills the spills to merge. * @param outputFile the file to write the merged data to. @@ -328,36 +365,53 @@ private long[] mergeSpillsWithFileStream( assert (spills.length >= 2); final int numPartitions = partitioner.numPartitions(); final long[] partitionLengths = new long[numPartitions]; - final InputStream[] spillInputStreams = new FileInputStream[spills.length]; - OutputStream mergedFileOutputStream = null; + final InputStream[] spillInputStreams = new InputStream[spills.length]; + + final OutputStream bos = new BufferedOutputStream( + java.nio.file.Files.newOutputStream(outputFile.toPath()), + outputBufferSizeInBytes); + // Use a counting output stream to avoid having to close the underlying file and ask + // the file system for its size after each partition is written. + final CountingOutputStream mergedFileOutputStream = new CountingOutputStream(bos); boolean threwException = true; try { for (int i = 0; i < spills.length; i++) { - spillInputStreams[i] = new FileInputStream(spills[i].file); + spillInputStreams[i] = new NioBufferedFileInputStream( + spills[i].file, + inputBufferSizeInBytes); } for (int partition = 0; partition < numPartitions; partition++) { - final long initialFileLength = outputFile.length(); - mergedFileOutputStream = - new TimeTrackingOutputStream(writeMetrics, new FileOutputStream(outputFile, true)); + final long initialFileLength = mergedFileOutputStream.getByteCount(); + // Shield the underlying output stream from close() and flush() calls, so that we can close + // the higher level streams to make sure all data is really flushed and internal state is + // cleaned. + OutputStream partitionOutput = new CloseAndFlushShieldOutputStream( + new TimeTrackingOutputStream(writeMetrics, mergedFileOutputStream)); + partitionOutput = blockManager.serializerManager().wrapForEncryption(partitionOutput); if (compressionCodec != null) { - mergedFileOutputStream = compressionCodec.compressedOutputStream(mergedFileOutputStream); + partitionOutput = compressionCodec.compressedOutputStream(partitionOutput); } - for (int i = 0; i < spills.length; i++) { final long partitionLengthInSpill = spills[i].partitionLengths[partition]; if (partitionLengthInSpill > 0) { - InputStream partitionInputStream = - new LimitedInputStream(spillInputStreams[i], partitionLengthInSpill); - if (compressionCodec != null) { - partitionInputStream = compressionCodec.compressedInputStream(partitionInputStream); + InputStream partitionInputStream = new LimitedInputStream(spillInputStreams[i], + partitionLengthInSpill, false); + try { + partitionInputStream = blockManager.serializerManager().wrapForEncryption( + partitionInputStream); + if (compressionCodec != null) { + partitionInputStream = compressionCodec.compressedInputStream(partitionInputStream); + } + ByteStreams.copy(partitionInputStream, partitionOutput); + } finally { + partitionInputStream.close(); } - ByteStreams.copy(partitionInputStream, mergedFileOutputStream); } } - mergedFileOutputStream.flush(); - mergedFileOutputStream.close(); - partitionLengths[partition] = (outputFile.length() - initialFileLength); + partitionOutput.flush(); + partitionOutput.close(); + partitionLengths[partition] = (mergedFileOutputStream.getByteCount() - initialFileLength); } threwException = false; } finally { @@ -389,28 +443,25 @@ private long[] mergeSpillsWithTransferTo(SpillInfo[] spills, File outputFile) th boolean threwException = true; try { for (int i = 0; i < spills.length; i++) { - spillInputChannels[i] = new FileInputStream(spills[i].file).getChannel(); + spillInputChannels[i] = FileChannel.open(spills[i].file.toPath(), READ); } // This file needs to opened in append mode in order to work around a Linux kernel bug that // affects transferTo; see SPARK-3948 for more details. - mergedFileOutputChannel = new FileOutputStream(outputFile, true).getChannel(); + mergedFileOutputChannel = FileChannel.open(outputFile.toPath(), WRITE, CREATE, APPEND); long bytesWrittenToMergedFile = 0; for (int partition = 0; partition < numPartitions; partition++) { for (int i = 0; i < spills.length; i++) { final long partitionLengthInSpill = spills[i].partitionLengths[partition]; - long bytesToTransfer = partitionLengthInSpill; final FileChannel spillInputChannel = spillInputChannels[i]; final long writeStartTime = System.nanoTime(); - while (bytesToTransfer > 0) { - final long actualBytesTransferred = spillInputChannel.transferTo( - spillInputChannelPositions[i], - bytesToTransfer, - mergedFileOutputChannel); - spillInputChannelPositions[i] += actualBytesTransferred; - bytesToTransfer -= actualBytesTransferred; - } - writeMetrics.incShuffleWriteTime(System.nanoTime() - writeStartTime); + Utils.copyFileStreamNIO( + spillInputChannel, + mergedFileOutputChannel, + spillInputChannelPositions[i], + partitionLengthInSpill); + spillInputChannelPositions[i] += partitionLengthInSpill; + writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); bytesWrittenToMergedFile += partitionLengthInSpill; partitionLengths[partition] += partitionLengthInSpill; } @@ -444,13 +495,7 @@ private long[] mergeSpillsWithTransferTo(SpillInfo[] spills, File outputFile) th @Override public Option stop(boolean success) { try { - // Update task metrics from accumulators (null in UnsafeShuffleWriterSuite) - Map> internalAccumulators = - taskContext.internalMetricsToAccumulators(); - if (internalAccumulators != null) { - internalAccumulators.apply(InternalAccumulator.PEAK_EXECUTION_MEMORY()) - .add(getPeakMemoryUsedBytes()); - } + taskContext.taskMetrics().incPeakExecutionMemory(getPeakMemoryUsedBytes()); if (stopping) { return Option.apply(null); @@ -462,8 +507,6 @@ public Option stop(boolean success) { } return Option.apply(mapStatus); } else { - // The map task failed, so delete our output data. - shuffleBlockResolver.removeDataByMap(shuffleId, mapId); return Option.apply(null); } } diff --git a/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java b/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java index f19ed01d5aeb..dff4f5df6878 100644 --- a/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java +++ b/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java @@ -19,7 +19,9 @@ import org.apache.spark.util.EnumUtil; +import java.util.Collections; import java.util.HashSet; +import java.util.Locale; import java.util.Set; public enum TaskSorting { @@ -28,15 +30,13 @@ public enum TaskSorting { DECREASING_RUNTIME("-runtime"); private final Set alternateNames; - private TaskSorting(String... names) { - alternateNames = new HashSet(); - for (String n: names) { - alternateNames.add(n); - } + TaskSorting(String... names) { + alternateNames = new HashSet<>(); + Collections.addAll(alternateNames, names); } public static TaskSorting fromString(String str) { - String lower = str.toLowerCase(); + String lower = str.toLowerCase(Locale.ROOT); for (TaskSorting t: values()) { if (t.alternateNames.contains(lower)) { return t; diff --git a/core/src/main/java/org/apache/spark/storage/TimeTrackingOutputStream.java b/core/src/main/java/org/apache/spark/storage/TimeTrackingOutputStream.java index dc2aa30466cc..5d0555a8c28e 100644 --- a/core/src/main/java/org/apache/spark/storage/TimeTrackingOutputStream.java +++ b/core/src/main/java/org/apache/spark/storage/TimeTrackingOutputStream.java @@ -42,34 +42,34 @@ public TimeTrackingOutputStream(ShuffleWriteMetrics writeMetrics, OutputStream o public void write(int b) throws IOException { final long startTime = System.nanoTime(); outputStream.write(b); - writeMetrics.incShuffleWriteTime(System.nanoTime() - startTime); + writeMetrics.incWriteTime(System.nanoTime() - startTime); } @Override public void write(byte[] b) throws IOException { final long startTime = System.nanoTime(); outputStream.write(b); - writeMetrics.incShuffleWriteTime(System.nanoTime() - startTime); + writeMetrics.incWriteTime(System.nanoTime() - startTime); } @Override public void write(byte[] b, int off, int len) throws IOException { final long startTime = System.nanoTime(); outputStream.write(b, off, len); - writeMetrics.incShuffleWriteTime(System.nanoTime() - startTime); + writeMetrics.incWriteTime(System.nanoTime() - startTime); } @Override public void flush() throws IOException { final long startTime = System.nanoTime(); outputStream.flush(); - writeMetrics.incShuffleWriteTime(System.nanoTime() - startTime); + writeMetrics.incWriteTime(System.nanoTime() - startTime); } @Override public void close() throws IOException { final long startTime = System.nanoTime(); outputStream.close(); - writeMetrics.incShuffleWriteTime(System.nanoTime() - startTime); + writeMetrics.incWriteTime(System.nanoTime() - startTime); } } diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 6656fd1d0bc5..4fadfe36cd71 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -20,11 +20,11 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.util.Arrays; import java.util.Iterator; import java.util.LinkedList; import com.google.common.annotations.VisibleForTesting; +import com.google.common.io.Closeables; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,13 +32,14 @@ import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.serializer.SerializerManager; import org.apache.spark.storage.BlockManager; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.UnsafeAlignedOffset; import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.array.LongArray; import org.apache.spark.unsafe.hash.Murmur3_x86_32; import org.apache.spark.unsafe.memory.MemoryBlock; -import org.apache.spark.unsafe.memory.MemoryLocation; import org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillReader; import org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillWriter; @@ -56,16 +57,15 @@ * Bytes 4 to 8: len(k) * Bytes 8 to 8 + len(k): key data * Bytes 8 + len(k) to 8 + len(k) + len(v): value data + * Bytes 8 + len(k) + len(v) to 8 + len(k) + len(v) + 8: pointer to next pair * * This means that the first four bytes store the entire record (key + value) length. This format - * is consistent with {@link org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter}, + * is compatible with {@link org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter}, * so we can pass records from this map directly into the sorter to sort records in place. */ public final class BytesToBytesMap extends MemoryConsumer { - private final Logger logger = LoggerFactory.getLogger(BytesToBytesMap.class); - - private static final Murmur3_x86_32 HASHER = new Murmur3_x86_32(0); + private static final Logger logger = LoggerFactory.getLogger(BytesToBytesMap.class); private static final HashMapGrowthStrategy growthStrategy = HashMapGrowthStrategy.DOUBLING; @@ -134,7 +134,12 @@ public final class BytesToBytesMap extends MemoryConsumer { /** * Number of keys defined in the map. */ - private int numElements; + private int numKeys; + + /** + * Number of values defined in the map. A key could have multiple values. + */ + private int numValues; /** * The map will be expanded once the number of keys exceeds this threshold. @@ -155,30 +160,31 @@ public final class BytesToBytesMap extends MemoryConsumer { private final boolean enablePerfMetrics; - private long timeSpentResizingNs = 0; - private long numProbes = 0; private long numKeyLookups = 0; - private long numHashCollisions = 0; - private long peakMemoryUsedBytes = 0L; + private final int initialCapacity; + private final BlockManager blockManager; + private final SerializerManager serializerManager; private volatile MapIterator destructiveIterator = null; private LinkedList spillWriters = new LinkedList<>(); public BytesToBytesMap( TaskMemoryManager taskMemoryManager, BlockManager blockManager, + SerializerManager serializerManager, int initialCapacity, double loadFactor, long pageSizeBytes, boolean enablePerfMetrics) { - super(taskMemoryManager, pageSizeBytes); + super(taskMemoryManager, pageSizeBytes, taskMemoryManager.getTungstenMemoryMode()); this.taskMemoryManager = taskMemoryManager; this.blockManager = blockManager; + this.serializerManager = serializerManager; this.loadFactor = loadFactor; this.loc = new Location(); this.pageSizeBytes = pageSizeBytes; @@ -194,6 +200,7 @@ public BytesToBytesMap( throw new IllegalArgumentException("Page size " + pageSizeBytes + " cannot exceed " + TaskMemoryManager.MAXIMUM_PAGE_SIZE_BYTES); } + this.initialCapacity = initialCapacity; allocate(initialCapacity); } @@ -212,8 +219,10 @@ public BytesToBytesMap( this( taskMemoryManager, SparkEnv.get() != null ? SparkEnv.get().blockManager() : null, + SparkEnv.get() != null ? SparkEnv.get().serializerManager() : null, initialCapacity, - 0.70, + // In order to re-use the longArray for sorting, the load factor cannot be larger than 0.5. + 0.5, pageSizeBytes, enablePerfMetrics); } @@ -221,7 +230,12 @@ public BytesToBytesMap( /** * Returns the number of keys defined in the map. */ - public int numElements() { return numElements; } + public int numKeys() { return numKeys; } + + /** + * Returns the number of values defined in the map. A key could have multiple values. + */ + public int numValues() { return numValues; } public final class MapIterator implements Iterator { @@ -244,6 +258,11 @@ private MapIterator(int numRecords, Location loc, boolean destructive) { this.destructive = destructive; if (destructive) { destructiveIterator = this; + // longArray will not be used anymore if destructive is true, release it now. + if (longArray != null) { + freeArray(longArray); + longArray = null; + } } } @@ -259,21 +278,16 @@ private void advanceToNextPage() { currentPage = dataPages.get(nextIdx); pageBaseObject = currentPage.getBaseObject(); offsetInPage = currentPage.getBaseOffset(); - recordsInPage = Platform.getInt(pageBaseObject, offsetInPage); - offsetInPage += 4; + recordsInPage = UnsafeAlignedOffset.getSize(pageBaseObject, offsetInPage); + offsetInPage += UnsafeAlignedOffset.getUaoSize(); } else { currentPage = null; if (reader != null) { - // remove the spill file from disk - File file = spillWriters.removeFirst().getFile(); - if (file != null && file.exists()) { - if (!file.delete()) { - logger.error("Was unable to delete spill file {}", file.getAbsolutePath()); - } - } + handleFailedDelete(); } try { - reader = spillWriters.getFirst().getReader(blockManager); + Closeables.close(reader, /* swallowIOException = */ false); + reader = spillWriters.getFirst().getReader(serializerManager); recordsInPage = -1; } catch (IOException e) { // Scala iterator does not handle exception @@ -287,13 +301,7 @@ private void advanceToNextPage() { public boolean hasNext() { if (numRecords == 0) { if (reader != null) { - // remove the spill file from disk - File file = spillWriters.removeFirst().getFile(); - if (file != null && file.exists()) { - if (!file.delete()) { - logger.error("Was unable to delete spill file {}", file.getAbsolutePath()); - } - } + handleFailedDelete(); } } return numRecords > 0; @@ -306,9 +314,10 @@ public Location next() { } numRecords--; if (currentPage != null) { - int totalLength = Platform.getInt(pageBaseObject, offsetInPage); + int totalLength = UnsafeAlignedOffset.getSize(pageBaseObject, offsetInPage); loc.with(currentPage, offsetInPage); - offsetInPage += 4 + totalLength; + // [total size] [key size] [key] [value] [pointer to next] + offsetInPage += UnsafeAlignedOffset.getUaoSize() + totalLength + 8; recordsInPage --; return loc; } else { @@ -319,6 +328,11 @@ public Location next() { try { reader.loadNext(); } catch (IOException e) { + try { + reader.close(); + } catch(IOException e2) { + logger.error("Error while closing spill reader", e2); + } // Scala iterator does not handle exception Platform.throwException(e); } @@ -346,14 +360,15 @@ public long spill(long numBytes) throws IOException { Object base = block.getBaseObject(); long offset = block.getBaseOffset(); - int numRecords = Platform.getInt(base, offset); - offset += 4; + int numRecords = UnsafeAlignedOffset.getSize(base, offset); + int uaoSize = UnsafeAlignedOffset.getUaoSize(); + offset += uaoSize; final UnsafeSorterSpillWriter writer = new UnsafeSorterSpillWriter(blockManager, 32 * 1024, writeMetrics, numRecords); while (numRecords > 0) { - int length = Platform.getInt(base, offset); - writer.write(base, offset + 4, length, 0); - offset += 4 + length; + int length = UnsafeAlignedOffset.getSize(base, offset); + writer.write(base, offset + uaoSize, length, 0); + offset += uaoSize + length + 8; numRecords--; } writer.close(); @@ -376,6 +391,14 @@ public long spill(long numBytes) throws IOException { public void remove() { throw new UnsupportedOperationException(); } + + private void handleFailedDelete() { + // remove the spill file from disk + File file = spillWriters.removeFirst().getFile(); + if (file != null && file.exists() && !file.delete()) { + logger.error("Was unable to delete spill file {}", file.getAbsolutePath()); + } + } } /** @@ -387,7 +410,7 @@ public void remove() { * `lookup()`, the behavior of the returned iterator is undefined. */ public MapIterator iterator() { - return new MapIterator(numElements, loc, false); + return new MapIterator(numValues, loc, false); } /** @@ -401,7 +424,7 @@ public MapIterator iterator() { * `lookup()`, the behavior of the returned iterator is undefined. */ public MapIterator destructiveIterator() { - return new MapIterator(numElements, loc, true); + return new MapIterator(numValues, loc, true); } /** @@ -411,7 +434,19 @@ public MapIterator destructiveIterator() { * This function always return the same {@link Location} instance to avoid object allocation. */ public Location lookup(Object keyBase, long keyOffset, int keyLength) { - safeLookup(keyBase, keyOffset, keyLength, loc); + safeLookup(keyBase, keyOffset, keyLength, loc, + Murmur3_x86_32.hashUnsafeWords(keyBase, keyOffset, keyLength, 42)); + return loc; + } + + /** + * Looks up a key, and return a {@link Location} handle that can be used to test existence + * and read/write values. + * + * This function always return the same {@link Location} instance to avoid object allocation. + */ + public Location lookup(Object keyBase, long keyOffset, int keyLength, int hash) { + safeLookup(keyBase, keyOffset, keyLength, loc, hash); return loc; } @@ -420,14 +455,13 @@ public Location lookup(Object keyBase, long keyOffset, int keyLength) { * * This is a thread-safe version of `lookup`, could be used by multiple threads. */ - public void safeLookup(Object keyBase, long keyOffset, int keyLength, Location loc) { + public void safeLookup(Object keyBase, long keyOffset, int keyLength, Location loc, int hash) { assert(longArray != null); if (enablePerfMetrics) { numKeyLookups++; } - final int hashcode = HASHER.hashUnsafeWords(keyBase, keyOffset, keyLength); - int pos = hashcode & mask; + int pos = hash & mask; int step = 1; while (true) { if (enablePerfMetrics) { @@ -435,30 +469,23 @@ public void safeLookup(Object keyBase, long keyOffset, int keyLength, Location l } if (longArray.get(pos * 2) == 0) { // This is a new key. - loc.with(pos, hashcode, false); + loc.with(pos, hash, false); return; } else { long stored = longArray.get(pos * 2 + 1); - if ((int) (stored) == hashcode) { + if ((int) (stored) == hash) { // Full hash code matches. Let's compare the keys for equality. - loc.with(pos, hashcode, true); + loc.with(pos, hash, true); if (loc.getKeyLength() == keyLength) { - final MemoryLocation keyAddress = loc.getKeyAddress(); - final Object storedkeyBase = keyAddress.getBaseObject(); - final long storedkeyOffset = keyAddress.getBaseOffset(); final boolean areEqual = ByteArrayMethods.arrayEquals( keyBase, keyOffset, - storedkeyBase, - storedkeyOffset, + loc.getKeyBase(), + loc.getKeyOffset(), keyLength ); if (areEqual) { return; - } else { - if (enablePerfMetrics) { - numHashCollisions++; - } } } } @@ -478,13 +505,14 @@ public final class Location { private boolean isDefined; /** * The hashcode of the most recent key passed to - * {@link BytesToBytesMap#lookup(Object, long, int)}. Caching this hashcode here allows us to - * avoid re-hashing the key when storing a value for that key. + * {@link BytesToBytesMap#lookup(Object, long, int, int)}. Caching this hashcode here allows us + * to avoid re-hashing the key when storing a value for that key. */ private int keyHashcode; - private final MemoryLocation keyMemoryLocation = new MemoryLocation(); - private final MemoryLocation valueMemoryLocation = new MemoryLocation(); + private Object baseObject; // the base object for key and value + private long keyOffset; private int keyLength; + private long valueOffset; private int valueLength; /** @@ -498,18 +526,16 @@ private void updateAddressesAndSizes(long fullKeyAddress) { taskMemoryManager.getOffsetInPage(fullKeyAddress)); } - private void updateAddressesAndSizes(final Object base, final long offset) { - long position = offset; - final int totalLength = Platform.getInt(base, position); - position += 4; - keyLength = Platform.getInt(base, position); - position += 4; - valueLength = totalLength - keyLength - 4; - - keyMemoryLocation.setObjAndOffset(base, position); - - position += keyLength; - valueMemoryLocation.setObjAndOffset(base, position); + private void updateAddressesAndSizes(final Object base, long offset) { + baseObject = base; + final int totalLength = UnsafeAlignedOffset.getSize(base, offset); + int uaoSize = UnsafeAlignedOffset.getUaoSize(); + offset += uaoSize; + keyLength = UnsafeAlignedOffset.getSize(base, offset); + offset += uaoSize; + keyOffset = offset; + valueOffset = offset + keyLength; + valueLength = totalLength - keyLength - uaoSize; } private Location with(int pos, int keyHashcode, boolean isDefined) { @@ -537,13 +563,29 @@ private Location with(MemoryBlock page, long offsetInPage) { private Location with(Object base, long offset, int length) { this.isDefined = true; this.memoryPage = null; - keyLength = Platform.getInt(base, offset); - valueLength = length - 4 - keyLength; - keyMemoryLocation.setObjAndOffset(base, offset + 4); - valueMemoryLocation.setObjAndOffset(base, offset + 4 + keyLength); + baseObject = base; + int uaoSize = UnsafeAlignedOffset.getUaoSize(); + keyOffset = offset + uaoSize; + keyLength = UnsafeAlignedOffset.getSize(base, offset); + valueOffset = offset + uaoSize + keyLength; + valueLength = length - uaoSize - keyLength; return this; } + /** + * Find the next pair that has the same key as current one. + */ + public boolean nextValue() { + assert isDefined; + long nextAddr = Platform.getLong(baseObject, valueOffset + valueLength); + if (nextAddr == 0) { + return false; + } else { + updateAddressesAndSizes(nextAddr); + return true; + } + } + /** * Returns the memory page that contains the current record. * This is only valid if this is returned by {@link BytesToBytesMap#iterator()}. @@ -560,34 +602,44 @@ public boolean isDefined() { } /** - * Returns the address of the key defined at this position. - * This points to the first byte of the key data. - * Unspecified behavior if the key is not defined. - * For efficiency reasons, calls to this method always returns the same MemoryLocation object. + * Returns the base object for key. */ - public MemoryLocation getKeyAddress() { + public Object getKeyBase() { assert (isDefined); - return keyMemoryLocation; + return baseObject; } /** - * Returns the length of the key defined at this position. - * Unspecified behavior if the key is not defined. + * Returns the offset for key. */ - public int getKeyLength() { + public long getKeyOffset() { assert (isDefined); - return keyLength; + return keyOffset; + } + + /** + * Returns the base object for value. + */ + public Object getValueBase() { + assert (isDefined); + return baseObject; + } + + /** + * Returns the offset for value. + */ + public long getValueOffset() { + assert (isDefined); + return valueOffset; } /** - * Returns the address of the value defined at this position. - * This points to the first byte of the value data. + * Returns the length of the key defined at this position. * Unspecified behavior if the key is not defined. - * For efficiency reasons, calls to this method always returns the same MemoryLocation object. */ - public MemoryLocation getValueAddress() { + public int getKeyLength() { assert (isDefined); - return valueMemoryLocation; + return keyLength; } /** @@ -600,10 +652,9 @@ public int getValueLength() { } /** - * Store a new key and value. This method may only be called once for a given key; if you want - * to update the value associated with a key, then you can directly manipulate the bytes stored - * at the value address. The return value indicates whether the put succeeded or whether it - * failed because additional memory could not be acquired. + * Append a new value for the key. This method could be called multiple times for a given key. + * The return value indicates whether the put succeeded or whether it failed because additional + * memory could not be acquired. *

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

@@ -612,7 +663,7 @@ public int getValueLength() { *

*

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

*

* As an example usage, here's the proper way to store a new key: @@ -620,7 +671,7 @@ public int getValueLength() { *

      *   Location loc = map.lookup(keyBase, keyOffset, keyLength);
      *   if (!loc.isDefined()) {
-     *     if (!loc.putNewKey(keyBase, keyOffset, keyLength, ...)) {
+     *     if (!loc.append(keyBase, keyOffset, keyLength, ...)) {
      *       // handle failure to grow map (by spilling, for example)
      *     }
      *   }
@@ -632,28 +683,26 @@ public int getValueLength() {
      * @return true if the put() was successful and false if the put() failed because memory could
      *         not be acquired.
      */
-    public boolean putNewKey(Object keyBase, long keyOffset, int keyLength,
-        Object valueBase, long valueOffset, int valueLength) {
-      assert (!isDefined) : "Can only set value once for a key";
-      assert (keyLength % 8 == 0);
-      assert (valueLength % 8 == 0);
-      assert(longArray != null);
-
+    public boolean append(Object kbase, long koff, int klen, Object vbase, long voff, int vlen) {
+      assert (klen % 8 == 0);
+      assert (vlen % 8 == 0);
+      assert (longArray != null);
 
-      if (numElements == MAX_CAPACITY
+      if (numKeys == MAX_CAPACITY
         // The map could be reused from last spill (because of no enough memory to grow),
         // then we don't try to grow again if hit the `growthThreshold`.
-        || !canGrowArray && numElements > growthThreshold) {
+        || !canGrowArray && numKeys >= growthThreshold) {
         return false;
       }
 
       // Here, we'll copy the data into our data pages. Because we only store a relative offset from
       // the key address instead of storing the absolute address of the value, the key and value
       // must be stored in the same memory page.
-      // (8 byte key length) (key) (value)
-      final long recordLength = 8 + keyLength + valueLength;
+      // (8 byte key length) (key) (value) (8 byte pointer to next value)
+      int uaoSize = UnsafeAlignedOffset.getUaoSize();
+      final long recordLength = (2 * uaoSize) + klen + vlen + 8;
       if (currentPage == null || currentPage.size() - pageCursor < recordLength) {
-        if (!acquireNewPage(recordLength + 4L)) {
+        if (!acquireNewPage(recordLength + uaoSize)) {
           return false;
         }
       }
@@ -662,30 +711,36 @@ public boolean putNewKey(Object keyBase, long keyOffset, int keyLength,
       final Object base = currentPage.getBaseObject();
       long offset = currentPage.getBaseOffset() + pageCursor;
       final long recordOffset = offset;
-      Platform.putInt(base, offset, keyLength + valueLength + 4);
-      Platform.putInt(base, offset + 4, keyLength);
-      offset += 8;
-      Platform.copyMemory(keyBase, keyOffset, base, offset, keyLength);
-      offset += keyLength;
-      Platform.copyMemory(valueBase, valueOffset, base, offset, valueLength);
-
-      // --- Update bookkeeping data structures -----------------------------------------------------
+      UnsafeAlignedOffset.putSize(base, offset, klen + vlen + uaoSize);
+      UnsafeAlignedOffset.putSize(base, offset + uaoSize, klen);
+      offset += (2 * uaoSize);
+      Platform.copyMemory(kbase, koff, base, offset, klen);
+      offset += klen;
+      Platform.copyMemory(vbase, voff, base, offset, vlen);
+      offset += vlen;
+      // put this value at the beginning of the list
+      Platform.putLong(base, offset, isDefined ? longArray.get(pos * 2) : 0);
+
+      // --- Update bookkeeping data structures ----------------------------------------------------
       offset = currentPage.getBaseOffset();
-      Platform.putInt(base, offset, Platform.getInt(base, offset) + 1);
+      UnsafeAlignedOffset.putSize(base, offset, UnsafeAlignedOffset.getSize(base, offset) + 1);
       pageCursor += recordLength;
-      numElements++;
       final long storedKeyAddress = taskMemoryManager.encodePageNumberAndOffset(
         currentPage, recordOffset);
       longArray.set(pos * 2, storedKeyAddress);
-      longArray.set(pos * 2 + 1, keyHashcode);
       updateAddressesAndSizes(storedKeyAddress);
-      isDefined = true;
+      numValues++;
+      if (!isDefined) {
+        numKeys++;
+        longArray.set(pos * 2 + 1, keyHashcode);
+        isDefined = true;
 
-      if (numElements > growthThreshold && longArray.size() < MAX_CAPACITY) {
-        try {
-          growAndRehash();
-        } catch (OutOfMemoryError oom) {
-          canGrowArray = false;
+        if (numKeys >= growthThreshold && longArray.size() < MAX_CAPACITY) {
+          try {
+            growAndRehash();
+          } catch (OutOfMemoryError oom) {
+            canGrowArray = false;
+          }
         }
       }
       return true;
@@ -703,8 +758,8 @@ private boolean acquireNewPage(long required) {
       return false;
     }
     dataPages.add(currentPage);
-    Platform.putInt(currentPage.getBaseObject(), currentPage.getBaseOffset(), 0);
-    pageCursor = 4;
+    UnsafeAlignedOffset.putSize(currentPage.getBaseObject(), currentPage.getBaseOffset(), 0);
+    pageCursor = UnsafeAlignedOffset.getUaoSize();
     return true;
   }
 
@@ -724,11 +779,10 @@ public long spill(long size, MemoryConsumer trigger) throws IOException {
    */
   private void allocate(int capacity) {
     assert (capacity >= 0);
-    // The capacity needs to be divisible by 64 so that our bit set can be sized properly
     capacity = Math.max((int) Math.min(MAX_CAPACITY, ByteArrayMethods.nextPowerOf2(capacity)), 64);
     assert (capacity <= MAX_CAPACITY);
-    acquireMemory(capacity * 16);
-    longArray = new LongArray(MemoryBlock.fromLongArray(new long[capacity * 2]));
+    longArray = allocateArray(capacity * 2);
+    longArray.zeroOut();
 
     this.growthThreshold = (int) (capacity * loadFactor);
     this.mask = capacity - 1;
@@ -743,9 +797,8 @@ private void allocate(int capacity) {
   public void free() {
     updatePeakMemoryUsed();
     if (longArray != null) {
-      long used = longArray.memoryBlock().size();
+      freeArray(longArray);
       longArray = null;
-      releaseMemory(used);
     }
     Iterator dataPagesIterator = dataPages.iterator();
     while (dataPagesIterator.hasNext()) {
@@ -799,16 +852,6 @@ public long getPeakMemoryUsedBytes() {
     return peakMemoryUsedBytes;
   }
 
-  /**
-   * Returns the total amount of time spent resizing this map (in nanoseconds).
-   */
-  public long getTimeSpentResizingNs() {
-    if (!enablePerfMetrics) {
-      throw new IllegalStateException();
-    }
-    return timeSpentResizingNs;
-  }
-
   /**
    * Returns the average number of probes per key lookup.
    */
@@ -819,13 +862,6 @@ public double getAverageProbesPerLookup() {
     return (1.0 * numProbes) / numKeyLookups;
   }
 
-  public long getNumHashCollisions() {
-    if (!enablePerfMetrics) {
-      throw new IllegalStateException();
-    }
-    return numHashCollisions;
-  }
-
   @VisibleForTesting
   public int getNumDataPages() {
     return dataPages.size();
@@ -834,21 +870,24 @@ public int getNumDataPages() {
   /**
    * Returns the underline long[] of longArray.
    */
-  public long[] getArray() {
+  public LongArray getArray() {
     assert(longArray != null);
-    return (long[]) longArray.memoryBlock().getBaseObject();
+    return longArray;
   }
 
   /**
    * Reset this map to initialized state.
    */
   public void reset() {
-    numElements = 0;
-    Arrays.fill(getArray(), 0);
+    numKeys = 0;
+    numValues = 0;
+    freeArray(longArray);
     while (dataPages.size() > 0) {
       MemoryBlock dataPage = dataPages.removeLast();
       freePage(dataPage);
     }
+    allocate(initialCapacity);
+    canGrowArray = true;
     currentPage = null;
     pageCursor = 0;
   }
@@ -860,10 +899,6 @@ public void reset() {
   void growAndRehash() {
     assert(longArray != null);
 
-    long resizeStartTime = -1;
-    if (enablePerfMetrics) {
-      resizeStartTime = System.nanoTime();
-    }
     // Store references to the old data structures to be used when we re-hash
     final LongArray oldLongArray = longArray;
     final int oldCapacity = (int) oldLongArray.size() / 2;
@@ -887,10 +922,6 @@ void growAndRehash() {
       longArray.set(newPos * 2, keyPointer);
       longArray.set(newPos * 2 + 1, hashcode);
     }
-    releaseMemory(oldLongArray.memoryBlock().size());
-
-    if (enablePerfMetrics) {
-      timeSpentResizingNs += System.nanoTime() - resizeStartTime;
-    }
+    freeArray(oldLongArray);
   }
 }
diff --git a/core/src/main/java/org/apache/spark/unsafe/map/HashMapGrowthStrategy.java b/core/src/main/java/org/apache/spark/unsafe/map/HashMapGrowthStrategy.java
index 20654e4eeaa0..b8c2294c7b7a 100644
--- a/core/src/main/java/org/apache/spark/unsafe/map/HashMapGrowthStrategy.java
+++ b/core/src/main/java/org/apache/spark/unsafe/map/HashMapGrowthStrategy.java
@@ -30,11 +30,17 @@ public interface HashMapGrowthStrategy {
   HashMapGrowthStrategy DOUBLING = new Doubling();
 
   class Doubling implements HashMapGrowthStrategy {
+
+    // Some JVMs can't allocate arrays of length Integer.MAX_VALUE; actual max is somewhat
+    // smaller. Be conservative and lower the cap a little.
+    private static final int ARRAY_MAX = Integer.MAX_VALUE - 8;
+
     @Override
     public int nextCapacity(int currentCapacity) {
       assert (currentCapacity > 0);
+      int doubleCapacity = currentCapacity * 2;
       // Guard against overflow
-      return (currentCapacity * 2 > 0) ? (currentCapacity * 2) : Integer.MAX_VALUE;
+      return (doubleCapacity > 0 && doubleCapacity <= ARRAY_MAX) ? doubleCapacity : ARRAY_MAX;
     }
   }
 
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java
index d2bf297c6c17..0910db22af00 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java
@@ -20,97 +20,153 @@
 import com.google.common.primitives.UnsignedLongs;
 
 import org.apache.spark.annotation.Private;
-import org.apache.spark.unsafe.Platform;
 import org.apache.spark.unsafe.types.ByteArray;
 import org.apache.spark.unsafe.types.UTF8String;
-import org.apache.spark.util.Utils;
 
 @Private
 public class PrefixComparators {
   private PrefixComparators() {}
 
-  public static final StringPrefixComparator STRING = new StringPrefixComparator();
-  public static final StringPrefixComparatorDesc STRING_DESC = new StringPrefixComparatorDesc();
-  public static final BinaryPrefixComparator BINARY = new BinaryPrefixComparator();
-  public static final BinaryPrefixComparatorDesc BINARY_DESC = new BinaryPrefixComparatorDesc();
-  public static final LongPrefixComparator LONG = new LongPrefixComparator();
-  public static final LongPrefixComparatorDesc LONG_DESC = new LongPrefixComparatorDesc();
-  public static final DoublePrefixComparator DOUBLE = new DoublePrefixComparator();
-  public static final DoublePrefixComparatorDesc DOUBLE_DESC = new DoublePrefixComparatorDesc();
-
-  public static final class StringPrefixComparator extends PrefixComparator {
-    @Override
-    public int compare(long aPrefix, long bPrefix) {
-      return UnsignedLongs.compare(aPrefix, bPrefix);
-    }
-
+  public static final PrefixComparator STRING = new UnsignedPrefixComparator();
+  public static final PrefixComparator STRING_DESC = new UnsignedPrefixComparatorDesc();
+  public static final PrefixComparator STRING_NULLS_LAST = new UnsignedPrefixComparatorNullsLast();
+  public static final PrefixComparator STRING_DESC_NULLS_FIRST =
+    new UnsignedPrefixComparatorDescNullsFirst();
+
+  public static final PrefixComparator BINARY = new UnsignedPrefixComparator();
+  public static final PrefixComparator BINARY_DESC = new UnsignedPrefixComparatorDesc();
+  public static final PrefixComparator BINARY_NULLS_LAST = new UnsignedPrefixComparatorNullsLast();
+  public static final PrefixComparator BINARY_DESC_NULLS_FIRST =
+    new UnsignedPrefixComparatorDescNullsFirst();
+
+  public static final PrefixComparator LONG = new SignedPrefixComparator();
+  public static final PrefixComparator LONG_DESC = new SignedPrefixComparatorDesc();
+  public static final PrefixComparator LONG_NULLS_LAST = new SignedPrefixComparatorNullsLast();
+  public static final PrefixComparator LONG_DESC_NULLS_FIRST =
+    new SignedPrefixComparatorDescNullsFirst();
+
+  public static final PrefixComparator DOUBLE = new UnsignedPrefixComparator();
+  public static final PrefixComparator DOUBLE_DESC = new UnsignedPrefixComparatorDesc();
+  public static final PrefixComparator DOUBLE_NULLS_LAST = new UnsignedPrefixComparatorNullsLast();
+  public static final PrefixComparator DOUBLE_DESC_NULLS_FIRST =
+    new UnsignedPrefixComparatorDescNullsFirst();
+
+  public static final class StringPrefixComparator {
     public static long computePrefix(UTF8String value) {
       return value == null ? 0L : value.getPrefix();
     }
   }
 
-  public static final class StringPrefixComparatorDesc extends PrefixComparator {
-    @Override
-    public int compare(long bPrefix, long aPrefix) {
+  public static final class BinaryPrefixComparator {
+    public static long computePrefix(byte[] bytes) {
+      return ByteArray.getPrefix(bytes);
+    }
+  }
+
+  public static final class DoublePrefixComparator {
+    /**
+     * Converts the double into a value that compares correctly as an unsigned long. For more
+     * details see http://stereopsis.com/radix.html.
+     */
+    public static long computePrefix(double value) {
+      // Java's doubleToLongBits already canonicalizes all NaN values to the smallest possible
+      // positive NaN, so there's nothing special we need to do for NaNs.
+      long bits = Double.doubleToLongBits(value);
+      // Negative floats compare backwards due to their sign-magnitude representation, so flip
+      // all the bits in this case.
+      long mask = -(bits >>> 63) | 0x8000000000000000L;
+      return bits ^ mask;
+    }
+  }
+
+  /**
+   * Provides radix sort parameters. Comparators implementing this also are indicating that the
+   * ordering they define is compatible with radix sort.
+   */
+  public abstract static class RadixSortSupport extends PrefixComparator {
+    /** @return Whether the sort should be descending in binary sort order. */
+    public abstract boolean sortDescending();
+
+    /** @return Whether the sort should take into account the sign bit. */
+    public abstract boolean sortSigned();
+
+    /** @return Whether the sort should put nulls first or last. */
+    public abstract boolean nullsFirst();
+  }
+
+  //
+  // Standard prefix comparator implementations
+  //
+
+  public static final class UnsignedPrefixComparator extends RadixSortSupport {
+    @Override public boolean sortDescending() { return false; }
+    @Override public boolean sortSigned() { return false; }
+    @Override public boolean nullsFirst() { return true; }
+    public int compare(long aPrefix, long bPrefix) {
       return UnsignedLongs.compare(aPrefix, bPrefix);
     }
   }
 
-  public static final class BinaryPrefixComparator extends PrefixComparator {
-    @Override
+  public static final class UnsignedPrefixComparatorNullsLast extends RadixSortSupport {
+    @Override public boolean sortDescending() { return false; }
+    @Override public boolean sortSigned() { return false; }
+    @Override public boolean nullsFirst() { return false; }
     public int compare(long aPrefix, long bPrefix) {
       return UnsignedLongs.compare(aPrefix, bPrefix);
     }
+  }
 
-    public static long computePrefix(byte[] bytes) {
-      return ByteArray.getPrefix(bytes);
+  public static final class UnsignedPrefixComparatorDescNullsFirst extends RadixSortSupport {
+    @Override public boolean sortDescending() { return true; }
+    @Override public boolean sortSigned() { return false; }
+    @Override public boolean nullsFirst() { return true; }
+    public int compare(long bPrefix, long aPrefix) {
+      return UnsignedLongs.compare(aPrefix, bPrefix);
     }
   }
 
-  public static final class BinaryPrefixComparatorDesc extends PrefixComparator {
-    @Override
+  public static final class UnsignedPrefixComparatorDesc extends RadixSortSupport {
+    @Override public boolean sortDescending() { return true; }
+    @Override public boolean sortSigned() { return false; }
+    @Override public boolean nullsFirst() { return false; }
     public int compare(long bPrefix, long aPrefix) {
       return UnsignedLongs.compare(aPrefix, bPrefix);
     }
   }
 
-  public static final class LongPrefixComparator extends PrefixComparator {
-    @Override
+  public static final class SignedPrefixComparator extends RadixSortSupport {
+    @Override public boolean sortDescending() { return false; }
+    @Override public boolean sortSigned() { return true; }
+    @Override public boolean nullsFirst() { return true; }
     public int compare(long a, long b) {
       return (a < b) ? -1 : (a > b) ? 1 : 0;
     }
   }
 
-  public static final class LongPrefixComparatorDesc extends PrefixComparator {
-    @Override
-    public int compare(long b, long a) {
+  public static final class SignedPrefixComparatorNullsLast extends RadixSortSupport {
+    @Override public boolean sortDescending() { return false; }
+    @Override public boolean sortSigned() { return true; }
+    @Override public boolean nullsFirst() { return false; }
+    public int compare(long a, long b) {
       return (a < b) ? -1 : (a > b) ? 1 : 0;
     }
   }
 
-  public static final class DoublePrefixComparator extends PrefixComparator {
-    @Override
-    public int compare(long aPrefix, long bPrefix) {
-      double a = Double.longBitsToDouble(aPrefix);
-      double b = Double.longBitsToDouble(bPrefix);
-      return Utils.nanSafeCompareDoubles(a, b);
-    }
-
-    public static long computePrefix(double value) {
-      return Double.doubleToLongBits(value);
+  public static final class SignedPrefixComparatorDescNullsFirst extends RadixSortSupport {
+    @Override public boolean sortDescending() { return true; }
+    @Override public boolean sortSigned() { return true; }
+    @Override public boolean nullsFirst() { return true; }
+    public int compare(long b, long a) {
+      return (a < b) ? -1 : (a > b) ? 1 : 0;
     }
   }
 
-  public static final class DoublePrefixComparatorDesc extends PrefixComparator {
-    @Override
-    public int compare(long bPrefix, long aPrefix) {
-      double a = Double.longBitsToDouble(aPrefix);
-      double b = Double.longBitsToDouble(bPrefix);
-      return Utils.nanSafeCompareDoubles(a, b);
-    }
-
-    public static long computePrefix(double value) {
-      return Double.doubleToLongBits(value);
+  public static final class SignedPrefixComparatorDesc extends RadixSortSupport {
+    @Override public boolean sortDescending() { return true; }
+    @Override public boolean sortSigned() { return true; }
+    @Override public boolean nullsFirst() { return false; }
+    public int compare(long b, long a) {
+      return (a < b) ? -1 : (a > b) ? 1 : 0;
     }
   }
 }
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RadixSort.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RadixSort.java
new file mode 100644
index 000000000000..3dd318471008
--- /dev/null
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RadixSort.java
@@ -0,0 +1,261 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util.collection.unsafe.sort;
+
+import com.google.common.primitives.Ints;
+
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.array.LongArray;
+
+public class RadixSort {
+
+  /**
+   * Sorts a given array of longs using least-significant-digit radix sort. This routine assumes
+   * you have extra space at the end of the array at least equal to the number of records. The
+   * sort is destructive and may relocate the data positioned within the array.
+   *
+   * @param array array of long elements followed by at least that many empty slots.
+   * @param numRecords number of data records in the array.
+   * @param startByteIndex the first byte (in range [0, 7]) to sort each long by, counting from the
+   *                       least significant byte.
+   * @param endByteIndex the last byte (in range [0, 7]) to sort each long by, counting from the
+   *                     least significant byte. Must be greater than startByteIndex.
+   * @param desc whether this is a descending (binary-order) sort.
+   * @param signed whether this is a signed (two's complement) sort.
+   *
+   * @return The starting index of the sorted data within the given array. We return this instead
+   *         of always copying the data back to position zero for efficiency.
+   */
+  public static int sort(
+      LongArray array, long numRecords, int startByteIndex, int endByteIndex,
+      boolean desc, boolean signed) {
+    assert startByteIndex >= 0 : "startByteIndex (" + startByteIndex + ") should >= 0";
+    assert endByteIndex <= 7 : "endByteIndex (" + endByteIndex + ") should <= 7";
+    assert endByteIndex > startByteIndex;
+    assert numRecords * 2 <= array.size();
+    long inIndex = 0;
+    long outIndex = numRecords;
+    if (numRecords > 0) {
+      long[][] counts = getCounts(array, numRecords, startByteIndex, endByteIndex);
+      for (int i = startByteIndex; i <= endByteIndex; i++) {
+        if (counts[i] != null) {
+          sortAtByte(
+            array, numRecords, counts[i], i, inIndex, outIndex,
+            desc, signed && i == endByteIndex);
+          long tmp = inIndex;
+          inIndex = outIndex;
+          outIndex = tmp;
+        }
+      }
+    }
+    return Ints.checkedCast(inIndex);
+  }
+
+  /**
+   * Performs a partial sort by copying data into destination offsets for each byte value at the
+   * specified byte offset.
+   *
+   * @param array array to partially sort.
+   * @param numRecords number of data records in the array.
+   * @param counts counts for each byte value. This routine destructively modifies this array.
+   * @param byteIdx the byte in a long to sort at, counting from the least significant byte.
+   * @param inIndex the starting index in the array where input data is located.
+   * @param outIndex the starting index where sorted output data should be written.
+   * @param desc whether this is a descending (binary-order) sort.
+   * @param signed whether this is a signed (two's complement) sort (only applies to last byte).
+   */
+  private static void sortAtByte(
+      LongArray array, long numRecords, long[] counts, int byteIdx, long inIndex, long outIndex,
+      boolean desc, boolean signed) {
+    assert counts.length == 256;
+    long[] offsets = transformCountsToOffsets(
+      counts, numRecords, array.getBaseOffset() + outIndex * 8L, 8, desc, signed);
+    Object baseObject = array.getBaseObject();
+    long baseOffset = array.getBaseOffset() + inIndex * 8L;
+    long maxOffset = baseOffset + numRecords * 8L;
+    for (long offset = baseOffset; offset < maxOffset; offset += 8) {
+      long value = Platform.getLong(baseObject, offset);
+      int bucket = (int)((value >>> (byteIdx * 8)) & 0xff);
+      Platform.putLong(baseObject, offsets[bucket], value);
+      offsets[bucket] += 8;
+    }
+  }
+
+  /**
+   * Computes a value histogram for each byte in the given array.
+   *
+   * @param array array to count records in.
+   * @param numRecords number of data records in the array.
+   * @param startByteIndex the first byte to compute counts for (the prior are skipped).
+   * @param endByteIndex the last byte to compute counts for.
+   *
+   * @return an array of eight 256-byte count arrays, one for each byte starting from the least
+   *         significant byte. If the byte does not need sorting the array will be null.
+   */
+  private static long[][] getCounts(
+      LongArray array, long numRecords, int startByteIndex, int endByteIndex) {
+    long[][] counts = new long[8][];
+    // Optimization: do a fast pre-pass to determine which byte indices we can skip for sorting.
+    // If all the byte values at a particular index are the same we don't need to count it.
+    long bitwiseMax = 0;
+    long bitwiseMin = -1L;
+    long maxOffset = array.getBaseOffset() + numRecords * 8L;
+    Object baseObject = array.getBaseObject();
+    for (long offset = array.getBaseOffset(); offset < maxOffset; offset += 8) {
+      long value = Platform.getLong(baseObject, offset);
+      bitwiseMax |= value;
+      bitwiseMin &= value;
+    }
+    long bitsChanged = bitwiseMin ^ bitwiseMax;
+    // Compute counts for each byte index.
+    for (int i = startByteIndex; i <= endByteIndex; i++) {
+      if (((bitsChanged >>> (i * 8)) & 0xff) != 0) {
+        counts[i] = new long[256];
+        // TODO(ekl) consider computing all the counts in one pass.
+        for (long offset = array.getBaseOffset(); offset < maxOffset; offset += 8) {
+          counts[i][(int)((Platform.getLong(baseObject, offset) >>> (i * 8)) & 0xff)]++;
+        }
+      }
+    }
+    return counts;
+  }
+
+  /**
+   * Transforms counts into the proper unsafe output offsets for the sort type.
+   *
+   * @param counts counts for each byte value. This routine destructively modifies this array.
+   * @param numRecords number of data records in the original data array.
+   * @param outputOffset output offset in bytes from the base array object.
+   * @param bytesPerRecord size of each record (8 for plain sort, 16 for key-prefix sort).
+   * @param desc whether this is a descending (binary-order) sort.
+   * @param signed whether this is a signed (two's complement) sort.
+   *
+   * @return the input counts array.
+   */
+  private static long[] transformCountsToOffsets(
+      long[] counts, long numRecords, long outputOffset, long bytesPerRecord,
+      boolean desc, boolean signed) {
+    assert counts.length == 256;
+    int start = signed ? 128 : 0;  // output the negative records first (values 129-255).
+    if (desc) {
+      long pos = numRecords;
+      for (int i = start; i < start + 256; i++) {
+        pos -= counts[i & 0xff];
+        counts[i & 0xff] = outputOffset + pos * bytesPerRecord;
+      }
+    } else {
+      long pos = 0;
+      for (int i = start; i < start + 256; i++) {
+        long tmp = counts[i & 0xff];
+        counts[i & 0xff] = outputOffset + pos * bytesPerRecord;
+        pos += tmp;
+      }
+    }
+    return counts;
+  }
+
+  /**
+   * Specialization of sort() for key-prefix arrays. In this type of array, each record consists
+   * of two longs, only the second of which is sorted on.
+   *
+   * @param startIndex starting index in the array to sort from. This parameter is not supported
+   *    in the plain sort() implementation.
+   */
+  public static int sortKeyPrefixArray(
+      LongArray array,
+      long startIndex,
+      long numRecords,
+      int startByteIndex,
+      int endByteIndex,
+      boolean desc,
+      boolean signed) {
+    assert startByteIndex >= 0 : "startByteIndex (" + startByteIndex + ") should >= 0";
+    assert endByteIndex <= 7 : "endByteIndex (" + endByteIndex + ") should <= 7";
+    assert endByteIndex > startByteIndex;
+    assert numRecords * 4 <= array.size();
+    long inIndex = startIndex;
+    long outIndex = startIndex + numRecords * 2L;
+    if (numRecords > 0) {
+      long[][] counts = getKeyPrefixArrayCounts(
+        array, startIndex, numRecords, startByteIndex, endByteIndex);
+      for (int i = startByteIndex; i <= endByteIndex; i++) {
+        if (counts[i] != null) {
+          sortKeyPrefixArrayAtByte(
+            array, numRecords, counts[i], i, inIndex, outIndex,
+            desc, signed && i == endByteIndex);
+          long tmp = inIndex;
+          inIndex = outIndex;
+          outIndex = tmp;
+        }
+      }
+    }
+    return Ints.checkedCast(inIndex);
+  }
+
+  /**
+   * Specialization of getCounts() for key-prefix arrays. We could probably combine this with
+   * getCounts with some added parameters but that seems to hurt in benchmarks.
+   */
+  private static long[][] getKeyPrefixArrayCounts(
+      LongArray array, long startIndex, long numRecords, int startByteIndex, int endByteIndex) {
+    long[][] counts = new long[8][];
+    long bitwiseMax = 0;
+    long bitwiseMin = -1L;
+    long baseOffset = array.getBaseOffset() + startIndex * 8L;
+    long limit = baseOffset + numRecords * 16L;
+    Object baseObject = array.getBaseObject();
+    for (long offset = baseOffset; offset < limit; offset += 16) {
+      long value = Platform.getLong(baseObject, offset + 8);
+      bitwiseMax |= value;
+      bitwiseMin &= value;
+    }
+    long bitsChanged = bitwiseMin ^ bitwiseMax;
+    for (int i = startByteIndex; i <= endByteIndex; i++) {
+      if (((bitsChanged >>> (i * 8)) & 0xff) != 0) {
+        counts[i] = new long[256];
+        for (long offset = baseOffset; offset < limit; offset += 16) {
+          counts[i][(int)((Platform.getLong(baseObject, offset + 8) >>> (i * 8)) & 0xff)]++;
+        }
+      }
+    }
+    return counts;
+  }
+
+  /**
+   * Specialization of sortAtByte() for key-prefix arrays.
+   */
+  private static void sortKeyPrefixArrayAtByte(
+      LongArray array, long numRecords, long[] counts, int byteIdx, long inIndex, long outIndex,
+      boolean desc, boolean signed) {
+    assert counts.length == 256;
+    long[] offsets = transformCountsToOffsets(
+      counts, numRecords, array.getBaseOffset() + outIndex * 8L, 16, desc, signed);
+    Object baseObject = array.getBaseObject();
+    long baseOffset = array.getBaseOffset() + inIndex * 8L;
+    long maxOffset = baseOffset + numRecords * 16L;
+    for (long offset = baseOffset; offset < maxOffset; offset += 16) {
+      long key = Platform.getLong(baseObject, offset);
+      long prefix = Platform.getLong(baseObject, offset + 8);
+      int bucket = (int)((prefix >>> (byteIdx * 8)) & 0xff);
+      long dest = offsets[bucket];
+      Platform.putLong(baseObject, dest, key);
+      Platform.putLong(baseObject, dest + 8, prefix);
+      offsets[bucket] += 16;
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java
index dbf6770e0739..e9571aa8bb05 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java
@@ -17,11 +17,9 @@
 
 package org.apache.spark.util.collection.unsafe.sort;
 
-import org.apache.spark.memory.TaskMemoryManager;
-
-final class RecordPointerAndKeyPrefix {
+public final class RecordPointerAndKeyPrefix {
   /**
-   * A pointer to a record; see {@link TaskMemoryManager} for a
+   * A pointer to a record; see {@link org.apache.spark.memory.TaskMemoryManager} for a
    * description of how these addresses are encoded.
    */
   public long recordPointer;
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
index cba043bc48cc..39eda00dd7ef 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
@@ -21,6 +21,8 @@
 import java.io.File;
 import java.io.IOException;
 import java.util.LinkedList;
+import java.util.Queue;
+import java.util.function.Supplier;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
@@ -30,10 +32,12 @@
 import org.apache.spark.executor.ShuffleWriteMetrics;
 import org.apache.spark.memory.MemoryConsumer;
 import org.apache.spark.memory.TaskMemoryManager;
+import org.apache.spark.serializer.SerializerManager;
 import org.apache.spark.storage.BlockManager;
 import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.UnsafeAlignedOffset;
+import org.apache.spark.unsafe.array.LongArray;
 import org.apache.spark.unsafe.memory.MemoryBlock;
-import org.apache.spark.util.TaskCompletionListener;
 import org.apache.spark.util.Utils;
 
 /**
@@ -41,18 +45,35 @@
  */
 public final class UnsafeExternalSorter extends MemoryConsumer {
 
-  private final Logger logger = LoggerFactory.getLogger(UnsafeExternalSorter.class);
+  private static final Logger logger = LoggerFactory.getLogger(UnsafeExternalSorter.class);
 
+  @Nullable
   private final PrefixComparator prefixComparator;
-  private final RecordComparator recordComparator;
+
+  /**
+   * {@link RecordComparator} may probably keep the reference to the records they compared last
+   * time, so we should not keep a {@link RecordComparator} instance inside
+   * {@link UnsafeExternalSorter}, because {@link UnsafeExternalSorter} is referenced by
+   * {@link TaskContext} and thus can not be garbage collected until the end of the task.
+   */
+  @Nullable
+  private final Supplier recordComparatorSupplier;
+
   private final TaskMemoryManager taskMemoryManager;
   private final BlockManager blockManager;
+  private final SerializerManager serializerManager;
   private final TaskContext taskContext;
-  private ShuffleWriteMetrics writeMetrics;
 
   /** The buffer size to use when writing spills using DiskBlockObjectWriter */
   private final int fileBufferSizeBytes;
 
+  /**
+   * Force this sorter to spill when there are this many elements in memory. The default value is
+   * 1024 * 1024 * 1024 / 2 which allows the maximum size of the pointer array to be 8G.
+   */
+  public static final long DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD = 1024 * 1024 * 1024 / 2;
+
+  private final long numElementsForSpillThreshold;
   /**
    * Memory pages that hold the records being sorted. The pages in this list are freed when
    * spilling, although in principle we could recycle these pages across spills (on the other hand,
@@ -69,19 +90,24 @@ public final class UnsafeExternalSorter extends MemoryConsumer {
   private MemoryBlock currentPage = null;
   private long pageCursor = -1;
   private long peakMemoryUsedBytes = 0;
+  private long totalSpillBytes = 0L;
+  private long totalSortTimeNanos = 0L;
   private volatile SpillableIterator readingIterator = null;
 
   public static UnsafeExternalSorter createWithExistingInMemorySorter(
       TaskMemoryManager taskMemoryManager,
       BlockManager blockManager,
+      SerializerManager serializerManager,
       TaskContext taskContext,
-      RecordComparator recordComparator,
+      Supplier recordComparatorSupplier,
       PrefixComparator prefixComparator,
       int initialSize,
       long pageSizeBytes,
+      long numElementsForSpillThreshold,
       UnsafeInMemorySorter inMemorySorter) throws IOException {
     UnsafeExternalSorter sorter = new UnsafeExternalSorter(taskMemoryManager, blockManager,
-      taskContext, recordComparator, prefixComparator, initialSize, pageSizeBytes, inMemorySorter);
+      serializerManager, taskContext, recordComparatorSupplier, prefixComparator, initialSize,
+        numElementsForSpillThreshold, pageSizeBytes, inMemorySorter, false /* ignored */);
     sorter.spill(Long.MAX_VALUE, sorter);
     // The external sorter will be used to insert records, in-memory sorter is not needed.
     sorter.inMemSorter = null;
@@ -91,57 +117,66 @@ public static UnsafeExternalSorter createWithExistingInMemorySorter(
   public static UnsafeExternalSorter create(
       TaskMemoryManager taskMemoryManager,
       BlockManager blockManager,
+      SerializerManager serializerManager,
       TaskContext taskContext,
-      RecordComparator recordComparator,
+      Supplier recordComparatorSupplier,
       PrefixComparator prefixComparator,
       int initialSize,
-      long pageSizeBytes) {
-    return new UnsafeExternalSorter(taskMemoryManager, blockManager,
-      taskContext, recordComparator, prefixComparator, initialSize, pageSizeBytes, null);
+      long pageSizeBytes,
+      long numElementsForSpillThreshold,
+      boolean canUseRadixSort) {
+    return new UnsafeExternalSorter(taskMemoryManager, blockManager, serializerManager,
+      taskContext, recordComparatorSupplier, prefixComparator, initialSize, pageSizeBytes,
+      numElementsForSpillThreshold, null, canUseRadixSort);
   }
 
   private UnsafeExternalSorter(
       TaskMemoryManager taskMemoryManager,
       BlockManager blockManager,
+      SerializerManager serializerManager,
       TaskContext taskContext,
-      RecordComparator recordComparator,
+      Supplier recordComparatorSupplier,
       PrefixComparator prefixComparator,
       int initialSize,
       long pageSizeBytes,
-      @Nullable UnsafeInMemorySorter existingInMemorySorter) {
-    super(taskMemoryManager, pageSizeBytes);
+      long numElementsForSpillThreshold,
+      @Nullable UnsafeInMemorySorter existingInMemorySorter,
+      boolean canUseRadixSort) {
+    super(taskMemoryManager, pageSizeBytes, taskMemoryManager.getTungstenMemoryMode());
     this.taskMemoryManager = taskMemoryManager;
     this.blockManager = blockManager;
+    this.serializerManager = serializerManager;
     this.taskContext = taskContext;
-    this.recordComparator = recordComparator;
+    this.recordComparatorSupplier = recordComparatorSupplier;
     this.prefixComparator = prefixComparator;
     // Use getSizeAsKb (not bytes) to maintain backwards compatibility for units
-    // this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024;
+    // this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024
     this.fileBufferSizeBytes = 32 * 1024;
-    // TODO: metrics tracking + integration with shuffle write metrics
-    // need to connect the write metrics to task metrics so we count the spill IO somewhere.
-    this.writeMetrics = new ShuffleWriteMetrics();
 
     if (existingInMemorySorter == null) {
-      this.inMemSorter =
-        new UnsafeInMemorySorter(taskMemoryManager, recordComparator, prefixComparator, initialSize);
-      acquireMemory(inMemSorter.getMemoryUsage());
+      RecordComparator comparator = null;
+      if (recordComparatorSupplier != null) {
+        comparator = recordComparatorSupplier.get();
+      }
+      this.inMemSorter = new UnsafeInMemorySorter(
+        this,
+        taskMemoryManager,
+        comparator,
+        prefixComparator,
+        initialSize,
+        canUseRadixSort);
     } else {
       this.inMemSorter = existingInMemorySorter;
     }
     this.peakMemoryUsedBytes = getMemoryUsage();
+    this.numElementsForSpillThreshold = numElementsForSpillThreshold;
 
     // Register a cleanup task with TaskContext to ensure that memory is guaranteed to be freed at
     // the end of the task. This is necessary to avoid memory leaks in when the downstream operator
     // does not fully consume the sorter's output (e.g. sort followed by limit).
-    taskContext.addTaskCompletionListener(
-      new TaskCompletionListener() {
-        @Override
-        public void onTaskCompletion(TaskContext context) {
-          cleanupResources();
-        }
-      }
-    );
+    taskContext.addTaskCompletionListener(context -> {
+      cleanupResources();
+    });
   }
 
   /**
@@ -177,31 +212,28 @@ public long spill(long size, MemoryConsumer trigger) throws IOException {
       spillWriters.size(),
       spillWriters.size() > 1 ? " times" : " time");
 
+    ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics();
     // We only write out contents of the inMemSorter if it is not empty.
     if (inMemSorter.numRecords() > 0) {
       final UnsafeSorterSpillWriter spillWriter =
         new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, writeMetrics,
           inMemSorter.numRecords());
       spillWriters.add(spillWriter);
-      final UnsafeSorterIterator sortedRecords = inMemSorter.getSortedIterator();
-      while (sortedRecords.hasNext()) {
-        sortedRecords.loadNext();
-        final Object baseObject = sortedRecords.getBaseObject();
-        final long baseOffset = sortedRecords.getBaseOffset();
-        final int recordLength = sortedRecords.getRecordLength();
-        spillWriter.write(baseObject, baseOffset, recordLength, sortedRecords.getKeyPrefix());
-      }
-      spillWriter.close();
-
-      inMemSorter.reset();
+      spillIterator(inMemSorter.getSortedIterator(), spillWriter);
     }
 
     final long spillSize = freeMemory();
     // Note that this is more-or-less going to be a multiple of the page size, so wasted space in
     // pages will currently be counted as memory spilled even though that space isn't actually
     // written to disk. This also counts the space needed to store the sorter's pointer array.
-    taskContext.taskMetrics().incMemoryBytesSpilled(spillSize);
+    inMemSorter.reset();
+    // Reset the in-memory sorter's pointer array only after freeing up the memory pages holding the
+    // records. Otherwise, if the task is over allocated memory, then without freeing the memory
+    // pages, we might not be able to get memory for the pointer array.
 
+    taskContext.taskMetrics().incMemoryBytesSpilled(spillSize);
+    taskContext.taskMetrics().incDiskBytesSpilled(writeMetrics.bytesWritten());
+    totalSpillBytes += spillSize;
     return spillSize;
   }
 
@@ -232,6 +264,24 @@ public long getPeakMemoryUsedBytes() {
     return peakMemoryUsedBytes;
   }
 
+  /**
+   * @return the total amount of time spent sorting data (in-memory only).
+   */
+  public long getSortTimeNanos() {
+    UnsafeInMemorySorter sorter = inMemSorter;
+    if (sorter != null) {
+      return sorter.getSortTimeNanos();
+    }
+    return totalSortTimeNanos;
+  }
+
+  /**
+   * Return the total number of bytes that has been spilled into disk so far.
+   */
+  public long getSpillSize() {
+    return totalSpillBytes;
+  }
+
   @VisibleForTesting
   public int getNumberOfAllocatedPages() {
     return allocatedPages.size();
@@ -277,9 +327,8 @@ public void cleanupResources() {
       deleteSpillFiles();
       freeMemory();
       if (inMemSorter != null) {
-        long used = inMemSorter.getMemoryUsage();
+        inMemSorter.free();
         inMemSorter = null;
-        releaseMemory(used);
       }
     }
   }
@@ -289,30 +338,27 @@ public void cleanupResources() {
    * array and grows the array if additional space is required. If the required space cannot be
    * obtained, then the in-memory data will be spilled to disk.
    */
-  private void growPointerArrayIfNecessary() throws IOException {
+  private void growPointerArrayIfNecessary() {
     assert(inMemSorter != null);
     if (!inMemSorter.hasSpaceForAnotherRecord()) {
       long used = inMemSorter.getMemoryUsage();
-      long needed = used + inMemSorter.getMemoryToExpand();
+      LongArray array;
       try {
-        acquireMemory(needed);  // could trigger spilling
+        // could trigger spilling
+        array = allocateArray(used / 8 * 2);
       } catch (OutOfMemoryError e) {
         // should have trigger spilling
-        assert(inMemSorter.hasSpaceForAnotherRecord());
+        if (!inMemSorter.hasSpaceForAnotherRecord()) {
+          logger.error("Unable to grow the pointer array");
+          throw e;
+        }
         return;
       }
       // check if spilling is triggered or not
       if (inMemSorter.hasSpaceForAnotherRecord()) {
-        releaseMemory(needed);
+        freeArray(array);
       } else {
-        try {
-          inMemSorter.expandPointerArray();
-          releaseMemory(used);
-        } catch (OutOfMemoryError oom) {
-          // Just in case that JVM had run out of memory
-          releaseMemory(needed);
-          spill();
-        }
+        inMemSorter.expandPointerArray(array);
       }
     }
   }
@@ -339,22 +385,30 @@ private void acquireNewPageIfNecessary(int required) {
   /**
    * Write a record to the sorter.
    */
-  public void insertRecord(Object recordBase, long recordOffset, int length, long prefix)
+  public void insertRecord(
+      Object recordBase, long recordOffset, int length, long prefix, boolean prefixIsNull)
     throws IOException {
 
+    assert(inMemSorter != null);
+    if (inMemSorter.numRecords() >= numElementsForSpillThreshold) {
+      logger.info("Spilling data because number of spilledRecords crossed the threshold " +
+        numElementsForSpillThreshold);
+      spill();
+    }
+
     growPointerArrayIfNecessary();
+    int uaoSize = UnsafeAlignedOffset.getUaoSize();
     // Need 4 bytes to store the record length.
-    final int required = length + 4;
+    final int required = length + uaoSize;
     acquireNewPageIfNecessary(required);
 
     final Object base = currentPage.getBaseObject();
     final long recordAddress = taskMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor);
-    Platform.putInt(base, pageCursor, length);
-    pageCursor += 4;
+    UnsafeAlignedOffset.putSize(base, pageCursor, length);
+    pageCursor += uaoSize;
     Platform.copyMemory(recordBase, recordOffset, base, pageCursor, length);
     pageCursor += length;
-    assert(inMemSorter != null);
-    inMemSorter.insertRecord(recordAddress, prefix);
+    inMemSorter.insertRecord(recordAddress, prefix, prefixIsNull);
   }
 
   /**
@@ -366,26 +420,27 @@ public void insertRecord(Object recordBase, long recordOffset, int length, long
    * record length = key length + value length + 4
    */
   public void insertKVRecord(Object keyBase, long keyOffset, int keyLen,
-      Object valueBase, long valueOffset, int valueLen, long prefix)
+      Object valueBase, long valueOffset, int valueLen, long prefix, boolean prefixIsNull)
     throws IOException {
 
     growPointerArrayIfNecessary();
-    final int required = keyLen + valueLen + 4 + 4;
+    int uaoSize = UnsafeAlignedOffset.getUaoSize();
+    final int required = keyLen + valueLen + (2 * uaoSize);
     acquireNewPageIfNecessary(required);
 
     final Object base = currentPage.getBaseObject();
     final long recordAddress = taskMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor);
-    Platform.putInt(base, pageCursor, keyLen + valueLen + 4);
-    pageCursor += 4;
-    Platform.putInt(base, pageCursor, keyLen);
-    pageCursor += 4;
+    UnsafeAlignedOffset.putSize(base, pageCursor, keyLen + valueLen + uaoSize);
+    pageCursor += uaoSize;
+    UnsafeAlignedOffset.putSize(base, pageCursor, keyLen);
+    pageCursor += uaoSize;
     Platform.copyMemory(keyBase, keyOffset, base, pageCursor, keyLen);
     pageCursor += keyLen;
     Platform.copyMemory(valueBase, valueOffset, base, pageCursor, valueLen);
     pageCursor += valueLen;
 
     assert(inMemSorter != null);
-    inMemSorter.insertRecord(recordAddress, prefix);
+    inMemSorter.insertRecord(recordAddress, prefix, prefixIsNull);
   }
 
   /**
@@ -406,15 +461,16 @@ public void merge(UnsafeExternalSorter other) throws IOException {
    * after consuming this iterator.
    */
   public UnsafeSorterIterator getSortedIterator() throws IOException {
+    assert(recordComparatorSupplier != null);
     if (spillWriters.isEmpty()) {
       assert(inMemSorter != null);
       readingIterator = new SpillableIterator(inMemSorter.getSortedIterator());
       return readingIterator;
     } else {
-      final UnsafeSorterSpillMerger spillMerger =
-        new UnsafeSorterSpillMerger(recordComparator, prefixComparator, spillWriters.size());
+      final UnsafeSorterSpillMerger spillMerger = new UnsafeSorterSpillMerger(
+        recordComparatorSupplier.get(), prefixComparator, spillWriters.size());
       for (UnsafeSorterSpillWriter spillWriter : spillWriters) {
-        spillMerger.addSpillIfNotEmpty(spillWriter.getReader(blockManager));
+        spillMerger.addSpillIfNotEmpty(spillWriter.getReader(serializerManager));
       }
       if (inMemSorter != null) {
         readingIterator = new SpillableIterator(inMemSorter.getSortedIterator());
@@ -424,6 +480,18 @@ public UnsafeSorterIterator getSortedIterator() throws IOException {
     }
   }
 
+  private static void spillIterator(UnsafeSorterIterator inMemIterator,
+      UnsafeSorterSpillWriter spillWriter) throws IOException {
+    while (inMemIterator.hasNext()) {
+      inMemIterator.loadNext();
+      final Object baseObject = inMemIterator.getBaseObject();
+      final long baseOffset = inMemIterator.getBaseOffset();
+      final int recordLength = inMemIterator.getRecordLength();
+      spillWriter.write(baseObject, baseOffset, recordLength, inMemIterator.getKeyPrefix());
+    }
+    spillWriter.close();
+  }
+
   /**
    * An UnsafeSorterIterator that support spilling.
    */
@@ -434,9 +502,14 @@ class SpillableIterator extends UnsafeSorterIterator {
     private boolean loaded = false;
     private int numRecords = 0;
 
-    public SpillableIterator(UnsafeInMemorySorter.SortedIterator inMemIterator) {
+    SpillableIterator(UnsafeSorterIterator inMemIterator) {
       this.upstream = inMemIterator;
-      this.numRecords = inMemIterator.numRecordsLeft();
+      this.numRecords = inMemIterator.getNumRecords();
+    }
+
+    @Override
+    public int getNumRecords() {
+      return numRecords;
     }
 
     public long spill() throws IOException {
@@ -449,24 +522,22 @@ public long spill() throws IOException {
         UnsafeInMemorySorter.SortedIterator inMemIterator =
           ((UnsafeInMemorySorter.SortedIterator) upstream).clone();
 
+       ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics();
+        // Iterate over the records that have not been returned and spill them.
         final UnsafeSorterSpillWriter spillWriter =
           new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, writeMetrics, numRecords);
-        while (inMemIterator.hasNext()) {
-          inMemIterator.loadNext();
-          final Object baseObject = inMemIterator.getBaseObject();
-          final long baseOffset = inMemIterator.getBaseOffset();
-          final int recordLength = inMemIterator.getRecordLength();
-          spillWriter.write(baseObject, baseOffset, recordLength, inMemIterator.getKeyPrefix());
-        }
-        spillWriter.close();
+        spillIterator(inMemIterator, spillWriter);
         spillWriters.add(spillWriter);
-        nextUpstream = spillWriter.getReader(blockManager);
+        nextUpstream = spillWriter.getReader(serializerManager);
 
         long released = 0L;
         synchronized (UnsafeExternalSorter.this) {
-          // release the pages except the one that is used
+          // release the pages except the one that is used. There can still be a caller that
+          // is accessing the current record. We free this page in that caller's next loadNext()
+          // call.
           for (MemoryBlock page : allocatedPages) {
-            if (!loaded || page.getBaseObject() != inMemIterator.getBaseObject()) {
+            if (!loaded || page.pageNumber !=
+                    ((UnsafeInMemorySorter.SortedIterator)upstream).getCurrentPageNumber()) {
               released += page.size();
               freePage(page);
             } else {
@@ -475,6 +546,16 @@ public long spill() throws IOException {
           }
           allocatedPages.clear();
         }
+
+        // in-memory sorter will not be used after spilling
+        assert(inMemSorter != null);
+        released += inMemSorter.getMemoryUsage();
+        totalSortTimeNanos += inMemSorter.getSortTimeNanos();
+        inMemSorter.free();
+        inMemSorter = null;
+        taskContext.taskMetrics().incMemoryBytesSpilled(released);
+        taskContext.taskMetrics().incDiskBytesSpilled(writeMetrics.bytesWritten());
+        totalSpillBytes += released;
         return released;
       }
     }
@@ -496,11 +577,6 @@ public void loadNext() throws IOException {
           }
           upstream = nextUpstream;
           nextUpstream = null;
-
-          assert(inMemSorter != null);
-          long used = inMemSorter.getMemoryUsage();
-          inMemSorter = null;
-          releaseMemory(used);
         }
         numRecords--;
         upstream.loadNext();
@@ -527,4 +603,106 @@ public long getKeyPrefix() {
       return upstream.getKeyPrefix();
     }
   }
+
+  /**
+   * Returns an iterator starts from startIndex, which will return the rows in the order as
+   * inserted.
+   *
+   * It is the caller's responsibility to call `cleanupResources()`
+   * after consuming this iterator.
+   *
+   * TODO: support forced spilling
+   */
+  public UnsafeSorterIterator getIterator(int startIndex) throws IOException {
+    if (spillWriters.isEmpty()) {
+      assert(inMemSorter != null);
+      UnsafeSorterIterator iter = inMemSorter.getSortedIterator();
+      moveOver(iter, startIndex);
+      return iter;
+    } else {
+      LinkedList queue = new LinkedList<>();
+      int i = 0;
+      for (UnsafeSorterSpillWriter spillWriter : spillWriters) {
+        if (i + spillWriter.recordsSpilled() > startIndex) {
+          UnsafeSorterIterator iter = spillWriter.getReader(serializerManager);
+          moveOver(iter, startIndex - i);
+          queue.add(iter);
+        }
+        i += spillWriter.recordsSpilled();
+      }
+      if (inMemSorter != null) {
+        UnsafeSorterIterator iter = inMemSorter.getSortedIterator();
+        moveOver(iter, startIndex - i);
+        queue.add(iter);
+      }
+      return new ChainedIterator(queue);
+    }
+  }
+
+  private void moveOver(UnsafeSorterIterator iter, int steps)
+      throws IOException {
+    if (steps > 0) {
+      for (int i = 0; i < steps; i++) {
+        if (iter.hasNext()) {
+          iter.loadNext();
+        } else {
+          throw new ArrayIndexOutOfBoundsException("Failed to move the iterator " + steps +
+            " steps forward");
+        }
+      }
+    }
+  }
+
+  /**
+   * Chain multiple UnsafeSorterIterator together as single one.
+   */
+  static class ChainedIterator extends UnsafeSorterIterator {
+
+    private final Queue iterators;
+    private UnsafeSorterIterator current;
+    private int numRecords;
+
+    ChainedIterator(Queue iterators) {
+      assert iterators.size() > 0;
+      this.numRecords = 0;
+      for (UnsafeSorterIterator iter: iterators) {
+        this.numRecords += iter.getNumRecords();
+      }
+      this.iterators = iterators;
+      this.current = iterators.remove();
+    }
+
+    @Override
+    public int getNumRecords() {
+      return numRecords;
+    }
+
+    @Override
+    public boolean hasNext() {
+      while (!current.hasNext() && !iterators.isEmpty()) {
+        current = iterators.remove();
+      }
+      return current.hasNext();
+    }
+
+    @Override
+    public void loadNext() throws IOException {
+      while (!current.hasNext() && !iterators.isEmpty()) {
+        current = iterators.remove();
+      }
+      current.loadNext();
+    }
+
+    @Override
+    public Object getBaseObject() { return current.getBaseObject(); }
+
+    @Override
+    public long getBaseOffset() { return current.getBaseOffset(); }
+
+    @Override
+    public int getRecordLength() { return current.getRecordLength(); }
+
+    @Override
+    public long getKeyPrefix() { return current.getKeyPrefix(); }
+  }
 }
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
index d57213b9b8bf..c14c12664f5a 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
@@ -18,9 +18,17 @@
 package org.apache.spark.util.collection.unsafe.sort;
 
 import java.util.Comparator;
+import java.util.LinkedList;
 
+import org.apache.avro.reflect.Nullable;
+
+import org.apache.spark.TaskContext;
+import org.apache.spark.memory.MemoryConsumer;
 import org.apache.spark.memory.TaskMemoryManager;
 import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.UnsafeAlignedOffset;
+import org.apache.spark.unsafe.array.LongArray;
+import org.apache.spark.unsafe.memory.MemoryBlock;
 import org.apache.spark.util.collection.Sorter;
 
 /**
@@ -50,11 +58,14 @@ private static final class SortComparator implements Comparator sorter;
+  @Nullable
   private final Comparator sortComparator;
 
   /**
-   * Within this buffer, position {@code 2 * i} holds a pointer pointer to the record at
+   * If non-null, specifies the radix sort parameters and that radix sort will be used.
+   */
+  @Nullable
+  private final PrefixComparators.RadixSortSupport radixSortSupport;
+
+  /**
+   * Within this buffer, position {@code 2 * i} holds a pointer to the record at
    * index {@code i}, while position {@code 2 * i + 1} in the array holds an 8-byte key prefix.
+   *
+   * Only part of the array will be used to store the pointers, the rest part is preserved as
+   * temporary buffer for sorting.
    */
-  private long[] array;
+  private LongArray array;
 
   /**
    * The position in the sort buffer where new records can be inserted.
    */
   private int pos = 0;
 
+  /**
+   * If sorting with radix sort, specifies the starting position in the sort buffer where records
+   * with non-null prefixes are kept. Positions [0..nullBoundaryPos) will contain null-prefixed
+   * records, and positions [nullBoundaryPos..pos) non-null prefixed records. This lets us avoid
+   * radix sorting over null values.
+   */
+  private int nullBoundaryPos = 0;
+
+  /*
+   * How many records could be inserted, because part of the array should be left for sorting.
+   */
+  private int usableCapacity = 0;
+
+  private long initialSize;
+
+  private long totalSortTimeNanos = 0L;
+
   public UnsafeInMemorySorter(
+    final MemoryConsumer consumer,
     final TaskMemoryManager memoryManager,
     final RecordComparator recordComparator,
     final PrefixComparator prefixComparator,
-    int initialSize) {
-    this(memoryManager, recordComparator, prefixComparator, new long[initialSize * 2]);
+    int initialSize,
+    boolean canUseRadixSort) {
+    this(consumer, memoryManager, recordComparator, prefixComparator,
+      consumer.allocateArray(initialSize * 2), canUseRadixSort);
   }
 
   public UnsafeInMemorySorter(
+      final MemoryConsumer consumer,
       final TaskMemoryManager memoryManager,
       final RecordComparator recordComparator,
       final PrefixComparator prefixComparator,
-      long[] array) {
-    this.array = array;
+      LongArray array,
+      boolean canUseRadixSort) {
+    this.consumer = consumer;
     this.memoryManager = memoryManager;
-    this.sorter = new Sorter<>(UnsafeSortDataFormat.INSTANCE);
-    this.sortComparator = new SortComparator(recordComparator, prefixComparator, memoryManager);
+    this.initialSize = array.size();
+    if (recordComparator != null) {
+      this.sortComparator = new SortComparator(recordComparator, prefixComparator, memoryManager);
+      if (canUseRadixSort && prefixComparator instanceof PrefixComparators.RadixSortSupport) {
+        this.radixSortSupport = (PrefixComparators.RadixSortSupport)prefixComparator;
+      } else {
+        this.radixSortSupport = null;
+      }
+    } else {
+      this.sortComparator = null;
+      this.radixSortSupport = null;
+    }
+    this.array = array;
+    this.usableCapacity = getUsableCapacity();
+  }
+
+  private int getUsableCapacity() {
+    // Radix sort requires same amount of used memory as buffer, Tim sort requires
+    // half of the used memory as buffer.
+    return (int) (array.size() / (radixSortSupport != null ? 2 : 1.5));
+  }
+
+  /**
+   * Free the memory used by pointer array.
+   */
+  public void free() {
+    if (consumer != null) {
+      consumer.freeArray(array);
+      array = null;
+    }
   }
 
   public void reset() {
+    if (consumer != null) {
+      consumer.freeArray(array);
+      array = consumer.allocateArray(initialSize);
+      usableCapacity = getUsableCapacity();
+    }
     pos = 0;
+    nullBoundaryPos = 0;
   }
 
   /**
@@ -107,26 +184,34 @@ public int numRecords() {
     return pos / 2;
   }
 
-  private int newLength() {
-    return array.length < Integer.MAX_VALUE / 2 ? (array.length * 2) : Integer.MAX_VALUE;
-  }
-
-  public long getMemoryToExpand() {
-    return (long) (newLength() - array.length) * 8L;
+  /**
+   * @return the total amount of time spent sorting data (in-memory only).
+   */
+  public long getSortTimeNanos() {
+    return totalSortTimeNanos;
   }
 
   public long getMemoryUsage() {
-    return array.length * 8L;
+    return array.size() * 8;
   }
 
   public boolean hasSpaceForAnotherRecord() {
-    return pos + 2 <= array.length;
+    return pos + 1 < usableCapacity;
   }
 
-  public void expandPointerArray() {
-    final long[] oldArray = array;
-    array = new long[newLength()];
-    System.arraycopy(oldArray, 0, array, 0, oldArray.length);
+  public void expandPointerArray(LongArray newArray) {
+    if (newArray.size() < array.size()) {
+      throw new OutOfMemoryError("Not enough memory to grow pointer array");
+    }
+    Platform.copyMemory(
+      array.getBaseObject(),
+      array.getBaseOffset(),
+      newArray.getBaseObject(),
+      newArray.getBaseOffset(),
+      pos * 8L);
+    consumer.freeArray(array);
+    array = newArray;
+    usableCapacity = getUsableCapacity();
   }
 
   /**
@@ -136,63 +221,87 @@ public void expandPointerArray() {
    * @param recordPointer pointer to a record in a data page, encoded by {@link TaskMemoryManager}.
    * @param keyPrefix a user-defined key prefix
    */
-  public void insertRecord(long recordPointer, long keyPrefix) {
+  public void insertRecord(long recordPointer, long keyPrefix, boolean prefixIsNull) {
     if (!hasSpaceForAnotherRecord()) {
-      expandPointerArray();
+      throw new IllegalStateException("There is no space for new record");
+    }
+    if (prefixIsNull && radixSortSupport != null) {
+      // Swap forward a non-null record to make room for this one at the beginning of the array.
+      array.set(pos, array.get(nullBoundaryPos));
+      pos++;
+      array.set(pos, array.get(nullBoundaryPos + 1));
+      pos++;
+      // Place this record in the vacated position.
+      array.set(nullBoundaryPos, recordPointer);
+      nullBoundaryPos++;
+      array.set(nullBoundaryPos, keyPrefix);
+      nullBoundaryPos++;
+    } else {
+      array.set(pos, recordPointer);
+      pos++;
+      array.set(pos, keyPrefix);
+      pos++;
     }
-    array[pos] = recordPointer;
-    pos++;
-    array[pos] = keyPrefix;
-    pos++;
   }
 
-  public static final class SortedIterator extends UnsafeSorterIterator {
+  public final class SortedIterator extends UnsafeSorterIterator implements Cloneable {
 
-    private final TaskMemoryManager memoryManager;
-    private final int sortBufferInsertPosition;
-    private final long[] sortBuffer;
-    private int position = 0;
+    private final int numRecords;
+    private int position;
+    private int offset;
     private Object baseObject;
     private long baseOffset;
     private long keyPrefix;
     private int recordLength;
+    private long currentPageNumber;
+    private final TaskContext taskContext = TaskContext.get();
 
-    private SortedIterator(
-        TaskMemoryManager memoryManager,
-        int sortBufferInsertPosition,
-        long[] sortBuffer) {
-      this.memoryManager = memoryManager;
-      this.sortBufferInsertPosition = sortBufferInsertPosition;
-      this.sortBuffer = sortBuffer;
+    private SortedIterator(int numRecords, int offset) {
+      this.numRecords = numRecords;
+      this.position = 0;
+      this.offset = offset;
     }
 
-    public SortedIterator clone () {
-      SortedIterator iter = new SortedIterator(memoryManager, sortBufferInsertPosition, sortBuffer);
+    public SortedIterator clone() {
+      SortedIterator iter = new SortedIterator(numRecords, offset);
       iter.position = position;
       iter.baseObject = baseObject;
       iter.baseOffset = baseOffset;
       iter.keyPrefix = keyPrefix;
       iter.recordLength = recordLength;
+      iter.currentPageNumber = currentPageNumber;
       return iter;
     }
 
     @Override
-    public boolean hasNext() {
-      return position < sortBufferInsertPosition;
+    public int getNumRecords() {
+      return numRecords;
     }
 
-    public int numRecordsLeft() {
-      return (sortBufferInsertPosition - position) / 2;
+    @Override
+    public boolean hasNext() {
+      return position / 2 < numRecords;
     }
 
     @Override
     public void loadNext() {
+      // Kill the task in case it has been marked as killed. This logic is from
+      // InterruptibleIterator, but we inline it here instead of wrapping the iterator in order
+      // to avoid performance overhead. This check is added here in `loadNext()` instead of in
+      // `hasNext()` because it's technically possible for the caller to be relying on
+      // `getNumRecords()` instead of `hasNext()` to know when to stop.
+      if (taskContext != null) {
+        taskContext.killTaskIfInterrupted();
+      }
       // This pointer points to a 4-byte record length, followed by the record's bytes
-      final long recordPointer = sortBuffer[position];
+      final long recordPointer = array.get(offset + position);
+      currentPageNumber = TaskMemoryManager.decodePageNumber(recordPointer);
+      int uaoSize = UnsafeAlignedOffset.getUaoSize();
       baseObject = memoryManager.getPage(recordPointer);
-      baseOffset = memoryManager.getOffsetInPage(recordPointer) + 4;  // Skip over record length
-      recordLength = Platform.getInt(baseObject, baseOffset - 4);
-      keyPrefix = sortBuffer[position + 1];
+      // Skip over record length
+      baseOffset = memoryManager.getOffsetInPage(recordPointer) + uaoSize;
+      recordLength = UnsafeAlignedOffset.getSize(baseObject, baseOffset - uaoSize);
+      keyPrefix = array.get(offset + position + 1);
       position += 2;
     }
 
@@ -202,6 +311,10 @@ public void loadNext() {
     @Override
     public long getBaseOffset() { return baseOffset; }
 
+    public long getCurrentPageNumber() {
+      return currentPageNumber;
+    }
+
     @Override
     public int getRecordLength() { return recordLength; }
 
@@ -213,8 +326,41 @@ public void loadNext() {
    * Return an iterator over record pointers in sorted order. For efficiency, all calls to
    * {@code next()} will return the same mutable object.
    */
-  public SortedIterator getSortedIterator() {
-    sorter.sort(array, 0, pos / 2, sortComparator);
-    return new SortedIterator(memoryManager, pos, array);
+  public UnsafeSorterIterator getSortedIterator() {
+    int offset = 0;
+    long start = System.nanoTime();
+    if (sortComparator != null) {
+      if (this.radixSortSupport != null) {
+        offset = RadixSort.sortKeyPrefixArray(
+          array, nullBoundaryPos, (pos - nullBoundaryPos) / 2L, 0, 7,
+          radixSortSupport.sortDescending(), radixSortSupport.sortSigned());
+      } else {
+        MemoryBlock unused = new MemoryBlock(
+          array.getBaseObject(),
+          array.getBaseOffset() + pos * 8L,
+          (array.size() - pos) * 8L);
+        LongArray buffer = new LongArray(unused);
+        Sorter sorter =
+          new Sorter<>(new UnsafeSortDataFormat(buffer));
+        sorter.sort(array, 0, pos / 2, sortComparator);
+      }
+    }
+    totalSortTimeNanos += System.nanoTime() - start;
+    if (nullBoundaryPos > 0) {
+      assert radixSortSupport != null : "Nulls are only stored separately with radix sort";
+      LinkedList queue = new LinkedList<>();
+
+      // The null order is either LAST or FIRST, regardless of sorting direction (ASC|DESC)
+      if (radixSortSupport.nullsFirst()) {
+        queue.add(new SortedIterator(nullBoundaryPos / 2, 0));
+        queue.add(new SortedIterator((pos - nullBoundaryPos) / 2, offset));
+      } else {
+        queue.add(new SortedIterator((pos - nullBoundaryPos) / 2, offset));
+        queue.add(new SortedIterator(nullBoundaryPos / 2, 0));
+      }
+      return new UnsafeExternalSorter.ChainedIterator(queue);
+    } else {
+      return new SortedIterator(pos / 2, offset);
+    }
   }
 }
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java
index d09c728a7a63..d9f84d10e905 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java
@@ -17,23 +17,28 @@
 
 package org.apache.spark.util.collection.unsafe.sort;
 
+import org.apache.spark.unsafe.Platform;
+import org.apache.spark.unsafe.array.LongArray;
 import org.apache.spark.util.collection.SortDataFormat;
 
 /**
  * Supports sorting an array of (record pointer, key prefix) pairs.
  * Used in {@link UnsafeInMemorySorter}.
  * 

- * Within each long[] buffer, position {@code 2 * i} holds a pointer pointer to the record at + * Within each long[] buffer, position {@code 2 * i} holds a pointer to the record at * index {@code i}, while position {@code 2 * i + 1} in the array holds an 8-byte key prefix. */ -final class UnsafeSortDataFormat extends SortDataFormat { +public final class UnsafeSortDataFormat + extends SortDataFormat { - public static final UnsafeSortDataFormat INSTANCE = new UnsafeSortDataFormat(); + private final LongArray buffer; - private UnsafeSortDataFormat() { } + public UnsafeSortDataFormat(LongArray buffer) { + this.buffer = buffer; + } @Override - public RecordPointerAndKeyPrefix getKey(long[] data, int pos) { + public RecordPointerAndKeyPrefix getKey(LongArray data, int pos) { // Since we re-use keys, this method shouldn't be called. throw new UnsupportedOperationException(); } @@ -44,37 +49,44 @@ public RecordPointerAndKeyPrefix newKey() { } @Override - public RecordPointerAndKeyPrefix getKey(long[] data, int pos, RecordPointerAndKeyPrefix reuse) { - reuse.recordPointer = data[pos * 2]; - reuse.keyPrefix = data[pos * 2 + 1]; + public RecordPointerAndKeyPrefix getKey(LongArray data, int pos, + RecordPointerAndKeyPrefix reuse) { + reuse.recordPointer = data.get(pos * 2); + reuse.keyPrefix = data.get(pos * 2 + 1); return reuse; } @Override - public void swap(long[] data, int pos0, int pos1) { - long tempPointer = data[pos0 * 2]; - long tempKeyPrefix = data[pos0 * 2 + 1]; - data[pos0 * 2] = data[pos1 * 2]; - data[pos0 * 2 + 1] = data[pos1 * 2 + 1]; - data[pos1 * 2] = tempPointer; - data[pos1 * 2 + 1] = tempKeyPrefix; + public void swap(LongArray data, int pos0, int pos1) { + long tempPointer = data.get(pos0 * 2); + long tempKeyPrefix = data.get(pos0 * 2 + 1); + data.set(pos0 * 2, data.get(pos1 * 2)); + data.set(pos0 * 2 + 1, data.get(pos1 * 2 + 1)); + data.set(pos1 * 2, tempPointer); + data.set(pos1 * 2 + 1, tempKeyPrefix); } @Override - public void copyElement(long[] src, int srcPos, long[] dst, int dstPos) { - dst[dstPos * 2] = src[srcPos * 2]; - dst[dstPos * 2 + 1] = src[srcPos * 2 + 1]; + public void copyElement(LongArray src, int srcPos, LongArray dst, int dstPos) { + dst.set(dstPos * 2, src.get(srcPos * 2)); + dst.set(dstPos * 2 + 1, src.get(srcPos * 2 + 1)); } @Override - public void copyRange(long[] src, int srcPos, long[] dst, int dstPos, int length) { - System.arraycopy(src, srcPos * 2, dst, dstPos * 2, length * 2); + public void copyRange(LongArray src, int srcPos, LongArray dst, int dstPos, int length) { + Platform.copyMemory( + src.getBaseObject(), + src.getBaseOffset() + srcPos * 16L, + dst.getBaseObject(), + dst.getBaseOffset() + dstPos * 16L, + length * 16L); } @Override - public long[] allocate(int length) { - assert (length < Integer.MAX_VALUE / 2) : "Length " + length + " is too large"; - return new long[length * 2]; + public LongArray allocate(int length) { + assert (length * 2 <= buffer.size()) : + "the buffer is smaller than required: " + buffer.size() + " < " + (length * 2); + return buffer; } } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java index 16ac2e8d821b..1b3167fcc250 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java @@ -32,4 +32,6 @@ public abstract class UnsafeSorterIterator { public abstract int getRecordLength(); public abstract long getKeyPrefix(); + + public abstract int getNumRecords(); } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java index 3874a9f9cbdb..cf4dfde86ca9 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java @@ -23,28 +23,25 @@ final class UnsafeSorterSpillMerger { + private int numRecords = 0; private final PriorityQueue priorityQueue; - public UnsafeSorterSpillMerger( - final RecordComparator recordComparator, - final PrefixComparator prefixComparator, - final int numSpills) { - final Comparator comparator = new Comparator() { - - @Override - public int compare(UnsafeSorterIterator left, UnsafeSorterIterator right) { - final int prefixComparisonResult = - prefixComparator.compare(left.getKeyPrefix(), right.getKeyPrefix()); - if (prefixComparisonResult == 0) { - return recordComparator.compare( - left.getBaseObject(), left.getBaseOffset(), - right.getBaseObject(), right.getBaseOffset()); - } else { - return prefixComparisonResult; - } + UnsafeSorterSpillMerger( + RecordComparator recordComparator, + PrefixComparator prefixComparator, + int numSpills) { + Comparator comparator = (left, right) -> { + int prefixComparisonResult = + prefixComparator.compare(left.getKeyPrefix(), right.getKeyPrefix()); + if (prefixComparisonResult == 0) { + return recordComparator.compare( + left.getBaseObject(), left.getBaseOffset(), + right.getBaseObject(), right.getBaseOffset()); + } else { + return prefixComparisonResult; } }; - priorityQueue = new PriorityQueue(numSpills, comparator); + priorityQueue = new PriorityQueue<>(numSpills, comparator); } /** @@ -56,9 +53,10 @@ public void addSpillIfNotEmpty(UnsafeSorterIterator spillReader) throws IOExcept // make sure the hasNext method of UnsafeSorterIterator returned by getSortedIterator // does not return wrong result because hasNext will returns true // at least priorityQueue.size() times. If we allow n spillReaders in the - // priorityQueue, we will have n extra empty records in the result of the UnsafeSorterIterator. + // priorityQueue, we will have n extra empty records in the result of UnsafeSorterIterator. spillReader.loadNext(); priorityQueue.add(spillReader); + numRecords += spillReader.getNumRecords(); } } @@ -67,6 +65,11 @@ public UnsafeSorterIterator getSortedIterator() throws IOException { private UnsafeSorterIterator spillReader; + @Override + public int getNumRecords() { + return numRecords; + } + @Override public boolean hasNext() { return !priorityQueue.isEmpty() || (spillReader != null && spillReader.hasNext()); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java index 039e940a357e..e2f48e5508af 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java @@ -17,46 +17,88 @@ package org.apache.spark.util.collection.unsafe.sort; -import java.io.*; - import com.google.common.io.ByteStreams; +import com.google.common.io.Closeables; +import org.apache.spark.SparkEnv; +import org.apache.spark.TaskContext; +import org.apache.spark.io.NioBufferedFileInputStream; +import org.apache.spark.io.ReadAheadInputStream; +import org.apache.spark.serializer.SerializerManager; +import org.apache.spark.storage.BlockId; +import org.apache.spark.unsafe.Platform; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.spark.storage.BlockId; -import org.apache.spark.storage.BlockManager; -import org.apache.spark.unsafe.Platform; +import java.io.*; /** * Reads spill files written by {@link UnsafeSorterSpillWriter} (see that class for a description * of the file format). */ -public final class UnsafeSorterSpillReader extends UnsafeSorterIterator { +public final class UnsafeSorterSpillReader extends UnsafeSorterIterator implements Closeable { private static final Logger logger = LoggerFactory.getLogger(UnsafeSorterSpillReader.class); + private static final int DEFAULT_BUFFER_SIZE_BYTES = 1024 * 1024; // 1 MB + private static final int MAX_BUFFER_SIZE_BYTES = 16777216; // 16 mb - private final File file; private InputStream in; private DataInputStream din; // Variables that change with every record read: private int recordLength; private long keyPrefix; + private int numRecords; private int numRecordsRemaining; private byte[] arr = new byte[1024 * 1024]; private Object baseObject = arr; private final long baseOffset = Platform.BYTE_ARRAY_OFFSET; + private final TaskContext taskContext = TaskContext.get(); public UnsafeSorterSpillReader( - BlockManager blockManager, + SerializerManager serializerManager, File file, BlockId blockId) throws IOException { assert (file.length() > 0); - this.file = file; - final BufferedInputStream bs = new BufferedInputStream(new FileInputStream(file)); - this.in = blockManager.wrapForCompression(blockId, bs); - this.din = new DataInputStream(this.in); - numRecordsRemaining = din.readInt(); + long bufferSizeBytes = + SparkEnv.get() == null ? + DEFAULT_BUFFER_SIZE_BYTES: + SparkEnv.get().conf().getSizeAsBytes("spark.unsafe.sorter.spill.reader.buffer.size", + DEFAULT_BUFFER_SIZE_BYTES); + if (bufferSizeBytes > MAX_BUFFER_SIZE_BYTES || bufferSizeBytes < DEFAULT_BUFFER_SIZE_BYTES) { + // fall back to a sane default value + logger.warn("Value of config \"spark.unsafe.sorter.spill.reader.buffer.size\" = {} not in " + + "allowed range [{}, {}). Falling back to default value : {} bytes", bufferSizeBytes, + DEFAULT_BUFFER_SIZE_BYTES, MAX_BUFFER_SIZE_BYTES, DEFAULT_BUFFER_SIZE_BYTES); + bufferSizeBytes = DEFAULT_BUFFER_SIZE_BYTES; + } + + final double readAheadFraction = + SparkEnv.get() == null ? 0.5 : + SparkEnv.get().conf().getDouble("spark.unsafe.sorter.spill.read.ahead.fraction", 0.5); + + final boolean readAheadEnabled = SparkEnv.get() != null && + SparkEnv.get().conf().getBoolean("spark.unsafe.sorter.spill.read.ahead.enabled", true); + + final InputStream bs = + new NioBufferedFileInputStream(file, (int) bufferSizeBytes); + try { + if (readAheadEnabled) { + this.in = new ReadAheadInputStream(serializerManager.wrapStream(blockId, bs), + (int) bufferSizeBytes, (int) (bufferSizeBytes * readAheadFraction)); + } else { + this.in = serializerManager.wrapStream(blockId, bs); + } + this.din = new DataInputStream(this.in); + numRecords = numRecordsRemaining = din.readInt(); + } catch (IOException e) { + Closeables.close(bs, /* swallowIOException = */ true); + throw e; + } + } + + @Override + public int getNumRecords() { + return numRecords; } @Override @@ -66,6 +108,14 @@ public boolean hasNext() { @Override public void loadNext() throws IOException { + // Kill the task in case it has been marked as killed. This logic is from + // InterruptibleIterator, but we inline it here instead of wrapping the iterator in order + // to avoid performance overhead. This check is added here in `loadNext()` instead of in + // `hasNext()` because it's technically possible for the caller to be relying on + // `getNumRecords()` instead of `hasNext()` to know when to stop. + if (taskContext != null) { + taskContext.killTaskIfInterrupted(); + } recordLength = din.readInt(); keyPrefix = din.readLong(); if (recordLength > arr.length) { @@ -75,12 +125,7 @@ public void loadNext() throws IOException { ByteStreams.readFully(in, arr, 0, recordLength); numRecordsRemaining--; if (numRecordsRemaining == 0) { - in.close(); - if (!file.delete() && file.exists()) { - logger.warn("Unable to delete spill file {}", file.getPath()); - } - in = null; - din = null; + close(); } } @@ -103,4 +148,16 @@ public int getRecordLength() { public long getKeyPrefix() { return keyPrefix; } + + @Override + public void close() throws IOException { + if (in != null) { + try { + in.close(); + } finally { + in = null; + din = null; + } + } + } } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java index 234e21140a1d..9399024f0178 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java @@ -22,6 +22,8 @@ import scala.Tuple2; +import org.apache.spark.SparkConf; +import org.apache.spark.serializer.SerializerManager; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.serializer.DummySerializerInstance; import org.apache.spark.storage.BlockId; @@ -29,6 +31,7 @@ import org.apache.spark.storage.DiskBlockObjectWriter; import org.apache.spark.storage.TempLocalBlockId; import org.apache.spark.unsafe.Platform; +import org.apache.spark.internal.config.package$; /** * Spills a list of sorted records to disk. Spill files have the following format: @@ -37,12 +40,16 @@ */ public final class UnsafeSorterSpillWriter { - static final int DISK_WRITE_BUFFER_SIZE = 1024 * 1024; + private final SparkConf conf = new SparkConf(); + + /** The buffer size to use when writing the sorted records to an on-disk file */ + private final int diskWriteBufferSize = + (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); // Small writes to DiskBlockObjectWriter will be fairly inefficient. Since there doesn't seem to // be an API to directly transfer bytes from managed memory to the disk writer, we buffer // data through a byte array. - private byte[] writeBuffer = new byte[DISK_WRITE_BUFFER_SIZE]; + private byte[] writeBuffer = new byte[diskWriteBufferSize]; private final File file; private final BlockId blockId; @@ -72,7 +79,7 @@ public UnsafeSorterSpillWriter( } // Based on DataOutputStream.writeLong. - private void writeLongToBuffer(long v, int offset) throws IOException { + private void writeLongToBuffer(long v, int offset) { writeBuffer[offset + 0] = (byte)(v >>> 56); writeBuffer[offset + 1] = (byte)(v >>> 48); writeBuffer[offset + 2] = (byte)(v >>> 40); @@ -84,7 +91,7 @@ private void writeLongToBuffer(long v, int offset) throws IOException { } // Based on DataOutputStream.writeInt. - private void writeIntToBuffer(int v, int offset) throws IOException { + private void writeIntToBuffer(int v, int offset) { writeBuffer[offset + 0] = (byte)(v >>> 24); writeBuffer[offset + 1] = (byte)(v >>> 16); writeBuffer[offset + 2] = (byte)(v >>> 8); @@ -113,7 +120,7 @@ public void write( writeIntToBuffer(recordLength, 0); writeLongToBuffer(keyPrefix, 4); int dataRemaining = recordLength; - int freeSpaceInWriteBuffer = DISK_WRITE_BUFFER_SIZE - 4 - 8; // space used by prefix + len + int freeSpaceInWriteBuffer = diskWriteBufferSize - 4 - 8; // space used by prefix + len long recordReadPosition = baseOffset; while (dataRemaining > 0) { final int toTransfer = Math.min(freeSpaceInWriteBuffer, dataRemaining); @@ -121,21 +128,22 @@ public void write( baseObject, recordReadPosition, writeBuffer, - Platform.BYTE_ARRAY_OFFSET + (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer), + Platform.BYTE_ARRAY_OFFSET + (diskWriteBufferSize - freeSpaceInWriteBuffer), toTransfer); - writer.write(writeBuffer, 0, (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer) + toTransfer); + writer.write(writeBuffer, 0, (diskWriteBufferSize - freeSpaceInWriteBuffer) + toTransfer); recordReadPosition += toTransfer; dataRemaining -= toTransfer; - freeSpaceInWriteBuffer = DISK_WRITE_BUFFER_SIZE; + freeSpaceInWriteBuffer = diskWriteBufferSize; } - if (freeSpaceInWriteBuffer < DISK_WRITE_BUFFER_SIZE) { - writer.write(writeBuffer, 0, (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer)); + if (freeSpaceInWriteBuffer < diskWriteBufferSize) { + writer.write(writeBuffer, 0, (diskWriteBufferSize - freeSpaceInWriteBuffer)); } writer.recordWritten(); } public void close() throws IOException { - writer.commitAndClose(); + writer.commitAndGet(); + writer.close(); writer = null; writeBuffer = null; } @@ -144,7 +152,11 @@ public File getFile() { return file; } - public UnsafeSorterSpillReader getReader(BlockManager blockManager) throws IOException { - return new UnsafeSorterSpillReader(blockManager, file, blockId); + public UnsafeSorterSpillReader getReader(SerializerManager serializerManager) throws IOException { + return new UnsafeSorterSpillReader(serializerManager, file, blockId); + } + + public int recordsSpilled() { + return numRecordsSpilled; } } diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults-repl.properties b/core/src/main/resources/org/apache/spark/log4j-defaults-repl.properties deleted file mode 100644 index c85abc35b93b..000000000000 --- a/core/src/main/resources/org/apache/spark/log4j-defaults-repl.properties +++ /dev/null @@ -1,33 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Set everything to be logged to the console -log4j.rootCategory=WARN, console -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.target=System.err -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n - -# Settings to quiet third party logs that are too verbose -log4j.logger.org.spark-project.jetty=WARN -log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR -log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO -log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO - -# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support -log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL -log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties index d44cc85dcbd8..277010015072 100644 --- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -22,12 +22,21 @@ log4j.appender.console.target=System.err log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n +# Set the default spark-shell log level to WARN. When running the spark-shell, the +# log level for this class is used to overwrite the root logger's log level, so that +# the user can have different defaults for the shell and regular Spark apps. +log4j.logger.org.apache.spark.repl.Main=WARN + # Settings to quiet third party logs that are too verbose -log4j.logger.org.spark-project.jetty=WARN -log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO # SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR + +# Parquet related logging +log4j.logger.org.apache.parquet.CorruptStatistics=ERROR +log4j.logger.parquet.CorruptStatistics=ERROR diff --git a/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js b/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js index 2d9262b972a5..6fe8136c87ae 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js +++ b/core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js @@ -1,4 +1,5 @@ -/* This is a custom version of dagre-d3 on top of v0.4.3. The full list of commits can be found at http://github.com/andrewor14/dagre-d3/ */!function(e){if("object"==typeof exports&&"undefined"!=typeof module)module.exports=e();else if("function"==typeof define&&define.amd)define([],e);else{var f;"undefined"!=typeof window?f=window:"undefined"!=typeof global?f=global:"undefined"!=typeof self&&(f=self),f.dagreD3=e()}}(function(){var define,module,exports;return function e(t,n,r){function s(o,u){if(!n[o]){if(!t[o]){var a=typeof require=="function"&&require;if(!u&&a)return a(o,!0);if(i)return i(o,!0);var f=new Error("Cannot find module '"+o+"'");throw f.code="MODULE_NOT_FOUND",f}var l=n[o]={exports:{}};t[o][0].call(l.exports,function(e){var n=t[o][1][e];return s(n?n:e)},l,l.exports,e,t,n,r)}return n[o].exports}var i=typeof require=="function"&&require;for(var o=0;o0}},{}],14:[function(require,module,exports){module.exports=intersectNode;function intersectNode(node,point){return node.intersect(point)}},{}],15:[function(require,module,exports){var intersectLine=require("./intersect-line");module.exports=intersectPolygon;function intersectPolygon(node,polyPoints,point){var x1=node.x;var y1=node.y;var intersections=[];var minX=Number.POSITIVE_INFINITY,minY=Number.POSITIVE_INFINITY;polyPoints.forEach(function(entry){minX=Math.min(minX,entry.x);minY=Math.min(minY,entry.y)});var left=x1-node.width/2-minX;var top=y1-node.height/2-minY;for(var i=0;i1){intersections.sort(function(p,q){var pdx=p.x-point.x,pdy=p.y-point.y,distp=Math.sqrt(pdx*pdx+pdy*pdy),qdx=q.x-point.x,qdy=q.y-point.y,distq=Math.sqrt(qdx*qdx+qdy*qdy);return distpMath.abs(dx)*h){if(dy<0){h=-h}sx=dy===0?0:h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=dx===0?0:w*dy/dx}return{x:x+sx,y:y+sy}}},{}],17:[function(require,module,exports){var util=require("../util");module.exports=addHtmlLabel;function addHtmlLabel(root,node){var fo=root.append("foreignObject").attr("width","100000");var div=fo.append("xhtml:div");var label=node.label;switch(typeof label){case"function":div.insert(label);break;case"object":div.insert(function(){return label});break;default:div.html(label)}util.applyStyle(div,node.labelStyle);div.style("display","inline-block");div.style("white-space","nowrap");var w,h;div.each(function(){w=this.clientWidth;h=this.clientHeight});fo.attr("width",w).attr("height",h);return fo}},{"../util":25}],18:[function(require,module,exports){var addTextLabel=require("./add-text-label"),addHtmlLabel=require("./add-html-label");module.exports=addLabel;function addLabel(root,node){var label=node.label;var labelSvg=root.append("g");if(typeof label!=="string"||node.labelType==="html"){addHtmlLabel(labelSvg,node)}else{addTextLabel(labelSvg,node)}var labelBBox=labelSvg.node().getBBox();labelSvg.attr("transform","translate("+-labelBBox.width/2+","+-labelBBox.height/2+")");return labelSvg}},{"./add-html-label":17,"./add-text-label":19}],19:[function(require,module,exports){var util=require("../util");module.exports=addTextLabel;function addTextLabel(root,node){var domNode=root.append("text");var lines=processEscapeSequences(node.label).split("\n");for(var i=0;imaxPadding){maxPadding=child.paddingTop}}return maxPadding}function getRank(g,v){var maxRank=0;var children=g.children(v);for(var i=0;imaxRank){maxRank=thisRank}}return maxRank}function orderByRank(g,nodes){return nodes.sort(function(x,y){return getRank(g,x)-getRank(g,y)})}function edgeToId(e){return escapeId(e.v)+":"+escapeId(e.w)+":"+escapeId(e.name)}var ID_DELIM=/:/g;function escapeId(str){return str?String(str).replace(ID_DELIM,"\\:"):""}function applyStyle(dom,styleFn){if(styleFn){dom.attr("style",styleFn)}}function applyClass(dom,classFn,otherClasses){if(classFn){dom.attr("class",classFn).attr("class",otherClasses+" "+dom.attr("class"))}}function applyTransition(selection,g){var graph=g.graph();if(_.isPlainObject(graph)){var transition=graph.transition;if(_.isFunction(transition)){return transition(selection)}}return selection}},{"./lodash":20}],26:[function(require,module,exports){module.exports="0.4.4-pre"},{}],27:[function(require,module,exports){module.exports={graphlib:require("./lib/graphlib"),layout:require("./lib/layout"),debug:require("./lib/debug"),util:{time:require("./lib/util").time,notime:require("./lib/util").notime},version:require("./lib/version")}},{"./lib/debug":32,"./lib/graphlib":33,"./lib/layout":35,"./lib/util":55,"./lib/version":56}],28:[function(require,module,exports){"use strict";var _=require("./lodash"),greedyFAS=require("./greedy-fas");module.exports={run:run,undo:undo};function run(g){var fas=g.graph().acyclicer==="greedy"?greedyFAS(g,weightFn(g)):dfsFAS(g);_.each(fas,function(e){var label=g.edge(e);g.removeEdge(e);label.forwardName=e.name;label.reversed=true;g.setEdge(e.w,e.v,label,_.uniqueId("rev"))});function weightFn(g){return function(e){return g.edge(e).weight}}}function dfsFAS(g){var fas=[],stack={},visited={};function dfs(v){if(_.has(visited,v)){return}visited[v]=true;stack[v]=true;_.each(g.outEdges(v),function(e){if(_.has(stack,e.w)){fas.push(e)}else{dfs(e.w)}});delete stack[v]}_.each(g.nodes(),dfs);return fas}function undo(g){_.each(g.edges(),function(e){var label=g.edge(e);if(label.reversed){g.removeEdge(e);var forwardName=label.forwardName;delete label.reversed;delete label.forwardName;g.setEdge(e.w,e.v,label,forwardName)}})}},{"./greedy-fas":34,"./lodash":36}],29:[function(require,module,exports){var _=require("./lodash"),util=require("./util");module.exports=addBorderSegments;function addBorderSegments(g){function dfs(v){var children=g.children(v),node=g.node(v);if(children.length){_.each(children,dfs)}if(_.has(node,"minRank")){node.borderLeft=[];node.borderRight=[];for(var rank=node.minRank,maxRank=node.maxRank+1;rank0;--i){entry=buckets[i].dequeue();if(entry){results=results.concat(removeNode(g,buckets,zeroIdx,entry,true));break}}}}return results}function removeNode(g,buckets,zeroIdx,entry,collectPredecessors){var results=collectPredecessors?[]:undefined;_.each(g.inEdges(entry.v),function(edge){var weight=g.edge(edge),uEntry=g.node(edge.v);if(collectPredecessors){results.push({v:edge.v,w:edge.w})}uEntry.out-=weight;assignBucket(buckets,zeroIdx,uEntry)});_.each(g.outEdges(entry.v),function(edge){var weight=g.edge(edge),w=edge.w,wEntry=g.node(w);wEntry["in"]-=weight;assignBucket(buckets,zeroIdx,wEntry)});g.removeNode(entry.v);return results}function buildState(g,weightFn){var fasGraph=new Graph,maxIn=0,maxOut=0;_.each(g.nodes(),function(v){fasGraph.setNode(v,{v:v,"in":0,out:0})});_.each(g.edges(),function(e){var prevWeight=fasGraph.edge(e.v,e.w)||0,weight=weightFn(e),edgeWeight=prevWeight+weight;fasGraph.setEdge(e.v,e.w,edgeWeight);maxOut=Math.max(maxOut,fasGraph.node(e.v).out+=weight);maxIn=Math.max(maxIn,fasGraph.node(e.w)["in"]+=weight)});var buckets=_.range(maxOut+maxIn+3).map(function(){return new List});var zeroIdx=maxIn+1;_.each(fasGraph.nodes(),function(v){assignBucket(buckets,zeroIdx,fasGraph.node(v))});return{graph:fasGraph,buckets:buckets,zeroIdx:zeroIdx}}function assignBucket(buckets,zeroIdx,entry){if(!entry.out){buckets[0].enqueue(entry)}else if(!entry["in"]){buckets[buckets.length-1].enqueue(entry)}else{buckets[entry.out-entry["in"]+zeroIdx].enqueue(entry)}}},{"./data/list":31,"./graphlib":33,"./lodash":36}],35:[function(require,module,exports){"use strict";var _=require("./lodash"),acyclic=require("./acyclic"),normalize=require("./normalize"),rank=require("./rank"),normalizeRanks=require("./util").normalizeRanks,parentDummyChains=require("./parent-dummy-chains"),removeEmptyRanks=require("./util").removeEmptyRanks,nestingGraph=require("./nesting-graph"),addBorderSegments=require("./add-border-segments"),coordinateSystem=require("./coordinate-system"),order=require("./order"),position=require("./position"),util=require("./util"),Graph=require("./graphlib").Graph;module.exports=layout;function layout(g,opts){var time=opts&&opts.debugTiming?util.time:util.notime;time("layout",function(){var layoutGraph=time(" buildLayoutGraph",function(){return buildLayoutGraph(g)});time(" runLayout",function(){runLayout(layoutGraph,time)});time(" updateInputGraph",function(){updateInputGraph(g,layoutGraph)})})}function runLayout(g,time){time(" makeSpaceForEdgeLabels",function(){makeSpaceForEdgeLabels(g)});time(" removeSelfEdges",function(){removeSelfEdges(g)});time(" acyclic",function(){acyclic.run(g)});time(" nestingGraph.run",function(){nestingGraph.run(g)});time(" rank",function(){rank(util.asNonCompoundGraph(g))});time(" injectEdgeLabelProxies",function(){injectEdgeLabelProxies(g)});time(" removeEmptyRanks",function(){removeEmptyRanks(g)});time(" nestingGraph.cleanup",function(){nestingGraph.cleanup(g)});time(" normalizeRanks",function(){normalizeRanks(g)});time(" assignRankMinMax",function(){assignRankMinMax(g)});time(" removeEdgeLabelProxies",function(){removeEdgeLabelProxies(g)});time(" normalize.run",function(){normalize.run(g)});time(" parentDummyChains",function(){ -parentDummyChains(g)});time(" addBorderSegments",function(){addBorderSegments(g)});time(" order",function(){order(g)});time(" insertSelfEdges",function(){insertSelfEdges(g)});time(" adjustCoordinateSystem",function(){coordinateSystem.adjust(g)});time(" position",function(){position(g)});time(" positionSelfEdges",function(){positionSelfEdges(g)});time(" removeBorderNodes",function(){removeBorderNodes(g)});time(" normalize.undo",function(){normalize.undo(g)});time(" fixupEdgeLabelCoords",function(){fixupEdgeLabelCoords(g)});time(" undoCoordinateSystem",function(){coordinateSystem.undo(g)});time(" translateGraph",function(){translateGraph(g)});time(" assignNodeIntersects",function(){assignNodeIntersects(g)});time(" reversePoints",function(){reversePointsForReversedEdges(g)});time(" acyclic.undo",function(){acyclic.undo(g)})}function updateInputGraph(inputGraph,layoutGraph){_.each(inputGraph.nodes(),function(v){var inputLabel=inputGraph.node(v),layoutLabel=layoutGraph.node(v);if(inputLabel){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y;if(layoutGraph.children(v).length){inputLabel.width=layoutLabel.width;inputLabel.height=layoutLabel.height}}});_.each(inputGraph.edges(),function(e){var inputLabel=inputGraph.edge(e),layoutLabel=layoutGraph.edge(e);inputLabel.points=layoutLabel.points;if(_.has(layoutLabel,"x")){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y}});inputGraph.graph().width=layoutGraph.graph().width;inputGraph.graph().height=layoutGraph.graph().height}var graphNumAttrs=["nodesep","edgesep","ranksep","marginx","marginy"],graphDefaults={ranksep:50,edgesep:20,nodesep:50,rankdir:"tb"},graphAttrs=["acyclicer","ranker","rankdir","align"],nodeNumAttrs=["width","height"],nodeDefaults={width:0,height:0},edgeNumAttrs=["minlen","weight","width","height","labeloffset"],edgeDefaults={minlen:1,weight:1,width:0,height:0,labeloffset:10,labelpos:"r"},edgeAttrs=["labelpos"];function buildLayoutGraph(inputGraph){var g=new Graph({multigraph:true,compound:true}),graph=canonicalize(inputGraph.graph());g.setGraph(_.merge({},graphDefaults,selectNumberAttrs(graph,graphNumAttrs),_.pick(graph,graphAttrs)));_.each(inputGraph.nodes(),function(v){var node=canonicalize(inputGraph.node(v));g.setNode(v,_.defaults(selectNumberAttrs(node,nodeNumAttrs),nodeDefaults));g.setParent(v,inputGraph.parent(v))});_.each(inputGraph.edges(),function(e){var edge=canonicalize(inputGraph.edge(e));g.setEdge(e,_.merge({},edgeDefaults,selectNumberAttrs(edge,edgeNumAttrs),_.pick(edge,edgeAttrs)))});return g}function makeSpaceForEdgeLabels(g){var graph=g.graph();graph.ranksep/=2;_.each(g.edges(),function(e){var edge=g.edge(e);edge.minlen*=2;if(edge.labelpos.toLowerCase()!=="c"){if(graph.rankdir==="TB"||graph.rankdir==="BT"){edge.width+=edge.labeloffset}else{edge.height+=edge.labeloffset}}})}function injectEdgeLabelProxies(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.width&&edge.height){var v=g.node(e.v),w=g.node(e.w),label={rank:(w.rank-v.rank)/2+v.rank,e:e};util.addDummyNode(g,"edge-proxy",label,"_ep")}})}function assignRankMinMax(g){var maxRank=0;_.each(g.nodes(),function(v){var node=g.node(v);if(node.borderTop){node.minRank=g.node(node.borderTop).rank;node.maxRank=g.node(node.borderBottom).rank;maxRank=_.max(maxRank,node.maxRank)}});g.graph().maxRank=maxRank}function removeEdgeLabelProxies(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="edge-proxy"){g.edge(node.e).labelRank=node.rank;g.removeNode(v)}})}function translateGraph(g){var minX=Number.POSITIVE_INFINITY,maxX=0,minY=Number.POSITIVE_INFINITY,maxY=0,graphLabel=g.graph(),marginX=graphLabel.marginx||0,marginY=graphLabel.marginy||0;function getExtremes(attrs){var x=attrs.x,y=attrs.y,w=attrs.width,h=attrs.height;minX=Math.min(minX,x-w/2);maxX=Math.max(maxX,x+w/2);minY=Math.min(minY,y-h/2);maxY=Math.max(maxY,y+h/2)}_.each(g.nodes(),function(v){getExtremes(g.node(v))});_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){getExtremes(edge)}});minX-=marginX;minY-=marginY;_.each(g.nodes(),function(v){var node=g.node(v);node.x-=minX;node.y-=minY});_.each(g.edges(),function(e){var edge=g.edge(e);_.each(edge.points,function(p){p.x-=minX;p.y-=minY});if(_.has(edge,"x")){edge.x-=minX}if(_.has(edge,"y")){edge.y-=minY}});graphLabel.width=maxX-minX+marginX;graphLabel.height=maxY-minY+marginY}function assignNodeIntersects(g){_.each(g.edges(),function(e){var edge=g.edge(e),nodeV=g.node(e.v),nodeW=g.node(e.w),p1,p2;if(!edge.points){edge.points=[];p1=nodeW;p2=nodeV}else{p1=edge.points[0];p2=edge.points[edge.points.length-1]}edge.points.unshift(util.intersectRect(nodeV,p1));edge.points.push(util.intersectRect(nodeW,p2))})}function fixupEdgeLabelCoords(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){if(edge.labelpos==="l"||edge.labelpos==="r"){edge.width-=edge.labeloffset}switch(edge.labelpos){case"l":edge.x-=edge.width/2+edge.labeloffset;break;case"r":edge.x+=edge.width/2+edge.labeloffset;break}}})}function reversePointsForReversedEdges(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.reversed){edge.points.reverse()}})}function removeBorderNodes(g){_.each(g.nodes(),function(v){if(g.children(v).length){var node=g.node(v),t=g.node(node.borderTop),b=g.node(node.borderBottom),l=g.node(_.last(node.borderLeft)),r=g.node(_.last(node.borderRight));node.width=Math.abs(r.x-l.x);node.height=Math.abs(b.y-t.y);node.x=l.x+node.width/2;node.y=t.y+node.height/2}});_.each(g.nodes(),function(v){if(g.node(v).dummy==="border"){g.removeNode(v)}})}function removeSelfEdges(g){_.each(g.edges(),function(e){if(e.v===e.w){var node=g.node(e.v);if(!node.selfEdges){node.selfEdges=[]}node.selfEdges.push({e:e,label:g.edge(e)});g.removeEdge(e)}})}function insertSelfEdges(g){var layers=util.buildLayerMatrix(g);_.each(layers,function(layer){var orderShift=0;_.each(layer,function(v,i){var node=g.node(v);node.order=i+orderShift;_.each(node.selfEdges,function(selfEdge){util.addDummyNode(g,"selfedge",{width:selfEdge.label.width,height:selfEdge.label.height,rank:node.rank,order:i+ ++orderShift,e:selfEdge.e,label:selfEdge.label},"_se")});delete node.selfEdges})})}function positionSelfEdges(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="selfedge"){var selfNode=g.node(node.e.v),x=selfNode.x+selfNode.width/2,y=selfNode.y,dx=node.x-x,dy=selfNode.height/2;g.setEdge(node.e,node.label);g.removeNode(v);node.label.points=[{x:x+2*dx/3,y:y-dy},{x:x+5*dx/6,y:y-dy},{x:x+dx,y:y},{x:x+5*dx/6,y:y+dy},{x:x+2*dx/3,y:y+dy}];node.label.x=node.x;node.label.y=node.y}})}function selectNumberAttrs(obj,attrs){return _.mapValues(_.pick(obj,attrs),Number)}function canonicalize(attrs){var newAttrs={};_.each(attrs,function(v,k){newAttrs[k.toLowerCase()]=v});return newAttrs}},{"./acyclic":28,"./add-border-segments":29,"./coordinate-system":30,"./graphlib":33,"./lodash":36,"./nesting-graph":37,"./normalize":38,"./order":43,"./parent-dummy-chains":48,"./position":50,"./rank":52,"./util":55}],36:[function(require,module,exports){module.exports=require(20)},{"/Users/andrew/Documents/dev/dagre-d3/lib/lodash.js":20,lodash:77}],37:[function(require,module,exports){var _=require("./lodash"),util=require("./util");module.exports={run:run,cleanup:cleanup};function run(g){var root=util.addDummyNode(g,"root",{},"_root"),depths=treeDepths(g),height=_.max(depths)-1,nodeSep=2*height+1;g.graph().nestingRoot=root;_.each(g.edges(),function(e){g.edge(e).minlen*=nodeSep});var weight=sumWeights(g)+1;_.each(g.children(),function(child){dfs(g,root,nodeSep,weight,height,depths,child)});g.graph().nodeRankFactor=nodeSep}function dfs(g,root,nodeSep,weight,height,depths,v){var children=g.children(v);if(!children.length){if(v!==root){g.setEdge(root,v,{weight:0,minlen:nodeSep})}return}var top=util.addBorderNode(g,"_bt"),bottom=util.addBorderNode(g,"_bb"),label=g.node(v);g.setParent(top,v);label.borderTop=top;g.setParent(bottom,v);label.borderBottom=bottom;_.each(children,function(child){dfs(g,root,nodeSep,weight,height,depths,child);var childNode=g.node(child),childTop=childNode.borderTop?childNode.borderTop:child,childBottom=childNode.borderBottom?childNode.borderBottom:child,thisWeight=childNode.borderTop?weight:2*weight,minlen=childTop!==childBottom?1:height-depths[v]+1;g.setEdge(top,childTop,{weight:thisWeight,minlen:minlen,nestingEdge:true});g.setEdge(childBottom,bottom,{weight:thisWeight,minlen:minlen,nestingEdge:true})});if(!g.parent(v)){g.setEdge(root,top,{weight:0,minlen:height+depths[v]})}}function treeDepths(g){var depths={};function dfs(v,depth){var children=g.children(v);if(children&&children.length){_.each(children,function(child){dfs(child,depth+1)})}depths[v]=depth}_.each(g.children(),function(v){dfs(v,1)});return depths}function sumWeights(g){return _.reduce(g.edges(),function(acc,e){return acc+g.edge(e).weight},0)}function cleanup(g){var graphLabel=g.graph();g.removeNode(graphLabel.nestingRoot);delete graphLabel.nestingRoot;_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.nestingEdge){g.removeEdge(e)}})}},{"./lodash":36,"./util":55}],38:[function(require,module,exports){"use strict";var _=require("./lodash"),util=require("./util");module.exports={run:run,undo:undo};function run(g){g.graph().dummyChains=[];_.each(g.edges(),function(edge){normalizeEdge(g,edge)})}function normalizeEdge(g,e){var v=e.v,vRank=g.node(v).rank,w=e.w,wRank=g.node(w).rank,name=e.name,edgeLabel=g.edge(e),labelRank=edgeLabel.labelRank;if(wRank===vRank+1)return;g.removeEdge(e);var dummy,attrs,i;for(i=0,++vRank;vRank0){if(index%2){weightSum+=tree[index+1]}index=index-1>>1;tree[index]+=entry.weight}cc+=entry.weight*weightSum}));return cc}},{"../lodash":36}],43:[function(require,module,exports){"use strict";var _=require("../lodash"),initOrder=require("./init-order"),crossCount=require("./cross-count"),sortSubgraph=require("./sort-subgraph"),buildLayerGraph=require("./build-layer-graph"),addSubgraphConstraints=require("./add-subgraph-constraints"),Graph=require("../graphlib").Graph,util=require("../util");module.exports=order;function order(g){var maxRank=util.maxRank(g),downLayerGraphs=buildLayerGraphs(g,_.range(1,maxRank+1),"inEdges"),upLayerGraphs=buildLayerGraphs(g,_.range(maxRank-1,-1,-1),"outEdges");var layering=initOrder(g);assignOrder(g,layering);var bestCC=Number.POSITIVE_INFINITY,best;for(var i=0,lastBest=0;lastBest<4;++i,++lastBest){sweepLayerGraphs(i%2?downLayerGraphs:upLayerGraphs,i%4>=2);layering=util.buildLayerMatrix(g);var cc=crossCount(g,layering);if(cc=vEntry.barycenter){mergeEntries(vEntry,uEntry)}}}function handleOut(vEntry){return function(wEntry){wEntry["in"].push(vEntry);if(--wEntry.indegree===0){sourceSet.push(wEntry)}}}while(sourceSet.length){var entry=sourceSet.pop();entries.push(entry);_.each(entry["in"].reverse(),handleIn(entry));_.each(entry.out,handleOut(entry))}return _.chain(entries).filter(function(entry){return!entry.merged}).map(function(entry){return _.pick(entry,["vs","i","barycenter","weight"])}).value()}function mergeEntries(target,source){var sum=0,weight=0;if(target.weight){sum+=target.barycenter*target.weight;weight+=target.weight}if(source.weight){sum+=source.barycenter*source.weight;weight+=source.weight}target.vs=source.vs.concat(target.vs);target.barycenter=sum/weight;target.weight=weight;target.i=Math.min(source.i,target.i);source.merged=true}},{"../lodash":36}],46:[function(require,module,exports){var _=require("../lodash"),barycenter=require("./barycenter"),resolveConflicts=require("./resolve-conflicts"),sort=require("./sort");module.exports=sortSubgraph;function sortSubgraph(g,v,cg,biasRight){var movable=g.children(v),node=g.node(v),bl=node?node.borderLeft:undefined,br=node?node.borderRight:undefined,subgraphs={};if(bl){movable=_.filter(movable,function(w){return w!==bl&&w!==br})}var barycenters=barycenter(g,movable);_.each(barycenters,function(entry){if(g.children(entry.v).length){var subgraphResult=sortSubgraph(g,entry.v,cg,biasRight);subgraphs[entry.v]=subgraphResult;if(_.has(subgraphResult,"barycenter")){mergeBarycenters(entry,subgraphResult)}}});var entries=resolveConflicts(barycenters,cg);expandSubgraphs(entries,subgraphs);var result=sort(entries,biasRight);if(bl){result.vs=_.flatten([bl,result.vs,br],true);if(g.predecessors(bl).length){var blPred=g.node(g.predecessors(bl)[0]),brPred=g.node(g.predecessors(br)[0]);if(!_.has(result,"barycenter")){result.barycenter=0;result.weight=0}result.barycenter=(result.barycenter*result.weight+blPred.order+brPred.order)/(result.weight+2);result.weight+=2}}return result}function expandSubgraphs(entries,subgraphs){_.each(entries,function(entry){entry.vs=_.flatten(entry.vs.map(function(v){if(subgraphs[v]){return subgraphs[v].vs}return v}),true)})}function mergeBarycenters(target,other){if(!_.isUndefined(target.barycenter)){target.barycenter=(target.barycenter*target.weight+other.barycenter*other.weight)/(target.weight+other.weight);target.weight+=other.weight}else{target.barycenter=other.barycenter;target.weight=other.weight}}},{"../lodash":36,"./barycenter":40,"./resolve-conflicts":45,"./sort":47}],47:[function(require,module,exports){var _=require("../lodash"),util=require("../util");module.exports=sort;function sort(entries,biasRight){var parts=util.partition(entries,function(entry){return _.has(entry,"barycenter")});var sortable=parts.lhs,unsortable=_.sortBy(parts.rhs,function(entry){return-entry.i}),vs=[],sum=0,weight=0,vsIndex=0;sortable.sort(compareWithBias(!!biasRight));vsIndex=consumeUnsortable(vs,unsortable,vsIndex);_.each(sortable,function(entry){vsIndex+=entry.vs.length;vs.push(entry.vs);sum+=entry.barycenter*entry.weight;weight+=entry.weight;vsIndex=consumeUnsortable(vs,unsortable,vsIndex)});var result={vs:_.flatten(vs,true)};if(weight){result.barycenter=sum/weight;result.weight=weight}return result}function consumeUnsortable(vs,unsortable,index){var last;while(unsortable.length&&(last=_.last(unsortable)).i<=index){unsortable.pop();vs.push(last.vs);index++}return index}function compareWithBias(bias){return function(entryV,entryW){if(entryV.barycenterentryW.barycenter){return 1}return!bias?entryV.i-entryW.i:entryW.i-entryV.i}}},{"../lodash":36,"../util":55}],48:[function(require,module,exports){var _=require("./lodash");module.exports=parentDummyChains;function parentDummyChains(g){var postorderNums=postorder(g);_.each(g.graph().dummyChains,function(v){var node=g.node(v),edgeObj=node.edgeObj,pathData=findPath(g,postorderNums,edgeObj.v,edgeObj.w),path=pathData.path,lca=pathData.lca,pathIdx=0,pathV=path[pathIdx],ascending=true;while(v!==edgeObj.w){node=g.node(v);if(ascending){while((pathV=path[pathIdx])!==lca&&g.node(pathV).maxRanklow||lim>postorderNums[parent].lim));lca=parent;parent=w;while((parent=g.parent(parent))!==lca){wPath.push(parent)}return{path:vPath.concat(wPath.reverse()),lca:lca}}function postorder(g){var result={},lim=0;function dfs(v){var low=lim;_.each(g.children(v),dfs);result[v]={low:low,lim:lim++}}_.each(g.children(),dfs);return result}},{"./lodash":36}],49:[function(require,module,exports){"use strict";var _=require("../lodash"),Graph=require("../graphlib").Graph,util=require("../util");module.exports={positionX:positionX,findType1Conflicts:findType1Conflicts,findType2Conflicts:findType2Conflicts,addConflict:addConflict,hasConflict:hasConflict,verticalAlignment:verticalAlignment,horizontalCompaction:horizontalCompaction,alignCoordinates:alignCoordinates,findSmallestWidthAlignment:findSmallestWidthAlignment,balance:balance};function findType1Conflicts(g,layering){var conflicts={};function visitLayer(prevLayer,layer){var k0=0,scanPos=0,prevLayerLength=prevLayer.length,lastNode=_.last(layer);_.each(layer,function(v,i){var w=findOtherInnerSegmentNode(g,v),k1=w?g.node(w).order:prevLayerLength;if(w||v===lastNode){_.each(layer.slice(scanPos,i+1),function(scanNode){_.each(g.predecessors(scanNode),function(u){var uLabel=g.node(u),uPos=uLabel.order;if((uPosnextNorthBorder)){addConflict(conflicts,u,v)}})}})}function visitLayer(north,south){var prevNorthPos=-1,nextNorthPos,southPos=0;_.each(south,function(v,southLookahead){if(g.node(v).dummy==="border"){var predecessors=g.predecessors(v);if(predecessors.length){nextNorthPos=g.node(predecessors[0]).order;scan(south,southPos,southLookahead,prevNorthPos,nextNorthPos);southPos=southLookahead;prevNorthPos=nextNorthPos}}scan(south,southPos,south.length,nextNorthPos,north.length)});return south}_.reduce(layering,visitLayer);return conflicts}function findOtherInnerSegmentNode(g,v){if(g.node(v).dummy){return _.find(g.predecessors(v),function(u){return g.node(u).dummy})}}function addConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}var conflictsV=conflicts[v];if(!conflictsV){conflicts[v]=conflictsV={}}conflictsV[w]=true}function hasConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}return _.has(conflicts[v],w)}function verticalAlignment(g,layering,conflicts,neighborFn){var root={},align={},pos={};_.each(layering,function(layer){_.each(layer,function(v,order){root[v]=v;align[v]=v;pos[v]=order})});_.each(layering,function(layer){var prevIdx=-1;_.each(layer,function(v){var ws=neighborFn(v);if(ws.length){ws=_.sortBy(ws,function(w){return pos[w]});var mp=(ws.length-1)/2;for(var i=Math.floor(mp),il=Math.ceil(mp);i<=il;++i){var w=ws[i];if(align[v]===v&&prevIdx0}},{}],14:[function(require,module,exports){module.exports=intersectNode;function intersectNode(node,point){return node.intersect(point)}},{}],15:[function(require,module,exports){var intersectLine=require("./intersect-line");module.exports=intersectPolygon;function intersectPolygon(node,polyPoints,point){var x1=node.x;var y1=node.y;var intersections=[];var minX=Number.POSITIVE_INFINITY,minY=Number.POSITIVE_INFINITY;polyPoints.forEach(function(entry){minX=Math.min(minX,entry.x);minY=Math.min(minY,entry.y)});var left=x1-node.width/2-minX;var top=y1-node.height/2-minY;for(var i=0;i1){intersections.sort(function(p,q){var pdx=p.x-point.x,pdy=p.y-point.y,distp=Math.sqrt(pdx*pdx+pdy*pdy),qdx=q.x-point.x,qdy=q.y-point.y,distq=Math.sqrt(qdx*qdx+qdy*qdy);return distpMath.abs(dx)*h){if(dy<0){h=-h}sx=dy===0?0:h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=dx===0?0:w*dy/dx}return{x:x+sx,y:y+sy}}},{}],17:[function(require,module,exports){var util=require("../util");module.exports=addHtmlLabel;function addHtmlLabel(root,node){var fo=root.append("foreignObject").attr("width","100000");var div=fo.append("xhtml:div");var label=node.label;switch(typeof label){case"function":div.insert(label);break;case"object":div.insert(function(){return label});break;default:div.html(label)}util.applyStyle(div,node.labelStyle);div.style("display","inline-block");div.style("white-space","nowrap");var w,h;div.each(function(){w=this.clientWidth;h=this.clientHeight});fo.attr("width",w).attr("height",h);return fo}},{"../util":25}],18:[function(require,module,exports){var addTextLabel=require("./add-text-label"),addHtmlLabel=require("./add-html-label");module.exports=addLabel;function addLabel(root,node){var label=node.label;var labelSvg=root.append("g");if(typeof label!=="string"||node.labelType==="html"){addHtmlLabel(labelSvg,node)}else{addTextLabel(labelSvg,node)}var labelBBox=labelSvg.node().getBBox();labelSvg.attr("transform","translate("+-labelBBox.width/2+","+-labelBBox.height/2+")");return labelSvg}},{"./add-html-label":17,"./add-text-label":19}],19:[function(require,module,exports){var util=require("../util");module.exports=addTextLabel;function addTextLabel(root,node){var domNode=root.append("text");var lines=processEscapeSequences(node.label).split("\n");for(var i=0;imaxPadding){maxPadding=child.paddingTop}}return maxPadding}function getRank(g,v){var maxRank=0;var children=g.children(v);for(var i=0;imaxRank){maxRank=thisRank}}return maxRank}function orderByRank(g,nodes){return nodes.sort(function(x,y){return getRank(g,x)-getRank(g,y)})}function edgeToId(e){return escapeId(e.v)+":"+escapeId(e.w)+":"+escapeId(e.name)}var ID_DELIM=/:/g;function escapeId(str){return str?String(str).replace(ID_DELIM,"\\:"):""}function applyStyle(dom,styleFn){if(styleFn){dom.attr("style",styleFn)}}function applyClass(dom,classFn,otherClasses){if(classFn){dom.attr("class",classFn).attr("class",otherClasses+" "+dom.attr("class"))}}function applyTransition(selection,g){var graph=g.graph();if(_.isPlainObject(graph)){var transition=graph.transition;if(_.isFunction(transition)){return transition(selection)}}return selection}},{"./lodash":20}],26:[function(require,module,exports){module.exports="0.4.4-pre"},{}],27:[function(require,module,exports){module.exports={graphlib:require("./lib/graphlib"),layout:require("./lib/layout"),debug:require("./lib/debug"),util:{time:require("./lib/util").time,notime:require("./lib/util").notime},version:require("./lib/version")}},{"./lib/debug":32,"./lib/graphlib":33,"./lib/layout":35,"./lib/util":55,"./lib/version":56}],28:[function(require,module,exports){"use strict";var _=require("./lodash"),greedyFAS=require("./greedy-fas");module.exports={run:run,undo:undo};function run(g){var fas=g.graph().acyclicer==="greedy"?greedyFAS(g,weightFn(g)):dfsFAS(g);_.each(fas,function(e){var label=g.edge(e);g.removeEdge(e);label.forwardName=e.name;label.reversed=true;g.setEdge(e.w,e.v,label,_.uniqueId("rev"))});function weightFn(g){return function(e){return g.edge(e).weight}}}function dfsFAS(g){var fas=[],stack={},visited={};function dfs(v){if(_.has(visited,v)){return}visited[v]=true;stack[v]=true;_.each(g.outEdges(v),function(e){if(_.has(stack,e.w)){fas.push(e)}else{dfs(e.w)}});delete stack[v]}_.each(g.nodes(),dfs);return fas}function undo(g){_.each(g.edges(),function(e){var label=g.edge(e);if(label.reversed){g.removeEdge(e);var forwardName=label.forwardName;delete label.reversed;delete label.forwardName;g.setEdge(e.w,e.v,label,forwardName)}})}},{"./greedy-fas":34,"./lodash":36}],29:[function(require,module,exports){var _=require("./lodash"),util=require("./util");module.exports=addBorderSegments;function addBorderSegments(g){function dfs(v){var children=g.children(v),node=g.node(v);if(children.length){_.each(children,dfs)}if(_.has(node,"minRank")){node.borderLeft=[];node.borderRight=[];for(var rank=node.minRank,maxRank=node.maxRank+1;rank0;--i){entry=buckets[i].dequeue();if(entry){results=results.concat(removeNode(g,buckets,zeroIdx,entry,true));break}}}}return results}function removeNode(g,buckets,zeroIdx,entry,collectPredecessors){var results=collectPredecessors?[]:undefined;_.each(g.inEdges(entry.v),function(edge){var weight=g.edge(edge),uEntry=g.node(edge.v);if(collectPredecessors){results.push({v:edge.v,w:edge.w})}uEntry.out-=weight;assignBucket(buckets,zeroIdx,uEntry)});_.each(g.outEdges(entry.v),function(edge){var weight=g.edge(edge),w=edge.w,wEntry=g.node(w);wEntry["in"]-=weight;assignBucket(buckets,zeroIdx,wEntry)});g.removeNode(entry.v);return results}function buildState(g,weightFn){var fasGraph=new Graph,maxIn=0,maxOut=0;_.each(g.nodes(),function(v){fasGraph.setNode(v,{v:v,"in":0,out:0})});_.each(g.edges(),function(e){var prevWeight=fasGraph.edge(e.v,e.w)||0,weight=weightFn(e),edgeWeight=prevWeight+weight;fasGraph.setEdge(e.v,e.w,edgeWeight);maxOut=Math.max(maxOut,fasGraph.node(e.v).out+=weight);maxIn=Math.max(maxIn,fasGraph.node(e.w)["in"]+=weight)});var buckets=_.range(maxOut+maxIn+3).map(function(){return new List});var zeroIdx=maxIn+1;_.each(fasGraph.nodes(),function(v){assignBucket(buckets,zeroIdx,fasGraph.node(v))});return{graph:fasGraph,buckets:buckets,zeroIdx:zeroIdx}}function assignBucket(buckets,zeroIdx,entry){if(!entry.out){buckets[0].enqueue(entry)}else if(!entry["in"]){buckets[buckets.length-1].enqueue(entry)}else{buckets[entry.out-entry["in"]+zeroIdx].enqueue(entry)}}},{"./data/list":31,"./graphlib":33,"./lodash":36}],35:[function(require,module,exports){"use strict";var _=require("./lodash"),acyclic=require("./acyclic"),normalize=require("./normalize"),rank=require("./rank"),normalizeRanks=require("./util").normalizeRanks,parentDummyChains=require("./parent-dummy-chains"),removeEmptyRanks=require("./util").removeEmptyRanks,nestingGraph=require("./nesting-graph"),addBorderSegments=require("./add-border-segments"),coordinateSystem=require("./coordinate-system"),order=require("./order"),position=require("./position"),util=require("./util"),Graph=require("./graphlib").Graph;module.exports=layout;function layout(g,opts){var time=opts&&opts.debugTiming?util.time:util.notime;time("layout",function(){var layoutGraph=time(" buildLayoutGraph",function(){return buildLayoutGraph(g)});time(" runLayout",function(){runLayout(layoutGraph,time)});time(" updateInputGraph",function(){updateInputGraph(g,layoutGraph)})})}function runLayout(g,time){time(" makeSpaceForEdgeLabels",function(){makeSpaceForEdgeLabels(g)});time(" removeSelfEdges",function(){removeSelfEdges(g)});time(" acyclic",function(){acyclic.run(g)});time(" nestingGraph.run",function(){nestingGraph.run(g)});time(" rank",function(){rank(util.asNonCompoundGraph(g))});time(" injectEdgeLabelProxies",function(){injectEdgeLabelProxies(g)});time(" removeEmptyRanks",function(){removeEmptyRanks(g)});time(" nestingGraph.cleanup",function(){nestingGraph.cleanup(g)});time(" normalizeRanks",function(){normalizeRanks(g)});time(" assignRankMinMax",function(){assignRankMinMax(g)});time(" removeEdgeLabelProxies",function(){removeEdgeLabelProxies(g)});time(" normalize.run",function(){ +normalize.run(g)});time(" parentDummyChains",function(){parentDummyChains(g)});time(" addBorderSegments",function(){addBorderSegments(g)});time(" order",function(){order(g)});time(" insertSelfEdges",function(){insertSelfEdges(g)});time(" adjustCoordinateSystem",function(){coordinateSystem.adjust(g)});time(" position",function(){position(g)});time(" positionSelfEdges",function(){positionSelfEdges(g)});time(" removeBorderNodes",function(){removeBorderNodes(g)});time(" normalize.undo",function(){normalize.undo(g)});time(" fixupEdgeLabelCoords",function(){fixupEdgeLabelCoords(g)});time(" undoCoordinateSystem",function(){coordinateSystem.undo(g)});time(" translateGraph",function(){translateGraph(g)});time(" assignNodeIntersects",function(){assignNodeIntersects(g)});time(" reversePoints",function(){reversePointsForReversedEdges(g)});time(" acyclic.undo",function(){acyclic.undo(g)})}function updateInputGraph(inputGraph,layoutGraph){_.each(inputGraph.nodes(),function(v){var inputLabel=inputGraph.node(v),layoutLabel=layoutGraph.node(v);if(inputLabel){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y;if(layoutGraph.children(v).length){inputLabel.width=layoutLabel.width;inputLabel.height=layoutLabel.height}}});_.each(inputGraph.edges(),function(e){var inputLabel=inputGraph.edge(e),layoutLabel=layoutGraph.edge(e);inputLabel.points=layoutLabel.points;if(_.has(layoutLabel,"x")){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y}});inputGraph.graph().width=layoutGraph.graph().width;inputGraph.graph().height=layoutGraph.graph().height}var graphNumAttrs=["nodesep","edgesep","ranksep","marginx","marginy"],graphDefaults={ranksep:50,edgesep:20,nodesep:50,rankdir:"tb"},graphAttrs=["acyclicer","ranker","rankdir","align"],nodeNumAttrs=["width","height"],nodeDefaults={width:0,height:0},edgeNumAttrs=["minlen","weight","width","height","labeloffset"],edgeDefaults={minlen:1,weight:1,width:0,height:0,labeloffset:10,labelpos:"r"},edgeAttrs=["labelpos"];function buildLayoutGraph(inputGraph){var g=new Graph({multigraph:true,compound:true}),graph=canonicalize(inputGraph.graph());g.setGraph(_.merge({},graphDefaults,selectNumberAttrs(graph,graphNumAttrs),_.pick(graph,graphAttrs)));_.each(inputGraph.nodes(),function(v){var node=canonicalize(inputGraph.node(v));g.setNode(v,_.defaults(selectNumberAttrs(node,nodeNumAttrs),nodeDefaults));g.setParent(v,inputGraph.parent(v))});_.each(inputGraph.edges(),function(e){var edge=canonicalize(inputGraph.edge(e));g.setEdge(e,_.merge({},edgeDefaults,selectNumberAttrs(edge,edgeNumAttrs),_.pick(edge,edgeAttrs)))});return g}function makeSpaceForEdgeLabels(g){var graph=g.graph();graph.ranksep/=2;_.each(g.edges(),function(e){var edge=g.edge(e);edge.minlen*=2;if(edge.labelpos.toLowerCase()!=="c"){if(graph.rankdir==="TB"||graph.rankdir==="BT"){edge.width+=edge.labeloffset}else{edge.height+=edge.labeloffset}}})}function injectEdgeLabelProxies(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.width&&edge.height){var v=g.node(e.v),w=g.node(e.w),label={rank:(w.rank-v.rank)/2+v.rank,e:e};util.addDummyNode(g,"edge-proxy",label,"_ep")}})}function assignRankMinMax(g){var maxRank=0;_.each(g.nodes(),function(v){var node=g.node(v);if(node.borderTop){node.minRank=g.node(node.borderTop).rank;node.maxRank=g.node(node.borderBottom).rank;maxRank=_.max(maxRank,node.maxRank)}});g.graph().maxRank=maxRank}function removeEdgeLabelProxies(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="edge-proxy"){g.edge(node.e).labelRank=node.rank;g.removeNode(v)}})}function translateGraph(g){var minX=Number.POSITIVE_INFINITY,maxX=0,minY=Number.POSITIVE_INFINITY,maxY=0,graphLabel=g.graph(),marginX=graphLabel.marginx||0,marginY=graphLabel.marginy||0;function getExtremes(attrs){var x=attrs.x,y=attrs.y,w=attrs.width,h=attrs.height;minX=Math.min(minX,x-w/2);maxX=Math.max(maxX,x+w/2);minY=Math.min(minY,y-h/2);maxY=Math.max(maxY,y+h/2)}_.each(g.nodes(),function(v){getExtremes(g.node(v))});_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){getExtremes(edge)}});minX-=marginX;minY-=marginY;_.each(g.nodes(),function(v){var node=g.node(v);node.x-=minX;node.y-=minY});_.each(g.edges(),function(e){var edge=g.edge(e);_.each(edge.points,function(p){p.x-=minX;p.y-=minY});if(_.has(edge,"x")){edge.x-=minX}if(_.has(edge,"y")){edge.y-=minY}});graphLabel.width=maxX-minX+marginX;graphLabel.height=maxY-minY+marginY}function assignNodeIntersects(g){_.each(g.edges(),function(e){var edge=g.edge(e),nodeV=g.node(e.v),nodeW=g.node(e.w),p1,p2;if(!edge.points){edge.points=[];p1=nodeW;p2=nodeV}else{p1=edge.points[0];p2=edge.points[edge.points.length-1]}edge.points.unshift(util.intersectRect(nodeV,p1));edge.points.push(util.intersectRect(nodeW,p2))})}function fixupEdgeLabelCoords(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){if(edge.labelpos==="l"||edge.labelpos==="r"){edge.width-=edge.labeloffset}switch(edge.labelpos){case"l":edge.x-=edge.width/2+edge.labeloffset;break;case"r":edge.x+=edge.width/2+edge.labeloffset;break}}})}function reversePointsForReversedEdges(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.reversed){edge.points.reverse()}})}function removeBorderNodes(g){_.each(g.nodes(),function(v){if(g.children(v).length){var node=g.node(v),t=g.node(node.borderTop),b=g.node(node.borderBottom),l=g.node(_.last(node.borderLeft)),r=g.node(_.last(node.borderRight));node.width=Math.abs(r.x-l.x);node.height=Math.abs(b.y-t.y);node.x=l.x+node.width/2;node.y=t.y+node.height/2}});_.each(g.nodes(),function(v){if(g.node(v).dummy==="border"){g.removeNode(v)}})}function removeSelfEdges(g){_.each(g.edges(),function(e){if(e.v===e.w){var node=g.node(e.v);if(!node.selfEdges){node.selfEdges=[]}node.selfEdges.push({e:e,label:g.edge(e)});g.removeEdge(e)}})}function insertSelfEdges(g){var layers=util.buildLayerMatrix(g);_.each(layers,function(layer){var orderShift=0;_.each(layer,function(v,i){var node=g.node(v);node.order=i+orderShift;_.each(node.selfEdges,function(selfEdge){util.addDummyNode(g,"selfedge",{width:selfEdge.label.width,height:selfEdge.label.height,rank:node.rank,order:i+ ++orderShift,e:selfEdge.e,label:selfEdge.label},"_se")});delete node.selfEdges})})}function positionSelfEdges(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="selfedge"){var selfNode=g.node(node.e.v),x=selfNode.x+selfNode.width/2,y=selfNode.y,dx=node.x-x,dy=selfNode.height/2;g.setEdge(node.e,node.label);g.removeNode(v);node.label.points=[{x:x+2*dx/3,y:y-dy},{x:x+5*dx/6,y:y-dy},{x:x+dx,y:y},{x:x+5*dx/6,y:y+dy},{x:x+2*dx/3,y:y+dy}];node.label.x=node.x;node.label.y=node.y}})}function selectNumberAttrs(obj,attrs){return _.mapValues(_.pick(obj,attrs),Number)}function canonicalize(attrs){var newAttrs={};_.each(attrs,function(v,k){newAttrs[k.toLowerCase()]=v});return newAttrs}},{"./acyclic":28,"./add-border-segments":29,"./coordinate-system":30,"./graphlib":33,"./lodash":36,"./nesting-graph":37,"./normalize":38,"./order":43,"./parent-dummy-chains":48,"./position":50,"./rank":52,"./util":55}],36:[function(require,module,exports){module.exports=require(20)},{"/Users/andrew/Documents/dev/dagre-d3/lib/lodash.js":20,lodash:77}],37:[function(require,module,exports){var _=require("./lodash"),util=require("./util");module.exports={run:run,cleanup:cleanup};function run(g){var root=util.addDummyNode(g,"root",{},"_root"),depths=treeDepths(g),height=_.max(depths)-1,nodeSep=2*height+1;g.graph().nestingRoot=root;_.each(g.edges(),function(e){g.edge(e).minlen*=nodeSep});var weight=sumWeights(g)+1;_.each(g.children(),function(child){dfs(g,root,nodeSep,weight,height,depths,child)});g.graph().nodeRankFactor=nodeSep}function dfs(g,root,nodeSep,weight,height,depths,v){var children=g.children(v);if(!children.length){if(v!==root){g.setEdge(root,v,{weight:0,minlen:nodeSep})}return}var top=util.addBorderNode(g,"_bt"),bottom=util.addBorderNode(g,"_bb"),label=g.node(v);g.setParent(top,v);label.borderTop=top;g.setParent(bottom,v);label.borderBottom=bottom;_.each(children,function(child){dfs(g,root,nodeSep,weight,height,depths,child);var childNode=g.node(child),childTop=childNode.borderTop?childNode.borderTop:child,childBottom=childNode.borderBottom?childNode.borderBottom:child,thisWeight=childNode.borderTop?weight:2*weight,minlen=childTop!==childBottom?1:height-depths[v]+1;g.setEdge(top,childTop,{weight:thisWeight,minlen:minlen,nestingEdge:true});g.setEdge(childBottom,bottom,{weight:thisWeight,minlen:minlen,nestingEdge:true})});if(!g.parent(v)){g.setEdge(root,top,{weight:0,minlen:height+depths[v]})}}function treeDepths(g){var depths={};function dfs(v,depth){var children=g.children(v);if(children&&children.length){_.each(children,function(child){dfs(child,depth+1)})}depths[v]=depth}_.each(g.children(),function(v){dfs(v,1)});return depths}function sumWeights(g){return _.reduce(g.edges(),function(acc,e){return acc+g.edge(e).weight},0)}function cleanup(g){var graphLabel=g.graph();g.removeNode(graphLabel.nestingRoot);delete graphLabel.nestingRoot;_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.nestingEdge){g.removeEdge(e)}})}},{"./lodash":36,"./util":55}],38:[function(require,module,exports){"use strict";var _=require("./lodash"),util=require("./util");module.exports={run:run,undo:undo};function run(g){g.graph().dummyChains=[];_.each(g.edges(),function(edge){normalizeEdge(g,edge)})}function normalizeEdge(g,e){var v=e.v,vRank=g.node(v).rank,w=e.w,wRank=g.node(w).rank,name=e.name,edgeLabel=g.edge(e),labelRank=edgeLabel.labelRank;if(wRank===vRank+1)return;g.removeEdge(e);var dummy,attrs,i;for(i=0,++vRank;vRank0){if(index%2){weightSum+=tree[index+1]}index=index-1>>1;tree[index]+=entry.weight}cc+=entry.weight*weightSum}));return cc}},{"../lodash":36}],43:[function(require,module,exports){"use strict";var _=require("../lodash"),initOrder=require("./init-order"),crossCount=require("./cross-count"),sortSubgraph=require("./sort-subgraph"),buildLayerGraph=require("./build-layer-graph"),addSubgraphConstraints=require("./add-subgraph-constraints"),Graph=require("../graphlib").Graph,util=require("../util");module.exports=order;function order(g){var maxRank=util.maxRank(g),downLayerGraphs=buildLayerGraphs(g,_.range(1,maxRank+1),"inEdges"),upLayerGraphs=buildLayerGraphs(g,_.range(maxRank-1,-1,-1),"outEdges");var layering=initOrder(g);assignOrder(g,layering);var bestCC=Number.POSITIVE_INFINITY,best;for(var i=0,lastBest=0;lastBest<4;++i,++lastBest){sweepLayerGraphs(i%2?downLayerGraphs:upLayerGraphs,i%4>=2);layering=util.buildLayerMatrix(g);var cc=crossCount(g,layering);if(cc=vEntry.barycenter){mergeEntries(vEntry,uEntry)}}}function handleOut(vEntry){return function(wEntry){wEntry["in"].push(vEntry);if(--wEntry.indegree===0){sourceSet.push(wEntry)}}}while(sourceSet.length){var entry=sourceSet.pop();entries.push(entry);_.each(entry["in"].reverse(),handleIn(entry));_.each(entry.out,handleOut(entry))}return _.chain(entries).filter(function(entry){return!entry.merged}).map(function(entry){return _.pick(entry,["vs","i","barycenter","weight"])}).value()}function mergeEntries(target,source){var sum=0,weight=0;if(target.weight){sum+=target.barycenter*target.weight;weight+=target.weight}if(source.weight){sum+=source.barycenter*source.weight;weight+=source.weight}target.vs=source.vs.concat(target.vs);target.barycenter=sum/weight;target.weight=weight;target.i=Math.min(source.i,target.i);source.merged=true}},{"../lodash":36}],46:[function(require,module,exports){var _=require("../lodash"),barycenter=require("./barycenter"),resolveConflicts=require("./resolve-conflicts"),sort=require("./sort");module.exports=sortSubgraph;function sortSubgraph(g,v,cg,biasRight){var movable=g.children(v),node=g.node(v),bl=node?node.borderLeft:undefined,br=node?node.borderRight:undefined,subgraphs={};if(bl){movable=_.filter(movable,function(w){return w!==bl&&w!==br})}var barycenters=barycenter(g,movable);_.each(barycenters,function(entry){if(g.children(entry.v).length){var subgraphResult=sortSubgraph(g,entry.v,cg,biasRight);subgraphs[entry.v]=subgraphResult;if(_.has(subgraphResult,"barycenter")){mergeBarycenters(entry,subgraphResult)}}});var entries=resolveConflicts(barycenters,cg);expandSubgraphs(entries,subgraphs);var result=sort(entries,biasRight);if(bl){result.vs=_.flatten([bl,result.vs,br],true);if(g.predecessors(bl).length){var blPred=g.node(g.predecessors(bl)[0]),brPred=g.node(g.predecessors(br)[0]);if(!_.has(result,"barycenter")){result.barycenter=0;result.weight=0}result.barycenter=(result.barycenter*result.weight+blPred.order+brPred.order)/(result.weight+2);result.weight+=2}}return result}function expandSubgraphs(entries,subgraphs){_.each(entries,function(entry){entry.vs=_.flatten(entry.vs.map(function(v){if(subgraphs[v]){return subgraphs[v].vs}return v}),true)})}function mergeBarycenters(target,other){if(!_.isUndefined(target.barycenter)){target.barycenter=(target.barycenter*target.weight+other.barycenter*other.weight)/(target.weight+other.weight);target.weight+=other.weight}else{target.barycenter=other.barycenter;target.weight=other.weight}}},{"../lodash":36,"./barycenter":40,"./resolve-conflicts":45,"./sort":47}],47:[function(require,module,exports){var _=require("../lodash"),util=require("../util");module.exports=sort;function sort(entries,biasRight){var parts=util.partition(entries,function(entry){return _.has(entry,"barycenter")});var sortable=parts.lhs,unsortable=_.sortBy(parts.rhs,function(entry){return-entry.i}),vs=[],sum=0,weight=0,vsIndex=0;sortable.sort(compareWithBias(!!biasRight));vsIndex=consumeUnsortable(vs,unsortable,vsIndex);_.each(sortable,function(entry){vsIndex+=entry.vs.length;vs.push(entry.vs);sum+=entry.barycenter*entry.weight;weight+=entry.weight;vsIndex=consumeUnsortable(vs,unsortable,vsIndex)});var result={vs:_.flatten(vs,true)};if(weight){result.barycenter=sum/weight;result.weight=weight}return result}function consumeUnsortable(vs,unsortable,index){var last;while(unsortable.length&&(last=_.last(unsortable)).i<=index){unsortable.pop();vs.push(last.vs);index++}return index}function compareWithBias(bias){return function(entryV,entryW){if(entryV.barycenterentryW.barycenter){return 1}return!bias?entryV.i-entryW.i:entryW.i-entryV.i}}},{"../lodash":36,"../util":55}],48:[function(require,module,exports){var _=require("./lodash");module.exports=parentDummyChains;function parentDummyChains(g){var postorderNums=postorder(g);_.each(g.graph().dummyChains,function(v){var node=g.node(v),edgeObj=node.edgeObj,pathData=findPath(g,postorderNums,edgeObj.v,edgeObj.w),path=pathData.path,lca=pathData.lca,pathIdx=0,pathV=path[pathIdx],ascending=true;while(v!==edgeObj.w){node=g.node(v);if(ascending){while((pathV=path[pathIdx])!==lca&&g.node(pathV).maxRanklow||lim>postorderNums[parent].lim));lca=parent;parent=w;while((parent=g.parent(parent))!==lca){wPath.push(parent)}return{path:vPath.concat(wPath.reverse()),lca:lca}}function postorder(g){var result={},lim=0;function dfs(v){var low=lim;_.each(g.children(v),dfs);result[v]={low:low,lim:lim++}}_.each(g.children(),dfs);return result}},{"./lodash":36}],49:[function(require,module,exports){"use strict";var _=require("../lodash"),Graph=require("../graphlib").Graph,util=require("../util");module.exports={positionX:positionX,findType1Conflicts:findType1Conflicts,findType2Conflicts:findType2Conflicts,addConflict:addConflict,hasConflict:hasConflict,verticalAlignment:verticalAlignment,horizontalCompaction:horizontalCompaction,alignCoordinates:alignCoordinates,findSmallestWidthAlignment:findSmallestWidthAlignment,balance:balance};function findType1Conflicts(g,layering){var conflicts={};function visitLayer(prevLayer,layer){var k0=0,scanPos=0,prevLayerLength=prevLayer.length,lastNode=_.last(layer);_.each(layer,function(v,i){var w=findOtherInnerSegmentNode(g,v),k1=w?g.node(w).order:prevLayerLength;if(w||v===lastNode){_.each(layer.slice(scanPos,i+1),function(scanNode){_.each(g.predecessors(scanNode),function(u){var uLabel=g.node(u),uPos=uLabel.order;if((uPosnextNorthBorder)){addConflict(conflicts,u,v)}})}})}function visitLayer(north,south){var prevNorthPos=-1,nextNorthPos,southPos=0;_.each(south,function(v,southLookahead){if(g.node(v).dummy==="border"){var predecessors=g.predecessors(v);if(predecessors.length){nextNorthPos=g.node(predecessors[0]).order;scan(south,southPos,southLookahead,prevNorthPos,nextNorthPos);southPos=southLookahead;prevNorthPos=nextNorthPos}}scan(south,southPos,south.length,nextNorthPos,north.length)});return south}_.reduce(layering,visitLayer);return conflicts}function findOtherInnerSegmentNode(g,v){if(g.node(v).dummy){return _.find(g.predecessors(v),function(u){return g.node(u).dummy})}}function addConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}var conflictsV=conflicts[v];if(!conflictsV){conflicts[v]=conflictsV={}}conflictsV[w]=true}function hasConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}return _.has(conflicts[v],w)}function verticalAlignment(g,layering,conflicts,neighborFn){var root={},align={},pos={};_.each(layering,function(layer){_.each(layer,function(v,order){root[v]=v;align[v]=v;pos[v]=order})});_.each(layering,function(layer){var prevIdx=-1;_.each(layer,function(v){var ws=neighborFn(v);if(ws.length){ws=_.sortBy(ws,function(w){return pos[w]});var mp=(ws.length-1)/2;for(var i=Math.floor(mp),il=Math.ceil(mp);i<=il;++i){var w=ws[i];if(align[v]===v&&prevIdxwLabel.lim){tailLabel=wLabel;flip=true}var candidates=_.filter(g.edges(),function(edge){return flip===isDescendant(t,t.node(edge.v),tailLabel)&&flip!==isDescendant(t,t.node(edge.w),tailLabel)});return _.min(candidates,function(edge){return slack(g,edge)})}function exchangeEdges(t,g,e,f){var v=e.v,w=e.w;t.removeEdge(v,w);t.setEdge(f.v,f.w,{});initLowLimValues(t);initCutValues(t,g);updateRanks(t,g)}function updateRanks(t,g){var root=_.find(t.nodes(),function(v){return!g.node(v).parent}),vs=preorder(t,root);vs=vs.slice(1);_.each(vs,function(v){var parent=t.node(v).parent,edge=g.edge(v,parent),flipped=false;if(!edge){edge=g.edge(parent,v);flipped=true}g.node(v).rank=g.node(parent).rank+(flipped?edge.minlen:-edge.minlen)})}function isTreeEdge(tree,u,v){return tree.hasEdge(u,v)}function isDescendant(tree,vLabel,rootLabel){return rootLabel.low<=vLabel.lim&&vLabel.lim<=rootLabel.lim}},{"../graphlib":33,"../lodash":36,"../util":55,"./feasible-tree":51,"./util":54}],54:[function(require,module,exports){"use strict";var _=require("../lodash");module.exports={longestPath:longestPath,slack:slack};function longestPath(g){var visited={};function dfs(v){var label=g.node(v);if(_.has(visited,v)){return label.rank}visited[v]=true;var rank=_.min(_.map(g.outEdges(v),function(e){return dfs(e.w)-g.edge(e).minlen}));if(rank===Number.POSITIVE_INFINITY){rank=0}return label.rank=rank}_.each(g.sources(),dfs)}function slack(g,e){return g.node(e.w).rank-g.node(e.v).rank-g.edge(e).minlen}},{"../lodash":36}],55:[function(require,module,exports){"use strict";var _=require("./lodash"),Graph=require("./graphlib").Graph;module.exports={addDummyNode:addDummyNode,simplify:simplify,asNonCompoundGraph:asNonCompoundGraph,successorWeights:successorWeights,predecessorWeights:predecessorWeights,intersectRect:intersectRect,buildLayerMatrix:buildLayerMatrix,normalizeRanks:normalizeRanks,removeEmptyRanks:removeEmptyRanks,addBorderNode:addBorderNode,maxRank:maxRank,partition:partition,time:time,notime:notime};function addDummyNode(g,type,attrs,name){var v;do{v=_.uniqueId(name)}while(g.hasNode(v));attrs.dummy=type;g.setNode(v,attrs);return v}function simplify(g){var simplified=(new Graph).setGraph(g.graph());_.each(g.nodes(),function(v){simplified.setNode(v,g.node(v))});_.each(g.edges(),function(e){var simpleLabel=simplified.edge(e.v,e.w)||{weight:0,minlen:1},label=g.edge(e);simplified.setEdge(e.v,e.w,{weight:simpleLabel.weight+label.weight,minlen:Math.max(simpleLabel.minlen,label.minlen)})});return simplified}function asNonCompoundGraph(g){var simplified=new Graph({multigraph:g.isMultigraph()}).setGraph(g.graph());_.each(g.nodes(),function(v){if(!g.children(v).length){simplified.setNode(v,g.node(v))}});_.each(g.edges(),function(e){simplified.setEdge(e,g.edge(e))});return simplified}function successorWeights(g){var weightMap=_.map(g.nodes(),function(v){var sucs={};_.each(g.outEdges(v),function(e){sucs[e.w]=(sucs[e.w]||0)+g.edge(e).weight});return sucs});return _.zipObject(g.nodes(),weightMap)}function predecessorWeights(g){var weightMap=_.map(g.nodes(),function(v){var preds={};_.each(g.inEdges(v),function(e){preds[e.v]=(preds[e.v]||0)+g.edge(e).weight});return preds});return _.zipObject(g.nodes(),weightMap)}function intersectRect(rect,point){var x=rect.x;var y=rect.y;var dx=point.x-x;var dy=point.y-y;var w=rect.width/2;var h=rect.height/2;if(!dx&&!dy){throw new Error("Not possible to find intersection inside of the rectangle")}var sx,sy;if(Math.abs(dy)*w>Math.abs(dx)*h){if(dy<0){h=-h}sx=h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=w*dy/dx}return{x:x+sx,y:y+sy}}function buildLayerMatrix(g){var layering=_.map(_.range(maxRank(g)+1),function(){return[]});_.each(g.nodes(),function(v){var node=g.node(v),rank=node.rank;if(!_.isUndefined(rank)){layering[rank][node.order]=v}});return layering}function normalizeRanks(g){var min=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));_.each(g.nodes(),function(v){var node=g.node(v);if(_.has(node,"rank")){node.rank-=min}})}function removeEmptyRanks(g){var offset=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));var layers=[];_.each(g.nodes(),function(v){var rank=g.node(v).rank-offset;if(!_.has(layers,rank)){layers[rank]=[]}layers[rank].push(v)});var delta=0,nodeRankFactor=g.graph().nodeRankFactor;_.each(layers,function(vs,i){if(_.isUndefined(vs)&&i%nodeRankFactor!==0){--delta}else if(delta){_.each(vs,function(v){g.node(v).rank+=delta})}})}function addBorderNode(g,prefix,rank,order){var node={width:0,height:0};if(arguments.length>=4){node.rank=rank;node.order=order}return addDummyNode(g,"border",node,prefix)}function maxRank(g){return _.max(_.map(g.nodes(),function(v){var rank=g.node(v).rank;if(!_.isUndefined(rank)){return rank}}))}function partition(collection,fn){var result={lhs:[],rhs:[]};_.each(collection,function(value){if(fn(value)){result.lhs.push(value)}else{result.rhs.push(value)}});return result}function time(name,fn){var start=_.now();try{return fn()}finally{console.log(name+" time: "+(_.now()-start)+"ms")}}function notime(name,fn){return fn()}},{"./graphlib":33,"./lodash":36}],56:[function(require,module,exports){module.exports="0.7.1"},{}],57:[function(require,module,exports){var lib=require("./lib");module.exports={Graph:lib.Graph,json:require("./lib/json"),alg:require("./lib/alg"),version:lib.version}},{"./lib":73,"./lib/alg":64,"./lib/json":74}],58:[function(require,module,exports){var _=require("../lodash");module.exports=components;function components(g){var visited={},cmpts=[],cmpt;function dfs(v){if(_.has(visited,v))return;visited[v]=true;cmpt.push(v);_.each(g.successors(v),dfs);_.each(g.predecessors(v),dfs)}_.each(g.nodes(),function(v){cmpt=[];dfs(v);if(cmpt.length){cmpts.push(cmpt)}});return cmpts}},{"../lodash":75}],59:[function(require,module,exports){var _=require("../lodash");module.exports=dfs;function dfs(g,vs,order){if(!_.isArray(vs)){vs=[vs]}var acc=[],visited={};_.each(vs,function(v){if(!g.hasNode(v)){throw new Error("Graph does not have node: "+v)}doDfs(g,v,order==="post",visited,acc)});return acc}function doDfs(g,v,postorder,visited,acc){if(!_.has(visited,v)){visited[v]=true;if(!postorder){acc.push(v)}_.each(g.neighbors(v),function(w){doDfs(g,w,postorder,visited,acc)});if(postorder){acc.push(v)}}}},{"../lodash":75}],60:[function(require,module,exports){var dijkstra=require("./dijkstra"),_=require("../lodash");module.exports=dijkstraAll;function dijkstraAll(g,weightFunc,edgeFunc){return _.transform(g.nodes(),function(acc,v){acc[v]=dijkstra(g,v,weightFunc,edgeFunc)},{})}},{"../lodash":75,"./dijkstra":61}],61:[function(require,module,exports){var _=require("../lodash"),PriorityQueue=require("../data/priority-queue");module.exports=dijkstra;var DEFAULT_WEIGHT_FUNC=_.constant(1);function dijkstra(g,source,weightFn,edgeFn){return runDijkstra(g,String(source),weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runDijkstra(g,source,weightFn,edgeFn){var results={},pq=new PriorityQueue,v,vEntry;var updateNeighbors=function(edge){var w=edge.v!==v?edge.v:edge.w,wEntry=results[w],weight=weightFn(edge),distance=vEntry.distance+weight;if(weight<0){throw new Error("dijkstra does not allow negative edge weights. "+"Bad edge: "+edge+" Weight: "+weight)}if(distance0){v=pq.removeMin();vEntry=results[v];if(vEntry.distance===Number.POSITIVE_INFINITY){break}edgeFn(v).forEach(updateNeighbors)}return results}},{"../data/priority-queue":71,"../lodash":75}],62:[function(require,module,exports){var _=require("../lodash"),tarjan=require("./tarjan");module.exports=findCycles;function findCycles(g){return _.filter(tarjan(g),function(cmpt){return cmpt.length>1})}},{"../lodash":75,"./tarjan":69}],63:[function(require,module,exports){var _=require("../lodash");module.exports=floydWarshall;var DEFAULT_WEIGHT_FUNC=_.constant(1);function floydWarshall(g,weightFn,edgeFn){return runFloydWarshall(g,weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runFloydWarshall(g,weightFn,edgeFn){var results={},nodes=g.nodes();nodes.forEach(function(v){results[v]={};results[v][v]={distance:0};nodes.forEach(function(w){if(v!==w){results[v][w]={distance:Number.POSITIVE_INFINITY}}});edgeFn(v).forEach(function(edge){var w=edge.v===v?edge.w:edge.v,d=weightFn(edge);results[v][w]={distance:d,predecessor:v}})});nodes.forEach(function(k){var rowK=results[k];nodes.forEach(function(i){var rowI=results[i];nodes.forEach(function(j){var ik=rowI[k];var kj=rowK[j];var ij=rowI[j];var altDistance=ik.distance+kj.distance;if(altDistance0){v=pq.removeMin();if(_.has(parents,v)){result.setEdge(v,parents[v])}else if(init){throw new Error("Input graph is not connected: "+g)}else{init=true}g.nodeEdges(v).forEach(updateNeighbors)}return result}},{"../data/priority-queue":71,"../graph":72,"../lodash":75}],69:[function(require,module,exports){var _=require("../lodash");module.exports=tarjan;function tarjan(g){var index=0,stack=[],visited={},results=[];function dfs(v){var entry=visited[v]={onStack:true,lowlink:index,index:index++};stack.push(v);g.successors(v).forEach(function(w){if(!_.has(visited,w)){dfs(w);entry.lowlink=Math.min(entry.lowlink,visited[w].lowlink)}else if(visited[w].onStack){entry.lowlink=Math.min(entry.lowlink,visited[w].index)}});if(entry.lowlink===entry.index){var cmpt=[],w;do{w=stack.pop();visited[w].onStack=false;cmpt.push(w)}while(v!==w);results.push(cmpt)}}g.nodes().forEach(function(v){if(!_.has(visited,v)){dfs(v)}});return results}},{"../lodash":75}],70:[function(require,module,exports){var _=require("../lodash");module.exports=topsort;topsort.CycleException=CycleException;function topsort(g){var visited={},stack={},results=[];function visit(node){if(_.has(stack,node)){throw new CycleException}if(!_.has(visited,node)){stack[node]=true;visited[node]=true;_.each(g.predecessors(node),visit);delete stack[node];results.push(node)}}_.each(g.sinks(),visit);if(_.size(visited)!==g.nodeCount()){throw new CycleException}return results}function CycleException(){}},{"../lodash":75}],71:[function(require,module,exports){var _=require("../lodash");module.exports=PriorityQueue;function PriorityQueue(){this._arr=[];this._keyIndices={}}PriorityQueue.prototype.size=function(){return this._arr.length};PriorityQueue.prototype.keys=function(){return this._arr.map(function(x){return x.key})};PriorityQueue.prototype.has=function(key){return _.has(this._keyIndices,key)};PriorityQueue.prototype.priority=function(key){var index=this._keyIndices[key];if(index!==undefined){return this._arr[index].priority}};PriorityQueue.prototype.min=function(){if(this.size()===0){throw new Error("Queue underflow")}return this._arr[0].key};PriorityQueue.prototype.add=function(key,priority){var keyIndices=this._keyIndices;key=String(key);if(!_.has(keyIndices,key)){var arr=this._arr;var index=arr.length;keyIndices[key]=index;arr.push({key:key,priority:priority});this._decrease(index);return true}return false};PriorityQueue.prototype.removeMin=function(){this._swap(0,this._arr.length-1);var min=this._arr.pop();delete this._keyIndices[min.key];this._heapify(0);return min.key};PriorityQueue.prototype.decrease=function(key,priority){var index=this._keyIndices[key];if(priority>this._arr[index].priority){throw new Error("New priority is greater than current priority. "+"Key: "+key+" Old: "+this._arr[index].priority+" New: "+priority)}this._arr[index].priority=priority;this._decrease(index)};PriorityQueue.prototype._heapify=function(i){var arr=this._arr;var l=2*i,r=l+1,largest=i;if(l>1;if(arr[parent].priority1){this.setNode(v,value)}else{this.setNode(v)}},this);return this};Graph.prototype.setNode=function(v,value){if(_.has(this._nodes,v)){if(arguments.length>1){this._nodes[v]=value}return this}this._nodes[v]=arguments.length>1?value:this._defaultNodeLabelFn(v);if(this._isCompound){this._parent[v]=GRAPH_NODE;this._children[v]={};this._children[GRAPH_NODE][v]=true}this._in[v]={};this._preds[v]={};this._out[v]={};this._sucs[v]={};++this._nodeCount;return this};Graph.prototype.node=function(v){return this._nodes[v]};Graph.prototype.hasNode=function(v){return _.has(this._nodes,v)};Graph.prototype.removeNode=function(v){var self=this;if(_.has(this._nodes,v)){var removeEdge=function(e){self.removeEdge(self._edgeObjs[e])};delete this._nodes[v];if(this._isCompound){this._removeFromParentsChildList(v);delete this._parent[v];_.each(this.children(v),function(child){this.setParent(child)},this);delete this._children[v]}_.each(_.keys(this._in[v]),removeEdge);delete this._in[v];delete this._preds[v];_.each(_.keys(this._out[v]),removeEdge);delete this._out[v];delete this._sucs[v];--this._nodeCount}return this};Graph.prototype.setParent=function(v,parent){if(!this._isCompound){throw new Error("Cannot set parent in a non-compound graph")}if(_.isUndefined(parent)){parent=GRAPH_NODE}else{for(var ancestor=parent;!_.isUndefined(ancestor);ancestor=this.parent(ancestor)){if(ancestor===v){throw new Error("Setting "+parent+" as parent of "+v+" would create create a cycle")}}this.setNode(parent)}this.setNode(v);this._removeFromParentsChildList(v);this._parent[v]=parent;this._children[parent][v]=true;return this};Graph.prototype._removeFromParentsChildList=function(v){delete this._children[this._parent[v]][v]};Graph.prototype.parent=function(v){if(this._isCompound){var parent=this._parent[v];if(parent!==GRAPH_NODE){return parent}}};Graph.prototype.children=function(v){if(_.isUndefined(v)){v=GRAPH_NODE}if(this._isCompound){var children=this._children[v];if(children){return _.keys(children)}}else if(v===GRAPH_NODE){return this.nodes()}else if(this.hasNode(v)){return[]}};Graph.prototype.predecessors=function(v){var predsV=this._preds[v];if(predsV){return _.keys(predsV)}};Graph.prototype.successors=function(v){var sucsV=this._sucs[v];if(sucsV){return _.keys(sucsV)}};Graph.prototype.neighbors=function(v){var preds=this.predecessors(v);if(preds){return _.union(preds,this.successors(v))}};Graph.prototype.setDefaultEdgeLabel=function(newDefault){if(!_.isFunction(newDefault)){newDefault=_.constant(newDefault)}this._defaultEdgeLabelFn=newDefault;return this};Graph.prototype.edgeCount=function(){return this._edgeCount};Graph.prototype.edges=function(){return _.values(this._edgeObjs)};Graph.prototype.setPath=function(vs,value){var self=this,args=arguments;_.reduce(vs,function(v,w){if(args.length>1){self.setEdge(v,w,value)}else{self.setEdge(v,w)}return w});return this};Graph.prototype.setEdge=function(){var v,w,name,value,valueSpecified=false;if(_.isPlainObject(arguments[0])){v=arguments[0].v;w=arguments[0].w;name=arguments[0].name;if(arguments.length===2){value=arguments[1];valueSpecified=true}}else{v=arguments[0];w=arguments[1];name=arguments[3];if(arguments.length>2){value=arguments[2];valueSpecified=true}}v=""+v;w=""+w;if(!_.isUndefined(name)){name=""+name}var e=edgeArgsToId(this._isDirected,v,w,name);if(_.has(this._edgeLabels,e)){if(valueSpecified){this._edgeLabels[e]=value}return this}if(!_.isUndefined(name)&&!this._isMultigraph){throw new Error("Cannot set a named edge when isMultigraph = false")}this.setNode(v);this.setNode(w);this._edgeLabels[e]=valueSpecified?value:this._defaultEdgeLabelFn(v,w,name);var edgeObj=edgeArgsToObj(this._isDirected,v,w,name);v=edgeObj.v;w=edgeObj.w;Object.freeze(edgeObj);this._edgeObjs[e]=edgeObj;incrementOrInitEntry(this._preds[w],v);incrementOrInitEntry(this._sucs[v],w);this._in[w][e]=edgeObj;this._out[v][e]=edgeObj;this._edgeCount++;return this};Graph.prototype.edge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return this._edgeLabels[e]};Graph.prototype.hasEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return _.has(this._edgeLabels,e)};Graph.prototype.removeEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name),edge=this._edgeObjs[e];if(edge){v=edge.v;w=edge.w;delete this._edgeLabels[e];delete this._edgeObjs[e];decrementOrRemoveEntry(this._preds[w],v);decrementOrRemoveEntry(this._sucs[v],w);delete this._in[w][e];delete this._out[v][e];this._edgeCount--}return this};Graph.prototype.inEdges=function(v,u){var inV=this._in[v];if(inV){var edges=_.values(inV);if(!u){return edges}return _.filter(edges,function(edge){return edge.v===u})}};Graph.prototype.outEdges=function(v,w){var outV=this._out[v];if(outV){var edges=_.values(outV);if(!w){return edges}return _.filter(edges,function(edge){return edge.w===w})}};Graph.prototype.nodeEdges=function(v,w){var inEdges=this.inEdges(v,w);if(inEdges){return inEdges.concat(this.outEdges(v,w))}};function incrementOrInitEntry(map,k){if(_.has(map,k)){map[k]++}else{map[k]=1}}function decrementOrRemoveEntry(map,k){if(!--map[k]){delete map[k]}}function edgeArgsToId(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}return v+EDGE_KEY_DELIM+w+EDGE_KEY_DELIM+(_.isUndefined(name)?DEFAULT_EDGE_NAME:name)}function edgeArgsToObj(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}var edgeObj={v:v,w:w};if(name){edgeObj.name=name}return edgeObj}function edgeObjToId(isDirected,edgeObj){return edgeArgsToId(isDirected,edgeObj.v,edgeObj.w,edgeObj.name)}},{"./lodash":75}],73:[function(require,module,exports){module.exports={Graph:require("./graph"),version:require("./version")}},{"./graph":72,"./version":76}],74:[function(require,module,exports){var _=require("./lodash"),Graph=require("./graph");module.exports={write:write,read:read};function write(g){var json={options:{directed:g.isDirected(),multigraph:g.isMultigraph(),compound:g.isCompound()},nodes:writeNodes(g),edges:writeEdges(g)};if(!_.isUndefined(g.graph())){json.value=_.clone(g.graph())}return json}function writeNodes(g){return _.map(g.nodes(),function(v){var nodeValue=g.node(v),parent=g.parent(v),node={v:v};if(!_.isUndefined(nodeValue)){node.value=nodeValue}if(!_.isUndefined(parent)){node.parent=parent}return node})}function writeEdges(g){return _.map(g.edges(),function(e){var edgeValue=g.edge(e),edge={v:e.v,w:e.w};if(!_.isUndefined(e.name)){edge.name=e.name}if(!_.isUndefined(edgeValue)){edge.value=edgeValue}return edge})}function read(json){var g=new Graph(json.options).setGraph(json.value);_.each(json.nodes,function(entry){g.setNode(entry.v,entry.value);if(entry.parent){g.setParent(entry.v,entry.parent)}});_.each(json.edges,function(entry){g.setEdge({v:entry.v,w:entry.w,name:entry.name},entry.value)});return g}},{"./graph":72,"./lodash":75}],75:[function(require,module,exports){module.exports=require(20)},{"/Users/andrew/Documents/dev/dagre-d3/lib/lodash.js":20,lodash:77}],76:[function(require,module,exports){module.exports="1.0.1"},{}],77:[function(require,module,exports){(function(global){(function(){var undefined;var arrayPool=[],objectPool=[];var idCounter=0;var keyPrefix=+new Date+"";var largeArraySize=75;var maxPoolSize=40;var whitespace=" \f \ufeff"+"\n\r\u2028\u2029"+" ᠎              ";var reEmptyStringLeading=/\b__p \+= '';/g,reEmptyStringMiddle=/\b(__p \+=) '' \+/g,reEmptyStringTrailing=/(__e\(.*?\)|\b__t\)) \+\n'';/g;var reEsTemplate=/\$\{([^\\}]*(?:\\.[^\\}]*)*)\}/g;var reFlags=/\w*$/;var reFuncName=/^\s*function[ \n\r\t]+\w/;var reInterpolate=/<%=([\s\S]+?)%>/g;var reLeadingSpacesAndZeros=RegExp("^["+whitespace+"]*0+(?=.$)");var reNoMatch=/($^)/;var reThis=/\bthis\b/;var reUnescapedString=/['\n\r\t\u2028\u2029\\]/g;var contextProps=["Array","Boolean","Date","Function","Math","Number","Object","RegExp","String","_","attachEvent","clearTimeout","isFinite","isNaN","parseInt","setTimeout"];var templateCounter=0;var argsClass="[object Arguments]",arrayClass="[object Array]",boolClass="[object Boolean]",dateClass="[object Date]",funcClass="[object Function]",numberClass="[object Number]",objectClass="[object Object]",regexpClass="[object RegExp]",stringClass="[object String]";var cloneableClasses={};cloneableClasses[funcClass]=false;cloneableClasses[argsClass]=cloneableClasses[arrayClass]=cloneableClasses[boolClass]=cloneableClasses[dateClass]=cloneableClasses[numberClass]=cloneableClasses[objectClass]=cloneableClasses[regexpClass]=cloneableClasses[stringClass]=true;var debounceOptions={leading:false,maxWait:0,trailing:false};var descriptor={configurable:false,enumerable:false,value:null,writable:false};var objectTypes={"boolean":false,"function":true,object:true,number:false,string:false,undefined:false};var stringEscapes={"\\":"\\","'":"'","\n":"n","\r":"r"," ":"t","\u2028":"u2028","\u2029":"u2029"};var root=objectTypes[typeof window]&&window||this;var freeExports=objectTypes[typeof exports]&&exports&&!exports.nodeType&&exports;var freeModule=objectTypes[typeof module]&&module&&!module.nodeType&&module;var moduleExports=freeModule&&freeModule.exports===freeExports&&freeExports;var freeGlobal=objectTypes[typeof global]&&global;if(freeGlobal&&(freeGlobal.global===freeGlobal||freeGlobal.window===freeGlobal)){root=freeGlobal}function baseIndexOf(array,value,fromIndex){var index=(fromIndex||0)-1,length=array?array.length:0;while(++index-1?0:-1:cache?0:-1}function cachePush(value){var cache=this.cache,type=typeof value;if(type=="boolean"||value==null){cache[value]=true}else{if(type!="number"&&type!="string"){type="object"}var key=type=="number"?value:keyPrefix+value,typeCache=cache[type]||(cache[type]={});if(type=="object"){(typeCache[key]||(typeCache[key]=[])).push(value)}else{typeCache[key]=true}}}function charAtCallback(value){return value.charCodeAt(0)}function compareAscending(a,b){var ac=a.criteria,bc=b.criteria,index=-1,length=ac.length;while(++indexother||typeof value=="undefined"){return 1}if(value/g,evaluate:/<%([\s\S]+?)%>/g,interpolate:reInterpolate,variable:"",imports:{_:lodash}};function baseBind(bindData){var func=bindData[0],partialArgs=bindData[2],thisArg=bindData[4];function bound(){if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(this instanceof bound){var thisBinding=baseCreate(func.prototype),result=func.apply(thisBinding,args||arguments);return isObject(result)?result:thisBinding}return func.apply(thisArg,args||arguments)}setBindData(bound,bindData);return bound}function baseClone(value,isDeep,callback,stackA,stackB){if(callback){var result=callback(value);if(typeof result!="undefined"){return result}}var isObj=isObject(value);if(isObj){var className=toString.call(value);if(!cloneableClasses[className]){return value}var ctor=ctorByClass[className];switch(className){case boolClass:case dateClass:return new ctor(+value);case numberClass:case stringClass:return new ctor(value);case regexpClass:result=ctor(value.source,reFlags.exec(value));result.lastIndex=value.lastIndex;return result}}else{return value}var isArr=isArray(value);if(isDeep){var initedStack=!stackA;stackA||(stackA=getArray());stackB||(stackB=getArray());var length=stackA.length;while(length--){if(stackA[length]==value){return stackB[length]}}result=isArr?ctor(value.length):{}}else{result=isArr?slice(value):assign({},value)}if(isArr){if(hasOwnProperty.call(value,"index")){result.index=value.index}if(hasOwnProperty.call(value,"input")){result.input=value.input}}if(!isDeep){return result}stackA.push(value);stackB.push(result);(isArr?forEach:forOwn)(value,function(objValue,key){result[key]=baseClone(objValue,isDeep,callback,stackA,stackB)});if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseCreate(prototype,properties){return isObject(prototype)?nativeCreate(prototype):{}; +})}function enterEdge(t,g,edge){var v=edge.v,w=edge.w;if(!g.hasEdge(v,w)){v=edge.w;w=edge.v}var vLabel=t.node(v),wLabel=t.node(w),tailLabel=vLabel,flip=false;if(vLabel.lim>wLabel.lim){tailLabel=wLabel;flip=true}var candidates=_.filter(g.edges(),function(edge){return flip===isDescendant(t,t.node(edge.v),tailLabel)&&flip!==isDescendant(t,t.node(edge.w),tailLabel)});return _.min(candidates,function(edge){return slack(g,edge)})}function exchangeEdges(t,g,e,f){var v=e.v,w=e.w;t.removeEdge(v,w);t.setEdge(f.v,f.w,{});initLowLimValues(t);initCutValues(t,g);updateRanks(t,g)}function updateRanks(t,g){var root=_.find(t.nodes(),function(v){return!g.node(v).parent}),vs=preorder(t,root);vs=vs.slice(1);_.each(vs,function(v){var parent=t.node(v).parent,edge=g.edge(v,parent),flipped=false;if(!edge){edge=g.edge(parent,v);flipped=true}g.node(v).rank=g.node(parent).rank+(flipped?edge.minlen:-edge.minlen)})}function isTreeEdge(tree,u,v){return tree.hasEdge(u,v)}function isDescendant(tree,vLabel,rootLabel){return rootLabel.low<=vLabel.lim&&vLabel.lim<=rootLabel.lim}},{"../graphlib":33,"../lodash":36,"../util":55,"./feasible-tree":51,"./util":54}],54:[function(require,module,exports){"use strict";var _=require("../lodash");module.exports={longestPath:longestPath,slack:slack};function longestPath(g){var visited={};function dfs(v){var label=g.node(v);if(_.has(visited,v)){return label.rank}visited[v]=true;var rank=_.min(_.map(g.outEdges(v),function(e){return dfs(e.w)-g.edge(e).minlen}));if(rank===Number.POSITIVE_INFINITY){rank=0}return label.rank=rank}_.each(g.sources(),dfs)}function slack(g,e){return g.node(e.w).rank-g.node(e.v).rank-g.edge(e).minlen}},{"../lodash":36}],55:[function(require,module,exports){"use strict";var _=require("./lodash"),Graph=require("./graphlib").Graph;module.exports={addDummyNode:addDummyNode,simplify:simplify,asNonCompoundGraph:asNonCompoundGraph,successorWeights:successorWeights,predecessorWeights:predecessorWeights,intersectRect:intersectRect,buildLayerMatrix:buildLayerMatrix,normalizeRanks:normalizeRanks,removeEmptyRanks:removeEmptyRanks,addBorderNode:addBorderNode,maxRank:maxRank,partition:partition,time:time,notime:notime};function addDummyNode(g,type,attrs,name){var v;do{v=_.uniqueId(name)}while(g.hasNode(v));attrs.dummy=type;g.setNode(v,attrs);return v}function simplify(g){var simplified=(new Graph).setGraph(g.graph());_.each(g.nodes(),function(v){simplified.setNode(v,g.node(v))});_.each(g.edges(),function(e){var simpleLabel=simplified.edge(e.v,e.w)||{weight:0,minlen:1},label=g.edge(e);simplified.setEdge(e.v,e.w,{weight:simpleLabel.weight+label.weight,minlen:Math.max(simpleLabel.minlen,label.minlen)})});return simplified}function asNonCompoundGraph(g){var simplified=new Graph({multigraph:g.isMultigraph()}).setGraph(g.graph());_.each(g.nodes(),function(v){if(!g.children(v).length){simplified.setNode(v,g.node(v))}});_.each(g.edges(),function(e){simplified.setEdge(e,g.edge(e))});return simplified}function successorWeights(g){var weightMap=_.map(g.nodes(),function(v){var sucs={};_.each(g.outEdges(v),function(e){sucs[e.w]=(sucs[e.w]||0)+g.edge(e).weight});return sucs});return _.zipObject(g.nodes(),weightMap)}function predecessorWeights(g){var weightMap=_.map(g.nodes(),function(v){var preds={};_.each(g.inEdges(v),function(e){preds[e.v]=(preds[e.v]||0)+g.edge(e).weight});return preds});return _.zipObject(g.nodes(),weightMap)}function intersectRect(rect,point){var x=rect.x;var y=rect.y;var dx=point.x-x;var dy=point.y-y;var w=rect.width/2;var h=rect.height/2;if(!dx&&!dy){throw new Error("Not possible to find intersection inside of the rectangle")}var sx,sy;if(Math.abs(dy)*w>Math.abs(dx)*h){if(dy<0){h=-h}sx=h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=w*dy/dx}return{x:x+sx,y:y+sy}}function buildLayerMatrix(g){var layering=_.map(_.range(maxRank(g)+1),function(){return[]});_.each(g.nodes(),function(v){var node=g.node(v),rank=node.rank;if(!_.isUndefined(rank)){layering[rank][node.order]=v}});return layering}function normalizeRanks(g){var min=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));_.each(g.nodes(),function(v){var node=g.node(v);if(_.has(node,"rank")){node.rank-=min}})}function removeEmptyRanks(g){var offset=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));var layers=[];_.each(g.nodes(),function(v){var rank=g.node(v).rank-offset;if(!_.has(layers,rank)){layers[rank]=[]}layers[rank].push(v)});var delta=0,nodeRankFactor=g.graph().nodeRankFactor;_.each(layers,function(vs,i){if(_.isUndefined(vs)&&i%nodeRankFactor!==0){--delta}else if(delta){_.each(vs,function(v){g.node(v).rank+=delta})}})}function addBorderNode(g,prefix,rank,order){var node={width:0,height:0};if(arguments.length>=4){node.rank=rank;node.order=order}return addDummyNode(g,"border",node,prefix)}function maxRank(g){return _.max(_.map(g.nodes(),function(v){var rank=g.node(v).rank;if(!_.isUndefined(rank)){return rank}}))}function partition(collection,fn){var result={lhs:[],rhs:[]};_.each(collection,function(value){if(fn(value)){result.lhs.push(value)}else{result.rhs.push(value)}});return result}function time(name,fn){var start=_.now();try{return fn()}finally{console.log(name+" time: "+(_.now()-start)+"ms")}}function notime(name,fn){return fn()}},{"./graphlib":33,"./lodash":36}],56:[function(require,module,exports){module.exports="0.7.1"},{}],57:[function(require,module,exports){var lib=require("./lib");module.exports={Graph:lib.Graph,json:require("./lib/json"),alg:require("./lib/alg"),version:lib.version}},{"./lib":73,"./lib/alg":64,"./lib/json":74}],58:[function(require,module,exports){var _=require("../lodash");module.exports=components;function components(g){var visited={},cmpts=[],cmpt;function dfs(v){if(_.has(visited,v))return;visited[v]=true;cmpt.push(v);_.each(g.successors(v),dfs);_.each(g.predecessors(v),dfs)}_.each(g.nodes(),function(v){cmpt=[];dfs(v);if(cmpt.length){cmpts.push(cmpt)}});return cmpts}},{"../lodash":75}],59:[function(require,module,exports){var _=require("../lodash");module.exports=dfs;function dfs(g,vs,order){if(!_.isArray(vs)){vs=[vs]}var acc=[],visited={};_.each(vs,function(v){if(!g.hasNode(v)){throw new Error("Graph does not have node: "+v)}doDfs(g,v,order==="post",visited,acc)});return acc}function doDfs(g,v,postorder,visited,acc){if(!_.has(visited,v)){visited[v]=true;if(!postorder){acc.push(v)}_.each(g.neighbors(v),function(w){doDfs(g,w,postorder,visited,acc)});if(postorder){acc.push(v)}}}},{"../lodash":75}],60:[function(require,module,exports){var dijkstra=require("./dijkstra"),_=require("../lodash");module.exports=dijkstraAll;function dijkstraAll(g,weightFunc,edgeFunc){return _.transform(g.nodes(),function(acc,v){acc[v]=dijkstra(g,v,weightFunc,edgeFunc)},{})}},{"../lodash":75,"./dijkstra":61}],61:[function(require,module,exports){var _=require("../lodash"),PriorityQueue=require("../data/priority-queue");module.exports=dijkstra;var DEFAULT_WEIGHT_FUNC=_.constant(1);function dijkstra(g,source,weightFn,edgeFn){return runDijkstra(g,String(source),weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runDijkstra(g,source,weightFn,edgeFn){var results={},pq=new PriorityQueue,v,vEntry;var updateNeighbors=function(edge){var w=edge.v!==v?edge.v:edge.w,wEntry=results[w],weight=weightFn(edge),distance=vEntry.distance+weight;if(weight<0){throw new Error("dijkstra does not allow negative edge weights. "+"Bad edge: "+edge+" Weight: "+weight)}if(distance0){v=pq.removeMin();vEntry=results[v];if(vEntry.distance===Number.POSITIVE_INFINITY){break}edgeFn(v).forEach(updateNeighbors)}return results}},{"../data/priority-queue":71,"../lodash":75}],62:[function(require,module,exports){var _=require("../lodash"),tarjan=require("./tarjan");module.exports=findCycles;function findCycles(g){return _.filter(tarjan(g),function(cmpt){return cmpt.length>1})}},{"../lodash":75,"./tarjan":69}],63:[function(require,module,exports){var _=require("../lodash");module.exports=floydWarshall;var DEFAULT_WEIGHT_FUNC=_.constant(1);function floydWarshall(g,weightFn,edgeFn){return runFloydWarshall(g,weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runFloydWarshall(g,weightFn,edgeFn){var results={},nodes=g.nodes();nodes.forEach(function(v){results[v]={};results[v][v]={distance:0};nodes.forEach(function(w){if(v!==w){results[v][w]={distance:Number.POSITIVE_INFINITY}}});edgeFn(v).forEach(function(edge){var w=edge.v===v?edge.w:edge.v,d=weightFn(edge);results[v][w]={distance:d,predecessor:v}})});nodes.forEach(function(k){var rowK=results[k];nodes.forEach(function(i){var rowI=results[i];nodes.forEach(function(j){var ik=rowI[k];var kj=rowK[j];var ij=rowI[j];var altDistance=ik.distance+kj.distance;if(altDistance0){v=pq.removeMin();if(_.has(parents,v)){result.setEdge(v,parents[v])}else if(init){throw new Error("Input graph is not connected: "+g)}else{init=true}g.nodeEdges(v).forEach(updateNeighbors)}return result}},{"../data/priority-queue":71,"../graph":72,"../lodash":75}],69:[function(require,module,exports){var _=require("../lodash");module.exports=tarjan;function tarjan(g){var index=0,stack=[],visited={},results=[];function dfs(v){var entry=visited[v]={onStack:true,lowlink:index,index:index++};stack.push(v);g.successors(v).forEach(function(w){if(!_.has(visited,w)){dfs(w);entry.lowlink=Math.min(entry.lowlink,visited[w].lowlink)}else if(visited[w].onStack){entry.lowlink=Math.min(entry.lowlink,visited[w].index)}});if(entry.lowlink===entry.index){var cmpt=[],w;do{w=stack.pop();visited[w].onStack=false;cmpt.push(w)}while(v!==w);results.push(cmpt)}}g.nodes().forEach(function(v){if(!_.has(visited,v)){dfs(v)}});return results}},{"../lodash":75}],70:[function(require,module,exports){var _=require("../lodash");module.exports=topsort;topsort.CycleException=CycleException;function topsort(g){var visited={},stack={},results=[];function visit(node){if(_.has(stack,node)){throw new CycleException}if(!_.has(visited,node)){stack[node]=true;visited[node]=true;_.each(g.predecessors(node),visit);delete stack[node];results.push(node)}}_.each(g.sinks(),visit);if(_.size(visited)!==g.nodeCount()){throw new CycleException}return results}function CycleException(){}},{"../lodash":75}],71:[function(require,module,exports){var _=require("../lodash");module.exports=PriorityQueue;function PriorityQueue(){this._arr=[];this._keyIndices={}}PriorityQueue.prototype.size=function(){return this._arr.length};PriorityQueue.prototype.keys=function(){return this._arr.map(function(x){return x.key})};PriorityQueue.prototype.has=function(key){return _.has(this._keyIndices,key)};PriorityQueue.prototype.priority=function(key){var index=this._keyIndices[key];if(index!==undefined){return this._arr[index].priority}};PriorityQueue.prototype.min=function(){if(this.size()===0){throw new Error("Queue underflow")}return this._arr[0].key};PriorityQueue.prototype.add=function(key,priority){var keyIndices=this._keyIndices;key=String(key);if(!_.has(keyIndices,key)){var arr=this._arr;var index=arr.length;keyIndices[key]=index;arr.push({key:key,priority:priority});this._decrease(index);return true}return false};PriorityQueue.prototype.removeMin=function(){this._swap(0,this._arr.length-1);var min=this._arr.pop();delete this._keyIndices[min.key];this._heapify(0);return min.key};PriorityQueue.prototype.decrease=function(key,priority){var index=this._keyIndices[key];if(priority>this._arr[index].priority){throw new Error("New priority is greater than current priority. "+"Key: "+key+" Old: "+this._arr[index].priority+" New: "+priority)}this._arr[index].priority=priority;this._decrease(index)};PriorityQueue.prototype._heapify=function(i){var arr=this._arr;var l=2*i,r=l+1,largest=i;if(l>1;if(arr[parent].priority1){this.setNode(v,value)}else{this.setNode(v)}},this);return this};Graph.prototype.setNode=function(v,value){if(_.has(this._nodes,v)){if(arguments.length>1){this._nodes[v]=value}return this}this._nodes[v]=arguments.length>1?value:this._defaultNodeLabelFn(v);if(this._isCompound){this._parent[v]=GRAPH_NODE;this._children[v]={};this._children[GRAPH_NODE][v]=true}this._in[v]={};this._preds[v]={};this._out[v]={};this._sucs[v]={};++this._nodeCount;return this};Graph.prototype.node=function(v){return this._nodes[v]};Graph.prototype.hasNode=function(v){return _.has(this._nodes,v)};Graph.prototype.removeNode=function(v){var self=this;if(_.has(this._nodes,v)){var removeEdge=function(e){self.removeEdge(self._edgeObjs[e])};delete this._nodes[v];if(this._isCompound){this._removeFromParentsChildList(v);delete this._parent[v];_.each(this.children(v),function(child){this.setParent(child)},this);delete this._children[v]}_.each(_.keys(this._in[v]),removeEdge);delete this._in[v];delete this._preds[v];_.each(_.keys(this._out[v]),removeEdge);delete this._out[v];delete this._sucs[v];--this._nodeCount}return this};Graph.prototype.setParent=function(v,parent){if(!this._isCompound){throw new Error("Cannot set parent in a non-compound graph")}if(_.isUndefined(parent)){parent=GRAPH_NODE}else{for(var ancestor=parent;!_.isUndefined(ancestor);ancestor=this.parent(ancestor)){if(ancestor===v){throw new Error("Setting "+parent+" as parent of "+v+" would create create a cycle")}}this.setNode(parent)}this.setNode(v);this._removeFromParentsChildList(v);this._parent[v]=parent;this._children[parent][v]=true;return this};Graph.prototype._removeFromParentsChildList=function(v){delete this._children[this._parent[v]][v]};Graph.prototype.parent=function(v){if(this._isCompound){var parent=this._parent[v];if(parent!==GRAPH_NODE){return parent}}};Graph.prototype.children=function(v){if(_.isUndefined(v)){v=GRAPH_NODE}if(this._isCompound){var children=this._children[v];if(children){return _.keys(children)}}else if(v===GRAPH_NODE){return this.nodes()}else if(this.hasNode(v)){return[]}};Graph.prototype.predecessors=function(v){var predsV=this._preds[v];if(predsV){return _.keys(predsV)}};Graph.prototype.successors=function(v){var sucsV=this._sucs[v];if(sucsV){return _.keys(sucsV)}};Graph.prototype.neighbors=function(v){var preds=this.predecessors(v);if(preds){return _.union(preds,this.successors(v))}};Graph.prototype.setDefaultEdgeLabel=function(newDefault){if(!_.isFunction(newDefault)){newDefault=_.constant(newDefault)}this._defaultEdgeLabelFn=newDefault;return this};Graph.prototype.edgeCount=function(){return this._edgeCount};Graph.prototype.edges=function(){return _.values(this._edgeObjs)};Graph.prototype.setPath=function(vs,value){var self=this,args=arguments;_.reduce(vs,function(v,w){if(args.length>1){self.setEdge(v,w,value)}else{self.setEdge(v,w)}return w});return this};Graph.prototype.setEdge=function(){var v,w,name,value,valueSpecified=false;if(_.isPlainObject(arguments[0])){v=arguments[0].v;w=arguments[0].w;name=arguments[0].name;if(arguments.length===2){value=arguments[1];valueSpecified=true}}else{v=arguments[0];w=arguments[1];name=arguments[3];if(arguments.length>2){value=arguments[2];valueSpecified=true}}v=""+v;w=""+w;if(!_.isUndefined(name)){name=""+name}var e=edgeArgsToId(this._isDirected,v,w,name);if(_.has(this._edgeLabels,e)){if(valueSpecified){this._edgeLabels[e]=value}return this}if(!_.isUndefined(name)&&!this._isMultigraph){throw new Error("Cannot set a named edge when isMultigraph = false")}this.setNode(v);this.setNode(w);this._edgeLabels[e]=valueSpecified?value:this._defaultEdgeLabelFn(v,w,name);var edgeObj=edgeArgsToObj(this._isDirected,v,w,name);v=edgeObj.v;w=edgeObj.w;Object.freeze(edgeObj);this._edgeObjs[e]=edgeObj;incrementOrInitEntry(this._preds[w],v);incrementOrInitEntry(this._sucs[v],w);this._in[w][e]=edgeObj;this._out[v][e]=edgeObj;this._edgeCount++;return this};Graph.prototype.edge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return this._edgeLabels[e]};Graph.prototype.hasEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return _.has(this._edgeLabels,e)};Graph.prototype.removeEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name),edge=this._edgeObjs[e];if(edge){v=edge.v;w=edge.w;delete this._edgeLabels[e];delete this._edgeObjs[e];decrementOrRemoveEntry(this._preds[w],v);decrementOrRemoveEntry(this._sucs[v],w);delete this._in[w][e];delete this._out[v][e];this._edgeCount--}return this};Graph.prototype.inEdges=function(v,u){var inV=this._in[v];if(inV){var edges=_.values(inV);if(!u){return edges}return _.filter(edges,function(edge){return edge.v===u})}};Graph.prototype.outEdges=function(v,w){var outV=this._out[v];if(outV){var edges=_.values(outV);if(!w){return edges}return _.filter(edges,function(edge){return edge.w===w})}};Graph.prototype.nodeEdges=function(v,w){var inEdges=this.inEdges(v,w);if(inEdges){return inEdges.concat(this.outEdges(v,w))}};function incrementOrInitEntry(map,k){if(_.has(map,k)){map[k]++}else{map[k]=1}}function decrementOrRemoveEntry(map,k){if(!--map[k]){delete map[k]}}function edgeArgsToId(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}return v+EDGE_KEY_DELIM+w+EDGE_KEY_DELIM+(_.isUndefined(name)?DEFAULT_EDGE_NAME:name)}function edgeArgsToObj(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}var edgeObj={v:v,w:w};if(name){edgeObj.name=name}return edgeObj}function edgeObjToId(isDirected,edgeObj){return edgeArgsToId(isDirected,edgeObj.v,edgeObj.w,edgeObj.name)}},{"./lodash":75}],73:[function(require,module,exports){module.exports={Graph:require("./graph"),version:require("./version")}},{"./graph":72,"./version":76}],74:[function(require,module,exports){var _=require("./lodash"),Graph=require("./graph");module.exports={write:write,read:read};function write(g){var json={options:{directed:g.isDirected(),multigraph:g.isMultigraph(),compound:g.isCompound()},nodes:writeNodes(g),edges:writeEdges(g)};if(!_.isUndefined(g.graph())){json.value=_.clone(g.graph())}return json}function writeNodes(g){return _.map(g.nodes(),function(v){var nodeValue=g.node(v),parent=g.parent(v),node={v:v};if(!_.isUndefined(nodeValue)){node.value=nodeValue}if(!_.isUndefined(parent)){node.parent=parent}return node})}function writeEdges(g){return _.map(g.edges(),function(e){var edgeValue=g.edge(e),edge={v:e.v,w:e.w};if(!_.isUndefined(e.name)){edge.name=e.name}if(!_.isUndefined(edgeValue)){edge.value=edgeValue}return edge})}function read(json){var g=new Graph(json.options).setGraph(json.value);_.each(json.nodes,function(entry){g.setNode(entry.v,entry.value);if(entry.parent){g.setParent(entry.v,entry.parent)}});_.each(json.edges,function(entry){g.setEdge({v:entry.v,w:entry.w,name:entry.name},entry.value)});return g}},{"./graph":72,"./lodash":75}],75:[function(require,module,exports){module.exports=require(20)},{"/Users/andrew/Documents/dev/dagre-d3/lib/lodash.js":20,lodash:77}],76:[function(require,module,exports){module.exports="1.0.1"},{}],77:[function(require,module,exports){(function(global){(function(){var undefined;var arrayPool=[],objectPool=[];var idCounter=0;var keyPrefix=+new Date+"";var largeArraySize=75;var maxPoolSize=40;var whitespace=" \f \ufeff"+"\n\r\u2028\u2029"+" ᠎              ";var reEmptyStringLeading=/\b__p \+= '';/g,reEmptyStringMiddle=/\b(__p \+=) '' \+/g,reEmptyStringTrailing=/(__e\(.*?\)|\b__t\)) \+\n'';/g;var reEsTemplate=/\$\{([^\\}]*(?:\\.[^\\}]*)*)\}/g;var reFlags=/\w*$/;var reFuncName=/^\s*function[ \n\r\t]+\w/;var reInterpolate=/<%=([\s\S]+?)%>/g;var reLeadingSpacesAndZeros=RegExp("^["+whitespace+"]*0+(?=.$)");var reNoMatch=/($^)/;var reThis=/\bthis\b/;var reUnescapedString=/['\n\r\t\u2028\u2029\\]/g;var contextProps=["Array","Boolean","Date","Function","Math","Number","Object","RegExp","String","_","attachEvent","clearTimeout","isFinite","isNaN","parseInt","setTimeout"];var templateCounter=0;var argsClass="[object Arguments]",arrayClass="[object Array]",boolClass="[object Boolean]",dateClass="[object Date]",funcClass="[object Function]",numberClass="[object Number]",objectClass="[object Object]",regexpClass="[object RegExp]",stringClass="[object String]";var cloneableClasses={};cloneableClasses[funcClass]=false;cloneableClasses[argsClass]=cloneableClasses[arrayClass]=cloneableClasses[boolClass]=cloneableClasses[dateClass]=cloneableClasses[numberClass]=cloneableClasses[objectClass]=cloneableClasses[regexpClass]=cloneableClasses[stringClass]=true;var debounceOptions={leading:false,maxWait:0,trailing:false};var descriptor={configurable:false,enumerable:false,value:null,writable:false};var objectTypes={"boolean":false,"function":true,object:true,number:false,string:false,undefined:false};var stringEscapes={"\\":"\\","'":"'","\n":"n","\r":"r"," ":"t","\u2028":"u2028","\u2029":"u2029"};var root=objectTypes[typeof window]&&window||this;var freeExports=objectTypes[typeof exports]&&exports&&!exports.nodeType&&exports;var freeModule=objectTypes[typeof module]&&module&&!module.nodeType&&module;var moduleExports=freeModule&&freeModule.exports===freeExports&&freeExports;var freeGlobal=objectTypes[typeof global]&&global;if(freeGlobal&&(freeGlobal.global===freeGlobal||freeGlobal.window===freeGlobal)){root=freeGlobal}function baseIndexOf(array,value,fromIndex){var index=(fromIndex||0)-1,length=array?array.length:0;while(++index-1?0:-1:cache?0:-1}function cachePush(value){var cache=this.cache,type=typeof value;if(type=="boolean"||value==null){cache[value]=true}else{if(type!="number"&&type!="string"){type="object"}var key=type=="number"?value:keyPrefix+value,typeCache=cache[type]||(cache[type]={});if(type=="object"){(typeCache[key]||(typeCache[key]=[])).push(value)}else{typeCache[key]=true}}}function charAtCallback(value){return value.charCodeAt(0)}function compareAscending(a,b){var ac=a.criteria,bc=b.criteria,index=-1,length=ac.length;while(++indexother||typeof value=="undefined"){return 1}if(value/g,evaluate:/<%([\s\S]+?)%>/g,interpolate:reInterpolate,variable:"",imports:{_:lodash}};function baseBind(bindData){var func=bindData[0],partialArgs=bindData[2],thisArg=bindData[4];function bound(){if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(this instanceof bound){var thisBinding=baseCreate(func.prototype),result=func.apply(thisBinding,args||arguments);return isObject(result)?result:thisBinding}return func.apply(thisArg,args||arguments)}setBindData(bound,bindData);return bound}function baseClone(value,isDeep,callback,stackA,stackB){if(callback){var result=callback(value);if(typeof result!="undefined"){return result}}var isObj=isObject(value);if(isObj){var className=toString.call(value);if(!cloneableClasses[className]){return value}var ctor=ctorByClass[className];switch(className){case boolClass:case dateClass:return new ctor(+value);case numberClass:case stringClass:return new ctor(value);case regexpClass:result=ctor(value.source,reFlags.exec(value));result.lastIndex=value.lastIndex;return result}}else{return value}var isArr=isArray(value);if(isDeep){var initedStack=!stackA;stackA||(stackA=getArray());stackB||(stackB=getArray());var length=stackA.length;while(length--){if(stackA[length]==value){return stackB[length]}}result=isArr?ctor(value.length):{}}else{result=isArr?slice(value):assign({},value)}if(isArr){if(hasOwnProperty.call(value,"index")){result.index=value.index}if(hasOwnProperty.call(value,"input")){result.input=value.input}}if(!isDeep){return result}stackA.push(value);stackB.push(result);(isArr?forEach:forOwn)(value,function(objValue,key){result[key]=baseClone(objValue,isDeep,callback,stackA,stackB)});if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseCreate(prototype,properties){ +return isObject(prototype)?nativeCreate(prototype):{}}if(!nativeCreate){baseCreate=function(){function Object(){}return function(prototype){if(isObject(prototype)){Object.prototype=prototype;var result=new Object;Object.prototype=null}return result||context.Object()}}()}function baseCreateCallback(func,thisArg,argCount){if(typeof func!="function"){return identity}if(typeof thisArg=="undefined"||!("prototype"in func)){return func}var bindData=func.__bindData__;if(typeof bindData=="undefined"){if(support.funcNames){bindData=!func.name}bindData=bindData||!support.funcDecomp;if(!bindData){var source=fnToString.call(func);if(!support.funcNames){bindData=!reFuncName.test(source)}if(!bindData){bindData=reThis.test(source);setBindData(func,bindData)}}}if(bindData===false||bindData!==true&&bindData[1]&1){return func}switch(argCount){case 1:return function(value){return func.call(thisArg,value)};case 2:return function(a,b){return func.call(thisArg,a,b)};case 3:return function(value,index,collection){return func.call(thisArg,value,index,collection)};case 4:return function(accumulator,value,index,collection){return func.call(thisArg,accumulator,value,index,collection)}}return bind(func,thisArg)}function baseCreateWrapper(bindData){var func=bindData[0],bitmask=bindData[1],partialArgs=bindData[2],partialRightArgs=bindData[3],thisArg=bindData[4],arity=bindData[5];var isBind=bitmask&1,isBindKey=bitmask&2,isCurry=bitmask&4,isCurryBound=bitmask&8,key=func;function bound(){var thisBinding=isBind?thisArg:this;if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(partialRightArgs||isCurry){args||(args=slice(arguments));if(partialRightArgs){push.apply(args,partialRightArgs)}if(isCurry&&args.length=largeArraySize&&indexOf===baseIndexOf,result=[];if(isLarge){var cache=createCache(values);if(cache){indexOf=cacheIndexOf;values=cache}else{isLarge=false}}while(++index-1}})}}stackA.pop();stackB.pop();if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseMerge(object,source,callback,stackA,stackB){(isArray(source)?forEach:forOwn)(source,function(source,key){var found,isArr,result=source,value=object[key];if(source&&((isArr=isArray(source))||isPlainObject(source))){var stackLength=stackA.length;while(stackLength--){if(found=stackA[stackLength]==source){value=stackB[stackLength];break}}if(!found){var isShallow;if(callback){result=callback(value,source);if(isShallow=typeof result!="undefined"){value=result}}if(!isShallow){value=isArr?isArray(value)?value:[]:isPlainObject(value)?value:{}}stackA.push(source);stackB.push(value);if(!isShallow){baseMerge(value,source,callback,stackA,stackB)}}}else{if(callback){result=callback(value,source);if(typeof result=="undefined"){result=source}}if(typeof result!="undefined"){value=result}}object[key]=value})}function baseRandom(min,max){return min+floor(nativeRandom()*(max-min+1))}function baseUniq(array,isSorted,callback){var index=-1,indexOf=getIndexOf(),length=array?array.length:0,result=[];var isLarge=!isSorted&&length>=largeArraySize&&indexOf===baseIndexOf,seen=callback||isLarge?getArray():result;if(isLarge){var cache=createCache(seen);indexOf=cacheIndexOf;seen=cache}while(++index":">",'"':""","'":"'"};var htmlUnescapes=invert(htmlEscapes);var reEscapedHtml=RegExp("("+keys(htmlUnescapes).join("|")+")","g"),reUnescapedHtml=RegExp("["+keys(htmlEscapes).join("")+"]","g");var assign=function(object,source,guard){var index,iterable=object,result=iterable;if(!iterable)return result;var args=arguments,argsIndex=0,argsLength=typeof guard=="number"?2:args.length;if(argsLength>3&&typeof args[argsLength-2]=="function"){var callback=baseCreateCallback(args[--argsLength-1],args[argsLength--],2)}else if(argsLength>2&&typeof args[argsLength-1]=="function"){callback=args[--argsLength]}while(++argsIndex3&&typeof args[length-2]=="function"){var callback=baseCreateCallback(args[--length-1],args[length--],2)}else if(length>2&&typeof args[length-1]=="function"){callback=args[--length]}var sources=slice(arguments,1,length),index=-1,stackA=getArray(),stackB=getArray();while(++index-1}else if(typeof length=="number"){result=(isString(collection)?collection.indexOf(target,fromIndex):indexOf(collection,target,fromIndex))>-1}else{forOwn(collection,function(value){if(++index>=fromIndex){return!(result=value===target)}})}return result}var countBy=createAggregator(function(result,value,key){hasOwnProperty.call(result,key)?result[key]++:result[key]=1});function every(collection,callback,thisArg){var result=true;callback=lodash.createCallback(callback,thisArg,3);var index=-1,length=collection?collection.length:0;if(typeof length=="number"){while(++indexresult){result=value}}}else{callback=callback==null&&isString(collection)?charAtCallback:lodash.createCallback(callback,thisArg,3);forEach(collection,function(value,index,collection){var current=callback(value,index,collection);if(current>computed){computed=current;result=value}})}return result}function min(collection,callback,thisArg){var computed=Infinity,result=computed;if(typeof callback!="function"&&thisArg&&thisArg[callback]===collection){callback=null}if(callback==null&&isArray(collection)){var index=-1,length=collection.length;while(++index=largeArraySize&&createCache(argsIndex?args[argsIndex]:seen))}}var array=args[0],index=-1,length=array?array.length:0,result=[];outer:while(++index>>1;callback(array[mid])=largeArraySize&&indexOf===baseIndexOf,result=[];if(isLarge){var cache=createCache(values);if(cache){indexOf=cacheIndexOf;values=cache}else{isLarge=false}}while(++index-1}})}}stackA.pop();stackB.pop();if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseMerge(object,source,callback,stackA,stackB){(isArray(source)?forEach:forOwn)(source,function(source,key){var found,isArr,result=source,value=object[key];if(source&&((isArr=isArray(source))||isPlainObject(source))){var stackLength=stackA.length;while(stackLength--){if(found=stackA[stackLength]==source){value=stackB[stackLength];break}}if(!found){var isShallow;if(callback){result=callback(value,source);if(isShallow=typeof result!="undefined"){value=result}}if(!isShallow){value=isArr?isArray(value)?value:[]:isPlainObject(value)?value:{}}stackA.push(source);stackB.push(value);if(!isShallow){baseMerge(value,source,callback,stackA,stackB)}}}else{if(callback){result=callback(value,source);if(typeof result=="undefined"){result=source}}if(typeof result!="undefined"){value=result}}object[key]=value})}function baseRandom(min,max){return min+floor(nativeRandom()*(max-min+1))}function baseUniq(array,isSorted,callback){var index=-1,indexOf=getIndexOf(),length=array?array.length:0,result=[];var isLarge=!isSorted&&length>=largeArraySize&&indexOf===baseIndexOf,seen=callback||isLarge?getArray():result;if(isLarge){var cache=createCache(seen);indexOf=cacheIndexOf;seen=cache}while(++index":">",'"':""","'":"'"};var htmlUnescapes=invert(htmlEscapes);var reEscapedHtml=RegExp("("+keys(htmlUnescapes).join("|")+")","g"),reUnescapedHtml=RegExp("["+keys(htmlEscapes).join("")+"]","g");var assign=function(object,source,guard){var index,iterable=object,result=iterable;if(!iterable)return result;var args=arguments,argsIndex=0,argsLength=typeof guard=="number"?2:args.length;if(argsLength>3&&typeof args[argsLength-2]=="function"){var callback=baseCreateCallback(args[--argsLength-1],args[argsLength--],2)}else if(argsLength>2&&typeof args[argsLength-1]=="function"){callback=args[--argsLength]}while(++argsIndex3&&typeof args[length-2]=="function"){var callback=baseCreateCallback(args[--length-1],args[length--],2)}else if(length>2&&typeof args[length-1]=="function"){callback=args[--length]}var sources=slice(arguments,1,length),index=-1,stackA=getArray(),stackB=getArray();while(++index-1}else if(typeof length=="number"){result=(isString(collection)?collection.indexOf(target,fromIndex):indexOf(collection,target,fromIndex))>-1}else{forOwn(collection,function(value){if(++index>=fromIndex){return!(result=value===target)}})}return result}var countBy=createAggregator(function(result,value,key){hasOwnProperty.call(result,key)?result[key]++:result[key]=1});function every(collection,callback,thisArg){var result=true;callback=lodash.createCallback(callback,thisArg,3);var index=-1,length=collection?collection.length:0;if(typeof length=="number"){while(++indexresult){result=value}}}else{callback=callback==null&&isString(collection)?charAtCallback:lodash.createCallback(callback,thisArg,3);forEach(collection,function(value,index,collection){var current=callback(value,index,collection);if(current>computed){computed=current;result=value}})}return result}function min(collection,callback,thisArg){var computed=Infinity,result=computed;if(typeof callback!="function"&&thisArg&&thisArg[callback]===collection){callback=null}if(callback==null&&isArray(collection)){var index=-1,length=collection.length;while(++index=largeArraySize&&createCache(argsIndex?args[argsIndex]:seen))}}var array=args[0],index=-1,length=array?array.length:0,result=[];outer:while(++index>>1;callback(array[mid])1?arguments:arguments[0],index=-1,length=array?max(pluck(array,"length")):0,result=Array(length<0?0:length);while(++index2?createWrapper(func,17,slice(arguments,2),null,thisArg):createWrapper(func,1,null,null,thisArg)}function bindAll(object){var funcs=arguments.length>1?baseFlatten(arguments,true,false,1):functions(object),index=-1,length=funcs.length;while(++index2?createWrapper(key,19,slice(arguments,2),null,object):createWrapper(key,3,null,null,object)}function compose(){var funcs=arguments,length=funcs.length;while(length--){if(!isFunction(funcs[length])){throw new TypeError}}return function(){var args=arguments,length=funcs.length;while(length--){args=[funcs[length].apply(this,args)]}return args[0]}}function curry(func,arity){arity=typeof arity=="number"?arity:+arity||func.length;return createWrapper(func,4,null,null,null,arity)}function debounce(func,wait,options){var args,maxTimeoutId,result,stamp,thisArg,timeoutId,trailingCall,lastCalled=0,maxWait=false,trailing=true;if(!isFunction(func)){throw new TypeError}wait=nativeMax(0,wait)||0;if(options===true){var leading=true;trailing=false}else if(isObject(options)){leading=options.leading;maxWait="maxWait"in options&&(nativeMax(wait,options.maxWait)||0);trailing="trailing"in options?options.trailing:trailing}var delayed=function(){var remaining=wait-(now()-stamp);if(remaining<=0){if(maxTimeoutId){clearTimeout(maxTimeoutId)}var isCalled=trailingCall;maxTimeoutId=timeoutId=trailingCall=undefined;if(isCalled){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}}else{timeoutId=setTimeout(delayed,remaining)}};var maxDelayed=function(){if(timeoutId){clearTimeout(timeoutId)}maxTimeoutId=timeoutId=trailingCall=undefined;if(trailing||maxWait!==wait){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}};return function(){args=arguments;stamp=now();thisArg=this;trailingCall=trailing&&(timeoutId||!leading);if(maxWait===false){var leadingCall=leading&&!timeoutId}else{if(!maxTimeoutId&&!leading){lastCalled=stamp}var remaining=maxWait-(stamp-lastCalled),isCalled=remaining<=0;if(isCalled){if(maxTimeoutId){maxTimeoutId=clearTimeout(maxTimeoutId)}lastCalled=stamp;result=func.apply(thisArg,args)}else if(!maxTimeoutId){maxTimeoutId=setTimeout(maxDelayed,remaining)}}if(isCalled&&timeoutId){timeoutId=clearTimeout(timeoutId)}else if(!timeoutId&&wait!==maxWait){timeoutId=setTimeout(delayed,wait)}if(leadingCall){isCalled=true;result=func.apply(thisArg,args)}if(isCalled&&!timeoutId&&!maxTimeoutId){args=thisArg=null}return result}}function defer(func){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,1);return setTimeout(function(){func.apply(undefined,args)},1)}function delay(func,wait){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,2);return setTimeout(function(){func.apply(undefined,args)},wait)}function memoize(func,resolver){if(!isFunction(func)){throw new TypeError}var memoized=function(){var cache=memoized.cache,key=resolver?resolver.apply(this,arguments):keyPrefix+arguments[0];return hasOwnProperty.call(cache,key)?cache[key]:cache[key]=func.apply(this,arguments)};memoized.cache={};return memoized}function once(func){var ran,result;if(!isFunction(func)){throw new TypeError}return function(){if(ran){return result}ran=true;result=func.apply(this,arguments);func=null;return result}}function partial(func){return createWrapper(func,16,slice(arguments,1))}function partialRight(func){return createWrapper(func,32,null,slice(arguments,1))}function throttle(func,wait,options){var leading=true,trailing=true;if(!isFunction(func)){throw new TypeError}if(options===false){leading=false}else if(isObject(options)){leading="leading"in options?options.leading:leading;trailing="trailing"in options?options.trailing:trailing}debounceOptions.leading=leading;debounceOptions.maxWait=wait;debounceOptions.trailing=trailing;return debounce(func,wait,debounceOptions)}function wrap(value,wrapper){return createWrapper(wrapper,16,[value])}function constant(value){return function(){return value}}function createCallback(func,thisArg,argCount){var type=typeof func;if(func==null||type=="function"){return baseCreateCallback(func,thisArg,argCount)}if(type!="object"){return property(func)}var props=keys(func),key=props[0],a=func[key];if(props.length==1&&a===a&&!isObject(a)){return function(object){var b=object[key];return a===b&&(a!==0||1/a==1/b)}}return function(object){var length=props.length,result=false;while(length--){if(!(result=baseIsEqual(object[props[length]],func[props[length]],null,true))){break}}return result}}function escape(string){return string==null?"":String(string).replace(reUnescapedHtml,escapeHtmlChar)}function identity(value){return value}function mixin(object,source,options){var chain=true,methodNames=source&&functions(source);if(!source||!options&&!methodNames.length){if(options==null){options=source}ctor=lodashWrapper;source=object;object=lodash;methodNames=functions(source)}if(options===false){chain=false}else if(isObject(options)&&"chain"in options){chain=options.chain}var ctor=object,isFunc=isFunction(ctor);forEach(methodNames,function(methodName){var func=object[methodName]=source[methodName];if(isFunc){ctor.prototype[methodName]=function(){var chainAll=this.__chain__,value=this.__wrapped__,args=[value];push.apply(args,arguments);var result=func.apply(object,args);if(chain||chainAll){if(value===result&&isObject(result)){return this}result=new ctor(result);result.__chain__=chainAll}return result}}})}function noConflict(){context._=oldDash;return this}function noop(){}var now=isNative(now=Date.now)&&now||function(){return(new Date).getTime()};var parseInt=nativeParseInt(whitespace+"08")==8?nativeParseInt:function(value,radix){return nativeParseInt(isString(value)?value.replace(reLeadingSpacesAndZeros,""):value,radix||0)};function property(key){return function(object){return object[key]}}function random(min,max,floating){var noMin=min==null,noMax=max==null;if(floating==null){if(typeof min=="boolean"&&noMax){floating=min;min=1}else if(!noMax&&typeof max=="boolean"){floating=max;noMax=true}}if(noMin&&noMax){max=1}min=+min||0;if(noMax){max=min;min=0}else{max=+max||0}if(floating||min%1||max%1){var rand=nativeRandom();return nativeMin(min+rand*(max-min+parseFloat("1e-"+((rand+"").length-1))),max)}return baseRandom(min,max)}function result(object,key){if(object){var value=object[key];return isFunction(value)?object[key]():value}}function template(text,data,options){var settings=lodash.templateSettings;text=String(text||"");options=defaults({},options,settings);var imports=defaults({},options.imports,settings.imports),importsKeys=keys(imports),importsValues=values(imports);var isEvaluating,index=0,interpolate=options.interpolate||reNoMatch,source="__p += '";var reDelimiters=RegExp((options.escape||reNoMatch).source+"|"+interpolate.source+"|"+(interpolate===reInterpolate?reEsTemplate:reNoMatch).source+"|"+(options.evaluate||reNoMatch).source+"|$","g");text.replace(reDelimiters,function(match,escapeValue,interpolateValue,esTemplateValue,evaluateValue,offset){interpolateValue||(interpolateValue=esTemplateValue);source+=text.slice(index,offset).replace(reUnescapedString,escapeStringChar);if(escapeValue){source+="' +\n__e("+escapeValue+") +\n'"}if(evaluateValue){isEvaluating=true;source+="';\n"+evaluateValue+";\n__p += '"}if(interpolateValue){source+="' +\n((__t = ("+interpolateValue+")) == null ? '' : __t) +\n'"}index=offset+match.length;return match});source+="';\n";var variable=options.variable,hasVariable=variable;if(!hasVariable){variable="obj";source="with ("+variable+") {\n"+source+"\n}\n"}source=(isEvaluating?source.replace(reEmptyStringLeading,""):source).replace(reEmptyStringMiddle,"$1").replace(reEmptyStringTrailing,"$1;");source="function("+variable+") {\n"+(hasVariable?"":variable+" || ("+variable+" = {});\n")+"var __t, __p = '', __e = _.escape"+(isEvaluating?", __j = Array.prototype.join;\n"+"function print() { __p += __j.call(arguments, '') }\n":";\n")+source+"return __p\n}";var sourceURL="\n/*\n//# sourceURL="+(options.sourceURL||"/lodash/template/source["+templateCounter++ +"]")+"\n*/";try{var result=Function(importsKeys,"return "+source+sourceURL).apply(undefined,importsValues)}catch(e){e.source=source;throw e}if(data){return result(data)}result.source=source;return result}function times(n,callback,thisArg){n=(n=+n)>-1?n:0;var index=-1,result=Array(n);callback=baseCreateCallback(callback,thisArg,1);while(++index1?arguments:arguments[0],index=-1,length=array?max(pluck(array,"length")):0,result=Array(length<0?0:length);while(++index2?createWrapper(func,17,slice(arguments,2),null,thisArg):createWrapper(func,1,null,null,thisArg)}function bindAll(object){var funcs=arguments.length>1?baseFlatten(arguments,true,false,1):functions(object),index=-1,length=funcs.length;while(++index2?createWrapper(key,19,slice(arguments,2),null,object):createWrapper(key,3,null,null,object)}function compose(){var funcs=arguments,length=funcs.length;while(length--){if(!isFunction(funcs[length])){throw new TypeError}}return function(){var args=arguments,length=funcs.length;while(length--){args=[funcs[length].apply(this,args)]}return args[0]}}function curry(func,arity){arity=typeof arity=="number"?arity:+arity||func.length;return createWrapper(func,4,null,null,null,arity)}function debounce(func,wait,options){var args,maxTimeoutId,result,stamp,thisArg,timeoutId,trailingCall,lastCalled=0,maxWait=false,trailing=true;if(!isFunction(func)){throw new TypeError}wait=nativeMax(0,wait)||0;if(options===true){var leading=true;trailing=false}else if(isObject(options)){leading=options.leading;maxWait="maxWait"in options&&(nativeMax(wait,options.maxWait)||0);trailing="trailing"in options?options.trailing:trailing}var delayed=function(){var remaining=wait-(now()-stamp);if(remaining<=0){if(maxTimeoutId){clearTimeout(maxTimeoutId)}var isCalled=trailingCall;maxTimeoutId=timeoutId=trailingCall=undefined;if(isCalled){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}}else{timeoutId=setTimeout(delayed,remaining)}};var maxDelayed=function(){if(timeoutId){clearTimeout(timeoutId)}maxTimeoutId=timeoutId=trailingCall=undefined;if(trailing||maxWait!==wait){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}};return function(){args=arguments;stamp=now();thisArg=this;trailingCall=trailing&&(timeoutId||!leading);if(maxWait===false){var leadingCall=leading&&!timeoutId}else{if(!maxTimeoutId&&!leading){lastCalled=stamp}var remaining=maxWait-(stamp-lastCalled),isCalled=remaining<=0;if(isCalled){if(maxTimeoutId){maxTimeoutId=clearTimeout(maxTimeoutId)}lastCalled=stamp;result=func.apply(thisArg,args)}else if(!maxTimeoutId){maxTimeoutId=setTimeout(maxDelayed,remaining)}}if(isCalled&&timeoutId){timeoutId=clearTimeout(timeoutId)}else if(!timeoutId&&wait!==maxWait){timeoutId=setTimeout(delayed,wait)}if(leadingCall){isCalled=true;result=func.apply(thisArg,args)}if(isCalled&&!timeoutId&&!maxTimeoutId){args=thisArg=null}return result}}function defer(func){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,1);return setTimeout(function(){func.apply(undefined,args)},1)}function delay(func,wait){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,2);return setTimeout(function(){func.apply(undefined,args)},wait)}function memoize(func,resolver){if(!isFunction(func)){throw new TypeError}var memoized=function(){var cache=memoized.cache,key=resolver?resolver.apply(this,arguments):keyPrefix+arguments[0];return hasOwnProperty.call(cache,key)?cache[key]:cache[key]=func.apply(this,arguments)};memoized.cache={};return memoized}function once(func){var ran,result;if(!isFunction(func)){throw new TypeError}return function(){if(ran){return result}ran=true;result=func.apply(this,arguments);func=null;return result}}function partial(func){return createWrapper(func,16,slice(arguments,1))}function partialRight(func){return createWrapper(func,32,null,slice(arguments,1))}function throttle(func,wait,options){var leading=true,trailing=true;if(!isFunction(func)){throw new TypeError}if(options===false){leading=false}else if(isObject(options)){leading="leading"in options?options.leading:leading;trailing="trailing"in options?options.trailing:trailing}debounceOptions.leading=leading;debounceOptions.maxWait=wait;debounceOptions.trailing=trailing;return debounce(func,wait,debounceOptions)}function wrap(value,wrapper){return createWrapper(wrapper,16,[value])}function constant(value){return function(){return value}}function createCallback(func,thisArg,argCount){var type=typeof func;if(func==null||type=="function"){return baseCreateCallback(func,thisArg,argCount)}if(type!="object"){return property(func)}var props=keys(func),key=props[0],a=func[key];if(props.length==1&&a===a&&!isObject(a)){return function(object){var b=object[key];return a===b&&(a!==0||1/a==1/b)}}return function(object){var length=props.length,result=false;while(length--){if(!(result=baseIsEqual(object[props[length]],func[props[length]],null,true))){break}}return result}}function escape(string){return string==null?"":String(string).replace(reUnescapedHtml,escapeHtmlChar)}function identity(value){return value}function mixin(object,source,options){var chain=true,methodNames=source&&functions(source);if(!source||!options&&!methodNames.length){if(options==null){options=source}ctor=lodashWrapper;source=object;object=lodash;methodNames=functions(source)}if(options===false){chain=false}else if(isObject(options)&&"chain"in options){chain=options.chain}var ctor=object,isFunc=isFunction(ctor);forEach(methodNames,function(methodName){var func=object[methodName]=source[methodName];if(isFunc){ctor.prototype[methodName]=function(){var chainAll=this.__chain__,value=this.__wrapped__,args=[value];push.apply(args,arguments);var result=func.apply(object,args);if(chain||chainAll){if(value===result&&isObject(result)){return this}result=new ctor(result);result.__chain__=chainAll}return result}}})}function noConflict(){context._=oldDash;return this}function noop(){}var now=isNative(now=Date.now)&&now||function(){return(new Date).getTime()};var parseInt=nativeParseInt(whitespace+"08")==8?nativeParseInt:function(value,radix){return nativeParseInt(isString(value)?value.replace(reLeadingSpacesAndZeros,""):value,radix||0)};function property(key){return function(object){return object[key]}}function random(min,max,floating){var noMin=min==null,noMax=max==null;if(floating==null){if(typeof min=="boolean"&&noMax){floating=min;min=1}else if(!noMax&&typeof max=="boolean"){floating=max;noMax=true}}if(noMin&&noMax){max=1}min=+min||0;if(noMax){max=min;min=0}else{max=+max||0}if(floating||min%1||max%1){var rand=nativeRandom();return nativeMin(min+rand*(max-min+parseFloat("1e-"+((rand+"").length-1))),max)}return baseRandom(min,max)}function result(object,key){if(object){var value=object[key];return isFunction(value)?object[key]():value}}function template(text,data,options){var settings=lodash.templateSettings;text=String(text||"");options=defaults({},options,settings);var imports=defaults({},options.imports,settings.imports),importsKeys=keys(imports),importsValues=values(imports);var isEvaluating,index=0,interpolate=options.interpolate||reNoMatch,source="__p += '";var reDelimiters=RegExp((options.escape||reNoMatch).source+"|"+interpolate.source+"|"+(interpolate===reInterpolate?reEsTemplate:reNoMatch).source+"|"+(options.evaluate||reNoMatch).source+"|$","g");text.replace(reDelimiters,function(match,escapeValue,interpolateValue,esTemplateValue,evaluateValue,offset){interpolateValue||(interpolateValue=esTemplateValue);source+=text.slice(index,offset).replace(reUnescapedString,escapeStringChar);if(escapeValue){source+="' +\n__e("+escapeValue+") +\n'"}if(evaluateValue){isEvaluating=true;source+="';\n"+evaluateValue+";\n__p += '"}if(interpolateValue){source+="' +\n((__t = ("+interpolateValue+")) == null ? '' : __t) +\n'"}index=offset+match.length;return match});source+="';\n";var variable=options.variable,hasVariable=variable;if(!hasVariable){variable="obj";source="with ("+variable+") {\n"+source+"\n}\n"}source=(isEvaluating?source.replace(reEmptyStringLeading,""):source).replace(reEmptyStringMiddle,"$1").replace(reEmptyStringTrailing,"$1;");source="function("+variable+") {\n"+(hasVariable?"":variable+" || ("+variable+" = {});\n")+"var __t, __p = '', __e = _.escape"+(isEvaluating?", __j = Array.prototype.join;\n"+"function print() { __p += __j.call(arguments, '') }\n":";\n")+source+"return __p\n}";var sourceURL="\n/*\n//# sourceURL="+(options.sourceURL||"/lodash/template/source["+templateCounter++ +"]")+"\n*/";try{var result=Function(importsKeys,"return "+source+sourceURL).apply(undefined,importsValues)}catch(e){e.source=source;throw e}if(data){return result(data)}result.source=source;return result}function times(n,callback,thisArg){n=(n=+n)>-1?n:0;var index=-1,result=Array(n);callback=baseCreateCallback(callback,thisArg,1);while(++index tr > th { + padding-left: 18px; + padding-right: 18px; +} + +table.dataTable th:active { + outline: none; +} + +/* Scrolling */ +div.dataTables_scrollHead table { + margin-bottom: 0 !important; + border-bottom-left-radius: 0; + border-bottom-right-radius: 0; +} + +div.dataTables_scrollHead table thead tr:last-child th:first-child, +div.dataTables_scrollHead table thead tr:last-child td:first-child { + border-bottom-left-radius: 0 !important; + border-bottom-right-radius: 0 !important; +} + +div.dataTables_scrollBody table { + border-top: none; + margin-top: 0 !important; + margin-bottom: 0 !important; +} + +div.dataTables_scrollBody tbody tr:first-child th, +div.dataTables_scrollBody tbody tr:first-child td { + border-top: none; +} + +div.dataTables_scrollFoot table { + margin-top: 0 !important; + border-top: none; +} + +/* Frustratingly the border-collapse:collapse used by Bootstrap makes the column + width calculations when using scrolling impossible to align columns. We have + to use separate + */ +table.table-bordered.dataTable { + border-collapse: separate !important; +} +table.table-bordered thead th, +table.table-bordered thead td { + border-left-width: 0; + border-top-width: 0; +} +table.table-bordered tbody th, +table.table-bordered tbody td { + border-left-width: 0; + border-bottom-width: 0; +} +table.table-bordered th:last-child, +table.table-bordered td:last-child { + border-right-width: 0; +} +div.dataTables_scrollHead table.table-bordered { + border-bottom-width: 0; +} + + + + +/* + * TableTools styles + */ +.table.dataTable tbody tr.active td, +.table.dataTable tbody tr.active th { + background-color: #08C; + color: white; +} + +.table.dataTable tbody tr.active:hover td, +.table.dataTable tbody tr.active:hover th { + background-color: #0075b0 !important; +} + +.table.dataTable tbody tr.active th > a, +.table.dataTable tbody tr.active td > a { + color: white; +} + +.table-striped.dataTable tbody tr.active:nth-child(odd) td, +.table-striped.dataTable tbody tr.active:nth-child(odd) th { + background-color: #017ebc; +} + +table.DTTT_selectable tbody tr { + cursor: pointer; +} + +div.DTTT .btn { + color: #333 !important; + font-size: 12px; +} + +div.DTTT .btn:hover { + text-decoration: none !important; +} + +ul.DTTT_dropdown.dropdown-menu { + z-index: 2003; +} + +ul.DTTT_dropdown.dropdown-menu a { + color: #333 !important; /* needed only when demo_page.css is included */ +} + +ul.DTTT_dropdown.dropdown-menu li { + position: relative; +} + +ul.DTTT_dropdown.dropdown-menu li:hover a { + background-color: #0088cc; + color: white !important; +} + +div.DTTT_collection_background { + z-index: 2002; +} + +/* TableTools information display */ +div.DTTT_print_info { + position: fixed; + top: 50%; + left: 50%; + width: 400px; + height: 150px; + margin-left: -200px; + margin-top: -75px; + text-align: center; + color: #333; + padding: 10px 30px; + opacity: 0.95; + + background-color: white; + border: 1px solid rgba(0, 0, 0, 0.2); + border-radius: 6px; + + -webkit-box-shadow: 0 3px 7px rgba(0, 0, 0, 0.5); + box-shadow: 0 3px 7px rgba(0, 0, 0, 0.5); +} + +div.DTTT_print_info h6 { + font-weight: normal; + font-size: 28px; + line-height: 28px; + margin: 1em; +} + +div.DTTT_print_info p { + font-size: 14px; + line-height: 20px; +} + +div.dataTables_processing { + position: absolute; + top: 50%; + left: 50%; + width: 100%; + height: 60px; + margin-left: -50%; + margin-top: -25px; + padding-top: 20px; + padding-bottom: 20px; + text-align: center; + font-size: 1.2em; + background-color: white; + background: -webkit-gradient(linear, left top, right top, color-stop(0%, rgba(255,255,255,0)), color-stop(25%, rgba(255,255,255,0.9)), color-stop(75%, rgba(255,255,255,0.9)), color-stop(100%, rgba(255,255,255,0))); + background: -webkit-linear-gradient(left, rgba(255,255,255,0) 0%, rgba(255,255,255,0.9) 25%, rgba(255,255,255,0.9) 75%, rgba(255,255,255,0) 100%); + background: -moz-linear-gradient(left, rgba(255,255,255,0) 0%, rgba(255,255,255,0.9) 25%, rgba(255,255,255,0.9) 75%, rgba(255,255,255,0) 100%); + background: -ms-linear-gradient(left, rgba(255,255,255,0) 0%, rgba(255,255,255,0.9) 25%, rgba(255,255,255,0.9) 75%, rgba(255,255,255,0) 100%); + background: -o-linear-gradient(left, rgba(255,255,255,0) 0%, rgba(255,255,255,0.9) 25%, rgba(255,255,255,0.9) 75%, rgba(255,255,255,0) 100%); + background: linear-gradient(to right, rgba(255,255,255,0) 0%, rgba(255,255,255,0.9) 25%, rgba(255,255,255,0.9) 75%, rgba(255,255,255,0) 100%); +} + + + +/* + * FixedColumns styles + */ +div.DTFC_LeftHeadWrapper table, +div.DTFC_LeftFootWrapper table, +div.DTFC_RightHeadWrapper table, +div.DTFC_RightFootWrapper table, +table.DTFC_Cloned tr.even { + background-color: white; + margin-bottom: 0; +} + +div.DTFC_RightHeadWrapper table , +div.DTFC_LeftHeadWrapper table { + border-bottom: none !important; + margin-bottom: 0 !important; + border-top-right-radius: 0 !important; + border-bottom-left-radius: 0 !important; + border-bottom-right-radius: 0 !important; +} + +div.DTFC_RightHeadWrapper table thead tr:last-child th:first-child, +div.DTFC_RightHeadWrapper table thead tr:last-child td:first-child, +div.DTFC_LeftHeadWrapper table thead tr:last-child th:first-child, +div.DTFC_LeftHeadWrapper table thead tr:last-child td:first-child { + border-bottom-left-radius: 0 !important; + border-bottom-right-radius: 0 !important; +} + +div.DTFC_RightBodyWrapper table, +div.DTFC_LeftBodyWrapper table { + border-top: none; + margin: 0 !important; +} + +div.DTFC_RightBodyWrapper tbody tr:first-child th, +div.DTFC_RightBodyWrapper tbody tr:first-child td, +div.DTFC_LeftBodyWrapper tbody tr:first-child th, +div.DTFC_LeftBodyWrapper tbody tr:first-child td { + border-top: none; +} + +div.DTFC_RightFootWrapper table, +div.DTFC_LeftFootWrapper table { + border-top: none; + margin-top: 0 !important; +} + + +/* + * FixedHeader styles + */ +div.FixedHeader_Cloned table { + margin: 0 !important +} + diff --git a/core/src/main/resources/org/apache/spark/ui/static/dataTables.bootstrap.min.js b/core/src/main/resources/org/apache/spark/ui/static/dataTables.bootstrap.min.js new file mode 100644 index 000000000000..f0d09b9d5266 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/dataTables.bootstrap.min.js @@ -0,0 +1,8 @@ +/*! + DataTables Bootstrap 3 integration + ©2011-2014 SpryMedia Ltd - datatables.net/license +*/ +(function(){var f=function(c,b){c.extend(!0,b.defaults,{dom:"<'row'<'col-sm-6'l><'col-sm-6'f>><'row'<'col-sm-12'tr>><'row'<'col-sm-6'i><'col-sm-6'p>>",renderer:"bootstrap"});c.extend(b.ext.classes,{sWrapper:"dataTables_wrapper form-inline dt-bootstrap",sFilterInput:"form-control input-sm",sLengthSelect:"form-control input-sm"});b.ext.renderer.pageButton.bootstrap=function(g,f,p,k,h,l){var q=new b.Api(g),r=g.oClasses,i=g.oLanguage.oPaginate,d,e,o=function(b,f){var j,m,n,a,k=function(a){a.preventDefault(); +c(a.currentTarget).hasClass("disabled")||q.page(a.data.action).draw(!1)};j=0;for(m=f.length;j",{"class":r.sPageButton+" "+ +e,"aria-controls":g.sTableId,tabindex:g.iTabIndex,id:0===p&&"string"===typeof a?g.sTableId+"_"+a:null}).append(c("",{href:"#"}).html(d)).appendTo(b),g.oApi._fnBindAction(n,{action:a},k))}};o(c(f).empty().html('

    ').children("ul"),k)};b.TableTools&&(c.extend(!0,b.TableTools.classes,{container:"DTTT btn-group",buttons:{normal:"btn btn-default",disabled:"disabled"},collection:{container:"DTTT_dropdown dropdown-menu",buttons:{normal:"",disabled:"disabled"}},print:{info:"DTTT_print_info"}, +select:{row:"active"}}),c.extend(!0,b.TableTools.DEFAULTS.oTags,{collection:{container:"ul",button:"li",liner:"a"}}))};"function"===typeof define&&define.amd?define(["jquery","datatables"],f):"object"===typeof exports?f(require("jquery"),require("datatables")):jQuery&&f(jQuery,jQuery.fn.dataTable)})(window,document); diff --git a/core/src/main/resources/org/apache/spark/ui/static/dataTables.rowsGroup.js b/core/src/main/resources/org/apache/spark/ui/static/dataTables.rowsGroup.js new file mode 100644 index 000000000000..983c3a564fb1 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/dataTables.rowsGroup.js @@ -0,0 +1,224 @@ +/*! RowsGroup for DataTables v1.0.0 + * 2015 Alexey Shildyakov ashl1future@gmail.com + */ + +/** + * @summary RowsGroup + * @description Group rows by specified columns + * @version 1.0.0 + * @file dataTables.rowsGroup.js + * @author Alexey Shildyakov (ashl1future@gmail.com) + * @contact ashl1future@gmail.com + * @copyright Alexey Shildyakov + * + * License MIT - http://datatables.net/license/mit + * + * This feature plug-in for DataTables automatically merges columns cells + * based on it's values equality. It supports multi-column row grouping + * in according to the requested order with dependency from each previous + * requested columns. Now it supports ordering and searching. + * Please see the example.html for details. + * + * Rows grouping in DataTables can be enabled by using any one of the following + * options: + * + * * Setting the `rowsGroup` parameter in the DataTables initialisation + * to array which contains columns selectors + * (https://datatables.net/reference/type/column-selector) used for grouping. i.e. + * rowsGroup = [1, 'columnName:name', ] + * * Setting the `rowsGroup` parameter in the DataTables defaults + * (thus causing all tables to have this feature) - i.e. + * `$.fn.dataTable.defaults.RowsGroup = [0]`. + * * Creating a new instance: `new $.fn.dataTable.RowsGroup( table, columnsForGrouping );` + * where `table` is a DataTable's API instance and `columnsForGrouping` is the array + * described above. + * + * For more detailed information please see: + * + */ + +(function($){ + +ShowedDataSelectorModifier = { + order: 'current', + page: 'current', + search: 'applied', +} + +GroupedColumnsOrderDir = 'desc'; // change + + +/* + * columnsForGrouping: array of DTAPI:cell-selector for columns for which rows grouping is applied + */ +var RowsGroup = function ( dt, columnsForGrouping ) +{ + this.table = dt.table(); + this.columnsForGrouping = columnsForGrouping; + // set to True when new reorder is applied by RowsGroup to prevent order() looping + this.orderOverrideNow = false; + this.order = [] + + self = this; + $(document).on('order.dt', function ( e, settings) { + if (!self.orderOverrideNow) { + self._updateOrderAndDraw() + } + self.orderOverrideNow = false; + }) + + $(document).on('draw.dt', function ( e, settings) { + self._mergeCells() + }) + + this._updateOrderAndDraw(); +}; + + +RowsGroup.prototype = { + _getOrderWithGroupColumns: function (order, groupedColumnsOrderDir) + { + if (groupedColumnsOrderDir === undefined) + groupedColumnsOrderDir = GroupedColumnsOrderDir + + var self = this; + var groupedColumnsIndexes = this.columnsForGrouping.map(function(columnSelector){ + return self.table.column(columnSelector).index() + }) + var groupedColumnsKnownOrder = order.filter(function(columnOrder){ + return groupedColumnsIndexes.indexOf(columnOrder[0]) >= 0 + }) + var nongroupedColumnsOrder = order.filter(function(columnOrder){ + return groupedColumnsIndexes.indexOf(columnOrder[0]) < 0 + }) + var groupedColumnsKnownOrderIndexes = groupedColumnsKnownOrder.map(function(columnOrder){ + return columnOrder[0] + }) + var groupedColumnsOrder = groupedColumnsIndexes.map(function(iColumn){ + var iInOrderIndexes = groupedColumnsKnownOrderIndexes.indexOf(iColumn) + if (iInOrderIndexes >= 0) + return [iColumn, groupedColumnsKnownOrder[iInOrderIndexes][1]] + else + return [iColumn, groupedColumnsOrderDir] + }) + + groupedColumnsOrder.push.apply(groupedColumnsOrder, nongroupedColumnsOrder) + return groupedColumnsOrder; + }, + + // Workaround: the DT reset ordering to 'desc' from multi-ordering if user order on one column (without shift) + // but because we always has multi-ordering due to grouped rows this happens every time + _getInjectedMonoSelectWorkaround: function(order) + { + if (order.length === 1) { + // got mono order - workaround here + var orderingColumn = order[0][0] + var previousOrder = this.order.map(function(val){ + return val[0] + }) + var iColumn = previousOrder.indexOf(orderingColumn); + if (iColumn >= 0) { + // assume change the direction, because we already has that in previous order + return [[orderingColumn, this._toogleDirection(this.order[iColumn][1])]] + } // else This is the new ordering column. Proceed as is. + } // else got multi order - work normal + return order; + }, + + _mergeCells: function() + { + var columnsIndexes = this.table.columns(this.columnsForGrouping, ShowedDataSelectorModifier).indexes().toArray() + var showedRowsCount = this.table.rows(ShowedDataSelectorModifier)[0].length + this._mergeColumn(0, showedRowsCount - 1, columnsIndexes) + }, + + // the index is relative to the showed data + // (selector-modifier = {order: 'current', page: 'current', search: 'applied'}) index + _mergeColumn: function(iStartRow, iFinishRow, columnsIndexes) + { + var columnsIndexesCopy = columnsIndexes.slice() + currentColumn = columnsIndexesCopy.shift() + currentColumn = this.table.column(currentColumn, ShowedDataSelectorModifier) + + var columnNodes = currentColumn.nodes() + var columnValues = currentColumn.data() + + var newSequenceRow = iStartRow, + iRow; + for (iRow = iStartRow + 1; iRow <= iFinishRow; ++iRow) { + + if (columnValues[iRow] === columnValues[newSequenceRow]) { + $(columnNodes[iRow]).hide() + } else { + $(columnNodes[newSequenceRow]).show() + $(columnNodes[newSequenceRow]).attr('rowspan', (iRow-1) - newSequenceRow + 1) + + if (columnsIndexesCopy.length > 0) + this._mergeColumn(newSequenceRow, (iRow-1), columnsIndexesCopy) + + newSequenceRow = iRow; + } + + } + $(columnNodes[newSequenceRow]).show() + $(columnNodes[newSequenceRow]).attr('rowspan', (iRow-1)- newSequenceRow + 1) + if (columnsIndexesCopy.length > 0) + this._mergeColumn(newSequenceRow, (iRow-1), columnsIndexesCopy) + }, + + _toogleDirection: function(dir) + { + return dir == 'asc'? 'desc': 'asc'; + }, + + _updateOrderAndDraw: function() + { + this.orderOverrideNow = true; + + var currentOrder = this.table.order(); + currentOrder = this._getInjectedMonoSelectWorkaround(currentOrder); + this.order = this._getOrderWithGroupColumns(currentOrder) + // this.table.order($.extend(true, Array(), this.order)) // disable this line in order to support sorting on non-grouped columns + this.table.draw(false) + }, +}; + + +$.fn.dataTable.RowsGroup = RowsGroup; +$.fn.DataTable.RowsGroup = RowsGroup; + +// Automatic initialisation listener +$(document).on( 'init.dt', function ( e, settings ) { + if ( e.namespace !== 'dt' ) { + return; + } + + var api = new $.fn.dataTable.Api( settings ); + + if ( settings.oInit.rowsGroup || + $.fn.dataTable.defaults.rowsGroup ) + { + options = settings.oInit.rowsGroup? + settings.oInit.rowsGroup: + $.fn.dataTable.defaults.rowsGroup; + new RowsGroup( api, options ); + } +} ); + +}(jQuery)); + +/* + +TODO: Provide function which determines the all s and s with "rowspan" html-attribute is parent (groupped) for the specified or . To use in selections, editing or hover styles. + +TODO: Feature +Use saved order direction for grouped columns + Split the columns into grouped and ungrouped. + + user = grouped+ungrouped + grouped = grouped + saved = grouped+ungrouped + + For grouped uses following order: user -> saved (because 'saved' include 'grouped' after first initialisation). This should be done with saving order like for 'groupedColumns' + For ungrouped: uses only 'user' input ordering +*/ \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html new file mode 100644 index 000000000000..5c91304e49fd --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -0,0 +1,126 @@ + + + diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js new file mode 100644 index 000000000000..d430d8c5fb35 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -0,0 +1,603 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +var threadDumpEnabled = false; + +function setThreadDumpEnabled(val) { + threadDumpEnabled = val; +} + +function getThreadDumpEnabled() { + return threadDumpEnabled; +} + +function formatStatus(status, type) { + if (status) { + return "Active" + } else { + return "Dead" + } +} + +jQuery.extend(jQuery.fn.dataTableExt.oSort, { + "title-numeric-pre": function (a) { + var x = a.match(/title="*(-?[0-9\.]+)/)[1]; + return parseFloat(x); + }, + + "title-numeric-asc": function (a, b) { + return ((a < b) ? -1 : ((a > b) ? 1 : 0)); + }, + + "title-numeric-desc": function (a, b) { + return ((a < b) ? 1 : ((a > b) ? -1 : 0)); + } +}); + +$(document).ajaxStop($.unblockUI); +$(document).ajaxStart(function () { + $.blockUI({message: '

    Loading Executors Page...

    '}); +}); + +function createTemplateURI(appId) { + var words = document.baseURI.split('/'); + var ind = words.indexOf("proxy"); + if (ind > 0) { + var baseURI = words.slice(0, ind + 1).join('/') + '/' + appId + '/static/executorspage-template.html'; + return baseURI; + } + ind = words.indexOf("history"); + if(ind > 0) { + var baseURI = words.slice(0, ind).join('/') + '/static/executorspage-template.html'; + return baseURI; + } + return location.origin + "/static/executorspage-template.html"; +} + +function getStandAloneppId(cb) { + var words = document.baseURI.split('/'); + var ind = words.indexOf("proxy"); + if (ind > 0) { + var appId = words[ind + 1]; + cb(appId); + return; + } + ind = words.indexOf("history"); + if (ind > 0) { + var appId = words[ind + 1]; + cb(appId); + return; + } + //Looks like Web UI is running in standalone mode + //Let's get application-id using REST End Point + $.getJSON(location.origin + "/api/v1/applications", function(response, status, jqXHR) { + if (response && response.length > 0) { + var appId = response[0].id + cb(appId); + return; + } + }); +} + +function createRESTEndPoint(appId) { + var words = document.baseURI.split('/'); + var ind = words.indexOf("proxy"); + if (ind > 0) { + var appId = words[ind + 1]; + var newBaseURI = words.slice(0, ind + 2).join('/'); + return newBaseURI + "/api/v1/applications/" + appId + "/allexecutors" + } + ind = words.indexOf("history"); + if (ind > 0) { + var appId = words[ind + 1]; + var attemptId = words[ind + 2]; + var newBaseURI = words.slice(0, ind).join('/'); + if (isNaN(attemptId)) { + return newBaseURI + "/api/v1/applications/" + appId + "/allexecutors"; + } else { + return newBaseURI + "/api/v1/applications/" + appId + "/" + attemptId + "/allexecutors"; + } + } + return location.origin + "/api/v1/applications/" + appId + "/allexecutors"; +} + +function formatLogsCells(execLogs, type) { + if (type !== 'display') return Object.keys(execLogs); + if (!execLogs) return; + var result = ''; + $.each(execLogs, function (logName, logUrl) { + result += '
    ' + }); + return result; +} + +function logsExist(execs) { + return execs.some(function(exec) { + return !($.isEmptyObject(exec["executorLogs"])); + }); +} + +// Determine Color Opacity from 0.5-1 +// activeTasks range from 0 to maxTasks +function activeTasksAlpha(activeTasks, maxTasks) { + return maxTasks > 0 ? ((activeTasks / maxTasks) * 0.5 + 0.5) : 1; +} + +function activeTasksStyle(activeTasks, maxTasks) { + return activeTasks > 0 ? ("hsla(240, 100%, 50%, " + activeTasksAlpha(activeTasks, maxTasks) + ")") : ""; +} + +// failedTasks range max at 10% failure, alpha max = 1 +function failedTasksAlpha(failedTasks, totalTasks) { + return totalTasks > 0 ? + (Math.min(10 * failedTasks / totalTasks, 1) * 0.5 + 0.5) : 1; +} + +function failedTasksStyle(failedTasks, totalTasks) { + return failedTasks > 0 ? + ("hsla(0, 100%, 50%, " + failedTasksAlpha(failedTasks, totalTasks) + ")") : ""; +} + +// totalDuration range from 0 to 50% GC time, alpha max = 1 +function totalDurationAlpha(totalGCTime, totalDuration) { + return totalDuration > 0 ? + (Math.min(totalGCTime / totalDuration + 0.5, 1)) : 1; +} + +// When GCTimePercent is edited change ToolTips.TASK_TIME to match +var GCTimePercent = 0.1; + +function totalDurationStyle(totalGCTime, totalDuration) { + // Red if GC time over GCTimePercent of total time + return (totalGCTime > GCTimePercent * totalDuration) ? + ("hsla(0, 100%, 50%, " + totalDurationAlpha(totalGCTime, totalDuration) + ")") : ""; +} + +function totalDurationColor(totalGCTime, totalDuration) { + return (totalGCTime > GCTimePercent * totalDuration) ? "white" : "black"; +} + +$(document).ready(function () { + $.extend($.fn.dataTable.defaults, { + stateSave: true, + lengthMenu: [[20, 40, 60, 100, -1], [20, 40, 60, 100, "All"]], + pageLength: 20 + }); + + executorsSummary = $("#active-executors"); + + getStandAloneppId(function (appId) { + + var endPoint = createRESTEndPoint(appId); + $.getJSON(endPoint, function (response, status, jqXHR) { + var summary = []; + var allExecCnt = 0; + var allRDDBlocks = 0; + var allMemoryUsed = 0; + var allMaxMemory = 0; + var allOnHeapMemoryUsed = 0; + var allOnHeapMaxMemory = 0; + var allOffHeapMemoryUsed = 0; + var allOffHeapMaxMemory = 0; + var allDiskUsed = 0; + var allTotalCores = 0; + var allMaxTasks = 0; + var allActiveTasks = 0; + var allFailedTasks = 0; + var allCompletedTasks = 0; + var allTotalTasks = 0; + var allTotalDuration = 0; + var allTotalGCTime = 0; + var allTotalInputBytes = 0; + var allTotalShuffleRead = 0; + var allTotalShuffleWrite = 0; + var allTotalBlacklisted = 0; + + var activeExecCnt = 0; + var activeRDDBlocks = 0; + var activeMemoryUsed = 0; + var activeMaxMemory = 0; + var activeOnHeapMemoryUsed = 0; + var activeOnHeapMaxMemory = 0; + var activeOffHeapMemoryUsed = 0; + var activeOffHeapMaxMemory = 0; + var activeDiskUsed = 0; + var activeTotalCores = 0; + var activeMaxTasks = 0; + var activeActiveTasks = 0; + var activeFailedTasks = 0; + var activeCompletedTasks = 0; + var activeTotalTasks = 0; + var activeTotalDuration = 0; + var activeTotalGCTime = 0; + var activeTotalInputBytes = 0; + var activeTotalShuffleRead = 0; + var activeTotalShuffleWrite = 0; + var activeTotalBlacklisted = 0; + + var deadExecCnt = 0; + var deadRDDBlocks = 0; + var deadMemoryUsed = 0; + var deadMaxMemory = 0; + var deadOnHeapMemoryUsed = 0; + var deadOnHeapMaxMemory = 0; + var deadOffHeapMemoryUsed = 0; + var deadOffHeapMaxMemory = 0; + var deadDiskUsed = 0; + var deadTotalCores = 0; + var deadMaxTasks = 0; + var deadActiveTasks = 0; + var deadFailedTasks = 0; + var deadCompletedTasks = 0; + var deadTotalTasks = 0; + var deadTotalDuration = 0; + var deadTotalGCTime = 0; + var deadTotalInputBytes = 0; + var deadTotalShuffleRead = 0; + var deadTotalShuffleWrite = 0; + var deadTotalBlacklisted = 0; + + response.forEach(function (exec) { + var memoryMetrics = { + usedOnHeapStorageMemory: 0, + usedOffHeapStorageMemory: 0, + totalOnHeapStorageMemory: 0, + totalOffHeapStorageMemory: 0 + }; + + exec.memoryMetrics = exec.hasOwnProperty('memoryMetrics') ? exec.memoryMetrics : memoryMetrics; + }); + + response.forEach(function (exec) { + allExecCnt += 1; + allRDDBlocks += exec.rddBlocks; + allMemoryUsed += exec.memoryUsed; + allMaxMemory += exec.maxMemory; + allOnHeapMemoryUsed += exec.memoryMetrics.usedOnHeapStorageMemory; + allOnHeapMaxMemory += exec.memoryMetrics.totalOnHeapStorageMemory; + allOffHeapMemoryUsed += exec.memoryMetrics.usedOffHeapStorageMemory; + allOffHeapMaxMemory += exec.memoryMetrics.totalOffHeapStorageMemory; + allDiskUsed += exec.diskUsed; + allTotalCores += exec.totalCores; + allMaxTasks += exec.maxTasks; + allActiveTasks += exec.activeTasks; + allFailedTasks += exec.failedTasks; + allCompletedTasks += exec.completedTasks; + allTotalTasks += exec.totalTasks; + allTotalDuration += exec.totalDuration; + allTotalGCTime += exec.totalGCTime; + allTotalInputBytes += exec.totalInputBytes; + allTotalShuffleRead += exec.totalShuffleRead; + allTotalShuffleWrite += exec.totalShuffleWrite; + allTotalBlacklisted += exec.isBlacklisted ? 1 : 0; + if (exec.isActive) { + activeExecCnt += 1; + activeRDDBlocks += exec.rddBlocks; + activeMemoryUsed += exec.memoryUsed; + activeMaxMemory += exec.maxMemory; + activeOnHeapMemoryUsed += exec.memoryMetrics.usedOnHeapStorageMemory; + activeOnHeapMaxMemory += exec.memoryMetrics.totalOnHeapStorageMemory; + activeOffHeapMemoryUsed += exec.memoryMetrics.usedOffHeapStorageMemory; + activeOffHeapMaxMemory += exec.memoryMetrics.totalOffHeapStorageMemory; + activeDiskUsed += exec.diskUsed; + activeTotalCores += exec.totalCores; + activeMaxTasks += exec.maxTasks; + activeActiveTasks += exec.activeTasks; + activeFailedTasks += exec.failedTasks; + activeCompletedTasks += exec.completedTasks; + activeTotalTasks += exec.totalTasks; + activeTotalDuration += exec.totalDuration; + activeTotalGCTime += exec.totalGCTime; + activeTotalInputBytes += exec.totalInputBytes; + activeTotalShuffleRead += exec.totalShuffleRead; + activeTotalShuffleWrite += exec.totalShuffleWrite; + activeTotalBlacklisted += exec.isBlacklisted ? 1 : 0; + } else { + deadExecCnt += 1; + deadRDDBlocks += exec.rddBlocks; + deadMemoryUsed += exec.memoryUsed; + deadMaxMemory += exec.maxMemory; + deadOnHeapMemoryUsed += exec.memoryMetrics.usedOnHeapStorageMemory; + deadOnHeapMaxMemory += exec.memoryMetrics.totalOnHeapStorageMemory; + deadOffHeapMemoryUsed += exec.memoryMetrics.usedOffHeapStorageMemory; + deadOffHeapMaxMemory += exec.memoryMetrics.totalOffHeapStorageMemory; + deadDiskUsed += exec.diskUsed; + deadTotalCores += exec.totalCores; + deadMaxTasks += exec.maxTasks; + deadActiveTasks += exec.activeTasks; + deadFailedTasks += exec.failedTasks; + deadCompletedTasks += exec.completedTasks; + deadTotalTasks += exec.totalTasks; + deadTotalDuration += exec.totalDuration; + deadTotalGCTime += exec.totalGCTime; + deadTotalInputBytes += exec.totalInputBytes; + deadTotalShuffleRead += exec.totalShuffleRead; + deadTotalShuffleWrite += exec.totalShuffleWrite; + deadTotalBlacklisted += exec.isBlacklisted ? 1 : 0; + } + }); + + var totalSummary = { + "execCnt": ( "Total(" + allExecCnt + ")"), + "allRDDBlocks": allRDDBlocks, + "allMemoryUsed": allMemoryUsed, + "allMaxMemory": allMaxMemory, + "allOnHeapMemoryUsed": allOnHeapMemoryUsed, + "allOnHeapMaxMemory": allOnHeapMaxMemory, + "allOffHeapMemoryUsed": allOffHeapMemoryUsed, + "allOffHeapMaxMemory": allOffHeapMaxMemory, + "allDiskUsed": allDiskUsed, + "allTotalCores": allTotalCores, + "allMaxTasks": allMaxTasks, + "allActiveTasks": allActiveTasks, + "allFailedTasks": allFailedTasks, + "allCompletedTasks": allCompletedTasks, + "allTotalTasks": allTotalTasks, + "allTotalDuration": allTotalDuration, + "allTotalGCTime": allTotalGCTime, + "allTotalInputBytes": allTotalInputBytes, + "allTotalShuffleRead": allTotalShuffleRead, + "allTotalShuffleWrite": allTotalShuffleWrite, + "allTotalBlacklisted": allTotalBlacklisted + }; + var activeSummary = { + "execCnt": ( "Active(" + activeExecCnt + ")"), + "allRDDBlocks": activeRDDBlocks, + "allMemoryUsed": activeMemoryUsed, + "allMaxMemory": activeMaxMemory, + "allOnHeapMemoryUsed": activeOnHeapMemoryUsed, + "allOnHeapMaxMemory": activeOnHeapMaxMemory, + "allOffHeapMemoryUsed": activeOffHeapMemoryUsed, + "allOffHeapMaxMemory": activeOffHeapMaxMemory, + "allDiskUsed": activeDiskUsed, + "allTotalCores": activeTotalCores, + "allMaxTasks": activeMaxTasks, + "allActiveTasks": activeActiveTasks, + "allFailedTasks": activeFailedTasks, + "allCompletedTasks": activeCompletedTasks, + "allTotalTasks": activeTotalTasks, + "allTotalDuration": activeTotalDuration, + "allTotalGCTime": activeTotalGCTime, + "allTotalInputBytes": activeTotalInputBytes, + "allTotalShuffleRead": activeTotalShuffleRead, + "allTotalShuffleWrite": activeTotalShuffleWrite, + "allTotalBlacklisted": activeTotalBlacklisted + }; + var deadSummary = { + "execCnt": ( "Dead(" + deadExecCnt + ")" ), + "allRDDBlocks": deadRDDBlocks, + "allMemoryUsed": deadMemoryUsed, + "allMaxMemory": deadMaxMemory, + "allOnHeapMemoryUsed": deadOnHeapMemoryUsed, + "allOnHeapMaxMemory": deadOnHeapMaxMemory, + "allOffHeapMemoryUsed": deadOffHeapMemoryUsed, + "allOffHeapMaxMemory": deadOffHeapMaxMemory, + "allDiskUsed": deadDiskUsed, + "allTotalCores": deadTotalCores, + "allMaxTasks": deadMaxTasks, + "allActiveTasks": deadActiveTasks, + "allFailedTasks": deadFailedTasks, + "allCompletedTasks": deadCompletedTasks, + "allTotalTasks": deadTotalTasks, + "allTotalDuration": deadTotalDuration, + "allTotalGCTime": deadTotalGCTime, + "allTotalInputBytes": deadTotalInputBytes, + "allTotalShuffleRead": deadTotalShuffleRead, + "allTotalShuffleWrite": deadTotalShuffleWrite, + "allTotalBlacklisted": deadTotalBlacklisted + }; + + var data = {executors: response, "execSummary": [activeSummary, deadSummary, totalSummary]}; + $.get(createTemplateURI(appId), function (template) { + + executorsSummary.append(Mustache.render($(template).filter("#executors-summary-template").html(), data)); + var selector = "#active-executors-table"; + var conf = { + "data": response, + "columns": [ + { + data: function (row, type) { + return type !== 'display' ? (isNaN(row.id) ? 0 : row.id ) : row.id; + } + }, + {data: 'hostPort'}, + {data: 'isActive', render: function (data, type, row) { + if (row.isBlacklisted) return "Blacklisted"; + else return formatStatus (data, type); + } + }, + {data: 'rddBlocks'}, + { + data: function (row, type) { + if (type !== 'display') + return row.memoryUsed; + else + return (formatBytes(row.memoryUsed, type) + ' / ' + + formatBytes(row.maxMemory, type)); + } + }, + { + data: function (row, type) { + if (type !== 'display') + return row.memoryMetrics.usedOnHeapStorageMemory; + else + return (formatBytes(row.memoryMetrics.usedOnHeapStorageMemory, type) + ' / ' + + formatBytes(row.memoryMetrics.totalOnHeapStorageMemory, type)); + }, + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + $(nTd).addClass('on_heap_memory') + } + }, + { + data: function (row, type) { + if (type !== 'display') + return row.memoryMetrics.usedOffHeapStorageMemory; + else + return (formatBytes(row.memoryMetrics.usedOffHeapStorageMemory, type) + ' / ' + + formatBytes(row.memoryMetrics.totalOffHeapStorageMemory, type)); + }, + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + $(nTd).addClass('off_heap_memory') + } + }, + {data: 'diskUsed', render: formatBytes}, + {data: 'totalCores'}, + { + data: 'activeTasks', + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + if (sData > 0) { + $(nTd).css('color', 'white'); + $(nTd).css('background', activeTasksStyle(oData.activeTasks, oData.maxTasks)); + } + } + }, + { + data: 'failedTasks', + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + if (sData > 0) { + $(nTd).css('color', 'white'); + $(nTd).css('background', failedTasksStyle(oData.failedTasks, oData.totalTasks)); + } + } + }, + {data: 'completedTasks'}, + {data: 'totalTasks'}, + { + data: function (row, type) { + return type === 'display' ? (formatDuration(row.totalDuration) + ' (' + formatDuration(row.totalGCTime) + ')') : row.totalDuration + }, + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + if (oData.totalDuration > 0) { + $(nTd).css('color', totalDurationColor(oData.totalGCTime, oData.totalDuration)); + $(nTd).css('background', totalDurationStyle(oData.totalGCTime, oData.totalDuration)); + } + } + }, + {data: 'totalInputBytes', render: formatBytes}, + {data: 'totalShuffleRead', render: formatBytes}, + {data: 'totalShuffleWrite', render: formatBytes}, + {name: 'executorLogsCol', data: 'executorLogs', render: formatLogsCells}, + { + name: 'threadDumpCol', + data: 'id', render: function (data, type) { + return type === 'display' ? ("Thread Dump" ) : data; + } + } + ], + "order": [[0, "asc"]] + }; + + var dt = $(selector).DataTable(conf); + dt.column('executorLogsCol:name').visible(logsExist(response)); + dt.column('threadDumpCol:name').visible(getThreadDumpEnabled()); + $('#active-executors [data-toggle="tooltip"]').tooltip(); + + var sumSelector = "#summary-execs-table"; + var sumConf = { + "data": [activeSummary, deadSummary, totalSummary], + "columns": [ + { + data: 'execCnt', + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + $(nTd).css('font-weight', 'bold'); + } + }, + {data: 'allRDDBlocks'}, + { + data: function (row, type) { + if (type !== 'display') + return row.allMemoryUsed + else + return (formatBytes(row.allMemoryUsed, type) + ' / ' + + formatBytes(row.allMaxMemory, type)); + } + }, + { + data: function (row, type) { + if (type !== 'display') + return row.allOnHeapMemoryUsed; + else + return (formatBytes(row.allOnHeapMemoryUsed, type) + ' / ' + + formatBytes(row.allOnHeapMaxMemory, type)); + }, + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + $(nTd).addClass('on_heap_memory') + } + }, + { + data: function (row, type) { + if (type !== 'display') + return row.allOffHeapMemoryUsed; + else + return (formatBytes(row.allOffHeapMemoryUsed, type) + ' / ' + + formatBytes(row.allOffHeapMaxMemory, type)); + }, + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + $(nTd).addClass('off_heap_memory') + } + }, + {data: 'allDiskUsed', render: formatBytes}, + {data: 'allTotalCores'}, + { + data: 'allActiveTasks', + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + if (sData > 0) { + $(nTd).css('color', 'white'); + $(nTd).css('background', activeTasksStyle(oData.allActiveTasks, oData.allMaxTasks)); + } + } + }, + { + data: 'allFailedTasks', + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + if (sData > 0) { + $(nTd).css('color', 'white'); + $(nTd).css('background', failedTasksStyle(oData.allFailedTasks, oData.allTotalTasks)); + } + } + }, + {data: 'allCompletedTasks'}, + {data: 'allTotalTasks'}, + { + data: function (row, type) { + return type === 'display' ? (formatDuration(row.allTotalDuration, type) + ' (' + formatDuration(row.allTotalGCTime, type) + ')') : row.allTotalDuration + }, + "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { + if (oData.allTotalDuration > 0) { + $(nTd).css('color', totalDurationColor(oData.allTotalGCTime, oData.allTotalDuration)); + $(nTd).css('background', totalDurationStyle(oData.allTotalGCTime, oData.allTotalDuration)); + } + } + }, + {data: 'allTotalInputBytes', render: formatBytes}, + {data: 'allTotalShuffleRead', render: formatBytes}, + {data: 'allTotalShuffleWrite', render: formatBytes}, + {data: 'allTotalBlacklisted'} + ], + "paging": false, + "searching": false, + "info": false + + }; + + $(sumSelector).DataTable(sumConf); + $('#execSummary [data-toggle="tooltip"]').tooltip(); + + }); + }); + }); +}); diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage-common.js b/core/src/main/resources/org/apache/spark/ui/static/historypage-common.js new file mode 100644 index 000000000000..55d540d8317a --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage-common.js @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +$(document).ready(function() { + if ($('#last-updated').length) { + var lastUpdatedMillis = Number($('#last-updated').text()); + var updatedDate = new Date(lastUpdatedMillis); + $('#last-updated').text(updatedDate.toLocaleDateString()+", "+updatedDate.toLocaleTimeString()) + } +}); diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html b/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html new file mode 100644 index 000000000000..18d921ab67be --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html @@ -0,0 +1,94 @@ + + + diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js new file mode 100644 index 000000000000..aa7e86037255 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +var appLimit = -1; + +function setAppLimit(val) { + appLimit = val; +} + +function makeIdNumeric(id) { + var strs = id.split("_"); + if (strs.length < 3) { + return id; + } + var appSeqNum = strs[2]; + var resl = strs[0] + "_" + strs[1] + "_"; + var diff = 10 - appSeqNum.length; + while (diff > 0) { + resl += "0"; // padding 0 before the app sequence number to make sure it has 10 characters + diff--; + } + resl += appSeqNum; + return resl; +} + +function formatDate(date) { + if (date <= 0) return "-"; + else return date.split(".")[0].replace("T", " "); +} + +function getParameterByName(name, searchString) { + var regex = new RegExp("[\\?&]" + name + "=([^&#]*)"), + results = regex.exec(searchString); + return results === null ? "" : decodeURIComponent(results[1].replace(/\+/g, " ")); +} + +function removeColumnByName(columns, columnName) { + return columns.filter(function(col) {return col.name != columnName}) +} + +function getColumnIndex(columns, columnName) { + for(var i = 0; i < columns.length; i++) { + if (columns[i].name == columnName) + return i; + } + return -1; +} + +jQuery.extend( jQuery.fn.dataTableExt.oSort, { + "title-numeric-pre": function ( a ) { + var x = a.match(/title="*(-?[0-9\.]+)/)[1]; + return parseFloat( x ); + }, + + "title-numeric-asc": function ( a, b ) { + return ((a < b) ? -1 : ((a > b) ? 1 : 0)); + }, + + "title-numeric-desc": function ( a, b ) { + return ((a < b) ? 1 : ((a > b) ? -1 : 0)); + } +} ); + +jQuery.extend( jQuery.fn.dataTableExt.oSort, { + "appid-numeric-pre": function ( a ) { + var x = a.match(/title="*(-?[0-9a-zA-Z\-\_]+)/)[1]; + return makeIdNumeric(x); + }, + + "appid-numeric-asc": function ( a, b ) { + return ((a < b) ? -1 : ((a > b) ? 1 : 0)); + }, + + "appid-numeric-desc": function ( a, b ) { + return ((a < b) ? 1 : ((a > b) ? -1 : 0)); + } +} ); + +jQuery.extend( jQuery.fn.dataTableExt.ofnSearch, { + "appid-numeric": function ( a ) { + return a.replace(/[\r\n]/g, " ").replace(/<.*?>/g, ""); + } +} ); + +$(document).ajaxStop($.unblockUI); +$(document).ajaxStart(function(){ + $.blockUI({ message: '

    Loading history summary...

    '}); +}); + +$(document).ready(function() { + $.extend( $.fn.dataTable.defaults, { + stateSave: true, + lengthMenu: [[20,40,60,100,-1], [20, 40, 60, 100, "All"]], + pageLength: 20 + }); + + historySummary = $("#history-summary"); + searchString = historySummary["context"]["location"]["search"]; + requestedIncomplete = getParameterByName("showIncomplete", searchString); + requestedIncomplete = (requestedIncomplete == "true" ? true : false); + + $.getJSON("api/v1/applications?limit=" + appLimit, function(response,status,jqXHR) { + var array = []; + var hasMultipleAttempts = false; + for (i in response) { + var app = response[i]; + if (app["attempts"][0]["completed"] == requestedIncomplete) { + continue; // if we want to show for Incomplete, we skip the completed apps; otherwise skip incomplete ones. + } + var id = app["id"]; + var name = app["name"]; + if (app["attempts"].length > 1) { + hasMultipleAttempts = true; + } + var num = app["attempts"].length; + for (j in app["attempts"]) { + var attempt = app["attempts"][j]; + attempt["startTime"] = formatDate(attempt["startTime"]); + attempt["endTime"] = formatDate(attempt["endTime"]); + attempt["lastUpdated"] = formatDate(attempt["lastUpdated"]); + attempt["log"] = uiRoot + "/api/v1/applications/" + id + "/" + + (attempt.hasOwnProperty("attemptId") ? attempt["attemptId"] + "/" : "") + "logs"; + attempt["durationMillisec"] = attempt["duration"]; + attempt["duration"] = formatDuration(attempt["duration"]); + var app_clone = {"id" : id, "name" : name, "num" : num, "attempts" : [attempt]}; + array.push(app_clone); + } + } + if(array.length < 20) { + $.fn.dataTable.defaults.paging = false; + } + + var data = { + "uiroot": uiRoot, + "applications": array, + "hasMultipleAttempts": hasMultipleAttempts, + "showCompletedColumns": !requestedIncomplete, + } + + $.get("static/historypage-template.html", function(template) { + var sibling = historySummary.prev(); + historySummary.detach(); + var apps = $(Mustache.render($(template).filter("#history-summary-template").html(),data)); + var attemptIdColumnName = 'attemptId'; + var startedColumnName = 'started'; + var defaultSortColumn = completedColumnName = 'completed'; + var durationColumnName = 'duration'; + var conf = { + "columns": [ + {name: 'appId', type: "appid-numeric"}, + {name: 'appName'}, + {name: attemptIdColumnName}, + {name: startedColumnName}, + {name: completedColumnName}, + {name: durationColumnName, type: "title-numeric"}, + {name: 'user'}, + {name: 'lastUpdated'}, + {name: 'eventLog'}, + ], + "autoWidth": false, + }; + + if (hasMultipleAttempts) { + conf.rowsGroup = [ + 'appId:name', + 'appName:name' + ]; + } else { + conf.columns = removeColumnByName(conf.columns, attemptIdColumnName); + } + + var defaultSortColumn = completedColumnName; + if (requestedIncomplete) { + defaultSortColumn = startedColumnName; + conf.columns = removeColumnByName(conf.columns, completedColumnName); + conf.columns = removeColumnByName(conf.columns, durationColumnName); + } + conf.order = [[ getColumnIndex(conf.columns, defaultSortColumn), "desc" ]]; + conf.columnDefs = [ + {"searchable": false, "targets": [getColumnIndex(conf.columns, durationColumnName)]} + ]; + historySummary.append(apps); + apps.DataTable(conf); + sibling.after(historySummary); + $('#history-summary [data-toggle="tooltip"]').tooltip(); + }); + }); +}); diff --git a/core/src/main/resources/org/apache/spark/ui/static/jquery.blockUI.min.js b/core/src/main/resources/org/apache/spark/ui/static/jquery.blockUI.min.js new file mode 100644 index 000000000000..1e84b3ec21c4 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/jquery.blockUI.min.js @@ -0,0 +1,6 @@ +/* +* jQuery BlockUI; v20131009 +* http://jquery.malsup.com/block/ +* Copyright (c) 2013 M. Alsup; Dual licensed: MIT/GPL +*/ +(function(){"use strict";function e(e){function o(o,i){var s,h,k=o==window,v=i&&void 0!==i.message?i.message:void 0;if(i=e.extend({},e.blockUI.defaults,i||{}),!i.ignoreIfBlocked||!e(o).data("blockUI.isBlocked")){if(i.overlayCSS=e.extend({},e.blockUI.defaults.overlayCSS,i.overlayCSS||{}),s=e.extend({},e.blockUI.defaults.css,i.css||{}),i.onOverlayClick&&(i.overlayCSS.cursor="pointer"),h=e.extend({},e.blockUI.defaults.themedCSS,i.themedCSS||{}),v=void 0===v?i.message:v,k&&b&&t(window,{fadeOut:0}),v&&"string"!=typeof v&&(v.parentNode||v.jquery)){var y=v.jquery?v[0]:v,m={};e(o).data("blockUI.history",m),m.el=y,m.parent=y.parentNode,m.display=y.style.display,m.position=y.style.position,m.parent&&m.parent.removeChild(y)}e(o).data("blockUI.onUnblock",i.onUnblock);var g,I,w,U,x=i.baseZ;g=r||i.forceIframe?e(''):e(''),I=i.theme?e(''):e(''),i.theme&&k?(U='"):i.theme?(U='"):U=k?'':'',w=e(U),v&&(i.theme?(w.css(h),w.addClass("ui-widget-content")):w.css(s)),i.theme||I.css(i.overlayCSS),I.css("position",k?"fixed":"absolute"),(r||i.forceIframe)&&g.css("opacity",0);var C=[g,I,w],S=k?e("body"):e(o);e.each(C,function(){this.appendTo(S)}),i.theme&&i.draggable&&e.fn.draggable&&w.draggable({handle:".ui-dialog-titlebar",cancel:"li"});var O=f&&(!e.support.boxModel||e("object,embed",k?null:o).length>0);if(u||O){if(k&&i.allowBodyStretch&&e.support.boxModel&&e("html,body").css("height","100%"),(u||!e.support.boxModel)&&!k)var E=d(o,"borderTopWidth"),T=d(o,"borderLeftWidth"),M=E?"(0 - "+E+")":0,B=T?"(0 - "+T+")":0;e.each(C,function(e,o){var t=o[0].style;if(t.position="absolute",2>e)k?t.setExpression("height","Math.max(document.body.scrollHeight, document.body.offsetHeight) - (jQuery.support.boxModel?0:"+i.quirksmodeOffsetHack+') + "px"'):t.setExpression("height",'this.parentNode.offsetHeight + "px"'),k?t.setExpression("width",'jQuery.support.boxModel && document.documentElement.clientWidth || document.body.clientWidth + "px"'):t.setExpression("width",'this.parentNode.offsetWidth + "px"'),B&&t.setExpression("left",B),M&&t.setExpression("top",M);else if(i.centerY)k&&t.setExpression("top",'(document.documentElement.clientHeight || document.body.clientHeight) / 2 - (this.offsetHeight / 2) + (blah = document.documentElement.scrollTop ? document.documentElement.scrollTop : document.body.scrollTop) + "px"'),t.marginTop=0;else if(!i.centerY&&k){var n=i.css&&i.css.top?parseInt(i.css.top,10):0,s="((document.documentElement.scrollTop ? document.documentElement.scrollTop : document.body.scrollTop) + "+n+') + "px"';t.setExpression("top",s)}})}if(v&&(i.theme?w.find(".ui-widget-content").append(v):w.append(v),(v.jquery||v.nodeType)&&e(v).show()),(r||i.forceIframe)&&i.showOverlay&&g.show(),i.fadeIn){var j=i.onBlock?i.onBlock:c,H=i.showOverlay&&!v?j:c,z=v?j:c;i.showOverlay&&I._fadeIn(i.fadeIn,H),v&&w._fadeIn(i.fadeIn,z)}else i.showOverlay&&I.show(),v&&w.show(),i.onBlock&&i.onBlock();if(n(1,o,i),k?(b=w[0],p=e(i.focusableElements,b),i.focusInput&&setTimeout(l,20)):a(w[0],i.centerX,i.centerY),i.timeout){var W=setTimeout(function(){k?e.unblockUI(i):e(o).unblock(i)},i.timeout);e(o).data("blockUI.timeout",W)}}}function t(o,t){var s,l=o==window,a=e(o),d=a.data("blockUI.history"),c=a.data("blockUI.timeout");c&&(clearTimeout(c),a.removeData("blockUI.timeout")),t=e.extend({},e.blockUI.defaults,t||{}),n(0,o,t),null===t.onUnblock&&(t.onUnblock=a.data("blockUI.onUnblock"),a.removeData("blockUI.onUnblock"));var r;r=l?e("body").children().filter(".blockUI").add("body > .blockUI"):a.find(">.blockUI"),t.cursorReset&&(r.length>1&&(r[1].style.cursor=t.cursorReset),r.length>2&&(r[2].style.cursor=t.cursorReset)),l&&(b=p=null),t.fadeOut?(s=r.length,r.stop().fadeOut(t.fadeOut,function(){0===--s&&i(r,d,t,o)})):i(r,d,t,o)}function i(o,t,i,n){var s=e(n);if(!s.data("blockUI.isBlocked")){o.each(function(){this.parentNode&&this.parentNode.removeChild(this)}),t&&t.el&&(t.el.style.display=t.display,t.el.style.position=t.position,t.parent&&t.parent.appendChild(t.el),s.removeData("blockUI.history")),s.data("blockUI.static")&&s.css("position","static"),"function"==typeof i.onUnblock&&i.onUnblock(n,i);var l=e(document.body),a=l.width(),d=l[0].style.width;l.width(a-1).width(a),l[0].style.width=d}}function n(o,t,i){var n=t==window,l=e(t);if((o||(!n||b)&&(n||l.data("blockUI.isBlocked")))&&(l.data("blockUI.isBlocked",o),n&&i.bindEvents&&(!o||i.showOverlay))){var a="mousedown mouseup keydown keypress keyup touchstart touchend touchmove";o?e(document).bind(a,i,s):e(document).unbind(a,s)}}function s(o){if("keydown"===o.type&&o.keyCode&&9==o.keyCode&&b&&o.data.constrainTabKey){var t=p,i=!o.shiftKey&&o.target===t[t.length-1],n=o.shiftKey&&o.target===t[0];if(i||n)return setTimeout(function(){l(n)},10),!1}var s=o.data,a=e(o.target);return a.hasClass("blockOverlay")&&s.onOverlayClick&&s.onOverlayClick(o),a.parents("div."+s.blockMsgClass).length>0?!0:0===a.parents().children().filter("div.blockUI").length}function l(e){if(p){var o=p[e===!0?p.length-1:0];o&&o.focus()}}function a(e,o,t){var i=e.parentNode,n=e.style,s=(i.offsetWidth-e.offsetWidth)/2-d(i,"borderLeftWidth"),l=(i.offsetHeight-e.offsetHeight)/2-d(i,"borderTopWidth");o&&(n.left=s>0?s+"px":"0"),t&&(n.top=l>0?l+"px":"0")}function d(o,t){return parseInt(e.css(o,t),10)||0}e.fn._fadeIn=e.fn.fadeIn;var c=e.noop||function(){},r=/MSIE/.test(navigator.userAgent),u=/MSIE 6.0/.test(navigator.userAgent)&&!/MSIE 8.0/.test(navigator.userAgent);document.documentMode||0;var f=e.isFunction(document.createElement("div").style.setExpression);e.blockUI=function(e){o(window,e)},e.unblockUI=function(e){t(window,e)},e.growlUI=function(o,t,i,n){var s=e('
    ');o&&s.append("

    "+o+"

    "),t&&s.append("

    "+t+"

    "),void 0===i&&(i=3e3);var l=function(o){o=o||{},e.blockUI({message:s,fadeIn:o.fadeIn!==void 0?o.fadeIn:700,fadeOut:o.fadeOut!==void 0?o.fadeOut:1e3,timeout:o.timeout!==void 0?o.timeout:i,centerY:!1,showOverlay:!1,onUnblock:n,css:e.blockUI.defaults.growlCSS})};l(),s.css("opacity"),s.mouseover(function(){l({fadeIn:0,timeout:3e4});var o=e(".blockMsg");o.stop(),o.fadeTo(300,1)}).mouseout(function(){e(".blockMsg").fadeOut(1e3)})},e.fn.block=function(t){if(this[0]===window)return e.blockUI(t),this;var i=e.extend({},e.blockUI.defaults,t||{});return this.each(function(){var o=e(this);i.ignoreIfBlocked&&o.data("blockUI.isBlocked")||o.unblock({fadeOut:0})}),this.each(function(){"static"==e.css(this,"position")&&(this.style.position="relative",e(this).data("blockUI.static",!0)),this.style.zoom=1,o(this,t)})},e.fn.unblock=function(o){return this[0]===window?(e.unblockUI(o),this):this.each(function(){t(this,o)})},e.blockUI.version=2.66,e.blockUI.defaults={message:"

    Please wait...

    ",title:null,draggable:!0,theme:!1,css:{padding:0,margin:0,width:"30%",top:"40%",left:"35%",textAlign:"center",color:"#000",border:"3px solid #aaa",backgroundColor:"#fff",cursor:"wait"},themedCSS:{width:"30%",top:"40%",left:"35%"},overlayCSS:{backgroundColor:"#000",opacity:.6,cursor:"wait"},cursorReset:"default",growlCSS:{width:"350px",top:"10px",left:"",right:"10px",border:"none",padding:"5px",opacity:.6,cursor:"default",color:"#fff",backgroundColor:"#000","-webkit-border-radius":"10px","-moz-border-radius":"10px","border-radius":"10px"},iframeSrc:/^https/i.test(window.location.href||"")?"javascript:false":"about:blank",forceIframe:!1,baseZ:1e3,centerX:!0,centerY:!0,allowBodyStretch:!0,bindEvents:!0,constrainTabKey:!0,fadeIn:200,fadeOut:400,timeout:0,showOverlay:!0,focusInput:!0,focusableElements:":input:enabled:visible",onBlock:null,onUnblock:null,onOverlayClick:null,quirksmodeOffsetHack:4,blockMsgClass:"blockMsg",ignoreIfBlocked:!1};var b=null,p=[]}"function"==typeof define&&define.amd&&define.amd.jQuery?define(["jquery"],e):e(jQuery)})(); \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/jquery.cookies.2.2.0.min.js b/core/src/main/resources/org/apache/spark/ui/static/jquery.cookies.2.2.0.min.js new file mode 100644 index 000000000000..bd2dacb4eeeb --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/jquery.cookies.2.2.0.min.js @@ -0,0 +1,18 @@ +/** + * Copyright (c) 2005 - 2010, James Auldridge + * All rights reserved. + * + * Licensed under the BSD, MIT, and GPL (your choice!) Licenses: + * http://code.google.com/p/cookies/wiki/License + * + */ +var jaaulde=window.jaaulde||{};jaaulde.utils=jaaulde.utils||{};jaaulde.utils.cookies=(function(){var resolveOptions,assembleOptionsString,parseCookies,constructor,defaultOptions={expiresAt:null,path:'/',domain:null,secure:false};resolveOptions=function(options){var returnValue,expireDate;if(typeof options!=='object'||options===null){returnValue=defaultOptions;}else +{returnValue={expiresAt:defaultOptions.expiresAt,path:defaultOptions.path,domain:defaultOptions.domain,secure:defaultOptions.secure};if(typeof options.expiresAt==='object'&&options.expiresAt instanceof Date){returnValue.expiresAt=options.expiresAt;}else if(typeof options.hoursToLive==='number'&&options.hoursToLive!==0){expireDate=new Date();expireDate.setTime(expireDate.getTime()+(options.hoursToLive*60*60*1000));returnValue.expiresAt=expireDate;}if(typeof options.path==='string'&&options.path!==''){returnValue.path=options.path;}if(typeof options.domain==='string'&&options.domain!==''){returnValue.domain=options.domain;}if(options.secure===true){returnValue.secure=options.secure;}}return returnValue;};assembleOptionsString=function(options){options=resolveOptions(options);return((typeof options.expiresAt==='object'&&options.expiresAt instanceof Date?'; expires='+options.expiresAt.toGMTString():'')+'; path='+options.path+(typeof options.domain==='string'?'; domain='+options.domain:'')+(options.secure===true?'; secure':''));};parseCookies=function(){var cookies={},i,pair,name,value,separated=document.cookie.split(';'),unparsedValue;for(i=0;i.sorting_1,table.dataTable.order-column tbody tr>.sorting_2,table.dataTable.order-column tbody tr>.sorting_3,table.dataTable.display tbody tr>.sorting_1,table.dataTable.display tbody tr>.sorting_2,table.dataTable.display tbody tr>.sorting_3{background-color:#f9f9f9}table.dataTable.order-column tbody tr.selected>.sorting_1,table.dataTable.order-column tbody tr.selected>.sorting_2,table.dataTable.order-column tbody tr.selected>.sorting_3,table.dataTable.display tbody tr.selected>.sorting_1,table.dataTable.display tbody tr.selected>.sorting_2,table.dataTable.display tbody tr.selected>.sorting_3{background-color:#acbad4}table.dataTable.display tbody tr.odd>.sorting_1,table.dataTable.order-column.stripe tbody tr.odd>.sorting_1{background-color:#f1f1f1}table.dataTable.display tbody tr.odd>.sorting_2,table.dataTable.order-column.stripe tbody tr.odd>.sorting_2{background-color:#f3f3f3}table.dataTable.display tbody tr.odd>.sorting_3,table.dataTable.order-column.stripe tbody tr.odd>.sorting_3{background-color:#f5f5f5}table.dataTable.display tbody tr.odd.selected>.sorting_1,table.dataTable.order-column.stripe tbody tr.odd.selected>.sorting_1{background-color:#a6b3cd}table.dataTable.display tbody tr.odd.selected>.sorting_2,table.dataTable.order-column.stripe tbody tr.odd.selected>.sorting_2{background-color:#a7b5ce}table.dataTable.display tbody tr.odd.selected>.sorting_3,table.dataTable.order-column.stripe tbody tr.odd.selected>.sorting_3{background-color:#a9b6d0}table.dataTable.display tbody tr.even>.sorting_1,table.dataTable.order-column.stripe tbody tr.even>.sorting_1{background-color:#f9f9f9}table.dataTable.display tbody tr.even>.sorting_2,table.dataTable.order-column.stripe tbody tr.even>.sorting_2{background-color:#fbfbfb}table.dataTable.display tbody tr.even>.sorting_3,table.dataTable.order-column.stripe tbody tr.even>.sorting_3{background-color:#fdfdfd}table.dataTable.display tbody tr.even.selected>.sorting_1,table.dataTable.order-column.stripe tbody tr.even.selected>.sorting_1{background-color:#acbad4}table.dataTable.display tbody tr.even.selected>.sorting_2,table.dataTable.order-column.stripe tbody tr.even.selected>.sorting_2{background-color:#adbbd6}table.dataTable.display tbody tr.even.selected>.sorting_3,table.dataTable.order-column.stripe tbody tr.even.selected>.sorting_3{background-color:#afbdd8}table.dataTable.display tbody tr:hover>.sorting_1,table.dataTable.display tbody tr.odd:hover>.sorting_1,table.dataTable.display tbody tr.even:hover>.sorting_1,table.dataTable.order-column.hover tbody tr:hover>.sorting_1,table.dataTable.order-column.hover tbody tr.odd:hover>.sorting_1,table.dataTable.order-column.hover tbody tr.even:hover>.sorting_1{background-color:#eaeaea}table.dataTable.display tbody tr:hover>.sorting_2,table.dataTable.display tbody tr.odd:hover>.sorting_2,table.dataTable.display tbody tr.even:hover>.sorting_2,table.dataTable.order-column.hover tbody tr:hover>.sorting_2,table.dataTable.order-column.hover tbody tr.odd:hover>.sorting_2,table.dataTable.order-column.hover tbody tr.even:hover>.sorting_2{background-color:#ebebeb}table.dataTable.display tbody tr:hover>.sorting_3,table.dataTable.display tbody tr.odd:hover>.sorting_3,table.dataTable.display tbody tr.even:hover>.sorting_3,table.dataTable.order-column.hover tbody tr:hover>.sorting_3,table.dataTable.order-column.hover tbody tr.odd:hover>.sorting_3,table.dataTable.order-column.hover tbody tr.even:hover>.sorting_3{background-color:#eee}table.dataTable.display tbody tr:hover.selected>.sorting_1,table.dataTable.display tbody tr.odd:hover.selected>.sorting_1,table.dataTable.display tbody tr.even:hover.selected>.sorting_1,table.dataTable.order-column.hover tbody tr:hover.selected>.sorting_1,table.dataTable.order-column.hover tbody tr.odd:hover.selected>.sorting_1,table.dataTable.order-column.hover tbody tr.even:hover.selected>.sorting_1{background-color:#a1aec7}table.dataTable.display tbody tr:hover.selected>.sorting_2,table.dataTable.display tbody tr.odd:hover.selected>.sorting_2,table.dataTable.display tbody tr.even:hover.selected>.sorting_2,table.dataTable.order-column.hover tbody tr:hover.selected>.sorting_2,table.dataTable.order-column.hover tbody tr.odd:hover.selected>.sorting_2,table.dataTable.order-column.hover tbody tr.even:hover.selected>.sorting_2{background-color:#a2afc8}table.dataTable.display tbody tr:hover.selected>.sorting_3,table.dataTable.display tbody tr.odd:hover.selected>.sorting_3,table.dataTable.display tbody tr.even:hover.selected>.sorting_3,table.dataTable.order-column.hover tbody tr:hover.selected>.sorting_3,table.dataTable.order-column.hover tbody tr.odd:hover.selected>.sorting_3,table.dataTable.order-column.hover tbody tr.even:hover.selected>.sorting_3{background-color:#a4b2cb}table.dataTable.no-footer{border-bottom:1px solid #111}table.dataTable.nowrap th,table.dataTable.nowrap td{white-space:nowrap}table.dataTable.compact thead th,table.dataTable.compact thead td{padding:5px 9px}table.dataTable.compact tfoot th,table.dataTable.compact tfoot td{padding:5px 9px 3px 9px}table.dataTable.compact tbody th,table.dataTable.compact tbody td{padding:4px 5px}table.dataTable th.dt-left,table.dataTable td.dt-left{text-align:left}table.dataTable th.dt-center,table.dataTable td.dt-center,table.dataTable td.dataTables_empty{text-align:center}table.dataTable th.dt-right,table.dataTable td.dt-right{text-align:right}table.dataTable th.dt-justify,table.dataTable td.dt-justify{text-align:justify}table.dataTable th.dt-nowrap,table.dataTable td.dt-nowrap{white-space:nowrap}table.dataTable thead th.dt-head-left,table.dataTable thead td.dt-head-left,table.dataTable tfoot th.dt-head-left,table.dataTable tfoot td.dt-head-left{text-align:left}table.dataTable thead th.dt-head-center,table.dataTable thead td.dt-head-center,table.dataTable tfoot th.dt-head-center,table.dataTable tfoot td.dt-head-center{text-align:center}table.dataTable thead th.dt-head-right,table.dataTable thead td.dt-head-right,table.dataTable tfoot th.dt-head-right,table.dataTable tfoot td.dt-head-right{text-align:right}table.dataTable thead th.dt-head-justify,table.dataTable thead td.dt-head-justify,table.dataTable tfoot th.dt-head-justify,table.dataTable tfoot td.dt-head-justify{text-align:justify}table.dataTable thead th.dt-head-nowrap,table.dataTable thead td.dt-head-nowrap,table.dataTable tfoot th.dt-head-nowrap,table.dataTable tfoot td.dt-head-nowrap{white-space:nowrap}table.dataTable tbody th.dt-body-left,table.dataTable tbody td.dt-body-left{text-align:left}table.dataTable tbody th.dt-body-center,table.dataTable tbody td.dt-body-center{text-align:center}table.dataTable tbody th.dt-body-right,table.dataTable tbody td.dt-body-right{text-align:right}table.dataTable tbody th.dt-body-justify,table.dataTable tbody td.dt-body-justify{text-align:justify}table.dataTable tbody th.dt-body-nowrap,table.dataTable tbody td.dt-body-nowrap{white-space:nowrap}table.dataTable,table.dataTable th,table.dataTable td{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box}.dataTables_wrapper{position:relative;clear:both;*zoom:1;zoom:1}.dataTables_wrapper .dataTables_length{float:left}.dataTables_wrapper .dataTables_filter{float:right;text-align:right}.dataTables_wrapper .dataTables_filter input{margin-left:0.5em}.dataTables_wrapper .dataTables_info{clear:both;float:left;padding-top:0.755em}.dataTables_wrapper .dataTables_paginate{float:right;text-align:right;padding-top:0.25em}.dataTables_wrapper .dataTables_paginate .paginate_button{box-sizing:border-box;display:inline-block;min-width:1.5em;padding:0.5em 1em;margin-left:2px;text-align:center;text-decoration:none !important;cursor:pointer;*cursor:hand;color:#333 !important;border:1px solid transparent}.dataTables_wrapper .dataTables_paginate .paginate_button.current,.dataTables_wrapper .dataTables_paginate .paginate_button.current:hover{color:#333 !important;border:1px solid #cacaca;background-color:#fff;background:-webkit-gradient(linear, left top, left bottom, color-stop(0%, #fff), color-stop(100%, #dcdcdc));background:-webkit-linear-gradient(top, #fff 0%, #dcdcdc 100%);background:-moz-linear-gradient(top, #fff 0%, #dcdcdc 100%);background:-ms-linear-gradient(top, #fff 0%, #dcdcdc 100%);background:-o-linear-gradient(top, #fff 0%, #dcdcdc 100%);background:linear-gradient(to bottom, #fff 0%, #dcdcdc 100%)}.dataTables_wrapper .dataTables_paginate .paginate_button.disabled,.dataTables_wrapper .dataTables_paginate .paginate_button.disabled:hover,.dataTables_wrapper .dataTables_paginate .paginate_button.disabled:active{cursor:default;color:#666 !important;border:1px solid transparent;background:transparent;box-shadow:none}.dataTables_wrapper .dataTables_paginate .paginate_button:hover{color:white !important;border:1px solid #111;background-color:#585858;background:-webkit-gradient(linear, left top, left bottom, color-stop(0%, #585858), color-stop(100%, #111));background:-webkit-linear-gradient(top, #585858 0%, #111 100%);background:-moz-linear-gradient(top, #585858 0%, #111 100%);background:-ms-linear-gradient(top, #585858 0%, #111 100%);background:-o-linear-gradient(top, #585858 0%, #111 100%);background:linear-gradient(to bottom, #585858 0%, #111 100%)}.dataTables_wrapper .dataTables_paginate .paginate_button:active{outline:none;background-color:#2b2b2b;background:-webkit-gradient(linear, left top, left bottom, color-stop(0%, #2b2b2b), color-stop(100%, #0c0c0c));background:-webkit-linear-gradient(top, #2b2b2b 0%, #0c0c0c 100%);background:-moz-linear-gradient(top, #2b2b2b 0%, #0c0c0c 100%);background:-ms-linear-gradient(top, #2b2b2b 0%, #0c0c0c 100%);background:-o-linear-gradient(top, #2b2b2b 0%, #0c0c0c 100%);background:linear-gradient(to bottom, #2b2b2b 0%, #0c0c0c 100%);box-shadow:inset 0 0 3px #111}.dataTables_wrapper .dataTables_processing{position:absolute;top:50%;left:50%;width:100%;height:40px;margin-left:-50%;margin-top:-25px;padding-top:20px;text-align:center;font-size:1.2em;background-color:white;background:-webkit-gradient(linear, left top, right top, color-stop(0%, rgba(255,255,255,0)), color-stop(25%, rgba(255,255,255,0.9)), color-stop(75%, rgba(255,255,255,0.9)), color-stop(100%, rgba(255,255,255,0)));background:-webkit-linear-gradient(left, rgba(255,255,255,0) 0%, rgba(255,255,255,0.9) 25%, rgba(255,255,255,0.9) 75%, rgba(255,255,255,0) 100%);background:-moz-linear-gradient(left, rgba(255,255,255,0) 0%, rgba(255,255,255,0.9) 25%, rgba(255,255,255,0.9) 75%, rgba(255,255,255,0) 100%);background:-ms-linear-gradient(left, rgba(255,255,255,0) 0%, rgba(255,255,255,0.9) 25%, rgba(255,255,255,0.9) 75%, rgba(255,255,255,0) 100%);background:-o-linear-gradient(left, rgba(255,255,255,0) 0%, rgba(255,255,255,0.9) 25%, rgba(255,255,255,0.9) 75%, rgba(255,255,255,0) 100%);background:linear-gradient(to right, rgba(255,255,255,0) 0%, rgba(255,255,255,0.9) 25%, rgba(255,255,255,0.9) 75%, rgba(255,255,255,0) 100%)}.dataTables_wrapper .dataTables_length,.dataTables_wrapper .dataTables_filter,.dataTables_wrapper .dataTables_info,.dataTables_wrapper .dataTables_processing,.dataTables_wrapper .dataTables_paginate{color:#333}.dataTables_wrapper .dataTables_scroll{clear:both}.dataTables_wrapper .dataTables_scroll div.dataTables_scrollBody{*margin-top:-1px;-webkit-overflow-scrolling:touch}.dataTables_wrapper .dataTables_scroll div.dataTables_scrollBody th>div.dataTables_sizing,.dataTables_wrapper .dataTables_scroll div.dataTables_scrollBody td>div.dataTables_sizing{height:0;overflow:hidden;margin:0 !important;padding:0 !important}.dataTables_wrapper.no-footer .dataTables_scrollBody{border-bottom:1px solid #111}.dataTables_wrapper.no-footer div.dataTables_scrollHead table,.dataTables_wrapper.no-footer div.dataTables_scrollBody table{border-bottom:none}.dataTables_wrapper:after{visibility:hidden;display:block;content:"";clear:both;height:0}@media screen and (max-width: 767px){.dataTables_wrapper .dataTables_info,.dataTables_wrapper .dataTables_paginate{float:none;text-align:center}.dataTables_wrapper .dataTables_paginate{margin-top:0.5em}}@media screen and (max-width: 640px){.dataTables_wrapper .dataTables_length,.dataTables_wrapper .dataTables_filter{float:none;text-align:center}.dataTables_wrapper .dataTables_filter{margin-top:0.5em}} diff --git a/core/src/main/resources/org/apache/spark/ui/static/jquery.dataTables.1.10.4.min.js b/core/src/main/resources/org/apache/spark/ui/static/jquery.dataTables.1.10.4.min.js new file mode 100644 index 000000000000..8885017c35d0 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/jquery.dataTables.1.10.4.min.js @@ -0,0 +1,157 @@ +/*! DataTables 1.10.4 + * ©2008-2014 SpryMedia Ltd - datatables.net/license + */ +(function(Da,P,l){var O=function(g){function V(a){var b,c,e={};g.each(a,function(d){if((b=d.match(/^([^A-Z]+?)([A-Z])/))&&-1!=="a aa ai ao as b fn i m o s ".indexOf(b[1]+" "))c=d.replace(b[0],b[2].toLowerCase()),e[c]=d,"o"===b[1]&&V(a[d])});a._hungarianMap=e}function G(a,b,c){a._hungarianMap||V(a);var e;g.each(b,function(d){e=a._hungarianMap[d];if(e!==l&&(c||b[e]===l))"o"===e.charAt(0)?(b[e]||(b[e]={}),g.extend(!0,b[e],b[d]),G(a[e],b[e],c)):b[e]=b[d]})}function O(a){var b=p.defaults.oLanguage,c=a.sZeroRecords; +!a.sEmptyTable&&(c&&"No data available in table"===b.sEmptyTable)&&D(a,a,"sZeroRecords","sEmptyTable");!a.sLoadingRecords&&(c&&"Loading..."===b.sLoadingRecords)&&D(a,a,"sZeroRecords","sLoadingRecords");a.sInfoThousands&&(a.sThousands=a.sInfoThousands);(a=a.sDecimal)&&cb(a)}function db(a){z(a,"ordering","bSort");z(a,"orderMulti","bSortMulti");z(a,"orderClasses","bSortClasses");z(a,"orderCellsTop","bSortCellsTop");z(a,"order","aaSorting");z(a,"orderFixed","aaSortingFixed");z(a,"paging","bPaginate"); +z(a,"pagingType","sPaginationType");z(a,"pageLength","iDisplayLength");z(a,"searching","bFilter");if(a=a.aoSearchCols)for(var b=0,c=a.length;b").css({position:"absolute",top:0,left:0,height:1,width:1,overflow:"hidden"}).append(g("
    ").css({position:"absolute",top:1,left:1,width:100, +overflow:"scroll"}).append(g('
    ').css({width:"100%",height:10}))).appendTo("body"),c=b.find(".test");a.bScrollOversize=100===c[0].offsetWidth;a.bScrollbarLeft=1!==c.offset().left;b.remove()}function gb(a,b,c,e,d,f){var h,i=!1;c!==l&&(h=c,i=!0);for(;e!==d;)a.hasOwnProperty(e)&&(h=i?b(h,a[e],e,a):a[e],i=!0,e+=f);return h}function Ea(a,b){var c=p.defaults.column,e=a.aoColumns.length,c=g.extend({},p.models.oColumn,c,{nTh:b?b:P.createElement("th"),sTitle:c.sTitle?c.sTitle:b?b.innerHTML: +"",aDataSort:c.aDataSort?c.aDataSort:[e],mData:c.mData?c.mData:e,idx:e});a.aoColumns.push(c);c=a.aoPreSearchCols;c[e]=g.extend({},p.models.oSearch,c[e]);ja(a,e,null)}function ja(a,b,c){var b=a.aoColumns[b],e=a.oClasses,d=g(b.nTh);if(!b.sWidthOrig){b.sWidthOrig=d.attr("width")||null;var f=(d.attr("style")||"").match(/width:\s*(\d+[pxem%]+)/);f&&(b.sWidthOrig=f[1])}c!==l&&null!==c&&(eb(c),G(p.defaults.column,c),c.mDataProp!==l&&!c.mData&&(c.mData=c.mDataProp),c.sType&&(b._sManualType=c.sType),c.className&& +!c.sClass&&(c.sClass=c.className),g.extend(b,c),D(b,c,"sWidth","sWidthOrig"),"number"===typeof c.iDataSort&&(b.aDataSort=[c.iDataSort]),D(b,c,"aDataSort"));var h=b.mData,i=W(h),j=b.mRender?W(b.mRender):null,c=function(a){return"string"===typeof a&&-1!==a.indexOf("@")};b._bAttrSrc=g.isPlainObject(h)&&(c(h.sort)||c(h.type)||c(h.filter));b.fnGetData=function(a,b,c){var e=i(a,b,l,c);return j&&b?j(e,b,a,c):e};b.fnSetData=function(a,b,c){return Q(h)(a,b,c)};"number"!==typeof h&&(a._rowReadObject=!0);a.oFeatures.bSort|| +(b.bSortable=!1,d.addClass(e.sSortableNone));a=-1!==g.inArray("asc",b.asSorting);c=-1!==g.inArray("desc",b.asSorting);!b.bSortable||!a&&!c?(b.sSortingClass=e.sSortableNone,b.sSortingClassJUI=""):a&&!c?(b.sSortingClass=e.sSortableAsc,b.sSortingClassJUI=e.sSortJUIAscAllowed):!a&&c?(b.sSortingClass=e.sSortableDesc,b.sSortingClassJUI=e.sSortJUIDescAllowed):(b.sSortingClass=e.sSortable,b.sSortingClassJUI=e.sSortJUI)}function X(a){if(!1!==a.oFeatures.bAutoWidth){var b=a.aoColumns;Fa(a);for(var c=0,e=b.length;c< +e;c++)b[c].nTh.style.width=b[c].sWidth}b=a.oScroll;(""!==b.sY||""!==b.sX)&&Y(a);u(a,null,"column-sizing",[a])}function ka(a,b){var c=Z(a,"bVisible");return"number"===typeof c[b]?c[b]:null}function $(a,b){var c=Z(a,"bVisible"),c=g.inArray(b,c);return-1!==c?c:null}function aa(a){return Z(a,"bVisible").length}function Z(a,b){var c=[];g.map(a.aoColumns,function(a,d){a[b]&&c.push(d)});return c}function Ga(a){var b=a.aoColumns,c=a.aoData,e=p.ext.type.detect,d,f,h,i,j,g,m,o,k;d=0;for(f=b.length;do[f])e(m.length+o[f],n);else if("string"===typeof o[f]){i=0;for(j=m.length;ib&&a[d]--; -1!=e&&c===l&& +a.splice(e,1)}function ca(a,b,c,e){var d=a.aoData[b],f,h=function(c,f){for(;c.childNodes.length;)c.removeChild(c.firstChild);c.innerHTML=v(a,b,f,"display")};if("dom"===c||(!c||"auto"===c)&&"dom"===d.src)d._aData=ma(a,d,e,e===l?l:d._aData).data;else{var i=d.anCells;if(i)if(e!==l)h(i[e],e);else{c=0;for(f=i.length;c").appendTo(h));b=0;for(c= +m.length;btr").attr("role","row");g(h).find(">tr>th, >tr>td").addClass(n.sHeaderTH);g(i).find(">tr>th, >tr>td").addClass(n.sFooterTH);if(null!==i){a=a.aoFooter[0];b=0;for(c=a.length;b=a.fnRecordsDisplay()?0:h,a.iInitDisplayStart=-1);var h=a._iDisplayStart,n=a.fnDisplayEnd();if(a.bDeferLoading)a.bDeferLoading= +!1,a.iDraw++,B(a,!1);else if(i){if(!a.bDestroying&&!jb(a))return}else a.iDraw++;if(0!==j.length){f=i?a.aoData.length:n;for(i=i?0:h;i",{"class":d? +e[0]:""}).append(g("",{valign:"top",colSpan:aa(a),"class":a.oClasses.sRowEmpty}).html(c))[0];u(a,"aoHeaderCallback","header",[g(a.nTHead).children("tr")[0],Ka(a),h,n,j]);u(a,"aoFooterCallback","footer",[g(a.nTFoot).children("tr")[0],Ka(a),h,n,j]);e=g(a.nTBody);e.children().detach();e.append(g(b));u(a,"aoDrawCallback","draw",[a]);a.bSorted=!1;a.bFiltered=!1;a.bDrawing=!1}}function M(a,b){var c=a.oFeatures,e=c.bFilter;c.bSort&&kb(a);e?fa(a,a.oPreviousSearch):a.aiDisplay=a.aiDisplayMaster.slice(); +!0!==b&&(a._iDisplayStart=0);a._drawHold=b;L(a);a._drawHold=!1}function lb(a){var b=a.oClasses,c=g(a.nTable),c=g("
    ").insertBefore(c),e=a.oFeatures,d=g("
    ",{id:a.sTableId+"_wrapper","class":b.sWrapper+(a.nTFoot?"":" "+b.sNoFooter)});a.nHolding=c[0];a.nTableWrapper=d[0];a.nTableReinsertBefore=a.nTable.nextSibling;for(var f=a.sDom.split(""),h,i,j,n,m,o,k=0;k")[0];n=f[k+1];if("'"==n||'"'==n){m="";for(o=2;f[k+o]!=n;)m+=f[k+o],o++;"H"==m?m=b.sJUIHeader: +"F"==m&&(m=b.sJUIFooter);-1!=m.indexOf(".")?(n=m.split("."),j.id=n[0].substr(1,n[0].length-1),j.className=n[1]):"#"==m.charAt(0)?j.id=m.substr(1,m.length-1):j.className=m;k+=o}d.append(j);d=g(j)}else if(">"==i)d=d.parent();else if("l"==i&&e.bPaginate&&e.bLengthChange)h=mb(a);else if("f"==i&&e.bFilter)h=nb(a);else if("r"==i&&e.bProcessing)h=ob(a);else if("t"==i)h=pb(a);else if("i"==i&&e.bInfo)h=qb(a);else if("p"==i&&e.bPaginate)h=rb(a);else if(0!==p.ext.feature.length){j=p.ext.feature;o=0;for(n=j.length;o< +n;o++)if(i==j[o].cFeature){h=j[o].fnInit(a);break}}h&&(j=a.aanFeatures,j[i]||(j[i]=[]),j[i].push(h),d.append(h))}c.replaceWith(d)}function da(a,b){var c=g(b).children("tr"),e,d,f,h,i,j,n,m,o,k;a.splice(0,a.length);f=0;for(j=c.length;f',i=e.sSearch,i=i.match(/_INPUT_/)?i.replace("_INPUT_",h):i+h,b=g("
    ",{id:!f.f?c+"_filter":null,"class":b.sFilter}).append(g("
    ").addClass(b.sLength);a.aanFeatures.l||(j[0].id=c+"_length");j.children().append(a.oLanguage.sLengthMenu.replace("_MENU_",d[0].outerHTML));g("select",j).val(a._iDisplayLength).bind("change.DT", +function(){Qa(a,g(this).val());L(a)});g(a.nTable).bind("length.dt.DT",function(b,c,f){a===c&&g("select",j).val(f)});return j[0]}function rb(a){var b=a.sPaginationType,c=p.ext.pager[b],e="function"===typeof c,d=function(a){L(a)},b=g("
    ").addClass(a.oClasses.sPaging+b)[0],f=a.aanFeatures;e||c.fnInit(a,b,d);f.p||(b.id=a.sTableId+"_paginate",a.aoDrawCallback.push({fn:function(a){if(e){var b=a._iDisplayStart,g=a._iDisplayLength,n=a.fnRecordsDisplay(),m=-1===g,b=m?0:Math.ceil(b/g),g=m?1:Math.ceil(n/ +g),n=c(b,g),o,m=0;for(o=f.p.length;mf&&(e=0)):"first"==b?e=0:"previous"==b?(e=0<=d?e-d:0,0>e&&(e=0)):"next"==b?e+d",{id:!a.aanFeatures.r?a.sTableId+"_processing":null,"class":a.oClasses.sProcessing}).html(a.oLanguage.sProcessing).insertBefore(a.nTable)[0]}function B(a,b){a.oFeatures.bProcessing&&g(a.aanFeatures.r).css("display",b?"block":"none");u(a,null,"processing",[a,b])}function pb(a){var b=g(a.nTable);b.attr("role","grid");var c=a.oScroll;if(""===c.sX&&""===c.sY)return a.nTable;var e=c.sX,d=c.sY,f=a.oClasses,h=b.children("caption"),i=h.length?h[0]._captionSide:null, +j=g(b[0].cloneNode(!1)),n=g(b[0].cloneNode(!1)),m=b.children("tfoot");c.sX&&"100%"===b.attr("width")&&b.removeAttr("width");m.length||(m=null);c=g("
    ",{"class":f.sScrollWrapper}).append(g("
    ",{"class":f.sScrollHead}).css({overflow:"hidden",position:"relative",border:0,width:e?!e?null:s(e):"100%"}).append(g("
    ",{"class":f.sScrollHeadInner}).css({"box-sizing":"content-box",width:c.sXInner||"100%"}).append(j.removeAttr("id").css("margin-left",0).append("top"===i?h:null).append(b.children("thead"))))).append(g("
    ", +{"class":f.sScrollBody}).css({overflow:"auto",height:!d?null:s(d),width:!e?null:s(e)}).append(b));m&&c.append(g("
    ",{"class":f.sScrollFoot}).css({overflow:"hidden",border:0,width:e?!e?null:s(e):"100%"}).append(g("
    ",{"class":f.sScrollFootInner}).append(n.removeAttr("id").css("margin-left",0).append("bottom"===i?h:null).append(b.children("tfoot")))));var b=c.children(),o=b[0],f=b[1],k=m?b[2]:null;e&&g(f).scroll(function(){var a=this.scrollLeft;o.scrollLeft=a;m&&(k.scrollLeft=a)});a.nScrollHead= +o;a.nScrollBody=f;a.nScrollFoot=k;a.aoDrawCallback.push({fn:Y,sName:"scrolling"});return c[0]}function Y(a){var b=a.oScroll,c=b.sX,e=b.sXInner,d=b.sY,f=b.iBarWidth,h=g(a.nScrollHead),i=h[0].style,j=h.children("div"),n=j[0].style,m=j.children("table"),j=a.nScrollBody,o=g(j),k=j.style,l=g(a.nScrollFoot).children("div"),p=l.children("table"),r=g(a.nTHead),q=g(a.nTable),t=q[0],N=t.style,J=a.nTFoot?g(a.nTFoot):null,u=a.oBrowser,w=u.bScrollOversize,y,v,x,K,z,A=[],B=[],C=[],D,E=function(a){a=a.style;a.paddingTop= +"0";a.paddingBottom="0";a.borderTopWidth="0";a.borderBottomWidth="0";a.height=0};q.children("thead, tfoot").remove();z=r.clone().prependTo(q);y=r.find("tr");x=z.find("tr");z.find("th, td").removeAttr("tabindex");J&&(K=J.clone().prependTo(q),v=J.find("tr"),K=K.find("tr"));c||(k.width="100%",h[0].style.width="100%");g.each(pa(a,z),function(b,c){D=ka(a,b);c.style.width=a.aoColumns[D].sWidth});J&&F(function(a){a.style.width=""},K);b.bCollapse&&""!==d&&(k.height=o[0].offsetHeight+r[0].offsetHeight+"px"); +h=q.outerWidth();if(""===c){if(N.width="100%",w&&(q.find("tbody").height()>j.offsetHeight||"scroll"==o.css("overflow-y")))N.width=s(q.outerWidth()-f)}else""!==e?N.width=s(e):h==o.width()&&o.height()h-f&&(N.width=s(h))):N.width=s(h);h=q.outerWidth();F(E,x);F(function(a){C.push(a.innerHTML);A.push(s(g(a).css("width")))},x);F(function(a,b){a.style.width=A[b]},y);g(x).height(0);J&&(F(E,K),F(function(a){B.push(s(g(a).css("width")))},K),F(function(a,b){a.style.width= +B[b]},v),g(K).height(0));F(function(a,b){a.innerHTML='
    '+C[b]+"
    ";a.style.width=A[b]},x);J&&F(function(a,b){a.innerHTML="";a.style.width=B[b]},K);if(q.outerWidth()j.offsetHeight||"scroll"==o.css("overflow-y")?h+f:h;if(w&&(j.scrollHeight>j.offsetHeight||"scroll"==o.css("overflow-y")))N.width=s(v-f);(""===c||""!==e)&&R(a,1,"Possible column misalignment",6)}else v="100%";k.width=s(v);i.width=s(v);J&&(a.nScrollFoot.style.width= +s(v));!d&&w&&(k.height=s(t.offsetHeight+f));d&&b.bCollapse&&(k.height=s(d),b=c&&t.offsetWidth>j.offsetWidth?f:0,t.offsetHeightj.clientHeight||"scroll"==o.css("overflow-y");u="padding"+(u.bScrollbarLeft?"Left":"Right");n[u]=m?f+"px":"0px";J&&(p[0].style.width=s(b),l[0].style.width=s(b),l[0].style[u]=m?f+"px":"0px");o.scroll();if((a.bSorted||a.bFiltered)&&!a._drawHold)j.scrollTop=0}function F(a, +b,c){for(var e=0,d=0,f=b.length,h,g;d"));i.find("tfoot th, tfoot td").css("width","");var p=i.find("tbody tr"),j=pa(a,i.find("thead")[0]);for(k=0;k").css("width",s(a)).appendTo(b||P.body),e=c[0].offsetWidth;c.remove();return e}function Eb(a,b){var c=a.oScroll;if(c.sX||c.sY)c=!c.sX?c.iBarWidth:0,b.style.width=s(g(b).outerWidth()-c)}function Db(a,b){var c=Fb(a,b);if(0>c)return null; +var e=a.aoData[c];return!e.nTr?g("").html(v(a,c,b,"display"))[0]:e.anCells[b]}function Fb(a,b){for(var c,e=-1,d=-1,f=0,h=a.aoData.length;fe&&(e=c.length,d=f);return d}function s(a){return null===a?"0px":"number"==typeof a?0>a?"0px":a+"px":a.match(/\d$/)?a+"px":a}function Gb(){if(!p.__scrollbarWidth){var a=g("

    ").css({width:"100%",height:200,padding:0})[0],b=g("

    ").css({position:"absolute",top:0,left:0,width:200,height:150,padding:0, +overflow:"hidden",visibility:"hidden"}).append(a).appendTo("body"),c=a.offsetWidth;b.css("overflow","scroll");a=a.offsetWidth;c===a&&(a=b[0].clientWidth);b.remove();p.__scrollbarWidth=c-a}return p.__scrollbarWidth}function T(a){var b,c,e=[],d=a.aoColumns,f,h,i,j;b=a.aaSortingFixed;c=g.isPlainObject(b);var n=[];f=function(a){a.length&&!g.isArray(a[0])?n.push(a):n.push.apply(n,a)};g.isArray(b)&&f(b);c&&b.pre&&f(b.pre);f(a.aaSorting);c&&b.post&&f(b.post);for(a=0;ad?1:0,0!==c)return"asc"===g.dir?c:-c;c=e[a];d=e[b];return cd?1:0}):j.sort(function(a,b){var c,h,g,i,j=n.length,l=f[a]._aSortData,p=f[b]._aSortData;for(g=0;gh?1:0})}a.bSorted=!0}function Ib(a){for(var b,c,e=a.aoColumns,d=T(a),a=a.oLanguage.oAria,f=0,h=e.length;f/g,"");var j=c.nTh;j.removeAttribute("aria-sort");c.bSortable&&(0d?d+1:3));d=0;for(f=e.length;dd?d+1:3))}a.aLastSort=e}function Hb(a,b){var c=a.aoColumns[b],e=p.ext.order[c.sSortDataType],d;e&&(d=e.call(a.oInstance,a,b,$(a,b)));for(var f,h=p.ext.type.order[c.sType+"-pre"],g=0,j=a.aoData.length;g< +j;g++)if(c=a.aoData[g],c._aSortData||(c._aSortData=[]),!c._aSortData[b]||e)f=e?d[g]:v(a,g,b,"sort"),c._aSortData[b]=h?h(f):f}function xa(a){if(a.oFeatures.bStateSave&&!a.bDestroying){var b={time:+new Date,start:a._iDisplayStart,length:a._iDisplayLength,order:g.extend(!0,[],a.aaSorting),search:yb(a.oPreviousSearch),columns:g.map(a.aoColumns,function(b,e){return{visible:b.bVisible,search:yb(a.aoPreSearchCols[e])}})};u(a,"aoStateSaveParams","stateSaveParams",[a,b]);a.oSavedState=b;a.fnStateSaveCallback.call(a.oInstance, +a,b)}}function Jb(a){var b,c,e=a.aoColumns;if(a.oFeatures.bStateSave){var d=a.fnStateLoadCallback.call(a.oInstance,a);if(d&&d.time&&(b=u(a,"aoStateLoadParams","stateLoadParams",[a,d]),-1===g.inArray(!1,b)&&(b=a.iStateDuration,!(0=e.length?[0,c[1]]:c)});g.extend(a.oPreviousSearch, +zb(d.search));b=0;for(c=d.columns.length;b=c&&(b=c-e);b-=b%e;if(-1===e||0>b)b=0;a._iDisplayStart=b}function Oa(a,b){var c=a.renderer,e=p.ext.renderer[b];return g.isPlainObject(c)&& +c[b]?e[c[b]]||e._:"string"===typeof c?e[c]||e._:e._}function A(a){return a.oFeatures.bServerSide?"ssp":a.ajax||a.sAjaxSource?"ajax":"dom"}function Va(a,b){var c=[],c=Lb.numbers_length,e=Math.floor(c/2);b<=c?c=U(0,b):a<=e?(c=U(0,c-2),c.push("ellipsis"),c.push(b-1)):(a>=b-1-e?c=U(b-(c-2),b):(c=U(a-1,a+2),c.push("ellipsis"),c.push(b-1)),c.splice(0,0,"ellipsis"),c.splice(0,0,0));c.DT_el="span";return c}function cb(a){g.each({num:function(b){return za(b,a)},"num-fmt":function(b){return za(b,a,Wa)},"html-num":function(b){return za(b, +a,Aa)},"html-num-fmt":function(b){return za(b,a,Aa,Wa)}},function(b,c){w.type.order[b+a+"-pre"]=c;b.match(/^html\-/)&&(w.type.search[b+a]=w.type.search.html)})}function Mb(a){return function(){var b=[ya(this[p.ext.iApiIndex])].concat(Array.prototype.slice.call(arguments));return p.ext.internal[a].apply(this,b)}}var p,w,q,r,t,Xa={},Nb=/[\r\n]/g,Aa=/<.*?>/g,$b=/^[\w\+\-]/,ac=/[\w\+\-]$/,Xb=RegExp("(\\/|\\.|\\*|\\+|\\?|\\||\\(|\\)|\\[|\\]|\\{|\\}|\\\\|\\$|\\^|\\-)","g"),Wa=/[',$\u00a3\u20ac\u00a5%\u2009\u202F]/g, +H=function(a){return!a||!0===a||"-"===a?!0:!1},Ob=function(a){var b=parseInt(a,10);return!isNaN(b)&&isFinite(a)?b:null},Pb=function(a,b){Xa[b]||(Xa[b]=RegExp(ua(b),"g"));return"string"===typeof a&&"."!==b?a.replace(/\./g,"").replace(Xa[b],"."):a},Ya=function(a,b,c){var e="string"===typeof a;b&&e&&(a=Pb(a,b));c&&e&&(a=a.replace(Wa,""));return H(a)||!isNaN(parseFloat(a))&&isFinite(a)},Qb=function(a,b,c){return H(a)?!0:!(H(a)||"string"===typeof a)?null:Ya(a.replace(Aa,""),b,c)?!0:null},C=function(a, +b,c){var e=[],d=0,f=a.length;if(c!==l)for(;d")[0],Yb=va.textContent!==l,Zb=/<.*?>/g;p=function(a){this.$=function(a,b){return this.api(!0).$(a,b)};this._=function(a,b){return this.api(!0).rows(a,b).data()};this.api=function(a){return a?new q(ya(this[w.iApiIndex])):new q(this)};this.fnAddData=function(a,b){var c=this.api(!0),e=g.isArray(a)&&(g.isArray(a[0])||g.isPlainObject(a[0]))? +c.rows.add(a):c.row.add(a);(b===l||b)&&c.draw();return e.flatten().toArray()};this.fnAdjustColumnSizing=function(a){var b=this.api(!0).columns.adjust(),c=b.settings()[0],e=c.oScroll;a===l||a?b.draw(!1):(""!==e.sX||""!==e.sY)&&Y(c)};this.fnClearTable=function(a){var b=this.api(!0).clear();(a===l||a)&&b.draw()};this.fnClose=function(a){this.api(!0).row(a).child.hide()};this.fnDeleteRow=function(a,b,c){var e=this.api(!0),a=e.rows(a),d=a.settings()[0],g=d.aoData[a[0][0]];a.remove();b&&b.call(this,d,g); +(c===l||c)&&e.draw();return g};this.fnDestroy=function(a){this.api(!0).destroy(a)};this.fnDraw=function(a){this.api(!0).draw(!a)};this.fnFilter=function(a,b,c,e,d,g){d=this.api(!0);null===b||b===l?d.search(a,c,e,g):d.column(b).search(a,c,e,g);d.draw()};this.fnGetData=function(a,b){var c=this.api(!0);if(a!==l){var e=a.nodeName?a.nodeName.toLowerCase():"";return b!==l||"td"==e||"th"==e?c.cell(a,b).data():c.row(a).data()||null}return c.data().toArray()};this.fnGetNodes=function(a){var b=this.api(!0); +return a!==l?b.row(a).node():b.rows().nodes().flatten().toArray()};this.fnGetPosition=function(a){var b=this.api(!0),c=a.nodeName.toUpperCase();return"TR"==c?b.row(a).index():"TD"==c||"TH"==c?(a=b.cell(a).index(),[a.row,a.columnVisible,a.column]):null};this.fnIsOpen=function(a){return this.api(!0).row(a).child.isShown()};this.fnOpen=function(a,b,c){return this.api(!0).row(a).child(b,c).show().child()[0]};this.fnPageChange=function(a,b){var c=this.api(!0).page(a);(b===l||b)&&c.draw(!1)};this.fnSetColumnVis= +function(a,b,c){a=this.api(!0).column(a).visible(b);(c===l||c)&&a.columns.adjust().draw()};this.fnSettings=function(){return ya(this[w.iApiIndex])};this.fnSort=function(a){this.api(!0).order(a).draw()};this.fnSortListener=function(a,b,c){this.api(!0).order.listener(a,b,c)};this.fnUpdate=function(a,b,c,e,d){var g=this.api(!0);c===l||null===c?g.row(b).data(a):g.cell(b,c).data(a);(d===l||d)&&g.columns.adjust();(e===l||e)&&g.draw();return 0};this.fnVersionCheck=w.fnVersionCheck;var b=this,c=a===l,e=this.length; +c&&(a={});this.oApi=this.internal=w.internal;for(var d in p.ext.internal)d&&(this[d]=Mb(d));this.each(function(){var d={},d=1t<"F"ip>'),k.renderer)? +g.isPlainObject(k.renderer)&&!k.renderer.header&&(k.renderer.header="jqueryui"):k.renderer="jqueryui":g.extend(j,p.ext.classes,d.oClasses);g(this).addClass(j.sTable);if(""!==k.oScroll.sX||""!==k.oScroll.sY)k.oScroll.iBarWidth=Gb();!0===k.oScroll.sX&&(k.oScroll.sX="100%");k.iInitDisplayStart===l&&(k.iInitDisplayStart=d.iDisplayStart,k._iDisplayStart=d.iDisplayStart);null!==d.iDeferLoading&&(k.bDeferLoading=!0,h=g.isArray(d.iDeferLoading),k._iRecordsDisplay=h?d.iDeferLoading[0]:d.iDeferLoading,k._iRecordsTotal= +h?d.iDeferLoading[1]:d.iDeferLoading);var r=k.oLanguage;g.extend(!0,r,d.oLanguage);""!==r.sUrl&&(g.ajax({dataType:"json",url:r.sUrl,success:function(a){O(a);G(m.oLanguage,a);g.extend(true,r,a);ga(k)},error:function(){ga(k)}}),n=!0);null===d.asStripeClasses&&(k.asStripeClasses=[j.sStripeOdd,j.sStripeEven]);var h=k.asStripeClasses,q=g("tbody tr:eq(0)",this);-1!==g.inArray(!0,g.map(h,function(a){return q.hasClass(a)}))&&(g("tbody tr",this).removeClass(h.join(" ")),k.asDestroyStripes=h.slice());var o= +[],s,h=this.getElementsByTagName("thead");0!==h.length&&(da(k.aoHeader,h[0]),o=pa(k));if(null===d.aoColumns){s=[];h=0;for(i=o.length;h").appendTo(this));k.nTHead=i[0];i=g(this).children("tbody");0===i.length&&(i=g("").appendTo(this));k.nTBody=i[0];i=g(this).children("tfoot");if(0===i.length&&0").appendTo(this);0===i.length||0===i.children().length?g(this).addClass(j.sNoFooter): +0a?new q(b[a],this[a]):null},filter:function(a){var b=[];if(y.filter)b=y.filter.call(this,a,this);else for(var c=0,e=this.length;c").addClass(b);g("td",c).addClass(b).html(a)[0].colSpan=aa(e);d.push(c[0])}};if(g.isArray(a)||a instanceof g)for(var h=0,i=a.length;h=0?b:h.length+b];if(typeof a==="function"){var d=Ba(c,f);return g.map(h,function(b,f){return a(f,Vb(c,f,0,0,d),j[f])?f:null})}var k=typeof a==="string"?a.match(cc):"";if(k)switch(k[2]){case "visIdx":case "visible":b= +parseInt(k[1],10);if(b<0){var l=g.map(h,function(a,b){return a.bVisible?b:null});return[l[l.length+b]]}return[ka(c,b)];case "name":return g.map(i,function(a,b){return a===k[1]?b:null})}else return g(j).filter(a).map(function(){return g.inArray(this,j)}).toArray()})},1);c.selector.cols=a;c.selector.opts=b;return c});t("columns().header()","column().header()",function(){return this.iterator("column",function(a,b){return a.aoColumns[b].nTh},1)});t("columns().footer()","column().footer()",function(){return this.iterator("column", +function(a,b){return a.aoColumns[b].nTf},1)});t("columns().data()","column().data()",function(){return this.iterator("column-rows",Vb,1)});t("columns().dataSrc()","column().dataSrc()",function(){return this.iterator("column",function(a,b){return a.aoColumns[b].mData},1)});t("columns().cache()","column().cache()",function(a){return this.iterator("column-rows",function(b,c,e,d,f){return ha(b.aoData,f,"search"===a?"_aFilterData":"_aSortData",c)},1)});t("columns().nodes()","column().nodes()",function(){return this.iterator("column-rows", +function(a,b,c,e,d){return ha(a.aoData,d,"anCells",b)},1)});t("columns().visible()","column().visible()",function(a,b){return this.iterator("column",function(c,e){if(a===l)return c.aoColumns[e].bVisible;var d=c.aoColumns,f=d[e],h=c.aoData,i,j,n;if(a!==l&&f.bVisible!==a){if(a){var m=g.inArray(!0,C(d,"bVisible"),e+1);i=0;for(j=h.length;ie;return!0};p.isDataTable=p.fnIsDataTable=function(a){var b=g(a).get(0),c=!1;g.each(p.settings,function(a,d){if(d.nTable===b||d.nScrollHead===b||d.nScrollFoot===b)c=!0});return c};p.tables=p.fnTables=function(a){return g.map(p.settings,function(b){if(!a||a&&g(b.nTable).is(":visible"))return b.nTable})};p.util={throttle:ta,escapeRegex:ua}; +p.camelToHungarian=G;r("$()",function(a,b){var c=this.rows(b).nodes(),c=g(c);return g([].concat(c.filter(a).toArray(),c.find(a).toArray()))});g.each(["on","one","off"],function(a,b){r(b+"()",function(){var a=Array.prototype.slice.call(arguments);a[0].match(/\.dt\b/)||(a[0]+=".dt");var e=g(this.tables().nodes());e[b].apply(e,a);return this})});r("clear()",function(){return this.iterator("table",function(a){na(a)})});r("settings()",function(){return new q(this.context,this.context)});r("data()",function(){return this.iterator("table", +function(a){return C(a.aoData,"_aData")}).flatten()});r("destroy()",function(a){a=a||!1;return this.iterator("table",function(b){var c=b.nTableWrapper.parentNode,e=b.oClasses,d=b.nTable,f=b.nTBody,h=b.nTHead,i=b.nTFoot,j=g(d),f=g(f),l=g(b.nTableWrapper),m=g.map(b.aoData,function(a){return a.nTr}),o;b.bDestroying=!0;u(b,"aoDestroyCallback","destroy",[b]);a||(new q(b)).columns().visible(!0);l.unbind(".DT").find(":not(tbody *)").unbind(".DT");g(Da).unbind(".DT-"+b.sInstance);d!=h.parentNode&&(j.children("thead").detach(), +j.append(h));i&&d!=i.parentNode&&(j.children("tfoot").detach(),j.append(i));j.detach();l.detach();b.aaSorting=[];b.aaSortingFixed=[];wa(b);g(m).removeClass(b.asStripeClasses.join(" "));g("th, td",h).removeClass(e.sSortable+" "+e.sSortableAsc+" "+e.sSortableDesc+" "+e.sSortableNone);b.bJUI&&(g("th span."+e.sSortIcon+", td span."+e.sSortIcon,h).detach(),g("th, td",h).each(function(){var a=g("div."+e.sSortJUIWrapper,this);g(this).append(a.contents());a.detach()}));!a&&c&&c.insertBefore(d,b.nTableReinsertBefore); +f.children().detach();f.append(m);j.css("width",b.sDestroyWidth).removeClass(e.sTable);(o=b.asDestroyStripes.length)&&f.children().each(function(a){g(this).addClass(b.asDestroyStripes[a%o])});c=g.inArray(b,p.settings);-1!==c&&p.settings.splice(c,1)})});p.version="1.10.4";p.settings=[];p.models={};p.models.oSearch={bCaseInsensitive:!0,sSearch:"",bRegex:!1,bSmart:!0};p.models.oRow={nTr:null,anCells:null,_aData:[],_aSortData:null,_aFilterData:null,_sFilterRow:null,_sRowStripe:"",src:null};p.models.oColumn= +{idx:null,aDataSort:null,asSorting:null,bSearchable:null,bSortable:null,bVisible:null,_sManualType:null,_bAttrSrc:!1,fnCreatedCell:null,fnGetData:null,fnSetData:null,mData:null,mRender:null,nTh:null,nTf:null,sClass:null,sContentPadding:null,sDefaultContent:null,sName:null,sSortDataType:"std",sSortingClass:null,sSortingClassJUI:null,sTitle:null,sType:null,sWidth:null,sWidthOrig:null};p.defaults={aaData:null,aaSorting:[[0,"asc"]],aaSortingFixed:[],ajax:null,aLengthMenu:[10,25,50,100],aoColumns:null, +aoColumnDefs:null,aoSearchCols:[],asStripeClasses:null,bAutoWidth:!0,bDeferRender:!1,bDestroy:!1,bFilter:!0,bInfo:!0,bJQueryUI:!1,bLengthChange:!0,bPaginate:!0,bProcessing:!1,bRetrieve:!1,bScrollCollapse:!1,bServerSide:!1,bSort:!0,bSortMulti:!0,bSortCellsTop:!1,bSortClasses:!0,bStateSave:!1,fnCreatedRow:null,fnDrawCallback:null,fnFooterCallback:null,fnFormatNumber:function(a){return a.toString().replace(/\B(?=(\d{3})+(?!\d))/g,this.oLanguage.sThousands)},fnHeaderCallback:null,fnInfoCallback:null, +fnInitComplete:null,fnPreDrawCallback:null,fnRowCallback:null,fnServerData:null,fnServerParams:null,fnStateLoadCallback:function(a){try{return JSON.parse((-1===a.iStateDuration?sessionStorage:localStorage).getItem("DataTables_"+a.sInstance+"_"+location.pathname))}catch(b){}},fnStateLoadParams:null,fnStateLoaded:null,fnStateSaveCallback:function(a,b){try{(-1===a.iStateDuration?sessionStorage:localStorage).setItem("DataTables_"+a.sInstance+"_"+location.pathname,JSON.stringify(b))}catch(c){}},fnStateSaveParams:null, +iStateDuration:7200,iDeferLoading:null,iDisplayLength:10,iDisplayStart:0,iTabIndex:0,oClasses:{},oLanguage:{oAria:{sSortAscending:": activate to sort column ascending",sSortDescending:": activate to sort column descending"},oPaginate:{sFirst:"First",sLast:"Last",sNext:"Next",sPrevious:"Previous"},sEmptyTable:"No data available in table",sInfo:"Showing _START_ to _END_ of _TOTAL_ entries",sInfoEmpty:"Showing 0 to 0 of 0 entries",sInfoFiltered:"(filtered from _MAX_ total entries)",sInfoPostFix:"",sDecimal:"", +sThousands:",",sLengthMenu:"Show _MENU_ entries",sLoadingRecords:"Loading...",sProcessing:"Processing...",sSearch:"Search:",sSearchPlaceholder:"",sUrl:"",sZeroRecords:"No matching records found"},oSearch:g.extend({},p.models.oSearch),sAjaxDataProp:"data",sAjaxSource:null,sDom:"lfrtip",searchDelay:null,sPaginationType:"simple_numbers",sScrollX:"",sScrollXInner:"",sScrollY:"",sServerMethod:"GET",renderer:null};V(p.defaults);p.defaults.column={aDataSort:null,iDataSort:-1,asSorting:["asc","desc"],bSearchable:!0, +bSortable:!0,bVisible:!0,fnCreatedCell:null,mData:null,mRender:null,sCellType:"td",sClass:"",sContentPadding:"",sDefaultContent:null,sName:"",sSortDataType:"std",sTitle:null,sType:null,sWidth:null};V(p.defaults.column);p.models.oSettings={oFeatures:{bAutoWidth:null,bDeferRender:null,bFilter:null,bInfo:null,bLengthChange:null,bPaginate:null,bProcessing:null,bServerSide:null,bSort:null,bSortMulti:null,bSortClasses:null,bStateSave:null},oScroll:{bCollapse:null,iBarWidth:0,sX:null,sXInner:null,sY:null}, +oLanguage:{fnInfoCallback:null},oBrowser:{bScrollOversize:!1,bScrollbarLeft:!1},ajax:null,aanFeatures:[],aoData:[],aiDisplay:[],aiDisplayMaster:[],aoColumns:[],aoHeader:[],aoFooter:[],oPreviousSearch:{},aoPreSearchCols:[],aaSorting:null,aaSortingFixed:[],asStripeClasses:null,asDestroyStripes:[],sDestroyWidth:0,aoRowCallback:[],aoHeaderCallback:[],aoFooterCallback:[],aoDrawCallback:[],aoRowCreatedCallback:[],aoPreDrawCallback:[],aoInitComplete:[],aoStateSaveParams:[],aoStateLoadParams:[],aoStateLoaded:[], +sTableId:"",nTable:null,nTHead:null,nTFoot:null,nTBody:null,nTableWrapper:null,bDeferLoading:!1,bInitialised:!1,aoOpenRows:[],sDom:null,searchDelay:null,sPaginationType:"two_button",iStateDuration:0,aoStateSave:[],aoStateLoad:[],oSavedState:null,oLoadedState:null,sAjaxSource:null,sAjaxDataProp:null,bAjaxDataGet:!0,jqXHR:null,json:l,oAjaxData:l,fnServerData:null,aoServerParams:[],sServerMethod:null,fnFormatNumber:null,aLengthMenu:null,iDraw:0,bDrawing:!1,iDrawError:-1,_iDisplayLength:10,_iDisplayStart:0, +_iRecordsTotal:0,_iRecordsDisplay:0,bJUI:null,oClasses:{},bFiltered:!1,bSorted:!1,bSortCellsTop:null,oInit:null,aoDestroyCallback:[],fnRecordsTotal:function(){return"ssp"==A(this)?1*this._iRecordsTotal:this.aiDisplayMaster.length},fnRecordsDisplay:function(){return"ssp"==A(this)?1*this._iRecordsDisplay:this.aiDisplay.length},fnDisplayEnd:function(){var a=this._iDisplayLength,b=this._iDisplayStart,c=b+a,e=this.aiDisplay.length,d=this.oFeatures,f=d.bPaginate;return d.bServerSide?!1===f||-1===a?b+e: +Math.min(b+a,this._iRecordsDisplay):!f||c>e||-1===a?e:c},oInstance:null,sInstance:null,iTabIndex:0,nScrollHead:null,nScrollFoot:null,aLastSort:[],oPlugins:{}};p.ext=w={classes:{},errMode:"alert",feature:[],search:[],internal:{},legacy:{ajax:null},pager:{},renderer:{pageButton:{},header:{}},order:{},type:{detect:[],search:{},order:{}},_unique:0,fnVersionCheck:p.fnVersionCheck,iApiIndex:0,oJUIClasses:{},sVersion:p.version};g.extend(w,{afnFiltering:w.search,aTypes:w.type.detect,ofnSearch:w.type.search, +oSort:w.type.order,afnSortData:w.order,aoFeatures:w.feature,oApi:w.internal,oStdClasses:w.classes,oPagination:w.pager});g.extend(p.ext.classes,{sTable:"dataTable",sNoFooter:"no-footer",sPageButton:"paginate_button",sPageButtonActive:"current",sPageButtonDisabled:"disabled",sStripeOdd:"odd",sStripeEven:"even",sRowEmpty:"dataTables_empty",sWrapper:"dataTables_wrapper",sFilter:"dataTables_filter",sInfo:"dataTables_info",sPaging:"dataTables_paginate paging_",sLength:"dataTables_length",sProcessing:"dataTables_processing", +sSortAsc:"sorting_asc",sSortDesc:"sorting_desc",sSortable:"sorting",sSortableAsc:"sorting_asc_disabled",sSortableDesc:"sorting_desc_disabled",sSortableNone:"sorting_disabled",sSortColumn:"sorting_",sFilterInput:"",sLengthSelect:"",sScrollWrapper:"dataTables_scroll",sScrollHead:"dataTables_scrollHead",sScrollHeadInner:"dataTables_scrollHeadInner",sScrollBody:"dataTables_scrollBody",sScrollFoot:"dataTables_scrollFoot",sScrollFootInner:"dataTables_scrollFootInner",sHeaderTH:"",sFooterTH:"",sSortJUIAsc:"", +sSortJUIDesc:"",sSortJUI:"",sSortJUIAscAllowed:"",sSortJUIDescAllowed:"",sSortJUIWrapper:"",sSortIcon:"",sJUIHeader:"",sJUIFooter:""});var Ca="",Ca="",E=Ca+"ui-state-default",ia=Ca+"css_right ui-icon ui-icon-",Wb=Ca+"fg-toolbar ui-toolbar ui-widget-header ui-helper-clearfix";g.extend(p.ext.oJUIClasses,p.ext.classes,{sPageButton:"fg-button ui-button "+E,sPageButtonActive:"ui-state-disabled",sPageButtonDisabled:"ui-state-disabled",sPaging:"dataTables_paginate fg-buttonset ui-buttonset fg-buttonset-multi ui-buttonset-multi paging_", +sSortAsc:E+" sorting_asc",sSortDesc:E+" sorting_desc",sSortable:E+" sorting",sSortableAsc:E+" sorting_asc_disabled",sSortableDesc:E+" sorting_desc_disabled",sSortableNone:E+" sorting_disabled",sSortJUIAsc:ia+"triangle-1-n",sSortJUIDesc:ia+"triangle-1-s",sSortJUI:ia+"carat-2-n-s",sSortJUIAscAllowed:ia+"carat-1-n",sSortJUIDescAllowed:ia+"carat-1-s",sSortJUIWrapper:"DataTables_sort_wrapper",sSortIcon:"DataTables_sort_icon",sScrollHead:"dataTables_scrollHead "+E,sScrollFoot:"dataTables_scrollFoot "+E, +sHeaderTH:E,sFooterTH:E,sJUIHeader:Wb+" ui-corner-tl ui-corner-tr",sJUIFooter:Wb+" ui-corner-bl ui-corner-br"});var Lb=p.ext.pager;g.extend(Lb,{simple:function(){return["previous","next"]},full:function(){return["first","previous","next","last"]},simple_numbers:function(a,b){return["previous",Va(a,b),"next"]},full_numbers:function(a,b){return["first","previous",Va(a,b),"next","last"]},_numbers:Va,numbers_length:7});g.extend(!0,p.ext.renderer,{pageButton:{_:function(a,b,c,e,d,f){var h=a.oClasses,i= +a.oLanguage.oPaginate,j,l,m=0,o=function(b,e){var k,p,r,q,s=function(b){Sa(a,b.data.action,true)};k=0;for(p=e.length;k").appendTo(b);o(r,q)}else{l=j="";switch(q){case "ellipsis":b.append("");break;case "first":j=i.sFirst;l=q+(d>0?"":" "+h.sPageButtonDisabled);break;case "previous":j=i.sPrevious;l=q+(d>0?"":" "+h.sPageButtonDisabled);break;case "next":j=i.sNext;l=q+(d",{"class":h.sPageButton+" "+l,"aria-controls":a.sTableId,"data-dt-idx":m,tabindex:a.iTabIndex,id:c===0&&typeof q==="string"?a.sTableId+"_"+q:null}).html(j).appendTo(b);Ua(r,{action:q},s);m++}}}};try{var k=g(P.activeElement).data("dt-idx");o(g(b).empty(),e);k!==null&&g(b).find("[data-dt-idx="+k+"]").focus()}catch(p){}}}});g.extend(p.ext.type.detect,[function(a,b){var c=b.oLanguage.sDecimal; +return Ya(a,c)?"num"+c:null},function(a){if(a&&!(a instanceof Date)&&(!$b.test(a)||!ac.test(a)))return null;var b=Date.parse(a);return null!==b&&!isNaN(b)||H(a)?"date":null},function(a,b){var c=b.oLanguage.sDecimal;return Ya(a,c,!0)?"num-fmt"+c:null},function(a,b){var c=b.oLanguage.sDecimal;return Qb(a,c)?"html-num"+c:null},function(a,b){var c=b.oLanguage.sDecimal;return Qb(a,c,!0)?"html-num-fmt"+c:null},function(a){return H(a)||"string"===typeof a&&-1!==a.indexOf("<")?"html":null}]);g.extend(p.ext.type.search, +{html:function(a){return H(a)?a:"string"===typeof a?a.replace(Nb," ").replace(Aa,""):""},string:function(a){return H(a)?a:"string"===typeof a?a.replace(Nb," "):a}});var za=function(a,b,c,e){if(0!==a&&(!a||"-"===a))return-Infinity;b&&(a=Pb(a,b));a.replace&&(c&&(a=a.replace(c,"")),e&&(a=a.replace(e,"")));return 1*a};g.extend(w.type.order,{"date-pre":function(a){return Date.parse(a)||0},"html-pre":function(a){return H(a)?"":a.replace?a.replace(/<.*?>/g,"").toLowerCase():a+""},"string-pre":function(a){return H(a)? +"":"string"===typeof a?a.toLowerCase():!a.toString?"":a.toString()},"string-asc":function(a,b){return ab?1:0},"string-desc":function(a,b){return ab?-1:0}});cb("");g.extend(!0,p.ext.renderer,{header:{_:function(a,b,c,e){g(a.nTable).on("order.dt.DT",function(d,f,h,g){if(a===f){d=c.idx;b.removeClass(c.sSortingClass+" "+e.sSortAsc+" "+e.sSortDesc).addClass(g[d]=="asc"?e.sSortAsc:g[d]=="desc"?e.sSortDesc:c.sSortingClass)}})},jqueryui:function(a,b,c,e){g("
    ").addClass(e.sSortJUIWrapper).append(b.contents()).append(g("").addClass(e.sSortIcon+ +" "+c.sSortingClassJUI)).appendTo(b);g(a.nTable).on("order.dt.DT",function(d,f,g,i){if(a===f){d=c.idx;b.removeClass(e.sSortAsc+" "+e.sSortDesc).addClass(i[d]=="asc"?e.sSortAsc:i[d]=="desc"?e.sSortDesc:c.sSortingClass);b.find("span."+e.sSortIcon).removeClass(e.sSortJUIAsc+" "+e.sSortJUIDesc+" "+e.sSortJUI+" "+e.sSortJUIAscAllowed+" "+e.sSortJUIDescAllowed).addClass(i[d]=="asc"?e.sSortJUIAsc:i[d]=="desc"?e.sSortJUIDesc:c.sSortingClassJUI)}})}}});p.render={number:function(a,b,c,e){return{display:function(d){var f= +0>d?"-":"",d=Math.abs(parseFloat(d)),g=parseInt(d,10),d=c?b+(d-g).toFixed(c).substring(2):"";return f+(e||"")+g.toString().replace(/\B(?=(\d{3})+(?!\d))/g,a)+d}}}};g.extend(p.ext.internal,{_fnExternApiFunc:Mb,_fnBuildAjax:qa,_fnAjaxUpdate:jb,_fnAjaxParameters:sb,_fnAjaxUpdateDraw:tb,_fnAjaxDataSrc:ra,_fnAddColumn:Ea,_fnColumnOptions:ja,_fnAdjustColumnSizing:X,_fnVisibleToColumnIndex:ka,_fnColumnIndexToVisible:$,_fnVisbleColumns:aa,_fnGetColumns:Z,_fnColumnTypes:Ga,_fnApplyColumnDefs:hb,_fnHungarianMap:V, +_fnCamelToHungarian:G,_fnLanguageCompat:O,_fnBrowserDetect:fb,_fnAddData:I,_fnAddTr:la,_fnNodeToDataIndex:function(a,b){return b._DT_RowIndex!==l?b._DT_RowIndex:null},_fnNodeToColumnIndex:function(a,b,c){return g.inArray(c,a.aoData[b].anCells)},_fnGetCellData:v,_fnSetCellData:Ha,_fnSplitObjNotation:Ja,_fnGetObjectDataFn:W,_fnSetObjectDataFn:Q,_fnGetDataMaster:Ka,_fnClearTable:na,_fnDeleteIndex:oa,_fnInvalidate:ca,_fnGetRowElements:ma,_fnCreateTr:Ia,_fnBuildHead:ib,_fnDrawHead:ea,_fnDraw:L,_fnReDraw:M, +_fnAddOptionsHtml:lb,_fnDetectHeader:da,_fnGetUniqueThs:pa,_fnFeatureHtmlFilter:nb,_fnFilterComplete:fa,_fnFilterCustom:wb,_fnFilterColumn:vb,_fnFilter:ub,_fnFilterCreateSearch:Pa,_fnEscapeRegex:ua,_fnFilterData:xb,_fnFeatureHtmlInfo:qb,_fnUpdateInfo:Ab,_fnInfoMacros:Bb,_fnInitialise:ga,_fnInitComplete:sa,_fnLengthChange:Qa,_fnFeatureHtmlLength:mb,_fnFeatureHtmlPaginate:rb,_fnPageChange:Sa,_fnFeatureHtmlProcessing:ob,_fnProcessingDisplay:B,_fnFeatureHtmlTable:pb,_fnScrollDraw:Y,_fnApplyToChildren:F, +_fnCalculateColumnWidths:Fa,_fnThrottle:ta,_fnConvertToWidth:Cb,_fnScrollingWidthAdjust:Eb,_fnGetWidestNode:Db,_fnGetMaxLenString:Fb,_fnStringToCss:s,_fnScrollBarWidth:Gb,_fnSortFlatten:T,_fnSort:kb,_fnSortAria:Ib,_fnSortListener:Ta,_fnSortAttachListener:Na,_fnSortingClasses:wa,_fnSortData:Hb,_fnSaveState:xa,_fnLoadState:Jb,_fnSettingsFromNode:ya,_fnLog:R,_fnMap:D,_fnBindAction:Ua,_fnCallbackReg:x,_fnCallbackFire:u,_fnLengthOverflow:Ra,_fnRenderer:Oa,_fnDataSource:A,_fnRowAttributes:La,_fnCalculateEnd:function(){}}); +g.fn.dataTable=p;g.fn.dataTableSettings=p.settings;g.fn.dataTableExt=p.ext;g.fn.DataTable=function(a){return g(this).dataTable(a).api()};g.each(p,function(a,b){g.fn.DataTable[a]=b});return g.fn.dataTable};"function"===typeof define&&define.amd?define("datatables",["jquery"],O):"object"===typeof exports?O(require("jquery")):jQuery&&!jQuery.fn.dataTable&&O(jQuery)})(window,document); diff --git a/core/src/main/resources/org/apache/spark/ui/static/jquery.mustache.js b/core/src/main/resources/org/apache/spark/ui/static/jquery.mustache.js new file mode 100644 index 000000000000..14925bf93d0f --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/jquery.mustache.js @@ -0,0 +1,592 @@ +/* +Shameless port of a shameless port +@defunkt => @janl => @aq + +See http://github.com/defunkt/mustache for more info. +*/ + +;(function($) { + +/*! + * mustache.js - Logic-less {{mustache}} templates with JavaScript + * http://github.com/janl/mustache.js + */ + +/*global define: false*/ + +(function (root, factory) { + if (typeof exports === "object" && exports) { + factory(exports); // CommonJS + } else { + var mustache = {}; + factory(mustache); + if (typeof define === "function" && define.amd) { + define(mustache); // AMD + } else { + root.Mustache = mustache; // +
    +
    +
      + {providerConfig.map { case (k, v) =>
    • {k}: {v}
    • }} +
    + { + if (eventLogsUnderProcessCount > 0) { +

    There are {eventLogsUnderProcessCount} event log(s) currently being + processed which may result in additional applications getting listed on this page. + Refresh the page to view updates.

    + } + } + + { + if (lastUpdatedTime > 0) { +

    Last updated: {lastUpdatedTime}

    + } + } -

    - Showing {actualFirst + 1}-{last + 1} of {allAppsSize} - {if (requestedIncomplete) "(Incomplete applications)"} - - { - if (actualPage > 1) { - < - 1 - } - } - {if (actualPage - plusOrMinus > secondPageFromLeft) " ... "} - {leftSideIndices} - {actualPage} - {rightSideIndices} - {if (actualPage + plusOrMinus < secondPageFromRight) " ... "} - { - if (actualPage < pageCount) { - {pageCount} - > - } - } - -

    ++ - appTable + { + if (allAppsSize > 0) { + ++ + ++ + ++ + ++ + } else if (requestedIncomplete) {

    No incomplete applications found!

    + } else if (eventLogsUnderProcessCount > 0) { +

    No completed applications found!

    } else { -

    No completed applications found!

    ++ -

    Did you specify the correct logging directory? - Please verify your setting of - spark.history.fs.logDirectory and whether you have the permissions to - access it.
    It is also possible that your application did not run to - completion or did not stop the SparkContext. -

    +

    No completed applications found!

    ++ parent.emptyListingHtml } - } - - { + } + + + { if (requestedIncomplete) { "Back to completed applications" } else { "Show incomplete applications" } - } - -
    + } + +
    - UIUtils.basicSparkPage(content, "History Server") - } - - private val appHeader = Seq( - "App ID", - "App Name", - "Started", - "Completed", - "Duration", - "Spark User", - "Last Updated") - - private val appWithAttemptHeader = Seq( - "App ID", - "App Name", - "Attempt ID", - "Started", - "Completed", - "Duration", - "Spark User", - "Last Updated") - - private def rangeIndices( - range: Seq[Int], - condition: Int => Boolean, - showIncomplete: Boolean): Seq[Node] = { - range.filter(condition).map(nextPage => - {nextPage} ) - } - - private def attemptRow( - renderAttemptIdColumn: Boolean, - info: ApplicationHistoryInfo, - attempt: ApplicationAttemptInfo, - isFirst: Boolean): Seq[Node] = { - val uiAddress = UIUtils.prependBaseUri(HistoryServer.getAttemptURI(info.id, attempt.attemptId)) - val startTime = UIUtils.formatDate(attempt.startTime) - val endTime = if (attempt.endTime > 0) UIUtils.formatDate(attempt.endTime) else "-" - val duration = - if (attempt.endTime > 0) { - UIUtils.formatDuration(attempt.endTime - attempt.startTime) - } else { - "-" - } - val lastUpdated = UIUtils.formatDate(attempt.lastUpdated) - - { - if (isFirst) { - if (info.attempts.size > 1 || renderAttemptIdColumn) { - - {info.id} - - {info.name} - } else { - {info.id} - {info.name} - } - } else { - Nil - } - } - { - if (renderAttemptIdColumn) { - if (info.attempts.size > 1 && attempt.attemptId.isDefined) { - {attempt.attemptId.get} - } else { -   - } - } else { - Nil - } - } - {startTime} - {endTime} - - {duration} - {attempt.sparkUser} - {lastUpdated} - - } - - private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { - attemptRow(false, info, info.attempts.head, true) - } - - private def appWithAttemptRow(info: ApplicationHistoryInfo): Seq[Node] = { - attemptRow(true, info, info.attempts.head, true) ++ - info.attempts.drop(1).flatMap(attemptRow(true, info, _, false)) + UIUtils.basicSparkPage(content, "History Server", true) } - private def makePageLink(linkPage: Int, showIncomplete: Boolean): String = { - UIUtils.prependBaseUri("/?" + Array( - "page=" + linkPage, - "showIncomplete=" + showIncomplete - ).mkString("&")) + private def makePageLink(showIncomplete: Boolean): String = { + UIUtils.prependBaseUri("/?" + "showIncomplete=" + showIncomplete) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index d4f327cc588f..d9c8fda99ef9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -21,16 +21,19 @@ import java.util.NoSuchElementException import java.util.zip.ZipOutputStream import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} -import com.google.common.cache._ +import scala.util.control.NonFatal +import scala.xml.Node + import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, ApplicationsListResource, - UIRoot} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, ApplicationsListResource, UIRoot} import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.{ShutdownHookManager, SignalLogger, Utils} +import org.apache.spark.util.{ShutdownHookManager, SystemClock, Utils} /** * A web server that renders SparkUIs of completed applications. @@ -48,31 +51,20 @@ class HistoryServer( provider: ApplicationHistoryProvider, securityManager: SecurityManager, port: Int) - extends WebUI(securityManager, port, conf) with Logging with UIRoot { + extends WebUI(securityManager, securityManager.getSSLOptions("historyServer"), port, conf) + with Logging with UIRoot with ApplicationCacheOperations { // How many applications to retain private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50) - private val appLoader = new CacheLoader[String, SparkUI] { - override def load(key: String): SparkUI = { - val parts = key.split("/") - require(parts.length == 1 || parts.length == 2, s"Invalid app key $key") - val ui = provider - .getAppUI(parts(0), if (parts.length > 1) Some(parts(1)) else None) - .getOrElse(throw new NoSuchElementException(s"no app with key $key")) - attachSparkUI(ui) - ui - } - } + // How many applications the summary ui displays + private[history] val maxApplications = conf.get(HISTORY_UI_MAX_APPS); - private val appCache = CacheBuilder.newBuilder() - .maximumSize(retainedApplications) - .removalListener(new RemovalListener[String, SparkUI] { - override def onRemoval(rm: RemovalNotification[String, SparkUI]): Unit = { - detachSparkUI(rm.getValue()) - } - }) - .build(appLoader) + // application + private val appCache = new ApplicationCache(this, retainedApplications, new SystemClock()) + + // and its metrics, for testing as well as monitoring + val cacheMetrics = appCache.metrics private val loaderServlet = new HttpServlet { protected override def doGet(req: HttpServletRequest, res: HttpServletResponse): Unit = { @@ -103,7 +95,9 @@ class HistoryServer( // Note we don't use the UI retrieved from the cache; the cache loader above will register // the app's UI, and all we need to do is redirect the user to the same URI that was // requested, and the proper data should be served at that point. - res.sendRedirect(res.encodeRedirectURL(req.getRequestURI())) + // Also, make sure that the redirect url contains the query string present in the request. + val requestURI = req.getRequestURI + Option(req.getQueryString).map("?" + _).getOrElse("") + res.sendRedirect(res.encodeRedirectURL(requestURI)) } // SPARK-5983 ensure TRACE is not supported @@ -113,7 +107,7 @@ class HistoryServer( } def getSparkUI(appKey: String): Option[SparkUI] = { - Option(appCache.get(appKey)) + appCache.getSparkUI(appKey) } initialize() @@ -146,32 +140,59 @@ class HistoryServer( override def stop() { super.stop() provider.stop() + appCache.stop() } /** Attach a reconstructed UI to this server. Only valid after bind(). */ - private def attachSparkUI(ui: SparkUI) { + override def attachSparkUI( + appId: String, + attemptId: Option[String], + ui: SparkUI, + completed: Boolean) { assert(serverInfo.isDefined, "HistoryServer must be bound before attaching SparkUIs") ui.getHandlers.foreach(attachHandler) addFilters(ui.getHandlers, conf) } /** Detach a reconstructed UI from this server. Only valid after bind(). */ - private def detachSparkUI(ui: SparkUI) { + override def detachSparkUI(appId: String, attemptId: Option[String], ui: SparkUI): Unit = { assert(serverInfo.isDefined, "HistoryServer must be bound before detaching SparkUIs") ui.getHandlers.foreach(detachHandler) } + /** + * Get the application UI and whether or not it is completed + * @param appId application ID + * @param attemptId attempt ID + * @return If found, the Spark UI and any history information to be used in the cache + */ + override def getAppUI(appId: String, attemptId: Option[String]): Option[LoadedAppUI] = { + provider.getAppUI(appId, attemptId) + } + /** * Returns a list of available applications, in descending order according to their end time. * * @return List of all known applications. */ - def getApplicationList(): Iterable[ApplicationHistoryInfo] = { + def getApplicationList(): Iterator[ApplicationHistoryInfo] = { provider.getListing() } + def getEventLogsUnderProcess(): Int = { + provider.getEventLogsUnderProcess() + } + + def getLastUpdatedTime(): Long = { + provider.getLastUpdatedTime() + } + def getApplicationInfoList: Iterator[ApplicationInfo] = { - getApplicationList().iterator.map(ApplicationsListResource.appHistoryInfoToPublicAppInfo) + getApplicationList().map(ApplicationsListResource.appHistoryInfoToPublicAppInfo) + } + + def getApplicationInfo(appId: String): Option[ApplicationInfo] = { + provider.getApplicationInfo(appId).map(ApplicationsListResource.appHistoryInfoToPublicAppInfo) } override def writeEventLogs( @@ -181,6 +202,13 @@ class HistoryServer( provider.writeEventLogs(appId, attemptId, zipStream) } + /** + * @return html text to display when the application list is empty + */ + def emptyListingHtml(): Seq[Node] = { + provider.getEmptyListingHtml() + } + /** * Returns the provider configuration to show in the listing page. * @@ -188,12 +216,18 @@ class HistoryServer( */ def getProviderConfig(): Map[String, String] = provider.getConfig() + /** + * Load an application UI and attach it to the web server. + * @param appId application ID + * @param attemptId optional attempt ID + * @return true if the application was found and loaded. + */ private def loadAppUi(appId: String, attemptId: Option[String]): Boolean = { try { - appCache.get(appId + attemptId.map { id => s"/$id" }.getOrElse("")) + appCache.get(appId, attemptId) true } catch { - case e: Exception => e.getCause() match { + case NonFatal(e) => e.getCause() match { case nsee: NoSuchElementException => false @@ -202,6 +236,17 @@ class HistoryServer( } } + /** + * String value for diagnostics. + * @return a multi-line description of the server state. + */ + override def toString: String = { + s""" + | History Server; + | provider = $provider + | cache = $appCache + """.stripMargin + } } /** @@ -220,11 +265,11 @@ object HistoryServer extends Logging { val UI_PATH_PREFIX = "/history" - def main(argStrings: Array[String]) { - SignalLogger.register(log) + def main(argStrings: Array[String]): Unit = { + Utils.initDaemon(log) new HistoryServerArguments(conf, argStrings) initSecurity() - val securityManager = new SecurityManager(conf) + val securityManager = createSecurityManager(conf) val providerName = conf.getOption("spark.history.provider") .getOrElse(classOf[FsHistoryProvider].getName()) @@ -244,6 +289,29 @@ object HistoryServer extends Logging { while(true) { Thread.sleep(Int.MaxValue) } } + /** + * Create a security manager. + * This turns off security in the SecurityManager, so that the History Server can start + * in a Spark cluster where security is enabled. + * @param config configuration for the SecurityManager constructor + * @return the security manager for use in constructing the History Server. + */ + private[history] def createSecurityManager(config: SparkConf): SecurityManager = { + if (config.getBoolean(SecurityManager.SPARK_AUTH_CONF, false)) { + logDebug(s"Clearing ${SecurityManager.SPARK_AUTH_CONF}") + config.set(SecurityManager.SPARK_AUTH_CONF, "false") + } + + if (config.getBoolean("spark.acls.enable", config.getBoolean("spark.ui.acls.enable", false))) { + logInfo("Either spark.acls.enable or spark.ui.acls.enable is configured, clearing it and " + + "only using spark.history.ui.acl.enable") + config.set("spark.acls.enable", "false") + config.set("spark.ui.acls.enable", "false") + } + + new SecurityManager(config) + } + def initSecurity() { // If we are accessing HDFS and it has security enabled (Kerberos), we have to login // from a keytab file so that we can access HDFS beyond the kerberos ticket expiration. diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index d03bab3820bb..080ba12c2f0d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -17,11 +17,14 @@ package org.apache.spark.deploy.history -import org.apache.spark.{Logging, SparkConf} +import scala.annotation.tailrec + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** - * Command-line parser for the master. + * Command-line parser for the [[HistoryServer]]. */ private[history] class HistoryServerArguments(conf: SparkConf, args: Array[String]) extends Logging { @@ -29,6 +32,7 @@ private[history] class HistoryServerArguments(conf: SparkConf, args: Array[Strin parse(args.toList) + @tailrec private def parse(args: List[String]): Unit = { if (args.length == 1) { setLogDirectory(args.head) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index ac553b71115d..53564d0e9515 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -41,7 +41,6 @@ private[spark] class ApplicationInfo( @transient var coresGranted: Int = _ @transient var endTime: Long = _ @transient var appSource: ApplicationSource = _ - @transient @volatile var appUIUrlAtHistoryServer: Option[String] = None // A cap on the number of executors this application can have at any given time. // By default, this is infinite. Only after the first allocation request is issued by the @@ -65,7 +64,7 @@ private[spark] class ApplicationInfo( appSource = new ApplicationSource(this) nextExecutorId = 0 removedExecutors = new ArrayBuffer[ExecutorDesc] - executorLimit = Integer.MAX_VALUE + executorLimit = desc.initialExecutorLimit.getOrElse(Integer.MAX_VALUE) } private def newExecutorId(useID: Option[Int] = None): Int = { @@ -135,11 +134,4 @@ private[spark] class ApplicationInfo( System.currentTimeMillis() - startTime } } - - /** - * Returns the original application UI url unless there is its address at history server - * is defined - */ - def curAppUIUrl: String = appUIUrlAtHistoryServer.getOrElse(desc.appUiUrl) - } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala index 37bfcdfdf477..097728c82157 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala @@ -22,6 +22,4 @@ private[master] object ApplicationState extends Enumeration { type ApplicationState = Value val WAITING, RUNNING, FINISHED, FAILED, KILLED, UNKNOWN = Value - - val MAX_NUM_RETRY = 10 } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala index b197dbcbfe29..8d5edae0501e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala @@ -19,7 +19,6 @@ package org.apache.spark.deploy.master import java.util.Date -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.DriverDescription import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index 1aa8cd5013b4..f2b5ea7e23ec 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -21,7 +21,7 @@ import java.io._ import scala.reflect.ClassTag -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.serializer.{DeserializationStream, SerializationStream, Serializer} import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala index 70f21fbe0de8..52e2854961ed 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala @@ -32,8 +32,8 @@ trait LeaderElectionAgent { @DeveloperApi trait LeaderElectable { - def electedLeader() - def revokedLeadership() + def electedLeader(): Unit + def revokedLeadership(): Unit } /** Single-node implementation of LeaderElectionAgent -- we're initially and always the leader. */ 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 b25a487806c7..e030cac60a8e 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 @@ -17,33 +17,26 @@ package org.apache.spark.deploy.master -import java.io.FileNotFoundException -import java.net.URLEncoder import java.text.SimpleDateFormat -import java.util.Date +import java.util.{Date, Locale} import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.language.postfixOps import scala.util.Random -import org.apache.hadoop.fs.Path - -import org.apache.spark.rpc._ -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} +import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages._ -import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.deploy.rest.StandaloneRestServer +import org.apache.spark.internal.Logging import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} +import org.apache.spark.rpc._ import org.apache.spark.serializer.{JavaSerializer, Serializer} -import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{ThreadUtils, SignalLogger, Utils} +import org.apache.spark.util.{SparkUncaughtExceptionHandler, ThreadUtils, Utils} private[deploy] class Master( override val rpcEnv: RpcEnv, @@ -58,17 +51,19 @@ private[deploy] class Master( private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs + // For application IDs + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) private val WORKER_TIMEOUT_MS = conf.getLong("spark.worker.timeout", 60) * 1000 private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) private val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) private val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) private val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") + private val MAX_EXECUTOR_RETRIES = conf.getInt("spark.deploy.maxExecutorRetries", 10) val workers = new HashSet[WorkerInfo] val idToApp = new HashMap[String, ApplicationInfo] - val waitingApps = new ArrayBuffer[ApplicationInfo] + private val waitingApps = new ArrayBuffer[ApplicationInfo] val apps = new HashSet[ApplicationInfo] private val idToWorker = new HashMap[String, WorkerInfo] @@ -78,7 +73,6 @@ private[deploy] class Master( private val addressToApp = new HashMap[RpcAddress, ApplicationInfo] private val completedApps = new ArrayBuffer[ApplicationInfo] private var nextAppNumber = 0 - private val appIdToUI = new HashMap[String, SparkUI] private val drivers = new HashSet[DriverInfo] private val completedDrivers = new ArrayBuffer[DriverInfo] @@ -86,7 +80,7 @@ private[deploy] class Master( private val waitingDrivers = new ArrayBuffer[DriverInfo] private var nextDriverNumber = 0 - Utils.checkHost(address.host, "Expected hostname") + Utils.checkHost(address.host) private val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf, securityMgr) private val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf, @@ -121,6 +115,7 @@ private[deploy] class Master( // Default maxCores for applications that don't specify it (i.e. pass Int.MaxValue) private val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue) + val reverseProxy = conf.getBoolean("spark.ui.reverseProxy", false) if (defaultCores < 1) { throw new SparkException("spark.deploy.defaultCores must be positive") } @@ -136,6 +131,12 @@ private[deploy] class Master( webUi = new MasterWebUI(this, webUiPort) webUi.bind() masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort + if (reverseProxy) { + masterWebUiUrl = conf.get("spark.ui.reverseProxyUrl", masterWebUiUrl) + webUi.addProxy() + logInfo(s"Spark Master is acting as a reverse proxy. Master, Workers and " + + s"Applications UIs are available at $masterWebUiUrl") + } checkForWorkerTimeOutTask = forwardMessageThread.scheduleAtFixedRate(new Runnable { override def run(): Unit = Utils.tryLogNonFatalError { self.send(CheckForWorkerTimeOut) @@ -208,7 +209,7 @@ private[deploy] class Master( } override def receive: PartialFunction[Any, Unit] = { - case ElectedLeader => { + case ElectedLeader => val (storedApps, storedDrivers, storedWorkers) = persistenceEngine.readPersistedData(rpcEnv) state = if (storedApps.isEmpty && storedDrivers.isEmpty && storedWorkers.isEmpty) { RecoveryState.ALIVE @@ -224,16 +225,38 @@ private[deploy] class Master( } }, WORKER_TIMEOUT_MS, TimeUnit.MILLISECONDS) } - } case CompleteRecovery => completeRecovery() - case RevokedLeadership => { + case RevokedLeadership => logError("Leadership has been revoked -- master shutting down.") System.exit(0) - } - case RegisterApplication(description, driver) => { + case RegisterWorker( + id, workerHost, workerPort, workerRef, cores, memory, workerWebUiUrl, masterAddress) => + logInfo("Registering worker %s:%d with %d cores, %s RAM".format( + workerHost, workerPort, cores, Utils.megabytesToString(memory))) + if (state == RecoveryState.STANDBY) { + workerRef.send(MasterInStandby) + } else if (idToWorker.contains(id)) { + workerRef.send(RegisterWorkerFailed("Duplicate worker ID")) + } else { + val worker = new WorkerInfo(id, workerHost, workerPort, cores, memory, + workerRef, workerWebUiUrl) + if (registerWorker(worker)) { + persistenceEngine.addWorker(worker) + workerRef.send(RegisteredWorker(self, masterWebUiUrl, masterAddress)) + schedule() + } else { + val workerAddress = worker.endpoint.address + logWarning("Worker registration failed. Attempted to re-register worker at same " + + "address: " + workerAddress) + workerRef.send(RegisterWorkerFailed("Attempted to re-register worker at same address: " + + workerAddress)) + } + } + + case RegisterApplication(description, driver) => // TODO Prevent repeated registrations from some driver if (state == RecoveryState.STANDBY) { // ignore, don't send response @@ -246,16 +269,23 @@ private[deploy] class Master( driver.send(RegisteredApplication(app.id, self)) schedule() } - } - case ExecutorStateChanged(appId, execId, state, message, exitStatus) => { + case ExecutorStateChanged(appId, execId, state, message, exitStatus) => val execOption = idToApp.get(appId).flatMap(app => app.executors.get(execId)) execOption match { - case Some(exec) => { + case Some(exec) => val appInfo = idToApp(appId) + val oldState = exec.state exec.state = state - if (state == ExecutorState.RUNNING) { appInfo.resetRetryCount() } - exec.application.driver.send(ExecutorUpdated(execId, state, message, exitStatus)) + + if (state == ExecutorState.RUNNING) { + assert(oldState == ExecutorState.LAUNCHING, + s"executor $execId state transfer from $oldState to RUNNING is illegal") + appInfo.resetRetryCount() + } + + exec.application.driver.send(ExecutorUpdated(execId, state, message, exitStatus, false)) + if (ExecutorState.isFinished(state)) { // Remove this executor from the worker and app logInfo(s"Removing executor ${exec.fullId} because it is $state") @@ -268,35 +298,33 @@ private[deploy] class Master( val normalExit = exitStatus == Some(0) // Only retry certain number of times so we don't go into an infinite loop. - if (!normalExit) { - if (appInfo.incrementRetryCount() < ApplicationState.MAX_NUM_RETRY) { - schedule() - } else { - val execs = appInfo.executors.values - if (!execs.exists(_.state == ExecutorState.RUNNING)) { - logError(s"Application ${appInfo.desc.name} with ID ${appInfo.id} failed " + - s"${appInfo.retryCount} times; removing it") - removeApplication(appInfo, ApplicationState.FAILED) - } + // Important note: this code path is not exercised by tests, so be very careful when + // changing this `if` condition. + if (!normalExit + && appInfo.incrementRetryCount() >= MAX_EXECUTOR_RETRIES + && MAX_EXECUTOR_RETRIES >= 0) { // < 0 disables this application-killing path + val execs = appInfo.executors.values + if (!execs.exists(_.state == ExecutorState.RUNNING)) { + logError(s"Application ${appInfo.desc.name} with ID ${appInfo.id} failed " + + s"${appInfo.retryCount} times; removing it") + removeApplication(appInfo, ApplicationState.FAILED) } } } - } + schedule() case None => logWarning(s"Got status update for unknown executor $appId/$execId") } - } - case DriverStateChanged(driverId, state, exception) => { + case DriverStateChanged(driverId, state, exception) => state match { case DriverState.ERROR | DriverState.FINISHED | DriverState.KILLED | DriverState.FAILED => removeDriver(driverId, state, exception) case _ => throw new Exception(s"Received unexpected state update for driver $driverId: $state") } - } - case Heartbeat(workerId, worker) => { + case Heartbeat(workerId, worker) => idToWorker.get(workerId) match { case Some(workerInfo) => workerInfo.lastHeartbeat = System.currentTimeMillis() @@ -310,9 +338,8 @@ private[deploy] class Master( " This worker was never registered, so ignoring the heartbeat.") } } - } - case MasterChangeAcknowledged(appId) => { + case MasterChangeAcknowledged(appId) => idToApp.get(appId) match { case Some(app) => logInfo("Application has been re-registered: " + appId) @@ -322,9 +349,8 @@ private[deploy] class Master( } if (canCompleteRecovery) { completeRecovery() } - } - case WorkerSchedulerStateResponse(workerId, executors, driverIds) => { + case WorkerSchedulerStateResponse(workerId, executors, driverIds) => idToWorker.get(workerId) match { case Some(worker) => logInfo("Worker has been re-registered: " + workerId) @@ -342,7 +368,7 @@ private[deploy] class Master( drivers.find(_.id == driverId).foreach { driver => driver.worker = Some(worker) driver.state = DriverState.RUNNING - worker.drivers(driverId) = driver + worker.addDriver(driver) } } case None => @@ -350,44 +376,42 @@ private[deploy] class Master( } if (canCompleteRecovery) { completeRecovery() } - } + + case WorkerLatestState(workerId, executors, driverIds) => + idToWorker.get(workerId) match { + case Some(worker) => + for (exec <- executors) { + val executorMatches = worker.executors.exists { + case (_, e) => e.application.id == exec.appId && e.id == exec.execId + } + if (!executorMatches) { + // master doesn't recognize this executor. So just tell worker to kill it. + worker.endpoint.send(KillExecutor(masterUrl, exec.appId, exec.execId)) + } + } + + for (driverId <- driverIds) { + val driverMatches = worker.drivers.exists { case (id, _) => id == driverId } + if (!driverMatches) { + // master doesn't recognize this driver. So just tell worker to kill it. + worker.endpoint.send(KillDriver(driverId)) + } + } + case None => + logWarning("Worker state from unknown worker: " + workerId) + } case UnregisterApplication(applicationId) => logInfo(s"Received unregister request from application $applicationId") idToApp.get(applicationId).foreach(finishApplication) - case CheckForWorkerTimeOut => { + case CheckForWorkerTimeOut => timeOutDeadWorkers() - } + } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RegisterWorker( - id, workerHost, workerPort, workerRef, cores, memory, workerUiPort, publicAddress) => { - logInfo("Registering worker %s:%d with %d cores, %s RAM".format( - workerHost, workerPort, cores, Utils.megabytesToString(memory))) - if (state == RecoveryState.STANDBY) { - context.reply(MasterInStandby) - } else if (idToWorker.contains(id)) { - context.reply(RegisterWorkerFailed("Duplicate worker ID")) - } else { - val worker = new WorkerInfo(id, workerHost, workerPort, cores, memory, - workerRef, workerUiPort, publicAddress) - if (registerWorker(worker)) { - persistenceEngine.addWorker(worker) - context.reply(RegisteredWorker(self, masterWebUiUrl)) - schedule() - } else { - val workerAddress = worker.endpoint.address - logWarning("Worker registration failed. Attempted to re-register worker at same " + - "address: " + workerAddress) - context.reply(RegisterWorkerFailed("Attempted to re-register worker at same address: " - + workerAddress)) - } - } - } - - case RequestSubmitDriver(description) => { + case RequestSubmitDriver(description) => if (state != RecoveryState.ALIVE) { val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " + "Can only accept driver submissions in ALIVE state." @@ -406,9 +430,8 @@ private[deploy] class Master( context.reply(SubmitDriverResponse(self, true, Some(driver.id), s"Driver successfully submitted as ${driver.id}")) } - } - case RequestKillDriver(driverId) => { + case RequestKillDriver(driverId) => if (state != RecoveryState.ALIVE) { val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " + s"Can only kill drivers in ALIVE state." @@ -439,9 +462,8 @@ private[deploy] class Master( context.reply(KillDriverResponse(self, driverId, success = false, msg)) } } - } - case RequestDriverStatus(driverId) => { + case RequestDriverStatus(driverId) => if (state != RecoveryState.ALIVE) { val msg = s"${Utils.BACKUP_STANDALONE_MASTER_PREFIX}: $state. " + "Can only request driver status in ALIVE state." @@ -456,18 +478,15 @@ private[deploy] class Master( context.reply(DriverStatusResponse(found = false, None, None, None, None)) } } - } - case RequestMasterState => { + case RequestMasterState => context.reply(MasterStateResponse( address.host, address.port, restServerBoundPort, workers.toArray, apps.toArray, completedApps.toArray, drivers.toArray, completedDrivers.toArray, state)) - } - case BoundPortsRequest => { + case BoundPortsRequest => context.reply(BoundPortsResponse(address.port, webUi.boundPort, restServerBoundPort)) - } case RequestExecutors(appId, requestedTotal) => context.reply(handleRequestExecutors(appId, requestedTotal)) @@ -480,7 +499,7 @@ private[deploy] class Master( override def onDisconnected(address: RpcAddress): Unit = { // The disconnected client could've been either a worker or an app; remove whichever it was logInfo(s"$address got disassociated, removing it.") - addressToWorker.get(address).foreach(removeWorker) + addressToWorker.get(address).foreach(removeWorker(_, s"${address} got disassociated")) addressToApp.get(address).foreach(finishApplication) if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() } } @@ -526,9 +545,13 @@ private[deploy] class Master( state = RecoveryState.COMPLETING_RECOVERY // Kill off any workers and apps that didn't respond to us. - workers.filter(_.state == WorkerState.UNKNOWN).foreach(removeWorker) + workers.filter(_.state == WorkerState.UNKNOWN).foreach( + removeWorker(_, "Not responding for recovery")) apps.filter(_.state == ApplicationState.UNKNOWN).foreach(finishApplication) + // Update the state of recovered apps to RUNNING + apps.filter(_.state == ApplicationState.WAITING).foreach(_.state = ApplicationState.RUNNING) + // Reschedule drivers which were not claimed by any workers drivers.filter(_.worker.isEmpty).foreach { d => logWarning(s"Driver ${d.id} was not found after master recovery") @@ -637,19 +660,22 @@ private[deploy] class Master( private def startExecutorsOnWorkers(): Unit = { // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app // in the queue, then the second app, etc. - for (app <- waitingApps if app.coresLeft > 0) { - val coresPerExecutor: Option[Int] = app.desc.coresPerExecutor - // Filter out workers that don't have enough resources to launch an executor - val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) - .filter(worker => worker.memoryFree >= app.desc.memoryPerExecutorMB && - worker.coresFree >= coresPerExecutor.getOrElse(1)) - .sortBy(_.coresFree).reverse - val assignedCores = scheduleExecutorsOnWorkers(app, usableWorkers, spreadOutApps) - - // Now that we've decided how many cores to allocate on each worker, let's allocate them - for (pos <- 0 until usableWorkers.length if assignedCores(pos) > 0) { - allocateWorkerResourceToExecutors( - app, assignedCores(pos), coresPerExecutor, usableWorkers(pos)) + for (app <- waitingApps) { + val coresPerExecutor = app.desc.coresPerExecutor.getOrElse(1) + // If the cores left is less than the coresPerExecutor,the cores left will not be allocated + if (app.coresLeft >= coresPerExecutor) { + // Filter out workers that don't have enough resources to launch an executor + val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) + .filter(worker => worker.memoryFree >= app.desc.memoryPerExecutorMB && + worker.coresFree >= coresPerExecutor) + .sortBy(_.coresFree).reverse + val assignedCores = scheduleExecutorsOnWorkers(app, usableWorkers, spreadOutApps) + + // Now that we've decided how many cores to allocate on each worker, let's allocate them + for (pos <- 0 until usableWorkers.length if assignedCores(pos) > 0) { + allocateWorkerResourceToExecutors( + app, assignedCores(pos), app.desc.coresPerExecutor, usableWorkers(pos)) + } } } } @@ -683,15 +709,28 @@ private[deploy] class Master( * every time a new app joins or resource availability changes. */ private def schedule(): Unit = { - if (state != RecoveryState.ALIVE) { return } + if (state != RecoveryState.ALIVE) { + return + } // Drivers take strict precedence over executors - val shuffledWorkers = Random.shuffle(workers) // Randomization helps balance drivers - for (worker <- shuffledWorkers if worker.state == WorkerState.ALIVE) { - for (driver <- waitingDrivers) { + val shuffledAliveWorkers = Random.shuffle(workers.toSeq.filter(_.state == WorkerState.ALIVE)) + val numWorkersAlive = shuffledAliveWorkers.size + var curPos = 0 + for (driver <- waitingDrivers.toList) { // iterate over a copy of waitingDrivers + // We assign workers to each waiting driver in a round-robin fashion. For each driver, we + // start from the last worker that was assigned a driver, and continue onwards until we have + // explored all alive workers. + var launched = false + var numWorkersVisited = 0 + while (numWorkersVisited < numWorkersAlive && !launched) { + val worker = shuffledAliveWorkers(curPos) + numWorkersVisited += 1 if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { launchDriver(worker, driver) waitingDrivers -= driver + launched = true } + curPos = (curPos + 1) % numWorkersAlive } } startExecutorsOnWorkers() @@ -702,8 +741,8 @@ private[deploy] class Master( worker.addExecutor(exec) worker.endpoint.send(LaunchExecutor(masterUrl, exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory)) - exec.application.driver.send(ExecutorAdded( - exec.id, worker.id, worker.hostPort, exec.cores, exec.memory)) + exec.application.driver.send( + ExecutorAdded(exec.id, worker.id, worker.hostPort, exec.cores, exec.memory)) } private def registerWorker(worker: WorkerInfo): Boolean = { @@ -721,7 +760,7 @@ private[deploy] class Master( if (oldWorker.state == WorkerState.UNKNOWN) { // A worker registering from UNKNOWN implies that the worker was restarted during recovery. // The old worker must thus be dead, so we will remove it and accept the new worker. - removeWorker(oldWorker) + removeWorker(oldWorker, "Worker replaced by a new worker with same address") } else { logInfo("Attempted to re-register worker at same address: " + workerAddress) return false @@ -734,15 +773,17 @@ private[deploy] class Master( true } - private def removeWorker(worker: WorkerInfo) { + private def removeWorker(worker: WorkerInfo, msg: String) { logInfo("Removing worker " + worker.id + " on " + worker.host + ":" + worker.port) worker.setState(WorkerState.DEAD) idToWorker -= worker.id addressToWorker -= worker.endpoint.address + for (exec <- worker.executors.values) { logInfo("Telling app of lost executor: " + exec.id) exec.application.driver.send(ExecutorUpdated( - exec.id, ExecutorState.LOST, Some("worker lost"), None)) + exec.id, ExecutorState.LOST, Some("worker lost"), None, workerLost = true)) + exec.state = ExecutorState.LOST exec.application.removeExecutor(exec) } for (driver <- worker.drivers.values) { @@ -754,13 +795,27 @@ private[deploy] class Master( removeDriver(driver.id, DriverState.ERROR, None) } } + logInfo(s"Telling app of lost worker: " + worker.id) + apps.filterNot(completedApps.contains(_)).foreach { app => + app.driver.send(WorkerRemoved(worker.id, worker.host, msg)) + } persistenceEngine.removeWorker(worker) } private def relaunchDriver(driver: DriverInfo) { - driver.worker = None - driver.state = DriverState.RELAUNCHING - waitingDrivers += driver + // We must setup a new driver with a new driver id here, because the original driver may + // be still running. Consider this scenario: a worker is network partitioned with master, + // the master then relaunches driver driverID1 with a driver id driverID2, then the worker + // reconnects to master. From this point on, if driverID2 is equal to driverID1, then master + // can not distinguish the statusUpdate of the original driver and the newly relaunched one, + // for example, when DriverStateChanged(driverID1, KILLED) arrives at master, master will + // remove driverID1, so the newly relaunched driver disappears too. See SPARK-19900 for details. + removeDriver(driver.id, DriverState.RELAUNCHING, None) + val newDriver = createDriver(driver.desc) + persistenceEngine.addDriver(newDriver) + drivers.add(newDriver) + waitingDrivers += newDriver + schedule() } @@ -798,20 +853,17 @@ private[deploy] class Master( idToApp -= app.id endpointToApp -= app.driver addressToApp -= app.driver.address + if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) - completedApps.take(toRemove).foreach( a => { - appIdToUI.remove(a.id).foreach { ui => webUi.detachSparkUI(ui) } + completedApps.take(toRemove).foreach { a => applicationMetricsSystem.removeSource(a.appSource) - }) + } completedApps.trimStart(toRemove) } completedApps += app // Remember it in our history waitingApps -= app - // If application events are logged, use them to rebuild the UI - rebuildSparkUI(app) - for (exec <- app.executors.values) { killExecutor(exec) } @@ -910,77 +962,7 @@ private[deploy] class Master( exec.state = ExecutorState.KILLED } - /** - * Rebuild a new SparkUI from the given application's event logs. - * Return the UI if successful, else None - */ - private[master] def rebuildSparkUI(app: ApplicationInfo): Option[SparkUI] = { - val appName = app.desc.name - val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" - try { - val eventLogDir = app.desc.eventLogDir - .getOrElse { - // Event logging is not enabled for this application - app.appUIUrlAtHistoryServer = Some(notFoundBasePath) - return None - } - - val eventLogFilePrefix = EventLoggingListener.getLogPath( - eventLogDir, app.id, app.desc.eventLogCodec) - val fs = Utils.getHadoopFileSystem(eventLogDir, hadoopConf) - val inProgressExists = fs.exists(new Path(eventLogFilePrefix + - EventLoggingListener.IN_PROGRESS)) - - if (inProgressExists) { - // Event logging is enabled for this application, but the application is still in progress - logWarning(s"Application $appName is still in progress, it may be terminated abnormally.") - } - - val (eventLogFile, status) = if (inProgressExists) { - (eventLogFilePrefix + EventLoggingListener.IN_PROGRESS, " (in progress)") - } else { - (eventLogFilePrefix, " (completed)") - } - - val logInput = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) - val replayBus = new ReplayListenerBus() - val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), - appName, HistoryServer.UI_PATH_PREFIX + s"/${app.id}", app.startTime) - val maybeTruncated = eventLogFile.endsWith(EventLoggingListener.IN_PROGRESS) - try { - replayBus.replay(logInput, eventLogFile, maybeTruncated) - } finally { - logInput.close() - } - appIdToUI(app.id) = ui - webUi.attachSparkUI(ui) - // Application UI is successfully rebuilt, so link the Master UI to it - app.appUIUrlAtHistoryServer = Some(ui.basePath) - Some(ui) - } catch { - case fnf: FileNotFoundException => - // Event logging is enabled for this application, but no event logs are found - val title = s"Application history not found (${app.id})" - var msg = s"No event logs found for application $appName in ${app.desc.eventLogDir.get}." - logWarning(msg) - msg += " Did you specify the correct logging directory?" - msg = URLEncoder.encode(msg, "UTF-8") - app.appUIUrlAtHistoryServer = Some(notFoundBasePath + s"?msg=$msg&title=$title") - None - case e: Exception => - // Relay exception message to application UI page - val title = s"Application history load error (${app.id})" - val exception = URLEncoder.encode(Utils.exceptionString(e), "UTF-8") - var msg = s"Exception in replaying log for application $appName!" - logError(msg, e) - msg = URLEncoder.encode(msg, "UTF-8") - app.appUIUrlAtHistoryServer = - Some(notFoundBasePath + s"?msg=$msg&exception=$exception&title=$title") - None - } - } - - /** Generate a new app ID given a app's submission date */ + /** Generate a new app ID given an app's submission date */ private def newApplicationId(submitDate: Date): String = { val appId = "app-%s-%04d".format(createDateFormat.format(submitDate), nextAppNumber) nextAppNumber += 1 @@ -996,7 +978,7 @@ private[deploy] class Master( if (worker.state != WorkerState.DEAD) { logWarning("Removing %s because we got no heartbeat in %d seconds".format( worker.id, WORKER_TIMEOUT_MS / 1000)) - removeWorker(worker) + removeWorker(worker, s"Not receiving heartbeat for ${WORKER_TIMEOUT_MS / 1000} seconds") } else { if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT_MS)) { workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it @@ -1054,7 +1036,9 @@ private[deploy] object Master extends Logging { val ENDPOINT_NAME = "Master" def main(argStrings: Array[String]) { - SignalLogger.register(log) + Thread.setDefaultUncaughtExceptionHandler(new SparkUncaughtExceptionHandler( + exitOnUncaughtException = false)) + Utils.initDaemon(log) val conf = new SparkConf val args = new MasterArguments(argStrings, conf) val (rpcEnv, _, _) = startRpcEnvAndEndpoint(args.host, args.port, args.webUiPort, conf) @@ -1076,7 +1060,7 @@ private[deploy] object Master extends Logging { val rpcEnv = RpcEnv.create(SYSTEM_NAME, host, port, conf, securityMgr) val masterEndpoint = rpcEnv.setupEndpoint(ENDPOINT_NAME, new Master(rpcEnv, rpcEnv.address, webUiPort, securityMgr, conf)) - val portsResponse = masterEndpoint.askWithRetry[BoundPortsResponse](BoundPortsRequest) + val portsResponse = masterEndpoint.askSync[BoundPortsResponse](BoundPortsRequest) (rpcEnv, portsResponse.webUIPort, portsResponse.restPort) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index 44cefbc77f08..615d2533cf08 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -17,19 +17,27 @@ package org.apache.spark.deploy.master +import scala.annotation.tailrec + import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.util.{IntParam, Utils} /** * Command-line parser for the master. */ -private[master] class MasterArguments(args: Array[String], conf: SparkConf) { +private[master] class MasterArguments(args: Array[String], conf: SparkConf) extends Logging { var host = Utils.localHostName() var port = 7077 var webUiPort = 8080 var propertiesFile: String = null // Check for settings in environment variables + if (System.getenv("SPARK_MASTER_IP") != null) { + logWarning("SPARK_MASTER_IP is deprecated, please use SPARK_MASTER_HOST") + host = System.getenv("SPARK_MASTER_IP") + } + if (System.getenv("SPARK_MASTER_HOST") != null) { host = System.getenv("SPARK_MASTER_HOST") } @@ -49,14 +57,15 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) { webUiPort = conf.get("spark.master.ui.port").toInt } + @tailrec private def parse(args: List[String]): Unit = args match { case ("--ip" | "-i") :: value :: tail => - Utils.checkHost(value, "ip no longer supported, please use hostname " + value) + Utils.checkHost(value) host = value parse(tail) case ("--host" | "-h") :: value :: tail => - Utils.checkHost(value, "Please use hostname " + value) + Utils.checkHost(value) host = value parse(tail) @@ -75,7 +84,7 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) { case ("--help") :: tail => printUsageAndExit(0) - case Nil => {} + case Nil => // No-op case _ => printUsageAndExit(1) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala index 66a9ff38678c..fb07c39dd02e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala @@ -32,7 +32,7 @@ private[spark] class MasterSource(val master: Master) extends Source { // Gauge for alive worker numbers in cluster metricRegistry.register(MetricRegistry.name("aliveWorkers"), new Gauge[Int]{ - override def getValue: Int = master.workers.filter(_.state == WorkerState.ALIVE).size + override def getValue: Int = master.workers.count(_.state == WorkerState.ALIVE) }) // Gauge for application numbers in cluster @@ -42,6 +42,6 @@ private[spark] class MasterSource(val master: Master) extends Source { // Gauge for waiting application numbers in cluster metricRegistry.register(MetricRegistry.name("waitingApps"), new Gauge[Int] { - override def getValue: Int = master.waitingApps.size + override def getValue: Int = master.apps.count(_.state == ApplicationState.WAITING) }) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala index 58a00bceee6a..b30bc821b732 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala @@ -17,11 +17,11 @@ package org.apache.spark.deploy.master +import scala.reflect.ClassTag + import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rpc.RpcEnv -import scala.reflect.ClassTag - /** * Allows Master to persist any state that is necessary in order to recover from a failure. * The following semantics are required: @@ -40,12 +40,12 @@ abstract class PersistenceEngine { * Defines how the object is serialized and persisted. Implementation will * depend on the store used. */ - def persist(name: String, obj: Object) + def persist(name: String, obj: Object): Unit /** * Defines how the object referred by its name is removed from the store. */ - def unpersist(name: String) + def unpersist(name: String): Unit /** * Gives all objects, matching a prefix. This defines how objects are diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index c4c3283fb73f..ffdd635be4f5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -17,8 +17,9 @@ package org.apache.spark.deploy.master -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.serializer.Serializer /** diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index f75196660520..c87d6e24b78c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -29,11 +29,10 @@ private[spark] class WorkerInfo( val cores: Int, val memory: Int, val endpoint: RpcEndpointRef, - val webUiPort: Int, - val publicAddress: String) + val webUiAddress: String) extends Serializable { - Utils.checkHost(host, "Expected hostname") + Utils.checkHost(host) assert (port > 0) @transient var executors: mutable.HashMap[String, ExecutorDesc] = _ // executorId => info @@ -98,10 +97,6 @@ private[spark] class WorkerInfo( coresUsed -= driver.desc.cores } - def webUiAddress : String = { - "http://" + this.publicAddress + ":" + this.webUiPort - } - def setState(state: WorkerState.Value): Unit = { this.state = state } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index d317206a614f..1e8dabfbe6b0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -17,10 +17,12 @@ package org.apache.spark.deploy.master -import org.apache.spark.{Logging, SparkConf} import org.apache.curator.framework.CuratorFramework -import org.apache.curator.framework.recipes.leader.{LeaderLatchListener, LeaderLatch} +import org.apache.curator.framework.recipes.leader.{LeaderLatch, LeaderLatchListener} + +import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkCuratorUtil +import org.apache.spark.internal.Logging private[master] class ZooKeeperLeaderElectionAgent(val masterInstance: LeaderElectable, conf: SparkConf) extends LeaderLatchListener with LeaderElectionAgent with Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 540e802420ce..af850e4871e5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -25,8 +25,9 @@ import scala.reflect.ClassTag import org.apache.curator.framework.CuratorFramework import org.apache.zookeeper.CreateMode -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkCuratorUtil +import org.apache.spark.internal.Logging import org.apache.spark.serializer.Serializer @@ -50,7 +51,7 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializer override def read[T: ClassTag](prefix: String): Seq[T] = { zk.getChildren.forPath(WORKING_DIR).asScala - .filter(_.startsWith(prefix)).map(deserializeFromFile[T]).flatten + .filter(_.startsWith(prefix)).flatMap(deserializeFromFile[T]) } override def close() { @@ -69,11 +70,10 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializer try { Some(serializer.newInstance().deserialize[T](ByteBuffer.wrap(fileData))) } catch { - case e: Exception => { + case e: Exception => logWarning("Exception while reading persisted file, deleting", e) zk.delete().forPath(WORKING_DIR + "/" + filename) None - } } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index f405aa2bdc8b..68e57b7564ad 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -21,10 +21,10 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.deploy.ExecutorState import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} +import org.apache.spark.deploy.ExecutorState import org.apache.spark.deploy.master.ExecutorDesc -import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} import org.apache.spark.util.Utils private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") { @@ -33,11 +33,11 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") /** Executor details for a particular application */ def render(request: HttpServletRequest): Seq[Node] = { - val appId = request.getParameter("appId") - val state = master.askWithRetry[MasterStateResponse](RequestMasterState) - val app = state.activeApps.find(_.id == appId).getOrElse({ - state.completedApps.find(_.id == appId).getOrElse(null) - }) + // stripXSS is called first to remove suspicious characters used in XSS attacks + val appId = UIUtils.stripXSS(request.getParameter("appId")) + val state = master.askSync[MasterStateResponse](RequestMasterState) + val app = state.activeApps.find(_.id == appId) + .getOrElse(state.completedApps.find(_.id == appId).orNull) if (app == null) { val msg =
    No running application with ID {appId}
    return UIUtils.basicSparkPage(msg, "Not Found") @@ -70,24 +70,41 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") } } +
  • + + Executor Limit: + { + if (app.executorLimit == Int.MaxValue) "Unlimited" else app.executorLimit + } + ({app.executors.size} granted) + +
  • Executor Memory: {Utils.megabytesToString(app.desc.memoryPerExecutorMB)}
  • -
  • Submit Date: {app.submitDate}
  • +
  • Submit Date: {UIUtils.formatDate(app.submitDate)}
  • State: {app.state}
  • -
  • Application Detail UI
  • + { + if (!app.isFinished) { +
  • + Application Detail UI +
  • + } + }
-

Executor Summary

+

Executor Summary ({allExecutors.length})

{executorsTable} { if (removedExecutors.nonEmpty) { -

Removed Executors

++ +

Removed Executors ({removedExecutors.length})

++ removedExecutorsTable } } @@ -97,19 +114,21 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") } private def executorRow(executor: ExecutorDesc): Seq[Node] = { + val workerUrlRef = UIUtils.makeHref(parent.master.reverseProxy, + executor.worker.id, executor.worker.webUiAddress) {executor.id} - {executor.worker.id} + {executor.worker.id} {executor.cores} {executor.memory} {executor.state} - stdout - stderr + stdout + stderr } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala deleted file mode 100644 index e021f1eef794..000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.master.ui - -import java.net.URLDecoder -import javax.servlet.http.HttpServletRequest - -import scala.xml.Node - -import org.apache.spark.ui.{UIUtils, WebUIPage} - -private[ui] class HistoryNotFoundPage(parent: MasterWebUI) - extends WebUIPage("history/not-found") { - - /** - * Render a page that conveys failure in loading application history. - * - * This accepts 3 HTTP parameters: - * msg = message to display to the user - * title = title of the page - * exception = detailed description of the exception in loading application history (if any) - * - * Parameters "msg" and "exception" are assumed to be UTF-8 encoded. - */ - def render(request: HttpServletRequest): Seq[Node] = { - val titleParam = request.getParameter("title") - val msgParam = request.getParameter("msg") - val exceptionParam = request.getParameter("exception") - - // If no parameters are specified, assume the user did not enable event logging - val defaultTitle = "Event logging is not enabled" - val defaultContent = -
-
- No event logs were found for this application! To - enable event logging, - set spark.eventLog.enabled to true and - spark.eventLog.dir to the directory to which your - event logs are written. -
-
- - val title = Option(titleParam).getOrElse(defaultTitle) - val content = Option(msgParam) - .map { msg => URLDecoder.decode(msg, "UTF-8") } - .map { msg => -
-
{msg}
-
++ - Option(exceptionParam) - .map { e => URLDecoder.decode(e, "UTF-8") } - .map { e =>
{e}
} - .getOrElse(Seq.empty) - }.getOrElse(defaultContent) - - UIUtils.basicSparkPage(content, title) - } -} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index ee539dd1f511..bc0bf6a1d970 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -23,17 +23,17 @@ import scala.xml.Node import org.json4s.JValue +import org.apache.spark.deploy.DeployMessages.{KillDriverResponse, MasterStateResponse, RequestKillDriver, RequestMasterState} import org.apache.spark.deploy.JsonProtocol -import org.apache.spark.deploy.DeployMessages.{KillDriverResponse, RequestKillDriver, MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master._ -import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private val master = parent.masterEndpointRef def getMasterState: MasterStateResponse = { - master.askWithRetry[MasterStateResponse](RequestMasterState) + master.askSync[MasterStateResponse](RequestMasterState) } override def renderJson(request: HttpServletRequest): JValue = { @@ -57,8 +57,10 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private def handleKillRequest(request: HttpServletRequest, action: String => Unit): Unit = { if (parent.killEnabled && parent.master.securityMgr.checkModifyPermissions(request.getRemoteUser)) { - val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean - val id = Option(request.getParameter("id")) + // stripXSS is called first to remove suspicious characters used in XSS attacks + val killFlag = + Option(UIUtils.stripXSS(request.getParameter("terminate"))).getOrElse("false").toBoolean + val id = Option(UIUtils.stripXSS(request.getParameter("id"))) if (id.isDefined && killFlag) { action(id.get) } @@ -76,7 +78,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { val aliveWorkers = state.workers.filter(_.state == WorkerState.ALIVE) val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers) - val appHeaders = Seq("Application ID", "Name", "Cores", "Memory per Node", "Submitted Time", + val appHeaders = Seq("Application ID", "Name", "Cores", "Memory per Executor", "Submitted Time", "User", "State", "Duration") val activeApps = state.activeApps.sortBy(_.startTime).reverse val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps) @@ -107,18 +109,18 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { }.getOrElse { Seq.empty } } -
  • Alive Workers: {aliveWorkers.size}
  • +
  • Alive Workers: {aliveWorkers.length}
  • Cores in use: {aliveWorkers.map(_.cores).sum} Total, {aliveWorkers.map(_.coresUsed).sum} Used
  • Memory in use: {Utils.megabytesToString(aliveWorkers.map(_.memory).sum)} Total, {Utils.megabytesToString(aliveWorkers.map(_.memoryUsed).sum)} Used
  • Applications: - {state.activeApps.size} Running, - {state.completedApps.size} Completed
  • + {state.activeApps.length} Running, + {state.completedApps.length} Completed
  • Drivers: - {state.activeDrivers.size} Running, - {state.completedDrivers.size} Completed
  • + {state.activeDrivers.length} Running, + {state.completedDrivers.length} Completed
  • Status: {state.status}
  • @@ -126,14 +128,14 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
    -

    Workers

    +

    Workers ({workers.length})

    {workerTable}
    -

    Running Applications

    +

    Running Applications ({activeApps.length})

    {activeAppsTable}
    @@ -142,7 +144,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {if (hasDrivers) {
    -

    Running Drivers

    +

    Running Drivers ({activeDrivers.length})

    {activeDriversTable}
    @@ -152,7 +154,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
    -

    Completed Applications

    +

    Completed Applications ({completedApps.length})

    {completedAppsTable}
    @@ -162,7 +164,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { if (hasDrivers) {
    -

    Completed Drivers

    +

    Completed Drivers ({completedDrivers.length})

    {completedDriversTable}
    @@ -176,7 +178,15 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private def workerRow(worker: WorkerInfo): Seq[Node] = { - {worker.id} + { + if (worker.isAlive()) { + + {worker.id} + + } else { + worker.id + } + } {worker.host}:{worker.port} {worker.state} @@ -206,7 +216,14 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {killLink} - {app.desc.name} + { + if (app.isFinished) { + app.desc.name + } else { + {app.desc.name} + } + } {app.coresGranted} @@ -237,8 +254,15 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { } {driver.id} {killLink} - {driver.submitDate} - {driver.worker.map(w => {w.id.toString}).getOrElse("None")} + {UIUtils.formatDate(driver.submitDate)} + {driver.worker.map(w => + if (w.isAlive()) { + + {w.id.toString} + + } else { + w.id.toString + }).getOrElse("None")} {driver.state} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 6174fc11f83d..35b7ddd46e4d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -17,10 +17,9 @@ package org.apache.spark.deploy.master.ui -import org.apache.spark.Logging +import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.Master -import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationsListResource, ApplicationInfo, - UIRoot} +import org.apache.spark.internal.Logging import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ @@ -28,59 +27,42 @@ import org.apache.spark.ui.JettyUtils._ * Web UI server for the standalone master. */ private[master] -class MasterWebUI(val master: Master, requestedPort: Int) - extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging - with UIRoot { +class MasterWebUI( + val master: Master, + requestedPort: Int) + extends WebUI(master.securityMgr, master.securityMgr.getSSLOptions("standalone"), + requestedPort, master.conf, name = "MasterUI") with Logging { val masterEndpointRef = master.self val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true) - val masterPage = new MasterPage(this) - initialize() /** Initialize all components of the server. */ def initialize() { val masterPage = new MasterPage(this) attachPage(new ApplicationPage(this)) - attachPage(new HistoryNotFoundPage(this)) attachPage(masterPage) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler(ApiRootResource.getServletHandler(this)) attachHandler(createRedirectHandler( "/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST"))) attachHandler(createRedirectHandler( "/driver/kill", "/", masterPage.handleDriverKillRequest, httpMethods = Set("POST"))) } - /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ - def attachSparkUI(ui: SparkUI) { - assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") - ui.getHandlers.foreach(attachHandler) + def addProxy(): Unit = { + val handler = createProxyHandler(idToUiAddress) + attachHandler(handler) } - /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ - def detachSparkUI(ui: SparkUI) { - assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") - ui.getHandlers.foreach(detachHandler) - } + def idToUiAddress(id: String): Option[String] = { + val state = masterEndpointRef.askSync[MasterStateResponse](RequestMasterState) + val maybeWorkerUiAddress = state.workers.find(_.id == id).map(_.webUiAddress) + val maybeAppUiAddress = state.activeApps.find(_.id == id).map(_.desc.appUiUrl) - def getApplicationInfoList: Iterator[ApplicationInfo] = { - val state = masterPage.getMasterState - val activeApps = state.activeApps.sortBy(_.startTime).reverse - val completedApps = state.completedApps.sortBy(_.endTime).reverse - activeApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, false) } ++ - completedApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, true) } + maybeWorkerUiAddress.orElse(maybeAppUiAddress) } - def getSparkUI(appId: String): Option[SparkUI] = { - val state = masterPage.getMasterState - val activeApps = state.activeApps.sortBy(_.startTime).reverse - val completedApps = state.completedApps.sortBy(_.endTime).reverse - (activeApps ++ completedApps).find { _.id == appId }.flatMap { - master.rebuildSparkUI - } - } } private[master] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala deleted file mode 100644 index 5accaf78d0a5..000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.mesos - -import org.apache.spark.SparkConf -import org.apache.spark.util.{IntParam, Utils} - - -private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf: SparkConf) { - var host = Utils.localHostName() - var port = 7077 - var name = "Spark Cluster" - var webUiPort = 8081 - var masterUrl: String = _ - var zookeeperUrl: Option[String] = None - var propertiesFile: String = _ - - parse(args.toList) - - propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) - - private def parse(args: List[String]): Unit = args match { - case ("--host" | "-h") :: value :: tail => - Utils.checkHost(value, "Please use hostname " + value) - host = value - parse(tail) - - case ("--port" | "-p") :: IntParam(value) :: tail => - port = value - parse(tail) - - case ("--webui-port" | "-p") :: IntParam(value) :: tail => - webUiPort = value - parse(tail) - - case ("--zk" | "-z") :: value :: tail => - zookeeperUrl = Some(value) - parse(tail) - - case ("--master" | "-m") :: value :: tail => - if (!value.startsWith("mesos://")) { - // scalastyle:off println - System.err.println("Cluster dispatcher only supports mesos (uri begins with mesos://)") - // scalastyle:on println - System.exit(1) - } - masterUrl = value.stripPrefix("mesos://") - parse(tail) - - case ("--name") :: value :: tail => - name = value - parse(tail) - - case ("--properties-file") :: value :: tail => - propertiesFile = value - parse(tail) - - case ("--help") :: tail => - printUsageAndExit(0) - - case Nil => { - if (masterUrl == null) { - // scalastyle:off println - System.err.println("--master is required") - // scalastyle:on println - printUsageAndExit(1) - } - } - - case _ => - printUsageAndExit(1) - } - - private def printUsageAndExit(exitCode: Int): Unit = { - // scalastyle:off println - System.err.println( - "Usage: MesosClusterDispatcher [options]\n" + - "\n" + - "Options:\n" + - " -h HOST, --host HOST Hostname to listen on\n" + - " -p PORT, --port PORT Port to listen on (default: 7077)\n" + - " --webui-port WEBUI_PORT WebUI Port to listen on (default: 8081)\n" + - " --name NAME Framework name to show in Mesos UI\n" + - " -m --master MASTER URI for connecting to Mesos master\n" + - " -z --zk ZOOKEEPER Comma delimited URLs for connecting to \n" + - " Zookeeper for persistence\n" + - " --properties-file FILE Path to a custom Spark properties file.\n" + - " Default is conf/spark-defaults.conf.") - // scalastyle:on println - System.exit(exitCode) - } -} diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala deleted file mode 100644 index 12337a940a41..000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.mesos - -import java.net.SocketAddress - -import scala.collection.mutable - -import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.deploy.ExternalShuffleService -import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} -import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage -import org.apache.spark.network.shuffle.protocol.mesos.RegisterDriver -import org.apache.spark.network.util.TransportConf - -/** - * An RPC endpoint that receives registration requests from Spark drivers running on Mesos. - * It detects driver termination and calls the cleanup callback to [[ExternalShuffleService]]. - */ -private[mesos] class MesosExternalShuffleBlockHandler(transportConf: TransportConf) - extends ExternalShuffleBlockHandler(transportConf, null) with Logging { - - // Stores a map of driver socket addresses to app ids - private val connectedApps = new mutable.HashMap[SocketAddress, String] - - protected override def handleMessage( - message: BlockTransferMessage, - client: TransportClient, - callback: RpcResponseCallback): Unit = { - message match { - case RegisterDriverParam(appId) => - val address = client.getSocketAddress - logDebug(s"Received registration request from app $appId (remote address $address).") - if (connectedApps.contains(address)) { - val existingAppId = connectedApps(address) - if (!existingAppId.equals(appId)) { - logError(s"A new app '$appId' has connected to existing address $address, " + - s"removing previously registered app '$existingAppId'.") - applicationRemoved(existingAppId, true) - } - } - connectedApps(address) = appId - callback.onSuccess(new Array[Byte](0)) - case _ => super.handleMessage(message, client, callback) - } - } - - /** - * On connection termination, clean up shuffle files written by the associated application. - */ - override def connectionTerminated(client: TransportClient): Unit = { - val address = client.getSocketAddress - if (connectedApps.contains(address)) { - val appId = connectedApps(address) - logInfo(s"Application $appId disconnected (address was $address).") - applicationRemoved(appId, true /* cleanupLocalDirs */) - connectedApps.remove(address) - } else { - logWarning(s"Unknown $address disconnected.") - } - } - - /** An extractor object for matching [[RegisterDriver]] message. */ - private object RegisterDriverParam { - def unapply(r: RegisterDriver): Option[String] = Some(r.getAppId) - } -} - -/** - * A wrapper of [[ExternalShuffleService]] that provides an additional endpoint for drivers - * to associate with. This allows the shuffle service to detect when a driver is terminated - * and can clean up the associated shuffle files. - */ -private[mesos] class MesosExternalShuffleService(conf: SparkConf, securityManager: SecurityManager) - extends ExternalShuffleService(conf, securityManager) { - - protected override def newShuffleBlockHandler( - conf: TransportConf): ExternalShuffleBlockHandler = { - new MesosExternalShuffleBlockHandler(conf) - } -} - -private[spark] object MesosExternalShuffleService extends Logging { - - def main(args: Array[String]): Unit = { - ExternalShuffleService.main(args, - (conf: SparkConf, sm: SecurityManager) => new MesosExternalShuffleService(conf, sm)) - } -} - - diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala deleted file mode 100644 index 7419fa969964..000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.mesos.ui - -import javax.servlet.http.HttpServletRequest - -import scala.xml.Node - -import org.apache.mesos.Protos.TaskStatus -import org.apache.spark.deploy.mesos.MesosDriverDescription -import org.apache.spark.scheduler.cluster.mesos.MesosClusterSubmissionState -import org.apache.spark.ui.{UIUtils, WebUIPage} - -private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage("") { - def render(request: HttpServletRequest): Seq[Node] = { - val state = parent.scheduler.getSchedulerState() - val queuedHeaders = Seq("Driver ID", "Submit Date", "Main Class", "Driver Resources") - val driverHeaders = queuedHeaders ++ - Seq("Start Date", "Mesos Slave ID", "State") - val retryHeaders = Seq("Driver ID", "Submit Date", "Description") ++ - Seq("Last Failed Status", "Next Retry Time", "Attempt Count") - val queuedTable = UIUtils.listingTable(queuedHeaders, queuedRow, state.queuedDrivers) - val launchedTable = UIUtils.listingTable(driverHeaders, driverRow, state.launchedDrivers) - val finishedTable = UIUtils.listingTable(driverHeaders, driverRow, state.finishedDrivers) - val retryTable = UIUtils.listingTable(retryHeaders, retryRow, state.pendingRetryDrivers) - val content = -

    Mesos Framework ID: {state.frameworkId}

    -
    -
    -

    Queued Drivers:

    - {queuedTable} -

    Launched Drivers:

    - {launchedTable} -

    Finished Drivers:

    - {finishedTable} -

    Supervise drivers waiting for retry:

    - {retryTable} -
    -
    ; - UIUtils.basicSparkPage(content, "Spark Drivers for Mesos cluster") - } - - private def queuedRow(submission: MesosDriverDescription): Seq[Node] = { - val id = submission.submissionId - - {id} - {submission.submissionDate} - {submission.command.mainClass} - cpus: {submission.cores}, mem: {submission.mem} - - } - - private def driverRow(state: MesosClusterSubmissionState): Seq[Node] = { - val id = state.driverDescription.submissionId - - {id} - {state.driverDescription.submissionDate} - {state.driverDescription.command.mainClass} - cpus: {state.driverDescription.cores}, mem: {state.driverDescription.mem} - {state.startDate} - {state.slaveId.getValue} - {stateString(state.mesosTaskStatus)} - - } - - private def retryRow(submission: MesosDriverDescription): Seq[Node] = { - val id = submission.submissionId - - {id} - {submission.submissionDate} - {submission.command.mainClass} - {submission.retryState.get.lastFailureStatus} - {submission.retryState.get.nextRetry} - {submission.retryState.get.retries} - - } - - private def stateString(status: Option[TaskStatus]): String = { - if (status.isEmpty) { - return "" - } - val sb = new StringBuilder - val s = status.get - sb.append(s"State: ${s.getState}") - if (status.get.hasMessage) { - sb.append(s", Message: ${s.getMessage}") - } - if (status.get.hasHealthy) { - sb.append(s", Healthy: ${s.getHealthy}") - } - if (status.get.hasSource) { - sb.append(s", Source: ${s.getSource}") - } - if (status.get.hasReason) { - sb.append(s", Reason: ${s.getReason}") - } - if (status.get.hasTimestamp) { - sb.append(s", Time: ${s.getTimestamp}") - } - sb.toString() - } -} diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index 957a928bc402..21cb94142b15 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -19,15 +19,20 @@ package org.apache.spark.deploy.rest import java.io.{DataOutputStream, FileNotFoundException} import java.net.{ConnectException, HttpURLConnection, SocketException, URL} +import java.nio.charset.StandardCharsets +import java.util.concurrent.TimeoutException import javax.servlet.http.HttpServletResponse import scala.collection.mutable +import scala.concurrent.{Await, Future} +import scala.concurrent.duration._ import scala.io.Source +import scala.util.control.NonFatal import com.fasterxml.jackson.core.JsonProcessingException -import com.google.common.base.Charsets -import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} +import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** @@ -208,7 +213,7 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { try { val out = new DataOutputStream(conn.getOutputStream) Utils.tryWithSafeFinally { - out.write(json.getBytes(Charsets.UTF_8)) + out.write(json.getBytes(StandardCharsets.UTF_8)) } { out.close() } @@ -225,7 +230,8 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { * Exposed for testing. */ private[rest] def readResponse(connection: HttpURLConnection): SubmitRestProtocolResponse = { - try { + import scala.concurrent.ExecutionContext.Implicits.global + val responseFuture = Future { val dataStream = if (connection.getResponseCode == HttpServletResponse.SC_OK) { connection.getInputStream @@ -251,11 +257,19 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { throw new SubmitRestProtocolException( s"Message received from server was not a response:\n${unexpected.toJson}") } - } catch { + } + + // scalastyle:off awaitresult + try { Await.result(responseFuture, 10.seconds) } catch { + // scalastyle:on awaitresult case unreachable @ (_: FileNotFoundException | _: SocketException) => throw new SubmitRestConnectionException("Unable to connect to server", unreachable) case malformed @ (_: JsonProcessingException | _: SubmitRestProtocolException) => throw new SubmitRestProtocolException("Malformed response received from server", malformed) + case timeout: TimeoutException => + throw new SubmitRestConnectionException("No response from server", timeout) + case NonFatal(t) => + throw new SparkException("Exception while waiting for response", t) } } @@ -374,7 +388,7 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { logWarning(s"Unable to connect to server ${masterUrl}.") lostMasters += masterUrl } - lostMasters.size >= masters.size + lostMasters.size >= masters.length } } @@ -404,13 +418,13 @@ private[spark] object RestSubmissionClient { } def main(args: Array[String]): Unit = { - if (args.size < 2) { + if (args.length < 2) { sys.error("Usage: RestSubmissionClient [app resource] [main class] [app args*]") sys.exit(1) } val appResource = args(0) val mainClass = args(1) - val appArgs = args.slice(2, args.size) + val appArgs = args.slice(2, args.length) val conf = new SparkConf val env = filterSystemEnvironment(sys.env) run(appResource, mainClass, appArgs, conf, env) @@ -420,8 +434,10 @@ private[spark] object RestSubmissionClient { * Filter non-spark environment variables from any environment. */ private[rest] def filterSystemEnvironment(env: Map[String, String]): Map[String, String] = { - env.filter { case (k, _) => - (k.startsWith("SPARK_") && k != "SPARK_ENV_LOADED") || k.startsWith("MESOS_") + env.filterKeys { k => + // SPARK_HOME is filtered out because it is usually wrong on the remote machine (SPARK-12345) + (k.startsWith("SPARK_") && k != "SPARK_ENV_LOADED" && k != "SPARK_HOME") || + k.startsWith("MESOS_") } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index 2e78d03e5c0c..e88195d95f27 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -11,24 +11,25 @@ * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and + * See the License for the specific language governing permissions and * limitations under the License. */ package org.apache.spark.deploy.rest -import java.net.InetSocketAddress import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.io.Source + 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 +import org.eclipse.jetty.server.{HttpConnectionFactory, Server, ServerConnector} +import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} +import org.eclipse.jetty.util.thread.{QueuedThreadPool, ScheduledExecutorScheduler} import org.json4s._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} +import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** @@ -78,18 +79,32 @@ private[spark] abstract class RestSubmissionServer( * Return a 2-tuple of the started server and the bound port. */ private def doStart(startPort: Int): (Server, Int) = { - val server = new Server(new InetSocketAddress(host, startPort)) val threadPool = new QueuedThreadPool threadPool.setDaemon(true) - server.setThreadPool(threadPool) + val server = new Server(threadPool) + + val connector = new ServerConnector( + server, + null, + // Call this full constructor to set this, which forces daemon threads: + new ScheduledExecutorScheduler("RestSubmissionServer-JettyScheduler", true), + null, + -1, + -1, + new HttpConnectionFactory()) + connector.setHost(host) + connector.setPort(startPort) + server.addConnector(connector) + val mainHandler = new ServletContextHandler + mainHandler.setServer(server) mainHandler.setContextPath("/") contextToServlet.foreach { case (prefix, servlet) => mainHandler.addServlet(new ServletHolder(servlet), prefix) } server.setHandler(mainHandler) server.start() - val boundPort = server.getConnectors()(0).getLocalPort + val boundPort = connector.getLocalPort (server, boundPort) } 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 d5b9bcab1423..0164084ab129 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 @@ -11,7 +11,7 @@ * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and + * See the License for the specific language governing permissions and * limitations under the License. */ @@ -20,11 +20,11 @@ package org.apache.spark.deploy.rest import java.io.File import javax.servlet.http.HttpServletResponse -import org.apache.spark.deploy.ClientArguments._ +import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription} +import org.apache.spark.deploy.ClientArguments._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils -import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} /** * A server that responds to requests submitted by the [[RestSubmissionClient]]. @@ -71,7 +71,7 @@ private[rest] class StandaloneKillRequestServlet(masterEndpoint: RpcEndpointRef, extends KillRequestServlet { protected def handleKill(submissionId: String): KillSubmissionResponse = { - val response = masterEndpoint.askWithRetry[DeployMessages.KillDriverResponse]( + val response = masterEndpoint.askSync[DeployMessages.KillDriverResponse]( DeployMessages.RequestKillDriver(submissionId)) val k = new KillSubmissionResponse k.serverSparkVersion = sparkVersion @@ -89,7 +89,7 @@ private[rest] class StandaloneStatusRequestServlet(masterEndpoint: RpcEndpointRe extends StatusRequestServlet { protected def handleStatus(submissionId: String): SubmissionStatusResponse = { - val response = masterEndpoint.askWithRetry[DeployMessages.DriverStatusResponse]( + val response = masterEndpoint.askSync[DeployMessages.DriverStatusResponse]( DeployMessages.RequestDriverStatus(submissionId)) val message = response.exception.map { s"Exception from the cluster:\n" + formatException(_) } val d = new SubmissionStatusResponse @@ -174,7 +174,7 @@ private[rest] class StandaloneSubmitRequestServlet( requestMessage match { case submitRequest: CreateSubmissionRequest => val driverDescription = buildDriverDescription(submitRequest) - val response = masterEndpoint.askWithRetry[DeployMessages.SubmitDriverResponse]( + val response = masterEndpoint.askSync[DeployMessages.SubmitDriverResponse]( DeployMessages.RequestSubmitDriver(driverDescription)) val submitResponse = new CreateSubmissionResponse submitResponse.serverSparkVersion = sparkVersion diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala new file mode 100644 index 000000000000..78b0e6b2cbf3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.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.deploy.security + +import scala.reflect.runtime.universe +import scala.util.control.NonFatal + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.Credentials +import org.apache.hadoop.security.token.{Token, TokenIdentifier} + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +private[security] class HBaseDelegationTokenProvider + extends HadoopDelegationTokenProvider with Logging { + + override def serviceName: String = "hbase" + + override def obtainDelegationTokens( + hadoopConf: Configuration, + sparkConf: SparkConf, + creds: Credentials): Option[Long] = { + try { + val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader) + val obtainToken = mirror.classLoader. + loadClass("org.apache.hadoop.hbase.security.token.TokenUtil"). + getMethod("obtainToken", classOf[Configuration]) + + logDebug("Attempting to fetch HBase security token.") + val token = obtainToken.invoke(null, hbaseConf(hadoopConf)) + .asInstanceOf[Token[_ <: TokenIdentifier]] + logInfo(s"Get token from HBase: ${token.toString}") + creds.addToken(token.getService, token) + } catch { + case NonFatal(e) => + logDebug(s"Failed to get token from service $serviceName", e) + } + + None + } + + override def delegationTokensRequired(hadoopConf: Configuration): Boolean = { + hbaseConf(hadoopConf).get("hbase.security.authentication") == "kerberos" + } + + private def hbaseConf(conf: Configuration): Configuration = { + try { + val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader) + val confCreate = mirror.classLoader. + loadClass("org.apache.hadoop.hbase.HBaseConfiguration"). + getMethod("create", classOf[Configuration]) + confCreate.invoke(null, conf).asInstanceOf[Configuration] + } catch { + case NonFatal(e) => + logDebug("Fail to invoke HBaseConfiguration", e) + conf + } + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala new file mode 100644 index 000000000000..c134b7ebe38f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.security + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.security.Credentials + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging + +/** + * Manages all the registered HadoopDelegationTokenProviders and offer APIs for other modules to + * obtain delegation tokens and their renewal time. By default [[HadoopFSDelegationTokenProvider]], + * [[HiveDelegationTokenProvider]] and [[HBaseDelegationTokenProvider]] will be loaded in if not + * explicitly disabled. + * + * Also, each HadoopDelegationTokenProvider is controlled by + * spark.security.credentials.{service}.enabled, and will not be loaded if this config is set to + * false. For example, Hive's delegation token provider [[HiveDelegationTokenProvider]] can be + * enabled/disabled by the configuration spark.security.credentials.hive.enabled. + * + * @param sparkConf Spark configuration + * @param hadoopConf Hadoop configuration + * @param fileSystems Delegation tokens will be fetched for these Hadoop filesystems. + */ +private[spark] class HadoopDelegationTokenManager( + sparkConf: SparkConf, + hadoopConf: Configuration, + fileSystems: Configuration => Set[FileSystem]) + extends Logging { + + private val deprecatedProviderEnabledConfigs = List( + "spark.yarn.security.tokens.%s.enabled", + "spark.yarn.security.credentials.%s.enabled") + private val providerEnabledConfig = "spark.security.credentials.%s.enabled" + + // Maintain all the registered delegation token providers + private val delegationTokenProviders = getDelegationTokenProviders + logDebug(s"Using the following delegation token providers: " + + s"${delegationTokenProviders.keys.mkString(", ")}.") + + /** Construct a [[HadoopDelegationTokenManager]] for the default Hadoop filesystem */ + def this(sparkConf: SparkConf, hadoopConf: Configuration) = { + this( + sparkConf, + hadoopConf, + hadoopConf => Set(FileSystem.get(hadoopConf).getHomeDirectory.getFileSystem(hadoopConf))) + } + + private def getDelegationTokenProviders: Map[String, HadoopDelegationTokenProvider] = { + val providers = List(new HadoopFSDelegationTokenProvider(fileSystems), + new HiveDelegationTokenProvider, + new HBaseDelegationTokenProvider) + + // Filter out providers for which spark.security.credentials.{service}.enabled is false. + providers + .filter { p => isServiceEnabled(p.serviceName) } + .map { p => (p.serviceName, p) } + .toMap + } + + def isServiceEnabled(serviceName: String): Boolean = { + val key = providerEnabledConfig.format(serviceName) + + deprecatedProviderEnabledConfigs.foreach { pattern => + val deprecatedKey = pattern.format(serviceName) + if (sparkConf.contains(deprecatedKey)) { + logWarning(s"${deprecatedKey} is deprecated. Please use ${key} instead.") + } + } + + val isEnabledDeprecated = deprecatedProviderEnabledConfigs.forall { pattern => + sparkConf + .getOption(pattern.format(serviceName)) + .map(_.toBoolean) + .getOrElse(true) + } + + sparkConf + .getOption(key) + .map(_.toBoolean) + .getOrElse(isEnabledDeprecated) + } + + /** + * Get delegation token provider for the specified service. + */ + def getServiceDelegationTokenProvider(service: String): Option[HadoopDelegationTokenProvider] = { + delegationTokenProviders.get(service) + } + + /** + * Writes delegation tokens to creds. Delegation tokens are fetched from all registered + * providers. + * + * @return Time after which the fetched delegation tokens should be renewed. + */ + def obtainDelegationTokens( + hadoopConf: Configuration, + creds: Credentials): Long = { + delegationTokenProviders.values.flatMap { provider => + if (provider.delegationTokensRequired(hadoopConf)) { + provider.obtainDelegationTokens(hadoopConf, sparkConf, creds) + } else { + logDebug(s"Service ${provider.serviceName} does not require a token." + + s" Check your configuration to see if security is disabled or not.") + None + } + }.foldLeft(Long.MaxValue)(math.min) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala new file mode 100644 index 000000000000..1ba245e84af4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.security + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.Credentials + +import org.apache.spark.SparkConf + +/** + * Hadoop delegation token provider. + */ +private[spark] trait HadoopDelegationTokenProvider { + + /** + * Name of the service to provide delegation tokens. This name should be unique. Spark will + * internally use this name to differentiate delegation token providers. + */ + def serviceName: String + + /** + * Returns true if delegation tokens are required for this service. By default, it is based on + * whether Hadoop security is enabled. + */ + def delegationTokensRequired(hadoopConf: Configuration): Boolean + + /** + * Obtain delegation tokens for this service and get the time of the next renewal. + * @param hadoopConf Configuration of current Hadoop Compatible system. + * @param creds Credentials to add tokens and security keys to. + * @return If the returned tokens are renewable and can be renewed, return the time of the next + * renewal, otherwise None should be returned. + */ + def obtainDelegationTokens( + hadoopConf: Configuration, + sparkConf: SparkConf, + creds: Credentials): Option[Long] +} diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala new file mode 100644 index 000000000000..300773c58b18 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.security + +import scala.collection.JavaConverters._ +import scala.util.Try + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.mapred.Master +import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ + +private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: Configuration => Set[FileSystem]) + extends HadoopDelegationTokenProvider with Logging { + + // This tokenRenewalInterval will be set in the first call to obtainDelegationTokens. + // If None, no token renewer is specified or no token can be renewed, + // so we cannot get the token renewal interval. + private var tokenRenewalInterval: Option[Long] = null + + override val serviceName: String = "hadoopfs" + + override def obtainDelegationTokens( + hadoopConf: Configuration, + sparkConf: SparkConf, + creds: Credentials): Option[Long] = { + + val fsToGetTokens = fileSystems(hadoopConf) + val fetchCreds = fetchDelegationTokens(getTokenRenewer(hadoopConf), fsToGetTokens, creds) + + // Get the token renewal interval if it is not set. It will only be called once. + if (tokenRenewalInterval == null) { + tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf, fsToGetTokens) + } + + // Get the time of next renewal. + val nextRenewalDate = tokenRenewalInterval.flatMap { interval => + val nextRenewalDates = fetchCreds.getAllTokens.asScala + .filter(_.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]) + .map { token => + val identifier = token + .decodeIdentifier() + .asInstanceOf[AbstractDelegationTokenIdentifier] + identifier.getIssueDate + interval + } + if (nextRenewalDates.isEmpty) None else Some(nextRenewalDates.min) + } + + nextRenewalDate + } + + def delegationTokensRequired(hadoopConf: Configuration): Boolean = { + UserGroupInformation.isSecurityEnabled + } + + private def getTokenRenewer(hadoopConf: Configuration): String = { + val tokenRenewer = Master.getMasterPrincipal(hadoopConf) + logDebug("Delegation token renewer is: " + tokenRenewer) + + if (tokenRenewer == null || tokenRenewer.length() == 0) { + val errorMessage = "Can't get Master Kerberos principal for use as renewer." + logError(errorMessage) + throw new SparkException(errorMessage) + } + + tokenRenewer + } + + private def fetchDelegationTokens( + renewer: String, + filesystems: Set[FileSystem], + creds: Credentials): Credentials = { + + filesystems.foreach { fs => + logInfo("getting token for: " + fs) + fs.addDelegationTokens(renewer, creds) + } + + creds + } + + private def getTokenRenewalInterval( + hadoopConf: Configuration, + sparkConf: SparkConf, + filesystems: Set[FileSystem]): Option[Long] = { + // We cannot use the tokens generated with renewer yarn. Trying to renew + // those will fail with an access control issue. So create new tokens with the logged in + // user as renewer. + sparkConf.get(PRINCIPAL).flatMap { renewer => + val creds = new Credentials() + fetchDelegationTokens(renewer, filesystems, creds) + + val renewIntervals = creds.getAllTokens.asScala.filter { + _.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier] + }.flatMap { token => + Try { + val newExpiration = token.renew(hadoopConf) + val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] + val interval = newExpiration - identifier.getIssueDate + logInfo(s"Renewal interval is $interval for token ${token.getKind.toString}") + interval + }.toOption + } + if (renewIntervals.isEmpty) None else Some(renewIntervals.min) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala new file mode 100644 index 000000000000..b31cc595ed83 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.security + +import java.lang.reflect.UndeclaredThrowableException +import java.security.PrivilegedExceptionAction + +import scala.util.control.NonFatal + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.hadoop.io.Text +import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.Token + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +private[security] class HiveDelegationTokenProvider + extends HadoopDelegationTokenProvider with Logging { + + override def serviceName: String = "hive" + + private val classNotFoundErrorStr = s"You are attempting to use the " + + s"${getClass.getCanonicalName}, but your Spark distribution is not built with Hive libraries." + + private def hiveConf(hadoopConf: Configuration): Configuration = { + try { + new HiveConf(hadoopConf, classOf[HiveConf]) + } catch { + case NonFatal(e) => + logDebug("Fail to create Hive Configuration", e) + hadoopConf + case e: NoClassDefFoundError => + logWarning(classNotFoundErrorStr) + hadoopConf + } + } + + override def delegationTokensRequired(hadoopConf: Configuration): Boolean = { + UserGroupInformation.isSecurityEnabled && + hiveConf(hadoopConf).getTrimmed("hive.metastore.uris", "").nonEmpty + } + + override def obtainDelegationTokens( + hadoopConf: Configuration, + sparkConf: SparkConf, + creds: Credentials): Option[Long] = { + try { + val conf = hiveConf(hadoopConf) + + val principalKey = "hive.metastore.kerberos.principal" + val principal = conf.getTrimmed(principalKey, "") + require(principal.nonEmpty, s"Hive principal $principalKey undefined") + val metastoreUri = conf.getTrimmed("hive.metastore.uris", "") + require(metastoreUri.nonEmpty, "Hive metastore uri undefined") + + val currentUser = UserGroupInformation.getCurrentUser() + logDebug(s"Getting Hive delegation token for ${currentUser.getUserName()} against " + + s"$principal at $metastoreUri") + + doAsRealUser { + val hive = Hive.get(conf, classOf[HiveConf]) + val tokenStr = hive.getDelegationToken(currentUser.getUserName(), principal) + + val hive2Token = new Token[DelegationTokenIdentifier]() + hive2Token.decodeFromUrlString(tokenStr) + logInfo(s"Get Token from hive metastore: ${hive2Token.toString}") + creds.addToken(new Text("hive.server2.delegation.token"), hive2Token) + } + + None + } catch { + case NonFatal(e) => + logDebug(s"Failed to get token from service $serviceName", e) + None + case e: NoClassDefFoundError => + logWarning(classNotFoundErrorStr) + None + } finally { + Utils.tryLogNonFatalError { + Hive.closeCurrent() + } + } + } + + /** + * Run some code as the real logged in user (which may differ from the current user, for + * example, when using proxying). + */ + private def doAsRealUser[T](fn: => T): T = { + val currentUser = UserGroupInformation.getCurrentUser() + val realUser = Option(currentUser.getRealUser()).getOrElse(currentUser) + + // For some reason the Scala-generated anonymous class ends up causing an + // UndeclaredThrowableException, even if you annotate the method with @throws. + try { + realUser.doAs(new PrivilegedExceptionAction[T]() { + override def run(): T = fn + }) + } catch { + case e: UndeclaredThrowableException => throw Option(e.getCause()).getOrElse(e) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index ce02ee203a4b..12e0dae3f5e5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -22,14 +22,14 @@ import java.io.{File, FileOutputStream, InputStream, IOException} import scala.collection.JavaConverters._ import scala.collection.Map -import org.apache.spark.Logging import org.apache.spark.SecurityManager import org.apache.spark.deploy.Command +import org.apache.spark.internal.Logging import org.apache.spark.launcher.WorkerCommandBuilder import org.apache.spark.util.Utils /** - ** Utilities for running commands with the spark classpath. + * Utilities for running commands with the spark classpath. */ private[deploy] object CommandUtils extends Logging { @@ -44,7 +44,7 @@ object CommandUtils extends Logging { memory: Int, sparkHome: String, substituteArguments: String => String, - classPaths: Seq[String] = Seq[String](), + classPaths: Seq[String] = Seq.empty, env: Map[String, String] = sys.env): ProcessBuilder = { val localCommand = buildLocalCommand( command, securityMgr, substituteArguments, classPaths, env) @@ -73,7 +73,7 @@ object CommandUtils extends Logging { command: Command, securityMgr: SecurityManager, substituteArguments: String => String, - classPath: Seq[String] = Seq[String](), + classPath: Seq[String] = Seq.empty, env: Map[String, String]): Command = { val libraryPathName = Utils.libraryPathEnvName val libraryPathEntries = command.libraryPathEntries @@ -96,7 +96,7 @@ object CommandUtils extends Logging { command.arguments.map(substituteArguments), newEnvironment, command.classPathEntries ++ classPath, - Seq[String](), // library path already captured in environment variable + Seq.empty, // library path already captured in environment variable // filter out auth secret from java options command.javaOpts.filterNot(_.startsWith("-D" + SecurityManager.SPARK_AUTH_SECRET_CONF))) } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 89159ff5e2b3..58a181128eb4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -18,20 +18,21 @@ package org.apache.spark.deploy.worker import java.io._ +import java.net.URI +import java.nio.charset.StandardCharsets import scala.collection.JavaConverters._ -import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files -import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkConf, SecurityManager} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{DriverDescription, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages.DriverStateChanged import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.master.DriverState.DriverState +import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.util.{Utils, Clock, SystemClock} +import org.apache.spark.util.{Clock, ShutdownHookManager, SystemClock, Utils} /** * Manages the execution of one driver, including automatically restarting the driver on failure. @@ -52,9 +53,12 @@ private[deploy] class DriverRunner( @volatile private var killed = false // Populated once finished - private[worker] var finalState: Option[DriverState] = None - private[worker] var finalException: Option[Exception] = None - private var finalExitCode: Option[Int] = None + @volatile private[worker] var finalState: Option[DriverState] = None + @volatile private[worker] var finalException: Option[Exception] = None + + // Timeout to wait for when trying to terminate a driver. + private val DRIVER_TERMINATE_TIMEOUT_MS = + conf.getTimeAsMs("spark.worker.driverTerminateTimeout", "10s") // Decoupled for testing def setClock(_clock: Clock): Unit = { @@ -67,56 +71,63 @@ private[deploy] class DriverRunner( private var clock: Clock = new SystemClock() private var sleeper = new Sleeper { - def sleep(seconds: Int): Unit = (0 until seconds).takeWhile(f => {Thread.sleep(1000); !killed}) + def sleep(seconds: Int): Unit = (0 until seconds).takeWhile { _ => + Thread.sleep(1000) + !killed + } } /** Starts a thread to run and manage the driver. */ private[worker] def start() = { new Thread("DriverRunner for " + driverId) { override def run() { + var shutdownHook: AnyRef = null try { - val driverDir = createWorkingDirectory() - val localJarFilename = downloadUserJar(driverDir) - - def substituteVariables(argument: String): String = argument match { - case "{{WORKER_URL}}" => workerUrl - case "{{USER_JAR}}" => localJarFilename - case other => other + shutdownHook = ShutdownHookManager.addShutdownHook { () => + logInfo(s"Worker shutting down, killing driver $driverId") + kill() } - // TODO: If we add ability to submit multiple jars they should also be added here - val builder = CommandUtils.buildProcessBuilder(driverDesc.command, securityManager, - driverDesc.mem, sparkHome.getAbsolutePath, substituteVariables) - launchDriver(builder, driverDir, driverDesc.supervise) - } - catch { - case e: Exception => finalException = Some(e) - } + // prepare driver jars and run driver + val exitCode = prepareAndRunDriver() - val state = - if (killed) { - DriverState.KILLED - } else if (finalException.isDefined) { - DriverState.ERROR + // set final state depending on if forcibly killed and process exit code + finalState = if (exitCode == 0) { + Some(DriverState.FINISHED) + } else if (killed) { + Some(DriverState.KILLED) } else { - finalExitCode match { - case Some(0) => DriverState.FINISHED - case _ => DriverState.FAILED - } + Some(DriverState.FAILED) } + } catch { + case e: Exception => + kill() + finalState = Some(DriverState.ERROR) + finalException = Some(e) + } finally { + if (shutdownHook != null) { + ShutdownHookManager.removeShutdownHook(shutdownHook) + } + } - finalState = Some(state) - - worker.send(DriverStateChanged(driverId, state, finalException)) + // notify worker of final driver state, possible exception + worker.send(DriverStateChanged(driverId, finalState.get, finalException)) } }.start() } /** Terminate this driver (or prevent it from ever starting if not yet started) */ - private[worker] def kill() { + private[worker] def kill(): Unit = { + logInfo("Killing driver process!") + killed = true synchronized { - process.foreach(p => p.destroy()) - killed = true + process.foreach { p => + val exitCode = Utils.terminateProcess(p, DRIVER_TERMINATE_TIMEOUT_MS) + if (exitCode.isEmpty) { + logWarning("Failed to terminate driver process: " + p + + ". This process will likely be orphaned.") + } + } } } @@ -137,34 +148,44 @@ private[deploy] class DriverRunner( * Will throw an exception if there are errors downloading the jar. */ private def downloadUserJar(driverDir: File): String = { - val jarPath = new Path(driverDesc.jarUrl) - - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - val destPath = new File(driverDir.getAbsolutePath, jarPath.getName) - val jarFileName = jarPath.getName + val jarFileName = new URI(driverDesc.jarUrl).getPath.split("/").last val localJarFile = new File(driverDir, jarFileName) - val localJarFilename = localJarFile.getAbsolutePath - if (!localJarFile.exists()) { // May already exist if running multiple workers on one node - logInfo(s"Copying user jar $jarPath to $destPath") + logInfo(s"Copying user jar ${driverDesc.jarUrl} to $localJarFile") Utils.fetchFile( driverDesc.jarUrl, driverDir, conf, securityManager, - hadoopConf, + SparkHadoopUtil.get.newConfiguration(conf), System.currentTimeMillis(), useCache = false) + if (!localJarFile.exists()) { // Verify copy succeeded + throw new IOException( + s"Can not find expected jar $jarFileName which should have been loaded in $driverDir") + } } + localJarFile.getAbsolutePath + } + + private[worker] def prepareAndRunDriver(): Int = { + val driverDir = createWorkingDirectory() + val localJarFilename = downloadUserJar(driverDir) - if (!localJarFile.exists()) { // Verify copy succeeded - throw new Exception(s"Did not see expected jar $jarFileName in $driverDir") + def substituteVariables(argument: String): String = argument match { + case "{{WORKER_URL}}" => workerUrl + case "{{USER_JAR}}" => localJarFilename + case other => other } - localJarFilename + // TODO: If we add ability to submit multiple jars they should also be added here + val builder = CommandUtils.buildProcessBuilder(driverDesc.command, securityManager, + driverDesc.mem, sparkHome.getAbsolutePath, substituteVariables) + + runDriver(builder, driverDir, driverDesc.supervise) } - private def launchDriver(builder: ProcessBuilder, baseDir: File, supervise: Boolean) { + private def runDriver(builder: ProcessBuilder, baseDir: File, supervise: Boolean): Int = { builder.directory(baseDir) def initialize(process: Process): Unit = { // Redirect stdout and stderr to files @@ -174,50 +195,51 @@ private[deploy] class DriverRunner( val stderr = new File(baseDir, "stderr") val formattedCommand = builder.command.asScala.mkString("\"", "\" \"", "\"") val header = "Launch Command: %s\n%s\n\n".format(formattedCommand, "=" * 40) - Files.append(header, stderr, UTF_8) + Files.append(header, stderr, StandardCharsets.UTF_8) CommandUtils.redirectStream(process.getErrorStream, stderr) } runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise) } - def runCommandWithRetry( - command: ProcessBuilderLike, initialize: Process => Unit, supervise: Boolean): Unit = { + private[worker] def runCommandWithRetry( + command: ProcessBuilderLike, initialize: Process => Unit, supervise: Boolean): Int = { + var exitCode = -1 // Time to wait between submission retries. var waitSeconds = 1 // A run of this many seconds resets the exponential back-off. val successfulRunDuration = 5 - var keepTrying = !killed while (keepTrying) { logInfo("Launch Command: " + command.command.mkString("\"", "\" \"", "\"")) synchronized { - if (killed) { return } + if (killed) { return exitCode } process = Some(command.start()) initialize(process.get) } val processStart = clock.getTimeMillis() - val exitCode = process.get.waitFor() - if (clock.getTimeMillis() - processStart > successfulRunDuration * 1000) { - waitSeconds = 1 - } + exitCode = process.get.waitFor() - if (supervise && exitCode != 0 && !killed) { + // check if attempting another run + keepTrying = supervise && exitCode != 0 && !killed + if (keepTrying) { + if (clock.getTimeMillis() - processStart > successfulRunDuration * 1000) { + waitSeconds = 1 + } logInfo(s"Command exited with status $exitCode, re-launching after $waitSeconds s.") sleeper.sleep(waitSeconds) waitSeconds = waitSeconds * 2 // exponential back-off } - - keepTrying = supervise && exitCode != 0 && !killed - finalExitCode = Some(exitCode) } + + exitCode } } private[deploy] trait Sleeper { - def sleep(seconds: Int) + def sleep(seconds: Int): Unit } // Needed because ProcessBuilder is a final class and cannot be mocked 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 6799f78ec0c1..c1671192e0c6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -19,7 +19,10 @@ package org.apache.spark.deploy.worker import java.io.File +import org.apache.commons.lang3.StringUtils + import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.deploy.{DependencyUtils, SparkHadoopUtil, SparkSubmit} import org.apache.spark.rpc.RpcEnv import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils} @@ -51,6 +54,7 @@ object DriverWrapper { new MutableURLClassLoader(Array(userJarUrl), currentLoader) } Thread.currentThread.setContextClassLoader(loader) + setupDependencies(loader, userJar) // Delegate to supplied main class val clazz = Utils.classForName(mainClass) @@ -66,4 +70,28 @@ object DriverWrapper { System.exit(-1) } } + + private def setupDependencies(loader: MutableURLClassLoader, userJar: String): Unit = { + val sparkConf = new SparkConf() + val secMgr = new SecurityManager(sparkConf) + val hadoopConf = SparkHadoopUtil.newConfiguration(sparkConf) + + val Seq(packagesExclusions, packages, repositories, ivyRepoPath) = + Seq("spark.jars.excludes", "spark.jars.packages", "spark.jars.repositories", "spark.jars.ivy") + .map(sys.props.get(_).orNull) + + val resolvedMavenCoordinates = DependencyUtils.resolveMavenDependencies(packagesExclusions, + packages, repositories, ivyRepoPath) + val jars = { + val jarsProp = sys.props.get("spark.jars").orNull + if (!StringUtils.isBlank(resolvedMavenCoordinates)) { + SparkSubmit.mergeFileLists(jarsProp, resolvedMavenCoordinates) + } else { + jarsProp + } + } + val localJars = DependencyUtils.resolveAndDownloadJars(jars, userJar, sparkConf, hadoopConf, + secMgr) + DependencyUtils.addJarsToClassPath(localJars, loader) + } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 3aef0515cbf6..d4d8521cc820 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -18,16 +18,17 @@ package org.apache.spark.deploy.worker import java.io._ +import java.nio.charset.StandardCharsets import scala.collection.JavaConverters._ -import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files -import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.{SecurityManager, SparkConf, Logging} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import org.apache.spark.internal.Logging +import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.{ShutdownHookManager, Utils} import org.apache.spark.util.logging.FileAppender @@ -60,6 +61,9 @@ private[deploy] class ExecutorRunner( private var stdoutAppender: FileAppender = null private var stderrAppender: FileAppender = null + // Timeout to wait for when trying to terminate an executor. + private val EXECUTOR_TERMINATE_TIMEOUT_MS = 10 * 1000 + // NOTE: This is now redundant with the automated shut-down enforced by the Executor. It might // make sense to remove this in the future. private var shutdownHook: AnyRef = null @@ -71,6 +75,11 @@ private[deploy] class ExecutorRunner( workerThread.start() // Shutdown hook that kills actors on shutdown. shutdownHook = ShutdownHookManager.addShutdownHook { () => + // It's possible that we arrive here before calling `fetchAndRunExecutor`, then `state` will + // be `ExecutorState.RUNNING`. In this case, we should set `state` to `FAILED`. + if (state == ExecutorState.RUNNING) { + state = ExecutorState.FAILED + } killProcess(Some("Worker shutting down")) } } @@ -89,10 +98,17 @@ private[deploy] class ExecutorRunner( if (stderrAppender != null) { stderrAppender.stop() } - process.destroy() - exitCode = Some(process.waitFor()) + exitCode = Utils.terminateProcess(process, EXECUTOR_TERMINATE_TIMEOUT_MS) + if (exitCode.isEmpty) { + logWarning("Failed to terminate process: " + process + + ". This process will likely be orphaned.") + } + } + try { + worker.send(ExecutorStateChanged(appId, execId, state, message, exitCode)) + } catch { + case e: IllegalStateException => logWarning(e.getMessage(), e) } - worker.send(ExecutorStateChanged(appId, execId, state, message, exitCode)) } /** Stop this executor runner, including killing the process it launched */ @@ -140,7 +156,11 @@ private[deploy] class ExecutorRunner( // Add webUI log urls val baseUrl = - s"http://$publicAddress:$webUiPort/logPage/?appId=$appId&executorId=$execId&logType=" + if (conf.getBoolean("spark.ui.reverseProxy", false)) { + s"/proxy/$workerId/logPage/?appId=$appId&executorId=$execId&logType=" + } else { + s"http://$publicAddress:$webUiPort/logPage/?appId=$appId&executorId=$execId&logType=" + } builder.environment.put("SPARK_LOG_URL_STDERR", s"${baseUrl}stderr") builder.environment.put("SPARK_LOG_URL_STDOUT", s"${baseUrl}stdout") @@ -153,7 +173,7 @@ private[deploy] class ExecutorRunner( stdoutAppender = FileAppender(process.getInputStream, stdout, conf) val stderr = new File(executorDir, "stderr") - Files.write(header, stderr, UTF_8) + Files.write(header, stderr, StandardCharsets.UTF_8) stderrAppender = FileAppender(process.getErrorStream, stderr, conf) // Wait for it to exit; executor may exit with code 0 (when driver instructs it to shutdown) @@ -163,16 +183,14 @@ private[deploy] class ExecutorRunner( val message = "Command exited with code " + exitCode worker.send(ExecutorStateChanged(appId, execId, state, Some(message), Some(exitCode))) } catch { - case interrupted: InterruptedException => { + case interrupted: InterruptedException => logInfo("Runner thread for executor " + fullId + " interrupted") state = ExecutorState.KILLED killProcess(None) - } - case e: Exception => { + case e: Exception => logError("Error running executor", e) state = ExecutorState.FAILED killProcess(Some(e.toString)) - } } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index a45867e7680e..ed5fa4b839cd 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -20,24 +20,25 @@ package org.apache.spark.deploy.worker import java.io.File import java.io.IOException import java.text.SimpleDateFormat -import java.util.{UUID, Date} +import java.util.{Date, Locale, UUID} import java.util.concurrent._ import java.util.concurrent.{Future => JFuture, ScheduledFuture => JScheduledFuture} import scala.collection.mutable.{HashMap, HashSet, LinkedHashMap} import scala.concurrent.ExecutionContext -import scala.util.{Failure, Random, Success} +import scala.util.Random import scala.util.control.NonFatal -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{Command, ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.ExternalShuffleService import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI +import org.apache.spark.internal.Logging import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rpc._ -import org.apache.spark.util.{ThreadUtils, SignalLogger, Utils} +import org.apache.spark.util.{SparkUncaughtExceptionHandler, ThreadUtils, Utils} private[deploy] class Worker( override val rpcEnv: RpcEnv, @@ -45,7 +46,6 @@ private[deploy] class Worker( cores: Int, memory: Int, masterRpcAddresses: Array[RpcAddress], - systemName: String, endpointName: String, workDirPath: String = null, val conf: SparkConf, @@ -55,20 +55,20 @@ private[deploy] class Worker( private val host = rpcEnv.address.host private val port = rpcEnv.address.port - Utils.checkHost(host, "Expected hostname") + Utils.checkHost(host) assert (port > 0) // A scheduled executor used to send messages at the specified time. private val forwordMessageScheduler = ThreadUtils.newDaemonSingleThreadScheduledExecutor("worker-forward-message-scheduler") - // A separated thread to clean up the workDir. Used to provide the implicit parameter of `Future` - // methods. + // A separated thread to clean up the workDir and the directories of finished applications. + // Used to provide the implicit parameter of `Future` methods. private val cleanupThreadExecutor = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonSingleThreadExecutor("worker-cleanup-thread")) // For worker and executor IDs - private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) // Send a heartbeat every (heartbeat timeout) / 4 milliseconds private val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 @@ -99,9 +99,24 @@ private[deploy] class Worker( private val testing: Boolean = sys.props.contains("spark.testing") private var master: Option[RpcEndpointRef] = None + + /** + * Whether to use the master address in `masterRpcAddresses` if possible. If it's disabled, Worker + * will just use the address received from Master. + */ + private val preferConfiguredMasterAddress = + conf.getBoolean("spark.worker.preferConfiguredMasterAddress", false) + /** + * The master address to connect in case of failure. When the connection is broken, worker will + * use this address to connect. This is usually just one of `masterRpcAddresses`. However, when + * a master is restarted or takes over leadership, it will be an address sent from master, which + * may not be in `masterRpcAddresses`. + */ + private var masterAddressToConnect: Option[RpcAddress] = None private var activeMasterUrl: String = "" private[worker] var activeMasterWebUiUrl : String = "" - private val workerUri = rpcEnv.uriOf(systemName, rpcEnv.address, endpointName) + private var workerWebUiUrl: String = "" + private val workerUri = RpcEndpointAddress(rpcEnv.address, endpointName).toString private var registered = false private var connected = false private val workerId = generateWorkerId() @@ -140,18 +155,18 @@ private[deploy] class Worker( private val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) private val workerSource = new WorkerSource(this) + val reverseProxy = conf.getBoolean("spark.ui.reverseProxy", false) + private var registerMasterFutures: Array[JFuture[_]] = null private var registrationRetryTimer: Option[JScheduledFuture[_]] = None // A thread pool for registering with masters. Because registering with a master is a blocking // action, this thread pool must be able to create "masterRpcAddresses.size" threads at the same // time so that we can register with all masters. - private val registerMasterThreadPool = new ThreadPoolExecutor( - 0, - masterRpcAddresses.size, // Make sure we can register with all masters at the same time - 60L, TimeUnit.SECONDS, - new SynchronousQueue[Runnable](), - ThreadUtils.namedThreadFactory("worker-register-master-threadpool")) + private val registerMasterThreadPool = ThreadUtils.newDaemonCachedThreadPool( + "worker-register-master-threadpool", + masterRpcAddresses.length // Make sure we can register with all masters at the same time + ) var coresUsed = 0 var memoryUsed = 0 @@ -187,6 +202,8 @@ private[deploy] class Worker( shuffleService.startIfEnabled() webUi = new WorkerWebUI(this, workDir, webUiPort) webUi.bind() + + workerWebUiUrl = s"http://$publicAddress:${webUi.boundPort}" registerWithMaster() metricsSystem.registerSource(workerSource) @@ -195,12 +212,24 @@ private[deploy] class Worker( metricsSystem.getServletHandlers.foreach(webUi.attachHandler) } - private def changeMaster(masterRef: RpcEndpointRef, uiUrl: String) { + /** + * Change to use the new master. + * + * @param masterRef the new master ref + * @param uiUrl the new master Web UI address + * @param masterAddress the new master address which the worker should use to connect in case of + * failure + */ + private def changeMaster(masterRef: RpcEndpointRef, uiUrl: String, masterAddress: RpcAddress) { // activeMasterUrl it's a valid Spark url since we receive it from master. activeMasterUrl = masterRef.address.toSparkURL activeMasterWebUiUrl = uiUrl + masterAddressToConnect = Some(masterAddress) master = Some(masterRef) connected = true + if (reverseProxy) { + logInfo(s"WorkerWebUI is available at $activeMasterWebUiUrl/proxy/$workerId") + } // Cancel any outstanding re-registration attempts because we found a new master cancelLastRegistrationRetry() } @@ -211,9 +240,8 @@ private[deploy] class Worker( override def run(): Unit = { try { logInfo("Connecting to master " + masterAddress + "...") - val masterEndpoint = - rpcEnv.setupEndpointRef(Master.SYSTEM_NAME, masterAddress, Master.ENDPOINT_NAME) - registerWithMaster(masterEndpoint) + val masterEndpoint = rpcEnv.setupEndpointRef(masterAddress, Master.ENDPOINT_NAME) + sendRegisterMessageToMaster(masterEndpoint) } catch { case ie: InterruptedException => // Cancelled case NonFatal(e) => logWarning(s"Failed to connect to master $masterAddress", e) @@ -263,14 +291,14 @@ private[deploy] class Worker( if (registerMasterFutures != null) { registerMasterFutures.foreach(_.cancel(true)) } - val masterAddress = masterRef.address + val masterAddress = + if (preferConfiguredMasterAddress) masterAddressToConnect.get else masterRef.address registerMasterFutures = Array(registerMasterThreadPool.submit(new Runnable { override def run(): Unit = { try { logInfo("Connecting to master " + masterAddress + "...") - val masterEndpoint = - rpcEnv.setupEndpointRef(Master.SYSTEM_NAME, masterAddress, Master.ENDPOINT_NAME) - registerWithMaster(masterEndpoint) + val masterEndpoint = rpcEnv.setupEndpointRef(masterAddress, Master.ENDPOINT_NAME) + sendRegisterMessageToMaster(masterEndpoint) } catch { case ie: InterruptedException => // Cancelled case NonFatal(e) => logWarning(s"Failed to connect to master $masterAddress", e) @@ -339,27 +367,28 @@ private[deploy] class Worker( } } - private def registerWithMaster(masterEndpoint: RpcEndpointRef): Unit = { - masterEndpoint.ask[RegisterWorkerResponse](RegisterWorker( - workerId, host, port, self, cores, memory, webUi.boundPort, publicAddress)) - .onComplete { - // This is a very fast action so we can use "ThreadUtils.sameThread" - case Success(msg) => - Utils.tryLogNonFatalError { - handleRegisterResponse(msg) - } - case Failure(e) => - logError(s"Cannot register with master: ${masterEndpoint.address}", e) - System.exit(1) - }(ThreadUtils.sameThread) + private def sendRegisterMessageToMaster(masterEndpoint: RpcEndpointRef): Unit = { + masterEndpoint.send(RegisterWorker( + workerId, + host, + port, + self, + cores, + memory, + workerWebUiUrl, + masterEndpoint.address)) } private def handleRegisterResponse(msg: RegisterWorkerResponse): Unit = synchronized { msg match { - case RegisteredWorker(masterRef, masterWebUiUrl) => - logInfo("Successfully registered with master " + masterRef.address.toSparkURL) + case RegisteredWorker(masterRef, masterWebUiUrl, masterAddress) => + if (preferConfiguredMasterAddress) { + logInfo("Successfully registered with master " + masterAddress.toSparkURL) + } else { + logInfo("Successfully registered with master " + masterRef.address.toSparkURL) + } registered = true - changeMaster(masterRef, masterWebUiUrl) + changeMaster(masterRef, masterWebUiUrl, masterAddress) forwordMessageScheduler.scheduleAtFixedRate(new Runnable { override def run(): Unit = Utils.tryLogNonFatalError { self.send(SendHeartbeat) @@ -375,6 +404,11 @@ private[deploy] class Worker( }, CLEANUP_INTERVAL_MILLIS, CLEANUP_INTERVAL_MILLIS, TimeUnit.MILLISECONDS) } + val execs = executors.values.map { e => + new ExecutorDescription(e.appId, e.execId, e.cores, e.state) + } + masterRef.send(WorkerLatestState(workerId, execs.toList, drivers.keys.toSeq)) + case RegisterWorkerFailed(message) => if (!registered) { logError("Worker registration failed: " + message) @@ -387,6 +421,9 @@ private[deploy] class Worker( } override def receive: PartialFunction[Any, Unit] = synchronized { + case msg: RegisterWorkerResponse => + handleRegisterResponse(msg) + case SendHeartbeat => if (connected) { sendToMaster(Heartbeat(workerId, self)) } @@ -395,7 +432,7 @@ private[deploy] class Worker( // rpcEndpoint. // Copy ids so that it can be used in the cleanup thread. val appIds = executors.values.map(_.appId).toSet - val cleanupFuture = concurrent.future { + val cleanupFuture = concurrent.Future { val appDirs = workDir.listFiles() if (appDirs == null) { throw new IOException("ERROR: Failed to list files in " + appDirs) @@ -413,14 +450,13 @@ private[deploy] class Worker( } }(cleanupThreadExecutor) - cleanupFuture.onFailure { - case e: Throwable => - logError("App dir cleanup failed: " + e.getMessage, e) - }(cleanupThreadExecutor) + cleanupFuture.failed.foreach(e => + logError("App dir cleanup failed: " + e.getMessage, e) + )(cleanupThreadExecutor) case MasterChanged(masterRef, masterWebUiUrl) => logInfo("Master has changed, new master is at " + masterRef.address.toSparkURL) - changeMaster(masterRef, masterWebUiUrl) + changeMaster(masterRef, masterWebUiUrl, masterRef.address) val execs = executors.values. map(e => new ExecutorDescription(e.appId, e.execId, e.cores, e.state)) @@ -446,13 +482,25 @@ private[deploy] class Worker( // Create local dirs for the executor. These are passed to the executor via the // SPARK_EXECUTOR_DIRS environment variable, and deleted by the Worker when the // application finishes. - val appLocalDirs = appDirectories.get(appId).getOrElse { - Utils.getOrCreateLocalRootDirs(conf).map { dir => - val appDir = Utils.createDirectory(dir, namePrefix = "executor") - Utils.chmod700(appDir) - appDir.getAbsolutePath() + val appLocalDirs = appDirectories.getOrElse(appId, { + val localRootDirs = Utils.getOrCreateLocalRootDirs(conf) + val dirs = localRootDirs.flatMap { dir => + try { + val appDir = Utils.createDirectory(dir, namePrefix = "executor") + Utils.chmod700(appDir) + Some(appDir.getAbsolutePath()) + } catch { + case e: IOException => + logWarning(s"${e.getMessage}. Ignoring this directory.") + None + } }.toSeq - } + if (dirs.isEmpty) { + throw new IOException("No subfolder can be created in " + + s"${localRootDirs.mkString(",")}.") + } + dirs + }) appDirectories(appId) = appLocalDirs val manager = new ExecutorRunner( appId, @@ -469,14 +517,14 @@ private[deploy] class Worker( executorDir, workerUri, conf, - appLocalDirs, ExecutorState.LOADING) + appLocalDirs, ExecutorState.RUNNING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ memoryUsed += memory_ sendToMaster(ExecutorStateChanged(appId, execId, manager.state, None, None)) } catch { - case e: Exception => { + case e: Exception => logError(s"Failed to launch executor $appId/$execId for ${appDesc.name}.", e) if (executors.contains(appId + "/" + execId)) { executors(appId + "/" + execId).kill() @@ -484,7 +532,6 @@ private[deploy] class Worker( } sendToMaster(ExecutorStateChanged(appId, execId, ExecutorState.FAILED, Some(e.toString), None)) - } } } @@ -493,7 +540,7 @@ private[deploy] class Worker( case KillExecutor(masterUrl, appId, execId) => if (masterUrl != activeMasterUrl) { - logWarning("Invalid Master (" + masterUrl + ") attempted to launch executor " + execId) + logWarning("Invalid Master (" + masterUrl + ") attempted to kill executor " + execId) } else { val fullId = appId + "/" + execId executors.get(fullId) match { @@ -505,7 +552,7 @@ private[deploy] class Worker( } } - case LaunchDriver(driverId, driverDesc) => { + case LaunchDriver(driverId, driverDesc) => logInfo(s"Asked to launch driver $driverId") val driver = new DriverRunner( conf, @@ -521,9 +568,8 @@ private[deploy] class Worker( coresUsed += driverDesc.cores memoryUsed += driverDesc.mem - } - case KillDriver(driverId) => { + case KillDriver(driverId) => logInfo(s"Asked to kill driver $driverId") drivers.get(driverId) match { case Some(runner) => @@ -531,11 +577,9 @@ private[deploy] class Worker( case None => logError(s"Asked to kill unknown driver $driverId") } - } - case driverStateChanged @ DriverStateChanged(driverId, state, exception) => { + case driverStateChanged @ DriverStateChanged(driverId, state, exception) => handleDriverStateChanged(driverStateChanged) - } case ReregisterWithMaster => reregisterWithMaster() @@ -554,7 +598,8 @@ private[deploy] class Worker( } override def onDisconnected(remoteAddress: RpcAddress): Unit = { - if (master.exists(_.address == remoteAddress)) { + if (master.exists(_.address == remoteAddress) || + masterAddressToConnect.exists(_ == remoteAddress)) { logInfo(s"$remoteAddress Disassociated !") masterDisconnected() } @@ -571,10 +616,14 @@ private[deploy] class Worker( if (shouldCleanup) { finishedApps -= id appDirectories.remove(id).foreach { dirList => - logInfo(s"Cleaning up local directories for application $id") - dirList.foreach { dir => - Utils.deleteRecursively(new File(dir)) - } + concurrent.Future { + logInfo(s"Cleaning up local directories for application $id") + dirList.foreach { dir => + Utils.deleteRecursively(new File(dir)) + } + }(cleanupThreadExecutor).failed.foreach(e => + logError(s"Clean up app dir $dirList failed: ${e.getMessage}", e) + )(cleanupThreadExecutor) } shuffleService.applicationRemoved(id) } @@ -688,11 +737,24 @@ private[deploy] object Worker extends Logging { val ENDPOINT_NAME = "Worker" def main(argStrings: Array[String]) { - SignalLogger.register(log) + Thread.setDefaultUncaughtExceptionHandler(new SparkUncaughtExceptionHandler( + exitOnUncaughtException = false)) + Utils.initDaemon(log) val conf = new SparkConf val args = new WorkerArguments(argStrings, conf) val rpcEnv = startRpcEnvAndEndpoint(args.host, args.port, args.webUiPort, args.cores, - args.memory, args.masters, args.workDir) + args.memory, args.masters, args.workDir, conf = conf) + // With external shuffle service enabled, if we request to launch multiple workers on one host, + // we can only successfully launch the first worker and the rest fails, because with the port + // bound, we may launch no more than one external shuffle service on each host. + // When this happens, we should give explicit reason of failure instead of fail silently. For + // more detail see SPARK-20989. + val externalShuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false) + val sparkWorkerInstances = scala.sys.env.getOrElse("SPARK_WORKER_INSTANCES", "1").toInt + require(externalShuffleServiceEnabled == false || sparkWorkerInstances <= 1, + "Starting multiple workers on one host is failed because we may launch no more than one " + + "external shuffle service on each host, please set spark.shuffle.service.enabled to " + + "false or set SPARK_WORKER_INSTANCES to 1 to resolve the conflict.") rpcEnv.awaitTermination() } @@ -713,7 +775,7 @@ private[deploy] object Worker extends Logging { val rpcEnv = RpcEnv.create(systemName, host, port, conf, securityMgr) val masterAddresses = masterUrls.map(RpcAddress.fromSparkURL(_)) rpcEnv.setupEndpoint(ENDPOINT_NAME, new Worker(rpcEnv, webUiPort, cores, memory, - masterAddresses, systemName, ENDPOINT_NAME, workDir, conf, securityMgr)) + masterAddresses, ENDPOINT_NAME, workDir, conf, securityMgr)) rpcEnv } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 5181142c5f80..580281288b06 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -19,8 +19,10 @@ package org.apache.spark.deploy.worker import java.lang.management.ManagementFactory -import org.apache.spark.util.{IntParam, MemoryParam, Utils} +import scala.annotation.tailrec + import org.apache.spark.SparkConf +import org.apache.spark.util.{IntParam, MemoryParam, Utils} /** * Command-line parser for the worker. @@ -63,14 +65,15 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { checkWorkerMemory() + @tailrec private def parse(args: List[String]): Unit = args match { case ("--ip" | "-i") :: value :: tail => - Utils.checkHost(value, "ip no longer supported, please use hostname " + value) + Utils.checkHost(value) host = value parse(tail) case ("--host" | "-h") :: value :: tail => - Utils.checkHost(value, "Please use hostname " + value) + Utils.checkHost(value) host = value parse(tail) @@ -162,12 +165,11 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { } // scalastyle:on classforname } catch { - case e: Exception => { + case e: Exception => totalMb = 2*1024 // scalastyle:off println System.out.println("Failed to get total physical memory. Using " + totalMb + " MB") // scalastyle:on println - } } // Leave out 1 GB for the operating system, but don't return a negative memory size math.max(totalMb - 1024, Utils.DEFAULT_DRIVER_MEM_MB) @@ -175,7 +177,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { def checkWorkerMemory(): Unit = { if (memory <= 0) { - val message = "Memory can't be 0, missing a M or G on the end of the memory specification?" + val message = "Memory is below 1MB, or missing a M/G at the end of the memory specification?" throw new IllegalStateException(message) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index ab56fde938ba..23efcab6caad 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -17,11 +17,12 @@ package org.apache.spark.deploy.worker -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.rpc._ /** - * Actor which connects to a worker process and terminates the JVM if the connection is severed. + * Endpoint which connects to a worker process and terminates the JVM if the + * connection is severed. * Provides fate sharing between a worker and its associated child processes. */ private[spark] class WorkerWatcher( diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index 5a1d06eb87db..2f5a5642d3ca 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -18,36 +18,37 @@ package org.apache.spark.deploy.worker.ui import java.io.File -import java.net.URI import javax.servlet.http.HttpServletRequest -import scala.xml.Node +import scala.xml.{Node, Unparsed} -import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.internal.Logging +import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils -import org.apache.spark.Logging import org.apache.spark.util.logging.RollingFileAppender private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with Logging { private val worker = parent.worker - private val workDir = parent.workDir + private val workDir = new File(parent.workDir.toURI.normalize().getPath) private val supportedLogTypes = Set("stderr", "stdout") + private val defaultBytes = 100 * 1024 + // stripXSS is called first to remove suspicious characters used in XSS attacks def renderLog(request: HttpServletRequest): String = { - val defaultBytes = 100 * 1024 - - val appId = Option(request.getParameter("appId")) - val executorId = Option(request.getParameter("executorId")) - val driverId = Option(request.getParameter("driverId")) - val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) + val appId = Option(UIUtils.stripXSS(request.getParameter("appId"))) + val executorId = Option(UIUtils.stripXSS(request.getParameter("executorId"))) + val driverId = Option(UIUtils.stripXSS(request.getParameter("driverId"))) + val logType = UIUtils.stripXSS(request.getParameter("logType")) + val offset = Option(UIUtils.stripXSS(request.getParameter("offset"))).map(_.toLong) + val byteLength = + Option(UIUtils.stripXSS(request.getParameter("byteLength"))).map(_.toInt) + .getOrElse(defaultBytes) val logDir = (appId, executorId, driverId) match { case (Some(a), Some(e), None) => - s"${workDir.getPath}/$appId/$executorId/" + s"${workDir.getPath}/$a/$e/" case (None, None, Some(d)) => - s"${workDir.getPath}/$driverId/" + s"${workDir.getPath}/$d/" case _ => throw new Exception("Request must specify either application or driver identifiers") } @@ -57,14 +58,16 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with pre + logText } + // stripXSS is called first to remove suspicious characters used in XSS attacks def render(request: HttpServletRequest): Seq[Node] = { - val defaultBytes = 100 * 1024 - val appId = Option(request.getParameter("appId")) - val executorId = Option(request.getParameter("executorId")) - val driverId = Option(request.getParameter("driverId")) - val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) + val appId = Option(UIUtils.stripXSS(request.getParameter("appId"))) + val executorId = Option(UIUtils.stripXSS(request.getParameter("executorId"))) + val driverId = Option(UIUtils.stripXSS(request.getParameter("driverId"))) + val logType = UIUtils.stripXSS(request.getParameter("logType")) + val offset = Option(UIUtils.stripXSS(request.getParameter("offset"))).map(_.toLong) + val byteLength = + Option(UIUtils.stripXSS(request.getParameter("byteLength"))).map(_.toInt) + .getOrElse(defaultBytes) val (logDir, params, pageName) = (appId, executorId, driverId) match { case (Some(a), Some(e), None) => @@ -77,51 +80,44 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with val (logText, startByte, endByte, logLength) = getLog(logDir, logType, offset, byteLength) val linkToMaster =

    Back to Master

    - val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} - - val backButton = - if (startByte > 0) { - - - - } else { - - } - - val nextButton = - if (endByte < logLength) { - - - - } else { - - } + val curLogLength = endByte - startByte + val range = + + Showing {curLogLength} Bytes: {startByte.toString} - {endByte.toString} of {logLength} + + + val moreButton = + + + val newButton = + + + val alert = + + + val logParams = "?%s&logType=%s".format(params, logType) + val jsOnload = "window.onload = " + + s"initLogPage('$logParams', $curLogLength, $startByte, $endByte, $logLength, $byteLength);" val content = - - - {linkToMaster} -
    -
    {backButton}
    -
    {range}
    -
    {nextButton}
    -
    -
    -
    -
    {logText}
    -
    - - +
    + {linkToMaster} + {range} +
    +
    {moreButton}
    +
    {logText}
    + {alert} +
    {newButton}
    +
    + +
    + UIUtils.basicSparkPage(content, logType + " log page for " + pageName) } @@ -138,7 +134,7 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with } // Verify that the normalized path of the log directory is in the working directory - val normalizedUri = new URI(logDirectory).normalize() + val normalizedUri = new File(logDirectory).toURI.normalize() val normalizedLogDir = new File(normalizedUri.getPath) if (!Utils.isInDirectory(workDir, normalizedLogDir)) { return ("Error: invalid log directory " + logDirectory, 0, 0, 0) @@ -148,7 +144,8 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with val files = RollingFileAppender.getSortedRolledOverFiles(logDirectory, logType) logDebug(s"Sorted log files of type $logType in $logDirectory:\n${files.mkString("\n")}") - val totalLength = files.map { _.length }.sum + val fileLengths: Seq[Long] = files.map(Utils.getFileLength(_, worker.conf)) + val totalLength = fileLengths.sum val offset = offsetOption.getOrElse(totalLength - byteLength) val startIndex = { if (offset < 0) { @@ -161,7 +158,7 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with } val endIndex = math.min(startIndex + byteLength, totalLength) logDebug(s"Getting log from $startIndex to $endIndex") - val logText = Utils.offsetBytes(files, startIndex, endIndex) + val logText = Utils.offsetBytes(files, fileLengths, startIndex, endIndex) logDebug(s"Got log of length ${logText.length} bytes") (logText, startIndex, endIndex, totalLength) } catch { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala index fd905feb97e9..ce84bc4dae32 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala @@ -17,28 +17,29 @@ package org.apache.spark.deploy.worker.ui +import javax.servlet.http.HttpServletRequest + import scala.xml.Node -import javax.servlet.http.HttpServletRequest import org.json4s.JValue -import org.apache.spark.deploy.JsonProtocol +import org.apache.spark.deploy.{ExecutorState, JsonProtocol} import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils private[ui] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { private val workerEndpoint = parent.worker.self override def renderJson(request: HttpServletRequest): JValue = { - val workerState = workerEndpoint.askWithRetry[WorkerStateResponse](RequestWorkerState) + val workerState = workerEndpoint.askSync[WorkerStateResponse](RequestWorkerState) JsonProtocol.writeWorkerState(workerState) } def render(request: HttpServletRequest): Seq[Node] = { - val workerState = workerEndpoint.askWithRetry[WorkerStateResponse](RequestWorkerState) + val workerState = workerEndpoint.askSync[WorkerStateResponse](RequestWorkerState) val executorHeaders = Seq("ExecutorID", "Cores", "State", "Memory", "Job Details", "Logs") val runningExecutors = workerState.executors @@ -50,9 +51,11 @@ private[ui] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { val driverHeaders = Seq("DriverID", "Main Class", "State", "Cores", "Memory", "Logs", "Notes") val runningDrivers = workerState.drivers.sortBy(_.driverId).reverse - val runningDriverTable = UIUtils.listingTable(driverHeaders, driverRow, runningDrivers) + val runningDriverTable = UIUtils.listingTable[DriverRunner](driverHeaders, + driverRow(workerState.workerId, _), runningDrivers) val finishedDrivers = workerState.finishedDrivers.sortBy(_.driverId).reverse - val finishedDriverTable = UIUtils.listingTable(driverHeaders, driverRow, finishedDrivers) + val finishedDriverTable = UIUtils.listingTable[DriverRunner](driverHeaders, + driverRow(workerState.workerId, _), finishedDrivers) // For now we only show driver information if the user has submitted drivers to the cluster. // This is until we integrate the notion of drivers and applications in the UI. @@ -101,6 +104,11 @@ private[ui] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { } def executorRow(executor: ExecutorRunner): Seq[Node] = { + val workerUrlRef = UIUtils.makeHref(parent.worker.reverseProxy, executor.workerId, + parent.webUrl) + val appUrlRef = UIUtils.makeHref(parent.worker.reverseProxy, executor.appId, + executor.appDesc.appUiUrl) + {executor.execId} {executor.cores} @@ -111,21 +119,30 @@ private[ui] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") {
    • ID: {executor.appId}
    • -
    • Name: {executor.appDesc.name}
    • +
    • Name: + { + if ({executor.state == ExecutorState.RUNNING} && executor.appDesc.appUiUrl.nonEmpty) { + {executor.appDesc.name} + } else { + {executor.appDesc.name} + } + } +
    • User: {executor.appDesc.user}
    - stdout - stderr + stdout + stderr } - def driverRow(driver: DriverRunner): Seq[Node] = { + def driverRow(workerId: String, driver: DriverRunner): Seq[Node] = { + val workerUrlRef = UIUtils.makeHref(parent.worker.reverseProxy, workerId, parent.webUrl) {driver.driverId} {driver.driverDesc.command.arguments(2)} @@ -137,8 +154,8 @@ private[ui] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { {Utils.megabytesToString(driver.driverDesc.mem)} - stdout - stderr + stdout + stderr {driver.finalException.getOrElse("")} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 1a0598e50dcf..db696b04384b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -20,8 +20,8 @@ package org.apache.spark.deploy.worker.ui import java.io.File import javax.servlet.http.HttpServletRequest -import org.apache.spark.Logging import org.apache.spark.deploy.worker.Worker +import org.apache.spark.internal.Logging import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.RpcUtils @@ -34,7 +34,8 @@ class WorkerWebUI( val worker: Worker, val workDir: File, requestedPort: Int) - extends WebUI(worker.securityMgr, requestedPort, worker.conf, name = "WorkerUI") + extends WebUI(worker.securityMgr, worker.securityMgr.getSSLOptions("standalone"), + requestedPort, worker.conf, name = "WorkerUI") with Logging { private[ui] val timeout = RpcUtils.askRpcTimeout(worker.conf) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index c2ebf3059621..d27362ae85be 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -19,32 +19,35 @@ package org.apache.spark.executor import java.net.URL import java.nio.ByteBuffer - -import org.apache.hadoop.conf.Configuration +import java.util.Locale +import java.util.concurrent.atomic.AtomicBoolean import scala.collection.mutable import scala.util.{Failure, Success} +import scala.util.control.NonFatal -import org.apache.spark.rpc._ import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher -import org.apache.spark.scheduler.TaskDescription +import org.apache.spark.internal.Logging +import org.apache.spark.rpc._ +import org.apache.spark.scheduler.{ExecutorLossReason, TaskDescription} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.{ThreadUtils, SignalLogger, Utils} +import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class CoarseGrainedExecutorBackend( override val rpcEnv: RpcEnv, driverUrl: String, executorId: String, - hostPort: String, + hostname: String, cores: Int, userClassPath: Seq[URL], env: SparkEnv) extends ThreadSafeRpcEndpoint with ExecutorBackend with Logging { + private[this] val stopping = new AtomicBoolean(false) var executor: Executor = null @volatile var driver: Option[RpcEndpointRef] = None @@ -57,70 +60,77 @@ private[spark] class CoarseGrainedExecutorBackend( rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref => // This is a very fast action so we can use "ThreadUtils.sameThread" driver = Some(ref) - ref.ask[RegisterExecutorResponse]( - RegisterExecutor(executorId, self, hostPort, cores, extractLogUrls)) + ref.ask[Boolean](RegisterExecutor(executorId, self, hostname, cores, extractLogUrls)) }(ThreadUtils.sameThread).onComplete { // This is a very fast action so we can use "ThreadUtils.sameThread" - case Success(msg) => Utils.tryLogNonFatalError { - Option(self).foreach(_.send(msg)) // msg must be RegisterExecutorResponse - } - case Failure(e) => { - logError(s"Cannot register with driver: $driverUrl", e) - System.exit(1) - } + case Success(msg) => + // Always receive `true`. Just ignore it + case Failure(e) => + exitExecutor(1, s"Cannot register with driver: $driverUrl", e, notifyDriver = false) }(ThreadUtils.sameThread) } def extractLogUrls: Map[String, String] = { val prefix = "SPARK_LOG_URL_" sys.env.filterKeys(_.startsWith(prefix)) - .map(e => (e._1.substring(prefix.length).toLowerCase, e._2)) + .map(e => (e._1.substring(prefix.length).toLowerCase(Locale.ROOT), e._2)) } override def receive: PartialFunction[Any, Unit] = { - case RegisteredExecutor(hostname) => + case RegisteredExecutor => logInfo("Successfully registered with driver") - executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) + try { + executor = new Executor(executorId, hostname, env, userClassPath, isLocal = false) + } catch { + case NonFatal(e) => + exitExecutor(1, "Unable to create executor due to " + e.getMessage, e) + } case RegisterExecutorFailed(message) => - logError("Slave registration failed: " + message) - System.exit(1) + exitExecutor(1, "Slave registration failed: " + message) case LaunchTask(data) => if (executor == null) { - logError("Received LaunchTask command but executor was null") - System.exit(1) + exitExecutor(1, "Received LaunchTask command but executor was null") } else { - val taskDesc = ser.deserialize[TaskDescription](data.value) + val taskDesc = TaskDescription.decode(data.value) logInfo("Got assigned task " + taskDesc.taskId) - executor.launchTask(this, taskId = taskDesc.taskId, attemptNumber = taskDesc.attemptNumber, - taskDesc.name, taskDesc.serializedTask) + executor.launchTask(this, taskDesc) } - case KillTask(taskId, _, interruptThread) => + case KillTask(taskId, _, interruptThread, reason) => if (executor == null) { - logError("Received KillTask command but executor was null") - System.exit(1) + exitExecutor(1, "Received KillTask command but executor was null") } else { - executor.killTask(taskId, interruptThread) + executor.killTask(taskId, interruptThread, reason) } case StopExecutor => + stopping.set(true) logInfo("Driver commanded a shutdown") // Cannot shutdown here because an ack may need to be sent back to the caller. So send // a message to self to actually do the shutdown. self.send(Shutdown) case Shutdown => - executor.stop() - stop() - rpcEnv.shutdown() + stopping.set(true) + new Thread("CoarseGrainedExecutorBackend-stop-executor") { + override def run(): Unit = { + // executor.stop() will call `SparkEnv.stop()` which waits until RpcEnv stops totally. + // However, if `executor.stop()` runs in some thread of RpcEnv, RpcEnv won't be able to + // stop until `executor.stop()` returns, which becomes a dead-lock (See SPARK-14180). + // Therefore, we put this line in a new thread. + executor.stop() + } + }.start() } override def onDisconnected(remoteAddress: RpcAddress): Unit = { - if (driver.exists(_.address == remoteAddress)) { - logError(s"Driver $remoteAddress disassociated! Shutting down.") - System.exit(1) + if (stopping.get()) { + logInfo(s"Driver from $remoteAddress disconnected during shutdown") + } else if (driver.exists(_.address == remoteAddress)) { + exitExecutor(1, s"Driver $remoteAddress disassociated! Shutting down.", null, + notifyDriver = false) } else { logWarning(s"An unknown ($remoteAddress) driver disconnected.") } @@ -133,6 +143,33 @@ private[spark] class CoarseGrainedExecutorBackend( case None => logWarning(s"Drop $msg because has not yet connected to driver") } } + + /** + * This function can be overloaded by other child classes to handle + * executor exits differently. For e.g. when an executor goes down, + * back-end may not want to take the parent process down. + */ + protected def exitExecutor(code: Int, + reason: String, + throwable: Throwable = null, + notifyDriver: Boolean = true) = { + val message = "Executor self-exiting due to : " + reason + if (throwable != null) { + logError(message, throwable) + } else { + logError(message) + } + + if (notifyDriver && driver.nonEmpty) { + driver.get.ask[Boolean]( + RemoveExecutor(executorId, new ExecutorLossReason(reason)) + ).failed.foreach(e => + logWarning(s"Unable to notify the driver due to " + e.getMessage, e) + )(ThreadUtils.sameThread) + } + + System.exit(code) + } } private[spark] object CoarseGrainedExecutorBackend extends Logging { @@ -146,7 +183,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { workerUrl: Option[String], userClassPath: Seq[URL]) { - SignalLogger.register(log) + Utils.initDaemon(log) SparkHadoopUtil.get.runAsSparkUser { () => // Debug code @@ -154,17 +191,16 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { // Bootstrap to fetch the driver's Spark properties. val executorConf = new SparkConf - val port = executorConf.getInt("spark.executor.port", 0) val fetcher = RpcEnv.create( "driverPropsFetcher", hostname, - port, + -1, executorConf, new SecurityManager(executorConf), clientMode = true) val driver = fetcher.setupEndpointRefByURI(driverUrl) - val props = driver.askWithRetry[Seq[(String, String)]](RetrieveSparkProps) ++ - Seq[(String, String)](("spark.app.id", appId)) + val cfg = driver.askSync[SparkAppConfig](RetrieveSparkAppConfig) + val props = cfg.sparkProperties ++ Seq[(String, String)](("spark.app.id", appId)) fetcher.shutdown() // Create SparkEnv using properties we fetched from the driver. @@ -180,25 +216,24 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { if (driverConf.contains("spark.yarn.credentials.file")) { logInfo("Will periodically update credentials from: " + driverConf.get("spark.yarn.credentials.file")) - SparkHadoopUtil.get.startExecutorDelegationTokenRenewer(driverConf) + SparkHadoopUtil.get.startCredentialUpdater(driverConf) + } + + cfg.hadoopDelegationCreds.foreach { hadoopCreds => + val creds = SparkHadoopUtil.get.deserialize(hadoopCreds) + SparkHadoopUtil.get.addCurrentUserCredentials(creds) } val env = SparkEnv.createExecutorEnv( - driverConf, executorId, hostname, port, cores, isLocal = false) - - // SparkEnv will set spark.executor.port if the rpc env is listening for incoming - // connections (e.g., if it's using akka). Otherwise, the executor is running in - // client mode only, and does not accept incoming connections. - val sparkHostPort = env.conf.getOption("spark.executor.port").map { port => - hostname + ":" + port - }.orNull + driverConf, executorId, hostname, cores, cfg.ioEncryptionKey, isLocal = false) + env.rpcEnv.setupEndpoint("Executor", new CoarseGrainedExecutorBackend( - env.rpcEnv, driverUrl, executorId, sparkHostPort, cores, userClassPath, env)) + env.rpcEnv, driverUrl, executorId, hostname, cores, userClassPath, env)) workerUrl.foreach { url => env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url)) } env.rpcEnv.awaitTermination() - SparkHadoopUtil.get.stopExecutorDelegationTokenRenewer() + SparkHadoopUtil.get.stopCredentialUpdater() } } @@ -251,13 +286,14 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } run(driverUrl, executorId, hostname, cores, appId, workerUrl, userClassPath) + System.exit(0) } private def printUsageAndExit() = { // scalastyle:off println System.err.println( """ - |"Usage: CoarseGrainedExecutorBackend [options] + |Usage: CoarseGrainedExecutorBackend [options] | | Options are: | --driver-url diff --git a/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala b/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala index 7d84889a2def..3e0d52cb4ccb 100644 --- a/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala +++ b/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import org.apache.spark.{TaskCommitDenied, TaskEndReason} +import org.apache.spark.TaskCommitDenied /** * Exception thrown when a task attempts to commit output to HDFS but is denied by the driver. @@ -29,5 +29,5 @@ private[spark] class CommitDeniedException( attemptNumber: Int) extends Exception(msg) { - def toTaskEndReason: TaskEndReason = TaskCommitDenied(jobID, splitID, attemptNumber) + def toTaskCommitDeniedReason: TaskCommitDenied = TaskCommitDenied(jobID, splitID, attemptNumber) } 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 9e88d488c037..2ecbb749d1fb 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -18,28 +18,36 @@ package org.apache.spark.executor import java.io.{File, NotSerializableException} +import java.lang.Thread.UncaughtExceptionHandler import java.lang.management.ManagementFactory -import java.net.URL +import java.net.{URI, URL} import java.nio.ByteBuffer -import java.util.concurrent.{ConcurrentHashMap, TimeUnit} +import java.util.Properties +import java.util.concurrent._ +import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.{ArrayBuffer, HashMap, Map} import scala.util.control.NonFatal +import com.google.common.util.concurrent.ThreadFactoryBuilder + import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.memory.TaskMemoryManager -import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task} +import org.apache.spark.rpc.RpcTimeout +import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task, TaskDescription} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} import org.apache.spark.util._ +import org.apache.spark.util.io.ChunkedByteBuffer /** * Spark executor, backed by a threadpool to run tasks. * * This can be used with Mesos, YARN, and the standalone scheduler. - * An internal RPC interface (at the moment Akka) is used for communication with the driver, + * An internal RPC interface is used for communication with the driver, * except in the case of Mesos fine-grained mode. */ private[spark] class Executor( @@ -47,7 +55,8 @@ private[spark] class Executor( executorHostname: String, env: SparkEnv, userClassPath: Seq[URL] = Nil, - isLocal: Boolean = false) + isLocal: Boolean = false, + uncaughtExceptionHandler: UncaughtExceptionHandler = new SparkUncaughtExceptionHandler) extends Logging { logInfo(s"Starting executor ID $executorId on host $executorHostname") @@ -62,7 +71,7 @@ private[spark] class Executor( private val conf = env.conf // No ip or host:port - just hostname - Utils.checkHost(executorHostname, "Expected executed slave to be a hostname") + Utils.checkHost(executorHostname) // must not have port specified. assert (0 == Utils.parseHostPort(executorHostname)._2) @@ -73,25 +82,48 @@ private[spark] class Executor( // Setup an uncaught exception handler for non-local mode. // Make any thread terminations due to uncaught exceptions kill the entire // executor process to avoid surprising stalls. - Thread.setDefaultUncaughtExceptionHandler(SparkUncaughtExceptionHandler) + Thread.setDefaultUncaughtExceptionHandler(uncaughtExceptionHandler) } // Start worker thread pool - private val threadPool = ThreadUtils.newDaemonCachedThreadPool("Executor task launch worker") + private val threadPool = { + val threadFactory = new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("Executor task launch worker-%d") + .setThreadFactory(new ThreadFactory { + override def newThread(r: Runnable): Thread = + // Use UninterruptibleThread to run tasks so that we can allow running codes without being + // interrupted by `Thread.interrupt()`. Some issues, such as KAFKA-1894, HADOOP-10622, + // will hang forever if some methods are interrupted. + new UninterruptibleThread(r, "unused") // thread name will be set by ThreadFactoryBuilder + }) + .build() + Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor] + } private val executorSource = new ExecutorSource(threadPool, executorId) + // Pool used for threads that supervise task killing / cancellation + private val taskReaperPool = ThreadUtils.newDaemonCachedThreadPool("Task reaper") + // For tasks which are in the process of being killed, this map holds the most recently created + // TaskReaper. All accesses to this map should be synchronized on the map itself (this isn't + // a ConcurrentHashMap because we use the synchronization for purposes other than simply guarding + // the integrity of the map's internal state). The purpose of this map is to prevent the creation + // of a separate TaskReaper for every killTask() of a given task. Instead, this map allows us to + // track whether an existing TaskReaper fulfills the role of a TaskReaper that we would otherwise + // create. The map key is a task id. + private val taskReaperForTask: HashMap[Long, TaskReaper] = HashMap[Long, TaskReaper]() if (!isLocal) { - env.metricsSystem.registerSource(executorSource) env.blockManager.initialize(conf.getAppId) + env.metricsSystem.registerSource(executorSource) + env.metricsSystem.registerSource(env.blockManager.shuffleMetricsSource) } - // Create an RpcEndpoint for receiving RPCs from the driver - private val executorEndpoint = env.rpcEnv.setupEndpoint( - ExecutorEndpoint.EXECUTOR_ENDPOINT_NAME, new ExecutorEndpoint(env.rpcEnv, executorId)) - // Whether to load classes in user jars before those in Spark jars private val userClassPathFirst = conf.getBoolean("spark.executor.userClassPathFirst", false) + // Whether to monitor killed / interrupted tasks + private val taskReaperEnabled = conf.getBoolean("spark.task.reaper.enabled", false) + // Create our ClassLoader // do this after SparkEnv creation so can access the SecurityManager private val urlClassLoader = createClassLoader() @@ -99,10 +131,15 @@ private[spark] class Executor( // Set the classloader for serializer env.serializer.setDefaultClassLoader(replClassLoader) + // SPARK-21928. SerializerManager's internal instance of Kryo might get used in netty threads + // for fetching remote cached RDD blocks, so need to make sure it uses the right classloader too. + env.serializerManager.setDefaultClassLoader(replClassLoader) - // Akka's message frame size. If task result is bigger than this, we use the block manager + // Max size of direct result. If task result is bigger than this, we use the block manager // to send the result back. - private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) + private val maxDirectResultSize = Math.min( + conf.getSizeAsBytes("spark.task.maxDirectResultSize", 1L << 20), + RpcUtils.maxMessageSizeBytes(conf)) // Limit of bytes for total size of results (default is 1GB) private val maxResultSize = Utils.getMaxResultSize(conf) @@ -113,30 +150,72 @@ private[spark] class Executor( // Executor for the heartbeat task. private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater") + // must be initialized before running startDriverHeartbeat() + private val heartbeatReceiverRef = + RpcUtils.makeDriverRef(HeartbeatReceiver.ENDPOINT_NAME, conf, env.rpcEnv) + + /** + * When an executor is unable to send heartbeats to the driver more than `HEARTBEAT_MAX_FAILURES` + * times, it should kill itself. The default value is 60. It means we will retry to send + * heartbeats about 10 minutes because the heartbeat interval is 10s. + */ + private val HEARTBEAT_MAX_FAILURES = conf.getInt("spark.executor.heartbeat.maxFailures", 60) + + /** + * Count the failure times of heartbeat. It should only be accessed in the heartbeat thread. Each + * successful heartbeat will reset it to 0. + */ + private var heartbeatFailures = 0 + startDriverHeartbeater() - def launchTask( - context: ExecutorBackend, - taskId: Long, - attemptNumber: Int, - taskName: String, - serializedTask: ByteBuffer): Unit = { - val tr = new TaskRunner(context, taskId = taskId, attemptNumber = attemptNumber, taskName, - serializedTask) - runningTasks.put(taskId, tr) + private[executor] def numRunningTasks: Int = runningTasks.size() + + def launchTask(context: ExecutorBackend, taskDescription: TaskDescription): Unit = { + val tr = new TaskRunner(context, taskDescription) + runningTasks.put(taskDescription.taskId, tr) threadPool.execute(tr) } - def killTask(taskId: Long, interruptThread: Boolean): Unit = { - val tr = runningTasks.get(taskId) - if (tr != null) { - tr.kill(interruptThread) + def killTask(taskId: Long, interruptThread: Boolean, reason: String): Unit = { + val taskRunner = runningTasks.get(taskId) + if (taskRunner != null) { + if (taskReaperEnabled) { + val maybeNewTaskReaper: Option[TaskReaper] = taskReaperForTask.synchronized { + val shouldCreateReaper = taskReaperForTask.get(taskId) match { + case None => true + case Some(existingReaper) => interruptThread && !existingReaper.interruptThread + } + if (shouldCreateReaper) { + val taskReaper = new TaskReaper( + taskRunner, interruptThread = interruptThread, reason = reason) + taskReaperForTask(taskId) = taskReaper + Some(taskReaper) + } else { + None + } + } + // Execute the TaskReaper from outside of the synchronized block. + maybeNewTaskReaper.foreach(taskReaperPool.execute) + } else { + taskRunner.kill(interruptThread = interruptThread, reason = reason) + } } } + /** + * Function to kill the running tasks in an executor. + * This can be called by executor back-ends to kill the + * tasks instead of taking the JVM down. + * @param interruptThread whether to interrupt the task thread + */ + def killAllTasks(interruptThread: Boolean, reason: String) : Unit = { + runningTasks.keys().asScala.foreach(t => + killTask(t, interruptThread = interruptThread, reason = reason)) + } + def stop(): Unit = { env.metricsSystem.report() - env.rpcEnv.stop(executorEndpoint) heartbeater.shutdown() heartbeater.awaitTermination(10, TimeUnit.SECONDS) threadPool.shutdown() @@ -152,14 +231,25 @@ private[spark] class Executor( class TaskRunner( execBackend: ExecutorBackend, - val taskId: Long, - val attemptNumber: Int, - taskName: String, - serializedTask: ByteBuffer) + private val taskDescription: TaskDescription) extends Runnable { - /** Whether this task has been killed. */ - @volatile private var killed = false + val taskId = taskDescription.taskId + val threadName = s"Executor task launch worker for task $taskId" + private val taskName = taskDescription.name + + /** If specified, this task has been killed and this option contains the reason. */ + @volatile private var reasonIfKilled: Option[String] = None + + @volatile private var threadId: Long = -1 + + def getThreadId: Long = threadId + + /** Whether this task has been finished. */ + @GuardedBy("TaskRunner.this") + private var finished = false + + def isFinished: Boolean = synchronized { finished } /** How much the JVM process has spent in GC when the task starts to run. */ @volatile var startGCTime: Long = _ @@ -170,89 +260,156 @@ private[spark] class Executor( */ @volatile var task: Task[Any] = _ - def kill(interruptThread: Boolean): Unit = { - logInfo(s"Executor is trying to kill $taskName (TID $taskId)") - killed = true + def kill(interruptThread: Boolean, reason: String): Unit = { + logInfo(s"Executor is trying to kill $taskName (TID $taskId), reason: $reason") + reasonIfKilled = Some(reason) if (task != null) { - task.kill(interruptThread) + synchronized { + if (!finished) { + task.kill(interruptThread, reason) + } + } } } + /** + * Set the finished flag to true and clear the current thread's interrupt status + */ + private def setTaskFinishedAndClearInterruptStatus(): Unit = synchronized { + this.finished = true + // SPARK-14234 - Reset the interrupted status of the thread to avoid the + // ClosedByInterruptException during execBackend.statusUpdate which causes + // Executor to crash + Thread.interrupted() + // Notify any waiting TaskReapers. Generally there will only be one reaper per task but there + // is a rare corner-case where one task can have two reapers in case cancel(interrupt=False) + // is followed by cancel(interrupt=True). Thus we use notifyAll() to avoid a lost wakeup: + notifyAll() + } + override def run(): Unit = { + threadId = Thread.currentThread.getId + Thread.currentThread.setName(threadName) + val threadMXBean = ManagementFactory.getThreadMXBean val taskMemoryManager = new TaskMemoryManager(env.memoryManager, taskId) val deserializeStartTime = System.currentTimeMillis() + val deserializeStartCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { + threadMXBean.getCurrentThreadCpuTime + } else 0L Thread.currentThread.setContextClassLoader(replClassLoader) val ser = env.closureSerializer.newInstance() logInfo(s"Running $taskName (TID $taskId)") execBackend.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) var taskStart: Long = 0 + var taskStartCpu: Long = 0 startGCTime = computeTotalGcTime() try { - val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) - updateDependencies(taskFiles, taskJars) - task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader) + // Must be set before updateDependencies() is called, in case fetching dependencies + // requires access to properties contained within (e.g. for access control). + Executor.taskDeserializationProps.set(taskDescription.properties) + + updateDependencies(taskDescription.addedFiles, taskDescription.addedJars) + task = ser.deserialize[Task[Any]]( + taskDescription.serializedTask, Thread.currentThread.getContextClassLoader) + task.localProperties = taskDescription.properties task.setTaskMemoryManager(taskMemoryManager) // If this task has been killed before we deserialized it, let's quit now. Otherwise, // continue executing the task. - if (killed) { + val killReason = reasonIfKilled + if (killReason.isDefined) { // Throw an exception rather than returning, because returning within a try{} block // causes a NonLocalReturnControl exception to be thrown. The NonLocalReturnControl // exception will be caught by the catch block, leading to an incorrect ExceptionFailure // for the task. - throw new TaskKilledException + throw new TaskKilledException(killReason.get) } - logDebug("Task " + taskId + "'s epoch is " + task.epoch) - env.mapOutputTracker.updateEpoch(task.epoch) + // The purpose of updating the epoch here is to invalidate executor map output status cache + // in case FetchFailures have occurred. In local mode `env.mapOutputTracker` will be + // MapOutputTrackerMaster and its cache invalidation is not based on epoch numbers so + // we don't need to make any special calls here. + if (!isLocal) { + logDebug("Task " + taskId + "'s epoch is " + task.epoch) + env.mapOutputTracker.asInstanceOf[MapOutputTrackerWorker].updateEpoch(task.epoch) + } // Run the actual task and measure its runtime. taskStart = System.currentTimeMillis() + taskStartCpu = if (threadMXBean.isCurrentThreadCpuTimeSupported) { + threadMXBean.getCurrentThreadCpuTime + } else 0L var threwException = true - val (value, accumUpdates) = try { + val value = try { val res = task.run( taskAttemptId = taskId, - attemptNumber = attemptNumber, + attemptNumber = taskDescription.attemptNumber, metricsSystem = env.metricsSystem) threwException = false res } finally { + val releasedLocks = env.blockManager.releaseAllLocksForTask(taskId) val freedMemory = taskMemoryManager.cleanUpAllAllocatedMemory() - if (freedMemory > 0) { + + if (freedMemory > 0 && !threwException) { val errMsg = s"Managed memory leak detected; size = $freedMemory bytes, TID = $taskId" - if (conf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false) && !threwException) { + if (conf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false)) { + throw new SparkException(errMsg) + } else { + logWarning(errMsg) + } + } + + if (releasedLocks.nonEmpty && !threwException) { + val errMsg = + s"${releasedLocks.size} block locks were not released by TID = $taskId:\n" + + releasedLocks.mkString("[", ", ", "]") + if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false)) { throw new SparkException(errMsg) } else { - logError(errMsg) + logInfo(errMsg) } } } + task.context.fetchFailed.foreach { fetchFailure => + // uh-oh. it appears the user code has caught the fetch-failure without throwing any + // other exceptions. Its *possible* this is what the user meant to do (though highly + // unlikely). So we will log an error and keep going. + logError(s"TID ${taskId} completed successfully though internally it encountered " + + s"unrecoverable fetch failures! Most likely this means user code is incorrectly " + + s"swallowing Spark's internal ${classOf[FetchFailedException]}", fetchFailure) + } val taskFinish = System.currentTimeMillis() + val taskFinishCpu = if (threadMXBean.isCurrentThreadCpuTimeSupported) { + threadMXBean.getCurrentThreadCpuTime + } else 0L // If the task has been killed, let's fail it. - if (task.killed) { - throw new TaskKilledException - } + task.context.killTaskIfInterrupted() val resultSer = env.serializer.newInstance() val beforeSerialization = System.currentTimeMillis() val valueBytes = resultSer.serialize(value) val afterSerialization = System.currentTimeMillis() - for (m <- task.metrics) { - // Deserialization happens in two parts: first, we deserialize a Task object, which - // includes the Partition. Second, Task.run() deserializes the RDD and function to be run. - m.setExecutorDeserializeTime( - (taskStart - deserializeStartTime) + task.executorDeserializeTime) - // We need to subtract Task.run()'s deserialization time to avoid double-counting - m.setExecutorRunTime((taskFinish - taskStart) - task.executorDeserializeTime) - m.setJvmGCTime(computeTotalGcTime() - startGCTime) - m.setResultSerializationTime(afterSerialization - beforeSerialization) - m.updateAccumulators() - } - - val directResult = new DirectTaskResult(valueBytes, accumUpdates, task.metrics.orNull) + // Deserialization happens in two parts: first, we deserialize a Task object, which + // includes the Partition. Second, Task.run() deserializes the RDD and function to be run. + task.metrics.setExecutorDeserializeTime( + (taskStart - deserializeStartTime) + task.executorDeserializeTime) + task.metrics.setExecutorDeserializeCpuTime( + (taskStartCpu - deserializeStartCpuTime) + task.executorDeserializeCpuTime) + // We need to subtract Task.run()'s deserialization time to avoid double-counting + task.metrics.setExecutorRunTime((taskFinish - taskStart) - task.executorDeserializeTime) + task.metrics.setExecutorCpuTime( + (taskFinishCpu - taskStartCpu) - task.executorDeserializeCpuTime) + task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime) + task.metrics.setResultSerializationTime(afterSerialization - beforeSerialization) + + // Note: accumulator updates must be collected after TaskMetrics is updated + val accumUpdates = task.collectAccumulatorUpdates() + // TODO: do not serialize value twice + val directResult = new DirectTaskResult(valueBytes, accumUpdates) val serializedDirectResult = ser.serialize(directResult) val resultSize = serializedDirectResult.limit @@ -263,10 +420,12 @@ private[spark] class Executor( s"(${Utils.bytesToString(resultSize)} > ${Utils.bytesToString(maxResultSize)}), " + s"dropping it.") ser.serialize(new IndirectTaskResult[Any](TaskResultBlockId(taskId), resultSize)) - } else if (resultSize >= akkaFrameSize - AkkaUtils.reservedSizeBytes) { + } else if (resultSize > maxDirectResultSize) { val blockId = TaskResultBlockId(taskId) env.blockManager.putBytes( - blockId, serializedDirectResult, StorageLevel.MEMORY_AND_DISK_SER) + blockId, + new ChunkedByteBuffer(serializedDirectResult.duplicate()), + StorageLevel.MEMORY_AND_DISK_SER) logInfo( s"Finished $taskName (TID $taskId). $resultSize bytes result sent via BlockManager)") ser.serialize(new IndirectTaskResult[Any](blockId, resultSize)) @@ -276,20 +435,41 @@ private[spark] class Executor( } } + setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult) } catch { - case ffe: FetchFailedException => - val reason = ffe.toTaskEndReason + case t: Throwable if hasFetchFailure && !Utils.isFatalError(t) => + val reason = task.context.fetchFailed.get.toTaskFailedReason + if (!t.isInstanceOf[FetchFailedException]) { + // there was a fetch failure in the task, but some user code wrapped that exception + // and threw something else. Regardless, we treat it as a fetch failure. + val fetchFailedCls = classOf[FetchFailedException].getName + logWarning(s"TID ${taskId} encountered a ${fetchFailedCls} and " + + s"failed, but the ${fetchFailedCls} was hidden by another " + + s"exception. Spark is handling this like a fetch failure and ignoring the " + + s"other exception: $t") + } + setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) - case _: TaskKilledException | _: InterruptedException if task.killed => - logInfo(s"Executor killed $taskName (TID $taskId)") - execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) + case t: TaskKilledException => + logInfo(s"Executor killed $taskName (TID $taskId), reason: ${t.reason}") + setTaskFinishedAndClearInterruptStatus() + execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled(t.reason))) - case cDE: CommitDeniedException => - val reason = cDE.toTaskEndReason - execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) + case _: InterruptedException | NonFatal(_) if + task != null && task.reasonIfKilled.isDefined => + val killReason = task.reasonIfKilled.getOrElse("unknown reason") + logInfo(s"Executor interrupted and killed $taskName (TID $taskId), reason: $killReason") + setTaskFinishedAndClearInterruptStatus() + execBackend.statusUpdate( + taskId, TaskState.KILLED, ser.serialize(TaskKilled(killReason))) + + case CausedBy(cDE: CommitDeniedException) => + val reason = cDE.toTaskCommitDeniedReason + setTaskFinishedAndClearInterruptStatus() + execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(reason)) case t: Throwable => // Attempt to exit cleanly by informing the driver of our failure. @@ -297,35 +477,165 @@ private[spark] class Executor( // the default uncaught exception handler, which will terminate the Executor. logError(s"Exception in $taskName (TID $taskId)", t) - val metrics: Option[TaskMetrics] = Option(task).flatMap { task => - task.metrics.map { m => - m.setExecutorRunTime(System.currentTimeMillis() - taskStart) - m.setJvmGCTime(computeTotalGcTime() - startGCTime) - m.updateAccumulators() - m - } - } - val serializedTaskEndReason = { - try { - ser.serialize(new ExceptionFailure(t, metrics)) - } catch { - case _: NotSerializableException => - // t is not serializable so just send the stacktrace - ser.serialize(new ExceptionFailure(t, metrics, false)) + // SPARK-20904: Do not report failure to driver if if happened during shut down. Because + // libraries may set up shutdown hooks that race with running tasks during shutdown, + // spurious failures may occur and can result in improper accounting in the driver (e.g. + // the task failure would not be ignored if the shutdown happened because of premption, + // instead of an app issue). + if (!ShutdownHookManager.inShutdown()) { + // Collect latest accumulator values to report back to the driver + val accums: Seq[AccumulatorV2[_, _]] = + if (task != null) { + task.metrics.setExecutorRunTime(System.currentTimeMillis() - taskStart) + task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime) + task.collectAccumulatorUpdates(taskFailed = true) + } else { + Seq.empty + } + + val accUpdates = accums.map(acc => acc.toInfo(Some(acc.value), None)) + + val serializedTaskEndReason = { + try { + ser.serialize(new ExceptionFailure(t, accUpdates).withAccums(accums)) + } catch { + case _: NotSerializableException => + // t is not serializable so just send the stacktrace + ser.serialize(new ExceptionFailure(t, accUpdates, false).withAccums(accums)) + } } + setTaskFinishedAndClearInterruptStatus() + execBackend.statusUpdate(taskId, TaskState.FAILED, serializedTaskEndReason) + } else { + logInfo("Not reporting error to driver during JVM shutdown.") } - execBackend.statusUpdate(taskId, TaskState.FAILED, serializedTaskEndReason) // Don't forcibly exit unless the exception was inherently fatal, to avoid // stopping other tasks unnecessarily. if (Utils.isFatalError(t)) { - SparkUncaughtExceptionHandler.uncaughtException(t) + uncaughtExceptionHandler.uncaughtException(Thread.currentThread(), t) } } finally { runningTasks.remove(taskId) } } + + private def hasFetchFailure: Boolean = { + task != null && task.context != null && task.context.fetchFailed.isDefined + } + } + + /** + * Supervises the killing / cancellation of a task by sending the interrupted flag, optionally + * sending a Thread.interrupt(), and monitoring the task until it finishes. + * + * Spark's current task cancellation / task killing mechanism is "best effort" because some tasks + * may not be interruptable or may not respond to their "killed" flags being set. If a significant + * fraction of a cluster's task slots are occupied by tasks that have been marked as killed but + * remain running then this can lead to a situation where new jobs and tasks are starved of + * resources that are being used by these zombie tasks. + * + * The TaskReaper was introduced in SPARK-18761 as a mechanism to monitor and clean up zombie + * tasks. For backwards-compatibility / backportability this component is disabled by default + * and must be explicitly enabled by setting `spark.task.reaper.enabled=true`. + * + * A TaskReaper is created for a particular task when that task is killed / cancelled. Typically + * a task will have only one TaskReaper, but it's possible for a task to have up to two reapers + * in case kill is called twice with different values for the `interrupt` parameter. + * + * Once created, a TaskReaper will run until its supervised task has finished running. If the + * TaskReaper has not been configured to kill the JVM after a timeout (i.e. if + * `spark.task.reaper.killTimeout < 0`) then this implies that the TaskReaper may run indefinitely + * if the supervised task never exits. + */ + private class TaskReaper( + taskRunner: TaskRunner, + val interruptThread: Boolean, + val reason: String) + extends Runnable { + + private[this] val taskId: Long = taskRunner.taskId + + private[this] val killPollingIntervalMs: Long = + conf.getTimeAsMs("spark.task.reaper.pollingInterval", "10s") + + private[this] val killTimeoutMs: Long = conf.getTimeAsMs("spark.task.reaper.killTimeout", "-1") + + private[this] val takeThreadDump: Boolean = + conf.getBoolean("spark.task.reaper.threadDump", true) + + override def run(): Unit = { + val startTimeMs = System.currentTimeMillis() + def elapsedTimeMs = System.currentTimeMillis() - startTimeMs + def timeoutExceeded(): Boolean = killTimeoutMs > 0 && elapsedTimeMs > killTimeoutMs + try { + // Only attempt to kill the task once. If interruptThread = false then a second kill + // attempt would be a no-op and if interruptThread = true then it may not be safe or + // effective to interrupt multiple times: + taskRunner.kill(interruptThread = interruptThread, reason = reason) + // Monitor the killed task until it exits. The synchronization logic here is complicated + // because we don't want to synchronize on the taskRunner while possibly taking a thread + // dump, but we also need to be careful to avoid races between checking whether the task + // has finished and wait()ing for it to finish. + var finished: Boolean = false + while (!finished && !timeoutExceeded()) { + taskRunner.synchronized { + // We need to synchronize on the TaskRunner while checking whether the task has + // finished in order to avoid a race where the task is marked as finished right after + // we check and before we call wait(). + if (taskRunner.isFinished) { + finished = true + } else { + taskRunner.wait(killPollingIntervalMs) + } + } + if (taskRunner.isFinished) { + finished = true + } else { + logWarning(s"Killed task $taskId is still running after $elapsedTimeMs ms") + if (takeThreadDump) { + try { + Utils.getThreadDumpForThread(taskRunner.getThreadId).foreach { thread => + if (thread.threadName == taskRunner.threadName) { + logWarning(s"Thread dump from task $taskId:\n${thread.stackTrace}") + } + } + } catch { + case NonFatal(e) => + logWarning("Exception thrown while obtaining thread dump: ", e) + } + } + } + } + + if (!taskRunner.isFinished && timeoutExceeded()) { + if (isLocal) { + logError(s"Killed task $taskId could not be stopped within $killTimeoutMs ms; " + + "not killing JVM because we are running in local mode.") + } else { + // In non-local-mode, the exception thrown here will bubble up to the uncaught exception + // handler and cause the executor JVM to exit. + throw new SparkException( + s"Killing executor JVM because killed task $taskId could not be stopped within " + + s"$killTimeoutMs ms.") + } + } + } finally { + // Clean up entries in the taskReaperForTask map. + taskReaperForTask.synchronized { + taskReaperForTask.get(taskId).foreach { taskReaperInMap => + if (taskReaperInMap eq this) { + taskReaperForTask.remove(taskId) + } else { + // This must have been a TaskReaper where interruptThread == false where a subsequent + // killTask() call for the same task had interruptThread == true and overwrote the + // map entry. + } + } + } + } + } } /** @@ -365,9 +675,9 @@ private[spark] class Executor( val _userClassPathFirst: java.lang.Boolean = userClassPathFirst val klass = Utils.classForName("org.apache.spark.repl.ExecutorClassLoader") .asInstanceOf[Class[_ <: ClassLoader]] - val constructor = klass.getConstructor(classOf[SparkConf], classOf[String], - classOf[ClassLoader], classOf[Boolean]) - constructor.newInstance(conf, classUri, parent, _userClassPathFirst) + val constructor = klass.getConstructor(classOf[SparkConf], classOf[SparkEnv], + classOf[String], classOf[ClassLoader], classOf[Boolean]) + constructor.newInstance(conf, env, classUri, parent, _userClassPathFirst) } catch { case _: ClassNotFoundException => logError("Could not find org.apache.spark.repl.ExecutorClassLoader on classpath!") @@ -383,7 +693,7 @@ private[spark] class Executor( * Download any missing dependencies if we receive a new set of files and JARs from the * SparkContext. Also adds any new JARs we fetched to the class loader. */ - private def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) { + private def updateDependencies(newFiles: Map[String, Long], newJars: Map[String, Long]) { lazy val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) synchronized { // Fetch missing dependencies @@ -395,7 +705,7 @@ private[spark] class Executor( currentFiles(name) = timestamp } for ((name, timestamp) <- newJars) { - val localName = name.split("/").last + val localName = new URI(name).getPath.split("/").last val currentTimeStamp = currentJars.get(name) .orElse(currentJars.get(localName)) .getOrElse(-1L) @@ -416,46 +726,38 @@ private[spark] class Executor( } } - private val heartbeatReceiverRef = - RpcUtils.makeDriverRef(HeartbeatReceiver.ENDPOINT_NAME, conf, env.rpcEnv) - /** Reports heartbeat and metrics for active tasks to the driver. */ private def reportHeartBeat(): Unit = { - // list of (task id, metrics) to send back to the driver - val tasksMetrics = new ArrayBuffer[(Long, TaskMetrics)]() + // list of (task id, accumUpdates) to send back to the driver + val accumUpdates = new ArrayBuffer[(Long, Seq[AccumulatorV2[_, _]])]() val curGCTime = computeTotalGcTime() for (taskRunner <- runningTasks.values().asScala) { if (taskRunner.task != null) { - taskRunner.task.metrics.foreach { metrics => - metrics.updateShuffleReadMetrics() - metrics.updateInputMetrics() - metrics.setJvmGCTime(curGCTime - taskRunner.startGCTime) - metrics.updateAccumulators() - - if (isLocal) { - // JobProgressListener will hold an reference of it during - // onExecutorMetricsUpdate(), then JobProgressListener can not see - // the changes of metrics any more, so make a deep copy of it - val copiedMetrics = Utils.deserialize[TaskMetrics](Utils.serialize(metrics)) - tasksMetrics += ((taskRunner.taskId, copiedMetrics)) - } else { - // It will be copied by serialization - tasksMetrics += ((taskRunner.taskId, metrics)) - } - } + taskRunner.task.metrics.mergeShuffleReadMetrics() + taskRunner.task.metrics.setJvmGCTime(curGCTime - taskRunner.startGCTime) + accumUpdates += ((taskRunner.taskId, taskRunner.task.metrics.accumulators())) } } - val message = Heartbeat(executorId, tasksMetrics.toArray, env.blockManager.blockManagerId) + val message = Heartbeat(executorId, accumUpdates.toArray, env.blockManager.blockManagerId) try { - val response = heartbeatReceiverRef.askWithRetry[HeartbeatResponse](message) + val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( + message, RpcTimeout(conf, "spark.executor.heartbeatInterval", "10s")) if (response.reregisterBlockManager) { logInfo("Told to re-register on heartbeat") env.blockManager.reregister() } + heartbeatFailures = 0 } catch { - case NonFatal(e) => logWarning("Issue communicating with driver in heartbeater", e) + case NonFatal(e) => + logWarning("Issue communicating with driver in heartbeater", e) + heartbeatFailures += 1 + if (heartbeatFailures >= HEARTBEAT_MAX_FAILURES) { + logError(s"Exit as unable to send heartbeats to driver " + + s"more than $HEARTBEAT_MAX_FAILURES times") + System.exit(ExecutorExitCode.HEARTBEAT_FAILURE) + } } } @@ -474,3 +776,10 @@ private[spark] class Executor( heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS) } } + +private[spark] object Executor { + // This is reserved for internal use by components that need to read task properties before a + // task is fully deserialized. When possible, the TaskContext.getLocalProperty call should be + // used instead. + val taskDeserializationProps: ThreadLocal[Properties] = new ThreadLocal[Properties] +} diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala index e07cb31cbe4b..7153323d01a0 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala @@ -25,6 +25,6 @@ import org.apache.spark.TaskState.TaskState * A pluggable interface used by the Executor to send updates to the cluster scheduler. */ private[spark] trait ExecutorBackend { - def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) + def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer): Unit } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorEndpoint.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorEndpoint.scala deleted file mode 100644 index cf362f846473..000000000000 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorEndpoint.scala +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.executor - -import org.apache.spark.rpc.{RpcEnv, RpcCallContext, RpcEndpoint} -import org.apache.spark.util.Utils - -/** - * Driver -> Executor message to trigger a thread dump. - */ -private[spark] case object TriggerThreadDump - -/** - * [[RpcEndpoint]] that runs inside of executors to enable driver -> executor RPC. - */ -private[spark] -class ExecutorEndpoint(override val rpcEnv: RpcEnv, executorId: String) extends RpcEndpoint { - - override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case TriggerThreadDump => - context.reply(Utils.getThreadDump()) - } - -} - -object ExecutorEndpoint { - val EXECUTOR_ENDPOINT_NAME = "ExecutorEndpoint" -} diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala index ea36fb60bd54..99858f785600 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -39,6 +39,12 @@ object ExecutorExitCode { /** ExternalBlockStore failed to create a local temporary directory after many attempts. */ val EXTERNAL_BLOCK_STORE_FAILED_TO_CREATE_DIR = 55 + /** + * Executor is unable to send heartbeats to the driver more than + * "spark.executor.heartbeat.maxFailures" times. + */ + val HEARTBEAT_FAILURE = 56 + def explainExitCode(exitCode: Int): String = { exitCode match { case UNCAUGHT_EXCEPTION => "Uncaught exception" @@ -51,6 +57,8 @@ object ExecutorExitCode { // TODO: replace external block store with concrete implementation name case EXTERNAL_BLOCK_STORE_FAILED_TO_CREATE_DIR => "ExternalBlockStore failed to create a local temporary directory." + case HEARTBEAT_FAILURE => + "Unable to send heartbeats to driver." case _ => "Unknown executor exit code (" + exitCode + ")" + ( if (exitCode > 128) { diff --git a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala new file mode 100644 index 000000000000..3d15f3a0396e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator + + +/** + * :: DeveloperApi :: + * Method by which input data was read. Network means that the data was read over the network + * from a remote block manager (which may have stored the data on-disk or in-memory). + * Operations are not thread-safe. + */ +@DeveloperApi +object DataReadMethod extends Enumeration with Serializable { + type DataReadMethod = Value + val Memory, Disk, Hadoop, Network = Value +} + + +/** + * :: DeveloperApi :: + * A collection of accumulators that represents metrics about reading data from external systems. + */ +@DeveloperApi +class InputMetrics private[spark] () extends Serializable { + private[executor] val _bytesRead = new LongAccumulator + private[executor] val _recordsRead = new LongAccumulator + + /** + * Total number of bytes read. + */ + def bytesRead: Long = _bytesRead.sum + + /** + * Total number of records read. + */ + def recordsRead: Long = _recordsRead.sum + + private[spark] def incBytesRead(v: Long): Unit = _bytesRead.add(v) + private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) + private[spark] def setBytesRead(v: Long): Unit = _bytesRead.setValue(v) +} diff --git a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala new file mode 100644 index 000000000000..dada9697c1cf --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator + + +/** + * :: DeveloperApi :: + * Method by which output data was written. + * Operations are not thread-safe. + */ +@DeveloperApi +object DataWriteMethod extends Enumeration with Serializable { + type DataWriteMethod = Value + val Hadoop = Value +} + + +/** + * :: DeveloperApi :: + * A collection of accumulators that represents metrics about writing data to external systems. + */ +@DeveloperApi +class OutputMetrics private[spark] () extends Serializable { + private[executor] val _bytesWritten = new LongAccumulator + private[executor] val _recordsWritten = new LongAccumulator + + /** + * Total number of bytes written. + */ + def bytesWritten: Long = _bytesWritten.sum + + /** + * Total number of records written. + */ + def recordsWritten: Long = _recordsWritten.sum + + private[spark] def setBytesWritten(v: Long): Unit = _bytesWritten.setValue(v) + private[spark] def setRecordsWritten(v: Long): Unit = _recordsWritten.setValue(v) +} diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala new file mode 100644 index 000000000000..4be395c8358b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator + + +/** + * :: DeveloperApi :: + * A collection of accumulators that represent metrics about reading shuffle data. + * Operations are not thread-safe. + */ +@DeveloperApi +class ShuffleReadMetrics private[spark] () extends Serializable { + private[executor] val _remoteBlocksFetched = new LongAccumulator + private[executor] val _localBlocksFetched = new LongAccumulator + private[executor] val _remoteBytesRead = new LongAccumulator + private[executor] val _remoteBytesReadToDisk = new LongAccumulator + private[executor] val _localBytesRead = new LongAccumulator + private[executor] val _fetchWaitTime = new LongAccumulator + private[executor] val _recordsRead = new LongAccumulator + + /** + * Number of remote blocks fetched in this shuffle by this task. + */ + def remoteBlocksFetched: Long = _remoteBlocksFetched.sum + + /** + * Number of local blocks fetched in this shuffle by this task. + */ + def localBlocksFetched: Long = _localBlocksFetched.sum + + /** + * Total number of remote bytes read from the shuffle by this task. + */ + def remoteBytesRead: Long = _remoteBytesRead.sum + + /** + * Total number of remotes bytes read to disk from the shuffle by this task. + */ + def remoteBytesReadToDisk: Long = _remoteBytesReadToDisk.sum + + /** + * Shuffle data that was read from the local disk (as opposed to from a remote executor). + */ + def localBytesRead: Long = _localBytesRead.sum + + /** + * Time the task spent waiting for remote shuffle blocks. This only includes the time + * blocking on shuffle input data. For instance if block B is being fetched while the task is + * still not finished processing block A, it is not considered to be blocking on block B. + */ + def fetchWaitTime: Long = _fetchWaitTime.sum + + /** + * Total number of records read from the shuffle by this task. + */ + def recordsRead: Long = _recordsRead.sum + + /** + * Total bytes fetched in the shuffle by this task (both remote and local). + */ + def totalBytesRead: Long = remoteBytesRead + localBytesRead + + /** + * Number of blocks fetched in this shuffle by this task (remote or local). + */ + def totalBlocksFetched: Long = remoteBlocksFetched + localBlocksFetched + + private[spark] def incRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched.add(v) + private[spark] def incLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.add(v) + private[spark] def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead.add(v) + private[spark] def incRemoteBytesReadToDisk(v: Long): Unit = _remoteBytesReadToDisk.add(v) + private[spark] def incLocalBytesRead(v: Long): Unit = _localBytesRead.add(v) + private[spark] def incFetchWaitTime(v: Long): Unit = _fetchWaitTime.add(v) + private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) + + private[spark] def setRemoteBlocksFetched(v: Int): Unit = _remoteBlocksFetched.setValue(v) + private[spark] def setLocalBlocksFetched(v: Int): Unit = _localBlocksFetched.setValue(v) + private[spark] def setRemoteBytesRead(v: Long): Unit = _remoteBytesRead.setValue(v) + private[spark] def setRemoteBytesReadToDisk(v: Long): Unit = _remoteBytesReadToDisk.setValue(v) + private[spark] def setLocalBytesRead(v: Long): Unit = _localBytesRead.setValue(v) + private[spark] def setFetchWaitTime(v: Long): Unit = _fetchWaitTime.setValue(v) + private[spark] def setRecordsRead(v: Long): Unit = _recordsRead.setValue(v) + + /** + * Resets the value of the current metrics (`this`) and merges all the independent + * [[TempShuffleReadMetrics]] into `this`. + */ + private[spark] def setMergeValues(metrics: Seq[TempShuffleReadMetrics]): Unit = { + _remoteBlocksFetched.setValue(0) + _localBlocksFetched.setValue(0) + _remoteBytesRead.setValue(0) + _remoteBytesReadToDisk.setValue(0) + _localBytesRead.setValue(0) + _fetchWaitTime.setValue(0) + _recordsRead.setValue(0) + metrics.foreach { metric => + _remoteBlocksFetched.add(metric.remoteBlocksFetched) + _localBlocksFetched.add(metric.localBlocksFetched) + _remoteBytesRead.add(metric.remoteBytesRead) + _remoteBytesReadToDisk.add(metric.remoteBytesReadToDisk) + _localBytesRead.add(metric.localBytesRead) + _fetchWaitTime.add(metric.fetchWaitTime) + _recordsRead.add(metric.recordsRead) + } + } +} + +/** + * A temporary shuffle read metrics holder that is used to collect shuffle read metrics for each + * shuffle dependency, and all temporary metrics will be merged into the [[ShuffleReadMetrics]] at + * last. + */ +private[spark] class TempShuffleReadMetrics { + private[this] var _remoteBlocksFetched = 0L + private[this] var _localBlocksFetched = 0L + private[this] var _remoteBytesRead = 0L + private[this] var _remoteBytesReadToDisk = 0L + private[this] var _localBytesRead = 0L + private[this] var _fetchWaitTime = 0L + private[this] var _recordsRead = 0L + + def incRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched += v + def incLocalBlocksFetched(v: Long): Unit = _localBlocksFetched += v + def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead += v + def incRemoteBytesReadToDisk(v: Long): Unit = _remoteBytesReadToDisk += v + def incLocalBytesRead(v: Long): Unit = _localBytesRead += v + def incFetchWaitTime(v: Long): Unit = _fetchWaitTime += v + def incRecordsRead(v: Long): Unit = _recordsRead += v + + def remoteBlocksFetched: Long = _remoteBlocksFetched + def localBlocksFetched: Long = _localBlocksFetched + def remoteBytesRead: Long = _remoteBytesRead + def remoteBytesReadToDisk: Long = _remoteBytesReadToDisk + def localBytesRead: Long = _localBytesRead + def fetchWaitTime: Long = _fetchWaitTime + def recordsRead: Long = _recordsRead +} diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala new file mode 100644 index 000000000000..ada2e1bc0859 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.LongAccumulator + + +/** + * :: DeveloperApi :: + * A collection of accumulators that represent metrics about writing shuffle data. + * Operations are not thread-safe. + */ +@DeveloperApi +class ShuffleWriteMetrics private[spark] () extends Serializable { + private[executor] val _bytesWritten = new LongAccumulator + private[executor] val _recordsWritten = new LongAccumulator + private[executor] val _writeTime = new LongAccumulator + + /** + * Number of bytes written for the shuffle by this task. + */ + def bytesWritten: Long = _bytesWritten.sum + + /** + * Total number of records written to the shuffle by this task. + */ + def recordsWritten: Long = _recordsWritten.sum + + /** + * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds. + */ + def writeTime: Long = _writeTime.sum + + private[spark] def incBytesWritten(v: Long): Unit = _bytesWritten.add(v) + private[spark] def incRecordsWritten(v: Long): Unit = _recordsWritten.add(v) + private[spark] def incWriteTime(v: Long): Unit = _writeTime.add(v) + private[spark] def decBytesWritten(v: Long): Unit = { + _bytesWritten.setValue(bytesWritten - v) + } + private[spark] def decRecordsWritten(v: Long): Unit = { + _recordsWritten.setValue(recordsWritten - v) + } + + // Legacy methods for backward compatibility. + // TODO: remove these once we make this class private. + @deprecated("use bytesWritten instead", "2.0.0") + def shuffleBytesWritten: Long = bytesWritten + @deprecated("use writeTime instead", "2.0.0") + def shuffleWriteTime: Long = writeTime + @deprecated("use recordsWritten instead", "2.0.0") + def shuffleRecordsWritten: Long = recordsWritten + +} 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 42207a955359..85b2745a2aec 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,421 +17,311 @@ package org.apache.spark.executor -import java.io.{IOException, ObjectInputStream} -import java.util.concurrent.ConcurrentHashMap - -import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConverters._ +import scala.collection.mutable.{ArrayBuffer, LinkedHashMap} +import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.DataReadMethod.DataReadMethod +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.{BlockId, BlockStatus} -import org.apache.spark.util.Utils +import org.apache.spark.util._ + /** * :: DeveloperApi :: * Metrics tracked during the execution of a task. * - * This class is used to house metrics both for in-progress and completed tasks. In executors, - * both the task thread and the heartbeat thread write to the TaskMetrics. The heartbeat thread - * reads it to send in-progress metrics, and the task thread reads it to send metrics along with - * the completed task. + * This class is wrapper around a collection of internal accumulators that represent metrics + * associated with a task. The local values of these accumulators are sent from the executor + * to the driver when the task completes. These values are then merged into the corresponding + * accumulator previously registered on the driver. * - * So, when adding new fields, take into consideration that the whole object can be serialized for - * shipping off at any time to consumers of the SparkListener interface. + * The accumulator updates are also sent to the driver periodically (on executor heartbeat) + * and when the task failed with an exception. The [[TaskMetrics]] object itself should never + * be sent to the driver. */ @DeveloperApi -class TaskMetrics extends Serializable { - /** - * Host's name the task runs on - */ - private var _hostname: String = _ - def hostname: String = _hostname - private[spark] def setHostname(value: String) = _hostname = value +class TaskMetrics private[spark] () extends Serializable { + // Each metric is internally represented as an accumulator + private val _executorDeserializeTime = new LongAccumulator + private val _executorDeserializeCpuTime = new LongAccumulator + private val _executorRunTime = new LongAccumulator + private val _executorCpuTime = new LongAccumulator + private val _resultSize = new LongAccumulator + private val _jvmGCTime = new LongAccumulator + private val _resultSerializationTime = new LongAccumulator + private val _memoryBytesSpilled = new LongAccumulator + private val _diskBytesSpilled = new LongAccumulator + private val _peakExecutionMemory = new LongAccumulator + private val _updatedBlockStatuses = new CollectionAccumulator[(BlockId, BlockStatus)] /** - * Time taken on the executor to deserialize this task + * Time taken on the executor to deserialize this task. */ - private var _executorDeserializeTime: Long = _ - def executorDeserializeTime: Long = _executorDeserializeTime - private[spark] def setExecutorDeserializeTime(value: Long) = _executorDeserializeTime = value - + def executorDeserializeTime: Long = _executorDeserializeTime.sum /** - * Time the executor spends actually running the task (including fetching shuffle data) + * CPU Time taken on the executor to deserialize this task in nanoseconds. */ - private var _executorRunTime: Long = _ - def executorRunTime: Long = _executorRunTime - private[spark] def setExecutorRunTime(value: Long) = _executorRunTime = value + def executorDeserializeCpuTime: Long = _executorDeserializeCpuTime.sum /** - * The number of bytes this task transmitted back to the driver as the TaskResult + * Time the executor spends actually running the task (including fetching shuffle data). */ - private var _resultSize: Long = _ - def resultSize: Long = _resultSize - private[spark] def setResultSize(value: Long) = _resultSize = value - + def executorRunTime: Long = _executorRunTime.sum /** - * Amount of time the JVM spent in garbage collection while executing this task + * CPU Time the executor spends actually running the task + * (including fetching shuffle data) in nanoseconds. */ - private var _jvmGCTime: Long = _ - def jvmGCTime: Long = _jvmGCTime - private[spark] def setJvmGCTime(value: Long) = _jvmGCTime = value + def executorCpuTime: Long = _executorCpuTime.sum /** - * Amount of time spent serializing the task result + * The number of bytes this task transmitted back to the driver as the TaskResult. */ - private var _resultSerializationTime: Long = _ - def resultSerializationTime: Long = _resultSerializationTime - private[spark] def setResultSerializationTime(value: Long) = _resultSerializationTime = value + def resultSize: Long = _resultSize.sum /** - * The number of in-memory bytes spilled by this task + * Amount of time the JVM spent in garbage collection while executing this task. */ - private var _memoryBytesSpilled: Long = _ - def memoryBytesSpilled: Long = _memoryBytesSpilled - private[spark] def incMemoryBytesSpilled(value: Long): Unit = _memoryBytesSpilled += value - private[spark] def decMemoryBytesSpilled(value: Long): Unit = _memoryBytesSpilled -= value + def jvmGCTime: Long = _jvmGCTime.sum /** - * The number of on-disk bytes spilled by this task + * Amount of time spent serializing the task result. */ - private var _diskBytesSpilled: Long = _ - def diskBytesSpilled: Long = _diskBytesSpilled - private[spark] def incDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled += value - private[spark] def decDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled -= value + def resultSerializationTime: Long = _resultSerializationTime.sum /** - * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read - * are stored here. + * The number of in-memory bytes spilled by this task. */ - private var _inputMetrics: Option[InputMetrics] = None - - def inputMetrics: Option[InputMetrics] = _inputMetrics + def memoryBytesSpilled: Long = _memoryBytesSpilled.sum /** - * This should only be used when recreating TaskMetrics, not when updating input metrics in - * executors + * The number of on-disk bytes spilled by this task. */ - private[spark] def setInputMetrics(inputMetrics: Option[InputMetrics]) { - _inputMetrics = inputMetrics - } + def diskBytesSpilled: Long = _diskBytesSpilled.sum /** - * If this task writes data externally (e.g. to a distributed filesystem), metrics on how much - * data was written are stored here. + * Peak memory used by internal data structures created during shuffles, aggregations and + * joins. The value of this accumulator should be approximately the sum of the peak sizes + * across all such data structures created in this task. For SQL jobs, this only tracks all + * unsafe operators and ExternalSort. */ - var outputMetrics: Option[OutputMetrics] = None + def peakExecutionMemory: Long = _peakExecutionMemory.sum /** - * If this task reads from shuffle output, metrics on getting shuffle data will be collected here. - * This includes read metrics aggregated over all the task's shuffle dependencies. - */ - private var _shuffleReadMetrics: Option[ShuffleReadMetrics] = None + * Storage statuses of any blocks that have been updated as a result of this task. + * + * Tracking the _updatedBlockStatuses can use a lot of memory. + * It is not used anywhere inside of Spark so we would ideally remove it, but its exposed to + * the user in SparkListenerTaskEnd so the api is kept for compatibility. + * Tracking can be turned off to save memory via config + * TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES. + */ + def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = { + // This is called on driver. All accumulator updates have a fixed value. So it's safe to use + // `asScala` which accesses the internal values using `java.util.Iterator`. + _updatedBlockStatuses.value.asScala + } - def shuffleReadMetrics: Option[ShuffleReadMetrics] = _shuffleReadMetrics + // Setters and increment-ers + private[spark] def setExecutorDeserializeTime(v: Long): Unit = + _executorDeserializeTime.setValue(v) + private[spark] def setExecutorDeserializeCpuTime(v: Long): Unit = + _executorDeserializeCpuTime.setValue(v) + private[spark] def setExecutorRunTime(v: Long): Unit = _executorRunTime.setValue(v) + private[spark] def setExecutorCpuTime(v: Long): Unit = _executorCpuTime.setValue(v) + private[spark] def setResultSize(v: Long): Unit = _resultSize.setValue(v) + private[spark] def setJvmGCTime(v: Long): Unit = _jvmGCTime.setValue(v) + private[spark] def setResultSerializationTime(v: Long): Unit = + _resultSerializationTime.setValue(v) + private[spark] def incMemoryBytesSpilled(v: Long): Unit = _memoryBytesSpilled.add(v) + private[spark] def incDiskBytesSpilled(v: Long): Unit = _diskBytesSpilled.add(v) + private[spark] def incPeakExecutionMemory(v: Long): Unit = _peakExecutionMemory.add(v) + private[spark] def incUpdatedBlockStatuses(v: (BlockId, BlockStatus)): Unit = + _updatedBlockStatuses.add(v) + private[spark] def setUpdatedBlockStatuses(v: java.util.List[(BlockId, BlockStatus)]): Unit = + _updatedBlockStatuses.setValue(v) + private[spark] def setUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]): Unit = + _updatedBlockStatuses.setValue(v.asJava) /** - * This should only be used when recreating TaskMetrics, not when updating read metrics in - * executors. + * Metrics related to reading data from a [[org.apache.spark.rdd.HadoopRDD]] or from persisted + * data, defined only in tasks with input. */ - private[spark] def setShuffleReadMetrics(shuffleReadMetrics: Option[ShuffleReadMetrics]) { - _shuffleReadMetrics = shuffleReadMetrics - } + val inputMetrics: InputMetrics = new InputMetrics() /** - * ShuffleReadMetrics per dependency for collecting independently while task is in progress. + * Metrics related to writing data externally (e.g. to a distributed filesystem), + * defined only in tasks with output. */ - @transient private lazy val depsShuffleReadMetrics: ArrayBuffer[ShuffleReadMetrics] = - new ArrayBuffer[ShuffleReadMetrics]() + val outputMetrics: OutputMetrics = new OutputMetrics() /** - * If this task writes to shuffle output, metrics on the written shuffle data will be collected - * here + * Metrics related to shuffle read aggregated across all shuffle dependencies. + * This is defined only if there are shuffle dependencies in this task. */ - var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None + val shuffleReadMetrics: ShuffleReadMetrics = new ShuffleReadMetrics() /** - * Storage statuses of any blocks that have been updated as a result of this task. + * Metrics related to shuffle write, defined only in shuffle map stages. */ - var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None + val shuffleWriteMetrics: ShuffleWriteMetrics = new ShuffleWriteMetrics() /** + * A list of [[TempShuffleReadMetrics]], one per shuffle dependency. + * * A task may have multiple shuffle readers for multiple dependencies. To avoid synchronization - * issues from readers in different threads, in-progress tasks use a ShuffleReadMetrics for each - * dependency, and merge these metrics before reporting them to the driver. This method returns - * a ShuffleReadMetrics for a dependency and registers it for merging later. + * issues from readers in different threads, in-progress tasks use a [[TempShuffleReadMetrics]] + * for each dependency and merge these metrics before reporting them to the driver. */ - private [spark] def createShuffleReadMetricsForDependency(): ShuffleReadMetrics = synchronized { - val readMetrics = new ShuffleReadMetrics() - depsShuffleReadMetrics += readMetrics - readMetrics - } + @transient private lazy val tempShuffleReadMetrics = new ArrayBuffer[TempShuffleReadMetrics] /** - * Returns the input metrics object that the task should use. Currently, if - * there exists an input metric with the same readMethod, we return that one - * so the caller can accumulate bytes read. If the readMethod is different - * than previously seen by this task, we return a new InputMetric but don't - * record it. + * Create a [[TempShuffleReadMetrics]] for a particular shuffle dependency. * - * Once https://issues.apache.org/jira/browse/SPARK-5225 is addressed, - * we can store all the different inputMetrics (one per readMethod). + * All usages are expected to be followed by a call to [[mergeShuffleReadMetrics]], which + * merges the temporary values synchronously. Otherwise, all temporary data collected will + * be lost. */ - private[spark] def getInputMetricsForReadMethod(readMethod: DataReadMethod): InputMetrics = { - synchronized { - _inputMetrics match { - case None => - val metrics = new InputMetrics(readMethod) - _inputMetrics = Some(metrics) - metrics - case Some(metrics @ InputMetrics(method)) if method == readMethod => - metrics - case Some(InputMetrics(method)) => - new InputMetrics(readMethod) - } - } + private[spark] def createTempShuffleReadMetrics(): TempShuffleReadMetrics = synchronized { + val readMetrics = new TempShuffleReadMetrics + tempShuffleReadMetrics += readMetrics + readMetrics } /** - * Aggregates shuffle read metrics for all registered dependencies into shuffleReadMetrics. + * Merge values across all temporary [[ShuffleReadMetrics]] into `_shuffleReadMetrics`. + * This is expected to be called on executor heartbeat and at the end of a task. */ - private[spark] def updateShuffleReadMetrics(): Unit = synchronized { - if (!depsShuffleReadMetrics.isEmpty) { - val merged = new ShuffleReadMetrics() - for (depMetrics <- depsShuffleReadMetrics) { - merged.incFetchWaitTime(depMetrics.fetchWaitTime) - merged.incLocalBlocksFetched(depMetrics.localBlocksFetched) - merged.incRemoteBlocksFetched(depMetrics.remoteBlocksFetched) - merged.incRemoteBytesRead(depMetrics.remoteBytesRead) - merged.incLocalBytesRead(depMetrics.localBytesRead) - merged.incRecordsRead(depMetrics.recordsRead) - } - _shuffleReadMetrics = Some(merged) + private[spark] def mergeShuffleReadMetrics(): Unit = synchronized { + if (tempShuffleReadMetrics.nonEmpty) { + shuffleReadMetrics.setMergeValues(tempShuffleReadMetrics) } } - private[spark] def updateInputMetrics(): Unit = synchronized { - inputMetrics.foreach(_.updateBytesRead()) - } - - @throws(classOf[IOException]) - private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { - in.defaultReadObject() - // Get the hostname from cached data, since hostname is the order of number of nodes in - // cluster, so using cached hostname will decrease the object number and alleviate the GC - // overhead. - _hostname = TaskMetrics.getCachedHostName(_hostname) - } - - private var _accumulatorUpdates: Map[Long, Any] = Map.empty - @transient private var _accumulatorsUpdater: () => Map[Long, Any] = null - - private[spark] def updateAccumulators(): Unit = synchronized { - _accumulatorUpdates = _accumulatorsUpdater() + // Only used for test + private[spark] val testAccum = sys.props.get("spark.testing").map(_ => new LongAccumulator) + + + import InternalAccumulator._ + @transient private[spark] lazy val nameToAccums = LinkedHashMap( + EXECUTOR_DESERIALIZE_TIME -> _executorDeserializeTime, + EXECUTOR_DESERIALIZE_CPU_TIME -> _executorDeserializeCpuTime, + EXECUTOR_RUN_TIME -> _executorRunTime, + EXECUTOR_CPU_TIME -> _executorCpuTime, + RESULT_SIZE -> _resultSize, + JVM_GC_TIME -> _jvmGCTime, + RESULT_SERIALIZATION_TIME -> _resultSerializationTime, + MEMORY_BYTES_SPILLED -> _memoryBytesSpilled, + DISK_BYTES_SPILLED -> _diskBytesSpilled, + PEAK_EXECUTION_MEMORY -> _peakExecutionMemory, + UPDATED_BLOCK_STATUSES -> _updatedBlockStatuses, + shuffleRead.REMOTE_BLOCKS_FETCHED -> shuffleReadMetrics._remoteBlocksFetched, + shuffleRead.LOCAL_BLOCKS_FETCHED -> shuffleReadMetrics._localBlocksFetched, + shuffleRead.REMOTE_BYTES_READ -> shuffleReadMetrics._remoteBytesRead, + shuffleRead.REMOTE_BYTES_READ_TO_DISK -> shuffleReadMetrics._remoteBytesReadToDisk, + shuffleRead.LOCAL_BYTES_READ -> shuffleReadMetrics._localBytesRead, + shuffleRead.FETCH_WAIT_TIME -> shuffleReadMetrics._fetchWaitTime, + shuffleRead.RECORDS_READ -> shuffleReadMetrics._recordsRead, + shuffleWrite.BYTES_WRITTEN -> shuffleWriteMetrics._bytesWritten, + shuffleWrite.RECORDS_WRITTEN -> shuffleWriteMetrics._recordsWritten, + shuffleWrite.WRITE_TIME -> shuffleWriteMetrics._writeTime, + input.BYTES_READ -> inputMetrics._bytesRead, + input.RECORDS_READ -> inputMetrics._recordsRead, + output.BYTES_WRITTEN -> outputMetrics._bytesWritten, + output.RECORDS_WRITTEN -> outputMetrics._recordsWritten + ) ++ testAccum.map(TEST_ACCUM -> _) + + @transient private[spark] lazy val internalAccums: Seq[AccumulatorV2[_, _]] = + nameToAccums.values.toIndexedSeq + + /* ========================== * + | OTHER THINGS | + * ========================== */ + + private[spark] def register(sc: SparkContext): Unit = { + nameToAccums.foreach { + case (name, acc) => acc.register(sc, name = Some(name), countFailedValues = true) + } } /** - * Return the latest updates of accumulators in this task. + * External accumulators registered with this task. */ - def accumulatorUpdates(): Map[Long, Any] = _accumulatorUpdates + @transient private[spark] lazy val externalAccums = new ArrayBuffer[AccumulatorV2[_, _]] - private[spark] def setAccumulatorsUpdater(accumulatorsUpdater: () => Map[Long, Any]): Unit = { - _accumulatorsUpdater = accumulatorsUpdater + private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit = { + externalAccums += a } -} - -private[spark] object TaskMetrics { - private val hostNameCache = new ConcurrentHashMap[String, String]() - def empty: TaskMetrics = new TaskMetrics + private[spark] def accumulators(): Seq[AccumulatorV2[_, _]] = internalAccums ++ externalAccums - def getCachedHostName(host: String): String = { - val canonicalHost = hostNameCache.putIfAbsent(host, host) - if (canonicalHost != null) canonicalHost else host + private[spark] def nonZeroInternalAccums(): Seq[AccumulatorV2[_, _]] = { + // RESULT_SIZE accumulator is always zero at executor, we need to send it back as its + // value will be updated at driver side. + internalAccums.filter(a => !a.isZero || a == _resultSize) } } -/** - * :: DeveloperApi :: - * Method by which input data was read. Network means that the data was read over the network - * from a remote block manager (which may have stored the data on-disk or in-memory). - */ -@DeveloperApi -object DataReadMethod extends Enumeration with Serializable { - type DataReadMethod = Value - val Memory, Disk, Hadoop, Network = Value -} -/** - * :: DeveloperApi :: - * Method by which output data was written. - */ -@DeveloperApi -object DataWriteMethod extends Enumeration with Serializable { - type DataWriteMethod = Value - val Hadoop = Value -} - -/** - * :: DeveloperApi :: - * Metrics about reading input data. - */ -@DeveloperApi -case class InputMetrics(readMethod: DataReadMethod.Value) { +private[spark] object TaskMetrics extends Logging { + import InternalAccumulator._ /** - * This is volatile so that it is visible to the updater thread. + * Create an empty task metrics that doesn't register its accumulators. */ - @volatile @transient var bytesReadCallback: Option[() => Long] = None - - /** - * Total bytes read. - */ - private var _bytesRead: Long = _ - def bytesRead: Long = _bytesRead - def incBytesRead(bytes: Long): Unit = _bytesRead += bytes - - /** - * Total records read. - */ - private var _recordsRead: Long = _ - def recordsRead: Long = _recordsRead - def incRecordsRead(records: Long): Unit = _recordsRead += records - - /** - * Invoke the bytesReadCallback and mutate bytesRead. - */ - def updateBytesRead() { - bytesReadCallback.foreach { c => - _bytesRead = c() + def empty: TaskMetrics = { + val tm = new TaskMetrics + tm.nameToAccums.foreach { case (name, acc) => + acc.metadata = AccumulatorMetadata(AccumulatorContext.newId(), Some(name), true) } + tm } - /** - * Register a function that can be called to get up-to-date information on how many bytes the task - * has read from an input source. - */ - def setBytesReadCallback(f: Option[() => Long]) { - bytesReadCallback = f + def registered: TaskMetrics = { + val tm = empty + tm.internalAccums.foreach(AccumulatorContext.register) + tm } -} - -/** - * :: DeveloperApi :: - * Metrics about writing output data. - */ -@DeveloperApi -case class OutputMetrics(writeMethod: DataWriteMethod.Value) { - /** - * Total bytes written - */ - private var _bytesWritten: Long = _ - def bytesWritten: Long = _bytesWritten - private[spark] def setBytesWritten(value : Long): Unit = _bytesWritten = value - - /** - * Total records written - */ - private var _recordsWritten: Long = 0L - def recordsWritten: Long = _recordsWritten - private[spark] def setRecordsWritten(value: Long): Unit = _recordsWritten = value -} - -/** - * :: DeveloperApi :: - * Metrics pertaining to shuffle data read in a given task. - */ -@DeveloperApi -class ShuffleReadMetrics extends Serializable { - /** - * Number of remote blocks fetched in this shuffle by this task - */ - private var _remoteBlocksFetched: Int = _ - def remoteBlocksFetched: Int = _remoteBlocksFetched - private[spark] def incRemoteBlocksFetched(value: Int) = _remoteBlocksFetched += value - private[spark] def decRemoteBlocksFetched(value: Int) = _remoteBlocksFetched -= value - - /** - * Number of local blocks fetched in this shuffle by this task - */ - private var _localBlocksFetched: Int = _ - def localBlocksFetched: Int = _localBlocksFetched - private[spark] def incLocalBlocksFetched(value: Int) = _localBlocksFetched += value - private[spark] def decLocalBlocksFetched(value: Int) = _localBlocksFetched -= value - - /** - * Time the task spent waiting for remote shuffle blocks. This only includes the time - * blocking on shuffle input data. For instance if block B is being fetched while the task is - * still not finished processing block A, it is not considered to be blocking on block B. - */ - private var _fetchWaitTime: Long = _ - def fetchWaitTime: Long = _fetchWaitTime - private[spark] def incFetchWaitTime(value: Long) = _fetchWaitTime += value - private[spark] def decFetchWaitTime(value: Long) = _fetchWaitTime -= value - - /** - * Total number of remote bytes read from the shuffle by this task - */ - private var _remoteBytesRead: Long = _ - def remoteBytesRead: Long = _remoteBytesRead - private[spark] def incRemoteBytesRead(value: Long) = _remoteBytesRead += value - private[spark] def decRemoteBytesRead(value: Long) = _remoteBytesRead -= value /** - * Shuffle data that was read from the local disk (as opposed to from a remote executor). - */ - private var _localBytesRead: Long = _ - def localBytesRead: Long = _localBytesRead - private[spark] def incLocalBytesRead(value: Long) = _localBytesRead += value - - /** - * Total bytes fetched in the shuffle by this task (both remote and local). - */ - def totalBytesRead: Long = _remoteBytesRead + _localBytesRead - - /** - * Number of blocks fetched in this shuffle by this task (remote or local) - */ - def totalBlocksFetched: Int = _remoteBlocksFetched + _localBlocksFetched - - /** - * Total number of records read from the shuffle by this task - */ - private var _recordsRead: Long = _ - def recordsRead: Long = _recordsRead - private[spark] def incRecordsRead(value: Long) = _recordsRead += value - private[spark] def decRecordsRead(value: Long) = _recordsRead -= value -} - -/** - * :: DeveloperApi :: - * Metrics pertaining to shuffle data written in a given task. - */ -@DeveloperApi -class ShuffleWriteMetrics extends Serializable { - /** - * Number of bytes written for the shuffle by this task - */ - @volatile private var _shuffleBytesWritten: Long = _ - def shuffleBytesWritten: Long = _shuffleBytesWritten - private[spark] def incShuffleBytesWritten(value: Long) = _shuffleBytesWritten += value - private[spark] def decShuffleBytesWritten(value: Long) = _shuffleBytesWritten -= value - - /** - * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds - */ - @volatile private var _shuffleWriteTime: Long = _ - def shuffleWriteTime: Long = _shuffleWriteTime - private[spark] def incShuffleWriteTime(value: Long) = _shuffleWriteTime += value - private[spark] def decShuffleWriteTime(value: Long) = _shuffleWriteTime -= value + * Construct a [[TaskMetrics]] object from a list of [[AccumulableInfo]], called on driver only. + * The returned [[TaskMetrics]] is only used to get some internal metrics, we don't need to take + * care of external accumulator info passed in. + */ + def fromAccumulatorInfos(infos: Seq[AccumulableInfo]): TaskMetrics = { + val tm = new TaskMetrics + infos.filter(info => info.name.isDefined && info.update.isDefined).foreach { info => + val name = info.name.get + val value = info.update.get + if (name == UPDATED_BLOCK_STATUSES) { + tm.setUpdatedBlockStatuses(value.asInstanceOf[java.util.List[(BlockId, BlockStatus)]]) + } else { + tm.nameToAccums.get(name).foreach( + _.asInstanceOf[LongAccumulator].setValue(value.asInstanceOf[Long]) + ) + } + } + tm + } /** - * Total number of records written to the shuffle by this task + * Construct a [[TaskMetrics]] object from a list of accumulator updates, called on driver only. */ - @volatile private var _shuffleRecordsWritten: Long = _ - def shuffleRecordsWritten: Long = _shuffleRecordsWritten - private[spark] def incShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten += value - private[spark] def decShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten -= value - private[spark] def setShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten = value + def fromAccumulators(accums: Seq[AccumulatorV2[_, _]]): TaskMetrics = { + val tm = new TaskMetrics + for (acc <- accums) { + val name = acc.name + if (name.isDefined && tm.nameToAccums.contains(name.get)) { + val tmAcc = tm.nameToAccums(name.get).asInstanceOf[AccumulatorV2[Any, Any]] + tmAcc.metadata = acc.metadata + tmAcc.merge(acc.asInstanceOf[AccumulatorV2[Any, Any]]) + } else { + tm.externalAccums += acc + } + } + tm + } } diff --git a/core/src/main/scala/org/apache/spark/executor/package-info.java b/core/src/main/scala/org/apache/spark/executor/package-info.java index dd3b6815fb45..fb280964c490 100644 --- a/core/src/main/scala/org/apache/spark/executor/package-info.java +++ b/core/src/main/scala/org/apache/spark/executor/package-info.java @@ -18,4 +18,4 @@ /** * Package for executor components used with various cluster managers. */ -package org.apache.spark.executor; \ No newline at end of file +package org.apache.spark.executor; diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala index 532850dd5771..978afaffab30 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala @@ -19,11 +19,10 @@ package org.apache.spark.input import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{BytesWritable, LongWritable} -import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAttemptContext} +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat -import org.apache.spark.Logging -import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging /** * Custom Input Format for reading and splitting flat binary files that contain records, @@ -36,7 +35,7 @@ private[spark] object FixedLengthBinaryInputFormat { /** Retrieves the record length property from a Hadoop configuration */ def getRecordLength(context: JobContext): Int = { - SparkHadoopUtil.get.getConfigurationFromJobContext(context).get(RECORD_LENGTH_PROPERTY).toInt + context.getConfiguration.get(RECORD_LENGTH_PROPERTY).toInt } } diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala index 67a96925da01..549395314ba6 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala @@ -20,11 +20,10 @@ package org.apache.spark.input import java.io.IOException import org.apache.hadoop.fs.FSDataInputStream -import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.hadoop.io.{BytesWritable, LongWritable} +import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.input.FileSplit -import org.apache.spark.deploy.SparkHadoopUtil /** * FixedLengthBinaryRecordReader is returned by FixedLengthBinaryInputFormat. @@ -83,16 +82,16 @@ private[spark] class FixedLengthBinaryRecordReader // the actual file we will be reading from val file = fileSplit.getPath // job configuration - val job = SparkHadoopUtil.get.getConfigurationFromJobContext(context) + val conf = context.getConfiguration // check compression - val codec = new CompressionCodecFactory(job).getCodec(file) + val codec = new CompressionCodecFactory(conf).getCodec(file) if (codec != null) { throw new IOException("FixedLengthRecordReader does not support reading compressed files") } // get the record length recordLength = FixedLengthBinaryInputFormat.getRecordLength(context) // get the filesystem - val fs = file.getFileSystem(job) + val fs = file.getFileSystem(conf) // open the File fileInputStream = fs.open(file) // seek to the splitStart position diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index 280e7a5fe893..17cdba4f1305 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -21,13 +21,15 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, Da import scala.collection.JavaConverters._ -import com.google.common.io.{Closeables, ByteStreams} +import com.google.common.io.{ByteStreams, Closeables} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.input.{CombineFileInputFormat, CombineFileRecordReader, CombineFileSplit} -import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Since +import org.apache.spark.internal.config /** * A general format for reading whole files in as streams, byte arrays, @@ -42,10 +44,14 @@ private[spark] abstract class StreamFileInputFormat[T] * Allow minPartitions set by end-user in order to keep compatibility with old Hadoop API * which is set through setMaxSplitSize */ - def setMinPartitions(context: JobContext, minPartitions: Int) { + def setMinPartitions(sc: SparkContext, context: JobContext, minPartitions: Int) { + val defaultMaxSplitBytes = sc.getConf.get(config.FILES_MAX_PARTITION_BYTES) + val openCostInBytes = sc.getConf.get(config.FILES_OPEN_COST_IN_BYTES) + val defaultParallelism = sc.defaultParallelism val files = listStatus(context).asScala - val totalLen = files.map(file => if (file.isDir) 0L else file.getLen).sum - val maxSplitSize = Math.ceil(totalLen * 1.0 / files.size).toLong + val totalBytes = files.filterNot(_.isDirectory).map(_.getLen + openCostInBytes).sum + val bytesPerCore = totalBytes / defaultParallelism + val maxSplitSize = Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore)) super.setMaxSplitSize(maxSplitSize) } @@ -135,8 +141,7 @@ class PortableDataStream( private val confBytes = { val baos = new ByteArrayOutputStream() - SparkHadoopUtil.get.getConfigurationFromJobContext(context). - write(new DataOutputStream(baos)) + context.getConfiguration.write(new DataOutputStream(baos)) baos.toByteArray } @@ -171,6 +176,7 @@ class PortableDataStream( * Create a new DataInputStream from the split and context. The user of this method is responsible * for closing the stream after usage. */ + @Since("1.2.0") def open(): DataInputStream = { val pathp = split.getPath(index) val fs = pathp.getFileSystem(conf) @@ -180,6 +186,7 @@ class PortableDataStream( /** * Read the file as a byte array */ + @Since("1.2.0") def toArray(): Array[Byte] = { val stream = open() try { @@ -189,15 +196,10 @@ class PortableDataStream( } } - /** - * Closing the PortableDataStream is not needed anymore. The user either can use the - * PortableDataStream to get a DataInputStream (which the user needs to close after usage), - * or a byte array. - */ - @deprecated("Closing the PortableDataStream is not needed anymore.", "1.6.0") - def close(): Unit = { - } - + @Since("1.2.0") def getPath(): String = path + + @Since("2.2.0") + def getConfiguration: Configuration = conf } diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala index 1ba34a11414a..f47cd38d712c 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala @@ -20,11 +20,9 @@ package org.apache.spark.input import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce.InputSplit -import org.apache.hadoop.mapreduce.JobContext +import org.apache.hadoop.io.Text +import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat -import org.apache.hadoop.mapreduce.RecordReader -import org.apache.hadoop.mapreduce.TaskAttemptContext /** * A [[org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat CombineFileInputFormat]] for @@ -33,14 +31,13 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext */ private[spark] class WholeTextFileInputFormat - extends CombineFileInputFormat[String, String] with Configurable { + extends CombineFileInputFormat[Text, Text] with Configurable { override protected def isSplitable(context: JobContext, file: Path): Boolean = false override def createRecordReader( split: InputSplit, - context: TaskAttemptContext): RecordReader[String, String] = { - + context: TaskAttemptContext): RecordReader[Text, Text] = { val reader = new ConfigurableCombineFileRecordReader(split, context, classOf[WholeTextFileRecordReader]) reader.setConf(getConf) @@ -53,7 +50,7 @@ private[spark] class WholeTextFileInputFormat */ def setMinPartitions(context: JobContext, minPartitions: Int) { val files = listStatus(context).asScala - val totalLen = files.map(file => if (file.isDir) 0L else file.getLen).sum + val totalLen = files.map(file => if (file.isDirectory) 0L else file.getLen).sum val maxSplitSize = Math.ceil(totalLen * 1.0 / (if (minPartitions == 0) 1 else minPartitions)).toLong super.setMaxSplitSize(maxSplitSize) diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala index 31bde8a78f3c..6b7f086678e9 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala @@ -17,17 +17,14 @@ package org.apache.spark.input -import org.apache.hadoop.conf.{Configuration, Configurable => HConfigurable} import com.google.common.io.{ByteStreams, Closeables} - +import org.apache.hadoop.conf.{Configurable => HConfigurable, Configuration} import org.apache.hadoop.io.Text import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.hadoop.mapreduce.InputSplit -import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, CombineFileRecordReader} import org.apache.hadoop.mapreduce.RecordReader import org.apache.hadoop.mapreduce.TaskAttemptContext -import org.apache.spark.deploy.SparkHadoopUtil - +import org.apache.hadoop.mapreduce.lib.input.{CombineFileRecordReader, CombineFileSplit} /** * A trait to implement [[org.apache.hadoop.conf.Configurable Configurable]] interface. @@ -49,17 +46,16 @@ private[spark] class WholeTextFileRecordReader( split: CombineFileSplit, context: TaskAttemptContext, index: Integer) - extends RecordReader[String, String] with Configurable { + extends RecordReader[Text, Text] with Configurable { private[this] val path = split.getPath(index) - private[this] val fs = path.getFileSystem( - SparkHadoopUtil.get.getConfigurationFromJobContext(context)) + private[this] val fs = path.getFileSystem(context.getConfiguration) // True means the current file has been processed, then skip it. private[this] var processed = false - private[this] val key = path.toString - private[this] var value: String = null + private[this] val key: Text = new Text(path.toString) + private[this] var value: Text = null override def initialize(split: InputSplit, context: TaskAttemptContext): Unit = {} @@ -67,9 +63,9 @@ private[spark] class WholeTextFileRecordReader( override def getProgress: Float = if (processed) 1.0f else 0.0f - override def getCurrentKey: String = key + override def getCurrentKey: Text = key - override def getCurrentValue: String = value + override def getCurrentValue: Text = value override def nextKeyValue(): Boolean = { if (!processed) { @@ -83,7 +79,7 @@ private[spark] class WholeTextFileRecordReader( ByteStreams.toByteArray(fileIn) } - value = new Text(innerBuffer).toString + value = new Text(innerBuffer) Closeables.close(fileIn, false) processed = true true diff --git a/core/src/main/scala/org/apache/spark/internal/Logging.scala b/core/src/main/scala/org/apache/spark/internal/Logging.scala new file mode 100644 index 000000000000..c0d709ad25f2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/Logging.scala @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal + +import org.apache.log4j.{Level, LogManager, PropertyConfigurator} +import org.slf4j.{Logger, LoggerFactory} +import org.slf4j.impl.StaticLoggerBinder + +import org.apache.spark.util.Utils + +/** + * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows + * logging messages at different levels using methods that only evaluate parameters lazily if the + * log level is enabled. + */ +trait Logging { + + // Make the log field transient so that objects with Logging can + // be serialized and used on another machine + @transient private var log_ : Logger = null + + // Method to get the logger name for this object + protected def logName = { + // Ignore trailing $'s in the class names for Scala objects + this.getClass.getName.stripSuffix("$") + } + + // Method to get or create the logger for this object + protected def log: Logger = { + if (log_ == null) { + initializeLogIfNecessary(false) + log_ = LoggerFactory.getLogger(logName) + } + log_ + } + + // Log methods that take only a String + protected def logInfo(msg: => String) { + if (log.isInfoEnabled) log.info(msg) + } + + protected def logDebug(msg: => String) { + if (log.isDebugEnabled) log.debug(msg) + } + + protected def logTrace(msg: => String) { + if (log.isTraceEnabled) log.trace(msg) + } + + protected def logWarning(msg: => String) { + if (log.isWarnEnabled) log.warn(msg) + } + + protected def logError(msg: => String) { + if (log.isErrorEnabled) log.error(msg) + } + + // Log methods that take Throwables (Exceptions/Errors) too + protected def logInfo(msg: => String, throwable: Throwable) { + if (log.isInfoEnabled) log.info(msg, throwable) + } + + protected def logDebug(msg: => String, throwable: Throwable) { + if (log.isDebugEnabled) log.debug(msg, throwable) + } + + protected def logTrace(msg: => String, throwable: Throwable) { + if (log.isTraceEnabled) log.trace(msg, throwable) + } + + protected def logWarning(msg: => String, throwable: Throwable) { + if (log.isWarnEnabled) log.warn(msg, throwable) + } + + protected def logError(msg: => String, throwable: Throwable) { + if (log.isErrorEnabled) log.error(msg, throwable) + } + + protected def isTraceEnabled(): Boolean = { + log.isTraceEnabled + } + + protected def initializeLogIfNecessary(isInterpreter: Boolean): Unit = { + initializeLogIfNecessary(isInterpreter, silent = false) + } + + protected def initializeLogIfNecessary( + isInterpreter: Boolean, + silent: Boolean = false): Boolean = { + if (!Logging.initialized) { + Logging.initLock.synchronized { + if (!Logging.initialized) { + initializeLogging(isInterpreter, silent) + return true + } + } + } + false + } + + private def initializeLogging(isInterpreter: Boolean, silent: Boolean): Unit = { + // Don't use a logger in here, as this is itself occurring during initialization of a logger + // If Log4j 1.2 is being used, but is not initialized, load a default properties file + if (Logging.isLog4j12()) { + val log4j12Initialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements + // scalastyle:off println + if (!log4j12Initialized) { + Logging.defaultSparkLog4jConfig = true + val defaultLogProps = "org/apache/spark/log4j-defaults.properties" + Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match { + case Some(url) => + PropertyConfigurator.configure(url) + if (!silent) { + System.err.println(s"Using Spark's default log4j profile: $defaultLogProps") + } + case None => + System.err.println(s"Spark was unable to load $defaultLogProps") + } + } + + val rootLogger = LogManager.getRootLogger() + if (Logging.defaultRootLevel == null) { + Logging.defaultRootLevel = rootLogger.getLevel() + } + + if (isInterpreter) { + // Use the repl's main class to define the default log level when running the shell, + // overriding the root logger's config if they're different. + val replLogger = LogManager.getLogger(logName) + val replLevel = Option(replLogger.getLevel()).getOrElse(Level.WARN) + if (replLevel != rootLogger.getEffectiveLevel()) { + if (!silent) { + System.err.printf("Setting default log level to \"%s\".\n", replLevel) + System.err.println("To adjust logging level use sc.setLogLevel(newLevel). " + + "For SparkR, use setLogLevel(newLevel).") + } + rootLogger.setLevel(replLevel) + } + } + // scalastyle:on println + } + Logging.initialized = true + + // Force a call into slf4j to initialize it. Avoids this happening from multiple threads + // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html + log + } +} + +private[spark] object Logging { + @volatile private var initialized = false + @volatile private var defaultRootLevel: Level = null + @volatile private var defaultSparkLog4jConfig = false + + val initLock = new Object() + try { + // We use reflection here to handle the case where users remove the + // slf4j-to-jul bridge order to route their logs to JUL. + val bridgeClass = Utils.classForName("org.slf4j.bridge.SLF4JBridgeHandler") + bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) + val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] + if (!installed) { + bridgeClass.getMethod("install").invoke(null) + } + } catch { + case e: ClassNotFoundException => // can't log anything yet so just fail silently + } + + /** + * Marks the logging system as not initialized. This does a best effort at resetting the + * logging system to its initial state so that the next class to use logging triggers + * initialization again. + */ + def uninitialize(): Unit = initLock.synchronized { + if (isLog4j12()) { + if (defaultSparkLog4jConfig) { + defaultSparkLog4jConfig = false + LogManager.resetConfiguration() + } else { + LogManager.getRootLogger().setLevel(defaultRootLevel) + } + } + this.initialized = false + } + + private def isLog4j12(): Boolean = { + // This distinguishes the log4j 1.2 binding, currently + // org.slf4j.impl.Log4jLoggerFactory, from the log4j 2.0 binding, currently + // org.apache.logging.slf4j.Log4jLoggerFactory + val binderClass = StaticLoggerBinder.getSingleton.getLoggerFactoryClassStr + "org.slf4j.impl.Log4jLoggerFactory".equals(binderClass) + } +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala new file mode 100644 index 000000000000..8f4c1b60920d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.config + +import java.util.concurrent.TimeUnit +import java.util.regex.PatternSyntaxException + +import scala.util.matching.Regex + +import org.apache.spark.network.util.{ByteUnit, JavaUtils} + +private object ConfigHelpers { + + def toNumber[T](s: String, converter: String => T, key: String, configType: String): T = { + try { + converter(s.trim) + } catch { + case _: NumberFormatException => + throw new IllegalArgumentException(s"$key should be $configType, but was $s") + } + } + + def toBoolean(s: String, key: String): Boolean = { + try { + s.trim.toBoolean + } catch { + case _: IllegalArgumentException => + throw new IllegalArgumentException(s"$key should be boolean, but was $s") + } + } + + def stringToSeq[T](str: String, converter: String => T): Seq[T] = { + str.split(",").map(_.trim()).filter(_.nonEmpty).map(converter) + } + + def seqToString[T](v: Seq[T], stringConverter: T => String): String = { + v.map(stringConverter).mkString(",") + } + + def timeFromString(str: String, unit: TimeUnit): Long = JavaUtils.timeStringAs(str, unit) + + def timeToString(v: Long, unit: TimeUnit): String = TimeUnit.MILLISECONDS.convert(v, unit) + "ms" + + def byteFromString(str: String, unit: ByteUnit): Long = { + val (input, multiplier) = + if (str.length() > 0 && str.charAt(0) == '-') { + (str.substring(1), -1) + } else { + (str, 1) + } + multiplier * JavaUtils.byteStringAs(input, unit) + } + + def byteToString(v: Long, unit: ByteUnit): String = unit.convertTo(v, ByteUnit.BYTE) + "b" + + def regexFromString(str: String, key: String): Regex = { + try str.r catch { + case e: PatternSyntaxException => + throw new IllegalArgumentException(s"$key should be a regex, but was $str", e) + } + } + +} + +/** + * A type-safe config builder. Provides methods for transforming the input data (which can be + * used, e.g., for validation) and creating the final config entry. + * + * One of the methods that return a [[ConfigEntry]] must be called to create a config entry that + * can be used with [[SparkConf]]. + */ +private[spark] class TypedConfigBuilder[T]( + val parent: ConfigBuilder, + val converter: String => T, + val stringConverter: T => String) { + + import ConfigHelpers._ + + def this(parent: ConfigBuilder, converter: String => T) = { + this(parent, converter, Option(_).map(_.toString).orNull) + } + + /** Apply a transformation to the user-provided values of the config entry. */ + def transform(fn: T => T): TypedConfigBuilder[T] = { + new TypedConfigBuilder(parent, s => fn(converter(s)), stringConverter) + } + + /** Checks if the user-provided value for the config matches the validator. */ + def checkValue(validator: T => Boolean, errorMsg: String): TypedConfigBuilder[T] = { + transform { v => + if (!validator(v)) throw new IllegalArgumentException(errorMsg) + v + } + } + + /** Check that user-provided values for the config match a pre-defined set. */ + def checkValues(validValues: Set[T]): TypedConfigBuilder[T] = { + transform { v => + if (!validValues.contains(v)) { + throw new IllegalArgumentException( + s"The value of ${parent.key} should be one of ${validValues.mkString(", ")}, but was $v") + } + v + } + } + + /** Turns the config entry into a sequence of values of the underlying type. */ + def toSequence: TypedConfigBuilder[Seq[T]] = { + new TypedConfigBuilder(parent, stringToSeq(_, converter), seqToString(_, stringConverter)) + } + + /** Creates a [[ConfigEntry]] that does not have a default value. */ + def createOptional: OptionalConfigEntry[T] = { + val entry = new OptionalConfigEntry[T](parent.key, parent._alternatives, converter, + stringConverter, parent._doc, parent._public) + parent._onCreate.foreach(_(entry)) + entry + } + + /** Creates a [[ConfigEntry]] that has a default value. */ + def createWithDefault(default: T): ConfigEntry[T] = { + // Treat "String" as a special case, so that both createWithDefault and createWithDefaultString + // behave the same w.r.t. variable expansion of default values. + if (default.isInstanceOf[String]) { + createWithDefaultString(default.asInstanceOf[String]) + } else { + val transformedDefault = converter(stringConverter(default)) + val entry = new ConfigEntryWithDefault[T](parent.key, parent._alternatives, + transformedDefault, converter, stringConverter, parent._doc, parent._public) + parent._onCreate.foreach(_(entry)) + entry + } + } + + /** Creates a [[ConfigEntry]] with a function to determine the default value */ + def createWithDefaultFunction(defaultFunc: () => T): ConfigEntry[T] = { + val entry = new ConfigEntryWithDefaultFunction[T](parent.key, parent._alternatives, defaultFunc, + converter, stringConverter, parent._doc, parent._public) + parent._onCreate.foreach(_ (entry)) + entry + } + + /** + * Creates a [[ConfigEntry]] that has a default value. The default value is provided as a + * [[String]] and must be a valid value for the entry. + */ + def createWithDefaultString(default: String): ConfigEntry[T] = { + val entry = new ConfigEntryWithDefaultString[T](parent.key, parent._alternatives, default, + converter, stringConverter, parent._doc, parent._public) + parent._onCreate.foreach(_(entry)) + entry + } + +} + +/** + * Basic builder for Spark configurations. Provides methods for creating type-specific builders. + * + * @see TypedConfigBuilder + */ +private[spark] case class ConfigBuilder(key: String) { + + import ConfigHelpers._ + + private[config] var _public = true + private[config] var _doc = "" + private[config] var _onCreate: Option[ConfigEntry[_] => Unit] = None + private[config] var _alternatives = List.empty[String] + + def internal(): ConfigBuilder = { + _public = false + this + } + + def doc(s: String): ConfigBuilder = { + _doc = s + this + } + + /** + * Registers a callback for when the config entry is finally instantiated. Currently used by + * SQLConf to keep track of SQL configuration entries. + */ + def onCreate(callback: ConfigEntry[_] => Unit): ConfigBuilder = { + _onCreate = Option(callback) + this + } + + def withAlternative(key: String): ConfigBuilder = { + _alternatives = _alternatives :+ key + this + } + + def intConf: TypedConfigBuilder[Int] = { + new TypedConfigBuilder(this, toNumber(_, _.toInt, key, "int")) + } + + def longConf: TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, toNumber(_, _.toLong, key, "long")) + } + + def doubleConf: TypedConfigBuilder[Double] = { + new TypedConfigBuilder(this, toNumber(_, _.toDouble, key, "double")) + } + + def booleanConf: TypedConfigBuilder[Boolean] = { + new TypedConfigBuilder(this, toBoolean(_, key)) + } + + def stringConf: TypedConfigBuilder[String] = { + new TypedConfigBuilder(this, v => v) + } + + def timeConf(unit: TimeUnit): TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, timeFromString(_, unit), timeToString(_, unit)) + } + + def bytesConf(unit: ByteUnit): TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, byteFromString(_, unit), byteToString(_, unit)) + } + + def fallbackConf[T](fallback: ConfigEntry[T]): ConfigEntry[T] = { + new FallbackConfigEntry(key, _alternatives, _doc, _public, fallback) + } + + def regexConf: TypedConfigBuilder[Regex] = { + new TypedConfigBuilder(this, regexFromString(_, this.key), _.toString) + } +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala new file mode 100644 index 000000000000..f1190289244e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.config + +/** + * An entry contains all meta information for a configuration. + * + * When applying variable substitution to config values, only references starting with "spark." are + * considered in the default namespace. For known Spark configuration keys (i.e. those created using + * `ConfigBuilder`), references will also consider the default value when it exists. + * + * Variable expansion is also applied to the default values of config entries that have a default + * value declared as a string. + * + * @param key the key for the configuration + * @param valueConverter how to convert a string to the value. It should throw an exception if the + * string does not have the required format. + * @param stringConverter how to convert a value to a string that the user can use it as a valid + * string value. It's usually `toString`. But sometimes, a custom converter + * is necessary. E.g., if T is List[String], `a, b, c` is better than + * `List(a, b, c)`. + * @param doc the documentation for the configuration + * @param isPublic if this configuration is public to the user. If it's `false`, this + * configuration is only used internally and we should not expose it to users. + * @tparam T the value type + */ +private[spark] abstract class ConfigEntry[T] ( + val key: String, + val alternatives: List[String], + val valueConverter: String => T, + val stringConverter: T => String, + val doc: String, + val isPublic: Boolean) { + + import ConfigEntry._ + + registerEntry(this) + + def defaultValueString: String + + protected def readString(reader: ConfigReader): Option[String] = { + alternatives.foldLeft(reader.get(key))((res, nextKey) => res.orElse(reader.get(nextKey))) + } + + def readFrom(reader: ConfigReader): T + + def defaultValue: Option[T] = None + + override def toString: String = { + s"ConfigEntry(key=$key, defaultValue=$defaultValueString, doc=$doc, public=$isPublic)" + } +} + +private class ConfigEntryWithDefault[T] ( + key: String, + alternatives: List[String], + _defaultValue: T, + valueConverter: String => T, + stringConverter: T => String, + doc: String, + isPublic: Boolean) + extends ConfigEntry(key, alternatives, valueConverter, stringConverter, doc, isPublic) { + + override def defaultValue: Option[T] = Some(_defaultValue) + + override def defaultValueString: String = stringConverter(_defaultValue) + + def readFrom(reader: ConfigReader): T = { + readString(reader).map(valueConverter).getOrElse(_defaultValue) + } +} + +private class ConfigEntryWithDefaultFunction[T] ( + key: String, + alternatives: List[String], + _defaultFunction: () => T, + valueConverter: String => T, + stringConverter: T => String, + doc: String, + isPublic: Boolean) + extends ConfigEntry(key, alternatives, valueConverter, stringConverter, doc, isPublic) { + + override def defaultValue: Option[T] = Some(_defaultFunction()) + + override def defaultValueString: String = stringConverter(_defaultFunction()) + + def readFrom(reader: ConfigReader): T = { + readString(reader).map(valueConverter).getOrElse(_defaultFunction()) + } +} + +private class ConfigEntryWithDefaultString[T] ( + key: String, + alternatives: List[String], + _defaultValue: String, + valueConverter: String => T, + stringConverter: T => String, + doc: String, + isPublic: Boolean) + extends ConfigEntry(key, alternatives, valueConverter, stringConverter, doc, isPublic) { + + override def defaultValue: Option[T] = Some(valueConverter(_defaultValue)) + + override def defaultValueString: String = _defaultValue + + def readFrom(reader: ConfigReader): T = { + val value = readString(reader).getOrElse(reader.substitute(_defaultValue)) + valueConverter(value) + } +} + + +/** + * A config entry that does not have a default value. + */ +private[spark] class OptionalConfigEntry[T]( + key: String, + alternatives: List[String], + val rawValueConverter: String => T, + val rawStringConverter: T => String, + doc: String, + isPublic: Boolean) + extends ConfigEntry[Option[T]](key, alternatives, + s => Some(rawValueConverter(s)), + v => v.map(rawStringConverter).orNull, doc, isPublic) { + + override def defaultValueString: String = "" + + override def readFrom(reader: ConfigReader): Option[T] = { + readString(reader).map(rawValueConverter) + } +} + +/** + * A config entry whose default value is defined by another config entry. + */ +private class FallbackConfigEntry[T] ( + key: String, + alternatives: List[String], + doc: String, + isPublic: Boolean, + private[config] val fallback: ConfigEntry[T]) + extends ConfigEntry[T](key, alternatives, + fallback.valueConverter, fallback.stringConverter, doc, isPublic) { + + override def defaultValueString: String = s"" + + override def readFrom(reader: ConfigReader): T = { + readString(reader).map(valueConverter).getOrElse(fallback.readFrom(reader)) + } +} + +private[spark] object ConfigEntry { + + private val knownConfigs = new java.util.concurrent.ConcurrentHashMap[String, ConfigEntry[_]]() + + def registerEntry(entry: ConfigEntry[_]): Unit = { + val existing = knownConfigs.putIfAbsent(entry.key, entry) + require(existing == null, s"Config entry ${entry.key} already registered!") + } + + def findEntry(key: String): ConfigEntry[_] = knownConfigs.get(key) + +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigProvider.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigProvider.scala new file mode 100644 index 000000000000..5d98a1185f05 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigProvider.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.config + +import java.util.{Map => JMap} + +/** + * A source of configuration values. + */ +private[spark] trait ConfigProvider { + + def get(key: String): Option[String] + +} + +private[spark] class EnvProvider extends ConfigProvider { + + override def get(key: String): Option[String] = sys.env.get(key) + +} + +private[spark] class SystemProvider extends ConfigProvider { + + override def get(key: String): Option[String] = sys.props.get(key) + +} + +private[spark] class MapProvider(conf: JMap[String, String]) extends ConfigProvider { + + override def get(key: String): Option[String] = Option(conf.get(key)) + +} + +/** + * A config provider that only reads Spark config keys. + */ +private[spark] class SparkConfigProvider(conf: JMap[String, String]) extends ConfigProvider { + + override def get(key: String): Option[String] = { + if (key.startsWith("spark.")) { + Option(conf.get(key)) + } else { + None + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigReader.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigReader.scala new file mode 100644 index 000000000000..c1ab22150d02 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigReader.scala @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.config + +import java.util.{Map => JMap} + +import scala.collection.mutable.HashMap +import scala.util.matching.Regex + +private object ConfigReader { + + private val REF_RE = "\\$\\{(?:(\\w+?):)?(\\S+?)\\}".r + +} + +/** + * A helper class for reading config entries and performing variable substitution. + * + * If a config value contains variable references of the form "${prefix:variableName}", the + * reference will be replaced with the value of the variable depending on the prefix. By default, + * the following prefixes are handled: + * + * - no prefix: use the default config provider + * - system: looks for the value in the system properties + * - env: looks for the value in the environment + * + * Different prefixes can be bound to a `ConfigProvider`, which is used to read configuration + * values from the data source for the prefix, and both the system and env providers can be + * overridden. + * + * If the reference cannot be resolved, the original string will be retained. + * + * @param conf The config provider for the default namespace (no prefix). + */ +private[spark] class ConfigReader(conf: ConfigProvider) { + + def this(conf: JMap[String, String]) = this(new MapProvider(conf)) + + private val bindings = new HashMap[String, ConfigProvider]() + bind(null, conf) + bindEnv(new EnvProvider()) + bindSystem(new SystemProvider()) + + /** + * Binds a prefix to a provider. This method is not thread-safe and should be called + * before the instance is used to expand values. + */ + def bind(prefix: String, provider: ConfigProvider): ConfigReader = { + bindings(prefix) = provider + this + } + + def bind(prefix: String, values: JMap[String, String]): ConfigReader = { + bind(prefix, new MapProvider(values)) + } + + def bindEnv(provider: ConfigProvider): ConfigReader = bind("env", provider) + + def bindSystem(provider: ConfigProvider): ConfigReader = bind("system", provider) + + /** + * Reads a configuration key from the default provider, and apply variable substitution. + */ + def get(key: String): Option[String] = conf.get(key).map(substitute) + + /** + * Perform variable substitution on the given input string. + */ + def substitute(input: String): String = substitute(input, Set()) + + private def substitute(input: String, usedRefs: Set[String]): String = { + if (input != null) { + ConfigReader.REF_RE.replaceAllIn(input, { m => + val prefix = m.group(1) + val name = m.group(2) + val ref = if (prefix == null) name else s"$prefix:$name" + require(!usedRefs.contains(ref), s"Circular reference in $input: $ref") + + val replacement = bindings.get(prefix) + .flatMap(getOrDefault(_, name)) + .map { v => substitute(v, usedRefs + ref) } + .getOrElse(m.matched) + Regex.quoteReplacement(replacement) + }) + } else { + input + } + } + + /** + * Gets the value of a config from the given `ConfigProvider`. If no value is found for this + * config, and the `ConfigEntry` defines this config has default value, return the default value. + */ + private def getOrDefault(conf: ConfigProvider, key: String): Option[String] = { + conf.get(key).orElse { + ConfigEntry.findEntry(key) match { + case e: ConfigEntryWithDefault[_] => Option(e.defaultValueString) + case e: ConfigEntryWithDefaultString[_] => Option(e.defaultValueString) + case e: ConfigEntryWithDefaultFunction[_] => Option(e.defaultValueString) + case e: FallbackConfigEntry[_] => getOrDefault(conf, e.fallback.key) + case _ => None + } + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala new file mode 100644 index 000000000000..44a2815b81a7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -0,0 +1,413 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal + +import java.util.concurrent.TimeUnit + +import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.network.util.ByteUnit +import org.apache.spark.util.Utils + +package object config { + + private[spark] val DRIVER_CLASS_PATH = + ConfigBuilder(SparkLauncher.DRIVER_EXTRA_CLASSPATH).stringConf.createOptional + + private[spark] val DRIVER_JAVA_OPTIONS = + ConfigBuilder(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS).stringConf.createOptional + + private[spark] val DRIVER_LIBRARY_PATH = + ConfigBuilder(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH).stringConf.createOptional + + private[spark] val DRIVER_USER_CLASS_PATH_FIRST = + ConfigBuilder("spark.driver.userClassPathFirst").booleanConf.createWithDefault(false) + + private[spark] val DRIVER_MEMORY = ConfigBuilder("spark.driver.memory") + .bytesConf(ByteUnit.MiB) + .createWithDefaultString("1g") + + private[spark] val EXECUTOR_CLASS_PATH = + ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH).stringConf.createOptional + + private[spark] val EXECUTOR_JAVA_OPTIONS = + ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS).stringConf.createOptional + + private[spark] val EXECUTOR_LIBRARY_PATH = + ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH).stringConf.createOptional + + private[spark] val EXECUTOR_USER_CLASS_PATH_FIRST = + ConfigBuilder("spark.executor.userClassPathFirst").booleanConf.createWithDefault(false) + + private[spark] val EXECUTOR_MEMORY = ConfigBuilder("spark.executor.memory") + .bytesConf(ByteUnit.MiB) + .createWithDefaultString("1g") + + private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal() + .booleanConf.createWithDefault(false) + + private[spark] val CPUS_PER_TASK = ConfigBuilder("spark.task.cpus").intConf.createWithDefault(1) + + private[spark] val DYN_ALLOCATION_MIN_EXECUTORS = + ConfigBuilder("spark.dynamicAllocation.minExecutors").intConf.createWithDefault(0) + + private[spark] val DYN_ALLOCATION_INITIAL_EXECUTORS = + ConfigBuilder("spark.dynamicAllocation.initialExecutors") + .fallbackConf(DYN_ALLOCATION_MIN_EXECUTORS) + + private[spark] val DYN_ALLOCATION_MAX_EXECUTORS = + ConfigBuilder("spark.dynamicAllocation.maxExecutors").intConf.createWithDefault(Int.MaxValue) + + private[spark] val SHUFFLE_SERVICE_ENABLED = + ConfigBuilder("spark.shuffle.service.enabled").booleanConf.createWithDefault(false) + + private[spark] val KEYTAB = ConfigBuilder("spark.yarn.keytab") + .doc("Location of user's keytab.") + .stringConf.createOptional + + private[spark] val PRINCIPAL = ConfigBuilder("spark.yarn.principal") + .doc("Name of the Kerberos principal.") + .stringConf.createOptional + + private[spark] val EXECUTOR_INSTANCES = ConfigBuilder("spark.executor.instances") + .intConf + .createOptional + + private[spark] val PY_FILES = ConfigBuilder("spark.yarn.dist.pyFiles") + .internal() + .stringConf + .toSequence + .createWithDefault(Nil) + + private[spark] val MAX_TASK_FAILURES = + ConfigBuilder("spark.task.maxFailures") + .intConf + .createWithDefault(4) + + // Blacklist confs + private[spark] val BLACKLIST_ENABLED = + ConfigBuilder("spark.blacklist.enabled") + .booleanConf + .createOptional + + private[spark] val MAX_TASK_ATTEMPTS_PER_EXECUTOR = + ConfigBuilder("spark.blacklist.task.maxTaskAttemptsPerExecutor") + .intConf + .createWithDefault(1) + + private[spark] val MAX_TASK_ATTEMPTS_PER_NODE = + ConfigBuilder("spark.blacklist.task.maxTaskAttemptsPerNode") + .intConf + .createWithDefault(2) + + private[spark] val MAX_FAILURES_PER_EXEC = + ConfigBuilder("spark.blacklist.application.maxFailedTasksPerExecutor") + .intConf + .createWithDefault(2) + + private[spark] val MAX_FAILURES_PER_EXEC_STAGE = + ConfigBuilder("spark.blacklist.stage.maxFailedTasksPerExecutor") + .intConf + .createWithDefault(2) + + private[spark] val MAX_FAILED_EXEC_PER_NODE = + ConfigBuilder("spark.blacklist.application.maxFailedExecutorsPerNode") + .intConf + .createWithDefault(2) + + private[spark] val MAX_FAILED_EXEC_PER_NODE_STAGE = + ConfigBuilder("spark.blacklist.stage.maxFailedExecutorsPerNode") + .intConf + .createWithDefault(2) + + private[spark] val BLACKLIST_TIMEOUT_CONF = + ConfigBuilder("spark.blacklist.timeout") + .timeConf(TimeUnit.MILLISECONDS) + .createOptional + + private[spark] val BLACKLIST_KILL_ENABLED = + ConfigBuilder("spark.blacklist.killBlacklistedExecutors") + .booleanConf + .createWithDefault(false) + + private[spark] val BLACKLIST_LEGACY_TIMEOUT_CONF = + ConfigBuilder("spark.scheduler.executorTaskBlacklistTime") + .internal() + .timeConf(TimeUnit.MILLISECONDS) + .createOptional + + private[spark] val BLACKLIST_FETCH_FAILURE_ENABLED = + ConfigBuilder("spark.blacklist.application.fetchFailure.enabled") + .booleanConf + .createWithDefault(false) + // End blacklist confs + + private[spark] val UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE = + ConfigBuilder("spark.files.fetchFailure.unRegisterOutputOnHost") + .doc("Whether to un-register all the outputs on the host in condition that we receive " + + " a FetchFailure. This is set default to false, which means, we only un-register the " + + " outputs related to the exact executor(instead of the host) on a FetchFailure.") + .booleanConf + .createWithDefault(false) + + private[spark] val LISTENER_BUS_EVENT_QUEUE_CAPACITY = + ConfigBuilder("spark.scheduler.listenerbus.eventqueue.capacity") + .withAlternative("spark.scheduler.listenerbus.eventqueue.size") + .intConf + .checkValue(_ > 0, "The capacity of listener bus event queue must not be negative") + .createWithDefault(10000) + + private[spark] val LISTENER_BUS_METRICS_MAX_LISTENER_CLASSES_TIMED = + ConfigBuilder("spark.scheduler.listenerbus.metrics.maxListenerClassesTimed") + .internal() + .intConf + .createWithDefault(128) + + // This property sets the root namespace for metrics reporting + private[spark] val METRICS_NAMESPACE = ConfigBuilder("spark.metrics.namespace") + .stringConf + .createOptional + + private[spark] val PYSPARK_DRIVER_PYTHON = ConfigBuilder("spark.pyspark.driver.python") + .stringConf + .createOptional + + private[spark] val PYSPARK_PYTHON = ConfigBuilder("spark.pyspark.python") + .stringConf + .createOptional + + // To limit memory usage, we only track information for a fixed number of tasks + private[spark] val UI_RETAINED_TASKS = ConfigBuilder("spark.ui.retainedTasks") + .intConf + .createWithDefault(100000) + + // To limit how many applications are shown in the History Server summary ui + private[spark] val HISTORY_UI_MAX_APPS = + ConfigBuilder("spark.history.ui.maxApplications").intConf.createWithDefault(Integer.MAX_VALUE) + + private[spark] val IO_ENCRYPTION_ENABLED = ConfigBuilder("spark.io.encryption.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val IO_ENCRYPTION_KEYGEN_ALGORITHM = + ConfigBuilder("spark.io.encryption.keygen.algorithm") + .stringConf + .createWithDefault("HmacSHA1") + + private[spark] val IO_ENCRYPTION_KEY_SIZE_BITS = ConfigBuilder("spark.io.encryption.keySizeBits") + .intConf + .checkValues(Set(128, 192, 256)) + .createWithDefault(128) + + private[spark] val IO_CRYPTO_CIPHER_TRANSFORMATION = + ConfigBuilder("spark.io.crypto.cipher.transformation") + .internal() + .stringConf + .createWithDefaultString("AES/CTR/NoPadding") + + private[spark] val DRIVER_HOST_ADDRESS = ConfigBuilder("spark.driver.host") + .doc("Address of driver endpoints.") + .stringConf + .createWithDefault(Utils.localCanonicalHostName()) + + private[spark] val DRIVER_BIND_ADDRESS = ConfigBuilder("spark.driver.bindAddress") + .doc("Address where to bind network listen sockets on the driver.") + .fallbackConf(DRIVER_HOST_ADDRESS) + + private[spark] val BLOCK_MANAGER_PORT = ConfigBuilder("spark.blockManager.port") + .doc("Port to use for the block manager when a more specific setting is not provided.") + .intConf + .createWithDefault(0) + + private[spark] val DRIVER_BLOCK_MANAGER_PORT = ConfigBuilder("spark.driver.blockManager.port") + .doc("Port to use for the block manager on the driver.") + .fallbackConf(BLOCK_MANAGER_PORT) + + private[spark] val IGNORE_CORRUPT_FILES = ConfigBuilder("spark.files.ignoreCorruptFiles") + .doc("Whether to ignore corrupt files. If true, the Spark jobs will continue to run when " + + "encountering corrupted or non-existing files and contents that have been read will still " + + "be returned.") + .booleanConf + .createWithDefault(false) + + private[spark] val APP_CALLER_CONTEXT = ConfigBuilder("spark.log.callerContext") + .stringConf + .createOptional + + private[spark] val FILES_MAX_PARTITION_BYTES = ConfigBuilder("spark.files.maxPartitionBytes") + .doc("The maximum number of bytes to pack into a single partition when reading files.") + .longConf + .createWithDefault(128 * 1024 * 1024) + + private[spark] val FILES_OPEN_COST_IN_BYTES = ConfigBuilder("spark.files.openCostInBytes") + .doc("The estimated cost to open a file, measured by the number of bytes could be scanned in" + + " the same time. This is used when putting multiple files into a partition. It's better to" + + " over estimate, then the partitions with small files will be faster than partitions with" + + " bigger files.") + .longConf + .createWithDefault(4 * 1024 * 1024) + + private[spark] val SECRET_REDACTION_PATTERN = + ConfigBuilder("spark.redaction.regex") + .doc("Regex to decide which Spark configuration properties and environment variables in " + + "driver and executor environments contain sensitive information. When this regex matches " + + "a property key or value, the value is redacted from the environment UI and various logs " + + "like YARN and event logs.") + .regexConf + .createWithDefault("(?i)secret|password".r) + + private[spark] val STRING_REDACTION_PATTERN = + ConfigBuilder("spark.redaction.string.regex") + .doc("Regex to decide which parts of strings produced by Spark contain sensitive " + + "information. When this regex matches a string part, that string part is replaced by a " + + "dummy value. This is currently used to redact the output of SQL explain commands.") + .regexConf + .createOptional + + private[spark] val NETWORK_AUTH_ENABLED = + ConfigBuilder("spark.authenticate") + .booleanConf + .createWithDefault(false) + + private[spark] val SASL_ENCRYPTION_ENABLED = + ConfigBuilder("spark.authenticate.enableSaslEncryption") + .booleanConf + .createWithDefault(false) + + private[spark] val NETWORK_ENCRYPTION_ENABLED = + ConfigBuilder("spark.network.crypto.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val BUFFER_WRITE_CHUNK_SIZE = + ConfigBuilder("spark.buffer.write.chunkSize") + .internal() + .doc("The chunk size during writing out the bytes of ChunkedByteBuffer.") + .bytesConf(ByteUnit.BYTE) + .checkValue(_ <= Int.MaxValue, "The chunk size during writing out the bytes of" + + " ChunkedByteBuffer should not larger than Int.MaxValue.") + .createWithDefault(64 * 1024 * 1024) + + private[spark] val CHECKPOINT_COMPRESS = + ConfigBuilder("spark.checkpoint.compress") + .doc("Whether to compress RDD checkpoints. Generally a good idea. Compression will use " + + "spark.io.compression.codec.") + .booleanConf + .createWithDefault(false) + + private[spark] val SHUFFLE_ACCURATE_BLOCK_THRESHOLD = + ConfigBuilder("spark.shuffle.accurateBlockThreshold") + .doc("When we compress the size of shuffle blocks in HighlyCompressedMapStatus, we will " + + "record the size accurately if it's above this config. This helps to prevent OOM by " + + "avoiding underestimating shuffle block size when fetch shuffle blocks.") + .bytesConf(ByteUnit.BYTE) + .createWithDefault(100 * 1024 * 1024) + + private[spark] val SHUFFLE_REGISTRATION_TIMEOUT = + ConfigBuilder("spark.shuffle.registration.timeout") + .doc("Timeout in milliseconds for registration to the external shuffle service.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefault(5000) + + private[spark] val SHUFFLE_REGISTRATION_MAX_ATTEMPTS = + ConfigBuilder("spark.shuffle.registration.maxAttempts") + .doc("When we fail to register to the external shuffle service, we will " + + "retry for maxAttempts times.") + .intConf + .createWithDefault(3) + + private[spark] val REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS = + ConfigBuilder("spark.reducer.maxBlocksInFlightPerAddress") + .doc("This configuration limits the number of remote blocks being fetched per reduce task" + + " from a given host port. When a large number of blocks are being requested from a given" + + " address in a single fetch or simultaneously, this could crash the serving executor or" + + " Node Manager. This is especially useful to reduce the load on the Node Manager when" + + " external shuffle is enabled. You can mitigate the issue by setting it to a lower value.") + .intConf + .checkValue(_ > 0, "The max no. of blocks in flight cannot be non-positive.") + .createWithDefault(Int.MaxValue) + + private[spark] val REDUCER_MAX_REQ_SIZE_SHUFFLE_TO_MEM = + ConfigBuilder("spark.reducer.maxReqSizeShuffleToMem") + .doc("The blocks of a shuffle request will be fetched to disk when size of the request is " + + "above this threshold. This is to avoid a giant request takes too much memory. We can " + + "enable this config by setting a specific value(e.g. 200m). Note that this config can " + + "be enabled only when the shuffle shuffle service is newer than Spark-2.2 or the shuffle" + + " service is disabled.") + .bytesConf(ByteUnit.BYTE) + .createWithDefault(Long.MaxValue) + + private[spark] val TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES = + ConfigBuilder("spark.taskMetrics.trackUpdatedBlockStatuses") + .doc("Enable tracking of updatedBlockStatuses in the TaskMetrics. Off by default since " + + "tracking the block statuses can use a lot of memory and its not used anywhere within " + + "spark.") + .booleanConf + .createWithDefault(false) + + private[spark] val SHUFFLE_FILE_BUFFER_SIZE = + ConfigBuilder("spark.shuffle.file.buffer") + .doc("Size of the in-memory buffer for each shuffle file output stream. " + + "These buffers reduce the number of disk seeks and system calls made " + + "in creating intermediate shuffle files.") + .bytesConf(ByteUnit.KiB) + .checkValue(v => v > 0 && v <= Int.MaxValue / 1024, + s"The file buffer size must be greater than 0 and less than ${Int.MaxValue / 1024}.") + .createWithDefaultString("32k") + + private[spark] val SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE = + ConfigBuilder("spark.shuffle.unsafe.file.output.buffer") + .doc("The file system for this buffer size after each partition " + + "is written in unsafe shuffle writer.") + .bytesConf(ByteUnit.KiB) + .checkValue(v => v > 0 && v <= Int.MaxValue / 1024, + s"The buffer size must be greater than 0 and less than ${Int.MaxValue / 1024}.") + .createWithDefaultString("32k") + + private[spark] val SHUFFLE_DISK_WRITE_BUFFER_SIZE = + ConfigBuilder("spark.shuffle.spill.diskWriteBufferSize") + .doc("The buffer size to use when writing the sorted records to an on-disk file.") + .bytesConf(ByteUnit.BYTE) + .checkValue(v => v > 0 && v <= Int.MaxValue, + s"The buffer size must be greater than 0 and less than ${Int.MaxValue}.") + .createWithDefault(1024 * 1024) + + private[spark] val UNROLL_MEMORY_CHECK_PERIOD = + ConfigBuilder("spark.storage.unrollMemoryCheckPeriod") + .internal() + .doc("The memory check period is used to determine how often we should check whether " + + "there is a need to request more memory when we try to unroll the given block in memory.") + .longConf + .createWithDefault(16) + + private[spark] val UNROLL_MEMORY_GROWTH_FACTOR = + ConfigBuilder("spark.storage.unrollMemoryGrowthFactor") + .internal() + .doc("Memory to request as a multiple of the size that used to unroll the block.") + .doubleConf + .createWithDefault(1.5) + + private[spark] val FORCE_DOWNLOAD_SCHEMES = + ConfigBuilder("spark.yarn.dist.forceDownloadSchemes") + .doc("Comma-separated list of schemes for which files will be downloaded to the " + + "local disk prior to being added to YARN's distributed cache. For use in cases " + + "where the YARN service does not support schemes that are supported by Spark, like http, " + + "https and ftp.") + .stringConf + .toSequence + .createWithDefault(Nil) +} diff --git a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala new file mode 100644 index 000000000000..50f51e1af453 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.io + +import org.apache.hadoop.fs._ +import org.apache.hadoop.mapreduce._ + +import org.apache.spark.util.Utils + + +/** + * An interface to define how a single Spark job commits its outputs. Three notes: + * + * 1. Implementations must be serializable, as the committer instance instantiated on the driver + * will be used for tasks on executors. + * 2. Implementations should have a constructor with 2 arguments: + * (jobId: String, path: String) + * 3. A committer should not be reused across multiple Spark jobs. + * + * The proper call sequence is: + * + * 1. Driver calls setupJob. + * 2. As part of each task's execution, executor calls setupTask and then commitTask + * (or abortTask if task failed). + * 3. When all necessary tasks completed successfully, the driver calls commitJob. If the job + * failed to execute (e.g. too many failed tasks), the job should call abortJob. + */ +abstract class FileCommitProtocol { + import FileCommitProtocol._ + + /** + * Setups up a job. Must be called on the driver before any other methods can be invoked. + */ + def setupJob(jobContext: JobContext): Unit + + /** + * Commits a job after the writes succeed. Must be called on the driver. + */ + def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit + + /** + * Aborts a job after the writes fail. Must be called on the driver. + * + * Calling this function is a best-effort attempt, because it is possible that the driver + * just crashes (or killed) before it can call abort. + */ + def abortJob(jobContext: JobContext): Unit + + /** + * Sets up a task within a job. + * Must be called before any other task related methods can be invoked. + */ + def setupTask(taskContext: TaskAttemptContext): Unit + + /** + * Notifies the commit protocol to add a new file, and gets back the full path that should be + * used. Must be called on the executors when running tasks. + * + * Note that the returned temp file may have an arbitrary path. The commit protocol only + * promises that the file will be at the location specified by the arguments after job commit. + * + * A full file path consists of the following parts: + * 1. the base path + * 2. some sub-directory within the base path, used to specify partitioning + * 3. file prefix, usually some unique job id with the task id + * 4. bucket id + * 5. source specific file extension, e.g. ".snappy.parquet" + * + * The "dir" parameter specifies 2, and "ext" parameter specifies both 4 and 5, and the rest + * are left to the commit protocol implementation to decide. + * + * Important: it is the caller's responsibility to add uniquely identifying content to "ext" + * if a task is going to write out multiple files to the same dir. The file commit protocol only + * guarantees that files written by different tasks will not conflict. + */ + def newTaskTempFile(taskContext: TaskAttemptContext, dir: Option[String], ext: String): String + + /** + * Similar to newTaskTempFile(), but allows files to committed to an absolute output location. + * Depending on the implementation, there may be weaker guarantees around adding files this way. + * + * Important: it is the caller's responsibility to add uniquely identifying content to "ext" + * if a task is going to write out multiple files to the same dir. The file commit protocol only + * guarantees that files written by different tasks will not conflict. + */ + def newTaskTempFileAbsPath( + taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String + + /** + * Commits a task after the writes succeed. Must be called on the executors when running tasks. + */ + def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage + + /** + * Aborts a task after the writes have failed. Must be called on the executors when running tasks. + * + * Calling this function is a best-effort attempt, because it is possible that the executor + * just crashes (or killed) before it can call abort. + */ + def abortTask(taskContext: TaskAttemptContext): Unit + + /** + * Specifies that a file should be deleted with the commit of this job. The default + * implementation deletes the file immediately. + */ + def deleteWithJob(fs: FileSystem, path: Path, recursive: Boolean): Boolean = { + fs.delete(path, recursive) + } + + /** + * Called on the driver after a task commits. This can be used to access task commit messages + * before the job has finished. These same task commit messages will be passed to commitJob() + * if the entire job succeeds. + */ + def onTaskCommit(taskCommit: TaskCommitMessage): Unit = {} +} + + +object FileCommitProtocol { + class TaskCommitMessage(val obj: Any) extends Serializable + + object EmptyTaskCommitMessage extends TaskCommitMessage(null) + + /** + * Instantiates a FileCommitProtocol using the given className. + */ + def instantiate(className: String, jobId: String, outputPath: String) + : FileCommitProtocol = { + val clazz = Utils.classForName(className).asInstanceOf[Class[FileCommitProtocol]] + val ctor = clazz.getDeclaredConstructor(classOf[String], classOf[String]) + ctor.newInstance(jobId, outputPath) + } +} diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapRedCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapRedCommitProtocol.scala new file mode 100644 index 000000000000..ddbd624b380d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapRedCommitProtocol.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.io + +import org.apache.hadoop.mapred._ +import org.apache.hadoop.mapreduce.{TaskAttemptContext => NewTaskAttemptContext} + +/** + * An [[FileCommitProtocol]] implementation backed by an underlying Hadoop OutputCommitter + * (from the old mapred API). + * + * Unlike Hadoop's OutputCommitter, this implementation is serializable. + */ +class HadoopMapRedCommitProtocol(jobId: String, path: String) + extends HadoopMapReduceCommitProtocol(jobId, path) { + + override def setupCommitter(context: NewTaskAttemptContext): OutputCommitter = { + val config = context.getConfiguration.asInstanceOf[JobConf] + config.getOutputCommitter + } +} diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala new file mode 100644 index 000000000000..b1d07ab2c919 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.io + +import java.util.{Date, UUID} + +import scala.collection.mutable + +import org.apache.hadoop.conf.Configurable +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl + +import org.apache.spark.internal.Logging +import org.apache.spark.mapred.SparkHadoopMapRedUtil + +/** + * An [[FileCommitProtocol]] implementation backed by an underlying Hadoop OutputCommitter + * (from the newer mapreduce API, not the old mapred API). + * + * Unlike Hadoop's OutputCommitter, this implementation is serializable. + */ +class HadoopMapReduceCommitProtocol(jobId: String, path: String) + extends FileCommitProtocol with Serializable with Logging { + + import FileCommitProtocol._ + + /** OutputCommitter from Hadoop is not serializable so marking it transient. */ + @transient private var committer: OutputCommitter = _ + + /** + * Tracks files staged by this task for absolute output paths. These outputs are not managed by + * the Hadoop OutputCommitter, so we must move these to their final locations on job commit. + * + * The mapping is from the temp output path to the final desired output path of the file. + */ + @transient private var addedAbsPathFiles: mutable.Map[String, String] = null + + /** + * The staging directory for all files committed with absolute output paths. + */ + private def absPathStagingDir: Path = new Path(path, "_temporary-" + jobId) + + protected def setupCommitter(context: TaskAttemptContext): OutputCommitter = { + val format = context.getOutputFormatClass.newInstance() + // If OutputFormat is Configurable, we should set conf to it. + format match { + case c: Configurable => c.setConf(context.getConfiguration) + case _ => () + } + format.getOutputCommitter(context) + } + + override def newTaskTempFile( + taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = { + val filename = getFilename(taskContext, ext) + + val stagingDir: String = committer match { + // For FileOutputCommitter it has its own staging path called "work path". + case f: FileOutputCommitter => + Option(f.getWorkPath).map(_.toString).getOrElse(path) + case _ => path + } + + dir.map { d => + new Path(new Path(stagingDir, d), filename).toString + }.getOrElse { + new Path(stagingDir, filename).toString + } + } + + override def newTaskTempFileAbsPath( + taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { + val filename = getFilename(taskContext, ext) + val absOutputPath = new Path(absoluteDir, filename).toString + + // Include a UUID here to prevent file collisions for one task writing to different dirs. + // In principle we could include hash(absoluteDir) instead but this is simpler. + val tmpOutputPath = new Path( + absPathStagingDir, UUID.randomUUID().toString() + "-" + filename).toString + + addedAbsPathFiles(tmpOutputPath) = absOutputPath + tmpOutputPath + } + + private def getFilename(taskContext: TaskAttemptContext, ext: String): String = { + // The file name looks like part-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003-c000.parquet + // Note that %05d does not truncate the split number, so if we have more than 100000 tasks, + // the file name is fine and won't overflow. + val split = taskContext.getTaskAttemptID.getTaskID.getId + f"part-$split%05d-$jobId$ext" + } + + override def setupJob(jobContext: JobContext): Unit = { + // Setup IDs + val jobId = SparkHadoopWriterUtils.createJobID(new Date, 0) + val taskId = new TaskID(jobId, TaskType.MAP, 0) + val taskAttemptId = new TaskAttemptID(taskId, 0) + + // Set up the configuration object + jobContext.getConfiguration.set("mapreduce.job.id", jobId.toString) + jobContext.getConfiguration.set("mapreduce.task.id", taskAttemptId.getTaskID.toString) + jobContext.getConfiguration.set("mapreduce.task.attempt.id", taskAttemptId.toString) + jobContext.getConfiguration.setBoolean("mapreduce.task.ismap", true) + jobContext.getConfiguration.setInt("mapreduce.task.partition", 0) + + val taskAttemptContext = new TaskAttemptContextImpl(jobContext.getConfiguration, taskAttemptId) + committer = setupCommitter(taskAttemptContext) + committer.setupJob(jobContext) + } + + override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = { + committer.commitJob(jobContext) + val filesToMove = taskCommits.map(_.obj.asInstanceOf[Map[String, String]]) + .foldLeft(Map[String, String]())(_ ++ _) + logDebug(s"Committing files staged for absolute locations $filesToMove") + val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration) + for ((src, dst) <- filesToMove) { + fs.rename(new Path(src), new Path(dst)) + } + fs.delete(absPathStagingDir, true) + } + + override def abortJob(jobContext: JobContext): Unit = { + committer.abortJob(jobContext, JobStatus.State.FAILED) + val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration) + fs.delete(absPathStagingDir, true) + } + + override def setupTask(taskContext: TaskAttemptContext): Unit = { + committer = setupCommitter(taskContext) + committer.setupTask(taskContext) + addedAbsPathFiles = mutable.Map[String, String]() + } + + override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = { + val attemptId = taskContext.getTaskAttemptID + SparkHadoopMapRedUtil.commitTask( + committer, taskContext, attemptId.getJobID.getId, attemptId.getTaskID.getId) + new TaskCommitMessage(addedAbsPathFiles.toMap) + } + + override def abortTask(taskContext: TaskAttemptContext): Unit = { + committer.abortTask(taskContext) + // best effort cleanup of other staged files + for ((src, _) <- addedAbsPathFiles) { + val tmp = new Path(src) + tmp.getFileSystem(taskContext.getConfiguration).delete(tmp, false) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopWriteConfigUtil.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopWriteConfigUtil.scala new file mode 100644 index 000000000000..9b987e0e1bb6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopWriteConfigUtil.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.io + +import scala.reflect.ClassTag + +import org.apache.hadoop.mapreduce._ + +import org.apache.spark.SparkConf + +/** + * Interface for create output format/committer/writer used during saving an RDD using a Hadoop + * OutputFormat (both from the old mapred API and the new mapreduce API) + * + * Notes: + * 1. Implementations should throw [[IllegalArgumentException]] when wrong hadoop API is + * referenced; + * 2. Implementations must be serializable, as the instance instantiated on the driver + * will be used for tasks on executors; + * 3. Implementations should have a constructor with exactly one argument: + * (conf: SerializableConfiguration) or (conf: SerializableJobConf). + */ +abstract class HadoopWriteConfigUtil[K, V: ClassTag] extends Serializable { + + // -------------------------------------------------------------------------- + // Create JobContext/TaskAttemptContext + // -------------------------------------------------------------------------- + + def createJobContext(jobTrackerId: String, jobId: Int): JobContext + + def createTaskAttemptContext( + jobTrackerId: String, + jobId: Int, + splitId: Int, + taskAttemptId: Int): TaskAttemptContext + + // -------------------------------------------------------------------------- + // Create committer + // -------------------------------------------------------------------------- + + def createCommitter(jobId: Int): HadoopMapReduceCommitProtocol + + // -------------------------------------------------------------------------- + // Create writer + // -------------------------------------------------------------------------- + + def initWriter(taskContext: TaskAttemptContext, splitId: Int): Unit + + def write(pair: (K, V)): Unit + + def closeWriter(taskContext: TaskAttemptContext): Unit + + // -------------------------------------------------------------------------- + // Create OutputFormat + // -------------------------------------------------------------------------- + + def initOutputFormat(jobContext: JobContext): Unit + + // -------------------------------------------------------------------------- + // Verify hadoop config + // -------------------------------------------------------------------------- + + def assertConf(jobContext: JobContext, conf: SparkConf): Unit +} diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala new file mode 100644 index 000000000000..949d8c677998 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala @@ -0,0 +1,390 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.io + +import java.text.NumberFormat +import java.util.{Date, Locale} + +import scala.reflect.ClassTag + +import org.apache.hadoop.conf.{Configurable, Configuration} +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.mapred._ +import org.apache.hadoop.mapreduce.{JobContext => NewJobContext, +OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, +TaskAttemptContext => NewTaskAttemptContext, TaskAttemptID => NewTaskAttemptID, TaskType} +import org.apache.hadoop.mapreduce.task.{TaskAttemptContextImpl => NewTaskAttemptContextImpl} + +import org.apache.spark.{SerializableWritable, SparkConf, SparkException, TaskContext} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage +import org.apache.spark.rdd.{HadoopRDD, RDD} +import org.apache.spark.util.{SerializableConfiguration, SerializableJobConf, Utils} + +/** + * A helper object that saves an RDD using a Hadoop OutputFormat. + */ +private[spark] +object SparkHadoopWriter extends Logging { + import SparkHadoopWriterUtils._ + + /** + * Basic work flow of this command is: + * 1. Driver side setup, prepare the data source and hadoop configuration for the write job to + * be issued. + * 2. Issues a write job consists of one or more executor side tasks, each of which writes all + * rows within an RDD partition. + * 3. If no exception is thrown in a task, commits that task, otherwise aborts that task; If any + * exception is thrown during task commitment, also aborts that task. + * 4. If all tasks are committed, commit the job, otherwise aborts the job; If any exception is + * thrown during job commitment, also aborts the job. + */ + def write[K, V: ClassTag]( + rdd: RDD[(K, V)], + config: HadoopWriteConfigUtil[K, V]): Unit = { + // Extract context and configuration from RDD. + val sparkContext = rdd.context + val stageId = rdd.id + + // Set up a job. + val jobTrackerId = createJobTrackerID(new Date()) + val jobContext = config.createJobContext(jobTrackerId, stageId) + config.initOutputFormat(jobContext) + + // Assert the output format/key/value class is set in JobConf. + config.assertConf(jobContext, rdd.conf) + + val committer = config.createCommitter(stageId) + committer.setupJob(jobContext) + + // Try to write all RDD partitions as a Hadoop OutputFormat. + try { + val ret = sparkContext.runJob(rdd, (context: TaskContext, iter: Iterator[(K, V)]) => { + executeTask( + context = context, + config = config, + jobTrackerId = jobTrackerId, + sparkStageId = context.stageId, + sparkPartitionId = context.partitionId, + sparkAttemptNumber = context.attemptNumber, + committer = committer, + iterator = iter) + }) + + committer.commitJob(jobContext, ret) + logInfo(s"Job ${jobContext.getJobID} committed.") + } catch { + case cause: Throwable => + logError(s"Aborting job ${jobContext.getJobID}.", cause) + committer.abortJob(jobContext) + throw new SparkException("Job aborted.", cause) + } + } + + /** Write a RDD partition out in a single Spark task. */ + private def executeTask[K, V: ClassTag]( + context: TaskContext, + config: HadoopWriteConfigUtil[K, V], + jobTrackerId: String, + sparkStageId: Int, + sparkPartitionId: Int, + sparkAttemptNumber: Int, + committer: FileCommitProtocol, + iterator: Iterator[(K, V)]): TaskCommitMessage = { + // Set up a task. + val taskContext = config.createTaskAttemptContext( + jobTrackerId, sparkStageId, sparkPartitionId, sparkAttemptNumber) + committer.setupTask(taskContext) + + val (outputMetrics, callback) = initHadoopOutputMetrics(context) + + // Initiate the writer. + config.initWriter(taskContext, sparkPartitionId) + var recordsWritten = 0L + + // Write all rows in RDD partition. + try { + val ret = Utils.tryWithSafeFinallyAndFailureCallbacks { + while (iterator.hasNext) { + val pair = iterator.next() + config.write(pair) + + // Update bytes written metric every few records + maybeUpdateOutputMetrics(outputMetrics, callback, recordsWritten) + recordsWritten += 1 + } + + config.closeWriter(taskContext) + committer.commitTask(taskContext) + }(catchBlock = { + // If there is an error, release resource and then abort the task. + try { + config.closeWriter(taskContext) + } finally { + committer.abortTask(taskContext) + logError(s"Task ${taskContext.getTaskAttemptID} aborted.") + } + }) + + outputMetrics.setBytesWritten(callback()) + outputMetrics.setRecordsWritten(recordsWritten) + + ret + } catch { + case t: Throwable => + throw new SparkException("Task failed while writing rows", t) + } + } +} + +/** + * A helper class that reads JobConf from older mapred API, creates output Format/Committer/Writer. + */ +private[spark] +class HadoopMapRedWriteConfigUtil[K, V: ClassTag](conf: SerializableJobConf) + extends HadoopWriteConfigUtil[K, V] with Logging { + + private var outputFormat: Class[_ <: OutputFormat[K, V]] = null + private var writer: RecordWriter[K, V] = null + + private def getConf: JobConf = conf.value + + // -------------------------------------------------------------------------- + // Create JobContext/TaskAttemptContext + // -------------------------------------------------------------------------- + + override def createJobContext(jobTrackerId: String, jobId: Int): NewJobContext = { + val jobAttemptId = new SerializableWritable(new JobID(jobTrackerId, jobId)) + new JobContextImpl(getConf, jobAttemptId.value) + } + + override def createTaskAttemptContext( + jobTrackerId: String, + jobId: Int, + splitId: Int, + taskAttemptId: Int): NewTaskAttemptContext = { + // Update JobConf. + HadoopRDD.addLocalConfiguration(jobTrackerId, jobId, splitId, taskAttemptId, conf.value) + // Create taskContext. + val attemptId = new TaskAttemptID(jobTrackerId, jobId, TaskType.MAP, splitId, taskAttemptId) + new TaskAttemptContextImpl(getConf, attemptId) + } + + // -------------------------------------------------------------------------- + // Create committer + // -------------------------------------------------------------------------- + + override def createCommitter(jobId: Int): HadoopMapReduceCommitProtocol = { + // Update JobConf. + HadoopRDD.addLocalConfiguration("", 0, 0, 0, getConf) + // Create commit protocol. + FileCommitProtocol.instantiate( + className = classOf[HadoopMapRedCommitProtocol].getName, + jobId = jobId.toString, + outputPath = getConf.get("mapred.output.dir") + ).asInstanceOf[HadoopMapReduceCommitProtocol] + } + + // -------------------------------------------------------------------------- + // Create writer + // -------------------------------------------------------------------------- + + override def initWriter(taskContext: NewTaskAttemptContext, splitId: Int): Unit = { + val numfmt = NumberFormat.getInstance(Locale.US) + numfmt.setMinimumIntegerDigits(5) + numfmt.setGroupingUsed(false) + + val outputName = "part-" + numfmt.format(splitId) + val path = FileOutputFormat.getOutputPath(getConf) + val fs: FileSystem = { + if (path != null) { + path.getFileSystem(getConf) + } else { + FileSystem.get(getConf) + } + } + + writer = getConf.getOutputFormat + .getRecordWriter(fs, getConf, outputName, Reporter.NULL) + .asInstanceOf[RecordWriter[K, V]] + + require(writer != null, "Unable to obtain RecordWriter") + } + + override def write(pair: (K, V)): Unit = { + require(writer != null, "Must call createWriter before write.") + writer.write(pair._1, pair._2) + } + + override def closeWriter(taskContext: NewTaskAttemptContext): Unit = { + if (writer != null) { + writer.close(Reporter.NULL) + } + } + + // -------------------------------------------------------------------------- + // Create OutputFormat + // -------------------------------------------------------------------------- + + override def initOutputFormat(jobContext: NewJobContext): Unit = { + if (outputFormat == null) { + outputFormat = getConf.getOutputFormat.getClass + .asInstanceOf[Class[_ <: OutputFormat[K, V]]] + } + } + + private def getOutputFormat(): OutputFormat[K, V] = { + require(outputFormat != null, "Must call initOutputFormat first.") + + outputFormat.newInstance() + } + + // -------------------------------------------------------------------------- + // Verify hadoop config + // -------------------------------------------------------------------------- + + override def assertConf(jobContext: NewJobContext, conf: SparkConf): Unit = { + val outputFormatInstance = getOutputFormat() + val keyClass = getConf.getOutputKeyClass + val valueClass = getConf.getOutputValueClass + if (outputFormatInstance == null) { + throw new SparkException("Output format class not set") + } + if (keyClass == null) { + throw new SparkException("Output key class not set") + } + if (valueClass == null) { + throw new SparkException("Output value class not set") + } + SparkHadoopUtil.get.addCredentials(getConf) + + logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + + valueClass.getSimpleName + ")") + + if (SparkHadoopWriterUtils.isOutputSpecValidationEnabled(conf)) { + // FileOutputFormat ignores the filesystem parameter + val ignoredFs = FileSystem.get(getConf) + getOutputFormat().checkOutputSpecs(ignoredFs, getConf) + } + } +} + +/** + * A helper class that reads Configuration from newer mapreduce API, creates output + * Format/Committer/Writer. + */ +private[spark] +class HadoopMapReduceWriteConfigUtil[K, V: ClassTag](conf: SerializableConfiguration) + extends HadoopWriteConfigUtil[K, V] with Logging { + + private var outputFormat: Class[_ <: NewOutputFormat[K, V]] = null + private var writer: NewRecordWriter[K, V] = null + + private def getConf: Configuration = conf.value + + // -------------------------------------------------------------------------- + // Create JobContext/TaskAttemptContext + // -------------------------------------------------------------------------- + + override def createJobContext(jobTrackerId: String, jobId: Int): NewJobContext = { + val jobAttemptId = new NewTaskAttemptID(jobTrackerId, jobId, TaskType.MAP, 0, 0) + new NewTaskAttemptContextImpl(getConf, jobAttemptId) + } + + override def createTaskAttemptContext( + jobTrackerId: String, + jobId: Int, + splitId: Int, + taskAttemptId: Int): NewTaskAttemptContext = { + val attemptId = new NewTaskAttemptID( + jobTrackerId, jobId, TaskType.REDUCE, splitId, taskAttemptId) + new NewTaskAttemptContextImpl(getConf, attemptId) + } + + // -------------------------------------------------------------------------- + // Create committer + // -------------------------------------------------------------------------- + + override def createCommitter(jobId: Int): HadoopMapReduceCommitProtocol = { + FileCommitProtocol.instantiate( + className = classOf[HadoopMapReduceCommitProtocol].getName, + jobId = jobId.toString, + outputPath = getConf.get("mapreduce.output.fileoutputformat.outputdir") + ).asInstanceOf[HadoopMapReduceCommitProtocol] + } + + // -------------------------------------------------------------------------- + // Create writer + // -------------------------------------------------------------------------- + + override def initWriter(taskContext: NewTaskAttemptContext, splitId: Int): Unit = { + val taskFormat = getOutputFormat() + // If OutputFormat is Configurable, we should set conf to it. + taskFormat match { + case c: Configurable => c.setConf(getConf) + case _ => () + } + + writer = taskFormat.getRecordWriter(taskContext) + .asInstanceOf[NewRecordWriter[K, V]] + + require(writer != null, "Unable to obtain RecordWriter") + } + + override def write(pair: (K, V)): Unit = { + require(writer != null, "Must call createWriter before write.") + writer.write(pair._1, pair._2) + } + + override def closeWriter(taskContext: NewTaskAttemptContext): Unit = { + if (writer != null) { + writer.close(taskContext) + writer = null + } else { + logWarning("Writer has been closed.") + } + } + + // -------------------------------------------------------------------------- + // Create OutputFormat + // -------------------------------------------------------------------------- + + override def initOutputFormat(jobContext: NewJobContext): Unit = { + if (outputFormat == null) { + outputFormat = jobContext.getOutputFormatClass + .asInstanceOf[Class[_ <: NewOutputFormat[K, V]]] + } + } + + private def getOutputFormat(): NewOutputFormat[K, V] = { + require(outputFormat != null, "Must call initOutputFormat first.") + + outputFormat.newInstance() + } + + // -------------------------------------------------------------------------- + // Verify hadoop config + // -------------------------------------------------------------------------- + + override def assertConf(jobContext: NewJobContext, conf: SparkConf): Unit = { + if (SparkHadoopWriterUtils.isOutputSpecValidationEnabled(conf)) { + getOutputFormat().checkOutputSpecs(jobContext) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriterUtils.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriterUtils.scala new file mode 100644 index 000000000000..de828a6d6156 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriterUtils.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.io + +import java.text.SimpleDateFormat +import java.util.{Date, Locale} + +import scala.util.DynamicVariable + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapred.{JobConf, JobID} + +import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.OutputMetrics + +/** + * A helper object that provide common utils used during saving an RDD using a Hadoop OutputFormat + * (both from the old mapred API and the new mapreduce API) + */ +private[spark] +object SparkHadoopWriterUtils { + + private val RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES = 256 + + def createJobID(time: Date, id: Int): JobID = { + val jobtrackerID = createJobTrackerID(time) + new JobID(jobtrackerID, id) + } + + def createJobTrackerID(time: Date): String = { + new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(time) + } + + def createPathFromString(path: String, conf: JobConf): Path = { + if (path == null) { + throw new IllegalArgumentException("Output path is null") + } + val outputPath = new Path(path) + val fs = outputPath.getFileSystem(conf) + if (fs == null) { + throw new IllegalArgumentException("Incorrectly formatted output path") + } + outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + } + + // Note: this needs to be a function instead of a 'val' so that the disableOutputSpecValidation + // setting can take effect: + def isOutputSpecValidationEnabled(conf: SparkConf): Boolean = { + val validationDisabled = disableOutputSpecValidation.value + val enabledInConf = conf.getBoolean("spark.hadoop.validateOutputSpecs", true) + enabledInConf && !validationDisabled + } + + // TODO: these don't seem like the right abstractions. + // We should abstract the duplicate code in a less awkward way. + + def initHadoopOutputMetrics(context: TaskContext): (OutputMetrics, () => Long) = { + val bytesWrittenCallback = SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback() + (context.taskMetrics().outputMetrics, bytesWrittenCallback) + } + + def maybeUpdateOutputMetrics( + outputMetrics: OutputMetrics, + callback: () => Long, + recordsWritten: Long): Unit = { + if (recordsWritten % RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES == 0) { + outputMetrics.setBytesWritten(callback()) + outputMetrics.setRecordsWritten(recordsWritten) + } + } + + /** + * Allows for the `spark.hadoop.validateOutputSpecs` checks to be disabled on a case-by-case + * basis; see SPARK-4835 for more details. + */ + val disableOutputSpecValidation: DynamicVariable[Boolean] = new DynamicVariable[Boolean](false) +} diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index ca74eedf89be..27f2e429395d 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -17,7 +17,8 @@ package org.apache.spark.io -import java.io.{IOException, InputStream, OutputStream} +import java.io._ +import java.util.Locale import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import net.jpountz.lz4.{LZ4BlockInputStream, LZ4BlockOutputStream} @@ -32,9 +33,8 @@ import org.apache.spark.util.Utils * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. * - * Note: The wire protocol for a codec is not guaranteed compatible across versions of Spark. - * This is intended for use as an internal compression utility within a single - * Spark application. + * @note The wire protocol for a codec is not guaranteed compatible across versions of Spark. + * This is intended for use as an internal compression utility within a single Spark application. */ @DeveloperApi trait CompressionCodec { @@ -49,7 +49,8 @@ private[spark] object CompressionCodec { private val configKey = "spark.io.compression.codec" private[spark] def supportsConcatenationOfSerializedStreams(codec: CompressionCodec): Boolean = { - codec.isInstanceOf[SnappyCompressionCodec] || codec.isInstanceOf[LZFCompressionCodec] + (codec.isInstanceOf[SnappyCompressionCodec] || codec.isInstanceOf[LZFCompressionCodec] + || codec.isInstanceOf[LZ4CompressionCodec]) } private val shortCompressionCodecNames = Map( @@ -66,13 +67,13 @@ private[spark] object CompressionCodec { } def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { - val codecClass = shortCompressionCodecNames.getOrElse(codecName.toLowerCase, codecName) + val codecClass = + shortCompressionCodecNames.getOrElse(codecName.toLowerCase(Locale.ROOT), codecName) val codec = try { val ctor = Utils.classForName(codecClass).getConstructor(classOf[SparkConf]) Some(ctor.newInstance(conf).asInstanceOf[CompressionCodec]) } catch { - case e: ClassNotFoundException => None - case e: IllegalArgumentException => None + case _: ClassNotFoundException | _: IllegalArgumentException => None } codec.getOrElse(throw new IllegalArgumentException(s"Codec [$codecName] is not available. " + s"Consider setting $configKey=$FALLBACK_COMPRESSION_CODEC")) @@ -92,20 +93,19 @@ private[spark] object CompressionCodec { } } - val FALLBACK_COMPRESSION_CODEC = "lzf" - val DEFAULT_COMPRESSION_CODEC = "snappy" + val FALLBACK_COMPRESSION_CODEC = "snappy" + val DEFAULT_COMPRESSION_CODEC = "lz4" val ALL_COMPRESSION_CODECS = shortCompressionCodecNames.values.toSeq } - /** * :: DeveloperApi :: * LZ4 implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by `spark.io.compression.lz4.blockSize`. * - * Note: The wire protocol for this codec is not guaranteed to be compatible across versions - * of Spark. This is intended for use as an internal compression utility within a single Spark - * application. + * @note The wire protocol for this codec is not guaranteed to be compatible across versions + * of Spark. This is intended for use as an internal compression utility within a single Spark + * application. */ @DeveloperApi class LZ4CompressionCodec(conf: SparkConf) extends CompressionCodec { @@ -115,7 +115,10 @@ class LZ4CompressionCodec(conf: SparkConf) extends CompressionCodec { new LZ4BlockOutputStream(s, blockSize) } - override def compressedInputStream(s: InputStream): InputStream = new LZ4BlockInputStream(s) + override def compressedInputStream(s: InputStream): InputStream = { + val disableConcatenationOfByteStream = false + new LZ4BlockInputStream(s, disableConcatenationOfByteStream) + } } @@ -123,9 +126,9 @@ class LZ4CompressionCodec(conf: SparkConf) extends CompressionCodec { * :: DeveloperApi :: * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. * - * Note: The wire protocol for this codec is not guaranteed to be compatible across versions - * of Spark. This is intended for use as an internal compression utility within a single Spark - * application. + * @note The wire protocol for this codec is not guaranteed to be compatible across versions + * of Spark. This is intended for use as an internal compression utility within a single Spark + * application. */ @DeveloperApi class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { @@ -143,18 +146,13 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by `spark.io.compression.snappy.blockSize`. * - * Note: The wire protocol for this codec is not guaranteed to be compatible across versions - * of Spark. This is intended for use as an internal compression utility within a single Spark - * application. + * @note The wire protocol for this codec is not guaranteed to be compatible across versions + * of Spark. This is intended for use as an internal compression utility within a single Spark + * application. */ @DeveloperApi class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { - - try { - Snappy.getNativeLibraryVersion - } catch { - case e: Error => throw new IllegalArgumentException(e) - } + val version = SnappyCompressionCodec.version override def compressedOutputStream(s: OutputStream): OutputStream = { val blockSize = conf.getSizeAsBytes("spark.io.compression.snappy.blockSize", "32k").toInt @@ -165,7 +163,20 @@ class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { } /** - * Wrapper over [[SnappyOutputStream]] which guards against write-after-close and double-close + * Object guards against memory leak bug in snappy-java library: + * (https://github.com/xerial/snappy-java/issues/131). + * Before a new version of the library, we only call the method once and cache the result. + */ +private final object SnappyCompressionCodec { + private lazy val version: String = try { + Snappy.getNativeLibraryVersion + } catch { + case e: Error => throw new IllegalArgumentException(e) + } +} + +/** + * Wrapper over `SnappyOutputStream` which guards against write-after-close and double-close * issues. See SPARK-7660 for more details. This wrapping can be removed if we upgrade to a version * of snappy-java that contains the fix for https://github.com/xerial/snappy-java/issues/107. */ diff --git a/core/src/main/scala/org/apache/spark/io/package-info.java b/core/src/main/scala/org/apache/spark/io/package-info.java index bea1bfdb6375..1a466602806e 100644 --- a/core/src/main/scala/org/apache/spark/io/package-info.java +++ b/core/src/main/scala/org/apache/spark/io/package-info.java @@ -18,4 +18,4 @@ /** * IO codecs used for compression. */ -package org.apache.spark.io; \ No newline at end of file +package org.apache.spark.io; diff --git a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala index 3ea984c501e0..a5d41a1eeb47 100644 --- a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala +++ b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala @@ -21,7 +21,7 @@ import java.net.{InetAddress, Socket} import org.apache.spark.SPARK_VERSION import org.apache.spark.launcher.LauncherProtocol._ -import org.apache.spark.util.ThreadUtils +import org.apache.spark.util.{ThreadUtils, Utils} /** * A class that can be used to talk to a launcher server. Users should extend this class to @@ -88,12 +88,20 @@ private[spark] abstract class LauncherBackend { */ protected def onDisconnected() : Unit = { } + private def fireStopRequest(): Unit = { + val thread = LauncherBackend.threadFactory.newThread(new Runnable() { + override def run(): Unit = Utils.tryLogNonFatalError { + onStopRequest() + } + }) + thread.start() + } private class BackendConnection(s: Socket) extends LauncherConnection(s) { override protected def handle(m: Message): Unit = m match { case _: Stop => - onStopRequest() + fireStopRequest() case _ => throw new IllegalArgumentException(s"Unexpected message type: ${m.getClass().getName()}") diff --git a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala index a2add6161728..4216b2627309 100644 --- a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala +++ b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala @@ -37,11 +37,8 @@ private[spark] class WorkerCommandBuilder(sparkHome: String, memoryMb: Int, comm override def buildCommand(env: JMap[String, String]): JList[String] = { val cmd = buildJavaCommand(command.classPathEntries.mkString(File.pathSeparator)) - cmd.add(s"-Xms${memoryMb}M") cmd.add(s"-Xmx${memoryMb}M") command.javaOpts.foreach(cmd.add) - CommandBuilderUtils.addPermGenSizeOpt(cmd) - addOptionString(cmd, getenv("SPARK_JAVA_OPTS")) cmd } diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala index f7298e8d5c62..607283a306b8 100644 --- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala @@ -18,61 +18,13 @@ package org.apache.spark.mapred import java.io.IOException -import java.lang.reflect.Modifier -import org.apache.hadoop.mapred._ import org.apache.hadoop.mapreduce.{TaskAttemptContext => MapReduceTaskAttemptContext} import org.apache.hadoop.mapreduce.{OutputCommitter => MapReduceOutputCommitter} -import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.executor.CommitDeniedException -import org.apache.spark.{Logging, SparkEnv, TaskContext} -import org.apache.spark.util.{Utils => SparkUtils} - -private[spark] -trait SparkHadoopMapRedUtil { - def newJobContext(conf: JobConf, jobId: JobID): JobContext = { - val klass = firstAvailableClass("org.apache.hadoop.mapred.JobContextImpl", - "org.apache.hadoop.mapred.JobContext") - val ctor = klass.getDeclaredConstructor(classOf[JobConf], - classOf[org.apache.hadoop.mapreduce.JobID]) - // In Hadoop 1.0.x, JobContext is an interface, and JobContextImpl is package private. - // Make it accessible if it's not in order to access it. - if (!Modifier.isPublic(ctor.getModifiers)) { - ctor.setAccessible(true) - } - ctor.newInstance(conf, jobId).asInstanceOf[JobContext] - } - - def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = { - val klass = firstAvailableClass("org.apache.hadoop.mapred.TaskAttemptContextImpl", - "org.apache.hadoop.mapred.TaskAttemptContext") - val ctor = klass.getDeclaredConstructor(classOf[JobConf], classOf[TaskAttemptID]) - // See above - if (!Modifier.isPublic(ctor.getModifiers)) { - ctor.setAccessible(true) - } - ctor.newInstance(conf, attemptId).asInstanceOf[TaskAttemptContext] - } - - def newTaskAttemptID( - jtIdentifier: String, - jobId: Int, - isMap: Boolean, - taskId: Int, - attemptId: Int): TaskAttemptID = { - new TaskAttemptID(jtIdentifier, jobId, isMap, taskId, attemptId) - } - - private def firstAvailableClass(first: String, second: String): Class[_] = { - try { - SparkUtils.classForName(first) - } catch { - case e: ClassNotFoundException => - SparkUtils.classForName(second) - } - } -} +import org.apache.spark.internal.Logging object SparkHadoopMapRedUtil extends Logging { /** @@ -81,11 +33,8 @@ object SparkHadoopMapRedUtil extends Logging { * the driver in order to determine whether this attempt can commit (please see SPARK-4879 for * details). * - * Output commit coordinator is only contacted when the following two configurations are both set - * to `true`: - * - * - `spark.speculation` - * - `spark.hadoop.outputCommitCoordination.enabled` + * Output commit coordinator is only used when `spark.hadoop.outputCommitCoordination.enabled` + * is set to true (which is the default). */ def commitTask( committer: MapReduceOutputCommitter, @@ -93,7 +42,7 @@ object SparkHadoopMapRedUtil extends Logging { jobId: Int, splitId: Int): Unit = { - val mrTaskAttemptID = SparkHadoopUtil.get.getTaskAttemptIDFromTaskAttemptContext(mrTaskContext) + val mrTaskAttemptID = mrTaskContext.getTaskAttemptID // Called after we have decided to commit def performCommit(): Unit = { @@ -112,11 +61,10 @@ object SparkHadoopMapRedUtil extends Logging { if (committer.needsTaskCommit(mrTaskContext)) { val shouldCoordinateWithDriver: Boolean = { val sparkConf = SparkEnv.get.conf - // We only need to coordinate with the driver if there are multiple concurrent task - // attempts, which should only occur if speculation is enabled - val speculationEnabled = sparkConf.getBoolean("spark.speculation", defaultValue = false) - // This (undocumented) setting is an escape-hatch in case the commit code introduces bugs - sparkConf.getBoolean("spark.hadoop.outputCommitCoordination.enabled", speculationEnabled) + // We only need to coordinate with the driver if there are concurrent task attempts. + // Note that this could happen even when speculation is not enabled (e.g. see SPARK-8029). + // This (undocumented) setting is an escape-hatch in case the commit code introduces bugs. + sparkConf.getBoolean("spark.hadoop.outputCommitCoordination.enabled", defaultValue = true) } if (shouldCoordinateWithDriver) { diff --git a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala deleted file mode 100644 index 943ebcb7bd0a..000000000000 --- a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala +++ /dev/null @@ -1,81 +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.mapreduce - -import java.lang.{Boolean => JBoolean, Integer => JInteger} - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapreduce.{JobContext, JobID, TaskAttemptContext, TaskAttemptID} -import org.apache.spark.util.Utils - -private[spark] -trait SparkHadoopMapReduceUtil { - def newJobContext(conf: Configuration, jobId: JobID): JobContext = { - val klass = firstAvailableClass( - "org.apache.hadoop.mapreduce.task.JobContextImpl", // hadoop2, hadoop2-yarn - "org.apache.hadoop.mapreduce.JobContext") // hadoop1 - val ctor = klass.getDeclaredConstructor(classOf[Configuration], classOf[JobID]) - ctor.newInstance(conf, jobId).asInstanceOf[JobContext] - } - - def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = { - val klass = firstAvailableClass( - "org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl", // hadoop2, hadoop2-yarn - "org.apache.hadoop.mapreduce.TaskAttemptContext") // hadoop1 - val ctor = klass.getDeclaredConstructor(classOf[Configuration], classOf[TaskAttemptID]) - ctor.newInstance(conf, attemptId).asInstanceOf[TaskAttemptContext] - } - - def newTaskAttemptID( - jtIdentifier: String, - jobId: Int, - isMap: Boolean, - taskId: Int, - attemptId: Int): TaskAttemptID = { - val klass = Utils.classForName("org.apache.hadoop.mapreduce.TaskAttemptID") - try { - // First, attempt to use the old-style constructor that takes a boolean isMap - // (not available in YARN) - val ctor = klass.getDeclaredConstructor(classOf[String], classOf[Int], classOf[Boolean], - classOf[Int], classOf[Int]) - ctor.newInstance(jtIdentifier, new JInteger(jobId), new JBoolean(isMap), new JInteger(taskId), - new JInteger(attemptId)).asInstanceOf[TaskAttemptID] - } catch { - case exc: NoSuchMethodException => { - // If that failed, look for the new constructor that takes a TaskType (not available in 1.x) - val taskTypeClass = Utils.classForName("org.apache.hadoop.mapreduce.TaskType") - .asInstanceOf[Class[Enum[_]]] - val taskType = taskTypeClass.getMethod("valueOf", classOf[String]).invoke( - taskTypeClass, if (isMap) "MAP" else "REDUCE") - val ctor = klass.getDeclaredConstructor(classOf[String], classOf[Int], taskTypeClass, - classOf[Int], classOf[Int]) - ctor.newInstance(jtIdentifier, new JInteger(jobId), taskType, new JInteger(taskId), - new JInteger(attemptId)).asInstanceOf[TaskAttemptID] - } - } - } - - private def firstAvailableClass(first: String, second: String): Class[_] = { - try { - Utils.classForName(first) - } catch { - case e: ClassNotFoundException => - Utils.classForName(second) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala new file mode 100644 index 000000000000..f1915857ea43 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.memory + +import javax.annotation.concurrent.GuardedBy + +import scala.collection.mutable + +import org.apache.spark.internal.Logging + +/** + * Implements policies and bookkeeping for sharing an adjustable-sized pool of memory between tasks. + * + * Tries to ensure that each task gets a reasonable share of memory, instead of some task ramping up + * to a large amount first and then causing others to spill to disk repeatedly. + * + * If there are N tasks, it ensures that each task can acquire at least 1 / 2N of the memory + * before it has to spill, and at most 1 / N. Because N varies dynamically, we keep track of the + * set of active tasks and redo the calculations of 1 / 2N and 1 / N in waiting tasks whenever this + * set changes. This is all done by synchronizing access to mutable state and using wait() and + * notifyAll() to signal changes to callers. Prior to Spark 1.6, this arbitration of memory across + * tasks was performed by the ShuffleMemoryManager. + * + * @param lock a [[MemoryManager]] instance to synchronize on + * @param memoryMode the type of memory tracked by this pool (on- or off-heap) + */ +private[memory] class ExecutionMemoryPool( + lock: Object, + memoryMode: MemoryMode + ) extends MemoryPool(lock) with Logging { + + private[this] val poolName: String = memoryMode match { + case MemoryMode.ON_HEAP => "on-heap execution" + case MemoryMode.OFF_HEAP => "off-heap execution" + } + + /** + * Map from taskAttemptId -> memory consumption in bytes + */ + @GuardedBy("lock") + private val memoryForTask = new mutable.HashMap[Long, Long]() + + override def memoryUsed: Long = lock.synchronized { + memoryForTask.values.sum + } + + /** + * Returns the memory consumption, in bytes, for the given task. + */ + def getMemoryUsageForTask(taskAttemptId: Long): Long = lock.synchronized { + memoryForTask.getOrElse(taskAttemptId, 0L) + } + + /** + * Try to acquire up to `numBytes` of memory for the given task and return the number of bytes + * obtained, or 0 if none can be allocated. + * + * This call may block until there is enough free memory in some situations, to make sure each + * task has a chance to ramp up to at least 1 / 2N of the total memory pool (where N is the # of + * active tasks) before it is forced to spill. This can happen if the number of tasks increase + * but an older task had a lot of memory already. + * + * @param numBytes number of bytes to acquire + * @param taskAttemptId the task attempt acquiring memory + * @param maybeGrowPool a callback that potentially grows the size of this pool. It takes in + * one parameter (Long) that represents the desired amount of memory by + * which this pool should be expanded. + * @param computeMaxPoolSize a callback that returns the maximum allowable size of this pool + * at this given moment. This is not a field because the max pool + * size is variable in certain cases. For instance, in unified + * memory management, the execution pool can be expanded by evicting + * cached blocks, thereby shrinking the storage pool. + * + * @return the number of bytes granted to the task. + */ + private[memory] def acquireMemory( + numBytes: Long, + taskAttemptId: Long, + maybeGrowPool: Long => Unit = (additionalSpaceNeeded: Long) => Unit, + computeMaxPoolSize: () => Long = () => poolSize): Long = lock.synchronized { + assert(numBytes > 0, s"invalid number of bytes requested: $numBytes") + + // TODO: clean up this clunky method signature + + // Add this task to the taskMemory map just so we can keep an accurate count of the number + // of active tasks, to let other tasks ramp down their memory in calls to `acquireMemory` + if (!memoryForTask.contains(taskAttemptId)) { + memoryForTask(taskAttemptId) = 0L + // This will later cause waiting tasks to wake up and check numTasks again + lock.notifyAll() + } + + // Keep looping until we're either sure that we don't want to grant this request (because this + // task would have more than 1 / numActiveTasks of the memory) or we have enough free + // memory to give it (we always let each task get at least 1 / (2 * numActiveTasks)). + // TODO: simplify this to limit each task to its own slot + while (true) { + val numActiveTasks = memoryForTask.keys.size + val curMem = memoryForTask(taskAttemptId) + + // In every iteration of this loop, we should first try to reclaim any borrowed execution + // space from storage. This is necessary because of the potential race condition where new + // storage blocks may steal the free execution memory that this task was waiting for. + maybeGrowPool(numBytes - memoryFree) + + // Maximum size the pool would have after potentially growing the pool. + // This is used to compute the upper bound of how much memory each task can occupy. This + // must take into account potential free memory as well as the amount this pool currently + // occupies. Otherwise, we may run into SPARK-12155 where, in unified memory management, + // we did not take into account space that could have been freed by evicting cached blocks. + val maxPoolSize = computeMaxPoolSize() + val maxMemoryPerTask = maxPoolSize / numActiveTasks + val minMemoryPerTask = poolSize / (2 * numActiveTasks) + + // How much we can grant this task; keep its share within 0 <= X <= 1 / numActiveTasks + val maxToGrant = math.min(numBytes, math.max(0, maxMemoryPerTask - curMem)) + // Only give it as much memory as is free, which might be none if it reached 1 / numTasks + val toGrant = math.min(maxToGrant, memoryFree) + + // We want to let each task get at least 1 / (2 * numActiveTasks) before blocking; + // if we can't give it this much now, wait for other tasks to free up memory + // (this happens if older tasks allocated lots of memory before N grew) + if (toGrant < numBytes && curMem + toGrant < minMemoryPerTask) { + logInfo(s"TID $taskAttemptId waiting for at least 1/2N of $poolName pool to be free") + lock.wait() + } else { + memoryForTask(taskAttemptId) += toGrant + return toGrant + } + } + 0L // Never reached + } + + /** + * Release `numBytes` of memory acquired by the given task. + */ + def releaseMemory(numBytes: Long, taskAttemptId: Long): Unit = lock.synchronized { + val curMem = memoryForTask.getOrElse(taskAttemptId, 0L) + var memoryToFree = if (curMem < numBytes) { + logWarning( + s"Internal error: release called on $numBytes bytes but task only has $curMem bytes " + + s"of memory from the $poolName pool") + curMem + } else { + numBytes + } + if (memoryForTask.contains(taskAttemptId)) { + memoryForTask(taskAttemptId) -= memoryToFree + if (memoryForTask(taskAttemptId) <= 0) { + memoryForTask.remove(taskAttemptId) + } + } + lock.notifyAll() // Notify waiters in acquireMemory() that memory has been freed + } + + /** + * Release all memory for the given task and mark it as inactive (e.g. when a task ends). + * @return the number of bytes freed. + */ + def releaseAllMemoryForTask(taskAttemptId: Long): Long = lock.synchronized { + val numBytesToFree = getMemoryUsageForTask(taskAttemptId) + releaseMemory(numBytesToFree, taskAttemptId) + numBytesToFree + } + +} diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index b0cf2696a397..82442cf56154 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -19,14 +19,11 @@ package org.apache.spark.memory import javax.annotation.concurrent.GuardedBy -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer - -import com.google.common.annotations.VisibleForTesting - -import org.apache.spark.util.Utils -import org.apache.spark.{SparkException, TaskContext, SparkConf, Logging} -import org.apache.spark.storage.{BlockId, BlockStatus, MemoryStore} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.storage.BlockId +import org.apache.spark.storage.memory.MemoryStore +import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.memory.MemoryAllocator @@ -36,65 +33,62 @@ import org.apache.spark.unsafe.memory.MemoryAllocator * In this context, execution memory refers to that used for computation in shuffles, joins, * sorts and aggregations, while storage memory refers to that used for caching and propagating * internal data across the cluster. There exists one MemoryManager per JVM. - * - * The MemoryManager abstract base class itself implements policies for sharing execution memory - * between tasks; it tries to ensure that each task gets a reasonable share of memory, instead of - * some task ramping up to a large amount first and then causing others to spill to disk repeatedly. - * If there are N tasks, it ensures that each task can acquire at least 1 / 2N of the memory - * before it has to spill, and at most 1 / N. Because N varies dynamically, we keep track of the - * set of active tasks and redo the calculations of 1 / 2N and 1 / N in waiting tasks whenever - * this set changes. This is all done by synchronizing access to mutable state and using wait() and - * notifyAll() to signal changes to callers. Prior to Spark 1.6, this arbitration of memory across - * tasks was performed by the ShuffleMemoryManager. */ -private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) extends Logging { +private[spark] abstract class MemoryManager( + conf: SparkConf, + numCores: Int, + onHeapStorageMemory: Long, + onHeapExecutionMemory: Long) extends Logging { // -- Methods related to memory allocation policies and bookkeeping ------------------------------ - // The memory store used to evict cached blocks - private var _memoryStore: MemoryStore = _ - protected def memoryStore: MemoryStore = { - if (_memoryStore == null) { - throw new IllegalArgumentException("memory store not initialized yet") - } - _memoryStore - } + @GuardedBy("this") + protected val onHeapStorageMemoryPool = new StorageMemoryPool(this, MemoryMode.ON_HEAP) + @GuardedBy("this") + protected val offHeapStorageMemoryPool = new StorageMemoryPool(this, MemoryMode.OFF_HEAP) + @GuardedBy("this") + protected val onHeapExecutionMemoryPool = new ExecutionMemoryPool(this, MemoryMode.ON_HEAP) + @GuardedBy("this") + protected val offHeapExecutionMemoryPool = new ExecutionMemoryPool(this, MemoryMode.OFF_HEAP) + + onHeapStorageMemoryPool.incrementPoolSize(onHeapStorageMemory) + onHeapExecutionMemoryPool.incrementPoolSize(onHeapExecutionMemory) - // Amount of execution/storage memory in use, accesses must be synchronized on `this` - @GuardedBy("this") protected var _executionMemoryUsed: Long = 0 - @GuardedBy("this") protected var _storageMemoryUsed: Long = 0 - // Map from taskAttemptId -> memory consumption in bytes - @GuardedBy("this") private val executionMemoryForTask = new mutable.HashMap[Long, Long]() + protected[this] val maxOffHeapMemory = conf.getSizeAsBytes("spark.memory.offHeap.size", 0) + protected[this] val offHeapStorageMemory = + (maxOffHeapMemory * conf.getDouble("spark.memory.storageFraction", 0.5)).toLong + + offHeapExecutionMemoryPool.incrementPoolSize(maxOffHeapMemory - offHeapStorageMemory) + offHeapStorageMemoryPool.incrementPoolSize(offHeapStorageMemory) /** - * Set the [[MemoryStore]] used by this manager to evict cached blocks. - * This must be set after construction due to initialization ordering constraints. + * Total available on heap memory for storage, in bytes. This amount can vary over time, + * depending on the MemoryManager implementation. + * In this model, this is equivalent to the amount of memory not occupied by execution. */ - final def setMemoryStore(store: MemoryStore): Unit = { - _memoryStore = store - } + def maxOnHeapStorageMemory: Long /** - * Total available memory for execution, in bytes. + * Total available off heap memory for storage, in bytes. This amount can vary over time, + * depending on the MemoryManager implementation. */ - def maxExecutionMemory: Long + def maxOffHeapStorageMemory: Long /** - * Total available memory for storage, in bytes. + * Set the [[MemoryStore]] used by this manager to evict cached blocks. + * This must be set after construction due to initialization ordering constraints. */ - def maxStorageMemory: Long - - // TODO: avoid passing evicted blocks around to simplify method signatures (SPARK-10985) + final def setMemoryStore(store: MemoryStore): Unit = synchronized { + onHeapStorageMemoryPool.setMemoryStore(store) + offHeapStorageMemoryPool.setMemoryStore(store) + } /** * Acquire N bytes of memory to cache the given block, evicting existing ones if necessary. - * Blocks evicted in the process, if any, are added to `evictedBlocks`. + * * @return whether all N bytes were successfully granted. */ - def acquireStorageMemory( - blockId: BlockId, - numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean + def acquireStorageMemory(blockId: BlockId, numBytes: Long, memoryMode: MemoryMode): Boolean /** * Acquire N bytes of memory to unroll the given block, evicting existing ones if necessary. @@ -102,197 +96,115 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte * This extra method allows subclasses to differentiate behavior between acquiring storage * memory and acquiring unroll memory. For instance, the memory management model in Spark * 1.5 and before places a limit on the amount of space that can be freed from unrolling. - * Blocks evicted in the process, if any, are added to `evictedBlocks`. * * @return whether all N bytes were successfully granted. */ - def acquireUnrollMemory( - blockId: BlockId, - numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { - acquireStorageMemory(blockId, numBytes, evictedBlocks) - } - - /** - * Acquire N bytes of memory for execution, evicting cached blocks if necessary. - * Blocks evicted in the process, if any, are added to `evictedBlocks`. - * @return number of bytes successfully granted (<= N). - */ - @VisibleForTesting - private[memory] def doAcquireExecutionMemory( - numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Long + def acquireUnrollMemory(blockId: BlockId, numBytes: Long, memoryMode: MemoryMode): Boolean /** - * Try to acquire up to `numBytes` of execution memory for the current task and return the number - * of bytes obtained, or 0 if none can be allocated. + * Try to acquire up to `numBytes` of execution memory for the current task and return the + * number of bytes obtained, or 0 if none can be allocated. * * This call may block until there is enough free memory in some situations, to make sure each * task has a chance to ramp up to at least 1 / 2N of the total memory pool (where N is the # of * active tasks) before it is forced to spill. This can happen if the number of tasks increase * but an older task had a lot of memory already. - * - * Subclasses should override `doAcquireExecutionMemory` in order to customize the policies - * that control global sharing of memory between execution and storage. */ private[memory] - final def acquireExecutionMemory(numBytes: Long, taskAttemptId: Long): Long = synchronized { - assert(numBytes > 0, "invalid number of bytes requested: " + numBytes) - - // Add this task to the taskMemory map just so we can keep an accurate count of the number - // of active tasks, to let other tasks ramp down their memory in calls to tryToAcquire - if (!executionMemoryForTask.contains(taskAttemptId)) { - executionMemoryForTask(taskAttemptId) = 0L - // This will later cause waiting tasks to wake up and check numTasks again - notifyAll() - } - - // Once the cross-task memory allocation policy has decided to grant more memory to a task, - // this method is called in order to actually obtain that execution memory, potentially - // triggering eviction of storage memory: - def acquire(toGrant: Long): Long = synchronized { - val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - val acquired = doAcquireExecutionMemory(toGrant, evictedBlocks) - // Register evicted blocks, if any, with the active task metrics - Option(TaskContext.get()).foreach { tc => - val metrics = tc.taskMetrics() - val lastUpdatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) - metrics.updatedBlocks = Some(lastUpdatedBlocks ++ evictedBlocks.toSeq) - } - executionMemoryForTask(taskAttemptId) += acquired - acquired - } - - // Keep looping until we're either sure that we don't want to grant this request (because this - // task would have more than 1 / numActiveTasks of the memory) or we have enough free - // memory to give it (we always let each task get at least 1 / (2 * numActiveTasks)). - // TODO: simplify this to limit each task to its own slot - while (true) { - val numActiveTasks = executionMemoryForTask.keys.size - val curMem = executionMemoryForTask(taskAttemptId) - val freeMemory = maxExecutionMemory - executionMemoryForTask.values.sum - - // How much we can grant this task; don't let it grow to more than 1 / numActiveTasks; - // don't let it be negative - val maxToGrant = - math.min(numBytes, math.max(0, (maxExecutionMemory / numActiveTasks) - curMem)) - // Only give it as much memory as is free, which might be none if it reached 1 / numTasks - val toGrant = math.min(maxToGrant, freeMemory) - - if (curMem < maxExecutionMemory / (2 * numActiveTasks)) { - // We want to let each task get at least 1 / (2 * numActiveTasks) before blocking; - // if we can't give it this much now, wait for other tasks to free up memory - // (this happens if older tasks allocated lots of memory before N grew) - if ( - freeMemory >= math.min(maxToGrant, maxExecutionMemory / (2 * numActiveTasks) - curMem)) { - return acquire(toGrant) - } else { - logInfo( - s"TID $taskAttemptId waiting for at least 1/2N of execution memory pool to be free") - wait() - } - } else { - return acquire(toGrant) - } - } - 0L // Never reached - } - - @VisibleForTesting - private[memory] def releaseExecutionMemory(numBytes: Long): Unit = synchronized { - if (numBytes > _executionMemoryUsed) { - logWarning(s"Attempted to release $numBytes bytes of execution " + - s"memory when we only have ${_executionMemoryUsed} bytes") - _executionMemoryUsed = 0 - } else { - _executionMemoryUsed -= numBytes - } - } + def acquireExecutionMemory( + numBytes: Long, + taskAttemptId: Long, + memoryMode: MemoryMode): Long /** * Release numBytes of execution memory belonging to the given task. */ private[memory] - final def releaseExecutionMemory(numBytes: Long, taskAttemptId: Long): Unit = synchronized { - val curMem = executionMemoryForTask.getOrElse(taskAttemptId, 0L) - if (curMem < numBytes) { - if (Utils.isTesting) { - throw new SparkException( - s"Internal error: release called on $numBytes bytes but task only has $curMem") - } else { - logWarning(s"Internal error: release called on $numBytes bytes but task only has $curMem") - } - } - if (executionMemoryForTask.contains(taskAttemptId)) { - executionMemoryForTask(taskAttemptId) -= numBytes - if (executionMemoryForTask(taskAttemptId) <= 0) { - executionMemoryForTask.remove(taskAttemptId) - } - releaseExecutionMemory(numBytes) + def releaseExecutionMemory( + numBytes: Long, + taskAttemptId: Long, + memoryMode: MemoryMode): Unit = synchronized { + memoryMode match { + case MemoryMode.ON_HEAP => onHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId) + case MemoryMode.OFF_HEAP => offHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId) } - notifyAll() // Notify waiters in acquireExecutionMemory() that memory has been freed } /** * Release all memory for the given task and mark it as inactive (e.g. when a task ends). + * * @return the number of bytes freed. */ private[memory] def releaseAllExecutionMemoryForTask(taskAttemptId: Long): Long = synchronized { - val numBytesToFree = getExecutionMemoryUsageForTask(taskAttemptId) - releaseExecutionMemory(numBytesToFree, taskAttemptId) - numBytesToFree + onHeapExecutionMemoryPool.releaseAllMemoryForTask(taskAttemptId) + + offHeapExecutionMemoryPool.releaseAllMemoryForTask(taskAttemptId) } /** * Release N bytes of storage memory. */ - def releaseStorageMemory(numBytes: Long): Unit = synchronized { - if (numBytes > _storageMemoryUsed) { - logWarning(s"Attempted to release $numBytes bytes of storage " + - s"memory when we only have ${_storageMemoryUsed} bytes") - _storageMemoryUsed = 0 - } else { - _storageMemoryUsed -= numBytes + def releaseStorageMemory(numBytes: Long, memoryMode: MemoryMode): Unit = synchronized { + memoryMode match { + case MemoryMode.ON_HEAP => onHeapStorageMemoryPool.releaseMemory(numBytes) + case MemoryMode.OFF_HEAP => offHeapStorageMemoryPool.releaseMemory(numBytes) } } /** * Release all storage memory acquired. */ - def releaseAllStorageMemory(): Unit = synchronized { - _storageMemoryUsed = 0 + final def releaseAllStorageMemory(): Unit = synchronized { + onHeapStorageMemoryPool.releaseAllMemory() + offHeapStorageMemoryPool.releaseAllMemory() } /** * Release N bytes of unroll memory. */ - def releaseUnrollMemory(numBytes: Long): Unit = synchronized { - releaseStorageMemory(numBytes) + final def releaseUnrollMemory(numBytes: Long, memoryMode: MemoryMode): Unit = synchronized { + releaseStorageMemory(numBytes, memoryMode) } /** * Execution memory currently in use, in bytes. */ final def executionMemoryUsed: Long = synchronized { - _executionMemoryUsed + onHeapExecutionMemoryPool.memoryUsed + offHeapExecutionMemoryPool.memoryUsed } /** * Storage memory currently in use, in bytes. */ final def storageMemoryUsed: Long = synchronized { - _storageMemoryUsed + onHeapStorageMemoryPool.memoryUsed + offHeapStorageMemoryPool.memoryUsed } /** * Returns the execution memory consumption, in bytes, for the given task. */ private[memory] def getExecutionMemoryUsageForTask(taskAttemptId: Long): Long = synchronized { - executionMemoryForTask.getOrElse(taskAttemptId, 0L) + onHeapExecutionMemoryPool.getMemoryUsageForTask(taskAttemptId) + + offHeapExecutionMemoryPool.getMemoryUsageForTask(taskAttemptId) } // -- Fields related to Tungsten managed memory ------------------------------------------------- + /** + * Tracks whether Tungsten memory will be allocated on the JVM heap or off-heap using + * sun.misc.Unsafe. + */ + final val tungstenMemoryMode: MemoryMode = { + if (conf.getBoolean("spark.memory.offHeap.enabled", false)) { + require(conf.getSizeAsBytes("spark.memory.offHeap.size", 0) > 0, + "spark.memory.offHeap.size must be > 0 when spark.memory.offHeap.enabled == true") + require(Platform.unaligned(), + "No support for unaligned Unsafe. Set spark.memory.offHeap.enabled to false.") + MemoryMode.OFF_HEAP + } else { + MemoryMode.ON_HEAP + } + } + /** * The default page size, in bytes. * @@ -306,21 +218,22 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte val cores = if (numCores > 0) numCores else Runtime.getRuntime.availableProcessors() // Because of rounding to next power of 2, we may have safetyFactor as 8 in worst case val safetyFactor = 16 - val size = ByteArrayMethods.nextPowerOf2(maxExecutionMemory / cores / safetyFactor) + val maxTungstenMemory: Long = tungstenMemoryMode match { + case MemoryMode.ON_HEAP => onHeapExecutionMemoryPool.poolSize + case MemoryMode.OFF_HEAP => offHeapExecutionMemoryPool.poolSize + } + val size = ByteArrayMethods.nextPowerOf2(maxTungstenMemory / cores / safetyFactor) val default = math.min(maxPageSize, math.max(minPageSize, size)) conf.getSizeAsBytes("spark.buffer.pageSize", default) } - /** - * Tracks whether Tungsten memory will be allocated on the JVM heap or off-heap using - * sun.misc.Unsafe. - */ - final val tungstenMemoryIsAllocatedInHeap: Boolean = - !conf.getBoolean("spark.unsafe.offHeap", false) - /** * Allocates memory for use by Unsafe/Tungsten code. */ - private[memory] final val tungstenMemoryAllocator: MemoryAllocator = - if (tungstenMemoryIsAllocatedInHeap) MemoryAllocator.HEAP else MemoryAllocator.UNSAFE + private[memory] final val tungstenMemoryAllocator: MemoryAllocator = { + tungstenMemoryMode match { + case MemoryMode.ON_HEAP => MemoryAllocator.HEAP + case MemoryMode.OFF_HEAP => MemoryAllocator.UNSAFE + } + } } diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala new file mode 100644 index 000000000000..1b9edf9c43bd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.memory + +import javax.annotation.concurrent.GuardedBy + +/** + * Manages bookkeeping for an adjustable-sized region of memory. This class is internal to + * the [[MemoryManager]]. See subclasses for more details. + * + * @param lock a [[MemoryManager]] instance, used for synchronization. We purposely erase the type + * to `Object` to avoid programming errors, since this object should only be used for + * synchronization purposes. + */ +private[memory] abstract class MemoryPool(lock: Object) { + + @GuardedBy("lock") + private[this] var _poolSize: Long = 0 + + /** + * Returns the current size of the pool, in bytes. + */ + final def poolSize: Long = lock.synchronized { + _poolSize + } + + /** + * Returns the amount of free memory in the pool, in bytes. + */ + final def memoryFree: Long = lock.synchronized { + _poolSize - memoryUsed + } + + /** + * Expands the pool by `delta` bytes. + */ + final def incrementPoolSize(delta: Long): Unit = lock.synchronized { + require(delta >= 0) + _poolSize += delta + } + + /** + * Shrinks the pool by `delta` bytes. + */ + final def decrementPoolSize(delta: Long): Unit = lock.synchronized { + require(delta >= 0) + require(delta <= _poolSize) + require(_poolSize - delta >= memoryUsed) + _poolSize -= delta + } + + /** + * Returns the amount of used memory in this pool (in bytes). + */ + def memoryUsed: Long +} diff --git a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala index 9c2c2e90a228..a6f7db0600e6 100644 --- a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala @@ -17,11 +17,8 @@ package org.apache.spark.memory -import scala.collection.mutable - import org.apache.spark.SparkConf -import org.apache.spark.storage.{BlockId, BlockStatus} - +import org.apache.spark.storage.BlockId /** * A [[MemoryManager]] that statically partitions the heap space into disjoint regions. @@ -32,10 +29,14 @@ import org.apache.spark.storage.{BlockId, BlockStatus} */ private[spark] class StaticMemoryManager( conf: SparkConf, - override val maxExecutionMemory: Long, - override val maxStorageMemory: Long, + maxOnHeapExecutionMemory: Long, + override val maxOnHeapStorageMemory: Long, numCores: Int) - extends MemoryManager(conf, numCores) { + extends MemoryManager( + conf, + numCores, + maxOnHeapStorageMemory, + maxOnHeapExecutionMemory) { def this(conf: SparkConf, numCores: Int) { this( @@ -45,86 +46,68 @@ private[spark] class StaticMemoryManager( numCores) } + // The StaticMemoryManager does not support off-heap storage memory: + offHeapExecutionMemoryPool.incrementPoolSize(offHeapStorageMemoryPool.poolSize) + offHeapStorageMemoryPool.decrementPoolSize(offHeapStorageMemoryPool.poolSize) + // Max number of bytes worth of blocks to evict when unrolling - private val maxMemoryToEvictForUnroll: Long = { - (maxStorageMemory * conf.getDouble("spark.storage.unrollFraction", 0.2)).toLong + private val maxUnrollMemory: Long = { + (maxOnHeapStorageMemory * conf.getDouble("spark.storage.unrollFraction", 0.2)).toLong } - /** - * Acquire N bytes of memory for execution. - * @return number of bytes successfully granted (<= N). - */ - override def doAcquireExecutionMemory( - numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Long = synchronized { - assert(numBytes >= 0) - assert(_executionMemoryUsed <= maxExecutionMemory) - val bytesToGrant = math.min(numBytes, maxExecutionMemory - _executionMemoryUsed) - _executionMemoryUsed += bytesToGrant - bytesToGrant - } + override def maxOffHeapStorageMemory: Long = 0L - /** - * Acquire N bytes of memory to cache the given block, evicting existing ones if necessary. - * Blocks evicted in the process, if any, are added to `evictedBlocks`. - * @return whether all N bytes were successfully granted. - */ override def acquireStorageMemory( blockId: BlockId, numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { - acquireStorageMemory(blockId, numBytes, numBytes, evictedBlocks) + memoryMode: MemoryMode): Boolean = synchronized { + require(memoryMode != MemoryMode.OFF_HEAP, + "StaticMemoryManager does not support off-heap storage memory") + if (numBytes > maxOnHeapStorageMemory) { + // Fail fast if the block simply won't fit + logInfo(s"Will not store $blockId as the required space ($numBytes bytes) exceeds our " + + s"memory limit ($maxOnHeapStorageMemory bytes)") + false + } else { + onHeapStorageMemoryPool.acquireMemory(blockId, numBytes) + } } - /** - * Acquire N bytes of memory to unroll the given block, evicting existing ones if necessary. - * - * This evicts at most M bytes worth of existing blocks, where M is a fraction of the storage - * space specified by `spark.storage.unrollFraction`. Blocks evicted in the process, if any, - * are added to `evictedBlocks`. - * - * @return whether all N bytes were successfully granted. - */ override def acquireUnrollMemory( blockId: BlockId, numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { - val currentUnrollMemory = memoryStore.currentUnrollMemory - val maxNumBytesToFree = math.max(0, maxMemoryToEvictForUnroll - currentUnrollMemory) - val numBytesToFree = math.min(numBytes, maxNumBytesToFree) - acquireStorageMemory(blockId, numBytes, numBytesToFree, evictedBlocks) + memoryMode: MemoryMode): Boolean = synchronized { + require(memoryMode != MemoryMode.OFF_HEAP, + "StaticMemoryManager does not support off-heap unroll memory") + val currentUnrollMemory = onHeapStorageMemoryPool.memoryStore.currentUnrollMemory + val freeMemory = onHeapStorageMemoryPool.memoryFree + // When unrolling, we will use all of the existing free memory, and, if necessary, + // some extra space freed from evicting cached blocks. We must place a cap on the + // amount of memory to be evicted by unrolling, however, otherwise unrolling one + // big block can blow away the entire cache. + val maxNumBytesToFree = math.max(0, maxUnrollMemory - currentUnrollMemory - freeMemory) + // Keep it within the range 0 <= X <= maxNumBytesToFree + val numBytesToFree = math.max(0, math.min(maxNumBytesToFree, numBytes - freeMemory)) + onHeapStorageMemoryPool.acquireMemory(blockId, numBytes, numBytesToFree) } - /** - * Acquire N bytes of storage memory for the given block, evicting existing ones if necessary. - * - * @param blockId the ID of the block we are acquiring storage memory for - * @param numBytesToAcquire the size of this block - * @param numBytesToFree the size of space to be freed through evicting blocks - * @param evictedBlocks a holder for blocks evicted in the process - * @return whether all N bytes were successfully granted. - */ - private def acquireStorageMemory( - blockId: BlockId, - numBytesToAcquire: Long, - numBytesToFree: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { - assert(numBytesToAcquire >= 0) - assert(numBytesToFree >= 0) - memoryStore.ensureFreeSpace(blockId, numBytesToFree, evictedBlocks) - assert(_storageMemoryUsed <= maxStorageMemory) - val enoughMemory = _storageMemoryUsed + numBytesToAcquire <= maxStorageMemory - if (enoughMemory) { - _storageMemoryUsed += numBytesToAcquire + private[memory] + override def acquireExecutionMemory( + numBytes: Long, + taskAttemptId: Long, + memoryMode: MemoryMode): Long = synchronized { + memoryMode match { + case MemoryMode.ON_HEAP => onHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) + case MemoryMode.OFF_HEAP => offHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) } - enoughMemory } - } private[spark] object StaticMemoryManager { + private val MIN_MEMORY_BYTES = 32 * 1024 * 1024 + /** * Return the total amount of memory available for the storage region, in bytes. */ @@ -135,12 +118,25 @@ private[spark] object StaticMemoryManager { (systemMaxMemory * memoryFraction * safetyFraction).toLong } - /** * Return the total amount of memory available for the execution region, in bytes. */ private def getMaxExecutionMemory(conf: SparkConf): Long = { val systemMaxMemory = conf.getLong("spark.testing.memory", Runtime.getRuntime.maxMemory) + + if (systemMaxMemory < MIN_MEMORY_BYTES) { + throw new IllegalArgumentException(s"System memory $systemMaxMemory must " + + s"be at least $MIN_MEMORY_BYTES. Please increase heap size using the --driver-memory " + + s"option or spark.driver.memory in Spark configuration.") + } + if (conf.contains("spark.executor.memory")) { + val executorMemory = conf.getSizeAsBytes("spark.executor.memory") + if (executorMemory < MIN_MEMORY_BYTES) { + throw new IllegalArgumentException(s"Executor memory $executorMemory must be at least " + + s"$MIN_MEMORY_BYTES. Please increase executor memory using the " + + s"--executor-memory option or spark.executor.memory in Spark configuration.") + } + } val memoryFraction = conf.getDouble("spark.shuffle.memoryFraction", 0.2) val safetyFraction = conf.getDouble("spark.shuffle.safetyFraction", 0.8) (systemMaxMemory * memoryFraction * safetyFraction).toLong diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala new file mode 100644 index 000000000000..4c6b639015a9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.memory + +import javax.annotation.concurrent.GuardedBy + +import org.apache.spark.internal.Logging +import org.apache.spark.storage.BlockId +import org.apache.spark.storage.memory.MemoryStore + +/** + * Performs bookkeeping for managing an adjustable-size pool of memory that is used for storage + * (caching). + * + * @param lock a [[MemoryManager]] instance to synchronize on + * @param memoryMode the type of memory tracked by this pool (on- or off-heap) + */ +private[memory] class StorageMemoryPool( + lock: Object, + memoryMode: MemoryMode + ) extends MemoryPool(lock) with Logging { + + private[this] val poolName: String = memoryMode match { + case MemoryMode.ON_HEAP => "on-heap storage" + case MemoryMode.OFF_HEAP => "off-heap storage" + } + + @GuardedBy("lock") + private[this] var _memoryUsed: Long = 0L + + override def memoryUsed: Long = lock.synchronized { + _memoryUsed + } + + private var _memoryStore: MemoryStore = _ + def memoryStore: MemoryStore = { + if (_memoryStore == null) { + throw new IllegalStateException("memory store not initialized yet") + } + _memoryStore + } + + /** + * Set the [[MemoryStore]] used by this manager to evict cached blocks. + * This must be set after construction due to initialization ordering constraints. + */ + final def setMemoryStore(store: MemoryStore): Unit = { + _memoryStore = store + } + + /** + * Acquire N bytes of memory to cache the given block, evicting existing ones if necessary. + * + * @return whether all N bytes were successfully granted. + */ + def acquireMemory(blockId: BlockId, numBytes: Long): Boolean = lock.synchronized { + val numBytesToFree = math.max(0, numBytes - memoryFree) + acquireMemory(blockId, numBytes, numBytesToFree) + } + + /** + * Acquire N bytes of storage memory for the given block, evicting existing ones if necessary. + * + * @param blockId the ID of the block we are acquiring storage memory for + * @param numBytesToAcquire the size of this block + * @param numBytesToFree the amount of space to be freed through evicting blocks + * @return whether all N bytes were successfully granted. + */ + def acquireMemory( + blockId: BlockId, + numBytesToAcquire: Long, + numBytesToFree: Long): Boolean = lock.synchronized { + assert(numBytesToAcquire >= 0) + assert(numBytesToFree >= 0) + assert(memoryUsed <= poolSize) + if (numBytesToFree > 0) { + memoryStore.evictBlocksToFreeSpace(Some(blockId), numBytesToFree, memoryMode) + } + // NOTE: If the memory store evicts blocks, then those evictions will synchronously call + // back into this StorageMemoryPool in order to free memory. Therefore, these variables + // should have been updated. + val enoughMemory = numBytesToAcquire <= memoryFree + if (enoughMemory) { + _memoryUsed += numBytesToAcquire + } + enoughMemory + } + + def releaseMemory(size: Long): Unit = lock.synchronized { + if (size > _memoryUsed) { + logWarning(s"Attempted to release $size bytes of storage " + + s"memory when we only have ${_memoryUsed} bytes") + _memoryUsed = 0 + } else { + _memoryUsed -= size + } + } + + def releaseAllMemory(): Unit = lock.synchronized { + _memoryUsed = 0 + } + + /** + * Free space to shrink the size of this storage memory pool by `spaceToFree` bytes. + * Note: this method doesn't actually reduce the pool size but relies on the caller to do so. + * + * @return number of bytes to be removed from the pool's capacity. + */ + def freeSpaceToShrinkPool(spaceToFree: Long): Long = lock.synchronized { + val spaceFreedByReleasingUnusedMemory = math.min(spaceToFree, memoryFree) + val remainingSpaceToFree = spaceToFree - spaceFreedByReleasingUnusedMemory + if (remainingSpaceToFree > 0) { + // If reclaiming free memory did not adequately shrink the pool, begin evicting blocks: + val spaceFreedByEviction = + memoryStore.evictBlocksToFreeSpace(None, remainingSpaceToFree, memoryMode) + // When a block is released, BlockManager.dropFromMemory() calls releaseMemory(), so we do + // not need to decrement _memoryUsed here. However, we do need to decrement the pool size. + spaceFreedByReleasingUnusedMemory + spaceFreedByEviction + } else { + spaceFreedByReleasingUnusedMemory + } + } +} diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index a3093030a0f9..78edd2c4d7fa 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -17,20 +17,17 @@ package org.apache.spark.memory -import scala.collection.mutable - import org.apache.spark.SparkConf -import org.apache.spark.storage.{BlockStatus, BlockId} - +import org.apache.spark.storage.BlockId /** * A [[MemoryManager]] that enforces a soft boundary between execution and storage such that * either side can borrow memory from the other. * - * The region shared between execution and storage is a fraction of the total heap space - * configurable through `spark.memory.fraction` (default 0.75). The position of the boundary + * The region shared between execution and storage is a fraction of (the total heap space - 300MB) + * configurable through `spark.memory.fraction` (default 0.6). The position of the boundary * within this space is further determined by `spark.memory.storageFraction` (default 0.5). - * This means the size of the storage region is 0.75 * 0.5 = 0.375 of the heap space by default. + * This means the size of the storage region is 0.6 * 0.5 = 0.3 of the heap space by default. * * Storage can borrow as much execution memory as is free until execution reclaims its space. * When this happens, cached blocks will be evicted from memory until sufficient borrowed @@ -41,105 +38,197 @@ import org.apache.spark.storage.{BlockStatus, BlockId} * The implication is that attempts to cache blocks may fail if execution has already eaten * up most of the storage space, in which case the new blocks will be evicted immediately * according to their respective storage levels. + * + * @param onHeapStorageRegionSize Size of the storage region, in bytes. + * This region is not statically reserved; execution can borrow from + * it if necessary. Cached blocks can be evicted only if actual + * storage memory usage exceeds this region. */ -private[spark] class UnifiedMemoryManager( +private[spark] class UnifiedMemoryManager private[memory] ( conf: SparkConf, - maxMemory: Long, + val maxHeapMemory: Long, + onHeapStorageRegionSize: Long, numCores: Int) - extends MemoryManager(conf, numCores) { - - def this(conf: SparkConf, numCores: Int) { - this(conf, UnifiedMemoryManager.getMaxMemory(conf), numCores) - } + extends MemoryManager( + conf, + numCores, + onHeapStorageRegionSize, + maxHeapMemory - onHeapStorageRegionSize) { - /** - * Size of the storage region, in bytes. - * - * This region is not statically reserved; execution can borrow from it if necessary. - * Cached blocks can be evicted only if actual storage memory usage exceeds this region. - */ - private val storageRegionSize: Long = { - (maxMemory * conf.getDouble("spark.memory.storageFraction", 0.5)).toLong + private def assertInvariants(): Unit = { + assert(onHeapExecutionMemoryPool.poolSize + onHeapStorageMemoryPool.poolSize == maxHeapMemory) + assert( + offHeapExecutionMemoryPool.poolSize + offHeapStorageMemoryPool.poolSize == maxOffHeapMemory) } - /** - * Total amount of memory, in bytes, not currently occupied by either execution or storage. - */ - private def totalFreeMemory: Long = synchronized { - assert(_executionMemoryUsed <= maxMemory) - assert(_storageMemoryUsed <= maxMemory) - assert(_executionMemoryUsed + _storageMemoryUsed <= maxMemory) - maxMemory - _executionMemoryUsed - _storageMemoryUsed - } + assertInvariants() - /** - * Total available memory for execution, in bytes. - * In this model, this is equivalent to the amount of memory not occupied by storage. - */ - override def maxExecutionMemory: Long = synchronized { - maxMemory - _storageMemoryUsed + override def maxOnHeapStorageMemory: Long = synchronized { + maxHeapMemory - onHeapExecutionMemoryPool.memoryUsed } - /** - * Total available memory for storage, in bytes. - * In this model, this is equivalent to the amount of memory not occupied by execution. - */ - override def maxStorageMemory: Long = synchronized { - maxMemory - _executionMemoryUsed + override def maxOffHeapStorageMemory: Long = synchronized { + maxOffHeapMemory - offHeapExecutionMemoryPool.memoryUsed } /** - * Acquire N bytes of memory for execution, evicting cached blocks if necessary. + * Try to acquire up to `numBytes` of execution memory for the current task and return the + * number of bytes obtained, or 0 if none can be allocated. * - * This method evicts blocks only up to the amount of memory borrowed by storage. - * Blocks evicted in the process, if any, are added to `evictedBlocks`. - * @return number of bytes successfully granted (<= N). + * This call may block until there is enough free memory in some situations, to make sure each + * task has a chance to ramp up to at least 1 / 2N of the total memory pool (where N is the # of + * active tasks) before it is forced to spill. This can happen if the number of tasks increase + * but an older task had a lot of memory already. */ - private[memory] override def doAcquireExecutionMemory( + override private[memory] def acquireExecutionMemory( numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Long = synchronized { + taskAttemptId: Long, + memoryMode: MemoryMode): Long = synchronized { + assertInvariants() assert(numBytes >= 0) - val memoryBorrowedByStorage = math.max(0, _storageMemoryUsed - storageRegionSize) - // If there is not enough free memory AND storage has borrowed some execution memory, - // then evict as much memory borrowed by storage as needed to grant this request - val shouldEvictStorage = totalFreeMemory < numBytes && memoryBorrowedByStorage > 0 - if (shouldEvictStorage) { - val spaceToEnsure = math.min(numBytes, memoryBorrowedByStorage) - memoryStore.ensureFreeSpace(spaceToEnsure, evictedBlocks) + val (executionPool, storagePool, storageRegionSize, maxMemory) = memoryMode match { + case MemoryMode.ON_HEAP => ( + onHeapExecutionMemoryPool, + onHeapStorageMemoryPool, + onHeapStorageRegionSize, + maxHeapMemory) + case MemoryMode.OFF_HEAP => ( + offHeapExecutionMemoryPool, + offHeapStorageMemoryPool, + offHeapStorageMemory, + maxOffHeapMemory) + } + + /** + * Grow the execution pool by evicting cached blocks, thereby shrinking the storage pool. + * + * When acquiring memory for a task, the execution pool may need to make multiple + * attempts. Each attempt must be able to evict storage in case another task jumps in + * and caches a large block between the attempts. This is called once per attempt. + */ + def maybeGrowExecutionPool(extraMemoryNeeded: Long): Unit = { + if (extraMemoryNeeded > 0) { + // There is not enough free memory in the execution pool, so try to reclaim memory from + // storage. We can reclaim any free memory from the storage pool. If the storage pool + // has grown to become larger than `storageRegionSize`, we can evict blocks and reclaim + // the memory that storage has borrowed from execution. + val memoryReclaimableFromStorage = math.max( + storagePool.memoryFree, + storagePool.poolSize - storageRegionSize) + if (memoryReclaimableFromStorage > 0) { + // Only reclaim as much space as is necessary and available: + val spaceToReclaim = storagePool.freeSpaceToShrinkPool( + math.min(extraMemoryNeeded, memoryReclaimableFromStorage)) + storagePool.decrementPoolSize(spaceToReclaim) + executionPool.incrementPoolSize(spaceToReclaim) + } + } + } + + /** + * The size the execution pool would have after evicting storage memory. + * + * The execution memory pool divides this quantity among the active tasks evenly to cap + * the execution memory allocation for each task. It is important to keep this greater + * than the execution pool size, which doesn't take into account potential memory that + * could be freed by evicting storage. Otherwise we may hit SPARK-12155. + * + * Additionally, this quantity should be kept below `maxMemory` to arbitrate fairness + * in execution memory allocation across tasks, Otherwise, a task may occupy more than + * its fair share of execution memory, mistakenly thinking that other tasks can acquire + * the portion of storage memory that cannot be evicted. + */ + def computeMaxExecutionPoolSize(): Long = { + maxMemory - math.min(storagePool.memoryUsed, storageRegionSize) } - val bytesToGrant = math.min(numBytes, totalFreeMemory) - _executionMemoryUsed += bytesToGrant - bytesToGrant + + executionPool.acquireMemory( + numBytes, taskAttemptId, maybeGrowExecutionPool, () => computeMaxExecutionPoolSize) } - /** - * Acquire N bytes of memory to cache the given block, evicting existing ones if necessary. - * Blocks evicted in the process, if any, are added to `evictedBlocks`. - * @return whether all N bytes were successfully granted. - */ override def acquireStorageMemory( blockId: BlockId, numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { + memoryMode: MemoryMode): Boolean = synchronized { + assertInvariants() assert(numBytes >= 0) - memoryStore.ensureFreeSpace(blockId, numBytes, evictedBlocks) - val enoughMemory = totalFreeMemory >= numBytes - if (enoughMemory) { - _storageMemoryUsed += numBytes + val (executionPool, storagePool, maxMemory) = memoryMode match { + case MemoryMode.ON_HEAP => ( + onHeapExecutionMemoryPool, + onHeapStorageMemoryPool, + maxOnHeapStorageMemory) + case MemoryMode.OFF_HEAP => ( + offHeapExecutionMemoryPool, + offHeapStorageMemoryPool, + maxOffHeapStorageMemory) + } + if (numBytes > maxMemory) { + // Fail fast if the block simply won't fit + logInfo(s"Will not store $blockId as the required space ($numBytes bytes) exceeds our " + + s"memory limit ($maxMemory bytes)") + return false + } + if (numBytes > storagePool.memoryFree) { + // There is not enough free memory in the storage pool, so try to borrow free memory from + // the execution pool. + val memoryBorrowedFromExecution = Math.min(executionPool.memoryFree, + numBytes - storagePool.memoryFree) + executionPool.decrementPoolSize(memoryBorrowedFromExecution) + storagePool.incrementPoolSize(memoryBorrowedFromExecution) } - enoughMemory + storagePool.acquireMemory(blockId, numBytes) } + override def acquireUnrollMemory( + blockId: BlockId, + numBytes: Long, + memoryMode: MemoryMode): Boolean = synchronized { + acquireStorageMemory(blockId, numBytes, memoryMode) + } } -private object UnifiedMemoryManager { +object UnifiedMemoryManager { + + // Set aside a fixed amount of memory for non-storage, non-execution purposes. + // This serves a function similar to `spark.memory.fraction`, but guarantees that we reserve + // sufficient memory for the system even for small heaps. E.g. if we have a 1GB JVM, then + // the memory used for execution and storage will be (1024 - 300) * 0.6 = 434MB by default. + private val RESERVED_SYSTEM_MEMORY_BYTES = 300 * 1024 * 1024 + + def apply(conf: SparkConf, numCores: Int): UnifiedMemoryManager = { + val maxMemory = getMaxMemory(conf) + new UnifiedMemoryManager( + conf, + maxHeapMemory = maxMemory, + onHeapStorageRegionSize = + (maxMemory * conf.getDouble("spark.memory.storageFraction", 0.5)).toLong, + numCores = numCores) + } /** * Return the total amount of memory shared between execution and storage, in bytes. */ private def getMaxMemory(conf: SparkConf): Long = { - val systemMaxMemory = conf.getLong("spark.testing.memory", Runtime.getRuntime.maxMemory) - val memoryFraction = conf.getDouble("spark.memory.fraction", 0.75) - (systemMaxMemory * memoryFraction).toLong + val systemMemory = conf.getLong("spark.testing.memory", Runtime.getRuntime.maxMemory) + val reservedMemory = conf.getLong("spark.testing.reservedMemory", + if (conf.contains("spark.testing")) 0 else RESERVED_SYSTEM_MEMORY_BYTES) + val minSystemMemory = (reservedMemory * 1.5).ceil.toLong + if (systemMemory < minSystemMemory) { + throw new IllegalArgumentException(s"System memory $systemMemory must " + + s"be at least $minSystemMemory. Please increase heap size using the --driver-memory " + + s"option or spark.driver.memory in Spark configuration.") + } + // SPARK-12759 Check executor memory to fail fast if memory is insufficient + if (conf.contains("spark.executor.memory")) { + val executorMemory = conf.getSizeAsBytes("spark.executor.memory") + if (executorMemory < minSystemMemory) { + throw new IllegalArgumentException(s"Executor memory $executorMemory must be at least " + + s"$minSystemMemory. Please increase executor memory using the " + + s"--executor-memory option or spark.executor.memory in Spark configuration.") + } + } + val usableMemory = systemMemory - reservedMemory + val memoryFraction = conf.getDouble("spark.memory.fraction", 0.6) + (usableMemory * memoryFraction).toLong } } diff --git a/core/src/main/scala/org/apache/spark/memory/package.scala b/core/src/main/scala/org/apache/spark/memory/package.scala new file mode 100644 index 000000000000..3d00cd9cb637 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/memory/package.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * This package implements Spark's memory management system. This system consists of two main + * components, a JVM-wide memory manager and a per-task manager: + * + * - [[org.apache.spark.memory.MemoryManager]] manages Spark's overall memory usage within a JVM. + * This component implements the policies for dividing the available memory across tasks and for + * allocating memory between storage (memory used caching and data transfer) and execution + * (memory used by computations, such as shuffles, joins, sorts, and aggregations). + * - [[org.apache.spark.memory.TaskMemoryManager]] manages the memory allocated by individual + * tasks. Tasks interact with TaskMemoryManager and never directly interact with the JVM-wide + * MemoryManager. + * + * Internally, each of these components have additional abstractions for memory bookkeeping: + * + * - [[org.apache.spark.memory.MemoryConsumer]]s are clients of the TaskMemoryManager and + * correspond to individual operators and data structures within a task. The TaskMemoryManager + * receives memory allocation requests from MemoryConsumers and issues callbacks to consumers + * in order to trigger spilling when running low on memory. + * - [[org.apache.spark.memory.MemoryPool]]s are a bookkeeping abstraction used by the + * MemoryManager to track the division of memory between storage and execution. + * + * Diagrammatically: + * + * {{{ + * +-------------+ + * | MemConsumer |----+ +------------------------+ + * +-------------+ | +-------------------+ | MemoryManager | + * +--->| TaskMemoryManager |----+ | | + * +-------------+ | +-------------------+ | | +------------------+ | + * | MemConsumer |----+ | | | StorageMemPool | | + * +-------------+ +-------------------+ | | +------------------+ | + * | TaskMemoryManager |----+ | | + * +-------------------+ | | +------------------+ | + * +---->| |OnHeapExecMemPool | | + * * | | +------------------+ | + * * | | | + * +-------------+ * | | +------------------+ | + * | MemConsumer |----+ | | |OffHeapExecMemPool| | + * +-------------+ | +-------------------+ | | +------------------+ | + * +--->| TaskMemoryManager |----+ | | + * +-------------------+ +------------------------+ + * }}} + * + * + * There are two implementations of [[org.apache.spark.memory.MemoryManager]] which vary in how + * they handle the sizing of their memory pools: + * + * - [[org.apache.spark.memory.UnifiedMemoryManager]], the default in Spark 1.6+, enforces soft + * boundaries between storage and execution memory, allowing requests for memory in one region + * to be fulfilled by borrowing memory from the other. + * - [[org.apache.spark.memory.StaticMemoryManager]] enforces hard boundaries between storage + * and execution memory by statically partitioning Spark's memory and preventing storage and + * execution from borrowing memory from each other. This mode is retained only for legacy + * compatibility purposes. + */ +package object memory 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 dd2d325d8703..a4056508c181 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -24,8 +24,9 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.matching.Regex +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils -import org.apache.spark.{Logging, SparkConf} private[spark] class MetricsConfig(conf: SparkConf) extends Logging { @@ -34,7 +35,7 @@ private[spark] class MetricsConfig(conf: SparkConf) extends Logging { private val DEFAULT_METRICS_CONF_FILENAME = "metrics.properties" private[metrics] val properties = new Properties() - private[metrics] var propertyCategories: mutable.HashMap[String, Properties] = null + private[metrics] var perInstanceSubProperties: mutable.HashMap[String, Properties] = null private def setDefaultProperties(prop: Properties) { prop.setProperty("*.sink.servlet.class", "org.apache.spark.metrics.sink.MetricsServlet") @@ -43,6 +44,10 @@ private[spark] class MetricsConfig(conf: SparkConf) extends Logging { prop.setProperty("applications.sink.servlet.path", "/metrics/applications/json") } + /** + * Load properties from various places, based on precedence + * If the same property is set again latter on in the method, it overwrites the previous value + */ def initialize() { // Add default properties in case there's no properties file setDefaultProperties(properties) @@ -57,16 +62,47 @@ private[spark] class MetricsConfig(conf: SparkConf) extends Logging { case _ => } - propertyCategories = subProperties(properties, INSTANCE_REGEX) - if (propertyCategories.contains(DEFAULT_PREFIX)) { - val defaultProperty = propertyCategories(DEFAULT_PREFIX).asScala - for((inst, prop) <- propertyCategories if (inst != DEFAULT_PREFIX); - (k, v) <- defaultProperty if (prop.get(k) == null)) { + // Now, let's populate a list of sub-properties per instance, instance being the prefix that + // appears before the first dot in the property name. + // Add to the sub-properties per instance, the default properties (those with prefix "*"), if + // they don't have that exact same sub-property already defined. + // + // For example, if properties has ("*.class"->"default_class", "*.path"->"default_path, + // "driver.path"->"driver_path"), for driver specific sub-properties, we'd like the output to be + // ("driver"->Map("path"->"driver_path", "class"->"default_class") + // Note how class got added to based on the default property, but path remained the same + // since "driver.path" already existed and took precedence over "*.path" + // + perInstanceSubProperties = subProperties(properties, INSTANCE_REGEX) + if (perInstanceSubProperties.contains(DEFAULT_PREFIX)) { + val defaultSubProperties = perInstanceSubProperties(DEFAULT_PREFIX).asScala + for ((instance, prop) <- perInstanceSubProperties if (instance != DEFAULT_PREFIX); + (k, v) <- defaultSubProperties if (prop.get(k) == null)) { prop.put(k, v) } } } + /** + * Take a simple set of properties and a regex that the instance names (part before the first dot) + * have to conform to. And, return a map of the first order prefix (before the first dot) to the + * sub-properties under that prefix. + * + * For example, if the properties sent were Properties("*.sink.servlet.class"->"class1", + * "*.sink.servlet.path"->"path1"), the returned map would be + * Map("*" -> Properties("sink.servlet.class" -> "class1", "sink.servlet.path" -> "path1")) + * Note in the subProperties (value of the returned Map), only the suffixes are used as property + * keys. + * If, in the passed properties, there is only one property with a given prefix, it is still + * "unflattened". For example, if the input was Properties("*.sink.servlet.class" -> "class1" + * the returned Map would contain one key-value pair + * Map("*" -> Properties("sink.servlet.class" -> "class1")) + * Any passed in properties, not complying with the regex are ignored. + * + * @param prop the flat list of properties to "unflatten" based on prefixes + * @param regex the regex that the prefix has to comply with + * @return an unflatted map, mapping prefix with sub-properties under that prefix + */ def subProperties(prop: Properties, regex: Regex): mutable.HashMap[String, Properties] = { val subProperties = new mutable.HashMap[String, Properties] prop.asScala.foreach { kv => @@ -79,9 +115,9 @@ private[spark] class MetricsConfig(conf: SparkConf) extends Logging { } def getInstance(inst: String): Properties = { - propertyCategories.get(inst) match { + perInstanceSubProperties.get(inst) match { case Some(s) => s - case None => propertyCategories.getOrElse(DEFAULT_PREFIX, new Properties) + case None => perInstanceSubProperties.getOrElse(DEFAULT_PREFIX, new Properties) } } 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 fdf76d312db3..3457a2632277 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -20,36 +20,37 @@ package org.apache.spark.metrics import java.util.Properties import java.util.concurrent.TimeUnit -import org.apache.spark.util.Utils - import scala.collection.mutable import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} import org.eclipse.jetty.servlet.ServletContextHandler -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ import org.apache.spark.metrics.sink.{MetricsServlet, Sink} -import org.apache.spark.metrics.source.Source +import org.apache.spark.metrics.source.{Source, StaticSources} +import org.apache.spark.util.Utils /** - * Spark Metrics System, created by specific "instance", combined by source, - * sink, periodically poll source metrics data to sink destinations. + * Spark Metrics System, created by a specific "instance", combined by source, + * sink, periodically polls source metrics data to sink destinations. * - * "instance" specify "who" (the role) use metrics system. In spark there are several roles - * like master, worker, executor, client driver, these roles will create metrics system - * for monitoring. So instance represents these roles. Currently in Spark, several instances + * "instance" specifies "who" (the role) uses the metrics system. In Spark, there are several roles + * like master, worker, executor, client driver. These roles will create metrics system + * for monitoring. So, "instance" represents these roles. Currently in Spark, several instances * have already implemented: master, worker, executor, driver, applications. * - * "source" specify "where" (source) to collect metrics data. In metrics system, there exists + * "source" specifies "where" (source) to collect metrics data from. In metrics system, there exists * two kinds of source: * 1. Spark internal source, like MasterSource, WorkerSource, etc, which will collect * Spark component's internal state, these sources are related to instance and will be - * added after specific metrics system is created. + * added after a specific metrics system is created. * 2. Common source, like JvmSource, which will collect low level state, is configured by * configuration and loaded through reflection. * - * "sink" specify "where" (destination) to output metrics data to. Several sinks can be - * coexisted and flush metrics to all these sinks. + * "sink" specifies "where" (destination) to output metrics data to. Several sinks can + * coexist and metrics can be flushed to all these sinks. * * Metrics configuration format is like below: * [instance].[sink|source].[name].[options] = xxxx @@ -62,9 +63,9 @@ import org.apache.spark.metrics.source.Source * [sink|source] means this property belongs to source or sink. This field can only be * source or sink. * - * [name] specify the name of sink or source, it is custom defined. + * [name] specify the name of sink or source, if it is custom defined. * - * [options] is the specific property of this source or sink. + * [options] represent the specific property of this source or sink. */ private[spark] class MetricsSystem private ( val instance: String, @@ -96,6 +97,7 @@ private[spark] class MetricsSystem private ( def start() { require(!running, "Attempting to start a MetricsSystem that is already running") running = true + StaticSources.allSources.foreach(registerSource) registerSources() registerSinks() sinks.foreach(_.start) @@ -124,19 +126,25 @@ private[spark] class MetricsSystem private ( * application, executor/driver and metric source. */ private[spark] def buildRegistryName(source: Source): String = { - val appId = conf.getOption("spark.app.id") + val metricsNamespace = conf.get(METRICS_NAMESPACE).orElse(conf.getOption("spark.app.id")) + val executorId = conf.getOption("spark.executor.id") val defaultName = MetricRegistry.name(source.sourceName) if (instance == "driver" || instance == "executor") { - if (appId.isDefined && executorId.isDefined) { - MetricRegistry.name(appId.get, executorId.get, source.sourceName) + if (metricsNamespace.isDefined && executorId.isDefined) { + MetricRegistry.name(metricsNamespace.get, executorId.get, source.sourceName) } else { // Only Driver and Executor set spark.app.id and spark.executor.id. // Other instance types, e.g. Master and Worker, are not related to a specific application. - val warningMsg = s"Using default name $defaultName for source because %s is not set." - if (appId.isEmpty) { logWarning(warningMsg.format("spark.app.id")) } - if (executorId.isEmpty) { logWarning(warningMsg.format("spark.executor.id")) } + if (metricsNamespace.isEmpty) { + logWarning(s"Using default name $defaultName for source because neither " + + s"${METRICS_NAMESPACE.key} nor spark.app.id is set.") + } + if (executorId.isEmpty) { + logWarning(s"Using default name $defaultName for source because spark.executor.id is " + + s"not set.") + } defaultName } } else { defaultName } @@ -196,10 +204,9 @@ private[spark] class MetricsSystem private ( sinks += sink.asInstanceOf[Sink] } } catch { - case e: Exception => { + case e: Exception => logError("Sink class " + classPath + " cannot be instantiated") throw e - } } } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala index 81b9056b40fb..fce556fd0382 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala @@ -17,7 +17,7 @@ package org.apache.spark.metrics.sink -import java.util.Properties +import java.util.{Locale, Properties} import java.util.concurrent.TimeUnit import com.codahale.metrics.{ConsoleReporter, MetricRegistry} @@ -39,7 +39,7 @@ private[spark] class ConsoleSink(val property: Properties, val registry: MetricR } val pollUnit: TimeUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match { - case Some(s) => TimeUnit.valueOf(s.toUpperCase()) + case Some(s) => TimeUnit.valueOf(s.toUpperCase(Locale.ROOT)) case None => TimeUnit.valueOf(CONSOLE_DEFAULT_UNIT) } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 9d5f2ae9328a..88bba2fdbd1c 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -42,7 +42,7 @@ private[spark] class CsvSink(val property: Properties, val registry: MetricRegis } val pollUnit: TimeUnit = Option(property.getProperty(CSV_KEY_UNIT)) match { - case Some(s) => TimeUnit.valueOf(s.toUpperCase()) + case Some(s) => TimeUnit.valueOf(s.toUpperCase(Locale.ROOT)) case None => TimeUnit.valueOf(CSV_DEFAULT_UNIT) } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala index 2d25ebd66159..ac33e68abb49 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala @@ -18,11 +18,11 @@ package org.apache.spark.metrics.sink import java.net.InetSocketAddress -import java.util.Properties +import java.util.{Locale, Properties} import java.util.concurrent.TimeUnit import com.codahale.metrics.MetricRegistry -import com.codahale.metrics.graphite.{GraphiteUDP, Graphite, GraphiteReporter} +import com.codahale.metrics.graphite.{Graphite, GraphiteReporter, GraphiteUDP} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem @@ -59,7 +59,7 @@ private[spark] class GraphiteSink(val property: Properties, val registry: Metric } val pollUnit: TimeUnit = propertyToOption(GRAPHITE_KEY_UNIT) match { - case Some(s) => TimeUnit.valueOf(s.toUpperCase()) + case Some(s) => TimeUnit.valueOf(s.toUpperCase(Locale.ROOT)) case None => TimeUnit.valueOf(GRAPHITE_DEFAULT_UNIT) } @@ -67,9 +67,9 @@ private[spark] class GraphiteSink(val property: Properties, val registry: Metric MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) - val graphite = propertyToOption(GRAPHITE_KEY_PROTOCOL).map(_.toLowerCase) match { - case Some("udp") => new GraphiteUDP(new InetSocketAddress(host, port)) - case Some("tcp") | None => new Graphite(new InetSocketAddress(host, port)) + val graphite = propertyToOption(GRAPHITE_KEY_PROTOCOL).map(_.toLowerCase(Locale.ROOT)) match { + case Some("udp") => new GraphiteUDP(host, port) + case Some("tcp") | None => new Graphite(host, port) case Some(p) => throw new Exception(s"Invalid Graphite protocol: $p") } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala index 2588fe2c9edb..1992b42ac7f6 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala @@ -20,6 +20,7 @@ package org.apache.spark.metrics.sink import java.util.Properties import com.codahale.metrics.{JmxReporter, MetricRegistry} + import org.apache.spark.SecurityManager private[spark] class JmxSink(val property: Properties, val registry: MetricRegistry, diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 4193e1d21d3c..68b58b849064 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -19,7 +19,6 @@ package org.apache.spark.metrics.sink import java.util.Properties import java.util.concurrent.TimeUnit - import javax.servlet.http.HttpServletRequest import com.codahale.metrics.MetricRegistry @@ -27,7 +26,7 @@ import com.codahale.metrics.json.MetricsModule import com.fasterxml.jackson.databind.ObjectMapper import org.eclipse.jetty.servlet.ServletContextHandler -import org.apache.spark.{SparkConf, SecurityManager} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.ui.JettyUtils._ private[spark] class MetricsServlet( diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/Slf4jSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/Slf4jSink.scala index 11dfcfe2f04e..7fa4ba762298 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/Slf4jSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/Slf4jSink.scala @@ -17,10 +17,10 @@ package org.apache.spark.metrics.sink -import java.util.Properties +import java.util.{Locale, Properties} import java.util.concurrent.TimeUnit -import com.codahale.metrics.{Slf4jReporter, MetricRegistry} +import com.codahale.metrics.{MetricRegistry, Slf4jReporter} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem @@ -42,7 +42,7 @@ private[spark] class Slf4jSink( } val pollUnit: TimeUnit = Option(property.getProperty(SLF4J_KEY_UNIT)) match { - case Some(s) => TimeUnit.valueOf(s.toUpperCase()) + case Some(s) => TimeUnit.valueOf(s.toUpperCase(Locale.ROOT)) case None => TimeUnit.valueOf(SLF4J_DEFAULT_UNIT) } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/StatsdReporter.scala b/core/src/main/scala/org/apache/spark/metrics/sink/StatsdReporter.scala new file mode 100644 index 000000000000..ba75aa1c65cc --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/sink/StatsdReporter.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.metrics.sink + +import java.io.IOException +import java.net.{DatagramPacket, DatagramSocket, InetSocketAddress} +import java.nio.charset.StandardCharsets.UTF_8 +import java.util.SortedMap +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.util.{Failure, Success, Try} + +import com.codahale.metrics._ +import org.apache.hadoop.net.NetUtils + +import org.apache.spark.internal.Logging + +/** + * @see + * StatsD metric types + */ +private[spark] object StatsdMetricType { + val COUNTER = "c" + val GAUGE = "g" + val TIMER = "ms" + val Set = "s" +} + +private[spark] class StatsdReporter( + registry: MetricRegistry, + host: String = "127.0.0.1", + port: Int = 8125, + prefix: String = "", + filter: MetricFilter = MetricFilter.ALL, + rateUnit: TimeUnit = TimeUnit.SECONDS, + durationUnit: TimeUnit = TimeUnit.MILLISECONDS) + extends ScheduledReporter(registry, "statsd-reporter", filter, rateUnit, durationUnit) + with Logging { + + import StatsdMetricType._ + + private val address = new InetSocketAddress(host, port) + private val whitespace = "[\\s]+".r + + override def report( + gauges: SortedMap[String, Gauge[_]], + counters: SortedMap[String, Counter], + histograms: SortedMap[String, Histogram], + meters: SortedMap[String, Meter], + timers: SortedMap[String, Timer]): Unit = + Try(new DatagramSocket) match { + case Failure(ioe: IOException) => logWarning("StatsD datagram socket construction failed", + NetUtils.wrapException(host, port, NetUtils.getHostname(), 0, ioe)) + case Failure(e) => logWarning("StatsD datagram socket construction failed", e) + case Success(s) => + implicit val socket = s + val localAddress = Try(socket.getLocalAddress).map(_.getHostAddress).getOrElse(null) + val localPort = socket.getLocalPort + Try { + gauges.entrySet.asScala.foreach(e => reportGauge(e.getKey, e.getValue)) + counters.entrySet.asScala.foreach(e => reportCounter(e.getKey, e.getValue)) + histograms.entrySet.asScala.foreach(e => reportHistogram(e.getKey, e.getValue)) + meters.entrySet.asScala.foreach(e => reportMetered(e.getKey, e.getValue)) + timers.entrySet.asScala.foreach(e => reportTimer(e.getKey, e.getValue)) + } recover { + case ioe: IOException => + logDebug(s"Unable to send packets to StatsD", NetUtils.wrapException( + address.getHostString, address.getPort, localAddress, localPort, ioe)) + case e: Throwable => logDebug(s"Unable to send packets to StatsD at '$host:$port'", e) + } + Try(socket.close()) recover { + case ioe: IOException => + logDebug("Error when close socket to StatsD", NetUtils.wrapException( + address.getHostString, address.getPort, localAddress, localPort, ioe)) + case e: Throwable => logDebug("Error when close socket to StatsD", e) + } + } + + private def reportGauge(name: String, gauge: Gauge[_])(implicit socket: DatagramSocket): Unit = + formatAny(gauge.getValue).foreach(v => send(fullName(name), v, GAUGE)) + + private def reportCounter(name: String, counter: Counter)(implicit socket: DatagramSocket): Unit = + send(fullName(name), format(counter.getCount), COUNTER) + + private def reportHistogram(name: String, histogram: Histogram) + (implicit socket: DatagramSocket): Unit = { + val snapshot = histogram.getSnapshot + send(fullName(name, "count"), format(histogram.getCount), GAUGE) + send(fullName(name, "max"), format(snapshot.getMax), TIMER) + send(fullName(name, "mean"), format(snapshot.getMean), TIMER) + send(fullName(name, "min"), format(snapshot.getMin), TIMER) + send(fullName(name, "stddev"), format(snapshot.getStdDev), TIMER) + send(fullName(name, "p50"), format(snapshot.getMedian), TIMER) + send(fullName(name, "p75"), format(snapshot.get75thPercentile), TIMER) + send(fullName(name, "p95"), format(snapshot.get95thPercentile), TIMER) + send(fullName(name, "p98"), format(snapshot.get98thPercentile), TIMER) + send(fullName(name, "p99"), format(snapshot.get99thPercentile), TIMER) + send(fullName(name, "p999"), format(snapshot.get999thPercentile), TIMER) + } + + private def reportMetered(name: String, meter: Metered)(implicit socket: DatagramSocket): Unit = { + send(fullName(name, "count"), format(meter.getCount), GAUGE) + send(fullName(name, "m1_rate"), format(convertRate(meter.getOneMinuteRate)), TIMER) + send(fullName(name, "m5_rate"), format(convertRate(meter.getFiveMinuteRate)), TIMER) + send(fullName(name, "m15_rate"), format(convertRate(meter.getFifteenMinuteRate)), TIMER) + send(fullName(name, "mean_rate"), format(convertRate(meter.getMeanRate)), TIMER) + } + + private def reportTimer(name: String, timer: Timer)(implicit socket: DatagramSocket): Unit = { + val snapshot = timer.getSnapshot + send(fullName(name, "max"), format(convertDuration(snapshot.getMax)), TIMER) + send(fullName(name, "mean"), format(convertDuration(snapshot.getMean)), TIMER) + send(fullName(name, "min"), format(convertDuration(snapshot.getMin)), TIMER) + send(fullName(name, "stddev"), format(convertDuration(snapshot.getStdDev)), TIMER) + send(fullName(name, "p50"), format(convertDuration(snapshot.getMedian)), TIMER) + send(fullName(name, "p75"), format(convertDuration(snapshot.get75thPercentile)), TIMER) + send(fullName(name, "p95"), format(convertDuration(snapshot.get95thPercentile)), TIMER) + send(fullName(name, "p98"), format(convertDuration(snapshot.get98thPercentile)), TIMER) + send(fullName(name, "p99"), format(convertDuration(snapshot.get99thPercentile)), TIMER) + send(fullName(name, "p999"), format(convertDuration(snapshot.get999thPercentile)), TIMER) + + reportMetered(name, timer) + } + + private def send(name: String, value: String, metricType: String) + (implicit socket: DatagramSocket): Unit = { + val bytes = sanitize(s"$name:$value|$metricType").getBytes(UTF_8) + val packet = new DatagramPacket(bytes, bytes.length, address) + socket.send(packet) + } + + private def fullName(names: String*): String = MetricRegistry.name(prefix, names : _*) + + private def sanitize(s: String): String = whitespace.replaceAllIn(s, "-") + + private def format(v: Any): String = formatAny(v).getOrElse("") + + private def formatAny(v: Any): Option[String] = + v match { + case f: Float => Some("%2.2f".format(f)) + case d: Double => Some("%2.2f".format(d)) + case b: BigDecimal => Some("%2.2f".format(b)) + case n: Number => Some(v.toString) + case _ => None + } +} + diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/StatsdSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/StatsdSink.scala new file mode 100644 index 000000000000..859a2f6bcd45 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/sink/StatsdSink.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.metrics.sink + +import java.util.Properties +import java.util.concurrent.TimeUnit + +import com.codahale.metrics.MetricRegistry + +import org.apache.spark.SecurityManager +import org.apache.spark.internal.Logging +import org.apache.spark.metrics.MetricsSystem + +private[spark] object StatsdSink { + val STATSD_KEY_HOST = "host" + val STATSD_KEY_PORT = "port" + val STATSD_KEY_PERIOD = "period" + val STATSD_KEY_UNIT = "unit" + val STATSD_KEY_PREFIX = "prefix" + + val STATSD_DEFAULT_HOST = "127.0.0.1" + val STATSD_DEFAULT_PORT = "8125" + val STATSD_DEFAULT_PERIOD = "10" + val STATSD_DEFAULT_UNIT = "SECONDS" + val STATSD_DEFAULT_PREFIX = "" +} + +private[spark] class StatsdSink( + val property: Properties, + val registry: MetricRegistry, + securityMgr: SecurityManager) + extends Sink with Logging { + import StatsdSink._ + + val host = property.getProperty(STATSD_KEY_HOST, STATSD_DEFAULT_HOST) + val port = property.getProperty(STATSD_KEY_PORT, STATSD_DEFAULT_PORT).toInt + + val pollPeriod = property.getProperty(STATSD_KEY_PERIOD, STATSD_DEFAULT_PERIOD).toInt + val pollUnit = + TimeUnit.valueOf(property.getProperty(STATSD_KEY_UNIT, STATSD_DEFAULT_UNIT).toUpperCase) + + val prefix = property.getProperty(STATSD_KEY_PREFIX, STATSD_DEFAULT_PREFIX) + + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + + val reporter = new StatsdReporter(registry, host, port, prefix) + + override def start(): Unit = { + reporter.start(pollPeriod, pollUnit) + logInfo(s"StatsdSink started with prefix: '$prefix'") + } + + override def stop(): Unit = { + reporter.stop() + logInfo("StatsdSink stopped.") + } + + override def report(): Unit = reporter.report() +} + diff --git a/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala b/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala new file mode 100644 index 000000000000..99ec78633ab7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.metrics.source + +import com.codahale.metrics.MetricRegistry + +import org.apache.spark.annotation.Experimental + +private[spark] object StaticSources { + /** + * The set of all static sources. These sources may be reported to from any class, including + * static classes, without requiring reference to a SparkEnv. + */ + val allSources = Seq(CodegenMetrics, HiveCatalogMetrics) +} + +/** + * :: Experimental :: + * Metrics for code generation. + */ +@Experimental +object CodegenMetrics extends Source { + override val sourceName: String = "CodeGenerator" + override val metricRegistry: MetricRegistry = new MetricRegistry() + + /** + * Histogram of the length of source code text compiled by CodeGenerator (in characters). + */ + val METRIC_SOURCE_CODE_SIZE = metricRegistry.histogram(MetricRegistry.name("sourceCodeSize")) + + /** + * Histogram of the time it took to compile source code text (in milliseconds). + */ + val METRIC_COMPILATION_TIME = metricRegistry.histogram(MetricRegistry.name("compilationTime")) + + /** + * Histogram of the bytecode size of each class generated by CodeGenerator. + */ + val METRIC_GENERATED_CLASS_BYTECODE_SIZE = + metricRegistry.histogram(MetricRegistry.name("generatedClassSize")) + + /** + * Histogram of the bytecode size of each method in classes generated by CodeGenerator. + */ + val METRIC_GENERATED_METHOD_BYTECODE_SIZE = + metricRegistry.histogram(MetricRegistry.name("generatedMethodSize")) +} + +/** + * :: Experimental :: + * Metrics for access to the hive external catalog. + */ +@Experimental +object HiveCatalogMetrics extends Source { + override val sourceName: String = "HiveExternalCatalog" + override val metricRegistry: MetricRegistry = new MetricRegistry() + + /** + * Tracks the total number of partition metadata entries fetched via the client api. + */ + val METRIC_PARTITIONS_FETCHED = metricRegistry.counter(MetricRegistry.name("partitionsFetched")) + + /** + * Tracks the total number of files discovered off of the filesystem by InMemoryFileIndex. + */ + val METRIC_FILES_DISCOVERED = metricRegistry.counter(MetricRegistry.name("filesDiscovered")) + + /** + * Tracks the total number of files served from the file status cache instead of discovered. + */ + val METRIC_FILE_CACHE_HITS = metricRegistry.counter(MetricRegistry.name("fileCacheHits")) + + /** + * Tracks the total number of Hive client calls (e.g. to lookup a table). + */ + val METRIC_HIVE_CLIENT_CALLS = metricRegistry.counter(MetricRegistry.name("hiveClientCalls")) + + /** + * Tracks the total number of Spark jobs launched for parallel file listing. + */ + val METRIC_PARALLEL_LISTING_JOB_COUNT = metricRegistry.counter( + MetricRegistry.name("parallelListingJobCount")) + + /** + * Resets the values of all metrics to zero. This is useful in tests. + */ + def reset(): Unit = { + METRIC_PARTITIONS_FETCHED.dec(METRIC_PARTITIONS_FETCHED.getCount()) + METRIC_FILES_DISCOVERED.dec(METRIC_FILES_DISCOVERED.getCount()) + METRIC_FILE_CACHE_HITS.dec(METRIC_FILE_CACHE_HITS.getCount()) + METRIC_HIVE_CLIENT_CALLS.dec(METRIC_HIVE_CLIENT_CALLS.getCount()) + METRIC_PARALLEL_LISTING_JOB_COUNT.dec(METRIC_PARALLEL_LISTING_JOB_COUNT.getCount()) + } + + // clients can use these to avoid classloader issues with the codahale classes + def incrementFetchedPartitions(n: Int): Unit = METRIC_PARTITIONS_FETCHED.inc(n) + def incrementFilesDiscovered(n: Int): Unit = METRIC_FILES_DISCOVERED.inc(n) + def incrementFileCacheHits(n: Int): Unit = METRIC_FILE_CACHE_HITS.inc(n) + def incrementHiveClientCalls(n: Int): Unit = METRIC_HIVE_CLIENT_CALLS.inc(n) + def incrementParallelListingJobCount(n: Int): Unit = METRIC_PARALLEL_LISTING_JOB_COUNT.inc(n) +} diff --git a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala index 1745d52c8192..b3f8bfe8b1d4 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala @@ -17,6 +17,8 @@ package org.apache.spark.network +import scala.reflect.ClassTag + import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.storage.{BlockId, StorageLevel} @@ -31,6 +33,18 @@ trait BlockDataManager { /** * Put the block locally, using the given storage level. + * + * Returns true if the block was stored and false if the put operation failed or the block + * already existed. + */ + def putBlockData( + blockId: BlockId, + data: ManagedBuffer, + level: StorageLevel, + classTag: ClassTag[_]): Boolean + + /** + * Release locks acquired by [[putBlockData()]] and [[getBlockData()]]. */ - def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Unit + def releaseLock(blockId: BlockId, taskAttemptId: Option[Long]): Unit } diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index dcbda5a8515d..fe5fd2da039b 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -20,13 +20,15 @@ package org.apache.spark.network import java.io.Closeable import java.nio.ByteBuffer -import scala.concurrent.{Promise, Await, Future} +import scala.concurrent.{Future, Promise} import scala.concurrent.duration.Duration +import scala.reflect.ClassTag -import org.apache.spark.Logging -import org.apache.spark.network.buffer.{NioManagedBuffer, ManagedBuffer} -import org.apache.spark.network.shuffle.{ShuffleClient, BlockFetchingListener} -import org.apache.spark.storage.{BlockManagerId, BlockId, StorageLevel} +import org.apache.spark.internal.Logging +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient, TempShuffleFileManager} +import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.util.ThreadUtils private[spark] abstract class BlockTransferService extends ShuffleClient with Closeable with Logging { @@ -35,7 +37,7 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo * Initialize the transfer service by giving it the BlockDataManager that can be used to fetch * local blocks or put local blocks. */ - def init(blockDataManager: BlockDataManager) + def init(blockDataManager: BlockDataManager): Unit /** * Tear down the transfer service. @@ -65,7 +67,8 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo port: Int, execId: String, blockIds: Array[String], - listener: BlockFetchingListener): Unit + listener: BlockFetchingListener, + tempShuffleFileManager: TempShuffleFileManager): Unit /** * Upload a single block to a remote node, available only after [[init]] is invoked. @@ -76,7 +79,8 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo execId: String, blockId: BlockId, blockData: ManagedBuffer, - level: StorageLevel): Future[Unit] + level: StorageLevel, + classTag: ClassTag[_]): Future[Unit] /** * A special case of [[fetchBlocks]], as it fetches only one block and is blocking. @@ -97,9 +101,8 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo ret.flip() result.success(new NioManagedBuffer(ret)) } - }) - - Await.result(result.future, Duration.Inf) + }, tempShuffleFileManager = null) + ThreadUtils.awaitResult(result.future, Duration.Inf) } /** @@ -114,7 +117,9 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo execId: String, blockId: BlockId, blockData: ManagedBuffer, - level: StorageLevel): Unit = { - Await.result(uploadBlock(hostname, port, execId, blockId, blockData, level), Duration.Inf) + level: StorageLevel, + classTag: ClassTag[_]): Unit = { + val future = uploadBlock(hostname, port, execId, blockId, blockData, level, classTag) + ThreadUtils.awaitResult(future, Duration.Inf) } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index 76968249fb62..eb4cf94164fd 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -20,10 +20,12 @@ package org.apache.spark.network.netty import java.nio.ByteBuffer import scala.collection.JavaConverters._ +import scala.language.existentials +import scala.reflect.ClassTag -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.network.BlockDataManager -import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.buffer.NioManagedBuffer import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.server.{OneForOneStreamManager, RpcHandler, StreamManager} import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, OpenBlocks, StreamHandle, UploadBlock} @@ -47,26 +49,32 @@ class NettyBlockRpcServer( override def receive( client: TransportClient, - messageBytes: Array[Byte], + rpcMessage: ByteBuffer, responseContext: RpcResponseCallback): Unit = { - val message = BlockTransferMessage.Decoder.fromByteArray(messageBytes) + val message = BlockTransferMessage.Decoder.fromByteBuffer(rpcMessage) logTrace(s"Received request: $message") message match { case openBlocks: OpenBlocks => - val blocks: Seq[ManagedBuffer] = - openBlocks.blockIds.map(BlockId.apply).map(blockManager.getBlockData) + val blocksNum = openBlocks.blockIds.length + val blocks = for (i <- (0 until blocksNum).view) + yield blockManager.getBlockData(BlockId.apply(openBlocks.blockIds(i))) val streamId = streamManager.registerStream(appId, blocks.iterator.asJava) - logTrace(s"Registered streamId $streamId with ${blocks.size} buffers") - responseContext.onSuccess(new StreamHandle(streamId, blocks.size).toByteArray) + logTrace(s"Registered streamId $streamId with $blocksNum buffers") + responseContext.onSuccess(new StreamHandle(streamId, blocksNum).toByteBuffer) case uploadBlock: UploadBlock => - // StorageLevel is serialized as bytes using our JavaSerializer. - val level: StorageLevel = - serializer.newInstance().deserialize(ByteBuffer.wrap(uploadBlock.metadata)) + // StorageLevel and ClassTag are serialized as bytes using our JavaSerializer. + val (level: StorageLevel, classTag: ClassTag[_]) = { + serializer + .newInstance() + .deserialize(ByteBuffer.wrap(uploadBlock.metadata)) + .asInstanceOf[(StorageLevel, ClassTag[_])] + } val data = new NioManagedBuffer(ByteBuffer.wrap(uploadBlock.blockData)) - blockManager.putBlockData(BlockId(uploadBlock.blockId), data, level) - responseContext.onSuccess(new Array[Byte](0)) + val blockId = BlockId(uploadBlock.blockId) + blockManager.putBlockData(blockId, data, level, classTag) + responseContext.onSuccess(ByteBuffer.allocate(0)) } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 70a42f9045e6..ac4d85004bad 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -17,31 +17,44 @@ package org.apache.spark.network.netty +import java.nio.ByteBuffer +import java.util.{HashMap => JHashMap, Map => JMap} + import scala.collection.JavaConverters._ import scala.concurrent.{Future, Promise} +import scala.reflect.ClassTag + +import com.codahale.metrics.{Metric, MetricSet} import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.network._ import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.network.client.{TransportClientBootstrap, RpcResponseCallback, TransportClientFactory} -import org.apache.spark.network.sasl.{SaslClientBootstrap, SaslServerBootstrap} +import org.apache.spark.network.client.{RpcResponseCallback, TransportClientBootstrap, TransportClientFactory} +import org.apache.spark.network.crypto.{AuthClientBootstrap, AuthServerBootstrap} import org.apache.spark.network.server._ -import org.apache.spark.network.shuffle.{RetryingBlockFetcher, BlockFetchingListener, OneForOneBlockFetcher} +import org.apache.spark.network.shuffle.{BlockFetchingListener, OneForOneBlockFetcher, RetryingBlockFetcher, TempShuffleFileManager} import org.apache.spark.network.shuffle.protocol.UploadBlock +import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.util.Utils /** - * A BlockTransferService that uses Netty to fetch a set of blocks at at time. + * A BlockTransferService that uses Netty to fetch a set of blocks at time. */ -class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManager, numCores: Int) +private[spark] class NettyBlockTransferService( + conf: SparkConf, + securityManager: SecurityManager, + bindAddress: String, + override val hostName: String, + _port: Int, + numCores: Int) extends BlockTransferService { // TODO: Don't use Java serialization, use a more cross-version compatible serialization format. private val serializer = new JavaSerializer(conf) private val authEnabled = securityManager.isAuthenticationEnabled() - private val transportConf = SparkTransportConf.fromSparkConf(conf, numCores) + private val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numCores) private[this] var transportContext: TransportContext = _ private[this] var server: TransportServer = _ @@ -53,26 +66,37 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage var serverBootstrap: Option[TransportServerBootstrap] = None var clientBootstrap: Option[TransportClientBootstrap] = None if (authEnabled) { - serverBootstrap = Some(new SaslServerBootstrap(transportConf, securityManager)) - clientBootstrap = Some(new SaslClientBootstrap(transportConf, conf.getAppId, securityManager, - securityManager.isSaslEncryptionEnabled())) + serverBootstrap = Some(new AuthServerBootstrap(transportConf, securityManager)) + clientBootstrap = Some(new AuthClientBootstrap(transportConf, conf.getAppId, securityManager)) } transportContext = new TransportContext(transportConf, rpcHandler) clientFactory = transportContext.createClientFactory(clientBootstrap.toSeq.asJava) server = createServer(serverBootstrap.toList) appId = conf.getAppId - logInfo("Server created on " + server.getPort) + logInfo(s"Server created on ${hostName}:${server.getPort}") } /** Creates and binds the TransportServer, possibly trying multiple ports. */ private def createServer(bootstraps: List[TransportServerBootstrap]): TransportServer = { def startService(port: Int): (TransportServer, Int) = { - val server = transportContext.createServer(port, bootstraps.asJava) + val server = transportContext.createServer(bindAddress, port, bootstraps.asJava) (server, server.getPort) } - val portToTry = conf.getInt("spark.blockManager.port", 0) - Utils.startServiceOnPort(portToTry, startService, conf, getClass.getName)._1 + Utils.startServiceOnPort(_port, startService, conf, getClass.getName)._1 + } + + override def shuffleMetrics(): MetricSet = { + require(server != null && clientFactory != null, "NettyBlockTransferServer is not initialized") + + new MetricSet { + val allMetrics = new JHashMap[String, Metric]() + override def getMetrics: JMap[String, Metric] = { + allMetrics.putAll(clientFactory.getAllMetrics.getMetrics) + allMetrics.putAll(server.getAllMetrics.getMetrics) + allMetrics + } + } } override def fetchBlocks( @@ -80,13 +104,15 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage port: Int, execId: String, blockIds: Array[String], - listener: BlockFetchingListener): Unit = { + listener: BlockFetchingListener, + tempShuffleFileManager: TempShuffleFileManager): Unit = { logTrace(s"Fetch blocks from $host:$port (executor id $execId)") try { val blockFetchStarter = new RetryingBlockFetcher.BlockFetchStarter { override def createAndStart(blockIds: Array[String], listener: BlockFetchingListener) { val client = clientFactory.createClient(host, port) - new OneForOneBlockFetcher(client, appId, execId, blockIds.toArray, listener).start() + new OneForOneBlockFetcher(client, appId, execId, blockIds, listener, + transportConf, tempShuffleFileManager).start() } } @@ -105,8 +131,6 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage } } - override def hostName: String = Utils.localHostName() - override def port: Int = server.getPort override def uploadBlock( @@ -115,27 +139,21 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage execId: String, blockId: BlockId, blockData: ManagedBuffer, - level: StorageLevel): Future[Unit] = { + level: StorageLevel, + classTag: ClassTag[_]): Future[Unit] = { val result = Promise[Unit]() val client = clientFactory.createClient(hostname, port) - // StorageLevel is serialized as bytes using our JavaSerializer. Everything else is encoded - // using our binary protocol. - val levelBytes = serializer.newInstance().serialize(level).array() + // StorageLevel and ClassTag are serialized as bytes using our JavaSerializer. + // Everything else is encoded using our binary protocol. + val metadata = JavaUtils.bufferToArray(serializer.newInstance().serialize((level, classTag))) // Convert or copy nio buffer into array in order to serialize it. - val nioBuffer = blockData.nioByteBuffer() - val array = if (nioBuffer.hasArray) { - nioBuffer.array() - } else { - val data = new Array[Byte](nioBuffer.remaining()) - nioBuffer.get(data) - data - } + val array = JavaUtils.bufferToArray(blockData.nioByteBuffer()) - client.sendRpc(new UploadBlock(appId, execId, blockId.toString, levelBytes, array).toByteArray, + client.sendRpc(new UploadBlock(appId, execId, blockId.toString, metadata, array).toByteBuffer, new RpcResponseCallback { - override def onSuccess(response: Array[Byte]): Unit = { + override def onSuccess(response: ByteBuffer): Unit = { logTrace(s"Successfully uploaded block $blockId") result.success((): Unit) } diff --git a/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala b/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala index cef203006d68..25f7bcb9801b 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala @@ -17,8 +17,10 @@ package org.apache.spark.network.netty +import scala.collection.JavaConverters._ + import org.apache.spark.SparkConf -import org.apache.spark.network.util.{TransportConf, ConfigProvider} +import org.apache.spark.network.util.{ConfigProvider, TransportConf} /** * Provides a utility for transforming from a SparkConf inside a Spark JVM (e.g., Executor, @@ -40,24 +42,28 @@ object SparkTransportConf { /** * Utility for creating a [[TransportConf]] from a [[SparkConf]]. + * @param _conf the [[SparkConf]] + * @param module the module name * @param numUsableCores if nonzero, this will restrict the server and client threads to only * use the given number of cores, rather than all of the machine's cores. * This restriction will only occur if these properties are not already set. */ - def fromSparkConf(_conf: SparkConf, numUsableCores: Int = 0): TransportConf = { + def fromSparkConf(_conf: SparkConf, module: String, numUsableCores: Int = 0): TransportConf = { val conf = _conf.clone // Specify thread configuration based on our JVM's allocation of cores (rather than necessarily // assuming we have all the machine's cores). // NB: Only set if serverThreads/clientThreads not already set. val numThreads = defaultNumThreads(numUsableCores) - conf.set("spark.shuffle.io.serverThreads", - conf.get("spark.shuffle.io.serverThreads", numThreads.toString)) - conf.set("spark.shuffle.io.clientThreads", - conf.get("spark.shuffle.io.clientThreads", numThreads.toString)) + conf.setIfMissing(s"spark.$module.io.serverThreads", numThreads.toString) + conf.setIfMissing(s"spark.$module.io.clientThreads", numThreads.toString) - new TransportConf(new ConfigProvider { + new TransportConf(module, new ConfigProvider { override def get(name: String): String = conf.get(name) + override def get(name: String, defaultValue: String): String = conf.get(name, defaultValue) + override def getAll(): java.lang.Iterable[java.util.Map.Entry[String, String]] = { + conf.getAll.toMap.asJava.entrySet() + } }) } diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 7515aad09db7..8058a4d5dbde 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -17,6 +17,8 @@ package org.apache +import java.util.Properties + /** * Core Spark functionality. [[org.apache.spark.SparkContext]] serves as the main entry point to * Spark, while [[org.apache.spark.rdd.RDD]] is the data type representing a distributed collection, @@ -40,8 +42,57 @@ package org.apache * Developer API are intended for advanced users want to extend Spark through lower * level interfaces. These are subject to changes or removal in minor releases. */ - package object spark { - // For package docs only - val SPARK_VERSION = "1.6.0-SNAPSHOT" + + private object SparkBuildInfo { + + val ( + spark_version: String, + spark_branch: String, + spark_revision: String, + spark_build_user: String, + spark_repo_url: String, + spark_build_date: String) = { + + val resourceStream = Thread.currentThread().getContextClassLoader. + getResourceAsStream("spark-version-info.properties") + if (resourceStream == null) { + throw new SparkException("Could not find spark-version-info.properties") + } + + try { + val unknownProp = "" + val props = new Properties() + props.load(resourceStream) + ( + props.getProperty("version", unknownProp), + props.getProperty("branch", unknownProp), + props.getProperty("revision", unknownProp), + props.getProperty("user", unknownProp), + props.getProperty("url", unknownProp), + props.getProperty("date", unknownProp) + ) + } catch { + case e: Exception => + throw new SparkException("Error loading properties from spark-version-info.properties", e) + } finally { + if (resourceStream != null) { + try { + resourceStream.close() + } catch { + case e: Exception => + throw new SparkException("Error closing spark build info resource stream", e) + } + } + } + } + } + + val SPARK_VERSION = SparkBuildInfo.spark_version + val SPARK_BRANCH = SparkBuildInfo.spark_branch + val SPARK_REVISION = SparkBuildInfo.spark_revision + val SPARK_BUILD_USER = SparkBuildInfo.spark_build_user + val SPARK_REPO_URL = SparkBuildInfo.spark_repo_url + val SPARK_BUILD_DATE = SparkBuildInfo.spark_build_date } + diff --git a/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala index d25452daf760..b089bbd7e972 100644 --- a/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala +++ b/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala @@ -38,7 +38,7 @@ private[spark] class ApproximateActionListener[T, U, R]( extends JobListener { val startTime = System.currentTimeMillis() - val totalTasks = rdd.partitions.size + val totalTasks = rdd.partitions.length var finishedTasks = 0 var failure: Option[Exception] = None // Set if the job has failed (permanently) var resultObject: Option[PartialResult[R]] = None // Set if we've already returned a PartialResult diff --git a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala index 48b943415317..8f579c5a3033 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -21,5 +21,22 @@ package org.apache.spark.partial * A Double value with error bars and associated confidence. */ class BoundedDouble(val mean: Double, val confidence: Double, val low: Double, val high: Double) { + override def toString(): String = "[%.3f, %.3f]".format(low, high) + + override def hashCode: Int = + this.mean.hashCode ^ this.confidence.hashCode ^ this.low.hashCode ^ this.high.hashCode + + /** + * @note Consistent with Double, any NaN value will make equality false + */ + override def equals(that: Any): Boolean = + that match { + case that: BoundedDouble => + this.mean == that.mean && + this.confidence == that.confidence && + this.low == that.low && + this.high == that.high + case _ => false + } } diff --git a/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala index 637492a97551..cbee13687101 100644 --- a/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala @@ -17,21 +17,18 @@ package org.apache.spark.partial -import org.apache.commons.math3.distribution.NormalDistribution +import org.apache.commons.math3.distribution.PoissonDistribution /** * An ApproximateEvaluator for counts. - * - * TODO: There's currently a lot of shared code between this and GroupedCountEvaluator. It might - * be best to make this a special case of GroupedCountEvaluator with one group. */ private[spark] class CountEvaluator(totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[Long, BoundedDouble] { - var outputsMerged = 0 - var sum: Long = 0 + private var outputsMerged = 0 + private var sum: Long = 0 - override def merge(outputId: Int, taskResult: Long) { + override def merge(outputId: Int, taskResult: Long): Unit = { outputsMerged += 1 sum += taskResult } @@ -39,18 +36,29 @@ private[spark] class CountEvaluator(totalOutputs: Int, confidence: Double) override def currentResult(): BoundedDouble = { if (outputsMerged == totalOutputs) { new BoundedDouble(sum, 1.0, sum, sum) - } else if (outputsMerged == 0) { - new BoundedDouble(0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity) + } else if (outputsMerged == 0 || sum == 0) { + new BoundedDouble(0, 0.0, 0.0, Double.PositiveInfinity) } else { val p = outputsMerged.toDouble / totalOutputs - val mean = (sum + 1 - p) / p - val variance = (sum + 1) * (1 - p) / (p * p) - val stdev = math.sqrt(variance) - val confFactor = new NormalDistribution(). - inverseCumulativeProbability(1 - (1 - confidence) / 2) - val low = mean - confFactor * stdev - val high = mean + confFactor * stdev - new BoundedDouble(mean, confidence, low, high) + CountEvaluator.bound(confidence, sum, p) } } } + +private[partial] object CountEvaluator { + + def bound(confidence: Double, sum: Long, p: Double): BoundedDouble = { + // "sum" elements have been observed having scanned a fraction + // p of the data. This suggests data is counted at a rate of sum / p across the whole data + // set. The total expected count from the rest is distributed as + // (1-p) Poisson(sum / p) = Poisson(sum*(1-p)/p) + val dist = new PoissonDistribution(sum * (1 - p) / p) + // Not quite symmetric; calculate interval straight from discrete distribution + val low = dist.inverseCumulativeProbability((1 - confidence) / 2) + val high = dist.inverseCumulativeProbability((1 + confidence) / 2) + // Add 'sum' to each because distribution is just of remaining count, not observed + new BoundedDouble(sum + dist.getNumericalMean, confidence, sum + low, sum + high) + } + + +} diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala index 5afce75680f9..d2b4187df5d5 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala @@ -17,15 +17,10 @@ package org.apache.spark.partial -import java.util.{HashMap => JHashMap} - -import scala.collection.JavaConverters._ import scala.collection.Map import scala.collection.mutable.HashMap import scala.reflect.ClassTag -import org.apache.commons.math3.distribution.NormalDistribution - import org.apache.spark.util.collection.OpenHashMap /** @@ -34,10 +29,10 @@ import org.apache.spark.util.collection.OpenHashMap private[spark] class GroupedCountEvaluator[T : ClassTag](totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[OpenHashMap[T, Long], Map[T, BoundedDouble]] { - var outputsMerged = 0 - var sums = new OpenHashMap[T, Long]() // Sum of counts for each key + private var outputsMerged = 0 + private val sums = new OpenHashMap[T, Long]() // Sum of counts for each key - override def merge(outputId: Int, taskResult: OpenHashMap[T, Long]) { + override def merge(outputId: Int, taskResult: OpenHashMap[T, Long]): Unit = { outputsMerged += 1 taskResult.foreach { case (key, value) => sums.changeValue(key, value, _ + value) @@ -46,27 +41,12 @@ private[spark] class GroupedCountEvaluator[T : ClassTag](totalOutputs: Int, conf override def currentResult(): Map[T, BoundedDouble] = { if (outputsMerged == totalOutputs) { - val result = new JHashMap[T, BoundedDouble](sums.size) - sums.foreach { case (key, sum) => - result.put(key, new BoundedDouble(sum, 1.0, sum, sum)) - } - result.asScala + sums.map { case (key, sum) => (key, new BoundedDouble(sum, 1.0, sum, sum)) }.toMap } else if (outputsMerged == 0) { new HashMap[T, BoundedDouble] } else { val p = outputsMerged.toDouble / totalOutputs - val confFactor = new NormalDistribution(). - inverseCumulativeProbability(1 - (1 - confidence) / 2) - val result = new JHashMap[T, BoundedDouble](sums.size) - sums.foreach { case (key, sum) => - val mean = (sum + 1 - p) / p - val variance = (sum + 1) * (1 - p) / (p * p) - val stdev = math.sqrt(variance) - val low = mean - confFactor * stdev - val high = mean + confFactor * stdev - result.put(key, new BoundedDouble(mean, confidence, low, high)) - } - result.asScala + sums.map { case (key, sum) => (key, CountEvaluator.bound(confidence, sum, p)) }.toMap } } } diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala deleted file mode 100644 index a16404068480..000000000000 --- a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala +++ /dev/null @@ -1,80 +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.partial - -import java.util.{HashMap => JHashMap} - -import scala.collection.JavaConverters._ -import scala.collection.Map -import scala.collection.mutable.HashMap - -import org.apache.spark.util.StatCounter - -/** - * An ApproximateEvaluator for means by key. Returns a map of key to confidence interval. - */ -private[spark] class GroupedMeanEvaluator[T](totalOutputs: Int, confidence: Double) - extends ApproximateEvaluator[JHashMap[T, StatCounter], Map[T, BoundedDouble]] { - - var outputsMerged = 0 - var sums = new JHashMap[T, StatCounter] // Sum of counts for each key - - override def merge(outputId: Int, taskResult: JHashMap[T, StatCounter]) { - outputsMerged += 1 - val iter = taskResult.entrySet.iterator() - while (iter.hasNext) { - val entry = iter.next() - val old = sums.get(entry.getKey) - if (old != null) { - old.merge(entry.getValue) - } else { - sums.put(entry.getKey, entry.getValue) - } - } - } - - override def currentResult(): Map[T, BoundedDouble] = { - if (outputsMerged == totalOutputs) { - val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.entrySet.iterator() - while (iter.hasNext) { - val entry = iter.next() - val mean = entry.getValue.mean - result.put(entry.getKey, new BoundedDouble(mean, 1.0, mean, mean)) - } - result.asScala - } else if (outputsMerged == 0) { - new HashMap[T, BoundedDouble] - } else { - val studentTCacher = new StudentTCacher(confidence) - val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.entrySet.iterator() - while (iter.hasNext) { - val entry = iter.next() - val counter = entry.getValue - val mean = counter.mean - val stdev = math.sqrt(counter.sampleVariance / counter.count) - val confFactor = studentTCacher.get(counter.count) - val low = mean - confFactor * stdev - val high = mean + confFactor * stdev - result.put(entry.getKey, new BoundedDouble(mean, confidence, low, high)) - } - result.asScala - } - } -} diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala deleted file mode 100644 index 54a1beab3514..000000000000 --- a/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala +++ /dev/null @@ -1,88 +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.partial - -import java.util.{HashMap => JHashMap} - -import scala.collection.JavaConverters._ -import scala.collection.Map -import scala.collection.mutable.HashMap - -import org.apache.spark.util.StatCounter - -/** - * An ApproximateEvaluator for sums by key. Returns a map of key to confidence interval. - */ -private[spark] class GroupedSumEvaluator[T](totalOutputs: Int, confidence: Double) - extends ApproximateEvaluator[JHashMap[T, StatCounter], Map[T, BoundedDouble]] { - - var outputsMerged = 0 - var sums = new JHashMap[T, StatCounter] // Sum of counts for each key - - override def merge(outputId: Int, taskResult: JHashMap[T, StatCounter]) { - outputsMerged += 1 - val iter = taskResult.entrySet.iterator() - while (iter.hasNext) { - val entry = iter.next() - val old = sums.get(entry.getKey) - if (old != null) { - old.merge(entry.getValue) - } else { - sums.put(entry.getKey, entry.getValue) - } - } - } - - override def currentResult(): Map[T, BoundedDouble] = { - if (outputsMerged == totalOutputs) { - val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.entrySet.iterator() - while (iter.hasNext) { - val entry = iter.next() - val sum = entry.getValue.sum - result.put(entry.getKey, new BoundedDouble(sum, 1.0, sum, sum)) - } - result.asScala - } else if (outputsMerged == 0) { - new HashMap[T, BoundedDouble] - } else { - val p = outputsMerged.toDouble / totalOutputs - val studentTCacher = new StudentTCacher(confidence) - val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.entrySet.iterator() - while (iter.hasNext) { - val entry = iter.next() - val counter = entry.getValue - val meanEstimate = counter.mean - val meanVar = counter.sampleVariance / counter.count - val countEstimate = (counter.count + 1 - p) / p - val countVar = (counter.count + 1) * (1 - p) / (p * p) - val sumEstimate = meanEstimate * countEstimate - val sumVar = (meanEstimate * meanEstimate * countVar) + - (countEstimate * countEstimate * meanVar) + - (meanVar * countVar) - val sumStdev = math.sqrt(sumVar) - val confFactor = studentTCacher.get(counter.count) - val low = sumEstimate - confFactor * sumStdev - val high = sumEstimate + confFactor * sumStdev - result.put(entry.getKey, new BoundedDouble(sumEstimate, confidence, low, high)) - } - result.asScala - } - } -} diff --git a/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala index 787a21a61fdc..3fb2d30a800b 100644 --- a/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala @@ -27,10 +27,10 @@ import org.apache.spark.util.StatCounter private[spark] class MeanEvaluator(totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[StatCounter, BoundedDouble] { - var outputsMerged = 0 - var counter = new StatCounter + private var outputsMerged = 0 + private val counter = new StatCounter() - override def merge(outputId: Int, taskResult: StatCounter) { + override def merge(outputId: Int, taskResult: StatCounter): Unit = { outputsMerged += 1 counter.merge(taskResult) } @@ -38,19 +38,24 @@ private[spark] class MeanEvaluator(totalOutputs: Int, confidence: Double) override def currentResult(): BoundedDouble = { if (outputsMerged == totalOutputs) { new BoundedDouble(counter.mean, 1.0, counter.mean, counter.mean) - } else if (outputsMerged == 0) { + } else if (outputsMerged == 0 || counter.count == 0) { new BoundedDouble(0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity) + } else if (counter.count == 1) { + new BoundedDouble(counter.mean, confidence, Double.NegativeInfinity, Double.PositiveInfinity) } else { val mean = counter.mean val stdev = math.sqrt(counter.sampleVariance / counter.count) - val confFactor = { - if (counter.count > 100) { - new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2) + val confFactor = if (counter.count > 100) { + // For large n, the normal distribution is a good approximation to t-distribution + new NormalDistribution().inverseCumulativeProbability((1 + confidence) / 2) } else { + // t-distribution describes distribution of actual population mean + // note that if this goes to 0, TDistribution will throw an exception. + // Hence special casing 1 above. val degreesOfFreedom = (counter.count - 1).toInt - new TDistribution(degreesOfFreedom).inverseCumulativeProbability(1 - (1 - confidence) / 2) + new TDistribution(degreesOfFreedom).inverseCumulativeProbability((1 + confidence) / 2) } - } + // Symmetric, so confidence interval is symmetric about mean of distribution val low = mean - confFactor * stdev val high = mean + confFactor * stdev new BoundedDouble(mean, confidence, low, high) diff --git a/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala b/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala deleted file mode 100644 index 828bf96c2c0b..000000000000 --- a/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala +++ /dev/null @@ -1,46 +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.partial - -import org.apache.commons.math3.distribution.{TDistribution, NormalDistribution} - -/** - * A utility class for caching Student's T distribution values for a given confidence level - * and various sample sizes. This is used by the MeanEvaluator to efficiently calculate - * confidence intervals for many keys. - */ -private[spark] class StudentTCacher(confidence: Double) { - - val NORMAL_APPROX_SAMPLE_SIZE = 100 // For samples bigger than this, use Gaussian approximation - - val normalApprox = new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2) - val cache = Array.fill[Double](NORMAL_APPROX_SAMPLE_SIZE)(-1.0) - - def get(sampleSize: Long): Double = { - if (sampleSize >= NORMAL_APPROX_SAMPLE_SIZE) { - normalApprox - } else { - val size = sampleSize.toInt - if (cache(size) < 0) { - val tDist = new TDistribution(size - 1) - cache(size) = tDist.inverseCumulativeProbability(1 - (1 - confidence) / 2) - } - cache(size) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala index 1753c2561b67..1988052b733e 100644 --- a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala @@ -17,7 +17,7 @@ package org.apache.spark.partial -import org.apache.commons.math3.distribution.{TDistribution, NormalDistribution} +import org.apache.commons.math3.distribution.{NormalDistribution, TDistribution} import org.apache.spark.util.StatCounter @@ -29,10 +29,11 @@ import org.apache.spark.util.StatCounter private[spark] class SumEvaluator(totalOutputs: Int, confidence: Double) extends ApproximateEvaluator[StatCounter, BoundedDouble] { - var outputsMerged = 0 - var counter = new StatCounter + // modified in merge + private var outputsMerged = 0 + private val counter = new StatCounter() - override def merge(outputId: Int, taskResult: StatCounter) { + override def merge(outputId: Int, taskResult: StatCounter): Unit = { outputsMerged += 1 counter.merge(taskResult) } @@ -40,30 +41,50 @@ private[spark] class SumEvaluator(totalOutputs: Int, confidence: Double) override def currentResult(): BoundedDouble = { if (outputsMerged == totalOutputs) { new BoundedDouble(counter.sum, 1.0, counter.sum, counter.sum) - } else if (outputsMerged == 0) { + } else if (outputsMerged == 0 || counter.count == 0) { new BoundedDouble(0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity) } else { val p = outputsMerged.toDouble / totalOutputs + // Expected value of unobserved is presumed equal to that of the observed data val meanEstimate = counter.mean - val meanVar = counter.sampleVariance / counter.count - val countEstimate = (counter.count + 1 - p) / p - val countVar = (counter.count + 1) * (1 - p) / (p * p) + // Expected size of rest of the data is proportional + val countEstimate = counter.count * (1 - p) / p + // Expected sum is simply their product val sumEstimate = meanEstimate * countEstimate - val sumVar = (meanEstimate * meanEstimate * countVar) + - (countEstimate * countEstimate * meanVar) + - (meanVar * countVar) - val sumStdev = math.sqrt(sumVar) - val confFactor = { - if (counter.count > 100) { - new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2) + + // Variance of unobserved data is presumed equal to that of the observed data + val meanVar = counter.sampleVariance / counter.count + + // branch at this point because count == 1 implies counter.sampleVariance == Nan + // and we don't want to ever return a bound of NaN + if (meanVar.isNaN || counter.count == 1) { + // add sum because estimate is of unobserved data sum + new BoundedDouble( + counter.sum + sumEstimate, confidence, Double.NegativeInfinity, Double.PositiveInfinity) + } else { + // See CountEvaluator. Variance of population count here follows from negative binomial + val countVar = counter.count * (1 - p) / (p * p) + // Var(Sum) = Var(Mean*Count) = + // [E(Mean)]^2 * Var(Count) + [E(Count)]^2 * Var(Mean) + Var(Mean) * Var(Count) + val sumVar = (meanEstimate * meanEstimate * countVar) + + (countEstimate * countEstimate * meanVar) + + (meanVar * countVar) + val sumStdev = math.sqrt(sumVar) + val confFactor = if (counter.count > 100) { + new NormalDistribution().inverseCumulativeProbability((1 + confidence) / 2) } else { + // note that if this goes to 0, TDistribution will throw an exception. + // Hence special casing 1 above. val degreesOfFreedom = (counter.count - 1).toInt - new TDistribution(degreesOfFreedom).inverseCumulativeProbability(1 - (1 - confidence) / 2) + new TDistribution(degreesOfFreedom).inverseCumulativeProbability((1 + confidence) / 2) } + // Symmetric, so confidence interval is symmetric about mean of distribution + val low = sumEstimate - confFactor * sumStdev + val high = sumEstimate + confFactor * sumStdev + // add sum because estimate is of unobserved data sum + new BoundedDouble( + counter.sum + sumEstimate, confidence, counter.sum + low, counter.sum + high) } - val low = sumEstimate - confFactor * sumStdev - val high = sumEstimate + confFactor * sumStdev - new BoundedDouble(sumEstimate, confidence, low, high) } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index ca1eb1f4e4a9..c9ed12f4e1bd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -19,13 +19,13 @@ package org.apache.spark.rdd import java.util.concurrent.atomic.AtomicLong -import org.apache.spark.util.ThreadUtils - import scala.collection.mutable.ArrayBuffer -import scala.concurrent.ExecutionContext +import scala.concurrent.{ExecutionContext, Future} import scala.reflect.ClassTag -import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} +import org.apache.spark.{ComplexFutureAction, FutureAction, JobSubmitter} +import org.apache.spark.internal.Logging +import org.apache.spark.util.ThreadUtils /** * A set of asynchronous RDD actions available through an implicit conversion. @@ -65,18 +65,26 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi * Returns a future for retrieving the first num elements of the RDD. */ def takeAsync(num: Int): FutureAction[Seq[T]] = self.withScope { - val f = new ComplexFutureAction[Seq[T]] - - f.run { - // This is a blocking action so we should use "AsyncRDDActions.futureExecutionContext" which - // is a cached thread pool. - val results = new ArrayBuffer[T](num) - val totalParts = self.partitions.length - var partsScanned = 0 - while (results.size < num && partsScanned < totalParts) { + val callSite = self.context.getCallSite + val localProperties = self.context.getLocalProperties + // Cached thread pool to handle aggregation of subtasks. + implicit val executionContext = AsyncRDDActions.futureExecutionContext + val results = new ArrayBuffer[T] + val totalParts = self.partitions.length + + /* + Recursively triggers jobs to scan partitions until either the requested + number of elements are retrieved, or the partitions to scan are exhausted. + This implementation is non-blocking, asynchronously handling the + results of each job and triggering the next job using callbacks on futures. + */ + def continue(partsScanned: Int)(implicit jobSubmitter: JobSubmitter): Future[Seq[T]] = + if (results.size >= num || partsScanned >= totalParts) { + Future.successful(results.toSeq) + } else { // The number of partitions to try in this iteration. It is ok for this number to be // greater than totalParts because we actually cap it at totalParts in runJob. - var numPartsToTry = 1 + var numPartsToTry = 1L if (partsScanned > 0) { // If we didn't find any rows after the previous iteration, quadruple and retry. // Otherwise, interpolate the number of partitions we need to try, but overestimate it @@ -92,22 +100,23 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi } val left = num - results.size - val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts) + val p = partsScanned.until(math.min(partsScanned + numPartsToTry, totalParts).toInt) val buf = new Array[Array[T]](p.size) - f.runJob(self, + self.context.setCallSite(callSite) + self.context.setLocalProperties(localProperties) + val job = jobSubmitter.submitJob(self, (it: Iterator[T]) => it.take(left).toArray, p, (index: Int, data: Array[T]) => buf(index) = data, Unit) - - buf.foreach(results ++= _.take(num - results.size)) - partsScanned += numPartsToTry + job.flatMap { _ => + buf.foreach(results ++= _.take(num - results.size)) + continue(partsScanned + p.size) + } } - results.toSeq - }(AsyncRDDActions.futureExecutionContext) - f + new ComplexFutureAction[Seq[T]](continue(0)(_)) } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index aedced7408cd..a14bad47dfe1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -17,14 +17,17 @@ package org.apache.spark.rdd -import org.apache.hadoop.conf.{ Configurable, Configuration } +import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat +import org.apache.hadoop.mapreduce.task.JobContextImpl + +import org.apache.spark.{Partition, SparkContext} import org.apache.spark.input.StreamFileInputFormat -import org.apache.spark.{ Partition, SparkContext } private[spark] class BinaryFileRDD[T]( - sc: SparkContext, + @transient private val sc: SparkContext, inputFormatClass: Class[_ <: StreamFileInputFormat[T]], keyClass: Class[String], valueClass: Class[T], @@ -33,15 +36,19 @@ private[spark] class BinaryFileRDD[T]( extends NewHadoopRDD[String, T](sc, inputFormatClass, keyClass, valueClass, conf) { override def getPartitions: Array[Partition] = { - val inputFormat = inputFormatClass.newInstance val conf = getConf + // setMinPartitions below will call FileInputFormat.listStatus(), which can be quite slow when + // traversing a large number of directories and files. Parallelize it. + conf.setIfUnset(FileInputFormat.LIST_STATUS_NUM_THREADS, + Runtime.getRuntime.availableProcessors().toString) + val inputFormat = inputFormatClass.newInstance inputFormat match { case configurable: Configurable => configurable.setConf(conf) case _ => } - val jobContext = newJobContext(conf, jobId) - inputFormat.setMinPartitions(jobContext, minPartitions) + val jobContext = new JobContextImpl(conf, jobId) + inputFormat.setMinPartitions(sc, jobContext, minPartitions) val rawSplits = inputFormat.getSplits(jobContext).toArray val result = new Array[Partition](rawSplits.size) for (i <- 0 until rawSplits.size) { diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index fc1710fbad0a..4e036c2ed49b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -21,7 +21,6 @@ import scala.reflect.ClassTag import org.apache.spark._ import org.apache.spark.storage.{BlockId, BlockManager} -import scala.Some private[spark] class BlockRDDPartition(val blockId: BlockId, idx: Int) extends Partition { val index = idx @@ -36,19 +35,19 @@ class BlockRDD[T: ClassTag](sc: SparkContext, @transient val blockIds: Array[Blo override def getPartitions: Array[Partition] = { assertValid() - (0 until blockIds.length).map(i => { + (0 until blockIds.length).map { i => new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] - }).toArray + }.toArray } override def compute(split: Partition, context: TaskContext): Iterator[T] = { assertValid() val blockManager = SparkEnv.get.blockManager val blockId = split.asInstanceOf[BlockRDDPartition].blockId - blockManager.get(blockId) match { + blockManager.get[T](blockId) match { case Some(block) => block.data.asInstanceOf[Iterator[T]] case None => - throw new Exception("Could not compute split, block " + blockId + " not found") + throw new Exception(s"Could not compute split, block $blockId of RDD $id not found") } } diff --git a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala index 18e8cddbc40d..57108dcedcf0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala @@ -50,7 +50,7 @@ class CartesianRDD[T: ClassTag, U: ClassTag]( sc: SparkContext, var rdd1 : RDD[T], var rdd2 : RDD[U]) - extends RDD[Pair[T, U]](sc, Nil) + extends RDD[(T, U)](sc, Nil) with Serializable { val numPartitionsInRdd2 = rdd2.partitions.length diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 935c3babd8ea..4574c3724962 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -17,23 +17,24 @@ package org.apache.spark.rdd -import scala.language.existentials - import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer +import scala.language.existentials import scala.reflect.ClassTag import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.util.collection.{CompactBuffer, ExternalAppendOnlyMap} -import org.apache.spark.util.Utils import org.apache.spark.serializer.Serializer +import org.apache.spark.util.Utils +import org.apache.spark.util.collection.{CompactBuffer, ExternalAppendOnlyMap} -/** The references to rdd and splitIndex are transient because redundant information is stored - * in the CoGroupedRDD object. Because CoGroupedRDD is serialized separately from - * CoGroupPartition, if rdd and splitIndex aren't transient, they'll be included twice in the - * task closure. */ +/** + * The references to rdd and splitIndex are transient because redundant information is stored + * in the CoGroupedRDD object. Because CoGroupedRDD is serialized separately from + * CoGroupPartition, if rdd and splitIndex aren't transient, they'll be included twice in the + * task closure. + */ private[spark] case class NarrowCoGroupSplitDep( @transient rdd: RDD[_], @transient splitIndex: Int, @@ -57,22 +58,22 @@ private[spark] case class NarrowCoGroupSplitDep( * narrowDeps should always be equal to the number of parents. */ private[spark] class CoGroupPartition( - idx: Int, val narrowDeps: Array[Option[NarrowCoGroupSplitDep]]) + override val index: Int, val narrowDeps: Array[Option[NarrowCoGroupSplitDep]]) extends Partition with Serializable { - override val index: Int = idx - override def hashCode(): Int = idx + override def hashCode(): Int = index + override def equals(other: Any): Boolean = super.equals(other) } /** * :: DeveloperApi :: - * A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a + * An RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a * tuple with the list of values for that key. * - * Note: This is an internal API. We recommend users use RDD.cogroup(...) instead of - * instantiating this directly. - * @param rdds parent RDDs. * @param part partitioner used to partition the shuffle output + * + * @note This is an internal API. We recommend users use RDD.cogroup(...) instead of + * instantiating this directly. */ @DeveloperApi class CoGroupedRDD[K: ClassTag]( @@ -87,11 +88,11 @@ class CoGroupedRDD[K: ClassTag]( private type CoGroupValue = (Any, Int) // Int is dependency number private type CoGroupCombiner = Array[CoGroup] - private var serializer: Option[Serializer] = None + private var serializer: Serializer = SparkEnv.get.serializer /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ def setSerializer(serializer: Serializer): CoGroupedRDD[K] = { - this.serializer = Option(serializer) + this.serializer = serializer this } @@ -154,8 +155,7 @@ class CoGroupedRDD[K: ClassTag]( } context.taskMetrics().incMemoryBytesSpilled(map.memoryBytesSpilled) context.taskMetrics().incDiskBytesSpilled(map.diskBytesSpilled) - context.internalMetricsToAccumulators( - InternalAccumulator.PEAK_EXECUTION_MEMORY).add(map.peakMemoryUsedBytes) + context.taskMetrics().incPeakExecutionMemory(map.peakMemoryUsedBytes) new InterruptibleIterator(context, map.iterator.asInstanceOf[Iterator[(K, Array[Iterable[_]])]]) } diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 90d9735cb3f6..10451a324b0f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -70,23 +70,27 @@ private[spark] case class CoalescedRDDPartition( * parent partitions * @param prev RDD to be coalesced * @param maxPartitions number of desired partitions in the coalesced RDD (must be positive) - * @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance + * @param partitionCoalescer [[PartitionCoalescer]] implementation to use for coalescing */ private[spark] class CoalescedRDD[T: ClassTag]( @transient var prev: RDD[T], maxPartitions: Int, - balanceSlack: Double = 0.10) + partitionCoalescer: Option[PartitionCoalescer] = None) extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies require(maxPartitions > 0 || maxPartitions == prev.partitions.length, s"Number of partitions ($maxPartitions) must be positive.") + if (partitionCoalescer.isDefined) { + require(partitionCoalescer.get.isInstanceOf[Serializable], + "The partition coalescer passed in must be serializable.") + } override def getPartitions: Array[Partition] = { - val pc = new PartitionCoalescer(maxPartitions, prev, balanceSlack) + val pc = partitionCoalescer.getOrElse(new DefaultPartitionCoalescer()) - pc.run().zipWithIndex.map { + pc.coalesce(maxPartitions, prev).zipWithIndex.map { case (pg, i) => - val ids = pg.arr.map(_.index).toArray + val ids = pg.partitions.map(_.index).toArray new CoalescedRDDPartition(i, prev, ids, pg.prefLoc) } } @@ -144,15 +148,15 @@ private[spark] class CoalescedRDD[T: ClassTag]( * desired partitions is greater than the number of preferred machines (can happen), it needs to * start picking duplicate preferred machines. This is determined using coupon collector estimation * (2n log(n)). The load balancing is done using power-of-two randomized bins-balls with one twist: - * it tries to also achieve locality. This is done by allowing a slack (balanceSlack) between two - * bins. If two bins are within the slack in terms of balance, the algorithm will assign partitions - * according to locality. (contact alig for questions) - * + * it tries to also achieve locality. This is done by allowing a slack (balanceSlack, where + * 1.0 is all locality, 0 is all balance) between two bins. If two bins are within the slack + * in terms of balance, the algorithm will assign partitions according to locality. + * (contact alig for questions) */ -private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { - - def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size +private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) + extends PartitionCoalescer { + def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.numPartitions < o2.numPartitions def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = if (o1 == None) false else if (o2 == None) true else compare(o1.get, o2.get) @@ -167,47 +171,43 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: // hash used for the first maxPartitions (to avoid duplicates) val initialHash = mutable.Set[Partition]() - // determines the tradeoff between load-balancing the partitions sizes and their locality - // e.g. balanceSlack=0.10 means that it allows up to 10% imbalance in favor of locality - val slack = (balanceSlack * prev.partitions.length).toInt - var noLocality = true // if true if no preferredLocations exists for parent RDD // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) - def currPrefLocs(part: Partition): Seq[String] = { + def currPrefLocs(part: Partition, prev: RDD[_]): Seq[String] = { prev.context.getPreferredLocs(prev, part.index).map(tl => tl.host) } - // this class just keeps iterating and rotating infinitely over the partitions of the RDD - // next() returns the next preferred machine that a partition is replicated on - // the rotator first goes through the first replica copy of each partition, then second, third - // the iterators return type is a tuple: (replicaString, partition) - class LocationIterator(prev: RDD[_]) extends Iterator[(String, Partition)] { - - var it: Iterator[(String, Partition)] = resetIterator() - - override val isEmpty = !it.hasNext - - // initializes/resets to start iterating from the beginning - def resetIterator(): Iterator[(String, Partition)] = { - val iterators = (0 to 2).map( x => - prev.partitions.iterator.flatMap(p => { - if (currPrefLocs(p).size > x) Some((currPrefLocs(p)(x), p)) else None - } ) + class PartitionLocations(prev: RDD[_]) { + + // contains all the partitions from the previous RDD that don't have preferred locations + val partsWithoutLocs = ArrayBuffer[Partition]() + // contains all the partitions from the previous RDD that have preferred locations + val partsWithLocs = ArrayBuffer[(String, Partition)]() + + getAllPrefLocs(prev) + + // gets all the preferred locations of the previous RDD and splits them into partitions + // with preferred locations and ones without + def getAllPrefLocs(prev: RDD[_]): Unit = { + val tmpPartsWithLocs = mutable.LinkedHashMap[Partition, Seq[String]]() + // first get the locations for each partition, only do this once since it can be expensive + prev.partitions.foreach(p => { + val locs = currPrefLocs(p, prev) + if (locs.nonEmpty) { + tmpPartsWithLocs.put(p, locs) + } else { + partsWithoutLocs += p + } + } ) - iterators.reduceLeft((x, y) => x ++ y) - } - - // hasNext() is false iff there are no preferredLocations for any of the partitions of the RDD - override def hasNext: Boolean = { !isEmpty } - - // return the next preferredLocation of some partition of the RDD - override def next(): (String, Partition) = { - if (it.hasNext) { - it.next() - } else { - it = resetIterator() // ran out of preferred locations, reset and rotate to the beginning - it.next() + // convert it into an array of host to partition + for (x <- 0 to 2) { + tmpPartsWithLocs.foreach { parts => + val p = parts._1 + val locs = parts._2 + if (locs.size > x) partsWithLocs += ((locs(x), p)) + } } } } @@ -215,8 +215,9 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: /** * Sorts and gets the least element of the list associated with key in groupHash * The returned PartitionGroup is the least loaded of all groups that represent the machine "key" + * * @param key string representing a partitioned group on preferred machine key - * @return Option of PartitionGroup that has least elements for key + * @return Option of [[PartitionGroup]] that has least elements for key */ def getLeastGroupHash(key: String): Option[PartitionGroup] = { groupHash.get(key).map(_.sortWith(compare).head) @@ -224,78 +225,91 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: def addPartToPGroup(part: Partition, pgroup: PartitionGroup): Boolean = { if (!initialHash.contains(part)) { - pgroup.arr += part // already assign this element + pgroup.partitions += part // already assign this element initialHash += part // needed to avoid assigning partitions to multiple buckets true } else { false } } /** - * Initializes targetLen partition groups and assigns a preferredLocation - * This uses coupon collector to estimate how many preferredLocations it must rotate through - * until it has seen most of the preferred locations (2 * n log(n)) + * Initializes targetLen partition groups. If there are preferred locations, each group + * is assigned a preferredLocation. This uses coupon collector to estimate how many + * preferredLocations it must rotate through until it has seen most of the preferred + * locations (2 * n log(n)) * @param targetLen */ - def setupGroups(targetLen: Int) { - val rotIt = new LocationIterator(prev) - + def setupGroups(targetLen: Int, partitionLocs: PartitionLocations) { // deal with empty case, just create targetLen partition groups with no preferred location - if (!rotIt.hasNext) { - (1 to targetLen).foreach(x => groupArr += PartitionGroup()) + if (partitionLocs.partsWithLocs.isEmpty) { + (1 to targetLen).foreach(x => groupArr += new PartitionGroup()) return } noLocality = false - // number of iterations needed to be certain that we've seen most preferred locations val expectedCoupons2 = 2 * (math.log(targetLen)*targetLen + targetLen + 0.5).toInt var numCreated = 0 var tries = 0 // rotate through until either targetLen unique/distinct preferred locations have been created - // OR we've rotated expectedCoupons2, in which case we have likely seen all preferred locations, - // i.e. likely targetLen >> number of preferred locations (more buckets than there are machines) - while (numCreated < targetLen && tries < expectedCoupons2) { + // OR (we have went through either all partitions OR we've rotated expectedCoupons2 - in + // which case we have likely seen all preferred locations) + val numPartsToLookAt = math.min(expectedCoupons2, partitionLocs.partsWithLocs.length) + while (numCreated < targetLen && tries < numPartsToLookAt) { + val (nxt_replica, nxt_part) = partitionLocs.partsWithLocs(tries) tries += 1 - val (nxt_replica, nxt_part) = rotIt.next() if (!groupHash.contains(nxt_replica)) { - val pgroup = PartitionGroup(nxt_replica) + val pgroup = new PartitionGroup(Some(nxt_replica)) groupArr += pgroup addPartToPGroup(nxt_part, pgroup) groupHash.put(nxt_replica, ArrayBuffer(pgroup)) // list in case we have multiple numCreated += 1 } } - - while (numCreated < targetLen) { // if we don't have enough partition groups, create duplicates - var (nxt_replica, nxt_part) = rotIt.next() - val pgroup = PartitionGroup(nxt_replica) + tries = 0 + // if we don't have enough partition groups, create duplicates + while (numCreated < targetLen) { + val (nxt_replica, nxt_part) = partitionLocs.partsWithLocs(tries) + tries += 1 + val pgroup = new PartitionGroup(Some(nxt_replica)) groupArr += pgroup groupHash.getOrElseUpdate(nxt_replica, ArrayBuffer()) += pgroup - var tries = 0 - while (!addPartToPGroup(nxt_part, pgroup) && tries < targetLen) { // ensure at least one part - nxt_part = rotIt.next()._2 - tries += 1 - } + addPartToPGroup(nxt_part, pgroup) numCreated += 1 + if (tries >= partitionLocs.partsWithLocs.length) tries = 0 } - } /** * Takes a parent RDD partition and decides which of the partition groups to put it in * Takes locality into account, but also uses power of 2 choices to load balance - * It strikes a balance between the two use the balanceSlack variable + * It strikes a balance between the two using the balanceSlack variable * @param p partition (ball to be thrown) + * @param balanceSlack determines the trade-off between load-balancing the partitions sizes and + * their locality. e.g., balanceSlack=0.10 means that it allows up to 10% + * imbalance in favor of locality * @return partition group (bin to be put in) */ - def pickBin(p: Partition): PartitionGroup = { - val pref = currPrefLocs(p).map(getLeastGroupHash(_)).sortWith(compare) // least loaded pref locs + def pickBin( + p: Partition, + prev: RDD[_], + balanceSlack: Double, + partitionLocs: PartitionLocations): PartitionGroup = { + val slack = (balanceSlack * prev.partitions.length).toInt + // least loaded pref locs + val pref = currPrefLocs(p, prev).map(getLeastGroupHash(_)).sortWith(compare) val prefPart = if (pref == Nil) None else pref.head val r1 = rnd.nextInt(groupArr.size) val r2 = rnd.nextInt(groupArr.size) - val minPowerOfTwo = if (groupArr(r1).size < groupArr(r2).size) groupArr(r1) else groupArr(r2) + val minPowerOfTwo = { + if (groupArr(r1).numPartitions < groupArr(r2).numPartitions) { + groupArr(r1) + } + else { + groupArr(r2) + } + } if (prefPart.isEmpty) { // if no preferred locations, just use basic power of two return minPowerOfTwo @@ -303,55 +317,82 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: val prefPartActual = prefPart.get - if (minPowerOfTwo.size + slack <= prefPartActual.size) { // more imbalance than the slack allows + // more imbalance than the slack allows + if (minPowerOfTwo.numPartitions + slack <= prefPartActual.numPartitions) { minPowerOfTwo // prefer balance over locality } else { prefPartActual // prefer locality over balance } } - def throwBalls() { + def throwBalls( + maxPartitions: Int, + prev: RDD[_], + balanceSlack: Double, partitionLocs: PartitionLocations) { if (noLocality) { // no preferredLocations in parent RDD, no randomization needed if (maxPartitions > groupArr.size) { // just return prev.partitions for ((p, i) <- prev.partitions.zipWithIndex) { - groupArr(i).arr += p + groupArr(i).partitions += p } } else { // no locality available, then simply split partitions based on positions in array for (i <- 0 until maxPartitions) { val rangeStart = ((i.toLong * prev.partitions.length) / maxPartitions).toInt val rangeEnd = (((i.toLong + 1) * prev.partitions.length) / maxPartitions).toInt - (rangeStart until rangeEnd).foreach{ j => groupArr(i).arr += prev.partitions(j) } + (rangeStart until rangeEnd).foreach{ j => groupArr(i).partitions += prev.partitions(j) } } } } else { + // It is possible to have unionRDD where one rdd has preferred locations and another rdd + // that doesn't. To make sure we end up with the requested number of partitions, + // make sure to put a partition in every group. + + // if we don't have a partition assigned to every group first try to fill them + // with the partitions with preferred locations + val partIter = partitionLocs.partsWithLocs.iterator + groupArr.filter(pg => pg.numPartitions == 0).foreach { pg => + while (partIter.hasNext && pg.numPartitions == 0) { + var (nxt_replica, nxt_part) = partIter.next() + if (!initialHash.contains(nxt_part)) { + pg.partitions += nxt_part + initialHash += nxt_part + } + } + } + + // if we didn't get one partitions per group from partitions with preferred locations + // use partitions without preferred locations + val partNoLocIter = partitionLocs.partsWithoutLocs.iterator + groupArr.filter(pg => pg.numPartitions == 0).foreach { pg => + while (partNoLocIter.hasNext && pg.numPartitions == 0) { + var nxt_part = partNoLocIter.next() + if (!initialHash.contains(nxt_part)) { + pg.partitions += nxt_part + initialHash += nxt_part + } + } + } + + // finally pick bin for the rest for (p <- prev.partitions if (!initialHash.contains(p))) { // throw every partition into group - pickBin(p).arr += p + pickBin(p, prev, balanceSlack, partitionLocs).partitions += p } } } - def getPartitions: Array[PartitionGroup] = groupArr.filter( pg => pg.size > 0).toArray + def getPartitions: Array[PartitionGroup] = groupArr.filter( pg => pg.numPartitions > 0).toArray /** * Runs the packing algorithm and returns an array of PartitionGroups that if possible are * load balanced and grouped by locality - * @return array of partition groups + * + * @return array of partition groups */ - def run(): Array[PartitionGroup] = { - setupGroups(math.min(prev.partitions.length, maxPartitions)) // setup the groups (bins) - throwBalls() // assign partitions (balls) to each group (bins) + def coalesce(maxPartitions: Int, prev: RDD[_]): Array[PartitionGroup] = { + val partitionLocs = new PartitionLocations(prev) + // setup the groups (bins) + setupGroups(math.min(prev.partitions.length, maxPartitions), partitionLocs) + // assign partitions (balls) to each group (bins) + throwBalls(maxPartitions, prev, balanceSlack, partitionLocs) getPartitions } } - -private case class PartitionGroup(prefLoc: Option[String] = None) { - var arr = mutable.ArrayBuffer[Partition]() - def size: Int = arr.size -} - -private object PartitionGroup { - def apply(prefLoc: String): PartitionGroup = { - require(prefLoc != "", "Preferred location must not be empty") - PartitionGroup(Some(prefLoc)) - } -} diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 7fbaadcea3a3..943abae17a91 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -17,8 +17,9 @@ package org.apache.spark.rdd -import org.apache.spark.annotation.Experimental -import org.apache.spark.{TaskContext, Logging} +import org.apache.spark.TaskContext +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.MeanEvaluator import org.apache.spark.partial.PartialResult @@ -47,12 +48,12 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { stats().mean } - /** Compute the variance of this RDD's elements. */ + /** Compute the population variance of this RDD's elements. */ def variance(): Double = self.withScope { stats().variance } - /** Compute the standard deviation of this RDD's elements. */ + /** Compute the population standard deviation of this RDD's elements. */ def stdev(): Double = self.withScope { stats().stdev } @@ -73,6 +74,22 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { stats().sampleVariance } + /** + * Compute the population standard deviation of this RDD's elements. + */ + @Since("2.1.0") + def popStdev(): Double = self.withScope { + stats().popStdev + } + + /** + * Compute the population variance of this RDD's elements. + */ + @Since("2.1.0") + def popVariance(): Double = self.withScope { + stats().popVariance + } + /** * Approximate operation to return the mean within a timeout. */ @@ -103,7 +120,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * If the RDD contains infinity, NaN throws an exception * If the elements in RDD do not vary (max == min) always returns a single bucket. */ - def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = self.withScope { + def histogram(bucketCount: Int): (Array[Double], Array[Long]) = self.withScope { // Scala's built-in range has issues. See #SI-8782 def customRange(min: Double, max: Double, steps: Int): IndexedSeq[Double] = { val span = max - min @@ -111,9 +128,9 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { } // Compute the minimum and the maximum val (max: Double, min: Double) = self.mapPartitions { items => - Iterator(items.foldRight(Double.NegativeInfinity, - Double.PositiveInfinity)((e: Double, x: Pair[Double, Double]) => - (x._1.max(e), x._2.min(e)))) + Iterator( + items.foldRight((Double.NegativeInfinity, Double.PositiveInfinity) + )((e: Double, x: (Double, Double)) => (x._1.max(e), x._2.min(e)))) }.reduce { (maxmin1, maxmin2) => (maxmin1._1.max(maxmin2._1), maxmin1._2.min(maxmin2._2)) } @@ -135,14 +152,14 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** * Compute a histogram using the provided buckets. The buckets are all open - * to the right except for the last which is closed + * to the right except for the last which is closed. * e.g. for the array * [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50] - * e.g 1<=x<10 , 10<=x<20, 20<=x<=50 + * e.g {@code <=x<10, 10<=x<20, 20<=x<=50} * And on the input of 1 and 50 we would have a histogram of 1, 0, 1 * - * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched - * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets + * @note If your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched + * from an O(log n) insertion to O(1) per element. (where n = # buckets) if you set evenBuckets * to true. * buckets must be sorted and not contain any duplicates. * buckets array must be at least two elements @@ -166,8 +183,8 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { val counters = new Array[Long](buckets.length - 1) while (iter.hasNext) { bucketFunction(iter.next()) match { - case Some(x: Int) => {counters(x) += 1} - case _ => {} + case Some(x: Int) => counters(x) += 1 + case _ => // No-Op } } Iterator(counters) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index d841f05ec52c..76ea8b86c53d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -17,48 +17,41 @@ package org.apache.spark.rdd +import java.io.IOException import java.text.SimpleDateFormat -import java.util.Date -import java.io.EOFException +import java.util.{Date, Locale} import scala.collection.immutable.Map import scala.reflect.ClassTag -import scala.collection.mutable.ListBuffer import org.apache.hadoop.conf.{Configurable, Configuration} -import org.apache.hadoop.mapred.FileSplit -import org.apache.hadoop.mapred.InputFormat -import org.apache.hadoop.mapred.InputSplit -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.RecordReader -import org.apache.hadoop.mapred.Reporter -import org.apache.hadoop.mapred.JobID -import org.apache.hadoop.mapred.TaskAttemptID -import org.apache.hadoop.mapred.TaskID +import org.apache.hadoop.mapred._ import org.apache.hadoop.mapred.lib.CombineFileSplit +import org.apache.hadoop.mapreduce.TaskType import org.apache.hadoop.util.ReflectionUtils import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.DataReadMethod +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.IGNORE_CORRUPT_FILES import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD -import org.apache.spark.util.{SerializableConfiguration, ShutdownHookManager, NextIterator, Utils} -import org.apache.spark.scheduler.{HostTaskLocation, HDFSCacheTaskLocation} +import org.apache.spark.scheduler.{HDFSCacheTaskLocation, HostTaskLocation} import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.{NextIterator, SerializableConfiguration, ShutdownHookManager} /** * A Spark split class that wraps around a Hadoop InputSplit. */ -private[spark] class HadoopPartition(rddId: Int, idx: Int, s: InputSplit) +private[spark] class HadoopPartition(rddId: Int, override val index: Int, s: InputSplit) extends Partition { val inputSplit = new SerializableWritable[InputSplit](s) - override def hashCode(): Int = 41 * (41 + rddId) + idx + override def hashCode(): Int = 31 * (31 + rddId) + index - override val index: Int = idx + override def equals(other: Any): Boolean = super.equals(other) /** * Get any environment variables that should be added to the users environment when running pipes @@ -68,7 +61,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, s: InputSplit) val envVars: Map[String, String] = if (inputSplit.value.isInstanceOf[FileSplit]) { val is: FileSplit = inputSplit.value.asInstanceOf[FileSplit] // map_input_file is deprecated in favor of mapreduce_map_input_file but set both - // since its not removed yet + // since it's not removed yet Map("map_input_file" -> is.getPath().toString(), "mapreduce_map_input_file" -> is.getPath().toString()) } else { @@ -83,19 +76,19 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, s: InputSplit) * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the older MapReduce API (`org.apache.hadoop.mapred`). * - * Note: Instantiating this class directly is not recommended, please use - * [[org.apache.spark.SparkContext.hadoopRDD()]] - * * @param sc The SparkContext to associate the RDD with. * @param broadcastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed - * variabe references an instance of JobConf, then that JobConf will be used for the Hadoop job. - * Otherwise, a new JobConf will be created on each slave using the enclosed Configuration. + * variable references an instance of JobConf, then that JobConf will be used for the Hadoop job. + * Otherwise, a new JobConf will be created on each slave using the enclosed Configuration. * @param initLocalJobConfFuncOpt Optional closure used to initialize any JobConf that HadoopRDD * creates. * @param inputFormatClass Storage format of the data to be read. * @param keyClass Class of the key associated with the inputFormatClass. * @param valueClass Class of the value associated with the inputFormatClass. * @param minPartitions Minimum number of HadoopRDD partitions (Hadoop Splits) to generate. + * + * @note Instantiating this class directly is not recommended, please use + * `org.apache.spark.SparkContext.hadoopRDD()` */ @DeveloperApi class HadoopRDD[K, V]( @@ -123,22 +116,24 @@ class HadoopRDD[K, V]( sc, sc.broadcast(new SerializableConfiguration(conf)) .asInstanceOf[Broadcast[SerializableConfiguration]], - None /* initLocalJobConfFuncOpt */, + initLocalJobConfFuncOpt = None, inputFormatClass, keyClass, valueClass, minPartitions) } - protected val jobConfCacheKey = "rdd_%d_job_conf".format(id) + protected val jobConfCacheKey: String = "rdd_%d_job_conf".format(id) - protected val inputFormatCacheKey = "rdd_%d_input_format".format(id) + protected val inputFormatCacheKey: String = "rdd_%d_input_format".format(id) // used to build JobTracker ID private val createTime = new Date() private val shouldCloneJobConf = sparkContext.conf.getBoolean("spark.hadoop.cloneConf", false) + private val ignoreCorruptFiles = sparkContext.conf.get(IGNORE_CORRUPT_FILES) + // Returns a JobConf that will be used on slaves to obtain input splits for Hadoop reads. protected def getJobConf(): JobConf = { val conf: Configuration = broadcastedConf.value.value @@ -154,7 +149,7 @@ class HadoopRDD[K, V]( logDebug("Cloning Hadoop Configuration") val newJobConf = new JobConf(conf) if (!conf.isInstanceOf[JobConf]) { - initLocalJobConfFuncOpt.map(f => f(newJobConf)) + initLocalJobConfFuncOpt.foreach(f => f(newJobConf)) } newJobConf } @@ -173,7 +168,7 @@ class HadoopRDD[K, V]( HadoopRDD.CONFIGURATION_INSTANTIATION_LOCK.synchronized { logDebug("Creating new JobConf and caching it for later re-use") val newJobConf = new JobConf(conf) - initLocalJobConfFuncOpt.map(f => f(newJobConf)) + initLocalJobConfFuncOpt.foreach(f => f(newJobConf)) HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf) newJobConf } @@ -184,8 +179,9 @@ class HadoopRDD[K, V]( protected def getInputFormat(conf: JobConf): InputFormat[K, V] = { val newInputFormat = ReflectionUtils.newInstance(inputFormatClass.asInstanceOf[Class[_]], conf) .asInstanceOf[InputFormat[K, V]] - if (newInputFormat.isInstanceOf[Configurable]) { - newInputFormat.asInstanceOf[Configurable].setConf(conf) + newInputFormat match { + case c: Configurable => c.setConf(conf) + case _ => } newInputFormat } @@ -195,9 +191,6 @@ class HadoopRDD[K, V]( // add the credentials here as this can be called before SparkContext initialized SparkHadoopUtil.get.addCredentials(jobConf) val inputFormat = getInputFormat(jobConf) - if (inputFormat.isInstanceOf[Configurable]) { - inputFormat.asInstanceOf[Configurable].setConf(jobConf) - } val inputSplits = inputFormat.getSplits(jobConf, minPartitions) val array = new Array[Partition](inputSplits.size) for (i <- 0 until inputSplits.size) { @@ -209,53 +202,85 @@ class HadoopRDD[K, V]( override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = { val iter = new NextIterator[(K, V)] { - val split = theSplit.asInstanceOf[HadoopPartition] + private val split = theSplit.asInstanceOf[HadoopPartition] logInfo("Input split: " + split.inputSplit) - val jobConf = getJobConf() + private val jobConf = getJobConf() + + private val inputMetrics = context.taskMetrics().inputMetrics + private val existingBytesRead = inputMetrics.bytesRead - val inputMetrics = context.taskMetrics.getInputMetricsForReadMethod(DataReadMethod.Hadoop) + // Sets InputFileBlockHolder for the file block's information + split.inputSplit.value match { + case fs: FileSplit => + InputFileBlockHolder.set(fs.getPath.toString, fs.getStart, fs.getLength) + case _ => + InputFileBlockHolder.unset() + } // Find a function that will return the FileSystem bytes read by this thread. Do this before // creating RecordReader, because RecordReader's constructor might read some bytes - val bytesReadCallback = inputMetrics.bytesReadCallback.orElse { - split.inputSplit.value match { - case _: FileSplit | _: CombineFileSplit => - SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() - case _ => None + private val getBytesReadCallback: Option[() => Long] = split.inputSplit.value match { + case _: FileSplit | _: CombineFileSplit => + Some(SparkHadoopUtil.get.getFSBytesReadOnThreadCallback()) + case _ => None + } + + // We get our input bytes from thread-local Hadoop FileSystem statistics. + // If we do a coalesce, however, we are likely to compute multiple partitions in the same + // task and in the same thread, in which case we need to avoid override values written by + // previous partitions (SPARK-13071). + private def updateBytesRead(): Unit = { + getBytesReadCallback.foreach { getBytesRead => + inputMetrics.setBytesRead(existingBytesRead + getBytesRead()) } } - inputMetrics.setBytesReadCallback(bytesReadCallback) - var reader: RecordReader[K, V] = null - val inputFormat = getInputFormat(jobConf) - HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmm").format(createTime), + private var reader: RecordReader[K, V] = null + private val inputFormat = getInputFormat(jobConf) + HadoopRDD.addLocalConfiguration( + new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(createTime), context.stageId, theSplit.index, context.attemptNumber, jobConf) - reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL) + reader = + try { + inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL) + } catch { + case e: IOException if ignoreCorruptFiles => + logWarning(s"Skipped the rest content in the corrupted file: ${split.inputSplit}", e) + finished = true + null + } // Register an on-task-completion callback to close the input stream. - context.addTaskCompletionListener{ context => closeIfNeeded() } - val key: K = reader.createKey() - val value: V = reader.createValue() + context.addTaskCompletionListener { context => + // Update the bytes read before closing is to make sure lingering bytesRead statistics in + // this thread get correctly added. + updateBytesRead() + closeIfNeeded() + } + + private val key: K = if (reader == null) null.asInstanceOf[K] else reader.createKey() + private val value: V = if (reader == null) null.asInstanceOf[V] else reader.createValue() override def getNext(): (K, V) = { try { finished = !reader.next(key, value) } catch { - case eof: EOFException => + case e: IOException if ignoreCorruptFiles => + logWarning(s"Skipped the rest content in the corrupted file: ${split.inputSplit}", e) finished = true } if (!finished) { inputMetrics.incRecordsRead(1) } + if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { + updateBytesRead() + } (key, value) } - override def close() { + override def close(): Unit = { if (reader != null) { - // Close the reader and release it. Note: it's very important that we don't close the - // reader more than once, since that exposes us to MAPREDUCE-5918 when running against - // Hadoop 1.x and older Hadoop 2.x releases. That bug can lead to non-deterministic - // corruption issues when reading compressed input. + InputFileBlockHolder.unset() try { reader.close() } catch { @@ -266,8 +291,8 @@ class HadoopRDD[K, V]( } finally { reader = null } - if (bytesReadCallback.isDefined) { - inputMetrics.updateBytesRead() + if (getBytesReadCallback.isDefined) { + updateBytesRead() } else if (split.inputSplit.value.isInstanceOf[FileSplit] || split.inputSplit.value.isInstanceOf[CombineFileSplit]) { // If we can't get the bytes read from the FS stats, fall back to the split size, @@ -295,18 +320,10 @@ class HadoopRDD[K, V]( override def getPreferredLocations(split: Partition): Seq[String] = { val hsplit = split.asInstanceOf[HadoopPartition].inputSplit.value - val locs: Option[Seq[String]] = HadoopRDD.SPLIT_INFO_REFLECTIONS match { - case Some(c) => - try { - val lsplit = c.inputSplitWithLocationInfo.cast(hsplit) - val infos = c.getLocationInfo.invoke(lsplit).asInstanceOf[Array[AnyRef]] - Some(HadoopRDD.convertSplitLocationInfo(infos)) - } catch { - case e: Exception => - logDebug("Failed to use InputSplitWithLocations.", e) - None - } - case None => None + val locs = hsplit match { + case lsplit: InputSplitWithLocationInfo => + HadoopRDD.convertSplitLocationInfo(lsplit.getLocationInfo) + case _ => None } locs.getOrElse(hsplit.getLocations.filter(_ != "localhost")) } @@ -317,7 +334,7 @@ class HadoopRDD[K, V]( override def persist(storageLevel: StorageLevel): this.type = { if (storageLevel.deserialized) { - logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + + logWarning("Caching HadoopRDDs as deserialized objects usually leads to undesired" + " behavior because Hadoop's RecordReader reuses the same Writable object for all records." + " Use a map transformation to make copies of the records.") } @@ -352,13 +369,13 @@ private[spark] object HadoopRDD extends Logging { def addLocalConfiguration(jobTrackerId: String, jobId: Int, splitId: Int, attemptId: Int, conf: JobConf) { val jobID = new JobID(jobTrackerId, jobId) - val taId = new TaskAttemptID(new TaskID(jobID, true, splitId), attemptId) + val taId = new TaskAttemptID(new TaskID(jobID, TaskType.MAP, splitId), attemptId) - conf.set("mapred.tip.id", taId.getTaskID.toString) - conf.set("mapred.task.id", taId.toString) - conf.setBoolean("mapred.task.is.map", true) - conf.setInt("mapred.task.partition", splitId) - conf.set("mapred.job.id", jobID.toString) + conf.set("mapreduce.task.id", taId.getTaskID.toString) + conf.set("mapreduce.task.attempt.id", taId.toString) + conf.setBoolean("mapreduce.task.ismap", true) + conf.setInt("mapreduce.task.partition", splitId) + conf.set("mapreduce.job.id", jobID.toString) } /** @@ -382,41 +399,20 @@ private[spark] object HadoopRDD extends Logging { } } - private[spark] class SplitInfoReflections { - val inputSplitWithLocationInfo = - Utils.classForName("org.apache.hadoop.mapred.InputSplitWithLocationInfo") - val getLocationInfo = inputSplitWithLocationInfo.getMethod("getLocationInfo") - val newInputSplit = Utils.classForName("org.apache.hadoop.mapreduce.InputSplit") - val newGetLocationInfo = newInputSplit.getMethod("getLocationInfo") - val splitLocationInfo = Utils.classForName("org.apache.hadoop.mapred.SplitLocationInfo") - val isInMemory = splitLocationInfo.getMethod("isInMemory") - val getLocation = splitLocationInfo.getMethod("getLocation") - } - - private[spark] val SPLIT_INFO_REFLECTIONS: Option[SplitInfoReflections] = try { - Some(new SplitInfoReflections) - } catch { - case e: Exception => - logDebug("SplitLocationInfo and other new Hadoop classes are " + - "unavailable. Using the older Hadoop location info code.", e) - None - } - - private[spark] def convertSplitLocationInfo(infos: Array[AnyRef]): Seq[String] = { - val out = ListBuffer[String]() - infos.foreach { loc => { - val locationStr = HadoopRDD.SPLIT_INFO_REFLECTIONS.get. - getLocation.invoke(loc).asInstanceOf[String] + private[spark] def convertSplitLocationInfo( + infos: Array[SplitLocationInfo]): Option[Seq[String]] = { + Option(infos).map(_.flatMap { loc => + val locationStr = loc.getLocation if (locationStr != "localhost") { - if (HadoopRDD.SPLIT_INFO_REFLECTIONS.get.isInMemory. - invoke(loc).asInstanceOf[Boolean]) { - logDebug("Partition " + locationStr + " is cached by Hadoop.") - out += new HDFSCacheTaskLocation(locationStr).toString + if (loc.isInMemory) { + logDebug(s"Partition $locationStr is cached by Hadoop.") + Some(HDFSCacheTaskLocation(locationStr).toString) } else { - out += new HostTaskLocation(locationStr).toString + Some(HostTaskLocation(locationStr).toString) } + } else { + None } - }} - out.seq + }) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala b/core/src/main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala new file mode 100644 index 000000000000..ff2f58d81142 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/InputFileBlockHolder.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.rdd + +import org.apache.spark.unsafe.types.UTF8String + +/** + * This holds file names of the current Spark task. This is used in HadoopRDD, + * FileScanRDD, NewHadoopRDD and InputFileName function in Spark SQL. + */ +private[spark] object InputFileBlockHolder { + /** + * A wrapper around some input file information. + * + * @param filePath path of the file read, or empty string if not available. + * @param startOffset starting offset, in bytes, or -1 if not available. + * @param length size of the block, in bytes, or -1 if not available. + */ + private class FileBlock(val filePath: UTF8String, val startOffset: Long, val length: Long) { + def this() { + this(UTF8String.fromString(""), -1, -1) + } + } + + /** + * The thread variable for the name of the current file being read. This is used by + * the InputFileName function in Spark SQL. + */ + private[this] val inputBlock: InheritableThreadLocal[FileBlock] = + new InheritableThreadLocal[FileBlock] { + override protected def initialValue(): FileBlock = new FileBlock + } + + /** + * Returns the holding file name or empty string if it is unknown. + */ + def getInputFilePath: UTF8String = inputBlock.get().filePath + + /** + * Returns the starting offset of the block currently being read, or -1 if it is unknown. + */ + def getStartOffset: Long = inputBlock.get().startOffset + + /** + * Returns the length of the block being read, or -1 if it is unknown. + */ + def getLength: Long = inputBlock.get().length + + /** + * Sets the thread-local input block. + */ + def set(filePath: String, startOffset: Long, length: Long): Unit = { + require(filePath != null, "filePath cannot be null") + require(startOffset >= 0, s"startOffset ($startOffset) cannot be negative") + require(length >= 0, s"length ($length) cannot be negative") + inputBlock.set(new FileBlock(UTF8String.fromString(filePath), startOffset, length)) + } + + /** + * Clears the input file block to default value. + */ + def unset(): Unit = inputBlock.remove() +} diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 0c28f045e46e..aab46b8954bf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -17,15 +17,16 @@ package org.apache.spark.rdd -import java.sql.{PreparedStatement, Connection, ResultSet} +import java.sql.{Connection, ResultSet} import scala.reflect.ClassTag +import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.internal.Logging import org.apache.spark.util.NextIterator -import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { override def index: Int = idx @@ -33,14 +34,17 @@ private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) e // TODO: Expose a jdbcRDD function in SparkContext and mark this as semi-private /** - * An RDD that executes an SQL query on a JDBC connection and reads results. + * An RDD that executes a SQL query on a JDBC connection and reads results. * For usage example, see test case JdbcRDDSuite. * * @param getConnection a function that returns an open Connection. * The RDD takes care of closing the connection. * @param sql the text of the query. * The query must contain two ? placeholders for parameters used to partition the results. - * E.g. "select title, author from books where ? <= id and id <= ?" + * For example, + * {{{ + * select title, author from books where ? <= id and id <= ? + * }}} * @param lowerBound the minimum value of the first placeholder * @param upperBound the maximum value of the second placeholder * The lower and upper bounds are inclusive. @@ -64,11 +68,11 @@ class JdbcRDD[T: ClassTag]( override def getPartitions: Array[Partition] = { // bounds are inclusive, hence the + 1 here and - 1 on end val length = BigInt(1) + upperBound - lowerBound - (0 until numPartitions).map(i => { + (0 until numPartitions).map { i => val start = lowerBound + ((i * length) / numPartitions) val end = lowerBound + (((i + 1) * length) / numPartitions) - 1 new JdbcPartition(i, start.toLong, end.toLong) - }).toArray + }.toArray } override def compute(thePart: Partition, context: TaskContext): Iterator[T] = new NextIterator[T] @@ -78,14 +82,20 @@ class JdbcRDD[T: ClassTag]( val conn = getConnection() val stmt = conn.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY) - // setFetchSize(Integer.MIN_VALUE) is a mysql driver specific way to force streaming results, - // rather than pulling entire resultset into memory. - // see http://dev.mysql.com/doc/refman/5.0/en/connector-j-reference-implementation-notes.html - if (conn.getMetaData.getURL.matches("jdbc:mysql:.*")) { + val url = conn.getMetaData.getURL + if (url.startsWith("jdbc:mysql:")) { + // setFetchSize(Integer.MIN_VALUE) is a mysql driver specific way to force + // streaming results, rather than pulling entire resultset into memory. + // See the below URL + // dev.mysql.com/doc/connector-j/5.1/en/connector-j-reference-implementation-notes.html + stmt.setFetchSize(Integer.MIN_VALUE) - logInfo("statement fetch size set to: " + stmt.getFetchSize + " to force MySQL streaming ") + } else { + stmt.setFetchSize(100) } + logInfo(s"statement fetch size set to: ${stmt.getFetchSize}") + stmt.setLong(1, part.lower) stmt.setLong(2, part.upper) val rs = stmt.executeQuery() @@ -137,14 +147,17 @@ object JdbcRDD { } /** - * Create an RDD that executes an SQL query on a JDBC connection and reads results. + * Create an RDD that executes a SQL query on a JDBC connection and reads results. * For usage example, see test case JavaAPISuite.testJavaJdbcRDD. * * @param connectionFactory a factory that returns an open Connection. * The RDD takes care of closing the connection. * @param sql the text of the query. * The query must contain two ? placeholders for parameters used to partition the results. - * E.g. "select title, author from books where ? <= id and id <= ?" + * For example, + * {{{ + * select title, author from books where ? <= id and id <= ? + * }}} * @param lowerBound the minimum value of the first placeholder * @param upperBound the maximum value of the second placeholder * The lower and upper bounds are inclusive. @@ -177,14 +190,17 @@ object JdbcRDD { } /** - * Create an RDD that executes an SQL query on a JDBC connection and reads results. Each row is + * Create an RDD that executes a SQL query on a JDBC connection and reads results. Each row is * converted into a `Object` array. For usage example, see test case JavaAPISuite.testJavaJdbcRDD. * * @param connectionFactory a factory that returns an open Connection. * The RDD takes care of closing the connection. * @param sql the text of the query. * The query must contain two ? placeholders for parameters used to partition the results. - * E.g. "select title, author from books where ? <= id and id <= ?" + * For example, + * {{{ + * select title, author from books where ? <= id and id <= ? + * }}} * @param lowerBound the minimum value of the first placeholder * @param upperBound the maximum value of the second placeholder * The lower and upper bounds are inclusive. diff --git a/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala index bfe19195fcd3..503aa0dffc9f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{Partition, SparkContext, SparkEnv, SparkException, TaskContext} +import org.apache.spark.{Partition, SparkContext, SparkException, TaskContext} import org.apache.spark.storage.RDDBlockId /** @@ -41,7 +41,7 @@ private[spark] class LocalCheckpointRDD[T: ClassTag]( extends CheckpointRDD[T](sc) { def this(rdd: RDD[T]) { - this(rdd.context, rdd.id, rdd.partitions.size) + this(rdd.context, rdd.id, rdd.partitions.length) } protected override def getPartitions: Array[Partition] = { diff --git a/core/src/main/scala/org/apache/spark/rdd/LocalRDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/LocalRDDCheckpointData.scala index c115e0ff74d3..56f53714cbe3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/LocalRDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/LocalRDDCheckpointData.scala @@ -19,7 +19,8 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{Logging, SparkEnv, SparkException, TaskContext} +import org.apache.spark.{SparkEnv, TaskContext} +import org.apache.spark.internal.Logging import org.apache.spark.storage.{RDDBlockId, StorageLevel} import org.apache.spark.util.Utils @@ -72,12 +73,6 @@ private[spark] object LocalRDDCheckpointData { * This method is idempotent. */ def transformStorageLevel(level: StorageLevel): StorageLevel = { - // If this RDD is to be cached off-heap, fail fast since we cannot provide any - // correctness guarantees about subsequent computations after the first one - if (level.useOffHeap) { - throw new SparkException("Local checkpointing is not compatible with off-heap caching.") - } - StorageLevel(useDisk = true, level.useMemory, level.deserialized, level.replication) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 4312d3a41775..e4587c96eae1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -25,7 +25,7 @@ import org.apache.spark.{Partition, TaskContext} * An RDD that applies the provided function to every partition of the parent RDD. */ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( - prev: RDD[T], + var prev: RDD[T], f: (TaskContext, Int, Iterator[T]) => Iterator[U], // (TaskContext, partition index, iterator) preservesPartitioning: Boolean = false) extends RDD[U](prev) { @@ -36,4 +36,9 @@ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[U] = f(context, split.index, firstParent[T].iterator(split, context)) + + override def clearDependencies() { + super.clearDependencies() + prev = null + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 9c4b70844bdb..482875e6c1ac 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -17,25 +17,27 @@ package org.apache.spark.rdd +import java.io.IOException import java.text.SimpleDateFormat -import java.util.Date +import java.util.{Date, Locale} import scala.reflect.ClassTag import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, FileSplit} +import org.apache.hadoop.mapreduce.task.{JobContextImpl, TaskAttemptContextImpl} -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.input.WholeTextFileInputFormat import org.apache.spark._ -import org.apache.spark.executor.DataReadMethod -import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil -import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD -import org.apache.spark.util.{SerializableConfiguration, ShutdownHookManager, Utils} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.IGNORE_CORRUPT_FILES +import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.{SerializableConfiguration, ShutdownHookManager} private[spark] class NewHadoopPartition( rddId: Int, @@ -44,7 +46,10 @@ private[spark] class NewHadoopPartition( extends Partition { val serializableHadoopSplit = new SerializableWritable(rawSplit) - override def hashCode(): Int = 41 * (41 + rddId) + index + + override def hashCode(): Int = 31 * (31 + rddId) + index + + override def equals(other: Any): Boolean = super.equals(other) } /** @@ -52,14 +57,13 @@ private[spark] class NewHadoopPartition( * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the new MapReduce API (`org.apache.hadoop.mapreduce`). * - * Note: Instantiating this class directly is not recommended, please use - * [[org.apache.spark.SparkContext.newAPIHadoopRDD()]] - * * @param sc The SparkContext to associate the RDD with. * @param inputFormatClass Storage format of the data to be read. * @param keyClass Class of the key associated with the inputFormatClass. * @param valueClass Class of the value associated with the inputFormatClass. - * @param conf The Hadoop configuration. + * + * @note Instantiating this class directly is not recommended, please use + * `org.apache.spark.SparkContext.newAPIHadoopRDD()` */ @DeveloperApi class NewHadoopRDD[K, V]( @@ -68,16 +72,14 @@ class NewHadoopRDD[K, V]( keyClass: Class[K], valueClass: Class[V], @transient private val _conf: Configuration) - extends RDD[(K, V)](sc, Nil) - with SparkHadoopMapReduceUtil - with Logging { + extends RDD[(K, V)](sc, Nil) with Logging { // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it private val confBroadcast = sc.broadcast(new SerializableConfiguration(_conf)) // private val serializableConf = new SerializableWritable(_conf) private val jobTrackerId: String = { - val formatter = new SimpleDateFormat("yyyyMMddHHmm") + val formatter = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) formatter.format(new Date()) } @@ -85,6 +87,8 @@ class NewHadoopRDD[K, V]( private val shouldCloneJobConf = sparkContext.conf.getBoolean("spark.hadoop.cloneConf", false) + private val ignoreCorruptFiles = sparkContext.conf.get(IGNORE_CORRUPT_FILES) + def getConf: Configuration = { val conf: Configuration = confBroadcast.value.value if (shouldCloneJobConf) { @@ -97,7 +101,13 @@ class NewHadoopRDD[K, V]( // issues, this cloning is disabled by default. NewHadoopRDD.CONFIGURATION_INSTANTIATION_LOCK.synchronized { logDebug("Cloning Hadoop Configuration") - new Configuration(conf) + // The Configuration passed in is actually a JobConf and possibly contains credentials. + // To keep those credentials properly we have to create a new JobConf not a Configuration. + if (conf.isInstanceOf[JobConf]) { + new JobConf(conf) + } else { + new Configuration(conf) + } } } else { conf @@ -111,7 +121,7 @@ class NewHadoopRDD[K, V]( configurable.setConf(_conf) case _ => } - val jobContext = newJobContext(_conf, jobId) + val jobContext = new JobContextImpl(_conf, jobId) val rawSplits = inputFormat.getSplits(jobContext).toArray val result = new Array[Partition](rawSplits.size) for (i <- 0 until rawSplits.size) { @@ -122,45 +132,86 @@ class NewHadoopRDD[K, V]( override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = { val iter = new Iterator[(K, V)] { - val split = theSplit.asInstanceOf[NewHadoopPartition] + private val split = theSplit.asInstanceOf[NewHadoopPartition] logInfo("Input split: " + split.serializableHadoopSplit) - val conf = getConf + private val conf = getConf - val inputMetrics = context.taskMetrics - .getInputMetricsForReadMethod(DataReadMethod.Hadoop) + private val inputMetrics = context.taskMetrics().inputMetrics + private val existingBytesRead = inputMetrics.bytesRead + + // Sets InputFileBlockHolder for the file block's information + split.serializableHadoopSplit.value match { + case fs: FileSplit => + InputFileBlockHolder.set(fs.getPath.toString, fs.getStart, fs.getLength) + case _ => + InputFileBlockHolder.unset() + } // Find a function that will return the FileSystem bytes read by this thread. Do this before // creating RecordReader, because RecordReader's constructor might read some bytes - val bytesReadCallback = inputMetrics.bytesReadCallback.orElse { + private val getBytesReadCallback: Option[() => Long] = split.serializableHadoopSplit.value match { case _: FileSplit | _: CombineFileSplit => - SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() + Some(SparkHadoopUtil.get.getFSBytesReadOnThreadCallback()) case _ => None } + + // We get our input bytes from thread-local Hadoop FileSystem statistics. + // If we do a coalesce, however, we are likely to compute multiple partitions in the same + // task and in the same thread, in which case we need to avoid override values written by + // previous partitions (SPARK-13071). + private def updateBytesRead(): Unit = { + getBytesReadCallback.foreach { getBytesRead => + inputMetrics.setBytesRead(existingBytesRead + getBytesRead()) + } } - inputMetrics.setBytesReadCallback(bytesReadCallback) - val attemptId = newTaskAttemptID(jobTrackerId, id, isMap = true, split.index, 0) - val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) - val format = inputFormatClass.newInstance + private val format = inputFormatClass.newInstance format match { case configurable: Configurable => configurable.setConf(conf) case _ => } - private var reader = format.createRecordReader( - split.serializableHadoopSplit.value, hadoopAttemptContext) - reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) + private val attemptId = new TaskAttemptID(jobTrackerId, id, TaskType.MAP, split.index, 0) + private val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId) + private var finished = false + private var reader = + try { + val _reader = format.createRecordReader( + split.serializableHadoopSplit.value, hadoopAttemptContext) + _reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) + _reader + } catch { + case e: IOException if ignoreCorruptFiles => + logWarning( + s"Skipped the rest content in the corrupted file: ${split.serializableHadoopSplit}", + e) + finished = true + null + } // Register an on-task-completion callback to close the input stream. - context.addTaskCompletionListener(context => close()) - var havePair = false - var finished = false - var recordsSinceMetricsUpdate = 0 + context.addTaskCompletionListener { context => + // Update the bytesRead before closing is to make sure lingering bytesRead statistics in + // this thread get correctly added. + updateBytesRead() + close() + } + + private var havePair = false + private var recordsSinceMetricsUpdate = 0 override def hasNext: Boolean = { if (!finished && !havePair) { - finished = !reader.nextKeyValue + try { + finished = !reader.nextKeyValue + } catch { + case e: IOException if ignoreCorruptFiles => + logWarning( + s"Skipped the rest content in the corrupted file: ${split.serializableHadoopSplit}", + e) + finished = true + } if (finished) { // Close and release the reader here; close() will also be called when the task // completes, but for tasks that read from many files, it helps to release the @@ -180,15 +231,15 @@ class NewHadoopRDD[K, V]( if (!finished) { inputMetrics.incRecordsRead(1) } + if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { + updateBytesRead() + } (reader.getCurrentKey, reader.getCurrentValue) } - private def close() { + private def close(): Unit = { if (reader != null) { - // Close the reader and release it. Note: it's very important that we don't close the - // reader more than once, since that exposes us to MAPREDUCE-5918 when running against - // Hadoop 1.x and older Hadoop 2.x releases. That bug can lead to non-deterministic - // corruption issues when reading compressed input. + InputFileBlockHolder.unset() try { reader.close() } catch { @@ -199,8 +250,8 @@ class NewHadoopRDD[K, V]( } finally { reader = null } - if (bytesReadCallback.isDefined) { - inputMetrics.updateBytesRead() + if (getBytesReadCallback.isDefined) { + updateBytesRead() } else if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit] || split.serializableHadoopSplit.value.isInstanceOf[CombineFileSplit]) { // If we can't get the bytes read from the FS stats, fall back to the split size, @@ -228,18 +279,7 @@ class NewHadoopRDD[K, V]( override def getPreferredLocations(hsplit: Partition): Seq[String] = { val split = hsplit.asInstanceOf[NewHadoopPartition].serializableHadoopSplit.value - val locs = HadoopRDD.SPLIT_INFO_REFLECTIONS match { - case Some(c) => - try { - val infos = c.newGetLocationInfo.invoke(split).asInstanceOf[Array[AnyRef]] - Some(HadoopRDD.convertSplitLocationInfo(infos)) - } catch { - case e : Exception => - logDebug("Failed to use InputSplit#getLocationInfo.", e) - None - } - case None => None - } + val locs = HadoopRDD.convertSplitLocationInfo(split.getLocationInfo) locs.getOrElse(split.getLocations.filter(_ != "localhost")) } @@ -282,32 +322,3 @@ private[spark] object NewHadoopRDD { } } } - -private[spark] class WholeTextFileRDD( - sc : SparkContext, - inputFormatClass: Class[_ <: WholeTextFileInputFormat], - keyClass: Class[String], - valueClass: Class[String], - conf: Configuration, - minPartitions: Int) - extends NewHadoopRDD[String, String](sc, inputFormatClass, keyClass, valueClass, conf) { - - override def getPartitions: Array[Partition] = { - val inputFormat = inputFormatClass.newInstance - val conf = getConf - inputFormat match { - case configurable: Configurable => - configurable.setConf(conf) - case _ => - } - val jobContext = newJobContext(conf, jobId) - inputFormat.setMinPartitions(jobContext, minPartitions) - val rawSplits = inputFormat.getSplits(jobContext).toArray - val result = new Array[Partition](rawSplits.size) - for (i <- 0 until rawSplits.size) { - result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) - } - result - } -} - diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index d71bb6300090..a5992022d083 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -19,8 +19,9 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{Logging, Partitioner, RangePartitioner} +import org.apache.spark.{Partitioner, RangePartitioner} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through @@ -45,8 +46,7 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag, P <: Product2[K, V] : ClassTag] @DeveloperApi() ( self: RDD[P]) - extends Logging with Serializable -{ + extends Logging with Serializable { private val ordering = implicitly[Ordering[K]] /** @@ -76,7 +76,7 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, } /** - * Returns an RDD containing only the elements in the the inclusive range `lower` to `upper`. + * Returns an RDD containing only the elements in the inclusive range `lower` to `upper`. * If the RDD has been partitioned using a `RangePartitioner`, then this operation can be * performed efficiently by only scanning the partitions that might contain matching elements. * Otherwise, a standard `filter` is applied to all partitions. @@ -86,12 +86,11 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, def inRange(k: K): Boolean = ordering.gteq(k, lower) && ordering.lteq(k, upper) val rddToFilter: RDD[P] = self.partitioner match { - case Some(rp: RangePartitioner[K, V]) => { + case Some(rp: RangePartitioner[K, V]) => val partitionIndicies = (rp.getPartition(lower), rp.getPartition(upper)) match { case (l, u) => Math.min(l, u) to Math.max(l, u) } PartitionPruningRDD.create(self, partitionIndicies.contains) - } case _ => self } diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index c6181902ace6..e68c6b1366c7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -18,33 +18,28 @@ package org.apache.spark.rdd import java.nio.ByteBuffer -import java.text.SimpleDateFormat -import java.util.{Date, HashMap => JHashMap} +import java.util.{HashMap => JHashMap} -import scala.collection.{Map, mutable} +import scala.collection.{mutable, Map} import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import scala.util.DynamicVariable import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus -import org.apache.hadoop.conf.{Configurable, Configuration} -import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} -import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewOutputFormat, - RecordWriter => NewRecordWriter} +import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewOutputFormat} import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.annotation.Experimental -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.{DataWriteMethod, OutputMetrics} -import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io._ import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer -import org.apache.spark.util.{SerializableConfiguration, Utils} +import org.apache.spark.util.{SerializableConfiguration, SerializableJobConf, Utils} import org.apache.spark.util.collection.CompactBuffer import org.apache.spark.util.random.StratifiedSamplingUtils @@ -53,24 +48,24 @@ import org.apache.spark.util.random.StratifiedSamplingUtils */ class PairRDDFunctions[K, V](self: RDD[(K, V)]) (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) - extends Logging - with SparkHadoopMapReduceUtil - with Serializable -{ + extends Logging with Serializable { /** * :: Experimental :: * Generic function to combine the elements for each key using a custom set of aggregation * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C - * Note that V and C can be different -- for example, one might group an RDD of type - * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: * - * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) - * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) - * - `mergeCombiners`, to combine two C's into a single one. + * Users provide three functions: + * + * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) + * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) + * - `mergeCombiners`, to combine two C's into a single one. * * In addition, users can control the partitioning of the output RDD, and whether to perform * map-side aggregation (if a mapper can produce multiple items with the same key). + * + * @note V and C can be different -- for example, one might group an RDD of type + * (Int, Int) into an RDD of type (Int, Seq[Int]). */ @Experimental def combineByKeyWithClassTag[C]( @@ -86,7 +81,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) throw new SparkException("Cannot use map-side combining with array keys.") } if (partitioner.isInstanceOf[HashPartitioner]) { - throw new SparkException("Default partitioner cannot partition array keys.") + throw new SparkException("HashPartitioner cannot partition array keys.") } } val aggregator = new Aggregator[K, V, C]( @@ -111,7 +106,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * functions. This method is here for backward compatibility. It does not provide combiner * classtag information to the shuffle. * - * @see [[combineByKeyWithClassTag]] + * @see `combineByKeyWithClassTag` */ def combineByKey[C]( createCombiner: V => C, @@ -129,7 +124,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * This method is here for backward compatibility. It does not provide combiner * classtag information to the shuffle. * - * @see [[combineByKeyWithClassTag]] + * @see `combineByKeyWithClassTag` */ def combineByKey[C]( createCombiner: V => C, @@ -304,27 +299,27 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. + * Merge the values for each key using an associative and commutative reduce function. This will + * also perform the merging locally on each mapper before sending results to a reducer, similarly + * to a "combiner" in MapReduce. */ def reduceByKey(partitioner: Partitioner, func: (V, V) => V): RDD[(K, V)] = self.withScope { combineByKeyWithClassTag[V]((v: V) => v, func, func, partitioner) } /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. + * Merge the values for each key using an associative and commutative reduce function. This will + * also perform the merging locally on each mapper before sending results to a reducer, similarly + * to a "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. */ def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = self.withScope { reduceByKey(new HashPartitioner(numPartitions), func) } /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ + * Merge the values for each key using an associative and commutative reduce function. This will + * also perform the merging locally on each mapper before sending results to a reducer, similarly + * to a "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ * parallelism level. */ def reduceByKey(func: (V, V) => V): RDD[(K, V)] = self.withScope { @@ -332,9 +327,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } /** - * Merge the values for each key using an associative reduce function, but return the results - * immediately to the master as a Map. This will also perform the merging locally on each mapper - * before sending results to a reducer, similarly to a "combiner" in MapReduce. + * Merge the values for each key using an associative and commutative reduce function, but return + * the results immediately to the master as a Map. This will also perform the merging locally on + * each mapper before sending results to a reducer, similarly to a "combiner" in MapReduce. */ def reduceByKeyLocally(func: (V, V) => V): Map[K, V] = self.withScope { val cleanedF = self.sparkContext.clean(func) @@ -363,16 +358,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) self.mapPartitions(reducePartition).reduce(mergeMaps).asScala } - /** Alias for reduceByKeyLocally */ - @deprecated("Use reduceByKeyLocally", "1.0.0") - def reduceByKeyToDriver(func: (V, V) => V): Map[K, V] = self.withScope { - reduceByKeyLocally(func) - } - /** * Count the number of elements for each key, collecting the results to a local Map. * - * Note that this method should only be used if the resulting map is expected to be small, as + * @note This method should only be used if the resulting map is expected to be small, as * the whole thing is loaded into the driver's memory. * To handle very large results, consider using rdd.mapValues(_ => 1L).reduceByKey(_ + _), which * returns an RDD[T, Long] instead of a map. @@ -384,6 +373,16 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. + * + * The confidence is the probability that the error bounds of the result will + * contain the true value. That is, if countApprox were called repeatedly + * with confidence 0.9, we would expect 90% of the results to contain the + * true count. The confidence must be in the range [0,1] or an exception will + * be thrown. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[Map[K, BoundedDouble]] = self.withScope { @@ -397,9 +396,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available * here. * - * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero `sp > p` - * would trigger sparse representation of registers, which may reduce the memory consumption - * and increase accuracy when the cardinality is small. + * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero (`sp` is + * greater than `p`) would trigger sparse representation of registers, which may reduce the + * memory consumption and increase accuracy when the cardinality is small. * * @param p The precision value for the normal set. * `p` must be a value between 4 and `sp` if `sp` is not zero (32 max). @@ -489,12 +488,12 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * The ordering of elements within each group is not guaranteed, and may even differ * each time the resulting RDD is evaluated. * - * Note: This operation may be very expensive. If you are grouping in order to perform an - * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] - * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. + * @note This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey` + * or `PairRDDFunctions.reduceByKey` will provide much better performance. * - * Note: As currently implemented, groupByKey must be able to hold all the key-value pairs for any - * key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]]. + * @note As currently implemented, groupByKey must be able to hold all the key-value pairs for any + * key in memory. If a key has too many values, it can result in an `OutOfMemoryError`. */ def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = self.withScope { // groupByKey shouldn't use map side combine because map side combine does not @@ -513,12 +512,12 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * resulting RDD with into `numPartitions` partitions. The ordering of elements within * each group is not guaranteed, and may even differ each time the resulting RDD is evaluated. * - * Note: This operation may be very expensive. If you are grouping in order to perform an - * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] - * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. + * @note This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey` + * or `PairRDDFunctions.reduceByKey` will provide much better performance. * - * Note: As currently implemented, groupByKey must be able to hold all the key-value pairs for any - * key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]]. + * @note As currently implemented, groupByKey must be able to hold all the key-value pairs for any + * key in memory. If a key has too many values, it can result in an `OutOfMemoryError`. */ def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = self.withScope { groupByKey(new HashPartitioner(numPartitions)) @@ -529,7 +528,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def partitionBy(partitioner: Partitioner): RDD[(K, V)] = self.withScope { if (keyClass.isArray && partitioner.isInstanceOf[HashPartitioner]) { - throw new SparkException("Default partitioner cannot partition array keys.") + throw new SparkException("HashPartitioner cannot partition array keys.") } if (self.partitioner == Some(partitioner)) { self @@ -606,7 +605,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * existing partitioner/parallelism level. This method is here for backward compatibility. It * does not provide combiner classtag information to the shuffle. * - * @see [[combineByKeyWithClassTag]] + * @see `combineByKeyWithClassTag` */ def combineByKey[C]( createCombiner: V => C, @@ -634,9 +633,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * within each group is not guaranteed, and may even differ each time the resulting RDD is * evaluated. * - * Note: This operation may be very expensive. If you are grouping in order to perform an - * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] - * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. + * @note This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey` + * or `PairRDDFunctions.reduceByKey` will provide much better performance. */ def groupByKey(): RDD[(K, Iterable[V])] = self.withScope { groupByKey(defaultPartitioner(self)) @@ -736,6 +735,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * * Warning: this doesn't return a multimap (so if you have multiple values to the same key, only * one value per key is preserved in the map returned) + * + * @note this method should only be used if the resulting data is expected to be small, as + * all the data is loaded into the driver's memory. */ def collectAsMap(): Map[K, V] = self.withScope { val data = self.collect() @@ -780,7 +782,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) partitioner: Partitioner) : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = self.withScope { if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { - throw new SparkException("Default partitioner cannot partition array keys.") + throw new SparkException("HashPartitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2, other3), partitioner) cg.mapValues { case Array(vs, w1s, w2s, w3s) => @@ -798,7 +800,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Iterable[V], Iterable[W]))] = self.withScope { if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { - throw new SparkException("Default partitioner cannot partition array keys.") + throw new SparkException("HashPartitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other), partitioner) cg.mapValues { case Array(vs, w1s) => @@ -813,7 +815,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = self.withScope { if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { - throw new SparkException("Default partitioner cannot partition array keys.") + throw new SparkException("HashPartitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) cg.mapValues { case Array(vs, w1s, w2s) => @@ -903,20 +905,24 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Return an RDD with the pairs from `this` whose keys are not in `other`. * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting - * RDD will be <= us. + * RDD will be less than or equal to us. */ def subtractByKey[W: ClassTag](other: RDD[(K, W)]): RDD[(K, V)] = self.withScope { subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.length))) } - /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ + /** + * Return an RDD with the pairs from `this` whose keys are not in `other`. + */ def subtractByKey[W: ClassTag]( other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] = self.withScope { subtractByKey(other, new HashPartitioner(numPartitions)) } - /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ + /** + * Return an RDD with the pairs from `this` whose keys are not in `other`. + */ def subtractByKey[W: ClassTag](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] = self.withScope { new SubtractedRDD[K, V, W](self, other, p) } @@ -985,12 +991,12 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) conf: Configuration = self.context.hadoopConfiguration): Unit = self.withScope { // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). val hadoopConf = conf - val job = new NewAPIHadoopJob(hadoopConf) + val job = NewAPIHadoopJob.getInstance(hadoopConf) job.setOutputKeyClass(keyClass) job.setOutputValueClass(valueClass) job.setOutputFormatClass(outputFormatClass) - val jobConfiguration = SparkHadoopUtil.get.getConfigurationFromJobContext(job) - jobConfiguration.set("mapred.output.dir", path) + val jobConfiguration = job.getConfiguration + jobConfiguration.set("mapreduce.output.fileoutputformat.outputdir", path) saveAsNewAPIHadoopDataset(jobConfiguration) } @@ -1012,7 +1018,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class * supporting the key and value types K and V in this RDD. * - * Note that, we should make sure our tasks are idempotent when speculation is enabled, i.e. do + * @note We should make sure our tasks are idempotent when speculation is enabled, i.e. do * not use output committer that writes data directly. * There is an example in https://issues.apache.org/jira/browse/SPARK-10063 to show the bad * result of using direct output committer with speculation enabled. @@ -1031,10 +1037,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) conf.setOutputFormat(outputFormatClass) for (c <- codec) { hadoopConf.setCompressMapOutput(true) - hadoopConf.set("mapred.output.compress", "true") + hadoopConf.set("mapreduce.output.fileoutputformat.compress", "true") hadoopConf.setMapOutputCompressorClass(c) - hadoopConf.set("mapred.output.compression.codec", c.getCanonicalName) - hadoopConf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) + hadoopConf.set("mapreduce.output.fileoutputformat.compress.codec", c.getCanonicalName) + hadoopConf.set("mapreduce.output.fileoutputformat.compress.type", + CompressionType.BLOCK.toString) } // Use configured output committer if already set @@ -1050,13 +1057,13 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val warningMessage = s"$outputCommitterClass may be an output committer that writes data directly to " + "the final location. Because speculation is enabled, this output committer may " + - "cause data loss (see the case in SPARK-10063). If possible, please use a output " + + "cause data loss (see the case in SPARK-10063). If possible, please use an output " + "committer that does not have this behavior (e.g. FileOutputCommitter)." logWarning(warningMessage) } FileOutputFormat.setOutputPath(hadoopConf, - SparkHadoopWriter.createPathFromString(path, hadoopConf)) + SparkHadoopWriterUtils.createPathFromString(path, hadoopConf)) saveAsHadoopDataset(hadoopConf) } @@ -1066,85 +1073,16 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * output paths required (e.g. a table name to write to) in the same way as it would be * configured for a Hadoop MapReduce job. * - * Note that, we should make sure our tasks are idempotent when speculation is enabled, i.e. do + * @note We should make sure our tasks are idempotent when speculation is enabled, i.e. do * not use output committer that writes data directly. * There is an example in https://issues.apache.org/jira/browse/SPARK-10063 to show the bad * result of using direct output committer with speculation enabled. */ def saveAsNewAPIHadoopDataset(conf: Configuration): Unit = self.withScope { - // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). - val hadoopConf = conf - val job = new NewAPIHadoopJob(hadoopConf) - val formatter = new SimpleDateFormat("yyyyMMddHHmm") - val jobtrackerID = formatter.format(new Date()) - val stageId = self.id - val jobConfiguration = SparkHadoopUtil.get.getConfigurationFromJobContext(job) - val wrappedConf = new SerializableConfiguration(jobConfiguration) - val outfmt = job.getOutputFormatClass - val jobFormat = outfmt.newInstance - - if (isOutputSpecValidationEnabled) { - // FileOutputFormat ignores the filesystem parameter - jobFormat.checkOutputSpecs(job) - } - - val writeShard = (context: TaskContext, iter: Iterator[(K, V)]) => { - val config = wrappedConf.value - /* "reduce task" */ - val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, - context.attemptNumber) - val hadoopContext = newTaskAttemptContext(config, attemptId) - val format = outfmt.newInstance - format match { - case c: Configurable => c.setConf(config) - case _ => () - } - val committer = format.getOutputCommitter(hadoopContext) - committer.setupTask(hadoopContext) - - val (outputMetrics, bytesWrittenCallback) = initHadoopOutputMetrics(context) - - val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K, V]] - require(writer != null, "Unable to obtain RecordWriter") - var recordsWritten = 0L - Utils.tryWithSafeFinally { - while (iter.hasNext) { - val pair = iter.next() - writer.write(pair._1, pair._2) - - // Update bytes written metric every few records - maybeUpdateOutputMetrics(bytesWrittenCallback, outputMetrics, recordsWritten) - recordsWritten += 1 - } - } { - writer.close(hadoopContext) - } - committer.commitTask(hadoopContext) - bytesWrittenCallback.foreach { fn => outputMetrics.setBytesWritten(fn()) } - outputMetrics.setRecordsWritten(recordsWritten) - 1 - } : Int - - val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = true, 0, 0) - val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId) - val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext) - - // When speculation is on and output committer class name contains "Direct", we should warn - // users that they may loss data if they are using a direct output committer. - val speculationEnabled = self.conf.getBoolean("spark.speculation", false) - val outputCommitterClass = jobCommitter.getClass.getSimpleName - if (speculationEnabled && outputCommitterClass.contains("Direct")) { - val warningMessage = - s"$outputCommitterClass may be an output committer that writes data directly to " + - "the final location. Because speculation is enabled, this output committer may " + - "cause data loss (see the case in SPARK-10063). If possible, please use a output " + - "committer that does not have this behavior (e.g. FileOutputCommitter)." - logWarning(warningMessage) - } - - jobCommitter.setupJob(jobTaskContext) - self.context.runJob(self, writeShard) - jobCommitter.commitJob(jobTaskContext) + val config = new HadoopMapReduceWriteConfigUtil[K, V](new SerializableConfiguration(conf)) + SparkHadoopWriter.write( + rdd = self, + config = config) } /** @@ -1154,81 +1092,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * MapReduce job. */ def saveAsHadoopDataset(conf: JobConf): Unit = self.withScope { - // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). - val hadoopConf = conf - val outputFormatInstance = hadoopConf.getOutputFormat - val keyClass = hadoopConf.getOutputKeyClass - val valueClass = hadoopConf.getOutputValueClass - if (outputFormatInstance == null) { - throw new SparkException("Output format class not set") - } - if (keyClass == null) { - throw new SparkException("Output key class not set") - } - if (valueClass == null) { - throw new SparkException("Output value class not set") - } - SparkHadoopUtil.get.addCredentials(hadoopConf) - - logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + - valueClass.getSimpleName + ")") - - if (isOutputSpecValidationEnabled) { - // FileOutputFormat ignores the filesystem parameter - val ignoredFs = FileSystem.get(hadoopConf) - hadoopConf.getOutputFormat.checkOutputSpecs(ignoredFs, hadoopConf) - } - - val writer = new SparkHadoopWriter(hadoopConf) - writer.preSetup() - - val writeToFile = (context: TaskContext, iter: Iterator[(K, V)]) => { - // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it - // around by taking a mod. We expect that no task will be attempted 2 billion times. - val taskAttemptId = (context.taskAttemptId % Int.MaxValue).toInt - - val (outputMetrics, bytesWrittenCallback) = initHadoopOutputMetrics(context) - - writer.setup(context.stageId, context.partitionId, taskAttemptId) - writer.open() - var recordsWritten = 0L - - Utils.tryWithSafeFinally { - while (iter.hasNext) { - val record = iter.next() - writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) - - // Update bytes written metric every few records - maybeUpdateOutputMetrics(bytesWrittenCallback, outputMetrics, recordsWritten) - recordsWritten += 1 - } - } { - writer.close() - } - writer.commit() - bytesWrittenCallback.foreach { fn => outputMetrics.setBytesWritten(fn()) } - outputMetrics.setRecordsWritten(recordsWritten) - } - - self.context.runJob(self, writeToFile) - writer.commitJob() - } - - private def initHadoopOutputMetrics(context: TaskContext): (OutputMetrics, Option[() => Long]) = { - val bytesWrittenCallback = SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback() - val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop) - if (bytesWrittenCallback.isDefined) { - context.taskMetrics.outputMetrics = Some(outputMetrics) - } - (outputMetrics, bytesWrittenCallback) - } - - private def maybeUpdateOutputMetrics(bytesWrittenCallback: Option[() => Long], - outputMetrics: OutputMetrics, recordsWritten: Long): Unit = { - if (recordsWritten % PairRDDFunctions.RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES == 0) { - bytesWrittenCallback.foreach { fn => outputMetrics.setBytesWritten(fn()) } - outputMetrics.setRecordsWritten(recordsWritten) - } + val config = new HadoopMapRedWriteConfigUtil[K, V](new SerializableJobConf(conf)) + SparkHadoopWriter.write( + rdd = self, + config = config) } /** @@ -1246,22 +1113,4 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) private[spark] def valueClass: Class[_] = vt.runtimeClass private[spark] def keyOrdering: Option[Ordering[K]] = Option(ord) - - // Note: this needs to be a function instead of a 'val' so that the disableOutputSpecValidation - // setting can take effect: - private def isOutputSpecValidationEnabled: Boolean = { - val validationDisabled = PairRDDFunctions.disableOutputSpecValidation.value - val enabledInConf = self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true) - enabledInConf && !validationDisabled - } -} - -private[spark] object PairRDDFunctions { - val RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES = 256 - - /** - * Allows for the `spark.hadoop.validateOutputSpecs` checks to be disabled on a case-by-case - * basis; see SPARK-4835 for more details. - */ - val disableOutputSpecValidation: DynamicVariable[Boolean] = new DynamicVariable[Boolean](false) } diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 582fa93afe34..9f8019b80a4d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -32,8 +32,8 @@ import org.apache.spark.util.Utils private[spark] class ParallelCollectionPartition[T: ClassTag]( var rddId: Long, var slice: Int, - var values: Seq[T]) - extends Partition with Serializable { + var values: Seq[T] + ) extends Partition with Serializable { def iterator: Iterator[T] = values.iterator @@ -116,20 +116,20 @@ private object ParallelCollectionRDD { */ def slice[T: ClassTag](seq: Seq[T], numSlices: Int): Seq[Seq[T]] = { if (numSlices < 1) { - throw new IllegalArgumentException("Positive number of slices required") + throw new IllegalArgumentException("Positive number of partitions required") } // Sequences need to be sliced at the same set of index positions for operations // like RDD.zip() to behave as expected def positions(length: Long, numSlices: Int): Iterator[(Int, Int)] = { - (0 until numSlices).iterator.map(i => { + (0 until numSlices).iterator.map { i => val start = ((i * length) / numSlices).toInt val end = (((i + 1) * length) / numSlices).toInt (start, end) - }) + } } seq match { - case r: Range => { - positions(r.length, numSlices).zipWithIndex.map({ case ((start, end), index) => + case r: Range => + positions(r.length, numSlices).zipWithIndex.map { case ((start, end), index) => // If the range is inclusive, use inclusive range for the last slice if (r.isInclusive && index == numSlices - 1) { new Range.Inclusive(r.start + start * r.step, r.end, r.step) @@ -137,9 +137,8 @@ private object ParallelCollectionRDD { else { new Range(r.start + start * r.step, r.start + end * r.step, r.step) } - }).toSeq.asInstanceOf[Seq[Seq[T]]] - } - case nr: NumericRange[_] => { + }.toSeq.asInstanceOf[Seq[Seq[T]]] + case nr: NumericRange[_] => // For ranges of Long, Double, BigInteger, etc val slices = new ArrayBuffer[Seq[T]](numSlices) var r = nr @@ -149,14 +148,11 @@ private object ParallelCollectionRDD { r = r.drop(sliceSize) } slices - } - case _ => { + case _ => val array = seq.toArray // To prevent O(n^2) operations for List etc - positions(array.length, numSlices).map({ - case (start, end) => + positions(array.length, numSlices).map { case (start, end) => array.slice(start, end).toSeq - }).toSeq - } + }.toSeq } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index d6a37e8cc5da..ce75a16031a3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -48,7 +48,7 @@ private[spark] class PruneDependency[T](rdd: RDD[T], partitionFilterFunc: Int => /** * :: DeveloperApi :: - * A RDD used to prune RDD partitions/partitions so we can avoid launching tasks on + * An RDD used to prune RDD partitions/partitions so we can avoid launching tasks on * all partitions. An example use case: If we know the RDD is partitioned by range, * and the execution DAG has a filter on the key, we can avoid launching tasks * on partitions that don't have the range covering the key. @@ -65,7 +65,7 @@ class PartitionPruningRDD[T: ClassTag]( } override protected def getPartitions: Array[Partition] = - getDependencies.head.asInstanceOf[PruneDependency[T]].partitions + dependencies.head.asInstanceOf[PruneDependency[T]].partitions } diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala index 9e3880714a79..d744d6759254 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -31,12 +31,13 @@ import org.apache.spark.util.Utils private[spark] class PartitionerAwareUnionRDDPartition( @transient val rdds: Seq[RDD[_]], - val idx: Int + override val index: Int ) extends Partition { - var parents = rdds.map(_.partitions(idx)).toArray + var parents = rdds.map(_.partitions(index)).toArray - override val index = idx - override def hashCode(): Int = idx + override def hashCode(): Int = index + + override def equals(other: Any): Boolean = super.equals(other) @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { @@ -59,7 +60,7 @@ class PartitionerAwareUnionRDD[T: ClassTag]( sc: SparkContext, var rdds: Seq[RDD[T]] ) extends RDD[T](sc, rdds.map(x => new OneToOneDependency(x))) { - require(rdds.length > 0) + require(rdds.nonEmpty) require(rdds.forall(_.partitioner.isDefined)) require(rdds.flatMap(_.partitioner).toSet.size == 1, "Parent RDDs have different partitioners: " + rdds.flatMap(_.partitioner)) @@ -68,9 +69,9 @@ class PartitionerAwareUnionRDD[T: ClassTag]( override def getPartitions: Array[Partition] = { val numPartitions = partitioner.get.numPartitions - (0 until numPartitions).map(index => { + (0 until numPartitions).map { index => new PartitionerAwareUnionRDDPartition(rdds, index) - }).toArray + }.toArray } // Get the location where most of the partitions of parent RDDs are located @@ -78,11 +79,10 @@ class PartitionerAwareUnionRDD[T: ClassTag]( logDebug("Finding preferred location for " + this + ", partition " + s.index) val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].parents val locations = rdds.zip(parentPartitions).flatMap { - case (rdd, part) => { + case (rdd, part) => val parentLocations = currPrefLocs(rdd, part) logDebug("Location of " + rdd + " partition " + part.index + " = " + parentLocations) parentLocations - } } val location = if (locations.isEmpty) { None diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala index 3b1acacf409b..15691a8fc8ea 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala @@ -22,8 +22,8 @@ import java.util.Random import scala.reflect.ClassTag import org.apache.spark.{Partition, TaskContext} -import org.apache.spark.util.random.RandomSampler import org.apache.spark.util.Utils +import org.apache.spark.util.random.RandomSampler private[spark] class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) @@ -32,7 +32,7 @@ class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) } /** - * A RDD sampled from its parent RDD partition-wise. For each partition of the parent RDD, + * An RDD sampled from its parent RDD partition-wise. For each partition of the parent RDD, * a user-specified [[org.apache.spark.util.random.RandomSampler]] instance is used to obtain * a random sample of the records in the partition. The random seeds assigned to the samplers * are guaranteed to have different values. diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index afbe566b7656..02b28b72fb0e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -17,11 +17,14 @@ package org.apache.spark.rdd +import java.io.BufferedWriter import java.io.File import java.io.FilenameFilter import java.io.IOException +import java.io.OutputStreamWriter import java.io.PrintWriter import java.util.StringTokenizer +import java.util.concurrent.atomic.AtomicReference import scala.collection.JavaConverters._ import scala.collection.Map @@ -43,22 +46,11 @@ private[spark] class PipedRDD[T: ClassTag]( envVars: Map[String, String], printPipeContext: (String => Unit) => Unit, printRDDElement: (T, String => Unit) => Unit, - separateWorkingDir: Boolean) + separateWorkingDir: Boolean, + bufferSize: Int, + encoding: String) extends RDD[String](prev) { - // Similar to Runtime.exec(), if we are given a single string, split it into words - // using a standard StringTokenizer (i.e. by spaces) - def this( - prev: RDD[T], - command: String, - envVars: Map[String, String] = Map(), - printPipeContext: (String => Unit) => Unit = null, - printRDDElement: (T, String => Unit) => Unit = null, - separateWorkingDir: Boolean = false) = - this(prev, PipedRDD.tokenize(command), envVars, printPipeContext, printRDDElement, - separateWorkingDir) - - override def getPartitions: Array[Partition] = firstParent[T].partitions /** @@ -118,63 +110,99 @@ private[spark] class PipedRDD[T: ClassTag]( val proc = pb.start() val env = SparkEnv.get + val childThreadException = new AtomicReference[Throwable](null) // Start a thread to print the process's stderr to ours - new Thread("stderr reader for " + command) { - override def run() { - for (line <- Source.fromInputStream(proc.getErrorStream).getLines) { - // scalastyle:off println - System.err.println(line) - // scalastyle:on println + new Thread(s"stderr reader for $command") { + override def run(): Unit = { + val err = proc.getErrorStream + try { + for (line <- Source.fromInputStream(err)(encoding).getLines) { + // scalastyle:off println + System.err.println(line) + // scalastyle:on println + } + } catch { + case t: Throwable => childThreadException.set(t) + } finally { + err.close() } } }.start() // Start a thread to feed the process input from our parent's iterator - new Thread("stdin writer for " + command) { - override def run() { + new Thread(s"stdin writer for $command") { + override def run(): Unit = { TaskContext.setTaskContext(context) - val out = new PrintWriter(proc.getOutputStream) - - // scalastyle:off println - // input the pipe context firstly - if (printPipeContext != null) { - printPipeContext(out.println(_)) - } - for (elem <- firstParent[T].iterator(split, context)) { - if (printRDDElement != null) { - printRDDElement(elem, out.println(_)) - } else { - out.println(elem) + val out = new PrintWriter(new BufferedWriter( + new OutputStreamWriter(proc.getOutputStream, encoding), bufferSize)) + try { + // scalastyle:off println + // input the pipe context firstly + if (printPipeContext != null) { + printPipeContext(out.println) } + for (elem <- firstParent[T].iterator(split, context)) { + if (printRDDElement != null) { + printRDDElement(elem, out.println) + } else { + out.println(elem) + } + } + // scalastyle:on println + } catch { + case t: Throwable => childThreadException.set(t) + } finally { + out.close() } - // scalastyle:on println - out.close() } }.start() // Return an iterator that read lines from the process's stdout - val lines = Source.fromInputStream(proc.getInputStream).getLines() + val lines = Source.fromInputStream(proc.getInputStream)(encoding).getLines new Iterator[String] { - def next(): String = lines.next() - def hasNext: Boolean = { - if (lines.hasNext) { + def next(): String = { + if (!hasNext()) { + throw new NoSuchElementException() + } + lines.next() + } + + def hasNext(): Boolean = { + val result = if (lines.hasNext) { true } else { val exitStatus = proc.waitFor() + cleanup() if (exitStatus != 0) { - throw new Exception("Subprocess exited with status " + exitStatus) + throw new IllegalStateException(s"Subprocess exited with status $exitStatus. " + + s"Command ran: " + command.mkString(" ")) } + false + } + propagateChildException() + result + } - // cleanup task working directory if used - if (workInTaskDirectory) { - scala.util.control.Exception.ignoring(classOf[IOException]) { - Utils.deleteRecursively(new File(taskDirectory)) - } - logDebug("Removed task working directory " + taskDirectory) + private def cleanup(): Unit = { + // cleanup task working directory if used + if (workInTaskDirectory) { + scala.util.control.Exception.ignoring(classOf[IOException]) { + Utils.deleteRecursively(new File(taskDirectory)) } + logDebug(s"Removed task working directory $taskDirectory") + } + } - false + private def propagateChildException(): Unit = { + val t = childThreadException.get() + if (t != null) { + val commandRan = command.mkString(" ") + logError(s"Caught exception while running pipe() operator. Command ran: $commandRan. " + + s"Exception: ${t.getMessage}") + proc.destroy() + cleanup() + throw t } } } 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 800ef53cbef0..8798dfc92536 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -21,8 +21,10 @@ import java.util.Random import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer +import scala.io.Codec import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} +import scala.util.hashing import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import org.apache.hadoop.io.{BytesWritable, NullWritable, Text} @@ -31,16 +33,17 @@ import org.apache.hadoop.mapred.TextOutputFormat import org.apache.spark._ import org.apache.spark.Partitioner._ -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult -import org.apache.spark.storage.StorageLevel +import org.apache.spark.storage.{RDDBlockId, StorageLevel} import org.apache.spark.util.{BoundedPriorityQueue, Utils} -import org.apache.spark.util.collection.OpenHashMap -import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, BernoulliCellSampler, +import org.apache.spark.util.collection.{OpenHashMap, Utils => collectionUtils} +import org.apache.spark.util.random.{BernoulliCellSampler, BernoulliSampler, PoissonSampler, SamplingUtils} /** @@ -53,7 +56,7 @@ import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, Bernoulli * Doubles; and * [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that * can be saved as SequenceFiles. - * All operations are automatically available on any RDD of the right type (e.g. RDD[(Int, Int)] + * All operations are automatically available on any RDD of the right type (e.g. RDD[(Int, Int)]) * through implicit. * * Internally, each RDD is characterized by five main properties: @@ -68,8 +71,8 @@ import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, Bernoulli * All of the scheduling and execution in Spark is done based on these methods, allowing each RDD * to implement its own way of computing itself. Indeed, users can implement custom RDDs (e.g. for * reading data from a new storage system) by overriding these functions. Please refer to the - * [[http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details - * on RDD internals. + * Spark paper + * for more details on RDD internals. */ abstract class RDD[T: ClassTag]( @transient private var _sc: SparkContext, @@ -85,17 +88,21 @@ abstract class RDD[T: ClassTag]( private def sc: SparkContext = { if (_sc == null) { throw new SparkException( - "RDD transformations and actions can only be invoked by the driver, not inside of other " + - "transformations; for example, rdd1.map(x => rdd2.values.count() * x) is invalid because " + - "the values transformation and count action cannot be performed inside of the rdd1.map " + - "transformation. For more information, see SPARK-5063.") + "This RDD lacks a SparkContext. It could happen in the following cases: \n(1) RDD " + + "transformations and actions are NOT invoked by the driver, but inside of other " + + "transformations; for example, rdd1.map(x => rdd2.values.count() * x) is invalid " + + "because the values transformation and count action cannot be performed inside of the " + + "rdd1.map transformation. For more information, see SPARK-5063.\n(2) When a Spark " + + "Streaming job recovers from checkpoint, this exception will be hit if a reference to " + + "an RDD not defined by the streaming job is used in DStream operations. For more " + + "information, See SPARK-13758.") } _sc } /** Construct an RDD with just a one-to-one dependency on one parent */ def this(@transient oneParent: RDD[_]) = - this(oneParent.context , List(new OneToOneDependency(oneParent))) + this(oneParent.context, List(new OneToOneDependency(oneParent))) private[spark] def conf = sc.conf // ======================================================================= @@ -112,6 +119,9 @@ abstract class RDD[T: ClassTag]( /** * Implemented by subclasses to return the set of partitions in this RDD. This method will only * be called once, so it is safe to implement a time-consuming computation in it. + * + * The partitions in this array must satisfy the following property: + * `rdd.partitions.zipWithIndex.forall { case (partition, index) => partition.index == index }` */ protected def getPartitions: Array[Partition] @@ -186,10 +196,14 @@ abstract class RDD[T: ClassTag]( } } - /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + /** + * Persist this RDD with the default storage level (`MEMORY_ONLY`). + */ def persist(): this.type = persist(StorageLevel.MEMORY_ONLY) - /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + /** + * Persist this RDD with the default storage level (`MEMORY_ONLY`). + */ def cache(): this.type = persist() /** @@ -237,11 +251,21 @@ abstract class RDD[T: ClassTag]( checkpointRDD.map(_.partitions).getOrElse { if (partitions_ == null) { partitions_ = getPartitions + partitions_.zipWithIndex.foreach { case (partition, index) => + require(partition.index == index, + s"partitions($index).partition == ${partition.index}, but it should equal $index") + } } partitions_ } } + /** + * Returns the number of partitions of this RDD. + */ + @Since("1.6.0") + final def getNumPartitions: Int = partitions.length + /** * Get the preferred locations of a partition, taking into account whether the * RDD is checkpointed. @@ -259,7 +283,7 @@ abstract class RDD[T: ClassTag]( */ final def iterator(split: Partition, context: TaskContext): Iterator[T] = { if (storageLevel != StorageLevel.NONE) { - SparkEnv.get.cacheManager.getOrCompute(this, split, context, storageLevel) + getOrCompute(split, context) } else { computeOrReadCheckpoint(split, context) } @@ -301,6 +325,35 @@ abstract class RDD[T: ClassTag]( } } + /** + * Gets or computes an RDD partition. Used by RDD.iterator() when an RDD is cached. + */ + private[spark] def getOrCompute(partition: Partition, context: TaskContext): Iterator[T] = { + val blockId = RDDBlockId(id, partition.index) + var readCachedBlock = true + // This method is called on executors, so we need call SparkEnv.get instead of sc.env. + SparkEnv.get.blockManager.getOrElseUpdate(blockId, storageLevel, elementClassTag, () => { + readCachedBlock = false + computeOrReadCheckpoint(partition, context) + }) match { + case Left(blockResult) => + if (readCachedBlock) { + val existingMetrics = context.taskMetrics().inputMetrics + existingMetrics.incBytesRead(blockResult.bytes) + new InterruptibleIterator[T](context, blockResult.data.asInstanceOf[Iterator[T]]) { + override def next(): T = { + existingMetrics.incRecordsRead(1) + delegate.next() + } + } + } else { + new InterruptibleIterator(context, blockResult.data.asInstanceOf[Iterator[T]]) + } + case Right(iter) => + new InterruptibleIterator(context, iter.asInstanceOf[Iterator[T]]) + } + } + /** * Execute a block of code in a scope such that all new RDDs created in this body will * be part of the same scope. For more detail, see {{org.apache.spark.rdd.RDDOperationScope}}. @@ -371,7 +424,8 @@ abstract class RDD[T: ClassTag]( * * This results in a narrow dependency, e.g. if you go from 1000 partitions * to 100 partitions, there will not be a shuffle, instead each of the 100 - * new partitions will claim 10 of the current partitions. + * new partitions will claim 10 of the current partitions. If a larger number + * of partitions is requested, it will stay at the current number of partitions. * * However, if you're doing a drastic coalesce, e.g. to numPartitions = 1, * this may result in your computation taking place on fewer nodes than @@ -380,18 +434,22 @@ abstract class RDD[T: ClassTag]( * current upstream partitions will be executed in parallel (per whatever * the current partitioning is). * - * Note: With shuffle = true, you can actually coalesce to a larger number + * @note With shuffle = true, you can actually coalesce to a larger number * of partitions. This is useful if you have a small number of partitions, * say 100, potentially with a few partitions being abnormally large. Calling * coalesce(1000, shuffle = true) will result in 1000 partitions with the - * data distributed using a hash partitioner. + * data distributed using a hash partitioner. The optional partition coalescer + * passed in must be serializable. */ - def coalesce(numPartitions: Int, shuffle: Boolean = false)(implicit ord: Ordering[T] = null) + def coalesce(numPartitions: Int, shuffle: Boolean = false, + partitionCoalescer: Option[PartitionCoalescer] = Option.empty) + (implicit ord: Ordering[T] = null) : RDD[T] = withScope { + require(numPartitions > 0, s"Number of partitions ($numPartitions) must be positive.") if (shuffle) { /** Distributes elements evenly across output partitions, starting from a random partition. */ val distributePartition = (index: Int, items: Iterator[T]) => { - var position = (new Random(index)).nextInt(numPartitions) + var position = (new Random(hashing.byteswap32(index))).nextInt(numPartitions) items.map { t => // Note that the hash code of the key will just be the key itself. The HashPartitioner // will mod it with the number of total partitions. @@ -404,9 +462,10 @@ abstract class RDD[T: ClassTag]( new CoalescedRDD( new ShuffledRDD[Int, T, T](mapPartitionsWithIndex(distributePartition), new HashPartitioner(numPartitions)), - numPartitions).values + numPartitions, + partitionCoalescer).values } else { - new CoalescedRDD(this, numPartitions) + new CoalescedRDD(this, numPartitions, partitionCoalescer) } } @@ -416,18 +475,27 @@ abstract class RDD[T: ClassTag]( * @param withReplacement can elements be sampled multiple times (replaced when sampled out) * @param fraction expected size of the sample as a fraction of this RDD's size * without replacement: probability that each element is chosen; fraction must be [0, 1] - * with replacement: expected number of times each element is chosen; fraction must be >= 0 + * with replacement: expected number of times each element is chosen; fraction must be greater + * than or equal to 0 * @param seed seed for the random number generator + * + * @note This is NOT guaranteed to provide exactly the fraction of the count + * of the given [[RDD]]. */ def sample( withReplacement: Boolean, fraction: Double, - seed: Long = Utils.random.nextLong): RDD[T] = withScope { - require(fraction >= 0.0, "Negative fraction value: " + fraction) - if (withReplacement) { - new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), true, seed) - } else { - new PartitionwiseSampledRDD[T, T](this, new BernoulliSampler[T](fraction), true, seed) + seed: Long = Utils.random.nextLong): RDD[T] = { + require(fraction >= 0, + s"Fraction must be nonnegative, but got ${fraction}") + + withScope { + require(fraction >= 0.0, "Negative fraction value: " + fraction) + if (withReplacement) { + new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), true, seed) + } else { + new PartitionwiseSampledRDD[T, T](this, new BernoulliSampler[T](fraction), true, seed) + } } } @@ -441,14 +509,22 @@ abstract class RDD[T: ClassTag]( */ def randomSplit( weights: Array[Double], - seed: Long = Utils.random.nextLong): Array[RDD[T]] = withScope { - val sum = weights.sum - val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) - normalizedCumWeights.sliding(2).map { x => - randomSampleWithRange(x(0), x(1), seed) - }.toArray + seed: Long = Utils.random.nextLong): Array[RDD[T]] = { + require(weights.forall(_ >= 0), + s"Weights must be nonnegative, but got ${weights.mkString("[", ",", "]")}") + require(weights.sum > 0, + s"Sum of weights must be positive, but got ${weights.mkString("[", ",", "]")}") + + withScope { + val sum = weights.sum + val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) + normalizedCumWeights.sliding(2).map { x => + randomSampleWithRange(x(0), x(1), seed) + }.toArray + } } + /** * Internal method exposed for Random Splits in DataFrames. Samples an RDD given a probability * range. @@ -472,6 +548,9 @@ abstract class RDD[T: ClassTag]( * @param num size of the returned sample * @param seed seed for the random number generator * @return sample of specified size in an array + * + * @note this method should only be used if the resulting array is expected to be small, as + * all the data is loaded into the driver's memory. */ def takeSample( withReplacement: Boolean, @@ -518,11 +597,7 @@ abstract class RDD[T: ClassTag]( * times (use `.distinct()` to eliminate them). */ def union(other: RDD[T]): RDD[T] = withScope { - if (partitioner.isDefined && other.partitioner == partitioner) { - new PartitionerAwareUnionRDD(sc, Array(this, other)) - } else { - new UnionRDD(sc, Array(this, other)) - } + sc.union(this, other) } /** @@ -550,7 +625,7 @@ abstract class RDD[T: ClassTag]( * Return the intersection of this RDD and another one. The output will not contain any duplicate * elements, even if the input RDDs did. * - * Note that this method performs a shuffle internally. + * @note This method performs a shuffle internally. */ def intersection(other: RDD[T]): RDD[T] = withScope { this.map(v => (v, null)).cogroup(other.map(v => (v, null))) @@ -562,7 +637,7 @@ abstract class RDD[T: ClassTag]( * Return the intersection of this RDD and another one. The output will not contain any duplicate * elements, even if the input RDDs did. * - * Note that this method performs a shuffle internally. + * @note This method performs a shuffle internally. * * @param partitioner Partitioner to use for the resulting RDD */ @@ -578,7 +653,7 @@ abstract class RDD[T: ClassTag]( * Return the intersection of this RDD and another one. The output will not contain any duplicate * elements, even if the input RDDs did. Performs a hash partition across the cluster * - * Note that this method performs a shuffle internally. + * @note This method performs a shuffle internally. * * @param numPartitions How many partitions to use in the resulting RDD */ @@ -606,9 +681,9 @@ abstract class RDD[T: ClassTag]( * mapping to that key. The ordering of elements within each group is not guaranteed, and * may even differ each time the resulting RDD is evaluated. * - * Note: This operation may be very expensive. If you are grouping in order to perform an - * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] - * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. + * @note This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey` + * or `PairRDDFunctions.reduceByKey` will provide much better performance. */ def groupBy[K](f: T => K)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = withScope { groupBy[K](f, defaultPartitioner(this)) @@ -619,9 +694,9 @@ abstract class RDD[T: ClassTag]( * mapping to that key. The ordering of elements within each group is not guaranteed, and * may even differ each time the resulting RDD is evaluated. * - * Note: This operation may be very expensive. If you are grouping in order to perform an - * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] - * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. + * @note This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey` + * or `PairRDDFunctions.reduceByKey` will provide much better performance. */ def groupBy[K]( f: T => K, @@ -634,9 +709,9 @@ abstract class RDD[T: ClassTag]( * mapping to that key. The ordering of elements within each group is not guaranteed, and * may even differ each time the resulting RDD is evaluated. * - * Note: This operation may be very expensive. If you are grouping in order to perform an - * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] - * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. + * @note This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using `PairRDDFunctions.aggregateByKey` + * or `PairRDDFunctions.reduceByKey` will provide much better performance. */ def groupBy[K](f: T => K, p: Partitioner)(implicit kt: ClassTag[K], ord: Ordering[K] = null) : RDD[(K, Iterable[T])] = withScope { @@ -648,18 +723,28 @@ abstract class RDD[T: ClassTag]( * Return an RDD created by piping elements to a forked external process. */ def pipe(command: String): RDD[String] = withScope { - new PipedRDD(this, command) + // Similar to Runtime.exec(), if we are given a single string, split it into words + // using a standard StringTokenizer (i.e. by spaces) + pipe(PipedRDD.tokenize(command)) } /** * Return an RDD created by piping elements to a forked external process. */ def pipe(command: String, env: Map[String, String]): RDD[String] = withScope { - new PipedRDD(this, command, env) + // Similar to Runtime.exec(), if we are given a single string, split it into words + // using a standard StringTokenizer (i.e. by spaces) + pipe(PipedRDD.tokenize(command), env) } /** - * Return an RDD created by piping elements to a forked external process. + * Return an RDD created by piping elements to a forked external process. The resulting RDD + * is computed by executing the given process once per partition. All elements + * of each input partition are written to a process's stdin as lines of input separated + * by a newline. The resulting partition consists of the process's stdout output, with + * each line of stdout resulting in one element of the output partition. A process is invoked + * even for empty partitions. + * * The print behavior can be customized by providing two functions. * * @param command command to run in forked process. @@ -672,9 +757,14 @@ abstract class RDD[T: ClassTag]( * print line function (like out.println()) as the 2nd parameter. * An example of pipe the RDD data of groupBy() in a streaming way, * instead of constructing a huge String to concat all the elements: - * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = - * for (e <- record._2){f(e)} + * {{{ + * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = + * for (e <- record._2) {f(e)} + * }}} * @param separateWorkingDir Use separate working directories for each task. + * @param bufferSize Buffer size for the stdin writer for the piped process. + * @param encoding Char encoding used for interacting (via stdin, stdout and stderr) with + * the piped process * @return the result RDD */ def pipe( @@ -682,11 +772,15 @@ abstract class RDD[T: ClassTag]( env: Map[String, String] = Map(), printPipeContext: (String => Unit) => Unit = null, printRDDElement: (T, String => Unit) => Unit = null, - separateWorkingDir: Boolean = false): RDD[String] = withScope { + separateWorkingDir: Boolean = false, + bufferSize: Int = 8192, + encoding: String = Codec.defaultCharsetCodec.name): RDD[String] = withScope { new PipedRDD(this, command, env, if (printPipeContext ne null) sc.clean(printPipeContext) else null, if (printRDDElement ne null) sc.clean(printRDDElement) else null, - separateWorkingDir) + separateWorkingDir, + bufferSize, + encoding) } /** @@ -706,113 +800,50 @@ abstract class RDD[T: ClassTag]( } /** - * Return a new RDD by applying a function to each partition of this RDD, while tracking the index - * of the original partition. + * [performance] Spark's internal mapPartitionsWithIndex method that skips closure cleaning. + * It is a performance API to be used carefully only if we are sure that the RDD elements are + * serializable and don't require closure cleaning. * - * `preservesPartitioning` indicates whether the input function preserves the partitioner, which - * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. + * @param preservesPartitioning indicates whether the input function preserves the partitioner, + * which should be `false` unless this is a pair RDD and the input function doesn't modify + * the keys. */ - def mapPartitionsWithIndex[U: ClassTag]( + private[spark] def mapPartitionsWithIndexInternal[U: ClassTag]( f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = withScope { - val cleanedF = sc.clean(f) new MapPartitionsRDD( this, - (context: TaskContext, index: Int, iter: Iterator[T]) => cleanedF(index, iter), + (context: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter), preservesPartitioning) } /** - * :: DeveloperApi :: - * Return a new RDD by applying a function to each partition of this RDD. This is a variant of - * mapPartitions that also passes the TaskContext into the closure. - * - * `preservesPartitioning` indicates whether the input function preserves the partitioner, which - * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. + * [performance] Spark's internal mapPartitions method that skips closure cleaning. */ - @DeveloperApi - @deprecated("use TaskContext.get", "1.2.0") - def mapPartitionsWithContext[U: ClassTag]( - f: (TaskContext, Iterator[T]) => Iterator[U], + private[spark] def mapPartitionsInternal[U: ClassTag]( + f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = withScope { - val cleanF = sc.clean(f) - val func = (context: TaskContext, index: Int, iter: Iterator[T]) => cleanF(context, iter) - new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning) + new MapPartitionsRDD( + this, + (context: TaskContext, index: Int, iter: Iterator[T]) => f(iter), + preservesPartitioning) } /** * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. + * + * `preservesPartitioning` indicates whether the input function preserves the partitioner, which + * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. */ - @deprecated("use mapPartitionsWithIndex", "0.7.0") - def mapPartitionsWithSplit[U: ClassTag]( + def mapPartitionsWithIndex[U: ClassTag]( f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = withScope { - mapPartitionsWithIndex(f, preservesPartitioning) - } - - /** - * Maps f over this RDD, where f takes an additional parameter of type A. This - * additional parameter is produced by constructA, which is called in each - * partition with the index of that partition. - */ - @deprecated("use mapPartitionsWithIndex", "1.0.0") - def mapWith[A, U: ClassTag] - (constructA: Int => A, preservesPartitioning: Boolean = false) - (f: (T, A) => U): RDD[U] = withScope { - val cleanF = sc.clean(f) - val cleanA = sc.clean(constructA) - mapPartitionsWithIndex((index, iter) => { - val a = cleanA(index) - iter.map(t => cleanF(t, a)) - }, preservesPartitioning) - } - - /** - * FlatMaps f over this RDD, where f takes an additional parameter of type A. This - * additional parameter is produced by constructA, which is called in each - * partition with the index of that partition. - */ - @deprecated("use mapPartitionsWithIndex and flatMap", "1.0.0") - def flatMapWith[A, U: ClassTag] - (constructA: Int => A, preservesPartitioning: Boolean = false) - (f: (T, A) => Seq[U]): RDD[U] = withScope { - val cleanF = sc.clean(f) - val cleanA = sc.clean(constructA) - mapPartitionsWithIndex((index, iter) => { - val a = cleanA(index) - iter.flatMap(t => cleanF(t, a)) - }, preservesPartitioning) - } - - /** - * Applies f to each element of this RDD, where f takes an additional parameter of type A. - * This additional parameter is produced by constructA, which is called in each - * partition with the index of that partition. - */ - @deprecated("use mapPartitionsWithIndex and foreach", "1.0.0") - def foreachWith[A](constructA: Int => A)(f: (T, A) => Unit): Unit = withScope { - val cleanF = sc.clean(f) - val cleanA = sc.clean(constructA) - mapPartitionsWithIndex { (index, iter) => - val a = cleanA(index) - iter.map(t => {cleanF(t, a); t}) - } - } - - /** - * Filters this RDD with p, where p takes an additional parameter of type A. This - * additional parameter is produced by constructA, which is called in each - * partition with the index of that partition. - */ - @deprecated("use mapPartitionsWithIndex and filter", "1.0.0") - def filterWith[A](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = withScope { - val cleanP = sc.clean(p) - val cleanA = sc.clean(constructA) - mapPartitionsWithIndex((index, iter) => { - val a = cleanA(index) - iter.filter(t => cleanP(t, a)) - }, preservesPartitioning = true) + val cleanedF = sc.clean(f) + new MapPartitionsRDD( + this, + (context: TaskContext, index: Int, iter: Iterator[T]) => cleanedF(index, iter), + preservesPartitioning) } /** @@ -898,6 +929,9 @@ abstract class RDD[T: ClassTag]( /** * Return an array that contains all of the elements in this RDD. + * + * @note This method should only be used if the resulting array is expected to be small, as + * all the data is loaded into the driver's memory. */ def collect(): Array[T] = withScope { val results = sc.runJob(this, (iter: Iterator[T]) => iter.toArray) @@ -909,7 +943,7 @@ abstract class RDD[T: ClassTag]( * * The iterator will consume as much memory as the largest partition in this RDD. * - * Note: this results in multiple Spark jobs, and if the input RDD is the result + * @note This results in multiple Spark jobs, and if the input RDD is the result * of a wide transformation (e.g. join with different partitioners), to avoid * recomputing the input RDD should be cached first. */ @@ -920,14 +954,6 @@ abstract class RDD[T: ClassTag]( (0 until partitions.length).iterator.flatMap(i => collectPartition(i)) } - /** - * Return an array that contains all of the elements in this RDD. - */ - @deprecated("use collect", "1.0.0") - def toArray(): Array[T] = withScope { - collect() - } - /** * Return an RDD that contains all matching values by applying `f`. */ @@ -1037,7 +1063,7 @@ abstract class RDD[T: ClassTag]( /** * Aggregate the elements of each partition, and then the results for all the partitions, using a - * given associative and commutative function and a neutral "zero value". The function + * given associative function and a neutral "zero value". The function * op(t1, t2) is allowed to modify t1 and return it as its result value to avoid object * allocation; however, it should not modify t2. * @@ -1047,6 +1073,13 @@ abstract class RDD[T: ClassTag]( * apply the fold to each element sequentially in some defined ordering. For functions * that are not commutative, the result may differ from that of a fold applied to a * non-distributed collection. + * + * @param zeroValue the initial value for the accumulated result of each partition for the `op` + * operator, and also the initial value for the combine results from different + * partitions for the `op` operator - this will typically be the neutral + * element (e.g. `Nil` for list concatenation or `0` for summation) + * @param op an operator used to both accumulate results within a partition and combine results + * from different partitions */ def fold(zeroValue: T)(op: (T, T) => T): T = withScope { // Clone the zero value since we will also be serializing it as part of tasks @@ -1065,6 +1098,13 @@ abstract class RDD[T: ClassTag]( * and one operation for merging two U's, as in scala.TraversableOnce. Both of these functions are * allowed to modify and return their first argument instead of creating a new U to avoid memory * allocation. + * + * @param zeroValue the initial value for the accumulated result of each partition for the + * `seqOp` operator, and also the initial value for the combine results from + * different partitions for the `combOp` operator - this will typically be the + * neutral element (e.g. `Nil` for list concatenation or `0` for summation) + * @param seqOp an operator used to accumulate results within a partition + * @param combOp an associative operator used to combine results from different partitions */ def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = withScope { // Clone the zero value since we will also be serializing it as part of tasks @@ -1079,9 +1119,9 @@ abstract class RDD[T: ClassTag]( /** * Aggregates the elements of this RDD in a multi-level tree pattern. + * This method is semantically identical to [[org.apache.spark.rdd.RDD#aggregate]]. * * @param depth suggested depth of the tree (default: 2) - * @see [[org.apache.spark.rdd.RDD#aggregate]] */ def treeAggregate[U: ClassTag](zeroValue: U)( seqOp: (U, T) => U, @@ -1095,7 +1135,7 @@ abstract class RDD[T: ClassTag]( val cleanCombOp = context.clean(combOp) val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp) - var partiallyAggregated = mapPartitions(it => Iterator(aggregatePartition(it))) + var partiallyAggregated: RDD[U] = mapPartitions(it => Iterator(aggregatePartition(it))) var numPartitions = partiallyAggregated.partitions.length val scale = math.max(math.ceil(math.pow(numPartitions, 1.0 / depth)).toInt, 2) // If creating an extra level doesn't help reduce @@ -1107,9 +1147,10 @@ abstract class RDD[T: ClassTag]( val curNumPartitions = numPartitions partiallyAggregated = partiallyAggregated.mapPartitionsWithIndex { (i, iter) => iter.map((i % curNumPartitions, _)) - }.reduceByKey(new HashPartitioner(curNumPartitions), cleanCombOp).values + }.foldByKey(zeroValue, new HashPartitioner(curNumPartitions))(cleanCombOp).values } - partiallyAggregated.reduce(cleanCombOp) + val copiedZeroValue = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) + partiallyAggregated.fold(copiedZeroValue)(cleanCombOp) } } @@ -1121,10 +1162,21 @@ abstract class RDD[T: ClassTag]( /** * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. + * + * The confidence is the probability that the error bounds of the result will + * contain the true value. That is, if countApprox were called repeatedly + * with confidence 0.9, we would expect 90% of the results to contain the + * true count. The confidence must be in the range [0,1] or an exception will + * be thrown. + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countApprox( timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = withScope { + require(0.0 <= confidence && confidence <= 1.0, s"confidence ($confidence) must be in [0,1]") val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) => var result = 0L while (iter.hasNext) { @@ -1140,10 +1192,15 @@ abstract class RDD[T: ClassTag]( /** * Return the count of each unique value in this RDD as a local map of (value, count) pairs. * - * Note that this method should only be used if the resulting map is expected to be small, as + * @note This method should only be used if the resulting map is expected to be small, as * the whole thing is loaded into the driver's memory. - * To handle very large results, consider using rdd.map(x => (x, 1L)).reduceByKey(_ + _), which - * returns an RDD[T, Long] instead of a map. + * To handle very large results, consider using + * + * {{{ + * rdd.map(x => (x, 1L)).reduceByKey(_ + _) + * }}} + * + * , which returns an RDD[T, Long] instead of a map. */ def countByValue()(implicit ord: Ordering[T] = null): Map[T, Long] = withScope { map(value => (value, null)).countByKey() @@ -1151,10 +1208,15 @@ abstract class RDD[T: ClassTag]( /** * Approximate version of countByValue(). + * + * @param timeout maximum time to wait for the job, in milliseconds + * @param confidence the desired statistical confidence in the result + * @return a potentially incomplete result, with error bounds */ def countByValueApprox(timeout: Long, confidence: Double = 0.95) (implicit ord: Ordering[T] = null) : PartialResult[Map[T, BoundedDouble]] = withScope { + require(0.0 <= confidence && confidence <= 1.0, s"confidence ($confidence) must be in [0,1]") if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } @@ -1176,9 +1238,9 @@ abstract class RDD[T: ClassTag]( * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available * here. * - * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero `sp > p` - * would trigger sparse representation of registers, which may reduce the memory consumption - * and increase accuracy when the cardinality is small. + * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero (`sp` is greater + * than `p`) would trigger sparse representation of registers, which may reduce the memory + * consumption and increase accuracy when the cardinality is small. * * @param p The precision value for the normal set. * `p` must be a value between 4 and `sp` if `sp` is not zero (32 max). @@ -1225,7 +1287,7 @@ abstract class RDD[T: ClassTag]( * This is similar to Scala's zipWithIndex but it uses Long instead of Int as the index type. * This method needs to trigger a spark job when this RDD contains more than one partitions. * - * Note that some RDDs, such as those returned by groupBy(), do not guarantee order of + * @note Some RDDs, such as those returned by groupBy(), do not guarantee order of * elements in a partition. The index assigned to each element is therefore not guaranteed, * and may even change if the RDD is reevaluated. If a fixed ordering is required to guarantee * the same index assignments, you should sort the RDD with sortByKey() or save it to a file. @@ -1239,7 +1301,7 @@ abstract class RDD[T: ClassTag]( * 2*n+k, ..., where n is the number of partitions. So there may exist gaps, but this method * won't trigger a spark job, which is different from [[org.apache.spark.rdd.RDD#zipWithIndex]]. * - * Note that some RDDs, such as those returned by groupBy(), do not guarantee order of + * @note Some RDDs, such as those returned by groupBy(), do not guarantee order of * elements in a partition. The unique ID assigned to each element is therefore not guaranteed, * and may even change if the RDD is reevaluated. If a fixed ordering is required to guarantee * the same index assignments, you should sort the RDD with sortByKey() or save it to a file. @@ -1247,7 +1309,7 @@ abstract class RDD[T: ClassTag]( def zipWithUniqueId(): RDD[(T, Long)] = withScope { val n = this.partitions.length.toLong this.mapPartitionsWithIndex { case (k, iter) => - iter.zipWithIndex.map { case (item, i) => + Utils.getIteratorZipWithIndex(iter, 0L).map { case (item, i) => (item, i * n + k) } } @@ -1258,10 +1320,14 @@ abstract class RDD[T: ClassTag]( * results from that partition to estimate the number of additional partitions needed to satisfy * the limit. * - * @note due to complications in the internal implementation, this method will raise + * @note This method should only be used if the resulting array is expected to be small, as + * all the data is loaded into the driver's memory. + * + * @note Due to complications in the internal implementation, this method will raise * an exception if called on an RDD of `Nothing` or `Null`. */ def take(num: Int): Array[T] = withScope { + val scaleUpFactor = Math.max(conf.getInt("spark.rdd.limit.scaleUpFactor", 4), 2) if (num == 0) { new Array[T](0) } else { @@ -1271,26 +1337,26 @@ abstract class RDD[T: ClassTag]( while (buf.size < num && partsScanned < totalParts) { // The number of partitions to try in this iteration. It is ok for this number to be // greater than totalParts because we actually cap it at totalParts in runJob. - var numPartsToTry = 1 + var numPartsToTry = 1L if (partsScanned > 0) { // If we didn't find any rows after the previous iteration, quadruple and retry. // Otherwise, interpolate the number of partitions we need to try, but overestimate // it by 50%. We also cap the estimation in the end. - if (buf.size == 0) { - numPartsToTry = partsScanned * 4 + if (buf.isEmpty) { + numPartsToTry = partsScanned * scaleUpFactor } else { // the left side of max is >=1 whenever partsScanned >= 2 numPartsToTry = Math.max((1.5 * num * partsScanned / buf.size).toInt - partsScanned, 1) - numPartsToTry = Math.min(numPartsToTry, partsScanned * 4) + numPartsToTry = Math.min(numPartsToTry, partsScanned * scaleUpFactor) } } val left = num - buf.size - val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts) + val p = partsScanned.until(math.min(partsScanned + numPartsToTry, totalParts).toInt) val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, p) res.foreach(buf ++= _.take(num - buf.size)) - partsScanned += numPartsToTry + partsScanned += p.size } buf.toArray @@ -1309,7 +1375,8 @@ abstract class RDD[T: ClassTag]( /** * Returns the top k (largest) elements from this RDD as defined by the specified - * implicit Ordering[T]. This does the opposite of [[takeOrdered]]. For example: + * implicit Ordering[T] and maintains the ordering. This does the opposite of + * [[takeOrdered]]. For example: * {{{ * sc.parallelize(Seq(10, 4, 2, 12, 3)).top(1) * // returns Array(12) @@ -1318,6 +1385,9 @@ abstract class RDD[T: ClassTag]( * // returns Array(6, 5) * }}} * + * @note This method should only be used if the resulting array is expected to be small, as + * all the data is loaded into the driver's memory. + * * @param num k, the number of top elements to return * @param ord the implicit ordering for T * @return an array of top elements @@ -1338,6 +1408,9 @@ abstract class RDD[T: ClassTag]( * // returns Array(2, 3) * }}} * + * @note This method should only be used if the resulting array is expected to be small, as + * all the data is loaded into the driver's memory. + * * @param num k, the number of elements to return * @param ord the implicit ordering for T * @return an array of top elements @@ -1349,7 +1422,7 @@ abstract class RDD[T: ClassTag]( val mapRDDs = mapPartitions { items => // Priority keeps the largest elements, so let's reverse the ordering. val queue = new BoundedPriorityQueue[T](num)(ord.reverse) - queue ++= util.collection.Utils.takeOrdered(items, num)(ord) + queue ++= collectionUtils.takeOrdered(items, num)(ord) Iterator.single(queue) } if (mapRDDs.partitions.length == 0) { @@ -1380,7 +1453,7 @@ abstract class RDD[T: ClassTag]( } /** - * @note due to complications in the internal implementation, this method will raise an + * @note Due to complications in the internal implementation, this method will raise an * exception if called on an RDD of `Nothing` or `Null`. This may be come up in practice * because, for example, the type of `parallelize(Seq())` is `RDD[Nothing]`. * (`parallelize(Seq())` should be avoided anyway in favor of `parallelize(Seq[T]())`.) @@ -1540,14 +1613,15 @@ abstract class RDD[T: ClassTag]( /** * Return whether this RDD is checkpointed and materialized, either reliably or locally. */ - def isCheckpointed: Boolean = checkpointData.exists(_.isCheckpointed) + def isCheckpointed: Boolean = isCheckpointedAndMaterialized /** * Return whether this RDD is checkpointed and materialized, either reliably or locally. * This is introduced as an alias for `isCheckpointed` to clarify the semantics of the * return value. Exposed for testing. */ - private[spark] def isCheckpointedAndMaterialized: Boolean = isCheckpointed + private[spark] def isCheckpointedAndMaterialized: Boolean = + checkpointData.exists(_.isCheckpointed) /** * Return whether this RDD is marked for local checkpointing. @@ -1597,6 +1671,15 @@ abstract class RDD[T: ClassTag]( private[spark] var checkpointData: Option[RDDCheckpointData[T]] = None + // Whether to checkpoint all ancestor RDDs that are marked for checkpointing. By default, + // we stop as soon as we find the first such RDD, an optimization that allows us to write + // less data but is not safe for all workloads. E.g. in streaming we may checkpoint both + // an RDD and its parent in every batch, in which case the parent may never be checkpointed + // and its lineage never truncated, leading to OOMs in the long run (SPARK-6847). + private val checkpointAllMarkedAncestors = + Option(sc.getLocalProperty(RDD.CHECKPOINT_ALL_MARKED_ANCESTORS)) + .map(_.toBoolean).getOrElse(false) + /** Returns the first parent RDD */ protected[spark] def firstParent[U: ClassTag]: RDD[U] = { dependencies.head.rdd.asInstanceOf[RDD[U]] @@ -1640,6 +1723,13 @@ abstract class RDD[T: ClassTag]( if (!doCheckpointCalled) { doCheckpointCalled = true if (checkpointData.isDefined) { + if (checkpointAllMarkedAncestors) { + // TODO We can collect all the RDDs that needs to be checkpointed, and then checkpoint + // them in parallel. + // Checkpoint parents first because our lineage will be truncated after we + // checkpoint ourselves + dependencies.foreach(_.rdd.doCheckpoint()) + } checkpointData.get.checkpoint() } else { dependencies.foreach(_.rdd.doCheckpoint()) @@ -1660,7 +1750,7 @@ abstract class RDD[T: ClassTag]( /** * Clears the dependencies of this RDD. This method must ensure that all references - * to the original parent RDDs is removed to enable the parent RDDs to be garbage + * to the original parent RDDs are removed to enable the parent RDDs to be garbage * collected. Subclasses of RDD may override this method for implementing their own cleaning * logic. See [[org.apache.spark.rdd.UnionRDD]] for an example. */ @@ -1755,10 +1845,13 @@ abstract class RDD[T: ClassTag]( * Defines implicit functions that provide extra functionalities on RDDs of specific types. * * For example, [[RDD.rddToPairRDDFunctions]] converts an RDD into a [[PairRDDFunctions]] for - * key-value-pair RDDs, and enabling extra functionalities such as [[PairRDDFunctions.reduceByKey]]. + * key-value-pair RDDs, and enabling extra functionalities such as `PairRDDFunctions.reduceByKey`. */ object RDD { + private[spark] val CHECKPOINT_ALL_MARKED_ANCESTORS = + "spark.checkpoint.checkpointAllMarkedAncestors" + // The following implicit functions were in SparkContext before 1.3 and users had to // `import SparkContext._` to enable them. Now we move them here to make the compiler find // them automatically. However, we still keep the old functions in SparkContext for backward diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index 429514b4f6be..6c552d4d1251 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -23,7 +23,8 @@ import org.apache.spark.Partition /** * Enumeration to manage state transitions of an RDD through checkpointing - * [ Initialized --> checkpointing in progress --> checkpointed ]. + * + * [ Initialized --{@literal >} checkpointing in progress --{@literal >} checkpointed ] */ private[spark] object CheckpointState extends Enumeration { type CheckpointState = Value @@ -32,7 +33,7 @@ private[spark] object CheckpointState extends Enumeration { /** * This class contains all the information related to RDD checkpointing. Each instance of this - * class is associated with a RDD. It manages process of checkpointing of the associated RDD, + * class is associated with an RDD. It manages process of checkpointing of the associated RDD, * as well as, manages the post-checkpoint state by providing the updated partitions, * iterator and preferred locations of the checkpointed RDD. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala b/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala index 540cbd688b63..53d69ba26811 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala @@ -25,7 +25,8 @@ import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import com.google.common.base.Objects -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging /** * A general, named code block representing an operation that instantiates RDDs. diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala index a69be6a068bb..979152b55f95 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala @@ -17,15 +17,19 @@ package org.apache.spark.rdd -import java.io.IOException +import java.io.{FileNotFoundException, IOException} +import java.util.concurrent.TimeUnit import scala.reflect.ClassTag +import scala.util.control.NonFatal import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.broadcast.Broadcast -import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.CHECKPOINT_COMPRESS +import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{SerializableConfiguration, Utils} /** @@ -33,8 +37,9 @@ import org.apache.spark.util.{SerializableConfiguration, Utils} */ private[spark] class ReliableCheckpointRDD[T: ClassTag]( sc: SparkContext, - val checkpointPath: String) - extends CheckpointRDD[T](sc) { + val checkpointPath: String, + _partitioner: Option[Partitioner] = None + ) extends CheckpointRDD[T](sc) { @transient private val hadoopConf = sc.hadoopConfiguration @transient private val cpath = new Path(checkpointPath) @@ -47,7 +52,13 @@ private[spark] class ReliableCheckpointRDD[T: ClassTag]( /** * Return the path of the checkpoint directory this RDD reads data from. */ - override def getCheckpointFile: Option[String] = Some(checkpointPath) + override val getCheckpointFile: Option[String] = Some(checkpointPath) + + override val partitioner: Option[Partitioner] = { + _partitioner.orElse { + ReliableCheckpointRDD.readCheckpointedPartitionerFile(context, checkpointPath) + } + } /** * Return partitions described by the files in the checkpoint directory. @@ -61,10 +72,10 @@ private[spark] class ReliableCheckpointRDD[T: ClassTag]( val inputFiles = fs.listStatus(cpath) .map(_.getPath) .filter(_.getName.startsWith("part-")) - .sortBy(_.toString) + .sortBy(_.getName.stripPrefix("part-").toInt) // Fail fast if input files are invalid inputFiles.zipWithIndex.foreach { case (path, i) => - if (!path.toString.endsWith(ReliableCheckpointRDD.checkpointFileName(i))) { + if (path.getName != ReliableCheckpointRDD.checkpointFileName(i)) { throw new SparkException(s"Invalid checkpoint file: $path") } } @@ -100,10 +111,59 @@ private[spark] object ReliableCheckpointRDD extends Logging { "part-%05d".format(partitionIndex) } + private def checkpointPartitionerFileName(): String = { + "_partitioner" + } + + /** + * Write RDD to checkpoint files and return a ReliableCheckpointRDD representing the RDD. + */ + def writeRDDToCheckpointDirectory[T: ClassTag]( + originalRDD: RDD[T], + checkpointDir: String, + blockSize: Int = -1): ReliableCheckpointRDD[T] = { + val checkpointStartTimeNs = System.nanoTime() + + val sc = originalRDD.sparkContext + + // Create the output path for the checkpoint + val checkpointDirPath = new Path(checkpointDir) + val fs = checkpointDirPath.getFileSystem(sc.hadoopConfiguration) + if (!fs.mkdirs(checkpointDirPath)) { + throw new SparkException(s"Failed to create checkpoint path $checkpointDirPath") + } + + // Save to file, and reload it as an RDD + val broadcastedConf = sc.broadcast( + new SerializableConfiguration(sc.hadoopConfiguration)) + // TODO: This is expensive because it computes the RDD again unnecessarily (SPARK-8582) + sc.runJob(originalRDD, + writePartitionToCheckpointFile[T](checkpointDirPath.toString, broadcastedConf) _) + + if (originalRDD.partitioner.nonEmpty) { + writePartitionerToCheckpointDir(sc, originalRDD.partitioner.get, checkpointDirPath) + } + + val checkpointDurationMs = + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - checkpointStartTimeNs) + logInfo(s"Checkpointing took $checkpointDurationMs ms.") + + val newRDD = new ReliableCheckpointRDD[T]( + sc, checkpointDirPath.toString, originalRDD.partitioner) + if (newRDD.partitions.length != originalRDD.partitions.length) { + throw new SparkException( + "Checkpoint RDD has a different number of partitions from original RDD. Original " + + s"RDD [ID: ${originalRDD.id}, num of partitions: ${originalRDD.partitions.length}]; " + + s"Checkpoint RDD [ID: ${newRDD.id}, num of partitions: " + + s"${newRDD.partitions.length}].") + } + newRDD + } + /** - * Write this partition's values to a checkpoint file. + * Write an RDD partition's data to a checkpoint file. */ - def writeCheckpointFile[T: ClassTag]( + def writePartitionToCheckpointFile[T: ClassTag]( path: String, broadcastedConf: Broadcast[SerializableConfiguration], blockSize: Int = -1)(ctx: TaskContext, iterator: Iterator[T]) { @@ -116,16 +176,19 @@ private[spark] object ReliableCheckpointRDD extends Logging { val tempOutputPath = new Path(outputDir, s".$finalOutputName-attempt-${ctx.attemptNumber()}") - if (fs.exists(tempOutputPath)) { - throw new IOException(s"Checkpoint failed: temporary path $tempOutputPath already exists") - } val bufferSize = env.conf.getInt("spark.buffer.size", 65536) val fileOutputStream = if (blockSize < 0) { - fs.create(tempOutputPath, false, bufferSize) + val fileStream = fs.create(tempOutputPath, false, bufferSize) + if (env.conf.get(CHECKPOINT_COMPRESS)) { + CompressionCodec.createCodec(env.conf).compressedOutputStream(fileStream) + } else { + fileStream + } } else { // This is mainly for testing purpose - fs.create(tempOutputPath, false, bufferSize, fs.getDefaultReplication, blockSize) + fs.create(tempOutputPath, false, bufferSize, + fs.getDefaultReplication(fs.getWorkingDirectory), blockSize) } val serializer = env.serializer.newInstance() val serializeStream = serializer.serializeStream(fileOutputStream) @@ -151,6 +214,70 @@ private[spark] object ReliableCheckpointRDD extends Logging { } } + /** + * Write a partitioner to the given RDD checkpoint directory. This is done on a best-effort + * basis; any exception while writing the partitioner is caught, logged and ignored. + */ + private def writePartitionerToCheckpointDir( + sc: SparkContext, partitioner: Partitioner, checkpointDirPath: Path): Unit = { + try { + val partitionerFilePath = new Path(checkpointDirPath, checkpointPartitionerFileName) + val bufferSize = sc.conf.getInt("spark.buffer.size", 65536) + val fs = partitionerFilePath.getFileSystem(sc.hadoopConfiguration) + val fileOutputStream = fs.create(partitionerFilePath, false, bufferSize) + val serializer = SparkEnv.get.serializer.newInstance() + val serializeStream = serializer.serializeStream(fileOutputStream) + Utils.tryWithSafeFinally { + serializeStream.writeObject(partitioner) + } { + serializeStream.close() + } + logDebug(s"Written partitioner to $partitionerFilePath") + } catch { + case NonFatal(e) => + logWarning(s"Error writing partitioner $partitioner to $checkpointDirPath") + } + } + + + /** + * Read a partitioner from the given RDD checkpoint directory, if it exists. + * This is done on a best-effort basis; any exception while reading the partitioner is + * caught, logged and ignored. + */ + private def readCheckpointedPartitionerFile( + sc: SparkContext, + checkpointDirPath: String): Option[Partitioner] = { + try { + val bufferSize = sc.conf.getInt("spark.buffer.size", 65536) + val partitionerFilePath = new Path(checkpointDirPath, checkpointPartitionerFileName) + val fs = partitionerFilePath.getFileSystem(sc.hadoopConfiguration) + val fileInputStream = fs.open(partitionerFilePath, bufferSize) + val serializer = SparkEnv.get.serializer.newInstance() + val partitioner = Utils.tryWithSafeFinally { + val deserializeStream = serializer.deserializeStream(fileInputStream) + Utils.tryWithSafeFinally { + deserializeStream.readObject[Partitioner] + } { + deserializeStream.close() + } + } { + fileInputStream.close() + } + + logDebug(s"Read partitioner from $partitionerFilePath") + Some(partitioner) + } catch { + case e: FileNotFoundException => + logDebug("No partitioner file", e) + None + case NonFatal(e) => + logWarning(s"Error reading partitioner from $checkpointDirPath, " + + s"partitioner will not be recovered which may lead to performance loss", e) + None + } + } + /** * Read the content of the specified checkpoint file. */ @@ -161,7 +288,14 @@ private[spark] object ReliableCheckpointRDD extends Logging { val env = SparkEnv.get val fs = path.getFileSystem(broadcastedConf.value.value) val bufferSize = env.conf.getInt("spark.buffer.size", 65536) - val fileInputStream = fs.open(path, bufferSize) + val fileInputStream = { + val fileStream = fs.open(path, bufferSize) + if (env.conf.get(CHECKPOINT_COMPRESS)) { + CompressionCodec.createCodec(env.conf).compressedInputStream(fileStream) + } else { + fileStream + } + } val serializer = env.serializer.newInstance() val deserializeStream = serializer.deserializeStream(fileInputStream) diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala index 91cad6662e4d..b6d723c68279 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.hadoop.fs.Path import org.apache.spark._ -import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.internal.Logging /** * An implementation of checkpointing that writes the RDD data to reliable storage. @@ -55,25 +55,7 @@ private[spark] class ReliableRDDCheckpointData[T: ClassTag](@transient private v * This is called immediately after the first action invoked on this RDD has completed. */ protected override def doCheckpoint(): CheckpointRDD[T] = { - - // Create the output path for the checkpoint - val path = new Path(cpDir) - val fs = path.getFileSystem(rdd.context.hadoopConfiguration) - if (!fs.mkdirs(path)) { - throw new SparkException(s"Failed to create checkpoint path $cpDir") - } - - // Save to file, and reload it as an RDD - val broadcastedConf = rdd.context.broadcast( - new SerializableConfiguration(rdd.context.hadoopConfiguration)) - // TODO: This is expensive because it computes the RDD again unnecessarily (SPARK-8582) - rdd.context.runJob(rdd, ReliableCheckpointRDD.writeCheckpointFile[T](cpDir, broadcastedConf) _) - val newRDD = new ReliableCheckpointRDD[T](rdd.context, cpDir) - if (newRDD.partitions.length != rdd.partitions.length) { - throw new SparkException( - s"Checkpoint RDD $newRDD(${newRDD.partitions.length}) has different " + - s"number of partitions from original RDD $rdd(${rdd.partitions.length})") - } + val newRDD = ReliableCheckpointRDD.writeRDDToCheckpointDirectory(rdd, cpDir) // Optionally clean our checkpoint files if the reference is out of scope if (rdd.conf.getBoolean("spark.cleaner.referenceTracking.cleanCheckpoints", false)) { @@ -83,7 +65,6 @@ private[spark] class ReliableRDDCheckpointData[T: ClassTag](@transient private v } logInfo(s"Done checkpointing RDD ${rdd.id} to $cpDir, new parent is RDD ${newRDD.id}") - newRDD } @@ -99,12 +80,7 @@ private[spark] object ReliableRDDCheckpointData extends Logging { /** Clean up the files associated with the checkpoint data for this RDD. */ def cleanCheckpoint(sc: SparkContext, rddId: Int): Unit = { checkpointPath(sc, rddId).foreach { path => - val fs = path.getFileSystem(sc.hadoopConfiguration) - if (fs.exists(path)) { - if (!fs.delete(path, true)) { - logWarning(s"Error deleting ${path.toString()}") - } - } + path.getFileSystem(sc.hadoopConfiguration).delete(path, true) } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala deleted file mode 100644 index 9e8cee5331cf..000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.rdd - -import java.util.Random - -import scala.reflect.ClassTag - -import org.apache.commons.math3.distribution.PoissonDistribution - -import org.apache.spark.{Partition, TaskContext} - -@deprecated("Replaced by PartitionwiseSampledRDDPartition", "1.0.0") -private[spark] -class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition with Serializable { - override val index: Int = prev.index -} - -@deprecated("Replaced by PartitionwiseSampledRDD", "1.0.0") -private[spark] class SampledRDD[T: ClassTag]( - prev: RDD[T], - withReplacement: Boolean, - frac: Double, - seed: Int) - extends RDD[T](prev) { - - override def getPartitions: Array[Partition] = { - val rg = new Random(seed) - firstParent[T].partitions.map(x => new SampledRDDPartition(x, rg.nextInt)) - } - - override def getPreferredLocations(split: Partition): Seq[String] = - firstParent[T].preferredLocations(split.asInstanceOf[SampledRDDPartition].prev) - - override def compute(splitIn: Partition, context: TaskContext): Iterator[T] = { - val split = splitIn.asInstanceOf[SampledRDDPartition] - if (withReplacement) { - // For large datasets, the expected number of occurrences of each element in a sample with - // replacement is Poisson(frac). We use that to get a count for each element. - val poisson = new PoissonDistribution(frac) - poisson.reseedRandomGenerator(split.seed) - - firstParent[T].iterator(split.prev, context).flatMap { element => - val count = poisson.sample() - if (count == 0) { - Iterator.empty // Avoid object allocation when we return 0 items, which is quite often - } else { - Iterator.fill(count)(element) - } - } - } else { // Sampling without replacement - val rand = new Random(split.seed) - firstParent[T].iterator(split.prev, context).filter(x => (rand.nextDouble <= frac)) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index 4b5f15dd06b8..02def89dd8c2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -16,20 +16,21 @@ */ package org.apache.spark.rdd -import scala.reflect.{ClassTag, classTag} +import scala.reflect.ClassTag import org.apache.hadoop.io.Writable import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.SequenceFileOutputFormat -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile, - * through an implicit conversion. Note that this can't be part of PairRDDFunctions because - * we need more implicit parameters to convert our keys and values to Writable. + * through an implicit conversion. * + * @note This can't be part of PairRDDFunctions because we need more implicit parameters to + * convert our keys and values to Writable. */ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag]( self: RDD[(K, V)], @@ -38,45 +39,8 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag extends Logging with Serializable { - @deprecated("It's used to provide backward compatibility for pre 1.3.0.", "1.3.0") - def this(self: RDD[(K, V)]) { - this(self, null, null) - } - - private val keyWritableClass = - if (_keyWritableClass == null) { - // pre 1.3.0, we need to use Reflection to get the Writable class - getWritableClass[K]() - } else { - _keyWritableClass - } - - private val valueWritableClass = - if (_valueWritableClass == null) { - // pre 1.3.0, we need to use Reflection to get the Writable class - getWritableClass[V]() - } else { - _valueWritableClass - } - - private def getWritableClass[T <% Writable: ClassTag](): Class[_ <: Writable] = { - val c = { - if (classOf[Writable].isAssignableFrom(classTag[T].runtimeClass)) { - classTag[T].runtimeClass - } else { - // We get the type of the Writable class by looking at the apply method which converts - // from T to Writable. Since we have two apply methods we filter out the one which - // is not of the form "java.lang.Object apply(java.lang.Object)" - implicitly[T => Writable].getClass.getDeclaredMethods().filter( - m => m.getReturnType().toString != "class java.lang.Object" && - m.getName() == "apply")(0).getReturnType - - } - // TODO: use something like WritableConverter to avoid reflection - } - c.asInstanceOf[Class[_ <: Writable]] - } - + // TODO the context bound (<%) above should be replaced with simple type bound and implicit + // conversion but is a breaking change. This should be fixed in Spark 3.x. /** * Output the RDD as a Hadoop SequenceFile using the Writable types we infer from the RDD's key @@ -94,24 +58,24 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag // valueWritableClass at the compile time. To implement that, we need to add type parameters to // SequenceFileRDDFunctions. however, SequenceFileRDDFunctions is a public class so it will be a // breaking change. - val convertKey = self.keyClass != keyWritableClass - val convertValue = self.valueClass != valueWritableClass + val convertKey = self.keyClass != _keyWritableClass + val convertValue = self.valueClass != _valueWritableClass - logInfo("Saving as sequence file of type (" + keyWritableClass.getSimpleName + "," + - valueWritableClass.getSimpleName + ")" ) + logInfo("Saving as sequence file of type " + + s"(${_keyWritableClass.getSimpleName},${_valueWritableClass.getSimpleName})" ) val format = classOf[SequenceFileOutputFormat[Writable, Writable]] val jobConf = new JobConf(self.context.hadoopConfiguration) if (!convertKey && !convertValue) { - self.saveAsHadoopFile(path, keyWritableClass, valueWritableClass, format, jobConf, codec) + self.saveAsHadoopFile(path, _keyWritableClass, _valueWritableClass, format, jobConf, codec) } else if (!convertKey && convertValue) { self.map(x => (x._1, anyToWritable(x._2))).saveAsHadoopFile( - path, keyWritableClass, valueWritableClass, format, jobConf, codec) + path, _keyWritableClass, _valueWritableClass, format, jobConf, codec) } else if (convertKey && !convertValue) { self.map(x => (anyToWritable(x._1), x._2)).saveAsHadoopFile( - path, keyWritableClass, valueWritableClass, format, jobConf, codec) + path, _keyWritableClass, _valueWritableClass, format, jobConf, codec) } else if (convertKey && convertValue) { self.map(x => (anyToWritable(x._1), anyToWritable(x._2))).saveAsHadoopFile( - path, keyWritableClass, valueWritableClass, format, jobConf, codec) + path, _keyWritableClass, _valueWritableClass, format, jobConf, codec) } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index a013c3f66a3a..26eaa9aa3d03 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -25,7 +25,6 @@ import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { override val index: Int = idx - override def hashCode(): Int = idx } /** @@ -44,7 +43,7 @@ class ShuffledRDD[K: ClassTag, V: ClassTag, C: ClassTag]( part: Partitioner) extends RDD[(K, C)](prev.context, Nil) { - private var serializer: Option[Serializer] = None + private var userSpecifiedSerializer: Option[Serializer] = None private var keyOrdering: Option[Ordering[K]] = None @@ -54,7 +53,7 @@ class ShuffledRDD[K: ClassTag, V: ClassTag, C: ClassTag]( /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ def setSerializer(serializer: Serializer): ShuffledRDD[K, V, C] = { - this.serializer = Option(serializer) + this.userSpecifiedSerializer = Option(serializer) this } @@ -77,6 +76,14 @@ class ShuffledRDD[K: ClassTag, V: ClassTag, C: ClassTag]( } override def getDependencies: Seq[Dependency[_]] = { + val serializer = userSpecifiedSerializer.getOrElse { + val serializerManager = SparkEnv.get.serializerManager + if (mapSideCombine) { + serializerManager.getSerializer(implicitly[ClassTag[K]], implicitly[ClassTag[C]]) + } else { + serializerManager.getSerializer(implicitly[ClassTag[K]], implicitly[ClassTag[V]]) + } + } List(new ShuffleDependency(prev, part, serializer, keyOrdering, aggregator, mapSideCombine)) } @@ -86,7 +93,7 @@ class ShuffledRDD[K: ClassTag, V: ClassTag, C: ClassTag]( Array.tabulate[Partition](part.numPartitions)(i => new ShuffledRDDPartition(i)) } - override def getPreferredLocations(partition: Partition): Seq[String] = { + override protected def getPreferredLocations(partition: Partition): Seq[String] = { val tracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] val dep = dependencies.head.asInstanceOf[ShuffleDependency[K, V, C]] tracker.getPreferredLocationsForShuffle(dep, partition.index) diff --git a/core/src/main/scala/org/apache/spark/rdd/SqlNewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SqlNewHadoopRDD.scala deleted file mode 100644 index 264dae7f3908..000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/SqlNewHadoopRDD.scala +++ /dev/null @@ -1,290 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.rdd - -import java.text.SimpleDateFormat -import java.util.Date - -import scala.reflect.ClassTag - -import org.apache.hadoop.conf.{Configurable, Configuration} -import org.apache.hadoop.io.Writable -import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, FileSplit} -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.DataReadMethod -import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil -import org.apache.spark.unsafe.types.UTF8String -import org.apache.spark.{Partition => SparkPartition, _} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{SerializableConfiguration, ShutdownHookManager, Utils} - - -private[spark] class SqlNewHadoopPartition( - rddId: Int, - val index: Int, - rawSplit: InputSplit with Writable) - extends SparkPartition { - - val serializableHadoopSplit = new SerializableWritable(rawSplit) - - override def hashCode(): Int = 41 * (41 + rddId) + index -} - -/** - * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, - * sources in HBase, or S3), using the new MapReduce API (`org.apache.hadoop.mapreduce`). - * It is based on [[org.apache.spark.rdd.NewHadoopRDD]]. It has three additions. - * 1. A shared broadcast Hadoop Configuration. - * 2. An optional closure `initDriverSideJobFuncOpt` that set configurations at the driver side - * to the shared Hadoop Configuration. - * 3. An optional closure `initLocalJobFuncOpt` that set configurations at both the driver side - * and the executor side to the shared Hadoop Configuration. - * - * Note: This is RDD is basically a cloned version of [[org.apache.spark.rdd.NewHadoopRDD]] with - * changes based on [[org.apache.spark.rdd.HadoopRDD]]. - */ -private[spark] class SqlNewHadoopRDD[V: ClassTag]( - sc : SparkContext, - broadcastedConf: Broadcast[SerializableConfiguration], - @transient private val initDriverSideJobFuncOpt: Option[Job => Unit], - initLocalJobFuncOpt: Option[Job => Unit], - inputFormatClass: Class[_ <: InputFormat[Void, V]], - valueClass: Class[V]) - extends RDD[V](sc, Nil) - with SparkHadoopMapReduceUtil - with Logging { - - protected def getJob(): Job = { - val conf: Configuration = broadcastedConf.value.value - // "new Job" will make a copy of the conf. Then, it is - // safe to mutate conf properties with initLocalJobFuncOpt - // and initDriverSideJobFuncOpt. - val newJob = new Job(conf) - initLocalJobFuncOpt.map(f => f(newJob)) - newJob - } - - def getConf(isDriverSide: Boolean): Configuration = { - val job = getJob() - if (isDriverSide) { - initDriverSideJobFuncOpt.map(f => f(job)) - } - SparkHadoopUtil.get.getConfigurationFromJobContext(job) - } - - private val jobTrackerId: String = { - val formatter = new SimpleDateFormat("yyyyMMddHHmm") - formatter.format(new Date()) - } - - @transient protected val jobId = new JobID(jobTrackerId, id) - - override def getPartitions: Array[SparkPartition] = { - val conf = getConf(isDriverSide = true) - val inputFormat = inputFormatClass.newInstance - inputFormat match { - case configurable: Configurable => - configurable.setConf(conf) - case _ => - } - val jobContext = newJobContext(conf, jobId) - val rawSplits = inputFormat.getSplits(jobContext).toArray - val result = new Array[SparkPartition](rawSplits.size) - for (i <- 0 until rawSplits.size) { - result(i) = - new SqlNewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) - } - result - } - - override def compute( - theSplit: SparkPartition, - context: TaskContext): Iterator[V] = { - val iter = new Iterator[V] { - val split = theSplit.asInstanceOf[SqlNewHadoopPartition] - logInfo("Input split: " + split.serializableHadoopSplit) - val conf = getConf(isDriverSide = false) - - val inputMetrics = context.taskMetrics - .getInputMetricsForReadMethod(DataReadMethod.Hadoop) - - // Sets the thread local variable for the file's name - split.serializableHadoopSplit.value match { - case fs: FileSplit => SqlNewHadoopRDD.setInputFileName(fs.getPath.toString) - case _ => SqlNewHadoopRDD.unsetInputFileName() - } - - // Find a function that will return the FileSystem bytes read by this thread. Do this before - // creating RecordReader, because RecordReader's constructor might read some bytes - val bytesReadCallback = inputMetrics.bytesReadCallback.orElse { - split.serializableHadoopSplit.value match { - case _: FileSplit | _: CombineFileSplit => - SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() - case _ => None - } - } - inputMetrics.setBytesReadCallback(bytesReadCallback) - - val attemptId = newTaskAttemptID(jobTrackerId, id, isMap = true, split.index, 0) - val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) - val format = inputFormatClass.newInstance - format match { - case configurable: Configurable => - configurable.setConf(conf) - case _ => - } - private[this] var reader = format.createRecordReader( - split.serializableHadoopSplit.value, hadoopAttemptContext) - reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) - - // Register an on-task-completion callback to close the input stream. - context.addTaskCompletionListener(context => close()) - - private[this] var havePair = false - private[this] var finished = false - - override def hasNext: Boolean = { - if (context.isInterrupted) { - throw new TaskKilledException - } - if (!finished && !havePair) { - finished = !reader.nextKeyValue - if (finished) { - // Close and release the reader here; close() will also be called when the task - // completes, but for tasks that read from many files, it helps to release the - // resources early. - close() - } - havePair = !finished - } - !finished - } - - override def next(): V = { - if (!hasNext) { - throw new java.util.NoSuchElementException("End of stream") - } - havePair = false - if (!finished) { - inputMetrics.incRecordsRead(1) - } - reader.getCurrentValue - } - - private def close() { - if (reader != null) { - SqlNewHadoopRDD.unsetInputFileName() - // Close the reader and release it. Note: it's very important that we don't close the - // reader more than once, since that exposes us to MAPREDUCE-5918 when running against - // Hadoop 1.x and older Hadoop 2.x releases. That bug can lead to non-deterministic - // corruption issues when reading compressed input. - try { - reader.close() - } catch { - case e: Exception => - if (!ShutdownHookManager.inShutdown()) { - logWarning("Exception in RecordReader.close()", e) - } - } finally { - reader = null - } - if (bytesReadCallback.isDefined) { - inputMetrics.updateBytesRead() - } else if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit] || - split.serializableHadoopSplit.value.isInstanceOf[CombineFileSplit]) { - // If we can't get the bytes read from the FS stats, fall back to the split size, - // which may be inaccurate. - try { - inputMetrics.incBytesRead(split.serializableHadoopSplit.value.getLength) - } catch { - case e: java.io.IOException => - logWarning("Unable to get input size to set InputMetrics for task", e) - } - } - } - } - } - iter - } - - override def getPreferredLocations(hsplit: SparkPartition): Seq[String] = { - val split = hsplit.asInstanceOf[SqlNewHadoopPartition].serializableHadoopSplit.value - val locs = HadoopRDD.SPLIT_INFO_REFLECTIONS match { - case Some(c) => - try { - val infos = c.newGetLocationInfo.invoke(split).asInstanceOf[Array[AnyRef]] - Some(HadoopRDD.convertSplitLocationInfo(infos)) - } catch { - case e : Exception => - logDebug("Failed to use InputSplit#getLocationInfo.", e) - None - } - case None => None - } - locs.getOrElse(split.getLocations.filter(_ != "localhost")) - } - - override def persist(storageLevel: StorageLevel): this.type = { - if (storageLevel.deserialized) { - logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + - " behavior because Hadoop's RecordReader reuses the same Writable object for all records." + - " Use a map transformation to make copies of the records.") - } - super.persist(storageLevel) - } -} - -private[spark] object SqlNewHadoopRDD { - - /** - * The thread variable for the name of the current file being read. This is used by - * the InputFileName function in Spark SQL. - */ - private[this] val inputFileName: ThreadLocal[UTF8String] = new ThreadLocal[UTF8String] { - override protected def initialValue(): UTF8String = UTF8String.fromString("") - } - - def getInputFileName(): UTF8String = inputFileName.get() - - private[spark] def setInputFileName(file: String) = inputFileName.set(UTF8String.fromString(file)) - - private[spark] def unsetInputFileName(): Unit = inputFileName.remove() - - /** - * Analogous to [[org.apache.spark.rdd.MapPartitionsRDD]], but passes in an InputSplit to - * the given function rather than the index of the partition. - */ - private[spark] class NewHadoopMapPartitionsWithSplitRDD[U: ClassTag, T: ClassTag]( - prev: RDD[T], - f: (InputSplit, Iterator[T]) => Iterator[U], - preservesPartitioning: Boolean = false) - extends RDD[U](prev) { - - override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None - - override def getPartitions: Array[SparkPartition] = firstParent[T].partitions - - override def compute(split: SparkPartition, context: TaskContext): Iterator[U] = { - val partition = split.asInstanceOf[SqlNewHadoopPartition] - val inputSplit = partition.serializableHadoopSplit.value - f(inputSplit, firstParent[T].iterator(split, context)) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index 25ec685eff5a..a733eaa5d7e5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -30,7 +30,6 @@ import org.apache.spark.Partitioner import org.apache.spark.ShuffleDependency import org.apache.spark.SparkEnv import org.apache.spark.TaskContext -import org.apache.spark.serializer.Serializer /** * An optimized version of cogroup for set difference/subtraction. @@ -54,13 +53,6 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( part: Partitioner) extends RDD[(K, V)](rdd1.context, Nil) { - private var serializer: Option[Serializer] = None - - /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ - def setSerializer(serializer: Serializer): SubtractedRDD[K, V, W] = { - this.serializer = Option(serializer) - this - } override def getDependencies: Seq[Dependency[_]] = { def rddDependency[T1: ClassTag, T2: ClassTag](rdd: RDD[_ <: Product2[T1, T2]]) @@ -70,7 +62,7 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( new OneToOneDependency(rdd) } else { logDebug("Adding shuffle dependency with " + rdd) - new ShuffleDependency[T1, T2, Any](rdd, part, serializer) + new ShuffleDependency[T1, T2, Any](rdd, part) } } Seq(rddDependency[K, V](rdd1), rddDependency[K, W](rdd2)) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 66cf4369da2e..60e383afadf1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -20,6 +20,8 @@ package org.apache.spark.rdd import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer +import scala.collection.parallel.ForkJoinTaskSupport +import scala.concurrent.forkjoin.ForkJoinPool import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} @@ -56,14 +58,30 @@ private[spark] class UnionPartition[T: ClassTag]( } } +object UnionRDD { + private[spark] lazy val partitionEvalTaskSupport = + new ForkJoinTaskSupport(new ForkJoinPool(8)) +} + @DeveloperApi class UnionRDD[T: ClassTag]( sc: SparkContext, var rdds: Seq[RDD[T]]) extends RDD[T](sc, Nil) { // Nil since we implement getDependencies + // visible for testing + private[spark] val isPartitionListingParallel: Boolean = + rdds.length > conf.getInt("spark.rdd.parallelListingThreshold", 10) + override def getPartitions: Array[Partition] = { - val array = new Array[Partition](rdds.map(_.partitions.length).sum) + val parRDDs = if (isPartitionListingParallel) { + val parArray = rdds.par + parArray.tasksupport = UnionRDD.partitionEvalTaskSupport + parArray + } else { + rdds + } + val array = new Array[Partition](parRDDs.map(_.partitions.length).seq.sum) var pos = 0 for ((rdd, rddIndex) <- rdds.zipWithIndex; split <- rdd.partitions) { array(pos) = new UnionPartition(pos, rdd, rddIndex, split.index) diff --git a/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala new file mode 100644 index 000000000000..9f3d0745c33c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.rdd + +import org.apache.hadoop.conf.{Configurable, Configuration} +import org.apache.hadoop.io.{Text, Writable} +import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat +import org.apache.hadoop.mapreduce.task.JobContextImpl + +import org.apache.spark.{Partition, SparkContext} +import org.apache.spark.input.WholeTextFileInputFormat + +/** + * An RDD that reads a bunch of text files in, and each text file becomes one record. + */ +private[spark] class WholeTextFileRDD( + sc : SparkContext, + inputFormatClass: Class[_ <: WholeTextFileInputFormat], + keyClass: Class[Text], + valueClass: Class[Text], + conf: Configuration, + minPartitions: Int) + extends NewHadoopRDD[Text, Text](sc, inputFormatClass, keyClass, valueClass, conf) { + + override def getPartitions: Array[Partition] = { + val conf = getConf + // setMinPartitions below will call FileInputFormat.listStatus(), which can be quite slow when + // traversing a large number of directories and files. Parallelize it. + conf.setIfUnset(FileInputFormat.LIST_STATUS_NUM_THREADS, + Runtime.getRuntime.availableProcessors().toString) + val inputFormat = inputFormatClass.newInstance + inputFormat match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => + } + val jobContext = new JobContextImpl(conf, jobId) + inputFormat.setMinPartitions(jobContext, minPartitions) + val rawSplits = inputFormat.getSplits(jobContext).toArray + val result = new Array[Partition](rawSplits.size) + for (i <- 0 until rawSplits.size) { + result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) + } + result + } +} diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index 4333a679c8aa..3cb1231bd347 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -54,7 +54,8 @@ private[spark] abstract class ZippedPartitionsBaseRDD[V: ClassTag]( override def getPartitions: Array[Partition] = { val numParts = rdds.head.partitions.length if (!rdds.forall(rdd => rdd.partitions.length == numParts)) { - throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions") + throw new IllegalArgumentException( + s"Can't zip RDDs with unequal numbers of partitions: ${rdds.map(_.partitions.length)}") } Array.tabulate[Partition](numParts) { i => val prefs = rdds.map(rdd => rdd.preferredLocations(rdd.partitions(i))) diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala index 32931d59acb1..8425b211d6ec 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala @@ -29,7 +29,7 @@ class ZippedWithIndexRDDPartition(val prev: Partition, val startIndex: Long) } /** - * Represents a RDD zipped with its element indices. The ordering is first based on the partition + * Represents an RDD zipped with its element indices. The ordering is first based on the partition * index and then the ordering of items within each partition. So the first item in the first * partition gets index 0, and the last item in the last partition receives the largest index. * @@ -43,7 +43,7 @@ class ZippedWithIndexRDD[T: ClassTag](prev: RDD[T]) extends RDD[(T, Long)](prev) @transient private val startIndices: Array[Long] = { val n = prev.partitions.length if (n == 0) { - Array[Long]() + Array.empty } else if (n == 1) { Array(0L) } else { @@ -64,8 +64,7 @@ class ZippedWithIndexRDD[T: ClassTag](prev: RDD[T]) extends RDD[(T, Long)](prev) override def compute(splitIn: Partition, context: TaskContext): Iterator[(T, Long)] = { val split = splitIn.asInstanceOf[ZippedWithIndexRDDPartition] - firstParent[T].iterator(split.prev, context).zipWithIndex.map { x => - (x._1, split.startIndex + x._2) - } + val parentIter = firstParent[T].iterator(split.prev, context) + Utils.getIteratorZipWithIndex(parentIter, split.startIndex) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/coalesce-public.scala b/core/src/main/scala/org/apache/spark/rdd/coalesce-public.scala new file mode 100644 index 000000000000..1f8ab784a92b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/coalesce-public.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.rdd + +import scala.collection.mutable + +import org.apache.spark.Partition +import org.apache.spark.annotation.DeveloperApi + +/** + * ::DeveloperApi:: + * A PartitionCoalescer defines how to coalesce the partitions of a given RDD. + */ +@DeveloperApi +trait PartitionCoalescer { + + /** + * Coalesce the partitions of the given RDD. + * + * @param maxPartitions the maximum number of partitions to have after coalescing + * @param parent the parent RDD whose partitions to coalesce + * @return an array of [[PartitionGroup]]s, where each element is itself an array of + * `Partition`s and represents a partition after coalescing is performed. + */ + def coalesce(maxPartitions: Int, parent: RDD[_]): Array[PartitionGroup] +} + +/** + * ::DeveloperApi:: + * A group of `Partition`s + * @param prefLoc preferred location for the partition group + */ +@DeveloperApi +class PartitionGroup(val prefLoc: Option[String] = None) { + val partitions = mutable.ArrayBuffer[Partition]() + def numPartitions: Int = partitions.size +} diff --git a/core/src/main/scala/org/apache/spark/rdd/package-info.java b/core/src/main/scala/org/apache/spark/rdd/package-info.java index 176cc58179fb..d9aa9bebe56d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/package-info.java +++ b/core/src/main/scala/org/apache/spark/rdd/package-info.java @@ -18,4 +18,4 @@ /** * Provides implementation's of various RDDs. */ -package org.apache.spark.rdd; \ No newline at end of file +package org.apache.spark.rdd; diff --git a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicRDDCheckpointer.scala b/core/src/main/scala/org/apache/spark/rdd/util/PeriodicRDDCheckpointer.scala similarity index 95% rename from mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicRDDCheckpointer.scala rename to core/src/main/scala/org/apache/spark/rdd/util/PeriodicRDDCheckpointer.scala index f31ed2aa90a6..facbb830a60d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicRDDCheckpointer.scala +++ b/core/src/main/scala/org/apache/spark/rdd/util/PeriodicRDDCheckpointer.scala @@ -15,11 +15,12 @@ * limitations under the License. */ -package org.apache.spark.mllib.impl +package org.apache.spark.rdd.util import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.PeriodicCheckpointer /** @@ -49,6 +50,7 @@ import org.apache.spark.storage.StorageLevel * {{{ * val (rdd1, rdd2, rdd3, ...) = ... * val cp = new PeriodicRDDCheckpointer(2, sc) + * cp.update(rdd1) * rdd1.count(); * // persisted: rdd1 * cp.update(rdd2) @@ -74,7 +76,7 @@ import org.apache.spark.storage.StorageLevel * * TODO: Move this out of MLlib? */ -private[mllib] class PeriodicRDDCheckpointer[T]( +private[spark] class PeriodicRDDCheckpointer[T]( checkpointInterval: Int, sc: SparkContext) extends PeriodicCheckpointer[RDD[T]](checkpointInterval, sc) { diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala b/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala index f527ec86ab7b..117f51c5b8f2 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala @@ -18,7 +18,7 @@ package org.apache.spark.rpc /** - * A callback that [[RpcEndpoint]] can use it to send back a message or failure. It's thread-safe + * A callback that [[RpcEndpoint]] can use to send back a message or failure. It's thread-safe * and can be called in any thread. */ private[spark] trait RpcCallContext { diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala index 0ba95169529e..97eed540b8f5 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpoint.scala @@ -35,7 +35,7 @@ private[spark] trait RpcEnvFactory { * * The life-cycle of an endpoint is: * - * constructor -> onStart -> receive* -> onStop + * {@code constructor -> onStart -> receive* -> onStop} * * Note: `receive` can be called concurrently. If you want `receive` to be thread-safe, please use * [[ThreadSafeRpcEndpoint]] @@ -63,16 +63,16 @@ private[spark] trait RpcEndpoint { } /** - * Process messages from [[RpcEndpointRef.send]] or [[RpcCallContext.reply)]]. If receiving a - * unmatched message, [[SparkException]] will be thrown and sent to `onError`. + * Process messages from `RpcEndpointRef.send` or `RpcCallContext.reply`. If receiving a + * unmatched message, `SparkException` will be thrown and sent to `onError`. */ def receive: PartialFunction[Any, Unit] = { case _ => throw new SparkException(self + " does not implement 'receive'") } /** - * Process messages from [[RpcEndpointRef.ask]]. If receiving a unmatched message, - * [[SparkException]] will be thrown and sent to `onError`. + * Process messages from `RpcEndpointRef.ask`. If receiving a unmatched message, + * `SparkException` will be thrown and sent to `onError`. */ def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case _ => context.sendFailure(new SparkException(self + " won't reply anything")) diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/RpcEndpointAddress.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointAddress.scala similarity index 83% rename from core/src/main/scala/org/apache/spark/rpc/netty/RpcEndpointAddress.scala rename to core/src/main/scala/org/apache/spark/rpc/RpcEndpointAddress.scala index d2e94f943aba..fdbccc9e74c3 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/RpcEndpointAddress.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointAddress.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.rpc.netty +package org.apache.spark.rpc import org.apache.spark.SparkException -import org.apache.spark.rpc.RpcAddress /** * An address identifier for an RPC endpoint. @@ -26,10 +25,11 @@ import org.apache.spark.rpc.RpcAddress * The `rpcAddress` may be null, in which case the endpoint is registered via a client-only * connection and can only be reached via the client that sent the endpoint reference. * - * @param rpcAddress The socket address of the endpint. + * @param rpcAddress The socket address of the endpoint. It's `null` when this address pointing to + * an endpoint in a client `NettyRpcEnv`. * @param name Name of the endpoint. */ -private[netty] case class RpcEndpointAddress(val rpcAddress: RpcAddress, val name: String) { +private[spark] case class RpcEndpointAddress(rpcAddress: RpcAddress, name: String) { require(name != null, "RpcEndpoint name must be provided.") @@ -44,7 +44,11 @@ private[netty] case class RpcEndpointAddress(val rpcAddress: RpcAddress, val nam } } -private[netty] object RpcEndpointAddress { +private[spark] object RpcEndpointAddress { + + def apply(host: String, port: Int, name: String): RpcEndpointAddress = { + new RpcEndpointAddress(host, port, name) + } def apply(sparkUrl: String): RpcEndpointAddress = { try { diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala index 623da3e9c11b..4d39f144dd19 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala @@ -20,8 +20,9 @@ package org.apache.spark.rpc import scala.concurrent.Future import scala.reflect.ClassTag +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.util.RpcUtils -import org.apache.spark.{SparkException, Logging, SparkConf} /** * A reference for a remote [[RpcEndpoint]]. [[RpcEndpointRef]] is thread-safe. @@ -62,25 +63,21 @@ private[spark] abstract class RpcEndpointRef(conf: SparkConf) def ask[T: ClassTag](message: Any): Future[T] = ask(message, defaultAskTimeout) /** - * Send a message to the corresponding [[RpcEndpoint]] and get its result within a default - * timeout, or throw a SparkException if this fails even after the default number of retries. - * The default `timeout` will be used in every trial of calling `sendWithReply`. Because this - * method retries, the message handling in the receiver side should be idempotent. + * Send a message to the corresponding [[RpcEndpoint.receiveAndReply]] and get its result within a + * default timeout, throw an exception if this fails. * * Note: this is a blocking action which may cost a lot of time, so don't call it in a message * loop of [[RpcEndpoint]]. - * + * @param message the message to send * @tparam T type of the reply message * @return the reply message from the corresponding [[RpcEndpoint]] */ - def askWithRetry[T: ClassTag](message: Any): T = askWithRetry(message, defaultAskTimeout) + def askSync[T: ClassTag](message: Any): T = askSync(message, defaultAskTimeout) /** - * Send a message to the corresponding [[RpcEndpoint.receive]] and get its result within a - * specified timeout, throw a SparkException if this fails even after the specified number of - * retries. `timeout` will be used in every trial of calling `sendWithReply`. Because this method - * retries, the message handling in the receiver side should be idempotent. + * Send a message to the corresponding [[RpcEndpoint.receiveAndReply]] and get its result within a + * specified timeout, throw an exception if this fails. * * Note: this is a blocking action which may cost a lot of time, so don't call it in a message * loop of [[RpcEndpoint]]. @@ -90,33 +87,9 @@ private[spark] abstract class RpcEndpointRef(conf: SparkConf) * @tparam T type of the reply message * @return the reply message from the corresponding [[RpcEndpoint]] */ - def askWithRetry[T: ClassTag](message: Any, timeout: RpcTimeout): T = { - // TODO: Consider removing multiple attempts - var attempts = 0 - var lastException: Exception = null - while (attempts < maxRetries) { - attempts += 1 - try { - val future = ask[T](message, timeout) - val result = timeout.awaitResult(future) - if (result == null) { - throw new SparkException("RpcEndpoint returned null") - } - return result - } catch { - case ie: InterruptedException => throw ie - case e: Exception => - lastException = e - logWarning(s"Error sending message [message = $message] in $attempts attempts", e) - } - - if (attempts < maxRetries) { - Thread.sleep(retryWaitMs) - } - } - - throw new SparkException( - s"Error sending message [message = $message]", lastException) + def askSync[T: ClassTag](message: Any, timeout: RpcTimeout): T = { + val future = ask[T](message, timeout) + timeout.awaitResult(future) } } diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala index a560fd10cdf7..de2cc56bc6b1 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala @@ -17,10 +17,14 @@ package org.apache.spark.rpc +import java.io.File +import java.nio.channels.ReadableByteChannel + import scala.concurrent.Future import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.util.{RpcUtils, Utils} +import org.apache.spark.rpc.netty.NettyRpcEnvFactory +import org.apache.spark.util.RpcUtils /** @@ -29,15 +33,6 @@ import org.apache.spark.util.{RpcUtils, Utils} */ private[spark] object RpcEnv { - private def getRpcEnvFactory(conf: SparkConf): RpcEnvFactory = { - val rpcEnvNames = Map( - "akka" -> "org.apache.spark.rpc.akka.AkkaRpcEnvFactory", - "netty" -> "org.apache.spark.rpc.netty.NettyRpcEnvFactory") - val rpcEnvName = conf.get("spark.rpc", "netty") - val rpcEnvFactoryClassName = rpcEnvNames.getOrElse(rpcEnvName.toLowerCase, rpcEnvName) - Utils.classForName(rpcEnvFactoryClassName).newInstance().asInstanceOf[RpcEnvFactory] - } - def create( name: String, host: String, @@ -45,9 +40,21 @@ private[spark] object RpcEnv { conf: SparkConf, securityManager: SecurityManager, clientMode: Boolean = false): RpcEnv = { - // Using Reflection to create the RpcEnv to avoid to depend on Akka directly - val config = RpcEnvConfig(conf, name, host, port, securityManager, clientMode) - getRpcEnvFactory(conf).create(config) + create(name, host, host, port, conf, securityManager, 0, clientMode) + } + + def create( + name: String, + bindAddress: String, + advertiseAddress: String, + port: Int, + conf: SparkConf, + securityManager: SecurityManager, + numUsableCores: Int, + clientMode: Boolean): RpcEnv = { + val config = RpcEnvConfig(conf, name, bindAddress, advertiseAddress, port, securityManager, + numUsableCores, clientMode) + new NettyRpcEnvFactory().create(config) } } @@ -95,12 +102,11 @@ private[spark] abstract class RpcEnv(conf: SparkConf) { } /** - * Retrieve the [[RpcEndpointRef]] represented by `systemName`, `address` and `endpointName`. + * Retrieve the [[RpcEndpointRef]] represented by `address` and `endpointName`. * This is a blocking action. */ - def setupEndpointRef( - systemName: String, address: RpcAddress, endpointName: String): RpcEndpointRef = { - setupEndpointRefByURI(uriOf(systemName, address, endpointName)) + def setupEndpointRef(address: RpcAddress, endpointName: String): RpcEndpointRef = { + setupEndpointRefByURI(RpcEndpointAddress(address, endpointName).toString) } /** @@ -121,24 +127,80 @@ private[spark] abstract class RpcEnv(conf: SparkConf) { */ def awaitTermination(): Unit - /** - * Create a URI used to create a [[RpcEndpointRef]]. Use this one to create the URI instead of - * creating it manually because different [[RpcEnv]] may have different formats. - */ - def uriOf(systemName: String, address: RpcAddress, endpointName: String): String - /** * [[RpcEndpointRef]] cannot be deserialized without [[RpcEnv]]. So when deserializing any object * that contains [[RpcEndpointRef]]s, the deserialization codes should be wrapped by this method. */ def deserialize[T](deserializationAction: () => T): T + + /** + * Return the instance of the file server used to serve files. This may be `null` if the + * RpcEnv is not operating in server mode. + */ + def fileServer: RpcEnvFileServer + + /** + * Open a channel to download a file from the given URI. If the URIs returned by the + * RpcEnvFileServer use the "spark" scheme, this method will be called by the Utils class to + * retrieve the files. + * + * @param uri URI with location of the file. + */ + def openChannel(uri: String): ReadableByteChannel } +/** + * A server used by the RpcEnv to server files to other processes owned by the application. + * + * The file server can return URIs handled by common libraries (such as "http" or "hdfs"), or + * it can return "spark" URIs which will be handled by `RpcEnv#fetchFile`. + */ +private[spark] trait RpcEnvFileServer { + + /** + * Adds a file to be served by this RpcEnv. This is used to serve files from the driver + * to executors when they're stored on the driver's local file system. + * + * @param file Local file to serve. + * @return A URI for the location of the file. + */ + def addFile(file: File): String + + /** + * Adds a jar to be served by this RpcEnv. Similar to `addFile` but for jars added using + * `SparkContext.addJar`. + * + * @param file Local file to serve. + * @return A URI for the location of the file. + */ + def addJar(file: File): String + + /** + * Adds a local directory to be served via this file server. + * + * @param baseUri Leading URI path (files can be retrieved by appending their relative + * path to this base URI). This cannot be "files" nor "jars". + * @param path Path to the local directory. + * @return URI for the root of the directory in the file server. + */ + def addDirectory(baseUri: String, path: File): String + + /** Validates and normalizes the base URI for directories. */ + protected def validateDirectoryUri(baseUri: String): String = { + val fixedBaseUri = "/" + baseUri.stripPrefix("/").stripSuffix("/") + require(fixedBaseUri != "/files" && fixedBaseUri != "/jars", + "Directory URI cannot be /files nor /jars.") + fixedBaseUri + } + +} private[spark] case class RpcEnvConfig( conf: SparkConf, name: String, - host: String, + bindAddress: String, + advertiseAddress: String, port: Int, securityManager: SecurityManager, + numUsableCores: Int, clientMode: Boolean) diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnvStoppedException.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnvStoppedException.scala new file mode 100644 index 000000000000..c296cc23f12b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnvStoppedException.scala @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.rpc + +private[rpc] class RpcEnvStoppedException() + extends IllegalStateException("RpcEnv already stopped.") diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala b/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala index 285786ebf9f1..3dc41f7f1279 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala @@ -19,15 +19,14 @@ package org.apache.spark.rpc import java.util.concurrent.TimeoutException -import scala.concurrent.{Awaitable, Await} +import scala.concurrent.Future import scala.concurrent.duration._ import org.apache.spark.SparkConf -import org.apache.spark.util.Utils - +import org.apache.spark.util.{ThreadUtils, Utils} /** - * An exception thrown if RpcTimeout modifies a [[TimeoutException]]. + * An exception thrown if RpcTimeout modifies a `TimeoutException`. */ private[rpc] class RpcTimeoutException(message: String, cause: TimeoutException) extends TimeoutException(message) { initCause(cause) } @@ -66,13 +65,14 @@ private[spark] class RpcTimeout(val duration: FiniteDuration, val timeoutProp: S /** * Wait for the completed result and return it. If the result is not available within this * timeout, throw a [[RpcTimeoutException]] to indicate which configuration controls the timeout. - * @param awaitable the `Awaitable` to be awaited - * @throws RpcTimeoutException if after waiting for the specified time `awaitable` + * + * @param future the `Future` to be awaited + * @throws RpcTimeoutException if after waiting for the specified time `future` * is still not ready */ - def awaitResult[T](awaitable: Awaitable[T]): T = { + def awaitResult[T](future: Future[T]): T = { try { - Await.result(awaitable, duration) + ThreadUtils.awaitResult(future, duration) } catch addMessageIfTimeout } } @@ -83,6 +83,7 @@ private[spark] object RpcTimeout { /** * Lookup the timeout property in the configuration and create * a RpcTimeout with the property key in the description. + * * @param conf configuration properties containing the timeout * @param timeoutProp property key for the timeout in seconds * @throws NoSuchElementException if property is not set @@ -96,6 +97,7 @@ private[spark] object RpcTimeout { * Lookup the timeout property in the configuration and create * a RpcTimeout with the property key in the description. * Uses the given default value if property is not set + * * @param conf configuration properties containing the timeout * @param timeoutProp property key for the timeout in seconds * @param defaultValue default timeout value in seconds if property not found @@ -110,6 +112,7 @@ private[spark] object RpcTimeout { * and create a RpcTimeout with the first set property key in the * description. * Uses the given default value if property is not set + * * @param conf configuration properties containing the timeout * @param timeoutPropList prioritized list of property keys for the timeout in seconds * @param defaultValue default timeout value in seconds if no properties found @@ -120,11 +123,11 @@ private[spark] object RpcTimeout { // Find the first set property or use the default value with the first property val itr = timeoutPropList.iterator var foundProp: Option[(String, String)] = None - while (itr.hasNext && foundProp.isEmpty){ + while (itr.hasNext && foundProp.isEmpty) { val propKey = itr.next() - conf.getOption(propKey).foreach { prop => foundProp = Some(propKey, prop) } + conf.getOption(propKey).foreach { prop => foundProp = Some((propKey, prop)) } } - val finalProp = foundProp.getOrElse(timeoutPropList.head, defaultValue) + val finalProp = foundProp.getOrElse((timeoutPropList.head, defaultValue)) val timeout = { Utils.timeStringAsSeconds(finalProp._2).seconds } new RpcTimeout(timeout, finalProp._1) } diff --git a/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala deleted file mode 100644 index 3fad595a0d0b..000000000000 --- a/core/src/main/scala/org/apache/spark/rpc/akka/AkkaRpcEnv.scala +++ /dev/null @@ -1,345 +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.rpc.akka - -import java.util.concurrent.ConcurrentHashMap - -import scala.concurrent.Future -import scala.language.postfixOps -import scala.reflect.ClassTag -import scala.util.control.NonFatal - -import akka.actor.{ActorSystem, ExtendedActorSystem, Actor, ActorRef, Props, Address} -import akka.event.Logging.Error -import akka.pattern.{ask => akkaAsk} -import akka.remote.{AssociationEvent, AssociatedEvent, DisassociatedEvent, AssociationErrorEvent} -import akka.serialization.JavaSerializer - -import org.apache.spark.{SparkException, Logging, SparkConf} -import org.apache.spark.rpc._ -import org.apache.spark.util.{ActorLogReceive, AkkaUtils, ThreadUtils} - -/** - * A RpcEnv implementation based on Akka. - * - * TODO Once we remove all usages of Akka in other place, we can move this file to a new project and - * remove Akka from the dependencies. - */ -private[spark] class AkkaRpcEnv private[akka] ( - val actorSystem: ActorSystem, conf: SparkConf, boundPort: Int) - extends RpcEnv(conf) with Logging { - - private val defaultAddress: RpcAddress = { - val address = actorSystem.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress - // In some test case, ActorSystem doesn't bind to any address. - // So just use some default value since they are only some unit tests - RpcAddress(address.host.getOrElse("localhost"), address.port.getOrElse(boundPort)) - } - - override val address: RpcAddress = defaultAddress - - /** - * A lookup table to search a [[RpcEndpointRef]] for a [[RpcEndpoint]]. We need it to make - * [[RpcEndpoint.self]] work. - */ - private val endpointToRef = new ConcurrentHashMap[RpcEndpoint, RpcEndpointRef]() - - /** - * Need this map to remove `RpcEndpoint` from `endpointToRef` via a `RpcEndpointRef` - */ - private val refToEndpoint = new ConcurrentHashMap[RpcEndpointRef, RpcEndpoint]() - - private def registerEndpoint(endpoint: RpcEndpoint, endpointRef: RpcEndpointRef): Unit = { - endpointToRef.put(endpoint, endpointRef) - refToEndpoint.put(endpointRef, endpoint) - } - - private def unregisterEndpoint(endpointRef: RpcEndpointRef): Unit = { - val endpoint = refToEndpoint.remove(endpointRef) - if (endpoint != null) { - endpointToRef.remove(endpoint) - } - } - - /** - * Retrieve the [[RpcEndpointRef]] of `endpoint`. - */ - override def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef = endpointToRef.get(endpoint) - - override def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = { - @volatile var endpointRef: AkkaRpcEndpointRef = null - // Use defered function because the Actor needs to use `endpointRef`. - // So `actorRef` should be created after assigning `endpointRef`. - val actorRef = () => actorSystem.actorOf(Props(new Actor with ActorLogReceive with Logging { - - assert(endpointRef != null) - - override def preStart(): Unit = { - // Listen for remote client network events - context.system.eventStream.subscribe(self, classOf[AssociationEvent]) - safelyCall(endpoint) { - endpoint.onStart() - } - } - - override def receiveWithLogging: Receive = { - case AssociatedEvent(_, remoteAddress, _) => - safelyCall(endpoint) { - endpoint.onConnected(akkaAddressToRpcAddress(remoteAddress)) - } - - case DisassociatedEvent(_, remoteAddress, _) => - safelyCall(endpoint) { - endpoint.onDisconnected(akkaAddressToRpcAddress(remoteAddress)) - } - - case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound, _) => - safelyCall(endpoint) { - endpoint.onNetworkError(cause, akkaAddressToRpcAddress(remoteAddress)) - } - - case e: AssociationEvent => - // TODO ignore? - - case m: AkkaMessage => - logDebug(s"Received RPC message: $m") - safelyCall(endpoint) { - processMessage(endpoint, m, sender) - } - - case AkkaFailure(e) => - safelyCall(endpoint) { - throw e - } - - case message: Any => { - logWarning(s"Unknown message: $message") - } - - } - - override def postStop(): Unit = { - unregisterEndpoint(endpoint.self) - safelyCall(endpoint) { - endpoint.onStop() - } - } - - }), name = name) - endpointRef = new AkkaRpcEndpointRef(defaultAddress, actorRef, conf, initInConstructor = false) - registerEndpoint(endpoint, endpointRef) - // Now actorRef can be created safely - endpointRef.init() - endpointRef - } - - private def processMessage(endpoint: RpcEndpoint, m: AkkaMessage, _sender: ActorRef): Unit = { - val message = m.message - val needReply = m.needReply - val pf: PartialFunction[Any, Unit] = - if (needReply) { - endpoint.receiveAndReply(new RpcCallContext { - override def sendFailure(e: Throwable): Unit = { - _sender ! AkkaFailure(e) - } - - override def reply(response: Any): Unit = { - _sender ! AkkaMessage(response, false) - } - - // Use "lazy" because most of RpcEndpoints don't need "senderAddress" - override lazy val senderAddress: RpcAddress = - new AkkaRpcEndpointRef(defaultAddress, _sender, conf).address - }) - } else { - endpoint.receive - } - try { - pf.applyOrElse[Any, Unit](message, { message => - throw new SparkException(s"Unmatched message $message from ${_sender}") - }) - } catch { - case NonFatal(e) => - _sender ! AkkaFailure(e) - if (!needReply) { - // If the sender does not require a reply, it may not handle the exception. So we rethrow - // "e" to make sure it will be processed. - throw e - } - } - } - - /** - * Run `action` safely to avoid to crash the thread. If any non-fatal exception happens, it will - * call `endpoint.onError`. If `endpoint.onError` throws any non-fatal exception, just log it. - */ - private def safelyCall(endpoint: RpcEndpoint)(action: => Unit): Unit = { - try { - action - } catch { - case NonFatal(e) => { - try { - endpoint.onError(e) - } catch { - case NonFatal(e) => logError(s"Ignore error: ${e.getMessage}", e) - } - } - } - } - - private def akkaAddressToRpcAddress(address: Address): RpcAddress = { - RpcAddress(address.host.getOrElse(defaultAddress.host), - address.port.getOrElse(defaultAddress.port)) - } - - override def asyncSetupEndpointRefByURI(uri: String): Future[RpcEndpointRef] = { - import actorSystem.dispatcher - actorSystem.actorSelection(uri).resolveOne(defaultLookupTimeout.duration). - map(new AkkaRpcEndpointRef(defaultAddress, _, conf)). - // this is just in case there is a timeout from creating the future in resolveOne, we want the - // exception to indicate the conf that determines the timeout - recover(defaultLookupTimeout.addMessageIfTimeout) - } - - override def uriOf(systemName: String, address: RpcAddress, endpointName: String): String = { - AkkaUtils.address( - AkkaUtils.protocol(actorSystem), systemName, address.host, address.port, endpointName) - } - - override def shutdown(): Unit = { - actorSystem.shutdown() - } - - override def stop(endpoint: RpcEndpointRef): Unit = { - require(endpoint.isInstanceOf[AkkaRpcEndpointRef]) - actorSystem.stop(endpoint.asInstanceOf[AkkaRpcEndpointRef].actorRef) - } - - override def awaitTermination(): Unit = { - actorSystem.awaitTermination() - } - - override def toString: String = s"${getClass.getSimpleName}($actorSystem)" - - override def deserialize[T](deserializationAction: () => T): T = { - JavaSerializer.currentSystem.withValue(actorSystem.asInstanceOf[ExtendedActorSystem]) { - deserializationAction() - } - } -} - -private[spark] class AkkaRpcEnvFactory extends RpcEnvFactory { - - def create(config: RpcEnvConfig): RpcEnv = { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem( - config.name, config.host, config.port, config.conf, config.securityManager) - actorSystem.actorOf(Props(classOf[ErrorMonitor]), "ErrorMonitor") - new AkkaRpcEnv(actorSystem, config.conf, boundPort) - } -} - -/** - * Monitor errors reported by Akka and log them. - */ -private[akka] class ErrorMonitor extends Actor with ActorLogReceive with Logging { - - override def preStart(): Unit = { - context.system.eventStream.subscribe(self, classOf[Error]) - } - - override def receiveWithLogging: Actor.Receive = { - case Error(cause: Throwable, _, _, message: String) => logError(message, cause) - } -} - -private[akka] class AkkaRpcEndpointRef( - @transient private val defaultAddress: RpcAddress, - @transient private val _actorRef: () => ActorRef, - conf: SparkConf, - initInConstructor: Boolean) - extends RpcEndpointRef(conf) with Logging { - - def this( - defaultAddress: RpcAddress, - _actorRef: ActorRef, - conf: SparkConf) = { - this(defaultAddress, () => _actorRef, conf, true) - } - - lazy val actorRef = _actorRef() - - override lazy val address: RpcAddress = { - val akkaAddress = actorRef.path.address - RpcAddress(akkaAddress.host.getOrElse(defaultAddress.host), - akkaAddress.port.getOrElse(defaultAddress.port)) - } - - override lazy val name: String = actorRef.path.name - - private[akka] def init(): Unit = { - // Initialize the lazy vals - actorRef - address - name - } - - if (initInConstructor) { - init() - } - - override def send(message: Any): Unit = { - actorRef ! AkkaMessage(message, false) - } - - override def ask[T: ClassTag](message: Any, timeout: RpcTimeout): Future[T] = { - actorRef.ask(AkkaMessage(message, true))(timeout.duration).flatMap { - // The function will run in the calling thread, so it should be short and never block. - case msg @ AkkaMessage(message, reply) => - if (reply) { - logError(s"Receive $msg but the sender cannot reply") - Future.failed(new SparkException(s"Receive $msg but the sender cannot reply")) - } else { - Future.successful(message) - } - case AkkaFailure(e) => - Future.failed(e) - }(ThreadUtils.sameThread).mapTo[T]. - recover(timeout.addMessageIfTimeout)(ThreadUtils.sameThread) - } - - override def toString: String = s"${getClass.getSimpleName}($actorRef)" - - final override def equals(that: Any): Boolean = that match { - case other: AkkaRpcEndpointRef => actorRef == other.actorRef - case _ => false - } - - final override def hashCode(): Int = if (actorRef == null) 0 else actorRef.hashCode() -} - -/** - * A wrapper to `message` so that the receiver knows if the sender expects a reply. - * @param message - * @param needReply if the sender expects a reply message - */ -private[akka] case class AkkaMessage(message: Any, needReply: Boolean) - -/** - * A reply with the failure error from the receiver to the sender - */ -private[akka] case class AkkaFailure(e: Throwable) diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala index eb25d6c7b721..904c4d02dd2a 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala @@ -17,22 +17,26 @@ package org.apache.spark.rpc.netty -import java.util.concurrent.{ThreadPoolExecutor, ConcurrentHashMap, LinkedBlockingQueue, TimeUnit} +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit} import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ import scala.concurrent.Promise import scala.util.control.NonFatal -import org.apache.spark.{SparkException, Logging} +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging import org.apache.spark.network.client.RpcResponseCallback import org.apache.spark.rpc._ import org.apache.spark.util.ThreadUtils /** * A message dispatcher, responsible for routing RPC messages to the appropriate endpoint(s). + * + * @param numUsableCores Number of CPU cores allocated to the process, for sizing the thread pool. + * If 0, will consider the available CPUs on the host. */ -private[netty] class Dispatcher(nettyEnv: NettyRpcEnv) extends Logging { +private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) extends Logging { private class EndpointData( val name: String, @@ -41,8 +45,10 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv) extends Logging { val inbox = new Inbox(ref, endpoint) } - private val endpoints = new ConcurrentHashMap[String, EndpointData] - private val endpointRefs = new ConcurrentHashMap[RpcEndpoint, RpcEndpointRef] + private val endpoints: ConcurrentMap[String, EndpointData] = + new ConcurrentHashMap[String, EndpointData] + private val endpointRefs: ConcurrentMap[RpcEndpoint, RpcEndpointRef] = + new ConcurrentHashMap[RpcEndpoint, RpcEndpointRef] // Track the receivers whose inboxes may contain messages. private val receivers = new LinkedBlockingQueue[EndpointData] @@ -106,71 +112,60 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv) extends Logging { val iter = endpoints.keySet().iterator() while (iter.hasNext) { val name = iter.next - postMessage( - name, - _ => message, - () => { logWarning(s"Drop $message because $name has been stopped") }) - } + postMessage(name, message, (e) => { e match { + case e: RpcEnvStoppedException => logDebug (s"Message $message dropped. ${e.getMessage}") + case e: Throwable => logWarning(s"Message $message dropped. ${e.getMessage}") + }} + )} } /** Posts a message sent by a remote endpoint. */ def postRemoteMessage(message: RequestMessage, callback: RpcResponseCallback): Unit = { - def createMessage(sender: NettyRpcEndpointRef): InboxMessage = { - val rpcCallContext = - new RemoteNettyRpcCallContext( - nettyEnv, sender, callback, message.senderAddress, message.needReply) - ContentMessage(message.senderAddress, message.content, message.needReply, rpcCallContext) - } - - def onEndpointStopped(): Unit = { - callback.onFailure( - new SparkException(s"Could not find ${message.receiver.name} or it has been stopped")) - } - - postMessage(message.receiver.name, createMessage, onEndpointStopped) + val rpcCallContext = + new RemoteNettyRpcCallContext(nettyEnv, callback, message.senderAddress) + val rpcMessage = RpcMessage(message.senderAddress, message.content, rpcCallContext) + postMessage(message.receiver.name, rpcMessage, (e) => callback.onFailure(e)) } /** Posts a message sent by a local endpoint. */ def postLocalMessage(message: RequestMessage, p: Promise[Any]): Unit = { - def createMessage(sender: NettyRpcEndpointRef): InboxMessage = { - val rpcCallContext = - new LocalNettyRpcCallContext(sender, message.senderAddress, message.needReply, p) - ContentMessage(message.senderAddress, message.content, message.needReply, rpcCallContext) - } - - def onEndpointStopped(): Unit = { - p.tryFailure( - new SparkException(s"Could not find ${message.receiver.name} or it has been stopped")) - } + val rpcCallContext = + new LocalNettyRpcCallContext(message.senderAddress, p) + val rpcMessage = RpcMessage(message.senderAddress, message.content, rpcCallContext) + postMessage(message.receiver.name, rpcMessage, (e) => p.tryFailure(e)) + } - postMessage(message.receiver.name, createMessage, onEndpointStopped) + /** Posts a one-way message. */ + def postOneWayMessage(message: RequestMessage): Unit = { + postMessage(message.receiver.name, OneWayMessage(message.senderAddress, message.content), + (e) => throw e) } /** * Posts a message to a specific endpoint. * * @param endpointName name of the endpoint. - * @param createMessageFn function to create the message. + * @param message the message to post * @param callbackIfStopped callback function if the endpoint is stopped. */ private def postMessage( endpointName: String, - createMessageFn: NettyRpcEndpointRef => InboxMessage, - callbackIfStopped: () => Unit): Unit = { - val shouldCallOnStop = synchronized { + message: InboxMessage, + callbackIfStopped: (Exception) => Unit): Unit = { + val error = synchronized { val data = endpoints.get(endpointName) - if (stopped || data == null) { - true + if (stopped) { + Some(new RpcEnvStoppedException()) + } else if (data == null) { + Some(new SparkException(s"Could not find $endpointName.")) } else { - data.inbox.post(createMessageFn(data.ref)) + data.inbox.post(message) receivers.offer(data) - false + None } } - if (shouldCallOnStop) { - // We don't need to call `onStop` in the `synchronized` block - callbackIfStopped() - } + // We don't need to call `onStop` in the `synchronized` block + error.foreach(callbackIfStopped) } def stop(): Unit = { @@ -200,8 +195,10 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv) extends Logging { /** Thread pool used for dispatching messages. */ private val threadpool: ThreadPoolExecutor = { + val availableCores = + if (numUsableCores > 0) numUsableCores else Runtime.getRuntime.availableProcessors() val numThreads = nettyEnv.conf.getInt("spark.rpc.netty.dispatcher.numThreads", - Runtime.getRuntime.availableProcessors()) + math.max(2, availableCores)) val pool = ThreadUtils.newDaemonFixedThreadPool(numThreads, "dispatcher-event-loop") for (i <- 0 until numThreads) { pool.execute(new MessageLoop) diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala index c72b588db57f..d32eba64e13e 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala @@ -21,18 +21,20 @@ import javax.annotation.concurrent.GuardedBy import scala.util.control.NonFatal -import com.google.common.annotations.VisibleForTesting - -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcAddress, RpcEndpoint, ThreadSafeRpcEndpoint} private[netty] sealed trait InboxMessage -private[netty] case class ContentMessage( +private[netty] case class OneWayMessage( + senderAddress: RpcAddress, + content: Any) extends InboxMessage + +private[netty] case class RpcMessage( senderAddress: RpcAddress, content: Any, - needReply: Boolean, context: NettyRpcCallContext) extends InboxMessage private[netty] case object OnStart extends InboxMessage @@ -50,7 +52,7 @@ private[netty] case class RemoteProcessConnectionError(cause: Throwable, remoteA extends InboxMessage /** - * A inbox that stores messages for an [[RpcEndpoint]] and posts messages to it thread-safely. + * An inbox that stores messages for an [[RpcEndpoint]] and posts messages to it thread-safely. */ private[netty] class Inbox( val endpointRef: NettyRpcEndpointRef, @@ -98,29 +100,24 @@ private[netty] class Inbox( while (true) { safelyCall(endpoint) { message match { - case ContentMessage(_sender, content, needReply, context) => - // The partial function to call - val pf = if (needReply) endpoint.receiveAndReply(context) else endpoint.receive + case RpcMessage(_sender, content, context) => try { - pf.applyOrElse[Any, Unit](content, { msg => + endpoint.receiveAndReply(context).applyOrElse[Any, Unit](content, { msg => throw new SparkException(s"Unsupported message $message from ${_sender}") }) - if (!needReply) { - context.finish() - } } catch { case NonFatal(e) => - if (needReply) { - // If the sender asks a reply, we should send the error back to the sender - context.sendFailure(e) - } else { - context.finish() - } + context.sendFailure(e) // Throw the exception -- this exception will be caught by the safelyCall function. // The endpoint's onError function will be called. throw e } + case OneWayMessage(_sender, content) => + endpoint.receive.applyOrElse[Any, Unit](content, { msg => + throw new SparkException(s"Unsupported message $message from ${_sender}") + }) + case OnStart => endpoint.onStart() if (!endpoint.isInstanceOf[ThreadSafeRpcEndpoint]) { @@ -193,8 +190,10 @@ private[netty] class Inbox( def isEmpty: Boolean = inbox.synchronized { messages.isEmpty } - /** Called when we are dropping a message. Test cases override this to test message dropping. */ - @VisibleForTesting + /** + * Called when we are dropping a message. Test cases override this to test message dropping. + * Exposed for testing. + */ protected def onDrop(message: InboxMessage): Unit = { logWarning(s"Drop $message because $endpointRef is stopped") } @@ -206,7 +205,12 @@ private[netty] class Inbox( try action catch { case NonFatal(e) => try endpoint.onError(e) catch { - case NonFatal(ee) => logError(s"Ignoring error", ee) + case NonFatal(ee) => + if (stopped) { + logDebug("Ignoring error", ee) + } else { + logError("Ignoring error", ee) + } } } } diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcCallContext.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcCallContext.scala index 21d5bb4923d1..7dd7e610a28e 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcCallContext.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcCallContext.scala @@ -19,53 +19,32 @@ package org.apache.spark.rpc.netty import scala.concurrent.Promise -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.network.client.RpcResponseCallback import org.apache.spark.rpc.{RpcAddress, RpcCallContext} -private[netty] abstract class NettyRpcCallContext( - endpointRef: NettyRpcEndpointRef, - override val senderAddress: RpcAddress, - needReply: Boolean) +private[netty] abstract class NettyRpcCallContext(override val senderAddress: RpcAddress) extends RpcCallContext with Logging { protected def send(message: Any): Unit override def reply(response: Any): Unit = { - if (needReply) { - send(AskResponse(endpointRef, response)) - } else { - throw new IllegalStateException( - s"Cannot send $response to the sender because the sender does not expect a reply") - } + send(response) } override def sendFailure(e: Throwable): Unit = { - if (needReply) { - send(AskResponse(endpointRef, RpcFailure(e))) - } else { - logError(e.getMessage, e) - throw new IllegalStateException( - "Cannot send reply to the sender because the sender won't handle it") - } + send(RpcFailure(e)) } - def finish(): Unit = { - if (!needReply) { - send(Ack(endpointRef)) - } - } } /** * If the sender and the receiver are in the same process, the reply can be sent back via `Promise`. */ private[netty] class LocalNettyRpcCallContext( - endpointRef: NettyRpcEndpointRef, senderAddress: RpcAddress, - needReply: Boolean, p: Promise[Any]) - extends NettyRpcCallContext(endpointRef, senderAddress, needReply) { + extends NettyRpcCallContext(senderAddress) { override protected def send(message: Any): Unit = { p.success(message) @@ -77,11 +56,9 @@ private[netty] class LocalNettyRpcCallContext( */ private[netty] class RemoteNettyRpcCallContext( nettyEnv: NettyRpcEnv, - endpointRef: NettyRpcEndpointRef, callback: RpcResponseCallback, - senderAddress: RpcAddress, - needReply: Boolean) - extends NettyRpcCallContext(endpointRef, senderAddress, needReply) { + senderAddress: RpcAddress) + extends NettyRpcCallContext(senderAddress) { override protected def send(message: Any): Unit = { val reply = nettyEnv.serialize(message) diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala index 09093819bb22..f951591e02a5 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala @@ -17,57 +17,69 @@ package org.apache.spark.rpc.netty import java.io._ -import java.lang.{Boolean => JBoolean} import java.net.{InetSocketAddress, URI} import java.nio.ByteBuffer +import java.nio.channels.{Pipe, ReadableByteChannel, WritableByteChannel} import java.util.concurrent._ import java.util.concurrent.atomic.AtomicBoolean -import javax.annotation.Nullable; -import javax.annotation.concurrent.GuardedBy +import javax.annotation.Nullable -import scala.collection.mutable import scala.concurrent.{Future, Promise} import scala.reflect.ClassTag -import scala.util.{DynamicVariable, Failure, Success} +import scala.util.{DynamicVariable, Failure, Success, Try} import scala.util.control.NonFatal -import com.google.common.base.Preconditions -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.internal.Logging import org.apache.spark.network.TransportContext import org.apache.spark.network.client._ +import org.apache.spark.network.crypto.{AuthClientBootstrap, AuthServerBootstrap} import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.sasl.{SaslClientBootstrap, SaslServerBootstrap} import org.apache.spark.network.server._ import org.apache.spark.rpc._ -import org.apache.spark.serializer.{JavaSerializer, JavaSerializerInstance} -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.serializer.{JavaSerializer, JavaSerializerInstance, SerializationStream} +import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, ThreadUtils, Utils} private[netty] class NettyRpcEnv( val conf: SparkConf, javaSerializerInstance: JavaSerializerInstance, host: String, - securityManager: SecurityManager) extends RpcEnv(conf) with Logging { + securityManager: SecurityManager, + numUsableCores: Int) extends RpcEnv(conf) with Logging { - private val transportConf = SparkTransportConf.fromSparkConf( - conf.clone.set("spark.shuffle.io.numConnectionsPerPeer", "1"), + private[netty] val transportConf = SparkTransportConf.fromSparkConf( + conf.clone.set("spark.rpc.io.numConnectionsPerPeer", "1"), + "rpc", conf.getInt("spark.rpc.io.threads", 0)) - private val dispatcher: Dispatcher = new Dispatcher(this) + private val dispatcher: Dispatcher = new Dispatcher(this, numUsableCores) + + private val streamManager = new NettyStreamManager(this) private val transportContext = new TransportContext(transportConf, - new NettyRpcHandler(dispatcher, this)) + new NettyRpcHandler(dispatcher, this, streamManager)) - private val clientFactory = { - val bootstraps: java.util.List[TransportClientBootstrap] = - if (securityManager.isAuthenticationEnabled()) { - java.util.Arrays.asList(new SaslClientBootstrap(transportConf, "", securityManager, - securityManager.isSaslEncryptionEnabled())) - } else { - java.util.Collections.emptyList[TransportClientBootstrap] - } - transportContext.createClientFactory(bootstraps) + private def createClientBootstraps(): java.util.List[TransportClientBootstrap] = { + if (securityManager.isAuthenticationEnabled()) { + java.util.Arrays.asList(new AuthClientBootstrap(transportConf, + securityManager.getSaslUser(), securityManager)) + } else { + java.util.Collections.emptyList[TransportClientBootstrap] + } } + private val clientFactory = transportContext.createClientFactory(createClientBootstraps()) + + /** + * A separate client factory for file downloads. This avoids using the same RPC handler as + * the main RPC context, so that events caused by these clients are kept isolated from the + * main RPC traffic. + * + * It also allows for different configuration of certain properties, such as the number of + * connections per peer. + */ + @volatile private var fileDownloadFactory: TransportClientFactory = _ + val timeoutScheduler = ThreadUtils.newDaemonSingleThreadScheduledExecutor("netty-rpc-env-timeout") // Because TransportClientFactory.createClient is blocking, we need to run it in this thread pool @@ -97,14 +109,14 @@ private[netty] class NettyRpcEnv( } } - def startServer(port: Int): Unit = { + def startServer(bindAddress: String, port: Int): Unit = { val bootstraps: java.util.List[TransportServerBootstrap] = if (securityManager.isAuthenticationEnabled()) { - java.util.Arrays.asList(new SaslServerBootstrap(transportConf, securityManager)) + java.util.Arrays.asList(new AuthServerBootstrap(transportConf, securityManager)) } else { java.util.Collections.emptyList() } - server = transportContext.createServer(port, bootstraps) + server = transportContext.createServer(bindAddress, port, bootstraps) dispatcher.registerRpcEndpoint( RpcEndpointVerifier.NAME, new RpcEndpointVerifier(this, dispatcher)) } @@ -139,7 +151,7 @@ private[netty] class NettyRpcEnv( private def postToOutbox(receiver: NettyRpcEndpointRef, message: OutboxMessage): Unit = { if (receiver.client != null) { - receiver.client.sendRpc(message.content, message.createCallback(receiver.client)); + message.sendWith(receiver.client) } else { require(receiver.address != null, "Cannot send message to client endpoint with no listen address.") @@ -171,25 +183,14 @@ private[netty] class NettyRpcEnv( val remoteAddr = message.receiver.address if (remoteAddr == address) { // Message to a local RPC endpoint. - val promise = Promise[Any]() - dispatcher.postLocalMessage(message, promise) - promise.future.onComplete { - case Success(response) => - val ack = response.asInstanceOf[Ack] - logTrace(s"Received ack from ${ack.sender}") - case Failure(e) => - logWarning(s"Exception when sending $message", e) - }(ThreadUtils.sameThread) + try { + dispatcher.postOneWayMessage(message) + } catch { + case e: RpcEnvStoppedException => logDebug(e.getMessage) + } } else { // Message to a remote RPC endpoint. - postToOutbox(message.receiver, OutboxMessage(serialize(message), - (e) => { - logWarning(s"Exception when sending $message", e) - }, - (client, response) => { - val ack = deserialize[Ack](client, response) - logDebug(s"Receive ack from ${ack.sender}") - })) + postToOutbox(message.receiver, OneWayOutboxMessage(message.serialize(this))) } } @@ -197,58 +198,77 @@ private[netty] class NettyRpcEnv( clientFactory.createClient(address.host, address.port) } - private[netty] def ask(message: RequestMessage): Future[Any] = { + private[netty] def ask[T: ClassTag](message: RequestMessage, timeout: RpcTimeout): Future[T] = { val promise = Promise[Any]() val remoteAddr = message.receiver.address - if (remoteAddr == address) { - val p = Promise[Any]() - dispatcher.postLocalMessage(message, p) - p.future.onComplete { - case Success(response) => - val reply = response.asInstanceOf[AskResponse] - if (reply.reply.isInstanceOf[RpcFailure]) { - if (!promise.tryFailure(reply.reply.asInstanceOf[RpcFailure].e)) { - logWarning(s"Ignore failure: ${reply.reply}") - } - } else if (!promise.trySuccess(reply.reply)) { - logWarning(s"Ignore message: ${reply}") - } - case Failure(e) => - if (!promise.tryFailure(e)) { - logWarning("Ignore Exception", e) - } + + def onFailure(e: Throwable): Unit = { + if (!promise.tryFailure(e)) { + e match { + case e : RpcEnvStoppedException => logDebug (s"Ignored failure: $e") + case _ => logWarning(s"Ignored failure: $e") + } + } + } + + def onSuccess(reply: Any): Unit = reply match { + case RpcFailure(e) => onFailure(e) + case rpcReply => + if (!promise.trySuccess(rpcReply)) { + logWarning(s"Ignored message: $reply") + } + } + + try { + if (remoteAddr == address) { + val p = Promise[Any]() + p.future.onComplete { + case Success(response) => onSuccess(response) + case Failure(e) => onFailure(e) + }(ThreadUtils.sameThread) + dispatcher.postLocalMessage(message, p) + } else { + val rpcMessage = RpcOutboxMessage(message.serialize(this), + onFailure, + (client, response) => onSuccess(deserialize[Any](client, response))) + postToOutbox(message.receiver, rpcMessage) + promise.future.failed.foreach { + case _: TimeoutException => rpcMessage.onTimeout() + case _ => + }(ThreadUtils.sameThread) + } + + val timeoutCancelable = timeoutScheduler.schedule(new Runnable { + override def run(): Unit = { + onFailure(new TimeoutException(s"Cannot receive any reply from ${remoteAddr} " + + s"in ${timeout.duration}")) + } + }, timeout.duration.toNanos, TimeUnit.NANOSECONDS) + promise.future.onComplete { v => + timeoutCancelable.cancel(true) }(ThreadUtils.sameThread) - } else { - postToOutbox(message.receiver, OutboxMessage(serialize(message), - (e) => { - if (!promise.tryFailure(e)) { - logWarning("Ignore Exception", e) - } - }, - (client, response) => { - val reply = deserialize[AskResponse](client, response) - if (reply.reply.isInstanceOf[RpcFailure]) { - if (!promise.tryFailure(reply.reply.asInstanceOf[RpcFailure].e)) { - logWarning(s"Ignore failure: ${reply.reply}") - } - } else if (!promise.trySuccess(reply.reply)) { - logWarning(s"Ignore message: ${reply}") - } - })) + } catch { + case NonFatal(e) => + onFailure(e) } - promise.future + promise.future.mapTo[T].recover(timeout.addMessageIfTimeout)(ThreadUtils.sameThread) + } + + private[netty] def serialize(content: Any): ByteBuffer = { + javaSerializerInstance.serialize(content) } - private[netty] def serialize(content: Any): Array[Byte] = { - val buffer = javaSerializerInstance.serialize(content) - java.util.Arrays.copyOfRange( - buffer.array(), buffer.arrayOffset + buffer.position, buffer.arrayOffset + buffer.limit) + /** + * Returns [[SerializationStream]] that forwards the serialized bytes to `out`. + */ + private[netty] def serializeStream(out: OutputStream): SerializationStream = { + javaSerializerInstance.serializeStream(out) } - private[netty] def deserialize[T: ClassTag](client: TransportClient, bytes: Array[Byte]): T = { + private[netty] def deserialize[T: ClassTag](client: TransportClient, bytes: ByteBuffer): T = { NettyRpcEnv.currentClient.withValue(client) { deserialize { () => - javaSerializerInstance.deserialize[T](ByteBuffer.wrap(bytes)) + javaSerializerInstance.deserialize[T](bytes) } } } @@ -257,9 +277,6 @@ private[netty] class NettyRpcEnv( dispatcher.getRpcEndpointRef(endpoint) } - override def uriOf(systemName: String, address: RpcAddress, endpointName: String): String = - new RpcEndpointAddress(address, endpointName).toString - override def shutdown(): Unit = { cleanup() } @@ -282,18 +299,21 @@ private[netty] class NettyRpcEnv( if (timeoutScheduler != null) { timeoutScheduler.shutdownNow() } + if (dispatcher != null) { + dispatcher.stop() + } if (server != null) { server.close() } if (clientFactory != null) { clientFactory.close() } - if (dispatcher != null) { - dispatcher.stop() - } if (clientConnectionExecutor != null) { clientConnectionExecutor.shutdownNow() } + if (fileDownloadFactory != null) { + fileDownloadFactory.close() + } } override def deserialize[T](deserializationAction: () => T): T = { @@ -302,10 +322,106 @@ private[netty] class NettyRpcEnv( } } + override def fileServer: RpcEnvFileServer = streamManager + + override def openChannel(uri: String): ReadableByteChannel = { + val parsedUri = new URI(uri) + require(parsedUri.getHost() != null, "Host name must be defined.") + require(parsedUri.getPort() > 0, "Port must be defined.") + require(parsedUri.getPath() != null && parsedUri.getPath().nonEmpty, "Path must be defined.") + + val pipe = Pipe.open() + val source = new FileDownloadChannel(pipe.source()) + try { + val client = downloadClient(parsedUri.getHost(), parsedUri.getPort()) + val callback = new FileDownloadCallback(pipe.sink(), source, client) + client.stream(parsedUri.getPath(), callback) + } catch { + case e: Exception => + pipe.sink().close() + source.close() + throw e + } + + source + } + + private def downloadClient(host: String, port: Int): TransportClient = { + if (fileDownloadFactory == null) synchronized { + if (fileDownloadFactory == null) { + val module = "files" + val prefix = "spark.rpc.io." + val clone = conf.clone() + + // Copy any RPC configuration that is not overridden in the spark.files namespace. + conf.getAll.foreach { case (key, value) => + if (key.startsWith(prefix)) { + val opt = key.substring(prefix.length()) + clone.setIfMissing(s"spark.$module.io.$opt", value) + } + } + + val ioThreads = clone.getInt("spark.files.io.threads", 1) + val downloadConf = SparkTransportConf.fromSparkConf(clone, module, ioThreads) + val downloadContext = new TransportContext(downloadConf, new NoOpRpcHandler(), true) + fileDownloadFactory = downloadContext.createClientFactory(createClientBootstraps()) + } + } + fileDownloadFactory.createClient(host, port) + } + + private class FileDownloadChannel(source: ReadableByteChannel) extends ReadableByteChannel { + + @volatile private var error: Throwable = _ + + def setError(e: Throwable): Unit = { + error = e + source.close() + } + + override def read(dst: ByteBuffer): Int = { + Try(source.read(dst)) match { + case Success(bytesRead) => bytesRead + case Failure(readErr) => + if (error != null) { + throw error + } else { + throw readErr + } + } + } + + override def close(): Unit = source.close() + + override def isOpen(): Boolean = source.isOpen() + + } + + private class FileDownloadCallback( + sink: WritableByteChannel, + source: FileDownloadChannel, + client: TransportClient) extends StreamCallback { + + override def onData(streamId: String, buf: ByteBuffer): Unit = { + while (buf.remaining() > 0) { + sink.write(buf) + } + } + + override def onComplete(streamId: String): Unit = { + sink.close() + } + + override def onFailure(streamId: String, cause: Throwable): Unit = { + logDebug(s"Error downloading stream $streamId.", cause) + source.setError(cause) + sink.close() + } + + } } private[netty] object NettyRpcEnv extends Logging { - /** * When deserializing the [[NettyRpcEndpointRef]], it needs a reference to [[NettyRpcEnv]]. * Use `currentEnv` to wrap the deserialization codes. E.g., @@ -326,7 +442,7 @@ private[netty] object NettyRpcEnv extends Logging { } -private[netty] class NettyRpcEnvFactory extends RpcEnvFactory with Logging { +private[rpc] class NettyRpcEnvFactory extends RpcEnvFactory with Logging { def create(config: RpcEnvConfig): RpcEnv = { val sparkConf = config.conf @@ -335,14 +451,15 @@ private[netty] class NettyRpcEnvFactory extends RpcEnvFactory with Logging { val javaSerializerInstance = new JavaSerializer(sparkConf).newInstance().asInstanceOf[JavaSerializerInstance] val nettyEnv = - new NettyRpcEnv(sparkConf, javaSerializerInstance, config.host, config.securityManager) + new NettyRpcEnv(sparkConf, javaSerializerInstance, config.advertiseAddress, + config.securityManager, config.numUsableCores) if (!config.clientMode) { val startNettyRpcEnv: Int => (NettyRpcEnv, Int) = { actualPort => - nettyEnv.startServer(actualPort) - (nettyEnv, actualPort) + nettyEnv.startServer(config.bindAddress, actualPort) + (nettyEnv, nettyEnv.address.port) } try { - Utils.startServiceOnPort(config.port, startNettyRpcEnv, sparkConf, "NettyRpcEnv")._1 + Utils.startServiceOnPort(config.port, startNettyRpcEnv, sparkConf, config.name)._1 } catch { case NonFatal(e) => nettyEnv.shutdown() @@ -372,20 +489,16 @@ private[netty] class NettyRpcEnvFactory extends RpcEnvFactory with Logging { * @param conf Spark configuration. * @param endpointAddress The address where the endpoint is listening. * @param nettyEnv The RpcEnv associated with this ref. - * @param local Whether the referenced endpoint lives in the same process. */ private[netty] class NettyRpcEndpointRef( @transient private val conf: SparkConf, - endpointAddress: RpcEndpointAddress, - @transient @volatile private var nettyEnv: NettyRpcEnv) - extends RpcEndpointRef(conf) with Serializable with Logging { + private val endpointAddress: RpcEndpointAddress, + @transient @volatile private var nettyEnv: NettyRpcEnv) extends RpcEndpointRef(conf) { @transient @volatile var client: TransportClient = _ - private val _address = if (endpointAddress.rpcAddress != null) endpointAddress else null - private val _name = endpointAddress.name - - override def address: RpcAddress = if (_address != null) _address.rpcAddress else null + override def address: RpcAddress = + if (endpointAddress.rpcAddress != null) endpointAddress.rpcAddress else null private def readObject(in: ObjectInputStream): Unit = { in.defaultReadObject() @@ -397,64 +510,103 @@ private[netty] class NettyRpcEndpointRef( out.defaultWriteObject() } - override def name: String = _name + override def name: String = endpointAddress.name override def ask[T: ClassTag](message: Any, timeout: RpcTimeout): Future[T] = { - val promise = Promise[Any]() - val timeoutCancelable = nettyEnv.timeoutScheduler.schedule(new Runnable { - override def run(): Unit = { - promise.tryFailure(new TimeoutException("Cannot receive any reply in " + timeout.duration)) - } - }, timeout.duration.toNanos, TimeUnit.NANOSECONDS) - val f = nettyEnv.ask(RequestMessage(nettyEnv.address, this, message, true)) - f.onComplete { v => - timeoutCancelable.cancel(true) - if (!promise.tryComplete(v)) { - logWarning(s"Ignore message $v") - } - }(ThreadUtils.sameThread) - promise.future.mapTo[T].recover(timeout.addMessageIfTimeout)(ThreadUtils.sameThread) + nettyEnv.ask(new RequestMessage(nettyEnv.address, this, message), timeout) } override def send(message: Any): Unit = { require(message != null, "Message is null") - nettyEnv.send(RequestMessage(nettyEnv.address, this, message, false)) + nettyEnv.send(new RequestMessage(nettyEnv.address, this, message)) } - override def toString: String = s"NettyRpcEndpointRef(${_address})" - - def toURI: URI = new URI(s"spark://${_address}") + override def toString: String = s"NettyRpcEndpointRef(${endpointAddress})" final override def equals(that: Any): Boolean = that match { - case other: NettyRpcEndpointRef => _address == other._address + case other: NettyRpcEndpointRef => endpointAddress == other.endpointAddress case _ => false } - final override def hashCode(): Int = if (_address == null) 0 else _address.hashCode() + final override def hashCode(): Int = + if (endpointAddress == null) 0 else endpointAddress.hashCode() } /** * The message that is sent from the sender to the receiver. + * + * @param senderAddress the sender address. It's `null` if this message is from a client + * `NettyRpcEnv`. + * @param receiver the receiver of this message. + * @param content the message content. */ -private[netty] case class RequestMessage( - senderAddress: RpcAddress, receiver: NettyRpcEndpointRef, content: Any, needReply: Boolean) +private[netty] class RequestMessage( + val senderAddress: RpcAddress, + val receiver: NettyRpcEndpointRef, + val content: Any) { + + /** Manually serialize [[RequestMessage]] to minimize the size. */ + def serialize(nettyEnv: NettyRpcEnv): ByteBuffer = { + val bos = new ByteBufferOutputStream() + val out = new DataOutputStream(bos) + try { + writeRpcAddress(out, senderAddress) + writeRpcAddress(out, receiver.address) + out.writeUTF(receiver.name) + val s = nettyEnv.serializeStream(out) + try { + s.writeObject(content) + } finally { + s.close() + } + } finally { + out.close() + } + bos.toByteBuffer + } -/** - * The base trait for all messages that are sent back from the receiver to the sender. - */ -private[netty] trait ResponseMessage + private def writeRpcAddress(out: DataOutputStream, rpcAddress: RpcAddress): Unit = { + if (rpcAddress == null) { + out.writeBoolean(false) + } else { + out.writeBoolean(true) + out.writeUTF(rpcAddress.host) + out.writeInt(rpcAddress.port) + } + } -/** - * The reply for `ask` from the receiver side. - */ -private[netty] case class AskResponse(sender: NettyRpcEndpointRef, reply: Any) - extends ResponseMessage + override def toString: String = s"RequestMessage($senderAddress, $receiver, $content)" +} -/** - * A message to send back to the receiver side. It's necessary because [[TransportClient]] only - * clean the resources when it receives a reply. - */ -private[netty] case class Ack(sender: NettyRpcEndpointRef) extends ResponseMessage +private[netty] object RequestMessage { + + private def readRpcAddress(in: DataInputStream): RpcAddress = { + val hasRpcAddress = in.readBoolean() + if (hasRpcAddress) { + RpcAddress(in.readUTF(), in.readInt()) + } else { + null + } + } + + def apply(nettyEnv: NettyRpcEnv, client: TransportClient, bytes: ByteBuffer): RequestMessage = { + val bis = new ByteBufferInputStream(bytes) + val in = new DataInputStream(bis) + try { + val senderAddress = readRpcAddress(in) + val endpointAddress = RpcEndpointAddress(readRpcAddress(in), in.readUTF()) + val ref = new NettyRpcEndpointRef(nettyEnv.conf, endpointAddress, nettyEnv) + ref.client = client + new RequestMessage( + senderAddress, + ref, + // The remaining bytes in `bytes` are the message content. + nettyEnv.deserialize(client, bytes)) + } finally { + in.close() + } + } +} /** * A response that indicates some failure happens in the receiver side. @@ -474,40 +626,60 @@ private[netty] case class RpcFailure(e: Throwable) * with different `RpcAddress` information). */ private[netty] class NettyRpcHandler( - dispatcher: Dispatcher, nettyEnv: NettyRpcEnv) extends RpcHandler with Logging { + dispatcher: Dispatcher, + nettyEnv: NettyRpcEnv, + streamManager: StreamManager) extends RpcHandler with Logging { - // TODO: Can we add connection callback (channel registered) to the underlying framework? - // A variable to track whether we should dispatch the RemoteProcessConnected message. - private val clients = new ConcurrentHashMap[TransportClient, JBoolean]() + // A variable to track the remote RpcEnv addresses of all clients + private val remoteAddresses = new ConcurrentHashMap[RpcAddress, RpcAddress]() override def receive( client: TransportClient, - message: Array[Byte], + message: ByteBuffer, callback: RpcResponseCallback): Unit = { + val messageToDispatch = internalReceive(client, message) + dispatcher.postRemoteMessage(messageToDispatch, callback) + } + + override def receive( + client: TransportClient, + message: ByteBuffer): Unit = { + val messageToDispatch = internalReceive(client, message) + dispatcher.postOneWayMessage(messageToDispatch) + } + + private def internalReceive(client: TransportClient, message: ByteBuffer): RequestMessage = { val addr = client.getChannel().remoteAddress().asInstanceOf[InetSocketAddress] assert(addr != null) - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) - if (clients.putIfAbsent(client, JBoolean.TRUE) == null) { - dispatcher.postToAll(RemoteProcessConnected(clientAddr)) - } - val requestMessage = nettyEnv.deserialize[RequestMessage](client, message) - val messageToDispatch = if (requestMessage.senderAddress == null) { - // Create a new message with the socket address of the client as the sender. - RequestMessage(clientAddr, requestMessage.receiver, requestMessage.content, - requestMessage.needReply) - } else { - requestMessage + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) + val requestMessage = RequestMessage(nettyEnv, client, message) + if (requestMessage.senderAddress == null) { + // Create a new message with the socket address of the client as the sender. + new RequestMessage(clientAddr, requestMessage.receiver, requestMessage.content) + } else { + // The remote RpcEnv listens to some port, we should also fire a RemoteProcessConnected for + // the listening address + val remoteEnvAddress = requestMessage.senderAddress + if (remoteAddresses.putIfAbsent(clientAddr, remoteEnvAddress) == null) { + dispatcher.postToAll(RemoteProcessConnected(remoteEnvAddress)) } - dispatcher.postRemoteMessage(messageToDispatch, callback) + requestMessage + } } - override def getStreamManager: StreamManager = new OneForOneStreamManager + override def getStreamManager: StreamManager = streamManager override def exceptionCaught(cause: Throwable, client: TransportClient): Unit = { val addr = client.getChannel.remoteAddress().asInstanceOf[InetSocketAddress] if (addr != null) { - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) dispatcher.postToAll(RemoteProcessConnectionError(cause, clientAddr)) + // If the remove RpcEnv listens to some address, we should also fire a + // RemoteProcessConnectionError for the remote RpcEnv listening address + val remoteEnvAddress = remoteAddresses.get(clientAddr) + if (remoteEnvAddress != null) { + dispatcher.postToAll(RemoteProcessConnectionError(cause, remoteEnvAddress)) + } } else { // If the channel is closed before connecting, its remoteAddress will be null. // See java.net.Socket.getRemoteSocketAddress @@ -516,13 +688,25 @@ private[netty] class NettyRpcHandler( } } - override def connectionTerminated(client: TransportClient): Unit = { + override def channelActive(client: TransportClient): Unit = { + val addr = client.getChannel().remoteAddress().asInstanceOf[InetSocketAddress] + assert(addr != null) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) + dispatcher.postToAll(RemoteProcessConnected(clientAddr)) + } + + override def channelInactive(client: TransportClient): Unit = { val addr = client.getChannel.remoteAddress().asInstanceOf[InetSocketAddress] if (addr != null) { - val clientAddr = RpcAddress(addr.getHostName, addr.getPort) - clients.remove(client) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) nettyEnv.removeOutbox(clientAddr) dispatcher.postToAll(RemoteProcessDisconnected(clientAddr)) + val remoteEnvAddress = remoteAddresses.remove(clientAddr) + // If the remove RpcEnv listens to some address, we should also fire a + // RemoteProcessDisconnected for the remote RpcEnv listening address + if (remoteEnvAddress != null) { + dispatcher.postToAll(RemoteProcessDisconnected(remoteEnvAddress)) + } } else { // If the channel is closed before connecting, its remoteAddress will be null. In this case, // we can ignore it since we don't fire "Associated". diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyStreamManager.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyStreamManager.scala new file mode 100644 index 000000000000..780fadd5bda8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyStreamManager.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.rpc.netty + +import java.io.File +import java.util.concurrent.ConcurrentHashMap + +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.server.StreamManager +import org.apache.spark.rpc.RpcEnvFileServer +import org.apache.spark.util.Utils + +/** + * StreamManager implementation for serving files from a NettyRpcEnv. + * + * Three kinds of resources can be registered in this manager, all backed by actual files: + * + * - "/files": a flat list of files; used as the backend for [[SparkContext.addFile]]. + * - "/jars": a flat list of files; used as the backend for [[SparkContext.addJar]]. + * - arbitrary directories; all files under the directory become available through the manager, + * respecting the directory's hierarchy. + * + * Only streaming (openStream) is supported. + */ +private[netty] class NettyStreamManager(rpcEnv: NettyRpcEnv) + extends StreamManager with RpcEnvFileServer { + + private val files = new ConcurrentHashMap[String, File]() + private val jars = new ConcurrentHashMap[String, File]() + private val dirs = new ConcurrentHashMap[String, File]() + + override def getChunk(streamId: Long, chunkIndex: Int): ManagedBuffer = { + throw new UnsupportedOperationException() + } + + override def openStream(streamId: String): ManagedBuffer = { + val Array(ftype, fname) = streamId.stripPrefix("/").split("/", 2) + val file = ftype match { + case "files" => files.get(fname) + case "jars" => jars.get(fname) + case other => + val dir = dirs.get(ftype) + require(dir != null, s"Invalid stream URI: $ftype not found.") + new File(dir, fname) + } + + if (file != null && file.isFile()) { + new FileSegmentManagedBuffer(rpcEnv.transportConf, file, 0, file.length()) + } else { + null + } + } + + override def addFile(file: File): String = { + val existingPath = files.putIfAbsent(file.getName, file) + require(existingPath == null || existingPath == file, + s"File ${file.getName} was already registered with a different path " + + s"(old path = $existingPath, new path = $file") + s"${rpcEnv.address.toSparkURL}/files/${Utils.encodeFileNameToURIRawPath(file.getName())}" + } + + override def addJar(file: File): String = { + val existingPath = jars.putIfAbsent(file.getName, file) + require(existingPath == null || existingPath == file, + s"File ${file.getName} was already registered with a different path " + + s"(old path = $existingPath, new path = $file") + s"${rpcEnv.address.toSparkURL}/jars/${Utils.encodeFileNameToURIRawPath(file.getName())}" + } + + override def addDirectory(baseUri: String, path: File): String = { + val fixedBaseUri = validateDirectoryUri(baseUri) + require(dirs.putIfAbsent(fixedBaseUri.stripPrefix("/"), path) == null, + s"URI '$fixedBaseUri' already registered.") + s"${rpcEnv.address.toSparkURL}$fixedBaseUri" + } + +} diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala index 2f6817f2eb93..b7e068aa6835 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala @@ -17,29 +17,71 @@ package org.apache.spark.rpc.netty +import java.nio.ByteBuffer import java.util.concurrent.Callable import javax.annotation.concurrent.GuardedBy import scala.util.control.NonFatal import org.apache.spark.SparkException +import org.apache.spark.internal.Logging import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} -import org.apache.spark.rpc.RpcAddress +import org.apache.spark.rpc.{RpcAddress, RpcEnvStoppedException} -private[netty] case class OutboxMessage(content: Array[Byte], - _onFailure: (Throwable) => Unit, - _onSuccess: (TransportClient, Array[Byte]) => Unit) { +private[netty] sealed trait OutboxMessage { - def createCallback(client: TransportClient): RpcResponseCallback = new RpcResponseCallback() { - override def onFailure(e: Throwable): Unit = { - _onFailure(e) + def sendWith(client: TransportClient): Unit + + def onFailure(e: Throwable): Unit + +} + +private[netty] case class OneWayOutboxMessage(content: ByteBuffer) extends OutboxMessage + with Logging { + + override def sendWith(client: TransportClient): Unit = { + client.send(content) + } + + override def onFailure(e: Throwable): Unit = { + e match { + case e1: RpcEnvStoppedException => logDebug(e1.getMessage) + case e1: Throwable => logWarning(s"Failed to send one-way RPC.", e1) } + } + +} + +private[netty] case class RpcOutboxMessage( + content: ByteBuffer, + _onFailure: (Throwable) => Unit, + _onSuccess: (TransportClient, ByteBuffer) => Unit) + extends OutboxMessage with RpcResponseCallback with Logging { - override def onSuccess(response: Array[Byte]): Unit = { - _onSuccess(client, response) + private var client: TransportClient = _ + private var requestId: Long = _ + + override def sendWith(client: TransportClient): Unit = { + this.client = client + this.requestId = client.sendRpc(content, this) + } + + def onTimeout(): Unit = { + if (client != null) { + client.removeRpcRequest(requestId) + } else { + logError("Ask timeout before connecting successfully") } } + override def onFailure(e: Throwable): Unit = { + _onFailure(e) + } + + override def onSuccess(response: ByteBuffer): Unit = { + _onSuccess(client, response) + } + } private[netty] class Outbox(nettyEnv: NettyRpcEnv, val address: RpcAddress) { @@ -82,7 +124,7 @@ private[netty] class Outbox(nettyEnv: NettyRpcEnv, val address: RpcAddress) { } } if (dropped) { - message._onFailure(new SparkException("Message is dropped because Outbox is stopped")) + message.onFailure(new SparkException("Message is dropped because Outbox is stopped")) } else { drainOutbox() } @@ -122,7 +164,7 @@ private[netty] class Outbox(nettyEnv: NettyRpcEnv, val address: RpcAddress) { try { val _client = synchronized { client } if (_client != null) { - _client.sendRpc(message.content, message.createCallback(_client)) + message.sendWith(_client) } else { assert(stopped == true) } @@ -195,17 +237,14 @@ private[netty] class Outbox(nettyEnv: NettyRpcEnv, val address: RpcAddress) { // update messages and it's safe to just drain the queue. var message = messages.poll() while (message != null) { - message._onFailure(e) + message.onFailure(e) message = messages.poll() } assert(messages.isEmpty) } private def closeClient(): Unit = synchronized { - // Not sure if `client.close` is idempotent. Just for safety. - if (client != null) { - client.close() - } + // Just set client to null. Don't close it in order to reuse the connection. client = null } @@ -229,7 +268,7 @@ private[netty] class Outbox(nettyEnv: NettyRpcEnv, val address: RpcAddress) { // update messages and it's safe to just drain the queue. var message = messages.poll() while (message != null) { - message._onFailure(new SparkException("Message is dropped because Outbox is stopped")) + message.onFailure(new SparkException("Message is dropped because Outbox is stopped")) message = messages.poll() } } diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/RpcEndpointVerifier.scala b/core/src/main/scala/org/apache/spark/rpc/netty/RpcEndpointVerifier.scala index 99f20da2d66a..430dcc50ba71 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/RpcEndpointVerifier.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/RpcEndpointVerifier.scala @@ -20,7 +20,7 @@ package org.apache.spark.rpc.netty import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEnv} /** - * An [[RpcEndpoint]] for remote [[RpcEnv]]s to query if an [[RpcEndpoint]] exists. + * An [[RpcEndpoint]] for remote [[RpcEnv]]s to query if an `RpcEndpoint` exists. * * This is used when setting up a remote endpoint reference. */ @@ -35,6 +35,6 @@ private[netty] class RpcEndpointVerifier(override val rpcEnv: RpcEnv, dispatcher private[netty] object RpcEndpointVerifier { val NAME = "endpoint-verifier" - /** A message used to ask the remote [[RpcEndpointVerifier]] if an [[RpcEndpoint]] exists. */ + /** A message used to ask the remote [[RpcEndpointVerifier]] if an `RpcEndpoint` exists. */ case class CheckExistence(name: String) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala index 146cfb9ba803..0a5fe5a1d3ee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala @@ -19,47 +19,61 @@ package org.apache.spark.scheduler import org.apache.spark.annotation.DeveloperApi + /** * :: DeveloperApi :: * Information about an [[org.apache.spark.Accumulable]] modified during a task or stage. + * + * @param id accumulator ID + * @param name accumulator name + * @param update partial value from a task, may be None if used on driver to describe a stage + * @param value total accumulated value so far, maybe None if used on executors to describe a task + * @param internal whether this accumulator was internal + * @param countFailedValues whether to count this accumulator's partial value if the task failed + * @param metadata internal metadata associated with this accumulator, if any + * + * @note Once this is JSON serialized the types of `update` and `value` will be lost and be + * cast to strings. This is because the user can define an accumulator of any type and it will + * be difficult to preserve the type in consumers of the event log. This does not apply to + * internal accumulators that represent task level metrics. */ @DeveloperApi -class AccumulableInfo private[spark] ( - val id: Long, - val name: String, - val update: Option[String], // represents a partial update within a task - val value: String, - val internal: Boolean) { - - override def equals(other: Any): Boolean = other match { - case acc: AccumulableInfo => - this.id == acc.id && this.name == acc.name && - this.update == acc.update && this.value == acc.value && - this.internal == acc.internal - case _ => false - } +case class AccumulableInfo private[spark] ( + id: Long, + name: Option[String], + update: Option[Any], // represents a partial update within a task + value: Option[Any], + private[spark] val internal: Boolean, + private[spark] val countFailedValues: Boolean, + // TODO: use this to identify internal task metrics instead of encoding it in the name + private[spark] val metadata: Option[String] = None) - override def hashCode(): Int = { - val state = Seq(id, name, update, value, internal) - state.map(_.hashCode).reduceLeft(31 * _ + _) - } -} +/** + * A collection of deprecated constructors. This will be removed soon. + */ object AccumulableInfo { + + @deprecated("do not create AccumulableInfo", "2.0.0") def apply( id: Long, name: String, update: Option[String], value: String, internal: Boolean): AccumulableInfo = { - new AccumulableInfo(id, name, update, value, internal) + new AccumulableInfo( + id, Option(name), update, Option(value), internal, countFailedValues = false) } + @deprecated("do not create AccumulableInfo", "2.0.0") def apply(id: Long, name: String, update: Option[String], value: String): AccumulableInfo = { - new AccumulableInfo(id, name, update, value, internal = false) + new AccumulableInfo( + id, Option(name), update, Option(value), internal = false, countFailedValues = false) } + @deprecated("do not create AccumulableInfo", "2.0.0") def apply(id: Long, name: String, value: String): AccumulableInfo = { - new AccumulableInfo(id, name, None, value, internal = false) + new AccumulableInfo( + id, Option(name), None, Option(value), internal = false, countFailedValues = false) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala index a3d2db31301b..949e88f60627 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala @@ -19,7 +19,6 @@ package org.apache.spark.scheduler import java.util.Properties -import org.apache.spark.TaskContext import org.apache.spark.util.CallSite /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index 9f218c64cac2..6da8865cd10d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -32,6 +32,9 @@ private[spark] class ApplicationEventListener extends SparkListener { var endTime: Option[Long] = None var viewAcls: Option[String] = None var adminAcls: Option[String] = None + var viewAclsGroups: Option[String] = None + var adminAclsGroups: Option[String] = None + var appSparkVersion: Option[String] = None override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { appName = Some(applicationStart.appName) @@ -51,6 +54,14 @@ private[spark] class ApplicationEventListener extends SparkListener { val allProperties = environmentDetails("Spark Properties").toMap viewAcls = allProperties.get("spark.ui.view.acls") adminAcls = allProperties.get("spark.admin.acls") + viewAclsGroups = allProperties.get("spark.ui.view.acls.groups") + adminAclsGroups = allProperties.get("spark.admin.acls.groups") } } + + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { + case SparkListenerLogStart(sparkVersion) => + appSparkVersion = Some(sparkVersion) + case _ => + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala new file mode 100644 index 000000000000..8605e1da161c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import java.util.concurrent.LinkedBlockingQueue +import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} + +import com.codahale.metrics.{Gauge, Timer} + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.util.Utils + +/** + * An asynchronous queue for events. All events posted to this queue will be delivered to the child + * listeners in a separate thread. + * + * Delivery will only begin when the `start()` method is called. The `stop()` method should be + * called when no more events need to be delivered. + */ +private class AsyncEventQueue(val name: String, conf: SparkConf, metrics: LiveListenerBusMetrics) + extends SparkListenerBus + with Logging { + + import AsyncEventQueue._ + + // Cap the capacity of the queue so we get an explicit error (rather than an OOM exception) if + // it's perpetually being added to more quickly than it's being drained. + private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent]( + conf.get(LISTENER_BUS_EVENT_QUEUE_CAPACITY)) + + // Keep the event count separately, so that waitUntilEmpty() can be implemented properly; + // this allows that method to return only when the events in the queue have been fully + // processed (instead of just dequeued). + private val eventCount = new AtomicLong() + + /** A counter for dropped events. It will be reset every time we log it. */ + private val droppedEventsCounter = new AtomicLong(0L) + + /** When `droppedEventsCounter` was logged last time in milliseconds. */ + @volatile private var lastReportTimestamp = 0L + + private val logDroppedEvent = new AtomicBoolean(false) + + private var sc: SparkContext = null + + private val started = new AtomicBoolean(false) + private val stopped = new AtomicBoolean(false) + + private val droppedEvents = metrics.metricRegistry.counter(s"queue.$name.numDroppedEvents") + private val processingTime = metrics.metricRegistry.timer(s"queue.$name.listenerProcessingTime") + + // Remove the queue size gauge first, in case it was created by a previous incarnation of + // this queue that was removed from the listener bus. + metrics.metricRegistry.remove(s"queue.$name.size") + metrics.metricRegistry.register(s"queue.$name.size", new Gauge[Int] { + override def getValue: Int = eventQueue.size() + }) + + private val dispatchThread = new Thread(s"spark-listener-group-$name") { + setDaemon(true) + override def run(): Unit = Utils.tryOrStopSparkContext(sc) { + dispatch() + } + } + + private def dispatch(): Unit = LiveListenerBus.withinListenerThread.withValue(true) { + try { + var next: SparkListenerEvent = eventQueue.take() + while (next != POISON_PILL) { + val ctx = processingTime.time() + try { + super.postToAll(next) + } finally { + ctx.stop() + } + eventCount.decrementAndGet() + next = eventQueue.take() + } + eventCount.decrementAndGet() + } catch { + case ie: InterruptedException => + logInfo(s"Stopping listener queue $name.", ie) + } + } + + override protected def getTimer(listener: SparkListenerInterface): Option[Timer] = { + metrics.getTimerForListenerClass(listener.getClass.asSubclass(classOf[SparkListenerInterface])) + } + + /** + * Start an asynchronous thread to dispatch events to the underlying listeners. + * + * @param sc Used to stop the SparkContext in case the async dispatcher fails. + */ + private[scheduler] def start(sc: SparkContext): Unit = { + if (started.compareAndSet(false, true)) { + this.sc = sc + dispatchThread.start() + } else { + throw new IllegalStateException(s"$name already started!") + } + } + + /** + * Stop the listener bus. It will wait until the queued events have been processed, but new + * events will be dropped. + */ + private[scheduler] def stop(): Unit = { + if (!started.get()) { + throw new IllegalStateException(s"Attempted to stop $name that has not yet started!") + } + if (stopped.compareAndSet(false, true)) { + eventQueue.put(POISON_PILL) + eventCount.incrementAndGet() + } + dispatchThread.join() + } + + def post(event: SparkListenerEvent): Unit = { + if (stopped.get()) { + return + } + + eventCount.incrementAndGet() + if (eventQueue.offer(event)) { + return + } + + eventCount.decrementAndGet() + droppedEvents.inc() + droppedEventsCounter.incrementAndGet() + if (logDroppedEvent.compareAndSet(false, true)) { + // Only log the following message once to avoid duplicated annoying logs. + logError(s"Dropping event from queue $name. " + + "This likely means one of the listeners is too slow and cannot keep up with " + + "the rate at which tasks are being started by the scheduler.") + } + logTrace(s"Dropping event $event") + + val droppedCount = droppedEventsCounter.get + if (droppedCount > 0) { + // Don't log too frequently + if (System.currentTimeMillis() - lastReportTimestamp >= 60 * 1000) { + // There may be multiple threads trying to decrease droppedEventsCounter. + // Use "compareAndSet" to make sure only one thread can win. + // And if another thread is increasing droppedEventsCounter, "compareAndSet" will fail and + // then that thread will update it. + if (droppedEventsCounter.compareAndSet(droppedCount, 0)) { + val prevLastReportTimestamp = lastReportTimestamp + lastReportTimestamp = System.currentTimeMillis() + val previous = new java.util.Date(prevLastReportTimestamp) + logWarning(s"Dropped $droppedEvents events from $name since $previous.") + } + } + } + } + + /** + * For testing only. Wait until there are no more events in the queue. + * + * @return true if the queue is empty. + */ + def waitUntilEmpty(deadline: Long): Boolean = { + while (eventCount.get() != 0) { + if (System.currentTimeMillis > deadline) { + return false + } + Thread.sleep(10) + } + true + } + +} + +private object AsyncEventQueue { + + val POISON_PILL = new SparkListenerEvent() { } + +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala new file mode 100644 index 000000000000..cd8e61d6d020 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -0,0 +1,466 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.concurrent.atomic.AtomicReference + +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} + +import org.apache.spark.{ExecutorAllocationClient, SparkConf, SparkContext} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config +import org.apache.spark.util.{Clock, SystemClock, Utils} + +/** + * BlacklistTracker is designed to track problematic executors and nodes. It supports blacklisting + * executors and nodes across an entire application (with a periodic expiry). TaskSetManagers add + * additional blacklisting of executors and nodes for individual tasks and stages which works in + * concert with the blacklisting here. + * + * The tracker needs to deal with a variety of workloads, eg.: + * + * * bad user code -- this may lead to many task failures, but that should not count against + * individual executors + * * many small stages -- this may prevent a bad executor for having many failures within one + * stage, but still many failures over the entire application + * * "flaky" executors -- they don't fail every task, but are still faulty enough to merit + * blacklisting + * + * See the design doc on SPARK-8425 for a more in-depth discussion. + * + * THREADING: As with most helpers of TaskSchedulerImpl, this is not thread-safe. Though it is + * called by multiple threads, callers must already have a lock on the TaskSchedulerImpl. The + * one exception is [[nodeBlacklist()]], which can be called without holding a lock. + */ +private[scheduler] class BlacklistTracker ( + private val listenerBus: LiveListenerBus, + conf: SparkConf, + allocationClient: Option[ExecutorAllocationClient], + clock: Clock = new SystemClock()) extends Logging { + + def this(sc: SparkContext, allocationClient: Option[ExecutorAllocationClient]) = { + this(sc.listenerBus, sc.conf, allocationClient) + } + + BlacklistTracker.validateBlacklistConfs(conf) + private val MAX_FAILURES_PER_EXEC = conf.get(config.MAX_FAILURES_PER_EXEC) + private val MAX_FAILED_EXEC_PER_NODE = conf.get(config.MAX_FAILED_EXEC_PER_NODE) + val BLACKLIST_TIMEOUT_MILLIS = BlacklistTracker.getBlacklistTimeout(conf) + private val BLACKLIST_FETCH_FAILURE_ENABLED = conf.get(config.BLACKLIST_FETCH_FAILURE_ENABLED) + + /** + * A map from executorId to information on task failures. Tracks the time of each task failure, + * so that we can avoid blacklisting executors due to failures that are very far apart. We do not + * actively remove from this as soon as tasks hit their timeouts, to avoid the time it would take + * to do so. But it will not grow too large, because as soon as an executor gets too many + * failures, we blacklist the executor and remove its entry here. + */ + private val executorIdToFailureList = new HashMap[String, ExecutorFailureList]() + val executorIdToBlacklistStatus = new HashMap[String, BlacklistedExecutor]() + val nodeIdToBlacklistExpiryTime = new HashMap[String, Long]() + /** + * An immutable copy of the set of nodes that are currently blacklisted. Kept in an + * AtomicReference to make [[nodeBlacklist()]] thread-safe. + */ + private val _nodeBlacklist = new AtomicReference[Set[String]](Set()) + /** + * Time when the next blacklist will expire. Used as a + * shortcut to avoid iterating over all entries in the blacklist when none will have expired. + */ + var nextExpiryTime: Long = Long.MaxValue + /** + * Mapping from nodes to all of the executors that have been blacklisted on that node. We do *not* + * remove from this when executors are removed from spark, so we can track when we get multiple + * successive blacklisted executors on one node. Nonetheless, it will not grow too large because + * there cannot be many blacklisted executors on one node, before we stop requesting more + * executors on that node, and we clean up the list of blacklisted executors once an executor has + * been blacklisted for BLACKLIST_TIMEOUT_MILLIS. + */ + val nodeToBlacklistedExecs = new HashMap[String, HashSet[String]]() + + /** + * Un-blacklists executors and nodes that have been blacklisted for at least + * BLACKLIST_TIMEOUT_MILLIS + */ + def applyBlacklistTimeout(): Unit = { + val now = clock.getTimeMillis() + // quickly check if we've got anything to expire from blacklist -- if not, avoid doing any work + if (now > nextExpiryTime) { + // Apply the timeout to blacklisted nodes and executors + val execsToUnblacklist = executorIdToBlacklistStatus.filter(_._2.expiryTime < now).keys + if (execsToUnblacklist.nonEmpty) { + // Un-blacklist any executors that have been blacklisted longer than the blacklist timeout. + logInfo(s"Removing executors $execsToUnblacklist from blacklist because the blacklist " + + s"for those executors has timed out") + execsToUnblacklist.foreach { exec => + val status = executorIdToBlacklistStatus.remove(exec).get + val failedExecsOnNode = nodeToBlacklistedExecs(status.node) + listenerBus.post(SparkListenerExecutorUnblacklisted(now, exec)) + failedExecsOnNode.remove(exec) + if (failedExecsOnNode.isEmpty) { + nodeToBlacklistedExecs.remove(status.node) + } + } + } + val nodesToUnblacklist = nodeIdToBlacklistExpiryTime.filter(_._2 < now).keys + if (nodesToUnblacklist.nonEmpty) { + // Un-blacklist any nodes that have been blacklisted longer than the blacklist timeout. + logInfo(s"Removing nodes $nodesToUnblacklist from blacklist because the blacklist " + + s"has timed out") + nodesToUnblacklist.foreach { node => + nodeIdToBlacklistExpiryTime.remove(node) + listenerBus.post(SparkListenerNodeUnblacklisted(now, node)) + } + _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) + } + updateNextExpiryTime() + } + } + + private def updateNextExpiryTime(): Unit = { + val execMinExpiry = if (executorIdToBlacklistStatus.nonEmpty) { + executorIdToBlacklistStatus.map{_._2.expiryTime}.min + } else { + Long.MaxValue + } + val nodeMinExpiry = if (nodeIdToBlacklistExpiryTime.nonEmpty) { + nodeIdToBlacklistExpiryTime.values.min + } else { + Long.MaxValue + } + nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry) + } + + private def killBlacklistedExecutor(exec: String): Unit = { + if (conf.get(config.BLACKLIST_KILL_ENABLED)) { + allocationClient match { + case Some(a) => + logInfo(s"Killing blacklisted executor id $exec " + + s"since ${config.BLACKLIST_KILL_ENABLED.key} is set.") + a.killExecutors(Seq(exec), true, true) + case None => + logWarning(s"Not attempting to kill blacklisted executor id $exec " + + s"since allocation client is not defined.") + } + } + } + + private def killExecutorsOnBlacklistedNode(node: String): Unit = { + if (conf.get(config.BLACKLIST_KILL_ENABLED)) { + allocationClient match { + case Some(a) => + logInfo(s"Killing all executors on blacklisted host $node " + + s"since ${config.BLACKLIST_KILL_ENABLED.key} is set.") + if (a.killExecutorsOnHost(node) == false) { + logError(s"Killing executors on node $node failed.") + } + case None => + logWarning(s"Not attempting to kill executors on blacklisted host $node " + + s"since allocation client is not defined.") + } + } + } + + def updateBlacklistForFetchFailure(host: String, exec: String): Unit = { + if (BLACKLIST_FETCH_FAILURE_ENABLED) { + // If we blacklist on fetch failures, we are implicitly saying that we believe the failure is + // non-transient, and can't be recovered from (even if this is the first fetch failure, + // stage is retried after just one failure, so we don't always get a chance to collect + // multiple fetch failures). + // If the external shuffle-service is on, then every other executor on this node would + // be suffering from the same issue, so we should blacklist (and potentially kill) all + // of them immediately. + + val now = clock.getTimeMillis() + val expiryTimeForNewBlacklists = now + BLACKLIST_TIMEOUT_MILLIS + + if (conf.get(config.SHUFFLE_SERVICE_ENABLED)) { + if (!nodeIdToBlacklistExpiryTime.contains(host)) { + logInfo(s"blacklisting node $host due to fetch failure of external shuffle service") + + nodeIdToBlacklistExpiryTime.put(host, expiryTimeForNewBlacklists) + listenerBus.post(SparkListenerNodeBlacklisted(now, host, 1)) + _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) + killExecutorsOnBlacklistedNode(host) + updateNextExpiryTime() + } + } else if (!executorIdToBlacklistStatus.contains(exec)) { + logInfo(s"Blacklisting executor $exec due to fetch failure") + + executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(host, expiryTimeForNewBlacklists)) + // We hardcoded number of failure tasks to 1 for fetch failure, because there's no + // reattempt for such failure. + listenerBus.post(SparkListenerExecutorBlacklisted(now, exec, 1)) + updateNextExpiryTime() + killBlacklistedExecutor(exec) + + val blacklistedExecsOnNode = nodeToBlacklistedExecs.getOrElseUpdate(exec, HashSet[String]()) + blacklistedExecsOnNode += exec + } + } + } + + def updateBlacklistForSuccessfulTaskSet( + stageId: Int, + stageAttemptId: Int, + failuresByExec: HashMap[String, ExecutorFailuresInTaskSet]): Unit = { + // if any tasks failed, we count them towards the overall failure count for the executor at + // this point. + val now = clock.getTimeMillis() + failuresByExec.foreach { case (exec, failuresInTaskSet) => + val appFailuresOnExecutor = + executorIdToFailureList.getOrElseUpdate(exec, new ExecutorFailureList) + appFailuresOnExecutor.addFailures(stageId, stageAttemptId, failuresInTaskSet) + appFailuresOnExecutor.dropFailuresWithTimeoutBefore(now) + val newTotal = appFailuresOnExecutor.numUniqueTaskFailures + + val expiryTimeForNewBlacklists = now + BLACKLIST_TIMEOUT_MILLIS + // If this pushes the total number of failures over the threshold, blacklist the executor. + // If its already blacklisted, we avoid "re-blacklisting" (which can happen if there were + // other tasks already running in another taskset when it got blacklisted), because it makes + // some of the logic around expiry times a little more confusing. But it also wouldn't be a + // problem to re-blacklist, with a later expiry time. + if (newTotal >= MAX_FAILURES_PER_EXEC && !executorIdToBlacklistStatus.contains(exec)) { + logInfo(s"Blacklisting executor id: $exec because it has $newTotal" + + s" task failures in successful task sets") + val node = failuresInTaskSet.node + executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTimeForNewBlacklists)) + listenerBus.post(SparkListenerExecutorBlacklisted(now, exec, newTotal)) + executorIdToFailureList.remove(exec) + updateNextExpiryTime() + killBlacklistedExecutor(exec) + + // In addition to blacklisting the executor, we also update the data for failures on the + // node, and potentially put the entire node into a blacklist as well. + val blacklistedExecsOnNode = nodeToBlacklistedExecs.getOrElseUpdate(node, HashSet[String]()) + blacklistedExecsOnNode += exec + // If the node is already in the blacklist, we avoid adding it again with a later expiry + // time. + if (blacklistedExecsOnNode.size >= MAX_FAILED_EXEC_PER_NODE && + !nodeIdToBlacklistExpiryTime.contains(node)) { + logInfo(s"Blacklisting node $node because it has ${blacklistedExecsOnNode.size} " + + s"executors blacklisted: ${blacklistedExecsOnNode}") + nodeIdToBlacklistExpiryTime.put(node, expiryTimeForNewBlacklists) + listenerBus.post(SparkListenerNodeBlacklisted(now, node, blacklistedExecsOnNode.size)) + _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) + killExecutorsOnBlacklistedNode(node) + } + } + } + } + + def isExecutorBlacklisted(executorId: String): Boolean = { + executorIdToBlacklistStatus.contains(executorId) + } + + /** + * Get the full set of nodes that are blacklisted. Unlike other methods in this class, this *IS* + * thread-safe -- no lock required on a taskScheduler. + */ + def nodeBlacklist(): Set[String] = { + _nodeBlacklist.get() + } + + def isNodeBlacklisted(node: String): Boolean = { + nodeIdToBlacklistExpiryTime.contains(node) + } + + def handleRemovedExecutor(executorId: String): Unit = { + // We intentionally do not clean up executors that are already blacklisted in + // nodeToBlacklistedExecs, so that if another executor on the same node gets blacklisted, we can + // blacklist the entire node. We also can't clean up executorIdToBlacklistStatus, so we can + // eventually remove the executor after the timeout. Despite not clearing those structures + // here, we don't expect they will grow too big since you won't get too many executors on one + // node, and the timeout will clear it up periodically in any case. + executorIdToFailureList -= executorId + } + + + /** + * Tracks all failures for one executor (that have not passed the timeout). + * + * In general we actually expect this to be extremely small, since it won't contain more than the + * maximum number of task failures before an executor is failed (default 2). + */ + private[scheduler] final class ExecutorFailureList extends Logging { + + private case class TaskId(stage: Int, stageAttempt: Int, taskIndex: Int) + + /** + * All failures on this executor in successful task sets. + */ + private var failuresAndExpiryTimes = ArrayBuffer[(TaskId, Long)]() + /** + * As an optimization, we track the min expiry time over all entries in failuresAndExpiryTimes + * so its quick to tell if there are any failures with expiry before the current time. + */ + private var minExpiryTime = Long.MaxValue + + def addFailures( + stage: Int, + stageAttempt: Int, + failuresInTaskSet: ExecutorFailuresInTaskSet): Unit = { + failuresInTaskSet.taskToFailureCountAndFailureTime.foreach { + case (taskIdx, (_, failureTime)) => + val expiryTime = failureTime + BLACKLIST_TIMEOUT_MILLIS + failuresAndExpiryTimes += ((TaskId(stage, stageAttempt, taskIdx), expiryTime)) + if (expiryTime < minExpiryTime) { + minExpiryTime = expiryTime + } + } + } + + /** + * The number of unique tasks that failed on this executor. Only counts failures within the + * timeout, and in successful tasksets. + */ + def numUniqueTaskFailures: Int = failuresAndExpiryTimes.size + + def isEmpty: Boolean = failuresAndExpiryTimes.isEmpty + + /** + * Apply the timeout to individual tasks. This is to prevent one-off failures that are very + * spread out in time (and likely have nothing to do with problems on the executor) from + * triggering blacklisting. However, note that we do *not* remove executors and nodes from + * the blacklist as we expire individual task failures -- each have their own timeout. Eg., + * suppose: + * * timeout = 10, maxFailuresPerExec = 2 + * * Task 1 fails on exec 1 at time 0 + * * Task 2 fails on exec 1 at time 5 + * --> exec 1 is blacklisted from time 5 - 15. + * This is to simplify the implementation, as well as keep the behavior easier to understand + * for the end user. + */ + def dropFailuresWithTimeoutBefore(dropBefore: Long): Unit = { + if (minExpiryTime < dropBefore) { + var newMinExpiry = Long.MaxValue + val newFailures = new ArrayBuffer[(TaskId, Long)] + failuresAndExpiryTimes.foreach { case (task, expiryTime) => + if (expiryTime >= dropBefore) { + newFailures += ((task, expiryTime)) + if (expiryTime < newMinExpiry) { + newMinExpiry = expiryTime + } + } + } + failuresAndExpiryTimes = newFailures + minExpiryTime = newMinExpiry + } + } + + override def toString(): String = { + s"failures = $failuresAndExpiryTimes" + } + } + +} + +private[scheduler] object BlacklistTracker extends Logging { + + private val DEFAULT_TIMEOUT = "1h" + + /** + * Returns true if the blacklist is enabled, based on checking the configuration in the following + * order: + * 1. Is it specifically enabled or disabled? + * 2. Is it enabled via the legacy timeout conf? + * 3. Default is off + */ + def isBlacklistEnabled(conf: SparkConf): Boolean = { + conf.get(config.BLACKLIST_ENABLED) match { + case Some(enabled) => + enabled + case None => + // if they've got a non-zero setting for the legacy conf, always enable the blacklist, + // otherwise, use the default. + val legacyKey = config.BLACKLIST_LEGACY_TIMEOUT_CONF.key + conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).exists { legacyTimeout => + if (legacyTimeout == 0) { + logWarning(s"Turning off blacklisting due to legacy configuration: $legacyKey == 0") + false + } else { + logWarning(s"Turning on blacklisting due to legacy configuration: $legacyKey > 0") + true + } + } + } + } + + def getBlacklistTimeout(conf: SparkConf): Long = { + conf.get(config.BLACKLIST_TIMEOUT_CONF).getOrElse { + conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).getOrElse { + Utils.timeStringAsMs(DEFAULT_TIMEOUT) + } + } + } + + /** + * Verify that blacklist configurations are consistent; if not, throw an exception. Should only + * be called if blacklisting is enabled. + * + * The configuration for the blacklist is expected to adhere to a few invariants. Default + * values follow these rules of course, but users may unwittingly change one configuration + * without making the corresponding adjustment elsewhere. This ensures we fail-fast when + * there are such misconfigurations. + */ + def validateBlacklistConfs(conf: SparkConf): Unit = { + + def mustBePos(k: String, v: String): Unit = { + throw new IllegalArgumentException(s"$k was $v, but must be > 0.") + } + + Seq( + config.MAX_TASK_ATTEMPTS_PER_EXECUTOR, + config.MAX_TASK_ATTEMPTS_PER_NODE, + config.MAX_FAILURES_PER_EXEC_STAGE, + config.MAX_FAILED_EXEC_PER_NODE_STAGE, + config.MAX_FAILURES_PER_EXEC, + config.MAX_FAILED_EXEC_PER_NODE + ).foreach { config => + val v = conf.get(config) + if (v <= 0) { + mustBePos(config.key, v.toString) + } + } + + val timeout = getBlacklistTimeout(conf) + if (timeout <= 0) { + // first, figure out where the timeout came from, to include the right conf in the message. + conf.get(config.BLACKLIST_TIMEOUT_CONF) match { + case Some(t) => + mustBePos(config.BLACKLIST_TIMEOUT_CONF.key, timeout.toString) + case None => + mustBePos(config.BLACKLIST_LEGACY_TIMEOUT_CONF.key, timeout.toString) + } + } + + val maxTaskFailures = conf.get(config.MAX_TASK_FAILURES) + val maxNodeAttempts = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) + + if (maxNodeAttempts >= maxTaskFailures) { + throw new IllegalArgumentException(s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + + s"( = ${maxNodeAttempts}) was >= ${config.MAX_TASK_FAILURES.key} " + + s"( = ${maxTaskFailures} ). Though blacklisting is enabled, with this configuration, " + + s"Spark will not be robust to one bad node. Decrease " + + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase ${config.MAX_TASK_FAILURES.key}, " + + s"or disable blacklisting with ${config.BLACKLIST_ENABLED.key}") + } + } +} + +private final case class BlacklistedExecutor(node: String, expiryTime: Long) 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 a1f0fd05f661..9153751d03c1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -22,8 +22,9 @@ import java.util.Properties import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicInteger +import scala.annotation.tailrec import scala.collection.Map -import scala.collection.mutable.{HashMap, HashSet, Stack} +import scala.collection.mutable.{ArrayStack, HashMap, HashSet} import scala.concurrent.duration._ import scala.language.existentials import scala.language.postfixOps @@ -34,12 +35,15 @@ import org.apache.commons.lang3.SerializationUtils import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config +import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.rpc.RpcTimeout import org.apache.spark.storage._ -import org.apache.spark.util._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat +import org.apache.spark.util._ /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of @@ -55,7 +59,7 @@ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat * set of map output files, and another to read those files after a barrier). In the end, every * stage will have only shuffle dependencies on other stages, and may compute multiple operations * inside it. The actual pipelining of these operations happens in the RDD.compute() functions of - * various RDDs (MappedRDD, FilteredRDD, etc). + * various RDDs * * In addition to coming up with a DAG of stages, the DAGScheduler also determines the preferred * locations to run each task on, based on the current cache status, and passes these to the @@ -130,7 +134,7 @@ class DAGScheduler( def this(sc: SparkContext) = this(sc, sc.taskScheduler) - private[scheduler] val metricsSource: DAGSchedulerSource = new DAGSchedulerSource(this) + private[spark] val metricsSource: DAGSchedulerSource = new DAGSchedulerSource(this) private[scheduler] val nextJobId = new AtomicInteger(0) private[scheduler] def numTotalJobs: Int = nextJobId.get() @@ -138,7 +142,13 @@ class DAGScheduler( private[scheduler] val jobIdToStageIds = new HashMap[Int, HashSet[Int]] private[scheduler] val stageIdToStage = new HashMap[Int, Stage] - private[scheduler] val shuffleToMapStage = new HashMap[Int, ShuffleMapStage] + /** + * Mapping from shuffle dependency ID to the ShuffleMapStage that will generate the data for + * that dependency. Only includes stages that are part of currently running job (when the job(s) + * that require the shuffle stage complete, the mapping will be removed, and the only record of + * the shuffle data will be in the MapOutputTracker). + */ + private[scheduler] val shuffleIdToMapStage = new HashMap[Int, ShuffleMapStage] private[scheduler] val jobIdToActiveJob = new HashMap[Int, ActiveJob] // Stages we need to run whose parents aren't done @@ -178,6 +188,21 @@ class DAGScheduler( /** If enabled, FetchFailed will not cause stage retry, in order to surface the problem. */ private val disallowStageRetryForTest = sc.getConf.getBoolean("spark.test.noStageRetry", false) + /** + * Whether to unregister all the outputs on the host in condition that we receive a FetchFailure, + * this is set default to false, which means, we only unregister the outputs related to the exact + * executor(instead of the host) on a FetchFailure. + */ + private[scheduler] val unRegisterOutputOnHostOnFetchFailure = + sc.getConf.get(config.UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE) + + /** + * Number of consecutive stage attempts allowed before a stage is aborted. + */ + private[scheduler] val maxConsecutiveStageAttempts = + sc.getConf.getInt("spark.stage.maxConsecutiveAttempts", + DAGScheduler.DEFAULT_MAX_CONSECUTIVE_STAGE_ATTEMPTS) + private val messageScheduler = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dag-scheduler-message") @@ -206,11 +231,10 @@ class DAGScheduler( task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Map[Long, Any], - taskInfo: TaskInfo, - taskMetrics: TaskMetrics): Unit = { + accumUpdates: Seq[AccumulatorV2[_, _]], + taskInfo: TaskInfo): Unit = { eventProcessLoop.post( - CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics)) + CompletionEvent(task, reason, result, accumUpdates, taskInfo)) } /** @@ -220,18 +244,26 @@ class DAGScheduler( */ def executorHeartbeatReceived( execId: String, - taskMetrics: Array[(Long, Int, Int, TaskMetrics)], // (taskId, stageId, stateAttempt, metrics) + // (taskId, stageId, stageAttemptId, accumUpdates) + accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])], blockManagerId: BlockManagerId): Boolean = { - listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, taskMetrics)) - blockManagerMaster.driverEndpoint.askWithRetry[Boolean]( + listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates)) + blockManagerMaster.driverEndpoint.askSync[Boolean]( BlockManagerHeartbeat(blockManagerId), new RpcTimeout(600 seconds, "BlockManagerHeartbeat")) } /** * Called by TaskScheduler implementation when an executor fails. */ - def executorLost(execId: String): Unit = { - eventProcessLoop.post(ExecutorLost(execId)) + def executorLost(execId: String, reason: ExecutorLossReason): Unit = { + eventProcessLoop.post(ExecutorLost(execId, reason)) + } + + /** + * Called by TaskScheduler implementation when a worker is removed. + */ + def workerRemoved(workerId: String, host: String, message: String): Unit = { + eventProcessLoop.post(WorkerRemoved(workerId, host, message)) } /** @@ -249,6 +281,13 @@ class DAGScheduler( eventProcessLoop.post(TaskSetFailed(taskSet, reason, exception)) } + /** + * Called by the TaskSetManager when it decides a speculative task is needed. + */ + def speculativeTaskSubmitted(task: Task[_]): Unit = { + eventProcessLoop.post(SpeculativeTaskSubmitted(task)) + } + private[scheduler] def getCacheLocs(rdd: RDD[_]): IndexedSeq[Seq[TaskLocation]] = cacheLocs.synchronized { // Note: this doesn't use `getOrElse()` because this method is called O(num tasks) times @@ -273,159 +312,141 @@ class DAGScheduler( } /** - * Get or create a shuffle map stage for the given shuffle dependency's map side. + * Gets a shuffle map stage if one exists in shuffleIdToMapStage. Otherwise, if the + * shuffle map stage doesn't already exist, this method will create the shuffle map stage in + * addition to any missing ancestor shuffle map stages. */ - private def getShuffleMapStage( + private def getOrCreateShuffleMapStage( shuffleDep: ShuffleDependency[_, _, _], firstJobId: Int): ShuffleMapStage = { - shuffleToMapStage.get(shuffleDep.shuffleId) match { - case Some(stage) => stage + shuffleIdToMapStage.get(shuffleDep.shuffleId) match { + case Some(stage) => + stage + case None => - // We are going to register ancestor shuffle dependencies - getAncestorShuffleDependencies(shuffleDep.rdd).foreach { dep => - shuffleToMapStage(dep.shuffleId) = newOrUsedShuffleStage(dep, firstJobId) + // Create stages for all missing ancestor shuffle dependencies. + getMissingAncestorShuffleDependencies(shuffleDep.rdd).foreach { dep => + // Even though getMissingAncestorShuffleDependencies only returns shuffle dependencies + // that were not already in shuffleIdToMapStage, it's possible that by the time we + // get to a particular dependency in the foreach loop, it's been added to + // shuffleIdToMapStage by the stage creation process for an earlier dependency. See + // SPARK-13902 for more information. + if (!shuffleIdToMapStage.contains(dep.shuffleId)) { + createShuffleMapStage(dep, firstJobId) + } } - // Then register current shuffleDep - val stage = newOrUsedShuffleStage(shuffleDep, firstJobId) - shuffleToMapStage(shuffleDep.shuffleId) = stage - stage + // Finally, create a stage for the given shuffle dependency. + createShuffleMapStage(shuffleDep, firstJobId) } } /** - * Helper function to eliminate some code re-use when creating new stages. + * Creates a ShuffleMapStage that generates the given shuffle dependency's partitions. If a + * previously run stage generated the same shuffle data, this function will copy the output + * locations that are still available from the previous shuffle to avoid unnecessarily + * regenerating data. */ - private def getParentStagesAndId(rdd: RDD[_], firstJobId: Int): (List[Stage], Int) = { - val parentStages = getParentStages(rdd, firstJobId) + def createShuffleMapStage(shuffleDep: ShuffleDependency[_, _, _], jobId: Int): ShuffleMapStage = { + val rdd = shuffleDep.rdd + val numTasks = rdd.partitions.length + val parents = getOrCreateParentStages(rdd, jobId) val id = nextStageId.getAndIncrement() - (parentStages, id) - } - - /** - * Create a ShuffleMapStage as part of the (re)-creation of a shuffle map stage in - * newOrUsedShuffleStage. The stage will be associated with the provided firstJobId. - * Production of shuffle map stages should always use newOrUsedShuffleStage, not - * newShuffleMapStage directly. - */ - private def newShuffleMapStage( - rdd: RDD[_], - numTasks: Int, - shuffleDep: ShuffleDependency[_, _, _], - firstJobId: Int, - callSite: CallSite): ShuffleMapStage = { - val (parentStages: List[Stage], id: Int) = getParentStagesAndId(rdd, firstJobId) - val stage: ShuffleMapStage = new ShuffleMapStage(id, rdd, numTasks, parentStages, - firstJobId, callSite, shuffleDep) + val stage = new ShuffleMapStage( + id, rdd, numTasks, parents, jobId, rdd.creationSite, shuffleDep, mapOutputTracker) stageIdToStage(id) = stage - updateJobIdStageIdMaps(firstJobId, stage) + shuffleIdToMapStage(shuffleDep.shuffleId) = stage + updateJobIdStageIdMaps(jobId, stage) + + if (!mapOutputTracker.containsShuffle(shuffleDep.shuffleId)) { + // Kind of ugly: need to register RDDs with the cache and map output tracker here + // since we can't do it in the RDD constructor because # of partitions is unknown + logInfo("Registering RDD " + rdd.id + " (" + rdd.getCreationSite + ")") + mapOutputTracker.registerShuffle(shuffleDep.shuffleId, rdd.partitions.length) + } stage } /** * Create a ResultStage associated with the provided jobId. */ - private def newResultStage( + private def createResultStage( rdd: RDD[_], func: (TaskContext, Iterator[_]) => _, partitions: Array[Int], jobId: Int, callSite: CallSite): ResultStage = { - val (parentStages: List[Stage], id: Int) = getParentStagesAndId(rdd, jobId) - val stage = new ResultStage(id, rdd, func, partitions, parentStages, jobId, callSite) + val parents = getOrCreateParentStages(rdd, jobId) + val id = nextStageId.getAndIncrement() + val stage = new ResultStage(id, rdd, func, partitions, parents, jobId, callSite) stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) stage } - /** - * Create a shuffle map Stage for the given RDD. The stage will also be associated with the - * provided firstJobId. If a stage for the shuffleId existed previously so that the shuffleId is - * present in the MapOutputTracker, then the number and location of available outputs are - * recovered from the MapOutputTracker - */ - private def newOrUsedShuffleStage( - shuffleDep: ShuffleDependency[_, _, _], - firstJobId: Int): ShuffleMapStage = { - val rdd = shuffleDep.rdd - val numTasks = rdd.partitions.length - val stage = newShuffleMapStage(rdd, numTasks, shuffleDep, firstJobId, rdd.creationSite) - if (mapOutputTracker.containsShuffle(shuffleDep.shuffleId)) { - val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId) - val locs = MapOutputTracker.deserializeMapStatuses(serLocs) - (0 until locs.length).foreach { i => - if (locs(i) ne null) { - // locs(i) will be null if missing - stage.addOutputLoc(i, locs(i)) - } - } - } else { - // Kind of ugly: need to register RDDs with the cache and map output tracker here - // since we can't do it in the RDD constructor because # of partitions is unknown - logInfo("Registering RDD " + rdd.id + " (" + rdd.getCreationSite + ")") - mapOutputTracker.registerShuffle(shuffleDep.shuffleId, rdd.partitions.length) - } - stage - } - /** * Get or create the list of parent stages for a given RDD. The new Stages will be created with * the provided firstJobId. */ - private def getParentStages(rdd: RDD[_], firstJobId: Int): List[Stage] = { - val parents = new HashSet[Stage] - val visited = new HashSet[RDD[_]] - // We are manually maintaining a stack here to prevent StackOverflowError - // caused by recursively visiting - val waitingForVisit = new Stack[RDD[_]] - def visit(r: RDD[_]) { - if (!visited(r)) { - visited += r - // Kind of ugly: need to register RDDs with the cache here since - // we can't do it in its constructor because # of partitions is unknown - for (dep <- r.dependencies) { - dep match { - case shufDep: ShuffleDependency[_, _, _] => - parents += getShuffleMapStage(shufDep, firstJobId) - case _ => - waitingForVisit.push(dep.rdd) - } - } - } - } - waitingForVisit.push(rdd) - while (waitingForVisit.nonEmpty) { - visit(waitingForVisit.pop()) - } - parents.toList + private def getOrCreateParentStages(rdd: RDD[_], firstJobId: Int): List[Stage] = { + getShuffleDependencies(rdd).map { shuffleDep => + getOrCreateShuffleMapStage(shuffleDep, firstJobId) + }.toList } /** Find ancestor shuffle dependencies that are not registered in shuffleToMapStage yet */ - private def getAncestorShuffleDependencies(rdd: RDD[_]): Stack[ShuffleDependency[_, _, _]] = { - val parents = new Stack[ShuffleDependency[_, _, _]] + private def getMissingAncestorShuffleDependencies( + rdd: RDD[_]): ArrayStack[ShuffleDependency[_, _, _]] = { + val ancestors = new ArrayStack[ShuffleDependency[_, _, _]] val visited = new HashSet[RDD[_]] // We are manually maintaining a stack here to prevent StackOverflowError // caused by recursively visiting - val waitingForVisit = new Stack[RDD[_]] - def visit(r: RDD[_]) { - if (!visited(r)) { - visited += r - for (dep <- r.dependencies) { - dep match { - case shufDep: ShuffleDependency[_, _, _] => - if (!shuffleToMapStage.contains(shufDep.shuffleId)) { - parents.push(shufDep) - } - case _ => - } - waitingForVisit.push(dep.rdd) + val waitingForVisit = new ArrayStack[RDD[_]] + waitingForVisit.push(rdd) + while (waitingForVisit.nonEmpty) { + val toVisit = waitingForVisit.pop() + if (!visited(toVisit)) { + visited += toVisit + getShuffleDependencies(toVisit).foreach { shuffleDep => + if (!shuffleIdToMapStage.contains(shuffleDep.shuffleId)) { + ancestors.push(shuffleDep) + waitingForVisit.push(shuffleDep.rdd) + } // Otherwise, the dependency and its ancestors have already been registered. } } } + ancestors + } + /** + * Returns shuffle dependencies that are immediate parents of the given RDD. + * + * This function will not return more distant ancestors. For example, if C has a shuffle + * dependency on B which has a shuffle dependency on A: + * + * A <-- B <-- C + * + * calling this function with rdd C will only return the B <-- C dependency. + * + * This function is scheduler-visible for the purpose of unit testing. + */ + private[scheduler] def getShuffleDependencies( + rdd: RDD[_]): HashSet[ShuffleDependency[_, _, _]] = { + val parents = new HashSet[ShuffleDependency[_, _, _]] + val visited = new HashSet[RDD[_]] + val waitingForVisit = new ArrayStack[RDD[_]] waitingForVisit.push(rdd) while (waitingForVisit.nonEmpty) { - visit(waitingForVisit.pop()) + val toVisit = waitingForVisit.pop() + if (!visited(toVisit)) { + visited += toVisit + toVisit.dependencies.foreach { + case shuffleDep: ShuffleDependency[_, _, _] => + parents += shuffleDep + case dependency => + waitingForVisit.push(dependency.rdd) + } + } } parents } @@ -435,7 +456,7 @@ class DAGScheduler( val visited = new HashSet[RDD[_]] // We are manually maintaining a stack here to prevent StackOverflowError // caused by recursively visiting - val waitingForVisit = new Stack[RDD[_]] + val waitingForVisit = new ArrayStack[RDD[_]] def visit(rdd: RDD[_]) { if (!visited(rdd)) { visited += rdd @@ -444,7 +465,7 @@ class DAGScheduler( for (dep <- rdd.dependencies) { dep match { case shufDep: ShuffleDependency[_, _, _] => - val mapStage = getShuffleMapStage(shufDep, stage.firstJobId) + val mapStage = getOrCreateShuffleMapStage(shufDep, stage.firstJobId) if (!mapStage.isAvailable) { missing += mapStage } @@ -467,13 +488,13 @@ class DAGScheduler( * all of that stage's ancestors. */ private def updateJobIdStageIdMaps(jobId: Int, stage: Stage): Unit = { + @tailrec def updateJobIdStageIdMapsList(stages: List[Stage]) { if (stages.nonEmpty) { val s = stages.head s.jobIds += jobId jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) += s.id - val parents: List[Stage] = getParentStages(s.rdd, jobId) - val parentsWithoutThisJobId = parents.filter { ! _.jobIds.contains(jobId) } + val parentsWithoutThisJobId = s.parents.filter { ! _.jobIds.contains(jobId) } updateJobIdStageIdMapsList(parentsWithoutThisJobId ++ stages.tail) } } @@ -506,8 +527,8 @@ class DAGScheduler( logDebug("Removing running stage %d".format(stageId)) runningStages -= stage } - for ((k, v) <- shuffleToMapStage.find(_._2 == stage)) { - shuffleToMapStage.remove(k) + for ((k, v) <- shuffleIdToMapStage.find(_._2 == stage)) { + shuffleIdToMapStage.remove(k) } if (waitingStages.contains(stage)) { logDebug("Removing stage %d from waiting set.".format(stageId)) @@ -541,8 +562,7 @@ class DAGScheduler( } /** - * Submit an action job to the scheduler and get a JobWaiter object back. The JobWaiter object - * can be used to block until the the job finishes executing or can be used to cancel the job. + * Submit an action job to the scheduler. * * @param rdd target RDD to run tasks on * @param func a function to run on each partition of the RDD @@ -551,6 +571,11 @@ class DAGScheduler( * @param callSite where in the user program this job was called * @param resultHandler callback to pass each result to * @param properties scheduler properties to attach to this job, e.g. fair scheduler pool name + * + * @return a JobWaiter object that can be used to block until the job finishes executing + * or can be used to cancel the job. + * + * @throws IllegalArgumentException when partitions ids are illegal */ def submitJob[T, U]( rdd: RDD[T], @@ -584,7 +609,7 @@ class DAGScheduler( /** * Run an action job on the given RDD and pass all the results to the resultHandler function as - * they arrive. Throws an exception if the job fials, or returns normally if successful. + * they arrive. * * @param rdd target RDD to run tasks on * @param func a function to run on each partition of the RDD @@ -593,6 +618,8 @@ class DAGScheduler( * @param callSite where in the user program this job was called * @param resultHandler callback to pass each result to * @param properties scheduler properties to attach to this job, e.g. fair scheduler pool name + * + * @note Throws `Exception` when the job fails */ def runJob[T, U]( rdd: RDD[T], @@ -603,11 +630,12 @@ class DAGScheduler( properties: Properties): Unit = { val start = System.nanoTime val waiter = submitJob(rdd, func, partitions, callSite, resultHandler, properties) - waiter.awaitResult() match { - case JobSucceeded => + ThreadUtils.awaitReady(waiter.completionFuture, Duration.Inf) + waiter.completionFuture.value.get match { + case scala.util.Success(_) => logInfo("Job %d finished: %s, took %f s".format (waiter.jobId, callSite.shortForm, (System.nanoTime - start) / 1e9)) - case JobFailed(exception: Exception) => + case scala.util.Failure(exception) => logInfo("Job %d failed: %s, took %f s".format (waiter.jobId, callSite.shortForm, (System.nanoTime - start) / 1e9)) // SPARK-8644: Include user stack trace in exceptions coming from DAGScheduler. @@ -623,7 +651,7 @@ class DAGScheduler( * * @param rdd target RDD to run tasks on * @param func a function to run on each partition of the RDD - * @param evaluator [[ApproximateEvaluator]] to receive the partial results + * @param evaluator `ApproximateEvaluator` to receive the partial results * @param callSite where in the user program this job was called * @param timeout maximum time to wait for the job, in milliseconds * @param properties scheduler properties to attach to this job, e.g. fair scheduler pool name @@ -646,7 +674,7 @@ class DAGScheduler( /** * Submit a shuffle map stage to run independently and get a JobWaiter object back. The waiter - * can be used to block until the the job finishes executing or can be used to cancel the job. + * can be used to block until the job finishes executing or can be used to cancel the job. * This method is used for adaptive query planning, to run map stages and look at statistics * about their outputs before submitting downstream stages. * @@ -682,9 +710,9 @@ class DAGScheduler( /** * Cancel a job that is running or waiting in the queue. */ - def cancelJob(jobId: Int): Unit = { + def cancelJob(jobId: Int, reason: Option[String]): Unit = { logInfo("Asked to cancel job " + jobId) - eventProcessLoop.post(JobCancelled(jobId)) + eventProcessLoop.post(JobCancelled(jobId, reason)) } /** @@ -705,17 +733,25 @@ class DAGScheduler( private[scheduler] def doCancelAllJobs() { // Cancel all running jobs. runningStages.map(_.firstJobId).foreach(handleJobCancellation(_, - reason = "as part of cancellation of all jobs")) + Option("as part of cancellation of all jobs"))) activeJobs.clear() // These should already be empty by this point, jobIdToActiveJob.clear() // but just in case we lost track of some jobs... - submitWaitingStages() } /** * Cancel all jobs associated with a running or scheduled stage. */ - def cancelStage(stageId: Int) { - eventProcessLoop.post(StageCancelled(stageId)) + def cancelStage(stageId: Int, reason: Option[String]) { + eventProcessLoop.post(StageCancelled(stageId, reason)) + } + + /** + * Kill a given task. It will be retried. + * + * @return Whether the task was successfully killed. + */ + def killTaskAttempt(taskId: Long, interruptThread: Boolean, reason: String): Boolean = { + taskScheduler.killTaskAttempt(taskId, interruptThread, reason) } /** @@ -734,23 +770,21 @@ class DAGScheduler( submitStage(stage) } } - submitWaitingStages() } /** - * Check for waiting or failed stages which are now eligible for resubmission. - * Ordinarily run on every iteration of the event loop. + * Check for waiting stages which are now eligible for resubmission. + * Submits stages that depend on the given parent stage. Called when the parent stage completes + * successfully. */ - private def submitWaitingStages() { - // TODO: We might want to run this less often, when we are sure that something has become - // runnable that wasn't before. - logTrace("Checking for newly runnable parent stages") + private def submitWaitingChildStages(parent: Stage) { + logTrace(s"Checking if any dependencies of $parent are now runnable") logTrace("running: " + runningStages) logTrace("waiting: " + waitingStages) logTrace("failed: " + failedStages) - val waitingStagesCopy = waitingStages.toArray - waitingStages.clear() - for (stage <- waitingStagesCopy.sortBy(_.firstJobId)) { + val childStages = waitingStages.filter(_.parents.contains(parent)).toArray + waitingStages --= childStages + for (stage <- childStages.sortBy(_.firstJobId)) { submitStage(stage) } } @@ -774,8 +808,8 @@ class DAGScheduler( } } val jobIds = activeInGroup.map(_.jobId) - jobIds.foreach(handleJobCancellation(_, "part of cancelled job group %s".format(groupId))) - submitWaitingStages() + jobIds.foreach(handleJobCancellation(_, + Option("part of cancelled job group %s".format(groupId)))) } private[scheduler] def handleBeginEvent(task: Task[_], taskInfo: TaskInfo) { @@ -783,7 +817,10 @@ class DAGScheduler( // In that case, we wouldn't have the stage anymore in stageIdToStage. val stageAttemptId = stageIdToStage.get(task.stageId).map(_.latestInfo.attemptId).getOrElse(-1) listenerBus.post(SparkListenerTaskStart(task.stageId, stageAttemptId, taskInfo)) - submitWaitingStages() + } + + private[scheduler] def handleSpeculativeTaskSubmitted(task: Task[_]): Unit = { + listenerBus.post(SparkListenerSpeculativeTaskSubmitted(task.stageId)) } private[scheduler] def handleTaskSetFailed( @@ -791,12 +828,12 @@ class DAGScheduler( reason: String, exception: Option[Throwable]): Unit = { stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason, exception) } - submitWaitingStages() } private[scheduler] def cleanUpAfterSchedulerStop() { for (job <- activeJobs) { - val error = new SparkException("Job cancelled because SparkContext was shut down") + val error = + new SparkException(s"Job ${job.jobId} cancelled because SparkContext was shut down") job.listener.jobFailed(error) // Tell the listeners that all of the running stages have ended. Don't bother // cancelling the stages because if the DAG scheduler is stopped, the entire application @@ -813,7 +850,6 @@ class DAGScheduler( private[scheduler] def handleGetTaskResult(taskInfo: TaskInfo) { listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) - submitWaitingStages() } private[scheduler] def handleJobSubmitted(jobId: Int, @@ -827,7 +863,7 @@ class DAGScheduler( try { // New stage creation may throw an exception if, for example, jobs are run on a // HadoopRDD whose underlying HDFS files have been deleted. - finalStage = newResultStage(finalRDD, func, partitions, jobId, callSite) + finalStage = createResultStage(finalRDD, func, partitions, jobId, callSite) } catch { case e: Exception => logWarning("Creating new stage failed due to exception - job: " + jobId, e) @@ -852,8 +888,6 @@ class DAGScheduler( listenerBus.post( SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, properties)) submitStage(finalStage) - - submitWaitingStages() } private[scheduler] def handleMapStageSubmitted(jobId: Int, @@ -867,7 +901,7 @@ class DAGScheduler( try { // New stage creation may throw an exception if, for example, jobs are run on a // HadoopRDD whose underlying HDFS files have been deleted. - finalStage = getShuffleMapStage(dependency, jobId) + finalStage = getOrCreateShuffleMapStage(dependency, jobId) } catch { case e: Exception => logWarning("Creating new stage failed due to exception - job: " + jobId, e) @@ -897,8 +931,6 @@ class DAGScheduler( if (finalStage.isAvailable) { markMapStageJobAsFinished(job, mapOutputTracker.getStatistics(dependency)) } - - submitWaitingStages() } /** Submits stage, but first recursively submits any missing parents. */ @@ -927,20 +959,13 @@ class DAGScheduler( /** Called when stage's parents are available and we can now do its task. */ private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") - // Get our pending tasks and remember them in our pendingTasks entry - stage.pendingPartitions.clear() // First figure out the indexes of partition ids to compute. val partitionsToCompute: Seq[Int] = stage.findMissingPartitions() - // Create internal accumulators if the stage has no accumulators initialized. - // Reset internal accumulators only if this stage is not partially submitted - // Otherwise, we may override existing accumulator values from some tasks - if (stage.internalAccumulators.isEmpty || stage.numPartitions == partitionsToCompute.size) { - stage.resetInternalAccumulators() - } - - val properties = jobIdToActiveJob.get(stage.firstJobId).map(_.properties).orNull + // Use the scheduling pool, job group, description, etc. from an ActiveJob associated + // with this Stage + val properties = jobIdToActiveJob(jobId).properties runningStages += stage // SparkListenerStageSubmitted should be posted before testing whether tasks are @@ -959,7 +984,6 @@ class DAGScheduler( case s: ShuffleMapStage => partitionsToCompute.map { id => (id, getPreferredLocs(stage.rdd, id))}.toMap case s: ResultStage => - val job = s.activeJob.get partitionsToCompute.map { id => val p = s.partitions(id) (id, getPreferredLocs(stage.rdd, p)) @@ -969,12 +993,19 @@ class DAGScheduler( case NonFatal(e) => stage.makeNewStageAttempt(partitionsToCompute.size) listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties)) - abortStage(stage, s"Task creation failed: $e\n${e.getStackTraceString}", Some(e)) + abortStage(stage, s"Task creation failed: $e\n${Utils.exceptionString(e)}", Some(e)) runningStages -= stage return } stage.makeNewStageAttempt(partitionsToCompute.size, taskIdToLocations.values.toSeq) + + // If there are tasks to execute, record the submission time of the stage. Otherwise, + // post the even without the submission time, which indicates that this stage was + // skipped. + if (partitionsToCompute.nonEmpty) { + stage.latestInfo.submissionTime = Some(clock.getTimeMillis()) + } listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties)) // TODO: Maybe we can keep the taskBinary in Stage to avoid serializing it multiple times. @@ -989,9 +1020,10 @@ class DAGScheduler( // For ResultTask, serialize and broadcast (rdd, func). val taskBinaryBytes: Array[Byte] = stage match { case stage: ShuffleMapStage => - closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef).array() + JavaUtils.bufferToArray( + closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef)) case stage: ResultStage => - closureSerializer.serialize((stage.rdd, stage.func): AnyRef).array() + JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, stage.func): AnyRef)) } taskBinary = sc.broadcast(taskBinaryBytes) @@ -1004,45 +1036,47 @@ class DAGScheduler( // Abort execution return case NonFatal(e) => - abortStage(stage, s"Task serialization failed: $e\n${e.getStackTraceString}", Some(e)) + abortStage(stage, s"Task serialization failed: $e\n${Utils.exceptionString(e)}", Some(e)) runningStages -= stage return } val tasks: Seq[Task[_]] = try { + val serializedTaskMetrics = closureSerializer.serialize(stage.latestInfo.taskMetrics).array() stage match { case stage: ShuffleMapStage => + stage.pendingPartitions.clear() partitionsToCompute.map { id => val locs = taskIdToLocations(id) val part = stage.rdd.partitions(id) + stage.pendingPartitions += id new ShuffleMapTask(stage.id, stage.latestInfo.attemptId, - taskBinary, part, locs, stage.internalAccumulators) + taskBinary, part, locs, properties, serializedTaskMetrics, Option(jobId), + Option(sc.applicationId), sc.applicationAttemptId) } case stage: ResultStage => - val job = stage.activeJob.get partitionsToCompute.map { id => val p: Int = stage.partitions(id) val part = stage.rdd.partitions(p) val locs = taskIdToLocations(id) new ResultTask(stage.id, stage.latestInfo.attemptId, - taskBinary, part, locs, id, stage.internalAccumulators) + taskBinary, part, locs, id, properties, serializedTaskMetrics, + Option(jobId), Option(sc.applicationId), sc.applicationAttemptId) } } } catch { case NonFatal(e) => - abortStage(stage, s"Task creation failed: $e\n${e.getStackTraceString}", Some(e)) + abortStage(stage, s"Task creation failed: $e\n${Utils.exceptionString(e)}", Some(e)) runningStages -= stage return } if (tasks.size > 0) { - logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") - stage.pendingPartitions ++= tasks.map(_.partitionId) - logDebug("New pending partitions: " + stage.pendingPartitions) + logInfo(s"Submitting ${tasks.size} missing tasks from $stage (${stage.rdd}) (first 15 " + + s"tasks are for partitions ${tasks.take(15).map(_.partitionId)})") taskScheduler.submitTasks(new TaskSet( - tasks.toArray, stage.id, stage.latestInfo.attemptId, stage.firstJobId, properties)) - stage.latestInfo.submissionTime = Some(clock.getTimeMillis()) + tasks.toArray, stage.id, stage.latestInfo.attemptId, jobId, properties)) } else { // Because we posted SparkListenerStageSubmitted earlier, we should mark // the stage as completed here in case there are no tasks to run @@ -1058,51 +1092,73 @@ class DAGScheduler( s"Stage ${stage} is actually done; (partitions: ${stage.numPartitions})" } logDebug(debugString) + + submitWaitingChildStages(stage) } } - /** Merge updates from a task to our local accumulator values */ + /** + * Merge local values from a task into the corresponding accumulators previously registered + * here on the driver. + * + * Although accumulators themselves are not thread-safe, this method is called only from one + * thread, the one that runs the scheduling loop. This means we only handle one task + * completion event at a time so we don't need to worry about locking the accumulators. + * This still doesn't stop the caller from updating the accumulator outside the scheduler, + * but that's not our problem since there's nothing we can do about that. + */ private def updateAccumulators(event: CompletionEvent): Unit = { val task = event.task val stage = stageIdToStage(task.stageId) - if (event.accumUpdates != null) { - try { - Accumulators.add(event.accumUpdates) - - event.accumUpdates.foreach { case (id, partialValue) => - // In this instance, although the reference in Accumulators.originals is a WeakRef, - // it's guaranteed to exist since the event.accumUpdates Map exists - - val acc = Accumulators.originals(id).get match { - case Some(accum) => accum.asInstanceOf[Accumulable[Any, Any]] - case None => throw new NullPointerException("Non-existent reference to Accumulator") - } - - // To avoid UI cruft, ignore cases where value wasn't updated - if (acc.name.isDefined && partialValue != acc.zero) { - val name = acc.name.get - val value = s"${acc.value}" - stage.latestInfo.accumulables(id) = - new AccumulableInfo(id, name, None, value, acc.isInternal) - event.taskInfo.accumulables += - new AccumulableInfo(id, name, Some(s"$partialValue"), value, acc.isInternal) - } + try { + event.accumUpdates.foreach { updates => + val id = updates.id + // Find the corresponding accumulator on the driver and update it + val acc: AccumulatorV2[Any, Any] = AccumulatorContext.get(id) match { + case Some(accum) => accum.asInstanceOf[AccumulatorV2[Any, Any]] + case None => + throw new SparkException(s"attempted to access non-existent accumulator $id") + } + acc.merge(updates.asInstanceOf[AccumulatorV2[Any, Any]]) + // To avoid UI cruft, ignore cases where value wasn't updated + if (acc.name.isDefined && !updates.isZero) { + stage.latestInfo.accumulables(id) = acc.toInfo(None, Some(acc.value)) + event.taskInfo.setAccumulables( + acc.toInfo(Some(updates.value), Some(acc.value)) +: event.taskInfo.accumulables) } - } catch { - // If we see an exception during accumulator update, just log the - // error and move on. - case e: Exception => - logError(s"Failed to update accumulators for $task", e) } + } catch { + case NonFatal(e) => + logError(s"Failed to update accumulators for task ${task.partitionId}", e) } } + private def postTaskEnd(event: CompletionEvent): Unit = { + val taskMetrics: TaskMetrics = + if (event.accumUpdates.nonEmpty) { + try { + TaskMetrics.fromAccumulators(event.accumUpdates) + } catch { + case NonFatal(e) => + val taskId = event.taskInfo.taskId + logError(s"Error when attempting to reconstruct metrics for task $taskId", e) + null + } + } else { + null + } + + listenerBus.post(SparkListenerTaskEnd(event.task.stageId, event.task.stageAttemptId, + Utils.getFormattedClassName(event.task), event.reason, event.taskInfo, taskMetrics)) + } + /** * Responds to a task finishing. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. */ private[scheduler] def handleTaskCompletion(event: CompletionEvent) { val task = event.task + val taskId = event.taskInfo.id val stageId = task.stageId val taskType = Utils.getFormattedClassName(task) @@ -1112,25 +1168,38 @@ class DAGScheduler( event.taskInfo.attemptNumber, // this is a task attempt number event.reason) - // The success case is dealt with separately below, since we need to compute accumulator - // updates before posting. - if (event.reason != Success) { - val attemptId = task.stageAttemptId - listenerBus.post(SparkListenerTaskEnd(stageId, attemptId, taskType, event.reason, - event.taskInfo, event.taskMetrics)) - } - if (!stageIdToStage.contains(task.stageId)) { + // The stage may have already finished when we get this event -- eg. maybe it was a + // speculative task. It is important that we send the TaskEnd event in any case, so listeners + // are properly notified and can chose to handle it. For instance, some listeners are + // doing their own accounting and if they don't get the task end event they think + // tasks are still running when they really aren't. + postTaskEnd(event) + // Skip all the actions if the stage has been cancelled. return } val stage = stageIdToStage(task.stageId) + + // Make sure the task's accumulators are updated before any other processing happens, so that + // we can post a task end event before any jobs or stages are updated. The accumulators are + // only updated in certain cases. + event.reason match { + case Success => + stage match { + case rs: ResultStage if rs.activeJob.isEmpty => + // Ignore update if task's job has finished. + case _ => + updateAccumulators(event) + } + case _: ExceptionFailure => updateAccumulators(event) + case _ => + } + postTaskEnd(event) + event.reason match { case Success => - listenerBus.post(SparkListenerTaskEnd(stageId, stage.latestInfo.attemptId, taskType, - event.reason, event.taskInfo, event.taskMetrics)) - stage.pendingPartitions -= task.partitionId task match { case rt: ResultTask[_, _] => // Cast to ResultStage here because it's part of the ResultTask @@ -1139,7 +1208,6 @@ class DAGScheduler( resultStage.activeJob match { case Some(job) => if (!job.finished(rt.outputId)) { - updateAccumulators(event) job.finished(rt.outputId) = true job.numFinished += 1 // If the whole job has finished, remove it @@ -1166,14 +1234,33 @@ class DAGScheduler( case smt: ShuffleMapTask => val shuffleStage = stage.asInstanceOf[ShuffleMapStage] - updateAccumulators(event) val status = event.result.asInstanceOf[MapStatus] val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) + if (stageIdToStage(task.stageId).latestInfo.attemptId == task.stageAttemptId) { + // This task was for the currently running attempt of the stage. Since the task + // completed successfully from the perspective of the TaskSetManager, mark it as + // no longer pending (the TaskSetManager may consider the task complete even + // when the output needs to be ignored because the task's epoch is too small below. + // In this case, when pending partitions is empty, there will still be missing + // output locations, which will cause the DAGScheduler to resubmit the stage below.) + shuffleStage.pendingPartitions -= task.partitionId + } if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) { logInfo(s"Ignoring possibly bogus $smt completion from executor $execId") } else { - shuffleStage.addOutputLoc(smt.partitionId, status) + // The epoch of the task is acceptable (i.e., the task was launched after the most + // recent failure we're aware of for the executor), so mark the task's output as + // available. + mapOutputTracker.registerMapOutput( + shuffleStage.shuffleDep.shuffleId, smt.partitionId, status) + // Remove the task's partition from pending partitions. This may have already been + // done above, but will not have been done yet in cases where the task attempt was + // from an earlier attempt of the stage (i.e., not the attempt that's currently + // running). This allows the DAGScheduler to mark the stage as complete when one + // copy of each task has finished successfully, even if the currently active stage + // still has tasks running. + shuffleStage.pendingPartitions -= task.partitionId } if (runningStages.contains(shuffleStage) && shuffleStage.pendingPartitions.isEmpty) { @@ -1183,21 +1270,19 @@ class DAGScheduler( logInfo("waiting: " + waitingStages) logInfo("failed: " + failedStages) - // We supply true to increment the epoch number here in case this is a - // recomputation of the map outputs. In that case, some nodes may have cached - // locations with holes (from when we detected the error) and will need the - // epoch incremented to refetch them. - // TODO: Only increment the epoch number if this is not the first time - // we registered these map outputs. - mapOutputTracker.registerMapOutputs( - shuffleStage.shuffleDep.shuffleId, - shuffleStage.outputLocInMapOutputTrackerFormat(), - changeEpoch = true) + // This call to increment the epoch may not be strictly necessary, but it is retained + // for now in order to minimize the changes in behavior from an earlier version of the + // code. This existing behavior of always incrementing the epoch following any + // successful shuffle map stage completion may have benefits by causing unneeded + // cached map outputs to be cleaned up earlier on executors. In the future we can + // consider removing this call, but this will require some extra investigation. + // See https://github.com/apache/spark/pull/17955/files#r117385673 for more details. + mapOutputTracker.incrementEpoch() clearCacheLocs() if (!shuffleStage.isAvailable) { - // Some tasks had failed; let's resubmit this shuffleStage + // Some tasks had failed; let's resubmit this shuffleStage. // TODO: Lower-level scheduler should also deal with this logInfo("Resubmitting " + shuffleStage + " (" + shuffleStage.name + ") because some of its tasks had failed: " + @@ -1211,19 +1296,25 @@ class DAGScheduler( markMapStageJobAsFinished(job, stats) } } + submitWaitingChildStages(shuffleStage) } - - // Note: newly runnable stages will be submitted below when we submit waiting stages } } case Resubmitted => logInfo("Resubmitted " + task + ", so marking it as still running") - stage.pendingPartitions += task.partitionId + stage match { + case sms: ShuffleMapStage => + sms.pendingPartitions += task.partitionId + + case _ => + assert(false, "TaskSetManagers should only send Resubmitted task statuses for " + + "tasks in ShuffleMapStages.") + } case FetchFailed(bmAddress, shuffleId, mapId, reduceId, failureMessage) => val failedStage = stageIdToStage(task.stageId) - val mapStage = shuffleToMapStage(shuffleId) + val mapStage = shuffleIdToMapStage(shuffleId) if (failedStage.latestInfo.attemptId != task.stageAttemptId) { logInfo(s"Ignoring fetch failure from $task as it's from $failedStage attempt" + @@ -1242,35 +1333,70 @@ class DAGScheduler( s"longer running") } - if (disallowStageRetryForTest) { - abortStage(failedStage, "Fetch failure will not retry stage due to testing config", - None) - } else if (failedStage.failedOnFetchAndShouldAbort(task.stageAttemptId)) { - abortStage(failedStage, s"$failedStage (${failedStage.name}) " + - s"has failed the maximum allowable number of " + - s"times: ${Stage.MAX_CONSECUTIVE_FETCH_FAILURES}. " + - s"Most recent failure reason: ${failureMessage}", None) - } else if (failedStages.isEmpty) { - // Don't schedule an event to resubmit failed stages if failed isn't empty, because - // in that case the event will already have been scheduled. - // TODO: Cancel running tasks in the stage - logInfo(s"Resubmitting $mapStage (${mapStage.name}) and " + - s"$failedStage (${failedStage.name}) due to fetch failure") - messageScheduler.schedule(new Runnable { - override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages) - }, DAGScheduler.RESUBMIT_TIMEOUT, TimeUnit.MILLISECONDS) + failedStage.fetchFailedAttemptIds.add(task.stageAttemptId) + val shouldAbortStage = + failedStage.fetchFailedAttemptIds.size >= maxConsecutiveStageAttempts || + disallowStageRetryForTest + + if (shouldAbortStage) { + val abortMessage = if (disallowStageRetryForTest) { + "Fetch failure will not retry stage due to testing config" + } else { + s"""$failedStage (${failedStage.name}) + |has failed the maximum allowable number of + |times: $maxConsecutiveStageAttempts. + |Most recent failure reason: $failureMessage""".stripMargin.replaceAll("\n", " ") + } + abortStage(failedStage, abortMessage, None) + } else { // update failedStages and make sure a ResubmitFailedStages event is enqueued + // TODO: Cancel running tasks in the failed stage -- cf. SPARK-17064 + val noResubmitEnqueued = !failedStages.contains(failedStage) + failedStages += failedStage + failedStages += mapStage + if (noResubmitEnqueued) { + // We expect one executor failure to trigger many FetchFailures in rapid succession, + // but all of those task failures can typically be handled by a single resubmission of + // the failed stage. We avoid flooding the scheduler's event queue with resubmit + // messages by checking whether a resubmit is already in the event queue for the + // failed stage. If there is already a resubmit enqueued for a different failed + // stage, that event would also be sufficient to handle the current failed stage, but + // producing a resubmit for each failed stage makes debugging and logging a little + // simpler while not producing an overwhelming number of scheduler events. + logInfo( + s"Resubmitting $mapStage (${mapStage.name}) and " + + s"$failedStage (${failedStage.name}) due to fetch failure" + ) + messageScheduler.schedule( + new Runnable { + override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages) + }, + DAGScheduler.RESUBMIT_TIMEOUT, + TimeUnit.MILLISECONDS + ) + } } - failedStages += failedStage - failedStages += mapStage // Mark the map whose fetch failed as broken in the map stage if (mapId != -1) { - mapStage.removeOutputLoc(mapId, bmAddress) mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) } // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { - handleExecutorLost(bmAddress.executorId, fetchFailed = true, Some(task.epoch)) + val hostToUnregisterOutputs = if (env.blockManager.externalShuffleServiceEnabled && + unRegisterOutputOnHostOnFetchFailure) { + // We had a fetch failure with the external shuffle service, so we + // assume all shuffle data on the node is bad. + Some(bmAddress.host) + } else { + // Unregister shuffle data just for one executor (we don't have any + // reason to believe shuffle data has been lost for the entire host). + None + } + removeExecutorAndUnregisterOutputs( + execId = bmAddress.executorId, + fileLost = true, + hostToUnregisterOutputs = hostToUnregisterOutputs, + maybeEpoch = Some(task.epoch)) } } @@ -1278,16 +1404,15 @@ class DAGScheduler( // Do nothing here, left up to the TaskScheduler to decide how to handle denied commits case exceptionFailure: ExceptionFailure => - // Do nothing here, left up to the TaskScheduler to decide how to handle user failures + // Nothing left to do, already handled above for accumulator updates. case TaskResultLost => // Do nothing here; the TaskScheduler handles these failures and resubmits the task. - case other => + case _: ExecutorLostFailure | _: TaskKilled | UnknownReason => // Unrecognized failure - also do nothing. If the task fails repeatedly, the TaskScheduler // will abort the job. } - submitWaitingStages() } /** @@ -1295,41 +1420,72 @@ class DAGScheduler( * modify the scheduler's internal state. Use executorLost() to post a loss event from outside. * * We will also assume that we've lost all shuffle blocks associated with the executor if the - * executor serves its own blocks (i.e., we're not using external shuffle) OR a FetchFailed - * occurred, in which case we presume all shuffle data related to this executor to be lost. + * executor serves its own blocks (i.e., we're not using external shuffle), the entire slave + * is lost (likely including the shuffle service), or a FetchFailed occurred, in which case we + * presume all shuffle data related to this executor to be lost. * * Optionally the epoch during which the failure was caught can be passed to avoid allowing * stray fetch failures from possibly retriggering the detection of a node as lost. */ private[scheduler] def handleExecutorLost( execId: String, - fetchFailed: Boolean, - maybeEpoch: Option[Long] = None) { + workerLost: Boolean): Unit = { + // if the cluster manager explicitly tells us that the entire worker was lost, then + // we know to unregister shuffle output. (Note that "worker" specifically refers to the process + // from a Standalone cluster, where the shuffle service lives in the Worker.) + val fileLost = workerLost || !env.blockManager.externalShuffleServiceEnabled + removeExecutorAndUnregisterOutputs( + execId = execId, + fileLost = fileLost, + hostToUnregisterOutputs = None, + maybeEpoch = None) + } + + private def removeExecutorAndUnregisterOutputs( + execId: String, + fileLost: Boolean, + hostToUnregisterOutputs: Option[String], + maybeEpoch: Option[Long] = None): Unit = { val currentEpoch = maybeEpoch.getOrElse(mapOutputTracker.getEpoch) if (!failedEpoch.contains(execId) || failedEpoch(execId) < currentEpoch) { failedEpoch(execId) = currentEpoch logInfo("Executor lost: %s (epoch %d)".format(execId, currentEpoch)) blockManagerMaster.removeExecutor(execId) - - if (!env.blockManager.externalShuffleServiceEnabled || fetchFailed) { - // TODO: This will be really slow if we keep accumulating shuffle map stages - for ((shuffleId, stage) <- shuffleToMapStage) { - stage.removeOutputsOnExecutor(execId) - mapOutputTracker.registerMapOutputs( - shuffleId, - stage.outputLocInMapOutputTrackerFormat(), - changeEpoch = true) - } - if (shuffleToMapStage.isEmpty) { - mapOutputTracker.incrementEpoch() + if (fileLost) { + hostToUnregisterOutputs match { + case Some(host) => + logInfo("Shuffle files lost for host: %s (epoch %d)".format(host, currentEpoch)) + mapOutputTracker.removeOutputsOnHost(host) + case None => + logInfo("Shuffle files lost for executor: %s (epoch %d)".format(execId, currentEpoch)) + mapOutputTracker.removeOutputsOnExecutor(execId) } clearCacheLocs() + + } else { + logDebug("Additional executor lost message for %s (epoch %d)".format(execId, currentEpoch)) } - } else { - logDebug("Additional executor lost message for " + execId + - "(epoch " + currentEpoch + ")") } - submitWaitingStages() + } + + /** + * Responds to a worker being removed. This is called inside the event loop, so it assumes it can + * modify the scheduler's internal state. Use workerRemoved() to post a loss event from outside. + * + * We will assume that we've lost all shuffle blocks associated with the host if a worker is + * removed, so we will remove them all from MapStatus. + * + * @param workerId identifier of the worker that is removed. + * @param host host of the worker that is removed. + * @param message the reason why the worker is removed. + */ + private[scheduler] def handleWorkerRemoved( + workerId: String, + host: String, + message: String): Unit = { + logInfo("Shuffle files lost for worker %s on host %s".format(workerId, host)) + mapOutputTracker.removeOutputsOnHost(host) + clearCacheLocs() } private[scheduler] def handleExecutorAdded(execId: String, host: String) { @@ -1338,30 +1494,33 @@ class DAGScheduler( logInfo("Host added was in lost list earlier: " + host) failedEpoch -= execId } - submitWaitingStages() } - private[scheduler] def handleStageCancellation(stageId: Int) { + private[scheduler] def handleStageCancellation(stageId: Int, reason: Option[String]) { stageIdToStage.get(stageId) match { case Some(stage) => val jobsThatUseStage: Array[Int] = stage.jobIds.toArray jobsThatUseStage.foreach { jobId => - handleJobCancellation(jobId, s"because Stage $stageId was cancelled") + val reasonStr = reason match { + case Some(originalReason) => + s"because $originalReason" + case None => + s"because Stage $stageId was cancelled" + } + handleJobCancellation(jobId, Option(reasonStr)) } case None => logInfo("No active jobs to kill for Stage " + stageId) } - submitWaitingStages() } - private[scheduler] def handleJobCancellation(jobId: Int, reason: String = "") { + private[scheduler] def handleJobCancellation(jobId: Int, reason: Option[String]) { if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { failJobAndIndependentStages( - jobIdToActiveJob(jobId), "Job %d cancelled %s".format(jobId, reason)) + jobIdToActiveJob(jobId), "Job %d cancelled %s".format(jobId, reason.getOrElse(""))) } - submitWaitingStages() } /** @@ -1383,7 +1542,7 @@ class DAGScheduler( stage.clearFailures() } else { stage.latestInfo.stageFailed(errorMessage.get) - logInfo("%s (%s) failed in %s s".format(stage, stage.name, serviceTime)) + logInfo(s"$stage (${stage.name}) failed in $serviceTime s due to ${errorMessage.get}") } outputCommitCoordinator.stageEnd(stage.id) @@ -1458,8 +1617,10 @@ class DAGScheduler( } if (ableToCancelStages) { - job.listener.jobFailed(error) + // SPARK-15783 important to cleanup state first, just for tests where we have some asserts + // against the state. Otherwise we have a *little* bit of flakiness in the tests. cleanupStateForJobAndIndependentStages(job) + job.listener.jobFailed(error) listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), JobFailed(error))) } } @@ -1472,14 +1633,14 @@ class DAGScheduler( val visitedRdds = new HashSet[RDD[_]] // We are manually maintaining a stack here to prevent StackOverflowError // caused by recursively visiting - val waitingForVisit = new Stack[RDD[_]] + val waitingForVisit = new ArrayStack[RDD[_]] def visit(rdd: RDD[_]) { if (!visitedRdds(rdd)) { visitedRdds += rdd for (dep <- rdd.dependencies) { dep match { case shufDep: ShuffleDependency[_, _, _] => - val mapStage = getShuffleMapStage(shufDep, stage.firstJobId) + val mapStage = getOrCreateShuffleMapStage(shufDep, stage.firstJobId) if (!mapStage.isAvailable) { waitingForVisit.push(mapStage.rdd) } // Otherwise there's no need to follow the dependency back @@ -1568,14 +1729,11 @@ class DAGScheduler( } def stop() { - logInfo("Stopping DAGScheduler") messageScheduler.shutdownNow() eventProcessLoop.stop() taskScheduler.stop() } - // Start the event thread and register the metrics source at the end of the constructor - env.metricsSystem.registerSource(metricsSource) eventProcessLoop.start() } @@ -1603,11 +1761,11 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler case MapStageSubmitted(jobId, dependency, callSite, listener, properties) => dagScheduler.handleMapStageSubmitted(jobId, dependency, callSite, listener, properties) - case StageCancelled(stageId) => - dagScheduler.handleStageCancellation(stageId) + case StageCancelled(stageId, reason) => + dagScheduler.handleStageCancellation(stageId, reason) - case JobCancelled(jobId) => - dagScheduler.handleJobCancellation(jobId) + case JobCancelled(jobId, reason) => + dagScheduler.handleJobCancellation(jobId, reason) case JobGroupCancelled(groupId) => dagScheduler.handleJobGroupCancelled(groupId) @@ -1618,16 +1776,26 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler case ExecutorAdded(execId, host) => dagScheduler.handleExecutorAdded(execId, host) - case ExecutorLost(execId) => - dagScheduler.handleExecutorLost(execId, fetchFailed = false) + case ExecutorLost(execId, reason) => + val workerLost = reason match { + case SlaveLost(_, true) => true + case _ => false + } + dagScheduler.handleExecutorLost(execId, workerLost) + + case WorkerRemoved(workerId, host, message) => + dagScheduler.handleWorkerRemoved(workerId, host, message) case BeginEvent(task, taskInfo) => dagScheduler.handleBeginEvent(task, taskInfo) + case SpeculativeTaskSubmitted(task) => + dagScheduler.handleSpeculativeTaskSubmitted(task) + case GettingResultEvent(taskInfo) => dagScheduler.handleGetTaskResult(taskInfo) - case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => + case completion: CompletionEvent => dagScheduler.handleTaskCompletion(completion) case TaskSetFailed(taskSet, reason, exception) => @@ -1644,7 +1812,7 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler } catch { case t: Throwable => logError("DAGScheduler failed to cancel all jobs.", t) } - dagScheduler.sc.stop() + dagScheduler.sc.stopInNewThread() } override def onStop(): Unit = { @@ -1658,4 +1826,7 @@ private[spark] object DAGScheduler { // this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one // as more failure events come in val RESUBMIT_TIMEOUT = 200 + + // Number of consecutive stage attempts allowed before a stage is aborted + val DEFAULT_MAX_CONSECUTIVE_STAGE_ATTEMPTS = 4 } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index dda3b6cc7f96..54ab8f8b3e1d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -19,13 +19,11 @@ package org.apache.spark.scheduler import java.util.Properties -import scala.collection.Map import scala.language.existentials import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics import org.apache.spark.rdd.RDD -import org.apache.spark.util.CallSite +import org.apache.spark.util.{AccumulatorV2, CallSite} /** * Types of events that can be handled by the DAGScheduler. The DAGScheduler uses an event queue @@ -55,9 +53,15 @@ private[scheduler] case class MapStageSubmitted( properties: Properties = null) extends DAGSchedulerEvent -private[scheduler] case class StageCancelled(stageId: Int) extends DAGSchedulerEvent +private[scheduler] case class StageCancelled( + stageId: Int, + reason: Option[String]) + extends DAGSchedulerEvent -private[scheduler] case class JobCancelled(jobId: Int) extends DAGSchedulerEvent +private[scheduler] case class JobCancelled( + jobId: Int, + reason: Option[String]) + extends DAGSchedulerEvent private[scheduler] case class JobGroupCancelled(groupId: String) extends DAGSchedulerEvent @@ -73,17 +77,24 @@ private[scheduler] case class CompletionEvent( task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Map[Long, Any], - taskInfo: TaskInfo, - taskMetrics: TaskMetrics) + accumUpdates: Seq[AccumulatorV2[_, _]], + taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] case class ExecutorAdded(execId: String, host: String) extends DAGSchedulerEvent -private[scheduler] case class ExecutorLost(execId: String) extends DAGSchedulerEvent +private[scheduler] case class ExecutorLost(execId: String, reason: ExecutorLossReason) + extends DAGSchedulerEvent + +private[scheduler] case class WorkerRemoved(workerId: String, host: String, message: String) + extends DAGSchedulerEvent private[scheduler] case class TaskSetFailed(taskSet: TaskSet, reason: String, exception: Option[Throwable]) extends DAGSchedulerEvent private[scheduler] case object ResubmitFailedStages extends DAGSchedulerEvent + +private[scheduler] +case class SpeculativeTaskSubmitted(task: Task[_]) extends DAGSchedulerEvent + diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 000a021a528c..9dafa0b7646b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -19,19 +19,24 @@ package org.apache.spark.scheduler import java.io._ import java.net.URI +import java.nio.charset.StandardCharsets +import java.util.EnumSet +import java.util.Locale import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import com.google.common.base.Charsets import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} import org.apache.hadoop.fs.permission.FsPermission +import org.apache.hadoop.hdfs.DFSOutputStream +import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf, SPARK_VERSION} +import org.apache.spark.{SPARK_VERSION, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, Utils} @@ -77,14 +82,6 @@ private[spark] class EventLoggingListener( // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None - // The Hadoop APIs have changed over time, so we use reflection to figure out - // the correct method to use to flush a hadoop data stream. See SPARK-1518 - // for details. - private val hadoopFlushMethod = { - val cls = classOf[FSDataOutputStream] - scala.util.Try(cls.getMethod("hflush")).getOrElse(cls.getMethod("sync")) - } - private var writer: Option[PrintWriter] = None // For testing. Keep track of all JSON serialized events that have been logged. @@ -97,21 +94,18 @@ private[spark] class EventLoggingListener( * Creates the log file in the configured log directory. */ def start() { - if (!fileSystem.getFileStatus(new Path(logBaseDir)).isDir) { - throw new IllegalArgumentException(s"Log directory $logBaseDir does not exist.") + if (!fileSystem.getFileStatus(new Path(logBaseDir)).isDirectory) { + throw new IllegalArgumentException(s"Log directory $logBaseDir is not a directory.") } val workingPath = logPath + IN_PROGRESS - val uri = new URI(workingPath) val path = new Path(workingPath) + val uri = path.toUri val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme val isDefaultLocal = defaultFs == null || defaultFs == "file" - if (shouldOverwrite && fileSystem.exists(path)) { + if (shouldOverwrite && fileSystem.delete(path, true)) { logWarning(s"Event log $path already exists. Overwriting...") - if (!fileSystem.delete(path, true)) { - logWarning(s"Error deleting $path") - } } /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). @@ -128,7 +122,7 @@ private[spark] class EventLoggingListener( val cstream = compressionCodec.map(_.compressedOutputStream(dstream)).getOrElse(dstream) val bstream = new BufferedOutputStream(cstream, outputBufferSize) - EventLoggingListener.initEventLog(bstream) + EventLoggingListener.initEventLog(bstream, testing, loggedEvents) fileSystem.setPermission(path, LOG_FILE_PERMISSIONS) writer = Some(new PrintWriter(bstream)) logInfo("Logging events to %s".format(logPath)) @@ -147,7 +141,10 @@ private[spark] class EventLoggingListener( // scalastyle:on println if (flushLogger) { writer.foreach(_.flush()) - hadoopDataStream.foreach(hadoopFlushMethod.invoke(_)) + hadoopDataStream.foreach(ds => ds.getWrappedStream match { + case wrapped: DFSOutputStream => wrapped.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH)) + case _ => ds.hflush() + }) } if (testing) { loggedEvents += eventJson @@ -163,7 +160,9 @@ private[spark] class EventLoggingListener( override def onTaskEnd(event: SparkListenerTaskEnd): Unit = logEvent(event) - override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Unit = logEvent(event) + override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Unit = { + logEvent(redactEvent(event)) + } // Events that trigger a flush override def onStageCompleted(event: SparkListenerStageCompleted): Unit = { @@ -201,12 +200,34 @@ private[spark] class EventLoggingListener( logEvent(event, flushLogger = true) } + override def onExecutorBlacklisted(event: SparkListenerExecutorBlacklisted): Unit = { + logEvent(event, flushLogger = true) + } + + override def onExecutorUnblacklisted(event: SparkListenerExecutorUnblacklisted): Unit = { + logEvent(event, flushLogger = true) + } + + override def onNodeBlacklisted(event: SparkListenerNodeBlacklisted): Unit = { + logEvent(event, flushLogger = true) + } + + override def onNodeUnblacklisted(event: SparkListenerNodeUnblacklisted): Unit = { + logEvent(event, flushLogger = true) + } + // No-op because logging every update would be overkill override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = {} // No-op because logging every update would be overkill override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { } + override def onOtherEvent(event: SparkListenerEvent): Unit = { + if (event.logEvent) { + logEvent(event, flushLogger = true) + } + } + /** * Stop logging events. The event log file will be renamed so that it loses the * ".inprogress" suffix. @@ -226,6 +247,28 @@ private[spark] class EventLoggingListener( } } fileSystem.rename(new Path(logPath + IN_PROGRESS), target) + // touch file to ensure modtime is current across those filesystems where rename() + // does not set it, -and which support setTimes(); it's a no-op on most object stores + try { + fileSystem.setTimes(target, System.currentTimeMillis(), -1) + } catch { + case e: Exception => logDebug(s"failed to set time of $target", e) + } + } + + private[spark] def redactEvent( + event: SparkListenerEnvironmentUpdate): SparkListenerEnvironmentUpdate = { + // environmentDetails maps a string descriptor to a set of properties + // Similar to: + // "JVM Information" -> jvmInformation, + // "Spark Properties" -> sparkProperties, + // ... + // where jvmInformation, sparkProperties, etc. are sequence of tuples. + // We go through the various of properties and redact sensitive information from them. + val redactedProps = event.environmentDetails.map{ case (name, props) => + name -> Utils.redact(sparkConf, props) + } + SparkListenerEnvironmentUpdate(redactedProps) } } @@ -234,8 +277,6 @@ private[spark] object EventLoggingListener extends Logging { // Suffix applied to the names of files still being written by applications. val IN_PROGRESS = ".inprogress" val DEFAULT_LOG_DIR = "/tmp/spark-events" - val SPARK_VERSION_KEY = "SPARK_VERSION" - val COMPRESSION_CODEC_KEY = "COMPRESSION_CODEC" private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) @@ -248,10 +289,17 @@ private[spark] object EventLoggingListener extends Logging { * * @param logStream Raw output stream to the event log file. */ - def initEventLog(logStream: OutputStream): Unit = { + def initEventLog( + logStream: OutputStream, + testing: Boolean, + loggedEvents: ArrayBuffer[JValue]): Unit = { val metadata = SparkListenerLogStart(SPARK_VERSION) - val metadataJson = compact(JsonProtocol.logStartToJson(metadata)) + "\n" - logStream.write(metadataJson.getBytes(Charsets.UTF_8)) + val eventJson = JsonProtocol.logStartToJson(metadata) + val metadataJson = compact(eventJson) + "\n" + logStream.write(metadataJson.getBytes(StandardCharsets.UTF_8)) + if (testing && loggedEvents != null) { + loggedEvents += eventJson + } } /** @@ -278,7 +326,7 @@ private[spark] object EventLoggingListener extends Logging { appId: String, appAttemptId: Option[String], compressionCodecName: Option[String] = None): String = { - val base = logBaseDir.toString.stripSuffix("/") + "/" + sanitize(appId) + val base = new Path(logBaseDir).toString.stripSuffix("/") + "/" + sanitize(appId) val codec = compressionCodecName.map("." + _).getOrElse("") if (appAttemptId.isDefined) { base + "_" + sanitize(appAttemptId.get) + codec @@ -288,7 +336,7 @@ private[spark] object EventLoggingListener extends Logging { } private def sanitize(str: String): String = { - str.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase + str.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase(Locale.ROOT) } /** @@ -297,26 +345,20 @@ private[spark] object EventLoggingListener extends Logging { * @return input stream that holds one JSON record per line. */ def openEventLog(log: Path, fs: FileSystem): InputStream = { - // It's not clear whether FileSystem.open() throws FileNotFoundException or just plain - // IOException when a file does not exist, so try our best to throw a proper exception. - if (!fs.exists(log)) { - throw new FileNotFoundException(s"File $log does not exist.") - } - val in = new BufferedInputStream(fs.open(log)) // Compression codec is encoded as an extension, e.g. app_123.lzf // Since we sanitize the app ID to not include periods, it is safe to split on it val logName = log.getName.stripSuffix(IN_PROGRESS) val codecName: Option[String] = logName.split("\\.").tail.lastOption - val codec = codecName.map { c => - codecMap.getOrElseUpdate(c, CompressionCodec.createCodec(new SparkConf, c)) - } try { + val codec = codecName.map { c => + codecMap.getOrElseUpdate(c, CompressionCodec.createCodec(new SparkConf, c)) + } codec.map(_.compressedInputStream(in)).getOrElse(in) } catch { - case e: Exception => + case e: Throwable => in.close() throw e } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala new file mode 100644 index 000000000000..70553d8be28b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.scheduler + +import scala.collection.mutable.HashMap + +/** + * Small helper for tracking failed tasks for blacklisting purposes. Info on all failures on one + * executor, within one task set. + */ +private[scheduler] class ExecutorFailuresInTaskSet(val node: String) { + /** + * Mapping from index of the tasks in the taskset, to the number of times it has failed on this + * executor and the most recent failure time. + */ + val taskToFailureCountAndFailureTime = HashMap[Int, (Int, Long)]() + + def updateWithFailure(taskIndex: Int, failureTime: Long): Unit = { + val (prevFailureCount, prevFailureTime) = + taskToFailureCountAndFailureTime.getOrElse(taskIndex, (0, -1L)) + // these times always come from the driver, so we don't need to worry about skew, but might + // as well still be defensive in case there is non-monotonicity in the clock + val newFailureTime = math.max(prevFailureTime, failureTime) + taskToFailureCountAndFailureTime(taskIndex) = (prevFailureCount + 1, newFailureTime) + } + + def numUniqueTasksWithFailures: Int = taskToFailureCountAndFailureTime.size + + /** + * Return the number of times this executor has failed on the given task index. + */ + def getNumTaskFailures(index: Int): Int = { + taskToFailureCountAndFailureTime.getOrElse(index, (0, 0))._1 + } + + override def toString(): String = { + s"numUniqueTasksWithFailures = $numUniqueTasksWithFailures; " + + s"tasksToFailureCount = $taskToFailureCountAndFailureTime" + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala index 47a5cbff4930..46a35b6a2eaf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler import org.apache.spark.executor.ExecutorExitCode /** - * Represents an explanation for a executor or whole slave failing or exiting. + * Represents an explanation for an executor or whole slave failing or exiting. */ private[spark] class ExecutorLossReason(val message: String) extends Serializable { @@ -40,6 +40,8 @@ private[spark] object ExecutorExited { } } +private[spark] object ExecutorKilled extends ExecutorLossReason("Executor killed by driver.") + /** * A loss reason that means we don't yet know why the executor exited. * @@ -49,6 +51,10 @@ private[spark] object ExecutorExited { */ private [spark] object LossReasonPending extends ExecutorLossReason("Pending loss reason.") +/** + * @param _message human readable loss reason + * @param workerLost whether the worker is confirmed lost too (i.e. including shuffle service) + */ private[spark] -case class SlaveLost(_message: String = "Slave lost") +case class SlaveLost(_message: String = "Slave lost", workerLost: Boolean = false) extends ExecutorLossReason(_message) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExternalClusterManager.scala b/core/src/main/scala/org/apache/spark/scheduler/ExternalClusterManager.scala new file mode 100644 index 000000000000..47f3527a32c0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ExternalClusterManager.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import org.apache.spark.SparkContext + +/** + * A cluster manager interface to plugin external scheduler. + */ +private[spark] trait ExternalClusterManager { + + /** + * Check if this cluster manager instance can create scheduler components + * for a certain master URL. + * @param masterURL the master URL + * @return True if the cluster manager can create scheduler backend/ + */ + def canCreate(masterURL: String): Boolean + + /** + * Create a task scheduler instance for the given SparkContext + * @param sc SparkContext + * @param masterURL the master URL + * @return TaskScheduler that will be responsible for task handling + */ + def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler + + /** + * Create a scheduler backend for the given SparkContext and scheduler. This is + * called after task scheduler is created using `ExternalClusterManager.createTaskScheduler()`. + * @param sc SparkContext + * @param masterURL the master URL + * @param scheduler TaskScheduler that will be used with the scheduler backend. + * @return SchedulerBackend that works with a TaskScheduler + */ + def createSchedulerBackend(sc: SparkContext, + masterURL: String, + scheduler: TaskScheduler): SchedulerBackend + + /** + * Initialize task scheduler and backend scheduler. This is called after the + * scheduler components are created + * @param scheduler TaskScheduler that will be responsible for task handling + * @param backend SchedulerBackend that works with a TaskScheduler + */ + def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index 0e438ab4366d..66ab9a52b778 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -26,9 +26,9 @@ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.util.ReflectionUtils -import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging /** * :: DeveloperApi :: @@ -57,11 +57,10 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl // Since we are not doing canonicalization of path, this can be wrong : like relative vs // absolute path .. which is fine, this is best case effort to remove duplicates - right ? override def equals(other: Any): Boolean = other match { - case that: InputFormatInfo => { + case that: InputFormatInfo => // not checking config - that should be fine, right ? this.inputFormatClazz == that.inputFormatClazz && this.path == that.path - } case _ => false } @@ -86,10 +85,9 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl } } catch { - case e: ClassNotFoundException => { + case e: ClassNotFoundException => throw new IllegalArgumentException("Specified inputformat " + inputFormatClazz + " cannot be found ?", e) - } } } @@ -103,7 +101,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl val instance: org.apache.hadoop.mapreduce.InputFormat[_, _] = ReflectionUtils.newInstance(inputFormatClazz.asInstanceOf[Class[_]], conf).asInstanceOf[ org.apache.hadoop.mapreduce.InputFormat[_, _]] - val job = new Job(conf) + val job = Job.getInstance(conf) val retval = new ArrayBuffer[SplitInfo]() val list = instance.getSplits(job) @@ -155,9 +153,9 @@ object InputFormatInfo { a) For each host, count number of splits hosted on that host. b) Decrement the currently allocated containers on that host. - c) Compute rack info for each host and update rack -> count map based on (b). + c) Compute rack info for each host and update rack to count map based on (b). d) Allocate nodes based on (c) - e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node + e) On the allocation result, ensure that we don't allocate "too many" jobs on a single node (even if data locality on that is very high) : this is to prevent fragility of job if a single (or small set of) hosts go down. @@ -173,7 +171,7 @@ object InputFormatInfo { for (inputSplit <- formats) { val splits = inputSplit.findPreferredLocations() - for (split <- splits){ + for (split <- splits) { val location = split.hostLocation val set = nodeToSplit.getOrElseUpdate(location, new HashSet[SplitInfo]) set += split diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala b/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala index 50c2b9acd609..e0f7c8f02132 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala @@ -23,6 +23,6 @@ package org.apache.spark.scheduler * job fails (and no further taskSucceeded events will happen). */ private[spark] trait JobListener { - def taskSucceeded(index: Int, result: Any) - def jobFailed(exception: Exception) + def taskSucceeded(index: Int, result: Any): Unit + def jobFailed(exception: Exception): Unit } diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala deleted file mode 100644 index f96eb8ca0ae0..000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ /dev/null @@ -1,277 +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.scheduler - -import java.io.{File, FileNotFoundException, IOException, PrintWriter} -import java.text.SimpleDateFormat -import java.util.{Date, Properties} - -import scala.collection.mutable.HashMap - -import org.apache.spark._ -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.TaskMetrics - -/** - * :: DeveloperApi :: - * A logger class to record runtime information for jobs in Spark. This class outputs one log file - * for each Spark job, containing tasks start/stop and shuffle information. JobLogger is a subclass - * of SparkListener, use addSparkListener to add JobLogger to a SparkContext after the SparkContext - * is created. Note that each JobLogger only works for one SparkContext - * - * NOTE: The functionality of this class is heavily stripped down to accommodate for a general - * refactor of the SparkListener interface. In its place, the EventLoggingListener is introduced - * to log application information as SparkListenerEvents. To enable this functionality, set - * spark.eventLog.enabled to true. - */ -@DeveloperApi -@deprecated("Log application information by setting spark.eventLog.enabled.", "1.0.0") -class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { - - def this() = this(System.getProperty("user.name", ""), - String.valueOf(System.currentTimeMillis())) - - private val logDir = - if (System.getenv("SPARK_LOG_DIR") != null) { - System.getenv("SPARK_LOG_DIR") - } else { - "/tmp/spark-%s".format(user) - } - - private val jobIdToPrintWriter = new HashMap[Int, PrintWriter] - private val stageIdToJobId = new HashMap[Int, Int] - private val jobIdToStageIds = new HashMap[Int, Seq[Int]] - private val dateFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - } - - createLogDir() - - /** Create a folder for log files, the folder's name is the creation time of jobLogger */ - protected def createLogDir() { - val dir = new File(logDir + "/" + logDirName + "/") - if (dir.exists()) { - return - } - if (!dir.mkdirs()) { - // JobLogger should throw a exception rather than continue to construct this object. - throw new IOException("create log directory error:" + logDir + "/" + logDirName + "/") - } - } - - /** - * Create a log file for one job - * @param jobId ID of the job - * @throws FileNotFoundException Fail to create log file - */ - protected def createLogWriter(jobId: Int) { - try { - val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobId) - jobIdToPrintWriter += (jobId -> fileWriter) - } catch { - case e: FileNotFoundException => e.printStackTrace() - } - } - - /** - * Close log file, and clean the stage relationship in stageIdToJobId - * @param jobId ID of the job - */ - protected def closeLogWriter(jobId: Int) { - jobIdToPrintWriter.get(jobId).foreach { fileWriter => - fileWriter.close() - jobIdToStageIds.get(jobId).foreach(_.foreach { stageId => - stageIdToJobId -= stageId - }) - jobIdToPrintWriter -= jobId - jobIdToStageIds -= jobId - } - } - - /** - * Build up the maps that represent stage-job relationships - * @param jobId ID of the job - * @param stageIds IDs of the associated stages - */ - protected def buildJobStageDependencies(jobId: Int, stageIds: Seq[Int]) = { - jobIdToStageIds(jobId) = stageIds - stageIds.foreach { stageId => stageIdToJobId(stageId) = jobId } - } - - /** - * Write info into log file - * @param jobId ID of the job - * @param info Info to be recorded - * @param withTime Controls whether to record time stamp before the info, default is true - */ - protected def jobLogInfo(jobId: Int, info: String, withTime: Boolean = true) { - var writeInfo = info - if (withTime) { - val date = new Date(System.currentTimeMillis()) - writeInfo = dateFormat.get.format(date) + ": " + info - } - // scalastyle:off println - jobIdToPrintWriter.get(jobId).foreach(_.println(writeInfo)) - // scalastyle:on println - } - - /** - * Write info into log file - * @param stageId ID of the stage - * @param info Info to be recorded - * @param withTime Controls whether to record time stamp before the info, default is true - */ - protected def stageLogInfo(stageId: Int, info: String, withTime: Boolean = true) { - stageIdToJobId.get(stageId).foreach(jobId => jobLogInfo(jobId, info, withTime)) - } - - /** - * Record task metrics into job log files, including execution info and shuffle metrics - * @param stageId Stage ID of the task - * @param status Status info of the task - * @param taskInfo Task description info - * @param taskMetrics Task running metrics - */ - protected def recordTaskMetrics(stageId: Int, status: String, - taskInfo: TaskInfo, taskMetrics: TaskMetrics) { - val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageId + - " START_TIME=" + taskInfo.launchTime + " FINISH_TIME=" + taskInfo.finishTime + - " EXECUTOR_ID=" + taskInfo.executorId + " HOST=" + taskMetrics.hostname - val executorRunTime = " EXECUTOR_RUN_TIME=" + taskMetrics.executorRunTime - val gcTime = " GC_TIME=" + taskMetrics.jvmGCTime - val inputMetrics = taskMetrics.inputMetrics match { - case Some(metrics) => - " READ_METHOD=" + metrics.readMethod.toString + - " INPUT_BYTES=" + metrics.bytesRead - case None => "" - } - val outputMetrics = taskMetrics.outputMetrics match { - case Some(metrics) => - " OUTPUT_BYTES=" + metrics.bytesWritten - case None => "" - } - val shuffleReadMetrics = taskMetrics.shuffleReadMetrics match { - case Some(metrics) => - " BLOCK_FETCHED_TOTAL=" + metrics.totalBlocksFetched + - " BLOCK_FETCHED_LOCAL=" + metrics.localBlocksFetched + - " BLOCK_FETCHED_REMOTE=" + metrics.remoteBlocksFetched + - " REMOTE_FETCH_WAIT_TIME=" + metrics.fetchWaitTime + - " REMOTE_BYTES_READ=" + metrics.remoteBytesRead + - " LOCAL_BYTES_READ=" + metrics.localBytesRead - case None => "" - } - val writeMetrics = taskMetrics.shuffleWriteMetrics match { - case Some(metrics) => - " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten + - " SHUFFLE_WRITE_TIME=" + metrics.shuffleWriteTime - case None => "" - } - stageLogInfo(stageId, status + info + executorRunTime + gcTime + inputMetrics + outputMetrics + - shuffleReadMetrics + writeMetrics) - } - - /** - * When stage is submitted, record stage submit info - * @param stageSubmitted Stage submitted event - */ - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { - val stageInfo = stageSubmitted.stageInfo - stageLogInfo(stageInfo.stageId, "STAGE_ID=%d STATUS=SUBMITTED TASK_SIZE=%d".format( - stageInfo.stageId, stageInfo.numTasks)) - } - - /** - * When stage is completed, record stage completion status - * @param stageCompleted Stage completed event - */ - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { - val stageId = stageCompleted.stageInfo.stageId - if (stageCompleted.stageInfo.failureReason.isEmpty) { - stageLogInfo(stageId, s"STAGE_ID=$stageId STATUS=COMPLETED") - } else { - stageLogInfo(stageId, s"STAGE_ID=$stageId STATUS=FAILED") - } - } - - /** - * When task ends, record task completion status and metrics - * @param taskEnd Task end event - */ - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val taskInfo = taskEnd.taskInfo - var taskStatus = "TASK_TYPE=%s".format(taskEnd.taskType) - val taskMetrics = if (taskEnd.taskMetrics != null) taskEnd.taskMetrics else TaskMetrics.empty - taskEnd.reason match { - case Success => taskStatus += " STATUS=SUCCESS" - recordTaskMetrics(taskEnd.stageId, taskStatus, taskInfo, taskMetrics) - case Resubmitted => - taskStatus += " STATUS=RESUBMITTED TID=" + taskInfo.taskId + - " STAGE_ID=" + taskEnd.stageId - stageLogInfo(taskEnd.stageId, taskStatus) - case FetchFailed(bmAddress, shuffleId, mapId, reduceId, message) => - taskStatus += " STATUS=FETCHFAILED TID=" + taskInfo.taskId + " STAGE_ID=" + - taskEnd.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + - mapId + " REDUCE_ID=" + reduceId - stageLogInfo(taskEnd.stageId, taskStatus) - case _ => - } - } - - /** - * When job ends, recording job completion status and close log file - * @param jobEnd Job end event - */ - override def onJobEnd(jobEnd: SparkListenerJobEnd) { - val jobId = jobEnd.jobId - var info = "JOB_ID=" + jobId - jobEnd.jobResult match { - case JobSucceeded => info += " STATUS=SUCCESS" - case JobFailed(exception) => - info += " STATUS=FAILED REASON=" - exception.getMessage.split("\\s+").foreach(info += _ + "_") - case _ => - } - jobLogInfo(jobId, info.substring(0, info.length - 1).toUpperCase) - closeLogWriter(jobId) - } - - /** - * Record job properties into job log file - * @param jobId ID of the job - * @param properties Properties of the job - */ - protected def recordJobProperties(jobId: Int, properties: Properties) { - if (properties != null) { - val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "") - jobLogInfo(jobId, description, withTime = false) - } - } - - /** - * When job starts, record job property and stage graph - * @param jobStart Job start event - */ - override def onJobStart(jobStart: SparkListenerJobStart) { - val jobId = jobStart.jobId - val properties = jobStart.properties - createLogWriter(jobId) - recordJobProperties(jobId, properties) - buildJobStageDependencies(jobId, jobStart.stageIds) - jobLogInfo(jobId, "JOB_ID=" + jobId + " STATUS=STARTED") - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index 4cd6cbe189aa..4a304a078d65 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -29,5 +29,4 @@ sealed trait JobResult @DeveloperApi case object JobSucceeded extends JobResult -@DeveloperApi private[spark] case class JobFailed(exception: Exception) extends JobResult diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index 382b09422a4a..65d7184231e2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -17,6 +17,12 @@ package org.apache.spark.scheduler +import java.util.concurrent.atomic.AtomicInteger + +import scala.concurrent.{Future, Promise} + +import org.apache.spark.internal.Logging + /** * An object that waits for a DAGScheduler job to complete. As tasks finish, it passes their * results to the given handler function. @@ -26,19 +32,17 @@ private[spark] class JobWaiter[T]( val jobId: Int, totalTasks: Int, resultHandler: (Int, T) => Unit) - extends JobListener { - - private var finishedTasks = 0 - - // Is the job as a whole finished (succeeded or failed)? - @volatile - private var _jobFinished = totalTasks == 0 - - def jobFinished: Boolean = _jobFinished + extends JobListener with Logging { + private val finishedTasks = new AtomicInteger(0) // If the job is finished, this will be its result. In the case of 0 task jobs (e.g. zero // partition RDDs), we set the jobResult directly to JobSucceeded. - private var jobResult: JobResult = if (jobFinished) JobSucceeded else null + private val jobPromise: Promise[Unit] = + if (totalTasks == 0) Promise.successful(()) else Promise() + + def jobFinished: Boolean = jobPromise.isCompleted + + def completionFuture: Future[Unit] = jobPromise.future /** * Sends a signal to the DAGScheduler to cancel the job. The cancellation itself is handled @@ -46,32 +50,23 @@ private[spark] class JobWaiter[T]( * will fail this job with a SparkException. */ def cancel() { - dagScheduler.cancelJob(jobId) + dagScheduler.cancelJob(jobId, None) } - override def taskSucceeded(index: Int, result: Any): Unit = synchronized { - if (_jobFinished) { - throw new UnsupportedOperationException("taskSucceeded() called on a finished JobWaiter") + override def taskSucceeded(index: Int, result: Any): Unit = { + // resultHandler call must be synchronized in case resultHandler itself is not thread safe. + synchronized { + resultHandler(index, result.asInstanceOf[T]) } - resultHandler(index, result.asInstanceOf[T]) - finishedTasks += 1 - if (finishedTasks == totalTasks) { - _jobFinished = true - jobResult = JobSucceeded - this.notifyAll() + if (finishedTasks.incrementAndGet() == totalTasks) { + jobPromise.success(()) } } - override def jobFailed(exception: Exception): Unit = synchronized { - _jobFinished = true - jobResult = JobFailed(exception) - this.notifyAll() - } - - def awaitResult(): JobResult = synchronized { - while (!_jobFinished) { - this.wait() + override def jobFailed(exception: Exception): Unit = { + if (!jobPromise.tryFailure(exception)) { + logWarning("Ignore failure", exception) } - return jobResult } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index be23056e7d42..2f93c497c577 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -17,29 +17,241 @@ package org.apache.spark.scheduler -import java.util.concurrent.atomic.AtomicBoolean +import java.util.{List => JList} +import java.util.concurrent._ +import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} -import org.apache.spark.util.AsynchronousListenerBus +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.reflect.ClassTag +import scala.util.DynamicVariable + +import com.codahale.metrics.{Counter, MetricRegistry, Timer} + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.metrics.source.Source /** * Asynchronously passes SparkListenerEvents to registered SparkListeners. * - * Until start() is called, all posted events are only buffered. Only after this listener bus + * Until `start()` is called, all posted events are only buffered. Only after this listener bus * has started will events be actually propagated to all attached listeners. This listener bus - * is stopped when it receives a SparkListenerShutdown event, which is posted using stop(). + * is stopped when `stop()` is called, and it will drop further events after stopping. */ -private[spark] class LiveListenerBus - extends AsynchronousListenerBus[SparkListener, SparkListenerEvent]("SparkListenerBus") - with SparkListenerBus { - - private val logDroppedEvent = new AtomicBoolean(false) - - override def onDropEvent(event: SparkListenerEvent): Unit = { - if (logDroppedEvent.compareAndSet(false, true)) { - // Only log the following message once to avoid duplicated annoying logs. - logError("Dropping SparkListenerEvent because no remaining room in event queue. " + - "This likely means one of the SparkListeners is too slow and cannot keep up with " + - "the rate at which tasks are being started by the scheduler.") +private[spark] class LiveListenerBus(conf: SparkConf) { + + import LiveListenerBus._ + + private var sparkContext: SparkContext = _ + + private[spark] val metrics = new LiveListenerBusMetrics(conf) + + // Indicate if `start()` is called + private val started = new AtomicBoolean(false) + // Indicate if `stop()` is called + private val stopped = new AtomicBoolean(false) + + /** A counter for dropped events. It will be reset every time we log it. */ + private val droppedEventsCounter = new AtomicLong(0L) + + /** When `droppedEventsCounter` was logged last time in milliseconds. */ + @volatile private var lastReportTimestamp = 0L + + private val queues = new CopyOnWriteArrayList[AsyncEventQueue]() + + /** Add a listener to queue shared by all non-internal listeners. */ + def addToSharedQueue(listener: SparkListenerInterface): Unit = { + addToQueue(listener, SHARED_QUEUE) + } + + /** Add a listener to the executor management queue. */ + def addToManagementQueue(listener: SparkListenerInterface): Unit = { + addToQueue(listener, EXECUTOR_MANAGEMENT_QUEUE) + } + + /** Add a listener to the application status queue. */ + def addToStatusQueue(listener: SparkListenerInterface): Unit = { + addToQueue(listener, APP_STATUS_QUEUE) + } + + /** Add a listener to the event log queue. */ + def addToEventLogQueue(listener: SparkListenerInterface): Unit = { + addToQueue(listener, EVENT_LOG_QUEUE) + } + + /** + * Add a listener to a specific queue, creating a new queue if needed. Queues are independent + * of each other (each one uses a separate thread for delivering events), allowing slower + * listeners to be somewhat isolated from others. + */ + private def addToQueue(listener: SparkListenerInterface, queue: String): Unit = synchronized { + if (stopped.get()) { + throw new IllegalStateException("LiveListenerBus is stopped.") + } + + queues.asScala.find(_.name == queue) match { + case Some(queue) => + queue.addListener(listener) + + case None => + val newQueue = new AsyncEventQueue(queue, conf, metrics) + newQueue.addListener(listener) + if (started.get()) { + newQueue.start(sparkContext) + } + queues.add(newQueue) + } + } + + def removeListener(listener: SparkListenerInterface): Unit = synchronized { + // Remove listener from all queues it was added to, and stop queues that have become empty. + queues.asScala + .filter { queue => + queue.removeListener(listener) + queue.listeners.isEmpty() + } + .foreach { toRemove => + if (started.get() && !stopped.get()) { + toRemove.stop() + } + queues.remove(toRemove) + } + } + + /** Post an event to all queues. */ + def post(event: SparkListenerEvent): Unit = { + if (!stopped.get()) { + metrics.numEventsPosted.inc() + val it = queues.iterator() + while (it.hasNext()) { + it.next().post(event) + } + } + } + + /** + * Start sending events to attached listeners. + * + * This first sends out all buffered events posted before this listener bus has started, then + * listens for any additional events asynchronously while the listener bus is still running. + * This should only be called once. + * + * @param sc Used to stop the SparkContext in case the listener thread dies. + */ + def start(sc: SparkContext, metricsSystem: MetricsSystem): Unit = synchronized { + if (!started.compareAndSet(false, true)) { + throw new IllegalStateException("LiveListenerBus already started.") + } + + this.sparkContext = sc + queues.asScala.foreach(_.start(sc)) + metricsSystem.registerSource(metrics) + } + + /** + * For testing only. Wait until there are no more events in the queue, or until the specified + * time has elapsed. Throw `TimeoutException` if the specified time elapsed before the queue + * emptied. + * Exposed for testing. + */ + @throws(classOf[TimeoutException]) + def waitUntilEmpty(timeoutMillis: Long): Unit = { + val deadline = System.currentTimeMillis + timeoutMillis + queues.asScala.foreach { queue => + if (!queue.waitUntilEmpty(deadline)) { + throw new TimeoutException(s"The event queue is not empty after $timeoutMillis ms.") + } + } + } + + /** + * Stop the listener bus. It will wait until the queued events have been processed, but drop the + * new events after stopping. + */ + def stop(): Unit = { + if (!started.get()) { + throw new IllegalStateException(s"Attempted to stop bus that has not yet started!") + } + + if (!stopped.compareAndSet(false, true)) { + return + } + + synchronized { + queues.asScala.foreach(_.stop()) + queues.clear() + } + } + + // For testing only. + private[spark] def findListenersByClass[T <: SparkListenerInterface : ClassTag](): Seq[T] = { + queues.asScala.flatMap { queue => queue.findListenersByClass[T]() } + } + + // For testing only. + private[spark] def listeners: JList[SparkListenerInterface] = { + queues.asScala.flatMap(_.listeners.asScala).asJava + } + + // For testing only. + private[scheduler] def activeQueues(): Set[String] = { + queues.asScala.map(_.name).toSet + } + +} + +private[spark] object LiveListenerBus { + // Allows for Context to check whether stop() call is made within listener thread + val withinListenerThread: DynamicVariable[Boolean] = new DynamicVariable[Boolean](false) + + private[scheduler] val SHARED_QUEUE = "shared" + + private[scheduler] val APP_STATUS_QUEUE = "appStatus" + + private[scheduler] val EXECUTOR_MANAGEMENT_QUEUE = "executorManagement" + + private[scheduler] val EVENT_LOG_QUEUE = "eventLog" +} + +private[spark] class LiveListenerBusMetrics(conf: SparkConf) + extends Source with Logging { + + override val sourceName: String = "LiveListenerBus" + override val metricRegistry: MetricRegistry = new MetricRegistry + + /** + * The total number of events posted to the LiveListenerBus. This is a count of the total number + * of events which have been produced by the application and sent to the listener bus, NOT a + * count of the number of events which have been processed and delivered to listeners (or dropped + * without being delivered). + */ + val numEventsPosted: Counter = metricRegistry.counter(MetricRegistry.name("numEventsPosted")) + + // Guarded by synchronization. + private val perListenerClassTimers = mutable.Map[String, Timer]() + + /** + * Returns a timer tracking the processing time of the given listener class. + * events processed by that listener. This method is thread-safe. + */ + def getTimerForListenerClass(cls: Class[_ <: SparkListenerInterface]): Option[Timer] = { + synchronized { + val className = cls.getName + val maxTimed = conf.get(LISTENER_BUS_METRICS_MAX_LISTENER_CLASSES_TIMED) + perListenerClassTimers.get(className).orElse { + if (perListenerClassTimers.size == maxTimed) { + logError(s"Not measuring processing time for listener class $className because a " + + s"maximum of $maxTimed listener classes are already timed.") + None + } else { + perListenerClassTimers(className) = + metricRegistry.timer(MetricRegistry.name("listenerProcessingTime", className)) + perListenerClassTimers.get(className) + } + } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 180c8d1827e1..5e45b375ddd4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -19,8 +19,14 @@ package org.apache.spark.scheduler import java.io.{Externalizable, ObjectInput, ObjectOutput} +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.roaringbitmap.RoaringBitmap + +import org.apache.spark.SparkEnv +import org.apache.spark.internal.config import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.collection.BitSet import org.apache.spark.util.Utils /** @@ -120,35 +126,41 @@ private[spark] class CompressedMapStatus( } /** - * A [[MapStatus]] implementation that only stores the average size of non-empty blocks, - * plus a bitmap for tracking which blocks are empty. During serialization, this bitmap - * is compressed. + * A [[MapStatus]] implementation that stores the accurate size of huge blocks, which are larger + * than spark.shuffle.accurateBlockThreshold. It stores the average size of other non-empty blocks, + * plus a bitmap for tracking which blocks are empty. * * @param loc location where the task is being executed * @param numNonEmptyBlocks the number of non-empty blocks * @param emptyBlocks a bitmap tracking which blocks are empty - * @param avgSize average size of the non-empty blocks + * @param avgSize average size of the non-empty and non-huge blocks + * @param hugeBlockSizes sizes of huge blocks by their reduceId. */ private[spark] class HighlyCompressedMapStatus private ( private[this] var loc: BlockManagerId, private[this] var numNonEmptyBlocks: Int, - private[this] var emptyBlocks: BitSet, - private[this] var avgSize: Long) + private[this] var emptyBlocks: RoaringBitmap, + private[this] var avgSize: Long, + private var hugeBlockSizes: Map[Int, Byte]) extends MapStatus with Externalizable { // loc could be null when the default constructor is called during deserialization - require(loc == null || avgSize > 0 || numNonEmptyBlocks == 0, + require(loc == null || avgSize > 0 || hugeBlockSizes.size > 0 || numNonEmptyBlocks == 0, "Average size can only be zero for map stages that produced no output") - protected def this() = this(null, -1, null, -1) // For deserialization only + protected def this() = this(null, -1, null, -1, null) // For deserialization only override def location: BlockManagerId = loc override def getSizeForBlock(reduceId: Int): Long = { - if (emptyBlocks.get(reduceId)) { + assert(hugeBlockSizes != null) + if (emptyBlocks.contains(reduceId)) { 0 } else { - avgSize + hugeBlockSizes.get(reduceId) match { + case Some(size) => MapStatus.decompressSize(size) + case None => avgSize + } } } @@ -156,13 +168,26 @@ private[spark] class HighlyCompressedMapStatus private ( loc.writeExternal(out) emptyBlocks.writeExternal(out) out.writeLong(avgSize) + out.writeInt(hugeBlockSizes.size) + hugeBlockSizes.foreach { kv => + out.writeInt(kv._1) + out.writeByte(kv._2) + } } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { loc = BlockManagerId(in) - emptyBlocks = new BitSet + emptyBlocks = new RoaringBitmap() emptyBlocks.readExternal(in) avgSize = in.readLong() + val count = in.readInt() + val hugeBlockSizesArray = mutable.ArrayBuffer[Tuple2[Int, Byte]]() + (0 until count).foreach { _ => + val block = in.readInt() + val size = in.readByte() + hugeBlockSizesArray += Tuple2(block, size) + } + hugeBlockSizes = hugeBlockSizesArray.toMap } } @@ -176,15 +201,25 @@ private[spark] object HighlyCompressedMapStatus { // From a compression standpoint, it shouldn't matter whether we track empty or non-empty // blocks. From a performance standpoint, we benefit from tracking empty blocks because // we expect that there will be far fewer of them, so we will perform fewer bitmap insertions. + val emptyBlocks = new RoaringBitmap() val totalNumBlocks = uncompressedSizes.length - val emptyBlocks = new BitSet(totalNumBlocks) + val threshold = Option(SparkEnv.get) + .map(_.conf.get(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD)) + .getOrElse(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD.defaultValue.get) + val hugeBlockSizesArray = ArrayBuffer[Tuple2[Int, Byte]]() while (i < totalNumBlocks) { - var size = uncompressedSizes(i) + val size = uncompressedSizes(i) if (size > 0) { numNonEmptyBlocks += 1 - totalSize += size + // Huge blocks are not included in the calculation for average size, thus size for smaller + // blocks is more accurate. + if (size < threshold) { + totalSize += size + } else { + hugeBlockSizesArray += Tuple2(i, MapStatus.compressSize(uncompressedSizes(i))) + } } else { - emptyBlocks.set(i) + emptyBlocks.add(i) } i += 1 } @@ -193,6 +228,9 @@ private[spark] object HighlyCompressedMapStatus { } else { 0 } - new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize) + emptyBlocks.trim() + emptyBlocks.runOptimize() + new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize, + hugeBlockSizesArray.toMap) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala index 4d146678174f..83d87b548a43 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala @@ -20,7 +20,9 @@ package org.apache.spark.scheduler import scala.collection.mutable import org.apache.spark._ -import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, RpcEndpoint} +import org.apache.spark.internal.Logging +import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} +import org.apache.spark.util.{RpcUtils, ThreadUtils} private sealed trait OutputCommitCoordinationMessage extends Serializable @@ -46,25 +48,29 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean) private type StageId = Int private type PartitionId = Int private type TaskAttemptNumber = Int - private val NO_AUTHORIZED_COMMITTER: TaskAttemptNumber = -1 + private case class StageState(numPartitions: Int) { + val authorizedCommitters = Array.fill[TaskAttemptNumber](numPartitions)(NO_AUTHORIZED_COMMITTER) + val failures = mutable.Map[PartitionId, mutable.Set[TaskAttemptNumber]]() + } /** - * Map from active stages's id => partition id => task attempt with exclusive lock on committing - * output for that partition. + * Map from active stages's id => authorized task attempts for each partition id, which hold an + * exclusive lock on committing task output for that partition, as well as any known failed + * attempts in the stage. * * 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 = mutable.Map[StageId, Array[TaskAttemptNumber]]() + private val stageStates = mutable.Map[StageId, StageState]() /** * Returns whether the OutputCommitCoordinator's internal data structures are all empty. */ def isEmpty: Boolean = { - authorizedCommittersByStage.isEmpty + stageStates.isEmpty } /** @@ -87,7 +93,8 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean) val msg = AskPermissionToCommitOutput(stage, partition, attemptNumber) coordinatorRef match { case Some(endpointRef) => - endpointRef.askWithRetry[Boolean](msg) + ThreadUtils.awaitResult(endpointRef.ask[Boolean](msg), + RpcUtils.askRpcTimeout(conf).duration) case None => logError( "canCommit called after coordinator was stopped (is SparkEnv shutdown in progress)?") @@ -102,19 +109,13 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean) * @param maxPartitionId the maximum partition id that could appear in this stage's tasks (i.e. * the maximum possible value of `context.partitionId`). */ - private[scheduler] def stageStart( - stage: StageId, - maxPartitionId: Int): Unit = { - val arr = new Array[TaskAttemptNumber](maxPartitionId + 1) - java.util.Arrays.fill(arr, NO_AUTHORIZED_COMMITTER) - synchronized { - authorizedCommittersByStage(stage) = arr - } + private[scheduler] def stageStart(stage: StageId, maxPartitionId: Int): Unit = synchronized { + stageStates(stage) = new StageState(maxPartitionId + 1) } // Called by DAGScheduler private[scheduler] def stageEnd(stage: StageId): Unit = synchronized { - authorizedCommittersByStage.remove(stage) + stageStates.remove(stage) } // Called by DAGScheduler @@ -123,7 +124,7 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean) partition: PartitionId, attemptNumber: TaskAttemptNumber, reason: TaskEndReason): Unit = synchronized { - val authorizedCommitters = authorizedCommittersByStage.getOrElse(stage, { + val stageState = stageStates.getOrElse(stage, { logDebug(s"Ignoring task completion for completed stage") return }) @@ -134,10 +135,12 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean) logInfo(s"Task was denied committing, stage: $stage, partition: $partition, " + s"attempt: $attemptNumber") case otherReason => - if (authorizedCommitters(partition) == attemptNumber) { + // Mark the attempt as failed to blacklist from future commit protocol + stageState.failures.getOrElseUpdate(partition, mutable.Set()) += attemptNumber + if (stageState.authorizedCommitters(partition) == attemptNumber) { logDebug(s"Authorized committer (attemptNumber=$attemptNumber, stage=$stage, " + s"partition=$partition) failed; clearing lock") - authorizedCommitters(partition) = NO_AUTHORIZED_COMMITTER + stageState.authorizedCommitters(partition) = NO_AUTHORIZED_COMMITTER } } } @@ -146,7 +149,7 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean) if (isDriver) { coordinatorRef.foreach(_ send StopCoordinator) coordinatorRef = None - authorizedCommittersByStage.clear() + stageStates.clear() } } @@ -155,25 +158,45 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean) stage: StageId, partition: PartitionId, attemptNumber: TaskAttemptNumber): Boolean = synchronized { - authorizedCommittersByStage.get(stage) match { - case Some(authorizedCommitters) => - authorizedCommitters(partition) match { + stageStates.get(stage) match { + case Some(state) if attemptFailed(state, partition, attemptNumber) => + logInfo(s"Denying attemptNumber=$attemptNumber to commit for stage=$stage," + + s" partition=$partition as task attempt $attemptNumber has already failed.") + false + case Some(state) => + state.authorizedCommitters(partition) match { case NO_AUTHORIZED_COMMITTER => logDebug(s"Authorizing attemptNumber=$attemptNumber to commit for stage=$stage, " + s"partition=$partition") - authorizedCommitters(partition) = attemptNumber + state.authorizedCommitters(partition) = attemptNumber true case existingCommitter => - logDebug(s"Denying attemptNumber=$attemptNumber to commit for stage=$stage, " + - s"partition=$partition; existingCommitter = $existingCommitter") - false + // Coordinator should be idempotent when receiving AskPermissionToCommit. + if (existingCommitter == attemptNumber) { + logWarning(s"Authorizing duplicate request to commit for " + + s"attemptNumber=$attemptNumber to commit for stage=$stage," + + s" partition=$partition; existingCommitter = $existingCommitter." + + s" This can indicate dropped network traffic.") + true + } else { + logDebug(s"Denying attemptNumber=$attemptNumber to commit for stage=$stage, " + + s"partition=$partition; existingCommitter = $existingCommitter") + false + } } case None => - logDebug(s"Stage $stage has completed, so not allowing attempt number $attemptNumber of" + - s"partition $partition to commit") + logDebug(s"Stage $stage has completed, so not allowing" + + s" attempt number $attemptNumber of partition $partition to commit") false } } + + private def attemptFailed( + stageState: StageState, + partition: PartitionId, + attempt: TaskAttemptNumber): Boolean = synchronized { + stageState.failures.get(partition).exists(_.contains(attempt)) + } } private[spark] object OutputCommitCoordinator { @@ -183,6 +206,8 @@ private[spark] object OutputCommitCoordinator { override val rpcEnv: RpcEnv, outputCommitCoordinator: OutputCommitCoordinator) extends RpcEndpoint with Logging { + logDebug("init") // force eager creation of logger + override def receive: PartialFunction[Any, Unit] = { case StopCoordinator => logInfo("OutputCommitCoordinator stopped!") diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 551e39a81b69..f4b0ab10155a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -22,39 +22,40 @@ import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode /** - * An Schedulable entity that represent collection of Pools or TaskSetManagers + * A Schedulable entity that represents collection of Pools or TaskSetManagers */ - private[spark] class Pool( val poolName: String, val schedulingMode: SchedulingMode, initMinShare: Int, initWeight: Int) - extends Schedulable - with Logging { + extends Schedulable with Logging { val schedulableQueue = new ConcurrentLinkedQueue[Schedulable] val schedulableNameToSchedulable = new ConcurrentHashMap[String, Schedulable] - var weight = initWeight - var minShare = initMinShare + val weight = initWeight + val minShare = initMinShare var runningTasks = 0 - var priority = 0 + val priority = 0 // A pool's stage id is used to break the tie in scheduling. var stageId = -1 - var name = poolName + val name = poolName var parent: Pool = null - var taskSetSchedulingAlgorithm: SchedulingAlgorithm = { + private val taskSetSchedulingAlgorithm: SchedulingAlgorithm = { schedulingMode match { case SchedulingMode.FAIR => new FairSchedulingAlgorithm() case SchedulingMode.FIFO => new FIFOSchedulingAlgorithm() + case _ => + val msg = s"Unsupported scheduling mode: $schedulingMode. Use FAIR or FIFO instead." + throw new IllegalArgumentException(msg) } } @@ -87,16 +88,16 @@ private[spark] class Pool( schedulableQueue.asScala.foreach(_.executorLost(executorId, host, reason)) } - override def checkSpeculatableTasks(): Boolean = { + override def checkSpeculatableTasks(minTimeToSpeculation: Int): Boolean = { var shouldRevive = false for (schedulable <- schedulableQueue.asScala) { - shouldRevive |= schedulable.checkSpeculatableTasks() + shouldRevive |= schedulable.checkSpeculatableTasks(minTimeToSpeculation) } shouldRevive } override def getSortedTaskSetQueue: ArrayBuffer[TaskSetManager] = { - var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] + val sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] val sortedSchedulableQueue = schedulableQueue.asScala.toSeq.sortWith(taskSetSchedulingAlgorithm.comparator) for (schedulable <- sortedSchedulableQueue) { 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 c6d957b65f3f..26a6a3effc9a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -17,14 +17,16 @@ package org.apache.spark.scheduler -import java.io.{InputStream, IOException} +import java.io.{EOFException, InputStream, IOException} import scala.io.Source import com.fasterxml.jackson.core.JsonParseException +import com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException import org.json4s.jackson.JsonMethods._ -import org.apache.spark.Logging +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.ReplayListenerBus._ import org.apache.spark.util.JsonProtocol /** @@ -43,32 +45,69 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { * @param sourceName Filename (or other source identifier) from whence @logData is being read * @param maybeTruncated Indicate whether log file might be truncated (some abnormal situations * encountered, log file might not finished writing) or not + * @param eventsFilter Filter function to select JSON event strings in the log data stream that + * should be parsed and replayed. When not specified, all event strings in the log data + * are parsed and replayed. */ def replay( logData: InputStream, sourceName: String, - maybeTruncated: Boolean = false): Unit = { + maybeTruncated: Boolean = false, + eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): Unit = { + val lines = Source.fromInputStream(logData).getLines() + replay(lines, sourceName, maybeTruncated, eventsFilter) + } + + /** + * Overloaded variant of [[replay()]] which accepts an iterator of lines instead of an + * [[InputStream]]. Exposed for use by custom ApplicationHistoryProvider implementations. + */ + def replay( + lines: Iterator[String], + sourceName: String, + maybeTruncated: Boolean, + eventsFilter: ReplayEventsFilter): Unit = { var currentLine: String = null - var lineNumber: Int = 1 + var lineNumber: Int = 0 + try { - val lines = Source.fromInputStream(logData).getLines() - while (lines.hasNext) { - currentLine = lines.next() + val lineEntries = lines + .zipWithIndex + .filter { case (line, _) => eventsFilter(line) } + + while (lineEntries.hasNext) { try { + val entry = lineEntries.next() + + currentLine = entry._1 + lineNumber = entry._2 + 1 + postToAll(JsonProtocol.sparkEventFromJson(parse(currentLine))) } catch { + case e: ClassNotFoundException if KNOWN_REMOVED_CLASSES.contains(e.getMessage) => + // Ignore events generated by Structured Streaming in Spark 2.0.0 and 2.0.1. + // It's safe since no place uses them. + logWarning(s"Dropped incompatible Structured Streaming log: $currentLine") + case e: UnrecognizedPropertyException if e.getMessage != null && e.getMessage.startsWith( + "Unrecognized field \"queryStatus\" " + + "(class org.apache.spark.sql.streaming.StreamingQueryListener$") => + // Ignore events generated by Structured Streaming in Spark 2.0.2 + // It's safe since no place uses them. + logWarning(s"Dropped incompatible Structured Streaming log: $currentLine") case jpe: JsonParseException => // We can only ignore exception from last line of the file that might be truncated - if (!maybeTruncated || lines.hasNext) { + // the last entry may not be the very last line in the event log, but we treat it + // as such in a best effort to replay the given input + if (!maybeTruncated || lineEntries.hasNext) { throw jpe } else { logWarning(s"Got JsonParseException from log file $sourceName" + s" at line $lineNumber, the file might not have finished writing cleanly.") } } - lineNumber += 1 } } catch { + case _: EOFException if maybeTruncated => case ioe: IOException => throw ioe case e: Exception => @@ -78,3 +117,21 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { } } + + +private[spark] object ReplayListenerBus { + + type ReplayEventsFilter = (String) => Boolean + + // utility filter that selects all event logs during replay + val SELECT_ALL_FILTER: ReplayEventsFilter = { (eventString: String) => true } + + /** + * Classes that were removed. Structured Streaming doesn't use them any more. However, parsing + * old json may fail and we can just ignore these failures. + */ + val KNOWN_REMOVED_CLASSES = Set( + "org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgress", + "org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated" + ) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index fb693721a9cb..e36c759a4255 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -17,9 +17,10 @@ package org.apache.spark.scheduler -import java.nio.ByteBuffer - import java.io._ +import java.lang.management.ManagementFactory +import java.nio.ByteBuffer +import java.util.Properties import org.apache.spark._ import org.apache.spark.broadcast.Broadcast @@ -31,6 +32,7 @@ import org.apache.spark.rdd.RDD * See [[Task]] for more information. * * @param stageId id of the stage this task belongs to + * @param stageAttemptId attempt id of the stage this task belongs to * @param taskBinary broadcasted version of the serialized RDD and the function to apply on each * partition of the given RDD. Once deserialized, the type should be * (RDD[T], (TaskContext, Iterator[T]) => U). @@ -38,7 +40,15 @@ import org.apache.spark.rdd.RDD * @param locs preferred task execution locations for locality scheduling * @param outputId index of the task in this job (a job can launch tasks on only a subset of the * input RDD's partitions). - */ + * @param localProperties copy of thread-local properties set by the user on the driver side. + * @param serializedTaskMetrics a `TaskMetrics` that is created and serialized on the driver side + * and sent to executor side. + * + * The parameters below are optional: + * @param jobId id of the job this task belongs to + * @param appId id of the app this task belongs to + * @param appAttemptId attempt id of the app this task belongs to + */ private[spark] class ResultTask[T, U]( stageId: Int, stageAttemptId: Int, @@ -46,8 +56,13 @@ private[spark] class ResultTask[T, U]( partition: Partition, locs: Seq[TaskLocation], val outputId: Int, - internalAccumulators: Seq[Accumulator[Long]]) - extends Task[U](stageId, stageAttemptId, partition.index, internalAccumulators) + localProperties: Properties, + serializedTaskMetrics: Array[Byte], + jobId: Option[Int] = None, + appId: Option[String] = None, + appAttemptId: Option[String] = None) + extends Task[U](stageId, stageAttemptId, partition.index, localProperties, serializedTaskMetrics, + jobId, appId, appAttemptId) with Serializable { @transient private[this] val preferredLocs: Seq[TaskLocation] = { @@ -56,13 +71,19 @@ private[spark] class ResultTask[T, U]( override def runTask(context: TaskContext): U = { // Deserialize the RDD and the func using the broadcast variables. + val threadMXBean = ManagementFactory.getThreadMXBean val deserializeStartTime = System.currentTimeMillis() + val deserializeStartCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { + threadMXBean.getCurrentThreadCpuTime + } else 0L val ser = SparkEnv.get.closureSerializer.newInstance() val (rdd, func) = ser.deserialize[(RDD[T], (TaskContext, Iterator[T]) => U)]( ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) _executorDeserializeTime = System.currentTimeMillis() - deserializeStartTime + _executorDeserializeCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { + threadMXBean.getCurrentThreadCpuTime - deserializeStartCpuTime + } else 0L - metrics = Some(context.taskMetrics) func(context, rdd.iterator(partition, context)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala index ab00bc8f0bf4..b6f88ed0a93a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala @@ -43,6 +43,6 @@ private[spark] trait Schedulable { def removeSchedulable(schedulable: Schedulable): Unit def getSchedulableByName(name: String): Schedulable def executorLost(executorId: String, host: String, reason: ExecutorLossReason): Unit - def checkSpeculatableTasks(): Boolean + def checkSpeculatableTasks(minTimeToSpeculation: Int): Boolean def getSortedTaskSetQueue: ArrayBuffer[TaskSetManager] } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index 6c5827f75e63..5f3c280ec31e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -18,11 +18,14 @@ package org.apache.spark.scheduler import java.io.{FileInputStream, InputStream} -import java.util.{NoSuchElementException, Properties} +import java.util.{Locale, NoSuchElementException, Properties} -import scala.xml.XML +import scala.util.control.NonFatal +import scala.xml.{Node, XML} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.util.Utils /** @@ -33,9 +36,9 @@ import org.apache.spark.util.Utils private[spark] trait SchedulableBuilder { def rootPool: Pool - def buildPools() + def buildPools(): Unit - def addTaskSetManager(manager: Schedulable, properties: Properties) + def addTaskSetManager(manager: Schedulable, properties: Properties): Unit } private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) @@ -53,7 +56,8 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) extends SchedulableBuilder with Logging { - val schedulerAllocFile = conf.getOption("spark.scheduler.allocation.file") + val SCHEDULER_ALLOCATION_FILE_PROPERTY = "spark.scheduler.allocation.file" + val schedulerAllocFile = conf.getOption(SCHEDULER_ALLOCATION_FILE_PROPERTY) val DEFAULT_SCHEDULER_FILE = "fairscheduler.xml" val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.pool" val DEFAULT_POOL_NAME = "default" @@ -67,19 +71,35 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) val DEFAULT_WEIGHT = 1 override def buildPools() { - var is: Option[InputStream] = None + var fileData: Option[(InputStream, String)] = None try { - is = Option { - schedulerAllocFile.map { f => - new FileInputStream(f) - }.getOrElse { - Utils.getSparkClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE) + fileData = schedulerAllocFile.map { f => + val fis = new FileInputStream(f) + logInfo(s"Creating Fair Scheduler pools from $f") + Some((fis, f)) + }.getOrElse { + val is = Utils.getSparkClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE) + if (is != null) { + logInfo(s"Creating Fair Scheduler pools from default file: $DEFAULT_SCHEDULER_FILE") + Some((is, DEFAULT_SCHEDULER_FILE)) + } else { + logWarning("Fair Scheduler configuration file not found so jobs will be scheduled in " + + s"FIFO order. To use fair scheduling, configure pools in $DEFAULT_SCHEDULER_FILE or " + + s"set $SCHEDULER_ALLOCATION_FILE_PROPERTY to a file that contains the configuration.") + None } } - is.foreach { i => buildFairSchedulerPool(i) } + fileData.foreach { case (is, fileName) => buildFairSchedulerPool(is, fileName) } + } catch { + case NonFatal(t) => + val defaultMessage = "Error while building the fair scheduler pools" + val message = fileData.map { case (is, fileName) => s"$defaultMessage from $fileName" } + .getOrElse(defaultMessage) + logError(message, t) + throw t } finally { - is.foreach(_.close()) + fileData.foreach { case (is, fileName) => is.close() } } // finally create "default" pool @@ -91,62 +111,93 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) val pool = new Pool(DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) rootPool.addSchedulable(pool) - logInfo("Created default pool %s, schedulingMode: %s, minShare: %d, weight: %d".format( + logInfo("Created default pool: %s, schedulingMode: %s, minShare: %d, weight: %d".format( DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) } } - private def buildFairSchedulerPool(is: InputStream) { + private def buildFairSchedulerPool(is: InputStream, fileName: String) { val xml = XML.load(is) for (poolNode <- (xml \\ POOLS_PROPERTY)) { val poolName = (poolNode \ POOL_NAME_PROPERTY).text - var schedulingMode = DEFAULT_SCHEDULING_MODE - var minShare = DEFAULT_MINIMUM_SHARE - var weight = DEFAULT_WEIGHT - - val xmlSchedulingMode = (poolNode \ SCHEDULING_MODE_PROPERTY).text - if (xmlSchedulingMode != "") { - try { - schedulingMode = SchedulingMode.withName(xmlSchedulingMode) - } catch { - case e: NoSuchElementException => - logWarning("Error xml schedulingMode, using default schedulingMode") - } - } - val xmlMinShare = (poolNode \ MINIMUM_SHARES_PROPERTY).text - if (xmlMinShare != "") { - minShare = xmlMinShare.toInt - } + val schedulingMode = getSchedulingModeValue(poolNode, poolName, + DEFAULT_SCHEDULING_MODE, fileName) + val minShare = getIntValue(poolNode, poolName, MINIMUM_SHARES_PROPERTY, + DEFAULT_MINIMUM_SHARE, fileName) + val weight = getIntValue(poolNode, poolName, WEIGHT_PROPERTY, + DEFAULT_WEIGHT, fileName) - val xmlWeight = (poolNode \ WEIGHT_PROPERTY).text - if (xmlWeight != "") { - weight = xmlWeight.toInt - } + rootPool.addSchedulable(new Pool(poolName, schedulingMode, minShare, weight)) - val pool = new Pool(poolName, schedulingMode, minShare, weight) - rootPool.addSchedulable(pool) - logInfo("Created pool %s, schedulingMode: %s, minShare: %d, weight: %d".format( + logInfo("Created pool: %s, schedulingMode: %s, minShare: %d, weight: %d".format( poolName, schedulingMode, minShare, weight)) } } + private def getSchedulingModeValue( + poolNode: Node, + poolName: String, + defaultValue: SchedulingMode, + fileName: String): SchedulingMode = { + + val xmlSchedulingMode = + (poolNode \ SCHEDULING_MODE_PROPERTY).text.trim.toUpperCase(Locale.ROOT) + val warningMessage = s"Unsupported schedulingMode: $xmlSchedulingMode found in " + + s"Fair Scheduler configuration file: $fileName, using " + + s"the default schedulingMode: $defaultValue for pool: $poolName" + try { + if (SchedulingMode.withName(xmlSchedulingMode) != SchedulingMode.NONE) { + SchedulingMode.withName(xmlSchedulingMode) + } else { + logWarning(warningMessage) + defaultValue + } + } catch { + case e: NoSuchElementException => + logWarning(warningMessage) + defaultValue + } + } + + private def getIntValue( + poolNode: Node, + poolName: String, + propertyName: String, + defaultValue: Int, + fileName: String): Int = { + + val data = (poolNode \ propertyName).text.trim + try { + data.toInt + } catch { + case e: NumberFormatException => + logWarning(s"Error while loading fair scheduler configuration from $fileName: " + + s"$propertyName is blank or invalid: $data, using the default $propertyName: " + + s"$defaultValue for pool: $poolName") + defaultValue + } + } + override def addTaskSetManager(manager: Schedulable, properties: Properties) { - var poolName = DEFAULT_POOL_NAME - var parentPool = rootPool.getSchedulableByName(poolName) - if (properties != null) { - poolName = properties.getProperty(FAIR_SCHEDULER_PROPERTIES, DEFAULT_POOL_NAME) - parentPool = rootPool.getSchedulableByName(poolName) - if (parentPool == null) { - // we will create a new pool that user has configured in app - // instead of being defined in xml file - parentPool = new Pool(poolName, DEFAULT_SCHEDULING_MODE, - DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) - rootPool.addSchedulable(parentPool) - logInfo("Created pool %s, schedulingMode: %s, minShare: %d, weight: %d".format( - poolName, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) + val poolName = if (properties != null) { + properties.getProperty(FAIR_SCHEDULER_PROPERTIES, DEFAULT_POOL_NAME) + } else { + DEFAULT_POOL_NAME } + var parentPool = rootPool.getSchedulableByName(poolName) + if (parentPool == null) { + // we will create a new pool that user has configured in app + // instead of being defined in xml file + parentPool = new Pool(poolName, DEFAULT_SCHEDULING_MODE, + DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) + rootPool.addSchedulable(parentPool) + logWarning(s"A job was submitted with scheduler pool $poolName, which has not been " + + "configured. This can happen when the file that pools are read from isn't set, or " + + s"when that file doesn't contain $poolName. Created $poolName with default " + + s"configuration (schedulingMode: $DEFAULT_SCHEDULING_MODE, " + + s"minShare: $DEFAULT_MINIMUM_SHARE, weight: $DEFAULT_WEIGHT)") } parentPool.addSchedulable(manager) logInfo("Added task set " + manager.name + " tasks to pool " + poolName) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 8801a761afae..22db3350abfa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -30,8 +30,21 @@ private[spark] trait SchedulerBackend { def reviveOffers(): Unit def defaultParallelism(): Int - def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = + /** + * Requests that an executor kills a running task. + * + * @param taskId Id of the task. + * @param executorId Id of the executor the task is running on. + * @param interruptThread Whether the executor should interrupt the task thread. + * @param reason The reason for the task kill. + */ + def killTask( + taskId: Long, + executorId: String, + interruptThread: Boolean, + reason: String): Unit = throw new UnsupportedOperationException + def isReady(): Boolean = true /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala index 864941d468af..18ebbbe78a5b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala @@ -36,11 +36,7 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { val stageId2 = s2.stageId res = math.signum(stageId1 - stageId2) } - if (res < 0) { - true - } else { - false - } + res < 0 } } @@ -52,12 +48,12 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { val runningTasks2 = s2.runningTasks val s1Needy = runningTasks1 < minShare1 val s2Needy = runningTasks2 < minShare2 - val minShareRatio1 = runningTasks1.toDouble / math.max(minShare1, 1.0).toDouble - val minShareRatio2 = runningTasks2.toDouble / math.max(minShare2, 1.0).toDouble + val minShareRatio1 = runningTasks1.toDouble / math.max(minShare1, 1.0) + val minShareRatio2 = runningTasks2.toDouble / math.max(minShare2, 1.0) val taskToWeightRatio1 = runningTasks1.toDouble / s1.weight.toDouble val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble - var compare: Int = 0 + var compare = 0 if (s1Needy && !s2Needy) { return true } else if (!s1Needy && s2Needy) { @@ -67,7 +63,6 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { } else { compare = taskToWeightRatio1.compareTo(taskToWeightRatio2) } - if (compare < 0) { true } else if (compare > 0) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index 51416e5ce97f..1b44d0aee319 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -17,9 +17,10 @@ package org.apache.spark.scheduler -import org.apache.spark.ShuffleDependency +import scala.collection.mutable.HashSet + +import org.apache.spark.{MapOutputTrackerMaster, ShuffleDependency} import org.apache.spark.rdd.RDD -import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.CallSite /** @@ -40,19 +41,22 @@ private[spark] class ShuffleMapStage( parents: List[Stage], firstJobId: Int, callSite: CallSite, - val shuffleDep: ShuffleDependency[_, _, _]) + val shuffleDep: ShuffleDependency[_, _, _], + mapOutputTrackerMaster: MapOutputTrackerMaster) extends Stage(id, rdd, numTasks, parents, firstJobId, callSite) { private[this] var _mapStageJobs: List[ActiveJob] = Nil - private[this] var _numAvailableOutputs: Int = 0 - /** - * List of [[MapStatus]] for each partition. The index of the array is the map partition id, - * and each value in the array is the list of possible [[MapStatus]] for a partition - * (a single task might run multiple times). + * Partitions that either haven't yet been computed, or that were computed on an executor + * that has since been lost, so should be re-computed. This variable is used by the + * DAGScheduler to determine when a stage has completed. Task successes in both the active + * attempt for the stage or in earlier attempts for this stage can cause paritition ids to get + * removed from pendingPartitions. As a result, this variable may be inconsistent with the pending + * tasks in the TaskSetManager for the active attempt for the stage (the partitions stored here + * will always be a subset of the partitions that the TaskSetManager thinks are pending). */ - private[this] val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) + val pendingPartitions = new HashSet[Int] override def toString: String = "ShuffleMapStage " + id @@ -75,69 +79,18 @@ private[spark] class ShuffleMapStage( /** * Number of partitions that have shuffle outputs. * When this reaches [[numPartitions]], this map stage is ready. - * This should be kept consistent as `outputLocs.filter(!_.isEmpty).size`. */ - def numAvailableOutputs: Int = _numAvailableOutputs + def numAvailableOutputs: Int = mapOutputTrackerMaster.getNumAvailableOutputs(shuffleDep.shuffleId) /** * Returns true if the map stage is ready, i.e. all partitions have shuffle outputs. - * This should be the same as `outputLocs.contains(Nil)`. */ - def isAvailable: Boolean = _numAvailableOutputs == numPartitions + def isAvailable: Boolean = numAvailableOutputs == numPartitions /** Returns the sequence of partition ids that are missing (i.e. needs to be computed). */ override def findMissingPartitions(): Seq[Int] = { - val missing = (0 until numPartitions).filter(id => outputLocs(id).isEmpty) - assert(missing.size == numPartitions - _numAvailableOutputs, - s"${missing.size} missing, expected ${numPartitions - _numAvailableOutputs}") - missing - } - - def addOutputLoc(partition: Int, status: MapStatus): Unit = { - val prevList = outputLocs(partition) - outputLocs(partition) = status :: prevList - if (prevList == Nil) { - _numAvailableOutputs += 1 - } - } - - def removeOutputLoc(partition: Int, bmAddress: BlockManagerId): Unit = { - val prevList = outputLocs(partition) - val newList = prevList.filterNot(_.location == bmAddress) - outputLocs(partition) = newList - if (prevList != Nil && newList == Nil) { - _numAvailableOutputs -= 1 - } - } - - /** - * Returns an array of [[MapStatus]] (index by partition id). For each partition, the returned - * value contains only one (i.e. the first) [[MapStatus]]. If there is no entry for the partition, - * that position is filled with null. - */ - def outputLocInMapOutputTrackerFormat(): Array[MapStatus] = { - outputLocs.map(_.headOption.orNull) - } - - /** - * Removes all shuffle outputs associated with this executor. Note that this will also remove - * outputs which are served by an external shuffle server (if one exists), as they are still - * registered with this execId. - */ - def removeOutputsOnExecutor(execId: String): Unit = { - var becameUnavailable = false - for (partition <- 0 until numPartitions) { - val prevList = outputLocs(partition) - val newList = prevList.filterNot(_.location.executorId == execId) - outputLocs(partition) = newList - if (prevList != Nil && newList == Nil) { - becameUnavailable = true - _numAvailableOutputs -= 1 - } - } - if (becameUnavailable) { - logInfo("%s is now unavailable on executor %s (%d/%d, %s)".format( - this, execId, _numAvailableOutputs, numPartitions, isAvailable)) - } + mapOutputTrackerMaster + .findMissingPartitions(shuffleDep.shuffleId) + .getOrElse(0 until numPartitions) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index f478f9982afe..7a25c47e2cab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -17,26 +17,38 @@ package org.apache.spark.scheduler +import java.lang.management.ManagementFactory import java.nio.ByteBuffer +import java.util.Properties import scala.language.existentials import org.apache.spark._ import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.shuffle.ShuffleWriter /** -* A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner -* specified in the ShuffleDependency). -* -* See [[org.apache.spark.scheduler.Task]] for more information. -* + * A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner + * specified in the ShuffleDependency). + * + * See [[org.apache.spark.scheduler.Task]] for more information. + * * @param stageId id of the stage this task belongs to + * @param stageAttemptId attempt id of the stage this task belongs to * @param taskBinary broadcast version of the RDD and the ShuffleDependency. Once deserialized, * the type should be (RDD[_], ShuffleDependency[_, _, _]). * @param partition partition of the RDD this task is associated with * @param locs preferred task execution locations for locality scheduling + * @param localProperties copy of thread-local properties set by the user on the driver side. + * @param serializedTaskMetrics a `TaskMetrics` that is created and serialized on the driver side + * and sent to executor side. + * + * The parameters below are optional: + * @param jobId id of the job this task belongs to + * @param appId id of the app this task belongs to + * @param appAttemptId attempt id of the app this task belongs to */ private[spark] class ShuffleMapTask( stageId: Int, @@ -44,13 +56,18 @@ private[spark] class ShuffleMapTask( taskBinary: Broadcast[Array[Byte]], partition: Partition, @transient private var locs: Seq[TaskLocation], - internalAccumulators: Seq[Accumulator[Long]]) - extends Task[MapStatus](stageId, stageAttemptId, partition.index, internalAccumulators) + localProperties: Properties, + serializedTaskMetrics: Array[Byte], + jobId: Option[Int] = None, + appId: Option[String] = None, + appAttemptId: Option[String] = None) + extends Task[MapStatus](stageId, stageAttemptId, partition.index, localProperties, + serializedTaskMetrics, jobId, appId, appAttemptId) with Logging { /** A constructor used only in test suites. This does not require passing in an RDD. */ def this(partitionId: Int) { - this(0, 0, null, new Partition { override def index: Int = 0 }, null, null) + this(0, 0, null, new Partition { override def index: Int = 0 }, null, new Properties, null) } @transient private val preferredLocs: Seq[TaskLocation] = { @@ -59,13 +76,19 @@ private[spark] class ShuffleMapTask( override def runTask(context: TaskContext): MapStatus = { // Deserialize the RDD using the broadcast variable. + val threadMXBean = ManagementFactory.getThreadMXBean val deserializeStartTime = System.currentTimeMillis() + val deserializeStartCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { + threadMXBean.getCurrentThreadCpuTime + } else 0L val ser = SparkEnv.get.closureSerializer.newInstance() val (rdd, dep) = ser.deserialize[(RDD[_], ShuffleDependency[_, _, _])]( ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) _executorDeserializeTime = System.currentTimeMillis() - deserializeStartTime + _executorDeserializeCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { + threadMXBean.getCurrentThreadCpuTime - deserializeStartCpuTime + } else 0L - metrics = Some(context.taskMetrics) var writer: ShuffleWriter[Any, Any] = null try { val manager = SparkEnv.get.shuffleManager diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 896f1743332f..b76e560669d5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -18,19 +18,25 @@ package org.apache.spark.scheduler import java.util.Properties +import javax.annotation.Nullable import scala.collection.Map -import scala.collection.mutable -import org.apache.spark.{Logging, TaskEndReason} +import com.fasterxml.jackson.annotation.JsonTypeInfo + +import org.apache.spark.{SparkConf, TaskEndReason} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.{BlockManagerId, BlockUpdatedInfo} -import org.apache.spark.util.{Distribution, Utils} +import org.apache.spark.ui.SparkUI @DeveloperApi -sealed trait SparkListenerEvent +@JsonTypeInfo(use = JsonTypeInfo.Id.CLASS, include = JsonTypeInfo.As.PROPERTY, property = "Event") +trait SparkListenerEvent { + /* Whether output this event to the event log */ + protected[spark] def logEvent: Boolean = true +} @DeveloperApi case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) @@ -46,6 +52,9 @@ case class SparkListenerTaskStart(stageId: Int, stageAttemptId: Int, taskInfo: T @DeveloperApi case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerSpeculativeTaskSubmitted(stageId: Int) extends SparkListenerEvent + @DeveloperApi case class SparkListenerTaskEnd( stageId: Int, @@ -53,7 +62,8 @@ case class SparkListenerTaskEnd( taskType: String, reason: TaskEndReason, taskInfo: TaskInfo, - taskMetrics: TaskMetrics) + // may be null if the task has failed + @Nullable taskMetrics: TaskMetrics) extends SparkListenerEvent @DeveloperApi @@ -80,8 +90,13 @@ case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(S extends SparkListenerEvent @DeveloperApi -case class SparkListenerBlockManagerAdded(time: Long, blockManagerId: BlockManagerId, maxMem: Long) - extends SparkListenerEvent +case class SparkListenerBlockManagerAdded( + time: Long, + blockManagerId: BlockManagerId, + maxMem: Long, + maxOnHeapMem: Option[Long] = None, + maxOffHeapMem: Option[Long] = None) extends SparkListenerEvent { +} @DeveloperApi case class SparkListenerBlockManagerRemoved(time: Long, blockManagerId: BlockManagerId) @@ -98,18 +113,40 @@ case class SparkListenerExecutorAdded(time: Long, executorId: String, executorIn case class SparkListenerExecutorRemoved(time: Long, executorId: String, reason: String) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerExecutorBlacklisted( + time: Long, + executorId: String, + taskFailures: Int) + extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerExecutorUnblacklisted(time: Long, executorId: String) + extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerNodeBlacklisted( + time: Long, + hostId: String, + executorFailures: Int) + extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerNodeUnblacklisted(time: Long, hostId: String) + extends SparkListenerEvent + @DeveloperApi case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends SparkListenerEvent /** * Periodic updates from executors. * @param execId executor id - * @param taskMetrics sequence of (task id, stage id, stage attempt, metrics) + * @param accumUpdates sequence of (taskId, stageId, stageAttemptId, accumUpdates) */ @DeveloperApi case class SparkListenerExecutorMetricsUpdate( execId: String, - taskMetrics: Seq[(Long, Int, Int, TaskMetrics)]) + accumUpdates: Seq[(Long, Int, Int, Seq[AccumulableInfo])]) extends SparkListenerEvent @DeveloperApi @@ -126,263 +163,207 @@ case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent /** * An internal class that describes the metadata of an event log. - * This event is not meant to be posted to listeners downstream. */ -private[spark] case class SparkListenerLogStart(sparkVersion: String) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerLogStart(sparkVersion: String) extends SparkListenerEvent /** - * :: DeveloperApi :: - * Interface for listening to events from the Spark scheduler. Note that this is an internal - * interface which might change in different Spark releases. Java clients should extend - * {@link JavaSparkListener} + * Interface for creating history listeners defined in other modules like SQL, which are used to + * rebuild the history UI. */ -@DeveloperApi -trait SparkListener { +private[spark] trait SparkHistoryListenerFactory { + /** + * Create listeners used to rebuild the history UI. + */ + def createListeners(conf: SparkConf, sparkUI: SparkUI): Seq[SparkListener] +} + + +/** + * Interface for listening to events from the Spark scheduler. Most applications should probably + * extend SparkListener or SparkFirehoseListener directly, rather than implementing this class. + * + * Note that this is an internal interface which might change in different Spark releases. + */ +private[spark] trait SparkListenerInterface { + /** * Called when a stage completes successfully or fails, with information on the completed stage. */ - def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { } + def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit /** * Called when a stage is submitted */ - def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { } + def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit /** * Called when a task starts */ - def onTaskStart(taskStart: SparkListenerTaskStart) { } + def onTaskStart(taskStart: SparkListenerTaskStart): Unit /** * Called when a task begins remotely fetching its result (will not be called for tasks that do * not need to fetch the result remotely). */ - def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { } + def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult): Unit /** * Called when a task ends */ - def onTaskEnd(taskEnd: SparkListenerTaskEnd) { } + def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit /** * Called when a job starts */ - def onJobStart(jobStart: SparkListenerJobStart) { } + def onJobStart(jobStart: SparkListenerJobStart): Unit /** * Called when a job ends */ - def onJobEnd(jobEnd: SparkListenerJobEnd) { } + def onJobEnd(jobEnd: SparkListenerJobEnd): Unit /** * Called when environment properties have been updated */ - def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { } + def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate): Unit /** * Called when a new block manager has joined */ - def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) { } + def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit /** * Called when an existing block manager has been removed */ - def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved) { } + def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit /** * Called when an RDD is manually unpersisted by the application */ - def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { } + def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit /** * Called when the application starts */ - def onApplicationStart(applicationStart: SparkListenerApplicationStart) { } + def onApplicationStart(applicationStart: SparkListenerApplicationStart): Unit /** * Called when the application ends */ - def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { } + def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit /** * Called when the driver receives task metrics from an executor in a heartbeat. */ - def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { } + def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit /** * Called when the driver registers a new executor. */ - def onExecutorAdded(executorAdded: SparkListenerExecutorAdded) { } + def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit /** * Called when the driver removes an executor. */ - def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved) { } + def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit + + /** + * Called when the driver blacklists an executor for a Spark application. + */ + def onExecutorBlacklisted(executorBlacklisted: SparkListenerExecutorBlacklisted): Unit + + /** + * Called when the driver re-enables a previously blacklisted executor. + */ + def onExecutorUnblacklisted(executorUnblacklisted: SparkListenerExecutorUnblacklisted): Unit + + /** + * Called when the driver blacklists a node for a Spark application. + */ + def onNodeBlacklisted(nodeBlacklisted: SparkListenerNodeBlacklisted): Unit + + /** + * Called when the driver re-enables a previously blacklisted node. + */ + def onNodeUnblacklisted(nodeUnblacklisted: SparkListenerNodeUnblacklisted): Unit /** * Called when the driver receives a block update info. */ - def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated) { } + def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit + + /** + * Called when a speculative task is submitted + */ + def onSpeculativeTaskSubmitted(speculativeTask: SparkListenerSpeculativeTaskSubmitted): Unit + + /** + * Called when other events like SQL-specific events are posted. + */ + def onOtherEvent(event: SparkListenerEvent): Unit } + /** * :: DeveloperApi :: - * Simple SparkListener that logs a few summary statistics when each stage completes + * A default implementation for `SparkListenerInterface` that has no-op implementations for + * all callbacks. + * + * Note that this is an internal interface which might change in different Spark releases. */ @DeveloperApi -class StatsReportListener extends SparkListener with Logging { - - import org.apache.spark.scheduler.StatsReportListener._ - - private val taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]() - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val info = taskEnd.taskInfo - val metrics = taskEnd.taskMetrics - if (info != null && metrics != null) { - taskInfoMetrics += ((info, metrics)) - } - } - - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { - implicit val sc = stageCompleted - this.logInfo("Finished stage: " + stageCompleted.stageInfo) - showMillisDistribution("task runtime:", (info, _) => Some(info.duration), taskInfoMetrics) - - // Shuffle write - showBytesDistribution("shuffle bytes written:", - (_, metric) => metric.shuffleWriteMetrics.map(_.shuffleBytesWritten), taskInfoMetrics) - - // Fetch & I/O - showMillisDistribution("fetch wait time:", - (_, metric) => metric.shuffleReadMetrics.map(_.fetchWaitTime), taskInfoMetrics) - showBytesDistribution("remote bytes read:", - (_, metric) => metric.shuffleReadMetrics.map(_.remoteBytesRead), taskInfoMetrics) - showBytesDistribution("task result size:", - (_, metric) => Some(metric.resultSize), taskInfoMetrics) - - // Runtime breakdown - val runtimePcts = taskInfoMetrics.map { case (info, metrics) => - RuntimePercentage(info.duration, metrics) - } - showDistribution("executor (non-fetch) time pct: ", - Distribution(runtimePcts.map(_.executorPct * 100)), "%2.0f %%") - showDistribution("fetch wait time pct: ", - Distribution(runtimePcts.flatMap(_.fetchPct.map(_ * 100))), "%2.0f %%") - showDistribution("other time pct: ", Distribution(runtimePcts.map(_.other * 100)), "%2.0f %%") - taskInfoMetrics.clear() - } +abstract class SparkListener extends SparkListenerInterface { + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { } -} + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { } -private[spark] object StatsReportListener extends Logging { - - // For profiling, the extremes are more interesting - val percentiles = Array[Int](0, 5, 10, 25, 50, 75, 90, 95, 100) - val probabilities = percentiles.map(_ / 100.0) - val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" - - def extractDoubleDistribution( - taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)], - getMetric: (TaskInfo, TaskMetrics) => Option[Double]): Option[Distribution] = { - Distribution(taskInfoMetrics.flatMap { case (info, metric) => getMetric(info, metric) }) - } - - // Is there some way to setup the types that I can get rid of this completely? - def extractLongDistribution( - taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)], - getMetric: (TaskInfo, TaskMetrics) => Option[Long]): Option[Distribution] = { - extractDoubleDistribution( - taskInfoMetrics, - (info, metric) => { getMetric(info, metric).map(_.toDouble) }) - } - - def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) { - val stats = d.statCounter - val quantiles = d.getQuantiles(probabilities).map(formatNumber) - logInfo(heading + stats) - logInfo(percentilesHeader) - logInfo("\t" + quantiles.mkString("\t")) - } - - def showDistribution( - heading: String, - dOpt: Option[Distribution], - formatNumber: Double => String) { - dOpt.foreach { d => showDistribution(heading, d, formatNumber)} - } - - def showDistribution(heading: String, dOpt: Option[Distribution], format: String) { - def f(d: Double): String = format.format(d) - showDistribution(heading, dOpt, f _) - } - - def showDistribution( - heading: String, - format: String, - getMetric: (TaskInfo, TaskMetrics) => Option[Double], - taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { - showDistribution(heading, extractDoubleDistribution(taskInfoMetrics, getMetric), format) - } - - def showBytesDistribution( - heading: String, - getMetric: (TaskInfo, TaskMetrics) => Option[Long], - taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { - showBytesDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric)) - } - - def showBytesDistribution(heading: String, dOpt: Option[Distribution]) { - dOpt.foreach { dist => showBytesDistribution(heading, dist) } - } - - def showBytesDistribution(heading: String, dist: Distribution) { - showDistribution(heading, dist, (d => Utils.bytesToString(d.toLong)): Double => String) - } - - def showMillisDistribution(heading: String, dOpt: Option[Distribution]) { - showDistribution(heading, dOpt, - (d => StatsReportListener.millisToString(d.toLong)): Double => String) - } - - def showMillisDistribution( - heading: String, - getMetric: (TaskInfo, TaskMetrics) => Option[Long], - taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { - showMillisDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric)) - } - - val seconds = 1000L - val minutes = seconds * 60 - val hours = minutes * 60 + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { } - /** - * Reformat a time interval in milliseconds to a prettier format for output - */ - def millisToString(ms: Long): String = { - val (size, units) = - if (ms > hours) { - (ms.toDouble / hours, "hours") - } else if (ms > minutes) { - (ms.toDouble / minutes, "min") - } else if (ms > seconds) { - (ms.toDouble / seconds, "s") - } else { - (ms.toDouble, "ms") - } - "%.1f %s".format(size, units) - } -} + override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult): Unit = { } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { } + + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { } + + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { } + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate): Unit = { } + + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { } + + override def onBlockManagerRemoved( + blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = { } + + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = { } + + override def onApplicationStart(applicationStart: SparkListenerApplicationStart): Unit = { } + + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { } + + override def onExecutorMetricsUpdate( + executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = { } + + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { } + + override def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit = { } + + override def onExecutorBlacklisted( + executorBlacklisted: SparkListenerExecutorBlacklisted): Unit = { } + + override def onExecutorUnblacklisted( + executorUnblacklisted: SparkListenerExecutorUnblacklisted): Unit = { } + + override def onNodeBlacklisted( + nodeBlacklisted: SparkListenerNodeBlacklisted): Unit = { } + + override def onNodeUnblacklisted( + nodeUnblacklisted: SparkListenerNodeUnblacklisted): Unit = { } + + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { } + + override def onSpeculativeTaskSubmitted( + speculativeTask: SparkListenerSpeculativeTaskSubmitted): Unit = { } -private case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double) - -private object RuntimePercentage { - def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { - val denom = totalTime.toDouble - val fetchTime = metrics.shuffleReadMetrics.map(_.fetchWaitTime) - val fetch = fetchTime.map(_ / denom) - val exec = (metrics.executorRunTime - fetchTime.getOrElse(0L)) / denom - val other = 1.0 - (exec + fetch.getOrElse(0d)) - RuntimePercentage(exec, fetch, other) - } + override def onOtherEvent(event: SparkListenerEvent): Unit = { } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 04afde33f5aa..056c0cbded43 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -22,9 +22,12 @@ import org.apache.spark.util.ListenerBus /** * A [[SparkListenerEvent]] bus that relays [[SparkListenerEvent]]s to its listeners */ -private[spark] trait SparkListenerBus extends ListenerBus[SparkListener, SparkListenerEvent] { +private[spark] trait SparkListenerBus + extends ListenerBus[SparkListenerInterface, SparkListenerEvent] { - override def onPostEvent(listener: SparkListener, event: SparkListenerEvent): Unit = { + protected override def doPostEvent( + listener: SparkListenerInterface, + event: SparkListenerEvent): Unit = { event match { case stageSubmitted: SparkListenerStageSubmitted => listener.onStageSubmitted(stageSubmitted) @@ -58,9 +61,19 @@ private[spark] trait SparkListenerBus extends ListenerBus[SparkListener, SparkLi listener.onExecutorAdded(executorAdded) case executorRemoved: SparkListenerExecutorRemoved => listener.onExecutorRemoved(executorRemoved) + case executorBlacklisted: SparkListenerExecutorBlacklisted => + listener.onExecutorBlacklisted(executorBlacklisted) + case executorUnblacklisted: SparkListenerExecutorUnblacklisted => + listener.onExecutorUnblacklisted(executorUnblacklisted) + case nodeBlacklisted: SparkListenerNodeBlacklisted => + listener.onNodeBlacklisted(nodeBlacklisted) + case nodeUnblacklisted: SparkListenerNodeUnblacklisted => + listener.onNodeUnblacklisted(nodeUnblacklisted) case blockUpdated: SparkListenerBlockUpdated => listener.onBlockUpdated(blockUpdated) - case logStart: SparkListenerLogStart => // ignore event log metadata + case speculativeTaskSubmitted: SparkListenerSpeculativeTaskSubmitted => + listener.onSpeculativeTaskSubmitted(speculativeTaskSubmitted) + case _ => listener.onOtherEvent(event) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index 1ce83485f024..bc1431835e25 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -45,18 +45,17 @@ class SplitInfo( hashCode } - // This is practically useless since most of the Split impl's dont seem to implement equals :-( + // This is practically useless since most of the Split impl's don't seem to implement equals :-( // So unless there is identity equality between underlyingSplits, it will always fail even if it // is pointing to same block. override def equals(other: Any): Boolean = other match { - case that: SplitInfo => { + case that: SplitInfo => this.hostLocation == that.hostLocation && this.inputFormatClazz == that.inputFormatClazz && this.path == that.path && this.length == that.length && // other split specific checks (like start for FileSplit) this.underlyingSplit == that.underlyingSplit - } case _ => false } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 7ea24a217bd3..290fd073caf2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -19,7 +19,8 @@ package org.apache.spark.scheduler import scala.collection.mutable.HashSet -import org.apache.spark._ +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.util.CallSite @@ -66,32 +67,14 @@ private[scheduler] abstract class Stage( /** Set of jobs that this stage belongs to. */ val jobIds = new HashSet[Int] - val pendingPartitions = new HashSet[Int] - /** The ID to use for the next new attempt for this stage. */ private var nextAttemptId: Int = 0 val name: String = callSite.shortForm val details: String = callSite.longForm - private var _internalAccumulators: Seq[Accumulator[Long]] = Seq.empty - - /** Internal accumulators shared across all tasks in this stage. */ - def internalAccumulators: Seq[Accumulator[Long]] = _internalAccumulators - - /** - * Re-initialize the internal accumulators associated with this stage. - * - * This is called every time the stage is submitted, *except* when a subset of tasks - * belonging to this stage has already finished. Otherwise, reinitializing the internal - * accumulators here again will override partial values from the finished tasks. - */ - def resetInternalAccumulators(): Unit = { - _internalAccumulators = InternalAccumulator.create(rdd.sparkContext) - } - /** - * Pointer to the [StageInfo] object for the most recent attempt. This needs to be initialized + * Pointer to the [[StageInfo]] object for the most recent attempt. This needs to be initialized * here, before any attempts have actually been created, because the DAGScheduler uses this * StageInfo to tell SparkListeners when a job starts (which happens before any stage attempts * have been created). @@ -104,29 +87,20 @@ private[scheduler] abstract class Stage( * We keep track of each attempt ID that has failed to avoid recording duplicate failures if * multiple tasks from the same stage attempt fail (SPARK-5945). */ - private val fetchFailedAttemptIds = new HashSet[Int] + val fetchFailedAttemptIds = new HashSet[Int] private[scheduler] def clearFailures() : Unit = { fetchFailedAttemptIds.clear() } - /** - * Check whether we should abort the failedStage due to multiple consecutive fetch failures. - * - * This method updates the running set of failed stage attempts and returns - * true if the number of failures exceeds the allowable number of failures. - */ - private[scheduler] def failedOnFetchAndShouldAbort(stageAttemptId: Int): Boolean = { - fetchFailedAttemptIds.add(stageAttemptId) - fetchFailedAttemptIds.size >= Stage.MAX_CONSECUTIVE_FETCH_FAILURES - } - /** Creates a new attempt for this stage by creating a new StageInfo with a new attempt ID. */ def makeNewStageAttempt( numPartitionsToCompute: Int, taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty): Unit = { + val metrics = new TaskMetrics + metrics.register(rdd.sparkContext) _latestInfo = StageInfo.fromStage( - this, nextAttemptId, Some(numPartitionsToCompute), taskLocalityPreferences) + this, nextAttemptId, Some(numPartitionsToCompute), metrics, taskLocalityPreferences) nextAttemptId += 1 } @@ -143,8 +117,3 @@ private[scheduler] abstract class Stage( /** Returns the sequence of partition ids that are missing (i.e. needs to be computed). */ def findMissingPartitions(): Seq[Int] } - -private[scheduler] object Stage { - // The number of consecutive failures allowed before a stage is aborted - val MAX_CONSECUTIVE_FETCH_FAILURES = 4 -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 24796c14300b..c513ed36d168 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import scala.collection.mutable.HashMap import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.RDDInfo /** @@ -35,6 +36,7 @@ class StageInfo( val rddInfos: Seq[RDDInfo], val parentIds: Seq[Int], val details: String, + val taskMetrics: TaskMetrics = null, private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None @@ -42,7 +44,11 @@ class StageInfo( var completionTime: Option[Long] = None /** If the stage failed, the reason why. */ var failureReason: Option[String] = None - /** Terminal values of accumulables updated during this stage. */ + + /** + * Terminal values of accumulables updated during this stage, including all the user-defined + * accumulators. + */ val accumulables = HashMap[Long, AccumulableInfo]() def stageFailed(reason: String) { @@ -75,6 +81,7 @@ private[spark] object StageInfo { stage: Stage, attemptId: Int, numTasks: Option[Int] = None, + taskMetrics: TaskMetrics = null, taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty ): StageInfo = { val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) @@ -87,6 +94,7 @@ private[spark] object StageInfo { rddInfos, stage.parents.map(_.id), stage.details, + taskMetrics, taskLocalityPreferences) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala new file mode 100644 index 000000000000..3c8cab7504c1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/StatsReportListener.scala @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import scala.collection.mutable + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.Logging +import org.apache.spark.util.{Distribution, Utils} + + +/** + * :: DeveloperApi :: + * Simple SparkListener that logs a few summary statistics when each stage completes. + */ +@DeveloperApi +class StatsReportListener extends SparkListener with Logging { + + import org.apache.spark.scheduler.StatsReportListener._ + + private val taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]() + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val info = taskEnd.taskInfo + val metrics = taskEnd.taskMetrics + if (info != null && metrics != null) { + taskInfoMetrics += ((info, metrics)) + } + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { + implicit val sc = stageCompleted + this.logInfo(s"Finished stage: ${getStatusDetail(stageCompleted.stageInfo)}") + showMillisDistribution("task runtime:", (info, _) => info.duration, taskInfoMetrics) + + // Shuffle write + showBytesDistribution("shuffle bytes written:", + (_, metric) => metric.shuffleWriteMetrics.bytesWritten, taskInfoMetrics) + + // Fetch & I/O + showMillisDistribution("fetch wait time:", + (_, metric) => metric.shuffleReadMetrics.fetchWaitTime, taskInfoMetrics) + showBytesDistribution("remote bytes read:", + (_, metric) => metric.shuffleReadMetrics.remoteBytesRead, taskInfoMetrics) + showBytesDistribution("task result size:", + (_, metric) => metric.resultSize, taskInfoMetrics) + + // Runtime breakdown + val runtimePcts = taskInfoMetrics.map { case (info, metrics) => + RuntimePercentage(info.duration, metrics) + } + showDistribution("executor (non-fetch) time pct: ", + Distribution(runtimePcts.map(_.executorPct * 100)), "%2.0f %%") + showDistribution("fetch wait time pct: ", + Distribution(runtimePcts.flatMap(_.fetchPct.map(_ * 100))), "%2.0f %%") + showDistribution("other time pct: ", Distribution(runtimePcts.map(_.other * 100)), "%2.0f %%") + taskInfoMetrics.clear() + } + + private def getStatusDetail(info: StageInfo): String = { + val failureReason = info.failureReason.map("(" + _ + ")").getOrElse("") + val timeTaken = info.submissionTime.map( + x => info.completionTime.getOrElse(System.currentTimeMillis()) - x + ).getOrElse("-") + + s"Stage(${info.stageId}, ${info.attemptId}); Name: '${info.name}'; " + + s"Status: ${info.getStatusString}$failureReason; numTasks: ${info.numTasks}; " + + s"Took: $timeTaken msec" + } + +} + +private[spark] object StatsReportListener extends Logging { + + // For profiling, the extremes are more interesting + val percentiles = Array[Int](0, 5, 10, 25, 50, 75, 90, 95, 100) + val probabilities = percentiles.map(_ / 100.0) + val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" + + def extractDoubleDistribution( + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)], + getMetric: (TaskInfo, TaskMetrics) => Double): Option[Distribution] = { + Distribution(taskInfoMetrics.map { case (info, metric) => getMetric(info, metric) }) + } + + // Is there some way to setup the types that I can get rid of this completely? + def extractLongDistribution( + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)], + getMetric: (TaskInfo, TaskMetrics) => Long): Option[Distribution] = { + extractDoubleDistribution( + taskInfoMetrics, + (info, metric) => { getMetric(info, metric).toDouble }) + } + + def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) { + val stats = d.statCounter + val quantiles = d.getQuantiles(probabilities).map(formatNumber) + logInfo(heading + stats) + logInfo(percentilesHeader) + logInfo("\t" + quantiles.mkString("\t")) + } + + def showDistribution( + heading: String, + dOpt: Option[Distribution], + formatNumber: Double => String) { + dOpt.foreach { d => showDistribution(heading, d, formatNumber)} + } + + def showDistribution(heading: String, dOpt: Option[Distribution], format: String) { + def f(d: Double): String = format.format(d) + showDistribution(heading, dOpt, f _) + } + + def showDistribution( + heading: String, + format: String, + getMetric: (TaskInfo, TaskMetrics) => Double, + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showDistribution(heading, extractDoubleDistribution(taskInfoMetrics, getMetric), format) + } + + def showBytesDistribution( + heading: String, + getMetric: (TaskInfo, TaskMetrics) => Long, + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showBytesDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric)) + } + + def showBytesDistribution(heading: String, dOpt: Option[Distribution]) { + dOpt.foreach { dist => showBytesDistribution(heading, dist) } + } + + def showBytesDistribution(heading: String, dist: Distribution) { + showDistribution(heading, dist, (d => Utils.bytesToString(d.toLong)): Double => String) + } + + def showMillisDistribution(heading: String, dOpt: Option[Distribution]) { + showDistribution(heading, dOpt, + (d => StatsReportListener.millisToString(d.toLong)): Double => String) + } + + def showMillisDistribution( + heading: String, + getMetric: (TaskInfo, TaskMetrics) => Long, + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showMillisDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric)) + } + + val seconds = 1000L + val minutes = seconds * 60 + val hours = minutes * 60 + + /** + * Reformat a time interval in milliseconds to a prettier format for output + */ + def millisToString(ms: Long): String = { + val (size, units) = + if (ms > hours) { + (ms.toDouble / hours, "hours") + } else if (ms > minutes) { + (ms.toDouble / minutes, "min") + } else if (ms > seconds) { + (ms.toDouble / seconds, "s") + } else { + (ms.toDouble, "ms") + } + "%.1f %s".format(size, units) + } +} + +private case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double) + +private object RuntimePercentage { + def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { + val denom = totalTime.toDouble + val fetchTime = Some(metrics.shuffleReadMetrics.fetchWaitTime) + val fetch = fetchTime.map(_ / denom) + val exec = (metrics.executorRunTime - fetchTime.getOrElse(0L)) / denom + val other = 1.0 - (exec + fetch.getOrElse(0d)) + RuntimePercentage(exec, fetch, other) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 4fb32ba8cb18..7767ef1803a0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -17,24 +17,21 @@ package org.apache.spark.scheduler -import java.io.{ByteArrayOutputStream, DataInputStream, DataOutputStream} import java.nio.ByteBuffer +import java.util.Properties -import scala.collection.mutable.HashMap - -import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.{Accumulator, SparkEnv, TaskContextImpl, TaskContext} +import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.memory.TaskMemoryManager -import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.ByteBufferInputStream -import org.apache.spark.util.Utils - +import org.apache.spark.internal.config.APP_CALLER_CONTEXT +import org.apache.spark.memory.{MemoryMode, TaskMemoryManager} +import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.util._ /** * A unit of execution. We have two kinds of Task's in Spark: - * - [[org.apache.spark.scheduler.ShuffleMapTask]] - * - [[org.apache.spark.scheduler.ResultTask]] + * + * - [[org.apache.spark.scheduler.ShuffleMapTask]] + * - [[org.apache.spark.scheduler.ResultTask]] * * A Spark job consists of one or more stages. The very last stage in a job consists of multiple * ResultTasks, while earlier stages consist of ShuffleMapTasks. A ResultTask executes the task @@ -42,59 +39,109 @@ import org.apache.spark.util.Utils * and divides the task output to multiple buckets (based on the task's partitioner). * * @param stageId id of the stage this task belongs to + * @param stageAttemptId attempt id of the stage this task belongs to * @param partitionId index of the number in the RDD + * @param localProperties copy of thread-local properties set by the user on the driver side. + * @param serializedTaskMetrics a `TaskMetrics` that is created and serialized on the driver side + * and sent to executor side. + * + * The parameters below are optional: + * @param jobId id of the job this task belongs to + * @param appId id of the app this task belongs to + * @param appAttemptId attempt id of the app this task belongs to */ private[spark] abstract class Task[T]( val stageId: Int, val stageAttemptId: Int, val partitionId: Int, - internalAccumulators: Seq[Accumulator[Long]]) extends Serializable { + @transient var localProperties: Properties = new Properties, + // The default value is only used in tests. + serializedTaskMetrics: Array[Byte] = + SparkEnv.get.closureSerializer.newInstance().serialize(TaskMetrics.registered).array(), + val jobId: Option[Int] = None, + val appId: Option[String] = None, + val appAttemptId: Option[String] = None) extends Serializable { - /** - * The key of the Map is the accumulator id and the value of the Map is the latest accumulator - * local value. - */ - type AccumulatorUpdates = Map[Long, Any] + @transient lazy val metrics: TaskMetrics = + SparkEnv.get.closureSerializer.newInstance().deserialize(ByteBuffer.wrap(serializedTaskMetrics)) /** - * Called by [[Executor]] to run this task. + * Called by [[org.apache.spark.executor.Executor]] to run this task. * * @param taskAttemptId an identifier for this task attempt that is unique within a SparkContext. * @param attemptNumber how many times this task has been attempted (0 for the first attempt) * @return the result of the task along with updates of Accumulators. */ final def run( - taskAttemptId: Long, - attemptNumber: Int, - metricsSystem: MetricsSystem) - : (T, AccumulatorUpdates) = { + taskAttemptId: Long, + attemptNumber: Int, + metricsSystem: MetricsSystem): T = { + SparkEnv.get.blockManager.registerTask(taskAttemptId) context = new TaskContextImpl( stageId, partitionId, taskAttemptId, attemptNumber, taskMemoryManager, + localProperties, metricsSystem, - internalAccumulators, - runningLocally = false) + metrics) TaskContext.setTaskContext(context) - context.taskMetrics.setHostname(Utils.localHostName()) - context.taskMetrics.setAccumulatorsUpdater(context.collectInternalAccumulators) taskThread = Thread.currentThread() - if (_killed) { - kill(interruptThread = false) + + if (_reasonIfKilled != null) { + kill(interruptThread = false, _reasonIfKilled) } + + new CallerContext( + "TASK", + SparkEnv.get.conf.get(APP_CALLER_CONTEXT), + appId, + appAttemptId, + jobId, + Option(stageId), + Option(stageAttemptId), + Option(taskAttemptId), + Option(attemptNumber)).setCurrentContext() + try { - (runTask(context), context.collectAccumulators()) + runTask(context) + } catch { + case e: Throwable => + // Catch all errors; run task failure callbacks, and rethrow the exception. + try { + context.markTaskFailed(e) + } catch { + case t: Throwable => + e.addSuppressed(t) + } + context.markTaskCompleted(Some(e)) + throw e } finally { - context.markTaskCompleted() try { - Utils.tryLogNonFatalError { - // Release memory used by this thread for unrolling blocks - SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask() - } + // Call the task completion callbacks. If "markTaskCompleted" is called twice, the second + // one is no-op. + context.markTaskCompleted(None) } finally { - TaskContext.unset() + try { + Utils.tryLogNonFatalError { + // Release memory used by this thread for unrolling blocks + SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP) + SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask( + MemoryMode.OFF_HEAP) + // Notify any tasks waiting for execution memory to be freed to wake up and try to + // acquire memory again. This makes impossible the scenario where a task sleeps forever + // because there are no other tasks left to notify it. Since this is safe to do but may + // not be strictly necessary, we should revisit whether we can remove this in the + // future. + val memoryManager = SparkEnv.get.memoryManager + memoryManager.synchronized { memoryManager.notifyAll() } + } + } finally { + // Though we unset the ThreadLocal here, the context member variable itself is still + // queried directly in the TaskRunner to check for FetchFailedExceptions. + TaskContext.unset() + } } } } @@ -109,32 +156,48 @@ private[spark] abstract class Task[T]( def preferredLocations: Seq[TaskLocation] = Nil - // Map output tracker epoch. Will be set by TaskScheduler. + // Map output tracker epoch. Will be set by TaskSetManager. var epoch: Long = -1 - var metrics: Option[TaskMetrics] = None - // Task context, to be initialized in run(). - @transient protected var context: TaskContextImpl = _ + @transient var context: TaskContextImpl = _ // The actual Thread on which the task is running, if any. Initialized in run(). @volatile @transient private var taskThread: Thread = _ - // A flag to indicate whether the task is killed. This is used in case context is not yet - // initialized when kill() is invoked. - @volatile @transient private var _killed = false + // If non-null, this task has been killed and the reason is as specified. This is used in case + // context is not yet initialized when kill() is invoked. + @volatile @transient private var _reasonIfKilled: String = null protected var _executorDeserializeTime: Long = 0 + protected var _executorDeserializeCpuTime: Long = 0 /** - * Whether the task has been killed. + * If defined, this task has been killed and this option contains the reason. */ - def killed: Boolean = _killed + def reasonIfKilled: Option[String] = Option(_reasonIfKilled) /** * Returns the amount of time spent deserializing the RDD and function to be run. */ def executorDeserializeTime: Long = _executorDeserializeTime + def executorDeserializeCpuTime: Long = _executorDeserializeCpuTime + + /** + * Collect the latest values of accumulators used in this task. If the task failed, + * filter out the accumulators whose values should not be included on failures. + */ + def collectAccumulatorUpdates(taskFailed: Boolean = false): Seq[AccumulatorV2[_, _]] = { + if (context != null) { + // Note: internal accumulators representing task metrics always count failed values + context.taskMetrics.nonZeroInternalAccums() ++ + // zero value external accumulators may still be useful, e.g. SQLMetrics, we should not + // filter them out. + context.taskMetrics.externalAccums.filter(a => !taskFailed || a.countFailedValues) + } else { + Seq.empty + } + } /** * Kills a task by setting the interrupted flag to true. This relies on the upper level Spark @@ -142,88 +205,14 @@ private[spark] abstract class Task[T]( * be called multiple times. * If interruptThread is true, we will also call Thread.interrupt() on the Task's executor thread. */ - def kill(interruptThread: Boolean) { - _killed = true + def kill(interruptThread: Boolean, reason: String) { + require(reason != null) + _reasonIfKilled = reason if (context != null) { - context.markInterrupted() + context.markInterrupted(reason) } if (interruptThread && taskThread != null) { taskThread.interrupt() } } } - -/** - * Handles transmission of tasks and their dependencies, because this can be slightly tricky. We - * need to send the list of JARs and files added to the SparkContext with each task to ensure that - * worker nodes find out about it, but we can't make it part of the Task because the user's code in - * the task might depend on one of the JARs. Thus we serialize each task as multiple objects, by - * first writing out its dependencies. - */ -private[spark] object Task { - /** - * Serialize a task and the current app dependencies (files and JARs added to the SparkContext) - */ - def serializeWithDependencies( - task: Task[_], - currentFiles: HashMap[String, Long], - currentJars: HashMap[String, Long], - serializer: SerializerInstance) - : ByteBuffer = { - - val out = new ByteArrayOutputStream(4096) - val dataOut = new DataOutputStream(out) - - // Write currentFiles - dataOut.writeInt(currentFiles.size) - for ((name, timestamp) <- currentFiles) { - dataOut.writeUTF(name) - dataOut.writeLong(timestamp) - } - - // Write currentJars - dataOut.writeInt(currentJars.size) - for ((name, timestamp) <- currentJars) { - dataOut.writeUTF(name) - dataOut.writeLong(timestamp) - } - - // Write the task itself and finish - dataOut.flush() - val taskBytes = serializer.serialize(task).array() - out.write(taskBytes) - ByteBuffer.wrap(out.toByteArray) - } - - /** - * Deserialize the list of dependencies in a task serialized with serializeWithDependencies, - * and return the task itself as a serialized ByteBuffer. The caller can then update its - * ClassLoaders and deserialize the task. - * - * @return (taskFiles, taskJars, taskBytes) - */ - def deserializeWithDependencies(serializedTask: ByteBuffer) - : (HashMap[String, Long], HashMap[String, Long], ByteBuffer) = { - - val in = new ByteBufferInputStream(serializedTask) - val dataIn = new DataInputStream(in) - - // Read task's files - val taskFiles = new HashMap[String, Long]() - val numFiles = dataIn.readInt() - for (i <- 0 until numFiles) { - taskFiles(dataIn.readUTF()) = dataIn.readLong() - } - - // Read task's JARs - val taskJars = new HashMap[String, Long]() - val numJars = dataIn.readInt() - for (i <- 0 until numJars) { - taskJars(dataIn.readUTF()) = dataIn.readLong() - } - - // Create a sub-buffer for the rest of the data, which is the serialized Task object - val subBuffer = serializedTask.slice() // ByteBufferInputStream will have read just up to task - (taskFiles, taskJars, subBuffer) - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala index 1c7c81c488c3..c98b87148e40 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -17,13 +17,32 @@ package org.apache.spark.scheduler +import java.io.{DataInputStream, DataOutputStream} import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets +import java.util.Properties -import org.apache.spark.util.SerializableBuffer +import scala.collection.JavaConverters._ +import scala.collection.mutable.{HashMap, Map} + +import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Utils} /** * Description of a task that gets passed onto executors to be executed, usually created by - * [[TaskSetManager.resourceOffer]]. + * `TaskSetManager.resourceOffer`. + * + * TaskDescriptions and the associated Task need to be serialized carefully for two reasons: + * + * (1) When a TaskDescription is received by an Executor, the Executor needs to first get the + * list of JARs and files and add these to the classpath, and set the properties, before + * deserializing the Task object (serializedTask). This is why the Properties are included + * in the TaskDescription, even though they're also in the serialized task. + * (2) Because a TaskDescription is serialized and sent to an executor for each task, efficient + * serialization (both in terms of serialization time and serialized buffer size) is + * important. For this reason, we serialize TaskDescriptions ourselves with the + * TaskDescription.encode and TaskDescription.decode methods. This results in a smaller + * serialized size because it avoids serializing unnecessary fields in the Map objects + * (which can introduce significant overhead when the maps are small). */ private[spark] class TaskDescription( val taskId: Long, @@ -31,13 +50,95 @@ private[spark] class TaskDescription( val executorId: String, val name: String, val index: Int, // Index within this task's TaskSet - _serializedTask: ByteBuffer) - extends Serializable { + val addedFiles: Map[String, Long], + val addedJars: Map[String, Long], + val properties: Properties, + val serializedTask: ByteBuffer) { + + override def toString: String = "TaskDescription(TID=%d, index=%d)".format(taskId, index) +} - // Because ByteBuffers are not serializable, wrap the task in a SerializableBuffer - private val buffer = new SerializableBuffer(_serializedTask) +private[spark] object TaskDescription { + private def serializeStringLongMap(map: Map[String, Long], dataOut: DataOutputStream): Unit = { + dataOut.writeInt(map.size) + for ((key, value) <- map) { + dataOut.writeUTF(key) + dataOut.writeLong(value) + } + } - def serializedTask: ByteBuffer = buffer.value + def encode(taskDescription: TaskDescription): ByteBuffer = { + val bytesOut = new ByteBufferOutputStream(4096) + val dataOut = new DataOutputStream(bytesOut) - override def toString: String = "TaskDescription(TID=%d, index=%d)".format(taskId, index) + dataOut.writeLong(taskDescription.taskId) + dataOut.writeInt(taskDescription.attemptNumber) + dataOut.writeUTF(taskDescription.executorId) + dataOut.writeUTF(taskDescription.name) + dataOut.writeInt(taskDescription.index) + + // Write files. + serializeStringLongMap(taskDescription.addedFiles, dataOut) + + // Write jars. + serializeStringLongMap(taskDescription.addedJars, dataOut) + + // Write properties. + dataOut.writeInt(taskDescription.properties.size()) + taskDescription.properties.asScala.foreach { case (key, value) => + dataOut.writeUTF(key) + // SPARK-19796 -- writeUTF doesn't work for long strings, which can happen for property values + val bytes = value.getBytes(StandardCharsets.UTF_8) + dataOut.writeInt(bytes.length) + dataOut.write(bytes) + } + + // Write the task. The task is already serialized, so write it directly to the byte buffer. + Utils.writeByteBuffer(taskDescription.serializedTask, bytesOut) + + dataOut.close() + bytesOut.close() + bytesOut.toByteBuffer + } + + private def deserializeStringLongMap(dataIn: DataInputStream): HashMap[String, Long] = { + val map = new HashMap[String, Long]() + val mapSize = dataIn.readInt() + for (i <- 0 until mapSize) { + map(dataIn.readUTF()) = dataIn.readLong() + } + map + } + + def decode(byteBuffer: ByteBuffer): TaskDescription = { + val dataIn = new DataInputStream(new ByteBufferInputStream(byteBuffer)) + val taskId = dataIn.readLong() + val attemptNumber = dataIn.readInt() + val executorId = dataIn.readUTF() + val name = dataIn.readUTF() + val index = dataIn.readInt() + + // Read files. + val taskFiles = deserializeStringLongMap(dataIn) + + // Read jars. + val taskJars = deserializeStringLongMap(dataIn) + + // Read properties. + val properties = new Properties() + val numProperties = dataIn.readInt() + for (i <- 0 until numProperties) { + val key = dataIn.readUTF() + val valueLength = dataIn.readInt() + val valueBytes = new Array[Byte](valueLength) + dataIn.readFully(valueBytes) + properties.setProperty(key, new String(valueBytes, StandardCharsets.UTF_8)) + } + + // Create a sub-buffer for the serialized task into its own buffer (to be deserialized later). + val serializedTask = byteBuffer.slice() + + new TaskDescription(taskId, attemptNumber, executorId, name, index, taskFiles, taskJars, + properties, serializedTask) + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index f113c2b1b843..9843eab4f134 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,8 +17,8 @@ package org.apache.spark.scheduler -import scala.collection.mutable.ListBuffer - +import org.apache.spark.TaskState +import org.apache.spark.TaskState.TaskState import org.apache.spark.annotation.DeveloperApi /** @@ -28,6 +28,10 @@ import org.apache.spark.annotation.DeveloperApi @DeveloperApi class TaskInfo( val taskId: Long, + /** + * The index of this task within its task set. Not necessarily the same as the ID of the RDD + * partition that the task is computing. + */ val index: Int, val attemptNumber: Int, val launchTime: Long, @@ -48,7 +52,13 @@ class TaskInfo( * accumulable to be updated multiple times in a single task or for two accumulables with the * same name but different IDs to exist in a task. */ - val accumulables = ListBuffer[AccumulableInfo]() + def accumulables: Seq[AccumulableInfo] = _accumulables + + private[this] var _accumulables: Seq[AccumulableInfo] = Nil + + private[spark] def setAccumulables(newAccumulables: Seq[AccumulableInfo]): Unit = { + _accumulables = newAccumulables + } /** * The time when the task has completed successfully (including the time to remotely fetch @@ -58,24 +68,28 @@ class TaskInfo( var failed = false - private[spark] def markGettingResult(time: Long = System.currentTimeMillis) { - gettingResultTime = time - } + var killed = false - private[spark] def markSuccessful(time: Long = System.currentTimeMillis) { - finishTime = time + private[spark] def markGettingResult(time: Long) { + gettingResultTime = time } - private[spark] def markFailed(time: Long = System.currentTimeMillis) { + private[spark] def markFinished(state: TaskState, time: Long) { + // finishTime should be set larger than 0, otherwise "finished" below will return false. + assert(time > 0) finishTime = time - failed = true + if (state == TaskState.FAILED) { + failed = true + } else if (state == TaskState.KILLED) { + killed = true + } } def gettingResult: Boolean = gettingResultTime != 0 def finished: Boolean = finishTime != 0 - def successful: Boolean = finished && !failed + def successful: Boolean = finished && !failed && !killed def running: Boolean = !finished @@ -88,6 +102,8 @@ class TaskInfo( } } else if (failed) { "FAILED" + } else if (killed) { + "KILLED" } else if (successful) { "SUCCESS" } else { @@ -95,9 +111,6 @@ class TaskInfo( } } - @deprecated("Use attemptNumber", "1.6.0") - def attempt: Int = attemptNumber - def id: String = s"$index.$attemptNumber" def duration: Long = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala index 1eb6c1614fc0..06b52935c696 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala @@ -64,18 +64,18 @@ private[spark] object TaskLocation { /** * Create a TaskLocation from a string returned by getPreferredLocations. - * These strings have the form [hostname] or hdfs_cache_[hostname], depending on whether the - * location is cached. + * These strings have the form executor_[hostname]_[executorid], [hostname], or + * hdfs_cache_[hostname], depending on whether the location is cached. */ def apply(str: String): TaskLocation = { val hstr = str.stripPrefix(inMemoryLocationTag) if (hstr.equals(str)) { if (str.startsWith(executorLocationTag)) { - val splits = str.split("_") - if (splits.length != 3) { - throw new IllegalArgumentException("Illegal executor location format: " + str) - } - new ExecutorCacheTaskLocation(splits(1), splits(2)) + val hostAndExecutorId = str.stripPrefix(executorLocationTag) + val splits = hostAndExecutorId.split("_", 2) + require(splits.length == 2, "Illegal executor location format: " + str) + val Array(host, executorId) = splits + new ExecutorCacheTaskLocation(host, executorId) } else { new HostTaskLocation(str) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index b82c7f3fa54f..836769e1723d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -20,13 +20,12 @@ package org.apache.spark.scheduler import java.io._ import java.nio.ByteBuffer -import scala.collection.Map -import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkEnv -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.serializer.SerializerInstance import org.apache.spark.storage.BlockId -import org.apache.spark.util.Utils +import org.apache.spark.util.{AccumulatorV2, Utils} // Task result. Also contains updates to accumulator variables. private[spark] sealed trait TaskResult[T] @@ -36,31 +35,24 @@ private[spark] case class IndirectTaskResult[T](blockId: BlockId, size: Int) extends TaskResult[T] with Serializable /** A TaskResult that contains the task's return value and accumulator updates. */ -private[spark] -class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long, Any], - var metrics: TaskMetrics) +private[spark] class DirectTaskResult[T]( + var valueBytes: ByteBuffer, + var accumUpdates: Seq[AccumulatorV2[_, _]]) extends TaskResult[T] with Externalizable { private var valueObjectDeserialized = false private var valueObject: T = _ - def this() = this(null.asInstanceOf[ByteBuffer], null, null) + def this() = this(null.asInstanceOf[ByteBuffer], null) override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { - - out.writeInt(valueBytes.remaining); + out.writeInt(valueBytes.remaining) Utils.writeByteBuffer(valueBytes, out) - out.writeInt(accumUpdates.size) - for ((key, value) <- accumUpdates) { - out.writeLong(key) - out.writeObject(value) - } - out.writeObject(metrics) + accumUpdates.foreach(out.writeObject) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { - val blen = in.readInt() val byteVal = new Array[Byte](blen) in.readFully(byteVal) @@ -68,15 +60,14 @@ class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long val numUpdates = in.readInt if (numUpdates == 0) { - accumUpdates = null + accumUpdates = Seq.empty } else { - val _accumUpdates = mutable.Map[Long, Any]() + val _accumUpdates = new ArrayBuffer[AccumulatorV2[_, _]] for (i <- 0 until numUpdates) { - _accumUpdates(in.readLong()) = in.readObject() + _accumUpdates += in.readObject.asInstanceOf[AccumulatorV2[_, _]] } accumUpdates = _accumUpdates } - metrics = in.readObject().asInstanceOf[TaskMetrics] valueObjectDeserialized = false } @@ -87,14 +78,14 @@ class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long * * After the first time, `value()` is trivial and just returns the deserialized `valueObject`. */ - def value(): T = { + def value(resultSer: SerializerInstance = null): T = { if (valueObjectDeserialized) { valueObject } else { // This should not run when holding a lock because it may cost dozens of seconds for a large - // value. - val resultSer = SparkEnv.get.serializer.newInstance() - valueObject = resultSer.deserialize(valueBytes) + // value + val ser = if (resultSer == null) SparkEnv.get.serializer.newInstance() else resultSer + valueObject = ser.deserialize(valueBytes) valueObjectDeserialized = true valueObject } 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 46a6f6537e2e..a284f7956cd3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -18,15 +18,16 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer -import java.util.concurrent.RejectedExecutionException +import java.util.concurrent.{ExecutorService, RejectedExecutionException} import scala.language.existentials import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.TaskState.TaskState +import org.apache.spark.internal.Logging import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{LongAccumulator, ThreadUtils, Utils} /** * Runs a thread pool that deserializes and remotely fetches (if necessary) task results. @@ -35,17 +36,28 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul extends Logging { private val THREADS = sparkEnv.conf.getInt("spark.resultGetter.threads", 4) - private val getTaskResultExecutor = ThreadUtils.newDaemonFixedThreadPool( - THREADS, "task-result-getter") + // Exposed for testing. + protected val getTaskResultExecutor: ExecutorService = + ThreadUtils.newDaemonFixedThreadPool(THREADS, "task-result-getter") + + // Exposed for testing. protected val serializer = new ThreadLocal[SerializerInstance] { override def initialValue(): SerializerInstance = { sparkEnv.closureSerializer.newInstance() } } + protected val taskResultSerializer = new ThreadLocal[SerializerInstance] { + override def initialValue(): SerializerInstance = { + sparkEnv.serializer.newInstance() + } + } + def enqueueSuccessfulTask( - taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) { + taskSetManager: TaskSetManager, + tid: Long, + serializedData: ByteBuffer): Unit = { getTaskResultExecutor.execute(new Runnable { override def run(): Unit = Utils.logUncaughtExceptions { try { @@ -57,7 +69,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul // deserialize "value" without holding any lock so that it won't block other threads. // We should call it here, so that when it's called again in // "TaskSetManager.handleSuccessfulTask", it does not need to deserialize the value. - directResult.value() + directResult.value(taskResultSerializer.get()) (directResult, serializedData.limit()) case IndirectTaskResult(blockId, size) => if (!taskSetManager.canFetchMoreResults(size)) { @@ -77,12 +89,27 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul return } val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]]( - serializedTaskResult.get) + serializedTaskResult.get.toByteBuffer) + // force deserialization of referenced value + deserializedResult.value(taskResultSerializer.get()) sparkEnv.blockManager.master.removeBlock(blockId) (deserializedResult, size) } - result.metrics.setResultSize(size) + // Set the task result size in the accumulator updates received from the executors. + // We need to do this here on the driver because if we did this on the executors then + // we would have to serialize the result again after updating the size. + result.accumUpdates = result.accumUpdates.map { a => + if (a.name == Some(InternalAccumulator.RESULT_SIZE)) { + val acc = a.asInstanceOf[LongAccumulator] + assert(acc.sum == 0L, "task result size should not have been set on the executors") + acc.setValue(size.toLong) + acc + } else { + a + } + } + scheduler.handleSuccessfulTask(taskSetManager, tid, result) } catch { case cnf: ClassNotFoundException => @@ -99,25 +126,29 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul def enqueueFailedTask(taskSetManager: TaskSetManager, tid: Long, taskState: TaskState, serializedData: ByteBuffer) { - var reason : TaskEndReason = UnknownReason + var reason : TaskFailedReason = UnknownReason try { getTaskResultExecutor.execute(new Runnable { override def run(): Unit = Utils.logUncaughtExceptions { + val loader = Utils.getContextOrSparkClassLoader try { if (serializedData != null && serializedData.limit() > 0) { - reason = serializer.get().deserialize[TaskEndReason]( - serializedData, Utils.getSparkClassLoader) + reason = serializer.get().deserialize[TaskFailedReason]( + serializedData, loader) } } 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 => {} + case ex: Exception => // No-op + } finally { + // If there's an error while deserializing the TaskEndReason, this Runnable + // will die. Still tell the scheduler about the task failure, to avoid a hang + // where the scheduler thinks the task is still running. + scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) } - scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) } }) } catch { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index cb9a3008107d..90644fea23ab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -18,8 +18,8 @@ package org.apache.spark.scheduler import org.apache.spark.scheduler.SchedulingMode.SchedulingMode -import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.AccumulatorV2 /** * Low-level task scheduler interface, currently implemented exclusively by @@ -52,7 +52,14 @@ private[spark] trait TaskScheduler { def submitTasks(taskSet: TaskSet): Unit // Cancel a stage. - def cancelTasks(stageId: Int, interruptThread: Boolean) + def cancelTasks(stageId: Int, interruptThread: Boolean): Unit + + /** + * Kills a task attempt. + * + * @return Whether the task was successfully killed. + */ + def killTaskAttempt(taskId: Long, interruptThread: Boolean, reason: String): Boolean // Set the DAG scheduler for upcalls. This is guaranteed to be set before submitTasks is called. def setDAGScheduler(dagScheduler: DAGScheduler): Unit @@ -65,8 +72,10 @@ private[spark] trait TaskScheduler { * alive. Return true if the driver knows about the given block manager. Otherwise, return false, * indicating that the block manager should re-register. */ - def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], - blockManagerId: BlockManagerId): Boolean + def executorHeartbeatReceived( + execId: String, + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], + blockManagerId: BlockManagerId): Boolean /** * Get an application ID associated with the job. @@ -80,6 +89,11 @@ private[spark] trait TaskScheduler { */ def executorLost(executorId: String, reason: ExecutorLossReason): Unit + /** + * Process a removed worker + */ + def workerRemoved(workerId: String, host: String, message: String): Unit + /** * Get an application's attempt ID associated with the job. * 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 43d7d80b7aae..0c11806b3981 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -18,36 +18,35 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer -import java.util.{TimerTask, Timer} +import java.util.{Locale, Timer, TimerTask} import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicLong -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import scala.language.postfixOps +import scala.collection.Set +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality -import org.apache.spark.util.{ThreadUtils, Utils} -import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils} /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. - * It can also work with a local setup by using a LocalBackend and setting isLocal to true. - * It handles common logic, like determining a scheduling order across jobs, waking up to launch - * speculative tasks, etc. + * It can also work with a local setup by using a `LocalSchedulerBackend` and setting + * isLocal to true. It handles common logic, like determining a scheduling order across jobs, waking + * up to launch speculative tasks, etc. * * Clients should first call initialize() and start(), then submit task sets through the * runTasks method. * - * THREADING: SchedulerBackends and task-submitting clients can call this class from multiple + * THREADING: [[SchedulerBackend]]s and task-submitting clients can call this class from multiple * threads, so it needs locks in public API methods to maintain its state. In addition, some - * SchedulerBackends synchronize on themselves when they want to send events here, and then + * [[SchedulerBackend]]s synchronize on themselves when they want to send events here, and then * acquire a lock on us, so we need to make sure that we don't try to lock the backend while * we are holding a lock on ourselves. */ @@ -55,15 +54,28 @@ private[spark] class TaskSchedulerImpl( val sc: SparkContext, val maxTaskFailures: Int, isLocal: Boolean = false) - extends TaskScheduler with Logging -{ - def this(sc: SparkContext) = this(sc, sc.conf.getInt("spark.task.maxFailures", 4)) + extends TaskScheduler with Logging { + + import TaskSchedulerImpl._ + + def this(sc: SparkContext) = { + this(sc, sc.conf.get(config.MAX_TASK_FAILURES)) + } + + // Lazily initializing blackListTrackOpt to avoid getting empty ExecutorAllocationClient, + // because ExecutorAllocationClient is created after this TaskSchedulerImpl. + private[scheduler] lazy val blacklistTrackerOpt = maybeCreateBlacklistTracker(sc) val conf = sc.conf // How often to check for speculative tasks val SPECULATION_INTERVAL_MS = conf.getTimeAsMs("spark.speculation.interval", "100ms") + // Duplicate copies of a task will only be launched if the original copy has been running for + // at least this amount of time. This is to avoid the overhead of launching speculative copies + // of tasks that are very short. + val MIN_TIME_TO_SPECULATION = 100 + private val speculationScheduler = ThreadUtils.newDaemonSingleThreadScheduledExecutor("task-scheduler-speculation") @@ -77,6 +89,7 @@ private[spark] class TaskSchedulerImpl( // on this class. private val taskSetsByStageIdAndAttempt = new HashMap[Int, HashMap[Int, TaskSetManager]] + // Protected by `this` private[scheduler] val taskIdToTaskSetManager = new HashMap[Long, TaskSetManager] val taskIdToExecutorId = new HashMap[Long, String] @@ -87,12 +100,16 @@ private[spark] class TaskSchedulerImpl( // Incrementing task IDs val nextTaskId = new AtomicLong(0) - // Which executor IDs we have executors on - val activeExecutorIds = new HashSet[String] + // IDs of the tasks running on each executor + private val executorIdToRunningTaskIds = new HashMap[String, HashSet[Long]] + + def runningTasksByExecutors: Map[String, Int] = synchronized { + executorIdToRunningTaskIds.toMap.mapValues(_.size) + } // The set of executors we have on each host; this is used to compute hostsAlive, which // in turn is used to decide when we can attain data locality on a given host - protected val executorsByHost = new HashMap[String, HashSet[String]] + protected val hostToExecutors = new HashMap[String, HashSet[String]] protected val hostsByRack = new HashMap[String, HashSet[String]] @@ -103,18 +120,20 @@ private[spark] class TaskSchedulerImpl( var backend: SchedulerBackend = null - val mapOutputTracker = SparkEnv.get.mapOutputTracker + val mapOutputTracker = SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] - var schedulableBuilder: SchedulableBuilder = null - var rootPool: Pool = null + private var schedulableBuilder: SchedulableBuilder = null // default scheduler is FIFO - private val schedulingModeConf = conf.get("spark.scheduler.mode", "FIFO") - val schedulingMode: SchedulingMode = try { - SchedulingMode.withName(schedulingModeConf.toUpperCase) - } catch { - case e: java.util.NoSuchElementException => - throw new SparkException(s"Unrecognized spark.scheduler.mode: $schedulingModeConf") - } + private val schedulingModeConf = conf.get(SCHEDULER_MODE_PROPERTY, SchedulingMode.FIFO.toString) + val schedulingMode: SchedulingMode = + try { + SchedulingMode.withName(schedulingModeConf.toUpperCase(Locale.ROOT)) + } catch { + case e: java.util.NoSuchElementException => + throw new SparkException(s"Unrecognized $SCHEDULER_MODE_PROPERTY: $schedulingModeConf") + } + + val rootPool: Pool = new Pool("", schedulingMode, 0, 0) // This is a var so that we can reset it for testing purposes. private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) @@ -125,14 +144,15 @@ private[spark] class TaskSchedulerImpl( def initialize(backend: SchedulerBackend) { this.backend = backend - // temporarily set rootPool name to empty - rootPool = new Pool("", schedulingMode, 0, 0) schedulableBuilder = { schedulingMode match { case SchedulingMode.FIFO => new FIFOSchedulableBuilder(rootPool) case SchedulingMode.FAIR => new FairSchedulableBuilder(rootPool, conf) + case _ => + throw new IllegalArgumentException(s"Unsupported $SCHEDULER_MODE_PROPERTY: " + + s"$schedulingMode") } } schedulableBuilder.buildPools() @@ -145,7 +165,7 @@ private[spark] class TaskSchedulerImpl( if (!isLocal && conf.getBoolean("spark.speculation", false)) { logInfo("Starting speculative execution thread") - speculationScheduler.scheduleAtFixedRate(new Runnable { + speculationScheduler.scheduleWithFixedDelay(new Runnable { override def run(): Unit = Utils.tryOrStopSparkContext(sc) { checkSpeculatableTasks() } @@ -197,7 +217,7 @@ private[spark] class TaskSchedulerImpl( private[scheduler] def createTaskSetManager( taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { - new TaskSetManager(this, taskSet, maxTaskFailures) + new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt) } override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { @@ -211,8 +231,8 @@ private[spark] class TaskSchedulerImpl( // 2. The task set manager has been created but no tasks has been scheduled. In this case, // simply abort the stage. tsm.runningTasksSet.foreach { tid => - val execId = taskIdToExecutorId(tid) - backend.killTask(tid, execId, interruptThread) + taskIdToExecutorId.get(tid).foreach(execId => + backend.killTask(tid, execId, interruptThread, reason = "Stage cancelled")) } tsm.abort("Stage %s cancelled".format(stageId)) logInfo("Stage %d was cancelled".format(stageId)) @@ -220,6 +240,18 @@ private[spark] class TaskSchedulerImpl( } } + override def killTaskAttempt(taskId: Long, interruptThread: Boolean, reason: String): Boolean = { + logInfo(s"Killing task $taskId: $reason") + val execId = taskIdToExecutorId.get(taskId) + if (execId.isDefined) { + backend.killTask(taskId, execId.get, interruptThread, reason) + true + } else { + logWarning(s"Could not kill task $taskId because no task with that ID was found.") + false + } + } + /** * Called to indicate that all task attempts (including speculated tasks) associated with the * given TaskSetManager have completed, so state associated with the TaskSetManager should be @@ -233,8 +265,8 @@ private[spark] class TaskSchedulerImpl( } } manager.parent.removeSchedulable(manager) - logInfo("Removed TaskSet %s, whose tasks have all completed, from pool %s" - .format(manager.taskSet.id, manager.parent.name)) + logInfo(s"Removed TaskSet ${manager.taskSet.id}, whose tasks have all completed, from pool" + + s" ${manager.parent.name}") } private def resourceOfferSingleTaskSet( @@ -242,8 +274,10 @@ private[spark] class TaskSchedulerImpl( maxLocality: TaskLocality, shuffledOffers: Seq[WorkerOffer], availableCpus: Array[Int], - tasks: Seq[ArrayBuffer[TaskDescription]]) : Boolean = { + tasks: IndexedSeq[ArrayBuffer[TaskDescription]]) : Boolean = { var launchedTask = false + // nodes and executors that are blacklisted for the entire application have already been + // filtered out by this point for (i <- 0 until shuffledOffers.size) { val execId = shuffledOffers(i).executorId val host = shuffledOffers(i).host @@ -254,7 +288,7 @@ private[spark] class TaskSchedulerImpl( val tid = task.taskId taskIdToTaskSetManager(tid) = taskSet taskIdToExecutorId(tid) = execId - executorsByHost(host) += execId + executorIdToRunningTaskIds(execId).add(tid) availableCpus(i) -= CPUS_PER_TASK assert(availableCpus(i) >= 0) launchedTask = true @@ -276,16 +310,19 @@ private[spark] class TaskSchedulerImpl( * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so * that tasks are balanced across the cluster. */ - def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = synchronized { + def resourceOffers(offers: IndexedSeq[WorkerOffer]): Seq[Seq[TaskDescription]] = synchronized { // Mark each slave as alive and remember its hostname // Also track if new executor is added var newExecAvail = false for (o <- offers) { - executorIdToHost(o.executorId) = o.host - activeExecutorIds += o.executorId - if (!executorsByHost.contains(o.host)) { - executorsByHost(o.host) = new HashSet[String]() + if (!hostToExecutors.contains(o.host)) { + hostToExecutors(o.host) = new HashSet[String]() + } + if (!executorIdToRunningTaskIds.contains(o.executorId)) { + hostToExecutors(o.host) += o.executorId executorAdded(o.executorId, o.host) + executorIdToHost(o.executorId) = o.host + executorIdToRunningTaskIds(o.executorId) = HashSet[Long]() newExecAvail = true } for (rack <- getRackForHost(o.host)) { @@ -293,10 +330,21 @@ private[spark] class TaskSchedulerImpl( } } - // Randomly shuffle offers to avoid always placing tasks on the same set of workers. - val shuffledOffers = Random.shuffle(offers) + // Before making any offers, remove any nodes from the blacklist whose blacklist has expired. Do + // this here to avoid a separate thread and added synchronization overhead, and also because + // updating the blacklist is only relevant when task offers are being made. + blacklistTrackerOpt.foreach(_.applyBlacklistTimeout()) + + val filteredOffers = blacklistTrackerOpt.map { blacklistTracker => + offers.filter { offer => + !blacklistTracker.isNodeBlacklisted(offer.host) && + !blacklistTracker.isExecutorBlacklisted(offer.executorId) + } + }.getOrElse(offers) + + val shuffledOffers = shuffleOffers(filteredOffers) // Build a list of tasks to assign to each worker. - val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores)) + val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores / CPUS_PER_TASK)) val availableCpus = shuffledOffers.map(o => o.cores).toArray val sortedTaskSets = rootPool.getSortedTaskSetQueue for (taskSet <- sortedTaskSets) { @@ -310,12 +358,19 @@ private[spark] class TaskSchedulerImpl( // Take each TaskSet in our scheduling order, and then offer it each node in increasing order // of locality levels so that it gets a chance to launch local tasks on all of them. // NOTE: the preferredLocality order: PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY - var launchedTask = false - for (taskSet <- sortedTaskSets; maxLocality <- taskSet.myLocalityLevels) { - do { - launchedTask = resourceOfferSingleTaskSet( - taskSet, maxLocality, shuffledOffers, availableCpus, tasks) - } while (launchedTask) + for (taskSet <- sortedTaskSets) { + var launchedAnyTask = false + var launchedTaskAtCurrentMaxLocality = false + for (currentMaxLocality <- taskSet.myLocalityLevels) { + do { + launchedTaskAtCurrentMaxLocality = resourceOfferSingleTaskSet( + taskSet, currentMaxLocality, shuffledOffers, availableCpus, tasks) + launchedAnyTask |= launchedTaskAtCurrentMaxLocality + } while (launchedTaskAtCurrentMaxLocality) + } + if (!launchedAnyTask) { + taskSet.abortIfCompletelyBlacklisted(hostToExecutors) + } } if (tasks.size > 0) { @@ -324,36 +379,47 @@ private[spark] class TaskSchedulerImpl( return tasks } + /** + * Shuffle offers around to avoid always placing tasks on the same workers. Exposed to allow + * overriding in tests, so it can be deterministic. + */ + protected def shuffleOffers(offers: IndexedSeq[WorkerOffer]): IndexedSeq[WorkerOffer] = { + Random.shuffle(offers) + } + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { var failedExecutor: Option[String] = None + var reason: Option[ExecutorLossReason] = None synchronized { try { - if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) { - // We lost this entire executor, so remember that it's gone - val execId = taskIdToExecutorId(tid) - if (activeExecutorIds.contains(execId)) { - removeExecutor(execId, - SlaveLost(s"Task $tid was lost, so marking the executor as lost as well.")) - failedExecutor = Some(execId) - } - } taskIdToTaskSetManager.get(tid) match { case Some(taskSet) => - if (TaskState.isFinished(state)) { - taskIdToTaskSetManager.remove(tid) - taskIdToExecutorId.remove(tid) + if (state == TaskState.LOST) { + // TaskState.LOST is only used by the deprecated Mesos fine-grained scheduling mode, + // where each executor corresponds to a single task, so mark the executor as failed. + val execId = taskIdToExecutorId.getOrElse(tid, throw new IllegalStateException( + "taskIdToTaskSetManager.contains(tid) <=> taskIdToExecutorId.contains(tid)")) + if (executorIdToRunningTaskIds.contains(execId)) { + reason = Some( + SlaveLost(s"Task $tid was lost, so marking the executor as lost as well.")) + removeExecutor(execId, reason.get) + failedExecutor = Some(execId) + } } - if (state == TaskState.FINISHED) { - taskSet.removeRunningTask(tid) - taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) - } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { + if (TaskState.isFinished(state)) { + cleanupTaskState(tid) taskSet.removeRunningTask(tid) - taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData) + if (state == TaskState.FINISHED) { + taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) + } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { + taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData) + } } case None => logError( ("Ignoring update with state %s for TID %s because its task set is gone (this is " + - "likely the result of receiving duplicate task finished status updates)") + "likely the result of receiving duplicate task finished status updates) or its " + + "executor has been marked as failed.") .format(state, tid)) } } catch { @@ -362,7 +428,8 @@ private[spark] class TaskSchedulerImpl( } // Update the DAGScheduler without holding a lock on this, since that can deadlock if (failedExecutor.isDefined) { - dagScheduler.executorLost(failedExecutor.get) + assert(reason.isDefined) + dagScheduler.executorLost(failedExecutor.get, reason.get) backend.reviveOffers() } } @@ -374,17 +441,18 @@ private[spark] class TaskSchedulerImpl( */ override def executorHeartbeatReceived( execId: String, - taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean = { - - val metricsWithStageIds: Array[(Long, Int, Int, TaskMetrics)] = synchronized { - taskMetrics.flatMap { case (id, metrics) => + // (taskId, stageId, stageAttemptId, accumUpdates) + val accumUpdatesWithTaskIds: Array[(Long, Int, Int, Seq[AccumulableInfo])] = synchronized { + accumUpdates.flatMap { case (id, updates) => + val accInfos = updates.map(acc => acc.toInfo(Some(acc.value), None)) taskIdToTaskSetManager.get(id).map { taskSetMgr => - (id, taskSetMgr.stageId, taskSetMgr.taskSet.stageAttemptId, metrics) + (id, taskSetMgr.stageId, taskSetMgr.taskSet.stageAttemptId, accInfos) } } } - dagScheduler.executorHeartbeatReceived(execId, metricsWithStageIds, blockManagerId) + dagScheduler.executorHeartbeatReceived(execId, accumUpdatesWithTaskIds, blockManagerId) } def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long): Unit = synchronized { @@ -402,9 +470,9 @@ private[spark] class TaskSchedulerImpl( taskSetManager: TaskSetManager, tid: Long, taskState: TaskState, - reason: TaskEndReason): Unit = synchronized { + reason: TaskFailedReason): Unit = synchronized { taskSetManager.handleFailedTask(tid, taskState, reason) - if (!taskSetManager.isZombie && taskState != TaskState.KILLED) { + if (!taskSetManager.isZombie && !taskSetManager.someAttemptSucceeded(tid)) { // Need to revive offers again now that the task set manager state has been updated to // reflect failed tasks that need to be re-run. backend.reviveOffers() @@ -451,7 +519,7 @@ private[spark] class TaskSchedulerImpl( def checkSpeculatableTasks() { var shouldRevive = false synchronized { - shouldRevive = rootPool.checkSpeculatableTasks() + shouldRevive = rootPool.checkSpeculatableTasks(MIN_TIME_TO_SPECULATION) } if (shouldRevive) { backend.reviveOffers() @@ -462,47 +530,84 @@ private[spark] class TaskSchedulerImpl( var failedExecutor: Option[String] = None synchronized { - if (activeExecutorIds.contains(executorId)) { + if (executorIdToRunningTaskIds.contains(executorId)) { val hostPort = executorIdToHost(executorId) - logError("Lost executor %s on %s: %s".format(executorId, hostPort, reason)) + logExecutorLoss(executorId, hostPort, reason) removeExecutor(executorId, reason) failedExecutor = Some(executorId) } else { - executorIdToHost.get(executorId) match { - case Some(_) => - // If the host mapping still exists, it means we don't know the loss reason for the - // executor. So call removeExecutor() to update tasks running on that executor when - // the real loss reason is finally known. - removeExecutor(executorId, reason) - - case None => - // We may get multiple executorLost() calls with different loss reasons. For example, - // one may be triggered by a dropped connection from the slave while another may be a - // report of executor termination from Mesos. We produce log messages for both so we - // eventually report the termination reason. - logError("Lost an executor " + executorId + " (already removed): " + reason) - } + executorIdToHost.get(executorId) match { + case Some(hostPort) => + // If the host mapping still exists, it means we don't know the loss reason for the + // executor. So call removeExecutor() to update tasks running on that executor when + // the real loss reason is finally known. + logExecutorLoss(executorId, hostPort, reason) + removeExecutor(executorId, reason) + + case None => + // We may get multiple executorLost() calls with different loss reasons. For example, + // one may be triggered by a dropped connection from the slave while another may be a + // report of executor termination from Mesos. We produce log messages for both so we + // eventually report the termination reason. + logError(s"Lost an executor $executorId (already removed): $reason") + } } } // Call dagScheduler.executorLost without holding the lock on this to prevent deadlock if (failedExecutor.isDefined) { - dagScheduler.executorLost(failedExecutor.get) + dagScheduler.executorLost(failedExecutor.get, reason) backend.reviveOffers() } } + override def workerRemoved(workerId: String, host: String, message: String): Unit = { + logInfo(s"Handle removed worker $workerId: $message") + dagScheduler.workerRemoved(workerId, host, message) + } + + private def logExecutorLoss( + executorId: String, + hostPort: String, + reason: ExecutorLossReason): Unit = reason match { + case LossReasonPending => + logDebug(s"Executor $executorId on $hostPort lost, but reason not yet known.") + case ExecutorKilled => + logInfo(s"Executor $executorId on $hostPort killed by driver.") + case _ => + logError(s"Lost executor $executorId on $hostPort: $reason") + } + + /** + * Cleans up the TaskScheduler's state for tracking the given task. + */ + private def cleanupTaskState(tid: Long): Unit = { + taskIdToTaskSetManager.remove(tid) + taskIdToExecutorId.remove(tid).foreach { executorId => + executorIdToRunningTaskIds.get(executorId).foreach { _.remove(tid) } + } + } + /** * Remove an executor from all our data structures and mark it as lost. If the executor's loss * reason is not yet known, do not yet remove its association with its host nor update the status * of any running tasks, since the loss reason defines whether we'll fail those tasks. */ private def removeExecutor(executorId: String, reason: ExecutorLossReason) { - activeExecutorIds -= executorId + // The tasks on the lost executor may not send any more status updates (because the executor + // has been lost), so they should be cleaned up here. + executorIdToRunningTaskIds.remove(executorId).foreach { taskIds => + logDebug("Cleaning up TaskScheduler state for tasks " + + s"${taskIds.mkString("[", ",", "]")} on failed executor $executorId") + // We do not notify the TaskSetManager of the task failures because that will + // happen below in the rootPool.executorLost() call. + taskIds.foreach(cleanupTaskState) + } + val host = executorIdToHost(executorId) - val execs = executorsByHost.getOrElse(host, new HashSet) + val execs = hostToExecutors.getOrElse(host, new HashSet) execs -= executorId if (execs.isEmpty) { - executorsByHost -= host + hostToExecutors -= host for (rack <- getRackForHost(host); hosts <- hostsByRack.get(rack)) { hosts -= host if (hosts.isEmpty) { @@ -515,6 +620,7 @@ private[spark] class TaskSchedulerImpl( executorIdToHost -= executorId rootPool.executorLost(executorId, host, reason) } + blacklistTrackerOpt.foreach(_.handleRemovedExecutor(executorId)) } def executorAdded(execId: String, host: String) { @@ -522,11 +628,11 @@ private[spark] class TaskSchedulerImpl( } def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized { - executorsByHost.get(host).map(_.toSet) + hostToExecutors.get(host).map(_.toSet) } def hasExecutorsAliveOnHost(host: String): Boolean = synchronized { - executorsByHost.contains(host) + hostToExecutors.contains(host) } def hasHostAliveOnRack(rack: String): Boolean = synchronized { @@ -534,7 +640,19 @@ private[spark] class TaskSchedulerImpl( } def isExecutorAlive(execId: String): Boolean = synchronized { - activeExecutorIds.contains(execId) + executorIdToRunningTaskIds.contains(execId) + } + + def isExecutorBusy(execId: String): Boolean = synchronized { + executorIdToRunningTaskIds.get(execId).exists(_.nonEmpty) + } + + /** + * Get a snapshot of the currently blacklisted nodes for the entire application. This is + * thread-safe -- it can be called without a lock on the TaskScheduler. + */ + def nodeBlacklist(): scala.collection.immutable.Set[String] = { + blacklistTrackerOpt.map(_.nodeBlacklist()).getOrElse(scala.collection.immutable.Set()) } // By default, rack is unknown @@ -545,6 +663,11 @@ private[spark] class TaskSchedulerImpl( return } while (!backend.isReady) { + // Might take a while for backend to be ready if it is waiting on resources. + if (sc.stopped.get) { + // For example: the master removes the application for some reason + throw new IllegalStateException("Spark context stopped while waiting for backend") + } synchronized { this.wait(100) } @@ -570,16 +693,19 @@ private[spark] class TaskSchedulerImpl( private[spark] object TaskSchedulerImpl { + + val SCHEDULER_MODE_PROPERTY = "spark.scheduler.mode" + /** * Used to balance containers across hosts. * * Accepts a map of hosts to resource offers for that host, and returns a prioritized list of - * resource offers representing the order in which the offers should be used. The resource + * resource offers representing the order in which the offers should be used. The resource * offers are ordered such that we'll allocate one container on each host before allocating a * second container on any host, and so on, in order to reduce the damage if a host fails. * - * For example, given , , , returns - * [o1, o5, o4, 02, o6, o3] + * For example, given {@literal }, {@literal } and + * {@literal }, returns {@literal [o1, o5, o4, o2, o6, o3]}. */ def prioritizeContainers[K, T] (map: HashMap[K, ArrayBuffer[T]]): List[T] = { val _keyList = new ArrayBuffer[K](map.size) @@ -597,10 +723,10 @@ private[spark] object TaskSchedulerImpl { while (found) { found = false for (key <- keyList) { - val containerList: ArrayBuffer[T] = map.get(key).getOrElse(null) + val containerList: ArrayBuffer[T] = map.getOrElse(key, null) assert(containerList != null) // Get the index'th entry for this host - if present - if (index < containerList.size){ + if (index < containerList.size) { retval += containerList.apply(index) found = true } @@ -611,4 +737,16 @@ private[spark] object TaskSchedulerImpl { retval.toList } + private def maybeCreateBlacklistTracker(sc: SparkContext): Option[BlacklistTracker] = { + if (BlacklistTracker.isBlacklistEnabled(sc.conf)) { + val executorAllocClient: Option[ExecutorAllocationClient] = sc.schedulerBackend match { + case b: ExecutorAllocationClient => Some(b) + case _ => None + } + Some(new BlacklistTracker(sc, executorAllocClient)) + } else { + None + } + } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala index be8526ba9b94..517c8991aed7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala @@ -29,7 +29,7 @@ private[spark] class TaskSet( val stageAttemptId: Int, val priority: Int, val properties: Properties) { - val id: String = stageId + "." + stageAttemptId + val id: String = stageId + "." + stageAttemptId override def toString: String = "TaskSet " + id } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala new file mode 100644 index 000000000000..e815b7e0cf6c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.{HashMap, HashSet} + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config +import org.apache.spark.util.Clock + +/** + * Handles blacklisting executors and nodes within a taskset. This includes blacklisting specific + * (task, executor) / (task, nodes) pairs, and also completely blacklisting executors and nodes + * for the entire taskset. + * + * It also must store sufficient information in task failures for application level blacklisting, + * which is handled by [[BlacklistTracker]]. Note that BlacklistTracker does not know anything + * about task failures until a taskset completes successfully. + * + * THREADING: This class is a helper to [[TaskSetManager]]; as with the methods in + * [[TaskSetManager]] this class is designed only to be called from code with a lock on the + * TaskScheduler (e.g. its event handlers). It should not be called from other threads. + */ +private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int, val clock: Clock) + extends Logging { + + private val MAX_TASK_ATTEMPTS_PER_EXECUTOR = conf.get(config.MAX_TASK_ATTEMPTS_PER_EXECUTOR) + private val MAX_TASK_ATTEMPTS_PER_NODE = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) + private val MAX_FAILURES_PER_EXEC_STAGE = conf.get(config.MAX_FAILURES_PER_EXEC_STAGE) + private val MAX_FAILED_EXEC_PER_NODE_STAGE = conf.get(config.MAX_FAILED_EXEC_PER_NODE_STAGE) + + /** + * A map from each executor to the task failures on that executor. This is used for blacklisting + * within this taskset, and it is also relayed onto [[BlacklistTracker]] for app-level + * blacklisting if this taskset completes successfully. + */ + val execToFailures = new HashMap[String, ExecutorFailuresInTaskSet]() + + /** + * Map from node to all executors on it with failures. Needed because we want to know about + * executors on a node even after they have died. (We don't want to bother tracking the + * node -> execs mapping in the usual case when there aren't any failures). + */ + private val nodeToExecsWithFailures = new HashMap[String, HashSet[String]]() + private val nodeToBlacklistedTaskIndexes = new HashMap[String, HashSet[Int]]() + private val blacklistedExecs = new HashSet[String]() + private val blacklistedNodes = new HashSet[String]() + + /** + * Return true if this executor is blacklisted for the given task. This does *not* + * need to return true if the executor is blacklisted for the entire stage, or blacklisted + * for the entire application. That is to keep this method as fast as possible in the inner-loop + * of the scheduler, where those filters will have already been applied. + */ + def isExecutorBlacklistedForTask(executorId: String, index: Int): Boolean = { + execToFailures.get(executorId).exists { execFailures => + execFailures.getNumTaskFailures(index) >= MAX_TASK_ATTEMPTS_PER_EXECUTOR + } + } + + def isNodeBlacklistedForTask(node: String, index: Int): Boolean = { + nodeToBlacklistedTaskIndexes.get(node).exists(_.contains(index)) + } + + /** + * Return true if this executor is blacklisted for the given stage. Completely ignores whether + * the executor is blacklisted for the entire application (or anything to do with the node the + * executor is on). That is to keep this method as fast as possible in the inner-loop of the + * scheduler, where those filters will already have been applied. + */ + def isExecutorBlacklistedForTaskSet(executorId: String): Boolean = { + blacklistedExecs.contains(executorId) + } + + def isNodeBlacklistedForTaskSet(node: String): Boolean = { + blacklistedNodes.contains(node) + } + + private[scheduler] def updateBlacklistForFailedTask( + host: String, + exec: String, + index: Int): Unit = { + val execFailures = execToFailures.getOrElseUpdate(exec, new ExecutorFailuresInTaskSet(host)) + execFailures.updateWithFailure(index, clock.getTimeMillis()) + + // check if this task has also failed on other executors on the same host -- if its gone + // over the limit, blacklist this task from the entire host. + val execsWithFailuresOnNode = nodeToExecsWithFailures.getOrElseUpdate(host, new HashSet()) + execsWithFailuresOnNode += exec + val failuresOnHost = execsWithFailuresOnNode.toIterator.flatMap { exec => + execToFailures.get(exec).map { failures => + // We count task attempts here, not the number of unique executors with failures. This is + // because jobs are aborted based on the number task attempts; if we counted unique + // executors, it would be hard to config to ensure that you try another + // node before hitting the max number of task failures. + failures.getNumTaskFailures(index) + } + }.sum + if (failuresOnHost >= MAX_TASK_ATTEMPTS_PER_NODE) { + nodeToBlacklistedTaskIndexes.getOrElseUpdate(host, new HashSet()) += index + } + + // Check if enough tasks have failed on the executor to blacklist it for the entire stage. + if (execFailures.numUniqueTasksWithFailures >= MAX_FAILURES_PER_EXEC_STAGE) { + if (blacklistedExecs.add(exec)) { + logInfo(s"Blacklisting executor ${exec} for stage $stageId") + // This executor has been pushed into the blacklist for this stage. Let's check if it + // pushes the whole node into the blacklist. + val blacklistedExecutorsOnNode = + execsWithFailuresOnNode.filter(blacklistedExecs.contains(_)) + if (blacklistedExecutorsOnNode.size >= MAX_FAILED_EXEC_PER_NODE_STAGE) { + if (blacklistedNodes.add(host)) { + logInfo(s"Blacklisting ${host} for stage $stageId") + } + } + } + } + } +} 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 114468c48c44..3804ea863b4f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -19,20 +19,18 @@ package org.apache.spark.scheduler import java.io.NotSerializableException import java.nio.ByteBuffer -import java.util.Arrays import java.util.concurrent.ConcurrentLinkedQueue -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import scala.math.{min, max} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.math.max import scala.util.control.NonFatal import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.SchedulingMode._ import org.apache.spark.TaskState.TaskState -import org.apache.spark.util.{Clock, SystemClock, Utils} +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.SchedulingMode._ +import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, Utils} +import org.apache.spark.util.collection.MedianHeap /** * Schedules the tasks within a single TaskSet in the TaskSchedulerImpl. This class keeps track of @@ -53,19 +51,14 @@ private[spark] class TaskSetManager( sched: TaskSchedulerImpl, val taskSet: TaskSet, val maxTaskFailures: Int, - clock: Clock = new SystemClock()) - extends Schedulable with Logging { + blacklistTracker: Option[BlacklistTracker] = None, + clock: Clock = new SystemClock()) extends Schedulable with Logging { - val conf = sched.sc.conf + private val conf = sched.sc.conf - /* - * Sometimes if an executor is dead or in an otherwise invalid state, the driver - * does not realize right away leading to repeated task failures. If enabled, - * this temporarily prevents a task from re-launching on an executor where - * it just failed. - */ - private val EXECUTOR_TASK_BLACKLIST_TIMEOUT = - conf.getLong("spark.scheduler.executorTaskBlacklistTime", 0L) + // SPARK-21563 make a copy of the jars/files so they are consistent across the TaskSet + private val addedJars = HashMap[String, Long](sched.sc.addedJars.toSeq: _*) + private val addedFiles = HashMap[String, Long](sched.sc.addedFiles.toSeq: _*) // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = conf.getDouble("spark.speculation.quantile", 0.75) @@ -74,6 +67,8 @@ private[spark] class TaskSetManager( // Limit of bytes for total size of results (default is 1GB) val maxResultSize = Utils.getMaxResultSize(conf) + val speculationEnabled = conf.getBoolean("spark.speculation", false) + // Serializer for closures and tasks. val env = SparkEnv.get val ser = env.closureSerializer.newInstance() @@ -81,42 +76,59 @@ private[spark] class TaskSetManager( val tasks = taskSet.tasks val numTasks = tasks.length val copiesRunning = new Array[Int](numTasks) + + // For each task, tracks whether a copy of the task has succeeded. A task will also be + // marked as "succeeded" if it failed with a fetch failure, in which case it should not + // be re-run because the missing map data needs to be regenerated first. val successful = new Array[Boolean](numTasks) private val numFailures = new Array[Int](numTasks) - // key is taskId, value is a Map of executor id to when it failed - private val failedExecutors = new HashMap[Int, HashMap[String, Long]]() val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) - var tasksSuccessful = 0 + private[scheduler] var tasksSuccessful = 0 - var weight = 1 - var minShare = 0 + val weight = 1 + val minShare = 0 var priority = taskSet.priority var stageId = taskSet.stageId - var name = "TaskSet_" + taskSet.stageId.toString + val name = "TaskSet_" + taskSet.id var parent: Pool = null - var totalResultSize = 0L - var calculatedTasks = 0 + private var totalResultSize = 0L + private var calculatedTasks = 0 - val runningTasksSet = new HashSet[Long] + private[scheduler] val taskSetBlacklistHelperOpt: Option[TaskSetBlacklist] = { + blacklistTracker.map { _ => + new TaskSetBlacklist(conf, stageId, clock) + } + } + + private[scheduler] val runningTasksSet = new HashSet[Long] override def runningTasks: Int = runningTasksSet.size + def someAttemptSucceeded(tid: Long): Boolean = { + successful(taskInfos(tid).index) + } + // True once no more tasks should be launched for this task set manager. TaskSetManagers enter // the zombie state once at least one attempt of each task has completed successfully, or if the // task set is aborted (for example, because it was killed). TaskSetManagers remain in the zombie // state until all tasks have finished running; we keep TaskSetManagers that are in the zombie // state in order to continue to track and account for the running tasks. // TODO: We should kill any running task attempts when the task set manager becomes a zombie. - var isZombie = false + private[scheduler] var isZombie = false // Set of pending tasks for each executor. These collections are actually // treated as stacks, in which new tasks are added to the end of the // ArrayBuffer and removed from the end. This makes it faster to detect // tasks that repeatedly fail because whenever a task failed, it is put - // back at the head of the stack. They are also only cleaned up lazily; - // when a task is launched, it remains in all the pending lists except - // the one that it was launched from, but gets removed from them later. + // back at the head of the stack. These collections may contain duplicates + // for two reasons: + // (1): Tasks are only removed lazily; when a task is launched, it remains + // in all the pending lists except the one that it was launched from. + // (2): Tasks may be re-added to these lists multiple times as a result + // of failures. + // Duplicates are handled in dequeueTaskFromList, which ensures that a + // task hasn't already started running before launching it. private val pendingTasksForExecutor = new HashMap[String, ArrayBuffer[Int]] // Set of pending tasks for each host. Similar to pendingTasksForExecutor, @@ -127,17 +139,22 @@ private[spark] class TaskSetManager( private val pendingTasksForRack = new HashMap[String, ArrayBuffer[Int]] // Set containing pending tasks with no locality preferences. - var pendingTasksWithNoPrefs = new ArrayBuffer[Int] + private[scheduler] var pendingTasksWithNoPrefs = new ArrayBuffer[Int] // Set containing all pending tasks (also used as a stack, as above). - val allPendingTasks = new ArrayBuffer[Int] + private val allPendingTasks = new ArrayBuffer[Int] // Tasks that can be speculated. Since these will be a small fraction of total // tasks, we'll just hold them in a HashSet. - val speculatableTasks = new HashSet[Int] + private[scheduler] val speculatableTasks = new HashSet[Int] // Task index, start and finish time for each task attempt (indexed by task ID) - val taskInfos = new HashMap[Long, TaskInfo] + private val taskInfos = new HashMap[Long, TaskInfo] + + // Use a MedianHeap to record durations of successful tasks so we know when to launch + // speculative tasks. This is only used when speculation is enabled, to avoid the overhead + // of inserting into the heap when the heap won't be used. + val successfulTaskDurations = new MedianHeap() // How frequently to reprint duplicate exceptions in full, in milliseconds val EXCEPTION_PRINT_INTERVAL = @@ -146,7 +163,7 @@ private[spark] class TaskSetManager( // Map of recent exceptions (identified by string representation and top stack frame) to // duplicate count (how many times the same exception has appeared) and time the full exception // was printed. This should ideally be an LRU map that can drop old exceptions automatically. - val recentExceptions = HashMap[String, (Int, Long)]() + private val recentExceptions = HashMap[String, (Int, Long)]() // Figure out the current map output tracker epoch and set it on all tasks val epoch = sched.mapOutputTracker.getEpoch @@ -161,59 +178,57 @@ private[spark] class TaskSetManager( addPendingTask(i) } - // Figure out which locality levels we have in our TaskSet, so we can do delay scheduling - var myLocalityLevels = computeValidLocalityLevels() - var localityWaits = myLocalityLevels.map(getLocalityWait) // Time to wait at each level + /** + * Track the set of locality levels which are valid given the tasks locality preferences and + * the set of currently available executors. This is updated as executors are added and removed. + * This allows a performance optimization, of skipping levels that aren't relevant (eg., skip + * PROCESS_LOCAL if no tasks could be run PROCESS_LOCAL for the current set of executors). + */ + private[scheduler] var myLocalityLevels = computeValidLocalityLevels() + + // Time to wait at each level + private[scheduler] var localityWaits = myLocalityLevels.map(getLocalityWait) // Delay scheduling variables: we keep track of our current locality level and the time we // last launched a task at that level, and move up a level when localityWaits[curLevel] expires. // We then move down if we manage to launch a "more local" task. - var currentLocalityIndex = 0 // Index of our current locality level in validLocalityLevels - var lastLaunchTime = clock.getTimeMillis() // Time we last launched a task at this level + private var currentLocalityIndex = 0 // Index of our current locality level in validLocalityLevels + private var lastLaunchTime = clock.getTimeMillis() // Time we last launched a task at this level override def schedulableQueue: ConcurrentLinkedQueue[Schedulable] = null override def schedulingMode: SchedulingMode = SchedulingMode.NONE - var emittedTaskSizeWarning = false + private[scheduler] var emittedTaskSizeWarning = false /** Add a task to all the pending-task lists that it should be on. */ - private def addPendingTask(index: Int) { - // Utility method that adds `index` to a list only if it's not already there - def addTo(list: ArrayBuffer[Int]) { - if (!list.contains(index)) { - list += index - } - } - + private[spark] def addPendingTask(index: Int) { for (loc <- tasks(index).preferredLocations) { loc match { case e: ExecutorCacheTaskLocation => - addTo(pendingTasksForExecutor.getOrElseUpdate(e.executorId, new ArrayBuffer)) - case e: HDFSCacheTaskLocation => { + pendingTasksForExecutor.getOrElseUpdate(e.executorId, new ArrayBuffer) += index + case e: HDFSCacheTaskLocation => val exe = sched.getExecutorsAliveOnHost(loc.host) exe match { - case Some(set) => { + case Some(set) => for (e <- set) { - addTo(pendingTasksForExecutor.getOrElseUpdate(e, new ArrayBuffer)) + pendingTasksForExecutor.getOrElseUpdate(e, new ArrayBuffer) += index } logInfo(s"Pending task $index has a cached location at ${e.host} " + ", where there are executors " + set.mkString(",")) - } case None => logDebug(s"Pending task $index has a cached location at ${e.host} " + ", but there are no executors alive there.") } - } - case _ => Unit + case _ => } - addTo(pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer)) + pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer) += index for (rack <- sched.getRackForHost(loc.host)) { - addTo(pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer)) + pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index } } if (tasks(index).preferredLocations == Nil) { - addTo(pendingTasksWithNoPrefs) + pendingTasksWithNoPrefs += index } allPendingTasks += index // No point scanning this whole list to find the old task there @@ -249,12 +264,15 @@ private[spark] class TaskSetManager( * This method also cleans up any tasks in the list that have already * been launched, since we want that to happen lazily. */ - private def dequeueTaskFromList(execId: String, list: ArrayBuffer[Int]): Option[Int] = { + private def dequeueTaskFromList( + execId: String, + host: String, + list: ArrayBuffer[Int]): Option[Int] = { var indexOffset = list.size while (indexOffset > 0) { indexOffset -= 1 val index = list(indexOffset) - if (!executorIsBlacklisted(execId, index)) { + if (!isTaskBlacklistedOnExecOrNode(index, execId, host)) { // This should almost always be list.trimEnd(1) to remove tail list.remove(indexOffset) if (copiesRunning(index) == 0 && !successful(index)) { @@ -270,19 +288,11 @@ private[spark] class TaskSetManager( taskAttempts(taskIndex).exists(_.host == host) } - /** - * Is this re-execution of a failed task on an executor it already failed in before - * EXECUTOR_TASK_BLACKLIST_TIMEOUT has elapsed ? - */ - private def executorIsBlacklisted(execId: String, taskId: Int): Boolean = { - if (failedExecutors.contains(taskId)) { - val failed = failedExecutors.get(taskId).get - - return failed.contains(execId) && - clock.getTimeMillis() - failed.get(execId).get < EXECUTOR_TASK_BLACKLIST_TIMEOUT + private def isTaskBlacklistedOnExecOrNode(index: Int, execId: String, host: String): Boolean = { + taskSetBlacklistHelperOpt.exists { blacklist => + blacklist.isNodeBlacklistedForTask(host, index) || + blacklist.isExecutorBlacklistedForTask(execId, index) } - - false } /** @@ -296,8 +306,10 @@ private[spark] class TaskSetManager( { speculatableTasks.retain(index => !successful(index)) // Remove finished tasks from set - def canRunOnHost(index: Int): Boolean = - !hasAttemptOnHost(index, host) && !executorIsBlacklisted(execId, index) + def canRunOnHost(index: Int): Boolean = { + !hasAttemptOnHost(index, host) && + !isTaskBlacklistedOnExecOrNode(index, execId, host) + } if (!speculatableTasks.isEmpty) { // Check for process-local tasks; note that tasks can be process-local @@ -340,7 +352,7 @@ private[spark] class TaskSetManager( if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { for (rack <- sched.getRackForHost(host)) { for (index <- speculatableTasks if canRunOnHost(index)) { - val racks = tasks(index).preferredLocations.map(_.host).map(sched.getRackForHost) + val racks = tasks(index).preferredLocations.map(_.host).flatMap(sched.getRackForHost) if (racks.contains(rack)) { speculatableTasks -= index return Some((index, TaskLocality.RACK_LOCAL)) @@ -370,19 +382,19 @@ private[spark] class TaskSetManager( private def dequeueTask(execId: String, host: String, maxLocality: TaskLocality.Value) : Option[(Int, TaskLocality.Value, Boolean)] = { - for (index <- dequeueTaskFromList(execId, getPendingTasksForExecutor(execId))) { + for (index <- dequeueTaskFromList(execId, host, getPendingTasksForExecutor(execId))) { return Some((index, TaskLocality.PROCESS_LOCAL, false)) } if (TaskLocality.isAllowed(maxLocality, TaskLocality.NODE_LOCAL)) { - for (index <- dequeueTaskFromList(execId, getPendingTasksForHost(host))) { + for (index <- dequeueTaskFromList(execId, host, getPendingTasksForHost(host))) { return Some((index, TaskLocality.NODE_LOCAL, false)) } } if (TaskLocality.isAllowed(maxLocality, TaskLocality.NO_PREF)) { // Look for noPref tasks after NODE_LOCAL for minimize cross-rack traffic - for (index <- dequeueTaskFromList(execId, pendingTasksWithNoPrefs)) { + for (index <- dequeueTaskFromList(execId, host, pendingTasksWithNoPrefs)) { return Some((index, TaskLocality.PROCESS_LOCAL, false)) } } @@ -390,14 +402,14 @@ private[spark] class TaskSetManager( if (TaskLocality.isAllowed(maxLocality, TaskLocality.RACK_LOCAL)) { for { rack <- sched.getRackForHost(host) - index <- dequeueTaskFromList(execId, getPendingTasksForRack(rack)) + index <- dequeueTaskFromList(execId, host, getPendingTasksForRack(rack)) } { return Some((index, TaskLocality.RACK_LOCAL, false)) } } if (TaskLocality.isAllowed(maxLocality, TaskLocality.ANY)) { - for (index <- dequeueTaskFromList(execId, allPendingTasks)) { + for (index <- dequeueTaskFromList(execId, host, allPendingTasks)) { return Some((index, TaskLocality.ANY, false)) } } @@ -425,7 +437,11 @@ private[spark] class TaskSetManager( maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] = { - if (!isZombie) { + val offerBlacklisted = taskSetBlacklistHelperOpt.exists { blacklist => + blacklist.isNodeBlacklistedForTaskSet(host) || + blacklist.isExecutorBlacklistedForTaskSet(execId) + } + if (!isZombie && !offerBlacklisted) { val curTime = clock.getTimeMillis() var allowedLocality = maxLocality @@ -438,66 +454,77 @@ private[spark] class TaskSetManager( } } - dequeueTask(execId, host, allowedLocality) match { - case Some((index, taskLocality, speculative)) => { - // Found a task; do some bookkeeping and return a task description - val task = tasks(index) - val taskId = sched.newTaskId() - // Do various bookkeeping - copiesRunning(index) += 1 - val attemptNum = taskAttempts(index).size - val info = new TaskInfo(taskId, index, attemptNum, curTime, - execId, host, taskLocality, speculative) - taskInfos(taskId) = info - taskAttempts(index) = info :: taskAttempts(index) - // Update our locality level for delay scheduling - // NO_PREF will not affect the variables related to delay scheduling - if (maxLocality != TaskLocality.NO_PREF) { - currentLocalityIndex = getLocalityIndex(taskLocality) - lastLaunchTime = curTime - } - // Serialize and return the task - val startTime = clock.getTimeMillis() - val serializedTask: ByteBuffer = try { - Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) - } catch { - // If the task cannot be serialized, then there's no point to re-attempt the task, - // as it will always fail. So just abort the whole task-set. - case NonFatal(e) => - val msg = s"Failed to serialize task $taskId, not attempting to retry it." - logError(msg, e) - abort(s"$msg Exception during serialization: $e") - throw new TaskNotSerializableException(e) - } - if (serializedTask.limit > TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024 && - !emittedTaskSizeWarning) { - emittedTaskSizeWarning = true - logWarning(s"Stage ${task.stageId} contains a task of very large size " + - s"(${serializedTask.limit / 1024} KB). The maximum recommended task size is " + - s"${TaskSetManager.TASK_SIZE_TO_WARN_KB} KB.") - } - addRunningTask(taskId) - - // We used to log the time it takes to serialize the task, but task size is already - // a good proxy to task serialization time. - // val timeTaken = clock.getTime() - startTime - val taskName = s"task ${info.id} in stage ${taskSet.id}" - logInfo(s"Starting $taskName (TID $taskId, $host, partition ${task.partitionId}," + - s"$taskLocality, ${serializedTask.limit} bytes)") - - sched.dagScheduler.taskStarted(task, info) - return Some(new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, - taskName, index, serializedTask)) + dequeueTask(execId, host, allowedLocality).map { case ((index, taskLocality, speculative)) => + // Found a task; do some bookkeeping and return a task description + val task = tasks(index) + val taskId = sched.newTaskId() + // Do various bookkeeping + copiesRunning(index) += 1 + val attemptNum = taskAttempts(index).size + val info = new TaskInfo(taskId, index, attemptNum, curTime, + execId, host, taskLocality, speculative) + taskInfos(taskId) = info + taskAttempts(index) = info :: taskAttempts(index) + // Update our locality level for delay scheduling + // NO_PREF will not affect the variables related to delay scheduling + if (maxLocality != TaskLocality.NO_PREF) { + currentLocalityIndex = getLocalityIndex(taskLocality) + lastLaunchTime = curTime } - case _ => + // Serialize and return the task + val serializedTask: ByteBuffer = try { + ser.serialize(task) + } catch { + // If the task cannot be serialized, then there's no point to re-attempt the task, + // as it will always fail. So just abort the whole task-set. + case NonFatal(e) => + val msg = s"Failed to serialize task $taskId, not attempting to retry it." + logError(msg, e) + abort(s"$msg Exception during serialization: $e") + throw new TaskNotSerializableException(e) + } + if (serializedTask.limit > TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024 && + !emittedTaskSizeWarning) { + emittedTaskSizeWarning = true + logWarning(s"Stage ${task.stageId} contains a task of very large size " + + s"(${serializedTask.limit / 1024} KB). The maximum recommended task size is " + + s"${TaskSetManager.TASK_SIZE_TO_WARN_KB} KB.") + } + addRunningTask(taskId) + + // We used to log the time it takes to serialize the task, but task size is already + // a good proxy to task serialization time. + // val timeTaken = clock.getTime() - startTime + val taskName = s"task ${info.id} in stage ${taskSet.id}" + logInfo(s"Starting $taskName (TID $taskId, $host, executor ${info.executorId}, " + + s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit} bytes)") + + sched.dagScheduler.taskStarted(task, info) + new TaskDescription( + taskId, + attemptNum, + execId, + taskName, + index, + addedFiles, + addedJars, + task.localProperties, + serializedTask) } + } else { + None } - None } private def maybeFinishTaskSet() { if (isZombie && runningTasks == 0) { sched.taskSetFinished(this) + if (tasksSuccessful == numTasks) { + blacklistTracker.foreach(_.updateBlacklistForSuccessfulTaskSet( + taskSet.stageId, + taskSet.stageAttemptId, + taskSetBlacklistHelperOpt.get.execToFailures)) + } } } @@ -557,9 +584,9 @@ private[spark] class TaskSetManager( // Jump to the next locality level, and reset lastLaunchTime so that the next locality // wait timer doesn't immediately expire lastLaunchTime += localityWaits(currentLocalityIndex) - currentLocalityIndex += 1 - logDebug(s"Moving to ${myLocalityLevels(currentLocalityIndex)} after waiting for " + + logDebug(s"Moving to ${myLocalityLevels(currentLocalityIndex + 1)} after waiting for " + s"${localityWaits(currentLocalityIndex)}ms") + currentLocalityIndex += 1 } else { return myLocalityLevels(currentLocalityIndex) } @@ -580,12 +607,84 @@ private[spark] class TaskSetManager( index } + /** + * Check whether the given task set has been blacklisted to the point that it can't run anywhere. + * + * It is possible that this taskset has become impossible to schedule *anywhere* due to the + * blacklist. The most common scenario would be if there are fewer executors than + * spark.task.maxFailures. We need to detect this so we can fail the task set, otherwise the job + * will hang. + * + * There's a tradeoff here: we could make sure all tasks in the task set are schedulable, but that + * would add extra time to each iteration of the scheduling loop. Here, we take the approach of + * making sure at least one of the unscheduled tasks is schedulable. This means we may not detect + * the hang as quickly as we could have, but we'll always detect the hang eventually, and the + * method is faster in the typical case. In the worst case, this method can take + * O(maxTaskFailures + numTasks) time, but it will be faster when there haven't been any task + * failures (this is because the method picks one unscheduled task, and then iterates through each + * executor until it finds one that the task isn't blacklisted on). + */ + private[scheduler] def abortIfCompletelyBlacklisted( + hostToExecutors: HashMap[String, HashSet[String]]): Unit = { + taskSetBlacklistHelperOpt.foreach { taskSetBlacklist => + val appBlacklist = blacklistTracker.get + // Only look for unschedulable tasks when at least one executor has registered. Otherwise, + // task sets will be (unnecessarily) aborted in cases when no executors have registered yet. + if (hostToExecutors.nonEmpty) { + // find any task that needs to be scheduled + val pendingTask: Option[Int] = { + // usually this will just take the last pending task, but because of the lazy removal + // from each list, we may need to go deeper in the list. We poll from the end because + // failed tasks are put back at the end of allPendingTasks, so we're more likely to find + // an unschedulable task this way. + val indexOffset = allPendingTasks.lastIndexWhere { indexInTaskSet => + copiesRunning(indexInTaskSet) == 0 && !successful(indexInTaskSet) + } + if (indexOffset == -1) { + None + } else { + Some(allPendingTasks(indexOffset)) + } + } + + pendingTask.foreach { indexInTaskSet => + // try to find some executor this task can run on. Its possible that some *other* + // task isn't schedulable anywhere, but we will discover that in some later call, + // when that unschedulable task is the last task remaining. + val blacklistedEverywhere = hostToExecutors.forall { case (host, execsOnHost) => + // Check if the task can run on the node + val nodeBlacklisted = + appBlacklist.isNodeBlacklisted(host) || + taskSetBlacklist.isNodeBlacklistedForTaskSet(host) || + taskSetBlacklist.isNodeBlacklistedForTask(host, indexInTaskSet) + if (nodeBlacklisted) { + true + } else { + // Check if the task can run on any of the executors + execsOnHost.forall { exec => + appBlacklist.isExecutorBlacklisted(exec) || + taskSetBlacklist.isExecutorBlacklistedForTaskSet(exec) || + taskSetBlacklist.isExecutorBlacklistedForTask(exec, indexInTaskSet) + } + } + } + if (blacklistedEverywhere) { + val partition = tasks(indexInTaskSet).partitionId + abort(s"Aborting $taskSet because task $indexInTaskSet (partition $partition) " + + s"cannot run anywhere due to node and executor blacklist. Blacklisting behavior " + + s"can be configured via spark.blacklist.*.") + } + } + } + } + } + /** * Marks the task as getting result and notifies the DAG Scheduler */ def handleTaskGettingResult(tid: Long): Unit = { val info = taskInfos(tid) - info.markGettingResult() + info.markGettingResult(clock.getTimeMillis()) sched.dagScheduler.taskGettingResult(info) } @@ -608,25 +707,34 @@ private[spark] class TaskSetManager( } /** - * Marks the task as successful and notifies the DAGScheduler that a task has ended. + * Marks a task as successful and notifies the DAGScheduler that the task has ended. */ def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]): Unit = { val info = taskInfos(tid) val index = info.index - info.markSuccessful() + info.markFinished(TaskState.FINISHED, clock.getTimeMillis()) + if (speculationEnabled) { + successfulTaskDurations.insert(info.duration) + } removeRunningTask(tid) - // This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which holds the - // "TaskSchedulerImpl" lock until exiting. To avoid the SPARK-7655 issue, we should not - // "deserialize" the value when holding a lock to avoid blocking other threads. So we call - // "result.value()" in "TaskResultGetter.enqueueSuccessfulTask" before reaching here. - // Note: "result.value()" only deserializes the value when it's called at the first time, so - // here "result.value()" just returns the value and won't block other threads. - sched.dagScheduler.taskEnded( - tasks(index), Success, result.value(), result.accumUpdates, info, result.metrics) + + // Kill any other attempts for the same task (since those are unnecessary now that one + // attempt completed successfully). + for (attemptInfo <- taskAttempts(index) if attemptInfo.running) { + logInfo(s"Killing attempt ${attemptInfo.attemptNumber} for task ${attemptInfo.id} " + + s"in stage ${taskSet.id} (TID ${attemptInfo.taskId}) on ${attemptInfo.host} " + + s"as the attempt ${info.attemptNumber} succeeded on ${info.host}") + sched.backend.killTask( + attemptInfo.taskId, + attemptInfo.executorId, + interruptThread = true, + reason = "another attempt succeeded") + } if (!successful(index)) { tasksSuccessful += 1 - logInfo("Finished task %s in stage %s (TID %d) in %d ms on %s (%d/%d)".format( - info.id, taskSet.id, info.taskId, info.duration, info.host, tasksSuccessful, numTasks)) + logInfo(s"Finished task ${info.id} in stage ${taskSet.id} (TID ${info.taskId}) in" + + s" ${info.duration} ms on ${info.host} (executor ${info.executorId})" + + s" ($tasksSuccessful/$numTasks)") // Mark successful and stop if all the tasks have succeeded. successful(index) = true if (tasksSuccessful == numTasks) { @@ -636,7 +744,13 @@ private[spark] class TaskSetManager( logInfo("Ignoring task-finished event for " + info.id + " in stage " + taskSet.id + " because task " + index + " has already completed successfully") } - failedExecutors.remove(index) + // This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which holds the + // "TaskSchedulerImpl" lock until exiting. To avoid the SPARK-7655 issue, we should not + // "deserialize" the value when holding a lock to avoid blocking other threads. So we call + // "result.value()" in "TaskResultGetter.enqueueSuccessfulTask" before reaching here. + // Note: "result.value()" only deserializes the value when it's called at the first time, so + // here "result.value()" just returns the value and won't block other threads. + sched.dagScheduler.taskEnded(tasks(index), Success, result.value(), result.accumUpdates, info) maybeFinishTaskSet() } @@ -644,19 +758,18 @@ private[spark] class TaskSetManager( * Marks the task as failed, re-adds it to the list of pending tasks, and notifies the * DAG Scheduler. */ - def handleFailedTask(tid: Long, state: TaskState, reason: TaskEndReason) { + def handleFailedTask(tid: Long, state: TaskState, reason: TaskFailedReason) { val info = taskInfos(tid) - if (info.failed) { + if (info.failed || info.killed) { return } removeRunningTask(tid) - info.markFailed() + info.markFinished(state, clock.getTimeMillis()) val index = info.index copiesRunning(index) -= 1 - var taskMetrics : TaskMetrics = null - - val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}): " + - reason.asInstanceOf[TaskFailedReason].toErrorString + var accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty + val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}," + + s" executor ${info.executorId}): ${reason.toErrorString}" val failureException: Option[Throwable] = reason match { case fetchFailed: FetchFailed => logWarning(failureReason) @@ -664,12 +777,18 @@ private[spark] class TaskSetManager( successful(index) = true tasksSuccessful += 1 } - // Not adding to failed executors for FetchFailed. isZombie = true + + if (fetchFailed.bmAddress != null) { + blacklistTracker.foreach(_.updateBlacklistForFetchFailure( + fetchFailed.bmAddress.host, fetchFailed.bmAddress.executorId)) + } + None case ef: ExceptionFailure => - taskMetrics = ef.metrics.orNull + // ExceptionFailure's might have accumulator updates + accumUpdates = ef.accums if (ef.className == classOf[NotSerializableException].getName) { // If the task result wasn't serializable, there's no point in trying to re-execute it. logError("Task %s in stage %s (TID %d) had a not serializable result: %s; not retrying" @@ -699,13 +818,13 @@ private[spark] class TaskSetManager( logWarning(failureReason) } else { logInfo( - s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid) on executor ${info.host}: " + - s"${ef.className} (${ef.description}) [duplicate $dupCount]") + s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid) on ${info.host}, executor" + + s" ${info.executorId}: ${ef.className} (${ef.description}) [duplicate $dupCount]") } ef.exception case e: ExecutorLostFailure if !e.exitCausedByApp => - logInfo(s"Task $tid failed because while it was being computed, its executor" + + logInfo(s"Task $tid failed because while it was being computed, its executor " + "exited for a reason unrelated to the task. Not counting this failure towards the " + "maximum number of failures for the task.") None @@ -713,19 +832,13 @@ private[spark] class TaskSetManager( case e: TaskFailedReason => // TaskResultLost, TaskKilled, and others logWarning(failureReason) None - - case e: TaskEndReason => - logError("Unknown TaskEndReason: " + e) - None } - // always add to failed executors - failedExecutors.getOrElseUpdate(index, new HashMap[String, Long]()). - put(info.executorId, clock.getTimeMillis()) - sched.dagScheduler.taskEnded(tasks(index), reason, null, null, info, taskMetrics) - addPendingTask(index) - if (!isZombie && state != TaskState.KILLED - && reason.isInstanceOf[TaskFailedReason] - && reason.asInstanceOf[TaskFailedReason].countTowardsTaskFailures) { + + sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info) + + if (!isZombie && reason.countTowardsTaskFailures) { + taskSetBlacklistHelperOpt.foreach(_.updateBlacklistForFailedTask( + info.host, info.executorId, index)) assert (null != failureReason) numFailures(index) += 1 if (numFailures(index) >= maxTaskFailures) { @@ -736,6 +849,16 @@ private[spark] class TaskSetManager( return } } + + if (successful(index)) { + logInfo(s"Task ${info.id} in stage ${taskSet.id} (TID $tid) failed, but the task will not" + + s" be re-executed (either because the task failed with a shuffle data fetch failure," + + s" so the previous stage needs to be re-run, or because a different copy of the task" + + s" has already succeeded).") + } else { + addPendingTask(index) + } + maybeFinishTaskSet() } @@ -772,7 +895,7 @@ private[spark] class TaskSetManager( override def removeSchedulable(schedulable: Schedulable) {} override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { - var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager]() + val sortedTaskSetQueue = new ArrayBuffer[TaskSetManager]() sortedTaskSetQueue += this sortedTaskSetQueue } @@ -783,7 +906,8 @@ private[spark] class TaskSetManager( // and we are not using an external shuffle server which could serve the shuffle outputs. // The reason is the next stage wouldn't be able to fetch the data from this dead executor // so we would need to rerun these tasks on other executors. - if (tasks(0).isInstanceOf[ShuffleMapTask] && !env.blockManager.externalShuffleServiceEnabled) { + if (tasks(0).isInstanceOf[ShuffleMapTask] && !env.blockManager.externalShuffleServiceEnabled + && !isZombie) { for ((tid, info) <- taskInfos if info.executorId == execId) { val index = taskInfos(tid).index if (successful(index)) { @@ -793,13 +917,15 @@ private[spark] class TaskSetManager( addPendingTask(index) // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our // stage finishes when a total of tasks.size tasks finish. - sched.dagScheduler.taskEnded(tasks(index), Resubmitted, null, null, info, null) + sched.dagScheduler.taskEnded( + tasks(index), Resubmitted, null, Seq.empty, info) } } } for ((tid, info) <- taskInfos if info.running && info.executorId == execId) { val exitCausedByApp: Boolean = reason match { case exited: ExecutorExited => exited.exitCausedByApp + case ExecutorKilled => false case _ => true } handleFailedTask(tid, TaskState.FAILED, ExecutorLostFailure(info.executorId, exitCausedByApp, @@ -813,10 +939,8 @@ private[spark] class TaskSetManager( * Check for tasks to be speculated and return true if there are any. This is called periodically * by the TaskScheduler. * - * TODO: To make this scale to large jobs, we need to maintain a list of running tasks, so that - * we don't scan the whole task set. It might also help to make this sorted by launch time. */ - override def checkSpeculatableTasks(): Boolean = { + override def checkSpeculatableTasks(minTimeToSpeculation: Int): Boolean = { // Can't speculate if we only have one task, and no need to speculate if the task set is a // zombie. if (isZombie || numTasks == 1) { @@ -825,16 +949,16 @@ private[spark] class TaskSetManager( var foundTasks = false val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation) + if (tasksSuccessful >= minFinishedForSpeculation && tasksSuccessful > 0) { val time = clock.getTimeMillis() - val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray - Arrays.sort(durations) - val medianDuration = durations(min((0.5 * tasksSuccessful).round.toInt, durations.size - 1)) - val threshold = max(SPECULATION_MULTIPLIER * medianDuration, 100) + val medianDuration = successfulTaskDurations.median + val threshold = max(SPECULATION_MULTIPLIER * medianDuration, minTimeToSpeculation) // TODO: Threshold should also look at standard deviation of task durations and have a lower // bound based on that. logDebug("Task length threshold for speculation: " + threshold) - for ((tid, info) <- taskInfos) { + for (tid <- runningTasksSet) { + val info = taskInfos(tid) val index = info.index if (!successful(index) && copiesRunning(index) == 1 && info.timeRunning(time) > threshold && !speculatableTasks.contains(index)) { @@ -842,6 +966,7 @@ private[spark] class TaskSetManager( "Marking task %d in stage %s (on %s) as speculatable because it ran more than %.0f ms" .format(index, taskSet.id, info.host, threshold)) speculatableTasks += index + sched.dagScheduler.speculativeTaskSubmitted(tasks(index)) foundTasks = true } } @@ -873,18 +998,18 @@ private[spark] class TaskSetManager( private def computeValidLocalityLevels(): Array[TaskLocality.TaskLocality] = { import TaskLocality.{PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY} val levels = new ArrayBuffer[TaskLocality.TaskLocality] - if (!pendingTasksForExecutor.isEmpty && getLocalityWait(PROCESS_LOCAL) != 0 && + if (!pendingTasksForExecutor.isEmpty && pendingTasksForExecutor.keySet.exists(sched.isExecutorAlive(_))) { levels += PROCESS_LOCAL } - if (!pendingTasksForHost.isEmpty && getLocalityWait(NODE_LOCAL) != 0 && + if (!pendingTasksForHost.isEmpty && pendingTasksForHost.keySet.exists(sched.hasExecutorsAliveOnHost(_))) { levels += NODE_LOCAL } if (!pendingTasksWithNoPrefs.isEmpty) { levels += NO_PREF } - if (!pendingTasksForRack.isEmpty && getLocalityWait(RACK_LOCAL) != 0 && + if (!pendingTasksForRack.isEmpty && pendingTasksForRack.keySet.exists(sched.hasHostAliveOnRack(_))) { levels += RACK_LOCAL } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index f3d0d8547677..5d65731dfc30 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -22,24 +22,34 @@ import java.nio.ByteBuffer import org.apache.spark.TaskState.TaskState import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.ExecutorLossReason -import org.apache.spark.util.{SerializableBuffer, Utils} +import org.apache.spark.util.SerializableBuffer private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable private[spark] object CoarseGrainedClusterMessages { - case object RetrieveSparkProps extends CoarseGrainedClusterMessage + case object RetrieveSparkAppConfig extends CoarseGrainedClusterMessage + + case class SparkAppConfig( + sparkProperties: Seq[(String, String)], + ioEncryptionKey: Option[Array[Byte]], + hadoopDelegationCreds: Option[Array[Byte]]) + extends CoarseGrainedClusterMessage + + case object RetrieveLastAllocatedExecutorId extends CoarseGrainedClusterMessage // Driver to executors case class LaunchTask(data: SerializableBuffer) extends CoarseGrainedClusterMessage - case class KillTask(taskId: Long, executor: String, interruptThread: Boolean) + case class KillTask(taskId: Long, executor: String, interruptThread: Boolean, reason: String) + extends CoarseGrainedClusterMessage + + case class KillExecutorsOnHost(host: String) extends CoarseGrainedClusterMessage sealed trait RegisterExecutorResponse - case class RegisteredExecutor(hostname: String) extends CoarseGrainedClusterMessage - with RegisterExecutorResponse + case object RegisteredExecutor extends CoarseGrainedClusterMessage with RegisterExecutorResponse case class RegisterExecutorFailed(message: String) extends CoarseGrainedClusterMessage with RegisterExecutorResponse @@ -48,7 +58,7 @@ private[spark] object CoarseGrainedClusterMessages { case class RegisterExecutor( executorId: String, executorRef: RpcEndpointRef, - hostPort: String, + hostname: String, cores: Int, logUrls: Map[String, String]) extends CoarseGrainedClusterMessage @@ -76,6 +86,9 @@ private[spark] object CoarseGrainedClusterMessages { case class RemoveExecutor(executorId: String, reason: ExecutorLossReason) extends CoarseGrainedClusterMessage + case class RemoveWorker(workerId: String, host: String, message: String) + extends CoarseGrainedClusterMessage + case class SetupDriver(driver: RpcEndpointRef) extends CoarseGrainedClusterMessage // Exchanged between the driver and the AM in Yarn client mode @@ -93,7 +106,8 @@ private[spark] object CoarseGrainedClusterMessages { case class RequestExecutors( requestedTotal: Int, localityAwareTasks: Int, - hostToLocalTaskCount: Map[String, Int]) + hostToLocalTaskCount: Map[String, Int], + nodeBlacklist: Set[String]) extends CoarseGrainedClusterMessage // Check if an executor was force-killed but for a reason unrelated to the running tasks. diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index f71d98feac05..424e43b25c77 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -19,18 +19,25 @@ package org.apache.spark.scheduler.cluster import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicInteger +import javax.annotation.concurrent.GuardedBy import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.concurrent.Future +import org.apache.hadoop.security.UserGroupInformation + +import org.apache.spark.{ExecutorAllocationClient, SparkEnv, SparkException, TaskState} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.security.HadoopDelegationTokenManager +import org.apache.spark.internal.Logging import org.apache.spark.rpc._ -import org.apache.spark.{ExecutorAllocationClient, Logging, SparkEnv, SparkException, TaskState} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend.ENDPOINT_NAME -import org.apache.spark.util.{ThreadUtils, SerializableBuffer, AkkaUtils, Utils} +import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils, Utils} /** - * A scheduler backend that waits for coarse grained executors to connect to it through Akka. + * A scheduler backend that waits for coarse-grained executors to connect. * This backend holds onto each executor for the duration of the Spark job rather than relinquishing * executors whenever a task is done and asking the scheduler to launch a new executor for * each new task. Executors may be launched in a variety of ways, such as Mesos tasks for the @@ -39,52 +46,70 @@ import org.apache.spark.util.{ThreadUtils, SerializableBuffer, AkkaUtils, Utils} */ private[spark] class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: RpcEnv) - extends ExecutorAllocationClient with SchedulerBackend with Logging -{ + extends ExecutorAllocationClient with SchedulerBackend with Logging { + // Use an atomic variable to track total number of cores in the cluster for simplicity and speed - var totalCoreCount = new AtomicInteger(0) + protected val totalCoreCount = new AtomicInteger(0) // Total number of executors that are currently registered - var totalRegisteredExecutors = new AtomicInteger(0) - val conf = scheduler.sc.conf - private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) + protected val totalRegisteredExecutors = new AtomicInteger(0) + protected val conf = scheduler.sc.conf + private val maxRpcMessageSize = RpcUtils.maxMessageSizeBytes(conf) + private val defaultAskTimeout = RpcUtils.askRpcTimeout(conf) // Submit tasks only after (registered resources / total expected resources) // is equal to at least this value, that is double between 0 and 1. - var minRegisteredRatio = + private val _minRegisteredRatio = math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0)) // Submit tasks after maxRegisteredWaitingTime milliseconds // if minRegisteredRatio has not yet been reached - val maxRegisteredWaitingTimeMs = + private val maxRegisteredWaitingTimeMs = conf.getTimeAsMs("spark.scheduler.maxRegisteredResourcesWaitingTime", "30s") - val createTime = System.currentTimeMillis() + private val createTime = System.currentTimeMillis() + // Accessing `executorDataMap` in `DriverEndpoint.receive/receiveAndReply` doesn't need any + // protection. But accessing `executorDataMap` out of `DriverEndpoint.receive/receiveAndReply` + // must be protected by `CoarseGrainedSchedulerBackend.this`. Besides, `executorDataMap` should + // only be modified in `DriverEndpoint.receive/receiveAndReply` with protection by + // `CoarseGrainedSchedulerBackend.this`. private val executorDataMap = new HashMap[String, ExecutorData] + // Number of executors requested by the cluster manager, [[ExecutorAllocationManager]] + @GuardedBy("CoarseGrainedSchedulerBackend.this") + private var requestedTotalExecutors = 0 + // Number of executors requested from the cluster manager that have not registered yet + @GuardedBy("CoarseGrainedSchedulerBackend.this") private var numPendingExecutors = 0 private val listenerBus = scheduler.sc.listenerBus - // Executors we have requested the cluster manager to kill that have not died yet - private val executorsPendingToRemove = new HashSet[String] + // Executors we have requested the cluster manager to kill that have not died yet; maps + // the executor ID to whether it was explicitly killed by the driver (and thus shouldn't + // be considered an app-related failure). + @GuardedBy("CoarseGrainedSchedulerBackend.this") + private val executorsPendingToRemove = new HashMap[String, Boolean] // A map to store hostname with its possible task number running on it + @GuardedBy("CoarseGrainedSchedulerBackend.this") protected var hostToLocalTaskCount: Map[String, Int] = Map.empty // The number of pending tasks which is locality required + @GuardedBy("CoarseGrainedSchedulerBackend.this") protected var localityAwareTasks = 0 - // Executors that have been lost, but for which we don't yet know the real exit reason. - protected val executorsPendingLossReason = new HashSet[String] + // The num of current max ExecutorId used to re-register appMaster + @volatile protected var currentExecutorIdCounter = 0 + + // hadoop token manager used by some sub-classes (e.g. Mesos) + def hadoopDelegationTokenManager: Option[HadoopDelegationTokenManager] = None + + // Hadoop delegation tokens to be sent to the executors. + val hadoopDelegationCreds: Option[Array[Byte]] = getHadoopDelegationCreds() class DriverEndpoint(override val rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) extends ThreadSafeRpcEndpoint with Logging { - // If this DriverEndpoint is changed to support multiple threads, - // then this may need to be changed so that we don't share the serializer - // instance across threads - private val ser = SparkEnv.get.closureSerializer.newInstance() - - override protected def log = CoarseGrainedSchedulerBackend.this.log + // Executors that have been lost, but for which we don't yet know the real exit reason. + protected val executorsPendingLossReason = new HashSet[String] protected val addressToExecutorId = new HashMap[RpcAddress, String] @@ -120,21 +145,36 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp case ReviveOffers => makeOffers() - case KillTask(taskId, executorId, interruptThread) => + case KillTask(taskId, executorId, interruptThread, reason) => executorDataMap.get(executorId) match { case Some(executorInfo) => - executorInfo.executorEndpoint.send(KillTask(taskId, executorId, interruptThread)) + executorInfo.executorEndpoint.send( + KillTask(taskId, executorId, interruptThread, reason)) case None => // Ignoring the task kill since the executor is not registered. logWarning(s"Attempted to kill task $taskId for unknown executor $executorId.") } + + case KillExecutorsOnHost(host) => + scheduler.getExecutorsAliveOnHost(host).foreach { exec => + killExecutors(exec.toSeq, replace = true, force = true) + } } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RegisterExecutor(executorId, executorRef, hostPort, cores, logUrls) => + case RegisterExecutor(executorId, executorRef, hostname, cores, logUrls) => if (executorDataMap.contains(executorId)) { - context.reply(RegisterExecutorFailed("Duplicate executor ID: " + executorId)) + executorRef.send(RegisterExecutorFailed("Duplicate executor ID: " + executorId)) + context.reply(true) + } else if (scheduler.nodeBlacklist != null && + scheduler.nodeBlacklist.contains(hostname)) { + // If the cluster manager gives us an executor on a blacklisted node (because it + // already started allocating those resources before we informed it of our blacklist, + // or if it ignored our blacklist), then we reject that executor immediately. + logInfo(s"Rejecting $executorId as it has been blacklisted.") + executorRef.send(RegisterExecutorFailed(s"Executor is blacklisted: $executorId")) + context.reply(true) } else { // If the executor's rpc env is not listening for incoming connections, `hostPort` // will be null, and the client connection should be used to contact the executor. @@ -147,19 +187,23 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp addressToExecutorId(executorAddress) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) - val data = new ExecutorData(executorRef, executorRef.address, executorAddress.host, + val data = new ExecutorData(executorRef, executorRef.address, hostname, cores, cores, logUrls) // This must be synchronized because variables mutated // in this block are read when requesting executors CoarseGrainedSchedulerBackend.this.synchronized { executorDataMap.put(executorId, data) + if (currentExecutorIdCounter < executorId.toInt) { + currentExecutorIdCounter = executorId.toInt + } if (numPendingExecutors > 0) { numPendingExecutors -= 1 logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") } } + executorRef.send(RegisteredExecutor) // Note: some tests expect the reply to come after we put the executor in the map - context.reply(RegisteredExecutor(executorAddress.host)) + context.reply(true) listenerBus.post( SparkListenerExecutorAdded(System.currentTimeMillis(), executorId, data)) makeOffers() @@ -177,21 +221,40 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp context.reply(true) case RemoveExecutor(executorId, reason) => + // We will remove the executor's state and cannot restore it. However, the connection + // between the driver and the executor may be still alive so that the executor won't exit + // automatically, so try to tell the executor to stop itself. See SPARK-13519. + executorDataMap.get(executorId).foreach(_.executorEndpoint.send(StopExecutor)) removeExecutor(executorId, reason) context.reply(true) - case RetrieveSparkProps => - context.reply(sparkProperties) + case RemoveWorker(workerId, host, message) => + removeWorker(workerId, host, message) + context.reply(true) + + case RetrieveSparkAppConfig => + val reply = SparkAppConfig( + sparkProperties, + SparkEnv.get.securityManager.getIOEncryptionKey(), + hadoopDelegationCreds) + context.reply(reply) } // Make fake resource offers on all executors private def makeOffers() { - // Filter out executors under killing - val activeExecutors = executorDataMap.filterKeys(executorIsAlive) - val workOffers = activeExecutors.map { case (id, executorData) => - new WorkerOffer(id, executorData.executorHost, executorData.freeCores) - }.toSeq - launchTasks(scheduler.resourceOffers(workOffers)) + // Make sure no executor is killed while some task is launching on it + val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized { + // Filter out executors under killing + val activeExecutors = executorDataMap.filterKeys(executorIsAlive) + val workOffers = activeExecutors.map { + case (id, executorData) => + new WorkerOffer(id, executorData.executorHost, executorData.freeCores) + }.toIndexedSeq + scheduler.resourceOffers(workOffers) + } + if (!taskDescs.isEmpty) { + launchTasks(taskDescs) + } } override def onDisconnected(remoteAddress: RpcAddress): Unit = { @@ -204,12 +267,20 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Make fake resource offers on just one executor private def makeOffers(executorId: String) { - // Filter out executors under killing - if (executorIsAlive(executorId)) { - val executorData = executorDataMap(executorId) - val workOffers = Seq( - new WorkerOffer(executorId, executorData.executorHost, executorData.freeCores)) - launchTasks(scheduler.resourceOffers(workOffers)) + // Make sure no executor is killed while some task is launching on it + val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized { + // Filter out executors under killing + if (executorIsAlive(executorId)) { + val executorData = executorDataMap(executorId) + val workOffers = IndexedSeq( + new WorkerOffer(executorId, executorData.executorHost, executorData.freeCores)) + scheduler.resourceOffers(workOffers) + } else { + Seq.empty + } + } + if (!taskDescs.isEmpty) { + launchTasks(taskDescs) } } @@ -221,15 +292,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Launch tasks returned by a set of resource offers private def launchTasks(tasks: Seq[Seq[TaskDescription]]) { for (task <- tasks.flatten) { - val serializedTask = ser.serialize(task) - if (serializedTask.limit >= akkaFrameSize - AkkaUtils.reservedSizeBytes) { + val serializedTask = TaskDescription.encode(task) + if (serializedTask.limit >= maxRpcMessageSize) { scheduler.taskIdToTaskSetManager.get(task.taskId).foreach { taskSetMgr => try { var msg = "Serialized task %s:%d was %d bytes, which exceeds max allowed: " + - "spark.akka.frameSize (%d bytes) - reserved (%d bytes). Consider increasing " + - "spark.akka.frameSize or using broadcast variables for large values." - msg = msg.format(task.taskId, task.index, serializedTask.limit, akkaFrameSize, - AkkaUtils.reservedSizeBytes) + "spark.rpc.message.maxSize (%d bytes). Consider increasing " + + "spark.rpc.message.maxSize or using broadcast variables for large values." + msg = msg.format(task.taskId, task.index, serializedTask.limit, maxRpcMessageSize) taskSetMgr.abort(msg) } catch { case e: Exception => logError("Exception in error callback", e) @@ -239,37 +309,55 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp else { val executorData = executorDataMap(task.executorId) executorData.freeCores -= scheduler.CPUS_PER_TASK + + logDebug(s"Launching task ${task.taskId} on executor id: ${task.executorId} hostname: " + + s"${executorData.executorHost}.") + executorData.executorEndpoint.send(LaunchTask(new SerializableBuffer(serializedTask))) } } } // Remove a disconnected slave from the cluster - def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = { + private def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = { + logDebug(s"Asked to remove executor $executorId with reason $reason") executorDataMap.get(executorId) match { case Some(executorInfo) => // This must be synchronized because variables mutated // in this block are read when requesting executors - CoarseGrainedSchedulerBackend.this.synchronized { + val killed = CoarseGrainedSchedulerBackend.this.synchronized { addressToExecutorId -= executorInfo.executorAddress executorDataMap -= executorId - executorsPendingToRemove -= executorId executorsPendingLossReason -= executorId + executorsPendingToRemove.remove(executorId).getOrElse(false) } totalCoreCount.addAndGet(-executorInfo.totalCores) totalRegisteredExecutors.addAndGet(-1) - scheduler.executorLost(executorId, reason) + scheduler.executorLost(executorId, if (killed) ExecutorKilled else reason) listenerBus.post( SparkListenerExecutorRemoved(System.currentTimeMillis(), executorId, reason.toString)) - case None => logInfo(s"Asked to remove non-existent executor $executorId") + case None => + // SPARK-15262: If an executor is still alive even after the scheduler has removed + // its metadata, we may receive a heartbeat from that executor and tell its block + // manager to reregister itself. If that happens, the block manager master will know + // about the executor, but the scheduler will not. Therefore, we should remove the + // executor from the block manager when we hit this case. + scheduler.sc.env.blockManager.master.removeExecutorAsync(executorId) + logInfo(s"Asked to remove non-existent executor $executorId") } } + // Remove a lost worker from the cluster + private def removeWorker(workerId: String, host: String, message: String): Unit = { + logDebug(s"Asked to remove worker $workerId with reason $message") + scheduler.workerRemoved(workerId, host, message) + } + /** * Stop making resource offers for the given executor. The executor is marked as lost with * the loss reason still pending. * - * @return Whether executor was alive. + * @return Whether executor should be disabled */ protected def disableExecutor(executorId: String): Boolean = { val shouldDisable = CoarseGrainedSchedulerBackend.this.synchronized { @@ -277,7 +365,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorsPendingLossReason += executorId true } else { - false + // Returns true for explicitly killed executors, we also need to get pending loss reasons; + // For others return false. + executorsPendingToRemove.contains(executorId) } } @@ -295,7 +385,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } var driverEndpoint: RpcEndpointRef = null - val taskIdsOnSlave = new HashMap[String, HashSet[String]] + + protected def minRegisteredRatio: Double = _minRegisteredRatio override def start() { val properties = new ArrayBuffer[(String, String)] @@ -306,7 +397,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } // TODO (prashant) send conf instead of properties - driverEndpoint = rpcEnv.setupEndpoint(ENDPOINT_NAME, createDriverEndpoint(properties)) + driverEndpoint = createDriverEndpointRef(properties) + } + + protected def createDriverEndpointRef( + properties: ArrayBuffer[(String, String)]): RpcEndpointRef = { + rpcEnv.setupEndpoint(ENDPOINT_NAME, createDriverEndpoint(properties)) } protected def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = { @@ -317,7 +413,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp try { if (driverEndpoint != null) { logInfo("Shutting down all executors") - driverEndpoint.askWithRetry[Boolean](StopExecutors) + driverEndpoint.askSync[Boolean](StopExecutors) } } catch { case e: Exception => @@ -329,7 +425,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp stopExecutors() try { if (driverEndpoint != null) { - driverEndpoint.askWithRetry[Boolean](StopDriver) + driverEndpoint.askSync[Boolean](StopDriver) } } catch { case e: Exception => @@ -337,26 +433,51 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } } + /** + * Reset the state of CoarseGrainedSchedulerBackend to the initial state. Currently it will only + * be called in the yarn-client mode when AM re-registers after a failure. + * */ + protected def reset(): Unit = { + val executors: Set[String] = synchronized { + requestedTotalExecutors = 0 + numPendingExecutors = 0 + executorsPendingToRemove.clear() + executorDataMap.keys.toSet + } + + // Remove all the lingering executors that should be removed but not yet. The reason might be + // because (1) disconnected event is not yet received; (2) executors die silently. + executors.foreach { eid => + removeExecutor(eid, SlaveLost("Stale executor after cluster manager re-registered.")) + } + } + override def reviveOffers() { driverEndpoint.send(ReviveOffers) } - override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { - driverEndpoint.send(KillTask(taskId, executorId, interruptThread)) + override def killTask( + taskId: Long, executorId: String, interruptThread: Boolean, reason: String) { + driverEndpoint.send(KillTask(taskId, executorId, interruptThread, reason)) } override def defaultParallelism(): Int = { conf.getInt("spark.default.parallelism", math.max(totalCoreCount.get(), 2)) } - // Called by subclasses when notified of a lost worker - def removeExecutor(executorId: String, reason: ExecutorLossReason) { - try { - driverEndpoint.askWithRetry[Boolean](RemoveExecutor(executorId, reason)) - } catch { - case e: Exception => - throw new SparkException("Error notifying standalone scheduler's driver endpoint", e) - } + /** + * Called by subclasses when notified of a lost worker. It just fires the message and returns + * at once. + */ + protected def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = { + // Only log the failure since we don't care about the result. + driverEndpoint.ask[Boolean](RemoveExecutor(executorId, reason)).failed.foreach(t => + logError(t.getMessage, t))(ThreadUtils.sameThread) + } + + protected def removeWorker(workerId: String, host: String, message: String): Unit = { + driverEndpoint.ask[Boolean](RemoveWorker(workerId, host, message)).failed.foreach(t => + logError(t.getMessage, t))(ThreadUtils.sameThread) } def sufficientResourcesRegistered(): Boolean = true @@ -378,25 +499,43 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Return the number of executors currently registered with this backend. */ - def numExistingExecutors: Int = executorDataMap.size + private def numExistingExecutors: Int = executorDataMap.size + + override def getExecutorIds(): Seq[String] = { + executorDataMap.keySet.toSeq + } /** * Request an additional number of executors from the cluster manager. * @return whether the request is acknowledged. */ - final override def requestExecutors(numAdditionalExecutors: Int): Boolean = synchronized { + final override def requestExecutors(numAdditionalExecutors: Int): Boolean = { if (numAdditionalExecutors < 0) { throw new IllegalArgumentException( "Attempted to request a negative number of additional executor(s) " + s"$numAdditionalExecutors from the cluster manager. Please specify a positive number!") } logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager") - logDebug(s"Number of pending executors is now $numPendingExecutors") - numPendingExecutors += numAdditionalExecutors - // Account for executors pending to be added or removed - val newTotal = numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size - doRequestTotalExecutors(newTotal) + val response = synchronized { + requestedTotalExecutors += numAdditionalExecutors + numPendingExecutors += numAdditionalExecutors + logDebug(s"Number of pending executors is now $numPendingExecutors") + if (requestedTotalExecutors != + (numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size)) { + logDebug( + s"""requestExecutors($numAdditionalExecutors): Executor request doesn't match: + |requestedTotalExecutors = $requestedTotalExecutors + |numExistingExecutors = $numExistingExecutors + |numPendingExecutors = $numPendingExecutors + |executorsPendingToRemove = ${executorsPendingToRemove.size}""".stripMargin) + } + + // Account for executors pending to be added or removed + doRequestTotalExecutors(requestedTotalExecutors) + } + + defaultAskTimeout.awaitResult(response) } /** @@ -417,19 +556,25 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp numExecutors: Int, localityAwareTasks: Int, hostToLocalTaskCount: Map[String, Int] - ): Boolean = synchronized { + ): Boolean = { if (numExecutors < 0) { throw new IllegalArgumentException( "Attempted to request a negative number of executor(s) " + s"$numExecutors from the cluster manager. Please specify a positive number!") } - this.localityAwareTasks = localityAwareTasks - this.hostToLocalTaskCount = hostToLocalTaskCount + val response = synchronized { + this.requestedTotalExecutors = numExecutors + this.localityAwareTasks = localityAwareTasks + this.hostToLocalTaskCount = hostToLocalTaskCount + + numPendingExecutors = + math.max(numExecutors - numExistingExecutors + executorsPendingToRemove.size, 0) - numPendingExecutors = - math.max(numExecutors - numExistingExecutors + executorsPendingToRemove.size, 0) - doRequestTotalExecutors(numExecutors) + doRequestTotalExecutors(numExecutors) + } + + defaultAskTimeout.awaitResult(response) } /** @@ -442,55 +587,117 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp * insufficient resources to satisfy the first request. We make the assumption here that the * cluster manager will eventually fulfill all requests when resources free up. * - * @return whether the request is acknowledged. + * @return a future whose evaluation indicates whether the request is acknowledged. */ - protected def doRequestTotalExecutors(requestedTotal: Int): Boolean = false - - /** - * Request that the cluster manager kill the specified executors. - * @return whether the kill request is acknowledged. - */ - final override def killExecutors(executorIds: Seq[String]): Boolean = synchronized { - killExecutors(executorIds, replace = false) - } + protected def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = + Future.successful(false) /** * Request that the cluster manager kill the specified executors. * + * When asking the executor to be replaced, the executor loss is considered a failure, and + * killed tasks that are running on the executor will count towards the failure limits. If no + * replacement is being requested, then the tasks will not count towards the limit. + * * @param executorIds identifiers of executors to kill - * @param replace whether to replace the killed executors with new ones - * @return whether the kill request is acknowledged. + * @param replace whether to replace the killed executors with new ones, default false + * @param force whether to force kill busy executors, default false + * @return the ids of the executors acknowledged by the cluster manager to be removed. */ - final def killExecutors(executorIds: Seq[String], replace: Boolean): Boolean = synchronized { + final override def killExecutors( + executorIds: Seq[String], + replace: Boolean, + force: Boolean): Seq[String] = { logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}") - val (knownExecutors, unknownExecutors) = executorIds.partition(executorDataMap.contains) - unknownExecutors.foreach { id => - logWarning(s"Executor to kill $id does not exist!") - } - // If an executor is already pending to be removed, do not kill it again (SPARK-9795) - val executorsToKill = knownExecutors.filter { id => !executorsPendingToRemove.contains(id) } - executorsPendingToRemove ++= executorsToKill + val response = synchronized { + val (knownExecutors, unknownExecutors) = executorIds.partition(executorDataMap.contains) + unknownExecutors.foreach { id => + logWarning(s"Executor to kill $id does not exist!") + } - // If we do not wish to replace the executors we kill, sync the target number of executors - // with the cluster manager to avoid allocating new ones. When computing the new target, - // take into account executors that are pending to be added or removed. - if (!replace) { - doRequestTotalExecutors( - numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size) - } else { - numPendingExecutors += knownExecutors.size + // If an executor is already pending to be removed, do not kill it again (SPARK-9795) + // If this executor is busy, do not kill it unless we are told to force kill it (SPARK-9552) + val executorsToKill = knownExecutors + .filter { id => !executorsPendingToRemove.contains(id) } + .filter { id => force || !scheduler.isExecutorBusy(id) } + executorsToKill.foreach { id => executorsPendingToRemove(id) = !replace } + + logInfo(s"Actual list of executor(s) to be killed is ${executorsToKill.mkString(", ")}") + + // If we do not wish to replace the executors we kill, sync the target number of executors + // with the cluster manager to avoid allocating new ones. When computing the new target, + // take into account executors that are pending to be added or removed. + val adjustTotalExecutors = + if (!replace) { + requestedTotalExecutors = math.max(requestedTotalExecutors - executorsToKill.size, 0) + if (requestedTotalExecutors != + (numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size)) { + logDebug( + s"""killExecutors($executorIds, $replace, $force): Executor counts do not match: + |requestedTotalExecutors = $requestedTotalExecutors + |numExistingExecutors = $numExistingExecutors + |numPendingExecutors = $numPendingExecutors + |executorsPendingToRemove = ${executorsPendingToRemove.size}""".stripMargin) + } + doRequestTotalExecutors(requestedTotalExecutors) + } else { + numPendingExecutors += knownExecutors.size + Future.successful(true) + } + + val killExecutors: Boolean => Future[Boolean] = + if (!executorsToKill.isEmpty) { + _ => doKillExecutors(executorsToKill) + } else { + _ => Future.successful(false) + } + + val killResponse = adjustTotalExecutors.flatMap(killExecutors)(ThreadUtils.sameThread) + + killResponse.flatMap(killSuccessful => + Future.successful (if (killSuccessful) executorsToKill else Seq.empty[String]) + )(ThreadUtils.sameThread) } - doKillExecutors(executorsToKill) + defaultAskTimeout.awaitResult(response) } /** * Kill the given list of executors through the cluster manager. * @return whether the kill request is acknowledged. */ - protected def doKillExecutors(executorIds: Seq[String]): Boolean = false + protected def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = + Future.successful(false) + /** + * Request that the cluster manager kill all executors on a given host. + * @return whether the kill request is acknowledged. + */ + final override def killExecutorsOnHost(host: String): Boolean = { + logInfo(s"Requesting to kill any and all executors on host ${host}") + // A potential race exists if a new executor attempts to register on a host + // that is on the blacklist and is no no longer valid. To avoid this race, + // all executor registration and killing happens in the event loop. This way, either + // an executor will fail to register, or will be killed when all executors on a host + // are killed. + // Kill all the executors on this host in an event loop to ensure serialization. + driverEndpoint.send(KillExecutorsOnHost(host)) + true + } + + protected def getHadoopDelegationCreds(): Option[Array[Byte]] = { + if (UserGroupInformation.isSecurityEnabled && hadoopDelegationTokenManager.isDefined) { + hadoopDelegationTokenManager.map { manager => + val creds = UserGroupInformation.getCurrentUser.getCredentials + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + manager.obtainDelegationTokens(hadoopConf, creds) + SparkHadoopUtil.get.serialize(creds) + } + } else { + None + } + } } private[spark] object CoarseGrainedSchedulerBackend { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala index 626a2b7d69ab..b25a4bfb501f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.rpc.{RpcEndpointRef, RpcAddress} +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} /** * Grouping of data for an executor used by CoarseGrainedSchedulerBackend. diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala deleted file mode 100644 index 641638a77d5f..000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ /dev/null @@ -1,74 +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.scheduler.cluster - -import org.apache.hadoop.fs.{Path, FileSystem} - -import org.apache.spark.rpc.RpcAddress -import org.apache.spark.{Logging, SparkContext, SparkEnv} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.scheduler.TaskSchedulerImpl - -private[spark] class SimrSchedulerBackend( - scheduler: TaskSchedulerImpl, - sc: SparkContext, - driverFilePath: String) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) - with Logging { - - val tmpPath = new Path(driverFilePath + "_tmp") - val filePath = new Path(driverFilePath) - - val maxCores = conf.getInt("spark.simr.executor.cores", 1) - - override def start() { - super.start() - - val driverUrl = rpcEnv.uriOf(SparkEnv.driverActorSystemName, - RpcAddress(sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port").toInt), - CoarseGrainedSchedulerBackend.ENDPOINT_NAME) - - val conf = SparkHadoopUtil.get.newConfiguration(sc.conf) - val fs = FileSystem.get(conf) - val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") - - logInfo("Writing to HDFS file: " + driverFilePath) - logInfo("Writing Akka address: " + driverUrl) - logInfo("Writing Spark UI Address: " + appUIAddress) - - // Create temporary file to prevent race condition where executors get empty driverUrl file - val temp = fs.create(tmpPath, true) - temp.writeUTF(driverUrl) - temp.writeInt(maxCores) - temp.writeUTF(appUIAddress) - temp.close() - - // "Atomic" rename - fs.rename(tmpPath, filePath) - } - - override def stop() { - val conf = SparkHadoopUtil.get.newConfiguration(sc.conf) - val fs = FileSystem.get(conf) - if (!fs.delete(new Path(driverFilePath), false)) { - logWarning(s"error deleting ${driverFilePath}") - } - super.stop() - } - -} 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 deleted file mode 100644 index 05d9bc92f228..000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ /dev/null @@ -1,208 +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.scheduler.cluster - -import java.util.concurrent.Semaphore - -import org.apache.spark.rpc.RpcAddress -import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv} -import org.apache.spark.deploy.{ApplicationDescription, Command} -import org.apache.spark.deploy.client.{AppClient, AppClientListener} -import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} -import org.apache.spark.scheduler._ -import org.apache.spark.util.Utils - -private[spark] class SparkDeploySchedulerBackend( - scheduler: TaskSchedulerImpl, - sc: SparkContext, - masters: Array[String]) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) - with AppClientListener - with Logging { - - private var client: AppClient = null - private var stopping = false - private val launcherBackend = new LauncherBackend() { - override protected def onStopRequest(): Unit = stop(SparkAppHandle.State.KILLED) - } - - @volatile var shutdownCallback: SparkDeploySchedulerBackend => Unit = _ - @volatile private var appId: String = _ - - private val registrationBarrier = new Semaphore(0) - - private val maxCores = conf.getOption("spark.cores.max").map(_.toInt) - private val totalExpectedCores = maxCores.getOrElse(0) - - override def start() { - super.start() - launcherBackend.connect() - - // The endpoint for executors to talk to us - val driverUrl = rpcEnv.uriOf(SparkEnv.driverActorSystemName, - RpcAddress(sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port").toInt), - CoarseGrainedSchedulerBackend.ENDPOINT_NAME) - val args = Seq( - "--driver-url", driverUrl, - "--executor-id", "{{EXECUTOR_ID}}", - "--hostname", "{{HOSTNAME}}", - "--cores", "{{CORES}}", - "--app-id", "{{APP_ID}}", - "--worker-url", "{{WORKER_URL}}") - val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") - .map(Utils.splitCommandString).getOrElse(Seq.empty) - val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath") - .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil) - val libraryPathEntries = sc.conf.getOption("spark.executor.extraLibraryPath") - .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil) - - // When testing, expose the parent class path to the child. This is processed by - // compute-classpath.{cmd,sh} and makes all needed jars available to child processes - // when the assembly is built with the "*-provided" profiles enabled. - val testingClassPath = - if (sys.props.contains("spark.testing")) { - sys.props("java.class.path").split(java.io.File.pathSeparator).toSeq - } else { - Nil - } - - // Start executors with a few necessary configs for registering with the scheduler - val sparkJavaOpts = Utils.sparkJavaOpts(conf, SparkConf.isExecutorStartupConf) - val javaOpts = sparkJavaOpts ++ extraJavaOpts - val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", - args, sc.executorEnvs, classPathEntries ++ testingClassPath, libraryPathEntries, javaOpts) - val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") - val coresPerExecutor = conf.getOption("spark.executor.cores").map(_.toInt) - val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, - command, appUIAddress, sc.eventLogDir, sc.eventLogCodec, coresPerExecutor) - client = new AppClient(sc.env.rpcEnv, masters, appDesc, this, conf) - client.start() - launcherBackend.setState(SparkAppHandle.State.SUBMITTED) - waitForRegistration() - launcherBackend.setState(SparkAppHandle.State.RUNNING) - } - - override def stop(): Unit = synchronized { - stop(SparkAppHandle.State.FINISHED) - } - - override def connected(appId: String) { - logInfo("Connected to Spark cluster with app ID " + appId) - this.appId = appId - notifyContext() - launcherBackend.setAppId(appId) - } - - override def disconnected() { - notifyContext() - if (!stopping) { - logWarning("Disconnected from Spark cluster! Waiting for reconnection...") - } - } - - override def dead(reason: String) { - notifyContext() - if (!stopping) { - launcherBackend.setState(SparkAppHandle.State.KILLED) - logError("Application has been killed. Reason: " + reason) - try { - scheduler.error(reason) - } finally { - // Ensure the application terminates, as we can no longer run jobs. - sc.stop() - } - } - } - - override def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, - memory: Int) { - logInfo("Granted executor ID %s on hostPort %s with %d cores, %s RAM".format( - fullId, hostPort, cores, Utils.megabytesToString(memory))) - } - - override def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]) { - val reason: ExecutorLossReason = exitStatus match { - case Some(code) => ExecutorExited(code, exitCausedByApp = true, message) - case None => SlaveLost(message) - } - logInfo("Executor %s removed: %s".format(fullId, message)) - removeExecutor(fullId.split("/")(1), reason) - } - - override def sufficientResourcesRegistered(): Boolean = { - totalCoreCount.get() >= totalExpectedCores * minRegisteredRatio - } - - override def applicationId(): String = - Option(appId).getOrElse { - logWarning("Application ID is not initialized yet.") - super.applicationId - } - - /** - * Request executors from the Master by specifying the total number desired, - * including existing pending and running executors. - * - * @return whether the request is acknowledged. - */ - protected override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { - Option(client) match { - case Some(c) => c.requestTotalExecutors(requestedTotal) - case None => - logWarning("Attempted to request executors before driver fully initialized.") - false - } - } - - /** - * Kill the given list of executors through the Master. - * @return whether the kill request is acknowledged. - */ - protected override def doKillExecutors(executorIds: Seq[String]): Boolean = { - Option(client) match { - case Some(c) => c.killExecutors(executorIds) - case None => - logWarning("Attempted to kill executors before driver fully initialized.") - false - } - } - - private def waitForRegistration() = { - registrationBarrier.acquire() - } - - private def notifyContext() = { - registrationBarrier.release() - } - - private def stop(finalState: SparkAppHandle.State): Unit = synchronized { - stopping = true - - launcherBackend.setState(finalState) - launcherBackend.close() - - super.stop() - client.stop() - - val callback = shutdownCallback - if (callback != null) { - callback(this) - } - } - -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala new file mode 100644 index 000000000000..a4e2a7434128 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.cluster + +import java.util.concurrent.Semaphore +import java.util.concurrent.atomic.AtomicBoolean + +import scala.concurrent.Future + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.deploy.{ApplicationDescription, Command} +import org.apache.spark.deploy.client.{StandaloneAppClient, StandaloneAppClientListener} +import org.apache.spark.internal.Logging +import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} +import org.apache.spark.rpc.RpcEndpointAddress +import org.apache.spark.scheduler._ +import org.apache.spark.util.Utils + +/** + * A [[SchedulerBackend]] implementation for Spark's standalone cluster manager. + */ +private[spark] class StandaloneSchedulerBackend( + scheduler: TaskSchedulerImpl, + sc: SparkContext, + masters: Array[String]) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) + with StandaloneAppClientListener + with Logging { + + private var client: StandaloneAppClient = null + private val stopping = new AtomicBoolean(false) + private val launcherBackend = new LauncherBackend() { + override protected def onStopRequest(): Unit = stop(SparkAppHandle.State.KILLED) + } + + @volatile var shutdownCallback: StandaloneSchedulerBackend => Unit = _ + @volatile private var appId: String = _ + + private val registrationBarrier = new Semaphore(0) + + private val maxCores = conf.getOption("spark.cores.max").map(_.toInt) + private val totalExpectedCores = maxCores.getOrElse(0) + + override def start() { + super.start() + + // SPARK-21159. The scheduler backend should only try to connect to the launcher when in client + // mode. In cluster mode, the code that submits the application to the Master needs to connect + // to the launcher instead. + if (sc.deployMode == "client") { + launcherBackend.connect() + } + + // The endpoint for executors to talk to us + val driverUrl = RpcEndpointAddress( + sc.conf.get("spark.driver.host"), + sc.conf.get("spark.driver.port").toInt, + CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + val args = Seq( + "--driver-url", driverUrl, + "--executor-id", "{{EXECUTOR_ID}}", + "--hostname", "{{HOSTNAME}}", + "--cores", "{{CORES}}", + "--app-id", "{{APP_ID}}", + "--worker-url", "{{WORKER_URL}}") + val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") + .map(Utils.splitCommandString).getOrElse(Seq.empty) + val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath") + .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil) + val libraryPathEntries = sc.conf.getOption("spark.executor.extraLibraryPath") + .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil) + + // When testing, expose the parent class path to the child. This is processed by + // compute-classpath.{cmd,sh} and makes all needed jars available to child processes + // when the assembly is built with the "*-provided" profiles enabled. + val testingClassPath = + if (sys.props.contains("spark.testing")) { + sys.props("java.class.path").split(java.io.File.pathSeparator).toSeq + } else { + Nil + } + + // Start executors with a few necessary configs for registering with the scheduler + val sparkJavaOpts = Utils.sparkJavaOpts(conf, SparkConf.isExecutorStartupConf) + val javaOpts = sparkJavaOpts ++ extraJavaOpts + val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", + args, sc.executorEnvs, classPathEntries ++ testingClassPath, libraryPathEntries, javaOpts) + val webUrl = sc.ui.map(_.webUrl).getOrElse("") + val coresPerExecutor = conf.getOption("spark.executor.cores").map(_.toInt) + // If we're using dynamic allocation, set our initial executor limit to 0 for now. + // ExecutorAllocationManager will send the real initial limit to the Master later. + val initialExecutorLimit = + if (Utils.isDynamicAllocationEnabled(conf)) { + Some(0) + } else { + None + } + val appDesc = ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, + webUrl, sc.eventLogDir, sc.eventLogCodec, coresPerExecutor, initialExecutorLimit) + client = new StandaloneAppClient(sc.env.rpcEnv, masters, appDesc, this, conf) + client.start() + launcherBackend.setState(SparkAppHandle.State.SUBMITTED) + waitForRegistration() + launcherBackend.setState(SparkAppHandle.State.RUNNING) + } + + override def stop(): Unit = { + stop(SparkAppHandle.State.FINISHED) + } + + override def connected(appId: String) { + logInfo("Connected to Spark cluster with app ID " + appId) + this.appId = appId + notifyContext() + launcherBackend.setAppId(appId) + } + + override def disconnected() { + notifyContext() + if (!stopping.get) { + logWarning("Disconnected from Spark cluster! Waiting for reconnection...") + } + } + + override def dead(reason: String) { + notifyContext() + if (!stopping.get) { + launcherBackend.setState(SparkAppHandle.State.KILLED) + logError("Application has been killed. Reason: " + reason) + try { + scheduler.error(reason) + } finally { + // Ensure the application terminates, as we can no longer run jobs. + sc.stopInNewThread() + } + } + } + + override def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, + memory: Int) { + logInfo("Granted executor ID %s on hostPort %s with %d cores, %s RAM".format( + fullId, hostPort, cores, Utils.megabytesToString(memory))) + } + + override def executorRemoved( + fullId: String, message: String, exitStatus: Option[Int], workerLost: Boolean) { + val reason: ExecutorLossReason = exitStatus match { + case Some(code) => ExecutorExited(code, exitCausedByApp = true, message) + case None => SlaveLost(message, workerLost = workerLost) + } + logInfo("Executor %s removed: %s".format(fullId, message)) + removeExecutor(fullId.split("/")(1), reason) + } + + override def workerRemoved(workerId: String, host: String, message: String): Unit = { + logInfo("Worker %s removed: %s".format(workerId, message)) + removeWorker(workerId, host, message) + } + + override def sufficientResourcesRegistered(): Boolean = { + totalCoreCount.get() >= totalExpectedCores * minRegisteredRatio + } + + override def applicationId(): String = + Option(appId).getOrElse { + logWarning("Application ID is not initialized yet.") + super.applicationId + } + + /** + * Request executors from the Master by specifying the total number desired, + * including existing pending and running executors. + * + * @return whether the request is acknowledged. + */ + protected override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { + Option(client) match { + case Some(c) => c.requestTotalExecutors(requestedTotal) + case None => + logWarning("Attempted to request executors before driver fully initialized.") + Future.successful(false) + } + } + + /** + * Kill the given list of executors through the Master. + * @return whether the kill request is acknowledged. + */ + protected override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = { + Option(client) match { + case Some(c) => c.killExecutors(executorIds) + case None => + logWarning("Attempted to kill executors before driver fully initialized.") + Future.successful(false) + } + } + + private def waitForRegistration() = { + registrationBarrier.acquire() + } + + private def notifyContext() = { + registrationBarrier.release() + } + + private def stop(finalState: SparkAppHandle.State): Unit = { + if (stopping.compareAndSet(false, true)) { + try { + super.stop() + client.stop() + + val callback = shutdownCallback + if (callback != null) { + callback(this) + } + } finally { + launcherBackend.setState(finalState) + launcherBackend.close() + } + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala deleted file mode 100644 index 80da37b09b59..000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ /dev/null @@ -1,226 +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.scheduler.cluster - -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.{Future, ExecutionContext} - -import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.rpc._ -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.scheduler._ -import org.apache.spark.ui.JettyUtils -import org.apache.spark.util.{ThreadUtils, RpcUtils} - -import scala.util.control.NonFatal - -/** - * Abstract Yarn scheduler backend that contains common logic - * between the client and cluster Yarn scheduler backends. - */ -private[spark] abstract class YarnSchedulerBackend( - scheduler: TaskSchedulerImpl, - sc: SparkContext) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { - - if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { - minRegisteredRatio = 0.8 - } - - protected var totalExpectedExecutors = 0 - - private val yarnSchedulerEndpoint = new YarnSchedulerEndpoint(rpcEnv) - - private val yarnSchedulerEndpointRef = rpcEnv.setupEndpoint( - YarnSchedulerBackend.ENDPOINT_NAME, yarnSchedulerEndpoint) - - private implicit val askTimeout = RpcUtils.askRpcTimeout(sc.conf) - - /** - * Request executors from the ApplicationMaster by specifying the total number desired. - * This includes executors already pending or running. - */ - override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { - yarnSchedulerEndpointRef.askWithRetry[Boolean]( - RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount)) - } - - /** - * Request that the ApplicationMaster kill the specified executors. - */ - override def doKillExecutors(executorIds: Seq[String]): Boolean = { - yarnSchedulerEndpointRef.askWithRetry[Boolean](KillExecutors(executorIds)) - } - - override def sufficientResourcesRegistered(): Boolean = { - totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio - } - - /** - * Add filters to the SparkUI. - */ - private def addWebUIFilter( - filterName: String, - filterParams: Map[String, String], - proxyBase: String): Unit = { - if (proxyBase != null && proxyBase.nonEmpty) { - System.setProperty("spark.ui.proxyBase", proxyBase) - } - - val hasFilter = - filterName != null && filterName.nonEmpty && - filterParams != null && filterParams.nonEmpty - if (hasFilter) { - logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase") - conf.set("spark.ui.filters", filterName) - filterParams.foreach { case (k, v) => conf.set(s"spark.$filterName.param.$k", v) } - scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) } - } - } - - override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = { - new YarnDriverEndpoint(rpcEnv, properties) - } - - /** - * Override the DriverEndpoint to add extra logic for the case when an executor is disconnected. - * This endpoint communicates with the executors and queries the AM for an executor's exit - * status when the executor is disconnected. - */ - private class YarnDriverEndpoint(rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) - extends DriverEndpoint(rpcEnv, sparkProperties) { - - /** - * When onDisconnected is received at the driver endpoint, the superclass DriverEndpoint - * handles it by assuming the Executor was lost for a bad reason and removes the executor - * immediately. - * - * In YARN's case however it is crucial to talk to the application master and ask why the - * executor had exited. If the executor exited for some reason unrelated to the running tasks - * (e.g., preemption), according to the application master, then we pass that information down - * to the TaskSetManager to inform the TaskSetManager that tasks on that lost executor should - * not count towards a job failure. - */ - override def onDisconnected(rpcAddress: RpcAddress): Unit = { - addressToExecutorId.get(rpcAddress).foreach { executorId => - if (disableExecutor(executorId)) { - yarnSchedulerEndpoint.handleExecutorDisconnectedFromDriver(executorId, rpcAddress) - } - } - } - } - - /** - * An [[RpcEndpoint]] that communicates with the ApplicationMaster. - */ - private class YarnSchedulerEndpoint(override val rpcEnv: RpcEnv) - extends ThreadSafeRpcEndpoint with Logging { - private var amEndpoint: Option[RpcEndpointRef] = None - - private val askAmThreadPool = - ThreadUtils.newDaemonCachedThreadPool("yarn-scheduler-ask-am-thread-pool") - implicit val askAmExecutor = ExecutionContext.fromExecutor(askAmThreadPool) - - private[YarnSchedulerBackend] def handleExecutorDisconnectedFromDriver( - executorId: String, - executorRpcAddress: RpcAddress): Unit = { - amEndpoint match { - case Some(am) => - val lossReasonRequest = GetExecutorLossReason(executorId) - val future = am.ask[ExecutorLossReason](lossReasonRequest, askTimeout) - future onSuccess { - case reason: ExecutorLossReason => { - driverEndpoint.askWithRetry[Boolean](RemoveExecutor(executorId, reason)) - } - } - future onFailure { - case NonFatal(e) => { - logWarning(s"Attempted to get executor loss reason" + - s" for executor id ${executorId} at RPC address ${executorRpcAddress}," + - s" but got no response. Marking as slave lost.", e) - driverEndpoint.askWithRetry[Boolean](RemoveExecutor(executorId, SlaveLost())) - } - case t => throw t - } - case None => - logWarning("Attempted to check for an executor loss reason" + - " before the AM has registered!") - } - } - - override def receive: PartialFunction[Any, Unit] = { - case RegisterClusterManager(am) => - logInfo(s"ApplicationMaster registered as $am") - amEndpoint = Option(am) - - case AddWebUIFilter(filterName, filterParams, proxyBase) => - addWebUIFilter(filterName, filterParams, proxyBase) - - case RemoveExecutor(executorId, reason) => - logWarning(reason.toString) - removeExecutor(executorId, reason) - } - - - override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case r: RequestExecutors => - amEndpoint match { - case Some(am) => - Future { - context.reply(am.askWithRetry[Boolean](r)) - } onFailure { - case NonFatal(e) => - logError(s"Sending $r to AM was unsuccessful", e) - context.sendFailure(e) - } - case None => - logWarning("Attempted to request executors before the AM has registered!") - context.reply(false) - } - - case k: KillExecutors => - amEndpoint match { - case Some(am) => - Future { - context.reply(am.askWithRetry[Boolean](k)) - } onFailure { - case NonFatal(e) => - logError(s"Sending $k to AM was unsuccessful", e) - context.sendFailure(e) - } - case None => - logWarning("Attempted to kill executors before the AM has registered!") - context.reply(false) - } - } - - override def onDisconnected(remoteAddress: RpcAddress): Unit = { - if (amEndpoint.exists(_.address == remoteAddress)) { - logWarning(s"ApplicationMaster has disassociated: $remoteAddress") - } - } - - override def onStop(): Unit = { - askAmThreadPool.shutdownNow() - } - } -} - -private[spark] object YarnSchedulerBackend { - val ENDPOINT_NAME = "YarnScheduler" -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala deleted file mode 100644 index d10a77f8e5c7..000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ /dev/null @@ -1,436 +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.scheduler.cluster.mesos - -import java.io.File -import java.util.concurrent.locks.ReentrantLock -import java.util.{Collections, List => JList} - -import scala.collection.JavaConverters._ -import scala.collection.mutable.{HashMap, HashSet} - -import com.google.common.collect.HashBiMap -import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} -import org.apache.mesos.{Scheduler => MScheduler, SchedulerDriver} - -import org.apache.spark.{SecurityManager, SparkContext, SparkEnv, SparkException, TaskState} -import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient -import org.apache.spark.rpc.RpcAddress -import org.apache.spark.scheduler.{SlaveLost, TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.util.Utils - -/** - * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds - * onto each Mesos node for the duration of the Spark job instead of relinquishing cores whenever - * a task is done. It launches Spark tasks within the coarse-grained Mesos tasks using the - * CoarseGrainedSchedulerBackend mechanism. This class is useful for lower and more predictable - * latency. - * - * Unfortunately this has a bit of duplication from MesosSchedulerBackend, but it seems hard to - * remove this. - */ -private[spark] class CoarseMesosSchedulerBackend( - scheduler: TaskSchedulerImpl, - sc: SparkContext, - master: String, - securityManager: SecurityManager) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) - with MScheduler - with MesosSchedulerUtils { - - val MAX_SLAVE_FAILURES = 2 // Blacklist a slave after this many failures - - // Maximum number of cores to acquire (TODO: we'll need more flexible controls here) - val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt - - // If shuffle service is enabled, the Spark driver will register with the shuffle service. - // This is for cleaning up shuffle files reliably. - private val shuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false) - - // Cores we have acquired with each Mesos task ID - val coresByTaskId = new HashMap[Int, Int] - var totalCoresAcquired = 0 - - val slaveIdsWithExecutors = new HashSet[String] - - // Maping from slave Id to hostname - private val slaveIdToHost = new HashMap[String, String] - - val taskIdToSlaveId: HashBiMap[Int, String] = HashBiMap.create[Int, String] - // How many times tasks on each slave failed - val failuresBySlaveId: HashMap[String, Int] = new HashMap[String, Int] - - /** - * The total number of executors we aim to have. Undefined when not using dynamic allocation - * and before the ExecutorAllocatorManager calls [[doRequestTotalExecutors]]. - */ - private var executorLimitOption: Option[Int] = None - - /** - * Return the current executor limit, which may be [[Int.MaxValue]] - * before properly initialized. - */ - private[mesos] def executorLimit: Int = executorLimitOption.getOrElse(Int.MaxValue) - - private val pendingRemovedSlaveIds = new HashSet[String] - - // private lock object protecting mutable state above. Using the intrinsic lock - // may lead to deadlocks since the superclass might also try to lock - private val stateLock = new ReentrantLock - - val extraCoresPerSlave = conf.getInt("spark.mesos.extra.cores", 0) - - // Offer constraints - private val slaveOfferConstraints = - parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) - - // A client for talking to the external shuffle service, if it is a - private val mesosExternalShuffleClient: Option[MesosExternalShuffleClient] = { - if (shuffleServiceEnabled) { - Some(new MesosExternalShuffleClient( - SparkTransportConf.fromSparkConf(conf), - securityManager, - securityManager.isAuthenticationEnabled(), - securityManager.isSaslEncryptionEnabled())) - } else { - None - } - } - - var nextMesosTaskId = 0 - - @volatile var appId: String = _ - - def newMesosTaskId(): Int = { - val id = nextMesosTaskId - nextMesosTaskId += 1 - id - } - - override def start() { - super.start() - val driver = createSchedulerDriver( - master, - CoarseMesosSchedulerBackend.this, - sc.sparkUser, - sc.appName, - sc.conf, - sc.ui.map(_.appUIAddress)) - startScheduler(driver) - } - - def createCommand(offer: Offer, numCores: Int, taskId: Int): CommandInfo = { - val executorSparkHome = conf.getOption("spark.mesos.executor.home") - .orElse(sc.getSparkHome()) - .getOrElse { - throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") - } - val environment = Environment.newBuilder() - val extraClassPath = conf.getOption("spark.executor.extraClassPath") - extraClassPath.foreach { cp => - environment.addVariables( - Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) - } - val extraJavaOpts = conf.get("spark.executor.extraJavaOptions", "") - - // Set the environment variable through a command prefix - // to append to the existing value of the variable - val prefixEnv = conf.getOption("spark.executor.extraLibraryPath").map { p => - Utils.libraryPathEnvPrefix(Seq(p)) - }.getOrElse("") - - environment.addVariables( - Environment.Variable.newBuilder() - .setName("SPARK_EXECUTOR_OPTS") - .setValue(extraJavaOpts) - .build()) - - sc.executorEnvs.foreach { case (key, value) => - environment.addVariables(Environment.Variable.newBuilder() - .setName(key) - .setValue(value) - .build()) - } - val command = CommandInfo.newBuilder() - .setEnvironment(environment) - - val uri = conf.getOption("spark.executor.uri") - .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) - - if (uri.isEmpty) { - val runScript = new File(executorSparkHome, "./bin/spark-class").getCanonicalPath - command.setValue( - "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend" - .format(prefixEnv, runScript) + - s" --driver-url $driverURL" + - s" --executor-id ${offer.getSlaveId.getValue}" + - s" --hostname ${offer.getHostname}" + - s" --cores $numCores" + - s" --app-id $appId") - } else { - // Grab everything to the first '.'. We'll use that and '*' to - // glob the directory "correctly". - val basename = uri.get.split('/').last.split('.').head - val executorId = sparkExecutorId(offer.getSlaveId.getValue, taskId.toString) - command.setValue( - s"cd $basename*; $prefixEnv " + - "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend" + - s" --driver-url $driverURL" + - s" --executor-id $executorId" + - s" --hostname ${offer.getHostname}" + - s" --cores $numCores" + - s" --app-id $appId") - command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get)) - } - - conf.getOption("spark.mesos.uris").map { uris => - setupUris(uris, command) - } - - command.build() - } - - protected def driverURL: String = { - if (conf.contains("spark.testing")) { - "driverURL" - } else { - sc.env.rpcEnv.uriOf( - SparkEnv.driverActorSystemName, - RpcAddress(conf.get("spark.driver.host"), conf.get("spark.driver.port").toInt), - CoarseGrainedSchedulerBackend.ENDPOINT_NAME) - } - } - - override def offerRescinded(d: SchedulerDriver, o: OfferID) {} - - override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { - appId = frameworkId.getValue - mesosExternalShuffleClient.foreach(_.init(appId)) - logInfo("Registered as framework ID " + appId) - markRegistered() - } - - override def sufficientResourcesRegistered(): Boolean = { - totalCoresAcquired >= maxCores * minRegisteredRatio - } - - override def disconnected(d: SchedulerDriver) {} - - override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {} - - /** - * Method called by Mesos to offer resources on slaves. We respond by launching an executor, - * unless we've already launched more than we wanted to. - */ - override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { - stateLock.synchronized { - val filters = Filters.newBuilder().setRefuseSeconds(5).build() - for (offer <- offers.asScala) { - val offerAttributes = toAttributeMap(offer.getAttributesList) - val meetsConstraints = matchesAttributeRequirements(slaveOfferConstraints, offerAttributes) - val slaveId = offer.getSlaveId.getValue - val mem = getResource(offer.getResourcesList, "mem") - val cpus = getResource(offer.getResourcesList, "cpus").toInt - val id = offer.getId.getValue - if (taskIdToSlaveId.size < executorLimit && - totalCoresAcquired < maxCores && - meetsConstraints && - mem >= calculateTotalMemory(sc) && - cpus >= 1 && - failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES && - !slaveIdsWithExecutors.contains(slaveId)) { - // Launch an executor on the slave - val cpusToUse = math.min(cpus, maxCores - totalCoresAcquired) - totalCoresAcquired += cpusToUse - val taskId = newMesosTaskId() - taskIdToSlaveId.put(taskId, slaveId) - slaveIdsWithExecutors += slaveId - coresByTaskId(taskId) = cpusToUse - // Gather cpu resources from the available resources and use them in the task. - val (remainingResources, cpuResourcesToUse) = - partitionResources(offer.getResourcesList, "cpus", cpusToUse) - val (_, memResourcesToUse) = - partitionResources(remainingResources.asJava, "mem", calculateTotalMemory(sc)) - val taskBuilder = MesosTaskInfo.newBuilder() - .setTaskId(TaskID.newBuilder().setValue(taskId.toString).build()) - .setSlaveId(offer.getSlaveId) - .setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave, taskId)) - .setName("Task " + taskId) - .addAllResources(cpuResourcesToUse.asJava) - .addAllResources(memResourcesToUse.asJava) - - sc.conf.getOption("spark.mesos.executor.docker.image").foreach { image => - MesosSchedulerBackendUtil - .setupContainerBuilderDockerInfo(image, sc.conf, taskBuilder.getContainerBuilder()) - } - - // accept the offer and launch the task - logDebug(s"Accepting offer: $id with attributes: $offerAttributes mem: $mem cpu: $cpus") - slaveIdToHost(offer.getSlaveId.getValue) = offer.getHostname - d.launchTasks( - Collections.singleton(offer.getId), - Collections.singleton(taskBuilder.build()), filters) - } else { - // Decline the offer - logDebug(s"Declining offer: $id with attributes: $offerAttributes mem: $mem cpu: $cpus") - d.declineOffer(offer.getId) - } - } - } - } - - - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { - val taskId = status.getTaskId.getValue.toInt - val state = status.getState - logInfo(s"Mesos task $taskId is now $state") - val slaveId: String = status.getSlaveId.getValue - stateLock.synchronized { - // If the shuffle service is enabled, have the driver register with each one of the - // shuffle services. This allows the shuffle services to clean up state associated with - // this application when the driver exits. There is currently not a great way to detect - // this through Mesos, since the shuffle services are set up independently. - if (TaskState.fromMesos(state).equals(TaskState.RUNNING) && - slaveIdToHost.contains(slaveId) && - shuffleServiceEnabled) { - assume(mesosExternalShuffleClient.isDefined, - "External shuffle client was not instantiated even though shuffle service is enabled.") - // TODO: Remove this and allow the MesosExternalShuffleService to detect - // framework termination when new Mesos Framework HTTP API is available. - val externalShufflePort = conf.getInt("spark.shuffle.service.port", 7337) - val hostname = slaveIdToHost.remove(slaveId).get - logDebug(s"Connecting to shuffle service on slave $slaveId, " + - s"host $hostname, port $externalShufflePort for app ${conf.getAppId}") - mesosExternalShuffleClient.get - .registerDriverWithShuffleService(hostname, externalShufflePort) - } - - if (TaskState.isFinished(TaskState.fromMesos(state))) { - val slaveId = taskIdToSlaveId.get(taskId) - slaveIdsWithExecutors -= slaveId - taskIdToSlaveId.remove(taskId) - // Remove the cores we have remembered for this task, if it's in the hashmap - for (cores <- coresByTaskId.get(taskId)) { - totalCoresAcquired -= cores - coresByTaskId -= taskId - } - // If it was a failure, mark the slave as failed for blacklisting purposes - if (TaskState.isFailed(TaskState.fromMesos(state))) { - failuresBySlaveId(slaveId) = failuresBySlaveId.getOrElse(slaveId, 0) + 1 - if (failuresBySlaveId(slaveId) >= MAX_SLAVE_FAILURES) { - logInfo(s"Blacklisting Mesos slave $slaveId due to too many failures; " + - "is Spark installed on it?") - } - } - executorTerminated(d, slaveId, s"Executor finished with state $state") - // In case we'd rejected everything before but have now lost a node - d.reviveOffers() - } - } - } - - override def error(d: SchedulerDriver, message: String) { - logError(s"Mesos error: $message") - scheduler.error(message) - } - - override def stop() { - super.stop() - if (mesosDriver != null) { - mesosDriver.stop() - } - } - - override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} - - /** - * Called when a slave is lost or a Mesos task finished. Update local view on - * what tasks are running and remove the terminated slave from the list of pending - * slave IDs that we might have asked to be killed. It also notifies the driver - * that an executor was removed. - */ - private def executorTerminated(d: SchedulerDriver, slaveId: String, reason: String): Unit = { - stateLock.synchronized { - if (slaveIdsWithExecutors.contains(slaveId)) { - val slaveIdToTaskId = taskIdToSlaveId.inverse() - if (slaveIdToTaskId.containsKey(slaveId)) { - val taskId: Int = slaveIdToTaskId.get(slaveId) - taskIdToSlaveId.remove(taskId) - removeExecutor(sparkExecutorId(slaveId, taskId.toString), SlaveLost(reason)) - } - // TODO: This assumes one Spark executor per Mesos slave, - // which may no longer be true after SPARK-5095 - pendingRemovedSlaveIds -= slaveId - slaveIdsWithExecutors -= slaveId - } - } - } - - private def sparkExecutorId(slaveId: String, taskId: String): String = { - s"$slaveId/$taskId" - } - - override def slaveLost(d: SchedulerDriver, slaveId: SlaveID): Unit = { - logInfo(s"Mesos slave lost: ${slaveId.getValue}") - executorTerminated(d, slaveId.getValue, "Mesos slave lost: " + slaveId.getValue) - } - - override def executorLost(d: SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int): Unit = { - logInfo("Executor lost: %s, marking slave %s as lost".format(e.getValue, s.getValue)) - slaveLost(d, s) - } - - override def applicationId(): String = - Option(appId).getOrElse { - logWarning("Application ID is not initialized yet.") - super.applicationId - } - - override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { - // We don't truly know if we can fulfill the full amount of executors - // since at coarse grain it depends on the amount of slaves available. - logInfo("Capping the total amount of executors to " + requestedTotal) - executorLimitOption = Some(requestedTotal) - true - } - - override def doKillExecutors(executorIds: Seq[String]): Boolean = { - if (mesosDriver == null) { - logWarning("Asked to kill executors before the Mesos driver was started.") - return false - } - - val slaveIdToTaskId = taskIdToSlaveId.inverse() - for (executorId <- executorIds) { - val slaveId = executorId.split("/")(0) - if (slaveIdToTaskId.containsKey(slaveId)) { - mesosDriver.killTask( - TaskID.newBuilder().setValue(slaveIdToTaskId.get(slaveId).toString).build()) - pendingRemovedSlaveIds += slaveId - } else { - logWarning("Unable to find executor Id '" + executorId + "' in Mesos scheduler") - } - } - // no need to adjust `executorLimitOption` since the AllocationManager already communicated - // the desired limit through a call to `doRequestTotalExecutors`. - // See [[o.a.s.scheduler.cluster.CoarseGrainedSchedulerBackend.killExecutors]] - true - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala deleted file mode 100644 index a6d9374eb9e8..000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ /dev/null @@ -1,676 +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.scheduler.cluster.mesos - -import java.io.File -import java.util.concurrent.locks.ReentrantLock -import java.util.{Collections, Date, List => JList} - -import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer - -import org.apache.mesos.Protos.Environment.Variable -import org.apache.mesos.Protos.TaskStatus.Reason -import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} -import org.apache.mesos.{Scheduler, SchedulerDriver} -import org.apache.spark.deploy.mesos.MesosDriverDescription -import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KillSubmissionResponse, SubmissionStatusResponse} -import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.util.Utils -import org.apache.spark.{SecurityManager, SparkConf, SparkException, TaskState} - - -/** - * Tracks the current state of a Mesos Task that runs a Spark driver. - * @param driverDescription Submitted driver description from - * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]] - * @param taskId Mesos TaskID generated for the task - * @param slaveId Slave ID that the task is assigned to - * @param mesosTaskStatus The last known task status update. - * @param startDate The date the task was launched - */ -private[spark] class MesosClusterSubmissionState( - val driverDescription: MesosDriverDescription, - val taskId: TaskID, - val slaveId: SlaveID, - var mesosTaskStatus: Option[TaskStatus], - var startDate: Date, - var finishDate: Option[Date]) - extends Serializable { - - def copy(): MesosClusterSubmissionState = { - new MesosClusterSubmissionState( - driverDescription, taskId, slaveId, mesosTaskStatus, startDate, finishDate) - } -} - -/** - * Tracks the retry state of a driver, which includes the next time it should be scheduled - * and necessary information to do exponential backoff. - * This class is not thread-safe, and we expect the caller to handle synchronizing state. - * @param lastFailureStatus Last Task status when it failed. - * @param retries Number of times it has been retried. - * @param nextRetry Time at which it should be retried next - * @param waitTime The amount of time driver is scheduled to wait until next retry. - */ -private[spark] class MesosClusterRetryState( - val lastFailureStatus: TaskStatus, - val retries: Int, - val nextRetry: Date, - val waitTime: Int) extends Serializable { - def copy(): MesosClusterRetryState = - new MesosClusterRetryState(lastFailureStatus, retries, nextRetry, waitTime) -} - -/** - * The full state of the cluster scheduler, currently being used for displaying - * information on the UI. - * @param frameworkId Mesos Framework id for the cluster scheduler. - * @param masterUrl The Mesos master url - * @param queuedDrivers All drivers queued to be launched - * @param launchedDrivers All launched or running drivers - * @param finishedDrivers All terminated drivers - * @param pendingRetryDrivers All drivers pending to be retried - */ -private[spark] class MesosClusterSchedulerState( - val frameworkId: String, - val masterUrl: Option[String], - val queuedDrivers: Iterable[MesosDriverDescription], - val launchedDrivers: Iterable[MesosClusterSubmissionState], - val finishedDrivers: Iterable[MesosClusterSubmissionState], - val pendingRetryDrivers: Iterable[MesosDriverDescription]) - -/** - * The full state of a Mesos driver, that is being used to display driver information on the UI. - */ -private[spark] class MesosDriverState( - val state: String, - val description: MesosDriverDescription, - val submissionState: Option[MesosClusterSubmissionState] = None) - -/** - * A Mesos scheduler that is responsible for launching submitted Spark drivers in cluster mode - * as Mesos tasks in a Mesos cluster. - * All drivers are launched asynchronously by the framework, which will eventually be launched - * by one of the slaves in the cluster. The results of the driver will be stored in slave's task - * sandbox which is accessible by visiting the Mesos UI. - * This scheduler supports recovery by persisting all its state and performs task reconciliation - * on recover, which gets all the latest state for all the drivers from Mesos master. - */ -private[spark] class MesosClusterScheduler( - engineFactory: MesosClusterPersistenceEngineFactory, - conf: SparkConf) - extends Scheduler with MesosSchedulerUtils { - var frameworkUrl: String = _ - private val metricsSystem = - MetricsSystem.createMetricsSystem("mesos_cluster", conf, new SecurityManager(conf)) - private val master = conf.get("spark.master") - private val appName = conf.get("spark.app.name") - private val queuedCapacity = conf.getInt("spark.mesos.maxDrivers", 200) - private val retainedDrivers = conf.getInt("spark.mesos.retainedDrivers", 200) - private val maxRetryWaitTime = conf.getInt("spark.mesos.cluster.retry.wait.max", 60) // 1 minute - private val schedulerState = engineFactory.createEngine("scheduler") - private val stateLock = new ReentrantLock() - private val finishedDrivers = - new mutable.ArrayBuffer[MesosClusterSubmissionState](retainedDrivers) - private var frameworkId: String = null - // Holds all the launched drivers and current launch state, keyed by driver id. - private val launchedDrivers = new mutable.HashMap[String, MesosClusterSubmissionState]() - // Holds a map of driver id to expected slave id that is passed to Mesos for reconciliation. - // All drivers that are loaded after failover are added here, as we need get the latest - // state of the tasks from Mesos. - private val pendingRecover = new mutable.HashMap[String, SlaveID]() - // Stores all the submitted drivers that hasn't been launched. - private val queuedDrivers = new ArrayBuffer[MesosDriverDescription]() - // All supervised drivers that are waiting to retry after termination. - private val pendingRetryDrivers = new ArrayBuffer[MesosDriverDescription]() - private val queuedDriversState = engineFactory.createEngine("driverQueue") - private val launchedDriversState = engineFactory.createEngine("launchedDrivers") - private val pendingRetryDriversState = engineFactory.createEngine("retryList") - // Flag to mark if the scheduler is ready to be called, which is until the scheduler - // is registered with Mesos master. - @volatile protected var ready = false - private var masterInfo: Option[MasterInfo] = None - - def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse = { - val c = new CreateSubmissionResponse - if (!ready) { - c.success = false - c.message = "Scheduler is not ready to take requests" - return c - } - - stateLock.synchronized { - if (isQueueFull()) { - c.success = false - c.message = "Already reached maximum submission size" - return c - } - c.submissionId = desc.submissionId - queuedDriversState.persist(desc.submissionId, desc) - queuedDrivers += desc - c.success = true - } - c - } - - def killDriver(submissionId: String): KillSubmissionResponse = { - val k = new KillSubmissionResponse - if (!ready) { - k.success = false - k.message = "Scheduler is not ready to take requests" - return k - } - k.submissionId = submissionId - stateLock.synchronized { - // We look for the requested driver in the following places: - // 1. Check if submission is running or launched. - // 2. Check if it's still queued. - // 3. Check if it's in the retry list. - // 4. Check if it has already completed. - if (launchedDrivers.contains(submissionId)) { - val task = launchedDrivers(submissionId) - mesosDriver.killTask(task.taskId) - k.success = true - k.message = "Killing running driver" - } else if (removeFromQueuedDrivers(submissionId)) { - k.success = true - k.message = "Removed driver while it's still pending" - } else if (removeFromPendingRetryDrivers(submissionId)) { - k.success = true - k.message = "Removed driver while it's being retried" - } else if (finishedDrivers.exists(_.driverDescription.submissionId.equals(submissionId))) { - k.success = false - k.message = "Driver already terminated" - } else { - k.success = false - k.message = "Cannot find driver" - } - } - k - } - - def getDriverStatus(submissionId: String): SubmissionStatusResponse = { - val s = new SubmissionStatusResponse - if (!ready) { - s.success = false - s.message = "Scheduler is not ready to take requests" - return s - } - s.submissionId = submissionId - stateLock.synchronized { - if (queuedDrivers.exists(_.submissionId.equals(submissionId))) { - s.success = true - s.driverState = "QUEUED" - } else if (launchedDrivers.contains(submissionId)) { - s.success = true - s.driverState = "RUNNING" - launchedDrivers(submissionId).mesosTaskStatus.foreach(state => s.message = state.toString) - } else if (finishedDrivers.exists(_.driverDescription.submissionId.equals(submissionId))) { - s.success = true - s.driverState = "FINISHED" - finishedDrivers - .find(d => d.driverDescription.submissionId.equals(submissionId)).get.mesosTaskStatus - .foreach(state => s.message = state.toString) - } else if (pendingRetryDrivers.exists(_.submissionId.equals(submissionId))) { - val status = pendingRetryDrivers.find(_.submissionId.equals(submissionId)) - .get.retryState.get.lastFailureStatus - s.success = true - s.driverState = "RETRYING" - s.message = status.toString - } else { - s.success = false - s.driverState = "NOT_FOUND" - } - } - s - } - - /** - * Gets the driver state to be displayed on the Web UI. - */ - def getDriverState(submissionId: String): Option[MesosDriverState] = { - stateLock.synchronized { - queuedDrivers.find(_.submissionId.equals(submissionId)) - .map(d => new MesosDriverState("QUEUED", d)) - .orElse(launchedDrivers.get(submissionId) - .map(d => new MesosDriverState("RUNNING", d.driverDescription, Some(d)))) - .orElse(finishedDrivers.find(_.driverDescription.submissionId.equals(submissionId)) - .map(d => new MesosDriverState("FINISHED", d.driverDescription, Some(d)))) - .orElse(pendingRetryDrivers.find(_.submissionId.equals(submissionId)) - .map(d => new MesosDriverState("RETRYING", d))) - } - } - - private def isQueueFull(): Boolean = launchedDrivers.size >= queuedCapacity - - /** - * Recover scheduler state that is persisted. - * We still need to do task reconciliation to be up to date of the latest task states - * as it might have changed while the scheduler is failing over. - */ - private def recoverState(): Unit = { - stateLock.synchronized { - launchedDriversState.fetchAll[MesosClusterSubmissionState]().foreach { state => - launchedDrivers(state.taskId.getValue) = state - pendingRecover(state.taskId.getValue) = state.slaveId - } - queuedDriversState.fetchAll[MesosDriverDescription]().foreach(d => queuedDrivers += d) - // There is potential timing issue where a queued driver might have been launched - // but the scheduler shuts down before the queued driver was able to be removed - // from the queue. We try to mitigate this issue by walking through all queued drivers - // and remove if they're already launched. - queuedDrivers - .filter(d => launchedDrivers.contains(d.submissionId)) - .foreach(d => removeFromQueuedDrivers(d.submissionId)) - pendingRetryDriversState.fetchAll[MesosDriverDescription]() - .foreach(s => pendingRetryDrivers += s) - // TODO: Consider storing finished drivers so we can show them on the UI after - // failover. For now we clear the history on each recovery. - finishedDrivers.clear() - } - } - - /** - * Starts the cluster scheduler and wait until the scheduler is registered. - * This also marks the scheduler to be ready for requests. - */ - def start(): Unit = { - // TODO: Implement leader election to make sure only one framework running in the cluster. - val fwId = schedulerState.fetch[String]("frameworkId") - fwId.foreach { id => - frameworkId = id - } - recoverState() - metricsSystem.registerSource(new MesosClusterSchedulerSource(this)) - metricsSystem.start() - val driver = createSchedulerDriver( - master, - MesosClusterScheduler.this, - Utils.getCurrentUserName(), - appName, - conf, - Some(frameworkUrl), - Some(true), - Some(Integer.MAX_VALUE), - fwId) - - startScheduler(driver) - ready = true - } - - def stop(): Unit = { - ready = false - metricsSystem.report() - metricsSystem.stop() - mesosDriver.stop(true) - } - - override def registered( - driver: SchedulerDriver, - newFrameworkId: FrameworkID, - masterInfo: MasterInfo): Unit = { - logInfo("Registered as framework ID " + newFrameworkId.getValue) - if (newFrameworkId.getValue != frameworkId) { - frameworkId = newFrameworkId.getValue - schedulerState.persist("frameworkId", frameworkId) - } - markRegistered() - - stateLock.synchronized { - this.masterInfo = Some(masterInfo) - if (!pendingRecover.isEmpty) { - // Start task reconciliation if we need to recover. - val statuses = pendingRecover.collect { - case (taskId, slaveId) => - val newStatus = TaskStatus.newBuilder() - .setTaskId(TaskID.newBuilder().setValue(taskId).build()) - .setSlaveId(slaveId) - .setState(MesosTaskState.TASK_STAGING) - .build() - launchedDrivers.get(taskId).map(_.mesosTaskStatus.getOrElse(newStatus)) - .getOrElse(newStatus) - } - // TODO: Page the status updates to avoid trying to reconcile - // a large amount of tasks at once. - driver.reconcileTasks(statuses.toSeq.asJava) - } - } - } - - private def buildDriverCommand(desc: MesosDriverDescription): CommandInfo = { - val appJar = CommandInfo.URI.newBuilder() - .setValue(desc.jarUrl.stripPrefix("file:").stripPrefix("local:")).build() - val builder = CommandInfo.newBuilder().addUris(appJar) - val entries = - (conf.getOption("spark.executor.extraLibraryPath").toList ++ - desc.command.libraryPathEntries) - val prefixEnv = if (!entries.isEmpty) { - Utils.libraryPathEnvPrefix(entries) - } else { - "" - } - val envBuilder = Environment.newBuilder() - desc.command.environment.foreach { case (k, v) => - envBuilder.addVariables(Variable.newBuilder().setName(k).setValue(v).build()) - } - // Pass all spark properties to executor. - val executorOpts = desc.schedulerProperties.map { case (k, v) => s"-D$k=$v" }.mkString(" ") - envBuilder.addVariables( - Variable.newBuilder().setName("SPARK_EXECUTOR_OPTS").setValue(executorOpts)) - val dockerDefined = desc.schedulerProperties.contains("spark.mesos.executor.docker.image") - val executorUri = desc.schedulerProperties.get("spark.executor.uri") - .orElse(desc.command.environment.get("SPARK_EXECUTOR_URI")) - // Gets the path to run spark-submit, and the path to the Mesos sandbox. - val (executable, sandboxPath) = if (dockerDefined) { - // Application jar is automatically downloaded in the mounted sandbox by Mesos, - // and the path to the mounted volume is stored in $MESOS_SANDBOX env variable. - ("./bin/spark-submit", "$MESOS_SANDBOX") - } else if (executorUri.isDefined) { - builder.addUris(CommandInfo.URI.newBuilder().setValue(executorUri.get).build()) - val folderBasename = executorUri.get.split('/').last.split('.').head - val cmdExecutable = s"cd $folderBasename*; $prefixEnv bin/spark-submit" - // Sandbox path points to the parent folder as we chdir into the folderBasename. - (cmdExecutable, "..") - } else { - val executorSparkHome = desc.schedulerProperties.get("spark.mesos.executor.home") - .orElse(conf.getOption("spark.home")) - .orElse(Option(System.getenv("SPARK_HOME"))) - .getOrElse { - throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") - } - val cmdExecutable = new File(executorSparkHome, "./bin/spark-submit").getCanonicalPath - // Sandbox points to the current directory by default with Mesos. - (cmdExecutable, ".") - } - val primaryResource = new File(sandboxPath, desc.jarUrl.split("/").last).toString() - val cmdOptions = generateCmdOption(desc, sandboxPath).mkString(" ") - val appArguments = desc.command.arguments.mkString(" ") - builder.setValue(s"$executable $cmdOptions $primaryResource $appArguments") - builder.setEnvironment(envBuilder.build()) - conf.getOption("spark.mesos.uris").map { uris => - setupUris(uris, builder) - } - desc.schedulerProperties.get("spark.mesos.uris").map { uris => - setupUris(uris, builder) - } - desc.schedulerProperties.get("spark.submit.pyFiles").map { pyFiles => - setupUris(pyFiles, builder) - } - builder.build() - } - - private def generateCmdOption(desc: MesosDriverDescription, sandboxPath: String): Seq[String] = { - var options = Seq( - "--name", desc.schedulerProperties("spark.app.name"), - "--master", s"mesos://${conf.get("spark.master")}", - "--driver-cores", desc.cores.toString, - "--driver-memory", s"${desc.mem}M") - - // Assume empty main class means we're running python - if (!desc.command.mainClass.equals("")) { - options ++= Seq("--class", desc.command.mainClass) - } - - desc.schedulerProperties.get("spark.executor.memory").map { v => - options ++= Seq("--executor-memory", v) - } - desc.schedulerProperties.get("spark.cores.max").map { v => - options ++= Seq("--total-executor-cores", v) - } - desc.schedulerProperties.get("spark.submit.pyFiles").map { pyFiles => - val formattedFiles = pyFiles.split(",") - .map { path => new File(sandboxPath, path.split("/").last).toString() } - .mkString(",") - options ++= Seq("--py-files", formattedFiles) - } - options - } - - private class ResourceOffer(val offer: Offer, var cpu: Double, var mem: Double) { - override def toString(): String = { - s"Offer id: ${offer.getId.getValue}, cpu: $cpu, mem: $mem" - } - } - - /** - * This method takes all the possible candidates and attempt to schedule them with Mesos offers. - * Every time a new task is scheduled, the afterLaunchCallback is called to perform post scheduled - * logic on each task. - */ - private def scheduleTasks( - candidates: Seq[MesosDriverDescription], - afterLaunchCallback: (String) => Boolean, - currentOffers: List[ResourceOffer], - tasks: mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]): Unit = { - for (submission <- candidates) { - val driverCpu = submission.cores - val driverMem = submission.mem - logTrace(s"Finding offer to launch driver with cpu: $driverCpu, mem: $driverMem") - val offerOption = currentOffers.find { o => - o.cpu >= driverCpu && o.mem >= driverMem - } - if (offerOption.isEmpty) { - logDebug(s"Unable to find offer to launch driver id: ${submission.submissionId}, " + - s"cpu: $driverCpu, mem: $driverMem") - } else { - val offer = offerOption.get - offer.cpu -= driverCpu - offer.mem -= driverMem - val taskId = TaskID.newBuilder().setValue(submission.submissionId).build() - val cpuResource = createResource("cpus", driverCpu) - val memResource = createResource("mem", driverMem) - val commandInfo = buildDriverCommand(submission) - val appName = submission.schedulerProperties("spark.app.name") - val taskInfo = TaskInfo.newBuilder() - .setTaskId(taskId) - .setName(s"Driver for $appName") - .setSlaveId(offer.offer.getSlaveId) - .setCommand(commandInfo) - .addResources(cpuResource) - .addResources(memResource) - submission.schedulerProperties.get("spark.mesos.executor.docker.image").foreach { image => - val container = taskInfo.getContainerBuilder() - val volumes = submission.schedulerProperties - .get("spark.mesos.executor.docker.volumes") - .map(MesosSchedulerBackendUtil.parseVolumesSpec) - val portmaps = submission.schedulerProperties - .get("spark.mesos.executor.docker.portmaps") - .map(MesosSchedulerBackendUtil.parsePortMappingsSpec) - MesosSchedulerBackendUtil.addDockerInfo( - container, image, volumes = volumes, portmaps = portmaps) - taskInfo.setContainer(container.build()) - } - val queuedTasks = tasks.getOrElseUpdate(offer.offer.getId, new ArrayBuffer[TaskInfo]) - queuedTasks += taskInfo.build() - logTrace(s"Using offer ${offer.offer.getId.getValue} to launch driver " + - submission.submissionId) - val newState = new MesosClusterSubmissionState(submission, taskId, offer.offer.getSlaveId, - None, new Date(), None) - launchedDrivers(submission.submissionId) = newState - launchedDriversState.persist(submission.submissionId, newState) - afterLaunchCallback(submission.submissionId) - } - } - } - - override def resourceOffers(driver: SchedulerDriver, offers: JList[Offer]): Unit = { - val currentOffers = offers.asScala.map(o => - new ResourceOffer( - o, getResource(o.getResourcesList, "cpus"), getResource(o.getResourcesList, "mem")) - ).toList - logTrace(s"Received offers from Mesos: \n${currentOffers.mkString("\n")}") - val tasks = new mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]() - val currentTime = new Date() - - stateLock.synchronized { - // We first schedule all the supervised drivers that are ready to retry. - // This list will be empty if none of the drivers are marked as supervise. - val driversToRetry = pendingRetryDrivers.filter { d => - d.retryState.get.nextRetry.before(currentTime) - } - - scheduleTasks( - copyBuffer(driversToRetry), - removeFromPendingRetryDrivers, - currentOffers, - tasks) - - // Then we walk through the queued drivers and try to schedule them. - scheduleTasks( - copyBuffer(queuedDrivers), - removeFromQueuedDrivers, - currentOffers, - tasks) - } - tasks.foreach { case (offerId, taskInfos) => - driver.launchTasks(Collections.singleton(offerId), taskInfos.asJava) - } - offers.asScala - .filter(o => !tasks.keySet.contains(o.getId)) - .foreach(o => driver.declineOffer(o.getId)) - } - - private def copyBuffer( - buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { - val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) - buffer.copyToBuffer(newBuffer) - newBuffer - } - - def getSchedulerState(): MesosClusterSchedulerState = { - stateLock.synchronized { - new MesosClusterSchedulerState( - frameworkId, - masterInfo.map(m => s"http://${m.getIp}:${m.getPort}"), - copyBuffer(queuedDrivers), - launchedDrivers.values.map(_.copy()).toList, - finishedDrivers.map(_.copy()).toList, - copyBuffer(pendingRetryDrivers)) - } - } - - override def offerRescinded(driver: SchedulerDriver, offerId: OfferID): Unit = {} - override def disconnected(driver: SchedulerDriver): Unit = {} - override def reregistered(driver: SchedulerDriver, masterInfo: MasterInfo): Unit = { - logInfo(s"Framework re-registered with master ${masterInfo.getId}") - } - override def slaveLost(driver: SchedulerDriver, slaveId: SlaveID): Unit = {} - override def error(driver: SchedulerDriver, error: String): Unit = { - logError("Error received: " + error) - } - - /** - * Check if the task state is a recoverable state that we can relaunch the task. - * Task state like TASK_ERROR are not relaunchable state since it wasn't able - * to be validated by Mesos. - */ - private def shouldRelaunch(state: MesosTaskState): Boolean = { - state == MesosTaskState.TASK_FAILED || - state == MesosTaskState.TASK_KILLED || - state == MesosTaskState.TASK_LOST - } - - override def statusUpdate(driver: SchedulerDriver, status: TaskStatus): Unit = { - val taskId = status.getTaskId.getValue - stateLock.synchronized { - if (launchedDrivers.contains(taskId)) { - if (status.getReason == Reason.REASON_RECONCILIATION && - !pendingRecover.contains(taskId)) { - // Task has already received update and no longer requires reconciliation. - return - } - val state = launchedDrivers(taskId) - // Check if the driver is supervise enabled and can be relaunched. - if (state.driverDescription.supervise && shouldRelaunch(status.getState)) { - removeFromLaunchedDrivers(taskId) - state.finishDate = Some(new Date()) - val retryState: Option[MesosClusterRetryState] = state.driverDescription.retryState - val (retries, waitTimeSec) = retryState - .map { rs => (rs.retries + 1, Math.min(maxRetryWaitTime, rs.waitTime * 2)) } - .getOrElse{ (1, 1) } - val nextRetry = new Date(new Date().getTime + waitTimeSec * 1000L) - - val newDriverDescription = state.driverDescription.copy( - retryState = Some(new MesosClusterRetryState(status, retries, nextRetry, waitTimeSec))) - pendingRetryDrivers += newDriverDescription - pendingRetryDriversState.persist(taskId, newDriverDescription) - } else if (TaskState.isFinished(TaskState.fromMesos(status.getState))) { - removeFromLaunchedDrivers(taskId) - state.finishDate = Some(new Date()) - if (finishedDrivers.size >= retainedDrivers) { - val toRemove = math.max(retainedDrivers / 10, 1) - finishedDrivers.trimStart(toRemove) - } - finishedDrivers += state - } - state.mesosTaskStatus = Option(status) - } else { - logError(s"Unable to find driver $taskId in status update") - } - } - } - - override def frameworkMessage( - driver: SchedulerDriver, - executorId: ExecutorID, - slaveId: SlaveID, - message: Array[Byte]): Unit = {} - - override def executorLost( - driver: SchedulerDriver, - executorId: ExecutorID, - slaveId: SlaveID, - status: Int): Unit = {} - - private def removeFromQueuedDrivers(id: String): Boolean = { - val index = queuedDrivers.indexWhere(_.submissionId.equals(id)) - if (index != -1) { - queuedDrivers.remove(index) - queuedDriversState.expunge(id) - true - } else { - false - } - } - - private def removeFromLaunchedDrivers(id: String): Boolean = { - if (launchedDrivers.remove(id).isDefined) { - launchedDriversState.expunge(id) - true - } else { - false - } - } - - private def removeFromPendingRetryDrivers(id: String): Boolean = { - val index = pendingRetryDrivers.indexWhere(_.submissionId.equals(id)) - if (index != -1) { - pendingRetryDrivers.remove(index) - pendingRetryDriversState.expunge(id) - true - } else { - false - } - } - - def getQueuedDriversSize: Int = queuedDrivers.size - def getLaunchedDriversSize: Int = launchedDrivers.size - def getPendingRetryDriversSize: Int = pendingRetryDrivers.size -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala deleted file mode 100644 index aaffac604a88..000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ /dev/null @@ -1,416 +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.scheduler.cluster.mesos - -import java.io.File -import java.util.{ArrayList => JArrayList, Collections, List => JList} - -import scala.collection.JavaConverters._ -import scala.collection.mutable.{HashMap, HashSet} - -import org.apache.mesos.{Scheduler => MScheduler, _} -import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, TaskInfo => MesosTaskInfo, _} -import org.apache.mesos.protobuf.ByteString -import org.apache.spark.{SparkContext, SparkException, TaskState} -import org.apache.spark.executor.MesosExecutorBackend -import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.util.Utils - -/** - * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a - * separate Mesos task, allowing multiple applications to share cluster nodes both in space (tasks - * from multiple apps can run on different cores) and in time (a core can switch ownership). - */ -private[spark] class MesosSchedulerBackend( - scheduler: TaskSchedulerImpl, - sc: SparkContext, - master: String) - extends SchedulerBackend - with MScheduler - with MesosSchedulerUtils { - - // Stores the slave ids that has launched a Mesos executor. - val slaveIdToExecutorInfo = new HashMap[String, MesosExecutorInfo] - val taskIdToSlaveId = new HashMap[Long, String] - - // An ExecutorInfo for our tasks - var execArgs: Array[Byte] = null - - var classLoader: ClassLoader = null - - // The listener bus to publish executor added/removed events. - val listenerBus = sc.listenerBus - - private[mesos] val mesosExecutorCores = sc.conf.getDouble("spark.mesos.mesosExecutor.cores", 1) - - // Offer constraints - private[this] val slaveOfferConstraints = - parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) - - @volatile var appId: String = _ - - override def start() { - classLoader = Thread.currentThread.getContextClassLoader - val driver = createSchedulerDriver( - master, - MesosSchedulerBackend.this, - sc.sparkUser, - sc.appName, - sc.conf, - sc.ui.map(_.appUIAddress)) - startScheduler(driver) - } - - /** - * Creates a MesosExecutorInfo that is used to launch a Mesos executor. - * @param availableResources Available resources that is offered by Mesos - * @param execId The executor id to assign to this new executor. - * @return A tuple of the new mesos executor info and the remaining available resources. - */ - def createExecutorInfo( - availableResources: JList[Resource], - execId: String): (MesosExecutorInfo, JList[Resource]) = { - val executorSparkHome = sc.conf.getOption("spark.mesos.executor.home") - .orElse(sc.getSparkHome()) // Fall back to driver Spark home for backward compatibility - .getOrElse { - throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") - } - val environment = Environment.newBuilder() - sc.conf.getOption("spark.executor.extraClassPath").foreach { cp => - environment.addVariables( - Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) - } - val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions").getOrElse("") - - val prefixEnv = sc.conf.getOption("spark.executor.extraLibraryPath").map { p => - Utils.libraryPathEnvPrefix(Seq(p)) - }.getOrElse("") - - environment.addVariables( - Environment.Variable.newBuilder() - .setName("SPARK_EXECUTOR_OPTS") - .setValue(extraJavaOpts) - .build()) - sc.executorEnvs.foreach { case (key, value) => - environment.addVariables(Environment.Variable.newBuilder() - .setName(key) - .setValue(value) - .build()) - } - val command = CommandInfo.newBuilder() - .setEnvironment(environment) - val uri = sc.conf.getOption("spark.executor.uri") - .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) - - val executorBackendName = classOf[MesosExecutorBackend].getName - if (uri.isEmpty) { - val executorPath = new File(executorSparkHome, "/bin/spark-class").getCanonicalPath - command.setValue(s"$prefixEnv $executorPath $executorBackendName") - } else { - // Grab everything to the first '.'. We'll use that and '*' to - // glob the directory "correctly". - val basename = uri.get.split('/').last.split('.').head - command.setValue(s"cd ${basename}*; $prefixEnv ./bin/spark-class $executorBackendName") - command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get)) - } - val builder = MesosExecutorInfo.newBuilder() - val (resourcesAfterCpu, usedCpuResources) = - partitionResources(availableResources, "cpus", mesosExecutorCores) - val (resourcesAfterMem, usedMemResources) = - partitionResources(resourcesAfterCpu.asJava, "mem", calculateTotalMemory(sc)) - - builder.addAllResources(usedCpuResources.asJava) - builder.addAllResources(usedMemResources.asJava) - - sc.conf.getOption("spark.mesos.uris").foreach(setupUris(_, command)) - - val executorInfo = builder - .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) - .setCommand(command) - .setData(ByteString.copyFrom(createExecArg())) - - sc.conf.getOption("spark.mesos.executor.docker.image").foreach { image => - MesosSchedulerBackendUtil - .setupContainerBuilderDockerInfo(image, sc.conf, executorInfo.getContainerBuilder()) - } - - (executorInfo.build(), resourcesAfterMem.asJava) - } - - /** - * Create and serialize the executor argument to pass to Mesos. Our executor arg is an array - * containing all the spark.* system properties in the form of (String, String) pairs. - */ - private def createExecArg(): Array[Byte] = { - if (execArgs == null) { - val props = new HashMap[String, String] - for ((key, value) <- sc.conf.getAll) { - props(key) = value - } - // Serialize the map as an array of (String, String) pairs - execArgs = Utils.serialize(props.toArray) - } - execArgs - } - - override def offerRescinded(d: SchedulerDriver, o: OfferID) {} - - override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { - inClassLoader() { - appId = frameworkId.getValue - logInfo("Registered as framework ID " + appId) - markRegistered() - } - } - - private def inClassLoader()(fun: => Unit) = { - val oldClassLoader = Thread.currentThread.getContextClassLoader - Thread.currentThread.setContextClassLoader(classLoader) - try { - fun - } finally { - Thread.currentThread.setContextClassLoader(oldClassLoader) - } - } - - override def disconnected(d: SchedulerDriver) {} - - override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {} - - private def getTasksSummary(tasks: JArrayList[MesosTaskInfo]): String = { - val builder = new StringBuilder - tasks.asScala.foreach { t => - builder.append("Task id: ").append(t.getTaskId.getValue).append("\n") - .append("Slave id: ").append(t.getSlaveId.getValue).append("\n") - .append("Task resources: ").append(t.getResourcesList).append("\n") - .append("Executor resources: ").append(t.getExecutor.getResourcesList) - .append("---------------------------------------------\n") - } - builder.toString() - } - - /** - * Method called by Mesos to offer resources on slaves. We respond by asking our active task sets - * for tasks in order of priority. We fill each node with tasks in a round-robin manner so that - * tasks are balanced across the cluster. - */ - override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { - inClassLoader() { - // Fail-fast on offers we know will be rejected - val (usableOffers, unUsableOffers) = offers.asScala.partition { o => - val mem = getResource(o.getResourcesList, "mem") - val cpus = getResource(o.getResourcesList, "cpus") - val slaveId = o.getSlaveId.getValue - val offerAttributes = toAttributeMap(o.getAttributesList) - - // check if all constraints are satisfield - // 1. Attribute constraints - // 2. Memory requirements - // 3. CPU requirements - need at least 1 for executor, 1 for task - val meetsConstraints = matchesAttributeRequirements(slaveOfferConstraints, offerAttributes) - val meetsMemoryRequirements = mem >= calculateTotalMemory(sc) - val meetsCPURequirements = cpus >= (mesosExecutorCores + scheduler.CPUS_PER_TASK) - - val meetsRequirements = - (meetsConstraints && meetsMemoryRequirements && meetsCPURequirements) || - (slaveIdToExecutorInfo.contains(slaveId) && cpus >= scheduler.CPUS_PER_TASK) - - // add some debug messaging - val debugstr = if (meetsRequirements) "Accepting" else "Declining" - val id = o.getId.getValue - logDebug(s"$debugstr offer: $id with attributes: $offerAttributes mem: $mem cpu: $cpus") - - meetsRequirements - } - - // Decline offers we ruled out immediately - unUsableOffers.foreach(o => d.declineOffer(o.getId)) - - val workerOffers = usableOffers.map { o => - val cpus = if (slaveIdToExecutorInfo.contains(o.getSlaveId.getValue)) { - getResource(o.getResourcesList, "cpus").toInt - } else { - // If the Mesos executor has not been started on this slave yet, set aside a few - // cores for the Mesos executor by offering fewer cores to the Spark executor - (getResource(o.getResourcesList, "cpus") - mesosExecutorCores).toInt - } - new WorkerOffer( - o.getSlaveId.getValue, - o.getHostname, - cpus) - } - - val slaveIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap - val slaveIdToWorkerOffer = workerOffers.map(o => o.executorId -> o).toMap - val slaveIdToResources = new HashMap[String, JList[Resource]]() - usableOffers.foreach { o => - slaveIdToResources(o.getSlaveId.getValue) = o.getResourcesList - } - - val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]] - - val slavesIdsOfAcceptedOffers = HashSet[String]() - - // Call into the TaskSchedulerImpl - val acceptedOffers = scheduler.resourceOffers(workerOffers).filter(!_.isEmpty) - acceptedOffers - .foreach { offer => - offer.foreach { taskDesc => - val slaveId = taskDesc.executorId - slavesIdsOfAcceptedOffers += slaveId - taskIdToSlaveId(taskDesc.taskId) = slaveId - val (mesosTask, remainingResources) = createMesosTask( - taskDesc, - slaveIdToResources(slaveId), - slaveId) - mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo]) - .add(mesosTask) - slaveIdToResources(slaveId) = remainingResources - } - } - - // Reply to the offers - val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? - - mesosTasks.foreach { case (slaveId, tasks) => - slaveIdToWorkerOffer.get(slaveId).foreach(o => - listenerBus.post(SparkListenerExecutorAdded(System.currentTimeMillis(), slaveId, - // TODO: Add support for log urls for Mesos - new ExecutorInfo(o.host, o.cores, Map.empty))) - ) - logTrace(s"Launching Mesos tasks on slave '$slaveId', tasks:\n${getTasksSummary(tasks)}") - d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) - } - - // Decline offers that weren't used - // NOTE: This logic assumes that we only get a single offer for each host in a given batch - for (o <- usableOffers if !slavesIdsOfAcceptedOffers.contains(o.getSlaveId.getValue)) { - d.declineOffer(o.getId) - } - } - } - - /** Turn a Spark TaskDescription into a Mesos task and also resources unused by the task */ - def createMesosTask( - task: TaskDescription, - resources: JList[Resource], - slaveId: String): (MesosTaskInfo, JList[Resource]) = { - val taskId = TaskID.newBuilder().setValue(task.taskId.toString).build() - val (executorInfo, remainingResources) = if (slaveIdToExecutorInfo.contains(slaveId)) { - (slaveIdToExecutorInfo(slaveId), resources) - } else { - createExecutorInfo(resources, slaveId) - } - slaveIdToExecutorInfo(slaveId) = executorInfo - val (finalResources, cpuResources) = - partitionResources(remainingResources, "cpus", scheduler.CPUS_PER_TASK) - val taskInfo = MesosTaskInfo.newBuilder() - .setTaskId(taskId) - .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build()) - .setExecutor(executorInfo) - .setName(task.name) - .addAllResources(cpuResources.asJava) - .setData(MesosTaskLaunchData(task.serializedTask, task.attemptNumber).toByteString) - .build() - (taskInfo, finalResources.asJava) - } - - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { - inClassLoader() { - val tid = status.getTaskId.getValue.toLong - val state = TaskState.fromMesos(status.getState) - synchronized { - if (TaskState.isFailed(TaskState.fromMesos(status.getState)) - && taskIdToSlaveId.contains(tid)) { - // We lost the executor on this slave, so remember that it's gone - removeExecutor(taskIdToSlaveId(tid), "Lost executor") - } - if (TaskState.isFinished(state)) { - taskIdToSlaveId.remove(tid) - } - } - scheduler.statusUpdate(tid, state, status.getData.asReadOnlyByteBuffer) - } - } - - override def error(d: SchedulerDriver, message: String) { - inClassLoader() { - logError("Mesos error: " + message) - scheduler.error(message) - } - } - - override def stop() { - if (mesosDriver != null) { - mesosDriver.stop() - } - } - - override def reviveOffers() { - mesosDriver.reviveOffers() - } - - override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} - - /** - * Remove executor associated with slaveId in a thread safe manner. - */ - private def removeExecutor(slaveId: String, reason: String) = { - synchronized { - listenerBus.post(SparkListenerExecutorRemoved(System.currentTimeMillis(), slaveId, reason)) - slaveIdToExecutorInfo -= slaveId - } - } - - private def recordSlaveLost(d: SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) { - inClassLoader() { - logInfo("Mesos slave lost: " + slaveId.getValue) - removeExecutor(slaveId.getValue, reason.toString) - scheduler.executorLost(slaveId.getValue, reason) - } - } - - override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { - recordSlaveLost(d, slaveId, SlaveLost()) - } - - override def executorLost(d: SchedulerDriver, executorId: ExecutorID, - slaveId: SlaveID, status: Int) { - logInfo("Executor lost: %s, marking slave %s as lost".format(executorId.getValue, - slaveId.getValue)) - recordSlaveLost(d, slaveId, ExecutorExited(status, exitCausedByApp = true)) - } - - override def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = { - mesosDriver.killTask( - TaskID.newBuilder() - .setValue(taskId.toString).build() - ) - } - - // TODO: query Mesos for number of cores - override def defaultParallelism(): Int = sc.conf.getInt("spark.default.parallelism", 8) - - override def applicationId(): String = - Option(appId).getOrElse { - logWarning("Application ID is not initialized yet.") - super.applicationId - } - -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala deleted file mode 100644 index e79c543a9de2..000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala +++ /dev/null @@ -1,142 +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.scheduler.cluster.mesos - -import org.apache.mesos.Protos.{ContainerInfo, Volume} -import org.apache.mesos.Protos.ContainerInfo.DockerInfo - -import org.apache.spark.{Logging, SparkConf} - -/** - * A collection of utility functions which can be used by both the - * MesosSchedulerBackend and the CoarseMesosSchedulerBackend. - */ -private[mesos] object MesosSchedulerBackendUtil extends Logging { - /** - * Parse a comma-delimited list of volume specs, each of which - * takes the form [host-dir:]container-dir[:rw|:ro]. - */ - def parseVolumesSpec(volumes: String): List[Volume] = { - volumes.split(",").map(_.split(":")).flatMap { spec => - val vol: Volume.Builder = Volume - .newBuilder() - .setMode(Volume.Mode.RW) - spec match { - case Array(container_path) => - Some(vol.setContainerPath(container_path)) - case Array(container_path, "rw") => - Some(vol.setContainerPath(container_path)) - case Array(container_path, "ro") => - Some(vol.setContainerPath(container_path) - .setMode(Volume.Mode.RO)) - case Array(host_path, container_path) => - Some(vol.setContainerPath(container_path) - .setHostPath(host_path)) - case Array(host_path, container_path, "rw") => - Some(vol.setContainerPath(container_path) - .setHostPath(host_path)) - case Array(host_path, container_path, "ro") => - Some(vol.setContainerPath(container_path) - .setHostPath(host_path) - .setMode(Volume.Mode.RO)) - case spec => { - logWarning(s"Unable to parse volume specs: $volumes. " - + "Expected form: \"[host-dir:]container-dir[:rw|:ro](, ...)\"") - None - } - } - } - .map { _.build() } - .toList - } - - /** - * Parse a comma-delimited list of port mapping specs, each of which - * takes the form host_port:container_port[:udp|:tcp] - * - * Note: - * the docker form is [ip:]host_port:container_port, but the DockerInfo - * message has no field for 'ip', and instead has a 'protocol' field. - * Docker itself only appears to support TCP, so this alternative form - * anticipates the expansion of the docker form to allow for a protocol - * and leaves open the chance for mesos to begin to accept an 'ip' field - */ - def parsePortMappingsSpec(portmaps: String): List[DockerInfo.PortMapping] = { - portmaps.split(",").map(_.split(":")).flatMap { spec: Array[String] => - val portmap: DockerInfo.PortMapping.Builder = DockerInfo.PortMapping - .newBuilder() - .setProtocol("tcp") - spec match { - case Array(host_port, container_port) => - Some(portmap.setHostPort(host_port.toInt) - .setContainerPort(container_port.toInt)) - case Array(host_port, container_port, protocol) => - Some(portmap.setHostPort(host_port.toInt) - .setContainerPort(container_port.toInt) - .setProtocol(protocol)) - case spec => { - logWarning(s"Unable to parse port mapping specs: $portmaps. " - + "Expected form: \"host_port:container_port[:udp|:tcp](, ...)\"") - None - } - } - } - .map { _.build() } - .toList - } - - /** - * Construct a DockerInfo structure and insert it into a ContainerInfo - */ - def addDockerInfo( - container: ContainerInfo.Builder, - image: String, - volumes: Option[List[Volume]] = None, - network: Option[ContainerInfo.DockerInfo.Network] = None, - portmaps: Option[List[ContainerInfo.DockerInfo.PortMapping]] = None): Unit = { - - val docker = ContainerInfo.DockerInfo.newBuilder().setImage(image) - - network.foreach(docker.setNetwork) - portmaps.foreach(_.foreach(docker.addPortMappings)) - container.setType(ContainerInfo.Type.DOCKER) - container.setDocker(docker.build()) - volumes.foreach(_.foreach(container.addVolumes)) - } - - /** - * Setup a docker containerizer - */ - def setupContainerBuilderDockerInfo( - imageName: String, - conf: SparkConf, - builder: ContainerInfo.Builder): Unit = { - val volumes = conf - .getOption("spark.mesos.executor.docker.volumes") - .map(parseVolumesSpec) - val portmaps = conf - .getOption("spark.mesos.executor.docker.portmaps") - .map(parsePortMappingsSpec) - addDockerInfo( - builder, - imageName, - volumes = volumes, - portmaps = portmaps) - logDebug("setupContainerDockerInfo: using docker image: " + imageName) - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala deleted file mode 100644 index 860c8e097b3b..000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ /dev/null @@ -1,339 +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.scheduler.cluster.mesos - -import java.util.{List => JList} -import java.util.concurrent.CountDownLatch - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer -import scala.util.control.NonFatal - -import com.google.common.base.Splitter -import org.apache.mesos.{MesosSchedulerDriver, SchedulerDriver, Scheduler, Protos} -import org.apache.mesos.Protos._ -import org.apache.mesos.protobuf.{ByteString, GeneratedMessage} -import org.apache.spark.{SparkException, SparkConf, Logging, SparkContext} -import org.apache.spark.util.Utils - - -/** - * Shared trait for implementing a Mesos Scheduler. This holds common state and helper - * methods and Mesos scheduler will use. - */ -private[mesos] trait MesosSchedulerUtils extends Logging { - // Lock used to wait for scheduler to be registered - private final val registerLatch = new CountDownLatch(1) - - // Driver for talking to Mesos - protected var mesosDriver: SchedulerDriver = null - - /** - * Creates a new MesosSchedulerDriver that communicates to the Mesos master. - * @param masterUrl The url to connect to Mesos master - * @param scheduler the scheduler class to receive scheduler callbacks - * @param sparkUser User to impersonate with when running tasks - * @param appName The framework name to display on the Mesos UI - * @param conf Spark configuration - * @param webuiUrl The WebUI url to link from Mesos UI - * @param checkpoint Option to checkpoint tasks for failover - * @param failoverTimeout Duration Mesos master expect scheduler to reconnect on disconnect - * @param frameworkId The id of the new framework - */ - protected def createSchedulerDriver( - masterUrl: String, - scheduler: Scheduler, - sparkUser: String, - appName: String, - conf: SparkConf, - webuiUrl: Option[String] = None, - checkpoint: Option[Boolean] = None, - failoverTimeout: Option[Double] = None, - frameworkId: Option[String] = None): SchedulerDriver = { - val fwInfoBuilder = FrameworkInfo.newBuilder().setUser(sparkUser).setName(appName) - val credBuilder = Credential.newBuilder() - webuiUrl.foreach { url => fwInfoBuilder.setWebuiUrl(url) } - checkpoint.foreach { checkpoint => fwInfoBuilder.setCheckpoint(checkpoint) } - failoverTimeout.foreach { timeout => fwInfoBuilder.setFailoverTimeout(timeout) } - frameworkId.foreach { id => - fwInfoBuilder.setId(FrameworkID.newBuilder().setValue(id).build()) - } - conf.getOption("spark.mesos.principal").foreach { principal => - fwInfoBuilder.setPrincipal(principal) - credBuilder.setPrincipal(principal) - } - conf.getOption("spark.mesos.secret").foreach { secret => - credBuilder.setSecret(ByteString.copyFromUtf8(secret)) - } - if (credBuilder.hasSecret && !fwInfoBuilder.hasPrincipal) { - throw new SparkException( - "spark.mesos.principal must be configured when spark.mesos.secret is set") - } - conf.getOption("spark.mesos.role").foreach { role => - fwInfoBuilder.setRole(role) - } - if (credBuilder.hasPrincipal) { - new MesosSchedulerDriver( - scheduler, fwInfoBuilder.build(), masterUrl, credBuilder.build()) - } else { - new MesosSchedulerDriver(scheduler, fwInfoBuilder.build(), masterUrl) - } - } - - /** - * Starts the MesosSchedulerDriver and stores the current running driver to this new instance. - * This driver is expected to not be running. - * This method returns only after the scheduler has registered with Mesos. - */ - def startScheduler(newDriver: SchedulerDriver): Unit = { - synchronized { - if (mesosDriver != null) { - registerLatch.await() - return - } - - new Thread(Utils.getFormattedClassName(this) + "-mesos-driver") { - setDaemon(true) - - override def run() { - mesosDriver = newDriver - try { - val ret = mesosDriver.run() - logInfo("driver.run() returned with code " + ret) - if (ret != null && ret.equals(Status.DRIVER_ABORTED)) { - System.exit(1) - } - } catch { - case e: Exception => { - logError("driver.run() failed", e) - System.exit(1) - } - } - } - }.start() - - registerLatch.await() - } - } - - /** - * Signal that the scheduler has registered with Mesos. - */ - protected def getResource(res: JList[Resource], name: String): Double = { - // A resource can have multiple values in the offer since it can either be from - // a specific role or wildcard. - res.asScala.filter(_.getName == name).map(_.getScalar.getValue).sum - } - - protected def markRegistered(): Unit = { - registerLatch.countDown() - } - - def createResource(name: String, amount: Double, role: Option[String] = None): Resource = { - val builder = Resource.newBuilder() - .setName(name) - .setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(amount).build()) - - role.foreach { r => builder.setRole(r) } - - builder.build() - } - - /** - * Partition the existing set of resources into two groups, those remaining to be - * scheduled and those requested to be used for a new task. - * @param resources The full list of available resources - * @param resourceName The name of the resource to take from the available resources - * @param amountToUse The amount of resources to take from the available resources - * @return The remaining resources list and the used resources list. - */ - def partitionResources( - resources: JList[Resource], - resourceName: String, - amountToUse: Double): (List[Resource], List[Resource]) = { - var remain = amountToUse - var requestedResources = new ArrayBuffer[Resource] - val remainingResources = resources.asScala.map { - case r => { - if (remain > 0 && - r.getType == Value.Type.SCALAR && - r.getScalar.getValue > 0.0 && - r.getName == resourceName) { - val usage = Math.min(remain, r.getScalar.getValue) - requestedResources += createResource(resourceName, usage, Some(r.getRole)) - remain -= usage - createResource(resourceName, r.getScalar.getValue - usage, Some(r.getRole)) - } else { - r - } - } - } - - // Filter any resource that has depleted. - val filteredResources = - remainingResources.filter(r => r.getType != Value.Type.SCALAR || r.getScalar.getValue > 0.0) - - (filteredResources.toList, requestedResources.toList) - } - - /** Helper method to get the key,value-set pair for a Mesos Attribute protobuf */ - protected def getAttribute(attr: Attribute): (String, Set[String]) = { - (attr.getName, attr.getText.getValue.split(',').toSet) - } - - - /** Build a Mesos resource protobuf object */ - protected def createResource(resourceName: String, quantity: Double): Protos.Resource = { - Resource.newBuilder() - .setName(resourceName) - .setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(quantity).build()) - .build() - } - - /** - * Converts the attributes from the resource offer into a Map of name -> Attribute Value - * The attribute values are the mesos attribute types and they are - * @param offerAttributes - * @return - */ - protected def toAttributeMap(offerAttributes: JList[Attribute]): Map[String, GeneratedMessage] = { - offerAttributes.asScala.map(attr => { - val attrValue = attr.getType match { - case Value.Type.SCALAR => attr.getScalar - case Value.Type.RANGES => attr.getRanges - case Value.Type.SET => attr.getSet - case Value.Type.TEXT => attr.getText - } - (attr.getName, attrValue) - }).toMap - } - - - /** - * Match the requirements (if any) to the offer attributes. - * if attribute requirements are not specified - return true - * else if attribute is defined and no values are given, simple attribute presence is performed - * else if attribute name and value is specified, subset match is performed on slave attributes - */ - def matchesAttributeRequirements( - slaveOfferConstraints: Map[String, Set[String]], - offerAttributes: Map[String, GeneratedMessage]): Boolean = { - slaveOfferConstraints.forall { - // offer has the required attribute and subsumes the required values for that attribute - case (name, requiredValues) => - offerAttributes.get(name) match { - case None => false - case Some(_) if requiredValues.isEmpty => true // empty value matches presence - case Some(scalarValue: Value.Scalar) => - // check if provided values is less than equal to the offered values - requiredValues.map(_.toDouble).exists(_ <= scalarValue.getValue) - case Some(rangeValue: Value.Range) => - val offerRange = rangeValue.getBegin to rangeValue.getEnd - // Check if there is some required value that is between the ranges specified - // Note: We only support the ability to specify discrete values, in the future - // we may expand it to subsume ranges specified with a XX..YY value or something - // similar to that. - requiredValues.map(_.toLong).exists(offerRange.contains(_)) - case Some(offeredValue: Value.Set) => - // check if the specified required values is a subset of offered set - requiredValues.subsetOf(offeredValue.getItemList.asScala.toSet) - case Some(textValue: Value.Text) => - // check if the specified value is equal, if multiple values are specified - // we succeed if any of them match. - requiredValues.contains(textValue.getValue) - } - } - } - - /** - * Parses the attributes constraints provided to spark and build a matching data struct: - * Map[, Set[values-to-match]] - * The constraints are specified as ';' separated key-value pairs where keys and values - * are separated by ':'. The ':' implies equality (for singular values) and "is one of" for - * multiple values (comma separated). For example: - * {{{ - * parseConstraintString("tachyon:true;zone:us-east-1a,us-east-1b") - * // would result in - * - * Map( - * "tachyon" -> Set("true"), - * "zone": -> Set("us-east-1a", "us-east-1b") - * ) - * }}} - * - * Mesos documentation: http://mesos.apache.org/documentation/attributes-resources/ - * https://github.com/apache/mesos/blob/master/src/common/values.cpp - * https://github.com/apache/mesos/blob/master/src/common/attributes.cpp - * - * @param constraintsVal constaints string consisting of ';' separated key-value pairs (separated - * by ':') - * @return Map of constraints to match resources offers. - */ - def parseConstraintString(constraintsVal: String): Map[String, Set[String]] = { - /* - Based on mesos docs: - attributes : attribute ( ";" attribute )* - attribute : labelString ":" ( labelString | "," )+ - labelString : [a-zA-Z0-9_/.-] - */ - val splitter = Splitter.on(';').trimResults().withKeyValueSeparator(':') - // kv splitter - if (constraintsVal.isEmpty) { - Map() - } else { - try { - splitter.split(constraintsVal).asScala.toMap.mapValues(v => - if (v == null || v.isEmpty) { - Set[String]() - } else { - v.split(',').toSet - } - ) - } catch { - case NonFatal(e) => - throw new IllegalArgumentException(s"Bad constraint string: $constraintsVal", e) - } - } - } - - // These defaults copied from YARN - private val MEMORY_OVERHEAD_FRACTION = 0.10 - private val MEMORY_OVERHEAD_MINIMUM = 384 - - /** - * Return the amount of memory to allocate to each executor, taking into account - * container overheads. - * @param sc SparkContext to use to get `spark.mesos.executor.memoryOverhead` value - * @return memory requirement as (0.1 * ) or MEMORY_OVERHEAD_MINIMUM - * (whichever is larger) - */ - def calculateTotalMemory(sc: SparkContext): Int = { - sc.conf.getInt("spark.mesos.executor.memoryOverhead", - math.max(MEMORY_OVERHEAD_FRACTION * sc.executorMemory, MEMORY_OVERHEAD_MINIMUM).toInt) + - sc.executorMemory - } - - def setupUris(uris: String, builder: CommandInfo.Builder): Unit = { - uris.split(",").foreach { uri => - builder.addUris(CommandInfo.URI.newBuilder().setValue(uri.trim())) - } - } - -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala deleted file mode 100644 index 5e7e6567a3e0..000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import java.nio.ByteBuffer - -import org.apache.mesos.protobuf.ByteString - -import org.apache.spark.Logging - -/** - * Wrapper for serializing the data sent when launching Mesos tasks. - */ -private[spark] case class MesosTaskLaunchData( - serializedTask: ByteBuffer, - attemptNumber: Int) extends Logging { - - def toByteString: ByteString = { - val dataBuffer = ByteBuffer.allocate(4 + serializedTask.limit) - dataBuffer.putInt(attemptNumber) - dataBuffer.put(serializedTask) - dataBuffer.rewind - logDebug(s"ByteBuffer size: [${dataBuffer.remaining}]") - ByteString.copyFrom(dataBuffer) - } -} - -private[spark] object MesosTaskLaunchData extends Logging { - def fromByteString(byteString: ByteString): MesosTaskLaunchData = { - val byteBuffer = byteString.asReadOnlyByteBuffer() - logDebug(s"ByteBuffer size: [${byteBuffer.remaining}]") - val attemptNumber = byteBuffer.getInt // updates the position by 4 bytes - val serializedTask = byteBuffer.slice() // subsequence starting at the current position - MesosTaskLaunchData(serializedTask, attemptNumber) - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala deleted file mode 100644 index c633d860ae6e..000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ /dev/null @@ -1,165 +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.scheduler.local - -import java.io.File -import java.net.URL -import java.nio.ByteBuffer - -import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv, TaskState} -import org.apache.spark.TaskState.TaskState -import org.apache.spark.executor.{Executor, ExecutorBackend} -import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} -import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} -import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.ExecutorInfo - -private case class ReviveOffers() - -private case class StatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) - -private case class KillTask(taskId: Long, interruptThread: Boolean) - -private case class StopExecutor() - -/** - * Calls to LocalBackend are all serialized through LocalEndpoint. Using an RpcEndpoint makes the - * calls on LocalBackend asynchronous, which is necessary to prevent deadlock between LocalBackend - * and the TaskSchedulerImpl. - */ -private[spark] class LocalEndpoint( - override val rpcEnv: RpcEnv, - userClassPath: Seq[URL], - scheduler: TaskSchedulerImpl, - executorBackend: LocalBackend, - private val totalCores: Int) - extends ThreadSafeRpcEndpoint with Logging { - - private var freeCores = totalCores - - val localExecutorId = SparkContext.DRIVER_IDENTIFIER - val localExecutorHostname = "localhost" - - private val executor = new Executor( - localExecutorId, localExecutorHostname, SparkEnv.get, userClassPath, isLocal = true) - - override def receive: PartialFunction[Any, Unit] = { - case ReviveOffers => - reviveOffers() - - case StatusUpdate(taskId, state, serializedData) => - scheduler.statusUpdate(taskId, state, serializedData) - if (TaskState.isFinished(state)) { - freeCores += scheduler.CPUS_PER_TASK - reviveOffers() - } - - case KillTask(taskId, interruptThread) => - executor.killTask(taskId, interruptThread) - } - - override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case StopExecutor => - executor.stop() - context.reply(true) - } - - def reviveOffers() { - val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) - for (task <- scheduler.resourceOffers(offers).flatten) { - freeCores -= scheduler.CPUS_PER_TASK - executor.launchTask(executorBackend, taskId = task.taskId, attemptNumber = task.attemptNumber, - task.name, task.serializedTask) - } - } -} - -/** - * LocalBackend is used when running a local version of Spark where the executor, backend, and - * master all run in the same JVM. It sits behind a TaskSchedulerImpl and handles launching tasks - * on a single Executor (created by the LocalBackend) running locally. - */ -private[spark] class LocalBackend( - conf: SparkConf, - scheduler: TaskSchedulerImpl, - val totalCores: Int) - extends SchedulerBackend with ExecutorBackend with Logging { - - private val appId = "local-" + System.currentTimeMillis - private var localEndpoint: RpcEndpointRef = null - private val userClassPath = getUserClasspath(conf) - private val listenerBus = scheduler.sc.listenerBus - private val launcherBackend = new LauncherBackend() { - override def onStopRequest(): Unit = stop(SparkAppHandle.State.KILLED) - } - - /** - * Returns a list of URLs representing the user classpath. - * - * @param conf Spark configuration. - */ - def getUserClasspath(conf: SparkConf): Seq[URL] = { - val userClassPathStr = conf.getOption("spark.executor.extraClassPath") - userClassPathStr.map(_.split(File.pathSeparator)).toSeq.flatten.map(new File(_).toURI.toURL) - } - - launcherBackend.connect() - - override def start() { - val rpcEnv = SparkEnv.get.rpcEnv - val executorEndpoint = new LocalEndpoint(rpcEnv, userClassPath, scheduler, this, totalCores) - localEndpoint = rpcEnv.setupEndpoint("LocalBackendEndpoint", executorEndpoint) - listenerBus.post(SparkListenerExecutorAdded( - System.currentTimeMillis, - executorEndpoint.localExecutorId, - new ExecutorInfo(executorEndpoint.localExecutorHostname, totalCores, Map.empty))) - launcherBackend.setAppId(appId) - launcherBackend.setState(SparkAppHandle.State.RUNNING) - } - - override def stop() { - stop(SparkAppHandle.State.FINISHED) - } - - override def reviveOffers() { - localEndpoint.send(ReviveOffers) - } - - override def defaultParallelism(): Int = - scheduler.conf.getInt("spark.default.parallelism", totalCores) - - override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { - localEndpoint.send(KillTask(taskId, interruptThread)) - } - - override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) { - localEndpoint.send(StatusUpdate(taskId, state, serializedData)) - } - - override def applicationId(): String = appId - - private def stop(finalState: SparkAppHandle.State): Unit = { - localEndpoint.ask(StopExecutor) - try { - launcherBackend.setState(finalState) - } finally { - launcherBackend.close() - } - } - -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala new file mode 100644 index 000000000000..35509bc2f85b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.local + +import java.io.File +import java.net.URL +import java.nio.ByteBuffer + +import org.apache.spark.{SparkConf, SparkContext, SparkEnv, TaskState} +import org.apache.spark.TaskState.TaskState +import org.apache.spark.executor.{Executor, ExecutorBackend} +import org.apache.spark.internal.Logging +import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} +import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} +import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.ExecutorInfo + +private case class ReviveOffers() + +private case class StatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) + +private case class KillTask(taskId: Long, interruptThread: Boolean, reason: String) + +private case class StopExecutor() + +/** + * Calls to [[LocalSchedulerBackend]] are all serialized through LocalEndpoint. Using an + * RpcEndpoint makes the calls on [[LocalSchedulerBackend]] asynchronous, which is necessary + * to prevent deadlock between [[LocalSchedulerBackend]] and the [[TaskSchedulerImpl]]. + */ +private[spark] class LocalEndpoint( + override val rpcEnv: RpcEnv, + userClassPath: Seq[URL], + scheduler: TaskSchedulerImpl, + executorBackend: LocalSchedulerBackend, + private val totalCores: Int) + extends ThreadSafeRpcEndpoint with Logging { + + private var freeCores = totalCores + + val localExecutorId = SparkContext.DRIVER_IDENTIFIER + val localExecutorHostname = "localhost" + + private val executor = new Executor( + localExecutorId, localExecutorHostname, SparkEnv.get, userClassPath, isLocal = true) + + override def receive: PartialFunction[Any, Unit] = { + case ReviveOffers => + reviveOffers() + + case StatusUpdate(taskId, state, serializedData) => + scheduler.statusUpdate(taskId, state, serializedData) + if (TaskState.isFinished(state)) { + freeCores += scheduler.CPUS_PER_TASK + reviveOffers() + } + + case KillTask(taskId, interruptThread, reason) => + executor.killTask(taskId, interruptThread, reason) + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case StopExecutor => + executor.stop() + context.reply(true) + } + + def reviveOffers() { + val offers = IndexedSeq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) + for (task <- scheduler.resourceOffers(offers).flatten) { + freeCores -= scheduler.CPUS_PER_TASK + executor.launchTask(executorBackend, task) + } + } +} + +/** + * Used when running a local version of Spark where the executor, backend, and master all run in + * the same JVM. It sits behind a [[TaskSchedulerImpl]] and handles launching tasks on a single + * Executor (created by the [[LocalSchedulerBackend]]) running locally. + */ +private[spark] class LocalSchedulerBackend( + conf: SparkConf, + scheduler: TaskSchedulerImpl, + val totalCores: Int) + extends SchedulerBackend with ExecutorBackend with Logging { + + private val appId = "local-" + System.currentTimeMillis + private var localEndpoint: RpcEndpointRef = null + private val userClassPath = getUserClasspath(conf) + private val listenerBus = scheduler.sc.listenerBus + private val launcherBackend = new LauncherBackend() { + override def onStopRequest(): Unit = stop(SparkAppHandle.State.KILLED) + } + + /** + * Returns a list of URLs representing the user classpath. + * + * @param conf Spark configuration. + */ + def getUserClasspath(conf: SparkConf): Seq[URL] = { + val userClassPathStr = conf.getOption("spark.executor.extraClassPath") + userClassPathStr.map(_.split(File.pathSeparator)).toSeq.flatten.map(new File(_).toURI.toURL) + } + + launcherBackend.connect() + + override def start() { + val rpcEnv = SparkEnv.get.rpcEnv + val executorEndpoint = new LocalEndpoint(rpcEnv, userClassPath, scheduler, this, totalCores) + localEndpoint = rpcEnv.setupEndpoint("LocalSchedulerBackendEndpoint", executorEndpoint) + listenerBus.post(SparkListenerExecutorAdded( + System.currentTimeMillis, + executorEndpoint.localExecutorId, + new ExecutorInfo(executorEndpoint.localExecutorHostname, totalCores, Map.empty))) + launcherBackend.setAppId(appId) + launcherBackend.setState(SparkAppHandle.State.RUNNING) + } + + override def stop() { + stop(SparkAppHandle.State.FINISHED) + } + + override def reviveOffers() { + localEndpoint.send(ReviveOffers) + } + + override def defaultParallelism(): Int = + scheduler.conf.getInt("spark.default.parallelism", totalCores) + + override def killTask( + taskId: Long, executorId: String, interruptThread: Boolean, reason: String) { + localEndpoint.send(KillTask(taskId, interruptThread, reason)) + } + + override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) { + localEndpoint.send(StatusUpdate(taskId, state, serializedData)) + } + + override def applicationId(): String = appId + + private def stop(finalState: SparkAppHandle.State): Unit = { + localEndpoint.ask(StopExecutor) + try { + launcherBackend.setState(finalState) + } finally { + launcherBackend.close() + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/package-info.java b/core/src/main/scala/org/apache/spark/scheduler/package-info.java index 5b4a628d3cee..90fc65251eae 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/package-info.java +++ b/core/src/main/scala/org/apache/spark/scheduler/package-info.java @@ -18,4 +18,4 @@ /** * Spark's DAG scheduler. */ -package org.apache.spark.scheduler; \ No newline at end of file +package org.apache.spark.scheduler; diff --git a/core/src/main/scala/org/apache/spark/scheduler/package.scala b/core/src/main/scala/org/apache/spark/scheduler/package.scala index f0dbfc2ac5f4..4847c41710b2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/package.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/package.scala @@ -18,7 +18,7 @@ package org.apache.spark /** - * Spark's scheduling components. This includes the [[org.apache.spark.scheduler.DAGScheduler]] and - * lower level [[org.apache.spark.scheduler.TaskScheduler]]. + * Spark's scheduling components. This includes the `org.apache.spark.scheduler.DAGScheduler` and + * lower level `org.apache.spark.scheduler.TaskScheduler`. */ package object scheduler diff --git a/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala b/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala new file mode 100644 index 000000000000..00621976b77f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.security + +import java.io.{InputStream, OutputStream} +import java.nio.ByteBuffer +import java.nio.channels.{ReadableByteChannel, WritableByteChannel} +import java.util.Properties +import javax.crypto.KeyGenerator +import javax.crypto.spec.{IvParameterSpec, SecretKeySpec} + +import scala.collection.JavaConverters._ + +import com.google.common.io.ByteStreams +import org.apache.commons.crypto.random._ +import org.apache.commons.crypto.stream._ + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.network.util.{CryptoUtils, JavaUtils} + +/** + * A util class for manipulating IO encryption and decryption streams. + */ +private[spark] object CryptoStreamUtils extends Logging { + + // The initialization vector length in bytes. + val IV_LENGTH_IN_BYTES = 16 + // The prefix of IO encryption related configurations in Spark configuration. + val SPARK_IO_ENCRYPTION_COMMONS_CONFIG_PREFIX = "spark.io.encryption.commons.config." + + /** + * Helper method to wrap `OutputStream` with `CryptoOutputStream` for encryption. + */ + def createCryptoOutputStream( + os: OutputStream, + sparkConf: SparkConf, + key: Array[Byte]): OutputStream = { + val params = new CryptoParams(key, sparkConf) + val iv = createInitializationVector(params.conf) + os.write(iv) + new CryptoOutputStream(params.transformation, params.conf, os, params.keySpec, + new IvParameterSpec(iv)) + } + + /** + * Wrap a `WritableByteChannel` for encryption. + */ + def createWritableChannel( + channel: WritableByteChannel, + sparkConf: SparkConf, + key: Array[Byte]): WritableByteChannel = { + val params = new CryptoParams(key, sparkConf) + val iv = createInitializationVector(params.conf) + val helper = new CryptoHelperChannel(channel) + + helper.write(ByteBuffer.wrap(iv)) + new CryptoOutputStream(params.transformation, params.conf, helper, params.keySpec, + new IvParameterSpec(iv)) + } + + /** + * Helper method to wrap `InputStream` with `CryptoInputStream` for decryption. + */ + def createCryptoInputStream( + is: InputStream, + sparkConf: SparkConf, + key: Array[Byte]): InputStream = { + val iv = new Array[Byte](IV_LENGTH_IN_BYTES) + ByteStreams.readFully(is, iv) + val params = new CryptoParams(key, sparkConf) + new CryptoInputStream(params.transformation, params.conf, is, params.keySpec, + new IvParameterSpec(iv)) + } + + /** + * Wrap a `ReadableByteChannel` for decryption. + */ + def createReadableChannel( + channel: ReadableByteChannel, + sparkConf: SparkConf, + key: Array[Byte]): ReadableByteChannel = { + val iv = new Array[Byte](IV_LENGTH_IN_BYTES) + val buf = ByteBuffer.wrap(iv) + JavaUtils.readFully(channel, buf) + + val params = new CryptoParams(key, sparkConf) + new CryptoInputStream(params.transformation, params.conf, channel, params.keySpec, + new IvParameterSpec(iv)) + } + + def toCryptoConf(conf: SparkConf): Properties = { + CryptoUtils.toCryptoConf(SPARK_IO_ENCRYPTION_COMMONS_CONFIG_PREFIX, + conf.getAll.toMap.asJava.entrySet()) + } + + /** + * Creates a new encryption key. + */ + def createKey(conf: SparkConf): Array[Byte] = { + val keyLen = conf.get(IO_ENCRYPTION_KEY_SIZE_BITS) + val ioKeyGenAlgorithm = conf.get(IO_ENCRYPTION_KEYGEN_ALGORITHM) + val keyGen = KeyGenerator.getInstance(ioKeyGenAlgorithm) + keyGen.init(keyLen) + keyGen.generateKey().getEncoded() + } + + /** + * This method to generate an IV (Initialization Vector) using secure random. + */ + private[this] def createInitializationVector(properties: Properties): Array[Byte] = { + val iv = new Array[Byte](IV_LENGTH_IN_BYTES) + val initialIVStart = System.currentTimeMillis() + CryptoRandomFactory.getCryptoRandom(properties).nextBytes(iv) + val initialIVFinish = System.currentTimeMillis() + val initialIVTime = initialIVFinish - initialIVStart + if (initialIVTime > 2000) { + logWarning(s"It costs ${initialIVTime} milliseconds to create the Initialization Vector " + + s"used by CryptoStream") + } + iv + } + + /** + * This class is a workaround for CRYPTO-125, that forces all bytes to be written to the + * underlying channel. Since the callers of this API are using blocking I/O, there are no + * concerns with regards to CPU usage here. + */ + private class CryptoHelperChannel(sink: WritableByteChannel) extends WritableByteChannel { + + override def write(src: ByteBuffer): Int = { + val count = src.remaining() + while (src.hasRemaining()) { + sink.write(src) + } + count + } + + override def isOpen(): Boolean = sink.isOpen() + + override def close(): Unit = sink.close() + + } + + private class CryptoParams(key: Array[Byte], sparkConf: SparkConf) { + + val keySpec = new SecretKeySpec(key, "AES") + val transformation = sparkConf.get(IO_CRYPTO_CIPHER_TRANSFORMATION) + val conf = toCryptoConf(sparkConf) + + } + +} diff --git a/core/src/main/scala/org/apache/spark/security/GroupMappingServiceProvider.scala b/core/src/main/scala/org/apache/spark/security/GroupMappingServiceProvider.scala new file mode 100644 index 000000000000..ea047a4f75d5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/security/GroupMappingServiceProvider.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.security + +/** + * This Spark trait is used for mapping a given userName to a set of groups which it belongs to. + * This is useful for specifying a common group of admins/developers to provide them admin, modify + * and/or view access rights. Based on whether access control checks are enabled using + * spark.acls.enable, every time a user tries to access or modify the application, the + * SecurityManager gets the corresponding groups a user belongs to from the instance of the groups + * mapping provider specified by the entry spark.user.groups.mapping. + */ + +trait GroupMappingServiceProvider { + + /** + * Get the groups the user belongs to. + * @param userName User's Name + * @return set of groups that the user belongs to. Empty in case of an invalid user. + */ + def getGroups(userName : String) : Set[String] + +} diff --git a/core/src/main/scala/org/apache/spark/security/ShellBasedGroupsMappingProvider.scala b/core/src/main/scala/org/apache/spark/security/ShellBasedGroupsMappingProvider.scala new file mode 100644 index 000000000000..f71dd08246b2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/security/ShellBasedGroupsMappingProvider.scala @@ -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.security + +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +/** + * This class is responsible for getting the groups for a particular user in Unix based + * environments. This implementation uses the Unix Shell based id command to fetch the user groups + * for the specified user. It does not cache the user groups as the invocations are expected + * to be infrequent. + */ + +private[spark] class ShellBasedGroupsMappingProvider extends GroupMappingServiceProvider + with Logging { + + override def getGroups(username: String): Set[String] = { + val userGroups = getUnixGroups(username) + logDebug("User: " + username + " Groups: " + userGroups.mkString(",")) + userGroups + } + + // shells out a "bash -c id -Gn username" to get user groups + private def getUnixGroups(username: String): Set[String] = { + val cmdSeq = Seq("bash", "-c", "id -Gn " + username) + // we need to get rid of the trailing "\n" from the result of command execution + Utils.executeAndGetOutput(cmdSeq).stripLineEnd.split(" ").toSet + } +} diff --git a/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala index 62f8aae7f212..f0ed41f6903f 100644 --- a/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala @@ -19,6 +19,7 @@ package org.apache.spark.serializer import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets import scala.collection.mutable @@ -29,8 +30,9 @@ import org.apache.avro.generic.{GenericData, GenericRecord} import org.apache.avro.io._ import org.apache.commons.io.IOUtils -import org.apache.spark.{SparkException, SparkEnv} +import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.Utils /** * Custom serializer used for generic Avro records. If the user registers the schemas @@ -71,8 +73,11 @@ private[serializer] class GenericAvroSerializer(schemas: Map[Long, String]) def compress(schema: Schema): Array[Byte] = compressCache.getOrElseUpdate(schema, { val bos = new ByteArrayOutputStream() val out = codec.compressedOutputStream(bos) - out.write(schema.toString.getBytes("UTF-8")) - out.close() + Utils.tryWithSafeFinally { + out.write(schema.toString.getBytes(StandardCharsets.UTF_8)) + } { + out.close() + } bos.toByteArray }) @@ -81,9 +86,17 @@ private[serializer] class GenericAvroSerializer(schemas: Map[Long, String]) * seen values so to limit the number of times that decompression has to be done. */ def decompress(schemaBytes: ByteBuffer): Schema = decompressCache.getOrElseUpdate(schemaBytes, { - val bis = new ByteArrayInputStream(schemaBytes.array()) - val bytes = IOUtils.toByteArray(codec.compressedInputStream(bis)) - new Schema.Parser().parse(new String(bytes, "UTF-8")) + val bis = new ByteArrayInputStream( + schemaBytes.array(), + schemaBytes.arrayOffset() + schemaBytes.position(), + schemaBytes.remaining()) + val in = codec.compressedInputStream(bis) + val bytes = Utils.tryWithSafeFinally { + IOUtils.toByteArray(in) + } { + in.close() + } + new Schema.Parser().parse(new String(bytes, StandardCharsets.UTF_8)) }) /** diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index b463a71d5bd7..f60dcfddfdc2 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -24,8 +24,7 @@ import scala.reflect.ClassTag import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.util.ByteBufferInputStream -import org.apache.spark.util.Utils +import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Utils} private[spark] class JavaSerializationStream( out: OutputStream, counterReset: Int, extraDebugInfo: Boolean) @@ -69,7 +68,7 @@ private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoa // scalastyle:on classforname } catch { case e: ClassNotFoundException => - JavaDeserializationStream.primitiveMappings.get(desc.getName).getOrElse(throw e) + JavaDeserializationStream.primitiveMappings.getOrElse(desc.getName, throw e) } } @@ -96,11 +95,11 @@ private[spark] class JavaSerializerInstance( extends SerializerInstance { override def serialize[T: ClassTag](t: T): ByteBuffer = { - val bos = new ByteArrayOutputStream() + val bos = new ByteBufferOutputStream() val out = serializeStream(bos) out.writeObject(t) out.close() - ByteBuffer.wrap(bos.toByteArray) + bos.toByteBuffer } override def deserialize[T: ClassTag](bytes: ByteBuffer): T = { @@ -132,7 +131,7 @@ private[spark] class JavaSerializerInstance( * :: DeveloperApi :: * A Spark serializer that uses Java's built-in serialization. * - * Note that this serializer is not guaranteed to be wire-compatible across different versions of + * @note This serializer is not guaranteed to be wire-compatible across different versions of * Spark. It is intended to be used to serialize/de-serialize data within a single * Spark application. */ diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index bc51d4f2820c..58483c9577d2 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -17,33 +17,37 @@ package org.apache.spark.serializer -import java.io.{EOFException, IOException, InputStream, OutputStream} +import java.io._ import java.nio.ByteBuffer +import java.util.Locale import javax.annotation.Nullable import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import com.esotericsoftware.kryo.{Kryo, KryoException} +import com.esotericsoftware.kryo.{Kryo, KryoException, Serializer => KryoClassSerializer} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} +import com.esotericsoftware.kryo.io.{UnsafeInput => KryoUnsafeInput, UnsafeOutput => KryoUnsafeOutput} import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} import org.apache.avro.generic.{GenericData, GenericRecord} +import org.roaringbitmap.RoaringBitmap import org.apache.spark._ import org.apache.spark.api.python.PythonBroadcast -import org.apache.spark.broadcast.HttpBroadcast +import org.apache.spark.internal.Logging import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus} import org.apache.spark.storage._ -import org.apache.spark.util.{Utils, BoundedPriorityQueue, SerializableConfiguration, SerializableJobConf} -import org.apache.spark.util.collection.{BitSet, CompactBuffer} +import org.apache.spark.util.{BoundedPriorityQueue, SerializableConfiguration, SerializableJobConf, Utils} +import org.apache.spark.util.collection.CompactBuffer /** - * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. + * A Spark serializer that uses the + * Kryo serialization library. * - * Note that this serializer is not guaranteed to be wire-compatible across different versions of + * @note This serializer is not guaranteed to be wire-compatible across different versions of * Spark. It is intended to be used to serialize/de-serialize data within a single * Spark application. */ @@ -69,14 +73,23 @@ class KryoSerializer(conf: SparkConf) private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) private val registrationRequired = conf.getBoolean("spark.kryo.registrationRequired", false) - private val userRegistrator = conf.getOption("spark.kryo.registrator") + private val userRegistrators = conf.get("spark.kryo.registrator", "") + .split(',').map(_.trim) + .filter(!_.isEmpty) private val classesToRegister = conf.get("spark.kryo.classesToRegister", "") - .split(',') + .split(',').map(_.trim) .filter(!_.isEmpty) private val avroSchemas = conf.getAvroSchema + // whether to use unsafe based IO for serialization + private val useUnsafe = conf.getBoolean("spark.kryo.unsafe", false) - def newKryoOutput(): KryoOutput = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) + def newKryoOutput(): KryoOutput = + if (useUnsafe) { + new KryoUnsafeOutput(bufferSize, math.max(bufferSize, maxBufferSize)) + } else { + new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) + } def newKryo(): Kryo = { val instantiator = new EmptyScalaKryoInstantiator @@ -93,6 +106,9 @@ class KryoSerializer(conf: SparkConf) for (cls <- KryoSerializer.toRegister) { kryo.register(cls) } + for ((cls, ser) <- KryoSerializer.toRegisterSerializer) { + kryo.register(cls, ser) + } // For results returned by asJavaIterable. See JavaIterableWrapperSerializer. kryo.register(JavaIterableWrapperSerializer.wrapperClass, new JavaIterableWrapperSerializer) @@ -101,7 +117,6 @@ class KryoSerializer(conf: SparkConf) kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) kryo.register(classOf[SerializableConfiguration], new KryoJavaSerializer()) kryo.register(classOf[SerializableJobConf], new KryoJavaSerializer()) - kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) kryo.register(classOf[PythonBroadcast], new KryoJavaSerializer()) kryo.register(classOf[GenericRecord], new GenericAvroSerializer(avroSchemas)) @@ -115,7 +130,7 @@ class KryoSerializer(conf: SparkConf) classesToRegister .foreach { className => kryo.register(Class.forName(className, true, classLoader)) } // Allow the user to register their own classes by setting spark.kryo.registrator. - userRegistrator + userRegistrators .map(Class.forName(_, true, classLoader).newInstance().asInstanceOf[KryoRegistrator]) .foreach { reg => reg.registerClasses(kryo) } // scalastyle:on classforname @@ -160,6 +175,7 @@ class KryoSerializer(conf: SparkConf) kryo.register(None.getClass) kryo.register(Nil.getClass) kryo.register(Utils.classForName("scala.collection.immutable.$colon$colon")) + kryo.register(Utils.classForName("scala.collection.immutable.Map$EmptyMap$")) kryo.register(classOf[ArrayBuffer[Any]]) kryo.setClassLoader(classLoader) @@ -167,7 +183,7 @@ class KryoSerializer(conf: SparkConf) } override def newInstance(): SerializerInstance = { - new KryoSerializerInstance(this) + new KryoSerializerInstance(this, useUnsafe) } private[spark] override lazy val supportsRelocationOfSerializedObjects: Boolean = { @@ -181,9 +197,12 @@ class KryoSerializer(conf: SparkConf) private[spark] class KryoSerializationStream( serInstance: KryoSerializerInstance, - outStream: OutputStream) extends SerializationStream { + outStream: OutputStream, + useUnsafe: Boolean) extends SerializationStream { + + private[this] var output: KryoOutput = + if (useUnsafe) new KryoUnsafeOutput(outStream) else new KryoOutput(outStream) - private[this] var output: KryoOutput = new KryoOutput(outStream) private[this] var kryo: Kryo = serInstance.borrowKryo() override def writeObject[T: ClassTag](t: T): SerializationStream = { @@ -214,9 +233,12 @@ class KryoSerializationStream( private[spark] class KryoDeserializationStream( serInstance: KryoSerializerInstance, - inStream: InputStream) extends DeserializationStream { + inStream: InputStream, + useUnsafe: Boolean) extends DeserializationStream { + + private[this] var input: KryoInput = + if (useUnsafe) new KryoUnsafeInput(inStream) else new KryoInput(inStream) - private[this] var input: KryoInput = new KryoInput(inStream) private[this] var kryo: Kryo = serInstance.borrowKryo() override def readObject[T: ClassTag](): T = { @@ -224,7 +246,8 @@ class KryoDeserializationStream( kryo.readClassAndObject(input).asInstanceOf[T] } catch { // DeserializationStream uses the EOF exception to indicate stopping condition. - case e: KryoException if e.getMessage.toLowerCase.contains("buffer underflow") => + case e: KryoException + if e.getMessage.toLowerCase(Locale.ROOT).contains("buffer underflow") => throw new EOFException } } @@ -243,8 +266,8 @@ class KryoDeserializationStream( } } -private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { - +private[spark] class KryoSerializerInstance(ks: KryoSerializer, useUnsafe: Boolean) + extends SerializerInstance { /** * A re-used [[Kryo]] instance. Methods will borrow this instance by calling `borrowKryo()`, do * their work, then release the instance by calling `releaseKryo()`. Logically, this is a caching @@ -283,7 +306,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ // Make these lazy vals to avoid creating a buffer unless we use them. private lazy val output = ks.newKryoOutput() - private lazy val input = new KryoInput() + private lazy val input = if (useUnsafe) new KryoUnsafeInput() else new KryoInput() override def serialize[T: ClassTag](t: T): ByteBuffer = { output.clear() @@ -293,7 +316,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ } catch { case e: KryoException if e.getMessage.startsWith("Buffer overflow") => throw new SparkException(s"Kryo serialization failed: ${e.getMessage}. To avoid this, " + - "increase spark.kryoserializer.buffer.max value.") + "increase spark.kryoserializer.buffer.max value.", e) } finally { releaseKryo(kryo) } @@ -303,7 +326,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ override def deserialize[T: ClassTag](bytes: ByteBuffer): T = { val kryo = borrowKryo() try { - input.setBuffer(bytes.array) + input.setBuffer(bytes.array(), bytes.arrayOffset() + bytes.position(), bytes.remaining()) kryo.readClassAndObject(input).asInstanceOf[T] } finally { releaseKryo(kryo) @@ -315,7 +338,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ val oldClassLoader = kryo.getClassLoader try { kryo.setClassLoader(loader) - input.setBuffer(bytes.array) + input.setBuffer(bytes.array(), bytes.arrayOffset() + bytes.position(), bytes.remaining()) kryo.readClassAndObject(input).asInstanceOf[T] } finally { kryo.setClassLoader(oldClassLoader) @@ -324,11 +347,11 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ } override def serializeStream(s: OutputStream): SerializationStream = { - new KryoSerializationStream(this, s) + new KryoSerializationStream(this, s, useUnsafe) } override def deserializeStream(s: InputStream): DeserializationStream = { - new KryoDeserializationStream(this, s) + new KryoDeserializationStream(this, s, useUnsafe) } /** @@ -352,7 +375,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ * serialization. */ trait KryoRegistrator { - def registerClasses(kryo: Kryo) + def registerClasses(kryo: Kryo): Unit } private[serializer] object KryoSerializer { @@ -362,15 +385,87 @@ private[serializer] object KryoSerializer { classOf[StorageLevel], classOf[CompressedMapStatus], classOf[HighlyCompressedMapStatus], - classOf[BitSet], classOf[CompactBuffer[_]], classOf[BlockManagerId], + classOf[Array[Boolean]], classOf[Array[Byte]], classOf[Array[Short]], + classOf[Array[Int]], classOf[Array[Long]], + classOf[Array[Float]], + classOf[Array[Double]], + classOf[Array[Char]], + classOf[Array[String]], + classOf[Array[Array[String]]], classOf[BoundedPriorityQueue[_]], classOf[SparkConf] ) + + private val toRegisterSerializer = Map[Class[_], KryoClassSerializer[_]]( + classOf[RoaringBitmap] -> new KryoClassSerializer[RoaringBitmap]() { + override def write(kryo: Kryo, output: KryoOutput, bitmap: RoaringBitmap): Unit = { + bitmap.serialize(new KryoOutputObjectOutputBridge(kryo, output)) + } + override def read(kryo: Kryo, input: KryoInput, cls: Class[RoaringBitmap]): RoaringBitmap = { + val ret = new RoaringBitmap + ret.deserialize(new KryoInputObjectInputBridge(kryo, input)) + ret + } + } + ) +} + +/** + * This is a bridge class to wrap KryoInput as an InputStream and ObjectInput. It forwards all + * methods of InputStream and ObjectInput to KryoInput. It's usually helpful when an API expects + * an InputStream or ObjectInput but you want to use Kryo. + */ +private[spark] class KryoInputObjectInputBridge( + kryo: Kryo, input: KryoInput) extends FilterInputStream(input) with ObjectInput { + override def readLong(): Long = input.readLong() + override def readChar(): Char = input.readChar() + override def readFloat(): Float = input.readFloat() + override def readByte(): Byte = input.readByte() + override def readShort(): Short = input.readShort() + override def readUTF(): String = input.readString() // readString in kryo does utf8 + override def readInt(): Int = input.readInt() + override def readUnsignedShort(): Int = input.readShortUnsigned() + override def skipBytes(n: Int): Int = { + input.skip(n) + n + } + override def readFully(b: Array[Byte]): Unit = input.read(b) + override def readFully(b: Array[Byte], off: Int, len: Int): Unit = input.read(b, off, len) + override def readLine(): String = throw new UnsupportedOperationException("readLine") + override def readBoolean(): Boolean = input.readBoolean() + override def readUnsignedByte(): Int = input.readByteUnsigned() + override def readDouble(): Double = input.readDouble() + override def readObject(): AnyRef = kryo.readClassAndObject(input) +} + +/** + * This is a bridge class to wrap KryoOutput as an OutputStream and ObjectOutput. It forwards all + * methods of OutputStream and ObjectOutput to KryoOutput. It's usually helpful when an API expects + * an OutputStream or ObjectOutput but you want to use Kryo. + */ +private[spark] class KryoOutputObjectOutputBridge( + kryo: Kryo, output: KryoOutput) extends FilterOutputStream(output) with ObjectOutput { + override def writeFloat(v: Float): Unit = output.writeFloat(v) + // There is no "readChars" counterpart, except maybe "readLine", which is not supported + override def writeChars(s: String): Unit = throw new UnsupportedOperationException("writeChars") + override def writeDouble(v: Double): Unit = output.writeDouble(v) + override def writeUTF(s: String): Unit = output.writeString(s) // writeString in kryo does UTF8 + override def writeShort(v: Int): Unit = output.writeShort(v) + override def writeInt(v: Int): Unit = output.writeInt(v) + override def writeBoolean(v: Boolean): Unit = output.writeBoolean(v) + override def write(b: Int): Unit = output.write(b) + override def write(b: Array[Byte]): Unit = output.write(b) + override def write(b: Array[Byte], off: Int, len: Int): Unit = output.write(b, off, len) + override def writeBytes(s: String): Unit = output.writeString(s) + override def writeChar(v: Int): Unit = output.writeChar(v.toChar) + override def writeLong(v: Long): Unit = output.writeLong(v) + override def writeByte(v: Int): Unit = output.writeByte(v) + override def writeObject(obj: AnyRef): Unit = kryo.writeClassAndObject(output, obj) } /** @@ -406,8 +501,8 @@ private class JavaIterableWrapperSerializer private object JavaIterableWrapperSerializer extends Logging { // The class returned by JavaConverters.asJava // (scala.collection.convert.Wrappers$IterableWrapper). - val wrapperClass = - scala.collection.convert.WrapAsJava.asJavaIterable(Seq(1)).getClass + import scala.collection.JavaConverters._ + val wrapperClass = Seq(1).asJava.getClass // Get the underlying method so we can use it to get the Scala collection for serialization. private val underlyingMethodOpt = { diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala index a1b1e1631eaf..5e7a98c8aa89 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala @@ -25,7 +25,7 @@ import scala.annotation.tailrec import scala.collection.mutable import scala.util.control.NonFatal -import org.apache.spark.Logging +import org.apache.spark.internal.Logging private[spark] object SerializationDebugger extends Logging { @@ -53,12 +53,13 @@ private[spark] object SerializationDebugger extends Logging { /** * Find the path leading to a not serializable object. This method is modeled after OpenJDK's * serialization mechanism, and handles the following cases: - * - primitives - * - arrays of primitives - * - arrays of non-primitive objects - * - Serializable objects - * - Externalizable objects - * - writeReplace + * + * - primitives + * - arrays of primitives + * - arrays of non-primitive objects + * - Serializable objects + * - Externalizable objects + * - writeReplace * * It does not yet handle writeObject override, but that shouldn't be too hard to do either. */ @@ -154,7 +155,7 @@ private[spark] object SerializationDebugger extends Logging { // If the object has been replaced using writeReplace(), // then call visit() on it again to test its type again. - if (!finalObj.eq(o)) { + if (finalObj.getClass != o.getClass) { return visit(finalObj, s"writeReplace data (class: ${finalObj.getClass.getName})" :: stack) } @@ -264,8 +265,13 @@ private[spark] object SerializationDebugger extends Logging { if (!desc.hasWriteReplaceMethod) { (o, desc) } else { - // write place - findObjectAndDescriptor(desc.invokeWriteReplace(o)) + val replaced = desc.invokeWriteReplace(o) + // `writeReplace` recursion stops when the returned object has the same class. + if (replaced.getClass == o.getClass) { + (replaced, desc) + } else { + findObjectAndDescriptor(replaced) + } } } diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index bd2704dc8187..cb8b1cc07763 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -23,9 +23,8 @@ import javax.annotation.concurrent.NotThreadSafe import scala.reflect.ClassTag -import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.annotation.{DeveloperApi, Private} -import org.apache.spark.util.{Utils, ByteBufferInputStream, NextIterator} +import org.apache.spark.util.NextIterator /** * :: DeveloperApi :: @@ -40,7 +39,7 @@ import org.apache.spark.util.{Utils, ByteBufferInputStream, NextIterator} * * 2. Java serialization interface. * - * Note that serializers are not required to be wire-compatible across different versions of Spark. + * @note Serializers are not required to be wire-compatible across different versions of Spark. * They are intended to be used to serialize/de-serialize data within a single Spark application. */ @DeveloperApi @@ -78,7 +77,7 @@ abstract class Serializer { * position = 0 * serOut.write(obj1) * serOut.flush() - * position = # of bytes writen to stream so far + * position = # of bytes written to stream so far * obj1Bytes = output[0:position-1] * serOut.write(obj2) * serOut.flush() @@ -100,18 +99,6 @@ abstract class Serializer { } -@DeveloperApi -object Serializer { - def getSerializer(serializer: Serializer): Serializer = { - if (serializer == null) SparkEnv.get.serializer else serializer - } - - def getSerializer(serializer: Option[Serializer]): Serializer = { - serializer.getOrElse(SparkEnv.get.serializer) - } -} - - /** * :: DeveloperApi :: * An instance of a serializer, for use by one thread at a time. @@ -138,7 +125,7 @@ abstract class SerializerInstance { * A stream for writing serialized objects. */ @DeveloperApi -abstract class SerializationStream { +abstract class SerializationStream extends Closeable { /** The most general-purpose method to write an object. */ def writeObject[T: ClassTag](t: T): SerializationStream /** Writes the object representing the key of a key-value pair. */ @@ -146,7 +133,7 @@ abstract class SerializationStream { /** Writes the object representing the value of a key-value pair. */ def writeValue[T: ClassTag](value: T): SerializationStream = writeObject(value) def flush(): Unit - def close(): Unit + override def close(): Unit def writeAll[T: ClassTag](iter: Iterator[T]): SerializationStream = { while (iter.hasNext) { @@ -162,14 +149,14 @@ abstract class SerializationStream { * A stream for reading serialized objects. */ @DeveloperApi -abstract class DeserializationStream { +abstract class DeserializationStream extends Closeable { /** The most general-purpose method to read an object. */ def readObject[T: ClassTag](): T /** Reads the object representing the key of a key-value pair. */ def readKey[T: ClassTag](): T = readObject[T]() /** Reads the object representing the value of a key-value pair. */ def readValue[T: ClassTag](): T = readObject[T]() - def close(): Unit + override def close(): Unit /** * Read the elements of this stream through an iterator. This can only be called once, as @@ -200,10 +187,9 @@ abstract class DeserializationStream { try { (readKey[Any](), readValue[Any]()) } catch { - case eof: EOFException => { + case eof: EOFException => finished = true null - } } } diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala new file mode 100644 index 000000000000..311383e7ea2b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.serializer + +import java.io.{BufferedInputStream, BufferedOutputStream, InputStream, OutputStream} +import java.nio.ByteBuffer + +import scala.reflect.ClassTag + +import org.apache.spark.SparkConf +import org.apache.spark.io.CompressionCodec +import org.apache.spark.security.CryptoStreamUtils +import org.apache.spark.storage._ +import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} + +/** + * Component which configures serialization, compression and encryption for various Spark + * components, including automatic selection of which [[Serializer]] to use for shuffles. + */ +private[spark] class SerializerManager( + defaultSerializer: Serializer, + conf: SparkConf, + encryptionKey: Option[Array[Byte]]) { + + def this(defaultSerializer: Serializer, conf: SparkConf) = this(defaultSerializer, conf, None) + + private[this] val kryoSerializer = new KryoSerializer(conf) + + def setDefaultClassLoader(classLoader: ClassLoader): Unit = { + kryoSerializer.setDefaultClassLoader(classLoader) + } + + private[this] val stringClassTag: ClassTag[String] = implicitly[ClassTag[String]] + private[this] val primitiveAndPrimitiveArrayClassTags: Set[ClassTag[_]] = { + val primitiveClassTags = Set[ClassTag[_]]( + ClassTag.Boolean, + ClassTag.Byte, + ClassTag.Char, + ClassTag.Double, + ClassTag.Float, + ClassTag.Int, + ClassTag.Long, + ClassTag.Null, + ClassTag.Short + ) + val arrayClassTags = primitiveClassTags.map(_.wrap) + primitiveClassTags ++ arrayClassTags + } + + // Whether to compress broadcast variables that are stored + private[this] val compressBroadcast = conf.getBoolean("spark.broadcast.compress", true) + // Whether to compress shuffle output that are stored + private[this] val compressShuffle = conf.getBoolean("spark.shuffle.compress", true) + // Whether to compress RDD partitions that are stored serialized + private[this] val compressRdds = conf.getBoolean("spark.rdd.compress", false) + // Whether to compress shuffle output temporarily spilled to disk + private[this] val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", true) + + /* The compression codec to use. Note that the "lazy" val is necessary because we want to delay + * the initialization of the compression codec until it is first used. The reason is that a Spark + * program could be using a user-defined codec in a third party jar, which is loaded in + * Executor.updateDependencies. When the BlockManager is initialized, user level jars hasn't been + * loaded yet. */ + private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec(conf) + + def encryptionEnabled: Boolean = encryptionKey.isDefined + + def canUseKryo(ct: ClassTag[_]): Boolean = { + primitiveAndPrimitiveArrayClassTags.contains(ct) || ct == stringClassTag + } + + // SPARK-18617: As feature in SPARK-13990 can not be applied to Spark Streaming now. The worst + // result is streaming job based on `Receiver` mode can not run on Spark 2.x properly. It may be + // a rational choice to close `kryo auto pick` feature for streaming in the first step. + def getSerializer(ct: ClassTag[_], autoPick: Boolean): Serializer = { + if (autoPick && canUseKryo(ct)) { + kryoSerializer + } else { + defaultSerializer + } + } + + /** + * Pick the best serializer for shuffling an RDD of key-value pairs. + */ + def getSerializer(keyClassTag: ClassTag[_], valueClassTag: ClassTag[_]): Serializer = { + if (canUseKryo(keyClassTag) && canUseKryo(valueClassTag)) { + kryoSerializer + } else { + defaultSerializer + } + } + + private def shouldCompress(blockId: BlockId): Boolean = { + blockId match { + case _: ShuffleBlockId => compressShuffle + case _: BroadcastBlockId => compressBroadcast + case _: RDDBlockId => compressRdds + case _: TempLocalBlockId => compressShuffleSpill + case _: TempShuffleBlockId => compressShuffle + case _ => false + } + } + + /** + * Wrap an input stream for encryption and compression + */ + def wrapStream(blockId: BlockId, s: InputStream): InputStream = { + wrapForCompression(blockId, wrapForEncryption(s)) + } + + /** + * Wrap an output stream for encryption and compression + */ + def wrapStream(blockId: BlockId, s: OutputStream): OutputStream = { + wrapForCompression(blockId, wrapForEncryption(s)) + } + + /** + * Wrap an input stream for encryption if shuffle encryption is enabled + */ + def wrapForEncryption(s: InputStream): InputStream = { + encryptionKey + .map { key => CryptoStreamUtils.createCryptoInputStream(s, conf, key) } + .getOrElse(s) + } + + /** + * Wrap an output stream for encryption if shuffle encryption is enabled + */ + def wrapForEncryption(s: OutputStream): OutputStream = { + encryptionKey + .map { key => CryptoStreamUtils.createCryptoOutputStream(s, conf, key) } + .getOrElse(s) + } + + /** + * Wrap an output stream for compression if block compression is enabled for its block type + */ + def wrapForCompression(blockId: BlockId, s: OutputStream): OutputStream = { + if (shouldCompress(blockId)) compressionCodec.compressedOutputStream(s) else s + } + + /** + * Wrap an input stream for compression if block compression is enabled for its block type + */ + def wrapForCompression(blockId: BlockId, s: InputStream): InputStream = { + if (shouldCompress(blockId)) compressionCodec.compressedInputStream(s) else s + } + + /** Serializes into a stream. */ + def dataSerializeStream[T: ClassTag]( + blockId: BlockId, + outputStream: OutputStream, + values: Iterator[T]): Unit = { + val byteStream = new BufferedOutputStream(outputStream) + val autoPick = !blockId.isInstanceOf[StreamBlockId] + val ser = getSerializer(implicitly[ClassTag[T]], autoPick).newInstance() + ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close() + } + + /** Serializes into a chunked byte buffer. */ + def dataSerialize[T: ClassTag]( + blockId: BlockId, + values: Iterator[T]): ChunkedByteBuffer = { + dataSerializeWithExplicitClassTag(blockId, values, implicitly[ClassTag[T]]) + } + + /** Serializes into a chunked byte buffer. */ + def dataSerializeWithExplicitClassTag( + blockId: BlockId, + values: Iterator[_], + classTag: ClassTag[_]): ChunkedByteBuffer = { + val bbos = new ChunkedByteBufferOutputStream(1024 * 1024 * 4, ByteBuffer.allocate) + val byteStream = new BufferedOutputStream(bbos) + val autoPick = !blockId.isInstanceOf[StreamBlockId] + val ser = getSerializer(classTag, autoPick).newInstance() + ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close() + bbos.toChunkedByteBuffer + } + + /** + * Deserializes an InputStream into an iterator of values and disposes of it when the end of + * the iterator is reached. + */ + def dataDeserializeStream[T]( + blockId: BlockId, + inputStream: InputStream) + (classTag: ClassTag[T]): Iterator[T] = { + val stream = new BufferedInputStream(inputStream) + val autoPick = !blockId.isInstanceOf[StreamBlockId] + getSerializer(classTag, autoPick) + .newInstance() + .deserializeStream(wrapForCompression(blockId, inputStream)) + .asIterator.asInstanceOf[Iterator[T]] + } +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala b/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala index b36c457d6d51..04e4cf88d706 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BaseShuffleHandle.scala @@ -17,8 +17,7 @@ package org.apache.spark.shuffle -import org.apache.spark.{ShuffleDependency, Aggregator, Partitioner} -import org.apache.spark.serializer.Serializer +import org.apache.spark.ShuffleDependency /** * A basic ShuffleHandle implementation that just captures registerShuffle's parameters. diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index b0abda4a81b8..c8d146030093 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -18,7 +18,8 @@ package org.apache.spark.shuffle import org.apache.spark._ -import org.apache.spark.serializer.Serializer +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.serializer.SerializerManager import org.apache.spark.storage.{BlockManager, ShuffleBlockFetcherIterator} import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.ExternalSorter @@ -32,6 +33,7 @@ private[spark] class BlockStoreShuffleReader[K, C]( startPartition: Int, endPartition: Int, context: TaskContext, + serializerManager: SerializerManager = SparkEnv.get.serializerManager, blockManager: BlockManager = SparkEnv.get.blockManager, mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker) extends ShuffleReader[K, C] with Logging { @@ -40,24 +42,23 @@ private[spark] class BlockStoreShuffleReader[K, C]( /** Read the combined key-values for this reduce task */ override def read(): Iterator[Product2[K, C]] = { - val blockFetcherItr = new ShuffleBlockFetcherIterator( + val wrappedStreams = new ShuffleBlockFetcherIterator( context, blockManager.shuffleClient, blockManager, mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition), + serializerManager.wrapStream, // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility - SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024) + SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024, + SparkEnv.get.conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue), + SparkEnv.get.conf.get(config.REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS), + SparkEnv.get.conf.get(config.REDUCER_MAX_REQ_SIZE_SHUFFLE_TO_MEM), + SparkEnv.get.conf.getBoolean("spark.shuffle.detectCorrupt", true)) - // Wrap the streams for compression based on configuration - val wrappedStreams = blockFetcherItr.map { case (blockId, inputStream) => - blockManager.wrapForCompression(blockId, inputStream) - } - - val ser = Serializer.getSerializer(dep.serializer) - val serializerInstance = ser.newInstance() + val serializerInstance = dep.serializer.newInstance() // Create a key/value iterator for each stream - val recordIter = wrappedStreams.flatMap { wrappedStream => + val recordIter = wrappedStreams.flatMap { case (blockId, wrappedStream) => // Note: the asKeyValueIterator below wraps a key/value iterator inside of a // NextIterator. The NextIterator makes sure that close() is called on the // underlying InputStream when all records have been read. @@ -65,13 +66,13 @@ private[spark] class BlockStoreShuffleReader[K, C]( } // Update the context task metrics for each record read. - val readMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() + val readMetrics = context.taskMetrics.createTempShuffleReadMetrics() val metricIter = CompletionIterator[(Any, Any), Iterator[(Any, Any)]]( - recordIter.map(record => { + recordIter.map { record => readMetrics.incRecordsRead(1) record - }), - context.taskMetrics().updateShuffleReadMetrics()) + }, + context.taskMetrics().mergeShuffleReadMetrics()) // An interruptible iterator must be used here in order to support task cancellation val interruptibleIter = new InterruptibleIterator[(Any, Any)](context, metricIter) @@ -96,15 +97,13 @@ private[spark] class BlockStoreShuffleReader[K, C]( // Sort the output if there is a sort ordering defined. dep.keyOrdering match { case Some(keyOrd: Ordering[K]) => - // Create an ExternalSorter to sort the data. Note that if spark.shuffle.spill is disabled, - // the ExternalSorter won't spill to disk. + // Create an ExternalSorter to sort the data. val sorter = - new ExternalSorter[K, C, C](context, ordering = Some(keyOrd), serializer = Some(ser)) + new ExternalSorter[K, C, C](context, ordering = Some(keyOrd), serializer = dep.serializer) sorter.insertAll(aggregatedIter) context.taskMetrics().incMemoryBytesSpilled(sorter.memoryBytesSpilled) context.taskMetrics().incDiskBytesSpilled(sorter.diskBytesSpilled) - context.internalMetricsToAccumulators( - InternalAccumulator.PEAK_EXECUTION_MEMORY).add(sorter.peakMemoryUsedBytes) + context.taskMetrics().incPeakExecutionMemory(sorter.peakMemoryUsedBytes) CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]](sorter.iterator, sorter.stop()) case None => aggregatedIter diff --git a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala index be184464e0ae..265a8acfa8d6 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala @@ -17,8 +17,8 @@ package org.apache.spark.shuffle +import org.apache.spark.{FetchFailed, TaskContext, TaskFailedReason} import org.apache.spark.storage.BlockManagerId -import org.apache.spark.{FetchFailed, TaskEndReason} import org.apache.spark.util.Utils /** @@ -26,6 +26,11 @@ import org.apache.spark.util.Utils * back to DAGScheduler (through TaskEndReason) so we'd resubmit the previous stage. * * Note that bmAddress can be null. + * + * To prevent user code from hiding this fetch failure, in the constructor we call + * [[TaskContext.setFetchFailed()]]. This means that you *must* throw this exception immediately + * after creating it -- you cannot create it, check some condition, and then decide to ignore it + * (or risk triggering any other exceptions). See SPARK-19276. */ private[spark] class FetchFailedException( bmAddress: BlockManagerId, @@ -45,7 +50,13 @@ private[spark] class FetchFailedException( this(bmAddress, shuffleId, mapId, reduceId, cause.getMessage, cause) } - def toTaskEndReason: TaskEndReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, + // SPARK-19276. We set the fetch failure in the task context, so that even if there is user-code + // which intercepts this exception (possibly wrapping it), the Executor can still tell there was + // a fetch failure, and send the correct error msg back to the driver. We wrap with an Option + // because the TaskContext is not defined in some test cases. + Option(TaskContext.get()).map(_.setFetchFailed(this)) + + def toTaskFailedReason: TaskFailedReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, Utils.exceptionString(this)) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala deleted file mode 100644 index cd253a78c2b1..000000000000 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ /dev/null @@ -1,150 +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.shuffle - -import java.util.concurrent.ConcurrentLinkedQueue - -import scala.collection.JavaConverters._ - -import org.apache.spark.{Logging, SparkConf, SparkEnv} -import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} -import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.serializer.Serializer -import org.apache.spark.storage._ -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} - -/** A group of writers for a ShuffleMapTask, one writer per reducer. */ -private[spark] trait ShuffleWriterGroup { - val writers: Array[DiskBlockObjectWriter] - - /** @param success Indicates all writes were successful. If false, no blocks will be recorded. */ - def releaseWriters(success: Boolean) -} - -/** - * Manages assigning disk-based block writers to shuffle tasks. Each shuffle task gets one file - * per reducer. - */ -// Note: Changes to the format in this file should be kept in sync with -// org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getHashBasedShuffleBlockData(). -private[spark] class FileShuffleBlockResolver(conf: SparkConf) - extends ShuffleBlockResolver with Logging { - - private val transportConf = SparkTransportConf.fromSparkConf(conf) - - private lazy val blockManager = SparkEnv.get.blockManager - - // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided - private val bufferSize = conf.getSizeAsKb("spark.shuffle.file.buffer", "32k").toInt * 1024 - - /** - * Contains all the state related to a particular shuffle. - */ - private class ShuffleState(val numReducers: Int) { - /** - * The mapIds of all map tasks completed on this Executor for this shuffle. - */ - val completedMapTasks = new ConcurrentLinkedQueue[Int]() - } - - private val shuffleStates = new TimeStampedHashMap[ShuffleId, ShuffleState] - - private val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup, conf) - - /** - * Get a ShuffleWriterGroup for the given map task, which will register it as complete - * when the writers are closed successfully - */ - def forMapTask(shuffleId: Int, mapId: Int, numReducers: Int, serializer: Serializer, - writeMetrics: ShuffleWriteMetrics): ShuffleWriterGroup = { - new ShuffleWriterGroup { - shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numReducers)) - private val shuffleState = shuffleStates(shuffleId) - - val openStartTime = System.nanoTime - val serializerInstance = serializer.newInstance() - val writers: Array[DiskBlockObjectWriter] = { - Array.tabulate[DiskBlockObjectWriter](numReducers) { bucketId => - val blockId = ShuffleBlockId(shuffleId, mapId, bucketId) - val blockFile = blockManager.diskBlockManager.getFile(blockId) - // Because of previous failures, the shuffle file may already exist on this machine. - // If so, remove it. - if (blockFile.exists) { - if (blockFile.delete()) { - logInfo(s"Removed existing shuffle file $blockFile") - } else { - logWarning(s"Failed to remove existing shuffle file $blockFile") - } - } - blockManager.getDiskWriter(blockId, blockFile, serializerInstance, bufferSize, - writeMetrics) - } - } - // Creating the file to write to and creating a disk writer both involve interacting with - // the disk, so should be included in the shuffle write time. - writeMetrics.incShuffleWriteTime(System.nanoTime - openStartTime) - - override def releaseWriters(success: Boolean) { - shuffleState.completedMapTasks.add(mapId) - } - } - } - - override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { - val file = blockManager.diskBlockManager.getFile(blockId) - new FileSegmentManagedBuffer(transportConf, file, 0, file.length) - } - - /** Remove all the blocks / files and metadata related to a particular shuffle. */ - def removeShuffle(shuffleId: ShuffleId): Boolean = { - // Do not change the ordering of this, if shuffleStates should be removed only - // after the corresponding shuffle blocks have been removed - val cleaned = removeShuffleBlocks(shuffleId) - shuffleStates.remove(shuffleId) - cleaned - } - - /** Remove all the blocks / files related to a particular shuffle. */ - private def removeShuffleBlocks(shuffleId: ShuffleId): Boolean = { - shuffleStates.get(shuffleId) match { - case Some(state) => - for (mapId <- state.completedMapTasks.asScala; reduceId <- 0 until state.numReducers) { - val blockId = new ShuffleBlockId(shuffleId, mapId, reduceId) - val file = blockManager.diskBlockManager.getFile(blockId) - if (!file.delete()) { - logWarning(s"Error deleting ${file.getPath()}") - } - } - logInfo("Deleted all files for shuffle " + shuffleId) - true - case None => - logInfo("Could not find files for shuffle " + shuffleId + " for deleting") - false - } - } - - private def cleanup(cleanupTime: Long) { - shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => removeShuffleBlocks(shuffleId)) - } - - override def stop() { - metadataCleaner.cancel() - } -} diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 5e4c2b5d0a5c..94a3a78e9416 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -18,17 +18,19 @@ package org.apache.spark.shuffle import java.io._ +import java.nio.file.Files import com.google.common.io.ByteStreams -import org.apache.spark.{SparkConf, SparkEnv, Logging} +import org.apache.spark.{SparkConf, SparkEnv} +import org.apache.spark.internal.Logging +import org.apache.spark.io.NioBufferedFileInputStream import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.shuffle.IndexShuffleBlockResolver.NOOP_REDUCE_ID import org.apache.spark.storage._ import org.apache.spark.util.Utils -import IndexShuffleBlockResolver.NOOP_REDUCE_ID - /** * Create and maintain the shuffle blocks' mapping between logic block and physical file location. * Data of shuffle blocks from the same map task are stored in a single consolidated data file. @@ -40,12 +42,15 @@ import IndexShuffleBlockResolver.NOOP_REDUCE_ID */ // Note: Changes to the format in this file should be kept in sync with // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getSortBasedShuffleBlockData(). -private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleBlockResolver +private[spark] class IndexShuffleBlockResolver( + conf: SparkConf, + _blockManager: BlockManager = null) + extends ShuffleBlockResolver with Logging { - private lazy val blockManager = SparkEnv.get.blockManager + private lazy val blockManager = Option(_blockManager).getOrElse(SparkEnv.get.blockManager) - private val transportConf = SparkTransportConf.fromSparkConf(conf) + private val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") def getDataFile(shuffleId: Int, mapId: Int): File = { blockManager.diskBlockManager.getFile(ShuffleDataBlockId(shuffleId, mapId, NOOP_REDUCE_ID)) @@ -57,7 +62,7 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB /** * Remove data file and index file that contain the output data from one map. - * */ + */ def removeDataByMap(shuffleId: Int, mapId: Int): Unit = { var file = getDataFile(shuffleId, mapId) if (file.exists()) { @@ -74,24 +79,117 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB } } + /** + * Check whether the given index and data files match each other. + * If so, return the partition lengths in the data file. Otherwise return null. + */ + private def checkIndexAndDataFile(index: File, data: File, blocks: Int): Array[Long] = { + // the index file should have `block + 1` longs as offset. + if (index.length() != (blocks + 1) * 8) { + return null + } + val lengths = new Array[Long](blocks) + // Read the lengths of blocks + val in = try { + new DataInputStream(new NioBufferedFileInputStream(index)) + } catch { + case e: IOException => + return null + } + try { + // Convert the offsets into lengths of each block + var offset = in.readLong() + if (offset != 0L) { + return null + } + var i = 0 + while (i < blocks) { + val off = in.readLong() + lengths(i) = off - offset + offset = off + i += 1 + } + } catch { + case e: IOException => + return null + } finally { + in.close() + } + + // the size of data file should match with index file + if (data.length() == lengths.sum) { + lengths + } else { + null + } + } + /** * Write an index file with the offsets of each block, plus a final offset at the end for the * end of the output file. This will be used by getBlockData to figure out where each block * begins and ends. - * */ - def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]): Unit = { + * + * It will commit the data and index file as an atomic operation, use the existing ones, or + * replace them with new ones. + * + * Note: the `lengths` will be updated to match the existing index file if use the existing ones. + */ + def writeIndexFileAndCommit( + shuffleId: Int, + mapId: Int, + lengths: Array[Long], + dataTmp: File): Unit = { val indexFile = getIndexFile(shuffleId, mapId) - val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile))) - Utils.tryWithSafeFinally { - // We take in lengths of each block, need to convert it to offsets. - var offset = 0L - out.writeLong(offset) - for (length <- lengths) { - offset += length + val indexTmp = Utils.tempFileWith(indexFile) + try { + val out = new DataOutputStream( + new BufferedOutputStream(Files.newOutputStream(indexTmp.toPath))) + Utils.tryWithSafeFinally { + // We take in lengths of each block, need to convert it to offsets. + var offset = 0L out.writeLong(offset) + for (length <- lengths) { + offset += length + out.writeLong(offset) + } + } { + out.close() + } + + val dataFile = getDataFile(shuffleId, mapId) + // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure + // the following check and rename are atomic. + synchronized { + val existingLengths = checkIndexAndDataFile(indexFile, dataFile, lengths.length) + if (existingLengths != null) { + // Another attempt for the same task has already written our map outputs successfully, + // so just use the existing partition lengths and delete our temporary map outputs. + System.arraycopy(existingLengths, 0, lengths, 0, lengths.length) + if (dataTmp != null && dataTmp.exists()) { + dataTmp.delete() + } + indexTmp.delete() + } else { + // This is the first successful attempt in writing the map outputs for this task, + // so override any existing index and data files with the ones we wrote. + if (indexFile.exists()) { + indexFile.delete() + } + if (dataFile.exists()) { + dataFile.delete() + } + if (!indexTmp.renameTo(indexFile)) { + throw new IOException("fail to rename file " + indexTmp + " to " + indexFile) + } + if (dataTmp != null && dataTmp.exists() && !dataTmp.renameTo(dataFile)) { + throw new IOException("fail to rename file " + dataTmp + " to " + dataFile) + } + } + } + } finally { + if (indexTmp.exists() && !indexTmp.delete()) { + logError(s"Failed to delete temporary index file at ${indexTmp.getAbsolutePath}") } - } { - out.close() } } @@ -100,7 +198,7 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB // find out the consolidated file, then the offset within that from our index val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId) - val in = new DataInputStream(new FileInputStream(indexFile)) + val in = new DataInputStream(Files.newInputStream(indexFile.toPath)) try { ByteStreams.skipFully(in, blockId.reduceId * 8) val offset = in.readLong() diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala index 4342b0d598b1..d1ecbc1bf017 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala @@ -17,7 +17,6 @@ package org.apache.spark.shuffle -import java.nio.ByteBuffer import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.storage.ShuffleBlockId diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 978366d1a1d1..4ea8a7120a9c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -17,7 +17,7 @@ package org.apache.spark.shuffle -import org.apache.spark.{TaskContext, ShuffleDependency} +import org.apache.spark.{ShuffleDependency, TaskContext} /** * Pluggable interface for shuffle systems. A ShuffleManager is created in SparkEnv on the driver @@ -28,6 +28,7 @@ import org.apache.spark.{TaskContext, ShuffleDependency} * boolean isDriver as parameters. */ private[spark] trait ShuffleManager { + /** * Register a shuffle with the manager and obtain a handle for it to pass to tasks. */ @@ -50,9 +51,9 @@ private[spark] trait ShuffleManager { context: TaskContext): ShuffleReader[K, C] /** - * Remove a shuffle's metadata from the ShuffleManager. - * @return true if the metadata removed successfully, otherwise false. - */ + * Remove a shuffle's metadata from the ShuffleManager. + * @return true if the metadata removed successfully, otherwise false. + */ def unregisterShuffle(shuffleId: Int): Boolean /** diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala deleted file mode 100644 index d2e2fc4c110a..000000000000 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala +++ /dev/null @@ -1,78 +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.shuffle.hash - -import org.apache.spark._ -import org.apache.spark.shuffle._ - -/** - * A ShuffleManager using hashing, that creates one output file per reduce partition on each - * mapper (possibly reusing these across waves of tasks). - */ -private[spark] class HashShuffleManager(conf: SparkConf) extends ShuffleManager with Logging { - - if (!conf.getBoolean("spark.shuffle.spill", true)) { - logWarning( - "spark.shuffle.spill was set to false, but this configuration is ignored as of Spark 1.6+." + - " Shuffle will continue to spill to disk when necessary.") - } - - private val fileShuffleBlockResolver = new FileShuffleBlockResolver(conf) - - /* Register a shuffle with the manager and obtain a handle for it to pass to tasks. */ - override def registerShuffle[K, V, C]( - shuffleId: Int, - numMaps: Int, - dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { - new BaseShuffleHandle(shuffleId, numMaps, dependency) - } - - /** - * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive). - * Called on executors by reduce tasks. - */ - override def getReader[K, C]( - handle: ShuffleHandle, - startPartition: Int, - endPartition: Int, - context: TaskContext): ShuffleReader[K, C] = { - new BlockStoreShuffleReader( - handle.asInstanceOf[BaseShuffleHandle[K, _, C]], startPartition, endPartition, context) - } - - /** Get a writer for a given partition. Called on executors by map tasks. */ - override def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext) - : ShuffleWriter[K, V] = { - new HashShuffleWriter( - shuffleBlockResolver, handle.asInstanceOf[BaseShuffleHandle[K, V, _]], mapId, context) - } - - /** Remove a shuffle's metadata from the ShuffleManager. */ - override def unregisterShuffle(shuffleId: Int): Boolean = { - shuffleBlockResolver.removeShuffle(shuffleId) - } - - override def shuffleBlockResolver: FileShuffleBlockResolver = { - fileShuffleBlockResolver - } - - /** Shut down this ShuffleManager. */ - override def stop(): Unit = { - shuffleBlockResolver.stop() - } -} diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala deleted file mode 100644 index 41df70c602c3..000000000000 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ /dev/null @@ -1,119 +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.shuffle.hash - -import org.apache.spark._ -import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.scheduler.MapStatus -import org.apache.spark.serializer.Serializer -import org.apache.spark.shuffle._ -import org.apache.spark.storage.DiskBlockObjectWriter - -private[spark] class HashShuffleWriter[K, V]( - shuffleBlockResolver: FileShuffleBlockResolver, - handle: BaseShuffleHandle[K, V, _], - mapId: Int, - context: TaskContext) - extends ShuffleWriter[K, V] with Logging { - - private val dep = handle.dependency - private val numOutputSplits = dep.partitioner.numPartitions - private val metrics = context.taskMetrics - - // Are we in the process of stopping? Because map tasks can call stop() with success = true - // and then call stop() with success = false if they get an exception, we want to make sure - // we don't try deleting files, etc twice. - private var stopping = false - - private val writeMetrics = new ShuffleWriteMetrics() - metrics.shuffleWriteMetrics = Some(writeMetrics) - - private val blockManager = SparkEnv.get.blockManager - private val ser = Serializer.getSerializer(dep.serializer.getOrElse(null)) - private val shuffle = shuffleBlockResolver.forMapTask(dep.shuffleId, mapId, numOutputSplits, ser, - writeMetrics) - - /** Write a bunch of records to this task's output */ - override def write(records: Iterator[Product2[K, V]]): Unit = { - val iter = if (dep.aggregator.isDefined) { - if (dep.mapSideCombine) { - dep.aggregator.get.combineValuesByKey(records, context) - } else { - records - } - } else { - require(!dep.mapSideCombine, "Map-side combine without Aggregator specified!") - records - } - - for (elem <- iter) { - val bucketId = dep.partitioner.getPartition(elem._1) - shuffle.writers(bucketId).write(elem._1, elem._2) - } - } - - /** Close this writer, passing along whether the map completed */ - override def stop(initiallySuccess: Boolean): Option[MapStatus] = { - var success = initiallySuccess - try { - if (stopping) { - return None - } - stopping = true - if (success) { - try { - Some(commitWritesAndBuildStatus()) - } catch { - case e: Exception => - success = false - revertWrites() - throw e - } - } else { - revertWrites() - None - } - } finally { - // Release the writers back to the shuffle block manager. - if (shuffle != null && shuffle.writers != null) { - try { - shuffle.releaseWriters(success) - } catch { - case e: Exception => logError("Failed to release shuffle writers", e) - } - } - } - } - - private def commitWritesAndBuildStatus(): MapStatus = { - // Commit the writes. Get the size of each bucket block (total block size). - val sizes: Array[Long] = shuffle.writers.map { writer: DiskBlockObjectWriter => - writer.commitAndClose() - writer.fileSegment().length - } - MapStatus(blockManager.shuffleServerId, sizes) - } - - private def revertWrites(): Unit = { - if (shuffle != null && shuffle.writers != null) { - for (writer <- shuffle.writers) { - writer.revertPartialWritesAndClose() - } - } - } -} diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 66b6bbc61fe8..bfb4dc698e32 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -20,7 +20,7 @@ package org.apache.spark.shuffle.sort import java.util.concurrent.ConcurrentHashMap import org.apache.spark._ -import org.apache.spark.serializer.Serializer +import org.apache.spark.internal.Logging import org.apache.spark.shuffle._ /** @@ -82,13 +82,13 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager override val shuffleBlockResolver = new IndexShuffleBlockResolver(conf) /** - * Register a shuffle with the manager and obtain a handle for it to pass to tasks. + * Obtains a [[ShuffleHandle]] to pass to tasks. */ override def registerShuffle[K, V, C]( shuffleId: Int, numMaps: Int, dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { - if (SortShuffleWriter.shouldBypassMergeSort(SparkEnv.get.conf, dependency)) { + if (SortShuffleWriter.shouldBypassMergeSort(conf, dependency)) { // If there are fewer than spark.shuffle.sort.bypassMergeThreshold partitions and we don't // need map-side aggregation, then write numPartitions files directly and just concatenate // them at the end. This avoids doing serialization and deserialization twice to merge @@ -184,10 +184,9 @@ private[spark] object SortShuffleManager extends Logging { def canUseSerializedShuffle(dependency: ShuffleDependency[_, _, _]): Boolean = { val shufId = dependency.shuffleId val numPartitions = dependency.partitioner.numPartitions - val serializer = Serializer.getSerializer(dependency.serializer) - if (!serializer.supportsRelocationOfSerializedObjects) { + if (!dependency.serializer.supportsRelocationOfSerializedObjects) { log.debug(s"Can't use serialized shuffle for shuffle $shufId because the serializer, " + - s"${serializer.getClass.getName}, does not support object relocation") + s"${dependency.serializer.getClass.getName}, does not support object relocation") false } else if (dependency.aggregator.isDefined) { log.debug( diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 808317b017a0..636b88e792bf 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -18,10 +18,11 @@ package org.apache.spark.shuffle.sort import org.apache.spark._ -import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.MapStatus -import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleWriter, BaseShuffleHandle} +import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleWriter} import org.apache.spark.storage.ShuffleBlockId +import org.apache.spark.util.Utils import org.apache.spark.util.collection.ExternalSorter private[spark] class SortShuffleWriter[K, V, C]( @@ -44,8 +45,7 @@ private[spark] class SortShuffleWriter[K, V, C]( private var mapStatus: MapStatus = null - private val writeMetrics = new ShuffleWriteMetrics() - context.taskMetrics.shuffleWriteMetrics = Some(writeMetrics) + private val writeMetrics = context.taskMetrics().shuffleWriteMetrics /** Write a bunch of records to this task's output */ override def write(records: Iterator[Product2[K, V]]): Unit = { @@ -65,12 +65,18 @@ private[spark] class SortShuffleWriter[K, V, C]( // Don't bother including the time to open the merged output file in the shuffle write time, // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). - val outputFile = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) - val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) - val partitionLengths = sorter.writePartitionedFile(blockId, outputFile) - shuffleBlockResolver.writeIndexFile(dep.shuffleId, mapId, partitionLengths) - - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + val output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) + val tmp = Utils.tempFileWith(output) + try { + val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) + val partitionLengths = sorter.writePartitionedFile(blockId, tmp) + shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + } finally { + if (tmp.exists() && !tmp.delete()) { + logError(s"Error while deleting temp file ${tmp.getAbsolutePath}") + } + } } /** Close this writer, passing along whether the map completed */ @@ -83,8 +89,6 @@ private[spark] class SortShuffleWriter[K, V, C]( if (success) { return Option(mapStatus) } else { - // The map task failed, so delete our output data. - shuffleBlockResolver.removeDataByMap(dep.shuffleId, mapId) return None } } finally { @@ -92,8 +96,7 @@ private[spark] class SortShuffleWriter[K, V, C]( if (sorter != null) { val startTime = System.nanoTime() sorter.stop() - context.taskMetrics.shuffleWriteMetrics.foreach( - _.incShuffleWriteTime(System.nanoTime - startTime)) + writeMetrics.incWriteTime(System.nanoTime - startTime) sorter = null } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllExecutorListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllExecutorListResource.scala new file mode 100644 index 000000000000..eb5cc1b9a3bd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllExecutorListResource.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.status.api.v1 + +import javax.ws.rs.{GET, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.exec.ExecutorsPage + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class AllExecutorListResource(ui: SparkUI) { + + @GET + def executorList(): Seq[ExecutorSummary] = { + val listener = ui.executorsListener + listener.synchronized { + // The follow codes should be protected by `listener` to make sure no executors will be + // removed before we query their status. See SPARK-12784. + (0 until listener.activeStorageStatusList.size).map { statusId => + ExecutorsPage.getExecInfo(listener, statusId, isActive = true) + } ++ (0 until listener.deadStorageStatusList.size).map { statusId => + ExecutorsPage.getExecInfo(listener, statusId, isActive = false) + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala index 5783df5d8220..d0d9ef1165e8 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala @@ -68,7 +68,12 @@ private[v1] object AllJobsResource { listener: JobProgressListener, includeStageDetails: Boolean): JobData = { listener.synchronized { - val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max) + val lastStageInfo = + if (job.stageIds.isEmpty) { + None + } else { + listener.stageIdToInfo.get(job.stageIds.max) + } val lastStageData = lastStageInfo.flatMap { s => listener.stageIdToData.get((s.stageId, s.attemptId)) } @@ -86,7 +91,7 @@ private[v1] object AllJobsResource { numTasks = job.numTasks, numActiveTasks = job.numActiveTasks, numCompletedTasks = job.numCompletedTasks, - numSkippedTasks = job.numCompletedTasks, + numSkippedTasks = job.numSkippedTasks, numFailedTasks = job.numFailedTasks, numActiveStages = job.numActiveStages, numCompletedStages = job.completedStageIndices.size, diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala index 645ede26a087..1279b281ad8d 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala @@ -28,7 +28,7 @@ private[v1] class AllRDDResource(ui: SparkUI) { @GET def rddList(): Seq[RDDStorageInfo] = { - val storageStatusList = ui.storageListener.storageStatusList + val storageStatusList = ui.storageListener.activeStorageStatusList val rddInfos = ui.storageListener.rddInfoList rddInfos.map{rddInfo => AllRDDResource.getRDDStorageInfo(rddInfo.id, rddInfo, storageStatusList, @@ -44,7 +44,7 @@ private[spark] object AllRDDResource { rddId: Int, listener: StorageListener, includeDetails: Boolean): Option[RDDStorageInfo] = { - val storageStatusList = listener.storageStatusList + val storageStatusList = listener.activeStorageStatusList listener.rddInfoList.find { _.id == rddId }.map { rddInfo => getRDDStorageInfo(rddId, rddInfo, storageStatusList, includeDetails) } @@ -61,7 +61,7 @@ private[spark] object AllRDDResource { .flatMap { _.rddBlocksById(rddId) } .sortWith { _._1.name < _._1.name } .map { case (blockId, status) => - (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) + (blockId, status, blockLocations.getOrElse(blockId, Seq[String]("Unknown"))) } val dataDistribution = if (includeDetails) { @@ -70,7 +70,13 @@ private[spark] object AllRDDResource { address = status.blockManagerId.hostPort, memoryUsed = status.memUsedByRdd(rddId), memoryRemaining = status.memRemaining, - diskUsed = status.diskUsedByRdd(rddId) + diskUsed = status.diskUsedByRdd(rddId), + onHeapMemoryUsed = Some( + if (!rddInfo.storageLevel.useOffHeap) status.memUsedByRdd(rddId) else 0L), + offHeapMemoryUsed = Some( + if (rddInfo.storageLevel.useOffHeap) status.memUsedByRdd(rddId) else 0L), + onHeapMemoryRemaining = status.onHeapMemRemaining, + offHeapMemoryRemaining = status.offHeapMemRemaining ) } ) } else { None diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala index 24a0b5220695..4a4ed954d689 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -17,13 +17,13 @@ package org.apache.spark.status.api.v1 import java.util.{Arrays, Date, List => JList} -import javax.ws.rs.{GET, PathParam, Produces, QueryParam} +import javax.ws.rs.{GET, Produces, QueryParam} import javax.ws.rs.core.MediaType -import org.apache.spark.executor.{InputMetrics => InternalInputMetrics, OutputMetrics => InternalOutputMetrics, ShuffleReadMetrics => InternalShuffleReadMetrics, ShuffleWriteMetrics => InternalShuffleWriteMetrics, TaskMetrics => InternalTaskMetrics} import org.apache.spark.scheduler.{AccumulableInfo => InternalAccumulableInfo, StageInfo} import org.apache.spark.ui.SparkUI import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData} +import org.apache.spark.ui.jobs.UIData.{InputMetricsUIData => InternalInputMetrics, OutputMetricsUIData => InternalOutputMetrics, ShuffleReadMetricsUIData => InternalShuffleReadMetrics, ShuffleWriteMetricsUIData => InternalShuffleWriteMetrics, TaskMetricsUIData => InternalTaskMetrics} import org.apache.spark.util.Distribution @Produces(Array(MediaType.APPLICATION_JSON)) @@ -47,6 +47,7 @@ private[v1] class AllStagesResource(ui: SparkUI) { listener.stageIdToData.get((stageInfo.stageId, stageInfo.attemptId)) } } yield { + stageUiData.lastUpdateTime = ui.lastUpdateTime AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, includeDetails = false) } } @@ -59,8 +60,18 @@ private[v1] object AllStagesResource { stageUiData: StageUIData, includeDetails: Boolean): StageData = { + val taskLaunchTimes = stageUiData.taskData.values.map(_.taskInfo.launchTime).filter(_ > 0) + + val firstTaskLaunchedTime: Option[Date] = + if (taskLaunchTimes.nonEmpty) { + Some(new Date(taskLaunchTimes.min)) + } else { + None + } + val taskData = if (includeDetails) { - Some(stageUiData.taskData.map { case (k, v) => k -> convertTaskData(v) } ) + Some(stageUiData.taskData.map { case (k, v) => + k -> convertTaskData(v, stageUiData.lastUpdateTime) }) } else { None } @@ -92,6 +103,10 @@ private[v1] object AllStagesResource { numCompleteTasks = stageUiData.numCompleteTasks, numFailedTasks = stageUiData.numFailedTasks, executorRunTime = stageUiData.executorRunTime, + executorCpuTime = stageUiData.executorCpuTime, + submissionTime = stageInfo.submissionTime.map(new Date(_)), + firstTaskLaunchedTime, + completionTime = stageInfo.completionTime.map(new Date(_)), inputBytes = stageUiData.inputBytes, inputRecords = stageUiData.inputRecords, outputBytes = stageUiData.outputBytes, @@ -123,19 +138,21 @@ private[v1] object AllStagesResource { } } - def convertTaskData(uiData: TaskUIData): TaskData = { + def convertTaskData(uiData: TaskUIData, lastUpdateTime: Option[Long]): TaskData = { new TaskData( taskId = uiData.taskInfo.taskId, index = uiData.taskInfo.index, attempt = uiData.taskInfo.attemptNumber, launchTime = new Date(uiData.taskInfo.launchTime), + duration = uiData.taskDuration(lastUpdateTime), executorId = uiData.taskInfo.executorId, host = uiData.taskInfo.host, + status = uiData.taskInfo.status, taskLocality = uiData.taskInfo.taskLocality.toString(), speculative = uiData.taskInfo.speculative, accumulatorUpdates = uiData.taskInfo.accumulables.map { convertAccumulableInfo }, errorMessage = uiData.errorMessage, - taskMetrics = uiData.taskMetrics.map { convertUiTaskMetrics } + taskMetrics = uiData.metrics.map { convertUiTaskMetrics } ) } @@ -143,7 +160,7 @@ private[v1] object AllStagesResource { allTaskData: Iterable[TaskUIData], quantiles: Array[Double]): TaskMetricDistributions = { - val rawMetrics = allTaskData.flatMap{_.taskMetrics}.toSeq + val rawMetrics = allTaskData.flatMap{_.metrics}.toSeq def metricQuantiles(f: InternalTaskMetrics => Double): IndexedSeq[Double] = Distribution(rawMetrics.map { d => f(d) }).get.getQuantiles(quantiles) @@ -155,63 +172,63 @@ private[v1] object AllStagesResource { // to make it a little easier to deal w/ all of the nested options. Mostly it lets us just // implement one "build" method, which just builds the quantiles for each field. - val inputMetrics: Option[InputMetricDistributions] = + val inputMetrics: InputMetricDistributions = new MetricHelper[InternalInputMetrics, InputMetricDistributions](rawMetrics, quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): Option[InternalInputMetrics] = { - raw.inputMetrics - } + def getSubmetrics(raw: InternalTaskMetrics): InternalInputMetrics = raw.inputMetrics def build: InputMetricDistributions = new InputMetricDistributions( bytesRead = submetricQuantiles(_.bytesRead), recordsRead = submetricQuantiles(_.recordsRead) ) - }.metricOption + }.build - val outputMetrics: Option[OutputMetricDistributions] = + val outputMetrics: OutputMetricDistributions = new MetricHelper[InternalOutputMetrics, OutputMetricDistributions](rawMetrics, quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): Option[InternalOutputMetrics] = { - raw.outputMetrics - } + def getSubmetrics(raw: InternalTaskMetrics): InternalOutputMetrics = raw.outputMetrics + def build: OutputMetricDistributions = new OutputMetricDistributions( bytesWritten = submetricQuantiles(_.bytesWritten), recordsWritten = submetricQuantiles(_.recordsWritten) ) - }.metricOption + }.build - val shuffleReadMetrics: Option[ShuffleReadMetricDistributions] = + val shuffleReadMetrics: ShuffleReadMetricDistributions = new MetricHelper[InternalShuffleReadMetrics, ShuffleReadMetricDistributions](rawMetrics, quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): Option[InternalShuffleReadMetrics] = { + def getSubmetrics(raw: InternalTaskMetrics): InternalShuffleReadMetrics = raw.shuffleReadMetrics - } + def build: ShuffleReadMetricDistributions = new ShuffleReadMetricDistributions( readBytes = submetricQuantiles(_.totalBytesRead), readRecords = submetricQuantiles(_.recordsRead), remoteBytesRead = submetricQuantiles(_.remoteBytesRead), + remoteBytesReadToDisk = submetricQuantiles(_.remoteBytesReadToDisk), remoteBlocksFetched = submetricQuantiles(_.remoteBlocksFetched), localBlocksFetched = submetricQuantiles(_.localBlocksFetched), totalBlocksFetched = submetricQuantiles(_.totalBlocksFetched), fetchWaitTime = submetricQuantiles(_.fetchWaitTime) ) - }.metricOption + }.build - val shuffleWriteMetrics: Option[ShuffleWriteMetricDistributions] = + val shuffleWriteMetrics: ShuffleWriteMetricDistributions = new MetricHelper[InternalShuffleWriteMetrics, ShuffleWriteMetricDistributions](rawMetrics, quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): Option[InternalShuffleWriteMetrics] = { + def getSubmetrics(raw: InternalTaskMetrics): InternalShuffleWriteMetrics = raw.shuffleWriteMetrics - } + def build: ShuffleWriteMetricDistributions = new ShuffleWriteMetricDistributions( - writeBytes = submetricQuantiles(_.shuffleBytesWritten), - writeRecords = submetricQuantiles(_.shuffleRecordsWritten), - writeTime = submetricQuantiles(_.shuffleWriteTime) + writeBytes = submetricQuantiles(_.bytesWritten), + writeRecords = submetricQuantiles(_.recordsWritten), + writeTime = submetricQuantiles(_.writeTime) ) - }.metricOption + }.build new TaskMetricDistributions( quantiles = quantiles, executorDeserializeTime = metricQuantiles(_.executorDeserializeTime), + executorDeserializeCpuTime = metricQuantiles(_.executorDeserializeCpuTime), executorRunTime = metricQuantiles(_.executorRunTime), + executorCpuTime = metricQuantiles(_.executorCpuTime), resultSize = metricQuantiles(_.resultSize), jvmGcTime = metricQuantiles(_.jvmGCTime), resultSerializationTime = metricQuantiles(_.resultSerializationTime), @@ -225,22 +242,25 @@ private[v1] object AllStagesResource { } def convertAccumulableInfo(acc: InternalAccumulableInfo): AccumulableInfo = { - new AccumulableInfo(acc.id, acc.name, acc.update, acc.value) + new AccumulableInfo( + acc.id, acc.name.orNull, acc.update.map(_.toString), acc.value.map(_.toString).orNull) } def convertUiTaskMetrics(internal: InternalTaskMetrics): TaskMetrics = { new TaskMetrics( executorDeserializeTime = internal.executorDeserializeTime, + executorDeserializeCpuTime = internal.executorDeserializeCpuTime, executorRunTime = internal.executorRunTime, + executorCpuTime = internal.executorCpuTime, resultSize = internal.resultSize, jvmGcTime = internal.jvmGCTime, resultSerializationTime = internal.resultSerializationTime, memoryBytesSpilled = internal.memoryBytesSpilled, diskBytesSpilled = internal.diskBytesSpilled, - inputMetrics = internal.inputMetrics.map { convertInputMetrics }, - outputMetrics = Option(internal.outputMetrics).flatten.map { convertOutputMetrics }, - shuffleReadMetrics = internal.shuffleReadMetrics.map { convertShuffleReadMetrics }, - shuffleWriteMetrics = internal.shuffleWriteMetrics.map { convertShuffleWriteMetrics } + inputMetrics = convertInputMetrics(internal.inputMetrics), + outputMetrics = convertOutputMetrics(internal.outputMetrics), + shuffleReadMetrics = convertShuffleReadMetrics(internal.shuffleReadMetrics), + shuffleWriteMetrics = convertShuffleWriteMetrics(internal.shuffleWriteMetrics) ) } @@ -264,46 +284,36 @@ private[v1] object AllStagesResource { localBlocksFetched = internal.localBlocksFetched, fetchWaitTime = internal.fetchWaitTime, remoteBytesRead = internal.remoteBytesRead, - totalBlocksFetched = internal.totalBlocksFetched, + remoteBytesReadToDisk = internal.remoteBytesReadToDisk, + localBytesRead = internal.localBytesRead, recordsRead = internal.recordsRead ) } def convertShuffleWriteMetrics(internal: InternalShuffleWriteMetrics): ShuffleWriteMetrics = { new ShuffleWriteMetrics( - bytesWritten = internal.shuffleBytesWritten, - writeTime = internal.shuffleWriteTime, - recordsWritten = internal.shuffleRecordsWritten + bytesWritten = internal.bytesWritten, + writeTime = internal.writeTime, + recordsWritten = internal.recordsWritten ) } } /** - * Helper for getting distributions from nested metric types. Many of the metrics we want are - * contained in options inside TaskMetrics (eg., ShuffleWriteMetrics). This makes it easy to handle - * the options (returning None if the metrics are all empty), and extract the quantiles for each - * metric. After creating an instance, call metricOption to get the result type. + * Helper for getting distributions from nested metric types. */ private[v1] abstract class MetricHelper[I, O]( rawMetrics: Seq[InternalTaskMetrics], quantiles: Array[Double]) { - def getSubmetrics(raw: InternalTaskMetrics): Option[I] + def getSubmetrics(raw: InternalTaskMetrics): I def build: O - val data: Seq[I] = rawMetrics.flatMap(getSubmetrics) + val data: Seq[I] = rawMetrics.map(getSubmetrics) /** applies the given function to all input metrics, and returns the quantiles */ def submetricQuantiles(f: I => Double): IndexedSeq[Double] = { Distribution(data.map { d => f(d) }).get.getQuantiles(quantiles) } - - def metricOption: Option[O] = { - if (data.isEmpty) { - None - } else { - Some(build) - } - } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index 50b6ba67e993..f17b63775482 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -18,13 +18,14 @@ package org.apache.spark.status.api.v1 import java.util.zip.ZipOutputStream import javax.servlet.ServletContext +import javax.servlet.http.HttpServletRequest import javax.ws.rs._ import javax.ws.rs.core.{Context, Response} -import com.sun.jersey.api.core.ResourceConfig -import com.sun.jersey.spi.container.servlet.ServletContainer import org.eclipse.jetty.server.handler.ContextHandler import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} +import org.glassfish.jersey.server.ServerProperties +import org.glassfish.jersey.servlet.ServletContainer import org.apache.spark.SecurityManager import org.apache.spark.ui.SparkUI @@ -40,7 +41,7 @@ import org.apache.spark.ui.SparkUI * HistoryServerSuite. */ @Path("/v1") -private[v1] class ApiRootResource extends UIRootFromServletContext { +private[v1] class ApiRootResource extends ApiRequestContext { @Path("applications") def getApplicationList(): ApplicationListResource = { @@ -56,21 +57,21 @@ private[v1] class ApiRootResource extends UIRootFromServletContext { def getJobs( @PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): AllJobsResource = { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + withSparkUI(appId, Some(attemptId)) { ui => new AllJobsResource(ui) } } @Path("applications/{appId}/jobs") def getJobs(@PathParam("appId") appId: String): AllJobsResource = { - uiRoot.withSparkUI(appId, None) { ui => + withSparkUI(appId, None) { ui => new AllJobsResource(ui) } } @Path("applications/{appId}/jobs/{jobId: \\d+}") def getJob(@PathParam("appId") appId: String): OneJobResource = { - uiRoot.withSparkUI(appId, None) { ui => + withSparkUI(appId, None) { ui => new OneJobResource(ui) } } @@ -79,31 +80,46 @@ private[v1] class ApiRootResource extends UIRootFromServletContext { def getJob( @PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): OneJobResource = { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + withSparkUI(appId, Some(attemptId)) { ui => new OneJobResource(ui) } } @Path("applications/{appId}/executors") def getExecutors(@PathParam("appId") appId: String): ExecutorListResource = { - uiRoot.withSparkUI(appId, None) { ui => + withSparkUI(appId, None) { ui => new ExecutorListResource(ui) } } + @Path("applications/{appId}/allexecutors") + def getAllExecutors(@PathParam("appId") appId: String): AllExecutorListResource = { + withSparkUI(appId, None) { ui => + new AllExecutorListResource(ui) + } + } + @Path("applications/{appId}/{attemptId}/executors") def getExecutors( @PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): ExecutorListResource = { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + withSparkUI(appId, Some(attemptId)) { ui => new ExecutorListResource(ui) } } + @Path("applications/{appId}/{attemptId}/allexecutors") + def getAllExecutors( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): AllExecutorListResource = { + withSparkUI(appId, Some(attemptId)) { ui => + new AllExecutorListResource(ui) + } + } @Path("applications/{appId}/stages") def getStages(@PathParam("appId") appId: String): AllStagesResource = { - uiRoot.withSparkUI(appId, None) { ui => + withSparkUI(appId, None) { ui => new AllStagesResource(ui) } } @@ -112,14 +128,14 @@ private[v1] class ApiRootResource extends UIRootFromServletContext { def getStages( @PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): AllStagesResource = { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + withSparkUI(appId, Some(attemptId)) { ui => new AllStagesResource(ui) } } @Path("applications/{appId}/stages/{stageId: \\d+}") def getStage(@PathParam("appId") appId: String): OneStageResource = { - uiRoot.withSparkUI(appId, None) { ui => + withSparkUI(appId, None) { ui => new OneStageResource(ui) } } @@ -128,14 +144,14 @@ private[v1] class ApiRootResource extends UIRootFromServletContext { def getStage( @PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): OneStageResource = { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + withSparkUI(appId, Some(attemptId)) { ui => new OneStageResource(ui) } } @Path("applications/{appId}/storage/rdd") def getRdds(@PathParam("appId") appId: String): AllRDDResource = { - uiRoot.withSparkUI(appId, None) { ui => + withSparkUI(appId, None) { ui => new AllRDDResource(ui) } } @@ -144,14 +160,14 @@ private[v1] class ApiRootResource extends UIRootFromServletContext { def getRdds( @PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): AllRDDResource = { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + withSparkUI(appId, Some(attemptId)) { ui => new AllRDDResource(ui) } } @Path("applications/{appId}/storage/rdd/{rddId: \\d+}") def getRdd(@PathParam("appId") appId: String): OneRDDResource = { - uiRoot.withSparkUI(appId, None) { ui => + withSparkUI(appId, None) { ui => new OneRDDResource(ui) } } @@ -160,7 +176,7 @@ private[v1] class ApiRootResource extends UIRootFromServletContext { def getRdd( @PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): OneRDDResource = { - uiRoot.withSparkUI(appId, Some(attemptId)) { ui => + withSparkUI(appId, Some(attemptId)) { ui => new OneRDDResource(ui) } } @@ -168,14 +184,48 @@ private[v1] class ApiRootResource extends UIRootFromServletContext { @Path("applications/{appId}/logs") def getEventLogs( @PathParam("appId") appId: String): EventLogDownloadResource = { - new EventLogDownloadResource(uiRoot, appId, None) + try { + // withSparkUI will throw NotFoundException if attemptId exists for this application. + // So we need to try again with attempt id "1". + withSparkUI(appId, None) { _ => + new EventLogDownloadResource(uiRoot, appId, None) + } + } catch { + case _: NotFoundException => + withSparkUI(appId, Some("1")) { _ => + new EventLogDownloadResource(uiRoot, appId, None) + } + } } @Path("applications/{appId}/{attemptId}/logs") def getEventLogs( @PathParam("appId") appId: String, @PathParam("attemptId") attemptId: String): EventLogDownloadResource = { - new EventLogDownloadResource(uiRoot, appId, Some(attemptId)) + withSparkUI(appId, Some(attemptId)) { _ => + new EventLogDownloadResource(uiRoot, appId, Some(attemptId)) + } + } + + @Path("version") + def getVersion(): VersionResource = { + new VersionResource(uiRoot) + } + + @Path("applications/{appId}/environment") + def getEnvironment(@PathParam("appId") appId: String): ApplicationEnvironmentResource = { + withSparkUI(appId, None) { ui => + new ApplicationEnvironmentResource(ui) + } + } + + @Path("applications/{appId}/{attemptId}/environment") + def getEnvironment( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): ApplicationEnvironmentResource = { + withSparkUI(appId, Some(attemptId)) { ui => + new ApplicationEnvironmentResource(ui) + } } } @@ -185,12 +235,7 @@ private[spark] object ApiRootResource { val jerseyContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS) jerseyContext.setContextPath("/api") val holder: ServletHolder = new ServletHolder(classOf[ServletContainer]) - holder.setInitParameter("com.sun.jersey.config.property.resourceConfigClass", - "com.sun.jersey.api.core.PackagesResourceConfig") - holder.setInitParameter("com.sun.jersey.config.property.packages", - "org.apache.spark.status.api.v1") - holder.setInitParameter(ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS, - classOf[SecurityFilter].getCanonicalName) + holder.setInitParameter(ServerProperties.PROVIDER_PACKAGES, "org.apache.spark.status.api.v1") UIRootFromServletContext.setUiRoot(jerseyContext, uiRoot) jerseyContext.addServlet(holder, "/*") jerseyContext @@ -199,12 +244,13 @@ private[spark] object ApiRootResource { /** * This trait is shared by the all the root containers for application UI information -- - * the HistoryServer, the Master UI, and the application UI. This provides the common + * the HistoryServer and the application UI. This provides the common * interface needed for them all to expose application info as json. */ private[spark] trait UIRoot { def getSparkUI(appKey: String): Option[SparkUI] def getApplicationInfoList: Iterator[ApplicationInfo] + def getApplicationInfo(appId: String): Option[ApplicationInfo] /** * Write the event logs for the given app to the [[ZipOutputStream]] instance. If attemptId is @@ -216,19 +262,6 @@ private[spark] trait UIRoot { .status(Response.Status.SERVICE_UNAVAILABLE) .build() } - - /** - * Get the spark UI with the given appID, and apply a function - * to it. If there is no such app, throw an appropriate exception - */ - def withSparkUI[T](appId: String, attemptId: Option[String])(f: SparkUI => T): T = { - val appKey = attemptId.map(appId + "/" + _).getOrElse(appId) - getSparkUI(appKey) match { - case Some(ui) => - f(ui) - case None => throw new NotFoundException("no such app: " + appId) - } - } def securityManager: SecurityManager } @@ -245,13 +278,37 @@ private[v1] object UIRootFromServletContext { } } -private[v1] trait UIRootFromServletContext { +private[v1] trait ApiRequestContext { @Context - var servletContext: ServletContext = _ + protected var servletContext: ServletContext = _ + + @Context + protected var httpRequest: HttpServletRequest = _ def uiRoot: UIRoot = UIRootFromServletContext.getUiRoot(servletContext) + + + /** + * Get the spark UI with the given appID, and apply a function + * to it. If there is no such app, throw an appropriate exception + */ + def withSparkUI[T](appId: String, attemptId: Option[String])(f: SparkUI => T): T = { + val appKey = attemptId.map(appId + "/" + _).getOrElse(appId) + uiRoot.getSparkUI(appKey) match { + case Some(ui) => + val user = httpRequest.getRemoteUser() + if (!ui.securityManager.checkUIViewPermissions(user)) { + throw new ForbiddenException(raw"""user "$user" is not authorized""") + } + f(ui) + case None => throw new NotFoundException("no such app: " + appId) + } + } } +private[v1] class ForbiddenException(msg: String) extends WebApplicationException( + Response.status(Response.Status.FORBIDDEN).entity(msg).build()) + private[v1] class NotFoundException(msg: String) extends WebApplicationException( new NoSuchElementException(msg), Response diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationEnvironmentResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationEnvironmentResource.scala new file mode 100644 index 000000000000..739a8aceae86 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationEnvironmentResource.scala @@ -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.status.api.v1 + +import javax.ws.rs._ +import javax.ws.rs.core.MediaType + +import org.apache.spark.ui.SparkUI + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class ApplicationEnvironmentResource(ui: SparkUI) { + + @GET + def getEnvironmentInfo(): ApplicationEnvironmentInfo = { + val listener = ui.environmentListener + listener.synchronized { + val jvmInfo = Map(listener.jvmInformation: _*) + val runtime = new RuntimeInfo( + jvmInfo("Java Version"), + jvmInfo("Java Home"), + jvmInfo("Scala Version")) + + new ApplicationEnvironmentInfo( + runtime, + listener.sparkProperties, + listener.systemProperties, + listener.classpathEntries) + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala index 17b521f3e1d4..f039744e7f67 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala @@ -16,12 +16,11 @@ */ package org.apache.spark.status.api.v1 -import java.util.{Arrays, Date, List => JList} +import java.util.{Date, List => JList} import javax.ws.rs.{DefaultValue, GET, Produces, QueryParam} import javax.ws.rs.core.MediaType import org.apache.spark.deploy.history.ApplicationHistoryInfo -import org.apache.spark.deploy.master.{ApplicationInfo => InternalApplicationInfo} @Produces(Array(MediaType.APPLICATION_JSON)) private[v1] class ApplicationListResource(uiRoot: UIRoot) { @@ -30,30 +29,42 @@ private[v1] class ApplicationListResource(uiRoot: UIRoot) { def appList( @QueryParam("status") status: JList[ApplicationStatus], @DefaultValue("2010-01-01") @QueryParam("minDate") minDate: SimpleDateParam, - @DefaultValue("3000-01-01") @QueryParam("maxDate") maxDate: SimpleDateParam) + @DefaultValue("3000-01-01") @QueryParam("maxDate") maxDate: SimpleDateParam, + @DefaultValue("2010-01-01") @QueryParam("minEndDate") minEndDate: SimpleDateParam, + @DefaultValue("3000-01-01") @QueryParam("maxEndDate") maxEndDate: SimpleDateParam, + @QueryParam("limit") limit: Integer) : Iterator[ApplicationInfo] = { - val allApps = uiRoot.getApplicationInfoList - val adjStatus = { - if (status.isEmpty) { - Arrays.asList(ApplicationStatus.values(): _*) - } else { - status - } - } - val includeCompleted = adjStatus.contains(ApplicationStatus.COMPLETED) - val includeRunning = adjStatus.contains(ApplicationStatus.RUNNING) - allApps.filter { app => + + val numApps = Option(limit).map(_.toInt).getOrElse(Integer.MAX_VALUE) + val includeCompleted = status.isEmpty || status.contains(ApplicationStatus.COMPLETED) + val includeRunning = status.isEmpty || status.contains(ApplicationStatus.RUNNING) + + uiRoot.getApplicationInfoList.filter { app => val anyRunning = app.attempts.exists(!_.completed) - // if any attempt is still running, we consider the app to also still be running - val statusOk = (!anyRunning && includeCompleted) || - (anyRunning && includeRunning) + // if any attempt is still running, we consider the app to also still be running; // keep the app if *any* attempts fall in the right time window - val dateOk = app.attempts.exists { attempt => - attempt.startTime.getTime >= minDate.timestamp && - attempt.startTime.getTime <= maxDate.timestamp + ((!anyRunning && includeCompleted) || (anyRunning && includeRunning)) && + app.attempts.exists { attempt => + isAttemptInRange(attempt, minDate, maxDate, minEndDate, maxEndDate, anyRunning) } - statusOk && dateOk - } + }.take(numApps) + } + + private def isAttemptInRange( + attempt: ApplicationAttemptInfo, + minStartDate: SimpleDateParam, + maxStartDate: SimpleDateParam, + minEndDate: SimpleDateParam, + maxEndDate: SimpleDateParam, + anyRunning: Boolean): Boolean = { + val startTimeOk = attempt.startTime.getTime >= minStartDate.timestamp && + attempt.startTime.getTime <= maxStartDate.timestamp + // If the maxEndDate is in the past, exclude all running apps. + val endTimeOkForRunning = anyRunning && (maxEndDate.timestamp > System.currentTimeMillis()) + val endTimeOkForCompleted = !anyRunning && (attempt.endTime.getTime >= minEndDate.timestamp && + attempt.endTime.getTime <= maxEndDate.timestamp) + val endTimeOk = endTimeOkForRunning || endTimeOkForCompleted + startTimeOk && endTimeOk } } @@ -62,33 +73,27 @@ private[spark] object ApplicationsListResource { new ApplicationInfo( id = app.id, name = app.name, + coresGranted = None, + maxCores = None, + coresPerExecutor = None, + memoryPerExecutorMB = None, attempts = app.attempts.map { internalAttemptInfo => new ApplicationAttemptInfo( attemptId = internalAttemptInfo.attemptId, startTime = new Date(internalAttemptInfo.startTime), endTime = new Date(internalAttemptInfo.endTime), + duration = + if (internalAttemptInfo.endTime > 0) { + internalAttemptInfo.endTime - internalAttemptInfo.startTime + } else { + 0 + }, + lastUpdated = new Date(internalAttemptInfo.lastUpdated), sparkUser = internalAttemptInfo.sparkUser, - completed = internalAttemptInfo.completed + completed = internalAttemptInfo.completed, + appSparkVersion = internalAttemptInfo.appSparkVersion ) } ) } - - def convertApplicationInfo( - internal: InternalApplicationInfo, - completed: Boolean): ApplicationInfo = { - // standalone application info always has just one attempt - new ApplicationInfo( - id = internal.id, - name = internal.desc.name, - attempts = Seq(new ApplicationAttemptInfo( - attemptId = None, - startTime = new Date(internal.startTime), - endTime = new Date(internal.endTime), - sparkUser = internal.desc.user, - completed = completed - )) - ) - } - } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala index 22e21f0c62a2..c84022ddfeef 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/EventLogDownloadResource.scala @@ -23,8 +23,9 @@ import javax.ws.rs.core.{MediaType, Response, StreamingOutput} import scala.util.control.NonFatal -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging @Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) private[v1] class EventLogDownloadResource( diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala index 8ad4656b4dad..2f3b5e984002 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala @@ -1,22 +1,22 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.status.api.v1 -import javax.ws.rs.{GET, PathParam, Produces} +import javax.ws.rs.{GET, Produces} import javax.ws.rs.core.MediaType import org.apache.spark.ui.SparkUI @@ -28,9 +28,13 @@ private[v1] class ExecutorListResource(ui: SparkUI) { @GET def executorList(): Seq[ExecutorSummary] = { val listener = ui.executorsListener - val storageStatusList = listener.storageStatusList - (0 until storageStatusList.size).map { statusId => - ExecutorsPage.getExecInfo(listener, statusId) + listener.synchronized { + // The follow codes should be protected by `listener` to make sure no executors will be + // removed before we query their status. See SPARK-12784. + val storageStatusList = listener.activeStorageStatusList + (0 until storageStatusList.size).map { statusId => + ExecutorsPage.getExecInfo(listener, statusId, isActive = true) + } } } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala index 202a5191ad57..76af33c1a18d 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala @@ -19,8 +19,9 @@ package org.apache.spark.status.api.v1 import java.io.OutputStream import java.lang.annotation.Annotation import java.lang.reflect.Type +import java.nio.charset.StandardCharsets import java.text.SimpleDateFormat -import java.util.{Calendar, SimpleTimeZone} +import java.util.{Calendar, Locale, SimpleTimeZone} import javax.ws.rs.Produces import javax.ws.rs.core.{MediaType, MultivaluedMap} import javax.ws.rs.ext.{MessageBodyWriter, Provider} @@ -68,7 +69,7 @@ private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{ multivaluedMap: MultivaluedMap[String, AnyRef], outputStream: OutputStream): Unit = { t match { - case ErrorWrapper(err) => outputStream.write(err.getBytes("utf-8")) + case ErrorWrapper(err) => outputStream.write(err.getBytes(StandardCharsets.UTF_8)) case _ => mapper.writeValue(outputStream, t) } } @@ -85,7 +86,7 @@ private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{ private[spark] object JacksonMessageWriter { def makeISODateFormat: SimpleDateFormat = { - val iso8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'GMT'") + val iso8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'GMT'", Locale.US) val cal = Calendar.getInstance(new SimpleTimeZone(0, "GMT")) iso8601.setCalendar(cal) iso8601 diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala index b5ef72649e29..18c3e2f40736 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala @@ -16,15 +16,15 @@ */ package org.apache.spark.status.api.v1 +import javax.ws.rs.{GET, PathParam, Produces} import javax.ws.rs.core.MediaType -import javax.ws.rs.{Produces, PathParam, GET} @Produces(Array(MediaType.APPLICATION_JSON)) private[v1] class OneApplicationResource(uiRoot: UIRoot) { @GET def getApp(@PathParam("appId") appId: String): ApplicationInfo = { - val apps = uiRoot.getApplicationInfoList.find { _.id == appId } + val apps = uiRoot.getApplicationInfo(appId) apps.getOrElse(throw new NotFoundException("unknown app: " + appId)) } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala index 6d8a60d480ae..653150385c73 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.status.api.v1 -import javax.ws.rs.{PathParam, GET, Produces} +import javax.ws.rs.{GET, PathParam, Produces} import javax.ws.rs.core.MediaType import org.apache.spark.JobExecutionStatus @@ -30,7 +30,7 @@ private[v1] class OneJobResource(ui: SparkUI) { def oneJob(@PathParam("jobId") jobId: Int): JobData = { val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] = AllJobsResource.getStatusToJobs(ui) - val jobOpt = statusToJobs.map {_._2} .flatten.find { jobInfo => jobInfo.jobId == jobId} + val jobOpt = statusToJobs.flatMap(_._2).find { jobInfo => jobInfo.jobId == jobId} jobOpt.map { job => AllJobsResource.convertJobData(job, ui.jobProgressListener, false) }.getOrElse { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala index dfdc09c6caf3..237aeac18587 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.status.api.v1 -import javax.ws.rs.{PathParam, GET, Produces} +import javax.ws.rs.{GET, PathParam, Produces} import javax.ws.rs.core.MediaType import org.apache.spark.ui.SparkUI diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala index f9812f06cf52..f15073bccced 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala @@ -33,8 +33,9 @@ private[v1] class OneStageResource(ui: SparkUI) { @GET @Path("") def stageData(@PathParam("stageId") stageId: Int): Seq[StageData] = { - withStage(stageId){ stageAttempts => + withStage(stageId) { stageAttempts => stageAttempts.map { stage => + stage.ui.lastUpdateTime = ui.lastUpdateTime AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui, includeDetails = true) } @@ -47,6 +48,7 @@ private[v1] class OneStageResource(ui: SparkUI) { @PathParam("stageId") stageId: Int, @PathParam("stageAttemptId") stageAttemptId: Int): StageData = { withStageAttempt(stageId, stageAttemptId) { stage => + stage.ui.lastUpdateTime = ui.lastUpdateTime AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui, includeDetails = true) } @@ -81,7 +83,8 @@ private[v1] class OneStageResource(ui: SparkUI) { @DefaultValue("20") @QueryParam("length") length: Int, @DefaultValue("ID") @QueryParam("sortBy") sortBy: TaskSorting): Seq[TaskData] = { withStageAttempt(stageId, stageAttemptId) { stage => - val tasks = stage.ui.taskData.values.map{AllStagesResource.convertTaskData}.toIndexedSeq + val tasks = stage.ui.taskData.values + .map{ AllStagesResource.convertTaskData(_, ui.lastUpdateTime)}.toIndexedSeq .sorted(OneStageResource.ordering(sortBy)) tasks.slice(offset, offset + length) } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala index 95fbd96ade5a..1cd37185d660 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala @@ -16,21 +16,19 @@ */ package org.apache.spark.status.api.v1 -import javax.ws.rs.WebApplicationException +import javax.ws.rs.container.{ContainerRequestContext, ContainerRequestFilter} import javax.ws.rs.core.Response +import javax.ws.rs.ext.Provider -import com.sun.jersey.spi.container.{ContainerRequest, ContainerRequestFilter} - -private[v1] class SecurityFilter extends ContainerRequestFilter with UIRootFromServletContext { - def filter(req: ContainerRequest): ContainerRequest = { - val user = Option(req.getUserPrincipal).map { _.getName }.orNull - if (uiRoot.securityManager.checkUIViewPermissions(user)) { - req - } else { - throw new WebApplicationException( +@Provider +private[v1] class SecurityFilter extends ContainerRequestFilter with ApiRequestContext { + override def filter(req: ContainerRequestContext): Unit = { + val user = httpRequest.getRemoteUser() + if (!uiRoot.securityManager.checkUIViewPermissions(user)) { + req.abortWith( Response .status(Response.Status.FORBIDDEN) - .entity(raw"""user "$user"is not authorized""") + .entity(raw"""user "$user" is not authorized""") .build() ) } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala index 0c71cd238222..d8d5e8958b23 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala @@ -17,7 +17,7 @@ package org.apache.spark.status.api.v1 import java.text.{ParseException, SimpleDateFormat} -import java.util.TimeZone +import java.util.{Locale, TimeZone} import javax.ws.rs.WebApplicationException import javax.ws.rs.core.Response import javax.ws.rs.core.Response.Status @@ -25,12 +25,12 @@ import javax.ws.rs.core.Response.Status private[v1] class SimpleDateParam(val originalValue: String) { val timestamp: Long = { - val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSz") + val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSz", Locale.US) try { format.parse(originalValue).getTime() } catch { case _: ParseException => - val gmtDay = new SimpleDateFormat("yyyy-MM-dd") + val gmtDay = new SimpleDateFormat("yyyy-MM-dd", Locale.US) gmtDay.setTimeZone(TimeZone.getTimeZone("GMT")) try { gmtDay.parse(originalValue).getTime() diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/VersionResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/VersionResource.scala new file mode 100644 index 000000000000..673da1ce36b5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/api/v1/VersionResource.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.status.api.v1 + +import javax.ws.rs._ +import javax.ws.rs.core.MediaType + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class VersionResource(ui: UIRoot) { + + @GET + def getVersionInfo(): VersionInfo = new VersionInfo( + org.apache.spark.SPARK_VERSION + ) + +} diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 2bec64f2ef02..05948f266105 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -25,14 +25,25 @@ import org.apache.spark.JobExecutionStatus class ApplicationInfo private[spark]( val id: String, val name: String, + val coresGranted: Option[Int], + val maxCores: Option[Int], + val coresPerExecutor: Option[Int], + val memoryPerExecutorMB: Option[Int], val attempts: Seq[ApplicationAttemptInfo]) class ApplicationAttemptInfo private[spark]( val attemptId: Option[String], val startTime: Date, val endTime: Date, + val lastUpdated: Date, + val duration: Long, val sparkUser: String, - val completed: Boolean = false) + val completed: Boolean = false, + val appSparkVersion: String) { + def getStartTimeEpoch: Long = startTime.getTime + def getEndTimeEpoch: Long = endTime.getTime + def getLastUpdatedEpoch: Long = lastUpdated.getTime +} class ExecutorStageSummary private[spark]( val taskTime : Long, @@ -48,19 +59,31 @@ class ExecutorStageSummary private[spark]( class ExecutorSummary private[spark]( val id: String, val hostPort: String, + val isActive: Boolean, val rddBlocks: Int, val memoryUsed: Long, val diskUsed: Long, + val totalCores: Int, + val maxTasks: Int, val activeTasks: Int, val failedTasks: Int, val completedTasks: Int, val totalTasks: Int, val totalDuration: Long, + val totalGCTime: Long, val totalInputBytes: Long, val totalShuffleRead: Long, val totalShuffleWrite: Long, + val isBlacklisted: Boolean, val maxMemory: Long, - val executorLogs: Map[String, String]) + val executorLogs: Map[String, String], + val memoryMetrics: Option[MemoryMetrics]) + +class MemoryMetrics private[spark]( + val usedOnHeapStorageMemory: Long, + val usedOffHeapStorageMemory: Long, + val totalOnHeapStorageMemory: Long, + val totalOffHeapStorageMemory: Long) class JobData private[spark]( val jobId: Int, @@ -81,8 +104,6 @@ class JobData private[spark]( val numSkippedStages: Int, val numFailedStages: Int) -// Q: should Tachyon size go in here as well? currently the UI only shows it on the overall storage -// page ... does anybody pay attention to it? class RDDStorageInfo private[spark]( val id: Int, val name: String, @@ -98,7 +119,11 @@ class RDDDataDistribution private[spark]( val address: String, val memoryUsed: Long, val memoryRemaining: Long, - val diskUsed: Long) + val diskUsed: Long, + val onHeapMemoryUsed: Option[Long], + val offHeapMemoryUsed: Option[Long], + val onHeapMemoryRemaining: Option[Long], + val offHeapMemoryRemaining: Option[Long]) class RDDPartitionInfo private[spark]( val blockName: String, @@ -111,11 +136,15 @@ class StageData private[spark]( val status: StageStatus, val stageId: Int, val attemptId: Int, - val numActiveTasks: Int , + val numActiveTasks: Int, val numCompleteTasks: Int, val numFailedTasks: Int, val executorRunTime: Long, + val executorCpuTime: Long, + val submissionTime: Option[Date], + val firstTaskLaunchedTime: Option[Date], + val completionTime: Option[Date], val inputBytes: Long, val inputRecords: Long, @@ -141,8 +170,10 @@ class TaskData private[spark]( val index: Int, val attempt: Int, val launchTime: Date, + val duration: Option[Long] = None, val executorId: String, val host: String, + val status: String, val taskLocality: String, val speculative: Boolean, val accumulatorUpdates: Seq[AccumulableInfo], @@ -151,16 +182,18 @@ class TaskData private[spark]( class TaskMetrics private[spark]( val executorDeserializeTime: Long, + val executorDeserializeCpuTime: Long, val executorRunTime: Long, + val executorCpuTime: Long, val resultSize: Long, val jvmGcTime: Long, val resultSerializationTime: Long, val memoryBytesSpilled: Long, val diskBytesSpilled: Long, - val inputMetrics: Option[InputMetrics], - val outputMetrics: Option[OutputMetrics], - val shuffleReadMetrics: Option[ShuffleReadMetrics], - val shuffleWriteMetrics: Option[ShuffleWriteMetrics]) + val inputMetrics: InputMetrics, + val outputMetrics: OutputMetrics, + val shuffleReadMetrics: ShuffleReadMetrics, + val shuffleWriteMetrics: ShuffleWriteMetrics) class InputMetrics private[spark]( val bytesRead: Long, @@ -171,11 +204,12 @@ class OutputMetrics private[spark]( val recordsWritten: Long) class ShuffleReadMetrics private[spark]( - val remoteBlocksFetched: Int, - val localBlocksFetched: Int, + val remoteBlocksFetched: Long, + val localBlocksFetched: Long, val fetchWaitTime: Long, val remoteBytesRead: Long, - val totalBlocksFetched: Int, + val remoteBytesReadToDisk: Long, + val localBytesRead: Long, val recordsRead: Long) class ShuffleWriteMetrics private[spark]( @@ -187,17 +221,19 @@ class TaskMetricDistributions private[spark]( val quantiles: IndexedSeq[Double], val executorDeserializeTime: IndexedSeq[Double], + val executorDeserializeCpuTime: IndexedSeq[Double], val executorRunTime: IndexedSeq[Double], + val executorCpuTime: IndexedSeq[Double], val resultSize: IndexedSeq[Double], val jvmGcTime: IndexedSeq[Double], val resultSerializationTime: IndexedSeq[Double], val memoryBytesSpilled: IndexedSeq[Double], val diskBytesSpilled: IndexedSeq[Double], - val inputMetrics: Option[InputMetricDistributions], - val outputMetrics: Option[OutputMetricDistributions], - val shuffleReadMetrics: Option[ShuffleReadMetricDistributions], - val shuffleWriteMetrics: Option[ShuffleWriteMetricDistributions]) + val inputMetrics: InputMetricDistributions, + val outputMetrics: OutputMetricDistributions, + val shuffleReadMetrics: ShuffleReadMetricDistributions, + val shuffleWriteMetrics: ShuffleWriteMetricDistributions) class InputMetricDistributions private[spark]( val bytesRead: IndexedSeq[Double], @@ -214,6 +250,7 @@ class ShuffleReadMetricDistributions private[spark]( val localBlocksFetched: IndexedSeq[Double], val fetchWaitTime: IndexedSeq[Double], val remoteBytesRead: IndexedSeq[Double], + val remoteBytesReadToDisk: IndexedSeq[Double], val totalBlocksFetched: IndexedSeq[Double]) class ShuffleWriteMetricDistributions private[spark]( @@ -226,3 +263,17 @@ class AccumulableInfo private[spark]( val name: String, val update: Option[String], val value: String) + +class VersionInfo private[spark]( + val spark: String) + +class ApplicationEnvironmentInfo private[spark] ( + val runtime: RuntimeInfo, + val sparkProperties: Seq[(String, String)], + val systemProperties: Seq[(String, String)], + val classpathEntries: Seq[(String, String)]) + +class RuntimeInfo private[spark]( + val javaVersion: String, + val javaHome: String, + val scalaVersion: String) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala deleted file mode 100644 index f6e46ae9a481..000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala +++ /dev/null @@ -1,24 +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.storage - -import org.apache.spark.SparkException - -private[spark] -case class BlockFetchException(messages: String, throwable: Throwable) - extends SparkException(messages, throwable) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala deleted file mode 100644 index 22fdf73e9d1f..000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala +++ /dev/null @@ -1,83 +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.storage - -import java.util.concurrent.ConcurrentHashMap - -private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) { - // To save space, 'pending' and 'failed' are encoded as special sizes: - @volatile var size: Long = BlockInfo.BLOCK_PENDING - private def pending: Boolean = size == BlockInfo.BLOCK_PENDING - private def failed: Boolean = size == BlockInfo.BLOCK_FAILED - private def initThread: Thread = BlockInfo.blockInfoInitThreads.get(this) - - setInitThread() - - private def setInitThread() { - /* Set current thread as init thread - waitForReady will not block this thread - * (in case there is non trivial initialization which ends up calling waitForReady - * as part of initialization itself) */ - BlockInfo.blockInfoInitThreads.put(this, Thread.currentThread()) - } - - /** - * Wait for this BlockInfo to be marked as ready (i.e. block is finished writing). - * Return true if the block is available, false otherwise. - */ - def waitForReady(): Boolean = { - if (pending && initThread != Thread.currentThread()) { - synchronized { - while (pending) { - this.wait() - } - } - } - !failed - } - - /** Mark this BlockInfo as ready (i.e. block is finished writing) */ - def markReady(sizeInBytes: Long) { - require(sizeInBytes >= 0, s"sizeInBytes was negative: $sizeInBytes") - assert(pending) - size = sizeInBytes - BlockInfo.blockInfoInitThreads.remove(this) - synchronized { - this.notifyAll() - } - } - - /** Mark this BlockInfo as ready but failed */ - def markFailure() { - assert(pending) - size = BlockInfo.BLOCK_FAILED - BlockInfo.blockInfoInitThreads.remove(this) - synchronized { - this.notifyAll() - } - } -} - -private object BlockInfo { - /* initThread is logically a BlockInfo field, but we store it here because - * it's only needed while this block is in the 'pending' state and we want - * to minimize BlockInfo's memory footprint. */ - private val blockInfoInitThreads = new ConcurrentHashMap[BlockInfo, Thread] - - private val BLOCK_PENDING: Long = -1L - private val BLOCK_FAILED: Long = -2L -} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala new file mode 100644 index 000000000000..219a0e799cc7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -0,0 +1,446 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import javax.annotation.concurrent.GuardedBy + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.reflect.ClassTag + +import com.google.common.collect.{ConcurrentHashMultiset, ImmutableMultiset} + +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.internal.Logging + + +/** + * Tracks metadata for an individual block. + * + * Instances of this class are _not_ thread-safe and are protected by locks in the + * [[BlockInfoManager]]. + * + * @param level the block's storage level. This is the requested persistence level, not the + * effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this + * does not imply that the block is actually resident in memory). + * @param classTag the block's [[ClassTag]], used to select the serializer + * @param tellMaster whether state changes for this block should be reported to the master. This + * is true for most blocks, but is false for broadcast blocks. + */ +private[storage] class BlockInfo( + val level: StorageLevel, + val classTag: ClassTag[_], + val tellMaster: Boolean) { + + /** + * The size of the block (in bytes) + */ + def size: Long = _size + def size_=(s: Long): Unit = { + _size = s + checkInvariants() + } + private[this] var _size: Long = 0 + + /** + * The number of times that this block has been locked for reading. + */ + def readerCount: Int = _readerCount + def readerCount_=(c: Int): Unit = { + _readerCount = c + checkInvariants() + } + private[this] var _readerCount: Int = 0 + + /** + * The task attempt id of the task which currently holds the write lock for this block, or + * [[BlockInfo.NON_TASK_WRITER]] if the write lock is held by non-task code, or + * [[BlockInfo.NO_WRITER]] if this block is not locked for writing. + */ + def writerTask: Long = _writerTask + def writerTask_=(t: Long): Unit = { + _writerTask = t + checkInvariants() + } + private[this] var _writerTask: Long = BlockInfo.NO_WRITER + + private def checkInvariants(): Unit = { + // A block's reader count must be non-negative: + assert(_readerCount >= 0) + // A block is either locked for reading or for writing, but not for both at the same time: + assert(_readerCount == 0 || _writerTask == BlockInfo.NO_WRITER) + } + + checkInvariants() +} + +private[storage] object BlockInfo { + + /** + * Special task attempt id constant used to mark a block's write lock as being unlocked. + */ + val NO_WRITER: Long = -1 + + /** + * Special task attempt id constant used to mark a block's write lock as being held by + * a non-task thread (e.g. by a driver thread or by unit test code). + */ + val NON_TASK_WRITER: Long = -1024 +} + +/** + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking. + * + * The locking interface exposed by this class is readers-writer lock. Every lock acquisition is + * automatically associated with a running task and locks are automatically released upon task + * completion or failure. + * + * This class is thread-safe. + */ +private[storage] class BlockInfoManager extends Logging { + + private type TaskAttemptId = Long + + /** + * Used to look up metadata for individual blocks. Entries are added to this map via an atomic + * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed + * by [[removeBlock()]]. + */ + @GuardedBy("this") + private[this] val infos = new mutable.HashMap[BlockId, BlockInfo] + + /** + * Tracks the set of blocks that each task has locked for writing. + */ + @GuardedBy("this") + private[this] val writeLocksByTask = + new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]] + with mutable.MultiMap[TaskAttemptId, BlockId] + + /** + * Tracks the set of blocks that each task has locked for reading, along with the number of times + * that a block has been locked (since our read locks are re-entrant). + */ + @GuardedBy("this") + private[this] val readLocksByTask = + new mutable.HashMap[TaskAttemptId, ConcurrentHashMultiset[BlockId]] + + // ---------------------------------------------------------------------------------------------- + + // Initialization for special task attempt ids: + registerTask(BlockInfo.NON_TASK_WRITER) + + // ---------------------------------------------------------------------------------------------- + + /** + * Called at the start of a task in order to register that task with this [[BlockInfoManager]]. + * This must be called prior to calling any other BlockInfoManager methods from that task. + */ + def registerTask(taskAttemptId: TaskAttemptId): Unit = synchronized { + require(!readLocksByTask.contains(taskAttemptId), + s"Task attempt $taskAttemptId is already registered") + readLocksByTask(taskAttemptId) = ConcurrentHashMultiset.create() + } + + /** + * Returns the current task's task attempt id (which uniquely identifies the task), or + * [[BlockInfo.NON_TASK_WRITER]] if called by a non-task thread. + */ + private def currentTaskAttemptId: TaskAttemptId = { + Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(BlockInfo.NON_TASK_WRITER) + } + + /** + * Lock a block for reading and return its metadata. + * + * If another task has already locked this block for reading, then the read lock will be + * immediately granted to the calling task and its lock count will be incremented. + * + * If another task has locked this block for writing, then this call will block until the write + * lock is released or will return immediately if `blocking = false`. + * + * A single task can lock a block multiple times for reading, in which case each lock will need + * to be released separately. + * + * @param blockId the block to lock. + * @param blocking if true (default), this call will block until the lock is acquired. If false, + * this call will return immediately if the lock acquisition fails. + * @return None if the block did not exist or was removed (in which case no lock is held), or + * Some(BlockInfo) (in which case the block is locked for reading). + */ + def lockForReading( + blockId: BlockId, + blocking: Boolean = true): Option[BlockInfo] = synchronized { + logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId") + do { + infos.get(blockId) match { + case None => return None + case Some(info) => + if (info.writerTask == BlockInfo.NO_WRITER) { + info.readerCount += 1 + readLocksByTask(currentTaskAttemptId).add(blockId) + logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId") + return Some(info) + } + } + if (blocking) { + wait() + } + } while (blocking) + None + } + + /** + * Lock a block for writing and return its metadata. + * + * If another task has already locked this block for either reading or writing, then this call + * will block until the other locks are released or will return immediately if `blocking = false`. + * + * @param blockId the block to lock. + * @param blocking if true (default), this call will block until the lock is acquired. If false, + * this call will return immediately if the lock acquisition fails. + * @return None if the block did not exist or was removed (in which case no lock is held), or + * Some(BlockInfo) (in which case the block is locked for writing). + */ + def lockForWriting( + blockId: BlockId, + blocking: Boolean = true): Option[BlockInfo] = synchronized { + logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId") + do { + infos.get(blockId) match { + case None => return None + case Some(info) => + if (info.writerTask == BlockInfo.NO_WRITER && info.readerCount == 0) { + info.writerTask = currentTaskAttemptId + writeLocksByTask.addBinding(currentTaskAttemptId, blockId) + logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId") + return Some(info) + } + } + if (blocking) { + wait() + } + } while (blocking) + None + } + + /** + * Throws an exception if the current task does not hold a write lock on the given block. + * Otherwise, returns the block's BlockInfo. + */ + def assertBlockIsLockedForWriting(blockId: BlockId): BlockInfo = synchronized { + infos.get(blockId) match { + case Some(info) => + if (info.writerTask != currentTaskAttemptId) { + throw new SparkException( + s"Task $currentTaskAttemptId has not locked block $blockId for writing") + } else { + info + } + case None => + throw new SparkException(s"Block $blockId does not exist") + } + } + + /** + * Get a block's metadata without acquiring any locks. This method is only exposed for use by + * [[BlockManager.getStatus()]] and should not be called by other code outside of this class. + */ + private[storage] def get(blockId: BlockId): Option[BlockInfo] = synchronized { + infos.get(blockId) + } + + /** + * Downgrades an exclusive write lock to a shared read lock. + */ + def downgradeLock(blockId: BlockId): Unit = synchronized { + logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId") + val info = get(blockId).get + require(info.writerTask == currentTaskAttemptId, + s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold on" + + s" block $blockId") + unlock(blockId) + val lockOutcome = lockForReading(blockId, blocking = false) + assert(lockOutcome.isDefined) + } + + /** + * Release a lock on the given block. + * In case a TaskContext is not propagated properly to all child threads for the task, we fail to + * get the TID from TaskContext, so we have to explicitly pass the TID value to release the lock. + * + * See SPARK-18406 for more discussion of this issue. + */ + def unlock(blockId: BlockId, taskAttemptId: Option[TaskAttemptId] = None): Unit = synchronized { + val taskId = taskAttemptId.getOrElse(currentTaskAttemptId) + logTrace(s"Task $taskId releasing lock for $blockId") + val info = get(blockId).getOrElse { + throw new IllegalStateException(s"Block $blockId not found") + } + if (info.writerTask != BlockInfo.NO_WRITER) { + info.writerTask = BlockInfo.NO_WRITER + writeLocksByTask.removeBinding(taskId, blockId) + } else { + assert(info.readerCount > 0, s"Block $blockId is not locked for reading") + info.readerCount -= 1 + val countsForTask = readLocksByTask(taskId) + val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1 + assert(newPinCountForTask >= 0, + s"Task $taskId release lock on block $blockId more times than it acquired it") + } + notifyAll() + } + + /** + * Attempt to acquire the appropriate lock for writing a new block. + * + * This enforces the first-writer-wins semantics. If we are the first to write the block, + * then just go ahead and acquire the write lock. Otherwise, if another thread is already + * writing the block, then we wait for the write to finish before acquiring the read lock. + * + * @return true if the block did not already exist, false otherwise. If this returns false, then + * a read lock on the existing block will be held. If this returns true, a write lock on + * the new block will be held. + */ + def lockNewBlockForWriting( + blockId: BlockId, + newBlockInfo: BlockInfo): Boolean = synchronized { + logTrace(s"Task $currentTaskAttemptId trying to put $blockId") + lockForReading(blockId) match { + case Some(info) => + // Block already exists. This could happen if another thread races with us to compute + // the same block. In this case, just keep the read lock and return. + false + case None => + // Block does not yet exist or is removed, so we are free to acquire the write lock + infos(blockId) = newBlockInfo + lockForWriting(blockId) + true + } + } + + /** + * Release all lock held by the given task, clearing that task's pin bookkeeping + * structures and updating the global pin counts. This method should be called at the + * end of a task (either by a task completion handler or in `TaskRunner.run()`). + * + * @return the ids of blocks whose pins were released + */ + def releaseAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = synchronized { + val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]() + + val readLocks = readLocksByTask.remove(taskAttemptId).getOrElse(ImmutableMultiset.of[BlockId]()) + val writeLocks = writeLocksByTask.remove(taskAttemptId).getOrElse(Seq.empty) + + for (blockId <- writeLocks) { + infos.get(blockId).foreach { info => + assert(info.writerTask == taskAttemptId) + info.writerTask = BlockInfo.NO_WRITER + } + blocksWithReleasedLocks += blockId + } + + readLocks.entrySet().iterator().asScala.foreach { entry => + val blockId = entry.getElement + val lockCount = entry.getCount + blocksWithReleasedLocks += blockId + get(blockId).foreach { info => + info.readerCount -= lockCount + assert(info.readerCount >= 0) + } + } + + notifyAll() + + blocksWithReleasedLocks + } + + /** Returns the number of locks held by the given task. Used only for testing. */ + private[storage] def getTaskLockCount(taskAttemptId: TaskAttemptId): Int = { + readLocksByTask.get(taskAttemptId).map(_.size()).getOrElse(0) + + writeLocksByTask.get(taskAttemptId).map(_.size).getOrElse(0) + } + + /** + * Returns the number of blocks tracked. + */ + def size: Int = synchronized { + infos.size + } + + /** + * Return the number of map entries in this pin counter's internal data structures. + * This is used in unit tests in order to detect memory leaks. + */ + private[storage] def getNumberOfMapEntries: Long = synchronized { + size + + readLocksByTask.size + + readLocksByTask.map(_._2.size()).sum + + writeLocksByTask.size + + writeLocksByTask.map(_._2.size).sum + } + + /** + * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries + * in this iterator are mutable and thus may reflect blocks that are deleted while the iterator + * is being traversed. + */ + def entries: Iterator[(BlockId, BlockInfo)] = synchronized { + infos.toArray.toIterator + } + + /** + * Removes the given block and releases the write lock on it. + * + * This can only be called while holding a write lock on the given block. + */ + def removeBlock(blockId: BlockId): Unit = synchronized { + logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId") + infos.get(blockId) match { + case Some(blockInfo) => + if (blockInfo.writerTask != currentTaskAttemptId) { + throw new IllegalStateException( + s"Task $currentTaskAttemptId called remove() on block $blockId without a write lock") + } else { + infos.remove(blockId) + blockInfo.readerCount = 0 + blockInfo.writerTask = BlockInfo.NO_WRITER + writeLocksByTask.removeBinding(currentTaskAttemptId, blockId) + } + case None => + throw new IllegalArgumentException( + s"Task $currentTaskAttemptId called remove() on non-existent block $blockId") + } + notifyAll() + } + + /** + * Delete all state. Called during shutdown. + */ + def clear(): Unit = synchronized { + infos.valuesIterator.foreach { blockInfo => + blockInfo.readerCount = 0 + blockInfo.writerTask = BlockInfo.NO_WRITER + } + infos.clear() + readLocksByTask.clear() + writeLocksByTask.clear() + notifyAll() + } + +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index c374b9376622..a98083df5bd8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -18,35 +18,36 @@ package org.apache.spark.storage import java.io._ -import java.nio.{ByteBuffer, MappedByteBuffer} +import java.nio.ByteBuffer +import java.nio.channels.Channels -import scala.collection.mutable.{ArrayBuffer, HashMap} -import scala.concurrent.{ExecutionContext, Await, Future} +import scala.collection.mutable +import scala.collection.mutable.HashMap +import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ -import scala.util.control.NonFatal +import scala.reflect.ClassTag import scala.util.Random +import scala.util.control.NonFatal -import sun.nio.ch.DirectBuffer +import com.codahale.metrics.{MetricRegistry, MetricSet} import org.apache.spark._ import org.apache.spark.executor.{DataReadMethod, ShuffleWriteMetrics} -import org.apache.spark.io.CompressionCodec -import org.apache.spark.memory.MemoryManager +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.memory.{MemoryManager, MemoryMode} +import org.apache.spark.metrics.source.Source import org.apache.spark.network._ -import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.ExternalShuffleClient import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo import org.apache.spark.rpc.RpcEnv -import org.apache.spark.serializer.{SerializerInstance, Serializer} +import org.apache.spark.serializer.{SerializerInstance, SerializerManager} import org.apache.spark.shuffle.ShuffleManager -import org.apache.spark.shuffle.hash.HashShuffleManager +import org.apache.spark.storage.memory._ +import org.apache.spark.unsafe.Platform import org.apache.spark.util._ - -private[spark] sealed trait BlockValues -private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues -private[spark] case class IteratorValues(iterator: Iterator[Any]) extends BlockValues -private[spark] case class ArrayValues(buffer: Array[Any]) extends BlockValues +import org.apache.spark.util.io.ChunkedByteBuffer /* Class for returning a fetched block and associated metrics. */ private[spark] class BlockResult( @@ -54,51 +55,103 @@ private[spark] class BlockResult( val readMethod: DataReadMethod.Value, val bytes: Long) +/** + * Abstracts away how blocks are stored and provides different ways to read the underlying block + * data. Callers should call [[dispose()]] when they're done with the block. + */ +private[spark] trait BlockData { + + def toInputStream(): InputStream + + /** + * Returns a Netty-friendly wrapper for the block's data. + * + * Please see `ManagedBuffer.convertToNetty()` for more details. + */ + def toNetty(): Object + + def toChunkedByteBuffer(allocator: Int => ByteBuffer): ChunkedByteBuffer + + def toByteBuffer(): ByteBuffer + + def size: Long + + def dispose(): Unit + +} + +private[spark] class ByteBufferBlockData( + val buffer: ChunkedByteBuffer, + val shouldDispose: Boolean) extends BlockData { + + override def toInputStream(): InputStream = buffer.toInputStream(dispose = false) + + override def toNetty(): Object = buffer.toNetty + + override def toChunkedByteBuffer(allocator: Int => ByteBuffer): ChunkedByteBuffer = { + buffer.copy(allocator) + } + + override def toByteBuffer(): ByteBuffer = buffer.toByteBuffer + + override def size: Long = buffer.size + + override def dispose(): Unit = { + if (shouldDispose) { + buffer.dispose() + } + } + +} + /** * Manager running on every node (driver and executors) which provides interfaces for putting and * retrieving blocks both locally and remotely into various stores (memory, disk, and off-heap). * - * Note that #initialize() must be called before the BlockManager is usable. + * Note that [[initialize()]] must be called before the BlockManager is usable. */ private[spark] class BlockManager( executorId: String, rpcEnv: RpcEnv, val master: BlockManagerMaster, - defaultSerializer: Serializer, + val serializerManager: SerializerManager, val conf: SparkConf, memoryManager: MemoryManager, mapOutputTracker: MapOutputTracker, shuffleManager: ShuffleManager, - blockTransferService: BlockTransferService, + val blockTransferService: BlockTransferService, securityManager: SecurityManager, numUsableCores: Int) - extends BlockDataManager with Logging { + extends BlockDataManager with BlockEvictionHandler with Logging { + + private[spark] val externalShuffleServiceEnabled = + conf.getBoolean("spark.shuffle.service.enabled", false) - val diskBlockManager = new DiskBlockManager(this, conf) + val diskBlockManager = { + // Only perform cleanup if an external service is not serving our shuffle files. + val deleteFilesOnStop = + !externalShuffleServiceEnabled || executorId == SparkContext.DRIVER_IDENTIFIER + new DiskBlockManager(conf, deleteFilesOnStop) + } - private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] + // Visible for testing + private[storage] val blockInfoManager = new BlockInfoManager private val futureExecutionContext = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("block-manager-future", 128)) // Actual storage of where blocks are kept - private var externalBlockStoreInitialized = false - private[spark] val memoryStore = new MemoryStore(this, memoryManager) - private[spark] val diskStore = new DiskStore(this, diskBlockManager) - private[spark] lazy val externalBlockStore: ExternalBlockStore = { - externalBlockStoreInitialized = true - new ExternalBlockStore(this, executorId) - } + private[spark] val memoryStore = + new MemoryStore(conf, blockInfoManager, serializerManager, memoryManager, this) + private[spark] val diskStore = new DiskStore(conf, diskBlockManager, securityManager) memoryManager.setMemoryStore(memoryStore) - // Note: depending on the memory manager, `maxStorageMemory` may actually vary over time. + // Note: depending on the memory manager, `maxMemory` may actually vary over time. // However, since we use this only for reporting and logging, what we actually want here is - // the absolute maximum value that `maxStorageMemory` can ever possibly reach. We may need + // the absolute maximum value that `maxMemory` can ever possibly reach. We may need // to revisit whether reporting this value as the "max" is intuitive to the user. - private val maxMemory = memoryManager.maxStorageMemory - - private[spark] - val externalShuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false) + private val maxOnHeapMemory = memoryManager.maxOnHeapStorageMemory + private val maxOffHeapMemory = memoryManager.maxOffHeapStorageMemory // Port used by the external shuffle service. In Yarn mode, this may be already be // set through the Hadoop configuration as the server is launched in the Yarn NM. @@ -123,21 +176,16 @@ private[spark] class BlockManager( // Client to read other executors' shuffle files. This is either an external service, or just the // standard BlockTransferService to directly connect to other Executors. private[spark] val shuffleClient = if (externalShuffleServiceEnabled) { - val transConf = SparkTransportConf.fromSparkConf(conf, numUsableCores) - new ExternalShuffleClient(transConf, securityManager, securityManager.isAuthenticationEnabled(), - securityManager.isSaslEncryptionEnabled()) + val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores) + new ExternalShuffleClient(transConf, securityManager, + securityManager.isAuthenticationEnabled(), conf.get(config.SHUFFLE_REGISTRATION_TIMEOUT)) } else { blockTransferService } - // Whether to compress broadcast variables that are stored - private val compressBroadcast = conf.getBoolean("spark.broadcast.compress", true) - // Whether to compress shuffle output that are stored - private val compressShuffle = conf.getBoolean("spark.shuffle.compress", true) - // Whether to compress RDD partitions that are stored serialized - private val compressRdds = conf.getBoolean("spark.rdd.compress", false) - // Whether to compress shuffle output temporarily spilled to disk - private val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", true) + // Max number of failures before this block manager refreshes the block locations from the driver + private val maxFailuresBeforeLocationRefresh = + conf.getInt("spark.block.failures.beforeLocationRefresh", 5) private val slaveEndpoint = rpcEnv.setupEndpoint( "BlockManagerEndpoint" + BlockManager.ID_GENERATOR.next, @@ -148,22 +196,12 @@ private[spark] class BlockManager( private var asyncReregisterTask: Future[Unit] = null private val asyncReregisterLock = new Object - private val metadataCleaner = new MetadataCleaner( - MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks, conf) - private val broadcastCleaner = new MetadataCleaner( - MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks, conf) - // Field related to peer block managers that are necessary for block replication @volatile private var cachedPeers: Seq[BlockManagerId] = _ private val peerFetchLock = new Object private var lastPeerFetchTime = 0L - /* The compression codec to use. Note that the "lazy" val is necessary because we want to delay - * the initialization of the compression codec until it is first used. The reason is that a Spark - * program could be using a user-defined codec in a third party jar, which is loaded in - * Executor.updateDependencies. When the BlockManager is initialized, user level jars hasn't been - * loaded yet. */ - private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec(conf) + private var blockReplicationPolicy: BlockReplicationPolicy = _ /** * Initializes the BlockManager with the given appId. This is not performed in the constructor as @@ -178,8 +216,25 @@ private[spark] class BlockManager( blockTransferService.init(this) shuffleClient.init(appId) - blockManagerId = BlockManagerId( - executorId, blockTransferService.hostName, blockTransferService.port) + blockReplicationPolicy = { + val priorityClass = conf.get( + "spark.storage.replication.policy", classOf[RandomBlockReplicationPolicy].getName) + val clazz = Utils.classForName(priorityClass) + val ret = clazz.newInstance.asInstanceOf[BlockReplicationPolicy] + logInfo(s"Using $priorityClass for block replication policy") + ret + } + + val id = + BlockManagerId(executorId, blockTransferService.hostName, blockTransferService.port, None) + + val idFromMaster = master.registerBlockManager( + id, + maxOnHeapMemory, + maxOffHeapMemory, + slaveEndpoint) + + blockManagerId = if (idFromMaster != null) idFromMaster else id shuffleServerId = if (externalShuffleServiceEnabled) { logInfo(s"external shuffle service port = $externalShuffleServicePort") @@ -188,12 +243,22 @@ private[spark] class BlockManager( blockManagerId } - master.registerBlockManager(blockManagerId, maxMemory, slaveEndpoint) - // Register Executors' configuration with the local shuffle service, if one should exist. if (externalShuffleServiceEnabled && !blockManagerId.isDriver) { registerWithExternalShuffleServer() } + + logInfo(s"Initialized BlockManager: $blockManagerId") + } + + def shuffleMetricsSource: Source = { + import BlockManager._ + + if (externalShuffleServiceEnabled) { + new ShuffleMetricsSource("ExternalShuffle", shuffleClient.shuffleMetrics()) + } else { + new ShuffleMetricsSource("NettyBlockTransfer", shuffleClient.shuffleMetrics()) + } } private def registerWithExternalShuffleServer() { @@ -203,7 +268,7 @@ private[spark] class BlockManager( diskBlockManager.subDirsPerLocalDir, shuffleManager.getClass.getName) - val MAX_ATTEMPTS = 3 + val MAX_ATTEMPTS = conf.get(config.SHUFFLE_REGISTRATION_MAX_ATTEMPTS) val SLEEP_TIME_SECS = 5 for (i <- 1 to MAX_ATTEMPTS) { @@ -217,6 +282,9 @@ private[spark] class BlockManager( logError(s"Failed to connect to external shuffle server, will retry ${MAX_ATTEMPTS - i}" + s" more times after waiting $SLEEP_TIME_SECS seconds...", e) Thread.sleep(SLEEP_TIME_SECS * 1000) + case NonFatal(e) => + throw new SparkException("Unable to register with external shuffle server due to : " + + e.getMessage, e) } } } @@ -232,10 +300,10 @@ private[spark] class BlockManager( * will be made then. */ private def reportAllBlocks(): Unit = { - logInfo(s"Reporting ${blockInfo.size} blocks to the master.") - for ((blockId, info) <- blockInfo) { + logInfo(s"Reporting ${blockInfoManager.size} blocks to the master.") + for ((blockId, info) <- blockInfoManager.entries) { val status = getCurrentBlockStatus(blockId, info) - if (!tryToReportBlockStatus(blockId, info, status)) { + if (info.tellMaster && !tryToReportBlockStatus(blockId, status)) { logError(s"Failed to report $blockId to master; giving up.") return } @@ -250,8 +318,8 @@ private[spark] class BlockManager( */ def reregister(): Unit = { // TODO: We might need to rate limit re-registering. - logInfo("BlockManager re-registering with master") - master.registerBlockManager(blockManagerId, maxMemory, slaveEndpoint) + logInfo(s"BlockManager $blockManagerId re-registering with master") + master.registerBlockManager(blockManagerId, maxOnHeapMemory, maxOffHeapMemory, slaveEndpoint) reportAllBlocks() } @@ -279,7 +347,12 @@ private[spark] class BlockManager( def waitForAsyncReregister(): Unit = { val task = asyncReregisterTask if (task != null) { - Await.ready(task, Duration.Inf) + try { + ThreadUtils.awaitReady(task, Duration.Inf) + } catch { + case NonFatal(t) => + throw new Exception("Error occurred while waiting for async. reregistration", t) + } } } @@ -291,34 +364,42 @@ private[spark] class BlockManager( if (blockId.isShuffle) { shuffleManager.shuffleBlockResolver.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) } else { - val blockBytesOpt = doGetLocal(blockId, asBlockResult = false) - .asInstanceOf[Option[ByteBuffer]] - if (blockBytesOpt.isDefined) { - val buffer = blockBytesOpt.get - new NioManagedBuffer(buffer) - } else { - throw new BlockNotFoundException(blockId.toString) + getLocalBytes(blockId) match { + case Some(blockData) => + new BlockManagerManagedBuffer(blockInfoManager, blockId, blockData, true) + case None => + // If this block manager receives a request for a block that it doesn't have then it's + // likely that the master has outdated block statuses for this block. Therefore, we send + // an RPC so that this block is marked as being unavailable from this block manager. + reportBlockStatus(blockId, BlockStatus.empty) + throw new BlockNotFoundException(blockId.toString) } } } /** * Put the block locally, using the given storage level. + * + * '''Important!''' Callers must not mutate or release the data buffer underlying `bytes`. Doing + * so may corrupt or change the data stored by the `BlockManager`. */ - override def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Unit = { - putBytes(blockId, data.nioByteBuffer(), level) + override def putBlockData( + blockId: BlockId, + data: ManagedBuffer, + level: StorageLevel, + classTag: ClassTag[_]): Boolean = { + putBytes(blockId, new ChunkedByteBuffer(data.nioByteBuffer()), level)(classTag) } /** * Get the BlockStatus for the block identified by the given ID, if it exists. - * NOTE: This is mainly for testing, and it doesn't fetch information from external block store. + * NOTE: This is mainly for testing. */ def getStatus(blockId: BlockId): Option[BlockStatus] = { - blockInfo.get(blockId).map { info => + blockInfoManager.get(blockId).map { info => val memSize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L val diskSize = if (diskStore.contains(blockId)) diskStore.getSize(blockId) else 0L - // Assume that block is not in external block store - BlockStatus(info.level, memSize, diskSize, 0L) + BlockStatus(info.level, memSize = memSize, diskSize = diskSize) } } @@ -328,7 +409,12 @@ private[spark] class BlockManager( * may not know of). */ def getMatchingBlockIds(filter: BlockId => Boolean): Seq[BlockId] = { - (blockInfo.keys ++ diskBlockManager.getAllBlocks()).filter(filter).toSeq + // The `toArray` is necessary here in order to force the list to be materialized so that we + // don't try to serialize a lazy iterator when responding to client requests. + (blockInfoManager.entries.map(_._1) ++ diskBlockManager.getAllBlocks()) + .filter(filter) + .toArray + .toSeq } /** @@ -342,10 +428,9 @@ private[spark] class BlockManager( */ private def reportBlockStatus( blockId: BlockId, - info: BlockInfo, status: BlockStatus, droppedMemorySize: Long = 0L): Unit = { - val needReregister = !tryToReportBlockStatus(blockId, info, status, droppedMemorySize) + val needReregister = !tryToReportBlockStatus(blockId, status, droppedMemorySize) if (needReregister) { logInfo(s"Got told to re-register updating block $blockId") // Re-registering will report our new block for free. @@ -361,19 +446,12 @@ private[spark] class BlockManager( */ private def tryToReportBlockStatus( blockId: BlockId, - info: BlockInfo, status: BlockStatus, droppedMemorySize: Long = 0L): Boolean = { - if (info.tellMaster) { - val storageLevel = status.storageLevel - val inMemSize = Math.max(status.memSize, droppedMemorySize) - val inExternalBlockStoreSize = status.externalBlockStoreSize - val onDiskSize = status.diskSize - master.updateBlockInfo( - blockManagerId, blockId, storageLevel, inMemSize, onDiskSize, inExternalBlockStoreSize) - } else { - true - } + val storageLevel = status.storageLevel + val inMemSize = Math.max(status.memSize, droppedMemorySize) + val onDiskSize = status.diskSize + master.updateBlockInfo(blockManagerId, blockId, storageLevel, inMemSize, onDiskSize) } /** @@ -385,20 +463,21 @@ private[spark] class BlockManager( info.synchronized { info.level match { case null => - BlockStatus(StorageLevel.NONE, 0L, 0L, 0L) + BlockStatus.empty case level => val inMem = level.useMemory && memoryStore.contains(blockId) - val inExternalBlockStore = level.useOffHeap && externalBlockStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) val deserialized = if (inMem) level.deserialized else false - val replication = if (inMem || inExternalBlockStore || onDisk) level.replication else 1 - val storageLevel = - StorageLevel(onDisk, inMem, inExternalBlockStore, deserialized, replication) + val replication = if (inMem || onDisk) level.replication else 1 + val storageLevel = StorageLevel( + useDisk = onDisk, + useMemory = inMem, + useOffHeap = level.useOffHeap, + deserialized = deserialized, + replication = replication) val memSize = if (inMem) memoryStore.getSize(blockId) else 0L - val externalBlockStoreSize = - if (inExternalBlockStore) externalBlockStore.getSize(blockId) else 0L val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L - BlockStatus(storageLevel, memSize, diskSize, externalBlockStoreSize) + BlockStatus(storageLevel, memSize, diskSize) } } } @@ -414,17 +493,72 @@ private[spark] class BlockManager( } /** - * Get block from local block manager. + * Cleanup code run in response to a failed local read. + * Must be called while holding a read lock on the block. + */ + private def handleLocalReadFailure(blockId: BlockId): Nothing = { + releaseLock(blockId) + // Remove the missing block so that its unavailability is reported to the driver + removeBlock(blockId) + throw new SparkException(s"Block $blockId was not found even though it's read-locked") + } + + /** + * Get block from local block manager as an iterator of Java objects. */ - def getLocal(blockId: BlockId): Option[BlockResult] = { + def getLocalValues(blockId: BlockId): Option[BlockResult] = { logDebug(s"Getting local block $blockId") - doGetLocal(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] + blockInfoManager.lockForReading(blockId) match { + case None => + logDebug(s"Block $blockId was not found") + None + case Some(info) => + val level = info.level + logDebug(s"Level for block $blockId is $level") + val taskAttemptId = Option(TaskContext.get()).map(_.taskAttemptId()) + if (level.useMemory && memoryStore.contains(blockId)) { + val iter: Iterator[Any] = if (level.deserialized) { + memoryStore.getValues(blockId).get + } else { + serializerManager.dataDeserializeStream( + blockId, memoryStore.getBytes(blockId).get.toInputStream())(info.classTag) + } + // We need to capture the current taskId in case the iterator completion is triggered + // from a different thread which does not have TaskContext set; see SPARK-18406 for + // discussion. + val ci = CompletionIterator[Any, Iterator[Any]](iter, { + releaseLock(blockId, taskAttemptId) + }) + Some(new BlockResult(ci, DataReadMethod.Memory, info.size)) + } else if (level.useDisk && diskStore.contains(blockId)) { + val diskData = diskStore.getBytes(blockId) + val iterToReturn: Iterator[Any] = { + if (level.deserialized) { + val diskValues = serializerManager.dataDeserializeStream( + blockId, + diskData.toInputStream())(info.classTag) + maybeCacheDiskValuesInMemory(info, blockId, level, diskValues) + } else { + val stream = maybeCacheDiskBytesInMemory(info, blockId, level, diskData) + .map { _.toInputStream(dispose = false) } + .getOrElse { diskData.toInputStream() } + serializerManager.dataDeserializeStream(blockId, stream)(info.classTag) + } + } + val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, { + releaseLockAndDispose(blockId, diskData, taskAttemptId) + }) + Some(new BlockResult(ci, DataReadMethod.Disk, info.size)) + } else { + handleLocalReadFailure(blockId) + } + } } /** * Get block from the local block manager as serialized bytes. */ - def getLocalBytes(blockId: BlockId): Option[ByteBuffer] = { + def getLocalBytes(blockId: BlockId): Option[BlockData] = { logDebug(s"Getting local block $blockId as bytes") // As an optimization for map output fetches, if the block is for a shuffle, return it // without acquiring a lock; the disk store never deletes (recent) items so this should work @@ -432,186 +566,136 @@ private[spark] class BlockManager( val shuffleBlockResolver = shuffleManager.shuffleBlockResolver // TODO: This should gracefully handle case where local block is not available. Currently // downstream code will throw an exception. - Option( + val buf = new ChunkedByteBuffer( shuffleBlockResolver.getBlockData(blockId.asInstanceOf[ShuffleBlockId]).nioByteBuffer()) + Some(new ByteBufferBlockData(buf, true)) } else { - doGetLocal(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] + blockInfoManager.lockForReading(blockId).map { info => doGetLocalBytes(blockId, info) } } } - private def doGetLocal(blockId: BlockId, asBlockResult: Boolean): Option[Any] = { - val info = blockInfo.get(blockId).orNull - if (info != null) { - info.synchronized { - // Double check to make sure the block is still there. There is a small chance that the - // block has been removed by removeBlock (which also synchronizes on the blockInfo object). - // Note that this only checks metadata tracking. If user intentionally deleted the block - // on disk or from off heap storage without using removeBlock, this conditional check will - // still pass but eventually we will get an exception because we can't find the block. - if (blockInfo.get(blockId).isEmpty) { - logWarning(s"Block $blockId had been removed") - return None - } - - // If another thread is writing the block, wait for it to become ready. - if (!info.waitForReady()) { - // If we get here, the block write failed. - logWarning(s"Block $blockId was marked as failure.") - return None - } - - val level = info.level - logDebug(s"Level for block $blockId is $level") - - // Look for the block in memory - if (level.useMemory) { - logDebug(s"Getting block $blockId from memory") - val result = if (asBlockResult) { - memoryStore.getValues(blockId).map(new BlockResult(_, DataReadMethod.Memory, info.size)) - } else { - memoryStore.getBytes(blockId) - } - result match { - case Some(values) => - return result - case None => - logDebug(s"Block $blockId not found in memory") - } - } - - // Look for the block in external block store - if (level.useOffHeap) { - logDebug(s"Getting block $blockId from ExternalBlockStore") - if (externalBlockStore.contains(blockId)) { - val result = if (asBlockResult) { - externalBlockStore.getValues(blockId) - .map(new BlockResult(_, DataReadMethod.Memory, info.size)) - } else { - externalBlockStore.getBytes(blockId) - } - result match { - case Some(values) => - return result - case None => - logDebug(s"Block $blockId not found in ExternalBlockStore") - } - } - } - - // Look for block on disk, potentially storing it back in memory if required - if (level.useDisk) { - logDebug(s"Getting block $blockId from disk") - val bytes: ByteBuffer = diskStore.getBytes(blockId) match { - case Some(b) => b - case None => - throw new BlockException( - blockId, s"Block $blockId not found on disk, though it should be") - } - assert(0 == bytes.position()) - - if (!level.useMemory) { - // If the block shouldn't be stored in memory, we can just return it - if (asBlockResult) { - return Some(new BlockResult(dataDeserialize(blockId, bytes), DataReadMethod.Disk, - info.size)) - } else { - return Some(bytes) - } - } else { - // Otherwise, we also have to store something in the memory store - if (!level.deserialized || !asBlockResult) { - /* We'll store the bytes in memory if the block's storage level includes - * "memory serialized", or if it should be cached as objects in memory - * but we only requested its serialized bytes. */ - memoryStore.putBytes(blockId, bytes.limit, () => { - // https://issues.apache.org/jira/browse/SPARK-6076 - // If the file size is bigger than the free memory, OOM will happen. So if we cannot - // put it into MemoryStore, copyForMemory should not be created. That's why this - // action is put into a `() => ByteBuffer` and created lazily. - val copyForMemory = ByteBuffer.allocate(bytes.limit) - copyForMemory.put(bytes) - }) - bytes.rewind() - } - if (!asBlockResult) { - return Some(bytes) - } else { - val values = dataDeserialize(blockId, bytes) - if (level.deserialized) { - // Cache the values before returning them - val putResult = memoryStore.putIterator( - blockId, values, level, returnValues = true, allowPersistToDisk = false) - // The put may or may not have succeeded, depending on whether there was enough - // space to unroll the block. Either way, the put here should return an iterator. - putResult.data match { - case Left(it) => - return Some(new BlockResult(it, DataReadMethod.Disk, info.size)) - case _ => - // This only happens if we dropped the values back to disk (which is never) - throw new SparkException("Memory store did not return an iterator!") - } - } else { - return Some(new BlockResult(values, DataReadMethod.Disk, info.size)) - } - } - } - } + /** + * Get block from the local block manager as serialized bytes. + * + * Must be called while holding a read lock on the block. + * Releases the read lock upon exception; keeps the read lock upon successful return. + */ + private def doGetLocalBytes(blockId: BlockId, info: BlockInfo): BlockData = { + val level = info.level + logDebug(s"Level for block $blockId is $level") + // In order, try to read the serialized bytes from memory, then from disk, then fall back to + // serializing in-memory objects, and, finally, throw an exception if the block does not exist. + if (level.deserialized) { + // Try to avoid expensive serialization by reading a pre-serialized copy from disk: + if (level.useDisk && diskStore.contains(blockId)) { + // Note: we purposely do not try to put the block back into memory here. Since this branch + // handles deserialized blocks, this block may only be cached in memory as objects, not + // serialized bytes. Because the caller only requested bytes, it doesn't make sense to + // cache the block's deserialized objects since that caching may not have a payoff. + diskStore.getBytes(blockId) + } else if (level.useMemory && memoryStore.contains(blockId)) { + // The block was not found on disk, so serialize an in-memory copy: + new ByteBufferBlockData(serializerManager.dataSerializeWithExplicitClassTag( + blockId, memoryStore.getValues(blockId).get, info.classTag), true) + } else { + handleLocalReadFailure(blockId) + } + } else { // storage level is serialized + if (level.useMemory && memoryStore.contains(blockId)) { + new ByteBufferBlockData(memoryStore.getBytes(blockId).get, false) + } else if (level.useDisk && diskStore.contains(blockId)) { + val diskData = diskStore.getBytes(blockId) + maybeCacheDiskBytesInMemory(info, blockId, level, diskData) + .map(new ByteBufferBlockData(_, false)) + .getOrElse(diskData) + } else { + handleLocalReadFailure(blockId) } - } else { - logDebug(s"Block $blockId not registered locally") } - None } /** * Get block from remote block managers. + * + * This does not acquire a lock on this block in this JVM. */ - def getRemote(blockId: BlockId): Option[BlockResult] = { - logDebug(s"Getting remote block $blockId") - doGetRemote(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] + private def getRemoteValues[T: ClassTag](blockId: BlockId): Option[BlockResult] = { + val ct = implicitly[ClassTag[T]] + getRemoteBytes(blockId).map { data => + val values = + serializerManager.dataDeserializeStream(blockId, data.toInputStream(dispose = true))(ct) + new BlockResult(values, DataReadMethod.Network, data.size) + } } /** - * Get block from remote block managers as serialized bytes. + * Return a list of locations for the given block, prioritizing the local machine since + * multiple block managers can share the same host, followed by hosts on the same rack. */ - def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { - logDebug(s"Getting remote block $blockId as bytes") - doGetRemote(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] + private def getLocations(blockId: BlockId): Seq[BlockManagerId] = { + val locs = Random.shuffle(master.getLocations(blockId)) + val (preferredLocs, otherLocs) = locs.partition { loc => blockManagerId.host == loc.host } + blockManagerId.topologyInfo match { + case None => preferredLocs ++ otherLocs + case Some(_) => + val (sameRackLocs, differentRackLocs) = otherLocs.partition { + loc => blockManagerId.topologyInfo == loc.topologyInfo + } + preferredLocs ++ sameRackLocs ++ differentRackLocs + } } - private def doGetRemote(blockId: BlockId, asBlockResult: Boolean): Option[Any] = { + /** + * Get block from remote block managers as serialized bytes. + */ + def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = { + logDebug(s"Getting remote block $blockId") require(blockId != null, "BlockId is null") - val locations = Random.shuffle(master.getLocations(blockId)) - var numFetchFailures = 0 - for (loc <- locations) { + var runningFailureCount = 0 + var totalFailureCount = 0 + val locations = getLocations(blockId) + val maxFetchFailures = locations.size + var locationIterator = locations.iterator + while (locationIterator.hasNext) { + val loc = locationIterator.next() logDebug(s"Getting remote block $blockId from $loc") val data = try { blockTransferService.fetchBlockSync( loc.host, loc.port, loc.executorId, blockId.toString).nioByteBuffer() } catch { case NonFatal(e) => - numFetchFailures += 1 - if (numFetchFailures == locations.size) { - // An exception is thrown while fetching this block from all locations - throw new BlockFetchException(s"Failed to fetch block from" + - s" ${locations.size} locations. Most recent failure cause:", e) - } else { - // This location failed, so we retry fetch from a different one by returning null here - logWarning(s"Failed to fetch remote block $blockId " + - s"from $loc (failed attempt $numFetchFailures)", e) - null + runningFailureCount += 1 + totalFailureCount += 1 + + if (totalFailureCount >= maxFetchFailures) { + // Give up trying anymore locations. Either we've tried all of the original locations, + // or we've refreshed the list of locations from the master, and have still + // hit failures after trying locations from the refreshed list. + logWarning(s"Failed to fetch block after $totalFailureCount fetch failures. " + + s"Most recent failure cause:", e) + return None + } + + logWarning(s"Failed to fetch remote block $blockId " + + s"from $loc (failed attempt $runningFailureCount)", e) + + // If there is a large number of executors then locations list can contain a + // large number of stale entries causing a large number of retries that may + // take a significant amount of time. To get rid of these stale entries + // we refresh the block locations after a certain number of fetch failures + if (runningFailureCount >= maxFailuresBeforeLocationRefresh) { + locationIterator = getLocations(blockId).iterator + logDebug(s"Refreshed locations from the driver " + + s"after ${runningFailureCount} fetch failures.") + runningFailureCount = 0 } + + // This location failed, so we retry fetch from a different one by returning null here + null } if (data != null) { - if (asBlockResult) { - return Some(new BlockResult( - dataDeserialize(blockId, data), - DataReadMethod.Network, - data.limit())) - } else { - return Some(data) - } + return Some(new ChunkedByteBuffer(data)) } logDebug(s"The value of block $blockId is null") } @@ -621,14 +705,18 @@ private[spark] class BlockManager( /** * Get a block from the block manager (either local or remote). + * + * This acquires a read lock on the block if the block was stored locally and does not acquire + * any locks if the block was fetched from a remote block manager. The read lock will + * automatically be freed once the result's `data` iterator is fully consumed. */ - def get(blockId: BlockId): Option[BlockResult] = { - val local = getLocal(blockId) + def get[T: ClassTag](blockId: BlockId): Option[BlockResult] = { + val local = getLocalValues(blockId) if (local.isDefined) { logInfo(s"Found block $blockId locally") return local } - val remote = getRemote(blockId) + val remote = getRemoteValues[T](blockId) if (remote.isDefined) { logInfo(s"Found block $blockId remotely") return remote @@ -636,14 +724,101 @@ private[spark] class BlockManager( None } - def putIterator( + /** + * Downgrades an exclusive write lock to a shared read lock. + */ + def downgradeLock(blockId: BlockId): Unit = { + blockInfoManager.downgradeLock(blockId) + } + + /** + * Release a lock on the given block with explicit TID. + * The param `taskAttemptId` should be passed in case we can't get the correct TID from + * TaskContext, for example, the input iterator of a cached RDD iterates to the end in a child + * thread. + */ + def releaseLock(blockId: BlockId, taskAttemptId: Option[Long] = None): Unit = { + blockInfoManager.unlock(blockId, taskAttemptId) + } + + /** + * Registers a task with the BlockManager in order to initialize per-task bookkeeping structures. + */ + def registerTask(taskAttemptId: Long): Unit = { + blockInfoManager.registerTask(taskAttemptId) + } + + /** + * Release all locks for the given task. + * + * @return the blocks whose locks were released. + */ + def releaseAllLocksForTask(taskAttemptId: Long): Seq[BlockId] = { + blockInfoManager.releaseAllLocksForTask(taskAttemptId) + } + + /** + * Retrieve the given block if it exists, otherwise call the provided `makeIterator` method + * to compute the block, persist it, and return its values. + * + * @return either a BlockResult if the block was successfully cached, or an iterator if the block + * could not be cached. + */ + def getOrElseUpdate[T]( blockId: BlockId, - values: Iterator[Any], level: StorageLevel, - tellMaster: Boolean = true, - effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { + classTag: ClassTag[T], + makeIterator: () => Iterator[T]): Either[BlockResult, Iterator[T]] = { + // Attempt to read the block from local or remote storage. If it's present, then we don't need + // to go through the local-get-or-put path. + get[T](blockId)(classTag) match { + case Some(block) => + return Left(block) + case _ => + // Need to compute the block. + } + // Initially we hold no locks on this block. + doPutIterator(blockId, makeIterator, level, classTag, keepReadLock = true) match { + case None => + // doPut() didn't hand work back to us, so the block already existed or was successfully + // stored. Therefore, we now hold a read lock on the block. + val blockResult = getLocalValues(blockId).getOrElse { + // Since we held a read lock between the doPut() and get() calls, the block should not + // have been evicted, so get() not returning the block indicates some internal error. + releaseLock(blockId) + throw new SparkException(s"get() failed for block $blockId even though we held a lock") + } + // We already hold a read lock on the block from the doPut() call and getLocalValues() + // acquires the lock again, so we need to call releaseLock() here so that the net number + // of lock acquisitions is 1 (since the caller will only call release() once). + releaseLock(blockId) + Left(blockResult) + case Some(iter) => + // The put failed, likely because the data was too large to fit in memory and could not be + // dropped to disk. Therefore, we need to pass the input iterator back to the caller so + // that they can decide what to do with the values (e.g. process them without caching). + Right(iter) + } + } + + /** + * @return true if the block was stored or false if an error occurred. + */ + def putIterator[T: ClassTag]( + blockId: BlockId, + values: Iterator[T], + level: StorageLevel, + tellMaster: Boolean = true): Boolean = { require(values != null, "Values is null") - doPut(blockId, IteratorValues(values), level, tellMaster, effectiveStorageLevel) + doPutIterator(blockId, () => values, level, implicitly[ClassTag[T]], tellMaster) match { + case None => + true + case Some(iter) => + // Caller doesn't care about the iterator values, so we can close the iterator here + // to free resources earlier + iter.close() + false + } } /** @@ -657,224 +832,395 @@ private[spark] class BlockManager( serializerInstance: SerializerInstance, bufferSize: Int, writeMetrics: ShuffleWriteMetrics): DiskBlockObjectWriter = { - val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) val syncWrites = conf.getBoolean("spark.shuffle.sync", false) - new DiskBlockObjectWriter(file, serializerInstance, bufferSize, compressStream, - syncWrites, writeMetrics) + new DiskBlockObjectWriter(file, serializerManager, serializerInstance, bufferSize, + syncWrites, writeMetrics, blockId) } /** - * Put a new block of values to the block manager. - * Return a list of blocks updated as a result of this put. + * Put a new block of serialized bytes to the block manager. + * + * '''Important!''' Callers must not mutate or release the data buffer underlying `bytes`. Doing + * so may corrupt or change the data stored by the `BlockManager`. + * + * @return true if the block was stored or false if an error occurred. */ - def putArray( + def putBytes[T: ClassTag]( blockId: BlockId, - values: Array[Any], + bytes: ChunkedByteBuffer, level: StorageLevel, - tellMaster: Boolean = true, - effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { - require(values != null, "Values is null") - doPut(blockId, ArrayValues(values), level, tellMaster, effectiveStorageLevel) + tellMaster: Boolean = true): Boolean = { + require(bytes != null, "Bytes is null") + doPutBytes(blockId, bytes, level, implicitly[ClassTag[T]], tellMaster) } /** - * Put a new block of serialized bytes to the block manager. - * Return a list of blocks updated as a result of this put. + * Put the given bytes according to the given level in one of the block stores, replicating + * the values if necessary. + * + * If the block already exists, this method will not overwrite it. + * + * '''Important!''' Callers must not mutate or release the data buffer underlying `bytes`. Doing + * so may corrupt or change the data stored by the `BlockManager`. + * + * @param keepReadLock if true, this method will hold the read lock when it returns (even if the + * block already exists). If false, this method will hold no locks when it + * returns. + * @return true if the block was already present or if the put succeeded, false otherwise. */ - def putBytes( + private def doPutBytes[T]( blockId: BlockId, - bytes: ByteBuffer, + bytes: ChunkedByteBuffer, level: StorageLevel, + classTag: ClassTag[T], tellMaster: Boolean = true, - effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { - require(bytes != null, "Bytes is null") - doPut(blockId, ByteBufferValues(bytes), level, tellMaster, effectiveStorageLevel) + keepReadLock: Boolean = false): Boolean = { + doPut(blockId, level, classTag, tellMaster = tellMaster, keepReadLock = keepReadLock) { info => + val startTimeMs = System.currentTimeMillis + // Since we're storing bytes, initiate the replication before storing them locally. + // This is faster as data is already serialized and ready to send. + val replicationFuture = if (level.replication > 1) { + Future { + // This is a blocking action and should run in futureExecutionContext which is a cached + // thread pool. The ByteBufferBlockData wrapper is not disposed of to avoid releasing + // buffers that are owned by the caller. + replicate(blockId, new ByteBufferBlockData(bytes, false), level, classTag) + }(futureExecutionContext) + } else { + null + } + + val size = bytes.size + + if (level.useMemory) { + // Put it in memory first, even if it also has useDisk set to true; + // We will drop it to disk later if the memory store can't hold it. + val putSucceeded = if (level.deserialized) { + val values = + serializerManager.dataDeserializeStream(blockId, bytes.toInputStream())(classTag) + memoryStore.putIteratorAsValues(blockId, values, classTag) match { + case Right(_) => true + case Left(iter) => + // If putting deserialized values in memory failed, we will put the bytes directly to + // disk, so we don't need this iterator and can close it to free resources earlier. + iter.close() + false + } + } else { + val memoryMode = level.memoryMode + memoryStore.putBytes(blockId, size, memoryMode, () => { + if (memoryMode == MemoryMode.OFF_HEAP && + bytes.chunks.exists(buffer => !buffer.isDirect)) { + bytes.copy(Platform.allocateDirectBuffer) + } else { + bytes + } + }) + } + if (!putSucceeded && level.useDisk) { + logWarning(s"Persisting block $blockId to disk instead.") + diskStore.putBytes(blockId, bytes) + } + } else if (level.useDisk) { + diskStore.putBytes(blockId, bytes) + } + + val putBlockStatus = getCurrentBlockStatus(blockId, info) + val blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid + if (blockWasSuccessfullyStored) { + // Now that the block is in either the memory or disk store, + // tell the master about it. + info.size = size + if (tellMaster && info.tellMaster) { + reportBlockStatus(blockId, putBlockStatus) + } + addUpdatedBlockStatusToTaskMetrics(blockId, putBlockStatus) + } + logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) + if (level.replication > 1) { + // Wait for asynchronous replication to finish + try { + ThreadUtils.awaitReady(replicationFuture, Duration.Inf) + } catch { + case NonFatal(t) => + throw new Exception("Error occurred while waiting for replication to finish", t) + } + } + if (blockWasSuccessfullyStored) { + None + } else { + Some(bytes) + } + }.isEmpty } /** - * Put the given block according to the given level in one of the block stores, replicating - * the values if necessary. + * Helper method used to abstract common code from [[doPutBytes()]] and [[doPutIterator()]]. * - * The effective storage level refers to the level according to which the block will actually be - * handled. This allows the caller to specify an alternate behavior of doPut while preserving - * the original level specified by the user. + * @param putBody a function which attempts the actual put() and returns None on success + * or Some on failure. */ - private def doPut( + private def doPut[T]( blockId: BlockId, - data: BlockValues, level: StorageLevel, - tellMaster: Boolean = true, - effectiveStorageLevel: Option[StorageLevel] = None) - : Seq[(BlockId, BlockStatus)] = { + classTag: ClassTag[_], + tellMaster: Boolean, + keepReadLock: Boolean)(putBody: BlockInfo => Option[T]): Option[T] = { require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") - effectiveStorageLevel.foreach { level => - require(level != null && level.isValid, "Effective StorageLevel is null or invalid") - } - - // Return value - val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - /* Remember the block's storage level so that we can correctly drop it to disk if it needs - * to be dropped right after it got put into memory. Note, however, that other threads will - * not be able to get() this block until we call markReady on its BlockInfo. */ val putBlockInfo = { - val tinfo = new BlockInfo(level, tellMaster) - // Do atomically ! - val oldBlockOpt = blockInfo.putIfAbsent(blockId, tinfo) - if (oldBlockOpt.isDefined) { - if (oldBlockOpt.get.waitForReady()) { - logWarning(s"Block $blockId already exists on this machine; not re-adding it") - return updatedBlocks - } - // TODO: So the block info exists - but previous attempt to load it (?) failed. - // What do we do now ? Retry on it ? - oldBlockOpt.get + val newInfo = new BlockInfo(level, classTag, tellMaster) + if (blockInfoManager.lockNewBlockForWriting(blockId, newInfo)) { + newInfo } else { - tinfo + logWarning(s"Block $blockId already exists on this machine; not re-adding it") + if (!keepReadLock) { + // lockNewBlockForWriting returned a read lock on the existing block, so we must free it: + releaseLock(blockId) + } + return None } } val startTimeMs = System.currentTimeMillis - - /* If we're storing values and we need to replicate the data, we'll want access to the values, - * but because our put will read the whole iterator, there will be no values left. For the - * case where the put serializes data, we'll remember the bytes, above; but for the case where - * it doesn't, such as deserialized storage, let's rely on the put returning an Iterator. */ - var valuesAfterPut: Iterator[Any] = null - - // Ditto for the bytes after the put - var bytesAfterPut: ByteBuffer = null - - // Size of the block in bytes - var size = 0L - - // The level we actually use to put the block - val putLevel = effectiveStorageLevel.getOrElse(level) - - // If we're storing bytes, then initiate the replication before storing them locally. - // This is faster as data is already serialized and ready to send. - val replicationFuture = data match { - case b: ByteBufferValues if putLevel.replication > 1 => - // Duplicate doesn't copy the bytes, but just creates a wrapper - val bufferView = b.buffer.duplicate() - Future { - // This is a blocking action and should run in futureExecutionContext which is a cached - // thread pool - replicate(blockId, bufferView, putLevel) - }(futureExecutionContext) - case _ => null + var exceptionWasThrown: Boolean = true + val result: Option[T] = try { + val res = putBody(putBlockInfo) + exceptionWasThrown = false + if (res.isEmpty) { + // the block was successfully stored + if (keepReadLock) { + blockInfoManager.downgradeLock(blockId) + } else { + blockInfoManager.unlock(blockId) + } + } else { + removeBlockInternal(blockId, tellMaster = false) + logWarning(s"Putting block $blockId failed") + } + res + } catch { + // Since removeBlockInternal may throw exception, + // we should print exception first to show root cause. + case NonFatal(e) => + logWarning(s"Putting block $blockId failed due to exception $e.") + throw e + } finally { + // This cleanup is performed in a finally block rather than a `catch` to avoid having to + // catch and properly re-throw InterruptedException. + if (exceptionWasThrown) { + // If an exception was thrown then it's possible that the code in `putBody` has already + // notified the master about the availability of this block, so we need to send an update + // to remove this block location. + removeBlockInternal(blockId, tellMaster = tellMaster) + // The `putBody` code may have also added a new block status to TaskMetrics, so we need + // to cancel that out by overwriting it with an empty block status. We only do this if + // the finally block was entered via an exception because doing this unconditionally would + // cause us to send empty block statuses for every block that failed to be cached due to + // a memory shortage (which is an expected failure, unlike an uncaught exception). + addUpdatedBlockStatusToTaskMetrics(blockId, BlockStatus.empty) + } } - - putBlockInfo.synchronized { - logTrace("Put for block %s took %s to get into synchronized block" + if (level.replication > 1) { + logDebug("Putting block %s with replication took %s" .format(blockId, Utils.getUsedTimeMs(startTimeMs))) + } else { + logDebug("Putting block %s without replication took %s" + .format(blockId, Utils.getUsedTimeMs(startTimeMs))) + } + result + } - var marked = false - try { - // returnValues - Whether to return the values put - // blockStore - The type of storage to put these values into - val (returnValues, blockStore: BlockStore) = { - if (putLevel.useMemory) { - // Put it in memory first, even if it also has useDisk set to true; - // We will drop it to disk later if the memory store can't hold it. - (true, memoryStore) - } else if (putLevel.useOffHeap) { - // Use external block store - (false, externalBlockStore) - } else if (putLevel.useDisk) { - // Don't get back the bytes from put unless we replicate them - (putLevel.replication > 1, diskStore) - } else { - assert(putLevel == StorageLevel.NONE) - throw new BlockException( - blockId, s"Attempted to put block $blockId without specifying storage level!") + /** + * Put the given block according to the given level in one of the block stores, replicating + * the values if necessary. + * + * If the block already exists, this method will not overwrite it. + * + * @param keepReadLock if true, this method will hold the read lock when it returns (even if the + * block already exists). If false, this method will hold no locks when it + * returns. + * @return None if the block was already present or if the put succeeded, or Some(iterator) + * if the put failed. + */ + private def doPutIterator[T]( + blockId: BlockId, + iterator: () => Iterator[T], + level: StorageLevel, + classTag: ClassTag[T], + tellMaster: Boolean = true, + keepReadLock: Boolean = false): Option[PartiallyUnrolledIterator[T]] = { + doPut(blockId, level, classTag, tellMaster = tellMaster, keepReadLock = keepReadLock) { info => + val startTimeMs = System.currentTimeMillis + var iteratorFromFailedMemoryStorePut: Option[PartiallyUnrolledIterator[T]] = None + // Size of the block in bytes + var size = 0L + if (level.useMemory) { + // Put it in memory first, even if it also has useDisk set to true; + // We will drop it to disk later if the memory store can't hold it. + if (level.deserialized) { + memoryStore.putIteratorAsValues(blockId, iterator(), classTag) match { + case Right(s) => + size = s + case Left(iter) => + // Not enough space to unroll this block; drop to disk if applicable + if (level.useDisk) { + logWarning(s"Persisting block $blockId to disk instead.") + diskStore.put(blockId) { channel => + val out = Channels.newOutputStream(channel) + serializerManager.dataSerializeStream(blockId, out, iter)(classTag) + } + size = diskStore.getSize(blockId) + } else { + iteratorFromFailedMemoryStorePut = Some(iter) + } + } + } else { // !level.deserialized + memoryStore.putIteratorAsBytes(blockId, iterator(), classTag, level.memoryMode) match { + case Right(s) => + size = s + case Left(partiallySerializedValues) => + // Not enough space to unroll this block; drop to disk if applicable + if (level.useDisk) { + logWarning(s"Persisting block $blockId to disk instead.") + diskStore.put(blockId) { channel => + val out = Channels.newOutputStream(channel) + partiallySerializedValues.finishWritingToStream(out) + } + size = diskStore.getSize(blockId) + } else { + iteratorFromFailedMemoryStorePut = Some(partiallySerializedValues.valuesIterator) + } } } - // Actually put the values - val result = data match { - case IteratorValues(iterator) => - blockStore.putIterator(blockId, iterator, putLevel, returnValues) - case ArrayValues(array) => - blockStore.putArray(blockId, array, putLevel, returnValues) - case ByteBufferValues(bytes) => - bytes.rewind() - blockStore.putBytes(blockId, bytes, putLevel) - } - size = result.size - result.data match { - case Left (newIterator) if putLevel.useMemory => valuesAfterPut = newIterator - case Right (newBytes) => bytesAfterPut = newBytes - case _ => + } else if (level.useDisk) { + diskStore.put(blockId) { channel => + val out = Channels.newOutputStream(channel) + serializerManager.dataSerializeStream(blockId, out, iterator())(classTag) } + size = diskStore.getSize(blockId) + } - // Keep track of which blocks are dropped from memory - if (putLevel.useMemory) { - result.droppedBlocks.foreach { updatedBlocks += _ } + val putBlockStatus = getCurrentBlockStatus(blockId, info) + val blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid + if (blockWasSuccessfullyStored) { + // Now that the block is in either the memory or disk store, tell the master about it. + info.size = size + if (tellMaster && info.tellMaster) { + reportBlockStatus(blockId, putBlockStatus) } - - val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) - if (putBlockStatus.storageLevel != StorageLevel.NONE) { - // Now that the block is in either the memory, externalBlockStore, or disk store, - // let other threads read it, and tell the master about it. - marked = true - putBlockInfo.markReady(size) - if (tellMaster) { - reportBlockStatus(blockId, putBlockInfo, putBlockStatus) + addUpdatedBlockStatusToTaskMetrics(blockId, putBlockStatus) + logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) + if (level.replication > 1) { + val remoteStartTime = System.currentTimeMillis + val bytesToReplicate = doGetLocalBytes(blockId, info) + // [SPARK-16550] Erase the typed classTag when using default serialization, since + // NettyBlockRpcServer crashes when deserializing repl-defined classes. + // TODO(ekl) remove this once the classloader issue on the remote end is fixed. + val remoteClassTag = if (!serializerManager.canUseKryo(classTag)) { + scala.reflect.classTag[Any] + } else { + classTag } - updatedBlocks += ((blockId, putBlockStatus)) - } - } finally { - // If we failed in putting the block to memory/disk, notify other possible readers - // that it has failed, and then remove it from the block info map. - if (!marked) { - // Note that the remove must happen before markFailure otherwise another thread - // could've inserted a new BlockInfo before we remove it. - blockInfo.remove(blockId) - putBlockInfo.markFailure() - logWarning(s"Putting block $blockId failed") + try { + replicate(blockId, bytesToReplicate, level, remoteClassTag) + } finally { + bytesToReplicate.dispose() + } + logDebug("Put block %s remotely took %s" + .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) } } + assert(blockWasSuccessfullyStored == iteratorFromFailedMemoryStorePut.isEmpty) + iteratorFromFailedMemoryStorePut } - logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) - - // Either we're storing bytes and we asynchronously started replication, or we're storing - // values and need to serialize and replicate them now: - if (putLevel.replication > 1) { - data match { - case ByteBufferValues(bytes) => - if (replicationFuture != null) { - Await.ready(replicationFuture, Duration.Inf) + } + + /** + * Attempts to cache spilled bytes read from disk into the MemoryStore in order to speed up + * subsequent reads. This method requires the caller to hold a read lock on the block. + * + * @return a copy of the bytes from the memory store if the put succeeded, otherwise None. + * If this returns bytes from the memory store then the original disk store bytes will + * automatically be disposed and the caller should not continue to use them. Otherwise, + * if this returns None then the original disk store bytes will be unaffected. + */ + private def maybeCacheDiskBytesInMemory( + blockInfo: BlockInfo, + blockId: BlockId, + level: StorageLevel, + diskData: BlockData): Option[ChunkedByteBuffer] = { + require(!level.deserialized) + if (level.useMemory) { + // Synchronize on blockInfo to guard against a race condition where two readers both try to + // put values read from disk into the MemoryStore. + blockInfo.synchronized { + if (memoryStore.contains(blockId)) { + diskData.dispose() + Some(memoryStore.getBytes(blockId).get) + } else { + val allocator = level.memoryMode match { + case MemoryMode.ON_HEAP => ByteBuffer.allocate _ + case MemoryMode.OFF_HEAP => Platform.allocateDirectBuffer _ } - case _ => - val remoteStartTime = System.currentTimeMillis - // Serialize the block if not already done - if (bytesAfterPut == null) { - if (valuesAfterPut == null) { - throw new SparkException( - "Underlying put returned neither an Iterator nor bytes! This shouldn't happen.") - } - bytesAfterPut = dataSerialize(blockId, valuesAfterPut) + val putSucceeded = memoryStore.putBytes(blockId, diskData.size, level.memoryMode, () => { + // https://issues.apache.org/jira/browse/SPARK-6076 + // If the file size is bigger than the free memory, OOM will happen. So if we + // cannot put it into MemoryStore, copyForMemory should not be created. That's why + // this action is put into a `() => ChunkedByteBuffer` and created lazily. + diskData.toChunkedByteBuffer(allocator) + }) + if (putSucceeded) { + diskData.dispose() + Some(memoryStore.getBytes(blockId).get) + } else { + None } - replicate(blockId, bytesAfterPut, putLevel) - logDebug("Put block %s remotely took %s" - .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) + } } + } else { + None } + } - BlockManager.dispose(bytesAfterPut) - - if (putLevel.replication > 1) { - logDebug("Putting block %s with replication took %s" - .format(blockId, Utils.getUsedTimeMs(startTimeMs))) + /** + * Attempts to cache spilled values read from disk into the MemoryStore in order to speed up + * subsequent reads. This method requires the caller to hold a read lock on the block. + * + * @return a copy of the iterator. The original iterator passed this method should no longer + * be used after this method returns. + */ + private def maybeCacheDiskValuesInMemory[T]( + blockInfo: BlockInfo, + blockId: BlockId, + level: StorageLevel, + diskIterator: Iterator[T]): Iterator[T] = { + require(level.deserialized) + val classTag = blockInfo.classTag.asInstanceOf[ClassTag[T]] + if (level.useMemory) { + // Synchronize on blockInfo to guard against a race condition where two readers both try to + // put values read from disk into the MemoryStore. + blockInfo.synchronized { + if (memoryStore.contains(blockId)) { + // Note: if we had a means to discard the disk iterator, we would do that here. + memoryStore.getValues(blockId).get + } else { + memoryStore.putIteratorAsValues(blockId, diskIterator, classTag) match { + case Left(iter) => + // The memory store put() failed, so it returned the iterator back to us: + iter + case Right(_) => + // The put() succeeded, so we can read the values back: + memoryStore.getValues(blockId).get + } + } + }.asInstanceOf[Iterator[T]] } else { - logDebug("Putting block %s without replication took %s" - .format(blockId, Utils.getUsedTimeMs(startTimeMs))) + diskIterator } - - updatedBlocks } /** @@ -894,199 +1240,208 @@ private[spark] class BlockManager( } /** - * Replicate block to another node. Not that this is a blocking call that returns after + * Called for pro-active replenishment of blocks lost due to executor failures + * + * @param blockId blockId being replicate + * @param existingReplicas existing block managers that have a replica + * @param maxReplicas maximum replicas needed + */ + def replicateBlock( + blockId: BlockId, + existingReplicas: Set[BlockManagerId], + maxReplicas: Int): Unit = { + logInfo(s"Using $blockManagerId to pro-actively replicate $blockId") + blockInfoManager.lockForReading(blockId).foreach { info => + val data = doGetLocalBytes(blockId, info) + val storageLevel = StorageLevel( + useDisk = info.level.useDisk, + useMemory = info.level.useMemory, + useOffHeap = info.level.useOffHeap, + deserialized = info.level.deserialized, + replication = maxReplicas) + // we know we are called as a result of an executor removal, so we refresh peer cache + // this way, we won't try to replicate to a missing executor with a stale reference + getPeers(forceFetch = true) + try { + replicate(blockId, data, storageLevel, info.classTag, existingReplicas) + } finally { + logDebug(s"Releasing lock for $blockId") + releaseLockAndDispose(blockId, data) + } + } + } + + /** + * Replicate block to another node. Note that this is a blocking call that returns after * the block has been replicated. */ - private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel): Unit = { + private def replicate( + blockId: BlockId, + data: BlockData, + level: StorageLevel, + classTag: ClassTag[_], + existingReplicas: Set[BlockManagerId] = Set.empty): Unit = { + val maxReplicationFailures = conf.getInt("spark.storage.maxReplicationFailures", 1) - val numPeersToReplicateTo = level.replication - 1 - val peersForReplication = new ArrayBuffer[BlockManagerId] - val peersReplicatedTo = new ArrayBuffer[BlockManagerId] - val peersFailedToReplicateTo = new ArrayBuffer[BlockManagerId] val tLevel = StorageLevel( - level.useDisk, level.useMemory, level.useOffHeap, level.deserialized, 1) - val startTime = System.currentTimeMillis - val random = new Random(blockId.hashCode) - - var replicationFailed = false - var failures = 0 - var done = false - - // Get cached list of peers - peersForReplication ++= getPeers(forceFetch = false) - - // Get a random peer. Note that this selection of a peer is deterministic on the block id. - // So assuming the list of peers does not change and no replication failures, - // if there are multiple attempts in the same node to replicate the same block, - // the same set of peers will be selected. - def getRandomPeer(): Option[BlockManagerId] = { - // If replication had failed, then force update the cached list of peers and remove the peers - // that have been already used - if (replicationFailed) { - peersForReplication.clear() - peersForReplication ++= getPeers(forceFetch = true) - peersForReplication --= peersReplicatedTo - peersForReplication --= peersFailedToReplicateTo - } - if (!peersForReplication.isEmpty) { - Some(peersForReplication(random.nextInt(peersForReplication.size))) - } else { - None - } - } + useDisk = level.useDisk, + useMemory = level.useMemory, + useOffHeap = level.useOffHeap, + deserialized = level.deserialized, + replication = 1) - // One by one choose a random peer and try uploading the block to it - // If replication fails (e.g., target peer is down), force the list of cached peers - // to be re-fetched from driver and then pick another random peer for replication. Also - // temporarily black list the peer for which replication failed. - // - // This selection of a peer and replication is continued in a loop until one of the - // following 3 conditions is fulfilled: - // (i) specified number of peers have been replicated to - // (ii) too many failures in replicating to peers - // (iii) no peer left to replicate to - // - while (!done) { - getRandomPeer() match { - case Some(peer) => - try { - val onePeerStartTime = System.currentTimeMillis - data.rewind() - logTrace(s"Trying to replicate $blockId of ${data.limit()} bytes to $peer") - blockTransferService.uploadBlockSync( - peer.host, peer.port, peer.executorId, blockId, new NioManagedBuffer(data), tLevel) - logTrace(s"Replicated $blockId of ${data.limit()} bytes to $peer in %s ms" - .format(System.currentTimeMillis - onePeerStartTime)) - peersReplicatedTo += peer - peersForReplication -= peer - replicationFailed = false - if (peersReplicatedTo.size == numPeersToReplicateTo) { - done = true // specified number of peers have been replicated to - } - } catch { - case e: Exception => - logWarning(s"Failed to replicate $blockId to $peer, failure #$failures", e) - failures += 1 - replicationFailed = true - peersFailedToReplicateTo += peer - if (failures > maxReplicationFailures) { // too many failures in replcating to peers - done = true - } + val numPeersToReplicateTo = level.replication - 1 + val startTime = System.nanoTime + + val peersReplicatedTo = mutable.HashSet.empty ++ existingReplicas + val peersFailedToReplicateTo = mutable.HashSet.empty[BlockManagerId] + var numFailures = 0 + + val initialPeers = getPeers(false).filterNot(existingReplicas.contains) + + var peersForReplication = blockReplicationPolicy.prioritize( + blockManagerId, + initialPeers, + peersReplicatedTo, + blockId, + numPeersToReplicateTo) + + while(numFailures <= maxReplicationFailures && + !peersForReplication.isEmpty && + peersReplicatedTo.size < numPeersToReplicateTo) { + val peer = peersForReplication.head + try { + val onePeerStartTime = System.nanoTime + logTrace(s"Trying to replicate $blockId of ${data.size} bytes to $peer") + blockTransferService.uploadBlockSync( + peer.host, + peer.port, + peer.executorId, + blockId, + new BlockManagerManagedBuffer(blockInfoManager, blockId, data, false), + tLevel, + classTag) + logTrace(s"Replicated $blockId of ${data.size} bytes to $peer" + + s" in ${(System.nanoTime - onePeerStartTime).toDouble / 1e6} ms") + peersForReplication = peersForReplication.tail + peersReplicatedTo += peer + } catch { + case NonFatal(e) => + logWarning(s"Failed to replicate $blockId to $peer, failure #$numFailures", e) + peersFailedToReplicateTo += peer + // we have a failed replication, so we get the list of peers again + // we don't want peers we have already replicated to and the ones that + // have failed previously + val filteredPeers = getPeers(true).filter { p => + !peersFailedToReplicateTo.contains(p) && !peersReplicatedTo.contains(p) } - case None => // no peer left to replicate to - done = true + + numFailures += 1 + peersForReplication = blockReplicationPolicy.prioritize( + blockManagerId, + filteredPeers, + peersReplicatedTo, + blockId, + numPeersToReplicateTo - peersReplicatedTo.size) } } - val timeTakeMs = (System.currentTimeMillis - startTime) - logDebug(s"Replicating $blockId of ${data.limit()} bytes to " + - s"${peersReplicatedTo.size} peer(s) took $timeTakeMs ms") + logDebug(s"Replicating $blockId of ${data.size} bytes to " + + s"${peersReplicatedTo.size} peer(s) took ${(System.nanoTime - startTime) / 1e6} ms") if (peersReplicatedTo.size < numPeersToReplicateTo) { logWarning(s"Block $blockId replicated to only " + s"${peersReplicatedTo.size} peer(s) instead of $numPeersToReplicateTo peers") } + + logDebug(s"block $blockId replicated to ${peersReplicatedTo.mkString(", ")}") } /** * Read a block consisting of a single object. */ - def getSingle(blockId: BlockId): Option[Any] = { - get(blockId).map(_.data.next()) + def getSingle[T: ClassTag](blockId: BlockId): Option[T] = { + get[T](blockId).map(_.data.next().asInstanceOf[T]) } /** * Write a block consisting of a single object. + * + * @return true if the block was stored or false if the block was already stored or an + * error occurred. */ - def putSingle( + def putSingle[T: ClassTag]( blockId: BlockId, - value: Any, + value: T, level: StorageLevel, - tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true): Boolean = { putIterator(blockId, Iterator(value), level, tellMaster) } - def dropFromMemory( - blockId: BlockId, - data: Either[Array[Any], ByteBuffer]): Option[BlockStatus] = { - dropFromMemory(blockId, () => data) - } - /** * Drop a block from memory, possibly putting it on disk if applicable. Called when the memory * store reaches its limit and needs to free up space. * * If `data` is not put on disk, it won't be created. * - * Return the block status if the given block has been updated, else None. + * The caller of this method must hold a write lock on the block before calling this method. + * This method does not release the write lock. + * + * @return the block's new effective StorageLevel. */ - def dropFromMemory( + private[storage] override def dropFromMemory[T: ClassTag]( blockId: BlockId, - data: () => Either[Array[Any], ByteBuffer]): Option[BlockStatus] = { - + data: () => Either[Array[T], ChunkedByteBuffer]): StorageLevel = { logInfo(s"Dropping block $blockId from memory") - val info = blockInfo.get(blockId).orNull - - // If the block has not already been dropped - if (info != null) { - info.synchronized { - // required ? As of now, this will be invoked only for blocks which are ready - // But in case this changes in future, adding for consistency sake. - if (!info.waitForReady()) { - // If we get here, the block write failed. - logWarning(s"Block $blockId was marked as failure. Nothing to drop") - return None - } else if (blockInfo.get(blockId).isEmpty) { - logWarning(s"Block $blockId was already dropped.") - return None - } - var blockIsUpdated = false - val level = info.level - - // Drop to disk, if storage level requires - if (level.useDisk && !diskStore.contains(blockId)) { - logInfo(s"Writing block $blockId to disk") - data() match { - case Left(elements) => - diskStore.putArray(blockId, elements, level, returnValues = false) - case Right(bytes) => - diskStore.putBytes(blockId, bytes, level) + val info = blockInfoManager.assertBlockIsLockedForWriting(blockId) + var blockIsUpdated = false + val level = info.level + + // Drop to disk, if storage level requires + if (level.useDisk && !diskStore.contains(blockId)) { + logInfo(s"Writing block $blockId to disk") + data() match { + case Left(elements) => + diskStore.put(blockId) { channel => + val out = Channels.newOutputStream(channel) + serializerManager.dataSerializeStream( + blockId, + out, + elements.toIterator)(info.classTag.asInstanceOf[ClassTag[T]]) } - blockIsUpdated = true - } + case Right(bytes) => + diskStore.putBytes(blockId, bytes) + } + blockIsUpdated = true + } - // Actually drop from memory store - val droppedMemorySize = - if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L - val blockIsRemoved = memoryStore.remove(blockId) - if (blockIsRemoved) { - blockIsUpdated = true - } else { - logWarning(s"Block $blockId could not be dropped from memory as it does not exist") - } + // Actually drop from memory store + val droppedMemorySize = + if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L + val blockIsRemoved = memoryStore.remove(blockId) + if (blockIsRemoved) { + blockIsUpdated = true + } else { + logWarning(s"Block $blockId could not be dropped from memory as it does not exist") + } - val status = getCurrentBlockStatus(blockId, info) - if (info.tellMaster) { - reportBlockStatus(blockId, info, status, droppedMemorySize) - } - if (!level.useDisk) { - // The block is completely gone from this node; forget it so we can put() it again later. - blockInfo.remove(blockId) - } - if (blockIsUpdated) { - return Some(status) - } - } + val status = getCurrentBlockStatus(blockId, info) + if (info.tellMaster) { + reportBlockStatus(blockId, status, droppedMemorySize) } - None + if (blockIsUpdated) { + addUpdatedBlockStatusToTaskMetrics(blockId, status) + } + status.storageLevel } /** * Remove all blocks belonging to the given RDD. + * * @return The number of blocks removed. */ def removeRdd(rddId: Int): Int = { // TODO: Avoid a linear scan by creating another mapping of RDD.id to blocks. logInfo(s"Removing RDD $rddId") - val blocksToRemove = blockInfo.keys.flatMap(_.asRDDId).filter(_.rddId == rddId) + val blocksToRemove = blockInfoManager.entries.flatMap(_._1.asRDDId).filter(_.rddId == rddId) blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster = false) } blocksToRemove.size } @@ -1096,7 +1451,7 @@ private[spark] class BlockManager( */ def removeBroadcast(broadcastId: Long, tellMaster: Boolean): Int = { logDebug(s"Removing broadcast $broadcastId") - val blocksToRemove = blockInfo.keys.collect { + val blocksToRemove = blockInfoManager.entries.map(_._1).collect { case bid @ BroadcastBlockId(`broadcastId`, _) => bid } blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster) } @@ -1108,128 +1463,47 @@ private[spark] class BlockManager( */ def removeBlock(blockId: BlockId, tellMaster: Boolean = true): Unit = { logDebug(s"Removing block $blockId") - val info = blockInfo.get(blockId).orNull - if (info != null) { - info.synchronized { - // Removals are idempotent in disk store and memory store. At worst, we get a warning. - val removedFromMemory = memoryStore.remove(blockId) - val removedFromDisk = diskStore.remove(blockId) - val removedFromExternalBlockStore = - if (externalBlockStoreInitialized) externalBlockStore.remove(blockId) else false - if (!removedFromMemory && !removedFromDisk && !removedFromExternalBlockStore) { - logWarning(s"Block $blockId could not be removed as it was not found in either " + - "the disk, memory, or external block store") - } - blockInfo.remove(blockId) - if (tellMaster && info.tellMaster) { - val status = getCurrentBlockStatus(blockId, info) - reportBlockStatus(blockId, info, status) - } - } - } else { - // The block has already been removed; do nothing. - logWarning(s"Asked to remove block $blockId, which does not exist") - } - } - - private def dropOldNonBroadcastBlocks(cleanupTime: Long): Unit = { - logInfo(s"Dropping non broadcast blocks older than $cleanupTime") - dropOldBlocks(cleanupTime, !_.isBroadcast) - } - - private def dropOldBroadcastBlocks(cleanupTime: Long): Unit = { - logInfo(s"Dropping broadcast blocks older than $cleanupTime") - dropOldBlocks(cleanupTime, _.isBroadcast) - } - - private def dropOldBlocks(cleanupTime: Long, shouldDrop: (BlockId => Boolean)): Unit = { - val iterator = blockInfo.getEntrySet.iterator - while (iterator.hasNext) { - val entry = iterator.next() - val (id, info, time) = (entry.getKey, entry.getValue.value, entry.getValue.timestamp) - if (time < cleanupTime && shouldDrop(id)) { - info.synchronized { - val level = info.level - if (level.useMemory) { memoryStore.remove(id) } - if (level.useDisk) { diskStore.remove(id) } - if (level.useOffHeap) { externalBlockStore.remove(id) } - iterator.remove() - logInfo(s"Dropped block $id") - } - val status = getCurrentBlockStatus(id, info) - reportBlockStatus(id, info, status) - } - } - } - - private def shouldCompress(blockId: BlockId): Boolean = { - blockId match { - case _: ShuffleBlockId => compressShuffle - case _: BroadcastBlockId => compressBroadcast - case _: RDDBlockId => compressRdds - case _: TempLocalBlockId => compressShuffleSpill - case _: TempShuffleBlockId => compressShuffle - case _ => false + blockInfoManager.lockForWriting(blockId) match { + case None => + // The block has already been removed; do nothing. + logWarning(s"Asked to remove block $blockId, which does not exist") + case Some(info) => + removeBlockInternal(blockId, tellMaster = tellMaster && info.tellMaster) + addUpdatedBlockStatusToTaskMetrics(blockId, BlockStatus.empty) } } /** - * Wrap an output stream for compression if block compression is enabled for its block type + * Internal version of [[removeBlock()]] which assumes that the caller already holds a write + * lock on the block. */ - def wrapForCompression(blockId: BlockId, s: OutputStream): OutputStream = { - if (shouldCompress(blockId)) compressionCodec.compressedOutputStream(s) else s - } - - /** - * Wrap an input stream for compression if block compression is enabled for its block type - */ - def wrapForCompression(blockId: BlockId, s: InputStream): InputStream = { - if (shouldCompress(blockId)) compressionCodec.compressedInputStream(s) else s - } - - /** Serializes into a stream. */ - def dataSerializeStream( - blockId: BlockId, - outputStream: OutputStream, - values: Iterator[Any], - serializer: Serializer = defaultSerializer): Unit = { - val byteStream = new BufferedOutputStream(outputStream) - val ser = serializer.newInstance() - ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close() + private def removeBlockInternal(blockId: BlockId, tellMaster: Boolean): Unit = { + // Removals are idempotent in disk store and memory store. At worst, we get a warning. + val removedFromMemory = memoryStore.remove(blockId) + val removedFromDisk = diskStore.remove(blockId) + if (!removedFromMemory && !removedFromDisk) { + logWarning(s"Block $blockId could not be removed as it was not found on disk or in memory") + } + blockInfoManager.removeBlock(blockId) + if (tellMaster) { + reportBlockStatus(blockId, BlockStatus.empty) + } } - /** Serializes into a byte buffer. */ - def dataSerialize( - blockId: BlockId, - values: Iterator[Any], - serializer: Serializer = defaultSerializer): ByteBuffer = { - val byteStream = new ByteArrayOutputStream(4096) - dataSerializeStream(blockId, byteStream, values, serializer) - ByteBuffer.wrap(byteStream.toByteArray) + private def addUpdatedBlockStatusToTaskMetrics(blockId: BlockId, status: BlockStatus): Unit = { + if (conf.get(config.TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES)) { + Option(TaskContext.get()).foreach { c => + c.taskMetrics().incUpdatedBlockStatuses(blockId -> status) + } + } } - /** - * Deserializes a ByteBuffer into an iterator of values and disposes of it when the end of - * the iterator is reached. - */ - def dataDeserialize( + def releaseLockAndDispose( blockId: BlockId, - bytes: ByteBuffer, - serializer: Serializer = defaultSerializer): Iterator[Any] = { - bytes.rewind() - dataDeserializeStream(blockId, new ByteBufferInputStream(bytes, true), serializer) - } - - /** - * Deserializes a InputStream into an iterator of values and disposes of it when the end of - * the iterator is reached. - */ - def dataDeserializeStream( - blockId: BlockId, - inputStream: InputStream, - serializer: Serializer = defaultSerializer): Iterator[Any] = { - val stream = new BufferedInputStream(inputStream) - serializer.newInstance().deserializeStream(wrapForCompression(blockId, stream)).asIterator + data: BlockData, + taskAttemptId: Option[Long] = None): Unit = { + releaseLock(blockId, taskAttemptId) + data.dispose() } def stop(): Unit = { @@ -1240,38 +1514,17 @@ private[spark] class BlockManager( } diskBlockManager.stop() rpcEnv.stop(slaveEndpoint) - blockInfo.clear() + blockInfoManager.clear() memoryStore.clear() - diskStore.clear() - if (externalBlockStoreInitialized) { - externalBlockStore.clear() - } - metadataCleaner.cancel() - broadcastCleaner.cancel() futureExecutionContext.shutdownNow() logInfo("BlockManager stopped") } } -private[spark] object BlockManager extends Logging { +private[spark] object BlockManager { private val ID_GENERATOR = new IdGenerator - /** - * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that - * might cause errors if one attempts to read from the unmapped buffer, but it's better than - * waiting for the GC to find it because that could lead to huge numbers of open files. There's - * unfortunately no standard API to do this. - */ - def dispose(buffer: ByteBuffer): Unit = { - if (buffer != null && buffer.isInstanceOf[MappedByteBuffer]) { - logTrace(s"Unmapping $buffer") - if (buffer.asInstanceOf[DirectBuffer].cleaner() != null) { - buffer.asInstanceOf[DirectBuffer].cleaner().clean() - } - } - } - def blockIdsToHosts( blockIds: Array[BlockId], env: SparkEnv, @@ -1291,4 +1544,12 @@ private[spark] object BlockManager extends Logging { } blockManagers.toMap } + + private class ShuffleMetricsSource( + override val sourceName: String, + metricSet: MetricSet) extends Source { + + override val metricRegistry = new MetricRegistry + metricRegistry.registerAll(metricSet) + } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index 69ac37511e73..2c3da0ee85e0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.Utils * :: DeveloperApi :: * This class represent an unique identifier for a BlockManager. * - * The first 2 constructors of this class is made private to ensure that BlockManagerId objects + * The first 2 constructors of this class are made private to ensure that BlockManagerId objects * can be created only using the apply method in the companion object. This allows de-duplication * of ID objects. Also, constructor parameters are private to ensure that parameters cannot be * modified from outside this class. @@ -37,15 +37,16 @@ import org.apache.spark.util.Utils class BlockManagerId private ( private var executorId_ : String, private var host_ : String, - private var port_ : Int) + private var port_ : Int, + private var topologyInfo_ : Option[String]) extends Externalizable { - private def this() = this(null, null, 0) // For deserialization only + private def this() = this(null, null, 0, None) // For deserialization only def executorId: String = executorId_ - if (null != host_){ - Utils.checkHost(host_, "Expected hostname") + if (null != host_) { + Utils.checkHost(host_) assert (port_ > 0) } @@ -60,6 +61,8 @@ class BlockManagerId private ( def port: Int = port_ + def topologyInfo: Option[String] = topologyInfo_ + def isDriver: Boolean = { executorId == SparkContext.DRIVER_IDENTIFIER || executorId == SparkContext.LEGACY_DRIVER_IDENTIFIER @@ -69,24 +72,33 @@ class BlockManagerId private ( out.writeUTF(executorId_) out.writeUTF(host_) out.writeInt(port_) + out.writeBoolean(topologyInfo_.isDefined) + // we only write topologyInfo if we have it + topologyInfo.foreach(out.writeUTF(_)) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { executorId_ = in.readUTF() host_ = in.readUTF() port_ = in.readInt() + val isTopologyInfoAvailable = in.readBoolean() + topologyInfo_ = if (isTopologyInfoAvailable) Option(in.readUTF()) else None } @throws(classOf[IOException]) private def readResolve(): Object = BlockManagerId.getCachedBlockManagerId(this) - override def toString: String = s"BlockManagerId($executorId, $host, $port)" + override def toString: String = s"BlockManagerId($executorId, $host, $port, $topologyInfo)" - override def hashCode: Int = (executorId.hashCode * 41 + host.hashCode) * 41 + port + override def hashCode: Int = + ((executorId.hashCode * 41 + host.hashCode) * 41 + port) * 41 + topologyInfo.hashCode override def equals(that: Any): Boolean = that match { case id: BlockManagerId => - executorId == id.executorId && port == id.port && host == id.host + executorId == id.executorId && + port == id.port && + host == id.host && + topologyInfo == id.topologyInfo case _ => false } @@ -101,10 +113,18 @@ private[spark] object BlockManagerId { * @param execId ID of the executor. * @param host Host name of the block manager. * @param port Port of the block manager. + * @param topologyInfo topology information for the blockmanager, if available + * This can be network topology information for use while choosing peers + * while replicating data blocks. More information available here: + * [[org.apache.spark.storage.TopologyMapper]] * @return A new [[org.apache.spark.storage.BlockManagerId]]. */ - def apply(execId: String, host: String, port: Int): BlockManagerId = - getCachedBlockManagerId(new BlockManagerId(execId, host, port)) + def apply( + execId: String, + host: String, + port: Int, + topologyInfo: Option[String] = None): BlockManagerId = + getCachedBlockManagerId(new BlockManagerId(execId, host, port, topologyInfo)) def apply(in: ObjectInput): BlockManagerId = { val obj = new BlockManagerId() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala new file mode 100644 index 000000000000..3d3806126676 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.io.InputStream +import java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicInteger + +import org.apache.spark.network.buffer.ManagedBuffer + +/** + * This [[ManagedBuffer]] wraps a [[BlockData]] instance retrieved from the [[BlockManager]] + * so that the corresponding block's read lock can be released once this buffer's references + * are released. + * + * If `dispose` is set to true, the [[BlockData]]will be disposed when the buffer's reference + * count drops to zero. + * + * This is effectively a wrapper / bridge to connect the BlockManager's notion of read locks + * to the network layer's notion of retain / release counts. + */ +private[storage] class BlockManagerManagedBuffer( + blockInfoManager: BlockInfoManager, + blockId: BlockId, + data: BlockData, + dispose: Boolean) extends ManagedBuffer { + + private val refCount = new AtomicInteger(1) + + override def size(): Long = data.size + + override def nioByteBuffer(): ByteBuffer = data.toByteBuffer() + + override def createInputStream(): InputStream = data.toInputStream() + + override def convertToNetty(): Object = data.toNetty() + + override def retain(): ManagedBuffer = { + refCount.incrementAndGet() + val locked = blockInfoManager.lockForReading(blockId, blocking = false) + assert(locked.isDefined) + this + } + + override def release(): ManagedBuffer = { + blockInfoManager.unlock(blockId) + if (refCount.decrementAndGet() == 0 && dispose) { + data.dispose() + } + this + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index f45bff34d4db..8b1dc0ba6356 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -19,12 +19,13 @@ package org.apache.spark.storage import scala.collection.Iterable import scala.collection.generic.CanBuildFrom -import scala.concurrent.{Await, Future} +import scala.concurrent.Future +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.util.{ThreadUtils, RpcUtils} +import org.apache.spark.util.{RpcUtils, ThreadUtils} private[spark] class BlockManagerMaster( @@ -41,12 +42,29 @@ class BlockManagerMaster( logInfo("Removed " + execId + " successfully in removeExecutor") } - /** Register the BlockManager's id with the driver. */ + /** Request removal of a dead executor from the driver endpoint. + * This is only called on the driver side. Non-blocking + */ + def removeExecutorAsync(execId: String) { + driverEndpoint.ask[Boolean](RemoveExecutor(execId)) + logInfo("Removal of executor " + execId + " requested") + } + + /** + * Register the BlockManager's id with the driver. The input BlockManagerId does not contain + * topology information. This information is obtained from the master and we respond with an + * updated BlockManagerId fleshed out with this information. + */ def registerBlockManager( - blockManagerId: BlockManagerId, maxMemSize: Long, slaveEndpoint: RpcEndpointRef): Unit = { - logInfo("Trying to register BlockManager") - tell(RegisterBlockManager(blockManagerId, maxMemSize, slaveEndpoint)) - logInfo("Registered BlockManager") + blockManagerId: BlockManagerId, + maxOnHeapMemSize: Long, + maxOffHeapMemSize: Long, + slaveEndpoint: RpcEndpointRef): BlockManagerId = { + logInfo(s"Registering BlockManager $blockManagerId") + val updatedId = driverEndpoint.askSync[BlockManagerId]( + RegisterBlockManager(blockManagerId, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint)) + logInfo(s"Registered BlockManager $updatedId") + updatedId } def updateBlockInfo( @@ -54,23 +72,21 @@ class BlockManagerMaster( blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long, - externalBlockStoreSize: Long): Boolean = { - val res = driverEndpoint.askWithRetry[Boolean]( - UpdateBlockInfo(blockManagerId, blockId, storageLevel, - memSize, diskSize, externalBlockStoreSize)) + diskSize: Long): Boolean = { + val res = driverEndpoint.askSync[Boolean]( + UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize)) logDebug(s"Updated info of block $blockId") res } /** Get locations of the blockId from the driver */ def getLocations(blockId: BlockId): Seq[BlockManagerId] = { - driverEndpoint.askWithRetry[Seq[BlockManagerId]](GetLocations(blockId)) + driverEndpoint.askSync[Seq[BlockManagerId]](GetLocations(blockId)) } /** Get locations of multiple blockIds from the driver */ def getLocations(blockIds: Array[BlockId]): IndexedSeq[Seq[BlockManagerId]] = { - driverEndpoint.askWithRetry[IndexedSeq[Seq[BlockManagerId]]]( + driverEndpoint.askSync[IndexedSeq[Seq[BlockManagerId]]]( GetLocationsMultipleBlockIds(blockIds)) } @@ -84,11 +100,11 @@ class BlockManagerMaster( /** Get ids of other nodes in the cluster from the driver */ def getPeers(blockManagerId: BlockManagerId): Seq[BlockManagerId] = { - driverEndpoint.askWithRetry[Seq[BlockManagerId]](GetPeers(blockManagerId)) + driverEndpoint.askSync[Seq[BlockManagerId]](GetPeers(blockManagerId)) } - def getRpcHostPortForExecutor(executorId: String): Option[(String, Int)] = { - driverEndpoint.askWithRetry[Option[(String, Int)]](GetRpcHostPortForExecutor(executorId)) + def getExecutorEndpointRef(executorId: String): Option[RpcEndpointRef] = { + driverEndpoint.askSync[Option[RpcEndpointRef]](GetExecutorEndpointRef(executorId)) } /** @@ -96,16 +112,15 @@ class BlockManagerMaster( * blocks that the driver knows about. */ def removeBlock(blockId: BlockId) { - driverEndpoint.askWithRetry[Boolean](RemoveBlock(blockId)) + driverEndpoint.askSync[Boolean](RemoveBlock(blockId)) } /** Remove all blocks belonging to the given RDD. */ def removeRdd(rddId: Int, blocking: Boolean) { - val future = driverEndpoint.askWithRetry[Future[Seq[Int]]](RemoveRdd(rddId)) - future.onFailure { - case e: Exception => - logWarning(s"Failed to remove RDD $rddId - ${e.getMessage}", e) - }(ThreadUtils.sameThread) + val future = driverEndpoint.askSync[Future[Seq[Int]]](RemoveRdd(rddId)) + future.failed.foreach(e => + logWarning(s"Failed to remove RDD $rddId - ${e.getMessage}", e) + )(ThreadUtils.sameThread) if (blocking) { timeout.awaitResult(future) } @@ -113,11 +128,10 @@ class BlockManagerMaster( /** Remove all blocks belonging to the given shuffle. */ def removeShuffle(shuffleId: Int, blocking: Boolean) { - val future = driverEndpoint.askWithRetry[Future[Seq[Boolean]]](RemoveShuffle(shuffleId)) - future.onFailure { - case e: Exception => - logWarning(s"Failed to remove shuffle $shuffleId - ${e.getMessage}", e) - }(ThreadUtils.sameThread) + val future = driverEndpoint.askSync[Future[Seq[Boolean]]](RemoveShuffle(shuffleId)) + future.failed.foreach(e => + logWarning(s"Failed to remove shuffle $shuffleId - ${e.getMessage}", e) + )(ThreadUtils.sameThread) if (blocking) { timeout.awaitResult(future) } @@ -125,13 +139,12 @@ class BlockManagerMaster( /** Remove all blocks belonging to the given broadcast. */ def removeBroadcast(broadcastId: Long, removeFromMaster: Boolean, blocking: Boolean) { - val future = driverEndpoint.askWithRetry[Future[Seq[Int]]]( + val future = driverEndpoint.askSync[Future[Seq[Int]]]( RemoveBroadcast(broadcastId, removeFromMaster)) - future.onFailure { - case e: Exception => - logWarning(s"Failed to remove broadcast $broadcastId" + - s" with removeFromMaster = $removeFromMaster - ${e.getMessage}", e) - }(ThreadUtils.sameThread) + future.failed.foreach(e => + logWarning(s"Failed to remove broadcast $broadcastId" + + s" with removeFromMaster = $removeFromMaster - ${e.getMessage}", e) + )(ThreadUtils.sameThread) if (blocking) { timeout.awaitResult(future) } @@ -144,11 +157,11 @@ class BlockManagerMaster( * amount of remaining memory. */ def getMemoryStatus: Map[BlockManagerId, (Long, Long)] = { - driverEndpoint.askWithRetry[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus) + driverEndpoint.askSync[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus) } def getStorageStatus: Array[StorageStatus] = { - driverEndpoint.askWithRetry[Array[StorageStatus]](GetStorageStatus) + driverEndpoint.askSync[Array[StorageStatus]](GetStorageStatus) } /** @@ -169,7 +182,7 @@ class BlockManagerMaster( * master endpoint for a response to a prior message. */ val response = driverEndpoint. - askWithRetry[Map[BlockManagerId, Future[Option[BlockStatus]]]](msg) + askSync[Map[BlockManagerId, Future[Option[BlockStatus]]]](msg) val (blockManagerIds, futures) = response.unzip implicit val sameThread = ThreadUtils.sameThread val cbf = @@ -199,7 +212,7 @@ class BlockManagerMaster( filter: BlockId => Boolean, askSlaves: Boolean): Seq[BlockId] = { val msg = GetMatchingBlockIds(filter, askSlaves) - val future = driverEndpoint.askWithRetry[Future[Seq[BlockId]]](msg) + val future = driverEndpoint.askSync[Future[Seq[BlockId]]](msg) timeout.awaitResult(future) } @@ -208,7 +221,7 @@ class BlockManagerMaster( * since they are not reported the master. */ def hasCachedBlocks(executorId: String): Boolean = { - driverEndpoint.askWithRetry[Boolean](HasCachedBlocks(executorId)) + driverEndpoint.askSync[Boolean](HasCachedBlocks(executorId)) } /** Stop the driver endpoint, called only on the Spark driver node */ @@ -222,7 +235,7 @@ class BlockManagerMaster( /** Send a one-way message to the master endpoint, to which we expect it to reply with true. */ private def tell(message: Any) { - if (!driverEndpoint.askWithRetry[Boolean](message)) { + if (!driverEndpoint.askSync[Boolean](message)) { throw new SparkException("BlockManagerMasterEndpoint returned false, expected true.") } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 7db6035553ae..df0a5f5e229f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -19,14 +19,15 @@ package org.apache.spark.storage import java.util.{HashMap => JHashMap} -import scala.collection.immutable.HashSet -import scala.collection.mutable import scala.collection.JavaConverters._ +import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future} +import scala.util.Random -import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv, RpcCallContext, ThreadSafeRpcEndpoint} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging +import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{ThreadUtils, Utils} @@ -55,15 +56,27 @@ class BlockManagerMasterEndpoint( private val askThreadPool = ThreadUtils.newDaemonCachedThreadPool("block-manager-ask-thread-pool") private implicit val askExecutionContext = ExecutionContext.fromExecutorService(askThreadPool) + private val topologyMapper = { + val topologyMapperClassName = conf.get( + "spark.storage.replication.topologyMapper", classOf[DefaultTopologyMapper].getName) + val clazz = Utils.classForName(topologyMapperClassName) + val mapper = + clazz.getConstructor(classOf[SparkConf]).newInstance(conf).asInstanceOf[TopologyMapper] + logInfo(s"Using $topologyMapperClassName for getting topology information") + mapper + } + + val proactivelyReplicate = conf.get("spark.storage.replication.proactive", "false").toBoolean + + logInfo("BlockManagerMasterEndpoint up") + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RegisterBlockManager(blockManagerId, maxMemSize, slaveEndpoint) => - register(blockManagerId, maxMemSize, slaveEndpoint) - context.reply(true) + case RegisterBlockManager(blockManagerId, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint) => + context.reply(register(blockManagerId, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint)) - case _updateBlockInfo @ UpdateBlockInfo( - blockManagerId, blockId, storageLevel, deserializedSize, size, externalBlockStoreSize) => - context.reply(updateBlockInfo( - blockManagerId, blockId, storageLevel, deserializedSize, size, externalBlockStoreSize)) + case _updateBlockInfo @ + UpdateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) => + context.reply(updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size)) listenerBus.post(SparkListenerBlockUpdated(BlockUpdatedInfo(_updateBlockInfo))) case GetLocations(blockId) => @@ -75,8 +88,8 @@ class BlockManagerMasterEndpoint( case GetPeers(blockManagerId) => context.reply(getPeers(blockManagerId)) - case GetRpcHostPortForExecutor(executorId) => - context.reply(getRpcHostPortForExecutor(executorId)) + case GetExecutorEndpointRef(executorId) => + context.reply(getExecutorEndpointRef(executorId)) case GetMemoryStatus => context.reply(memoryStatus) @@ -185,17 +198,38 @@ class BlockManagerMasterEndpoint( // Remove it from blockManagerInfo and remove all the blocks. blockManagerInfo.remove(blockManagerId) + val iterator = info.blocks.keySet.iterator while (iterator.hasNext) { val blockId = iterator.next val locations = blockLocations.get(blockId) locations -= blockManagerId + // De-register the block if none of the block managers have it. Otherwise, if pro-active + // replication is enabled, and a block is either an RDD or a test block (the latter is used + // for unit testing), we send a message to a randomly chosen executor location to replicate + // the given block. Note that we ignore other block types (such as broadcast/shuffle blocks + // etc.) as replication doesn't make much sense in that context. if (locations.size == 0) { blockLocations.remove(blockId) + logWarning(s"No more replicas available for $blockId !") + } else if (proactivelyReplicate && (blockId.isRDD || blockId.isInstanceOf[TestBlockId])) { + // As a heursitic, assume single executor failure to find out the number of replicas that + // existed before failure + val maxReplicas = locations.size + 1 + val i = (new Random(blockId.hashCode)).nextInt(locations.size) + val blockLocations = locations.toSeq + val candidateBMId = blockLocations(i) + blockManagerInfo.get(candidateBMId).foreach { bm => + val remainingLocations = locations.toSeq.filter(bm => bm != candidateBMId) + val replicateMsg = ReplicateBlock(blockId, remainingLocations, maxReplicas) + bm.slaveEndpoint.ask[Boolean](replicateMsg) + } } } + listenerBus.post(SparkListenerBlockManagerRemoved(System.currentTimeMillis(), blockManagerId)) logInfo(s"Removing block manager $blockManagerId") + } private def removeExecutor(execId: String) { @@ -242,7 +276,8 @@ class BlockManagerMasterEndpoint( private def storageStatus: Array[StorageStatus] = { blockManagerInfo.map { case (blockManagerId, info) => - new StorageStatus(blockManagerId, info.maxMem, info.blocks.asScala) + new StorageStatus(blockManagerId, info.maxMem, Some(info.maxOnHeapMem), + Some(info.maxOffHeapMem), info.blocks.asScala) }.toArray } @@ -299,7 +334,22 @@ class BlockManagerMasterEndpoint( ).map(_.flatten.toSeq) } - private def register(id: BlockManagerId, maxMemSize: Long, slaveEndpoint: RpcEndpointRef) { + /** + * Returns the BlockManagerId with topology information populated, if available. + */ + private def register( + idWithoutTopologyInfo: BlockManagerId, + maxOnHeapMemSize: Long, + maxOffHeapMemSize: Long, + slaveEndpoint: RpcEndpointRef): BlockManagerId = { + // the dummy id is not expected to contain the topology information. + // we get that info here and respond back with a more fleshed out block manager id + val id = BlockManagerId( + idWithoutTopologyInfo.executorId, + idWithoutTopologyInfo.host, + idWithoutTopologyInfo.port, + topologyMapper.getTopologyForHost(idWithoutTopologyInfo.host)) + val time = System.currentTimeMillis() if (!blockManagerInfo.contains(id)) { blockManagerIdByExecutor.get(id.executorId) match { @@ -311,14 +361,16 @@ class BlockManagerMasterEndpoint( case None => } logInfo("Registering block manager %s with %s RAM, %s".format( - id.hostPort, Utils.bytesToString(maxMemSize), id)) + id.hostPort, Utils.bytesToString(maxOnHeapMemSize + maxOffHeapMemSize), id)) blockManagerIdByExecutor(id.executorId) = id blockManagerInfo(id) = new BlockManagerInfo( - id, System.currentTimeMillis(), maxMemSize, slaveEndpoint) + id, System.currentTimeMillis(), maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint) } - listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxMemSize)) + listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxOnHeapMemSize + maxOffHeapMemSize, + Some(maxOnHeapMemSize), Some(maxOffHeapMemSize))) + id } private def updateBlockInfo( @@ -326,8 +378,7 @@ class BlockManagerMasterEndpoint( blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long, - externalBlockStoreSize: Long): Boolean = { + diskSize: Long): Boolean = { if (!blockManagerInfo.contains(blockManagerId)) { if (blockManagerId.isDriver && !isLocal) { @@ -344,8 +395,7 @@ class BlockManagerMasterEndpoint( return true } - blockManagerInfo(blockManagerId).updateBlockInfo( - blockId, storageLevel, memSize, diskSize, externalBlockStoreSize) + blockManagerInfo(blockManagerId).updateBlockInfo(blockId, storageLevel, memSize, diskSize) var locations: mutable.HashSet[BlockManagerId] = null if (blockLocations.containsKey(blockId)) { @@ -388,15 +438,14 @@ class BlockManagerMasterEndpoint( } /** - * Returns the hostname and port of an executor, based on the [[RpcEnv]] address of its - * [[BlockManagerSlaveEndpoint]]. + * Returns an [[RpcEndpointRef]] of the [[BlockManagerSlaveEndpoint]] for sending RPC messages. */ - private def getRpcHostPortForExecutor(executorId: String): Option[(String, Int)] = { + private def getExecutorEndpointRef(executorId: String): Option[RpcEndpointRef] = { for ( blockManagerId <- blockManagerIdByExecutor.get(executorId); info <- blockManagerInfo.get(blockManagerId) ) yield { - (info.slaveEndpoint.address.host, info.slaveEndpoint.address.port) + info.slaveEndpoint } } @@ -406,26 +455,25 @@ class BlockManagerMasterEndpoint( } @DeveloperApi -case class BlockStatus( - storageLevel: StorageLevel, - memSize: Long, - diskSize: Long, - externalBlockStoreSize: Long) { - def isCached: Boolean = memSize + diskSize + externalBlockStoreSize > 0 +case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) { + def isCached: Boolean = memSize + diskSize > 0 } @DeveloperApi object BlockStatus { - def empty: BlockStatus = BlockStatus(StorageLevel.NONE, 0L, 0L, 0L) + def empty: BlockStatus = BlockStatus(StorageLevel.NONE, memSize = 0L, diskSize = 0L) } private[spark] class BlockManagerInfo( val blockManagerId: BlockManagerId, timeMs: Long, - val maxMem: Long, + val maxOnHeapMem: Long, + val maxOffHeapMem: Long, val slaveEndpoint: RpcEndpointRef) extends Logging { + val maxMem = maxOnHeapMem + maxOffHeapMem + private var _lastSeenMs: Long = timeMs private var _remainingMem: Long = maxMem @@ -445,16 +493,21 @@ private[spark] class BlockManagerInfo( blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long, - externalBlockStoreSize: Long) { + diskSize: Long) { updateLastSeenMs() - if (_blocks.containsKey(blockId)) { + val blockExists = _blocks.containsKey(blockId) + var originalMemSize: Long = 0 + var originalDiskSize: Long = 0 + var originalLevel: StorageLevel = StorageLevel.NONE + + if (blockExists) { // The block exists on the slave already. val blockStatus: BlockStatus = _blocks.get(blockId) - val originalLevel: StorageLevel = blockStatus.storageLevel - val originalMemSize: Long = blockStatus.memSize + originalLevel = blockStatus.storageLevel + originalMemSize = blockStatus.memSize + originalDiskSize = blockStatus.diskSize if (originalLevel.useMemory) { _remainingMem += originalMemSize @@ -462,7 +515,7 @@ private[spark] class BlockManagerInfo( } if (storageLevel.isValid) { - /* isValid means it is either stored in-memory, on-disk or on-externalBlockStore. + /* isValid means it is either stored in-memory or on-disk. * The memSize here indicates the data size in or dropped from memory, * externalBlockStoreSize here indicates the data size in or dropped from externalBlockStore, * and the diskSize here indicates the data size in or dropped to disk. @@ -470,46 +523,47 @@ private[spark] class BlockManagerInfo( * Therefore, a safe way to set BlockStatus is to set its info in accurate modes. */ var blockStatus: BlockStatus = null if (storageLevel.useMemory) { - blockStatus = BlockStatus(storageLevel, memSize, 0, 0) + blockStatus = BlockStatus(storageLevel, memSize = memSize, diskSize = 0) _blocks.put(blockId, blockStatus) _remainingMem -= memSize - logInfo("Added %s in memory on %s (size: %s, free: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(memSize), - Utils.bytesToString(_remainingMem))) + if (blockExists) { + logInfo(s"Updated $blockId in memory on ${blockManagerId.hostPort}" + + s" (current size: ${Utils.bytesToString(memSize)}," + + s" original size: ${Utils.bytesToString(originalMemSize)}," + + s" free: ${Utils.bytesToString(_remainingMem)})") + } else { + logInfo(s"Added $blockId in memory on ${blockManagerId.hostPort}" + + s" (size: ${Utils.bytesToString(memSize)}," + + s" free: ${Utils.bytesToString(_remainingMem)})") + } } if (storageLevel.useDisk) { - blockStatus = BlockStatus(storageLevel, 0, diskSize, 0) + blockStatus = BlockStatus(storageLevel, memSize = 0, diskSize = diskSize) _blocks.put(blockId, blockStatus) - logInfo("Added %s on disk on %s (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) - } - if (storageLevel.useOffHeap) { - blockStatus = BlockStatus(storageLevel, 0, 0, externalBlockStoreSize) - _blocks.put(blockId, blockStatus) - logInfo("Added %s on ExternalBlockStore on %s (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(externalBlockStoreSize))) + if (blockExists) { + logInfo(s"Updated $blockId on disk on ${blockManagerId.hostPort}" + + s" (current size: ${Utils.bytesToString(diskSize)}," + + s" original size: ${Utils.bytesToString(originalDiskSize)})") + } else { + logInfo(s"Added $blockId on disk on ${blockManagerId.hostPort}" + + s" (size: ${Utils.bytesToString(diskSize)})") + } } if (!blockId.isBroadcast && blockStatus.isCached) { _cachedBlocks += blockId } - } else if (_blocks.containsKey(blockId)) { + } else if (blockExists) { // If isValid is not true, drop the block. - val blockStatus: BlockStatus = _blocks.get(blockId) _blocks.remove(blockId) _cachedBlocks -= blockId - if (blockStatus.storageLevel.useMemory) { - logInfo("Removed %s on %s in memory (size: %s, free: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.memSize), - Utils.bytesToString(_remainingMem))) - } - if (blockStatus.storageLevel.useDisk) { - logInfo("Removed %s on %s on disk (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) + if (originalLevel.useMemory) { + logInfo(s"Removed $blockId on ${blockManagerId.hostPort} in memory" + + s" (size: ${Utils.bytesToString(originalMemSize)}," + + s" free: ${Utils.bytesToString(_remainingMem)})") } - if (blockStatus.storageLevel.useOffHeap) { - logInfo("Removed %s on %s on externalBlockStore (size: %s)".format( - blockId, blockManagerId.hostPort, - Utils.bytesToString(blockStatus.externalBlockStoreSize))) + if (originalLevel.useDisk) { + logInfo(s"Removed $blockId on ${blockManagerId.hostPort} on disk" + + s" (size: ${Utils.bytesToString(originalDiskSize)})") } } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 376e9eb48843..0c0ff144596a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -32,6 +32,10 @@ private[spark] object BlockManagerMessages { // blocks that the master knows about. case class RemoveBlock(blockId: BlockId) extends ToBlockManagerSlave + // Replicate blocks that were lost due to executor failure + case class ReplicateBlock(blockId: BlockId, replicas: Seq[BlockManagerId], maxReplicas: Int) + extends ToBlockManagerSlave + // Remove all blocks belonging to a specific RDD. case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave @@ -42,6 +46,11 @@ private[spark] object BlockManagerMessages { case class RemoveBroadcast(broadcastId: Long, removeFromDriver: Boolean = true) extends ToBlockManagerSlave + /** + * Driver to Executor message to trigger a thread dump. + */ + case object TriggerThreadDump extends ToBlockManagerSlave + ////////////////////////////////////////////////////////////////////////////////// // Messages from slaves to the master. ////////////////////////////////////////////////////////////////////////////////// @@ -49,7 +58,8 @@ private[spark] object BlockManagerMessages { case class RegisterBlockManager( blockManagerId: BlockManagerId, - maxMemSize: Long, + maxOnHeapMemSize: Long, + maxOffHeapMemSize: Long, sender: RpcEndpointRef) extends ToBlockManagerMaster @@ -58,12 +68,11 @@ private[spark] object BlockManagerMessages { var blockId: BlockId, var storageLevel: StorageLevel, var memSize: Long, - var diskSize: Long, - var externalBlockStoreSize: Long) + var diskSize: Long) extends ToBlockManagerMaster with Externalizable { - def this() = this(null, null, null, 0, 0, 0) // For deserialization only + def this() = this(null, null, null, 0, 0) // For deserialization only override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { blockManagerId.writeExternal(out) @@ -71,7 +80,6 @@ private[spark] object BlockManagerMessages { storageLevel.writeExternal(out) out.writeLong(memSize) out.writeLong(diskSize) - out.writeLong(externalBlockStoreSize) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -80,7 +88,6 @@ private[spark] object BlockManagerMessages { storageLevel = StorageLevel(in) memSize = in.readLong() diskSize = in.readLong() - externalBlockStoreSize = in.readLong() } } @@ -90,7 +97,7 @@ private[spark] object BlockManagerMessages { case class GetPeers(blockManagerId: BlockManagerId) extends ToBlockManagerMaster - case class GetRpcHostPortForExecutor(executorId: String) extends ToBlockManagerMaster + case class GetExecutorEndpointRef(executorId: String) extends ToBlockManagerMaster case class RemoveExecutor(execId: String) extends ToBlockManagerMaster diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala index e749631bf6f1..742cf4fe393f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala @@ -19,10 +19,11 @@ package org.apache.spark.storage import scala.concurrent.{ExecutionContext, Future} -import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcEnv, RpcCallContext, RpcEndpoint} -import org.apache.spark.util.ThreadUtils -import org.apache.spark.{Logging, MapOutputTracker, SparkEnv} +import org.apache.spark.{MapOutputTracker, SparkEnv} +import org.apache.spark.internal.Logging +import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.storage.BlockManagerMessages._ +import org.apache.spark.util.{ThreadUtils, Utils} /** * An RpcEndpoint to take commands from the master to execute options. For example, @@ -70,6 +71,13 @@ class BlockManagerSlaveEndpoint( case GetMatchingBlockIds(filter, _) => context.reply(blockManager.getMatchingBlockIds(filter)) + + case TriggerThreadDump => + context.reply(Utils.getThreadDump()) + + case ReplicateBlock(blockId, replicas, maxReplicas) => + context.reply(blockManager.replicateBlock(blockId, replicas.toSet, maxReplicas)) + } private def doAsync[T](actionMessage: String, context: RpcCallContext)(body: => T) { @@ -77,13 +85,13 @@ class BlockManagerSlaveEndpoint( logDebug(actionMessage) body } - future.onSuccess { case response => - logDebug("Done " + actionMessage + ", response is " + response) + future.foreach { response => + logDebug(s"Done $actionMessage, response is $response") context.reply(response) - logDebug("Sent response: " + response + " to " + context.senderAddress) + logDebug(s"Sent response: $response to ${context.senderAddress}") } - future.onFailure { case t: Throwable => - logError("Error in " + actionMessage, t) + future.failed.foreach { t => + logError(s"Error in $actionMessage", t) context.sendFailure(t) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index c5ba9af3e265..197a01762c0c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -26,35 +26,39 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager) override val metricRegistry = new MetricRegistry() override val sourceName = "BlockManager" - metricRegistry.register(MetricRegistry.name("memory", "maxMem_MB"), new Gauge[Long] { - override def getValue: Long = { - val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).sum - maxMem / 1024 / 1024 - } - }) - - metricRegistry.register(MetricRegistry.name("memory", "remainingMem_MB"), new Gauge[Long] { - override def getValue: Long = { - val storageStatusList = blockManager.master.getStorageStatus - val remainingMem = storageStatusList.map(_.memRemaining).sum - remainingMem / 1024 / 1024 - } - }) - - metricRegistry.register(MetricRegistry.name("memory", "memUsed_MB"), new Gauge[Long] { - override def getValue: Long = { - val storageStatusList = blockManager.master.getStorageStatus - val memUsed = storageStatusList.map(_.memUsed).sum - memUsed / 1024 / 1024 - } - }) - - metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed_MB"), new Gauge[Long] { - override def getValue: Long = { - val storageStatusList = blockManager.master.getStorageStatus - val diskSpaceUsed = storageStatusList.map(_.diskUsed).sum - diskSpaceUsed / 1024 / 1024 - } - }) + private def registerGauge(name: String, func: BlockManagerMaster => Long): Unit = { + metricRegistry.register(name, new Gauge[Long] { + override def getValue: Long = func(blockManager.master) / 1024 / 1024 + }) + } + + registerGauge(MetricRegistry.name("memory", "maxMem_MB"), + _.getStorageStatus.map(_.maxMem).sum) + + registerGauge(MetricRegistry.name("memory", "maxOnHeapMem_MB"), + _.getStorageStatus.map(_.maxOnHeapMem.getOrElse(0L)).sum) + + registerGauge(MetricRegistry.name("memory", "maxOffHeapMem_MB"), + _.getStorageStatus.map(_.maxOffHeapMem.getOrElse(0L)).sum) + + registerGauge(MetricRegistry.name("memory", "remainingMem_MB"), + _.getStorageStatus.map(_.memRemaining).sum) + + registerGauge(MetricRegistry.name("memory", "remainingOnHeapMem_MB"), + _.getStorageStatus.map(_.onHeapMemRemaining.getOrElse(0L)).sum) + + registerGauge(MetricRegistry.name("memory", "remainingOffHeapMem_MB"), + _.getStorageStatus.map(_.offHeapMemRemaining.getOrElse(0L)).sum) + + registerGauge(MetricRegistry.name("memory", "memUsed_MB"), + _.getStorageStatus.map(_.memUsed).sum) + + registerGauge(MetricRegistry.name("memory", "onHeapMemUsed_MB"), + _.getStorageStatus.map(_.onHeapMemUsed.getOrElse(0L)).sum) + + registerGauge(MetricRegistry.name("memory", "offHeapMemUsed_MB"), + _.getStorageStatus.map(_.offHeapMemUsed.getOrElse(0L)).sum) + + registerGauge(MetricRegistry.name("disk", "diskSpaceUsed_MB"), + _.getStorageStatus.map(_.diskUsed).sum) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala b/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala new file mode 100644 index 000000000000..353eac60df17 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockReplicationPolicy.scala @@ -0,0 +1,224 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging + +/** + * ::DeveloperApi:: + * BlockReplicationPrioritization provides logic for prioritizing a sequence of peers for + * replicating blocks. BlockManager will replicate to each peer returned in order until the + * desired replication order is reached. If a replication fails, prioritize() will be called + * again to get a fresh prioritization. + */ +@DeveloperApi +trait BlockReplicationPolicy { + + /** + * Method to prioritize a bunch of candidate peers of a block + * + * @param blockManagerId Id of the current BlockManager for self identification + * @param peers A list of peers of a BlockManager + * @param peersReplicatedTo Set of peers already replicated to + * @param blockId BlockId of the block being replicated. This can be used as a source of + * randomness if needed. + * @param numReplicas Number of peers we need to replicate to + * @return A prioritized list of peers. Lower the index of a peer, higher its priority. + * This returns a list of size at most `numPeersToReplicateTo`. + */ + def prioritize( + blockManagerId: BlockManagerId, + peers: Seq[BlockManagerId], + peersReplicatedTo: mutable.HashSet[BlockManagerId], + blockId: BlockId, + numReplicas: Int): List[BlockManagerId] +} + +object BlockReplicationUtils { + // scalastyle:off line.size.limit + /** + * Uses sampling algorithm by Robert Floyd. Finds a random sample in O(n) while + * minimizing space usage. Please see + * here. + * + * @param n total number of indices + * @param m number of samples needed + * @param r random number generator + * @return list of m random unique indices + */ + // scalastyle:on line.size.limit + private def getSampleIds(n: Int, m: Int, r: Random): List[Int] = { + val indices = (n - m + 1 to n).foldLeft(mutable.LinkedHashSet.empty[Int]) {case (set, i) => + val t = r.nextInt(i) + 1 + if (set.contains(t)) set + i else set + t + } + indices.map(_ - 1).toList + } + + /** + * Get a random sample of size m from the elems + * + * @param elems + * @param m number of samples needed + * @param r random number generator + * @tparam T + * @return a random list of size m. If there are fewer than m elements in elems, we just + * randomly shuffle elems + */ + def getRandomSample[T](elems: Seq[T], m: Int, r: Random): List[T] = { + if (elems.size > m) { + getSampleIds(elems.size, m, r).map(elems(_)) + } else { + r.shuffle(elems).toList + } + } +} + +@DeveloperApi +class RandomBlockReplicationPolicy + extends BlockReplicationPolicy + with Logging { + + /** + * Method to prioritize a bunch of candidate peers of a block. This is a basic implementation, + * that just makes sure we put blocks on different hosts, if possible + * + * @param blockManagerId Id of the current BlockManager for self identification + * @param peers A list of peers of a BlockManager + * @param peersReplicatedTo Set of peers already replicated to + * @param blockId BlockId of the block being replicated. This can be used as a source of + * randomness if needed. + * @param numReplicas Number of peers we need to replicate to + * @return A prioritized list of peers. Lower the index of a peer, higher its priority + */ + override def prioritize( + blockManagerId: BlockManagerId, + peers: Seq[BlockManagerId], + peersReplicatedTo: mutable.HashSet[BlockManagerId], + blockId: BlockId, + numReplicas: Int): List[BlockManagerId] = { + val random = new Random(blockId.hashCode) + logDebug(s"Input peers : ${peers.mkString(", ")}") + val prioritizedPeers = if (peers.size > numReplicas) { + BlockReplicationUtils.getRandomSample(peers, numReplicas, random) + } else { + if (peers.size < numReplicas) { + logWarning(s"Expecting ${numReplicas} replicas with only ${peers.size} peer/s.") + } + random.shuffle(peers).toList + } + logDebug(s"Prioritized peers : ${prioritizedPeers.mkString(", ")}") + prioritizedPeers + } +} + +@DeveloperApi +class BasicBlockReplicationPolicy + extends BlockReplicationPolicy + with Logging { + + /** + * Method to prioritize a bunch of candidate peers of a block manager. This implementation + * replicates the behavior of block replication in HDFS. For a given number of replicas needed, + * we choose a peer within the rack, one outside and remaining blockmanagers are chosen at + * random, in that order till we meet the number of replicas needed. + * This works best with a total replication factor of 3, like HDFS. + * + * @param blockManagerId Id of the current BlockManager for self identification + * @param peers A list of peers of a BlockManager + * @param peersReplicatedTo Set of peers already replicated to + * @param blockId BlockId of the block being replicated. This can be used as a source of + * randomness if needed. + * @param numReplicas Number of peers we need to replicate to + * @return A prioritized list of peers. Lower the index of a peer, higher its priority + */ + override def prioritize( + blockManagerId: BlockManagerId, + peers: Seq[BlockManagerId], + peersReplicatedTo: mutable.HashSet[BlockManagerId], + blockId: BlockId, + numReplicas: Int): List[BlockManagerId] = { + + logDebug(s"Input peers : $peers") + logDebug(s"BlockManagerId : $blockManagerId") + + val random = new Random(blockId.hashCode) + + // if block doesn't have topology info, we can't do much, so we randomly shuffle + // if there is, we see what's needed from peersReplicatedTo and based on numReplicas, + // we choose whats needed + if (blockManagerId.topologyInfo.isEmpty || numReplicas == 0) { + // no topology info for the block. The best we can do is randomly choose peers + BlockReplicationUtils.getRandomSample(peers, numReplicas, random) + } else { + // we have topology information, we see what is left to be done from peersReplicatedTo + val doneWithinRack = peersReplicatedTo.exists(_.topologyInfo == blockManagerId.topologyInfo) + val doneOutsideRack = peersReplicatedTo.exists { p => + p.topologyInfo.isDefined && p.topologyInfo != blockManagerId.topologyInfo + } + + if (doneOutsideRack && doneWithinRack) { + // we are done, we just return a random sample + BlockReplicationUtils.getRandomSample(peers, numReplicas, random) + } else { + // we separate peers within and outside rack + val (inRackPeers, outOfRackPeers) = peers + .filter(_.host != blockManagerId.host) + .partition(_.topologyInfo == blockManagerId.topologyInfo) + + val peerWithinRack = if (doneWithinRack) { + // we are done with in-rack replication, so don't need anymore peers + Seq.empty + } else { + if (inRackPeers.isEmpty) { + Seq.empty + } else { + Seq(inRackPeers(random.nextInt(inRackPeers.size))) + } + } + + val peerOutsideRack = if (doneOutsideRack || numReplicas - peerWithinRack.size <= 0) { + Seq.empty + } else { + if (outOfRackPeers.isEmpty) { + Seq.empty + } else { + Seq(outOfRackPeers(random.nextInt(outOfRackPeers.size))) + } + } + + val priorityPeers = peerWithinRack ++ peerOutsideRack + val numRemainingPeers = numReplicas - priorityPeers.size + val remainingPeers = if (numRemainingPeers > 0) { + val rPeers = peers.filter(p => !priorityPeers.contains(p)) + BlockReplicationUtils.getRandomSample(rPeers, numRemainingPeers, random) + } else { + Seq.empty + } + + (priorityPeers ++ remainingPeers).toList + } + + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/BlockStatusListener.scala index 2789e25b8d3a..0a14fcadf53e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStatusListener.scala @@ -26,8 +26,7 @@ private[spark] case class BlockUIData( location: String, storageLevel: StorageLevel, memSize: Long, - diskSize: Long, - externalBlockStoreSize: Long) + diskSize: Long) /** * The aggregated status of stream blocks in an executor @@ -41,8 +40,6 @@ private[spark] case class ExecutorStreamBlockStatus( def totalDiskSize: Long = blocks.map(_.diskSize).sum - def totalExternalBlockStoreSize: Long = blocks.map(_.externalBlockStoreSize).sum - def numStreamBlocks: Int = blocks.size } @@ -62,7 +59,6 @@ private[spark] class BlockStatusListener extends SparkListener { val storageLevel = blockUpdated.blockUpdatedInfo.storageLevel val memSize = blockUpdated.blockUpdatedInfo.memSize val diskSize = blockUpdated.blockUpdatedInfo.diskSize - val externalBlockStoreSize = blockUpdated.blockUpdatedInfo.externalBlockStoreSize synchronized { // Drop the update info if the block manager is not registered @@ -74,8 +70,7 @@ private[spark] class BlockStatusListener extends SparkListener { blockManagerId.hostPort, storageLevel, memSize, - diskSize, - externalBlockStoreSize) + diskSize) ) } else { // If isValid is not true, it means we should drop the block. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala deleted file mode 100644 index 69985c9759e2..000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ /dev/null @@ -1,71 +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.storage - -import java.nio.ByteBuffer - -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.Logging - -/** - * Abstract class to store blocks. - */ -private[spark] abstract class BlockStore(val blockManager: BlockManager) extends Logging { - - def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult - - /** - * Put in a block and, possibly, also return its content as either bytes or another Iterator. - * This is used to efficiently write the values to multiple locations (e.g. for replication). - * - * @return a PutResult that contains the size of the data, as well as the values put if - * returnValues is true (if not, the result's data field can be null) - */ - def putIterator( - blockId: BlockId, - values: Iterator[Any], - level: StorageLevel, - returnValues: Boolean): PutResult - - def putArray( - blockId: BlockId, - values: Array[Any], - level: StorageLevel, - returnValues: Boolean): PutResult - - /** - * Return the size of a block in bytes. - */ - def getSize(blockId: BlockId): Long - - def getBytes(blockId: BlockId): Option[ByteBuffer] - - def getValues(blockId: BlockId): Option[Iterator[Any]] - - /** - * Remove a block, if it exists. - * @param blockId the block to remove. - * @return True if the block was found and removed, False otherwise. - */ - def remove(blockId: BlockId): Boolean - - def contains(blockId: BlockId): Boolean - - def clear() { } -} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockUpdatedInfo.scala b/core/src/main/scala/org/apache/spark/storage/BlockUpdatedInfo.scala index a5790e4454a8..e070bf658acb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockUpdatedInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockUpdatedInfo.scala @@ -30,8 +30,7 @@ case class BlockUpdatedInfo( blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long, - externalBlockStoreSize: Long) + diskSize: Long) private[spark] object BlockUpdatedInfo { @@ -41,7 +40,6 @@ private[spark] object BlockUpdatedInfo { updateBlockInfo.blockId, updateBlockInfo.storageLevel, updateBlockInfo.memSize, - updateBlockInfo.diskSize, - updateBlockInfo.externalBlockStoreSize) + updateBlockInfo.diskSize) } } 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 f7e84a2c2e14..3d43e3c367aa 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -17,27 +17,24 @@ package org.apache.spark.storage +import java.io.{File, IOException} import java.util.UUID -import java.io.{IOException, File} -import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.SparkConf import org.apache.spark.executor.ExecutorExitCode +import org.apache.spark.internal.Logging import org.apache.spark.util.{ShutdownHookManager, Utils} /** * Creates and maintains the logical mapping between logical blocks and physical on-disk - * locations. By default, one block is mapped to one file with a name given by its BlockId. - * However, it is also possible to have a block map to only a segment of a file, by calling - * mapBlockToFileSegment(). + * locations. One block is mapped to one file with a name given by its BlockId. * * Block files are hashed among the directories listed in spark.local.dir (or in * SPARK_LOCAL_DIRS, if it's set). */ -private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkConf) - extends Logging { +private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolean) extends Logging { - private[spark] - val subDirsPerLocalDir = blockManager.conf.getInt("spark.diskStore.subDirectories", 64) + private[spark] val subDirsPerLocalDir = conf.getInt("spark.diskStore.subDirectories", 64) /* Create one local directory for each path mentioned in spark.local.dir; then, inside this * directory, create multiple subdirectories that we will hash files into, in order to avoid @@ -144,6 +141,7 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon } private def addShutdownHook(): AnyRef = { + logDebug("Adding shutdown hook") // force eager creation of logger ShutdownHookManager.addShutdownHook(ShutdownHookManager.TEMP_DIR_SHUTDOWN_PRIORITY + 1) { () => logInfo("Shutdown hook called") DiskBlockManager.this.doStop() @@ -163,10 +161,7 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon } private def doStop(): Unit = { - // Only perform cleanup if an external service is not serving our shuffle files. - // Also blockManagerId could be null if block manager is not initialized properly. - if (!blockManager.externalShuffleServiceEnabled || - (blockManager.blockManagerId != null && blockManager.blockManagerId.isDriver)) { + if (deleteFilesOnStop) { localDirs.foreach { localDir => if (localDir.isDirectory() && localDir.exists()) { try { diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index 80d426fadc65..a024c83d8d8b 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -17,130 +17,185 @@ package org.apache.spark.storage -import java.io.{BufferedOutputStream, FileOutputStream, File, OutputStream} +import java.io.{BufferedOutputStream, File, FileOutputStream, OutputStream} import java.nio.channels.FileChannel -import org.apache.spark.Logging -import org.apache.spark.serializer.{SerializerInstance, SerializationStream} import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.internal.Logging +import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} import org.apache.spark.util.Utils /** * A class for writing JVM objects directly to a file on disk. This class allows data to be appended - * to an existing block and can guarantee atomicity in the case of faults as it allows the caller to - * revert partial writes. + * to an existing block. For efficiency, it retains the underlying file channel across + * multiple commits. This channel is kept open until close() is called. In case of faults, + * callers should instead close with revertPartialWritesAndClose() to atomically revert the + * uncommitted partial writes. * * This class does not support concurrent writes. Also, once the writer has been opened it cannot be * reopened again. */ private[spark] class DiskBlockObjectWriter( - file: File, + val file: File, + serializerManager: SerializerManager, serializerInstance: SerializerInstance, bufferSize: Int, - compressStream: OutputStream => OutputStream, syncWrites: Boolean, // These write metrics concurrently shared with other active DiskBlockObjectWriters who // are themselves performing writes. All updates must be relative. - writeMetrics: ShuffleWriteMetrics) + writeMetrics: ShuffleWriteMetrics, + val blockId: BlockId = null) extends OutputStream with Logging { + /** + * Guards against close calls, e.g. from a wrapping stream. + * Call manualClose to close the stream that was extended by this trait. + * Commit uses this trait to close object streams without paying the + * cost of closing and opening the underlying file. + */ + private trait ManualCloseOutputStream extends OutputStream { + abstract override def close(): Unit = { + flush() + } + + def manualClose(): Unit = { + super.close() + } + } + /** The file channel, used for repositioning / truncating the file. */ private var channel: FileChannel = null + private var mcs: ManualCloseOutputStream = null private var bs: OutputStream = null private var fos: FileOutputStream = null private var ts: TimeTrackingOutputStream = null private var objOut: SerializationStream = null private var initialized = false + private var streamOpen = false private var hasBeenClosed = false - private var commitAndCloseHasBeenCalled = false /** * Cursors used to represent positions in the file. * - * xxxxxxxx|--------|--- | - * ^ ^ ^ - * | | finalPosition - * | reportedPosition - * initialPosition + * xxxxxxxxxx|----------|-----| + * ^ ^ ^ + * | | channel.position() + * | reportedPosition + * committedPosition * - * initialPosition: Offset in the file where we start writing. Immutable. * reportedPosition: Position at the time of the last update to the write metrics. - * finalPosition: Offset where we stopped writing. Set on closeAndCommit() then never changed. + * committedPosition: Offset after last committed write. * -----: Current writes to the underlying file. - * xxxxx: Existing contents of the file. + * xxxxx: Committed contents of the file. */ - private val initialPosition = file.length() - private var finalPosition: Long = -1 - private var reportedPosition = initialPosition + private var committedPosition = file.length() + private var reportedPosition = committedPosition /** * Keep track of number of records written and also use this to periodically * output bytes written since the latter is expensive to do for each record. + * And we reset it after every commitAndGet called. */ private var numRecordsWritten = 0 + private def initialize(): Unit = { + fos = new FileOutputStream(file, true) + channel = fos.getChannel() + ts = new TimeTrackingOutputStream(writeMetrics, fos) + class ManualCloseBufferedOutputStream + extends BufferedOutputStream(ts, bufferSize) with ManualCloseOutputStream + mcs = new ManualCloseBufferedOutputStream + } + def open(): DiskBlockObjectWriter = { if (hasBeenClosed) { throw new IllegalStateException("Writer already closed. Cannot be reopened.") } - fos = new FileOutputStream(file, true) - ts = new TimeTrackingOutputStream(writeMetrics, fos) - channel = fos.getChannel() - bs = compressStream(new BufferedOutputStream(ts, bufferSize)) + if (!initialized) { + initialize() + initialized = true + } + + bs = serializerManager.wrapStream(blockId, mcs) objOut = serializerInstance.serializeStream(bs) - initialized = true + streamOpen = true this } - override def close() { + /** + * Close and cleanup all resources. + * Should call after committing or reverting partial writes. + */ + private def closeResources(): Unit = { if (initialized) { Utils.tryWithSafeFinally { - if (syncWrites) { - // Force outstanding writes to disk and track how long it takes - objOut.flush() - val start = System.nanoTime() - fos.getFD.sync() - writeMetrics.incShuffleWriteTime(System.nanoTime() - start) - } + mcs.manualClose() } { - objOut.close() + channel = null + mcs = null + bs = null + fos = null + ts = null + objOut = null + initialized = false + streamOpen = false + hasBeenClosed = true } - - channel = null - bs = null - fos = null - ts = null - objOut = null - initialized = false - hasBeenClosed = true } } - def isOpen: Boolean = objOut != null + /** + * Commits any remaining partial writes and closes resources. + */ + override def close() { + if (initialized) { + Utils.tryWithSafeFinally { + commitAndGet() + } { + closeResources() + } + } + } /** * Flush the partial writes and commit them as a single atomic block. + * A commit may write additional bytes to frame the atomic block. + * + * @return file segment with previous offset and length committed on this call. */ - def commitAndClose(): Unit = { - if (initialized) { + def commitAndGet(): FileSegment = { + if (streamOpen) { // NOTE: Because Kryo doesn't flush the underlying stream we explicitly flush both the // serializer stream and the lower level stream. objOut.flush() bs.flush() - close() - finalPosition = file.length() - // In certain compression codecs, more bytes are written after close() is called - writeMetrics.incShuffleBytesWritten(finalPosition - reportedPosition) + objOut.close() + streamOpen = false + + if (syncWrites) { + // Force outstanding writes to disk and track how long it takes + val start = System.nanoTime() + fos.getFD.sync() + writeMetrics.incWriteTime(System.nanoTime() - start) + } + + val pos = channel.position() + val fileSegment = new FileSegment(file, committedPosition, pos - committedPosition) + committedPosition = pos + // In certain compression codecs, more bytes are written after streams are closed + writeMetrics.incBytesWritten(committedPosition - reportedPosition) + reportedPosition = committedPosition + numRecordsWritten = 0 + fileSegment } else { - finalPosition = file.length() + new FileSegment(file, committedPosition, 0) } - commitAndCloseHasBeenCalled = true } /** - * Reverts writes that haven't been flushed yet. Callers should invoke this function + * Reverts writes that haven't been committed yet. Callers should invoke this function * when there are runtime exceptions. This method will not throw, though it may be * unsuccessful in truncating written data. * @@ -149,34 +204,36 @@ private[spark] class DiskBlockObjectWriter( def revertPartialWritesAndClose(): File = { // Discard current writes. We do this by flushing the outstanding writes and then // truncating the file to its initial position. - try { + Utils.tryWithSafeFinally { if (initialized) { - writeMetrics.decShuffleBytesWritten(reportedPosition - initialPosition) - writeMetrics.decShuffleRecordsWritten(numRecordsWritten) - objOut.flush() - bs.flush() - close() + writeMetrics.decBytesWritten(reportedPosition - committedPosition) + writeMetrics.decRecordsWritten(numRecordsWritten) + streamOpen = false + closeResources() } - - val truncateStream = new FileOutputStream(file, true) + } { + var truncateStream: FileOutputStream = null try { - truncateStream.getChannel.truncate(initialPosition) - file + truncateStream = new FileOutputStream(file, true) + truncateStream.getChannel.truncate(committedPosition) + } catch { + case e: Exception => + logError("Uncaught exception while reverting partial writes to file " + file, e) } finally { - truncateStream.close() + if (truncateStream != null) { + truncateStream.close() + truncateStream = null + } } - } catch { - case e: Exception => - logError("Uncaught exception while reverting partial writes to file " + file, e) - file } + file } /** * Writes a key-value pair. */ def write(key: Any, value: Any) { - if (!initialized) { + if (!streamOpen) { open() } @@ -188,7 +245,7 @@ private[spark] class DiskBlockObjectWriter( override def write(b: Int): Unit = throw new UnsupportedOperationException() override def write(kvBytes: Array[Byte], offs: Int, len: Int): Unit = { - if (!initialized) { + if (!streamOpen) { open() } @@ -200,32 +257,20 @@ private[spark] class DiskBlockObjectWriter( */ def recordWritten(): Unit = { numRecordsWritten += 1 - writeMetrics.incShuffleRecordsWritten(1) + writeMetrics.incRecordsWritten(1) - if (numRecordsWritten % 32 == 0) { + if (numRecordsWritten % 16384 == 0) { updateBytesWritten() } } - /** - * Returns the file segment of committed data that this Writer has written. - * This is only valid after commitAndClose() has been called. - */ - def fileSegment(): FileSegment = { - if (!commitAndCloseHasBeenCalled) { - throw new IllegalStateException( - "fileSegment() is only valid after commitAndClose() has been called") - } - new FileSegment(file, initialPosition, finalPosition - initialPosition) - } - /** * Report the number of bytes written in this writer's shuffle write metrics. * Note that this is only valid before the underlying streams are closed. */ private def updateBytesWritten() { val pos = channel.position() - writeMetrics.incShuffleBytesWritten(pos - reportedPosition) + writeMetrics.incBytesWritten(pos - reportedPosition) reportedPosition = pos } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index c008b9dc1632..3579acf8d83d 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -17,158 +17,310 @@ package org.apache.spark.storage -import java.io.{IOException, File, FileOutputStream, RandomAccessFile} +import java.io._ import java.nio.ByteBuffer +import java.nio.channels.{Channels, ReadableByteChannel, WritableByteChannel} import java.nio.channels.FileChannel.MapMode +import java.util.concurrent.ConcurrentHashMap -import org.apache.spark.Logging -import org.apache.spark.serializer.Serializer +import scala.collection.mutable.ListBuffer + +import com.google.common.io.Closeables +import io.netty.channel.{DefaultFileRegion, FileRegion} +import io.netty.util.AbstractReferenceCounted + +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.internal.Logging +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.security.CryptoStreamUtils import org.apache.spark.util.Utils +import org.apache.spark.util.io.ChunkedByteBuffer /** * Stores BlockManager blocks on disk. */ -private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) - extends BlockStore(blockManager) with Logging { +private[spark] class DiskStore( + conf: SparkConf, + diskManager: DiskBlockManager, + securityManager: SecurityManager) extends Logging { - val minMemoryMapBytes = blockManager.conf.getSizeAsBytes("spark.storage.memoryMapThreshold", "2m") + private val minMemoryMapBytes = conf.getSizeAsBytes("spark.storage.memoryMapThreshold", "2m") + private val maxMemoryMapBytes = conf.getSizeAsBytes("spark.storage.memoryMapLimitForTests", + Int.MaxValue.toString) + private val blockSizes = new ConcurrentHashMap[String, Long]() - override def getSize(blockId: BlockId): Long = { - diskManager.getFile(blockId.name).length - } + def getSize(blockId: BlockId): Long = blockSizes.get(blockId.name) - override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): PutResult = { - // So that we do not modify the input offsets ! - // duplicate does not copy buffer, so inexpensive - val bytes = _bytes.duplicate() + /** + * Invokes the provided callback function to write the specific block. + * + * @throws IllegalStateException if the block already exists in the disk store. + */ + def put(blockId: BlockId)(writeFunc: WritableByteChannel => Unit): Unit = { + if (contains(blockId)) { + throw new IllegalStateException(s"Block $blockId is already present in the disk store") + } logDebug(s"Attempting to put block $blockId") val startTime = System.currentTimeMillis val file = diskManager.getFile(blockId) - val channel = new FileOutputStream(file).getChannel - Utils.tryWithSafeFinally { - while (bytes.remaining > 0) { - channel.write(bytes) + val out = new CountingWritableChannel(openForWrite(file)) + var threwException: Boolean = true + try { + writeFunc(out) + blockSizes.put(blockId.name, out.getCount) + threwException = false + } finally { + try { + out.close() + } catch { + case ioe: IOException => + if (!threwException) { + threwException = true + throw ioe + } + } finally { + if (threwException) { + remove(blockId) + } } - } { - channel.close() } val finishTime = System.currentTimeMillis logDebug("Block %s stored as %s file on disk in %d ms".format( - file.getName, Utils.bytesToString(bytes.limit), finishTime - startTime)) - PutResult(bytes.limit(), Right(bytes.duplicate())) + file.getName, + Utils.bytesToString(file.length()), + finishTime - startTime)) } - override def putArray( - blockId: BlockId, - values: Array[Any], - level: StorageLevel, - returnValues: Boolean): PutResult = { - putIterator(blockId, values.toIterator, level, returnValues) + def putBytes(blockId: BlockId, bytes: ChunkedByteBuffer): Unit = { + put(blockId) { channel => + bytes.writeFully(channel) + } } - override def putIterator( - blockId: BlockId, - values: Iterator[Any], - level: StorageLevel, - returnValues: Boolean): PutResult = { + def getBytes(blockId: BlockId): BlockData = { + val file = diskManager.getFile(blockId.name) + val blockSize = getSize(blockId) - logDebug(s"Attempting to write values for block $blockId") - val startTime = System.currentTimeMillis - val file = diskManager.getFile(blockId) - val outputStream = new FileOutputStream(file) - try { - Utils.tryWithSafeFinally { - blockManager.dataSerializeStream(blockId, outputStream, values) - } { - // Close outputStream here because it should be closed before file is deleted. - outputStream.close() + securityManager.getIOEncryptionKey() match { + case Some(key) => + // Encrypted blocks cannot be memory mapped; return a special object that does decryption + // and provides InputStream / FileRegion implementations for reading the data. + new EncryptedBlockData(file, blockSize, conf, key) + + case _ => + new DiskBlockData(minMemoryMapBytes, maxMemoryMapBytes, file, blockSize) + } + } + + def remove(blockId: BlockId): Boolean = { + blockSizes.remove(blockId.name) + val file = diskManager.getFile(blockId.name) + if (file.exists()) { + val ret = file.delete() + if (!ret) { + logWarning(s"Error deleting ${file.getPath()}") } + ret + } else { + false + } + } + + def contains(blockId: BlockId): Boolean = { + val file = diskManager.getFile(blockId.name) + file.exists() + } + + private def openForWrite(file: File): WritableByteChannel = { + val out = new FileOutputStream(file).getChannel() + try { + securityManager.getIOEncryptionKey().map { key => + CryptoStreamUtils.createWritableChannel(out, conf, key) + }.getOrElse(out) } catch { - case e: Throwable => - if (file.exists()) { - if (!file.delete()) { - logWarning(s"Error deleting ${file}") - } - } + case e: Exception => + Closeables.close(out, true) + file.delete() throw e } + } - val length = file.length +} - val timeTaken = System.currentTimeMillis - startTime - logDebug("Block %s stored as %s file on disk in %d ms".format( - file.getName, Utils.bytesToString(length), timeTaken)) +private class DiskBlockData( + minMemoryMapBytes: Long, + maxMemoryMapBytes: Long, + file: File, + blockSize: Long) extends BlockData { - if (returnValues) { - // Return a byte buffer for the contents of the file - val buffer = getBytes(blockId).get - PutResult(length, Right(buffer)) - } else { - PutResult(length, null) + override def toInputStream(): InputStream = new FileInputStream(file) + + /** + * Returns a Netty-friendly wrapper for the block's data. + * + * Please see `ManagedBuffer.convertToNetty()` for more details. + */ + override def toNetty(): AnyRef = new DefaultFileRegion(file, 0, size) + + override def toChunkedByteBuffer(allocator: (Int) => ByteBuffer): ChunkedByteBuffer = { + Utils.tryWithResource(open()) { channel => + var remaining = blockSize + val chunks = new ListBuffer[ByteBuffer]() + while (remaining > 0) { + val chunkSize = math.min(remaining, maxMemoryMapBytes) + val chunk = allocator(chunkSize.toInt) + remaining -= chunkSize + JavaUtils.readFully(channel, chunk) + chunk.flip() + chunks += chunk + } + new ChunkedByteBuffer(chunks.toArray) } } - private def getBytes(file: File, offset: Long, length: Long): Option[ByteBuffer] = { - val channel = new RandomAccessFile(file, "r").getChannel - Utils.tryWithSafeFinally { - // For small files, directly read rather than memory map - if (length < minMemoryMapBytes) { - val buf = ByteBuffer.allocate(length.toInt) - channel.position(offset) - while (buf.remaining() != 0) { - if (channel.read(buf) == -1) { - throw new IOException("Reached EOF before filling buffer\n" + - s"offset=$offset\nfile=${file.getAbsolutePath}\nbuf.remaining=${buf.remaining}") - } - } + override def toByteBuffer(): ByteBuffer = { + require(blockSize < maxMemoryMapBytes, + s"can't create a byte buffer of size $blockSize" + + s" since it exceeds ${Utils.bytesToString(maxMemoryMapBytes)}.") + Utils.tryWithResource(open()) { channel => + if (blockSize < minMemoryMapBytes) { + // For small files, directly read rather than memory map. + val buf = ByteBuffer.allocate(blockSize.toInt) + JavaUtils.readFully(channel, buf) buf.flip() - Some(buf) + buf } else { - Some(channel.map(MapMode.READ_ONLY, offset, length)) + channel.map(MapMode.READ_ONLY, 0, file.length) } - } { - channel.close() } } - override def getBytes(blockId: BlockId): Option[ByteBuffer] = { - val file = diskManager.getFile(blockId.name) - getBytes(file, 0, file.length) - } + override def size: Long = blockSize + + override def dispose(): Unit = {} - def getBytes(segment: FileSegment): Option[ByteBuffer] = { - getBytes(segment.file, segment.offset, segment.length) + private def open() = new FileInputStream(file).getChannel +} + +private class EncryptedBlockData( + file: File, + blockSize: Long, + conf: SparkConf, + key: Array[Byte]) extends BlockData { + + override def toInputStream(): InputStream = Channels.newInputStream(open()) + + override def toNetty(): Object = new ReadableChannelFileRegion(open(), blockSize) + + override def toChunkedByteBuffer(allocator: Int => ByteBuffer): ChunkedByteBuffer = { + val source = open() + try { + var remaining = blockSize + val chunks = new ListBuffer[ByteBuffer]() + while (remaining > 0) { + val chunkSize = math.min(remaining, Int.MaxValue) + val chunk = allocator(chunkSize.toInt) + remaining -= chunkSize + JavaUtils.readFully(source, chunk) + chunk.flip() + chunks += chunk + } + + new ChunkedByteBuffer(chunks.toArray) + } finally { + source.close() + } } - override def getValues(blockId: BlockId): Option[Iterator[Any]] = { - getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer)) + override def toByteBuffer(): ByteBuffer = { + // This is used by the block transfer service to replicate blocks. The upload code reads + // all bytes into memory to send the block to the remote executor, so it's ok to do this + // as long as the block fits in a Java array. + assert(blockSize <= Int.MaxValue, "Block is too large to be wrapped in a byte buffer.") + val dst = ByteBuffer.allocate(blockSize.toInt) + val in = open() + try { + JavaUtils.readFully(in, dst) + dst.flip() + dst + } finally { + Closeables.close(in, true) + } } - /** - * A version of getValues that allows a custom serializer. This is used as part of the - * shuffle short-circuit code. - */ - def getValues(blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = { - // TODO: Should bypass getBytes and use a stream based implementation, so that - // we won't use a lot of memory during e.g. external sort merge. - getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes, serializer)) + override def size: Long = blockSize + + override def dispose(): Unit = { } + + private def open(): ReadableByteChannel = { + val channel = new FileInputStream(file).getChannel() + try { + CryptoStreamUtils.createReadableChannel(channel, conf, key) + } catch { + case e: Exception => + Closeables.close(channel, true) + throw e + } } - override def remove(blockId: BlockId): Boolean = { - val file = diskManager.getFile(blockId.name) - if (file.exists()) { - val ret = file.delete() - if (!ret) { - logWarning(s"Error deleting ${file.getPath()}") +} + +private class ReadableChannelFileRegion(source: ReadableByteChannel, blockSize: Long) + extends AbstractReferenceCounted with FileRegion { + + private var _transferred = 0L + + private val buffer = ByteBuffer.allocateDirect(64 * 1024) + buffer.flip() + + override def count(): Long = blockSize + + override def position(): Long = 0 + + override def transfered(): Long = _transferred + + override def transferTo(target: WritableByteChannel, pos: Long): Long = { + assert(pos == transfered(), "Invalid position.") + + var written = 0L + var lastWrite = -1L + while (lastWrite != 0) { + if (!buffer.hasRemaining()) { + buffer.clear() + source.read(buffer) + buffer.flip() + } + if (buffer.hasRemaining()) { + lastWrite = target.write(buffer) + written += lastWrite + } else { + lastWrite = 0 } - ret - } else { - false } + + _transferred += written + written } - override def contains(blockId: BlockId): Boolean = { - val file = diskManager.getFile(blockId.name) - file.exists() + override def deallocate(): Unit = source.close() +} + +private class CountingWritableChannel(sink: WritableByteChannel) extends WritableByteChannel { + + private var count = 0L + + def getCount: Long = count + + override def write(src: ByteBuffer): Int = { + val written = sink.write(src) + if (written > 0) { + count += written + } + written } + + override def isOpen(): Boolean = sink.isOpen() + + override def close(): Unit = sink.close() + } diff --git a/core/src/main/scala/org/apache/spark/storage/ExternalBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ExternalBlockManager.scala deleted file mode 100644 index f39325a12d24..000000000000 --- a/core/src/main/scala/org/apache/spark/storage/ExternalBlockManager.scala +++ /dev/null @@ -1,122 +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.storage - -import java.nio.ByteBuffer - -/** - * An abstract class that the concrete external block manager has to inherit. - * The class has to have a no-argument constructor, and will be initialized by init, - * which is invoked by ExternalBlockStore. The main input parameter is blockId for all - * the methods, which is the unique identifier for Block in one Spark application. - * - * The underlying external block manager should avoid any name space conflicts among multiple - * Spark applications. For example, creating different directory for different applications - * by randomUUID - * - */ -private[spark] abstract class ExternalBlockManager { - - protected var blockManager: BlockManager = _ - - override def toString: String = {"External Block Store"} - - /** - * Initialize a concrete block manager implementation. Subclass should initialize its internal - * data structure, e.g, file system, in this function, which is invoked by ExternalBlockStore - * right after the class is constructed. The function should throw IOException on failure - * - * @throws java.io.IOException if there is any file system failure during the initialization. - */ - def init(blockManager: BlockManager, executorId: String): Unit = { - this.blockManager = blockManager - } - - /** - * Drop the block from underlying external block store, if it exists.. - * @return true on successfully removing the block - * false if the block could not be removed as it was not found - * - * @throws java.io.IOException if there is any file system failure in removing the block. - */ - def removeBlock(blockId: BlockId): Boolean - - /** - * Used by BlockManager to check the existence of the block in the underlying external - * block store. - * @return true if the block exists. - * false if the block does not exists. - * - * @throws java.io.IOException if there is any file system failure in checking - * the block existence. - */ - def blockExists(blockId: BlockId): Boolean - - /** - * Put the given block to the underlying external block store. Note that in normal case, - * putting a block should never fail unless something wrong happens to the underlying - * external block store, e.g., file system failure, etc. In this case, IOException - * should be thrown. - * - * @throws java.io.IOException if there is any file system failure in putting the block. - */ - def putBytes(blockId: BlockId, bytes: ByteBuffer): Unit - - def putValues(blockId: BlockId, values: Iterator[_]): Unit = { - val bytes = blockManager.dataSerialize(blockId, values) - putBytes(blockId, bytes) - } - - /** - * Retrieve the block bytes. - * @return Some(ByteBuffer) if the block bytes is successfully retrieved - * None if the block does not exist in the external block store. - * - * @throws java.io.IOException if there is any file system failure in getting the block. - */ - def getBytes(blockId: BlockId): Option[ByteBuffer] - - /** - * Retrieve the block data. - * @return Some(Iterator[Any]) if the block data is successfully retrieved - * None if the block does not exist in the external block store. - * - * @throws java.io.IOException if there is any file system failure in getting the block. - */ - def getValues(blockId: BlockId): Option[Iterator[_]] = { - getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer)) - } - - /** - * Get the size of the block saved in the underlying external block store, - * which is saved before by putBytes. - * @return size of the block - * 0 if the block does not exist - * - * @throws java.io.IOException if there is any file system failure in getting the block size. - */ - def getSize(blockId: BlockId): Long - - /** - * Clean up any information persisted in the underlying external block store, - * e.g., the directory, files, etc,which is invoked by the shutdown hook of ExternalBlockStore - * during system shutdown. - * - */ - def shutdown() -} diff --git a/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala b/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala deleted file mode 100644 index db965d54bafd..000000000000 --- a/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala +++ /dev/null @@ -1,217 +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.storage - -import java.nio.ByteBuffer - -import scala.util.control.NonFatal - -import org.apache.spark.Logging -import org.apache.spark.util.Utils - - -/** - * Stores BlockManager blocks on ExternalBlockStore. - * We capture any potential exception from underlying implementation - * and return with the expected failure value - */ -private[spark] class ExternalBlockStore(blockManager: BlockManager, executorId: String) - extends BlockStore(blockManager: BlockManager) with Logging { - - lazy val externalBlockManager: Option[ExternalBlockManager] = createBlkManager() - - logInfo("ExternalBlockStore started") - - override def getSize(blockId: BlockId): Long = { - try { - externalBlockManager.map(_.getSize(blockId)).getOrElse(0) - } catch { - case NonFatal(t) => - logError(s"Error in getSize($blockId)", t) - 0L - } - } - - override def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult = { - putIntoExternalBlockStore(blockId, bytes, returnValues = true) - } - - override def putArray( - blockId: BlockId, - values: Array[Any], - level: StorageLevel, - returnValues: Boolean): PutResult = { - putIntoExternalBlockStore(blockId, values.toIterator, returnValues) - } - - override def putIterator( - blockId: BlockId, - values: Iterator[Any], - level: StorageLevel, - returnValues: Boolean): PutResult = { - putIntoExternalBlockStore(blockId, values, returnValues) - } - - private def putIntoExternalBlockStore( - blockId: BlockId, - values: Iterator[_], - returnValues: Boolean): PutResult = { - logTrace(s"Attempting to put block $blockId into ExternalBlockStore") - // we should never hit here if externalBlockManager is None. Handle it anyway for safety. - try { - val startTime = System.currentTimeMillis - if (externalBlockManager.isDefined) { - externalBlockManager.get.putValues(blockId, values) - val size = getSize(blockId) - val data = if (returnValues) { - Left(getValues(blockId).get) - } else { - null - } - val finishTime = System.currentTimeMillis - logDebug("Block %s stored as %s file in ExternalBlockStore in %d ms".format( - blockId, Utils.bytesToString(size), finishTime - startTime)) - PutResult(size, data) - } else { - logError(s"Error in putValues($blockId): no ExternalBlockManager has been configured") - PutResult(-1, null, Seq((blockId, BlockStatus.empty))) - } - } catch { - case NonFatal(t) => - logError(s"Error in putValues($blockId)", t) - PutResult(-1, null, Seq((blockId, BlockStatus.empty))) - } - } - - private def putIntoExternalBlockStore( - blockId: BlockId, - bytes: ByteBuffer, - returnValues: Boolean): PutResult = { - logTrace(s"Attempting to put block $blockId into ExternalBlockStore") - // we should never hit here if externalBlockManager is None. Handle it anyway for safety. - try { - val startTime = System.currentTimeMillis - if (externalBlockManager.isDefined) { - val byteBuffer = bytes.duplicate() - byteBuffer.rewind() - externalBlockManager.get.putBytes(blockId, byteBuffer) - val size = bytes.limit() - val data = if (returnValues) { - Right(bytes) - } else { - null - } - val finishTime = System.currentTimeMillis - logDebug("Block %s stored as %s file in ExternalBlockStore in %d ms".format( - blockId, Utils.bytesToString(size), finishTime - startTime)) - PutResult(size, data) - } else { - logError(s"Error in putBytes($blockId): no ExternalBlockManager has been configured") - PutResult(-1, null, Seq((blockId, BlockStatus.empty))) - } - } catch { - case NonFatal(t) => - logError(s"Error in putBytes($blockId)", t) - PutResult(-1, null, Seq((blockId, BlockStatus.empty))) - } - } - - // We assume the block is removed even if exception thrown - override def remove(blockId: BlockId): Boolean = { - try { - externalBlockManager.map(_.removeBlock(blockId)).getOrElse(true) - } catch { - case NonFatal(t) => - logError(s"Error in removeBlock($blockId)", t) - true - } - } - - override def getValues(blockId: BlockId): Option[Iterator[Any]] = { - try { - externalBlockManager.flatMap(_.getValues(blockId)) - } catch { - case NonFatal(t) => - logError(s"Error in getValues($blockId)", t) - None - } - } - - override def getBytes(blockId: BlockId): Option[ByteBuffer] = { - try { - externalBlockManager.flatMap(_.getBytes(blockId)) - } catch { - case NonFatal(t) => - logError(s"Error in getBytes($blockId)", t) - None - } - } - - override def contains(blockId: BlockId): Boolean = { - try { - val ret = externalBlockManager.map(_.blockExists(blockId)).getOrElse(false) - if (!ret) { - logInfo(s"Remove block $blockId") - blockManager.removeBlock(blockId, true) - } - ret - } catch { - case NonFatal(t) => - logError(s"Error in getBytes($blockId)", t) - false - } - } - - private def addShutdownHook() { - Runtime.getRuntime.addShutdownHook(new Thread("ExternalBlockStore shutdown hook") { - override def run(): Unit = Utils.logUncaughtExceptions { - logDebug("Shutdown hook called") - externalBlockManager.map(_.shutdown()) - } - }) - } - - // Create concrete block manager and fall back to Tachyon by default for backward compatibility. - private def createBlkManager(): Option[ExternalBlockManager] = { - val clsName = blockManager.conf.getOption(ExternalBlockStore.BLOCK_MANAGER_NAME) - .getOrElse(ExternalBlockStore.DEFAULT_BLOCK_MANAGER_NAME) - - try { - val instance = Utils.classForName(clsName) - .newInstance() - .asInstanceOf[ExternalBlockManager] - instance.init(blockManager, executorId) - addShutdownHook(); - Some(instance) - } catch { - case NonFatal(t) => - logError("Cannot initialize external block store", t) - None - } - } -} - -private[spark] object ExternalBlockStore extends Logging { - val MAX_DIR_CREATION_ATTEMPTS = 10 - val SUB_DIRS_PER_DIR = "64" - val BASE_DIR = "spark.externalBlockStore.baseDir" - val FOLD_NAME = "spark.externalBlockStore.folderName" - val MASTER_URL = "spark.externalBlockStore.url" - val BLOCK_MANAGER_NAME = "spark.externalBlockStore.blockManager" - val DEFAULT_BLOCK_MANAGER_NAME = "org.apache.spark.storage.TachyonBlockManager" -} diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala deleted file mode 100644 index 4dbac388e098..000000000000 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ /dev/null @@ -1,625 +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.storage - -import java.nio.ByteBuffer -import java.util.LinkedHashMap - -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.TaskContext -import org.apache.spark.memory.MemoryManager -import org.apache.spark.util.{SizeEstimator, Utils} -import org.apache.spark.util.collection.SizeTrackingVector - -private case class MemoryEntry(value: Any, size: Long, deserialized: Boolean) - -/** - * Stores blocks in memory, either as Arrays of deserialized Java objects or as - * serialized ByteBuffers. - */ -private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: MemoryManager) - extends BlockStore(blockManager) { - - // Note: all changes to memory allocations, notably putting blocks, evicting blocks, and - // acquiring or releasing unroll memory, must be synchronized on `memoryManager`! - - private val conf = blockManager.conf - private val entries = new LinkedHashMap[BlockId, MemoryEntry](32, 0.75f, true) - - // A mapping from taskAttemptId to amount of memory used for unrolling a block (in bytes) - // All accesses of this map are assumed to have manually synchronized on `memoryManager` - private val unrollMemoryMap = mutable.HashMap[Long, Long]() - // Same as `unrollMemoryMap`, but for pending unroll memory as defined below. - // Pending unroll memory refers to the intermediate memory occupied by a task - // after the unroll but before the actual putting of the block in the cache. - // This chunk of memory is expected to be released *as soon as* we finish - // caching the corresponding block as opposed to until after the task finishes. - // This is only used if a block is successfully unrolled in its entirety in - // memory (SPARK-4777). - private val pendingUnrollMemoryMap = mutable.HashMap[Long, Long]() - - // Initial memory to request before unrolling any block - private val unrollMemoryThreshold: Long = - conf.getLong("spark.storage.unrollMemoryThreshold", 1024 * 1024) - - /** Total amount of memory available for storage, in bytes. */ - private def maxMemory: Long = memoryManager.maxStorageMemory - - if (maxMemory < unrollMemoryThreshold) { - logWarning(s"Max memory ${Utils.bytesToString(maxMemory)} is less than the initial memory " + - s"threshold ${Utils.bytesToString(unrollMemoryThreshold)} needed to store a block in " + - s"memory. Please configure Spark with more memory.") - } - - logInfo("MemoryStore started with capacity %s".format(Utils.bytesToString(maxMemory))) - - /** Total storage memory used including unroll memory, in bytes. */ - private def memoryUsed: Long = memoryManager.storageMemoryUsed - - /** - * Amount of storage memory, in bytes, used for caching blocks. - * This does not include memory used for unrolling. - */ - private def blocksMemoryUsed: Long = memoryManager.synchronized { - memoryUsed - currentUnrollMemory - } - - override def getSize(blockId: BlockId): Long = { - entries.synchronized { - entries.get(blockId).size - } - } - - override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): PutResult = { - // Work on a duplicate - since the original input might be used elsewhere. - val bytes = _bytes.duplicate() - bytes.rewind() - if (level.deserialized) { - val values = blockManager.dataDeserialize(blockId, bytes) - putIterator(blockId, values, level, returnValues = true) - } else { - val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - tryToPut(blockId, bytes, bytes.limit, deserialized = false, droppedBlocks) - PutResult(bytes.limit(), Right(bytes.duplicate()), droppedBlocks) - } - } - - /** - * Use `size` to test if there is enough space in MemoryStore. If so, create the ByteBuffer and - * put it into MemoryStore. Otherwise, the ByteBuffer won't be created. - * - * The caller should guarantee that `size` is correct. - */ - def putBytes(blockId: BlockId, size: Long, _bytes: () => ByteBuffer): PutResult = { - // Work on a duplicate - since the original input might be used elsewhere. - lazy val bytes = _bytes().duplicate().rewind().asInstanceOf[ByteBuffer] - val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - val putSuccess = tryToPut(blockId, () => bytes, size, deserialized = false, droppedBlocks) - val data = - if (putSuccess) { - assert(bytes.limit == size) - Right(bytes.duplicate()) - } else { - null - } - PutResult(size, data, droppedBlocks) - } - - override def putArray( - blockId: BlockId, - values: Array[Any], - level: StorageLevel, - returnValues: Boolean): PutResult = { - val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - if (level.deserialized) { - val sizeEstimate = SizeEstimator.estimate(values.asInstanceOf[AnyRef]) - tryToPut(blockId, values, sizeEstimate, deserialized = true, droppedBlocks) - PutResult(sizeEstimate, Left(values.iterator), droppedBlocks) - } else { - val bytes = blockManager.dataSerialize(blockId, values.iterator) - tryToPut(blockId, bytes, bytes.limit, deserialized = false, droppedBlocks) - PutResult(bytes.limit(), Right(bytes.duplicate()), droppedBlocks) - } - } - - override def putIterator( - blockId: BlockId, - values: Iterator[Any], - level: StorageLevel, - returnValues: Boolean): PutResult = { - putIterator(blockId, values, level, returnValues, allowPersistToDisk = true) - } - - /** - * Attempt to put the given block in memory store. - * - * There may not be enough space to fully unroll the iterator in memory, in which case we - * optionally drop the values to disk if - * (1) the block's storage level specifies useDisk, and - * (2) `allowPersistToDisk` is true. - * - * One scenario in which `allowPersistToDisk` is false is when the BlockManager reads a block - * back from disk and attempts to cache it in memory. In this case, we should not persist the - * block back on disk again, as it is already in disk store. - */ - private[storage] def putIterator( - blockId: BlockId, - values: Iterator[Any], - level: StorageLevel, - returnValues: Boolean, - allowPersistToDisk: Boolean): PutResult = { - val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - val unrolledValues = unrollSafely(blockId, values, droppedBlocks) - unrolledValues match { - case Left(arrayValues) => - // Values are fully unrolled in memory, so store them as an array - val res = putArray(blockId, arrayValues, level, returnValues) - droppedBlocks ++= res.droppedBlocks - PutResult(res.size, res.data, droppedBlocks) - case Right(iteratorValues) => - // Not enough space to unroll this block; drop to disk if applicable - if (level.useDisk && allowPersistToDisk) { - logWarning(s"Persisting block $blockId to disk instead.") - val res = blockManager.diskStore.putIterator(blockId, iteratorValues, level, returnValues) - PutResult(res.size, res.data, droppedBlocks) - } else { - PutResult(0, Left(iteratorValues), droppedBlocks) - } - } - } - - override def getBytes(blockId: BlockId): Option[ByteBuffer] = { - val entry = entries.synchronized { - entries.get(blockId) - } - if (entry == null) { - None - } else if (entry.deserialized) { - Some(blockManager.dataSerialize(blockId, entry.value.asInstanceOf[Array[Any]].iterator)) - } else { - Some(entry.value.asInstanceOf[ByteBuffer].duplicate()) // Doesn't actually copy the data - } - } - - override def getValues(blockId: BlockId): Option[Iterator[Any]] = { - val entry = entries.synchronized { - entries.get(blockId) - } - if (entry == null) { - None - } else if (entry.deserialized) { - Some(entry.value.asInstanceOf[Array[Any]].iterator) - } else { - val buffer = entry.value.asInstanceOf[ByteBuffer].duplicate() // Doesn't actually copy data - Some(blockManager.dataDeserialize(blockId, buffer)) - } - } - - override def remove(blockId: BlockId): Boolean = memoryManager.synchronized { - val entry = entries.synchronized { entries.remove(blockId) } - if (entry != null) { - memoryManager.releaseStorageMemory(entry.size) - logDebug(s"Block $blockId of size ${entry.size} dropped " + - s"from memory (free ${maxMemory - blocksMemoryUsed})") - true - } else { - false - } - } - - override def clear(): Unit = memoryManager.synchronized { - entries.synchronized { - entries.clear() - } - unrollMemoryMap.clear() - pendingUnrollMemoryMap.clear() - memoryManager.releaseAllStorageMemory() - logInfo("MemoryStore cleared") - } - - /** - * Unroll the given block in memory safely. - * - * The safety of this operation refers to avoiding potential OOM exceptions caused by - * unrolling the entirety of the block in memory at once. This is achieved by periodically - * checking whether the memory restrictions for unrolling blocks are still satisfied, - * stopping immediately if not. This check is a safeguard against the scenario in which - * there is not enough free memory to accommodate the entirety of a single block. - * - * This method returns either an array with the contents of the entire block or an iterator - * containing the values of the block (if the array would have exceeded available memory). - */ - def unrollSafely( - blockId: BlockId, - values: Iterator[Any], - droppedBlocks: ArrayBuffer[(BlockId, BlockStatus)]) - : Either[Array[Any], Iterator[Any]] = { - - // Number of elements unrolled so far - var elementsUnrolled = 0 - // Whether there is still enough memory for us to continue unrolling this block - var keepUnrolling = true - // Initial per-task memory to request for unrolling blocks (bytes). Exposed for testing. - val initialMemoryThreshold = unrollMemoryThreshold - // How often to check whether we need to request more memory - val memoryCheckPeriod = 16 - // Memory currently reserved by this task for this particular unrolling operation - var memoryThreshold = initialMemoryThreshold - // Memory to request as a multiple of current vector size - val memoryGrowthFactor = 1.5 - // Previous unroll memory held by this task, for releasing later (only at the very end) - val previousMemoryReserved = currentUnrollMemoryForThisTask - // Underlying vector for unrolling the block - var vector = new SizeTrackingVector[Any] - - // Request enough memory to begin unrolling - keepUnrolling = reserveUnrollMemoryForThisTask(blockId, initialMemoryThreshold, droppedBlocks) - - if (!keepUnrolling) { - logWarning(s"Failed to reserve initial memory threshold of " + - s"${Utils.bytesToString(initialMemoryThreshold)} for computing block $blockId in memory.") - } - - // Unroll this block safely, checking whether we have exceeded our threshold periodically - try { - while (values.hasNext && keepUnrolling) { - vector += values.next() - if (elementsUnrolled % memoryCheckPeriod == 0) { - // If our vector's size has exceeded the threshold, request more memory - val currentSize = vector.estimateSize() - if (currentSize >= memoryThreshold) { - val amountToRequest = (currentSize * memoryGrowthFactor - memoryThreshold).toLong - keepUnrolling = reserveUnrollMemoryForThisTask( - blockId, amountToRequest, droppedBlocks) - // New threshold is currentSize * memoryGrowthFactor - memoryThreshold += amountToRequest - } - } - elementsUnrolled += 1 - } - - if (keepUnrolling) { - // We successfully unrolled the entirety of this block - Left(vector.toArray) - } else { - // We ran out of space while unrolling the values for this block - logUnrollFailureMessage(blockId, vector.estimateSize()) - Right(vector.iterator ++ values) - } - - } finally { - // If we return an array, the values returned here will be cached in `tryToPut` later. - // In this case, we should release the memory only after we cache the block there. - if (keepUnrolling) { - val taskAttemptId = currentTaskAttemptId() - memoryManager.synchronized { - // Since we continue to hold onto the array until we actually cache it, we cannot - // release the unroll memory yet. Instead, we transfer it to pending unroll memory - // so `tryToPut` can further transfer it to normal storage memory later. - // TODO: we can probably express this without pending unroll memory (SPARK-10907) - val amountToTransferToPending = currentUnrollMemoryForThisTask - previousMemoryReserved - unrollMemoryMap(taskAttemptId) -= amountToTransferToPending - pendingUnrollMemoryMap(taskAttemptId) = - pendingUnrollMemoryMap.getOrElse(taskAttemptId, 0L) + amountToTransferToPending - } - } else { - // Otherwise, if we return an iterator, we can only release the unroll memory when - // the task finishes since we don't know when the iterator will be consumed. - } - } - } - - /** - * Return the RDD ID that a given block ID is from, or None if it is not an RDD block. - */ - private def getRddId(blockId: BlockId): Option[Int] = { - blockId.asRDDId.map(_.rddId) - } - - private def tryToPut( - blockId: BlockId, - value: Any, - size: Long, - deserialized: Boolean, - droppedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = { - tryToPut(blockId, () => value, size, deserialized, droppedBlocks) - } - - /** - * Try to put in a set of values, if we can free up enough space. The value should either be - * an Array if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) size - * must also be passed by the caller. - * - * `value` will be lazily created. If it cannot be put into MemoryStore or disk, `value` won't be - * created to avoid OOM since it may be a big ByteBuffer. - * - * Synchronize on `memoryManager` to ensure that all the put requests and its associated block - * dropping is done by only on thread at a time. Otherwise while one thread is dropping - * blocks to free memory for one block, another thread may use up the freed space for - * another block. - * - * All blocks evicted in the process, if any, will be added to `droppedBlocks`. - * - * @return whether put was successful. - */ - private def tryToPut( - blockId: BlockId, - value: () => Any, - size: Long, - deserialized: Boolean, - droppedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = { - - /* TODO: Its possible to optimize the locking by locking entries only when selecting blocks - * to be dropped. Once the to-be-dropped blocks have been selected, and lock on entries has - * been released, it must be ensured that those to-be-dropped blocks are not double counted - * for freeing up more space for another block that needs to be put. Only then the actually - * dropping of blocks (and writing to disk if necessary) can proceed in parallel. */ - - memoryManager.synchronized { - // Note: if we have previously unrolled this block successfully, then pending unroll - // memory should be non-zero. This is the amount that we already reserved during the - // unrolling process. In this case, we can just reuse this space to cache our block. - // The synchronization on `memoryManager` here guarantees that the release and acquire - // happen atomically. This relies on the assumption that all memory acquisitions are - // synchronized on the same lock. - releasePendingUnrollMemoryForThisTask() - val enoughMemory = memoryManager.acquireStorageMemory(blockId, size, droppedBlocks) - if (enoughMemory) { - // We acquired enough memory for the block, so go ahead and put it - val entry = new MemoryEntry(value(), size, deserialized) - entries.synchronized { - entries.put(blockId, entry) - } - val valuesOrBytes = if (deserialized) "values" else "bytes" - logInfo("Block %s stored as %s in memory (estimated size %s, free %s)".format( - blockId, valuesOrBytes, Utils.bytesToString(size), Utils.bytesToString(blocksMemoryUsed))) - } else { - // Tell the block manager that we couldn't put it in memory so that it can drop it to - // disk if the block allows disk storage. - lazy val data = if (deserialized) { - Left(value().asInstanceOf[Array[Any]]) - } else { - Right(value().asInstanceOf[ByteBuffer].duplicate()) - } - val droppedBlockStatus = blockManager.dropFromMemory(blockId, () => data) - droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) } - } - enoughMemory - } - } - - /** - * Try to free up a given amount of space by evicting existing blocks. - * - * @param space the amount of memory to free, in bytes - * @param droppedBlocks a holder for blocks evicted in the process - * @return whether the requested free space is freed. - */ - private[spark] def ensureFreeSpace( - space: Long, - droppedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = { - ensureFreeSpace(None, space, droppedBlocks) - } - - /** - * Try to free up a given amount of space to store a block by evicting existing ones. - * - * @param space the amount of memory to free, in bytes - * @param droppedBlocks a holder for blocks evicted in the process - * @return whether the requested free space is freed. - */ - private[spark] def ensureFreeSpace( - blockId: BlockId, - space: Long, - droppedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = { - ensureFreeSpace(Some(blockId), space, droppedBlocks) - } - - /** - * Try to free up a given amount of space to store a particular block, but can fail if - * either the block is bigger than our memory or it would require replacing another block - * from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that - * don't fit into memory that we want to avoid). - * - * @param blockId the ID of the block we are freeing space for, if any - * @param space the size of this block - * @param droppedBlocks a holder for blocks evicted in the process - * @return whether the requested free space is freed. - */ - private def ensureFreeSpace( - blockId: Option[BlockId], - space: Long, - droppedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = { - memoryManager.synchronized { - val freeMemory = maxMemory - memoryUsed - val rddToAdd = blockId.flatMap(getRddId) - val selectedBlocks = new ArrayBuffer[BlockId] - var selectedMemory = 0L - - logInfo(s"Ensuring $space bytes of free space " + - blockId.map { id => s"for block $id" }.getOrElse("") + - s"(free: $freeMemory, max: $maxMemory)") - - // Fail fast if the block simply won't fit - if (space > maxMemory) { - logInfo("Will not " + blockId.map { id => s"store $id" }.getOrElse("free memory") + - s" as the required space ($space bytes) exceeds our memory limit ($maxMemory bytes)") - return false - } - - // No need to evict anything if there is already enough free space - if (freeMemory >= space) { - return true - } - - // This is synchronized to ensure that the set of entries is not changed - // (because of getValue or getBytes) while traversing the iterator, as that - // can lead to exceptions. - entries.synchronized { - val iterator = entries.entrySet().iterator() - while (freeMemory + selectedMemory < space && iterator.hasNext) { - val pair = iterator.next() - val blockId = pair.getKey - if (rddToAdd.isEmpty || rddToAdd != getRddId(blockId)) { - selectedBlocks += blockId - selectedMemory += pair.getValue.size - } - } - } - - if (freeMemory + selectedMemory >= space) { - logInfo(s"${selectedBlocks.size} blocks selected for dropping") - for (blockId <- selectedBlocks) { - val entry = entries.synchronized { entries.get(blockId) } - // This should never be null as only one task should be dropping - // blocks and removing entries. However the check is still here for - // future safety. - if (entry != null) { - val data = if (entry.deserialized) { - Left(entry.value.asInstanceOf[Array[Any]]) - } else { - Right(entry.value.asInstanceOf[ByteBuffer].duplicate()) - } - val droppedBlockStatus = blockManager.dropFromMemory(blockId, data) - droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) } - } - } - true - } else { - blockId.foreach { id => - logInfo(s"Will not store $id as it would require dropping another block " + - "from the same RDD") - } - false - } - } - } - - override def contains(blockId: BlockId): Boolean = { - entries.synchronized { entries.containsKey(blockId) } - } - - private def currentTaskAttemptId(): Long = { - // In case this is called on the driver, return an invalid task attempt id. - Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1L) - } - - /** - * Reserve memory for unrolling the given block for this task. - * @return whether the request is granted. - */ - def reserveUnrollMemoryForThisTask( - blockId: BlockId, - memory: Long, - droppedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = { - memoryManager.synchronized { - val success = memoryManager.acquireUnrollMemory(blockId, memory, droppedBlocks) - if (success) { - val taskAttemptId = currentTaskAttemptId() - unrollMemoryMap(taskAttemptId) = unrollMemoryMap.getOrElse(taskAttemptId, 0L) + memory - } - success - } - } - - /** - * Release memory used by this task for unrolling blocks. - * If the amount is not specified, remove the current task's allocation altogether. - */ - def releaseUnrollMemoryForThisTask(memory: Long = Long.MaxValue): Unit = { - val taskAttemptId = currentTaskAttemptId() - memoryManager.synchronized { - if (unrollMemoryMap.contains(taskAttemptId)) { - val memoryToRelease = math.min(memory, unrollMemoryMap(taskAttemptId)) - if (memoryToRelease > 0) { - unrollMemoryMap(taskAttemptId) -= memoryToRelease - if (unrollMemoryMap(taskAttemptId) == 0) { - unrollMemoryMap.remove(taskAttemptId) - } - memoryManager.releaseUnrollMemory(memoryToRelease) - } - } - } - } - - /** - * Release pending unroll memory of current unroll successful block used by this task - */ - def releasePendingUnrollMemoryForThisTask(memory: Long = Long.MaxValue): Unit = { - val taskAttemptId = currentTaskAttemptId() - memoryManager.synchronized { - if (pendingUnrollMemoryMap.contains(taskAttemptId)) { - val memoryToRelease = math.min(memory, pendingUnrollMemoryMap(taskAttemptId)) - if (memoryToRelease > 0) { - pendingUnrollMemoryMap(taskAttemptId) -= memoryToRelease - if (pendingUnrollMemoryMap(taskAttemptId) == 0) { - pendingUnrollMemoryMap.remove(taskAttemptId) - } - memoryManager.releaseUnrollMemory(memoryToRelease) - } - } - } - } - - /** - * Return the amount of memory currently occupied for unrolling blocks across all tasks. - */ - def currentUnrollMemory: Long = memoryManager.synchronized { - unrollMemoryMap.values.sum + pendingUnrollMemoryMap.values.sum - } - - /** - * Return the amount of memory currently occupied for unrolling blocks by this task. - */ - def currentUnrollMemoryForThisTask: Long = memoryManager.synchronized { - unrollMemoryMap.getOrElse(currentTaskAttemptId(), 0L) - } - - /** - * Return the number of tasks currently unrolling blocks. - */ - private def numTasksUnrolling: Int = memoryManager.synchronized { unrollMemoryMap.keys.size } - - /** - * Log information about current memory usage. - */ - private def logMemoryUsage(): Unit = { - logInfo( - s"Memory use = ${Utils.bytesToString(blocksMemoryUsed)} (blocks) + " + - s"${Utils.bytesToString(currentUnrollMemory)} (scratch space shared across " + - s"$numTasksUnrolling tasks(s)) = ${Utils.bytesToString(memoryUsed)}. " + - s"Storage limit = ${Utils.bytesToString(maxMemory)}." - ) - } - - /** - * Log a warning for failing to unroll a block. - * - * @param blockId ID of the block we are trying to unroll. - * @param finalVectorSize Final size of the vector before unrolling failed. - */ - private def logUnrollFailureMessage(blockId: BlockId, finalVectorSize: Long): Unit = { - logWarning( - s"Not enough space to cache $blockId in memory! " + - s"(computed ${Utils.bytesToString(finalVectorSize)} so far)" - ) - logMemoryUsage() - } -} diff --git a/core/src/main/scala/org/apache/spark/storage/PutResult.scala b/core/src/main/scala/org/apache/spark/storage/PutResult.scala deleted file mode 100644 index f0eac7594ecf..000000000000 --- a/core/src/main/scala/org/apache/spark/storage/PutResult.scala +++ /dev/null @@ -1,32 +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.storage - -import java.nio.ByteBuffer - -/** - * Result of adding a block into a BlockStore. This case class contains a few things: - * (1) The estimated size of the put, - * (2) The values put if the caller asked for them to be returned (e.g. for chaining - * replication), and - * (3) A list of blocks dropped as a result of this put. This is always empty for DiskStore. - */ -private[spark] case class PutResult( - size: Long, - data: Either[Iterator[_], ByteBuffer], - droppedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty) diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 96062626b504..e5abbf745cc4 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -18,16 +18,17 @@ package org.apache.spark.storage import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.rdd.{RDDOperationScope, RDD} +import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.util.Utils @DeveloperApi class RDDInfo( val id: Int, - val name: String, + var name: String, val numPartitions: Int, var storageLevel: StorageLevel, val parentIds: Seq[Int], + val callSite: String = "", val scope: Option[RDDOperationScope] = None) extends Ordered[RDDInfo] { @@ -36,15 +37,14 @@ class RDDInfo( var diskSize = 0L var externalBlockStoreSize = 0L - def isCached: Boolean = - (memSize + diskSize + externalBlockStoreSize > 0) && numCachedPartitions > 0 + def isCached: Boolean = (memSize + diskSize > 0) && numCachedPartitions > 0 override def toString: String = { import Utils.bytesToString ("RDD \"%s\" (%d) StorageLevel: %s; CachedPartitions: %d; TotalPartitions: %d; " + - "MemorySize: %s; ExternalBlockStoreSize: %s; DiskSize: %s").format( + "MemorySize: %s; DiskSize: %s").format( name, id, storageLevel.toString, numCachedPartitions, numPartitions, - bytesToString(memSize), bytesToString(externalBlockStoreSize), bytesToString(diskSize)) + bytesToString(memSize), bytesToString(diskSize)) } override def compare(that: RDDInfo): Int = { @@ -56,6 +56,7 @@ private[spark] object RDDInfo { def fromRdd(rdd: RDD[_]): RDDInfo = { val rddName = Option(rdd.name).getOrElse(Utils.getFormattedClassName(rdd)) val parentIds = rdd.dependencies.map(_.rdd.id) - new RDDInfo(rdd.id, rddName, rdd.partitions.length, rdd.getStorageLevel, parentIds, rdd.scope) + new RDDInfo(rdd.id, rddName, rdd.partitions.length, + rdd.getStorageLevel, parentIds, rdd.creationSite.shortForm, rdd.scope) } } 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 0d0448feb5b0..2d176b62f8b3 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -17,17 +17,21 @@ package org.apache.spark.storage -import java.io.InputStream +import java.io.{File, InputStream, IOException} +import java.nio.ByteBuffer import java.util.concurrent.LinkedBlockingQueue +import javax.annotation.concurrent.GuardedBy -import scala.collection.mutable.{ArrayBuffer, HashSet, Queue} -import scala.util.control.NonFatal +import scala.collection.mutable +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} -import org.apache.spark.{Logging, SparkException, TaskContext} -import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient} +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient, TempShuffleFileManager} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.util.Utils +import org.apache.spark.util.io.ChunkedByteBufferOutputStream /** * An iterator that fetches multiple blocks. For local blocks, it fetches from the local block @@ -36,7 +40,7 @@ import org.apache.spark.util.Utils * This creates an iterator of (BlockID, InputStream) tuples so the caller can handle blocks * in a pipelined fashion as they are received. * - * The implementation throttles the remote fetches to they don't exceed maxBytesInFlight to avoid + * The implementation throttles the remote fetches so they don't exceed maxBytesInFlight to avoid * using too much memory. * * @param context [[TaskContext]], used for metrics update @@ -45,7 +49,13 @@ import org.apache.spark.util.Utils * @param blocksByAddress list of blocks to fetch grouped by the [[BlockManagerId]]. * For each block we also require the size (in bytes as a long field) in * order to throttle the memory usage. + * @param streamWrapper A function to wrap the returned input stream. * @param maxBytesInFlight max size (in bytes) of remote blocks to fetch at any given point. + * @param maxReqsInFlight max number of remote requests to fetch blocks at any given point. + * @param maxBlocksInFlightPerAddress max number of shuffle blocks being fetched at any given point + * for a given remote host:port. + * @param maxReqSizeShuffleToMem max size (in bytes) of a request that can be shuffled to memory. + * @param detectCorrupt whether to detect any corruption in fetched blocks. */ private[spark] final class ShuffleBlockFetcherIterator( @@ -53,8 +63,13 @@ final class ShuffleBlockFetcherIterator( shuffleClient: ShuffleClient, blockManager: BlockManager, blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], - maxBytesInFlight: Long) - extends Iterator[(BlockId, InputStream)] with Logging { + streamWrapper: (BlockId, InputStream) => InputStream, + maxBytesInFlight: Long, + maxReqsInFlight: Int, + maxBlocksInFlightPerAddress: Int, + maxReqSizeShuffleToMem: Long, + detectCorrupt: Boolean) + extends Iterator[(BlockId, InputStream)] with TempShuffleFileManager with Logging { import ShuffleBlockFetcherIterator._ @@ -67,7 +82,7 @@ final class ShuffleBlockFetcherIterator( private[this] var numBlocksToFetch = 0 /** - * The number of blocks proccessed by the caller. The iterator is exhausted when + * The number of blocks processed by the caller. The iterator is exhausted when * [[numBlocksProcessed]] == [[numBlocksToFetch]]. */ private[this] var numBlocksProcessed = 0 @@ -90,7 +105,7 @@ final class ShuffleBlockFetcherIterator( * Current [[FetchResult]] being processed. We track this so we can release the current buffer * in case of a runtime exception when processing the current buffer. */ - @volatile private[this] var currentResult: FetchResult = null + @volatile private[this] var currentResult: SuccessFetchResult = null /** * Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that @@ -98,16 +113,42 @@ final class ShuffleBlockFetcherIterator( */ private[this] val fetchRequests = new Queue[FetchRequest] + /** + * Queue of fetch requests which could not be issued the first time they were dequeued. These + * requests are tried again when the fetch constraints are satisfied. + */ + private[this] val deferredFetchRequests = new HashMap[BlockManagerId, Queue[FetchRequest]]() + /** Current bytes in flight from our requests */ private[this] var bytesInFlight = 0L - private[this] val shuffleMetrics = context.taskMetrics().createShuffleReadMetricsForDependency() + /** Current number of requests in flight */ + private[this] var reqsInFlight = 0 + + /** Current number of blocks in flight per host:port */ + private[this] val numBlocksInFlightPerAddress = new HashMap[BlockManagerId, Int]() + + /** + * The blocks that can't be decompressed successfully, it is used to guarantee that we retry + * at most once for those corrupted blocks. + */ + private[this] val corruptedBlocks = mutable.HashSet[BlockId]() + + private[this] val shuffleMetrics = context.taskMetrics().createTempShuffleReadMetrics() /** * Whether the iterator is still active. If isZombie is true, the callback interface will no * longer place fetched blocks into [[results]]. */ - @volatile private[this] var isZombie = false + @GuardedBy("this") + private[this] var isZombie = false + + /** + * A set to store the files used for shuffling remote huge blocks. Files in this set will be + * deleted when cleanup. This is a layer of defensiveness against disk file leaks. + */ + @GuardedBy("this") + private[this] val shuffleFilesSet = mutable.HashSet[File]() initialize() @@ -115,62 +156,103 @@ final class ShuffleBlockFetcherIterator( // The currentResult is set to null to prevent releasing the buffer again on cleanup() private[storage] def releaseCurrentResultBuffer(): Unit = { // Release the current buffer if necessary - currentResult match { - case SuccessFetchResult(_, _, _, buf) => buf.release() - case _ => + if (currentResult != null) { + currentResult.buf.release() } currentResult = null } + override def createTempShuffleFile(): File = { + blockManager.diskBlockManager.createTempLocalBlock()._2 + } + + override def registerTempShuffleFileToClean(file: File): Boolean = synchronized { + if (isZombie) { + false + } else { + shuffleFilesSet += file + true + } + } + /** * Mark the iterator as zombie, and release all buffers that haven't been deserialized yet. */ private[this] def cleanup() { - isZombie = true + synchronized { + isZombie = true + } releaseCurrentResultBuffer() // Release buffers in the results queue val iter = results.iterator() while (iter.hasNext) { val result = iter.next() result match { - case SuccessFetchResult(_, _, _, buf) => buf.release() + case SuccessFetchResult(_, address, _, buf, _) => + if (address != blockManager.blockManagerId) { + shuffleMetrics.incRemoteBytesRead(buf.size) + if (buf.isInstanceOf[FileSegmentManagedBuffer]) { + shuffleMetrics.incRemoteBytesReadToDisk(buf.size) + } + shuffleMetrics.incRemoteBlocksFetched(1) + } + buf.release() case _ => } } + shuffleFilesSet.foreach { file => + if (!file.delete()) { + logWarning("Failed to cleanup shuffle fetch temp file " + file.getAbsolutePath()) + } + } } private[this] def sendRequest(req: FetchRequest) { logDebug("Sending request for %d blocks (%s) from %s".format( req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) bytesInFlight += req.size + reqsInFlight += 1 // so we can look up the size of each blockID val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap + val remainingBlocks = new HashSet[String]() ++= sizeMap.keys val blockIds = req.blocks.map(_._1.toString) - val address = req.address - shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray, - new BlockFetchingListener { - override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = { - // Only add the buffer to results queue if the iterator is not zombie, - // i.e. cleanup() has not been called yet. + + val blockFetchingListener = new BlockFetchingListener { + override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = { + // Only add the buffer to results queue if the iterator is not zombie, + // i.e. cleanup() has not been called yet. + ShuffleBlockFetcherIterator.this.synchronized { if (!isZombie) { // Increment the ref count because we need to pass this to a different thread. // This needs to be released after use. buf.retain() - results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf)) - shuffleMetrics.incRemoteBytesRead(buf.size) - shuffleMetrics.incRemoteBlocksFetched(1) + remainingBlocks -= blockId + results.put(new SuccessFetchResult(BlockId(blockId), address, sizeMap(blockId), buf, + remainingBlocks.isEmpty)) + logDebug("remainingBlocks: " + remainingBlocks) } - logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } + logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) + } - override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = { - logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e) - results.put(new FailureFetchResult(BlockId(blockId), address, e)) - } + override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = { + logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e) + results.put(new FailureFetchResult(BlockId(blockId), address, e)) } - ) + } + + // Fetch remote shuffle blocks to disk when the request is too large. Since the shuffle data is + // already encrypted and compressed over the wire(w.r.t. the related configs), we can just fetch + // the data and write it to file directly. + if (req.size > maxReqSizeShuffleToMem) { + shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray, + blockFetchingListener, this) + } else { + shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray, + blockFetchingListener, null) + } } private[this] def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { @@ -178,7 +260,8 @@ final class ShuffleBlockFetcherIterator( // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 // nodes, rather than blocking on reading output from one node. val targetRequestSize = math.max(maxBytesInFlight / 5, 1L) - logDebug("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) + logDebug("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize + + ", maxBlocksInFlightPerAddress: " + maxBlocksInFlightPerAddress) // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. @@ -207,11 +290,13 @@ final class ShuffleBlockFetcherIterator( } else if (size < 0) { throw new BlockException(blockId, "Negative block size " + size) } - if (curRequestSize >= targetRequestSize) { + if (curRequestSize >= targetRequestSize || + curBlocks.size >= maxBlocksInFlightPerAddress) { // Add this FetchRequest remoteRequests += new FetchRequest(address, curBlocks) + logDebug(s"Creating fetch request of $curRequestSize at $address " + + s"with ${curBlocks.size} blocks") curBlocks = new ArrayBuffer[(BlockId, Long)] - logDebug(s"Creating fetch request of $curRequestSize at $address") curRequestSize = 0 } } @@ -227,7 +312,7 @@ final class ShuffleBlockFetcherIterator( /** * Fetch the local blocks while we are fetching remote blocks. This is ok because - * [[ManagedBuffer]]'s memory is allocated lazily when we create the input stream, so all we + * `ManagedBuffer`'s memory is allocated lazily when we create the input stream, so all we * track in-memory are the ManagedBuffer references themselves. */ private[this] def fetchLocalBlocks() { @@ -239,7 +324,7 @@ final class ShuffleBlockFetcherIterator( shuffleMetrics.incLocalBlocksFetched(1) shuffleMetrics.incLocalBytesRead(buf.size) buf.retain() - results.put(new SuccessFetchResult(blockId, blockManager.blockManagerId, 0, buf)) + results.put(new SuccessFetchResult(blockId, blockManager.blockManagerId, 0, buf, false)) } catch { case e: Exception => // If we see an exception, stop immediately. @@ -258,6 +343,9 @@ final class ShuffleBlockFetcherIterator( val remoteRequests = splitLocalRemoteBlocks() // Add the remote requests into our queue in a random order fetchRequests ++= Utils.randomize(remoteRequests) + assert ((0 == reqsInFlight) == (0 == bytesInFlight), + "expected reqsInFlight = 0 but found reqsInFlight = " + reqsInFlight + + ", expected bytesInFlight = 0 but found bytesInFlight = " + bytesInFlight) // Send out initial requests for blocks, up to our maxBytesInFlight fetchUpToMaxBytes() @@ -281,39 +369,147 @@ final class ShuffleBlockFetcherIterator( * Throws a FetchFailedException if the next block could not be fetched. */ override def next(): (BlockId, InputStream) = { - numBlocksProcessed += 1 - val startFetchWait = System.currentTimeMillis() - currentResult = results.take() - val result = currentResult - val stopFetchWait = System.currentTimeMillis() - shuffleMetrics.incFetchWaitTime(stopFetchWait - startFetchWait) - - result match { - case SuccessFetchResult(_, _, size, _) => bytesInFlight -= size - case _ => + if (!hasNext) { + throw new NoSuchElementException } - // Send fetch requests up to maxBytesInFlight - fetchUpToMaxBytes() - result match { - case FailureFetchResult(blockId, address, e) => - throwFetchFailedException(blockId, address, e) + numBlocksProcessed += 1 + + var result: FetchResult = null + var input: InputStream = null + // Take the next fetched result and try to decompress it to detect data corruption, + // then fetch it one more time if it's corrupt, throw FailureFetchResult if the second fetch + // is also corrupt, so the previous stage could be retried. + // For local shuffle block, throw FailureFetchResult for the first IOException. + while (result == null) { + val startFetchWait = System.currentTimeMillis() + result = results.take() + val stopFetchWait = System.currentTimeMillis() + shuffleMetrics.incFetchWaitTime(stopFetchWait - startFetchWait) - case SuccessFetchResult(blockId, address, _, buf) => - try { - (result.blockId, new BufferReleasingInputStream(buf.createInputStream(), this)) - } catch { - case NonFatal(t) => - throwFetchFailedException(blockId, address, t) - } + result match { + case r @ SuccessFetchResult(blockId, address, size, buf, isNetworkReqDone) => + if (address != blockManager.blockManagerId) { + numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 + shuffleMetrics.incRemoteBytesRead(buf.size) + if (buf.isInstanceOf[FileSegmentManagedBuffer]) { + shuffleMetrics.incRemoteBytesReadToDisk(buf.size) + } + shuffleMetrics.incRemoteBlocksFetched(1) + } + bytesInFlight -= size + if (isNetworkReqDone) { + reqsInFlight -= 1 + logDebug("Number of requests in flight " + reqsInFlight) + } + + val in = try { + buf.createInputStream() + } catch { + // The exception could only be throwed by local shuffle block + case e: IOException => + assert(buf.isInstanceOf[FileSegmentManagedBuffer]) + logError("Failed to create input stream from local block", e) + buf.release() + throwFetchFailedException(blockId, address, e) + } + + input = streamWrapper(blockId, in) + // Only copy the stream if it's wrapped by compression or encryption, also the size of + // block is small (the decompressed block is smaller than maxBytesInFlight) + if (detectCorrupt && !input.eq(in) && size < maxBytesInFlight / 3) { + val originalInput = input + val out = new ChunkedByteBufferOutputStream(64 * 1024, ByteBuffer.allocate) + try { + // Decompress the whole block at once to detect any corruption, which could increase + // the memory usage tne potential increase the chance of OOM. + // TODO: manage the memory used here, and spill it into disk in case of OOM. + Utils.copyStream(input, out) + out.close() + input = out.toChunkedByteBuffer.toInputStream(dispose = true) + } catch { + case e: IOException => + buf.release() + if (buf.isInstanceOf[FileSegmentManagedBuffer] + || corruptedBlocks.contains(blockId)) { + throwFetchFailedException(blockId, address, e) + } else { + logWarning(s"got an corrupted block $blockId from $address, fetch again", e) + corruptedBlocks += blockId + fetchRequests += FetchRequest(address, Array((blockId, size))) + result = null + } + } finally { + // TODO: release the buf here to free memory earlier + originalInput.close() + in.close() + } + } + + case FailureFetchResult(blockId, address, e) => + throwFetchFailedException(blockId, address, e) + } + + // Send fetch requests up to maxBytesInFlight + fetchUpToMaxBytes() } + + currentResult = result.asInstanceOf[SuccessFetchResult] + (currentResult.blockId, new BufferReleasingInputStream(input, this)) } private def fetchUpToMaxBytes(): Unit = { - // Send fetch requests up to maxBytesInFlight - while (fetchRequests.nonEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) + // Send fetch requests up to maxBytesInFlight. If you cannot fetch from a remote host + // immediately, defer the request until the next time it can be processed. + + // Process any outstanding deferred fetch requests if possible. + if (deferredFetchRequests.nonEmpty) { + for ((remoteAddress, defReqQueue) <- deferredFetchRequests) { + while (isRemoteBlockFetchable(defReqQueue) && + !isRemoteAddressMaxedOut(remoteAddress, defReqQueue.front)) { + val request = defReqQueue.dequeue() + logDebug(s"Processing deferred fetch request for $remoteAddress with " + + s"${request.blocks.length} blocks") + send(remoteAddress, request) + if (defReqQueue.isEmpty) { + deferredFetchRequests -= remoteAddress + } + } + } + } + + // Process any regular fetch requests if possible. + while (isRemoteBlockFetchable(fetchRequests)) { + val request = fetchRequests.dequeue() + val remoteAddress = request.address + if (isRemoteAddressMaxedOut(remoteAddress, request)) { + logDebug(s"Deferring fetch request for $remoteAddress with ${request.blocks.size} blocks") + val defReqQueue = deferredFetchRequests.getOrElse(remoteAddress, new Queue[FetchRequest]()) + defReqQueue.enqueue(request) + deferredFetchRequests(remoteAddress) = defReqQueue + } else { + send(remoteAddress, request) + } + } + + def send(remoteAddress: BlockManagerId, request: FetchRequest): Unit = { + sendRequest(request) + numBlocksInFlightPerAddress(remoteAddress) = + numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + request.blocks.size + } + + def isRemoteBlockFetchable(fetchReqQueue: Queue[FetchRequest]): Boolean = { + fetchReqQueue.nonEmpty && + (bytesInFlight == 0 || + (reqsInFlight + 1 <= maxReqsInFlight && + bytesInFlight + fetchReqQueue.front.size <= maxBytesInFlight)) + } + + // Checks if sending a new fetch request will exceed the max no. of blocks being fetched from a + // given remote address. + def isRemoteAddressMaxedOut(remoteAddress: BlockManagerId, request: FetchRequest): Boolean = { + numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + request.blocks.size > + maxBlocksInFlightPerAddress } } @@ -329,7 +525,7 @@ final class ShuffleBlockFetcherIterator( } /** - * Helper class that ensures a ManagedBuffer is release upon InputStream.close() + * Helper class that ensures a ManagedBuffer is released upon InputStream.close() */ private class BufferReleasingInputStream( private val delegate: InputStream, @@ -389,14 +585,15 @@ object ShuffleBlockFetcherIterator { * @param address BlockManager that the block was fetched from. * @param size estimated size of the block, used to calculate bytesInFlight. * Note that this is NOT the exact bytes. - * @param buf [[ManagedBuffer]] for the content. + * @param buf `ManagedBuffer` for the content. + * @param isNetworkReqDone Is this the last network request for this host in this fetch request. */ private[storage] case class SuccessFetchResult( blockId: BlockId, address: BlockManagerId, size: Long, - buf: ManagedBuffer) - extends FetchResult { + buf: ManagedBuffer, + isNetworkReqDone: Boolean) extends FetchResult { require(buf != null) require(size >= 0) } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 703bce3e6b85..4c6998d7a8e2 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -21,6 +21,7 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.memory.MemoryMode import org.apache.spark.util.Utils /** @@ -30,7 +31,7 @@ import org.apache.spark.util.Utils * ExternalBlockStore, whether to keep the data in memory in a serialized format, and whether * to replicate the RDD partitions on multiple nodes. * - * The [[org.apache.spark.storage.StorageLevel$]] singleton object contains some static constants + * The [[org.apache.spark.storage.StorageLevel]] singleton object contains some static constants * for commonly useful storage levels. To create your own storage level object, use the * factory method of the singleton object (`StorageLevel(...)`). */ @@ -59,10 +60,12 @@ class StorageLevel private( assert(replication < 40, "Replication restricted to be less than 40 for calculating hash codes") if (useOffHeap) { - require(!useDisk, "Off-heap storage level does not support using disk") - require(!useMemory, "Off-heap storage level does not support using heap memory") require(!deserialized, "Off-heap storage level does not support deserialized storage") - require(replication == 1, "Off-heap storage level does not support multiple replication") + } + + private[spark] def memoryMode: MemoryMode = { + if (useOffHeap) MemoryMode.OFF_HEAP + else MemoryMode.ON_HEAP } override def clone(): StorageLevel = { @@ -80,7 +83,7 @@ class StorageLevel private( false } - def isValid: Boolean = (useMemory || useDisk || useOffHeap) && (replication > 0) + def isValid: Boolean = (useMemory || useDisk) && (replication > 0) def toInt: Int = { var ret = 0 @@ -117,7 +120,14 @@ class StorageLevel private( private def readResolve(): Object = StorageLevel.getCachedStorageLevel(this) override def toString: String = { - s"StorageLevel($useDisk, $useMemory, $useOffHeap, $deserialized, $replication)" + val disk = if (useDisk) "disk" else "" + val memory = if (useMemory) "memory" else "" + val heap = if (useOffHeap) "offheap" else "" + val deserialize = if (deserialized) "deserialized" else "" + + val output = + Seq(disk, memory, heap, deserialize, s"$replication replicas").filter(_.nonEmpty) + s"StorageLevel(${output.mkString(", ")})" } override def hashCode(): Int = toInt * 41 + replication @@ -125,8 +135,9 @@ class StorageLevel private( def description: String = { var result = "" result += (if (useDisk) "Disk " else "") - result += (if (useMemory) "Memory " else "") - result += (if (useOffHeap) "ExternalBlockStore " else "") + if (useMemory) { + result += (if (useOffHeap) "Memory (off heap) " else "Memory ") + } result += (if (deserialized) "Deserialized " else "Serialized ") result += s"${replication}x Replicated" result @@ -150,7 +161,7 @@ object StorageLevel { val MEMORY_AND_DISK_2 = new StorageLevel(true, true, false, true, 2) val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, false) val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) - val OFF_HEAP = new StorageLevel(false, false, true, false) + val OFF_HEAP = new StorageLevel(true, true, true, false, 1) /** * :: DeveloperApi :: @@ -175,7 +186,7 @@ object StorageLevel { /** * :: DeveloperApi :: - * Create a new StorageLevel object without setting useOffHeap. + * Create a new StorageLevel object. */ @DeveloperApi def apply( @@ -190,7 +201,7 @@ object StorageLevel { /** * :: DeveloperApi :: - * Create a new StorageLevel object. + * Create a new StorageLevel object without setting useOffHeap. */ @DeveloperApi def apply( diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index ec711480ebf3..ac60f795915a 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -19,6 +19,7 @@ package org.apache.spark.storage import scala.collection.mutable +import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ @@ -29,14 +30,21 @@ import org.apache.spark.scheduler._ * This class is thread-safe (unlike JobProgressListener) */ @DeveloperApi -class StorageStatusListener extends SparkListener { +@deprecated("This class will be removed in a future release.", "2.2.0") +class StorageStatusListener(conf: SparkConf) extends SparkListener { // This maintains only blocks that are cached (i.e. storage level is not StorageLevel.NONE) private[storage] val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() + private[storage] val deadExecutorStorageStatus = new mutable.ListBuffer[StorageStatus]() + private[this] val retainedDeadExecutors = conf.getInt("spark.ui.retainedDeadExecutors", 100) def storageStatusList: Seq[StorageStatus] = synchronized { executorIdToStorageStatus.values.toSeq } + def deadStorageStatusList: Seq[StorageStatus] = synchronized { + deadExecutorStorageStatus + } + /** Update storage status list to reflect updated block statuses */ private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { executorIdToStorageStatus.get(execId).foreach { storageStatus => @@ -59,17 +67,6 @@ class StorageStatusListener extends SparkListener { } } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { - val info = taskEnd.taskInfo - val metrics = taskEnd.taskMetrics - if (info != null && metrics != null) { - val updatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) - if (updatedBlocks.length > 0) { - updateStorageStatus(info.executorId, updatedBlocks) - } - } - } - override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = synchronized { updateStorageStatus(unpersistRDD.rddId) } @@ -78,17 +75,37 @@ class StorageStatusListener extends SparkListener { synchronized { val blockManagerId = blockManagerAdded.blockManagerId val executorId = blockManagerId.executorId - val maxMem = blockManagerAdded.maxMem - val storageStatus = new StorageStatus(blockManagerId, maxMem) + // The onHeap and offHeap memory are always defined for new applications, + // but they can be missing if we are replaying old event logs. + val storageStatus = new StorageStatus(blockManagerId, blockManagerAdded.maxMem, + blockManagerAdded.maxOnHeapMem, blockManagerAdded.maxOffHeapMem) executorIdToStorageStatus(executorId) = storageStatus + + // Try to remove the dead storage status if same executor register the block manager twice. + deadExecutorStorageStatus.zipWithIndex.find(_._1.blockManagerId.executorId == executorId) + .foreach(toRemoveExecutor => deadExecutorStorageStatus.remove(toRemoveExecutor._2)) } } override def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved) { synchronized { val executorId = blockManagerRemoved.blockManagerId.executorId - executorIdToStorageStatus.remove(executorId) + executorIdToStorageStatus.remove(executorId).foreach { status => + deadExecutorStorageStatus += status + } + if (deadExecutorStorageStatus.size > retainedDeadExecutors) { + deadExecutorStorageStatus.trimStart(1) + } } } + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { + val executorId = blockUpdated.blockUpdatedInfo.blockManagerId.executorId + val blockId = blockUpdated.blockUpdatedInfo.blockId + val storageLevel = blockUpdated.blockUpdatedInfo.storageLevel + val memSize = blockUpdated.blockUpdatedInfo.memSize + val diskSize = blockUpdated.blockUpdatedInfo.diskSize + val blockStatus = BlockStatus(storageLevel, memSize, diskSize) + updateStorageStatus(executorId, Seq((blockId, blockStatus))) + } } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index c4ac30092f80..e9694fdbca2d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -17,10 +17,15 @@ package org.apache.spark.storage +import java.nio.{ByteBuffer, MappedByteBuffer} + import scala.collection.Map import scala.collection.mutable +import sun.nio.ch.DirectBuffer + import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging /** * :: DeveloperApi :: @@ -30,7 +35,12 @@ import org.apache.spark.annotation.DeveloperApi * class cannot mutate the source of the information. Accesses are not thread-safe. */ @DeveloperApi -class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { +@deprecated("This class may be removed or made private in a future release.", "2.2.0") +class StorageStatus( + val blockManagerId: BlockManagerId, + val maxMemory: Long, + val maxOnHeapMem: Option[Long], + val maxOffHeapMem: Option[Long]) { /** * Internal representation of the blocks stored in this block manager. @@ -41,32 +51,28 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { private val _rddBlocks = new mutable.HashMap[Int, mutable.Map[BlockId, BlockStatus]] private val _nonRddBlocks = new mutable.HashMap[BlockId, BlockStatus] - /** - * Storage information of the blocks that entails memory, disk, and off-heap memory usage. - * - * As with the block maps, we store the storage information separately for RDD blocks and - * non-RDD blocks for the same reason. In particular, RDD storage information is stored - * in a map indexed by the RDD ID to the following 4-tuple: - * - * (memory size, disk size, off-heap size, storage level) - * - * We assume that all the blocks that belong to the same RDD have the same storage level. - * This field is not relevant to non-RDD blocks, however, so the storage information for - * non-RDD blocks contains only the first 3 fields (in the same order). - */ - private val _rddStorageInfo = new mutable.HashMap[Int, (Long, Long, Long, StorageLevel)] - private var _nonRddStorageInfo: (Long, Long, Long) = (0L, 0L, 0L) + private case class RddStorageInfo(memoryUsage: Long, diskUsage: Long, level: StorageLevel) + private val _rddStorageInfo = new mutable.HashMap[Int, RddStorageInfo] + + private case class NonRddStorageInfo(var onHeapUsage: Long, var offHeapUsage: Long, + var diskUsage: Long) + private val _nonRddStorageInfo = NonRddStorageInfo(0L, 0L, 0L) /** Create a storage status with an initial set of blocks, leaving the source unmodified. */ - def this(bmid: BlockManagerId, maxMem: Long, initialBlocks: Map[BlockId, BlockStatus]) { - this(bmid, maxMem) + def this( + bmid: BlockManagerId, + maxMemory: Long, + maxOnHeapMem: Option[Long], + maxOffHeapMem: Option[Long], + initialBlocks: Map[BlockId, BlockStatus]) { + this(bmid, maxMemory, maxOnHeapMem, maxOffHeapMem) initialBlocks.foreach { case (bid, bstatus) => addBlock(bid, bstatus) } } /** * Return the blocks stored in this block manager. * - * Note that this is somewhat expensive, as it involves cloning the underlying maps and then + * @note This is somewhat expensive, as it involves cloning the underlying maps and then * concatenating them together. Much faster alternatives exist for common operations such as * contains, get, and size. */ @@ -75,16 +81,14 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { /** * Return the RDD blocks stored in this block manager. * - * Note that this is somewhat expensive, as it involves cloning the underlying maps and then + * @note This is somewhat expensive, as it involves cloning the underlying maps and then * concatenating them together. Much faster alternatives exist for common operations such as * getting the memory, disk, and off-heap memory sizes occupied by this RDD. */ def rddBlocks: Map[BlockId, BlockStatus] = _rddBlocks.flatMap { case (_, blocks) => blocks } /** Return the blocks that belong to the given RDD stored in this block manager. */ - def rddBlocksById(rddId: Int): Map[BlockId, BlockStatus] = { - _rddBlocks.get(rddId).getOrElse(Map.empty) - } + def rddBlocksById(rddId: Int): Map[BlockId, BlockStatus] = _rddBlocks.getOrElse(rddId, Map.empty) /** Add the given block to this storage status. If it already exists, overwrite it. */ private[spark] def addBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { @@ -125,7 +129,8 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { /** * Return whether the given block is stored in this block manager in O(1) time. - * Note that this is much faster than `this.blocks.contains`, which is O(blocks) time. + * + * @note This is much faster than `this.blocks.contains`, which is O(blocks) time. */ def containsBlock(blockId: BlockId): Boolean = { blockId match { @@ -138,12 +143,13 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { /** * Return the given block stored in this block manager in O(1) time. - * Note that this is much faster than `this.blocks.get`, which is O(blocks) time. + * + * @note This is much faster than `this.blocks.get`, which is O(blocks) time. */ def getBlock(blockId: BlockId): Option[BlockStatus] = { blockId match { case RDDBlockId(rddId, _) => - _rddBlocks.get(rddId).map(_.get(blockId)).flatten + _rddBlocks.get(rddId).flatMap(_.get(blockId)) case _ => _nonRddBlocks.get(blockId) } @@ -151,46 +157,77 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { /** * Return the number of blocks stored in this block manager in O(RDDs) time. - * Note that this is much faster than `this.blocks.size`, which is O(blocks) time. + * + * @note This is much faster than `this.blocks.size`, which is O(blocks) time. */ def numBlocks: Int = _nonRddBlocks.size + numRddBlocks /** * Return the number of RDD blocks stored in this block manager in O(RDDs) time. - * Note that this is much faster than `this.rddBlocks.size`, which is O(RDD blocks) time. + * + * @note This is much faster than `this.rddBlocks.size`, which is O(RDD blocks) time. */ def numRddBlocks: Int = _rddBlocks.values.map(_.size).sum /** * Return the number of blocks that belong to the given RDD in O(1) time. - * Note that this is much faster than `this.rddBlocksById(rddId).size`, which is + * + * @note This is much faster than `this.rddBlocksById(rddId).size`, which is * O(blocks in this RDD) time. */ def numRddBlocksById(rddId: Int): Int = _rddBlocks.get(rddId).map(_.size).getOrElse(0) + /** Return the max memory can be used by this block manager. */ + def maxMem: Long = maxMemory + /** Return the memory remaining in this block manager. */ def memRemaining: Long = maxMem - memUsed + /** Return the memory used by caching RDDs */ + def cacheSize: Long = onHeapCacheSize.getOrElse(0L) + offHeapCacheSize.getOrElse(0L) + /** Return the memory used by this block manager. */ - def memUsed: Long = _nonRddStorageInfo._1 + _rddBlocks.keys.toSeq.map(memUsedByRdd).sum + def memUsed: Long = onHeapMemUsed.getOrElse(0L) + offHeapMemUsed.getOrElse(0L) - /** Return the disk space used by this block manager. */ - def diskUsed: Long = _nonRddStorageInfo._2 + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum + /** Return the on-heap memory remaining in this block manager. */ + def onHeapMemRemaining: Option[Long] = + for (m <- maxOnHeapMem; o <- onHeapMemUsed) yield m - o + + /** Return the off-heap memory remaining in this block manager. */ + def offHeapMemRemaining: Option[Long] = + for (m <- maxOffHeapMem; o <- offHeapMemUsed) yield m - o + + /** Return the on-heap memory used by this block manager. */ + def onHeapMemUsed: Option[Long] = onHeapCacheSize.map(_ + _nonRddStorageInfo.onHeapUsage) - /** Return the off-heap space used by this block manager. */ - def offHeapUsed: Long = _nonRddStorageInfo._3 + _rddBlocks.keys.toSeq.map(offHeapUsedByRdd).sum + /** Return the off-heap memory used by this block manager. */ + def offHeapMemUsed: Option[Long] = offHeapCacheSize.map(_ + _nonRddStorageInfo.offHeapUsage) + + /** Return the memory used by on-heap caching RDDs */ + def onHeapCacheSize: Option[Long] = maxOnHeapMem.map { _ => + _rddStorageInfo.collect { + case (_, storageInfo) if !storageInfo.level.useOffHeap => storageInfo.memoryUsage + }.sum + } + + /** Return the memory used by off-heap caching RDDs */ + def offHeapCacheSize: Option[Long] = maxOffHeapMem.map { _ => + _rddStorageInfo.collect { + case (_, storageInfo) if storageInfo.level.useOffHeap => storageInfo.memoryUsage + }.sum + } + + /** Return the disk space used by this block manager. */ + def diskUsed: Long = _nonRddStorageInfo.diskUsage + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum /** Return the memory used by the given RDD in this block manager in O(1) time. */ - def memUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._1).getOrElse(0L) + def memUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_.memoryUsage).getOrElse(0L) /** Return the disk space used by the given RDD in this block manager in O(1) time. */ - def diskUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._2).getOrElse(0L) - - /** Return the off-heap space used by the given RDD in this block manager in O(1) time. */ - def offHeapUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._3).getOrElse(0L) + def diskUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_.diskUsage).getOrElse(0L) /** Return the storage level, if any, used by the given RDD in this block manager. */ - def rddStorageLevel(rddId: Int): Option[StorageLevel] = _rddStorageInfo.get(rddId).map(_._4) + def rddStorageLevel(rddId: Int): Option[StorageLevel] = _rddStorageInfo.get(rddId).map(_.level) /** * Update the relevant storage info, taking into account any existing status for this block. @@ -199,41 +236,65 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { val oldBlockStatus = getBlock(blockId).getOrElse(BlockStatus.empty) val changeInMem = newBlockStatus.memSize - oldBlockStatus.memSize val changeInDisk = newBlockStatus.diskSize - oldBlockStatus.diskSize - val changeInExternalBlockStore = - newBlockStatus.externalBlockStoreSize - oldBlockStatus.externalBlockStoreSize val level = newBlockStatus.storageLevel // Compute new info from old info - val (oldMem, oldDisk, oldExternalBlockStore) = blockId match { + val (oldMem, oldDisk) = blockId match { case RDDBlockId(rddId, _) => _rddStorageInfo.get(rddId) - .map { case (mem, disk, externalBlockStore, _) => (mem, disk, externalBlockStore) } - .getOrElse((0L, 0L, 0L)) - case _ => - _nonRddStorageInfo + .map { case RddStorageInfo(mem, disk, _) => (mem, disk) } + .getOrElse((0L, 0L)) + case _ if !level.useOffHeap => + (_nonRddStorageInfo.onHeapUsage, _nonRddStorageInfo.diskUsage) + case _ if level.useOffHeap => + (_nonRddStorageInfo.offHeapUsage, _nonRddStorageInfo.diskUsage) } val newMem = math.max(oldMem + changeInMem, 0L) val newDisk = math.max(oldDisk + changeInDisk, 0L) - val newExternalBlockStore = math.max(oldExternalBlockStore + changeInExternalBlockStore, 0L) // Set the correct info blockId match { case RDDBlockId(rddId, _) => // If this RDD is no longer persisted, remove it - if (newMem + newDisk + newExternalBlockStore == 0) { + if (newMem + newDisk == 0) { _rddStorageInfo.remove(rddId) } else { - _rddStorageInfo(rddId) = (newMem, newDisk, newExternalBlockStore, level) + _rddStorageInfo(rddId) = RddStorageInfo(newMem, newDisk, level) } case _ => - _nonRddStorageInfo = (newMem, newDisk, newExternalBlockStore) + if (!level.useOffHeap) { + _nonRddStorageInfo.onHeapUsage = newMem + } else { + _nonRddStorageInfo.offHeapUsage = newMem + } + _nonRddStorageInfo.diskUsage = newDisk } } - } /** Helper methods for storage-related objects. */ -private[spark] object StorageUtils { +private[spark] object StorageUtils extends Logging { + /** + * Attempt to clean up a ByteBuffer if it is direct or memory-mapped. This uses an *unsafe* Sun + * API that will cause errors if one attempts to read from the disposed buffer. However, neither + * the bytes allocated to direct buffers nor file descriptors opened for memory-mapped buffers put + * pressure on the garbage collector. Waiting for garbage collection may lead to the depletion of + * off-heap memory or huge numbers of open files. There's unfortunately no standard API to + * manually dispose of these kinds of buffers. + */ + def dispose(buffer: ByteBuffer): Unit = { + if (buffer != null && buffer.isInstanceOf[MappedByteBuffer]) { + logTrace(s"Disposing of $buffer") + cleanDirectBuffer(buffer.asInstanceOf[DirectBuffer]) + } + } + + private def cleanDirectBuffer(buffer: DirectBuffer) = { + val cleaner = buffer.cleaner() + if (cleaner != null) { + cleaner.clean() + } + } /** * Update the given list of RDDInfo with the given list of storage statuses. @@ -248,13 +309,11 @@ private[spark] object StorageUtils { val numCachedPartitions = statuses.map(_.numRddBlocksById(rddId)).sum val memSize = statuses.map(_.memUsedByRdd(rddId)).sum val diskSize = statuses.map(_.diskUsedByRdd(rddId)).sum - val externalBlockStoreSize = statuses.map(_.offHeapUsedByRdd(rddId)).sum rddInfo.storageLevel = storageLevel rddInfo.numCachedPartitions = numCachedPartitions rddInfo.memSize = memSize rddInfo.diskSize = diskSize - rddInfo.externalBlockStoreSize = externalBlockStoreSize } } diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala deleted file mode 100644 index 22878783fca6..000000000000 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ /dev/null @@ -1,253 +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.storage - -import java.io.IOException -import java.nio.ByteBuffer -import java.text.SimpleDateFormat -import java.util.{Date, Random} - -import scala.util.control.NonFatal - -import com.google.common.io.ByteStreams - -import tachyon.client.{ReadType, WriteType, TachyonFS, TachyonFile} -import tachyon.conf.TachyonConf -import tachyon.TachyonURI - -import org.apache.spark.Logging -import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.util.{ShutdownHookManager, Utils} - - -/** - * Creates and maintains the logical mapping between logical blocks and tachyon fs locations. By - * default, one block is mapped to one file with a name given by its BlockId. - * - */ -private[spark] class TachyonBlockManager() extends ExternalBlockManager with Logging { - - var rootDirs: String = _ - var master: String = _ - var client: tachyon.client.TachyonFS = _ - private var subDirsPerTachyonDir: Int = _ - - // Create one Tachyon directory for each path mentioned in spark.tachyonStore.folderName; - // then, inside this directory, create multiple subdirectories that we will hash files into, - // in order to avoid having really large inodes at the top level in Tachyon. - private var tachyonDirs: Array[TachyonFile] = _ - private var subDirs: Array[Array[tachyon.client.TachyonFile]] = _ - - - override def init(blockManager: BlockManager, executorId: String): Unit = { - super.init(blockManager, executorId) - val storeDir = blockManager.conf.get(ExternalBlockStore.BASE_DIR, "/tmp_spark_tachyon") - val appFolderName = blockManager.conf.get(ExternalBlockStore.FOLD_NAME) - - rootDirs = s"$storeDir/$appFolderName/$executorId" - master = blockManager.conf.get(ExternalBlockStore.MASTER_URL, "tachyon://localhost:19998") - client = if (master != null && master != "") { - TachyonFS.get(new TachyonURI(master), new TachyonConf()) - } else { - null - } - // original implementation call System.exit, we change it to run without extblkstore support - if (client == null) { - logError("Failed to connect to the Tachyon as the master address is not configured") - throw new IOException("Failed to connect to the Tachyon as the master " + - "address is not configured") - } - subDirsPerTachyonDir = blockManager.conf.get("spark.externalBlockStore.subDirectories", - ExternalBlockStore.SUB_DIRS_PER_DIR).toInt - - // Create one Tachyon directory for each path mentioned in spark.tachyonStore.folderName; - // then, inside this directory, create multiple subdirectories that we will hash files into, - // in order to avoid having really large inodes at the top level in Tachyon. - tachyonDirs = createTachyonDirs() - subDirs = Array.fill(tachyonDirs.length)(new Array[TachyonFile](subDirsPerTachyonDir)) - tachyonDirs.foreach(tachyonDir => ShutdownHookManager.registerShutdownDeleteDir(tachyonDir)) - } - - override def toString: String = {"ExternalBlockStore-Tachyon"} - - override def removeBlock(blockId: BlockId): Boolean = { - val file = getFile(blockId) - if (fileExists(file)) { - removeFile(file) - } else { - false - } - } - - override def blockExists(blockId: BlockId): Boolean = { - val file = getFile(blockId) - fileExists(file) - } - - override def putBytes(blockId: BlockId, bytes: ByteBuffer): Unit = { - val file = getFile(blockId) - val os = file.getOutStream(WriteType.TRY_CACHE) - try { - os.write(bytes.array()) - } catch { - case NonFatal(e) => - logWarning(s"Failed to put bytes of block $blockId into Tachyon", e) - os.cancel() - } finally { - os.close() - } - } - - override def putValues(blockId: BlockId, values: Iterator[_]): Unit = { - val file = getFile(blockId) - val os = file.getOutStream(WriteType.TRY_CACHE) - try { - blockManager.dataSerializeStream(blockId, os, values) - } catch { - case NonFatal(e) => - logWarning(s"Failed to put values of block $blockId into Tachyon", e) - os.cancel() - } finally { - os.close() - } - } - - override def getBytes(blockId: BlockId): Option[ByteBuffer] = { - val file = getFile(blockId) - if (file == null || file.getLocationHosts.size == 0) { - return None - } - val is = file.getInStream(ReadType.CACHE) - try { - val size = file.length - val bs = new Array[Byte](size.asInstanceOf[Int]) - ByteStreams.readFully(is, bs) - Some(ByteBuffer.wrap(bs)) - } catch { - case NonFatal(e) => - logWarning(s"Failed to get bytes of block $blockId from Tachyon", e) - None - } finally { - is.close() - } - } - - override def getValues(blockId: BlockId): Option[Iterator[_]] = { - val file = getFile(blockId) - if (file == null || file.getLocationHosts().size() == 0) { - return None - } - val is = file.getInStream(ReadType.CACHE) - Option(is).map { is => - blockManager.dataDeserializeStream(blockId, is) - } - } - - override def getSize(blockId: BlockId): Long = { - getFile(blockId.name).length - } - - def removeFile(file: TachyonFile): Boolean = { - client.delete(new TachyonURI(file.getPath()), false) - } - - def fileExists(file: TachyonFile): Boolean = { - client.exist(new TachyonURI(file.getPath())) - } - - def getFile(filename: String): TachyonFile = { - // Figure out which tachyon directory it hashes to, and which subdirectory in that - val hash = Utils.nonNegativeHash(filename) - val dirId = hash % tachyonDirs.length - val subDirId = (hash / tachyonDirs.length) % subDirsPerTachyonDir - - // Create the subdirectory if it doesn't already exist - var subDir = subDirs(dirId)(subDirId) - if (subDir == null) { - subDir = subDirs(dirId).synchronized { - val old = subDirs(dirId)(subDirId) - if (old != null) { - old - } else { - val path = new TachyonURI(s"${tachyonDirs(dirId)}/${"%02x".format(subDirId)}") - client.mkdir(path) - val newDir = client.getFile(path) - subDirs(dirId)(subDirId) = newDir - newDir - } - } - } - val filePath = new TachyonURI(s"$subDir/$filename") - if(!client.exist(filePath)) { - client.createFile(filePath) - } - val file = client.getFile(filePath) - file - } - - def getFile(blockId: BlockId): TachyonFile = getFile(blockId.name) - - // TODO: Some of the logic here could be consolidated/de-duplicated with that in the DiskStore. - private def createTachyonDirs(): Array[TachyonFile] = { - logDebug("Creating tachyon directories at root dirs '" + rootDirs + "'") - val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") - rootDirs.split(",").map { rootDir => - var foundLocalDir = false - var tachyonDir: TachyonFile = null - var tachyonDirId: String = null - var tries = 0 - val rand = new Random() - while (!foundLocalDir && tries < ExternalBlockStore.MAX_DIR_CREATION_ATTEMPTS) { - tries += 1 - try { - tachyonDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536)) - val path = new TachyonURI(s"$rootDir/spark-tachyon-$tachyonDirId") - if (!client.exist(path)) { - foundLocalDir = client.mkdir(path) - tachyonDir = client.getFile(path) - } - } catch { - case NonFatal(e) => - logWarning("Attempt " + tries + " to create tachyon dir " + tachyonDir + " failed", e) - } - } - if (!foundLocalDir) { - logError("Failed " + ExternalBlockStore.MAX_DIR_CREATION_ATTEMPTS - + " attempts to create tachyon dir in " + rootDir) - System.exit(ExecutorExitCode.EXTERNAL_BLOCK_STORE_FAILED_TO_CREATE_DIR) - } - logInfo("Created tachyon directory at " + tachyonDir) - tachyonDir - } - } - - override def shutdown() { - logDebug("Shutdown hook called") - tachyonDirs.foreach { tachyonDir => - try { - if (!ShutdownHookManager.hasRootAsShutdownDeleteDir(tachyonDir)) { - Utils.deleteRecursively(tachyonDir, client) - } - } catch { - case NonFatal(e) => - logError("Exception while deleting tachyon spark dir: " + tachyonDir, e) - } - } - client.close() - } -} diff --git a/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala b/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala new file mode 100644 index 000000000000..a150a8e3636e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import org.apache.spark.SparkConf +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +/** + * ::DeveloperApi:: + * TopologyMapper provides topology information for a given host + * @param conf SparkConf to get required properties, if needed + */ +@DeveloperApi +abstract class TopologyMapper(conf: SparkConf) { + /** + * Gets the topology information given the host name + * + * @param hostname Hostname + * @return topology information for the given hostname. One can use a 'topology delimiter' + * to make this topology information nested. + * For example : ‘/myrack/myhost’, where ‘/’ is the topology delimiter, + * ‘myrack’ is the topology identifier, and ‘myhost’ is the individual host. + * This function only returns the topology information without the hostname. + * This information can be used when choosing executors for block replication + * to discern executors from a different rack than a candidate executor, for example. + * + * An implementation can choose to use empty strings or None in case topology info + * is not available. This would imply that all such executors belong to the same rack. + */ + def getTopologyForHost(hostname: String): Option[String] +} + +/** + * A TopologyMapper that assumes all nodes are in the same rack + */ +@DeveloperApi +class DefaultTopologyMapper(conf: SparkConf) extends TopologyMapper(conf) with Logging { + override def getTopologyForHost(hostname: String): Option[String] = { + logDebug(s"Got a request for $hostname") + None + } +} + +/** + * A simple file based topology mapper. This expects topology information provided as a + * `java.util.Properties` file. The name of the file is obtained from SparkConf property + * `spark.storage.replication.topologyFile`. To use this topology mapper, set the + * `spark.storage.replication.topologyMapper` property to + * [[org.apache.spark.storage.FileBasedTopologyMapper]] + * @param conf SparkConf object + */ +@DeveloperApi +class FileBasedTopologyMapper(conf: SparkConf) extends TopologyMapper(conf) with Logging { + val topologyFile = conf.getOption("spark.storage.replication.topologyFile") + require(topologyFile.isDefined, "Please specify topology file via " + + "spark.storage.replication.topologyFile for FileBasedTopologyMapper.") + val topologyMap = Utils.getPropertiesFromFile(topologyFile.get) + + override def getTopologyForHost(hostname: String): Option[String] = { + val topology = topologyMap.get(hostname) + if (topology.isDefined) { + logDebug(s"$hostname -> ${topology.get}") + } else { + logWarning(s"$hostname does not have any topology information") + } + topology + } +} + diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala new file mode 100644 index 000000000000..651e9c7b2ab6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -0,0 +1,889 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage.memory + +import java.io.OutputStream +import java.nio.ByteBuffer +import java.util.LinkedHashMap + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + +import com.google.common.io.ByteStreams + +import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.{UNROLL_MEMORY_CHECK_PERIOD, UNROLL_MEMORY_GROWTH_FACTOR} +import org.apache.spark.memory.{MemoryManager, MemoryMode} +import org.apache.spark.serializer.{SerializationStream, SerializerManager} +import org.apache.spark.storage._ +import org.apache.spark.unsafe.Platform +import org.apache.spark.util.{SizeEstimator, Utils} +import org.apache.spark.util.collection.SizeTrackingVector +import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} + +private sealed trait MemoryEntry[T] { + def size: Long + def memoryMode: MemoryMode + def classTag: ClassTag[T] +} +private case class DeserializedMemoryEntry[T]( + value: Array[T], + size: Long, + classTag: ClassTag[T]) extends MemoryEntry[T] { + val memoryMode: MemoryMode = MemoryMode.ON_HEAP +} +private case class SerializedMemoryEntry[T]( + buffer: ChunkedByteBuffer, + memoryMode: MemoryMode, + classTag: ClassTag[T]) extends MemoryEntry[T] { + def size: Long = buffer.size +} + +private[storage] trait BlockEvictionHandler { + /** + * Drop a block from memory, possibly putting it on disk if applicable. Called when the memory + * store reaches its limit and needs to free up space. + * + * If `data` is not put on disk, it won't be created. + * + * The caller of this method must hold a write lock on the block before calling this method. + * This method does not release the write lock. + * + * @return the block's new effective StorageLevel. + */ + private[storage] def dropFromMemory[T: ClassTag]( + blockId: BlockId, + data: () => Either[Array[T], ChunkedByteBuffer]): StorageLevel +} + +/** + * Stores blocks in memory, either as Arrays of deserialized Java objects or as + * serialized ByteBuffers. + */ +private[spark] class MemoryStore( + conf: SparkConf, + blockInfoManager: BlockInfoManager, + serializerManager: SerializerManager, + memoryManager: MemoryManager, + blockEvictionHandler: BlockEvictionHandler) + extends Logging { + + // Note: all changes to memory allocations, notably putting blocks, evicting blocks, and + // acquiring or releasing unroll memory, must be synchronized on `memoryManager`! + + private val entries = new LinkedHashMap[BlockId, MemoryEntry[_]](32, 0.75f, true) + + // A mapping from taskAttemptId to amount of memory used for unrolling a block (in bytes) + // All accesses of this map are assumed to have manually synchronized on `memoryManager` + private val onHeapUnrollMemoryMap = mutable.HashMap[Long, Long]() + // Note: off-heap unroll memory is only used in putIteratorAsBytes() because off-heap caching + // always stores serialized values. + private val offHeapUnrollMemoryMap = mutable.HashMap[Long, Long]() + + // Initial memory to request before unrolling any block + private val unrollMemoryThreshold: Long = + conf.getLong("spark.storage.unrollMemoryThreshold", 1024 * 1024) + + /** Total amount of memory available for storage, in bytes. */ + private def maxMemory: Long = { + memoryManager.maxOnHeapStorageMemory + memoryManager.maxOffHeapStorageMemory + } + + if (maxMemory < unrollMemoryThreshold) { + logWarning(s"Max memory ${Utils.bytesToString(maxMemory)} is less than the initial memory " + + s"threshold ${Utils.bytesToString(unrollMemoryThreshold)} needed to store a block in " + + s"memory. Please configure Spark with more memory.") + } + + logInfo("MemoryStore started with capacity %s".format(Utils.bytesToString(maxMemory))) + + /** Total storage memory used including unroll memory, in bytes. */ + private def memoryUsed: Long = memoryManager.storageMemoryUsed + + /** + * Amount of storage memory, in bytes, used for caching blocks. + * This does not include memory used for unrolling. + */ + private def blocksMemoryUsed: Long = memoryManager.synchronized { + memoryUsed - currentUnrollMemory + } + + def getSize(blockId: BlockId): Long = { + entries.synchronized { + entries.get(blockId).size + } + } + + /** + * Use `size` to test if there is enough space in MemoryStore. If so, create the ByteBuffer and + * put it into MemoryStore. Otherwise, the ByteBuffer won't be created. + * + * The caller should guarantee that `size` is correct. + * + * @return true if the put() succeeded, false otherwise. + */ + def putBytes[T: ClassTag]( + blockId: BlockId, + size: Long, + memoryMode: MemoryMode, + _bytes: () => ChunkedByteBuffer): Boolean = { + require(!contains(blockId), s"Block $blockId is already present in the MemoryStore") + if (memoryManager.acquireStorageMemory(blockId, size, memoryMode)) { + // We acquired enough memory for the block, so go ahead and put it + val bytes = _bytes() + assert(bytes.size == size) + val entry = new SerializedMemoryEntry[T](bytes, memoryMode, implicitly[ClassTag[T]]) + entries.synchronized { + entries.put(blockId, entry) + } + logInfo("Block %s stored as bytes in memory (estimated size %s, free %s)".format( + blockId, Utils.bytesToString(size), Utils.bytesToString(maxMemory - blocksMemoryUsed))) + true + } else { + false + } + } + + /** + * Attempt to put the given block in memory store as values. + * + * It's possible that the iterator is too large to materialize and store in memory. To avoid + * OOM exceptions, this method will gradually unroll the iterator while periodically checking + * whether there is enough free memory. If the block is successfully materialized, then the + * temporary unroll memory used during the materialization is "transferred" to storage memory, + * so we won't acquire more memory than is actually needed to store the block. + * + * @return in case of success, the estimated size of the stored data. In case of failure, return + * an iterator containing the values of the block. The returned iterator will be backed + * by the combination of the partially-unrolled block and the remaining elements of the + * original input iterator. The caller must either fully consume this iterator or call + * `close()` on it in order to free the storage memory consumed by the partially-unrolled + * block. + */ + private[storage] def putIteratorAsValues[T]( + blockId: BlockId, + values: Iterator[T], + classTag: ClassTag[T]): Either[PartiallyUnrolledIterator[T], Long] = { + + require(!contains(blockId), s"Block $blockId is already present in the MemoryStore") + + // Number of elements unrolled so far + var elementsUnrolled = 0 + // Whether there is still enough memory for us to continue unrolling this block + var keepUnrolling = true + // Initial per-task memory to request for unrolling blocks (bytes). + val initialMemoryThreshold = unrollMemoryThreshold + // How often to check whether we need to request more memory + val memoryCheckPeriod = conf.get(UNROLL_MEMORY_CHECK_PERIOD) + // Memory currently reserved by this task for this particular unrolling operation + var memoryThreshold = initialMemoryThreshold + // Memory to request as a multiple of current vector size + val memoryGrowthFactor = conf.get(UNROLL_MEMORY_GROWTH_FACTOR) + // Keep track of unroll memory used by this particular block / putIterator() operation + var unrollMemoryUsedByThisBlock = 0L + // Underlying vector for unrolling the block + var vector = new SizeTrackingVector[T]()(classTag) + + // Request enough memory to begin unrolling + keepUnrolling = + reserveUnrollMemoryForThisTask(blockId, initialMemoryThreshold, MemoryMode.ON_HEAP) + + if (!keepUnrolling) { + logWarning(s"Failed to reserve initial memory threshold of " + + s"${Utils.bytesToString(initialMemoryThreshold)} for computing block $blockId in memory.") + } else { + unrollMemoryUsedByThisBlock += initialMemoryThreshold + } + + // Unroll this block safely, checking whether we have exceeded our threshold periodically + while (values.hasNext && keepUnrolling) { + vector += values.next() + if (elementsUnrolled % memoryCheckPeriod == 0) { + // If our vector's size has exceeded the threshold, request more memory + val currentSize = vector.estimateSize() + if (currentSize >= memoryThreshold) { + val amountToRequest = (currentSize * memoryGrowthFactor - memoryThreshold).toLong + keepUnrolling = + reserveUnrollMemoryForThisTask(blockId, amountToRequest, MemoryMode.ON_HEAP) + if (keepUnrolling) { + unrollMemoryUsedByThisBlock += amountToRequest + } + // New threshold is currentSize * memoryGrowthFactor + memoryThreshold += amountToRequest + } + } + elementsUnrolled += 1 + } + + if (keepUnrolling) { + // We successfully unrolled the entirety of this block + val arrayValues = vector.toArray + vector = null + val entry = + new DeserializedMemoryEntry[T](arrayValues, SizeEstimator.estimate(arrayValues), classTag) + val size = entry.size + def transferUnrollToStorage(amount: Long): Unit = { + // Synchronize so that transfer is atomic + memoryManager.synchronized { + releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, amount) + val success = memoryManager.acquireStorageMemory(blockId, amount, MemoryMode.ON_HEAP) + assert(success, "transferring unroll memory to storage memory failed") + } + } + // Acquire storage memory if necessary to store this block in memory. + val enoughStorageMemory = { + if (unrollMemoryUsedByThisBlock <= size) { + val acquiredExtra = + memoryManager.acquireStorageMemory( + blockId, size - unrollMemoryUsedByThisBlock, MemoryMode.ON_HEAP) + if (acquiredExtra) { + transferUnrollToStorage(unrollMemoryUsedByThisBlock) + } + acquiredExtra + } else { // unrollMemoryUsedByThisBlock > size + // If this task attempt already owns more unroll memory than is necessary to store the + // block, then release the extra memory that will not be used. + val excessUnrollMemory = unrollMemoryUsedByThisBlock - size + releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, excessUnrollMemory) + transferUnrollToStorage(size) + true + } + } + if (enoughStorageMemory) { + entries.synchronized { + entries.put(blockId, entry) + } + logInfo("Block %s stored as values in memory (estimated size %s, free %s)".format( + blockId, Utils.bytesToString(size), Utils.bytesToString(maxMemory - blocksMemoryUsed))) + Right(size) + } else { + assert(currentUnrollMemoryForThisTask >= unrollMemoryUsedByThisBlock, + "released too much unroll memory") + Left(new PartiallyUnrolledIterator( + this, + MemoryMode.ON_HEAP, + unrollMemoryUsedByThisBlock, + unrolled = arrayValues.toIterator, + rest = Iterator.empty)) + } + } else { + // We ran out of space while unrolling the values for this block + logUnrollFailureMessage(blockId, vector.estimateSize()) + Left(new PartiallyUnrolledIterator( + this, + MemoryMode.ON_HEAP, + unrollMemoryUsedByThisBlock, + unrolled = vector.iterator, + rest = values)) + } + } + + /** + * Attempt to put the given block in memory store as bytes. + * + * It's possible that the iterator is too large to materialize and store in memory. To avoid + * OOM exceptions, this method will gradually unroll the iterator while periodically checking + * whether there is enough free memory. If the block is successfully materialized, then the + * temporary unroll memory used during the materialization is "transferred" to storage memory, + * so we won't acquire more memory than is actually needed to store the block. + * + * @return in case of success, the estimated size of the stored data. In case of failure, + * return a handle which allows the caller to either finish the serialization by + * spilling to disk or to deserialize the partially-serialized block and reconstruct + * the original input iterator. The caller must either fully consume this result + * iterator or call `discard()` on it in order to free the storage memory consumed by the + * partially-unrolled block. + */ + private[storage] def putIteratorAsBytes[T]( + blockId: BlockId, + values: Iterator[T], + classTag: ClassTag[T], + memoryMode: MemoryMode): Either[PartiallySerializedBlock[T], Long] = { + + require(!contains(blockId), s"Block $blockId is already present in the MemoryStore") + + val allocator = memoryMode match { + case MemoryMode.ON_HEAP => ByteBuffer.allocate _ + case MemoryMode.OFF_HEAP => Platform.allocateDirectBuffer _ + } + + // Whether there is still enough memory for us to continue unrolling this block + var keepUnrolling = true + // Number of elements unrolled so far + var elementsUnrolled = 0L + // How often to check whether we need to request more memory + val memoryCheckPeriod = conf.get(UNROLL_MEMORY_CHECK_PERIOD) + // Memory to request as a multiple of current bbos size + val memoryGrowthFactor = conf.get(UNROLL_MEMORY_GROWTH_FACTOR) + // Initial per-task memory to request for unrolling blocks (bytes). + val initialMemoryThreshold = unrollMemoryThreshold + // Keep track of unroll memory used by this particular block / putIterator() operation + var unrollMemoryUsedByThisBlock = 0L + // Underlying buffer for unrolling the block + val redirectableStream = new RedirectableOutputStream + val chunkSize = if (initialMemoryThreshold > Int.MaxValue) { + logWarning(s"Initial memory threshold of ${Utils.bytesToString(initialMemoryThreshold)} " + + s"is too large to be set as chunk size. Chunk size has been capped to " + + s"${Utils.bytesToString(Int.MaxValue)}") + Int.MaxValue + } else { + initialMemoryThreshold.toInt + } + val bbos = new ChunkedByteBufferOutputStream(chunkSize, allocator) + redirectableStream.setOutputStream(bbos) + val serializationStream: SerializationStream = { + val autoPick = !blockId.isInstanceOf[StreamBlockId] + val ser = serializerManager.getSerializer(classTag, autoPick).newInstance() + ser.serializeStream(serializerManager.wrapForCompression(blockId, redirectableStream)) + } + + // Request enough memory to begin unrolling + keepUnrolling = reserveUnrollMemoryForThisTask(blockId, initialMemoryThreshold, memoryMode) + + if (!keepUnrolling) { + logWarning(s"Failed to reserve initial memory threshold of " + + s"${Utils.bytesToString(initialMemoryThreshold)} for computing block $blockId in memory.") + } else { + unrollMemoryUsedByThisBlock += initialMemoryThreshold + } + + def reserveAdditionalMemoryIfNecessary(): Unit = { + if (bbos.size > unrollMemoryUsedByThisBlock) { + val amountToRequest = (bbos.size * memoryGrowthFactor - unrollMemoryUsedByThisBlock).toLong + keepUnrolling = reserveUnrollMemoryForThisTask(blockId, amountToRequest, memoryMode) + if (keepUnrolling) { + unrollMemoryUsedByThisBlock += amountToRequest + } + } + } + + // Unroll this block safely, checking whether we have exceeded our threshold + while (values.hasNext && keepUnrolling) { + serializationStream.writeObject(values.next())(classTag) + elementsUnrolled += 1 + if (elementsUnrolled % memoryCheckPeriod == 0) { + reserveAdditionalMemoryIfNecessary() + } + } + + // Make sure that we have enough memory to store the block. By this point, it is possible that + // the block's actual memory usage has exceeded the unroll memory by a small amount, so we + // perform one final call to attempt to allocate additional memory if necessary. + if (keepUnrolling) { + serializationStream.close() + reserveAdditionalMemoryIfNecessary() + } + + if (keepUnrolling) { + val entry = SerializedMemoryEntry[T](bbos.toChunkedByteBuffer, memoryMode, classTag) + // Synchronize so that transfer is atomic + memoryManager.synchronized { + releaseUnrollMemoryForThisTask(memoryMode, unrollMemoryUsedByThisBlock) + val success = memoryManager.acquireStorageMemory(blockId, entry.size, memoryMode) + assert(success, "transferring unroll memory to storage memory failed") + } + entries.synchronized { + entries.put(blockId, entry) + } + logInfo("Block %s stored as bytes in memory (estimated size %s, free %s)".format( + blockId, Utils.bytesToString(entry.size), + Utils.bytesToString(maxMemory - blocksMemoryUsed))) + Right(entry.size) + } else { + // We ran out of space while unrolling the values for this block + logUnrollFailureMessage(blockId, bbos.size) + Left( + new PartiallySerializedBlock( + this, + serializerManager, + blockId, + serializationStream, + redirectableStream, + unrollMemoryUsedByThisBlock, + memoryMode, + bbos, + values, + classTag)) + } + } + + def getBytes(blockId: BlockId): Option[ChunkedByteBuffer] = { + val entry = entries.synchronized { entries.get(blockId) } + entry match { + case null => None + case e: DeserializedMemoryEntry[_] => + throw new IllegalArgumentException("should only call getBytes on serialized blocks") + case SerializedMemoryEntry(bytes, _, _) => Some(bytes) + } + } + + def getValues(blockId: BlockId): Option[Iterator[_]] = { + val entry = entries.synchronized { entries.get(blockId) } + entry match { + case null => None + case e: SerializedMemoryEntry[_] => + throw new IllegalArgumentException("should only call getValues on deserialized blocks") + case DeserializedMemoryEntry(values, _, _) => + val x = Some(values) + x.map(_.iterator) + } + } + + def remove(blockId: BlockId): Boolean = memoryManager.synchronized { + val entry = entries.synchronized { + entries.remove(blockId) + } + if (entry != null) { + entry match { + case SerializedMemoryEntry(buffer, _, _) => buffer.dispose() + case _ => + } + memoryManager.releaseStorageMemory(entry.size, entry.memoryMode) + logDebug(s"Block $blockId of size ${entry.size} dropped " + + s"from memory (free ${maxMemory - blocksMemoryUsed})") + true + } else { + false + } + } + + def clear(): Unit = memoryManager.synchronized { + entries.synchronized { + entries.clear() + } + onHeapUnrollMemoryMap.clear() + offHeapUnrollMemoryMap.clear() + memoryManager.releaseAllStorageMemory() + logInfo("MemoryStore cleared") + } + + /** + * Return the RDD ID that a given block ID is from, or None if it is not an RDD block. + */ + private def getRddId(blockId: BlockId): Option[Int] = { + blockId.asRDDId.map(_.rddId) + } + + /** + * Try to evict blocks to free up a given amount of space to store a particular block. + * Can fail if either the block is bigger than our memory or it would require replacing + * another block from the same RDD (which leads to a wasteful cyclic replacement pattern for + * RDDs that don't fit into memory that we want to avoid). + * + * @param blockId the ID of the block we are freeing space for, if any + * @param space the size of this block + * @param memoryMode the type of memory to free (on- or off-heap) + * @return the amount of memory (in bytes) freed by eviction + */ + private[spark] def evictBlocksToFreeSpace( + blockId: Option[BlockId], + space: Long, + memoryMode: MemoryMode): Long = { + assert(space > 0) + memoryManager.synchronized { + var freedMemory = 0L + val rddToAdd = blockId.flatMap(getRddId) + val selectedBlocks = new ArrayBuffer[BlockId] + def blockIsEvictable(blockId: BlockId, entry: MemoryEntry[_]): Boolean = { + entry.memoryMode == memoryMode && (rddToAdd.isEmpty || rddToAdd != getRddId(blockId)) + } + // This is synchronized to ensure that the set of entries is not changed + // (because of getValue or getBytes) while traversing the iterator, as that + // can lead to exceptions. + entries.synchronized { + val iterator = entries.entrySet().iterator() + while (freedMemory < space && iterator.hasNext) { + val pair = iterator.next() + val blockId = pair.getKey + val entry = pair.getValue + if (blockIsEvictable(blockId, entry)) { + // We don't want to evict blocks which are currently being read, so we need to obtain + // an exclusive write lock on blocks which are candidates for eviction. We perform a + // non-blocking "tryLock" here in order to ignore blocks which are locked for reading: + if (blockInfoManager.lockForWriting(blockId, blocking = false).isDefined) { + selectedBlocks += blockId + freedMemory += pair.getValue.size + } + } + } + } + + def dropBlock[T](blockId: BlockId, entry: MemoryEntry[T]): Unit = { + val data = entry match { + case DeserializedMemoryEntry(values, _, _) => Left(values) + case SerializedMemoryEntry(buffer, _, _) => Right(buffer) + } + val newEffectiveStorageLevel = + blockEvictionHandler.dropFromMemory(blockId, () => data)(entry.classTag) + if (newEffectiveStorageLevel.isValid) { + // The block is still present in at least one store, so release the lock + // but don't delete the block info + blockInfoManager.unlock(blockId) + } else { + // The block isn't present in any store, so delete the block info so that the + // block can be stored again + blockInfoManager.removeBlock(blockId) + } + } + + if (freedMemory >= space) { + var lastSuccessfulBlock = -1 + try { + logInfo(s"${selectedBlocks.size} blocks selected for dropping " + + s"(${Utils.bytesToString(freedMemory)} bytes)") + (0 until selectedBlocks.size).foreach { idx => + val blockId = selectedBlocks(idx) + val entry = entries.synchronized { + entries.get(blockId) + } + // This should never be null as only one task should be dropping + // blocks and removing entries. However the check is still here for + // future safety. + if (entry != null) { + dropBlock(blockId, entry) + afterDropAction(blockId) + } + lastSuccessfulBlock = idx + } + logInfo(s"After dropping ${selectedBlocks.size} blocks, " + + s"free memory is ${Utils.bytesToString(maxMemory - blocksMemoryUsed)}") + freedMemory + } finally { + // like BlockManager.doPut, we use a finally rather than a catch to avoid having to deal + // with InterruptedException + if (lastSuccessfulBlock != selectedBlocks.size - 1) { + // the blocks we didn't process successfully are still locked, so we have to unlock them + (lastSuccessfulBlock + 1 until selectedBlocks.size).foreach { idx => + val blockId = selectedBlocks(idx) + blockInfoManager.unlock(blockId) + } + } + } + } else { + blockId.foreach { id => + logInfo(s"Will not store $id") + } + selectedBlocks.foreach { id => + blockInfoManager.unlock(id) + } + 0L + } + } + } + + // hook for testing, so we can simulate a race + protected def afterDropAction(blockId: BlockId): Unit = {} + + def contains(blockId: BlockId): Boolean = { + entries.synchronized { entries.containsKey(blockId) } + } + + private def currentTaskAttemptId(): Long = { + // In case this is called on the driver, return an invalid task attempt id. + Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1L) + } + + /** + * Reserve memory for unrolling the given block for this task. + * + * @return whether the request is granted. + */ + def reserveUnrollMemoryForThisTask( + blockId: BlockId, + memory: Long, + memoryMode: MemoryMode): Boolean = { + memoryManager.synchronized { + val success = memoryManager.acquireUnrollMemory(blockId, memory, memoryMode) + if (success) { + val taskAttemptId = currentTaskAttemptId() + val unrollMemoryMap = memoryMode match { + case MemoryMode.ON_HEAP => onHeapUnrollMemoryMap + case MemoryMode.OFF_HEAP => offHeapUnrollMemoryMap + } + unrollMemoryMap(taskAttemptId) = unrollMemoryMap.getOrElse(taskAttemptId, 0L) + memory + } + success + } + } + + /** + * Release memory used by this task for unrolling blocks. + * If the amount is not specified, remove the current task's allocation altogether. + */ + def releaseUnrollMemoryForThisTask(memoryMode: MemoryMode, memory: Long = Long.MaxValue): Unit = { + val taskAttemptId = currentTaskAttemptId() + memoryManager.synchronized { + val unrollMemoryMap = memoryMode match { + case MemoryMode.ON_HEAP => onHeapUnrollMemoryMap + case MemoryMode.OFF_HEAP => offHeapUnrollMemoryMap + } + if (unrollMemoryMap.contains(taskAttemptId)) { + val memoryToRelease = math.min(memory, unrollMemoryMap(taskAttemptId)) + if (memoryToRelease > 0) { + unrollMemoryMap(taskAttemptId) -= memoryToRelease + memoryManager.releaseUnrollMemory(memoryToRelease, memoryMode) + } + if (unrollMemoryMap(taskAttemptId) == 0) { + unrollMemoryMap.remove(taskAttemptId) + } + } + } + } + + /** + * Return the amount of memory currently occupied for unrolling blocks across all tasks. + */ + def currentUnrollMemory: Long = memoryManager.synchronized { + onHeapUnrollMemoryMap.values.sum + offHeapUnrollMemoryMap.values.sum + } + + /** + * Return the amount of memory currently occupied for unrolling blocks by this task. + */ + def currentUnrollMemoryForThisTask: Long = memoryManager.synchronized { + onHeapUnrollMemoryMap.getOrElse(currentTaskAttemptId(), 0L) + + offHeapUnrollMemoryMap.getOrElse(currentTaskAttemptId(), 0L) + } + + /** + * Return the number of tasks currently unrolling blocks. + */ + private def numTasksUnrolling: Int = memoryManager.synchronized { + (onHeapUnrollMemoryMap.keys ++ offHeapUnrollMemoryMap.keys).toSet.size + } + + /** + * Log information about current memory usage. + */ + private def logMemoryUsage(): Unit = { + logInfo( + s"Memory use = ${Utils.bytesToString(blocksMemoryUsed)} (blocks) + " + + s"${Utils.bytesToString(currentUnrollMemory)} (scratch space shared across " + + s"$numTasksUnrolling tasks(s)) = ${Utils.bytesToString(memoryUsed)}. " + + s"Storage limit = ${Utils.bytesToString(maxMemory)}." + ) + } + + /** + * Log a warning for failing to unroll a block. + * + * @param blockId ID of the block we are trying to unroll. + * @param finalVectorSize Final size of the vector before unrolling failed. + */ + private def logUnrollFailureMessage(blockId: BlockId, finalVectorSize: Long): Unit = { + logWarning( + s"Not enough space to cache $blockId in memory! " + + s"(computed ${Utils.bytesToString(finalVectorSize)} so far)" + ) + logMemoryUsage() + } +} + +/** + * The result of a failed [[MemoryStore.putIteratorAsValues()]] call. + * + * @param memoryStore the memoryStore, used for freeing memory. + * @param memoryMode the memory mode (on- or off-heap). + * @param unrollMemory the amount of unroll memory used by the values in `unrolled`. + * @param unrolled an iterator for the partially-unrolled values. + * @param rest the rest of the original iterator passed to + * [[MemoryStore.putIteratorAsValues()]]. + */ +private[storage] class PartiallyUnrolledIterator[T]( + memoryStore: MemoryStore, + memoryMode: MemoryMode, + unrollMemory: Long, + private[this] var unrolled: Iterator[T], + rest: Iterator[T]) + extends Iterator[T] { + + private def releaseUnrollMemory(): Unit = { + memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory) + // SPARK-17503: Garbage collects the unrolling memory before the life end of + // PartiallyUnrolledIterator. + unrolled = null + } + + override def hasNext: Boolean = { + if (unrolled == null) { + rest.hasNext + } else if (!unrolled.hasNext) { + releaseUnrollMemory() + rest.hasNext + } else { + true + } + } + + override def next(): T = { + if (unrolled == null || !unrolled.hasNext) { + rest.next() + } else { + unrolled.next() + } + } + + /** + * Called to dispose of this iterator and free its memory. + */ + def close(): Unit = { + if (unrolled != null) { + releaseUnrollMemory() + } + } +} + +/** + * A wrapper which allows an open [[OutputStream]] to be redirected to a different sink. + */ +private[storage] class RedirectableOutputStream extends OutputStream { + private[this] var os: OutputStream = _ + def setOutputStream(s: OutputStream): Unit = { os = s } + override def write(b: Int): Unit = os.write(b) + override def write(b: Array[Byte]): Unit = os.write(b) + override def write(b: Array[Byte], off: Int, len: Int): Unit = os.write(b, off, len) + override def flush(): Unit = os.flush() + override def close(): Unit = os.close() +} + +/** + * The result of a failed [[MemoryStore.putIteratorAsBytes()]] call. + * + * @param memoryStore the MemoryStore, used for freeing memory. + * @param serializerManager the SerializerManager, used for deserializing values. + * @param blockId the block id. + * @param serializationStream a serialization stream which writes to [[redirectableOutputStream]]. + * @param redirectableOutputStream an OutputStream which can be redirected to a different sink. + * @param unrollMemory the amount of unroll memory used by the values in `unrolled`. + * @param memoryMode whether the unroll memory is on- or off-heap + * @param bbos byte buffer output stream containing the partially-serialized values. + * [[redirectableOutputStream]] initially points to this output stream. + * @param rest the rest of the original iterator passed to + * [[MemoryStore.putIteratorAsValues()]]. + * @param classTag the [[ClassTag]] for the block. + */ +private[storage] class PartiallySerializedBlock[T]( + memoryStore: MemoryStore, + serializerManager: SerializerManager, + blockId: BlockId, + private val serializationStream: SerializationStream, + private val redirectableOutputStream: RedirectableOutputStream, + val unrollMemory: Long, + memoryMode: MemoryMode, + bbos: ChunkedByteBufferOutputStream, + rest: Iterator[T], + classTag: ClassTag[T]) { + + private lazy val unrolledBuffer: ChunkedByteBuffer = { + bbos.close() + bbos.toChunkedByteBuffer + } + + // If the task does not fully consume `valuesIterator` or otherwise fails to consume or dispose of + // this PartiallySerializedBlock then we risk leaking of direct buffers, so we use a task + // completion listener here in order to ensure that `unrolled.dispose()` is called at least once. + // The dispose() method is idempotent, so it's safe to call it unconditionally. + Option(TaskContext.get()).foreach { taskContext => + taskContext.addTaskCompletionListener { _ => + // When a task completes, its unroll memory will automatically be freed. Thus we do not call + // releaseUnrollMemoryForThisTask() here because we want to avoid double-freeing. + unrolledBuffer.dispose() + } + } + + // Exposed for testing + private[storage] def getUnrolledChunkedByteBuffer: ChunkedByteBuffer = unrolledBuffer + + private[this] var discarded = false + private[this] var consumed = false + + private def verifyNotConsumedAndNotDiscarded(): Unit = { + if (consumed) { + throw new IllegalStateException( + "Can only call one of finishWritingToStream() or valuesIterator() and can only call once.") + } + if (discarded) { + throw new IllegalStateException("Cannot call methods on a discarded PartiallySerializedBlock") + } + } + + /** + * Called to dispose of this block and free its memory. + */ + def discard(): Unit = { + if (!discarded) { + try { + // We want to close the output stream in order to free any resources associated with the + // serializer itself (such as Kryo's internal buffers). close() might cause data to be + // written, so redirect the output stream to discard that data. + redirectableOutputStream.setOutputStream(ByteStreams.nullOutputStream()) + serializationStream.close() + } finally { + discarded = true + unrolledBuffer.dispose() + memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory) + } + } + } + + /** + * Finish writing this block to the given output stream by first writing the serialized values + * and then serializing the values from the original input iterator. + */ + def finishWritingToStream(os: OutputStream): Unit = { + verifyNotConsumedAndNotDiscarded() + consumed = true + // `unrolled`'s underlying buffers will be freed once this input stream is fully read: + ByteStreams.copy(unrolledBuffer.toInputStream(dispose = true), os) + memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory) + redirectableOutputStream.setOutputStream(os) + while (rest.hasNext) { + serializationStream.writeObject(rest.next())(classTag) + } + serializationStream.close() + } + + /** + * Returns an iterator over the values in this block by first deserializing the serialized + * values and then consuming the rest of the original input iterator. + * + * If the caller does not plan to fully consume the resulting iterator then they must call + * `close()` on it to free its resources. + */ + def valuesIterator: PartiallyUnrolledIterator[T] = { + verifyNotConsumedAndNotDiscarded() + consumed = true + // Close the serialization stream so that the serializer's internal buffers are freed and any + // "end-of-stream" markers can be written out so that `unrolled` is a valid serialized stream. + serializationStream.close() + // `unrolled`'s underlying buffers will be freed once this input stream is fully read: + val unrolledIter = serializerManager.dataDeserializeStream( + blockId, unrolledBuffer.toInputStream(dispose = true))(classTag) + // The unroll memory will be freed once `unrolledIter` is fully consumed in + // PartiallyUnrolledIterator. If the iterator is not consumed by the end of the task then any + // extra unroll memory will automatically be freed by a `finally` block in `Task`. + new PartiallyUnrolledIterator( + memoryStore, + memoryMode, + unrollMemory, + unrolled = unrolledIter, + rest = rest) + } +} 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 77c0bc8b5360..3ae80ecfd22e 100644 --- a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala +++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala @@ -20,6 +20,7 @@ package org.apache.spark.ui import java.util.{Timer, TimerTask} import org.apache.spark._ +import org.apache.spark.internal.Logging /** * ConsoleProgressBar shows the progress of stages in the next line of the console. It poll the @@ -28,23 +29,24 @@ 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' + // Carriage return + private val CR = '\r' // Update period of progress bar, in milliseconds - val UPDATE_PERIOD = 200L + private val updatePeriodMSec = + sc.getConf.getTimeAsMs("spark.ui.consoleProgress.update.interval", "200") // Delay to show up a progress bar, in milliseconds - val FIRST_DELAY = 500L + private val firstDelayMSec = 500L // The width of terminal - val TerminalWidth = if (!sys.env.getOrElse("COLUMNS", "").isEmpty) { + private val TerminalWidth = if (!sys.env.getOrElse("COLUMNS", "").isEmpty) { sys.env.get("COLUMNS").get.toInt } else { 80 } - var lastFinishTime = 0L - var lastUpdateTime = 0L - var lastProgressBar = "" + private var lastFinishTime = 0L + private var lastUpdateTime = 0L + private var lastProgressBar = "" // Schedule a refresh thread to run periodically private val timer = new Timer("refresh progress", true) @@ -52,19 +54,19 @@ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging { override def run() { refresh() } - }, FIRST_DELAY, UPDATE_PERIOD) + }, firstDelayMSec, updatePeriodMSec) /** * Try to refresh the progress bar in every cycle */ private def refresh(): Unit = synchronized { val now = System.currentTimeMillis() - if (now - lastFinishTime < FIRST_DELAY) { + if (now - lastFinishTime < firstDelayMSec) { return } val stageIds = sc.statusTracker.getActiveStageIds() - val stages = stageIds.map(sc.statusTracker.getStageInfo).flatten.filter(_.numTasks() > 1) - .filter(now - _.submissionTime() > FIRST_DELAY).sortBy(_.stageId()) + val stages = stageIds.flatMap(sc.statusTracker.getStageInfo).filter(_.numTasks() > 1) + .filter(now - _.submissionTime() > firstDelayMSec).sortBy(_.stageId()) if (stages.length > 0) { show(now, stages.take(3)) // display at most 3 stages in same time } @@ -93,7 +95,7 @@ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging { header + bar + tailer }.mkString("") - // only refresh if it's changed of after 1 minute (or the ssh connection will be closed + // only refresh if it's changed OR after 1 minute (or the ssh connection will be closed // after idle some time) if (bar != lastProgressBar || now - lastUpdateTime > 60 * 1000L) { System.err.print(CR + bar) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index b796a44fe01a..5ee04dad6ed4 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -17,21 +17,28 @@ package org.apache.spark.ui -import java.net.{InetSocketAddress, URL} +import java.net.{URI, URL} import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.language.implicitConversions import scala.xml.Node -import org.eclipse.jetty.server.Server +import org.eclipse.jetty.client.HttpClient +import org.eclipse.jetty.client.api.Response +import org.eclipse.jetty.client.http.HttpClientTransportOverHTTP +import org.eclipse.jetty.proxy.ProxyServlet +import org.eclipse.jetty.server._ import org.eclipse.jetty.server.handler._ +import org.eclipse.jetty.server.handler.gzip.GzipHandler import org.eclipse.jetty.servlet._ -import org.eclipse.jetty.util.thread.QueuedThreadPool +import org.eclipse.jetty.util.component.LifeCycle +import org.eclipse.jetty.util.thread.{QueuedThreadPool, ScheduledExecutorScheduler} import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf, SSLOptions} +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** @@ -39,11 +46,14 @@ import org.apache.spark.util.Utils */ private[spark] object JettyUtils extends Logging { + val SPARK_CONNECTOR_NAME = "Spark" + val REDIRECT_CONNECTOR_NAME = "HttpsRedirect" + // Base type for a function that returns something based on an HTTP request. Allows for // implicit conversion from many types of functions to jetty Handlers. type Responder[T] = HttpServletRequest => T - class ServletParams[T <% AnyRef](val responder: Responder[T], + class ServletParams[T <: AnyRef](val responder: Responder[T], val contentType: String, val extractFn: T => String = (in: Any) => in.toString) {} @@ -57,7 +67,7 @@ private[spark] object JettyUtils extends Logging { implicit def textResponderToServlet(responder: Responder[String]): ServletParams[String] = new ServletParams(responder, "text/plain") - def createServlet[T <% AnyRef]( + def createServlet[T <: AnyRef]( servletParams: ServletParams[T], securityMgr: SecurityManager, conf: SparkConf): HttpServlet = { @@ -79,13 +89,11 @@ private[spark] object JettyUtils extends Logging { val result = servletParams.responder(request) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") response.setHeader("X-Frame-Options", xFrameOptionsValue) - // scalastyle:off println - response.getWriter.println(servletParams.extractFn(result)) - // scalastyle:on println + response.getWriter.print(servletParams.extractFn(result)) } else { - response.setStatus(HttpServletResponse.SC_UNAUTHORIZED) + response.setStatus(HttpServletResponse.SC_FORBIDDEN) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") - response.sendError(HttpServletResponse.SC_UNAUTHORIZED, + response.sendError(HttpServletResponse.SC_FORBIDDEN, "User is not authorized to access this page.") } } catch { @@ -104,7 +112,7 @@ private[spark] object JettyUtils extends Logging { } /** Create a context handler that responds to a request with the given path prefix */ - def createServletHandler[T <% AnyRef]( + def createServletHandler[T <: AnyRef]( path: String, servletParams: ServletParams[T], securityMgr: SecurityManager, @@ -184,6 +192,62 @@ private[spark] object JettyUtils extends Logging { contextHandler } + /** Create a handler for proxying request to Workers and Application Drivers */ + def createProxyHandler(idToUiAddress: String => Option[String]): ServletContextHandler = { + val servlet = new ProxyServlet { + override def rewriteTarget(request: HttpServletRequest): String = { + val path = request.getPathInfo + if (path == null) return null + + val prefixTrailingSlashIndex = path.indexOf('/', 1) + val prefix = if (prefixTrailingSlashIndex == -1) { + path + } else { + path.substring(0, prefixTrailingSlashIndex) + } + val id = prefix.drop(1) + + // Query master state for id's corresponding UI address + // If that address exists, try to turn it into a valid, target URI string + // Otherwise, return null + idToUiAddress(id) + .map(createProxyURI(prefix, _, path, request.getQueryString)) + .filter(uri => uri != null && validateDestination(uri.getHost, uri.getPort)) + .map(_.toString) + .orNull + } + + override def newHttpClient(): HttpClient = { + // SPARK-21176: Use the Jetty logic to calculate the number of selector threads (#CPUs/2), + // but limit it to 8 max. + val numSelectors = math.max(1, math.min(8, Runtime.getRuntime().availableProcessors() / 2)) + new HttpClient(new HttpClientTransportOverHTTP(numSelectors), null) + } + + override def filterServerResponseHeader( + clientRequest: HttpServletRequest, + serverResponse: Response, + headerName: String, + headerValue: String): String = { + if (headerName.equalsIgnoreCase("location")) { + val newHeader = createProxyLocationHeader(headerValue, clientRequest, + serverResponse.getRequest().getURI()) + if (newHeader != null) { + return newHeader + } + } + super.filterServerResponseHeader( + clientRequest, serverResponse, headerName, headerValue) + } + } + + val contextHandler = new ServletContextHandler + val holder = new ServletHolder(servlet) + contextHandler.setContextPath("/proxy") + contextHandler.addServlet(holder, "/*") + contextHandler + } + /** Add filters, if any, to the given list of ServletContextHandlers */ def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) @@ -224,47 +288,238 @@ private[spark] object JettyUtils extends Logging { def startJettyServer( hostName: String, port: Int, + sslOptions: SSLOptions, handlers: Seq[ServletContextHandler], conf: SparkConf, serverName: String = ""): ServerInfo = { addFilters(handlers, conf) - val collection = new ContextHandlerCollection - val gzipHandlers = handlers.map { h => - val gzipHandler = new GzipHandler - gzipHandler.setHandler(h) - gzipHandler + // Start the server first, with no connectors. + val pool = new QueuedThreadPool + if (serverName.nonEmpty) { + pool.setName(serverName) } - collection.setHandlers(gzipHandlers.toArray) - - // Bind to the given port, or throw a java.net.BindException if the port is occupied - def connect(currentPort: Int): (Server, Int) = { - val server = new Server(new InetSocketAddress(hostName, currentPort)) - val pool = new QueuedThreadPool - pool.setDaemon(true) - server.setThreadPool(pool) - val errorHandler = new ErrorHandler() - errorHandler.setShowStacks(true) - server.addBean(errorHandler) - server.setHandler(collection) - try { - server.start() - (server, server.getConnectors.head.getLocalPort) - } catch { - case e: Exception => - server.stop() + pool.setDaemon(true) + + val server = new Server(pool) + + val errorHandler = new ErrorHandler() + errorHandler.setShowStacks(true) + errorHandler.setServer(server) + server.addBean(errorHandler) + + val collection = new ContextHandlerCollection + server.setHandler(collection) + + // Executor used to create daemon threads for the Jetty connectors. + val serverExecutor = new ScheduledExecutorScheduler(s"$serverName-JettyScheduler", true) + + try { + server.start() + + // As each acceptor and each selector will use one thread, the number of threads should at + // least be the number of acceptors and selectors plus 1. (See SPARK-13776) + var minThreads = 1 + + def newConnector( + connectionFactories: Array[ConnectionFactory], + port: Int): (ServerConnector, Int) = { + val connector = new ServerConnector( + server, + null, + serverExecutor, + null, + -1, + -1, + connectionFactories: _*) + connector.setPort(port) + connector.start() + + // Currently we only use "SelectChannelConnector" + // Limit the max acceptor number to 8 so that we don't waste a lot of threads + connector.setAcceptQueueSize(math.min(connector.getAcceptors, 8)) + connector.setHost(hostName) + // The number of selectors always equals to the number of acceptors + minThreads += connector.getAcceptors * 2 + + (connector, connector.getLocalPort()) + } + + // If SSL is configured, create the secure connector first. + val securePort = sslOptions.createJettySslContextFactory().map { factory => + val securePort = sslOptions.port.getOrElse(if (port > 0) Utils.userPort(port, 400) else 0) + val secureServerName = if (serverName.nonEmpty) s"$serverName (HTTPS)" else serverName + val connectionFactories = AbstractConnectionFactory.getFactories(factory, + new HttpConnectionFactory()) + + def sslConnect(currentPort: Int): (ServerConnector, Int) = { + newConnector(connectionFactories, currentPort) + } + + val (connector, boundPort) = Utils.startServiceOnPort[ServerConnector](securePort, + sslConnect, conf, secureServerName) + connector.setName(SPARK_CONNECTOR_NAME) + server.addConnector(connector) + boundPort + } + + // Bind the HTTP port. + def httpConnect(currentPort: Int): (ServerConnector, Int) = { + newConnector(Array(new HttpConnectionFactory()), currentPort) + } + + val (httpConnector, httpPort) = Utils.startServiceOnPort[ServerConnector](port, httpConnect, + conf, serverName) + + // If SSL is configured, then configure redirection in the HTTP connector. + securePort match { + case Some(p) => + httpConnector.setName(REDIRECT_CONNECTOR_NAME) + val redirector = createRedirectHttpsHandler(p, "https") + collection.addHandler(redirector) + redirector.start() + + case None => + httpConnector.setName(SPARK_CONNECTOR_NAME) + } + + server.addConnector(httpConnector) + + // Add all the known handlers now that connectors are configured. + handlers.foreach { h => + h.setVirtualHosts(toVirtualHosts(SPARK_CONNECTOR_NAME)) + val gzipHandler = new GzipHandler() + gzipHandler.setHandler(h) + collection.addHandler(gzipHandler) + gzipHandler.start() + } + + pool.setMaxThreads(math.max(pool.getMaxThreads, minThreads)) + ServerInfo(server, httpPort, securePort, collection) + } catch { + case e: Exception => + server.stop() + if (serverExecutor.isStarted()) { + serverExecutor.stop() + } + if (pool.isStarted()) { pool.stop() - throw e + } + throw e + } + } + + private def createRedirectHttpsHandler(securePort: Int, scheme: String): ContextHandler = { + val redirectHandler: ContextHandler = new ContextHandler + redirectHandler.setContextPath("/") + redirectHandler.setVirtualHosts(toVirtualHosts(REDIRECT_CONNECTOR_NAME)) + redirectHandler.setHandler(new AbstractHandler { + override def handle( + target: String, + baseRequest: Request, + request: HttpServletRequest, + response: HttpServletResponse): Unit = { + if (baseRequest.isSecure) { + return + } + val httpsURI = createRedirectURI(scheme, baseRequest.getServerName, securePort, + baseRequest.getRequestURI, baseRequest.getQueryString) + response.setContentLength(0) + response.sendRedirect(response.encodeRedirectURL(httpsURI)) + baseRequest.setHandled(true) + } + }) + redirectHandler + } + + def createProxyURI(prefix: String, target: String, path: String, query: String): URI = { + if (!path.startsWith(prefix)) { + return null + } + + val uri = new StringBuilder(target) + val rest = path.substring(prefix.length()) + + if (!rest.isEmpty()) { + if (!rest.startsWith("/") && !uri.endsWith("/")) { + uri.append("/") } + uri.append(rest) + } + + val rewrittenURI = URI.create(uri.toString()) + if (query != null) { + return new URI( + rewrittenURI.getScheme(), + rewrittenURI.getAuthority(), + rewrittenURI.getPath(), + query, + rewrittenURI.getFragment() + ).normalize() } + rewrittenURI.normalize() + } + + def createProxyLocationHeader( + headerValue: String, + clientRequest: HttpServletRequest, + targetUri: URI): String = { + val toReplace = targetUri.getScheme() + "://" + targetUri.getAuthority() + if (headerValue.startsWith(toReplace)) { + val id = clientRequest.getPathInfo.substring("/proxy/".length).takeWhile(_ != '/') + val headerPath = headerValue.substring(toReplace.length) + + s"${clientRequest.getScheme}://${clientRequest.getHeader("host")}/proxy/$id$headerPath" + } else { + null + } + } - val (server, boundPort) = Utils.startServiceOnPort[Server](port, connect, conf, serverName) - ServerInfo(server, boundPort, collection) + // Create a new URI from the arguments, handling IPv6 host encoding and default ports. + private def createRedirectURI( + scheme: String, server: String, port: Int, path: String, query: String) = { + val redirectServer = if (server.contains(":") && !server.startsWith("[")) { + s"[${server}]" + } else { + server + } + val authority = s"$redirectServer:$port" + new URI(scheme, authority, path, query, null).toString } + + def toVirtualHosts(connectors: String*): Array[String] = connectors.map("@" + _).toArray + } private[spark] case class ServerInfo( server: Server, boundPort: Int, - rootHandler: ContextHandlerCollection) + securePort: Option[Int], + private val rootHandler: ContextHandlerCollection) { + + def addHandler(handler: ContextHandler): Unit = { + handler.setVirtualHosts(JettyUtils.toVirtualHosts(JettyUtils.SPARK_CONNECTOR_NAME)) + rootHandler.addHandler(handler) + if (!handler.isStarted()) { + handler.start() + } + } + + def removeHandler(handler: ContextHandler): Unit = { + rootHandler.removeHandler(handler) + if (handler.isStarted) { + handler.stop() + } + } + + def stop(): Unit = { + server.stop() + // Stop the ThreadPool if it supports stop() method (through LifeCycle). + // It is needed because stopping the Server won't stop the ThreadPool it uses. + val threadPool = server.getThreadPool + if (threadPool != null && threadPool.isInstanceOf[LifeCycle]) { + threadPool.asInstanceOf[LifeCycle].stop + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/PagedTable.scala b/core/src/main/scala/org/apache/spark/ui/PagedTable.scala index 6e2375477a68..65fa38387b9e 100644 --- a/core/src/main/scala/org/apache/spark/ui/PagedTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/PagedTable.scala @@ -17,8 +17,15 @@ package org.apache.spark.ui +import java.net.URLDecoder + +import scala.collection.JavaConverters._ import scala.xml.{Node, Unparsed} +import com.google.common.base.Splitter + +import org.apache.spark.util.Utils + /** * A data source that provides data for a page. * @@ -71,6 +78,12 @@ private[ui] trait PagedTable[T] { def tableCssClass: String + def pageSizeFormField: String + + def prevPageSizeFormField: String + + def pageNumberFormField: String + def dataSource: PagedDataSource[T] def headers: Seq[Node] @@ -81,21 +94,28 @@ private[ui] trait PagedTable[T] { val _dataSource = dataSource try { val PageData(totalPages, data) = _dataSource.pageData(page) + val pageNavi = pageNavigation(page, _dataSource.pageSize, totalPages)
    - {pageNavigation(page, _dataSource.pageSize, totalPages)} + {pageNavi} {headers} {data.map(row)}
    + {pageNavi}
    } catch { case e: IndexOutOfBoundsException => val PageData(totalPages, _) = _dataSource.pageData(1)
    {pageNavigation(1, _dataSource.pageSize, totalPages)} -
    {e.getMessage}
    +
    +

    Error while rendering table:

    +
    +              {Utils.exceptionString(e)}
    +            
    +
    } } @@ -151,36 +171,58 @@ private[ui] trait PagedTable[T] { // The current page should be disabled so that it cannot be clicked.
  • {p}
  • } else { -
  • {p}
  • +
  • {p}
  • + } + } + + val hiddenFormFields = { + if (goButtonFormPath.contains('?')) { + val queryString = goButtonFormPath.split("\\?", 2)(1) + val search = queryString.split("#")(0) + Splitter + .on('&') + .trimResults() + .omitEmptyStrings() + .withKeyValueSeparator("=") + .split(search) + .asScala + .filterKeys(_ != pageSizeFormField) + .filterKeys(_ != prevPageSizeFormField) + .filterKeys(_ != pageNumberFormField) + .mapValues(URLDecoder.decode(_, "UTF-8")) + .map { case (k, v) => + + } + } else { + Seq.empty } } - val (goButtonJsFuncName, goButtonJsFunc) = goButtonJavascriptFunction - // When clicking the "Go" button, it will call this javascript method and then call - // "goButtonJsFuncName" - val formJs = - s"""$$(function(){ - | $$( "#form-$tableId-page" ).submit(function(event) { - | var page = $$("#form-$tableId-page-no").val() - | var pageSize = $$("#form-$tableId-page-size").val() - | pageSize = pageSize ? pageSize: 100; - | if (page != "") { - | ${goButtonJsFuncName}(page, pageSize); - | } - | event.preventDefault(); - | }); - |}); - """.stripMargin
    + method="get" + action={Unparsed(goButtonFormPath)} + class="form-inline pull-right" + style="margin-bottom: 0px;"> + + {hiddenFormFields} - + + + id={s"form-$tableId-page-size"} + name={pageSizeFormField} + value={pageSize.toString} + class="span1" /> +
    @@ -189,7 +231,7 @@ private[ui] trait PagedTable[T] {
    -
    } } @@ -239,10 +276,7 @@ private[ui] trait PagedTable[T] { def pageLink(page: Int): String /** - * Only the implementation knows how to create the url with a page number and the page size, so we - * leave this one to the implementation. The implementation should create a JavaScript method that - * accepts a page number along with the page size and jumps to the page. The return value is this - * method name and its JavaScript codes. + * Returns the submission path for the "go to page #" form. */ - def goButtonJavascriptFunction: (String, String) + def goButtonFormPath: String } diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 99085ada9f0a..6e94073238a5 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,19 +17,23 @@ package org.apache.spark.ui -import java.util.Date +import java.util.{Date, ServiceLoader} +import scala.collection.JavaConverters._ + +import org.apache.spark.{SecurityManager, SparkConf, SparkContext} +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler._ import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationAttemptInfo, ApplicationInfo, UIRoot} -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} -import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.{EnvironmentListener, EnvironmentTab} import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} -import org.apache.spark.ui.jobs.{JobsTab, JobProgressListener, StagesTab} -import org.apache.spark.ui.storage.{StorageListener, StorageTab} +import org.apache.spark.ui.jobs.{JobProgressListener, JobsTab, StagesTab} import org.apache.spark.ui.scope.RDDOperationGraphListener +import org.apache.spark.ui.storage.{StorageListener, StorageTab} +import org.apache.spark.util.Utils /** * Top level user interface for a Spark application. @@ -46,21 +50,26 @@ private[spark] class SparkUI private ( val operationGraphListener: RDDOperationGraphListener, var appName: String, val basePath: String, + val lastUpdateTime: Option[Long] = None, val startTime: Long) - extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath, "SparkUI") + extends WebUI(securityManager, securityManager.getSSLOptions("ui"), SparkUI.getUIPort(conf), + conf, basePath, "SparkUI") with Logging with UIRoot { val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + var appId: String = _ - val stagesTab = new StagesTab(this) + var appSparkVersion = org.apache.spark.SPARK_VERSION - var appId: String = _ + private var streamingJobProgressListener: Option[SparkListener] = None /** Initialize all components of the server. */ def initialize() { - attachTab(new JobsTab(this)) + val jobsTab = new JobsTab(this) + attachTab(jobsTab) + val stagesTab = new StagesTab(this) attachTab(stagesTab) attachTab(new StorageTab(this)) attachTab(new EnvironmentTab(this)) @@ -68,13 +77,21 @@ private[spark] class SparkUI private ( attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) attachHandler(createRedirectHandler("/", "/jobs/", basePath = basePath)) attachHandler(ApiRootResource.getServletHandler(this)) - // This should be POST only, but, the YARN AM proxy won't proxy POSTs + // These should be POST only, but, the YARN AM proxy won't proxy POSTs + attachHandler(createRedirectHandler( + "/jobs/job/kill", "/jobs/", jobsTab.handleKillRequest, httpMethods = Set("GET", "POST"))) attachHandler(createRedirectHandler( "/stages/stage/kill", "/stages/", stagesTab.handleKillRequest, httpMethods = Set("GET", "POST"))) } initialize() + def getSparkUser: String = { + environmentListener.sparkUser + .orElse(environmentListener.systemProperties.toMap.get("user.name")) + .getOrElse("") + } + def getAppName: String = appName def setAppId(id: String): Unit = { @@ -84,16 +101,9 @@ private[spark] class SparkUI private ( /** Stop the server behind this web interface. Only valid after bind(). */ override def stop() { super.stop() - logInfo("Stopped Spark web UI at %s".format(appUIAddress)) + logInfo(s"Stopped Spark web UI at $webUrl") } - /** - * Return the application UI host:port. This does not include the scheme (http://). - */ - private[spark] def appUIHostPort = publicHostName + ":" + boundPort - - private[spark] def appUIAddress = s"http://$appUIHostPort" - def getSparkUI(appId: String): Option[SparkUI] = { if (appId == this.appId) Some(this) else None } @@ -102,22 +112,40 @@ private[spark] class SparkUI private ( Iterator(new ApplicationInfo( id = appId, name = appName, + coresGranted = None, + maxCores = None, + coresPerExecutor = None, + memoryPerExecutorMB = None, attempts = Seq(new ApplicationAttemptInfo( attemptId = None, startTime = new Date(startTime), endTime = new Date(-1), - sparkUser = "", - completed = false + duration = 0, + lastUpdated = new Date(startTime), + sparkUser = getSparkUser, + completed = false, + appSparkVersion = appSparkVersion )) )) } + + def getApplicationInfo(appId: String): Option[ApplicationInfo] = { + getApplicationInfoList.find(_.id == appId) + } + + def getStreamingJobProgressListener: Option[SparkListener] = streamingJobProgressListener + + def setStreamingJobProgressListener(sparkListener: SparkListener): Unit = { + streamingJobProgressListener = Option(sparkListener) + } } private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String) extends WebUITab(parent, prefix) { - def appName: String = parent.getAppName + def appName: String = parent.appName + def appSparkVersion: String = parent.appSparkVersion } private[spark] object SparkUI { @@ -134,13 +162,14 @@ private[spark] object SparkUI { def createLiveUI( sc: SparkContext, conf: SparkConf, - listenerBus: SparkListenerBus, jobProgressListener: JobProgressListener, securityManager: SecurityManager, appName: String, startTime: Long): SparkUI = { - create(Some(sc), conf, listenerBus, securityManager, appName, - jobProgressListener = Some(jobProgressListener), startTime = startTime) + create(Some(sc), conf, + sc.listenerBus.addToStatusQueue, + securityManager, appName, jobProgressListener = Some(jobProgressListener), + startTime = startTime) } def createHistoryUI( @@ -149,8 +178,18 @@ private[spark] object SparkUI { securityManager: SecurityManager, appName: String, basePath: String, + lastUpdateTime: Option[Long], startTime: Long): SparkUI = { - create(None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime) + val sparkUI = create(None, conf, listenerBus.addListener, securityManager, appName, basePath, + lastUpdateTime = lastUpdateTime, startTime = startTime) + + val listenerFactories = ServiceLoader.load(classOf[SparkHistoryListenerFactory], + Utils.getContextOrSparkClassLoader).asScala + listenerFactories.foreach { listenerFactory => + val listeners = listenerFactory.createListeners(conf, sparkUI) + listeners.foreach(listenerBus.addListener) + } + sparkUI } /** @@ -163,33 +202,34 @@ private[spark] object SparkUI { private def create( sc: Option[SparkContext], conf: SparkConf, - listenerBus: SparkListenerBus, + addListenerFn: SparkListenerInterface => Unit, securityManager: SecurityManager, appName: String, basePath: String = "", jobProgressListener: Option[JobProgressListener] = None, + lastUpdateTime: Option[Long] = None, startTime: Long): SparkUI = { val _jobProgressListener: JobProgressListener = jobProgressListener.getOrElse { val listener = new JobProgressListener(conf) - listenerBus.addListener(listener) + addListenerFn(listener) listener } val environmentListener = new EnvironmentListener - val storageStatusListener = new StorageStatusListener - val executorsListener = new ExecutorsListener(storageStatusListener) + val storageStatusListener = new StorageStatusListener(conf) + val executorsListener = new ExecutorsListener(storageStatusListener, conf) val storageListener = new StorageListener(storageStatusListener) val operationGraphListener = new RDDOperationGraphListener(conf) - listenerBus.addListener(environmentListener) - listenerBus.addListener(storageStatusListener) - listenerBus.addListener(executorsListener) - listenerBus.addListener(storageListener) - listenerBus.addListener(operationGraphListener) + addListenerFn(environmentListener) + addListenerFn(storageStatusListener) + addListenerFn(executorsListener) + addListenerFn(storageListener) + addListenerFn(operationGraphListener) new SparkUI(sc, conf, securityManager, environmentListener, storageStatusListener, executorsListener, _jobProgressListener, storageListener, operationGraphListener, - appName, basePath, startTime) + appName, basePath, lastUpdateTime, startTime) } } 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 cb122eaed83d..766cc65084f0 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -87,4 +87,16 @@ private[spark] object ToolTips { multiple operations (e.g. two map() functions) if they can be pipelined. Some operations also create multiple RDDs internally. Cached RDDs are shown in green. """ + + val TASK_TIME = + "Shaded red when garbage collection (GC) time is over 10% of task time" + + val BLACKLISTED = + "Shows if this executor has been blacklisted by the scheduler due to task failures." + + val APPLICATION_EXECUTOR_LIMIT = + """Maximum number of executors that this application will use. This limit is finite only when + dynamic allocation is enabled. The number of granted executors may exceed the limit + ephemerally when executors are being killed. + """ } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 25dcb604d9e5..ba798df13c95 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -17,14 +17,17 @@ package org.apache.spark.ui +import java.net.URLDecoder import java.text.SimpleDateFormat -import java.util.{Date, Locale} +import java.util.{Date, Locale, TimeZone} import scala.util.control.NonFatal import scala.xml._ import scala.xml.transform.{RewriteRule, RuleTransformer} -import org.apache.spark.Logging +import org.apache.commons.lang3.StringEscapeUtils + +import org.apache.spark.internal.Logging import org.apache.spark.ui.scope.RDDOperationGraph /** Utility functions for generating XML pages with spark content. */ @@ -33,9 +36,12 @@ private[spark] object UIUtils extends Logging { val TABLE_CLASS_STRIPED = TABLE_CLASS_NOT_STRIPED + " table-striped" val TABLE_CLASS_STRIPED_SORTABLE = TABLE_CLASS_STRIPED + " sortable" + private val NEWLINE_AND_SINGLE_QUOTE_REGEX = raw"(?i)(\r\n|\n|\r|%0D%0A|%0A|%0D|'|%27)".r + // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. private val dateFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + override def initialValue(): SimpleDateFormat = + new SimpleDateFormat("yyyy/MM/dd HH:mm:ss", Locale.US) } def formatDate(date: Date): String = dateFormat.get.format(date) @@ -167,6 +173,9 @@ private[spark] object UIUtils extends Logging { + + + } def vizHeaderNodes: Seq[Node] = { @@ -177,6 +186,20 @@ private[spark] object UIUtils extends Logging { } + def dataTablesHeaderNodes: Seq[Node] = { + + + + + + + + + + } + /** Returns a spark page with correctly formatted headers */ def headerSparkPage( title: String, @@ -184,7 +207,8 @@ private[spark] object UIUtils extends Logging { activeTab: SparkUITab, refreshInterval: Option[Int] = None, helpText: Option[String] = None, - showVisualization: Boolean = false): Seq[Node] = { + showVisualization: Boolean = false, + useDataTables: Boolean = false): Seq[Node] = { val appName = activeTab.appName val shortAppName = if (appName.length < 36) appName else appName.take(32) + "..." @@ -199,6 +223,7 @@ private[spark] object UIUtils extends Logging { {commonHeaderNodes} {if (showVisualization) vizHeaderNodes else Seq.empty} + {if (useDataTables) dataTablesHeaderNodes else Seq.empty} {appName} - {title} @@ -207,13 +232,13 @@ private[spark] object UIUtils extends Logging { - +
    @@ -232,10 +257,14 @@ private[spark] object UIUtils extends Logging { } /** Returns a page with the spark css/js and a simple format. Used for scheduler UI. */ - def basicSparkPage(content: => Seq[Node], title: String): Seq[Node] = { + def basicSparkPage( + content: => Seq[Node], + title: String, + useDataTables: Boolean = false): Seq[Node] = { {commonHeaderNodes} + {if (useDataTables) dataTablesHeaderNodes else Seq.empty} {title} @@ -317,15 +346,23 @@ private[spark] object UIUtils extends Logging { completed: Int, failed: Int, skipped: Int, + reasonToNumKilled: Map[String, Int], total: Int): Seq[Node] = { val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) - val startWidth = "width: %s%%".format((started.toDouble/total)*100) + // started + completed can be > total when there are speculative tasks + val boundedStarted = math.min(started, total - completed) + val startWidth = "width: %s%%".format((boundedStarted.toDouble/total)*100)
    {completed}/{total} + { if (failed == 0 && skipped == 0 && started > 0) s"($started running)" } { if (failed > 0) s"($failed failed)" } { if (skipped > 0) s"($skipped skipped)" } + { reasonToNumKilled.toSeq.sortBy(-_._2).map { + case (reason, count) => s"($count killed: $reason)" + } + }
    @@ -387,23 +424,24 @@ private[spark] object UIUtils extends Logging { } - /** Return a script element that automatically expands the DAG visualization on page load. */ - def expandDagVizOnLoad(forJob: Boolean): Seq[Node] = { - - } - /** * Returns HTML rendering of a job or stage description. It will try to parse the string as HTML * and make sure that it only contains anchors with root-relative links. Otherwise, * the whole string will rendered as a simple escaped text. * * Note: In terms of security, only anchor tags with root relative links are supported. So any - * attempts to embed links outside Spark UI, or other tags like ++ + ++ + + }
    -
    -
    - {execTable} -
    -
    ; - UIUtils.headerSparkPage("Executors (" + execInfo.size + ")", content, parent) + UIUtils.headerSparkPage("Executors", content, parent, useDataTables = true) } - - /** Render an HTML row representing an executor */ - private def execRow(info: ExecutorSummary, logsExist: Boolean): Seq[Node] = { - val maximumMemory = info.maxMemory - val memoryUsed = info.memoryUsed - val diskUsed = info.diskUsed - - {info.id} - {info.hostPort} - {info.rddBlocks} - - {Utils.bytesToString(memoryUsed)} / - {Utils.bytesToString(maximumMemory)} - - - {Utils.bytesToString(diskUsed)} - - {info.activeTasks} - {info.failedTasks} - {info.completedTasks} - {info.totalTasks} - - {Utils.msDurationToString(info.totalDuration)} - - - {Utils.bytesToString(info.totalInputBytes)} - - - {Utils.bytesToString(info.totalShuffleRead)} - - - {Utils.bytesToString(info.totalShuffleWrite)} - - { - if (logsExist) { - - { - info.executorLogs.map { case (logName, logUrl) => - - } - } - - } - } - { - if (threadDumpEnabled) { - val encodedId = URLEncoder.encode(info.id, "UTF-8") - - Thread Dump - - } else { - Seq.empty - } - } - - } - } private[spark] object ExecutorsPage { + private val ON_HEAP_MEMORY_TOOLTIP = "Memory used / total available memory for on heap " + + "storage of data like RDD partitions cached in memory." + private val OFF_HEAP_MEMORY_TOOLTIP = "Memory used / total available memory for off heap " + + "storage of data like RDD partitions cached in memory." + /** Represent an executor's info as a map given a storage status index */ - def getExecInfo(listener: ExecutorsListener, statusId: Int): ExecutorSummary = { - val status = listener.storageStatusList(statusId) + def getExecInfo( + listener: ExecutorsListener, + statusId: Int, + isActive: Boolean): ExecutorSummary = { + val status = if (isActive) { + listener.activeStorageStatusList(statusId) + } else { + listener.deadStorageStatusList(statusId) + } val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort val rddBlocks = status.numBlocks val memUsed = status.memUsed val maxMem = status.maxMem + val memoryMetrics = for { + onHeapUsed <- status.onHeapMemUsed + offHeapUsed <- status.offHeapMemUsed + maxOnHeap <- status.maxOnHeapMem + maxOffHeap <- status.maxOffHeapMem + } yield { + new MemoryMetrics(onHeapUsed, offHeapUsed, maxOnHeap, maxOffHeap) + } + + val diskUsed = status.diskUsed - val activeTasks = listener.executorToTasksActive.getOrElse(execId, 0) - val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0) - val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0) - val totalTasks = activeTasks + failedTasks + completedTasks - val totalDuration = listener.executorToDuration.getOrElse(execId, 0L) - val totalInputBytes = listener.executorToInputBytes.getOrElse(execId, 0L) - val totalShuffleRead = listener.executorToShuffleRead.getOrElse(execId, 0L) - val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0L) - val executorLogs = listener.executorToLogUrls.getOrElse(execId, Map.empty) + val taskSummary = listener.executorToTaskSummary.getOrElse(execId, ExecutorTaskSummary(execId)) new ExecutorSummary( execId, hostPort, + isActive, rddBlocks, memUsed, diskUsed, - activeTasks, - failedTasks, - completedTasks, - totalTasks, - totalDuration, - totalInputBytes, - totalShuffleRead, - totalShuffleWrite, + taskSummary.totalCores, + taskSummary.tasksMax, + taskSummary.tasksActive, + taskSummary.tasksFailed, + taskSummary.tasksComplete, + taskSummary.tasksActive + taskSummary.tasksFailed + taskSummary.tasksComplete, + taskSummary.duration, + taskSummary.jvmGCTime, + taskSummary.inputBytes, + taskSummary.shuffleRead, + taskSummary.shuffleWrite, + taskSummary.isBlacklisted, maxMem, - executorLogs + taskSummary.executorLogs, + memoryMetrics ) } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index a88fc4c37d3c..64a1a292a384 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -11,20 +11,19 @@ * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and + * See the License for the specific language governing permissions and * limitations under the License. */ package org.apache.spark.ui.exec -import scala.collection.mutable.HashMap +import scala.collection.mutable.{LinkedHashMap, ListBuffer} -import org.apache.spark.{Resubmitted, ExceptionFailure, SparkContext} +import org.apache.spark.{Resubmitted, SparkConf, SparkContext} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.storage.{StorageStatus, StorageStatusListener} import org.apache.spark.ui.{SparkUI, SparkUITab} -import org.apache.spark.ui.jobs.UIData.ExecutorUIData private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "executors") { val listener = parent.executorsListener @@ -38,101 +37,172 @@ private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "exec } } +private[ui] case class ExecutorTaskSummary( + var executorId: String, + var totalCores: Int = 0, + var tasksMax: Int = 0, + var tasksActive: Int = 0, + var tasksFailed: Int = 0, + var tasksComplete: Int = 0, + var duration: Long = 0L, + var jvmGCTime: Long = 0L, + var inputBytes: Long = 0L, + var inputRecords: Long = 0L, + var outputBytes: Long = 0L, + var outputRecords: Long = 0L, + var shuffleRead: Long = 0L, + var shuffleWrite: Long = 0L, + var executorLogs: Map[String, String] = Map.empty, + var isAlive: Boolean = true, + var isBlacklisted: Boolean = false +) + /** * :: DeveloperApi :: * A SparkListener that prepares information to be displayed on the ExecutorsTab */ @DeveloperApi -class ExecutorsListener(storageStatusListener: StorageStatusListener) extends SparkListener { - val executorToTasksActive = HashMap[String, Int]() - val executorToTasksComplete = HashMap[String, Int]() - val executorToTasksFailed = HashMap[String, Int]() - val executorToDuration = HashMap[String, Long]() - val executorToInputBytes = HashMap[String, Long]() - val executorToInputRecords = HashMap[String, Long]() - val executorToOutputBytes = HashMap[String, Long]() - val executorToOutputRecords = HashMap[String, Long]() - val executorToShuffleRead = HashMap[String, Long]() - val executorToShuffleWrite = HashMap[String, Long]() - val executorToLogUrls = HashMap[String, Map[String, String]]() - val executorIdToData = HashMap[String, ExecutorUIData]() - - def storageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList - - override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = synchronized { +@deprecated("This class will be removed in a future release.", "2.2.0") +class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: SparkConf) + extends SparkListener { + val executorToTaskSummary = LinkedHashMap[String, ExecutorTaskSummary]() + var executorEvents = new ListBuffer[SparkListenerEvent]() + + private val maxTimelineExecutors = conf.getInt("spark.ui.timeline.executors.maximum", 1000) + private val retainedDeadExecutors = conf.getInt("spark.ui.retainedDeadExecutors", 100) + + def activeStorageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList + + def deadStorageStatusList: Seq[StorageStatus] = storageStatusListener.deadStorageStatusList + + override def onExecutorAdded( + executorAdded: SparkListenerExecutorAdded): Unit = synchronized { val eid = executorAdded.executorId - executorToLogUrls(eid) = executorAdded.executorInfo.logUrlMap - executorIdToData(eid) = ExecutorUIData(executorAdded.time) + val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) + taskSummary.executorLogs = executorAdded.executorInfo.logUrlMap + taskSummary.totalCores = executorAdded.executorInfo.totalCores + taskSummary.tasksMax = taskSummary.totalCores / conf.getInt("spark.task.cpus", 1) + executorEvents += executorAdded + if (executorEvents.size > maxTimelineExecutors) { + executorEvents.remove(0) + } + + val deadExecutors = executorToTaskSummary.filter(e => !e._2.isAlive) + if (deadExecutors.size > retainedDeadExecutors) { + val head = deadExecutors.head + executorToTaskSummary.remove(head._1) + } } override def onExecutorRemoved( executorRemoved: SparkListenerExecutorRemoved): Unit = synchronized { - val eid = executorRemoved.executorId - val uiData = executorIdToData(eid) - uiData.finishTime = Some(executorRemoved.time) - uiData.finishReason = Some(executorRemoved.reason) + executorEvents += executorRemoved + if (executorEvents.size > maxTimelineExecutors) { + executorEvents.remove(0) + } + executorToTaskSummary.get(executorRemoved.executorId).foreach(e => e.isAlive = false) } - override def onApplicationStart(applicationStart: SparkListenerApplicationStart): Unit = { + override def onApplicationStart( + applicationStart: SparkListenerApplicationStart): Unit = { applicationStart.driverLogs.foreach { logs => - val storageStatus = storageStatusList.find { s => + val storageStatus = activeStorageStatusList.find { s => s.blockManagerId.executorId == SparkContext.LEGACY_DRIVER_IDENTIFIER || s.blockManagerId.executorId == SparkContext.DRIVER_IDENTIFIER } - storageStatus.foreach { s => executorToLogUrls(s.blockManagerId.executorId) = logs.toMap } + storageStatus.foreach { s => + val eid = s.blockManagerId.executorId + val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) + taskSummary.executorLogs = logs.toMap + } } } - override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { + override def onTaskStart( + taskStart: SparkListenerTaskStart): Unit = synchronized { val eid = taskStart.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) + taskSummary.tasksActive += 1 } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { + override def onTaskEnd( + taskEnd: SparkListenerTaskEnd): Unit = synchronized { val info = taskEnd.taskInfo if (info != null) { val eid = info.executorId - taskEnd.reason match { - case Resubmitted => - // Note: For resubmitted tasks, we continue to use the metrics that belong to the - // first attempt of this task. This may not be 100% accurate because the first attempt - // could have failed half-way through. The correct fix would be to keep track of the - // metrics added by each attempt, but this is much more complicated. - return - case e: ExceptionFailure => - executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - case _ => - executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 + val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) + // Note: For resubmitted tasks, we continue to use the metrics that belong to the + // first attempt of this task. This may not be 100% accurate because the first attempt + // could have failed half-way through. The correct fix would be to keep track of the + // metrics added by each attempt, but this is much more complicated. + if (taskEnd.reason == Resubmitted) { + return } - - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 - executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration + if (info.successful) { + taskSummary.tasksComplete += 1 + } else { + taskSummary.tasksFailed += 1 + } + if (taskSummary.tasksActive >= 1) { + taskSummary.tasksActive -= 1 + } + taskSummary.duration += info.duration // Update shuffle read/write val metrics = taskEnd.taskMetrics if (metrics != null) { - metrics.inputMetrics.foreach { inputMetrics => - executorToInputBytes(eid) = - executorToInputBytes.getOrElse(eid, 0L) + inputMetrics.bytesRead - executorToInputRecords(eid) = - executorToInputRecords.getOrElse(eid, 0L) + inputMetrics.recordsRead - } - metrics.outputMetrics.foreach { outputMetrics => - executorToOutputBytes(eid) = - executorToOutputBytes.getOrElse(eid, 0L) + outputMetrics.bytesWritten - executorToOutputRecords(eid) = - executorToOutputRecords.getOrElse(eid, 0L) + outputMetrics.recordsWritten - } - metrics.shuffleReadMetrics.foreach { shuffleRead => - executorToShuffleRead(eid) = - executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead - } - metrics.shuffleWriteMetrics.foreach { shuffleWrite => - executorToShuffleWrite(eid) = - executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten - } + taskSummary.inputBytes += metrics.inputMetrics.bytesRead + taskSummary.inputRecords += metrics.inputMetrics.recordsRead + taskSummary.outputBytes += metrics.outputMetrics.bytesWritten + taskSummary.outputRecords += metrics.outputMetrics.recordsWritten + + taskSummary.shuffleRead += metrics.shuffleReadMetrics.remoteBytesRead + taskSummary.shuffleWrite += metrics.shuffleWriteMetrics.bytesWritten + taskSummary.jvmGCTime += metrics.jvmGCTime } } } + private def updateExecutorBlacklist( + eid: String, + isBlacklisted: Boolean): Unit = { + val execTaskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) + execTaskSummary.isBlacklisted = isBlacklisted + } + + override def onExecutorBlacklisted( + executorBlacklisted: SparkListenerExecutorBlacklisted) + : Unit = synchronized { + updateExecutorBlacklist(executorBlacklisted.executorId, true) + } + + override def onExecutorUnblacklisted( + executorUnblacklisted: SparkListenerExecutorUnblacklisted) + : Unit = synchronized { + updateExecutorBlacklist(executorUnblacklisted.executorId, false) + } + + override def onNodeBlacklisted( + nodeBlacklisted: SparkListenerNodeBlacklisted) + : Unit = synchronized { + // Implicitly blacklist every executor associated with this node, and show this in the UI. + activeStorageStatusList.foreach { status => + if (status.blockManagerId.host == nodeBlacklisted.hostId) { + updateExecutorBlacklist(status.blockManagerId.executorId, true) + } + } + } + + override def onNodeUnblacklisted( + nodeUnblacklisted: SparkListenerNodeUnblacklisted) + : Unit = synchronized { + // Implicitly unblacklist every executor associated with this node, regardless of how + // they may have been blacklisted initially (either explicitly through executor blacklisting + // or implicitly through node blacklisting). Show this in the UI. + activeStorageStatusList.foreach { status => + if (status.blockManagerId.host == nodeUnblacklisted.hostId) { + updateExecutorBlacklist(status.blockManagerId.executorId, false) + } + } + } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index d467dd9e1f29..a7f2caafe04b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -17,15 +17,21 @@ package org.apache.spark.ui.jobs +import java.net.URLEncoder import java.util.Date import javax.servlet.http.HttpServletRequest +import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, ListBuffer} import scala.xml._ +import org.apache.commons.lang3.StringEscapeUtils + import org.apache.spark.JobExecutionStatus -import org.apache.spark.ui.jobs.UIData.{ExecutorUIData, JobUIData} -import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} +import org.apache.spark.scheduler._ +import org.apache.spark.ui._ +import org.apache.spark.ui.jobs.UIData.{JobUIData, StageUIData} +import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished jobs */ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { @@ -71,7 +77,12 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val jobId = jobUIData.jobId val status = jobUIData.status val (jobName, jobDescription) = getLastStageNameAndDescription(jobUIData) - val displayJobDescription = if (jobDescription.isEmpty) jobName else jobDescription + val displayJobDescription = + if (jobDescription.isEmpty) { + jobName + } else { + UIUtils.makeDescription(jobDescription, "", plainText = true).text + } val submissionTime = jobUIData.submissionTime.get val completionTimeOpt = jobUIData.completionTime val completionTime = completionTimeOpt.getOrElse(System.currentTimeMillis()) @@ -82,9 +93,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { case JobExecutionStatus.UNKNOWN => "unknown" } - // The timeline library treats contents as HTML, so we have to escape them; for the - // data-title attribute string we have to escape them twice since that's in a string. + // The timeline library treats contents as HTML, so we have to escape them. We need to add + // extra layers of escaping in order to embed this in a Javascript string literal. val escapedDesc = Utility.escape(displayJobDescription) + val jsEscapedDesc = StringEscapeUtils.escapeEcmaScript(escapedDesc) val jobEventJsonAsStr = s""" |{ @@ -94,7 +106,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { | 'end': new Date(${completionTime}), | 'content': '
    ' + | 'Status: ${status}
    ' + | 'Submitted: ${UIUtils.formatDate(new Date(submissionTime))}' + | '${ @@ -104,62 +116,62 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { "" } }">' + - | '${escapedDesc} (Job ${jobId})
    ' + | '${jsEscapedDesc} (Job ${jobId})
    ' |} """.stripMargin jobEventJsonAsStr } } - private def makeExecutorEvent(executorUIDatas: HashMap[String, ExecutorUIData]): Seq[String] = { + private def makeExecutorEvent(executorUIDatas: Seq[SparkListenerEvent]): + Seq[String] = { val events = ListBuffer[String]() executorUIDatas.foreach { - case (executorId, event) => + case a: SparkListenerExecutorAdded => val addedEvent = s""" |{ | 'className': 'executor added', | 'group': 'executors', - | 'start': new Date(${event.startTime}), + | 'start': new Date(${a.time}), | 'content': '
    Executor ${executorId} added
    ' + | 'data-title="Executor ${a.executorId}
    ' + + | 'Added at ${UIUtils.formatDate(new Date(a.time))}"' + + | 'data-html="true">Executor ${a.executorId} added' |} """.stripMargin events += addedEvent + case e: SparkListenerExecutorRemoved => + val removedEvent = + s""" + |{ + | 'className': 'executor removed', + | 'group': 'executors', + | 'start': new Date(${e.time}), + | 'content': '
    Reason: ${e.reason.replace("\n", " ")}""" + } else { + "" + } + }"' + + | 'data-html="true">Executor ${e.executorId} removed
    ' + |} + """.stripMargin + events += removedEvent - if (event.finishTime.isDefined) { - val removedEvent = - s""" - |{ - | 'className': 'executor removed', - | 'group': 'executors', - | 'start': new Date(${event.finishTime.get}), - | 'content': '
    Reason: ${event.finishReason.get}""" - } else { - "" - } - }"' + - | 'data-html="true">Executor ${executorId} removed
    ' - |} - """.stripMargin - events += removedEvent - } } events.toSeq } private def makeTimeline( jobs: Seq[JobUIData], - executors: HashMap[String, ExecutorUIData], + executors: Seq[SparkListenerEvent], startTime: Long): Seq[Node] = { val jobEventJsonAsStrSeq = makeJobEvent(jobs) @@ -198,67 +210,79 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { ++ } - private def jobsTable(jobs: Seq[JobUIData]): Seq[Node] = { + private def jobsTable( + request: HttpServletRequest, + tableHeaderId: String, + jobTag: String, + jobs: Seq[JobUIData], + killEnabled: Boolean): Seq[Node] = { + // stripXSS is called to remove suspicious characters used in XSS attacks + val allParameters = request.getParameterMap.asScala.toMap.mapValues(_.map(UIUtils.stripXSS)) + val parameterOtherTable = allParameters.filterNot(_._1.startsWith(jobTag)) + .map(para => para._1 + "=" + para._2(0)) + val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined) + val jobIdTitle = if (someJobHasJobGroup) "Job Id (Job Group)" else "Job Id" - val columns: Seq[Node] = { - {if (someJobHasJobGroup) "Job Id (Job Group)" else "Job Id"} - Description - Submitted - Duration - Stages: Succeeded/Total - Tasks (for all stages): Succeeded/Total - } + // stripXSS is called first to remove suspicious characters used in XSS attacks + val parameterJobPage = UIUtils.stripXSS(request.getParameter(jobTag + ".page")) + val parameterJobSortColumn = UIUtils.stripXSS(request.getParameter(jobTag + ".sort")) + val parameterJobSortDesc = UIUtils.stripXSS(request.getParameter(jobTag + ".desc")) + val parameterJobPageSize = UIUtils.stripXSS(request.getParameter(jobTag + ".pageSize")) + val parameterJobPrevPageSize = UIUtils.stripXSS(request.getParameter(jobTag + ".prevPageSize")) - def makeRow(job: JobUIData): Seq[Node] = { - val (lastStageName, lastStageDescription) = getLastStageNameAndDescription(job) - val duration: Option[Long] = { - job.submissionTime.map { start => - val end = job.completionTime.getOrElse(System.currentTimeMillis()) - end - start - } + val jobPage = Option(parameterJobPage).map(_.toInt).getOrElse(1) + val jobSortColumn = Option(parameterJobSortColumn).map { sortColumn => + UIUtils.decodeURLParameter(sortColumn) + }.getOrElse(jobIdTitle) + val jobSortDesc = Option(parameterJobSortDesc).map(_.toBoolean).getOrElse( + // New jobs should be shown above old jobs by default. + jobSortColumn == jobIdTitle + ) + val jobPageSize = Option(parameterJobPageSize).map(_.toInt).getOrElse(100) + val jobPrevPageSize = Option(parameterJobPrevPageSize).map(_.toInt).getOrElse(jobPageSize) + + val page: Int = { + // If the user has changed to a larger page size, then go to page 1 in order to avoid + // IndexOutOfBoundsException. + if (jobPageSize <= jobPrevPageSize) { + jobPage + } else { + 1 } - val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") - val formattedSubmissionTime = job.submissionTime.map(UIUtils.formatDate).getOrElse("Unknown") - val jobDescription = UIUtils.makeDescription(lastStageDescription, parent.basePath) - - val detailUrl = - "%s/jobs/job?id=%s".format(UIUtils.prependBaseUri(parent.basePath), job.jobId) - - - {job.jobId} {job.jobGroup.map(id => s"($id)").getOrElse("")} - - - {jobDescription} - {lastStageName} - - - {formattedSubmissionTime} - - {formattedDuration} - - {job.completedStageIndices.size}/{job.stageIds.size - job.numSkippedStages} - {if (job.numFailedStages > 0) s"(${job.numFailedStages} failed)"} - {if (job.numSkippedStages > 0) s"(${job.numSkippedStages} skipped)"} - - - {UIUtils.makeProgressBar(started = job.numActiveTasks, completed = job.numCompletedTasks, - failed = job.numFailedTasks, skipped = job.numSkippedTasks, - total = job.numTasks - job.numSkippedTasks)} - - } + val currentTime = System.currentTimeMillis() - - {columns} - - {jobs.map(makeRow)} - -
    + try { + new JobPagedTable( + jobs, + tableHeaderId, + jobTag, + UIUtils.prependBaseUri(parent.basePath), + "jobs", // subPath + parameterOtherTable, + parent.jobProgresslistener.stageIdToInfo, + parent.jobProgresslistener.stageIdToData, + killEnabled, + currentTime, + jobIdTitle, + pageSize = jobPageSize, + sortColumn = jobSortColumn, + desc = jobSortDesc + ).table(page) + } catch { + case e @ (_ : IllegalArgumentException | _ : IndexOutOfBoundsException) => +
    +

    Error while rendering job table:

    +
    +            {Utils.exceptionString(e)}
    +          
    +
    + } } def render(request: HttpServletRequest): Seq[Node] = { @@ -267,15 +291,15 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val startTime = listener.startTime val endTime = listener.endTime val activeJobs = listener.activeJobs.values.toSeq - val completedJobs = listener.completedJobs.reverse.toSeq - val failedJobs = listener.failedJobs.reverse.toSeq + val completedJobs = listener.completedJobs.reverse + val failedJobs = listener.failedJobs.reverse val activeJobsTable = - jobsTable(activeJobs.sortBy(_.submissionTime.getOrElse(-1L)).reverse) + jobsTable(request, "active", "activeJob", activeJobs, killEnabled = parent.killEnabled) val completedJobsTable = - jobsTable(completedJobs.sortBy(_.completionTime.getOrElse(-1L)).reverse) + jobsTable(request, "completed", "completedJob", completedJobs, killEnabled = false) val failedJobsTable = - jobsTable(failedJobs.sortBy(_.completionTime.getOrElse(-1L)).reverse) + jobsTable(request, "failed", "failedJob", failedJobs, killEnabled = false) val shouldShowActiveJobs = activeJobs.nonEmpty val shouldShowCompletedJobs = completedJobs.nonEmpty @@ -290,6 +314,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val summary: NodeSeq =
      +
    • + User: + {parent.getSparkUser} +
    • Total Uptime: { @@ -334,7 +362,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { var content = summary val executorListener = parent.executorListener content ++= makeTimeline(activeJobs ++ completedJobs ++ failedJobs, - executorListener.executorIdToData, startTime) + executorListener.executorEvents, startTime) if (shouldShowActiveJobs) { content ++=

      Active Jobs ({activeJobs.size})

      ++ @@ -356,3 +384,258 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } } } + +private[ui] class JobTableRowData( + val jobData: JobUIData, + val lastStageName: String, + val lastStageDescription: String, + val duration: Long, + val formattedDuration: String, + val submissionTime: Long, + val formattedSubmissionTime: String, + val jobDescription: NodeSeq, + val detailUrl: String) + +private[ui] class JobDataSource( + jobs: Seq[JobUIData], + stageIdToInfo: HashMap[Int, StageInfo], + stageIdToData: HashMap[(Int, Int), StageUIData], + basePath: String, + currentTime: Long, + pageSize: Int, + sortColumn: String, + desc: Boolean) extends PagedDataSource[JobTableRowData](pageSize) { + + // Convert JobUIData to JobTableRowData which contains the final contents to show in the table + // so that we can avoid creating duplicate contents during sorting the data + private val data = jobs.map(jobRow).sorted(ordering(sortColumn, desc)) + + private var _slicedJobIds: Set[Int] = null + + override def dataSize: Int = data.size + + override def sliceData(from: Int, to: Int): Seq[JobTableRowData] = { + val r = data.slice(from, to) + _slicedJobIds = r.map(_.jobData.jobId).toSet + r + } + + private def getLastStageNameAndDescription(job: JobUIData): (String, String) = { + val lastStageInfo = Option(job.stageIds) + .filter(_.nonEmpty) + .flatMap { ids => stageIdToInfo.get(ids.max)} + val lastStageData = lastStageInfo.flatMap { s => + stageIdToData.get((s.stageId, s.attemptId)) + } + val name = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") + val description = lastStageData.flatMap(_.description).getOrElse("") + (name, description) + } + + private def jobRow(jobData: JobUIData): JobTableRowData = { + val (lastStageName, lastStageDescription) = getLastStageNameAndDescription(jobData) + val duration: Option[Long] = { + jobData.submissionTime.map { start => + val end = jobData.completionTime.getOrElse(System.currentTimeMillis()) + end - start + } + } + val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") + val submissionTime = jobData.submissionTime + val formattedSubmissionTime = submissionTime.map(UIUtils.formatDate).getOrElse("Unknown") + val jobDescription = UIUtils.makeDescription(lastStageDescription, basePath, plainText = false) + + val detailUrl = "%s/jobs/job?id=%s".format(basePath, jobData.jobId) + + new JobTableRowData ( + jobData, + lastStageName, + lastStageDescription, + duration.getOrElse(-1), + formattedDuration, + submissionTime.getOrElse(-1), + formattedSubmissionTime, + jobDescription, + detailUrl + ) + } + + /** + * Return Ordering according to sortColumn and desc + */ + private def ordering(sortColumn: String, desc: Boolean): Ordering[JobTableRowData] = { + val ordering: Ordering[JobTableRowData] = sortColumn match { + case "Job Id" | "Job Id (Job Group)" => Ordering.by(_.jobData.jobId) + case "Description" => Ordering.by(x => (x.lastStageDescription, x.lastStageName)) + case "Submitted" => Ordering.by(_.submissionTime) + case "Duration" => Ordering.by(_.duration) + case "Stages: Succeeded/Total" | "Tasks (for all stages): Succeeded/Total" => + throw new IllegalArgumentException(s"Unsortable column: $sortColumn") + case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") + } + if (desc) { + ordering.reverse + } else { + ordering + } + } + +} +private[ui] class JobPagedTable( + data: Seq[JobUIData], + tableHeaderId: String, + jobTag: String, + basePath: String, + subPath: String, + parameterOtherTable: Iterable[String], + stageIdToInfo: HashMap[Int, StageInfo], + stageIdToData: HashMap[(Int, Int), StageUIData], + killEnabled: Boolean, + currentTime: Long, + jobIdTitle: String, + pageSize: Int, + sortColumn: String, + desc: Boolean + ) extends PagedTable[JobTableRowData] { + val parameterPath = basePath + s"/$subPath/?" + parameterOtherTable.mkString("&") + + override def tableId: String = jobTag + "-table" + + override def tableCssClass: String = + "table table-bordered table-condensed table-striped " + + "table-head-clickable table-cell-width-limited" + + override def pageSizeFormField: String = jobTag + ".pageSize" + + override def prevPageSizeFormField: String = jobTag + ".prevPageSize" + + override def pageNumberFormField: String = jobTag + ".page" + + override val dataSource = new JobDataSource( + data, + stageIdToInfo, + stageIdToData, + basePath, + currentTime, + pageSize, + sortColumn, + desc) + + override def pageLink(page: Int): String = { + val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") + parameterPath + + s"&$pageNumberFormField=$page" + + s"&$jobTag.sort=$encodedSortColumn" + + s"&$jobTag.desc=$desc" + + s"&$pageSizeFormField=$pageSize" + + s"#$tableHeaderId" + } + + override def goButtonFormPath: String = { + val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") + s"$parameterPath&$jobTag.sort=$encodedSortColumn&$jobTag.desc=$desc#$tableHeaderId" + } + + override def headers: Seq[Node] = { + // Information for each header: title, cssClass, and sortable + val jobHeadersAndCssClasses: Seq[(String, String, Boolean)] = + Seq( + (jobIdTitle, "", true), + ("Description", "", true), ("Submitted", "", true), ("Duration", "", true), + ("Stages: Succeeded/Total", "", false), + ("Tasks (for all stages): Succeeded/Total", "", false) + ) + + if (!jobHeadersAndCssClasses.filter(_._3).map(_._1).contains(sortColumn)) { + throw new IllegalArgumentException(s"Unknown column: $sortColumn") + } + + val headerRow: Seq[Node] = { + jobHeadersAndCssClasses.map { case (header, cssClass, sortable) => + if (header == sortColumn) { + val headerLink = Unparsed( + parameterPath + + s"&$jobTag.sort=${URLEncoder.encode(header, "UTF-8")}" + + s"&$jobTag.desc=${!desc}" + + s"&$jobTag.pageSize=$pageSize" + + s"#$tableHeaderId") + val arrow = if (desc) "▾" else "▴" // UP or DOWN + + + + {header} +  {Unparsed(arrow)} + + + + } else { + if (sortable) { + val headerLink = Unparsed( + parameterPath + + s"&$jobTag.sort=${URLEncoder.encode(header, "UTF-8")}" + + s"&$jobTag.pageSize=$pageSize" + + s"#$tableHeaderId") + + + + {header} + + + } else { + + {header} + + } + } + } + } + {headerRow} + } + + override def row(jobTableRow: JobTableRowData): Seq[Node] = { + val job = jobTableRow.jobData + + val killLink = if (killEnabled) { + val confirm = + s"if (window.confirm('Are you sure you want to kill job ${job.jobId} ?')) " + + "{ this.parentNode.submit(); return true; } else { return false; }" + // SPARK-6846 this should be POST-only but YARN AM won't proxy POST + /* + val killLinkUri = s"$basePathUri/jobs/job/kill/" +
      + + (kill) +
      + */ + val killLinkUri = s"$basePath/jobs/job/kill/?id=${job.jobId}" + (kill) + } else { + Seq.empty + } + + + + {job.jobId} {job.jobGroup.map(id => s"($id)").getOrElse("")} + + + {jobTableRow.jobDescription} {killLink} + {jobTableRow.lastStageName} + + + {jobTableRow.formattedSubmissionTime} + + {jobTableRow.formattedDuration} + + {job.completedStageIndices.size}/{job.stageIds.size - job.numSkippedStages} + {if (job.numFailedStages > 0) s"(${job.numFailedStages} failed)"} + {if (job.numSkippedStages > 0) s"(${job.numSkippedStages} skipped)"} + + + {UIUtils.makeProgressBar(started = job.numActiveTasks, + completed = job.completedIndices.size, + failed = job.numFailedTasks, skipped = job.numSkippedTasks, + reasonToNumKilled = job.reasonToNumKilled, total = job.numTasks - job.numSkippedTasks)} + + + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index 5e52942b64f3..a30c13592947 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -22,7 +22,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, NodeSeq} import org.apache.spark.scheduler.Schedulable -import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.ui.{UIUtils, WebUIPage} /** Page showing list of all ongoing and recently finished stages and pools */ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { @@ -34,26 +34,28 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { listener.synchronized { val activeStages = listener.activeStages.values.toSeq val pendingStages = listener.pendingStages.values.toSeq - val completedStages = listener.completedStages.reverse.toSeq + val completedStages = listener.completedStages.reverse val numCompletedStages = listener.numCompletedStages - val failedStages = listener.failedStages.reverse.toSeq + val failedStages = listener.failedStages.reverse val numFailedStages = listener.numFailedStages - val now = System.currentTimeMillis + val subPath = "stages" val activeStagesTable = - new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, - parent.basePath, parent.progressListener, isFairScheduler = parent.isFairScheduler, - killEnabled = parent.killEnabled) + new StageTableBase(request, activeStages, "active", "activeStage", parent.basePath, subPath, + parent.progressListener, parent.isFairScheduler, + killEnabled = parent.killEnabled, isFailedStage = false) val pendingStagesTable = - new StageTableBase(pendingStages.sortBy(_.submissionTime).reverse, - parent.basePath, parent.progressListener, isFairScheduler = parent.isFairScheduler, - killEnabled = false) + new StageTableBase(request, pendingStages, "pending", "pendingStage", parent.basePath, + subPath, parent.progressListener, parent.isFairScheduler, + killEnabled = false, isFailedStage = false) val completedStagesTable = - new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, - parent.progressListener, isFairScheduler = parent.isFairScheduler, killEnabled = false) + new StageTableBase(request, completedStages, "completed", "completedStage", parent.basePath, + subPath, parent.progressListener, parent.isFairScheduler, + killEnabled = false, isFailedStage = false) val failedStagesTable = - new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, - parent.progressListener, isFairScheduler = parent.isFairScheduler) + new StageTableBase(request, failedStages, "failed", "failedStage", parent.basePath, subPath, + parent.progressListener, parent.isFairScheduler, + killEnabled = false, isFailedStage = true) // For now, pool information is only accessible in live UIs val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]) @@ -113,7 +115,7 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { if (sc.isDefined && isFairScheduler) {

      {pools.size} Fair Scheduler Pools

      ++ poolTable.toNodeSeq } else { - Seq[Node]() + Seq.empty[Node] } } if (shouldShowActiveStages) { @@ -136,3 +138,4 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { } } } + diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index be144f6065ba..382a6f979f2e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -18,7 +18,7 @@ package org.apache.spark.ui.jobs import scala.collection.mutable -import scala.xml.Node +import scala.xml.{Node, Unparsed} import org.apache.spark.ui.{ToolTips, UIUtils} import org.apache.spark.ui.jobs.UIData.StageUIData @@ -42,21 +42,22 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage var hasShuffleWrite = false var hasShuffleRead = false var hasBytesSpilled = false - stageData.foreach(data => { + stageData.foreach { data => hasInput = data.hasInput hasOutput = data.hasOutput hasShuffleRead = data.hasShuffleRead hasShuffleWrite = data.hasShuffleWrite hasBytesSpilled = data.hasBytesSpilled - }) + } - + + {if (hasInput) { }} + {createExecutorTable()}
      Executor IDExecutor ID Address Task Time Total Tasks Failed TasksKilled Tasks Succeeded Tasks @@ -84,11 +85,25 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage Shuffle Spill (Memory) Shuffle Spill (Disk) + + Blacklisted + +
      + } private def createExecutorTable() : Seq[Node] = { @@ -104,11 +119,23 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage case Some(stageData: StageUIData) => stageData.executorSummary.toSeq.sortBy(_._1).map { case (k, v) => - {k} + +
      {k}
      +
      + { + val logs = parent.executorsListener.executorToTaskSummary.get(k) + .map(_.executorLogs).getOrElse(Map.empty) + logs.map { + case (logName, logUrl) => + } + } +
      + {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} {UIUtils.formatDuration(v.taskTime)} - {v.failedTasks + v.succeededTasks} + {v.failedTasks + v.succeededTasks + v.reasonToNumKilled.values.sum} {v.failedTasks} + {v.reasonToNumKilled.values.sum} {v.succeededTasks} {if (stageData.hasInput) { @@ -138,6 +165,7 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage {Utils.bytesToString(v.diskBytesSpilled)} }} + {v.isBlacklisted} } case None => diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 2cad0a796913..9fb011a049b7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -17,17 +17,17 @@ package org.apache.spark.ui.jobs -import java.util.Date +import java.util.{Date, Locale} +import javax.servlet.http.HttpServletRequest -import scala.collection.mutable.{Buffer, HashMap, ListBuffer} -import scala.xml.{NodeSeq, Node, Unparsed, Utility} +import scala.collection.mutable.{Buffer, ListBuffer} +import scala.xml.{Node, NodeSeq, Unparsed, Utility} -import javax.servlet.http.HttpServletRequest +import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.JobExecutionStatus -import org.apache.spark.scheduler.StageInfo +import org.apache.spark.scheduler._ import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} -import org.apache.spark.ui.jobs.UIData.ExecutorUIData /** Page showing statistics and stage list for a given job */ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { @@ -64,9 +64,10 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val submissionTime = stage.submissionTime.get val completionTime = stage.completionTime.getOrElse(System.currentTimeMillis()) - // The timeline library treats contents as HTML, so we have to escape them; for the - // data-title attribute string we have to escape them twice since that's in a string. + // The timeline library treats contents as HTML, so we have to escape them. We need to add + // extra layers of escaping in order to embed this in a Javascript string literal. val escapedName = Utility.escape(name) + val jsEscapedName = StringEscapeUtils.escapeEcmaScript(escapedName) s""" |{ | 'className': 'stage job-timeline-object ${status}', @@ -75,8 +76,8 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { | 'end': new Date(${completionTime}), | 'content': '
      ' + + | 'Status: ${status.toUpperCase(Locale.ROOT)}
      ' + | 'Submitted: ${UIUtils.formatDate(new Date(submissionTime))}' + | '${ if (status != "running") { @@ -85,61 +86,61 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { "" } }">' + - | '${escapedName} (Stage ${stageId}.${attemptId})
      ', + | '${jsEscapedName} (Stage ${stageId}.${attemptId})
    ', |} """.stripMargin } } - def makeExecutorEvent(executorUIDatas: HashMap[String, ExecutorUIData]): Seq[String] = { + def makeExecutorEvent(executorUIDatas: Seq[SparkListenerEvent]): Seq[String] = { val events = ListBuffer[String]() executorUIDatas.foreach { - case (executorId, event) => + case a: SparkListenerExecutorAdded => val addedEvent = s""" |{ | 'className': 'executor added', | 'group': 'executors', - | 'start': new Date(${event.startTime}), + | 'start': new Date(${a.time}), | 'content': '
    Executor ${executorId} added
    ' + | 'data-title="Executor ${a.executorId}
    ' + + | 'Added at ${UIUtils.formatDate(new Date(a.time))}"' + + | 'data-html="true">Executor ${a.executorId} added' |} """.stripMargin events += addedEvent - if (event.finishTime.isDefined) { - val removedEvent = - s""" - |{ - | 'className': 'executor removed', - | 'group': 'executors', - | 'start': new Date(${event.finishTime.get}), - | 'content': '
    Reason: ${event.finishReason.get}""" - } else { - "" - } - }"' + - | 'data-html="true">Executor ${executorId} removed
    ' - |} - """.stripMargin - events += removedEvent - } + case e: SparkListenerExecutorRemoved => + val removedEvent = + s""" + |{ + | 'className': 'executor removed', + | 'group': 'executors', + | 'start': new Date(${e.time}), + | 'content': '
    Reason: ${e.reason.replace("\n", " ")}""" + } else { + "" + } + }"' + + | 'data-html="true">Executor ${e.executorId} removed
    ' + |} + """.stripMargin + events += removedEvent + } events.toSeq } private def makeTimeline( stages: Seq[StageInfo], - executors: HashMap[String, ExecutorUIData], + executors: Seq[SparkListenerEvent], appStartTime: Long): Seq[Node] = { val stageEventJsonAsStrSeq = makeStageEvent(stages) @@ -177,7 +178,8 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { ++ } @@ -185,7 +187,8 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val listener = parent.jobProgresslistener listener.synchronized { - val parameterId = request.getParameter("id") + // stripXSS is called first to remove suspicious characters used in XSS attacks + val parameterId = UIUtils.stripXSS(request.getParameter("id")) require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") val jobId = parameterId.toInt @@ -226,20 +229,31 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { } } + val basePath = "jobs/job" + + val pendingOrSkippedTableId = + if (isComplete) { + "pending" + } else { + "skipped" + } + val activeStagesTable = - new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, - parent.basePath, parent.jobProgresslistener, isFairScheduler = parent.isFairScheduler, - killEnabled = parent.killEnabled) + new StageTableBase(request, activeStages, "active", "activeStage", parent.basePath, + basePath, parent.jobProgresslistener, parent.isFairScheduler, + killEnabled = parent.killEnabled, isFailedStage = false) val pendingOrSkippedStagesTable = - new StageTableBase(pendingOrSkippedStages.sortBy(_.stageId).reverse, - parent.basePath, parent.jobProgresslistener, isFairScheduler = parent.isFairScheduler, - killEnabled = false) + new StageTableBase(request, pendingOrSkippedStages, pendingOrSkippedTableId, "pendingStage", + parent.basePath, basePath, parent.jobProgresslistener, parent.isFairScheduler, + killEnabled = false, isFailedStage = false) val completedStagesTable = - new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, - parent.jobProgresslistener, isFairScheduler = parent.isFairScheduler, killEnabled = false) + new StageTableBase(request, completedStages, "completed", "completedStage", parent.basePath, + basePath, parent.jobProgresslistener, parent.isFairScheduler, + killEnabled = false, isFailedStage = false) val failedStagesTable = - new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, - parent.jobProgresslistener, isFairScheduler = parent.isFairScheduler) + new StageTableBase(request, failedStages, "failed", "failedStage", parent.basePath, + basePath, parent.jobProgresslistener, parent.isFairScheduler, + killEnabled = false, isFailedStage = true) val shouldShowActiveStages = activeStages.nonEmpty val shouldShowPendingStages = !isComplete && pendingOrSkippedStages.nonEmpty @@ -312,7 +326,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val operationGraphListener = parent.operationGraphListener content ++= makeTimeline(activeStages ++ completedStages ++ failedStages, - executorListener.executorIdToData, appStartTime) + executorListener.executorEvents, appStartTime) content ++= UIUtils.showDagVizForJob( jobId, operationGraphListener.getOperationGraphForJob(jobId)) 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 77d034fa5ba2..a18e86ec0a73 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 @@ -19,13 +19,13 @@ package org.apache.spark.ui.jobs import java.util.concurrent.TimeoutException -import scala.collection.mutable.{HashMap, HashSet, ListBuffer} - -import com.google.common.annotations.VisibleForTesting +import scala.collection.mutable.{HashMap, HashSet, LinkedHashMap, ListBuffer} import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId @@ -41,6 +41,7 @@ import org.apache.spark.ui.jobs.UIData._ * updating the internal data structures concurrently. */ @DeveloperApi +@deprecated("This class will be removed in a future release.", "2.2.0") class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // Define a handful of type aliases so that data structures' types can serve as documentation. @@ -94,6 +95,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val retainedStages = conf.getInt("spark.ui.retainedStages", SparkUI.DEFAULT_RETAINED_STAGES) val retainedJobs = conf.getInt("spark.ui.retainedJobs", SparkUI.DEFAULT_RETAINED_JOBS) + val retainedTasks = conf.get(UI_RETAINED_TASKS) // We can test for memory leaks by ensuring that collections that track non-active jobs and // stages do not grow without bound and that collections for active jobs/stages eventually become @@ -141,7 +143,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { /** If stages is too large, remove and garbage collect old stages */ private def trimStagesIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { if (stages.size > retainedStages) { - val toRemove = math.max(retainedStages / 10, 1) + val toRemove = calculateNumberToRemove(stages.size, retainedStages) stages.take(toRemove).foreach { s => stageIdToData.remove((s.stageId, s.attemptId)) stageIdToInfo.remove(s.stageId) @@ -153,7 +155,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { /** If jobs is too large, remove and garbage collect old jobs */ private def trimJobsIfNecessary(jobs: ListBuffer[JobUIData]) = synchronized { if (jobs.size > retainedJobs) { - val toRemove = math.max(retainedJobs / 10, 1) + val toRemove = calculateNumberToRemove(jobs.size, retainedJobs) jobs.take(toRemove).foreach { job => // Remove the job's UI data, if it exists jobIdToData.remove(job.jobId).foreach { removedJob => @@ -225,7 +227,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { trimJobsIfNecessary(completedJobs) jobData.status = JobExecutionStatus.SUCCEEDED numCompletedJobs += 1 - case JobFailed(exception) => + case JobFailed(_) => failedJobs += jobData trimJobsIfNecessary(failedJobs) jobData.status = JobExecutionStatus.FAILED @@ -283,7 +285,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { ) { jobData.numActiveStages -= 1 if (stage.failureReason.isEmpty) { - if (!stage.submissionTime.isEmpty) { + if (stage.submissionTime.isDefined) { jobData.completedStageIndices.add(stage.stageId) } } else { @@ -332,7 +334,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { new StageUIData }) stageData.numActiveTasks += 1 - stageData.taskData.put(taskInfo.taskId, new TaskUIData(taskInfo)) + stageData.taskData.put(taskInfo.taskId, TaskUIData(taskInfo)) } for ( activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskStart.stageId); @@ -369,36 +371,59 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { taskEnd.reason match { case Success => execSummary.succeededTasks += 1 + case kill: TaskKilled => + execSummary.reasonToNumKilled = execSummary.reasonToNumKilled.updated( + kill.reason, execSummary.reasonToNumKilled.getOrElse(kill.reason, 0) + 1) + case commitDenied: TaskCommitDenied => + execSummary.reasonToNumKilled = execSummary.reasonToNumKilled.updated( + commitDenied.toErrorString, execSummary.reasonToNumKilled.getOrElse( + commitDenied.toErrorString, 0) + 1) case _ => execSummary.failedTasks += 1 } execSummary.taskTime += info.duration stageData.numActiveTasks -= 1 - val (errorMessage, metrics): (Option[String], Option[TaskMetrics]) = + val errorMessage: Option[String] = taskEnd.reason match { case org.apache.spark.Success => stageData.completedIndices.add(info.index) stageData.numCompleteTasks += 1 - (None, Option(taskEnd.taskMetrics)) - case e: ExceptionFailure => // Handle ExceptionFailure because we might have metrics + None + case kill: TaskKilled => + stageData.reasonToNumKilled = stageData.reasonToNumKilled.updated( + kill.reason, stageData.reasonToNumKilled.getOrElse(kill.reason, 0) + 1) + Some(kill.toErrorString) + case commitDenied: TaskCommitDenied => + stageData.reasonToNumKilled = stageData.reasonToNumKilled.updated( + commitDenied.toErrorString, stageData.reasonToNumKilled.getOrElse( + commitDenied.toErrorString, 0) + 1) + Some(commitDenied.toErrorString) + case e: ExceptionFailure => // Handle ExceptionFailure because we might have accumUpdates stageData.numFailedTasks += 1 - (Some(e.toErrorString), e.metrics) - case e: TaskFailedReason => // All other failure cases + Some(e.toErrorString) + case e: TaskFailedReason => // All other failure cases stageData.numFailedTasks += 1 - (Some(e.toErrorString), None) + Some(e.toErrorString) } - if (!metrics.isEmpty) { - val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.taskMetrics) - updateAggregateMetrics(stageData, info.executorId, metrics.get, oldMetrics) + val taskMetrics = Option(taskEnd.taskMetrics) + taskMetrics.foreach { m => + val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.metrics) + updateAggregateMetrics(stageData, info.executorId, m, oldMetrics) } - val taskData = stageData.taskData.getOrElseUpdate(info.taskId, new TaskUIData(info)) - taskData.taskInfo = info - taskData.taskMetrics = metrics + val taskData = stageData.taskData.getOrElseUpdate(info.taskId, TaskUIData(info)) + taskData.updateTaskInfo(info) + taskData.updateTaskMetrics(taskMetrics) taskData.errorMessage = errorMessage + // If Tasks is too large, remove and garbage collect old tasks + if (stageData.taskData.size > retainedTasks) { + stageData.taskData = stageData.taskData.drop( + calculateNumberToRemove(stageData.taskData.size, retainedTasks)) + } + for ( activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskEnd.stageId); jobId <- activeJobsDependentOnStage; @@ -407,7 +432,15 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { jobData.numActiveTasks -= 1 taskEnd.reason match { case Success => + jobData.completedIndices.add((taskEnd.stageId, info.index)) jobData.numCompletedTasks += 1 + case kill: TaskKilled => + jobData.reasonToNumKilled = jobData.reasonToNumKilled.updated( + kill.reason, jobData.reasonToNumKilled.getOrElse(kill.reason, 0) + 1) + case commitDenied: TaskCommitDenied => + jobData.reasonToNumKilled = jobData.reasonToNumKilled.updated( + commitDenied.toErrorString, jobData.reasonToNumKilled.getOrElse( + commitDenied.toErrorString, 0) + 1) case _ => jobData.numFailedTasks += 1 } @@ -415,6 +448,13 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } } + /** + * Remove at least (maxRetained / 10) items to reduce friction. + */ + private def calculateNumberToRemove(dataSize: Int, retainedSize: Int): Int = { + math.max(retainedSize / 10, dataSize - retainedSize) + } + /** * Upon receiving new metrics for a task, updates the per-stage and per-executor-per-stage * aggregate metrics by calculating deltas between the currently recorded metrics and the new @@ -424,54 +464,54 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageData: StageUIData, execId: String, taskMetrics: TaskMetrics, - oldMetrics: Option[TaskMetrics]) { + oldMetrics: Option[TaskMetricsUIData]) { val execSummary = stageData.executorSummary.getOrElseUpdate(execId, new ExecutorSummary) val shuffleWriteDelta = - (taskMetrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L) - - oldMetrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten).getOrElse(0L)) + taskMetrics.shuffleWriteMetrics.bytesWritten - + oldMetrics.map(_.shuffleWriteMetrics.bytesWritten).getOrElse(0L) stageData.shuffleWriteBytes += shuffleWriteDelta execSummary.shuffleWrite += shuffleWriteDelta val shuffleWriteRecordsDelta = - (taskMetrics.shuffleWriteMetrics.map(_.shuffleRecordsWritten).getOrElse(0L) - - oldMetrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleRecordsWritten).getOrElse(0L)) + taskMetrics.shuffleWriteMetrics.recordsWritten - + oldMetrics.map(_.shuffleWriteMetrics.recordsWritten).getOrElse(0L) stageData.shuffleWriteRecords += shuffleWriteRecordsDelta execSummary.shuffleWriteRecords += shuffleWriteRecordsDelta val shuffleReadDelta = - (taskMetrics.shuffleReadMetrics.map(_.totalBytesRead).getOrElse(0L) - - oldMetrics.flatMap(_.shuffleReadMetrics).map(_.totalBytesRead).getOrElse(0L)) + taskMetrics.shuffleReadMetrics.totalBytesRead - + oldMetrics.map(_.shuffleReadMetrics.totalBytesRead).getOrElse(0L) stageData.shuffleReadTotalBytes += shuffleReadDelta execSummary.shuffleRead += shuffleReadDelta val shuffleReadRecordsDelta = - (taskMetrics.shuffleReadMetrics.map(_.recordsRead).getOrElse(0L) - - oldMetrics.flatMap(_.shuffleReadMetrics).map(_.recordsRead).getOrElse(0L)) + taskMetrics.shuffleReadMetrics.recordsRead - + oldMetrics.map(_.shuffleReadMetrics.recordsRead).getOrElse(0L) stageData.shuffleReadRecords += shuffleReadRecordsDelta execSummary.shuffleReadRecords += shuffleReadRecordsDelta val inputBytesDelta = - (taskMetrics.inputMetrics.map(_.bytesRead).getOrElse(0L) - - oldMetrics.flatMap(_.inputMetrics).map(_.bytesRead).getOrElse(0L)) + taskMetrics.inputMetrics.bytesRead - + oldMetrics.map(_.inputMetrics.bytesRead).getOrElse(0L) stageData.inputBytes += inputBytesDelta execSummary.inputBytes += inputBytesDelta val inputRecordsDelta = - (taskMetrics.inputMetrics.map(_.recordsRead).getOrElse(0L) - - oldMetrics.flatMap(_.inputMetrics).map(_.recordsRead).getOrElse(0L)) + taskMetrics.inputMetrics.recordsRead - + oldMetrics.map(_.inputMetrics.recordsRead).getOrElse(0L) stageData.inputRecords += inputRecordsDelta execSummary.inputRecords += inputRecordsDelta val outputBytesDelta = - (taskMetrics.outputMetrics.map(_.bytesWritten).getOrElse(0L) - - oldMetrics.flatMap(_.outputMetrics).map(_.bytesWritten).getOrElse(0L)) + taskMetrics.outputMetrics.bytesWritten - + oldMetrics.map(_.outputMetrics.bytesWritten).getOrElse(0L) stageData.outputBytes += outputBytesDelta execSummary.outputBytes += outputBytesDelta val outputRecordsDelta = - (taskMetrics.outputMetrics.map(_.recordsWritten).getOrElse(0L) - - oldMetrics.flatMap(_.outputMetrics).map(_.recordsWritten).getOrElse(0L)) + taskMetrics.outputMetrics.recordsWritten - + oldMetrics.map(_.outputMetrics.recordsWritten).getOrElse(0L) stageData.outputRecords += outputRecordsDelta execSummary.outputRecords += outputRecordsDelta @@ -488,22 +528,25 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val timeDelta = taskMetrics.executorRunTime - oldMetrics.map(_.executorRunTime).getOrElse(0L) stageData.executorRunTime += timeDelta + + val cpuTimeDelta = + taskMetrics.executorCpuTime - oldMetrics.map(_.executorCpuTime).getOrElse(0L) + stageData.executorCpuTime += cpuTimeDelta } override def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { - for ((taskId, sid, sAttempt, taskMetrics) <- executorMetricsUpdate.taskMetrics) { + for ((taskId, sid, sAttempt, accumUpdates) <- executorMetricsUpdate.accumUpdates) { val stageData = stageIdToData.getOrElseUpdate((sid, sAttempt), { logWarning("Metrics update for task in unknown stage " + sid) new StageUIData }) val taskData = stageData.taskData.get(taskId) - taskData.map { t => + val metrics = TaskMetrics.fromAccumulatorInfos(accumUpdates) + taskData.foreach { t => if (!t.taskInfo.finished) { - updateAggregateMetrics(stageData, executorMetricsUpdate.execId, taskMetrics, - t.taskMetrics) - + updateAggregateMetrics(stageData, executorMetricsUpdate.execId, metrics, t.metrics) // Overwrite task metrics - t.taskMetrics = Some(taskMetrics) + t.updateTaskMetrics(Some(metrics)) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala index 77ca60b000a9..cc173381879a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -17,8 +17,10 @@ package org.apache.spark.ui.jobs +import javax.servlet.http.HttpServletRequest + import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.ui.{SparkUI, SparkUITab} +import org.apache.spark.ui.{SparkUI, SparkUITab, UIUtils} /** Web UI showing progress status of all jobs in the given SparkContext. */ private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") { @@ -29,8 +31,26 @@ private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") { val operationGraphListener = parent.operationGraphListener def isFairScheduler: Boolean = - jobProgresslistener.schedulingMode.exists(_ == SchedulingMode.FAIR) + jobProgresslistener.schedulingMode == Some(SchedulingMode.FAIR) + + def getSparkUser: String = parent.getSparkUser attachPage(new AllJobsPage(this)) attachPage(new JobPage(this)) + + def handleKillRequest(request: HttpServletRequest): Unit = { + if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) { + // stripXSS is called first to remove suspicious characters used in XSS attacks + val jobId = Option(UIUtils.stripXSS(request.getParameter("id"))).map(_.toInt) + jobId.foreach { id => + if (jobProgresslistener.activeJobs.contains(id)) { + sc.foreach(_.cancelJob(id)) + // Do a quick pause here to give Spark time to kill the job so it shows up as + // killed after the refresh. Note that this will block the serving thread so the + // time should be limited in duration. + Thread.sleep(100) + } + } + } + } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index f3e0b38523f3..819fe57e14b2 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -22,7 +22,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.scheduler.StageInfo -import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.ui.{UIUtils, WebUIPage} /** Page showing specific pool details */ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { @@ -31,25 +31,34 @@ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { - val poolName = request.getParameter("poolname") - require(poolName != null && poolName.nonEmpty, "Missing poolname parameter") + // stripXSS is called first to remove suspicious characters used in XSS attacks + val poolName = Option(UIUtils.stripXSS(request.getParameter("poolname"))).map { poolname => + UIUtils.decodeURLParameter(poolname) + }.getOrElse { + throw new IllegalArgumentException(s"Missing poolname parameter") + } val poolToActiveStages = listener.poolToActiveStages val activeStages = poolToActiveStages.get(poolName) match { case Some(s) => s.values.toSeq - case None => Seq[StageInfo]() + case None => Seq.empty[StageInfo] } - val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, - parent.basePath, parent.progressListener, isFairScheduler = parent.isFairScheduler, - killEnabled = parent.killEnabled) + val shouldShowActiveStages = activeStages.nonEmpty + val activeStagesTable = + new StageTableBase(request, activeStages, "", "activeStage", parent.basePath, "stages/pool", + parent.progressListener, parent.isFairScheduler, parent.killEnabled, + isFailedStage = false) // For now, pool information is only accessible in live UIs - val pools = sc.map(_.getPoolForName(poolName).get).toSeq + val pools = sc.map(_.getPoolForName(poolName).getOrElse { + throw new IllegalArgumentException(s"Unknown poolname: $poolName") + }).toSeq val poolTable = new PoolTable(pools, parent) - val content = -

    Summary

    ++ poolTable.toNodeSeq ++ -

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq + var content =

    Summary

    ++ poolTable.toNodeSeq + if (shouldShowActiveStages) { + content ++=

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq + } UIUtils.headerSparkPage("Fair Scheduler Pool: " + poolName, content, parent) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index 9ba2af54dacf..ea02968733ca 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -17,6 +17,8 @@ package org.apache.spark.ui.jobs +import java.net.URLEncoder + import scala.collection.mutable.HashMap import scala.xml.Node @@ -59,7 +61,7 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: StagesTab) { case None => 0 } val href = "%s/stages/pool?poolname=%s" - .format(UIUtils.prependBaseUri(parent.basePath), p.name) + .format(UIUtils.prependBaseUri(parent.basePath), URLEncoder.encode(p.name, "UTF-8")) {p.name} 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 51425e599e74..4d80308eb0a6 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 @@ -26,12 +26,13 @@ import scala.xml.{Elem, Node, Unparsed} import org.apache.commons.lang3.StringEscapeUtils -import org.apache.spark.{InternalAccumulator, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} +import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo, TaskLocality} import org.apache.spark.ui._ +import org.apache.spark.ui.exec.ExecutorsListener import org.apache.spark.ui.jobs.UIData._ -import org.apache.spark.util.{Utils, Distribution} +import org.apache.spark.util.{Distribution, Utils} /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { @@ -39,6 +40,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { private val progressListener = parent.progressListener private val operationGraphListener = parent.operationGraphListener + private val executorsListener = parent.executorsListener private val TIMELINE_LEGEND = {
    @@ -68,29 +70,43 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { // if we find that it's okay. private val MAX_TIMELINE_TASKS = parent.conf.getInt("spark.ui.timeline.tasks.maximum", 1000) - private val displayPeakExecutionMemory = parent.conf.getBoolean("spark.sql.unsafe.enabled", true) + private def getLocalitySummaryString(stageData: StageUIData): String = { + val localities = stageData.taskData.values.map(_.taskInfo.taskLocality) + val localityCounts = localities.groupBy(identity).mapValues(_.size) + val localityNamesAndCounts = localityCounts.toSeq.map { case (locality, count) => + val localityName = locality match { + case TaskLocality.PROCESS_LOCAL => "Process local" + case TaskLocality.NODE_LOCAL => "Node local" + case TaskLocality.RACK_LOCAL => "Rack local" + case TaskLocality.ANY => "Any" + } + s"$localityName: $count" + } + localityNamesAndCounts.sorted.mkString("; ") + } def render(request: HttpServletRequest): Seq[Node] = { progressListener.synchronized { - val parameterId = request.getParameter("id") + // stripXSS is called first to remove suspicious characters used in XSS attacks + val parameterId = UIUtils.stripXSS(request.getParameter("id")) require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val parameterAttempt = request.getParameter("attempt") + val parameterAttempt = UIUtils.stripXSS(request.getParameter("attempt")) require(parameterAttempt != null && parameterAttempt.nonEmpty, "Missing attempt parameter") - val parameterTaskPage = request.getParameter("task.page") - val parameterTaskSortColumn = request.getParameter("task.sort") - val parameterTaskSortDesc = request.getParameter("task.desc") - val parameterTaskPageSize = request.getParameter("task.pageSize") + val parameterTaskPage = UIUtils.stripXSS(request.getParameter("task.page")) + val parameterTaskSortColumn = UIUtils.stripXSS(request.getParameter("task.sort")) + val parameterTaskSortDesc = UIUtils.stripXSS(request.getParameter("task.desc")) + val parameterTaskPageSize = UIUtils.stripXSS(request.getParameter("task.pageSize")) + val parameterTaskPrevPageSize = UIUtils.stripXSS(request.getParameter("task.prevPageSize")) val taskPage = Option(parameterTaskPage).map(_.toInt).getOrElse(1) - val taskSortColumn = Option(parameterTaskSortColumn).getOrElse("Index") + val taskSortColumn = Option(parameterTaskSortColumn).map { sortColumn => + UIUtils.decodeURLParameter(sortColumn) + }.getOrElse("Index") val taskSortDesc = Option(parameterTaskSortDesc).map(_.toBoolean).getOrElse(false) val taskPageSize = Option(parameterTaskPageSize).map(_.toInt).getOrElse(100) - - // If this is set, expand the dag visualization by default - val expandDagVizParam = request.getParameter("expandDagViz") - val expandDagViz = expandDagVizParam != null && expandDagVizParam.toBoolean + val taskPrevPageSize = Option(parameterTaskPrevPageSize).map(_.toInt).getOrElse(taskPageSize) val stageId = parameterId.toInt val stageAttemptId = parameterAttempt.toInt @@ -116,11 +132,18 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val stageData = stageDataOption.get val tasks = stageData.taskData.values.toSeq.sortBy(_.taskInfo.launchTime) - val numCompleted = tasks.count(_.taskInfo.finished) + val numCompleted = stageData.numCompleteTasks + val totalTasks = stageData.numActiveTasks + + stageData.numCompleteTasks + stageData.numFailedTasks + val totalTasksNumStr = if (totalTasks == tasks.size) { + s"$totalTasks" + } else { + s"$totalTasks, showing ${tasks.size}" + } val allAccumulables = progressListener.stageIdToData((stageId, stageAttemptId)).accumulables val externalAccumulables = allAccumulables.values.filter { acc => !acc.internal } - val hasAccumulators = externalAccumulables.size > 0 + val hasAccumulators = externalAccumulables.nonEmpty val summary =
    @@ -129,6 +152,10 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { Total Time Across All Tasks: {UIUtils.formatDuration(stageData.executorRunTime)} +
  • + Locality Level Summary: + {getLocalitySummaryString(stageData)} +
  • {if (stageData.hasInput) {
  • Input Size / Records: @@ -224,15 +251,13 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { Getting Result Time
  • - {if (displayPeakExecutionMemory) { -
  • - - - Peak Execution Memory - -
  • - }} +
  • + + + Peak Execution Memory + +
  • @@ -240,21 +265,27 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val dagViz = UIUtils.showDagVizForStage( stageId, operationGraphListener.getOperationGraphForStage(stageId)) - val maybeExpandDagViz: Seq[Node] = - if (expandDagViz) { - UIUtils.expandDagVizOnLoad(forJob = false) - } else { - Seq.empty - } - val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value") - def accumulableRow(acc: AccumulableInfo): Elem = - {acc.name}{acc.value} + def accumulableRow(acc: AccumulableInfo): Seq[Node] = { + (acc.name, acc.value) match { + case (Some(name), Some(value)) => {name}{value} + case _ => Seq.empty[Node] + } + } val accumulableTable = UIUtils.listingTable( accumulableHeaders, accumulableRow, externalAccumulables.toSeq) + val page: Int = { + // If the user has changed to a larger page size, then go to page 1 in order to avoid + // IndexOutOfBoundsException. + if (taskPageSize <= taskPrevPageSize) { + taskPage + } else { + 1 + } + } val currentTime = System.currentTimeMillis() val (taskTable, taskTableHTML) = try { val _taskTable = new TaskPagedTable( @@ -268,15 +299,24 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { stageData.hasShuffleRead, stageData.hasShuffleWrite, stageData.hasBytesSpilled, + parent.lastUpdateTime, currentTime, pageSize = taskPageSize, sortColumn = taskSortColumn, - desc = taskSortDesc + desc = taskSortDesc, + executorsListener = executorsListener ) - (_taskTable, _taskTable.table(taskPage)) + (_taskTable, _taskTable.table(page)) } catch { case e @ (_ : IllegalArgumentException | _ : IndexOutOfBoundsException) => - (null,
    {e.getMessage}
    ) + val errorMessage = +
    +

    Error while rendering stage table:

    +
    +                {Utils.exceptionString(e)}
    +              
    +
    + (null, errorMessage) } val jsForScrollingDownToTaskTable = @@ -298,10 +338,10 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { else taskTable.dataSource.slicedTaskIds // Excludes tasks which failed and have incomplete metrics - val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.taskMetrics.isDefined) + val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.metrics.isDefined) val summaryTable: Option[Seq[Node]] = - if (validTasks.size == 0) { + if (validTasks.isEmpty) { None } else { @@ -316,8 +356,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { getDistributionQuantiles(data).map(d => {Utils.bytesToString(d.toLong)}) } - val deserializationTimes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.executorDeserializeTime.toDouble + val deserializationTimes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.executorDeserializeTime.toDouble } val deserializationQuantiles = @@ -327,13 +367,13 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { +: getFormattedTimeQuantiles(deserializationTimes) - val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.executorRunTime.toDouble + val serviceTimes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.executorRunTime.toDouble } val serviceQuantiles = Duration +: getFormattedTimeQuantiles(serviceTimes) - val gcTimes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.jvmGCTime.toDouble + val gcTimes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.jvmGCTime.toDouble } val gcQuantiles = @@ -342,8 +382,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { +: getFormattedTimeQuantiles(gcTimes) - val serializationTimes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.resultSerializationTime.toDouble + val serializationTimes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.resultSerializationTime.toDouble } val serializationQuantiles = @@ -353,8 +393,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { +: getFormattedTimeQuantiles(serializationTimes) - val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => - getGettingResultTime(info, currentTime).toDouble + val gettingResultTimes = validTasks.map { taskUIData: TaskUIData => + getGettingResultTime(taskUIData.taskInfo, currentTime).toDouble } val gettingResultQuantiles = @@ -365,12 +405,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { +: getFormattedTimeQuantiles(gettingResultTimes) - val peakExecutionMemory = validTasks.map { case TaskUIData(info, _, _) => - info.accumulables - .find { acc => acc.name == InternalAccumulator.PEAK_EXECUTION_MEMORY } - .map { acc => acc.update.getOrElse("0").toLong } - .getOrElse(0L) - .toDouble + val peakExecutionMemory = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.peakExecutionMemory.toDouble } val peakExecutionMemoryQuantiles = { @@ -384,8 +420,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { // The scheduler delay includes the network delay to send the task to the worker // machine and to send back the result (but not the time to fetch the task result, // if it needed to be fetched from the block manager on the worker). - val schedulerDelays = validTasks.map { case TaskUIData(info, metrics, _) => - getSchedulerDelay(info, metrics.get, currentTime).toDouble + val schedulerDelays = validTasks.map { taskUIData: TaskUIData => + getSchedulerDelay(taskUIData.taskInfo, taskUIData.metrics.get, currentTime).toDouble } val schedulerDelayTitle = Scheduler Delay @@ -399,30 +435,30 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { ) } - val inputSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.inputMetrics.map(_.bytesRead).getOrElse(0L).toDouble + val inputSizes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.inputMetrics.bytesRead.toDouble } - val inputRecords = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.inputMetrics.map(_.recordsRead).getOrElse(0L).toDouble + val inputRecords = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.inputMetrics.recordsRead.toDouble } val inputQuantiles = Input Size / Records +: getFormattedSizeQuantilesWithRecords(inputSizes, inputRecords) - val outputSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.outputMetrics.map(_.bytesWritten).getOrElse(0L).toDouble + val outputSizes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.outputMetrics.bytesWritten.toDouble } - val outputRecords = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.outputMetrics.map(_.recordsWritten).getOrElse(0L).toDouble + val outputRecords = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.outputMetrics.recordsWritten.toDouble } val outputQuantiles = Output Size / Records +: getFormattedSizeQuantilesWithRecords(outputSizes, outputRecords) - val shuffleReadBlockedTimes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleReadMetrics.map(_.fetchWaitTime).getOrElse(0L).toDouble + val shuffleReadBlockedTimes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.shuffleReadMetrics.fetchWaitTime.toDouble } val shuffleReadBlockedQuantiles = @@ -433,11 +469,11 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { +: getFormattedTimeQuantiles(shuffleReadBlockedTimes) - val shuffleReadTotalSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleReadMetrics.map(_.totalBytesRead).getOrElse(0L).toDouble + val shuffleReadTotalSizes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.shuffleReadMetrics.totalBytesRead.toDouble } - val shuffleReadTotalRecords = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleReadMetrics.map(_.recordsRead).getOrElse(0L).toDouble + val shuffleReadTotalRecords = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.shuffleReadMetrics.recordsRead.toDouble } val shuffleReadTotalQuantiles = @@ -448,8 +484,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { +: getFormattedSizeQuantilesWithRecords(shuffleReadTotalSizes, shuffleReadTotalRecords) - val shuffleReadRemoteSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble + val shuffleReadRemoteSizes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.shuffleReadMetrics.remoteBytesRead.toDouble } val shuffleReadRemoteQuantiles = @@ -460,25 +496,25 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { +: getFormattedSizeQuantiles(shuffleReadRemoteSizes) - val shuffleWriteSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble + val shuffleWriteSizes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.shuffleWriteMetrics.bytesWritten.toDouble } - val shuffleWriteRecords = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.shuffleWriteMetrics.map(_.shuffleRecordsWritten).getOrElse(0L).toDouble + val shuffleWriteRecords = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.shuffleWriteMetrics.recordsWritten.toDouble } val shuffleWriteQuantiles = Shuffle Write Size / Records +: getFormattedSizeQuantilesWithRecords(shuffleWriteSizes, shuffleWriteRecords) - val memoryBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.memoryBytesSpilled.toDouble + val memoryBytesSpilledSizes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.memoryBytesSpilled.toDouble } val memoryBytesSpilledQuantiles = Shuffle spill (memory) +: getFormattedSizeQuantiles(memoryBytesSpilledSizes) - val diskBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.diskBytesSpilled.toDouble + val diskBytesSpilledSizes = validTasks.map { taskUIData: TaskUIData => + taskUIData.metrics.get.diskBytesSpilled.toDouble } val diskBytesSpilledQuantiles = Shuffle spill (disk) +: getFormattedSizeQuantiles(diskBytesSpilledSizes) @@ -494,13 +530,9 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {serializationQuantiles} , {gettingResultQuantiles}, - if (displayPeakExecutionMemory) { - - {peakExecutionMemoryQuantiles} - - } else { - Nil - }, + + {peakExecutionMemoryQuantiles} + , if (stageData.hasInput) {inputQuantiles} else Nil, if (stageData.hasOutput) {outputQuantiles} else Nil, if (stageData.hasShuffleRead) { @@ -534,22 +566,34 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val executorTable = new ExecutorTable(stageId, stageAttemptId, parent) val maybeAccumulableTable: Seq[Node] = - if (hasAccumulators) {

    Accumulators

    ++ accumulableTable } else Seq() + if (hasAccumulators) {

    Accumulators

    ++ accumulableTable } else Seq.empty + + val aggMetrics = + +

    + + Aggregated Metrics by Executor +

    +
    +
    + {executorTable.toNodeSeq} +
    val content = summary ++ dagViz ++ - maybeExpandDagViz ++ showAdditionalMetrics ++ makeTimeline( // Only show the tasks in the table stageData.taskData.values.toSeq.filter(t => taskIdsInPage.contains(t.taskInfo.taskId)), currentTime) ++ -

    Summary Metrics for {numCompleted} Completed Tasks

    ++ +

    Summary Metrics for {numCompleted} Completed Tasks

    ++
    {summaryTable.getOrElse("No tasks have reported metrics yet.")}
    ++ -

    Aggregated Metrics by Executor

    ++ executorTable.toNodeSeq ++ + aggMetrics ++ maybeAccumulableTable ++ -

    Tasks

    ++ taskTableHTML ++ jsForScrollingDownToTaskTable +

    Tasks ({totalTasksNumStr})

    ++ + taskTableHTML ++ jsForScrollingDownToTaskTable UIUtils.headerSparkPage(stageHeader, content, parent, showVisualization = true) } } @@ -574,13 +618,12 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { def toProportion(time: Long) = time.toDouble / totalExecutionTime * 100 - val metricsOpt = taskUIData.taskMetrics + val metricsOpt = taskUIData.metrics val shuffleReadTime = - metricsOpt.flatMap(_.shuffleReadMetrics.map(_.fetchWaitTime)).getOrElse(0L) + metricsOpt.map(_.shuffleReadMetrics.fetchWaitTime).getOrElse(0L) val shuffleReadTimeProportion = toProportion(shuffleReadTime) val shuffleWriteTime = - (metricsOpt.flatMap(_.shuffleWriteMetrics - .map(_.shuffleWriteTime)).getOrElse(0L) / 1e6).toLong + (metricsOpt.map(_.shuffleWriteMetrics.writeTime).getOrElse(0L) / 1e6).toLong val shuffleWriteTimeProportion = toProportion(shuffleWriteTime) val serializationTime = metricsOpt.map(_.resultSerializationTime).getOrElse(0L) @@ -602,9 +645,9 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { } val executorComputingTime = executorRunTime - shuffleReadTime - shuffleWriteTime val executorComputingTimeProportion = - (100 - schedulerDelayProportion - shuffleReadTimeProportion - + math.max(100 - schedulerDelayProportion - shuffleReadTimeProportion - shuffleWriteTimeProportion - serializationTimeProportion - - deserializationTimeProportion - gettingResultTimeProportion) + deserializationTimeProportion - gettingResultTimeProportion, 0) val schedulerDelayProportionPos = 0 val deserializationTimeProportionPos = @@ -720,7 +763,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { ++ } @@ -741,11 +785,11 @@ private[ui] object StagePage { } private[ui] def getSchedulerDelay( - info: TaskInfo, metrics: TaskMetrics, currentTime: Long): Long = { + info: TaskInfo, metrics: TaskMetricsUIData, currentTime: Long): Long = { if (info.finished) { val totalExecutionTime = info.finishTime - info.launchTime - val executorOverhead = (metrics.executorDeserializeTime + - metrics.resultSerializationTime) + val executorOverhead = metrics.executorDeserializeTime + + metrics.resultSerializationTime math.max( 0, totalExecutionTime - metrics.executorRunTime - executorOverhead - @@ -792,7 +836,8 @@ private[ui] class TaskTableRowData( val speculative: Boolean, val status: String, val taskLocality: String, - val executorIdAndHost: String, + val executorId: String, + val host: String, val launchTime: Long, val duration: Long, val formatDuration: String, @@ -808,7 +853,8 @@ private[ui] class TaskTableRowData( val shuffleRead: Option[TaskTableRowShuffleReadData], val shuffleWrite: Option[TaskTableRowShuffleWriteData], val bytesSpilled: Option[TaskTableRowBytesSpilledData], - val error: String) + val error: String, + val logs: Map[String, String]) private[ui] class TaskDataSource( tasks: Seq[TaskUIData], @@ -818,17 +864,19 @@ private[ui] class TaskDataSource( hasShuffleRead: Boolean, hasShuffleWrite: Boolean, hasBytesSpilled: Boolean, + lastUpdateTime: Option[Long], currentTime: Long, pageSize: Int, sortColumn: String, - desc: Boolean) extends PagedDataSource[TaskTableRowData](pageSize) { + desc: Boolean, + executorsListener: ExecutorsListener) extends PagedDataSource[TaskTableRowData](pageSize) { import StagePage._ // Convert TaskUIData to TaskTableRowData which contains the final contents to show in the table // so that we can avoid creating duplicate contents during sorting the data private val data = tasks.map(taskRow).sorted(ordering(sortColumn, desc)) - private var _slicedTaskIds: Set[Long] = null + private var _slicedTaskIds: Set[Long] = _ override def dataSize: Int = data.size @@ -841,42 +889,42 @@ private[ui] class TaskDataSource( def slicedTaskIds: Set[Long] = _slicedTaskIds private def taskRow(taskData: TaskUIData): TaskTableRowData = { - val TaskUIData(info, metrics, errorMessage) = taskData - val duration = if (info.status == "RUNNING") info.timeRunning(currentTime) - else metrics.map(_.executorRunTime).getOrElse(1L) - val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration) - else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("") + val info = taskData.taskInfo + val metrics = taskData.metrics + val duration = taskData.taskDuration(lastUpdateTime).getOrElse(1L) + val formatDuration = + taskData.taskDuration(lastUpdateTime).map(d => UIUtils.formatDuration(d)).getOrElse("") val schedulerDelay = metrics.map(getSchedulerDelay(info, _, currentTime)).getOrElse(0L) val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val taskDeserializationTime = metrics.map(_.executorDeserializeTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) val gettingResultTime = getGettingResultTime(info, currentTime) - val (taskInternalAccumulables, taskExternalAccumulables) = - info.accumulables.partition(_.internal) - val externalAccumulableReadable = taskExternalAccumulables.map { acc => - StringEscapeUtils.escapeHtml4(s"${acc.name}: ${acc.update.get}") - } - val peakExecutionMemoryUsed = taskInternalAccumulables - .find { acc => acc.name == InternalAccumulator.PEAK_EXECUTION_MEMORY } - .map { acc => acc.update.getOrElse("0").toLong } - .getOrElse(0L) + val externalAccumulableReadable = info.accumulables + .filterNot(_.internal) + .flatMap { a => + (a.name, a.update) match { + case (Some(name), Some(update)) => Some(StringEscapeUtils.escapeHtml4(s"$name: $update")) + case _ => None + } + } + val peakExecutionMemoryUsed = metrics.map(_.peakExecutionMemory).getOrElse(0L) - val maybeInput = metrics.flatMap(_.inputMetrics) + val maybeInput = metrics.map(_.inputMetrics) val inputSortable = maybeInput.map(_.bytesRead).getOrElse(0L) val inputReadable = maybeInput - .map(m => s"${Utils.bytesToString(m.bytesRead)} (${m.readMethod.toString.toLowerCase()})") + .map(m => s"${Utils.bytesToString(m.bytesRead)}") .getOrElse("") val inputRecords = maybeInput.map(_.recordsRead.toString).getOrElse("") - val maybeOutput = metrics.flatMap(_.outputMetrics) + val maybeOutput = metrics.map(_.outputMetrics) val outputSortable = maybeOutput.map(_.bytesWritten).getOrElse(0L) val outputReadable = maybeOutput .map(m => s"${Utils.bytesToString(m.bytesWritten)}") .getOrElse("") val outputRecords = maybeOutput.map(_.recordsWritten.toString).getOrElse("") - val maybeShuffleRead = metrics.flatMap(_.shuffleReadMetrics) + val maybeShuffleRead = metrics.map(_.shuffleReadMetrics) val shuffleReadBlockedTimeSortable = maybeShuffleRead.map(_.fetchWaitTime).getOrElse(0L) val shuffleReadBlockedTimeReadable = maybeShuffleRead.map(ms => UIUtils.formatDuration(ms.fetchWaitTime)).getOrElse("") @@ -890,14 +938,14 @@ private[ui] class TaskDataSource( val shuffleReadRemoteSortable = remoteShuffleBytes.getOrElse(0L) val shuffleReadRemoteReadable = remoteShuffleBytes.map(Utils.bytesToString).getOrElse("") - val maybeShuffleWrite = metrics.flatMap(_.shuffleWriteMetrics) - val shuffleWriteSortable = maybeShuffleWrite.map(_.shuffleBytesWritten).getOrElse(0L) + val maybeShuffleWrite = metrics.map(_.shuffleWriteMetrics) + val shuffleWriteSortable = maybeShuffleWrite.map(_.bytesWritten).getOrElse(0L) val shuffleWriteReadable = maybeShuffleWrite - .map(m => s"${Utils.bytesToString(m.shuffleBytesWritten)}").getOrElse("") + .map(m => s"${Utils.bytesToString(m.bytesWritten)}").getOrElse("") val shuffleWriteRecords = maybeShuffleWrite - .map(_.shuffleRecordsWritten.toString).getOrElse("") + .map(_.recordsWritten.toString).getOrElse("") - val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) + val maybeWriteTime = metrics.map(_.shuffleWriteMetrics.writeTime) val writeTimeSortable = maybeWriteTime.getOrElse(0L) val writeTimeReadable = maybeWriteTime.map(t => t / (1000 * 1000)).map { ms => if (ms == 0) "" else UIUtils.formatDuration(ms) @@ -964,6 +1012,8 @@ private[ui] class TaskDataSource( None } + val logs = executorsListener.executorToTaskSummary.get(info.executorId) + .map(_.executorLogs).getOrElse(Map.empty) new TaskTableRowData( info.index, info.taskId, @@ -971,7 +1021,8 @@ private[ui] class TaskDataSource( info.speculative, info.status, info.taskLocality.toString, - s"${info.executorId} / ${info.host}", + info.executorId, + info.host, info.launchTime, duration, formatDuration, @@ -987,96 +1038,47 @@ private[ui] class TaskDataSource( shuffleRead, shuffleWrite, bytesSpilled, - errorMessage.getOrElse("")) + taskData.errorMessage.getOrElse(""), + logs) } /** * Return Ordering according to sortColumn and desc */ private def ordering(sortColumn: String, desc: Boolean): Ordering[TaskTableRowData] = { - val ordering = sortColumn match { - case "Index" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Int.compare(x.index, y.index) - } - case "ID" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.taskId, y.taskId) - } - case "Attempt" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Int.compare(x.attempt, y.attempt) - } - case "Status" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.String.compare(x.status, y.status) - } - case "Locality Level" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.String.compare(x.taskLocality, y.taskLocality) - } - case "Executor ID / Host" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.String.compare(x.executorIdAndHost, y.executorIdAndHost) - } - case "Launch Time" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.launchTime, y.launchTime) - } - case "Duration" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.duration, y.duration) - } - case "Scheduler Delay" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.schedulerDelay, y.schedulerDelay) - } - case "Task Deserialization Time" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.taskDeserializationTime, y.taskDeserializationTime) - } - case "GC Time" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.gcTime, y.gcTime) - } - case "Result Serialization Time" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.serializationTime, y.serializationTime) - } - case "Getting Result Time" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.gettingResultTime, y.gettingResultTime) - } - case "Peak Execution Memory" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.peakExecutionMemoryUsed, y.peakExecutionMemoryUsed) - } + val ordering: Ordering[TaskTableRowData] = sortColumn match { + case "Index" => Ordering.by(_.index) + case "ID" => Ordering.by(_.taskId) + case "Attempt" => Ordering.by(_.attempt) + case "Status" => Ordering.by(_.status) + case "Locality Level" => Ordering.by(_.taskLocality) + case "Executor ID" => Ordering.by(_.executorId) + case "Host" => Ordering.by(_.host) + case "Launch Time" => Ordering.by(_.launchTime) + case "Duration" => Ordering.by(_.duration) + case "Scheduler Delay" => Ordering.by(_.schedulerDelay) + case "Task Deserialization Time" => Ordering.by(_.taskDeserializationTime) + case "GC Time" => Ordering.by(_.gcTime) + case "Result Serialization Time" => Ordering.by(_.serializationTime) + case "Getting Result Time" => Ordering.by(_.gettingResultTime) + case "Peak Execution Memory" => Ordering.by(_.peakExecutionMemoryUsed) case "Accumulators" => if (hasAccumulators) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.String.compare(x.accumulators.get, y.accumulators.get) - } + Ordering.by(_.accumulators.get) } else { throw new IllegalArgumentException( "Cannot sort by Accumulators because of no accumulators") } case "Input Size / Records" => if (hasInput) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.input.get.inputSortable, y.input.get.inputSortable) - } + Ordering.by(_.input.get.inputSortable) } else { throw new IllegalArgumentException( "Cannot sort by Input Size / Records because of no inputs") } case "Output Size / Records" => if (hasOutput) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.output.get.outputSortable, y.output.get.outputSortable) - } + Ordering.by(_.output.get.outputSortable) } else { throw new IllegalArgumentException( "Cannot sort by Output Size / Records because of no outputs") @@ -1084,33 +1086,21 @@ private[ui] class TaskDataSource( // ShuffleRead case "Shuffle Read Blocked Time" => if (hasShuffleRead) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.shuffleRead.get.shuffleReadBlockedTimeSortable, - y.shuffleRead.get.shuffleReadBlockedTimeSortable) - } + Ordering.by(_.shuffleRead.get.shuffleReadBlockedTimeSortable) } else { throw new IllegalArgumentException( "Cannot sort by Shuffle Read Blocked Time because of no shuffle reads") } case "Shuffle Read Size / Records" => if (hasShuffleRead) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.shuffleRead.get.shuffleReadSortable, - y.shuffleRead.get.shuffleReadSortable) - } + Ordering.by(_.shuffleRead.get.shuffleReadSortable) } else { throw new IllegalArgumentException( "Cannot sort by Shuffle Read Size / Records because of no shuffle reads") } case "Shuffle Remote Reads" => if (hasShuffleRead) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.shuffleRead.get.shuffleReadRemoteSortable, - y.shuffleRead.get.shuffleReadRemoteSortable) - } + Ordering.by(_.shuffleRead.get.shuffleReadRemoteSortable) } else { throw new IllegalArgumentException( "Cannot sort by Shuffle Remote Reads because of no shuffle reads") @@ -1118,22 +1108,14 @@ private[ui] class TaskDataSource( // ShuffleWrite case "Write Time" => if (hasShuffleWrite) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.shuffleWrite.get.writeTimeSortable, - y.shuffleWrite.get.writeTimeSortable) - } + Ordering.by(_.shuffleWrite.get.writeTimeSortable) } else { throw new IllegalArgumentException( "Cannot sort by Write Time because of no shuffle writes") } case "Shuffle Write Size / Records" => if (hasShuffleWrite) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.shuffleWrite.get.shuffleWriteSortable, - y.shuffleWrite.get.shuffleWriteSortable) - } + Ordering.by(_.shuffleWrite.get.shuffleWriteSortable) } else { throw new IllegalArgumentException( "Cannot sort by Shuffle Write Size / Records because of no shuffle writes") @@ -1141,30 +1123,19 @@ private[ui] class TaskDataSource( // BytesSpilled case "Shuffle Spill (Memory)" => if (hasBytesSpilled) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.bytesSpilled.get.memoryBytesSpilledSortable, - y.bytesSpilled.get.memoryBytesSpilledSortable) - } + Ordering.by(_.bytesSpilled.get.memoryBytesSpilledSortable) } else { throw new IllegalArgumentException( "Cannot sort by Shuffle Spill (Memory) because of no spills") } case "Shuffle Spill (Disk)" => if (hasBytesSpilled) { - new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.Long.compare(x.bytesSpilled.get.diskBytesSpilledSortable, - y.bytesSpilled.get.diskBytesSpilledSortable) - } + Ordering.by(_.bytesSpilled.get.diskBytesSpilledSortable) } else { throw new IllegalArgumentException( "Cannot sort by Shuffle Spill (Disk) because of no spills") } - case "Errors" => new Ordering[TaskTableRowData] { - override def compare(x: TaskTableRowData, y: TaskTableRowData): Int = - Ordering.String.compare(x.error, y.error) - } + case "Errors" => Ordering.by(_.error) case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") } if (desc) { @@ -1186,17 +1157,23 @@ private[ui] class TaskPagedTable( hasShuffleRead: Boolean, hasShuffleWrite: Boolean, hasBytesSpilled: Boolean, + lastUpdateTime: Option[Long], currentTime: Long, pageSize: Int, sortColumn: String, - desc: Boolean) extends PagedTable[TaskTableRowData] { - - // We only track peak memory used for unsafe operators - private val displayPeakExecutionMemory = conf.getBoolean("spark.sql.unsafe.enabled", true) + desc: Boolean, + executorsListener: ExecutorsListener) extends PagedTable[TaskTableRowData] { override def tableId: String = "task-table" - override def tableCssClass: String = "table table-bordered table-condensed table-striped" + override def tableCssClass: String = + "table table-bordered table-condensed table-striped table-head-clickable" + + override def pageSizeFormField: String = "task.pageSize" + + override def prevPageSizeFormField: String = "task.prevPageSize" + + override def pageNumberFormField: String = "task.page" override val dataSource: TaskDataSource = new TaskDataSource( data, @@ -1206,50 +1183,38 @@ private[ui] class TaskPagedTable( hasShuffleRead, hasShuffleWrite, hasBytesSpilled, + lastUpdateTime, currentTime, pageSize, sortColumn, - desc) + desc, + executorsListener) override def pageLink(page: Int): String = { val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") - s"${basePath}&task.page=$page&task.sort=${encodedSortColumn}&task.desc=${desc}" + - s"&task.pageSize=${pageSize}" + basePath + + s"&$pageNumberFormField=$page" + + s"&task.sort=$encodedSortColumn" + + s"&task.desc=$desc" + + s"&$pageSizeFormField=$pageSize" } - override def goButtonJavascriptFunction: (String, String) = { - val jsFuncName = "goToTaskPage" + override def goButtonFormPath: String = { val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") - val jsFunc = s""" - |currentTaskPageSize = ${pageSize} - |function goToTaskPage(page, pageSize) { - | // Set page to 1 if the page size changes - | page = pageSize == currentTaskPageSize ? page : 1; - | var url = "${basePath}&task.sort=${encodedSortColumn}&task.desc=${desc}" + - | "&task.page=" + page + "&task.pageSize=" + pageSize; - | window.location.href = url; - |} - """.stripMargin - (jsFuncName, jsFunc) + s"$basePath&task.sort=$encodedSortColumn&task.desc=$desc" } def headers: Seq[Node] = { val taskHeadersAndCssClasses: Seq[(String, String)] = Seq( ("Index", ""), ("ID", ""), ("Attempt", ""), ("Status", ""), ("Locality Level", ""), - ("Executor ID / Host", ""), ("Launch Time", ""), ("Duration", ""), + ("Executor ID", ""), ("Host", ""), ("Launch Time", ""), ("Duration", ""), ("Scheduler Delay", TaskDetailsClassNames.SCHEDULER_DELAY), ("Task Deserialization Time", TaskDetailsClassNames.TASK_DESERIALIZATION_TIME), ("GC Time", ""), ("Result Serialization Time", TaskDetailsClassNames.RESULT_SERIALIZATION_TIME), - ("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME)) ++ - { - if (displayPeakExecutionMemory) { - Seq(("Peak Execution Memory", TaskDetailsClassNames.PEAK_EXECUTION_MEMORY)) - } else { - Nil - } - } ++ + ("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME), + ("Peak Execution Memory", TaskDetailsClassNames.PEAK_EXECUTION_MEMORY)) ++ {if (hasAccumulators) Seq(("Accumulators", "")) else Nil} ++ {if (hasInput) Seq(("Input Size / Records", "")) else Nil} ++ {if (hasOutput) Seq(("Output Size / Records", "")) else Nil} ++ @@ -1279,21 +1244,27 @@ private[ui] class TaskPagedTable( val headerRow: Seq[Node] = { taskHeadersAndCssClasses.map { case (header, cssClass) => if (header == sortColumn) { - val headerLink = - s"$basePath&task.sort=${URLEncoder.encode(header, "UTF-8")}&task.desc=${!desc}" + - s"&task.pageSize=${pageSize}" - val js = Unparsed(s"window.location.href='${headerLink}'") + val headerLink = Unparsed( + basePath + + s"&task.sort=${URLEncoder.encode(header, "UTF-8")}" + + s"&task.desc=${!desc}" + + s"&task.pageSize=$pageSize") val arrow = if (desc) "▾" else "▴" // UP or DOWN - - {header} -  {Unparsed(arrow)} + + + {header} +  {Unparsed(arrow)} + } else { - val headerLink = - s"$basePath&task.sort=${URLEncoder.encode(header, "UTF-8")}&task.pageSize=${pageSize}" - val js = Unparsed(s"window.location.href='${headerLink}'") - - {header} + val headerLink = Unparsed( + basePath + + s"&task.sort=${URLEncoder.encode(header, "UTF-8")}" + + s"&task.pageSize=$pageSize") + + + {header} + } } @@ -1308,7 +1279,17 @@ private[ui] class TaskPagedTable( {if (task.speculative) s"${task.attempt} (speculative)" else task.attempt.toString} {task.status} {task.taskLocality} - {task.executorIdAndHost} + {task.executorId} + +
    {task.host}
    +
    + { + task.logs.map { + case (logName, logUrl) => + } + } +
    + {UIUtils.formatDate(new Date(task.launchTime))} {task.formatDuration} @@ -1326,11 +1307,9 @@ private[ui] class TaskPagedTable( {UIUtils.formatDuration(task.gettingResultTime)} - {if (displayPeakExecutionMemory) { - - {Utils.bytesToString(task.peakExecutionMemoryUsed)} - - }} + + {Utils.bytesToString(task.peakExecutionMemoryUsed)} + {if (task.accumulators.nonEmpty) { {Unparsed(task.accumulators.get)} }} 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 ea806d09b600..f0a12a28de06 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 @@ -17,61 +17,334 @@ package org.apache.spark.ui.jobs +import java.net.URLEncoder import java.util.Date +import javax.servlet.http.HttpServletRequest -import scala.xml.{Node, Text} +import scala.collection.JavaConverters._ +import scala.xml._ import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.scheduler.StageInfo -import org.apache.spark.ui.{ToolTips, UIUtils} +import org.apache.spark.ui._ +import org.apache.spark.ui.jobs.UIData.StageUIData import org.apache.spark.util.Utils -/** Page showing list of all ongoing and recently finished stages */ private[ui] class StageTableBase( + request: HttpServletRequest, stages: Seq[StageInfo], + tableHeaderID: String, + stageTag: String, basePath: String, + subPath: String, + progressListener: JobProgressListener, + isFairScheduler: Boolean, + killEnabled: Boolean, + isFailedStage: Boolean) { + // stripXSS is called to remove suspicious characters used in XSS attacks + val allParameters = request.getParameterMap.asScala.toMap.mapValues(_.map(UIUtils.stripXSS)) + val parameterOtherTable = allParameters.filterNot(_._1.startsWith(stageTag)) + .map(para => para._1 + "=" + para._2(0)) + + val parameterStagePage = UIUtils.stripXSS(request.getParameter(stageTag + ".page")) + val parameterStageSortColumn = UIUtils.stripXSS(request.getParameter(stageTag + ".sort")) + val parameterStageSortDesc = UIUtils.stripXSS(request.getParameter(stageTag + ".desc")) + val parameterStagePageSize = UIUtils.stripXSS(request.getParameter(stageTag + ".pageSize")) + val parameterStagePrevPageSize = + UIUtils.stripXSS(request.getParameter(stageTag + ".prevPageSize")) + + val stagePage = Option(parameterStagePage).map(_.toInt).getOrElse(1) + val stageSortColumn = Option(parameterStageSortColumn).map { sortColumn => + UIUtils.decodeURLParameter(sortColumn) + }.getOrElse("Stage Id") + val stageSortDesc = Option(parameterStageSortDesc).map(_.toBoolean).getOrElse( + // New stages should be shown above old jobs by default. + stageSortColumn == "Stage Id" + ) + val stagePageSize = Option(parameterStagePageSize).map(_.toInt).getOrElse(100) + val stagePrevPageSize = Option(parameterStagePrevPageSize).map(_.toInt) + .getOrElse(stagePageSize) + + val page: Int = { + // If the user has changed to a larger page size, then go to page 1 in order to avoid + // IndexOutOfBoundsException. + if (stagePageSize <= stagePrevPageSize) { + stagePage + } else { + 1 + } + } + val currentTime = System.currentTimeMillis() + + val toNodeSeq = try { + new StagePagedTable( + stages, + tableHeaderID, + stageTag, + basePath, + subPath, + progressListener, + isFairScheduler, + killEnabled, + currentTime, + stagePageSize, + stageSortColumn, + stageSortDesc, + isFailedStage, + parameterOtherTable + ).table(page) + } catch { + case e @ (_ : IllegalArgumentException | _ : IndexOutOfBoundsException) => +
    +

    Error while rendering stage table:

    +
    +          {Utils.exceptionString(e)}
    +        
    +
    + } +} + +private[ui] class StageTableRowData( + val stageInfo: StageInfo, + val stageData: Option[StageUIData], + val stageId: Int, + val attemptId: Int, + val schedulingPool: String, + val descriptionOption: Option[String], + val submissionTime: Long, + val formattedSubmissionTime: String, + val duration: Long, + val formattedDuration: String, + val inputRead: Long, + val inputReadWithUnit: String, + val outputWrite: Long, + val outputWriteWithUnit: String, + val shuffleRead: Long, + val shuffleReadWithUnit: String, + val shuffleWrite: Long, + val shuffleWriteWithUnit: String) + +private[ui] class MissingStageTableRowData( + stageInfo: StageInfo, + stageId: Int, + attemptId: Int) extends StageTableRowData( + stageInfo, None, stageId, attemptId, "", None, 0, "", -1, "", 0, "", 0, "", 0, "", 0, "") + +/** Page showing list of all ongoing and recently finished stages */ +private[ui] class StagePagedTable( + stages: Seq[StageInfo], + tableHeaderId: String, + stageTag: String, + basePath: String, + subPath: String, listener: JobProgressListener, isFairScheduler: Boolean, - killEnabled: Boolean) { - - protected def columns: Seq[Node] = { - Stage Id ++ - {if (isFairScheduler) {Pool Name} else Seq.empty} ++ - Description - Submitted - Duration - Tasks: Succeeded/Total - Input - Output - Shuffle Read - - - - Shuffle Write - - + killEnabled: Boolean, + currentTime: Long, + pageSize: Int, + sortColumn: String, + desc: Boolean, + isFailedStage: Boolean, + parameterOtherTable: Iterable[String]) extends PagedTable[StageTableRowData] { + + override def tableId: String = stageTag + "-table" + + override def tableCssClass: String = + "table table-bordered table-condensed table-striped " + + "table-head-clickable table-cell-width-limited" + + override def pageSizeFormField: String = stageTag + ".pageSize" + + override def prevPageSizeFormField: String = stageTag + ".prevPageSize" + + override def pageNumberFormField: String = stageTag + ".page" + + val parameterPath = UIUtils.prependBaseUri(basePath) + s"/$subPath/?" + + parameterOtherTable.mkString("&") + + override val dataSource = new StageDataSource( + stages, + listener, + currentTime, + pageSize, + sortColumn, + desc + ) + + override def pageLink(page: Int): String = { + val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") + parameterPath + + s"&$pageNumberFormField=$page" + + s"&$stageTag.sort=$encodedSortColumn" + + s"&$stageTag.desc=$desc" + + s"&$pageSizeFormField=$pageSize" + + s"#$tableHeaderId" } - def toNodeSeq: Seq[Node] = { - listener.synchronized { - stageTable(renderStageRow, stages) + override def goButtonFormPath: String = { + val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") + s"$parameterPath&$stageTag.sort=$encodedSortColumn&$stageTag.desc=$desc#$tableHeaderId" + } + + override def headers: Seq[Node] = { + // stageHeadersAndCssClasses has three parts: header title, tooltip information, and sortable. + // The tooltip information could be None, which indicates it does not have a tooltip. + // Otherwise, it has two parts: tooltip text, and position (true for left, false for default). + val stageHeadersAndCssClasses: Seq[(String, Option[(String, Boolean)], Boolean)] = + Seq(("Stage Id", None, true)) ++ + {if (isFairScheduler) {Seq(("Pool Name", None, true))} else Seq.empty} ++ + Seq( + ("Description", None, true), ("Submitted", None, true), ("Duration", None, true), + ("Tasks: Succeeded/Total", None, false), + ("Input", Some((ToolTips.INPUT, false)), true), + ("Output", Some((ToolTips.OUTPUT, false)), true), + ("Shuffle Read", Some((ToolTips.SHUFFLE_READ, false)), true), + ("Shuffle Write", Some((ToolTips.SHUFFLE_WRITE, true)), true) + ) ++ + {if (isFailedStage) {Seq(("Failure Reason", None, false))} else Seq.empty} + + if (!stageHeadersAndCssClasses.filter(_._3).map(_._1).contains(sortColumn)) { + throw new IllegalArgumentException(s"Unknown column: $sortColumn") } + + val headerRow: Seq[Node] = { + stageHeadersAndCssClasses.map { case (header, tooltip, sortable) => + val headerSpan = tooltip.map { case (title, left) => + if (left) { + /* Place the shuffle write tooltip on the left (rather than the default position + of on top) because the shuffle write column is the last column on the right side and + the tooltip is wider than the column, so it doesn't fit on top. */ + + {header} + + } else { + + {header} + + } + }.getOrElse( + {header} + ) + + if (header == sortColumn) { + val headerLink = Unparsed( + parameterPath + + s"&$stageTag.sort=${URLEncoder.encode(header, "UTF-8")}" + + s"&$stageTag.desc=${!desc}" + + s"&$stageTag.pageSize=$pageSize") + + s"#$tableHeaderId" + val arrow = if (desc) "▾" else "▴" // UP or DOWN + + + + {headerSpan} +  {Unparsed(arrow)} + + + + } else { + if (sortable) { + val headerLink = Unparsed( + parameterPath + + s"&$stageTag.sort=${URLEncoder.encode(header, "UTF-8")}" + + s"&$stageTag.pageSize=$pageSize") + + s"#$tableHeaderId" + + + + {headerSpan} + + + } else { + + {headerSpan} + + } + } + } + } + {headerRow} } - /** Special table that merges two header cells. */ - protected def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { - - {columns} - - {rows.map(r => makeRow(r))} - -
    + override def row(data: StageTableRowData): Seq[Node] = { + + {rowContent(data)} + } - private def makeDescription(s: StageInfo): Seq[Node] = { + private def rowContent(data: StageTableRowData): Seq[Node] = { + data.stageData match { + case None => missingStageRow(data.stageId) + case Some(stageData) => + val info = data.stageInfo + + {if (data.attemptId > 0) { + {data.stageId} (retry {data.attemptId}) + } else { + {data.stageId} + }} ++ + {if (isFairScheduler) { + + + {data.schedulingPool} + + + } else { + Seq.empty + }} ++ + {makeDescription(info, data.descriptionOption)} + + {data.formattedSubmissionTime} + + {data.formattedDuration} + + {UIUtils.makeProgressBar(started = stageData.numActiveTasks, + completed = stageData.completedIndices.size, failed = stageData.numFailedTasks, + skipped = 0, reasonToNumKilled = stageData.reasonToNumKilled, total = info.numTasks)} + + {data.inputReadWithUnit} + {data.outputWriteWithUnit} + {data.shuffleReadWithUnit} + {data.shuffleWriteWithUnit} ++ + { + if (isFailedStage) { + failureReasonHtml(info) + } else { + Seq.empty + } + } + } + } + + private def failureReasonHtml(s: StageInfo): Seq[Node] = { + val failureReason = s.failureReason.getOrElse("") + val isMultiline = failureReason.indexOf('\n') >= 0 + // Display the first line by default + val failureReasonSummary = StringEscapeUtils.escapeHtml4( + if (isMultiline) { + failureReason.substring(0, failureReason.indexOf('\n')) + } else { + failureReason + }) + val details = if (isMultiline) { + // scalastyle:off + + +details + ++ + + // scalastyle:on + } else { + "" + } + {failureReasonSummary}{details} + } + + private def makeDescription(s: StageInfo, descriptionOption: Option[String]): Seq[Node] = { val basePathUri = UIUtils.prependBaseUri(basePath) val killLink = if (killEnabled) { @@ -83,12 +356,13 @@ private[ui] class StageTableBase( val killLinkUri = s"$basePathUri/stages/stage/kill/"
    - (kill)
    */ - val killLinkUri = s"$basePathUri/stages/stage/kill/?id=${s.stageId}&terminate=true" + val killLinkUri = s"$basePathUri/stages/stage/kill/?id=${s.stageId}" (kill) + } else { + Seq.empty } val nameLinkUri = s"$basePathUri/stages/stage?id=${s.stageId}&attempt=${s.attemptId}" @@ -111,12 +385,7 @@ private[ui] class StageTableBase( } - val stageDesc = for { - stageData <- listener.stageIdToData.get((s.stageId, s.attemptId)) - desc <- stageData.description - } yield { - UIUtils.makeDescription(desc, basePathUri) - } + val stageDesc = descriptionOption.map(UIUtils.makeDescription(_, basePathUri))
    {stageDesc.getOrElse("")} {killLink} {nameLink} {details}
    } @@ -132,22 +401,60 @@ private[ui] class StageTableBase( ++ // Shuffle Read // Shuffle Write } +} + +private[ui] class StageDataSource( + stages: Seq[StageInfo], + listener: JobProgressListener, + currentTime: Long, + pageSize: Int, + sortColumn: String, + desc: Boolean) extends PagedDataSource[StageTableRowData](pageSize) { + // Convert StageInfo to StageTableRowData which contains the final contents to show in the table + // so that we can avoid creating duplicate contents during sorting the data + private val data = stages.map(stageRow).sorted(ordering(sortColumn, desc)) + + private var _slicedStageIds: Set[Int] = _ - protected def stageRow(s: StageInfo): Seq[Node] = { + override def dataSize: Int = data.size + + override def sliceData(from: Int, to: Int): Seq[StageTableRowData] = { + val r = data.slice(from, to) + _slicedStageIds = r.map(_.stageId).toSet + r + } + + private def stageRow(s: StageInfo): StageTableRowData = { val stageDataOption = listener.stageIdToData.get((s.stageId, s.attemptId)) + if (stageDataOption.isEmpty) { - return missingStageRow(s.stageId) + return new MissingStageTableRowData(s, s.stageId, s.attemptId) } - val stageData = stageDataOption.get - val submissionTime = s.submissionTime match { + + val description = stageData.description + + val formattedSubmissionTime = s.submissionTime match { case Some(t) => UIUtils.formatDate(new Date(t)) case None => "Unknown" } - val finishTime = s.completionTime.getOrElse(System.currentTimeMillis) - val duration = s.submissionTime.map { t => - if (finishTime > t) finishTime - t else System.currentTimeMillis - t - } + val finishTime = s.completionTime.getOrElse(currentTime) + + // The submission time for a stage is misleading because it counts the time + // the stage waits to be launched. (SPARK-10930) + val taskLaunchTimes = + stageData.taskData.values.map(_.taskInfo.launchTime).filter(_ > 0) + val duration: Option[Long] = + if (taskLaunchTimes.nonEmpty) { + val startTime = taskLaunchTimes.min + if (finishTime > startTime) { + Some(finishTime - startTime) + } else { + Some(currentTime - startTime) + } + } else { + None + } val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") val inputRead = stageData.inputBytes @@ -159,76 +466,51 @@ private[ui] class StageTableBase( val shuffleWrite = stageData.shuffleWriteBytes val shuffleWriteWithUnit = if (shuffleWrite > 0) Utils.bytesToString(shuffleWrite) else "" - {if (s.attemptId > 0) { - {s.stageId} (retry {s.attemptId}) - } else { - {s.stageId} - }} ++ - {if (isFairScheduler) { - - - {stageData.schedulingPool} - - - } else { - Seq.empty - }} ++ - {makeDescription(s)} - - {submissionTime} - - {formattedDuration} - - {UIUtils.makeProgressBar(started = stageData.numActiveTasks, - completed = stageData.completedIndices.size, failed = stageData.numFailedTasks, - skipped = 0, total = s.numTasks)} - - {inputReadWithUnit} - {outputWriteWithUnit} - {shuffleReadWithUnit} - {shuffleWriteWithUnit} - } - /** Render an HTML row that represents a stage */ - private def renderStageRow(s: StageInfo): Seq[Node] = - {stageRow(s)} -} - -private[ui] class FailedStageTable( - stages: Seq[StageInfo], - basePath: String, - listener: JobProgressListener, - isFairScheduler: Boolean) - extends StageTableBase(stages, basePath, listener, isFairScheduler, killEnabled = false) { - - override protected def columns: Seq[Node] = super.columns ++ Failure Reason + new StageTableRowData( + s, + stageDataOption, + s.stageId, + s.attemptId, + stageData.schedulingPool, + description, + s.submissionTime.getOrElse(0), + formattedSubmissionTime, + duration.getOrElse(-1), + formattedDuration, + inputRead, + inputReadWithUnit, + outputWrite, + outputWriteWithUnit, + shuffleRead, + shuffleReadWithUnit, + shuffleWrite, + shuffleWriteWithUnit + ) + } - override protected def stageRow(s: StageInfo): Seq[Node] = { - val basicColumns = super.stageRow(s) - val failureReason = s.failureReason.getOrElse("") - val isMultiline = failureReason.indexOf('\n') >= 0 - // Display the first line by default - val failureReasonSummary = StringEscapeUtils.escapeHtml4( - if (isMultiline) { - failureReason.substring(0, failureReason.indexOf('\n')) - } else { - failureReason - }) - val details = if (isMultiline) { - // scalastyle:off - - +details - ++ - - // scalastyle:on + /** + * Return Ordering according to sortColumn and desc + */ + private def ordering(sortColumn: String, desc: Boolean): Ordering[StageTableRowData] = { + val ordering: Ordering[StageTableRowData] = sortColumn match { + case "Stage Id" => Ordering.by(_.stageId) + case "Pool Name" => Ordering.by(_.schedulingPool) + case "Description" => Ordering.by(x => (x.descriptionOption, x.stageInfo.name)) + case "Submitted" => Ordering.by(_.submissionTime) + case "Duration" => Ordering.by(_.duration) + case "Input" => Ordering.by(_.inputRead) + case "Output" => Ordering.by(_.outputWrite) + case "Shuffle Read" => Ordering.by(_.shuffleRead) + case "Shuffle Write" => Ordering.by(_.shuffleWrite) + case "Tasks: Succeeded/Total" => + throw new IllegalArgumentException(s"Unsortable column: $sortColumn") + case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") + } + if (desc) { + ordering.reverse } else { - "" + ordering } - val failureReasonHtml = {failureReasonSummary}{details} - basicColumns ++ failureReasonHtml } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala index 5989f0035b27..0787ea662590 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.ui.{SparkUI, SparkUITab} +import org.apache.spark.ui.{SparkUI, SparkUITab, UIUtils} /** Web UI showing progress status of all stages in the given SparkContext. */ private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages") { @@ -29,24 +29,28 @@ private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages" val killEnabled = parent.killEnabled val progressListener = parent.jobProgressListener val operationGraphListener = parent.operationGraphListener + val executorsListener = parent.executorsListener + val lastUpdateTime = parent.lastUpdateTime attachPage(new AllStagesPage(this)) attachPage(new StagePage(this)) attachPage(new PoolPage(this)) - def isFairScheduler: Boolean = progressListener.schedulingMode.exists(_ == SchedulingMode.FAIR) + def isFairScheduler: Boolean = progressListener.schedulingMode == Some(SchedulingMode.FAIR) def handleKillRequest(request: HttpServletRequest): Unit = { if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) { - val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean - val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt - if (stageId >= 0 && killFlag && progressListener.activeStages.contains(stageId)) { - sc.get.cancelStage(stageId) + // stripXSS is called first to remove suspicious characters used in XSS attacks + val stageId = Option(UIUtils.stripXSS(request.getParameter("id"))).map(_.toInt) + stageId.foreach { id => + if (progressListener.activeStages.contains(id)) { + sc.foreach(_.cancelStage(id, "killed via the Web UI")) + // Do a quick pause here to give Spark time to kill the stage so it shows up as + // killed after the refresh. Note that this will block the serving thread so the + // time should be limited in duration. + Thread.sleep(100) + } } - // Do a quick pause here to give Spark time to kill the stage so it shows up as - // killed after the refresh. Note that this will block the serving thread so the - // time should be limited in duration. - Thread.sleep(100) } } 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 f008d4018061..5acec0d0f54c 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 @@ -17,20 +17,24 @@ package org.apache.spark.ui.jobs +import scala.collection.mutable +import scala.collection.mutable.{HashMap, LinkedHashMap} + +import com.google.common.collect.Interners + import org.apache.spark.JobExecutionStatus -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor._ import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} +import org.apache.spark.util.AccumulatorContext import org.apache.spark.util.collection.OpenHashSet -import scala.collection.mutable -import scala.collection.mutable.HashMap - private[spark] object UIData { class ExecutorSummary { var taskTime : Long = 0 var failedTasks : Int = 0 var succeededTasks : Int = 0 + var reasonToNumKilled : Map[String, Int] = Map.empty var inputBytes : Long = 0 var inputRecords : Long = 0 var outputBytes : Long = 0 @@ -41,6 +45,7 @@ private[spark] object UIData { var shuffleWriteRecords : Long = 0 var memoryBytesSpilled : Long = 0 var diskBytesSpilled : Long = 0 + var isBlacklisted : Int = 0 } class JobUIData( @@ -59,8 +64,10 @@ private[spark] object UIData { var numTasks: Int = 0, var numActiveTasks: Int = 0, var numCompletedTasks: Int = 0, + var completedIndices: OpenHashSet[(Int, Int)] = new OpenHashSet[(Int, Int)](), var numSkippedTasks: Int = 0, var numFailedTasks: Int = 0, + var reasonToNumKilled: Map[String, Int] = Map.empty, /* Stages */ var numActiveStages: Int = 0, // This needs to be a set instead of a simple count to prevent double-counting of rerun stages: @@ -74,8 +81,10 @@ private[spark] object UIData { var numCompleteTasks: Int = _ var completedIndices = new OpenHashSet[Int]() var numFailedTasks: Int = _ + var reasonToNumKilled: Map[String, Int] = Map.empty var executorRunTime: Long = _ + var executorCpuTime: Long = _ var inputBytes: Long = _ var inputRecords: Long = _ @@ -87,31 +96,216 @@ private[spark] object UIData { var shuffleWriteRecords: Long = _ var memoryBytesSpilled: Long = _ var diskBytesSpilled: Long = _ + var isBlacklisted: Int = _ + var lastUpdateTime: Option[Long] = None var schedulingPool: String = "" var description: Option[String] = None var accumulables = new HashMap[Long, AccumulableInfo] - var taskData = new HashMap[Long, TaskUIData] + var taskData = new LinkedHashMap[Long, TaskUIData] var executorSummary = new HashMap[String, ExecutorSummary] def hasInput: Boolean = inputBytes > 0 def hasOutput: Boolean = outputBytes > 0 def hasShuffleRead: Boolean = shuffleReadTotalBytes > 0 def hasShuffleWrite: Boolean = shuffleWriteBytes > 0 - def hasBytesSpilled: Boolean = memoryBytesSpilled > 0 && diskBytesSpilled > 0 + def hasBytesSpilled: Boolean = memoryBytesSpilled > 0 || diskBytesSpilled > 0 } /** * These are kept mutable and reused throughout a task's lifetime to avoid excessive reallocation. */ - case class TaskUIData( - var taskInfo: TaskInfo, - var taskMetrics: Option[TaskMetrics] = None, - var errorMessage: Option[String] = None) - - case class ExecutorUIData( - val startTime: Long, - var finishTime: Option[Long] = None, - var finishReason: Option[String] = None) + class TaskUIData private(private var _taskInfo: TaskInfo) { + + private[this] var _metrics: Option[TaskMetricsUIData] = Some(TaskMetricsUIData.EMPTY) + + var errorMessage: Option[String] = None + + def taskInfo: TaskInfo = _taskInfo + + def metrics: Option[TaskMetricsUIData] = _metrics + + def updateTaskInfo(taskInfo: TaskInfo): Unit = { + _taskInfo = TaskUIData.dropInternalAndSQLAccumulables(taskInfo) + } + + def updateTaskMetrics(metrics: Option[TaskMetrics]): Unit = { + _metrics = metrics.map(TaskMetricsUIData.fromTaskMetrics) + } + + def taskDuration(lastUpdateTime: Option[Long] = None): Option[Long] = { + if (taskInfo.status == "RUNNING") { + Some(_taskInfo.timeRunning(lastUpdateTime.getOrElse(System.currentTimeMillis))) + } else { + _metrics.map(_.executorRunTime) + } + } + } + + object TaskUIData { + + private val stringInterner = Interners.newWeakInterner[String]() + + /** String interning to reduce the memory usage. */ + private def weakIntern(s: String): String = { + stringInterner.intern(s) + } + + def apply(taskInfo: TaskInfo): TaskUIData = { + new TaskUIData(dropInternalAndSQLAccumulables(taskInfo)) + } + + /** + * We don't need to store internal or SQL accumulables as their values will be shown in other + * places, so drop them to reduce the memory usage. + */ + private[spark] def dropInternalAndSQLAccumulables(taskInfo: TaskInfo): TaskInfo = { + val newTaskInfo = new TaskInfo( + taskId = taskInfo.taskId, + index = taskInfo.index, + attemptNumber = taskInfo.attemptNumber, + launchTime = taskInfo.launchTime, + executorId = weakIntern(taskInfo.executorId), + host = weakIntern(taskInfo.host), + taskLocality = taskInfo.taskLocality, + speculative = taskInfo.speculative + ) + newTaskInfo.gettingResultTime = taskInfo.gettingResultTime + newTaskInfo.setAccumulables(taskInfo.accumulables.filter { + accum => !accum.internal && accum.metadata != Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER) + }) + newTaskInfo.finishTime = taskInfo.finishTime + newTaskInfo.failed = taskInfo.failed + newTaskInfo.killed = taskInfo.killed + newTaskInfo + } + } + + case class TaskMetricsUIData( + executorDeserializeTime: Long, + executorDeserializeCpuTime: Long, + executorRunTime: Long, + executorCpuTime: Long, + resultSize: Long, + jvmGCTime: Long, + resultSerializationTime: Long, + memoryBytesSpilled: Long, + diskBytesSpilled: Long, + peakExecutionMemory: Long, + inputMetrics: InputMetricsUIData, + outputMetrics: OutputMetricsUIData, + shuffleReadMetrics: ShuffleReadMetricsUIData, + shuffleWriteMetrics: ShuffleWriteMetricsUIData) + + object TaskMetricsUIData { + def fromTaskMetrics(m: TaskMetrics): TaskMetricsUIData = { + TaskMetricsUIData( + executorDeserializeTime = m.executorDeserializeTime, + executorDeserializeCpuTime = m.executorDeserializeCpuTime, + executorRunTime = m.executorRunTime, + executorCpuTime = m.executorCpuTime, + resultSize = m.resultSize, + jvmGCTime = m.jvmGCTime, + resultSerializationTime = m.resultSerializationTime, + memoryBytesSpilled = m.memoryBytesSpilled, + diskBytesSpilled = m.diskBytesSpilled, + peakExecutionMemory = m.peakExecutionMemory, + inputMetrics = InputMetricsUIData(m.inputMetrics), + outputMetrics = OutputMetricsUIData(m.outputMetrics), + shuffleReadMetrics = ShuffleReadMetricsUIData(m.shuffleReadMetrics), + shuffleWriteMetrics = ShuffleWriteMetricsUIData(m.shuffleWriteMetrics)) + } + + val EMPTY: TaskMetricsUIData = fromTaskMetrics(TaskMetrics.empty) + } + + case class InputMetricsUIData(bytesRead: Long, recordsRead: Long) + object InputMetricsUIData { + def apply(metrics: InputMetrics): InputMetricsUIData = { + if (metrics.bytesRead == 0 && metrics.recordsRead == 0) { + EMPTY + } else { + new InputMetricsUIData( + bytesRead = metrics.bytesRead, + recordsRead = metrics.recordsRead) + } + } + private val EMPTY = InputMetricsUIData(0, 0) + } + + case class OutputMetricsUIData(bytesWritten: Long, recordsWritten: Long) + object OutputMetricsUIData { + def apply(metrics: OutputMetrics): OutputMetricsUIData = { + if (metrics.bytesWritten == 0 && metrics.recordsWritten == 0) { + EMPTY + } else { + new OutputMetricsUIData( + bytesWritten = metrics.bytesWritten, + recordsWritten = metrics.recordsWritten) + } + } + private val EMPTY = OutputMetricsUIData(0, 0) + } + + case class ShuffleReadMetricsUIData( + remoteBlocksFetched: Long, + localBlocksFetched: Long, + remoteBytesRead: Long, + remoteBytesReadToDisk: Long, + localBytesRead: Long, + fetchWaitTime: Long, + recordsRead: Long, + totalBytesRead: Long, + totalBlocksFetched: Long) + + object ShuffleReadMetricsUIData { + def apply(metrics: ShuffleReadMetrics): ShuffleReadMetricsUIData = { + if ( + metrics.remoteBlocksFetched == 0 && + metrics.localBlocksFetched == 0 && + metrics.remoteBytesRead == 0 && + metrics.localBytesRead == 0 && + metrics.fetchWaitTime == 0 && + metrics.recordsRead == 0 && + metrics.totalBytesRead == 0 && + metrics.totalBlocksFetched == 0) { + EMPTY + } else { + new ShuffleReadMetricsUIData( + remoteBlocksFetched = metrics.remoteBlocksFetched, + localBlocksFetched = metrics.localBlocksFetched, + remoteBytesRead = metrics.remoteBytesRead, + remoteBytesReadToDisk = metrics.remoteBytesReadToDisk, + localBytesRead = metrics.localBytesRead, + fetchWaitTime = metrics.fetchWaitTime, + recordsRead = metrics.recordsRead, + totalBytesRead = metrics.totalBytesRead, + totalBlocksFetched = metrics.totalBlocksFetched + ) + } + } + private val EMPTY = ShuffleReadMetricsUIData(0, 0, 0, 0, 0, 0, 0, 0, 0) + } + + case class ShuffleWriteMetricsUIData( + bytesWritten: Long, + recordsWritten: Long, + writeTime: Long) + + object ShuffleWriteMetricsUIData { + def apply(metrics: ShuffleWriteMetrics): ShuffleWriteMetricsUIData = { + if (metrics.bytesWritten == 0 && metrics.recordsWritten == 0 && metrics.writeTime == 0) { + EMPTY + } else { + new ShuffleWriteMetricsUIData( + bytesWritten = metrics.bytesWritten, + recordsWritten = metrics.recordsWritten, + writeTime = metrics.writeTime + ) + } + } + private val EMPTY = ShuffleWriteMetricsUIData(0, 0, 0) + } + } diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala index 81f168a447ea..bb763248cd7e 100644 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala @@ -17,10 +17,14 @@ package org.apache.spark.ui.scope +import java.util.Objects + import scala.collection.mutable -import scala.collection.mutable.{StringBuilder, ListBuffer} +import scala.collection.mutable.{ListBuffer, StringBuilder} + +import org.apache.commons.lang3.StringEscapeUtils -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.StageInfo import org.apache.spark.storage.StorageLevel @@ -38,7 +42,7 @@ private[ui] case class RDDOperationGraph( rootCluster: RDDOperationCluster) /** A node in an RDDOperationGraph. This represents an RDD. */ -private[ui] case class RDDOperationNode(id: Int, name: String, cached: Boolean) +private[ui] case class RDDOperationNode(id: Int, name: String, cached: Boolean, callsite: String) /** * A directed edge connecting two nodes in an RDDOperationGraph. @@ -70,6 +74,22 @@ private[ui] class RDDOperationCluster(val id: String, private var _name: String) def getCachedNodes: Seq[RDDOperationNode] = { _childNodes.filter(_.cached) ++ _childClusters.flatMap(_.getCachedNodes) } + + def canEqual(other: Any): Boolean = other.isInstanceOf[RDDOperationCluster] + + override def equals(other: Any): Boolean = other match { + case that: RDDOperationCluster => + (that canEqual this) && + _childClusters == that._childClusters && + id == that.id && + _name == that._name + case _ => false + } + + override def hashCode(): Int = { + val state = Seq(_childClusters, id, _name) + state.map(Objects.hashCode).foldLeft(0)((a, b) => 31 * a + b) + } } private[ui] object RDDOperationGraph extends Logging { @@ -87,7 +107,7 @@ private[ui] object RDDOperationGraph extends Logging { * supporting in the future if we decide to group certain stages within the same job under * a common scope (e.g. part of a SQL query). */ - def makeOperationGraph(stage: StageInfo): RDDOperationGraph = { + def makeOperationGraph(stage: StageInfo, retainedNodes: Int): RDDOperationGraph = { val edges = new ListBuffer[RDDOperationEdge] val nodes = new mutable.HashMap[Int, RDDOperationNode] val clusters = new mutable.HashMap[String, RDDOperationCluster] // indexed by cluster ID @@ -99,18 +119,37 @@ private[ui] object RDDOperationGraph extends Logging { { if (stage.attemptId == 0) "" else s" (attempt ${stage.attemptId})" } val rootCluster = new RDDOperationCluster(stageClusterId, stageClusterName) + var rootNodeCount = 0 + val addRDDIds = new mutable.HashSet[Int]() + val dropRDDIds = new mutable.HashSet[Int]() + // Find nodes, edges, and operation scopes that belong to this stage - stage.rddInfos.foreach { rdd => - edges ++= rdd.parentIds.map { parentId => RDDOperationEdge(parentId, rdd.id) } + stage.rddInfos.sortBy(_.id).foreach { rdd => + val parentIds = rdd.parentIds + val isAllowed = + if (parentIds.isEmpty) { + rootNodeCount += 1 + rootNodeCount <= retainedNodes + } else { + parentIds.exists(id => addRDDIds.contains(id) || !dropRDDIds.contains(id)) + } - // TODO: differentiate between the intention to cache an RDD and whether it's actually cached - val node = nodes.getOrElseUpdate( - rdd.id, RDDOperationNode(rdd.id, rdd.name, rdd.storageLevel != StorageLevel.NONE)) + if (isAllowed) { + addRDDIds += rdd.id + edges ++= parentIds.filter(id => !dropRDDIds.contains(id)).map(RDDOperationEdge(_, rdd.id)) + } else { + dropRDDIds += rdd.id + } + // TODO: differentiate between the intention to cache an RDD and whether it's actually cached + val node = nodes.getOrElseUpdate(rdd.id, RDDOperationNode( + rdd.id, rdd.name, rdd.storageLevel != StorageLevel.NONE, rdd.callSite)) if (rdd.scope.isEmpty) { // This RDD has no encompassing scope, so we put it directly in the root cluster // This should happen only if an RDD is instantiated outside of a public RDD API - rootCluster.attachChildNode(node) + if (isAllowed) { + rootCluster.attachChildNode(node) + } } else { // Otherwise, this RDD belongs to an inner cluster, // which may be nested inside of other clusters @@ -129,8 +168,14 @@ private[ui] object RDDOperationGraph extends Logging { } } // Attach the outermost cluster to the root cluster, and the RDD to the innermost cluster - rddClusters.headOption.foreach { cluster => rootCluster.attachChildCluster(cluster) } - rddClusters.lastOption.foreach { cluster => cluster.attachChildNode(node) } + rddClusters.headOption.foreach { cluster => + if (!rootCluster.childClusters.contains(cluster)) { + rootCluster.attachChildCluster(cluster) + } + } + if (isAllowed) { + rddClusters.lastOption.foreach { cluster => cluster.attachChildNode(node) } + } } } @@ -177,7 +222,13 @@ private[ui] object RDDOperationGraph extends Logging { /** Return the dot representation of a node in an RDDOperationGraph. */ private def makeDotNode(node: RDDOperationNode): String = { - s"""${node.id} [label="${node.name} [${node.id}]"]""" + val isCached = if (node.cached) { + " [Cached]" + } else { + "" + } + val label = s"${node.name} [${node.id}]$isCached\n${node.callsite}" + s"""${node.id} [label="${StringEscapeUtils.escapeJava(label)}"]""" } /** Update the dot representation of the RDDOperationGraph in cluster to subgraph. */ @@ -186,7 +237,7 @@ private[ui] object RDDOperationGraph extends Logging { cluster: RDDOperationCluster, indent: String): Unit = { subgraph.append(indent).append(s"subgraph cluster${cluster.id} {\n") - subgraph.append(indent).append(s""" label="${cluster.name}";\n""") + .append(indent).append(s""" label="${StringEscapeUtils.escapeJava(cluster.name)}";\n""") cluster.childNodes.foreach { node => subgraph.append(indent).append(s" ${makeDotNode(node)};\n") } diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala index 89119cd3579e..37a12a864693 100644 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala @@ -41,6 +41,10 @@ private[ui] class RDDOperationGraphListener(conf: SparkConf) extends SparkListen private[ui] val jobIds = new mutable.ArrayBuffer[Int] private[ui] val stageIds = new mutable.ArrayBuffer[Int] + // How many root nodes to retain in DAG Graph + private[ui] val retainedNodes = + conf.getInt("spark.ui.dagGraph.retainedRootRDDs", Int.MaxValue) + // How many jobs or stages to retain graph metadata for private val retainedJobs = conf.getInt("spark.ui.retainedJobs", SparkUI.DEFAULT_RETAINED_JOBS) @@ -52,9 +56,8 @@ private[ui] class RDDOperationGraphListener(conf: SparkConf) extends SparkListen * An empty list is returned if one or more of its stages has been cleaned up. */ def getOperationGraphForJob(jobId: Int): Seq[RDDOperationGraph] = synchronized { - val skippedStageIds = jobIdToSkippedStageIds.get(jobId).getOrElse(Seq.empty) - val graphs = jobIdToStageIds.get(jobId) - .getOrElse(Seq.empty) + val skippedStageIds = jobIdToSkippedStageIds.getOrElse(jobId, Seq.empty) + val graphs = jobIdToStageIds.getOrElse(jobId, Seq.empty) .flatMap { sid => stageIdToGraph.get(sid) } // Mark any skipped stages as such graphs.foreach { g => @@ -83,7 +86,7 @@ private[ui] class RDDOperationGraphListener(conf: SparkConf) extends SparkListen val stageId = stageInfo.stageId stageIds += stageId stageIdToJobId(stageId) = jobId - stageIdToGraph(stageId) = RDDOperationGraph.makeOperationGraph(stageInfo) + stageIdToGraph(stageId) = RDDOperationGraph.makeOperationGraph(stageInfo, retainedNodes) trimStagesIfNecessary() } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index fd6cc3ed759b..e8ff08f7d88f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -31,24 +31,27 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - val parameterId = request.getParameter("id") + // stripXSS is called first to remove suspicious characters used in XSS attacks + val parameterId = UIUtils.stripXSS(request.getParameter("id")) require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val parameterBlockPage = request.getParameter("block.page") - val parameterBlockSortColumn = request.getParameter("block.sort") - val parameterBlockSortDesc = request.getParameter("block.desc") - val parameterBlockPageSize = request.getParameter("block.pageSize") + val parameterBlockPage = UIUtils.stripXSS(request.getParameter("block.page")) + val parameterBlockSortColumn = UIUtils.stripXSS(request.getParameter("block.sort")) + val parameterBlockSortDesc = UIUtils.stripXSS(request.getParameter("block.desc")) + val parameterBlockPageSize = UIUtils.stripXSS(request.getParameter("block.pageSize")) + val parameterBlockPrevPageSize = UIUtils.stripXSS(request.getParameter("block.prevPageSize")) val blockPage = Option(parameterBlockPage).map(_.toInt).getOrElse(1) val blockSortColumn = Option(parameterBlockSortColumn).getOrElse("Block Name") val blockSortDesc = Option(parameterBlockSortDesc).map(_.toBoolean).getOrElse(false) val blockPageSize = Option(parameterBlockPageSize).map(_.toInt).getOrElse(100) + val blockPrevPageSize = Option(parameterBlockPrevPageSize).map(_.toInt).getOrElse(blockPageSize) val rddId = parameterId.toInt val rddStorageInfo = AllRDDResource.getRDDStorageInfo(rddId, listener, includeDetails = true) .getOrElse { // Rather than crashing, render an "RDD Not Found" page - return UIUtils.headerSparkPage("RDD Not Found", Seq[Node](), parent) + return UIUtils.headerSparkPage("RDD Not Found", Seq.empty[Node], parent) } // Worker table @@ -56,17 +59,26 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { rddStorageInfo.dataDistribution.get, id = Some("rdd-storage-by-worker-table")) // Block table - val (blockTable, blockTableHTML) = try { + val page: Int = { + // If the user has changed to a larger page size, then go to page 1 in order to avoid + // IndexOutOfBoundsException. + if (blockPageSize <= blockPrevPageSize) { + blockPage + } else { + 1 + } + } + val blockTableHTML = try { val _blockTable = new BlockPagedTable( UIUtils.prependBaseUri(parent.basePath) + s"/storage/rdd/?id=${rddId}", rddStorageInfo.partitions.get, blockPageSize, blockSortColumn, blockSortDesc) - (_blockTable, _blockTable.table(blockPage)) + _blockTable.table(page) } catch { case e @ (_ : IllegalArgumentException | _ : IndexOutOfBoundsException) => - (null,
    {e.getMessage}
    ) +
    {e.getMessage}
    } val jsForScrollingDownToBlockTable = @@ -136,7 +148,8 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { /** Header fields for the worker table */ private def workerHeader = Seq( "Host", - "Memory Usage", + "On Heap Memory Usage", + "Off Heap Memory Usage", "Disk Usage") /** Render an HTML row representing a worker */ @@ -144,8 +157,12 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { {worker.address} - {Utils.bytesToString(worker.memoryUsed)} - ({Utils.bytesToString(worker.memoryRemaining)} Remaining) + {Utils.bytesToString(worker.onHeapMemoryUsed.getOrElse(0L))} + ({Utils.bytesToString(worker.onHeapMemoryRemaining.getOrElse(0L))} Remaining) + + + {Utils.bytesToString(worker.offHeapMemoryUsed.getOrElse(0L))} + ({Utils.bytesToString(worker.offHeapMemoryRemaining.getOrElse(0L))} Remaining) {Utils.bytesToString(worker.diskUsed)} @@ -186,27 +203,12 @@ private[ui] class BlockDataSource( * Return Ordering according to sortColumn and desc */ private def ordering(sortColumn: String, desc: Boolean): Ordering[BlockTableRowData] = { - val ordering = sortColumn match { - case "Block Name" => new Ordering[BlockTableRowData] { - override def compare(x: BlockTableRowData, y: BlockTableRowData): Int = - Ordering.String.compare(x.blockName, y.blockName) - } - case "Storage Level" => new Ordering[BlockTableRowData] { - override def compare(x: BlockTableRowData, y: BlockTableRowData): Int = - Ordering.String.compare(x.storageLevel, y.storageLevel) - } - case "Size in Memory" => new Ordering[BlockTableRowData] { - override def compare(x: BlockTableRowData, y: BlockTableRowData): Int = - Ordering.Long.compare(x.memoryUsed, y.memoryUsed) - } - case "Size on Disk" => new Ordering[BlockTableRowData] { - override def compare(x: BlockTableRowData, y: BlockTableRowData): Int = - Ordering.Long.compare(x.diskUsed, y.diskUsed) - } - case "Executors" => new Ordering[BlockTableRowData] { - override def compare(x: BlockTableRowData, y: BlockTableRowData): Int = - Ordering.String.compare(x.executors, y.executors) - } + val ordering: Ordering[BlockTableRowData] = sortColumn match { + case "Block Name" => Ordering.by(_.blockName) + case "Storage Level" => Ordering.by(_.storageLevel) + case "Size in Memory" => Ordering.by(_.memoryUsed) + case "Size on Disk" => Ordering.by(_.diskUsed) + case "Executors" => Ordering.by(_.executors) case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") } if (desc) { @@ -226,7 +228,14 @@ private[ui] class BlockPagedTable( override def tableId: String = "rdd-storage-by-block-table" - override def tableCssClass: String = "table table-bordered table-condensed table-striped" + override def tableCssClass: String = + "table table-bordered table-condensed table-striped table-head-clickable" + + override def pageSizeFormField: String = "block.pageSize" + + override def prevPageSizeFormField: String = "block.prevPageSize" + + override def pageNumberFormField: String = "block.page" override val dataSource: BlockDataSource = new BlockDataSource( rddPartitions, @@ -236,24 +245,16 @@ private[ui] class BlockPagedTable( override def pageLink(page: Int): String = { val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") - s"${basePath}&block.page=$page&block.sort=${encodedSortColumn}&block.desc=${desc}" + - s"&block.pageSize=${pageSize}" + basePath + + s"&$pageNumberFormField=$page" + + s"&block.sort=$encodedSortColumn" + + s"&block.desc=$desc" + + s"&$pageSizeFormField=$pageSize" } - override def goButtonJavascriptFunction: (String, String) = { - val jsFuncName = "goToBlockPage" + override def goButtonFormPath: String = { val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") - val jsFunc = s""" - |currentBlockPageSize = ${pageSize} - |function goToBlockPage(page, pageSize) { - | // Set page to 1 if the page size changes - | page = pageSize == currentBlockPageSize ? page : 1; - | var url = "${basePath}&block.sort=${encodedSortColumn}&block.desc=${desc}" + - | "&block.page=" + page + "&block.pageSize=" + pageSize; - | window.location.href = url; - |} - """.stripMargin - (jsFuncName, jsFunc) + s"$basePath&block.sort=$encodedSortColumn&block.desc=$desc" } override def headers: Seq[Node] = { @@ -271,22 +272,27 @@ private[ui] class BlockPagedTable( val headerRow: Seq[Node] = { blockHeaders.map { header => if (header == sortColumn) { - val headerLink = - s"$basePath&block.sort=${URLEncoder.encode(header, "UTF-8")}&block.desc=${!desc}" + - s"&block.pageSize=${pageSize}" - val js = Unparsed(s"window.location.href='${headerLink}'") + val headerLink = Unparsed( + basePath + + s"&block.sort=${URLEncoder.encode(header, "UTF-8")}" + + s"&block.desc=${!desc}" + + s"&block.pageSize=$pageSize") val arrow = if (desc) "▾" else "▴" // UP or DOWN - - {header} -  {Unparsed(arrow)} + + + {header} +  {Unparsed(arrow)} + } else { - val headerLink = - s"$basePath&block.sort=${URLEncoder.encode(header, "UTF-8")}" + - s"&block.pageSize=${pageSize}" - val js = Unparsed(s"window.location.href='${headerLink}'") - - {header} + val headerLink = Unparsed( + basePath + + s"&block.sort=${URLEncoder.encode(header, "UTF-8")}" + + s"&block.pageSize=$pageSize") + + + {header} + } } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 04f584621e71..aa84788f1df8 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -54,7 +54,6 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { "Cached Partitions", "Fraction Cached", "Size in Memory", - "Size in ExternalBlockStore", "Size on Disk") /** Render an HTML row representing an RDD */ @@ -71,7 +70,6 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { {rdd.numCachedPartitions.toString} {"%.0f%%".format(rdd.numCachedPartitions * 100.0 / rdd.numPartitions)} {Utils.bytesToString(rdd.memSize)} - {Utils.bytesToString(rdd.externalBlockStoreSize)} {Utils.bytesToString(rdd.diskSize)} // scalastyle:on @@ -104,7 +102,6 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { "Executor ID", "Address", "Total Size in Memory", - "Total Size in ExternalBlockStore", "Total Size on Disk", "Stream Blocks") @@ -119,9 +116,6 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { {Utils.bytesToString(status.totalMemSize)} - - {Utils.bytesToString(status.totalExternalBlockStoreSize)} - {Utils.bytesToString(status.totalDiskSize)} @@ -157,12 +151,12 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { /** Render a stream block */ private def streamBlockTableRow(block: (BlockId, Seq[BlockUIData])): Seq[Node] = { val replications = block._2 - assert(replications.size > 0) // This must be true because it's the result of "groupBy" + assert(replications.nonEmpty) // This must be true because it's the result of "groupBy" if (replications.size == 1) { streamBlockTableSubrow(block._1, replications.head, replications.size, true) } else { streamBlockTableSubrow(block._1, replications.head, replications.size, true) ++ - replications.tail.map(streamBlockTableSubrow(block._1, _, replications.size, false)).flatten + replications.tail.flatMap(streamBlockTableSubrow(block._1, _, replications.size, false)) } } @@ -195,8 +189,6 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { ("Memory", block.memSize) } else if (block.storageLevel.useMemory && !block.storageLevel.deserialized) { ("Memory Serialized", block.memSize) - } else if (block.storageLevel.useOffHeap) { - ("External", block.externalBlockStoreSize) } else { throw new IllegalStateException(s"Invalid Storage Level: ${block.storageLevel}") } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 22e2993b3b5b..148efb134e14 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -20,9 +20,9 @@ package org.apache.spark.ui.storage import scala.collection.mutable import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.ui._ import org.apache.spark.scheduler._ import org.apache.spark.storage._ +import org.apache.spark.ui._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storage") { @@ -39,11 +39,12 @@ private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storag * This class is thread-safe (unlike JobProgressListener) */ @DeveloperApi +@deprecated("This class will be removed in a future release.", "2.2.0") class StorageListener(storageStatusListener: StorageStatusListener) extends BlockStatusListener { private[ui] val _rddInfoMap = mutable.Map[Int, RDDInfo]() // exposed for testing - def storageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList + def activeStorageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList /** Filter RDD info to include only those with cached partitions */ def rddInfoList: Seq[RDDInfo] = synchronized { @@ -54,23 +55,12 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Bloc private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = { val rddIdsToUpdate = updatedBlocks.flatMap { case (bid, _) => bid.asRDDId.map(_.rddId) }.toSet val rddInfosToUpdate = _rddInfoMap.values.toSeq.filter { s => rddIdsToUpdate.contains(s.id) } - StorageUtils.updateRddInfo(rddInfosToUpdate, storageStatusList) - } - - /** - * Assumes the storage status list is fully up-to-date. This implies the corresponding - * StorageStatusSparkListener must process the SparkListenerTaskEnd event before this listener. - */ - override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { - val metrics = taskEnd.taskMetrics - if (metrics != null && metrics.updatedBlocks.isDefined) { - updateRDDInfo(metrics.updatedBlocks.get) - } + StorageUtils.updateRddInfo(rddInfosToUpdate, activeStorageStatusList) } override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized { val rddInfos = stageSubmitted.stageInfo.rddInfos - rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info) } + rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info).name = info.name } } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = synchronized { @@ -84,4 +74,14 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Bloc override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = synchronized { _rddInfoMap.remove(unpersistRDD.rddId) } + + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { + super.onBlockUpdated(blockUpdated) + val blockId = blockUpdated.blockUpdatedInfo.blockId + val storageLevel = blockUpdated.blockUpdatedInfo.storageLevel + val memSize = blockUpdated.blockUpdatedInfo.memSize + val diskSize = blockUpdated.blockUpdatedInfo.diskSize + val blockStatus = BlockStatus(storageLevel, memSize, diskSize) + updateRDDInfo(Seq((blockId, blockStatus))) + } } diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala new file mode 100644 index 000000000000..f4a736d6d439 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -0,0 +1,503 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.{lang => jl} +import java.io.ObjectInputStream +import java.util.{ArrayList, Collections} +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.atomic.AtomicLong + +import org.apache.spark.{InternalAccumulator, SparkContext, TaskContext} +import org.apache.spark.scheduler.AccumulableInfo + +private[spark] case class AccumulatorMetadata( + id: Long, + name: Option[String], + countFailedValues: Boolean) extends Serializable + + +/** + * The base class for accumulators, that can accumulate inputs of type `IN`, and produce output of + * type `OUT`. + * + * `OUT` should be a type that can be read atomically (e.g., Int, Long), or thread-safely + * (e.g., synchronized collections) because it will be read from other threads. + */ +abstract class AccumulatorV2[IN, OUT] extends Serializable { + private[spark] var metadata: AccumulatorMetadata = _ + private[this] var atDriverSide = true + + private[spark] def register( + sc: SparkContext, + name: Option[String] = None, + countFailedValues: Boolean = false): Unit = { + if (this.metadata != null) { + throw new IllegalStateException("Cannot register an Accumulator twice.") + } + this.metadata = AccumulatorMetadata(AccumulatorContext.newId(), name, countFailedValues) + AccumulatorContext.register(this) + sc.cleaner.foreach(_.registerAccumulatorForCleanup(this)) + } + + /** + * Returns true if this accumulator has been registered. + * + * @note All accumulators must be registered before use, or it will throw exception. + */ + final def isRegistered: Boolean = + metadata != null && AccumulatorContext.get(metadata.id).isDefined + + private def assertMetadataNotNull(): Unit = { + if (metadata == null) { + throw new IllegalStateException("The metadata of this accumulator has not been assigned yet.") + } + } + + /** + * Returns the id of this accumulator, can only be called after registration. + */ + final def id: Long = { + assertMetadataNotNull() + metadata.id + } + + /** + * Returns the name of this accumulator, can only be called after registration. + */ + final def name: Option[String] = { + assertMetadataNotNull() + + if (atDriverSide) { + metadata.name.orElse(AccumulatorContext.get(id).flatMap(_.metadata.name)) + } else { + metadata.name + } + } + + /** + * Whether to accumulate values from failed tasks. This is set to true for system and time + * metrics like serialization time or bytes spilled, and false for things with absolute values + * like number of input rows. This should be used for internal metrics only. + */ + private[spark] final def countFailedValues: Boolean = { + assertMetadataNotNull() + metadata.countFailedValues + } + + /** + * Creates an [[AccumulableInfo]] representation of this [[AccumulatorV2]] with the provided + * values. + */ + private[spark] def toInfo(update: Option[Any], value: Option[Any]): AccumulableInfo = { + val isInternal = name.exists(_.startsWith(InternalAccumulator.METRICS_PREFIX)) + new AccumulableInfo(id, name, update, value, isInternal, countFailedValues) + } + + final private[spark] def isAtDriverSide: Boolean = atDriverSide + + /** + * Returns if this accumulator is zero value or not. e.g. for a counter accumulator, 0 is zero + * value; for a list accumulator, Nil is zero value. + */ + def isZero: Boolean + + /** + * Creates a new copy of this accumulator, which is zero value. i.e. call `isZero` on the copy + * must return true. + */ + def copyAndReset(): AccumulatorV2[IN, OUT] = { + val copyAcc = copy() + copyAcc.reset() + copyAcc + } + + /** + * Creates a new copy of this accumulator. + */ + def copy(): AccumulatorV2[IN, OUT] + + /** + * Resets this accumulator, which is zero value. i.e. call `isZero` must + * return true. + */ + def reset(): Unit + + /** + * Takes the inputs and accumulates. + */ + def add(v: IN): Unit + + /** + * Merges another same-type accumulator into this one and update its state, i.e. this should be + * merge-in-place. + */ + def merge(other: AccumulatorV2[IN, OUT]): Unit + + /** + * Defines the current value of this accumulator + */ + def value: OUT + + // Called by Java when serializing an object + final protected def writeReplace(): Any = { + if (atDriverSide) { + if (!isRegistered) { + throw new UnsupportedOperationException( + "Accumulator must be registered before send to executor") + } + val copyAcc = copyAndReset() + assert(copyAcc.isZero, "copyAndReset must return a zero value copy") + val isInternalAcc = name.isDefined && name.get.startsWith(InternalAccumulator.METRICS_PREFIX) + if (isInternalAcc) { + // Do not serialize the name of internal accumulator and send it to executor. + copyAcc.metadata = metadata.copy(name = None) + } else { + // For non-internal accumulators, we still need to send the name because users may need to + // access the accumulator name at executor side, or they may keep the accumulators sent from + // executors and access the name when the registered accumulator is already garbage + // collected(e.g. SQLMetrics). + copyAcc.metadata = metadata + } + copyAcc + } else { + this + } + } + + // Called by Java when deserializing an object + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { + in.defaultReadObject() + if (atDriverSide) { + atDriverSide = false + + // Automatically register the accumulator when it is deserialized with the task closure. + // This is for external accumulators and internal ones that do not represent task level + // metrics, e.g. internal SQL metrics, which are per-operator. + val taskContext = TaskContext.get() + if (taskContext != null) { + taskContext.registerAccumulator(this) + } + } else { + atDriverSide = true + } + } + + override def toString: String = { + if (metadata == null) { + "Un-registered Accumulator: " + getClass.getSimpleName + } else { + getClass.getSimpleName + s"(id: $id, name: $name, value: $value)" + } + } +} + + +/** + * An internal class used to track accumulators by Spark itself. + */ +private[spark] object AccumulatorContext { + + /** + * This global map holds the original accumulator objects that are created on the driver. + * It keeps weak references to these objects so that accumulators can be garbage-collected + * once the RDDs and user-code that reference them are cleaned up. + * TODO: Don't use a global map; these should be tied to a SparkContext (SPARK-13051). + */ + private val originals = new ConcurrentHashMap[Long, jl.ref.WeakReference[AccumulatorV2[_, _]]] + + private[this] val nextId = new AtomicLong(0L) + + /** + * Returns a globally unique ID for a new [[AccumulatorV2]]. + * Note: Once you copy the [[AccumulatorV2]] the ID is no longer unique. + */ + def newId(): Long = nextId.getAndIncrement + + /** Returns the number of accumulators registered. Used in testing. */ + def numAccums: Int = originals.size + + /** + * Registers an [[AccumulatorV2]] created on the driver such that it can be used on the executors. + * + * All accumulators registered here can later be used as a container for accumulating partial + * values across multiple tasks. This is what `org.apache.spark.scheduler.DAGScheduler` does. + * Note: if an accumulator is registered here, it should also be registered with the active + * context cleaner for cleanup so as to avoid memory leaks. + * + * If an [[AccumulatorV2]] with the same ID was already registered, this does nothing instead + * of overwriting it. We will never register same accumulator twice, this is just a sanity check. + */ + def register(a: AccumulatorV2[_, _]): Unit = { + originals.putIfAbsent(a.id, new jl.ref.WeakReference[AccumulatorV2[_, _]](a)) + } + + /** + * Unregisters the [[AccumulatorV2]] with the given ID, if any. + */ + def remove(id: Long): Unit = { + originals.remove(id) + } + + /** + * Returns the [[AccumulatorV2]] registered with the given ID, if any. + */ + def get(id: Long): Option[AccumulatorV2[_, _]] = { + Option(originals.get(id)).map { ref => + // Since we are storing weak references, we must check whether the underlying data is valid. + val acc = ref.get + if (acc eq null) { + throw new IllegalStateException(s"Attempted to access garbage collected accumulator $id") + } + acc + } + } + + /** + * Clears all registered [[AccumulatorV2]]s. For testing only. + */ + def clear(): Unit = { + originals.clear() + } + + // Identifier for distinguishing SQL metrics from other accumulators + private[spark] val SQL_ACCUM_IDENTIFIER = "sql" +} + + +/** + * An [[AccumulatorV2 accumulator]] for computing sum, count, and average of 64-bit integers. + * + * @since 2.0.0 + */ +class LongAccumulator extends AccumulatorV2[jl.Long, jl.Long] { + private var _sum = 0L + private var _count = 0L + + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + override def isZero: Boolean = _sum == 0L && _count == 0 + + override def copy(): LongAccumulator = { + val newAcc = new LongAccumulator + newAcc._count = this._count + newAcc._sum = this._sum + newAcc + } + + override def reset(): Unit = { + _sum = 0L + _count = 0L + } + + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + override def add(v: jl.Long): Unit = { + _sum += v + _count += 1 + } + + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + def add(v: Long): Unit = { + _sum += v + _count += 1 + } + + /** + * Returns the number of elements added to the accumulator. + * @since 2.0.0 + */ + def count: Long = _count + + /** + * Returns the sum of elements added to the accumulator. + * @since 2.0.0 + */ + def sum: Long = _sum + + /** + * Returns the average of elements added to the accumulator. + * @since 2.0.0 + */ + def avg: Double = _sum.toDouble / _count + + override def merge(other: AccumulatorV2[jl.Long, jl.Long]): Unit = other match { + case o: LongAccumulator => + _sum += o.sum + _count += o.count + case _ => + throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + } + + private[spark] def setValue(newValue: Long): Unit = _sum = newValue + + override def value: jl.Long = _sum +} + + +/** + * An [[AccumulatorV2 accumulator]] for computing sum, count, and averages for double precision + * floating numbers. + * + * @since 2.0.0 + */ +class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] { + private var _sum = 0.0 + private var _count = 0L + + override def isZero: Boolean = _sum == 0.0 && _count == 0 + + override def copy(): DoubleAccumulator = { + val newAcc = new DoubleAccumulator + newAcc._count = this._count + newAcc._sum = this._sum + newAcc + } + + override def reset(): Unit = { + _sum = 0.0 + _count = 0L + } + + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + override def add(v: jl.Double): Unit = { + _sum += v + _count += 1 + } + + /** + * Adds v to the accumulator, i.e. increment sum by v and count by 1. + * @since 2.0.0 + */ + def add(v: Double): Unit = { + _sum += v + _count += 1 + } + + /** + * Returns the number of elements added to the accumulator. + * @since 2.0.0 + */ + def count: Long = _count + + /** + * Returns the sum of elements added to the accumulator. + * @since 2.0.0 + */ + def sum: Double = _sum + + /** + * Returns the average of elements added to the accumulator. + * @since 2.0.0 + */ + def avg: Double = _sum / _count + + override def merge(other: AccumulatorV2[jl.Double, jl.Double]): Unit = other match { + case o: DoubleAccumulator => + _sum += o.sum + _count += o.count + case _ => + throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + } + + private[spark] def setValue(newValue: Double): Unit = _sum = newValue + + override def value: jl.Double = _sum +} + + +/** + * An [[AccumulatorV2 accumulator]] for collecting a list of elements. + * + * @since 2.0.0 + */ +class CollectionAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { + private val _list: java.util.List[T] = Collections.synchronizedList(new ArrayList[T]()) + + override def isZero: Boolean = _list.isEmpty + + override def copyAndReset(): CollectionAccumulator[T] = new CollectionAccumulator + + override def copy(): CollectionAccumulator[T] = { + val newAcc = new CollectionAccumulator[T] + _list.synchronized { + newAcc._list.addAll(_list) + } + newAcc + } + + override def reset(): Unit = _list.clear() + + override def add(v: T): Unit = _list.add(v) + + override def merge(other: AccumulatorV2[T, java.util.List[T]]): Unit = other match { + case o: CollectionAccumulator[T] => _list.addAll(o.value) + case _ => throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + } + + override def value: java.util.List[T] = _list.synchronized { + java.util.Collections.unmodifiableList(new ArrayList[T](_list)) + } + + private[spark] def setValue(newValue: java.util.List[T]): Unit = { + _list.clear() + _list.addAll(newValue) + } +} + + +class LegacyAccumulatorWrapper[R, T]( + initialValue: R, + param: org.apache.spark.AccumulableParam[R, T]) extends AccumulatorV2[T, R] { + private[spark] var _value = initialValue // Current value on driver + + override def isZero: Boolean = _value == param.zero(initialValue) + + override def copy(): LegacyAccumulatorWrapper[R, T] = { + val acc = new LegacyAccumulatorWrapper(initialValue, param) + acc._value = _value + acc + } + + override def reset(): Unit = { + _value = param.zero(initialValue) + } + + override def add(v: T): Unit = _value = param.addAccumulator(_value, v) + + override def merge(other: AccumulatorV2[T, R]): Unit = other match { + case o: LegacyAccumulatorWrapper[R, T] => _value = param.addInPlace(_value, o.value) + case _ => throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + } + + override def value: R = _value +} diff --git a/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala b/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala deleted file mode 100644 index 81a7cbde01ce..000000000000 --- a/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import akka.actor.Actor -import org.slf4j.Logger - -/** - * A trait to enable logging all Akka actor messages. Here's an example of using this: - * - * {{{ - * class BlockManagerMasterActor extends Actor with ActorLogReceive with Logging { - * ... - * override def receiveWithLogging = { - * case GetLocations(blockId) => - * sender ! getLocations(blockId) - * ... - * } - * ... - * } - * }}} - * - */ -private[spark] trait ActorLogReceive { - self: Actor => - - override def receive: Actor.Receive = new Actor.Receive { - - private val _receiveWithLogging = receiveWithLogging - - override def isDefinedAt(o: Any): Boolean = { - val handled = _receiveWithLogging.isDefinedAt(o) - if (!handled) { - log.debug(s"Received unexpected actor system event: $o") - } - handled - } - - override def apply(o: Any): Unit = { - if (log.isDebugEnabled) { - log.debug(s"[actor] received message $o from ${self.sender}") - } - val start = System.nanoTime - _receiveWithLogging.apply(o) - val timeTaken = (System.nanoTime - start).toDouble / 1000000 - if (log.isDebugEnabled) { - log.debug(s"[actor] handled message ($timeTaken ms) $o from ${self.sender}") - } - } - } - - def receiveWithLogging: Actor.Receive - - protected def log: Logger -} diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala deleted file mode 100644 index 1738258a0c79..000000000000 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ /dev/null @@ -1,242 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import scala.collection.JavaConverters._ - -import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem} -import akka.pattern.ask - -import com.typesafe.config.ConfigFactory -import org.apache.log4j.{Level, Logger} - -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv, SparkException} -import org.apache.spark.rpc.RpcTimeout - -/** - * Various utility classes for working with Akka. - */ -private[spark] object AkkaUtils extends Logging { - - /** - * Creates an ActorSystem ready for remoting, with various Spark features. Returns both the - * ActorSystem itself and its port (which is hard to get from Akka). - * - * Note: the `name` parameter is important, as even if a client sends a message to right - * host + port, if the system name is incorrect, Akka will drop the message. - * - * If indestructible is set to true, the Actor System will continue running in the event - * of a fatal exception. This is used by [[org.apache.spark.executor.Executor]]. - */ - def createActorSystem( - name: String, - host: String, - port: Int, - conf: SparkConf, - securityManager: SecurityManager): (ActorSystem, Int) = { - val startService: Int => (ActorSystem, Int) = { actualPort => - doCreateActorSystem(name, host, actualPort, conf, securityManager) - } - Utils.startServiceOnPort(port, startService, conf, name) - } - - private def doCreateActorSystem( - name: String, - host: String, - port: Int, - conf: SparkConf, - securityManager: SecurityManager): (ActorSystem, Int) = { - - val akkaThreads = conf.getInt("spark.akka.threads", 4) - val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) - val akkaTimeoutS = conf.getTimeAsSeconds("spark.akka.timeout", - conf.get("spark.network.timeout", "120s")) - val akkaFrameSize = maxFrameSizeBytes(conf) - val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) - val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" - if (!akkaLogLifecycleEvents) { - // As a workaround for Akka issue #3787, we coerce the "EndpointWriter" log to be silent. - // See: https://www.assembla.com/spaces/akka/tickets/3787#/ - Option(Logger.getLogger("akka.remote.EndpointWriter")).map(l => l.setLevel(Level.FATAL)) - } - - val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" - - val akkaHeartBeatPausesS = conf.getTimeAsSeconds("spark.akka.heartbeat.pauses", "6000s") - val akkaHeartBeatIntervalS = conf.getTimeAsSeconds("spark.akka.heartbeat.interval", "1000s") - - val secretKey = securityManager.getSecretKey() - val isAuthOn = securityManager.isAuthenticationEnabled() - if (isAuthOn && secretKey == null) { - throw new Exception("Secret key is null with authentication on") - } - val requireCookie = if (isAuthOn) "on" else "off" - val secureCookie = if (isAuthOn) secretKey else "" - logDebug(s"In createActorSystem, requireCookie is: $requireCookie") - - val akkaSslConfig = securityManager.akkaSSLOptions.createAkkaConfig - .getOrElse(ConfigFactory.empty()) - - val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap.asJava) - .withFallback(akkaSslConfig).withFallback(ConfigFactory.parseString( - s""" - |akka.daemonic = on - |akka.loggers = [""akka.event.slf4j.Slf4jLogger""] - |akka.stdout-loglevel = "ERROR" - |akka.jvm-exit-on-fatal-error = off - |akka.remote.require-cookie = "$requireCookie" - |akka.remote.secure-cookie = "$secureCookie" - |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatIntervalS s - |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPausesS s - |akka.actor.provider = "akka.remote.RemoteActorRefProvider" - |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" - |akka.remote.netty.tcp.hostname = "$host" - |akka.remote.netty.tcp.port = $port - |akka.remote.netty.tcp.tcp-nodelay = on - |akka.remote.netty.tcp.connection-timeout = $akkaTimeoutS s - |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}B - |akka.remote.netty.tcp.execution-pool-size = $akkaThreads - |akka.actor.default-dispatcher.throughput = $akkaBatchSize - |akka.log-config-on-start = $logAkkaConfig - |akka.remote.log-remote-lifecycle-events = $lifecycleEvents - |akka.log-dead-letters = $lifecycleEvents - |akka.log-dead-letters-during-shutdown = $lifecycleEvents - """.stripMargin)) - - val actorSystem = ActorSystem(name, akkaConf) - val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider - val boundPort = provider.getDefaultAddress.port.get - (actorSystem, boundPort) - } - - private val AKKA_MAX_FRAME_SIZE_IN_MB = Int.MaxValue / 1024 / 1024 - - /** Returns the configured max frame size for Akka messages in bytes. */ - def maxFrameSizeBytes(conf: SparkConf): Int = { - val frameSizeInMB = conf.getInt("spark.akka.frameSize", 128) - if (frameSizeInMB > AKKA_MAX_FRAME_SIZE_IN_MB) { - throw new IllegalArgumentException( - s"spark.akka.frameSize should not be greater than $AKKA_MAX_FRAME_SIZE_IN_MB MB") - } - frameSizeInMB * 1024 * 1024 - } - - /** Space reserved for extra data in an Akka message besides serialized task or task result. */ - val reservedSizeBytes = 200 * 1024 - - /** - * Send a message to the given actor and get its result within a default timeout, or - * throw a SparkException if this fails. - */ - def askWithReply[T]( - message: Any, - actor: ActorRef, - timeout: RpcTimeout): T = { - askWithReply[T](message, actor, maxAttempts = 1, retryInterval = Int.MaxValue, timeout) - } - - /** - * Send a message to the given actor and get its result within a default timeout, or - * throw a SparkException if this fails even after the specified number of retries. - */ - def askWithReply[T]( - message: Any, - actor: ActorRef, - maxAttempts: Int, - retryInterval: Long, - timeout: RpcTimeout): T = { - // TODO: Consider removing multiple attempts - if (actor == null) { - throw new SparkException(s"Error sending message [message = $message]" + - " as actor is null ") - } - var attempts = 0 - var lastException: Exception = null - while (attempts < maxAttempts) { - attempts += 1 - try { - val future = actor.ask(message)(timeout.duration) - val result = timeout.awaitResult(future) - if (result == null) { - throw new SparkException("Actor returned null") - } - return result.asInstanceOf[T] - } catch { - case ie: InterruptedException => throw ie - case e: Exception => - lastException = e - logWarning(s"Error sending message [message = $message] in $attempts attempts", e) - } - if (attempts < maxAttempts) { - Thread.sleep(retryInterval) - } - } - - throw new SparkException( - s"Error sending message [message = $message]", lastException) - } - - def makeDriverRef(name: String, conf: SparkConf, actorSystem: ActorSystem): ActorRef = { - val driverActorSystemName = SparkEnv.driverActorSystemName - val driverHost: String = conf.get("spark.driver.host", "localhost") - val driverPort: Int = conf.getInt("spark.driver.port", 7077) - Utils.checkHost(driverHost, "Expected hostname") - val url = address(protocol(actorSystem), driverActorSystemName, driverHost, driverPort, name) - val timeout = RpcUtils.lookupRpcTimeout(conf) - logInfo(s"Connecting to $name: $url") - timeout.awaitResult(actorSystem.actorSelection(url).resolveOne(timeout.duration)) - } - - def makeExecutorRef( - name: String, - conf: SparkConf, - host: String, - port: Int, - actorSystem: ActorSystem): ActorRef = { - val executorActorSystemName = SparkEnv.executorActorSystemName - Utils.checkHost(host, "Expected hostname") - val url = address(protocol(actorSystem), executorActorSystemName, host, port, name) - val timeout = RpcUtils.lookupRpcTimeout(conf) - logInfo(s"Connecting to $name: $url") - timeout.awaitResult(actorSystem.actorSelection(url).resolveOne(timeout.duration)) - } - - def protocol(actorSystem: ActorSystem): String = { - val akkaConf = actorSystem.settings.config - val sslProp = "akka.remote.netty.tcp.enable-ssl" - protocol(akkaConf.hasPath(sslProp) && akkaConf.getBoolean(sslProp)) - } - - def protocol(ssl: Boolean = false): String = { - if (ssl) { - "akka.ssl.tcp" - } else { - "akka.tcp" - } - } - - def address( - protocol: String, - systemName: String, - host: String, - port: Int, - actorName: String): String = { - s"$protocol://$systemName@$host:$port/user/$actorName" - } - -} diff --git a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala deleted file mode 100644 index 61b5a4cecddc..000000000000 --- a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala +++ /dev/null @@ -1,180 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import java.util.concurrent._ -import java.util.concurrent.atomic.AtomicBoolean - -import com.google.common.annotations.VisibleForTesting -import org.apache.spark.SparkContext - -/** - * Asynchronously passes events to registered listeners. - * - * Until `start()` is called, all posted events are only buffered. Only after this listener bus - * has started will events be actually propagated to all attached listeners. This listener bus - * is stopped when `stop()` is called, and it will drop further events after stopping. - * - * @param name name of the listener bus, will be the name of the listener thread. - * @tparam L type of listener - * @tparam E type of event - */ -private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: String) - extends ListenerBus[L, E] { - - self => - - private var sparkContext: SparkContext = null - - /* Cap the capacity of the event queue so we get an explicit error (rather than - * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ - private val EVENT_QUEUE_CAPACITY = 10000 - private val eventQueue = new LinkedBlockingQueue[E](EVENT_QUEUE_CAPACITY) - - // Indicate if `start()` is called - private val started = new AtomicBoolean(false) - // Indicate if `stop()` is called - private val stopped = new AtomicBoolean(false) - - // Indicate if we are processing some event - // Guarded by `self` - private var processingEvent = false - - // A counter that represents the number of events produced and consumed in the queue - private val eventLock = new Semaphore(0) - - private val listenerThread = new Thread(name) { - setDaemon(true) - override def run(): Unit = Utils.tryOrStopSparkContext(sparkContext) { - while (true) { - eventLock.acquire() - self.synchronized { - processingEvent = true - } - try { - val event = eventQueue.poll - if (event == null) { - // Get out of the while loop and shutdown the daemon thread - if (!stopped.get) { - throw new IllegalStateException("Polling `null` from eventQueue means" + - " the listener bus has been stopped. So `stopped` must be true") - } - return - } - postToAll(event) - } finally { - self.synchronized { - processingEvent = false - } - } - } - } - } - - /** - * Start sending events to attached listeners. - * - * This first sends out all buffered events posted before this listener bus has started, then - * listens for any additional events asynchronously while the listener bus is still running. - * This should only be called once. - * - * @param sc Used to stop the SparkContext in case the listener thread dies. - */ - def start(sc: SparkContext) { - if (started.compareAndSet(false, true)) { - sparkContext = sc - listenerThread.start() - } else { - throw new IllegalStateException(s"$name already started!") - } - } - - def post(event: E) { - if (stopped.get) { - // Drop further events to make `listenerThread` exit ASAP - logError(s"$name has already stopped! Dropping event $event") - return - } - val eventAdded = eventQueue.offer(event) - if (eventAdded) { - eventLock.release() - } else { - onDropEvent(event) - } - } - - /** - * For testing only. Wait until there are no more events in the queue, or until the specified - * time has elapsed. Throw `TimeoutException` if the specified time elapsed before the queue - * emptied. - */ - @VisibleForTesting - @throws(classOf[TimeoutException]) - def waitUntilEmpty(timeoutMillis: Long): Unit = { - val finishTime = System.currentTimeMillis + timeoutMillis - while (!queueIsEmpty) { - if (System.currentTimeMillis > finishTime) { - throw new TimeoutException( - s"The event queue is not empty after $timeoutMillis milliseconds") - } - /* Sleep rather than using wait/notify, because this is used only for testing and - * wait/notify add overhead in the general case. */ - Thread.sleep(10) - } - } - - /** - * For testing only. Return whether the listener daemon thread is still alive. - */ - @VisibleForTesting - def listenerThreadIsAlive: Boolean = listenerThread.isAlive - - /** - * Return whether the event queue is empty. - * - * The use of synchronized here guarantees that all events that once belonged to this queue - * have already been processed by all attached listeners, if this returns true. - */ - private def queueIsEmpty: Boolean = synchronized { eventQueue.isEmpty && !processingEvent } - - /** - * Stop the listener bus. It will wait until the queued events have been processed, but drop the - * new events after stopping. - */ - def stop() { - if (!started.get()) { - throw new IllegalStateException(s"Attempted to stop $name that has not yet started!") - } - if (stopped.compareAndSet(false, true)) { - // Call eventLock.release() so that listenerThread will poll `null` from `eventQueue` and know - // `stop` is called. - eventLock.release() - listenerThread.join() - } else { - // Keep quiet - } - } - - /** - * If the event queue exceeds its capacity, the new events will be dropped. The subclasses will be - * notified with the dropped events. - * - * Note: `onDropEvent` can be called in any thread. - */ - def onDropEvent(event: E): Unit -} diff --git a/core/src/main/scala/org/apache/spark/util/Benchmark.scala b/core/src/main/scala/org/apache/spark/util/Benchmark.scala new file mode 100644 index 000000000000..7def44bd2a2b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/Benchmark.scala @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.{OutputStream, PrintStream} + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.util.Try + +import org.apache.commons.io.output.TeeOutputStream +import org.apache.commons.lang3.SystemUtils + +/** + * Utility class to benchmark components. An example of how to use this is: + * val benchmark = new Benchmark("My Benchmark", valuesPerIteration) + * benchmark.addCase("V1")() + * benchmark.addCase("V2")() + * benchmark.run + * This will output the average time to run each function and the rate of each function. + * + * The benchmark function takes one argument that is the iteration that's being run. + * + * @param name name of this benchmark. + * @param valuesPerIteration number of values used in the test case, used to compute rows/s. + * @param minNumIters the min number of iterations that will be run per case, not counting warm-up. + * @param warmupTime amount of time to spend running dummy case iterations for JIT warm-up. + * @param minTime further iterations will be run for each case until this time is used up. + * @param outputPerIteration if true, the timing for each run will be printed to stdout. + * @param output optional output stream to write benchmark results to + */ +private[spark] class Benchmark( + name: String, + valuesPerIteration: Long, + minNumIters: Int = 2, + warmupTime: FiniteDuration = 2.seconds, + minTime: FiniteDuration = 2.seconds, + outputPerIteration: Boolean = false, + output: Option[OutputStream] = None) { + import Benchmark._ + val benchmarks = mutable.ArrayBuffer.empty[Benchmark.Case] + + val out = if (output.isDefined) { + new PrintStream(new TeeOutputStream(System.out, output.get)) + } else { + System.out + } + + /** + * Adds a case to run when run() is called. The given function will be run for several + * iterations to collect timing statistics. + * + * @param name of the benchmark case + * @param numIters if non-zero, forces exactly this many iterations to be run + */ + def addCase(name: String, numIters: Int = 0)(f: Int => Unit): Unit = { + addTimerCase(name, numIters) { timer => + timer.startTiming() + f(timer.iteration) + timer.stopTiming() + } + } + + /** + * Adds a case with manual timing control. When the function is run, timing does not start + * until timer.startTiming() is called within the given function. The corresponding + * timer.stopTiming() method must be called before the function returns. + * + * @param name of the benchmark case + * @param numIters if non-zero, forces exactly this many iterations to be run + */ + def addTimerCase(name: String, numIters: Int = 0)(f: Benchmark.Timer => Unit): Unit = { + benchmarks += Benchmark.Case(name, f, numIters) + } + + /** + * Runs the benchmark and outputs the results to stdout. This should be copied and added as + * a comment with the benchmark. Although the results vary from machine to machine, it should + * provide some baseline. + */ + def run(): Unit = { + require(benchmarks.nonEmpty) + // scalastyle:off + println("Running benchmark: " + name) + + val results = benchmarks.map { c => + println(" Running case: " + c.name) + measure(valuesPerIteration, c.numIters)(c.fn) + } + println + + val firstBest = results.head.bestMs + // The results are going to be processor specific so it is useful to include that. + out.println(Benchmark.getJVMOSInfo()) + out.println(Benchmark.getProcessorName()) + out.printf("%-40s %16s %12s %13s %10s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", + "Per Row(ns)", "Relative") + out.println("-" * 96) + results.zip(benchmarks).foreach { case (result, benchmark) => + out.printf("%-40s %16s %12s %13s %10s\n", + benchmark.name, + "%5.0f / %4.0f" format (result.bestMs, result.avgMs), + "%10.1f" format result.bestRate, + "%6.1f" format (1000 / result.bestRate), + "%3.1fX" format (firstBest / result.bestMs)) + } + out.println + // scalastyle:on + } + + /** + * Runs a single function `f` for iters, returning the average time the function took and + * the rate of the function. + */ + def measure(num: Long, overrideNumIters: Int)(f: Timer => Unit): Result = { + System.gc() // ensures garbage from previous cases don't impact this one + val warmupDeadline = warmupTime.fromNow + while (!warmupDeadline.isOverdue) { + f(new Benchmark.Timer(-1)) + } + val minIters = if (overrideNumIters != 0) overrideNumIters else minNumIters + val minDuration = if (overrideNumIters != 0) 0 else minTime.toNanos + val runTimes = ArrayBuffer[Long]() + var i = 0 + while (i < minIters || runTimes.sum < minDuration) { + val timer = new Benchmark.Timer(i) + f(timer) + val runTime = timer.totalTime() + runTimes += runTime + + if (outputPerIteration) { + // scalastyle:off + println(s"Iteration $i took ${runTime / 1000} microseconds") + // scalastyle:on + } + i += 1 + } + // scalastyle:off + println(s" Stopped after $i iterations, ${runTimes.sum / 1000000} ms") + // scalastyle:on + val best = runTimes.min + val avg = runTimes.sum / runTimes.size + Result(avg / 1000000.0, num / (best / 1000.0), best / 1000000.0) + } +} + +private[spark] object Benchmark { + + /** + * Object available to benchmark code to control timing e.g. to exclude set-up time. + * + * @param iteration specifies this is the nth iteration of running the benchmark case + */ + class Timer(val iteration: Int) { + private var accumulatedTime: Long = 0L + private var timeStart: Long = 0L + + def startTiming(): Unit = { + assert(timeStart == 0L, "Already started timing.") + timeStart = System.nanoTime + } + + def stopTiming(): Unit = { + assert(timeStart != 0L, "Have not started timing.") + accumulatedTime += System.nanoTime - timeStart + timeStart = 0L + } + + def totalTime(): Long = { + assert(timeStart == 0L, "Have not stopped timing.") + accumulatedTime + } + } + + case class Case(name: String, fn: Timer => Unit, numIters: Int) + case class Result(avgMs: Double, bestRate: Double, bestMs: Double) + + /** + * This should return a user helpful processor information. Getting at this depends on the OS. + * This should return something like "Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz" + */ + def getProcessorName(): String = { + val cpu = if (SystemUtils.IS_OS_MAC_OSX) { + Utils.executeAndGetOutput(Seq("/usr/sbin/sysctl", "-n", "machdep.cpu.brand_string")) + } else if (SystemUtils.IS_OS_LINUX) { + Try { + val grepPath = Utils.executeAndGetOutput(Seq("which", "grep")).stripLineEnd + Utils.executeAndGetOutput(Seq(grepPath, "-m", "1", "model name", "/proc/cpuinfo")) + .stripLineEnd.replaceFirst("model name[\\s*]:[\\s*]", "") + }.getOrElse("Unknown processor") + } else { + System.getenv("PROCESSOR_IDENTIFIER") + } + cpu + } + + /** + * This should return a user helpful JVM & OS information. + * This should return something like + * "OpenJDK 64-Bit Server VM 1.8.0_65-b17 on Linux 4.1.13-100.fc21.x86_64" + */ + def getJVMOSInfo(): String = { + val vmName = System.getProperty("java.vm.name") + val runtimeVersion = System.getProperty("java.runtime.version") + val osName = System.getProperty("os.name") + val osVersion = System.getProperty("os.version") + s"${vmName} ${runtimeVersion} on ${osName} ${osVersion}" + } +} diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index 1b2b1932e0c3..eff0aa4453f0 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -51,6 +51,10 @@ private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Orderin this } + def poll(): A = { + underlying.poll() + } + override def +=(elem1: A, elem2: A, elems: A*): this.type = { this += elem1 += elem2 ++= elems } diff --git a/core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala b/core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala index 54de4d4ee8ca..a938cb07724c 100644 --- a/core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala @@ -20,14 +20,11 @@ package org.apache.spark.util import java.io.InputStream import java.nio.ByteBuffer -import org.apache.spark.storage.BlockManager - /** - * Reads data from a ByteBuffer, and optionally cleans it up using BlockManager.dispose() - * at the end of the stream (e.g. to close a memory-mapped file). + * Reads data from a ByteBuffer. */ private[spark] -class ByteBufferInputStream(private var buffer: ByteBuffer, dispose: Boolean = false) +class ByteBufferInputStream(private var buffer: ByteBuffer) extends InputStream { override def read(): Int = { @@ -68,13 +65,10 @@ class ByteBufferInputStream(private var buffer: ByteBuffer, dispose: Boolean = f } /** - * Clean up the buffer, and potentially dispose of it using BlockManager.dispose(). + * Clean up the buffer, and potentially dispose of it using StorageUtils.dispose(). */ private def cleanUp() { if (buffer != null) { - if (dispose) { - BlockManager.dispose(buffer) - } buffer = null } } diff --git a/core/src/main/scala/org/apache/spark/util/ByteBufferOutputStream.scala b/core/src/main/scala/org/apache/spark/util/ByteBufferOutputStream.scala new file mode 100644 index 000000000000..9077b86f9ba1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/ByteBufferOutputStream.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.ByteArrayOutputStream +import java.nio.ByteBuffer + +/** + * Provide a zero-copy way to convert data in ByteArrayOutputStream to ByteBuffer + */ +private[spark] class ByteBufferOutputStream(capacity: Int) extends ByteArrayOutputStream(capacity) { + + def this() = this(32) + + def getCount(): Int = count + + private[this] var closed: Boolean = false + + override def write(b: Int): Unit = { + require(!closed, "cannot write to a closed ByteBufferOutputStream") + super.write(b) + } + + override def write(b: Array[Byte], off: Int, len: Int): Unit = { + require(!closed, "cannot write to a closed ByteBufferOutputStream") + super.write(b, off, len) + } + + override def reset(): Unit = { + require(!closed, "cannot reset a closed ByteBufferOutputStream") + super.reset() + } + + override def close(): Unit = { + if (!closed) { + super.close() + closed = true + } + } + + def toByteBuffer: ByteBuffer = { + require(closed, "can only call toByteBuffer() after ByteBufferOutputStream has been closed") + ByteBuffer.wrap(buf, 0, count) + } +} diff --git a/core/src/main/scala/org/apache/spark/util/CausedBy.scala b/core/src/main/scala/org/apache/spark/util/CausedBy.scala new file mode 100644 index 000000000000..73df446d981c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/CausedBy.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +/** + * Extractor Object for pulling out the root cause of an error. + * If the error contains no cause, it will return the error itself. + * + * Usage: + * try { + * ... + * } catch { + * case CausedBy(ex: CommitDeniedException) => ... + * } + */ +private[spark] object CausedBy { + + def unapply(e: Throwable): Option[Throwable] = { + Option(e.getCause).flatMap(cause => unapply(cause)).orElse(Some(e)) + } +} diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index 1b49dca9dc78..48a1d7b84b61 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -19,12 +19,14 @@ package org.apache.spark.util import java.io.{ByteArrayInputStream, ByteArrayOutputStream} -import scala.collection.mutable.{Map, Set} +import scala.collection.mutable.{Map, Set, Stack} +import scala.language.existentials -import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} -import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ +import org.apache.xbean.asm5.{ClassReader, ClassVisitor, MethodVisitor, Type} +import org.apache.xbean.asm5.Opcodes._ -import org.apache.spark.{Logging, SparkEnv, SparkException} +import org.apache.spark.{SparkEnv, SparkException} +import org.apache.spark.internal.Logging /** * A cleaner that renders closures serializable if they can be done so safely. @@ -76,35 +78,19 @@ private[spark] object ClosureCleaner extends Logging { */ private def getInnerClosureClasses(obj: AnyRef): List[Class[_]] = { val seen = Set[Class[_]](obj.getClass) - var stack = List[Class[_]](obj.getClass) + val stack = Stack[Class[_]](obj.getClass) while (!stack.isEmpty) { - val cr = getClassReader(stack.head) - stack = stack.tail - val set = Set[Class[_]]() + val cr = getClassReader(stack.pop()) + val set = Set.empty[Class[_]] cr.accept(new InnerClosureFinder(set), 0) for (cls <- set -- seen) { seen += cls - stack = cls :: stack + stack.push(cls) } } (seen - obj.getClass).toList } - private def createNullValue(cls: Class[_]): AnyRef = { - if (cls.isPrimitive) { - cls match { - case java.lang.Boolean.TYPE => new java.lang.Boolean(false) - case java.lang.Character.TYPE => new java.lang.Character('\u0000') - case java.lang.Void.TYPE => - // This should not happen because `Foo(void x) {}` does not compile. - throw new IllegalStateException("Unexpected void parameter in constructor") - case _ => new java.lang.Byte(0: Byte) - } - } else { - null - } - } - /** * Clean the given closure in place. * @@ -194,16 +180,18 @@ private[spark] object ClosureCleaner extends Logging { val declaredFields = func.getClass.getDeclaredFields val declaredMethods = func.getClass.getDeclaredMethods - logDebug(" + declared fields: " + declaredFields.size) - declaredFields.foreach { f => logDebug(" " + f) } - logDebug(" + declared methods: " + declaredMethods.size) - declaredMethods.foreach { m => logDebug(" " + m) } - logDebug(" + inner classes: " + innerClasses.size) - innerClasses.foreach { c => logDebug(" " + c.getName) } - logDebug(" + outer classes: " + outerClasses.size) - outerClasses.foreach { c => logDebug(" " + c.getName) } - logDebug(" + outer objects: " + outerObjects.size) - outerObjects.foreach { o => logDebug(" " + o) } + if (log.isDebugEnabled) { + logDebug(" + declared fields: " + declaredFields.size) + declaredFields.foreach { f => logDebug(" " + f) } + logDebug(" + declared methods: " + declaredMethods.size) + declaredMethods.foreach { m => logDebug(" " + m) } + logDebug(" + inner classes: " + innerClasses.size) + innerClasses.foreach { c => logDebug(" " + c.getName) } + logDebug(" + outer classes: " + outerClasses.size) + outerClasses.foreach { c => logDebug(" " + c.getName) } + logDebug(" + outer objects: " + outerObjects.size) + outerObjects.foreach { o => logDebug(" " + o) } + } // Fail fast if we detect return statements in closures getClassReader(func.getClass).accept(new ReturnStatementFinder(), 0) @@ -215,7 +203,7 @@ private[spark] object ClosureCleaner extends Logging { // Initialize accessed fields with the outer classes first // This step is needed to associate the fields to the correct classes later for (cls <- outerClasses) { - accessedFields(cls) = Set[String]() + accessedFields(cls) = Set.empty[String] } // Populate accessed fields by visiting all fields and methods accessed by this and // all of its inner closures. If transitive cleaning is enabled, this may recursively @@ -232,16 +220,24 @@ private[spark] object ClosureCleaner extends Logging { // Note that all outer objects but the outermost one (first one in this list) must be closures var outerPairs: List[(Class[_], AnyRef)] = (outerClasses zip outerObjects).reverse var parent: AnyRef = null - if (outerPairs.size > 0 && !isClosure(outerPairs.head._1)) { - // The closure is ultimately nested inside a class; keep the object of that - // class without cloning it since we don't want to clone the user's objects. - // Note that we still need to keep around the outermost object itself because - // we need it to clone its child closure later (see below). - logDebug(s" + outermost object is not a closure, so do not clone it: ${outerPairs.head}") - parent = outerPairs.head._2 // e.g. SparkContext - outerPairs = outerPairs.tail - } else if (outerPairs.size > 0) { - logDebug(s" + outermost object is a closure, so we just keep it: ${outerPairs.head}") + if (outerPairs.size > 0) { + val (outermostClass, outermostObject) = outerPairs.head + if (isClosure(outermostClass)) { + logDebug(s" + outermost object is a closure, so we clone it: ${outerPairs.head}") + } else if (outermostClass.getName.startsWith("$line")) { + // SPARK-14558: if the outermost object is a REPL line object, we should clone and clean it + // as it may carray a lot of unnecessary information, e.g. hadoop conf, spark conf, etc. + logDebug(s" + outermost object is a REPL line object, so we clone it: ${outerPairs.head}") + } else { + // The closure is ultimately nested inside a class; keep the object of that + // class without cloning it since we don't want to clone the user's objects. + // Note that we still need to keep around the outermost object itself because + // we need it to clone its child closure later (see below). + logDebug(" + outermost object is not a closure or REPL line object, so do not clone it: " + + outerPairs.head) + parent = outermostObject // e.g. SparkContext + outerPairs = outerPairs.tail + } } else { logDebug(" + there are no enclosing objects!") } @@ -325,11 +321,11 @@ private[spark] object ClosureCleaner extends Logging { private[spark] class ReturnStatementInClosureException extends SparkException("Return statements aren't allowed in Spark closures") -private class ReturnStatementFinder extends ClassVisitor(ASM4) { +private class ReturnStatementFinder extends ClassVisitor(ASM5) { override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { if (name.contains("apply")) { - new MethodVisitor(ASM4) { + new MethodVisitor(ASM5) { override def visitTypeInsn(op: Int, tp: String) { if (op == NEW && tp.contains("scala/runtime/NonLocalReturnControl")) { throw new ReturnStatementInClosureException @@ -337,7 +333,7 @@ private class ReturnStatementFinder extends ClassVisitor(ASM4) { } } } else { - new MethodVisitor(ASM4) {} + new MethodVisitor(ASM5) {} } } } @@ -361,7 +357,7 @@ private[util] class FieldAccessFinder( findTransitively: Boolean, specificMethod: Option[MethodIdentifier[_]] = None, visitedMethods: Set[MethodIdentifier[_]] = Set.empty) - extends ClassVisitor(ASM4) { + extends ClassVisitor(ASM5) { override def visitMethod( access: Int, @@ -376,7 +372,7 @@ private[util] class FieldAccessFinder( return null } - new MethodVisitor(ASM4) { + new MethodVisitor(ASM5) { override def visitFieldInsn(op: Int, owner: String, name: String, desc: String) { if (op == GETFIELD) { for (cl <- fields.keys if cl.getName == owner.replace('/', '.')) { @@ -385,7 +381,8 @@ private[util] class FieldAccessFinder( } } - override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { + override def visitMethodInsn( + op: Int, owner: String, name: String, desc: String, itf: Boolean) { for (cl <- fields.keys if cl.getName == owner.replace('/', '.')) { // Check for calls a getter method for a variable in an interpreter wrapper object. // This means that the corresponding field will be accessed, so we should save it. @@ -408,7 +405,7 @@ private[util] class FieldAccessFinder( } } -private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM4) { +private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM5) { var myName: String = null // TODO: Recursively find inner closures that we indirectly reference, e.g. @@ -423,9 +420,9 @@ private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { - new MethodVisitor(ASM4) { - override def visitMethodInsn(op: Int, owner: String, name: String, - desc: String) { + new MethodVisitor(ASM5) { + override def visitMethodInsn( + op: Int, owner: String, name: String, desc: String, itf: Boolean) { val argTypes = Type.getArgumentTypes(desc) if (op == INVOKESPECIAL && name == "" && argTypes.length > 0 && argTypes(0).toString.startsWith("L") // is it an object? diff --git a/core/src/main/scala/org/apache/spark/util/CommandLineUtils.scala b/core/src/main/scala/org/apache/spark/util/CommandLineUtils.scala new file mode 100644 index 000000000000..d73901686b70 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/CommandLineUtils.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.PrintStream + +import org.apache.spark.SparkException + +/** + * Contains basic command line parsing functionality and methods to parse some common Spark CLI + * options. + */ +private[spark] trait CommandLineUtils { + + // Exposed for testing + private[spark] var exitFn: Int => Unit = (exitCode: Int) => System.exit(exitCode) + + private[spark] var printStream: PrintStream = System.err + + // scalastyle:off println + + 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(1) + } + + // scalastyle:on println + + private[spark] def parseSparkConfProperty(pair: String): (String, String) = { + pair.split("=", 2).toSeq match { + case Seq(k, v) => (k, v) + case _ => printErrorAndExit(s"Spark config without '=': $pair") + throw new SparkException(s"Spark config without '=': $pair") + } + } + + def main(args: Array[String]): Unit +} diff --git a/core/src/main/scala/org/apache/spark/util/EventLoop.scala b/core/src/main/scala/org/apache/spark/util/EventLoop.scala index e9b2b8d24b47..3ea9139e1102 100644 --- a/core/src/main/scala/org/apache/spark/util/EventLoop.scala +++ b/core/src/main/scala/org/apache/spark/util/EventLoop.scala @@ -17,12 +17,12 @@ package org.apache.spark.util -import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.{BlockingQueue, LinkedBlockingDeque} +import java.util.concurrent.atomic.AtomicBoolean import scala.util.control.NonFatal -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * An event loop to receive events from the caller and process all events in the event thread. It @@ -47,13 +47,12 @@ private[spark] abstract class EventLoop[E](name: String) extends Logging { try { onReceive(event) } catch { - case NonFatal(e) => { + case NonFatal(e) => try { onError(e) } catch { case NonFatal(e) => logError("Unexpected error in " + name, e) } - } } } } catch { 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 ee2eb58cf5e2..8406826a228d 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -19,19 +19,21 @@ package org.apache.spark.util import java.util.{Properties, UUID} -import org.apache.spark.scheduler.cluster.ExecutorInfo - import scala.collection.JavaConverters._ import scala.collection.Map +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule import org.json4s.DefaultFormats -import org.json4s.JsonDSL._ import org.json4s.JsonAST._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage._ /** @@ -54,6 +56,8 @@ private[spark] object JsonProtocol { private implicit val format = DefaultFormats + private val mapper = new ObjectMapper().registerModule(DefaultScalaModule) + /** ------------------------------------------------- * * JSON serialization methods for SparkListenerEvents | * -------------------------------------------------- */ @@ -96,26 +100,27 @@ private[spark] object JsonProtocol { executorMetricsUpdateToJson(metricsUpdate) case blockUpdated: SparkListenerBlockUpdated => throw new MatchError(blockUpdated) // TODO(ekl) implement this + case _ => parse(mapper.writeValueAsString(event)) } } def stageSubmittedToJson(stageSubmitted: SparkListenerStageSubmitted): JValue = { val stageInfo = stageInfoToJson(stageSubmitted.stageInfo) val properties = propertiesToJson(stageSubmitted.properties) - ("Event" -> Utils.getFormattedClassName(stageSubmitted)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.stageSubmitted) ~ ("Stage Info" -> stageInfo) ~ ("Properties" -> properties) } def stageCompletedToJson(stageCompleted: SparkListenerStageCompleted): JValue = { val stageInfo = stageInfoToJson(stageCompleted.stageInfo) - ("Event" -> Utils.getFormattedClassName(stageCompleted)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.stageCompleted) ~ ("Stage Info" -> stageInfo) } def taskStartToJson(taskStart: SparkListenerTaskStart): JValue = { val taskInfo = taskStart.taskInfo - ("Event" -> Utils.getFormattedClassName(taskStart)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.taskStart) ~ ("Stage ID" -> taskStart.stageId) ~ ("Stage Attempt ID" -> taskStart.stageAttemptId) ~ ("Task Info" -> taskInfoToJson(taskInfo)) @@ -123,7 +128,7 @@ private[spark] object JsonProtocol { def taskGettingResultToJson(taskGettingResult: SparkListenerTaskGettingResult): JValue = { val taskInfo = taskGettingResult.taskInfo - ("Event" -> Utils.getFormattedClassName(taskGettingResult)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.taskGettingResult) ~ ("Task Info" -> taskInfoToJson(taskInfo)) } @@ -132,7 +137,7 @@ private[spark] object JsonProtocol { val taskInfo = taskEnd.taskInfo val taskMetrics = taskEnd.taskMetrics val taskMetricsJson = if (taskMetrics != null) taskMetricsToJson(taskMetrics) else JNothing - ("Event" -> Utils.getFormattedClassName(taskEnd)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.taskEnd) ~ ("Stage ID" -> taskEnd.stageId) ~ ("Stage Attempt ID" -> taskEnd.stageAttemptId) ~ ("Task Type" -> taskEnd.taskType) ~ @@ -143,7 +148,7 @@ private[spark] object JsonProtocol { def jobStartToJson(jobStart: SparkListenerJobStart): JValue = { val properties = propertiesToJson(jobStart.properties) - ("Event" -> Utils.getFormattedClassName(jobStart)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.jobStart) ~ ("Job ID" -> jobStart.jobId) ~ ("Submission Time" -> jobStart.time) ~ ("Stage Infos" -> jobStart.stageInfos.map(stageInfoToJson)) ~ // Added in Spark 1.2.0 @@ -153,7 +158,7 @@ private[spark] object JsonProtocol { def jobEndToJson(jobEnd: SparkListenerJobEnd): JValue = { val jobResult = jobResultToJson(jobEnd.jobResult) - ("Event" -> Utils.getFormattedClassName(jobEnd)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.jobEnd) ~ ("Job ID" -> jobEnd.jobId) ~ ("Completion Time" -> jobEnd.time) ~ ("Job Result" -> jobResult) @@ -165,7 +170,7 @@ private[spark] object JsonProtocol { val sparkProperties = mapToJson(environmentDetails("Spark Properties").toMap) val systemProperties = mapToJson(environmentDetails("System Properties").toMap) val classpathEntries = mapToJson(environmentDetails("Classpath Entries").toMap) - ("Event" -> Utils.getFormattedClassName(environmentUpdate)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.environmentUpdate) ~ ("JVM Information" -> jvmInformation) ~ ("Spark Properties" -> sparkProperties) ~ ("System Properties" -> systemProperties) ~ @@ -174,26 +179,28 @@ private[spark] object JsonProtocol { def blockManagerAddedToJson(blockManagerAdded: SparkListenerBlockManagerAdded): JValue = { val blockManagerId = blockManagerIdToJson(blockManagerAdded.blockManagerId) - ("Event" -> Utils.getFormattedClassName(blockManagerAdded)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.blockManagerAdded) ~ ("Block Manager ID" -> blockManagerId) ~ ("Maximum Memory" -> blockManagerAdded.maxMem) ~ - ("Timestamp" -> blockManagerAdded.time) + ("Timestamp" -> blockManagerAdded.time) ~ + ("Maximum Onheap Memory" -> blockManagerAdded.maxOnHeapMem) ~ + ("Maximum Offheap Memory" -> blockManagerAdded.maxOffHeapMem) } def blockManagerRemovedToJson(blockManagerRemoved: SparkListenerBlockManagerRemoved): JValue = { val blockManagerId = blockManagerIdToJson(blockManagerRemoved.blockManagerId) - ("Event" -> Utils.getFormattedClassName(blockManagerRemoved)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.blockManagerRemoved) ~ ("Block Manager ID" -> blockManagerId) ~ ("Timestamp" -> blockManagerRemoved.time) } def unpersistRDDToJson(unpersistRDD: SparkListenerUnpersistRDD): JValue = { - ("Event" -> Utils.getFormattedClassName(unpersistRDD)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.unpersistRDD) ~ ("RDD ID" -> unpersistRDD.rddId) } def applicationStartToJson(applicationStart: SparkListenerApplicationStart): JValue = { - ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.applicationStart) ~ ("App Name" -> applicationStart.appName) ~ ("App ID" -> applicationStart.appId.map(JString(_)).getOrElse(JNothing)) ~ ("Timestamp" -> applicationStart.time) ~ @@ -203,39 +210,39 @@ private[spark] object JsonProtocol { } def applicationEndToJson(applicationEnd: SparkListenerApplicationEnd): JValue = { - ("Event" -> Utils.getFormattedClassName(applicationEnd)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.applicationEnd) ~ ("Timestamp" -> applicationEnd.time) } def executorAddedToJson(executorAdded: SparkListenerExecutorAdded): JValue = { - ("Event" -> Utils.getFormattedClassName(executorAdded)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.executorAdded) ~ ("Timestamp" -> executorAdded.time) ~ ("Executor ID" -> executorAdded.executorId) ~ ("Executor Info" -> executorInfoToJson(executorAdded.executorInfo)) } def executorRemovedToJson(executorRemoved: SparkListenerExecutorRemoved): JValue = { - ("Event" -> Utils.getFormattedClassName(executorRemoved)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.executorRemoved) ~ ("Timestamp" -> executorRemoved.time) ~ ("Executor ID" -> executorRemoved.executorId) ~ ("Removed Reason" -> executorRemoved.reason) } def logStartToJson(logStart: SparkListenerLogStart): JValue = { - ("Event" -> Utils.getFormattedClassName(logStart)) ~ + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.logStart) ~ ("Spark Version" -> SPARK_VERSION) } def executorMetricsUpdateToJson(metricsUpdate: SparkListenerExecutorMetricsUpdate): JValue = { val execId = metricsUpdate.execId - val taskMetrics = metricsUpdate.taskMetrics - ("Event" -> Utils.getFormattedClassName(metricsUpdate)) ~ + val accumUpdates = metricsUpdate.accumUpdates + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.metricsUpdate) ~ ("Executor ID" -> execId) ~ - ("Metrics Updated" -> taskMetrics.map { case (taskId, stageId, stageAttemptId, metrics) => + ("Metrics Updated" -> accumUpdates.map { case (taskId, stageId, stageAttemptId, updates) => ("Task ID" -> taskId) ~ ("Stage ID" -> stageId) ~ ("Stage Attempt ID" -> stageAttemptId) ~ - ("Task Metrics" -> taskMetricsToJson(metrics)) + ("Accumulator Updates" -> JArray(updates.map(accumulableInfoToJson).toList)) }) } @@ -259,8 +266,7 @@ private[spark] object JsonProtocol { ("Submission Time" -> submissionTime) ~ ("Completion Time" -> completionTime) ~ ("Failure Reason" -> failureReason) ~ - ("Accumulables" -> JArray( - stageInfo.accumulables.values.map(accumulableInfoToJson).toList)) + ("Accumulables" -> accumulablesToJson(stageInfo.accumulables.values)) } def taskInfoToJson(taskInfo: TaskInfo): JValue = { @@ -275,36 +281,86 @@ private[spark] object JsonProtocol { ("Getting Result Time" -> taskInfo.gettingResultTime) ~ ("Finish Time" -> taskInfo.finishTime) ~ ("Failed" -> taskInfo.failed) ~ - ("Accumulables" -> JArray(taskInfo.accumulables.map(accumulableInfoToJson).toList)) + ("Killed" -> taskInfo.killed) ~ + ("Accumulables" -> accumulablesToJson(taskInfo.accumulables)) + } + + private lazy val accumulableBlacklist = Set("internal.metrics.updatedBlockStatuses") + + def accumulablesToJson(accumulables: Traversable[AccumulableInfo]): JArray = { + JArray(accumulables + .filterNot(_.name.exists(accumulableBlacklist.contains)) + .toList.map(accumulableInfoToJson)) } def accumulableInfoToJson(accumulableInfo: AccumulableInfo): JValue = { + val name = accumulableInfo.name ("ID" -> accumulableInfo.id) ~ - ("Name" -> accumulableInfo.name) ~ - ("Update" -> accumulableInfo.update.map(new JString(_)).getOrElse(JNothing)) ~ - ("Value" -> accumulableInfo.value) ~ - ("Internal" -> accumulableInfo.internal) + ("Name" -> name) ~ + ("Update" -> accumulableInfo.update.map { v => accumValueToJson(name, v) }) ~ + ("Value" -> accumulableInfo.value.map { v => accumValueToJson(name, v) }) ~ + ("Internal" -> accumulableInfo.internal) ~ + ("Count Failed Values" -> accumulableInfo.countFailedValues) ~ + ("Metadata" -> accumulableInfo.metadata) + } + + /** + * Serialize the value of an accumulator to JSON. + * + * For accumulators representing internal task metrics, this looks up the relevant + * [[AccumulatorParam]] to serialize the value accordingly. For all other accumulators, + * this will simply serialize the value as a string. + * + * The behavior here must match that of [[accumValueFromJson]]. Exposed for testing. + */ + private[util] def accumValueToJson(name: Option[String], value: Any): JValue = { + if (name.exists(_.startsWith(InternalAccumulator.METRICS_PREFIX))) { + value match { + case v: Int => JInt(v) + case v: Long => JInt(v) + // We only have 3 kind of internal accumulator types, so if it's not int or long, it must be + // the blocks accumulator, whose type is `java.util.List[(BlockId, BlockStatus)]` + case v => + JArray(v.asInstanceOf[java.util.List[(BlockId, BlockStatus)]].asScala.toList.map { + case (id, status) => + ("Block ID" -> id.toString) ~ + ("Status" -> blockStatusToJson(status)) + }) + } + } else { + // For all external accumulators, just use strings + JString(value.toString) + } } def taskMetricsToJson(taskMetrics: TaskMetrics): JValue = { - val shuffleReadMetrics = - taskMetrics.shuffleReadMetrics.map(shuffleReadMetricsToJson).getOrElse(JNothing) - val shuffleWriteMetrics = - taskMetrics.shuffleWriteMetrics.map(shuffleWriteMetricsToJson).getOrElse(JNothing) - val inputMetrics = - taskMetrics.inputMetrics.map(inputMetricsToJson).getOrElse(JNothing) - val outputMetrics = - taskMetrics.outputMetrics.map(outputMetricsToJson).getOrElse(JNothing) + val shuffleReadMetrics: JValue = + ("Remote Blocks Fetched" -> taskMetrics.shuffleReadMetrics.remoteBlocksFetched) ~ + ("Local Blocks Fetched" -> taskMetrics.shuffleReadMetrics.localBlocksFetched) ~ + ("Fetch Wait Time" -> taskMetrics.shuffleReadMetrics.fetchWaitTime) ~ + ("Remote Bytes Read" -> taskMetrics.shuffleReadMetrics.remoteBytesRead) ~ + ("Remote Bytes Read To Disk" -> taskMetrics.shuffleReadMetrics.remoteBytesReadToDisk) ~ + ("Local Bytes Read" -> taskMetrics.shuffleReadMetrics.localBytesRead) ~ + ("Total Records Read" -> taskMetrics.shuffleReadMetrics.recordsRead) + val shuffleWriteMetrics: JValue = + ("Shuffle Bytes Written" -> taskMetrics.shuffleWriteMetrics.bytesWritten) ~ + ("Shuffle Write Time" -> taskMetrics.shuffleWriteMetrics.writeTime) ~ + ("Shuffle Records Written" -> taskMetrics.shuffleWriteMetrics.recordsWritten) + val inputMetrics: JValue = + ("Bytes Read" -> taskMetrics.inputMetrics.bytesRead) ~ + ("Records Read" -> taskMetrics.inputMetrics.recordsRead) + val outputMetrics: JValue = + ("Bytes Written" -> taskMetrics.outputMetrics.bytesWritten) ~ + ("Records Written" -> taskMetrics.outputMetrics.recordsWritten) val updatedBlocks = - taskMetrics.updatedBlocks.map { blocks => - JArray(blocks.toList.map { case (id, status) => - ("Block ID" -> id.toString) ~ + JArray(taskMetrics.updatedBlockStatuses.toList.map { case (id, status) => + ("Block ID" -> id.toString) ~ ("Status" -> blockStatusToJson(status)) - }) - }.getOrElse(JNothing) - ("Host Name" -> taskMetrics.hostname) ~ + }) ("Executor Deserialize Time" -> taskMetrics.executorDeserializeTime) ~ + ("Executor Deserialize CPU Time" -> taskMetrics.executorDeserializeCpuTime) ~ ("Executor Run Time" -> taskMetrics.executorRunTime) ~ + ("Executor CPU Time" -> taskMetrics.executorCpuTime) ~ ("Result Size" -> taskMetrics.resultSize) ~ ("JVM GC Time" -> taskMetrics.jvmGCTime) ~ ("Result Serialization Time" -> taskMetrics.resultSerializationTime) ~ @@ -317,33 +373,6 @@ private[spark] object JsonProtocol { ("Updated Blocks" -> updatedBlocks) } - def shuffleReadMetricsToJson(shuffleReadMetrics: ShuffleReadMetrics): JValue = { - ("Remote Blocks Fetched" -> shuffleReadMetrics.remoteBlocksFetched) ~ - ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ - ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ - ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead) ~ - ("Local Bytes Read" -> shuffleReadMetrics.localBytesRead) ~ - ("Total Records Read" -> shuffleReadMetrics.recordsRead) - } - - def shuffleWriteMetricsToJson(shuffleWriteMetrics: ShuffleWriteMetrics): JValue = { - ("Shuffle Bytes Written" -> shuffleWriteMetrics.shuffleBytesWritten) ~ - ("Shuffle Write Time" -> shuffleWriteMetrics.shuffleWriteTime) ~ - ("Shuffle Records Written" -> shuffleWriteMetrics.shuffleRecordsWritten) - } - - def inputMetricsToJson(inputMetrics: InputMetrics): JValue = { - ("Data Read Method" -> inputMetrics.readMethod.toString) ~ - ("Bytes Read" -> inputMetrics.bytesRead) ~ - ("Records Read" -> inputMetrics.recordsRead) - } - - def outputMetricsToJson(outputMetrics: OutputMetrics): JValue = { - ("Data Write Method" -> outputMetrics.writeMethod.toString) ~ - ("Bytes Written" -> outputMetrics.bytesWritten) ~ - ("Records Written" -> outputMetrics.recordsWritten) - } - def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { val reason = Utils.getFormattedClassName(taskEndReason) val json: JObject = taskEndReason match { @@ -357,12 +386,12 @@ private[spark] object JsonProtocol { ("Message" -> fetchFailed.message) case exceptionFailure: ExceptionFailure => val stackTrace = stackTraceToJson(exceptionFailure.stackTrace) - val metrics = exceptionFailure.metrics.map(taskMetricsToJson).getOrElse(JNothing) + val accumUpdates = accumulablesToJson(exceptionFailure.accumUpdates) ("Class Name" -> exceptionFailure.className) ~ ("Description" -> exceptionFailure.description) ~ ("Stack Trace" -> stackTrace) ~ ("Full Stack Trace" -> exceptionFailure.fullStackTrace) ~ - ("Metrics" -> metrics) + ("Accumulator Updates" -> accumUpdates) case taskCommitDenied: TaskCommitDenied => ("Job ID" -> taskCommitDenied.jobID) ~ ("Partition ID" -> taskCommitDenied.partitionID) ~ @@ -371,6 +400,8 @@ private[spark] object JsonProtocol { ("Executor ID" -> executorId) ~ ("Exit Caused By App" -> exitCausedByApp) ~ ("Loss Reason" -> reason.map(_.toString)) + case taskKilled: TaskKilled => + ("Kill Reason" -> taskKilled.reason) case _ => Utils.emptyJson } ("Reason" -> reason) ~ json @@ -398,19 +429,18 @@ private[spark] object JsonProtocol { ("RDD ID" -> rddInfo.id) ~ ("Name" -> rddInfo.name) ~ ("Scope" -> rddInfo.scope.map(_.toJson)) ~ + ("Callsite" -> rddInfo.callSite) ~ ("Parent IDs" -> parentIds) ~ ("Storage Level" -> storageLevel) ~ ("Number of Partitions" -> rddInfo.numPartitions) ~ ("Number of Cached Partitions" -> rddInfo.numCachedPartitions) ~ ("Memory Size" -> rddInfo.memSize) ~ - ("ExternalBlockStore Size" -> rddInfo.externalBlockStoreSize) ~ ("Disk Size" -> rddInfo.diskSize) } def storageLevelToJson(storageLevel: StorageLevel): JValue = { ("Use Disk" -> storageLevel.useDisk) ~ ("Use Memory" -> storageLevel.useMemory) ~ - ("Use ExternalBlockStore" -> storageLevel.useOffHeap) ~ ("Deserialized" -> storageLevel.deserialized) ~ ("Replication" -> storageLevel.replication) } @@ -419,7 +449,6 @@ private[spark] object JsonProtocol { val storageLevel = storageLevelToJson(blockStatus.storageLevel) ("Storage Level" -> storageLevel) ~ ("Memory Size" -> blockStatus.memSize) ~ - ("ExternalBlockStore Size" -> blockStatus.externalBlockStoreSize) ~ ("Disk Size" -> blockStatus.diskSize) } @@ -468,7 +497,7 @@ private[spark] object JsonProtocol { * JSON deserialization methods for SparkListenerEvents | * ---------------------------------------------------- */ - def sparkEventFromJson(json: JValue): SparkListenerEvent = { + private object SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES { val stageSubmitted = Utils.getFormattedClassName(SparkListenerStageSubmitted) val stageCompleted = Utils.getFormattedClassName(SparkListenerStageCompleted) val taskStart = Utils.getFormattedClassName(SparkListenerTaskStart) @@ -486,6 +515,10 @@ private[spark] object JsonProtocol { val executorRemoved = Utils.getFormattedClassName(SparkListenerExecutorRemoved) val logStart = Utils.getFormattedClassName(SparkListenerLogStart) val metricsUpdate = Utils.getFormattedClassName(SparkListenerExecutorMetricsUpdate) + } + + def sparkEventFromJson(json: JValue): SparkListenerEvent = { + import SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES._ (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) @@ -505,6 +538,8 @@ private[spark] object JsonProtocol { case `executorRemoved` => executorRemovedFromJson(json) case `logStart` => logStartFromJson(json) case `metricsUpdate` => executorMetricsUpdateFromJson(json) + case other => mapper.readValue(compact(render(json)), Utils.classForName(other)) + .asInstanceOf[SparkListenerEvent] } } @@ -521,7 +556,8 @@ private[spark] object JsonProtocol { def taskStartFromJson(json: JValue): SparkListenerTaskStart = { val stageId = (json \ "Stage ID").extract[Int] - val stageAttemptId = (json \ "Stage Attempt ID").extractOpt[Int].getOrElse(0) + val stageAttemptId = + Utils.jsonOption(json \ "Stage Attempt ID").map(_.extract[Int]).getOrElse(0) val taskInfo = taskInfoFromJson(json \ "Task Info") SparkListenerTaskStart(stageId, stageAttemptId, taskInfo) } @@ -533,7 +569,8 @@ private[spark] object JsonProtocol { def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { val stageId = (json \ "Stage ID").extract[Int] - val stageAttemptId = (json \ "Stage Attempt ID").extractOpt[Int].getOrElse(0) + val stageAttemptId = + Utils.jsonOption(json \ "Stage Attempt ID").map(_.extract[Int]).getOrElse(0) val taskType = (json \ "Task Type").extract[String] val taskEndReason = taskEndReasonFromJson(json \ "Task End Reason") val taskInfo = taskInfoFromJson(json \ "Task Info") @@ -550,7 +587,9 @@ private[spark] object JsonProtocol { // The "Stage Infos" field was added in Spark 1.2.0 val stageInfos = Utils.jsonOption(json \ "Stage Infos") .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse { - stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown")) + stageIds.map { id => + new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown") + } } SparkListenerJobStart(jobId, submissionTime, stageInfos, properties) } @@ -576,7 +615,9 @@ private[spark] object JsonProtocol { val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") val maxMem = (json \ "Maximum Memory").extract[Long] val time = Utils.jsonOption(json \ "Timestamp").map(_.extract[Long]).getOrElse(-1L) - SparkListenerBlockManagerAdded(time, blockManagerId, maxMem) + val maxOnHeapMem = Utils.jsonOption(json \ "Maximum Onheap Memory").map(_.extract[Long]) + val maxOffHeapMem = Utils.jsonOption(json \ "Maximum Offheap Memory").map(_.extract[Long]) + SparkListenerBlockManagerAdded(time, blockManagerId, maxMem, maxOnHeapMem, maxOffHeapMem) } def blockManagerRemovedFromJson(json: JValue): SparkListenerBlockManagerRemoved = { @@ -624,14 +665,15 @@ private[spark] object JsonProtocol { def executorMetricsUpdateFromJson(json: JValue): SparkListenerExecutorMetricsUpdate = { val execInfo = (json \ "Executor ID").extract[String] - val taskMetrics = (json \ "Metrics Updated").extract[List[JValue]].map { json => + val accumUpdates = (json \ "Metrics Updated").extract[List[JValue]].map { json => val taskId = (json \ "Task ID").extract[Long] val stageId = (json \ "Stage ID").extract[Int] val stageAttemptId = (json \ "Stage Attempt ID").extract[Int] - val metrics = taskMetricsFromJson(json \ "Task Metrics") - (taskId, stageId, stageAttemptId, metrics) + val updates = + (json \ "Accumulator Updates").extract[List[JValue]].map(accumulableInfoFromJson) + (taskId, stageId, stageAttemptId, updates) } - SparkListenerExecutorMetricsUpdate(execInfo, taskMetrics) + SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates) } /** --------------------------------------------------------------------- * @@ -640,20 +682,22 @@ private[spark] object JsonProtocol { def stageInfoFromJson(json: JValue): StageInfo = { val stageId = (json \ "Stage ID").extract[Int] - val attemptId = (json \ "Stage Attempt ID").extractOpt[Int].getOrElse(0) + val attemptId = Utils.jsonOption(json \ "Stage Attempt ID").map(_.extract[Int]).getOrElse(0) val stageName = (json \ "Stage Name").extract[String] val numTasks = (json \ "Number of Tasks").extract[Int] val rddInfos = (json \ "RDD Info").extract[List[JValue]].map(rddInfoFromJson) val parentIds = Utils.jsonOption(json \ "Parent IDs") .map { l => l.extract[List[JValue]].map(_.extract[Int]) } .getOrElse(Seq.empty) - val details = (json \ "Details").extractOpt[String].getOrElse("") + val details = Utils.jsonOption(json \ "Details").map(_.extract[String]).getOrElse("") val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]) val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) val failureReason = Utils.jsonOption(json \ "Failure Reason").map(_.extract[String]) - val accumulatedValues = (json \ "Accumulables").extractOpt[List[JValue]] match { - case Some(values) => values.map(accumulableInfoFromJson(_)) - case None => Seq[AccumulableInfo]() + val accumulatedValues = { + Utils.jsonOption(json \ "Accumulables").map(_.extract[List[JValue]]) match { + case Some(values) => values.map(accumulableInfoFromJson) + case None => Seq.empty[AccumulableInfo] + } } val stageInfo = new StageInfo( @@ -670,18 +714,19 @@ private[spark] object JsonProtocol { def taskInfoFromJson(json: JValue): TaskInfo = { val taskId = (json \ "Task ID").extract[Long] val index = (json \ "Index").extract[Int] - val attempt = (json \ "Attempt").extractOpt[Int].getOrElse(1) + val attempt = Utils.jsonOption(json \ "Attempt").map(_.extract[Int]).getOrElse(1) val launchTime = (json \ "Launch Time").extract[Long] - val executorId = (json \ "Executor ID").extract[String] - val host = (json \ "Host").extract[String] + val executorId = (json \ "Executor ID").extract[String].intern() + val host = (json \ "Host").extract[String].intern() val taskLocality = TaskLocality.withName((json \ "Locality").extract[String]) - val speculative = (json \ "Speculative").extractOpt[Boolean].getOrElse(false) + val speculative = Utils.jsonOption(json \ "Speculative").exists(_.extract[Boolean]) val gettingResultTime = (json \ "Getting Result Time").extract[Long] val finishTime = (json \ "Finish Time").extract[Long] val failed = (json \ "Failed").extract[Boolean] - val accumulables = (json \ "Accumulables").extractOpt[Seq[JValue]] match { - case Some(values) => values.map(accumulableInfoFromJson(_)) - case None => Seq[AccumulableInfo]() + val killed = Utils.jsonOption(json \ "Killed").exists(_.extract[Boolean]) + val accumulables = Utils.jsonOption(json \ "Accumulables").map(_.extract[Seq[JValue]]) match { + case Some(values) => values.map(accumulableInfoFromJson) + case None => Seq.empty[AccumulableInfo] } val taskInfo = @@ -689,88 +734,126 @@ private[spark] object JsonProtocol { taskInfo.gettingResultTime = gettingResultTime taskInfo.finishTime = finishTime taskInfo.failed = failed - accumulables.foreach { taskInfo.accumulables += _ } + taskInfo.killed = killed + taskInfo.setAccumulables(accumulables) taskInfo } def accumulableInfoFromJson(json: JValue): AccumulableInfo = { val id = (json \ "ID").extract[Long] - val name = (json \ "Name").extract[String] - val update = Utils.jsonOption(json \ "Update").map(_.extract[String]) - val value = (json \ "Value").extract[String] - val internal = (json \ "Internal").extractOpt[Boolean].getOrElse(false) - AccumulableInfo(id, name, update, value, internal) + val name = Utils.jsonOption(json \ "Name").map(_.extract[String]) + val update = Utils.jsonOption(json \ "Update").map { v => accumValueFromJson(name, v) } + val value = Utils.jsonOption(json \ "Value").map { v => accumValueFromJson(name, v) } + val internal = Utils.jsonOption(json \ "Internal").exists(_.extract[Boolean]) + val countFailedValues = + Utils.jsonOption(json \ "Count Failed Values").exists(_.extract[Boolean]) + val metadata = Utils.jsonOption(json \ "Metadata").map(_.extract[String]) + new AccumulableInfo(id, name, update, value, internal, countFailedValues, metadata) + } + + /** + * Deserialize the value of an accumulator from JSON. + * + * For accumulators representing internal task metrics, this looks up the relevant + * [[AccumulatorParam]] to deserialize the value accordingly. For all other + * accumulators, this will simply deserialize the value as a string. + * + * The behavior here must match that of [[accumValueToJson]]. Exposed for testing. + */ + private[util] def accumValueFromJson(name: Option[String], value: JValue): Any = { + if (name.exists(_.startsWith(InternalAccumulator.METRICS_PREFIX))) { + value match { + case JInt(v) => v.toLong + case JArray(v) => + v.map { blockJson => + val id = BlockId((blockJson \ "Block ID").extract[String]) + val status = blockStatusFromJson(blockJson \ "Status") + (id, status) + }.asJava + case _ => throw new IllegalArgumentException(s"unexpected json value $value for " + + "accumulator " + name.get) + } + } else { + value.extract[String] + } } def taskMetricsFromJson(json: JValue): TaskMetrics = { + val metrics = TaskMetrics.empty if (json == JNothing) { - return TaskMetrics.empty + return metrics } - val metrics = new TaskMetrics - metrics.setHostname((json \ "Host Name").extract[String]) metrics.setExecutorDeserializeTime((json \ "Executor Deserialize Time").extract[Long]) + metrics.setExecutorDeserializeCpuTime((json \ "Executor Deserialize CPU Time") match { + case JNothing => 0 + case x => x.extract[Long] + }) metrics.setExecutorRunTime((json \ "Executor Run Time").extract[Long]) + metrics.setExecutorCpuTime((json \ "Executor CPU Time") match { + case JNothing => 0 + case x => x.extract[Long] + }) metrics.setResultSize((json \ "Result Size").extract[Long]) metrics.setJvmGCTime((json \ "JVM GC Time").extract[Long]) metrics.setResultSerializationTime((json \ "Result Serialization Time").extract[Long]) metrics.incMemoryBytesSpilled((json \ "Memory Bytes Spilled").extract[Long]) metrics.incDiskBytesSpilled((json \ "Disk Bytes Spilled").extract[Long]) - metrics.setShuffleReadMetrics( - Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson)) - metrics.shuffleWriteMetrics = - Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) - metrics.setInputMetrics( - Utils.jsonOption(json \ "Input Metrics").map(inputMetricsFromJson)) - metrics.outputMetrics = - Utils.jsonOption(json \ "Output Metrics").map(outputMetricsFromJson) - metrics.updatedBlocks = - Utils.jsonOption(json \ "Updated Blocks").map { value => - value.extract[List[JValue]].map { block => - val id = BlockId((block \ "Block ID").extract[String]) - val status = blockStatusFromJson(block \ "Status") - (id, status) - } - } - metrics - } - def shuffleReadMetricsFromJson(json: JValue): ShuffleReadMetrics = { - val metrics = new ShuffleReadMetrics - metrics.incRemoteBlocksFetched((json \ "Remote Blocks Fetched").extract[Int]) - 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.incLocalBytesRead((json \ "Local Bytes Read").extractOpt[Long].getOrElse(0)) - metrics.incRecordsRead((json \ "Total Records Read").extractOpt[Long].getOrElse(0)) - metrics - } + // Shuffle read metrics + Utils.jsonOption(json \ "Shuffle Read Metrics").foreach { readJson => + val readMetrics = metrics.createTempShuffleReadMetrics() + readMetrics.incRemoteBlocksFetched((readJson \ "Remote Blocks Fetched").extract[Int]) + readMetrics.incLocalBlocksFetched((readJson \ "Local Blocks Fetched").extract[Int]) + readMetrics.incRemoteBytesRead((readJson \ "Remote Bytes Read").extract[Long]) + Utils.jsonOption(readJson \ "Remote Bytes Read To Disk") + .foreach { v => readMetrics.incRemoteBytesReadToDisk(v.extract[Long])} + readMetrics.incLocalBytesRead( + Utils.jsonOption(readJson \ "Local Bytes Read").map(_.extract[Long]).getOrElse(0L)) + readMetrics.incFetchWaitTime((readJson \ "Fetch Wait Time").extract[Long]) + readMetrics.incRecordsRead( + Utils.jsonOption(readJson \ "Total Records Read").map(_.extract[Long]).getOrElse(0L)) + metrics.mergeShuffleReadMetrics() + } - def shuffleWriteMetricsFromJson(json: JValue): ShuffleWriteMetrics = { - val metrics = new ShuffleWriteMetrics - metrics.incShuffleBytesWritten((json \ "Shuffle Bytes Written").extract[Long]) - metrics.incShuffleWriteTime((json \ "Shuffle Write Time").extract[Long]) - metrics.setShuffleRecordsWritten((json \ "Shuffle Records Written") - .extractOpt[Long].getOrElse(0)) - metrics - } + // Shuffle write metrics + // TODO: Drop the redundant "Shuffle" since it's inconsistent with related classes. + Utils.jsonOption(json \ "Shuffle Write Metrics").foreach { writeJson => + val writeMetrics = metrics.shuffleWriteMetrics + writeMetrics.incBytesWritten((writeJson \ "Shuffle Bytes Written").extract[Long]) + writeMetrics.incRecordsWritten( + Utils.jsonOption(writeJson \ "Shuffle Records Written").map(_.extract[Long]).getOrElse(0L)) + writeMetrics.incWriteTime((writeJson \ "Shuffle Write Time").extract[Long]) + } - def inputMetricsFromJson(json: JValue): InputMetrics = { - val metrics = new InputMetrics( - DataReadMethod.withName((json \ "Data Read Method").extract[String])) - metrics.incBytesRead((json \ "Bytes Read").extract[Long]) - metrics.incRecordsRead((json \ "Records Read").extractOpt[Long].getOrElse(0)) - metrics - } + // Output metrics + Utils.jsonOption(json \ "Output Metrics").foreach { outJson => + val outputMetrics = metrics.outputMetrics + outputMetrics.setBytesWritten((outJson \ "Bytes Written").extract[Long]) + outputMetrics.setRecordsWritten( + Utils.jsonOption(outJson \ "Records Written").map(_.extract[Long]).getOrElse(0L)) + } + + // Input metrics + Utils.jsonOption(json \ "Input Metrics").foreach { inJson => + val inputMetrics = metrics.inputMetrics + inputMetrics.incBytesRead((inJson \ "Bytes Read").extract[Long]) + inputMetrics.incRecordsRead( + Utils.jsonOption(inJson \ "Records Read").map(_.extract[Long]).getOrElse(0L)) + } + + // Updated blocks + Utils.jsonOption(json \ "Updated Blocks").foreach { blocksJson => + metrics.setUpdatedBlockStatuses(blocksJson.extract[List[JValue]].map { blockJson => + val id = BlockId((blockJson \ "Block ID").extract[String]) + val status = blockStatusFromJson(blockJson \ "Status") + (id, status) + }) + } - def outputMetricsFromJson(json: JValue): OutputMetrics = { - val metrics = new OutputMetrics( - DataWriteMethod.withName((json \ "Data Write Method").extract[String])) - metrics.setBytesWritten((json \ "Bytes Written").extract[Long]) - metrics.setRecordsWritten((json \ "Records Written").extractOpt[Long].getOrElse(0)) metrics } - def taskEndReasonFromJson(json: JValue): TaskEndReason = { + private object TASK_END_REASON_FORMATTED_CLASS_NAMES { val success = Utils.getFormattedClassName(Success) val resubmitted = Utils.getFormattedClassName(Resubmitted) val fetchFailed = Utils.getFormattedClassName(FetchFailed) @@ -780,6 +863,10 @@ private[spark] object JsonProtocol { val taskCommitDenied = Utils.getFormattedClassName(TaskCommitDenied) val executorLostFailure = Utils.getFormattedClassName(ExecutorLostFailure) val unknownReason = Utils.getFormattedClassName(UnknownReason) + } + + def taskEndReasonFromJson(json: JValue): TaskEndReason = { + import TASK_END_REASON_FORMATTED_CLASS_NAMES._ (json \ "Reason").extract[String] match { case `success` => Success @@ -796,12 +883,20 @@ private[spark] object JsonProtocol { val className = (json \ "Class Name").extract[String] val description = (json \ "Description").extract[String] val stackTrace = stackTraceFromJson(json \ "Stack Trace") - val fullStackTrace = Utils.jsonOption(json \ "Full Stack Trace"). - map(_.extract[String]).orNull - val metrics = Utils.jsonOption(json \ "Metrics").map(taskMetricsFromJson) - ExceptionFailure(className, description, stackTrace, fullStackTrace, metrics, None) + val fullStackTrace = + Utils.jsonOption(json \ "Full Stack Trace").map(_.extract[String]).orNull + // Fallback on getting accumulator updates from TaskMetrics, which was logged in Spark 1.x + val accumUpdates = Utils.jsonOption(json \ "Accumulator Updates") + .map(_.extract[List[JValue]].map(accumulableInfoFromJson)) + .getOrElse(taskMetricsFromJson(json \ "Metrics").accumulators().map(acc => { + acc.toInfo(Some(acc.value), None) + })) + ExceptionFailure(className, description, stackTrace, fullStackTrace, None, accumUpdates) case `taskResultLost` => TaskResultLost - case `taskKilled` => TaskKilled + case `taskKilled` => + val killReason = Utils.jsonOption(json \ "Kill Reason") + .map(_.extract[String]).getOrElse("unknown reason") + TaskKilled(killReason) case `taskCommitDenied` => // Unfortunately, the `TaskCommitDenied` message was introduced in 1.3.0 but the JSON // de/serialization logic was not added until 1.5.1. To provide backward compatibility @@ -827,15 +922,19 @@ private[spark] object JsonProtocol { if (json == JNothing) { return null } - val executorId = (json \ "Executor ID").extract[String] - val host = (json \ "Host").extract[String] + val executorId = (json \ "Executor ID").extract[String].intern() + val host = (json \ "Host").extract[String].intern() val port = (json \ "Port").extract[Int] BlockManagerId(executorId, host, port) } - def jobResultFromJson(json: JValue): JobResult = { + private object JOB_RESULT_FORMATTED_CLASS_NAMES { val jobSucceeded = Utils.getFormattedClassName(JobSucceeded) val jobFailed = Utils.getFormattedClassName(JobFailed) + } + + def jobResultFromJson(json: JValue): JobResult = { + import JOB_RESULT_FORMATTED_CLASS_NAMES._ (json \ "Result").extract[String] match { case `jobSucceeded` => JobSucceeded @@ -851,6 +950,7 @@ private[spark] object JsonProtocol { val scope = Utils.jsonOption(json \ "Scope") .map(_.extract[String]) .map(RDDOperationScope.fromJson) + val callsite = Utils.jsonOption(json \ "Callsite").map(_.extract[String]).getOrElse("") val parentIds = Utils.jsonOption(json \ "Parent IDs") .map { l => l.extract[List[JValue]].map(_.extract[Int]) } .getOrElse(Seq.empty) @@ -858,15 +958,11 @@ private[spark] object JsonProtocol { val numPartitions = (json \ "Number of Partitions").extract[Int] val numCachedPartitions = (json \ "Number of Cached Partitions").extract[Int] val memSize = (json \ "Memory Size").extract[Long] - // fallback to tachyon for backward compatibility - val externalBlockStoreSize = (json \ "ExternalBlockStore Size").toSome - .getOrElse(json \ "Tachyon Size").extract[Long] val diskSize = (json \ "Disk Size").extract[Long] - val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel, parentIds, scope) + val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel, parentIds, callsite, scope) rddInfo.numCachedPartitions = numCachedPartitions rddInfo.memSize = memSize - rddInfo.externalBlockStoreSize = externalBlockStoreSize rddInfo.diskSize = diskSize rddInfo } @@ -874,22 +970,16 @@ private[spark] object JsonProtocol { def storageLevelFromJson(json: JValue): StorageLevel = { val useDisk = (json \ "Use Disk").extract[Boolean] val useMemory = (json \ "Use Memory").extract[Boolean] - // fallback to tachyon for backward compatability - val useExternalBlockStore = (json \ "Use ExternalBlockStore").toSome - .getOrElse(json \ "Use Tachyon").extract[Boolean] val deserialized = (json \ "Deserialized").extract[Boolean] val replication = (json \ "Replication").extract[Int] - StorageLevel(useDisk, useMemory, useExternalBlockStore, deserialized, replication) + StorageLevel(useDisk, useMemory, deserialized, replication) } def blockStatusFromJson(json: JValue): BlockStatus = { val storageLevel = storageLevelFromJson(json \ "Storage Level") val memorySize = (json \ "Memory Size").extract[Long] val diskSize = (json \ "Disk Size").extract[Long] - // fallback to tachyon for backward compatability - val externalBlockStoreSize = (json \ "ExternalBlockStore Size").toSome - .getOrElse(json \ "Tachyon Size").extract[Long] - BlockStatus(storageLevel, memorySize, diskSize, externalBlockStoreSize) + BlockStatus(storageLevel, memorySize, diskSize) } def executorInfoFromJson(json: JValue): ExecutorInfo = { diff --git a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala index 13cb516b583e..76a56298aaeb 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -23,48 +23,79 @@ import scala.collection.JavaConverters._ import scala.reflect.ClassTag import scala.util.control.NonFatal -import org.apache.spark.Logging +import com.codahale.metrics.Timer + +import org.apache.spark.internal.Logging /** * An event bus which posts events to its listeners. */ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { + private[this] val listenersPlusTimers = new CopyOnWriteArrayList[(L, Option[Timer])] + // Marked `private[spark]` for access in tests. - private[spark] val listeners = new CopyOnWriteArrayList[L] + private[spark] def listeners = listenersPlusTimers.asScala.map(_._1).asJava + + /** + * Returns a CodaHale metrics Timer for measuring the listener's event processing time. + * This method is intended to be overridden by subclasses. + */ + protected def getTimer(listener: L): Option[Timer] = None /** * Add a listener to listen events. This method is thread-safe and can be called in any thread. */ - final def addListener(listener: L) { - listeners.add(listener) + final def addListener(listener: L): Unit = { + listenersPlusTimers.add((listener, getTimer(listener))) + } + + /** + * Remove a listener and it won't receive any events. This method is thread-safe and can be called + * in any thread. + */ + final def removeListener(listener: L): Unit = { + listenersPlusTimers.asScala.find(_._1 eq listener).foreach { listenerAndTimer => + listenersPlusTimers.remove(listenerAndTimer) + } } /** * Post the event to all registered listeners. The `postToAll` caller should guarantee calling * `postToAll` in the same thread for all events. */ - final def postToAll(event: E): Unit = { + def postToAll(event: E): Unit = { // JavaConverters can create a JIterableWrapper if we use asScala. - // However, this method will be called frequently. To avoid the wrapper cost, here ewe use + // However, this method will be called frequently. To avoid the wrapper cost, here we use // Java Iterator directly. - val iter = listeners.iterator + val iter = listenersPlusTimers.iterator while (iter.hasNext) { - val listener = iter.next() + val listenerAndMaybeTimer = iter.next() + val listener = listenerAndMaybeTimer._1 + val maybeTimer = listenerAndMaybeTimer._2 + val maybeTimerContext = if (maybeTimer.isDefined) { + maybeTimer.get.time() + } else { + null + } try { - onPostEvent(listener, event) + doPostEvent(listener, event) } catch { case NonFatal(e) => logError(s"Listener ${Utils.getFormattedClassName(listener)} threw an exception", e) + } finally { + if (maybeTimerContext != null) { + maybeTimerContext.stop() + } } } } /** * Post an event to the specified listener. `onPostEvent` is guaranteed to be called in the same - * thread. + * thread for all listeners. */ - def onPostEvent(listener: L, event: E): Unit + protected def doPostEvent(listener: L, event: E): Unit private[spark] def findListenersByClass[T <: L : ClassTag](): Seq[T] = { val c = implicitly[ClassTag[T]].runtimeClass diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala deleted file mode 100644 index a8bbad086849..000000000000 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import java.util.{Timer, TimerTask} - -import org.apache.spark.{Logging, SparkConf} - -/** - * Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries) - */ -private[spark] class MetadataCleaner( - cleanerType: MetadataCleanerType.MetadataCleanerType, - cleanupFunc: (Long) => Unit, - conf: SparkConf) - extends Logging -{ - val name = cleanerType.toString - - private val delaySeconds = MetadataCleaner.getDelaySeconds(conf, cleanerType) - private val periodSeconds = math.max(10, delaySeconds / 10) - private val timer = new Timer(name + " cleanup timer", true) - - - private val task = new TimerTask { - override def run() { - try { - cleanupFunc(System.currentTimeMillis() - (delaySeconds * 1000)) - logInfo("Ran metadata cleaner for " + name) - } catch { - case e: Exception => logError("Error running cleanup task for " + name, e) - } - } - } - - if (delaySeconds > 0) { - logDebug( - "Starting metadata cleaner for " + name + " with delay of " + delaySeconds + " seconds " + - "and period of " + periodSeconds + " secs") - timer.schedule(task, delaySeconds * 1000, periodSeconds * 1000) - } - - def cancel() { - timer.cancel() - } -} - -private[spark] object MetadataCleanerType extends Enumeration { - - val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, BLOCK_MANAGER, - SHUFFLE_BLOCK_MANAGER, BROADCAST_VARS = Value - - type MetadataCleanerType = Value - - def systemProperty(which: MetadataCleanerType.MetadataCleanerType): String = { - "spark.cleaner.ttl." + which.toString - } -} - -// TODO: This mutates a Conf to set properties right now, which is kind of ugly when used in the -// initialization of StreamingContext. It's okay for users trying to configure stuff themselves. -private[spark] object MetadataCleaner { - def getDelaySeconds(conf: SparkConf): Int = { - conf.getTimeAsSeconds("spark.cleaner.ttl", "-1").toInt - } - - def getDelaySeconds( - conf: SparkConf, - cleanerType: MetadataCleanerType.MetadataCleanerType): Int = { - conf.get(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString).toInt - } - - def setDelaySeconds( - conf: SparkConf, - cleanerType: MetadataCleanerType.MetadataCleanerType, - delay: Int) { - conf.set(MetadataCleanerType.systemProperty(cleanerType), delay.toString) - } - - /** - * Set the default delay time (in seconds). - * @param conf SparkConf instance - * @param delay default delay time to set - * @param resetAll whether to reset all to default - */ - def setDelaySeconds(conf: SparkConf, delay: Int, resetAll: Boolean = true) { - conf.set("spark.cleaner.ttl", delay.toString) - if (resetAll) { - for (cleanerType <- MetadataCleanerType.values) { - System.clearProperty(MetadataCleanerType.systemProperty(cleanerType)) - } - } - } -} - diff --git a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala index 945217203be7..034826c57ef1 100644 --- a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala @@ -17,9 +17,8 @@ package org.apache.spark.util -import java.net.{URLClassLoader, URL} +import java.net.{URL, URLClassLoader} import java.util.Enumeration -import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ @@ -48,32 +47,12 @@ private[spark] class ChildFirstURLClassLoader(urls: Array[URL], parent: ClassLoa private val parentClassLoader = new ParentClassLoader(parent) - /** - * Used to implement fine-grained class loading locks similar to what is done by Java 7. This - * prevents deadlock issues when using non-hierarchical class loaders. - * - * Note that due to some issues with implementing class loaders in - * Scala, Java 7's `ClassLoader.registerAsParallelCapable` method is not called. - */ - private val locks = new ConcurrentHashMap[String, Object]() - override def loadClass(name: String, resolve: Boolean): Class[_] = { - var lock = locks.get(name) - if (lock == null) { - val newLock = new Object() - lock = locks.putIfAbsent(name, newLock) - if (lock == null) { - lock = newLock - } - } - - lock.synchronized { - try { - super.loadClass(name, resolve) - } catch { - case e: ClassNotFoundException => - parentClassLoader.loadClass(name, resolve) - } + try { + super.loadClass(name, resolve) + } catch { + case e: ClassNotFoundException => + parentClassLoader.loadClass(name, resolve) } } diff --git a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala index 73d126ff6254..c9b7493fcdc1 100644 --- a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala @@ -18,7 +18,7 @@ package org.apache.spark.util /** - * A class loader which makes some protected methods in ClassLoader accesible. + * A class loader which makes some protected methods in ClassLoader accessible. */ private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) { diff --git a/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala b/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala new file mode 100644 index 000000000000..ce06e18879a4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import scala.collection.mutable + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging +import org.apache.spark.storage.StorageLevel + + +/** + * This abstraction helps with persisting and checkpointing RDDs and types derived from RDDs + * (such as Graphs and DataFrames). In documentation, we use the phrase "Dataset" to refer to + * the distributed data type (RDD, Graph, etc.). + * + * Specifically, this abstraction automatically handles persisting and (optionally) checkpointing, + * as well as unpersisting and removing checkpoint files. + * + * Users should call update() when a new Dataset has been created, + * before the Dataset has been materialized. After updating [[PeriodicCheckpointer]], users are + * responsible for materializing the Dataset to ensure that persisting and checkpointing actually + * occur. + * + * When update() is called, this does the following: + * - Persist new Dataset (if not yet persisted), and put in queue of persisted Datasets. + * - Unpersist Datasets from queue until there are at most 3 persisted Datasets. + * - If using checkpointing and the checkpoint interval has been reached, + * - Checkpoint the new Dataset, and put in a queue of checkpointed Datasets. + * - Remove older checkpoints. + * + * WARNINGS: + * - This class should NOT be copied (since copies may conflict on which Datasets should be + * checkpointed). + * - This class removes checkpoint files once later Datasets have been checkpointed. + * However, references to the older Datasets will still return isCheckpointed = true. + * + * @param checkpointInterval Datasets will be checkpointed at this interval. + * If this interval was set as -1, then checkpointing will be disabled. + * @param sc SparkContext for the Datasets given to this checkpointer + * @tparam T Dataset type, such as RDD[Double] + */ +private[spark] abstract class PeriodicCheckpointer[T]( + val checkpointInterval: Int, + val sc: SparkContext) extends Logging { + + /** FIFO queue of past checkpointed Datasets */ + private val checkpointQueue = mutable.Queue[T]() + + /** FIFO queue of past persisted Datasets */ + private val persistedQueue = mutable.Queue[T]() + + /** Number of times [[update()]] has been called */ + private var updateCount = 0 + + /** + * Update with a new Dataset. Handle persistence and checkpointing as needed. + * Since this handles persistence and checkpointing, this should be called before the Dataset + * has been materialized. + * + * @param newData New Dataset created from previous Datasets in the lineage. + */ + def update(newData: T): Unit = { + persist(newData) + persistedQueue.enqueue(newData) + // We try to maintain 2 Datasets in persistedQueue to support the semantics of this class: + // Users should call [[update()]] when a new Dataset has been created, + // before the Dataset has been materialized. + while (persistedQueue.size > 3) { + val dataToUnpersist = persistedQueue.dequeue() + unpersist(dataToUnpersist) + } + updateCount += 1 + + // Handle checkpointing (after persisting) + if (checkpointInterval != -1 && (updateCount % checkpointInterval) == 0 + && sc.getCheckpointDir.nonEmpty) { + // Add new checkpoint before removing old checkpoints. + checkpoint(newData) + checkpointQueue.enqueue(newData) + // Remove checkpoints before the latest one. + var canDelete = true + while (checkpointQueue.size > 1 && canDelete) { + // Delete the oldest checkpoint only if the next checkpoint exists. + if (isCheckpointed(checkpointQueue.head)) { + removeCheckpointFile() + } else { + canDelete = false + } + } + } + } + + /** Checkpoint the Dataset */ + protected def checkpoint(data: T): Unit + + /** Return true iff the Dataset is checkpointed */ + protected def isCheckpointed(data: T): Boolean + + /** + * Persist the Dataset. + * Note: This should handle checking the current [[StorageLevel]] of the Dataset. + */ + protected def persist(data: T): Unit + + /** Unpersist the Dataset */ + protected def unpersist(data: T): Unit + + /** Get list of checkpoint files for this given Dataset */ + protected def getCheckpointFiles(data: T): Iterable[String] + + /** + * Call this to unpersist the Dataset. + */ + def unpersistDataSet(): Unit = { + while (persistedQueue.nonEmpty) { + val dataToUnpersist = persistedQueue.dequeue() + unpersist(dataToUnpersist) + } + } + + /** + * Call this at the end to delete any remaining checkpoint files. + */ + def deleteAllCheckpoints(): Unit = { + while (checkpointQueue.nonEmpty) { + removeCheckpointFile() + } + } + + /** + * Call this at the end to delete any remaining checkpoint files, except for the last checkpoint. + * Note that there may not be any checkpoints at all. + */ + def deleteAllCheckpointsButLast(): Unit = { + while (checkpointQueue.size > 1) { + removeCheckpointFile() + } + } + + /** + * Get all current checkpoint files. + * This is useful in combination with [[deleteAllCheckpointsButLast()]]. + */ + def getAllCheckpointFiles: Array[String] = { + checkpointQueue.flatMap(getCheckpointFiles).toArray + } + + /** + * Dequeue the oldest checkpointed Dataset, and remove its checkpoint files. + * This prints a warning but does not fail if the files cannot be removed. + */ + private def removeCheckpointFile(): Unit = { + val old = checkpointQueue.dequeue() + // Since the old checkpoint is not deleted by Spark, we manually delete it. + getCheckpointFiles(old).foreach( + PeriodicCheckpointer.removeCheckpointFile(_, sc.hadoopConfiguration)) + } +} + +private[spark] object PeriodicCheckpointer extends Logging { + + /** Delete a checkpoint file, and log a warning if deletion fails. */ + def removeCheckpointFile(checkpointFile: String, conf: Configuration): Unit = { + try { + val path = new Path(checkpointFile) + val fs = path.getFileSystem(conf) + fs.delete(path, true) + } catch { + case e: Exception => + logWarning("PeriodicCheckpointer could not remove old checkpoint file: " + + checkpointFile) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala index 7578a3b1d85f..e5cccf39f945 100644 --- a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala @@ -17,23 +17,19 @@ package org.apache.spark.util -import scala.concurrent.duration.FiniteDuration -import scala.language.postfixOps - -import org.apache.spark.{SparkEnv, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, RpcTimeout} -object RpcUtils { +private[spark] object RpcUtils { /** - * Retrieve a [[RpcEndpointRef]] which is located in the driver via its name. + * Retrieve a `RpcEndpointRef` which is located in the driver via its name. */ def makeDriverRef(name: String, conf: SparkConf, rpcEnv: RpcEnv): RpcEndpointRef = { - val driverActorSystemName = SparkEnv.driverActorSystemName val driverHost: String = conf.get("spark.driver.host", "localhost") val driverPort: Int = conf.getInt("spark.driver.port", 7077) - Utils.checkHost(driverHost, "Expected hostname") - rpcEnv.setupEndpointRef(driverActorSystemName, RpcAddress(driverHost, driverPort), name) + Utils.checkHost(driverHost) + rpcEnv.setupEndpointRef(RpcAddress(driverHost, driverPort), name) } /** Returns the configured number of times to retry connecting */ @@ -47,22 +43,24 @@ object RpcUtils { } /** Returns the default Spark timeout to use for RPC ask operations. */ - private[spark] def askRpcTimeout(conf: SparkConf): RpcTimeout = { + def askRpcTimeout(conf: SparkConf): RpcTimeout = { RpcTimeout(conf, Seq("spark.rpc.askTimeout", "spark.network.timeout"), "120s") } - @deprecated("use askRpcTimeout instead, this method was not intended to be public", "1.5.0") - def askTimeout(conf: SparkConf): FiniteDuration = { - askRpcTimeout(conf).duration - } - /** Returns the default Spark timeout to use for RPC remote endpoint lookup. */ - private[spark] def lookupRpcTimeout(conf: SparkConf): RpcTimeout = { + def lookupRpcTimeout(conf: SparkConf): RpcTimeout = { RpcTimeout(conf, Seq("spark.rpc.lookupTimeout", "spark.network.timeout"), "120s") } - @deprecated("use lookupRpcTimeout instead, this method was not intended to be public", "1.5.0") - def lookupTimeout(conf: SparkConf): FiniteDuration = { - lookupRpcTimeout(conf).duration + private val MAX_MESSAGE_SIZE_IN_MB = Int.MaxValue / 1024 / 1024 + + /** Returns the configured max message size for messages in bytes. */ + def maxMessageSizeBytes(conf: SparkConf): Int = { + val maxSizeInMB = conf.getInt("spark.rpc.message.maxSize", 128) + if (maxSizeInMB > MAX_MESSAGE_SIZE_IN_MB) { + throw new IllegalArgumentException( + s"spark.rpc.message.maxSize should not be greater than $MAX_MESSAGE_SIZE_IN_MB MB") + } + maxSizeInMB * 1024 * 1024 } } diff --git a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala index db4a8b304ec3..4001fac3c3d5 100644 --- a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala +++ b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala @@ -20,11 +20,11 @@ package org.apache.spark.util import java.io.File import java.util.PriorityQueue -import scala.util.{Failure, Success, Try} -import tachyon.client.TachyonFile +import scala.util.Try import org.apache.hadoop.fs.FileSystem -import org.apache.spark.Logging + +import org.apache.spark.internal.Logging /** * Various utility methods used by Spark. @@ -52,12 +52,14 @@ private[spark] object ShutdownHookManager extends Logging { } private val shutdownDeletePaths = new scala.collection.mutable.HashSet[String]() - private val shutdownDeleteTachyonPaths = new scala.collection.mutable.HashSet[String]() // Add a shutdown hook to delete the temp dirs when the JVM exits + logDebug("Adding shutdown hook") // force eager creation of logger addShutdownHook(TEMP_DIR_SHUTDOWN_PRIORITY) { () => logInfo("Shutdown hook called") - shutdownDeletePaths.foreach { dirPath => + // we need to materialize the paths to delete because deleteRecursively removes items from + // shutdownDeletePaths as we are traversing through it. + shutdownDeletePaths.toArray.foreach { dirPath => try { logInfo("Deleting directory " + dirPath) Utils.deleteRecursively(new File(dirPath)) @@ -75,14 +77,6 @@ private[spark] object ShutdownHookManager extends Logging { } } - // Register the tachyon path to be deleted via shutdown hook - def registerShutdownDeleteDir(tachyonfile: TachyonFile) { - val absolutePath = tachyonfile.getPath() - shutdownDeleteTachyonPaths.synchronized { - shutdownDeleteTachyonPaths += absolutePath - } - } - // Remove the path to be deleted via shutdown hook def removeShutdownDeleteDir(file: File) { val absolutePath = file.getAbsolutePath() @@ -91,14 +85,6 @@ private[spark] object ShutdownHookManager extends Logging { } } - // Remove the tachyon path to be deleted via shutdown hook - def removeShutdownDeleteDir(tachyonfile: TachyonFile) { - val absolutePath = tachyonfile.getPath() - shutdownDeleteTachyonPaths.synchronized { - shutdownDeleteTachyonPaths.remove(absolutePath) - } - } - // Is the path already registered to be deleted via a shutdown hook ? def hasShutdownDeleteDir(file: File): Boolean = { val absolutePath = file.getAbsolutePath() @@ -107,14 +93,6 @@ private[spark] object ShutdownHookManager extends Logging { } } - // Is the path already registered to be deleted via a shutdown hook ? - def hasShutdownDeleteTachyonDir(file: TachyonFile): Boolean = { - val absolutePath = file.getPath() - shutdownDeleteTachyonPaths.synchronized { - shutdownDeleteTachyonPaths.contains(absolutePath) - } - } - // Note: if file is child of some registered path, while not equal to it, then return true; // else false. This is to ensure that two shutdown hooks do not try to delete each others // paths - resulting in IOException and incomplete cleanup. @@ -131,22 +109,6 @@ private[spark] object ShutdownHookManager extends Logging { retval } - // Note: if file is child of some registered path, while not equal to it, then return true; - // else false. This is to ensure that two shutdown hooks do not try to delete each others - // paths - resulting in Exception and incomplete cleanup. - def hasRootAsShutdownDeleteDir(file: TachyonFile): Boolean = { - val absolutePath = file.getPath() - val retval = shutdownDeleteTachyonPaths.synchronized { - shutdownDeleteTachyonPaths.exists { path => - !absolutePath.equals(path) && absolutePath.startsWith(path) - } - } - if (retval) { - logInfo("path = " + file + ", already present as root for deletion.") - } - retval - } - /** * Detect whether this thread might be executing a shutdown hook. Will always return true if * the current thread is a running a shutdown hook but may spuriously return true otherwise (e.g. @@ -160,7 +122,9 @@ private[spark] object ShutdownHookManager extends Logging { val hook = new Thread { override def run() {} } + // scalastyle:off runtimeaddshutdownhook Runtime.getRuntime.addShutdownHook(hook) + // scalastyle:on runtimeaddshutdownhook Runtime.getRuntime.removeShutdownHook(hook) } catch { case ise: IllegalStateException => return true @@ -204,54 +168,40 @@ private[spark] object ShutdownHookManager extends Logging { private [util] class SparkShutdownHookManager { private val hooks = new PriorityQueue[SparkShutdownHook]() - private var shuttingDown = false + @volatile private var shuttingDown = false /** - * Install a hook to run at shutdown and run all registered hooks in order. Hadoop 1.x does not - * have `ShutdownHookManager`, so in that case we just use the JVM's `Runtime` object and hope for - * the best. + * Install a hook to run at shutdown and run all registered hooks in order. */ def install(): Unit = { val hookTask = new Runnable() { override def run(): Unit = runAll() } - Try(Utils.classForName("org.apache.hadoop.util.ShutdownHookManager")) match { - case Success(shmClass) => - val fsPriority = classOf[FileSystem] - .getField("SHUTDOWN_HOOK_PRIORITY") - .get(null) // static field, the value is not used - .asInstanceOf[Int] - val shm = shmClass.getMethod("get").invoke(null) - shm.getClass().getMethod("addShutdownHook", classOf[Runnable], classOf[Int]) - .invoke(shm, hookTask, Integer.valueOf(fsPriority + 30)) - - case Failure(_) => - Runtime.getRuntime.addShutdownHook(new Thread(hookTask, "Spark Shutdown Hook")); - } + org.apache.hadoop.util.ShutdownHookManager.get().addShutdownHook( + hookTask, FileSystem.SHUTDOWN_HOOK_PRIORITY + 30) } - def runAll(): Unit = synchronized { + def runAll(): Unit = { shuttingDown = true - while (!hooks.isEmpty()) { - Try(Utils.logUncaughtExceptions(hooks.poll().run())) + var nextHook: SparkShutdownHook = null + while ({ nextHook = hooks.synchronized { hooks.poll() }; nextHook != null }) { + Try(Utils.logUncaughtExceptions(nextHook.run())) } } - def add(priority: Int, hook: () => Unit): AnyRef = synchronized { - checkState() - val hookRef = new SparkShutdownHook(priority, hook) - hooks.add(hookRef) - hookRef - } - - def remove(ref: AnyRef): Boolean = synchronized { - hooks.remove(ref) + def add(priority: Int, hook: () => Unit): AnyRef = { + hooks.synchronized { + if (shuttingDown) { + throw new IllegalStateException("Shutdown hooks cannot be modified during shutdown.") + } + val hookRef = new SparkShutdownHook(priority, hook) + hooks.add(hookRef) + hookRef + } } - private def checkState(): Unit = { - if (shuttingDown) { - throw new IllegalStateException("Shutdown hooks cannot be modified during shutdown.") - } + def remove(ref: AnyRef): Boolean = { + hooks.synchronized { hooks.remove(ref) } } } diff --git a/core/src/main/scala/org/apache/spark/util/SignalLogger.scala b/core/src/main/scala/org/apache/spark/util/SignalLogger.scala deleted file mode 100644 index f77488ef3d44..000000000000 --- a/core/src/main/scala/org/apache/spark/util/SignalLogger.scala +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import org.apache.commons.lang3.SystemUtils -import org.slf4j.Logger -import sun.misc.{Signal, SignalHandler} - -/** - * Used to log signals received. This can be very useful in debugging crashes or kills. - * - * Inspired by Colin Patrick McCabe's similar class from Hadoop. - */ -private[spark] object SignalLogger { - - private var registered = false - - /** Register a signal handler to log signals on UNIX-like systems. */ - def register(log: Logger): Unit = synchronized { - if (SystemUtils.IS_OS_UNIX) { - require(!registered, "Can't re-install the signal handlers") - registered = true - - val signals = Seq("TERM", "HUP", "INT") - for (signal <- signals) { - try { - new SignalLoggerHandler(signal, log) - } catch { - case e: Exception => log.warn("Failed to register signal handler " + signal, e) - } - } - log.info("Registered signal handlers for [" + signals.mkString(", ") + "]") - } - } -} - -private sealed class SignalLoggerHandler(name: String, log: Logger) extends SignalHandler { - - val prevHandler = Signal.handle(new Signal(name), this) - - override def handle(signal: Signal): Unit = { - log.error("RECEIVED SIGNAL " + signal.getNumber() + ": SIG" + signal.getName()) - prevHandler.handle(signal) - } -} diff --git a/core/src/main/scala/org/apache/spark/util/SignalUtils.scala b/core/src/main/scala/org/apache/spark/util/SignalUtils.scala new file mode 100644 index 000000000000..5a24965170ce --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/SignalUtils.scala @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.util.Collections + +import scala.collection.JavaConverters._ + +import org.apache.commons.lang3.SystemUtils +import org.slf4j.Logger +import sun.misc.{Signal, SignalHandler} + +import org.apache.spark.internal.Logging + +/** + * Contains utilities for working with posix signals. + */ +private[spark] object SignalUtils extends Logging { + + /** A flag to make sure we only register the logger once. */ + private var loggerRegistered = false + + /** Register a signal handler to log signals on UNIX-like systems. */ + def registerLogger(log: Logger): Unit = synchronized { + if (!loggerRegistered) { + Seq("TERM", "HUP", "INT").foreach { sig => + SignalUtils.register(sig) { + log.error("RECEIVED SIGNAL " + sig) + false + } + } + loggerRegistered = true + } + } + + /** + * Adds an action to be run when a given signal is received by this process. + * + * Note that signals are only supported on unix-like operating systems and work on a best-effort + * basis: if a signal is not available or cannot be intercepted, only a warning is emitted. + * + * All actions for a given signal are run in a separate thread. + */ + def register(signal: String)(action: => Boolean): Unit = synchronized { + if (SystemUtils.IS_OS_UNIX) { + try { + val handler = handlers.getOrElseUpdate(signal, { + logInfo("Registered signal handler for " + signal) + new ActionHandler(new Signal(signal)) + }) + handler.register(action) + } catch { + case ex: Exception => logWarning(s"Failed to register signal handler for " + signal, ex) + } + } + } + + /** + * A handler for the given signal that runs a collection of actions. + */ + private class ActionHandler(signal: Signal) extends SignalHandler { + + /** + * List of actions upon the signal; the callbacks should return true if the signal is "handled", + * i.e. should not escalate to the next callback. + */ + private val actions = Collections.synchronizedList(new java.util.LinkedList[() => Boolean]) + + // original signal handler, before this handler was attached + private val prevHandler: SignalHandler = Signal.handle(signal, this) + + /** + * Called when this handler's signal is received. Note that if the same signal is received + * before this method returns, it is escalated to the previous handler. + */ + override def handle(sig: Signal): Unit = { + // register old handler, will receive incoming signals while this handler is running + Signal.handle(signal, prevHandler) + + // Run all actions, escalate to parent handler if no action catches the signal + // (i.e. all actions return false). Note that calling `map` is to ensure that + // all actions are run, `forall` is short-circuited and will stop evaluating + // after reaching a first false predicate. + val escalate = actions.asScala.map(action => action()).forall(_ == false) + if (escalate) { + prevHandler.handle(sig) + } + + // re-register this handler + Signal.handle(signal, this) + } + + /** + * Adds an action to be run by this handler. + * @param action An action to be run when a signal is received. Return true if the signal + * should be stopped with this handler, false if it should be escalated. + */ + def register(action: => Boolean): Unit = actions.add(() => action) + } + + /** Mapping from signal to their respective handlers. */ + private val handlers = new scala.collection.mutable.HashMap[String, ActionHandler] +} diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index 23ee4eff0881..3bfdf95db84c 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -17,20 +17,34 @@ package org.apache.spark.util -import com.google.common.collect.MapMaker - import java.lang.management.ManagementFactory import java.lang.reflect.{Field, Modifier} import java.util.{IdentityHashMap, Random} -import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.ArrayBuffer import scala.runtime.ScalaRunTime -import org.apache.spark.Logging +import com.google.common.collect.MapMaker + import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.util.collection.OpenHashSet +/** + * A trait that allows a class to give [[SizeEstimator]] more accurate size estimation. + * When a class extends it, [[SizeEstimator]] will query the `estimatedSize` first. + * If `estimatedSize` does not return [[None]], [[SizeEstimator]] will use the returned size + * as the size of the object. Otherwise, [[SizeEstimator]] will do the estimation work. + * The difference between a [[KnownSizeEstimation]] and + * [[org.apache.spark.util.collection.SizeTracker]] is that, a + * [[org.apache.spark.util.collection.SizeTracker]] still uses [[SizeEstimator]] to + * estimate the size. However, a [[KnownSizeEstimation]] can provide a better estimation without + * using [[SizeEstimator]]. + */ +private[spark] trait KnownSizeEstimation { + def estimatedSize: Long +} + /** * :: DeveloperApi :: * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in @@ -137,13 +151,12 @@ object SizeEstimator extends Logging { // TODO: We could use reflection on the VMOption returned ? getVMMethod.invoke(bean, "UseCompressedOops").toString.contains("true") } catch { - case e: Exception => { + case e: Exception => // Guess whether they've enabled UseCompressedOops based on whether maxMemory < 32 GB val guess = Runtime.getRuntime.maxMemory < (32L*1024*1024*1024) val guessInWords = if (guess) "yes" else "not" logWarning("Failed to check whether UseCompressedOops is set; assuming " + guessInWords) return guess - } } } @@ -194,15 +207,23 @@ object SizeEstimator extends Logging { val cls = obj.getClass if (cls.isArray) { visitArray(obj, cls, state) + } else if (cls.getName.startsWith("scala.reflect")) { + // Many objects in the scala.reflect package reference global reflection objects which, in + // turn, reference many other large global objects. Do nothing in this case. } else if (obj.isInstanceOf[ClassLoader] || obj.isInstanceOf[Class[_]]) { // Hadoop JobConfs created in the interpreter have a ClassLoader, which greatly confuses // the size estimator since it references the whole REPL. Do nothing in this case. In // general all ClassLoaders and Classes will be shared between objects anyway. } else { - val classInfo = getClassInfo(cls) - state.size += alignSize(classInfo.shellSize) - for (field <- classInfo.pointerFields) { - state.enqueue(field.get(obj)) + obj match { + case s: KnownSizeEstimation => + state.size += s.estimatedSize + case _ => + val classInfo = getClassInfo(cls) + state.size += alignSize(classInfo.shellSize) + for (field <- classInfo.pointerFields) { + state.enqueue(field.get(obj)) + } } } } @@ -234,7 +255,7 @@ object SizeEstimator extends Logging { } else { // Estimate the size of a large array by sampling elements without replacement. // To exclude the shared objects that the array elements may link, sample twice - // and use the min one to caculate array size. + // and use the min one to calculate array size. val rand = new Random(42) val drawn = new OpenHashSet[Int](2 * ARRAY_SAMPLE_SIZE) val s1 = sampleArray(array, state, rand, drawn, length) @@ -329,7 +350,7 @@ object SizeEstimator extends Logging { // 3. consistent fields layouts throughout the hierarchy: This means we should layout // superclass first. And we can use superclass's shellSize as a starting point to layout the // other fields in this class. - // 4. class alignment: HotSpot rounds field blocks up to to HeapOopSize not 4 bytes, confirmed + // 4. class alignment: HotSpot rounds field blocks up to HeapOopSize not 4 bytes, confirmed // with Aleksey. see https://bugs.openjdk.java.net/browse/CODETOOLS-7901322 // // The real world field layout is much more complicated. There are three kinds of fields diff --git a/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala b/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala index 724818724733..e0f5af5250e7 100644 --- a/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala +++ b/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala @@ -17,26 +17,31 @@ package org.apache.spark.util -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** - * The default uncaught exception handler for Executors terminates the whole process, to avoid - * getting into a bad state indefinitely. Since Executors are relatively lightweight, it's better - * to fail fast when things go wrong. + * The default uncaught exception handler for Spark daemons. It terminates the whole process for + * any Errors, and also terminates the process for Exceptions when the exitOnException flag is true. + * + * @param exitOnUncaughtException Whether to exit the process on UncaughtException. */ -private[spark] object SparkUncaughtExceptionHandler +private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException: Boolean = true) extends Thread.UncaughtExceptionHandler with Logging { override def uncaughtException(thread: Thread, exception: Throwable) { try { - logError("Uncaught exception in thread " + thread, exception) + // Make it explicit that uncaught exceptions are thrown when container is shutting down. + // It will help users when they analyze the executor logs + val inShutdownMsg = if (ShutdownHookManager.inShutdown()) "[Container in shutdown] " else "" + val errMsg = "Uncaught exception in thread " + logError(inShutdownMsg + errMsg + thread, exception) // We may have been called from a shutdown hook. If so, we must not call System.exit(). // (If we do, we will deadlock.) if (!ShutdownHookManager.inShutdown()) { if (exception.isInstanceOf[OutOfMemoryError]) { System.exit(SparkExitCode.OOM) - } else { + } else if (exitOnUncaughtException) { System.exit(SparkExitCode.UNCAUGHT_EXCEPTION) } } diff --git a/core/src/main/scala/org/apache/spark/util/StatCounter.scala b/core/src/main/scala/org/apache/spark/util/StatCounter.scala index 8586da1996cf..1e02638591f8 100644 --- a/core/src/main/scala/org/apache/spark/util/StatCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala @@ -17,11 +17,13 @@ package org.apache.spark.util +import org.apache.spark.annotation.Since + /** * A class for tracking the statistics of a set of numbers (count, mean and variance) in a * numerically robust way. Includes support for merging two StatCounters. Based on Welford - * and Chan's [[http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance algorithms]] - * for running variance. + * and Chan's + * algorithms for running variance. * * @constructor Initialize the StatCounter with the given values. */ @@ -104,8 +106,14 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { def min: Double = minValue - /** Return the variance of the values. */ - def variance: Double = { + /** Return the population variance of the values. */ + def variance: Double = popVariance + + /** + * Return the population variance of the values. + */ + @Since("2.1.0") + def popVariance: Double = { if (n == 0) { Double.NaN } else { @@ -125,8 +133,14 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { } } - /** Return the standard deviation of the values. */ - def stdev: Double = math.sqrt(variance) + /** Return the population standard deviation of the values. */ + def stdev: Double = popStdev + + /** + * Return the population standard deviation of the values. + */ + @Since("2.1.0") + def popStdev: Double = math.sqrt(popVariance) /** * Return the sample standard deviation of the values, which corrects for bias in estimating the diff --git a/core/src/main/scala/org/apache/spark/util/TaskCompletionListener.scala b/core/src/main/scala/org/apache/spark/util/TaskCompletionListener.scala deleted file mode 100644 index c1b8bf052c0c..000000000000 --- a/core/src/main/scala/org/apache/spark/util/TaskCompletionListener.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import java.util.EventListener - -import org.apache.spark.TaskContext -import org.apache.spark.annotation.DeveloperApi - -/** - * :: DeveloperApi :: - * - * Listener providing a callback function to invoke when a task's execution completes. - */ -@DeveloperApi -trait TaskCompletionListener extends EventListener { - def onTaskCompletion(context: TaskContext) -} diff --git a/core/src/main/scala/org/apache/spark/util/TaskCompletionListenerException.scala b/core/src/main/scala/org/apache/spark/util/TaskCompletionListenerException.scala deleted file mode 100644 index f64e069cd172..000000000000 --- a/core/src/main/scala/org/apache/spark/util/TaskCompletionListenerException.scala +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -/** - * Exception thrown when there is an exception in - * executing the callback in TaskCompletionListener. - */ -private[spark] -class TaskCompletionListenerException(errorMessages: Seq[String]) extends Exception { - - override def getMessage: String = { - if (errorMessages.size == 1) { - errorMessages.head - } else { - errorMessages.zipWithIndex.map { case (msg, i) => s"Exception $i: $msg" }.mkString("\n") - } - } -} diff --git a/core/src/main/scala/org/apache/spark/util/ThreadStackTrace.scala b/core/src/main/scala/org/apache/spark/util/ThreadStackTrace.scala index d4e0ad93b966..b1217980faf1 100644 --- a/core/src/main/scala/org/apache/spark/util/ThreadStackTrace.scala +++ b/core/src/main/scala/org/apache/spark/util/ThreadStackTrace.scala @@ -24,4 +24,8 @@ private[spark] case class ThreadStackTrace( threadId: Long, threadName: String, threadState: Thread.State, - stackTrace: String) + stackTrace: String, + blockedByThreadId: Option[Long], + blockedByLock: String, + holdingLocks: Seq[String]) + diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala index 53283448c87b..81aaf79db0c1 100644 --- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala @@ -19,11 +19,15 @@ package org.apache.spark.util import java.util.concurrent._ -import scala.concurrent.{ExecutionContext, ExecutionContextExecutor} +import scala.concurrent.{Awaitable, ExecutionContext, ExecutionContextExecutor} +import scala.concurrent.duration.Duration +import scala.concurrent.forkjoin.{ForkJoinPool => SForkJoinPool, ForkJoinWorkerThread => SForkJoinWorkerThread} import scala.util.control.NonFatal import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} +import org.apache.spark.SparkException + private[spark] object ThreadUtils { private val sameThreadExecutionContext = @@ -56,10 +60,18 @@ private[spark] object ThreadUtils { * Create a cached thread pool whose max number of threads is `maxThreadNumber`. Thread names * are formatted as prefix-ID, where ID is a unique, sequentially assigned integer. */ - def newDaemonCachedThreadPool(prefix: String, maxThreadNumber: Int): ThreadPoolExecutor = { + def newDaemonCachedThreadPool( + prefix: String, maxThreadNumber: Int, keepAliveSeconds: Int = 60): ThreadPoolExecutor = { val threadFactory = namedThreadFactory(prefix) - new ThreadPoolExecutor( - 0, maxThreadNumber, 60L, TimeUnit.SECONDS, new SynchronousQueue[Runnable], threadFactory) + val threadPool = new ThreadPoolExecutor( + maxThreadNumber, // corePoolSize: the max number of threads to create before queuing the tasks + maxThreadNumber, // maximumPoolSize: because we use LinkedBlockingDeque, this one is not used + keepAliveSeconds, + TimeUnit.SECONDS, + new LinkedBlockingQueue[Runnable], + threadFactory) + threadPool.allowCoreThreadTimeOut(true) + threadPool } /** @@ -148,4 +160,71 @@ private[spark] object ThreadUtils { result } } + + /** + * Construct a new Scala ForkJoinPool with a specified max parallelism and name prefix. + */ + def newForkJoinPool(prefix: String, maxThreadNumber: Int): SForkJoinPool = { + // Custom factory to set thread names + val factory = new SForkJoinPool.ForkJoinWorkerThreadFactory { + override def newThread(pool: SForkJoinPool) = + new SForkJoinWorkerThread(pool) { + setName(prefix + "-" + super.getName) + } + } + new SForkJoinPool(maxThreadNumber, factory, + null, // handler + false // asyncMode + ) + } + + // scalastyle:off awaitresult + /** + * Preferred alternative to `Await.result()`. + * + * This method wraps and re-throws any exceptions thrown by the underlying `Await` call, ensuring + * that this thread's stack trace appears in logs. + * + * In addition, it calls `Awaitable.result` directly to avoid using `ForkJoinPool`'s + * `BlockingContext`. Codes running in the user's thread may be in a thread of Scala ForkJoinPool. + * As concurrent executions in ForkJoinPool may see some [[ThreadLocal]] value unexpectedly, this + * method basically prevents ForkJoinPool from running other tasks in the current waiting thread. + * In general, we should use this method because many places in Spark use [[ThreadLocal]] and it's + * hard to debug when [[ThreadLocal]]s leak to other tasks. + */ + @throws(classOf[SparkException]) + def awaitResult[T](awaitable: Awaitable[T], atMost: Duration): T = { + try { + // `awaitPermission` is not actually used anywhere so it's safe to pass in null here. + // See SPARK-13747. + val awaitPermission = null.asInstanceOf[scala.concurrent.CanAwait] + awaitable.result(atMost)(awaitPermission) + } catch { + // TimeoutException is thrown in the current thread, so not need to warp the exception. + case NonFatal(t) if !t.isInstanceOf[TimeoutException] => + throw new SparkException("Exception thrown in awaitResult: ", t) + } + } + // scalastyle:on awaitresult + + // scalastyle:off awaitready + /** + * Preferred alternative to `Await.ready()`. + * + * @see [[awaitResult]] + */ + @throws(classOf[SparkException]) + def awaitReady[T](awaitable: Awaitable[T], atMost: Duration): awaitable.type = { + try { + // `awaitPermission` is not actually used anywhere so it's safe to pass in null here. + // See SPARK-13747. + val awaitPermission = null.asInstanceOf[scala.concurrent.CanAwait] + awaitable.ready(atMost)(awaitPermission) + } catch { + // TimeoutException is thrown in the current thread, so not need to warp the exception. + case NonFatal(t) if !t.isInstanceOf[TimeoutException] => + throw new SparkException("Exception thrown in awaitResult: ", t) + } + } + // scalastyle:on awaitready } diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala index d7e5143c3095..32af0127bbf3 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala @@ -17,14 +17,14 @@ package org.apache.spark.util -import java.util.Set import java.util.Map.Entry +import java.util.Set import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.Logging +import org.apache.spark.internal.Logging private[spark] case class TimeStampedValue[V](value: V, timestamp: Long) diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala deleted file mode 100644 index 65efeb1f4c19..000000000000 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import java.util.concurrent.ConcurrentHashMap - -import scala.collection.JavaConverters._ -import scala.collection.mutable.Set - -private[spark] class TimeStampedHashSet[A] extends Set[A] { - val internalMap = new ConcurrentHashMap[A, Long]() - - def contains(key: A): Boolean = { - internalMap.contains(key) - } - - def iterator: Iterator[A] = { - val jIterator = internalMap.entrySet().iterator() - jIterator.asScala.map(_.getKey) - } - - override def + (elem: A): Set[A] = { - val newSet = new TimeStampedHashSet[A] - newSet ++= this - newSet += elem - newSet - } - - override def - (elem: A): Set[A] = { - val newSet = new TimeStampedHashSet[A] - newSet ++= this - newSet -= elem - newSet - } - - override def += (key: A): this.type = { - internalMap.put(key, currentTime) - this - } - - override def -= (key: A): this.type = { - internalMap.remove(key) - this - } - - override def empty: Set[A] = new TimeStampedHashSet[A]() - - override def size(): Int = internalMap.size() - - override def foreach[U](f: (A) => U): Unit = { - val iterator = internalMap.entrySet().iterator() - while(iterator.hasNext) { - f(iterator.next.getKey) - } - } - - /** - * Removes old values that have timestamp earlier than `threshTime` - */ - def clearOldValues(threshTime: Long) { - val iterator = internalMap.entrySet().iterator() - while(iterator.hasNext) { - val entry = iterator.next() - if (entry.getValue < threshTime) { - iterator.remove() - } - } - } - - private def currentTime: Long = System.currentTimeMillis() -} diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala deleted file mode 100644 index 310c0c109416..000000000000 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala +++ /dev/null @@ -1,171 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import java.lang.ref.WeakReference -import java.util.concurrent.atomic.AtomicInteger - -import scala.collection.mutable -import scala.language.implicitConversions - -import org.apache.spark.Logging - -/** - * A wrapper of TimeStampedHashMap that ensures the values are weakly referenced and timestamped. - * - * If the value is garbage collected and the weak reference is null, get() will return a - * non-existent value. These entries are removed from the map periodically (every N inserts), as - * their values are no longer strongly reachable. Further, key-value pairs whose timestamps are - * older than a particular threshold can be removed using the clearOldValues method. - * - * TimeStampedWeakValueHashMap exposes a scala.collection.mutable.Map interface, which allows it - * to be a drop-in replacement for Scala HashMaps. Internally, it uses a Java ConcurrentHashMap, - * so all operations on this HashMap are thread-safe. - * - * @param updateTimeStampOnGet Whether timestamp of a pair will be updated when it is accessed. - */ -private[spark] class TimeStampedWeakValueHashMap[A, B](updateTimeStampOnGet: Boolean = false) - extends mutable.Map[A, B]() with Logging { - - import TimeStampedWeakValueHashMap._ - - private val internalMap = new TimeStampedHashMap[A, WeakReference[B]](updateTimeStampOnGet) - private val insertCount = new AtomicInteger(0) - - /** Return a map consisting only of entries whose values are still strongly reachable. */ - private def nonNullReferenceMap = internalMap.filter { case (_, ref) => ref.get != null } - - def get(key: A): Option[B] = internalMap.get(key) - - def iterator: Iterator[(A, B)] = nonNullReferenceMap.iterator - - override def + [B1 >: B](kv: (A, B1)): mutable.Map[A, B1] = { - val newMap = new TimeStampedWeakValueHashMap[A, B1] - val oldMap = nonNullReferenceMap.asInstanceOf[mutable.Map[A, WeakReference[B1]]] - newMap.internalMap.putAll(oldMap.toMap) - newMap.internalMap += kv - newMap - } - - override def - (key: A): mutable.Map[A, B] = { - val newMap = new TimeStampedWeakValueHashMap[A, B] - newMap.internalMap.putAll(nonNullReferenceMap.toMap) - newMap.internalMap -= key - newMap - } - - override def += (kv: (A, B)): this.type = { - internalMap += kv - if (insertCount.incrementAndGet() % CLEAR_NULL_VALUES_INTERVAL == 0) { - clearNullValues() - } - this - } - - override def -= (key: A): this.type = { - internalMap -= key - this - } - - override def update(key: A, value: B): Unit = this += ((key, value)) - - override def apply(key: A): B = internalMap.apply(key) - - override def filter(p: ((A, B)) => Boolean): mutable.Map[A, B] = nonNullReferenceMap.filter(p) - - override def empty: mutable.Map[A, B] = new TimeStampedWeakValueHashMap[A, B]() - - override def size: Int = internalMap.size - - override def foreach[U](f: ((A, B)) => U): Unit = nonNullReferenceMap.foreach(f) - - def putIfAbsent(key: A, value: B): Option[B] = internalMap.putIfAbsent(key, value) - - def toMap: Map[A, B] = iterator.toMap - - /** Remove old key-value pairs with timestamps earlier than `threshTime`. */ - def clearOldValues(threshTime: Long): Unit = internalMap.clearOldValues(threshTime) - - /** Remove entries with values that are no longer strongly reachable. */ - def clearNullValues() { - val it = internalMap.getEntrySet.iterator - while (it.hasNext) { - val entry = it.next() - if (entry.getValue.value.get == null) { - logDebug("Removing key " + entry.getKey + " because it is no longer strongly reachable.") - it.remove() - } - } - } - - // For testing - - def getTimestamp(key: A): Option[Long] = { - internalMap.getTimeStampedValue(key).map(_.timestamp) - } - - def getReference(key: A): Option[WeakReference[B]] = { - internalMap.getTimeStampedValue(key).map(_.value) - } -} - -/** - * Helper methods for converting to and from WeakReferences. - */ -private object TimeStampedWeakValueHashMap { - - // Number of inserts after which entries with null references are removed - val CLEAR_NULL_VALUES_INTERVAL = 100 - - /* Implicit conversion methods to WeakReferences. */ - - implicit def toWeakReference[V](v: V): WeakReference[V] = new WeakReference[V](v) - - implicit def toWeakReferenceTuple[K, V](kv: (K, V)): (K, WeakReference[V]) = { - kv match { case (k, v) => (k, toWeakReference(v)) } - } - - implicit def toWeakReferenceFunction[K, V, R](p: ((K, V)) => R): ((K, WeakReference[V])) => R = { - (kv: (K, WeakReference[V])) => p(kv) - } - - /* Implicit conversion methods from WeakReferences. */ - - implicit def fromWeakReference[V](ref: WeakReference[V]): V = ref.get - - implicit def fromWeakReferenceOption[V](v: Option[WeakReference[V]]): Option[V] = { - v match { - case Some(ref) => Option(fromWeakReference(ref)) - case None => None - } - } - - implicit def fromWeakReferenceTuple[K, V](kv: (K, WeakReference[V])): (K, V) = { - kv match { case (k, v) => (k, fromWeakReference(v)) } - } - - implicit def fromWeakReferenceIterator[K, V]( - it: Iterator[(K, WeakReference[V])]): Iterator[(K, V)] = { - it.map(fromWeakReferenceTuple) - } - - implicit def fromWeakReferenceMap[K, V]( - map: mutable.Map[K, WeakReference[V]]) : mutable.Map[K, V] = { - mutable.Map(map.mapValues(fromWeakReference).toSeq: _*) - } -} diff --git a/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala new file mode 100644 index 000000000000..6a58ec142dd7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import javax.annotation.concurrent.GuardedBy + +/** + * A special Thread that provides "runUninterruptibly" to allow running codes without being + * interrupted by `Thread.interrupt()`. If `Thread.interrupt()` is called during runUninterruptibly + * is running, it won't set the interrupted status. Instead, setting the interrupted status will be + * deferred until it's returning from "runUninterruptibly". + * + * Note: "runUninterruptibly" should be called only in `this` thread. + */ +private[spark] class UninterruptibleThread( + target: Runnable, + name: String) extends Thread(target, name) { + + def this(name: String) { + this(null, name) + } + + /** A monitor to protect "uninterruptible" and "interrupted" */ + private val uninterruptibleLock = new Object + + /** + * Indicates if `this` thread are in the uninterruptible status. If so, interrupting + * "this" will be deferred until `this` enters into the interruptible status. + */ + @GuardedBy("uninterruptibleLock") + private var uninterruptible = false + + /** + * Indicates if we should interrupt `this` when we are leaving the uninterruptible zone. + */ + @GuardedBy("uninterruptibleLock") + private var shouldInterruptThread = false + + /** + * Run `f` uninterruptibly in `this` thread. The thread won't be interrupted before returning + * from `f`. + * + * Note: this method should be called only in `this` thread. + */ + def runUninterruptibly[T](f: => T): T = { + if (Thread.currentThread() != this) { + throw new IllegalStateException(s"Call runUninterruptibly in a wrong thread. " + + s"Expected: $this but was ${Thread.currentThread()}") + } + + if (uninterruptibleLock.synchronized { uninterruptible }) { + // We are already in the uninterruptible status. So just run "f" and return + return f + } + + uninterruptibleLock.synchronized { + // Clear the interrupted status if it's set. + shouldInterruptThread = Thread.interrupted() || shouldInterruptThread + uninterruptible = true + } + try { + f + } finally { + uninterruptibleLock.synchronized { + uninterruptible = false + if (shouldInterruptThread) { + // Recover the interrupted status + super.interrupt() + shouldInterruptThread = false + } + } + } + } + + /** + * Interrupt `this` thread if possible. If `this` is in the uninterruptible status, it won't be + * interrupted until it enters into the interruptible status. + */ + override def interrupt(): Unit = { + uninterruptibleLock.synchronized { + if (uninterruptible) { + shouldInterruptThread = true + } else { + super.interrupt() + } + } + } +} 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 5a976ee839b1..836e33c36d9a 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -18,22 +18,32 @@ package org.apache.spark.util import java.io._ -import java.lang.management.ManagementFactory +import java.lang.management.{LockInfo, ManagementFactory, MonitorInfo, ThreadInfo} +import java.math.{MathContext, RoundingMode} import java.net._ import java.nio.ByteBuffer -import java.util.{Properties, Locale, Random, UUID} +import java.nio.channels.{Channels, FileChannel} +import java.nio.charset.StandardCharsets +import java.nio.file.{Files, Paths} +import java.util.{Locale, Properties, Random, UUID} import java.util.concurrent._ +import java.util.concurrent.atomic.AtomicBoolean +import java.util.zip.GZIPInputStream import javax.net.ssl.HttpsURLConnection +import scala.annotation.tailrec import scala.collection.JavaConverters._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag -import scala.util.{Failure, Success, Try} +import scala.util.Try import scala.util.control.{ControlThrowable, NonFatal} +import scala.util.matching.Regex -import com.google.common.io.{ByteStreams, Files} +import _root_.io.netty.channel.unix.Errors.NativeIoException +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} +import com.google.common.io.{ByteStreams, Files => GFiles} import com.google.common.net.InetAddresses import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration @@ -42,12 +52,12 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.log4j.PropertyConfigurator import org.eclipse.jetty.util.MultiException import org.json4s._ - -import tachyon.TachyonURI -import tachyon.client.{TachyonFS, TachyonFile} +import org.slf4j.Logger import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} @@ -57,6 +67,7 @@ private[spark] case class CallSite(shortForm: String, longForm: String) private[spark] object CallSite { val SHORT_FORM = "callSite.short" val LONG_FORM = "callSite.long" + val empty = CallSite("", "") } /** @@ -65,6 +76,8 @@ private[spark] object CallSite { private[spark] object Utils extends Logging { val random = new Random() + private val sparkUncaughtExceptionHandler = new SparkUncaughtExceptionHandler + /** * Define a default value for driver memory here since this value is referenced across the code * base and nearly all files already use Utils.scala @@ -74,6 +87,52 @@ private[spark] object Utils extends Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 @volatile private var localRootDirs: Array[String] = null + /** + * The performance overhead of creating and logging strings for wide schemas can be large. To + * limit the impact, we bound the number of fields to include by default. This can be overridden + * by setting the 'spark.debug.maxToStringFields' conf in SparkEnv. + */ + val DEFAULT_MAX_TO_STRING_FIELDS = 25 + + private def maxNumToStringFields = { + if (SparkEnv.get != null) { + SparkEnv.get.conf.getInt("spark.debug.maxToStringFields", DEFAULT_MAX_TO_STRING_FIELDS) + } else { + DEFAULT_MAX_TO_STRING_FIELDS + } + } + + /** Whether we have warned about plan string truncation yet. */ + private val truncationWarningPrinted = new AtomicBoolean(false) + + /** + * Format a sequence with semantics similar to calling .mkString(). Any elements beyond + * maxNumToStringFields will be dropped and replaced by a "... N more fields" placeholder. + * + * @return the trimmed and formatted string. + */ + def truncatedString[T]( + seq: Seq[T], + start: String, + sep: String, + end: String, + maxNumFields: Int = maxNumToStringFields): String = { + if (seq.length > maxNumFields) { + if (truncationWarningPrinted.compareAndSet(false, true)) { + logWarning( + "Truncated the string representation of a plan since it was too large. This " + + "behavior can be adjusted by setting 'spark.debug.maxToStringFields' in SparkEnv.conf.") + } + val numFields = math.max(0, maxNumFields - 1) + seq.take(numFields).mkString( + start, sep, sep + "... " + (seq.length - numFields) + " more fields" + end) + } else { + seq.mkString(start, sep, end) + } + } + + /** Shorthand for calling truncatedString() without start or end strings. */ + def truncatedString[T](seq: Seq[T], sep: String): String = truncatedString(seq, "", sep, "") /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { @@ -177,13 +236,30 @@ private[spark] object Utils extends Logging { /** * Primitive often used when writing [[java.nio.ByteBuffer]] to [[java.io.DataOutput]] */ - def writeByteBuffer(bb: ByteBuffer, out: ObjectOutput): Unit = { + def writeByteBuffer(bb: ByteBuffer, out: DataOutput): Unit = { + if (bb.hasArray) { + out.write(bb.array(), bb.arrayOffset() + bb.position(), bb.remaining()) + } else { + val originalPosition = bb.position() + val bbval = new Array[Byte](bb.remaining()) + bb.get(bbval) + out.write(bbval) + bb.position(originalPosition) + } + } + + /** + * Primitive often used when writing [[java.nio.ByteBuffer]] to [[java.io.OutputStream]] + */ + def writeByteBuffer(bb: ByteBuffer, out: OutputStream): Unit = { if (bb.hasArray) { out.write(bb.array(), bb.arrayOffset() + bb.position(), bb.remaining()) } else { + val originalPosition = bb.position() val bbval = new Array[Byte](bb.remaining()) bb.get(bbval) out.write(bbval) + bb.position(originalPosition) } } @@ -239,45 +315,27 @@ private[spark] object Utils extends Logging { dir } - /** Copy all data from an InputStream to an OutputStream. NIO way of file stream to file stream - * copying is disabled by default unless explicitly set transferToEnabled as true, - * the parameter transferToEnabled should be configured by spark.file.transferTo = [true|false]. - */ - def copyStream(in: InputStream, - out: OutputStream, - closeStreams: Boolean = false, - transferToEnabled: Boolean = false): Long = - { - var count = 0L + /** + * Copy all data from an InputStream to an OutputStream. NIO way of file stream to file stream + * copying is disabled by default unless explicitly set transferToEnabled as true, + * the parameter transferToEnabled should be configured by spark.file.transferTo = [true|false]. + */ + def copyStream( + in: InputStream, + out: OutputStream, + closeStreams: Boolean = false, + transferToEnabled: Boolean = false): Long = { tryWithSafeFinally { if (in.isInstanceOf[FileInputStream] && out.isInstanceOf[FileOutputStream] && transferToEnabled) { // When both streams are File stream, use transferTo to improve copy performance. val inChannel = in.asInstanceOf[FileInputStream].getChannel() val outChannel = out.asInstanceOf[FileOutputStream].getChannel() - val initialPos = outChannel.position() val size = inChannel.size() - - // In case transferTo method transferred less data than we have required. - while (count < size) { - count += inChannel.transferTo(count, size - count, outChannel) - } - - // Check the position after transferTo loop to see if it is in the right position and - // give user information if not. - // Position will not be increased to the expected length after calling transferTo in - // kernel version 2.6.32, this issue can be seen in - // https://bugs.openjdk.java.net/browse/JDK-7052359 - // This will lead to stream corruption issue when using sort-based shuffle (SPARK-3948). - val finalPos = outChannel.position() - assert(finalPos == initialPos + size, - s""" - |Current position $finalPos do not equal to expected position ${initialPos + size} - |after transferTo, please check your kernel version to see if it is 2.6.32, - |this is a kernel bug which will lead to unexpected behavior when using transferTo. - |You can set spark.file.transferTo = false to disable this NIO feature. - """.stripMargin) + copyFileStreamNIO(inChannel, outChannel, 0, size) + size } else { + var count = 0L val buf = new Array[Byte](8192) var n = 0 while (n != -1) { @@ -287,8 +345,8 @@ private[spark] object Utils extends Logging { count += n } } + count } - count } { if (closeStreams) { try { @@ -300,6 +358,37 @@ private[spark] object Utils extends Logging { } } + def copyFileStreamNIO( + input: FileChannel, + output: FileChannel, + startPosition: Long, + bytesToCopy: Long): Unit = { + val initialPos = output.position() + var count = 0L + // In case transferTo method transferred less data than we have required. + while (count < bytesToCopy) { + count += input.transferTo(count + startPosition, bytesToCopy - count, output) + } + assert(count == bytesToCopy, + s"request to copy $bytesToCopy bytes, but actually copied $count bytes.") + + // Check the position after transferTo loop to see if it is in the right position and + // give user information if not. + // Position will not be increased to the expected length after calling transferTo in + // kernel version 2.6.32, this issue can be seen in + // https://bugs.openjdk.java.net/browse/JDK-7052359 + // This will lead to stream corruption issue when using sort-based shuffle (SPARK-3948). + val finalPos = output.position() + val expectedPos = initialPos + bytesToCopy + assert(finalPos == expectedPos, + s""" + |Current position $finalPos do not equal to expected position $expectedPos + |after transferTo, please check your kernel version to see if it is 2.6.32, + |this is a kernel bug which will lead to unexpected behavior when using transferTo. + |You can set spark.file.transferTo = false to disable this NIO feature. + """.stripMargin) + } + /** * Construct a URI container information used for authentication. * This also sets the default authenticator to properly negotiation the @@ -317,6 +406,30 @@ private[spark] object Utils extends Logging { } /** + * A file name may contain some invalid URI characters, such as " ". This method will convert the + * file name to a raw path accepted by `java.net.URI(String)`. + * + * Note: the file name must not contain "/" or "\" + */ + def encodeFileNameToURIRawPath(fileName: String): String = { + require(!fileName.contains("/") && !fileName.contains("\\")) + // `file` and `localhost` are not used. Just to prevent URI from parsing `fileName` as + // scheme or host. The prefix "/" is required because URI doesn't accept a relative path. + // We should remove it after we get the raw path. + new URI("file", null, "localhost", -1, "/" + fileName, null, null).getRawPath.substring(1) + } + + /** + * Get the file name from uri's raw path and decode it. If the raw path of uri ends with "/", + * return the name before the last "/". + */ + def decodeFileNameInURI(uri: URI): String = { + val rawPath = uri.getRawPath + val rawFileName = rawPath.split("/").last + new URI("file:///" + rawFileName).getPath.substring(1) + } + + /** * Download a file or directory to target directory. Supports fetching the file in a variety of * ways, including HTTP, Hadoop-compatible filesystems, and files on a standard filesystem, based * on the URL parameter. Fetching directories is only supported from Hadoop-compatible @@ -336,8 +449,8 @@ private[spark] object Utils extends Logging { securityMgr: SecurityManager, hadoopConf: Configuration, timestamp: Long, - useCache: Boolean) { - val fileName = url.split("/").last + useCache: Boolean): File = { + val fileName = decodeFileNameInURI(new URI(url)) val targetFile = new File(targetDir, fileName) val fetchCacheEnabled = conf.getBoolean("spark.files.useFetchCache", defaultValue = true) if (useCache && fetchCacheEnabled) { @@ -385,6 +498,8 @@ private[spark] object Utils extends Logging { if (isWindows) { FileUtil.chmod(targetFile.getAbsolutePath, "u+r") } + + targetFile } /** @@ -479,7 +594,7 @@ private[spark] object Utils extends Logging { // The file does not exist in the target directory. Copy or move it there. if (removeSourceFile) { - Files.move(sourceFile, destFile) + Files.move(sourceFile.toPath, destFile.toPath) } else { logInfo(s"Copying ${sourceFile.getAbsolutePath} to ${destFile.getAbsolutePath}") copyRecursive(sourceFile, destFile) @@ -497,7 +612,7 @@ private[spark] object Utils extends Logging { case (f1, f2) => filesEqualRecursive(f1, f2) } } else if (file1.isFile && file2.isFile) { - Files.equal(file1, file2) + GFiles.equal(file1, file2) } else { false } @@ -511,7 +626,7 @@ private[spark] object Utils extends Logging { val subfiles = source.listFiles() subfiles.foreach(f => copyRecursive(f, new File(dest, f.getName))) } else { - Files.copy(source, dest) + Files.copy(source.toPath, dest.toPath) } } @@ -524,17 +639,25 @@ private[spark] object Utils extends Logging { * Throws SparkException if the target file already exists and has different contents than * the requested file. */ - private def doFetchFile( + def doFetchFile( url: String, targetDir: File, filename: String, conf: SparkConf, securityMgr: SecurityManager, - hadoopConf: Configuration) { + hadoopConf: Configuration): File = { val targetFile = new File(targetDir, filename) val uri = new URI(url) val fileOverwrite = conf.getBoolean("spark.files.overwrite", defaultValue = false) Option(uri.getScheme).getOrElse("file") match { + case "spark" => + if (SparkEnv.get == null) { + throw new IllegalStateException( + "Cannot retrieve files with 'spark' scheme without an active SparkEnv.") + } + val source = SparkEnv.get.rpcEnv.openChannel(url) + val is = Channels.newInputStream(source) + downloadFile(url, is, targetFile, fileOverwrite) case "http" | "https" | "ftp" => var uc: URLConnection = null if (securityMgr.isAuthenticationEnabled()) { @@ -566,6 +689,8 @@ private[spark] object Utils extends Logging { fetchHcfsFile(path, targetDir, fs, conf, hadoopConf, fileOverwrite, filename = Some(filename)) } + + targetFile } /** @@ -599,6 +724,26 @@ private[spark] object Utils extends Logging { } } + /** + * Validate that a given URI is actually a valid URL as well. + * @param uri The URI to validate + */ + @throws[MalformedURLException]("when the URI is an invalid URL") + def validateURL(uri: URI): Unit = { + Option(uri.getScheme).getOrElse("file") match { + case "http" | "https" | "ftp" => + try { + uri.toURL + } catch { + case e: MalformedURLException => + val ex = new MalformedURLException(s"URI (${uri.toString}) is not a valid URL.") + ex.initCause(e) + throw ex + } + case _ => // will not be turned into a URL anyway + } + } + /** * Get the path of a temporary directory. Spark's local directories can be configured through * multiple settings, which are used with the following precedence: @@ -612,14 +757,16 @@ private[spark] object Utils extends Logging { * always return a single directory. */ def getLocalDir(conf: SparkConf): String = { - getOrCreateLocalRootDirs(conf)(0) + getOrCreateLocalRootDirs(conf).headOption.getOrElse { + val configuredLocalDirs = getConfiguredLocalDirs(conf) + throw new IOException( + s"Failed to get a temp directory under [${configuredLocalDirs.mkString(",")}].") + } } private[spark] def isRunningInYarnContainer(conf: SparkConf): Boolean = { // These environment variables are set by YARN. - // For Hadoop 0.23.X, we check for YARN_LOCAL_DIRS (we use this below in getYarnLocalDirs()) - // For Hadoop 2.X, we check for CONTAINER_ID. - conf.getenv("CONTAINER_ID") != null || conf.getenv("YARN_LOCAL_DIRS") != null + conf.getenv("CONTAINER_ID") != null } /** @@ -695,17 +842,12 @@ private[spark] object Utils extends Logging { logError(s"Failed to create local root dir in $root. Ignoring this directory.") None } - }.toArray + } } /** Get the Yarn approved local directories. */ private def getYarnLocalDirs(conf: SparkConf): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(conf.getenv("YARN_LOCAL_DIRS")) - .getOrElse(Option(conf.getenv("LOCAL_DIRS")) - .getOrElse("")) + val localDirs = Option(conf.getenv("LOCAL_DIRS")).getOrElse("") if (localDirs.isEmpty) { throw new Exception("Yarn Local dirs can't be empty") @@ -733,7 +875,7 @@ private[spark] object Utils extends Logging { */ def randomizeInPlace[T](arr: Array[T], rand: Random = new Random): Array[T] = { for (i <- (arr.length - 1) to 1 by -1) { - val j = rand.nextInt(i) + val j = rand.nextInt(i + 1) val tmp = arr(j) arr(j) = arr(i) arr(i) = tmp @@ -798,6 +940,13 @@ private[spark] object Utils extends Logging { customHostname = Some(hostname) } + /** + * Get the local machine's FQDN. + */ + def localCanonicalHostName(): String = { + customHostname.getOrElse(localIpAddress.getCanonicalHostName) + } + /** * Get the local machine's hostname. */ @@ -812,12 +961,13 @@ private[spark] object Utils extends Logging { customHostname.getOrElse(InetAddresses.toUriString(localIpAddress)) } - def checkHost(host: String, message: String = "") { - assert(host.indexOf(':') == -1, message) + def checkHost(host: String) { + assert(host != null && host.indexOf(':') == -1, s"Expected hostname (not IP) but got $host") } - def checkHostPort(hostPort: String, message: String = "") { - assert(hostPort.indexOf(':') != -1, message) + def checkHostPort(hostPort: String) { + assert(hostPort != null && hostPort.indexOf(':') != -1, + s"Expected host and port but got $hostPort") } // Typically, this will be of order of number of nodes in cluster @@ -865,6 +1015,15 @@ private[spark] object Utils extends Logging { } } + /** + * Lists files recursively. + */ + def recursiveList(f: File): Array[File] = { + require(f.isDirectory) + val current = f.listFiles + current ++ current.filter(_.isDirectory).flatMap(recursiveList) + } + /** * Delete a file or directory and its contents recursively. * Don't follow directories if they are symlinks. @@ -889,7 +1048,9 @@ private[spark] object Utils extends Logging { ShutdownHookManager.removeShutdownDeleteDir(file) } } finally { - if (!file.delete()) { + if (file.delete()) { + logTrace(s"${file.getAbsolutePath} has been deleted") + } else { // Delete can also fail if the file simply did not exist if (file.exists()) { throw new IOException("Failed to delete: " + file.getAbsolutePath) @@ -899,28 +1060,11 @@ private[spark] object Utils extends Logging { } } - /** - * Delete a file or directory and its contents recursively. - */ - def deleteRecursively(dir: TachyonFile, client: TachyonFS) { - if (!client.delete(new TachyonURI(dir.getPath()), true)) { - throw new IOException("Failed to delete the tachyon dir: " + dir) - } - } - /** * Check to see if file is a symbolic link. */ def isSymlink(file: File): Boolean = { - if (file == null) throw new NullPointerException("File must not be null") - if (isWindows) return false - val fileInCanonicalDir = if (file.getParent() == null) { - file - } else { - new File(file.getParentFile().getCanonicalFile(), file.getName()) - } - - !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()) + return Files.isSymbolicLink(Paths.get(file.toURI)) } /** @@ -1007,26 +1151,39 @@ private[spark] object Utils extends Logging { /** * Convert a quantity in bytes to a human-readable string such as "4.0 MB". */ - def bytesToString(size: Long): String = { + def bytesToString(size: Long): String = bytesToString(BigInt(size)) + + def bytesToString(size: BigInt): String = { + val EB = 1L << 60 + val PB = 1L << 50 val TB = 1L << 40 val GB = 1L << 30 val MB = 1L << 20 val KB = 1L << 10 - val (value, unit) = { - if (size >= 2*TB) { - (size.asInstanceOf[Double] / TB, "TB") - } else if (size >= 2*GB) { - (size.asInstanceOf[Double] / GB, "GB") - } else if (size >= 2*MB) { - (size.asInstanceOf[Double] / MB, "MB") - } else if (size >= 2*KB) { - (size.asInstanceOf[Double] / KB, "KB") - } else { - (size.asInstanceOf[Double], "B") + if (size >= BigInt(1L << 11) * EB) { + // The number is too large, show it in scientific notation. + BigDecimal(size, new MathContext(3, RoundingMode.HALF_UP)).toString() + " B" + } else { + val (value, unit) = { + if (size >= 2 * EB) { + (BigDecimal(size) / EB, "EB") + } else if (size >= 2 * PB) { + (BigDecimal(size) / PB, "PB") + } else if (size >= 2 * TB) { + (BigDecimal(size) / TB, "TB") + } else if (size >= 2 * GB) { + (BigDecimal(size) / GB, "GB") + } else if (size >= 2 * MB) { + (BigDecimal(size) / MB, "MB") + } else if (size >= 2 * KB) { + (BigDecimal(size) / KB, "KB") + } else { + (BigDecimal(size), "B") + } } + "%.1f %s".formatLocal(Locale.US, value, unit) } - "%.1f %s".formatLocal(Locale.US, value, unit) } /** @@ -1036,16 +1193,17 @@ private[spark] object Utils extends Logging { val second = 1000 val minute = 60 * second val hour = 60 * minute + val locale = Locale.US ms match { case t if t < second => - "%d ms".format(t) + "%d ms".formatLocal(locale, t) case t if t < minute => - "%.1f s".format(t.toFloat / second) + "%.1f s".formatLocal(locale, t.toFloat / second) case t if t < hour => - "%.1f m".format(t.toFloat / minute) + "%.1f m".formatLocal(locale, t.toFloat / minute) case t => - "%.2f h".format(t.toFloat / hour) + "%.2f h".formatLocal(locale, t.toFloat / hour) } } @@ -1087,9 +1245,9 @@ private[spark] object Utils extends Logging { extraEnvironment: Map[String, String] = Map.empty, redirectStderr: Boolean = true): String = { val process = executeCommand(command, workingDir, extraEnvironment, redirectStderr) - val output = new StringBuffer + val output = new StringBuilder val threadName = "read stdout for " + command(0) - def appendToOutput(s: String): Unit = output.append(s) + def appendToOutput(s: String): Unit = output.append(s).append("\n") val stdoutThread = processStreamByLine(threadName, process.getInputStream, appendToOutput) val exitCode = process.waitFor() stdoutThread.join() // Wait for it to finish reading output @@ -1130,12 +1288,12 @@ private[spark] object Utils extends Logging { block } catch { case e: ControlThrowable => throw e - case t: Throwable => SparkUncaughtExceptionHandler.uncaughtException(t) + case t: Throwable => sparkUncaughtExceptionHandler.uncaughtException(t) } } /** - * Execute a block of code that evaluates to Unit, stop SparkContext is there is any uncaught + * Execute a block of code that evaluates to Unit, stop SparkContext if there is any uncaught * exception * * NOTE: This method is to be called by the driver-side components to avoid stopping the @@ -1151,7 +1309,7 @@ private[spark] object Utils extends Logging { val currentThreadName = Thread.currentThread().getName if (sc != null) { logError(s"uncaught error in thread $currentThreadName, stopping SparkContext", t) - sc.stop() + sc.stopInNewThread() } if (!NonFatal(t)) { logError(s"throw uncaught fatal error in thread $currentThreadName", t) @@ -1160,21 +1318,6 @@ private[spark] object Utils extends Logging { } } - /** - * Execute a block of code that evaluates to Unit, re-throwing any non-fatal uncaught - * exceptions as IOException. This is used when implementing Externalizable and Serializable's - * read and write methods, since Java's serializer will not report non-IOExceptions properly; - * see SPARK-4080 for more context. - */ - def tryOrIOException(block: => Unit) { - try { - block - } catch { - case e: IOException => throw e - case NonFatal(t) => throw new IOException(t) - } - } - /** * Execute a block of code that returns a value, re-throwing any non-fatal uncaught * exceptions as IOException. This is used when implementing Externalizable and Serializable's @@ -1185,8 +1328,12 @@ private[spark] object Utils extends Logging { try { block } catch { - case e: IOException => throw e - case NonFatal(t) => throw new IOException(t) + case e: IOException => + logError("Exception encountered", e) + throw e + case NonFatal(e) => + logError("Exception encountered", e) + throw new IOException(e) } } @@ -1211,7 +1358,6 @@ private[spark] object Utils extends Logging { * exception from the original `out.write` call. */ def tryWithSafeFinally[T](block: => T)(finallyBlock: => Unit): T = { - // It would be nice to find a method on Try that did this var originalThrowable: Throwable = null try { block @@ -1225,14 +1371,55 @@ private[spark] object Utils extends Logging { try { finallyBlock } catch { - case t: Throwable => - if (originalThrowable != null) { - originalThrowable.addSuppressed(t) - logWarning(s"Suppressing exception in finally: " + t.getMessage, t) - throw originalThrowable - } else { - throw t - } + case t: Throwable if (originalThrowable != null && originalThrowable != t) => + originalThrowable.addSuppressed(t) + logWarning(s"Suppressing exception in finally: ${t.getMessage}", t) + throw originalThrowable + } + } + } + + /** + * Execute a block of code and call the failure callbacks in the catch block. If exceptions occur + * in either the catch or the finally block, they are appended to the list of suppressed + * exceptions in original exception which is then rethrown. + * + * This is primarily an issue with `catch { abort() }` or `finally { out.close() }` blocks, + * where the abort/close needs to be called to clean up `out`, but if an exception happened + * in `out.write`, it's likely `out` may be corrupted and `abort` or `out.close` will + * fail as well. This would then suppress the original/likely more meaningful + * exception from the original `out.write` call. + */ + def tryWithSafeFinallyAndFailureCallbacks[T](block: => T) + (catchBlock: => Unit = (), finallyBlock: => Unit = ()): T = { + var originalThrowable: Throwable = null + try { + block + } catch { + case cause: Throwable => + // Purposefully not using NonFatal, because even fatal exceptions + // we don't want to have our finallyBlock suppress + originalThrowable = cause + try { + logError("Aborting task", originalThrowable) + TaskContext.get().asInstanceOf[TaskContextImpl].markTaskFailed(originalThrowable) + catchBlock + } catch { + case t: Throwable => + if (originalThrowable != t) { + originalThrowable.addSuppressed(t) + logWarning(s"Suppressing exception in catch: ${t.getMessage}", t) + } + } + throw originalThrowable + } finally { + try { + finallyBlock + } catch { + case t: Throwable if (originalThrowable != null && originalThrowable != t) => + originalThrowable.addSuppressed(t) + logWarning(s"Suppressing exception in finally: ${t.getMessage}", t) + throw originalThrowable } } } @@ -1268,7 +1455,7 @@ private[spark] object Utils extends Logging { var firstUserFile = "" var firstUserLine = 0 var insideSpark = true - var callStack = new ArrayBuffer[String]() :+ "" + val callStack = new ArrayBuffer[String]() :+ "" Thread.currentThread.getStackTrace().foreach { ste: StackTraceElement => // When running under some profilers, the current stack trace might contain some bogus @@ -1286,8 +1473,12 @@ private[spark] object Utils extends Logging { } callStack(0) = ste.toString // Put last Spark method on top of the stack trace. } else { - firstUserLine = ste.getLineNumber - firstUserFile = ste.getFileName + if (ste.getFileName != null) { + firstUserFile = ste.getFileName + if (ste.getLineNumber >= 0) { + firstUserLine = ste.getLineNumber + } + } callStack += ste.toString insideSpark = false } @@ -1311,14 +1502,77 @@ private[spark] object Utils extends Logging { CallSite(shortForm, longForm) } + private val UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF = + "spark.worker.ui.compressedLogFileLengthCacheSize" + private val DEFAULT_UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE = 100 + private var compressedLogFileLengthCache: LoadingCache[String, java.lang.Long] = null + private def getCompressedLogFileLengthCache( + sparkConf: SparkConf): LoadingCache[String, java.lang.Long] = this.synchronized { + if (compressedLogFileLengthCache == null) { + val compressedLogFileLengthCacheSize = sparkConf.getInt( + UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF, + DEFAULT_UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE) + compressedLogFileLengthCache = CacheBuilder.newBuilder() + .maximumSize(compressedLogFileLengthCacheSize) + .build[String, java.lang.Long](new CacheLoader[String, java.lang.Long]() { + override def load(path: String): java.lang.Long = { + Utils.getCompressedFileLength(new File(path)) + } + }) + } + compressedLogFileLengthCache + } + + /** + * Return the file length, if the file is compressed it returns the uncompressed file length. + * It also caches the uncompressed file size to avoid repeated decompression. The cache size is + * read from workerConf. + */ + def getFileLength(file: File, workConf: SparkConf): Long = { + if (file.getName.endsWith(".gz")) { + getCompressedLogFileLengthCache(workConf).get(file.getAbsolutePath) + } else { + file.length + } + } + + /** Return uncompressed file length of a compressed file. */ + private def getCompressedFileLength(file: File): Long = { + var gzInputStream: GZIPInputStream = null + try { + // Uncompress .gz file to determine file size. + var fileSize = 0L + gzInputStream = new GZIPInputStream(new FileInputStream(file)) + val bufSize = 1024 + val buf = new Array[Byte](bufSize) + var numBytes = ByteStreams.read(gzInputStream, buf, 0, bufSize) + while (numBytes > 0) { + fileSize += numBytes + numBytes = ByteStreams.read(gzInputStream, buf, 0, bufSize) + } + fileSize + } catch { + case e: Throwable => + logError(s"Cannot get file length of ${file}", e) + throw e + } finally { + if (gzInputStream != null) { + gzInputStream.close() + } + } + } + /** Return a string containing part of a file from byte 'start' to 'end'. */ - def offsetBytes(path: String, start: Long, end: Long): String = { + def offsetBytes(path: String, length: Long, start: Long, end: Long): String = { val file = new File(path) - val length = file.length() val effectiveEnd = math.min(length, end) val effectiveStart = math.max(0, start) val buff = new Array[Byte]((effectiveEnd-effectiveStart).toInt) - val stream = new FileInputStream(file) + val stream = if (path.endsWith(".gz")) { + new GZIPInputStream(new FileInputStream(file)) + } else { + new FileInputStream(file) + } try { ByteStreams.skipFully(stream, effectiveStart) @@ -1334,8 +1588,8 @@ private[spark] object Utils extends Logging { * and `endIndex` is based on the cumulative size of all the files take in * the given order. See figure below for more details. */ - def offsetBytes(files: Seq[File], start: Long, end: Long): String = { - val fileLengths = files.map { _.length } + def offsetBytes(files: Seq[File], fileLengths: Seq[Long], start: Long, end: Long): String = { + assert(files.length == fileLengths.length) val startIndex = math.max(start, 0) val endIndex = math.min(end, fileLengths.sum) val fileToLength = files.zip(fileLengths).toMap @@ -1343,7 +1597,7 @@ private[spark] object Utils extends Logging { val stringBuffer = new StringBuffer((endIndex - startIndex).toInt) var sum = 0L - for (file <- files) { + files.zip(fileLengths).foreach { case (file, fileLength) => val startIndexOfFile = sum val endIndexOfFile = sum + fileToLength(file) logDebug(s"Processing file $file, " + @@ -1362,19 +1616,19 @@ private[spark] object Utils extends Logging { if (startIndex <= startIndexOfFile && endIndex >= endIndexOfFile) { // Case C: read the whole file - stringBuffer.append(offsetBytes(file.getAbsolutePath, 0, fileToLength(file))) + stringBuffer.append(offsetBytes(file.getAbsolutePath, fileLength, 0, fileToLength(file))) } else if (startIndex > startIndexOfFile && startIndex < endIndexOfFile) { // Case A and B: read from [start of required range] to [end of file / end of range] val effectiveStartIndex = startIndex - startIndexOfFile val effectiveEndIndex = math.min(endIndex - startIndexOfFile, fileToLength(file)) stringBuffer.append(Utils.offsetBytes( - file.getAbsolutePath, effectiveStartIndex, effectiveEndIndex)) + file.getAbsolutePath, fileLength, effectiveStartIndex, effectiveEndIndex)) } else if (endIndex > startIndexOfFile && endIndex < endIndexOfFile) { // Case D: read from [start of file] to [end of require range] val effectiveStartIndex = math.max(startIndex - startIndexOfFile, 0) val effectiveEndIndex = endIndex - startIndexOfFile stringBuffer.append(Utils.offsetBytes( - file.getAbsolutePath, effectiveStartIndex, effectiveEndIndex)) + file.getAbsolutePath, fileLength, effectiveStartIndex, effectiveEndIndex)) } sum += fileToLength(file) logDebug(s"After processing file $file, string built is ${stringBuffer.toString}") @@ -1461,7 +1715,7 @@ private[spark] object Utils extends Logging { rawMod + (if (rawMod < 0) mod else 0) } - // Handles idiosyncracies with hash (add more as required) + // Handles idiosyncrasies with hash (add more as required) // This method should be kept in sync with // org.apache.spark.network.util.JavaUtils#nonNegativeHash(). def nonNegativeHash(obj: AnyRef): Int = { @@ -1478,8 +1732,8 @@ private[spark] object Utils extends Logging { } /** - * NaN-safe version of [[java.lang.Double.compare()]] which allows NaN values to be compared - * according to semantics where NaN == NaN and NaN > any non-NaN double. + * NaN-safe version of `java.lang.Double.compare()` which allows NaN values to be compared + * according to semantics where NaN == NaN and NaN is greater than any non-NaN double. */ def nanSafeCompareDoubles(x: Double, y: Double): Int = { val xIsNan: Boolean = java.lang.Double.isNaN(x) @@ -1492,8 +1746,8 @@ private[spark] object Utils extends Logging { } /** - * NaN-safe version of [[java.lang.Float.compare()]] which allows NaN values to be compared - * according to semantics where NaN == NaN and NaN > any non-NaN float. + * NaN-safe version of `java.lang.Float.compare()` which allows NaN values to be compared + * according to semantics where NaN == NaN and NaN is greater than any non-NaN float. */ def nanSafeCompareFloats(x: Float, y: Float): Int = { val xIsNan: Boolean = java.lang.Float.isNaN(x) @@ -1505,9 +1759,11 @@ private[spark] object Utils extends Logging { else -1 } - /** Returns the system properties map that is thread-safe to iterator over. It gets the - * properties which have been set explicitly, as well as those for which only a default value - * has been defined. */ + /** + * Returns the system properties map that is thread-safe to iterator over. It gets the + * properties which have been set explicitly, as well as those for which only a default value + * has been defined. + */ def getSystemProperties: Map[String, String] = { System.getProperties.stringPropertyNames().asScala .map(key => (key, System.getProperty(key))).toMap @@ -1527,11 +1783,12 @@ private[spark] object Utils extends Logging { /** * Timing method based on iterations that permit JVM JIT optimization. + * * @param numIters number of iterations * @param f function to be executed. If prepare is not None, the running time of each call to f * must be an order of magnitude longer than one millisecond for accurate timing. * @param prepare function to be executed before each call to f. Its running time doesn't count. - * @return the total time across all iterations (not couting preparation time) + * @return the total time across all iterations (not counting preparation time) */ def timeIt(numIters: Int)(f: => Unit, prepare: Option[() => Unit] = None): Long = { if (prepare.isEmpty) { @@ -1567,30 +1824,35 @@ private[spark] object Utils extends Logging { } /** - * Creates a symlink. Note jdk1.7 has Files.createSymbolicLink but not used here - * for jdk1.6 support. Supports windows by doing copy, everything else uses "ln -sf". + * Generate a zipWithIndex iterator, avoid index value overflowing problem + * in scala's zipWithIndex + */ + def getIteratorZipWithIndex[T](iterator: Iterator[T], startIndex: Long): Iterator[(T, Long)] = { + new Iterator[(T, Long)] { + require(startIndex >= 0, "startIndex should be >= 0.") + var index: Long = startIndex - 1L + def hasNext: Boolean = iterator.hasNext + def next(): (T, Long) = { + index += 1L + (iterator.next(), index) + } + } + } + + /** + * Creates a symlink. + * * @param src absolute path to the source * @param dst relative path for the destination */ - def symlink(src: File, dst: File) { + def symlink(src: File, dst: File): Unit = { if (!src.isAbsolute()) { throw new IOException("Source must be absolute") } if (dst.isAbsolute()) { throw new IOException("Destination must be relative") } - var cmdSuffix = "" - val linkCmd = if (isWindows) { - // refer to http://technet.microsoft.com/en-us/library/cc771254.aspx - cmdSuffix = " /s /e /k /h /y /i" - "cmd /c xcopy " - } else { - cmdSuffix = "" - "ln -sf " - } - import scala.sys.process._ - (linkCmd + src.getAbsolutePath() + " " + dst.getPath() + cmdSuffix) lines_! - ProcessLogger(line => logInfo(line)) + Files.createSymbolicLink(dst.toPath, src.toPath) } @@ -1663,26 +1925,30 @@ private[spark] object Utils extends Logging { } /** - * Wait for a process to terminate for at most the specified duration. - * Return whether the process actually terminated after the given timeout. - */ - def waitForProcess(process: Process, timeoutMs: Long): Boolean = { - var terminated = false - val startTime = System.currentTimeMillis - while (!terminated) { + * Terminates a process waiting for at most the specified duration. + * + * @return the process exit value if it was successfully terminated, else None + */ + def terminateProcess(process: Process, timeoutMs: Long): Option[Int] = { + // Politely destroy first + process.destroy() + if (process.waitFor(timeoutMs, TimeUnit.MILLISECONDS)) { + // Successful exit + Option(process.exitValue()) + } else { try { - process.exitValue() - terminated = true + process.destroyForcibly() } catch { - case e: IllegalThreadStateException => - // Process not terminated yet - if (System.currentTimeMillis - startTime > timeoutMs) { - return false - } - Thread.sleep(100) + case NonFatal(e) => logWarning("Exception when attempting to kill process", e) + } + // Wait, again, although this really should return almost immediately + if (process.waitFor(timeoutMs, TimeUnit.MILLISECONDS)) { + Option(process.exitValue()) + } else { + logWarning("Timed out waiting to forcibly kill process") + None } } - true } /** @@ -1690,7 +1956,7 @@ private[spark] object Utils extends Logging { * If the process does not terminate within the specified timeout, return None. */ def getStderr(process: Process, timeoutMs: Long): Option[String] = { - val terminated = Utils.waitForProcess(process, timeoutMs) + val terminated = process.waitFor(timeoutMs, TimeUnit.MILLISECONDS) if (terminated) { Some(Source.fromInputStream(process.getErrorStream).getLines().mkString("\n")) } else { @@ -1732,22 +1998,17 @@ private[spark] object Utils extends Logging { /** Returns true if the given exception was fatal. See docs for scala.util.control.NonFatal. */ def isFatalError(e: Throwable): Boolean = { e match { - case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => + case NonFatal(_) | + _: InterruptedException | + _: NotImplementedError | + _: ControlThrowable | + _: LinkageError => false case _ => true } } - lazy val isInInterpreter: Boolean = { - try { - val interpClass = classForName("org.apache.spark.repl.Main") - interpClass.getMethod("interp").invoke(null) != null - } catch { - case _: ClassNotFoundException => false - } - } - /** * Return a well-formed URI for the file described by a user input string. * @@ -1778,7 +2039,7 @@ private[spark] object Utils extends Logging { if (paths == null || paths.trim.isEmpty) { "" } else { - paths.split(",").map { p => Utils.resolveURI(p) }.mkString(",") + paths.split(",").filter(_.trim.nonEmpty).map { p => Utils.resolveURI(p) }.mkString(",") } } @@ -1818,13 +2079,27 @@ private[spark] object Utils extends Logging { path } + /** + * Updates Spark config with properties from a set of Properties. + * Provided properties have the highest priority. + */ + def updateSparkConfigFromProperties( + conf: SparkConf, + properties: Map[String, String]) : Unit = { + properties.filter { case (k, v) => + k.startsWith("spark.") + }.foreach { case (k, v) => + conf.set(k, v) + } + } + /** Load properties present in the given file. */ def getPropertiesFromFile(filename: String): Map[String, String] = { val file = new File(filename) require(file.exists(), s"Properties file $file does not exist") require(file.isFile(), s"Properties file $file is not a normal file") - val inReader = new InputStreamReader(new FileInputStream(file), "UTF-8") + val inReader = new InputStreamReader(new FileInputStream(file), StandardCharsets.UTF_8) try { val properties = new Properties() properties.load(inReader) @@ -1863,18 +2138,62 @@ private[spark] object Utils extends Logging { } } + private implicit class Lock(lock: LockInfo) { + def lockString: String = { + lock match { + case monitor: MonitorInfo => + s"Monitor(${lock.getClassName}@${lock.getIdentityHashCode}})" + case _ => + s"Lock(${lock.getClassName}@${lock.getIdentityHashCode}})" + } + } + } + /** Return a thread dump of all threads' stacktraces. Used to capture dumps for the web UI */ def getThreadDump(): Array[ThreadStackTrace] = { // We need to filter out null values here because dumpAllThreads() may return null array // elements for threads that are dead / don't exist. val threadInfos = ManagementFactory.getThreadMXBean.dumpAllThreads(true, true).filter(_ != null) - threadInfos.sortBy(_.getThreadId).map { case threadInfo => - val stackTrace = threadInfo.getStackTrace.map(_.toString).mkString("\n") - ThreadStackTrace(threadInfo.getThreadId, threadInfo.getThreadName, - threadInfo.getThreadState, stackTrace) + threadInfos.sortBy(_.getThreadId).map(threadInfoToThreadStackTrace) + } + + def getThreadDumpForThread(threadId: Long): Option[ThreadStackTrace] = { + if (threadId <= 0) { + None + } else { + // The Int.MaxValue here requests the entire untruncated stack trace of the thread: + val threadInfo = + Option(ManagementFactory.getThreadMXBean.getThreadInfo(threadId, Int.MaxValue)) + threadInfo.map(threadInfoToThreadStackTrace) } } + private def threadInfoToThreadStackTrace(threadInfo: ThreadInfo): ThreadStackTrace = { + val monitors = threadInfo.getLockedMonitors.map(m => m.getLockedStackFrame -> m).toMap + val stackTrace = threadInfo.getStackTrace.map { frame => + monitors.get(frame) match { + case Some(monitor) => + monitor.getLockedStackFrame.toString + s" => holding ${monitor.lockString}" + case None => + frame.toString + } + }.mkString("\n") + + // use a set to dedup re-entrant locks that are held at multiple places + val heldLocks = + (threadInfo.getLockedSynchronizers ++ threadInfo.getLockedMonitors).map(_.lockString).toSet + + ThreadStackTrace( + threadId = threadInfo.getThreadId, + threadName = threadInfo.getThreadName, + threadState = threadInfo.getThreadState, + stackTrace = stackTrace, + blockedByThreadId = + if (threadInfo.getLockOwnerId < 0) None else Some(threadInfo.getLockOwnerId), + blockedByLock = Option(threadInfo.getLockInfo).map(_.lockString).getOrElse(""), + holdingLocks = heldLocks.toSeq) + } + /** * Convert all spark properties set in the given SparkConf to a sequence of java options. */ @@ -1897,6 +2216,14 @@ private[spark] object Utils extends Logging { } } + /** + * Returns the user port to try when trying to bind a service. Handles wrapping and skipping + * privileged ports. + */ + def userPort(base: Int, offset: Int): Int = { + (base + offset - 1024) % (65536 - 1024) + 1024 + } + /** * Attempt to start a service on the given port, or fail after a number of attempts. * Each subsequent attempt uses 1 + the port used in the previous attempt (unless the port is 0). @@ -1924,8 +2251,7 @@ private[spark] object Utils extends Logging { val tryPort = if (startPort == 0) { startPort } else { - // If the new port wraps around, do not try a privilege port - ((startPort + offset - 1024) % (65536 - 1024)) + 1024 + userPort(startPort, offset) } try { val (service, port) = startService(tryPort) @@ -1934,15 +2260,32 @@ private[spark] object Utils extends Logging { } catch { case e: Exception if isBindCollision(e) => if (offset >= maxRetries) { - val exceptionMessage = - s"${e.getMessage}: Service$serviceString failed after $maxRetries retries!" + val exceptionMessage = if (startPort == 0) { + s"${e.getMessage}: Service$serviceString failed after " + + s"$maxRetries retries (on a random free port)! " + + s"Consider explicitly setting the appropriate binding address for " + + s"the service$serviceString (for example spark.driver.bindAddress " + + s"for SparkDriver) to the correct binding address." + } else { + s"${e.getMessage}: Service$serviceString failed after " + + s"$maxRetries retries (starting from $startPort)! Consider explicitly setting " + + s"the appropriate port for the service$serviceString (for example spark.ui.port " + + s"for SparkUI) to an available port or increasing spark.port.maxRetries." + } val exception = new BindException(exceptionMessage) // restore original stack trace exception.setStackTrace(e.getStackTrace) throw exception } - logWarning(s"Service$serviceString could not bind on port $tryPort. " + - s"Attempting port ${tryPort + 1}.") + if (startPort == 0) { + // As startPort 0 is for a random free port, it is most possibly binding address is + // not correct. + logWarning(s"Service$serviceString could not bind on a random free port. " + + "You may check whether configuring an appropriate binding address.") + } else { + logWarning(s"Service$serviceString could not bind on port $tryPort. " + + s"Attempting port ${tryPort + 1}.") + } } } // Should never happen @@ -1961,6 +2304,9 @@ private[spark] object Utils extends Logging { isBindCollision(e.getCause) case e: MultiException => e.getThrowables.asScala.exists(isBindCollision) + case e: NativeIoException => + (e.getMessage != null && e.getMessage.startsWith("bind() failed: ")) || + isBindCollision(e.getCause) case e: Exception => isBindCollision(e.getCause) case _ => false } @@ -2071,8 +2417,9 @@ private[spark] object Utils extends Logging { * A spark url (`spark://host:port`) is a special URI that its scheme is `spark` and only contains * host and port. * - * @throws SparkException if `sparkUrl` is invalid. + * @throws org.apache.spark.SparkException if sparkUrl is invalid. */ + @throws(classOf[SparkException]) def extractHostPortFromSparkUrl(sparkUrl: String): (String, Int) = { try { val uri = new java.net.URI(sparkUrl) @@ -2103,6 +2450,25 @@ private[spark] object Utils extends Logging { .getOrElse(UserGroupInformation.getCurrentUser().getShortUserName()) } + val EMPTY_USER_GROUPS = Set.empty[String] + + // Returns the groups to which the current user belongs. + def getCurrentUserGroups(sparkConf: SparkConf, username: String): Set[String] = { + val groupProviderClassName = sparkConf.get("spark.user.groups.mapping", + "org.apache.spark.security.ShellBasedGroupsMappingProvider") + if (groupProviderClassName != "") { + try { + val groupMappingServiceProvider = classForName(groupProviderClassName).newInstance. + asInstanceOf[org.apache.spark.security.GroupMappingServiceProvider] + val currentUserGroups = groupMappingServiceProvider.getGroups(username) + return currentUserGroups + } catch { + case e: Exception => logError(s"Error getting groups for user=$username", e) + } + } + EMPTY_USER_GROUPS + } + /** * Split the comma delimited string of master URLs into a list. * For instance, "spark://abc,def" becomes [spark://abc, spark://def]. @@ -2140,6 +2506,7 @@ private[spark] object Utils extends Logging { /** * Return whether the specified file is a parent directory of the child file. */ + @tailrec def isInDirectory(parent: File, child: File): Boolean = { if (child == null || parent == null) { return false @@ -2153,21 +2520,266 @@ private[spark] object Utils extends Logging { isInDirectory(parent, child.getParentFile) } + /** - * Return whether dynamic allocation is enabled in the given conf - * Dynamic allocation and explicitly setting the number of executors are inherently - * incompatible. In environments where dynamic allocation is turned on by default, - * the latter should override the former (SPARK-9092). + * + * @return whether it is local mode + */ + def isLocalMaster(conf: SparkConf): Boolean = { + val master = conf.get("spark.master", "") + master == "local" || master.startsWith("local[") + } + + /** + * Return whether dynamic allocation is enabled in the given conf. */ def isDynamicAllocationEnabled(conf: SparkConf): Boolean = { - conf.getBoolean("spark.dynamicAllocation.enabled", false) && - conf.getInt("spark.executor.instances", 0) == 0 + val dynamicAllocationEnabled = conf.getBoolean("spark.dynamicAllocation.enabled", false) + dynamicAllocationEnabled && + (!isLocalMaster(conf) || conf.getBoolean("spark.dynamicAllocation.testing", false)) + } + + /** + * Return the initial number of executors for dynamic allocation. + */ + def getDynamicAllocationInitialExecutors(conf: SparkConf): Int = { + if (conf.get(DYN_ALLOCATION_INITIAL_EXECUTORS) < conf.get(DYN_ALLOCATION_MIN_EXECUTORS)) { + logWarning(s"${DYN_ALLOCATION_INITIAL_EXECUTORS.key} less than " + + s"${DYN_ALLOCATION_MIN_EXECUTORS.key} is invalid, ignoring its setting, " + + "please update your configs.") + } + + if (conf.get(EXECUTOR_INSTANCES).getOrElse(0) < conf.get(DYN_ALLOCATION_MIN_EXECUTORS)) { + logWarning(s"${EXECUTOR_INSTANCES.key} less than " + + s"${DYN_ALLOCATION_MIN_EXECUTORS.key} is invalid, ignoring its setting, " + + "please update your configs.") + } + + val initialExecutors = Seq( + conf.get(DYN_ALLOCATION_MIN_EXECUTORS), + conf.get(DYN_ALLOCATION_INITIAL_EXECUTORS), + conf.get(EXECUTOR_INSTANCES).getOrElse(0)).max + + logInfo(s"Using initial executors = $initialExecutors, max of " + + s"${DYN_ALLOCATION_INITIAL_EXECUTORS.key}, ${DYN_ALLOCATION_MIN_EXECUTORS.key} and " + + s"${EXECUTOR_INSTANCES.key}") + initialExecutors } def tryWithResource[R <: Closeable, T](createResource: => R)(f: R => T): T = { val resource = createResource try f.apply(resource) finally resource.close() } + + /** + * Returns a path of temporary file which is in the same directory with `path`. + */ + def tempFileWith(path: File): File = { + new File(path.getAbsolutePath + "." + UUID.randomUUID()) + } + + /** + * Returns the name of this JVM process. This is OS dependent but typically (OSX, Linux, Windows), + * this is formatted as PID@hostname. + */ + def getProcessName(): String = { + ManagementFactory.getRuntimeMXBean().getName() + } + + /** + * Utility function that should be called early in `main()` for daemons to set up some common + * diagnostic state. + */ + def initDaemon(log: Logger): Unit = { + log.info(s"Started daemon with process name: ${Utils.getProcessName()}") + SignalUtils.registerLogger(log) + } + + /** + * Unions two comma-separated lists of files and filters out empty strings. + */ + def unionFileLists(leftList: Option[String], rightList: Option[String]): Set[String] = { + var allFiles = Set.empty[String] + leftList.foreach { value => allFiles ++= value.split(",") } + rightList.foreach { value => allFiles ++= value.split(",") } + allFiles.filter { _.nonEmpty } + } + + /** + * Return the jar files pointed by the "spark.jars" property. Spark internally will distribute + * these jars through file server. In the YARN mode, it will return an empty list, since YARN + * has its own mechanism to distribute jars. + */ + def getUserJars(conf: SparkConf): Seq[String] = { + val sparkJars = conf.getOption("spark.jars") + sparkJars.map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten + } + + /** + * Return the local jar files which will be added to REPL's classpath. These jar files are + * specified by --jars (spark.jars) or --packages, remote jars will be downloaded to local by + * SparkSubmit at first. + */ + def getLocalUserJarsForShell(conf: SparkConf): Seq[String] = { + val localJars = conf.getOption("spark.repl.local.jars") + localJars.map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten + } + + private[spark] val REDACTION_REPLACEMENT_TEXT = "*********(redacted)" + + /** + * Redact the sensitive values in the given map. If a map key matches the redaction pattern then + * its value is replaced with a dummy text. + */ + def redact(conf: SparkConf, kvs: Seq[(String, String)]): Seq[(String, String)] = { + val redactionPattern = conf.get(SECRET_REDACTION_PATTERN) + redact(redactionPattern, kvs) + } + + /** + * Redact the sensitive information in the given string. + */ + def redact(conf: SparkConf, text: String): String = { + if (text == null || text.isEmpty || conf == null || !conf.contains(STRING_REDACTION_PATTERN)) { + text + } else { + val regex = conf.get(STRING_REDACTION_PATTERN).get + regex.replaceAllIn(text, REDACTION_REPLACEMENT_TEXT) + } + } + + private def redact(redactionPattern: Regex, kvs: Seq[(String, String)]): Seq[(String, String)] = { + // If the sensitive information regex matches with either the key or the value, redact the value + // While the original intent was to only redact the value if the key matched with the regex, + // we've found that especially in verbose mode, the value of the property may contain sensitive + // information like so: + // "sun.java.command":"org.apache.spark.deploy.SparkSubmit ... \ + // --conf spark.executorEnv.HADOOP_CREDSTORE_PASSWORD=secret_password ... + // + // And, in such cases, simply searching for the sensitive information regex in the key name is + // not sufficient. The values themselves have to be searched as well and redacted if matched. + // This does mean we may be accounting more false positives - for example, if the value of an + // arbitrary property contained the term 'password', we may redact the value from the UI and + // logs. In order to work around it, user would have to make the spark.redaction.regex property + // more specific. + kvs.map { case (key, value) => + redactionPattern.findFirstIn(key) + .orElse(redactionPattern.findFirstIn(value)) + .map { _ => (key, REDACTION_REPLACEMENT_TEXT) } + .getOrElse((key, value)) + } + } + + /** + * Looks up the redaction regex from within the key value pairs and uses it to redact the rest + * of the key value pairs. No care is taken to make sure the redaction property itself is not + * redacted. So theoretically, the property itself could be configured to redact its own value + * when printing. + */ + def redact(kvs: Map[String, String]): Seq[(String, String)] = { + val redactionPattern = kvs.getOrElse( + SECRET_REDACTION_PATTERN.key, + SECRET_REDACTION_PATTERN.defaultValueString + ).r + redact(redactionPattern, kvs.toArray) + } + + def stringToSeq(str: String): Seq[String] = { + str.split(",").map(_.trim()).filter(_.nonEmpty) + } +} + +private[util] object CallerContext extends Logging { + val callerContextSupported: Boolean = { + SparkHadoopUtil.get.conf.getBoolean("hadoop.caller.context.enabled", false) && { + try { + Utils.classForName("org.apache.hadoop.ipc.CallerContext") + Utils.classForName("org.apache.hadoop.ipc.CallerContext$Builder") + true + } catch { + case _: ClassNotFoundException => + false + case NonFatal(e) => + logWarning("Fail to load the CallerContext class", e) + false + } + } + } +} + +/** + * An utility class used to set up Spark caller contexts to HDFS and Yarn. The `context` will be + * constructed by parameters passed in. + * When Spark applications run on Yarn and HDFS, its caller contexts will be written into Yarn RM + * audit log and hdfs-audit.log. That can help users to better diagnose and understand how + * specific applications impacting parts of the Hadoop system and potential problems they may be + * creating (e.g. overloading NN). As HDFS mentioned in HDFS-9184, for a given HDFS operation, it's + * very helpful to track which upper level job issues it. + * + * @param from who sets up the caller context (TASK, CLIENT, APPMASTER) + * + * The parameters below are optional: + * @param upstreamCallerContext caller context the upstream application passes in + * @param appId id of the app this task belongs to + * @param appAttemptId attempt id of the app this task belongs to + * @param jobId id of the job this task belongs to + * @param stageId id of the stage this task belongs to + * @param stageAttemptId attempt id of the stage this task belongs to + * @param taskId task id + * @param taskAttemptNumber task attempt id + */ +private[spark] class CallerContext( + from: String, + upstreamCallerContext: Option[String] = None, + appId: Option[String] = None, + appAttemptId: Option[String] = None, + jobId: Option[Int] = None, + stageId: Option[Int] = None, + stageAttemptId: Option[Int] = None, + taskId: Option[Long] = None, + taskAttemptNumber: Option[Int] = None) extends Logging { + + private val context = prepareContext("SPARK_" + + from + + appId.map("_" + _).getOrElse("") + + appAttemptId.map("_" + _).getOrElse("") + + jobId.map("_JId_" + _).getOrElse("") + + stageId.map("_SId_" + _).getOrElse("") + + stageAttemptId.map("_" + _).getOrElse("") + + taskId.map("_TId_" + _).getOrElse("") + + taskAttemptNumber.map("_" + _).getOrElse("") + + upstreamCallerContext.map("_" + _).getOrElse("")) + + private def prepareContext(context: String): String = { + // The default max size of Hadoop caller context is 128 + lazy val len = SparkHadoopUtil.get.conf.getInt("hadoop.caller.context.max.size", 128) + if (context == null || context.length <= len) { + context + } else { + val finalContext = context.substring(0, len) + logWarning(s"Truncated Spark caller context from $context to $finalContext") + finalContext + } + } + + /** + * Set up the caller context [[context]] by invoking Hadoop CallerContext API of + * [[org.apache.hadoop.ipc.CallerContext]], which was added in hadoop 2.8. + */ + def setCurrentContext(): Unit = { + if (CallerContext.callerContextSupported) { + try { + val callerContext = Utils.classForName("org.apache.hadoop.ipc.CallerContext") + val builder = Utils.classForName("org.apache.hadoop.ipc.CallerContext$Builder") + val builderInst = builder.getConstructor(classOf[String]).newInstance(context) + val hdfsContext = builder.getMethod("build").invoke(builderInst) + callerContext.getMethod("setCurrent", callerContext).invoke(null, hdfsContext) + } catch { + case NonFatal(e) => + logWarning("Fail to set Spark caller context", e) + } + } + } } /** @@ -2207,29 +2819,24 @@ private[spark] class RedirectThread( * the toString method. */ private[spark] class CircularBuffer(sizeInBytes: Int = 10240) extends java.io.OutputStream { - var pos: Int = 0 - var buffer = new Array[Int](sizeInBytes) + private var pos: Int = 0 + private var isBufferFull = false + private val buffer = new Array[Byte](sizeInBytes) - def write(i: Int): Unit = { - buffer(pos) = i + def write(input: Int): Unit = { + buffer(pos) = input.toByte pos = (pos + 1) % buffer.length + isBufferFull = isBufferFull || (pos == 0) } override def toString: String = { - val (end, start) = buffer.splitAt(pos) - val input = new java.io.InputStream { - val iterator = (start ++ end).iterator - - def read(): Int = if (iterator.hasNext) iterator.next() else -1 + if (!isBufferFull) { + return new String(buffer, 0, pos, StandardCharsets.UTF_8) } - val reader = new BufferedReader(new InputStreamReader(input)) - val stringBuilder = new StringBuilder - var line = reader.readLine() - while (line != null) { - stringBuilder.append(line) - stringBuilder.append("\n") - line = reader.readLine() - } - stringBuilder.toString() + + val nonCircularBuffer = new Array[Byte](sizeInBytes) + System.arraycopy(buffer, pos, nonCircularBuffer, 0, buffer.length - pos) + System.arraycopy(buffer, 0, nonCircularBuffer, buffer.length - pos, pos) + new String(nonCircularBuffer, StandardCharsets.UTF_8) } } diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala deleted file mode 100644 index 2ed827eab46d..000000000000 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import scala.language.implicitConversions -import scala.util.Random - -import org.apache.spark.util.random.XORShiftRandom - -@deprecated("Use Vectors.dense from Spark's mllib.linalg package instead.", "1.0.0") -class Vector(val elements: Array[Double]) extends Serializable { - def length: Int = elements.length - - def apply(index: Int): Double = elements(index) - - def + (other: Vector): Vector = { - if (length != other.length) { - throw new IllegalArgumentException("Vectors of different length") - } - Vector(length, i => this(i) + other(i)) - } - - def add(other: Vector): Vector = this + other - - def - (other: Vector): Vector = { - if (length != other.length) { - throw new IllegalArgumentException("Vectors of different length") - } - Vector(length, i => this(i) - other(i)) - } - - def subtract(other: Vector): Vector = this - other - - def dot(other: Vector): Double = { - if (length != other.length) { - throw new IllegalArgumentException("Vectors of different length") - } - var ans = 0.0 - var i = 0 - while (i < length) { - ans += this(i) * other(i) - i += 1 - } - ans - } - - /** - * return (this + plus) dot other, but without creating any intermediate storage - * @param plus - * @param other - * @return - */ - def plusDot(plus: Vector, other: Vector): Double = { - if (length != other.length) { - throw new IllegalArgumentException("Vectors of different length") - } - if (length != plus.length) { - throw new IllegalArgumentException("Vectors of different length") - } - var ans = 0.0 - var i = 0 - while (i < length) { - ans += (this(i) + plus(i)) * other(i) - i += 1 - } - ans - } - - def += (other: Vector): Vector = { - if (length != other.length) { - throw new IllegalArgumentException("Vectors of different length") - } - var i = 0 - while (i < length) { - elements(i) += other(i) - i += 1 - } - this - } - - def addInPlace(other: Vector): Vector = this +=other - - def * (scale: Double): Vector = Vector(length, i => this(i) * scale) - - def multiply (d: Double): Vector = this * d - - def / (d: Double): Vector = this * (1 / d) - - def divide (d: Double): Vector = this / d - - def unary_- : Vector = this * -1 - - def sum: Double = elements.reduceLeft(_ + _) - - def squaredDist(other: Vector): Double = { - var ans = 0.0 - var i = 0 - while (i < length) { - ans += (this(i) - other(i)) * (this(i) - other(i)) - i += 1 - } - ans - } - - def dist(other: Vector): Double = math.sqrt(squaredDist(other)) - - override def toString: String = elements.mkString("(", ", ", ")") -} - -object Vector { - def apply(elements: Array[Double]): Vector = new Vector(elements) - - def apply(elements: Double*): Vector = new Vector(elements.toArray) - - def apply(length: Int, initializer: Int => Double): Vector = { - val elements: Array[Double] = Array.tabulate(length)(initializer) - new Vector(elements) - } - - def zeros(length: Int): Vector = new Vector(new Array[Double](length)) - - def ones(length: Int): Vector = Vector(length, _ => 1) - - /** - * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers - * between 0.0 and 1.0. Optional scala.util.Random number generator can be provided. - */ - def random(length: Int, random: Random = new XORShiftRandom()): Vector = - Vector(length, _ => random.nextDouble()) - - class Multiplier(num: Double) { - def * (vec: Vector): Vector = vec * num - } - - implicit def doubleToMultiplier(num: Double): Multiplier = new Multiplier(num) - - implicit object VectorAccumParam extends org.apache.spark.AccumulatorParam[Vector] { - def addInPlace(t1: Vector, t2: Vector): Vector = t1 + t2 - - def zero(initialValue: Vector): Vector = Vector.zeros(initialValue.length) - } - -} diff --git a/core/src/main/scala/org/apache/spark/util/VersionUtils.scala b/core/src/main/scala/org/apache/spark/util/VersionUtils.scala new file mode 100644 index 000000000000..828153b86842 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/VersionUtils.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +/** + * Utilities for working with Spark version strings + */ +private[spark] object VersionUtils { + + private val majorMinorRegex = """^(\d+)\.(\d+)(\..*)?$""".r + + /** + * Given a Spark version string, return the major version number. + * E.g., for 2.0.1-SNAPSHOT, return 2. + */ + def majorVersion(sparkVersion: String): Int = majorMinorVersion(sparkVersion)._1 + + /** + * Given a Spark version string, return the minor version number. + * E.g., for 2.0.1-SNAPSHOT, return 0. + */ + def minorVersion(sparkVersion: String): Int = majorMinorVersion(sparkVersion)._2 + + /** + * Given a Spark version string, return the (major version number, minor version number). + * E.g., for 2.0.1-SNAPSHOT, return (2, 0). + */ + def majorMinorVersion(sparkVersion: String): (Int, Int) = { + majorMinorRegex.findFirstMatchIn(sparkVersion) match { + case Some(m) => + (m.group(1).toInt, m.group(2).toInt) + case None => + throw new IllegalArgumentException(s"Spark tried to parse '$sparkVersion' as a Spark" + + s" version string, but it could not find the major and minor version numbers.") + } + } +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index 4c1e16155462..bcb95b416dd2 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -17,7 +17,7 @@ package org.apache.spark.util.collection -import java.util.{Arrays, Comparator} +import java.util.Comparator import com.google.common.hash.Hashing @@ -140,16 +140,16 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) var i = 1 while (true) { val curKey = data(2 * pos) - if (k.eq(curKey) || k.equals(curKey)) { - val newValue = updateFunc(true, data(2 * pos + 1).asInstanceOf[V]) - data(2 * pos + 1) = newValue.asInstanceOf[AnyRef] - return newValue - } else if (curKey.eq(null)) { + if (curKey.eq(null)) { val newValue = updateFunc(false, null.asInstanceOf[V]) data(2 * pos) = k data(2 * pos + 1) = newValue.asInstanceOf[AnyRef] incrementSize() return newValue + } else if (k.eq(curKey) || k.equals(curKey)) { + val newValue = updateFunc(true, data(2 * pos + 1).asInstanceOf[V]) + data(2 * pos + 1) = newValue.asInstanceOf[AnyRef] + return newValue } else { val delta = i pos = (pos + delta) & mask diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index 85c5bdbfcebc..e63e0e3e1f68 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -17,21 +17,16 @@ package org.apache.spark.util.collection -import java.io.{Externalizable, ObjectInput, ObjectOutput} - -import org.apache.spark.util.{Utils => UUtils} - +import java.util.Arrays /** * A simple, fixed-size bit set implementation. This implementation is fast because it avoids * safety/bound checking. */ -class BitSet(private[this] var numBits: Int) extends Externalizable { - - private var words = new Array[Long](bit2words(numBits)) - private def numWords = words.length +class BitSet(numBits: Int) extends Serializable { - def this() = this(0) + private val words = new Array[Long](bit2words(numBits)) + private val numWords = words.length /** * Compute the capacity (number of bits) that can be represented @@ -42,21 +37,14 @@ class BitSet(private[this] var numBits: Int) extends Externalizable { /** * Clear all set bits. */ - def clear(): Unit = { - var i = 0 - while (i < numWords) { - words(i) = 0L - i += 1 - } - } + def clear(): Unit = Arrays.fill(words, 0) /** * Set all the bits up to a given index */ - def setUntil(bitIndex: Int) { + def setUntil(bitIndex: Int): Unit = { val wordIndex = bitIndex >> 6 // divide by 64 - var i = 0 - while(i < wordIndex) { words(i) = -1; i += 1 } + Arrays.fill(words, 0, wordIndex, -1) if(wordIndex < words.length) { // Set the remaining bits (note that the mask could still be zero) val mask = ~(-1L << (bitIndex & 0x3f)) @@ -64,6 +52,19 @@ class BitSet(private[this] var numBits: Int) extends Externalizable { } } + /** + * Clear all the bits up to a given index + */ + def clearUntil(bitIndex: Int): Unit = { + val wordIndex = bitIndex >> 6 // divide by 64 + Arrays.fill(words, 0, wordIndex, 0) + if(wordIndex < words.length) { + // Clear the remaining bits + val mask = -1L << (bitIndex & 0x3f) + words(wordIndex) &= mask + } + } + /** * Compute the bit-wise AND of the two sets returning the * result. @@ -237,19 +238,4 @@ class BitSet(private[this] var numBits: Int) extends Externalizable { /** Return the number of longs it would take to hold numBits. */ private def bit2words(numBits: Int) = ((numBits - 1) >> 6) + 1 - - override def writeExternal(out: ObjectOutput): Unit = UUtils.tryOrIOException { - out.writeInt(numBits) - words.foreach(out.writeLong(_)) - } - - override def readExternal(in: ObjectInput): Unit = UUtils.tryOrIOException { - numBits = in.readInt() - words = new Array[Long](bit2words(numBits)) - var index = 0 - while (index < words.length) { - words(index) = in.readLong() - index += 1 - } - } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala index 4d43d8d5cc8d..f5d2fa14e49c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala @@ -126,22 +126,22 @@ private[spark] class CompactBuffer[T: ClassTag] extends Seq[T] with Serializable /** Increase our size to newSize and grow the backing array if needed. */ private def growToSize(newSize: Int): Unit = { - if (newSize < 0) { - throw new UnsupportedOperationException("Can't grow buffer past Int.MaxValue elements") + // Some JVMs can't allocate arrays of length Integer.MAX_VALUE; actual max is somewhat + // smaller. Be conservative and lower the cap a little. + val arrayMax = Int.MaxValue - 8 + if (newSize < 0 || newSize - 2 > arrayMax) { + throw new UnsupportedOperationException(s"Can't grow buffer past $arrayMax elements") } val capacity = if (otherElements != null) otherElements.length + 2 else 2 if (newSize > capacity) { - var newArrayLen = 8 + var newArrayLen = 8L while (newSize - 2 > newArrayLen) { newArrayLen *= 2 - if (newArrayLen == Int.MinValue) { - // Prevent overflow if we double from 2^30 to 2^31, which will become Int.MinValue. - // Note that we set the new array length to Int.MaxValue - 2 so that our capacity - // calculation above still gives a positive integer. - newArrayLen = Int.MaxValue - 2 - } } - val newArray = new Array[T](newArrayLen) + if (newArrayLen > arrayMax) { + newArrayLen = arrayMax + } + val newArray = new Array[T](newArrayLen.toInt) if (otherElements != null) { System.arraycopy(otherElements, 0, newArray, 0, otherElements.length) } 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 f6d81ee5bf05..6f5b5bb3652d 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 @@ -18,6 +18,8 @@ package org.apache.spark.util.collection import java.io._ +import java.nio.channels.{Channels, FileChannel} +import java.nio.file.StandardOpenOption import java.util.Comparator import scala.collection.BufferedIterator @@ -26,14 +28,14 @@ import scala.collection.mutable.ArrayBuffer import com.google.common.io.ByteStreams -import org.apache.spark.{Logging, SparkEnv, TaskContext} +import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.memory.TaskMemoryManager -import org.apache.spark.serializer.{DeserializationStream, Serializer} +import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.internal.Logging +import org.apache.spark.serializer.{DeserializationStream, Serializer, SerializerManager} import org.apache.spark.storage.{BlockId, BlockManager} import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.ExternalAppendOnlyMap.HashComparator -import org.apache.spark.executor.ShuffleWriteMetrics /** * :: DeveloperApi :: @@ -58,11 +60,12 @@ class ExternalAppendOnlyMap[K, V, C]( mergeCombiners: (C, C) => C, serializer: Serializer = SparkEnv.get.serializer, blockManager: BlockManager = SparkEnv.get.blockManager, - context: TaskContext = TaskContext.get()) - extends Iterable[(K, C)] + context: TaskContext = TaskContext.get(), + serializerManager: SerializerManager = SparkEnv.get.serializerManager) + extends Spillable[SizeTracker](context.taskMemoryManager()) with Serializable with Logging - with Spillable[SizeTracker] { + with Iterable[(K, C)] { if (context == null) { throw new IllegalStateException( @@ -79,9 +82,7 @@ class ExternalAppendOnlyMap[K, V, C]( this(createCombiner, mergeValue, mergeCombiners, serializer, blockManager, TaskContext.get()) } - override protected[this] def taskMemoryManager: TaskMemoryManager = context.taskMemoryManager() - - private var currentMap = new SizeTrackingAppendOnlyMap[K, C] + @volatile private var currentMap = new SizeTrackingAppendOnlyMap[K, C] private val spilledMaps = new ArrayBuffer[DiskMapIterator] private val sparkConf = SparkEnv.get.conf private val diskBlockManager = blockManager.diskBlockManager @@ -105,8 +106,8 @@ class ExternalAppendOnlyMap[K, V, C]( private val fileBufferSize = sparkConf.getSizeAsKb("spark.shuffle.file.buffer", "32k").toInt * 1024 - // Write metrics for current spill - private var curWriteMetrics: ShuffleWriteMetrics = _ + // Write metrics + private val writeMetrics: ShuffleWriteMetrics = new ShuffleWriteMetrics() // Peak size of the in-memory map observed so far, in bytes private var _peakMemoryUsedBytes: Long = 0L @@ -115,6 +116,8 @@ class ExternalAppendOnlyMap[K, V, C]( private val keyComparator = new HashComparator[K] private val ser = serializer.newInstance() + @volatile private var readingIterator: SpillableIterator = null + /** * Number of files this map has spilled so far. * Exposed for testing. @@ -180,9 +183,38 @@ class ExternalAppendOnlyMap[K, V, C]( * Sort the existing contents of the in-memory map and spill them to a temporary file on disk. */ override protected[this] def spill(collection: SizeTracker): Unit = { + val inMemoryIterator = currentMap.destructiveSortedIterator(keyComparator) + val diskMapIterator = spillMemoryIteratorToDisk(inMemoryIterator) + spilledMaps += diskMapIterator + } + + /** + * Force to spilling the current in-memory collection to disk to release memory, + * It will be called by TaskMemoryManager when there is not enough memory for the task. + */ + override protected[this] def forceSpill(): Boolean = { + if (readingIterator != null) { + val isSpilled = readingIterator.spill() + if (isSpilled) { + currentMap = null + } + isSpilled + } else if (currentMap.size > 0) { + spill(currentMap) + currentMap = new SizeTrackingAppendOnlyMap[K, C] + true + } else { + false + } + } + + /** + * Spill the in-memory Iterator to a temporary file on disk. + */ + private[this] def spillMemoryIteratorToDisk(inMemoryIterator: Iterator[(K, C)]) + : DiskMapIterator = { val (blockId, file) = diskBlockManager.createTempLocalBlock() - curWriteMetrics = new ShuffleWriteMetrics() - var writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) + val writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, writeMetrics) var objectsWritten = 0 // List of batch sizes (bytes) in the order they are written to disk @@ -190,43 +222,35 @@ class ExternalAppendOnlyMap[K, V, C]( // Flush the disk writer's contents to disk, and update relevant variables def flush(): Unit = { - val w = writer - writer = null - w.commitAndClose() - _diskBytesSpilled += curWriteMetrics.shuffleBytesWritten - batchSizes.append(curWriteMetrics.shuffleBytesWritten) + val segment = writer.commitAndGet() + batchSizes += segment.length + _diskBytesSpilled += segment.length objectsWritten = 0 } var success = false try { - val it = currentMap.destructiveSortedIterator(keyComparator) - while (it.hasNext) { - val kv = it.next() + while (inMemoryIterator.hasNext) { + val kv = inMemoryIterator.next() writer.write(kv._1, kv._2) objectsWritten += 1 if (objectsWritten == serializerBatchSize) { flush() - curWriteMetrics = new ShuffleWriteMetrics() - writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) } } if (objectsWritten > 0) { flush() - } else if (writer != null) { - val w = writer - writer = null - w.revertPartialWritesAndClose() + writer.close() + } else { + writer.revertPartialWritesAndClose() } success = true } finally { if (!success) { // This code path only happens if an exception was thrown above before we set success; // close our stuff and let the exception be thrown further - if (writer != null) { - writer.revertPartialWritesAndClose() - } + writer.revertPartialWritesAndClose() if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting ${file}") @@ -235,7 +259,17 @@ class ExternalAppendOnlyMap[K, V, C]( } } - spilledMaps.append(new DiskMapIterator(file, blockId, batchSizes)) + new DiskMapIterator(file, blockId, batchSizes) + } + + /** + * Returns a destructive iterator for iterating over the entries of this map. + * If this iterator is forced spill to disk to release memory when there is not enough memory, + * it returns pairs from an on-disk map. + */ + def destructiveIterator(inMemoryIterator: Iterator[(K, C)]): Iterator[(K, C)] = { + readingIterator = new SpillableIterator(inMemoryIterator) + readingIterator } /** @@ -248,15 +282,18 @@ class ExternalAppendOnlyMap[K, V, C]( "ExternalAppendOnlyMap.iterator is destructive and should only be called once.") } if (spilledMaps.isEmpty) { - CompletionIterator[(K, C), Iterator[(K, C)]](currentMap.iterator, freeCurrentMap()) + CompletionIterator[(K, C), Iterator[(K, C)]]( + destructiveIterator(currentMap.iterator), freeCurrentMap()) } else { new ExternalIterator() } } private def freeCurrentMap(): Unit = { - currentMap = null // So that the memory can be garbage-collected - releaseMemory() + if (currentMap != null) { + currentMap = null // So that the memory can be garbage-collected + releaseMemory() + } } /** @@ -270,8 +307,8 @@ class ExternalAppendOnlyMap[K, V, C]( // Input streams are derived both from the in-memory map and spilled maps on disk // The in-memory map is sorted in place, while the spilled maps are already in sorted order - private val sortedMap = CompletionIterator[(K, C), Iterator[(K, C)]]( - currentMap.destructiveSortedIterator(keyComparator), freeCurrentMap()) + private val sortedMap = CompletionIterator[(K, C), Iterator[(K, C)]](destructiveIterator( + currentMap.destructiveSortedIterator(keyComparator)), freeCurrentMap()) private val inputStreams = (Seq(sortedMap) ++ spilledMaps).map(it => it.buffered) inputStreams.foreach { it => @@ -338,14 +375,14 @@ class ExternalAppendOnlyMap[K, V, C]( /** * Return true if there exists an input stream that still has unvisited pairs. */ - override def hasNext: Boolean = mergeHeap.length > 0 + override def hasNext: Boolean = mergeHeap.nonEmpty /** * Select a key with the minimum hash, then combine all values with the same key from all * input streams. */ override def next(): (K, C) = { - if (mergeHeap.length == 0) { + if (mergeHeap.isEmpty) { throw new NoSuchElementException } // Select a key from the StreamBuffer that holds the lowest key hash @@ -360,7 +397,7 @@ class ExternalAppendOnlyMap[K, V, C]( // For all other streams that may have this key (i.e. have the same minimum key hash), // merge in the corresponding value (if any) from that stream val mergedBuffers = ArrayBuffer[StreamBuffer](minBuffer) - while (mergeHeap.length > 0 && mergeHeap.head.minKeyHash == minHash) { + while (mergeHeap.nonEmpty && mergeHeap.head.minKeyHash == minHash) { val newBuffer = mergeHeap.dequeue() minCombiner = mergeIfKeyExists(minKey, minCombiner, newBuffer) mergedBuffers += newBuffer @@ -424,7 +461,7 @@ class ExternalAppendOnlyMap[K, V, C]( ) private var batchIndex = 0 // Which batch we're in - private var fileStream: FileInputStream = null + private var fileChannel: FileChannel = null // An intermediate stream that reads from exactly one batch // This guards against pre-fetching and other arbitrary behavior of higher level streams @@ -441,14 +478,14 @@ class ExternalAppendOnlyMap[K, V, C]( if (batchIndex < batchOffsets.length - 1) { if (deserializeStream != null) { deserializeStream.close() - fileStream.close() + fileChannel.close() deserializeStream = null - fileStream = null + fileChannel = null } val start = batchOffsets(batchIndex) - fileStream = new FileInputStream(file) - fileStream.getChannel.position(start) + fileChannel = FileChannel.open(file.toPath, StandardOpenOption.READ) + fileChannel.position(start) batchIndex += 1 val end = batchOffsets(batchIndex) @@ -456,9 +493,10 @@ class ExternalAppendOnlyMap[K, V, C]( assert(end >= start, "start = " + start + ", end = " + end + ", batchOffsets = " + batchOffsets.mkString("[", ", ", "]")) - val bufferedStream = new BufferedInputStream(ByteStreams.limit(fileStream, end - start)) - val compressedStream = blockManager.wrapForCompression(blockId, bufferedStream) - ser.deserializeStream(compressedStream) + val bufferedStream = new BufferedInputStream( + ByteStreams.limit(Channels.newInputStream(fileChannel), end - start)) + val wrappedStream = serializerManager.wrapStream(blockId, bufferedStream) + ser.deserializeStream(wrappedStream) } else { // No more batches left cleanup() @@ -516,9 +554,9 @@ class ExternalAppendOnlyMap[K, V, C]( ds.close() deserializeStream = null } - if (fileStream != null) { - fileStream.close() - fileStream = null + if (fileChannel != null) { + fileChannel.close() + fileChannel = null } if (file.exists()) { if (!file.delete()) { @@ -530,8 +568,56 @@ class ExternalAppendOnlyMap[K, V, C]( context.addTaskCompletionListener(context => cleanup()) } + private[this] class SpillableIterator(var upstream: Iterator[(K, C)]) + extends Iterator[(K, C)] { + + private val SPILL_LOCK = new Object() + + private var nextUpstream: Iterator[(K, C)] = null + + private var cur: (K, C) = readNext() + + private var hasSpilled: Boolean = false + + def spill(): Boolean = SPILL_LOCK.synchronized { + if (hasSpilled) { + false + } else { + logInfo(s"Task ${context.taskAttemptId} force spilling in-memory map to disk and " + + s"it will release ${org.apache.spark.util.Utils.bytesToString(getUsed())} memory") + nextUpstream = spillMemoryIteratorToDisk(upstream) + hasSpilled = true + true + } + } + + def readNext(): (K, C) = SPILL_LOCK.synchronized { + if (nextUpstream != null) { + upstream = nextUpstream + nextUpstream = null + } + if (upstream.hasNext) { + upstream.next() + } else { + null + } + } + + override def hasNext(): Boolean = cur != null + + override def next(): (K, C) = { + val r = cur + cur = readNext() + r + } + } + /** Convenience function to hash the given (K, C) pair by the key. */ private def hashKey(kc: (K, C)): Int = ExternalAppendOnlyMap.hash(kc._1) + + override def toString(): String = { + this.getClass.getName + "@" + java.lang.Integer.toHexString(this.hashCode()) + } } private[spark] object ExternalAppendOnlyMap { 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 a44e72b7c16d..3593cfd50778 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 @@ -18,18 +18,19 @@ package org.apache.spark.util.collection import java.io._ +import java.nio.channels.{Channels, FileChannel} +import java.nio.file.StandardOpenOption import java.util.Comparator -import scala.collection.mutable.ArrayBuffer import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer -import com.google.common.annotations.VisibleForTesting import com.google.common.io.ByteStreams import org.apache.spark._ -import org.apache.spark.memory.TaskMemoryManager -import org.apache.spark.serializer._ import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.internal.Logging +import org.apache.spark.serializer._ import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter} /** @@ -68,35 +69,33 @@ import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter} * * At a high level, this class works internally as follows: * - * - We repeatedly fill up buffers of in-memory data, using either a PartitionedAppendOnlyMap if - * we want to combine by key, or a PartitionedPairBuffer if we don't. - * Inside these buffers, we sort elements by partition ID and then possibly also by key. - * To avoid calling the partitioner multiple times with each key, we store the partition ID - * alongside each record. + * - We repeatedly fill up buffers of in-memory data, using either a PartitionedAppendOnlyMap if + * we want to combine by key, or a PartitionedPairBuffer if we don't. + * Inside these buffers, we sort elements by partition ID and then possibly also by key. + * To avoid calling the partitioner multiple times with each key, we store the partition ID + * alongside each record. * - * - When each buffer reaches our memory limit, we spill it to a file. This file is sorted first - * by partition ID and possibly second by key or by hash code of the key, if we want to do - * aggregation. For each file, we track how many objects were in each partition in memory, so we - * don't have to write out the partition ID for every element. + * - When each buffer reaches our memory limit, we spill it to a file. This file is sorted first + * by partition ID and possibly second by key or by hash code of the key, if we want to do + * aggregation. For each file, we track how many objects were in each partition in memory, so we + * don't have to write out the partition ID for every element. * - * - When the user requests an iterator or file output, the spilled files are merged, along with - * any remaining in-memory data, using the same sort order defined above (unless both sorting - * and aggregation are disabled). If we need to aggregate by key, we either use a total ordering - * from the ordering parameter, or read the keys with the same hash code and compare them with - * each other for equality to merge values. + * - When the user requests an iterator or file output, the spilled files are merged, along with + * any remaining in-memory data, using the same sort order defined above (unless both sorting + * and aggregation are disabled). If we need to aggregate by key, we either use a total ordering + * from the ordering parameter, or read the keys with the same hash code and compare them with + * each other for equality to merge values. * - * - Users are expected to call stop() at the end to delete all the intermediate files. + * - Users are expected to call stop() at the end to delete all the intermediate files. */ private[spark] class ExternalSorter[K, V, C]( context: TaskContext, aggregator: Option[Aggregator[K, V, C]] = None, partitioner: Option[Partitioner] = None, ordering: Option[Ordering[K]] = None, - serializer: Option[Serializer] = None) - extends Logging - with Spillable[WritablePartitionedPairCollection[K, C]] { - - override protected[this] def taskMemoryManager: TaskMemoryManager = context.taskMemoryManager() + serializer: Serializer = SparkEnv.get.serializer) + extends Spillable[WritablePartitionedPairCollection[K, C]](context.taskMemoryManager()) + with Logging { private val conf = SparkEnv.get.conf @@ -108,8 +107,8 @@ private[spark] class ExternalSorter[K, V, C]( private val blockManager = SparkEnv.get.blockManager private val diskBlockManager = blockManager.diskBlockManager - private val ser = Serializer.getSerializer(serializer) - private val serInstance = ser.newInstance() + private val serializerManager = SparkEnv.get.serializerManager + private val serInstance = serializer.newInstance() // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided private val fileBufferSize = conf.getSizeAsKb("spark.shuffle.file.buffer", "32k").toInt * 1024 @@ -126,8 +125,8 @@ private[spark] class ExternalSorter[K, V, C]( // Data structures to store in-memory objects before we spill. Depending on whether we have an // Aggregator set, we either put objects into an AppendOnlyMap where we combine them, or we // store them in an array buffer. - private var map = new PartitionedAppendOnlyMap[K, C] - private var buffer = new PartitionedPairBuffer[K, C] + @volatile private var map = new PartitionedAppendOnlyMap[K, C] + @volatile private var buffer = new PartitionedPairBuffer[K, C] // Total spilling statistics private var _diskBytesSpilled = 0L @@ -137,6 +136,10 @@ private[spark] class ExternalSorter[K, V, C]( private var _peakMemoryUsedBytes: Long = 0L def peakMemoryUsedBytes: Long = _peakMemoryUsedBytes + @volatile private var isShuffleSort: Boolean = true + private val forceSpillFiles = new ArrayBuffer[SpilledFile] + @volatile private var readingIterator: SpillableIterator = null + // A comparator for keys K that orders them within a partition to allow aggregation or sorting. // Can be a partial ordering by hash code if a total ordering is not provided through by the // user. (A partial ordering means that equal keys have comparator.compare(k, k) = 0, but some @@ -235,6 +238,34 @@ private[spark] class ExternalSorter[K, V, C]( * @param collection whichever collection we're using (map or buffer) */ override protected[this] def spill(collection: WritablePartitionedPairCollection[K, C]): Unit = { + val inMemoryIterator = collection.destructiveSortedWritablePartitionedIterator(comparator) + val spillFile = spillMemoryIteratorToDisk(inMemoryIterator) + spills += spillFile + } + + /** + * Force to spilling the current in-memory collection to disk to release memory, + * It will be called by TaskMemoryManager when there is not enough memory for the task. + */ + override protected[this] def forceSpill(): Boolean = { + if (isShuffleSort) { + false + } else { + assert(readingIterator != null) + val isSpilled = readingIterator.spill() + if (isSpilled) { + map = null + buffer = null + } + isSpilled + } + } + + /** + * Spill contents of in-memory iterator to a temporary file on disk. + */ + private[this] def spillMemoryIteratorToDisk(inMemoryIterator: WritablePartitionedIterator) + : SpilledFile = { // Because these files may be read during shuffle, their compression must be controlled by // spark.shuffle.compress instead of spark.shuffle.spill.compress, so we need to use // createTempShuffleBlock here; see SPARK-3426 for more context. @@ -242,14 +273,9 @@ private[spark] class ExternalSorter[K, V, C]( // These variables are reset after each flush var objectsWritten: Long = 0 - var spillMetrics: ShuffleWriteMetrics = null - var writer: DiskBlockObjectWriter = null - def openWriter(): Unit = { - assert (writer == null && spillMetrics == null) - spillMetrics = new ShuffleWriteMetrics - writer = blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, spillMetrics) - } - openWriter() + val spillMetrics: ShuffleWriteMetrics = new ShuffleWriteMetrics + val writer: DiskBlockObjectWriter = + blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, spillMetrics) // List of batch sizes (bytes) in the order they are written to disk val batchSizes = new ArrayBuffer[Long] @@ -258,48 +284,41 @@ private[spark] class ExternalSorter[K, V, C]( val elementsPerPartition = new Array[Long](numPartitions) // Flush the disk writer's contents to disk, and update relevant variables. - // The writer is closed at the end of this process, and cannot be reused. + // The writer is committed at the end of this process. def flush(): Unit = { - val w = writer - writer = null - w.commitAndClose() - _diskBytesSpilled += spillMetrics.shuffleBytesWritten - batchSizes.append(spillMetrics.shuffleBytesWritten) - spillMetrics = null + val segment = writer.commitAndGet() + batchSizes += segment.length + _diskBytesSpilled += segment.length objectsWritten = 0 } var success = false try { - val it = collection.destructiveSortedWritablePartitionedIterator(comparator) - while (it.hasNext) { - val partitionId = it.nextPartition() + while (inMemoryIterator.hasNext) { + val partitionId = inMemoryIterator.nextPartition() require(partitionId >= 0 && partitionId < numPartitions, s"partition Id: ${partitionId} should be in the range [0, ${numPartitions})") - it.writeNext(writer) + inMemoryIterator.writeNext(writer) elementsPerPartition(partitionId) += 1 objectsWritten += 1 if (objectsWritten == serializerBatchSize) { flush() - openWriter() } } if (objectsWritten > 0) { flush() - } else if (writer != null) { - val w = writer - writer = null - w.revertPartialWritesAndClose() + } else { + writer.revertPartialWritesAndClose() } success = true } finally { - if (!success) { + if (success) { + writer.close() + } else { // This code path only happens if an exception was thrown above before we set success; // close our stuff and let the exception be thrown further - if (writer != null) { - writer.revertPartialWritesAndClose() - } + writer.revertPartialWritesAndClose() if (file.exists()) { if (!file.delete()) { logWarning(s"Error deleting ${file}") @@ -308,7 +327,7 @@ private[spark] class ExternalSorter[K, V, C]( } } - spills.append(SpilledFile(file, blockId, batchSizes.toArray, elementsPerPartition)) + SpilledFile(file, blockId, batchSizes.toArray, elementsPerPartition) } /** @@ -475,7 +494,7 @@ private[spark] class ExternalSorter[K, V, C]( // Intermediate file and deserializer streams that read from exactly one batch // This guards against pre-fetching and other arbitrary behavior of higher level streams - var fileStream: FileInputStream = null + var fileChannel: FileChannel = null var deserializeStream = nextBatchStream() // Also sets fileStream var nextItem: (K, C) = null @@ -488,14 +507,14 @@ private[spark] class ExternalSorter[K, V, C]( if (batchId < batchOffsets.length - 1) { if (deserializeStream != null) { deserializeStream.close() - fileStream.close() + fileChannel.close() deserializeStream = null - fileStream = null + fileChannel = null } val start = batchOffsets(batchId) - fileStream = new FileInputStream(spill.file) - fileStream.getChannel.position(start) + fileChannel = FileChannel.open(spill.file.toPath, StandardOpenOption.READ) + fileChannel.position(start) batchId += 1 val end = batchOffsets(batchId) @@ -503,9 +522,11 @@ private[spark] class ExternalSorter[K, V, C]( assert(end >= start, "start = " + start + ", end = " + end + ", batchOffsets = " + batchOffsets.mkString("[", ", ", "]")) - val bufferedStream = new BufferedInputStream(ByteStreams.limit(fileStream, end - start)) - val compressedStream = blockManager.wrapForCompression(spill.blockId, bufferedStream) - serInstance.deserializeStream(compressedStream) + val bufferedStream = new BufferedInputStream( + ByteStreams.limit(Channels.newInputStream(fileChannel), end - start)) + + val wrappedStream = serializerManager.wrapStream(spill.blockId, bufferedStream) + serInstance.deserializeStream(wrappedStream) } else { // No more batches left cleanup() @@ -592,13 +613,29 @@ private[spark] class ExternalSorter[K, V, C]( batchId = batchOffsets.length // Prevent reading any other batch val ds = deserializeStream deserializeStream = null - fileStream = null - ds.close() + fileChannel = null + if (ds != null) { + ds.close() + } // NOTE: We don't do file.delete() here because that is done in ExternalSorter.stop(). // This should also be fixed in ExternalAppendOnlyMap. } } + /** + * Returns a destructive iterator for iterating over the entries of this map. + * If this iterator is forced spill to disk to release memory when there is not enough memory, + * it returns pairs from an on-disk map. + */ + def destructiveIterator(memoryIterator: Iterator[((Int, K), C)]): Iterator[((Int, K), C)] = { + if (isShuffleSort) { + memoryIterator + } else { + readingIterator = new SpillableIterator(memoryIterator) + readingIterator + } + } + /** * Return an iterator over all the data written to this object, grouped by partition and * aggregated by the requested aggregator. For each partition we then have an iterator over its @@ -608,8 +645,8 @@ private[spark] class ExternalSorter[K, V, C]( * * For now, we just merge all the spilled files in once pass, but this can be modified to * support hierarchical merging. + * Exposed for testing. */ - @VisibleForTesting def partitionedIterator: Iterator[(Int, Iterator[Product2[K, C]])] = { val usingMap = aggregator.isDefined val collection: WritablePartitionedPairCollection[K, C] = if (usingMap) map else buffer @@ -618,28 +655,32 @@ private[spark] class ExternalSorter[K, V, C]( // we don't even need to sort by anything other than partition ID if (!ordering.isDefined) { // The user hasn't requested sorted keys, so only sort by partition ID, not key - groupByPartition(collection.partitionedDestructiveSortedIterator(None)) + groupByPartition(destructiveIterator(collection.partitionedDestructiveSortedIterator(None))) } else { // We do need to sort by both partition ID and key - groupByPartition(collection.partitionedDestructiveSortedIterator(Some(keyComparator))) + groupByPartition(destructiveIterator( + collection.partitionedDestructiveSortedIterator(Some(keyComparator)))) } } else { // Merge spilled and in-memory data - merge(spills, collection.partitionedDestructiveSortedIterator(comparator)) + merge(spills, destructiveIterator( + collection.partitionedDestructiveSortedIterator(comparator))) } } /** * Return an iterator over all the data written to this object, aggregated by our aggregator. */ - def iterator: Iterator[Product2[K, C]] = partitionedIterator.flatMap(pair => pair._2) + def iterator: Iterator[Product2[K, C]] = { + isShuffleSort = false + partitionedIterator.flatMap(pair => pair._2) + } /** * Write all the data added into this ExternalSorter into a file in the disk store. This is * called by the SortShuffleWriter. * * @param blockId block ID to write to. The index file will be blockId.name + ".index". - * @param context a TaskContext for a running Spark task, for us to update shuffle metrics. * @return array of lengths, in bytes, of each partition of the file (used by map output tracker) */ def writePartitionedFile( @@ -648,52 +689,52 @@ private[spark] class ExternalSorter[K, V, C]( // Track location of each range in the output file val lengths = new Array[Long](numPartitions) + val writer = blockManager.getDiskWriter(blockId, outputFile, serInstance, fileBufferSize, + context.taskMetrics().shuffleWriteMetrics) if (spills.isEmpty) { // Case where we only have in-memory data val collection = if (aggregator.isDefined) map else buffer val it = collection.destructiveSortedWritablePartitionedIterator(comparator) while (it.hasNext) { - val writer = blockManager.getDiskWriter(blockId, outputFile, serInstance, fileBufferSize, - context.taskMetrics.shuffleWriteMetrics.get) val partitionId = it.nextPartition() while (it.hasNext && it.nextPartition() == partitionId) { it.writeNext(writer) } - writer.commitAndClose() - val segment = writer.fileSegment() + val segment = writer.commitAndGet() lengths(partitionId) = segment.length } } else { // We must perform merge-sort; get an iterator by partition and write everything directly. for ((id, elements) <- this.partitionedIterator) { if (elements.hasNext) { - val writer = blockManager.getDiskWriter(blockId, outputFile, serInstance, fileBufferSize, - context.taskMetrics.shuffleWriteMetrics.get) for (elem <- elements) { writer.write(elem._1, elem._2) } - writer.commitAndClose() - val segment = writer.fileSegment() + val segment = writer.commitAndGet() lengths(id) = segment.length } } } + writer.close() context.taskMetrics().incMemoryBytesSpilled(memoryBytesSpilled) context.taskMetrics().incDiskBytesSpilled(diskBytesSpilled) - context.internalMetricsToAccumulators( - InternalAccumulator.PEAK_EXECUTION_MEMORY).add(peakMemoryUsedBytes) + context.taskMetrics().incPeakExecutionMemory(peakMemoryUsedBytes) lengths } def stop(): Unit = { - map = null // So that the memory can be garbage-collected - buffer = null // So that the memory can be garbage-collected spills.foreach(s => s.file.delete()) spills.clear() - releaseMemory() + forceSpillFiles.foreach(s => s.file.delete()) + forceSpillFiles.clear() + if (map != null || buffer != null) { + map = null // So that the memory can be garbage-collected + buffer = null // So that the memory can be garbage-collected + releaseMemory() + } } /** @@ -727,4 +768,66 @@ private[spark] class ExternalSorter[K, V, C]( (elem._1._2, elem._2) } } + + private[this] class SpillableIterator(var upstream: Iterator[((Int, K), C)]) + extends Iterator[((Int, K), C)] { + + private val SPILL_LOCK = new Object() + + private var nextUpstream: Iterator[((Int, K), C)] = null + + private var cur: ((Int, K), C) = readNext() + + private var hasSpilled: Boolean = false + + def spill(): Boolean = SPILL_LOCK.synchronized { + if (hasSpilled) { + false + } else { + val inMemoryIterator = new WritablePartitionedIterator { + private[this] var cur = if (upstream.hasNext) upstream.next() else null + + def writeNext(writer: DiskBlockObjectWriter): Unit = { + writer.write(cur._1._2, cur._2) + cur = if (upstream.hasNext) upstream.next() else null + } + + def hasNext(): Boolean = cur != null + + def nextPartition(): Int = cur._1._1 + } + logInfo(s"Task ${context.taskAttemptId} force spilling in-memory map to disk and " + + s" it will release ${org.apache.spark.util.Utils.bytesToString(getUsed())} memory") + val spillFile = spillMemoryIteratorToDisk(inMemoryIterator) + forceSpillFiles += spillFile + val spillReader = new SpillReader(spillFile) + nextUpstream = (0 until numPartitions).iterator.flatMap { p => + val iterator = spillReader.readNextPartition() + iterator.map(cur => ((p, cur._1), cur._2)) + } + hasSpilled = true + true + } + } + + def readNext(): ((Int, K), C) = SPILL_LOCK.synchronized { + if (nextUpstream != null) { + upstream = nextUpstream + nextUpstream = null + } + if (upstream.hasNext) { + upstream.next() + } else { + null + } + } + + override def hasNext(): Boolean = cur != null + + override def next(): ((Int, K), C) = { + val r = cur + cur = readNext() + r + } + } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/MedianHeap.scala b/core/src/main/scala/org/apache/spark/util/collection/MedianHeap.scala new file mode 100644 index 000000000000..6e57c3c5bee8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/MedianHeap.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import scala.collection.mutable.PriorityQueue + +/** + * MedianHeap is designed to be used to quickly track the median of a group of numbers + * that may contain duplicates. Inserting a new number has O(log n) time complexity and + * determining the median has O(1) time complexity. + * The basic idea is to maintain two heaps: a smallerHalf and a largerHalf. The smallerHalf + * stores the smaller half of all numbers while the largerHalf stores the larger half. + * The sizes of two heaps need to be balanced each time when a new number is inserted so + * that their sizes will not be different by more than 1. Therefore each time when + * findMedian() is called we check if two heaps have the same size. If they do, we should + * return the average of the two top values of heaps. Otherwise we return the top of the + * heap which has one more element. + */ +private[spark] class MedianHeap(implicit val ord: Ordering[Double]) { + + /** + * Stores all the numbers less than the current median in a smallerHalf, + * i.e median is the maximum, at the root. + */ + private[this] var smallerHalf = PriorityQueue.empty[Double](ord) + + /** + * Stores all the numbers greater than the current median in a largerHalf, + * i.e median is the minimum, at the root. + */ + private[this] var largerHalf = PriorityQueue.empty[Double](ord.reverse) + + def isEmpty(): Boolean = { + smallerHalf.isEmpty && largerHalf.isEmpty + } + + def size(): Int = { + smallerHalf.size + largerHalf.size + } + + def insert(x: Double): Unit = { + // If both heaps are empty, we arbitrarily insert it into a heap, let's say, the largerHalf. + if (isEmpty) { + largerHalf.enqueue(x) + } else { + // If the number is larger than current median, it should be inserted into largerHalf, + // otherwise smallerHalf. + if (x > median) { + largerHalf.enqueue(x) + } else { + smallerHalf.enqueue(x) + } + } + rebalance() + } + + private[this] def rebalance(): Unit = { + if (largerHalf.size - smallerHalf.size > 1) { + smallerHalf.enqueue(largerHalf.dequeue()) + } + if (smallerHalf.size - largerHalf.size > 1) { + largerHalf.enqueue(smallerHalf.dequeue) + } + } + + def median: Double = { + if (isEmpty) { + throw new NoSuchElementException("MedianHeap is empty.") + } + if (largerHalf.size == smallerHalf.size) { + (largerHalf.head + smallerHalf.head) / 2.0 + } else if (largerHalf.size > smallerHalf.size) { + largerHalf.head + } else { + smallerHalf.head + } + } +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index efc2482c74dd..10ab0b3f8996 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -19,17 +19,16 @@ package org.apache.spark.util.collection import scala.reflect.ClassTag -import org.apache.spark.annotation.DeveloperApi - /** - * :: DeveloperApi :: * A fast hash map implementation for nullable keys. This hash map supports insertions and updates, * but not deletions. This map is about 5X faster than java.util.HashMap, while using much less * space overhead. * * Under the hood, it uses our OpenHashSet implementation. + * + * NOTE: when using numeric type as the value type, the user of this class should be careful to + * distinguish between the 0/0.0/0L and non-exist value */ -@DeveloperApi private[spark] class OpenHashMap[K : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( initialCapacity: Int) diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index 60bf4dd7469f..60f6f537c1d5 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -18,6 +18,7 @@ package org.apache.spark.util.collection import scala.reflect._ + import com.google.common.hash.Hashing import org.apache.spark.annotation.Private @@ -47,7 +48,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( require(initialCapacity <= OpenHashSet.MAX_CAPACITY, s"Can't make capacity bigger than ${OpenHashSet.MAX_CAPACITY} elements") - require(initialCapacity >= 1, "Invalid initial capacity") + require(initialCapacity >= 0, "Invalid initial capacity") require(loadFactor < 1.0, "Load factor must be less than 1.0") require(loadFactor > 0.0, "Load factor must be greater than 0.0") @@ -270,8 +271,12 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( private def hashcode(h: Int): Int = Hashing.murmur3_32().hashInt(h).asInt() private def nextPowerOf2(n: Int): Int = { - val highBit = Integer.highestOneBit(n) - if (highBit == n) n else highBit << 1 + if (n == 0) { + 1 + } else { + val highBit = Integer.highestOneBit(n) + if (highBit == n) n else highBit << 1 + } } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala index f5844d5353be..b755e5da5168 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala @@ -25,7 +25,7 @@ import org.apache.spark.util.collection.WritablePartitionedPairCollection._ * Append-only buffer of key-value pairs, each with a corresponding partition ID, that keeps track * of its estimated size in bytes. * - * The buffer can support up to `1073741823 (2 ^ 30 - 1)` elements. + * The buffer can support up to 1073741819 elements. */ private[spark] class PartitionedPairBuffer[K, V](initialCapacity: Int = 64) extends WritablePartitionedPairCollection[K, V] with SizeTracker @@ -59,7 +59,7 @@ private[spark] class PartitionedPairBuffer[K, V](initialCapacity: Int = 64) throw new IllegalStateException(s"Can't insert more than ${MAXIMUM_CAPACITY} elements") } val newCapacity = - if (capacity * 2 < 0 || capacity * 2 > MAXIMUM_CAPACITY) { // Overflow + if (capacity * 2 > MAXIMUM_CAPACITY) { // Overflow MAXIMUM_CAPACITY } else { capacity * 2 @@ -96,5 +96,7 @@ private[spark] class PartitionedPairBuffer[K, V](initialCapacity: Int = 64) } private object PartitionedPairBuffer { - val MAXIMUM_CAPACITY = Int.MaxValue / 2 // 2 ^ 30 - 1 + // Some JVMs can't allocate arrays of length Integer.MAX_VALUE; actual max is somewhat + // smaller. Be conservative and lower the cap a little. + val MAXIMUM_CAPACITY: Int = (Int.MaxValue - 8) / 2 } diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala index 9e002621a690..8183f825592c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -17,14 +17,16 @@ package org.apache.spark.util.collection -import org.apache.spark.memory.TaskMemoryManager -import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.SparkEnv +import org.apache.spark.internal.Logging +import org.apache.spark.memory.{MemoryConsumer, MemoryMode, TaskMemoryManager} /** * Spills contents of an in-memory collection to disk when the memory threshold * has been exceeded. */ -private[spark] trait Spillable[C] extends Logging { +private[spark] abstract class Spillable[C](taskMemoryManager: TaskMemoryManager) + extends MemoryConsumer(taskMemoryManager) with Logging { /** * Spills the current in-memory collection to disk, and releases the memory. * @@ -32,6 +34,12 @@ private[spark] trait Spillable[C] extends Logging { */ protected def spill(collection: C): Unit + /** + * Force to spilling the current in-memory collection to disk to release memory, + * It will be called by TaskMemoryManager when there is not enough memory for the task. + */ + protected def forceSpill(): Boolean + // Number of elements read from input since last spill protected def elementsRead: Long = _elementsRead @@ -39,9 +47,6 @@ private[spark] trait Spillable[C] extends Logging { // It's used for checking spilling frequency protected def addElementsRead(): Unit = { _elementsRead += 1 } - // Memory manager that can be used to acquire/release memory - protected[this] def taskMemoryManager: TaskMemoryManager - // Initial threshold for the size of a collection before we start tracking its memory usage // For testing only private[this] val initialMemoryThreshold: Long = @@ -54,13 +59,13 @@ private[spark] trait Spillable[C] extends Logging { // Threshold for this collection's size in bytes before we start tracking its memory usage // To avoid a large number of small spills, initialize this to a value orders of magnitude > 0 - private[this] var myMemoryThreshold = initialMemoryThreshold + @volatile private[this] var myMemoryThreshold = initialMemoryThreshold // Number of elements read from input since last spill private[this] var _elementsRead = 0L // Number of bytes spilled in total - private[this] var _memoryBytesSpilled = 0L + @volatile private[this] var _memoryBytesSpilled = 0L // Number of spills private[this] var _spillCount = 0 @@ -78,7 +83,7 @@ private[spark] trait Spillable[C] extends Logging { if (elementsRead % 32 == 0 && currentMemory >= myMemoryThreshold) { // Claim up to double our current memory from the shuffle memory pool val amountToRequest = 2 * currentMemory - myMemoryThreshold - val granted = taskMemoryManager.acquireExecutionMemory(amountToRequest, null) + val granted = acquireMemory(amountToRequest) myMemoryThreshold += granted // If we were granted too little memory to grow further (either tryToAcquire returned 0, // or we already had more memory than myMemoryThreshold), spill the current collection @@ -97,6 +102,26 @@ private[spark] trait Spillable[C] extends Logging { shouldSpill } + /** + * Spill some data to disk to release memory, which will be called by TaskMemoryManager + * when there is not enough memory for the task. + */ + override def spill(size: Long, trigger: MemoryConsumer): Long = { + if (trigger != this && taskMemoryManager.getTungstenMemoryMode == MemoryMode.ON_HEAP) { + val isSpilled = forceSpill() + if (!isSpilled) { + 0L + } else { + val freeMemory = myMemoryThreshold - initialMemoryThreshold + _memoryBytesSpilled += freeMemory + releaseMemory() + freeMemory + } + } else { + 0L + } + } + /** * @return number of bytes spilled in total */ @@ -106,8 +131,7 @@ private[spark] trait Spillable[C] extends Logging { * Release our memory back to the execution pool so that other tasks can grab it. */ def releaseMemory(): Unit = { - // The amount we requested does not include the initial memory tracking threshold - taskMemoryManager.releaseExecutionMemory(myMemoryThreshold - initialMemoryThreshold, null) + freeMemory(myMemoryThreshold - initialMemoryThreshold) myMemoryThreshold = initialMemoryThreshold } diff --git a/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala b/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala index 38848e9018c6..5232c2bd8d6f 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala @@ -23,9 +23,10 @@ import org.apache.spark.storage.DiskBlockObjectWriter /** * A common interface for size-tracking collections of key-value pairs that - * - Have an associated partition for each key-value pair. - * - Support a memory-efficient sorted iterator - * - Support a WritablePartitionedIterator for writing the contents directly as bytes. + * + * - Have an associated partition for each key-value pair. + * - Support a memory-efficient sorted iterator + * - Support a WritablePartitionedIterator for writing the contents directly as bytes. */ private[spark] trait WritablePartitionedPairCollection[K, V] { /** diff --git a/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala b/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala deleted file mode 100644 index daac6f971eb2..000000000000 --- a/core/src/main/scala/org/apache/spark/util/io/ByteArrayChunkOutputStream.scala +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util.io - -import java.io.OutputStream - -import scala.collection.mutable.ArrayBuffer - - -/** - * An OutputStream that writes to fixed-size chunks of byte arrays. - * - * @param chunkSize size of each chunk, in bytes. - */ -private[spark] -class ByteArrayChunkOutputStream(chunkSize: Int) extends OutputStream { - - private val chunks = new ArrayBuffer[Array[Byte]] - - /** Index of the last chunk. Starting with -1 when the chunks array is empty. */ - private var lastChunkIndex = -1 - - /** - * Next position to write in the last chunk. - * - * If this equals chunkSize, it means for next write we need to allocate a new chunk. - * This can also never be 0. - */ - private var position = chunkSize - - override def write(b: Int): Unit = { - allocateNewChunkIfNeeded() - chunks(lastChunkIndex)(position) = b.toByte - position += 1 - } - - override def write(bytes: Array[Byte], off: Int, len: Int): Unit = { - var written = 0 - while (written < len) { - allocateNewChunkIfNeeded() - val thisBatch = math.min(chunkSize - position, len - written) - System.arraycopy(bytes, written + off, chunks(lastChunkIndex), position, thisBatch) - written += thisBatch - position += thisBatch - } - } - - @inline - private def allocateNewChunkIfNeeded(): Unit = { - if (position == chunkSize) { - chunks += new Array[Byte](chunkSize) - lastChunkIndex += 1 - position = 0 - } - } - - def toArrays: Array[Array[Byte]] = { - if (lastChunkIndex == -1) { - new Array[Array[Byte]](0) - } else { - // Copy the first n-1 chunks to the output, and then create an array that fits the last chunk. - // An alternative would have been returning an array of ByteBuffers, with the last buffer - // bounded to only the last chunk's position. However, given our use case in Spark (to put - // the chunks in block manager), only limiting the view bound of the buffer would still - // require the block manager to store the whole chunk. - val ret = new Array[Array[Byte]](chunks.size) - for (i <- 0 until chunks.size - 1) { - ret(i) = chunks(i) - } - if (position == chunkSize) { - ret(lastChunkIndex) = chunks(lastChunkIndex) - } else { - ret(lastChunkIndex) = new Array[Byte](position) - System.arraycopy(chunks(lastChunkIndex), 0, ret(lastChunkIndex), 0, position) - } - ret - } - } -} diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala new file mode 100644 index 000000000000..c28570fb2456 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala @@ -0,0 +1,231 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.io + +import java.io.InputStream +import java.nio.ByteBuffer +import java.nio.channels.WritableByteChannel + +import com.google.common.primitives.UnsignedBytes +import io.netty.buffer.{ByteBuf, Unpooled} + +import org.apache.spark.SparkEnv +import org.apache.spark.internal.config +import org.apache.spark.network.util.ByteArrayWritableChannel +import org.apache.spark.storage.StorageUtils + +/** + * Read-only byte buffer which is physically stored as multiple chunks rather than a single + * contiguous array. + * + * @param chunks an array of [[ByteBuffer]]s. Each buffer in this array must have position == 0. + * Ownership of these buffers is transferred to the ChunkedByteBuffer, so if these + * buffers may also be used elsewhere then the caller is responsible for copying + * them as needed. + */ +private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) { + require(chunks != null, "chunks must not be null") + require(chunks.forall(_.position() == 0), "chunks' positions must be 0") + + // Chunk size in bytes + private val bufferWriteChunkSize = + Option(SparkEnv.get).map(_.conf.get(config.BUFFER_WRITE_CHUNK_SIZE)) + .getOrElse(config.BUFFER_WRITE_CHUNK_SIZE.defaultValue.get).toInt + + private[this] var disposed: Boolean = false + + /** + * This size of this buffer, in bytes. + */ + val size: Long = chunks.map(_.limit().asInstanceOf[Long]).sum + + def this(byteBuffer: ByteBuffer) = { + this(Array(byteBuffer)) + } + + /** + * Write this buffer to a channel. + */ + def writeFully(channel: WritableByteChannel): Unit = { + for (bytes <- getChunks()) { + while (bytes.remaining() > 0) { + val ioSize = Math.min(bytes.remaining(), bufferWriteChunkSize) + bytes.limit(bytes.position + ioSize) + channel.write(bytes) + } + } + } + + /** + * Wrap this buffer to view it as a Netty ByteBuf. + */ + def toNetty: ByteBuf = { + Unpooled.wrappedBuffer(chunks.length, getChunks(): _*) + } + + /** + * Copy this buffer into a new byte array. + * + * @throws UnsupportedOperationException if this buffer's size exceeds the maximum array size. + */ + def toArray: Array[Byte] = { + if (size >= Integer.MAX_VALUE) { + throw new UnsupportedOperationException( + s"cannot call toArray because buffer size ($size bytes) exceeds maximum array size") + } + val byteChannel = new ByteArrayWritableChannel(size.toInt) + writeFully(byteChannel) + byteChannel.close() + byteChannel.getData + } + + /** + * Convert this buffer to a ByteBuffer. If this buffer is backed by a single chunk, its underlying + * data will not be copied. Instead, it will be duplicated. If this buffer is backed by multiple + * chunks, the data underlying this buffer will be copied into a new byte buffer. As a result, it + * is suggested to use this method only if the caller does not need to manage the memory + * underlying this buffer. + * + * @throws UnsupportedOperationException if this buffer's size exceeds the max ByteBuffer size. + */ + def toByteBuffer: ByteBuffer = { + if (chunks.length == 1) { + chunks.head.duplicate() + } else { + ByteBuffer.wrap(toArray) + } + } + + /** + * Creates an input stream to read data from this ChunkedByteBuffer. + * + * @param dispose if true, [[dispose()]] will be called at the end of the stream + * in order to close any memory-mapped files which back this buffer. + */ + def toInputStream(dispose: Boolean = false): InputStream = { + new ChunkedByteBufferInputStream(this, dispose) + } + + /** + * Get duplicates of the ByteBuffers backing this ChunkedByteBuffer. + */ + def getChunks(): Array[ByteBuffer] = { + chunks.map(_.duplicate()) + } + + /** + * Make a copy of this ChunkedByteBuffer, copying all of the backing data into new buffers. + * The new buffer will share no resources with the original buffer. + * + * @param allocator a method for allocating byte buffers + */ + def copy(allocator: Int => ByteBuffer): ChunkedByteBuffer = { + val copiedChunks = getChunks().map { chunk => + val newChunk = allocator(chunk.limit()) + newChunk.put(chunk) + newChunk.flip() + newChunk + } + new ChunkedByteBuffer(copiedChunks) + } + + /** + * Attempt to clean up any ByteBuffer in this ChunkedByteBuffer which is direct or memory-mapped. + * See [[StorageUtils.dispose]] for more information. + */ + def dispose(): Unit = { + if (!disposed) { + chunks.foreach(StorageUtils.dispose) + disposed = true + } + } + +} + +/** + * Reads data from a ChunkedByteBuffer. + * + * @param dispose if true, `ChunkedByteBuffer.dispose()` will be called at the end of the stream + * in order to close any memory-mapped files which back the buffer. + */ +private[spark] class ChunkedByteBufferInputStream( + var chunkedByteBuffer: ChunkedByteBuffer, + dispose: Boolean) + extends InputStream { + + private[this] var chunks = chunkedByteBuffer.getChunks().iterator + private[this] var currentChunk: ByteBuffer = { + if (chunks.hasNext) { + chunks.next() + } else { + null + } + } + + override def read(): Int = { + if (currentChunk != null && !currentChunk.hasRemaining && chunks.hasNext) { + currentChunk = chunks.next() + } + if (currentChunk != null && currentChunk.hasRemaining) { + UnsignedBytes.toInt(currentChunk.get()) + } else { + close() + -1 + } + } + + override def read(dest: Array[Byte], offset: Int, length: Int): Int = { + if (currentChunk != null && !currentChunk.hasRemaining && chunks.hasNext) { + currentChunk = chunks.next() + } + if (currentChunk != null && currentChunk.hasRemaining) { + val amountToGet = math.min(currentChunk.remaining(), length) + currentChunk.get(dest, offset, amountToGet) + amountToGet + } else { + close() + -1 + } + } + + override def skip(bytes: Long): Long = { + if (currentChunk != null) { + val amountToSkip = math.min(bytes, currentChunk.remaining).toInt + currentChunk.position(currentChunk.position + amountToSkip) + if (currentChunk.remaining() == 0) { + if (chunks.hasNext) { + currentChunk = chunks.next() + } else { + close() + } + } + amountToSkip + } else { + 0L + } + } + + override def close(): Unit = { + if (chunkedByteBuffer != null && dispose) { + chunkedByteBuffer.dispose() + } + chunkedByteBuffer = null + chunks = null + currentChunk = null + } +} diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala new file mode 100644 index 000000000000..a625b3289538 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.io + +import java.io.OutputStream +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.storage.StorageUtils + +/** + * An OutputStream that writes to fixed-size chunks of byte arrays. + * + * @param chunkSize size of each chunk, in bytes. + */ +private[spark] class ChunkedByteBufferOutputStream( + chunkSize: Int, + allocator: Int => ByteBuffer) + extends OutputStream { + + private[this] var toChunkedByteBufferWasCalled = false + + private val chunks = new ArrayBuffer[ByteBuffer] + + /** Index of the last chunk. Starting with -1 when the chunks array is empty. */ + private[this] var lastChunkIndex = -1 + + /** + * Next position to write in the last chunk. + * + * If this equals chunkSize, it means for next write we need to allocate a new chunk. + * This can also never be 0. + */ + private[this] var position = chunkSize + private[this] var _size = 0 + private[this] var closed: Boolean = false + + def size: Long = _size + + override def close(): Unit = { + if (!closed) { + super.close() + closed = true + } + } + + override def write(b: Int): Unit = { + require(!closed, "cannot write to a closed ChunkedByteBufferOutputStream") + allocateNewChunkIfNeeded() + chunks(lastChunkIndex).put(b.toByte) + position += 1 + _size += 1 + } + + override def write(bytes: Array[Byte], off: Int, len: Int): Unit = { + require(!closed, "cannot write to a closed ChunkedByteBufferOutputStream") + var written = 0 + while (written < len) { + allocateNewChunkIfNeeded() + val thisBatch = math.min(chunkSize - position, len - written) + chunks(lastChunkIndex).put(bytes, written + off, thisBatch) + written += thisBatch + position += thisBatch + } + _size += len + } + + @inline + private def allocateNewChunkIfNeeded(): Unit = { + if (position == chunkSize) { + chunks += allocator(chunkSize) + lastChunkIndex += 1 + position = 0 + } + } + + def toChunkedByteBuffer: ChunkedByteBuffer = { + require(closed, "cannot call toChunkedByteBuffer() unless close() has been called") + require(!toChunkedByteBufferWasCalled, "toChunkedByteBuffer() can only be called once") + toChunkedByteBufferWasCalled = true + if (lastChunkIndex == -1) { + new ChunkedByteBuffer(Array.empty[ByteBuffer]) + } else { + // Copy the first n-1 chunks to the output, and then create an array that fits the last chunk. + // An alternative would have been returning an array of ByteBuffers, with the last buffer + // bounded to only the last chunk's position. However, given our use case in Spark (to put + // the chunks in block manager), only limiting the view bound of the buffer would still + // require the block manager to store the whole chunk. + val ret = new Array[ByteBuffer](chunks.size) + for (i <- 0 until chunks.size - 1) { + ret(i) = chunks(i) + ret(i).flip() + } + if (position == chunkSize) { + ret(lastChunkIndex) = chunks(lastChunkIndex) + ret(lastChunkIndex).flip() + } else { + ret(lastChunkIndex) = allocator(position) + chunks(lastChunkIndex).flip() + ret(lastChunkIndex).put(chunks(lastChunkIndex)) + ret(lastChunkIndex).flip() + StorageUtils.dispose(chunks(lastChunkIndex)) + } + new ChunkedByteBuffer(ret) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala index 14b6ba4af489..2f9ad4c8cc3e 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala @@ -17,9 +17,10 @@ package org.apache.spark.util.logging -import java.io.{File, FileOutputStream, InputStream} +import java.io.{File, FileOutputStream, InputStream, IOException} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.util.{IntParam, Utils} /** @@ -29,7 +30,6 @@ private[spark] class FileAppender(inputStream: InputStream, file: File, bufferSi extends Logging { @volatile private var outputStream: FileOutputStream = null @volatile private var markedForStop = false // has the appender been asked to stopped - @volatile private var stopped = false // has the appender stopped // Thread that reads the input stream and writes to file private val writingThread = new Thread("File appending thread for " + file) { @@ -47,11 +47,7 @@ private[spark] class FileAppender(inputStream: InputStream, file: File, bufferSi * or because of any error in appending */ def awaitTermination() { - synchronized { - if (!stopped) { - wait() - } - } + writingThread.join() } /** Stop the appender */ @@ -63,24 +59,28 @@ private[spark] class FileAppender(inputStream: InputStream, file: File, bufferSi protected def appendStreamToFile() { try { logDebug("Started appending thread") - openFile() - val buf = new Array[Byte](bufferSize) - var n = 0 - while (!markedForStop && n != -1) { - n = inputStream.read(buf) - if (n != -1) { - appendToFile(buf, n) + Utils.tryWithSafeFinally { + openFile() + val buf = new Array[Byte](bufferSize) + var n = 0 + while (!markedForStop && n != -1) { + try { + n = inputStream.read(buf) + } catch { + // An InputStream can throw IOException during read if the stream is closed + // asynchronously, so once appender has been flagged to stop these will be ignored + case _: IOException if markedForStop => // do nothing and proceed to stop appending + } + if (n > 0) { + appendToFile(buf, n) + } } + } { + closeFile() } } catch { case e: Exception => logError(s"Error writing stream to file $file", e) - } finally { - closeFile() - synchronized { - stopped = true - notifyAll() - } } } @@ -94,7 +94,7 @@ private[spark] class FileAppender(inputStream: InputStream, file: File, bufferSi /** Open the file output stream */ protected def openFile() { - outputStream = new FileOutputStream(file, false) + outputStream = new FileOutputStream(file, true) logDebug(s"Opened file $file") } @@ -125,16 +125,16 @@ private[spark] object FileAppender extends Logging { val validatedParams: Option[(Long, String)] = rollingInterval match { case "daily" => logInfo(s"Rolling executor logs enabled for $file with daily rolling") - Some(24 * 60 * 60 * 1000L, "--yyyy-MM-dd") + Some((24 * 60 * 60 * 1000L, "--yyyy-MM-dd")) case "hourly" => logInfo(s"Rolling executor logs enabled for $file with hourly rolling") - Some(60 * 60 * 1000L, "--yyyy-MM-dd--HH") + Some((60 * 60 * 1000L, "--yyyy-MM-dd--HH")) case "minutely" => logInfo(s"Rolling executor logs enabled for $file with rolling every minute") - Some(60 * 1000L, "--yyyy-MM-dd--HH-mm") + Some((60 * 1000L, "--yyyy-MM-dd--HH-mm")) case IntParam(seconds) => logInfo(s"Rolling executor logs enabled for $file with rolling $seconds seconds") - Some(seconds * 1000L, "--yyyy-MM-dd--HH-mm-ss") + Some((seconds * 1000L, "--yyyy-MM-dd--HH-mm-ss")) case _ => logWarning(s"Illegal interval for rolling executor logs [$rollingInterval], " + s"rolling logs not enabled") diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala index 1e8476c4a047..5d8cec8447b5 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala @@ -17,11 +17,13 @@ package org.apache.spark.util.logging -import java.io.{File, FileFilter, InputStream} +import java.io._ +import java.util.zip.GZIPOutputStream import com.google.common.io.Files +import org.apache.commons.io.IOUtils + import org.apache.spark.SparkConf -import RollingFileAppender._ /** * Continuously appends data from input stream into the given file, and rolls @@ -39,10 +41,13 @@ private[spark] class RollingFileAppender( activeFile: File, val rollingPolicy: RollingPolicy, conf: SparkConf, - bufferSize: Int = DEFAULT_BUFFER_SIZE + bufferSize: Int = RollingFileAppender.DEFAULT_BUFFER_SIZE ) extends FileAppender(inputStream, activeFile, bufferSize) { + import RollingFileAppender._ + private val maxRetainedFiles = conf.getInt(RETAINED_FILES_PROPERTY, -1) + private val enableCompression = conf.getBoolean(ENABLE_COMPRESSION, false) /** Stop the appender */ override def stop() { @@ -74,6 +79,33 @@ private[spark] class RollingFileAppender( } } + // Roll the log file and compress if enableCompression is true. + private def rotateFile(activeFile: File, rolloverFile: File): Unit = { + if (enableCompression) { + val gzFile = new File(rolloverFile.getAbsolutePath + GZIP_LOG_SUFFIX) + var gzOutputStream: GZIPOutputStream = null + var inputStream: InputStream = null + try { + inputStream = new FileInputStream(activeFile) + gzOutputStream = new GZIPOutputStream(new FileOutputStream(gzFile)) + IOUtils.copy(inputStream, gzOutputStream) + inputStream.close() + gzOutputStream.close() + activeFile.delete() + } finally { + IOUtils.closeQuietly(inputStream) + IOUtils.closeQuietly(gzOutputStream) + } + } else { + Files.move(activeFile, rolloverFile) + } + } + + // Check if the rollover file already exists. + private def rolloverFileExist(file: File): Boolean = { + file.exists || new File(file.getAbsolutePath + GZIP_LOG_SUFFIX).exists + } + /** Move the active log file to a new rollover file */ private def moveFile() { val rolloverSuffix = rollingPolicy.generateRolledOverFileSuffix() @@ -81,8 +113,8 @@ private[spark] class RollingFileAppender( activeFile.getParentFile, activeFile.getName + rolloverSuffix).getAbsoluteFile logDebug(s"Attempting to rollover file $activeFile to file $rolloverFile") if (activeFile.exists) { - if (!rolloverFile.exists) { - Files.move(activeFile, rolloverFile) + if (!rolloverFileExist(rolloverFile)) { + rotateFile(activeFile, rolloverFile) logInfo(s"Rolled over $activeFile to $rolloverFile") } else { // In case the rollover file name clashes, make a unique file name. @@ -95,11 +127,11 @@ private[spark] class RollingFileAppender( altRolloverFile = new File(activeFile.getParent, s"${activeFile.getName}$rolloverSuffix--$i").getAbsoluteFile i += 1 - } while (i < 10000 && altRolloverFile.exists) + } while (i < 10000 && rolloverFileExist(altRolloverFile)) logWarning(s"Rollover file $rolloverFile already exists, " + s"rolled over $activeFile to file $altRolloverFile") - Files.move(activeFile, altRolloverFile) + rotateFile(activeFile, altRolloverFile) } } else { logWarning(s"File $activeFile does not exist") @@ -115,7 +147,7 @@ private[spark] class RollingFileAppender( } }).sorted val filesToBeDeleted = rolledoverFiles.take( - math.max(0, rolledoverFiles.size - maxRetainedFiles)) + math.max(0, rolledoverFiles.length - maxRetainedFiles)) filesToBeDeleted.foreach { file => logInfo(s"Deleting file executor log file ${file.getAbsolutePath}") file.delete() @@ -140,6 +172,9 @@ private[spark] object RollingFileAppender { val SIZE_DEFAULT = (1024 * 1024).toString val RETAINED_FILES_PROPERTY = "spark.executor.logs.rolling.maxRetainedFiles" val DEFAULT_BUFFER_SIZE = 8192 + val ENABLE_COMPRESSION = "spark.executor.logs.rolling.enableCompression" + + val GZIP_LOG_SUFFIX = ".gz" /** * Get the sorted list of rolled over files. This assumes that the all the rolled @@ -156,6 +191,6 @@ private[spark] object RollingFileAppender { val file = new File(directory, activeFileName).getAbsoluteFile if (file.exists) Some(file) else None } - rolledOverFiles ++ activeFile + rolledOverFiles.sortBy(_.getName.stripSuffix(GZIP_LOG_SUFFIX)) ++ activeFile } } diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala index d7b7219e179d..1f263df57c85 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala @@ -18,9 +18,9 @@ package org.apache.spark.util.logging import java.text.SimpleDateFormat -import java.util.Calendar +import java.util.{Calendar, Locale} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Defines the policy based on which [[org.apache.spark.util.logging.RollingFileAppender]] will @@ -32,10 +32,10 @@ private[spark] trait RollingPolicy { def shouldRollover(bytesToBeWritten: Long): Boolean /** Notify that rollover has occurred */ - def rolledOver() + def rolledOver(): Unit /** Notify that bytes have been written */ - def bytesWritten(bytes: Long) + def bytesWritten(bytes: Long): Unit /** Get the desired name of the rollover file */ def generateRolledOverFileSuffix(): String @@ -59,7 +59,7 @@ private[spark] class TimeBasedRollingPolicy( } @volatile private var nextRolloverTime = calculateNextRolloverTime() - private val formatter = new SimpleDateFormat(rollingFileSuffixPattern) + private val formatter = new SimpleDateFormat(rollingFileSuffixPattern, Locale.US) /** Should rollover if current time has exceeded next rollover time */ def shouldRollover(bytesToBeWritten: Long): Boolean = { @@ -109,11 +109,11 @@ private[spark] class SizeBasedRollingPolicy( } @volatile private var bytesWrittenSinceRollover = 0L - val formatter = new SimpleDateFormat("--yyyy-MM-dd--HH-mm-ss--SSSS") + val formatter = new SimpleDateFormat("--yyyy-MM-dd--HH-mm-ss--SSSS", Locale.US) /** Should rollover if the next set of bytes is going to exceed the size limit */ def shouldRollover(bytesToBeWritten: Long): Boolean = { - logInfo(s"$bytesToBeWritten + $bytesWrittenSinceRollover > $rolloverSizeBytes") + logDebug(s"$bytesToBeWritten + $bytesWrittenSinceRollover > $rolloverSizeBytes") bytesToBeWritten + bytesWrittenSinceRollover > rolloverSizeBytes } diff --git a/core/src/main/scala/org/apache/spark/util/package-info.java b/core/src/main/scala/org/apache/spark/util/package-info.java index 819f54ee41a7..4c5d33d88d2b 100644 --- a/core/src/main/scala/org/apache/spark/util/package-info.java +++ b/core/src/main/scala/org/apache/spark/util/package-info.java @@ -18,4 +18,4 @@ /** * Spark utilities. */ -package org.apache.spark.util; \ No newline at end of file +package org.apache.spark.util; diff --git a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala index 70f3dd62b9b1..41f28f6e511e 100644 --- a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala @@ -26,5 +26,5 @@ import org.apache.spark.annotation.DeveloperApi @DeveloperApi trait Pseudorandom { /** Set random seed. */ - def setSeed(seed: Long) + def setSeed(seed: Long): Unit } diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index c156b03cdb7c..ea99a7e5b484 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -20,7 +20,6 @@ package org.apache.spark.util.random import java.util.Random import scala.reflect.ClassTag -import scala.collection.mutable.ArrayBuffer import org.apache.commons.math3.distribution.PoissonDistribution @@ -39,7 +38,14 @@ import org.apache.spark.annotation.DeveloperApi trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable { /** take a random sample */ - def sample(items: Iterator[T]): Iterator[U] + def sample(items: Iterator[T]): Iterator[U] = + items.filter(_ => sample > 0).asInstanceOf[Iterator[U]] + + /** + * Whether to sample the next item or not. + * Return how many times the next item will be sampled. Return 0 if it is not sampled. + */ + def sample(): Int /** return a copy of the RandomSampler object */ override def clone: RandomSampler[T, U] = @@ -54,7 +60,7 @@ object RandomSampler { /** * Default maximum gap-sampling fraction. * For sampling fractions <= this value, the gap sampling optimization will be applied. - * Above this value, it is assumed that "tradtional" Bernoulli sampling is faster. The + * Above this value, it is assumed that "traditional" Bernoulli sampling is faster. The * optimal value for this will depend on the RNG. More expensive RNGs will tend to make * the optimal value higher. The most reliable way to determine this value for a new RNG * is to experiment. When tuning for a new RNG, I would expect a value of 0.5 to be close @@ -107,21 +113,13 @@ class BernoulliCellSampler[T](lb: Double, ub: Double, complement: Boolean = fals override def setSeed(seed: Long): Unit = rng.setSeed(seed) - override def sample(items: Iterator[T]): Iterator[T] = { + override def sample(): Int = { if (ub - lb <= 0.0) { - if (complement) items else Iterator.empty + if (complement) 1 else 0 } else { - if (complement) { - items.filter { item => { - val x = rng.nextDouble() - (x < lb) || (x >= ub) - }} - } else { - items.filter { item => { - val x = rng.nextDouble() - (x >= lb) && (x < ub) - }} - } + val x = rng.nextDouble() + val n = if ((x >= lb) && (x < ub)) 1 else 0 + if (complement) 1 - n else n } } @@ -155,15 +153,22 @@ class BernoulliSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T override def setSeed(seed: Long): Unit = rng.setSeed(seed) - override def sample(items: Iterator[T]): Iterator[T] = { + private lazy val gapSampling: GapSampling = + new GapSampling(fraction, rng, RandomSampler.rngEpsilon) + + override def sample(): Int = { if (fraction <= 0.0) { - Iterator.empty + 0 } else if (fraction >= 1.0) { - items + 1 } else if (fraction <= RandomSampler.defaultMaxGapSamplingFraction) { - new GapSamplingIterator(items, fraction, rng, RandomSampler.rngEpsilon) + gapSampling.sample() } else { - items.filter { _ => rng.nextDouble() <= fraction } + if (rng.nextDouble() <= fraction) { + 1 + } else { + 0 + } } } @@ -180,7 +185,7 @@ class BernoulliSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T * @tparam T item type */ @DeveloperApi -class PoissonSampler[T: ClassTag]( +class PoissonSampler[T]( fraction: Double, useGapSamplingIfPossible: Boolean) extends RandomSampler[T, T] { @@ -201,15 +206,29 @@ class PoissonSampler[T: ClassTag]( rngGap.setSeed(seed) } - override def sample(items: Iterator[T]): Iterator[T] = { + private lazy val gapSamplingReplacement = + new GapSamplingReplacement(fraction, rngGap, RandomSampler.rngEpsilon) + + override def sample(): Int = { if (fraction <= 0.0) { - Iterator.empty + 0 } else if (useGapSamplingIfPossible && fraction <= RandomSampler.defaultMaxGapSamplingFraction) { - new GapSamplingReplacementIterator(items, fraction, rngGap, RandomSampler.rngEpsilon) + gapSamplingReplacement.sample() + } else { + rng.sample() + } + } + + override def sample(items: Iterator[T]): Iterator[T] = { + if (fraction <= 0.0) { + Iterator.empty } else { + val useGapSampling = useGapSamplingIfPossible && + fraction <= RandomSampler.defaultMaxGapSamplingFraction + items.flatMap { item => - val count = rng.sample() + val count = if (useGapSampling) gapSamplingReplacement.sample() else rng.sample() if (count == 0) Iterator.empty else Iterator.fill(count)(item) } } @@ -220,50 +239,36 @@ class PoissonSampler[T: ClassTag]( private[spark] -class GapSamplingIterator[T: ClassTag]( - var data: Iterator[T], +class GapSampling( f: Double, rng: Random = RandomSampler.newDefaultRNG, - epsilon: Double = RandomSampler.rngEpsilon) extends Iterator[T] { + epsilon: Double = RandomSampler.rngEpsilon) extends Serializable { require(f > 0.0 && f < 1.0, s"Sampling fraction ($f) must reside on open interval (0, 1)") require(epsilon > 0.0, s"epsilon ($epsilon) must be > 0") - /** implement efficient linear-sequence drop until Scala includes fix for jira SI-8835. */ - private val iterDrop: Int => Unit = { - val arrayClass = Array.empty[T].iterator.getClass - val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass - data.getClass match { - case `arrayClass` => - (n: Int) => { data = data.drop(n) } - case `arrayBufferClass` => - (n: Int) => { data = data.drop(n) } - case _ => - (n: Int) => { - var j = 0 - while (j < n && data.hasNext) { - data.next() - j += 1 - } - } - } - } - - override def hasNext: Boolean = data.hasNext + private val lnq = math.log1p(-f) - override def next(): T = { - val r = data.next() - advance() - r + /** Return 1 if the next item should be sampled. Otherwise, return 0. */ + def sample(): Int = { + if (countForDropping > 0) { + countForDropping -= 1 + 0 + } else { + advance() + 1 + } } - private val lnq = math.log1p(-f) + private var countForDropping: Int = 0 - /** skip elements that won't be sampled, according to geometric dist P(k) = (f)(1-f)^k. */ + /** + * Decide the number of elements that won't be sampled, + * according to geometric dist P(k) = (f)(1-f)^k. + */ private def advance(): Unit = { val u = math.max(rng.nextDouble(), epsilon) - val k = (math.log(u) / lnq).toInt - iterDrop(k) + countForDropping = (math.log(u) / lnq).toInt } /** advance to first sample as part of object construction. */ @@ -273,73 +278,24 @@ class GapSamplingIterator[T: ClassTag]( // work reliably. } + private[spark] -class GapSamplingReplacementIterator[T: ClassTag]( - var data: Iterator[T], - f: Double, - rng: Random = RandomSampler.newDefaultRNG, - epsilon: Double = RandomSampler.rngEpsilon) extends Iterator[T] { +class GapSamplingReplacement( + val f: Double, + val rng: Random = RandomSampler.newDefaultRNG, + epsilon: Double = RandomSampler.rngEpsilon) extends Serializable { require(f > 0.0, s"Sampling fraction ($f) must be > 0") require(epsilon > 0.0, s"epsilon ($epsilon) must be > 0") - /** implement efficient linear-sequence drop until scala includes fix for jira SI-8835. */ - private val iterDrop: Int => Unit = { - val arrayClass = Array.empty[T].iterator.getClass - val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass - data.getClass match { - case `arrayClass` => - (n: Int) => { data = data.drop(n) } - case `arrayBufferClass` => - (n: Int) => { data = data.drop(n) } - case _ => - (n: Int) => { - var j = 0 - while (j < n && data.hasNext) { - data.next() - j += 1 - } - } - } - } - - /** current sampling value, and its replication factor, as we are sampling with replacement. */ - private var v: T = _ - private var rep: Int = 0 - - override def hasNext: Boolean = data.hasNext || rep > 0 - - override def next(): T = { - val r = v - rep -= 1 - if (rep <= 0) advance() - r - } - - /** - * Skip elements with replication factor zero (i.e. elements that won't be sampled). - * Samples 'k' from geometric distribution P(k) = (1-q)(q)^k, where q = e^(-f), that is - * q is the probabililty of Poisson(0; f) - */ - private def advance(): Unit = { - val u = math.max(rng.nextDouble(), epsilon) - val k = (math.log(u) / (-f)).toInt - iterDrop(k) - // set the value and replication factor for the next value - if (data.hasNext) { - v = data.next() - rep = poissonGE1 - } - } - - private val q = math.exp(-f) + protected val q = math.exp(-f) /** * Sample from Poisson distribution, conditioned such that the sampled value is >= 1. * This is an adaptation from the algorithm for Generating Poisson distributed random variables: * http://en.wikipedia.org/wiki/Poisson_distribution */ - private def poissonGE1: Int = { + protected def poissonGE1: Int = { // simulate that the standard poisson sampling // gave us at least one iteration, for a sample of >= 1 var pp = q + ((1.0 - q) * rng.nextDouble()) @@ -353,6 +309,28 @@ class GapSamplingReplacementIterator[T: ClassTag]( } r } + private var countForDropping: Int = 0 + + def sample(): Int = { + if (countForDropping > 0) { + countForDropping -= 1 + 0 + } else { + val r = poissonGE1 + advance() + r + } + } + + /** + * Skip elements with replication factor zero (i.e. elements that won't be sampled). + * Samples 'k' from geometric distribution P(k) = (1-q)(q)^k, where q = e^(-f), that is + * q is the probability of Poisson(0; f) + */ + private def advance(): Unit = { + val u = math.max(rng.nextDouble(), epsilon) + countForDropping = (math.log(u) / (-f)).toInt + } /** advance to first sample as part of object construction. */ advance() diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala index c9a864ae6277..a7e0075debed 100644 --- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala @@ -34,7 +34,7 @@ private[spark] object SamplingUtils { input: Iterator[T], k: Int, seed: Long = Random.nextLong()) - : (Array[T], Int) = { + : (Array[T], Long) = { val reservoir = new Array[T](k) // Put the first k elements in the reservoir. var i = 0 @@ -52,31 +52,37 @@ private[spark] object SamplingUtils { (trimReservoir, i) } else { // If input size > k, continue the sampling process. + var l = i.toLong val rand = new XORShiftRandom(seed) while (input.hasNext) { val item = input.next() - val replacementIndex = rand.nextInt(i) + l += 1 + // There are k elements in the reservoir, and the l-th element has been + // consumed. It should be chosen with probability k/l. The expression + // below is a random long chosen uniformly from [0,l) + val replacementIndex = (rand.nextDouble() * l).toLong if (replacementIndex < k) { - reservoir(replacementIndex) = item + reservoir(replacementIndex.toInt) = item } - i += 1 } - (reservoir, i) + (reservoir, l) } } /** - * Returns a sampling rate that guarantees a sample of size >= sampleSizeLowerBound 99.99% of - * the time. + * Returns a sampling rate that guarantees a sample of size greater than or equal to + * sampleSizeLowerBound 99.99% of the time. * * How the sampling rate is determined: + * * Let p = num / total, where num is the sample size and total is the total number of - * datapoints in the RDD. We're trying to compute q > p such that + * datapoints in the RDD. We're trying to compute q {@literal >} p such that * - when sampling with replacement, we're drawing each datapoint with prob_i ~ Pois(q), - * where we want to guarantee Pr[s < num] < 0.0001 for s = sum(prob_i for i from 0 to total), - * i.e. the failure rate of not having a sufficiently large sample < 0.0001. + * where we want to guarantee + * Pr[s {@literal <} num] {@literal <} 0.0001 for s = sum(prob_i for i from 0 to total), + * i.e. the failure rate of not having a sufficiently large sample {@literal <} 0.0001. * Setting q = p + 5 * sqrt(p/total) is sufficient to guarantee 0.9999 success rate for - * num > 12, but we need a slightly larger q (9 empirically determined). + * num {@literal >} 12, but we need a slightly larger q (9 empirically determined). * - when sampling without replacement, we're drawing each datapoint with prob_i * ~ Binomial(total, fraction) and our choice of q guarantees 1-delta, or 0.9999 success * rate, where success rate is defined the same as in sampling with replacement. @@ -107,14 +113,14 @@ private[spark] object SamplingUtils { private[spark] object PoissonBounds { /** - * Returns a lambda such that Pr[X > s] is very small, where X ~ Pois(lambda). + * Returns a lambda such that Pr[X {@literal >} s] is very small, where X ~ Pois(lambda). */ def getLowerBound(s: Double): Double = { math.max(s - numStd(s) * math.sqrt(s), 1e-15) } /** - * Returns a lambda such that Pr[X < s] is very small, where X ~ Pois(lambda). + * Returns a lambda such that Pr[X {@literal <} s] is very small, where X ~ Pois(lambda). * * @param s sample size */ diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala index effe6fa2adcf..ce46fc8f201b 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala @@ -24,7 +24,7 @@ import scala.reflect.ClassTag import org.apache.commons.math3.distribution.PoissonDistribution -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD /** @@ -35,13 +35,14 @@ import org.apache.spark.rdd.RDD * high probability. This is achieved by maintaining a waitlist of size O(log(s)), where s is the * desired sample size for each stratum. * - * Like in simple random sampling, we generate a random value for each item from the - * uniform distribution [0.0, 1.0]. All items with values <= min(values of items in the waitlist) - * are accepted into the sample instantly. The threshold for instant accept is designed so that - * s - numAccepted = O(sqrt(s)), where s is again the desired sample size. Thus, by maintaining a - * waitlist size = O(sqrt(s)), we will be able to create a sample of the exact size s by adding - * a portion of the waitlist to the set of items that are instantly accepted. The exact threshold - * is computed by sorting the values in the waitlist and picking the value at (s - numAccepted). + * Like in simple random sampling, we generate a random value for each item from the uniform + * distribution [0.0, 1.0]. All items with values less than or equal to min(values of items in the + * waitlist) are accepted into the sample instantly. The threshold for instant accept is designed + * so that s - numAccepted = O(sqrt(s)), where s is again the desired sample size. Thus, by + * maintaining a waitlist size = O(sqrt(s)), we will be able to create a sample of the exact size + * s by adding a portion of the waitlist to the set of items that are instantly accepted. The exact + * threshold is computed by sorting the values in the waitlist and picking the value at + * (s - numAccepted). * * Note that since we use the same seed for the RNG when computing the thresholds and the actual * sample, our computed thresholds are guaranteed to produce the desired sample size. @@ -160,12 +161,20 @@ private[spark] object StratifiedSamplingUtils extends Logging { * * To do so, we compute sampleSize = math.ceil(size * samplingRate) for each stratum and compare * it to the number of items that were accepted instantly and the number of items in the waitlist - * for that stratum. Most of the time, numAccepted <= sampleSize <= (numAccepted + numWaitlisted), + * for that stratum. + * + * Most of the time, + * {{{ + * numAccepted <= sampleSize <= (numAccepted + numWaitlisted) + * }}} * which means we need to sort the elements in the waitlist by their associated values in order - * to find the value T s.t. |{elements in the stratum whose associated values <= T}| = sampleSize. - * Note that all elements in the waitlist have values >= bound for instant accept, so a T value - * in the waitlist range would allow all elements that were instantly accepted on the first pass - * to be included in the sample. + * to find the value T s.t. + * {{{ + * |{elements in the stratum whose associated values <= T}| = sampleSize + * }}}. + * Note that all elements in the waitlist have values greater than or equal to bound for instant + * accept, so a T value in the waitlist range would allow all elements that were instantly + * accepted on the first pass to be included in the sample. */ def computeThresholdByKey[K](finalResult: Map[K, AcceptanceResult], fractions: Map[K, Double]): Map[K, Double] = { diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index 85fb923cd9bc..e8cdb6e98bf3 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -60,9 +60,11 @@ private[spark] class XORShiftRandom(init: Long) extends JavaRandom(init) { private[spark] object XORShiftRandom { /** Hash seeds to have 0/1 bits throughout. */ - private def hashSeed(seed: Long): Long = { + private[random] def hashSeed(seed: Long): Long = { val bytes = ByteBuffer.allocate(java.lang.Long.SIZE).putLong(seed).array() - MurmurHash3.bytesHash(bytes) + val lowBits = MurmurHash3.bytesHash(bytes) + val highBits = MurmurHash3.bytesHash(bytes, lowBits) + (highBits.toLong << 32) | (lowBits.toLong & 0xFFFFFFFFL) } /** diff --git a/core/src/main/scala/org/apache/spark/util/random/package-info.java b/core/src/main/scala/org/apache/spark/util/random/package-info.java index 62c3762dd11b..e4f0c0febbbb 100644 --- a/core/src/main/scala/org/apache/spark/util/random/package-info.java +++ b/core/src/main/scala/org/apache/spark/util/random/package-info.java @@ -18,4 +18,4 @@ /** * Utilities for random number generation. */ -package org.apache.spark.util.random; \ No newline at end of file +package org.apache.spark.util.random; diff --git a/core/src/main/scala/org/apache/spark/util/taskListeners.scala b/core/src/main/scala/org/apache/spark/util/taskListeners.scala new file mode 100644 index 000000000000..51feccfb8342 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/taskListeners.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.util.EventListener + +import org.apache.spark.TaskContext +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * + * Listener providing a callback function to invoke when a task's execution completes. + */ +@DeveloperApi +trait TaskCompletionListener extends EventListener { + def onTaskCompletion(context: TaskContext): Unit +} + + +/** + * :: DeveloperApi :: + * + * Listener providing a callback function to invoke when a task's execution encounters an error. + * Operations defined here must be idempotent, as `onTaskFailure` can be called multiple times. + */ +@DeveloperApi +trait TaskFailureListener extends EventListener { + def onTaskFailure(context: TaskContext, error: Throwable): Unit +} + + +/** + * Exception thrown when there is an exception in executing the callback in TaskCompletionListener. + */ +private[spark] +class TaskCompletionListenerException( + errorMessages: Seq[String], + val previousError: Option[Throwable] = None) + extends RuntimeException { + + override def getMessage: String = { + val listenerErrorMessage = + if (errorMessages.size == 1) { + errorMessages.head + } else { + errorMessages.zipWithIndex.map { case (msg, i) => s"Exception $i: $msg" }.mkString("\n") + } + val previousErrorMessage = previousError.map { e => + "\n\nPrevious exception in task: " + e.getMessage + "\n" + + e.getStackTrace.mkString("\t", "\n\t", "") + }.getOrElse("") + listenerErrorMessage + previousErrorMessage + } +} diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java deleted file mode 100644 index fd8f7f39b7cc..000000000000 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ /dev/null @@ -1,1812 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark; - -import java.io.*; -import java.nio.channels.FileChannel; -import java.nio.ByteBuffer; -import java.net.URI; -import java.util.*; -import java.util.concurrent.*; - -import scala.Tuple2; -import scala.Tuple3; -import scala.Tuple4; -import scala.collection.JavaConverters; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Iterators; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.base.Throwables; -import com.google.common.base.Optional; -import com.google.common.base.Charsets; -import com.google.common.io.Files; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.compress.DefaultCodec; -import org.apache.hadoop.mapred.SequenceFileInputFormat; -import org.apache.hadoop.mapred.SequenceFileOutputFormat; -import org.apache.hadoop.mapreduce.Job; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import org.apache.spark.api.java.*; -import org.apache.spark.api.java.function.*; -import org.apache.spark.input.PortableDataStream; -import org.apache.spark.partial.BoundedDouble; -import org.apache.spark.partial.PartialResult; -import org.apache.spark.rdd.RDD; -import org.apache.spark.serializer.KryoSerializer; -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.util.StatCounter; - -// The test suite itself is Serializable so that anonymous Function implementations can be -// serialized, as an alternative to converting these anonymous classes to static inner classes; -// see http://stackoverflow.com/questions/758570/. -public class JavaAPISuite implements Serializable { - private transient JavaSparkContext sc; - private transient File tempDir; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaAPISuite"); - tempDir = Files.createTempDir(); - tempDir.deleteOnExit(); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } - - @SuppressWarnings("unchecked") - @Test - public void sparkContextUnion() { - // Union of non-specialized JavaRDDs - List strings = Arrays.asList("Hello", "World"); - JavaRDD s1 = sc.parallelize(strings); - JavaRDD s2 = sc.parallelize(strings); - // Varargs - JavaRDD sUnion = sc.union(s1, s2); - Assert.assertEquals(4, sUnion.count()); - // List - List> list = new ArrayList<>(); - list.add(s2); - sUnion = sc.union(s1, list); - Assert.assertEquals(4, sUnion.count()); - - // Union of JavaDoubleRDDs - List doubles = Arrays.asList(1.0, 2.0); - JavaDoubleRDD d1 = sc.parallelizeDoubles(doubles); - JavaDoubleRDD d2 = sc.parallelizeDoubles(doubles); - JavaDoubleRDD dUnion = sc.union(d1, d2); - Assert.assertEquals(4, dUnion.count()); - - // Union of JavaPairRDDs - List> pairs = new ArrayList<>(); - pairs.add(new Tuple2<>(1, 2)); - pairs.add(new Tuple2<>(3, 4)); - JavaPairRDD p1 = sc.parallelizePairs(pairs); - JavaPairRDD p2 = sc.parallelizePairs(pairs); - JavaPairRDD pUnion = sc.union(p1, p2); - Assert.assertEquals(4, pUnion.count()); - } - - @SuppressWarnings("unchecked") - @Test - public void intersection() { - List ints1 = Arrays.asList(1, 10, 2, 3, 4, 5); - List ints2 = Arrays.asList(1, 6, 2, 3, 7, 8); - JavaRDD s1 = sc.parallelize(ints1); - JavaRDD s2 = sc.parallelize(ints2); - - JavaRDD intersections = s1.intersection(s2); - Assert.assertEquals(3, intersections.count()); - - JavaRDD empty = sc.emptyRDD(); - JavaRDD emptyIntersection = empty.intersection(s2); - Assert.assertEquals(0, emptyIntersection.count()); - - List doubles = Arrays.asList(1.0, 2.0); - JavaDoubleRDD d1 = sc.parallelizeDoubles(doubles); - JavaDoubleRDD d2 = sc.parallelizeDoubles(doubles); - JavaDoubleRDD dIntersection = d1.intersection(d2); - Assert.assertEquals(2, dIntersection.count()); - - List> pairs = new ArrayList<>(); - pairs.add(new Tuple2<>(1, 2)); - pairs.add(new Tuple2<>(3, 4)); - JavaPairRDD p1 = sc.parallelizePairs(pairs); - JavaPairRDD p2 = sc.parallelizePairs(pairs); - JavaPairRDD pIntersection = p1.intersection(p2); - Assert.assertEquals(2, pIntersection.count()); - } - - @Test - public void sample() { - List ints = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); - JavaRDD rdd = sc.parallelize(ints); - JavaRDD sample20 = rdd.sample(true, 0.2, 3); - Assert.assertEquals(2, sample20.count()); - JavaRDD sample20WithoutReplacement = rdd.sample(false, 0.2, 5); - Assert.assertEquals(2, sample20WithoutReplacement.count()); - } - - @Test - public void randomSplit() { - List ints = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); - JavaRDD rdd = sc.parallelize(ints); - JavaRDD[] splits = rdd.randomSplit(new double[] { 0.4, 0.6, 1.0 }, 31); - Assert.assertEquals(3, splits.length); - Assert.assertEquals(1, splits[0].count()); - Assert.assertEquals(2, splits[1].count()); - Assert.assertEquals(7, splits[2].count()); - } - - @Test - public void sortByKey() { - List> pairs = new ArrayList<>(); - pairs.add(new Tuple2<>(0, 4)); - pairs.add(new Tuple2<>(3, 2)); - pairs.add(new Tuple2<>(-1, 1)); - - JavaPairRDD rdd = sc.parallelizePairs(pairs); - - // Default comparator - JavaPairRDD sortedRDD = rdd.sortByKey(); - Assert.assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); - List> sortedPairs = sortedRDD.collect(); - Assert.assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); - Assert.assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); - - // Custom comparator - sortedRDD = rdd.sortByKey(Collections.reverseOrder(), false); - Assert.assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); - sortedPairs = sortedRDD.collect(); - Assert.assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); - Assert.assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); - } - - @SuppressWarnings("unchecked") - @Test - public void repartitionAndSortWithinPartitions() { - List> pairs = new ArrayList<>(); - pairs.add(new Tuple2<>(0, 5)); - pairs.add(new Tuple2<>(3, 8)); - pairs.add(new Tuple2<>(2, 6)); - pairs.add(new Tuple2<>(0, 8)); - pairs.add(new Tuple2<>(3, 8)); - pairs.add(new Tuple2<>(1, 3)); - - JavaPairRDD rdd = sc.parallelizePairs(pairs); - - Partitioner partitioner = new Partitioner() { - @Override - public int numPartitions() { - return 2; - } - @Override - public int getPartition(Object key) { - return (Integer) key % 2; - } - }; - - JavaPairRDD repartitioned = - rdd.repartitionAndSortWithinPartitions(partitioner); - Assert.assertTrue(repartitioned.partitioner().isPresent()); - Assert.assertEquals(repartitioned.partitioner().get(), partitioner); - List>> partitions = repartitioned.glom().collect(); - Assert.assertEquals(partitions.get(0), - Arrays.asList(new Tuple2<>(0, 5), new Tuple2<>(0, 8), new Tuple2<>(2, 6))); - Assert.assertEquals(partitions.get(1), - Arrays.asList(new Tuple2<>(1, 3), new Tuple2<>(3, 8), new Tuple2<>(3, 8))); - } - - @Test - public void emptyRDD() { - JavaRDD rdd = sc.emptyRDD(); - Assert.assertEquals("Empty RDD shouldn't have any values", 0, rdd.count()); - } - - @Test - public void sortBy() { - List> pairs = new ArrayList<>(); - pairs.add(new Tuple2<>(0, 4)); - pairs.add(new Tuple2<>(3, 2)); - pairs.add(new Tuple2<>(-1, 1)); - - JavaRDD> rdd = sc.parallelize(pairs); - - // compare on first value - JavaRDD> sortedRDD = rdd.sortBy(new Function, Integer>() { - @Override - public Integer call(Tuple2 t) { - return t._1(); - } - }, true, 2); - - Assert.assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); - List> sortedPairs = sortedRDD.collect(); - Assert.assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); - Assert.assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); - - // compare on second value - sortedRDD = rdd.sortBy(new Function, Integer>() { - @Override - public Integer call(Tuple2 t) { - return t._2(); - } - }, true, 2); - Assert.assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); - sortedPairs = sortedRDD.collect(); - Assert.assertEquals(new Tuple2<>(3, 2), sortedPairs.get(1)); - Assert.assertEquals(new Tuple2<>(0, 4), sortedPairs.get(2)); - } - - @Test - public void foreach() { - final Accumulator accum = sc.accumulator(0); - JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); - rdd.foreach(new VoidFunction() { - @Override - public void call(String s) { - accum.add(1); - } - }); - Assert.assertEquals(2, accum.value().intValue()); - } - - @Test - public void foreachPartition() { - final Accumulator accum = sc.accumulator(0); - JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); - rdd.foreachPartition(new VoidFunction>() { - @Override - public void call(Iterator iter) { - while (iter.hasNext()) { - iter.next(); - accum.add(1); - } - } - }); - Assert.assertEquals(2, accum.value().intValue()); - } - - @Test - public void toLocalIterator() { - List correct = Arrays.asList(1, 2, 3, 4); - JavaRDD rdd = sc.parallelize(correct); - List result = Lists.newArrayList(rdd.toLocalIterator()); - Assert.assertEquals(correct, result); - } - - @Test - public void zipWithUniqueId() { - List dataArray = Arrays.asList(1, 2, 3, 4); - JavaPairRDD zip = sc.parallelize(dataArray).zipWithUniqueId(); - JavaRDD indexes = zip.values(); - Assert.assertEquals(4, new HashSet<>(indexes.collect()).size()); - } - - @Test - public void zipWithIndex() { - List dataArray = Arrays.asList(1, 2, 3, 4); - JavaPairRDD zip = sc.parallelize(dataArray).zipWithIndex(); - JavaRDD indexes = zip.values(); - List correctIndexes = Arrays.asList(0L, 1L, 2L, 3L); - Assert.assertEquals(correctIndexes, indexes.collect()); - } - - @SuppressWarnings("unchecked") - @Test - public void lookup() { - JavaPairRDD categories = sc.parallelizePairs(Arrays.asList( - new Tuple2<>("Apples", "Fruit"), - new Tuple2<>("Oranges", "Fruit"), - new Tuple2<>("Oranges", "Citrus") - )); - Assert.assertEquals(2, categories.lookup("Oranges").size()); - Assert.assertEquals(2, Iterables.size(categories.groupByKey().lookup("Oranges").get(0))); - } - - @Test - public void groupBy() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); - Function isOdd = new Function() { - @Override - public Boolean call(Integer x) { - return x % 2 == 0; - } - }; - JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); - Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds - - oddsAndEvens = rdd.groupBy(isOdd, 1); - Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds - } - - @Test - public void groupByOnPairRDD() { - // Regression test for SPARK-4459 - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); - Function, Boolean> areOdd = - new Function, Boolean>() { - @Override - public Boolean call(Tuple2 x) { - return (x._1() % 2 == 0) && (x._2() % 2 == 0); - } - }; - JavaPairRDD pairRDD = rdd.zip(rdd); - JavaPairRDD>> oddsAndEvens = pairRDD.groupBy(areOdd); - Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds - - oddsAndEvens = pairRDD.groupBy(areOdd, 1); - Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds - } - - @SuppressWarnings("unchecked") - @Test - public void keyByOnPairRDD() { - // Regression test for SPARK-4459 - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); - Function, String> sumToString = - new Function, String>() { - @Override - public String call(Tuple2 x) { - return String.valueOf(x._1() + x._2()); - } - }; - JavaPairRDD pairRDD = rdd.zip(rdd); - JavaPairRDD> keyed = pairRDD.keyBy(sumToString); - Assert.assertEquals(7, keyed.count()); - Assert.assertEquals(1, (long) keyed.lookup("2").get(0)._1()); - } - - @SuppressWarnings("unchecked") - @Test - public void cogroup() { - JavaPairRDD categories = sc.parallelizePairs(Arrays.asList( - new Tuple2<>("Apples", "Fruit"), - new Tuple2<>("Oranges", "Fruit"), - new Tuple2<>("Oranges", "Citrus") - )); - JavaPairRDD prices = sc.parallelizePairs(Arrays.asList( - new Tuple2<>("Oranges", 2), - new Tuple2<>("Apples", 3) - )); - JavaPairRDD, Iterable>> cogrouped = - categories.cogroup(prices); - Assert.assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); - Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); - - cogrouped.collect(); - } - - @SuppressWarnings("unchecked") - @Test - public void cogroup3() { - JavaPairRDD categories = sc.parallelizePairs(Arrays.asList( - new Tuple2<>("Apples", "Fruit"), - new Tuple2<>("Oranges", "Fruit"), - new Tuple2<>("Oranges", "Citrus") - )); - JavaPairRDD prices = sc.parallelizePairs(Arrays.asList( - new Tuple2<>("Oranges", 2), - new Tuple2<>("Apples", 3) - )); - JavaPairRDD quantities = sc.parallelizePairs(Arrays.asList( - new Tuple2<>("Oranges", 21), - new Tuple2<>("Apples", 42) - )); - - JavaPairRDD, Iterable, Iterable>> cogrouped = - categories.cogroup(prices, quantities); - Assert.assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); - Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); - Assert.assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3())); - - - cogrouped.collect(); - } - - @SuppressWarnings("unchecked") - @Test - public void cogroup4() { - JavaPairRDD categories = sc.parallelizePairs(Arrays.asList( - new Tuple2<>("Apples", "Fruit"), - new Tuple2<>("Oranges", "Fruit"), - new Tuple2<>("Oranges", "Citrus") - )); - JavaPairRDD prices = sc.parallelizePairs(Arrays.asList( - new Tuple2<>("Oranges", 2), - new Tuple2<>("Apples", 3) - )); - JavaPairRDD quantities = sc.parallelizePairs(Arrays.asList( - new Tuple2<>("Oranges", 21), - new Tuple2<>("Apples", 42) - )); - JavaPairRDD countries = sc.parallelizePairs(Arrays.asList( - new Tuple2<>("Oranges", "BR"), - new Tuple2<>("Apples", "US") - )); - - JavaPairRDD, Iterable, Iterable, Iterable>> cogrouped = - categories.cogroup(prices, quantities, countries); - Assert.assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); - Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); - Assert.assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3())); - Assert.assertEquals("[BR]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._4())); - - cogrouped.collect(); - } - - @SuppressWarnings("unchecked") - @Test - public void leftOuterJoin() { - JavaPairRDD rdd1 = sc.parallelizePairs(Arrays.asList( - new Tuple2<>(1, 1), - new Tuple2<>(1, 2), - new Tuple2<>(2, 1), - new Tuple2<>(3, 1) - )); - JavaPairRDD rdd2 = sc.parallelizePairs(Arrays.asList( - new Tuple2<>(1, 'x'), - new Tuple2<>(2, 'y'), - new Tuple2<>(2, 'z'), - new Tuple2<>(4, 'w') - )); - List>>> joined = - rdd1.leftOuterJoin(rdd2).collect(); - Assert.assertEquals(5, joined.size()); - Tuple2>> firstUnmatched = - rdd1.leftOuterJoin(rdd2).filter( - new Function>>, Boolean>() { - @Override - public Boolean call(Tuple2>> tup) { - return !tup._2()._2().isPresent(); - } - }).first(); - Assert.assertEquals(3, firstUnmatched._1().intValue()); - } - - @Test - public void foldReduce() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); - Function2 add = new Function2() { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - }; - - int sum = rdd.fold(0, add); - Assert.assertEquals(33, sum); - - sum = rdd.reduce(add); - Assert.assertEquals(33, sum); - } - - @Test - public void treeReduce() { - JavaRDD rdd = sc.parallelize(Arrays.asList(-5, -4, -3, -2, -1, 1, 2, 3, 4), 10); - Function2 add = new Function2() { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - }; - for (int depth = 1; depth <= 10; depth++) { - int sum = rdd.treeReduce(add, depth); - Assert.assertEquals(-5, sum); - } - } - - @Test - public void treeAggregate() { - JavaRDD rdd = sc.parallelize(Arrays.asList(-5, -4, -3, -2, -1, 1, 2, 3, 4), 10); - Function2 add = new Function2() { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - }; - for (int depth = 1; depth <= 10; depth++) { - int sum = rdd.treeAggregate(0, add, add, depth); - Assert.assertEquals(-5, sum); - } - } - - @SuppressWarnings("unchecked") - @Test - public void aggregateByKey() { - JavaPairRDD pairs = sc.parallelizePairs( - Arrays.asList( - new Tuple2<>(1, 1), - new Tuple2<>(1, 1), - new Tuple2<>(3, 2), - new Tuple2<>(5, 1), - new Tuple2<>(5, 3)), 2); - - Map> sets = pairs.aggregateByKey(new HashSet(), - new Function2, Integer, Set>() { - @Override - public Set call(Set a, Integer b) { - a.add(b); - return a; - } - }, - new Function2, Set, Set>() { - @Override - public Set call(Set a, Set b) { - a.addAll(b); - return a; - } - }).collectAsMap(); - Assert.assertEquals(3, sets.size()); - Assert.assertEquals(new HashSet<>(Arrays.asList(1)), sets.get(1)); - Assert.assertEquals(new HashSet<>(Arrays.asList(2)), sets.get(3)); - Assert.assertEquals(new HashSet<>(Arrays.asList(1, 3)), sets.get(5)); - } - - @SuppressWarnings("unchecked") - @Test - public void foldByKey() { - List> pairs = Arrays.asList( - new Tuple2<>(2, 1), - new Tuple2<>(2, 1), - new Tuple2<>(1, 1), - new Tuple2<>(3, 2), - new Tuple2<>(3, 1) - ); - JavaPairRDD rdd = sc.parallelizePairs(pairs); - JavaPairRDD sums = rdd.foldByKey(0, - new Function2() { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - }); - Assert.assertEquals(1, sums.lookup(1).get(0).intValue()); - Assert.assertEquals(2, sums.lookup(2).get(0).intValue()); - Assert.assertEquals(3, sums.lookup(3).get(0).intValue()); - } - - @SuppressWarnings("unchecked") - @Test - public void reduceByKey() { - List> pairs = Arrays.asList( - new Tuple2<>(2, 1), - new Tuple2<>(2, 1), - new Tuple2<>(1, 1), - new Tuple2<>(3, 2), - new Tuple2<>(3, 1) - ); - JavaPairRDD rdd = sc.parallelizePairs(pairs); - JavaPairRDD counts = rdd.reduceByKey( - new Function2() { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - }); - Assert.assertEquals(1, counts.lookup(1).get(0).intValue()); - Assert.assertEquals(2, counts.lookup(2).get(0).intValue()); - Assert.assertEquals(3, counts.lookup(3).get(0).intValue()); - - Map localCounts = counts.collectAsMap(); - Assert.assertEquals(1, localCounts.get(1).intValue()); - Assert.assertEquals(2, localCounts.get(2).intValue()); - Assert.assertEquals(3, localCounts.get(3).intValue()); - - localCounts = rdd.reduceByKeyLocally(new Function2() { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - }); - Assert.assertEquals(1, localCounts.get(1).intValue()); - Assert.assertEquals(2, localCounts.get(2).intValue()); - Assert.assertEquals(3, localCounts.get(3).intValue()); - } - - @Test - public void approximateResults() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); - Map countsByValue = rdd.countByValue(); - Assert.assertEquals(2, countsByValue.get(1).longValue()); - Assert.assertEquals(1, countsByValue.get(13).longValue()); - - PartialResult> approx = rdd.countByValueApprox(1); - Map finalValue = approx.getFinalValue(); - Assert.assertEquals(2.0, finalValue.get(1).mean(), 0.01); - Assert.assertEquals(1.0, finalValue.get(13).mean(), 0.01); - } - - @Test - public void take() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); - Assert.assertEquals(1, rdd.first().intValue()); - rdd.take(2); - rdd.takeSample(false, 2, 42); - } - - @Test - public void isEmpty() { - Assert.assertTrue(sc.emptyRDD().isEmpty()); - Assert.assertTrue(sc.parallelize(new ArrayList()).isEmpty()); - Assert.assertFalse(sc.parallelize(Arrays.asList(1)).isEmpty()); - Assert.assertTrue(sc.parallelize(Arrays.asList(1, 2, 3), 3).filter( - new Function() { - @Override - public Boolean call(Integer i) { - return i < 0; - } - }).isEmpty()); - Assert.assertFalse(sc.parallelize(Arrays.asList(1, 2, 3)).filter( - new Function() { - @Override - public Boolean call(Integer i) { - return i > 1; - } - }).isEmpty()); - } - - @Test - public void toArray() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3)); - List list = rdd.toArray(); - Assert.assertEquals(Arrays.asList(1, 2, 3), list); - } - - @Test - public void cartesian() { - JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); - JavaRDD stringRDD = sc.parallelize(Arrays.asList("Hello", "World")); - JavaPairRDD cartesian = stringRDD.cartesian(doubleRDD); - Assert.assertEquals(new Tuple2<>("Hello", 1.0), cartesian.first()); - } - - @Test - public void javaDoubleRDD() { - JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); - JavaDoubleRDD distinct = rdd.distinct(); - Assert.assertEquals(5, distinct.count()); - JavaDoubleRDD filter = rdd.filter(new Function() { - @Override - public Boolean call(Double x) { - return x > 2.0; - } - }); - Assert.assertEquals(3, filter.count()); - JavaDoubleRDD union = rdd.union(rdd); - Assert.assertEquals(12, union.count()); - union = union.cache(); - Assert.assertEquals(12, union.count()); - - Assert.assertEquals(20, rdd.sum(), 0.01); - StatCounter stats = rdd.stats(); - Assert.assertEquals(20, stats.sum(), 0.01); - Assert.assertEquals(20/6.0, rdd.mean(), 0.01); - Assert.assertEquals(20/6.0, rdd.mean(), 0.01); - Assert.assertEquals(6.22222, rdd.variance(), 0.01); - Assert.assertEquals(7.46667, rdd.sampleVariance(), 0.01); - Assert.assertEquals(2.49444, rdd.stdev(), 0.01); - Assert.assertEquals(2.73252, rdd.sampleStdev(), 0.01); - - rdd.first(); - rdd.take(5); - } - - @Test - public void javaDoubleRDDHistoGram() { - JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); - // Test using generated buckets - Tuple2 results = rdd.histogram(2); - double[] expected_buckets = {1.0, 2.5, 4.0}; - long[] expected_counts = {2, 2}; - Assert.assertArrayEquals(expected_buckets, results._1(), 0.1); - Assert.assertArrayEquals(expected_counts, results._2()); - // Test with provided buckets - long[] histogram = rdd.histogram(expected_buckets); - Assert.assertArrayEquals(expected_counts, histogram); - // SPARK-5744 - Assert.assertArrayEquals( - new long[] {0}, - sc.parallelizeDoubles(new ArrayList(0), 1).histogram(new double[]{0.0, 1.0})); - } - - private static class DoubleComparator implements Comparator, Serializable { - @Override - public int compare(Double o1, Double o2) { - return o1.compareTo(o2); - } - } - - @Test - public void max() { - JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); - double max = rdd.max(new DoubleComparator()); - Assert.assertEquals(4.0, max, 0.001); - } - - @Test - public void min() { - JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); - double max = rdd.min(new DoubleComparator()); - Assert.assertEquals(1.0, max, 0.001); - } - - @Test - public void naturalMax() { - JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); - double max = rdd.max(); - Assert.assertEquals(4.0, max, 0.0); - } - - @Test - public void naturalMin() { - JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); - double max = rdd.min(); - Assert.assertEquals(1.0, max, 0.0); - } - - @Test - public void takeOrdered() { - JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); - Assert.assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2, new DoubleComparator())); - Assert.assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2)); - } - - @Test - public void top() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); - List top2 = rdd.top(2); - Assert.assertEquals(Arrays.asList(4, 3), top2); - } - - private static class AddInts implements Function2 { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - } - - @Test - public void reduce() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); - int sum = rdd.reduce(new AddInts()); - Assert.assertEquals(10, sum); - } - - @Test - public void reduceOnJavaDoubleRDD() { - JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); - double sum = rdd.reduce(new Function2() { - @Override - public Double call(Double v1, Double v2) { - return v1 + v2; - } - }); - Assert.assertEquals(10.0, sum, 0.001); - } - - @Test - public void fold() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); - int sum = rdd.fold(0, new AddInts()); - Assert.assertEquals(10, sum); - } - - @Test - public void aggregate() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); - int sum = rdd.aggregate(0, new AddInts(), new AddInts()); - Assert.assertEquals(10, sum); - } - - @Test - public void map() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - JavaDoubleRDD doubles = rdd.mapToDouble(new DoubleFunction() { - @Override - public double call(Integer x) { - return x.doubleValue(); - } - }).cache(); - doubles.collect(); - JavaPairRDD pairs = rdd.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(Integer x) { - return new Tuple2<>(x, x); - } - }).cache(); - pairs.collect(); - JavaRDD strings = rdd.map(new Function() { - @Override - public String call(Integer x) { - return x.toString(); - } - }).cache(); - strings.collect(); - } - - @Test - public void flatMap() { - JavaRDD rdd = sc.parallelize(Arrays.asList("Hello World!", - "The quick brown fox jumps over the lazy dog.")); - JavaRDD words = rdd.flatMap(new FlatMapFunction() { - @Override - public Iterable call(String x) { - return Arrays.asList(x.split(" ")); - } - }); - Assert.assertEquals("Hello", words.first()); - Assert.assertEquals(11, words.count()); - - JavaPairRDD pairsRDD = rdd.flatMapToPair( - new PairFlatMapFunction() { - @Override - public Iterable> call(String s) { - List> pairs = new LinkedList<>(); - for (String word : s.split(" ")) { - pairs.add(new Tuple2<>(word, word)); - } - return pairs; - } - } - ); - Assert.assertEquals(new Tuple2<>("Hello", "Hello"), pairsRDD.first()); - Assert.assertEquals(11, pairsRDD.count()); - - JavaDoubleRDD doubles = rdd.flatMapToDouble(new DoubleFlatMapFunction() { - @Override - public Iterable call(String s) { - List lengths = new LinkedList<>(); - for (String word : s.split(" ")) { - lengths.add((double) word.length()); - } - return lengths; - } - }); - Assert.assertEquals(5.0, doubles.first(), 0.01); - Assert.assertEquals(11, pairsRDD.count()); - } - - @SuppressWarnings("unchecked") - @Test - public void mapsFromPairsToPairs() { - List> pairs = Arrays.asList( - new Tuple2<>(1, "a"), - new Tuple2<>(2, "aa"), - new Tuple2<>(3, "aaa") - ); - JavaPairRDD pairRDD = sc.parallelizePairs(pairs); - - // Regression test for SPARK-668: - JavaPairRDD swapped = pairRDD.flatMapToPair( - new PairFlatMapFunction, String, Integer>() { - @Override - public Iterable> call(Tuple2 item) { - return Collections.singletonList(item.swap()); - } - }); - swapped.collect(); - - // There was never a bug here, but it's worth testing: - pairRDD.mapToPair(new PairFunction, String, Integer>() { - @Override - public Tuple2 call(Tuple2 item) { - return item.swap(); - } - }).collect(); - } - - @Test - public void mapPartitions() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); - JavaRDD partitionSums = rdd.mapPartitions( - new FlatMapFunction, Integer>() { - @Override - public Iterable call(Iterator iter) { - int sum = 0; - while (iter.hasNext()) { - sum += iter.next(); - } - return Collections.singletonList(sum); - } - }); - Assert.assertEquals("[3, 7]", partitionSums.collect().toString()); - } - - - @Test - public void mapPartitionsWithIndex() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); - JavaRDD partitionSums = rdd.mapPartitionsWithIndex( - new Function2, Iterator>() { - @Override - public Iterator call(Integer index, Iterator iter) { - int sum = 0; - while (iter.hasNext()) { - sum += iter.next(); - } - return Collections.singletonList(sum).iterator(); - } - }, false); - Assert.assertEquals("[3, 7]", partitionSums.collect().toString()); - } - - - @Test - public void repartition() { - // Shrinking number of partitions - JavaRDD in1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 2); - JavaRDD repartitioned1 = in1.repartition(4); - List> result1 = repartitioned1.glom().collect(); - Assert.assertEquals(4, result1.size()); - for (List l : result1) { - Assert.assertFalse(l.isEmpty()); - } - - // Growing number of partitions - JavaRDD in2 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 4); - JavaRDD repartitioned2 = in2.repartition(2); - List> result2 = repartitioned2.glom().collect(); - Assert.assertEquals(2, result2.size()); - for (List l: result2) { - Assert.assertFalse(l.isEmpty()); - } - } - - @SuppressWarnings("unchecked") - @Test - public void persist() { - JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); - doubleRDD = doubleRDD.persist(StorageLevel.DISK_ONLY()); - Assert.assertEquals(20, doubleRDD.sum(), 0.1); - - List> pairs = Arrays.asList( - new Tuple2<>(1, "a"), - new Tuple2<>(2, "aa"), - new Tuple2<>(3, "aaa") - ); - JavaPairRDD pairRDD = sc.parallelizePairs(pairs); - pairRDD = pairRDD.persist(StorageLevel.DISK_ONLY()); - Assert.assertEquals("a", pairRDD.first()._2()); - - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - rdd = rdd.persist(StorageLevel.DISK_ONLY()); - Assert.assertEquals(1, rdd.first().intValue()); - } - - @Test - public void iterator() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = TaskContext$.MODULE$.empty(); - Assert.assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); - } - - @Test - public void glom() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); - Assert.assertEquals("[1, 2]", rdd.glom().first().toString()); - } - - // File input / output tests are largely adapted from FileSuite: - - @Test - public void textFiles() throws IOException { - String outputDir = new File(tempDir, "output").getAbsolutePath(); - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); - rdd.saveAsTextFile(outputDir); - // Read the plain text file and check it's OK - File outputFile = new File(outputDir, "part-00000"); - String content = Files.toString(outputFile, Charsets.UTF_8); - Assert.assertEquals("1\n2\n3\n4\n", content); - // Also try reading it in as a text file RDD - List expected = Arrays.asList("1", "2", "3", "4"); - JavaRDD readRDD = sc.textFile(outputDir); - Assert.assertEquals(expected, readRDD.collect()); - } - - @Test - public void wholeTextFiles() throws Exception { - byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); - byte[] content2 = "spark is also easy to use.\n".getBytes("utf-8"); - - String tempDirName = tempDir.getAbsolutePath(); - Files.write(content1, new File(tempDirName + "/part-00000")); - Files.write(content2, new File(tempDirName + "/part-00001")); - - Map container = new HashMap<>(); - container.put(tempDirName+"/part-00000", new Text(content1).toString()); - container.put(tempDirName+"/part-00001", new Text(content2).toString()); - - JavaPairRDD readRDD = sc.wholeTextFiles(tempDirName, 3); - List> result = readRDD.collect(); - - for (Tuple2 res : result) { - Assert.assertEquals(res._2(), container.get(new URI(res._1()).getPath())); - } - } - - @Test - public void textFilesCompressed() throws IOException { - String outputDir = new File(tempDir, "output").getAbsolutePath(); - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); - rdd.saveAsTextFile(outputDir, DefaultCodec.class); - - // Try reading it in as a text file RDD - List expected = Arrays.asList("1", "2", "3", "4"); - JavaRDD readRDD = sc.textFile(outputDir); - Assert.assertEquals(expected, readRDD.collect()); - } - - @SuppressWarnings("unchecked") - @Test - public void sequenceFile() { - String outputDir = new File(tempDir, "output").getAbsolutePath(); - List> pairs = Arrays.asList( - new Tuple2<>(1, "a"), - new Tuple2<>(2, "aa"), - new Tuple2<>(3, "aaa") - ); - JavaPairRDD rdd = sc.parallelizePairs(pairs); - - rdd.mapToPair(new PairFunction, IntWritable, Text>() { - @Override - public Tuple2 call(Tuple2 pair) { - return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2())); - } - }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); - - // Try reading the output back as an object file - JavaPairRDD readRDD = sc.sequenceFile(outputDir, IntWritable.class, - Text.class).mapToPair(new PairFunction, Integer, String>() { - @Override - public Tuple2 call(Tuple2 pair) { - return new Tuple2<>(pair._1().get(), pair._2().toString()); - } - }); - Assert.assertEquals(pairs, readRDD.collect()); - } - - @Test - public void binaryFiles() throws Exception { - // Reusing the wholeText files example - byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); - - String tempDirName = tempDir.getAbsolutePath(); - File file1 = new File(tempDirName + "/part-00000"); - - FileOutputStream fos1 = new FileOutputStream(file1); - - FileChannel channel1 = fos1.getChannel(); - ByteBuffer bbuf = ByteBuffer.wrap(content1); - channel1.write(bbuf); - channel1.close(); - JavaPairRDD readRDD = sc.binaryFiles(tempDirName, 3); - List> result = readRDD.collect(); - for (Tuple2 res : result) { - Assert.assertArrayEquals(content1, res._2().toArray()); - } - } - - @Test - public void binaryFilesCaching() throws Exception { - // Reusing the wholeText files example - byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); - - String tempDirName = tempDir.getAbsolutePath(); - File file1 = new File(tempDirName + "/part-00000"); - - FileOutputStream fos1 = new FileOutputStream(file1); - - FileChannel channel1 = fos1.getChannel(); - ByteBuffer bbuf = ByteBuffer.wrap(content1); - channel1.write(bbuf); - channel1.close(); - - JavaPairRDD readRDD = sc.binaryFiles(tempDirName).cache(); - readRDD.foreach(new VoidFunction>() { - @Override - public void call(Tuple2 pair) { - pair._2().toArray(); // force the file to read - } - }); - - List> result = readRDD.collect(); - for (Tuple2 res : result) { - Assert.assertArrayEquals(content1, res._2().toArray()); - } - } - - @Test - public void binaryRecords() throws Exception { - // Reusing the wholeText files example - byte[] content1 = "spark isn't always easy to use.\n".getBytes("utf-8"); - int numOfCopies = 10; - String tempDirName = tempDir.getAbsolutePath(); - File file1 = new File(tempDirName + "/part-00000"); - - FileOutputStream fos1 = new FileOutputStream(file1); - - FileChannel channel1 = fos1.getChannel(); - - for (int i = 0; i < numOfCopies; i++) { - ByteBuffer bbuf = ByteBuffer.wrap(content1); - channel1.write(bbuf); - } - channel1.close(); - - JavaRDD readRDD = sc.binaryRecords(tempDirName, content1.length); - Assert.assertEquals(numOfCopies,readRDD.count()); - List result = readRDD.collect(); - for (byte[] res : result) { - Assert.assertArrayEquals(content1, res); - } - } - - @SuppressWarnings("unchecked") - @Test - public void writeWithNewAPIHadoopFile() { - String outputDir = new File(tempDir, "output").getAbsolutePath(); - List> pairs = Arrays.asList( - new Tuple2<>(1, "a"), - new Tuple2<>(2, "aa"), - new Tuple2<>(3, "aaa") - ); - JavaPairRDD rdd = sc.parallelizePairs(pairs); - - rdd.mapToPair(new PairFunction, IntWritable, Text>() { - @Override - public Tuple2 call(Tuple2 pair) { - return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2())); - } - }).saveAsNewAPIHadoopFile( - outputDir, IntWritable.class, Text.class, - org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.class); - - JavaPairRDD output = sc.sequenceFile(outputDir, IntWritable.class, Text.class); - Assert.assertEquals(pairs.toString(), output.map(new Function, String>() { - @Override - public String call(Tuple2 x) { - return x.toString(); - } - }).collect().toString()); - } - - @SuppressWarnings("unchecked") - @Test - public void readWithNewAPIHadoopFile() throws IOException { - String outputDir = new File(tempDir, "output").getAbsolutePath(); - List> pairs = Arrays.asList( - new Tuple2<>(1, "a"), - new Tuple2<>(2, "aa"), - new Tuple2<>(3, "aaa") - ); - JavaPairRDD rdd = sc.parallelizePairs(pairs); - - rdd.mapToPair(new PairFunction, IntWritable, Text>() { - @Override - public Tuple2 call(Tuple2 pair) { - return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2())); - } - }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); - - JavaPairRDD output = sc.newAPIHadoopFile(outputDir, - org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat.class, - IntWritable.class, Text.class, new Job().getConfiguration()); - Assert.assertEquals(pairs.toString(), output.map(new Function, String>() { - @Override - public String call(Tuple2 x) { - return x.toString(); - } - }).collect().toString()); - } - - @Test - public void objectFilesOfInts() { - String outputDir = new File(tempDir, "output").getAbsolutePath(); - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); - rdd.saveAsObjectFile(outputDir); - // Try reading the output back as an object file - List expected = Arrays.asList(1, 2, 3, 4); - JavaRDD readRDD = sc.objectFile(outputDir); - Assert.assertEquals(expected, readRDD.collect()); - } - - @SuppressWarnings("unchecked") - @Test - public void objectFilesOfComplexTypes() { - String outputDir = new File(tempDir, "output").getAbsolutePath(); - List> pairs = Arrays.asList( - new Tuple2<>(1, "a"), - new Tuple2<>(2, "aa"), - new Tuple2<>(3, "aaa") - ); - JavaPairRDD rdd = sc.parallelizePairs(pairs); - rdd.saveAsObjectFile(outputDir); - // Try reading the output back as an object file - JavaRDD> readRDD = sc.objectFile(outputDir); - Assert.assertEquals(pairs, readRDD.collect()); - } - - @SuppressWarnings("unchecked") - @Test - public void hadoopFile() { - String outputDir = new File(tempDir, "output").getAbsolutePath(); - List> pairs = Arrays.asList( - new Tuple2<>(1, "a"), - new Tuple2<>(2, "aa"), - new Tuple2<>(3, "aaa") - ); - JavaPairRDD rdd = sc.parallelizePairs(pairs); - - rdd.mapToPair(new PairFunction, IntWritable, Text>() { - @Override - public Tuple2 call(Tuple2 pair) { - return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2())); - } - }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); - - JavaPairRDD output = sc.hadoopFile(outputDir, - SequenceFileInputFormat.class, IntWritable.class, Text.class); - Assert.assertEquals(pairs.toString(), output.map(new Function, String>() { - @Override - public String call(Tuple2 x) { - return x.toString(); - } - }).collect().toString()); - } - - @SuppressWarnings("unchecked") - @Test - public void hadoopFileCompressed() { - String outputDir = new File(tempDir, "output_compressed").getAbsolutePath(); - List> pairs = Arrays.asList( - new Tuple2<>(1, "a"), - new Tuple2<>(2, "aa"), - new Tuple2<>(3, "aaa") - ); - JavaPairRDD rdd = sc.parallelizePairs(pairs); - - rdd.mapToPair(new PairFunction, IntWritable, Text>() { - @Override - public Tuple2 call(Tuple2 pair) { - return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2())); - } - }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class, - DefaultCodec.class); - - JavaPairRDD output = sc.hadoopFile(outputDir, - SequenceFileInputFormat.class, IntWritable.class, Text.class); - - Assert.assertEquals(pairs.toString(), output.map(new Function, String>() { - @Override - public String call(Tuple2 x) { - return x.toString(); - } - }).collect().toString()); - } - - @Test - public void zip() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - JavaDoubleRDD doubles = rdd.mapToDouble(new DoubleFunction() { - @Override - public double call(Integer x) { - return x.doubleValue(); - } - }); - JavaPairRDD zipped = rdd.zip(doubles); - zipped.count(); - } - - @Test - public void zipPartitions() { - JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6), 2); - JavaRDD rdd2 = sc.parallelize(Arrays.asList("1", "2", "3", "4"), 2); - FlatMapFunction2, Iterator, Integer> sizesFn = - new FlatMapFunction2, Iterator, Integer>() { - @Override - public Iterable call(Iterator i, Iterator s) { - return Arrays.asList(Iterators.size(i), Iterators.size(s)); - } - }; - - JavaRDD sizes = rdd1.zipPartitions(rdd2, sizesFn); - Assert.assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); - } - - @Test - public void accumulators() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - - final Accumulator intAccum = sc.intAccumulator(10); - rdd.foreach(new VoidFunction() { - @Override - public void call(Integer x) { - intAccum.add(x); - } - }); - Assert.assertEquals((Integer) 25, intAccum.value()); - - final Accumulator doubleAccum = sc.doubleAccumulator(10.0); - rdd.foreach(new VoidFunction() { - @Override - public void call(Integer x) { - doubleAccum.add((double) x); - } - }); - Assert.assertEquals((Double) 25.0, doubleAccum.value()); - - // Try a custom accumulator type - AccumulatorParam floatAccumulatorParam = new AccumulatorParam() { - @Override - public Float addInPlace(Float r, Float t) { - return r + t; - } - - @Override - public Float addAccumulator(Float r, Float t) { - return r + t; - } - - @Override - public Float zero(Float initialValue) { - return 0.0f; - } - }; - - final Accumulator floatAccum = sc.accumulator(10.0f, floatAccumulatorParam); - rdd.foreach(new VoidFunction() { - @Override - public void call(Integer x) { - floatAccum.add((float) x); - } - }); - Assert.assertEquals((Float) 25.0f, floatAccum.value()); - - // Test the setValue method - floatAccum.setValue(5.0f); - Assert.assertEquals((Float) 5.0f, floatAccum.value()); - } - - @Test - public void keyBy() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2)); - List> s = rdd.keyBy(new Function() { - @Override - public String call(Integer t) { - return t.toString(); - } - }).collect(); - Assert.assertEquals(new Tuple2<>("1", 1), s.get(0)); - Assert.assertEquals(new Tuple2<>("2", 2), s.get(1)); - } - - @Test - public void checkpointAndComputation() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - sc.setCheckpointDir(tempDir.getAbsolutePath()); - Assert.assertFalse(rdd.isCheckpointed()); - rdd.checkpoint(); - rdd.count(); // Forces the DAG to cause a checkpoint - Assert.assertTrue(rdd.isCheckpointed()); - Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), rdd.collect()); - } - - @Test - public void checkpointAndRestore() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - sc.setCheckpointDir(tempDir.getAbsolutePath()); - Assert.assertFalse(rdd.isCheckpointed()); - rdd.checkpoint(); - rdd.count(); // Forces the DAG to cause a checkpoint - Assert.assertTrue(rdd.isCheckpointed()); - - Assert.assertTrue(rdd.getCheckpointFile().isPresent()); - JavaRDD recovered = sc.checkpointFile(rdd.getCheckpointFile().get()); - Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect()); - } - - @Test - public void combineByKey() { - JavaRDD originalRDD = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6)); - Function keyFunction = new Function() { - @Override - public Integer call(Integer v1) { - return v1 % 3; - } - }; - Function createCombinerFunction = new Function() { - @Override - public Integer call(Integer v1) { - return v1; - } - }; - - Function2 mergeValueFunction = new Function2() { - @Override - public Integer call(Integer v1, Integer v2) { - return v1 + v2; - } - }; - - JavaPairRDD combinedRDD = originalRDD.keyBy(keyFunction) - .combineByKey(createCombinerFunction, mergeValueFunction, mergeValueFunction); - Map results = combinedRDD.collectAsMap(); - ImmutableMap expected = ImmutableMap.of(0, 9, 1, 5, 2, 7); - Assert.assertEquals(expected, results); - - Partitioner defaultPartitioner = Partitioner.defaultPartitioner( - combinedRDD.rdd(), - JavaConverters.collectionAsScalaIterableConverter( - Collections.>emptyList()).asScala().toSeq()); - combinedRDD = originalRDD.keyBy(keyFunction) - .combineByKey( - createCombinerFunction, - mergeValueFunction, - mergeValueFunction, - defaultPartitioner, - false, - new KryoSerializer(new SparkConf())); - results = combinedRDD.collectAsMap(); - Assert.assertEquals(expected, results); - } - - @SuppressWarnings("unchecked") - @Test - public void mapOnPairRDD() { - JavaRDD rdd1 = sc.parallelize(Arrays.asList(1,2,3,4)); - JavaPairRDD rdd2 = rdd1.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(Integer i) { - return new Tuple2<>(i, i % 2); - } - }); - JavaPairRDD rdd3 = rdd2.mapToPair( - new PairFunction, Integer, Integer>() { - @Override - public Tuple2 call(Tuple2 in) { - return new Tuple2<>(in._2(), in._1()); - } - }); - Assert.assertEquals(Arrays.asList( - new Tuple2<>(1, 1), - new Tuple2<>(0, 2), - new Tuple2<>(1, 3), - new Tuple2<>(0, 4)), rdd3.collect()); - - } - - @SuppressWarnings("unchecked") - @Test - public void collectPartitions() { - JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); - - JavaPairRDD rdd2 = rdd1.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(Integer i) { - return new Tuple2<>(i, i % 2); - } - }); - - List[] parts = rdd1.collectPartitions(new int[] {0}); - Assert.assertEquals(Arrays.asList(1, 2), parts[0]); - - parts = rdd1.collectPartitions(new int[] {1, 2}); - Assert.assertEquals(Arrays.asList(3, 4), parts[0]); - Assert.assertEquals(Arrays.asList(5, 6, 7), parts[1]); - - Assert.assertEquals(Arrays.asList(new Tuple2<>(1, 1), - new Tuple2<>(2, 0)), - rdd2.collectPartitions(new int[] {0})[0]); - - List>[] parts2 = rdd2.collectPartitions(new int[] {1, 2}); - Assert.assertEquals(Arrays.asList(new Tuple2<>(3, 1), - new Tuple2<>(4, 0)), - parts2[0]); - Assert.assertEquals(Arrays.asList(new Tuple2<>(5, 1), - new Tuple2<>(6, 0), - new Tuple2<>(7, 1)), - parts2[1]); - } - - @Test - public void countApproxDistinct() { - List arrayData = new ArrayList<>(); - int size = 100; - for (int i = 0; i < 100000; i++) { - arrayData.add(i % size); - } - JavaRDD simpleRdd = sc.parallelize(arrayData, 10); - Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.05) - size) / (size * 1.0)) <= 0.1); - } - - @Test - public void countApproxDistinctByKey() { - List> arrayData = new ArrayList<>(); - for (int i = 10; i < 100; i++) { - for (int j = 0; j < i; j++) { - arrayData.add(new Tuple2<>(i, j)); - } - } - double relativeSD = 0.001; - JavaPairRDD pairRdd = sc.parallelizePairs(arrayData); - List> res = pairRdd.countApproxDistinctByKey(relativeSD, 8).collect(); - for (Tuple2 resItem : res) { - double count = (double)resItem._1(); - Long resCount = (Long)resItem._2(); - Double error = Math.abs((resCount - count) / count); - Assert.assertTrue(error < 0.1); - } - - } - - @Test - public void collectAsMapWithIntArrayValues() { - // Regression test for SPARK-1040 - JavaRDD rdd = sc.parallelize(Arrays.asList(1)); - JavaPairRDD pairRDD = rdd.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(Integer x) { - return new Tuple2<>(x, new int[]{x}); - } - }); - pairRDD.collect(); // Works fine - pairRDD.collectAsMap(); // Used to crash with ClassCastException - } - - @SuppressWarnings("unchecked") - @Test - public void collectAsMapAndSerialize() throws Exception { - JavaPairRDD rdd = - sc.parallelizePairs(Arrays.asList(new Tuple2<>("foo", 1))); - Map map = rdd.collectAsMap(); - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - new ObjectOutputStream(bytes).writeObject(map); - Map deserializedMap = (Map) - new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray())).readObject(); - Assert.assertEquals(1, deserializedMap.get("foo").intValue()); - } - - @Test - @SuppressWarnings("unchecked") - public void sampleByKey() { - JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3); - JavaPairRDD rdd2 = rdd1.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(Integer i) { - return new Tuple2<>(i % 2, 1); - } - }); - Map fractions = Maps.newHashMap(); - fractions.put(0, 0.5); - fractions.put(1, 1.0); - JavaPairRDD wr = rdd2.sampleByKey(true, fractions, 1L); - Map wrCounts = (Map) (Object) wr.countByKey(); - Assert.assertEquals(2, wrCounts.size()); - Assert.assertTrue(wrCounts.get(0) > 0); - Assert.assertTrue(wrCounts.get(1) > 0); - JavaPairRDD wor = rdd2.sampleByKey(false, fractions, 1L); - Map worCounts = (Map) (Object) wor.countByKey(); - Assert.assertEquals(2, worCounts.size()); - Assert.assertTrue(worCounts.get(0) > 0); - Assert.assertTrue(worCounts.get(1) > 0); - } - - @Test - @SuppressWarnings("unchecked") - public void sampleByKeyExact() { - JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3); - JavaPairRDD rdd2 = rdd1.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(Integer i) { - return new Tuple2<>(i % 2, 1); - } - }); - Map fractions = Maps.newHashMap(); - fractions.put(0, 0.5); - fractions.put(1, 1.0); - JavaPairRDD wrExact = rdd2.sampleByKeyExact(true, fractions, 1L); - Map wrExactCounts = (Map) (Object) wrExact.countByKey(); - Assert.assertEquals(2, wrExactCounts.size()); - Assert.assertTrue(wrExactCounts.get(0) == 2); - Assert.assertTrue(wrExactCounts.get(1) == 4); - JavaPairRDD worExact = rdd2.sampleByKeyExact(false, fractions, 1L); - Map worExactCounts = (Map) (Object) worExact.countByKey(); - Assert.assertEquals(2, worExactCounts.size()); - Assert.assertTrue(worExactCounts.get(0) == 2); - Assert.assertTrue(worExactCounts.get(1) == 4); - } - - private static class SomeCustomClass implements Serializable { - SomeCustomClass() { - // Intentionally left blank - } - } - - @Test - public void collectUnderlyingScalaRDD() { - List data = new ArrayList<>(); - for (int i = 0; i < 100; i++) { - data.add(new SomeCustomClass()); - } - JavaRDD rdd = sc.parallelize(data); - SomeCustomClass[] collected = (SomeCustomClass[]) rdd.rdd().retag(SomeCustomClass.class).collect(); - Assert.assertEquals(data.size(), collected.length); - } - - private static final class BuggyMapFunction implements Function { - - @Override - public T call(T x) { - throw new IllegalStateException("Custom exception!"); - } - } - - @Test - public void collectAsync() throws Exception { - List data = Arrays.asList(1, 2, 3, 4, 5); - JavaRDD rdd = sc.parallelize(data, 1); - JavaFutureAction> future = rdd.collectAsync(); - List result = future.get(); - Assert.assertEquals(data, result); - Assert.assertFalse(future.isCancelled()); - Assert.assertTrue(future.isDone()); - Assert.assertEquals(1, future.jobIds().size()); - } - - @Test - public void takeAsync() throws Exception { - List data = Arrays.asList(1, 2, 3, 4, 5); - JavaRDD rdd = sc.parallelize(data, 1); - JavaFutureAction> future = rdd.takeAsync(1); - List result = future.get(); - Assert.assertEquals(1, result.size()); - Assert.assertEquals((Integer) 1, result.get(0)); - Assert.assertFalse(future.isCancelled()); - Assert.assertTrue(future.isDone()); - Assert.assertEquals(1, future.jobIds().size()); - } - - @Test - public void foreachAsync() throws Exception { - List data = Arrays.asList(1, 2, 3, 4, 5); - JavaRDD rdd = sc.parallelize(data, 1); - JavaFutureAction future = rdd.foreachAsync( - new VoidFunction() { - @Override - public void call(Integer integer) { - // intentionally left blank. - } - } - ); - future.get(); - Assert.assertFalse(future.isCancelled()); - Assert.assertTrue(future.isDone()); - Assert.assertEquals(1, future.jobIds().size()); - } - - @Test - public void countAsync() throws Exception { - List data = Arrays.asList(1, 2, 3, 4, 5); - JavaRDD rdd = sc.parallelize(data, 1); - JavaFutureAction future = rdd.countAsync(); - long count = future.get(); - Assert.assertEquals(data.size(), count); - Assert.assertFalse(future.isCancelled()); - Assert.assertTrue(future.isDone()); - Assert.assertEquals(1, future.jobIds().size()); - } - - @Test - public void testAsyncActionCancellation() throws Exception { - List data = Arrays.asList(1, 2, 3, 4, 5); - JavaRDD rdd = sc.parallelize(data, 1); - JavaFutureAction future = rdd.foreachAsync(new VoidFunction() { - @Override - public void call(Integer integer) throws InterruptedException { - Thread.sleep(10000); // To ensure that the job won't finish before it's cancelled. - } - }); - future.cancel(true); - Assert.assertTrue(future.isCancelled()); - Assert.assertTrue(future.isDone()); - try { - future.get(2000, TimeUnit.MILLISECONDS); - Assert.fail("Expected future.get() for cancelled job to throw CancellationException"); - } catch (CancellationException ignored) { - // pass - } - } - - @Test - public void testAsyncActionErrorWrapping() throws Exception { - List data = Arrays.asList(1, 2, 3, 4, 5); - JavaRDD rdd = sc.parallelize(data, 1); - JavaFutureAction future = rdd.map(new BuggyMapFunction()).countAsync(); - try { - future.get(2, TimeUnit.SECONDS); - Assert.fail("Expected future.get() for failed job to throw ExcecutionException"); - } catch (ExecutionException ee) { - Assert.assertTrue(Throwables.getStackTraceAsString(ee).contains("Custom exception!")); - } - Assert.assertTrue(future.isDone()); - } - - - /** - * Test for SPARK-3647. This test needs to use the maven-built assembly to trigger the issue, - * since that's the only artifact where Guava classes have been relocated. - */ - @Test - public void testGuavaOptional() { - // Stop the context created in setUp() and start a local-cluster one, to force usage of the - // assembly. - sc.stop(); - JavaSparkContext localCluster = new JavaSparkContext("local-cluster[1,1,1024]", "JavaAPISuite"); - try { - JavaRDD rdd1 = localCluster.parallelize(Arrays.asList(1, 2, null), 3); - JavaRDD> rdd2 = rdd1.map( - new Function>() { - @Override - public Optional call(Integer i) { - return Optional.fromNullable(i); - } - }); - rdd2.collect(); - } finally { - localCluster.stop(); - } - } - - static class Class1 {} - static class Class2 {} - - @Test - public void testRegisterKryoClasses() { - SparkConf conf = new SparkConf(); - conf.registerKryoClasses(new Class[]{ Class1.class, Class2.class }); - Assert.assertEquals( - Class1.class.getName() + "," + Class2.class.getName(), - conf.get("spark.kryo.classesToRegister")); - } - -} diff --git a/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java b/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java index 7fe452a48d89..a6589d289814 100644 --- a/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java +++ b/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java @@ -20,14 +20,11 @@ import java.sql.Connection; import java.sql.DriverManager; import java.sql.PreparedStatement; -import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; 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.Function2; import org.apache.spark.rdd.JdbcRDD; import org.junit.After; import org.junit.Assert; @@ -89,30 +86,13 @@ public void tearDown() throws SQLException { public void testJavaJdbcRDD() throws Exception { JavaRDD rdd = JdbcRDD.create( sc, - new JdbcRDD.ConnectionFactory() { - @Override - public Connection getConnection() throws SQLException { - return DriverManager.getConnection("jdbc:derby:target/JavaJdbcRDDSuiteDb"); - } - }, + () -> DriverManager.getConnection("jdbc:derby:target/JavaJdbcRDDSuiteDb"), "SELECT DATA FROM FOO WHERE ? <= ID AND ID <= ?", 1, 100, 1, - new Function() { - @Override - public Integer call(ResultSet r) throws Exception { - return r.getInt(1); - } - } + r -> r.getInt(1) ).cache(); Assert.assertEquals(100, rdd.count()); - Assert.assertEquals( - Integer.valueOf(10100), - rdd.reduce(new Function2() { - @Override - public Integer call(Integer i1, Integer i2) { - return i1 + i2; - } - })); + Assert.assertEquals(Integer.valueOf(10100), rdd.reduce((i1, i2) -> i1 + i2)); } } diff --git a/core/src/test/java/org/apache/spark/api/java/OptionalSuite.java b/core/src/test/java/org/apache/spark/api/java/OptionalSuite.java new file mode 100644 index 000000000000..4b97c18198c1 --- /dev/null +++ b/core/src/test/java/org/apache/spark/api/java/OptionalSuite.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java; + +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests {@link Optional}. + */ +public class OptionalSuite { + + @Test + public void testEmpty() { + Assert.assertFalse(Optional.empty().isPresent()); + Assert.assertNull(Optional.empty().orNull()); + Assert.assertEquals("foo", Optional.empty().or("foo")); + Assert.assertEquals("foo", Optional.empty().orElse("foo")); + } + + @Test(expected = NullPointerException.class) + public void testEmptyGet() { + Optional.empty().get(); + } + + @Test + public void testAbsent() { + Assert.assertFalse(Optional.absent().isPresent()); + Assert.assertNull(Optional.absent().orNull()); + Assert.assertEquals("foo", Optional.absent().or("foo")); + Assert.assertEquals("foo", Optional.absent().orElse("foo")); + } + + @Test(expected = NullPointerException.class) + public void testAbsentGet() { + Optional.absent().get(); + } + + @Test + public void testOf() { + Assert.assertTrue(Optional.of(1).isPresent()); + Assert.assertNotNull(Optional.of(1).orNull()); + Assert.assertEquals(Integer.valueOf(1), Optional.of(1).get()); + Assert.assertEquals(Integer.valueOf(1), Optional.of(1).or(2)); + Assert.assertEquals(Integer.valueOf(1), Optional.of(1).orElse(2)); + } + + @Test(expected = NullPointerException.class) + public void testOfWithNull() { + Optional.of(null); + } + + @Test + public void testOfNullable() { + Assert.assertTrue(Optional.ofNullable(1).isPresent()); + Assert.assertNotNull(Optional.ofNullable(1).orNull()); + Assert.assertEquals(Integer.valueOf(1), Optional.ofNullable(1).get()); + Assert.assertEquals(Integer.valueOf(1), Optional.ofNullable(1).or(2)); + Assert.assertEquals(Integer.valueOf(1), Optional.ofNullable(1).orElse(2)); + Assert.assertFalse(Optional.ofNullable(null).isPresent()); + Assert.assertNull(Optional.ofNullable(null).orNull()); + Assert.assertEquals(Integer.valueOf(2), Optional.ofNullable(null).or(2)); + Assert.assertEquals(Integer.valueOf(2), Optional.ofNullable(null).orElse(2)); + } + + @Test + public void testFromNullable() { + Assert.assertTrue(Optional.fromNullable(1).isPresent()); + Assert.assertNotNull(Optional.fromNullable(1).orNull()); + Assert.assertEquals(Integer.valueOf(1), Optional.fromNullable(1).get()); + Assert.assertEquals(Integer.valueOf(1), Optional.fromNullable(1).or(2)); + Assert.assertEquals(Integer.valueOf(1), Optional.fromNullable(1).orElse(2)); + Assert.assertFalse(Optional.fromNullable(null).isPresent()); + Assert.assertNull(Optional.fromNullable(null).orNull()); + Assert.assertEquals(Integer.valueOf(2), Optional.fromNullable(null).or(2)); + Assert.assertEquals(Integer.valueOf(2), Optional.fromNullable(null).orElse(2)); + } + +} diff --git a/core/src/test/java/org/apache/spark/io/GenericFileInputStreamSuite.java b/core/src/test/java/org/apache/spark/io/GenericFileInputStreamSuite.java new file mode 100644 index 000000000000..3440e1aea2f4 --- /dev/null +++ b/core/src/test/java/org/apache/spark/io/GenericFileInputStreamSuite.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.io; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.RandomUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; + +import static org.junit.Assert.assertEquals; + +/** + * Tests functionality of {@link NioBufferedFileInputStream} + */ +public abstract class GenericFileInputStreamSuite { + + private byte[] randomBytes; + + protected File inputFile; + + protected InputStream inputStream; + + @Before + public void setUp() throws IOException { + // Create a byte array of size 2 MB with random bytes + randomBytes = RandomUtils.nextBytes(2 * 1024 * 1024); + inputFile = File.createTempFile("temp-file", ".tmp"); + FileUtils.writeByteArrayToFile(inputFile, randomBytes); + } + + @After + public void tearDown() { + inputFile.delete(); + } + + @Test + public void testReadOneByte() throws IOException { + for (int i = 0; i < randomBytes.length; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + } + + @Test + public void testReadMultipleBytes() throws IOException { + byte[] readBytes = new byte[8 * 1024]; + int i = 0; + while (i < randomBytes.length) { + int read = inputStream.read(readBytes, 0, 8 * 1024); + for (int j = 0; j < read; j++) { + assertEquals(randomBytes[i], readBytes[j]); + i++; + } + } + } + + @Test + public void testBytesSkipped() throws IOException { + assertEquals(1024, inputStream.skip(1024)); + for (int i = 1024; i < randomBytes.length; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + } + + @Test + public void testBytesSkippedAfterRead() throws IOException { + for (int i = 0; i < 1024; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + assertEquals(1024, inputStream.skip(1024)); + for (int i = 2048; i < randomBytes.length; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + } + + @Test + public void testNegativeBytesSkippedAfterRead() throws IOException { + for (int i = 0; i < 1024; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + // Skipping negative bytes should essential be a no-op + assertEquals(0, inputStream.skip(-1)); + assertEquals(0, inputStream.skip(-1024)); + assertEquals(0, inputStream.skip(Long.MIN_VALUE)); + assertEquals(1024, inputStream.skip(1024)); + for (int i = 2048; i < randomBytes.length; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + } + + @Test + public void testSkipFromFileChannel() throws IOException { + // Since the buffer is smaller than the skipped bytes, this will guarantee + // we skip from underlying file channel. + assertEquals(1024, inputStream.skip(1024)); + for (int i = 1024; i < 2048; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + assertEquals(256, inputStream.skip(256)); + assertEquals(256, inputStream.skip(256)); + assertEquals(512, inputStream.skip(512)); + for (int i = 3072; i < randomBytes.length; i++) { + assertEquals(randomBytes[i], (byte) inputStream.read()); + } + } + + @Test + public void testBytesSkippedAfterEOF() throws IOException { + assertEquals(randomBytes.length, inputStream.skip(randomBytes.length + 1)); + assertEquals(-1, inputStream.read()); + } +} diff --git a/core/src/test/java/org/apache/spark/io/NioBufferedInputStreamSuite.java b/core/src/test/java/org/apache/spark/io/NioBufferedInputStreamSuite.java new file mode 100644 index 000000000000..211b33a1a9fb --- /dev/null +++ b/core/src/test/java/org/apache/spark/io/NioBufferedInputStreamSuite.java @@ -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.io; + +import org.junit.Before; + +import java.io.IOException; + +/** + * Tests functionality of {@link NioBufferedFileInputStream} + */ +public class NioBufferedInputStreamSuite extends GenericFileInputStreamSuite { + + @Before + public void setUp() throws IOException { + super.setUp(); + inputStream = new NioBufferedFileInputStream(inputFile); + } +} diff --git a/core/src/test/java/org/apache/spark/io/ReadAheadInputStreamSuite.java b/core/src/test/java/org/apache/spark/io/ReadAheadInputStreamSuite.java new file mode 100644 index 000000000000..918ddc4517ec --- /dev/null +++ b/core/src/test/java/org/apache/spark/io/ReadAheadInputStreamSuite.java @@ -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.io; + +import org.junit.Before; + +import java.io.IOException; + +/** + * Tests functionality of {@link NioBufferedFileInputStream} + */ +public class ReadAheadInputStreamSuite extends GenericFileInputStreamSuite { + + @Before + public void setUp() throws IOException { + super.setUp(); + inputStream = new ReadAheadInputStream( + new NioBufferedFileInputStream(inputFile), 8 * 1024, 4 * 1024); + } +} diff --git a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index aa15e792e2b2..ac4391e3ef99 100644 --- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -17,9 +17,6 @@ package org.apache.spark.launcher; -import java.io.BufferedReader; -import java.io.InputStream; -import java.io.InputStreamReader; import java.util.Arrays; import java.util.HashMap; import java.util.Map; @@ -29,6 +26,10 @@ import org.slf4j.LoggerFactory; import org.slf4j.bridge.SLF4JBridgeHandler; import static org.junit.Assert.*; +import static org.junit.Assume.*; + +import org.apache.spark.internal.config.package$; +import org.apache.spark.util.Utils; /** * These tests require the Spark assembly to be built before they can be run. @@ -43,10 +44,10 @@ public class SparkLauncherSuite { private static final Logger LOG = LoggerFactory.getLogger(SparkLauncherSuite.class); private static final NamedThreadFactory TF = new NamedThreadFactory("SparkLauncherSuite-%d"); + private final SparkLauncher launcher = new SparkLauncher(); + @Test public void testSparkArgumentHandling() throws Exception { - SparkLauncher launcher = new SparkLauncher() - .setSparkHome(System.getProperty("spark.test.home")); SparkSubmitOptionParser opts = new SparkSubmitOptionParser(); launcher.addSparkArg(opts.HELP); @@ -86,18 +87,27 @@ public void testSparkArgumentHandling() throws Exception { launcher.setConf("spark.foo", "foo"); launcher.addSparkArg(opts.CONF, "spark.foo=bar"); assertEquals("bar", launcher.builder.conf.get("spark.foo")); + + launcher.setConf(SparkLauncher.PYSPARK_DRIVER_PYTHON, "python3.4"); + launcher.setConf(SparkLauncher.PYSPARK_PYTHON, "python3.5"); + assertEquals("python3.4", launcher.builder.conf.get( + package$.MODULE$.PYSPARK_DRIVER_PYTHON().key())); + assertEquals("python3.5", launcher.builder.conf.get(package$.MODULE$.PYSPARK_PYTHON().key())); } @Test public void testChildProcLauncher() throws Exception { + // This test is failed on Windows due to the failure of initiating executors + // by the path length limitation. See SPARK-18718. + assumeTrue(!Utils.isWindows()); + SparkSubmitOptionParser opts = new SparkSubmitOptionParser(); - Map env = new HashMap(); + Map env = new HashMap<>(); env.put("SPARK_PRINT_LAUNCH_COMMAND", "1"); - SparkLauncher launcher = new SparkLauncher(env) - .setSparkHome(System.getProperty("spark.test.home")) + launcher .setMaster("local") - .setAppResource("spark-internal") + .setAppResource(SparkLauncher.NO_RESOURCE) .addSparkArg(opts.CONF, String.format("%s=-Dfoo=ShouldBeOverriddenBelow", SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS)) .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, @@ -105,11 +115,11 @@ public void testChildProcLauncher() throws Exception { .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path")) .addSparkArg(opts.CLASS, "ShouldBeOverriddenBelow") .setMainClass(SparkLauncherTestApp.class.getName()) + .redirectError() .addAppArgs("proc"); final Process app = launcher.launch(); - new OutputRedirector(app.getInputStream(), TF); - new OutputRedirector(app.getErrorStream(), TF); + new OutputRedirector(app.getInputStream(), getClass().getName() + ".child", TF); assertEquals(0, app.waitFor()); } diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java index dab7b0592cb4..46b0516e3614 100644 --- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java +++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java @@ -17,8 +17,6 @@ package org.apache.spark.memory; -import java.io.IOException; - import org.junit.Assert; import org.junit.Test; @@ -27,53 +25,44 @@ public class TaskMemoryManagerSuite { - class TestMemoryConsumer extends MemoryConsumer { - TestMemoryConsumer(TaskMemoryManager memoryManager) { - super(memoryManager); - } - - @Override - public long spill(long size, MemoryConsumer trigger) throws IOException { - long used = getUsed(); - releaseMemory(used); - return used; - } - - void use(long size) { - acquireMemory(size); - } - - void free(long size) { - releaseMemory(size); - } - } - @Test public void leakedPageMemoryIsDetected() { final TaskMemoryManager manager = new TaskMemoryManager( - new TestMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")), 0); - manager.allocatePage(4096, null); // leak memory + new StaticMemoryManager( + new SparkConf().set("spark.memory.offHeap.enabled", "false"), + Long.MAX_VALUE, + Long.MAX_VALUE, + 1), + 0); + final MemoryConsumer c = new TestMemoryConsumer(manager); + manager.allocatePage(4096, c); // leak memory + Assert.assertEquals(4096, manager.getMemoryConsumptionForThisTask()); Assert.assertEquals(4096, manager.cleanUpAllAllocatedMemory()); } @Test public void encodePageNumberAndOffsetOffHeap() { - final TaskMemoryManager manager = new TaskMemoryManager( - new TestMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "true")), 0); - final MemoryBlock dataPage = manager.allocatePage(256, null); + final SparkConf conf = new SparkConf() + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "1000"); + final TaskMemoryManager manager = new TaskMemoryManager(new TestMemoryManager(conf), 0); + final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.OFF_HEAP); + final MemoryBlock dataPage = manager.allocatePage(256, c); // In off-heap mode, an offset is an absolute address that may require more than 51 bits to // encode. This test exercises that corner-case: final long offset = ((1L << TaskMemoryManager.OFFSET_BITS) + 10); final long encodedAddress = manager.encodePageNumberAndOffset(dataPage, offset); Assert.assertEquals(null, manager.getPage(encodedAddress)); Assert.assertEquals(offset, manager.getOffsetInPage(encodedAddress)); + manager.freePage(dataPage, c); } @Test public void encodePageNumberAndOffsetOnHeap() { final TaskMemoryManager manager = new TaskMemoryManager( - new TestMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")), 0); - final MemoryBlock dataPage = manager.allocatePage(256, null); + new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); + final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP); + final MemoryBlock dataPage = manager.allocatePage(256, c); final long encodedAddress = manager.encodePageNumberAndOffset(dataPage, 64); Assert.assertEquals(dataPage.getBaseObject(), manager.getPage(encodedAddress)); Assert.assertEquals(64, manager.getOffsetInPage(encodedAddress)); @@ -88,37 +77,102 @@ public void cooperativeSpilling() { TestMemoryConsumer c1 = new TestMemoryConsumer(manager); TestMemoryConsumer c2 = new TestMemoryConsumer(manager); c1.use(100); - assert(c1.getUsed() == 100); + Assert.assertEquals(100, c1.getUsed()); c2.use(100); - assert(c2.getUsed() == 100); - assert(c1.getUsed() == 0); // spilled + Assert.assertEquals(100, c2.getUsed()); + Assert.assertEquals(0, c1.getUsed()); // spilled c1.use(100); - assert(c1.getUsed() == 100); - assert(c2.getUsed() == 0); // spilled + Assert.assertEquals(100, c1.getUsed()); + Assert.assertEquals(0, c2.getUsed()); // spilled c1.use(50); - assert(c1.getUsed() == 50); // spilled - assert(c2.getUsed() == 0); + Assert.assertEquals(50, c1.getUsed()); // spilled + Assert.assertEquals(0, c2.getUsed()); c2.use(50); - assert(c1.getUsed() == 50); - assert(c2.getUsed() == 50); + Assert.assertEquals(50, c1.getUsed()); + Assert.assertEquals(50, c2.getUsed()); c1.use(100); - assert(c1.getUsed() == 100); - assert(c2.getUsed() == 0); // spilled + Assert.assertEquals(100, c1.getUsed()); + Assert.assertEquals(0, c2.getUsed()); // spilled c1.free(20); - assert(c1.getUsed() == 80); + Assert.assertEquals(80, c1.getUsed()); c2.use(10); - assert(c1.getUsed() == 80); - assert(c2.getUsed() == 10); + Assert.assertEquals(80, c1.getUsed()); + Assert.assertEquals(10, c2.getUsed()); c2.use(100); - assert(c2.getUsed() == 100); - assert(c1.getUsed() == 0); // spilled + Assert.assertEquals(100, c2.getUsed()); + Assert.assertEquals(0, c1.getUsed()); // spilled c1.free(0); c2.free(100); - assert(manager.cleanUpAllAllocatedMemory() == 0); + Assert.assertEquals(0, manager.cleanUpAllAllocatedMemory()); + } + + @Test + public void cooperativeSpilling2() { + final TestMemoryManager memoryManager = new TestMemoryManager(new SparkConf()); + memoryManager.limit(100); + final TaskMemoryManager manager = new TaskMemoryManager(memoryManager, 0); + + TestMemoryConsumer c1 = new TestMemoryConsumer(manager); + TestMemoryConsumer c2 = new TestMemoryConsumer(manager); + TestMemoryConsumer c3 = new TestMemoryConsumer(manager); + + c1.use(20); + Assert.assertEquals(20, c1.getUsed()); + c2.use(80); + Assert.assertEquals(80, c2.getUsed()); + c3.use(80); + Assert.assertEquals(20, c1.getUsed()); // c1: not spilled + Assert.assertEquals(0, c2.getUsed()); // c2: spilled as it has required size of memory + Assert.assertEquals(80, c3.getUsed()); + + c2.use(80); + Assert.assertEquals(20, c1.getUsed()); // c1: not spilled + Assert.assertEquals(0, c3.getUsed()); // c3: spilled as it has required size of memory + Assert.assertEquals(80, c2.getUsed()); + + c3.use(10); + Assert.assertEquals(0, c1.getUsed()); // c1: spilled as it has required size of memory + Assert.assertEquals(80, c2.getUsed()); // c2: not spilled as spilling c1 already satisfies c3 + Assert.assertEquals(10, c3.getUsed()); + + c1.free(0); + c2.free(80); + c3.free(10); + Assert.assertEquals(0, manager.cleanUpAllAllocatedMemory()); + } + + @Test + public void shouldNotForceSpillingInDifferentModes() { + final TestMemoryManager memoryManager = new TestMemoryManager(new SparkConf()); + memoryManager.limit(100); + final TaskMemoryManager manager = new TaskMemoryManager(memoryManager, 0); + + TestMemoryConsumer c1 = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP); + TestMemoryConsumer c2 = new TestMemoryConsumer(manager, MemoryMode.OFF_HEAP); + c1.use(80); + Assert.assertEquals(80, c1.getUsed()); + c2.use(80); + Assert.assertEquals(20, c2.getUsed()); // not enough memory + Assert.assertEquals(80, c1.getUsed()); // not spilled + + c2.use(10); + Assert.assertEquals(10, c2.getUsed()); // spilled + Assert.assertEquals(80, c1.getUsed()); // not spilled + } + + @Test + public void offHeapConfigurationBackwardsCompatibility() { + // Tests backwards-compatibility with the old `spark.unsafe.offHeap` configuration, which + // was deprecated in Spark 1.6 and replaced by `spark.memory.offHeap.enabled` (see SPARK-12251). + final SparkConf conf = new SparkConf() + .set("spark.unsafe.offHeap", "true") + .set("spark.memory.offHeap.size", "1000"); + final TaskMemoryManager manager = new TaskMemoryManager(new TestMemoryManager(conf), 0); + Assert.assertSame(MemoryMode.OFF_HEAP, manager.tungstenMemoryMode); } } diff --git a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java new file mode 100644 index 000000000000..db91329c94cb --- /dev/null +++ b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.memory; + +import java.io.IOException; + +public class TestMemoryConsumer extends MemoryConsumer { + public TestMemoryConsumer(TaskMemoryManager memoryManager, MemoryMode mode) { + super(memoryManager, 1024L, mode); + } + public TestMemoryConsumer(TaskMemoryManager memoryManager) { + this(memoryManager, MemoryMode.ON_HEAP); + } + + @Override + public long spill(long size, MemoryConsumer trigger) throws IOException { + long used = getUsed(); + free(used); + return used; + } + + void use(long size) { + long got = taskMemoryManager.acquireExecutionMemory(size, this); + used += got; + } + + void free(long size) { + used -= size; + taskMemoryManager.releaseExecutionMemory(size, this); + } +} + + diff --git a/core/src/test/java/org/apache/spark/serializer/TestJavaSerializerImpl.java b/core/src/test/java/org/apache/spark/serializer/TestJavaSerializerImpl.java index 3d50ab4fabe4..8aa063670099 100644 --- a/core/src/test/java/org/apache/spark/serializer/TestJavaSerializerImpl.java +++ b/core/src/test/java/org/apache/spark/serializer/TestJavaSerializerImpl.java @@ -21,7 +21,6 @@ import java.io.OutputStream; import java.nio.ByteBuffer; -import scala.Option; import scala.reflect.ClassTag; diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java index 9a43f1f3a923..354efe18dbde 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java @@ -22,8 +22,7 @@ import org.junit.Test; import org.apache.spark.SparkConf; -import org.apache.spark.memory.TestMemoryManager; -import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.memory.*; import org.apache.spark.unsafe.memory.MemoryBlock; import static org.apache.spark.shuffle.sort.PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES; @@ -35,11 +34,12 @@ public class PackedRecordPointerSuite { @Test public void heap() throws IOException { - final SparkConf conf = new SparkConf().set("spark.unsafe.offHeap", "false"); + final SparkConf conf = new SparkConf().set("spark.memory.offHeap.enabled", "false"); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock page0 = memoryManager.allocatePage(128, null); - final MemoryBlock page1 = memoryManager.allocatePage(128, null); + final MemoryConsumer c = new TestMemoryConsumer(memoryManager, MemoryMode.ON_HEAP); + final MemoryBlock page0 = memoryManager.allocatePage(128, c); + final MemoryBlock page1 = memoryManager.allocatePage(128, c); final long addressInPage1 = memoryManager.encodePageNumberAndOffset(page1, page1.getBaseOffset() + 42); PackedRecordPointer packedPointer = new PackedRecordPointer(); @@ -54,11 +54,14 @@ public void heap() throws IOException { @Test public void offHeap() throws IOException { - final SparkConf conf = new SparkConf().set("spark.unsafe.offHeap", "true"); + final SparkConf conf = new SparkConf() + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "10000"); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock page0 = memoryManager.allocatePage(128, null); - final MemoryBlock page1 = memoryManager.allocatePage(128, null); + final MemoryConsumer c = new TestMemoryConsumer(memoryManager, MemoryMode.OFF_HEAP); + final MemoryBlock page0 = memoryManager.allocatePage(128, c); + final MemoryBlock page1 = memoryManager.allocatePage(128, c); final long addressInPage1 = memoryManager.encodePageNumberAndOffset(page1, page1.getBaseOffset() + 42); PackedRecordPointer packedPointer = new PackedRecordPointer(); diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemoryRadixSorterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemoryRadixSorterSuite.java new file mode 100644 index 000000000000..6927d0a81590 --- /dev/null +++ b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemoryRadixSorterSuite.java @@ -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.shuffle.sort; + +public class ShuffleInMemoryRadixSorterSuite extends ShuffleInMemorySorterSuite { + @Override + protected boolean shouldUseRadixSort() { return true; } +} diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java index 2293b1bbc113..694352ee2af4 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java @@ -17,6 +17,7 @@ package org.apache.spark.shuffle.sort; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Random; @@ -25,24 +26,34 @@ import org.apache.spark.HashPartitioner; import org.apache.spark.SparkConf; -import org.apache.spark.unsafe.Platform; +import org.apache.spark.memory.MemoryConsumer; +import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.memory.TestMemoryConsumer; import org.apache.spark.memory.TestMemoryManager; +import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.memory.MemoryBlock; -import org.apache.spark.memory.TaskMemoryManager; public class ShuffleInMemorySorterSuite { + protected boolean shouldUseRadixSort() { return false; } + + final TestMemoryManager memoryManager = + new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")); + final TaskMemoryManager taskMemoryManager = new TaskMemoryManager(memoryManager, 0); + final TestMemoryConsumer consumer = new TestMemoryConsumer(taskMemoryManager); + private static String getStringFromDataPage(Object baseObject, long baseOffset, int strLength) { final byte[] strBytes = new byte[strLength]; Platform.copyMemory(baseObject, baseOffset, strBytes, Platform.BYTE_ARRAY_OFFSET, strLength); - return new String(strBytes); + return new String(strBytes, StandardCharsets.UTF_8); } @Test public void testSortingEmptyInput() { - final ShuffleInMemorySorter sorter = new ShuffleInMemorySorter(100); + final ShuffleInMemorySorter sorter = new ShuffleInMemorySorter( + consumer, 100, shouldUseRadixSort()); final ShuffleInMemorySorter.ShuffleSorterIterator iter = sorter.getSortedIterator(); - assert(!iter.hasNext()); + Assert.assertFalse(iter.hasNext()); } @Test @@ -58,19 +69,25 @@ public void testBasicSorting() throws Exception { "Lychee", "Mango" }; - final SparkConf conf = new SparkConf().set("spark.unsafe.offHeap", "false"); + final SparkConf conf = new SparkConf().set("spark.memory.offHeap.enabled", "false"); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); - final MemoryBlock dataPage = memoryManager.allocatePage(2048, null); + final MemoryConsumer c = new TestMemoryConsumer(memoryManager); + final MemoryBlock dataPage = memoryManager.allocatePage(2048, c); final Object baseObject = dataPage.getBaseObject(); - final ShuffleInMemorySorter sorter = new ShuffleInMemorySorter(4); + final ShuffleInMemorySorter sorter = new ShuffleInMemorySorter( + consumer, 4, shouldUseRadixSort()); final HashPartitioner hashPartitioner = new HashPartitioner(4); // Write the records into the data page and store pointers into the sorter long position = dataPage.getBaseOffset(); for (String str : dataToSort) { + if (!sorter.hasSpaceForAnotherRecord()) { + sorter.expandPointerArray( + consumer.allocateArray(sorter.getMemoryUsage() / 8 * 2)); + } final long recordAddress = memoryManager.encodePageNumberAndOffset(dataPage, position); - final byte[] strBytes = str.getBytes("utf-8"); + final byte[] strBytes = str.getBytes(StandardCharsets.UTF_8); Platform.putInt(baseObject, position, strBytes.length); position += 4; Platform.copyMemory( @@ -104,10 +121,13 @@ public void testBasicSorting() throws Exception { @Test public void testSortingManyNumbers() throws Exception { - ShuffleInMemorySorter sorter = new ShuffleInMemorySorter(4); + ShuffleInMemorySorter sorter = new ShuffleInMemorySorter(consumer, 4, shouldUseRadixSort()); int[] numbersToSort = new int[128000]; Random random = new Random(16); for (int i = 0; i < numbersToSort.length; i++) { + if (!sorter.hasSpaceForAnotherRecord()) { + sorter.expandPointerArray(consumer.allocateArray(sorter.getMemoryUsage() / 8 * 2)); + } numbersToSort[i] = random.nextInt(PackedRecordPointer.MAXIMUM_PARTITION_ID + 1); sorter.insertRecord(0, numbersToSort[i]); } diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 4763395d7d40..24a55df84a24 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -21,43 +21,47 @@ import java.nio.ByteBuffer; import java.util.*; -import scala.*; +import scala.Option; +import scala.Product2; +import scala.Tuple2; +import scala.Tuple2$; import scala.collection.Iterator; -import scala.runtime.AbstractFunction1; -import com.google.common.collect.Iterators; import com.google.common.collect.HashMultiset; -import com.google.common.io.ByteStreams; +import com.google.common.collect.Iterators; import org.junit.After; import org.junit.Before; import org.junit.Test; import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.lessThan; -import static org.junit.Assert.*; -import static org.mockito.Answers.RETURNS_SMART_NULLS; -import static org.mockito.Mockito.*; -import org.apache.spark.*; +import org.apache.spark.HashPartitioner; +import org.apache.spark.ShuffleDependency; +import org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; +import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.executor.TaskMetrics; import org.apache.spark.io.CompressionCodec$; import org.apache.spark.io.LZ4CompressionCodec; import org.apache.spark.io.LZFCompressionCodec; import org.apache.spark.io.SnappyCompressionCodec; -import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.executor.TaskMetrics; +import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.memory.TestMemoryManager; import org.apache.spark.network.util.LimitedInputStream; -import org.apache.spark.serializer.*; import org.apache.spark.scheduler.MapStatus; +import org.apache.spark.security.CryptoStreamUtils; +import org.apache.spark.serializer.*; import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.storage.*; -import org.apache.spark.memory.TestMemoryManager; -import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.util.Utils; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.lessThan; +import static org.junit.Assert.*; +import static org.mockito.Answers.RETURNS_SMART_NULLS; +import static org.mockito.Mockito.*; + public class UnsafeShuffleWriterSuite { static final int NUM_PARTITITONS = 4; @@ -67,7 +71,7 @@ public class UnsafeShuffleWriterSuite { File mergedOutputFile; File tempDir; long[] partitionSizesInMergedFile; - final LinkedList spillFilesCreated = new LinkedList(); + final LinkedList spillFilesCreated = new LinkedList<>(); SparkConf conf; final Serializer serializer = new KryoSerializer(new SparkConf()); TaskMetrics taskMetrics; @@ -78,17 +82,6 @@ public class UnsafeShuffleWriterSuite { @Mock(answer = RETURNS_SMART_NULLS) TaskContext taskContext; @Mock(answer = RETURNS_SMART_NULLS) ShuffleDependency shuffleDep; - private final class CompressStream extends AbstractFunction1 { - @Override - public OutputStream apply(OutputStream stream) { - if (conf.getBoolean("spark.shuffle.compress", true)) { - return CompressionCodec$.MODULE$.createCodec(conf).compressedOutputStream(stream); - } else { - return stream; - } - } - } - @After public void tearDown() { Utils.deleteRecursively(tempDir); @@ -108,10 +101,15 @@ public void setUp() throws IOException { spillFilesCreated.clear(); conf = new SparkConf() .set("spark.buffer.pageSize", "1m") - .set("spark.unsafe.offHeap", "false"); + .set("spark.memory.offHeap.enabled", "false"); taskMetrics = new TaskMetrics(); memoryManager = new TestMemoryManager(conf); - taskMemoryManager = new TaskMemoryManager(memoryManager, 0); + taskMemoryManager = new TaskMemoryManager(memoryManager, 0); + + // Some tests will override this manager because they change the configuration. This is a + // default for tests that don't need a specific one. + SerializerManager manager = new SerializerManager(serializer, conf); + when(blockManager.serializerManager()).thenReturn(manager); when(blockManager.diskBlockManager()).thenReturn(diskBlockManager); when(blockManager.getDiskWriter( @@ -119,87 +117,49 @@ public void setUp() throws IOException { any(File.class), any(SerializerInstance.class), anyInt(), - any(ShuffleWriteMetrics.class))).thenAnswer(new Answer() { - @Override - public DiskBlockObjectWriter answer(InvocationOnMock invocationOnMock) throws Throwable { + any(ShuffleWriteMetrics.class))).thenAnswer(invocationOnMock -> { Object[] args = invocationOnMock.getArguments(); - return new DiskBlockObjectWriter( (File) args[1], + blockManager.serializerManager(), (SerializerInstance) args[2], (Integer) args[3], - new CompressStream(), false, - (ShuffleWriteMetrics) args[4] + (ShuffleWriteMetrics) args[4], + (BlockId) args[0] ); - } - }); - when(blockManager.wrapForCompression(any(BlockId.class), any(InputStream.class))).thenAnswer( - new Answer() { - @Override - public InputStream answer(InvocationOnMock invocation) throws Throwable { - assert (invocation.getArguments()[0] instanceof TempShuffleBlockId); - InputStream is = (InputStream) invocation.getArguments()[1]; - if (conf.getBoolean("spark.shuffle.compress", true)) { - return CompressionCodec$.MODULE$.createCodec(conf).compressedInputStream(is); - } else { - return is; - } - } - } - ); - - when(blockManager.wrapForCompression(any(BlockId.class), any(OutputStream.class))).thenAnswer( - new Answer() { - @Override - public OutputStream answer(InvocationOnMock invocation) throws Throwable { - assert (invocation.getArguments()[0] instanceof TempShuffleBlockId); - OutputStream os = (OutputStream) invocation.getArguments()[1]; - if (conf.getBoolean("spark.shuffle.compress", true)) { - return CompressionCodec$.MODULE$.createCodec(conf).compressedOutputStream(os); - } else { - return os; - } - } - } - ); + }); when(shuffleBlockResolver.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocationOnMock) throws Throwable { - partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; - return null; - } - }).when(shuffleBlockResolver).writeIndexFile(anyInt(), anyInt(), any(long[].class)); - - when(diskBlockManager.createTempShuffleBlock()).thenAnswer( - new Answer>() { - @Override - public Tuple2 answer( - InvocationOnMock invocationOnMock) throws Throwable { - TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); - File file = File.createTempFile("spillFile", ".spill", tempDir); - spillFilesCreated.add(file); - return Tuple2$.MODULE$.apply(blockId, file); - } - }); + doAnswer(invocationOnMock -> { + partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; + File tmp = (File) invocationOnMock.getArguments()[3]; + mergedOutputFile.delete(); + tmp.renameTo(mergedOutputFile); + return null; + }).when(shuffleBlockResolver) + .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), any(File.class)); + + when(diskBlockManager.createTempShuffleBlock()).thenAnswer(invocationOnMock -> { + TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); + File file = File.createTempFile("spillFile", ".spill", tempDir); + spillFilesCreated.add(file); + return Tuple2$.MODULE$.apply(blockId, file); + }); when(taskContext.taskMetrics()).thenReturn(taskMetrics); - when(taskContext.internalMetricsToAccumulators()).thenReturn(null); - - when(shuffleDep.serializer()).thenReturn(Option.apply(serializer)); + when(shuffleDep.serializer()).thenReturn(serializer); when(shuffleDep.partitioner()).thenReturn(hashPartitioner); } private UnsafeShuffleWriter createWriter( boolean transferToEnabled) throws IOException { conf.set("spark.file.transferTo", String.valueOf(transferToEnabled)); - return new UnsafeShuffleWriter( + return new UnsafeShuffleWriter<>( blockManager, shuffleBlockResolver, taskMemoryManager, - new SerializedShuffleHandle(0, 1, shuffleDep), + new SerializedShuffleHandle<>(0, 1, shuffleDep), 0, // map id taskContext, conf @@ -214,14 +174,15 @@ private void assertSpillFilesWereCleanedUp() { } private List> readRecordsFromFile() throws IOException { - final ArrayList> recordsList = new ArrayList>(); + final ArrayList> recordsList = new ArrayList<>(); long startOffset = 0; for (int i = 0; i < NUM_PARTITITONS; i++) { final long partitionSize = partitionSizesInMergedFile[i]; if (partitionSize > 0) { - InputStream in = new FileInputStream(mergedOutputFile); - ByteStreams.skipFully(in, startOffset); - in = new LimitedInputStream(in, partitionSize); + FileInputStream fin = new FileInputStream(mergedOutputFile); + fin.getChannel().position(startOffset); + InputStream in = new LimitedInputStream(fin, partitionSize); + in = blockManager.serializerManager().wrapForEncryption(in); if (conf.getBoolean("spark.shuffle.compress", true)) { in = CompressionCodec$.MODULE$.createCodec(conf).compressedInputStream(in); } @@ -249,7 +210,7 @@ public void doNotNeedToCallWriteBeforeUnsuccessfulStop() throws IOException { createWriter(false).stop(false); } - class PandaException extends RuntimeException { + static class PandaException extends RuntimeException { } @Test(expected=PandaException.class) @@ -269,13 +230,13 @@ class BadRecords extends scala.collection.AbstractIterator writer = createWriter(true); - writer.write(Iterators.>emptyIterator()); + writer.write(Iterators.emptyIterator()); final Option mapStatus = writer.stop(true); assertTrue(mapStatus.isDefined()); assertTrue(mergedOutputFile.exists()); assertArrayEquals(new long[NUM_PARTITITONS], partitionSizesInMergedFile); - assertEquals(0, taskMetrics.shuffleWriteMetrics().get().shuffleRecordsWritten()); - assertEquals(0, taskMetrics.shuffleWriteMetrics().get().shuffleBytesWritten()); + assertEquals(0, taskMetrics.shuffleWriteMetrics().recordsWritten()); + assertEquals(0, taskMetrics.shuffleWriteMetrics().bytesWritten()); assertEquals(0, taskMetrics.diskBytesSpilled()); assertEquals(0, taskMetrics.memoryBytesSpilled()); } @@ -283,10 +244,9 @@ public void writeEmptyIterator() throws Exception { @Test public void writeWithoutSpilling() throws Exception { // In this example, each partition should have exactly one record: - final ArrayList> dataToWrite = - new ArrayList>(); + final ArrayList> dataToWrite = new ArrayList<>(); for (int i = 0; i < NUM_PARTITITONS; i++) { - dataToWrite.add(new Tuple2(i, i)); + dataToWrite.add(new Tuple2<>(i, i)); } final UnsafeShuffleWriter writer = createWriter(true); writer.write(dataToWrite.iterator()); @@ -305,27 +265,44 @@ public void writeWithoutSpilling() throws Exception { HashMultiset.create(dataToWrite), HashMultiset.create(readRecordsFromFile())); assertSpillFilesWereCleanedUp(); - ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics().get(); - assertEquals(dataToWrite.size(), shuffleWriteMetrics.shuffleRecordsWritten()); + ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics(); + assertEquals(dataToWrite.size(), shuffleWriteMetrics.recordsWritten()); assertEquals(0, taskMetrics.diskBytesSpilled()); assertEquals(0, taskMetrics.memoryBytesSpilled()); - assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.shuffleBytesWritten()); + assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.bytesWritten()); } private void testMergingSpills( - boolean transferToEnabled, - String compressionCodecName) throws IOException { + final boolean transferToEnabled, + String compressionCodecName, + boolean encrypt) throws Exception { if (compressionCodecName != null) { conf.set("spark.shuffle.compress", "true"); conf.set("spark.io.compression.codec", compressionCodecName); } else { conf.set("spark.shuffle.compress", "false"); } + conf.set(org.apache.spark.internal.config.package$.MODULE$.IO_ENCRYPTION_ENABLED(), encrypt); + + SerializerManager manager; + if (encrypt) { + manager = new SerializerManager(serializer, conf, + Option.apply(CryptoStreamUtils.createKey(conf))); + } else { + manager = new SerializerManager(serializer, conf); + } + + when(blockManager.serializerManager()).thenReturn(manager); + testMergingSpills(transferToEnabled, encrypt); + } + + private void testMergingSpills( + boolean transferToEnabled, + boolean encrypted) throws IOException { final UnsafeShuffleWriter writer = createWriter(transferToEnabled); - final ArrayList> dataToWrite = - new ArrayList>(); + final ArrayList> dataToWrite = new ArrayList<>(); for (int i : new int[] { 1, 2, 3, 4, 4, 2 }) { - dataToWrite.add(new Tuple2(i, i)); + dataToWrite.add(new Tuple2<>(i, i)); } writer.insertRecordIntoSorter(dataToWrite.get(0)); writer.insertRecordIntoSorter(dataToWrite.get(1)); @@ -344,109 +321,151 @@ private void testMergingSpills( for (long size: partitionSizesInMergedFile) { sumOfPartitionSizes += size; } + assertEquals(sumOfPartitionSizes, mergedOutputFile.length()); assertEquals(HashMultiset.create(dataToWrite), HashMultiset.create(readRecordsFromFile())); assertSpillFilesWereCleanedUp(); - ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics().get(); - assertEquals(dataToWrite.size(), shuffleWriteMetrics.shuffleRecordsWritten()); + ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics(); + assertEquals(dataToWrite.size(), shuffleWriteMetrics.recordsWritten()); assertThat(taskMetrics.diskBytesSpilled(), greaterThan(0L)); assertThat(taskMetrics.diskBytesSpilled(), lessThan(mergedOutputFile.length())); assertThat(taskMetrics.memoryBytesSpilled(), greaterThan(0L)); - assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.shuffleBytesWritten()); + assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.bytesWritten()); } @Test public void mergeSpillsWithTransferToAndLZF() throws Exception { - testMergingSpills(true, LZFCompressionCodec.class.getName()); + testMergingSpills(true, LZFCompressionCodec.class.getName(), false); } @Test public void mergeSpillsWithFileStreamAndLZF() throws Exception { - testMergingSpills(false, LZFCompressionCodec.class.getName()); + testMergingSpills(false, LZFCompressionCodec.class.getName(), false); } @Test public void mergeSpillsWithTransferToAndLZ4() throws Exception { - testMergingSpills(true, LZ4CompressionCodec.class.getName()); + testMergingSpills(true, LZ4CompressionCodec.class.getName(), false); } @Test public void mergeSpillsWithFileStreamAndLZ4() throws Exception { - testMergingSpills(false, LZ4CompressionCodec.class.getName()); + testMergingSpills(false, LZ4CompressionCodec.class.getName(), false); } @Test public void mergeSpillsWithTransferToAndSnappy() throws Exception { - testMergingSpills(true, SnappyCompressionCodec.class.getName()); + testMergingSpills(true, SnappyCompressionCodec.class.getName(), false); } @Test public void mergeSpillsWithFileStreamAndSnappy() throws Exception { - testMergingSpills(false, SnappyCompressionCodec.class.getName()); + testMergingSpills(false, SnappyCompressionCodec.class.getName(), false); } @Test public void mergeSpillsWithTransferToAndNoCompression() throws Exception { - testMergingSpills(true, null); + testMergingSpills(true, null, false); } @Test public void mergeSpillsWithFileStreamAndNoCompression() throws Exception { - testMergingSpills(false, null); + testMergingSpills(false, null, false); + } + + @Test + public void mergeSpillsWithCompressionAndEncryption() throws Exception { + // This should actually be translated to a "file stream merge" internally, just have the + // test to make sure that it's the case. + testMergingSpills(true, LZ4CompressionCodec.class.getName(), true); + } + + @Test + public void mergeSpillsWithFileStreamAndCompressionAndEncryption() throws Exception { + testMergingSpills(false, LZ4CompressionCodec.class.getName(), true); + } + + @Test + public void mergeSpillsWithCompressionAndEncryptionSlowPath() throws Exception { + conf.set("spark.shuffle.unsafe.fastMergeEnabled", "false"); + testMergingSpills(false, LZ4CompressionCodec.class.getName(), true); + } + + @Test + public void mergeSpillsWithEncryptionAndNoCompression() throws Exception { + // This should actually be translated to a "file stream merge" internally, just have the + // test to make sure that it's the case. + testMergingSpills(true, null, true); + } + + @Test + public void mergeSpillsWithFileStreamAndEncryptionAndNoCompression() throws Exception { + testMergingSpills(false, null, true); } @Test public void writeEnoughDataToTriggerSpill() throws Exception { memoryManager.limit(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES); final UnsafeShuffleWriter writer = createWriter(false); - final ArrayList> dataToWrite = new ArrayList>(); + final ArrayList> dataToWrite = new ArrayList<>(); final byte[] bigByteArray = new byte[PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES / 10]; for (int i = 0; i < 10 + 1; i++) { - dataToWrite.add(new Tuple2(i, bigByteArray)); + dataToWrite.add(new Tuple2<>(i, bigByteArray)); } writer.write(dataToWrite.iterator()); assertEquals(2, spillFilesCreated.size()); writer.stop(true); readRecordsFromFile(); assertSpillFilesWereCleanedUp(); - ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics().get(); - assertEquals(dataToWrite.size(), shuffleWriteMetrics.shuffleRecordsWritten()); + ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics(); + assertEquals(dataToWrite.size(), shuffleWriteMetrics.recordsWritten()); assertThat(taskMetrics.diskBytesSpilled(), greaterThan(0L)); assertThat(taskMetrics.diskBytesSpilled(), lessThan(mergedOutputFile.length())); assertThat(taskMetrics.memoryBytesSpilled(), greaterThan(0L)); - assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.shuffleBytesWritten()); + assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.bytesWritten()); } @Test - public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exception { - memoryManager.limit(UnsafeShuffleWriter.INITIAL_SORT_BUFFER_SIZE * 16); + public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpillRadixOff() throws Exception { + conf.set("spark.shuffle.sort.useRadixSort", "false"); + writeEnoughRecordsToTriggerSortBufferExpansionAndSpill(); + assertEquals(2, spillFilesCreated.size()); + } + + @Test + public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpillRadixOn() throws Exception { + conf.set("spark.shuffle.sort.useRadixSort", "true"); + writeEnoughRecordsToTriggerSortBufferExpansionAndSpill(); + assertEquals(3, spillFilesCreated.size()); + } + + private void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exception { + memoryManager.limit(UnsafeShuffleWriter.DEFAULT_INITIAL_SORT_BUFFER_SIZE * 16); final UnsafeShuffleWriter writer = createWriter(false); final ArrayList> dataToWrite = new ArrayList<>(); - for (int i = 0; i < UnsafeShuffleWriter.INITIAL_SORT_BUFFER_SIZE; i++) { - dataToWrite.add(new Tuple2(i, i)); + for (int i = 0; i < UnsafeShuffleWriter.DEFAULT_INITIAL_SORT_BUFFER_SIZE + 1; i++) { + dataToWrite.add(new Tuple2<>(i, i)); } writer.write(dataToWrite.iterator()); - assertEquals(2, spillFilesCreated.size()); writer.stop(true); readRecordsFromFile(); assertSpillFilesWereCleanedUp(); - ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics().get(); - assertEquals(dataToWrite.size(), shuffleWriteMetrics.shuffleRecordsWritten()); + ShuffleWriteMetrics shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics(); + assertEquals(dataToWrite.size(), shuffleWriteMetrics.recordsWritten()); assertThat(taskMetrics.diskBytesSpilled(), greaterThan(0L)); assertThat(taskMetrics.diskBytesSpilled(), lessThan(mergedOutputFile.length())); assertThat(taskMetrics.memoryBytesSpilled(), greaterThan(0L)); - assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.shuffleBytesWritten()); + assertEquals(mergedOutputFile.length(), shuffleWriteMetrics.bytesWritten()); } @Test public void writeRecordsThatAreBiggerThanDiskWriteBufferSize() throws Exception { final UnsafeShuffleWriter writer = createWriter(false); - final ArrayList> dataToWrite = - new ArrayList>(); + final ArrayList> dataToWrite = new ArrayList<>(); final byte[] bytes = new byte[(int) (ShuffleExternalSorter.DISK_WRITE_BUFFER_SIZE * 2.5)]; new Random(42).nextBytes(bytes); - dataToWrite.add(new Tuple2(1, ByteBuffer.wrap(bytes))); + dataToWrite.add(new Tuple2<>(1, ByteBuffer.wrap(bytes))); writer.write(dataToWrite.iterator()); writer.stop(true); assertEquals( @@ -458,16 +477,16 @@ public void writeRecordsThatAreBiggerThanDiskWriteBufferSize() throws Exception @Test public void writeRecordsThatAreBiggerThanMaxRecordSize() throws Exception { final UnsafeShuffleWriter writer = createWriter(false); - final ArrayList> dataToWrite = new ArrayList>(); - dataToWrite.add(new Tuple2(1, ByteBuffer.wrap(new byte[1]))); + final ArrayList> dataToWrite = new ArrayList<>(); + dataToWrite.add(new Tuple2<>(1, ByteBuffer.wrap(new byte[1]))); // We should be able to write a record that's right _at_ the max record size final byte[] atMaxRecordSize = new byte[(int) taskMemoryManager.pageSizeBytes() - 4]; new Random(42).nextBytes(atMaxRecordSize); - dataToWrite.add(new Tuple2(2, ByteBuffer.wrap(atMaxRecordSize))); + dataToWrite.add(new Tuple2<>(2, ByteBuffer.wrap(atMaxRecordSize))); // Inserting a record that's larger than the max record size final byte[] exceedsMaxRecordSize = new byte[(int) taskMemoryManager.pageSizeBytes()]; new Random(42).nextBytes(exceedsMaxRecordSize); - dataToWrite.add(new Tuple2(3, ByteBuffer.wrap(exceedsMaxRecordSize))); + dataToWrite.add(new Tuple2<>(3, ByteBuffer.wrap(exceedsMaxRecordSize))); writer.write(dataToWrite.iterator()); writer.stop(true); assertEquals( @@ -479,10 +498,10 @@ public void writeRecordsThatAreBiggerThanMaxRecordSize() throws Exception { @Test public void spillFilesAreDeletedWhenStoppingAfterError() throws IOException { final UnsafeShuffleWriter writer = createWriter(false); - writer.insertRecordIntoSorter(new Tuple2(1, 1)); - writer.insertRecordIntoSorter(new Tuple2(2, 2)); + writer.insertRecordIntoSorter(new Tuple2<>(1, 1)); + writer.insertRecordIntoSorter(new Tuple2<>(2, 2)); writer.forceSorterToSpill(); - writer.insertRecordIntoSorter(new Tuple2(2, 2)); + writer.insertRecordIntoSorter(new Tuple2<>(2, 2)); writer.stop(false); assertSpillFilesWereCleanedUp(); } @@ -495,7 +514,7 @@ public void testPeakMemoryUsed() throws Exception { taskMemoryManager = spy(taskMemoryManager); when(taskMemoryManager.pageSizeBytes()).thenReturn(pageSizeBytes); final UnsafeShuffleWriter writer = - new UnsafeShuffleWriter( + new UnsafeShuffleWriter<>( blockManager, shuffleBlockResolver, taskMemoryManager, @@ -540,4 +559,5 @@ public void testPeakMemoryUsed() throws Exception { writer.stop(false); } } + } diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java index 92bd45e5fa24..03cec8ed81b7 100644 --- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java +++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java @@ -19,14 +19,10 @@ import java.io.File; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.nio.ByteBuffer; import java.util.*; -import scala.Tuple2; import scala.Tuple2$; -import scala.runtime.AbstractFunction1; import org.junit.After; import org.junit.Assert; @@ -34,24 +30,23 @@ import org.junit.Test; import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import org.apache.spark.SparkConf; import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.memory.TestMemoryManager; import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.memory.TestMemoryManager; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.serializer.JavaSerializer; import org.apache.spark.serializer.SerializerInstance; +import org.apache.spark.serializer.SerializerManager; import org.apache.spark.storage.*; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; -import org.apache.spark.unsafe.memory.MemoryLocation; import org.apache.spark.util.Utils; import static org.hamcrest.Matchers.greaterThan; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.mockito.AdditionalAnswers.returnsSecondArg; import static org.mockito.Answers.RETURNS_SMART_NULLS; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; @@ -64,63 +59,56 @@ public abstract class AbstractBytesToBytesMapSuite { private TestMemoryManager memoryManager; private TaskMemoryManager taskMemoryManager; - private final long PAGE_SIZE_BYTES = 1L << 26; // 64 megabytes + private SerializerManager serializerManager = new SerializerManager( + new JavaSerializer(new SparkConf()), + new SparkConf().set("spark.shuffle.spill.compress", "false")); + private static final long PAGE_SIZE_BYTES = 1L << 26; // 64 megabytes - final LinkedList spillFilesCreated = new LinkedList(); + final LinkedList spillFilesCreated = new LinkedList<>(); File tempDir; @Mock(answer = RETURNS_SMART_NULLS) BlockManager blockManager; @Mock(answer = RETURNS_SMART_NULLS) DiskBlockManager diskBlockManager; - private static final class CompressStream extends AbstractFunction1 { - @Override - public OutputStream apply(OutputStream stream) { - return stream; - } - } - @Before public void setup() { memoryManager = new TestMemoryManager( - new SparkConf().set("spark.unsafe.offHeap", "" + useOffHeapMemoryAllocator())); + new SparkConf() + .set("spark.memory.offHeap.enabled", "" + useOffHeapMemoryAllocator()) + .set("spark.memory.offHeap.size", "256mb") + .set("spark.shuffle.spill.compress", "false") + .set("spark.shuffle.compress", "false")); taskMemoryManager = new TaskMemoryManager(memoryManager, 0); tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "unsafe-test"); spillFilesCreated.clear(); MockitoAnnotations.initMocks(this); when(blockManager.diskBlockManager()).thenReturn(diskBlockManager); - when(diskBlockManager.createTempLocalBlock()).thenAnswer(new Answer>() { - @Override - public Tuple2 answer(InvocationOnMock invocationOnMock) throws Throwable { - TempLocalBlockId blockId = new TempLocalBlockId(UUID.randomUUID()); - File file = File.createTempFile("spillFile", ".spill", tempDir); - spillFilesCreated.add(file); - return Tuple2$.MODULE$.apply(blockId, file); - } + when(diskBlockManager.createTempLocalBlock()).thenAnswer(invocationOnMock -> { + TempLocalBlockId blockId = new TempLocalBlockId(UUID.randomUUID()); + File file = File.createTempFile("spillFile", ".spill", tempDir); + spillFilesCreated.add(file); + return Tuple2$.MODULE$.apply(blockId, file); }); when(blockManager.getDiskWriter( any(BlockId.class), any(File.class), any(SerializerInstance.class), anyInt(), - any(ShuffleWriteMetrics.class))).thenAnswer(new Answer() { - @Override - public DiskBlockObjectWriter answer(InvocationOnMock invocationOnMock) throws Throwable { + any(ShuffleWriteMetrics.class))).thenAnswer(invocationOnMock -> { Object[] args = invocationOnMock.getArguments(); return new DiskBlockObjectWriter( (File) args[1], + serializerManager, (SerializerInstance) args[2], (Integer) args[3], - new CompressStream(), false, - (ShuffleWriteMetrics) args[4] + (ShuffleWriteMetrics) args[4], + (BlockId) args[0] ); - } - }); - when(blockManager.wrapForCompression(any(BlockId.class), any(InputStream.class))) - .then(returnsSecondArg()); + }); } @After @@ -128,8 +116,8 @@ public void tearDown() { Utils.deleteRecursively(tempDir); tempDir = null; - Assert.assertEquals(0L, taskMemoryManager.cleanUpAllAllocatedMemory()); if (taskMemoryManager != null) { + Assert.assertEquals(0L, taskMemoryManager.cleanUpAllAllocatedMemory()); long leakedMemory = taskMemoryManager.getMemoryConsumptionForThisTask(); taskMemoryManager = null; Assert.assertEquals(0L, leakedMemory); @@ -138,10 +126,9 @@ public void tearDown() { protected abstract boolean useOffHeapMemoryAllocator(); - private static byte[] getByteArray(MemoryLocation loc, int size) { + private static byte[] getByteArray(Object base, long offset, int size) { final byte[] arr = new byte[size]; - Platform.copyMemory( - loc.getBaseObject(), loc.getBaseOffset(), arr, Platform.BYTE_ARRAY_OFFSET, size); + Platform.copyMemory(base, offset, arr, Platform.BYTE_ARRAY_OFFSET, size); return arr; } @@ -159,13 +146,14 @@ private byte[] getRandomByteArray(int numWords) { */ private static boolean arrayEquals( byte[] expected, - MemoryLocation actualAddr, + Object base, + long offset, long actualLengthBytes) { return (actualLengthBytes == expected.length) && ByteArrayMethods.arrayEquals( expected, Platform.BYTE_ARRAY_OFFSET, - actualAddr.getBaseObject(), - actualAddr.getBaseOffset(), + base, + offset, expected.length ); } @@ -174,7 +162,7 @@ private static boolean arrayEquals( public void emptyMap() { BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager, 64, PAGE_SIZE_BYTES); try { - Assert.assertEquals(0, map.numElements()); + Assert.assertEquals(0, map.numKeys()); final int keyLengthInWords = 10; final int keyLengthInBytes = keyLengthInWords * 8; final byte[] key = getRandomByteArray(keyLengthInWords); @@ -196,7 +184,7 @@ public void setAndRetrieveAKey() { final BytesToBytesMap.Location loc = map.lookup(keyData, Platform.BYTE_ARRAY_OFFSET, recordLengthBytes); Assert.assertFalse(loc.isDefined()); - Assert.assertTrue(loc.putNewKey( + Assert.assertTrue(loc.append( keyData, Platform.BYTE_ARRAY_OFFSET, recordLengthBytes, @@ -208,19 +196,23 @@ public void setAndRetrieveAKey() { // reflect the result of this store without us having to call lookup() again on the same key. Assert.assertEquals(recordLengthBytes, loc.getKeyLength()); Assert.assertEquals(recordLengthBytes, loc.getValueLength()); - Assert.assertArrayEquals(keyData, getByteArray(loc.getKeyAddress(), recordLengthBytes)); - Assert.assertArrayEquals(valueData, getByteArray(loc.getValueAddress(), recordLengthBytes)); + Assert.assertArrayEquals(keyData, + getByteArray(loc.getKeyBase(), loc.getKeyOffset(), recordLengthBytes)); + Assert.assertArrayEquals(valueData, + getByteArray(loc.getValueBase(), loc.getValueOffset(), recordLengthBytes)); // After calling lookup() the location should still point to the correct data. Assert.assertTrue( map.lookup(keyData, Platform.BYTE_ARRAY_OFFSET, recordLengthBytes).isDefined()); Assert.assertEquals(recordLengthBytes, loc.getKeyLength()); Assert.assertEquals(recordLengthBytes, loc.getValueLength()); - Assert.assertArrayEquals(keyData, getByteArray(loc.getKeyAddress(), recordLengthBytes)); - Assert.assertArrayEquals(valueData, getByteArray(loc.getValueAddress(), recordLengthBytes)); + Assert.assertArrayEquals(keyData, + getByteArray(loc.getKeyBase(), loc.getKeyOffset(), recordLengthBytes)); + Assert.assertArrayEquals(valueData, + getByteArray(loc.getValueBase(), loc.getValueOffset(), recordLengthBytes)); try { - Assert.assertTrue(loc.putNewKey( + Assert.assertTrue(loc.append( keyData, Platform.BYTE_ARRAY_OFFSET, recordLengthBytes, @@ -248,7 +240,7 @@ private void iteratorTestBase(boolean destructive) throws Exception { Assert.assertFalse(loc.isDefined()); // Ensure that we store some zero-length keys if (i % 5 == 0) { - Assert.assertTrue(loc.putNewKey( + Assert.assertTrue(loc.append( null, Platform.LONG_ARRAY_OFFSET, 0, @@ -257,7 +249,7 @@ private void iteratorTestBase(boolean destructive) throws Exception { 8 )); } else { - Assert.assertTrue(loc.putNewKey( + Assert.assertTrue(loc.append( value, Platform.LONG_ARRAY_OFFSET, 8, @@ -279,15 +271,12 @@ private void iteratorTestBase(boolean destructive) throws Exception { while (iter.hasNext()) { final BytesToBytesMap.Location loc = iter.next(); Assert.assertTrue(loc.isDefined()); - final MemoryLocation keyAddress = loc.getKeyAddress(); - final MemoryLocation valueAddress = loc.getValueAddress(); - final long value = Platform.getLong( - valueAddress.getBaseObject(), valueAddress.getBaseOffset()); + final long value = Platform.getLong(loc.getValueBase(), loc.getValueOffset()); final long keyLength = loc.getKeyLength(); if (keyLength == 0) { Assert.assertTrue("value " + value + " was not divisible by 5", value % 5 == 0); } else { - final long key = Platform.getLong(keyAddress.getBaseObject(), keyAddress.getBaseOffset()); + final long key = Platform.getLong(loc.getKeyBase(), loc.getKeyOffset()); Assert.assertEquals(value, key); } valuesSeen.set((int) value); @@ -340,7 +329,7 @@ public void iteratingOverDataPagesWithWastedSpace() throws Exception { KEY_LENGTH ); Assert.assertFalse(loc.isDefined()); - Assert.assertTrue(loc.putNewKey( + Assert.assertTrue(loc.append( key, Platform.LONG_ARRAY_OFFSET, KEY_LENGTH, @@ -353,23 +342,23 @@ public void iteratingOverDataPagesWithWastedSpace() throws Exception { final java.util.BitSet valuesSeen = new java.util.BitSet(NUM_ENTRIES); final Iterator iter = map.iterator(); - final long key[] = new long[KEY_LENGTH / 8]; - final long value[] = new long[VALUE_LENGTH / 8]; + final long[] key = new long[KEY_LENGTH / 8]; + final long[] value = new long[VALUE_LENGTH / 8]; while (iter.hasNext()) { final BytesToBytesMap.Location loc = iter.next(); Assert.assertTrue(loc.isDefined()); Assert.assertEquals(KEY_LENGTH, loc.getKeyLength()); Assert.assertEquals(VALUE_LENGTH, loc.getValueLength()); Platform.copyMemory( - loc.getKeyAddress().getBaseObject(), - loc.getKeyAddress().getBaseOffset(), + loc.getKeyBase(), + loc.getKeyOffset(), key, Platform.LONG_ARRAY_OFFSET, KEY_LENGTH ); Platform.copyMemory( - loc.getValueAddress().getBaseObject(), - loc.getValueAddress().getBaseOffset(), + loc.getValueBase(), + loc.getValueOffset(), value, Platform.LONG_ARRAY_OFFSET, VALUE_LENGTH @@ -393,7 +382,7 @@ public void randomizedStressTest() { final int size = 65536; // Java arrays' hashCodes() aren't based on the arrays' contents, so we need to wrap arrays // into ByteBuffers in order to use them as keys here. - final Map expected = new HashMap(); + final Map expected = new HashMap<>(); final BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager, size, PAGE_SIZE_BYTES); try { // Fill the map to 90% full so that we can trigger probing @@ -408,7 +397,7 @@ public void randomizedStressTest() { key.length ); Assert.assertFalse(loc.isDefined()); - Assert.assertTrue(loc.putNewKey( + Assert.assertTrue(loc.append( key, Platform.BYTE_ARRAY_OFFSET, key.length, @@ -421,19 +410,22 @@ public void randomizedStressTest() { Assert.assertTrue(loc.isDefined()); Assert.assertEquals(key.length, loc.getKeyLength()); Assert.assertEquals(value.length, loc.getValueLength()); - Assert.assertTrue(arrayEquals(key, loc.getKeyAddress(), key.length)); - Assert.assertTrue(arrayEquals(value, loc.getValueAddress(), value.length)); + Assert.assertTrue(arrayEquals(key, loc.getKeyBase(), loc.getKeyOffset(), key.length)); + Assert.assertTrue( + arrayEquals(value, loc.getValueBase(), loc.getValueOffset(), value.length)); } } for (Map.Entry entry : expected.entrySet()) { - final byte[] key = entry.getKey().array(); + final byte[] key = JavaUtils.bufferToArray(entry.getKey()); final byte[] value = entry.getValue(); final BytesToBytesMap.Location loc = map.lookup(key, Platform.BYTE_ARRAY_OFFSET, key.length); Assert.assertTrue(loc.isDefined()); - Assert.assertTrue(arrayEquals(key, loc.getKeyAddress(), loc.getKeyLength())); - Assert.assertTrue(arrayEquals(value, loc.getValueAddress(), loc.getValueLength())); + Assert.assertTrue( + arrayEquals(key, loc.getKeyBase(), loc.getKeyOffset(), loc.getKeyLength())); + Assert.assertTrue( + arrayEquals(value, loc.getValueBase(), loc.getValueOffset(), loc.getValueLength())); } } finally { map.free(); @@ -446,7 +438,7 @@ public void randomizedTestWithRecordsLargerThanPageSize() { final BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager, 64, pageSizeBytes); // Java arrays' hashCodes() aren't based on the arrays' contents, so we need to wrap arrays // into ByteBuffers in order to use them as keys here. - final Map expected = new HashMap(); + final Map expected = new HashMap<>(); try { for (int i = 0; i < 1000; i++) { final byte[] key = getRandomByteArray(rand.nextInt(128)); @@ -459,7 +451,7 @@ public void randomizedTestWithRecordsLargerThanPageSize() { key.length ); Assert.assertFalse(loc.isDefined()); - Assert.assertTrue(loc.putNewKey( + Assert.assertTrue(loc.append( key, Platform.BYTE_ARRAY_OFFSET, key.length, @@ -472,18 +464,21 @@ public void randomizedTestWithRecordsLargerThanPageSize() { Assert.assertTrue(loc.isDefined()); Assert.assertEquals(key.length, loc.getKeyLength()); Assert.assertEquals(value.length, loc.getValueLength()); - Assert.assertTrue(arrayEquals(key, loc.getKeyAddress(), key.length)); - Assert.assertTrue(arrayEquals(value, loc.getValueAddress(), value.length)); + Assert.assertTrue(arrayEquals(key, loc.getKeyBase(), loc.getKeyOffset(), key.length)); + Assert.assertTrue( + arrayEquals(value, loc.getValueBase(), loc.getValueOffset(), value.length)); } } for (Map.Entry entry : expected.entrySet()) { - final byte[] key = entry.getKey().array(); + final byte[] key = JavaUtils.bufferToArray(entry.getKey()); final byte[] value = entry.getValue(); final BytesToBytesMap.Location loc = map.lookup(key, Platform.BYTE_ARRAY_OFFSET, key.length); Assert.assertTrue(loc.isDefined()); - Assert.assertTrue(arrayEquals(key, loc.getKeyAddress(), loc.getKeyLength())); - Assert.assertTrue(arrayEquals(value, loc.getValueAddress(), loc.getValueLength())); + Assert.assertTrue( + arrayEquals(key, loc.getKeyBase(), loc.getKeyOffset(), loc.getKeyLength())); + Assert.assertTrue( + arrayEquals(value, loc.getValueBase(), loc.getValueOffset(), loc.getValueLength())); } } finally { map.free(); @@ -499,7 +494,7 @@ public void failureToAllocateFirstPage() { final BytesToBytesMap.Location loc = map.lookup(emptyArray, Platform.LONG_ARRAY_OFFSET, 0); Assert.assertFalse(loc.isDefined()); - Assert.assertFalse(loc.putNewKey( + Assert.assertFalse(loc.append( emptyArray, Platform.LONG_ARRAY_OFFSET, 0, emptyArray, Platform.LONG_ARRAY_OFFSET, 0)); } finally { map.free(); @@ -520,7 +515,7 @@ public void failureToGrow() { final long[] arr = new long[]{i}; final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8); success = - loc.putNewKey(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); + loc.append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); if (!success) { break; } @@ -534,13 +529,14 @@ public void failureToGrow() { @Test public void spillInIterator() throws IOException { - BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager, blockManager, 1, 0.75, 1024, false); + BytesToBytesMap map = new BytesToBytesMap( + taskMemoryManager, blockManager, serializerManager, 1, 0.75, 1024, false); try { int i; for (i = 0; i < 1024; i++) { final long[] arr = new long[]{i}; final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8); - loc.putNewKey(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); + loc.append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); } BytesToBytesMap.MapIterator iter = map.iterator(); for (i = 0; i < 100; i++) { @@ -570,6 +566,44 @@ public void spillInIterator() throws IOException { } } + @Test + public void multipleValuesForSameKey() { + BytesToBytesMap map = + new BytesToBytesMap(taskMemoryManager, blockManager, serializerManager, 1, 0.5, 1024, false); + try { + int i; + for (i = 0; i < 1024; i++) { + final long[] arr = new long[]{i}; + map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8) + .append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); + } + assert map.numKeys() == 1024; + assert map.numValues() == 1024; + for (i = 0; i < 1024; i++) { + final long[] arr = new long[]{i}; + map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8) + .append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); + } + assert map.numKeys() == 1024; + assert map.numValues() == 2048; + for (i = 0; i < 1024; i++) { + final long[] arr = new long[]{i}; + final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8); + assert loc.isDefined(); + assert loc.nextValue(); + assert !loc.nextValue(); + } + BytesToBytesMap.MapIterator iter = map.iterator(); + for (i = 0; i < 2048; i++) { + assert iter.hasNext(); + final BytesToBytesMap.Location loc = iter.next(); + assert loc.isDefined(); + } + } finally { + map.free(); + } + } + @Test public void initialCapacityBoundsChecking() { try { @@ -592,7 +626,7 @@ public void initialCapacityBoundsChecking() { @Test public void testPeakMemoryUsed() { - final long recordLengthBytes = 24; + final long recordLengthBytes = 32; final long pageSizeBytes = 256 + 8; // 8 bytes for end-of-page marker final long numRecordsPerPage = (pageSizeBytes - 8) / recordLengthBytes; final BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager, 1024, pageSizeBytes); @@ -606,7 +640,7 @@ public void testPeakMemoryUsed() { try { for (long i = 0; i < numRecordsPerPage * 10; i++) { final long[] value = new long[]{i}; - map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8).putNewKey( + map.lookup(value, Platform.LONG_ARRAY_OFFSET, 8).append( value, Platform.LONG_ARRAY_OFFSET, 8, diff --git a/core/src/test/java/org/apache/spark/util/collection/TestTimSort.java b/core/src/test/java/org/apache/spark/util/collection/TestTimSort.java index 45772b6d3c20..e884b1bc123b 100644 --- a/core/src/test/java/org/apache/spark/util/collection/TestTimSort.java +++ b/core/src/test/java/org/apache/spark/util/collection/TestTimSort.java @@ -76,7 +76,7 @@ private static int[] createArray(List runs, int length) { * @param length The sum of all run lengths that will be added to runs. */ private static List runsJDKWorstCase(int minRun, int length) { - List runs = new ArrayList(); + List runs = new ArrayList<>(); long runningTotal = 0, Y = minRun + 4, X = minRun; diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterRadixSortSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterRadixSortSuite.java new file mode 100644 index 000000000000..bb38305a0785 --- /dev/null +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterRadixSortSuite.java @@ -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.util.collection.unsafe.sort; + +public class UnsafeExternalSorterRadixSortSuite extends UnsafeExternalSorterSuite { + @Override + protected boolean shouldUseRadixSort() { return true; } +} diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java index cfead0e5924b..5330a688e63e 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java @@ -19,23 +19,17 @@ import java.io.File; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.Arrays; import java.util.LinkedList; import java.util.UUID; -import scala.Tuple2; import scala.Tuple2$; -import scala.runtime.AbstractFunction1; import org.junit.After; import org.junit.Before; import org.junit.Test; import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import org.apache.spark.SparkConf; import org.apache.spark.TaskContext; @@ -43,30 +37,32 @@ import org.apache.spark.executor.TaskMetrics; import org.apache.spark.memory.TestMemoryManager; import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.serializer.JavaSerializer; import org.apache.spark.serializer.SerializerInstance; +import org.apache.spark.serializer.SerializerManager; import org.apache.spark.storage.*; import org.apache.spark.unsafe.Platform; import org.apache.spark.util.Utils; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.junit.Assert.*; -import static org.mockito.AdditionalAnswers.returnsSecondArg; import static org.mockito.Answers.RETURNS_SMART_NULLS; import static org.mockito.Mockito.*; public class UnsafeExternalSorterSuite { - final LinkedList spillFilesCreated = new LinkedList(); + private final SparkConf conf = new SparkConf(); + + final LinkedList spillFilesCreated = new LinkedList<>(); final TestMemoryManager memoryManager = - new TestMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")); + new TestMemoryManager(conf.clone().set("spark.memory.offHeap.enabled", "false")); final TaskMemoryManager taskMemoryManager = new TaskMemoryManager(memoryManager, 0); + final SerializerManager serializerManager = new SerializerManager( + new JavaSerializer(conf), + conf.clone().set("spark.shuffle.spill.compress", "false")); // Use integer comparison for comparing prefixes (which are partition ids, in this case) - final PrefixComparator prefixComparator = new PrefixComparator() { - @Override - public int compare(long prefix1, long prefix2) { - return (int) prefix1 - (int) prefix2; - } - }; + final PrefixComparator prefixComparator = PrefixComparators.LONG; // Since the key fits within the 8-byte prefix, we don't need to do any record comparison, so // use a dummy comparator final RecordComparator recordComparator = new RecordComparator() { @@ -80,62 +76,47 @@ public int compare( } }; - SparkConf sparkConf; File tempDir; @Mock(answer = RETURNS_SMART_NULLS) BlockManager blockManager; @Mock(answer = RETURNS_SMART_NULLS) DiskBlockManager diskBlockManager; @Mock(answer = RETURNS_SMART_NULLS) TaskContext taskContext; + protected boolean shouldUseRadixSort() { return false; } - private final long pageSizeBytes = new SparkConf().getSizeAsBytes("spark.buffer.pageSize", "4m"); - - private static final class CompressStream extends AbstractFunction1 { - @Override - public OutputStream apply(OutputStream stream) { - return stream; - } - } + private final long pageSizeBytes = conf.getSizeAsBytes("spark.buffer.pageSize", "4m"); @Before public void setUp() { MockitoAnnotations.initMocks(this); - sparkConf = new SparkConf(); tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "unsafe-test"); spillFilesCreated.clear(); taskContext = mock(TaskContext.class); when(taskContext.taskMetrics()).thenReturn(new TaskMetrics()); when(blockManager.diskBlockManager()).thenReturn(diskBlockManager); - when(diskBlockManager.createTempLocalBlock()).thenAnswer(new Answer>() { - @Override - public Tuple2 answer(InvocationOnMock invocationOnMock) throws Throwable { - TempLocalBlockId blockId = new TempLocalBlockId(UUID.randomUUID()); - File file = File.createTempFile("spillFile", ".spill", tempDir); - spillFilesCreated.add(file); - return Tuple2$.MODULE$.apply(blockId, file); - } + when(diskBlockManager.createTempLocalBlock()).thenAnswer(invocationOnMock -> { + TempLocalBlockId blockId = new TempLocalBlockId(UUID.randomUUID()); + File file = File.createTempFile("spillFile", ".spill", tempDir); + spillFilesCreated.add(file); + return Tuple2$.MODULE$.apply(blockId, file); }); when(blockManager.getDiskWriter( any(BlockId.class), any(File.class), any(SerializerInstance.class), anyInt(), - any(ShuffleWriteMetrics.class))).thenAnswer(new Answer() { - @Override - public DiskBlockObjectWriter answer(InvocationOnMock invocationOnMock) throws Throwable { + any(ShuffleWriteMetrics.class))).thenAnswer(invocationOnMock -> { Object[] args = invocationOnMock.getArguments(); return new DiskBlockObjectWriter( (File) args[1], + serializerManager, (SerializerInstance) args[2], (Integer) args[3], - new CompressStream(), false, - (ShuffleWriteMetrics) args[4] + (ShuffleWriteMetrics) args[4], + (BlockId) args[0] ); - } - }); - when(blockManager.wrapForCompression(any(BlockId.class), any(InputStream.class))) - .then(returnsSecondArg()); + }); } @After @@ -157,25 +138,28 @@ private void assertSpillFilesWereCleanedUp() { private static void insertNumber(UnsafeExternalSorter sorter, int value) throws Exception { final int[] arr = new int[]{ value }; - sorter.insertRecord(arr, Platform.INT_ARRAY_OFFSET, 4, value); + sorter.insertRecord(arr, Platform.INT_ARRAY_OFFSET, 4, value, false); } private static void insertRecord( UnsafeExternalSorter sorter, int[] record, long prefix) throws IOException { - sorter.insertRecord(record, Platform.INT_ARRAY_OFFSET, record.length * 4, prefix); + sorter.insertRecord(record, Platform.INT_ARRAY_OFFSET, record.length * 4, prefix, false); } private UnsafeExternalSorter newSorter() throws IOException { return UnsafeExternalSorter.create( taskMemoryManager, blockManager, + serializerManager, taskContext, - recordComparator, + () -> recordComparator, prefixComparator, /* initialSize */ 1024, - pageSizeBytes); + pageSizeBytes, + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD, + shouldUseRadixSort()); } @Test @@ -205,13 +189,13 @@ public void testSortingOnlyByPrefix() throws Exception { @Test public void testSortingEmptyArrays() throws Exception { final UnsafeExternalSorter sorter = newSorter(); - sorter.insertRecord(null, 0, 0, 0); - sorter.insertRecord(null, 0, 0, 0); + sorter.insertRecord(null, 0, 0, 0, false); + sorter.insertRecord(null, 0, 0, 0, false); sorter.spill(); - sorter.insertRecord(null, 0, 0, 0); + sorter.insertRecord(null, 0, 0, 0, false); sorter.spill(); - sorter.insertRecord(null, 0, 0, 0); - sorter.insertRecord(null, 0, 0, 0); + sorter.insertRecord(null, 0, 0, 0, false); + sorter.insertRecord(null, 0, 0, 0, false); UnsafeSorterIterator iter = sorter.getSortedIterator(); @@ -225,6 +209,25 @@ public void testSortingEmptyArrays() throws Exception { assertSpillFilesWereCleanedUp(); } + @Test + public void testSortTimeMetric() throws Exception { + final UnsafeExternalSorter sorter = newSorter(); + long prevSortTime = sorter.getSortTimeNanos(); + assertEquals(prevSortTime, 0); + + sorter.insertRecord(null, 0, 0, 0, false); + sorter.spill(); + assertThat(sorter.getSortTimeNanos(), greaterThan(prevSortTime)); + prevSortTime = sorter.getSortTimeNanos(); + + sorter.spill(); // no sort needed + assertEquals(sorter.getSortTimeNanos(), prevSortTime); + + sorter.insertRecord(null, 0, 0, 0, false); + UnsafeSorterIterator iter = sorter.getSortedIterator(); + assertThat(sorter.getSortTimeNanos(), greaterThan(prevSortTime)); + } + @Test public void spillingOccursInResponseToMemoryPressure() throws Exception { final UnsafeExternalSorter sorter = newSorter(); @@ -260,7 +263,7 @@ public void testFillingPage() throws Exception { final UnsafeExternalSorter sorter = newSorter(); byte[] record = new byte[16]; while (sorter.getNumberOfAllocatedPages() < 2) { - sorter.insertRecord(record, Platform.BYTE_ARRAY_OFFSET, record.length, 0); + sorter.insertRecord(record, Platform.BYTE_ARRAY_OFFSET, record.length, 0, false); } sorter.cleanupResources(); assertSpillFilesWereCleanedUp(); @@ -320,25 +323,25 @@ public void forcedSpillingWithReadIterator() throws Exception { int n = (int) pageSizeBytes / recordSize * 3; for (int i = 0; i < n; i++) { record[0] = (long) i; - sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0); + sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0, false); } - assert(sorter.getNumberOfAllocatedPages() >= 2); + assertTrue(sorter.getNumberOfAllocatedPages() >= 2); UnsafeExternalSorter.SpillableIterator iter = (UnsafeExternalSorter.SpillableIterator) sorter.getSortedIterator(); int lastv = 0; for (int i = 0; i < n / 3; i++) { iter.hasNext(); iter.loadNext(); - assert(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == i); + assertTrue(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == i); lastv = i; } - assert(iter.spill() > 0); - assert(iter.spill() == 0); - assert(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == lastv); + assertTrue(iter.spill() > 0); + assertEquals(0, iter.spill()); + assertTrue(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == lastv); for (int i = n / 3; i < n; i++) { iter.hasNext(); iter.loadNext(); - assert(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == i); + assertEquals(i, Platform.getLong(iter.getBaseObject(), iter.getBaseOffset())); } sorter.cleanupResources(); assertSpillFilesWereCleanedUp(); @@ -352,18 +355,77 @@ public void forcedSpillingWithNotReadIterator() throws Exception { int n = (int) pageSizeBytes / recordSize * 3; for (int i = 0; i < n; i++) { record[0] = (long) i; - sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0); + sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0, false); } - assert(sorter.getNumberOfAllocatedPages() >= 2); + assertTrue(sorter.getNumberOfAllocatedPages() >= 2); UnsafeExternalSorter.SpillableIterator iter = (UnsafeExternalSorter.SpillableIterator) sorter.getSortedIterator(); - assert(iter.spill() > 0); - assert(iter.spill() == 0); + assertTrue(iter.spill() > 0); + assertEquals(0, iter.spill()); + for (int i = 0; i < n; i++) { + iter.hasNext(); + iter.loadNext(); + assertEquals(i, Platform.getLong(iter.getBaseObject(), iter.getBaseOffset())); + } + sorter.cleanupResources(); + assertSpillFilesWereCleanedUp(); + } + + @Test + public void forcedSpillingWithoutComparator() throws Exception { + final UnsafeExternalSorter sorter = UnsafeExternalSorter.create( + taskMemoryManager, + blockManager, + serializerManager, + taskContext, + null, + null, + /* initialSize */ 1024, + pageSizeBytes, + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD, + shouldUseRadixSort()); + long[] record = new long[100]; + int recordSize = record.length * 8; + int n = (int) pageSizeBytes / recordSize * 3; + int batch = n / 4; + for (int i = 0; i < n; i++) { + record[0] = (long) i; + sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0, false); + if (i % batch == batch - 1) { + sorter.spill(); + } + } + UnsafeSorterIterator iter = sorter.getIterator(0); for (int i = 0; i < n; i++) { iter.hasNext(); iter.loadNext(); - assert(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == i); + assertEquals(i, Platform.getLong(iter.getBaseObject(), iter.getBaseOffset())); + } + sorter.cleanupResources(); + assertSpillFilesWereCleanedUp(); + } + + @Test + public void testDiskSpilledBytes() throws Exception { + final UnsafeExternalSorter sorter = newSorter(); + long[] record = new long[100]; + int recordSize = record.length * 8; + int n = (int) pageSizeBytes / recordSize * 3; + for (int i = 0; i < n; i++) { + record[0] = (long) i; + sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0, false); } + // We will have at-least 2 memory pages allocated because of rounding happening due to + // integer division of pageSizeBytes and recordSize. + assertTrue(sorter.getNumberOfAllocatedPages() >= 2); + assertTrue(taskContext.taskMetrics().diskBytesSpilled() == 0); + UnsafeExternalSorter.SpillableIterator iter = + (UnsafeExternalSorter.SpillableIterator) sorter.getSortedIterator(); + assertTrue(iter.spill() > 0); + assertTrue(taskContext.taskMetrics().diskBytesSpilled() > 0); + assertEquals(0, iter.spill()); + // Even if we did not spill second time, the disk spilled bytes should still be non-zero + assertTrue(taskContext.taskMetrics().diskBytesSpilled() > 0); sorter.cleanupResources(); assertSpillFilesWereCleanedUp(); } @@ -376,11 +438,14 @@ public void testPeakMemoryUsed() throws Exception { final UnsafeExternalSorter sorter = UnsafeExternalSorter.create( taskMemoryManager, blockManager, + serializerManager, taskContext, - recordComparator, + () -> recordComparator, prefixComparator, 1024, - pageSizeBytes); + pageSizeBytes, + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD, + shouldUseRadixSort()); // Peak memory should be monotonically increasing. More specifically, every time // we allocate a new page it should increase by exactly the size of the page. @@ -390,7 +455,6 @@ public void testPeakMemoryUsed() throws Exception { for (int i = 0; i < numRecordsPerPage * 10; i++) { insertNumber(sorter, i); newPeakMemory = sorter.getPeakMemoryUsedBytes(); - // The first page is pre-allocated on instantiation if (i % numRecordsPerPage == 0) { // We allocated a new page for this record, so peak memory should change assertEquals(previousPeakMemory + pageSizeBytes, newPeakMemory); @@ -415,5 +479,37 @@ public void testPeakMemoryUsed() throws Exception { } } + @Test + public void testGetIterator() throws Exception { + final UnsafeExternalSorter sorter = newSorter(); + for (int i = 0; i < 100; i++) { + insertNumber(sorter, i); + } + verifyIntIterator(sorter.getIterator(0), 0, 100); + verifyIntIterator(sorter.getIterator(79), 79, 100); + + sorter.spill(); + for (int i = 100; i < 200; i++) { + insertNumber(sorter, i); + } + sorter.spill(); + verifyIntIterator(sorter.getIterator(79), 79, 200); + + for (int i = 200; i < 300; i++) { + insertNumber(sorter, i); + } + verifyIntIterator(sorter.getIterator(79), 79, 300); + verifyIntIterator(sorter.getIterator(139), 139, 300); + verifyIntIterator(sorter.getIterator(279), 279, 300); + } + + private void verifyIntIterator(UnsafeSorterIterator iter, int start, int end) + throws IOException { + for (int i = start; i < end; i++) { + assert (iter.hasNext()); + iter.loadNext(); + assert (Platform.getInt(iter.getBaseObject(), iter.getBaseOffset()) == i); + } + } } diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterRadixSortSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterRadixSortSuite.java new file mode 100644 index 000000000000..ae69ededf76f --- /dev/null +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterRadixSortSuite.java @@ -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.util.collection.unsafe.sort; + +public class UnsafeInMemorySorterRadixSortSuite extends UnsafeInMemorySorterSuite { + @Override + protected boolean shouldUseRadixSort() { return true; } +} diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java index 642f6585f8a1..bd89085aa9a1 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java @@ -17,12 +17,15 @@ package org.apache.spark.util.collection.unsafe.sort; +import java.nio.charset.StandardCharsets; import java.util.Arrays; +import org.junit.Assert; import org.junit.Test; import org.apache.spark.HashPartitioner; import org.apache.spark.SparkConf; +import org.apache.spark.memory.TestMemoryConsumer; import org.apache.spark.memory.TestMemoryManager; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.unsafe.Platform; @@ -36,22 +39,27 @@ public class UnsafeInMemorySorterSuite { + protected boolean shouldUseRadixSort() { return false; } + private static String getStringFromDataPage(Object baseObject, long baseOffset, int length) { final byte[] strBytes = new byte[length]; Platform.copyMemory(baseObject, baseOffset, strBytes, Platform.BYTE_ARRAY_OFFSET, length); - return new String(strBytes); + return new String(strBytes, StandardCharsets.UTF_8); } @Test public void testSortingEmptyInput() { - final UnsafeInMemorySorter sorter = new UnsafeInMemorySorter( - new TaskMemoryManager( - new TestMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")), 0), + final TaskMemoryManager memoryManager = new TaskMemoryManager( + new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); + final TestMemoryConsumer consumer = new TestMemoryConsumer(memoryManager); + final UnsafeInMemorySorter sorter = new UnsafeInMemorySorter(consumer, + memoryManager, mock(RecordComparator.class), mock(PrefixComparator.class), - 100); + 100, + shouldUseRadixSort()); final UnsafeSorterIterator iter = sorter.getSortedIterator(); - assert(!iter.hasNext()); + Assert.assertFalse(iter.hasNext()); } @Test @@ -68,13 +76,14 @@ public void testSortingOnlyByIntegerPrefix() throws Exception { "Mango" }; final TaskMemoryManager memoryManager = new TaskMemoryManager( - new TestMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")), 0); - final MemoryBlock dataPage = memoryManager.allocatePage(2048, null); + new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); + final TestMemoryConsumer consumer = new TestMemoryConsumer(memoryManager); + final MemoryBlock dataPage = memoryManager.allocatePage(2048, consumer); final Object baseObject = dataPage.getBaseObject(); // Write the records into the data page: long position = dataPage.getBaseOffset(); for (String str : dataToSort) { - final byte[] strBytes = str.getBytes("utf-8"); + final byte[] strBytes = str.getBytes(StandardCharsets.UTF_8); Platform.putInt(baseObject, position, strBytes.length); position += 4; Platform.copyMemory( @@ -96,23 +105,22 @@ public int compare( // Compute key prefixes based on the records' partition ids final HashPartitioner hashPartitioner = new HashPartitioner(4); // Use integer comparison for comparing prefixes (which are partition ids, in this case) - final PrefixComparator prefixComparator = new PrefixComparator() { - @Override - public int compare(long prefix1, long prefix2) { - return (int) prefix1 - (int) prefix2; - } - }; - UnsafeInMemorySorter sorter = new UnsafeInMemorySorter(memoryManager, recordComparator, - prefixComparator, dataToSort.length); + final PrefixComparator prefixComparator = PrefixComparators.LONG; + UnsafeInMemorySorter sorter = new UnsafeInMemorySorter(consumer, memoryManager, + recordComparator, prefixComparator, dataToSort.length, shouldUseRadixSort()); // Given a page of records, insert those records into the sorter one-by-one: position = dataPage.getBaseOffset(); for (int i = 0; i < dataToSort.length; i++) { + if (!sorter.hasSpaceForAnotherRecord()) { + sorter.expandPointerArray( + consumer.allocateArray(sorter.getMemoryUsage() / 8 * 2)); + } // position now points to the start of a record (which holds its length). final int recordLength = Platform.getInt(baseObject, position); final long address = memoryManager.encodePageNumberAndOffset(dataPage, position); final String str = getStringFromDataPage(baseObject, position + 4, recordLength); final int partitionId = hashPartitioner.getPartition(str); - sorter.insertRecord(address, partitionId); + sorter.insertRecord(address, partitionId, false); position += 4 + recordLength; } final UnsafeSorterIterator iter = sorter.getSortedIterator(); diff --git a/core/src/test/java/test/org/apache/spark/Java8RDDAPISuite.java b/core/src/test/java/test/org/apache/spark/Java8RDDAPISuite.java new file mode 100644 index 000000000000..1d2b05ebc250 --- /dev/null +++ b/core/src/test/java/test/org/apache/spark/Java8RDDAPISuite.java @@ -0,0 +1,351 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark; + +import java.io.File; +import java.io.Serializable; +import java.util.*; + +import scala.Tuple2; + +import com.google.common.collect.Iterables; +import com.google.common.io.Files; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +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.Optional; +import org.apache.spark.api.java.function.*; +import org.apache.spark.util.Utils; + +/** + * Most of these tests replicate org.apache.spark.JavaAPISuite using java 8 + * lambda syntax. + */ +public class Java8RDDAPISuite implements Serializable { + private static int foreachCalls = 0; + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaAPISuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void foreachWithAnonymousClass() { + foreachCalls = 0; + JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); + rdd.foreach(s -> foreachCalls++); + Assert.assertEquals(2, foreachCalls); + } + + @Test + public void foreach() { + foreachCalls = 0; + JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); + rdd.foreach(x -> foreachCalls++); + Assert.assertEquals(2, foreachCalls); + } + + @Test + public void groupBy() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); + Function isOdd = x -> x % 2 == 0; + JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); + Assert.assertEquals(2, oddsAndEvens.count()); + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds + + oddsAndEvens = rdd.groupBy(isOdd, 1); + Assert.assertEquals(2, oddsAndEvens.count()); + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds + } + + @Test + public void leftOuterJoin() { + JavaPairRDD rdd1 = sc.parallelizePairs(Arrays.asList( + new Tuple2<>(1, 1), + new Tuple2<>(1, 2), + new Tuple2<>(2, 1), + new Tuple2<>(3, 1) + )); + JavaPairRDD rdd2 = sc.parallelizePairs(Arrays.asList( + new Tuple2<>(1, 'x'), + new Tuple2<>(2, 'y'), + new Tuple2<>(2, 'z'), + new Tuple2<>(4, 'w') + )); + List>>> joined = + rdd1.leftOuterJoin(rdd2).collect(); + Assert.assertEquals(5, joined.size()); + Tuple2>> firstUnmatched = + rdd1.leftOuterJoin(rdd2).filter(tup -> !tup._2()._2().isPresent()).first(); + Assert.assertEquals(3, firstUnmatched._1().intValue()); + } + + @Test + public void foldReduce() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); + Function2 add = (a, b) -> a + b; + + int sum = rdd.fold(0, add); + Assert.assertEquals(33, sum); + + sum = rdd.reduce(add); + Assert.assertEquals(33, sum); + } + + @Test + public void foldByKey() { + List> pairs = Arrays.asList( + new Tuple2<>(2, 1), + new Tuple2<>(2, 1), + new Tuple2<>(1, 1), + new Tuple2<>(3, 2), + new Tuple2<>(3, 1) + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + JavaPairRDD sums = rdd.foldByKey(0, (a, b) -> a + b); + Assert.assertEquals(1, sums.lookup(1).get(0).intValue()); + Assert.assertEquals(2, sums.lookup(2).get(0).intValue()); + Assert.assertEquals(3, sums.lookup(3).get(0).intValue()); + } + + @Test + public void reduceByKey() { + List> pairs = Arrays.asList( + new Tuple2<>(2, 1), + new Tuple2<>(2, 1), + new Tuple2<>(1, 1), + new Tuple2<>(3, 2), + new Tuple2<>(3, 1) + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + JavaPairRDD counts = rdd.reduceByKey((a, b) -> a + b); + Assert.assertEquals(1, counts.lookup(1).get(0).intValue()); + Assert.assertEquals(2, counts.lookup(2).get(0).intValue()); + Assert.assertEquals(3, counts.lookup(3).get(0).intValue()); + + Map localCounts = counts.collectAsMap(); + Assert.assertEquals(1, localCounts.get(1).intValue()); + Assert.assertEquals(2, localCounts.get(2).intValue()); + Assert.assertEquals(3, localCounts.get(3).intValue()); + + localCounts = rdd.reduceByKeyLocally((a, b) -> a + b); + Assert.assertEquals(1, localCounts.get(1).intValue()); + Assert.assertEquals(2, localCounts.get(2).intValue()); + Assert.assertEquals(3, localCounts.get(3).intValue()); + } + + @Test + public void map() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); + JavaDoubleRDD doubles = rdd.mapToDouble(x -> 1.0 * x).cache(); + doubles.collect(); + JavaPairRDD pairs = rdd.mapToPair(x -> new Tuple2<>(x, x)) + .cache(); + pairs.collect(); + JavaRDD strings = rdd.map(Object::toString).cache(); + strings.collect(); + } + + @Test + public void flatMap() { + JavaRDD rdd = sc.parallelize(Arrays.asList("Hello World!", + "The quick brown fox jumps over the lazy dog.")); + JavaRDD words = rdd.flatMap(x -> Arrays.asList(x.split(" ")).iterator()); + + Assert.assertEquals("Hello", words.first()); + Assert.assertEquals(11, words.count()); + + JavaPairRDD pairs = rdd.flatMapToPair(s -> { + List> pairs2 = new LinkedList<>(); + for (String word : s.split(" ")) { + pairs2.add(new Tuple2<>(word, word)); + } + return pairs2.iterator(); + }); + + Assert.assertEquals(new Tuple2<>("Hello", "Hello"), pairs.first()); + Assert.assertEquals(11, pairs.count()); + + JavaDoubleRDD doubles = rdd.flatMapToDouble(s -> { + List lengths = new LinkedList<>(); + for (String word : s.split(" ")) { + lengths.add((double) word.length()); + } + return lengths.iterator(); + }); + + Assert.assertEquals(5.0, doubles.first(), 0.01); + Assert.assertEquals(11, pairs.count()); + } + + @Test + public void mapsFromPairsToPairs() { + List> pairs = Arrays.asList( + new Tuple2<>(1, "a"), + new Tuple2<>(2, "aa"), + new Tuple2<>(3, "aaa") + ); + JavaPairRDD pairRDD = sc.parallelizePairs(pairs); + + // Regression test for SPARK-668: + JavaPairRDD swapped = + pairRDD.flatMapToPair(x -> Collections.singletonList(x.swap()).iterator()); + swapped.collect(); + + // There was never a bug here, but it's worth testing: + pairRDD.map(Tuple2::swap).collect(); + } + + @Test + public void mapPartitions() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + JavaRDD partitionSums = rdd.mapPartitions(iter -> { + int sum = 0; + while (iter.hasNext()) { + sum += iter.next(); + } + return Collections.singletonList(sum).iterator(); + }); + + Assert.assertEquals("[3, 7]", partitionSums.collect().toString()); + } + + @Test + public void sequenceFile() { + File tempDir = Files.createTempDir(); + tempDir.deleteOnExit(); + String outputDir = new File(tempDir, "output").getAbsolutePath(); + List> pairs = Arrays.asList( + new Tuple2<>(1, "a"), + new Tuple2<>(2, "aa"), + new Tuple2<>(3, "aaa") + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + + rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()))) + .saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); + + // Try reading the output back as an object file + JavaPairRDD readRDD = sc.sequenceFile(outputDir, IntWritable.class, Text.class) + .mapToPair(pair -> new Tuple2<>(pair._1().get(), pair._2().toString())); + Assert.assertEquals(pairs, readRDD.collect()); + Utils.deleteRecursively(tempDir); + } + + @Test + public void zip() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); + JavaDoubleRDD doubles = rdd.mapToDouble(x -> 1.0 * x); + JavaPairRDD zipped = rdd.zip(doubles); + zipped.count(); + } + + @Test + public void zipPartitions() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6), 2); + JavaRDD rdd2 = sc.parallelize(Arrays.asList("1", "2", "3", "4"), 2); + FlatMapFunction2, Iterator, Integer> sizesFn = + (Iterator i, Iterator s) -> { + int sizeI = 0; + while (i.hasNext()) { + sizeI += 1; + i.next(); + } + int sizeS = 0; + while (s.hasNext()) { + sizeS += 1; + s.next(); + } + return Arrays.asList(sizeI, sizeS).iterator(); + }; + JavaRDD sizes = rdd1.zipPartitions(rdd2, sizesFn); + Assert.assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); + } + + @Test + public void keyBy() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2)); + List> s = rdd.keyBy(Object::toString).collect(); + Assert.assertEquals(new Tuple2<>("1", 1), s.get(0)); + Assert.assertEquals(new Tuple2<>("2", 2), s.get(1)); + } + + @Test + public void mapOnPairRDD() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + JavaPairRDD rdd2 = + rdd1.mapToPair(i -> new Tuple2<>(i, i % 2)); + JavaPairRDD rdd3 = + rdd2.mapToPair(in -> new Tuple2<>(in._2(), in._1())); + Assert.assertEquals(Arrays.asList( + new Tuple2<>(1, 1), + new Tuple2<>(0, 2), + new Tuple2<>(1, 3), + new Tuple2<>(0, 4)), rdd3.collect()); + } + + @Test + public void collectPartitions() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); + + JavaPairRDD rdd2 = + rdd1.mapToPair(i -> new Tuple2<>(i, i % 2)); + List[] parts = rdd1.collectPartitions(new int[]{0}); + Assert.assertEquals(Arrays.asList(1, 2), parts[0]); + + parts = rdd1.collectPartitions(new int[]{1, 2}); + Assert.assertEquals(Arrays.asList(3, 4), parts[0]); + Assert.assertEquals(Arrays.asList(5, 6, 7), parts[1]); + + Assert.assertEquals(Arrays.asList(new Tuple2<>(1, 1), new Tuple2<>(2, 0)), + rdd2.collectPartitions(new int[]{0})[0]); + + List>[] parts2 = rdd2.collectPartitions(new int[]{1, 2}); + Assert.assertEquals(Arrays.asList(new Tuple2<>(3, 1), new Tuple2<>(4, 0)), parts2[0]); + Assert.assertEquals(Arrays.asList(new Tuple2<>(5, 1), new Tuple2<>(6, 0), new Tuple2<>(7, 1)), + parts2[1]); + } + + @Test + public void collectAsMapWithIntArrayValues() { + // Regression test for SPARK-1040 + JavaRDD rdd = sc.parallelize(Arrays.asList(1)); + JavaPairRDD pairRDD = + rdd.mapToPair(x -> new Tuple2<>(x, new int[]{x})); + pairRDD.collect(); // Works fine + pairRDD.collectAsMap(); // Used to crash with ClassCastException + } +} diff --git a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java new file mode 100644 index 000000000000..01b5fb7b4668 --- /dev/null +++ b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java @@ -0,0 +1,1553 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark; + +import java.io.*; +import java.nio.channels.FileChannel; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.*; + +import org.apache.spark.Accumulator; +import org.apache.spark.AccumulatorParam; +import org.apache.spark.Partitioner; +import org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; +import scala.Tuple2; +import scala.Tuple3; +import scala.Tuple4; +import scala.collection.JavaConverters; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import com.google.common.base.Throwables; +import com.google.common.io.Files; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.mapred.SequenceFileInputFormat; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.apache.hadoop.mapreduce.Job; +import org.junit.After; +import static org.junit.Assert.*; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaFutureAction; +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.Optional; +import org.apache.spark.api.java.function.*; +import org.apache.spark.input.PortableDataStream; +import org.apache.spark.partial.BoundedDouble; +import org.apache.spark.partial.PartialResult; +import org.apache.spark.rdd.RDD; +import org.apache.spark.serializer.KryoSerializer; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.util.LongAccumulator; +import org.apache.spark.util.StatCounter; + +// The test suite itself is Serializable so that anonymous Function implementations can be +// serialized, as an alternative to converting these anonymous classes to static inner classes; +// see http://stackoverflow.com/questions/758570/. +public class JavaAPISuite implements Serializable { + private transient JavaSparkContext sc; + private transient File tempDir; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaAPISuite"); + tempDir = Files.createTempDir(); + tempDir.deleteOnExit(); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @SuppressWarnings("unchecked") + @Test + public void sparkContextUnion() { + // Union of non-specialized JavaRDDs + List strings = Arrays.asList("Hello", "World"); + JavaRDD s1 = sc.parallelize(strings); + JavaRDD s2 = sc.parallelize(strings); + // Varargs + JavaRDD sUnion = sc.union(s1, s2); + assertEquals(4, sUnion.count()); + // List + List> list = new ArrayList<>(); + list.add(s2); + sUnion = sc.union(s1, list); + assertEquals(4, sUnion.count()); + + // Union of JavaDoubleRDDs + List doubles = Arrays.asList(1.0, 2.0); + JavaDoubleRDD d1 = sc.parallelizeDoubles(doubles); + JavaDoubleRDD d2 = sc.parallelizeDoubles(doubles); + JavaDoubleRDD dUnion = sc.union(d1, d2); + assertEquals(4, dUnion.count()); + + // Union of JavaPairRDDs + List> pairs = new ArrayList<>(); + pairs.add(new Tuple2<>(1, 2)); + pairs.add(new Tuple2<>(3, 4)); + JavaPairRDD p1 = sc.parallelizePairs(pairs); + JavaPairRDD p2 = sc.parallelizePairs(pairs); + JavaPairRDD pUnion = sc.union(p1, p2); + assertEquals(4, pUnion.count()); + } + + @SuppressWarnings("unchecked") + @Test + public void intersection() { + List ints1 = Arrays.asList(1, 10, 2, 3, 4, 5); + List ints2 = Arrays.asList(1, 6, 2, 3, 7, 8); + JavaRDD s1 = sc.parallelize(ints1); + JavaRDD s2 = sc.parallelize(ints2); + + JavaRDD intersections = s1.intersection(s2); + assertEquals(3, intersections.count()); + + JavaRDD empty = sc.emptyRDD(); + JavaRDD emptyIntersection = empty.intersection(s2); + assertEquals(0, emptyIntersection.count()); + + List doubles = Arrays.asList(1.0, 2.0); + JavaDoubleRDD d1 = sc.parallelizeDoubles(doubles); + JavaDoubleRDD d2 = sc.parallelizeDoubles(doubles); + JavaDoubleRDD dIntersection = d1.intersection(d2); + assertEquals(2, dIntersection.count()); + + List> pairs = new ArrayList<>(); + pairs.add(new Tuple2<>(1, 2)); + pairs.add(new Tuple2<>(3, 4)); + JavaPairRDD p1 = sc.parallelizePairs(pairs); + JavaPairRDD p2 = sc.parallelizePairs(pairs); + JavaPairRDD pIntersection = p1.intersection(p2); + assertEquals(2, pIntersection.count()); + } + + @Test + public void sample() { + List ints = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + JavaRDD rdd = sc.parallelize(ints); + // the seeds here are "magic" to make this work out nicely + JavaRDD sample20 = rdd.sample(true, 0.2, 8); + assertEquals(2, sample20.count()); + JavaRDD sample20WithoutReplacement = rdd.sample(false, 0.2, 2); + assertEquals(2, sample20WithoutReplacement.count()); + } + + @Test + public void randomSplit() { + List ints = new ArrayList<>(1000); + for (int i = 0; i < 1000; i++) { + ints.add(i); + } + JavaRDD rdd = sc.parallelize(ints); + JavaRDD[] splits = rdd.randomSplit(new double[] { 0.4, 0.6, 1.0 }, 31); + // the splits aren't perfect -- not enough data for them to be -- just check they're about right + assertEquals(3, splits.length); + long s0 = splits[0].count(); + long s1 = splits[1].count(); + long s2 = splits[2].count(); + assertTrue(s0 + " not within expected range", s0 > 150 && s0 < 250); + assertTrue(s1 + " not within expected range", s1 > 250 && s0 < 350); + assertTrue(s2 + " not within expected range", s2 > 430 && s2 < 570); + } + + @Test + public void sortByKey() { + List> pairs = new ArrayList<>(); + pairs.add(new Tuple2<>(0, 4)); + pairs.add(new Tuple2<>(3, 2)); + pairs.add(new Tuple2<>(-1, 1)); + + JavaPairRDD rdd = sc.parallelizePairs(pairs); + + // Default comparator + JavaPairRDD sortedRDD = rdd.sortByKey(); + assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); + List> sortedPairs = sortedRDD.collect(); + assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); + assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); + + // Custom comparator + sortedRDD = rdd.sortByKey(Collections.reverseOrder(), false); + assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); + sortedPairs = sortedRDD.collect(); + assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); + assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); + } + + @SuppressWarnings("unchecked") + @Test + public void repartitionAndSortWithinPartitions() { + List> pairs = new ArrayList<>(); + pairs.add(new Tuple2<>(0, 5)); + pairs.add(new Tuple2<>(3, 8)); + pairs.add(new Tuple2<>(2, 6)); + pairs.add(new Tuple2<>(0, 8)); + pairs.add(new Tuple2<>(3, 8)); + pairs.add(new Tuple2<>(1, 3)); + + JavaPairRDD rdd = sc.parallelizePairs(pairs); + + Partitioner partitioner = new Partitioner() { + @Override + public int numPartitions() { + return 2; + } + @Override + public int getPartition(Object key) { + return (Integer) key % 2; + } + }; + + JavaPairRDD repartitioned = + rdd.repartitionAndSortWithinPartitions(partitioner); + assertTrue(repartitioned.partitioner().isPresent()); + assertEquals(repartitioned.partitioner().get(), partitioner); + List>> partitions = repartitioned.glom().collect(); + assertEquals(partitions.get(0), + Arrays.asList(new Tuple2<>(0, 5), new Tuple2<>(0, 8), new Tuple2<>(2, 6))); + assertEquals(partitions.get(1), + Arrays.asList(new Tuple2<>(1, 3), new Tuple2<>(3, 8), new Tuple2<>(3, 8))); + } + + @Test + public void emptyRDD() { + JavaRDD rdd = sc.emptyRDD(); + assertEquals("Empty RDD shouldn't have any values", 0, rdd.count()); + } + + @Test + public void sortBy() { + List> pairs = new ArrayList<>(); + pairs.add(new Tuple2<>(0, 4)); + pairs.add(new Tuple2<>(3, 2)); + pairs.add(new Tuple2<>(-1, 1)); + + JavaRDD> rdd = sc.parallelize(pairs); + + // compare on first value + JavaRDD> sortedRDD = rdd.sortBy(Tuple2::_1, true, 2); + + assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); + List> sortedPairs = sortedRDD.collect(); + assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); + assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); + + // compare on second value + sortedRDD = rdd.sortBy(Tuple2::_2, true, 2); + assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); + sortedPairs = sortedRDD.collect(); + assertEquals(new Tuple2<>(3, 2), sortedPairs.get(1)); + assertEquals(new Tuple2<>(0, 4), sortedPairs.get(2)); + } + + @Test + public void foreach() { + LongAccumulator accum = sc.sc().longAccumulator(); + JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); + rdd.foreach(s -> accum.add(1)); + assertEquals(2, accum.value().intValue()); + } + + @Test + public void foreachPartition() { + LongAccumulator accum = sc.sc().longAccumulator(); + JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); + rdd.foreachPartition(iter -> { + while (iter.hasNext()) { + iter.next(); + accum.add(1); + } + }); + assertEquals(2, accum.value().intValue()); + } + + @Test + public void toLocalIterator() { + List correct = Arrays.asList(1, 2, 3, 4); + JavaRDD rdd = sc.parallelize(correct); + List result = Lists.newArrayList(rdd.toLocalIterator()); + assertEquals(correct, result); + } + + @Test + public void zipWithUniqueId() { + List dataArray = Arrays.asList(1, 2, 3, 4); + JavaPairRDD zip = sc.parallelize(dataArray).zipWithUniqueId(); + JavaRDD indexes = zip.values(); + assertEquals(4, new HashSet<>(indexes.collect()).size()); + } + + @Test + public void zipWithIndex() { + List dataArray = Arrays.asList(1, 2, 3, 4); + JavaPairRDD zip = sc.parallelize(dataArray).zipWithIndex(); + JavaRDD indexes = zip.values(); + List correctIndexes = Arrays.asList(0L, 1L, 2L, 3L); + assertEquals(correctIndexes, indexes.collect()); + } + + @SuppressWarnings("unchecked") + @Test + public void lookup() { + JavaPairRDD categories = sc.parallelizePairs(Arrays.asList( + new Tuple2<>("Apples", "Fruit"), + new Tuple2<>("Oranges", "Fruit"), + new Tuple2<>("Oranges", "Citrus") + )); + assertEquals(2, categories.lookup("Oranges").size()); + assertEquals(2, Iterables.size(categories.groupByKey().lookup("Oranges").get(0))); + } + + @Test + public void groupBy() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); + Function isOdd = x -> x % 2 == 0; + JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); + assertEquals(2, oddsAndEvens.count()); + assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds + + oddsAndEvens = rdd.groupBy(isOdd, 1); + assertEquals(2, oddsAndEvens.count()); + assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds + } + + @Test + public void groupByOnPairRDD() { + // Regression test for SPARK-4459 + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); + Function, Boolean> areOdd = + x -> (x._1() % 2 == 0) && (x._2() % 2 == 0); + JavaPairRDD pairRDD = rdd.zip(rdd); + JavaPairRDD>> oddsAndEvens = pairRDD.groupBy(areOdd); + assertEquals(2, oddsAndEvens.count()); + assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds + + oddsAndEvens = pairRDD.groupBy(areOdd, 1); + assertEquals(2, oddsAndEvens.count()); + assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds + } + + @SuppressWarnings("unchecked") + @Test + public void keyByOnPairRDD() { + // Regression test for SPARK-4459 + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); + Function, String> sumToString = x -> String.valueOf(x._1() + x._2()); + JavaPairRDD pairRDD = rdd.zip(rdd); + JavaPairRDD> keyed = pairRDD.keyBy(sumToString); + assertEquals(7, keyed.count()); + assertEquals(1, (long) keyed.lookup("2").get(0)._1()); + } + + @SuppressWarnings("unchecked") + @Test + public void cogroup() { + JavaPairRDD categories = sc.parallelizePairs(Arrays.asList( + new Tuple2<>("Apples", "Fruit"), + new Tuple2<>("Oranges", "Fruit"), + new Tuple2<>("Oranges", "Citrus") + )); + JavaPairRDD prices = sc.parallelizePairs(Arrays.asList( + new Tuple2<>("Oranges", 2), + new Tuple2<>("Apples", 3) + )); + JavaPairRDD, Iterable>> cogrouped = + categories.cogroup(prices); + assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); + + cogrouped.collect(); + } + + @SuppressWarnings("unchecked") + @Test + public void cogroup3() { + JavaPairRDD categories = sc.parallelizePairs(Arrays.asList( + new Tuple2<>("Apples", "Fruit"), + new Tuple2<>("Oranges", "Fruit"), + new Tuple2<>("Oranges", "Citrus") + )); + JavaPairRDD prices = sc.parallelizePairs(Arrays.asList( + new Tuple2<>("Oranges", 2), + new Tuple2<>("Apples", 3) + )); + JavaPairRDD quantities = sc.parallelizePairs(Arrays.asList( + new Tuple2<>("Oranges", 21), + new Tuple2<>("Apples", 42) + )); + + JavaPairRDD, Iterable, Iterable>> cogrouped = + categories.cogroup(prices, quantities); + assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); + assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3())); + + + cogrouped.collect(); + } + + @SuppressWarnings("unchecked") + @Test + public void cogroup4() { + JavaPairRDD categories = sc.parallelizePairs(Arrays.asList( + new Tuple2<>("Apples", "Fruit"), + new Tuple2<>("Oranges", "Fruit"), + new Tuple2<>("Oranges", "Citrus") + )); + JavaPairRDD prices = sc.parallelizePairs(Arrays.asList( + new Tuple2<>("Oranges", 2), + new Tuple2<>("Apples", 3) + )); + JavaPairRDD quantities = sc.parallelizePairs(Arrays.asList( + new Tuple2<>("Oranges", 21), + new Tuple2<>("Apples", 42) + )); + JavaPairRDD countries = sc.parallelizePairs(Arrays.asList( + new Tuple2<>("Oranges", "BR"), + new Tuple2<>("Apples", "US") + )); + + JavaPairRDD, Iterable, Iterable, + Iterable>> cogrouped = categories.cogroup(prices, quantities, countries); + assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); + assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3())); + assertEquals("[BR]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._4())); + + cogrouped.collect(); + } + + @SuppressWarnings("unchecked") + @Test + public void leftOuterJoin() { + JavaPairRDD rdd1 = sc.parallelizePairs(Arrays.asList( + new Tuple2<>(1, 1), + new Tuple2<>(1, 2), + new Tuple2<>(2, 1), + new Tuple2<>(3, 1) + )); + JavaPairRDD rdd2 = sc.parallelizePairs(Arrays.asList( + new Tuple2<>(1, 'x'), + new Tuple2<>(2, 'y'), + new Tuple2<>(2, 'z'), + new Tuple2<>(4, 'w') + )); + List>>> joined = + rdd1.leftOuterJoin(rdd2).collect(); + assertEquals(5, joined.size()); + Tuple2>> firstUnmatched = + rdd1.leftOuterJoin(rdd2).filter(tup -> !tup._2()._2().isPresent()).first(); + assertEquals(3, firstUnmatched._1().intValue()); + } + + @Test + public void foldReduce() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); + Function2 add = (a, b) -> a + b; + + int sum = rdd.fold(0, add); + assertEquals(33, sum); + + sum = rdd.reduce(add); + assertEquals(33, sum); + } + + @Test + public void treeReduce() { + JavaRDD rdd = sc.parallelize(Arrays.asList(-5, -4, -3, -2, -1, 1, 2, 3, 4), 10); + Function2 add = (a, b) -> a + b; + for (int depth = 1; depth <= 10; depth++) { + int sum = rdd.treeReduce(add, depth); + assertEquals(-5, sum); + } + } + + @Test + public void treeAggregate() { + JavaRDD rdd = sc.parallelize(Arrays.asList(-5, -4, -3, -2, -1, 1, 2, 3, 4), 10); + Function2 add = (a, b) -> a + b; + for (int depth = 1; depth <= 10; depth++) { + int sum = rdd.treeAggregate(0, add, add, depth); + assertEquals(-5, sum); + } + } + + @SuppressWarnings("unchecked") + @Test + public void aggregateByKey() { + JavaPairRDD pairs = sc.parallelizePairs( + Arrays.asList( + new Tuple2<>(1, 1), + new Tuple2<>(1, 1), + new Tuple2<>(3, 2), + new Tuple2<>(5, 1), + new Tuple2<>(5, 3)), 2); + + Map> sets = pairs.aggregateByKey(new HashSet(), + (a, b) -> { + a.add(b); + return a; + }, + (a, b) -> { + a.addAll(b); + return a; + }).collectAsMap(); + assertEquals(3, sets.size()); + assertEquals(new HashSet<>(Arrays.asList(1)), sets.get(1)); + assertEquals(new HashSet<>(Arrays.asList(2)), sets.get(3)); + assertEquals(new HashSet<>(Arrays.asList(1, 3)), sets.get(5)); + } + + @SuppressWarnings("unchecked") + @Test + public void foldByKey() { + List> pairs = Arrays.asList( + new Tuple2<>(2, 1), + new Tuple2<>(2, 1), + new Tuple2<>(1, 1), + new Tuple2<>(3, 2), + new Tuple2<>(3, 1) + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + JavaPairRDD sums = rdd.foldByKey(0, (a, b) -> a + b); + assertEquals(1, sums.lookup(1).get(0).intValue()); + assertEquals(2, sums.lookup(2).get(0).intValue()); + assertEquals(3, sums.lookup(3).get(0).intValue()); + } + + @SuppressWarnings("unchecked") + @Test + public void reduceByKey() { + List> pairs = Arrays.asList( + new Tuple2<>(2, 1), + new Tuple2<>(2, 1), + new Tuple2<>(1, 1), + new Tuple2<>(3, 2), + new Tuple2<>(3, 1) + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + JavaPairRDD counts = rdd.reduceByKey((a, b) -> a + b); + assertEquals(1, counts.lookup(1).get(0).intValue()); + assertEquals(2, counts.lookup(2).get(0).intValue()); + assertEquals(3, counts.lookup(3).get(0).intValue()); + + Map localCounts = counts.collectAsMap(); + assertEquals(1, localCounts.get(1).intValue()); + assertEquals(2, localCounts.get(2).intValue()); + assertEquals(3, localCounts.get(3).intValue()); + + localCounts = rdd.reduceByKeyLocally((a, b) -> a + b); + assertEquals(1, localCounts.get(1).intValue()); + assertEquals(2, localCounts.get(2).intValue()); + assertEquals(3, localCounts.get(3).intValue()); + } + + @Test + public void approximateResults() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); + Map countsByValue = rdd.countByValue(); + assertEquals(2, countsByValue.get(1).longValue()); + assertEquals(1, countsByValue.get(13).longValue()); + + PartialResult> approx = rdd.countByValueApprox(1); + Map finalValue = approx.getFinalValue(); + assertEquals(2.0, finalValue.get(1).mean(), 0.01); + assertEquals(1.0, finalValue.get(13).mean(), 0.01); + } + + @Test + public void take() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); + assertEquals(1, rdd.first().intValue()); + rdd.take(2); + rdd.takeSample(false, 2, 42); + } + + @Test + public void isEmpty() { + assertTrue(sc.emptyRDD().isEmpty()); + assertTrue(sc.parallelize(new ArrayList()).isEmpty()); + assertFalse(sc.parallelize(Arrays.asList(1)).isEmpty()); + assertTrue(sc.parallelize(Arrays.asList(1, 2, 3), 3).filter(i -> i < 0).isEmpty()); + assertFalse(sc.parallelize(Arrays.asList(1, 2, 3)).filter(i -> i > 1).isEmpty()); + } + + @Test + public void cartesian() { + JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); + JavaRDD stringRDD = sc.parallelize(Arrays.asList("Hello", "World")); + JavaPairRDD cartesian = stringRDD.cartesian(doubleRDD); + assertEquals(new Tuple2<>("Hello", 1.0), cartesian.first()); + } + + @Test + public void javaDoubleRDD() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); + JavaDoubleRDD distinct = rdd.distinct(); + assertEquals(5, distinct.count()); + JavaDoubleRDD filter = rdd.filter(x -> x > 2.0); + assertEquals(3, filter.count()); + JavaDoubleRDD union = rdd.union(rdd); + assertEquals(12, union.count()); + union = union.cache(); + assertEquals(12, union.count()); + + assertEquals(20, rdd.sum(), 0.01); + StatCounter stats = rdd.stats(); + assertEquals(20, stats.sum(), 0.01); + assertEquals(20/6.0, rdd.mean(), 0.01); + assertEquals(20/6.0, rdd.mean(), 0.01); + assertEquals(6.22222, rdd.variance(), 0.01); + assertEquals(rdd.variance(), rdd.popVariance(), 1e-14); + assertEquals(7.46667, rdd.sampleVariance(), 0.01); + assertEquals(2.49444, rdd.stdev(), 0.01); + assertEquals(rdd.stdev(), rdd.popStdev(), 1e-14); + assertEquals(2.73252, rdd.sampleStdev(), 0.01); + + rdd.first(); + rdd.take(5); + } + + @Test + public void javaDoubleRDDHistoGram() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + // Test using generated buckets + Tuple2 results = rdd.histogram(2); + double[] expected_buckets = {1.0, 2.5, 4.0}; + long[] expected_counts = {2, 2}; + assertArrayEquals(expected_buckets, results._1(), 0.1); + assertArrayEquals(expected_counts, results._2()); + // Test with provided buckets + long[] histogram = rdd.histogram(expected_buckets); + assertArrayEquals(expected_counts, histogram); + // SPARK-5744 + assertArrayEquals( + new long[] {0}, + sc.parallelizeDoubles(new ArrayList<>(0), 1).histogram(new double[]{0.0, 1.0})); + } + + private static class DoubleComparator implements Comparator, Serializable { + @Override + public int compare(Double o1, Double o2) { + return o1.compareTo(o2); + } + } + + @Test + public void max() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.max(new DoubleComparator()); + assertEquals(4.0, max, 0.001); + } + + @Test + public void min() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.min(new DoubleComparator()); + assertEquals(1.0, max, 0.001); + } + + @Test + public void naturalMax() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.max(); + assertEquals(4.0, max, 0.0); + } + + @Test + public void naturalMin() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.min(); + assertEquals(1.0, max, 0.0); + } + + @Test + public void takeOrdered() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2, new DoubleComparator())); + assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2)); + } + + @Test + public void top() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + List top2 = rdd.top(2); + assertEquals(Arrays.asList(4, 3), top2); + } + + private static class AddInts implements Function2 { + @Override + public Integer call(Integer a, Integer b) { + return a + b; + } + } + + @Test + public void reduce() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + int sum = rdd.reduce(new AddInts()); + assertEquals(10, sum); + } + + @Test + public void reduceOnJavaDoubleRDD() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double sum = rdd.reduce((v1, v2) -> v1 + v2); + assertEquals(10.0, sum, 0.001); + } + + @Test + public void fold() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + int sum = rdd.fold(0, new AddInts()); + assertEquals(10, sum); + } + + @Test + public void aggregate() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + int sum = rdd.aggregate(0, new AddInts(), new AddInts()); + assertEquals(10, sum); + } + + @Test + public void map() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); + JavaDoubleRDD doubles = rdd.mapToDouble(Integer::doubleValue).cache(); + doubles.collect(); + JavaPairRDD pairs = rdd.mapToPair(x -> new Tuple2<>(x, x)).cache(); + pairs.collect(); + JavaRDD strings = rdd.map(Object::toString).cache(); + strings.collect(); + } + + @Test + public void flatMap() { + JavaRDD rdd = sc.parallelize(Arrays.asList("Hello World!", + "The quick brown fox jumps over the lazy dog.")); + JavaRDD words = rdd.flatMap(x -> Arrays.asList(x.split(" ")).iterator()); + assertEquals("Hello", words.first()); + assertEquals(11, words.count()); + + JavaPairRDD pairsRDD = rdd.flatMapToPair(s -> { + List> pairs = new LinkedList<>(); + for (String word : s.split(" ")) { + pairs.add(new Tuple2<>(word, word)); + } + return pairs.iterator(); + } + ); + assertEquals(new Tuple2<>("Hello", "Hello"), pairsRDD.first()); + assertEquals(11, pairsRDD.count()); + + JavaDoubleRDD doubles = rdd.flatMapToDouble(s -> { + List lengths = new LinkedList<>(); + for (String word : s.split(" ")) { + lengths.add((double) word.length()); + } + return lengths.iterator(); + }); + assertEquals(5.0, doubles.first(), 0.01); + assertEquals(11, pairsRDD.count()); + } + + @SuppressWarnings("unchecked") + @Test + public void mapsFromPairsToPairs() { + List> pairs = Arrays.asList( + new Tuple2<>(1, "a"), + new Tuple2<>(2, "aa"), + new Tuple2<>(3, "aaa") + ); + JavaPairRDD pairRDD = sc.parallelizePairs(pairs); + + // Regression test for SPARK-668: + JavaPairRDD swapped = pairRDD.flatMapToPair( + item -> Collections.singletonList(item.swap()).iterator()); + swapped.collect(); + + // There was never a bug here, but it's worth testing: + pairRDD.mapToPair(Tuple2::swap).collect(); + } + + @Test + public void mapPartitions() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + JavaRDD partitionSums = rdd.mapPartitions(iter -> { + int sum = 0; + while (iter.hasNext()) { + sum += iter.next(); + } + return Collections.singletonList(sum).iterator(); + }); + assertEquals("[3, 7]", partitionSums.collect().toString()); + } + + + @Test + public void mapPartitionsWithIndex() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + JavaRDD partitionSums = rdd.mapPartitionsWithIndex((index, iter) -> { + int sum = 0; + while (iter.hasNext()) { + sum += iter.next(); + } + return Collections.singletonList(sum).iterator(); + }, false); + assertEquals("[3, 7]", partitionSums.collect().toString()); + } + + @Test + public void getNumPartitions(){ + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3); + JavaDoubleRDD rdd2 = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0), 2); + JavaPairRDD rdd3 = sc.parallelizePairs( + Arrays.asList( + new Tuple2<>("a", 1), + new Tuple2<>("aa", 2), + new Tuple2<>("aaa", 3) + ), + 2); + assertEquals(3, rdd1.getNumPartitions()); + assertEquals(2, rdd2.getNumPartitions()); + assertEquals(2, rdd3.getNumPartitions()); + } + + @Test + public void repartition() { + // Shrinking number of partitions + JavaRDD in1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 2); + JavaRDD repartitioned1 = in1.repartition(4); + List> result1 = repartitioned1.glom().collect(); + assertEquals(4, result1.size()); + for (List l : result1) { + assertFalse(l.isEmpty()); + } + + // Growing number of partitions + JavaRDD in2 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 4); + JavaRDD repartitioned2 = in2.repartition(2); + List> result2 = repartitioned2.glom().collect(); + assertEquals(2, result2.size()); + for (List l: result2) { + assertFalse(l.isEmpty()); + } + } + + @SuppressWarnings("unchecked") + @Test + public void persist() { + JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); + doubleRDD = doubleRDD.persist(StorageLevel.DISK_ONLY()); + assertEquals(20, doubleRDD.sum(), 0.1); + + List> pairs = Arrays.asList( + new Tuple2<>(1, "a"), + new Tuple2<>(2, "aa"), + new Tuple2<>(3, "aaa") + ); + JavaPairRDD pairRDD = sc.parallelizePairs(pairs); + pairRDD = pairRDD.persist(StorageLevel.DISK_ONLY()); + assertEquals("a", pairRDD.first()._2()); + + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); + rdd = rdd.persist(StorageLevel.DISK_ONLY()); + assertEquals(1, rdd.first().intValue()); + } + + @Test + public void iterator() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); + TaskContext context = TaskContext$.MODULE$.empty(); + assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); + } + + @Test + public void glom() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + assertEquals("[1, 2]", rdd.glom().first().toString()); + } + + // File input / output tests are largely adapted from FileSuite: + + @Test + public void textFiles() throws IOException { + String outputDir = new File(tempDir, "output").getAbsolutePath(); + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + rdd.saveAsTextFile(outputDir); + // Read the plain text file and check it's OK + File outputFile = new File(outputDir, "part-00000"); + String content = Files.toString(outputFile, StandardCharsets.UTF_8); + assertEquals("1\n2\n3\n4\n", content); + // Also try reading it in as a text file RDD + List expected = Arrays.asList("1", "2", "3", "4"); + JavaRDD readRDD = sc.textFile(outputDir); + assertEquals(expected, readRDD.collect()); + } + + @Test + public void wholeTextFiles() throws Exception { + byte[] content1 = "spark is easy to use.\n".getBytes(StandardCharsets.UTF_8); + byte[] content2 = "spark is also easy to use.\n".getBytes(StandardCharsets.UTF_8); + + String tempDirName = tempDir.getAbsolutePath(); + String path1 = new Path(tempDirName, "part-00000").toUri().getPath(); + String path2 = new Path(tempDirName, "part-00001").toUri().getPath(); + + Files.write(content1, new File(path1)); + Files.write(content2, new File(path2)); + + Map container = new HashMap<>(); + container.put(path1, new Text(content1).toString()); + container.put(path2, new Text(content2).toString()); + + JavaPairRDD readRDD = sc.wholeTextFiles(tempDirName, 3); + List> result = readRDD.collect(); + + for (Tuple2 res : result) { + // Note that the paths from `wholeTextFiles` are in URI format on Windows, + // for example, file:/C:/a/b/c. + assertEquals(res._2(), container.get(new Path(res._1()).toUri().getPath())); + } + } + + @Test + public void textFilesCompressed() throws IOException { + String outputDir = new File(tempDir, "output").getAbsolutePath(); + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + rdd.saveAsTextFile(outputDir, DefaultCodec.class); + + // Try reading it in as a text file RDD + List expected = Arrays.asList("1", "2", "3", "4"); + JavaRDD readRDD = sc.textFile(outputDir); + assertEquals(expected, readRDD.collect()); + } + + @SuppressWarnings("unchecked") + @Test + public void sequenceFile() { + String outputDir = new File(tempDir, "output").getAbsolutePath(); + List> pairs = Arrays.asList( + new Tuple2<>(1, "a"), + new Tuple2<>(2, "aa"), + new Tuple2<>(3, "aaa") + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + + rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()))) + .saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); + + // Try reading the output back as an object file + JavaPairRDD readRDD = sc.sequenceFile(outputDir, IntWritable.class, + Text.class).mapToPair(pair -> new Tuple2<>(pair._1().get(), pair._2().toString())); + assertEquals(pairs, readRDD.collect()); + } + + @Test + public void binaryFiles() throws Exception { + // Reusing the wholeText files example + byte[] content1 = "spark is easy to use.\n".getBytes(StandardCharsets.UTF_8); + + String tempDirName = tempDir.getAbsolutePath(); + File file1 = new File(tempDirName + "/part-00000"); + + FileOutputStream fos1 = new FileOutputStream(file1); + + FileChannel channel1 = fos1.getChannel(); + ByteBuffer bbuf = ByteBuffer.wrap(content1); + channel1.write(bbuf); + channel1.close(); + JavaPairRDD readRDD = sc.binaryFiles(tempDirName, 3); + List> result = readRDD.collect(); + for (Tuple2 res : result) { + assertArrayEquals(content1, res._2().toArray()); + } + } + + @Test + public void binaryFilesCaching() throws Exception { + // Reusing the wholeText files example + byte[] content1 = "spark is easy to use.\n".getBytes(StandardCharsets.UTF_8); + + String tempDirName = tempDir.getAbsolutePath(); + File file1 = new File(tempDirName + "/part-00000"); + + FileOutputStream fos1 = new FileOutputStream(file1); + + FileChannel channel1 = fos1.getChannel(); + ByteBuffer bbuf = ByteBuffer.wrap(content1); + channel1.write(bbuf); + channel1.close(); + + JavaPairRDD readRDD = sc.binaryFiles(tempDirName).cache(); + readRDD.foreach(pair -> pair._2().toArray()); // force the file to read + + List> result = readRDD.collect(); + for (Tuple2 res : result) { + assertArrayEquals(content1, res._2().toArray()); + } + } + + @Test + public void binaryRecords() throws Exception { + // Reusing the wholeText files example + byte[] content1 = "spark isn't always easy to use.\n".getBytes(StandardCharsets.UTF_8); + int numOfCopies = 10; + String tempDirName = tempDir.getAbsolutePath(); + File file1 = new File(tempDirName + "/part-00000"); + + FileOutputStream fos1 = new FileOutputStream(file1); + + FileChannel channel1 = fos1.getChannel(); + + for (int i = 0; i < numOfCopies; i++) { + ByteBuffer bbuf = ByteBuffer.wrap(content1); + channel1.write(bbuf); + } + channel1.close(); + + JavaRDD readRDD = sc.binaryRecords(tempDirName, content1.length); + assertEquals(numOfCopies,readRDD.count()); + List result = readRDD.collect(); + for (byte[] res : result) { + assertArrayEquals(content1, res); + } + } + + @SuppressWarnings("unchecked") + @Test + public void writeWithNewAPIHadoopFile() { + String outputDir = new File(tempDir, "output").getAbsolutePath(); + List> pairs = Arrays.asList( + new Tuple2<>(1, "a"), + new Tuple2<>(2, "aa"), + new Tuple2<>(3, "aaa") + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + + rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()))) + .saveAsNewAPIHadoopFile(outputDir, IntWritable.class, Text.class, + org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.class); + + JavaPairRDD output = + sc.sequenceFile(outputDir, IntWritable.class, Text.class); + assertEquals(pairs.toString(), output.map(Tuple2::toString).collect().toString()); + } + + @SuppressWarnings("unchecked") + @Test + public void readWithNewAPIHadoopFile() throws IOException { + String outputDir = new File(tempDir, "output").getAbsolutePath(); + List> pairs = Arrays.asList( + new Tuple2<>(1, "a"), + new Tuple2<>(2, "aa"), + new Tuple2<>(3, "aaa") + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + + rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()))) + .saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); + + JavaPairRDD output = sc.newAPIHadoopFile(outputDir, + org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat.class, + IntWritable.class, Text.class, Job.getInstance().getConfiguration()); + assertEquals(pairs.toString(), output.map(Tuple2::toString).collect().toString()); + } + + @Test + public void objectFilesOfInts() { + String outputDir = new File(tempDir, "output").getAbsolutePath(); + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + rdd.saveAsObjectFile(outputDir); + // Try reading the output back as an object file + List expected = Arrays.asList(1, 2, 3, 4); + JavaRDD readRDD = sc.objectFile(outputDir); + assertEquals(expected, readRDD.collect()); + } + + @SuppressWarnings("unchecked") + @Test + public void objectFilesOfComplexTypes() { + String outputDir = new File(tempDir, "output").getAbsolutePath(); + List> pairs = Arrays.asList( + new Tuple2<>(1, "a"), + new Tuple2<>(2, "aa"), + new Tuple2<>(3, "aaa") + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + rdd.saveAsObjectFile(outputDir); + // Try reading the output back as an object file + JavaRDD> readRDD = sc.objectFile(outputDir); + assertEquals(pairs, readRDD.collect()); + } + + @SuppressWarnings("unchecked") + @Test + public void hadoopFile() { + String outputDir = new File(tempDir, "output").getAbsolutePath(); + List> pairs = Arrays.asList( + new Tuple2<>(1, "a"), + new Tuple2<>(2, "aa"), + new Tuple2<>(3, "aaa") + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + + rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()))) + .saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); + + JavaPairRDD output = sc.hadoopFile(outputDir, + SequenceFileInputFormat.class, IntWritable.class, Text.class); + assertEquals(pairs.toString(), output.map(Tuple2::toString).collect().toString()); + } + + @SuppressWarnings("unchecked") + @Test + public void hadoopFileCompressed() { + String outputDir = new File(tempDir, "output_compressed").getAbsolutePath(); + List> pairs = Arrays.asList( + new Tuple2<>(1, "a"), + new Tuple2<>(2, "aa"), + new Tuple2<>(3, "aaa") + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + + rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()))) + .saveAsHadoopFile(outputDir, IntWritable.class, Text.class, + SequenceFileOutputFormat.class, DefaultCodec.class); + + JavaPairRDD output = sc.hadoopFile(outputDir, + SequenceFileInputFormat.class, IntWritable.class, Text.class); + + assertEquals(pairs.toString(), output.map(Tuple2::toString).collect().toString()); + } + + @Test + public void zip() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); + JavaDoubleRDD doubles = rdd.mapToDouble(Integer::doubleValue); + JavaPairRDD zipped = rdd.zip(doubles); + zipped.count(); + } + + @Test + public void zipPartitions() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6), 2); + JavaRDD rdd2 = sc.parallelize(Arrays.asList("1", "2", "3", "4"), 2); + FlatMapFunction2, Iterator, Integer> sizesFn = + (i, s) -> Arrays.asList(Iterators.size(i), Iterators.size(s)).iterator(); + + JavaRDD sizes = rdd1.zipPartitions(rdd2, sizesFn); + assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); + } + + @SuppressWarnings("deprecation") + @Test + public void accumulators() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); + + Accumulator intAccum = sc.intAccumulator(10); + rdd.foreach(intAccum::add); + assertEquals((Integer) 25, intAccum.value()); + + Accumulator doubleAccum = sc.doubleAccumulator(10.0); + rdd.foreach(x -> doubleAccum.add((double) x)); + assertEquals((Double) 25.0, doubleAccum.value()); + + // Try a custom accumulator type + AccumulatorParam floatAccumulatorParam = new AccumulatorParam() { + @Override + public Float addInPlace(Float r, Float t) { + return r + t; + } + + @Override + public Float addAccumulator(Float r, Float t) { + return r + t; + } + + @Override + public Float zero(Float initialValue) { + return 0.0f; + } + }; + + Accumulator floatAccum = sc.accumulator(10.0f, floatAccumulatorParam); + rdd.foreach(x -> floatAccum.add((float) x)); + assertEquals((Float) 25.0f, floatAccum.value()); + + // Test the setValue method + floatAccum.setValue(5.0f); + assertEquals((Float) 5.0f, floatAccum.value()); + } + + @Test + public void keyBy() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2)); + List> s = rdd.keyBy(Object::toString).collect(); + assertEquals(new Tuple2<>("1", 1), s.get(0)); + assertEquals(new Tuple2<>("2", 2), s.get(1)); + } + + @Test + public void checkpointAndComputation() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); + sc.setCheckpointDir(tempDir.getAbsolutePath()); + assertFalse(rdd.isCheckpointed()); + rdd.checkpoint(); + rdd.count(); // Forces the DAG to cause a checkpoint + assertTrue(rdd.isCheckpointed()); + assertEquals(Arrays.asList(1, 2, 3, 4, 5), rdd.collect()); + } + + @Test + public void checkpointAndRestore() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); + sc.setCheckpointDir(tempDir.getAbsolutePath()); + assertFalse(rdd.isCheckpointed()); + rdd.checkpoint(); + rdd.count(); // Forces the DAG to cause a checkpoint + assertTrue(rdd.isCheckpointed()); + + assertTrue(rdd.getCheckpointFile().isPresent()); + JavaRDD recovered = sc.checkpointFile(rdd.getCheckpointFile().get()); + assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect()); + } + + @Test + public void combineByKey() { + JavaRDD originalRDD = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6)); + Function keyFunction = v1 -> v1 % 3; + Function createCombinerFunction = v1 -> v1; + + Function2 mergeValueFunction = (v1, v2) -> v1 + v2; + + JavaPairRDD combinedRDD = originalRDD.keyBy(keyFunction) + .combineByKey(createCombinerFunction, mergeValueFunction, mergeValueFunction); + Map results = combinedRDD.collectAsMap(); + ImmutableMap expected = ImmutableMap.of(0, 9, 1, 5, 2, 7); + assertEquals(expected, results); + + Partitioner defaultPartitioner = Partitioner.defaultPartitioner( + combinedRDD.rdd(), + JavaConverters.collectionAsScalaIterableConverter( + Collections.>emptyList()).asScala().toSeq()); + combinedRDD = originalRDD.keyBy(keyFunction) + .combineByKey( + createCombinerFunction, + mergeValueFunction, + mergeValueFunction, + defaultPartitioner, + false, + new KryoSerializer(new SparkConf())); + results = combinedRDD.collectAsMap(); + assertEquals(expected, results); + } + + @SuppressWarnings("unchecked") + @Test + public void mapOnPairRDD() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1,2,3,4)); + JavaPairRDD rdd2 = rdd1.mapToPair(i -> new Tuple2<>(i, i % 2)); + JavaPairRDD rdd3 = rdd2.mapToPair(in -> new Tuple2<>(in._2(), in._1())); + assertEquals(Arrays.asList( + new Tuple2<>(1, 1), + new Tuple2<>(0, 2), + new Tuple2<>(1, 3), + new Tuple2<>(0, 4)), rdd3.collect()); + } + + @SuppressWarnings("unchecked") + @Test + public void collectPartitions() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); + + JavaPairRDD rdd2 = rdd1.mapToPair(i -> new Tuple2<>(i, i % 2)); + + List[] parts = rdd1.collectPartitions(new int[] {0}); + assertEquals(Arrays.asList(1, 2), parts[0]); + + parts = rdd1.collectPartitions(new int[] {1, 2}); + assertEquals(Arrays.asList(3, 4), parts[0]); + assertEquals(Arrays.asList(5, 6, 7), parts[1]); + + assertEquals( + Arrays.asList(new Tuple2<>(1, 1), new Tuple2<>(2, 0)), + rdd2.collectPartitions(new int[] {0})[0]); + + List>[] parts2 = rdd2.collectPartitions(new int[] {1, 2}); + assertEquals(Arrays.asList(new Tuple2<>(3, 1), new Tuple2<>(4, 0)), parts2[0]); + assertEquals( + Arrays.asList( + new Tuple2<>(5, 1), + new Tuple2<>(6, 0), + new Tuple2<>(7, 1)), + parts2[1]); + } + + @Test + public void countApproxDistinct() { + List arrayData = new ArrayList<>(); + int size = 100; + for (int i = 0; i < 100000; i++) { + arrayData.add(i % size); + } + JavaRDD simpleRdd = sc.parallelize(arrayData, 10); + assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.05) - size) / (size * 1.0)) <= 0.1); + } + + @Test + public void countApproxDistinctByKey() { + List> arrayData = new ArrayList<>(); + for (int i = 10; i < 100; i++) { + for (int j = 0; j < i; j++) { + arrayData.add(new Tuple2<>(i, j)); + } + } + double relativeSD = 0.001; + JavaPairRDD pairRdd = sc.parallelizePairs(arrayData); + List> res = pairRdd.countApproxDistinctByKey(relativeSD, 8).collect(); + for (Tuple2 resItem : res) { + double count = resItem._1(); + long resCount = resItem._2(); + double error = Math.abs((resCount - count) / count); + assertTrue(error < 0.1); + } + } + + @Test + public void collectAsMapWithIntArrayValues() { + // Regression test for SPARK-1040 + JavaRDD rdd = sc.parallelize(Arrays.asList(1)); + JavaPairRDD pairRDD = rdd.mapToPair(x -> new Tuple2<>(x, new int[]{x})); + pairRDD.collect(); // Works fine + pairRDD.collectAsMap(); // Used to crash with ClassCastException + } + + @SuppressWarnings("unchecked") + @Test + public void collectAsMapAndSerialize() throws Exception { + JavaPairRDD rdd = + sc.parallelizePairs(Arrays.asList(new Tuple2<>("foo", 1))); + Map map = rdd.collectAsMap(); + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + new ObjectOutputStream(bytes).writeObject(map); + Map deserializedMap = (Map) + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray())).readObject(); + assertEquals(1, deserializedMap.get("foo").intValue()); + } + + @Test + @SuppressWarnings("unchecked") + public void sampleByKey() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3); + JavaPairRDD rdd2 = rdd1.mapToPair(i -> new Tuple2<>(i % 2, 1)); + Map fractions = new HashMap<>(); + fractions.put(0, 0.5); + fractions.put(1, 1.0); + JavaPairRDD wr = rdd2.sampleByKey(true, fractions, 1L); + Map wrCounts = wr.countByKey(); + assertEquals(2, wrCounts.size()); + assertTrue(wrCounts.get(0) > 0); + assertTrue(wrCounts.get(1) > 0); + JavaPairRDD wor = rdd2.sampleByKey(false, fractions, 1L); + Map worCounts = wor.countByKey(); + assertEquals(2, worCounts.size()); + assertTrue(worCounts.get(0) > 0); + assertTrue(worCounts.get(1) > 0); + } + + @Test + @SuppressWarnings("unchecked") + public void sampleByKeyExact() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3); + JavaPairRDD rdd2 = rdd1.mapToPair(i -> new Tuple2<>(i % 2, 1)); + Map fractions = new HashMap<>(); + fractions.put(0, 0.5); + fractions.put(1, 1.0); + JavaPairRDD wrExact = rdd2.sampleByKeyExact(true, fractions, 1L); + Map wrExactCounts = wrExact.countByKey(); + assertEquals(2, wrExactCounts.size()); + assertTrue(wrExactCounts.get(0) == 2); + assertTrue(wrExactCounts.get(1) == 4); + JavaPairRDD worExact = rdd2.sampleByKeyExact(false, fractions, 1L); + Map worExactCounts = worExact.countByKey(); + assertEquals(2, worExactCounts.size()); + assertTrue(worExactCounts.get(0) == 2); + assertTrue(worExactCounts.get(1) == 4); + } + + private static class SomeCustomClass implements Serializable { + SomeCustomClass() { + // Intentionally left blank + } + } + + @Test + public void collectUnderlyingScalaRDD() { + List data = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + data.add(new SomeCustomClass()); + } + JavaRDD rdd = sc.parallelize(data); + SomeCustomClass[] collected = + (SomeCustomClass[]) rdd.rdd().retag(SomeCustomClass.class).collect(); + assertEquals(data.size(), collected.length); + } + + private static final class BuggyMapFunction implements Function { + + @Override + public T call(T x) { + throw new IllegalStateException("Custom exception!"); + } + } + + @Test + public void collectAsync() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction> future = rdd.collectAsync(); + List result = future.get(); + assertEquals(data, result); + assertFalse(future.isCancelled()); + assertTrue(future.isDone()); + assertEquals(1, future.jobIds().size()); + } + + @Test + public void takeAsync() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction> future = rdd.takeAsync(1); + List result = future.get(); + assertEquals(1, result.size()); + assertEquals((Integer) 1, result.get(0)); + assertFalse(future.isCancelled()); + assertTrue(future.isDone()); + assertEquals(1, future.jobIds().size()); + } + + @Test + public void foreachAsync() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction future = rdd.foreachAsync(integer -> {}); + future.get(); + assertFalse(future.isCancelled()); + assertTrue(future.isDone()); + assertEquals(1, future.jobIds().size()); + } + + @Test + public void countAsync() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction future = rdd.countAsync(); + long count = future.get(); + assertEquals(data.size(), count); + assertFalse(future.isCancelled()); + assertTrue(future.isDone()); + assertEquals(1, future.jobIds().size()); + } + + @Test + public void testAsyncActionCancellation() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction future = rdd.foreachAsync(integer -> { + Thread.sleep(10000); // To ensure that the job won't finish before it's cancelled. + }); + future.cancel(true); + assertTrue(future.isCancelled()); + assertTrue(future.isDone()); + try { + future.get(2000, TimeUnit.MILLISECONDS); + fail("Expected future.get() for cancelled job to throw CancellationException"); + } catch (CancellationException ignored) { + // pass + } + } + + @Test + public void testAsyncActionErrorWrapping() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction future = rdd.map(new BuggyMapFunction<>()).countAsync(); + try { + future.get(2, TimeUnit.SECONDS); + fail("Expected future.get() for failed job to throw ExcecutionException"); + } catch (ExecutionException ee) { + assertTrue(Throwables.getStackTraceAsString(ee).contains("Custom exception!")); + } + assertTrue(future.isDone()); + } + + static class Class1 {} + static class Class2 {} + + @Test + public void testRegisterKryoClasses() { + SparkConf conf = new SparkConf(); + conf.registerKryoClasses(new Class[]{ Class1.class, Class2.class }); + assertEquals( + Class1.class.getName() + "," + Class2.class.getName(), + conf.get("spark.kryo.classesToRegister")); + } + + @Test + public void testGetPersistentRDDs() { + java.util.Map> cachedRddsMap = sc.getPersistentRDDs(); + assertTrue(cachedRddsMap.isEmpty()); + JavaRDD rdd1 = sc.parallelize(Arrays.asList("a", "b")).setName("RDD1").cache(); + JavaRDD rdd2 = sc.parallelize(Arrays.asList("c", "d")).setName("RDD2").cache(); + cachedRddsMap = sc.getPersistentRDDs(); + assertEquals(2, cachedRddsMap.size()); + assertEquals("RDD1", cachedRddsMap.get(0).name()); + assertEquals("RDD2", cachedRddsMap.get(1).name()); + } + +} diff --git a/core/src/test/java/test/org/apache/spark/JavaSparkContextSuite.java b/core/src/test/java/test/org/apache/spark/JavaSparkContextSuite.java new file mode 100644 index 000000000000..7e9cc70d8651 --- /dev/null +++ b/core/src/test/java/test/org/apache/spark/JavaSparkContextSuite.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark; + +import java.io.*; + +import scala.collection.immutable.List; +import scala.collection.immutable.List$; +import scala.collection.immutable.Map; +import scala.collection.immutable.Map$; + +import org.junit.Test; + +import org.apache.spark.api.java.*; +import org.apache.spark.*; + +/** + * Java apps can uses both Java-friendly JavaSparkContext and Scala SparkContext. + */ +public class JavaSparkContextSuite implements Serializable { + + @Test + public void javaSparkContext() { + String[] jars = new String[] {}; + java.util.Map environment = new java.util.HashMap<>(); + + new JavaSparkContext(new SparkConf().setMaster("local").setAppName("name")).stop(); + new JavaSparkContext("local", "name", new SparkConf()).stop(); + new JavaSparkContext("local", "name").stop(); + new JavaSparkContext("local", "name", "sparkHome", "jarFile").stop(); + new JavaSparkContext("local", "name", "sparkHome", jars).stop(); + new JavaSparkContext("local", "name", "sparkHome", jars, environment).stop(); + } + + @Test + public void scalaSparkContext() { + List jars = List$.MODULE$.empty(); + Map environment = Map$.MODULE$.empty(); + + new SparkContext(new SparkConf().setMaster("local").setAppName("name")).stop(); + new SparkContext("local", "name", new SparkConf()).stop(); + new SparkContext("local", "name").stop(); + new SparkContext("local", "name", "sparkHome").stop(); + new SparkContext("local", "name", "sparkHome", jars).stop(); + new SparkContext("local", "name", "sparkHome", jars, environment).stop(); + } +} diff --git a/core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java b/core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java deleted file mode 100644 index e38bc38949d7..000000000000 --- a/core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package test.org.apache.spark; - -import org.apache.spark.TaskContext; -import org.apache.spark.util.TaskCompletionListener; - - -/** - * A simple implementation of TaskCompletionListener that makes sure TaskCompletionListener and - * TaskContext is Java friendly. - */ -public class JavaTaskCompletionListenerImpl implements TaskCompletionListener { - - @Override - public void onTaskCompletion(TaskContext context) { - context.isCompleted(); - context.isInterrupted(); - context.stageId(); - context.partitionId(); - context.isRunningLocally(); - context.addTaskCompletionListener(this); - } -} diff --git a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java index 4a918f725dc9..94f5805853e1 100644 --- a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java +++ b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java @@ -18,6 +18,8 @@ package test.org.apache.spark; import org.apache.spark.TaskContext; +import org.apache.spark.util.TaskCompletionListener; +import org.apache.spark.util.TaskFailureListener; /** * Something to make sure that TaskContext can be used in Java. @@ -29,13 +31,39 @@ public static void test() { tc.isCompleted(); tc.isInterrupted(); - tc.isRunningLocally(); tc.addTaskCompletionListener(new JavaTaskCompletionListenerImpl()); + tc.addTaskFailureListener(new JavaTaskFailureListenerImpl()); tc.attemptNumber(); tc.partitionId(); tc.stageId(); tc.taskAttemptId(); } + + /** + * A simple implementation of TaskCompletionListener that makes sure TaskCompletionListener and + * TaskContext is Java friendly. + */ + static class JavaTaskCompletionListenerImpl implements TaskCompletionListener { + @Override + public void onTaskCompletion(TaskContext context) { + context.isCompleted(); + context.isInterrupted(); + context.stageId(); + context.partitionId(); + context.addTaskCompletionListener(this); + } + } + + /** + * A simple implementation of TaskCompletionListener that makes sure TaskCompletionListener and + * TaskContext is Java friendly. + */ + static class JavaTaskFailureListenerImpl implements TaskFailureListener { + @Override + public void onTaskFailure(TaskContext context, Throwable error) { + } + } + } diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json index d575bf2f284b..f2c3ec5da889 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -1,11 +1,47 @@ [ { + "id" : "app-20161116163331-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-16T22:33:29.916GMT", + "endTime" : "2016-11-16T22:33:40.587GMT", + "lastUpdated" : "", + "duration" : 10671, + "sparkUser" : "jose", + "completed" : true, + "appSparkVersion" : "2.1.0-SNAPSHOT", + "endTimeEpoch" : 1479335620587, + "startTimeEpoch" : 1479335609916, + "lastUpdatedEpoch" : 0 + } ] +}, { + "id" : "app-20161115172038-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-15T23:20:37.079GMT", + "endTime" : "2016-11-15T23:22:18.874GMT", + "lastUpdated" : "", + "duration" : 101795, + "sparkUser" : "jose", + "completed" : true, + "appSparkVersion" : "2.1.0-SNAPSHOT", + "endTimeEpoch" : 1479252138874, + "startTimeEpoch" : 1479252037079, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "local-1430917381534", "name" : "Spark shell", "attempts" : [ { "startTime" : "2015-05-06T13:03:00.893GMT", "endTime" : "2015-05-06T13:03:11.398GMT", + "lastUpdated" : "", + "duration" : 10505, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "endTimeEpoch" : 1430917391398, + "startTimeEpoch" : 1430917380893, + "lastUpdatedEpoch" : 0 } ] }, { "id" : "local-1430917381535", @@ -14,14 +50,26 @@ "attemptId" : "2", "startTime" : "2015-05-06T13:03:00.893GMT", "endTime" : "2015-05-06T13:03:00.950GMT", + "lastUpdated" : "", + "duration" : 57, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "endTimeEpoch" : 1430917380950, + "startTimeEpoch" : 1430917380893, + "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", "startTime" : "2015-05-06T13:03:00.880GMT", "endTime" : "2015-05-06T13:03:00.890GMT", + "lastUpdated" : "", + "duration" : 10, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "endTimeEpoch" : 1430917380890, + "startTimeEpoch" : 1430917380880, + "lastUpdatedEpoch" : 0 } ] }, { "id" : "local-1426533911241", @@ -30,14 +78,26 @@ "attemptId" : "2", "startTime" : "2015-03-17T23:11:50.242GMT", "endTime" : "2015-03-17T23:12:25.177GMT", + "lastUpdated" : "", + "duration" : 34935, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1426633945177, + "startTimeEpoch" : 1426633910242, + "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", "startTime" : "2015-03-16T19:25:10.242GMT", "endTime" : "2015-03-16T19:25:45.177GMT", + "lastUpdated" : "", + "duration" : 34935, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1426533945177, + "startTimeEpoch" : 1426533910242, + "lastUpdatedEpoch" : 0 } ] }, { "id" : "local-1425081759269", @@ -45,8 +105,14 @@ "attempts" : [ { "startTime" : "2015-02-28T00:02:38.277GMT", "endTime" : "2015-02-28T00:02:46.912GMT", + "lastUpdated" : "", + "duration" : 8635, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1425081766912, + "startTimeEpoch" : 1425081758277, + "lastUpdatedEpoch" : 0 } ] }, { "id" : "local-1422981780767", @@ -54,8 +120,14 @@ "attempts" : [ { "startTime" : "2015-02-03T16:42:59.720GMT", "endTime" : "2015-02-03T16:43:08.731GMT", + "lastUpdated" : "", + "duration" : 9011, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1422981788731, + "startTimeEpoch" : 1422981779720, + "lastUpdatedEpoch" : 0 } ] }, { "id" : "local-1422981759269", @@ -63,7 +135,13 @@ "attempts" : [ { "startTime" : "2015-02-03T16:42:38.277GMT", "endTime" : "2015-02-03T16:42:46.912GMT", + "lastUpdated" : "", + "duration" : 8635, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1422981766912, + "startTimeEpoch" : 1422981758277, + "lastUpdatedEpoch" : 0 } ] -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json index 31ac9beea878..25c4fff77e0a 100644 --- a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json @@ -6,6 +6,10 @@ "numCompleteTasks" : 8, "numFailedTasks" : 0, "executorRunTime" : 162, + "executorCpuTime" : 0, + "submissionTime" : "2015-02-03T16:43:07.191GMT", + "firstTaskLaunchedTime" : "2015-02-03T16:43:07.191GMT", + "completionTime" : "2015-02-03T16:43:07.226GMT", "inputBytes" : 160, "inputRecords" : 0, "outputBytes" : 0, @@ -28,6 +32,10 @@ "numCompleteTasks" : 8, "numFailedTasks" : 0, "executorRunTime" : 3476, + "executorCpuTime" : 0, + "submissionTime" : "2015-02-03T16:43:05.829GMT", + "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT", + "completionTime" : "2015-02-03T16:43:06.286GMT", "inputBytes" : 28000128, "inputRecords" : 0, "outputBytes" : 0, @@ -50,6 +58,10 @@ "numCompleteTasks" : 8, "numFailedTasks" : 0, "executorRunTime" : 4338, + "executorCpuTime" : 0, + "submissionTime" : "2015-02-03T16:43:04.228GMT", + "firstTaskLaunchedTime" : "2015-02-03T16:43:04.234GMT", + "completionTime" : "2015-02-03T16:43:04.819GMT", "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, @@ -64,4 +76,4 @@ "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", "accumulatorUpdates" : [ ] -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index d575bf2f284b..c925c1dd8a4d 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -1,11 +1,47 @@ [ { + "id" : "app-20161116163331-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-16T22:33:29.916GMT", + "endTime" : "2016-11-16T22:33:40.587GMT", + "lastUpdated" : "", + "duration" : 10671, + "sparkUser" : "jose", + "completed" : true, + "appSparkVersion" : "2.1.0-SNAPSHOT", + "endTimeEpoch" : 1479335620587, + "startTimeEpoch" : 1479335609916, + "lastUpdatedEpoch" : 0 + } ] +}, { + "id" : "app-20161115172038-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-15T23:20:37.079GMT", + "endTime" : "2016-11-15T23:22:18.874GMT", + "lastUpdated" : "", + "duration" : 101795, + "sparkUser" : "jose", + "completed" : true, + "appSparkVersion" : "2.1.0-SNAPSHOT", + "endTimeEpoch" : 1479252138874, + "startTimeEpoch" : 1479252037079, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "local-1430917381534", "name" : "Spark shell", "attempts" : [ { "startTime" : "2015-05-06T13:03:00.893GMT", "endTime" : "2015-05-06T13:03:11.398GMT", + "lastUpdated" : "", + "duration" : 10505, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "endTimeEpoch" : 1430917391398, + "startTimeEpoch" : 1430917380893, + "lastUpdatedEpoch" : 0 } ] }, { "id" : "local-1430917381535", @@ -14,14 +50,26 @@ "attemptId" : "2", "startTime" : "2015-05-06T13:03:00.893GMT", "endTime" : "2015-05-06T13:03:00.950GMT", + "lastUpdated" : "", + "duration" : 57, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "endTimeEpoch" : 1430917380950, + "startTimeEpoch" : 1430917380893, + "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", "startTime" : "2015-05-06T13:03:00.880GMT", "endTime" : "2015-05-06T13:03:00.890GMT", + "lastUpdated" : "", + "duration" : 10, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "endTimeEpoch" : 1430917380890, + "startTimeEpoch" : 1430917380880, + "lastUpdatedEpoch" : 0 } ] }, { "id" : "local-1426533911241", @@ -30,14 +78,26 @@ "attemptId" : "2", "startTime" : "2015-03-17T23:11:50.242GMT", "endTime" : "2015-03-17T23:12:25.177GMT", + "lastUpdated" : "", + "duration" : 34935, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1426633945177, + "startTimeEpoch" : 1426633910242, + "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", "startTime" : "2015-03-16T19:25:10.242GMT", "endTime" : "2015-03-16T19:25:45.177GMT", + "lastUpdated" : "", + "duration" : 34935, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1426533945177, + "startTimeEpoch" : 1426533910242, + "lastUpdatedEpoch" : 0 } ] }, { "id" : "local-1425081759269", @@ -45,8 +105,15 @@ "attempts" : [ { "startTime" : "2015-02-28T00:02:38.277GMT", "endTime" : "2015-02-28T00:02:46.912GMT", + "lastUpdated" : "", + "duration" : 8635, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "appSparkVersion" : "", + "endTimeEpoch" : 1425081766912, + "startTimeEpoch" : 1425081758277, + "lastUpdatedEpoch" : 0 } ] }, { "id" : "local-1422981780767", @@ -54,8 +121,14 @@ "attempts" : [ { "startTime" : "2015-02-03T16:42:59.720GMT", "endTime" : "2015-02-03T16:43:08.731GMT", + "lastUpdated" : "", + "duration" : 9011, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1422981788731, + "startTimeEpoch" : 1422981779720, + "lastUpdatedEpoch" : 0 } ] }, { "id" : "local-1422981759269", @@ -63,7 +136,13 @@ "attempts" : [ { "startTime" : "2015-02-03T16:42:38.277GMT", "endTime" : "2015-02-03T16:42:46.912GMT", + "lastUpdated" : "", + "duration" : 8635, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1422981766912, + "startTimeEpoch" : 1422981758277, + "lastUpdatedEpoch" : 0 } ] -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json index cb622e147249..6b9f29e1a230 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json @@ -1,17 +1,22 @@ [ { "id" : "", "hostPort" : "localhost:57971", - "rddBlocks" : 8, - "memoryUsed" : 28000128, + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, "activeTasks" : 0, "failedTasks" : 1, "completedTasks" : 31, "totalTasks" : 32, "totalDuration" : 8820, + "totalGCTime" : 352, "totalInputBytes" : 28000288, "totalShuffleRead" : 0, "totalShuffleWrite" : 13180, + "isBlacklisted" : false, "maxMemory" : 278302556, "executorLogs" : { } -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json new file mode 100644 index 000000000000..0f94e3b255db --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json @@ -0,0 +1,148 @@ +[ { + "id" : "2", + "hostPort" : "172.22.0.167:51487", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 4, + "completedTasks" : 0, + "totalTasks" : 4, + "totalDuration" : 2537, + "totalGCTime" : 88, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr" + }, + "memoryMetrics": { + "usedOnHeapStorageMemory": 0, + "usedOffHeapStorageMemory": 0, + "totalOnHeapStorageMemory": 384093388, + "totalOffHeapStorageMemory": 524288000 + } +}, { + "id" : "driver", + "hostPort" : "172.22.0.167:51475", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { }, + "memoryMetrics": { + "usedOnHeapStorageMemory": 0, + "usedOffHeapStorageMemory": 0, + "totalOnHeapStorageMemory": 384093388, + "totalOffHeapStorageMemory": 524288000 + } +}, { + "id" : "1", + "hostPort" : "172.22.0.167:51490", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 4, + "totalTasks" : 4, + "totalDuration" : 3152, + "totalGCTime" : 68, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stdout", + "stderr" : "http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stderr" + }, + "memoryMetrics": { + "usedOnHeapStorageMemory": 0, + "usedOffHeapStorageMemory": 0, + "totalOnHeapStorageMemory": 384093388, + "totalOffHeapStorageMemory": 524288000 + } +}, { + "id" : "0", + "hostPort" : "172.22.0.167:51491", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 4, + "completedTasks" : 0, + "totalTasks" : 4, + "totalDuration" : 2551, + "totalGCTime" : 116, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stdout", + "stderr" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stderr" + }, + "memoryMetrics": { + "usedOnHeapStorageMemory": 0, + "usedOffHeapStorageMemory": 0, + "totalOnHeapStorageMemory": 384093388, + "totalOffHeapStorageMemory": 524288000 + } +}, { + "id" : "3", + "hostPort" : "172.22.0.167:51485", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 12, + "totalTasks" : 12, + "totalDuration" : 2453, + "totalGCTime" : 72, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout", + "stderr" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr" + }, + "memoryMetrics": { + "usedOnHeapStorageMemory": 0, + "usedOffHeapStorageMemory": 0, + "totalOnHeapStorageMemory": 384093388, + "totalOffHeapStorageMemory": 524288000 + } +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json new file mode 100644 index 000000000000..0f94e3b255db --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json @@ -0,0 +1,148 @@ +[ { + "id" : "2", + "hostPort" : "172.22.0.167:51487", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 4, + "completedTasks" : 0, + "totalTasks" : 4, + "totalDuration" : 2537, + "totalGCTime" : 88, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr" + }, + "memoryMetrics": { + "usedOnHeapStorageMemory": 0, + "usedOffHeapStorageMemory": 0, + "totalOnHeapStorageMemory": 384093388, + "totalOffHeapStorageMemory": 524288000 + } +}, { + "id" : "driver", + "hostPort" : "172.22.0.167:51475", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { }, + "memoryMetrics": { + "usedOnHeapStorageMemory": 0, + "usedOffHeapStorageMemory": 0, + "totalOnHeapStorageMemory": 384093388, + "totalOffHeapStorageMemory": 524288000 + } +}, { + "id" : "1", + "hostPort" : "172.22.0.167:51490", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 4, + "totalTasks" : 4, + "totalDuration" : 3152, + "totalGCTime" : 68, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stdout", + "stderr" : "http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stderr" + }, + "memoryMetrics": { + "usedOnHeapStorageMemory": 0, + "usedOffHeapStorageMemory": 0, + "totalOnHeapStorageMemory": 384093388, + "totalOffHeapStorageMemory": 524288000 + } +}, { + "id" : "0", + "hostPort" : "172.22.0.167:51491", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 4, + "completedTasks" : 0, + "totalTasks" : 4, + "totalDuration" : 2551, + "totalGCTime" : 116, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stdout", + "stderr" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stderr" + }, + "memoryMetrics": { + "usedOnHeapStorageMemory": 0, + "usedOffHeapStorageMemory": 0, + "totalOnHeapStorageMemory": 384093388, + "totalOffHeapStorageMemory": 524288000 + } +}, { + "id" : "3", + "hostPort" : "172.22.0.167:51485", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 12, + "totalTasks" : 12, + "totalDuration" : 2453, + "totalGCTime" : 72, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 908381388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout", + "stderr" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr" + }, + "memoryMetrics": { + "usedOnHeapStorageMemory": 0, + "usedOffHeapStorageMemory": 0, + "totalOnHeapStorageMemory": 384093388, + "totalOffHeapStorageMemory": 524288000 + } +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json new file mode 100644 index 000000000000..92e249c85111 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json @@ -0,0 +1,118 @@ +[ { + "id" : "2", + "hostPort" : "172.22.0.111:64539", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 6, + "completedTasks" : 0, + "totalTasks" : 6, + "totalDuration" : 2792, + "totalGCTime" : 128, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" + } +}, { + "id" : "driver", + "hostPort" : "172.22.0.111:64527", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { } +}, { + "id" : "1", + "hostPort" : "172.22.0.111:64541", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 12, + "totalTasks" : 12, + "totalDuration" : 2613, + "totalGCTime" : 84, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stdout", + "stderr" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stderr" + } +}, { + "id" : "0", + "hostPort" : "172.22.0.111:64540", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 4, + "completedTasks" : 0, + "totalTasks" : 4, + "totalDuration" : 2741, + "totalGCTime" : 120, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout", + "stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr" + } +}, { + "id" : "3", + "hostPort" : "172.22.0.111:64543", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 4, + "totalTasks" : 4, + "totalDuration" : 3457, + "totalGCTime" : 72, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stdout", + "stderr" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stderr" + } +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json index bff6a4f69d07..b86ba1e65de1 100644 --- a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json @@ -6,6 +6,10 @@ "numCompleteTasks" : 7, "numFailedTasks" : 1, "executorRunTime" : 278, + "executorCpuTime" : 0, + "submissionTime" : "2015-02-03T16:43:06.296GMT", + "firstTaskLaunchedTime" : "2015-02-03T16:43:06.296GMT", + "completionTime" : "2015-02-03T16:43:06.347GMT", "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, @@ -20,4 +24,4 @@ "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", "accumulatorUpdates" : [ ] -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json index 2e92e1fa0ec2..c108fa61a431 100644 --- a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json @@ -6,10 +6,10 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, "numFailedStages" : 0 -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json index 2e92e1fa0ec2..c108fa61a431 100644 --- a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json @@ -6,10 +6,10 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, "numFailedStages" : 0 -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json index cab4750270df..3d7407004d26 100644 --- a/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json @@ -6,7 +6,7 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, @@ -20,7 +20,7 @@ "numTasks" : 16, "numActiveTasks" : 0, "numCompletedTasks" : 15, - "numSkippedTasks" : 15, + "numSkippedTasks" : 0, "numFailedTasks" : 1, "numActiveStages" : 0, "numCompletedStages" : 1, @@ -34,10 +34,10 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, "numFailedStages" : 0 -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json new file mode 100644 index 000000000000..cc0b2b0022bd --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json @@ -0,0 +1,46 @@ +[ { + "id" : "app-20161116163331-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-16T22:33:29.916GMT", + "endTime" : "2016-11-16T22:33:40.587GMT", + "lastUpdated" : "", + "duration" : 10671, + "sparkUser" : "jose", + "completed" : true, + "appSparkVersion" : "2.1.0-SNAPSHOT", + "endTimeEpoch" : 1479335620587, + "startTimeEpoch" : 1479335609916, + "lastUpdatedEpoch" : 0 + } ] +}, { + "id" : "app-20161115172038-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-15T23:20:37.079GMT", + "endTime" : "2016-11-15T23:22:18.874GMT", + "lastUpdated" : "", + "duration" : 101795, + "sparkUser" : "jose", + "completed" : true, + "appSparkVersion" : "2.1.0-SNAPSHOT", + "endTimeEpoch" : 1479252138874, + "startTimeEpoch" : 1479252037079, + "lastUpdatedEpoch" : 0 + } ] +}, { + "id" : "local-1430917381534", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:11.398GMT", + "lastUpdated" : "", + "duration" : 10505, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "endTimeEpoch" : 1430917391398, + "startTimeEpoch" : 1430917380893, + "lastUpdatedEpoch" : 0 + } ] +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json index 483632a3956e..fa12413eeb0e 100644 --- a/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json @@ -4,7 +4,13 @@ "attempts" : [ { "startTime" : "2015-02-03T16:42:38.277GMT", "endTime" : "2015-02-03T16:42:46.912GMT", + "lastUpdated" : "", + "duration" : 8635, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1422981766912, + "startTimeEpoch" : 1422981758277, + "lastUpdatedEpoch" : 0 } ] -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json index 4b85690fd919..a0d4a0d1c455 100644 --- a/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json @@ -4,8 +4,14 @@ "attempts" : [ { "startTime" : "2015-02-03T16:42:59.720GMT", "endTime" : "2015-02-03T16:43:08.731GMT", + "lastUpdated" : "", + "duration" : 9011, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1422981788731, + "startTimeEpoch" : 1422981779720, + "lastUpdatedEpoch" : 0 } ] }, { "id" : "local-1422981759269", @@ -13,7 +19,13 @@ "attempts" : [ { "startTime" : "2015-02-03T16:42:38.277GMT", "endTime" : "2015-02-03T16:42:46.912GMT", + "lastUpdated" : "", + "duration" : 8635, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1422981766912, + "startTimeEpoch" : 1422981758277, + "lastUpdatedEpoch" : 0 } ] -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/maxEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/maxEndDate_app_list_json_expectation.json new file mode 100644 index 000000000000..dfa90010c6ca --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/maxEndDate_app_list_json_expectation.json @@ -0,0 +1,102 @@ +[ { + "id" : "local-1430917381535", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:00.950GMT", + "lastUpdated" : "", + "duration" : 57, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917380950 + }, { + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.880GMT", + "endTime" : "2015-05-06T13:03:00.890GMT", + "lastUpdated" : "", + "duration" : 10, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380880, + "endTimeEpoch" : 1430917380890 + } ] +}, { + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "lastUpdated" : "", + "duration" : 34935, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1426633910242, + "endTimeEpoch" : 1426633945177 + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "lastUpdated" : "", + "duration" : 34935, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1426533910242, + "endTimeEpoch" : 1426533945177 + } ] +}, { + "id" : "local-1425081759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-28T00:02:38.277GMT", + "endTime" : "2015-02-28T00:02:46.912GMT", + "lastUpdated" : "", + "duration" : 8635, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1425081758277, + "endTimeEpoch" : 1425081766912 + } ] +}, { + "id" : "local-1422981780767", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:59.720GMT", + "endTime" : "2015-02-03T16:43:08.731GMT", + "lastUpdated" : "", + "duration" : 9011, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1422981779720, + "endTimeEpoch" : 1422981788731 + } ] +}, { + "id" : "local-1422981759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-03T16:42:38.277GMT", + "endTime" : "2015-02-03T16:42:46.912GMT", + "lastUpdated" : "", + "duration" : 8635, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1422981758277, + "endTimeEpoch" : 1422981766912 + } ] +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_and_maxEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_and_maxEndDate_app_list_json_expectation.json new file mode 100644 index 000000000000..3ebe60e2cd03 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/minDate_and_maxEndDate_app_list_json_expectation.json @@ -0,0 +1,57 @@ +[ { + "id" : "local-1430917381535", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:00.950GMT", + "lastUpdated" : "", + "duration" : 57, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917380950 + }, { + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.880GMT", + "endTime" : "2015-05-06T13:03:00.890GMT", + "lastUpdated" : "", + "duration" : 10, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380880, + "endTimeEpoch" : 1430917380890 + } ] +}, { + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "lastUpdated" : "", + "duration" : 34935, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1426633910242, + "endTimeEpoch" : 1426633945177 + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "lastUpdated" : "", + "duration" : 34935, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1426533910242, + "endTimeEpoch" : 1426533945177 + } ] +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json index 15c2de8ef99e..5af50abd8533 100644 --- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -1,27 +1,75 @@ [ { + "id" : "app-20161116163331-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-16T22:33:29.916GMT", + "endTime" : "2016-11-16T22:33:40.587GMT", + "lastUpdated" : "", + "duration" : 10671, + "sparkUser" : "jose", + "completed" : true, + "appSparkVersion" : "2.1.0-SNAPSHOT", + "endTimeEpoch" : 1479335620587, + "startTimeEpoch" : 1479335609916, + "lastUpdatedEpoch" : 0 + } ] +}, { + "id" : "app-20161115172038-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-15T23:20:37.079GMT", + "endTime" : "2016-11-15T23:22:18.874GMT", + "lastUpdated" : "", + "duration" : 101795, + "sparkUser" : "jose", + "completed" : true, + "appSparkVersion" : "2.1.0-SNAPSHOT", + "endTimeEpoch" : 1479252138874, + "startTimeEpoch" : 1479252037079, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "local-1430917381534", "name" : "Spark shell", "attempts" : [ { "startTime" : "2015-05-06T13:03:00.893GMT", "endTime" : "2015-05-06T13:03:11.398GMT", + "lastUpdated" : "", + "duration" : 10505, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "endTimeEpoch" : 1430917391398, + "startTimeEpoch" : 1430917380893, + "lastUpdatedEpoch" : 0 } ] -}, { +}, { "id" : "local-1430917381535", "name" : "Spark shell", "attempts" : [ { "attemptId" : "2", "startTime" : "2015-05-06T13:03:00.893GMT", "endTime" : "2015-05-06T13:03:00.950GMT", + "lastUpdated" : "", + "duration" : 57, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "endTimeEpoch" : 1430917380950, + "startTimeEpoch" : 1430917380893, + "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", "startTime" : "2015-05-06T13:03:00.880GMT", "endTime" : "2015-05-06T13:03:00.890GMT", + "lastUpdated" : "", + "duration" : 10, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "endTimeEpoch" : 1430917380890, + "startTimeEpoch" : 1430917380880, + "lastUpdatedEpoch" : 0 } ] }, { "id" : "local-1426533911241", @@ -30,24 +78,40 @@ "attemptId" : "2", "startTime" : "2015-03-17T23:11:50.242GMT", "endTime" : "2015-03-17T23:12:25.177GMT", + "lastUpdated" : "", + "duration" : 34935, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1426633945177, + "startTimeEpoch" : 1426633910242, + "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", "startTime" : "2015-03-16T19:25:10.242GMT", "endTime" : "2015-03-16T19:25:45.177GMT", + "lastUpdated" : "", + "duration" : 34935, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1426533945177, + "startTimeEpoch" : 1426533910242, + "lastUpdatedEpoch" : 0 } ] }, { - "id": "local-1425081759269", - "name": "Spark shell", - "attempts": [ - { - "startTime": "2015-02-28T00:02:38.277GMT", - "endTime": "2015-02-28T00:02:46.912GMT", - "sparkUser": "irashid", - "completed": true - } - ] -} ] \ No newline at end of file + "id" : "local-1425081759269", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-02-28T00:02:38.277GMT", + "endTime" : "2015-02-28T00:02:46.912GMT", + "lastUpdated" : "", + "duration" : 8635, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1425081766912, + "startTimeEpoch" : 1425081758277, + "lastUpdatedEpoch" : 0 + } ] +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/minEndDate_and_maxEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minEndDate_and_maxEndDate_app_list_json_expectation.json new file mode 100644 index 000000000000..74a7b40a5927 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_and_maxEndDate_app_list_json_expectation.json @@ -0,0 +1,57 @@ +[ { + "id" : "local-1430917381535", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:00.950GMT", + "lastUpdated" : "", + "duration" : 57, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917380950 + }, { + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.880GMT", + "endTime" : "2015-05-06T13:03:00.890GMT", + "lastUpdated" : "", + "duration" : 10, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380880, + "endTimeEpoch" : 1430917380890 + } ] +}, { + "id" : "local-1426533911241", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-03-17T23:11:50.242GMT", + "endTime" : "2015-03-17T23:12:25.177GMT", + "lastUpdated" : "", + "duration" : 34935, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1426633910242, + "endTimeEpoch" : 1426633945177 + }, { + "attemptId" : "1", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "lastUpdated" : "", + "duration" : 34935, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1426533910242, + "endTimeEpoch" : 1426533945177 + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json new file mode 100644 index 000000000000..7f896c74b5be --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json @@ -0,0 +1,74 @@ +[ { + "id" : "app-20161116163331-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-16T22:33:29.916GMT", + "endTime" : "2016-11-16T22:33:40.587GMT", + "lastUpdated" : "", + "duration" : 10671, + "sparkUser" : "jose", + "completed" : true, + "appSparkVersion" : "2.1.0-SNAPSHOT", + "startTimeEpoch" : 1479335609916, + "lastUpdatedEpoch" : 0, + "endTimeEpoch" : 1479335620587 + } ] +}, { + "id" : "app-20161115172038-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-15T23:20:37.079GMT", + "endTime" : "2016-11-15T23:22:18.874GMT", + "lastUpdated" : "", + "duration" : 101795, + "sparkUser" : "jose", + "completed" : true, + "appSparkVersion" : "2.1.0-SNAPSHOT", + "startTimeEpoch" : 1479252037079, + "lastUpdatedEpoch" : 0, + "endTimeEpoch" : 1479252138874 + } ] +}, { + "id" : "local-1430917381534", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:11.398GMT", + "lastUpdated" : "", + "duration" : 10505, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917391398 + } ] +}, { + "id" : "local-1430917381535", + "name" : "Spark shell", + "attempts" : [ { + "attemptId" : "2", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:00.950GMT", + "lastUpdated" : "", + "duration" : 57, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380893, + "endTimeEpoch" : 1430917380950 + }, { + "attemptId" : "1", + "startTime" : "2015-05-06T13:03:00.880GMT", + "endTime" : "2015-05-06T13:03:00.890GMT", + "lastUpdated" : "", + "duration" : 10, + "sparkUser" : "irashid", + "completed" : true, + "appSparkVersion" : "1.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1430917380880, + "endTimeEpoch" : 1430917380890 + } ] +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json index 07489ad96414..24ec6a163fc2 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json @@ -4,7 +4,13 @@ "attempts" : [ { "startTime" : "2015-02-03T16:42:59.720GMT", "endTime" : "2015-02-03T16:43:08.731GMT", + "lastUpdated" : "", + "duration" : 9011, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1422981788731, + "startTimeEpoch" : 1422981779720, + "lastUpdatedEpoch" : 0 } ] -} \ No newline at end of file +} diff --git a/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json index 8f3d7160c723..94b6d6dba76e 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json @@ -5,13 +5,25 @@ "attemptId" : "2", "startTime" : "2015-03-17T23:11:50.242GMT", "endTime" : "2015-03-17T23:12:25.177GMT", + "lastUpdated" : "", + "duration" : 34935, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1426633945177, + "startTimeEpoch" : 1426633910242, + "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", "startTime" : "2015-03-16T19:25:10.242GMT", "endTime" : "2015-03-16T19:25:45.177GMT", + "lastUpdated" : "", + "duration" : 34935, "sparkUser" : "irashid", - "completed" : true + "completed" : true, + "appSparkVersion" : "", + "endTimeEpoch" : 1426533945177, + "startTimeEpoch" : 1426533910242, + "lastUpdatedEpoch" : 0 } ] -} \ No newline at end of file +} diff --git a/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json index 4a29072bdb6e..10c7e1c0b36f 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json @@ -6,10 +6,10 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, "numFailedStages" : 0 -} \ No newline at end of file +} diff --git a/core/src/test/resources/HistoryServerExpectations/one_rdd_storage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_rdd_storage_json_expectation.json deleted file mode 100644 index 38b5328ffbb0..000000000000 --- a/core/src/test/resources/HistoryServerExpectations/one_rdd_storage_json_expectation.json +++ /dev/null @@ -1,64 +0,0 @@ -{ - "id" : 0, - "name" : "0", - "numPartitions" : 8, - "numCachedPartitions" : 8, - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 28000128, - "diskUsed" : 0, - "dataDistribution" : [ { - "address" : "localhost:57971", - "memoryUsed" : 28000128, - "memoryRemaining" : 250302428, - "diskUsed" : 0 - } ], - "partitions" : [ { - "blockName" : "rdd_0_0", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - }, { - "blockName" : "rdd_0_1", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - }, { - "blockName" : "rdd_0_2", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - }, { - "blockName" : "rdd_0_3", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - }, { - "blockName" : "rdd_0_4", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - }, { - "blockName" : "rdd_0_5", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - }, { - "blockName" : "rdd_0_6", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - }, { - "blockName" : "rdd_0_7", - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 3500016, - "diskUsed" : 0, - "executors" : [ "localhost:57971" ] - } ] -} \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json index 111cb8163eb3..6fb40f6f1713 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json @@ -6,6 +6,10 @@ "numCompleteTasks" : 8, "numFailedTasks" : 0, "executorRunTime" : 3476, + "executorCpuTime" : 0, + "submissionTime" : "2015-02-03T16:43:05.829GMT", + "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT", + "completionTime" : "2015-02-03T16:43:06.286GMT", "inputBytes" : 28000128, "inputRecords" : 0, "outputBytes" : 0, @@ -26,14 +30,18 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.829GMT", + "duration" : 435, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 1, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 435, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 2, @@ -43,6 +51,19 @@ "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 94000, @@ -50,48 +71,69 @@ } } }, - "11" : { - "taskId" : 11, - "index" : 3, + "9" : { + "taskId" : 9, + "index" : 1, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 436, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, - "executorRunTime" : 434, + "executorDeserializeTime" : 1, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 436, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { - "bytesWritten" : 1647, - "writeTime" : 83000, + "bytesWritten" : 1648, + "writeTime" : 98000, "recordsWritten" : 0 } } }, - "14" : { - "taskId" : 14, - "index" : 6, + "10" : { + "taskId" : 10, + "index" : 2, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.832GMT", + "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -101,55 +143,89 @@ "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 88000, + "writeTime" : 76000, "recordsWritten" : 0 } } }, - "13" : { - "taskId" : 13, - "index" : 5, + "11" : { + "taskId" : 11, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", + "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 73000, + "bytesWritten" : 1647, + "writeTime" : 83000, "recordsWritten" : 0 } } }, - "10" : { - "taskId" : 10, - "index" : 2, + "12" : { + "taskId" : 12, + "index" : 4, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", + "launchTime" : "2015-02-03T16:43:05.831GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -159,55 +235,89 @@ "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 76000, + "bytesWritten" : 1645, + "writeTime" : 101000, "recordsWritten" : 0 } } }, - "9" : { - "taskId" : 9, - "index" : 1, + "13" : { + "taskId" : 13, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", + "launchTime" : "2015-02-03T16:43:05.831GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 436, + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 434, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 0, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 98000, + "writeTime" : 73000, "recordsWritten" : 0 } } }, - "12" : { - "taskId" : 12, - "index" : 4, + "14" : { + "taskId" : 14, + "index" : 6, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", + "launchTime" : "2015-02-03T16:43:05.832GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -217,9 +327,22 @@ "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { - "bytesWritten" : 1645, - "writeTime" : 101000, + "bytesWritten" : 1648, + "writeTime" : 88000, "recordsWritten" : 0 } } @@ -229,14 +352,18 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.833GMT", + "duration" : 435, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 1, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 435, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -246,6 +373,19 @@ "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 79000, @@ -267,4 +407,4 @@ "diskBytesSpilled" : 0 } } -} \ No newline at end of file +} diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json index ef339f89afa4..f5a89a210764 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json @@ -6,6 +6,10 @@ "numCompleteTasks" : 8, "numFailedTasks" : 0, "executorRunTime" : 3476, + "executorCpuTime" : 0, + "submissionTime" : "2015-02-03T16:43:05.829GMT", + "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT", + "completionTime" : "2015-02-03T16:43:06.286GMT", "inputBytes" : 28000128, "inputRecords" : 0, "outputBytes" : 0, @@ -26,14 +30,18 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.829GMT", + "duration" : 435, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 1, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 435, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 2, @@ -43,6 +51,19 @@ "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 94000, @@ -50,48 +71,69 @@ } } }, - "11" : { - "taskId" : 11, - "index" : 3, + "9" : { + "taskId" : 9, + "index" : 1, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 436, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, - "executorRunTime" : 434, + "executorDeserializeTime" : 1, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 436, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { - "bytesWritten" : 1647, - "writeTime" : 83000, + "bytesWritten" : 1648, + "writeTime" : 98000, "recordsWritten" : 0 } } }, - "14" : { - "taskId" : 14, - "index" : 6, + "10" : { + "taskId" : 10, + "index" : 2, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.832GMT", + "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -101,55 +143,89 @@ "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 88000, + "writeTime" : 76000, "recordsWritten" : 0 } } }, - "13" : { - "taskId" : 13, - "index" : 5, + "11" : { + "taskId" : 11, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", + "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 73000, + "bytesWritten" : 1647, + "writeTime" : 83000, "recordsWritten" : 0 } } }, - "10" : { - "taskId" : 10, - "index" : 2, + "12" : { + "taskId" : 12, + "index" : 4, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", + "launchTime" : "2015-02-03T16:43:05.831GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -159,55 +235,89 @@ "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 76000, + "bytesWritten" : 1645, + "writeTime" : 101000, "recordsWritten" : 0 } } }, - "9" : { - "taskId" : 9, - "index" : 1, + "13" : { + "taskId" : 13, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", + "launchTime" : "2015-02-03T16:43:05.831GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 1, - "executorRunTime" : 436, + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 434, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 0, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 98000, + "writeTime" : 73000, "recordsWritten" : 0 } } }, - "12" : { - "taskId" : 12, - "index" : 4, + "14" : { + "taskId" : 14, + "index" : 6, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", + "launchTime" : "2015-02-03T16:43:05.832GMT", + "duration" : 434, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 434, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -217,9 +327,22 @@ "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { - "bytesWritten" : 1645, - "writeTime" : 101000, + "bytesWritten" : 1648, + "writeTime" : 88000, "recordsWritten" : 0 } } @@ -229,14 +352,18 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.833GMT", + "duration" : 435, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 1, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 435, + "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, "resultSerializationTime" : 1, @@ -246,6 +373,19 @@ "bytesRead" : 3500016, "recordsRead" : 0 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 79000, @@ -267,4 +407,4 @@ "diskBytesSpilled" : 0 } } -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json index f79a31022d21..1e3ec7217afb 100644 --- a/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/rdd_list_storage_json_expectation.json @@ -1,9 +1 @@ -[ { - "id" : 0, - "name" : "0", - "numPartitions" : 8, - "numCachedPartitions" : 8, - "storageLevel" : "Memory Deserialized 1x Replicated", - "memoryUsed" : 28000128, - "diskUsed" : 0 -} ] \ No newline at end of file +[ ] diff --git a/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json index 8878e547a798..1e3ec7217afb 100644 --- a/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json @@ -1 +1 @@ -[ ] \ No newline at end of file +[ ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json index 056fac708859..6509df1508b3 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json @@ -6,6 +6,10 @@ "numCompleteTasks" : 8, "numFailedTasks" : 0, "executorRunTime" : 162, + "executorCpuTime" : 0, + "submissionTime" : "2015-02-03T16:43:07.191GMT", + "firstTaskLaunchedTime" : "2015-02-03T16:43:07.191GMT", + "completionTime" : "2015-02-03T16:43:07.226GMT", "inputBytes" : 160, "inputRecords" : 0, "outputBytes" : 0, @@ -28,6 +32,10 @@ "numCompleteTasks" : 8, "numFailedTasks" : 0, "executorRunTime" : 3476, + "executorCpuTime" : 0, + "submissionTime" : "2015-02-03T16:43:05.829GMT", + "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT", + "completionTime" : "2015-02-03T16:43:06.286GMT", "inputBytes" : 28000128, "inputRecords" : 0, "outputBytes" : 0, @@ -50,6 +58,10 @@ "numCompleteTasks" : 8, "numFailedTasks" : 0, "executorRunTime" : 4338, + "executorCpuTime" : 0, + "submissionTime" : "2015-02-03T16:43:04.228GMT", + "firstTaskLaunchedTime" : "2015-02-03T16:43:04.234GMT", + "completionTime" : "2015-02-03T16:43:04.819GMT", "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, @@ -72,6 +84,10 @@ "numCompleteTasks" : 7, "numFailedTasks" : 1, "executorRunTime" : 278, + "executorCpuTime" : 0, + "submissionTime" : "2015-02-03T16:43:06.296GMT", + "firstTaskLaunchedTime" : "2015-02-03T16:43:06.296GMT", + "completionTime" : "2015-02-03T16:43:06.347GMT", "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, @@ -86,4 +102,4 @@ "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", "accumulatorUpdates" : [ ] -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json index 79ccacd30969..8496863a9346 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json @@ -6,6 +6,10 @@ "numCompleteTasks" : 8, "numFailedTasks" : 0, "executorRunTime" : 120, + "executorCpuTime" : 0, + "submissionTime" : "2015-03-16T19:25:36.103GMT", + "firstTaskLaunchedTime" : "2015-03-16T19:25:36.515GMT", + "completionTime" : "2015-03-16T19:25:36.579GMT", "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, @@ -24,4 +28,4 @@ "name" : "my counter", "value" : "5050" } ] -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json index f2cb29b31c85..9b401b414f8d 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json @@ -3,14 +3,18 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.494GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 32, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -20,6 +24,19 @@ "bytesRead" : 49294, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 3842811, @@ -31,14 +48,18 @@ "index" : 1, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.502GMT", + "duration" : 350, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 31, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 0, @@ -48,6 +69,19 @@ "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 3934399, @@ -59,14 +93,18 @@ "index" : 2, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.503GMT", + "duration" : 348, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 32, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 348, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 2, @@ -76,6 +114,19 @@ "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 89885, @@ -87,14 +138,18 @@ "index" : 3, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 31, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 2, @@ -104,6 +159,19 @@ "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 1311694, @@ -115,14 +183,18 @@ "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 31, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -132,6 +204,19 @@ "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 83022, @@ -143,14 +228,18 @@ "index" : 5, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", + "duration" : 350, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 30, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -160,6 +249,19 @@ "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 3675510, @@ -171,14 +273,18 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", + "duration" : 351, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 29, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 351, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -188,6 +294,19 @@ "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 4016617, @@ -199,14 +318,18 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.506GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 31, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 0, @@ -216,6 +339,19 @@ "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 2579051, @@ -227,14 +363,18 @@ "index" : 8, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.914GMT", + "duration" : 80, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 80, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -244,6 +384,19 @@ "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 121551, @@ -255,14 +408,18 @@ "index" : 9, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.915GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 9, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -272,6 +429,19 @@ "bytesRead" : 60489, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 101664, @@ -283,14 +453,18 @@ "index" : 10, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.916GMT", + "duration" : 73, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 8, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 73, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -300,6 +474,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94709, @@ -311,14 +498,18 @@ "index" : 11, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.918GMT", + "duration" : 75, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 6, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 75, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -328,6 +519,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94507, @@ -339,14 +543,18 @@ "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", + "duration" : 77, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 9, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 77, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -356,6 +564,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 102476, @@ -367,14 +588,18 @@ "index" : 13, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.924GMT", + "duration" : 76, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 9, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -384,6 +609,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95004, @@ -395,14 +633,18 @@ "index" : 14, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.925GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 6, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -412,6 +654,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95646, @@ -423,14 +678,18 @@ "index" : 15, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.928GMT", + "duration" : 76, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -440,6 +699,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 602780, @@ -451,14 +723,18 @@ "index" : 16, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.001GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 10, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -468,6 +744,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 108320, @@ -479,14 +768,18 @@ "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", + "duration" : 91, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 11, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 91, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 1, @@ -496,6 +789,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 99944, @@ -507,14 +813,18 @@ "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", + "duration" : 92, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 10, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 92, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -524,6 +834,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100836, @@ -535,14 +858,18 @@ "index" : 19, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.012GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -552,10 +879,23 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95788, "recordsWritten" : 10 } } -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json index c3febc5fc944..2ebee66a6d7c 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json @@ -3,8 +3,10 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.515GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -15,20 +17,46 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 14, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, { "taskId" : 1, "index" : 1, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.521GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -39,20 +67,46 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 14, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, { "taskId" : 2, "index" : 2, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -63,20 +117,46 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 13, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, { "taskId" : 3, "index" : 3, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -87,20 +167,46 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 13, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, { "taskId" : 4, "index" : 4, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -111,20 +217,46 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 12, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, { "taskId" : 5, "index" : 5, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -135,20 +267,46 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 12, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, { "taskId" : 6, "index" : 6, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -159,20 +317,46 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 12, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, { "taskId" : 7, "index" : 7, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.524GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -183,11 +367,35 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 12, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json index 56d667d88917..965a31a4104c 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json @@ -3,8 +3,10 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.515GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -15,20 +17,46 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 14, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, { "taskId" : 1, "index" : 1, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.521GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -39,20 +67,46 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 14, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, { "taskId" : 2, "index" : 2, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -63,20 +117,46 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 13, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, { "taskId" : 3, "index" : 3, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -87,20 +167,46 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 13, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, { "taskId" : 4, "index" : 4, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -111,20 +217,46 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 12, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, { "taskId" : 5, "index" : 5, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.523GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -135,20 +267,46 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 12, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, { "taskId" : 6, "index" : 6, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.523GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -159,20 +317,46 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 12, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, { "taskId" : 7, "index" : 7, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.524GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -183,11 +367,35 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 12, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json index e5ec3bc4c712..31132e156937 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json @@ -3,14 +3,18 @@ "index" : 10, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.916GMT", + "duration" : 73, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 8, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 73, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -20,6 +24,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94709, @@ -31,14 +48,18 @@ "index" : 11, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.918GMT", + "duration" : 75, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 6, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 75, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -48,6 +69,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94507, @@ -59,14 +93,18 @@ "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", + "duration" : 77, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 9, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 77, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -76,6 +114,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 102476, @@ -87,14 +138,18 @@ "index" : 13, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.924GMT", + "duration" : 76, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 9, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -104,6 +159,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95004, @@ -115,14 +183,18 @@ "index" : 14, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.925GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 6, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -132,6 +204,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95646, @@ -143,14 +228,18 @@ "index" : 15, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.928GMT", + "duration" : 76, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -160,6 +249,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 602780, @@ -171,14 +273,18 @@ "index" : 16, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.001GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 10, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -188,6 +294,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 108320, @@ -199,14 +318,18 @@ "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", + "duration" : 91, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 11, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 91, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 1, @@ -216,6 +339,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 99944, @@ -227,14 +363,18 @@ "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", + "duration" : 92, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 10, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 92, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -244,6 +384,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100836, @@ -255,14 +408,18 @@ "index" : 19, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.012GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -272,6 +429,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95788, @@ -283,14 +453,18 @@ "index" : 20, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.014GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -300,6 +474,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 97716, @@ -311,14 +498,18 @@ "index" : 21, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.015GMT", + "duration" : 88, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 88, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -328,6 +519,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100270, @@ -339,14 +543,18 @@ "index" : 22, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.018GMT", + "duration" : 93, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 93, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -356,6 +564,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 143427, @@ -367,14 +588,18 @@ "index" : 23, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.031GMT", + "duration" : 65, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 65, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -384,6 +609,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 91844, @@ -395,14 +633,18 @@ "index" : 24, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.098GMT", + "duration" : 43, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 43, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 1, @@ -412,6 +654,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 157194, @@ -423,14 +678,18 @@ "index" : 25, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.103GMT", + "duration" : 49, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 49, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -440,6 +699,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94134, @@ -451,14 +723,18 @@ "index" : 26, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.105GMT", + "duration" : 38, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 6, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 38, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -468,6 +744,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 108213, @@ -479,14 +768,18 @@ "index" : 27, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.110GMT", + "duration" : 32, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 32, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -496,6 +789,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 102019, @@ -507,14 +813,18 @@ "index" : 28, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.113GMT", + "duration" : 29, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 29, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -524,6 +834,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 104299, @@ -535,14 +858,18 @@ "index" : 29, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.114GMT", + "duration" : 39, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 39, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -552,6 +879,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 114938, @@ -563,14 +903,18 @@ "index" : 30, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.118GMT", + "duration" : 34, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 4, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 34, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -580,6 +924,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 119770, @@ -591,14 +948,18 @@ "index" : 31, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.127GMT", + "duration" : 24, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 36, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 24, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -608,6 +969,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 92619, @@ -619,14 +993,18 @@ "index" : 32, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.148GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -636,6 +1014,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 89603, @@ -647,14 +1038,18 @@ "index" : 33, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.149GMT", + "duration" : 43, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 43, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -664,6 +1059,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 118329, @@ -675,14 +1083,18 @@ "index" : 34, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.156GMT", + "duration" : 27, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 6, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 27, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -692,6 +1104,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 127746, @@ -703,14 +1128,18 @@ "index" : 35, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.161GMT", + "duration" : 35, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 35, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -720,6 +1149,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 160963, @@ -731,14 +1173,18 @@ "index" : 36, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.164GMT", + "duration" : 29, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 29, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -748,6 +1194,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 123855, @@ -759,14 +1218,18 @@ "index" : 37, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.165GMT", + "duration" : 32, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 4, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 32, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -776,6 +1239,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 111869, @@ -787,14 +1263,18 @@ "index" : 38, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.166GMT", + "duration" : 31, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 31, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -804,6 +1284,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 131158, @@ -815,14 +1308,18 @@ "index" : 39, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.180GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -832,6 +1329,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 98748, @@ -843,14 +1353,18 @@ "index" : 40, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.197GMT", + "duration" : 14, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 4, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 14, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -860,6 +1374,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94792, @@ -871,14 +1398,18 @@ "index" : 41, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.200GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -888,6 +1419,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 90765, @@ -899,14 +1443,18 @@ "index" : 42, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.203GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 10, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -916,6 +1464,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 103713, @@ -927,14 +1488,18 @@ "index" : 43, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.204GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -944,6 +1509,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 171516, @@ -955,14 +1533,18 @@ "index" : 44, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.205GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -972,6 +1554,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 98293, @@ -983,14 +1578,18 @@ "index" : 45, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.206GMT", + "duration" : 19, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 19, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1000,6 +1599,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 92985, @@ -1011,14 +1623,18 @@ "index" : 46, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.210GMT", + "duration" : 31, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 1, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 31, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 6, "resultSerializationTime" : 0, @@ -1028,6 +1644,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 113322, @@ -1039,14 +1668,18 @@ "index" : 47, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.212GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1056,6 +1689,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 103015, @@ -1067,14 +1713,18 @@ "index" : 48, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.220GMT", + "duration" : 24, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 24, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 6, "resultSerializationTime" : 0, @@ -1084,6 +1734,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 139844, @@ -1095,14 +1758,18 @@ "index" : 49, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.223GMT", + "duration" : 23, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 7, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 23, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 6, "resultSerializationTime" : 0, @@ -1112,6 +1779,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94984, @@ -1123,14 +1803,18 @@ "index" : 50, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.240GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 4, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1140,6 +1824,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 90836, @@ -1151,14 +1848,18 @@ "index" : 51, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.242GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1168,6 +1869,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 96013, @@ -1179,14 +1893,18 @@ "index" : 52, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.243GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1196,6 +1914,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 89664, @@ -1207,14 +1938,18 @@ "index" : 53, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.244GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 6, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1224,6 +1959,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 92835, @@ -1235,14 +1983,18 @@ "index" : 54, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.244GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1252,6 +2004,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 90506, @@ -1263,14 +2028,18 @@ "index" : 55, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.246GMT", + "duration" : 21, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 4, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 21, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1280,6 +2049,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 108309, @@ -1291,14 +2073,18 @@ "index" : 56, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.249GMT", + "duration" : 20, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 20, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1308,6 +2094,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 90329, @@ -1319,14 +2118,18 @@ "index" : 57, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.257GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1336,6 +2139,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 96849, @@ -1347,14 +2163,18 @@ "index" : 58, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.263GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1364,6 +2184,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 97521, @@ -1375,14 +2208,18 @@ "index" : 59, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.265GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -1392,10 +2229,23 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100753, "recordsWritten" : 10 } } -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json index 5657123a2db1..6af1cfbeb8f7 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json @@ -3,14 +3,18 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", + "duration" : 351, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 29, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 351, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -20,6 +24,19 @@ "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 4016617, @@ -27,170 +44,272 @@ } } }, { - "taskId" : 5, - "index" : 5, + "taskId" : 1, + "index" : 1, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.505GMT", + "launchTime" : "2015-05-06T13:03:06.502GMT", + "duration" : 350, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 30, + "executorDeserializeTime" : 31, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3675510, + "writeTime" : 3934399, "recordsWritten" : 10 } } }, { - "taskId" : 1, - "index" : 1, + "taskId" : 5, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.502GMT", + "launchTime" : "2015-05-06T13:03:06.505GMT", + "duration" : 350, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 30, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3934399, + "writeTime" : 3675510, "recordsWritten" : 10 } } }, { - "taskId" : 4, - "index" : 4, + "taskId" : 0, + "index" : 0, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.504GMT", + "launchTime" : "2015-05-06T13:03:06.494GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 32, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 60488, + "bytesRead" : 49294, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 83022, + "writeTime" : 3842811, "recordsWritten" : 10 } } }, { - "taskId" : 7, - "index" : 7, + "taskId" : 3, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.506GMT", + "launchTime" : "2015-05-06T13:03:06.504GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 31, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 2579051, + "writeTime" : 1311694, "recordsWritten" : 10 } } }, { - "taskId" : 3, - "index" : 3, + "taskId" : 4, + "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 31, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 1311694, + "writeTime" : 83022, "recordsWritten" : 10 } } }, { - "taskId" : 0, - "index" : 0, + "taskId" : 7, + "index" : 7, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.494GMT", + "launchTime" : "2015-05-06T13:03:06.506GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 32, + "executorDeserializeTime" : 31, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 49294, + "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3842811, + "writeTime" : 2579051, "recordsWritten" : 10 } } @@ -199,14 +318,18 @@ "index" : 2, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.503GMT", + "duration" : 348, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 32, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 348, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 2, @@ -216,6 +339,19 @@ "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 89885, @@ -227,14 +363,18 @@ "index" : 22, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.018GMT", + "duration" : 93, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 93, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -244,6 +384,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 143427, @@ -255,14 +408,18 @@ "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", + "duration" : 92, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 10, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 92, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -272,6 +429,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100836, @@ -283,14 +453,18 @@ "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", + "duration" : 91, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 11, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 91, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 1, @@ -300,6 +474,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 99944, @@ -311,14 +498,18 @@ "index" : 21, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.015GMT", + "duration" : 88, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 88, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -328,6 +519,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100270, @@ -335,46 +539,67 @@ } } }, { - "taskId" : 16, - "index" : 16, + "taskId" : 9, + "index" : 9, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.001GMT", + "launchTime" : "2015-05-06T13:03:06.915GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 10, + "executorDeserializeTime" : 9, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, + "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 70564, + "bytesRead" : 60489, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 108320, + "writeTime" : 101664, "recordsWritten" : 10 } } }, { - "taskId" : 19, - "index" : 19, + "taskId" : 16, + "index" : 16, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.012GMT", + "launchTime" : "2015-05-06T13:03:07.001GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 5, + "executorDeserializeTime" : 10, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -384,55 +609,89 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95788, + "writeTime" : 108320, "recordsWritten" : 10 } } }, { - "taskId" : 9, - "index" : 9, + "taskId" : 19, + "index" : 19, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.915GMT", + "launchTime" : "2015-05-06T13:03:07.012GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 9, + "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, + "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 60489, + "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 101664, + "writeTime" : 95788, "recordsWritten" : 10 } } }, { - "taskId" : 20, - "index" : 20, + "taskId" : 14, + "index" : 14, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.014GMT", + "launchTime" : "2015-05-06T13:03:06.925GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 3, + "executorDeserializeTime" : 6, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, + "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, @@ -440,27 +699,44 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 97716, + "writeTime" : 95646, "recordsWritten" : 10 } } }, { - "taskId" : 14, - "index" : 14, + "taskId" : 20, + "index" : 20, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.925GMT", + "launchTime" : "2015-05-06T13:03:07.014GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 6, + "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, + "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, @@ -468,9 +744,22 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95646, + "writeTime" : 97716, "recordsWritten" : 10 } } @@ -479,14 +768,18 @@ "index" : 8, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.914GMT", + "duration" : 80, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 80, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -496,6 +789,19 @@ "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 121551, @@ -507,14 +813,18 @@ "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", + "duration" : 77, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 9, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 77, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -524,6 +834,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 102476, @@ -535,14 +858,18 @@ "index" : 13, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.924GMT", + "duration" : 76, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 9, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -552,10 +879,23 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95004, "recordsWritten" : 10 } } -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json index 5657123a2db1..6af1cfbeb8f7 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json @@ -3,14 +3,18 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", + "duration" : 351, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 29, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 351, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, @@ -20,6 +24,19 @@ "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 4016617, @@ -27,170 +44,272 @@ } } }, { - "taskId" : 5, - "index" : 5, + "taskId" : 1, + "index" : 1, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.505GMT", + "launchTime" : "2015-05-06T13:03:06.502GMT", + "duration" : 350, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 30, + "executorDeserializeTime" : 31, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3675510, + "writeTime" : 3934399, "recordsWritten" : 10 } } }, { - "taskId" : 1, - "index" : 1, + "taskId" : 5, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.502GMT", + "launchTime" : "2015-05-06T13:03:06.505GMT", + "duration" : 350, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 30, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3934399, + "writeTime" : 3675510, "recordsWritten" : 10 } } }, { - "taskId" : 4, - "index" : 4, + "taskId" : 0, + "index" : 0, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.504GMT", + "launchTime" : "2015-05-06T13:03:06.494GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 32, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 60488, + "bytesRead" : 49294, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 83022, + "writeTime" : 3842811, "recordsWritten" : 10 } } }, { - "taskId" : 7, - "index" : 7, + "taskId" : 3, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.506GMT", + "launchTime" : "2015-05-06T13:03:06.504GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 31, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 2579051, + "writeTime" : 1311694, "recordsWritten" : 10 } } }, { - "taskId" : 3, - "index" : 3, + "taskId" : 4, + "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 31, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 1311694, + "writeTime" : 83022, "recordsWritten" : 10 } } }, { - "taskId" : 0, - "index" : 0, + "taskId" : 7, + "index" : 7, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.494GMT", + "launchTime" : "2015-05-06T13:03:06.506GMT", + "duration" : 349, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 32, + "executorDeserializeTime" : 31, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 49294, + "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3842811, + "writeTime" : 2579051, "recordsWritten" : 10 } } @@ -199,14 +318,18 @@ "index" : 2, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.503GMT", + "duration" : 348, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 32, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 348, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, "resultSerializationTime" : 2, @@ -216,6 +339,19 @@ "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 89885, @@ -227,14 +363,18 @@ "index" : 22, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.018GMT", + "duration" : 93, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 93, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -244,6 +384,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 143427, @@ -255,14 +408,18 @@ "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", + "duration" : 92, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 10, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 92, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -272,6 +429,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100836, @@ -283,14 +453,18 @@ "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", + "duration" : 91, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 11, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 91, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 5, "resultSerializationTime" : 1, @@ -300,6 +474,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 99944, @@ -311,14 +498,18 @@ "index" : 21, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.015GMT", + "duration" : 88, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 88, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -328,6 +519,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100270, @@ -335,46 +539,67 @@ } } }, { - "taskId" : 16, - "index" : 16, + "taskId" : 9, + "index" : 9, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.001GMT", + "launchTime" : "2015-05-06T13:03:06.915GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 10, + "executorDeserializeTime" : 9, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, + "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 70564, + "bytesRead" : 60489, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 108320, + "writeTime" : 101664, "recordsWritten" : 10 } } }, { - "taskId" : 19, - "index" : 19, + "taskId" : 16, + "index" : 16, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.012GMT", + "launchTime" : "2015-05-06T13:03:07.001GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 5, + "executorDeserializeTime" : 10, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 5, "resultSerializationTime" : 0, @@ -384,55 +609,89 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95788, + "writeTime" : 108320, "recordsWritten" : 10 } } }, { - "taskId" : 9, - "index" : 9, + "taskId" : 19, + "index" : 19, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.915GMT", + "launchTime" : "2015-05-06T13:03:07.012GMT", + "duration" : 84, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 9, + "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, + "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 60489, + "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 101664, + "writeTime" : 95788, "recordsWritten" : 10 } } }, { - "taskId" : 20, - "index" : 20, + "taskId" : 14, + "index" : 14, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.014GMT", + "launchTime" : "2015-05-06T13:03:06.925GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 3, + "executorDeserializeTime" : 6, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, + "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, @@ -440,27 +699,44 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 97716, + "writeTime" : 95646, "recordsWritten" : 10 } } }, { - "taskId" : 14, - "index" : 14, + "taskId" : 20, + "index" : 20, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.925GMT", + "launchTime" : "2015-05-06T13:03:07.014GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 6, + "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, + "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, @@ -468,9 +744,22 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95646, + "writeTime" : 97716, "recordsWritten" : 10 } } @@ -479,14 +768,18 @@ "index" : 8, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.914GMT", + "duration" : 80, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 80, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -496,6 +789,19 @@ "bytesRead" : 60488, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 121551, @@ -507,14 +813,18 @@ "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", + "duration" : 77, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 9, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 77, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -524,6 +834,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 102476, @@ -535,14 +858,18 @@ "index" : 13, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.924GMT", + "duration" : 76, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 9, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, + "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -552,10 +879,23 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95004, "recordsWritten" : 10 } } -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json index 72fe017e9f85..c26daf4b8d7b 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json @@ -3,14 +3,18 @@ "index" : 40, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.197GMT", + "duration" : 14, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 4, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 14, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -20,6 +24,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94792, @@ -27,46 +44,67 @@ } } }, { - "taskId" : 86, - "index" : 86, + "taskId" : 41, + "index" : 41, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.374GMT", + "launchTime" : "2015-05-06T13:03:07.200GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 3, + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95848, + "writeTime" : 90765, "recordsWritten" : 10 } } }, { - "taskId" : 41, - "index" : 41, + "taskId" : 43, + "index" : 43, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.200GMT", + "launchTime" : "2015-05-06T13:03:07.204GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -76,25 +114,42 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 90765, + "writeTime" : 171516, "recordsWritten" : 10 } } }, { - "taskId" : 68, - "index" : 68, + "taskId" : 57, + "index" : 57, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.306GMT", + "launchTime" : "2015-05-06T13:03:07.257GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, + "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -104,9 +159,22 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 101750, + "writeTime" : 96849, "recordsWritten" : 10 } } @@ -115,14 +183,18 @@ "index" : 58, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.263GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -132,6 +204,19 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 97521, @@ -139,18 +224,22 @@ } } }, { - "taskId" : 43, - "index" : 43, + "taskId" : 68, + "index" : 68, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.204GMT", + "launchTime" : "2015-05-06T13:03:07.306GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -160,53 +249,87 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 171516, + "writeTime" : 101750, "recordsWritten" : 10 } } }, { - "taskId" : 57, - "index" : 57, + "taskId" : 86, + "index" : 86, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.257GMT", + "launchTime" : "2015-05-06T13:03:07.374GMT", + "duration" : 16, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 16, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 96849, + "writeTime" : 95848, "recordsWritten" : 10 } } }, { - "taskId" : 59, - "index" : 59, + "taskId" : 32, + "index" : 32, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.265GMT", + "launchTime" : "2015-05-06T13:03:07.148GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -216,25 +339,42 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 100753, + "writeTime" : 89603, "recordsWritten" : 10 } } }, { - "taskId" : 32, - "index" : 32, + "taskId" : 39, + "index" : 39, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.148GMT", + "launchTime" : "2015-05-06T13:03:07.180GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 3, + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -244,25 +384,42 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 89603, + "writeTime" : 98748, "recordsWritten" : 10 } } }, { - "taskId" : 87, - "index" : 87, + "taskId" : 42, + "index" : 42, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.374GMT", + "launchTime" : "2015-05-06T13:03:07.203GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 12, + "executorDeserializeTime" : 10, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -272,55 +429,89 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 102159, + "writeTime" : 103713, "recordsWritten" : 10 } } }, { - "taskId" : 99, - "index" : 99, + "taskId" : 51, + "index" : 51, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.426GMT", + "launchTime" : "2015-05-06T13:03:07.242GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 70565, + "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 133964, + "writeTime" : 96013, "recordsWritten" : 10 } } }, { - "taskId" : 63, - "index" : 63, + "taskId" : 59, + "index" : 59, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.276GMT", + "launchTime" : "2015-05-06T13:03:07.265GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 20, + "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, + "executorCpuTime" : 0, "resultSize" : 2065, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, @@ -328,27 +519,44 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 102779, + "writeTime" : 100753, "recordsWritten" : 10 } } }, { - "taskId" : 90, - "index" : 90, + "taskId" : 63, + "index" : 63, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.385GMT", + "launchTime" : "2015-05-06T13:03:07.276GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, + "executorDeserializeTime" : 20, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, + "executorCpuTime" : 0, "resultSize" : 2065, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, @@ -356,25 +564,42 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 98472, + "writeTime" : 102779, "recordsWritten" : 10 } } }, { - "taskId" : 39, - "index" : 39, + "taskId" : 87, + "index" : 87, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.180GMT", + "launchTime" : "2015-05-06T13:03:07.374GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, + "executorDeserializeTime" : 12, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -384,25 +609,42 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 98748, + "writeTime" : 102159, "recordsWritten" : 10 } } }, { - "taskId" : 42, - "index" : 42, + "taskId" : 90, + "index" : 90, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.203GMT", + "launchTime" : "2015-05-06T13:03:07.385GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 10, + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -412,53 +654,87 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 103713, + "writeTime" : 98472, "recordsWritten" : 10 } } }, { - "taskId" : 51, - "index" : 51, + "taskId" : 99, + "index" : 99, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.242GMT", + "launchTime" : "2015-05-06T13:03:07.426GMT", + "duration" : 17, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 17, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { - "bytesRead" : 70564, + "bytesRead" : 70565, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 96013, + "writeTime" : 133964, "recordsWritten" : 10 } } }, { - "taskId" : 50, - "index" : 50, + "taskId" : 44, + "index" : 44, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.240GMT", + "launchTime" : "2015-05-06T13:03:07.205GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 4, + "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -468,25 +744,42 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 90836, + "writeTime" : 98293, "recordsWritten" : 10 } } }, { - "taskId" : 53, - "index" : 53, + "taskId" : 47, + "index" : 47, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.244GMT", + "launchTime" : "2015-05-06T13:03:07.212GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 6, + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -496,25 +789,42 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 92835, + "writeTime" : 103015, "recordsWritten" : 10 } } }, { - "taskId" : 44, - "index" : 44, + "taskId" : 50, + "index" : 50, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.205GMT", + "launchTime" : "2015-05-06T13:03:07.240GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 3, + "executorDeserializeTime" : 4, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, + "executorCpuTime" : 0, "resultSize" : 2065, "jvmGcTime" : 0, "resultSerializationTime" : 0, @@ -524,27 +834,44 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 98293, + "writeTime" : 90836, "recordsWritten" : 10 } } }, { - "taskId" : 80, - "index" : 80, + "taskId" : 52, + "index" : 52, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.341GMT", + "launchTime" : "2015-05-06T13:03:07.243GMT", + "duration" : 18, "executorId" : "driver", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 13, + "executorDeserializeTime" : 5, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 18, + "executorCpuTime" : 0, "resultSize" : 2065, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, @@ -552,10 +879,23 @@ "bytesRead" : 70564, "recordsRead" : 10000 }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 98069, + "writeTime" : 89664, "recordsWritten" : 10 } } -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json index bc3c302813de..f8e27703c0de 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json @@ -1,7 +1,9 @@ { "quantiles" : [ 0.01, 0.5, 0.99 ], "executorDeserializeTime" : [ 1.0, 3.0, 36.0 ], + "executorDeserializeCpuTime" : [ 0.0, 0.0, 0.0 ], "executorRunTime" : [ 16.0, 28.0, 351.0 ], + "executorCpuTime" : [ 0.0, 0.0, 0.0 ], "resultSize" : [ 2010.0, 2065.0, 2065.0 ], "jvmGcTime" : [ 0.0, 0.0, 7.0 ], "resultSerializationTime" : [ 0.0, 0.0, 2.0 ], @@ -11,9 +13,23 @@ "bytesRead" : [ 60488.0, 70564.0, 70565.0 ], "recordsRead" : [ 10000.0, 10000.0, 10000.0 ] }, + "outputMetrics" : { + "bytesWritten" : [ 0.0, 0.0, 0.0 ], + "recordsWritten" : [ 0.0, 0.0, 0.0 ] + }, + "shuffleReadMetrics" : { + "readBytes" : [ 0.0, 0.0, 0.0 ], + "readRecords" : [ 0.0, 0.0, 0.0 ], + "remoteBlocksFetched" : [ 0.0, 0.0, 0.0 ], + "localBlocksFetched" : [ 0.0, 0.0, 0.0 ], + "fetchWaitTime" : [ 0.0, 0.0, 0.0 ], + "remoteBytesRead" : [ 0.0, 0.0, 0.0 ], + "remoteBytesReadToDisk" : [ 0.0, 0.0, 0.0 ], + "totalBlocksFetched" : [ 0.0, 0.0, 0.0 ] + }, "shuffleWriteMetrics" : { "writeBytes" : [ 1710.0, 1710.0, 1710.0 ], "writeRecords" : [ 10.0, 10.0, 10.0 ], "writeTime" : [ 89437.0, 102159.0, 4016617.0 ] } -} \ No newline at end of file +} diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json index e084c839f1d5..a28bda16a956 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json @@ -1,12 +1,22 @@ { "quantiles" : [ 0.05, 0.25, 0.5, 0.75, 0.95 ], "executorDeserializeTime" : [ 1.0, 2.0, 2.0, 2.0, 3.0 ], + "executorDeserializeCpuTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], "executorRunTime" : [ 30.0, 74.0, 75.0, 76.0, 79.0 ], + "executorCpuTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], "resultSize" : [ 1034.0, 1034.0, 1034.0, 1034.0, 1034.0 ], "jvmGcTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], "resultSerializationTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], "memoryBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], "diskBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "inputMetrics" : { + "bytesRead" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "recordsRead" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ] + }, + "outputMetrics" : { + "bytesWritten" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "recordsWritten" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ] + }, "shuffleReadMetrics" : { "readBytes" : [ 17100.0, 17100.0, 17100.0, 17100.0, 17100.0 ], "readRecords" : [ 100.0, 100.0, 100.0, 100.0, 100.0 ], @@ -14,6 +24,12 @@ "localBlocksFetched" : [ 100.0, 100.0, 100.0, 100.0, 100.0 ], "fetchWaitTime" : [ 0.0, 0.0, 0.0, 1.0, 1.0 ], "remoteBytesRead" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "remoteBytesReadToDisk" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], "totalBlocksFetched" : [ 100.0, 100.0, 100.0, 100.0, 100.0 ] + }, + "shuffleWriteMetrics" : { + "writeBytes" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "writeRecords" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "writeTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ] } -} \ No newline at end of file +} diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json index 6ac7811ce691..ede3eaed1d1d 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json @@ -1,7 +1,9 @@ { "quantiles" : [ 0.05, 0.25, 0.5, 0.75, 0.95 ], "executorDeserializeTime" : [ 2.0, 2.0, 3.0, 7.0, 31.0 ], + "executorDeserializeCpuTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], "executorRunTime" : [ 16.0, 18.0, 28.0, 49.0, 349.0 ], + "executorCpuTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], "resultSize" : [ 2010.0, 2065.0, 2065.0, 2065.0, 2065.0 ], "jvmGcTime" : [ 0.0, 0.0, 0.0, 5.0, 7.0 ], "resultSerializationTime" : [ 0.0, 0.0, 0.0, 0.0, 1.0 ], @@ -11,9 +13,23 @@ "bytesRead" : [ 60488.0, 70564.0, 70564.0, 70564.0, 70564.0 ], "recordsRead" : [ 10000.0, 10000.0, 10000.0, 10000.0, 10000.0 ] }, + "outputMetrics" : { + "bytesWritten" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "recordsWritten" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ] + }, + "shuffleReadMetrics" : { + "readBytes" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "readRecords" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "remoteBlocksFetched" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "localBlocksFetched" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "fetchWaitTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "remoteBytesRead" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "remoteBytesReadToDisk" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "totalBlocksFetched" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ] + }, "shuffleWriteMetrics" : { "writeBytes" : [ 1710.0, 1710.0, 1710.0, 1710.0, 1710.0 ], "writeRecords" : [ 10.0, 10.0, 10.0, 10.0, 10.0 ], "writeTime" : [ 90329.0, 95848.0, 102159.0, 121551.0, 2579051.0 ] } -} \ No newline at end of file +} diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json index 32d5731676ad..44b5f66efe33 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json @@ -6,6 +6,10 @@ "numCompleteTasks" : 8, "numFailedTasks" : 0, "executorRunTime" : 120, + "executorCpuTime" : 0, + "submissionTime" : "2015-03-16T19:25:36.103GMT", + "firstTaskLaunchedTime" : "2015-03-16T19:25:36.515GMT", + "completionTime" : "2015-03-16T19:25:36.579GMT", "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, @@ -25,154 +29,310 @@ "value" : "5050" } ], "tasks" : { - "2" : { - "taskId" : 2, - "index" : 2, + "0" : { + "taskId" : 0, + "index" : 0, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.522GMT", + "launchTime" : "2015-03-16T19:25:36.515GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "378", - "value" : "378" + "update" : "78", + "value" : "5050" } ], "taskMetrics" : { - "executorDeserializeTime" : 13, + "executorDeserializeTime" : 14, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, - "5" : { - "taskId" : 5, - "index" : 5, + "1" : { + "taskId" : 1, + "index" : 1, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.523GMT", + "launchTime" : "2015-03-16T19:25:36.521GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "897", - "value" : "3750" + "update" : "247", + "value" : "2175" } ], "taskMetrics" : { - "executorDeserializeTime" : 12, + "executorDeserializeTime" : 14, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, - "4" : { - "taskId" : 4, - "index" : 4, + "2" : { + "taskId" : 2, + "index" : 2, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "678", - "value" : "2853" + "update" : "378", + "value" : "378" } ], "taskMetrics" : { - "executorDeserializeTime" : 12, + "executorDeserializeTime" : 13, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, - "resultSerializationTime" : 1, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, - "7" : { - "taskId" : 7, - "index" : 7, + "3" : { + "taskId" : 3, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.524GMT", + "launchTime" : "2015-03-16T19:25:36.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "1222", - "value" : "4972" + "update" : "572", + "value" : "950" } ], "taskMetrics" : { - "executorDeserializeTime" : 12, + "executorDeserializeTime" : 13, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, - "1" : { - "taskId" : 1, - "index" : 1, + "4" : { + "taskId" : 4, + "index" : 4, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.521GMT", + "launchTime" : "2015-03-16T19:25:36.522GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "247", - "value" : "2175" + "update" : "678", + "value" : "2853" } ], "taskMetrics" : { - "executorDeserializeTime" : 14, + "executorDeserializeTime" : 12, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, - "3" : { - "taskId" : 3, - "index" : 3, + "5" : { + "taskId" : 5, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.522GMT", + "launchTime" : "2015-03-16T19:25:36.523GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "572", - "value" : "950" + "update" : "897", + "value" : "3750" } ], "taskMetrics" : { - "executorDeserializeTime" : 13, + "executorDeserializeTime" : 12, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, "6" : { @@ -180,8 +340,10 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { @@ -192,37 +354,87 @@ } ], "taskMetrics" : { "executorDeserializeTime" : 12, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } }, - "0" : { - "taskId" : 0, - "index" : 0, + "7" : { + "taskId" : 7, + "index" : 7, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.515GMT", + "launchTime" : "2015-03-16T19:25:36.524GMT", + "duration" : 15, "executorId" : "", "host" : "localhost", + "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "78", - "value" : "5050" + "update" : "1222", + "value" : "4972" } ], "taskMetrics" : { - "executorDeserializeTime" : 14, + "executorDeserializeTime" : 12, + "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, + "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0 + "diskBytesSpilled" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 0, + "recordsWritten" : 0 + } } } }, @@ -239,4 +451,4 @@ "diskBytesSpilled" : 0 } } -} \ No newline at end of file +} diff --git a/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json index cab4750270df..3d7407004d26 100644 --- a/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json @@ -6,7 +6,7 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, @@ -20,7 +20,7 @@ "numTasks" : 16, "numActiveTasks" : 0, "numCompletedTasks" : 15, - "numSkippedTasks" : 15, + "numSkippedTasks" : 0, "numFailedTasks" : 1, "numActiveStages" : 0, "numCompletedStages" : 1, @@ -34,10 +34,10 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, "numFailedStages" : 0 -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json index 6fd25befbf7e..6a9bafd6b219 100644 --- a/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json @@ -6,7 +6,7 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, @@ -20,10 +20,10 @@ "numTasks" : 8, "numActiveTasks" : 0, "numCompletedTasks" : 8, - "numSkippedTasks" : 8, + "numSkippedTasks" : 0, "numFailedTasks" : 0, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, "numFailedStages" : 0 -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager new file mode 100644 index 000000000000..cf8565c74e95 --- /dev/null +++ b/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager @@ -0,0 +1,3 @@ +org.apache.spark.scheduler.DummyExternalClusterManager +org.apache.spark.scheduler.MockExternalClusterManager +org.apache.spark.DummyLocalExternalClusterManager diff --git a/core/src/test/resources/TestUDTF.jar b/core/src/test/resources/TestUDTF.jar new file mode 100644 index 000000000000..514f2d5d26fd Binary files /dev/null and b/core/src/test/resources/TestUDTF.jar differ diff --git a/core/src/test/resources/fairscheduler-with-invalid-data.xml b/core/src/test/resources/fairscheduler-with-invalid-data.xml new file mode 100644 index 000000000000..a4d8d07b67ce --- /dev/null +++ b/core/src/test/resources/fairscheduler-with-invalid-data.xml @@ -0,0 +1,80 @@ + + + + + + INVALID_MIN_SHARE + 2 + FAIR + + + 1 + INVALID_WEIGHT + FAIR + + + 3 + 2 + INVALID_SCHEDULING_MODE + + + 2 + 1 + fair + + + 1 + 2 + NONE + + + + 2 + FAIR + + + 1 + + FAIR + + + 3 + 2 + + + + + 3 + FAIR + + + 2 + + FAIR + + + 2 + 2 + + + + 3 + 2 + FAIR + + diff --git a/core/src/test/resources/fairscheduler-with-valid-data.xml b/core/src/test/resources/fairscheduler-with-valid-data.xml new file mode 100644 index 000000000000..3d882331835c --- /dev/null +++ b/core/src/test/resources/fairscheduler-with-valid-data.xml @@ -0,0 +1,35 @@ + + + + + + 3 + 1 + FIFO + + + 4 + 2 + FAIR + + + 2 + 3 + FAIR + + \ No newline at end of file diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index a54d27de91ed..fb9d9851cb4d 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -33,5 +33,4 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%t: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN -org.spark-project.jetty.LEVEL=WARN +log4j.logger.org.spark_project.jetty=WARN diff --git a/core/src/test/resources/spark-events/app-20161115172038-0000 b/core/src/test/resources/spark-events/app-20161115172038-0000 new file mode 100755 index 000000000000..3af0451d0c39 --- /dev/null +++ b/core/src/test/resources/spark-events/app-20161115172038-0000 @@ -0,0 +1,75 @@ +{"Event":"SparkListenerLogStart","Spark Version":"2.1.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"172.22.0.111","Port":64527},"Maximum Memory":384093388,"Timestamp":1479252038836} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","Java Version":"1.8.0_92 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.blacklist.task.maxTaskAttemptsPerExecutor":"3","spark.blacklist.enabled":"TRUE","spark.driver.host":"172.22.0.111","spark.blacklist.task.maxTaskAttemptsPerNode":"3","spark.eventLog.enabled":"TRUE","spark.driver.port":"64511","spark.repl.class.uri":"spark://172.22.0.111:64511/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-f09ef9e2-7f15-433f-a5d1-30138d8764ca/repl-28d60911-dbc3-465f-b7b3-ee55c071595e","spark.app.name":"Spark shell","spark.blacklist.stage.maxFailedExecutorsPerNode":"3","spark.scheduler.mode":"FIFO","spark.eventLog.overwrite":"TRUE","spark.blacklist.stage.maxFailedTasksPerExecutor":"3","spark.executor.id":"driver","spark.blacklist.application.maxFailedExecutorsPerNode":"2","spark.submit.deployMode":"client","spark.master":"local-cluster[4,4,1024]","spark.home":"/Users/Jose/IdeaProjects/spark","spark.eventLog.dir":"/Users/jose/logs","spark.sql.catalogImplementation":"in-memory","spark.eventLog.compress":"FALSE","spark.blacklist.application.maxFailedTasksPerExecutor":"1","spark.blacklist.timeout":"10000","spark.app.id":"app-20161115172038-0000","spark.task.maxFailures":"4"},"System Properties":{"java.io.tmpdir":"/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/Jose","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib","user.dir":"/Users/Jose/IdeaProjects/spark","java.library.path":"/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.92-b14","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_92-b14","java.vm.info":"mixed mode","java.ext.dirs":"/Users/Jose/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","io.netty.maxDirectMemory":"0","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.11.6","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"jose","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=10000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","java.version":"1.8.0_92","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/arpack_combined_all-0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math3-3.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-scala_2.11-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-proxy-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/conf/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-6.1.26.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/jars/*":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/kryo-shaded-3.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/log4j-1.2.17.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-core-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-jvm-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/minlog-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-3.8.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/shapeless_2.11-2.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-common_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-streaming_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/validation-api-1.1.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"app-20161115172038-0000","Timestamp":1479252037079,"User":"jose"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479252042589,"Executor ID":"2","Executor Info":{"Host":"172.22.0.111","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout","stderr":"http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr"}}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479252042593,"Executor ID":"0","Executor Info":{"Host":"172.22.0.111","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout","stderr":"http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr"}}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479252042629,"Executor ID":"1","Executor Info":{"Host":"172.22.0.111","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stdout","stderr":"http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"172.22.0.111","Port":64540},"Maximum Memory":384093388,"Timestamp":1479252042687} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"172.22.0.111","Port":64539},"Maximum Memory":384093388,"Timestamp":1479252042689} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"172.22.0.111","Port":64541},"Maximum Memory":384093388,"Timestamp":1479252042692} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479252042711,"Executor ID":"3","Executor Info":{"Host":"172.22.0.111","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stdout","stderr":"http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"172.22.0.111","Port":64543},"Maximum Memory":384093388,"Timestamp":1479252042759} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1479252043855,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.(:35)\n$line16.$read$$iw$$iw$$iw$$iw.(:37)\n$line16.$read$$iw$$iw$$iw.(:39)\n$line16.$read$$iw$$iw.(:41)\n$line16.$read$$iw.(:43)\n$line16.$read.(:45)\n$line16.$read$.(:49)\n$line16.$read$.()\n$line16.$eval$.$print$lzycompute(:7)\n$line16.$eval$.$print(:6)\n$line16.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Accumulables":[]}],"Stage IDs":[0],"Properties":{}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.(:35)\n$line16.$read$$iw$$iw$$iw$$iw.(:37)\n$line16.$read$$iw$$iw$$iw.(:39)\n$line16.$read$$iw$$iw.(:41)\n$line16.$read$$iw.(:43)\n$line16.$read.(:45)\n$line16.$read$.(:49)\n$line16.$read$.()\n$line16.$eval$.$print$lzycompute(:7)\n$line16.$eval$.$print(:6)\n$line16.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Accumulables":[]},"Properties":{}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1479252044021,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1479252044052,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1479252044052,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1479252044053,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1479252044054,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1479252044055,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1479252044055,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1479252044056,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1479252044056,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1479252044057,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1479252044058,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1479252044058,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1479252044059,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1479252044060,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1479252044064,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1479252044065,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1479252044059,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044653,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":499,"Value":499,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":52390000,"Value":52390000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":18,"Value":18,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":7909000,"Value":7909000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":1123,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":21,"Value":21,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":499,"Executor Deserialize CPU Time":52390000,"Executor Run Time":18,"Executor CPU Time":7909000,"Result Size":1123,"JVM GC Time":21,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1479252044054,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044657,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":508,"Value":1007,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":36827000,"Value":89217000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":18,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":4333000,"Value":12242000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":2246,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":21,"Value":42,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":508,"Executor Deserialize CPU Time":36827000,"Executor Run Time":18,"Executor CPU Time":4333000,"Result Size":1123,"JVM GC Time":21,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1479252044056,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044658,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":509,"Value":1516,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":44100000,"Value":133317000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":17,"Value":53,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":11340000,"Value":23582000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":3369,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":21,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":509,"Executor Deserialize CPU Time":44100000,"Executor Run Time":17,"Executor CPU Time":11340000,"Result Size":1123,"JVM GC Time":21,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1479252044021,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044692,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":511,"Value":2027,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":227762000,"Value":361079000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":16,"Value":69,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":3631000,"Value":27213000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1938,"Value":5307,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":21,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":511,"Executor Deserialize CPU Time":227762000,"Executor Run Time":16,"Executor CPU Time":3631000,"Result Size":1938,"JVM GC Time":21,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":495,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1479252044055,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044720,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":495,"Value":564,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Value":114,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":495,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":30,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":494,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1479252044052,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044727,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":494,"Value":1058,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Value":144,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":494,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":30,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":494,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1479252044060,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044729,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":494,"Value":1552,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Value":174,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":494,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":30,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":13,"Attempt":1,"Launch Time":1479252044731,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":1,"Launch Time":1479252044731,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":5,"Attempt":1,"Launch Time":1479252044732,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1479252044058,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044736,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Value":2003,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Value":206,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":451,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":32,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":11,"Attempt":1,"Launch Time":1479252044736,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":446,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1479252044065,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044737,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":446,"Value":2449,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Value":238,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":446,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":32,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":15,"Attempt":1,"Launch Time":1479252044737,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":448,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1479252044056,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044741,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":448,"Value":2897,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Value":270,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":448,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":32,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":7,"Attempt":1,"Launch Time":1479252044742,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044752,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":1,"Launch Time":1479252044731,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044748,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":2035,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3655000,"Value":364734000,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":899000,"Value":28112000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":884,"Value":6191,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":3655000,"Executor Run Time":0,"Executor CPU Time":899000,"Result Size":884,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":19,"Index":11,"Attempt":1,"Launch Time":1479252044736,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044749,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":2899,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":2,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":11,"Attempt":2,"Launch Time":1479252044749,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":7,"Attempt":1,"Launch Time":1479252044742,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044752,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":2038,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3566000,"Value":368300000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":2900,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1004000,"Value":29116000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":7154,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3566000,"Executor Run Time":1,"Executor CPU Time":1004000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":10,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":20,"Index":15,"Attempt":1,"Launch Time":1479252044737,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044756,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":10,"Value":2910,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":10,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":15,"Attempt":2,"Launch Time":1479252044756,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":11,"Attempt":2,"Launch Time":1479252044749,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044759,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":2042,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3720000,"Value":372020000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":2911,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1009000,"Value":30125000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":8117,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3720000,"Executor Run Time":1,"Executor CPU Time":1009000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":5,"Attempt":1,"Launch Time":1479252044732,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044760,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":2047,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4303000,"Value":376323000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":2913,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":999000,"Value":31124000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":9080,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4303000,"Executor Run Time":2,"Executor CPU Time":999000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":15,"Attempt":2,"Launch Time":1479252044756,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044768,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":2053,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4946000,"Value":381269000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":2914,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1176000,"Value":32300000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":10043,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":4946000,"Executor Run Time":1,"Executor CPU Time":1176000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":13,"Attempt":1,"Launch Time":1479252044731,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044775,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":2060,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3406000,"Value":384675000,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1007000,"Value":33307000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":971,"Value":11014,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":3406000,"Executor Run Time":0,"Executor CPU Time":1007000,"Result Size":971,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":456,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1479252044053,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044778,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":456,"Value":3370,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Value":302,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":456,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":32,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":3,"Attempt":1,"Launch Time":1479252044778,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":503,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1479252044057,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044789,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":503,"Value":3873,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Value":332,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":503,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":30,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":9,"Attempt":1,"Launch Time":1479252044789,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":3,"Attempt":1,"Launch Time":1479252044778,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044791,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":2065,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2950000,"Value":387625000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3875,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":822000,"Value":34129000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":11977,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":2950000,"Executor Run Time":2,"Executor CPU Time":822000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":9,"Attempt":1,"Launch Time":1479252044789,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044798,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":2068,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2604000,"Value":390229000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3876,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":845000,"Value":34974000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":12940,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2604000,"Executor Run Time":1,"Executor CPU Time":845000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1479252044055,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044920,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":784,"Value":2852,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":56180000,"Value":446409000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":24,"Value":3900,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":6046000,"Value":41020000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":13976,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":350,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":784,"Executor Deserialize CPU Time":56180000,"Executor Run Time":24,"Executor CPU Time":6046000,"Result Size":1036,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1479252044052,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044921,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":789,"Value":3641,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":34766000,"Value":481175000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":22,"Value":3922,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":8189000,"Value":49209000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":15012,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":368,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":789,"Executor Deserialize CPU Time":34766000,"Executor Run Time":22,"Executor CPU Time":8189000,"Result Size":1036,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1479252044064,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044921,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":777,"Value":4418,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":29960000,"Value":511135000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":24,"Value":3946,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":9708000,"Value":58917000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":16048,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":386,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":777,"Executor Deserialize CPU Time":29960000,"Executor Run Time":24,"Executor CPU Time":9708000,"Result Size":1036,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1479252044058,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044924,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":791,"Value":5209,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":266560000,"Value":777695000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":16,"Value":3962,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":5884000,"Value":64801000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1851,"Value":17899,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":404,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":791,"Executor Deserialize CPU Time":266560000,"Executor Run Time":16,"Executor CPU Time":5884000,"Result Size":1851,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.(:35)\n$line16.$read$$iw$$iw$$iw$$iw.(:37)\n$line16.$read$$iw$$iw$$iw.(:39)\n$line16.$read$$iw$$iw.(:41)\n$line16.$read$$iw.(:43)\n$line16.$read.(:45)\n$line16.$read$.(:49)\n$line16.$read$.()\n$line16.$eval$.$print$lzycompute(:7)\n$line16.$eval$.$print(:6)\n$line16.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Submission Time":1479252044017,"Completion Time":1479252044926,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Value":3962,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Value":404,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Value":17899,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Value":777695000,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Value":64801000,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Value":6,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Value":5209,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1479252044931,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorBlacklisted","time":1479252044930,"executorId":"2","taskFailures":4} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorBlacklisted","time":1479252044930,"executorId":"0","taskFailures":4} +{"Event":"org.apache.spark.scheduler.SparkListenerNodeBlacklisted","time":1479252044930,"hostId":"172.22.0.111","executorFailures":2} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorUnblacklisted","time":1479252055635,"executorId":"2"} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorUnblacklisted","time":1479252055635,"executorId":"0"} +{"Event":"org.apache.spark.scheduler.SparkListenerNodeUnblacklisted","time":1479252055635,"hostId":"172.22.0.111"} +{"Event":"SparkListenerApplicationEnd","Timestamp":1479252138874} diff --git a/core/src/test/resources/spark-events/app-20161116163331-0000 b/core/src/test/resources/spark-events/app-20161116163331-0000 new file mode 100755 index 000000000000..57cfc5b97312 --- /dev/null +++ b/core/src/test/resources/spark-events/app-20161116163331-0000 @@ -0,0 +1,68 @@ +{"Event":"SparkListenerLogStart","Spark Version":"2.1.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"172.22.0.167","Port":51475},"Maximum Memory":908381388,"Timestamp":1479335611477,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":524288000} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","Java Version":"1.8.0_92 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.blacklist.task.maxTaskAttemptsPerExecutor":"3","spark.blacklist.enabled":"TRUE","spark.driver.host":"172.22.0.167","spark.blacklist.task.maxTaskAttemptsPerNode":"3","spark.eventLog.enabled":"TRUE","spark.driver.port":"51459","spark.repl.class.uri":"spark://172.22.0.167:51459/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-1cbc97d0-7fe6-4c9f-8c2c-f6fe51ee3cf2/repl-39929169-ac4c-4c6d-b116-f648e4dd62ed","spark.app.name":"Spark shell","spark.blacklist.stage.maxFailedExecutorsPerNode":"3","spark.scheduler.mode":"FIFO","spark.eventLog.overwrite":"TRUE","spark.blacklist.stage.maxFailedTasksPerExecutor":"3","spark.executor.id":"driver","spark.blacklist.application.maxFailedExecutorsPerNode":"2","spark.submit.deployMode":"client","spark.master":"local-cluster[4,4,1024]","spark.home":"/Users/Jose/IdeaProjects/spark","spark.eventLog.dir":"/Users/jose/logs","spark.sql.catalogImplementation":"in-memory","spark.eventLog.compress":"FALSE","spark.blacklist.application.maxFailedTasksPerExecutor":"1","spark.blacklist.timeout":"1000000","spark.app.id":"app-20161116163331-0000","spark.task.maxFailures":"4"},"System Properties":{"java.io.tmpdir":"/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/Jose","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib","user.dir":"/Users/Jose/IdeaProjects/spark","java.library.path":"/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.92-b14","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_92-b14","java.vm.info":"mixed mode","java.ext.dirs":"/Users/Jose/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","io.netty.maxDirectMemory":"0","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.11.6","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"jose","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=1000000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","java.version":"1.8.0_92","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/arpack_combined_all-0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math3-3.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-scala_2.11-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-proxy-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/conf/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-6.1.26.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/jars/*":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/kryo-shaded-3.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/log4j-1.2.17.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-core-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-jvm-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/minlog-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-3.8.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/shapeless_2.11-2.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-common_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-streaming_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/validation-api-1.1.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"app-20161116163331-0000","Timestamp":1479335609916,"User":"jose"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479335615320,"Executor ID":"3","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout","stderr":"http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"172.22.0.167","Port":51485},"Maximum Memory":908381388,"Timestamp":1479335615387,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":524288000} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479335615393,"Executor ID":"2","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout","stderr":"http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr"}}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479335615443,"Executor ID":"1","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stdout","stderr":"http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"172.22.0.167","Port":51487},"Maximum Memory":908381388,"Timestamp":1479335615448,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":524288000} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479335615462,"Executor ID":"0","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stdout","stderr":"http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"172.22.0.167","Port":51490},"Maximum Memory":908381388,"Timestamp":1479335615496,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":524288000} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"172.22.0.167","Port":51491},"Maximum Memory":908381388,"Timestamp":1479335615515,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":524288000} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1479335616467,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.(:35)\n$line16.$read$$iw$$iw$$iw$$iw.(:37)\n$line16.$read$$iw$$iw$$iw.(:39)\n$line16.$read$$iw$$iw.(:41)\n$line16.$read$$iw.(:43)\n$line16.$read.(:45)\n$line16.$read$.(:49)\n$line16.$read$.()\n$line16.$eval$.$print$lzycompute(:7)\n$line16.$eval$.$print(:6)\n$line16.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Accumulables":[]}],"Stage IDs":[0],"Properties":{}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.(:35)\n$line16.$read$$iw$$iw$$iw$$iw.(:37)\n$line16.$read$$iw$$iw$$iw.(:39)\n$line16.$read$$iw$$iw.(:41)\n$line16.$read$$iw.(:43)\n$line16.$read.(:45)\n$line16.$read$.(:49)\n$line16.$read$.()\n$line16.$eval$.$print$lzycompute(:7)\n$line16.$eval$.$print(:6)\n$line16.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Accumulables":[]},"Properties":{}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1479335616657,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1479335616687,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1479335616688,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1479335616688,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1479335616689,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1479335616690,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1479335616691,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1479335616692,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1479335616692,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1479335616693,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1479335616694,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1479335616694,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1479335616695,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1479335616696,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1479335616696,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1479335616697,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1479335616694,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617253,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":465,"Value":465,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":47305000,"Value":47305000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":22,"Value":22,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":7220000,"Value":7220000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":1123,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":18,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":465,"Executor Deserialize CPU Time":47305000,"Executor Run Time":22,"Executor CPU Time":7220000,"Result Size":1123,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1479335616697,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617257,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":464,"Value":929,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":20082000,"Value":67387000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":21,"Value":43,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":9084000,"Value":16304000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":2246,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":464,"Executor Deserialize CPU Time":20082000,"Executor Run Time":21,"Executor CPU Time":9084000,"Result Size":1123,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1479335616692,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617257,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":468,"Value":1397,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":29183000,"Value":96570000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":21,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":5753000,"Value":22057000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":3369,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":54,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":468,"Executor Deserialize CPU Time":29183000,"Executor Run Time":21,"Executor CPU Time":5753000,"Result Size":1123,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1479335616688,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617257,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":470,"Value":1867,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":233387000,"Value":329957000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":22,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":6783000,"Value":28840000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1938,"Value":5307,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":470,"Executor Deserialize CPU Time":233387000,"Executor Run Time":22,"Executor CPU Time":6783000,"Result Size":1938,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":453,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1479335616690,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617319,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":453,"Value":539,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Value":94,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":453,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":22,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":444,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1479335616696,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617326,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":444,"Value":983,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Value":123,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":444,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":29,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1479335616687,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617327,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Value":1434,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Value":145,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":451,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":22,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1479335616696,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617328,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Value":1885,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Value":167,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":451,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":22,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":450,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1479335616693,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617329,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":450,"Value":2335,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Value":189,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":450,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":22,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":444,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1479335616694,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617329,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":444,"Value":2779,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Value":218,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":444,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":29,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":442,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1479335616688,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617329,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":442,"Value":3221,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Value":247,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":442,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":29,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":2,"Attempt":1,"Launch Time":1479335617332,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617371,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":1903,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5136000,"Value":346556000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3673,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":958000,"Value":32856000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":9159,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":10,"Attempt":1,"Launch Time":1479335617333,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617370,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":1889,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3808000,"Value":341420000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3672,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1005000,"Value":31898000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":8196,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":9,"Attempt":1,"Launch Time":1479335617333,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617369,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":1879,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3737000,"Value":337612000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3670,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1066000,"Value":30893000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":7233,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":13,"Attempt":1,"Launch Time":1479335617334,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617368,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":1872,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3918000,"Value":333875000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3668,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":987000,"Value":29827000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":6270,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":446,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1479335616691,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617336,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":446,"Value":3667,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Value":276,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":446,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":29,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":6,"Attempt":1,"Launch Time":1479335617349,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617371,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1907,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3503000,"Value":350059000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3674,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1042000,"Value":33898000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":10122,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":13,"Attempt":1,"Launch Time":1479335617334,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617368,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":1872,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3918000,"Value":333875000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3668,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":987000,"Value":29827000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":6270,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":3918000,"Executor Run Time":1,"Executor CPU Time":987000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":1,"Attempt":1,"Launch Time":1479335617368,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617379,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1911,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3579000,"Value":353638000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3675,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":996000,"Value":34894000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":11085,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":9,"Attempt":1,"Launch Time":1479335617333,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617369,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":1879,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3737000,"Value":337612000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3670,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1066000,"Value":30893000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":7233,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":3737000,"Executor Run Time":2,"Executor CPU Time":1066000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":14,"Attempt":1,"Launch Time":1479335617369,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617380,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1915,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3412000,"Value":357050000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3676,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1014000,"Value":35908000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":12048,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":10,"Attempt":1,"Launch Time":1479335617333,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617370,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":1889,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3808000,"Value":341420000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3672,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1005000,"Value":31898000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":8196,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":3808000,"Executor Run Time":2,"Executor CPU Time":1005000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":5,"Attempt":1,"Launch Time":1479335617370,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617380,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":1918,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3482000,"Value":360532000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3678,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1142000,"Value":37050000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":13011,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":2,"Attempt":1,"Launch Time":1479335617332,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617371,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":1903,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5136000,"Value":346556000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3673,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":958000,"Value":32856000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":9159,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":5136000,"Executor Run Time":1,"Executor CPU Time":958000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":6,"Attempt":1,"Launch Time":1479335617349,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617371,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1907,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3503000,"Value":350059000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3674,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1042000,"Value":33898000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":10122,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3503000,"Executor Run Time":1,"Executor CPU Time":1042000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":1,"Attempt":1,"Launch Time":1479335617368,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617379,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1911,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3579000,"Value":353638000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3675,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":996000,"Value":34894000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":11085,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3579000,"Executor Run Time":1,"Executor CPU Time":996000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":14,"Attempt":1,"Launch Time":1479335617369,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617380,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1915,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3412000,"Value":357050000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3676,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1014000,"Value":35908000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":12048,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3412000,"Executor Run Time":1,"Executor CPU Time":1014000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":5,"Attempt":1,"Launch Time":1479335617370,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617380,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":1918,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3482000,"Value":360532000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3678,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1142000,"Value":37050000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":13011,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3482000,"Executor Run Time":2,"Executor CPU Time":1142000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1479335616657,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617470,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":714,"Value":2632,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":41300000,"Value":401832000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":18,"Value":3696,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":6640000,"Value":43690000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":14047,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":293,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":714,"Executor Deserialize CPU Time":41300000,"Executor Run Time":18,"Executor CPU Time":6640000,"Result Size":1036,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1479335616692,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617471,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":714,"Value":3346,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":43682000,"Value":445514000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":15,"Value":3711,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":9441000,"Value":53131000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":15083,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":310,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":714,"Executor Deserialize CPU Time":43682000,"Executor Run Time":15,"Executor CPU Time":9441000,"Result Size":1036,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1479335616695,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617471,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":691,"Value":4037,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":54811000,"Value":500325000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":16,"Value":3727,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":4571000,"Value":57702000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":16119,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":327,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":691,"Executor Deserialize CPU Time":54811000,"Executor Run Time":16,"Executor CPU Time":4571000,"Result Size":1036,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1479335616689,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617473,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":716,"Value":4753,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":220235000,"Value":720560000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":16,"Value":3743,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":5849000,"Value":63551000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1851,"Value":17970,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":344,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":716,"Executor Deserialize CPU Time":220235000,"Executor Run Time":16,"Executor CPU Time":5849000,"Result Size":1851,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.(:35)\n$line16.$read$$iw$$iw$$iw$$iw.(:37)\n$line16.$read$$iw$$iw$$iw.(:39)\n$line16.$read$$iw$$iw.(:41)\n$line16.$read$$iw.(:43)\n$line16.$read.(:45)\n$line16.$read$.(:49)\n$line16.$read$.()\n$line16.$eval$.$print$lzycompute(:7)\n$line16.$eval$.$print(:6)\n$line16.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Submission Time":1479335616653,"Completion Time":1479335617476,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Value":3743,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Value":344,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Value":17970,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Value":720560000,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Value":63551000,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Value":4,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Value":4753,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1479335617480,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorBlacklisted","time":1479335617478,"executorId":"2","taskFailures":4} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorBlacklisted","time":1479335617478,"executorId":"0","taskFailures":4} +{"Event":"org.apache.spark.scheduler.SparkListenerNodeBlacklisted","time":1479335617478,"hostId":"172.22.0.167","executorFailures":2} +{"Event":"SparkListenerApplicationEnd","Timestamp":1479335620587} diff --git a/core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1422981759269 similarity index 100% rename from core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1 rename to core/src/test/resources/spark-events/local-1422981759269 diff --git a/core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1422981780767 similarity index 100% rename from core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1 rename to core/src/test/resources/spark-events/local-1422981780767 diff --git a/core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1425081759269 similarity index 100% rename from core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1 rename to core/src/test/resources/spark-events/local-1425081759269 diff --git a/core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1426533911241 similarity index 100% rename from core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 rename to core/src/test/resources/spark-events/local-1426533911241 diff --git a/core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1426633911242 similarity index 100% rename from core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1 rename to core/src/test/resources/spark-events/local-1426633911242 diff --git a/core/src/test/resources/spark.keystore b/core/src/test/resources/spark.keystore new file mode 100644 index 000000000000..f30716b57b30 Binary files /dev/null and b/core/src/test/resources/spark.keystore differ diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 5b84acf40be4..3990ee1ec326 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -17,18 +17,33 @@ package org.apache.spark +import java.util.concurrent.Semaphore +import javax.annotation.concurrent.GuardedBy + import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.ref.WeakReference +import scala.util.control.NonFatal import org.scalatest.Matchers import org.scalatest.exceptions.TestFailedException +import org.apache.spark.AccumulatorParam.StringAccumulatorParam import org.apache.spark.scheduler._ +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, AccumulatorV2, LongAccumulator} class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContext { - import InternalAccumulator._ + import AccumulatorSuite.createLongAccum + + override def afterEach(): Unit = { + try { + AccumulatorContext.clear() + } finally { + super.afterEach() + } + } implicit def setAccum[A]: AccumulableParam[mutable.Set[A], A] = new AccumulableParam[mutable.Set[A], A] { @@ -45,9 +60,30 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex } } - test ("basic accumulation"){ + test("accumulator serialization") { + val ser = new JavaSerializer(new SparkConf).newInstance() + val acc = createLongAccum("x") + acc.add(5) + assert(acc.value == 5) + assert(acc.isAtDriverSide) + + // serialize and de-serialize it, to simulate sending accumulator to executor. + val acc2 = ser.deserialize[LongAccumulator](ser.serialize(acc)) + // value is reset on the executors + assert(acc2.value == 0) + assert(!acc2.isAtDriverSide) + + acc2.add(10) + // serialize and de-serialize it again, to simulate sending accumulator back to driver. + val acc3 = ser.deserialize[LongAccumulator](ser.serialize(acc2)) + // value is not reset on the driver + assert(acc3.value == 10) + assert(acc3.isAtDriverSide) + } + + test ("basic accumulation") { sc = new SparkContext("local", "test") - val acc : Accumulator[Int] = sc.accumulator(0) + val acc: Accumulator[Int] = sc.accumulator(0) val d = sc.parallelize(1 to 20) d.foreach{x => acc += x} @@ -59,12 +95,14 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex longAcc.value should be (210L + maxInt * 20) } - test ("value not assignable from tasks") { + test("value not assignable from tasks") { sc = new SparkContext("local", "test") - val acc : Accumulator[Int] = sc.accumulator(0) + val acc: Accumulator[Int] = sc.accumulator(0) val d = sc.parallelize(1 to 20) - an [Exception] should be thrownBy {d.foreach{x => acc.value = x}} + intercept[SparkException] { + d.foreach(x => acc.value = x) + } } test ("add value to collection accumulators") { @@ -84,7 +122,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex } } - test ("value not readable in tasks") { + test("value not readable in tasks") { val maxI = 1000 for (nThreads <- List(1, 10)) { // test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -135,7 +173,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex d.foreach { x => acc.localValue ++= x } - acc.value should be ( (0 to maxI).toSet) + acc.value should be ((0 to maxI).toSet) resetSparkContext() } } @@ -155,197 +193,95 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex System.gc() assert(ref.get.isEmpty) - Accumulators.remove(accId) - assert(!Accumulators.originals.get(accId).isDefined) + AccumulatorContext.remove(accId) + assert(!AccumulatorContext.get(accId).isDefined) } - test("internal accumulators in TaskContext") { - sc = new SparkContext("local", "test") - val accums = InternalAccumulator.create(sc) - val taskContext = new TaskContextImpl(0, 0, 0, 0, null, null, accums) - val internalMetricsToAccums = taskContext.internalMetricsToAccumulators - val collectedInternalAccums = taskContext.collectInternalAccumulators() - val collectedAccums = taskContext.collectAccumulators() - assert(internalMetricsToAccums.size > 0) - assert(internalMetricsToAccums.values.forall(_.isInternal)) - assert(internalMetricsToAccums.contains(TEST_ACCUMULATOR)) - val testAccum = internalMetricsToAccums(TEST_ACCUMULATOR) - assert(collectedInternalAccums.size === internalMetricsToAccums.size) - assert(collectedInternalAccums.size === collectedAccums.size) - assert(collectedInternalAccums.contains(testAccum.id)) - assert(collectedAccums.contains(testAccum.id)) - } + test("get accum") { + // Don't register with SparkContext for cleanup + var acc = createLongAccum("a") + val accId = acc.id + val ref = WeakReference(acc) + assert(ref.get.isDefined) - test("internal accumulators in a stage") { - val listener = new SaveInfoListener - val numPartitions = 10 - sc = new SparkContext("local", "test") - sc.addSparkListener(listener) - // Have each task add 1 to the internal accumulator - val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitions { iter => - TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 - iter - } - // Register asserts in job completion callback to avoid flakiness - listener.registerJobCompletionCallback { _ => - val stageInfos = listener.getCompletedStageInfos - val taskInfos = listener.getCompletedTaskInfos - assert(stageInfos.size === 1) - assert(taskInfos.size === numPartitions) - // The accumulator values should be merged in the stage - val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUMULATOR) - assert(stageAccum.value.toLong === numPartitions) - // The accumulator should be updated locally on each task - val taskAccumValues = taskInfos.map { taskInfo => - val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUMULATOR) - assert(taskAccum.update.isDefined) - assert(taskAccum.update.get.toLong === 1) - taskAccum.value.toLong - } - // Each task should keep track of the partial value on the way, i.e. 1, 2, ... numPartitions - assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) - } - rdd.count() - } + // Remove the explicit reference to it and allow weak reference to get garbage collected + acc = null + System.gc() + assert(ref.get.isEmpty) - test("internal accumulators in multiple stages") { - val listener = new SaveInfoListener - val numPartitions = 10 - sc = new SparkContext("local", "test") - sc.addSparkListener(listener) - // Each stage creates its own set of internal accumulators so the - // values for the same metric should not be mixed up across stages - val rdd = sc.parallelize(1 to 100, numPartitions) - .map { i => (i, i) } - .mapPartitions { iter => - TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 - iter - } - .reduceByKey { case (x, y) => x + y } - .mapPartitions { iter => - TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 10 - iter - } - .repartition(numPartitions * 2) - .mapPartitions { iter => - TaskContext.get().internalMetricsToAccumulators(TEST_ACCUMULATOR) += 100 - iter - } - // Register asserts in job completion callback to avoid flakiness - listener.registerJobCompletionCallback { _ => - // We ran 3 stages, and the accumulator values should be distinct - val stageInfos = listener.getCompletedStageInfos - assert(stageInfos.size === 3) - val (firstStageAccum, secondStageAccum, thirdStageAccum) = - (findAccumulableInfo(stageInfos(0).accumulables.values, TEST_ACCUMULATOR), - findAccumulableInfo(stageInfos(1).accumulables.values, TEST_ACCUMULATOR), - findAccumulableInfo(stageInfos(2).accumulables.values, TEST_ACCUMULATOR)) - assert(firstStageAccum.value.toLong === numPartitions) - assert(secondStageAccum.value.toLong === numPartitions * 10) - assert(thirdStageAccum.value.toLong === numPartitions * 2 * 100) + // Getting a garbage collected accum should throw error + intercept[IllegalStateException] { + AccumulatorContext.get(accId) } - rdd.count() - } - test("internal accumulators in fully resubmitted stages") { - testInternalAccumulatorsWithFailedTasks((i: Int) => true) // fail all tasks + // Getting a normal accumulator. Note: this has to be separate because referencing an + // accumulator above in an `assert` would keep it from being garbage collected. + val acc2 = createLongAccum("b") + assert(AccumulatorContext.get(acc2.id) === Some(acc2)) + + // Getting an accumulator that does not exist should return None + assert(AccumulatorContext.get(100000).isEmpty) } - test("internal accumulators in partially resubmitted stages") { - testInternalAccumulatorsWithFailedTasks((i: Int) => i % 2 == 0) // fail a subset + test("string accumulator param") { + val acc = new Accumulator("", StringAccumulatorParam, Some("darkness")) + assert(acc.value === "") + acc.setValue("feeds") + assert(acc.value === "feeds") + acc.add("your") + assert(acc.value === "your") // value is overwritten, not concatenated + acc += "soul" + assert(acc.value === "soul") + acc ++= "with" + assert(acc.value === "with") + acc.merge("kindness") + assert(acc.value === "kindness") } +} + +private[spark] object AccumulatorSuite { + import InternalAccumulator._ /** - * Return the accumulable info that matches the specified name. + * Create a long accumulator and register it to `AccumulatorContext`. */ - private def findAccumulableInfo( - accums: Iterable[AccumulableInfo], - name: String): AccumulableInfo = { - accums.find { a => a.name == name }.getOrElse { - throw new TestFailedException(s"internal accumulator '$name' not found", 0) - } + def createLongAccum( + name: String, + countFailedValues: Boolean = false, + initValue: Long = 0, + id: Long = AccumulatorContext.newId()): LongAccumulator = { + val acc = new LongAccumulator + acc.setValue(initValue) + acc.metadata = AccumulatorMetadata(id, Some(name), countFailedValues) + AccumulatorContext.register(acc) + acc } /** - * Test whether internal accumulators are merged properly if some tasks fail. + * Make an `AccumulableInfo` out of an [[Accumulable]] with the intent to use the + * info as an accumulator update. */ - private def testInternalAccumulatorsWithFailedTasks(failCondition: (Int => Boolean)): Unit = { - val listener = new SaveInfoListener - val numPartitions = 10 - val numFailedPartitions = (0 until numPartitions).count(failCondition) - // This says use 1 core and retry tasks up to 2 times - sc = new SparkContext("local[1, 2]", "test") - sc.addSparkListener(listener) - val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitionsWithIndex { case (i, iter) => - val taskContext = TaskContext.get() - taskContext.internalMetricsToAccumulators(TEST_ACCUMULATOR) += 1 - // Fail the first attempts of a subset of the tasks - if (failCondition(i) && taskContext.attemptNumber() == 0) { - throw new Exception("Failing a task intentionally.") - } - iter - } - // Register asserts in job completion callback to avoid flakiness - listener.registerJobCompletionCallback { _ => - val stageInfos = listener.getCompletedStageInfos - val taskInfos = listener.getCompletedTaskInfos - assert(stageInfos.size === 1) - assert(taskInfos.size === numPartitions + numFailedPartitions) - val stageAccum = findAccumulableInfo(stageInfos.head.accumulables.values, TEST_ACCUMULATOR) - // We should not double count values in the merged accumulator - assert(stageAccum.value.toLong === numPartitions) - val taskAccumValues = taskInfos.flatMap { taskInfo => - if (!taskInfo.failed) { - // If a task succeeded, its update value should always be 1 - val taskAccum = findAccumulableInfo(taskInfo.accumulables, TEST_ACCUMULATOR) - assert(taskAccum.update.isDefined) - assert(taskAccum.update.get.toLong === 1) - Some(taskAccum.value.toLong) - } else { - // If a task failed, we should not get its accumulator values - assert(taskInfo.accumulables.isEmpty) - None - } - } - assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) - } - rdd.count() - } - -} - -private[spark] object AccumulatorSuite { + def makeInfo(a: AccumulatorV2[_, _]): AccumulableInfo = a.toInfo(Some(a.value), None) /** - * Run one or more Spark jobs and verify that the peak execution memory accumulator - * is updated afterwards. + * Run one or more Spark jobs and verify that in at least one job the peak execution memory + * accumulator is updated afterwards. */ def verifyPeakExecutionMemorySet( sc: SparkContext, testName: String)(testBody: => Unit): Unit = { val listener = new SaveInfoListener sc.addSparkListener(listener) - // Register asserts in job completion callback to avoid flakiness - listener.registerJobCompletionCallback { jobId => - if (jobId == 0) { - // The first job is a dummy one to verify that the accumulator does not already exist - val accums = listener.getCompletedStageInfos.flatMap(_.accumulables.values) - assert(!accums.exists(_.name == InternalAccumulator.PEAK_EXECUTION_MEMORY)) - } else { - // In the subsequent jobs, verify that peak execution memory is updated - val accum = listener.getCompletedStageInfos - .flatMap(_.accumulables.values) - .find(_.name == InternalAccumulator.PEAK_EXECUTION_MEMORY) - .getOrElse { - throw new TestFailedException( - s"peak execution memory accumulator not set in '$testName'", 0) - } - assert(accum.value.toLong > 0) - } - } - // Run the jobs - sc.parallelize(1 to 10).count() testBody + // wait until all events have been processed before proceeding to assert things + sc.listenerBus.waitUntilEmpty(10 * 1000) + val accums = listener.getCompletedStageInfos.flatMap(_.accumulables.values) + val isSet = accums.exists { a => + a.name == Some(PEAK_EXECUTION_MEMORY) && a.value.exists(_.asInstanceOf[Long] > 0L) + } + if (!isSet) { + throw new TestFailedException(s"peak execution memory accumulator not set in '$testName'", 0) + } } } @@ -353,21 +289,56 @@ private[spark] object AccumulatorSuite { * A simple listener that keeps track of the TaskInfos and StageInfos of all completed jobs. */ private class SaveInfoListener extends SparkListener { - private val completedStageInfos: ArrayBuffer[StageInfo] = new ArrayBuffer[StageInfo] - private val completedTaskInfos: ArrayBuffer[TaskInfo] = new ArrayBuffer[TaskInfo] - private var jobCompletionCallback: (Int => Unit) = null // parameter is job ID + type StageId = Int + type StageAttemptId = Int + + private val completedStageInfos = new ArrayBuffer[StageInfo] + private val completedTaskInfos = + new mutable.HashMap[(StageId, StageAttemptId), ArrayBuffer[TaskInfo]] + + // Callback to call when a job completes. Parameter is job ID. + @GuardedBy("this") + private var jobCompletionCallback: () => Unit = null + private val jobCompletionSem = new Semaphore(0) + private var exception: Throwable = null def getCompletedStageInfos: Seq[StageInfo] = completedStageInfos.toArray.toSeq - def getCompletedTaskInfos: Seq[TaskInfo] = completedTaskInfos.toArray.toSeq + def getCompletedTaskInfos: Seq[TaskInfo] = completedTaskInfos.values.flatten.toSeq + def getCompletedTaskInfos(stageId: StageId, stageAttemptId: StageAttemptId): Seq[TaskInfo] = + completedTaskInfos.getOrElse((stageId, stageAttemptId), Seq.empty[TaskInfo]) + + /** + * If `jobCompletionCallback` is set, block until the next call has finished. + * If the callback failed with an exception, throw it. + */ + def awaitNextJobCompletion(): Unit = { + if (jobCompletionCallback != null) { + jobCompletionSem.acquire() + if (exception != null) { + throw exception + } + } + } - /** Register a callback to be called on job end. */ - def registerJobCompletionCallback(callback: (Int => Unit)): Unit = { + /** + * Register a callback to be called on job end. + * A call to this should be followed by [[awaitNextJobCompletion]]. + */ + def registerJobCompletionCallback(callback: () => Unit): Unit = { jobCompletionCallback = callback } override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { if (jobCompletionCallback != null) { - jobCompletionCallback(jobEnd.jobId) + try { + jobCompletionCallback() + } catch { + // Store any exception thrown here so we can throw them later in the main thread. + // Otherwise, if `jobCompletionCallback` threw something it wouldn't fail the test. + case NonFatal(e) => exception = e + } finally { + jobCompletionSem.release() + } } } @@ -376,6 +347,7 @@ private class SaveInfoListener extends SparkListener { } override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { - completedTaskInfos += taskEnd.taskInfo + completedTaskInfos.getOrElseUpdate( + (taskEnd.stageId, taskEnd.stageAttemptId), new ArrayBuffer[TaskInfo]) += taskEnd.taskInfo } } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala deleted file mode 100644 index cb8bd04e496a..000000000000 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark - -import org.mockito.Mockito._ -import org.scalatest.BeforeAndAfter -import org.scalatest.mock.MockitoSugar - -import org.apache.spark.executor.{DataReadMethod, TaskMetrics} -import org.apache.spark.rdd.RDD -import org.apache.spark.storage._ - -// TODO: Test the CacheManager's thread-safety aspects -class CacheManagerSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfter - with MockitoSugar { - - var blockManager: BlockManager = _ - var cacheManager: CacheManager = _ - var split: Partition = _ - /** An RDD which returns the values [1, 2, 3, 4]. */ - var rdd: RDD[Int] = _ - var rdd2: RDD[Int] = _ - var rdd3: RDD[Int] = _ - - before { - sc = new SparkContext("local", "test") - blockManager = mock[BlockManager] - cacheManager = new CacheManager(blockManager) - split = new Partition { override def index: Int = 0 } - rdd = new RDD[Int](sc, Nil) { - override def getPartitions: Array[Partition] = Array(split) - override val getDependencies = List[Dependency[_]]() - override def compute(split: Partition, context: TaskContext): Iterator[Int] = - Array(1, 2, 3, 4).iterator - } - rdd2 = new RDD[Int](sc, List(new OneToOneDependency(rdd))) { - override def getPartitions: Array[Partition] = firstParent[Int].partitions - override def compute(split: Partition, context: TaskContext): Iterator[Int] = - firstParent[Int].iterator(split, context) - }.cache() - rdd3 = new RDD[Int](sc, List(new OneToOneDependency(rdd2))) { - override def getPartitions: Array[Partition] = firstParent[Int].partitions - override def compute(split: Partition, context: TaskContext): Iterator[Int] = - firstParent[Int].iterator(split, context) - }.cache() - } - - 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. - blockManager = sc.env.blockManager - cacheManager = sc.env.cacheManager - val context = TaskContext.empty() - val computeValue = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) - val getValue = blockManager.get(RDDBlockId(rdd.id, split.index)) - assert(computeValue.toList === List(1, 2, 3, 4)) - assert(getValue.isDefined, "Block cached from getOrCompute is not found!") - assert(getValue.get.data.toList === List(1, 2, 3, 4)) - } - - test("get cached rdd") { - val result = new BlockResult(Array(5, 6, 7).iterator, DataReadMethod.Memory, 12) - when(blockManager.get(RDDBlockId(0, 0))).thenReturn(Some(result)) - - val context = TaskContext.empty() - val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) - assert(value.toList === List(5, 6, 7)) - } - - test("get uncached local rdd") { - // Local computation should not persist the resulting value, so don't expect a put(). - when(blockManager.get(RDDBlockId(0, 0))).thenReturn(None) - - val context = new TaskContextImpl(0, 0, 0, 0, null, null, Seq.empty, runningLocally = 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") { - cacheManager = sc.env.cacheManager - val context = TaskContext.empty() - cacheManager.getOrCompute(rdd3, split, context, StorageLevel.MEMORY_ONLY) - assert(context.taskMetrics.updatedBlocks.getOrElse(Seq()).size === 2) - } -} diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 119e5fc28e41..48408ccc8f81 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -21,17 +21,233 @@ import java.io.File import scala.reflect.ClassTag +import com.google.common.io.ByteStreams +import org.apache.hadoop.fs.Path + +import org.apache.spark.io.CompressionCodec import org.apache.spark.rdd._ import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId} import org.apache.spark.util.Utils +trait RDDCheckpointTester { self: SparkFunSuite => + + protected val partitioner = new HashPartitioner(2) + + private def defaultCollectFunc[T](rdd: RDD[T]): Any = rdd.collect() + + /** Implementations of this trait must implement this method */ + protected def sparkContext: SparkContext + + /** + * Test checkpointing of the RDD generated by the given operation. It tests whether the + * serialized size of the RDD is reduce after checkpointing or not. This function should be called + * on all RDDs that have a parent RDD (i.e., do not call on ParallelCollection, BlockRDD, etc.). + * + * @param op an operation to run on the RDD + * @param reliableCheckpoint if true, use reliable checkpoints, otherwise use local checkpoints + * @param collectFunc a function for collecting the values in the RDD, in case there are + * non-comparable types like arrays that we want to convert to something + * that supports == + */ + protected def testRDD[U: ClassTag]( + op: (RDD[Int]) => RDD[U], + reliableCheckpoint: Boolean, + collectFunc: RDD[U] => Any = defaultCollectFunc[U] _): Unit = { + // Generate the final RDD using given RDD operation + val baseRDD = generateFatRDD() + val operatedRDD = op(baseRDD) + val parentDependency = operatedRDD.dependencies.headOption.orNull + val rddType = operatedRDD.getClass.getSimpleName + val numPartitions = operatedRDD.partitions.length + + // Force initialization of all the data structures in RDDs + // Without this, serializing the RDD will give a wrong estimate of the size of the RDD + initializeRdd(operatedRDD) + + val partitionsBeforeCheckpoint = operatedRDD.partitions + + // Find serialized sizes before and after the checkpoint + logInfo("RDD before checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) + val (rddSizeBeforeCheckpoint, partitionSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD) + checkpoint(operatedRDD, reliableCheckpoint) + val result = collectFunc(operatedRDD) + operatedRDD.collect() // force re-initialization of post-checkpoint lazy variables + val (rddSizeAfterCheckpoint, partitionSizeAfterCheckpoint) = getSerializedSizes(operatedRDD) + logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) + + // Test whether the checkpoint file has been created + if (reliableCheckpoint) { + assert(operatedRDD.getCheckpointFile.nonEmpty) + val recoveredRDD = sparkContext.checkpointFile[U](operatedRDD.getCheckpointFile.get) + assert(collectFunc(recoveredRDD) === result) + assert(recoveredRDD.partitioner === operatedRDD.partitioner) + } + + // Test whether dependencies have been changed from its earlier parent RDD + assert(operatedRDD.dependencies.head != parentDependency) + + // Test whether the partitions have been changed from its earlier partitions + assert(operatedRDD.partitions.toList != partitionsBeforeCheckpoint.toList) + + // Test whether the partitions have been changed to the new Hadoop partitions + assert(operatedRDD.partitions.toList === operatedRDD.checkpointData.get.getPartitions.toList) + + // Test whether the number of partitions is same as before + assert(operatedRDD.partitions.length === numPartitions) + + // Test whether the data in the checkpointed RDD is same as original + assert(collectFunc(operatedRDD) === result) + + // Test whether serialized size of the RDD has reduced. + logInfo("Size of " + rddType + + " [" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]") + assert( + rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint, + "Size of " + rddType + " did not reduce after checkpointing " + + " [" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]" + ) + } + + /** + * Test whether checkpointing of the parent of the generated RDD also + * truncates the lineage or not. Some RDDs like CoGroupedRDD hold on to its parent + * RDDs partitions. So even if the parent RDD is checkpointed and its partitions changed, + * the generated RDD will remember the partitions and therefore potentially the whole lineage. + * This function should be called only those RDD whose partitions refer to parent RDD's + * partitions (i.e., do not call it on simple RDDs). + * + * @param op an operation to run on the RDD + * @param reliableCheckpoint if true, use reliable checkpoints, otherwise use local checkpoints + * @param collectFunc a function for collecting the values in the RDD, in case there are + * non-comparable types like arrays that we want to convert to something + * that supports == + */ + protected def testRDDPartitions[U: ClassTag]( + op: (RDD[Int]) => RDD[U], + reliableCheckpoint: Boolean, + collectFunc: RDD[U] => Any = defaultCollectFunc[U] _): Unit = { + // Generate the final RDD using given RDD operation + val baseRDD = generateFatRDD() + val operatedRDD = op(baseRDD) + val parentRDDs = operatedRDD.dependencies.map(_.rdd) + val rddType = operatedRDD.getClass.getSimpleName + + // Force initialization of all the data structures in RDDs + // Without this, serializing the RDD will give a wrong estimate of the size of the RDD + initializeRdd(operatedRDD) + + // Find serialized sizes before and after the checkpoint + logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) + val (rddSizeBeforeCheckpoint, partitionSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD) + // checkpoint the parent RDD, not the generated one + parentRDDs.foreach { rdd => + checkpoint(rdd, reliableCheckpoint) + } + val result = collectFunc(operatedRDD) // force checkpointing + operatedRDD.collect() // force re-initialization of post-checkpoint lazy variables + val (rddSizeAfterCheckpoint, partitionSizeAfterCheckpoint) = getSerializedSizes(operatedRDD) + logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) + + // Test whether the data in the checkpointed RDD is same as original + assert(collectFunc(operatedRDD) === result) + + // Test whether serialized size of the partitions has reduced + logInfo("Size of partitions of " + rddType + + " [" + partitionSizeBeforeCheckpoint + " --> " + partitionSizeAfterCheckpoint + "]") + assert( + partitionSizeAfterCheckpoint < partitionSizeBeforeCheckpoint, + "Size of " + rddType + " partitions did not reduce after checkpointing parent RDDs" + + " [" + partitionSizeBeforeCheckpoint + " --> " + partitionSizeAfterCheckpoint + "]" + ) + } + + /** + * Get serialized sizes of the RDD and its partitions, in order to test whether the size shrinks + * upon checkpointing. Ignores the checkpointData field, which may grow when we checkpoint. + */ + private def getSerializedSizes(rdd: RDD[_]): (Int, Int) = { + val rddSize = Utils.serialize(rdd).size + val rddCpDataSize = Utils.serialize(rdd.checkpointData).size + val rddPartitionSize = Utils.serialize(rdd.partitions).size + val rddDependenciesSize = Utils.serialize(rdd.dependencies).size + + // Print detailed size, helps in debugging + logInfo("Serialized sizes of " + rdd + + ": RDD = " + rddSize + + ", RDD checkpoint data = " + rddCpDataSize + + ", RDD partitions = " + rddPartitionSize + + ", RDD dependencies = " + rddDependenciesSize + ) + // this makes sure that serializing the RDD's checkpoint data does not + // serialize the whole RDD as well + assert( + rddSize > rddCpDataSize, + "RDD's checkpoint data (" + rddCpDataSize + ") is equal or larger than the " + + "whole RDD with checkpoint data (" + rddSize + ")" + ) + (rddSize - rddCpDataSize, rddPartitionSize) + } + + /** + * Serialize and deserialize an object. This is useful to verify the objects + * contents after deserialization (e.g., the contents of an RDD split after + * it is sent to a slave along with a task) + */ + protected def serializeDeserialize[T](obj: T): T = { + val bytes = Utils.serialize(obj) + Utils.deserialize[T](bytes) + } + + /** + * Recursively force the initialization of the all members of an RDD and it parents. + */ + private def initializeRdd(rdd: RDD[_]): Unit = { + rdd.partitions // forces the initialization of the partitions + rdd.dependencies.map(_.rdd).foreach(initializeRdd) + } + + /** Checkpoint the RDD either locally or reliably. */ + protected def checkpoint(rdd: RDD[_], reliableCheckpoint: Boolean): Unit = { + if (reliableCheckpoint) { + rdd.checkpoint() + } else { + rdd.localCheckpoint() + } + } + + /** Run a test twice, once for local checkpointing and once for reliable checkpointing. */ + protected def runTest( + name: String, + skipLocalCheckpoint: Boolean = false + )(body: Boolean => Unit): Unit = { + test(name + " [reliable checkpoint]")(body(true)) + if (!skipLocalCheckpoint) { + test(name + " [local checkpoint]")(body(false)) + } + } + + /** + * Generate an RDD such that both the RDD and its partitions have large size. + */ + protected def generateFatRDD(): RDD[Int] = { + new FatRDD(sparkContext.makeRDD(1 to 100, 4)).map(x => x) + } + + /** + * Generate an pair RDD (with partitioner) such that both the RDD and its partitions + * have large size. + */ + protected def generateFatPairRDD(): RDD[(Int, Int)] = { + new FatPairRDD(sparkContext.makeRDD(1 to 100, 4), partitioner).mapValues(x => x) + } +} + /** * Test suite for end-to-end checkpointing functionality. * This tests both reliable checkpoints and local checkpoints. */ -class CheckpointSuite extends SparkFunSuite with LocalSparkContext with Logging { +class CheckpointSuite extends SparkFunSuite with RDDCheckpointTester with LocalSparkContext { private var checkpointDir: File = _ - private val partitioner = new HashPartitioner(2) override def beforeEach(): Unit = { super.beforeEach() @@ -42,10 +258,15 @@ class CheckpointSuite extends SparkFunSuite with LocalSparkContext with Logging } override def afterEach(): Unit = { - super.afterEach() - Utils.deleteRecursively(checkpointDir) + try { + Utils.deleteRecursively(checkpointDir) + } finally { + super.afterEach() + } } + override def sparkContext: SparkContext = sc + runTest("basic checkpointing") { reliableCheckpoint: Boolean => val parCollection = sc.makeRDD(1 to 4) val flatMappedRDD = parCollection.flatMap(x => 1 to x) @@ -56,6 +277,49 @@ class CheckpointSuite extends SparkFunSuite with LocalSparkContext with Logging assert(flatMappedRDD.collect() === result) } + runTest("checkpointing partitioners", skipLocalCheckpoint = true) { _: Boolean => + + def testPartitionerCheckpointing( + partitioner: Partitioner, + corruptPartitionerFile: Boolean = false + ): Unit = { + val rddWithPartitioner = sc.makeRDD(1 to 4).map { _ -> 1 }.partitionBy(partitioner) + rddWithPartitioner.checkpoint() + rddWithPartitioner.count() + assert(rddWithPartitioner.getCheckpointFile.get.nonEmpty, + "checkpointing was not successful") + + if (corruptPartitionerFile) { + // Overwrite the partitioner file with garbage data + val checkpointDir = new Path(rddWithPartitioner.getCheckpointFile.get) + val fs = checkpointDir.getFileSystem(sc.hadoopConfiguration) + val partitionerFile = fs.listStatus(checkpointDir) + .find(_.getPath.getName.contains("partitioner")) + .map(_.getPath) + require(partitionerFile.nonEmpty, "could not find the partitioner file for testing") + val output = fs.create(partitionerFile.get, true) + output.write(100) + output.close() + } + + val newRDD = sc.checkpointFile[(Int, Int)](rddWithPartitioner.getCheckpointFile.get) + assert(newRDD.collect().toSet === rddWithPartitioner.collect().toSet, "RDD not recovered") + + if (!corruptPartitionerFile) { + assert(newRDD.partitioner != None, "partitioner not recovered") + assert(newRDD.partitioner === rddWithPartitioner.partitioner, + "recovered partitioner does not match") + } else { + assert(newRDD.partitioner == None, "partitioner unexpectedly recovered") + } + } + + testPartitionerCheckpointing(partitioner) + + // Test that corrupted partitioner file does not prevent recovery of RDD + testPartitionerCheckpointing(partitioner, corruptPartitionerFile = true) + } + runTest("RDDs with one-to-one dependencies") { reliableCheckpoint: Boolean => testRDD(_.map(x => x.toString), reliableCheckpoint) testRDD(_.flatMap(x => 1 to x), reliableCheckpoint) @@ -124,7 +388,7 @@ class CheckpointSuite extends SparkFunSuite with LocalSparkContext with Logging // the parent RDD has been checkpointed and parent partitions have been changed. // Note that this test is very specific to the current implementation of CartesianRDD. val ones = sc.makeRDD(1 to 100, 10).map(x => x) - checkpoint(ones, reliableCheckpoint) // checkpoint that MappedRDD + checkpoint(ones, reliableCheckpoint) val cartesian = new CartesianRDD(sc, ones, ones) val splitBeforeCheckpoint = serializeDeserialize(cartesian.partitions.head.asInstanceOf[CartesianPartition]) @@ -147,7 +411,7 @@ class CheckpointSuite extends SparkFunSuite with LocalSparkContext with Logging // Note that this test is very specific to the current implementation of // CoalescedRDDPartitions. val ones = sc.makeRDD(1 to 100, 10).map(x => x) - checkpoint(ones, reliableCheckpoint) // checkpoint that MappedRDD + checkpoint(ones, reliableCheckpoint) val coalesced = new CoalescedRDD(ones, 2) val splitBeforeCheckpoint = serializeDeserialize(coalesced.partitions.head.asInstanceOf[CoalescedRDDPartition]) @@ -238,7 +502,7 @@ class CheckpointSuite extends SparkFunSuite with LocalSparkContext with Logging } runTest("CheckpointRDD with zero partitions") { reliableCheckpoint: Boolean => - val rdd = new BlockRDD[Int](sc, Array[BlockId]()) + val rdd = new BlockRDD[Int](sc, Array.empty[BlockId]) assert(rdd.partitions.size === 0) assert(rdd.isCheckpointed === false) assert(rdd.isCheckpointedAndMaterialized === false) @@ -251,203 +515,26 @@ class CheckpointSuite extends SparkFunSuite with LocalSparkContext with Logging assert(rdd.partitions.size === 0) } - // Utility test methods - - /** Checkpoint the RDD either locally or reliably. */ - private def checkpoint(rdd: RDD[_], reliableCheckpoint: Boolean): Unit = { - if (reliableCheckpoint) { - rdd.checkpoint() - } else { - rdd.localCheckpoint() - } + runTest("checkpointAllMarkedAncestors") { reliableCheckpoint: Boolean => + testCheckpointAllMarkedAncestors(reliableCheckpoint, checkpointAllMarkedAncestors = true) + testCheckpointAllMarkedAncestors(reliableCheckpoint, checkpointAllMarkedAncestors = false) } - /** Run a test twice, once for local checkpointing and once for reliable checkpointing. */ - private def runTest(name: String)(body: Boolean => Unit): Unit = { - test(name + " [reliable checkpoint]")(body(true)) - test(name + " [local checkpoint]")(body(false)) - } - - private def defaultCollectFunc[T](rdd: RDD[T]): Any = rdd.collect() - - /** - * Test checkpointing of the RDD generated by the given operation. It tests whether the - * serialized size of the RDD is reduce after checkpointing or not. This function should be called - * on all RDDs that have a parent RDD (i.e., do not call on ParallelCollection, BlockRDD, etc.). - * - * @param op an operation to run on the RDD - * @param reliableCheckpoint if true, use reliable checkpoints, otherwise use local checkpoints - * @param collectFunc a function for collecting the values in the RDD, in case there are - * non-comparable types like arrays that we want to convert to something that supports == - */ - private def testRDD[U: ClassTag]( - op: (RDD[Int]) => RDD[U], - reliableCheckpoint: Boolean, - collectFunc: RDD[U] => Any = defaultCollectFunc[U] _): Unit = { - // Generate the final RDD using given RDD operation - val baseRDD = generateFatRDD() - val operatedRDD = op(baseRDD) - val parentRDD = operatedRDD.dependencies.headOption.orNull - val rddType = operatedRDD.getClass.getSimpleName - val numPartitions = operatedRDD.partitions.length - - // Force initialization of all the data structures in RDDs - // Without this, serializing the RDD will give a wrong estimate of the size of the RDD - initializeRdd(operatedRDD) - - val partitionsBeforeCheckpoint = operatedRDD.partitions - - // Find serialized sizes before and after the checkpoint - logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) - val (rddSizeBeforeCheckpoint, partitionSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD) - checkpoint(operatedRDD, reliableCheckpoint) - val result = collectFunc(operatedRDD) - operatedRDD.collect() // force re-initialization of post-checkpoint lazy variables - val (rddSizeAfterCheckpoint, partitionSizeAfterCheckpoint) = getSerializedSizes(operatedRDD) - logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) - - // Test whether the checkpoint file has been created - if (reliableCheckpoint) { - assert(collectFunc(sc.checkpointFile[U](operatedRDD.getCheckpointFile.get)) === result) - } - - // Test whether dependencies have been changed from its earlier parent RDD - assert(operatedRDD.dependencies.head.rdd != parentRDD) - - // Test whether the partitions have been changed from its earlier partitions - assert(operatedRDD.partitions.toList != partitionsBeforeCheckpoint.toList) - - // Test whether the partitions have been changed to the new Hadoop partitions - assert(operatedRDD.partitions.toList === operatedRDD.checkpointData.get.getPartitions.toList) - - // Test whether the number of partitions is same as before - assert(operatedRDD.partitions.length === numPartitions) - - // Test whether the data in the checkpointed RDD is same as original - assert(collectFunc(operatedRDD) === result) - - // Test whether serialized size of the RDD has reduced. - logInfo("Size of " + rddType + - " [" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]") - assert( - rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint, - "Size of " + rddType + " did not reduce after checkpointing " + - " [" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]" - ) - } - - /** - * Test whether checkpointing of the parent of the generated RDD also - * truncates the lineage or not. Some RDDs like CoGroupedRDD hold on to its parent - * RDDs partitions. So even if the parent RDD is checkpointed and its partitions changed, - * the generated RDD will remember the partitions and therefore potentially the whole lineage. - * This function should be called only those RDD whose partitions refer to parent RDD's - * partitions (i.e., do not call it on simple RDD like MappedRDD). - * - * @param op an operation to run on the RDD - * @param reliableCheckpoint if true, use reliable checkpoints, otherwise use local checkpoints - * @param collectFunc a function for collecting the values in the RDD, in case there are - * non-comparable types like arrays that we want to convert to something that supports == - */ - private def testRDDPartitions[U: ClassTag]( - op: (RDD[Int]) => RDD[U], - reliableCheckpoint: Boolean, - collectFunc: RDD[U] => Any = defaultCollectFunc[U] _): Unit = { - // Generate the final RDD using given RDD operation - val baseRDD = generateFatRDD() - val operatedRDD = op(baseRDD) - val parentRDDs = operatedRDD.dependencies.map(_.rdd) - val rddType = operatedRDD.getClass.getSimpleName - - // Force initialization of all the data structures in RDDs - // Without this, serializing the RDD will give a wrong estimate of the size of the RDD - initializeRdd(operatedRDD) - - // Find serialized sizes before and after the checkpoint - logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) - val (rddSizeBeforeCheckpoint, partitionSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD) - // checkpoint the parent RDD, not the generated one - parentRDDs.foreach { rdd => - checkpoint(rdd, reliableCheckpoint) + private def testCheckpointAllMarkedAncestors( + reliableCheckpoint: Boolean, checkpointAllMarkedAncestors: Boolean): Unit = { + sc.setLocalProperty(RDD.CHECKPOINT_ALL_MARKED_ANCESTORS, checkpointAllMarkedAncestors.toString) + try { + val rdd1 = sc.parallelize(1 to 10) + checkpoint(rdd1, reliableCheckpoint) + val rdd2 = rdd1.map(_ + 1) + checkpoint(rdd2, reliableCheckpoint) + rdd2.count() + assert(rdd1.isCheckpointed === checkpointAllMarkedAncestors) + assert(rdd2.isCheckpointed === true) + } finally { + sc.setLocalProperty(RDD.CHECKPOINT_ALL_MARKED_ANCESTORS, null) } - val result = collectFunc(operatedRDD) // force checkpointing - operatedRDD.collect() // force re-initialization of post-checkpoint lazy variables - val (rddSizeAfterCheckpoint, partitionSizeAfterCheckpoint) = getSerializedSizes(operatedRDD) - logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) - - // Test whether the data in the checkpointed RDD is same as original - assert(collectFunc(operatedRDD) === result) - - // Test whether serialized size of the partitions has reduced - logInfo("Size of partitions of " + rddType + - " [" + partitionSizeBeforeCheckpoint + " --> " + partitionSizeAfterCheckpoint + "]") - assert( - partitionSizeAfterCheckpoint < partitionSizeBeforeCheckpoint, - "Size of " + rddType + " partitions did not reduce after checkpointing parent RDDs" + - " [" + partitionSizeBeforeCheckpoint + " --> " + partitionSizeAfterCheckpoint + "]" - ) } - - /** - * Generate an RDD such that both the RDD and its partitions have large size. - */ - private def generateFatRDD(): RDD[Int] = { - new FatRDD(sc.makeRDD(1 to 100, 4)).map(x => x) - } - - /** - * Generate an pair RDD (with partitioner) such that both the RDD and its partitions - * have large size. - */ - private def generateFatPairRDD(): RDD[(Int, Int)] = { - new FatPairRDD(sc.makeRDD(1 to 100, 4), partitioner).mapValues(x => x) - } - - /** - * Get serialized sizes of the RDD and its partitions, in order to test whether the size shrinks - * upon checkpointing. Ignores the checkpointData field, which may grow when we checkpoint. - */ - private def getSerializedSizes(rdd: RDD[_]): (Int, Int) = { - val rddSize = Utils.serialize(rdd).size - val rddCpDataSize = Utils.serialize(rdd.checkpointData).size - val rddPartitionSize = Utils.serialize(rdd.partitions).size - val rddDependenciesSize = Utils.serialize(rdd.dependencies).size - - // Print detailed size, helps in debugging - logInfo("Serialized sizes of " + rdd + - ": RDD = " + rddSize + - ", RDD checkpoint data = " + rddCpDataSize + - ", RDD partitions = " + rddPartitionSize + - ", RDD dependencies = " + rddDependenciesSize - ) - // this makes sure that serializing the RDD's checkpoint data does not - // serialize the whole RDD as well - assert( - rddSize > rddCpDataSize, - "RDD's checkpoint data (" + rddCpDataSize + ") is equal or larger than the " + - "whole RDD with checkpoint data (" + rddSize + ")" - ) - (rddSize - rddCpDataSize, rddPartitionSize) - } - - /** - * Serialize and deserialize an object. This is useful to verify the objects - * contents after deserialization (e.g., the contents of an RDD split after - * it is sent to a slave along with a task) - */ - private def serializeDeserialize[T](obj: T): T = { - val bytes = Utils.serialize(obj) - Utils.deserialize[T](bytes) - } - - /** - * Recursively force the initialization of the all members of an RDD and it parents. - */ - private def initializeRdd(rdd: RDD[_]): Unit = { - rdd.partitions // forces the - rdd.dependencies.map(_.rdd).foreach(initializeRdd) - } - } /** RDD partition that has large serialized size. */ @@ -494,5 +581,43 @@ object CheckpointSuite { part ).asInstanceOf[RDD[(K, Array[Iterable[V]])]] } +} +class CheckpointCompressionSuite extends SparkFunSuite with LocalSparkContext { + + test("checkpoint compression") { + val checkpointDir = Utils.createTempDir() + try { + val conf = new SparkConf() + .set("spark.checkpoint.compress", "true") + .set("spark.ui.enabled", "false") + sc = new SparkContext("local", "test", conf) + sc.setCheckpointDir(checkpointDir.toString) + val rdd = sc.makeRDD(1 to 20, numSlices = 1) + rdd.checkpoint() + assert(rdd.collect().toSeq === (1 to 20)) + + // Verify that RDD is checkpointed + assert(rdd.firstParent.isInstanceOf[ReliableCheckpointRDD[_]]) + + val checkpointPath = new Path(rdd.getCheckpointFile.get) + val fs = checkpointPath.getFileSystem(sc.hadoopConfiguration) + val checkpointFile = + fs.listStatus(checkpointPath).map(_.getPath).find(_.getName.startsWith("part-")).get + + // Verify the checkpoint file is compressed, in other words, can be decompressed + val compressedInputStream = CompressionCodec.createCodec(conf) + .compressedInputStream(fs.open(checkpointFile)) + try { + ByteStreams.toByteArray(compressedInputStream) + } finally { + compressedInputStream.close() + } + + // Verify that the compressed content can be read back + assert(rdd.collect().toSeq === (1 to 20)) + } finally { + Utils.deleteRecursively(checkpointDir) + } + } } diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 0c14bef7befd..6724af952505 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -19,30 +19,27 @@ package org.apache.spark import java.lang.ref.WeakReference -import scala.collection.mutable.{HashSet, SynchronizedSet} +import scala.collection.mutable.HashSet import scala.language.existentials import scala.util.Random import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.PatienceConfiguration import org.scalatest.concurrent.Eventually._ +import org.scalatest.concurrent.PatienceConfiguration import org.scalatest.time.SpanSugar._ -import org.apache.spark.rdd.{ReliableRDDCheckpointData, RDD} -import org.apache.spark.storage._ -import org.apache.spark.shuffle.hash.HashShuffleManager +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData} import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.storage.BroadcastBlockId -import org.apache.spark.storage.RDDBlockId -import org.apache.spark.storage.ShuffleBlockId -import org.apache.spark.storage.ShuffleIndexBlockId +import org.apache.spark.storage._ +import org.apache.spark.util.Utils /** * An abstract base class for context cleaner tests, which sets up a context with a config * suitable for cleaner tests and provides some utility functions. Subclasses can use different * config options, in particular, a different shuffle manager class */ -abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[HashShuffleManager]) +abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[SortShuffleManager]) extends SparkFunSuite with BeforeAndAfter with LocalSparkContext { implicit val defaultTimeout = timeout(10000 millis) @@ -210,8 +207,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { } test("automatically cleanup normal checkpoint") { - val checkpointDir = java.io.File.createTempFile("temp", "") - checkpointDir.deleteOnExit() + val checkpointDir = Utils.createTempDir() checkpointDir.delete() var rdd = newPairRDD() sc.setCheckpointDir(checkpointDir.toString) @@ -356,84 +352,6 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { } -/** - * A copy of the shuffle tests for sort-based shuffle - */ -class SortShuffleContextCleanerSuite extends ContextCleanerSuiteBase(classOf[SortShuffleManager]) { - test("cleanup shuffle") { - val (rdd, shuffleDeps) = newRDDWithShuffleDependencies() - val collected = rdd.collect().toList - val tester = new CleanerTester(sc, shuffleIds = shuffleDeps.map(_.shuffleId)) - - // Explicit cleanup - shuffleDeps.foreach(s => cleaner.doCleanupShuffle(s.shuffleId, blocking = true)) - tester.assertCleanup() - - // Verify that shuffles can be re-executed after cleaning up - assert(rdd.collect().toList.equals(collected)) - } - - test("automatically cleanup shuffle") { - var rdd = newShuffleRDD() - rdd.count() - - // Test that GC does not cause shuffle cleanup due to a strong reference - val preGCTester = new CleanerTester(sc, shuffleIds = Seq(0)) - runGC() - intercept[Exception] { - preGCTester.assertCleanup()(timeout(1000 millis)) - } - rdd.count() // Defeat early collection by the JVM - - // Test that GC causes shuffle cleanup after dereferencing the RDD - val postGCTester = new CleanerTester(sc, shuffleIds = Seq(0)) - rdd = null // Make RDD out of scope, so that corresponding shuffle goes out of scope - runGC() - postGCTester.assertCleanup() - } - - test("automatically cleanup RDD + shuffle + broadcast in distributed mode") { - sc.stop() - - val conf2 = new SparkConf() - .setMaster("local-cluster[2, 1, 1024]") - .setAppName("ContextCleanerSuite") - .set("spark.cleaner.referenceTracking.blocking", "true") - .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") - .set("spark.shuffle.manager", shuffleManager.getName) - sc = new SparkContext(conf2) - - val numRdds = 10 - val numBroadcasts = 4 // Broadcasts are more costly - val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer - val broadcastBuffer = (1 to numBroadcasts).map(i => newBroadcast).toBuffer - val rddIds = sc.persistentRdds.keys.toSeq - val shuffleIds = 0 until sc.newShuffleId() - val broadcastIds = broadcastBuffer.map(_.id) - - val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) - runGC() - intercept[Exception] { - preGCTester.assertCleanup()(timeout(1000 millis)) - } - - // Test that GC triggers the cleanup of all variables after the dereferencing them - val postGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) - broadcastBuffer.clear() - rddBuffer.clear() - runGC() - postGCTester.assertCleanup() - - // Make sure the broadcasted task closure no longer exists after GC. - val taskClosureBroadcastId = broadcastIds.max + 1 - assert(sc.env.blockManager.master.getMatchingBlockIds({ - case BroadcastBlockId(`taskClosureBroadcastId`, _) => true - case _ => false - }, askSlaves = true).isEmpty) - } -} - - /** * Class to test whether RDDs, shuffles, etc. have been successfully cleaned. * The checkpoint here refers only to normal (reliable) checkpoints, not local checkpoints. @@ -446,25 +364,25 @@ class CleanerTester( checkpointIds: Seq[Long] = Seq.empty) extends Logging { - val toBeCleanedRDDIds = new HashSet[Int] with SynchronizedSet[Int] ++= rddIds - val toBeCleanedShuffleIds = new HashSet[Int] with SynchronizedSet[Int] ++= shuffleIds - val toBeCleanedBroadcstIds = new HashSet[Long] with SynchronizedSet[Long] ++= broadcastIds - val toBeCheckpointIds = new HashSet[Long] with SynchronizedSet[Long] ++= checkpointIds + val toBeCleanedRDDIds = new HashSet[Int] ++= rddIds + val toBeCleanedShuffleIds = new HashSet[Int] ++= shuffleIds + val toBeCleanedBroadcstIds = new HashSet[Long] ++= broadcastIds + val toBeCheckpointIds = new HashSet[Long] ++= checkpointIds val isDistributed = !sc.isLocal val cleanerListener = new CleanerListener { def rddCleaned(rddId: Int): Unit = { - toBeCleanedRDDIds -= rddId + toBeCleanedRDDIds.synchronized { toBeCleanedRDDIds -= rddId } logInfo("RDD " + rddId + " cleaned") } def shuffleCleaned(shuffleId: Int): Unit = { - toBeCleanedShuffleIds -= shuffleId + toBeCleanedShuffleIds.synchronized { toBeCleanedShuffleIds -= shuffleId } logInfo("Shuffle " + shuffleId + " cleaned") } def broadcastCleaned(broadcastId: Long): Unit = { - toBeCleanedBroadcstIds -= broadcastId + toBeCleanedBroadcstIds.synchronized { toBeCleanedBroadcstIds -= broadcastId } logInfo("Broadcast " + broadcastId + " cleaned") } @@ -473,7 +391,7 @@ class CleanerTester( } def checkpointCleaned(rddId: Long): Unit = { - toBeCheckpointIds -= rddId + toBeCheckpointIds.synchronized { toBeCheckpointIds -= rddId } logInfo("checkpoint " + rddId + " cleaned") } } @@ -489,7 +407,8 @@ class CleanerTester( def assertCleanup()(implicit waitTimeout: PatienceConfiguration.Timeout) { try { eventually(waitTimeout, interval(100 millis)) { - assert(isAllCleanedUp) + assert(isAllCleanedUp, + "The following resources were not cleaned up:\n" + uncleanedResourcesToString) } postCleanupValidate() } finally { @@ -581,18 +500,27 @@ class CleanerTester( } private def uncleanedResourcesToString = { + val s1 = toBeCleanedRDDIds.synchronized { + toBeCleanedRDDIds.toSeq.sorted.mkString("[", ", ", "]") + } + val s2 = toBeCleanedShuffleIds.synchronized { + toBeCleanedShuffleIds.toSeq.sorted.mkString("[", ", ", "]") + } + val s3 = toBeCleanedBroadcstIds.synchronized { + toBeCleanedBroadcstIds.toSeq.sorted.mkString("[", ", ", "]") + } s""" - |\tRDDs = ${toBeCleanedRDDIds.toSeq.sorted.mkString("[", ", ", "]")} - |\tShuffles = ${toBeCleanedShuffleIds.toSeq.sorted.mkString("[", ", ", "]")} - |\tBroadcasts = ${toBeCleanedBroadcstIds.toSeq.sorted.mkString("[", ", ", "]")} + |\tRDDs = $s1 + |\tShuffles = $s2 + |\tBroadcasts = $s3 """.stripMargin } private def isAllCleanedUp = - toBeCleanedRDDIds.isEmpty && - toBeCleanedShuffleIds.isEmpty && - toBeCleanedBroadcstIds.isEmpty && - toBeCheckpointIds.isEmpty + toBeCleanedRDDIds.synchronized { toBeCleanedRDDIds.isEmpty } && + toBeCleanedShuffleIds.synchronized { toBeCleanedShuffleIds.isEmpty } && + toBeCleanedBroadcstIds.synchronized { toBeCleanedBroadcstIds.isEmpty } && + toBeCheckpointIds.synchronized { toBeCheckpointIds.isEmpty } private def getRDDBlocks(rddId: Int): Seq[BlockId] = { blockManager.master.getMatchingBlockIds( _ match { diff --git a/core/src/test/scala/org/apache/spark/DebugFilesystem.scala b/core/src/test/scala/org/apache/spark/DebugFilesystem.scala new file mode 100644 index 000000000000..91355f736290 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/DebugFilesystem.scala @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import java.io.{FileDescriptor, InputStream} +import java.lang +import java.nio.ByteBuffer + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.hadoop.fs._ + +import org.apache.spark.internal.Logging + +object DebugFilesystem extends Logging { + // Stores the set of active streams and their creation sites. + private val openStreams = mutable.Map.empty[FSDataInputStream, Throwable] + + def addOpenStream(stream: FSDataInputStream): Unit = openStreams.synchronized { + openStreams.put(stream, new Throwable()) + } + + def clearOpenStreams(): Unit = openStreams.synchronized { + openStreams.clear() + } + + def removeOpenStream(stream: FSDataInputStream): Unit = openStreams.synchronized { + openStreams.remove(stream) + } + + def assertNoOpenStreams(): Unit = openStreams.synchronized { + val numOpen = openStreams.values.size + if (numOpen > 0) { + for (exc <- openStreams.values) { + logWarning("Leaked filesystem connection created at:") + exc.printStackTrace() + } + throw new IllegalStateException(s"There are $numOpen possibly leaked file streams.", + openStreams.values.head) + } + } +} + +/** + * DebugFilesystem wraps file open calls to track all open connections. This can be used in tests + * to check that connections are not leaked. + */ +// TODO(ekl) we should consider always interposing this to expose num open conns as a metric +class DebugFilesystem extends LocalFileSystem { + import DebugFilesystem._ + + override def open(f: Path, bufferSize: Int): FSDataInputStream = { + val wrapped: FSDataInputStream = super.open(f, bufferSize) + addOpenStream(wrapped) + new FSDataInputStream(wrapped.getWrappedStream) { + override def setDropBehind(dropBehind: lang.Boolean): Unit = wrapped.setDropBehind(dropBehind) + + override def getWrappedStream: InputStream = wrapped.getWrappedStream + + override def getFileDescriptor: FileDescriptor = wrapped.getFileDescriptor + + override def getPos: Long = wrapped.getPos + + override def seekToNewSource(targetPos: Long): Boolean = wrapped.seekToNewSource(targetPos) + + override def seek(desired: Long): Unit = wrapped.seek(desired) + + override def setReadahead(readahead: lang.Long): Unit = wrapped.setReadahead(readahead) + + override def read(position: Long, buffer: Array[Byte], offset: Int, length: Int): Int = + wrapped.read(position, buffer, offset, length) + + override def read(buf: ByteBuffer): Int = wrapped.read(buf) + + override def readFully(position: Long, buffer: Array[Byte], offset: Int, length: Int): Unit = + wrapped.readFully(position, buffer, offset, length) + + override def readFully(position: Long, buffer: Array[Byte]): Unit = + wrapped.readFully(position, buffer) + + override def available(): Int = wrapped.available() + + override def mark(readlimit: Int): Unit = wrapped.mark(readlimit) + + override def skip(n: Long): Long = wrapped.skip(n) + + override def markSupported(): Boolean = wrapped.markSupported() + + override def close(): Unit = { + wrapped.close() + removeOpenStream(wrapped) + } + + override def read(): Int = wrapped.read() + + override def reset(): Unit = wrapped.reset() + + override def toString: String = wrapped.toString + + override def equals(obj: scala.Any): Boolean = wrapped.equals(obj) + + override def hashCode(): Int = wrapped.hashCode() + } + } +} diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 1c3f2bc315dd..bea67b71a5a1 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -17,17 +17,20 @@ package org.apache.spark -import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers +import org.scalatest.concurrent.TimeLimits._ import org.scalatest.time.{Millis, Span} +import org.apache.spark.security.EncryptionFunSuite import org.apache.spark.storage.{RDDBlockId, StorageLevel} +import org.apache.spark.util.io.ChunkedByteBuffer class NotSerializableClass class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {} -class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContext { +class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContext + with EncryptionFunSuite { val clusterUrl = "local-cluster[2,1,1024]" @@ -50,18 +53,21 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex } test("local-cluster format") { - sc = new SparkContext("local-cluster[2,1,1024]", "test") - assert(sc.parallelize(1 to 2, 2).count() == 2) - resetSparkContext() - sc = new SparkContext("local-cluster[2 , 1 , 1024]", "test") - assert(sc.parallelize(1 to 2, 2).count() == 2) - resetSparkContext() - sc = new SparkContext("local-cluster[2, 1, 1024]", "test") - assert(sc.parallelize(1 to 2, 2).count() == 2) - resetSparkContext() - sc = new SparkContext("local-cluster[ 2, 1, 1024 ]", "test") - assert(sc.parallelize(1 to 2, 2).count() == 2) - resetSparkContext() + import SparkMasterRegex._ + + val masterStrings = Seq( + "local-cluster[2,1,1024]", + "local-cluster[2 , 1 , 1024]", + "local-cluster[2, 1, 1024]", + "local-cluster[ 2, 1, 1024 ]" + ) + + masterStrings.foreach { + case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => + assert(numSlaves.toInt == 2) + assert(coresPerSlave.toInt == 1) + assert(memoryPerSlave.toInt == 1024) + } } test("simple groupByKey") { @@ -88,8 +94,8 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex test("accumulators") { sc = new SparkContext(clusterUrl, "test") - val accum = sc.accumulator(0) - sc.parallelize(1 to 10, 10).foreach(x => accum += x) + val accum = sc.longAccumulator + sc.parallelize(1 to 10, 10).foreach(x => accum.add(x)) assert(accum.value === 55) } @@ -105,7 +111,6 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex test("repeatedly failing task") { sc = new SparkContext(clusterUrl, "test") - val accum = sc.accumulator(0) val thrown = intercept[SparkException] { // scalastyle:off println sc.parallelize(1 to 10, 10).foreach(x => println(x / 0)) @@ -131,75 +136,62 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex } } - test("caching") { - sc = new SparkContext(clusterUrl, "test") - val data = sc.parallelize(1 to 1000, 10).cache() - assert(data.count() === 1000) - assert(data.count() === 1000) - assert(data.count() === 1000) - } - - test("caching on disk") { - sc = new SparkContext(clusterUrl, "test") - val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.DISK_ONLY) - assert(data.count() === 1000) - assert(data.count() === 1000) - assert(data.count() === 1000) - } - - test("caching in memory, replicated") { - sc = new SparkContext(clusterUrl, "test") - val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_ONLY_2) - assert(data.count() === 1000) - assert(data.count() === 1000) - assert(data.count() === 1000) - } - - test("caching in memory, serialized, replicated") { - sc = new SparkContext(clusterUrl, "test") - val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_ONLY_SER_2) - assert(data.count() === 1000) - assert(data.count() === 1000) - assert(data.count() === 1000) - } - - test("caching on disk, replicated") { - sc = new SparkContext(clusterUrl, "test") - val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.DISK_ONLY_2) - assert(data.count() === 1000) - assert(data.count() === 1000) - assert(data.count() === 1000) - } - - test("caching in memory and disk, replicated") { + test("repeatedly failing task that crashes JVM with a zero exit code (SPARK-16925)") { + // Ensures that if a task which causes the JVM to exit with a zero exit code will cause the + // Spark job to eventually fail. sc = new SparkContext(clusterUrl, "test") - val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_AND_DISK_2) - assert(data.count() === 1000) - assert(data.count() === 1000) - assert(data.count() === 1000) + failAfter(Span(100000, Millis)) { + val thrown = intercept[SparkException] { + sc.parallelize(1 to 1, 1).foreachPartition { _ => System.exit(0) } + } + assert(thrown.getClass === classOf[SparkException]) + assert(thrown.getMessage.contains("failed 4 times")) + } + // Check that the cluster is still usable: + sc.parallelize(1 to 10).count() } - test("caching in memory and disk, serialized, replicated") { - sc = new SparkContext(clusterUrl, "test") - val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_AND_DISK_SER_2) - - assert(data.count() === 1000) - assert(data.count() === 1000) - assert(data.count() === 1000) + private def testCaching(conf: SparkConf, storageLevel: StorageLevel): Unit = { + sc = new SparkContext(conf.setMaster(clusterUrl).setAppName("test")) + sc.jobProgressListener.waitUntilExecutorsUp(2, 30000) + val data = sc.parallelize(1 to 1000, 10) + val cachedData = data.persist(storageLevel) + assert(cachedData.count === 1000) + assert(sc.getExecutorStorageStatus.map(_.rddBlocksById(cachedData.id).size).sum === + storageLevel.replication * data.getNumPartitions) + assert(cachedData.count === 1000) + assert(cachedData.count === 1000) // Get all the locations of the first partition and try to fetch the partitions // from those locations. val blockIds = data.partitions.indices.map(index => RDDBlockId(data.id, index)).toArray val blockId = blockIds(0) val blockManager = SparkEnv.get.blockManager - val blockTransfer = SparkEnv.get.blockTransferService + val blockTransfer = blockManager.blockTransferService + val serializerManager = SparkEnv.get.serializerManager blockManager.master.getLocations(blockId).foreach { cmId => val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, cmId.executorId, blockId.toString) - val deserialized = blockManager.dataDeserialize(blockId, bytes.nioByteBuffer()) - .asInstanceOf[Iterator[Int]].toList + val deserialized = serializerManager.dataDeserializeStream(blockId, + new ChunkedByteBuffer(bytes.nioByteBuffer()).toInputStream())(data.elementClassTag).toList assert(deserialized === (1 to 100).toList) } + // This will exercise the getRemoteBytes / getRemoteValues code paths: + assert(blockIds.flatMap(id => blockManager.get[Int](id).get.data).toSet === (1 to 1000).toSet) + } + + Seq( + "caching" -> StorageLevel.MEMORY_ONLY, + "caching on disk" -> StorageLevel.DISK_ONLY, + "caching in memory, replicated" -> StorageLevel.MEMORY_ONLY_2, + "caching in memory, serialized, replicated" -> StorageLevel.MEMORY_ONLY_SER_2, + "caching on disk, replicated" -> StorageLevel.DISK_ONLY_2, + "caching in memory and disk, replicated" -> StorageLevel.MEMORY_AND_DISK_2, + "caching in memory and disk, serialized, replicated" -> StorageLevel.MEMORY_AND_DISK_SER_2 + ).foreach { case (testName, storageLevel) => + encryptionTest(testName) { conf => + testCaching(conf, storageLevel) + } } test("compute without caching when no partitions fit in memory") { @@ -219,10 +211,10 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex test("compute when only some partitions fit in memory") { val size = 10000 - val numPartitions = 10 + val numPartitions = 20 val conf = new SparkConf() .set("spark.storage.unrollMemoryThreshold", "1024") - .set("spark.testing.memory", (size * numPartitions).toString) + .set("spark.testing.memory", size.toString) sc = new SparkContext(clusterUrl, "test", conf) val data = sc.parallelize(1 to size, numPartitions).persist(StorageLevel.MEMORY_ONLY) assert(data.count() === size) @@ -318,7 +310,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex Thread.sleep(200) } } catch { - case _: Throwable => { Thread.sleep(10) } + case _: Throwable => Thread.sleep(10) // Do nothing. We might see exceptions because block manager // is racing this thread to remove entries from the driver. } diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 454b7e607a51..be80d278fcea 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -19,13 +19,13 @@ package org.apache.spark import java.io.File -import org.scalatest.concurrent.Timeouts +import org.scalatest.concurrent.TimeLimits import org.scalatest.prop.TableDrivenPropertyChecks._ import org.scalatest.time.SpanSugar._ import org.apache.spark.util.Utils -class DriverSuite extends SparkFunSuite with Timeouts { +class DriverSuite extends SparkFunSuite with TimeLimits { ignore("driver should exit after finishing without cleanup (SPARK-530)") { val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 116f027a0f98..a91e09b7cb69 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -20,9 +20,12 @@ package org.apache.spark import scala.collection.mutable import org.scalatest.{BeforeAndAfter, PrivateMethodTester} + import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.ExternalClusterManager import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.scheduler.local.LocalSchedulerBackend import org.apache.spark.util.ManualClock /** @@ -46,9 +49,14 @@ class ExecutorAllocationManagerSuite contexts.foreach(_.stop()) } + private def post(bus: LiveListenerBus, event: SparkListenerEvent): Unit = { + bus.post(event) + bus.waitUntilEmpty(1000) + } + test("verify min/max executors") { val conf = new SparkConf() - .setMaster("local") + .setMaster("myDummyLocalExternalClusterManager") .setAppName("test-executor-allocation-manager") .set("spark.dynamicAllocation.enabled", "true") .set("spark.dynamicAllocation.testing", "true") @@ -92,7 +100,7 @@ class ExecutorAllocationManagerSuite test("add executors") { sc = createSparkContext(1, 10, 1) val manager = sc.executorAllocationManager.get - sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 1000))) + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 1000))) // Keep adding until the limit is reached assert(numExecutorsTarget(manager) === 1) @@ -137,7 +145,7 @@ class ExecutorAllocationManagerSuite test("add executors capped by num pending tasks") { sc = createSparkContext(0, 10, 0) val manager = sc.executorAllocationManager.get - sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 5))) + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 5))) // Verify that we're capped at number of tasks in the stage assert(numExecutorsTarget(manager) === 0) @@ -153,10 +161,10 @@ class ExecutorAllocationManagerSuite assert(numExecutorsToAdd(manager) === 1) // Verify that running a task doesn't affect the target - sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, 3))) - sc.listenerBus.postToAll(SparkListenerExecutorAdded( + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(1, 3))) + post(sc.listenerBus, SparkListenerExecutorAdded( 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) - sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) + post(sc.listenerBus, SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) assert(numExecutorsTarget(manager) === 5) assert(addExecutors(manager) === 1) assert(numExecutorsTarget(manager) === 6) @@ -169,9 +177,9 @@ class ExecutorAllocationManagerSuite assert(numExecutorsToAdd(manager) === 1) // Verify that re-running a task doesn't blow things up - sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(2, 3))) - sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, createTaskInfo(0, 0, "executor-1"))) - sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, createTaskInfo(1, 0, "executor-1"))) + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(2, 3))) + post(sc.listenerBus, SparkListenerTaskStart(2, 0, createTaskInfo(0, 0, "executor-1"))) + post(sc.listenerBus, SparkListenerTaskStart(2, 0, createTaskInfo(1, 0, "executor-1"))) assert(addExecutors(manager) === 1) assert(numExecutorsTarget(manager) === 9) assert(numExecutorsToAdd(manager) === 2) @@ -180,15 +188,49 @@ class ExecutorAllocationManagerSuite assert(numExecutorsToAdd(manager) === 1) // Verify that running a task once we're at our limit doesn't blow things up - sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, createTaskInfo(0, 1, "executor-1"))) + post(sc.listenerBus, SparkListenerTaskStart(2, 0, createTaskInfo(0, 1, "executor-1"))) assert(addExecutors(manager) === 0) assert(numExecutorsTarget(manager) === 10) } + test("add executors when speculative tasks added") { + sc = createSparkContext(0, 10, 0) + val manager = sc.executorAllocationManager.get + + // Verify that we're capped at number of tasks including the speculative ones in the stage + post(sc.listenerBus, SparkListenerSpeculativeTaskSubmitted(1)) + assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsToAdd(manager) === 1) + assert(addExecutors(manager) === 1) + post(sc.listenerBus, SparkListenerSpeculativeTaskSubmitted(1)) + post(sc.listenerBus, SparkListenerSpeculativeTaskSubmitted(1)) + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(1, 2))) + assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsToAdd(manager) === 2) + assert(addExecutors(manager) === 2) + assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsToAdd(manager) === 4) + assert(addExecutors(manager) === 2) + assert(numExecutorsTarget(manager) === 5) + assert(numExecutorsToAdd(manager) === 1) + + // Verify that running a task doesn't affect the target + post(sc.listenerBus, SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) + assert(numExecutorsTarget(manager) === 5) + assert(addExecutors(manager) === 0) + assert(numExecutorsToAdd(manager) === 1) + + // Verify that running a speculative task doesn't affect the target + post(sc.listenerBus, SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-2", true))) + assert(numExecutorsTarget(manager) === 5) + assert(addExecutors(manager) === 0) + assert(numExecutorsToAdd(manager) === 1) + } + test("cancel pending executors when no longer needed") { sc = createSparkContext(0, 10, 0) val manager = sc.executorAllocationManager.get - sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(2, 5))) + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(2, 5))) assert(numExecutorsTarget(manager) === 0) assert(numExecutorsToAdd(manager) === 1) @@ -199,15 +241,15 @@ class ExecutorAllocationManagerSuite assert(numExecutorsTarget(manager) === 3) val task1Info = createTaskInfo(0, 0, "executor-1") - sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, task1Info)) + post(sc.listenerBus, SparkListenerTaskStart(2, 0, task1Info)) assert(numExecutorsToAdd(manager) === 4) assert(addExecutors(manager) === 2) val task2Info = createTaskInfo(1, 0, "executor-1") - sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, task2Info)) - sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, Success, task1Info, null)) - sc.listenerBus.postToAll(SparkListenerTaskEnd(2, 0, null, Success, task2Info, null)) + post(sc.listenerBus, SparkListenerTaskStart(2, 0, task2Info)) + post(sc.listenerBus, SparkListenerTaskEnd(2, 0, null, Success, task1Info, null)) + post(sc.listenerBus, SparkListenerTaskEnd(2, 0, null, Success, task2Info, null)) assert(adjustRequestedExecutors(manager) === -1) } @@ -262,10 +304,99 @@ class ExecutorAllocationManagerSuite assert(executorsPendingToRemove(manager).isEmpty) } - test ("interleaving add and remove") { + test("remove multiple executors") { sc = createSparkContext(5, 10, 5) val manager = sc.executorAllocationManager.get - sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 1000))) + (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) } + + // Keep removing until the limit is reached + assert(executorsPendingToRemove(manager).isEmpty) + assert(removeExecutors(manager, Seq("1")) === Seq("1")) + assert(executorsPendingToRemove(manager).size === 1) + assert(executorsPendingToRemove(manager).contains("1")) + assert(removeExecutors(manager, Seq("2", "3")) === Seq("2", "3")) + assert(executorsPendingToRemove(manager).size === 3) + assert(executorsPendingToRemove(manager).contains("2")) + assert(executorsPendingToRemove(manager).contains("3")) + assert(!removeExecutor(manager, "100")) // remove non-existent executors + assert(removeExecutors(manager, Seq("101", "102")) !== Seq("101", "102")) + assert(executorsPendingToRemove(manager).size === 3) + assert(removeExecutor(manager, "4")) + assert(removeExecutors(manager, Seq("5")) === Seq("5")) + assert(!removeExecutor(manager, "6")) // reached the limit of 5 + assert(executorsPendingToRemove(manager).size === 5) + assert(executorsPendingToRemove(manager).contains("4")) + assert(executorsPendingToRemove(manager).contains("5")) + assert(!executorsPendingToRemove(manager).contains("6")) + + // Kill executors previously requested to remove + onExecutorRemoved(manager, "1") + assert(executorsPendingToRemove(manager).size === 4) + assert(!executorsPendingToRemove(manager).contains("1")) + onExecutorRemoved(manager, "2") + onExecutorRemoved(manager, "3") + assert(executorsPendingToRemove(manager).size === 2) + assert(!executorsPendingToRemove(manager).contains("2")) + assert(!executorsPendingToRemove(manager).contains("3")) + onExecutorRemoved(manager, "2") // duplicates should not count + onExecutorRemoved(manager, "3") + assert(executorsPendingToRemove(manager).size === 2) + onExecutorRemoved(manager, "4") + onExecutorRemoved(manager, "5") + assert(executorsPendingToRemove(manager).isEmpty) + + // Try removing again + // This should still fail because the number pending + running is still at the limit + assert(!removeExecutor(manager, "7")) + assert(executorsPendingToRemove(manager).isEmpty) + assert(removeExecutors(manager, Seq("8")) !== Seq("8")) + assert(executorsPendingToRemove(manager).isEmpty) + } + + test ("Removing with various numExecutorsTarget condition") { + sc = createSparkContext(5, 12, 5) + val manager = sc.executorAllocationManager.get + + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 8))) + + // Remove when numExecutorsTarget is the same as the current number of executors + assert(addExecutors(manager) === 1) + assert(addExecutors(manager) === 2) + (1 to 8).map { i => createTaskInfo(i, i, s"$i") }.foreach { + info => post(sc.listenerBus, SparkListenerTaskStart(0, 0, info)) } + assert(executorIds(manager).size === 8) + assert(numExecutorsTarget(manager) === 8) + assert(maxNumExecutorsNeeded(manager) == 8) + assert(!removeExecutor(manager, "1")) // won't work since numExecutorsTarget == numExecutors + + // Remove executors when numExecutorsTarget is lower than current number of executors + (1 to 3).map { i => createTaskInfo(i, i, s"$i") }.foreach { info => + post(sc.listenerBus, SparkListenerTaskEnd(0, 0, null, Success, info, null)) + } + adjustRequestedExecutors(manager) + assert(executorIds(manager).size === 8) + assert(numExecutorsTarget(manager) === 5) + assert(maxNumExecutorsNeeded(manager) == 5) + assert(removeExecutor(manager, "1")) + assert(removeExecutors(manager, Seq("2", "3"))=== Seq("2", "3")) + onExecutorRemoved(manager, "1") + onExecutorRemoved(manager, "2") + onExecutorRemoved(manager, "3") + + // numExecutorsTarget is lower than minNumExecutors + post(sc.listenerBus, + SparkListenerTaskEnd(0, 0, null, Success, createTaskInfo(4, 4, "4"), null)) + assert(executorIds(manager).size === 5) + assert(numExecutorsTarget(manager) === 5) + assert(maxNumExecutorsNeeded(manager) == 4) + assert(!removeExecutor(manager, "4")) // lower limit + assert(addExecutors(manager) === 0) // upper limit + } + + test ("interleaving add and remove") { + sc = createSparkContext(5, 12, 5) + val manager = sc.executorAllocationManager.get + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 1000))) // Add a few executors assert(addExecutors(manager) === 1) @@ -279,55 +410,59 @@ class ExecutorAllocationManagerSuite onExecutorAdded(manager, "7") onExecutorAdded(manager, "8") assert(executorIds(manager).size === 8) + assert(numExecutorsTarget(manager) === 8) - // Remove until limit - assert(removeExecutor(manager, "1")) - assert(removeExecutor(manager, "2")) - assert(removeExecutor(manager, "3")) - assert(!removeExecutor(manager, "4")) // lower limit reached - assert(!removeExecutor(manager, "5")) - onExecutorRemoved(manager, "1") - onExecutorRemoved(manager, "2") - onExecutorRemoved(manager, "3") - assert(executorIds(manager).size === 5) - // Add until limit - assert(addExecutors(manager) === 2) // upper limit reached - assert(addExecutors(manager) === 0) - assert(!removeExecutor(manager, "4")) // still at lower limit - assert(!removeExecutor(manager, "5")) + // Remove when numTargetExecutors is equal to the current number of executors + assert(!removeExecutor(manager, "1")) + assert(removeExecutors(manager, Seq("2", "3")) !== Seq("2", "3")) + + // Remove until limit onExecutorAdded(manager, "9") onExecutorAdded(manager, "10") onExecutorAdded(manager, "11") onExecutorAdded(manager, "12") - onExecutorAdded(manager, "13") - assert(executorIds(manager).size === 10) + assert(executorIds(manager).size === 12) + assert(numExecutorsTarget(manager) === 8) - // Remove succeeds again, now that we are no longer at the lower limit - assert(removeExecutor(manager, "4")) - assert(removeExecutor(manager, "5")) - assert(removeExecutor(manager, "6")) - assert(removeExecutor(manager, "7")) - assert(executorIds(manager).size === 10) - assert(addExecutors(manager) === 0) + assert(removeExecutor(manager, "1")) + assert(removeExecutors(manager, Seq("2", "3", "4")) === Seq("2", "3", "4")) + assert(!removeExecutor(manager, "5")) // lower limit reached + assert(!removeExecutor(manager, "6")) + onExecutorRemoved(manager, "1") + onExecutorRemoved(manager, "2") + onExecutorRemoved(manager, "3") onExecutorRemoved(manager, "4") - onExecutorRemoved(manager, "5") assert(executorIds(manager).size === 8) - // Number of executors pending restarts at 1 - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 0) - assert(executorIds(manager).size === 8) - onExecutorRemoved(manager, "6") - onExecutorRemoved(manager, "7") + // Add until limit + assert(!removeExecutor(manager, "7")) // still at lower limit + assert((manager, Seq("8")) !== Seq("8")) + onExecutorAdded(manager, "13") onExecutorAdded(manager, "14") onExecutorAdded(manager, "15") - assert(executorIds(manager).size === 8) - assert(addExecutors(manager) === 0) // still at upper limit onExecutorAdded(manager, "16") + assert(executorIds(manager).size === 12) + + // Remove succeeds again, now that we are no longer at the lower limit + assert(removeExecutors(manager, Seq("5", "6", "7")) === Seq("5", "6", "7")) + assert(removeExecutor(manager, "8")) + assert(executorIds(manager).size === 12) + onExecutorRemoved(manager, "5") + onExecutorRemoved(manager, "6") + assert(executorIds(manager).size === 10) + assert(numExecutorsToAdd(manager) === 4) + onExecutorRemoved(manager, "9") + onExecutorRemoved(manager, "10") + assert(addExecutors(manager) === 4) // at upper limit onExecutorAdded(manager, "17") + onExecutorAdded(manager, "18") assert(executorIds(manager).size === 10) - assert(numExecutorsTarget(manager) === 10) + assert(addExecutors(manager) === 0) // still at upper limit + onExecutorAdded(manager, "19") + onExecutorAdded(manager, "20") + assert(executorIds(manager).size === 12) + assert(numExecutorsTarget(manager) === 12) } test("starting/canceling add timer") { @@ -440,7 +575,7 @@ class ExecutorAllocationManagerSuite val clock = new ManualClock(2020L) val manager = sc.executorAllocationManager.get manager.setClock(clock) - sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 1000))) + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 1000))) // Scheduler queue backlogged onSchedulerBacklogged(manager) @@ -553,26 +688,26 @@ class ExecutorAllocationManagerSuite // Starting a stage should start the add timer val numTasks = 10 - sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, numTasks))) + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, numTasks))) assert(addTime(manager) !== NOT_SET) // Starting a subset of the tasks should not cancel the add timer val taskInfos = (0 to numTasks - 1).map { i => createTaskInfo(i, i, "executor-1") } - taskInfos.tail.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, info)) } + taskInfos.tail.foreach { info => post(sc.listenerBus, SparkListenerTaskStart(0, 0, info)) } assert(addTime(manager) !== NOT_SET) // Starting all remaining tasks should cancel the add timer - sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, taskInfos.head)) + post(sc.listenerBus, SparkListenerTaskStart(0, 0, taskInfos.head)) assert(addTime(manager) === NOT_SET) // Start two different stages // The add timer should be canceled only if all tasks in both stages start running - sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, numTasks))) - sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(2, numTasks))) + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(1, numTasks))) + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(2, numTasks))) assert(addTime(manager) !== NOT_SET) - taskInfos.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, info)) } + taskInfos.foreach { info => post(sc.listenerBus, SparkListenerTaskStart(1, 0, info)) } assert(addTime(manager) !== NOT_SET) - taskInfos.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, info)) } + taskInfos.foreach { info => post(sc.listenerBus, SparkListenerTaskStart(2, 0, info)) } assert(addTime(manager) === NOT_SET) } @@ -586,22 +721,22 @@ class ExecutorAllocationManagerSuite assert(removeTimes(manager).size === 5) // Starting a task cancel the remove timer for that executor - sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) - sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(1, 1, "executor-1"))) - sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(2, 2, "executor-2"))) + post(sc.listenerBus, SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) + post(sc.listenerBus, SparkListenerTaskStart(0, 0, createTaskInfo(1, 1, "executor-1"))) + post(sc.listenerBus, SparkListenerTaskStart(0, 0, createTaskInfo(2, 2, "executor-2"))) assert(removeTimes(manager).size === 3) assert(!removeTimes(manager).contains("executor-1")) assert(!removeTimes(manager).contains("executor-2")) // Finishing all tasks running on an executor should start the remove timer for that executor - sc.listenerBus.postToAll(SparkListenerTaskEnd( + post(sc.listenerBus, SparkListenerTaskEnd( 0, 0, "task-type", Success, createTaskInfo(0, 0, "executor-1"), new TaskMetrics)) - sc.listenerBus.postToAll(SparkListenerTaskEnd( + post(sc.listenerBus, SparkListenerTaskEnd( 0, 0, "task-type", Success, createTaskInfo(2, 2, "executor-2"), new TaskMetrics)) assert(removeTimes(manager).size === 4) assert(!removeTimes(manager).contains("executor-1")) // executor-1 has not finished yet assert(removeTimes(manager).contains("executor-2")) - sc.listenerBus.postToAll(SparkListenerTaskEnd( + post(sc.listenerBus, SparkListenerTaskEnd( 0, 0, "task-type", Success, createTaskInfo(1, 1, "executor-1"), new TaskMetrics)) assert(removeTimes(manager).size === 5) assert(removeTimes(manager).contains("executor-1")) // executor-1 has now finished @@ -614,13 +749,13 @@ class ExecutorAllocationManagerSuite assert(removeTimes(manager).isEmpty) // New executors have registered - sc.listenerBus.postToAll(SparkListenerExecutorAdded( + post(sc.listenerBus, SparkListenerExecutorAdded( 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) assert(executorIds(manager).size === 1) assert(executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 1) assert(removeTimes(manager).contains("executor-1")) - sc.listenerBus.postToAll(SparkListenerExecutorAdded( + post(sc.listenerBus, SparkListenerExecutorAdded( 0L, "executor-2", new ExecutorInfo("host2", 1, Map.empty))) assert(executorIds(manager).size === 2) assert(executorIds(manager).contains("executor-2")) @@ -628,14 +763,14 @@ class ExecutorAllocationManagerSuite assert(removeTimes(manager).contains("executor-2")) // Existing executors have disconnected - sc.listenerBus.postToAll(SparkListenerExecutorRemoved(0L, "executor-1", "")) + post(sc.listenerBus, SparkListenerExecutorRemoved(0L, "executor-1", "")) assert(executorIds(manager).size === 1) assert(!executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 1) assert(!removeTimes(manager).contains("executor-1")) // Unknown executor has disconnected - sc.listenerBus.postToAll(SparkListenerExecutorRemoved(0L, "executor-3", "")) + post(sc.listenerBus, SparkListenerExecutorRemoved(0L, "executor-3", "")) assert(executorIds(manager).size === 1) assert(removeTimes(manager).size === 1) } @@ -646,8 +781,8 @@ class ExecutorAllocationManagerSuite assert(executorIds(manager).isEmpty) assert(removeTimes(manager).isEmpty) - sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) - sc.listenerBus.postToAll(SparkListenerExecutorAdded( + post(sc.listenerBus, SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) + post(sc.listenerBus, SparkListenerExecutorAdded( 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) assert(executorIds(manager).size === 1) assert(executorIds(manager).contains("executor-1")) @@ -659,15 +794,15 @@ class ExecutorAllocationManagerSuite val manager = sc.executorAllocationManager.get assert(executorIds(manager).isEmpty) assert(removeTimes(manager).isEmpty) - sc.listenerBus.postToAll(SparkListenerExecutorAdded( + post(sc.listenerBus, SparkListenerExecutorAdded( 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) - sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) + post(sc.listenerBus, SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) assert(executorIds(manager).size === 1) assert(executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 0) - sc.listenerBus.postToAll(SparkListenerExecutorAdded( + post(sc.listenerBus, SparkListenerExecutorAdded( 0L, "executor-2", new ExecutorInfo("host1", 1, Map.empty))) assert(executorIds(manager).size === 2) assert(executorIds(manager).contains("executor-2")) @@ -680,7 +815,7 @@ class ExecutorAllocationManagerSuite sc = createSparkContext(0, 100000, 0) val manager = sc.executorAllocationManager.get val stage1 = createStageInfo(0, 1000) - sc.listenerBus.postToAll(SparkListenerStageSubmitted(stage1)) + post(sc.listenerBus, SparkListenerStageSubmitted(stage1)) assert(addExecutors(manager) === 1) assert(addExecutors(manager) === 2) @@ -691,12 +826,12 @@ class ExecutorAllocationManagerSuite onExecutorAdded(manager, s"executor-$i") } assert(executorIds(manager).size === 15) - sc.listenerBus.postToAll(SparkListenerStageCompleted(stage1)) + post(sc.listenerBus, SparkListenerStageCompleted(stage1)) adjustRequestedExecutors(manager) assert(numExecutorsTarget(manager) === 0) - sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, 1000))) + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(1, 1000))) addExecutors(manager) assert(numExecutorsTarget(manager) === 16) } @@ -713,7 +848,7 @@ class ExecutorAllocationManagerSuite // Verify whether the initial number of executors is kept with no pending tasks assert(numExecutorsTarget(manager) === 3) - sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, 2))) + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(1, 2))) clock.advance(100L) assert(maxNumExecutorsNeeded(manager) === 2) @@ -763,7 +898,7 @@ class ExecutorAllocationManagerSuite Seq.empty ) val stageInfo1 = createStageInfo(1, 5, localityPreferences1) - sc.listenerBus.postToAll(SparkListenerStageSubmitted(stageInfo1)) + post(sc.listenerBus, SparkListenerStageSubmitted(stageInfo1)) assert(localityAwareTasks(manager) === 3) assert(hostToLocalTaskCount(manager) === @@ -775,13 +910,13 @@ class ExecutorAllocationManagerSuite Seq.empty ) val stageInfo2 = createStageInfo(2, 3, localityPreferences2) - sc.listenerBus.postToAll(SparkListenerStageSubmitted(stageInfo2)) + post(sc.listenerBus, SparkListenerStageSubmitted(stageInfo2)) assert(localityAwareTasks(manager) === 5) assert(hostToLocalTaskCount(manager) === Map("host1" -> 2, "host2" -> 4, "host3" -> 4, "host4" -> 3, "host5" -> 2)) - sc.listenerBus.postToAll(SparkListenerStageCompleted(stageInfo1)) + post(sc.listenerBus, SparkListenerStageCompleted(stageInfo1)) assert(localityAwareTasks(manager) === 2) assert(hostToLocalTaskCount(manager) === Map("host2" -> 1, "host3" -> 2, "host4" -> 1, "host5" -> 2)) @@ -792,25 +927,114 @@ class ExecutorAllocationManagerSuite val manager = sc.executorAllocationManager.get assert(maxNumExecutorsNeeded(manager) === 0) - sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, 1))) + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 1))) assert(maxNumExecutorsNeeded(manager) === 1) val taskInfo = createTaskInfo(1, 1, "executor-1") - sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, taskInfo)) + post(sc.listenerBus, SparkListenerTaskStart(0, 0, taskInfo)) assert(maxNumExecutorsNeeded(manager) === 1) // If the task is failed, we expect it to be resubmitted later. - val taskEndReason = ExceptionFailure(null, null, null, null, null, None) - sc.listenerBus.postToAll(SparkListenerTaskEnd(0, 0, null, taskEndReason, taskInfo, null)) + val taskEndReason = ExceptionFailure(null, null, null, null, None) + post(sc.listenerBus, SparkListenerTaskEnd(0, 0, null, taskEndReason, taskInfo, null)) assert(maxNumExecutorsNeeded(manager) === 1) } + test("reset the state of allocation manager") { + sc = createSparkContext() + val manager = sc.executorAllocationManager.get + assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsToAdd(manager) === 1) + + // Allocation manager is reset when adding executor requests are sent without reporting back + // executor added. + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 10))) + + assert(addExecutors(manager) === 1) + assert(numExecutorsTarget(manager) === 2) + assert(addExecutors(manager) === 2) + assert(numExecutorsTarget(manager) === 4) + assert(addExecutors(manager) === 1) + assert(numExecutorsTarget(manager) === 5) + + manager.reset() + assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsToAdd(manager) === 1) + assert(executorIds(manager) === Set.empty) + + // Allocation manager is reset when executors are added. + post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 10))) + + addExecutors(manager) + addExecutors(manager) + addExecutors(manager) + assert(numExecutorsTarget(manager) === 5) + + onExecutorAdded(manager, "first") + onExecutorAdded(manager, "second") + onExecutorAdded(manager, "third") + onExecutorAdded(manager, "fourth") + onExecutorAdded(manager, "fifth") + assert(executorIds(manager) === Set("first", "second", "third", "fourth", "fifth")) + + // Cluster manager lost will make all the live executors lost, so here simulate this behavior + onExecutorRemoved(manager, "first") + onExecutorRemoved(manager, "second") + onExecutorRemoved(manager, "third") + onExecutorRemoved(manager, "fourth") + onExecutorRemoved(manager, "fifth") + + manager.reset() + assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsToAdd(manager) === 1) + assert(executorIds(manager) === Set.empty) + assert(removeTimes(manager) === Map.empty) + + // Allocation manager is reset when executors are pending to remove + addExecutors(manager) + addExecutors(manager) + addExecutors(manager) + assert(numExecutorsTarget(manager) === 5) + + onExecutorAdded(manager, "first") + onExecutorAdded(manager, "second") + onExecutorAdded(manager, "third") + onExecutorAdded(manager, "fourth") + onExecutorAdded(manager, "fifth") + onExecutorAdded(manager, "sixth") + onExecutorAdded(manager, "seventh") + onExecutorAdded(manager, "eighth") + assert(executorIds(manager) === Set("first", "second", "third", "fourth", "fifth", + "sixth", "seventh", "eighth")) + + removeExecutor(manager, "first") + removeExecutors(manager, Seq("second", "third")) + assert(executorsPendingToRemove(manager) === Set("first", "second", "third")) + assert(executorIds(manager) === Set("first", "second", "third", "fourth", "fifth", + "sixth", "seventh", "eighth")) + + + // Cluster manager lost will make all the live executors lost, so here simulate this behavior + onExecutorRemoved(manager, "first") + onExecutorRemoved(manager, "second") + onExecutorRemoved(manager, "third") + onExecutorRemoved(manager, "fourth") + onExecutorRemoved(manager, "fifth") + + manager.reset() + + assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsToAdd(manager) === 1) + assert(executorsPendingToRemove(manager) === Set.empty) + assert(removeTimes(manager) === Map.empty) + } + private def createSparkContext( minExecutors: Int = 1, maxExecutors: Int = 5, initialExecutors: Int = 1): SparkContext = { val conf = new SparkConf() - .setMaster("local") + .setMaster("myDummyLocalExternalClusterManager") .setAppName("test-executor-allocation-manager") .set("spark.dynamicAllocation.enabled", "true") .set("spark.dynamicAllocation.minExecutors", minExecutors.toString) @@ -843,14 +1067,19 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { numTasks: Int, taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty ): StageInfo = { - new StageInfo( - stageId, 0, "name", numTasks, Seq.empty, Seq.empty, "no details", taskLocalityPreferences) + new StageInfo(stageId, 0, "name", numTasks, Seq.empty, Seq.empty, "no details", + taskLocalityPreferences = taskLocalityPreferences) } - private def createTaskInfo(taskId: Int, taskIndex: Int, executorId: String): TaskInfo = { - new TaskInfo(taskId, taskIndex, 0, 0, executorId, "", TaskLocality.ANY, speculative = false) + private def createTaskInfo( + taskId: Int, + taskIndex: Int, + executorId: String, + speculative: Boolean = false): TaskInfo = { + new TaskInfo(taskId, taskIndex, 0, 0, executorId, "", TaskLocality.ANY, speculative) } + /* ------------------------------------------------------- * | Helper methods for accessing private methods and fields | * ------------------------------------------------------- */ @@ -868,6 +1097,7 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { private val _updateAndSyncNumExecutorsTarget = PrivateMethod[Int]('updateAndSyncNumExecutorsTarget) private val _removeExecutor = PrivateMethod[Boolean]('removeExecutor) + private val _removeExecutors = PrivateMethod[Seq[String]]('removeExecutors) private val _onExecutorAdded = PrivateMethod[Unit]('onExecutorAdded) private val _onExecutorRemoved = PrivateMethod[Unit]('onExecutorRemoved) private val _onSchedulerBacklogged = PrivateMethod[Unit]('onSchedulerBacklogged) @@ -876,6 +1106,7 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { private val _onExecutorBusy = PrivateMethod[Unit]('onExecutorBusy) private val _localityAwareTasks = PrivateMethod[Int]('localityAwareTasks) private val _hostToLocalTaskCount = PrivateMethod[Map[String, Int]]('hostToLocalTaskCount) + private val _onSpeculativeTaskSubmitted = PrivateMethod[Unit]('onSpeculativeTaskSubmitted) private def numExecutorsToAdd(manager: ExecutorAllocationManager): Int = { manager invokePrivate _numExecutorsToAdd() @@ -923,6 +1154,10 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { manager invokePrivate _removeExecutor(id) } + private def removeExecutors(manager: ExecutorAllocationManager, ids: Seq[String]): Seq[String] = { + manager invokePrivate _removeExecutors(ids) + } + private def onExecutorAdded(manager: ExecutorAllocationManager, id: String): Unit = { manager invokePrivate _onExecutorAdded(id) } @@ -947,6 +1182,10 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { manager invokePrivate _onExecutorBusy(id) } + private def onSpeculativeTaskSubmitted(manager: ExecutorAllocationManager, id: String) : Unit = { + manager invokePrivate _onSpeculativeTaskSubmitted(id) + } + private def localityAwareTasks(manager: ExecutorAllocationManager): Int = { manager invokePrivate _localityAwareTasks() } @@ -955,3 +1194,72 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { manager invokePrivate _hostToLocalTaskCount() } } + +/** + * A cluster manager which wraps around the scheduler and backend for local mode. It is used for + * testing the dynamic allocation policy. + */ +private class DummyLocalExternalClusterManager extends ExternalClusterManager { + + def canCreate(masterURL: String): Boolean = masterURL == "myDummyLocalExternalClusterManager" + + override def createTaskScheduler( + sc: SparkContext, + masterURL: String): TaskScheduler = new TaskSchedulerImpl(sc, 1, isLocal = true) + + override def createSchedulerBackend( + sc: SparkContext, + masterURL: String, + scheduler: TaskScheduler): SchedulerBackend = { + val sb = new LocalSchedulerBackend(sc.getConf, scheduler.asInstanceOf[TaskSchedulerImpl], 1) + new DummyLocalSchedulerBackend(sc, sb) + } + + override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { + val sc = scheduler.asInstanceOf[TaskSchedulerImpl] + sc.initialize(backend) + } +} + +/** + * A scheduler backend which wraps around local scheduler backend and exposes the executor + * allocation client interface for testing dynamic allocation. + */ +private class DummyLocalSchedulerBackend (sc: SparkContext, sb: SchedulerBackend) + extends SchedulerBackend with ExecutorAllocationClient { + + override private[spark] def getExecutorIds(): Seq[String] = sc.getExecutorIds() + + override private[spark] def requestTotalExecutors( + numExecutors: Int, + localityAwareTasks: Int, + hostToLocalTaskCount: Map[String, Int]): Boolean = + sc.requestTotalExecutors(numExecutors, localityAwareTasks, hostToLocalTaskCount) + + override def requestExecutors(numAdditionalExecutors: Int): Boolean = + sc.requestExecutors(numAdditionalExecutors) + + override def killExecutors( + executorIds: Seq[String], + replace: Boolean, + force: Boolean): Seq[String] = { + val response = sc.killExecutors(executorIds) + if (response) { + executorIds + } else { + Seq.empty[String] + } + } + + override def start(): Unit = sb.start() + + override def stop(): Unit = sb.stop() + + override def reviveOffers(): Unit = sb.reviveOffers() + + override def defaultParallelism(): Int = sb.defaultParallelism() + + override def killExecutorsOnHost(host: String): Boolean = { + false + } +} diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 231f4631e0a4..fe944031bc94 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.network.shuffle.{ExternalShuffleBlockHandler, ExternalSh /** * This suite creates an external shuffle server and routes all shuffle fetches through it. * Note that failures in this suite may arise due to changes in Spark that invalidate expectations - * set up in [[ExternalShuffleBlockHandler]], such as changing the format of shuffle files or how + * set up in `ExternalShuffleBlockHandler`, such as changing the format of shuffle files or how * we hash files into folders. */ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { @@ -35,7 +35,8 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { var rpcHandler: ExternalShuffleBlockHandler = _ override def beforeAll() { - val transportConf = SparkTransportConf.fromSparkConf(conf, numUsableCores = 2) + super.beforeAll() + val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores = 2) rpcHandler = new ExternalShuffleBlockHandler(transportConf, null) val transportContext = new TransportContext(transportConf, rpcHandler) server = transportContext.createServer() @@ -46,7 +47,11 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { } override def afterAll() { - server.close() + try { + server.close() + } finally { + super.afterAll() + } } // This test ensures that the external shuffle service is actually in use for the other tests. diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 203dab934ca1..d805c67714ff 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -17,10 +17,12 @@ package org.apache.spark -import org.apache.spark.util.NonSerializable - import java.io.{IOException, NotSerializableException, ObjectInputStream} +import org.apache.spark.memory.TestMemoryConsumer +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.NonSerializable + // Common state shared by FailureSuite-launched tasks. We use a global object // for this because any local variables used in the task closures will rightfully // be copied for each task, so there's no other way for them to share state. @@ -149,7 +151,8 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { // cause is preserved val thrownDueToTaskFailure = intercept[SparkException] { sc.parallelize(Seq(0)).mapPartitions { iter => - TaskContext.get().taskMemoryManager().allocatePage(128, null) + val c = new TestMemoryConsumer(TaskContext.get().taskMemoryManager()) + TaskContext.get().taskMemoryManager().allocatePage(128, c) throw new Exception("intentional task failure") iter }.count() @@ -159,7 +162,8 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { // If the task succeeded but memory was leaked, then the task should fail due to that leak val thrownDueToMemoryLeak = intercept[SparkException] { sc.parallelize(Seq(0)).mapPartitions { iter => - TaskContext.get().taskMemoryManager().allocatePage(128, null) + val c = new TestMemoryConsumer(TaskContext.get().taskMemoryManager()) + TaskContext.get().taskMemoryManager().allocatePage(128, c) iter }.count() } @@ -238,6 +242,26 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { FailureSuiteState.clear() } + test("failure because cached RDD partitions are missing from DiskStore (SPARK-15736)") { + sc = new SparkContext("local[1,2]", "test") + val rdd = sc.parallelize(1 to 2, 2).persist(StorageLevel.DISK_ONLY) + rdd.count() + // Directly delete all files from the disk store, triggering failures when reading cached data: + SparkEnv.get.blockManager.diskBlockManager.getAllFiles().foreach(_.delete()) + // Each task should fail once due to missing cached data, but then should succeed on its second + // attempt because the missing cache locations will be purged and the blocks will be recomputed. + rdd.count() + } + + test("SPARK-16304: Link error should not crash executor") { + sc = new SparkContext("local[1,2]", "test") + intercept[SparkException] { + sc.parallelize(1 to 2).foreach { i => + throw new LinkageError() + } + } + } + // TODO: Need to add tests with shuffle fetch failures. } diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala deleted file mode 100644 index 1255e71af6c0..000000000000 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ /dev/null @@ -1,261 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark - -import java.io._ -import java.net.URI -import java.util.jar.{JarEntry, JarOutputStream} -import javax.net.ssl.SSLException - -import com.google.common.io.{ByteStreams, Files} -import org.apache.commons.lang3.RandomUtils - -import org.apache.spark.util.Utils - -import SSLSampleConfigs._ - -class FileServerSuite extends SparkFunSuite with LocalSparkContext { - - @transient var tmpDir: File = _ - @transient var tmpFile: File = _ - @transient var tmpJarUrl: String = _ - - def newConf: SparkConf = new SparkConf(loadDefaults = false).set("spark.authenticate", "false") - - override def beforeEach() { - super.beforeEach() - resetSparkContext() - } - - override def beforeAll() { - super.beforeAll() - - tmpDir = Utils.createTempDir() - val testTempDir = new File(tmpDir, "test") - testTempDir.mkdir() - - val textFile = new File(testTempDir, "FileServerSuite.txt") - val pw = new PrintWriter(textFile) - // scalastyle:off println - pw.println("100") - // scalastyle:on println - pw.close() - - val jarFile = new File(testTempDir, "test.jar") - val jarStream = new FileOutputStream(jarFile) - val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) - - val jarEntry = new JarEntry(textFile.getName) - jar.putNextEntry(jarEntry) - - val in = new FileInputStream(textFile) - ByteStreams.copy(in, jar) - - in.close() - jar.close() - jarStream.close() - - tmpFile = textFile - tmpJarUrl = jarFile.toURI.toURL.toString - } - - override def afterAll() { - super.afterAll() - Utils.deleteRecursively(tmpDir) - } - - test("Distributing files locally") { - sc = new SparkContext("local[4]", "test", newConf) - sc.addFile(tmpFile.toString) - val testData = Array((1, 1), (1, 1), (2, 1), (3, 5), (2, 2), (3, 0)) - val result = sc.parallelize(testData).reduceByKey { - val path = SparkFiles.get("FileServerSuite.txt") - val in = new BufferedReader(new FileReader(path)) - val fileVal = in.readLine().toInt - in.close() - _ * fileVal + _ * fileVal - }.collect() - assert(result.toSet === Set((1, 200), (2, 300), (3, 500))) - } - - test("Distributing files locally security On") { - val sparkConf = new SparkConf(false) - sparkConf.set("spark.authenticate", "true") - sparkConf.set("spark.authenticate.secret", "good") - sc = new SparkContext("local[4]", "test", sparkConf) - - sc.addFile(tmpFile.toString) - assert(sc.env.securityManager.isAuthenticationEnabled() === true) - val testData = Array((1, 1), (1, 1), (2, 1), (3, 5), (2, 2), (3, 0)) - val result = sc.parallelize(testData).reduceByKey { - val path = SparkFiles.get("FileServerSuite.txt") - val in = new BufferedReader(new FileReader(path)) - val fileVal = in.readLine().toInt - in.close() - _ * fileVal + _ * fileVal - }.collect() - assert(result.toSet === Set((1, 200), (2, 300), (3, 500))) - } - - test("Distributing files locally using URL as input") { - // addFile("file:///....") - sc = new SparkContext("local[4]", "test", newConf) - sc.addFile(new File(tmpFile.toString).toURI.toString) - val testData = Array((1, 1), (1, 1), (2, 1), (3, 5), (2, 2), (3, 0)) - val result = sc.parallelize(testData).reduceByKey { - val path = SparkFiles.get("FileServerSuite.txt") - val in = new BufferedReader(new FileReader(path)) - val fileVal = in.readLine().toInt - in.close() - _ * fileVal + _ * fileVal - }.collect() - assert(result.toSet === Set((1, 200), (2, 300), (3, 500))) - } - - test ("Dynamically adding JARS locally") { - sc = new SparkContext("local[4]", "test", newConf) - sc.addJar(tmpJarUrl) - val testData = Array((1, 1)) - sc.parallelize(testData).foreach { x => - if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) { - throw new SparkException("jar not added") - } - } - } - - test("Distributing files on a standalone cluster") { - sc = new SparkContext("local-cluster[1,1,1024]", "test", newConf) - sc.addFile(tmpFile.toString) - val testData = Array((1, 1), (1, 1), (2, 1), (3, 5), (2, 2), (3, 0)) - val result = sc.parallelize(testData).reduceByKey { - val path = SparkFiles.get("FileServerSuite.txt") - val in = new BufferedReader(new FileReader(path)) - val fileVal = in.readLine().toInt - in.close() - _ * fileVal + _ * fileVal - }.collect() - assert(result.toSet === Set((1, 200), (2, 300), (3, 500))) - } - - test ("Dynamically adding JARS on a standalone cluster") { - sc = new SparkContext("local-cluster[1,1,1024]", "test", newConf) - sc.addJar(tmpJarUrl) - val testData = Array((1, 1)) - sc.parallelize(testData).foreach { x => - if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) { - throw new SparkException("jar not added") - } - } - } - - test ("Dynamically adding JARS on a standalone cluster using local: URL") { - sc = new SparkContext("local-cluster[1,1,1024]", "test", newConf) - sc.addJar(tmpJarUrl.replace("file", "local")) - val testData = Array((1, 1)) - sc.parallelize(testData).foreach { x => - if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) { - throw new SparkException("jar not added") - } - } - } - - test ("HttpFileServer should work with SSL") { - val sparkConf = sparkSSLConfig() - val sm = new SecurityManager(sparkConf) - val server = new HttpFileServer(sparkConf, sm, 0) - try { - server.initialize() - - fileTransferTest(server, sm) - } finally { - server.stop() - } - } - - test ("HttpFileServer should work with SSL and good credentials") { - val sparkConf = sparkSSLConfig() - sparkConf.set("spark.authenticate", "true") - sparkConf.set("spark.authenticate.secret", "good") - - val sm = new SecurityManager(sparkConf) - val server = new HttpFileServer(sparkConf, sm, 0) - try { - server.initialize() - - fileTransferTest(server, sm) - } finally { - server.stop() - } - } - - test ("HttpFileServer should not work with valid SSL and bad credentials") { - val sparkConf = sparkSSLConfig() - sparkConf.set("spark.authenticate", "true") - sparkConf.set("spark.authenticate.secret", "bad") - - val sm = new SecurityManager(sparkConf) - val server = new HttpFileServer(sparkConf, sm, 0) - try { - server.initialize() - - intercept[IOException] { - fileTransferTest(server) - } - } finally { - server.stop() - } - } - - test ("HttpFileServer should not work with SSL when the server is untrusted") { - val sparkConf = sparkSSLConfigUntrusted() - val sm = new SecurityManager(sparkConf) - val server = new HttpFileServer(sparkConf, sm, 0) - try { - server.initialize() - - intercept[SSLException] { - fileTransferTest(server) - } - } finally { - server.stop() - } - } - - def fileTransferTest(server: HttpFileServer, sm: SecurityManager = null): Unit = { - val randomContent = RandomUtils.nextBytes(100) - val file = File.createTempFile("FileServerSuite", "sslTests", tmpDir) - Files.write(randomContent, file) - server.addFile(file) - - val uri = new URI(server.serverUri + "/files/" + file.getName) - - val connection = if (sm != null && sm.isAuthenticationEnabled()) { - Utils.constructURIForAuthentication(uri, sm).toURL.openConnection() - } else { - uri.toURL.openConnection() - } - - if (sm != null) { - Utils.setupSecureURLConnection(connection, sm) - } - - val buf = ByteStreams.toByteArray(connection.getInputStream) - assert(buf === randomContent) - } - -} diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index fdb00aafc4a4..02728180ac82 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -17,22 +17,23 @@ package org.apache.spark -import java.io.{File, FileWriter} - -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.input.PortableDataStream -import org.apache.spark.storage.StorageLevel +import java.io._ +import java.nio.ByteBuffer +import java.util.zip.GZIPOutputStream import scala.io.Source +import org.apache.hadoop.fs.Path import org.apache.hadoop.io._ import org.apache.hadoop.io.compress.DefaultCodec -import org.apache.hadoop.mapred.{JobConf, FileAlreadyExistsException, FileSplit, TextInputFormat, TextOutputFormat} +import org.apache.hadoop.mapred.{FileAlreadyExistsException, FileSplit, JobConf, TextInputFormat, TextOutputFormat} import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.mapreduce.lib.input.{FileSplit => NewFileSplit, TextInputFormat => NewTextInputFormat} import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} -import org.apache.spark.rdd.{NewHadoopRDD, HadoopRDD} +import org.apache.spark.internal.config.IGNORE_CORRUPT_FILES +import org.apache.spark.rdd.{HadoopRDD, NewHadoopRDD} +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils class FileSuite extends SparkFunSuite with LocalSparkContext { @@ -44,8 +45,11 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { } override def afterEach() { - super.afterEach() - Utils.deleteRecursively(tempDir) + try { + Utils.deleteRecursively(tempDir) + } finally { + super.afterEach() + } } test("text files") { @@ -55,10 +59,15 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { nums.saveAsTextFile(outputDir) // Read the plain text file and check it's OK val outputFile = new File(outputDir, "part-00000") - val content = Source.fromFile(outputFile).mkString - assert(content === "1\n2\n3\n4\n") - // Also try reading it in as a text file RDD - assert(sc.textFile(outputDir).collect().toList === List("1", "2", "3", "4")) + val bufferSrc = Source.fromFile(outputFile) + Utils.tryWithSafeFinally { + val content = bufferSrc.mkString + assert(content === "1\n2\n3\n4\n") + // Also try reading it in as a text file RDD + assert(sc.textFile(outputDir).collect().toList === List("1", "2", "3", "4")) + } { + bufferSrc.close() + } } test("text files (compressed)") { @@ -104,11 +113,11 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { val normalFile = new File(normalDir, "part-00000") val normalContent = sc.sequenceFile[String, String](normalDir).collect - assert(normalContent === Array.fill(100)("abc", "abc")) + assert(normalContent === Array.fill(100)(("abc", "abc"))) val compressedFile = new File(compressedOutputDir, "part-00000" + codec.getDefaultExtension) val compressedContent = sc.sequenceFile[String, String](compressedOutputDir).collect - assert(compressedContent === Array.fill(100)("abc", "abc")) + assert(compressedContent === Array.fill(100)(("abc", "abc"))) assert(compressedFile.length < normalFile.length) } @@ -228,184 +237,82 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { assert(output.map(_.toString).collect().toList === List("(1,a)", "(2,aa)", "(3,aaa)")) } - test("binary file input as byte array") { - sc = new SparkContext("local", "test") + private def writeBinaryData(testOutput: Array[Byte], testOutputCopies: Int): File = { val outFile = new File(tempDir, "record-bytestream-00000.bin") - val outFileName = outFile.getAbsolutePath() - - // create file - val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) - val bbuf = java.nio.ByteBuffer.wrap(testOutput) - // write data to file - val file = new java.io.FileOutputStream(outFile) + val file = new FileOutputStream(outFile) val channel = file.getChannel - channel.write(bbuf) + for (i <- 0 until testOutputCopies) { + // Shift values by i so that they're different in the output + val alteredOutput = testOutput.map(b => (b + i).toByte) + channel.write(ByteBuffer.wrap(alteredOutput)) + } channel.close() file.close() + outFile + } - val inRdd = sc.binaryFiles(outFileName) - val (infile: String, indata: PortableDataStream) = inRdd.collect.head - + test("binary file input as byte array") { + sc = new SparkContext("local", "test") + val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) + val outFile = writeBinaryData(testOutput, 1) + val inRdd = sc.binaryFiles(outFile.getAbsolutePath) + val (infile, indata) = inRdd.collect().head // Make sure the name and array match - assert(infile.contains(outFileName)) // a prefix may get added + assert(infile.contains(outFile.toURI.getPath)) // a prefix may get added assert(indata.toArray === testOutput) } test("portabledatastream caching tests") { sc = new SparkContext("local", "test") - val outFile = new File(tempDir, "record-bytestream-00000.bin") - val outFileName = outFile.getAbsolutePath() - - // create file val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) - val bbuf = java.nio.ByteBuffer.wrap(testOutput) - // write data to file - val file = new java.io.FileOutputStream(outFile) - val channel = file.getChannel - channel.write(bbuf) - channel.close() - file.close() - - val inRdd = sc.binaryFiles(outFileName).cache() - inRdd.foreach{ - curData: (String, PortableDataStream) => - curData._2.toArray() // force the file to read - } - val mappedRdd = inRdd.map { - curData: (String, PortableDataStream) => - (curData._2.getPath(), curData._2) - } - val (infile: String, indata: PortableDataStream) = mappedRdd.collect.head - + val outFile = writeBinaryData(testOutput, 1) + val inRdd = sc.binaryFiles(outFile.getAbsolutePath).cache() + inRdd.foreach(_._2.toArray()) // force the file to read // Try reading the output back as an object file - - assert(indata.toArray === testOutput) + assert(inRdd.values.collect().head.toArray === testOutput) } test("portabledatastream persist disk storage") { sc = new SparkContext("local", "test") - val outFile = new File(tempDir, "record-bytestream-00000.bin") - val outFileName = outFile.getAbsolutePath() - - // create file val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) - val bbuf = java.nio.ByteBuffer.wrap(testOutput) - // write data to file - val file = new java.io.FileOutputStream(outFile) - val channel = file.getChannel - channel.write(bbuf) - channel.close() - file.close() - - val inRdd = sc.binaryFiles(outFileName).persist(StorageLevel.DISK_ONLY) - inRdd.foreach{ - curData: (String, PortableDataStream) => - curData._2.toArray() // force the file to read - } - val mappedRdd = inRdd.map { - curData: (String, PortableDataStream) => - (curData._2.getPath(), curData._2) - } - val (infile: String, indata: PortableDataStream) = mappedRdd.collect.head - - // Try reading the output back as an object file - - assert(indata.toArray === testOutput) + val outFile = writeBinaryData(testOutput, 1) + val inRdd = sc.binaryFiles(outFile.getAbsolutePath).persist(StorageLevel.DISK_ONLY) + inRdd.foreach(_._2.toArray()) // force the file to read + assert(inRdd.values.collect().head.toArray === testOutput) } test("portabledatastream flatmap tests") { sc = new SparkContext("local", "test") - val outFile = new File(tempDir, "record-bytestream-00000.bin") - val outFileName = outFile.getAbsolutePath() - - // create file val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) + val outFile = writeBinaryData(testOutput, 1) + val inRdd = sc.binaryFiles(outFile.getAbsolutePath) val numOfCopies = 3 - val bbuf = java.nio.ByteBuffer.wrap(testOutput) - // write data to file - val file = new java.io.FileOutputStream(outFile) - val channel = file.getChannel - channel.write(bbuf) - channel.close() - file.close() - - val inRdd = sc.binaryFiles(outFileName) - val mappedRdd = inRdd.map { - curData: (String, PortableDataStream) => - (curData._2.getPath(), curData._2) - } - val copyRdd = mappedRdd.flatMap { - curData: (String, PortableDataStream) => - for (i <- 1 to numOfCopies) yield (i, curData._2) - } - - val copyArr: Array[(Int, PortableDataStream)] = copyRdd.collect() - - // Try reading the output back as an object file + val copyRdd = inRdd.flatMap(curData => (0 until numOfCopies).map(_ => curData._2)) + val copyArr = copyRdd.collect() assert(copyArr.length == numOfCopies) - copyArr.foreach{ - cEntry: (Int, PortableDataStream) => - assert(cEntry._2.toArray === testOutput) + for (i <- copyArr.indices) { + assert(copyArr(i).toArray === testOutput) } - } test("fixed record length binary file as byte array") { - // a fixed length of 6 bytes - sc = new SparkContext("local", "test") - - val outFile = new File(tempDir, "record-bytestream-00000.bin") - val outFileName = outFile.getAbsolutePath() - - // create file val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) val testOutputCopies = 10 - - // write data to file - val file = new java.io.FileOutputStream(outFile) - val channel = file.getChannel - for(i <- 1 to testOutputCopies) { - val bbuf = java.nio.ByteBuffer.wrap(testOutput) - channel.write(bbuf) - } - channel.close() - file.close() - - val inRdd = sc.binaryRecords(outFileName, testOutput.length) - // make sure there are enough elements + val outFile = writeBinaryData(testOutput, testOutputCopies) + val inRdd = sc.binaryRecords(outFile.getAbsolutePath, testOutput.length) assert(inRdd.count == testOutputCopies) - - // now just compare the first one - val indata: Array[Byte] = inRdd.collect.head - assert(indata === testOutput) + val inArr = inRdd.collect() + for (i <- inArr.indices) { + assert(inArr(i) === testOutput.map(b => (b + i).toByte)) + } } test ("negative binary record length should raise an exception") { - // a fixed length of 6 bytes sc = new SparkContext("local", "test") - - val outFile = new File(tempDir, "record-bytestream-00000.bin") - val outFileName = outFile.getAbsolutePath() - - // create file - val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) - val testOutputCopies = 10 - - // write data to file - val file = new java.io.FileOutputStream(outFile) - val channel = file.getChannel - for(i <- 1 to testOutputCopies) { - val bbuf = java.nio.ByteBuffer.wrap(testOutput) - channel.write(bbuf) - } - channel.close() - file.close() - - val inRdd = sc.binaryRecords(outFileName, -1) - + val outFile = writeBinaryData(Array[Byte](1, 2, 3, 4, 5, 6), 1) intercept[SparkException] { - inRdd.count + sc.binaryRecords(outFile.getAbsolutePath, -1).count() } } @@ -494,7 +401,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { job.setOutputKeyClass(classOf[String]) job.setOutputValueClass(classOf[String]) job.set("mapred.output.format.class", classOf[TextOutputFormat[String, String]].getName) - job.set("mapred.output.dir", tempDir.getPath + "/outputDataset_old") + job.set("mapreduce.output.fileoutputformat.outputdir", tempDir.getPath + "/outputDataset_old") randomRDD.saveAsHadoopDataset(job) assert(new File(tempDir.getPath + "/outputDataset_old/part-00000").exists() === true) } @@ -503,12 +410,13 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { sc = new SparkContext("local", "test") val randomRDD = sc.parallelize( Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) - val job = new Job(sc.hadoopConfiguration) + val job = Job.getInstance(sc.hadoopConfiguration) job.setOutputKeyClass(classOf[String]) job.setOutputValueClass(classOf[String]) job.setOutputFormatClass(classOf[NewTextOutputFormat[String, String]]) - val jobConfig = SparkHadoopUtil.get.getConfigurationFromJobContext(job) - jobConfig.set("mapred.output.dir", tempDir.getPath + "/outputDataset_new") + val jobConfig = job.getConfiguration + jobConfig.set("mapreduce.output.fileoutputformat.outputdir", + tempDir.getPath + "/outputDataset_new") randomRDD.saveAsNewAPIHadoopDataset(jobConfig) assert(new File(tempDir.getPath + "/outputDataset_new/part-r-00000").exists() === true) } @@ -524,7 +432,9 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { .mapPartitionsWithInputSplit { (split, part) => Iterator(split.asInstanceOf[FileSplit].getPath.toUri.getPath) }.collect() - assert(inputPaths.toSet === Set(s"$outDir/part-00000", s"$outDir/part-00001")) + val outPathOne = new Path(outDir, "part-00000").toUri.getPath + val outPathTwo = new Path(outDir, "part-00001").toUri.getPath + assert(inputPaths.toSet === Set(outPathOne, outPathTwo)) } test("Get input files via new Hadoop API") { @@ -538,6 +448,66 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { .mapPartitionsWithInputSplit { (split, part) => Iterator(split.asInstanceOf[NewFileSplit].getPath.toUri.getPath) }.collect() - assert(inputPaths.toSet === Set(s"$outDir/part-00000", s"$outDir/part-00001")) + val outPathOne = new Path(outDir, "part-00000").toUri.getPath + val outPathTwo = new Path(outDir, "part-00001").toUri.getPath + assert(inputPaths.toSet === Set(outPathOne, outPathTwo)) } + + test("spark.files.ignoreCorruptFiles should work both HadoopRDD and NewHadoopRDD") { + val inputFile = File.createTempFile("input-", ".gz") + try { + // Create a corrupt gzip file + val byteOutput = new ByteArrayOutputStream() + val gzip = new GZIPOutputStream(byteOutput) + try { + gzip.write(Array[Byte](1, 2, 3, 4)) + } finally { + gzip.close() + } + val bytes = byteOutput.toByteArray + val o = new FileOutputStream(inputFile) + try { + // It's corrupt since we only write half of bytes into the file. + o.write(bytes.take(bytes.length / 2)) + } finally { + o.close() + } + + // Reading a corrupt gzip file should throw EOFException + sc = new SparkContext("local", "test") + // Test HadoopRDD + var e = intercept[SparkException] { + sc.textFile(inputFile.toURI.toString).collect() + } + assert(e.getCause.isInstanceOf[EOFException]) + assert(e.getCause.getMessage === "Unexpected end of input stream") + // Test NewHadoopRDD + e = intercept[SparkException] { + sc.newAPIHadoopFile( + inputFile.toURI.toString, + classOf[NewTextInputFormat], + classOf[LongWritable], + classOf[Text]).collect() + } + assert(e.getCause.isInstanceOf[EOFException]) + assert(e.getCause.getMessage === "Unexpected end of input stream") + sc.stop() + + val conf = new SparkConf().set(IGNORE_CORRUPT_FILES, true) + sc = new SparkContext("local", "test", conf) + // Test HadoopRDD + assert(sc.textFile(inputFile.toURI.toString).collect().isEmpty) + // Test NewHadoopRDD + assert { + sc.newAPIHadoopFile( + inputFile.toURI.toString, + classOf[NewTextInputFormat], + classOf[LongWritable], + classOf[Text]).collect().isEmpty + } + } finally { + inputFile.delete() + } + } + } diff --git a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala index 1102aea96b54..70b6309be7d5 100644 --- a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala +++ b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala @@ -17,11 +17,12 @@ package org.apache.spark -import scala.concurrent.Await import scala.concurrent.duration.Duration import org.scalatest.{BeforeAndAfter, Matchers} +import org.apache.spark.util.ThreadUtils + class FutureActionSuite extends SparkFunSuite @@ -36,7 +37,7 @@ class FutureActionSuite test("simple async action") { val rdd = sc.parallelize(1 to 10, 2) val job = rdd.countAsync() - val res = Await.result(job, Duration.Inf) + val res = ThreadUtils.awaitResult(job, Duration.Inf) res should be (10) job.jobIds.size should be (1) } @@ -44,7 +45,7 @@ class FutureActionSuite test("complex async action") { val rdd = sc.parallelize(1 to 15, 3) val job = rdd.takeAsync(10) - val res = Await.result(job, Duration.Inf) + val res = ThreadUtils.awaitResult(job, Duration.Inf) res should be (1 to 10) job.jobIds.size should be (2) } diff --git a/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala b/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala deleted file mode 100644 index 19180e88ebe0..000000000000 --- a/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark - -import org.scalatest.BeforeAndAfterAll - -class HashShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { - - // This test suite should run all tests in ShuffleSuite with hash-based shuffle. - - override def beforeAll() { - conf.set("spark.shuffle.manager", "hash") - } -} diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 3cd80c0f7d17..88916488c0de 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -21,22 +21,21 @@ import java.util.concurrent.{ExecutorService, TimeUnit} import scala.collection.Map import scala.collection.mutable -import scala.concurrent.Await +import scala.concurrent.Future import scala.concurrent.duration._ -import scala.language.postfixOps -import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} -import org.mockito.Mockito.{mock, spy, verify, when} import org.mockito.Matchers import org.mockito.Matchers._ +import org.mockito.Mockito.{mock, spy, verify, when} +import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEnv, RpcEndpointRef} +import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{ManualClock, ThreadUtils} /** * A test suite for the heartbeating behavior between the driver and the executors. @@ -47,8 +46,8 @@ class HeartbeatReceiverSuite with PrivateMethodTester with LocalSparkContext { - private val executorId1 = "executor-1" - private val executorId2 = "executor-2" + private val executorId1 = "1" + private val executorId2 = "2" // Shared state that must be reset before and after each test private var scheduler: TaskSchedulerImpl = null @@ -66,6 +65,7 @@ class HeartbeatReceiverSuite * that uses a manual clock. */ override def beforeEach(): Unit = { + super.beforeEach() val conf = new SparkConf() .setMaster("local[2]") .setAppName("test") @@ -93,12 +93,12 @@ class HeartbeatReceiverSuite test("task scheduler is set correctly") { assert(heartbeatReceiver.scheduler === null) - heartbeatReceiverRef.askWithRetry[Boolean](TaskSchedulerIsSet) + heartbeatReceiverRef.askSync[Boolean](TaskSchedulerIsSet) assert(heartbeatReceiver.scheduler !== null) } test("normal heartbeat") { - heartbeatReceiverRef.askWithRetry[Boolean](TaskSchedulerIsSet) + heartbeatReceiverRef.askSync[Boolean](TaskSchedulerIsSet) addExecutorAndVerify(executorId1) addExecutorAndVerify(executorId2) triggerHeartbeat(executorId1, executorShouldReregister = false) @@ -116,14 +116,14 @@ class HeartbeatReceiverSuite } test("reregister if heartbeat from unregistered executor") { - heartbeatReceiverRef.askWithRetry[Boolean](TaskSchedulerIsSet) + heartbeatReceiverRef.askSync[Boolean](TaskSchedulerIsSet) // Received heartbeat from unknown executor, so we ask it to re-register triggerHeartbeat(executorId1, executorShouldReregister = true) assert(getTrackedExecutors.isEmpty) } test("reregister if heartbeat from removed executor") { - heartbeatReceiverRef.askWithRetry[Boolean](TaskSchedulerIsSet) + heartbeatReceiverRef.askSync[Boolean](TaskSchedulerIsSet) addExecutorAndVerify(executorId1) addExecutorAndVerify(executorId2) // Remove the second executor but not the first @@ -140,7 +140,7 @@ class HeartbeatReceiverSuite test("expire dead hosts") { val executorTimeout = heartbeatReceiver.invokePrivate(_executorTimeoutMs()) - heartbeatReceiverRef.askWithRetry[Boolean](TaskSchedulerIsSet) + heartbeatReceiverRef.askSync[Boolean](TaskSchedulerIsSet) addExecutorAndVerify(executorId1) addExecutorAndVerify(executorId2) triggerHeartbeat(executorId1, executorShouldReregister = false) @@ -149,7 +149,7 @@ class HeartbeatReceiverSuite heartbeatReceiverClock.advance(executorTimeout / 2) triggerHeartbeat(executorId1, executorShouldReregister = false) heartbeatReceiverClock.advance(executorTimeout) - heartbeatReceiverRef.askWithRetry[Boolean](ExpireDeadHosts) + heartbeatReceiverRef.askSync[Boolean](ExpireDeadHosts) // Only the second executor should be expired as a dead host verify(scheduler).executorLost(Matchers.eq(executorId2), any()) val trackedExecutors = getTrackedExecutors @@ -173,11 +173,11 @@ class HeartbeatReceiverSuite val dummyExecutorEndpoint2 = new FakeExecutorEndpoint(rpcEnv) val dummyExecutorEndpointRef1 = rpcEnv.setupEndpoint("fake-executor-1", dummyExecutorEndpoint1) val dummyExecutorEndpointRef2 = rpcEnv.setupEndpoint("fake-executor-2", dummyExecutorEndpoint2) - fakeSchedulerBackend.driverEndpoint.askWithRetry[RegisterExecutorResponse]( - RegisterExecutor(executorId1, dummyExecutorEndpointRef1, "dummy:4040", 0, Map.empty)) - fakeSchedulerBackend.driverEndpoint.askWithRetry[RegisterExecutorResponse]( - RegisterExecutor(executorId2, dummyExecutorEndpointRef2, "dummy:4040", 0, Map.empty)) - heartbeatReceiverRef.askWithRetry[Boolean](TaskSchedulerIsSet) + fakeSchedulerBackend.driverEndpoint.askSync[Boolean]( + RegisterExecutor(executorId1, dummyExecutorEndpointRef1, "1.2.3.4", 0, Map.empty)) + fakeSchedulerBackend.driverEndpoint.askSync[Boolean]( + RegisterExecutor(executorId2, dummyExecutorEndpointRef2, "1.2.3.5", 0, Map.empty)) + heartbeatReceiverRef.askSync[Boolean](TaskSchedulerIsSet) addExecutorAndVerify(executorId1) addExecutorAndVerify(executorId2) triggerHeartbeat(executorId1, executorShouldReregister = false) @@ -195,7 +195,7 @@ class HeartbeatReceiverSuite // Here we use a timeout of O(seconds), but in practice this whole test takes O(10ms). val executorTimeout = heartbeatReceiver.invokePrivate(_executorTimeoutMs()) heartbeatReceiverClock.advance(executorTimeout * 2) - heartbeatReceiverRef.askWithRetry[Boolean](ExpireDeadHosts) + heartbeatReceiverRef.askSync[Boolean](ExpireDeadHosts) val killThread = heartbeatReceiver.invokePrivate(_killExecutorThread()) killThread.shutdown() // needed for awaitTermination killThread.awaitTermination(10L, TimeUnit.SECONDS) @@ -211,37 +211,39 @@ class HeartbeatReceiverSuite private def triggerHeartbeat( executorId: String, executorShouldReregister: Boolean): Unit = { - val metrics = new TaskMetrics + val metrics = TaskMetrics.empty val blockManagerId = BlockManagerId(executorId, "localhost", 12345) - val response = heartbeatReceiverRef.askWithRetry[HeartbeatResponse]( - Heartbeat(executorId, Array(1L -> metrics), blockManagerId)) + val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( + Heartbeat(executorId, Array(1L -> metrics.accumulators()), blockManagerId)) if (executorShouldReregister) { assert(response.reregisterBlockManager) } else { assert(!response.reregisterBlockManager) // Additionally verify that the scheduler callback is called with the correct parameters verify(scheduler).executorHeartbeatReceived( - Matchers.eq(executorId), Matchers.eq(Array(1L -> metrics)), Matchers.eq(blockManagerId)) + Matchers.eq(executorId), + Matchers.eq(Array(1L -> metrics.accumulators())), + Matchers.eq(blockManagerId)) } } private def addExecutorAndVerify(executorId: String): Unit = { assert( heartbeatReceiver.addExecutor(executorId).map { f => - Await.result(f, 10.seconds) + ThreadUtils.awaitResult(f, 10.seconds) } === Some(true)) } private def removeExecutorAndVerify(executorId: String): Unit = { assert( heartbeatReceiver.removeExecutor(executorId).map { f => - Await.result(f, 10.seconds) + ThreadUtils.awaitResult(f, 10.seconds) } === Some(true)) } private def getTrackedExecutors: Map[String, Long] = { - // We may receive undesired SparkListenerExecutorAdded from LocalBackend, so exclude it from - // the map. See SPARK-10800. + // We may receive undesired SparkListenerExecutorAdded from LocalSchedulerBackend, + // so exclude it from the map. See SPARK-10800. heartbeatReceiver.invokePrivate(_executorLastSeen()). filterKeys(_ != SparkContext.DRIVER_IDENTIFIER) } @@ -252,7 +254,12 @@ class HeartbeatReceiverSuite /** * Dummy RPC endpoint to simulate executors. */ -private class FakeExecutorEndpoint(override val rpcEnv: RpcEnv) extends RpcEndpoint +private class FakeExecutorEndpoint(override val rpcEnv: RpcEnv) extends RpcEndpoint { + + override def receive: PartialFunction[Any, Unit] = { + case _ => + } +} /** * Dummy scheduler backend to simulate executor allocation requests to the cluster manager. @@ -263,13 +270,13 @@ private class FakeSchedulerBackend( clusterManagerEndpoint: RpcEndpointRef) extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { - protected override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { - clusterManagerEndpoint.askWithRetry[Boolean]( - RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount)) + protected override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { + clusterManagerEndpoint.ask[Boolean]( + RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount, Set.empty[String])) } - protected override def doKillExecutors(executorIds: Seq[String]): Boolean = { - clusterManagerEndpoint.askWithRetry[Boolean](KillExecutors(executorIds)) + protected override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = { + clusterManagerEndpoint.ask[Boolean](KillExecutors(executorIds)) } } @@ -284,7 +291,7 @@ private class FakeClusterManager(override val rpcEnv: RpcEnv) extends RpcEndpoin def getExecutorIdsToKill: Set[String] = executorIdsToKill.toSet override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RequestExecutors(requestedTotal, _, _) => + case RequestExecutors(requestedTotal, _, _, _) => targetNumExecutors = requestedTotal context.reply(true) case KillExecutors(executorIds) => diff --git a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala index 4399f2562647..b9d18119b5a0 100644 --- a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD class ImplicitOrderingSuite extends SparkFunSuite with LocalSparkContext { // Tests that PairRDDFunctions grabs an implicit Ordering in various cases where it should. - test("basic inference of Orderings"){ + test("basic inference of Orderings") { sc = new SparkContext("local", "test") val rdd = sc.parallelize(1 to 10) @@ -30,11 +30,11 @@ class ImplicitOrderingSuite extends SparkFunSuite with LocalSparkContext { // Infer orderings after basic maps to particular types val basicMapExpectations = ImplicitOrderingSuite.basicMapExpectations(rdd) - basicMapExpectations.map({case (met, explain) => assert(met, explain)}) + basicMapExpectations.foreach { case (met, explain) => assert(met, explain) } // Infer orderings for other RDD methods val otherRDDMethodExpectations = ImplicitOrderingSuite.otherRDDMethodExpectations(rdd) - otherRDDMethodExpectations.map({case (met, explain) => assert(met, explain)}) + otherRDDMethodExpectations.foreach { case (met, explain) => assert(met, explain) } } } diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala new file mode 100644 index 000000000000..8d7be77f51fe --- /dev/null +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -0,0 +1,224 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler.AccumulableInfo +import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2} + + +class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { + import InternalAccumulator._ + + override def afterEach(): Unit = { + try { + AccumulatorContext.clear() + } finally { + super.afterEach() + } + } + + test("internal accumulators in TaskContext") { + val taskContext = TaskContext.empty() + val accumUpdates = taskContext.taskMetrics.accumulators() + assert(accumUpdates.size > 0) + val testAccum = taskContext.taskMetrics.testAccum.get + assert(accumUpdates.exists(_.id == testAccum.id)) + } + + test("internal accumulators in a stage") { + val listener = new SaveInfoListener + val numPartitions = 10 + sc = new SparkContext("local", "test") + sc.addSparkListener(listener) + // Have each task add 1 to the internal accumulator + val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitions { iter => + TaskContext.get().taskMetrics().testAccum.get.add(1) + iter + } + // Register asserts in job completion callback to avoid flakiness + listener.registerJobCompletionCallback { () => + val stageInfos = listener.getCompletedStageInfos + val taskInfos = listener.getCompletedTaskInfos + assert(stageInfos.size === 1) + assert(taskInfos.size === numPartitions) + // The accumulator values should be merged in the stage + val stageAccum = findTestAccum(stageInfos.head.accumulables.values) + assert(stageAccum.value.get.asInstanceOf[Long] === numPartitions) + // The accumulator should be updated locally on each task + val taskAccumValues = taskInfos.map { taskInfo => + val taskAccum = findTestAccum(taskInfo.accumulables) + assert(taskAccum.update.isDefined) + assert(taskAccum.update.get.asInstanceOf[Long] === 1L) + taskAccum.value.get.asInstanceOf[Long] + } + // Each task should keep track of the partial value on the way, i.e. 1, 2, ... numPartitions + assert(taskAccumValues.sorted === (1L to numPartitions).toSeq) + } + rdd.count() + listener.awaitNextJobCompletion() + } + + test("internal accumulators in multiple stages") { + val listener = new SaveInfoListener + val numPartitions = 10 + sc = new SparkContext("local", "test") + sc.addSparkListener(listener) + // Each stage creates its own set of internal accumulators so the + // values for the same metric should not be mixed up across stages + val rdd = sc.parallelize(1 to 100, numPartitions) + .map { i => (i, i) } + .mapPartitions { iter => + TaskContext.get().taskMetrics().testAccum.get.add(1) + iter + } + .reduceByKey { case (x, y) => x + y } + .mapPartitions { iter => + TaskContext.get().taskMetrics().testAccum.get.add(10) + iter + } + .repartition(numPartitions * 2) + .mapPartitions { iter => + TaskContext.get().taskMetrics().testAccum.get.add(100) + iter + } + // Register asserts in job completion callback to avoid flakiness + listener.registerJobCompletionCallback { () => + // We ran 3 stages, and the accumulator values should be distinct + val stageInfos = listener.getCompletedStageInfos + assert(stageInfos.size === 3) + val (firstStageAccum, secondStageAccum, thirdStageAccum) = + (findTestAccum(stageInfos(0).accumulables.values), + findTestAccum(stageInfos(1).accumulables.values), + findTestAccum(stageInfos(2).accumulables.values)) + assert(firstStageAccum.value.get.asInstanceOf[Long] === numPartitions) + assert(secondStageAccum.value.get.asInstanceOf[Long] === numPartitions * 10) + assert(thirdStageAccum.value.get.asInstanceOf[Long] === numPartitions * 2 * 100) + } + rdd.count() + } + + test("internal accumulators in resubmitted stages") { + val listener = new SaveInfoListener + val numPartitions = 10 + sc = new SparkContext("local", "test") + sc.addSparkListener(listener) + + // Simulate fetch failures in order to trigger a stage retry. Here we run 1 job with + // 2 stages. On the second stage, we trigger a fetch failure on the first stage attempt. + // This should retry both stages in the scheduler. Note that we only want to fail the + // first stage attempt because we want the stage to eventually succeed. + val x = sc.parallelize(1 to 100, numPartitions) + .mapPartitions { iter => TaskContext.get().taskMetrics().testAccum.get.add(1); iter } + .groupBy(identity) + val sid = x.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle.shuffleId + val rdd = x.mapPartitionsWithIndex { case (i, iter) => + // Fail the first stage attempt. Here we use the task attempt ID to determine this. + // This job runs 2 stages, and we're in the second stage. Therefore, any task attempt + // ID that's < 2 * numPartitions belongs to the first attempt of this stage. + val taskContext = TaskContext.get() + val isFirstStageAttempt = taskContext.taskAttemptId() < numPartitions * 2 + if (isFirstStageAttempt) { + throw new FetchFailedException( + SparkEnv.get.blockManager.blockManagerId, + sid, + taskContext.partitionId(), + taskContext.partitionId(), + "simulated fetch failure") + } else { + iter + } + } + + // Register asserts in job completion callback to avoid flakiness + listener.registerJobCompletionCallback { () => + val stageInfos = listener.getCompletedStageInfos + assert(stageInfos.size === 4) // 1 shuffle map stage + 1 result stage, both are retried + val mapStageId = stageInfos.head.stageId + val mapStageInfo1stAttempt = stageInfos.head + val mapStageInfo2ndAttempt = { + stageInfos.tail.find(_.stageId == mapStageId).getOrElse { + fail("expected two attempts of the same shuffle map stage.") + } + } + val stageAccum1stAttempt = findTestAccum(mapStageInfo1stAttempt.accumulables.values) + val stageAccum2ndAttempt = findTestAccum(mapStageInfo2ndAttempt.accumulables.values) + // Both map stages should have succeeded, since the fetch failure happened in the + // result stage, not the map stage. This means we should get the accumulator updates + // from all partitions. + assert(stageAccum1stAttempt.value.get.asInstanceOf[Long] === numPartitions) + assert(stageAccum2ndAttempt.value.get.asInstanceOf[Long] === numPartitions) + // Because this test resubmitted the map stage with all missing partitions, we should have + // created a fresh set of internal accumulators in the 2nd stage attempt. Assert this is + // the case by comparing the accumulator IDs between the two attempts. + // Note: it would be good to also test the case where the map stage is resubmitted where + // only a subset of the original partitions are missing. However, this scenario is very + // difficult to construct without potentially introducing flakiness. + assert(stageAccum1stAttempt.id != stageAccum2ndAttempt.id) + } + rdd.count() + listener.awaitNextJobCompletion() + } + + test("internal accumulators are registered for cleanups") { + sc = new SparkContext("local", "test") { + private val myCleaner = new SaveAccumContextCleaner(this) + override def cleaner: Option[ContextCleaner] = Some(myCleaner) + } + assert(AccumulatorContext.numAccums == 0) + sc.parallelize(1 to 100).map { i => (i, i) }.reduceByKey { _ + _ }.count() + val numInternalAccums = TaskMetrics.empty.internalAccums.length + // We ran 2 stages, so we should have 2 sets of internal accumulators, 1 for each stage + assert(AccumulatorContext.numAccums === numInternalAccums * 2) + val accumsRegistered = sc.cleaner match { + case Some(cleaner: SaveAccumContextCleaner) => cleaner.accumsRegisteredForCleanup + case _ => Seq.empty[Long] + } + // Make sure the same set of accumulators is registered for cleanup + assert(accumsRegistered.size === numInternalAccums * 2) + assert(accumsRegistered.toSet.size === AccumulatorContext.numAccums) + accumsRegistered.foreach(id => assert(AccumulatorContext.get(id) != None)) + } + + /** + * Return the accumulable info that matches the specified name. + */ + private def findTestAccum(accums: Iterable[AccumulableInfo]): AccumulableInfo = { + accums.find { a => a.name == Some(TEST_ACCUM) }.getOrElse { + fail(s"unable to find internal accumulator called $TEST_ACCUM") + } + } + + /** + * A special [[ContextCleaner]] that saves the IDs of the accumulators registered for cleanup. + */ + private class SaveAccumContextCleaner(sc: SparkContext) extends ContextCleaner(sc) { + private val accumsRegistered = new ArrayBuffer[Long] + + override def registerAccumulatorForCleanup(a: AccumulatorV2[_, _]): Unit = { + accumsRegistered += a.id + super.registerAccumulatorForCleanup(a) + } + + def accumsRegisteredForCleanup: Seq[Long] = accumsRegistered.toArray + } + +} diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 1168eb0b802f..8a77aea75a99 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -19,15 +19,15 @@ package org.apache.spark import java.util.concurrent.Semaphore -import scala.concurrent.Await import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.Future import scala.concurrent.duration._ -import scala.concurrent.future import org.scalatest.BeforeAndAfter import org.scalatest.Matchers import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart} +import org.apache.spark.util.ThreadUtils /** * Test suite for cancelling running jobs. We run the cancellation tasks for single job action @@ -38,8 +38,11 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft with LocalSparkContext { override def afterEach() { - super.afterEach() - resetSparkContext() + try { + resetSparkContext() + } finally { + super.afterEach() + } } test("local mode, FIFO scheduler") { @@ -100,7 +103,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft val rdd1 = rdd.map(x => x) - future { + Future { taskStartedSemaphore.acquire() sc.cancelAllJobs() taskCancelledSemaphore.release(100000) @@ -123,7 +126,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft }) // jobA is the one to be cancelled. - val jobA = future { + val jobA = Future { sc.setJobGroup("jobA", "this is a job to be cancelled") sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.count() } @@ -134,7 +137,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft sc.clearJobGroup() val jobB = sc.parallelize(1 to 100, 2).countAsync() sc.cancelJobGroup("jobA") - val e = intercept[SparkException] { Await.result(jobA, Duration.Inf) } + val e = intercept[SparkException] { ThreadUtils.awaitResult(jobA, Duration.Inf) }.getCause assert(e.getMessage contains "cancel") // Once A is cancelled, job B should finish fairly quickly. @@ -188,7 +191,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft }) // jobA is the one to be cancelled. - val jobA = future { + val jobA = Future { sc.setJobGroup("jobA", "this is a job to be cancelled", interruptOnCancel = true) sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(100000); i }.count() } @@ -199,13 +202,90 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft sc.clearJobGroup() val jobB = sc.parallelize(1 to 100, 2).countAsync() sc.cancelJobGroup("jobA") - val e = intercept[SparkException] { Await.result(jobA, 5.seconds) } + val e = intercept[SparkException] { ThreadUtils.awaitResult(jobA, 5.seconds) }.getCause assert(e.getMessage contains "cancel") // Once A is cancelled, job B should finish fairly quickly. assert(jobB.get() === 100) } + test("task reaper kills JVM if killed tasks keep running for too long") { + val conf = new SparkConf() + .set("spark.task.reaper.enabled", "true") + .set("spark.task.reaper.killTimeout", "5s") + sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) + + // Add a listener to release the semaphore once any tasks are launched. + val sem = new Semaphore(0) + sc.addSparkListener(new SparkListener { + override def onTaskStart(taskStart: SparkListenerTaskStart) { + sem.release() + } + }) + + // jobA is the one to be cancelled. + val jobA = Future { + sc.setJobGroup("jobA", "this is a job to be cancelled", interruptOnCancel = true) + sc.parallelize(1 to 10000, 2).map { i => + while (true) { } + }.count() + } + + // Block until both tasks of job A have started and cancel job A. + sem.acquire(2) + // Small delay to ensure tasks actually start executing the task body + Thread.sleep(1000) + + sc.clearJobGroup() + val jobB = sc.parallelize(1 to 100, 2).countAsync() + sc.cancelJobGroup("jobA") + val e = intercept[SparkException] { ThreadUtils.awaitResult(jobA, 15.seconds) }.getCause + assert(e.getMessage contains "cancel") + + // Once A is cancelled, job B should finish fairly quickly. + assert(ThreadUtils.awaitResult(jobB, 60.seconds) === 100) + } + + test("task reaper will not kill JVM if spark.task.killTimeout == -1") { + val conf = new SparkConf() + .set("spark.task.reaper.enabled", "true") + .set("spark.task.reaper.killTimeout", "-1") + .set("spark.task.reaper.PollingInterval", "1s") + .set("spark.deploy.maxExecutorRetries", "1") + sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) + + // Add a listener to release the semaphore once any tasks are launched. + val sem = new Semaphore(0) + sc.addSparkListener(new SparkListener { + override def onTaskStart(taskStart: SparkListenerTaskStart) { + sem.release() + } + }) + + // jobA is the one to be cancelled. + val jobA = Future { + sc.setJobGroup("jobA", "this is a job to be cancelled", interruptOnCancel = true) + sc.parallelize(1 to 2, 2).map { i => + val startTime = System.currentTimeMillis() + while (System.currentTimeMillis() < startTime + 10000) { } + }.count() + } + + // Block until both tasks of job A have started and cancel job A. + sem.acquire(2) + // Small delay to ensure tasks actually start executing the task body + Thread.sleep(1000) + + sc.clearJobGroup() + val jobB = sc.parallelize(1 to 100, 2).countAsync() + sc.cancelJobGroup("jobA") + val e = intercept[SparkException] { ThreadUtils.awaitResult(jobA, 15.seconds) }.getCause + assert(e.getMessage contains "cancel") + + // Once A is cancelled, job B should finish fairly quickly. + assert(ThreadUtils.awaitResult(jobB, 60.seconds) === 100) + } + test("two jobs sharing the same stage") { // sem1: make sure cancel is issued after some tasks are launched // twoJobsSharingStageSemaphore: @@ -228,7 +308,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft val f2 = rdd.countAsync() // Kill one of the action. - future { + Future { sem1.acquire() f1.cancel() JobCancellationSuite.twoJobsSharingStageSemaphore.release(10) @@ -244,8 +324,8 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft // Cancel before launching any tasks { val f = sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.countAsync() - future { f.cancel() } - val e = intercept[SparkException] { f.get() } + Future { f.cancel() } + val e = intercept[SparkException] { f.get() }.getCause assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed")) } @@ -260,12 +340,12 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft }) val f = sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.countAsync() - future { + Future { // Wait until some tasks were launched before we cancel the job. sem.acquire() f.cancel() } - val e = intercept[SparkException] { f.get() } + val e = intercept[SparkException] { f.get() }.getCause assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed")) } } @@ -274,8 +354,8 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft // Cancel before launching any tasks { val f = sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.takeAsync(5000) - future { f.cancel() } - val e = intercept[SparkException] { f.get() } + Future { f.cancel() } + val e = intercept[SparkException] { f.get() }.getCause assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed")) } @@ -289,11 +369,11 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft } }) val f = sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.takeAsync(5000) - future { + Future { sem.acquire() f.cancel() } - val e = intercept[SparkException] { f.get() } + val e = intercept[SparkException] { f.get() }.getCause assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed")) } } diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 8bf2e55defd0..1dd89bcbe36b 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -17,24 +17,27 @@ package org.apache.spark -import _root_.io.netty.util.internal.logging.{Slf4JLoggerFactory, InternalLoggerFactory} +import _root_.io.netty.util.internal.logging.{InternalLoggerFactory, Slf4JLoggerFactory} import org.scalatest.BeforeAndAfterAll import org.scalatest.BeforeAndAfterEach import org.scalatest.Suite -/** Manages a local `sc` {@link SparkContext} variable, correctly stopping it after each test. */ +/** Manages a local `sc` `SparkContext` variable, correctly stopping it after each test. */ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self: Suite => @transient var sc: SparkContext = _ override def beforeAll() { - InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory()) super.beforeAll() + InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory()) } override def afterEach() { - resetSparkContext() - super.afterEach() + try { + resetSparkContext() + } finally { + super.afterEach() + } } def resetSparkContext(): Unit = { @@ -49,7 +52,7 @@ object LocalSparkContext { if (sc != null) { sc.stop() } - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + // To avoid RPC rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 7e70308bb360..ebd826b0ba2f 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -19,10 +19,12 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer +import org.mockito.Matchers.any import org.mockito.Mockito._ -import org.mockito.Matchers.{any, isA} -import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcCallContext, RpcEnv} +import org.apache.spark.LocalSparkContext._ +import org.apache.spark.broadcast.BroadcastManager +import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEnv} import org.apache.spark.scheduler.{CompressedMapStatus, MapStatus} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{BlockManagerId, ShuffleBlockId} @@ -30,6 +32,12 @@ import org.apache.spark.storage.{BlockManagerId, ShuffleBlockId} class MapOutputTrackerSuite extends SparkFunSuite { private val conf = new SparkConf + private def newTrackerMaster(sparkConf: SparkConf = conf) = { + val broadcastManager = new BroadcastManager(true, sparkConf, + new SecurityManager(sparkConf)) + new MapOutputTrackerMaster(sparkConf, broadcastManager, true) + } + def createRpcEnv(name: String, host: String = "localhost", port: Int = 0, securityManager: SecurityManager = new SecurityManager(conf)): RpcEnv = { RpcEnv.create(name, host, port, conf, securityManager) @@ -37,7 +45,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { test("master start and stop") { val rpcEnv = createRpcEnv("test") - val tracker = new MapOutputTrackerMaster(conf) + val tracker = newTrackerMaster() tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.stop() @@ -46,7 +54,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { test("master register shuffle and fetch") { val rpcEnv = createRpcEnv("test") - val tracker = new MapOutputTrackerMaster(conf) + val tracker = newTrackerMaster() tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.registerShuffle(10, 2) @@ -62,13 +70,14 @@ class MapOutputTrackerSuite extends SparkFunSuite { Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))), (BlockManagerId("b", "hostB", 1000), ArrayBuffer((ShuffleBlockId(10, 1, 0), size10000)))) .toSet) + assert(0 == tracker.getNumCachedSerializedBroadcast) tracker.stop() rpcEnv.shutdown() } test("master register and unregister shuffle") { val rpcEnv = createRpcEnv("test") - val tracker = new MapOutputTrackerMaster(conf) + val tracker = newTrackerMaster() tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.registerShuffle(10, 2) @@ -80,6 +89,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { Array(compressedSize10000, compressedSize1000))) assert(tracker.containsShuffle(10)) assert(tracker.getMapSizesByExecutorId(10, 0).nonEmpty) + assert(0 == tracker.getNumCachedSerializedBroadcast) tracker.unregisterShuffle(10) assert(!tracker.containsShuffle(10)) assert(tracker.getMapSizesByExecutorId(10, 0).isEmpty) @@ -90,7 +100,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { test("master register shuffle and unregister map output and fetch") { val rpcEnv = createRpcEnv("test") - val tracker = new MapOutputTrackerMaster(conf) + val tracker = newTrackerMaster() tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) tracker.registerShuffle(10, 2) @@ -101,6 +111,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(compressedSize10000, compressedSize1000, compressedSize1000))) + assert(0 == tracker.getNumCachedSerializedBroadcast) // As if we had two simultaneous fetch failures tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) @@ -118,35 +129,37 @@ class MapOutputTrackerSuite extends SparkFunSuite { val hostname = "localhost" val rpcEnv = createRpcEnv("spark", hostname, 0, new SecurityManager(conf)) - val masterTracker = new MapOutputTrackerMaster(conf) + val masterTracker = newTrackerMaster() masterTracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, conf)) val slaveRpcEnv = createRpcEnv("spark-slave", hostname, 0, new SecurityManager(conf)) val slaveTracker = new MapOutputTrackerWorker(conf) slaveTracker.trackerEndpoint = - slaveRpcEnv.setupEndpointRef("spark", rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) + slaveRpcEnv.setupEndpointRef(rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) masterTracker.registerShuffle(10, 1) - masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) + // This is expected to fail because no outputs have been registered for the shuffle. intercept[FetchFailedException] { slaveTracker.getMapSizesByExecutorId(10, 0) } val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) masterTracker.registerMapOutput(10, 0, MapStatus( BlockManagerId("a", "hostA", 1000), Array(1000L))) - masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getMapSizesByExecutorId(10, 0) === Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))))) + assert(0 == masterTracker.getNumCachedSerializedBroadcast) + val masterTrackerEpochBeforeLossOfMapOutput = masterTracker.getEpoch masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) - masterTracker.incrementEpoch() + assert(masterTracker.getEpoch > masterTrackerEpochBeforeLossOfMapOutput) slaveTracker.updateEpoch(masterTracker.getEpoch) intercept[FetchFailedException] { slaveTracker.getMapSizesByExecutorId(10, 0) } // failure should be cached intercept[FetchFailedException] { slaveTracker.getMapSizesByExecutorId(10, 0) } + assert(0 == masterTracker.getNumCachedSerializedBroadcast) masterTracker.stop() slaveTracker.stop() @@ -154,17 +167,19 @@ class MapOutputTrackerSuite extends SparkFunSuite { slaveRpcEnv.shutdown() } - test("remote fetch below akka frame size") { + test("remote fetch below max RPC message size") { val newConf = new SparkConf - newConf.set("spark.akka.frameSize", "1") + newConf.set("spark.rpc.message.maxSize", "1") newConf.set("spark.rpc.askTimeout", "1") // Fail fast + newConf.set("spark.shuffle.mapOutput.minSizeForBroadcast", "1048576") - val masterTracker = new MapOutputTrackerMaster(conf) + val masterTracker = newTrackerMaster(newConf) val rpcEnv = createRpcEnv("spark") val masterEndpoint = new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, newConf) - rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) + masterTracker.trackerEndpoint = + rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) - // Frame size should be ~123B, and no exception should be thrown + // Message size should be ~123B, and no exception should be thrown masterTracker.registerShuffle(10, 1) masterTracker.registerMapOutput(10, 0, MapStatus( BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0))) @@ -172,45 +187,27 @@ class MapOutputTrackerSuite extends SparkFunSuite { val rpcCallContext = mock(classOf[RpcCallContext]) when(rpcCallContext.senderAddress).thenReturn(senderAddress) masterEndpoint.receiveAndReply(rpcCallContext)(GetMapOutputStatuses(10)) - verify(rpcCallContext).reply(any()) - verify(rpcCallContext, never()).sendFailure(any()) + // Default size for broadcast in this testsuite is set to -1 so should not cause broadcast + // to be used. + verify(rpcCallContext, timeout(30000)).reply(any()) + assert(0 == masterTracker.getNumCachedSerializedBroadcast) -// masterTracker.stop() // this throws an exception + masterTracker.stop() rpcEnv.shutdown() } - test("remote fetch exceeds akka frame size") { + test("min broadcast size exceeds max RPC message size") { val newConf = new SparkConf - newConf.set("spark.akka.frameSize", "1") + newConf.set("spark.rpc.message.maxSize", "1") newConf.set("spark.rpc.askTimeout", "1") // Fail fast + newConf.set("spark.shuffle.mapOutput.minSizeForBroadcast", Int.MaxValue.toString) - val masterTracker = new MapOutputTrackerMaster(conf) - val rpcEnv = createRpcEnv("test") - val masterEndpoint = new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, newConf) - rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) - - // Frame size should be ~1.1MB, and MapOutputTrackerMasterEndpoint should throw exception. - // Note that the size is hand-selected here because map output statuses are compressed before - // being sent. - masterTracker.registerShuffle(20, 100) - (0 until 100).foreach { i => - masterTracker.registerMapOutput(20, i, new CompressedMapStatus( - BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) - } - val senderAddress = RpcAddress("localhost", 12345) - val rpcCallContext = mock(classOf[RpcCallContext]) - when(rpcCallContext.senderAddress).thenReturn(senderAddress) - masterEndpoint.receiveAndReply(rpcCallContext)(GetMapOutputStatuses(20)) - verify(rpcCallContext, never()).reply(any()) - verify(rpcCallContext).sendFailure(isA(classOf[SparkException])) - -// masterTracker.stop() // this throws an exception - rpcEnv.shutdown() + intercept[IllegalArgumentException] { newTrackerMaster(newConf) } } test("getLocationsWithLargestOutputs with multiple outputs in same machine") { val rpcEnv = createRpcEnv("test") - val tracker = new MapOutputTrackerMaster(conf) + val tracker = newTrackerMaster() tracker.trackerEndpoint = rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, new MapOutputTrackerMasterEndpoint(rpcEnv, tracker, conf)) // Setup 3 map tasks @@ -242,4 +239,40 @@ class MapOutputTrackerSuite extends SparkFunSuite { tracker.stop() rpcEnv.shutdown() } + + test("remote fetch using broadcast") { + val newConf = new SparkConf + newConf.set("spark.rpc.message.maxSize", "1") + newConf.set("spark.rpc.askTimeout", "1") // Fail fast + newConf.set("spark.shuffle.mapOutput.minSizeForBroadcast", "10240") // 10 KB << 1MB framesize + + // needs TorrentBroadcast so need a SparkContext + withSpark(new SparkContext("local", "MapOutputTrackerSuite", newConf)) { sc => + val masterTracker = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + val rpcEnv = sc.env.rpcEnv + val masterEndpoint = new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, newConf) + rpcEnv.stop(masterTracker.trackerEndpoint) + rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) + + // Frame size should be ~1.1MB, and MapOutputTrackerMasterEndpoint should throw exception. + // Note that the size is hand-selected here because map output statuses are compressed before + // being sent. + masterTracker.registerShuffle(20, 100) + (0 until 100).foreach { i => + masterTracker.registerMapOutput(20, i, new CompressedMapStatus( + BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) + } + val senderAddress = RpcAddress("localhost", 12345) + val rpcCallContext = mock(classOf[RpcCallContext]) + when(rpcCallContext.senderAddress).thenReturn(senderAddress) + masterEndpoint.receiveAndReply(rpcCallContext)(GetMapOutputStatuses(20)) + // should succeed since majority of data is broadcast and actual serialized + // message size is small + verify(rpcCallContext, timeout(30000)).reply(any()) + assert(1 == masterTracker.getNumCachedSerializedBroadcast) + masterTracker.unregisterShuffle(20) + assert(0 == masterTracker.getNumCachedSerializedBroadcast) + } + } + } diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index aa8028792cb4..dfe4c25670ce 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -71,9 +71,9 @@ class PartitioningSuite extends SparkFunSuite with SharedSparkContext with Priva val partitionSizes = List(1, 2, 10, 100, 500, 1000, 1500) val partitioners = partitionSizes.map(p => (p, new RangePartitioner(p, rdd))) val decoratedRangeBounds = PrivateMethod[Array[Int]]('rangeBounds) - partitioners.map { case (numPartitions, partitioner) => + partitioners.foreach { case (numPartitions, partitioner) => val rangeBounds = partitioner.invokePrivate(decoratedRangeBounds()) - 1.to(1000).map { element => { + for (element <- 1 to 1000) { val partition = partitioner.getPartition(element) if (numPartitions > 1) { if (partition < rangeBounds.size) { @@ -85,7 +85,7 @@ class PartitioningSuite extends SparkFunSuite with SharedSparkContext with Priva } else { assert(partition === 0) } - }} + } } } @@ -163,8 +163,8 @@ class PartitioningSuite extends SparkFunSuite with SharedSparkContext with Priva val hashP2 = new HashPartitioner(2) assert(rangeP2 === rangeP2) assert(hashP2 === hashP2) - assert(hashP2 != rangeP2) - assert(rangeP2 != hashP2) + assert(hashP2 !== rangeP2) + assert(rangeP2 !== hashP2) } test("partitioner preservation") { @@ -244,11 +244,21 @@ class PartitioningSuite extends SparkFunSuite with SharedSparkContext with Priva assert(abs(6.0/2 - rdd.mean) < 0.01) assert(abs(1.0 - rdd.variance) < 0.01) assert(abs(1.0 - rdd.stdev) < 0.01) + assert(abs(rdd.variance - rdd.popVariance) < 1e-14) + assert(abs(rdd.stdev - rdd.popStdev) < 1e-14) + assert(abs(2.0 - rdd.sampleVariance) < 1e-14) + assert(abs(Math.sqrt(2.0) - rdd.sampleStdev) < 1e-14) assert(stats.max === 4.0) assert(stats.min === 2.0) // Add other tests here for classes that should be able to handle empty partitions correctly } + + test("Number of elements in RDD is less than number of partitions") { + val rdd = sc.parallelize(1 to 3).map(x => (x, x)) + val partitioner = new RangePartitioner(22, rdd) + assert(partitioner.numPartitions === 3) + } } diff --git a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala index 25b79bce6ab9..8eabc2b3cb95 100644 --- a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala @@ -20,10 +20,10 @@ package org.apache.spark import java.io.File import javax.net.ssl.SSLContext -import com.google.common.io.Files -import org.apache.spark.util.Utils import org.scalatest.BeforeAndAfterAll +import org.apache.spark.util.SparkConfWithEnv + class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll { test("test resolving property file as spark conf ") { @@ -81,7 +81,7 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll { conf.set("spark.ssl.protocol", "SSLv3") val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None) - val opts = SSLOptions.parse(conf, "spark.ui.ssl", defaults = Some(defaultOpts)) + val opts = SSLOptions.parse(conf, "spark.ssl.ui", defaults = Some(defaultOpts)) assert(opts.enabled === true) assert(opts.trustStore.isDefined === true) @@ -104,22 +104,24 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll { val conf = new SparkConf conf.set("spark.ssl.enabled", "true") - conf.set("spark.ui.ssl.enabled", "false") + conf.set("spark.ssl.ui.enabled", "false") + conf.set("spark.ssl.ui.port", "4242") conf.set("spark.ssl.keyStore", keyStorePath) conf.set("spark.ssl.keyStorePassword", "password") - conf.set("spark.ui.ssl.keyStorePassword", "12345") + conf.set("spark.ssl.ui.keyStorePassword", "12345") conf.set("spark.ssl.keyPassword", "password") conf.set("spark.ssl.trustStore", trustStorePath) conf.set("spark.ssl.trustStorePassword", "password") conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") - conf.set("spark.ui.ssl.enabledAlgorithms", "ABC, DEF") + conf.set("spark.ssl.ui.enabledAlgorithms", "ABC, DEF") conf.set("spark.ssl.protocol", "SSLv3") val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None) - val opts = SSLOptions.parse(conf, "spark.ui.ssl", defaults = Some(defaultOpts)) + val opts = SSLOptions.parse(conf, "spark.ssl.ui", defaults = Some(defaultOpts)) assert(opts.enabled === false) + assert(opts.port === Some(4242)) assert(opts.trustStore.isDefined === true) assert(opts.trustStore.get.getName === "truststore") assert(opts.trustStore.get.getAbsolutePath === trustStorePath) @@ -133,4 +135,18 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll { assert(opts.enabledAlgorithms === Set("ABC", "DEF")) } + test("variable substitution") { + val conf = new SparkConfWithEnv(Map( + "ENV1" -> "val1", + "ENV2" -> "val2")) + + conf.set("spark.ssl.enabled", "true") + conf.set("spark.ssl.keyStore", "${env:ENV1}") + conf.set("spark.ssl.trustStore", "${env:ENV2}") + + val opts = SSLOptions.parse(conf, "spark.ssl", defaults = None) + assert(opts.keyStore === Some(new File("val1"))) + assert(opts.trustStore === Some(new File("val2"))) + } + } diff --git a/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala index 2d14249855c9..33270bec6247 100644 --- a/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala +++ b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala @@ -41,7 +41,6 @@ object SSLSampleConfigs { def sparkSSLConfig(): SparkConf = { val conf = new SparkConf(loadDefaults = false) - conf.set("spark.rpc", "akka") conf.set("spark.ssl.enabled", "true") conf.set("spark.ssl.keyStore", keyStorePath) conf.set("spark.ssl.keyStorePassword", "password") @@ -55,7 +54,6 @@ object SSLSampleConfigs { def sparkSSLConfigUntrusted(): SparkConf = { val conf = new SparkConf(loadDefaults = false) - conf.set("spark.rpc", "akka") conf.set("spark.ssl.enabled", "true") conf.set("spark.ssl.keyStore", untrustedKeyStorePath) conf.set("spark.ssl.keyStorePassword", "password") diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index 26b95c06789f..9801b2638cc1 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -19,9 +19,19 @@ package org.apache.spark import java.io.File -import org.apache.spark.util.{SparkConfWithEnv, Utils} +import org.apache.spark.security.GroupMappingServiceProvider +import org.apache.spark.util.{ResetSystemProperties, SparkConfWithEnv, Utils} -class SecurityManagerSuite extends SparkFunSuite { +class DummyGroupMappingServiceProvider extends GroupMappingServiceProvider { + + val userGroups: Set[String] = Set[String]("group1", "group2", "group3") + + override def getGroups(username: String): Set[String] = { + userGroups + } +} + +class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with conf") { val conf = new SparkConf @@ -37,6 +47,45 @@ class SecurityManagerSuite extends SparkFunSuite { assert(securityManager.checkUIViewPermissions("user3") === false) } + test("set security with conf for groups") { + val conf = new SparkConf + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + conf.set("spark.ui.acls.enable", "true") + conf.set("spark.ui.view.acls.groups", "group1,group2") + // default ShellBasedGroupsMappingProvider is used to resolve user groups + val securityManager = new SecurityManager(conf); + // assuming executing user does not belong to group1,group2 + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + + val conf2 = new SparkConf + conf2.set("spark.authenticate", "true") + conf2.set("spark.authenticate.secret", "good") + conf2.set("spark.ui.acls.enable", "true") + conf2.set("spark.ui.view.acls.groups", "group1,group2") + // explicitly specify a custom GroupsMappingServiceProvider + conf2.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager2 = new SecurityManager(conf2); + // group4,group5 do not match + assert(securityManager2.checkUIViewPermissions("user1") === true) + assert(securityManager2.checkUIViewPermissions("user2") === true) + + val conf3 = new SparkConf + conf3.set("spark.authenticate", "true") + conf3.set("spark.authenticate.secret", "good") + conf3.set("spark.ui.acls.enable", "true") + conf3.set("spark.ui.view.acls.groups", "group4,group5") + // explicitly specify a bogus GroupsMappingServiceProvider + conf3.set("spark.user.groups.mapping", "BogusServiceProvider") + + val securityManager3 = new SecurityManager(conf3); + // BogusServiceProvider cannot be loaded and an error is logged returning an empty group set + assert(securityManager3.checkUIViewPermissions("user1") === false) + assert(securityManager3.checkUIViewPermissions("user2") === false) + } + test("set security with api") { val conf = new SparkConf conf.set("spark.ui.view.acls", "user1,user2") @@ -60,6 +109,40 @@ class SecurityManagerSuite extends SparkFunSuite { assert(securityManager.checkUIViewPermissions(null) === true) } + test("set security with api for groups") { + val conf = new SparkConf + conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + securityManager.setViewAclsGroups("group1,group2") + + // group1,group2 match + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user2") === true) + + // change groups so they do not match + securityManager.setViewAclsGroups("group4,group5") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + + val conf2 = new SparkConf + conf.set("spark.user.groups.mapping", "BogusServiceProvider") + + val securityManager2 = new SecurityManager(conf2) + securityManager2.setAcls(true) + securityManager2.setViewAclsGroups("group1,group2") + + // group1,group2 do not match because of BogusServiceProvider + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + + // setting viewAclsGroups to empty should still not match because of BogusServiceProvider + securityManager2.setViewAclsGroups("") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + } + test("set security modify acls") { val conf = new SparkConf conf.set("spark.modify.acls", "user1,user2") @@ -84,6 +167,29 @@ class SecurityManagerSuite extends SparkFunSuite { assert(securityManager.checkModifyPermissions(null) === true) } + test("set security modify acls for groups") { + val conf = new SparkConf + conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + securityManager.setModifyAclsGroups("group1,group2") + + // group1,group2 match + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkModifyPermissions("user2") === true) + + // change groups so they do not match + securityManager.setModifyAclsGroups("group4,group5") + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user2") === false) + + // change so they match again + securityManager.setModifyAclsGroups("group2,group3") + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkModifyPermissions("user2") === true) + } + test("set security admin acls") { val conf = new SparkConf conf.set("spark.admin.acls", "user1,user2") @@ -122,7 +228,48 @@ class SecurityManagerSuite extends SparkFunSuite { assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user3") === false) assert(securityManager.checkUIViewPermissions(null) === true) + } + + test("set security admin acls for groups") { + val conf = new SparkConf + conf.set("spark.admin.acls.groups", "group1") + conf.set("spark.ui.view.acls.groups", "group2") + conf.set("spark.modify.acls.groups", "group3") + conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + assert(securityManager.aclsEnabled() === true) + // group1,group2,group3 match + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user1") === true) + + // change admin groups so they do not match. view and modify groups are set to admin groups + securityManager.setAdminAclsGroups("group4,group5") + // invoke the set ui and modify to propagate the changes + securityManager.setViewAclsGroups("") + securityManager.setModifyAclsGroups("") + + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user1") === false) + + // change modify groups so they match + securityManager.setModifyAclsGroups("group3") + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user1") === false) + + // change view groups so they match + securityManager.setViewAclsGroups("group2") + securityManager.setModifyAclsGroups("group4") + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user1") === true) + + // change modify and view groups so they do not match + securityManager.setViewAclsGroups("group7") + securityManager.setModifyAclsGroups("group8") + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user1") === false) } test("set security with * in acls") { @@ -166,6 +313,57 @@ class SecurityManagerSuite extends SparkFunSuite { assert(securityManager.checkModifyPermissions("user8") === true) } + test("set security with * in acls for groups") { + val conf = new SparkConf + conf.set("spark.ui.acls.enable", "true") + conf.set("spark.admin.acls.groups", "group4,group5") + conf.set("spark.ui.view.acls.groups", "*") + conf.set("spark.modify.acls.groups", "group6") + + val securityManager = new SecurityManager(conf) + assert(securityManager.aclsEnabled() === true) + + // check for viewAclsGroups with * + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user2") === true) + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user2") === false) + + // check for modifyAcls with * + securityManager.setModifyAclsGroups("*") + securityManager.setViewAclsGroups("group6") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user2") === false) + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkModifyPermissions("user2") === true) + + // check for adminAcls with * + securityManager.setAdminAclsGroups("group9,*") + securityManager.setModifyAclsGroups("group4,group5") + securityManager.setViewAclsGroups("group6,group7") + assert(securityManager.checkUIViewPermissions("user5") === true) + assert(securityManager.checkUIViewPermissions("user6") === true) + assert(securityManager.checkModifyPermissions("user7") === true) + assert(securityManager.checkModifyPermissions("user8") === true) + } + + test("security for groups default behavior") { + // no groups or userToGroupsMapper provided + // this will default to the ShellBasedGroupsMappingProvider + val conf = new SparkConf + + val securityManager = new SecurityManager(conf) + securityManager.setAcls(true) + + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user1") === false) + + // set groups only + securityManager.setAdminAclsGroups("group1,group2") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user1") === false) + } + test("ssl on setup") { val conf = SSLSampleConfigs.sparkSSLConfig() val expectedAlgorithms = Set( @@ -183,7 +381,6 @@ class SecurityManagerSuite extends SparkFunSuite { val securityManager = new SecurityManager(conf) assert(securityManager.fileServerSSLOptions.enabled === true) - assert(securityManager.akkaSSLOptions.enabled === true) assert(securityManager.sslSocketFactory.isDefined === true) assert(securityManager.hostnameVerifier.isDefined === true) @@ -197,16 +394,6 @@ class SecurityManagerSuite extends SparkFunSuite { assert(securityManager.fileServerSSLOptions.keyPassword === Some("password")) assert(securityManager.fileServerSSLOptions.protocol === Some("TLSv1.2")) assert(securityManager.fileServerSSLOptions.enabledAlgorithms === expectedAlgorithms) - - assert(securityManager.akkaSSLOptions.trustStore.isDefined === true) - assert(securityManager.akkaSSLOptions.trustStore.get.getName === "truststore") - assert(securityManager.akkaSSLOptions.keyStore.isDefined === true) - assert(securityManager.akkaSSLOptions.keyStore.get.getName === "keystore") - assert(securityManager.akkaSSLOptions.trustStorePassword === Some("password")) - assert(securityManager.akkaSSLOptions.keyStorePassword === Some("password")) - assert(securityManager.akkaSSLOptions.keyPassword === Some("password")) - assert(securityManager.akkaSSLOptions.protocol === Some("TLSv1.2")) - assert(securityManager.akkaSSLOptions.enabledAlgorithms === expectedAlgorithms) } test("ssl off setup") { @@ -218,7 +405,6 @@ class SecurityManagerSuite extends SparkFunSuite { val securityManager = new SecurityManager(conf) assert(securityManager.fileServerSSLOptions.enabled === false) - assert(securityManager.akkaSSLOptions.enabled === false) assert(securityManager.sslSocketFactory.isDefined === false) assert(securityManager.hostnameVerifier.isDefined === false) } diff --git a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala index 3d2700b7e6be..6aedcb1271ff 100644 --- a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala @@ -17,11 +17,11 @@ package org.apache.spark -import org.scalatest.BeforeAndAfterAll +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} import org.scalatest.Suite /** Shares a local `SparkContext` between all tests in a suite and closes it at the end */ -trait SharedSparkContext extends BeforeAndAfterAll { self: Suite => +trait SharedSparkContext extends BeforeAndAfterAll with BeforeAndAfterEach { self: Suite => @transient private var _sc: SparkContext = _ @@ -30,13 +30,27 @@ trait SharedSparkContext extends BeforeAndAfterAll { self: Suite => var conf = new SparkConf(false) override def beforeAll() { - _sc = new SparkContext("local[4]", "test", conf) super.beforeAll() + _sc = new SparkContext( + "local[4]", "test", conf.set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName)) } override def afterAll() { - LocalSparkContext.stop(_sc) - _sc = null - super.afterAll() + try { + LocalSparkContext.stop(_sc) + _sc = null + } finally { + super.afterAll() + } + } + + protected override def beforeEach(): Unit = { + super.beforeEach() + DebugFilesystem.clearOpenStreams() + } + + protected override def afterEach(): Unit = { + super.afterEach() + DebugFilesystem.assertNoOpenStreams() } } diff --git a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala index d78c99c2e1e0..73638d9b131e 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala @@ -24,6 +24,7 @@ class ShuffleNettySuite extends ShuffleSuite with BeforeAndAfterAll { // This test suite should run all tests in ShuffleSuite with Netty shuffle mode. override def beforeAll() { + super.beforeAll() conf.set("spark.shuffle.blockTransferService", "netty") } } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 4a0877d86f2c..3931d53b4ae0 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -17,14 +17,19 @@ package org.apache.spark +import java.util.{Locale, Properties} +import java.util.concurrent.{Callable, CyclicBarrier, Executors, ExecutorService} + import org.scalatest.Matchers import org.apache.spark.ShuffleSuite.NonJavaSerializableClass +import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD} -import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} +import org.apache.spark.scheduler.{MapStatus, MyRDD, SparkListener, SparkListenerTaskEnd} import org.apache.spark.serializer.KryoSerializer -import org.apache.spark.storage.{ShuffleDataBlockId, ShuffleBlockId} -import org.apache.spark.util.MutablePair +import org.apache.spark.shuffle.ShuffleWriter +import org.apache.spark.storage.{ShuffleBlockId, ShuffleDataBlockId, ShuffleIndexBlockId} +import org.apache.spark.util.{MutablePair, Utils} abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkContext { @@ -234,7 +239,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC } assert(thrown.getClass === classOf[SparkException]) - assert(thrown.getMessage.toLowerCase.contains("serializable")) + assert(thrown.getMessage.toLowerCase(Locale.ROOT).contains("serializable")) } test("shuffle with different compression settings (SPARK-3426)") { @@ -272,7 +277,8 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // Delete one of the local shuffle blocks. val hashFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleBlockId(0, 0, 0)) val sortFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleDataBlockId(0, 0, 0)) - assert(hashFile.exists() || sortFile.exists()) + val indexFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleIndexBlockId(0, 0, 0)) + assert(hashFile.exists() || (sortFile.exists() && indexFile.exists())) if (hashFile.exists()) { hashFile.delete() @@ -280,11 +286,36 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC if (sortFile.exists()) { sortFile.delete() } + if (indexFile.exists()) { + indexFile.delete() + } // This count should retry the execution of the previous stage and rerun shuffle. rdd.count() } + test("cannot find its local shuffle file if no execution of the stage and rerun shuffle") { + sc = new SparkContext("local", "test", conf.clone()) + val rdd = sc.parallelize(1 to 10, 1).map((_, 1)).reduceByKey(_ + _) + + // Cannot find one of the local shuffle blocks. + val hashFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleBlockId(0, 0, 0)) + val sortFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleDataBlockId(0, 0, 0)) + val indexFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleIndexBlockId(0, 0, 0)) + assert(!hashFile.exists() && !sortFile.exists() && !indexFile.exists()) + + rdd.count() + + // Can find one of the local shuffle blocks. + val hashExistsFile = sc.env.blockManager.diskBlockManager + .getFile(new ShuffleBlockId(0, 0, 0)) + val sortExistsFile = sc.env.blockManager.diskBlockManager + .getFile(new ShuffleDataBlockId(0, 0, 0)) + val indexExistsFile = sc.env.blockManager.diskBlockManager + .getFile(new ShuffleIndexBlockId(0, 0, 0)) + assert(hashExistsFile.exists() || (sortExistsFile.exists() && indexExistsFile.exists())) + } + test("metrics for shuffle without aggregation") { sc = new SparkContext("local", "test", conf.clone()) val numRecords = 10000 @@ -317,6 +348,105 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC assert(metrics.bytesWritten === metrics.byresRead) assert(metrics.bytesWritten > 0) } + + test("multiple simultaneous attempts for one task (SPARK-8029)") { + sc = new SparkContext("local", "test", conf) + val mapTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + val manager = sc.env.shuffleManager + + val taskMemoryManager = new TaskMemoryManager(sc.env.memoryManager, 0L) + val metricsSystem = sc.env.metricsSystem + val shuffleMapRdd = new MyRDD(sc, 1, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(1)) + val shuffleHandle = manager.registerShuffle(0, 1, shuffleDep) + mapTrackerMaster.registerShuffle(0, 1) + + // first attempt -- its successful + val writer1 = manager.getWriter[Int, Int](shuffleHandle, 0, + new TaskContextImpl(0, 0, 0L, 0, taskMemoryManager, new Properties, metricsSystem)) + val data1 = (1 to 10).map { x => x -> x} + + // second attempt -- also successful. We'll write out different data, + // just to simulate the fact that the records may get written differently + // depending on what gets spilled, what gets combined, etc. + val writer2 = manager.getWriter[Int, Int](shuffleHandle, 0, + new TaskContextImpl(0, 0, 1L, 0, taskMemoryManager, new Properties, metricsSystem)) + val data2 = (11 to 20).map { x => x -> x} + + // interleave writes of both attempts -- we want to test that both attempts can occur + // simultaneously, and everything is still OK + + def writeAndClose( + writer: ShuffleWriter[Int, Int])( + iter: Iterator[(Int, Int)]): Option[MapStatus] = { + val files = writer.write(iter) + writer.stop(true) + } + val interleaver = new InterleaveIterators( + data1, writeAndClose(writer1), data2, writeAndClose(writer2)) + val (mapOutput1, mapOutput2) = interleaver.run() + + // check that we can read the map output and it has the right data + assert(mapOutput1.isDefined) + assert(mapOutput2.isDefined) + assert(mapOutput1.get.location === mapOutput2.get.location) + assert(mapOutput1.get.getSizeForBlock(0) === mapOutput1.get.getSizeForBlock(0)) + + // register one of the map outputs -- doesn't matter which one + mapOutput1.foreach { case mapStatus => + mapTrackerMaster.registerMapOutput(0, 0, mapStatus) + } + + val reader = manager.getReader[Int, Int](shuffleHandle, 0, 1, + new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, new Properties, metricsSystem)) + val readData = reader.read().toIndexedSeq + assert(readData === data1.toIndexedSeq || readData === data2.toIndexedSeq) + + manager.unregisterShuffle(0) + } +} + +/** + * Utility to help tests make sure that we can process two different iterators simultaneously + * in different threads. This makes sure that in your test, you don't completely process data1 with + * f1 before processing data2 with f2 (or vice versa). It adds a barrier so that the functions only + * process one element, before pausing to wait for the other function to "catch up". + */ +class InterleaveIterators[T, R]( + data1: Seq[T], + f1: Iterator[T] => R, + data2: Seq[T], + f2: Iterator[T] => R) { + + require(data1.size == data2.size) + + val barrier = new CyclicBarrier(2) + class BarrierIterator[E](id: Int, sub: Iterator[E]) extends Iterator[E] { + def hasNext: Boolean = sub.hasNext + + def next: E = { + barrier.await() + sub.next() + } + } + + val c1 = new Callable[R] { + override def call(): R = f1(new BarrierIterator(1, data1.iterator)) + } + val c2 = new Callable[R] { + override def call(): R = f2(new BarrierIterator(2, data2.iterator)) + } + + val e: ExecutorService = Executors.newFixedThreadPool(2) + + def run(): (R, R) = { + val future1 = e.submit(c1) + val future2 = e.submit(c2) + val r1 = future1.get() + val r2 = future2.get() + e.shutdown() + (r1, r2) + } } object ShuffleSuite { @@ -344,14 +474,10 @@ object ShuffleSuite { @volatile var bytesRead: Long = 0 val listener = new SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - taskEnd.taskMetrics.shuffleWriteMetrics.foreach { m => - recordsWritten += m.shuffleRecordsWritten - bytesWritten += m.shuffleBytesWritten - } - taskEnd.taskMetrics.shuffleReadMetrics.foreach { m => - recordsRead += m.recordsRead - bytesRead += m.totalBytesRead - } + recordsWritten += taskEnd.taskMetrics.shuffleWriteMetrics.recordsWritten + bytesWritten += taskEnd.taskMetrics.shuffleWriteMetrics.bytesWritten + recordsRead += taskEnd.taskMetrics.shuffleReadMetrics.recordsRead + bytesRead += taskEnd.taskMetrics.shuffleReadMetrics.totalBytesRead } } sc.addSparkListener(listener) diff --git a/core/src/test/scala/org/apache/spark/Smuggle.scala b/core/src/test/scala/org/apache/spark/Smuggle.scala new file mode 100644 index 000000000000..9d9217ea1b48 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/Smuggle.scala @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import java.util.UUID +import java.util.concurrent.locks.ReentrantReadWriteLock + +import scala.collection.mutable +import scala.language.implicitConversions + +/** + * Utility wrapper to "smuggle" objects into tasks while bypassing serialization. + * This is intended for testing purposes, primarily to make locks, semaphores, and + * other constructs that would not survive serialization available from within tasks. + * A Smuggle reference is itself serializable, but after being serialized and + * deserialized, it still refers to the same underlying "smuggled" object, as long + * as it was deserialized within the same JVM. This can be useful for tests that + * depend on the timing of task completion to be deterministic, since one can "smuggle" + * a lock or semaphore into the task, and then the task can block until the test gives + * the go-ahead to proceed via the lock. + */ +class Smuggle[T] private(val key: Symbol) extends Serializable { + def smuggledObject: T = Smuggle.get(key) +} + + +object Smuggle { + /** + * Wraps the specified object to be smuggled into a serialized task without + * being serialized itself. + * + * @param smuggledObject + * @tparam T + * @return Smuggle wrapper around smuggledObject. + */ + def apply[T](smuggledObject: T): Smuggle[T] = { + val key = Symbol(UUID.randomUUID().toString) + lock.writeLock().lock() + try { + smuggledObjects += key -> smuggledObject + } finally { + lock.writeLock().unlock() + } + new Smuggle(key) + } + + private val lock = new ReentrantReadWriteLock + private val smuggledObjects = mutable.WeakHashMap.empty[Symbol, Any] + + private def get[T](key: Symbol) : T = { + lock.readLock().lock() + try { + smuggledObjects(key).asInstanceOf[T] + } finally { + lock.readLock().unlock() + } + } + + /** + * Implicit conversion of a Smuggle wrapper to the object being smuggled. + * + * @param smuggle the wrapper to unpack. + * @tparam T + * @return the smuggled object represented by the wrapper. + */ + implicit def unpackSmuggledObject[T](smuggle : Smuggle[T]): T = smuggle.smuggledObject + +} diff --git a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala index b8ab227517cc..c0126e41ff7f 100644 --- a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala @@ -26,8 +26,8 @@ import org.apache.commons.io.filefilter.TrueFileFilter import org.scalatest.BeforeAndAfterAll import org.apache.spark.rdd.ShuffledRDD -import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.util.Utils class SortShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { @@ -37,10 +37,16 @@ class SortShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { private var tempDir: File = _ override def beforeAll() { + super.beforeAll() + // Once 'spark.local.dir' is set, it is cached. Unless this is manually cleared + // before/after a test, it could return the same directory even if this property + // is configured. + Utils.clearLocalRootDirs() conf.set("spark.shuffle.manager", "sort") } override def beforeEach(): Unit = { + super.beforeEach() tempDir = Utils.createTempDir() conf.set("spark.local.dir", tempDir.getAbsolutePath) } @@ -48,6 +54,7 @@ class SortShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { override def afterEach(): Unit = { try { Utils.deleteRecursively(tempDir) + Utils.clearLocalRootDirs() } finally { super.afterEach() } diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index ff9a92cc0a42..0897891ee175 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -17,18 +17,20 @@ package org.apache.spark -import java.util.concurrent.{TimeUnit, Executors} +import java.util.concurrent.{Executors, TimeUnit} import scala.collection.JavaConverters._ import scala.concurrent.duration._ import scala.language.postfixOps -import scala.util.{Try, Random} +import scala.util.{Random, Try} -import org.apache.spark.network.util.ByteUnit -import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer} -import org.apache.spark.util.{RpcUtils, ResetSystemProperties} import com.esotericsoftware.kryo.Kryo +import org.apache.spark.internal.config._ +import org.apache.spark.network.util.ByteUnit +import org.apache.spark.serializer.{JavaSerializer, KryoRegistrator, KryoSerializer} +import org.apache.spark.util.{ResetSystemProperties, RpcUtils} + class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSystemProperties { test("Test byteString conversion") { val conf = new SparkConf() @@ -50,8 +52,10 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst test("loading from system properties") { System.setProperty("spark.test.testProperty", "2") + System.setProperty("nonspark.test.testProperty", "0") val conf = new SparkConf() assert(conf.get("spark.test.testProperty") === "2") + assert(!conf.contains("nonspark.test.testProperty")) } test("initializing without loading defaults") { @@ -236,7 +240,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst conf.set(newName, "4") assert(conf.get(newName) === "4") - val count = conf.getAll.filter { case (k, v) => k.startsWith("spark.history.") }.size + val count = conf.getAll.count { case (k, v) => k.startsWith("spark.history.") } assert(count === 4) conf.set("spark.yarn.applicationMaster.waitTries", "42") @@ -266,6 +270,58 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst conf.set("spark.akka.lookupTimeout", "4") assert(RpcUtils.lookupRpcTimeout(conf).duration === (4 seconds)) } + + test("SPARK-13727") { + val conf = new SparkConf() + // set the conf in the deprecated way + conf.set("spark.io.compression.lz4.block.size", "12345") + // get the conf in the recommended way + assert(conf.get("spark.io.compression.lz4.blockSize") === "12345") + // we can still get the conf in the deprecated way + assert(conf.get("spark.io.compression.lz4.block.size") === "12345") + // the contains() also works as expected + assert(conf.contains("spark.io.compression.lz4.block.size")) + assert(conf.contains("spark.io.compression.lz4.blockSize")) + assert(conf.contains("spark.io.unknown") === false) + } + + val serializers = Map( + "java" -> new JavaSerializer(new SparkConf()), + "kryo" -> new KryoSerializer(new SparkConf())) + + serializers.foreach { case (name, ser) => + test(s"SPARK-17240: SparkConf should be serializable ($name)") { + val conf = new SparkConf() + conf.set(DRIVER_CLASS_PATH, "${" + DRIVER_JAVA_OPTIONS.key + "}") + conf.set(DRIVER_JAVA_OPTIONS, "test") + + val serializer = ser.newInstance() + val bytes = serializer.serialize(conf) + val deser = serializer.deserialize[SparkConf](bytes) + + assert(conf.get(DRIVER_CLASS_PATH) === deser.get(DRIVER_CLASS_PATH)) + } + } + + test("encryption requires authentication") { + val conf = new SparkConf() + conf.validateSettings() + + conf.set(NETWORK_ENCRYPTION_ENABLED, true) + intercept[IllegalArgumentException] { + conf.validateSettings() + } + + conf.set(NETWORK_ENCRYPTION_ENABLED, false) + conf.set(SASL_ENCRYPTION_ENABLED, true) + intercept[IllegalArgumentException] { + conf.validateSettings() + } + + conf.set(NETWORK_AUTH_ENABLED, true) + conf.validateSettings() + } + } class Class1 {} diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala index 2bdbd70c638a..8feb3dee050d 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark import org.scalatest.Assertions + import org.apache.spark.storage.StorageLevel class SparkContextInfoSuite extends SparkFunSuite with LocalSparkContext { @@ -81,20 +82,18 @@ package object testPackage extends Assertions { val curCallSite = sc.getCallSite().shortForm // note: 2 lines after definition of "rdd" val rddCreationLine = rddCreationSite match { - case CALL_SITE_REGEX(func, file, line) => { + case CALL_SITE_REGEX(func, file, line) => assert(func === "makeRDD") assert(file === "SparkContextInfoSuite.scala") line.toInt - } case _ => fail("Did not match expected call site format") } curCallSite match { - case CALL_SITE_REGEX(func, file, line) => { + case CALL_SITE_REGEX(func, file, line) => assert(func === "getCallSite") // this is correct because we called it from outside of Spark assert(file === "SparkContextInfoSuite.scala") assert(line.toInt === rddCreationLine.toInt + 2) - } case _ => fail("Did not match expected call site format") } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index e5a14a69ef05..f8938dfedee5 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -19,25 +19,31 @@ package org.apache.spark import org.scalatest.PrivateMethodTester -import org.apache.spark.util.Utils +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend} -import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import org.apache.spark.scheduler.local.LocalBackend +import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend +import org.apache.spark.scheduler.local.LocalSchedulerBackend + class SparkContextSchedulerCreationSuite extends SparkFunSuite with LocalSparkContext with PrivateMethodTester with Logging { def createTaskScheduler(master: String): TaskSchedulerImpl = - createTaskScheduler(master, new SparkConf()) + createTaskScheduler(master, "client") + + def createTaskScheduler(master: String, deployMode: String): TaskSchedulerImpl = + createTaskScheduler(master, deployMode, new SparkConf()) - def createTaskScheduler(master: String, conf: SparkConf): TaskSchedulerImpl = { + def createTaskScheduler( + master: String, + deployMode: String, + conf: SparkConf): TaskSchedulerImpl = { // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. sc = new SparkContext("local", "test", conf) val createTaskSchedulerMethod = PrivateMethod[Tuple2[SchedulerBackend, TaskScheduler]]('createTaskScheduler) - val (_, sched) = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) + val (_, sched) = SparkContext invokePrivate createTaskSchedulerMethod(sc, master, deployMode) sched.asInstanceOf[TaskSchedulerImpl] } @@ -51,7 +57,7 @@ class SparkContextSchedulerCreationSuite test("local") { val sched = createTaskScheduler("local") sched.backend match { - case s: LocalBackend => assert(s.totalCores === 1) + case s: LocalSchedulerBackend => assert(s.totalCores === 1) case _ => fail() } } @@ -59,7 +65,8 @@ class SparkContextSchedulerCreationSuite test("local-*") { val sched = createTaskScheduler("local[*]") sched.backend match { - case s: LocalBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case s: LocalSchedulerBackend => + assert(s.totalCores === Runtime.getRuntime.availableProcessors()) case _ => fail() } } @@ -68,7 +75,7 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local[5]") assert(sched.maxTaskFailures === 1) sched.backend match { - case s: LocalBackend => assert(s.totalCores === 5) + case s: LocalSchedulerBackend => assert(s.totalCores === 5) case _ => fail() } } @@ -77,7 +84,8 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local[* ,2]") assert(sched.maxTaskFailures === 2) sched.backend match { - case s: LocalBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case s: LocalSchedulerBackend => + assert(s.totalCores === Runtime.getRuntime.availableProcessors()) case _ => fail() } } @@ -86,7 +94,7 @@ class SparkContextSchedulerCreationSuite val sched = createTaskScheduler("local[4, 2]") assert(sched.maxTaskFailures === 2) sched.backend match { - case s: LocalBackend => assert(s.totalCores === 4) + case s: LocalSchedulerBackend => assert(s.totalCores === 4) case _ => fail() } } @@ -107,74 +115,18 @@ class SparkContextSchedulerCreationSuite test("local-default-parallelism") { val conf = new SparkConf().set("spark.default.parallelism", "16") - val sched = createTaskScheduler("local", conf) + val sched = createTaskScheduler("local", "client", conf) sched.backend match { - case s: LocalBackend => assert(s.defaultParallelism() === 16) - case _ => fail() - } - } - - test("simr") { - createTaskScheduler("simr://uri").backend match { - case s: SimrSchedulerBackend => // OK + case s: LocalSchedulerBackend => assert(s.defaultParallelism() === 16) case _ => fail() } } test("local-cluster") { createTaskScheduler("local-cluster[3, 14, 1024]").backend match { - case s: SparkDeploySchedulerBackend => // OK + case s: StandaloneSchedulerBackend => // OK case _ => fail() } } - - def testYarn(master: String, expectedClassName: String) { - try { - val sched = createTaskScheduler(master) - assert(sched.getClass === Utils.classForName(expectedClassName)) - } catch { - case e: SparkException => - assert(e.getMessage.contains("YARN mode not available")) - logWarning("YARN not available, could not test actual YARN scheduler creation") - case e: Throwable => fail(e) - } - } - - test("yarn-cluster") { - testYarn("yarn-cluster", "org.apache.spark.scheduler.cluster.YarnClusterScheduler") - } - - test("yarn-standalone") { - testYarn("yarn-standalone", "org.apache.spark.scheduler.cluster.YarnClusterScheduler") - } - - test("yarn-client") { - testYarn("yarn-client", "org.apache.spark.scheduler.cluster.YarnScheduler") - } - - def testMesos(master: String, expectedClass: Class[_], coarse: Boolean) { - val conf = new SparkConf().set("spark.mesos.coarse", coarse.toString) - try { - val sched = createTaskScheduler(master, conf) - assert(sched.backend.getClass === expectedClass) - } catch { - case e: UnsatisfiedLinkError => - assert(e.getMessage.contains("mesos")) - logWarning("Mesos not available, could not test actual Mesos scheduler creation") - case e: Throwable => fail(e) - } - } - - test("mesos fine-grained") { - testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend], coarse = false) - } - - test("mesos coarse-grained") { - testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend], coarse = true) - } - - test("mesos with zookeeper") { - testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend], coarse = false) - } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index d4f2ea87650a..0ed5f26863da 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -18,29 +18,38 @@ package org.apache.spark import java.io.File +import java.net.{MalformedURLException, URI} +import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit -import com.google.common.base.Charsets._ -import com.google.common.io.Files +import scala.concurrent.duration._ +import com.google.common.io.Files +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.{BytesWritable, LongWritable, Text} import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat} -import org.apache.spark.util.Utils - -import scala.concurrent.Await -import scala.concurrent.duration.Duration import org.scalatest.Matchers._ +import org.scalatest.concurrent.Eventually + +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart, SparkListenerTaskEnd, SparkListenerTaskStart} +import org.apache.spark.util.{ThreadUtils, Utils} -class SparkContextSuite extends SparkFunSuite with LocalSparkContext { + +class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventually { test("Only one SparkContext may be active at a time") { // Regression test for SPARK-4180 val conf = new SparkConf().setAppName("test").setMaster("local") .set("spark.driver.allowMultipleContexts", "false") sc = new SparkContext(conf) + val envBefore = SparkEnv.get // A SparkContext is already running, so we shouldn't be able to create a second one intercept[SparkException] { new SparkContext(conf) } + val envAfter = SparkEnv.get + // SparkEnv and other context variables should be the same + assert(envBefore == envAfter) // After stopping the running context, we should be able to create a new one resetSparkContext() sc = new SparkContext(conf) @@ -104,7 +113,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { assert(byteArray2.length === 0) } - test("addFile works") { + test("basic case for addFile and listFiles") { val dir = Utils.createTempDir() val file1 = File.createTempFile("someprefix1", "somesuffix1", dir) @@ -115,8 +124,8 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { val absolutePath2 = file2.getAbsolutePath try { - Files.write("somewords1", file1, UTF_8) - Files.write("somewords2", file2, UTF_8) + Files.write("somewords1", file1, StandardCharsets.UTF_8) + Files.write("somewords2", file2, StandardCharsets.UTF_8) val length1 = file1.length() val length2 = file2.length() @@ -152,6 +161,39 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { } x }).count() + assert(sc.listFiles().filter(_.contains("somesuffix1")).size == 1) + } finally { + sc.stop() + } + } + + test("add and list jar files") { + val jarPath = Thread.currentThread().getContextClassLoader.getResource("TestUDTF.jar") + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addJar(jarPath.toString) + assert(sc.listJars().filter(_.contains("TestUDTF.jar")).size == 1) + } finally { + sc.stop() + } + } + + test("SPARK-17650: malformed url's throw exceptions before bricking Executors") { + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + Seq("http", "https", "ftp").foreach { scheme => + val badURL = s"$scheme://user:pwd/path" + val e1 = intercept[MalformedURLException] { + sc.addFile(badURL) + } + assert(e1.getMessage.contains(badURL)) + val e2 = intercept[MalformedURLException] { + sc.addJar(badURL) + } + assert(e2.getMessage.contains(badURL)) + assert(sc.addedFiles.isEmpty) + assert(sc.addedJars.isEmpty) + } } finally { sc.stop() } @@ -200,12 +242,79 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { } } + test("cannot call addFile with different paths that have the same filename") { + val dir = Utils.createTempDir() + try { + val subdir1 = new File(dir, "subdir1") + val subdir2 = new File(dir, "subdir2") + assert(subdir1.mkdir()) + assert(subdir2.mkdir()) + val file1 = new File(subdir1, "file") + val file2 = new File(subdir2, "file") + Files.write("old", file1, StandardCharsets.UTF_8) + Files.write("new", file2, StandardCharsets.UTF_8) + sc = new SparkContext("local-cluster[1,1,1024]", "test") + sc.addFile(file1.getAbsolutePath) + def getAddedFileContents(): String = { + sc.parallelize(Seq(0)).map { _ => + scala.io.Source.fromFile(SparkFiles.get("file")).mkString + }.first() + } + assert(getAddedFileContents() === "old") + intercept[IllegalArgumentException] { + sc.addFile(file2.getAbsolutePath) + } + assert(getAddedFileContents() === "old") + } finally { + Utils.deleteRecursively(dir) + } + } + + // Regression tests for SPARK-16787 + for ( + schedulingMode <- Seq("local-mode", "non-local-mode"); + method <- Seq("addJar", "addFile") + ) { + val jarPath = Thread.currentThread().getContextClassLoader.getResource("TestUDTF.jar").toString + val master = schedulingMode match { + case "local-mode" => "local" + case "non-local-mode" => "local-cluster[1,1,1024]" + } + test(s"$method can be called twice with same file in $schedulingMode (SPARK-16787)") { + sc = new SparkContext(master, "test") + method match { + case "addJar" => + sc.addJar(jarPath) + sc.addJar(jarPath) + case "addFile" => + sc.addFile(jarPath) + sc.addFile(jarPath) + } + } + } + + test("add jar with invalid path") { + val tmpDir = Utils.createTempDir() + val tmpJar = File.createTempFile("test", ".jar", tmpDir) + + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addJar(tmpJar.getAbsolutePath) + + // Invalid jar path will only print the error log, will not add to file server. + sc.addJar("dummy.jar") + sc.addJar("") + sc.addJar(tmpDir.getAbsolutePath) + + assert(sc.listJars().size == 1) + assert(sc.listJars().head.contains(tmpJar.getName)) + } + test("Cancelling job group should not cause SparkContext to shutdown (SPARK-6414)") { try { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) val future = sc.parallelize(Seq(0)).foreachAsync(_ => {Thread.sleep(1000L)}) sc.cancelJobGroup("nonExistGroupId") - Await.ready(future, Duration(2, TimeUnit.SECONDS)) + ThreadUtils.awaitReady(future, Duration(2, TimeUnit.SECONDS)) // In SPARK-6414, sc.cancelJobGroup will cause NullPointerException and cause // SparkContext to shutdown, so the following assertion will fail. @@ -243,11 +352,12 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { try { // Create 5 text files. - Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", file1, UTF_8) - Files.write("someline1 in file2\nsomeline2 in file2", file2, UTF_8) - Files.write("someline1 in file3", file3, UTF_8) - Files.write("someline1 in file4\nsomeline2 in file4", file4, UTF_8) - Files.write("someline1 in file2\nsomeline2 in file5", file5, UTF_8) + Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", file1, + StandardCharsets.UTF_8) + Files.write("someline1 in file2\nsomeline2 in file2", file2, StandardCharsets.UTF_8) + Files.write("someline1 in file3", file3, StandardCharsets.UTF_8) + Files.write("someline1 in file4\nsomeline2 in file4", file4, StandardCharsets.UTF_8) + Files.write("someline1 in file2\nsomeline2 in file5", file5, StandardCharsets.UTF_8) sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) @@ -274,6 +384,31 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { } } + test("Default path for file based RDDs is properly set (SPARK-12517)") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + + // Test filetextFile, wholeTextFiles, binaryFiles, hadoopFile and + // newAPIHadoopFile for setting the default path as the RDD name + val mockPath = "default/path/for/" + + var targetPath = mockPath + "textFile" + assert(sc.textFile(targetPath).name === targetPath) + + targetPath = mockPath + "wholeTextFiles" + assert(sc.wholeTextFiles(targetPath).name === targetPath) + + targetPath = mockPath + "binaryFiles" + assert(sc.binaryFiles(targetPath).name === targetPath) + + targetPath = mockPath + "hadoopFile" + assert(sc.hadoopFile(targetPath).name === targetPath) + + targetPath = mockPath + "newAPIHadoopFile" + assert(sc.newAPIHadoopFile(targetPath).name === targetPath) + + sc.stop() + } + test("calling multiple sc.stop() must not throw any exception") { noException should be thrownBy { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) @@ -293,4 +428,195 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { assert(sc.getConf.getInt("spark.executor.instances", 0) === 6) } } + + + test("localProperties are inherited by spawned threads.") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.setLocalProperty("testProperty", "testValue") + var result = "unset"; + val thread = new Thread() { override def run() = {result = sc.getLocalProperty("testProperty")}} + thread.start() + thread.join() + sc.stop() + assert(result == "testValue") + } + + test("localProperties do not cross-talk between threads.") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + var result = "unset"; + val thread1 = new Thread() { + override def run() = {sc.setLocalProperty("testProperty", "testValue")}} + // testProperty should be unset and thus return null + val thread2 = new Thread() { + override def run() = {result = sc.getLocalProperty("testProperty")}} + thread1.start() + thread1.join() + thread2.start() + thread2.join() + sc.stop() + assert(result == null) + } + + test("log level case-insensitive and reset log level") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + val originalLevel = org.apache.log4j.Logger.getRootLogger().getLevel + try { + sc.setLogLevel("debug") + assert(org.apache.log4j.Logger.getRootLogger().getLevel === org.apache.log4j.Level.DEBUG) + sc.setLogLevel("INfo") + assert(org.apache.log4j.Logger.getRootLogger().getLevel === org.apache.log4j.Level.INFO) + } finally { + sc.setLogLevel(originalLevel.toString) + assert(org.apache.log4j.Logger.getRootLogger().getLevel === originalLevel) + sc.stop() + } + } + + test("register and deregister Spark listener from SparkContext") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + val sparkListener1 = new SparkListener { } + val sparkListener2 = new SparkListener { } + sc.addSparkListener(sparkListener1) + sc.addSparkListener(sparkListener2) + assert(sc.listenerBus.listeners.contains(sparkListener1)) + assert(sc.listenerBus.listeners.contains(sparkListener2)) + sc.removeSparkListener(sparkListener1) + assert(!sc.listenerBus.listeners.contains(sparkListener1)) + assert(sc.listenerBus.listeners.contains(sparkListener2)) + } + + test("Cancelling stages/jobs with custom reasons.") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + val REASON = "You shall not pass" + + val listener = new SparkListener { + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { + if (SparkContextSuite.cancelStage) { + eventually(timeout(10.seconds)) { + assert(SparkContextSuite.isTaskStarted) + } + sc.cancelStage(taskStart.stageId, REASON) + SparkContextSuite.cancelStage = false + } + } + + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + if (SparkContextSuite.cancelJob) { + eventually(timeout(10.seconds)) { + assert(SparkContextSuite.isTaskStarted) + } + sc.cancelJob(jobStart.jobId, REASON) + SparkContextSuite.cancelJob = false + } + } + } + sc.addSparkListener(listener) + + for (cancelWhat <- Seq("stage", "job")) { + SparkContextSuite.isTaskStarted = false + SparkContextSuite.cancelStage = (cancelWhat == "stage") + SparkContextSuite.cancelJob = (cancelWhat == "job") + + val ex = intercept[SparkException] { + sc.range(0, 10000L).mapPartitions { x => + org.apache.spark.SparkContextSuite.isTaskStarted = true + x + }.cartesian(sc.range(0, 10L))count() + } + + ex.getCause() match { + case null => + assert(ex.getMessage().contains(REASON)) + case cause: SparkException => + assert(cause.getMessage().contains(REASON)) + case cause: Throwable => + fail("Expected the cause to be SparkException, got " + cause.toString() + " instead.") + } + + eventually(timeout(20.seconds)) { + assert(sc.statusTracker.getExecutorInfos.map(_.numRunningTasks()).sum == 0) + } + } + } + + testCancellingTasks("that raise interrupted exception on cancel") { + Thread.sleep(9999999) + } + + // SPARK-20217 should not fail stage if task throws non-interrupted exception + testCancellingTasks("that raise runtime exception on cancel") { + try { + Thread.sleep(9999999) + } catch { + case t: Throwable => + throw new RuntimeException("killed") + } + } + + // Launches one task that will block forever. Once the SparkListener detects the task has + // started, kill and re-schedule it. The second run of the task will complete immediately. + // If this test times out, then the first version of the task wasn't killed successfully. + def testCancellingTasks(desc: String)(blockFn: => Unit): Unit = test(s"Killing tasks $desc") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + + SparkContextSuite.isTaskStarted = false + SparkContextSuite.taskKilled = false + SparkContextSuite.taskSucceeded = false + + val listener = new SparkListener { + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { + eventually(timeout(10.seconds)) { + assert(SparkContextSuite.isTaskStarted) + } + if (!SparkContextSuite.taskKilled) { + SparkContextSuite.taskKilled = true + sc.killTaskAttempt(taskStart.taskInfo.taskId, true, "first attempt will hang") + } + } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + if (taskEnd.taskInfo.attemptNumber == 1 && taskEnd.reason == Success) { + SparkContextSuite.taskSucceeded = true + } + } + } + sc.addSparkListener(listener) + eventually(timeout(20.seconds)) { + sc.parallelize(1 to 1).foreach { x => + // first attempt will hang + if (!SparkContextSuite.isTaskStarted) { + SparkContextSuite.isTaskStarted = true + blockFn + } + // second attempt succeeds immediately + } + } + eventually(timeout(10.seconds)) { + assert(SparkContextSuite.taskSucceeded) + } + } + + test("SPARK-19446: DebugFilesystem.assertNoOpenStreams should report " + + "open streams to help debugging") { + val fs = new DebugFilesystem() + fs.initialize(new URI("file:///"), new Configuration()) + val file = File.createTempFile("SPARK19446", "temp") + file.deleteOnExit() + Files.write(Array.ofDim[Byte](1000), file) + val path = new Path("file:///" + file.getCanonicalPath) + val stream = fs.open(path) + val exc = intercept[RuntimeException] { + DebugFilesystem.assertNoOpenStreams() + } + assert(exc != null) + assert(exc.getCause() != null) + stream.close() + } +} + +object SparkContextSuite { + @volatile var cancelJob = false + @volatile var cancelStage = false + @volatile var isTaskStarted = false + @volatile var taskKilled = false + @volatile var taskSucceeded = false } diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 9be9db01c7de..18077c08c9dc 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -18,14 +18,40 @@ package org.apache.spark // scalastyle:off -import org.scalatest.{FunSuite, Outcome} +import java.io.File + +import org.scalatest.{BeforeAndAfterAll, FunSuite, Outcome} + +import org.apache.spark.internal.Logging +import org.apache.spark.util.AccumulatorContext /** * Base abstract class for all unit tests in Spark for handling common functionality. */ -private[spark] abstract class SparkFunSuite extends FunSuite with Logging { +abstract class SparkFunSuite + extends FunSuite + with BeforeAndAfterAll + with Logging { // scalastyle:on + protected override def afterAll(): Unit = { + try { + // Avoid leaking map entries in tests that use accumulators without SparkContext + AccumulatorContext.clear() + } finally { + super.afterAll() + } + } + + // helper function + protected final def getTestResourceFile(file: String): File = { + new File(getClass.getClassLoader.getResource(file).getFile) + } + + protected final def getTestResourcePath(file: String): String = { + getTestResourceFile(file).getCanonicalPath + } + /** * Log the suite name and the test name before and after each test. * diff --git a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala index 46516e8d2529..5483f2b8434a 100644 --- a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala @@ -86,4 +86,30 @@ class StatusTrackerSuite extends SparkFunSuite with Matchers with LocalSparkCont Set(firstJobId, secondJobId)) } } + + test("getJobIdsForGroup() with takeAsync()") { + sc = new SparkContext("local", "test", new SparkConf(false)) + sc.setJobGroup("my-job-group2", "description") + sc.statusTracker.getJobIdsForGroup("my-job-group2") shouldBe empty + val firstJobFuture = sc.parallelize(1 to 1000, 1).takeAsync(1) + val firstJobId = eventually(timeout(10 seconds)) { + firstJobFuture.jobIds.head + } + eventually(timeout(10 seconds)) { + sc.statusTracker.getJobIdsForGroup("my-job-group2") should be (Seq(firstJobId)) + } + } + + test("getJobIdsForGroup() with takeAsync() across multiple partitions") { + sc = new SparkContext("local", "test", new SparkConf(false)) + sc.setJobGroup("my-job-group2", "description") + sc.statusTracker.getJobIdsForGroup("my-job-group2") shouldBe empty + val firstJobFuture = sc.parallelize(1 to 1000, 2).takeAsync(999) + val firstJobId = eventually(timeout(10 seconds)) { + firstJobFuture.jobIds.head + } + eventually(timeout(10 seconds)) { + sc.statusTracker.getJobIdsForGroup("my-job-group2") should have size 2 + } + } } diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala index 54c131cdae36..36273d722f50 100644 --- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -17,11 +17,10 @@ package org.apache.spark -import java.util.concurrent.{TimeUnit, Semaphore} -import java.util.concurrent.atomic.AtomicBoolean -import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.Semaphore +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} -import org.apache.spark.scheduler._ +import org.apache.spark.internal.Logging /** * Holds state shared across task threads in some ThreadingSuite tests. diff --git a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala index f7a13ab3996d..bc3f58cf2a35 100644 --- a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala +++ b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark -import org.scalatest.concurrent.Timeouts._ +import org.scalatest.concurrent.TimeLimits._ import org.scalatest.time.{Millis, Span} class UnpersistSuite extends SparkFunSuite with LocalSparkContext { @@ -35,7 +35,7 @@ class UnpersistSuite extends SparkFunSuite with LocalSparkContext { Thread.sleep(200) } } catch { - case _: Throwable => { Thread.sleep(10) } + case _: Throwable => Thread.sleep(10) // Do nothing. We might see exceptions because block manager // is racing this thread to remove entries from the driver. } diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala index 135c56bf5bc9..b38a3667abee 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.api.python -import scala.io.Source +import java.io.{File, PrintWriter} -import java.io.{PrintWriter, File} +import scala.io.Source import org.scalatest.Matchers diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala index 41f2a5c972b6..05b4e67412f2 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.api.python import java.io.{ByteArrayOutputStream, DataOutputStream} +import java.nio.charset.StandardCharsets import org.apache.spark.SparkFunSuite @@ -35,10 +36,12 @@ class PythonRDDSuite extends SparkFunSuite { // The correctness will be tested in Python PythonRDD.writeIteratorToStream(Iterator("a", null), buffer) PythonRDD.writeIteratorToStream(Iterator(null, "a"), buffer) - PythonRDD.writeIteratorToStream(Iterator("a".getBytes, null), buffer) - PythonRDD.writeIteratorToStream(Iterator(null, "a".getBytes), buffer) + PythonRDD.writeIteratorToStream(Iterator("a".getBytes(StandardCharsets.UTF_8), null), buffer) + PythonRDD.writeIteratorToStream(Iterator(null, "a".getBytes(StandardCharsets.UTF_8)), buffer) PythonRDD.writeIteratorToStream(Iterator((null, null), ("a", null), (null, "b")), buffer) - PythonRDD.writeIteratorToStream( - Iterator((null, null), ("a".getBytes, null), (null, "b".getBytes)), buffer) + PythonRDD.writeIteratorToStream(Iterator( + (null, null), + ("a".getBytes(StandardCharsets.UTF_8), null), + (null, "b".getBytes(StandardCharsets.UTF_8))), buffer) } } diff --git a/core/src/test/scala/org/apache/spark/api/r/JVMObjectTrackerSuite.scala b/core/src/test/scala/org/apache/spark/api/r/JVMObjectTrackerSuite.scala new file mode 100644 index 000000000000..6a979aefe6e9 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/api/r/JVMObjectTrackerSuite.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.r + +import org.apache.spark.SparkFunSuite + +class JVMObjectTrackerSuite extends SparkFunSuite { + test("JVMObjectId does not take null IDs") { + intercept[IllegalArgumentException] { + JVMObjectId(null) + } + } + + test("JVMObjectTracker") { + val tracker = new JVMObjectTracker + assert(tracker.size === 0) + withClue("an empty tracker can be cleared") { + tracker.clear() + } + val none = JVMObjectId("none") + assert(tracker.get(none) === None) + intercept[NoSuchElementException] { + tracker(JVMObjectId("none")) + } + + val obj1 = new Object + val id1 = tracker.addAndGetId(obj1) + assert(id1 != null) + assert(tracker.size === 1) + assert(tracker.get(id1).get.eq(obj1)) + assert(tracker(id1).eq(obj1)) + + val obj2 = new Object + val id2 = tracker.addAndGetId(obj2) + assert(id1 !== id2) + assert(tracker.size === 2) + assert(tracker(id2).eq(obj2)) + + val Some(obj1Removed) = tracker.remove(id1) + assert(obj1Removed.eq(obj1)) + assert(tracker.get(id1) === None) + assert(tracker.size === 1) + assert(tracker(id2).eq(obj2)) + + val obj3 = new Object + val id3 = tracker.addAndGetId(obj3) + assert(tracker.size === 2) + assert(id3 != id1) + assert(id3 != id2) + assert(tracker(id3).eq(obj3)) + + tracker.clear() + assert(tracker.size === 0) + assert(tracker.get(id1) === None) + assert(tracker.get(id2) === None) + assert(tracker.get(id3) === None) + } +} diff --git a/core/src/test/scala/org/apache/spark/api/r/RBackendSuite.scala b/core/src/test/scala/org/apache/spark/api/r/RBackendSuite.scala new file mode 100644 index 000000000000..085cc267ca74 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/api/r/RBackendSuite.scala @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.r + +import org.apache.spark.SparkFunSuite + +class RBackendSuite extends SparkFunSuite { + test("close() clears jvmObjectTracker") { + val backend = new RBackend + val tracker = backend.jvmObjectTracker + val id = tracker.addAndGetId(new Object) + backend.close() + assert(tracker.get(id) === None) + assert(tracker.size === 0) + } +} diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index ba21075ce6be..46f9ac6b0273 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.broadcast +import java.util.Locale + import scala.util.Random import org.scalatest.Assertions @@ -24,8 +26,10 @@ import org.scalatest.Assertions import org.apache.spark._ import org.apache.spark.io.SnappyCompressionCodec import org.apache.spark.rdd.RDD +import org.apache.spark.security.EncryptionFunSuite import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage._ +import org.apache.spark.util.io.ChunkedByteBuffer // Dummy class that creates a broadcast variable but doesn't use it class DummyBroadcastClass(rdd: RDD[Int]) extends Serializable { @@ -37,47 +41,16 @@ class DummyBroadcastClass(rdd: RDD[Int]) extends Serializable { rdd.map { x => val bm = SparkEnv.get.blockManager // Check if broadcast block was fetched - val isFound = bm.getLocal(BroadcastBlockId(bid)).isDefined + val isFound = bm.getLocalValues(BroadcastBlockId(bid)).isDefined (x, isFound) }.collect().toSet } } -class BroadcastSuite extends SparkFunSuite with LocalSparkContext { - - private val httpConf = broadcastConf("HttpBroadcastFactory") - private val torrentConf = broadcastConf("TorrentBroadcastFactory") - - test("Using HttpBroadcast locally") { - sc = new SparkContext("local", "test", httpConf) - val list = List[Int](1, 2, 3, 4) - val broadcast = sc.broadcast(list) - val results = sc.parallelize(1 to 2).map(x => (x, broadcast.value.sum)) - assert(results.collect().toSet === Set((1, 10), (2, 10))) - } - - test("Accessing HttpBroadcast variables from multiple threads") { - sc = new SparkContext("local[10]", "test", httpConf) - val list = List[Int](1, 2, 3, 4) - val broadcast = sc.broadcast(list) - val results = sc.parallelize(1 to 10).map(x => (x, broadcast.value.sum)) - assert(results.collect().toSet === (1 to 10).map(x => (x, 10)).toSet) - } - - test("Accessing HttpBroadcast variables in a local cluster") { - val numSlaves = 4 - val conf = httpConf.clone - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.broadcast.compress", "true") - sc = new SparkContext("local-cluster[%d, 1, 1024]".format(numSlaves), "test", conf) - val list = List[Int](1, 2, 3, 4) - val broadcast = sc.broadcast(list) - val results = sc.parallelize(1 to numSlaves).map(x => (x, broadcast.value.sum)) - assert(results.collect().toSet === (1 to numSlaves).map(x => (x, 10)).toSet) - } +class BroadcastSuite extends SparkFunSuite with LocalSparkContext with EncryptionFunSuite { test("Using TorrentBroadcast locally") { - sc = new SparkContext("local", "test", torrentConf) + sc = new SparkContext("local", "test") val list = List[Int](1, 2, 3, 4) val broadcast = sc.broadcast(list) val results = sc.parallelize(1 to 2).map(x => (x, broadcast.value.sum)) @@ -85,16 +58,15 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext { } test("Accessing TorrentBroadcast variables from multiple threads") { - sc = new SparkContext("local[10]", "test", torrentConf) + sc = new SparkContext("local[10]", "test") val list = List[Int](1, 2, 3, 4) val broadcast = sc.broadcast(list) val results = sc.parallelize(1 to 10).map(x => (x, broadcast.value.sum)) assert(results.collect().toSet === (1 to 10).map(x => (x, 10)).toSet) } - test("Accessing TorrentBroadcast variables in a local cluster") { + encryptionTest("Accessing TorrentBroadcast variables in a local cluster") { conf => val numSlaves = 4 - val conf = torrentConf.clone conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") conf.set("spark.broadcast.compress", "true") sc = new SparkContext("local-cluster[%d, 1, 1024]".format(numSlaves), "test", conf) @@ -116,7 +88,9 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext { val size = 1 + rand.nextInt(1024 * 10) val data: Array[Byte] = new Array[Byte](size) rand.nextBytes(data) - val blocks = blockifyObject(data, blockSize, serializer, compressionCodec) + val blocks = blockifyObject(data, blockSize, serializer, compressionCodec).map { b => + new ChunkedByteBuffer(b).toInputStream(dispose = true) + } val unblockified = unBlockifyObject[Array[Byte]](blocks, serializer, compressionCodec) assert(unblockified === data) } @@ -124,31 +98,13 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext { test("Test Lazy Broadcast variables with TorrentBroadcast") { val numSlaves = 2 - val conf = torrentConf.clone - sc = new SparkContext("local-cluster[%d, 1, 1024]".format(numSlaves), "test", conf) + sc = new SparkContext("local-cluster[%d, 1, 1024]".format(numSlaves), "test") val rdd = sc.parallelize(1 to numSlaves) - val results = new DummyBroadcastClass(rdd).doSomething() assert(results.toSet === (1 to numSlaves).map(x => (x, false)).toSet) } - test("Unpersisting HttpBroadcast on executors only in local mode") { - testUnpersistHttpBroadcast(distributed = false, removeFromDriver = false) - } - - test("Unpersisting HttpBroadcast on executors and driver in local mode") { - testUnpersistHttpBroadcast(distributed = false, removeFromDriver = true) - } - - test("Unpersisting HttpBroadcast on executors only in distributed mode") { - testUnpersistHttpBroadcast(distributed = true, removeFromDriver = false) - } - - test("Unpersisting HttpBroadcast on executors and driver in distributed mode") { - testUnpersistHttpBroadcast(distributed = true, removeFromDriver = true) - } - test("Unpersisting TorrentBroadcast on executors only in local mode") { testUnpersistTorrentBroadcast(distributed = false, removeFromDriver = false) } @@ -176,71 +132,29 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext { val thrown = intercept[IllegalStateException] { sc.broadcast(Seq(1, 2, 3)) } - assert(thrown.getMessage.toLowerCase.contains("stopped")) + assert(thrown.getMessage.toLowerCase(Locale.ROOT).contains("stopped")) } - /** - * Verify the persistence of state associated with an HttpBroadcast in either local mode or - * local-cluster mode (when distributed = true). - * - * This test creates a broadcast variable, uses it on all executors, and then unpersists it. - * In between each step, this test verifies that the broadcast blocks and the broadcast file - * are present only on the expected nodes. - */ - private def testUnpersistHttpBroadcast(distributed: Boolean, removeFromDriver: Boolean) { - val numSlaves = if (distributed) 2 else 0 - - // Verify that the broadcast file is created, and blocks are persisted only on the driver - def afterCreation(broadcastId: Long, bmm: BlockManagerMaster) { - val blockId = BroadcastBlockId(broadcastId) - val statuses = bmm.getBlockStatus(blockId, askSlaves = true) - assert(statuses.size === 1) - statuses.head match { case (bm, status) => - assert(bm.isDriver, "Block should only be on the driver") - assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) - assert(status.memSize > 0, "Block should be in memory store on the driver") - assert(status.diskSize === 0, "Block should not be in disk store on the driver") - } - if (distributed) { - // this file is only generated in distributed mode - assert(HttpBroadcast.getFile(blockId.broadcastId).exists, "Broadcast file not found!") - } - } - - // Verify that blocks are persisted in both the executors and the driver - def afterUsingBroadcast(broadcastId: Long, bmm: BlockManagerMaster) { - val blockId = BroadcastBlockId(broadcastId) - val statuses = bmm.getBlockStatus(blockId, askSlaves = true) - assert(statuses.size === numSlaves + 1) - statuses.foreach { case (_, status) => - assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) - assert(status.memSize > 0, "Block should be in memory store") - assert(status.diskSize === 0, "Block should not be in disk store") - } - } - - // Verify that blocks are unpersisted on all executors, and on all nodes if removeFromDriver - // is true. In the latter case, also verify that the broadcast file is deleted on the driver. - def afterUnpersist(broadcastId: Long, bmm: BlockManagerMaster) { - val blockId = BroadcastBlockId(broadcastId) - val statuses = bmm.getBlockStatus(blockId, askSlaves = true) - val expectedNumBlocks = if (removeFromDriver) 0 else 1 - val possiblyNot = if (removeFromDriver) "" else " not" - assert(statuses.size === expectedNumBlocks, - "Block should%s be unpersisted on the driver".format(possiblyNot)) - if (distributed && removeFromDriver) { - // this file is only generated in distributed mode - assert(!HttpBroadcast.getFile(blockId.broadcastId).exists, - "Broadcast file should%s be deleted".format(possiblyNot)) - } - } + test("Forbid broadcasting RDD directly") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + val rdd = sc.parallelize(1 to 4) + intercept[IllegalArgumentException] { sc.broadcast(rdd) } + sc.stop() + } - testUnpersistBroadcast(distributed, numSlaves, httpConf, afterCreation, - afterUsingBroadcast, afterUnpersist, removeFromDriver) + encryptionTest("Cache broadcast to disk") { conf => + conf.setMaster("local") + .setAppName("test") + .set("spark.memory.useLegacyMode", "true") + .set("spark.storage.memoryFraction", "0.0") + sc = new SparkContext(conf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + assert(broadcast.value.sum === 10) } /** - * Verify the persistence of state associated with an TorrentBroadcast in a local-cluster. + * Verify the persistence of state associated with a TorrentBroadcast in a local-cluster. * * This test creates a broadcast variable, uses it on all executors, and then unpersists it. * In between each step, this test verifies that the broadcast blocks are present only on the @@ -284,7 +198,7 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext { assert(statuses.size === expectedNumBlocks) } - testUnpersistBroadcast(distributed, numSlaves, torrentConf, afterCreation, + testUnpersistBroadcast(distributed, numSlaves, afterCreation, afterUsingBroadcast, afterUnpersist, removeFromDriver) } @@ -300,7 +214,6 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext { private def testUnpersistBroadcast( distributed: Boolean, numSlaves: Int, // used only when distributed = true - broadcastConf: SparkConf, afterCreation: (Long, BlockManagerMaster) => Unit, afterUsingBroadcast: (Long, BlockManagerMaster) => Unit, afterUnpersist: (Long, BlockManagerMaster) => Unit, @@ -308,7 +221,7 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext { sc = if (distributed) { val _sc = - new SparkContext("local-cluster[%d, 1, 1024]".format(numSlaves), "test", broadcastConf) + new SparkContext("local-cluster[%d, 1, 1024]".format(numSlaves), "test") // Wait until all salves are up try { _sc.jobProgressListener.waitUntilExecutorsUp(numSlaves, 60000) @@ -319,7 +232,7 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext { throw e } } else { - new SparkContext("local", "test", broadcastConf) + new SparkContext("local", "test") } val blockManagerMaster = sc.env.blockManager.master val list = List[Int](1, 2, 3, 4) @@ -356,13 +269,6 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext { assert(results.collect().toSet === (1 to partitions).map(x => (x, list.sum)).toSet) } } - - /** Helper method to create a SparkConf that uses the given broadcast factory. */ - private def broadcastConf(factoryName: String): SparkConf = { - val conf = new SparkConf - conf.set("spark.broadcast.factory", "org.apache.spark.broadcast.%s".format(factoryName)) - conf - } } package object testPackage extends Assertions { diff --git a/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala index 3164760b08a7..55a541d60ea3 100644 --- a/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala +++ b/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala @@ -20,9 +20,9 @@ package org.apache.spark.deploy import java.io.File import java.util.Date +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.{SecurityManager, SparkConf} private[deploy] object DeployTestUtils { def createAppDesc(): ApplicationDescription = { @@ -39,7 +39,7 @@ private[deploy] object DeployTestUtils { } def createDriverCommand(): Command = new Command( - "org.apache.spark.FakeClass", Seq("some arg --and-some options -g foo"), + "org.apache.spark.FakeClass", Seq("WORKER_URL", "USER_JAR", "mainClass"), Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Seq("-Dfoo") ) @@ -47,10 +47,10 @@ private[deploy] object DeployTestUtils { new DriverDescription("hdfs://some-dir/some.jar", 100, 3, false, createDriverCommand()) def createDriverInfo(): DriverInfo = new DriverInfo(3, "driver-3", - createDriverDesc(), new Date()) + createDriverDesc(), JsonConstants.submitDate) def createWorkerInfo(): WorkerInfo = { - val workerInfo = new WorkerInfo("id", "host", 8080, 4, 1234, null, 80, "publicAddress") + val workerInfo = new WorkerInfo("id", "host", 8080, 4, 1234, null, "http://publicAddress:80") workerInfo.lastHeartbeat = JsonConstants.currTimeInMillis workerInfo } @@ -69,7 +69,7 @@ private[deploy] object DeployTestUtils { "publicAddress", new File("sparkHome"), new File("workDir"), - "akka://worker", + "spark://worker", new SparkConf, Seq("localDir"), ExecutorState.RUNNING) @@ -84,7 +84,7 @@ private[deploy] object DeployTestUtils { new File("sparkHome"), createDriverDesc(), null, - "akka://worker", + "spark://worker", new SecurityManager(conf)) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala index d93febcfd23f..42b8cde65039 100644 --- a/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala +++ b/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala @@ -24,10 +24,8 @@ import java.util.jar.Manifest import scala.collection.mutable.ArrayBuffer -import com.google.common.io.{Files, ByteStreams} - +import com.google.common.io.{ByteStreams, Files} import org.apache.commons.io.FileUtils - import org.apache.ivy.core.settings.IvySettings import org.apache.spark.TestUtils.{createCompiledClass, JavaSourceFromString} @@ -144,7 +142,7 @@ private[deploy] object IvyTestUtils { |} """.stripMargin val sourceFile = - new JavaSourceFromString(new File(dir, className).getAbsolutePath, contents) + new JavaSourceFromString(new File(dir, className).toURI.getPath, contents) createCompiledClass(className, dir, sourceFile, Seq.empty) } @@ -245,16 +243,22 @@ private[deploy] object IvyTestUtils { withManifest: Option[Manifest] = None): File = { val jarFile = new File(dir, artifactName(artifact, useIvyLayout)) val jarFileStream = new FileOutputStream(jarFile) - val manifest = withManifest.getOrElse { - val mani = new Manifest() + val manifest: Manifest = withManifest.getOrElse { if (withR) { + val mani = new Manifest() val attr = mani.getMainAttributes attr.put(Name.MANIFEST_VERSION, "1.0") attr.put(new Name("Spark-HasRPackage"), "true") + mani + } else { + null } - mani } - val jarStream = new JarOutputStream(jarFileStream, manifest) + val jarStream = if (manifest != null) { + new JarOutputStream(jarFileStream, manifest) + } else { + new JarOutputStream(jarFileStream) + } for (file <- files) { val jarEntry = new JarEntry(file._1) @@ -307,7 +311,7 @@ private[deploy] object IvyTestUtils { val allFiles = ArrayBuffer[(String, File)](javaFile) if (withPython) { val pythonFile = createPythonFile(root) - allFiles.append((pythonFile.getName, pythonFile)) + allFiles += Tuple2(pythonFile.getName, pythonFile) } if (withR) { val rFiles = createRFiles(root, className, artifact.groupId) diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 0a9f128a3a6b..1903130cb694 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -23,10 +23,10 @@ import com.fasterxml.jackson.core.JsonParseException import org.json4s._ import org.json4s.jackson.JsonMethods +import org.apache.spark.{JsonTestUtils, SparkFunSuite} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, RecoveryState} import org.apache.spark.deploy.worker.ExecutorRunner -import org.apache.spark.{JsonTestUtils, SparkFunSuite} class JsonProtocolSuite extends SparkFunSuite with JsonTestUtils { @@ -65,7 +65,7 @@ class JsonProtocolSuite extends SparkFunSuite with JsonTestUtils { test("writeMasterState") { val workers = Array(createWorkerInfo(), createWorkerInfo()) val activeApps = Array(createAppInfo()) - val completedApps = Array[ApplicationInfo]() + val completedApps = Array.empty[ApplicationInfo] val activeDrivers = Array(createDriverInfo()) val completedDrivers = Array(createDriverInfo()) val stateResponse = new MasterStateResponse( @@ -104,8 +104,8 @@ object JsonConstants { val submitDate = new Date(123456789) val appInfoJsonStr = """ - |{"starttime":3,"id":"id","name":"name", - |"cores":4,"user":"%s", + |{"id":"id","starttime":3,"name":"name", + |"cores":0,"user":"%s", |"memoryperslave":1234,"submitdate":"%s", |"state":"WAITING","duration":%d} """.format(System.getProperty("user.name", ""), @@ -134,19 +134,24 @@ object JsonConstants { val driverInfoJsonStr = """ - |{"id":"driver-3","starttime":"3","state":"SUBMITTED","cores":3,"memory":100} - """.stripMargin + |{"id":"driver-3","starttime":"3", + |"state":"SUBMITTED","cores":3,"memory":100, + |"submitdate":"%s","worker":"None", + |"mainclass":"mainClass"} + """.format(submitDate.toString).stripMargin val masterStateJsonStr = """ |{"url":"spark://host:8080", |"workers":[%s,%s], + |"aliveworkers":2, |"cores":8,"coresused":0,"memory":2468,"memoryused":0, |"activeapps":[%s],"completedapps":[], |"activedrivers":[%s], + |"completeddrivers":[%s], |"status":"ALIVE"} """.format(workerInfoJsonStr, workerInfoJsonStr, - appInfoJsonStr, driverInfoJsonStr).stripMargin + appInfoJsonStr, driverInfoJsonStr, driverInfoJsonStr).stripMargin val workerStateJsonStr = """ diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala index 8dd31b4b6fdd..cbdf1755b0c5 100644 --- a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala @@ -22,9 +22,9 @@ import java.net.URL import scala.collection.mutable import scala.io.Source +import org.apache.spark.{LocalSparkContext, SparkContext, SparkFunSuite} +import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded} import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.{SparkListenerExecutorAdded, SparkListener} -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.util.SparkConfWithEnv class LogUrlsStandaloneSuite extends SparkFunSuite with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/deploy/RPackageUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/RPackageUtilsSuite.scala index 1ed4bae3ca21..32dd3ecc2f02 100644 --- a/core/src/test/scala/org/apache/spark/deploy/RPackageUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/RPackageUtilsSuite.scala @@ -17,10 +17,10 @@ package org.apache.spark.deploy -import java.io.{PrintStream, OutputStream, File} +import java.io.{File, OutputStream, PrintStream} import java.net.URI -import java.util.jar.Attributes.Name import java.util.jar.{JarFile, Manifest} +import java.util.jar.Attributes.Name import java.util.zip.ZipFile import scala.collection.JavaConverters._ @@ -33,8 +33,12 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkFunSuite import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate +import org.apache.spark.util.{ResetSystemProperties, Utils} -class RPackageUtilsSuite extends SparkFunSuite with BeforeAndAfterEach { +class RPackageUtilsSuite + extends SparkFunSuite + with BeforeAndAfterEach + with ResetSystemProperties { private val main = MavenCoordinate("a", "b", "c") private val dep1 = MavenCoordinate("a", "dep1", "c") @@ -60,11 +64,9 @@ class RPackageUtilsSuite extends SparkFunSuite with BeforeAndAfterEach { } } - def beforeAll() { - System.setProperty("spark.testing", "true") - } - override def beforeEach(): Unit = { + super.beforeEach() + System.setProperty("spark.testing", "true") lineBuffer.clear() } @@ -72,9 +74,13 @@ class RPackageUtilsSuite extends SparkFunSuite with BeforeAndAfterEach { val deps = Seq(dep1, dep2).mkString(",") IvyTestUtils.withRepository(main, Some(deps), None, withR = true) { repo => val jars = Seq(main, dep1, dep2).map(c => new JarFile(getJarPath(c, new File(new URI(repo))))) - assert(RPackageUtils.checkManifestForR(jars(0)), "should have R code") - assert(!RPackageUtils.checkManifestForR(jars(1)), "should not have R code") - assert(!RPackageUtils.checkManifestForR(jars(2)), "should not have R code") + Utils.tryWithSafeFinally { + assert(RPackageUtils.checkManifestForR(jars(0)), "should have R code") + assert(!RPackageUtils.checkManifestForR(jars(1)), "should not have R code") + assert(!RPackageUtils.checkManifestForR(jars(2)), "should not have R code") + } { + jars.foreach(_.close()) + } } } @@ -127,9 +133,20 @@ class RPackageUtilsSuite extends SparkFunSuite with BeforeAndAfterEach { } } + test("jars without manifest return false") { + IvyTestUtils.withRepository(main, None, None) { repo => + val jar = IvyTestUtils.packJar(new File(new URI(repo)), dep1, Nil, + useIvyLayout = false, withR = false, None) + Utils.tryWithResource(new JarFile(jar)) { jarFile => + assert(jarFile.getManifest == null, "jar file should have null manifest") + assert(!RPackageUtils.checkManifestForR(jarFile), "null manifest should return false") + } + } + } + test("SparkR zipping works properly") { val tempDir = Files.createTempDir() - try { + Utils.tryWithSafeFinally { IvyTestUtils.writeFile(tempDir, "test.R", "abc") val fakeSparkRDir = new File(tempDir, "SparkR") assert(fakeSparkRDir.mkdirs()) @@ -142,14 +159,19 @@ class RPackageUtilsSuite extends SparkFunSuite with BeforeAndAfterEach { IvyTestUtils.writeFile(fakePackageDir, "DESCRIPTION", "abc") val finalZip = RPackageUtils.zipRLibraries(tempDir, "sparkr.zip") assert(finalZip.exists()) - val entries = new ZipFile(finalZip).entries().asScala.map(_.getName).toSeq - assert(entries.contains("/test.R")) - assert(entries.contains("/SparkR/abc.R")) - assert(entries.contains("/SparkR/DESCRIPTION")) - assert(!entries.contains("/package.zip")) - assert(entries.contains("/packageTest/def.R")) - assert(entries.contains("/packageTest/DESCRIPTION")) - } finally { + val zipFile = new ZipFile(finalZip) + Utils.tryWithSafeFinally { + val entries = zipFile.entries().asScala.map(_.getName).toSeq + assert(entries.contains("/test.R")) + assert(entries.contains("/SparkR/abc.R")) + assert(entries.contains("/SparkR/DESCRIPTION")) + assert(!entries.contains("/package.zip")) + assert(entries.contains("/packageTest/def.R")) + assert(entries.contains("/packageTest/DESCRIPTION")) + } { + zipFile.close() + } + } { FileUtils.deleteDirectory(tempDir) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkHadoopUtilSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkHadoopUtilSuite.scala new file mode 100644 index 000000000000..ab24a76e20a3 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/SparkHadoopUtilSuite.scala @@ -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.deploy + +import java.security.PrivilegedExceptionAction + +import scala.util.Random + +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.permission.{FsAction, FsPermission} +import org.apache.hadoop.security.UserGroupInformation +import org.scalatest.Matchers + +import org.apache.spark.SparkFunSuite + +class SparkHadoopUtilSuite extends SparkFunSuite with Matchers { + test("check file permission") { + import FsAction._ + val testUser = s"user-${Random.nextInt(100)}" + val testGroups = Array(s"group-${Random.nextInt(100)}") + val testUgi = UserGroupInformation.createUserForTesting(testUser, testGroups) + + testUgi.doAs(new PrivilegedExceptionAction[Void] { + override def run(): Void = { + val sparkHadoopUtil = new SparkHadoopUtil + + // If file is owned by user and user has access permission + var status = fileStatus(testUser, testGroups.head, READ_WRITE, READ_WRITE, NONE) + sparkHadoopUtil.checkAccessPermission(status, READ) should be(true) + sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(true) + + // If file is owned by user but user has no access permission + status = fileStatus(testUser, testGroups.head, NONE, READ_WRITE, NONE) + sparkHadoopUtil.checkAccessPermission(status, READ) should be(false) + sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(false) + + val otherUser = s"test-${Random.nextInt(100)}" + val otherGroup = s"test-${Random.nextInt(100)}" + + // If file is owned by user's group and user's group has access permission + status = fileStatus(otherUser, testGroups.head, NONE, READ_WRITE, NONE) + sparkHadoopUtil.checkAccessPermission(status, READ) should be(true) + sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(true) + + // If file is owned by user's group but user's group has no access permission + status = fileStatus(otherUser, testGroups.head, READ_WRITE, NONE, NONE) + sparkHadoopUtil.checkAccessPermission(status, READ) should be(false) + sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(false) + + // If file is owned by other user and this user has access permission + status = fileStatus(otherUser, otherGroup, READ_WRITE, READ_WRITE, READ_WRITE) + sparkHadoopUtil.checkAccessPermission(status, READ) should be(true) + sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(true) + + // If file is owned by other user but this user has no access permission + status = fileStatus(otherUser, otherGroup, READ_WRITE, READ_WRITE, NONE) + sparkHadoopUtil.checkAccessPermission(status, READ) should be(false) + sparkHadoopUtil.checkAccessPermission(status, WRITE) should be(false) + + null + } + }) + } + + private def fileStatus( + owner: String, + group: String, + userAction: FsAction, + groupAction: FsAction, + otherAction: FsAction): FileStatus = { + new FileStatus(0L, + false, + 0, + 0L, + 0L, + 0L, + new FsPermission(userAction, groupAction, otherAction), + owner, + group, + null) + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 1fd470cd3b01..ad801bf8519a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -18,31 +18,34 @@ package org.apache.spark.deploy import java.io._ +import java.net.URI +import java.nio.charset.StandardCharsets +import java.nio.file.Files +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.io.Source -import com.google.common.base.Charsets.UTF_8 import com.google.common.io.ByteStreams -import org.scalatest.Matchers -import org.scalatest.concurrent.Timeouts +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FSDataInputStream, Path} +import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.scalatest.concurrent.TimeLimits import org.scalatest.time.SpanSugar._ import org.apache.spark._ +import org.apache.spark.TestUtils.JavaSourceFromString +import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.SparkSubmit._ import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate -import org.apache.spark.util.{ResetSystemProperties, Utils} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.scheduler.EventLoggingListener +import org.apache.spark.util.{CommandLineUtils, ResetSystemProperties, Utils} -// Note: this suite mixes in ResetSystemProperties because SparkSubmit.main() sets a bunch -// of properties that neeed to be cleared after tests. -class SparkSubmitSuite - extends SparkFunSuite - with Matchers - with ResetSystemProperties - with Timeouts { - - def beforeAll() { - System.setProperty("spark.testing", "true") - } +trait TestPrematureExit { + suite: SparkFunSuite => private val noOpOutputStream = new OutputStream { def write(b: Int) = {} @@ -59,16 +62,19 @@ class SparkSubmitSuite } /** Returns true if the script exits and the given search string is printed. */ - private def testPrematureExit(input: Array[String], searchString: String) = { + private[spark] def testPrematureExit( + input: Array[String], + searchString: String, + mainObject: CommandLineUtils = SparkSubmit) : Unit = { val printStream = new BufferPrintStream() - SparkSubmit.printStream = printStream + mainObject.printStream = printStream @volatile var exitedCleanly = false - SparkSubmit.exitFn = (_) => exitedCleanly = true + mainObject.exitFn = (_) => exitedCleanly = true val thread = new Thread { override def run() = try { - SparkSubmit.main(input) + mainObject.main(input) } catch { // If exceptions occur after the "exit" has happened, fine to ignore them. // These represent code paths not reachable during normal execution. @@ -82,10 +88,26 @@ class SparkSubmitSuite fail(s"Search string '$searchString' not found in $joined") } } +} + +// Note: this suite mixes in ResetSystemProperties because SparkSubmit.main() sets a bunch +// of properties that needed to be cleared after tests. +class SparkSubmitSuite + extends SparkFunSuite + with Matchers + with BeforeAndAfterEach + with ResetSystemProperties + with TimeLimits + with TestPrematureExit { + + override def beforeEach() { + super.beforeEach() + System.setProperty("spark.testing", "true") + } // scalastyle:off println test("prints usage on empty input") { - testPrematureExit(Array[String](), "Usage: spark-submit") + testPrematureExit(Array.empty[String], "Usage: spark-submit") } test("prints usage with only --help") { @@ -133,6 +155,58 @@ class SparkSubmitSuite appArgs.childArgs should be (Seq("--master", "local", "some", "--weird", "args")) } + test("print the right queue name") { + val clArgs = Seq( + "--name", "myApp", + "--class", "Foo", + "--conf", "spark.yarn.queue=thequeue", + "userjar.jar") + val appArgs = new SparkSubmitArguments(clArgs) + appArgs.queue should be ("thequeue") + appArgs.toString should include ("thequeue") + } + + test("specify deploy mode through configuration") { + val clArgs = Seq( + "--master", "yarn", + "--conf", "spark.submit.deployMode=client", + "--class", "org.SomeClass", + "thejar.jar" + ) + val appArgs = new SparkSubmitArguments(clArgs) + val (_, _, sysProps, _) = prepareSubmitEnvironment(appArgs) + + appArgs.deployMode should be ("client") + sysProps("spark.submit.deployMode") should be ("client") + + // Both cmd line and configuration are specified, cmdline option takes the priority + val clArgs1 = Seq( + "--master", "yarn", + "--deploy-mode", "cluster", + "--conf", "spark.submit.deployMode=client", + "-class", "org.SomeClass", + "thejar.jar" + ) + val appArgs1 = new SparkSubmitArguments(clArgs1) + val (_, _, sysProps1, _) = prepareSubmitEnvironment(appArgs1) + + appArgs1.deployMode should be ("cluster") + sysProps1("spark.submit.deployMode") should be ("cluster") + + // Neither cmdline nor configuration are specified, client mode is the default choice + val clArgs2 = Seq( + "--master", "yarn", + "--class", "org.SomeClass", + "thejar.jar" + ) + val appArgs2 = new SparkSubmitArguments(clArgs2) + appArgs2.deployMode should be (null) + + val (_, _, sysProps2, _) = prepareSubmitEnvironment(appArgs2) + appArgs2.deployMode should be ("client") + sysProps2("spark.submit.deployMode") should be ("client") + } + test("handles YARN cluster mode") { val clArgs = Seq( "--deploy-mode", "cluster", @@ -154,21 +228,26 @@ class SparkSubmitSuite val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) val childArgsStr = childArgs.mkString(" ") childArgsStr should include ("--class org.SomeClass") - childArgsStr should include ("--executor-memory 5g") - childArgsStr should include ("--driver-memory 4g") - childArgsStr should include ("--executor-cores 5") childArgsStr should include ("--arg arg1 --arg arg2") - childArgsStr should include ("--queue thequeue") childArgsStr should include regex ("--jar .*thejar.jar") - childArgsStr should include regex ("--addJars .*one.jar,.*two.jar,.*three.jar") - childArgsStr should include regex ("--files .*file1.txt,.*file2.txt") - childArgsStr should include regex ("--archives .*archive1.txt,.*archive2.txt") mainClass should be ("org.apache.spark.deploy.yarn.Client") - classpath should have length (0) + + // In yarn cluster mode, also adding jars to classpath + classpath(0) should endWith ("thejar.jar") + classpath(1) should endWith ("one.jar") + classpath(2) should endWith ("two.jar") + classpath(3) should endWith ("three.jar") + + sysProps("spark.executor.memory") should be ("5g") + sysProps("spark.driver.memory") should be ("4g") + sysProps("spark.executor.cores") should be ("5") + sysProps("spark.yarn.queue") should be ("thequeue") + sysProps("spark.yarn.dist.jars") should include regex (".*one.jar,.*two.jar,.*three.jar") + sysProps("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt") + sysProps("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt") sysProps("spark.app.name") should be ("beauty") sysProps("spark.ui.enabled") should be ("false") sysProps("SPARK_SUBMIT") should be ("true") - sysProps.keys should not contain ("spark.jars") } test("handles YARN client mode") { @@ -204,7 +283,8 @@ class SparkSubmitSuite sysProps("spark.executor.instances") should be ("6") sysProps("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt") sysProps("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt") - sysProps("spark.jars") should include regex (".*one.jar,.*two.jar,.*three.jar,.*thejar.jar") + sysProps("spark.yarn.dist.jars") should include + regex (".*one.jar,.*two.jar,.*three.jar,.*thejar.jar") sysProps("SPARK_SUBMIT") should be ("true") sysProps("spark.ui.enabled") should be ("false") } @@ -314,7 +394,8 @@ class SparkSubmitSuite val appArgs = new SparkSubmitArguments(clArgs) val (_, _, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) sysProps("spark.executor.memory") should be ("5g") - sysProps("spark.master") should be ("yarn-cluster") + sysProps("spark.master") should be ("yarn") + sysProps("spark.submit.deployMode") should be ("cluster") mainClass should be ("org.apache.spark.deploy.yarn.Client") } @@ -330,6 +411,37 @@ class SparkSubmitSuite runSparkSubmit(args) } + test("launch simple application with spark-submit with redaction") { + val testDir = Utils.createTempDir() + testDir.deleteOnExit() + val testDirPath = new Path(testDir.getAbsolutePath()) + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val fileSystem = Utils.getHadoopFileSystem("/", + SparkHadoopUtil.get.newConfiguration(new SparkConf())) + try { + val args = Seq( + "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--conf", "spark.executorEnv.HADOOP_CREDSTORE_PASSWORD=secret_password", + "--conf", "spark.eventLog.enabled=true", + "--conf", "spark.eventLog.testing=true", + "--conf", s"spark.eventLog.dir=${testDirPath.toUri.toString}", + "--conf", "spark.hadoop.fs.defaultFS=unsupported://example.com", + unusedJar.toString) + runSparkSubmit(args) + val listStatus = fileSystem.listStatus(testDirPath) + val logData = EventLoggingListener.openEventLog(listStatus.last.getPath, fileSystem) + Source.fromInputStream(logData).getLines().foreach { line => + assert(!line.contains("secret_password")) + } + } finally { + Utils.deleteRecursively(testDir) + } + } + test("includes jars passed in through --jars") { val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) @@ -366,15 +478,36 @@ class SparkSubmitSuite } } - test("correctly builds R packages included in a jar with --packages") { - // TODO(SPARK-9603): Building a package to $SPARK_HOME/R/lib is unavailable on Jenkins. - // It's hard to write the test in SparkR (because we can't create the repository dynamically) - /* + test("includes jars passed through spark.jars.packages and spark.jars.repositories") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val main = MavenCoordinate("my.great.lib", "mylib", "0.1") + val dep = MavenCoordinate("my.great.dep", "mylib", "0.1") + // Test using "spark.jars.packages" and "spark.jars.repositories" configurations. + IvyTestUtils.withRepository(main, Some(dep.toString), None) { repo => + val args = Seq( + "--class", JarCreationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local-cluster[2,1,1024]", + "--conf", "spark.jars.packages=my.great.lib:mylib:0.1,my.great.dep:mylib:0.1", + "--conf", s"spark.jars.repositories=$repo", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + unusedJar.toString, + "my.great.lib.MyLib", "my.great.dep.MyLib") + runSparkSubmit(args) + } + } + + // TODO(SPARK-9603): Building a package is flaky on Jenkins Maven builds. + // See https://gist.github.com/shivaram/3a2fecce60768a603dac for a error log + ignore("correctly builds R packages included in a jar with --packages") { assume(RUtils.isRInstalled, "R isn't installed on this machine.") + // Check if the SparkR package is installed + assume(RUtils.isSparkRInstalled, "SparkR is not installed in this build.") val main = MavenCoordinate("my.great.lib", "mylib", "0.1") val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) - val rScriptDir = - Seq(sparkHome, "R", "pkg", "inst", "tests", "packageInAJarTest.R").mkString(File.separator) + val rScriptDir = Seq( + sparkHome, "R", "pkg", "tests", "fulltests", "packageInAJarTest.R").mkString(File.separator) assert(new File(rScriptDir).exists) IvyTestUtils.withRepository(main, None, None, withR = true) { repo => val args = Seq( @@ -387,12 +520,46 @@ class SparkSubmitSuite rScriptDir) runSparkSubmit(args) } - */ + } + + test("include an external JAR in SparkR") { + assume(RUtils.isRInstalled, "R isn't installed on this machine.") + val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) + // Check if the SparkR package is installed + assume(RUtils.isSparkRInstalled, "SparkR is not installed in this build.") + val rScriptDir = + Seq(sparkHome, "R", "pkg", "tests", "fulltests", "jarTest.R").mkString(File.separator) + assert(new File(rScriptDir).exists) + + // compile a small jar containing a class that will be called from R code. + val tempDir = Utils.createTempDir() + val srcDir = new File(tempDir, "sparkrtest") + srcDir.mkdirs() + val excSource = new JavaSourceFromString(new File(srcDir, "DummyClass").toURI.getPath, + """package sparkrtest; + | + |public class DummyClass implements java.io.Serializable { + | public static String helloWorld(String arg) { return "Hello " + arg; } + | public static int addStuff(int arg1, int arg2) { return arg1 + arg2; } + |} + """.stripMargin) + val excFile = TestUtils.createCompiledClass("DummyClass", srcDir, excSource, Seq.empty) + val jarFile = new File(tempDir, "sparkRTestJar-%s.jar".format(System.currentTimeMillis())) + val jarURL = TestUtils.createJar(Seq(excFile), jarFile, directoryPrefix = Some("sparkrtest")) + + val args = Seq( + "--name", "testApp", + "--master", "local", + "--jars", jarURL.toString, + "--verbose", + "--conf", "spark.ui.enabled=false", + rScriptDir) + runSparkSubmit(args) } test("resolves command line argument paths correctly") { val jars = "/jar1,/jar2" // --jars - val files = "hdfs:/file1,file2" // --files + val files = "local:/file1,file2" // --files val archives = "file:/archive1,archive2" // --archives val pyFiles = "py-file1,py-file2" // --py-files @@ -412,7 +579,7 @@ class SparkSubmitSuite // Test files and archives (Yarn) val clArgs2 = Seq( - "--master", "yarn-client", + "--master", "yarn", "--class", "org.SomeClass", "--files", files, "--archives", archives, @@ -429,6 +596,8 @@ class SparkSubmitSuite val clArgs3 = Seq( "--master", "local", "--py-files", pyFiles, + "--conf", "spark.pyspark.driver.python=python3.4", + "--conf", "spark.pyspark.python=python3.5", "mister.py" ) val appArgs3 = new SparkSubmitArguments(clArgs3) @@ -436,11 +605,13 @@ class SparkSubmitSuite appArgs3.pyFiles should be (Utils.resolveURIs(pyFiles)) sysProps3("spark.submit.pyFiles") should be ( PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) + sysProps3(PYSPARK_DRIVER_PYTHON.key) should be ("python3.4") + sysProps3(PYSPARK_PYTHON.key) should be ("python3.5") } test("resolves config paths correctly") { val jars = "/jar1,/jar2" // spark.jars - val files = "hdfs:/file1,file2" // spark.files / spark.yarn.dist.files + val files = "local:/file1,file2" // spark.files / spark.yarn.dist.files val archives = "file:/archive1,archive2" // spark.yarn.dist.archives val pyFiles = "py-file1,py-file2" // spark.submit.pyFiles @@ -470,7 +641,7 @@ class SparkSubmitSuite writer2.println("spark.yarn.dist.archives " + archives) writer2.close() val clArgs2 = Seq( - "--master", "yarn-client", + "--master", "yarn", "--class", "org.SomeClass", "--properties-file", f2.getPath, "thejar.jar" @@ -494,6 +665,25 @@ class SparkSubmitSuite val sysProps3 = SparkSubmit.prepareSubmitEnvironment(appArgs3)._3 sysProps3("spark.submit.pyFiles") should be( PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) + + // Test remote python files + val f4 = File.createTempFile("test-submit-remote-python-files", "", tmpDir) + val writer4 = new PrintWriter(f4) + val remotePyFiles = "hdfs:///tmp/file1.py,hdfs:///tmp/file2.py" + writer4.println("spark.submit.pyFiles " + remotePyFiles) + writer4.close() + val clArgs4 = Seq( + "--master", "yarn", + "--deploy-mode", "cluster", + "--properties-file", f4.getPath, + "hdfs:///tmp/mister.py" + ) + val appArgs4 = new SparkSubmitArguments(clArgs4) + val sysProps4 = SparkSubmit.prepareSubmitEnvironment(appArgs4)._3 + // Should not format python path for yarn cluster mode + sysProps4("spark.submit.pyFiles") should be( + Utils.resolveURIs(remotePyFiles) + ) } test("user classpath first in driver") { @@ -525,13 +715,263 @@ class SparkSubmitSuite appArgs.executorMemory should be ("2.3g") } } + + test("comma separated list of files are unioned correctly") { + val left = Option("/tmp/a.jar,/tmp/b.jar") + val right = Option("/tmp/c.jar,/tmp/a.jar") + val emptyString = Option("") + Utils.unionFileLists(left, right) should be (Set("/tmp/a.jar", "/tmp/b.jar", "/tmp/c.jar")) + Utils.unionFileLists(emptyString, emptyString) should be (Set.empty) + Utils.unionFileLists(Option("/tmp/a.jar"), emptyString) should be (Set("/tmp/a.jar")) + Utils.unionFileLists(emptyString, Option("/tmp/a.jar")) should be (Set("/tmp/a.jar")) + Utils.unionFileLists(None, Option("/tmp/a.jar")) should be (Set("/tmp/a.jar")) + Utils.unionFileLists(Option("/tmp/a.jar"), None) should be (Set("/tmp/a.jar")) + } + + test("support glob path") { + val tmpJarDir = Utils.createTempDir() + val jar1 = TestUtils.createJarWithFiles(Map("test.resource" -> "1"), tmpJarDir) + val jar2 = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpJarDir) + + val tmpFileDir = Utils.createTempDir() + val file1 = File.createTempFile("tmpFile1", "", tmpFileDir) + val file2 = File.createTempFile("tmpFile2", "", tmpFileDir) + + val tmpPyFileDir = Utils.createTempDir() + val pyFile1 = File.createTempFile("tmpPy1", ".py", tmpPyFileDir) + val pyFile2 = File.createTempFile("tmpPy2", ".egg", tmpPyFileDir) + + val tmpArchiveDir = Utils.createTempDir() + val archive1 = File.createTempFile("archive1", ".zip", tmpArchiveDir) + val archive2 = File.createTempFile("archive2", ".zip", tmpArchiveDir) + + val args = Seq( + "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), + "--name", "testApp", + "--master", "yarn", + "--deploy-mode", "client", + "--jars", s"${tmpJarDir.getAbsolutePath}/*.jar", + "--files", s"${tmpFileDir.getAbsolutePath}/tmpFile*", + "--py-files", s"${tmpPyFileDir.getAbsolutePath}/tmpPy*", + "--archives", s"${tmpArchiveDir.getAbsolutePath}/*.zip", + jar2.toString) + + val appArgs = new SparkSubmitArguments(args) + val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs)._3 + sysProps("spark.yarn.dist.jars").split(",").toSet should be + (Set(jar1.toURI.toString, jar2.toURI.toString)) + sysProps("spark.yarn.dist.files").split(",").toSet should be + (Set(file1.toURI.toString, file2.toURI.toString)) + sysProps("spark.yarn.dist.pyFiles").split(",").toSet should be + (Set(pyFile1.getAbsolutePath, pyFile2.getAbsolutePath)) + sysProps("spark.yarn.dist.archives").split(",").toSet should be + (Set(archive1.toURI.toString, archive2.toURI.toString)) + } + // scalastyle:on println + private def checkDownloadedFile(sourcePath: String, outputPath: String): Unit = { + if (sourcePath == outputPath) { + return + } + + val sourceUri = new URI(sourcePath) + val outputUri = new URI(outputPath) + assert(outputUri.getScheme === "file") + + // The path and filename are preserved. + assert(outputUri.getPath.endsWith(new Path(sourceUri).getName)) + assert(FileUtils.readFileToString(new File(outputUri.getPath)) === + FileUtils.readFileToString(new File(sourceUri.getPath))) + } + + private def deleteTempOutputFile(outputPath: String): Unit = { + val outputFile = new File(new URI(outputPath).getPath) + if (outputFile.exists) { + outputFile.delete() + } + } + + test("downloadFile - invalid url") { + val sparkConf = new SparkConf(false) + intercept[IOException] { + DependencyUtils.downloadFile( + "abc:/my/file", Utils.createTempDir(), sparkConf, new Configuration(), + new SecurityManager(sparkConf)) + } + } + + test("downloadFile - file doesn't exist") { + val sparkConf = new SparkConf(false) + val hadoopConf = new Configuration() + val tmpDir = Utils.createTempDir() + updateConfWithFakeS3Fs(hadoopConf) + intercept[FileNotFoundException] { + DependencyUtils.downloadFile("s3a:/no/such/file", tmpDir, sparkConf, hadoopConf, + new SecurityManager(sparkConf)) + } + } + + test("downloadFile does not download local file") { + val sparkConf = new SparkConf(false) + val secMgr = new SecurityManager(sparkConf) + // empty path is considered as local file. + val tmpDir = Files.createTempDirectory("tmp").toFile + assert(DependencyUtils.downloadFile("", tmpDir, sparkConf, new Configuration(), secMgr) === "") + assert(DependencyUtils.downloadFile("/local/file", tmpDir, sparkConf, new Configuration(), + secMgr) === "/local/file") + } + + test("download one file to local") { + val sparkConf = new SparkConf(false) + val jarFile = File.createTempFile("test", ".jar") + jarFile.deleteOnExit() + val content = "hello, world" + FileUtils.write(jarFile, content) + val hadoopConf = new Configuration() + val tmpDir = Files.createTempDirectory("tmp").toFile + updateConfWithFakeS3Fs(hadoopConf) + val sourcePath = s"s3a://${jarFile.toURI.getPath}" + val outputPath = DependencyUtils.downloadFile(sourcePath, tmpDir, sparkConf, hadoopConf, + new SecurityManager(sparkConf)) + checkDownloadedFile(sourcePath, outputPath) + deleteTempOutputFile(outputPath) + } + + test("download list of files to local") { + val sparkConf = new SparkConf(false) + val jarFile = File.createTempFile("test", ".jar") + jarFile.deleteOnExit() + val content = "hello, world" + FileUtils.write(jarFile, content) + val hadoopConf = new Configuration() + val tmpDir = Files.createTempDirectory("tmp").toFile + updateConfWithFakeS3Fs(hadoopConf) + val sourcePaths = Seq("/local/file", s"s3a://${jarFile.toURI.getPath}") + val outputPaths = DependencyUtils + .downloadFileList(sourcePaths.mkString(","), tmpDir, sparkConf, hadoopConf, + new SecurityManager(sparkConf)) + .split(",") + + assert(outputPaths.length === sourcePaths.length) + sourcePaths.zip(outputPaths).foreach { case (sourcePath, outputPath) => + checkDownloadedFile(sourcePath, outputPath) + deleteTempOutputFile(outputPath) + } + } + + test("Avoid re-upload remote resources in yarn client mode") { + val hadoopConf = new Configuration() + updateConfWithFakeS3Fs(hadoopConf) + + val tmpDir = Utils.createTempDir() + val file = File.createTempFile("tmpFile", "", tmpDir) + val pyFile = File.createTempFile("tmpPy", ".egg", tmpDir) + val mainResource = File.createTempFile("tmpPy", ".py", tmpDir) + val tmpJar = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpDir) + val tmpJarPath = s"s3a://${new File(tmpJar.toURI).getAbsolutePath}" + + val args = Seq( + "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), + "--name", "testApp", + "--master", "yarn", + "--deploy-mode", "client", + "--jars", tmpJarPath, + "--files", s"s3a://${file.getAbsolutePath}", + "--py-files", s"s3a://${pyFile.getAbsolutePath}", + s"s3a://$mainResource" + ) + + val appArgs = new SparkSubmitArguments(args) + val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs, Some(hadoopConf))._3 + + // All the resources should still be remote paths, so that YARN client will not upload again. + sysProps("spark.yarn.dist.jars") should be (tmpJarPath) + sysProps("spark.yarn.dist.files") should be (s"s3a://${file.getAbsolutePath}") + sysProps("spark.yarn.dist.pyFiles") should be (s"s3a://${pyFile.getAbsolutePath}") + + // Local repl jars should be a local path. + sysProps("spark.repl.local.jars") should (startWith("file:")) + + // local py files should not be a URI format. + sysProps("spark.submit.pyFiles") should (startWith("/")) + } + + test("download remote resource if it is not supported by yarn service") { + testRemoteResources(isHttpSchemeBlacklisted = false, supportMockHttpFs = false) + } + + test("avoid downloading remote resource if it is supported by yarn service") { + testRemoteResources(isHttpSchemeBlacklisted = false, supportMockHttpFs = true) + } + + test("force download from blacklisted schemes") { + testRemoteResources(isHttpSchemeBlacklisted = true, supportMockHttpFs = true) + } + + private def testRemoteResources(isHttpSchemeBlacklisted: Boolean, + supportMockHttpFs: Boolean): Unit = { + val hadoopConf = new Configuration() + updateConfWithFakeS3Fs(hadoopConf) + if (supportMockHttpFs) { + hadoopConf.set("fs.http.impl", classOf[TestFileSystem].getCanonicalName) + hadoopConf.set("fs.http.impl.disable.cache", "true") + } + + val tmpDir = Utils.createTempDir() + val mainResource = File.createTempFile("tmpPy", ".py", tmpDir) + val tmpS3Jar = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpDir) + val tmpS3JarPath = s"s3a://${new File(tmpS3Jar.toURI).getAbsolutePath}" + val tmpHttpJar = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpDir) + val tmpHttpJarPath = s"http://${new File(tmpHttpJar.toURI).getAbsolutePath}" + + val args = Seq( + "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), + "--name", "testApp", + "--master", "yarn", + "--deploy-mode", "client", + "--jars", s"$tmpS3JarPath,$tmpHttpJarPath", + s"s3a://$mainResource" + ) ++ ( + if (isHttpSchemeBlacklisted) { + Seq("--conf", "spark.yarn.dist.forceDownloadSchemes=http,https") + } else { + Nil + } + ) + + val appArgs = new SparkSubmitArguments(args) + val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs, Some(hadoopConf))._3 + + val jars = sysProps("spark.yarn.dist.jars").split(",").toSet + + // The URI of remote S3 resource should still be remote. + assert(jars.contains(tmpS3JarPath)) + + if (supportMockHttpFs) { + // If Http FS is supported by yarn service, the URI of remote http resource should + // still be remote. + assert(jars.contains(tmpHttpJarPath)) + } else { + // If Http FS is not supported by yarn service, or http scheme is configured to be force + // downloading, the URI of remote http resource should be changed to a local one. + val jarName = new File(tmpHttpJar.toURI).getName + val localHttpJar = jars.filter(_.contains(jarName)) + localHttpJar.size should be(1) + localHttpJar.head should startWith("file:") + } + } + // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. private def runSparkSubmit(args: Seq[String]): Unit = { val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) + val sparkSubmitFile = if (Utils.isWindows) { + new File("..\\bin\\spark-submit.cmd") + } else { + new File("../bin/spark-submit") + } val process = Utils.executeCommand( - Seq("./bin/spark-submit") ++ args, + Seq(sparkSubmitFile.getCanonicalPath) ++ args, new File(sparkHome), Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) @@ -550,7 +990,7 @@ class SparkSubmitSuite val tmpDir = Utils.createTempDir() val defaultsConf = new File(tmpDir.getAbsolutePath, "spark-defaults.conf") - val writer = new OutputStreamWriter(new FileOutputStream(defaultsConf)) + val writer = new OutputStreamWriter(new FileOutputStream(defaultsConf), StandardCharsets.UTF_8) for ((key, value) <- defaults) writer.write(s"$key $value\n") writer.close() @@ -561,6 +1001,11 @@ class SparkSubmitSuite Utils.deleteRecursively(tmpDir) } } + + private def updateConfWithFakeS3Fs(conf: Configuration): Unit = { + conf.set("fs.s3a.impl", classOf[TestFileSystem].getCanonicalName) + conf.set("fs.s3a.impl.disable.cache", "true") + } } object JarCreationTest extends Logging { @@ -575,7 +1020,7 @@ object JarCreationTest extends Logging { Utils.classForName(args(1)) } catch { case t: Throwable => - exception = t + "\n" + t.getStackTraceString + exception = t + "\n" + Utils.exceptionString(t) exception = exception.replaceAll("\n", "\n\t") } Option(exception).toSeq.iterator @@ -618,9 +1063,39 @@ object UserClasspathFirstTest { val ccl = Thread.currentThread().getContextClassLoader() val resource = ccl.getResourceAsStream("test.resource") val bytes = ByteStreams.toByteArray(resource) - val contents = new String(bytes, 0, bytes.length, UTF_8) + val contents = new String(bytes, 0, bytes.length, StandardCharsets.UTF_8) if (contents != "USER") { throw new SparkException("Should have read user resource, but instead read: " + contents) } } } + +class TestFileSystem extends org.apache.hadoop.fs.LocalFileSystem { + private def local(path: Path): Path = { + // Ignore the scheme for testing. + new Path(path.toUri.getPath) + } + + private def toRemote(status: FileStatus): FileStatus = { + val path = s"s3a://${status.getPath.toUri.getPath}" + status.setPath(new Path(path)) + status + } + + override def isFile(path: Path): Boolean = super.isFile(local(path)) + + override def globStatus(pathPattern: Path): Array[FileStatus] = { + val newPath = new Path(pathPattern.toUri.getPath) + super.globStatus(newPath).map(toRemote) + } + + override def listStatus(path: Path): Array[FileStatus] = { + super.listStatus(local(path)).map(toRemote) + } + + override def copyToLocalFile(src: Path, dst: Path): Unit = { + super.copyToLocalFile(local(src), dst) + } + + override def open(path: Path): FSDataInputStream = super.open(local(path)) +} 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 63c346c1b890..eb8c203ae775 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -17,14 +17,16 @@ package org.apache.spark.deploy -import java.io.{File, PrintStream, OutputStream} +import java.io.{File, OutputStream, PrintStream} +import java.nio.charset.StandardCharsets import scala.collection.mutable.ArrayBuffer -import org.scalatest.BeforeAndAfterAll +import com.google.common.io.Files import org.apache.ivy.core.module.descriptor.MDArtifact import org.apache.ivy.core.settings.IvySettings -import org.apache.ivy.plugins.resolver.{AbstractResolver, FileSystemResolver, IBiblioResolver} +import org.apache.ivy.plugins.resolver.{AbstractResolver, ChainResolver, FileSystemResolver, IBiblioResolver} +import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkFunSuite import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate @@ -66,30 +68,34 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { test("create repo resolvers") { val settings = new IvySettings - val res1 = SparkSubmitUtils.createRepoResolvers(None, settings) + val res1 = SparkSubmitUtils.createRepoResolvers(settings.getDefaultIvyUserDir) // should have central and spark-packages by default assert(res1.getResolvers.size() === 4) assert(res1.getResolvers.get(0).asInstanceOf[IBiblioResolver].getName === "local-m2-cache") assert(res1.getResolvers.get(1).asInstanceOf[FileSystemResolver].getName === "local-ivy-cache") assert(res1.getResolvers.get(2).asInstanceOf[IBiblioResolver].getName === "central") assert(res1.getResolvers.get(3).asInstanceOf[IBiblioResolver].getName === "spark-packages") + } + test("create additional resolvers") { val repos = "a/1,b/2,c/3" - val resolver2 = SparkSubmitUtils.createRepoResolvers(Option(repos), settings) - assert(resolver2.getResolvers.size() === 7) + val settings = SparkSubmitUtils.buildIvySettings(Option(repos), None) + val resolver = settings.getDefaultResolver.asInstanceOf[ChainResolver] + assert(resolver.getResolvers.size() === 4) val expected = repos.split(",").map(r => s"$r/") - resolver2.getResolvers.toArray.zipWithIndex.foreach { case (resolver: AbstractResolver, i) => - if (i < 3) { - assert(resolver.getName === s"repo-${i + 1}") - assert(resolver.asInstanceOf[IBiblioResolver].getRoot === expected(i)) - } + resolver.getResolvers.toArray.map(_.asInstanceOf[AbstractResolver]).zipWithIndex.foreach { + case (r, i) => + if (1 < i && i < 3) { + assert(r.getName === s"repo-$i") + assert(r.asInstanceOf[IBiblioResolver].getRoot === expected(i - 1)) + } } } test("add dependencies works correctly") { val md = SparkSubmitUtils.getModuleDescriptor - val artifacts = SparkSubmitUtils.extractMavenCoordinates("com.databricks:spark-csv_2.10:0.1," + - "com.databricks:spark-avro_2.10:0.1") + val artifacts = SparkSubmitUtils.extractMavenCoordinates("com.databricks:spark-csv_2.11:0.1," + + "com.databricks:spark-avro_2.11:0.1") SparkSubmitUtils.addDependenciesToIvy(md, artifacts, "default") assert(md.getDependencies.length === 2) @@ -126,8 +132,10 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { val main = MavenCoordinate("my.awesome.lib", "mylib", "0.1") IvyTestUtils.withRepository(main, None, None) { repo => // end to end - val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, Option(repo), - Option(tempIvyPath), isTest = true) + val jarPath = SparkSubmitUtils.resolveMavenCoordinates( + main.toString, + SparkSubmitUtils.buildIvySettings(Option(repo), Option(tempIvyPath)), + isTest = true) assert(jarPath.indexOf(tempIvyPath) >= 0, "should use non-default ivy path") } } @@ -137,7 +145,9 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { val dep = "my.great.dep:mydep:0.5" // Local M2 repository IvyTestUtils.withRepository(main, Some(dep), Some(SparkSubmitUtils.m2Path)) { repo => - val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, None, None, + val jarPath = SparkSubmitUtils.resolveMavenCoordinates( + main.toString, + SparkSubmitUtils.buildIvySettings(None, None), isTest = true) assert(jarPath.indexOf("mylib") >= 0, "should find artifact") assert(jarPath.indexOf("mydep") >= 0, "should find dependency") @@ -146,7 +156,9 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { val settings = new IvySettings val ivyLocal = new File(settings.getDefaultIvyUserDir, "local" + File.separator) IvyTestUtils.withRepository(main, Some(dep), Some(ivyLocal), useIvyLayout = true) { repo => - val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, None, None, + val jarPath = SparkSubmitUtils.resolveMavenCoordinates( + main.toString, + SparkSubmitUtils.buildIvySettings(None, None), isTest = true) assert(jarPath.indexOf("mylib") >= 0, "should find artifact") assert(jarPath.indexOf("mydep") >= 0, "should find dependency") @@ -156,8 +168,10 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { settings.setDefaultIvyUserDir(new File(tempIvyPath)) IvyTestUtils.withRepository(main, Some(dep), Some(dummyIvyLocal), useIvyLayout = true, ivySettings = settings) { repo => - val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, None, - Some(tempIvyPath), isTest = true) + val jarPath = SparkSubmitUtils.resolveMavenCoordinates( + main.toString, + SparkSubmitUtils.buildIvySettings(None, Some(tempIvyPath)), + isTest = true) assert(jarPath.indexOf("mylib") >= 0, "should find artifact") assert(jarPath.indexOf(tempIvyPath) >= 0, "should be in new ivy path") assert(jarPath.indexOf("mydep") >= 0, "should find dependency") @@ -166,24 +180,29 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { test("dependency not found throws RuntimeException") { intercept[RuntimeException] { - SparkSubmitUtils.resolveMavenCoordinates("a:b:c", None, None, isTest = true) + SparkSubmitUtils.resolveMavenCoordinates( + "a:b:c", + SparkSubmitUtils.buildIvySettings(None, None), + isTest = true) } } test("neglects Spark and Spark's dependencies") { - val components = Seq("bagel_", "catalyst_", "core_", "graphx_", "hive_", "mllib_", "repl_", - "sql_", "streaming_", "yarn_", "network-common_", "network-shuffle_", "network-yarn_") + val coordinates = SparkSubmitUtils.IVY_DEFAULT_EXCLUDES + .map(comp => s"org.apache.spark:spark-${comp}2.11:2.1.1") + .mkString(",") + ",org.apache.spark:spark-core_fake:1.2.0" - val coordinates = - components.map(comp => s"org.apache.spark:spark-${comp}2.10:1.2.0").mkString(",") + - ",org.apache.spark:spark-core_fake:1.2.0" - - val path = SparkSubmitUtils.resolveMavenCoordinates(coordinates, None, None, isTest = true) + val path = SparkSubmitUtils.resolveMavenCoordinates( + coordinates, + SparkSubmitUtils.buildIvySettings(None, None), + isTest = true) assert(path === "", "should return empty path") - val main = MavenCoordinate("org.apache.spark", "spark-streaming-kafka-assembly_2.10", "1.2.0") + val main = MavenCoordinate("org.apache.spark", "spark-streaming-kafka-assembly_2.11", "1.2.0") IvyTestUtils.withRepository(main, None, None) { repo => - val files = SparkSubmitUtils.resolveMavenCoordinates(coordinates + "," + main.toString, - Some(repo), None, isTest = true) + val files = SparkSubmitUtils.resolveMavenCoordinates( + coordinates + "," + main.toString, + SparkSubmitUtils.buildIvySettings(Some(repo), None), + isTest = true) assert(files.indexOf(main.artifactId) >= 0, "Did not return artifact") } } @@ -192,10 +211,49 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { val main = new MavenCoordinate("my.great.lib", "mylib", "0.1") val dep = "my.great.dep:mydep:0.5" IvyTestUtils.withRepository(main, Some(dep), None) { repo => - val files = SparkSubmitUtils.resolveMavenCoordinates(main.toString, - Some(repo), None, Seq("my.great.dep:mydep"), isTest = true) + val files = SparkSubmitUtils.resolveMavenCoordinates( + main.toString, + SparkSubmitUtils.buildIvySettings(Some(repo), None), + Seq("my.great.dep:mydep"), + isTest = true) assert(files.indexOf(main.artifactId) >= 0, "Did not return artifact") assert(files.indexOf("my.great.dep") < 0, "Returned excluded artifact") } } + + test("load ivy settings file") { + val main = new MavenCoordinate("my.great.lib", "mylib", "0.1") + val dep = "my.great.dep:mydep:0.5" + val dummyIvyLocal = new File(tempIvyPath, "local" + File.separator) + val settingsText = + s""" + | + | + | + | + | + | + | + | + | + | + |""".stripMargin + + val settingsFile = new File(tempIvyPath, "ivysettings.xml") + Files.write(settingsText, settingsFile, StandardCharsets.UTF_8) + val settings = SparkSubmitUtils.loadIvySettings(settingsFile.toString, None, None) + settings.setDefaultIvyUserDir(new File(tempIvyPath)) // NOTE - can't set this through file + + val testUtilSettings = new IvySettings + testUtilSettings.setDefaultIvyUserDir(new File(tempIvyPath)) + IvyTestUtils.withRepository(main, Some(dep), Some(dummyIvyLocal), useIvyLayout = true, + ivySettings = testUtilSettings) { repo => + val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, settings, isTest = true) + assert(jarPath.indexOf("mylib") >= 0, "should find artifact") + assert(jarPath.indexOf(tempIvyPath) >= 0, "should be in new ivy path") + assert(jarPath.indexOf("mydep") >= 0, "should find dependency") + } + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index d145e78834b1..bf7480d79f8a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -17,10 +17,12 @@ package org.apache.spark.deploy +import scala.collection.mutable import scala.concurrent.duration._ -import org.mockito.Mockito.{mock, when} -import org.scalatest.BeforeAndAfterAll +import org.mockito.Matchers.any +import org.mockito.Mockito.{mock, verify, when} +import org.scalatest.{BeforeAndAfterAll, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.apache.spark._ @@ -28,9 +30,11 @@ import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMaste import org.apache.spark.deploy.master.ApplicationInfo import org.apache.spark.deploy.master.Master import org.apache.spark.deploy.worker.Worker +import org.apache.spark.internal.config import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} +import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster._ -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RegisterExecutor +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RegisterExecutorFailed} /** * End-to-end tests for dynamic allocation in standalone mode. @@ -38,7 +42,8 @@ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RegisterE class StandaloneDynamicAllocationSuite extends SparkFunSuite with LocalSparkContext - with BeforeAndAfterAll { + with BeforeAndAfterAll + with PrivateMethodTester { private val numWorkers = 2 private val conf = new SparkConf() @@ -68,15 +73,18 @@ class StandaloneDynamicAllocationSuite } override def afterAll(): Unit = { - masterRpcEnv.shutdown() - workerRpcEnvs.foreach(_.shutdown()) - master.stop() - workers.foreach(_.stop()) - masterRpcEnv = null - workerRpcEnvs = null - master = null - workers = null - super.afterAll() + try { + masterRpcEnv.shutdown() + workerRpcEnvs.foreach(_.shutdown()) + master.stop() + workers.foreach(_.stop()) + masterRpcEnv = null + workerRpcEnvs = null + master = null + workers = null + } finally { + super.afterAll() + } } test("dynamic allocation default behavior") { @@ -348,12 +356,13 @@ class StandaloneDynamicAllocationSuite test("kill the same executor twice (SPARK-9795)") { sc = new SparkContext(appConf) val appId = sc.applicationId + sc.requestExecutors(2) eventually(timeout(10.seconds), interval(10.millis)) { val apps = getApplications() assert(apps.size === 1) assert(apps.head.id === appId) assert(apps.head.executors.size === 2) - assert(apps.head.getExecutorLimit === Int.MaxValue) + assert(apps.head.getExecutorLimit === 2) } // sync executors between the Master and the driver, needed because // the driver refuses to kill executors it does not know about @@ -362,7 +371,7 @@ class StandaloneDynamicAllocationSuite val executors = getExecutorIds(sc) assert(executors.size === 2) assert(sc.killExecutor(executors.head)) - assert(sc.killExecutor(executors.head)) + assert(!sc.killExecutor(executors.head)) val apps = getApplications() assert(apps.head.executors.size === 1) // The limit should not be lowered twice @@ -372,36 +381,140 @@ class StandaloneDynamicAllocationSuite test("the pending replacement executors should not be lost (SPARK-10515)") { sc = new SparkContext(appConf) val appId = sc.applicationId + sc.requestExecutors(2) eventually(timeout(10.seconds), interval(10.millis)) { val apps = getApplications() assert(apps.size === 1) assert(apps.head.id === appId) assert(apps.head.executors.size === 2) - assert(apps.head.getExecutorLimit === Int.MaxValue) + assert(apps.head.getExecutorLimit === 2) } // sync executors between the Master and the driver, needed because // the driver refuses to kill executors it does not know about syncExecutors(sc) val executors = getExecutorIds(sc) + val executorIdsBefore = executors.toSet assert(executors.size === 2) - // kill executor 1, and replace it + // kill and replace an executor assert(sc.killAndReplaceExecutor(executors.head)) eventually(timeout(10.seconds), interval(10.millis)) { val apps = getApplications() assert(apps.head.executors.size === 2) + val executorIdsAfter = getExecutorIds(sc).toSet + // make sure the executor was killed and replaced + assert(executorIdsBefore != executorIdsAfter) } + // kill old executor (which is killedAndReplaced) should fail + assert(!sc.killExecutor(executors.head)) + + // refresh executors list + val newExecutors = getExecutorIds(sc) + syncExecutors(sc) + + // kill newly created executor and do not replace it + assert(sc.killExecutor(newExecutors(1))) + val apps = getApplications() + assert(apps.head.executors.size === 1) + assert(apps.head.getExecutorLimit === 1) + } + + test("disable force kill for busy executors (SPARK-9552)") { + sc = new SparkContext(appConf) + val appId = sc.applicationId + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(apps.size === 1) + assert(apps.head.id === appId) + assert(apps.head.executors.size === 2) + assert(apps.head.getExecutorLimit === Int.MaxValue) + } var apps = getApplications() - // kill executor 1 - assert(sc.killExecutor(executors.head)) + // sync executors between the Master and the driver, needed because + // the driver refuses to kill executors it does not know about + syncExecutors(sc) + val executors = getExecutorIds(sc) + assert(executors.size === 2) + + // simulate running a task on the executor + val getMap = + PrivateMethod[mutable.HashMap[String, mutable.HashSet[Long]]]('executorIdToRunningTaskIds) + val taskScheduler = sc.taskScheduler.asInstanceOf[TaskSchedulerImpl] + val executorIdToRunningTaskIds = taskScheduler invokePrivate getMap() + executorIdToRunningTaskIds(executors.head) = mutable.HashSet(1L) + // kill the busy executor without force; this should fail + assert(killExecutor(sc, executors.head, force = false).isEmpty) apps = getApplications() assert(apps.head.executors.size === 2) - assert(apps.head.getExecutorLimit === 2) - // kill executor 2 - assert(sc.killExecutor(executors(1))) + + // force kill busy executor + assert(killExecutor(sc, executors.head, force = true).nonEmpty) apps = getApplications() + // kill executor successfully assert(apps.head.executors.size === 1) - assert(apps.head.getExecutorLimit === 1) + } + + test("initial executor limit") { + val initialExecutorLimit = 1 + val myConf = appConf + .set("spark.dynamicAllocation.enabled", "true") + .set("spark.shuffle.service.enabled", "true") + .set("spark.dynamicAllocation.initialExecutors", initialExecutorLimit.toString) + sc = new SparkContext(myConf) + val appId = sc.applicationId + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(apps.size === 1) + assert(apps.head.id === appId) + assert(apps.head.executors.size === initialExecutorLimit) + assert(apps.head.getExecutorLimit === initialExecutorLimit) + } + } + + test("kill all executors on localhost") { + sc = new SparkContext(appConf) + val appId = sc.applicationId + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(apps.size === 1) + assert(apps.head.id === appId) + assert(apps.head.executors.size === 2) + assert(apps.head.getExecutorLimit === Int.MaxValue) + } + val beforeList = getApplications().head.executors.keys.toSet + assert(killExecutorsOnHost(sc, "localhost").equals(true)) + + syncExecutors(sc) + val afterList = getApplications().head.executors.keys.toSet + + eventually(timeout(10.seconds), interval(100.millis)) { + assert(beforeList.intersect(afterList).size == 0) + } + } + + test("executor registration on a blacklisted host must fail") { + sc = new SparkContext(appConf.set(config.BLACKLIST_ENABLED.key, "true")) + val endpointRef = mock(classOf[RpcEndpointRef]) + val mockAddress = mock(classOf[RpcAddress]) + when(endpointRef.address).thenReturn(mockAddress) + val message = RegisterExecutor("one", endpointRef, "blacklisted-host", 10, Map.empty) + + // Get "localhost" on a blacklist. + val taskScheduler = mock(classOf[TaskSchedulerImpl]) + when(taskScheduler.nodeBlacklist()).thenReturn(Set("blacklisted-host")) + when(taskScheduler.sc).thenReturn(sc) + sc.taskScheduler = taskScheduler + + // Create a fresh scheduler backend to blacklist "localhost". + sc.schedulerBackend.stop() + val backend = + new StandaloneSchedulerBackend(taskScheduler, sc, Array(masterRpcEnv.address.toSparkURL)) + backend.start() + + backend.driverEndpoint.ask[Boolean](message) + eventually(timeout(10.seconds), interval(100.millis)) { + verify(endpointRef).send(RegisterExecutorFailed(any())) + } } // =============================== @@ -428,7 +541,7 @@ class StandaloneDynamicAllocationSuite (0 until numWorkers).map { i => val rpcEnv = workerRpcEnvs(i) val worker = new Worker(rpcEnv, 0, cores, memory, Array(masterRpcEnv.address), - Worker.SYSTEM_NAME + i, Worker.ENDPOINT_NAME, null, conf, securityManager) + Worker.ENDPOINT_NAME, null, conf, securityManager) rpcEnv.setupEndpoint(Worker.ENDPOINT_NAME, worker) worker } @@ -436,10 +549,10 @@ class StandaloneDynamicAllocationSuite /** Get the Master state */ private def getMasterState: MasterStateResponse = { - master.self.askWithRetry[MasterStateResponse](RequestMasterState) + master.self.askSync[MasterStateResponse](RequestMasterState) } - /** Get the applictions that are active from Master */ + /** Get the applications that are active from Master */ private def getApplications(): Seq[ApplicationInfo] = { getMasterState.activeApps } @@ -455,6 +568,26 @@ class StandaloneDynamicAllocationSuite sc.killExecutors(getExecutorIds(sc).take(n)) } + /** Kill the given executor, specifying whether to force kill it. */ + private def killExecutor(sc: SparkContext, executorId: String, force: Boolean): Seq[String] = { + syncExecutors(sc) + sc.schedulerBackend match { + case b: CoarseGrainedSchedulerBackend => + b.killExecutors(Seq(executorId), replace = false, force) + case _ => fail("expected coarse grained scheduler") + } + } + + /** Kill the executors on a given host. */ + private def killExecutorsOnHost(sc: SparkContext, host: String): Boolean = { + syncExecutors(sc) + sc.schedulerBackend match { + case b: CoarseGrainedSchedulerBackend => + b.killExecutorsOnHost(host) + case _ => fail("expected coarse grained scheduler") + } + } + /** * Return a list of executor IDs belonging to this application. * @@ -484,13 +617,12 @@ class StandaloneDynamicAllocationSuite val missingExecutors = masterExecutors.toSet.diff(driverExecutors.toSet).toSeq.sorted missingExecutors.foreach { id => // Fake an executor registration so the driver knows about us - val port = System.currentTimeMillis % 65536 val endpointRef = mock(classOf[RpcEndpointRef]) val mockAddress = mock(classOf[RpcAddress]) when(endpointRef.address).thenReturn(mockAddress) - val message = RegisterExecutor(id, endpointRef, s"localhost:$port", 10, Map.empty) + val message = RegisterExecutor(id, endpointRef, "localhost", 10, Map.empty) val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend] - backend.driverEndpoint.askWithRetry[CoarseGrainedClusterMessage](message) + backend.driverEndpoint.askSync[Boolean](message) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala new file mode 100644 index 000000000000..a1707e6540b3 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala @@ -0,0 +1,231 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.client + +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.concurrent.duration._ + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.concurrent.{Eventually, ScalaFutures} + +import org.apache.spark._ +import org.apache.spark.deploy.{ApplicationDescription, Command} +import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} +import org.apache.spark.deploy.master.{ApplicationInfo, Master} +import org.apache.spark.deploy.worker.Worker +import org.apache.spark.internal.Logging +import org.apache.spark.rpc.RpcEnv +import org.apache.spark.util.Utils + +/** + * End-to-end tests for application client in standalone mode. + */ +class AppClientSuite + extends SparkFunSuite + with LocalSparkContext + with BeforeAndAfterAll + with Eventually + with ScalaFutures { + private val numWorkers = 2 + private val conf = new SparkConf() + private val securityManager = new SecurityManager(conf) + + private var masterRpcEnv: RpcEnv = null + private var workerRpcEnvs: Seq[RpcEnv] = null + private var master: Master = null + private var workers: Seq[Worker] = null + + /** + * Start the local cluster. + * Note: local-cluster mode is insufficient because we want a reference to the Master. + */ + override def beforeAll(): Unit = { + super.beforeAll() + masterRpcEnv = RpcEnv.create(Master.SYSTEM_NAME, "localhost", 0, conf, securityManager) + workerRpcEnvs = (0 until numWorkers).map { i => + RpcEnv.create(Worker.SYSTEM_NAME + i, "localhost", 0, conf, securityManager) + } + master = makeMaster() + workers = makeWorkers(10, 2048) + // Wait until all workers register with master successfully + eventually(timeout(60.seconds), interval(10.millis)) { + assert(getMasterState.workers.size === numWorkers) + } + } + + override def afterAll(): Unit = { + try { + workerRpcEnvs.foreach(_.shutdown()) + masterRpcEnv.shutdown() + workers.foreach(_.stop()) + master.stop() + workerRpcEnvs = null + masterRpcEnv = null + workers = null + master = null + } finally { + super.afterAll() + } + } + + test("interface methods of AppClient using local Master") { + val ci = new AppClientInst(masterRpcEnv.address.toSparkURL) + + ci.client.start() + + // Client should connect with one Master which registers the application + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(ci.listener.connectedIdList.size === 1, "client listener should have one connection") + assert(apps.size === 1, "master should have 1 registered app") + } + + // Send message to Master to request Executors, verify request by change in executor limit + val numExecutorsRequested = 1 + whenReady( + ci.client.requestTotalExecutors(numExecutorsRequested), + timeout(10.seconds), + interval(10.millis)) { acknowledged => + assert(acknowledged) + } + + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(apps.head.getExecutorLimit === numExecutorsRequested, s"executor request failed") + } + + // Send request to kill executor, verify request was made + val executorId: String = getApplications().head.executors.head._2.fullId + whenReady( + ci.client.killExecutors(Seq(executorId)), + timeout(10.seconds), + interval(10.millis)) { acknowledged => + assert(acknowledged) + } + + // Issue stop command for Client to disconnect from Master + ci.client.stop() + + // Verify Client is marked dead and unregistered from Master + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(ci.listener.deadReasonList.size === 1, "client should have been marked dead") + assert(apps.isEmpty, "master should have 0 registered apps") + } + } + + test("request from AppClient before initialized with master") { + val ci = new AppClientInst(masterRpcEnv.address.toSparkURL) + + // requests to master should fail immediately + whenReady(ci.client.requestTotalExecutors(3), timeout(1.seconds)) { success => + assert(success === false) + } + } + + // =============================== + // | Utility methods for testing | + // =============================== + + /** Return a SparkConf for applications that want to talk to our Master. */ + private def appConf: SparkConf = { + new SparkConf() + .setMaster(masterRpcEnv.address.toSparkURL) + .setAppName("test") + .set("spark.executor.memory", "256m") + } + + /** Make a master to which our application will send executor requests. */ + private def makeMaster(): Master = { + val master = new Master(masterRpcEnv, masterRpcEnv.address, 0, securityManager, conf) + masterRpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) + master + } + + /** Make a few workers that talk to our master. */ + private def makeWorkers(cores: Int, memory: Int): Seq[Worker] = { + (0 until numWorkers).map { i => + val rpcEnv = workerRpcEnvs(i) + val worker = new Worker(rpcEnv, 0, cores, memory, Array(masterRpcEnv.address), + Worker.ENDPOINT_NAME, null, conf, securityManager) + rpcEnv.setupEndpoint(Worker.ENDPOINT_NAME, worker) + worker + } + } + + /** Get the Master state */ + private def getMasterState: MasterStateResponse = { + master.self.askSync[MasterStateResponse](RequestMasterState) + } + + /** Get the applications that are active from Master */ + private def getApplications(): Seq[ApplicationInfo] = { + getMasterState.activeApps + } + + /** Application Listener to collect events */ + private class AppClientCollector extends StandaloneAppClientListener with Logging { + val connectedIdList = new ConcurrentLinkedQueue[String]() + @volatile var disconnectedCount: Int = 0 + val deadReasonList = new ConcurrentLinkedQueue[String]() + val execAddedList = new ConcurrentLinkedQueue[String]() + val execRemovedList = new ConcurrentLinkedQueue[String]() + + def connected(id: String): Unit = { + connectedIdList.add(id) + } + + def disconnected(): Unit = { + synchronized { + disconnectedCount += 1 + } + } + + def dead(reason: String): Unit = { + deadReasonList.add(reason) + } + + def executorAdded( + id: String, + workerId: String, + hostPort: String, + cores: Int, + memory: Int): Unit = { + execAddedList.add(id) + } + + def executorRemoved( + id: String, message: String, exitStatus: Option[Int], workerLost: Boolean): Unit = { + execRemovedList.add(id) + } + + def workerRemoved(workerId: String, host: String, message: String): Unit = {} + } + + /** Create AppClient and supporting objects */ + private class AppClientInst(masterUrl: String) { + val rpcEnv = RpcEnv.create("spark", Utils.localHostName(), 0, conf, securityManager) + private val cmd = new Command(TestExecutor.getClass.getCanonicalName.stripSuffix("$"), + List(), Map(), Seq(), Seq(), Seq()) + private val desc = new ApplicationDescription("AppClientSuite", Some(1), 512, cmd, "ignored") + val listener = new AppClientCollector + val client = new StandaloneAppClient(rpcEnv, Array(masterUrl), desc, listener, new SparkConf) + } + +} diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala b/core/src/test/scala/org/apache/spark/deploy/client/TestExecutor.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala rename to core/src/test/scala/org/apache/spark/deploy/client/TestExecutor.scala diff --git a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala new file mode 100644 index 000000000000..6e50e8454904 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala @@ -0,0 +1,488 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import java.util.{Date, NoSuchElementException} +import javax.servlet.Filter +import javax.servlet.http.{HttpServletRequest, HttpServletResponse} + +import scala.collection.mutable +import scala.collection.mutable.ListBuffer + +import com.codahale.metrics.Counter +import com.google.common.cache.LoadingCache +import com.google.common.util.concurrent.UncheckedExecutionException +import org.eclipse.jetty.servlet.ServletContextHandler +import org.mockito.Matchers._ +import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.Matchers +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging +import org.apache.spark.status.api.v1.{ApplicationAttemptInfo => AttemptInfo, ApplicationInfo} +import org.apache.spark.ui.SparkUI +import org.apache.spark.util.{Clock, ManualClock, Utils} + +class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar with Matchers { + + /** + * subclass with access to the cache internals + * @param retainedApplications number of retained applications + */ + class TestApplicationCache( + operations: ApplicationCacheOperations = new StubCacheOperations(), + retainedApplications: Int, + clock: Clock = new ManualClock(0)) + extends ApplicationCache(operations, retainedApplications, clock) { + + def cache(): LoadingCache[CacheKey, CacheEntry] = appCache + } + + /** + * Stub cache operations. + * The state is kept in a map of [[CacheKey]] to [[CacheEntry]], + * the `probeTime` field in the cache entry setting the timestamp of the entry + */ + class StubCacheOperations extends ApplicationCacheOperations with Logging { + + /** map to UI instances, including timestamps, which are used in update probes */ + val instances = mutable.HashMap.empty[CacheKey, CacheEntry] + + /** Map of attached spark UIs */ + val attached = mutable.HashMap.empty[CacheKey, SparkUI] + + var getAppUICount = 0L + var attachCount = 0L + var detachCount = 0L + var updateProbeCount = 0L + + override def getAppUI(appId: String, attemptId: Option[String]): Option[LoadedAppUI] = { + logDebug(s"getAppUI($appId, $attemptId)") + getAppUICount += 1 + instances.get(CacheKey(appId, attemptId)).map( e => + LoadedAppUI(e.ui, () => updateProbe(appId, attemptId, e.probeTime))) + } + + override def attachSparkUI( + appId: String, + attemptId: Option[String], + ui: SparkUI, + completed: Boolean): Unit = { + logDebug(s"attachSparkUI($appId, $attemptId, $ui)") + attachCount += 1 + attached += (CacheKey(appId, attemptId) -> ui) + } + + def putAndAttach( + appId: String, + attemptId: Option[String], + completed: Boolean, + started: Long, + ended: Long, + timestamp: Long): SparkUI = { + val ui = putAppUI(appId, attemptId, completed, started, ended, timestamp) + attachSparkUI(appId, attemptId, ui, completed) + ui + } + + def putAppUI( + appId: String, + attemptId: Option[String], + completed: Boolean, + started: Long, + ended: Long, + timestamp: Long): SparkUI = { + val ui = newUI(appId, attemptId, completed, started, ended) + putInstance(appId, attemptId, ui, completed, timestamp) + ui + } + + def putInstance( + appId: String, + attemptId: Option[String], + ui: SparkUI, + completed: Boolean, + timestamp: Long): Unit = { + instances += (CacheKey(appId, attemptId) -> + new CacheEntry(ui, completed, () => updateProbe(appId, attemptId, timestamp), timestamp)) + } + + /** + * Detach a reconstructed UI + * + * @param ui Spark UI + */ + override def detachSparkUI(appId: String, attemptId: Option[String], ui: SparkUI): Unit = { + logDebug(s"detachSparkUI($appId, $attemptId, $ui)") + detachCount += 1 + var name = ui.getAppName + val key = CacheKey(appId, attemptId) + attached.getOrElse(key, { throw new java.util.NoSuchElementException() }) + attached -= key + } + + /** + * Lookup from the internal cache of attached UIs + */ + def getAttached(appId: String, attemptId: Option[String]): Option[SparkUI] = { + attached.get(CacheKey(appId, attemptId)) + } + + /** + * The update probe. + * @param appId application to probe + * @param attemptId attempt to probe + * @param updateTime timestamp of this UI load + */ + private[history] def updateProbe( + appId: String, + attemptId: Option[String], + updateTime: Long)(): Boolean = { + updateProbeCount += 1 + logDebug(s"isUpdated($appId, $attemptId, ${updateTime})") + val entry = instances.get(CacheKey(appId, attemptId)).get + val updated = entry.probeTime > updateTime + logDebug(s"entry = $entry; updated = $updated") + updated + } + } + + /** + * Create a new UI. The info/attempt info classes here are from the package + * `org.apache.spark.status.api.v1`, not the near-equivalents from the history package + */ + def newUI( + name: String, + attemptId: Option[String], + completed: Boolean, + started: Long, + ended: Long): SparkUI = { + val info = new ApplicationInfo(name, name, Some(1), Some(1), Some(1), Some(64), + Seq(new AttemptInfo(attemptId, new Date(started), new Date(ended), + new Date(ended), ended - started, "user", completed, org.apache.spark.SPARK_VERSION))) + val ui = mock[SparkUI] + when(ui.getApplicationInfoList).thenReturn(List(info).iterator) + when(ui.getAppName).thenReturn(name) + when(ui.appName).thenReturn(name) + val handler = new ServletContextHandler() + when(ui.getHandlers).thenReturn(Seq(handler)) + ui + } + + /** + * Test operations on completed UIs: they are loaded on demand, entries + * are removed on overload. + * + * This effectively tests the original behavior of the history server's cache. + */ + test("Completed UI get") { + val operations = new StubCacheOperations() + val clock = new ManualClock(1) + implicit val cache = new ApplicationCache(operations, 2, clock) + val metrics = cache.metrics + // cache misses + val app1 = "app-1" + assertNotFound(app1, None) + assertMetric("lookupCount", metrics.lookupCount, 1) + assertMetric("lookupFailureCount", metrics.lookupFailureCount, 1) + assert(1 === operations.getAppUICount, "getAppUICount") + assertNotFound(app1, None) + assert(2 === operations.getAppUICount, "getAppUICount") + assert(0 === operations.attachCount, "attachCount") + + val now = clock.getTimeMillis() + // add the entry + operations.putAppUI(app1, None, true, now, now, now) + + // make sure its local + operations.getAppUI(app1, None).get + operations.getAppUICount = 0 + // now expect it to be found + val cacheEntry = cache.lookupCacheEntry(app1, None) + assert(1 === cacheEntry.probeTime) + assert(cacheEntry.completed) + // assert about queries made of the operations + assert(1 === operations.getAppUICount, "getAppUICount") + assert(1 === operations.attachCount, "attachCount") + + // and in the map of attached + assert(operations.getAttached(app1, None).isDefined, s"attached entry '1' from $cache") + + // go forward in time + clock.setTime(10) + val time2 = clock.getTimeMillis() + val cacheEntry2 = cache.get(app1) + // no more refresh as this is a completed app + assert(1 === operations.getAppUICount, "getAppUICount") + assert(0 === operations.updateProbeCount, "updateProbeCount") + assert(0 === operations.detachCount, "attachCount") + + // evict the entry + operations.putAndAttach("2", None, true, time2, time2, time2) + operations.putAndAttach("3", None, true, time2, time2, time2) + cache.get("2") + cache.get("3") + + // there should have been a detachment here + assert(1 === operations.detachCount, s"detach count from $cache") + // and entry app1 no longer attached + assert(operations.getAttached(app1, None).isEmpty, s"get($app1) in $cache") + val appId = "app1" + val attemptId = Some("_01") + val time3 = clock.getTimeMillis() + operations.putAppUI(appId, attemptId, false, time3, 0, time3) + // expect an error here + assertNotFound(appId, None) + } + + test("Test that if an attempt ID is set, it must be used in lookups") { + val operations = new StubCacheOperations() + val clock = new ManualClock(1) + implicit val cache = new ApplicationCache(operations, retainedApplications = 10, clock = clock) + val appId = "app1" + val attemptId = Some("_01") + operations.putAppUI(appId, attemptId, false, clock.getTimeMillis(), 0, 0) + assertNotFound(appId, None) + } + + /** + * Test that incomplete apps are not probed for updates during the time window, + * but that they are checked if that window has expired and they are not completed. + * Then, if they have changed, the old entry is replaced by a new one. + */ + test("Incomplete apps refreshed") { + val operations = new StubCacheOperations() + val clock = new ManualClock(50) + val window = 500 + implicit val cache = new ApplicationCache(operations, retainedApplications = 5, clock = clock) + val metrics = cache.metrics + // add the incomplete app + // add the entry + val started = clock.getTimeMillis() + val appId = "app1" + val attemptId = Some("001") + operations.putAppUI(appId, attemptId, false, started, 0, started) + val firstEntry = cache.lookupCacheEntry(appId, attemptId) + assert(started === firstEntry.probeTime, s"timestamp in $firstEntry") + assert(!firstEntry.completed, s"entry is complete: $firstEntry") + assertMetric("lookupCount", metrics.lookupCount, 1) + + assert(0 === operations.updateProbeCount, "expected no update probe on that first get") + + val checkTime = window * 2 + clock.setTime(checkTime) + val entry3 = cache.lookupCacheEntry(appId, attemptId) + assert(firstEntry !== entry3, s"updated entry test from $cache") + assertMetric("lookupCount", metrics.lookupCount, 2) + assertMetric("updateProbeCount", metrics.updateProbeCount, 1) + assertMetric("updateTriggeredCount", metrics.updateTriggeredCount, 0) + assert(1 === operations.updateProbeCount, s"refresh count in $cache") + assert(0 === operations.detachCount, s"detach count") + assert(entry3.probeTime === checkTime) + + val updateTime = window * 3 + // update the cached value + val updatedApp = operations.putAppUI(appId, attemptId, true, started, updateTime, updateTime) + val endTime = window * 10 + clock.setTime(endTime) + logDebug(s"Before operation = $cache") + val entry5 = cache.lookupCacheEntry(appId, attemptId) + assertMetric("lookupCount", metrics.lookupCount, 3) + assertMetric("updateProbeCount", metrics.updateProbeCount, 2) + // the update was triggered + assertMetric("updateTriggeredCount", metrics.updateTriggeredCount, 1) + assert(updatedApp === entry5.ui, s"UI {$updatedApp} did not match entry {$entry5} in $cache") + + // at which point, the refreshes stop + clock.setTime(window * 20) + assertCacheEntryEquals(appId, attemptId, entry5) + assertMetric("updateProbeCount", metrics.updateProbeCount, 2) + } + + /** + * Assert that a metric counter has a specific value; failure raises an exception + * including the cache's toString value + * @param name counter name (for exceptions) + * @param counter counter + * @param expected expected value. + * @param cache cache + */ + def assertMetric( + name: String, + counter: Counter, + expected: Long) + (implicit cache: ApplicationCache): Unit = { + val actual = counter.getCount + if (actual != expected) { + // this is here because Scalatest loses stack depth + throw new Exception(s"Wrong $name value - expected $expected but got $actual in $cache") + } + } + + /** + * Look up the cache entry and assert that it matches in the expected value. + * This assertion works if the two CacheEntries are different -it looks at the fields. + * UI are compared on object equality; the timestamp and completed flags directly. + * @param appId application ID + * @param attemptId attempt ID + * @param expected expected value + * @param cache app cache + */ + def assertCacheEntryEquals( + appId: String, + attemptId: Option[String], + expected: CacheEntry) + (implicit cache: ApplicationCache): Unit = { + val actual = cache.lookupCacheEntry(appId, attemptId) + val errorText = s"Expected get($appId, $attemptId) -> $expected, but got $actual from $cache" + assert(expected.ui === actual.ui, errorText + " SparkUI reference") + assert(expected.completed === actual.completed, errorText + " -completed flag") + assert(expected.probeTime === actual.probeTime, errorText + " -timestamp") + } + + /** + * Assert that a key wasn't found in cache or loaded. + * + * Looks for the specific nested exception raised by [[ApplicationCache]] + * @param appId application ID + * @param attemptId attempt ID + * @param cache app cache + */ + def assertNotFound( + appId: String, + attemptId: Option[String]) + (implicit cache: ApplicationCache): Unit = { + val ex = intercept[UncheckedExecutionException] { + cache.get(appId, attemptId) + } + var cause = ex.getCause + assert(cause !== null) + if (!cause.isInstanceOf[NoSuchElementException]) { + throw cause + } + } + + test("Large Scale Application Eviction") { + val operations = new StubCacheOperations() + val clock = new ManualClock(0) + val size = 5 + // only two entries are retained, so we expect evictions to occur on lookups + implicit val cache: ApplicationCache = new TestApplicationCache(operations, + retainedApplications = size, clock = clock) + + val attempt1 = Some("01") + + val ids = new ListBuffer[String]() + // build a list of applications + val count = 100 + for (i <- 1 to count ) { + val appId = f"app-$i%04d" + ids += appId + clock.advance(10) + val t = clock.getTimeMillis() + operations.putAppUI(appId, attempt1, true, t, t, t) + } + // now go through them in sequence reading them, expect evictions + ids.foreach { id => + cache.get(id, attempt1) + } + logInfo(cache.toString) + val metrics = cache.metrics + + assertMetric("loadCount", metrics.loadCount, count) + assertMetric("evictionCount", metrics.evictionCount, count - size) +} + + test("Attempts are Evicted") { + val operations = new StubCacheOperations() + implicit val cache: ApplicationCache = new TestApplicationCache(operations, + retainedApplications = 4) + val metrics = cache.metrics + val appId = "app1" + val attempt1 = Some("01") + val attempt2 = Some("02") + val attempt3 = Some("03") + operations.putAppUI(appId, attempt1, true, 100, 110, 110) + operations.putAppUI(appId, attempt2, true, 200, 210, 210) + operations.putAppUI(appId, attempt3, true, 300, 310, 310) + val attempt4 = Some("04") + operations.putAppUI(appId, attempt4, true, 400, 410, 410) + val attempt5 = Some("05") + operations.putAppUI(appId, attempt5, true, 500, 510, 510) + + def expectLoadAndEvictionCounts(expectedLoad: Int, expectedEvictionCount: Int): Unit = { + assertMetric("loadCount", metrics.loadCount, expectedLoad) + assertMetric("evictionCount", metrics.evictionCount, expectedEvictionCount) + } + + // first entry + cache.get(appId, attempt1) + expectLoadAndEvictionCounts(1, 0) + + // second + cache.get(appId, attempt2) + expectLoadAndEvictionCounts(2, 0) + + // no change + cache.get(appId, attempt2) + expectLoadAndEvictionCounts(2, 0) + + // eviction time + cache.get(appId, attempt3) + cache.size() should be(3) + cache.get(appId, attempt4) + expectLoadAndEvictionCounts(4, 0) + cache.get(appId, attempt5) + expectLoadAndEvictionCounts(5, 1) + cache.get(appId, attempt5) + expectLoadAndEvictionCounts(5, 1) + + } + + test("Instantiate Filter") { + // this is a regression test on the filter being constructable + val clazz = Utils.classForName(ApplicationCacheCheckFilterRelay.FILTER_NAME) + val instance = clazz.newInstance() + instance shouldBe a [Filter] + } + + test("redirect includes query params") { + val clazz = Utils.classForName(ApplicationCacheCheckFilterRelay.FILTER_NAME) + val filter = clazz.newInstance().asInstanceOf[ApplicationCacheCheckFilter] + filter.appId = "local-123" + val cache = mock[ApplicationCache] + when(cache.checkForUpdates(any(), any())).thenReturn(true) + ApplicationCacheCheckFilterRelay.setApplicationCache(cache) + val request = mock[HttpServletRequest] + when(request.getMethod()).thenReturn("GET") + when(request.getRequestURI()).thenReturn("http://localhost:18080/history/local-123/jobs/job/") + when(request.getQueryString()).thenReturn("id=2") + val resp = mock[HttpServletResponse] + when(resp.encodeRedirectURL(any())).thenAnswer(new Answer[String]() { + override def answer(invocationOnMock: InvocationOnMock): String = { + invocationOnMock.getArguments()(0).asInstanceOf[String] + } + }) + filter.doFilter(request, resp, null) + verify(resp).sendRedirect("http://localhost:18080/history/local-123/jobs/job/?id=2") + } + +} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 5cab17f8a38f..7109146ece37 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -17,40 +17,37 @@ package org.apache.spark.deploy.history -import java.io.{BufferedOutputStream, ByteArrayInputStream, ByteArrayOutputStream, File, - FileOutputStream, OutputStreamWriter} -import java.net.URI +import java.io._ +import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit import java.util.zip.{ZipInputStream, ZipOutputStream} -import scala.io.Source import scala.concurrent.duration._ import scala.language.postfixOps -import com.google.common.base.Charsets import com.google.common.io.{ByteStreams, Files} -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.hdfs.DistributedFileSystem import org.json4s.jackson.JsonMethods._ import org.mockito.Matchers.any -import org.mockito.Mockito.{doReturn, mock, spy, verify, when} +import org.mockito.Mockito.{mock, spy, verify} import org.scalatest.BeforeAndAfter import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ -import org.apache.spark.{Logging, SparkConf, SparkFunSuite} +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.io._ import org.apache.spark.scheduler._ +import org.apache.spark.security.GroupMappingServiceProvider import org.apache.spark.util.{Clock, JsonProtocol, ManualClock, Utils} class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matchers with Logging { - import FsHistoryProvider._ - private var testDir: File = null before { - testDir = Utils.createTempDir() + testDir = Utils.createTempDir(namePrefix = s"a b%20c+d") } after { @@ -65,12 +62,13 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc codec: Option[String] = None): File = { val ip = if (inProgress) EventLoggingListener.IN_PROGRESS else "" val logUri = EventLoggingListener.getLogPath(testDir.toURI, appId, appAttemptId) - val logPath = new URI(logUri).getPath + ip + val logPath = new Path(logUri).toUri.getPath + ip new File(logPath) } - test("Parse new and old application logs") { - val provider = new FsHistoryProvider(createTestConf()) + test("Parse application logs") { + val clock = new ManualClock(12345678) + val provider = new FsHistoryProvider(createTestConf(), clock) // Write a new-style application log. val newAppComplete = newLogFile("new1", None, inProgress = false) @@ -95,26 +93,11 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc None) ) - // Write an old-style application log. - val oldAppComplete = writeOldLog("old1", "1.0", None, true, - SparkListenerApplicationStart("old1", Some("old-app-complete"), 2L, "test", None), - SparkListenerApplicationEnd(3L) - ) - - // Check for logs so that we force the older unfinished app to be loaded, to make - // sure unfinished apps are also sorted correctly. - provider.checkForLogs() - - // Write an unfinished app, old-style. - val oldAppIncomplete = writeOldLog("old2", "1.0", None, false, - SparkListenerApplicationStart("old2", None, 2L, "test", None) - ) - - // Force a reload of data from the log directory, and check that both logs are loaded. + // Force a reload of data from the log directory, and check that logs are loaded. // Take the opportunity to check that the offset checks work as expected. updateAndCheck(provider) { list => - list.size should be (5) - list.count(_.attempts.head.completed) should be (3) + list.size should be (3) + list.count(_.attempts.head.completed) should be (2) def makeAppInfo( id: String, @@ -125,19 +108,18 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc user: String, completed: Boolean): ApplicationHistoryInfo = { ApplicationHistoryInfo(id, name, - List(ApplicationAttemptInfo(None, start, end, lastMod, user, completed))) + List(ApplicationAttemptInfo(None, start, end, lastMod, user, completed, ""))) } + // For completed files, lastUpdated would be lastModified time. list(0) should be (makeAppInfo("new-app-complete", newAppComplete.getName(), 1L, 5L, newAppComplete.lastModified(), "test", true)) list(1) should be (makeAppInfo("new-complete-lzf", newAppCompressedComplete.getName(), 1L, 4L, newAppCompressedComplete.lastModified(), "test", true)) - list(2) should be (makeAppInfo("old-app-complete", oldAppComplete.getName(), 2L, 3L, - oldAppComplete.lastModified(), "test", true)) - list(3) should be (makeAppInfo(oldAppIncomplete.getName(), oldAppIncomplete.getName(), 2L, - -1L, oldAppIncomplete.lastModified(), "test", false)) - list(4) should be (makeAppInfo("new-incomplete", newAppIncomplete.getName(), 1L, -1L, - newAppIncomplete.lastModified(), "test", false)) + + // For Inprogress files, lastUpdated would be current loading time. + list(2) should be (makeAppInfo("new-incomplete", newAppIncomplete.getName(), 1L, -1L, + clock.getTimeMillis(), "test", false)) // Make sure the UI can be rendered. list.foreach { case info => @@ -148,39 +130,19 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } } - test("Parse legacy logs with compression codec set") { - val provider = new FsHistoryProvider(createTestConf()) - val testCodecs = List((classOf[LZFCompressionCodec].getName(), true), - (classOf[SnappyCompressionCodec].getName(), true), - ("invalid.codec", false)) - - testCodecs.foreach { case (codecName, valid) => - val codec = if (valid) CompressionCodec.createCodec(new SparkConf(), codecName) else null - val logDir = new File(testDir, codecName) - logDir.mkdir() - createEmptyFile(new File(logDir, SPARK_VERSION_PREFIX + "1.0")) - writeFile(new File(logDir, LOG_PREFIX + "1"), false, Option(codec), - SparkListenerApplicationStart("app2", None, 2L, "test", None), - SparkListenerApplicationEnd(3L) - ) - createEmptyFile(new File(logDir, COMPRESSION_CODEC_PREFIX + codecName)) - - val logPath = new Path(logDir.getAbsolutePath()) - try { - val logInput = provider.openLegacyEventLog(logPath) - try { - Source.fromInputStream(logInput).getLines().toSeq.size should be (2) - } finally { - logInput.close() - } - } catch { - case e: IllegalArgumentException => - valid should be (false) + test("SPARK-3697: ignore files that cannot be read.") { + // setReadable(...) does not work on Windows. Please refer JDK-6728842. + assume(!Utils.isWindows) + + class TestFsHistoryProvider extends FsHistoryProvider(createTestConf()) { + var mergeApplicationListingCall = 0 + override protected def mergeApplicationListing(fileStatus: FileStatus): Unit = { + super.mergeApplicationListing(fileStatus) + mergeApplicationListingCall += 1 } } - } + val provider = new TestFsHistoryProvider - test("SPARK-3697: ignore directories that cannot be read.") { val logFile1 = newLogFile("new1", None, inProgress = false) writeFile(logFile1, true, None, SparkListenerApplicationStart("app1-1", Some("app1-1"), 1L, "test", None), @@ -193,10 +155,11 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc ) logFile2.setReadable(false, false) - val provider = new FsHistoryProvider(createTestConf()) updateAndCheck(provider) { list => list.size should be (1) } + + provider.mergeApplicationListingCall should be (1) } test("history file is renamed from inprogress to completed") { @@ -352,6 +315,48 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc assert(!log2.exists()) } + test("log cleaner for inProgress files") { + val firstFileModifiedTime = TimeUnit.SECONDS.toMillis(10) + val secondFileModifiedTime = TimeUnit.SECONDS.toMillis(20) + val maxAge = TimeUnit.SECONDS.toMillis(40) + val clock = new ManualClock(0) + val provider = new FsHistoryProvider( + createTestConf().set("spark.history.fs.cleaner.maxAge", s"${maxAge}ms"), clock) + + val log1 = newLogFile("inProgressApp1", None, inProgress = true) + writeFile(log1, true, None, + SparkListenerApplicationStart( + "inProgressApp1", Some("inProgressApp1"), 3L, "test", Some("attempt1")) + ) + + clock.setTime(firstFileModifiedTime) + provider.checkForLogs() + + val log2 = newLogFile("inProgressApp2", None, inProgress = true) + writeFile(log2, true, None, + SparkListenerApplicationStart( + "inProgressApp2", Some("inProgressApp2"), 23L, "test2", Some("attempt2")) + ) + + clock.setTime(secondFileModifiedTime) + provider.checkForLogs() + + // This should not trigger any cleanup + updateAndCheck(provider)(list => list.size should be(2)) + + // Should trigger cleanup for first file but not second one + clock.setTime(firstFileModifiedTime + maxAge + 1) + updateAndCheck(provider)(list => list.size should be(1)) + assert(!log1.exists()) + assert(log2.exists()) + + // Should cleanup the second file as well. + clock.setTime(secondFileModifiedTime + maxAge + 1) + updateAndCheck(provider)(list => list.size should be(0)) + assert(!log1.exists()) + assert(!log2.exists()) + } + test("Event log copy") { val provider = new FsHistoryProvider(createTestConf()) val logs = (1 to 2).map { i => @@ -375,8 +380,9 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc var entry = inputStream.getNextEntry entry should not be null while (entry != null) { - val actual = new String(ByteStreams.toByteArray(inputStream), Charsets.UTF_8) - val expected = Files.toString(logs.find(_.getName == entry.getName).get, Charsets.UTF_8) + val actual = new String(ByteStreams.toByteArray(inputStream), StandardCharsets.UTF_8) + val expected = + Files.toString(logs.find(_.getName == entry.getName).get, StandardCharsets.UTF_8) actual should be (expected) totalEntries += 1 entry = inputStream.getNextEntry @@ -395,21 +401,8 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc SparkListenerLogStart("1.4") ) - // Write a 1.2 log file with no start event (= no app id), it should be ignored. - writeOldLog("v12Log", "1.2", None, false) - - // Write 1.0 and 1.1 logs, which don't have app ids. - writeOldLog("v11Log", "1.1", None, true, - SparkListenerApplicationStart("v11Log", None, 2L, "test", None), - SparkListenerApplicationEnd(3L)) - writeOldLog("v10Log", "1.0", None, true, - SparkListenerApplicationStart("v10Log", None, 2L, "test", None), - SparkListenerApplicationEnd(4L)) - updateAndCheck(provider) { list => - list.size should be (2) - list(0).id should be ("v10Log") - list(1).id should be ("v11Log") + list.size should be (0) } } @@ -460,6 +453,135 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } } + test("ignore hidden files") { + + // FsHistoryProvider should ignore hidden files. (It even writes out a hidden file itself + // that should be ignored). + + // write out one totally bogus hidden file + val hiddenGarbageFile = new File(testDir, ".garbage") + val out = new PrintWriter(hiddenGarbageFile) + // scalastyle:off println + out.println("GARBAGE") + // scalastyle:on println + out.close() + + // also write out one real event log file, but since its a hidden file, we shouldn't read it + val tmpNewAppFile = newLogFile("hidden", None, inProgress = false) + val hiddenNewAppFile = new File(tmpNewAppFile.getParentFile, "." + tmpNewAppFile.getName) + tmpNewAppFile.renameTo(hiddenNewAppFile) + + // and write one real file, which should still get picked up just fine + val newAppComplete = newLogFile("real-app", None, inProgress = false) + writeFile(newAppComplete, true, None, + SparkListenerApplicationStart(newAppComplete.getName(), Some("new-app-complete"), 1L, "test", + None), + SparkListenerApplicationEnd(5L) + ) + + val provider = new FsHistoryProvider(createTestConf()) + updateAndCheck(provider) { list => + list.size should be (1) + list(0).name should be ("real-app") + } + } + + test("support history server ui admin acls") { + def createAndCheck(conf: SparkConf, properties: (String, String)*) + (checkFn: SecurityManager => Unit): Unit = { + // Empty the testDir for each test. + if (testDir.exists() && testDir.isDirectory) { + testDir.listFiles().foreach { f => if (f.isFile) f.delete() } + } + + var provider: FsHistoryProvider = null + try { + provider = new FsHistoryProvider(conf) + val log = newLogFile("app1", Some("attempt1"), inProgress = false) + writeFile(log, true, None, + SparkListenerApplicationStart("app1", Some("app1"), System.currentTimeMillis(), + "test", Some("attempt1")), + SparkListenerEnvironmentUpdate(Map( + "Spark Properties" -> properties.toSeq, + "JVM Information" -> Seq.empty, + "System Properties" -> Seq.empty, + "Classpath Entries" -> Seq.empty + )), + SparkListenerApplicationEnd(System.currentTimeMillis())) + + provider.checkForLogs() + val appUi = provider.getAppUI("app1", Some("attempt1")) + + assert(appUi.nonEmpty) + val securityManager = appUi.get.ui.securityManager + checkFn(securityManager) + } finally { + if (provider != null) { + provider.stop() + } + } + } + + // Test both history ui admin acls and application acls are configured. + val conf1 = createTestConf() + .set("spark.history.ui.acls.enable", "true") + .set("spark.history.ui.admin.acls", "user1,user2") + .set("spark.history.ui.admin.acls.groups", "group1") + .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) + + createAndCheck(conf1, ("spark.admin.acls", "user"), ("spark.admin.acls.groups", "group")) { + securityManager => + // Test whether user has permission to access UI. + securityManager.checkUIViewPermissions("user1") should be (true) + securityManager.checkUIViewPermissions("user2") should be (true) + securityManager.checkUIViewPermissions("user") should be (true) + securityManager.checkUIViewPermissions("abc") should be (false) + + // Test whether user with admin group has permission to access UI. + securityManager.checkUIViewPermissions("user3") should be (true) + securityManager.checkUIViewPermissions("user4") should be (true) + securityManager.checkUIViewPermissions("user5") should be (true) + securityManager.checkUIViewPermissions("user6") should be (false) + } + + // Test only history ui admin acls are configured. + val conf2 = createTestConf() + .set("spark.history.ui.acls.enable", "true") + .set("spark.history.ui.admin.acls", "user1,user2") + .set("spark.history.ui.admin.acls.groups", "group1") + .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) + createAndCheck(conf2) { securityManager => + // Test whether user has permission to access UI. + securityManager.checkUIViewPermissions("user1") should be (true) + securityManager.checkUIViewPermissions("user2") should be (true) + // Check the unknown "user" should return false + securityManager.checkUIViewPermissions("user") should be (false) + + // Test whether user with admin group has permission to access UI. + securityManager.checkUIViewPermissions("user3") should be (true) + securityManager.checkUIViewPermissions("user4") should be (true) + // Check the "user5" without mapping relation should return false + securityManager.checkUIViewPermissions("user5") should be (false) + } + + // Test neither history ui admin acls nor application acls are configured. + val conf3 = createTestConf() + .set("spark.history.ui.acls.enable", "true") + .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) + createAndCheck(conf3) { securityManager => + // Test whether user has permission to access UI. + securityManager.checkUIViewPermissions("user1") should be (false) + securityManager.checkUIViewPermissions("user2") should be (false) + securityManager.checkUIViewPermissions("user") should be (false) + + // Test whether user with admin group has permission to access UI. + // Check should be failed since we don't have acl group settings. + securityManager.checkUIViewPermissions("user3") should be (false) + securityManager.checkUIViewPermissions("user4") should be (false) + securityManager.checkUIViewPermissions("user5") should be (false) + } + } + /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: @@ -481,9 +603,15 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc val cstream = codec.map(_.compressedOutputStream(fstream)).getOrElse(fstream) val bstream = new BufferedOutputStream(cstream) if (isNewFormat) { - EventLoggingListener.initEventLog(new FileOutputStream(file)) + val newFormatStream = new FileOutputStream(file) + Utils.tryWithSafeFinally { + EventLoggingListener.initEventLog(newFormatStream, false, null) + } { + newFormatStream.close() + } } - val writer = new OutputStreamWriter(bstream, "UTF-8") + + val writer = new OutputStreamWriter(bstream, StandardCharsets.UTF_8) Utils.tryWithSafeFinally { events.foreach(e => writer.write(compact(render(JsonProtocol.sparkEventToJson(e))) + "\n")) } { @@ -499,25 +627,6 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc new SparkConf().set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) } - private def writeOldLog( - fname: String, - sparkVersion: String, - codec: Option[CompressionCodec], - completed: Boolean, - events: SparkListenerEvent*): File = { - val log = new File(testDir, fname) - log.mkdir() - - val oldEventLog = new File(log, LOG_PREFIX + "1") - createEmptyFile(new File(log, SPARK_VERSION_PREFIX + sparkVersion)) - writeFile(new File(log, LOG_PREFIX + "1"), false, codec, events: _*) - if (completed) { - createEmptyFile(new File(log, APPLICATION_COMPLETE)) - } - - log - } - private class SafeModeTestProvider(conf: SparkConf, clock: Clock) extends FsHistoryProvider(conf, clock) { @@ -531,3 +640,15 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } } + +class TestGroupsMappingProvider extends GroupMappingServiceProvider { + private val mappings = Map( + "user3" -> "group1", + "user4" -> "group1", + "user5" -> "group") + + override def getGroups(username: String): Set[String] = { + mappings.get(username).map(Set(_)).getOrElse(Set.empty) + } +} + diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala index 34f27ecaa07a..de321db845a6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala @@ -33,7 +33,7 @@ class HistoryServerArgumentsSuite extends SparkFunSuite { .set("spark.testing", "true") test("No Arguments Parsing") { - val argStrings = Array[String]() + val argStrings = Array.empty[String] val hsa = new HistoryServerArguments(conf, argStrings) assert(conf.get("spark.history.fs.logDirectory") === logDir.getAbsolutePath) assert(conf.get("spark.history.fs.updateInterval") === "1") diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 4b7fd4f13b69..18da8c18939e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -18,23 +18,39 @@ package org.apache.spark.deploy.history import java.io.{File, FileInputStream, FileWriter, InputStream, IOException} import java.net.{HttpURLConnection, URL} +import java.nio.charset.StandardCharsets import java.util.zip.ZipInputStream -import javax.servlet.http.{HttpServletRequest, HttpServletResponse} +import javax.servlet._ +import javax.servlet.http.{HttpServletRequest, HttpServletRequestWrapper, HttpServletResponse} -import com.google.common.base.Charsets +import scala.concurrent.duration._ +import scala.language.postfixOps + +import com.codahale.metrics.Counter import com.google.common.io.{ByteStreams, Files} import org.apache.commons.io.{FileUtils, IOUtils} -import org.mockito.Mockito.when +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.eclipse.jetty.proxy.ProxyServlet +import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} +import org.json4s.JsonAST._ +import org.json4s.jackson.JsonMethods +import org.json4s.jackson.JsonMethods._ +import org.openqa.selenium.WebDriver +import org.openqa.selenium.htmlunit.HtmlUnitDriver import org.scalatest.{BeforeAndAfter, Matchers} -import org.scalatest.mock.MockitoSugar +import org.scalatest.concurrent.Eventually +import org.scalatest.mockito.MockitoSugar +import org.scalatest.selenium.WebBrowser -import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.ui.{SparkUI, UIUtils} +import org.apache.spark._ +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.jobs.UIData.JobUIData +import org.apache.spark.util.{ResetSystemProperties, Utils} /** * A collection of tests against the historyserver, including comparing responses from the json * metrics api to a set of known "golden files". If new endpoints / parameters are added, - * cases should be added to this test suite. The expected outcomes can be genered by running + * cases should be added to this test suite. The expected outcomes can be generated by running * the HistoryServerSuite.main. Note that this will blindly generate new expectation files matching * the current behavior -- the developer must verify that behavior is correct. * @@ -43,23 +59,25 @@ import org.apache.spark.ui.{SparkUI, UIUtils} * are considered part of Spark's public api. */ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers with MockitoSugar - with JsonTestUtils { + with JsonTestUtils with Eventually with WebBrowser with LocalSparkContext + with ResetSystemProperties { - private val logDir = new File("src/test/resources/spark-events") - private val expRoot = new File("src/test/resources/HistoryServerExpectations/") + private val logDir = getTestResourcePath("spark-events") + private val expRoot = getTestResourceFile("HistoryServerExpectations") private var provider: FsHistoryProvider = null private var server: HistoryServer = null private var port: Int = -1 - def init(): Unit = { + def init(extraConf: (String, String)*): Unit = { val conf = new SparkConf() - .set("spark.history.fs.logDirectory", logDir.getAbsolutePath) - .set("spark.history.fs.updateInterval", "0") + .set("spark.history.fs.logDirectory", logDir) + .set("spark.history.fs.update.interval", "0") .set("spark.testing", "true") + conf.setAll(extraConf) provider = new FsHistoryProvider(conf) provider.checkForLogs() - val securityManager = new SecurityManager(conf) + val securityManager = HistoryServer.createSecurityManager(conf) server = new HistoryServer(conf, provider, securityManager, 18080) server.initialize() @@ -86,6 +104,13 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "minDate app list json" -> "applications?minDate=2015-02-10", "maxDate app list json" -> "applications?maxDate=2015-02-10", "maxDate2 app list json" -> "applications?maxDate=2015-02-03T16:42:40.000GMT", + "minEndDate app list json" -> "applications?minEndDate=2015-05-06T13:03:00.950GMT", + "maxEndDate app list json" -> "applications?maxEndDate=2015-05-06T13:03:00.950GMT", + "minEndDate and maxEndDate app list json" -> + "applications?minEndDate=2015-03-16&maxEndDate=2015-05-06T13:03:00.950GMT", + "minDate and maxEndDate app list json" -> + "applications?minDate=2015-03-16&maxEndDate=2015-05-06T13:03:00.950GMT", + "limit app list json" -> "applications?limit=3", "one app json" -> "applications/local-1422981780767", "one app multi-attempt json" -> "applications/local-1426533911241", "job list json" -> "applications/local-1422981780767/jobs", @@ -127,7 +152,11 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "applications/local-1426533911241/2/stages/0/0/taskList", "rdd list storage json" -> "applications/local-1422981780767/storage/rdd", - "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0" + "executor node blacklisting" -> "applications/app-20161116163331-0000/executors", + "executor node blacklisting unblacklisting" -> "applications/app-20161115172038-0000/executors", + "executor memory usage" -> "applications/app-20161116163331-0000/executors" + // Todo: enable this test when logging the even of onBlockUpdated. See: SPARK-13845 + // "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0" ) // run a bunch of characterization tests -- just verify the behavior is the same as what is saved @@ -138,18 +167,39 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers code should be (HttpServletResponse.SC_OK) jsonOpt should be ('defined) errOpt should be (None) - val json = jsonOpt.get + val exp = IOUtils.toString(new FileInputStream( new File(expRoot, HistoryServerSuite.sanitizePath(name) + "_expectation.json"))) // compare the ASTs so formatting differences don't cause failures import org.json4s._ import org.json4s.jackson.JsonMethods._ - val jsonAst = parse(json) + val jsonAst = parse(clearLastUpdated(jsonOpt.get)) val expAst = parse(exp) assertValidDataInJson(jsonAst, expAst) } } + // SPARK-10873 added the lastUpdated field for each application's attempt, + // the REST API returns the last modified time of EVENT LOG file for this field. + // It is not applicable to hard-code this dynamic field in a static expected file, + // so here we skip checking the lastUpdated field's value (setting it as ""). + private def clearLastUpdated(json: String): String = { + if (json.indexOf("lastUpdated") >= 0) { + val subStrings = json.split(",") + for (i <- subStrings.indices) { + if (subStrings(i).indexOf("lastUpdatedEpoch") >= 0) { + subStrings(i) = subStrings(i).replaceAll("(\\d+)", "0") + } else if (subStrings(i).indexOf("lastUpdated") >= 0) { + val regex = "\"lastUpdated\"\\s*:\\s*\".*\"".r + subStrings(i) = regex.replaceAllIn(subStrings(i), "\"lastUpdated\" : \"\"") + } + } + subStrings.mkString(",") + } else { + json + } + } + test("download all logs for app with multiple attempts") { doDownloadTest("local-1430917381535", None) } @@ -158,18 +208,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers (1 to 2).foreach { attemptId => doDownloadTest("local-1430917381535", Some(attemptId)) } } - test("download legacy logs - all attempts") { - doDownloadTest("local-1426533911241", None, legacy = true) - } - - test("download legacy logs - single attempts") { - (1 to 2). foreach { - attemptId => doDownloadTest("local-1426533911241", Some(attemptId), legacy = true) - } - } - // Test that the files are downloaded correctly, and validate them. - def doDownloadTest(appId: String, attemptId: Option[Int], legacy: Boolean = false): Unit = { + def doDownloadTest(appId: String, attemptId: Option[Int]): Unit = { val url = attemptId match { case Some(id) => @@ -187,25 +227,16 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers var entry = zipStream.getNextEntry entry should not be null val totalFiles = { - if (legacy) { - attemptId.map { x => 3 }.getOrElse(6) - } else { - attemptId.map { x => 1 }.getOrElse(2) - } + attemptId.map { x => 1 }.getOrElse(2) } var filesCompared = 0 while (entry != null) { if (!entry.isDirectory) { val expectedFile = { - if (legacy) { - val splits = entry.getName.split("/") - new File(new File(logDir, splits(0)), splits(1)) - } else { - new File(logDir, entry.getName) - } + new File(logDir, entry.getName) } - val expected = Files.toString(expectedFile, Charsets.UTF_8) - val actual = new String(ByteStreams.toByteArray(zipStream), Charsets.UTF_8) + val expected = Files.toString(expectedFile, StandardCharsets.UTF_8) + val actual = new String(ByteStreams.toByteArray(zipStream), StandardCharsets.UTF_8) actual should be (expected) filesCompared += 1 } @@ -240,32 +271,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers getContentAndCode("foobar")._1 should be (HttpServletResponse.SC_NOT_FOUND) } - test("generate history page with relative links") { - val historyServer = mock[HistoryServer] - val request = mock[HttpServletRequest] - val ui = mock[SparkUI] - val link = "/history/app1" - val info = new ApplicationHistoryInfo("app1", "app1", - List(ApplicationAttemptInfo(None, 0, 2, 1, "xxx", true))) - when(historyServer.getApplicationList()).thenReturn(Seq(info)) - when(ui.basePath).thenReturn(link) - when(historyServer.getProviderConfig()).thenReturn(Map[String, String]()) - val page = new HistoryPage(historyServer) - - // when - val response = page.render(request) - - // then - val links = response \\ "a" - val justHrefs = for { - l <- links - attrs <- l.attribute("href") - } yield (attrs.toString) - justHrefs should contain (UIUtils.prependBaseUri(resource = link)) - } - - test("relative links are prefixed with uiRoot (spark.ui.proxyBase)") { - val proxyBaseBeforeTest = System.getProperty("spark.ui.proxyBase") + test("static relative links are prefixed with uiRoot (spark.ui.proxyBase)") { val uiRoot = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")).getOrElse("/testwebproxybase") val page = new HistoryPage(server) val request = mock[HttpServletRequest] @@ -273,7 +279,6 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers // when System.setProperty("spark.ui.proxyBase", uiRoot) val response = page.render(request) - System.setProperty("spark.ui.proxyBase", Option(proxyBaseBeforeTest).getOrElse("")) // then val urls = response \\ "@href" map (_.toString) @@ -281,6 +286,324 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers all (siteRelativeLinks) should startWith (uiRoot) } + test("ajax rendered relative links are prefixed with uiRoot (spark.ui.proxyBase)") { + val uiRoot = "/testwebproxybase" + System.setProperty("spark.ui.proxyBase", uiRoot) + + server.stop() + + val conf = new SparkConf() + .set("spark.history.fs.logDirectory", logDir) + .set("spark.history.fs.update.interval", "0") + .set("spark.testing", "true") + + provider = new FsHistoryProvider(conf) + provider.checkForLogs() + val securityManager = HistoryServer.createSecurityManager(conf) + + server = new HistoryServer(conf, provider, securityManager, 18080) + server.initialize() + server.bind() + + val port = server.boundPort + + val servlet = new ProxyServlet { + override def rewriteTarget(request: HttpServletRequest): String = { + // servlet acts like a proxy that redirects calls made on + // spark.ui.proxyBase context path to the normal servlet handlers operating off "/" + val sb = request.getRequestURL() + + if (request.getQueryString() != null) { + sb.append(s"?${request.getQueryString()}") + } + + val proxyidx = sb.indexOf(uiRoot) + sb.delete(proxyidx, proxyidx + uiRoot.length).toString + } + } + + val contextHandler = new ServletContextHandler + val holder = new ServletHolder(servlet) + contextHandler.setContextPath(uiRoot) + contextHandler.addServlet(holder, "/") + server.attachHandler(contextHandler) + + implicit val webDriver: WebDriver = new HtmlUnitDriver(true) { + getWebClient.getOptions.setThrowExceptionOnScriptError(false) + } + + try { + val url = s"http://localhost:$port" + + go to s"$url$uiRoot" + + // expect the ajax call to finish in 5 seconds + implicitlyWait(org.scalatest.time.Span(5, org.scalatest.time.Seconds)) + + // once this findAll call returns, we know the ajax load of the table completed + findAll(ClassNameQuery("odd")) + + val links = findAll(TagNameQuery("a")) + .map(_.attribute("href")) + .filter(_.isDefined) + .map(_.get) + .filter(_.startsWith(url)).toList + + // there are atleast some URL links that were generated via javascript, + // and they all contain the spark.ui.proxyBase (uiRoot) + links.length should be > 4 + all(links) should startWith(url + uiRoot) + } finally { + contextHandler.stop() + quit() + } + + } + + /** + * Verify that the security manager needed for the history server can be instantiated + * when `spark.authenticate` is `true`, rather than raise an `IllegalArgumentException`. + */ + test("security manager starts with spark.authenticate set") { + val conf = new SparkConf() + .set("spark.testing", "true") + .set(SecurityManager.SPARK_AUTH_CONF, "true") + HistoryServer.createSecurityManager(conf) + } + + test("incomplete apps get refreshed") { + + implicit val webDriver: WebDriver = new HtmlUnitDriver + implicit val formats = org.json4s.DefaultFormats + + // this test dir is explicitly deleted on successful runs; retained for diagnostics when + // not + val logDir = Utils.createDirectory(System.getProperty("java.io.tmpdir", "logs")) + + // a new conf is used with the background thread set and running at its fastest + // allowed refresh rate (1Hz) + val myConf = new SparkConf() + .set("spark.history.fs.logDirectory", logDir.getAbsolutePath) + .set("spark.eventLog.dir", logDir.getAbsolutePath) + .set("spark.history.fs.update.interval", "1s") + .set("spark.eventLog.enabled", "true") + .set("spark.history.cache.window", "250ms") + .remove("spark.testing") + val provider = new FsHistoryProvider(myConf) + val securityManager = HistoryServer.createSecurityManager(myConf) + + sc = new SparkContext("local", "test", myConf) + val logDirUri = logDir.toURI + val logDirPath = new Path(logDirUri) + val fs = FileSystem.get(logDirUri, sc.hadoopConfiguration) + + def listDir(dir: Path): Seq[FileStatus] = { + val statuses = fs.listStatus(dir) + statuses.flatMap( + stat => if (stat.isDirectory) listDir(stat.getPath) else Seq(stat)) + } + + def dumpLogDir(msg: String = ""): Unit = { + if (log.isDebugEnabled) { + logDebug(msg) + listDir(logDirPath).foreach { status => + val s = status.toString + logDebug(s) + } + } + } + + // stop the server with the old config, and start the new one + server.stop() + server = new HistoryServer(myConf, provider, securityManager, 18080) + server.initialize() + server.bind() + val port = server.boundPort + val metrics = server.cacheMetrics + + // assert that a metric has a value; if not dump the whole metrics instance + def assertMetric(name: String, counter: Counter, expected: Long): Unit = { + val actual = counter.getCount + if (actual != expected) { + // this is here because Scalatest loses stack depth + fail(s"Wrong $name value - expected $expected but got $actual" + + s" in metrics\n$metrics") + } + } + + // build a URL for an app or app/attempt plus a page underneath + def buildURL(appId: String, suffix: String): URL = { + new URL(s"http://localhost:$port/history/$appId$suffix") + } + + // build a rest URL for the application and suffix. + def applications(appId: String, suffix: String): URL = { + new URL(s"http://localhost:$port/api/v1/applications/$appId$suffix") + } + + val historyServerRoot = new URL(s"http://localhost:$port/") + + // start initial job + val d = sc.parallelize(1 to 10) + d.count() + val stdInterval = interval(100 milliseconds) + val appId = eventually(timeout(20 seconds), stdInterval) { + val json = getContentAndCode("applications", port)._2.get + val apps = parse(json).asInstanceOf[JArray].arr + apps should have size 1 + (apps.head \ "id").extract[String] + } + + val appIdRoot = buildURL(appId, "") + val rootAppPage = HistoryServerSuite.getUrl(appIdRoot) + logDebug(s"$appIdRoot ->[${rootAppPage.length}] \n$rootAppPage") + // sanity check to make sure filter is chaining calls + rootAppPage should not be empty + + def getAppUI: SparkUI = { + provider.getAppUI(appId, None).get.ui + } + + // selenium isn't that useful on failures...add our own reporting + def getNumJobs(suffix: String): Int = { + val target = buildURL(appId, suffix) + val targetBody = HistoryServerSuite.getUrl(target) + try { + go to target.toExternalForm + findAll(cssSelector("tbody tr")).toIndexedSeq.size + } catch { + case ex: Exception => + throw new Exception(s"Against $target\n$targetBody", ex) + } + } + // use REST API to get #of jobs + def getNumJobsRestful(): Int = { + val json = HistoryServerSuite.getUrl(applications(appId, "/jobs")) + val jsonAst = parse(json) + val jobList = jsonAst.asInstanceOf[JArray] + jobList.values.size + } + + // get a list of app Ids of all apps in a given state. REST API + def listApplications(completed: Boolean): Seq[String] = { + val json = parse(HistoryServerSuite.getUrl(applications("", ""))) + logDebug(s"${JsonMethods.pretty(json)}") + json match { + case JNothing => Seq() + case apps: JArray => + apps.filter(app => { + (app \ "attempts") match { + case attempts: JArray => + val state = (attempts.children.head \ "completed").asInstanceOf[JBool] + state.value == completed + case _ => false + } + }).map(app => (app \ "id").asInstanceOf[JString].values) + case _ => Seq() + } + } + + def completedJobs(): Seq[JobUIData] = { + getAppUI.jobProgressListener.completedJobs + } + + def activeJobs(): Seq[JobUIData] = { + getAppUI.jobProgressListener.activeJobs.values.toSeq + } + + activeJobs() should have size 0 + completedJobs() should have size 1 + getNumJobs("") should be (1) + getNumJobs("/jobs") should be (1) + getNumJobsRestful() should be (1) + assert(metrics.lookupCount.getCount > 1, s"lookup count too low in $metrics") + + // dump state before the next bit of test, which is where update + // checking really gets stressed + dumpLogDir("filesystem before executing second job") + logDebug(s"History Server: $server") + + val d2 = sc.parallelize(1 to 10) + d2.count() + dumpLogDir("After second job") + + val stdTimeout = timeout(10 seconds) + logDebug("waiting for UI to update") + eventually(stdTimeout, stdInterval) { + assert(2 === getNumJobs(""), + s"jobs not updated, server=$server\n dir = ${listDir(logDirPath)}") + assert(2 === getNumJobs("/jobs"), + s"job count under /jobs not updated, server=$server\n dir = ${listDir(logDirPath)}") + getNumJobsRestful() should be(2) + } + + d.count() + d.count() + eventually(stdTimeout, stdInterval) { + assert(4 === getNumJobsRestful(), s"two jobs back-to-back not updated, server=$server\n") + } + val jobcount = getNumJobs("/jobs") + assert(!provider.getListing().next.completed) + + listApplications(false) should contain(appId) + + // stop the spark context + resetSparkContext() + // check the app is now found as completed + eventually(stdTimeout, stdInterval) { + assert(provider.getListing().next.completed, + s"application never completed, server=$server\n") + } + + // app becomes observably complete + eventually(stdTimeout, stdInterval) { + listApplications(true) should contain (appId) + } + // app is no longer incomplete + listApplications(false) should not contain(appId) + + assert(jobcount === getNumJobs("/jobs")) + + // no need to retain the test dir now the tests complete + logDir.deleteOnExit() + } + + test("ui and api authorization checks") { + val appId = "local-1430917381535" + val owner = "irashid" + val admin = "root" + val other = "alice" + + stop() + init( + "spark.ui.filters" -> classOf[FakeAuthFilter].getName(), + "spark.history.ui.acls.enable" -> "true", + "spark.history.ui.admin.acls" -> admin) + + val tests = Seq( + (owner, HttpServletResponse.SC_OK), + (admin, HttpServletResponse.SC_OK), + (other, HttpServletResponse.SC_FORBIDDEN), + // When the remote user is null, the code behaves as if auth were disabled. + (null, HttpServletResponse.SC_OK)) + + val port = server.boundPort + val testUrls = Seq( + s"http://localhost:$port/api/v1/applications/$appId/1/jobs", + s"http://localhost:$port/history/$appId/1/jobs/", + s"http://localhost:$port/api/v1/applications/$appId/logs", + s"http://localhost:$port/api/v1/applications/$appId/1/logs", + s"http://localhost:$port/api/v1/applications/$appId/2/logs") + + tests.foreach { case (user, expectedCode) => + testUrls.foreach { url => + val headers = if (user != null) Seq(FakeAuthFilter.FAKE_HTTP_USER -> user) else Nil + val sc = TestUtils.httpResponseCode(new URL(url), headers = headers) + assert(sc === expectedCode, s"Unexpected status code $sc for $url (user = $user)") + } + } + } + def getContentAndCode(path: String, port: Int = port): (Int, Option[String], Option[String]) = { HistoryServerSuite.getContentAndCode(new URL(s"http://localhost:$port/api/v1/$path")) } @@ -297,9 +620,11 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers val json = getUrl(path) val file = new File(expRoot, HistoryServerSuite.sanitizePath(name) + "_expectation.json") val out = new FileWriter(file) - out.write(json) + out.write(clearLastUpdated(json)) + out.write('\n') out.close() } + } object HistoryServerSuite { @@ -361,3 +686,26 @@ object HistoryServerSuite { } } } + +/** + * A filter used for auth tests; sets the request's user to the value of the "HTTP_USER" header. + */ +class FakeAuthFilter extends Filter { + + override def destroy(): Unit = { } + + override def init(config: FilterConfig): Unit = { } + + override def doFilter(req: ServletRequest, res: ServletResponse, chain: FilterChain): Unit = { + val hreq = req.asInstanceOf[HttpServletRequest] + val wrapped = new HttpServletRequestWrapper(hreq) { + override def getRemoteUser(): String = hreq.getHeader(FakeAuthFilter.FAKE_HTTP_USER) + } + chain.doFilter(wrapped, res) + } + +} + +object FakeAuthFilter { + val FAKE_HTTP_USER = "HTTP_USER" +} diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 242bf4b5566e..84b3a29b58bf 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -18,22 +18,86 @@ package org.apache.spark.deploy.master import java.util.Date +import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.atomic.AtomicInteger +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.collection.mutable.{HashMap, HashSet} import scala.concurrent.duration._ import scala.io.Source import scala.language.postfixOps +import scala.reflect.ClassTag import org.json4s._ import org.json4s.jackson.JsonMethods._ -import org.scalatest.{Matchers, PrivateMethodTester} +import org.mockito.Mockito.{mock, when} +import org.scalatest.{BeforeAndAfter, Matchers, PrivateMethodTester} import org.scalatest.concurrent.Eventually import other.supplier.{CustomPersistenceEngine, CustomRecoveryModeFactory} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy._ -import org.apache.spark.rpc.RpcEnv +import org.apache.spark.deploy.DeployMessages._ +import org.apache.spark.rpc.{RpcAddress, RpcEndpoint, RpcEndpointRef, RpcEnv} +import org.apache.spark.serializer -class MasterSuite extends SparkFunSuite with Matchers with Eventually with PrivateMethodTester { +object MockWorker { + val counter = new AtomicInteger(10000) +} + +class MockWorker(master: RpcEndpointRef, conf: SparkConf = new SparkConf) extends RpcEndpoint { + val seq = MockWorker.counter.incrementAndGet() + val id = seq.toString + override val rpcEnv: RpcEnv = RpcEnv.create("worker", "localhost", seq, + conf, new SecurityManager(conf)) + var apps = new mutable.HashMap[String, String]() + val driverIdToAppId = new mutable.HashMap[String, String]() + def newDriver(driverId: String): RpcEndpointRef = { + val name = s"driver_${drivers.size}" + rpcEnv.setupEndpoint(name, new RpcEndpoint { + override val rpcEnv: RpcEnv = MockWorker.this.rpcEnv + override def receive: PartialFunction[Any, Unit] = { + case RegisteredApplication(appId, _) => + apps(appId) = appId + driverIdToAppId(driverId) = appId + } + }) + } + + val appDesc = DeployTestUtils.createAppDesc() + val drivers = mutable.HashSet[String]() + override def receive: PartialFunction[Any, Unit] = { + case RegisteredWorker(masterRef, _, _) => + masterRef.send(WorkerLatestState(id, Nil, drivers.toSeq)) + case LaunchDriver(driverId, desc) => + drivers += driverId + master.send(RegisterApplication(appDesc, newDriver(driverId))) + case KillDriver(driverId) => + master.send(DriverStateChanged(driverId, DriverState.KILLED, None)) + drivers -= driverId + driverIdToAppId.get(driverId) match { + case Some(appId) => + apps.remove(appId) + master.send(UnregisterApplication(appId)) + case None => + } + driverIdToAppId.remove(driverId) + } +} + +class MasterSuite extends SparkFunSuite + with Matchers with Eventually with PrivateMethodTester with BeforeAndAfter { + + private var _master: Master = _ + + after { + if (_master != null) { + _master.rpcEnv.shutdown() + _master.rpcEnv.awaitTermination() + _master = null + } + } test("can use a custom recovery mode factory") { val conf = new SparkConf(loadDefaults = false) @@ -90,15 +154,14 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually with Priva cores = 0, memory = 0, endpoint = null, - webUiPort = 0, - publicAddress = "" + webUiAddress = "http://localhost:80" ) val (rpcEnv, _, _) = Master.startRpcEnvAndEndpoint("127.0.0.1", 0, 0, conf) try { - rpcEnv.setupEndpointRef(Master.SYSTEM_NAME, rpcEnv.address, Master.ENDPOINT_NAME) + rpcEnv.setupEndpointRef(rpcEnv.address, Master.ENDPOINT_NAME) CustomPersistenceEngine.lastInstance.isDefined shouldBe true val persistenceEngine = CustomPersistenceEngine.lastInstance.get @@ -121,6 +184,81 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually with Priva CustomRecoveryModeFactory.instantiationAttempts should be > instantiationAttempts } + test("master correctly recover the application") { + val conf = new SparkConf(loadDefaults = false) + conf.set("spark.deploy.recoveryMode", "CUSTOM") + conf.set("spark.deploy.recoveryMode.factory", + classOf[FakeRecoveryModeFactory].getCanonicalName) + conf.set("spark.master.rest.enabled", "false") + + val fakeAppInfo = makeAppInfo(1024) + val fakeWorkerInfo = makeWorkerInfo(8192, 16) + val fakeDriverInfo = new DriverInfo( + startTime = 0, + id = "test_driver", + desc = new DriverDescription( + jarUrl = "", + mem = 1024, + cores = 1, + supervise = false, + command = new Command("", Nil, Map.empty, Nil, Nil, Nil)), + submitDate = new Date()) + + // Build the fake recovery data + FakeRecoveryModeFactory.persistentData.put(s"app_${fakeAppInfo.id}", fakeAppInfo) + FakeRecoveryModeFactory.persistentData.put(s"driver_${fakeDriverInfo.id}", fakeDriverInfo) + FakeRecoveryModeFactory.persistentData.put(s"worker_${fakeWorkerInfo.id}", fakeWorkerInfo) + + var master: Master = null + try { + master = makeMaster(conf) + master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) + // Wait until Master recover from checkpoint data. + eventually(timeout(5 seconds), interval(100 milliseconds)) { + master.workers.size should be(1) + } + + master.idToApp.keySet should be(Set(fakeAppInfo.id)) + getDrivers(master) should be(Set(fakeDriverInfo)) + master.workers should be(Set(fakeWorkerInfo)) + + // Notify Master about the executor and driver info to make it correctly recovered. + val fakeExecutors = List( + new ExecutorDescription(fakeAppInfo.id, 0, 8, ExecutorState.RUNNING), + new ExecutorDescription(fakeAppInfo.id, 0, 7, ExecutorState.RUNNING)) + + fakeAppInfo.state should be(ApplicationState.UNKNOWN) + fakeWorkerInfo.coresFree should be(16) + fakeWorkerInfo.coresUsed should be(0) + + master.self.send(MasterChangeAcknowledged(fakeAppInfo.id)) + eventually(timeout(1 second), interval(10 milliseconds)) { + // Application state should be WAITING when "MasterChangeAcknowledged" event executed. + fakeAppInfo.state should be(ApplicationState.WAITING) + } + + master.self.send( + WorkerSchedulerStateResponse(fakeWorkerInfo.id, fakeExecutors, Seq(fakeDriverInfo.id))) + + eventually(timeout(5 seconds), interval(100 milliseconds)) { + getState(master) should be(RecoveryState.ALIVE) + } + + // If driver's resource is also counted, free cores should 0 + fakeWorkerInfo.coresFree should be(0) + fakeWorkerInfo.coresUsed should be(16) + // State of application should be RUNNING + fakeAppInfo.state should be(ApplicationState.RUNNING) + } finally { + if (master != null) { + master.rpcEnv.shutdown() + master.rpcEnv.awaitTermination() + master = null + FakeRecoveryModeFactory.persistentData.clear() + } + } + } + test("master/worker web ui available") { implicit val formats = org.json4s.DefaultFormats val conf = new SparkConf() @@ -144,6 +282,33 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually with Priva } } + test("master/worker web ui available with reverseProxy") { + implicit val formats = org.json4s.DefaultFormats + val reverseProxyUrl = "http://localhost:8080" + val conf = new SparkConf() + conf.set("spark.ui.reverseProxy", "true") + conf.set("spark.ui.reverseProxyUrl", reverseProxyUrl) + val localCluster = new LocalSparkCluster(2, 2, 512, conf) + localCluster.start() + try { + eventually(timeout(5 seconds), interval(100 milliseconds)) { + val json = Source.fromURL(s"http://localhost:${localCluster.masterWebUIPort}/json") + .getLines().mkString("\n") + val JArray(workers) = (parse(json) \ "workers") + workers.size should be (2) + workers.foreach { workerSummaryJson => + val JString(workerId) = workerSummaryJson \ "id" + val url = s"http://localhost:${localCluster.masterWebUIPort}/proxy/${workerId}/json" + val workerResponse = parse(Source.fromURL(url).getLines().mkString("\n")) + (workerResponse \ "cores").extract[Int] should be (2) + (workerResponse \ "masterwebuiurl").extract[String] should be (reverseProxyUrl) + } + } + } finally { + localCluster.stop() + } + } + test("basic scheduling - spread out") { basicScheduling(spreadOut = true) } @@ -354,14 +519,18 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually with Priva // ========================================== private val _scheduleExecutorsOnWorkers = PrivateMethod[Array[Int]]('scheduleExecutorsOnWorkers) + private val _drivers = PrivateMethod[HashSet[DriverInfo]]('drivers) + private val _state = PrivateMethod[RecoveryState.Value]('state) + private val workerInfo = makeWorkerInfo(4096, 10) private val workerInfos = Array(workerInfo, workerInfo, workerInfo) private def makeMaster(conf: SparkConf = new SparkConf): Master = { + assert(_master === null, "Some Master's RpcEnv is leaked in tests") val securityMgr = new SecurityManager(conf) val rpcEnv = RpcEnv.create(Master.SYSTEM_NAME, "localhost", 0, conf, securityMgr) - val master = new Master(rpcEnv, rpcEnv.address, 0, securityMgr, conf) - master + _master = new Master(rpcEnv, rpcEnv.address, 0, securityMgr, conf) + _master } private def makeAppInfo( @@ -371,12 +540,18 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually with Priva val desc = new ApplicationDescription( "test", maxCores, memoryPerExecutorMb, null, "", None, None, coresPerExecutor) val appId = System.currentTimeMillis.toString - new ApplicationInfo(0, appId, desc, new Date, null, Int.MaxValue) + val endpointRef = mock(classOf[RpcEndpointRef]) + val mockAddress = mock(classOf[RpcAddress]) + when(endpointRef.address).thenReturn(mockAddress) + new ApplicationInfo(0, appId, desc, new Date, endpointRef, Int.MaxValue) } private def makeWorkerInfo(memoryMb: Int, cores: Int): WorkerInfo = { val workerId = System.currentTimeMillis.toString - new WorkerInfo(workerId, "host", 100, cores, memoryMb, null, 101, "address") + val endpointRef = mock(classOf[RpcEndpointRef]) + val mockAddress = mock(classOf[RpcAddress]) + when(endpointRef.address).thenReturn(mockAddress) + new WorkerInfo(workerId, "host", 100, cores, memoryMb, endpointRef, "http://localhost:80") } private def scheduleExecutorsOnWorkers( @@ -387,4 +562,175 @@ class MasterSuite extends SparkFunSuite with Matchers with Eventually with Priva master.invokePrivate(_scheduleExecutorsOnWorkers(appInfo, workerInfos, spreadOut)) } + test("SPARK-13604: Master should ask Worker kill unknown executors and drivers") { + val master = makeMaster() + master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) + eventually(timeout(10.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + assert(masterState.status === RecoveryState.ALIVE, "Master is not alive") + } + + val killedExecutors = new ConcurrentLinkedQueue[(String, Int)]() + val killedDrivers = new ConcurrentLinkedQueue[String]() + val fakeWorker = master.rpcEnv.setupEndpoint("worker", new RpcEndpoint { + override val rpcEnv: RpcEnv = master.rpcEnv + + override def receive: PartialFunction[Any, Unit] = { + case KillExecutor(_, appId, execId) => killedExecutors.add((appId, execId)) + case KillDriver(driverId) => killedDrivers.add(driverId) + } + }) + + master.self.send(RegisterWorker( + "1", + "localhost", + 9999, + fakeWorker, + 10, + 1024, + "http://localhost:8080", + RpcAddress("localhost", 9999))) + val executors = (0 until 3).map { i => + new ExecutorDescription(appId = i.toString, execId = i, 2, ExecutorState.RUNNING) + } + master.self.send(WorkerLatestState("1", executors, driverIds = Seq("0", "1", "2"))) + + eventually(timeout(10.seconds)) { + assert(killedExecutors.asScala.toList.sorted === List("0" -> 0, "1" -> 1, "2" -> 2)) + assert(killedDrivers.asScala.toList.sorted === List("0", "1", "2")) + } + } + + test("SPARK-20529: Master should reply the address received from worker") { + val master = makeMaster() + master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) + eventually(timeout(10.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + assert(masterState.status === RecoveryState.ALIVE, "Master is not alive") + } + + @volatile var receivedMasterAddress: RpcAddress = null + val fakeWorker = master.rpcEnv.setupEndpoint("worker", new RpcEndpoint { + override val rpcEnv: RpcEnv = master.rpcEnv + + override def receive: PartialFunction[Any, Unit] = { + case RegisteredWorker(_, _, masterAddress) => + receivedMasterAddress = masterAddress + } + }) + + master.self.send(RegisterWorker( + "1", + "localhost", + 9999, + fakeWorker, + 10, + 1024, + "http://localhost:8080", + RpcAddress("localhost2", 10000))) + + eventually(timeout(10.seconds)) { + assert(receivedMasterAddress === RpcAddress("localhost2", 10000)) + } + } + + test("SPARK-19900: there should be a corresponding driver for the app after relaunching driver") { + val conf = new SparkConf().set("spark.worker.timeout", "1") + val master = makeMaster(conf) + master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) + eventually(timeout(10.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + assert(masterState.status === RecoveryState.ALIVE, "Master is not alive") + } + val worker1 = new MockWorker(master.self) + worker1.rpcEnv.setupEndpoint("worker", worker1) + val worker1Reg = RegisterWorker( + worker1.id, + "localhost", + 9998, + worker1.self, + 10, + 1024, + "http://localhost:8080", + RpcAddress("localhost2", 10000)) + master.self.send(worker1Reg) + val driver = DeployTestUtils.createDriverDesc().copy(supervise = true) + master.self.askSync[SubmitDriverResponse](RequestSubmitDriver(driver)) + + eventually(timeout(10.seconds)) { + assert(worker1.apps.nonEmpty) + } + + eventually(timeout(10.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + assert(masterState.workers(0).state == WorkerState.DEAD) + } + + val worker2 = new MockWorker(master.self) + worker2.rpcEnv.setupEndpoint("worker", worker2) + master.self.send(RegisterWorker( + worker2.id, + "localhost", + 9999, + worker2.self, + 10, + 1024, + "http://localhost:8081", + RpcAddress("localhost", 10001))) + eventually(timeout(10.seconds)) { + assert(worker2.apps.nonEmpty) + } + + master.self.send(worker1Reg) + eventually(timeout(10.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + + val worker = masterState.workers.filter(w => w.id == worker1.id) + assert(worker.length == 1) + // make sure the `DriverStateChanged` arrives at Master. + assert(worker(0).drivers.isEmpty) + assert(worker1.apps.isEmpty) + assert(worker1.drivers.isEmpty) + assert(worker2.apps.size == 1) + assert(worker2.drivers.size == 1) + assert(masterState.activeDrivers.length == 1) + assert(masterState.activeApps.length == 1) + } + } + + private def getDrivers(master: Master): HashSet[DriverInfo] = { + master.invokePrivate(_drivers()) + } + + private def getState(master: Master): RecoveryState.Value = { + master.invokePrivate(_state()) + } +} + +private class FakeRecoveryModeFactory(conf: SparkConf, ser: serializer.Serializer) + extends StandaloneRecoveryModeFactory(conf, ser) { + import FakeRecoveryModeFactory.persistentData + + override def createPersistenceEngine(): PersistenceEngine = new PersistenceEngine { + + override def unpersist(name: String): Unit = { + persistentData.remove(name) + } + + override def persist(name: String, obj: Object): Unit = { + persistentData(name) = obj + } + + override def read[T: ClassTag](prefix: String): Seq[T] = { + persistentData.filter(_._1.startsWith(prefix)).map(_._2.asInstanceOf[T]).toSeq + } + } + + override def createLeaderElectionAgent(master: LeaderElectable): LeaderElectionAgent = { + new MonarchyLeaderAgent(master) + } +} + +private object FakeRecoveryModeFactory { + val persistentData = new HashMap[String, Object]() } diff --git a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala index 34775577de8a..62fe0eaedfd2 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala @@ -25,7 +25,7 @@ import org.apache.curator.test.TestingServer import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.rpc.{RpcEndpoint, RpcEnv} -import org.apache.spark.serializer.{Serializer, JavaSerializer} +import org.apache.spark.serializer.{JavaSerializer, Serializer} import org.apache.spark.util.Utils class PersistenceEngineSuite extends SparkFunSuite { @@ -63,56 +63,57 @@ class PersistenceEngineSuite extends SparkFunSuite { conf: SparkConf, persistenceEngineCreator: Serializer => PersistenceEngine): Unit = { val serializer = new JavaSerializer(conf) val persistenceEngine = persistenceEngineCreator(serializer) - persistenceEngine.persist("test_1", "test_1_value") - assert(Seq("test_1_value") === persistenceEngine.read[String]("test_")) - persistenceEngine.persist("test_2", "test_2_value") - assert(Set("test_1_value", "test_2_value") === persistenceEngine.read[String]("test_").toSet) - persistenceEngine.unpersist("test_1") - assert(Seq("test_2_value") === persistenceEngine.read[String]("test_")) - persistenceEngine.unpersist("test_2") - assert(persistenceEngine.read[String]("test_").isEmpty) - - // Test deserializing objects that contain RpcEndpointRef - val testRpcEnv = RpcEnv.create("test", "localhost", 12345, conf, new SecurityManager(conf)) try { - // Create a real endpoint so that we can test RpcEndpointRef deserialization - val workerEndpoint = testRpcEnv.setupEndpoint("worker", new RpcEndpoint { - override val rpcEnv: RpcEnv = testRpcEnv - }) - - val workerToPersist = new WorkerInfo( - id = "test_worker", - host = "127.0.0.1", - port = 10000, - cores = 0, - memory = 0, - endpoint = workerEndpoint, - webUiPort = 0, - publicAddress = "" - ) - - persistenceEngine.addWorker(workerToPersist) - - val (storedApps, storedDrivers, storedWorkers) = - persistenceEngine.readPersistedData(testRpcEnv) - - assert(storedApps.isEmpty) - assert(storedDrivers.isEmpty) - - // Check deserializing WorkerInfo - assert(storedWorkers.size == 1) - val recoveryWorkerInfo = storedWorkers.head - assert(workerToPersist.id === recoveryWorkerInfo.id) - assert(workerToPersist.host === recoveryWorkerInfo.host) - assert(workerToPersist.port === recoveryWorkerInfo.port) - assert(workerToPersist.cores === recoveryWorkerInfo.cores) - assert(workerToPersist.memory === recoveryWorkerInfo.memory) - assert(workerToPersist.endpoint === recoveryWorkerInfo.endpoint) - assert(workerToPersist.webUiPort === recoveryWorkerInfo.webUiPort) - assert(workerToPersist.publicAddress === recoveryWorkerInfo.publicAddress) + persistenceEngine.persist("test_1", "test_1_value") + assert(Seq("test_1_value") === persistenceEngine.read[String]("test_")) + persistenceEngine.persist("test_2", "test_2_value") + assert(Set("test_1_value", "test_2_value") === persistenceEngine.read[String]("test_").toSet) + persistenceEngine.unpersist("test_1") + assert(Seq("test_2_value") === persistenceEngine.read[String]("test_")) + persistenceEngine.unpersist("test_2") + assert(persistenceEngine.read[String]("test_").isEmpty) + + // Test deserializing objects that contain RpcEndpointRef + val testRpcEnv = RpcEnv.create("test", "localhost", 12345, conf, new SecurityManager(conf)) + try { + // Create a real endpoint so that we can test RpcEndpointRef deserialization + val workerEndpoint = testRpcEnv.setupEndpoint("worker", new RpcEndpoint { + override val rpcEnv: RpcEnv = testRpcEnv + }) + + val workerToPersist = new WorkerInfo( + id = "test_worker", + host = "127.0.0.1", + port = 10000, + cores = 0, + memory = 0, + endpoint = workerEndpoint, + webUiAddress = "http://localhost:80") + + persistenceEngine.addWorker(workerToPersist) + + val (storedApps, storedDrivers, storedWorkers) = + persistenceEngine.readPersistedData(testRpcEnv) + + assert(storedApps.isEmpty) + assert(storedDrivers.isEmpty) + + // Check deserializing WorkerInfo + assert(storedWorkers.size == 1) + val recoveryWorkerInfo = storedWorkers.head + assert(workerToPersist.id === recoveryWorkerInfo.id) + assert(workerToPersist.host === recoveryWorkerInfo.host) + assert(workerToPersist.port === recoveryWorkerInfo.port) + assert(workerToPersist.cores === recoveryWorkerInfo.cores) + assert(workerToPersist.memory === recoveryWorkerInfo.memory) + assert(workerToPersist.endpoint === recoveryWorkerInfo.endpoint) + assert(workerToPersist.webUiAddress === recoveryWorkerInfo.webUiAddress) + } finally { + testRpcEnv.shutdown() + testRpcEnv.awaitTermination() + } } finally { - testRpcEnv.shutdown() - testRpcEnv.awaitTermination() + persistenceEngine.close() } } diff --git a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala new file mode 100644 index 000000000000..69a460fbc7db --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.master.ui + +import java.io.DataOutputStream +import java.net.{HttpURLConnection, URL} +import java.nio.charset.StandardCharsets +import java.util.Date + +import scala.collection.mutable.HashMap + +import org.mockito.Mockito.{mock, times, verify, when} +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.DeployMessages.{KillDriverResponse, RequestKillDriver} +import org.apache.spark.deploy.DeployTestUtils._ +import org.apache.spark.deploy.master._ +import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv} + + +class MasterWebUISuite extends SparkFunSuite with BeforeAndAfterAll { + + val conf = new SparkConf + val securityMgr = new SecurityManager(conf) + val rpcEnv = mock(classOf[RpcEnv]) + val master = mock(classOf[Master]) + val masterEndpointRef = mock(classOf[RpcEndpointRef]) + when(master.securityMgr).thenReturn(securityMgr) + when(master.conf).thenReturn(conf) + when(master.rpcEnv).thenReturn(rpcEnv) + when(master.self).thenReturn(masterEndpointRef) + val masterWebUI = new MasterWebUI(master, 0) + + override def beforeAll() { + super.beforeAll() + masterWebUI.bind() + } + + override def afterAll() { + masterWebUI.stop() + super.afterAll() + } + + test("kill application") { + val appDesc = createAppDesc() + // use new start date so it isn't filtered by UI + val activeApp = new ApplicationInfo( + new Date().getTime, "app-0", appDesc, new Date(), null, Int.MaxValue) + + when(master.idToApp).thenReturn(HashMap[String, ApplicationInfo]((activeApp.id, activeApp))) + + val url = s"http://localhost:${masterWebUI.boundPort}/app/kill/" + val body = convPostDataToString(Map(("id", activeApp.id), ("terminate", "true"))) + val conn = sendHttpRequest(url, "POST", body) + conn.getResponseCode + + // Verify the master was called to remove the active app + verify(master, times(1)).removeApplication(activeApp, ApplicationState.KILLED) + } + + test("kill driver") { + val activeDriverId = "driver-0" + val url = s"http://localhost:${masterWebUI.boundPort}/driver/kill/" + val body = convPostDataToString(Map(("id", activeDriverId), ("terminate", "true"))) + val conn = sendHttpRequest(url, "POST", body) + conn.getResponseCode + + // Verify that master was asked to kill driver with the correct id + verify(masterEndpointRef, times(1)).ask[KillDriverResponse](RequestKillDriver(activeDriverId)) + } + + private def convPostDataToString(data: Map[String, String]): String = { + (for ((name, value) <- data) yield s"$name=$value").mkString("&") + } + + /** + * Send an HTTP request to the given URL using the method and the body specified. + * Return the connection object. + */ + 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) + conn.setRequestProperty("Content-Type", "application/x-www-form-urlencoded") + conn.setRequestProperty("Content-Length", Integer.toString(body.length)) + val out = new DataOutputStream(conn.getOutputStream) + out.write(body.getBytes(StandardCharsets.UTF_8)) + out.close() + } + conn + } +} 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 9693e32bf6af..70887dc5dd97 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 @@ -11,7 +11,7 @@ * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and + * See the License for the specific language governing permissions and * limitations under the License. */ @@ -19,21 +19,21 @@ package org.apache.spark.deploy.rest import java.io.DataOutputStream import java.net.{HttpURLConnection, URL} +import java.nio.charset.StandardCharsets import javax.servlet.http.HttpServletResponse import scala.collection.mutable -import com.google.common.base.Charsets -import org.scalatest.BeforeAndAfterEach import org.json4s.JsonAST._ import org.json4s.jackson.JsonMethods._ +import org.scalatest.BeforeAndAfterEach import org.apache.spark._ -import org.apache.spark.rpc._ -import org.apache.spark.util.Utils -import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.{SparkSubmit, SparkSubmitArguments} +import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.DriverState._ +import org.apache.spark.rpc._ +import org.apache.spark.util.Utils /** * Tests for the REST application submission protocol used in standalone cluster mode. @@ -43,8 +43,12 @@ class StandaloneRestSubmitSuite extends SparkFunSuite with BeforeAndAfterEach { private var server: Option[RestSubmissionServer] = None override def afterEach() { - rpcEnv.foreach(_.shutdown()) - server.foreach(_.stop()) + try { + rpcEnv.foreach(_.shutdown()) + server.foreach(_.stop()) + } finally { + super.afterEach() + } } test("construct submit request") { @@ -404,7 +408,7 @@ class StandaloneRestSubmitSuite extends SparkFunSuite with BeforeAndAfterEach { /** * Start a [[StandaloneRestServer]] that communicates with the given endpoint. - * If `faulty` is true, start an [[FaultyStandaloneRestServer]] instead. + * If `faulty` is true, start a [[FaultyStandaloneRestServer]] instead. * Return the master URL that corresponds to the address of this server. */ private def startServer( @@ -494,7 +498,7 @@ class StandaloneRestSubmitSuite extends SparkFunSuite with BeforeAndAfterEach { if (body.nonEmpty) { conn.setDoOutput(true) val out = new DataOutputStream(conn.getOutputStream) - out.write(body.getBytes(Charsets.UTF_8)) + out.write(body.getBytes(StandardCharsets.UTF_8)) out.close() } conn diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala new file mode 100644 index 000000000000..eeffc36070b4 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.security + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.security.Credentials +import org.scalatest.Matchers + +import org.apache.spark.{SparkConf, SparkFunSuite} + +class HadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers { + private var delegationTokenManager: HadoopDelegationTokenManager = null + private var sparkConf: SparkConf = null + private var hadoopConf: Configuration = null + + override def beforeAll(): Unit = { + super.beforeAll() + + sparkConf = new SparkConf() + hadoopConf = new Configuration() + } + + test("Correctly load default credential providers") { + delegationTokenManager = new HadoopDelegationTokenManager( + sparkConf, + hadoopConf, + hadoopFSsToAccess) + + delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hive") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("bogus") should be (None) + } + + test("disable hive credential provider") { + sparkConf.set("spark.security.credentials.hive.enabled", "false") + delegationTokenManager = new HadoopDelegationTokenManager( + sparkConf, + hadoopConf, + hadoopFSsToAccess) + + delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hive") should be (None) + } + + test("using deprecated configurations") { + sparkConf.set("spark.yarn.security.tokens.hadoopfs.enabled", "false") + sparkConf.set("spark.yarn.security.credentials.hive.enabled", "false") + delegationTokenManager = new HadoopDelegationTokenManager( + sparkConf, + hadoopConf, + hadoopFSsToAccess) + + delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hive") should be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) + } + + test("verify no credentials are obtained") { + delegationTokenManager = new HadoopDelegationTokenManager( + sparkConf, + hadoopConf, + hadoopFSsToAccess) + val creds = new Credentials() + + // Tokens cannot be obtained from HDFS, Hive, HBase in unit tests. + delegationTokenManager.obtainDelegationTokens(hadoopConf, creds) + val tokens = creds.getAllTokens + tokens.size() should be (0) + } + + test("obtain tokens For HiveMetastore") { + val hadoopConf = new Configuration() + hadoopConf.set("hive.metastore.kerberos.principal", "bob") + // thrift picks up on port 0 and bails out, without trying to talk to endpoint + hadoopConf.set("hive.metastore.uris", "http://localhost:0") + + val hiveCredentialProvider = new HiveDelegationTokenProvider() + val credentials = new Credentials() + hiveCredentialProvider.obtainDelegationTokens(hadoopConf, sparkConf, credentials) + + credentials.getAllTokens.size() should be (0) + } + + test("Obtain tokens For HBase") { + val hadoopConf = new Configuration() + hadoopConf.set("hbase.security.authentication", "kerberos") + + val hbaseTokenProvider = new HBaseDelegationTokenProvider() + val creds = new Credentials() + hbaseTokenProvider.obtainDelegationTokens(hadoopConf, sparkConf, creds) + + creds.getAllTokens.size should be (0) + } + + private[spark] def hadoopFSsToAccess(hadoopConf: Configuration): Set[FileSystem] = { + Set(FileSystem.get(hadoopConf)) + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala index 7101cb9978df..607c0a4fac46 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/CommandUtilsSuite.scala @@ -17,10 +17,11 @@ package org.apache.spark.deploy.worker +import org.scalatest.{Matchers, PrivateMethodTester} + import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.Command import org.apache.spark.util.Utils -import org.scalatest.{Matchers, PrivateMethodTester} class CommandUtilsSuite extends SparkFunSuite with Matchers with PrivateMethodTester { diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index 6258c18d177f..52956045d598 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -19,13 +19,18 @@ package org.apache.spark.deploy.worker import java.io.File -import org.mockito.Mockito._ +import scala.concurrent.duration._ + import org.mockito.Matchers._ +import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer +import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.{Command, DriverDescription} +import org.apache.spark.deploy.master.DriverState +import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Clock class DriverRunnerTest extends SparkFunSuite { @@ -33,8 +38,10 @@ class DriverRunnerTest extends SparkFunSuite { val command = new Command("mainClass", Seq(), Map(), Seq(), Seq(), Seq()) val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) val conf = new SparkConf() - new DriverRunner(conf, "driverId", new File("workDir"), new File("sparkHome"), - driverDescription, null, "akka://1.2.3.4/worker/", new SecurityManager(conf)) + val worker = mock(classOf[RpcEndpointRef]) + doNothing().when(worker).send(any()) + spy(new DriverRunner(conf, "driverId", new File("workDir"), new File("sparkHome"), + driverDescription, worker, "spark://1.2.3.4/worker/", new SecurityManager(conf))) } private def createProcessBuilderAndProcess(): (ProcessBuilderLike, Process) = { @@ -45,6 +52,19 @@ class DriverRunnerTest extends SparkFunSuite { (processBuilder, process) } + private def createTestableDriverRunner( + processBuilder: ProcessBuilderLike, + superviseRetry: Boolean) = { + val runner = createDriverRunner() + runner.setSleeper(mock(classOf[Sleeper])) + doAnswer(new Answer[Int] { + def answer(invocation: InvocationOnMock): Int = { + runner.runCommandWithRetry(processBuilder, p => (), supervise = superviseRetry) + } + }).when(runner).prepareAndRunDriver() + runner + } + test("Process succeeds instantly") { val runner = createDriverRunner() @@ -145,4 +165,53 @@ class DriverRunnerTest extends SparkFunSuite { verify(sleeper, times(2)).sleep(2) } + test("Kill process finalized with state KILLED") { + val (processBuilder, process) = createProcessBuilderAndProcess() + val runner = createTestableDriverRunner(processBuilder, superviseRetry = true) + + when(process.waitFor()).thenAnswer(new Answer[Int] { + def answer(invocation: InvocationOnMock): Int = { + runner.kill() + -1 + } + }) + + runner.start() + + eventually(timeout(10.seconds), interval(100.millis)) { + assert(runner.finalState.get === DriverState.KILLED) + } + verify(process, times(1)).waitFor() + } + + test("Finalized with state FINISHED") { + val (processBuilder, process) = createProcessBuilderAndProcess() + val runner = createTestableDriverRunner(processBuilder, superviseRetry = true) + when(process.waitFor()).thenReturn(0) + runner.start() + eventually(timeout(10.seconds), interval(100.millis)) { + assert(runner.finalState.get === DriverState.FINISHED) + } + } + + test("Finalized with state FAILED") { + val (processBuilder, process) = createProcessBuilderAndProcess() + val runner = createTestableDriverRunner(processBuilder, superviseRetry = false) + when(process.waitFor()).thenReturn(-1) + runner.start() + eventually(timeout(10.seconds), interval(100.millis)) { + assert(runner.finalState.get === DriverState.FAILED) + } + } + + test("Handle exception starting process") { + val (processBuilder, process) = createProcessBuilderAndProcess() + val runner = createTestableDriverRunner(processBuilder, superviseRetry = false) + when(processBuilder.start()).thenThrow(new NullPointerException("bad command list")) + runner.start() + eventually(timeout(10.seconds), interval(100.millis)) { + assert(runner.finalState.get === DriverState.ERROR) + assert(runner.finalException.get.isInstanceOf[RuntimeException]) + } + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 98664dc1101e..0240bf8aed4c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -19,8 +19,8 @@ package org.apache.spark.deploy.worker import java.io.File -import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} class ExecutorRunnerTest extends SparkFunSuite { test("command includes appId") { diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala index faed4bdc6844..ce212a751331 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala @@ -17,15 +17,15 @@ package org.apache.spark.deploy.worker -import org.scalatest.Matchers +import org.scalatest.{BeforeAndAfter, Matchers} +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.{Command, ExecutorState} import org.apache.spark.deploy.DeployMessages.{DriverStateChanged, ExecutorStateChanged} import org.apache.spark.deploy.master.DriverState -import org.apache.spark.deploy.{Command, ExecutorState} import org.apache.spark.rpc.{RpcAddress, RpcEnv} -import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -class WorkerSuite extends SparkFunSuite with Matchers { +class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { import org.apache.spark.deploy.DeployTestUtils._ @@ -34,6 +34,25 @@ class WorkerSuite extends SparkFunSuite with Matchers { } def conf(opts: (String, String)*): SparkConf = new SparkConf(loadDefaults = false).setAll(opts) + private var _worker: Worker = _ + + private def makeWorker(conf: SparkConf): Worker = { + assert(_worker === null, "Some Worker's RpcEnv is leaked in tests") + val securityMgr = new SecurityManager(conf) + val rpcEnv = RpcEnv.create("test", "localhost", 12345, conf, securityMgr) + _worker = new Worker(rpcEnv, 50000, 20, 1234 * 5, Array.fill(1)(RpcAddress("1.2.3.4", 1234)), + "Worker", "/tmp", conf, securityMgr) + _worker + } + + after { + if (_worker != null) { + _worker.rpcEnv.shutdown() + _worker.rpcEnv.awaitTermination() + _worker = null + } + } + test("test isUseLocalNodeSSLConfig") { Worker.isUseLocalNodeSSLConfig(cmd("-Dasdf=dfgh")) shouldBe false Worker.isUseLocalNodeSSLConfig(cmd("-Dspark.ssl.useNodeLocalConf=true")) shouldBe true @@ -65,9 +84,7 @@ class WorkerSuite extends SparkFunSuite with Matchers { test("test clearing of finishedExecutors (small number of executors)") { val conf = new SparkConf() conf.set("spark.worker.ui.retainedExecutors", 2.toString) - val rpcEnv = RpcEnv.create("test", "localhost", 12345, conf, new SecurityManager(conf)) - val worker = new Worker(rpcEnv, 50000, 20, 1234 * 5, Array.fill(1)(RpcAddress("1.2.3.4", 1234)), - "sparkWorker1", "Worker", "/tmp", conf, new SecurityManager(conf)) + val worker = makeWorker(conf) // initialize workers for (i <- 0 until 5) { worker.executors += s"app1/$i" -> createExecutorRunner(i) @@ -91,9 +108,7 @@ class WorkerSuite extends SparkFunSuite with Matchers { test("test clearing of finishedExecutors (more executors)") { val conf = new SparkConf() conf.set("spark.worker.ui.retainedExecutors", 30.toString) - val rpcEnv = RpcEnv.create("test", "localhost", 12345, conf, new SecurityManager(conf)) - val worker = new Worker(rpcEnv, 50000, 20, 1234 * 5, Array.fill(1)(RpcAddress("1.2.3.4", 1234)), - "sparkWorker1", "Worker", "/tmp", conf, new SecurityManager(conf)) + val worker = makeWorker(conf) // initialize workers for (i <- 0 until 50) { worker.executors += s"app1/$i" -> createExecutorRunner(i) @@ -126,9 +141,7 @@ class WorkerSuite extends SparkFunSuite with Matchers { test("test clearing of finishedDrivers (small number of drivers)") { val conf = new SparkConf() conf.set("spark.worker.ui.retainedDrivers", 2.toString) - val rpcEnv = RpcEnv.create("test", "localhost", 12345, conf, new SecurityManager(conf)) - val worker = new Worker(rpcEnv, 50000, 20, 1234 * 5, Array.fill(1)(RpcAddress("1.2.3.4", 1234)), - "sparkWorker1", "Worker", "/tmp", conf, new SecurityManager(conf)) + val worker = makeWorker(conf) // initialize workers for (i <- 0 until 5) { val driverId = s"driverId-$i" @@ -152,9 +165,7 @@ class WorkerSuite extends SparkFunSuite with Matchers { test("test clearing of finishedDrivers (more drivers)") { val conf = new SparkConf() conf.set("spark.worker.ui.retainedDrivers", 30.toString) - val rpcEnv = RpcEnv.create("test", "localhost", 12345, conf, new SecurityManager(conf)) - val worker = new Worker(rpcEnv, 50000, 20, 1234 * 5, Array.fill(1)(RpcAddress("1.2.3.4", 1234)), - "sparkWorker1", "Worker", "/tmp", conf, new SecurityManager(conf)) + val worker = makeWorker(conf) // initialize workers for (i <- 0 until 50) { val driverId = s"driverId-$i" diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala index 40c24bdecc6c..31bea3293ae7 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala @@ -17,15 +17,14 @@ package org.apache.spark.deploy.worker -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.SecurityManager -import org.apache.spark.rpc.{RpcAddress, RpcEnv} +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress, RpcEnv} class WorkerWatcherSuite extends SparkFunSuite { test("WorkerWatcher shuts down on valid disassociation") { val conf = new SparkConf() val rpcEnv = RpcEnv.create("test", "localhost", 12345, conf, new SecurityManager(conf)) - val targetWorkerUrl = rpcEnv.uriOf("test", RpcAddress("1.2.3.4", 1234), "Worker") + val targetWorkerUrl = RpcEndpointAddress(RpcAddress("1.2.3.4", 1234), "Worker").toString val workerWatcher = new WorkerWatcher(rpcEnv, targetWorkerUrl, isTesting = true) rpcEnv.setupEndpoint("worker-watcher", workerWatcher) workerWatcher.onDisconnected(RpcAddress("1.2.3.4", 1234)) @@ -36,7 +35,7 @@ class WorkerWatcherSuite extends SparkFunSuite { test("WorkerWatcher stays alive on invalid disassociation") { val conf = new SparkConf() val rpcEnv = RpcEnv.create("test", "localhost", 12345, conf, new SecurityManager(conf)) - val targetWorkerUrl = rpcEnv.uriOf("test", RpcAddress("1.2.3.4", 1234), "Worker") + val targetWorkerUrl = RpcEndpointAddress(RpcAddress("1.2.3.4", 1234), "Worker").toString val otherRpcAddress = RpcAddress("4.3.2.1", 1234) val workerWatcher = new WorkerWatcher(rpcEnv, targetWorkerUrl, isTesting = true) rpcEnv.setupEndpoint("worker-watcher", workerWatcher) diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ui/LogPageSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ui/LogPageSuite.scala index 72eaffb41698..4c3e96777940 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ui/LogPageSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ui/LogPageSuite.scala @@ -22,16 +22,20 @@ import java.io.{File, FileWriter} import org.mockito.Mockito.{mock, when} import org.scalatest.PrivateMethodTester -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.worker.Worker class LogPageSuite extends SparkFunSuite with PrivateMethodTester { test("get logs simple") { val webui = mock(classOf[WorkerWebUI]) + val worker = mock(classOf[Worker]) val tmpDir = new File(sys.props("java.io.tmpdir")) val workDir = new File(tmpDir, "work-dir") workDir.mkdir() when(webui.workDir).thenReturn(workDir) + when(webui.worker).thenReturn(worker) + when(worker.conf).thenReturn(new SparkConf()) val logPage = new LogPage(webui) // Prepare some fake log files to read later diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala new file mode 100644 index 000000000000..105a178f2d94 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -0,0 +1,364 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import java.io.{Externalizable, ObjectInput, ObjectOutput} +import java.lang.Thread.UncaughtExceptionHandler +import java.nio.ByteBuffer +import java.util.Properties +import java.util.concurrent.{CountDownLatch, TimeUnit} + +import scala.collection.mutable.Map +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.mockito.ArgumentCaptor +import org.mockito.Matchers.{any, eq => meq} +import org.mockito.Mockito.{inOrder, verify, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.concurrent.Eventually +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark._ +import org.apache.spark.TaskState.TaskState +import org.apache.spark.memory.MemoryManager +import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.rdd.RDD +import org.apache.spark.rpc.RpcEnv +import org.apache.spark.scheduler.{FakeTask, ResultTask, TaskDescription} +import org.apache.spark.serializer.{JavaSerializer, SerializerManager} +import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.UninterruptibleThread + +class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar with Eventually { + + test("SPARK-15963: Catch `TaskKilledException` correctly in Executor.TaskRunner") { + // mock some objects to make Executor.launchTask() happy + val conf = new SparkConf + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + val serializedTask = serializer.newInstance().serialize(new FakeTask(0, 0)) + val taskDescription = createFakeTaskDescription(serializedTask) + + // we use latches to force the program to run in this order: + // +-----------------------------+---------------------------------------+ + // | main test thread | worker thread | + // +-----------------------------+---------------------------------------+ + // | executor.launchTask() | | + // | | TaskRunner.run() begins | + // | | ... | + // | | execBackend.statusUpdate // 1st time | + // | executor.killAllTasks(true) | | + // | | ... | + // | | task = ser.deserialize | + // | | ... | + // | | execBackend.statusUpdate // 2nd time | + // | | ... | + // | | TaskRunner.run() ends | + // | check results | | + // +-----------------------------+---------------------------------------+ + + val executorSuiteHelper = new ExecutorSuiteHelper + + val mockExecutorBackend = mock[ExecutorBackend] + when(mockExecutorBackend.statusUpdate(any(), any(), any())) + .thenAnswer(new Answer[Unit] { + var firstTime = true + override def answer(invocationOnMock: InvocationOnMock): Unit = { + if (firstTime) { + executorSuiteHelper.latch1.countDown() + // here between latch1 and latch2, executor.killAllTasks() is called + executorSuiteHelper.latch2.await() + firstTime = false + } + else { + // save the returned `taskState` and `testFailedReason` into `executorSuiteHelper` + val taskState = invocationOnMock.getArguments()(1).asInstanceOf[TaskState] + executorSuiteHelper.taskState = taskState + val taskEndReason = invocationOnMock.getArguments()(2).asInstanceOf[ByteBuffer] + executorSuiteHelper.testFailedReason = + serializer.newInstance().deserialize(taskEndReason) + // let the main test thread check `taskState` and `testFailedReason` + executorSuiteHelper.latch3.countDown() + } + } + }) + + var executor: Executor = null + try { + executor = new Executor("id", "localhost", env, userClassPath = Nil, isLocal = true) + // the task will be launched in a dedicated worker thread + executor.launchTask(mockExecutorBackend, taskDescription) + + if (!executorSuiteHelper.latch1.await(5, TimeUnit.SECONDS)) { + fail("executor did not send first status update in time") + } + // we know the task will be started, but not yet deserialized, because of the latches we + // use in mockExecutorBackend. + executor.killAllTasks(true, "test") + executorSuiteHelper.latch2.countDown() + if (!executorSuiteHelper.latch3.await(5, TimeUnit.SECONDS)) { + fail("executor did not send second status update in time") + } + + // `testFailedReason` should be `TaskKilled`; `taskState` should be `KILLED` + assert(executorSuiteHelper.testFailedReason === TaskKilled("test")) + assert(executorSuiteHelper.taskState === TaskState.KILLED) + } + finally { + if (executor != null) { + executor.stop() + } + } + } + + test("SPARK-19276: Handle FetchFailedExceptions that are hidden by user exceptions") { + val conf = new SparkConf().setMaster("local").setAppName("executor suite test") + sc = new SparkContext(conf) + val serializer = SparkEnv.get.closureSerializer.newInstance() + val resultFunc = (context: TaskContext, itr: Iterator[Int]) => itr.size + + // Submit a job where a fetch failure is thrown, but user code has a try/catch which hides + // the fetch failure. The executor should still tell the driver that the task failed due to a + // fetch failure, not a generic exception from user code. + val inputRDD = new FetchFailureThrowingRDD(sc) + val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = false) + val taskBinary = sc.broadcast(serializer.serialize((secondRDD, resultFunc)).array()) + val serializedTaskMetrics = serializer.serialize(TaskMetrics.registered).array() + val task = new ResultTask( + stageId = 1, + stageAttemptId = 0, + taskBinary = taskBinary, + partition = secondRDD.partitions(0), + locs = Seq(), + outputId = 0, + localProperties = new Properties(), + serializedTaskMetrics = serializedTaskMetrics + ) + + val serTask = serializer.serialize(task) + val taskDescription = createFakeTaskDescription(serTask) + + val failReason = runTaskAndGetFailReason(taskDescription) + assert(failReason.isInstanceOf[FetchFailed]) + } + + test("Executor's worker threads should be UninterruptibleThread") { + val conf = new SparkConf() + .setMaster("local") + .setAppName("executor thread test") + .set("spark.ui.enabled", "false") + sc = new SparkContext(conf) + val executorThread = sc.parallelize(Seq(1), 1).map { _ => + Thread.currentThread.getClass.getName + }.collect().head + assert(executorThread === classOf[UninterruptibleThread].getName) + } + + test("SPARK-19276: OOMs correctly handled with a FetchFailure") { + // when there is a fatal error like an OOM, we don't do normal fetch failure handling, since it + // may be a false positive. And we should call the uncaught exception handler. + val conf = new SparkConf().setMaster("local").setAppName("executor suite test") + sc = new SparkContext(conf) + val serializer = SparkEnv.get.closureSerializer.newInstance() + val resultFunc = (context: TaskContext, itr: Iterator[Int]) => itr.size + + // Submit a job where a fetch failure is thrown, but then there is an OOM. We should treat + // the fetch failure as a false positive, and just do normal OOM handling. + val inputRDD = new FetchFailureThrowingRDD(sc) + val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = true) + val taskBinary = sc.broadcast(serializer.serialize((secondRDD, resultFunc)).array()) + val serializedTaskMetrics = serializer.serialize(TaskMetrics.registered).array() + val task = new ResultTask( + stageId = 1, + stageAttemptId = 0, + taskBinary = taskBinary, + partition = secondRDD.partitions(0), + locs = Seq(), + outputId = 0, + localProperties = new Properties(), + serializedTaskMetrics = serializedTaskMetrics + ) + + val serTask = serializer.serialize(task) + val taskDescription = createFakeTaskDescription(serTask) + + val (failReason, uncaughtExceptionHandler) = + runTaskGetFailReasonAndExceptionHandler(taskDescription) + // make sure the task failure just looks like a OOM, not a fetch failure + assert(failReason.isInstanceOf[ExceptionFailure]) + val exceptionCaptor = ArgumentCaptor.forClass(classOf[Throwable]) + verify(uncaughtExceptionHandler).uncaughtException(any(), exceptionCaptor.capture()) + assert(exceptionCaptor.getAllValues.size === 1) + assert(exceptionCaptor.getAllValues.get(0).isInstanceOf[OutOfMemoryError]) + } + + test("Gracefully handle error in task deserialization") { + val conf = new SparkConf + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + val serializedTask = serializer.newInstance().serialize(new NonDeserializableTask) + val taskDescription = createFakeTaskDescription(serializedTask) + + val failReason = runTaskAndGetFailReason(taskDescription) + failReason match { + case ef: ExceptionFailure => + assert(ef.exception.isDefined) + assert(ef.exception.get.getMessage() === NonDeserializableTask.errorMsg) + case _ => + fail(s"unexpected failure type: $failReason") + } + } + + private def createMockEnv(conf: SparkConf, serializer: JavaSerializer): SparkEnv = { + val mockEnv = mock[SparkEnv] + val mockRpcEnv = mock[RpcEnv] + val mockMetricsSystem = mock[MetricsSystem] + val mockMemoryManager = mock[MemoryManager] + when(mockEnv.conf).thenReturn(conf) + when(mockEnv.serializer).thenReturn(serializer) + when(mockEnv.serializerManager).thenReturn(mock[SerializerManager]) + when(mockEnv.rpcEnv).thenReturn(mockRpcEnv) + when(mockEnv.metricsSystem).thenReturn(mockMetricsSystem) + when(mockEnv.memoryManager).thenReturn(mockMemoryManager) + when(mockEnv.closureSerializer).thenReturn(serializer) + SparkEnv.set(mockEnv) + mockEnv + } + + private def createFakeTaskDescription(serializedTask: ByteBuffer): TaskDescription = { + new TaskDescription( + taskId = 0, + attemptNumber = 0, + executorId = "", + name = "", + index = 0, + addedFiles = Map[String, Long](), + addedJars = Map[String, Long](), + properties = new Properties, + serializedTask) + } + + private def runTaskAndGetFailReason(taskDescription: TaskDescription): TaskFailedReason = { + runTaskGetFailReasonAndExceptionHandler(taskDescription)._1 + } + + private def runTaskGetFailReasonAndExceptionHandler( + taskDescription: TaskDescription): (TaskFailedReason, UncaughtExceptionHandler) = { + val mockBackend = mock[ExecutorBackend] + val mockUncaughtExceptionHandler = mock[UncaughtExceptionHandler] + var executor: Executor = null + try { + executor = new Executor("id", "localhost", SparkEnv.get, userClassPath = Nil, isLocal = true, + uncaughtExceptionHandler = mockUncaughtExceptionHandler) + // the task will be launched in a dedicated worker thread + executor.launchTask(mockBackend, taskDescription) + eventually(timeout(5.seconds), interval(10.milliseconds)) { + assert(executor.numRunningTasks === 0) + } + } finally { + if (executor != null) { + executor.stop() + } + } + val orderedMock = inOrder(mockBackend) + val statusCaptor = ArgumentCaptor.forClass(classOf[ByteBuffer]) + orderedMock.verify(mockBackend) + .statusUpdate(meq(0L), meq(TaskState.RUNNING), statusCaptor.capture()) + orderedMock.verify(mockBackend) + .statusUpdate(meq(0L), meq(TaskState.FAILED), statusCaptor.capture()) + // first statusUpdate for RUNNING has empty data + assert(statusCaptor.getAllValues().get(0).remaining() === 0) + // second update is more interesting + val failureData = statusCaptor.getAllValues.get(1) + val failReason = + SparkEnv.get.closureSerializer.newInstance().deserialize[TaskFailedReason](failureData) + (failReason, mockUncaughtExceptionHandler) + } +} + +class FetchFailureThrowingRDD(sc: SparkContext) extends RDD[Int](sc, Nil) { + override def compute(split: Partition, context: TaskContext): Iterator[Int] = { + new Iterator[Int] { + override def hasNext: Boolean = true + override def next(): Int = { + throw new FetchFailedException( + bmAddress = BlockManagerId("1", "hostA", 1234), + shuffleId = 0, + mapId = 0, + reduceId = 0, + message = "fake fetch failure" + ) + } + } + } + override protected def getPartitions: Array[Partition] = { + Array(new SimplePartition) + } +} + +class SimplePartition extends Partition { + override def index: Int = 0 +} + +class FetchFailureHidingRDD( + sc: SparkContext, + val input: FetchFailureThrowingRDD, + throwOOM: Boolean) extends RDD[Int](input) { + override def compute(split: Partition, context: TaskContext): Iterator[Int] = { + val inItr = input.compute(split, context) + try { + Iterator(inItr.size) + } catch { + case t: Throwable => + if (throwOOM) { + throw new OutOfMemoryError("OOM while handling another exception") + } else { + throw new RuntimeException("User Exception that hides the original exception", t) + } + } + } + + override protected def getPartitions: Array[Partition] = { + Array(new SimplePartition) + } +} + +// Helps to test("SPARK-15963") +private class ExecutorSuiteHelper { + + val latch1 = new CountDownLatch(1) + val latch2 = new CountDownLatch(1) + val latch3 = new CountDownLatch(1) + + @volatile var taskState: TaskState = _ + @volatile var testFailedReason: TaskFailedReason = _ +} + +private class NonDeserializableTask extends FakeTask(0, 0) with Externalizable { + def writeExternal(out: ObjectOutput): Unit = {} + def readExternal(in: ObjectInput): Unit = { + throw new RuntimeException(NonDeserializableTask.errorMsg) + } +} + +private object NonDeserializableTask { + val errorMsg = "failure in deserialization" +} diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index 8275fd87764c..7bcc2fb5231d 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -17,12 +17,231 @@ package org.apache.spark.executor -import org.apache.spark.SparkFunSuite +import org.scalatest.Assertions + +import org.apache.spark._ +import org.apache.spark.storage.{BlockStatus, StorageLevel, TestBlockId} +import org.apache.spark.util.AccumulatorV2 + class TaskMetricsSuite extends SparkFunSuite { - test("[SPARK-5701] updateShuffleReadMetrics: ShuffleReadMetrics not added when no shuffle deps") { - val taskMetrics = new TaskMetrics() - taskMetrics.updateShuffleReadMetrics() - assert(taskMetrics.shuffleReadMetrics.isEmpty) + import StorageLevel._ + + test("mutating values") { + val tm = new TaskMetrics + assert(tm.executorDeserializeTime == 0L) + assert(tm.executorRunTime == 0L) + assert(tm.resultSize == 0L) + assert(tm.jvmGCTime == 0L) + assert(tm.resultSerializationTime == 0L) + assert(tm.memoryBytesSpilled == 0L) + assert(tm.diskBytesSpilled == 0L) + assert(tm.peakExecutionMemory == 0L) + assert(tm.updatedBlockStatuses.isEmpty) + // set or increment values + tm.setExecutorDeserializeTime(100L) + tm.setExecutorDeserializeTime(1L) // overwrite + tm.setExecutorRunTime(200L) + tm.setExecutorRunTime(2L) + tm.setResultSize(300L) + tm.setResultSize(3L) + tm.setJvmGCTime(400L) + tm.setJvmGCTime(4L) + tm.setResultSerializationTime(500L) + tm.setResultSerializationTime(5L) + tm.incMemoryBytesSpilled(600L) + tm.incMemoryBytesSpilled(6L) // add + tm.incDiskBytesSpilled(700L) + tm.incDiskBytesSpilled(7L) + tm.incPeakExecutionMemory(800L) + tm.incPeakExecutionMemory(8L) + val block1 = (TestBlockId("a"), BlockStatus(MEMORY_ONLY, 1L, 2L)) + val block2 = (TestBlockId("b"), BlockStatus(MEMORY_ONLY, 3L, 4L)) + tm.incUpdatedBlockStatuses(block1) + tm.incUpdatedBlockStatuses(block2) + // assert new values exist + assert(tm.executorDeserializeTime == 1L) + assert(tm.executorRunTime == 2L) + assert(tm.resultSize == 3L) + assert(tm.jvmGCTime == 4L) + assert(tm.resultSerializationTime == 5L) + assert(tm.memoryBytesSpilled == 606L) + assert(tm.diskBytesSpilled == 707L) + assert(tm.peakExecutionMemory == 808L) + assert(tm.updatedBlockStatuses == Seq(block1, block2)) + } + + test("mutating shuffle read metrics values") { + val tm = new TaskMetrics + val sr = tm.shuffleReadMetrics + // initial values + assert(sr.remoteBlocksFetched == 0) + assert(sr.localBlocksFetched == 0) + assert(sr.remoteBytesRead == 0L) + assert(sr.localBytesRead == 0L) + assert(sr.fetchWaitTime == 0L) + assert(sr.recordsRead == 0L) + // set and increment values + sr.setRemoteBlocksFetched(100) + sr.setRemoteBlocksFetched(10) + sr.incRemoteBlocksFetched(1) // 10 + 1 + sr.incRemoteBlocksFetched(1) // 10 + 1 + 1 + sr.setLocalBlocksFetched(200) + sr.setLocalBlocksFetched(20) + sr.incLocalBlocksFetched(2) + sr.incLocalBlocksFetched(2) + sr.setRemoteBytesRead(300L) + sr.setRemoteBytesRead(30L) + sr.incRemoteBytesRead(3L) + sr.incRemoteBytesRead(3L) + sr.setRemoteBytesReadToDisk(10L) + sr.incRemoteBytesReadToDisk(8L) + sr.setLocalBytesRead(400L) + sr.setLocalBytesRead(40L) + sr.incLocalBytesRead(4L) + sr.incLocalBytesRead(4L) + sr.setFetchWaitTime(500L) + sr.setFetchWaitTime(50L) + sr.incFetchWaitTime(5L) + sr.incFetchWaitTime(5L) + sr.setRecordsRead(600L) + sr.setRecordsRead(60L) + sr.incRecordsRead(6L) + sr.incRecordsRead(6L) + // assert new values exist + assert(sr.remoteBlocksFetched == 12) + assert(sr.localBlocksFetched == 24) + assert(sr.remoteBytesRead == 36L) + assert(sr.remoteBytesReadToDisk == 18L) + assert(sr.localBytesRead == 48L) + assert(sr.fetchWaitTime == 60L) + assert(sr.recordsRead == 72L) + } + + test("mutating shuffle write metrics values") { + val tm = new TaskMetrics + val sw = tm.shuffleWriteMetrics + // initial values + assert(sw.bytesWritten == 0L) + assert(sw.recordsWritten == 0L) + assert(sw.writeTime == 0L) + // increment and decrement values + sw.incBytesWritten(100L) + sw.incBytesWritten(10L) // 100 + 10 + sw.decBytesWritten(1L) // 100 + 10 - 1 + sw.decBytesWritten(1L) // 100 + 10 - 1 - 1 + sw.incRecordsWritten(200L) + sw.incRecordsWritten(20L) + sw.decRecordsWritten(2L) + sw.decRecordsWritten(2L) + sw.incWriteTime(300L) + sw.incWriteTime(30L) + // assert new values exist + assert(sw.bytesWritten == 108L) + assert(sw.recordsWritten == 216L) + assert(sw.writeTime == 330L) + } + + test("mutating input metrics values") { + val tm = new TaskMetrics + val in = tm.inputMetrics + // initial values + assert(in.bytesRead == 0L) + assert(in.recordsRead == 0L) + // set and increment values + in.setBytesRead(1L) + in.setBytesRead(2L) + in.incRecordsRead(1L) + in.incRecordsRead(2L) + // assert new values exist + assert(in.bytesRead == 2L) + assert(in.recordsRead == 3L) + } + + test("mutating output metrics values") { + val tm = new TaskMetrics + val out = tm.outputMetrics + // initial values + assert(out.bytesWritten == 0L) + assert(out.recordsWritten == 0L) + // set values + out.setBytesWritten(1L) + out.setBytesWritten(2L) + out.setRecordsWritten(3L) + out.setRecordsWritten(4L) + // assert new values exist + assert(out.bytesWritten == 2L) + assert(out.recordsWritten == 4L) + } + + test("merging multiple shuffle read metrics") { + val tm = new TaskMetrics + val sr1 = tm.createTempShuffleReadMetrics() + val sr2 = tm.createTempShuffleReadMetrics() + val sr3 = tm.createTempShuffleReadMetrics() + sr1.incRecordsRead(10L) + sr2.incRecordsRead(10L) + sr1.incFetchWaitTime(1L) + sr2.incFetchWaitTime(2L) + sr3.incFetchWaitTime(3L) + tm.mergeShuffleReadMetrics() + assert(tm.shuffleReadMetrics.remoteBlocksFetched === 0L) + assert(tm.shuffleReadMetrics.recordsRead === 20L) + assert(tm.shuffleReadMetrics.fetchWaitTime === 6L) + + // SPARK-5701: calling merge without any shuffle deps does nothing + val tm2 = new TaskMetrics + tm2.mergeShuffleReadMetrics() + } + + test("additional accumulables") { + val tm = TaskMetrics.empty + val acc1 = AccumulatorSuite.createLongAccum("a") + val acc2 = AccumulatorSuite.createLongAccum("b") + val acc3 = AccumulatorSuite.createLongAccum("c") + val acc4 = AccumulatorSuite.createLongAccum("d", true) + tm.registerAccumulator(acc1) + tm.registerAccumulator(acc2) + tm.registerAccumulator(acc3) + tm.registerAccumulator(acc4) + acc1.add(1) + acc2.add(2) + val newUpdates = tm.accumulators() + .map(a => (a.id, a.asInstanceOf[AccumulatorV2[Any, Any]])).toMap + assert(newUpdates.contains(acc1.id)) + assert(newUpdates.contains(acc2.id)) + assert(newUpdates.contains(acc3.id)) + assert(newUpdates.contains(acc4.id)) + assert(newUpdates(acc1.id).name === Some("a")) + assert(newUpdates(acc2.id).name === Some("b")) + assert(newUpdates(acc3.id).name === Some("c")) + assert(newUpdates(acc4.id).name === Some("d")) + assert(newUpdates(acc1.id).value === 1) + assert(newUpdates(acc2.id).value === 2) + assert(newUpdates(acc3.id).value === 0) + assert(newUpdates(acc4.id).value === 0) + assert(!newUpdates(acc3.id).countFailedValues) + assert(newUpdates(acc4.id).countFailedValues) + assert(newUpdates.size === tm.internalAccums.size + 4) + } +} + + +private[spark] object TaskMetricsSuite extends Assertions { + + /** + * Assert that two lists of accumulator updates are equal. + * Note: this does NOT check accumulator ID equality. + */ + def assertUpdatesEquals( + updates1: Seq[AccumulatorV2[_, _]], + updates2: Seq[AccumulatorV2[_, _]]): Unit = { + assert(updates1.size === updates2.size) + updates1.zip(updates2).foreach { case (acc1, acc2) => + // do not assert ID equals here + assert(acc1.name === acc2.name) + assert(acc1.countFailedValues === acc2.countFailedValues) + assert(acc1.value == acc2.value) + } } } 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 8a199459c1dd..ddf73d637063 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -23,13 +23,13 @@ import java.io.FileOutputStream import scala.collection.immutable.IndexedSeq -import org.scalatest.BeforeAndAfterAll - import org.apache.hadoop.io.Text +import org.apache.hadoop.io.compress.{CompressionCodecFactory, GzipCodec} +import org.scalatest.BeforeAndAfterAll -import org.apache.spark.{Logging, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils -import org.apache.hadoop.io.compress.{DefaultCodec, CompressionCodecFactory, GzipCodec} /** * Tests the correctness of @@ -47,6 +47,7 @@ class WholeTextFileRecordReaderSuite extends SparkFunSuite with BeforeAndAfterAl // 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. + super.beforeAll() val conf = new SparkConf().set("spark.hadoop.fs.file.impl.disable.cache", "true") sc = new SparkContext("local", "test", conf) @@ -59,7 +60,11 @@ class WholeTextFileRecordReaderSuite extends SparkFunSuite with BeforeAndAfterAl } override def afterAll() { - sc.stop() + try { + sc.stop() + } finally { + super.afterAll() + } } private def createNativeFile(inputDir: File, fileName: String, contents: Array[Byte], diff --git a/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala b/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala new file mode 100644 index 000000000000..bf08276dbf97 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala @@ -0,0 +1,291 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.config + +import java.util.Locale +import java.util.concurrent.TimeUnit + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.network.util.ByteUnit +import org.apache.spark.util.SparkConfWithEnv + +class ConfigEntrySuite extends SparkFunSuite { + + private val PREFIX = "spark.ConfigEntrySuite" + + private def testKey(name: String): String = s"$PREFIX.$name" + + test("conf entry: int") { + val conf = new SparkConf() + val iConf = ConfigBuilder(testKey("int")).intConf.createWithDefault(1) + assert(conf.get(iConf) === 1) + conf.set(iConf, 2) + assert(conf.get(iConf) === 2) + } + + test("conf entry: long") { + val conf = new SparkConf() + val lConf = ConfigBuilder(testKey("long")).longConf.createWithDefault(0L) + conf.set(lConf, 1234L) + assert(conf.get(lConf) === 1234L) + } + + test("conf entry: double") { + val conf = new SparkConf() + val dConf = ConfigBuilder(testKey("double")).doubleConf.createWithDefault(0.0) + conf.set(dConf, 20.0) + assert(conf.get(dConf) === 20.0) + } + + test("conf entry: boolean") { + val conf = new SparkConf() + val bConf = ConfigBuilder(testKey("boolean")).booleanConf.createWithDefault(false) + assert(!conf.get(bConf)) + conf.set(bConf, true) + assert(conf.get(bConf)) + } + + test("conf entry: optional") { + val conf = new SparkConf() + val optionalConf = ConfigBuilder(testKey("optional")).intConf.createOptional + assert(conf.get(optionalConf) === None) + conf.set(optionalConf, 1) + assert(conf.get(optionalConf) === Some(1)) + } + + test("conf entry: fallback") { + val conf = new SparkConf() + val parentConf = ConfigBuilder(testKey("parent")).intConf.createWithDefault(1) + val confWithFallback = ConfigBuilder(testKey("fallback")).fallbackConf(parentConf) + assert(conf.get(confWithFallback) === 1) + conf.set(confWithFallback, 2) + assert(conf.get(parentConf) === 1) + assert(conf.get(confWithFallback) === 2) + } + + test("conf entry: time") { + val conf = new SparkConf() + val time = ConfigBuilder(testKey("time")).timeConf(TimeUnit.SECONDS) + .createWithDefaultString("1h") + assert(conf.get(time) === 3600L) + conf.set(time.key, "1m") + assert(conf.get(time) === 60L) + } + + test("conf entry: bytes") { + val conf = new SparkConf() + val bytes = ConfigBuilder(testKey("bytes")).bytesConf(ByteUnit.KiB) + .createWithDefaultString("1m") + assert(conf.get(bytes) === 1024L) + conf.set(bytes.key, "1k") + assert(conf.get(bytes) === 1L) + } + + test("conf entry: regex") { + val conf = new SparkConf() + val rConf = ConfigBuilder(testKey("regex")).regexConf.createWithDefault(".*".r) + + conf.set(rConf, "[0-9a-f]{8}".r) + assert(conf.get(rConf).toString === "[0-9a-f]{8}") + + conf.set(rConf.key, "[0-9a-f]{4}") + assert(conf.get(rConf).toString === "[0-9a-f]{4}") + + conf.set(rConf.key, "[.") + val e = intercept[IllegalArgumentException](conf.get(rConf)) + assert(e.getMessage.contains("regex should be a regex, but was")) + } + + test("conf entry: string seq") { + val conf = new SparkConf() + val seq = ConfigBuilder(testKey("seq")).stringConf.toSequence.createWithDefault(Seq()) + conf.set(seq.key, "1,,2, 3 , , 4") + assert(conf.get(seq) === Seq("1", "2", "3", "4")) + conf.set(seq, Seq("1", "2")) + assert(conf.get(seq) === Seq("1", "2")) + } + + test("conf entry: int seq") { + val conf = new SparkConf() + val seq = ConfigBuilder(testKey("intSeq")).intConf.toSequence.createWithDefault(Seq()) + conf.set(seq.key, "1,,2, 3 , , 4") + assert(conf.get(seq) === Seq(1, 2, 3, 4)) + conf.set(seq, Seq(1, 2)) + assert(conf.get(seq) === Seq(1, 2)) + } + + test("conf entry: transformation") { + val conf = new SparkConf() + val transformationConf = ConfigBuilder(testKey("transformation")) + .stringConf + .transform(_.toLowerCase(Locale.ROOT)) + .createWithDefault("FOO") + + assert(conf.get(transformationConf) === "foo") + conf.set(transformationConf, "BAR") + assert(conf.get(transformationConf) === "bar") + } + + test("conf entry: checkValue()") { + def createEntry(default: Int): ConfigEntry[Int] = + ConfigBuilder(testKey("checkValue")) + .intConf + .checkValue(value => value >= 0, "value must be non-negative") + .createWithDefault(default) + + val conf = new SparkConf() + + val entry = createEntry(10) + conf.set(entry, -1) + val e1 = intercept[IllegalArgumentException] { + conf.get(entry) + } + assert(e1.getMessage == "value must be non-negative") + + val e2 = intercept[IllegalArgumentException] { + createEntry(-1) + } + assert(e2.getMessage == "value must be non-negative") + } + + test("conf entry: valid values check") { + val conf = new SparkConf() + val enum = ConfigBuilder(testKey("enum")) + .stringConf + .checkValues(Set("a", "b", "c")) + .createWithDefault("a") + assert(conf.get(enum) === "a") + + conf.set(enum, "b") + assert(conf.get(enum) === "b") + + conf.set(enum, "d") + val enumError = intercept[IllegalArgumentException] { + conf.get(enum) + } + assert(enumError.getMessage === s"The value of ${enum.key} should be one of a, b, c, but was d") + } + + test("conf entry: conversion error") { + val conf = new SparkConf() + val conversionTest = ConfigBuilder(testKey("conversionTest")).doubleConf.createOptional + conf.set(conversionTest.key, "abc") + val conversionError = intercept[IllegalArgumentException] { + conf.get(conversionTest) + } + assert(conversionError.getMessage === s"${conversionTest.key} should be double, but was abc") + } + + test("default value handling is null-safe") { + val conf = new SparkConf() + val stringConf = ConfigBuilder(testKey("string")).stringConf.createWithDefault(null) + assert(conf.get(stringConf) === null) + } + + test("variable expansion of spark config entries") { + val env = Map("ENV1" -> "env1") + val conf = new SparkConfWithEnv(env) + + val stringConf = ConfigBuilder(testKey("stringForExpansion")) + .stringConf + .createWithDefault("string1") + val optionalConf = ConfigBuilder(testKey("optionForExpansion")) + .stringConf + .createOptional + val intConf = ConfigBuilder(testKey("intForExpansion")) + .intConf + .createWithDefault(42) + val fallbackConf = ConfigBuilder(testKey("fallbackForExpansion")) + .fallbackConf(intConf) + + val refConf = ConfigBuilder(testKey("configReferenceTest")) + .stringConf + .createWithDefault(null) + + def ref(entry: ConfigEntry[_]): String = "${" + entry.key + "}" + + def testEntryRef(entry: ConfigEntry[_], expected: String): Unit = { + conf.set(refConf, ref(entry)) + assert(conf.get(refConf) === expected) + } + + testEntryRef(stringConf, "string1") + testEntryRef(intConf, "42") + testEntryRef(fallbackConf, "42") + + testEntryRef(optionalConf, ref(optionalConf)) + + conf.set(optionalConf, ref(stringConf)) + testEntryRef(optionalConf, "string1") + + conf.set(optionalConf, ref(fallbackConf)) + testEntryRef(optionalConf, "42") + + // Default string values with variable references. + val parameterizedStringConf = ConfigBuilder(testKey("stringWithParams")) + .stringConf + .createWithDefault(ref(stringConf)) + assert(conf.get(parameterizedStringConf) === conf.get(stringConf)) + + // Make sure SparkConf's env override works. + conf.set(refConf, "${env:ENV1}") + assert(conf.get(refConf) === env("ENV1")) + + // Conf with null default value is not expanded. + val nullConf = ConfigBuilder(testKey("nullString")) + .stringConf + .createWithDefault(null) + testEntryRef(nullConf, ref(nullConf)) + } + + test("conf entry : default function") { + var data = 0 + val conf = new SparkConf() + val iConf = ConfigBuilder(testKey("intval")).intConf.createWithDefaultFunction(() => data) + assert(conf.get(iConf) === 0) + data = 2 + assert(conf.get(iConf) === 2) + } + + test("conf entry: alternative keys") { + val conf = new SparkConf() + val iConf = ConfigBuilder(testKey("a")) + .withAlternative(testKey("b")) + .withAlternative(testKey("c")) + .intConf.createWithDefault(0) + + // no key is set, return default value. + assert(conf.get(iConf) === 0) + + // the primary key is set, the alternative keys are not set, return the value of primary key. + conf.set(testKey("a"), "1") + assert(conf.get(iConf) === 1) + + // the primary key and alternative keys are all set, return the value of primary key. + conf.set(testKey("b"), "2") + conf.set(testKey("c"), "3") + assert(conf.get(iConf) === 1) + + // the primary key is not set, (some of) the alternative keys are set, return the value of the + // first alternative key that is set. + conf.remove(testKey("a")) + assert(conf.get(iConf) === 2) + conf.remove(testKey("b")) + assert(conf.get(iConf) === 3) + } +} diff --git a/core/src/test/scala/org/apache/spark/internal/config/ConfigReaderSuite.scala b/core/src/test/scala/org/apache/spark/internal/config/ConfigReaderSuite.scala new file mode 100644 index 000000000000..be57cc34e450 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/internal/config/ConfigReaderSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.config + +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite + +class ConfigReaderSuite extends SparkFunSuite { + + test("variable expansion") { + val env = Map("ENV1" -> "env1") + val conf = Map("key1" -> "value1", "key2" -> "value2") + + val reader = new ConfigReader(conf.asJava) + reader.bindEnv(new MapProvider(env.asJava)) + + assert(reader.substitute(null) === null) + assert(reader.substitute("${key1}") === "value1") + assert(reader.substitute("key1 is: ${key1}") === "key1 is: value1") + assert(reader.substitute("${key1} ${key2}") === "value1 value2") + assert(reader.substitute("${key3}") === "${key3}") + assert(reader.substitute("${env:ENV1}") === "env1") + assert(reader.substitute("${system:user.name}") === sys.props("user.name")) + assert(reader.substitute("${key1") === "${key1") + + // Unknown prefixes. + assert(reader.substitute("${unknown:value}") === "${unknown:value}") + } + + test("circular references") { + val conf = Map("key1" -> "${key2}", "key2" -> "${key1}") + val reader = new ConfigReader(conf.asJava) + val e = intercept[IllegalArgumentException] { + reader.substitute("${key1}") + } + assert(e.getMessage().contains("Circular")) + } + + test("spark conf provider filters config keys") { + val conf = Map("nonspark.key" -> "value", "spark.key" -> "value") + val reader = new ConfigReader(new SparkConfigProvider(conf.asJava)) + assert(reader.get("nonspark.key") === None) + assert(reader.get("spark.key") === Some("value")) + } + +} diff --git a/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala new file mode 100644 index 000000000000..3b798e36b049 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.io + +import java.nio.ByteBuffer + +import com.google.common.io.ByteStreams + +import org.apache.spark.SparkFunSuite +import org.apache.spark.network.util.ByteArrayWritableChannel +import org.apache.spark.util.io.ChunkedByteBuffer + +class ChunkedByteBufferSuite extends SparkFunSuite { + + test("no chunks") { + val emptyChunkedByteBuffer = new ChunkedByteBuffer(Array.empty[ByteBuffer]) + assert(emptyChunkedByteBuffer.size === 0) + assert(emptyChunkedByteBuffer.getChunks().isEmpty) + assert(emptyChunkedByteBuffer.toArray === Array.empty) + assert(emptyChunkedByteBuffer.toByteBuffer.capacity() === 0) + assert(emptyChunkedByteBuffer.toNetty.capacity() === 0) + emptyChunkedByteBuffer.toInputStream(dispose = false).close() + emptyChunkedByteBuffer.toInputStream(dispose = true).close() + } + + test("getChunks() duplicates chunks") { + val chunkedByteBuffer = new ChunkedByteBuffer(Array(ByteBuffer.allocate(8))) + chunkedByteBuffer.getChunks().head.position(4) + assert(chunkedByteBuffer.getChunks().head.position() === 0) + } + + test("copy() does not affect original buffer's position") { + val chunkedByteBuffer = new ChunkedByteBuffer(Array(ByteBuffer.allocate(8))) + chunkedByteBuffer.copy(ByteBuffer.allocate) + assert(chunkedByteBuffer.getChunks().head.position() === 0) + } + + test("writeFully() does not affect original buffer's position") { + val chunkedByteBuffer = new ChunkedByteBuffer(Array(ByteBuffer.allocate(8))) + chunkedByteBuffer.writeFully(new ByteArrayWritableChannel(chunkedByteBuffer.size.toInt)) + assert(chunkedByteBuffer.getChunks().head.position() === 0) + } + + test("toArray()") { + val empty = ByteBuffer.wrap(Array.empty[Byte]) + val bytes = ByteBuffer.wrap(Array.tabulate(8)(_.toByte)) + val chunkedByteBuffer = new ChunkedByteBuffer(Array(bytes, bytes, empty)) + assert(chunkedByteBuffer.toArray === bytes.array() ++ bytes.array()) + } + + test("toArray() throws UnsupportedOperationException if size exceeds 2GB") { + val fourMegabyteBuffer = ByteBuffer.allocate(1024 * 1024 * 4) + fourMegabyteBuffer.limit(fourMegabyteBuffer.capacity()) + val chunkedByteBuffer = new ChunkedByteBuffer(Array.fill(1024)(fourMegabyteBuffer)) + assert(chunkedByteBuffer.size === (1024L * 1024L * 1024L * 4L)) + intercept[UnsupportedOperationException] { + chunkedByteBuffer.toArray + } + } + + test("toInputStream()") { + val empty = ByteBuffer.wrap(Array.empty[Byte]) + val bytes1 = ByteBuffer.wrap(Array.tabulate(256)(_.toByte)) + val bytes2 = ByteBuffer.wrap(Array.tabulate(128)(_.toByte)) + val chunkedByteBuffer = new ChunkedByteBuffer(Array(empty, bytes1, bytes2)) + assert(chunkedByteBuffer.size === bytes1.limit() + bytes2.limit()) + + val inputStream = chunkedByteBuffer.toInputStream(dispose = false) + val bytesFromStream = new Array[Byte](chunkedByteBuffer.size.toInt) + ByteStreams.readFully(inputStream, bytesFromStream) + assert(bytesFromStream === bytes1.array() ++ bytes2.array()) + assert(chunkedByteBuffer.getChunks().head.position() === 0) + } +} diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala index 1553ab60bdda..9e9c2b0165e1 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -46,7 +46,7 @@ class CompressionCodecSuite extends SparkFunSuite { test("default compression codec") { val codec = CompressionCodec.createCodec(conf) - assert(codec.getClass === classOf[SnappyCompressionCodec]) + assert(codec.getClass === classOf[LZ4CompressionCodec]) testCodec(codec) } @@ -62,12 +62,10 @@ class CompressionCodecSuite extends SparkFunSuite { testCodec(codec) } - test("lz4 does not support concatenation of serialized streams") { + test("lz4 supports concatenation of serialized streams") { val codec = CompressionCodec.createCodec(conf, classOf[LZ4CompressionCodec].getName) assert(codec.getClass === classOf[LZ4CompressionCodec]) - intercept[Exception] { - testConcatenationOfSerializedStreams(codec) - } + testConcatenationOfSerializedStreams(codec) } test("lzf compression codec") { diff --git a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala index 639d1daa36c7..c88cc13654ce 100644 --- a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala @@ -26,7 +26,7 @@ import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ import org.apache.spark._ -import org.apache.spark.launcher._ +import org.apache.spark.util.Utils class LauncherBackendSuite extends SparkFunSuite with Matchers { @@ -36,6 +36,8 @@ class LauncherBackendSuite extends SparkFunSuite with Matchers { tests.foreach { case (name, master) => test(s"$name: launcher handle") { + // The tests here are failed due to the cmd length limitation up to 8K on Windows. + assume(!Utils.isWindows) testWithMaster(master) } } @@ -49,7 +51,7 @@ class LauncherBackendSuite extends SparkFunSuite with Matchers { .setConf("spark.ui.enabled", "false") .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, s"-Dtest.appender=console") .setMaster(master) - .setAppResource("spark-internal") + .setAppResource(SparkLauncher.NO_RESOURCE) .setMainClass(TestApp.getClass.getName().stripSuffix("$")) .startApplication() diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index 4a9479cf490f..85eeb5055ae0 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -19,72 +19,82 @@ package org.apache.spark.memory import java.util.concurrent.atomic.AtomicLong +import scala.collection.mutable +import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.Duration -import scala.concurrent.{Await, ExecutionContext, Future} import org.mockito.Matchers.{any, anyLong} -import org.mockito.Mockito.{mock, when} +import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfterEach import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkFunSuite -import org.apache.spark.storage.MemoryStore +import org.apache.spark.storage.{BlockId, BlockStatus, StorageLevel} +import org.apache.spark.storage.memory.MemoryStore +import org.apache.spark.util.ThreadUtils /** * Helper trait for sharing code among [[MemoryManager]] tests. */ -private[memory] trait MemoryManagerSuite extends SparkFunSuite { +private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAfterEach { - import MemoryManagerSuite.DEFAULT_ENSURE_FREE_SPACE_CALLED + protected val evictedBlocks = new mutable.ArrayBuffer[(BlockId, BlockStatus)] + + import MemoryManagerSuite.DEFAULT_EVICT_BLOCKS_TO_FREE_SPACE_CALLED // Note: Mockito's verify mechanism does not provide a way to reset method call counts // without also resetting stubbed methods. Since our test code relies on the latter, - // we need to use our own variable to track invocations of `ensureFreeSpace`. + // we need to use our own variable to track invocations of `evictBlocksToFreeSpace`. /** - * The amount of free space requested in the last call to [[MemoryStore.ensureFreeSpace]] + * The amount of space requested in the last call to [[MemoryStore.evictBlocksToFreeSpace]]. * - * This set whenever [[MemoryStore.ensureFreeSpace]] is called, and cleared when the test - * code makes explicit assertions on this variable through [[assertEnsureFreeSpaceCalled]]. + * This set whenever [[MemoryStore.evictBlocksToFreeSpace]] is called, and cleared when the test + * code makes explicit assertions on this variable through + * [[assertEvictBlocksToFreeSpaceCalled]]. */ - private val ensureFreeSpaceCalled = new AtomicLong(DEFAULT_ENSURE_FREE_SPACE_CALLED) + private val evictBlocksToFreeSpaceCalled = new AtomicLong(0) + + override def beforeEach(): Unit = { + super.beforeEach() + evictedBlocks.clear() + evictBlocksToFreeSpaceCalled.set(DEFAULT_EVICT_BLOCKS_TO_FREE_SPACE_CALLED) + } /** - * Make a mocked [[MemoryStore]] whose [[MemoryStore.ensureFreeSpace]] method is stubbed. + * Make a mocked [[MemoryStore]] whose [[MemoryStore.evictBlocksToFreeSpace]] method is stubbed. * - * This allows our test code to release storage memory when [[MemoryStore.ensureFreeSpace]] - * is called without relying on [[org.apache.spark.storage.BlockManager]] and all of its - * dependencies. + * This allows our test code to release storage memory when these methods are called + * without relying on [[org.apache.spark.storage.BlockManager]] and all of its dependencies. */ protected def makeMemoryStore(mm: MemoryManager): MemoryStore = { - val ms = mock(classOf[MemoryStore]) - when(ms.ensureFreeSpace(anyLong(), any())).thenAnswer(ensureFreeSpaceAnswer(mm, 0)) - when(ms.ensureFreeSpace(any(), anyLong(), any())).thenAnswer(ensureFreeSpaceAnswer(mm, 1)) + val ms = mock(classOf[MemoryStore], RETURNS_SMART_NULLS) + when(ms.evictBlocksToFreeSpace(any(), anyLong(), any())) + .thenAnswer(evictBlocksToFreeSpaceAnswer(mm)) mm.setMemoryStore(ms) ms } /** - * Make an [[Answer]] that stubs [[MemoryStore.ensureFreeSpace]] with the right arguments. + * Make a mocked [[MemoryStore]] whose [[MemoryStore.evictBlocksToFreeSpace]] method is + * stubbed to always throw [[RuntimeException]]. */ - private def ensureFreeSpaceAnswer(mm: MemoryManager, numBytesPos: Int): Answer[Boolean] = { - new Answer[Boolean] { - override def answer(invocation: InvocationOnMock): Boolean = { - val args = invocation.getArguments - require(args.size > numBytesPos, s"bad test: expected >$numBytesPos arguments " + - s"in ensureFreeSpace, found ${args.size}") - require(args(numBytesPos).isInstanceOf[Long], s"bad test: expected ensureFreeSpace " + - s"argument at index $numBytesPos to be a Long: ${args.mkString(", ")}") - val numBytes = args(numBytesPos).asInstanceOf[Long] - mockEnsureFreeSpace(mm, numBytes) + protected def makeBadMemoryStore(mm: MemoryManager): MemoryStore = { + val ms = mock(classOf[MemoryStore], RETURNS_SMART_NULLS) + when(ms.evictBlocksToFreeSpace(any(), anyLong(), any())).thenAnswer(new Answer[Long] { + override def answer(invocation: InvocationOnMock): Long = { + throw new RuntimeException("bad memory store!") } - } + }) + mm.setMemoryStore(ms) + ms } /** - * Simulate the part of [[MemoryStore.ensureFreeSpace]] that releases storage memory. + * Simulate the part of [[MemoryStore.evictBlocksToFreeSpace]] that releases storage memory. * * This is a significant simplification of the real method, which actually drops existing * blocks based on the size of each block. Instead, here we simply release as many bytes @@ -92,156 +102,172 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { * test without relying on the [[org.apache.spark.storage.BlockManager]], which brings in * many other dependencies. * - * Every call to this method will set a global variable, [[ensureFreeSpaceCalled]], that + * Every call to this method will set a global variable, [[evictBlocksToFreeSpaceCalled]], that * records the number of bytes this is called with. This variable is expected to be cleared - * by the test code later through [[assertEnsureFreeSpaceCalled]]. + * by the test code later through [[assertEvictBlocksToFreeSpaceCalled]]. */ - private def mockEnsureFreeSpace(mm: MemoryManager, numBytes: Long): Boolean = mm.synchronized { - require(ensureFreeSpaceCalled.get() === DEFAULT_ENSURE_FREE_SPACE_CALLED, - "bad test: ensure free space variable was not reset") - // Record the number of bytes we freed this call - ensureFreeSpaceCalled.set(numBytes) - if (numBytes <= mm.maxStorageMemory) { - def freeMemory = mm.maxStorageMemory - mm.storageMemoryUsed - val spaceToRelease = numBytes - freeMemory - if (spaceToRelease > 0) { - mm.releaseStorageMemory(spaceToRelease) + private def evictBlocksToFreeSpaceAnswer(mm: MemoryManager): Answer[Long] = { + new Answer[Long] { + override def answer(invocation: InvocationOnMock): Long = { + val args = invocation.getArguments + val numBytesToFree = args(1).asInstanceOf[Long] + assert(numBytesToFree > 0) + require(evictBlocksToFreeSpaceCalled.get() === DEFAULT_EVICT_BLOCKS_TO_FREE_SPACE_CALLED, + "bad test: evictBlocksToFreeSpace() variable was not reset") + evictBlocksToFreeSpaceCalled.set(numBytesToFree) + if (numBytesToFree <= mm.storageMemoryUsed) { + // We can evict enough blocks to fulfill the request for space + mm.releaseStorageMemory(numBytesToFree, mm.tungstenMemoryMode) + evictedBlocks += Tuple2(null, BlockStatus(StorageLevel.MEMORY_ONLY, numBytesToFree, 0L)) + numBytesToFree + } else { + // No blocks were evicted because eviction would not free enough space. + 0L + } } - freeMemory >= numBytes - } else { - // We attempted to free more bytes than our max allowable memory - false } } /** - * Assert that [[MemoryStore.ensureFreeSpace]] is called with the given parameters. + * Assert that [[MemoryStore.evictBlocksToFreeSpace]] is called with the given parameters. */ - protected def assertEnsureFreeSpaceCalled(ms: MemoryStore, numBytes: Long): Unit = { - assert(ensureFreeSpaceCalled.get() === numBytes, - s"expected ensure free space to be called with $numBytes") - ensureFreeSpaceCalled.set(DEFAULT_ENSURE_FREE_SPACE_CALLED) + protected def assertEvictBlocksToFreeSpaceCalled(ms: MemoryStore, numBytes: Long): Unit = { + assert(evictBlocksToFreeSpaceCalled.get() === numBytes, + s"expected evictBlocksToFreeSpace() to be called with $numBytes") + evictBlocksToFreeSpaceCalled.set(DEFAULT_EVICT_BLOCKS_TO_FREE_SPACE_CALLED) } /** - * Assert that [[MemoryStore.ensureFreeSpace]] is NOT called. + * Assert that [[MemoryStore.evictBlocksToFreeSpace]] is NOT called. */ - protected def assertEnsureFreeSpaceNotCalled[T](ms: MemoryStore): Unit = { - assert(ensureFreeSpaceCalled.get() === DEFAULT_ENSURE_FREE_SPACE_CALLED, - "ensure free space should not have been called!") + protected def assertEvictBlocksToFreeSpaceNotCalled[T](ms: MemoryStore): Unit = { + assert(evictBlocksToFreeSpaceCalled.get() === DEFAULT_EVICT_BLOCKS_TO_FREE_SPACE_CALLED, + "evictBlocksToFreeSpace() should not have been called!") + assert(evictedBlocks.isEmpty) } /** - * Create a MemoryManager with the specified execution memory limit and no storage memory. + * Create a MemoryManager with the specified execution memory limits and no storage memory. */ - protected def createMemoryManager(maxExecutionMemory: Long): MemoryManager + protected def createMemoryManager( + maxOnHeapExecutionMemory: Long, + maxOffHeapExecutionMemory: Long = 0L): MemoryManager // -- Tests of sharing of execution memory between tasks ---------------------------------------- // Prior to Spark 1.6, these tests were part of ShuffleMemoryManagerSuite. implicit val ec = ExecutionContext.global - test("single task requesting execution memory") { + test("single task requesting on-heap execution memory") { val manager = createMemoryManager(1000L) val taskMemoryManager = new TaskMemoryManager(manager, 0) + val c = new TestMemoryConsumer(taskMemoryManager) - assert(taskMemoryManager.acquireExecutionMemory(100L, null) === 100L) - assert(taskMemoryManager.acquireExecutionMemory(400L, null) === 400L) - assert(taskMemoryManager.acquireExecutionMemory(400L, null) === 400L) - assert(taskMemoryManager.acquireExecutionMemory(200L, null) === 100L) - assert(taskMemoryManager.acquireExecutionMemory(100L, null) === 0L) - assert(taskMemoryManager.acquireExecutionMemory(100L, null) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 100L) + assert(taskMemoryManager.acquireExecutionMemory(400L, c) === 400L) + assert(taskMemoryManager.acquireExecutionMemory(400L, c) === 400L) + assert(taskMemoryManager.acquireExecutionMemory(200L, c) === 100L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 0L) - taskMemoryManager.releaseExecutionMemory(500L, null) - assert(taskMemoryManager.acquireExecutionMemory(300L, null) === 300L) - assert(taskMemoryManager.acquireExecutionMemory(300L, null) === 200L) + taskMemoryManager.releaseExecutionMemory(500L, c) + assert(taskMemoryManager.acquireExecutionMemory(300L, c) === 300L) + assert(taskMemoryManager.acquireExecutionMemory(300L, c) === 200L) taskMemoryManager.cleanUpAllAllocatedMemory() - assert(taskMemoryManager.acquireExecutionMemory(1000L, null) === 1000L) - assert(taskMemoryManager.acquireExecutionMemory(100L, null) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(1000L, c) === 1000L) + assert(taskMemoryManager.acquireExecutionMemory(100L, c) === 0L) } - test("two tasks requesting full execution memory") { + test("two tasks requesting full on-heap execution memory") { val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // Have both tasks request 500 bytes, then wait until both requests have been granted: - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L, null) } - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, null) } - assert(Await.result(t1Result1, futureTimeout) === 500L) - assert(Await.result(t2Result1, futureTimeout) === 500L) + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L, c1) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } + assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 500L) + assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 500L) // Have both tasks each request 500 bytes more; both should immediately return 0 as they are // both now at 1 / N - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, null) } - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, null) } - assert(Await.result(t1Result2, 200.millis) === 0L) - assert(Await.result(t2Result2, 200.millis) === 0L) + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, c1) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } + assert(ThreadUtils.awaitResult(t1Result2, 200.millis) === 0L) + assert(ThreadUtils.awaitResult(t2Result2, 200.millis) === 0L) } - test("two tasks cannot grow past 1 / N of execution memory") { + test("two tasks cannot grow past 1 / N of on-heap execution memory") { val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // Have both tasks request 250 bytes, then wait until both requests have been granted: - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L, null) } - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, null) } - assert(Await.result(t1Result1, futureTimeout) === 250L) - assert(Await.result(t2Result1, futureTimeout) === 250L) + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L, c1) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, c2) } + assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 250L) + assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 250L) // Have both tasks each request 500 bytes more. // We should only grant 250 bytes to each of them on this second request - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, null) } - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, null) } - assert(Await.result(t1Result2, futureTimeout) === 250L) - assert(Await.result(t2Result2, futureTimeout) === 250L) + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, c1) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } + assert(ThreadUtils.awaitResult(t1Result2, futureTimeout) === 250L) + assert(ThreadUtils.awaitResult(t2Result2, futureTimeout) === 250L) } - test("tasks can block to get at least 1 / 2N of execution memory") { + test("tasks can block to get at least 1 / 2N of on-heap execution memory") { val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, null) } - assert(Await.result(t1Result1, futureTimeout) === 1000L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, c1) } + assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 1000L) + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, c2) } // Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult // to make sure the other thread blocks for some time otherwise. Thread.sleep(300) - t1MemManager.releaseExecutionMemory(250L, null) + t1MemManager.releaseExecutionMemory(250L, c1) // The memory freed from t1 should now be granted to t2. - assert(Await.result(t2Result1, futureTimeout) === 250L) + assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 250L) // Further requests by t2 should be denied immediately because it now has 1 / 2N of the memory. - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L, null) } - assert(Await.result(t2Result2, 200.millis) === 0L) + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L, c2) } + assert(ThreadUtils.awaitResult(t2Result2, 200.millis) === 0L) } test("TaskMemoryManager.cleanUpAllAllocatedMemory") { val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, null) } - assert(Await.result(t1Result1, futureTimeout) === 1000L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, c1) } + assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 1000L) + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } // Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult // to make sure the other thread blocks for some time otherwise. Thread.sleep(300) // t1 releases all of its memory, so t2 should be able to grab all of the memory t1MemManager.cleanUpAllAllocatedMemory() - assert(Await.result(t2Result1, futureTimeout) === 500L) - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, null) } - assert(Await.result(t2Result2, futureTimeout) === 500L) - val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L, null) } - assert(Await.result(t2Result3, 200.millis) === 0L) + assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 500L) + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } + assert(ThreadUtils.awaitResult(t2Result2, futureTimeout) === 500L) + val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L, c2) } + assert(ThreadUtils.awaitResult(t2Result3, 200.millis) === 0L) } test("tasks should not be granted a negative amount of execution memory") { @@ -249,19 +275,42 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) + val c1 = new TestMemoryConsumer(t1MemManager) + val c2 = new TestMemoryConsumer(t2MemManager) val futureTimeout: Duration = 20.seconds - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L, null) } - assert(Await.result(t1Result1, futureTimeout) === 700L) + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L, c1) } + assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 700L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L, null) } - assert(Await.result(t2Result1, futureTimeout) === 300L) + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L, c2) } + assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 300L) + + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L, c1) } + assert(ThreadUtils.awaitResult(t1Result2, 200.millis) === 0L) + } - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L, null) } - assert(Await.result(t1Result2, 200.millis) === 0L) + test("off-heap execution allocations cannot exceed limit") { + val memoryManager = createMemoryManager( + maxOnHeapExecutionMemory = 0L, + maxOffHeapExecutionMemory = 1000L) + + val tMemManager = new TaskMemoryManager(memoryManager, 1) + val c = new TestMemoryConsumer(tMemManager, MemoryMode.OFF_HEAP) + val result1 = Future { tMemManager.acquireExecutionMemory(1000L, c) } + assert(ThreadUtils.awaitResult(result1, 200.millis) === 1000L) + assert(tMemManager.getMemoryConsumptionForThisTask === 1000L) + + val result2 = Future { tMemManager.acquireExecutionMemory(300L, c) } + assert(ThreadUtils.awaitResult(result2, 200.millis) === 0L) + + assert(tMemManager.getMemoryConsumptionForThisTask === 1000L) + tMemManager.releaseExecutionMemory(500L, c) + assert(tMemManager.getMemoryConsumptionForThisTask === 500L) + tMemManager.releaseExecutionMemory(500L, c) + assert(tMemManager.getMemoryConsumptionForThisTask === 0L) } } private object MemoryManagerSuite { - private val DEFAULT_ENSURE_FREE_SPACE_CALLED = -1L + private val DEFAULT_EVICT_BLOCKS_TO_FREE_SPACE_CALLED = -1L } diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala index 4b4c3b031132..362cd861cc24 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryTestingUtils.scala @@ -17,7 +17,9 @@ package org.apache.spark.memory -import org.apache.spark.{SparkEnv, TaskContextImpl, TaskContext} +import java.util.Properties + +import org.apache.spark.{SparkEnv, TaskContext, TaskContextImpl} /** * Helper methods for mocking out memory-management-related classes in tests. @@ -31,7 +33,7 @@ object MemoryTestingUtils { taskAttemptId = 0, attemptNumber = 0, taskMemoryManager = taskMemoryManager, - metricsSystem = env.metricsSystem, - internalAccumulators = Seq.empty) + localProperties = new Properties, + metricsSystem = env.metricsSystem) } } diff --git a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala index 885c450d6d4f..4e31fb5589a9 100644 --- a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala @@ -17,17 +17,14 @@ package org.apache.spark.memory -import scala.collection.mutable.ArrayBuffer - import org.mockito.Mockito.when import org.apache.spark.SparkConf -import org.apache.spark.storage.{BlockId, BlockStatus, MemoryStore, TestBlockId} - +import org.apache.spark.storage.TestBlockId +import org.apache.spark.storage.memory.MemoryStore class StaticMemoryManagerSuite extends MemoryManagerSuite { private val conf = new SparkConf().set("spark.storage.unrollFraction", "0.4") - private val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] /** * Make a [[StaticMemoryManager]] and a [[MemoryStore]] with limited class dependencies. @@ -36,38 +33,48 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { maxExecutionMem: Long, maxStorageMem: Long): (StaticMemoryManager, MemoryStore) = { val mm = new StaticMemoryManager( - conf, maxExecutionMemory = maxExecutionMem, maxStorageMemory = maxStorageMem, numCores = 1) + conf, + maxOnHeapExecutionMemory = maxExecutionMem, + maxOnHeapStorageMemory = maxStorageMem, + numCores = 1) val ms = makeMemoryStore(mm) (mm, ms) } - override protected def createMemoryManager(maxMemory: Long): MemoryManager = { + override protected def createMemoryManager( + maxOnHeapExecutionMemory: Long, + maxOffHeapExecutionMemory: Long): StaticMemoryManager = { new StaticMemoryManager( - conf, - maxExecutionMemory = maxMemory, - maxStorageMemory = 0, + conf.clone + .set("spark.memory.fraction", "1") + .set("spark.testing.memory", maxOnHeapExecutionMemory.toString) + .set("spark.memory.offHeap.size", maxOffHeapExecutionMemory.toString), + maxOnHeapExecutionMemory = maxOnHeapExecutionMemory, + maxOnHeapStorageMemory = 0, numCores = 1) } test("basic execution memory") { val maxExecutionMem = 1000L + val taskAttemptId = 0L val (mm, _) = makeThings(maxExecutionMem, Long.MaxValue) + val memoryMode = MemoryMode.ON_HEAP assert(mm.executionMemoryUsed === 0L) - assert(mm.doAcquireExecutionMemory(10L, evictedBlocks) === 10L) + assert(mm.acquireExecutionMemory(10L, taskAttemptId, memoryMode) === 10L) assert(mm.executionMemoryUsed === 10L) - assert(mm.doAcquireExecutionMemory(100L, evictedBlocks) === 100L) + assert(mm.acquireExecutionMemory(100L, taskAttemptId, memoryMode) === 100L) // Acquire up to the max - assert(mm.doAcquireExecutionMemory(1000L, evictedBlocks) === 890L) + assert(mm.acquireExecutionMemory(1000L, taskAttemptId, memoryMode) === 890L) assert(mm.executionMemoryUsed === maxExecutionMem) - assert(mm.doAcquireExecutionMemory(1L, evictedBlocks) === 0L) + assert(mm.acquireExecutionMemory(1L, taskAttemptId, memoryMode) === 0L) assert(mm.executionMemoryUsed === maxExecutionMem) - mm.releaseExecutionMemory(800L) + mm.releaseExecutionMemory(800L, taskAttemptId, memoryMode) assert(mm.executionMemoryUsed === 200L) // Acquire after release - assert(mm.doAcquireExecutionMemory(1L, evictedBlocks) === 1L) + assert(mm.acquireExecutionMemory(1L, taskAttemptId, memoryMode) === 1L) assert(mm.executionMemoryUsed === 201L) // Release beyond what was acquired - mm.releaseExecutionMemory(maxExecutionMem) + mm.releaseExecutionMemory(maxExecutionMem, taskAttemptId, memoryMode) assert(mm.executionMemoryUsed === 0L) } @@ -75,60 +82,68 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { val maxStorageMem = 1000L val dummyBlock = TestBlockId("you can see the world you brought to live") val (mm, ms) = makeThings(Long.MaxValue, maxStorageMem) + val memoryMode = MemoryMode.ON_HEAP assert(mm.storageMemoryUsed === 0L) - assert(mm.acquireStorageMemory(dummyBlock, 10L, evictedBlocks)) - // `ensureFreeSpace` should be called with the number of bytes requested - assertEnsureFreeSpaceCalled(ms, 10L) + assert(mm.acquireStorageMemory(dummyBlock, 10L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) assert(mm.storageMemoryUsed === 10L) - assert(mm.acquireStorageMemory(dummyBlock, 100L, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, 100L) + + assert(mm.acquireStorageMemory(dummyBlock, 100L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) assert(mm.storageMemoryUsed === 110L) // Acquire more than the max, not granted - assert(!mm.acquireStorageMemory(dummyBlock, maxStorageMem + 1L, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, maxStorageMem + 1L) + assert(!mm.acquireStorageMemory(dummyBlock, maxStorageMem + 1L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) assert(mm.storageMemoryUsed === 110L) // Acquire up to the max, requests after this are still granted due to LRU eviction - assert(mm.acquireStorageMemory(dummyBlock, maxStorageMem, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, 1000L) + assert(mm.acquireStorageMemory(dummyBlock, maxStorageMem, memoryMode)) + assertEvictBlocksToFreeSpaceCalled(ms, 110L) assert(mm.storageMemoryUsed === 1000L) - assert(mm.acquireStorageMemory(dummyBlock, 1L, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, 1L) + assert(mm.acquireStorageMemory(dummyBlock, 1L, memoryMode)) + assertEvictBlocksToFreeSpaceCalled(ms, 1L) + assert(evictedBlocks.nonEmpty) + evictedBlocks.clear() + // Note: We evicted 1 byte to put another 1-byte block in, so the storage memory used remains at + // 1000 bytes. This is different from real behavior, where the 1-byte block would have evicted + // the 1000-byte block entirely. This is set up differently so we can write finer-grained tests. assert(mm.storageMemoryUsed === 1000L) - mm.releaseStorageMemory(800L) + mm.releaseStorageMemory(800L, memoryMode) assert(mm.storageMemoryUsed === 200L) // Acquire after release - assert(mm.acquireStorageMemory(dummyBlock, 1L, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, 1L) + assert(mm.acquireStorageMemory(dummyBlock, 1L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) assert(mm.storageMemoryUsed === 201L) mm.releaseAllStorageMemory() assert(mm.storageMemoryUsed === 0L) - assert(mm.acquireStorageMemory(dummyBlock, 1L, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, 1L) + assert(mm.acquireStorageMemory(dummyBlock, 1L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) assert(mm.storageMemoryUsed === 1L) // Release beyond what was acquired - mm.releaseStorageMemory(100L) + mm.releaseStorageMemory(100L, memoryMode) assert(mm.storageMemoryUsed === 0L) } test("execution and storage isolation") { val maxExecutionMem = 200L val maxStorageMem = 1000L + val taskAttemptId = 0L val dummyBlock = TestBlockId("ain't nobody love like you do") val (mm, ms) = makeThings(maxExecutionMem, maxStorageMem) + val memoryMode = MemoryMode.ON_HEAP // Only execution memory should increase - assert(mm.doAcquireExecutionMemory(100L, evictedBlocks) === 100L) + assert(mm.acquireExecutionMemory(100L, taskAttemptId, memoryMode) === 100L) assert(mm.storageMemoryUsed === 0L) assert(mm.executionMemoryUsed === 100L) - assert(mm.doAcquireExecutionMemory(1000L, evictedBlocks) === 100L) + assert(mm.acquireExecutionMemory(1000L, taskAttemptId, memoryMode) === 100L) assert(mm.storageMemoryUsed === 0L) assert(mm.executionMemoryUsed === 200L) // Only storage memory should increase - assert(mm.acquireStorageMemory(dummyBlock, 50L, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, 50L) + assert(mm.acquireStorageMemory(dummyBlock, 50L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) assert(mm.storageMemoryUsed === 50L) assert(mm.executionMemoryUsed === 200L) // Only execution memory should be released - mm.releaseExecutionMemory(133L) + mm.releaseExecutionMemory(133L, taskAttemptId, memoryMode) assert(mm.storageMemoryUsed === 50L) assert(mm.executionMemoryUsed === 67L) // Only storage memory should be released @@ -141,24 +156,34 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { val maxStorageMem = 1000L val dummyBlock = TestBlockId("lonely water") val (mm, ms) = makeThings(Long.MaxValue, maxStorageMem) - assert(mm.acquireUnrollMemory(dummyBlock, 100L, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, 100L) + val memoryMode = MemoryMode.ON_HEAP + assert(mm.acquireUnrollMemory(dummyBlock, 100L, memoryMode)) + when(ms.currentUnrollMemory).thenReturn(100L) + assertEvictBlocksToFreeSpaceNotCalled(ms) assert(mm.storageMemoryUsed === 100L) - mm.releaseUnrollMemory(40L) + mm.releaseUnrollMemory(40L, memoryMode) assert(mm.storageMemoryUsed === 60L) when(ms.currentUnrollMemory).thenReturn(60L) - assert(mm.acquireUnrollMemory(dummyBlock, 500L, evictedBlocks)) + assert(mm.acquireStorageMemory(dummyBlock, 800L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) + assert(mm.storageMemoryUsed === 860L) // `spark.storage.unrollFraction` is 0.4, so the max unroll space is 400 bytes. - // Since we already occupy 60 bytes, we will try to ensure only 400 - 60 = 340 bytes. - assertEnsureFreeSpaceCalled(ms, 340L) - assert(mm.storageMemoryUsed === 560L) - when(ms.currentUnrollMemory).thenReturn(560L) - assert(!mm.acquireUnrollMemory(dummyBlock, 800L, evictedBlocks)) - assert(mm.storageMemoryUsed === 560L) - // We already have 560 bytes > the max unroll space of 400 bytes, so no bytes are freed - assertEnsureFreeSpaceCalled(ms, 0L) + // As of this point, cache memory is 800 bytes and current unroll memory is 60 bytes. + // Requesting 240 more bytes of unroll memory will leave our total unroll memory at + // 300 bytes, still under the 400-byte limit. Therefore, all 240 bytes are granted. + assert(mm.acquireUnrollMemory(dummyBlock, 240L, memoryMode)) + assertEvictBlocksToFreeSpaceCalled(ms, 100L) // 860 + 240 - 1000 + when(ms.currentUnrollMemory).thenReturn(300L) // 60 + 240 + assert(mm.storageMemoryUsed === 1000L) + evictedBlocks.clear() + // We already have 300 bytes of unroll memory, so requesting 150 more will leave us + // above the 400-byte limit. Since there is not enough free memory, this request will + // fail even after evicting as much as we can (400 - 300 = 100 bytes). + assert(!mm.acquireUnrollMemory(dummyBlock, 150L, memoryMode)) + assertEvictBlocksToFreeSpaceCalled(ms, 100L) + assert(mm.storageMemoryUsed === 900L) // Release beyond what was acquired - mm.releaseUnrollMemory(maxStorageMem) + mm.releaseUnrollMemory(maxStorageMem, memoryMode) assert(mm.storageMemoryUsed === 0L) } diff --git a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala index 77e43554ee27..5f699df8211d 100644 --- a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala +++ b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala @@ -17,24 +17,23 @@ package org.apache.spark.memory -import scala.collection.mutable - import org.apache.spark.SparkConf -import org.apache.spark.storage.{BlockStatus, BlockId} +import org.apache.spark.storage.BlockId + +class TestMemoryManager(conf: SparkConf) + extends MemoryManager(conf, numCores = 1, Long.MaxValue, Long.MaxValue) { -class TestMemoryManager(conf: SparkConf) extends MemoryManager(conf, numCores = 1) { - private[memory] override def doAcquireExecutionMemory( + override private[memory] def acquireExecutionMemory( numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Long = synchronized { + taskAttemptId: Long, + memoryMode: MemoryMode): Long = { if (oomOnce) { oomOnce = false 0 } else if (available >= numBytes) { - _executionMemoryUsed += numBytes // To suppress warnings when freeing unallocated memory available -= numBytes numBytes } else { - _executionMemoryUsed += available val grant = available available = 0 grant @@ -43,18 +42,21 @@ class TestMemoryManager(conf: SparkConf) extends MemoryManager(conf, numCores = override def acquireStorageMemory( blockId: BlockId, numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = true + memoryMode: MemoryMode): Boolean = true override def acquireUnrollMemory( blockId: BlockId, numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = true - override def releaseExecutionMemory(numBytes: Long): Unit = { + memoryMode: MemoryMode): Boolean = true + override def releaseStorageMemory(numBytes: Long, memoryMode: MemoryMode): Unit = {} + override private[memory] def releaseExecutionMemory( + numBytes: Long, + taskAttemptId: Long, + memoryMode: MemoryMode): Unit = { available += numBytes - _executionMemoryUsed -= numBytes } - override def releaseStorageMemory(numBytes: Long): Unit = {} - override def maxExecutionMemory: Long = Long.MaxValue - override def maxStorageMemory: Long = Long.MaxValue + override def maxOnHeapStorageMemory: Long = Long.MaxValue + + override def maxOffHeapStorageMemory: Long = 0L private var oomOnce = false private var available = Long.MaxValue diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index 0c97f2bd8965..02b04cdbb2a5 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -17,196 +17,322 @@ package org.apache.spark.memory -import scala.collection.mutable.ArrayBuffer - import org.scalatest.PrivateMethodTester import org.apache.spark.SparkConf -import org.apache.spark.storage.{BlockId, BlockStatus, MemoryStore, TestBlockId} - +import org.apache.spark.storage.TestBlockId +import org.apache.spark.storage.memory.MemoryStore class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTester { - private val conf = new SparkConf().set("spark.memory.storageFraction", "0.5") private val dummyBlock = TestBlockId("--") - private val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + + private val storageFraction: Double = 0.5 /** * Make a [[UnifiedMemoryManager]] and a [[MemoryStore]] with limited class dependencies. */ private def makeThings(maxMemory: Long): (UnifiedMemoryManager, MemoryStore) = { - val mm = new UnifiedMemoryManager(conf, maxMemory, numCores = 1) + val mm = createMemoryManager(maxMemory) val ms = makeMemoryStore(mm) (mm, ms) } - override protected def createMemoryManager(maxMemory: Long): MemoryManager = { - new UnifiedMemoryManager(conf, maxMemory, numCores = 1) - } - - private def getStorageRegionSize(mm: UnifiedMemoryManager): Long = { - mm invokePrivate PrivateMethod[Long]('storageRegionSize)() - } - - test("storage region size") { - val maxMemory = 1000L - val (mm, _) = makeThings(maxMemory) - val storageFraction = conf.get("spark.memory.storageFraction").toDouble - val expectedStorageRegionSize = maxMemory * storageFraction - val actualStorageRegionSize = getStorageRegionSize(mm) - assert(expectedStorageRegionSize === actualStorageRegionSize) + override protected def createMemoryManager( + maxOnHeapExecutionMemory: Long, + maxOffHeapExecutionMemory: Long): UnifiedMemoryManager = { + val conf = new SparkConf() + .set("spark.memory.fraction", "1") + .set("spark.testing.memory", maxOnHeapExecutionMemory.toString) + .set("spark.memory.offHeap.size", maxOffHeapExecutionMemory.toString) + .set("spark.memory.storageFraction", storageFraction.toString) + UnifiedMemoryManager(conf, numCores = 1) } test("basic execution memory") { val maxMemory = 1000L + val taskAttemptId = 0L val (mm, _) = makeThings(maxMemory) + val memoryMode = MemoryMode.ON_HEAP assert(mm.executionMemoryUsed === 0L) - assert(mm.doAcquireExecutionMemory(10L, evictedBlocks) === 10L) + assert(mm.acquireExecutionMemory(10L, taskAttemptId, memoryMode) === 10L) assert(mm.executionMemoryUsed === 10L) - assert(mm.doAcquireExecutionMemory(100L, evictedBlocks) === 100L) + assert(mm.acquireExecutionMemory(100L, taskAttemptId, memoryMode) === 100L) // Acquire up to the max - assert(mm.doAcquireExecutionMemory(1000L, evictedBlocks) === 890L) + assert(mm.acquireExecutionMemory(1000L, taskAttemptId, memoryMode) === 890L) assert(mm.executionMemoryUsed === maxMemory) - assert(mm.doAcquireExecutionMemory(1L, evictedBlocks) === 0L) + assert(mm.acquireExecutionMemory(1L, taskAttemptId, memoryMode) === 0L) assert(mm.executionMemoryUsed === maxMemory) - mm.releaseExecutionMemory(800L) + mm.releaseExecutionMemory(800L, taskAttemptId, memoryMode) assert(mm.executionMemoryUsed === 200L) // Acquire after release - assert(mm.doAcquireExecutionMemory(1L, evictedBlocks) === 1L) + assert(mm.acquireExecutionMemory(1L, taskAttemptId, memoryMode) === 1L) assert(mm.executionMemoryUsed === 201L) // Release beyond what was acquired - mm.releaseExecutionMemory(maxMemory) + mm.releaseExecutionMemory(maxMemory, taskAttemptId, memoryMode) assert(mm.executionMemoryUsed === 0L) } test("basic storage memory") { val maxMemory = 1000L val (mm, ms) = makeThings(maxMemory) + val memoryMode = MemoryMode.ON_HEAP assert(mm.storageMemoryUsed === 0L) - assert(mm.acquireStorageMemory(dummyBlock, 10L, evictedBlocks)) - // `ensureFreeSpace` should be called with the number of bytes requested - assertEnsureFreeSpaceCalled(ms, 10L) + assert(mm.acquireStorageMemory(dummyBlock, 10L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) assert(mm.storageMemoryUsed === 10L) - assert(mm.acquireStorageMemory(dummyBlock, 100L, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, 100L) + + assert(mm.acquireStorageMemory(dummyBlock, 100L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) assert(mm.storageMemoryUsed === 110L) // Acquire more than the max, not granted - assert(!mm.acquireStorageMemory(dummyBlock, maxMemory + 1L, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, maxMemory + 1L) + assert(!mm.acquireStorageMemory(dummyBlock, maxMemory + 1L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) assert(mm.storageMemoryUsed === 110L) // Acquire up to the max, requests after this are still granted due to LRU eviction - assert(mm.acquireStorageMemory(dummyBlock, maxMemory, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, 1000L) + assert(mm.acquireStorageMemory(dummyBlock, maxMemory, memoryMode)) + assertEvictBlocksToFreeSpaceCalled(ms, 110L) assert(mm.storageMemoryUsed === 1000L) - assert(mm.acquireStorageMemory(dummyBlock, 1L, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, 1L) + assert(evictedBlocks.nonEmpty) + evictedBlocks.clear() + assert(mm.acquireStorageMemory(dummyBlock, 1L, memoryMode)) + assertEvictBlocksToFreeSpaceCalled(ms, 1L) + assert(evictedBlocks.nonEmpty) + evictedBlocks.clear() + // Note: We evicted 1 byte to put another 1-byte block in, so the storage memory used remains at + // 1000 bytes. This is different from real behavior, where the 1-byte block would have evicted + // the 1000-byte block entirely. This is set up differently so we can write finer-grained tests. assert(mm.storageMemoryUsed === 1000L) - mm.releaseStorageMemory(800L) + mm.releaseStorageMemory(800L, memoryMode) assert(mm.storageMemoryUsed === 200L) // Acquire after release - assert(mm.acquireStorageMemory(dummyBlock, 1L, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, 1L) + assert(mm.acquireStorageMemory(dummyBlock, 1L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) assert(mm.storageMemoryUsed === 201L) mm.releaseAllStorageMemory() assert(mm.storageMemoryUsed === 0L) - assert(mm.acquireStorageMemory(dummyBlock, 1L, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, 1L) + assert(mm.acquireStorageMemory(dummyBlock, 1L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) assert(mm.storageMemoryUsed === 1L) // Release beyond what was acquired - mm.releaseStorageMemory(100L) + mm.releaseStorageMemory(100L, memoryMode) assert(mm.storageMemoryUsed === 0L) } test("execution evicts storage") { val maxMemory = 1000L + val taskAttemptId = 0L val (mm, ms) = makeThings(maxMemory) - // First, ensure the test classes are set up as expected - val expectedStorageRegionSize = 500L - val expectedExecutionRegionSize = 500L - val storageRegionSize = getStorageRegionSize(mm) - val executionRegionSize = maxMemory - expectedStorageRegionSize - require(storageRegionSize === expectedStorageRegionSize, - "bad test: storage region size is unexpected") - require(executionRegionSize === expectedExecutionRegionSize, - "bad test: storage region size is unexpected") + val memoryMode = MemoryMode.ON_HEAP // Acquire enough storage memory to exceed the storage region - assert(mm.acquireStorageMemory(dummyBlock, 750L, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, 750L) + assert(mm.acquireStorageMemory(dummyBlock, 750L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) assert(mm.executionMemoryUsed === 0L) assert(mm.storageMemoryUsed === 750L) - require(mm.storageMemoryUsed > storageRegionSize, - s"bad test: storage memory used should exceed the storage region") // Execution needs to request 250 bytes to evict storage memory - assert(mm.doAcquireExecutionMemory(100L, evictedBlocks) === 100L) + assert(mm.acquireExecutionMemory(100L, taskAttemptId, memoryMode) === 100L) assert(mm.executionMemoryUsed === 100L) assert(mm.storageMemoryUsed === 750L) - assertEnsureFreeSpaceNotCalled(ms) + assertEvictBlocksToFreeSpaceNotCalled(ms) // Execution wants 200 bytes but only 150 are free, so storage is evicted - assert(mm.doAcquireExecutionMemory(200L, evictedBlocks) === 200L) - assertEnsureFreeSpaceCalled(ms, 200L) + assert(mm.acquireExecutionMemory(200L, taskAttemptId, memoryMode) === 200L) assert(mm.executionMemoryUsed === 300L) + assert(mm.storageMemoryUsed === 700L) + assertEvictBlocksToFreeSpaceCalled(ms, 50L) + assert(evictedBlocks.nonEmpty) + evictedBlocks.clear() mm.releaseAllStorageMemory() - require(mm.executionMemoryUsed < executionRegionSize, - s"bad test: execution memory used should be within the execution region") + require(mm.executionMemoryUsed === 300L) require(mm.storageMemoryUsed === 0, "bad test: all storage memory should have been released") // Acquire some storage memory again, but this time keep it within the storage region - assert(mm.acquireStorageMemory(dummyBlock, 400L, evictedBlocks)) - assertEnsureFreeSpaceCalled(ms, 400L) - require(mm.storageMemoryUsed < storageRegionSize, - s"bad test: storage memory used should be within the storage region") + assert(mm.acquireStorageMemory(dummyBlock, 400L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) + assert(mm.storageMemoryUsed === 400L) + assert(mm.executionMemoryUsed === 300L) // Execution cannot evict storage because the latter is within the storage fraction, // so grant only what's remaining without evicting anything, i.e. 1000 - 300 - 400 = 300 - assert(mm.doAcquireExecutionMemory(400L, evictedBlocks) === 300L) + assert(mm.acquireExecutionMemory(400L, taskAttemptId, memoryMode) === 300L) assert(mm.executionMemoryUsed === 600L) assert(mm.storageMemoryUsed === 400L) - assertEnsureFreeSpaceNotCalled(ms) + assertEvictBlocksToFreeSpaceNotCalled(ms) + } + + test("execution memory requests smaller than free memory should evict storage (SPARK-12165)") { + val maxMemory = 1000L + val taskAttemptId = 0L + val (mm, ms) = makeThings(maxMemory) + val memoryMode = MemoryMode.ON_HEAP + // Acquire enough storage memory to exceed the storage region size + assert(mm.acquireStorageMemory(dummyBlock, 700L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) + assert(mm.executionMemoryUsed === 0L) + assert(mm.storageMemoryUsed === 700L) + // SPARK-12165: previously, MemoryStore would not evict anything because it would + // mistakenly think that the 300 bytes of free space was still available even after + // using it to expand the execution pool. Consequently, no storage memory was released + // and the following call granted only 300 bytes to execution. + assert(mm.acquireExecutionMemory(500L, taskAttemptId, memoryMode) === 500L) + assertEvictBlocksToFreeSpaceCalled(ms, 200L) + assert(mm.storageMemoryUsed === 500L) + assert(mm.executionMemoryUsed === 500L) + assert(evictedBlocks.nonEmpty) } test("storage does not evict execution") { val maxMemory = 1000L + val taskAttemptId = 0L val (mm, ms) = makeThings(maxMemory) - // First, ensure the test classes are set up as expected - val expectedStorageRegionSize = 500L - val expectedExecutionRegionSize = 500L - val storageRegionSize = getStorageRegionSize(mm) - val executionRegionSize = maxMemory - expectedStorageRegionSize - require(storageRegionSize === expectedStorageRegionSize, - "bad test: storage region size is unexpected") - require(executionRegionSize === expectedExecutionRegionSize, - "bad test: storage region size is unexpected") + val memoryMode = MemoryMode.ON_HEAP // Acquire enough execution memory to exceed the execution region - assert(mm.doAcquireExecutionMemory(800L, evictedBlocks) === 800L) + assert(mm.acquireExecutionMemory(800L, taskAttemptId, memoryMode) === 800L) assert(mm.executionMemoryUsed === 800L) assert(mm.storageMemoryUsed === 0L) - assertEnsureFreeSpaceNotCalled(ms) - require(mm.executionMemoryUsed > executionRegionSize, - s"bad test: execution memory used should exceed the execution region") + assertEvictBlocksToFreeSpaceNotCalled(ms) // Storage should not be able to evict execution - assert(mm.acquireStorageMemory(dummyBlock, 100L, evictedBlocks)) + assert(mm.acquireStorageMemory(dummyBlock, 100L, memoryMode)) assert(mm.executionMemoryUsed === 800L) assert(mm.storageMemoryUsed === 100L) - assertEnsureFreeSpaceCalled(ms, 100L) - assert(!mm.acquireStorageMemory(dummyBlock, 250L, evictedBlocks)) + assertEvictBlocksToFreeSpaceNotCalled(ms) + assert(!mm.acquireStorageMemory(dummyBlock, 250L, memoryMode)) assert(mm.executionMemoryUsed === 800L) assert(mm.storageMemoryUsed === 100L) - assertEnsureFreeSpaceCalled(ms, 250L) - mm.releaseExecutionMemory(maxMemory) - mm.releaseStorageMemory(maxMemory) + // Do not attempt to evict blocks, since evicting will not free enough memory: + assertEvictBlocksToFreeSpaceNotCalled(ms) + mm.releaseExecutionMemory(maxMemory, taskAttemptId, memoryMode) + mm.releaseStorageMemory(maxMemory, memoryMode) // Acquire some execution memory again, but this time keep it within the execution region - assert(mm.doAcquireExecutionMemory(200L, evictedBlocks) === 200L) + assert(mm.acquireExecutionMemory(200L, taskAttemptId, memoryMode) === 200L) assert(mm.executionMemoryUsed === 200L) assert(mm.storageMemoryUsed === 0L) - assertEnsureFreeSpaceNotCalled(ms) - require(mm.executionMemoryUsed < executionRegionSize, - s"bad test: execution memory used should be within the execution region") + assertEvictBlocksToFreeSpaceNotCalled(ms) // Storage should still not be able to evict execution - assert(mm.acquireStorageMemory(dummyBlock, 750L, evictedBlocks)) + assert(mm.acquireStorageMemory(dummyBlock, 750L, memoryMode)) assert(mm.executionMemoryUsed === 200L) assert(mm.storageMemoryUsed === 750L) - assertEnsureFreeSpaceCalled(ms, 750L) - assert(!mm.acquireStorageMemory(dummyBlock, 850L, evictedBlocks)) + assertEvictBlocksToFreeSpaceNotCalled(ms) // since there were 800 bytes free + assert(!mm.acquireStorageMemory(dummyBlock, 850L, memoryMode)) assert(mm.executionMemoryUsed === 200L) assert(mm.storageMemoryUsed === 750L) - assertEnsureFreeSpaceCalled(ms, 850L) + // Do not attempt to evict blocks, since evicting will not free enough memory: + assertEvictBlocksToFreeSpaceNotCalled(ms) + } + + test("small heap") { + val systemMemory = 1024 * 1024 + val reservedMemory = 300 * 1024 + val memoryFraction = 0.8 + val conf = new SparkConf() + .set("spark.memory.fraction", memoryFraction.toString) + .set("spark.testing.memory", systemMemory.toString) + .set("spark.testing.reservedMemory", reservedMemory.toString) + val mm = UnifiedMemoryManager(conf, numCores = 1) + val expectedMaxMemory = ((systemMemory - reservedMemory) * memoryFraction).toLong + assert(mm.maxHeapMemory === expectedMaxMemory) + + // Try using a system memory that's too small + val conf2 = conf.clone().set("spark.testing.memory", (reservedMemory / 2).toString) + val exception = intercept[IllegalArgumentException] { + UnifiedMemoryManager(conf2, numCores = 1) + } + assert(exception.getMessage.contains("increase heap size")) + } + + test("insufficient executor memory") { + val systemMemory = 1024 * 1024 + val reservedMemory = 300 * 1024 + val memoryFraction = 0.8 + val conf = new SparkConf() + .set("spark.memory.fraction", memoryFraction.toString) + .set("spark.testing.memory", systemMemory.toString) + .set("spark.testing.reservedMemory", reservedMemory.toString) + val mm = UnifiedMemoryManager(conf, numCores = 1) + + // Try using an executor memory that's too small + val conf2 = conf.clone().set("spark.executor.memory", (reservedMemory / 2).toString) + val exception = intercept[IllegalArgumentException] { + UnifiedMemoryManager(conf2, numCores = 1) + } + assert(exception.getMessage.contains("increase executor memory")) + } + + test("execution can evict cached blocks when there are multiple active tasks (SPARK-12155)") { + val conf = new SparkConf() + .set("spark.memory.fraction", "1") + .set("spark.memory.storageFraction", "0") + .set("spark.testing.memory", "1000") + val mm = UnifiedMemoryManager(conf, numCores = 2) + val ms = makeMemoryStore(mm) + val memoryMode = MemoryMode.ON_HEAP + assert(mm.maxHeapMemory === 1000) + // Have two tasks each acquire some execution memory so that the memory pool registers that + // there are two active tasks: + assert(mm.acquireExecutionMemory(100L, 0, memoryMode) === 100L) + assert(mm.acquireExecutionMemory(100L, 1, memoryMode) === 100L) + // Fill up all of the remaining memory with storage. + assert(mm.acquireStorageMemory(dummyBlock, 800L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) + assert(mm.storageMemoryUsed === 800) + assert(mm.executionMemoryUsed === 200) + // A task should still be able to allocate 100 bytes execution memory by evicting blocks + assert(mm.acquireExecutionMemory(100L, 0, memoryMode) === 100L) + assertEvictBlocksToFreeSpaceCalled(ms, 100L) + assert(mm.executionMemoryUsed === 300) + assert(mm.storageMemoryUsed === 700) + assert(evictedBlocks.nonEmpty) } + test("SPARK-15260: atomically resize memory pools") { + val conf = new SparkConf() + .set("spark.memory.fraction", "1") + .set("spark.memory.storageFraction", "0") + .set("spark.testing.memory", "1000") + val mm = UnifiedMemoryManager(conf, numCores = 2) + makeBadMemoryStore(mm) + val memoryMode = MemoryMode.ON_HEAP + // Acquire 1000 then release 600 bytes of storage memory, leaving the + // storage memory pool at 1000 bytes but only 400 bytes of which are used. + assert(mm.acquireStorageMemory(dummyBlock, 1000L, memoryMode)) + mm.releaseStorageMemory(600L, memoryMode) + // Before the fix for SPARK-15260, we would first shrink the storage pool by the amount of + // unused storage memory (600 bytes), try to evict blocks, then enlarge the execution pool + // by the same amount. If the eviction threw an exception, then we would shrink one pool + // without enlarging the other, resulting in an assertion failure. + intercept[RuntimeException] { + mm.acquireExecutionMemory(1000L, 0, memoryMode) + } + val assertInvariants = PrivateMethod[Unit]('assertInvariants) + mm.invokePrivate[Unit](assertInvariants()) + } + + test("not enough free memory in the storage pool --OFF_HEAP") { + val conf = new SparkConf() + .set("spark.memory.offHeap.size", "1000") + .set("spark.testing.memory", "1000") + .set("spark.memory.offHeap.enabled", "true") + val taskAttemptId = 0L + val mm = UnifiedMemoryManager(conf, numCores = 1) + val ms = makeMemoryStore(mm) + val memoryMode = MemoryMode.OFF_HEAP + + assert(mm.acquireExecutionMemory(400L, taskAttemptId, memoryMode) === 400L) + assert(mm.storageMemoryUsed === 0L) + assert(mm.executionMemoryUsed === 400L) + + // Fail fast + assert(!mm.acquireStorageMemory(dummyBlock, 700L, memoryMode)) + assert(mm.storageMemoryUsed === 0L) + + assert(mm.acquireStorageMemory(dummyBlock, 100L, memoryMode)) + assert(mm.storageMemoryUsed === 100L) + assertEvictBlocksToFreeSpaceNotCalled(ms) + + // Borrow 50 from execution memory + assert(mm.acquireStorageMemory(dummyBlock, 450L, memoryMode)) + assertEvictBlocksToFreeSpaceNotCalled(ms) + assert(mm.storageMemoryUsed === 550L) + + // Borrow 50 from execution memory and evict 50 to free space + assert(mm.acquireStorageMemory(dummyBlock, 100L, memoryMode)) + assertEvictBlocksToFreeSpaceCalled(ms, 50) + assert(mm.storageMemoryUsed === 600L) + } } diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index 44eb5a046912..6f4203da1d86 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -25,23 +25,16 @@ import org.apache.commons.lang3.RandomUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.{LongWritable, Text} -import org.apache.hadoop.mapred.lib.{CombineFileInputFormat => OldCombineFileInputFormat, - CombineFileRecordReader => OldCombineFileRecordReader, CombineFileSplit => OldCombineFileSplit} -import org.apache.hadoop.mapred.{JobConf, Reporter, FileSplit => OldFileSplit, - InputSplit => OldInputSplit, LineRecordReader => OldLineRecordReader, - RecordReader => OldRecordReader, TextInputFormat => OldTextInputFormat} -import org.apache.hadoop.mapreduce.lib.input.{CombineFileInputFormat => NewCombineFileInputFormat, - CombineFileRecordReader => NewCombineFileRecordReader, CombineFileSplit => NewCombineFileSplit, - FileSplit => NewFileSplit, TextInputFormat => NewTextInputFormat} +import org.apache.hadoop.mapred.{FileSplit => OldFileSplit, InputSplit => OldInputSplit, JobConf, LineRecordReader => OldLineRecordReader, RecordReader => OldRecordReader, Reporter, TextInputFormat => OldTextInputFormat} +import org.apache.hadoop.mapred.lib.{CombineFileInputFormat => OldCombineFileInputFormat, CombineFileRecordReader => OldCombineFileRecordReader, CombineFileSplit => OldCombineFileSplit} +import org.apache.hadoop.mapreduce.{InputSplit => NewInputSplit, RecordReader => NewRecordReader, TaskAttemptContext} +import org.apache.hadoop.mapreduce.lib.input.{CombineFileInputFormat => NewCombineFileInputFormat, CombineFileRecordReader => NewCombineFileRecordReader, CombineFileSplit => NewCombineFileSplit, FileSplit => NewFileSplit, TextInputFormat => NewTextInputFormat} import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} -import org.apache.hadoop.mapreduce.{TaskAttemptContext, InputSplit => NewInputSplit, - RecordReader => NewRecordReader} import org.scalatest.BeforeAndAfter import org.apache.spark.{SharedSparkContext, SparkFunSuite} -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext with BeforeAndAfter { @@ -67,7 +60,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext pw.close() // Path to tmpFile - tmpFilePath = "file://" + tmpFile.getAbsolutePath + tmpFilePath = tmpFile.toURI.toString } after { @@ -98,45 +91,11 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext rdd.coalesce(4).count() } - // for count and coelesce, the same bytes should be read. + // for count and coalesce, the same bytes should be read. assert(bytesRead != 0) assert(bytesRead2 == bytesRead) } - /** - * This checks the situation where we have interleaved reads from - * different sources. Currently, we only accumulate fron the first - * read method we find in the task. This test uses cartesian to create - * the interleaved reads. - * - * Once https://issues.apache.org/jira/browse/SPARK-5225 is fixed - * this test should break. - */ - test("input metrics with mixed read method") { - // prime the cache manager - val numPartitions = 2 - val rdd = sc.parallelize(1 to 100, numPartitions).cache() - rdd.collect() - - val rdd2 = sc.textFile(tmpFilePath, numPartitions) - - val bytesRead = runAndReturnBytesRead { - rdd.count() - } - val bytesRead2 = runAndReturnBytesRead { - rdd2.count() - } - - val cartRead = runAndReturnBytesRead { - rdd.cartesian(rdd2).count() - } - - assert(cartRead != 0) - assert(bytesRead != 0) - // We read from the first rdd of the cartesian once per partition. - assert(cartRead == bytesRead * numPartitions) - } - test("input metrics for new Hadoop API with coalesce") { val bytesRead = runAndReturnBytesRead { sc.newAPIHadoopFile(tmpFilePath, classOf[NewTextInputFormat], classOf[LongWritable], @@ -183,7 +142,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext assert(records == numRecords) } - test("input metrics on recordsd read with cache") { + test("input metrics on records read with cache") { // prime the cache manager val rdd = sc.textFile(tmpFilePath, 4).cache() rdd.collect() @@ -209,10 +168,10 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext sc.addSparkListener(new SparkListener() { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val metrics = taskEnd.taskMetrics - metrics.inputMetrics.foreach(inputRead += _.recordsRead) - metrics.outputMetrics.foreach(outputWritten += _.recordsWritten) - metrics.shuffleReadMetrics.foreach(shuffleRead += _.recordsRead) - metrics.shuffleWriteMetrics.foreach(shuffleWritten += _.shuffleRecordsWritten) + inputRead += metrics.inputMetrics.recordsRead + outputWritten += metrics.outputMetrics.recordsWritten + shuffleRead += metrics.shuffleReadMetrics.recordsRead + shuffleWritten += metrics.shuffleWriteMetrics.recordsWritten } }) @@ -221,15 +180,12 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext sc.textFile(tmpFilePath, 4) .map(key => (key, 1)) .reduceByKey(_ + _) - .saveAsTextFile("file://" + tmpFile.getAbsolutePath) + .saveAsTextFile(tmpFile.toURI.toString) sc.listenerBus.waitUntilEmpty(500) assert(inputRead == numRecords) - // Only supported on newer Hadoop - if (SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback().isDefined) { - assert(outputWritten == numBuckets) - } + assert(outputWritten == numBuckets) assert(shuffleRead == shuffleWritten) } @@ -237,7 +193,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext val numPartitions = 2 val cartVector = 0 to 9 val cartFile = new File(tmpDir, getClass.getSimpleName + "_cart.txt") - val cartFilePath = "file://" + cartFile.getAbsolutePath + val cartFilePath = cartFile.toURI.toString // write files to disk so we can read them later. sc.parallelize(cartVector).saveAsTextFile(cartFilePath) @@ -272,19 +228,18 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext } private def runAndReturnBytesRead(job: => Unit): Long = { - runAndReturnMetrics(job, _.taskMetrics.inputMetrics.map(_.bytesRead)) + runAndReturnMetrics(job, _.taskMetrics.inputMetrics.bytesRead) } private def runAndReturnRecordsRead(job: => Unit): Long = { - runAndReturnMetrics(job, _.taskMetrics.inputMetrics.map(_.recordsRead)) + runAndReturnMetrics(job, _.taskMetrics.inputMetrics.recordsRead) } private def runAndReturnRecordsWritten(job: => Unit): Long = { - runAndReturnMetrics(job, _.taskMetrics.outputMetrics.map(_.recordsWritten)) + runAndReturnMetrics(job, _.taskMetrics.outputMetrics.recordsWritten) } - private def runAndReturnMetrics(job: => Unit, - collector: (SparkListenerTaskEnd) => Option[Long]): Long = { + private def runAndReturnMetrics(job: => Unit, collector: (SparkListenerTaskEnd) => Long): Long = { val taskMetrics = new ArrayBuffer[Long]() // Avoid receiving earlier taskEnd events @@ -292,7 +247,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext sc.addSparkListener(new SparkListener() { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - collector(taskEnd).foreach(taskMetrics += _) + taskMetrics += collector(taskEnd) } }) @@ -303,57 +258,49 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext } test("output metrics on records written") { - // Only supported on newer Hadoop - if (SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback().isDefined) { - val file = new File(tmpDir, getClass.getSimpleName) - val filePath = "file://" + file.getAbsolutePath + val file = new File(tmpDir, getClass.getSimpleName) + val filePath = file.toURI.toURL.toString - val records = runAndReturnRecordsWritten { - sc.parallelize(1 to numRecords).saveAsTextFile(filePath) - } - assert(records == numRecords) + val records = runAndReturnRecordsWritten { + sc.parallelize(1 to numRecords).saveAsTextFile(filePath) } + assert(records == numRecords) } test("output metrics on records written - new Hadoop API") { - // Only supported on newer Hadoop - if (SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback().isDefined) { - val file = new File(tmpDir, getClass.getSimpleName) - val filePath = "file://" + file.getAbsolutePath - - val records = runAndReturnRecordsWritten { - sc.parallelize(1 to numRecords).map(key => (key.toString, key.toString)) - .saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](filePath) - } - assert(records == numRecords) + val file = new File(tmpDir, getClass.getSimpleName) + val filePath = file.toURI.toURL.toString + + val records = runAndReturnRecordsWritten { + sc.parallelize(1 to numRecords).map(key => (key.toString, key.toString)) + .saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](filePath) } + assert(records == numRecords) } test("output metrics when writing text file") { val fs = FileSystem.getLocal(new Configuration()) val outPath = new Path(fs.getWorkingDirectory, "outdir") - if (SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback().isDefined) { - val taskBytesWritten = new ArrayBuffer[Long]() - sc.addSparkListener(new SparkListener() { - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - taskBytesWritten += taskEnd.taskMetrics.outputMetrics.get.bytesWritten - } - }) + val taskBytesWritten = new ArrayBuffer[Long]() + sc.addSparkListener(new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + taskBytesWritten += taskEnd.taskMetrics.outputMetrics.bytesWritten + } + }) - val rdd = sc.parallelize(Array("a", "b", "c", "d"), 2) + val rdd = sc.parallelize(Array("a", "b", "c", "d"), 2) - try { - rdd.saveAsTextFile(outPath.toString) - sc.listenerBus.waitUntilEmpty(500) - assert(taskBytesWritten.length == 2) - val outFiles = fs.listStatus(outPath).filter(_.getPath.getName != "_SUCCESS") - taskBytesWritten.zip(outFiles).foreach { case (bytes, fileStatus) => - assert(bytes >= fileStatus.getLen) - } - } finally { - fs.delete(outPath, true) + try { + rdd.saveAsTextFile(outPath.toString) + sc.listenerBus.waitUntilEmpty(500) + assert(taskBytesWritten.length == 2) + val outFiles = fs.listStatus(outPath).filter(_.getPath.getName != "_SUCCESS") + taskBytesWritten.zip(outFiles).foreach { case (bytes, fileStatus) => + assert(bytes >= fileStatus.getLen) } + } finally { + fs.delete(outPath, true) } } @@ -372,6 +319,35 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext } assert(bytesRead >= tmpFile.length()) } + + test("input metrics with old Hadoop API in different thread") { + val bytesRead = runAndReturnBytesRead { + sc.textFile(tmpFilePath, 4).mapPartitions { iter => + val buf = new ArrayBuffer[String]() + ThreadUtils.runInNewThread("testThread", false) { + iter.flatMap(_.split(" ")).foreach(buf.append(_)) + } + + buf.iterator + }.count() + } + assert(bytesRead >= tmpFile.length()) + } + + test("input metrics with new Hadoop API in different thread") { + val bytesRead = runAndReturnBytesRead { + sc.newAPIHadoopFile(tmpFilePath, classOf[NewTextInputFormat], classOf[LongWritable], + classOf[Text]).mapPartitions { iter => + val buf = new ArrayBuffer[String]() + ThreadUtils.runInNewThread("testThread", false) { + iter.map(_._2.toString).flatMap(_.split(" ")).foreach(buf.append(_)) + } + + buf.iterator + }.count() + } + assert(bytesRead >= tmpFile.length()) + } } /** 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 41f2ff725a17..a85011b42bbc 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala @@ -17,10 +17,9 @@ package org.apache.spark.metrics -import org.apache.spark.SparkConf - import org.scalatest.BeforeAndAfter +import org.apache.spark.SparkConf import org.apache.spark.SparkFunSuite class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { @@ -140,7 +139,7 @@ class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { val conf = new MetricsConfig(sparkConf) conf.initialize() - val propCategories = conf.propertyCategories + val propCategories = conf.perInstanceSubProperties assert(propCategories.size === 3) val masterProp = conf.getInstance("master") diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index 9c389c76bf3b..61db6af830cc 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -17,15 +17,15 @@ package org.apache.spark.metrics +import scala.collection.mutable.ArrayBuffer + +import com.codahale.metrics.MetricRegistry import org.scalatest.{BeforeAndAfter, PrivateMethodTester} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.master.MasterSource -import org.apache.spark.metrics.source.Source - -import com.codahale.metrics.MetricRegistry - -import scala.collection.mutable.ArrayBuffer +import org.apache.spark.internal.config._ +import org.apache.spark.metrics.source.{Source, StaticSources} class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateMethodTester{ var filePath: String = _ @@ -44,7 +44,7 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM val sources = PrivateMethod[ArrayBuffer[Source]]('sources) val sinks = PrivateMethod[ArrayBuffer[Source]]('sinks) - assert(metricsSystem.invokePrivate(sources()).length === 0) + assert(metricsSystem.invokePrivate(sources()).length === StaticSources.allSources.length) assert(metricsSystem.invokePrivate(sinks()).length === 0) assert(metricsSystem.getServletHandlers.nonEmpty) } @@ -55,13 +55,13 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM val sources = PrivateMethod[ArrayBuffer[Source]]('sources) val sinks = PrivateMethod[ArrayBuffer[Source]]('sinks) - assert(metricsSystem.invokePrivate(sources()).length === 0) + assert(metricsSystem.invokePrivate(sources()).length === StaticSources.allSources.length) assert(metricsSystem.invokePrivate(sinks()).length === 1) assert(metricsSystem.getServletHandlers.nonEmpty) val source = new MasterSource(null) metricsSystem.registerSource(source) - assert(metricsSystem.invokePrivate(sources()).length === 1) + assert(metricsSystem.invokePrivate(sources()).length === StaticSources.allSources.length + 1) } test("MetricsSystem with Driver instance") { @@ -184,4 +184,88 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM assert(metricName != s"$appId.$executorId.${source.sourceName}") assert(metricName === source.sourceName) } + + test("MetricsSystem with Executor instance, with custom namespace") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + val appName = "testName" + val executorId = "1" + conf.set("spark.app.id", appId) + conf.set("spark.app.name", appName) + conf.set("spark.executor.id", executorId) + conf.set(METRICS_NAMESPACE, "${spark.app.name}") + + val instanceName = "executor" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName === s"$appName.$executorId.${source.sourceName}") + } + + test("MetricsSystem with Executor instance, custom namespace which is not set") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val executorId = "1" + val namespaceToResolve = "${spark.doesnotexist}" + conf.set("spark.executor.id", executorId) + conf.set(METRICS_NAMESPACE, namespaceToResolve) + + val instanceName = "executor" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + // If the user set the spark.metrics.namespace property to an expansion of another property + // (say ${spark.doesnotexist}, the unresolved name (i.e. literally ${spark.doesnotexist}) + // is used as the root logger name. + assert(metricName === s"$namespaceToResolve.$executorId.${source.sourceName}") + } + + test("MetricsSystem with Executor instance, custom namespace, spark.executor.id not set") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + conf.set("spark.app.name", appId) + conf.set(METRICS_NAMESPACE, "${spark.app.name}") + + val instanceName = "executor" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName === source.sourceName) + } + + test("MetricsSystem with non-driver, non-executor instance with custom namespace") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + val appName = "testName" + val executorId = "dummyExecutorId" + conf.set("spark.app.id", appId) + conf.set("spark.app.name", appName) + conf.set(METRICS_NAMESPACE, "${spark.app.name}") + conf.set("spark.executor.id", executorId) + + val instanceName = "testInstance" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + + // Even if spark.app.id and spark.executor.id are set, they are not used for the metric name. + assert(metricName != s"$appId.$executorId.${source.sourceName}") + assert(metricName === source.sourceName) + } + } diff --git a/core/src/test/scala/org/apache/spark/metrics/sink/StatsdSinkSuite.scala b/core/src/test/scala/org/apache/spark/metrics/sink/StatsdSinkSuite.scala new file mode 100644 index 000000000000..0e21a36071c4 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/metrics/sink/StatsdSinkSuite.scala @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.metrics.sink + +import java.net.{DatagramPacket, DatagramSocket} +import java.nio.charset.StandardCharsets.UTF_8 +import java.util.Properties +import java.util.concurrent.TimeUnit._ + +import com.codahale.metrics._ + +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.metrics.sink.StatsdSink._ + +class StatsdSinkSuite extends SparkFunSuite { + private val securityMgr = new SecurityManager(new SparkConf(false)) + private val defaultProps = Map( + STATSD_KEY_PREFIX -> "spark", + STATSD_KEY_PERIOD -> "1", + STATSD_KEY_UNIT -> "seconds", + STATSD_KEY_HOST -> "127.0.0.1" + ) + private val socketTimeout = 30000 // milliseconds + private val socketBufferSize = 8192 + + private def withSocketAndSink(testCode: (DatagramSocket, StatsdSink) => Any): Unit = { + val socket = new DatagramSocket + socket.setReceiveBufferSize(socketBufferSize) + socket.setSoTimeout(socketTimeout) + val props = new Properties + defaultProps.foreach(e => props.put(e._1, e._2)) + props.put(STATSD_KEY_PORT, socket.getLocalPort.toString) + val registry = new MetricRegistry + val sink = new StatsdSink(props, registry, securityMgr) + try { + testCode(socket, sink) + } finally { + socket.close() + } + } + + test("metrics StatsD sink with Counter") { + withSocketAndSink { (socket, sink) => + val counter = new Counter + counter.inc(12) + sink.registry.register("counter", counter) + sink.report() + + val p = new DatagramPacket(new Array[Byte](socketBufferSize), socketBufferSize) + socket.receive(p) + + val result = new String(p.getData, 0, p.getLength, UTF_8) + assert(result === "spark.counter:12|c", "Counter metric received should match data sent") + } + } + + test("metrics StatsD sink with Gauge") { + withSocketAndSink { (socket, sink) => + val gauge = new Gauge[Double] { + override def getValue: Double = 1.23 + } + sink.registry.register("gauge", gauge) + sink.report() + + val p = new DatagramPacket(new Array[Byte](socketBufferSize), socketBufferSize) + socket.receive(p) + + val result = new String(p.getData, 0, p.getLength, UTF_8) + assert(result === "spark.gauge:1.23|g", "Gauge metric received should match data sent") + } + } + + test("metrics StatsD sink with Histogram") { + withSocketAndSink { (socket, sink) => + val p = new DatagramPacket(new Array[Byte](socketBufferSize), socketBufferSize) + val histogram = new Histogram(new UniformReservoir) + histogram.update(10) + histogram.update(20) + histogram.update(30) + sink.registry.register("histogram", histogram) + sink.report() + + val expectedResults = Set( + "spark.histogram.count:3|g", + "spark.histogram.max:30|ms", + "spark.histogram.mean:20.00|ms", + "spark.histogram.min:10|ms", + "spark.histogram.stddev:10.00|ms", + "spark.histogram.p50:20.00|ms", + "spark.histogram.p75:30.00|ms", + "spark.histogram.p95:30.00|ms", + "spark.histogram.p98:30.00|ms", + "spark.histogram.p99:30.00|ms", + "spark.histogram.p999:30.00|ms" + ) + + (1 to expectedResults.size).foreach { i => + socket.receive(p) + val result = new String(p.getData, 0, p.getLength, UTF_8) + logInfo(s"Received histogram result $i: '$result'") + assert(expectedResults.contains(result), + "Histogram metric received should match data sent") + } + } + } + + test("metrics StatsD sink with Timer") { + withSocketAndSink { (socket, sink) => + val p = new DatagramPacket(new Array[Byte](socketBufferSize), socketBufferSize) + val timer = new Timer() + timer.update(1, SECONDS) + timer.update(2, SECONDS) + timer.update(3, SECONDS) + sink.registry.register("timer", timer) + sink.report() + + val expectedResults = Set( + "spark.timer.max:3000.00|ms", + "spark.timer.mean:2000.00|ms", + "spark.timer.min:1000.00|ms", + "spark.timer.stddev:816.50|ms", + "spark.timer.p50:2000.00|ms", + "spark.timer.p75:3000.00|ms", + "spark.timer.p95:3000.00|ms", + "spark.timer.p98:3000.00|ms", + "spark.timer.p99:3000.00|ms", + "spark.timer.p999:3000.00|ms", + "spark.timer.count:3|g", + "spark.timer.m1_rate:0.00|ms", + "spark.timer.m5_rate:0.00|ms", + "spark.timer.m15_rate:0.00|ms" + ) + // mean rate varies on each test run + val oneMoreResult = """spark.timer.mean_rate:\d+\.\d\d\|ms""" + + (1 to (expectedResults.size + 1)).foreach { i => + socket.receive(p) + val result = new String(p.getData, 0, p.getLength, UTF_8) + logInfo(s"Received timer result $i: '$result'") + assert(expectedResults.contains(result) || result.matches(oneMoreResult), + "Timer metric received should match data sent") + } + } + } +} + diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index 3940527fb874..21138bd4a16b 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -19,24 +19,26 @@ package org.apache.spark.network.netty import java.io.InputStreamReader import java.nio._ -import java.nio.charset.Charset +import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit +import scala.concurrent.Promise import scala.concurrent.duration._ -import scala.concurrent.{Await, Promise} import scala.util.{Failure, Success, Try} import com.google.common.io.CharStreams +import org.mockito.Mockito._ +import org.scalatest.Matchers +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.network.{BlockDataManager, BlockTransferService} import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.shuffle.BlockFetchingListener -import org.apache.spark.network.{BlockDataManager, BlockTransferService} import org.apache.spark.storage.{BlockId, ShuffleBlockId} -import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.mockito.Mockito._ -import org.scalatest.mock.MockitoSugar -import org.scalatest.ShouldMatchers +import org.apache.spark.util.ThreadUtils -class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar with ShouldMatchers { +class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar with Matchers { test("security default off") { val conf = new SparkConf() .set("spark.app.id", "app-id") @@ -93,6 +95,20 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi } } + test("security with aes encryption") { + val conf = new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good") + .set("spark.app.id", "app-id") + .set("spark.network.crypto.enabled", "true") + .set("spark.network.crypto.saslFallback", "false") + testConnection(conf, conf) match { + case Success(_) => // expected + case Failure(t) => fail(t) + } + } + + /** * Creates two servers with different configurations and sees if they can talk. * Returns Success() if they can transfer a block, and Failure() if the block transfer was failed @@ -102,24 +118,27 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi val blockManager = mock[BlockDataManager] val blockId = ShuffleBlockId(0, 1, 2) val blockString = "Hello, world!" - val blockBuffer = new NioManagedBuffer(ByteBuffer.wrap(blockString.getBytes)) + val blockBuffer = new NioManagedBuffer(ByteBuffer.wrap( + blockString.getBytes(StandardCharsets.UTF_8))) when(blockManager.getBlockData(blockId)).thenReturn(blockBuffer) val securityManager0 = new SecurityManager(conf0) - val exec0 = new NettyBlockTransferService(conf0, securityManager0, numCores = 1) + val exec0 = new NettyBlockTransferService(conf0, securityManager0, "localhost", "localhost", 0, + 1) exec0.init(blockManager) val securityManager1 = new SecurityManager(conf1) - val exec1 = new NettyBlockTransferService(conf1, securityManager1, numCores = 1) + val exec1 = new NettyBlockTransferService(conf1, securityManager1, "localhost", "localhost", 0, + 1) exec1.init(blockManager) val result = fetchBlock(exec0, exec1, "1", blockId) match { case Success(buf) => val actualString = CharStreams.toString( - new InputStreamReader(buf.createInputStream(), Charset.forName("UTF-8"))) + new InputStreamReader(buf.createInputStream(), StandardCharsets.UTF_8)) actualString should equal(blockString) buf.release() - Success() + Success(()) case Failure(t) => Failure(t) } @@ -146,9 +165,9 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { promise.success(data.retain()) } - }) + }, null) - Await.ready(promise.future, FiniteDuration(1000, TimeUnit.MILLISECONDS)) + ThreadUtils.awaitReady(promise.future, FiniteDuration(10, TimeUnit.SECONDS)) promise.future.value.get } } diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala index 6f8e8a7ac603..f7bc3725d727 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferServiceSuite.scala @@ -17,28 +17,35 @@ package org.apache.spark.network.netty -import org.apache.spark.network.BlockDataManager -import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import scala.util.Random + import org.mockito.Mockito.mock import org.scalatest._ +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.network.BlockDataManager + class NettyBlockTransferServiceSuite extends SparkFunSuite with BeforeAndAfterEach - with ShouldMatchers { + with Matchers { private var service0: NettyBlockTransferService = _ private var service1: NettyBlockTransferService = _ override def afterEach() { - if (service0 != null) { - service0.close() - service0 = null - } + try { + if (service0 != null) { + service0.close() + service0 = null + } - if (service1 != null) { - service1.close() - service1 = null + if (service1 != null) { + service1.close() + service1 = null + } + } finally { + super.afterEach() } } @@ -54,28 +61,36 @@ class NettyBlockTransferServiceSuite } test("can bind to a specific port") { - val port = 17634 + val port = 17634 + Random.nextInt(10000) + logInfo("random port for test: " + port) service0 = createService(port) - service0.port should be >= port - service0.port should be <= (port + 10) // avoid testing equality in case of simultaneous tests + verifyServicePort(expectedPort = port, actualPort = service0.port) } test("can bind to a specific port twice and the second increments") { - val port = 17634 + val port = 17634 + Random.nextInt(10000) + logInfo("random port for test: " + port) service0 = createService(port) - service1 = createService(port) - service0.port should be >= port - service0.port should be <= (port + 10) - service1.port should be (service0.port + 1) + verifyServicePort(expectedPort = port, actualPort = service0.port) + service1 = createService(service0.port) + // `service0.port` is occupied, so `service1.port` should not be `service0.port` + verifyServicePort(expectedPort = service0.port + 1, actualPort = service1.port) + } + + private def verifyServicePort(expectedPort: Int, actualPort: Int): Unit = { + actualPort should be >= expectedPort + // avoid testing equality in case of simultaneous tests + // the default value for `spark.port.maxRetries` is 100 under test + actualPort should be <= (expectedPort + 100) } private def createService(port: Int): NettyBlockTransferService = { val conf = new SparkConf() .set("spark.app.id", s"test-${getClass.getName}") - .set("spark.blockManager.port", port.toString) val securityManager = new SecurityManager(conf) val blockDataManager = mock(classOf[BlockDataManager]) - val service = new NettyBlockTransferService(conf, securityManager, numCores = 1) + val service = new NettyBlockTransferService(conf, securityManager, "localhost", "localhost", + port, 1) service.init(blockDataManager) service } diff --git a/core/src/test/scala/org/apache/spark/partial/CountEvaluatorSuite.scala b/core/src/test/scala/org/apache/spark/partial/CountEvaluatorSuite.scala new file mode 100644 index 000000000000..3c1208c2c375 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/partial/CountEvaluatorSuite.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.partial + +import org.apache.spark.SparkFunSuite + +class CountEvaluatorSuite extends SparkFunSuite { + + test("test count 0") { + val evaluator = new CountEvaluator(10, 0.95) + assert(evaluator.currentResult() === new BoundedDouble(0.0, 0.0, 0.0, Double.PositiveInfinity)) + evaluator.merge(1, 0) + assert(evaluator.currentResult() === new BoundedDouble(0.0, 0.0, 0.0, Double.PositiveInfinity)) + } + + test("test count >= 1") { + val evaluator = new CountEvaluator(10, 0.95) + evaluator.merge(1, 1) + assert(evaluator.currentResult() === new BoundedDouble(10.0, 0.95, 5.0, 16.0)) + evaluator.merge(1, 3) + assert(evaluator.currentResult() === new BoundedDouble(20.0, 0.95, 13.0, 28.0)) + evaluator.merge(1, 8) + assert(evaluator.currentResult() === new BoundedDouble(40.0, 0.95, 30.0, 51.0)) + (4 to 10).foreach(_ => evaluator.merge(1, 10)) + assert(evaluator.currentResult() === new BoundedDouble(82.0, 1.0, 82.0, 82.0)) + } + +} diff --git a/core/src/test/scala/org/apache/spark/partial/MeanEvaluatorSuite.scala b/core/src/test/scala/org/apache/spark/partial/MeanEvaluatorSuite.scala new file mode 100644 index 000000000000..eaa1262b4199 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/partial/MeanEvaluatorSuite.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.partial + +import org.apache.spark.SparkFunSuite +import org.apache.spark.util.StatCounter + +class MeanEvaluatorSuite extends SparkFunSuite { + + test("test count 0") { + val evaluator = new MeanEvaluator(10, 0.95) + assert(new BoundedDouble(0.0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity) == + evaluator.currentResult()) + evaluator.merge(1, new StatCounter()) + assert(new BoundedDouble(0.0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity) == + evaluator.currentResult()) + evaluator.merge(1, new StatCounter(Seq(0.0))) + assert(new BoundedDouble(0.0, 0.95, Double.NegativeInfinity, Double.PositiveInfinity) == + evaluator.currentResult()) + } + + test("test count 1") { + val evaluator = new MeanEvaluator(10, 0.95) + evaluator.merge(1, new StatCounter(Seq(1.0))) + assert(new BoundedDouble(1.0, 0.95, Double.NegativeInfinity, Double.PositiveInfinity) == + evaluator.currentResult()) + } + + test("test count > 1") { + val evaluator = new MeanEvaluator(10, 0.95) + evaluator.merge(1, new StatCounter(Seq(1.0))) + evaluator.merge(1, new StatCounter(Seq(3.0))) + assert(new BoundedDouble(2.0, 0.95, -10.706204736174746, 14.706204736174746) == + evaluator.currentResult()) + evaluator.merge(1, new StatCounter(Seq(8.0))) + assert(new BoundedDouble(4.0, 0.95, -4.9566858949231225, 12.956685894923123) == + evaluator.currentResult()) + (4 to 10).foreach(_ => evaluator.merge(1, new StatCounter(Seq(9.0)))) + assert(new BoundedDouble(7.5, 1.0, 7.5, 7.5) == evaluator.currentResult()) + } + +} diff --git a/core/src/test/scala/org/apache/spark/partial/SumEvaluatorSuite.scala b/core/src/test/scala/org/apache/spark/partial/SumEvaluatorSuite.scala new file mode 100644 index 000000000000..e212db73627e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/partial/SumEvaluatorSuite.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.partial + +import org.apache.spark.SparkFunSuite +import org.apache.spark.util.StatCounter + +class SumEvaluatorSuite extends SparkFunSuite { + + test("correct handling of count 1") { + // sanity check: + assert(new BoundedDouble(2.0, 0.95, 1.1, 1.2) == new BoundedDouble(2.0, 0.95, 1.1, 1.2)) + + // count of 10 because it's larger than 1, + // and 0.95 because that's the default + val evaluator = new SumEvaluator(10, 0.95) + // arbitrarily assign id 1 + evaluator.merge(1, new StatCounter(Seq(2.0))) + assert(new BoundedDouble(20.0, 0.95, Double.NegativeInfinity, Double.PositiveInfinity) == + evaluator.currentResult()) + } + + test("correct handling of count 0") { + val evaluator = new SumEvaluator(10, 0.95) + evaluator.merge(1, new StatCounter()) + assert(new BoundedDouble(0, 0.0, Double.NegativeInfinity, Double.PositiveInfinity) == + evaluator.currentResult()) + } + + test("correct handling of NaN") { + val evaluator = new SumEvaluator(10, 0.95) + evaluator.merge(1, new StatCounter(Seq(1, Double.NaN, 2))) + val res = evaluator.currentResult() + // assert - note semantics of == in face of NaN + assert(res.mean.isNaN) + assert(res.confidence == 0.95) + assert(res.low == Double.NegativeInfinity) + assert(res.high == Double.PositiveInfinity) + } + + test("correct handling of > 1 values") { + val evaluator = new SumEvaluator(10, 0.95) + evaluator.merge(1, new StatCounter(Seq(1.0, 3.0, 2.0))) + val res = evaluator.currentResult() + assert(new BoundedDouble(60.0, 0.95, -101.7362525347778, 221.7362525347778) == + evaluator.currentResult()) + } + + test("test count > 1") { + val evaluator = new SumEvaluator(10, 0.95) + evaluator.merge(1, new StatCounter().merge(1.0)) + evaluator.merge(1, new StatCounter().merge(3.0)) + assert(new BoundedDouble(20.0, 0.95, -186.4513905077019, 226.4513905077019) == + evaluator.currentResult()) + evaluator.merge(1, new StatCounter().merge(8.0)) + assert(new BoundedDouble(40.0, 0.95, -72.75723361226733, 152.75723361226733) == + evaluator.currentResult()) + (4 to 10).foreach(_ => evaluator.merge(1, new StatCounter().merge(9.0))) + assert(new BoundedDouble(75.0, 1.0, 75.0, 75.0) == evaluator.currentResult()) + } + +} diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index ec99f2a1bad6..de0e71a332f2 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -19,27 +19,33 @@ package org.apache.spark.rdd import java.util.concurrent.Semaphore -import scala.concurrent.{Await, TimeoutException} -import scala.concurrent.duration.Duration +import scala.concurrent._ import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration.Duration import org.scalatest.BeforeAndAfterAll -import org.scalatest.concurrent.Timeouts +import org.scalatest.concurrent.TimeLimits import org.scalatest.time.SpanSugar._ -import org.apache.spark.{LocalSparkContext, SparkContext, SparkException, SparkFunSuite} +import org.apache.spark._ +import org.apache.spark.util.ThreadUtils -class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Timeouts { +class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with TimeLimits { @transient private var sc: SparkContext = _ override def beforeAll() { + super.beforeAll() sc = new SparkContext("local[2]", "test") } override def afterAll() { - LocalSparkContext.stop(sc) - sc = null + try { + LocalSparkContext.stop(sc) + sc = null + } finally { + super.afterAll() + } } lazy val zeroPartRdd = new EmptyRDD[Int](sc) @@ -59,9 +65,9 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim test("foreachAsync") { zeroPartRdd.foreachAsync(i => Unit).get() - val accum = sc.accumulator(0) + val accum = sc.longAccumulator sc.parallelize(1 to 1000, 3).foreachAsync { i => - accum += 1 + accum.add(1) }.get() assert(accum.value === 1000) } @@ -69,9 +75,9 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim test("foreachPartitionAsync") { zeroPartRdd.foreachPartitionAsync(iter => Unit).get() - val accum = sc.accumulator(0) + val accum = sc.longAccumulator sc.parallelize(1 to 1000, 9).foreachPartitionAsync { iter => - accum += 1 + accum.add(1) }.get() assert(accum.value === 9) } @@ -124,10 +130,10 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim info("Should not have reached this code path (onComplete matching Failure)") throw new Exception("Task should succeed") } - f.onSuccess { case a: Any => + f.foreach { a => sem.release() } - f.onFailure { case t => + f.failed.foreach { t => info("Should not have reached this code path (onFailure)") throw new Exception("Task should succeed") } @@ -158,11 +164,11 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim case scala.util.Failure(e) => sem.release() } - f.onSuccess { case a: Any => + f.foreach { a => info("Should not have reached this code path (onSuccess)") throw new Exception("Task should fail") } - f.onFailure { case t => + f.failed.foreach { t => sem.release() } intercept[SparkException] { @@ -180,13 +186,13 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim test("FutureAction result, infinite wait") { val f = sc.parallelize(1 to 100, 4) .countAsync() - assert(Await.result(f, Duration.Inf) === 100) + assert(ThreadUtils.awaitResult(f, Duration.Inf) === 100) } test("FutureAction result, finite wait") { val f = sc.parallelize(1 to 100, 4) .countAsync() - assert(Await.result(f, Duration(30, "seconds")) === 100) + assert(ThreadUtils.awaitResult(f, Duration(30, "seconds")) === 100) } test("FutureAction result, timeout") { @@ -194,7 +200,36 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim .mapPartitions(itr => { Thread.sleep(20); itr }) .countAsync() intercept[TimeoutException] { - Await.result(f, Duration(20, "milliseconds")) + ThreadUtils.awaitResult(f, Duration(20, "milliseconds")) + } + } + + private def testAsyncAction[R](action: RDD[Int] => FutureAction[R]): Unit = { + val executionContextInvoked = Promise[Unit] + val fakeExecutionContext = new ExecutionContext { + override def execute(runnable: Runnable): Unit = { + executionContextInvoked.success(()) + } + override def reportFailure(t: Throwable): Unit = () } + val starter = Smuggle(new Semaphore(0)) + starter.drainPermits() + val rdd = sc.parallelize(1 to 100, 4).mapPartitions {itr => starter.acquire(1); itr} + val f = action(rdd) + f.onComplete(_ => ())(fakeExecutionContext) + // Here we verify that registering the callback didn't cause a thread to be consumed. + assert(!executionContextInvoked.isCompleted) + // Now allow the executors to proceed with task processing. + starter.release(rdd.partitions.length) + // Waiting for the result verifies that the tasks were successfully processed. + ThreadUtils.awaitResult(executionContextInvoked.future, atMost = 15.seconds) + } + + test("SimpleFutureAction callback must not consume a thread while waiting") { + testAsyncAction(_.countAsync()) + } + + test("ComplexFutureAction callback must not consume a thread while waiting") { + testAsyncAction((_.takeAsync(100))) } } diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index 4e72b89bfcc4..864adddad342 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -164,8 +164,8 @@ class DoubleRDDSuite extends SparkFunSuite with SharedSparkContext { val expectedHistogramResults = Array(4, 2, 1, 2, 3) assert(histogramResults === expectedHistogramResults) } - // Make sure this works with a NaN end bucket and an inifity - test("WorksMixedRangeWithUnevenBucketsAndNaNAndNaNRangeAndInfity") { + // Make sure this works with a NaN end bucket and an infinity + test("WorksMixedRangeWithUnevenBucketsAndNaNAndNaNRangeAndInfinity") { // Make sure that it works with two unequally spaced buckets and elements in each val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1, 1.0/0.0, -1.0/0.0, Double.NaN)) @@ -178,7 +178,7 @@ class DoubleRDDSuite extends SparkFunSuite with SharedSparkContext { test("WorksWithOutOfRangeWithInfiniteBuckets") { // Verify that out of range works with two buckets val rdd = sc.parallelize(Seq(10.01, -0.01, Double.NaN)) - val buckets = Array(-1.0/0.0 , 0.0, 1.0/0.0) + val buckets = Array(-1.0/0.0, 0.0, 1.0/0.0) val histogramResults = rdd.histogram(buckets) val expectedHistogramResults = Array(1, 1) assert(histogramResults === expectedHistogramResults) diff --git a/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala b/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala index 5103eb74b245..478f0690f8e4 100644 --- a/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala @@ -17,9 +17,12 @@ package org.apache.spark.rdd -import org.apache.spark.{SparkException, SparkContext, LocalSparkContext, SparkFunSuite} +import scala.concurrent.duration._ +import scala.language.postfixOps -import org.mockito.Mockito.spy +import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} + +import org.apache.spark.{LocalSparkContext, SparkContext, SparkException, SparkFunSuite} import org.apache.spark.storage.{RDDBlockId, StorageLevel} /** @@ -29,6 +32,7 @@ import org.apache.spark.storage.{RDDBlockId, StorageLevel} class LocalCheckpointSuite extends SparkFunSuite with LocalSparkContext { override def beforeEach(): Unit = { + super.beforeEach() sc = new SparkContext("local[2]", "test") } @@ -45,10 +49,6 @@ class LocalCheckpointSuite extends SparkFunSuite with LocalSparkContext { assert(transform(StorageLevel.MEMORY_AND_DISK_SER) === StorageLevel.MEMORY_AND_DISK_SER) assert(transform(StorageLevel.MEMORY_AND_DISK_2) === StorageLevel.MEMORY_AND_DISK_2) assert(transform(StorageLevel.MEMORY_AND_DISK_SER_2) === StorageLevel.MEMORY_AND_DISK_SER_2) - // Off-heap is not supported and Spark should fail fast - intercept[SparkException] { - transform(StorageLevel.OFF_HEAP) - } } test("basic lineage truncation") { @@ -173,6 +173,10 @@ class LocalCheckpointSuite extends SparkFunSuite with LocalSparkContext { // Collecting the RDD should now fail with an informative exception val blockId = RDDBlockId(rdd.id, numPartitions - 1) bmm.removeBlock(blockId) + // Wait until the block has been removed successfully. + eventually(timeout(1 seconds), interval(100 milliseconds)) { + assert(bmm.getBlockStatus(blockId).isEmpty) + } try { rdd.collect() fail("Collect should have failed if local checkpoint block is removed...") diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 1321ec84735b..44dd955ce869 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -17,18 +17,22 @@ package org.apache.spark.rdd -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.mapred._ -import org.apache.hadoop.util.Progressable +import java.io.IOException import scala.collection.mutable.{ArrayBuffer, HashSet} import scala.util.Random +import org.apache.commons.math3.distribution.{BinomialDistribution, PoissonDistribution} import org.apache.hadoop.conf.{Configurable, Configuration} -import org.apache.hadoop.mapreduce.{JobContext => NewJobContext, OutputCommitter => NewOutputCommitter, -OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, -TaskAttemptContext => NewTaskAttempContext} -import org.apache.spark.{Partitioner, SharedSparkContext, SparkFunSuite} +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.mapred._ +import org.apache.hadoop.mapreduce.{JobContext => NewJobContext, + OutputCommitter => NewOutputCommitter, OutputFormat => NewOutputFormat, + RecordWriter => NewRecordWriter, TaskAttemptContext => NewTaskAttempContext} +import org.apache.hadoop.util.Progressable + +import org.apache.spark._ +import org.apache.spark.Partitioner import org.apache.spark.util.Utils class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { @@ -178,7 +182,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { assert(sums(2) === 1) } - test("reduceByKey with many output partitons") { + test("reduceByKey with many output partitions") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) val sums = pairs.reduceByKey(_ + _, 10).collect() assert(sums.toSet === Set((1, 7), (2, 1))) @@ -512,10 +516,10 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { pairs.saveAsNewAPIHadoopFile[NewFakeFormat]("ignored") /* - Check that configurable formats get configured: - ConfigTestFormat throws an exception if we try to write - to it when setConf hasn't been called first. - Assertion is in ConfigTestFormat.getRecordWriter. + * Check that configurable formats get configured: + * ConfigTestFormat throws an exception if we try to write + * to it when setConf hasn't been called first. + * Assertion is in ConfigTestFormat.getRecordWriter. */ pairs.saveAsNewAPIHadoopFile[ConfigTestFormat]("ignored") } @@ -532,6 +536,38 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { assert(FakeOutputCommitter.ran, "OutputCommitter was never called") } + test("failure callbacks should be called before calling writer.close() in saveNewAPIHadoopFile") { + val pairs = sc.parallelize(Array((new Integer(1), new Integer(2))), 1) + + FakeWriterWithCallback.calledBy = "" + FakeWriterWithCallback.exception = null + val e = intercept[SparkException] { + pairs.saveAsNewAPIHadoopFile[NewFakeFormatWithCallback]("ignored") + } + assert(e.getCause.getMessage contains "failed to write") + + assert(FakeWriterWithCallback.calledBy === "write,callback,close") + assert(FakeWriterWithCallback.exception != null, "exception should be captured") + assert(FakeWriterWithCallback.exception.getMessage contains "failed to write") + } + + test("failure callbacks should be called before calling writer.close() in saveAsHadoopFile") { + val pairs = sc.parallelize(Array((new Integer(1), new Integer(2))), 1) + val conf = new JobConf() + + FakeWriterWithCallback.calledBy = "" + FakeWriterWithCallback.exception = null + val e = intercept[SparkException] { + pairs.saveAsHadoopFile( + "ignored", pairs.keyClass, pairs.valueClass, classOf[FakeFormatWithCallback], conf) + } + assert(e.getCause.getMessage contains "failed to write") + + assert(FakeWriterWithCallback.calledBy === "write,callback,close") + assert(FakeWriterWithCallback.exception != null, "exception should be captured") + assert(FakeWriterWithCallback.exception.getMessage contains "failed to write") + } + test("lookup") { val pairs = sc.parallelize(Array((1, 2), (3, 4), (5, 6), (5, 7))) @@ -578,17 +614,36 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { (x: Int) => if (x % 10 < (10 * fractionPositive).toInt) "1" else "0" } - def checkSize(exact: Boolean, - withReplacement: Boolean, - expected: Long, - actual: Long, - p: Double): Boolean = { + def assertBinomialSample( + exact: Boolean, + actual: Int, + trials: Int, + p: Double): Unit = { + if (exact) { + assert(actual == math.ceil(p * trials).toInt) + } else { + val dist = new BinomialDistribution(trials, p) + val q = dist.cumulativeProbability(actual) + withClue(s"p = $p: trials = $trials") { + assert(q >= 0.001 && q <= 0.999) + } + } + } + + def assertPoissonSample( + exact: Boolean, + actual: Int, + trials: Int, + p: Double): Unit = { if (exact) { - return expected == actual + assert(actual == math.ceil(p * trials).toInt) + } else { + val dist = new PoissonDistribution(p * trials) + val q = dist.cumulativeProbability(actual) + withClue(s"p = $p: trials = $trials") { + assert(q >= 0.001 && q <= 0.999) + } } - val stdev = if (withReplacement) math.sqrt(expected) else math.sqrt(expected * p * (1 - p)) - // Very forgiving margin since we're dealing with very small sample sizes most of the time - math.abs(actual - expected) <= 6 * stdev } def testSampleExact(stratifiedData: RDD[(String, Int)], @@ -613,8 +668,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { samplingRate: Double, seed: Long, n: Long): Unit = { - val expectedSampleSize = stratifiedData.countByKey() - .mapValues(count => math.ceil(count * samplingRate).toInt) + val trials = stratifiedData.countByKey() val fractions = Map("1" -> samplingRate, "0" -> samplingRate) val sample = if (exact) { stratifiedData.sampleByKeyExact(false, fractions, seed) @@ -623,8 +677,10 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { } val sampleCounts = sample.countByKey() val takeSample = sample.collect() - sampleCounts.foreach { case(k, v) => - assert(checkSize(exact, false, expectedSampleSize(k), v, samplingRate)) } + sampleCounts.foreach { case (k, v) => + assertBinomialSample(exact = exact, actual = v.toInt, trials = trials(k).toInt, + p = samplingRate) + } assert(takeSample.size === takeSample.toSet.size) takeSample.foreach { x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]") } } @@ -635,6 +691,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { samplingRate: Double, seed: Long, n: Long): Unit = { + val trials = stratifiedData.countByKey() val expectedSampleSize = stratifiedData.countByKey().mapValues(count => math.ceil(count * samplingRate).toInt) val fractions = Map("1" -> samplingRate, "0" -> samplingRate) @@ -646,7 +703,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { val sampleCounts = sample.countByKey() val takeSample = sample.collect() sampleCounts.foreach { case (k, v) => - assert(checkSize(exact, true, expectedSampleSize(k), v, samplingRate)) + assertPoissonSample(exact, actual = v.toInt, trials = trials(k).toInt, p = samplingRate) } val groupedByKey = takeSample.groupBy(_._1) for ((key, v) <- groupedByKey) { @@ -657,7 +714,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { if (exact) { assert(v.toSet.size <= expectedSampleSize(key)) } else { - assert(checkSize(false, true, expectedSampleSize(key), v.toSet.size, samplingRate)) + assertPoissonSample(false, actual = v.toSet.size, trials(key).toInt, p = samplingRate) } } } @@ -668,8 +725,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { } /* - These classes are fakes for testing - "saveNewAPIHadoopFile should call setConf if format is configurable". + These classes are fakes for testing saveAsHadoopFile/saveNewAPIHadoopFile. Unfortunately, they have to be top level classes, and not defined in the test method, because otherwise Scala won't generate no-args constructors and the test will therefore throw InstantiationException when saveAsNewAPIHadoopFile @@ -754,6 +810,60 @@ class NewFakeFormat() extends NewOutputFormat[Integer, Integer]() { } } +object FakeWriterWithCallback { + var calledBy: String = "" + var exception: Throwable = _ + + def onFailure(ctx: TaskContext, e: Throwable): Unit = { + calledBy += "callback," + exception = e + } +} + +class FakeWriterWithCallback extends FakeWriter { + + override def close(p1: Reporter): Unit = { + FakeWriterWithCallback.calledBy += "close" + } + + override def write(p1: Integer, p2: Integer): Unit = { + FakeWriterWithCallback.calledBy += "write," + TaskContext.get().addTaskFailureListener { (t: TaskContext, e: Throwable) => + FakeWriterWithCallback.onFailure(t, e) + } + throw new IOException("failed to write") + } +} + +class FakeFormatWithCallback() extends FakeOutputFormat { + override def getRecordWriter( + ignored: FileSystem, + job: JobConf, name: String, + progress: Progressable): RecordWriter[Integer, Integer] = { + new FakeWriterWithCallback() + } +} + +class NewFakeWriterWithCallback extends NewFakeWriter { + override def close(p1: NewTaskAttempContext): Unit = { + FakeWriterWithCallback.calledBy += "close" + } + + override def write(p1: Integer, p2: Integer): Unit = { + FakeWriterWithCallback.calledBy += "write," + TaskContext.get().addTaskFailureListener { (t: TaskContext, e: Throwable) => + FakeWriterWithCallback.onFailure(t, e) + } + throw new IOException("failed to write") + } +} + +class NewFakeFormatWithCallback() extends NewFakeFormat { + override def getRecordWriter(p1: NewTaskAttempContext): NewRecordWriter[Integer, Integer] = { + new NewFakeWriterWithCallback() + } +} + class ConfigTestFormat() extends NewFakeFormat() with Configurable { var setConfCalled = false diff --git a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala index e7cc1617cdf1..31ce9483cf20 100644 --- a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala @@ -101,7 +101,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers { val data = 1 until 100 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_ + _) === 99) + assert(slices.map(_.size).sum === 99) assert(slices.forall(_.isInstanceOf[Range])) } @@ -109,7 +109,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers { val data = 1 to 100 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_ + _) === 100) + assert(slices.map(_.size).sum === 100) assert(slices.forall(_.isInstanceOf[Range])) } @@ -202,7 +202,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers { val data = 1L until 100L val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_ + _) === 99) + assert(slices.map(_.size).sum === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } @@ -210,7 +210,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers { val data = 1L to 100L val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_ + _) === 100) + assert(slices.map(_.size).sum === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } @@ -218,7 +218,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers { val data = 1.0 until 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_ + _) === 99) + assert(slices.map(_.size).sum === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } @@ -226,7 +226,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers { val data = 1.0 to 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_ + _) === 100) + assert(slices.map(_.size).sum === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala index 132a5fa9a80f..cb0de1c6beb6 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala @@ -29,6 +29,8 @@ class MockSampler extends RandomSampler[Long, Long] { s = seed } + override def sample(): Int = 1 + override def sample(items: Iterator[Long]): Iterator[Long] = { Iterator(s) } diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index 5f73ec867596..1a0eb250e7cd 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -19,133 +19,183 @@ package org.apache.spark.rdd import java.io.File +import scala.collection.Map +import scala.io.Codec + import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapred.{FileSplit, JobConf, TextInputFormat} -import scala.collection.Map -import scala.language.postfixOps -import scala.sys.process._ -import scala.util.Try - import org.apache.spark._ import org.apache.spark.util.Utils class PipedRDDSuite extends SparkFunSuite with SharedSparkContext { + val envCommand = if (Utils.isWindows) { + "cmd.exe /C set" + } else { + "printenv" + } test("basic pipe") { - if (testCommandAvailable("cat")) { - val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + assume(TestUtils.testCommandAvailable("cat")) + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val piped = nums.pipe(Seq("cat")) + val piped = nums.pipe(Seq("cat")) + val c = piped.collect() + assert(c.size === 4) + assert(c(0) === "1") + assert(c(1) === "2") + assert(c(2) === "3") + assert(c(3) === "4") + } + + test("basic pipe with tokenization") { + assume(TestUtils.testCommandAvailable("wc")) + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + + // verify that both RDD.pipe(command: String) and RDD.pipe(command: String, env) work good + for (piped <- Seq(nums.pipe("wc -l"), nums.pipe("wc -l", Map[String, String]()))) { val c = piped.collect() - assert(c.size === 4) - assert(c(0) === "1") - assert(c(1) === "2") - assert(c(2) === "3") - assert(c(3) === "4") - } else { - assert(true) + assert(c.size === 2) + assert(c(0).trim === "2") + assert(c(1).trim === "2") } } - test("advanced pipe") { - if (testCommandAvailable("cat")) { - val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val bl = sc.broadcast(List("0")) + test("failure in iterating over pipe input") { + assume(TestUtils.testCommandAvailable("cat")) + val nums = + sc.makeRDD(Array(1, 2, 3, 4), 2) + .mapPartitionsWithIndex((index, iterator) => { + new Iterator[Int] { + def hasNext = true + def next() = { + throw new SparkException("Exception to simulate bad scenario") + } + } + }) + + val piped = nums.pipe(Seq("cat")) - val piped = nums.pipe(Seq("cat"), + intercept[SparkException] { + piped.collect() + } + } + + test("advanced pipe") { + assume(TestUtils.testCommandAvailable("cat")) + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + val bl = sc.broadcast(List("0")) + + val piped = nums.pipe(Seq("cat"), + Map[String, String](), + (f: String => Unit) => { + bl.value.foreach(f); f("\u0001") + }, + (i: Int, f: String => Unit) => f(i + "_")) + + val c = piped.collect() + + assert(c.size === 8) + assert(c(0) === "0") + assert(c(1) === "\u0001") + assert(c(2) === "1_") + assert(c(3) === "2_") + assert(c(4) === "0") + assert(c(5) === "\u0001") + assert(c(6) === "3_") + assert(c(7) === "4_") + + val nums1 = sc.makeRDD(Array("a\t1", "b\t2", "a\t3", "b\t4"), 2) + val d = nums1.groupBy(str => str.split("\t")(0)). + pipe(Seq("cat"), Map[String, String](), (f: String => Unit) => { - bl.value.map(f(_)); f("\u0001") + bl.value.foreach(f); f("\u0001") }, - (i: Int, f: String => Unit) => f(i + "_")) - - val c = piped.collect() + (i: Tuple2[String, Iterable[String]], f: String => Unit) => { + for (e <- i._2) { + f(e + "_") + } + }).collect() + assert(d.size === 8) + assert(d(0) === "0") + assert(d(1) === "\u0001") + assert(d(2) === "b\t2_") + assert(d(3) === "b\t4_") + assert(d(4) === "0") + assert(d(5) === "\u0001") + assert(d(6) === "a\t1_") + assert(d(7) === "a\t3_") + } - assert(c.size === 8) - assert(c(0) === "0") - assert(c(1) === "\u0001") - assert(c(2) === "1_") - assert(c(3) === "2_") - assert(c(4) === "0") - assert(c(5) === "\u0001") - assert(c(6) === "3_") - assert(c(7) === "4_") - - val nums1 = sc.makeRDD(Array("a\t1", "b\t2", "a\t3", "b\t4"), 2) - val d = nums1.groupBy(str => str.split("\t")(0)). - pipe(Seq("cat"), - Map[String, String](), - (f: String => Unit) => { - bl.value.map(f(_)); f("\u0001") - }, - (i: Tuple2[String, Iterable[String]], f: String => Unit) => { - for (e <- i._2) { - f(e + "_") - } - }).collect() - assert(d.size === 8) - assert(d(0) === "0") - assert(d(1) === "\u0001") - assert(d(2) === "b\t2_") - assert(d(3) === "b\t4_") - assert(d(4) === "0") - assert(d(5) === "\u0001") - assert(d(6) === "a\t1_") - assert(d(7) === "a\t3_") + test("pipe with empty partition") { + val data = sc.parallelize(Seq("foo", "bing"), 8) + val piped = data.pipe("wc -c") + assert(piped.count == 8) + val charCounts = piped.map(_.trim.toInt).collect().toSet + val expected = if (Utils.isWindows) { + // Note that newline character on Windows is \r\n which are two. + Set(0, 5, 6) } else { - assert(true) + Set(0, 4, 5) } + assert(expected == charCounts) } test("pipe with env variable") { - if (testCommandAvailable("printenv")) { - val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val piped = nums.pipe(Seq("printenv", "MY_TEST_ENV"), Map("MY_TEST_ENV" -> "LALALA")) - val c = piped.collect() - assert(c.size === 2) - assert(c(0) === "LALALA") - assert(c(1) === "LALALA") - } else { - assert(true) + assume(TestUtils.testCommandAvailable(envCommand)) + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + val piped = nums.pipe(s"$envCommand MY_TEST_ENV", Map("MY_TEST_ENV" -> "LALALA")) + val c = piped.collect() + assert(c.length === 2) + // On Windows, `cmd.exe /C set` is used which prints out it as `varname=value` format + // whereas `printenv` usually prints out `value`. So, `varname=` is stripped here for both. + assert(c(0).stripPrefix("MY_TEST_ENV=") === "LALALA") + assert(c(1).stripPrefix("MY_TEST_ENV=") === "LALALA") + } + + test("pipe with process which cannot be launched due to bad command") { + assume(!TestUtils.testCommandAvailable("some_nonexistent_command")) + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + val command = Seq("some_nonexistent_command") + val piped = nums.pipe(command) + val exception = intercept[SparkException] { + piped.collect() } + assert(exception.getMessage.contains(command.mkString(" "))) } - test("pipe with non-zero exit status") { - if (testCommandAvailable("cat")) { - val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val piped = nums.pipe(Seq("cat nonexistent_file", "2>", "/dev/null")) - intercept[SparkException] { - piped.collect() - } - } else { - assert(true) + test("pipe with process which is launched but fails with non-zero exit status") { + assume(TestUtils.testCommandAvailable("cat")) + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + val command = Seq("cat", "nonexistent_file") + val piped = nums.pipe(command) + val exception = intercept[SparkException] { + piped.collect() } + assert(exception.getMessage.contains(command.mkString(" "))) } test("basic pipe with separate working directory") { - if (testCommandAvailable("cat")) { - val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val piped = nums.pipe(Seq("cat"), separateWorkingDir = true) - val c = piped.collect() - assert(c.size === 4) - assert(c(0) === "1") - assert(c(1) === "2") - assert(c(2) === "3") - assert(c(3) === "4") - val pipedPwd = nums.pipe(Seq("pwd"), separateWorkingDir = true) - val collectPwd = pipedPwd.collect() - assert(collectPwd(0).contains("tasks/")) - val pipedLs = nums.pipe(Seq("ls"), separateWorkingDir = true).collect() - // make sure symlinks were created - assert(pipedLs.length > 0) - // clean up top level tasks directory - Utils.deleteRecursively(new File("tasks")) - } else { - assert(true) - } + assume(TestUtils.testCommandAvailable("cat")) + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + val piped = nums.pipe(Seq("cat"), separateWorkingDir = true) + val c = piped.collect() + assert(c.size === 4) + assert(c(0) === "1") + assert(c(1) === "2") + assert(c(2) === "3") + assert(c(3) === "4") + val pipedPwd = nums.pipe(Seq("pwd"), separateWorkingDir = true) + val collectPwd = pipedPwd.collect() + assert(collectPwd(0).contains("tasks/")) + val pipedLs = nums.pipe(Seq("ls"), separateWorkingDir = true, bufferSize = 16384).collect() + // make sure symlinks were created + assert(pipedLs.length > 0) + // clean up top level tasks directory + Utils.deleteRecursively(new File("tasks")) } test("test pipe exports map_input_file") { @@ -156,32 +206,36 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext { testExportInputFile("mapreduce_map_input_file") } - def testCommandAvailable(command: String): Boolean = { - Try(Process(command) !!).isSuccess - } - def testExportInputFile(varName: String) { - if (testCommandAvailable("printenv")) { - val nums = new HadoopRDD(sc, new JobConf(), classOf[TextInputFormat], classOf[LongWritable], - classOf[Text], 2) { - override def getPartitions: Array[Partition] = Array(generateFakeHadoopPartition()) + assume(TestUtils.testCommandAvailable(envCommand)) + val nums = new HadoopRDD(sc, new JobConf(), classOf[TextInputFormat], classOf[LongWritable], + classOf[Text], 2) { + override def getPartitions: Array[Partition] = Array(generateFakeHadoopPartition()) - override val getDependencies = List[Dependency[_]]() + override val getDependencies = List[Dependency[_]]() - override def compute(theSplit: Partition, context: TaskContext) = { - new InterruptibleIterator[(LongWritable, Text)](context, Iterator((new LongWritable(1), - new Text("b")))) - } + override def compute(theSplit: Partition, context: TaskContext) = { + new InterruptibleIterator[(LongWritable, Text)](context, Iterator((new LongWritable(1), + new Text("b")))) } - val hadoopPart1 = generateFakeHadoopPartition() - val pipedRdd = new PipedRDD(nums, "printenv " + varName) - val tContext = TaskContext.empty() - val rddIter = pipedRdd.compute(hadoopPart1, tContext) - val arr = rddIter.toArray - assert(arr(0) == "/some/path") - } else { - // printenv isn't available so just pass the test } + val hadoopPart1 = generateFakeHadoopPartition() + val pipedRdd = + new PipedRDD( + nums, + PipedRDD.tokenize(s"$envCommand $varName"), + Map(), + null, + null, + false, + 4092, + Codec.defaultCharsetCodec.name) + val tContext = TaskContext.empty() + val rddIter = pipedRdd.compute(hadoopPart1, tContext) + val arr = rddIter.toArray + // On Windows, `cmd.exe /C set` is used which prints out it as `varname=value` format + // whereas `printenv` usually prints out `value`. So, `varname=` is stripped here for both. + assert(arr(0).stripPrefix(s"$varName=") === "/some/path") } def generateFakeHadoopPartition(): HadoopPartition = { diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 5f718ea9f7be..e994d724c462 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -17,23 +17,40 @@ package org.apache.spark.rdd -import java.io.{ObjectInputStream, ObjectOutputStream, IOException} +import java.io.{File, IOException, ObjectInputStream, ObjectOutputStream} -import com.esotericsoftware.kryo.KryoException - -import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.JavaConverters._ +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.reflect.ClassTag +import com.esotericsoftware.kryo.KryoException +import org.apache.hadoop.io.{LongWritable, Text} +import org.apache.hadoop.mapred.{FileSplit, TextInputFormat} + import org.apache.spark._ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.rdd.RDDSuiteUtils._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} class RDDSuite extends SparkFunSuite with SharedSparkContext { + var tempDir: File = _ + + override def beforeAll(): Unit = { + super.beforeAll() + tempDir = Utils.createTempDir() + } + + override def afterAll(): Unit = { + try { + Utils.deleteRecursively(tempDir) + } finally { + super.afterAll() + } + } test("basic operations") { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + assert(nums.getNumPartitions === 2) assert(nums.collect().toList === List(1, 2, 3, 4)) assert(nums.toLocalIterator.toList === List(1, 2, 3, 4)) val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) @@ -53,16 +70,16 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assert(!nums.isEmpty()) assert(nums.max() === 4) assert(nums.min() === 1) - val partitionSums = nums.mapPartitions(iter => Iterator(iter.reduceLeft(_ + _))) + val partitionSums = nums.mapPartitions(iter => Iterator(iter.sum)) assert(partitionSums.collect().toList === List(3, 7)) val partitionSumsWithSplit = nums.mapPartitionsWithIndex { - case(split, iter) => Iterator((split, iter.reduceLeft(_ + _))) + case(split, iter) => Iterator((split, iter.sum)) } assert(partitionSumsWithSplit.collect().toList === List((0, 3), (1, 7))) val partitionSumsWithIndex = nums.mapPartitionsWithIndex { - case(split, iter) => Iterator((split, iter.reduceLeft(_ + _))) + case(split, iter) => Iterator((split, iter.sum)) } assert(partitionSumsWithIndex.collect().toList === List((0, 3), (1, 7))) @@ -99,22 +116,39 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assert(sc.union(Seq(nums, nums)).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) } + test("SparkContext.union parallel partition listing") { + val nums1 = sc.makeRDD(Array(1, 2, 3, 4), 2) + val nums2 = sc.makeRDD(Array(5, 6, 7, 8), 2) + val serialUnion = sc.union(nums1, nums2) + val expected = serialUnion.collect().toList + + assert(serialUnion.asInstanceOf[UnionRDD[Int]].isPartitionListingParallel === false) + + sc.conf.set("spark.rdd.parallelListingThreshold", "1") + val parallelUnion = sc.union(nums1, nums2) + val actual = parallelUnion.collect().toList + sc.conf.remove("spark.rdd.parallelListingThreshold") + + assert(parallelUnion.asInstanceOf[UnionRDD[Int]].isPartitionListingParallel === true) + assert(expected === actual) + } + test("SparkContext.union creates UnionRDD if at least one RDD has no partitioner") { - val rddWithPartitioner = sc.parallelize(Seq(1->true)).partitionBy(new HashPartitioner(1)) - val rddWithNoPartitioner = sc.parallelize(Seq(2->true)) + val rddWithPartitioner = sc.parallelize(Seq(1 -> true)).partitionBy(new HashPartitioner(1)) + val rddWithNoPartitioner = sc.parallelize(Seq(2 -> true)) val unionRdd = sc.union(rddWithNoPartitioner, rddWithPartitioner) assert(unionRdd.isInstanceOf[UnionRDD[_]]) } test("SparkContext.union creates PartitionAwareUnionRDD if all RDDs have partitioners") { - val rddWithPartitioner = sc.parallelize(Seq(1->true)).partitionBy(new HashPartitioner(1)) + val rddWithPartitioner = sc.parallelize(Seq(1 -> true)).partitionBy(new HashPartitioner(1)) val unionRdd = sc.union(rddWithPartitioner, rddWithPartitioner) assert(unionRdd.isInstanceOf[PartitionerAwareUnionRDD[_]]) } test("PartitionAwareUnionRDD raises exception if at least one RDD has no partitioner") { - val rddWithPartitioner = sc.parallelize(Seq(1->true)).partitionBy(new HashPartitioner(1)) - val rddWithNoPartitioner = sc.parallelize(Seq(2->true)) + val rddWithPartitioner = sc.parallelize(Seq(1 -> true)).partitionBy(new HashPartitioner(1)) + val rddWithNoPartitioner = sc.parallelize(Seq(2 -> true)) intercept[IllegalArgumentException] { new PartitionerAwareUnionRDD(sc, Seq(rddWithNoPartitioner, rddWithPartitioner)) } @@ -158,6 +192,23 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assert(ser.serialize(union.partitions.head).limit() < 2000) } + test("fold") { + val rdd = sc.makeRDD(-1000 until 1000, 10) + def op: (Int, Int) => Int = (c: Int, x: Int) => c + x + val sum = rdd.fold(0)(op) + assert(sum === -1000) + } + + test("fold with op modifying first arg") { + val rdd = sc.makeRDD(-1000 until 1000, 10).map(x => Array(x)) + def op: (Array[Int], Array[Int]) => Array[Int] = { (c: Array[Int], x: Array[Int]) => + c(0) += x(0) + c + } + val sum = rdd.fold(Array(0))(op) + assert(sum(0) === -1000) + } + test("aggregate") { val pairs = sc.makeRDD(Array(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3))) type StringMap = HashMap[String, Int] @@ -184,7 +235,19 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { def combOp: (Long, Long) => Long = (c1: Long, c2: Long) => c1 + c2 for (depth <- 1 until 10) { val sum = rdd.treeAggregate(0L)(seqOp, combOp, depth) - assert(sum === -1000L) + assert(sum === -1000) + } + } + + test("treeAggregate with ops modifying first args") { + val rdd = sc.makeRDD(-1000 until 1000, 10).map(x => Array(x)) + def op: (Array[Int], Array[Int]) => Array[Int] = { (c: Array[Int], x: Array[Int]) => + c(0) += x(0) + c + } + for (depth <- 1 until 10) { + val sum = rdd.treeAggregate(Array(0))(op, op, depth) + assert(sum(0) === -1000) } } @@ -242,6 +305,10 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { test("repartitioned RDDs") { val data = sc.parallelize(1 to 1000, 10) + intercept[IllegalArgumentException] { + data.repartition(0) + } + // Coalesce partitions val repartitioned1 = data.repartition(2) assert(repartitioned1.partitions.size == 2) @@ -280,21 +347,27 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { val partitions = repartitioned.glom().collect() // assert all elements are present assert(repartitioned.collect().sortWith(_ > _).toSeq === input.toSeq.sortWith(_ > _).toSeq) - // assert no bucket is overloaded + // assert no bucket is overloaded or empty for (partition <- partitions) { val avg = input.size / finalPartitions val maxPossible = avg + initialPartitions - assert(partition.length <= maxPossible) + assert(partition.length <= maxPossible) + assert(!partition.isEmpty) } } testSplitPartitions(Array.fill(100)(1), 10, 20) testSplitPartitions(Array.fill(10000)(1) ++ Array.fill(10000)(2), 20, 100) + testSplitPartitions(Array.fill(1000)(1), 250, 128) } test("coalesced RDDs") { val data = sc.parallelize(1 to 10, 10) + intercept[IllegalArgumentException] { + data.coalesce(0) + } + val coalesced1 = data.coalesce(2) assert(coalesced1.collect().toList === (1 to 10).toList) assert(coalesced1.glom().collect().map(_.toList).toList === @@ -360,6 +433,33 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { map{x => List(x)}.toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back") } + test("coalesced RDDs with partial locality") { + // Make an RDD that has some locality preferences and some without. This can happen + // with UnionRDD + val data = sc.makeRDD((1 to 9).map(i => { + if (i > 4) { + (i, (i to (i + 2)).map { j => "m" + (j % 6) }) + } else { + (i, Vector()) + } + })) + val coalesced1 = data.coalesce(3) + assert(coalesced1.collect().toList.sorted === (1 to 9).toList, "Data got *lost* in coalescing") + + val splits = coalesced1.glom().collect().map(_.toList).toList + assert(splits.length === 3, "Supposed to coalesce to 3 but got " + splits.length) + + assert(splits.forall(_.length >= 1) === true, "Some partitions were empty") + + // If we try to coalesce into more partitions than the original RDD, it should just + // keep the original number of partitions. + val coalesced4 = data.coalesce(20) + val listOfLists = coalesced4.glom().collect().map(_.toList).toList + val sortedList = listOfLists.sortWith{ (x, y) => !x.isEmpty && (y.isEmpty || (x(0) < y(0))) } + assert(sortedList === (1 to 9). + map{x => List(x)}.toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back") + } + test("coalesced RDDs with locality, large scale (10K partitions)") { // large scale experiment import collection.mutable @@ -401,6 +501,48 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { } } + test("coalesced RDDs with partial locality, large scale (10K partitions)") { + // large scale experiment + import collection.mutable + val halfpartitions = 5000 + val partitions = 10000 + val numMachines = 50 + val machines = mutable.ListBuffer[String]() + (1 to numMachines).foreach(machines += "m" + _) + val rnd = scala.util.Random + for (seed <- 1 to 5) { + rnd.setSeed(seed) + + val firstBlocks = (1 to halfpartitions).map { i => + (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) + } + val blocksNoLocality = (halfpartitions + 1 to partitions).map { i => + (i, List()) + } + val blocks = firstBlocks ++ blocksNoLocality + + val data2 = sc.makeRDD(blocks) + + // first try going to same number of partitions + val coalesced2 = data2.coalesce(partitions) + + // test that we have 10000 partitions + assert(coalesced2.partitions.size == 10000, "Expected 10000 partitions, but got " + + coalesced2.partitions.size) + + // test that we have 100 partitions + val coalesced3 = data2.coalesce(numMachines * 2) + assert(coalesced3.partitions.size == 100, "Expected 100 partitions, but got " + + coalesced3.partitions.size) + + // test that the groups are load balanced with 100 +/- 20 elements in each + val maxImbalance3 = coalesced3.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) + .foldLeft(0)((dev, curr) => math.max(math.abs(100 - curr), dev)) + assert(maxImbalance3 <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance3) + } + } + // Test for SPARK-2412 -- ensure that the second pass of the algorithm does not throw an exception test("coalesced RDDs with locality, fail first pass") { val initialPartitions = 1000 @@ -440,66 +582,6 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assert(prunedData(0) === 10) } - test("mapWith") { - import java.util.Random - val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2) - @deprecated("suppress compile time deprecation warning", "1.0.0") - val randoms = ones.mapWith( - (index: Int) => new Random(index + 42)) - {(t: Int, prng: Random) => prng.nextDouble * t}.collect() - val prn42_3 = { - val prng42 = new Random(42) - prng42.nextDouble(); prng42.nextDouble(); prng42.nextDouble() - } - val prn43_3 = { - val prng43 = new Random(43) - prng43.nextDouble(); prng43.nextDouble(); prng43.nextDouble() - } - assert(randoms(2) === prn42_3) - assert(randoms(5) === prn43_3) - } - - test("flatMapWith") { - import java.util.Random - val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2) - @deprecated("suppress compile time deprecation warning", "1.0.0") - val randoms = ones.flatMapWith( - (index: Int) => new Random(index + 42)) - {(t: Int, prng: Random) => - val random = prng.nextDouble() - Seq(random * t, random * t * 10)}. - collect() - val prn42_3 = { - val prng42 = new Random(42) - prng42.nextDouble(); prng42.nextDouble(); prng42.nextDouble() - } - val prn43_3 = { - val prng43 = new Random(43) - prng43.nextDouble(); prng43.nextDouble(); prng43.nextDouble() - } - assert(randoms(5) === prn42_3 * 10) - assert(randoms(11) === prn43_3 * 10) - } - - test("filterWith") { - import java.util.Random - val ints = sc.makeRDD(Array(1, 2, 3, 4, 5, 6), 2) - @deprecated("suppress compile time deprecation warning", "1.0.0") - val sample = ints.filterWith( - (index: Int) => new Random(index + 42)) - {(t: Int, prng: Random) => prng.nextInt(3) == 0}. - collect() - val checkSample = { - val prng42 = new Random(42) - val prng43 = new Random(43) - Array(1, 2, 3, 4, 5, 6).filter{i => - if (i < 4) 0 == prng42.nextInt(3) else 0 == prng43.nextInt(3) - } - } - assert(sample.size === checkSample.size) - for (i <- 0 until sample.size) assert(sample(i) === checkSample(i)) - } - test("collect large number of empty partitions") { // Regression test for SPARK-4019 assert(sc.makeRDD(0 until 10, 1000).repartition(2001).collect().toSet === (0 until 10).toSet) @@ -541,6 +623,10 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assert(nums.take(501) === (1 to 501).toArray) assert(nums.take(999) === (1 to 999).toArray) assert(nums.take(1000) === (1 to 999).toArray) + + nums = sc.parallelize(1 to 2, 2) + assert(nums.take(2147483638).size === 2) + assert(nums.takeAsync(2147483638).get.size === 2) } test("top with predefined ordering") { @@ -631,27 +717,26 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { } { val sample = data.takeSample(withReplacement = true, num = 20) - assert(sample.size === 20) // Got exactly 100 elements - assert(sample.toSet.size <= 20, "sampling with replacement returned all distinct elements") + assert(sample.size === 20) // Got exactly 20 elements assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } { val sample = data.takeSample(withReplacement = true, num = n) - assert(sample.size === n) // Got exactly 100 elements - // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.size === n) // Got exactly n elements + // Chance of getting all distinct elements is astronomically low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") assert(sample.forall(x => 1 <= x && x <= n), s"elements not in [1, $n]") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement = true, n, seed) - assert(sample.size === n) // Got exactly 100 elements - // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.size === n) // Got exactly n elements + // Chance of getting all distinct elements is astronomically low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement = true, 2 * n, seed) - assert(sample.size === 2 * n) // Got exactly 200 elements - // Chance of getting all distinct elements is still quite low, so test we got < 100 + assert(sample.size === 2 * n) // Got exactly 2 * n elements + // Chance of getting all distinct elements is still quite low, so test we got < n assert(sample.toSet.size < n, "sampling with replacement returned all distinct elements") } } @@ -969,6 +1054,24 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { } } + test("RDD.partitions() fails fast when partitions indicies are incorrect (SPARK-13021)") { + class BadRDD[T: ClassTag](prev: RDD[T]) extends RDD[T](prev) { + + override def compute(part: Partition, context: TaskContext): Iterator[T] = { + prev.compute(part, context) + } + + override protected def getPartitions: Array[Partition] = { + prev.partitions.reverse // breaks contract, which is that `rdd.partitions(i).index == i` + } + } + val rdd = new BadRDD(sc.parallelize(1 to 100, 100)) + val e = intercept[IllegalArgumentException] { + rdd.partitions + } + assert(e.getMessage.contains("partitions")) + } + test("nested RDDs are not supported (SPARK-5063)") { val rdd: RDD[Int] = sc.parallelize(1 to 100) val rdd2: RDD[Int] = sc.parallelize(1 to 100) @@ -988,6 +1091,48 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assert(thrown.getMessage.contains("SPARK-5063")) } + test("custom RDD coalescer") { + val maxSplitSize = 512 + val outDir = new File(tempDir, "output").getAbsolutePath + sc.makeRDD(1 to 1000, 10).saveAsTextFile(outDir) + val hadoopRDD = + sc.hadoopFile(outDir, classOf[TextInputFormat], classOf[LongWritable], classOf[Text]) + val coalescedHadoopRDD = + hadoopRDD.coalesce(2, partitionCoalescer = Option(new SizeBasedCoalescer(maxSplitSize))) + assert(coalescedHadoopRDD.partitions.size <= 10) + var totalPartitionCount = 0L + coalescedHadoopRDD.partitions.foreach(partition => { + var splitSizeSum = 0L + partition.asInstanceOf[CoalescedRDDPartition].parents.foreach(partition => { + val split = partition.asInstanceOf[HadoopPartition].inputSplit.value.asInstanceOf[FileSplit] + splitSizeSum += split.getLength + totalPartitionCount += 1 + }) + assert(splitSizeSum <= maxSplitSize) + }) + assert(totalPartitionCount == 10) + } + + test("SPARK-18406: race between end-of-task and completion iterator read lock release") { + val rdd = sc.parallelize(1 to 1000, 10) + rdd.cache() + + rdd.mapPartitions { iter => + ThreadUtils.runInNewThread("TestThread") { + // Iterate to the end of the input iterator, to cause the CompletionIterator completion to + // fire outside of the task's main thread. + while (iter.hasNext) { + iter.next() + } + iter + } + }.collect() + } + + // NOTE + // Below tests calling sc.stop() have to be the last tests in this suite. If there are tests + // running after them and if they access sc those tests will fail as sc is already closed, because + // sc is shared (this suite mixins SharedSparkContext) test("cannot run actions after SparkContext has been stopped (SPARK-5063)") { val existingRDD = sc.parallelize(1 to 100) sc.stop() @@ -1008,5 +1153,60 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assertFails { sc.parallelize(1 to 100) } assertFails { sc.textFile("/nonexistent-path") } } +} +/** + * Coalesces partitions based on their size assuming that the parent RDD is a [[HadoopRDD]]. + * Took this class out of the test suite to prevent "Task not serializable" exceptions. + */ +class SizeBasedCoalescer(val maxSize: Int) extends PartitionCoalescer with Serializable { + override def coalesce(maxPartitions: Int, parent: RDD[_]): Array[PartitionGroup] = { + val partitions: Array[Partition] = parent.asInstanceOf[HadoopRDD[Any, Any]].getPartitions + val groups = ArrayBuffer[PartitionGroup]() + var currentGroup = new PartitionGroup() + var currentSum = 0L + var totalSum = 0L + var index = 0 + + // sort partitions based on the size of the corresponding input splits + partitions.sortWith((partition1, partition2) => { + val partition1Size = partition1.asInstanceOf[HadoopPartition].inputSplit.value.getLength + val partition2Size = partition2.asInstanceOf[HadoopPartition].inputSplit.value.getLength + partition1Size < partition2Size + }) + + def updateGroups(): Unit = { + groups += currentGroup + currentGroup = new PartitionGroup() + currentSum = 0 + } + + def addPartition(partition: Partition, splitSize: Long): Unit = { + currentGroup.partitions += partition + currentSum += splitSize + totalSum += splitSize + } + + while (index < partitions.size) { + val partition = partitions(index) + val fileSplit = + partition.asInstanceOf[HadoopPartition].inputSplit.value.asInstanceOf[FileSplit] + val splitSize = fileSplit.getLength + if (currentSum + splitSize < maxSize) { + addPartition(partition, splitSize) + index += 1 + if (index == partitions.size) { + updateGroups + } + } else { + if (currentGroup.partitions.size == 0) { + addPartition(partition, splitSize) + index += 1 + } else { + updateGroups + } + } + } + groups.toArray + } } diff --git a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala index a7de9cabe7cc..7f20206202cb 100644 --- a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.rdd import org.scalatest.Matchers -import org.apache.spark.{Logging, SharedSparkContext, SparkFunSuite} +import org.apache.spark.{SharedSparkContext, SparkFunSuite} +import org.apache.spark.internal.Logging class SortingSuite extends SparkFunSuite with SharedSparkContext with Matchers with Logging { @@ -134,7 +135,7 @@ class SortingSuite extends SparkFunSuite with SharedSparkContext with Matchers w } test("get a range of elements in an array not partitioned by a range partitioner") { - val pairArr = util.Random.shuffle((1 to 1000).toList).map(x => (x, x)) + val pairArr = scala.util.Random.shuffle((1 to 1000).toList).map(x => (x, x)) val pairs = sc.parallelize(pairArr, 10) val range = pairs.filterByRange(200, 800).collect() assert((800 to 200 by -1).toArray.sorted === range.map(_._1).sorted) diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index 834e4743df86..a799b1cfb076 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -17,18 +17,26 @@ package org.apache.spark.rpc -import java.io.NotSerializableException -import java.util.concurrent.{TimeUnit, CountDownLatch, TimeoutException} +import java.io.{File, NotSerializableException} +import java.nio.charset.StandardCharsets.UTF_8 +import java.util.UUID +import java.util.concurrent.{ConcurrentLinkedQueue, CountDownLatch, TimeUnit} +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.concurrent.Await import scala.concurrent.duration._ import scala.language.postfixOps +import com.google.common.io.Files +import org.mockito.Matchers.any +import org.mockito.Mockito.{mock, never, verify, when} import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ -import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.{SecurityManager, SparkConf, SparkEnv, SparkException, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.util.{ThreadUtils, Utils} /** * Common tests for an RpcEnv implementation. @@ -38,13 +46,23 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { var env: RpcEnv = _ override def beforeAll(): Unit = { + super.beforeAll() val conf = new SparkConf() - env = createRpcEnv(conf, "local", 12345) + env = createRpcEnv(conf, "local", 0) + + val sparkEnv = mock(classOf[SparkEnv]) + when(sparkEnv.rpcEnv).thenReturn(env) + SparkEnv.set(sparkEnv) } override def afterAll(): Unit = { - if (env != null) { - env.shutdown() + try { + if (env != null) { + env.shutdown() + } + SparkEnv.set(null) + } finally { + super.afterAll() } } @@ -76,9 +94,9 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } }) - val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345, clientMode = true) + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0, clientMode = true) // Use anotherEnv to find out the RpcEndpointRef - val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "send-remotely") + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "send-remotely") try { rpcEndpointRef.send("hello") eventually(timeout(5 seconds), interval(10 millis)) { @@ -100,8 +118,8 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } } val rpcEndpointRef = env.setupEndpoint("send-ref", endpoint) - val newRpcEndpointRef = rpcEndpointRef.askWithRetry[RpcEndpointRef]("Hello") - val reply = newRpcEndpointRef.askWithRetry[String]("Echo") + val newRpcEndpointRef = rpcEndpointRef.askSync[RpcEndpointRef]("Hello") + val reply = newRpcEndpointRef.askSync[String]("Echo") assert("Echo" === reply) } @@ -110,12 +128,11 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { override val rpcEnv = env override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case msg: String => { + case msg: String => context.reply(msg) - } } }) - val reply = rpcEndpointRef.askWithRetry[String]("hello") + val reply = rpcEndpointRef.askSync[String]("hello") assert("hello" === reply) } @@ -124,17 +141,16 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { override val rpcEnv = env override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case msg: String => { + case msg: String => context.reply(msg) - } } }) - val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345, clientMode = true) + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0, clientMode = true) // Use anotherEnv to find out the RpcEndpointRef - val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "ask-remotely") + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-remotely") try { - val reply = rpcEndpointRef.askWithRetry[String]("hello") + val reply = rpcEndpointRef.askSync[String]("hello") assert("hello" === reply) } finally { anotherEnv.shutdown() @@ -147,10 +163,9 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { override val rpcEnv = env override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case msg: String => { + case msg: String => Thread.sleep(100) context.reply(msg) - } } }) @@ -158,18 +173,17 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { val shortProp = "spark.rpc.short.timeout" conf.set("spark.rpc.retry.wait", "0") conf.set("spark.rpc.numRetries", "1") - val anotherEnv = createRpcEnv(conf, "remote", 13345, clientMode = true) + val anotherEnv = createRpcEnv(conf, "remote", 0, clientMode = true) // Use anotherEnv to find out the RpcEndpointRef - val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "ask-timeout") + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-timeout") try { - // Any exception thrown in askWithRetry is wrapped with a SparkException and set as the cause - val e = intercept[SparkException] { - rpcEndpointRef.askWithRetry[String]("hello", new RpcTimeout(1 millis, shortProp)) + val e = intercept[RpcTimeoutException] { + rpcEndpointRef.askSync[String]("hello", new RpcTimeout(1 millis, shortProp)) } // The SparkException cause should be a RpcTimeoutException with message indicating the // controlling timeout property - assert(e.getCause.isInstanceOf[RpcTimeoutException]) - assert(e.getCause.getMessage.contains(shortProp)) + assert(e.isInstanceOf[RpcTimeoutException]) + assert(e.getMessage.contains(shortProp)) } finally { anotherEnv.shutdown() anotherEnv.awaitTermination() @@ -300,10 +314,9 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { override val rpcEnv = env override def receive: PartialFunction[Any, Unit] = { - case m => { + case m => self callSelfSuccessfully = true - } } }) @@ -402,7 +415,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { }) val f = endpointRef.ask[String]("Hi") - val ack = Await.result(f, 5 seconds) + val ack = ThreadUtils.awaitResult(f, 5 seconds) assert("ack" === ack) env.stop(endpointRef) @@ -417,12 +430,12 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } }) - val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345, clientMode = true) + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0, clientMode = true) // Use anotherEnv to find out the RpcEndpointRef - val rpcEndpointRef = anotherEnv.setupEndpointRef("local", env.address, "sendWithReply-remotely") + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "sendWithReply-remotely") try { val f = rpcEndpointRef.ask[String]("hello") - val ack = Await.result(f, 5 seconds) + val ack = ThreadUtils.awaitResult(f, 5 seconds) assert("ack" === ack) } finally { anotherEnv.shutdown() @@ -441,9 +454,9 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { val f = endpointRef.ask[String]("Hi") val e = intercept[SparkException] { - Await.result(f, 5 seconds) + ThreadUtils.awaitResult(f, 5 seconds) } - assert("Oops" === e.getMessage) + assert("Oops" === e.getCause.getMessage) env.stop(endpointRef) } @@ -457,80 +470,139 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } }) - val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345, clientMode = true) + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0, clientMode = true) // Use anotherEnv to find out the RpcEndpointRef - val rpcEndpointRef = anotherEnv.setupEndpointRef( - "local", env.address, "sendWithReply-remotely-error") + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "sendWithReply-remotely-error") try { val f = rpcEndpointRef.ask[String]("hello") val e = intercept[SparkException] { - Await.result(f, 5 seconds) + ThreadUtils.awaitResult(f, 5 seconds) } - assert("Oops" === e.getMessage) + assert("Oops" === e.getCause.getMessage) } finally { anotherEnv.shutdown() anotherEnv.awaitTermination() } } - test("network events") { - val events = new mutable.ArrayBuffer[(Any, Any)] with mutable.SynchronizedBuffer[(Any, Any)] - env.setupEndpoint("network-events", new ThreadSafeRpcEndpoint { - override val rpcEnv = env + /** + * Setup an [[RpcEndpoint]] to collect all network events. + * + * @return the [[RpcEndpointRef]] and a `ConcurrentLinkedQueue` that contains network events. + */ + private def setupNetworkEndpoint( + _env: RpcEnv, + name: String): (RpcEndpointRef, ConcurrentLinkedQueue[(Any, Any)]) = { + val events = new ConcurrentLinkedQueue[(Any, Any)] + val ref = _env.setupEndpoint("network-events-non-client", new ThreadSafeRpcEndpoint { + override val rpcEnv = _env override def receive: PartialFunction[Any, Unit] = { case "hello" => - case m => events += "receive" -> m + case m => events.add("receive" -> m) } override def onConnected(remoteAddress: RpcAddress): Unit = { - events += "onConnected" -> remoteAddress + events.add("onConnected" -> remoteAddress) } override def onDisconnected(remoteAddress: RpcAddress): Unit = { - events += "onDisconnected" -> remoteAddress + events.add("onDisconnected" -> remoteAddress) } override def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = { - events += "onNetworkError" -> remoteAddress + events.add("onNetworkError" -> remoteAddress) } }) + (ref, events) + } - val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345, clientMode = true) - // Use anotherEnv to find out the RpcEndpointRef - val rpcEndpointRef = anotherEnv.setupEndpointRef( - "local", env.address, "network-events") - val remoteAddress = anotherEnv.address - rpcEndpointRef.send("hello") - eventually(timeout(5 seconds), interval(5 millis)) { - // anotherEnv is connected in client mode, so the remote address may be unknown depending on - // the implementation. Account for that when doing checks. - if (remoteAddress != null) { - assert(events === List(("onConnected", remoteAddress))) - } else { - assert(events.size === 1) - assert(events(0)._1 === "onConnected") + test("network events in sever RpcEnv when another RpcEnv is in server mode") { + val serverEnv1 = createRpcEnv(new SparkConf(), "server1", 0, clientMode = false) + val serverEnv2 = createRpcEnv(new SparkConf(), "server2", 0, clientMode = false) + val (_, events) = setupNetworkEndpoint(serverEnv1, "network-events") + val (serverRef2, _) = setupNetworkEndpoint(serverEnv2, "network-events") + try { + val serverRefInServer2 = serverEnv1.setupEndpointRef(serverRef2.address, serverRef2.name) + // Send a message to set up the connection + serverRefInServer2.send("hello") + + eventually(timeout(5 seconds), interval(5 millis)) { + assert(events.contains(("onConnected", serverEnv2.address))) + } + + serverEnv2.shutdown() + serverEnv2.awaitTermination() + + eventually(timeout(5 seconds), interval(5 millis)) { + assert(events.contains(("onConnected", serverEnv2.address))) + assert(events.contains(("onDisconnected", serverEnv2.address))) } + } finally { + serverEnv1.shutdown() + serverEnv2.shutdown() + serverEnv1.awaitTermination() + serverEnv2.awaitTermination() } + } - anotherEnv.shutdown() - anotherEnv.awaitTermination() - eventually(timeout(5 seconds), interval(5 millis)) { - // Account for anotherEnv not having an address due to running in client mode. - if (remoteAddress != null) { - assert(events === List( - ("onConnected", remoteAddress), - ("onNetworkError", remoteAddress), - ("onDisconnected", remoteAddress)) || - events === List( - ("onConnected", remoteAddress), - ("onDisconnected", remoteAddress))) - } else { - val eventNames = events.map(_._1) - assert(eventNames === List("onConnected", "onNetworkError", "onDisconnected") || - eventNames === List("onConnected", "onDisconnected")) + test("network events in sever RpcEnv when another RpcEnv is in client mode") { + val serverEnv = createRpcEnv(new SparkConf(), "server", 0, clientMode = false) + val (serverRef, events) = setupNetworkEndpoint(serverEnv, "network-events") + val clientEnv = createRpcEnv(new SparkConf(), "client", 0, clientMode = true) + try { + val serverRefInClient = clientEnv.setupEndpointRef(serverRef.address, serverRef.name) + // Send a message to set up the connection + serverRefInClient.send("hello") + + eventually(timeout(5 seconds), interval(5 millis)) { + // We don't know the exact client address but at least we can verify the message type + assert(events.asScala.map(_._1).exists(_ == "onConnected")) + } + + clientEnv.shutdown() + clientEnv.awaitTermination() + + eventually(timeout(5 seconds), interval(5 millis)) { + // We don't know the exact client address but at least we can verify the message type + assert(events.asScala.map(_._1).exists(_ == "onConnected")) + assert(events.asScala.map(_._1).exists(_ == "onDisconnected")) + } + } finally { + clientEnv.shutdown() + serverEnv.shutdown() + clientEnv.awaitTermination() + serverEnv.awaitTermination() + } + } + + test("network events in client RpcEnv when another RpcEnv is in server mode") { + val clientEnv = createRpcEnv(new SparkConf(), "client", 0, clientMode = true) + val serverEnv = createRpcEnv(new SparkConf(), "server", 0, clientMode = false) + val (_, events) = setupNetworkEndpoint(clientEnv, "network-events") + val (serverRef, _) = setupNetworkEndpoint(serverEnv, "network-events") + try { + val serverRefInClient = clientEnv.setupEndpointRef(serverRef.address, serverRef.name) + // Send a message to set up the connection + serverRefInClient.send("hello") + + eventually(timeout(5 seconds), interval(5 millis)) { + assert(events.contains(("onConnected", serverEnv.address))) + } + + serverEnv.shutdown() + serverEnv.awaitTermination() + + eventually(timeout(5 seconds), interval(5 millis)) { + assert(events.contains(("onConnected", serverEnv.address))) + assert(events.contains(("onDisconnected", serverEnv.address))) } + } finally { + clientEnv.shutdown() + serverEnv.shutdown() + clientEnv.awaitTermination() + serverEnv.awaitTermination() } } @@ -543,18 +615,16 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } }) - val anotherEnv = createRpcEnv(new SparkConf(), "remote", 13345, clientMode = true) + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0, clientMode = true) // Use anotherEnv to find out the RpcEndpointRef - val rpcEndpointRef = anotherEnv.setupEndpointRef( - "local", env.address, "sendWithReply-unserializable-error") + val rpcEndpointRef = + anotherEnv.setupEndpointRef(env.address, "sendWithReply-unserializable-error") try { val f = rpcEndpointRef.ask[String]("hello") - val e = intercept[Exception] { - Await.result(f, 1 seconds) + val e = intercept[SparkException] { + ThreadUtils.awaitResult(f, 1 seconds) } - assert(e.isInstanceOf[TimeoutException] || // For Akka - e.isInstanceOf[NotSerializableException] // For Netty - ) + assert(e.getCause.isInstanceOf[NotSerializableException]) } finally { anotherEnv.shutdown() anotherEnv.awaitTermination() @@ -563,16 +633,17 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { test("port conflict") { val anotherEnv = createRpcEnv(new SparkConf(), "remote", env.address.port) - assert(anotherEnv.address.port != env.address.port) + try { + assert(anotherEnv.address.port != env.address.port) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } } - test("send with authentication") { - val conf = new SparkConf - conf.set("spark.authenticate", "true") - conf.set("spark.authenticate.secret", "good") - - val localEnv = createRpcEnv(conf, "authentication-local", 13345) - val remoteEnv = createRpcEnv(conf, "authentication-remote", 14345, clientMode = true) + private def testSend(conf: SparkConf): Unit = { + val localEnv = createRpcEnv(conf, "authentication-local", 0) + val remoteEnv = createRpcEnv(conf, "authentication-remote", 0, clientMode = true) try { @volatile var message: String = null @@ -583,8 +654,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { case msg: String => message = msg } }) - val rpcEndpointRef = - remoteEnv.setupEndpointRef("authentication-local", localEnv.address, "send-authentication") + val rpcEndpointRef = remoteEnv.setupEndpointRef(localEnv.address, "send-authentication") rpcEndpointRef.send("hello") eventually(timeout(5 seconds), interval(10 millis)) { assert("hello" === message) @@ -597,27 +667,21 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } } - test("ask with authentication") { - val conf = new SparkConf - conf.set("spark.authenticate", "true") - conf.set("spark.authenticate.secret", "good") - - val localEnv = createRpcEnv(conf, "authentication-local", 13345) - val remoteEnv = createRpcEnv(conf, "authentication-remote", 14345, clientMode = true) + private def testAsk(conf: SparkConf): Unit = { + val localEnv = createRpcEnv(conf, "authentication-local", 0) + val remoteEnv = createRpcEnv(conf, "authentication-remote", 0, clientMode = true) try { localEnv.setupEndpoint("ask-authentication", new RpcEndpoint { override val rpcEnv = localEnv override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case msg: String => { + case msg: String => context.reply(msg) - } } }) - val rpcEndpointRef = - remoteEnv.setupEndpointRef("authentication-local", localEnv.address, "ask-authentication") - val reply = rpcEndpointRef.askWithRetry[String]("hello") + val rpcEndpointRef = remoteEnv.setupEndpointRef(localEnv.address, "ask-authentication") + val reply = rpcEndpointRef.askSync[String]("hello") assert("hello" === reply) } finally { localEnv.shutdown() @@ -627,6 +691,48 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } } + test("send with authentication") { + testSend(new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good")) + } + + test("send with SASL encryption") { + testSend(new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good") + .set("spark.authenticate.enableSaslEncryption", "true")) + } + + test("send with AES encryption") { + testSend(new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good") + .set("spark.network.crypto.enabled", "true") + .set("spark.network.crypto.saslFallback", "false")) + } + + test("ask with authentication") { + testAsk(new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good")) + } + + test("ask with SASL encryption") { + testAsk(new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good") + .set("spark.authenticate.enableSaslEncryption", "true")) + } + + test("ask with AES encryption") { + testAsk(new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good") + .set("spark.network.crypto.enabled", "true") + .set("spark.network.crypto.saslFallback", "false")) + } + test("construct RpcTimeout with conf property") { val conf = new SparkConf @@ -688,15 +794,17 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { // RpcTimeout.awaitResult should have added the property to the TimeoutException message assert(reply2.contains(shortTimeout.timeoutProp)) - // Ask with delayed response and allow the Future to timeout before Await.result + // Ask with delayed response and allow the Future to timeout before ThreadUtils.awaitResult val fut3 = rpcEndpointRef.ask[String](NeverReply("goodbye"), shortTimeout) + // scalastyle:off awaitresult // Allow future to complete with failure using plain Await.result, this will return // once the future is complete to verify addMessageIfTimeout was invoked val reply3 = intercept[RpcTimeoutException] { Await.result(fut3, 2000 millis) }.getMessage + // scalastyle:on awaitresult // When the future timed out, the recover callback should have used // RpcTimeout.addMessageIfTimeout to add the property to the TimeoutException message @@ -713,6 +821,93 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { assert(shortTimeout.timeoutProp.r.findAllIn(reply4).length === 1) } + test("file server") { + val conf = new SparkConf() + val tempDir = Utils.createTempDir() + val file = new File(tempDir, "file") + Files.write(UUID.randomUUID().toString(), file, UTF_8) + val fileWithSpecialChars = new File(tempDir, "file name") + Files.write(UUID.randomUUID().toString(), fileWithSpecialChars, UTF_8) + val empty = new File(tempDir, "empty") + Files.write("", empty, UTF_8); + val jar = new File(tempDir, "jar") + Files.write(UUID.randomUUID().toString(), jar, UTF_8) + + val dir1 = new File(tempDir, "dir1") + assert(dir1.mkdir()) + val subFile1 = new File(dir1, "file1") + Files.write(UUID.randomUUID().toString(), subFile1, UTF_8) + + val dir2 = new File(tempDir, "dir2") + assert(dir2.mkdir()) + val subFile2 = new File(dir2, "file2") + Files.write(UUID.randomUUID().toString(), subFile2, UTF_8) + + val fileUri = env.fileServer.addFile(file) + val fileWithSpecialCharsUri = env.fileServer.addFile(fileWithSpecialChars) + val emptyUri = env.fileServer.addFile(empty) + val jarUri = env.fileServer.addJar(jar) + val dir1Uri = env.fileServer.addDirectory("/dir1", dir1) + val dir2Uri = env.fileServer.addDirectory("/dir2", dir2) + + // Try registering directories with invalid names. + Seq("/files", "/jars").foreach { uri => + intercept[IllegalArgumentException] { + env.fileServer.addDirectory(uri, dir1) + } + } + + val destDir = Utils.createTempDir() + val sm = new SecurityManager(conf) + val hc = SparkHadoopUtil.get.conf + + val files = Seq( + (file, fileUri), + (fileWithSpecialChars, fileWithSpecialCharsUri), + (empty, emptyUri), + (jar, jarUri), + (subFile1, dir1Uri + "/file1"), + (subFile2, dir2Uri + "/file2")) + files.foreach { case (f, uri) => + val destFile = new File(destDir, f.getName()) + Utils.fetchFile(uri, destDir, conf, sm, hc, 0L, false) + assert(Files.equal(f, destFile)) + } + + // Try to download files that do not exist. + Seq("files", "jars", "dir1").foreach { root => + intercept[Exception] { + val uri = env.address.toSparkURL + s"/$root/doesNotExist" + Utils.fetchFile(uri, destDir, conf, sm, hc, 0L, false) + } + } + } + + test("SPARK-14699: RpcEnv.shutdown should not fire onDisconnected events") { + env.setupEndpoint("SPARK-14699", new RpcEndpoint { + override val rpcEnv: RpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case m => context.reply(m) + } + }) + + val anotherEnv = createRpcEnv(new SparkConf(), "remote", 0) + val endpoint = mock(classOf[RpcEndpoint]) + anotherEnv.setupEndpoint("SPARK-14699", endpoint) + + val ref = anotherEnv.setupEndpointRef(env.address, "SPARK-14699") + // Make sure the connect is set up + assert(ref.askSync[String]("hello") === "hello") + anotherEnv.shutdown() + anotherEnv.awaitTermination() + + env.stop(ref) + + verify(endpoint).onStop() + verify(endpoint, never()).onDisconnected(any()) + verify(endpoint, never()).onNetworkError(any(), any()) + } } class UnserializableClass diff --git a/core/src/test/scala/org/apache/spark/rpc/akka/AkkaRpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/akka/AkkaRpcEnvSuite.scala deleted file mode 100644 index 6478ab51c4da..000000000000 --- a/core/src/test/scala/org/apache/spark/rpc/akka/AkkaRpcEnvSuite.scala +++ /dev/null @@ -1,71 +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.rpc.akka - -import org.apache.spark.rpc._ -import org.apache.spark.{SSLSampleConfigs, SecurityManager, SparkConf} - -class AkkaRpcEnvSuite extends RpcEnvSuite { - - override def createRpcEnv(conf: SparkConf, - name: String, - port: Int, - clientMode: Boolean = false): RpcEnv = { - new AkkaRpcEnvFactory().create( - RpcEnvConfig(conf, name, "localhost", port, new SecurityManager(conf), clientMode)) - } - - test("setupEndpointRef: systemName, address, endpointName") { - val ref = env.setupEndpoint("test_endpoint", new RpcEndpoint { - override val rpcEnv = env - - override def receive = { - case _ => - } - }) - val conf = new SparkConf() - val newRpcEnv = new AkkaRpcEnvFactory().create( - RpcEnvConfig(conf, "test", "localhost", 12346, new SecurityManager(conf), false)) - try { - val newRef = newRpcEnv.setupEndpointRef("local", ref.address, "test_endpoint") - assert(s"akka.tcp://local@${env.address}/user/test_endpoint" === - newRef.asInstanceOf[AkkaRpcEndpointRef].actorRef.path.toString) - } finally { - newRpcEnv.shutdown() - } - } - - test("uriOf") { - val uri = env.uriOf("local", RpcAddress("1.2.3.4", 12345), "test_endpoint") - assert("akka.tcp://local@1.2.3.4:12345/user/test_endpoint" === uri) - } - - test("uriOf: ssl") { - val conf = SSLSampleConfigs.sparkSSLConfig() - val securityManager = new SecurityManager(conf) - val rpcEnv = new AkkaRpcEnvFactory().create( - RpcEnvConfig(conf, "test", "localhost", 12346, securityManager, false)) - try { - val uri = rpcEnv.uriOf("local", RpcAddress("1.2.3.4", 12345), "test_endpoint") - assert("akka.ssl.tcp://local@1.2.3.4:12345/user/test_endpoint" === uri) - } finally { - rpcEnv.shutdown() - } - } - -} diff --git a/core/src/test/scala/org/apache/spark/rpc/netty/InboxSuite.scala b/core/src/test/scala/org/apache/spark/rpc/netty/InboxSuite.scala index 276c077b3d13..e5539566e4b6 100644 --- a/core/src/test/scala/org/apache/spark/rpc/netty/InboxSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/netty/InboxSuite.scala @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger import org.mockito.Mockito._ import org.apache.spark.SparkFunSuite -import org.apache.spark.rpc.{RpcEnv, RpcEndpoint, RpcAddress, TestRpcEndpoint} +import org.apache.spark.rpc.{RpcAddress, TestRpcEndpoint} class InboxSuite extends SparkFunSuite { @@ -35,7 +35,7 @@ class InboxSuite extends SparkFunSuite { val dispatcher = mock(classOf[Dispatcher]) val inbox = new Inbox(endpointRef, endpoint) - val message = ContentMessage(null, "hi", false, null) + val message = OneWayMessage(null, "hi") inbox.post(message) inbox.process(dispatcher) assert(inbox.isEmpty) @@ -55,7 +55,7 @@ class InboxSuite extends SparkFunSuite { val dispatcher = mock(classOf[Dispatcher]) val inbox = new Inbox(endpointRef, endpoint) - val message = ContentMessage(null, "hi", true, null) + val message = RpcMessage(null, "hi", null) inbox.post(message) inbox.process(dispatcher) assert(inbox.isEmpty) @@ -83,7 +83,7 @@ class InboxSuite extends SparkFunSuite { new Thread { override def run(): Unit = { for (_ <- 0 until 100) { - val message = ContentMessage(null, "hi", false, null) + val message = OneWayMessage(null, "hi") inbox.post(message) } exitLatch.countDown() diff --git a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcAddressSuite.scala b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcAddressSuite.scala index 56743ba650b4..4fcdb619f930 100644 --- a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcAddressSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcAddressSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.rpc.netty import org.apache.spark.SparkFunSuite +import org.apache.spark.rpc.RpcEndpointAddress class NettyRpcAddressSuite extends SparkFunSuite { diff --git a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala index ce83087ec04d..f9481f875d43 100644 --- a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala @@ -17,27 +17,71 @@ package org.apache.spark.rpc.netty -import org.apache.spark.{SecurityManager, SparkConf} +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark._ +import org.apache.spark.network.client.TransportClient import org.apache.spark.rpc._ -class NettyRpcEnvSuite extends RpcEnvSuite { +class NettyRpcEnvSuite extends RpcEnvSuite with MockitoSugar { override def createRpcEnv( conf: SparkConf, name: String, port: Int, clientMode: Boolean = false): RpcEnv = { - val config = RpcEnvConfig(conf, "test", "localhost", port, new SecurityManager(conf), - clientMode) + val config = RpcEnvConfig(conf, "test", "localhost", "localhost", port, + new SecurityManager(conf), 0, clientMode) new NettyRpcEnvFactory().create(config) } test("non-existent endpoint") { - val uri = env.uriOf("test", env.address, "nonexist-endpoint") - val e = intercept[RpcEndpointNotFoundException] { - env.setupEndpointRef("test", env.address, "nonexist-endpoint") + val uri = RpcEndpointAddress(env.address, "nonexist-endpoint").toString + val e = intercept[SparkException] { + env.setupEndpointRef(env.address, "nonexist-endpoint") } - assert(e.getMessage.contains(uri)) + assert(e.getCause.isInstanceOf[RpcEndpointNotFoundException]) + assert(e.getCause.getMessage.contains(uri)) } + test("advertise address different from bind address") { + val sparkConf = new SparkConf() + val config = RpcEnvConfig(sparkConf, "test", "localhost", "example.com", 0, + new SecurityManager(sparkConf), 0, false) + val env = new NettyRpcEnvFactory().create(config) + try { + assert(env.address.hostPort.startsWith("example.com:")) + } finally { + env.shutdown() + } + } + + test("RequestMessage serialization") { + def assertRequestMessageEquals(expected: RequestMessage, actual: RequestMessage): Unit = { + assert(expected.senderAddress === actual.senderAddress) + assert(expected.receiver === actual.receiver) + assert(expected.content === actual.content) + } + + val nettyEnv = env.asInstanceOf[NettyRpcEnv] + val client = mock[TransportClient] + val senderAddress = RpcAddress("locahost", 12345) + val receiverAddress = RpcEndpointAddress("localhost", 54321, "test") + val receiver = new NettyRpcEndpointRef(nettyEnv.conf, receiverAddress, nettyEnv) + + val msg = new RequestMessage(senderAddress, receiver, "foo") + assertRequestMessageEquals( + msg, + RequestMessage(nettyEnv, client, msg.serialize(nettyEnv))) + + val msg2 = new RequestMessage(null, receiver, "foo") + assertRequestMessageEquals( + msg2, + RequestMessage(nettyEnv, client, msg2.serialize(nettyEnv))) + + val msg3 = new RequestMessage(senderAddress, receiver, null) + assertRequestMessageEquals( + msg3, + RequestMessage(nettyEnv, client, msg3.serialize(nettyEnv))) + } } diff --git a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala index f9d8e80c98b6..a71d8726e706 100644 --- a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala @@ -18,44 +18,47 @@ package org.apache.spark.rpc.netty import java.net.InetSocketAddress +import java.nio.ByteBuffer import io.netty.channel.Channel -import org.mockito.Mockito._ import org.mockito.Matchers._ +import org.mockito.Mockito._ import org.apache.spark.SparkFunSuite -import org.apache.spark.network.client.{TransportResponseHandler, TransportClient} +import org.apache.spark.network.client.{TransportClient, TransportResponseHandler} +import org.apache.spark.network.server.StreamManager import org.apache.spark.rpc._ class NettyRpcHandlerSuite extends SparkFunSuite { val env = mock(classOf[NettyRpcEnv]) - when(env.deserialize(any(classOf[TransportClient]), any(classOf[Array[Byte]]))(any())). - thenReturn(RequestMessage(RpcAddress("localhost", 12345), null, null, false)) + val sm = mock(classOf[StreamManager]) + when(env.deserialize(any(classOf[TransportClient]), any(classOf[ByteBuffer]))(any())) + .thenReturn(new RequestMessage(RpcAddress("localhost", 12345), null, null)) test("receive") { val dispatcher = mock(classOf[Dispatcher]) - val nettyRpcHandler = new NettyRpcHandler(dispatcher, env) + val nettyRpcHandler = new NettyRpcHandler(dispatcher, env, sm) val channel = mock(classOf[Channel]) val client = new TransportClient(channel, mock(classOf[TransportResponseHandler])) when(channel.remoteAddress()).thenReturn(new InetSocketAddress("localhost", 40000)) - nettyRpcHandler.receive(client, null, null) + nettyRpcHandler.channelActive(client) verify(dispatcher, times(1)).postToAll(RemoteProcessConnected(RpcAddress("localhost", 40000))) } test("connectionTerminated") { val dispatcher = mock(classOf[Dispatcher]) - val nettyRpcHandler = new NettyRpcHandler(dispatcher, env) + val nettyRpcHandler = new NettyRpcHandler(dispatcher, env, sm) val channel = mock(classOf[Channel]) val client = new TransportClient(channel, mock(classOf[TransportResponseHandler])) when(channel.remoteAddress()).thenReturn(new InetSocketAddress("localhost", 40000)) - nettyRpcHandler.receive(client, null, null) + nettyRpcHandler.channelActive(client) when(channel.remoteAddress()).thenReturn(new InetSocketAddress("localhost", 40000)) - nettyRpcHandler.connectionTerminated(client) + nettyRpcHandler.channelInactive(client) verify(dispatcher, times(1)).postToAll(RemoteProcessConnected(RpcAddress("localhost", 40000))) verify(dispatcher, times(1)).postToAll( diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala new file mode 100644 index 000000000000..f6015cd51c2b --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.scheduler + +import scala.concurrent.duration._ + +import org.apache.spark._ +import org.apache.spark.internal.config + +class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorMockBackend]{ + + val badHost = "host-0" + val duration = Duration(10, SECONDS) + + /** + * This backend just always fails if the task is executed on a bad host, but otherwise succeeds + * all tasks. + */ + def badHostBackend(): Unit = { + val (taskDescription, _) = backend.beginTask() + val host = backend.executorIdToExecutor(taskDescription.executorId).host + if (host == badHost) { + backend.taskFailed(taskDescription, new RuntimeException("I'm a bad host!")) + } else { + backend.taskSuccess(taskDescription, 42) + } + } + + // Test demonstrating the issue -- without a config change, the scheduler keeps scheduling + // according to locality preferences, and so the job fails + testScheduler("If preferred node is bad, without blacklist job will fail", + extraConfs = Seq( + config.BLACKLIST_ENABLED.key -> "false" + )) { + val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) + withBackend(badHostBackend _) { + val jobFuture = submit(rdd, (0 until 10).toArray) + awaitJobTermination(jobFuture, duration) + } + assertDataStructuresEmpty(noFailure = false) + } + + testScheduler( + "With default settings, job can succeed despite multiple bad executors on node", + extraConfs = Seq( + config.BLACKLIST_ENABLED.key -> "true", + config.MAX_TASK_FAILURES.key -> "4", + "spark.testing.nHosts" -> "2", + "spark.testing.nExecutorsPerHost" -> "5", + "spark.testing.nCoresPerExecutor" -> "10" + ) + ) { + // To reliably reproduce the failure that would occur without blacklisting, we have to use 1 + // task. That way, we ensure this 1 task gets rotated through enough bad executors on the host + // to fail the taskSet, before we have a bunch of different tasks fail in the executors so we + // blacklist them. + // But the point here is -- without blacklisting, we would never schedule anything on the good + // host-1 before we hit too many failures trying our preferred host-0. + val rdd = new MockRDDWithLocalityPrefs(sc, 1, Nil, badHost) + withBackend(badHostBackend _) { + val jobFuture = submit(rdd, (0 until 1).toArray) + awaitJobTermination(jobFuture, duration) + } + assertDataStructuresEmpty(noFailure = true) + } + + // Here we run with the blacklist on, and the default config takes care of having this + // robust to one bad node. + testScheduler( + "Bad node with multiple executors, job will still succeed with the right confs", + extraConfs = Seq( + config.BLACKLIST_ENABLED.key -> "true", + // just to avoid this test taking too long + "spark.locality.wait" -> "10ms" + ) + ) { + val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) + withBackend(badHostBackend _) { + val jobFuture = submit(rdd, (0 until 10).toArray) + awaitJobTermination(jobFuture, duration) + } + assert(results === (0 until 10).map { _ -> 42 }.toMap) + assertDataStructuresEmpty(noFailure = true) + } + + // Make sure that if we've failed on all executors, but haven't hit task.maxFailures yet, the job + // doesn't hang + testScheduler( + "SPARK-15865 Progress with fewer executors than maxTaskFailures", + extraConfs = Seq( + config.BLACKLIST_ENABLED.key -> "true", + "spark.testing.nHosts" -> "2", + "spark.testing.nExecutorsPerHost" -> "1", + "spark.testing.nCoresPerExecutor" -> "1" + ) + ) { + def runBackend(): Unit = { + val (taskDescription, _) = backend.beginTask() + backend.taskFailed(taskDescription, new RuntimeException("test task failure")) + } + withBackend(runBackend _) { + val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) + awaitJobTermination(jobFuture, duration) + val pattern = ("Aborting TaskSet 0.0 because task .* " + + "cannot run anywhere due to node and executor blacklist").r + assert(pattern.findFirstIn(failure.getMessage).isDefined, + s"Couldn't find $pattern in ${failure.getMessage()}") + } + assertDataStructuresEmpty(noFailure = false) + } +} + +class MultiExecutorMockBackend( + conf: SparkConf, + taskScheduler: TaskSchedulerImpl) extends MockBackend(conf, taskScheduler) { + + val nHosts = conf.getInt("spark.testing.nHosts", 5) + val nExecutorsPerHost = conf.getInt("spark.testing.nExecutorsPerHost", 4) + val nCoresPerExecutor = conf.getInt("spark.testing.nCoresPerExecutor", 2) + + override val executorIdToExecutor: Map[String, ExecutorTaskStatus] = { + (0 until nHosts).flatMap { hostIdx => + val hostName = "host-" + hostIdx + (0 until nExecutorsPerHost).map { subIdx => + val executorId = (hostIdx * nExecutorsPerHost + subIdx).toString + executorId -> + ExecutorTaskStatus(host = hostName, executorId = executorId, nCoresPerExecutor) + } + }.toMap + } + + override def defaultParallelism(): Int = nHosts * nExecutorsPerHost * nCoresPerExecutor +} + +class MockRDDWithLocalityPrefs( + sc: SparkContext, + numPartitions: Int, + shuffleDeps: Seq[ShuffleDependency[Int, Int, Nothing]], + val preferredLoc: String) extends MockRDD(sc, numPartitions, shuffleDeps) { + override def getPreferredLocations(split: Partition): Seq[String] = { + Seq(preferredLoc) + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala new file mode 100644 index 000000000000..a136d69b36d6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -0,0 +1,588 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import org.mockito.Matchers.any +import org.mockito.Mockito.{never, verify, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfterEach +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark._ +import org.apache.spark.internal.config +import org.apache.spark.util.ManualClock + +class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar + with LocalSparkContext { + + private val clock = new ManualClock(0) + + private var blacklist: BlacklistTracker = _ + private var listenerBusMock: LiveListenerBus = _ + private var scheduler: TaskSchedulerImpl = _ + private var conf: SparkConf = _ + + override def beforeEach(): Unit = { + conf = new SparkConf().setAppName("test").setMaster("local") + .set(config.BLACKLIST_ENABLED.key, "true") + scheduler = mockTaskSchedWithConf(conf) + + clock.setTime(0) + + listenerBusMock = mock[LiveListenerBus] + blacklist = new BlacklistTracker(listenerBusMock, conf, None, clock) + } + + override def afterEach(): Unit = { + if (blacklist != null) { + blacklist = null + } + if (scheduler != null) { + scheduler.stop() + scheduler = null + } + super.afterEach() + } + + // All executors and hosts used in tests should be in this set, so that [[assertEquivalentToSet]] + // works. Its OK if its got extraneous entries + val allExecutorAndHostIds = { + (('A' to 'Z')++ (1 to 100).map(_.toString)) + .flatMap{ suffix => + Seq(s"host$suffix", s"host-$suffix") + } + }.toSet + + /** + * Its easier to write our tests as if we could directly look at the sets of nodes & executors in + * the blacklist. However the api doesn't expose a set, so this is a simple way to test + * something similar, since we know the universe of values that might appear in these sets. + */ + def assertEquivalentToSet(f: String => Boolean, expected: Set[String]): Unit = { + allExecutorAndHostIds.foreach { id => + val actual = f(id) + val exp = expected.contains(id) + assert(actual === exp, raw"""for string "$id" """) + } + } + + def mockTaskSchedWithConf(conf: SparkConf): TaskSchedulerImpl = { + sc = new SparkContext(conf) + val scheduler = mock[TaskSchedulerImpl] + when(scheduler.sc).thenReturn(sc) + when(scheduler.mapOutputTracker).thenReturn( + SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster]) + scheduler + } + + def createTaskSetBlacklist(stageId: Int = 0): TaskSetBlacklist = { + new TaskSetBlacklist(conf, stageId, clock) + } + + test("executors can be blacklisted with only a few failures per stage") { + // For many different stages, executor 1 fails a task, then executor 2 succeeds the task, + // and then the task set is done. Not enough failures to blacklist the executor *within* + // any particular taskset, but we still blacklist the executor overall eventually. + // Also, we intentionally have a mix of task successes and failures -- there are even some + // successes after the executor is blacklisted. The idea here is those tasks get scheduled + // before the executor is blacklisted. We might get successes after blacklisting (because the + // executor might be flaky but not totally broken). But successes should not unblacklist the + // executor. + val failuresUntilBlacklisted = conf.get(config.MAX_FAILURES_PER_EXEC) + var failuresSoFar = 0 + (0 until failuresUntilBlacklisted * 10).foreach { stageId => + val taskSetBlacklist = createTaskSetBlacklist(stageId) + if (stageId % 2 == 0) { + // fail one task in every other taskset + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + failuresSoFar += 1 + } + blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, taskSetBlacklist.execToFailures) + assert(failuresSoFar == stageId / 2 + 1) + if (failuresSoFar < failuresUntilBlacklisted) { + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + } else { + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post( + SparkListenerExecutorBlacklisted(0, "1", failuresUntilBlacklisted)) + } + } + } + + // If an executor has many task failures, but the task set ends up failing, it shouldn't be + // counted against the executor. + test("executors aren't blacklisted as a result of tasks in failed task sets") { + val failuresUntilBlacklisted = conf.get(config.MAX_FAILURES_PER_EXEC) + // for many different stages, executor 1 fails a task, and then the taskSet fails. + (0 until failuresUntilBlacklisted * 10).foreach { stage => + val taskSetBlacklist = createTaskSetBlacklist(stage) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + } + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + } + + Seq(true, false).foreach { succeedTaskSet => + val label = if (succeedTaskSet) "success" else "failure" + test(s"stage blacklist updates correctly on stage $label") { + // Within one taskset, an executor fails a few times, so it's blacklisted for the taskset. + // But if the taskset fails, we shouldn't blacklist the executor after the stage. + val taskSetBlacklist = createTaskSetBlacklist(0) + // We trigger enough failures for both the taskset blacklist, and the application blacklist. + val numFailures = math.max(conf.get(config.MAX_FAILURES_PER_EXEC), + conf.get(config.MAX_FAILURES_PER_EXEC_STAGE)) + (0 until numFailures).foreach { index => + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = index) + } + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + if (succeedTaskSet) { + // The task set succeeded elsewhere, so we should count those failures against our executor, + // and it should be blacklisted for the entire application. + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist.execToFailures) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", numFailures)) + } else { + // The task set failed, so we don't count these failures against the executor for other + // stages. + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + } + } + } + + test("blacklisted executors and nodes get recovered with time") { + val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0) + // Fail 4 tasks in one task set on executor 1, so that executor gets blacklisted for the whole + // application. + (0 until 4).foreach { partition => + taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition) + } + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures) + assert(blacklist.nodeBlacklist() === Set()) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", 4)) + + val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) + // Fail 4 tasks in one task set on executor 2, so that executor gets blacklisted for the whole + // application. Since that's the second executor that is blacklisted on the same node, we also + // blacklist that node. + (0 until 4).foreach { partition => + taskSetBlacklist1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition) + } + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist1.execToFailures) + assert(blacklist.nodeBlacklist() === Set("hostA")) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA")) + verify(listenerBusMock).post(SparkListenerNodeBlacklisted(0, "hostA", 2)) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "2", 4)) + + // Advance the clock and then make sure hostA and executors 1 and 2 have been removed from the + // blacklist. + val timeout = blacklist.BLACKLIST_TIMEOUT_MILLIS + 1 + clock.advance(timeout) + blacklist.applyBlacklistTimeout() + assert(blacklist.nodeBlacklist() === Set()) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(timeout, "2")) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(timeout, "1")) + verify(listenerBusMock).post(SparkListenerNodeUnblacklisted(timeout, "hostA")) + + // Fail one more task, but executor isn't put back into blacklist since the count of failures + // on that executor should have been reset to 0. + val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 2) + taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + blacklist.updateBlacklistForSuccessfulTaskSet(2, 0, taskSetBlacklist2.execToFailures) + assert(blacklist.nodeBlacklist() === Set()) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + } + + test("blacklist can handle lost executors") { + // The blacklist should still work if an executor is killed completely. We should still + // be able to blacklist the entire node. + val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0) + // Lets say that executor 1 dies completely. We get some task failures, but + // the taskset then finishes successfully (elsewhere). + (0 until 4).foreach { partition => + taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition) + } + blacklist.handleRemovedExecutor("1") + blacklist.updateBlacklistForSuccessfulTaskSet( + stageId = 0, + stageAttemptId = 0, + taskSetBlacklist0.execToFailures) + assert(blacklist.isExecutorBlacklisted("1")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", 4)) + val t1 = blacklist.BLACKLIST_TIMEOUT_MILLIS / 2 + clock.advance(t1) + + // Now another executor gets spun up on that host, but it also dies. + val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) + (0 until 4).foreach { partition => + taskSetBlacklist1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition) + } + blacklist.handleRemovedExecutor("2") + blacklist.updateBlacklistForSuccessfulTaskSet( + stageId = 1, + stageAttemptId = 0, + taskSetBlacklist1.execToFailures) + // We've now had two bad executors on the hostA, so we should blacklist the entire node. + assert(blacklist.isExecutorBlacklisted("1")) + assert(blacklist.isExecutorBlacklisted("2")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(t1, "2", 4)) + assert(blacklist.isNodeBlacklisted("hostA")) + verify(listenerBusMock).post(SparkListenerNodeBlacklisted(t1, "hostA", 2)) + + // Advance the clock so that executor 1 should no longer be explicitly blacklisted, but + // everything else should still be blacklisted. + val t2 = blacklist.BLACKLIST_TIMEOUT_MILLIS / 2 + 1 + clock.advance(t2) + blacklist.applyBlacklistTimeout() + assert(!blacklist.isExecutorBlacklisted("1")) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(t1 + t2, "1")) + assert(blacklist.isExecutorBlacklisted("2")) + assert(blacklist.isNodeBlacklisted("hostA")) + // make sure we don't leak memory + assert(!blacklist.executorIdToBlacklistStatus.contains("1")) + assert(!blacklist.nodeToBlacklistedExecs("hostA").contains("1")) + // Advance the timeout again so now hostA should be removed from the blacklist. + clock.advance(t1) + blacklist.applyBlacklistTimeout() + assert(!blacklist.nodeIdToBlacklistExpiryTime.contains("hostA")) + verify(listenerBusMock).post(SparkListenerNodeUnblacklisted(t1 + t2 + t1, "hostA")) + // Even though unblacklisting a node implicitly unblacklists all of its executors, + // there will be no SparkListenerExecutorUnblacklisted sent here. + } + + test("task failures expire with time") { + // Verifies that 2 failures within the timeout period cause an executor to be blacklisted, but + // if task failures are spaced out by more than the timeout period, the first failure is timed + // out, and the executor isn't blacklisted. + var stageId = 0 + + def failOneTaskInTaskSet(exec: String): Unit = { + val taskSetBlacklist = createTaskSetBlacklist(stageId = stageId) + taskSetBlacklist.updateBlacklistForFailedTask("host-" + exec, exec, 0) + blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, taskSetBlacklist.execToFailures) + stageId += 1 + } + + failOneTaskInTaskSet(exec = "1") + // We have one sporadic failure on exec 2, but that's it. Later checks ensure that we never + // blacklist executor 2 despite this one failure. + failOneTaskInTaskSet(exec = "2") + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + assert(blacklist.nextExpiryTime === Long.MaxValue) + + // We advance the clock past the expiry time. + clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) + val t0 = clock.getTimeMillis() + blacklist.applyBlacklistTimeout() + assert(blacklist.nextExpiryTime === Long.MaxValue) + failOneTaskInTaskSet(exec = "1") + + // Because the 2nd failure on executor 1 happened past the expiry time, nothing should have been + // blacklisted. + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + + // Now we add one more failure, within the timeout, and it should be counted. + clock.setTime(t0 + blacklist.BLACKLIST_TIMEOUT_MILLIS - 1) + val t1 = clock.getTimeMillis() + failOneTaskInTaskSet(exec = "1") + blacklist.applyBlacklistTimeout() + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(t1, "1", 2)) + assert(blacklist.nextExpiryTime === t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS) + + // Add failures on executor 3, make sure it gets put on the blacklist. + clock.setTime(t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS - 1) + val t2 = clock.getTimeMillis() + failOneTaskInTaskSet(exec = "3") + failOneTaskInTaskSet(exec = "3") + blacklist.applyBlacklistTimeout() + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "3")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(t2, "3", 2)) + assert(blacklist.nextExpiryTime === t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS) + + // Now we go past the timeout for executor 1, so it should be dropped from the blacklist. + clock.setTime(t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) + blacklist.applyBlacklistTimeout() + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("3")) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(clock.getTimeMillis(), "1")) + assert(blacklist.nextExpiryTime === t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS) + + // Make sure that we update correctly when we go from having blacklisted executors to + // just having tasks with timeouts. + clock.setTime(t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS - 1) + failOneTaskInTaskSet(exec = "4") + blacklist.applyBlacklistTimeout() + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("3")) + assert(blacklist.nextExpiryTime === t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS) + + clock.setTime(t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) + blacklist.applyBlacklistTimeout() + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(clock.getTimeMillis(), "3")) + // we've got one task failure still, but we don't bother setting nextExpiryTime to it, to + // avoid wasting time checking for expiry of individual task failures. + assert(blacklist.nextExpiryTime === Long.MaxValue) + } + + test("task failure timeout works as expected for long-running tasksets") { + // This ensures that we don't trigger spurious blacklisting for long tasksets, when the taskset + // finishes long after the task failures. We create two tasksets, each with one failure. + // Individually they shouldn't cause any blacklisting since there is only one failure. + // Furthermore, we space the failures out so far that even when both tasksets have completed, + // we still don't trigger any blacklisting. + val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) + val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 2) + // Taskset1 has one failure immediately + taskSetBlacklist1.updateBlacklistForFailedTask("host-1", "1", 0) + // Then we have a *long* delay, much longer than the timeout, before any other failures or + // taskset completion + clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS * 5) + // After the long delay, we have one failure on taskset 2, on the same executor + taskSetBlacklist2.updateBlacklistForFailedTask("host-1", "1", 0) + // Finally, we complete both tasksets. Its important here to complete taskset2 *first*. We + // want to make sure that when taskset 1 finishes, even though we've now got two task failures, + // we realize that the task failure we just added was well before the timeout. + clock.advance(1) + blacklist.updateBlacklistForSuccessfulTaskSet(stageId = 2, 0, taskSetBlacklist2.execToFailures) + clock.advance(1) + blacklist.updateBlacklistForSuccessfulTaskSet(stageId = 1, 0, taskSetBlacklist1.execToFailures) + + // Make sure nothing was blacklisted + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + } + + test("only blacklist nodes for the application when enough executors have failed on that " + + "specific host") { + // we blacklist executors on two different hosts -- make sure that doesn't lead to any + // node blacklisting + val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0) + taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", 2)) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) + + val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) + taskSetBlacklist1.updateBlacklistForFailedTask("hostB", exec = "2", index = 0) + taskSetBlacklist1.updateBlacklistForFailedTask("hostB", exec = "2", index = 1) + blacklist.updateBlacklistForSuccessfulTaskSet(1, 0, taskSetBlacklist1.execToFailures) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "2", 2)) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) + + // Finally, blacklist another executor on the same node as the original blacklisted executor, + // and make sure this time we *do* blacklist the node. + val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 0) + taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "3", index = 0) + taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "3", index = 1) + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist2.execToFailures) + assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2", "3")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "3", 2)) + assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA")) + verify(listenerBusMock).post(SparkListenerNodeBlacklisted(0, "hostA", 2)) + } + + test("blacklist still respects legacy configs") { + val conf = new SparkConf().setMaster("local") + assert(!BlacklistTracker.isBlacklistEnabled(conf)) + conf.set(config.BLACKLIST_LEGACY_TIMEOUT_CONF, 5000L) + assert(BlacklistTracker.isBlacklistEnabled(conf)) + assert(5000 === BlacklistTracker.getBlacklistTimeout(conf)) + // the new conf takes precedence, though + conf.set(config.BLACKLIST_TIMEOUT_CONF, 1000L) + assert(1000 === BlacklistTracker.getBlacklistTimeout(conf)) + + // if you explicitly set the legacy conf to 0, that also would disable blacklisting + conf.set(config.BLACKLIST_LEGACY_TIMEOUT_CONF, 0L) + assert(!BlacklistTracker.isBlacklistEnabled(conf)) + // but again, the new conf takes precedence + conf.set(config.BLACKLIST_ENABLED, true) + assert(BlacklistTracker.isBlacklistEnabled(conf)) + assert(1000 === BlacklistTracker.getBlacklistTimeout(conf)) + } + + test("check blacklist configuration invariants") { + val conf = new SparkConf().setMaster("yarn-cluster") + Seq( + (2, 2), + (2, 3) + ).foreach { case (maxTaskFailures, maxNodeAttempts) => + conf.set(config.MAX_TASK_FAILURES, maxTaskFailures) + conf.set(config.MAX_TASK_ATTEMPTS_PER_NODE.key, maxNodeAttempts.toString) + val excMsg = intercept[IllegalArgumentException] { + BlacklistTracker.validateBlacklistConfs(conf) + }.getMessage() + assert(excMsg === s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + + s"( = ${maxNodeAttempts}) was >= ${config.MAX_TASK_FAILURES.key} " + + s"( = ${maxTaskFailures} ). Though blacklisting is enabled, with this configuration, " + + s"Spark will not be robust to one bad node. Decrease " + + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase ${config.MAX_TASK_FAILURES.key}, " + + s"or disable blacklisting with ${config.BLACKLIST_ENABLED.key}") + } + + conf.remove(config.MAX_TASK_FAILURES) + conf.remove(config.MAX_TASK_ATTEMPTS_PER_NODE) + + Seq( + config.MAX_TASK_ATTEMPTS_PER_EXECUTOR, + config.MAX_TASK_ATTEMPTS_PER_NODE, + config.MAX_FAILURES_PER_EXEC_STAGE, + config.MAX_FAILED_EXEC_PER_NODE_STAGE, + config.MAX_FAILURES_PER_EXEC, + config.MAX_FAILED_EXEC_PER_NODE, + config.BLACKLIST_TIMEOUT_CONF + ).foreach { config => + conf.set(config.key, "0") + val excMsg = intercept[IllegalArgumentException] { + BlacklistTracker.validateBlacklistConfs(conf) + }.getMessage() + assert(excMsg.contains(s"${config.key} was 0, but must be > 0.")) + conf.remove(config) + } + } + + test("blacklisting kills executors, configured by BLACKLIST_KILL_ENABLED") { + val allocationClientMock = mock[ExecutorAllocationClient] + when(allocationClientMock.killExecutors(any(), any(), any())).thenReturn(Seq("called")) + when(allocationClientMock.killExecutorsOnHost("hostA")).thenAnswer(new Answer[Boolean] { + // To avoid a race between blacklisting and killing, it is important that the nodeBlacklist + // is updated before we ask the executor allocation client to kill all the executors + // on a particular host. + override def answer(invocation: InvocationOnMock): Boolean = { + if (blacklist.nodeBlacklist.contains("hostA") == false) { + throw new IllegalStateException("hostA should be on the blacklist") + } + true + } + }) + blacklist = new BlacklistTracker(listenerBusMock, conf, Some(allocationClientMock), clock) + + // Disable auto-kill. Blacklist an executor and make sure killExecutors is not called. + conf.set(config.BLACKLIST_KILL_ENABLED, false) + + val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0) + // Fail 4 tasks in one task set on executor 1, so that executor gets blacklisted for the whole + // application. + (0 until 4).foreach { partition => + taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition) + } + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures) + + verify(allocationClientMock, never).killExecutor(any()) + + val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) + // Fail 4 tasks in one task set on executor 2, so that executor gets blacklisted for the whole + // application. Since that's the second executor that is blacklisted on the same node, we also + // blacklist that node. + (0 until 4).foreach { partition => + taskSetBlacklist1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition) + } + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist1.execToFailures) + + verify(allocationClientMock, never).killExecutors(any(), any(), any()) + verify(allocationClientMock, never).killExecutorsOnHost(any()) + + // Enable auto-kill. Blacklist an executor and make sure killExecutors is called. + conf.set(config.BLACKLIST_KILL_ENABLED, true) + blacklist = new BlacklistTracker(listenerBusMock, conf, Some(allocationClientMock), clock) + + val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 0) + // Fail 4 tasks in one task set on executor 1, so that executor gets blacklisted for the whole + // application. + (0 until 4).foreach { partition => + taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "1", index = partition) + } + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist2.execToFailures) + + verify(allocationClientMock).killExecutors(Seq("1"), true, true) + + val taskSetBlacklist3 = createTaskSetBlacklist(stageId = 1) + // Fail 4 tasks in one task set on executor 2, so that executor gets blacklisted for the whole + // application. Since that's the second executor that is blacklisted on the same node, we also + // blacklist that node. + (0 until 4).foreach { partition => + taskSetBlacklist3.updateBlacklistForFailedTask("hostA", exec = "2", index = partition) + } + blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist3.execToFailures) + + verify(allocationClientMock).killExecutors(Seq("2"), true, true) + verify(allocationClientMock).killExecutorsOnHost("hostA") + } + + test("fetch failure blacklisting kills executors, configured by BLACKLIST_KILL_ENABLED") { + val allocationClientMock = mock[ExecutorAllocationClient] + when(allocationClientMock.killExecutors(any(), any(), any())).thenReturn(Seq("called")) + when(allocationClientMock.killExecutorsOnHost("hostA")).thenAnswer(new Answer[Boolean] { + // To avoid a race between blacklisting and killing, it is important that the nodeBlacklist + // is updated before we ask the executor allocation client to kill all the executors + // on a particular host. + override def answer(invocation: InvocationOnMock): Boolean = { + if (blacklist.nodeBlacklist.contains("hostA") == false) { + throw new IllegalStateException("hostA should be on the blacklist") + } + true + } + }) + + conf.set(config.BLACKLIST_FETCH_FAILURE_ENABLED, true) + blacklist = new BlacklistTracker(listenerBusMock, conf, Some(allocationClientMock), clock) + + // Disable auto-kill. Blacklist an executor and make sure killExecutors is not called. + conf.set(config.BLACKLIST_KILL_ENABLED, false) + blacklist.updateBlacklistForFetchFailure("hostA", exec = "1") + + verify(allocationClientMock, never).killExecutors(any(), any(), any()) + verify(allocationClientMock, never).killExecutorsOnHost(any()) + + // Enable auto-kill. Blacklist an executor and make sure killExecutors is called. + conf.set(config.BLACKLIST_KILL_ENABLED, true) + blacklist = new BlacklistTracker(listenerBusMock, conf, Some(allocationClientMock), clock) + clock.advance(1000) + blacklist.updateBlacklistForFetchFailure("hostA", exec = "1") + + verify(allocationClientMock).killExecutors(Seq("1"), true, true) + verify(allocationClientMock, never).killExecutorsOnHost(any()) + + assert(blacklist.executorIdToBlacklistStatus.contains("1")) + assert(blacklist.executorIdToBlacklistStatus("1").node === "hostA") + assert(blacklist.executorIdToBlacklistStatus("1").expiryTime === + 1000 + blacklist.BLACKLIST_TIMEOUT_MILLIS) + assert(blacklist.nextExpiryTime === 1000 + blacklist.BLACKLIST_TIMEOUT_MILLIS) + assert(blacklist.nodeIdToBlacklistExpiryTime.isEmpty) + + // Enable external shuffle service to see if all the executors on this node will be killed. + conf.set(config.SHUFFLE_SERVICE_ENABLED, true) + clock.advance(1000) + blacklist.updateBlacklistForFetchFailure("hostA", exec = "2") + + verify(allocationClientMock, never).killExecutors(Seq("2"), true, true) + verify(allocationClientMock).killExecutorsOnHost("hostA") + + assert(blacklist.nodeIdToBlacklistExpiryTime.contains("hostA")) + assert(blacklist.nodeIdToBlacklistExpiryTime("hostA") === + 2000 + blacklist.BLACKLIST_TIMEOUT_MILLIS) + assert(blacklist.nextExpiryTime === 1000 + blacklist.BLACKLIST_TIMEOUT_MILLIS) + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index eef6aafa624e..04cccc67e328 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -18,16 +18,16 @@ package org.apache.spark.scheduler import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException, SparkFunSuite} -import org.apache.spark.util.{SerializableBuffer, AkkaUtils} +import org.apache.spark.util.{RpcUtils, SerializableBuffer} class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext { - test("serialized task larger than akka frame size") { + test("serialized task larger than max RPC message size") { val conf = new SparkConf - conf.set("spark.akka.frameSize", "1") + conf.set("spark.rpc.message.maxSize", "1") conf.set("spark.default.parallelism", "1") sc = new SparkContext("local-cluster[2, 1, 1024]", "test", conf) - val frameSize = AkkaUtils.maxFrameSizeBytes(sc.conf) + val frameSize = RpcUtils.maxMessageSizeBytes(sc.conf) val buffer = new SerializableBuffer(java.nio.ByteBuffer.allocate(2 * frameSize)) val larger = sc.parallelize(Seq(buffer)) val thrown = intercept[SparkException] { diff --git a/core/src/test/scala/org/apache/spark/scheduler/CustomShuffledRDD.scala b/core/src/test/scala/org/apache/spark/scheduler/CustomShuffledRDD.scala index d8d818ceed45..838686923767 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CustomShuffledRDD.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CustomShuffledRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.util.Arrays +import java.util.Objects import org.apache.spark._ import org.apache.spark.rdd.RDD @@ -53,6 +54,9 @@ class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: A parentPartitionMapping(parent.getPartition(key)) } + override def hashCode(): Int = + 31 * Objects.hashCode(parent) + Arrays.hashCode(partitionStartIndices) + override def equals(other: Any): Boolean = other match { case c: CoalescedPartitioner => c.parent == parent && Arrays.equals(c.partitionStartIndices, partitionStartIndices) @@ -66,6 +70,8 @@ private[spark] class CustomShuffledRDDPartition( extends Partition { override def hashCode(): Int = index + + override def equals(other: Any): Boolean = super.equals(other) } /** 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 3816b8c4a09a..6222e576d1ce 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -17,20 +17,24 @@ package org.apache.spark.scheduler -import scala.collection.mutable.{ArrayBuffer, HashSet, HashMap, Map} +import java.util.Properties +import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} + +import scala.annotation.meta.param +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import scala.language.reflectiveCalls import scala.util.control.NonFatal -import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.Timeouts +import org.scalatest.concurrent.TimeLimits import org.scalatest.time.SpanSugar._ import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode +import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException} import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} -import org.apache.spark.util.CallSite +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, LongAccumulator, Utils} class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) extends DAGSchedulerEventProcessLoop(dagScheduler) { @@ -43,6 +47,13 @@ class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) case NonFatal(e) => onError(e) } } + + override def onError(e: Throwable): Unit = { + logError("Error in DAGSchedulerEventLoop: ", e) + dagScheduler.stop() + throw e + } + } /** @@ -59,7 +70,7 @@ class MyRDD( numPartitions: Int, dependencies: List[Dependency[_]], locations: Seq[Seq[String]] = Nil, - @transient tracker: MapOutputTrackerMaster = null) + @(transient @param) tracker: MapOutputTrackerMaster = null) extends RDD[(Int, Int)](sc, dependencies) with Serializable { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = @@ -87,8 +98,9 @@ class MyRDD( class DAGSchedulerSuiteDummyException extends Exception -class DAGSchedulerSuite - extends SparkFunSuite with BeforeAndAfter with LocalSparkContext with Timeouts { +class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLimits { + + import DAGSchedulerSuite._ val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ @@ -98,12 +110,14 @@ class DAGSchedulerSuite val cancelledStages = new HashSet[Int]() val taskScheduler = new TaskScheduler() { - override def rootPool: Pool = null - override def schedulingMode: SchedulingMode = SchedulingMode.NONE + override def schedulingMode: SchedulingMode = SchedulingMode.FIFO + override def rootPool: Pool = new Pool("", schedulingMode, 0, 0) override def start() = {} override def stop() = {} - override def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], - blockManagerId: BlockManagerId): Boolean = true + override def executorHeartbeatReceived( + execId: String, + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], + blockManagerId: BlockManagerId): Boolean = true override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) @@ -112,9 +126,12 @@ class DAGSchedulerSuite override def cancelTasks(stageId: Int, interruptThread: Boolean) { cancelledStages += stageId } + override def killTaskAttempt( + taskId: Long, interruptThread: Boolean, reason: String): Boolean = false override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} + override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None } @@ -125,6 +142,7 @@ class DAGSchedulerSuite val successfulStages = new HashSet[Int] val failedStages = new ArrayBuffer[Int] val stageByOrderOfExecution = new ArrayBuffer[Int] + val endedTasks = new HashSet[Long] override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { submittedStageInfos += stageSubmitted.stageInfo @@ -139,9 +157,15 @@ class DAGSchedulerSuite failedStages += stageInfo.stageId } } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + endedTasks += taskEnd.taskInfo.taskId + } } var mapOutputTracker: MapOutputTrackerMaster = null + var broadcastManager: BroadcastManager = null + var securityMgr: SecurityManager = null var scheduler: DAGScheduler = null var dagEventProcessLoopTester: DAGSchedulerEventProcessLoop = null @@ -180,30 +204,49 @@ class DAGSchedulerSuite override def jobFailed(exception: Exception): Unit = { failure = exception } } - before { - sc = new SparkContext("local", "DAGSchedulerSuite") + override def beforeEach(): Unit = { + super.beforeEach() + init(new SparkConf()) + } + + private def init(testConf: SparkConf): Unit = { + sc = new SparkContext("local", "DAGSchedulerSuite", testConf) sparkListener.submittedStageInfos.clear() sparkListener.successfulStages.clear() sparkListener.failedStages.clear() + sparkListener.endedTasks.clear() failure = null sc.addSparkListener(sparkListener) taskSets.clear() cancelledStages.clear() cacheLocations.clear() results.clear() - mapOutputTracker = new MapOutputTrackerMaster(conf) + securityMgr = new SecurityManager(conf) + broadcastManager = new BroadcastManager(true, conf, securityMgr) + mapOutputTracker = new MapOutputTrackerMaster(conf, broadcastManager, true) { + override def sendTracker(message: Any): Unit = { + // no-op, just so we can stop this to avoid leaking threads + } + } scheduler = new DAGScheduler( - sc, - taskScheduler, - sc.listenerBus, - mapOutputTracker, - blockManagerMaster, - sc.env) + sc, + taskScheduler, + sc.listenerBus, + mapOutputTracker, + blockManagerMaster, + sc.env) dagEventProcessLoopTester = new DAGSchedulerEventProcessLoopTester(scheduler) } - after { - scheduler.stop() + override def afterEach(): Unit = { + try { + scheduler.stop() + dagEventProcessLoopTester.stop() + mapOutputTracker.stop() + broadcastManager.stop() + } finally { + super.afterEach() + } } override def afterAll() { @@ -233,26 +276,30 @@ class DAGSchedulerSuite * directly through CompletionEvents. */ private val jobComputeFunc = (context: TaskContext, it: Iterator[(_)]) => - it.next.asInstanceOf[Tuple2[_, _]]._1 + it.next.asInstanceOf[Tuple2[_, _]]._1 /** Send the given CompletionEvent messages for the tasks in the TaskSet. */ private def complete(taskSet: TaskSet, results: Seq[(TaskEndReason, Any)]) { assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { - runEvent(CompletionEvent( - taskSet.tasks(i), result._1, result._2, null, createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent(taskSet.tasks(i), result._1, result._2)) } } } - private def completeWithAccumulator(accumId: Long, taskSet: TaskSet, - results: Seq[(TaskEndReason, Any)]) { + private def completeWithAccumulator( + accumId: Long, + taskSet: TaskSet, + results: Seq[(TaskEndReason, Any)]) { 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, - Map[Long, Any]((accumId, 1)), createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent( + taskSet.tasks(i), + result._1, + result._2, + Seq(AccumulatorSuite.createLongAccum("", initValue = 1, id = accumId)))) } } } @@ -262,9 +309,10 @@ class DAGSchedulerSuite rdd: RDD[_], partitions: Array[Int], func: (TaskContext, Iterator[_]) => _ = jobComputeFunc, - listener: JobListener = jobListener): Int = { + listener: JobListener = jobListener, + properties: Properties = null): Int = { val jobId = scheduler.nextJobId.getAndIncrement() - runEvent(JobSubmitted(jobId, rdd, func, partitions, CallSite("", ""), listener)) + runEvent(JobSubmitted(jobId, rdd, func, partitions, CallSite("", ""), listener, properties)) jobId } @@ -284,7 +332,7 @@ class DAGSchedulerSuite /** Sends JobCancelled to the DAG scheduler. */ private def cancel(jobId: Int) { - runEvent(JobCancelled(jobId)) + runEvent(JobCancelled(jobId, None)) } test("[SPARK-3353] parent stage should have lower stage id") { @@ -295,15 +343,141 @@ class DAGSchedulerSuite assert(sparkListener.stageByOrderOfExecution(0) < sparkListener.stageByOrderOfExecution(1)) } + /** + * This test ensures that DAGScheduler build stage graph correctly. + * + * Suppose you have the following DAG: + * + * [A] <--(s_A)-- [B] <--(s_B)-- [C] <--(s_C)-- [D] + * \ / + * <------------- + * + * Here, RDD B has a shuffle dependency on RDD A, and RDD C has shuffle dependency on both + * B and A. The shuffle dependency IDs are numbers in the DAGScheduler, but to make the example + * easier to understand, let's call the shuffled data from A shuffle dependency ID s_A and the + * shuffled data from B shuffle dependency ID s_B. + * + * Note: [] means an RDD, () means a shuffle dependency. + */ + test("[SPARK-13902] Ensure no duplicate stages are created") { + val rddA = new MyRDD(sc, 1, Nil) + val shuffleDepA = new ShuffleDependency(rddA, new HashPartitioner(1)) + val s_A = shuffleDepA.shuffleId + + val rddB = new MyRDD(sc, 1, List(shuffleDepA), tracker = mapOutputTracker) + val shuffleDepB = new ShuffleDependency(rddB, new HashPartitioner(1)) + val s_B = shuffleDepB.shuffleId + + val rddC = new MyRDD(sc, 1, List(shuffleDepA, shuffleDepB), tracker = mapOutputTracker) + val shuffleDepC = new ShuffleDependency(rddC, new HashPartitioner(1)) + val s_C = shuffleDepC.shuffleId + + val rddD = new MyRDD(sc, 1, List(shuffleDepC), tracker = mapOutputTracker) + + submit(rddD, Array(0)) + + assert(scheduler.shuffleIdToMapStage.size === 3) + assert(scheduler.activeJobs.size === 1) + + val mapStageA = scheduler.shuffleIdToMapStage(s_A) + val mapStageB = scheduler.shuffleIdToMapStage(s_B) + val mapStageC = scheduler.shuffleIdToMapStage(s_C) + val finalStage = scheduler.activeJobs.head.finalStage + + assert(mapStageA.parents.isEmpty) + assert(mapStageB.parents === List(mapStageA)) + assert(mapStageC.parents === List(mapStageA, mapStageB)) + assert(finalStage.parents === List(mapStageC)) + + complete(taskSets(0), Seq((Success, makeMapStatus("hostA", 1)))) + complete(taskSets(1), Seq((Success, makeMapStatus("hostA", 1)))) + complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) + complete(taskSets(3), Seq((Success, 42))) + assert(results === Map(0 -> 42)) + assertDataStructuresEmpty() + } + + test("All shuffle files on the slave should be cleaned up when slave lost") { + // reset the test context with the right shuffle service config + afterEach() + val conf = new SparkConf() + conf.set("spark.shuffle.service.enabled", "true") + conf.set("spark.files.fetchFailure.unRegisterOutputOnHost", "true") + init(conf) + runEvent(ExecutorAdded("exec-hostA1", "hostA")) + runEvent(ExecutorAdded("exec-hostA2", "hostA")) + runEvent(ExecutorAdded("exec-hostB", "hostB")) + val firstRDD = new MyRDD(sc, 3, Nil) + val firstShuffleDep = new ShuffleDependency(firstRDD, new HashPartitioner(3)) + val firstShuffleId = firstShuffleDep.shuffleId + val shuffleMapRdd = new MyRDD(sc, 3, List(firstShuffleDep)) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(3)) + val secondShuffleId = shuffleDep.shuffleId + val reduceRdd = new MyRDD(sc, 1, List(shuffleDep)) + submit(reduceRdd, Array(0)) + // map stage1 completes successfully, with one task on each executor + complete(taskSets(0), Seq( + (Success, + MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2))), + (Success, + MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2))), + (Success, makeMapStatus("hostB", 1)) + )) + // map stage2 completes successfully, with one task on each executor + complete(taskSets(1), Seq( + (Success, + MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2))), + (Success, + MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2))), + (Success, makeMapStatus("hostB", 1)) + )) + // make sure our test setup is correct + val initialMapStatus1 = mapOutputTracker.shuffleStatuses(firstShuffleId).mapStatuses + // val initialMapStatus1 = mapOutputTracker.mapStatuses.get(0).get + assert(initialMapStatus1.count(_ != null) === 3) + assert(initialMapStatus1.map{_.location.executorId}.toSet === + Set("exec-hostA1", "exec-hostA2", "exec-hostB")) + + val initialMapStatus2 = mapOutputTracker.shuffleStatuses(secondShuffleId).mapStatuses + // val initialMapStatus1 = mapOutputTracker.mapStatuses.get(0).get + assert(initialMapStatus2.count(_ != null) === 3) + assert(initialMapStatus2.map{_.location.executorId}.toSet === + Set("exec-hostA1", "exec-hostA2", "exec-hostB")) + + // reduce stage fails with a fetch failure from one host + complete(taskSets(2), Seq( + (FetchFailed(BlockManagerId("exec-hostA2", "hostA", 12345), firstShuffleId, 0, 0, "ignored"), + null) + )) + + // Here is the main assertion -- make sure that we de-register + // the map outputs for both map stage from both executors on hostA + + val mapStatus1 = mapOutputTracker.shuffleStatuses(firstShuffleId).mapStatuses + assert(mapStatus1.count(_ != null) === 1) + assert(mapStatus1(2).location.executorId === "exec-hostB") + assert(mapStatus1(2).location.host === "hostB") + + val mapStatus2 = mapOutputTracker.shuffleStatuses(secondShuffleId).mapStatuses + assert(mapStatus2.count(_ != null) === 1) + assert(mapStatus2(2).location.executorId === "exec-hostB") + assert(mapStatus2(2).location.host === "hostB") + } + test("zero split job") { var numResults = 0 + var failureReason: Option[Exception] = None val fakeListener = new JobListener() { - override def taskSucceeded(partition: Int, value: Any) = numResults += 1 - override def jobFailed(exception: Exception) = throw exception + override def taskSucceeded(partition: Int, value: Any): Unit = numResults += 1 + override def jobFailed(exception: Exception): Unit = { + failureReason = Some(exception) + } } val jobId = submit(new MyRDD(sc, 0, Nil), Array(), listener = fakeListener) assert(numResults === 0) cancel(jobId) + assert(failureReason.isDefined) + assert(failureReason.get.getMessage() === "Job 0 cancelled ") } test("run trivial job") { @@ -323,9 +497,12 @@ class DAGSchedulerSuite } test("equals and hashCode AccumulableInfo") { - val accInfo1 = new AccumulableInfo(1, " Accumulable " + 1, Some("delta" + 1), "val" + 1, true) - val accInfo2 = new AccumulableInfo(1, " Accumulable " + 1, Some("delta" + 1), "val" + 1, false) - val accInfo3 = new AccumulableInfo(1, " Accumulable " + 1, Some("delta" + 1), "val" + 1, false) + val accInfo1 = new AccumulableInfo( + 1, Some("a1"), Some("delta1"), Some("val1"), internal = true, countFailedValues = false) + val accInfo2 = new AccumulableInfo( + 1, Some("a1"), Some("delta1"), Some("val1"), internal = false, countFailedValues = false) + val accInfo3 = new AccumulableInfo( + 1, Some("a1"), Some("delta1"), Some("val1"), internal = false, countFailedValues = false) assert(accInfo1 !== accInfo2) assert(accInfo2 === accInfo3) assert(accInfo2.hashCode() === accInfo3.hashCode()) @@ -435,8 +612,8 @@ class DAGSchedulerSuite // make sure that the DAGScheduler doesn't crash when the TaskScheduler // doesn't implement killTask() val noKillTaskScheduler = new TaskScheduler() { - override def rootPool: Pool = null - override def schedulingMode: SchedulingMode = SchedulingMode.NONE + override def schedulingMode: SchedulingMode = SchedulingMode.FIFO + override def rootPool: Pool = new Pool("", schedulingMode, 0, 0) override def start(): Unit = {} override def stop(): Unit = {} override def submitTasks(taskSet: TaskSet): Unit = { @@ -445,13 +622,18 @@ class DAGSchedulerSuite override def cancelTasks(stageId: Int, interruptThread: Boolean) { throw new UnsupportedOperationException } + override def killTaskAttempt( + taskId: Long, interruptThread: Boolean, reason: String): Boolean = { + throw new UnsupportedOperationException + } override def setDAGScheduler(dagScheduler: DAGScheduler): Unit = {} override def defaultParallelism(): Int = 2 override def executorHeartbeatReceived( execId: String, - taskMetrics: Array[(Long, TaskMetrics)], + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId): Boolean = true override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} + override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None } val noKillScheduler = new DAGScheduler( @@ -484,8 +666,8 @@ class DAGSchedulerSuite val reduceRdd = new MyRDD(sc, 1, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0)) complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)))) assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) complete(taskSets(1), Seq((Success, 42))) @@ -500,12 +682,12 @@ class DAGSchedulerSuite val reduceRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0, 1)) complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", reduceRdd.partitions.length)), - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)))) + (Success, makeMapStatus("hostA", reduceRdd.partitions.length)), + (Success, makeMapStatus("hostB", reduceRdd.partitions.length)))) // the 2nd ResultTask failed complete(taskSets(1), Seq( - (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) + (Success, 42), + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) // this will get called // blockManagerMaster.removeExecutor("exec-hostA") // ask the scheduler to try it again @@ -520,6 +702,46 @@ class DAGSchedulerSuite assertDataStructuresEmpty() } + private val shuffleFileLossTests = Seq( + ("slave lost with shuffle service", SlaveLost("", false), true, false), + ("worker lost with shuffle service", SlaveLost("", true), true, true), + ("worker lost without shuffle service", SlaveLost("", true), false, true), + ("executor failure with shuffle service", ExecutorKilled, true, false), + ("executor failure without shuffle service", ExecutorKilled, false, true)) + + for ((eventDescription, event, shuffleServiceOn, expectFileLoss) <- shuffleFileLossTests) { + val maybeLost = if (expectFileLoss) { + "lost" + } else { + "not lost" + } + test(s"shuffle files $maybeLost when $eventDescription") { + // reset the test context with the right shuffle service config + afterEach() + val conf = new SparkConf() + conf.set("spark.shuffle.service.enabled", shuffleServiceOn.toString) + init(conf) + assert(sc.env.blockManager.externalShuffleServiceEnabled == shuffleServiceOn) + + val shuffleMapRdd = new MyRDD(sc, 2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(1)) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = new MyRDD(sc, 1, List(shuffleDep), tracker = mapOutputTracker) + submit(reduceRdd, Array(0)) + complete(taskSets(0), Seq( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)))) + runEvent(ExecutorLost("exec-hostA", event)) + if (expectFileLoss) { + intercept[MetadataFetchFailedException] { + mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0) + } + } else { + assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === + HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) + } + } + } // Helper function to validate state when creating tests for task failures private def checkStageId(stageId: Int, attempt: Int, stageAttempt: TaskSet) { @@ -527,10 +749,9 @@ class DAGSchedulerSuite assert(stageAttempt.stageAttemptId == attempt) } - // Helper functions to extract commonly used code in Fetch Failure test cases private def setupStageAbortTest(sc: SparkContext) { - sc.listenerBus.addListener(new EndListener()) + sc.listenerBus.addToSharedQueue(new EndListener()) ended = false jobResult = null } @@ -594,11 +815,17 @@ class DAGSchedulerSuite * @param stageId - The current stageId * @param attemptIdx - The current attempt count */ - private def completeNextResultStageWithSuccess(stageId: Int, attemptIdx: Int): Unit = { + private def completeNextResultStageWithSuccess( + stageId: Int, + attemptIdx: Int, + partitionToResult: Int => Int = _ => 42): Unit = { val stageAttempt = taskSets.last checkStageId(stageId, attemptIdx, stageAttempt) assert(scheduler.stageIdToStage(stageId).isInstanceOf[ResultStage]) - complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map(_ => (Success, 42)).toSeq) + val taskResults = stageAttempt.tasks.zipWithIndex.map { case (task, idx) => + (Success, partitionToResult(idx)) + } + complete(stageAttempt, taskResults.toSeq) } /** @@ -629,7 +856,7 @@ class DAGSchedulerSuite completeShuffleMapStageSuccessfully(0, 1, numShufflePartitions = parts) completeNextResultStageWithSuccess(1, 1) - // Confirm job finished succesfully + // Confirm job finished successfully sc.listenerBus.waitUntilEmpty(1000) assert(ended === true) assert(results === (0 until parts).map { idx => idx -> 42 }.toMap) @@ -649,7 +876,7 @@ class DAGSchedulerSuite val reduceRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0, 1)) - for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FETCH_FAILURES) { + for (attempt <- 0 until scheduler.maxConsecutiveStageAttempts) { // Complete all the tasks for the current attempt of stage 0 successfully completeShuffleMapStageSuccessfully(0, attempt, numShufflePartitions = 2) @@ -661,7 +888,7 @@ class DAGSchedulerSuite // map output, for the next iteration through the loop scheduler.resubmitFailedStages() - if (attempt < Stage.MAX_CONSECUTIVE_FETCH_FAILURES - 1) { + if (attempt < scheduler.maxConsecutiveStageAttempts - 1) { assert(scheduler.runningStages.nonEmpty) assert(!ended) } else { @@ -695,11 +922,11 @@ class DAGSchedulerSuite // In the first two iterations, Stage 0 succeeds and stage 1 fails. In the next two iterations, // stage 2 fails. - for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FETCH_FAILURES) { + for (attempt <- 0 until scheduler.maxConsecutiveStageAttempts) { // Complete all the tasks for the current attempt of stage 0 successfully completeShuffleMapStageSuccessfully(0, attempt, numShufflePartitions = 2) - if (attempt < Stage.MAX_CONSECUTIVE_FETCH_FAILURES / 2) { + if (attempt < scheduler.maxConsecutiveStageAttempts / 2) { // Now we should have a new taskSet, for a new attempt of stage 1. // Fail all these tasks with FetchFailure completeNextStageWithFetchFailure(1, attempt, shuffleDepOne) @@ -707,8 +934,8 @@ class DAGSchedulerSuite completeShuffleMapStageSuccessfully(1, attempt, numShufflePartitions = 1) // Fail stage 2 - completeNextStageWithFetchFailure(2, attempt - Stage.MAX_CONSECUTIVE_FETCH_FAILURES / 2, - shuffleDepTwo) + completeNextStageWithFetchFailure(2, + attempt - scheduler.maxConsecutiveStageAttempts / 2, shuffleDepTwo) } // this will trigger a resubmission of stage 0, since we've lost some of its @@ -720,7 +947,7 @@ class DAGSchedulerSuite completeShuffleMapStageSuccessfully(1, 4, numShufflePartitions = 1) // Succeed stage2 with a "42" - completeNextResultStageWithSuccess(2, Stage.MAX_CONSECUTIVE_FETCH_FAILURES/2) + completeNextResultStageWithSuccess(2, scheduler.maxConsecutiveStageAttempts / 2) assert(results === Map(0 -> 42)) assertDataStructuresEmpty() @@ -743,7 +970,7 @@ class DAGSchedulerSuite submit(finalRdd, Array(0)) // First, execute stages 0 and 1, failing stage 1 up to MAX-1 times. - for (attempt <- 0 until Stage.MAX_CONSECUTIVE_FETCH_FAILURES - 1) { + for (attempt <- 0 until scheduler.maxConsecutiveStageAttempts - 1) { // Make each task in stage 0 success completeShuffleMapStageSuccessfully(0, attempt, numShufflePartitions = 2) @@ -808,23 +1035,17 @@ class DAGSchedulerSuite HashSet("hostA", "hostB")) // The first result task fails, with a fetch failure for the output from the first mapper. - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(0), FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), - null, - Map[Long, Any](), - createFakeTaskInfo(), null)) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(sparkListener.failedStages.contains(1)) // The second ResultTask fails, with a fetch failure for the output from the second mapper. - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(0), FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1, "ignored"), - null, - Map[Long, Any](), - createFakeTaskInfo(), null)) // The SparkListener should not receive redundant failure events. sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) @@ -861,12 +1082,9 @@ class DAGSchedulerSuite HashSet("hostA", "hostB")) // The first result task fails, with a fetch failure for the output from the first mapper. - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(0), FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), - null, - Map[Long, Any](), - createFakeTaskInfo(), null)) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(sparkListener.failedStages.contains(1)) @@ -879,12 +1097,9 @@ class DAGSchedulerSuite assert(countSubmittedMapStageAttempts() === 2) // The second ResultTask fails, with a fetch failure for the output from the second mapper. - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(1), FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1, 1, "ignored"), - null, - Map[Long, Any](), - createFakeTaskInfo(), null)) // Another ResubmitFailedStages event should not result in another attempt for the map @@ -899,11 +1114,11 @@ class DAGSchedulerSuite } /** - * This tests the case where a late FetchFailed comes in after the map stage has finished getting - * retried and a new reduce stage starts running. - */ + * This tests the case where a late FetchFailed comes in after the map stage has finished getting + * retried and a new reduce stage starts running. + */ test("extremely late fetch failures don't cause multiple concurrent attempts for " + - "the same stage") { + "the same stage") { val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) val shuffleId = shuffleDep.shuffleId @@ -931,12 +1146,9 @@ class DAGSchedulerSuite assert(countSubmittedReduceStageAttempts() === 1) // The first result task fails, with a fetch failure for the output from the first mapper. - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(0), FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), - null, - Map[Long, Any](), - createFakeTaskInfo(), null)) // Trigger resubmission of the failed map stage and finish the re-started map task. @@ -950,12 +1162,9 @@ class DAGSchedulerSuite assert(countSubmittedReduceStageAttempts() === 2) // A late FetchFailed arrives from the second task in the original reduce stage. - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(1), FetchFailed(makeBlockManagerId("hostB"), shuffleId, 1, 1, "ignored"), - null, - Map[Long, Any](), - createFakeTaskInfo(), null)) // Running ResubmitFailedStages shouldn't result in any more attempts for the map stage, because @@ -966,6 +1175,52 @@ class DAGSchedulerSuite assert(countSubmittedMapStageAttempts() === 2) } + test("task events always posted in speculation / when stage is killed") { + val baseRdd = new MyRDD(sc, 4, Nil) + val finalRdd = new MyRDD(sc, 4, List(new OneToOneDependency(baseRdd))) + submit(finalRdd, Array(0, 1, 2, 3)) + + // complete two tasks + runEvent(makeCompletionEvent( + taskSets(0).tasks(0), Success, 42, + Seq.empty, createFakeTaskInfoWithId(0))) + runEvent(makeCompletionEvent( + taskSets(0).tasks(1), Success, 42, + Seq.empty, createFakeTaskInfoWithId(1))) + sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + // verify stage exists + assert(scheduler.stageIdToStage.contains(0)) + assert(sparkListener.endedTasks.size == 2) + + // finish other 2 tasks + runEvent(makeCompletionEvent( + taskSets(0).tasks(2), Success, 42, + Seq.empty, createFakeTaskInfoWithId(2))) + runEvent(makeCompletionEvent( + taskSets(0).tasks(3), Success, 42, + Seq.empty, createFakeTaskInfoWithId(3))) + sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + assert(sparkListener.endedTasks.size == 4) + + // verify the stage is done + assert(!scheduler.stageIdToStage.contains(0)) + + // Stage should be complete. Finish one other Successful task to simulate what can happen + // with a speculative task and make sure the event is sent out + runEvent(makeCompletionEvent( + taskSets(0).tasks(3), Success, 42, + Seq.empty, createFakeTaskInfoWithId(5))) + sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + assert(sparkListener.endedTasks.size == 5) + + // make sure non successful tasks also send out event + runEvent(makeCompletionEvent( + taskSets(0).tasks(3), UnknownReason, 42, + Seq.empty, createFakeTaskInfoWithId(6))) + sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + assert(sparkListener.endedTasks.size == 6) + } + test("ignore late map task completions") { val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) @@ -975,7 +1230,7 @@ class DAGSchedulerSuite // pretend we were told hostA went away val oldEpoch = mapOutputTracker.getEpoch - runEvent(ExecutorLost("exec-hostA")) + runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) val newEpoch = mapOutputTracker.getEpoch assert(newEpoch > oldEpoch) @@ -986,48 +1241,36 @@ class DAGSchedulerSuite assert(shuffleStage.numAvailableOutputs === 0) // should be ignored for being too old - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSet.tasks(0), Success, - makeMapStatus("hostA", reduceRdd.partitions.size), - null, - createFakeTaskInfo(), - null)) + makeMapStatus("hostA", reduceRdd.partitions.size))) assert(shuffleStage.numAvailableOutputs === 0) // should work because it's a non-failed host (so the available map outputs will increase) - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSet.tasks(0), Success, - makeMapStatus("hostB", reduceRdd.partitions.size), - null, - createFakeTaskInfo(), - null)) + makeMapStatus("hostB", reduceRdd.partitions.size))) assert(shuffleStage.numAvailableOutputs === 1) // should be ignored for being too old - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSet.tasks(0), Success, - makeMapStatus("hostA", reduceRdd.partitions.size), - null, - createFakeTaskInfo(), - null)) + makeMapStatus("hostA", reduceRdd.partitions.size))) assert(shuffleStage.numAvailableOutputs === 1) // should work because it's a new epoch, which will increase the number of available map // outputs, and also finish the stage taskSet.tasks(1).epoch = newEpoch - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSet.tasks(1), Success, - makeMapStatus("hostA", reduceRdd.partitions.size), - null, - createFakeTaskInfo(), - null)) + makeMapStatus("hostA", reduceRdd.partitions.size))) assert(shuffleStage.numAvailableOutputs === 2) assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === - HashSet(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) + HashSet(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) // finish the next stage normally, which completes the job complete(taskSets(1), Seq((Success, 42), (Success, 43))) @@ -1054,6 +1297,47 @@ class DAGSchedulerSuite assertDataStructuresEmpty() } + /** + * Run two jobs, with a shared dependency. We simulate a fetch failure in the second job, which + * requires regenerating some outputs of the shared dependency. One key aspect of this test is + * that the second job actually uses a different stage for the shared dependency (a "skipped" + * stage). + */ + test("shuffle fetch failure in a reused shuffle dependency") { + // Run the first job successfully, which creates one shuffle dependency + + val shuffleMapRdd = new MyRDD(sc, 2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) + submit(reduceRdd, Array(0, 1)) + + completeShuffleMapStageSuccessfully(0, 0, 2) + completeNextResultStageWithSuccess(1, 0) + assert(results === Map(0 -> 42, 1 -> 42)) + assertDataStructuresEmpty() + + // submit another job w/ the shared dependency, and have a fetch failure + val reduce2 = new MyRDD(sc, 2, List(shuffleDep)) + submit(reduce2, Array(0, 1)) + // Note that the stage numbering here is only b/c the shared dependency produces a new, skipped + // stage. If instead it reused the existing stage, then this would be stage 2 + completeNextStageWithFetchFailure(3, 0, shuffleDep) + scheduler.resubmitFailedStages() + + // the scheduler now creates a new task set to regenerate the missing map output, but this time + // using a different stage, the "skipped" one + + // SPARK-9809 -- this stage is submitted without a task for each partition (because some of + // the shuffle map output is still available from stage 0); make sure we've still got internal + // accumulators setup + assert(scheduler.stageIdToStage(2).latestInfo.taskMetrics != null) + completeShuffleMapStageSuccessfully(2, 0, 2) + completeNextResultStageWithSuccess(3, 1, idx => idx + 1234) + assert(results === Map(0 -> 1234, 1 -> 1235)) + + assertDataStructuresEmpty() + } + /** * This test runs a three stage job, with a fetch failure in stage 1. but during the retry, we * have completions from both the first & second attempt of stage 1. So all the map output is @@ -1062,10 +1346,10 @@ class DAGSchedulerSuite */ test("don't submit stage until its dependencies map outputs are registered (SPARK-5259)") { val firstRDD = new MyRDD(sc, 3, Nil) - val firstShuffleDep = new ShuffleDependency(firstRDD, new HashPartitioner(2)) + val firstShuffleDep = new ShuffleDependency(firstRDD, new HashPartitioner(3)) val firstShuffleId = firstShuffleDep.shuffleId val shuffleMapRdd = new MyRDD(sc, 3, List(firstShuffleDep)) - val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(1)) val reduceRdd = new MyRDD(sc, 1, List(shuffleDep)) submit(reduceRdd, Array(0)) @@ -1077,13 +1361,10 @@ class DAGSchedulerSuite )) // then one executor dies, and a task fails in stage 1 - runEvent(ExecutorLost("exec-hostA")) - runEvent(CompletionEvent( + runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) + runEvent(makeCompletionEvent( taskSets(1).tasks(0), FetchFailed(null, firstShuffleId, 2, 0, "Fetch failed"), - null, - null, - createFakeTaskInfo(), null)) // so we resubmit stage 0, which completes happily @@ -1093,13 +1374,10 @@ class DAGSchedulerSuite assert(stage0Resubmit.stageAttemptId === 1) val task = stage0Resubmit.tasks(0) assert(task.partitionId === 2) - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( task, Success, - makeMapStatus("hostC", shuffleMapRdd.partitions.length), - null, - createFakeTaskInfo(), - null)) + makeMapStatus("hostC", shuffleMapRdd.partitions.length))) // now here is where things get tricky : we will now have a task set representing // the second attempt for stage 1, but we *also* have some tasks for the first attempt for @@ -1112,28 +1390,19 @@ class DAGSchedulerSuite // we'll have some tasks finish from the first attempt, and some finish from the second attempt, // so that we actually have all stage outputs, though no attempt has completed all its // tasks - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(3).tasks(0), Success, - makeMapStatus("hostC", reduceRdd.partitions.length), - null, - createFakeTaskInfo(), - null)) - runEvent(CompletionEvent( + makeMapStatus("hostC", reduceRdd.partitions.length))) + runEvent(makeCompletionEvent( taskSets(3).tasks(1), Success, - makeMapStatus("hostC", reduceRdd.partitions.length), - null, - createFakeTaskInfo(), - null)) + makeMapStatus("hostC", reduceRdd.partitions.length))) // late task finish from the first attempt - runEvent(CompletionEvent( + runEvent(makeCompletionEvent( taskSets(1).tasks(2), Success, - makeMapStatus("hostB", reduceRdd.partitions.length), - null, - createFakeTaskInfo(), - null)) + makeMapStatus("hostB", reduceRdd.partitions.length))) // What should happen now is that we submit stage 2. However, we might not see an error // b/c of DAGScheduler's error handling (it tends to swallow errors and just log them). But @@ -1180,21 +1449,21 @@ class DAGSchedulerSuite submit(reduceRdd, Array(0)) // complete some of the tasks from the first stage, on one host - runEvent(CompletionEvent( - taskSets(0).tasks(0), Success, - makeMapStatus("hostA", reduceRdd.partitions.length), null, createFakeTaskInfo(), null)) - runEvent(CompletionEvent( - taskSets(0).tasks(1), Success, - makeMapStatus("hostA", reduceRdd.partitions.length), null, createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent( + taskSets(0).tasks(0), + Success, + makeMapStatus("hostA", reduceRdd.partitions.length))) + runEvent(makeCompletionEvent( + taskSets(0).tasks(1), + Success, + makeMapStatus("hostA", reduceRdd.partitions.length))) // now that host goes down - runEvent(ExecutorLost("exec-hostA")) + runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) // so we resubmit those tasks - runEvent(CompletionEvent( - taskSets(0).tasks(0), Resubmitted, null, null, createFakeTaskInfo(), null)) - runEvent(CompletionEvent( - taskSets(0).tasks(1), Resubmitted, null, null, createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent(taskSets(0).tasks(0), Resubmitted, null)) + runEvent(makeCompletionEvent(taskSets(0).tasks(1), Resubmitted, null)) // now complete everything on a different host complete(taskSets(0), Seq( @@ -1275,24 +1544,145 @@ class DAGSchedulerSuite assertDataStructuresEmpty() } - test("run trivial shuffle with out-of-band failure and retry") { + def checkJobPropertiesAndPriority(taskSet: TaskSet, expected: String, priority: Int): Unit = { + assert(taskSet.properties != null) + assert(taskSet.properties.getProperty("testProperty") === expected) + assert(taskSet.priority === priority) + } + + def launchJobsThatShareStageAndCancelFirst(): ShuffleDependency[Int, Int, Nothing] = { + val baseRdd = new MyRDD(sc, 1, Nil) + val shuffleDep1 = new ShuffleDependency(baseRdd, new HashPartitioner(1)) + val intermediateRdd = new MyRDD(sc, 1, List(shuffleDep1)) + val shuffleDep2 = new ShuffleDependency(intermediateRdd, new HashPartitioner(1)) + val finalRdd1 = new MyRDD(sc, 1, List(shuffleDep2)) + val finalRdd2 = new MyRDD(sc, 1, List(shuffleDep2)) + val job1Properties = new Properties() + val job2Properties = new Properties() + job1Properties.setProperty("testProperty", "job1") + job2Properties.setProperty("testProperty", "job2") + + // Run jobs 1 & 2, both referencing the same stage, then cancel job1. + // Note that we have to submit job2 before we cancel job1 to have them actually share + // *Stages*, and not just shuffle dependencies, due to skipped stages (at least until + // we address SPARK-10193.) + val jobId1 = submit(finalRdd1, Array(0), properties = job1Properties) + val jobId2 = submit(finalRdd2, Array(0), properties = job2Properties) + assert(scheduler.activeJobs.nonEmpty) + val testProperty1 = scheduler.jobIdToActiveJob(jobId1).properties.getProperty("testProperty") + + // remove job1 as an ActiveJob + cancel(jobId1) + + // job2 should still be running + assert(scheduler.activeJobs.nonEmpty) + val testProperty2 = scheduler.jobIdToActiveJob(jobId2).properties.getProperty("testProperty") + assert(testProperty1 != testProperty2) + // NB: This next assert isn't necessarily the "desired" behavior; it's just to document + // the current behavior. We've already submitted the TaskSet for stage 0 based on job1, but + // even though we have cancelled that job and are now running it because of job2, we haven't + // updated the TaskSet's properties. Changing the properties to "job2" is likely the more + // correct behavior. + val job1Id = 0 // TaskSet priority for Stages run with "job1" as the ActiveJob + checkJobPropertiesAndPriority(taskSets(0), "job1", job1Id) + complete(taskSets(0), Seq((Success, makeMapStatus("hostA", 1)))) + + shuffleDep1 + } + + /** + * Makes sure that tasks for a stage used by multiple jobs are submitted with the properties of a + * later, active job if they were previously run under a job that is no longer active + */ + test("stage used by two jobs, the first no longer active (SPARK-6880)") { + launchJobsThatShareStageAndCancelFirst() + + // The next check is the key for SPARK-6880. For the stage which was shared by both job1 and + // job2 but never had any tasks submitted for job1, the properties of job2 are now used to run + // the stage. + checkJobPropertiesAndPriority(taskSets(1), "job2", 1) + + complete(taskSets(1), Seq((Success, makeMapStatus("hostA", 1)))) + assert(taskSets(2).properties != null) + complete(taskSets(2), Seq((Success, 42))) + assert(results === Map(0 -> 42)) + assert(scheduler.activeJobs.isEmpty) + + assertDataStructuresEmpty() + } + + /** + * Makes sure that tasks for a stage used by multiple jobs are submitted with the properties of a + * later, active job if they were previously run under a job that is no longer active, even when + * there are fetch failures + */ + test("stage used by two jobs, some fetch failures, and the first job no longer active " + + "(SPARK-6880)") { + val shuffleDep1 = launchJobsThatShareStageAndCancelFirst() + val job2Id = 1 // TaskSet priority for Stages run with "job2" as the ActiveJob + + // lets say there is a fetch failure in this task set, which makes us go back and + // run stage 0, attempt 1 + complete(taskSets(1), Seq( + (FetchFailed(makeBlockManagerId("hostA"), shuffleDep1.shuffleId, 0, 0, "ignored"), null))) + scheduler.resubmitFailedStages() + + // stage 0, attempt 1 should have the properties of job2 + assert(taskSets(2).stageId === 0) + assert(taskSets(2).stageAttemptId === 1) + checkJobPropertiesAndPriority(taskSets(2), "job2", job2Id) + + // run the rest of the stages normally, checking that they have the correct properties + complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) + checkJobPropertiesAndPriority(taskSets(3), "job2", job2Id) + complete(taskSets(3), Seq((Success, makeMapStatus("hostA", 1)))) + checkJobPropertiesAndPriority(taskSets(4), "job2", job2Id) + complete(taskSets(4), Seq((Success, 42))) + assert(results === Map(0 -> 42)) + assert(scheduler.activeJobs.isEmpty) + + assertDataStructuresEmpty() + } + + /** + * In this test, we run a map stage where one of the executors fails but we still receive a + * "zombie" complete message from a task that ran on that executor. We want to make sure the + * stage is resubmitted so that the task that ran on the failed executor is re-executed, and + * that the stage is only marked as finished once that task completes. + */ + test("run trivial shuffle with out-of-band executor failure and retry") { val shuffleMapRdd = new MyRDD(sc, 2, Nil) - val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(1)) val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 1, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0)) - // blockManagerMaster.removeExecutor("exec-hostA") - // pretend we were told hostA went away - runEvent(ExecutorLost("exec-hostA")) - // DAGScheduler will immediately resubmit the stage after it appears to have no pending tasks - // rather than marking it is as failed and waiting. + // Tell the DAGScheduler that hostA was lost. + runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)))) + + // At this point, no more tasks are running for the stage (and the TaskSetManager considers the + // stage complete), but the tasks that ran on HostA need to be re-run, so the DAGScheduler + // should re-submit the stage with one task (the task that originally ran on HostA). + assert(taskSets.size === 2) + assert(taskSets(1).tasks.size === 1) + + // Make sure that the stage that was re-submitted was the ShuffleMapStage (not the reduce + // stage, which shouldn't be run until all of the tasks in the ShuffleMapStage complete on + // alive executors). + assert(taskSets(1).tasks(0).isInstanceOf[ShuffleMapTask]) + // have hostC complete the resubmitted task complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === - HashSet(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) + HashSet(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) + + // Make sure that the reduce stage was now submitted. + assert(taskSets.size === 3) + assert(taskSets(2).tasks(0).isInstanceOf[ResultTask[_, _]]) + + // Complete the reduce stage. complete(taskSets(2), Seq((Success, 42))) assert(results === Map(0 -> 42)) assertDataStructuresEmpty() @@ -1307,15 +1697,15 @@ class DAGSchedulerSuite submit(finalRdd, Array(0)) // have the first stage complete normally complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) // have the second stage complete normally complete(taskSets(1), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostC", 1)))) + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostC", 1)))) // fail the third stage because hostA went down complete(taskSets(2), Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // have DAGScheduler try again @@ -1338,15 +1728,15 @@ class DAGSchedulerSuite cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) // complete stage 0 complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostB", 2)))) // complete stage 1 complete(taskSets(1), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)))) // pretend stage 2 failed because hostA went down complete(taskSets(2), Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun. @@ -1360,13 +1750,11 @@ class DAGSchedulerSuite } test("misbehaved accumulator should not crash DAGScheduler and SparkContext") { - val acc = new Accumulator[Int](0, new AccumulatorParam[Int] { - override def addAccumulator(t1: Int, t2: Int): Int = t1 + t2 - override def zero(initialValue: Int): Int = 0 - override def addInPlace(r1: Int, r2: Int): Int = { - throw new DAGSchedulerSuiteDummyException - } - }) + val acc = new LongAccumulator { + override def add(v: java.lang.Long): Unit = throw new DAGSchedulerSuiteDummyException + override def add(v: Long): Unit = throw new DAGSchedulerSuiteDummyException + } + sc.register(acc) // Run this on executors sc.parallelize(1 to 10, 2).foreach { item => acc.add(1) } @@ -1430,31 +1818,56 @@ class DAGSchedulerSuite test("accumulator not calculated for resubmitted result stage") { // just for register - val accum = new Accumulator[Int](0, AccumulatorParam.IntAccumulatorParam) + val accum = AccumulatorSuite.createLongAccum("a") val finalRdd = new MyRDD(sc, 1, Nil) submit(finalRdd, Array(0)) completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42))) completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) - val accVal = Accumulators.originals(accum.id).get.get.value - - assert(accVal === 1) - + assert(accum.value === 1) assertDataStructuresEmpty() } + test("accumulators are updated on exception failures") { + val acc1 = AccumulatorSuite.createLongAccum("ingenieur") + val acc2 = AccumulatorSuite.createLongAccum("boulanger") + val acc3 = AccumulatorSuite.createLongAccum("agriculteur") + assert(AccumulatorContext.get(acc1.id).isDefined) + assert(AccumulatorContext.get(acc2.id).isDefined) + assert(AccumulatorContext.get(acc3.id).isDefined) + val accUpdate1 = new LongAccumulator + accUpdate1.metadata = acc1.metadata + accUpdate1.setValue(15) + val accUpdate2 = new LongAccumulator + accUpdate2.metadata = acc2.metadata + accUpdate2.setValue(13) + val accUpdate3 = new LongAccumulator + accUpdate3.metadata = acc3.metadata + accUpdate3.setValue(18) + val accumUpdates = Seq(accUpdate1, accUpdate2, accUpdate3) + val accumInfo = accumUpdates.map(AccumulatorSuite.makeInfo) + val exceptionFailure = new ExceptionFailure( + new SparkException("fondue?"), + accumInfo).copy(accums = accumUpdates) + submit(new MyRDD(sc, 1, Nil), Array(0)) + runEvent(makeCompletionEvent(taskSets.head.tasks.head, exceptionFailure, "result")) + assert(AccumulatorContext.get(acc1.id).get.value === 15L) + assert(AccumulatorContext.get(acc2.id).get.value === 13L) + assert(AccumulatorContext.get(acc3.id).get.value === 18L) + } + test("reduce tasks should be placed locally with map output") { - // Create an shuffleMapRdd with 1 partition + // Create a shuffleMapRdd with 1 partition val shuffleMapRdd = new MyRDD(sc, 1, Nil) - val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(1)) val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 1, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0)) complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)))) + (Success, makeMapStatus("hostA", 1)))) assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === - HashSet(makeBlockManagerId("hostA"))) + HashSet(makeBlockManagerId("hostA"))) // Reducer should run on the same host that map task ran val reduceTaskSet = taskSets(1) @@ -1466,7 +1879,7 @@ class DAGSchedulerSuite test("reduce task locality preferences should only include machines with largest map outputs") { val numMapTasks = 4 - // Create an shuffleMapRdd with more partitions + // Create a shuffleMapRdd with more partitions val shuffleMapRdd = new MyRDD(sc, numMapTasks, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(1)) val shuffleId = shuffleDep.shuffleId @@ -1502,7 +1915,7 @@ class DAGSchedulerSuite assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"))) - // Reducer should run where RDD 2 has preferences, even though though it also has a shuffle dep + // Reducer should run where RDD 2 has preferences, even though it also has a shuffle dep val reduceTaskSet = taskSets(1) assertLocations(reduceTaskSet, Seq(Seq("hostB"))) complete(reduceTaskSet, Seq((Success, 42))) @@ -1516,7 +1929,7 @@ class DAGSchedulerSuite } // Does not include message, ONLY stack trace. - val stackTraceString = e.getStackTraceString + val stackTraceString = Utils.exceptionString(e) // should actually include the RDD operation that invoked the method: assert(stackTraceString.contains("org.apache.spark.rdd.RDD.count")) @@ -1525,6 +1938,18 @@ class DAGSchedulerSuite assert(stackTraceString.contains("org.scalatest.FunSuite")) } + test("catch errors in event loop") { + // this is a test of our testing framework -- make sure errors in event loop don't get ignored + + // just run some bad event that will throw an exception -- we'll give a null TaskEndReason + val rdd1 = new MyRDD(sc, 1, Nil) + submit(rdd1, Array(0)) + intercept[Exception] { + complete(taskSets(0), Seq( + (null, makeMapStatus("hostA", 1)))) + } + } + test("simple map stage submission") { val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(1)) @@ -1702,6 +2127,11 @@ class DAGSchedulerSuite * In this test, we run a map stage where one of the executors fails but we still receive a * "zombie" complete message from that executor. We want to make sure the stage is not reported * as done until all tasks have completed. + * + * Most of the functionality in this test is tested in "run trivial shuffle with out-of-band + * executor failure and retry". However, that test uses ShuffleMapStages that are followed by + * a ResultStage, whereas in this test, the ShuffleMapStage is tested in isolation, without a + * ResultStage after it. */ test("map stage submission with executor failure late map task completions") { val shuffleMapRdd = new MyRDD(sc, 3, Nil) @@ -1710,34 +2140,43 @@ class DAGSchedulerSuite submitMapStage(shuffleDep) val oldTaskSet = taskSets(0) - runEvent(CompletionEvent(oldTaskSet.tasks(0), Success, makeMapStatus("hostA", 2), - null, createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent(oldTaskSet.tasks(0), Success, makeMapStatus("hostA", 2))) assert(results.size === 0) // Map stage job should not be complete yet - // Pretend host A was lost + // Pretend host A was lost. This will cause the TaskSetManager to resubmit task 0, because it + // completed on hostA. val oldEpoch = mapOutputTracker.getEpoch - runEvent(ExecutorLost("exec-hostA")) + runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) val newEpoch = mapOutputTracker.getEpoch assert(newEpoch > oldEpoch) // Suppose we also get a completed event from task 1 on the same host; this should be ignored - runEvent(CompletionEvent(oldTaskSet.tasks(1), Success, makeMapStatus("hostA", 2), - null, createFakeTaskInfo(), null)) + runEvent(makeCompletionEvent(oldTaskSet.tasks(1), Success, makeMapStatus("hostA", 2))) assert(results.size === 0) // Map stage job should not be complete yet // A completion from another task should work because it's a non-failed host - runEvent(CompletionEvent(oldTaskSet.tasks(2), Success, makeMapStatus("hostB", 2), - null, createFakeTaskInfo(), null)) - assert(results.size === 0) // Map stage job should not be complete yet + runEvent(makeCompletionEvent(oldTaskSet.tasks(2), Success, makeMapStatus("hostB", 2))) + + // At this point, no more tasks are running for the stage (and the TaskSetManager considers + // the stage complete), but the task that ran on hostA needs to be re-run, so the map stage + // shouldn't be marked as complete, and the DAGScheduler should re-submit the stage. + assert(results.size === 0) + assert(taskSets.size === 2) // Now complete tasks in the second task set val newTaskSet = taskSets(1) - assert(newTaskSet.tasks.size === 2) // Both tasks 0 and 1 were on on hostA - runEvent(CompletionEvent(newTaskSet.tasks(0), Success, makeMapStatus("hostB", 2), - null, createFakeTaskInfo(), null)) - assert(results.size === 0) // Map stage job should not be complete yet - runEvent(CompletionEvent(newTaskSet.tasks(1), Success, makeMapStatus("hostB", 2), - null, createFakeTaskInfo(), null)) + // 2 tasks should have been re-submitted, for tasks 0 and 1 (which ran on hostA). + assert(newTaskSet.tasks.size === 2) + // Complete task 0 from the original task set (i.e., not hte one that's currently active). + // This should still be counted towards the job being complete (but there's still one + // outstanding task). + runEvent(makeCompletionEvent(newTaskSet.tasks(0), Success, makeMapStatus("hostB", 2))) + assert(results.size === 0) + + // Complete the final task, from the currently active task set. There's still one + // running task, task 0 in the currently active stage attempt, but the success of task 0 means + // the DAGScheduler can mark the stage as finished. + runEvent(makeCompletionEvent(newTaskSet.tasks(1), Success, makeMapStatus("hostB", 2))) assert(results.size === 1) // Map stage job should now finally be complete assertDataStructuresEmpty() @@ -1751,6 +2190,192 @@ class DAGSchedulerSuite assertDataStructuresEmpty() } + /** + * Checks the DAGScheduler's internal logic for traversing an RDD DAG by making sure that + * getShuffleDependencies correctly returns the direct shuffle dependencies of a particular + * RDD. The test creates the following RDD graph (where n denotes a narrow dependency and s + * denotes a shuffle dependency): + * + * A <------------s---------, + * \ + * B <--s-- C <--s-- D <--n---`-- E + * + * Here, the direct shuffle dependency of C is just the shuffle dependency on B. The direct + * shuffle dependencies of E are the shuffle dependency on A and the shuffle dependency on C. + */ + test("getShuffleDependencies correctly returns only direct shuffle parents") { + val rddA = new MyRDD(sc, 2, Nil) + val shuffleDepA = new ShuffleDependency(rddA, new HashPartitioner(1)) + val rddB = new MyRDD(sc, 2, Nil) + val shuffleDepB = new ShuffleDependency(rddB, new HashPartitioner(1)) + val rddC = new MyRDD(sc, 1, List(shuffleDepB)) + val shuffleDepC = new ShuffleDependency(rddC, new HashPartitioner(1)) + val rddD = new MyRDD(sc, 1, List(shuffleDepC)) + val narrowDepD = new OneToOneDependency(rddD) + val rddE = new MyRDD(sc, 1, List(shuffleDepA, narrowDepD), tracker = mapOutputTracker) + + assert(scheduler.getShuffleDependencies(rddA) === Set()) + assert(scheduler.getShuffleDependencies(rddB) === Set()) + assert(scheduler.getShuffleDependencies(rddC) === Set(shuffleDepB)) + assert(scheduler.getShuffleDependencies(rddD) === Set(shuffleDepC)) + assert(scheduler.getShuffleDependencies(rddE) === Set(shuffleDepA, shuffleDepC)) + } + + test("SPARK-17644: After one stage is aborted for too many failed attempts, subsequent stages" + + "still behave correctly on fetch failures") { + // Runs a job that always encounters a fetch failure, so should eventually be aborted + def runJobWithPersistentFetchFailure: Unit = { + val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).map(x => (x, 1)).groupByKey() + val shuffleHandle = + rdd1.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle + rdd1.map { + case (x, _) if (x == 1) => + throw new FetchFailedException( + BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") + case (x, _) => x + }.count() + } + + // Runs a job that encounters a single fetch failure but succeeds on the second attempt + def runJobWithTemporaryFetchFailure: Unit = { + object FailThisAttempt { + val _fail = new AtomicBoolean(true) + } + val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).map(x => (x, 1)).groupByKey() + val shuffleHandle = + rdd1.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle + rdd1.map { + case (x, _) if (x == 1) && FailThisAttempt._fail.getAndSet(false) => + throw new FetchFailedException( + BlockManagerId("1", "1", 1), shuffleHandle.shuffleId, 0, 0, "test") + } + } + + failAfter(10.seconds) { + val e = intercept[SparkException] { + runJobWithPersistentFetchFailure + } + assert(e.getMessage.contains("org.apache.spark.shuffle.FetchFailedException")) + } + + // Run a second job that will fail due to a fetch failure. + // This job will hang without the fix for SPARK-17644. + failAfter(10.seconds) { + val e = intercept[SparkException] { + runJobWithPersistentFetchFailure + } + assert(e.getMessage.contains("org.apache.spark.shuffle.FetchFailedException")) + } + + failAfter(10.seconds) { + try { + runJobWithTemporaryFetchFailure + } catch { + case e: Throwable => fail("A job with one fetch failure should eventually succeed") + } + } + } + + test("[SPARK-19263] DAGScheduler should not submit multiple active tasksets," + + " even with late completions from earlier stage attempts") { + // Create 3 RDDs with shuffle dependencies on each other: rddA <--- rddB <--- rddC + val rddA = new MyRDD(sc, 2, Nil) + val shuffleDepA = new ShuffleDependency(rddA, new HashPartitioner(2)) + val shuffleIdA = shuffleDepA.shuffleId + + val rddB = new MyRDD(sc, 2, List(shuffleDepA), tracker = mapOutputTracker) + val shuffleDepB = new ShuffleDependency(rddB, new HashPartitioner(2)) + + val rddC = new MyRDD(sc, 2, List(shuffleDepB), tracker = mapOutputTracker) + + submit(rddC, Array(0, 1)) + + // Complete both tasks in rddA. + assert(taskSets(0).stageId === 0 && taskSets(0).stageAttemptId === 0) + complete(taskSets(0), Seq( + (Success, makeMapStatus("hostA", 2)), + (Success, makeMapStatus("hostA", 2)))) + + // Fetch failed for task(stageId=1, stageAttemptId=0, partitionId=0) running on hostA + // and task(stageId=1, stageAttemptId=0, partitionId=1) is still running. + assert(taskSets(1).stageId === 1 && taskSets(1).stageAttemptId === 0) + runEvent(makeCompletionEvent( + taskSets(1).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleIdA, 0, 0, + "Fetch failure of task: stageId=1, stageAttempt=0, partitionId=0"), + result = null)) + + // Both original tasks in rddA should be marked as failed, because they ran on the + // failed hostA, so both should be resubmitted. Complete them on hostB successfully. + scheduler.resubmitFailedStages() + assert(taskSets(2).stageId === 0 && taskSets(2).stageAttemptId === 1 + && taskSets(2).tasks.size === 2) + complete(taskSets(2), Seq( + (Success, makeMapStatus("hostB", 2)), + (Success, makeMapStatus("hostB", 2)))) + + // Complete task(stageId=1, stageAttemptId=0, partitionId=1) running on failed hostA + // successfully. The success should be ignored because the task started before the + // executor failed, so the output may have been lost. + runEvent(makeCompletionEvent( + taskSets(1).tasks(1), Success, makeMapStatus("hostA", 2))) + + // Both tasks in rddB should be resubmitted, because none of them has succeeded truely. + // Complete the task(stageId=1, stageAttemptId=1, partitionId=0) successfully. + // Task(stageId=1, stageAttemptId=1, partitionId=1) of this new active stage attempt + // is still running. + assert(taskSets(3).stageId === 1 && taskSets(3).stageAttemptId === 1 + && taskSets(3).tasks.size === 2) + runEvent(makeCompletionEvent( + taskSets(3).tasks(0), Success, makeMapStatus("hostB", 2))) + + // There should be no new attempt of stage submitted, + // because task(stageId=1, stageAttempt=1, partitionId=1) is still running in + // the current attempt (and hasn't completed successfully in any earlier attempts). + assert(taskSets.size === 4) + + // Complete task(stageId=1, stageAttempt=1, partitionId=1) successfully. + runEvent(makeCompletionEvent( + taskSets(3).tasks(1), Success, makeMapStatus("hostB", 2))) + + // Now the ResultStage should be submitted, because all of the tasks of rddB have + // completed successfully on alive executors. + assert(taskSets.size === 5 && taskSets(4).tasks(0).isInstanceOf[ResultTask[_, _]]) + complete(taskSets(4), Seq( + (Success, 1), + (Success, 1))) + } + + test("task end event should have updated accumulators (SPARK-20342)") { + val tasks = 10 + + val accumId = new AtomicLong() + val foundCount = new AtomicLong() + val listener = new SparkListener() { + override def onTaskEnd(event: SparkListenerTaskEnd): Unit = { + event.taskInfo.accumulables.find(_.id == accumId.get).foreach { _ => + foundCount.incrementAndGet() + } + } + } + sc.addSparkListener(listener) + + // Try a few times in a loop to make sure. This is not guaranteed to fail when the bug exists, + // but it should at least make the test flaky. If the bug is fixed, this should always pass. + (1 to 10).foreach { i => + foundCount.set(0L) + + val accum = sc.longAccumulator(s"accum$i") + accumId.set(accum.id) + + sc.parallelize(1 to tasks, tasks).foreach { _ => + accum.add(1L) + } + sc.listenerBus.waitUntilEmpty(1000) + assert(foundCount.get() === tasks) + } + } + /** * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations. * Note that this checks only the host and not the executor ID. @@ -1762,12 +2387,6 @@ class DAGSchedulerSuite } } - private def makeMapStatus(host: String, reduces: Int, sizes: Byte = 2): MapStatus = - MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes)) - - private def makeBlockManagerId(host: String): BlockManagerId = - BlockManagerId("exec-" + host, host, 12345) - private def assertDataStructuresEmpty(): Unit = { assert(scheduler.activeJobs.isEmpty) assert(scheduler.failedStages.isEmpty) @@ -1775,7 +2394,7 @@ class DAGSchedulerSuite assert(scheduler.jobIdToStageIds.isEmpty) assert(scheduler.stageIdToStage.isEmpty) assert(scheduler.runningStages.isEmpty) - assert(scheduler.shuffleToMapStage.isEmpty) + assert(scheduler.shuffleIdToMapStage.isEmpty) assert(scheduler.waitingStages.isEmpty) assert(scheduler.outputCommitCoordinator.isEmpty) } @@ -1788,5 +2407,31 @@ class DAGSchedulerSuite info } + private def createFakeTaskInfoWithId(taskId: Long): TaskInfo = { + val info = new TaskInfo(taskId, 0, 0, 0L, "", "", TaskLocality.ANY, false) + info.finishTime = 1 // to prevent spurious errors in JobProgressListener + info + } + + private def makeCompletionEvent( + task: Task[_], + reason: TaskEndReason, + result: Any, + extraAccumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty, + taskInfo: TaskInfo = createFakeTaskInfo()): CompletionEvent = { + val accumUpdates = reason match { + case Success => task.metrics.accumulators() + case ef: ExceptionFailure => ef.accums + case _ => Seq.empty + } + CompletionEvent(task, reason, result, accumUpdates ++ extraAccumUpdates, taskInfo) + } } +object DAGSchedulerSuite { + def makeMapStatus(host: String, reduces: Int, sizes: Byte = 2): MapStatus = + MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes)) + + def makeBlockManagerId(host: String): BlockManagerId = + BlockManagerId("exec-" + host, host, 12345) +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 5cb2d4225d28..6b42775ccb0f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -18,18 +18,20 @@ package org.apache.spark.scheduler import java.io.{File, FileOutputStream, InputStream, IOException} -import java.net.URI import scala.collection.mutable import scala.io.Source import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ +import org.mockito.Mockito import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.io._ +import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{JsonProtocol, Utils} /** @@ -49,7 +51,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit private var testDirPath: Path = _ before { - testDir = Utils.createTempDir() + testDir = Utils.createTempDir(namePrefix = s"history log") testDir.deleteOnExit() testDirPath = new Path(testDir.getAbsolutePath()) } @@ -67,11 +69,11 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit val logPath = new Path(eventLogger.logPath + EventLoggingListener.IN_PROGRESS) assert(fileSystem.exists(logPath)) val logStatus = fileSystem.getFileStatus(logPath) - assert(!logStatus.isDir) + assert(!logStatus.isDirectory) // Verify log is renamed after stop() eventLogger.stop() - assert(!fileSystem.getFileStatus(new Path(eventLogger.logPath)).isDir) + assert(!fileSystem.getFileStatus(new Path(eventLogger.logPath)).isDirectory) } test("Basic event logging") { @@ -94,9 +96,21 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } } + test("Event logging with password redaction") { + val key = "spark.executorEnv.HADOOP_CREDSTORE_PASSWORD" + val secretPassword = "secret_password" + val conf = getLoggingConf(testDirPath, None) + .set(key, secretPassword) + val eventLogger = new EventLoggingListener("test", None, testDirPath.toUri(), conf) + val envDetails = SparkEnv.environmentDetails(conf, "FIFO", Seq.empty, Seq.empty) + val event = SparkListenerEnvironmentUpdate(envDetails) + val redactedProps = eventLogger.redactEvent(event).environmentDetails("Spark Properties").toMap + assert(redactedProps(key) == "*********(redacted)") + } + test("Log overwriting") { val logUri = EventLoggingListener.getLogPath(testDir.toURI, "test", None) - val logPath = new URI(logUri).getPath + val logPath = new Path(logUri).toUri.getPath // Create file before writing the event log new FileOutputStream(new File(logPath)).close() // Expected IOException, since we haven't enabled log overwrite. @@ -106,19 +120,20 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } test("Event log name") { + val baseDirUri = Utils.resolveURI("/base-dir") // without compression - assert(s"file:/base-dir/app1" === EventLoggingListener.getLogPath( - Utils.resolveURI("/base-dir"), "app1", None)) + assert(s"${baseDirUri.toString}/app1" === EventLoggingListener.getLogPath( + baseDirUri, "app1", None)) // with compression - assert(s"file:/base-dir/app1.lzf" === - EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), "app1", None, Some("lzf"))) + assert(s"${baseDirUri.toString}/app1.lzf" === + EventLoggingListener.getLogPath(baseDirUri, "app1", None, Some("lzf"))) // illegal characters in app ID - assert(s"file:/base-dir/a-fine-mind_dollar_bills__1" === - EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), + assert(s"${baseDirUri.toString}/a-fine-mind_dollar_bills__1" === + EventLoggingListener.getLogPath(baseDirUri, "a fine:mind$dollar{bills}.1", None)) // illegal characters in app ID with compression - assert(s"file:/base-dir/a-fine-mind_dollar_bills__1.lz4" === - EventLoggingListener.getLogPath(Utils.resolveURI("/base-dir"), + assert(s"${baseDirUri.toString}/a-fine-mind_dollar_bills__1.lz4" === + EventLoggingListener.getLogPath(baseDirUri, "a fine:mind$dollar{bills}.1", None, Some("lz4"))) } @@ -141,17 +156,18 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit extraConf.foreach { case (k, v) => conf.set(k, v) } val logName = compressionCodec.map("test-" + _).getOrElse("test") val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf) - val listenerBus = new LiveListenerBus + val listenerBus = new LiveListenerBus(conf) val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey", None) val applicationEnd = SparkListenerApplicationEnd(1000L) // A comprehensive test on JSON de/serialization of all events is in JsonProtocolSuite eventLogger.start() - listenerBus.start(sc) - listenerBus.addListener(eventLogger) - listenerBus.postToAll(applicationStart) - listenerBus.postToAll(applicationEnd) + listenerBus.start(Mockito.mock(classOf[SparkContext]), Mockito.mock(classOf[MetricsSystem])) + listenerBus.addToEventLogQueue(eventLogger) + listenerBus.post(applicationStart) + listenerBus.post(applicationEnd) + listenerBus.stop() eventLogger.stop() // Verify file contains exactly the two events logged @@ -180,7 +196,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // into SPARK-6688. val conf = getLoggingConf(testDirPath, compressionCodec) .set("spark.hadoop.fs.defaultFS", "unsupported://example.com") - val sc = new SparkContext("local-cluster[2,2,1024]", "test", conf) + sc = new SparkContext("local-cluster[2,2,1024]", "test", conf) assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get val eventLogPath = eventLogger.logPath @@ -201,8 +217,6 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // Make sure expected events exist in the log file. val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) - val logStart = SparkListenerLogStart(SPARK_VERSION) - val lines = readLines(logData) val eventSet = mutable.Set( SparkListenerApplicationStart, SparkListenerBlockManagerAdded, @@ -215,19 +229,25 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit SparkListenerTaskStart, SparkListenerTaskEnd, SparkListenerApplicationEnd).map(Utils.getFormattedClassName) - lines.foreach { line => - eventSet.foreach { event => - if (line.contains(event)) { - val parsedEvent = JsonProtocol.sparkEventFromJson(parse(line)) - val eventType = Utils.getFormattedClassName(parsedEvent) - if (eventType == event) { - eventSet.remove(event) + Utils.tryWithSafeFinally { + val logStart = SparkListenerLogStart(SPARK_VERSION) + val lines = readLines(logData) + lines.foreach { line => + eventSet.foreach { event => + if (line.contains(event)) { + val parsedEvent = JsonProtocol.sparkEventFromJson(parse(line)) + val eventType = Utils.getFormattedClassName(parsedEvent) + if (eventType == event) { + eventSet.remove(event) + } } } } + assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) + assert(eventSet.isEmpty, "The following events are missing: " + eventSet.toSeq) + } { + logData.close() } - assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) - assert(eventSet.isEmpty, "The following events are missing: " + eventSet.toSeq) } private def readLines(in: InputStream): Seq[String] = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala new file mode 100644 index 000000000000..a4e4ea7cd289 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.AccumulatorV2 + +class ExternalClusterManagerSuite extends SparkFunSuite with LocalSparkContext { + test("launch of backend and scheduler") { + val conf = new SparkConf().setMaster("myclusterManager"). + setAppName("testcm").set("spark.driver.allowMultipleContexts", "true") + sc = new SparkContext(conf) + // check if the scheduler components are created and initialized + sc.schedulerBackend match { + case dummy: DummySchedulerBackend => assert(dummy.initialized) + case other => fail(s"wrong scheduler backend: ${other}") + } + sc.taskScheduler match { + case dummy: DummyTaskScheduler => assert(dummy.initialized) + case other => fail(s"wrong task scheduler: ${other}") + } + } +} + +/** + * Super basic ExternalClusterManager, just to verify ExternalClusterManagers can be configured. + * + * Note that if you want a special ClusterManager for tests, you are probably much more interested + * in [[MockExternalClusterManager]] and the corresponding [[SchedulerIntegrationSuite]] + */ +private class DummyExternalClusterManager extends ExternalClusterManager { + + def canCreate(masterURL: String): Boolean = masterURL == "myclusterManager" + + def createTaskScheduler(sc: SparkContext, + masterURL: String): TaskScheduler = new DummyTaskScheduler + + def createSchedulerBackend(sc: SparkContext, + masterURL: String, + scheduler: TaskScheduler): SchedulerBackend = new DummySchedulerBackend() + + def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { + scheduler.asInstanceOf[DummyTaskScheduler].initialized = true + backend.asInstanceOf[DummySchedulerBackend].initialized = true + } + +} + +private class DummySchedulerBackend extends SchedulerBackend { + var initialized = false + def start() {} + def stop() {} + def reviveOffers() {} + def defaultParallelism(): Int = 1 +} + +private class DummyTaskScheduler extends TaskScheduler { + var initialized = false + override def schedulingMode: SchedulingMode = SchedulingMode.FIFO + override def rootPool: Pool = new Pool("", schedulingMode, 0, 0) + override def start(): Unit = {} + override def stop(): Unit = {} + override def submitTasks(taskSet: TaskSet): Unit = {} + override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = {} + override def killTaskAttempt( + taskId: Long, interruptThread: Boolean, reason: String): Boolean = false + override def setDAGScheduler(dagScheduler: DAGScheduler): Unit = {} + override def defaultParallelism(): Int = 2 + override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} + override def workerRemoved(workerId: String, host: String, message: String): Unit = {} + override def applicationAttemptId(): Option[String] = None + def executorHeartbeatReceived( + execId: String, + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], + blockManagerId: BlockManagerId): Boolean = true +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala index f7e16af9d3a9..fe6de2bd9885 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala @@ -17,12 +17,20 @@ package org.apache.spark.scheduler +import java.util.Properties + +import org.apache.spark.SparkEnv import org.apache.spark.TaskContext +import org.apache.spark.executor.TaskMetrics class FakeTask( stageId: Int, - prefLocs: Seq[TaskLocation] = Nil) - extends Task[Int](stageId, 0, 0, Seq.empty) { + partitionId: Int, + prefLocs: Seq[TaskLocation] = Nil, + serializedTaskMetrics: Array[Byte] = + SparkEnv.get.closureSerializer.newInstance().serialize(TaskMetrics.registered).array()) + extends Task[Int](stageId, 0, partitionId, new Properties, serializedTaskMetrics) { + override def runTask(context: TaskContext): Int = 0 override def preferredLocations: Seq[TaskLocation] = prefLocs } @@ -33,16 +41,21 @@ object FakeTask { * locations for each task (given as varargs) if this sequence is not empty. */ def createTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { - createTaskSet(numTasks, 0, prefLocs: _*) + createTaskSet(numTasks, stageAttemptId = 0, prefLocs: _*) } def createTaskSet(numTasks: Int, stageAttemptId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { + createTaskSet(numTasks, stageId = 0, stageAttemptId, prefLocs: _*) + } + + def createTaskSet(numTasks: Int, stageId: Int, stageAttemptId: Int, prefLocs: Seq[TaskLocation]*): + TaskSet = { if (prefLocs.size != 0 && prefLocs.size != numTasks) { throw new IllegalArgumentException("Wrong number of task locations") } val tasks = Array.tabulate[Task[_]](numTasks) { i => - new FakeTask(i, if (prefLocs.size != 0) prefLocs(i) else Nil) + new FakeTask(stageId, i, if (prefLocs.size != 0) prefLocs(i) else Nil) } - new TaskSet(tasks, 0, stageAttemptId, 0, null) + new TaskSet(tasks, stageId, stageAttemptId, priority = 0, null) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobWaiterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobWaiterSuite.scala new file mode 100644 index 000000000000..bc8e513fe5bc --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/JobWaiterSuite.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import scala.util.Failure + +import org.apache.spark.SparkFunSuite + +class JobWaiterSuite extends SparkFunSuite { + + test("call jobFailed multiple times") { + val waiter = new JobWaiter[Int](null, 0, totalTasks = 2, null) + + // Should not throw exception if calling jobFailed multiple times + waiter.jobFailed(new RuntimeException("Oops 1")) + waiter.jobFailed(new RuntimeException("Oops 2")) + waiter.jobFailed(new RuntimeException("Oops 3")) + + waiter.completionFuture.value match { + case Some(Failure(e)) => + // We should receive the first exception + assert("Oops 1" === e.getMessage) + case other => fail("Should receiver the first exception but it was " + other) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index b8e466fab450..144e5afdcdd7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -17,13 +17,19 @@ package org.apache.spark.scheduler -import org.apache.spark.storage.BlockManagerId - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.serializer.JavaSerializer +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream} import scala.util.Random +import org.mockito.Mockito._ +import org.roaringbitmap.RoaringBitmap + +import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkFunSuite} +import org.apache.spark.LocalSparkContext._ +import org.apache.spark.internal.config +import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.storage.BlockManagerId + class MapStatusSuite extends SparkFunSuite { test("compressSize") { @@ -78,7 +84,7 @@ class MapStatusSuite extends SparkFunSuite { test("HighlyCompressedMapStatus: estimated size should be the average non-empty block size") { val sizes = Array.tabulate[Long](3000) { i => i.toLong } - val avg = sizes.sum / sizes.filter(_ != 0).length + val avg = sizes.sum / sizes.count(_ != 0) val loc = BlockManagerId("a", "b", 10) val status = MapStatus(loc, sizes) val status1 = compressAndDecompressMapStatus(status) @@ -97,4 +103,67 @@ class MapStatusSuite extends SparkFunSuite { val buf = ser.newInstance().serialize(status) ser.newInstance().deserialize[MapStatus](buf) } + + test("RoaringBitmap: runOptimize succeeded") { + val r = new RoaringBitmap + (1 to 200000).foreach(i => + if (i % 200 != 0) { + r.add(i) + } + ) + val size1 = r.getSizeInBytes + val success = r.runOptimize() + r.trim() + val size2 = r.getSizeInBytes + assert(size1 > size2) + assert(success) + } + + test("RoaringBitmap: runOptimize failed") { + val r = new RoaringBitmap + (1 to 200000).foreach(i => + if (i % 200 == 0) { + r.add(i) + } + ) + val size1 = r.getSizeInBytes + val success = r.runOptimize() + r.trim() + val size2 = r.getSizeInBytes + assert(size1 === size2) + assert(!success) + } + + test("Blocks which are bigger than SHUFFLE_ACCURATE_BLOCK_THRESHOLD should not be " + + "underestimated.") { + val conf = new SparkConf().set(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD.key, "1000") + val env = mock(classOf[SparkEnv]) + doReturn(conf).when(env).conf + SparkEnv.set(env) + // Value of element in sizes is equal to the corresponding index. + val sizes = (0L to 2000L).toArray + val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes) + val arrayStream = new ByteArrayOutputStream(102400) + val objectOutputStream = new ObjectOutputStream(arrayStream) + assert(status1.isInstanceOf[HighlyCompressedMapStatus]) + objectOutputStream.writeObject(status1) + objectOutputStream.flush() + val array = arrayStream.toByteArray + val objectInput = new ObjectInputStream(new ByteArrayInputStream(array)) + val status2 = objectInput.readObject().asInstanceOf[HighlyCompressedMapStatus] + (1001 to 2000).foreach { + case part => assert(status2.getSizeForBlock(part) >= sizes(part)) + } + } + + test("SPARK-21133 HighlyCompressedMapStatus#writeExternal throws NPE") { + val conf = new SparkConf() + .set("spark.serializer", classOf[KryoSerializer].getName) + .setMaster("local") + .setAppName("SPARK-21133") + withSpark(new SparkContext(conf)) { sc => + val count = sc.parallelize(0 until 3000, 10).repartition(2001).collect().length + assert(count === 3000) + } + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala index f33324792495..255be6f46b06 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import java.io.{ObjectInputStream, ObjectOutputStream, IOException} +import java.io.{IOException, ObjectInputStream, ObjectOutputStream} import org.apache.spark.TaskContext @@ -25,7 +25,7 @@ import org.apache.spark.TaskContext * A Task implementation that fails to serialize. */ private[spark] class NotSerializableFakeTask(myId: Int, stageId: Int) - extends Task[Array[Byte]](stageId, 0, 0, Seq.empty) { + extends Task[Array[Byte]](stageId, 0, 0) { override def runTask(context: TaskContext): Array[Byte] = Array.empty[Byte] override def preferredLocations: Seq[TaskLocation] = Seq[TaskLocation]() diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala index 1ae5b030f083..a27dadcf49bf 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala @@ -18,10 +18,10 @@ package org.apache.spark.scheduler import org.apache.hadoop.mapred.{FileOutputCommitter, TaskAttemptContext} -import org.scalatest.concurrent.Timeouts -import org.scalatest.time.{Span, Seconds} +import org.scalatest.concurrent.TimeLimits +import org.scalatest.time.{Seconds, Span} -import org.apache.spark.{SparkConf, SparkContext, LocalSparkContext, SparkFunSuite, TaskContext} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite, TaskContext} import org.apache.spark.util.Utils /** @@ -32,13 +32,12 @@ import org.apache.spark.util.Utils class OutputCommitCoordinatorIntegrationSuite extends SparkFunSuite with LocalSparkContext - with Timeouts { + with TimeLimits { override def beforeAll(): Unit = { super.beforeAll() val conf = new SparkConf() - .set("master", "local[2,4]") - .set("spark.speculation", "true") + .set("spark.hadoop.outputCommitCoordination.enabled", "true") .set("spark.hadoop.mapred.output.committer.class", classOf[ThrowExceptionOnFirstAttemptOutputCommitter].getCanonicalName) sc = new SparkContext("local[2, 4]", "test", conf) diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index 7345508bfe99..03b190390249 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -18,23 +18,24 @@ package org.apache.spark.scheduler import java.io.File +import java.util.Date import java.util.concurrent.TimeoutException +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.apache.hadoop.mapred._ +import org.apache.hadoop.mapreduce.TaskType import org.mockito.Matchers import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter -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 +import org.apache.spark.internal.io.{FileCommitProtocol, HadoopMapRedCommitProtocol, SparkHadoopWriterUtils} +import org.apache.spark.rdd.{FakeOutputCommitter, RDD} +import org.apache.spark.util.{ThreadUtils, Utils} /** * Unit tests for the output commit coordination functionality. @@ -78,7 +79,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { val conf = new SparkConf() .setMaster("local[4]") .setAppName(classOf[OutputCommitCoordinatorSuite].getSimpleName) - .set("spark.speculation", "true") + .set("spark.hadoop.outputCommitCoordination.enabled", "true") sc = new SparkContext(conf) { override private[spark] def createSparkEnv( conf: SparkConf, @@ -114,7 +115,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { locality: TaskLocality.Value): Option[(Int, TaskLocality.Value)] = { if (!hasDequeuedSpeculatedTask) { hasDequeuedSpeculatedTask = true - Some(0, TaskLocality.PROCESS_LOCAL) + Some((0, TaskLocality.PROCESS_LOCAL)) } else { None } @@ -161,7 +162,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { // 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) + ThreadUtils.awaitResult(futureAction, 5 seconds) } assert(tempDir.list().size === 0) } @@ -177,13 +178,13 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { assert(!outputCommitCoordinator.canCommit(stage, partition, nonAuthorizedCommitter)) // The non-authorized committer fails outputCommitCoordinator.taskCompleted( - stage, partition, attemptNumber = nonAuthorizedCommitter, reason = TaskKilled) + stage, partition, attemptNumber = nonAuthorizedCommitter, reason = TaskKilled("test")) // New tasks should still not be able to commit because the authorized committer has not failed assert( !outputCommitCoordinator.canCommit(stage, partition, nonAuthorizedCommitter + 1)) // The authorized committer now fails, clearing the lock outputCommitCoordinator.taskCompleted( - stage, partition, attemptNumber = authorizedCommitter, reason = TaskKilled) + stage, partition, attemptNumber = authorizedCommitter, reason = TaskKilled("test")) // A new task should now be allowed to become the authorized committer assert( outputCommitCoordinator.canCommit(stage, partition, nonAuthorizedCommitter + 2)) @@ -191,6 +192,23 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { assert( !outputCommitCoordinator.canCommit(stage, partition, nonAuthorizedCommitter + 3)) } + + test("Duplicate calls to canCommit from the authorized committer gets idempotent responses.") { + val rdd = sc.parallelize(Seq(1), 1) + sc.runJob(rdd, OutputCommitFunctions(tempDir.getAbsolutePath).callCanCommitMultipleTimes _, + 0 until rdd.partitions.size) + } + + test("SPARK-19631: Do not allow failed attempts to be authorized for committing") { + val stage: Int = 1 + val partition: Int = 1 + val failedAttempt: Int = 0 + outputCommitCoordinator.stageStart(stage, maxPartitionId = 1) + outputCommitCoordinator.taskCompleted(stage, partition, attemptNumber = failedAttempt, + reason = ExecutorLostFailure("0", exitCausedByApp = true, None)) + assert(!outputCommitCoordinator.canCommit(stage, partition, failedAttempt)) + assert(outputCommitCoordinator.canCommit(stage, partition, failedAttempt + 1)) + } } /** @@ -198,6 +216,8 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { */ private case class OutputCommitFunctions(tempDirPath: String) { + private val jobId = new SerializableWritable(SparkHadoopWriterUtils.createJobID(new Date, 0)) + // Mock output committer that simulates a successful commit (after commit is authorized) private def successfulOutputCommitter = new FakeOutputCommitter { override def commitTask(context: TaskAttemptContext): Unit = { @@ -223,6 +243,16 @@ private case class OutputCommitFunctions(tempDirPath: String) { if (ctx.attemptNumber == 0) failingOutputCommitter else successfulOutputCommitter) } + // Receiver should be idempotent for AskPermissionToCommitOutput + def callCanCommitMultipleTimes(iter: Iterator[Int]): Unit = { + val ctx = TaskContext.get() + val canCommit1 = SparkEnv.get.outputCommitCoordinator + .canCommit(ctx.stageId(), ctx.partitionId(), ctx.attemptNumber()) + val canCommit2 = SparkEnv.get.outputCommitCoordinator + .canCommit(ctx.stageId(), ctx.partitionId(), ctx.attemptNumber()) + assert(canCommit1 && canCommit2) + } + private def runCommitWithProvidedCommitter( ctx: TaskContext, iter: Iterator[Int], @@ -230,14 +260,22 @@ private case class OutputCommitFunctions(tempDirPath: String) { 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() + + // Instantiate committer. + val committer = FileCommitProtocol.instantiate( + className = classOf[HadoopMapRedCommitProtocol].getName, + jobId = jobId.value.getId.toString, + outputPath = jobConf.get("mapred.output.dir")) + + // Create TaskAttemptContext. + // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it + // around by taking a mod. We expect that no task will be attempted 2 billion times. + val taskAttemptId = (ctx.taskAttemptId % Int.MaxValue).toInt + val attemptId = new TaskAttemptID( + new TaskID(jobId.value, TaskType.MAP, ctx.partitionId), taskAttemptId) + val taskContext = new TaskAttemptContextImpl(jobConf, attemptId) + + committer.setupTask(taskContext) + committer.commitTask(taskContext) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala index 467796d7c24b..5bd3955f5adb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala @@ -17,9 +17,11 @@ package org.apache.spark.scheduler +import java.io.FileNotFoundException import java.util.Properties import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.scheduler.SchedulingMode._ /** * Tests that pools and the associated scheduling algorithms for FIFO and fair scheduling work @@ -27,15 +29,20 @@ import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSui */ class PoolSuite extends SparkFunSuite with LocalSparkContext { + val LOCAL = "local" + val APP_NAME = "PoolSuite" + val SCHEDULER_ALLOCATION_FILE_PROPERTY = "spark.scheduler.allocation.file" + val TEST_POOL = "testPool" + def createTaskSetManager(stageId: Int, numTasks: Int, taskScheduler: TaskSchedulerImpl) : TaskSetManager = { val tasks = Array.tabulate[Task[_]](numTasks) { i => - new FakeTask(i, Nil) + new FakeTask(stageId, i, Nil) } new TaskSetManager(taskScheduler, new TaskSet(tasks, stageId, 0, 0, null), 0) } - def scheduleTaskAndVerifyId(taskId: Int, rootPool: Pool, expectedStageId: Int) { + def scheduleTaskAndVerifyId(taskId: Int, rootPool: Pool, expectedStageId: Int): Unit = { val taskSetQueue = rootPool.getSortedTaskSetQueue val nextTaskSetToSchedule = taskSetQueue.find(t => (t.runningTasks + t.tasksSuccessful) < t.numTasks) @@ -45,12 +52,11 @@ class PoolSuite extends SparkFunSuite with LocalSparkContext { } test("FIFO Scheduler Test") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") + sc = new SparkContext(LOCAL, APP_NAME) val taskScheduler = new TaskSchedulerImpl(sc) - val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0) + val rootPool = new Pool("", FIFO, 0, 0) val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) - schedulableBuilder.buildPools() val taskSetManager0 = createTaskSetManager(0, 2, taskScheduler) val taskSetManager1 = createTaskSetManager(1, 2, taskScheduler) @@ -74,30 +80,24 @@ class PoolSuite extends SparkFunSuite with LocalSparkContext { */ test("Fair Scheduler Test") { val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - val conf = new SparkConf().set("spark.scheduler.allocation.file", xmlPath) - sc = new SparkContext("local", "TaskSchedulerImplSuite", conf) + val conf = new SparkConf().set(SCHEDULER_ALLOCATION_FILE_PROPERTY, xmlPath) + sc = new SparkContext(LOCAL, APP_NAME, conf) val taskScheduler = new TaskSchedulerImpl(sc) - val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val rootPool = new Pool("", FAIR, 0, 0) val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) schedulableBuilder.buildPools() // Ensure that the XML file was read in correctly. - assert(rootPool.getSchedulableByName("default") != null) - assert(rootPool.getSchedulableByName("1") != null) - assert(rootPool.getSchedulableByName("2") != null) - assert(rootPool.getSchedulableByName("3") != null) - assert(rootPool.getSchedulableByName("1").minShare === 2) - assert(rootPool.getSchedulableByName("1").weight === 1) - assert(rootPool.getSchedulableByName("2").minShare === 3) - assert(rootPool.getSchedulableByName("2").weight === 1) - assert(rootPool.getSchedulableByName("3").minShare === 0) - assert(rootPool.getSchedulableByName("3").weight === 1) + verifyPool(rootPool, schedulableBuilder.DEFAULT_POOL_NAME, 0, 1, FIFO) + verifyPool(rootPool, "1", 2, 1, FIFO) + verifyPool(rootPool, "2", 3, 1, FIFO) + verifyPool(rootPool, "3", 0, 1, FIFO) val properties1 = new Properties() - properties1.setProperty("spark.scheduler.pool", "1") + properties1.setProperty(schedulableBuilder.FAIR_SCHEDULER_PROPERTIES, "1") val properties2 = new Properties() - properties2.setProperty("spark.scheduler.pool", "2") + properties2.setProperty(schedulableBuilder.FAIR_SCHEDULER_PROPERTIES, "2") val taskSetManager10 = createTaskSetManager(0, 1, taskScheduler) val taskSetManager11 = createTaskSetManager(1, 1, taskScheduler) @@ -134,22 +134,22 @@ class PoolSuite extends SparkFunSuite with LocalSparkContext { } test("Nested Pool Test") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") + sc = new SparkContext(LOCAL, APP_NAME) val taskScheduler = new TaskSchedulerImpl(sc) - val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) - val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1) - val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1) + val rootPool = new Pool("", FAIR, 0, 0) + val pool0 = new Pool("0", FAIR, 3, 1) + val pool1 = new Pool("1", FAIR, 4, 1) rootPool.addSchedulable(pool0) rootPool.addSchedulable(pool1) - val pool00 = new Pool("00", SchedulingMode.FAIR, 2, 2) - val pool01 = new Pool("01", SchedulingMode.FAIR, 1, 1) + val pool00 = new Pool("00", FAIR, 2, 2) + val pool01 = new Pool("01", FAIR, 1, 1) pool0.addSchedulable(pool00) pool0.addSchedulable(pool01) - val pool10 = new Pool("10", SchedulingMode.FAIR, 2, 2) - val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1) + val pool10 = new Pool("10", FAIR, 2, 2) + val pool11 = new Pool("11", FAIR, 2, 1) pool1.addSchedulable(pool10) pool1.addSchedulable(pool11) @@ -178,4 +178,170 @@ class PoolSuite extends SparkFunSuite with LocalSparkContext { scheduleTaskAndVerifyId(2, rootPool, 6) scheduleTaskAndVerifyId(3, rootPool, 2) } + + test("SPARK-17663: FairSchedulableBuilder sets default values for blank or invalid datas") { + val xmlPath = getClass.getClassLoader.getResource("fairscheduler-with-invalid-data.xml") + .getFile() + val conf = new SparkConf().set(SCHEDULER_ALLOCATION_FILE_PROPERTY, xmlPath) + + val rootPool = new Pool("", FAIR, 0, 0) + val schedulableBuilder = new FairSchedulableBuilder(rootPool, conf) + schedulableBuilder.buildPools() + + verifyPool(rootPool, schedulableBuilder.DEFAULT_POOL_NAME, 0, 1, FIFO) + verifyPool(rootPool, "pool_with_invalid_min_share", 0, 2, FAIR) + verifyPool(rootPool, "pool_with_invalid_weight", 1, 1, FAIR) + verifyPool(rootPool, "pool_with_invalid_scheduling_mode", 3, 2, FIFO) + verifyPool(rootPool, "pool_with_non_uppercase_scheduling_mode", 2, 1, FAIR) + verifyPool(rootPool, "pool_with_NONE_scheduling_mode", 1, 2, FIFO) + verifyPool(rootPool, "pool_with_whitespace_min_share", 0, 2, FAIR) + verifyPool(rootPool, "pool_with_whitespace_weight", 1, 1, FAIR) + verifyPool(rootPool, "pool_with_whitespace_scheduling_mode", 3, 2, FIFO) + verifyPool(rootPool, "pool_with_empty_min_share", 0, 3, FAIR) + verifyPool(rootPool, "pool_with_empty_weight", 2, 1, FAIR) + verifyPool(rootPool, "pool_with_empty_scheduling_mode", 2, 2, FIFO) + verifyPool(rootPool, "pool_with_surrounded_whitespace", 3, 2, FAIR) + } + + /** + * spark.scheduler.pool property should be ignored for the FIFO scheduler, + * because pools are only needed for fair scheduling. + */ + test("FIFO scheduler uses root pool and not spark.scheduler.pool property") { + sc = new SparkContext("local", "PoolSuite") + val taskScheduler = new TaskSchedulerImpl(sc) + + val rootPool = new Pool("", SchedulingMode.FIFO, initMinShare = 0, initWeight = 0) + val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) + + val taskSetManager0 = createTaskSetManager(stageId = 0, numTasks = 1, taskScheduler) + val taskSetManager1 = createTaskSetManager(stageId = 1, numTasks = 1, taskScheduler) + + val properties = new Properties() + properties.setProperty("spark.scheduler.pool", TEST_POOL) + + // When FIFO Scheduler is used and task sets are submitted, they should be added to + // the root pool, and no additional pools should be created + // (even though there's a configured default pool). + schedulableBuilder.addTaskSetManager(taskSetManager0, properties) + schedulableBuilder.addTaskSetManager(taskSetManager1, properties) + + assert(rootPool.getSchedulableByName(TEST_POOL) === null) + assert(rootPool.schedulableQueue.size === 2) + assert(rootPool.getSchedulableByName(taskSetManager0.name) === taskSetManager0) + assert(rootPool.getSchedulableByName(taskSetManager1.name) === taskSetManager1) + } + + test("FAIR Scheduler uses default pool when spark.scheduler.pool property is not set") { + sc = new SparkContext("local", "PoolSuite") + val taskScheduler = new TaskSchedulerImpl(sc) + + val rootPool = new Pool("", SchedulingMode.FAIR, initMinShare = 0, initWeight = 0) + val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) + schedulableBuilder.buildPools() + + // Submit a new task set manager with pool properties set to null. This should result + // in the task set manager getting added to the default pool. + val taskSetManager0 = createTaskSetManager(stageId = 0, numTasks = 1, taskScheduler) + schedulableBuilder.addTaskSetManager(taskSetManager0, null) + + val defaultPool = rootPool.getSchedulableByName(schedulableBuilder.DEFAULT_POOL_NAME) + assert(defaultPool !== null) + assert(defaultPool.schedulableQueue.size === 1) + assert(defaultPool.getSchedulableByName(taskSetManager0.name) === taskSetManager0) + + // When a task set manager is submitted with spark.scheduler.pool unset, it should be added to + // the default pool (as above). + val taskSetManager1 = createTaskSetManager(stageId = 1, numTasks = 1, taskScheduler) + schedulableBuilder.addTaskSetManager(taskSetManager1, new Properties()) + + assert(defaultPool.schedulableQueue.size === 2) + assert(defaultPool.getSchedulableByName(taskSetManager1.name) === taskSetManager1) + } + + test("FAIR Scheduler creates a new pool when spark.scheduler.pool property points to " + + "a non-existent pool") { + sc = new SparkContext("local", "PoolSuite") + val taskScheduler = new TaskSchedulerImpl(sc) + + val rootPool = new Pool("", SchedulingMode.FAIR, initMinShare = 0, initWeight = 0) + val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) + schedulableBuilder.buildPools() + + assert(rootPool.getSchedulableByName(TEST_POOL) === null) + + val taskSetManager = createTaskSetManager(stageId = 0, numTasks = 1, taskScheduler) + + val properties = new Properties() + properties.setProperty(schedulableBuilder.FAIR_SCHEDULER_PROPERTIES, TEST_POOL) + + // The fair scheduler should create a new pool with default values when spark.scheduler.pool + // points to a pool that doesn't exist yet (this can happen when the file that pools are read + // from isn't set, or when that file doesn't contain the pool name specified + // by spark.scheduler.pool). + schedulableBuilder.addTaskSetManager(taskSetManager, properties) + + verifyPool(rootPool, TEST_POOL, schedulableBuilder.DEFAULT_MINIMUM_SHARE, + schedulableBuilder.DEFAULT_WEIGHT, schedulableBuilder.DEFAULT_SCHEDULING_MODE) + val testPool = rootPool.getSchedulableByName(TEST_POOL) + assert(testPool.getSchedulableByName(taskSetManager.name) === taskSetManager) + } + + test("Pool should throw IllegalArgumentException when schedulingMode is not supported") { + intercept[IllegalArgumentException] { + new Pool("TestPool", SchedulingMode.NONE, 0, 1) + } + } + + test("Fair Scheduler should build fair scheduler when " + + "valid spark.scheduler.allocation.file property is set") { + val xmlPath = getClass.getClassLoader.getResource("fairscheduler-with-valid-data.xml").getFile() + val conf = new SparkConf().set(SCHEDULER_ALLOCATION_FILE_PROPERTY, xmlPath) + sc = new SparkContext(LOCAL, APP_NAME, conf) + + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) + schedulableBuilder.buildPools() + + verifyPool(rootPool, schedulableBuilder.DEFAULT_POOL_NAME, 0, 1, FIFO) + verifyPool(rootPool, "pool1", 3, 1, FIFO) + verifyPool(rootPool, "pool2", 4, 2, FAIR) + verifyPool(rootPool, "pool3", 2, 3, FAIR) + } + + test("Fair Scheduler should use default file(fairscheduler.xml) if it exists in classpath " + + "and spark.scheduler.allocation.file property is not set") { + val conf = new SparkConf() + sc = new SparkContext(LOCAL, APP_NAME, conf) + + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) + schedulableBuilder.buildPools() + + verifyPool(rootPool, schedulableBuilder.DEFAULT_POOL_NAME, 0, 1, FIFO) + verifyPool(rootPool, "1", 2, 1, FIFO) + verifyPool(rootPool, "2", 3, 1, FIFO) + verifyPool(rootPool, "3", 0, 1, FIFO) + } + + test("Fair Scheduler should throw FileNotFoundException " + + "when invalid spark.scheduler.allocation.file property is set") { + val conf = new SparkConf().set(SCHEDULER_ALLOCATION_FILE_PROPERTY, "INVALID_FILE_PATH") + sc = new SparkContext(LOCAL, APP_NAME, conf) + + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) + intercept[FileNotFoundException] { + schedulableBuilder.buildPools() + } + } + + private def verifyPool(rootPool: Pool, poolName: String, expectedInitMinShare: Int, + expectedInitWeight: Int, expectedSchedulingMode: SchedulingMode): Unit = { + val selectedPool = rootPool.getSchedulableByName(poolName) + assert(selectedPool !== null) + assert(selectedPool.minShare === expectedInitMinShare) + assert(selectedPool.weight === expectedInitWeight) + assert(selectedPool.schedulingMode === expectedSchedulingMode) + } } 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 103fc19369c9..d17e3864854a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -17,22 +17,23 @@ package org.apache.spark.scheduler -import java.io.{File, PrintWriter} +import java.io._ import java.net.URI +import java.util.concurrent.atomic.AtomicInteger +import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkConf, SparkContext, SPARK_VERSION} -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{JsonProtocol, Utils} +import org.apache.spark.io.{CompressionCodec, LZ4CompressionCodec} +import org.apache.spark.util.{JsonProtocol, JsonProtocolSuite, Utils} /** * Test whether ReplayListenerBus replays events from logs correctly. */ -class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter { +class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkContext { private val fileSystem = Utils.getHadoopFileSystem("/", SparkHadoopUtil.get.newConfiguration(new SparkConf())) private var testDir: File = _ @@ -73,6 +74,60 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(eventMonster.loggedEvents(1) === JsonProtocol.sparkEventToJson(applicationEnd)) } + /** + * Test replaying compressed spark history file that internally throws an EOFException. To + * avoid sensitivity to the compression specifics the test forces an EOFException to occur + * while reading bytes from the underlying stream (such as observed in actual history files + * in some cases) and forces specific failure handling. This validates correctness in both + * cases when maybeTruncated is true or false. + */ + test("Replay compressed inprogress log file succeeding on partial read") { + val buffered = new ByteArrayOutputStream + val codec = new LZ4CompressionCodec(new SparkConf()) + val compstream = codec.compressedOutputStream(buffered) + Utils.tryWithResource(new PrintWriter(compstream)) { writer => + + val applicationStart = SparkListenerApplicationStart("AppStarts", None, + 125L, "Mickey", None) + val applicationEnd = SparkListenerApplicationEnd(1000L) + + // scalastyle:off println + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) + // scalastyle:on println + } + + val logFilePath = Utils.getFilePath(testDir, "events.lz4.inprogress") + val bytes = buffered.toByteArray + Utils.tryWithResource(fileSystem.create(logFilePath)) { fstream => + fstream.write(bytes, 0, buffered.size) + } + + // Read the compressed .inprogress file and verify only first event was parsed. + val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) + val replayer = new ReplayListenerBus() + + val eventMonster = new EventMonster(conf) + replayer.addListener(eventMonster) + + // Verify the replay returns the events given the input maybe truncated. + val logData = EventLoggingListener.openEventLog(logFilePath, fileSystem) + Utils.tryWithResource(new EarlyEOFInputStream(logData, buffered.size - 10)) { failingStream => + replayer.replay(failingStream, logFilePath.toString, true) + + assert(eventMonster.loggedEvents.size === 1) + assert(failingStream.didFail) + } + + // Verify the replay throws the EOF exception since the input may not be truncated. + val logData2 = EventLoggingListener.openEventLog(logFilePath, fileSystem) + Utils.tryWithResource(new EarlyEOFInputStream(logData2, buffered.size - 10)) { failingStream2 => + intercept[EOFException] { + replayer.replay(failingStream2, logFilePath.toString, false) + } + } + } + // This assumes the correctness of EventLoggingListener test("End-to-end replay") { testApplicationReplay() @@ -98,11 +153,14 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter { * assumption that the event logging behavior is correct (tested in a separate suite). */ private def testApplicationReplay(codecName: Option[String] = None) { - val logDirPath = Utils.getFilePath(testDir, "test-replay") + val logDir = new File(testDir.getAbsolutePath, "test-replay") + // Here, it creates `Path` from the URI instead of the absolute path for the explicit file + // scheme so that the string representation of this `Path` has leading file scheme correctly. + val logDirPath = new Path(logDir.toURI) fileSystem.mkdirs(logDirPath) val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName) - val sc = new SparkContext("local-cluster[2,1,1024]", "Test replay", conf) + sc = new SparkContext("local-cluster[2,1,1024]", "Test replay", conf) // Run a few jobs sc.parallelize(1 to 100, 1).count() @@ -115,7 +173,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter { val applications = fileSystem.listStatus(logDirPath) assert(applications != null && applications.size > 0) val eventLog = applications.sortBy(_.getModificationTime).last - assert(!eventLog.isDir) + assert(!eventLog.isDirectory) // Replay events val logData = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem) @@ -132,7 +190,11 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(sc.eventLogger.isDefined) val originalEvents = sc.eventLogger.get.loggedEvents val replayedEvents = eventMonster.loggedEvents - originalEvents.zip(replayedEvents).foreach { case (e1, e2) => assert(e1 === e2) } + originalEvents.zip(replayedEvents).foreach { case (e1, e2) => + // Don't compare the JSON here because accumulators in StageInfo may be out of order + JsonProtocolSuite.assertEquals( + JsonProtocol.sparkEventFromJson(e1), JsonProtocol.sparkEventFromJson(e2)) + } } /** @@ -153,4 +215,25 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter { override def start() { } } + + /* + * This is a dummy input stream that wraps another input stream but ends prematurely when + * reading at the specified position, throwing an EOFExeption. + */ + private class EarlyEOFInputStream(in: InputStream, failAtPos: Int) extends InputStream { + private val countDown = new AtomicInteger(failAtPos) + + def didFail: Boolean = countDown.get == 0 + + @throws[IOException] + override def read(): Int = { + if (countDown.get == 0) { + throw new EOFException("Stream ended prematurely") + } + countDown.decrementAndGet() + in.read() + } + + override def close(): Unit = in.close() + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala new file mode 100644 index 000000000000..75ea409e16b4 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -0,0 +1,655 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.Properties +import java.util.concurrent.{TimeoutException, TimeUnit} +import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} + +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.concurrent.Future +import scala.concurrent.duration.{Duration, SECONDS} +import scala.language.existentials +import scala.reflect.ClassTag + +import org.scalactic.TripleEquals +import org.scalatest.Assertions.AssertionsHelper +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +import org.apache.spark._ +import org.apache.spark.TaskState._ +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.util.{CallSite, ThreadUtils, Utils} + +/** + * Tests for the entire scheduler code -- DAGScheduler, TaskSchedulerImpl, TaskSets, + * TaskSetManagers. + * + * Test cases are configured by providing a set of jobs to submit, and then simulating interaction + * with spark's executors via a mocked backend (eg., task completion, task failure, executors + * disconnecting, etc.). + */ +abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends SparkFunSuite + with LocalSparkContext { + + var taskScheduler: TestTaskScheduler = null + var scheduler: DAGScheduler = null + var backend: T = _ + + override def beforeEach(): Unit = { + if (taskScheduler != null) { + taskScheduler.runningTaskSets.clear() + } + results.clear() + failure = null + backendException.set(null) + super.beforeEach() + } + + override def afterEach(): Unit = { + super.afterEach() + taskScheduler.stop() + backend.stop() + scheduler.stop() + } + + def setupScheduler(conf: SparkConf): Unit = { + conf.setAppName(this.getClass().getSimpleName()) + val backendClassName = implicitly[ClassTag[T]].runtimeClass.getName() + conf.setMaster(s"mock[${backendClassName}]") + sc = new SparkContext(conf) + backend = sc.schedulerBackend.asInstanceOf[T] + taskScheduler = sc.taskScheduler.asInstanceOf[TestTaskScheduler] + taskScheduler.initialize(sc.schedulerBackend) + scheduler = new DAGScheduler(sc, taskScheduler) + taskScheduler.setDAGScheduler(scheduler) + } + + def testScheduler(name: String)(testBody: => Unit): Unit = { + testScheduler(name, Seq())(testBody) + } + + def testScheduler(name: String, extraConfs: Seq[(String, String)])(testBody: => Unit): Unit = { + test(name) { + val conf = new SparkConf() + extraConfs.foreach{ case (k, v) => conf.set(k, v)} + setupScheduler(conf) + testBody + } + } + + /** + * A map from partition to results for all tasks of a job when you call this test framework's + * [[submit]] method. Two important considerations: + * + * 1. If there is a job failure, results may or may not be empty. If any tasks succeed before + * the job has failed, they will get included in `results`. Instead, check for job failure by + * checking [[failure]]. (Also see `assertDataStructuresEmpty()`) + * + * 2. This only gets cleared between tests. So you'll need to do special handling if you submit + * more than one job in one test. + */ + val results = new HashMap[Int, Any]() + + /** + * If a call to [[submit]] results in a job failure, this will hold the exception, else it will + * be null. + * + * As with [[results]], this only gets cleared between tests, so care must be taken if you are + * submitting more than one job in one test. + */ + var failure: Throwable = _ + + /** + * When we submit dummy Jobs, this is the compute function we supply. + */ + private val jobComputeFunc: (TaskContext, scala.Iterator[_]) => Any = { + (context: TaskContext, it: Iterator[(_)]) => + throw new RuntimeException("jobComputeFunc shouldn't get called in this mock") + } + + /** Submits a job to the scheduler, and returns a future which does a bit of error handling. */ + protected def submit( + rdd: RDD[_], + partitions: Array[Int], + func: (TaskContext, Iterator[_]) => _ = jobComputeFunc): Future[Any] = { + val waiter: JobWaiter[Any] = scheduler.submitJob(rdd, func, partitions.toSeq, CallSite("", ""), + (index, res) => results(index) = res, new Properties()) + import scala.concurrent.ExecutionContext.Implicits.global + waiter.completionFuture.recover { case ex => + failure = ex + } + } + + /** + * Helper to run a few common asserts after a job has completed, in particular some internal + * datastructures for bookkeeping. This only does a very minimal check for whether the job + * failed or succeeded -- often you will want extra asserts on [[results]] or [[failure]]. + */ + protected def assertDataStructuresEmpty(noFailure: Boolean = true): Unit = { + if (noFailure) { + if (failure != null) { + // if there is a job failure, it can be a bit hard to tease the job failure msg apart + // from the test failure msg, so we do a little extra formatting + val msg = + raw""" + | There was a failed job. + | ----- Begin Job Failure Msg ----- + | ${Utils.exceptionString(failure)} + | ----- End Job Failure Msg ---- + """. + stripMargin + fail(msg) + } + // When a job fails, we terminate before waiting for all the task end events to come in, + // so there might still be a running task set. So we only check these conditions + // when the job succeeds. + // When the final task of a taskset completes, we post + // the event to the DAGScheduler event loop before we finish processing in the taskscheduler + // thread. It's possible the DAGScheduler thread processes the event, finishes the job, + // and notifies the job waiter before our original thread in the task scheduler finishes + // handling the event and marks the taskset as complete. So its ok if we need to wait a + // *little* bit longer for the original taskscheduler thread to finish up to deal w/ the race. + eventually(timeout(1 second), interval(10 millis)) { + assert(taskScheduler.runningTaskSets.isEmpty) + } + assert(!backend.hasTasks) + } else { + assert(failure != null) + } + assert(scheduler.activeJobs.isEmpty) + assert(backendException.get() == null) + } + + /** + * Looks at all shuffleMapOutputs that are dependencies of the given RDD, and makes sure + * they are all registered + */ + def assertMapOutputAvailable(targetRdd: MockRDD): Unit = { + val shuffleIds = targetRdd.shuffleDeps.map{_.shuffleId} + val nParts = targetRdd.numPartitions + for { + shuffleId <- shuffleIds + reduceIdx <- (0 until nParts) + } { + val statuses = taskScheduler.mapOutputTracker.getMapSizesByExecutorId(shuffleId, reduceIdx) + // really we should have already thrown an exception rather than fail either of these + // asserts, but just to be extra defensive let's double check the statuses are OK + assert(statuses != null) + assert(statuses.nonEmpty) + } + } + + /** models a stage boundary with a single dependency, like a shuffle */ + def shuffle(nParts: Int, input: MockRDD): MockRDD = { + val partitioner = new HashPartitioner(nParts) + val shuffleDep = new ShuffleDependency[Int, Int, Nothing](input, partitioner) + new MockRDD(sc, nParts, List(shuffleDep)) + } + + /** models a stage boundary with multiple dependencies, like a join */ + def join(nParts: Int, inputs: MockRDD*): MockRDD = { + val partitioner = new HashPartitioner(nParts) + val shuffleDeps = inputs.map { inputRDD => + new ShuffleDependency[Int, Int, Nothing](inputRDD, partitioner) + } + new MockRDD(sc, nParts, shuffleDeps) + } + + val backendException = new AtomicReference[Exception](null) + + /** + * Helper which makes it a little easier to setup a test, which starts a mock backend in another + * thread, responding to tasks with your custom function. You also supply the "body" of your + * test, where you submit jobs to your backend, wait for them to complete, then check + * whatever conditions you want. Note that this is *not* safe to all bad backends -- + * in particular, your `backendFunc` has to return quickly, it can't throw errors, (instead + * it should send back the right TaskEndReason) + */ + def withBackend[T](backendFunc: () => Unit)(testBody: => T): T = { + val backendContinue = new AtomicBoolean(true) + val backendThread = new Thread("mock backend thread") { + override def run(): Unit = { + while (backendContinue.get()) { + if (backend.hasTasksWaitingToRun) { + try { + backendFunc() + } catch { + case ex: Exception => + // Try to do a little error handling around exceptions that might occur here -- + // otherwise it can just look like a TimeoutException in the test itself. + logError("Exception in mock backend:", ex) + backendException.set(ex) + backendContinue.set(false) + throw ex + } + } else { + Thread.sleep(10) + } + } + } + } + try { + backendThread.start() + testBody + } finally { + backendContinue.set(false) + backendThread.join() + } + } + + /** + * Helper to do a little extra error checking while waiting for the job to terminate. Primarily + * just does a little extra error handling if there is an exception from the backend. + */ + def awaitJobTermination(jobFuture: Future[_], duration: Duration): Unit = { + try { + ThreadUtils.awaitReady(jobFuture, duration) + } catch { + case te: TimeoutException if backendException.get() != null => + val msg = raw""" + | ----- Begin Backend Failure Msg ----- + | ${Utils.exceptionString(backendException.get())} + | ----- End Backend Failure Msg ---- + """. + stripMargin + + fail(s"Future timed out after ${duration}, likely because of failure in backend: $msg") + } + } +} + +/** + * Helper for running a backend in integration tests, does a bunch of the book-keeping + * so individual tests can focus on just responding to tasks. Individual tests will use + * [[beginTask]], [[taskSuccess]], and [[taskFailed]]. + */ +private[spark] abstract class MockBackend( + conf: SparkConf, + val taskScheduler: TaskSchedulerImpl) extends SchedulerBackend with Logging { + + // Periodically revive offers to allow delay scheduling to work + private val reviveThread = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-revive-thread") + private val reviveIntervalMs = conf.getTimeAsMs("spark.scheduler.revive.interval", "10ms") + + /** + * Test backends should call this to get a task that has been assigned to them by the scheduler. + * Each task should be responded to with either [[taskSuccess]] or [[taskFailed]]. + */ + def beginTask(): (TaskDescription, Task[_]) = { + synchronized { + val toRun = assignedTasksWaitingToRun.remove(assignedTasksWaitingToRun.size - 1) + runningTasks += toRun._1.taskId + toRun + } + } + + /** + * Tell the scheduler the task completed successfully, with the given result. Also + * updates some internal state for this mock. + */ + def taskSuccess(task: TaskDescription, result: Any): Unit = { + val ser = env.serializer.newInstance() + val resultBytes = ser.serialize(result) + val directResult = new DirectTaskResult(resultBytes, Seq()) // no accumulator updates + taskUpdate(task, TaskState.FINISHED, directResult) + } + + /** + * Tell the scheduler the task failed, with the given state and result (probably ExceptionFailure + * or FetchFailed). Also updates some internal state for this mock. + */ + def taskFailed(task: TaskDescription, exc: Exception): Unit = { + taskUpdate(task, TaskState.FAILED, new ExceptionFailure(exc, Seq())) + } + + def taskFailed(task: TaskDescription, reason: TaskFailedReason): Unit = { + taskUpdate(task, TaskState.FAILED, reason) + } + + def taskUpdate(task: TaskDescription, state: TaskState, result: Any): Unit = { + val ser = env.serializer.newInstance() + val resultBytes = ser.serialize(result) + // statusUpdate is safe to call from multiple threads, its protected inside taskScheduler + taskScheduler.statusUpdate(task.taskId, state, resultBytes) + if (TaskState.isFinished(state)) { + synchronized { + runningTasks -= task.taskId + executorIdToExecutor(task.executorId).freeCores += taskScheduler.CPUS_PER_TASK + freeCores += taskScheduler.CPUS_PER_TASK + } + reviveOffers() + } + } + + // protected by this + private val assignedTasksWaitingToRun = new ArrayBuffer[(TaskDescription, Task[_])](10000) + // protected by this + private val runningTasks = HashSet[Long]() + + def hasTasks: Boolean = synchronized { + assignedTasksWaitingToRun.nonEmpty || runningTasks.nonEmpty + } + + def hasTasksWaitingToRun: Boolean = { + assignedTasksWaitingToRun.nonEmpty + } + + override def start(): Unit = { + reviveThread.scheduleAtFixedRate(new Runnable { + override def run(): Unit = Utils.tryLogNonFatalError { + reviveOffers() + } + }, 0, reviveIntervalMs, TimeUnit.MILLISECONDS) + } + + override def stop(): Unit = { + reviveThread.shutdown() + } + + val env = SparkEnv.get + + /** Accessed by both scheduling and backend thread, so should be protected by this. */ + var freeCores: Int = _ + + /** + * Accessed by both scheduling and backend thread, so should be protected by this. + * Most likely the only thing that needs to be protected are the inidividual ExecutorTaskStatus, + * but for simplicity in this mock just lock the whole backend. + */ + def executorIdToExecutor: Map[String, ExecutorTaskStatus] + + private def generateOffers(): IndexedSeq[WorkerOffer] = { + executorIdToExecutor.values.filter { exec => + exec.freeCores > 0 + }.map { exec => + WorkerOffer(executorId = exec.executorId, host = exec.host, + cores = exec.freeCores) + }.toIndexedSeq + } + + /** + * This is called by the scheduler whenever it has tasks it would like to schedule, when a tasks + * completes (which will be in a result-getter thread), and by the reviveOffers thread for delay + * scheduling. + */ + override def reviveOffers(): Unit = { + // Need a lock on the entire scheduler to protect freeCores -- otherwise, multiple threads + // may make offers at the same time, though they are using the same set of freeCores. + taskScheduler.synchronized { + val newTaskDescriptions = taskScheduler.resourceOffers(generateOffers()).flatten + // get the task now, since that requires a lock on TaskSchedulerImpl, to prevent individual + // tests from introducing a race if they need it. + val newTasks = newTaskDescriptions.map { taskDescription => + val taskSet = taskScheduler.taskIdToTaskSetManager(taskDescription.taskId).taskSet + val task = taskSet.tasks(taskDescription.index) + (taskDescription, task) + } + newTasks.foreach { case (taskDescription, _) => + executorIdToExecutor(taskDescription.executorId).freeCores -= taskScheduler.CPUS_PER_TASK + } + freeCores -= newTasks.size * taskScheduler.CPUS_PER_TASK + assignedTasksWaitingToRun ++= newTasks + } + } + + override def killTask( + taskId: Long, executorId: String, interruptThread: Boolean, reason: String): Unit = { + // We have to implement this b/c of SPARK-15385. + // Its OK for this to be a no-op, because even if a backend does implement killTask, + // it really can only be "best-effort" in any case, and the scheduler should be robust to that. + // And in fact its reasonably simulating a case where a real backend finishes tasks in between + // the time when the scheduler sends the msg to kill tasks, and the backend receives the msg. + } +} + +/** + * A very simple mock backend that can just run one task at a time. + */ +private[spark] class SingleCoreMockBackend( + conf: SparkConf, + taskScheduler: TaskSchedulerImpl) extends MockBackend(conf, taskScheduler) { + + val cores = 1 + + override def defaultParallelism(): Int = conf.getInt("spark.default.parallelism", cores) + + freeCores = cores + val localExecutorId = SparkContext.DRIVER_IDENTIFIER + val localExecutorHostname = "localhost" + + override val executorIdToExecutor: Map[String, ExecutorTaskStatus] = Map( + localExecutorId -> new ExecutorTaskStatus(localExecutorHostname, localExecutorId, freeCores) + ) +} + +case class ExecutorTaskStatus(host: String, executorId: String, var freeCores: Int) + +class MockRDD( + sc: SparkContext, + val numPartitions: Int, + val shuffleDeps: Seq[ShuffleDependency[Int, Int, Nothing]] +) extends RDD[(Int, Int)](sc, shuffleDeps) with Serializable { + + MockRDD.validate(numPartitions, shuffleDeps) + + override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = + throw new RuntimeException("should not be reached") + override def getPartitions: Array[Partition] = { + (0 until numPartitions).map(i => new Partition { + override def index: Int = i + }).toArray + } + override def getPreferredLocations(split: Partition): Seq[String] = Nil + override def toString: String = "MockRDD " + id +} + +object MockRDD extends AssertionsHelper with TripleEquals { + /** + * make sure all the shuffle dependencies have a consistent number of output partitions + * (mostly to make sure the test setup makes sense, not that Spark itself would get this wrong) + */ + def validate(numPartitions: Int, dependencies: Seq[ShuffleDependency[_, _, _]]): Unit = { + dependencies.foreach { dependency => + val partitioner = dependency.partitioner + assert(partitioner != null) + assert(partitioner.numPartitions === numPartitions) + } + } +} + +/** Simple cluster manager that wires up our mock backend. */ +private class MockExternalClusterManager extends ExternalClusterManager { + + val MOCK_REGEX = """mock\[(.*)\]""".r + def canCreate(masterURL: String): Boolean = MOCK_REGEX.findFirstIn(masterURL).isDefined + + def createTaskScheduler( + sc: SparkContext, + masterURL: String): TaskScheduler = { + new TestTaskScheduler(sc) + } + + def createSchedulerBackend( + sc: SparkContext, + masterURL: String, + scheduler: TaskScheduler): SchedulerBackend = { + masterURL match { + case MOCK_REGEX(backendClassName) => + val backendClass = Utils.classForName(backendClassName) + val ctor = backendClass.getConstructor(classOf[SparkConf], classOf[TaskSchedulerImpl]) + ctor.newInstance(sc.getConf, scheduler).asInstanceOf[SchedulerBackend] + } + } + + def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { + scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) + } +} + +/** TaskSchedulerImpl that just tracks a tiny bit more state to enable checks in tests. */ +class TestTaskScheduler(sc: SparkContext) extends TaskSchedulerImpl(sc) { + /** Set of TaskSets the DAGScheduler has requested executed. */ + val runningTaskSets = HashSet[TaskSet]() + + override def submitTasks(taskSet: TaskSet): Unit = { + runningTaskSets += taskSet + super.submitTasks(taskSet) + } + + override def taskSetFinished(manager: TaskSetManager): Unit = { + runningTaskSets -= manager.taskSet + super.taskSetFinished(manager) + } +} + +/** + * Some very basic tests just to demonstrate the use of the test framework (and verify that it + * works). + */ +class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCoreMockBackend] { + + /** + * Very simple one stage job. Backend successfully completes each task, one by one + */ + testScheduler("super simple job") { + def runBackend(): Unit = { + val (taskDescripition, _) = backend.beginTask() + backend.taskSuccess(taskDescripition, 42) + } + withBackend(runBackend _) { + val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) + val duration = Duration(1, SECONDS) + awaitJobTermination(jobFuture, duration) + } + assert(results === (0 until 10).map { _ -> 42 }.toMap) + assertDataStructuresEmpty() + } + + /** + * 5 stage job, diamond dependencies. + * + * a ----> b ----> d --> result + * \--> c --/ + * + * Backend successfully completes each task + */ + testScheduler("multi-stage job") { + + def shuffleIdToOutputParts(shuffleId: Int): Int = { + shuffleId match { + case 0 => 10 + case 1 => 20 + case _ => 30 + } + } + + val a = new MockRDD(sc, 2, Nil) + val b = shuffle(10, a) + val c = shuffle(20, a) + val d = join(30, b, c) + + def runBackend(): Unit = { + val (taskDescription, task) = backend.beginTask() + + // make sure the required map output is available + task.stageId match { + case 4 => assertMapOutputAvailable(d) + case _ => + // we can't check for the output for the two intermediate stages, unfortunately, + // b/c the stage numbering is non-deterministic, so stage number alone doesn't tell + // us what to check + } + (task.stageId, task.stageAttemptId, task.partitionId) match { + case (stage, 0, _) if stage < 4 => + val shuffleId = + scheduler.stageIdToStage(stage).asInstanceOf[ShuffleMapStage].shuffleDep.shuffleId + backend.taskSuccess(taskDescription, + DAGSchedulerSuite.makeMapStatus("hostA", shuffleIdToOutputParts(shuffleId))) + case (4, 0, partition) => + backend.taskSuccess(taskDescription, 4321 + partition) + } + } + withBackend(runBackend _) { + val jobFuture = submit(d, (0 until 30).toArray) + val duration = Duration(1, SECONDS) + awaitJobTermination(jobFuture, duration) + } + assert(results === (0 until 30).map { idx => idx -> (4321 + idx) }.toMap) + assertDataStructuresEmpty() + } + + /** + * 2 stage job, with a fetch failure. Make sure that: + * (a) map output is available whenever we run stage 1 + * (b) we get a second attempt for stage 0 & stage 1 + */ + testScheduler("job with fetch failure") { + val input = new MockRDD(sc, 2, Nil) + val shuffledRdd = shuffle(10, input) + val shuffleId = shuffledRdd.shuffleDeps.head.shuffleId + + val stageToAttempts = new HashMap[Int, HashSet[Int]]() + + def runBackend(): Unit = { + val (taskDescription, task) = backend.beginTask() + stageToAttempts.getOrElseUpdate(task.stageId, new HashSet()) += task.stageAttemptId + + // We cannot check if shuffle output is available, because the failed fetch will clear the + // shuffle output. Then we'd have a race, between the already-started task from the first + // attempt, and when the failure clears out the map output status. + + (task.stageId, task.stageAttemptId, task.partitionId) match { + case (0, _, _) => + backend.taskSuccess(taskDescription, DAGSchedulerSuite.makeMapStatus("hostA", 10)) + case (1, 0, 0) => + val fetchFailed = FetchFailed( + DAGSchedulerSuite.makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored") + backend.taskFailed(taskDescription, fetchFailed) + case (1, _, partition) => + backend.taskSuccess(taskDescription, 42 + partition) + case unmatched => + fail(s"Unexpected shuffle output $unmatched") + } + } + withBackend(runBackend _) { + val jobFuture = submit(shuffledRdd, (0 until 10).toArray) + val duration = Duration(1, SECONDS) + awaitJobTermination(jobFuture, duration) + } + assertDataStructuresEmpty() + assert(results === (0 until 10).map { idx => idx -> (42 + idx) }.toMap) + assert(stageToAttempts === Map(0 -> Set(0, 1), 1 -> Set(0, 1))) + } + + testScheduler("job failure after 4 attempts") { + def runBackend(): Unit = { + val (taskDescription, _) = backend.beginTask() + backend.taskFailed(taskDescription, new RuntimeException("test task failure")) + } + withBackend(runBackend _) { + val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) + val duration = Duration(1, SECONDS) + awaitJobTermination(jobFuture, duration) + assert(failure.getMessage.contains("test task failure")) + } + assertDataStructuresEmpty(noFailure = false) + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 53102b9f1c93..d061c7845f4a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -19,52 +19,101 @@ package org.apache.spark.scheduler import java.util.concurrent.Semaphore -import scala.collection.mutable import scala.collection.JavaConverters._ +import scala.collection.mutable +import org.mockito.Mockito import org.scalatest.Matchers +import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.ResetSystemProperties -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.config.LISTENER_BUS_EVENT_QUEUE_CAPACITY +import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.util.{ResetSystemProperties, RpcUtils} class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Matchers with ResetSystemProperties { + import LiveListenerBus._ + /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 val jobCompletionTime = 1421191296660L + private val mockSparkContext: SparkContext = Mockito.mock(classOf[SparkContext]) + private val mockMetricsSystem: MetricsSystem = Mockito.mock(classOf[MetricsSystem]) + + private def numDroppedEvents(bus: LiveListenerBus): Long = { + bus.metrics.metricRegistry.counter(s"queue.$SHARED_QUEUE.numDroppedEvents").getCount + } + + private def queueSize(bus: LiveListenerBus): Int = { + bus.metrics.metricRegistry.getGauges().get(s"queue.$SHARED_QUEUE.size").getValue() + .asInstanceOf[Int] + } + + private def eventProcessingTimeCount(bus: LiveListenerBus): Long = { + bus.metrics.metricRegistry.timer(s"queue.$SHARED_QUEUE.listenerProcessingTime").getCount() + } + + test("don't call sc.stop in listener") { + sc = new SparkContext("local", "SparkListenerSuite", new SparkConf()) + val listener = new SparkContextStoppingListener(sc) + + sc.listenerBus.addToSharedQueue(listener) + sc.listenerBus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) + sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + sc.stop() + + assert(listener.sparkExSeen) + } + test("basic creation and shutdown of LiveListenerBus") { + val conf = new SparkConf() val counter = new BasicJobCounter - val bus = new LiveListenerBus - bus.addListener(counter) + val bus = new LiveListenerBus(conf) + bus.addToSharedQueue(counter) - // Listener bus hasn't started yet, so posting events should not increment counter + // Metrics are initially empty. + assert(bus.metrics.numEventsPosted.getCount === 0) + assert(numDroppedEvents(bus) === 0) + assert(queueSize(bus) === 0) + assert(eventProcessingTimeCount(bus) === 0) + + // Post five events: (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } + + // Five messages should be marked as received and queued, but no messages should be posted to + // listeners yet because the the listener bus hasn't been started. + assert(bus.metrics.numEventsPosted.getCount === 5) + assert(queueSize(bus) === 5) assert(counter.count === 0) // Starting listener bus should flush all buffered events - bus.start(sc) + bus.start(mockSparkContext, mockMetricsSystem) + Mockito.verify(mockMetricsSystem).registerSource(bus.metrics) bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(counter.count === 5) + assert(queueSize(bus) === 0) + assert(eventProcessingTimeCount(bus) === 5) // After listener bus has stopped, posting events should not increment counter bus.stop() (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } assert(counter.count === 5) + assert(eventProcessingTimeCount(bus) === 5) // Listener bus must not be started twice intercept[IllegalStateException] { - val bus = new LiveListenerBus - bus.start(sc) - bus.start(sc) + val bus = new LiveListenerBus(conf) + bus.start(mockSparkContext, mockMetricsSystem) + bus.start(mockSparkContext, mockMetricsSystem) } // ... or stopped before starting intercept[IllegalStateException] { - val bus = new LiveListenerBus + val bus = new LiveListenerBus(conf) bus.stop() } } @@ -91,12 +140,11 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match drained = true } } - - val bus = new LiveListenerBus + val bus = new LiveListenerBus(new SparkConf()) val blockingListener = new BlockingListener - bus.addListener(blockingListener) - bus.start(sc) + bus.addToSharedQueue(blockingListener) + bus.start(mockSparkContext, mockMetricsSystem) bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) listenerStarted.acquire() @@ -122,6 +170,43 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match assert(drained) } + test("metrics for dropped listener events") { + val bus = new LiveListenerBus(new SparkConf().set(LISTENER_BUS_EVENT_QUEUE_CAPACITY, 1)) + + val listenerStarted = new Semaphore(0) + val listenerWait = new Semaphore(0) + + bus.addToSharedQueue(new SparkListener { + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { + listenerStarted.release() + listenerWait.acquire() + } + }) + + bus.start(mockSparkContext, mockMetricsSystem) + + // Post a message to the listener bus and wait for processing to begin: + bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) + listenerStarted.acquire() + assert(queueSize(bus) === 0) + assert(numDroppedEvents(bus) === 0) + + // If we post an additional message then it should remain in the queue because the listener is + // busy processing the first event: + bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) + assert(queueSize(bus) === 1) + assert(numDroppedEvents(bus) === 0) + + // The queue is now full, so any additional events posted to the listener will be dropped: + bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) + assert(queueSize(bus) === 1) + assert(numDroppedEvents(bus) === 1) + + // Allow the the remaining events to be processed so we can stop the listener bus: + listenerWait.release(2) + bus.stop() + } + test("basic creation of StageInfo") { sc = new SparkContext("local", "SparkListenerSuite") val listener = new SaveStageAndTaskInfo @@ -168,7 +253,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) listener.stageInfos.size should be {1} val stageInfo2 = listener.stageInfos.keys.find(_.stageId == 1).get - stageInfo2.rddInfos.size should be {3} // ParallelCollectionRDD, FilteredRDD, MappedRDD + stageInfo2.rddInfos.size should be {3} stageInfo2.rddInfos.forall(_.numPartitions == 4) should be {true} stageInfo2.rddInfos.exists(_.name == "Deux") should be {true} listener.stageInfos.clear() @@ -213,7 +298,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match } val numSlices = 16 - val d = sc.parallelize(0 to 1e3.toInt, numSlices).map(w) + val d = sc.parallelize(0 to 10000, numSlices).map(w) d.count() sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) listener.stageInfos.size should be (1) @@ -221,7 +306,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match val d2 = d.map { i => w(i) -> i * 2 }.setName("shuffle input 1") val d3 = d.map { i => w(i) -> (0 to (i % 5)) }.setName("shuffle input 2") val d4 = d2.cogroup(d3, numSlices).map { case (k, (v1, v2)) => - w(k) -> (v1.size, v2.size) + (w(k), (v1.size, v2.size)) } d4.setName("A Cogroup") d4.collectAsMap() @@ -251,36 +336,31 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match taskInfoMetrics.foreach { case (taskInfo, taskMetrics) => taskMetrics.resultSize should be > (0L) if (stageInfo.rddInfos.exists(info => info.name == d2.name || info.name == d3.name)) { - taskMetrics.inputMetrics should not be ('defined) - taskMetrics.outputMetrics should not be ('defined) - taskMetrics.shuffleWriteMetrics should be ('defined) - taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0L) + assert(taskMetrics.shuffleWriteMetrics.bytesWritten > 0L) } if (stageInfo.rddInfos.exists(_.name == d4.name)) { - taskMetrics.shuffleReadMetrics should be ('defined) - val sm = taskMetrics.shuffleReadMetrics.get - sm.totalBlocksFetched should be (2*numSlices) - sm.localBlocksFetched should be (2*numSlices) - sm.remoteBlocksFetched should be (0) - sm.remoteBytesRead should be (0L) + assert(taskMetrics.shuffleReadMetrics.totalBlocksFetched == 2 * numSlices) + assert(taskMetrics.shuffleReadMetrics.localBlocksFetched == 2 * numSlices) + assert(taskMetrics.shuffleReadMetrics.remoteBlocksFetched == 0) + assert(taskMetrics.shuffleReadMetrics.remoteBytesRead == 0L) } } } } test("onTaskGettingResult() called when result fetched remotely") { - sc = new SparkContext("local", "SparkListenerSuite") + val conf = new SparkConf().set("spark.rpc.message.maxSize", "1") + sc = new SparkContext("local", "SparkListenerSuite", conf) val listener = new SaveTaskEvents sc.addSparkListener(listener) - // Make a task whose result is larger than the akka frame size - System.setProperty("spark.akka.frameSize", "1") - val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt + // Make a task whose result is larger than the RPC message size + val maxRpcMessageSize = RpcUtils.maxMessageSizeBytes(conf) + assert(maxRpcMessageSize === 1024 * 1024) val result = sc.parallelize(Seq(1), 1) - .map { x => 1.to(akkaFrameSize).toArray } + .map { x => 1.to(maxRpcMessageSize).toArray } .reduce { case (x, y) => x } - assert(result === 1.to(akkaFrameSize).toArray) + assert(result === 1.to(maxRpcMessageSize).toArray) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) val TASK_INDEX = 0 @@ -294,7 +374,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match val listener = new SaveTaskEvents sc.addSparkListener(listener) - // Make a task whose result is larger than the akka frame size + // Make a task whose result is larger than the RPC message size val result = sc.parallelize(Seq(1), 1).map(2 * _).reduce { case (x, y) => x } assert(result === 2) @@ -343,32 +423,61 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match val badListener = new BadListener val jobCounter1 = new BasicJobCounter val jobCounter2 = new BasicJobCounter - val bus = new LiveListenerBus + val bus = new LiveListenerBus(new SparkConf()) // Propagate events to bad listener first - bus.addListener(badListener) - bus.addListener(jobCounter1) - bus.addListener(jobCounter2) - bus.start(sc) + bus.addToSharedQueue(badListener) + bus.addToSharedQueue(jobCounter1) + bus.addToSharedQueue(jobCounter2) + bus.start(mockSparkContext, mockMetricsSystem) // Post events to all listeners, and wait until the queue is drained (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) // The exception should be caught, and the event should be propagated to other listeners - assert(bus.listenerThreadIsAlive) assert(jobCounter1.count === 5) assert(jobCounter2.count === 5) } test("registering listeners via spark.extraListeners") { + val listeners = Seq( + classOf[ListenerThatAcceptsSparkConf], + classOf[FirehoseListenerThatAcceptsSparkConf], + classOf[BasicJobCounter]) val conf = new SparkConf().setMaster("local").setAppName("test") - .set("spark.extraListeners", classOf[ListenerThatAcceptsSparkConf].getName + "," + - classOf[BasicJobCounter].getName) + .set("spark.extraListeners", listeners.map(_.getName).mkString(",")) sc = new SparkContext(conf) sc.listenerBus.listeners.asScala.count(_.isInstanceOf[BasicJobCounter]) should be (1) sc.listenerBus.listeners.asScala .count(_.isInstanceOf[ListenerThatAcceptsSparkConf]) should be (1) + sc.listenerBus.listeners.asScala + .count(_.isInstanceOf[FirehoseListenerThatAcceptsSparkConf]) should be (1) + } + + test("add and remove listeners to/from LiveListenerBus queues") { + val bus = new LiveListenerBus(new SparkConf(false)) + val counter1 = new BasicJobCounter() + val counter2 = new BasicJobCounter() + val counter3 = new BasicJobCounter() + + bus.addToSharedQueue(counter1) + bus.addToStatusQueue(counter2) + bus.addToStatusQueue(counter3) + assert(bus.activeQueues() === Set(SHARED_QUEUE, APP_STATUS_QUEUE)) + assert(bus.findListenersByClass[BasicJobCounter]().size === 3) + + bus.removeListener(counter1) + assert(bus.activeQueues() === Set(APP_STATUS_QUEUE)) + assert(bus.findListenersByClass[BasicJobCounter]().size === 2) + + bus.removeListener(counter2) + assert(bus.activeQueues() === Set(APP_STATUS_QUEUE)) + assert(bus.findListenersByClass[BasicJobCounter]().size === 1) + + bus.removeListener(counter3) + assert(bus.activeQueues().isEmpty) + assert(bus.findListenersByClass[BasicJobCounter]().isEmpty) } /** @@ -442,7 +551,30 @@ private class BasicJobCounter extends SparkListener { override def onJobEnd(job: SparkListenerJobEnd): Unit = count += 1 } +/** + * A simple listener that tries to stop SparkContext. + */ +private class SparkContextStoppingListener(val sc: SparkContext) extends SparkListener { + @volatile var sparkExSeen = false + override def onJobEnd(job: SparkListenerJobEnd): Unit = { + try { + sc.stop() + } catch { + case se: SparkException => + sparkExSeen = true + } + } +} + private class ListenerThatAcceptsSparkConf(conf: SparkConf) extends SparkListener { var count = 0 override def onJobEnd(job: SparkListenerJobEnd): Unit = count += 1 } + +private class FirehoseListenerThatAcceptsSparkConf(conf: SparkConf) extends SparkFirehoseListener { + var count = 0 + override def onEvent(event: SparkListenerEvent): Unit = event match { + case job: SparkListenerJobEnd => count += 1 + case _ => + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 450ab7b9fe92..a1d9085fa085 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -17,16 +17,19 @@ package org.apache.spark.scheduler -import org.mockito.Mockito._ -import org.mockito.Matchers.any +import java.util.Properties +import org.mockito.Matchers.any +import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.apache.spark._ -import org.apache.spark.rdd.RDD -import org.apache.spark.util.{TaskCompletionListener, TaskCompletionListenerException} +import org.apache.spark.executor.{Executor, TaskMetrics, TaskMetricsSuite} +import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.source.JvmSource - +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.rdd.RDD +import org.apache.spark.util._ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkContext { @@ -51,35 +54,91 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark val rdd = new RDD[String](sc, List()) { override def getPartitions = Array[Partition](StubPartition(0)) override def compute(split: Partition, context: TaskContext) = { - context.addTaskCompletionListener(context => TaskContextSuite.completed = true) + context.addTaskCompletionListener(new TaskCompletionListener { + override def onTaskCompletion(context: TaskContext): Unit = + TaskContextSuite.completed = true + }) sys.error("failed") } } val closureSerializer = SparkEnv.get.closureSerializer.newInstance() val func = (c: TaskContext, i: Iterator[String]) => i.next() - val taskBinary = sc.broadcast(closureSerializer.serialize((rdd, func)).array) + val taskBinary = sc.broadcast(JavaUtils.bufferToArray(closureSerializer.serialize((rdd, func)))) val task = new ResultTask[String, String]( - 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, Seq.empty) + 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties, + closureSerializer.serialize(TaskMetrics.registered).array()) intercept[RuntimeException] { task.run(0, 0, null) } assert(TaskContextSuite.completed === true) } + test("calls TaskFailureListeners after failure") { + TaskContextSuite.lastError = null + sc = new SparkContext("local", "test") + val rdd = new RDD[String](sc, List()) { + override def getPartitions = Array[Partition](StubPartition(0)) + override def compute(split: Partition, context: TaskContext) = { + context.addTaskFailureListener((context, error) => TaskContextSuite.lastError = error) + sys.error("damn error") + } + } + val closureSerializer = SparkEnv.get.closureSerializer.newInstance() + val func = (c: TaskContext, i: Iterator[String]) => i.next() + val taskBinary = sc.broadcast(JavaUtils.bufferToArray(closureSerializer.serialize((rdd, func)))) + val task = new ResultTask[String, String]( + 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties, + closureSerializer.serialize(TaskMetrics.registered).array()) + intercept[RuntimeException] { + task.run(0, 0, null) + } + assert(TaskContextSuite.lastError.getMessage == "damn error") + } + test("all TaskCompletionListeners should be called even if some fail") { val context = TaskContext.empty() val listener = mock(classOf[TaskCompletionListener]) - context.addTaskCompletionListener(_ => throw new Exception("blah")) + context.addTaskCompletionListener(new TaskCompletionListener { + override def onTaskCompletion(context: TaskContext): Unit = throw new Exception("blah") + }) context.addTaskCompletionListener(listener) - context.addTaskCompletionListener(_ => throw new Exception("blah")) + context.addTaskCompletionListener(new TaskCompletionListener { + override def onTaskCompletion(context: TaskContext): Unit = throw new Exception("blah") + }) intercept[TaskCompletionListenerException] { - context.markTaskCompleted() + context.markTaskCompleted(None) } verify(listener, times(1)).onTaskCompletion(any()) } + test("all TaskFailureListeners should be called even if some fail") { + val context = TaskContext.empty() + val listener = mock(classOf[TaskFailureListener]) + context.addTaskFailureListener(new TaskFailureListener { + override def onTaskFailure(context: TaskContext, error: Throwable): Unit = + throw new Exception("exception in listener1") + }) + context.addTaskFailureListener(listener) + context.addTaskFailureListener(new TaskFailureListener { + override def onTaskFailure(context: TaskContext, error: Throwable): Unit = + throw new Exception("exception in listener3") + }) + + val e = intercept[TaskCompletionListenerException] { + context.markTaskFailed(new Exception("exception in task")) + } + + // Make sure listener 2 was called. + verify(listener, times(1)).onTaskFailure(any(), any()) + + // also need to check failure in TaskFailureListener does not mask earlier exception + assert(e.getMessage.contains("exception in listener1")) + assert(e.getMessage.contains("exception in listener3")) + assert(e.getMessage.contains("exception in task")) + } + test("TaskContext.attemptNumber should return attempt number, not task id (SPARK-4014)") { sc = new SparkContext("local[1,2]", "test") // use maxRetries = 2 because we test failed tasks // Check that attemptIds are 0 for all tasks' initial attempts @@ -99,17 +158,162 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark assert(attemptIdsWithFailedTask.toSet === Set(0, 1)) } - test("TaskContext.attemptId returns taskAttemptId for backwards-compatibility (SPARK-4014)") { + test("accumulators are updated on exception failures") { + // This means use 1 core and 4 max task failures + sc = new SparkContext("local[1,4]", "test") + // Create 2 accumulators, one that counts failed values and another that doesn't + val acc1 = AccumulatorSuite.createLongAccum("x", true) + val acc2 = AccumulatorSuite.createLongAccum("y", false) + // Fail first 3 attempts of every task. This means each task should be run 4 times. + sc.parallelize(1 to 10, 10).map { i => + acc1.add(1) + acc2.add(1) + if (TaskContext.get.attemptNumber() <= 2) { + throw new Exception("you did something wrong") + } else { + 0 + } + }.count() + // The one that counts failed values should be 4x the one that didn't, + // since we ran each task 4 times + assert(AccumulatorContext.get(acc1.id).get.value === 40L) + assert(AccumulatorContext.get(acc2.id).get.value === 10L) + } + + test("failed tasks collect only accumulators whose values count during failures") { sc = new SparkContext("local", "test") - val attemptIds = sc.parallelize(Seq(1, 2, 3, 4), 4).mapPartitions { iter => - Seq(TaskContext.get().attemptId).iterator - }.collect() - assert(attemptIds.toSet === Set(0, 1, 2, 3)) + val acc1 = AccumulatorSuite.createLongAccum("x", false) + val acc2 = AccumulatorSuite.createLongAccum("y", true) + acc1.add(1) + acc2.add(1) + // Create a dummy task. We won't end up running this; we just want to collect + // accumulator updates from it. + val taskMetrics = TaskMetrics.empty + val task = new Task[Int](0, 0, 0) { + context = new TaskContextImpl(0, 0, 0L, 0, + new TaskMemoryManager(SparkEnv.get.memoryManager, 0L), + new Properties, + SparkEnv.get.metricsSystem, + taskMetrics) + taskMetrics.registerAccumulator(acc1) + taskMetrics.registerAccumulator(acc2) + override def runTask(tc: TaskContext): Int = 0 + } + // First, simulate task success. This should give us all the accumulators. + val accumUpdates1 = task.collectAccumulatorUpdates(taskFailed = false) + TaskMetricsSuite.assertUpdatesEquals(accumUpdates1.takeRight(2), Seq(acc1, acc2)) + // Now, simulate task failures. This should give us only the accums that count failed values. + val accumUpdates2 = task.collectAccumulatorUpdates(taskFailed = true) + TaskMetricsSuite.assertUpdatesEquals(accumUpdates2.takeRight(1), Seq(acc2)) + } + + test("only updated internal accumulators will be sent back to driver") { + sc = new SparkContext("local", "test") + // Create a dummy task. We won't end up running this; we just want to collect + // accumulator updates from it. + val taskMetrics = TaskMetrics.registered + val task = new Task[Int](0, 0, 0) { + context = new TaskContextImpl(0, 0, 0L, 0, + new TaskMemoryManager(SparkEnv.get.memoryManager, 0L), + new Properties, + SparkEnv.get.metricsSystem, + taskMetrics) + taskMetrics.incMemoryBytesSpilled(10) + override def runTask(tc: TaskContext): Int = 0 + } + val updatedAccums = task.collectAccumulatorUpdates() + assert(updatedAccums.length == 2) + // the RESULT_SIZE accumulator will be sent back anyway. + assert(updatedAccums(0).name == Some(InternalAccumulator.RESULT_SIZE)) + assert(updatedAccums(0).value == 0) + assert(updatedAccums(1).name == Some(InternalAccumulator.MEMORY_BYTES_SPILLED)) + assert(updatedAccums(1).value == 10) + } + + test("localProperties are propagated to executors correctly") { + sc = new SparkContext("local", "test") + sc.setLocalProperty("testPropKey", "testPropValue") + val res = sc.parallelize(Array(1), 1).map(i => i).map(i => { + val inTask = TaskContext.get().getLocalProperty("testPropKey") + val inDeser = Executor.taskDeserializationProps.get().getProperty("testPropKey") + s"$inTask,$inDeser" + }).collect() + assert(res === Array("testPropValue,testPropValue")) + } + + test("immediately call a completion listener if the context is completed") { + var invocations = 0 + val context = TaskContext.empty() + context.markTaskCompleted(None) + context.addTaskCompletionListener(new TaskCompletionListener { + override def onTaskCompletion(context: TaskContext): Unit = + invocations += 1 + }) + assert(invocations == 1) + context.markTaskCompleted(None) + assert(invocations == 1) + } + + test("immediately call a failure listener if the context has failed") { + var invocations = 0 + var lastError: Throwable = null + val error = new RuntimeException + val context = TaskContext.empty() + context.markTaskFailed(error) + context.addTaskFailureListener(new TaskFailureListener { + override def onTaskFailure(context: TaskContext, e: Throwable): Unit = { + lastError = e + invocations += 1 + } + }) + assert(lastError == error) + assert(invocations == 1) + context.markTaskFailed(error) + assert(lastError == error) + assert(invocations == 1) + } + + test("TaskCompletionListenerException.getMessage should include previousError") { + val listenerErrorMessage = "exception in listener" + val taskErrorMessage = "exception in task" + val e = new TaskCompletionListenerException( + Seq(listenerErrorMessage), + Some(new RuntimeException(taskErrorMessage))) + assert(e.getMessage.contains(listenerErrorMessage) && e.getMessage.contains(taskErrorMessage)) } + + test("all TaskCompletionListeners should be called even if some fail or a task") { + val context = TaskContext.empty() + val listener = mock(classOf[TaskCompletionListener]) + context.addTaskCompletionListener(new TaskCompletionListener { + override def onTaskCompletion(context: TaskContext): Unit = + throw new Exception("exception in listener1") + }) + context.addTaskCompletionListener(listener) + context.addTaskCompletionListener(new TaskCompletionListener { + override def onTaskCompletion(context: TaskContext): Unit = + throw new Exception("exception in listener3") + }) + + val e = intercept[TaskCompletionListenerException] { + context.markTaskCompleted(Some(new Exception("exception in task"))) + } + + // Make sure listener 2 was called. + verify(listener, times(1)).onTaskCompletion(any()) + + // also need to check failure in TaskCompletionListener does not mask earlier exception + assert(e.getMessage.contains("exception in listener1")) + assert(e.getMessage.contains("exception in listener3")) + assert(e.getMessage.contains("exception in task")) + } + } private object TaskContextSuite { @volatile var completed = false + + @volatile var lastError: Throwable = _ } private case class StubPartition(index: Int) extends Partition diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala new file mode 100644 index 000000000000..97487ce1d2ca --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import java.io.{ByteArrayOutputStream, DataOutputStream, UTFDataFormatException} +import java.nio.ByteBuffer +import java.util.Properties + +import scala.collection.mutable.HashMap + +import org.apache.spark.SparkFunSuite + +class TaskDescriptionSuite extends SparkFunSuite { + test("encoding and then decoding a TaskDescription results in the same TaskDescription") { + val originalFiles = new HashMap[String, Long]() + originalFiles.put("fileUrl1", 1824) + originalFiles.put("fileUrl2", 2) + + val originalJars = new HashMap[String, Long]() + originalJars.put("jar1", 3) + + val originalProperties = new Properties() + originalProperties.put("property1", "18") + originalProperties.put("property2", "test value") + // SPARK-19796 -- large property values (like a large job description for a long sql query) + // can cause problems for DataOutputStream, make sure we handle correctly + val sb = new StringBuilder() + (0 to 10000).foreach(_ => sb.append("1234567890")) + val largeString = sb.toString() + originalProperties.put("property3", largeString) + // make sure we've got a good test case + intercept[UTFDataFormatException] { + val out = new DataOutputStream(new ByteArrayOutputStream()) + try { + out.writeUTF(largeString) + } finally { + out.close() + } + } + + // Create a dummy byte buffer for the task. + val taskBuffer = ByteBuffer.wrap(Array[Byte](1, 2, 3, 4)) + + val originalTaskDescription = new TaskDescription( + taskId = 1520589, + attemptNumber = 2, + executorId = "testExecutor", + name = "task for test", + index = 19, + originalFiles, + originalJars, + originalProperties, + taskBuffer + ) + + val serializedTaskDescription = TaskDescription.encode(originalTaskDescription) + val decodedTaskDescription = TaskDescription.decode(serializedTaskDescription) + + // Make sure that all of the fields in the decoded task description match the original. + assert(decodedTaskDescription.taskId === originalTaskDescription.taskId) + assert(decodedTaskDescription.attemptNumber === originalTaskDescription.attemptNumber) + assert(decodedTaskDescription.executorId === originalTaskDescription.executorId) + assert(decodedTaskDescription.name === originalTaskDescription.name) + assert(decodedTaskDescription.index === originalTaskDescription.index) + assert(decodedTaskDescription.addedFiles.equals(originalFiles)) + assert(decodedTaskDescription.addedJars.equals(originalJars)) + assert(decodedTaskDescription.properties.equals(originalTaskDescription.properties)) + assert(decodedTaskDescription.serializedTask.equals(taskBuffer)) + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 815caa79ff52..1bddba8f6c82 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -17,17 +17,27 @@ package org.apache.spark.scheduler +import java.io.{File, ObjectInputStream} +import java.net.URL import java.nio.ByteBuffer +import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.control.NonFatal +import com.google.common.util.concurrent.MoreExecutors +import org.mockito.ArgumentCaptor +import org.mockito.Matchers.{any, anyLong} +import org.mockito.Mockito.{spy, times, verify} import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, SparkFunSuite} +import org.apache.spark._ +import org.apache.spark.TestUtils.JavaSourceFromString import org.apache.spark.storage.TaskResultBlockId +import org.apache.spark.util.{MutableURLClassLoader, RpcUtils, Utils} + /** * Removes the TaskResult from the BlockManager before delegating to a normal TaskResultGetter. @@ -35,7 +45,7 @@ import org.apache.spark.storage.TaskResultBlockId * Used to test the case where a BlockManager evicts the task result (or dies) before the * TaskResult is retrieved. */ -class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedulerImpl) +private class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedulerImpl) extends TaskResultGetter(sparkEnv, scheduler) { var removedResult = false @@ -68,27 +78,52 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedule } } + +/** + * A [[TaskResultGetter]] that stores the [[DirectTaskResult]]s it receives from executors + * _before_ modifying the results in any way. + */ +private class MyTaskResultGetter(env: SparkEnv, scheduler: TaskSchedulerImpl) + extends TaskResultGetter(env, scheduler) { + + // Use the current thread so we can access its results synchronously + protected override val getTaskResultExecutor = MoreExecutors.sameThreadExecutor() + + // DirectTaskResults that we receive from the executors + private val _taskResults = new ArrayBuffer[DirectTaskResult[_]] + + def taskResults: Seq[DirectTaskResult[_]] = _taskResults + + override def enqueueSuccessfulTask(tsm: TaskSetManager, tid: Long, data: ByteBuffer): Unit = { + // work on a copy since the super class still needs to use the buffer + val newBuffer = data.duplicate() + _taskResults += env.closureSerializer.newInstance().deserialize[DirectTaskResult[_]](newBuffer) + super.enqueueSuccessfulTask(tsm, tid, data) + } +} + + /** * Tests related to handling task results (both direct and indirect). */ class TaskResultGetterSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkContext { - // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small + // Set the RPC message size to be as small as possible (it must be an integer, so 1 is as small // as we can make it) so the tests don't take too long. - def conf: SparkConf = new SparkConf().set("spark.akka.frameSize", "1") + def conf: SparkConf = new SparkConf().set("spark.rpc.message.maxSize", "1") - test("handling results smaller than Akka frame size") { + test("handling results smaller than max RPC message size") { sc = new SparkContext("local", "test", conf) val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x) assert(result === 2) } - test("handling results larger than Akka frame size") { + test("handling results larger than max RPC message size") { sc = new SparkContext("local", "test", conf) - val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt - val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) - assert(result === 1.to(akkaFrameSize).toArray) + val maxRpcMessageSize = RpcUtils.maxMessageSizeBytes(conf) + val result = + sc.parallelize(Seq(1), 1).map(x => 1.to(maxRpcMessageSize).toArray).reduce((x, y) => x) + assert(result === 1.to(maxRpcMessageSize).toArray) val RESULT_BLOCK_ID = TaskResultBlockId(0) assert(sc.env.blockManager.master.getLocations(RESULT_BLOCK_ID).size === 0, @@ -110,14 +145,127 @@ class TaskResultGetterSuite extends SparkFunSuite with BeforeAndAfter with Local } val resultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) scheduler.taskResultGetter = resultGetter - val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt - val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) + val maxRpcMessageSize = RpcUtils.maxMessageSizeBytes(conf) + val result = + sc.parallelize(Seq(1), 1).map(x => 1.to(maxRpcMessageSize).toArray).reduce((x, y) => x) assert(resultGetter.removeBlockSuccessfully) - assert(result === 1.to(akkaFrameSize).toArray) + assert(result === 1.to(maxRpcMessageSize).toArray) // Make sure two tasks were run (one failed one, and a second retried one). assert(scheduler.nextTaskId.get() === 2) } + + /** + * Make sure we are using the context classloader when deserializing failed TaskResults instead + * of the Spark classloader. + + * This test compiles a jar containing an exception and tests that when it is thrown on the + * executor, enqueueFailedTask can correctly deserialize the failure and identify the thrown + * exception as the cause. + + * Before this fix, enqueueFailedTask would throw a ClassNotFoundException when deserializing + * the exception, resulting in an UnknownReason for the TaskEndResult. + */ + test("failed task deserialized with the correct classloader (SPARK-11195)") { + // compile a small jar containing an exception that will be thrown on an executor. + val tempDir = Utils.createTempDir() + val srcDir = new File(tempDir, "repro/") + srcDir.mkdirs() + val excSource = new JavaSourceFromString(new File(srcDir, "MyException").toURI.getPath, + """package repro; + | + |public class MyException extends Exception { + |} + """.stripMargin) + val excFile = TestUtils.createCompiledClass("MyException", srcDir, excSource, Seq.empty) + val jarFile = new File(tempDir, "testJar-%s.jar".format(System.currentTimeMillis())) + TestUtils.createJar(Seq(excFile), jarFile, directoryPrefix = Some("repro")) + + // ensure we reset the classloader after the test completes + val originalClassLoader = Thread.currentThread.getContextClassLoader + val loader = new MutableURLClassLoader(new Array[URL](0), originalClassLoader) + Utils.tryWithSafeFinally { + // load the exception from the jar + loader.addURL(jarFile.toURI.toURL) + Thread.currentThread().setContextClassLoader(loader) + val excClass: Class[_] = Utils.classForName("repro.MyException") + + // NOTE: we must run the cluster with "local" so that the executor can load the compiled + // jar. + sc = new SparkContext("local", "test", conf) + val rdd = sc.parallelize(Seq(1), 1).map { _ => + val exc = excClass.newInstance().asInstanceOf[Exception] + throw exc + } + + // the driver should not have any problems resolving the exception class and determining + // why the task failed. + val exceptionMessage = intercept[SparkException] { + rdd.collect() + }.getMessage + + val expectedFailure = """(?s).*Lost task.*: repro.MyException.*""".r + val unknownFailure = """(?s).*Lost task.*: UnknownReason.*""".r + + assert(expectedFailure.findFirstMatchIn(exceptionMessage).isDefined) + assert(unknownFailure.findFirstMatchIn(exceptionMessage).isEmpty) + } { + Thread.currentThread.setContextClassLoader(originalClassLoader) + loader.close() + } + } + + test("task result size is set on the driver, not the executors") { + import InternalAccumulator._ + + // Set up custom TaskResultGetter and TaskSchedulerImpl spy + sc = new SparkContext("local", "test", conf) + val scheduler = sc.taskScheduler.asInstanceOf[TaskSchedulerImpl] + val spyScheduler = spy(scheduler) + val resultGetter = new MyTaskResultGetter(sc.env, spyScheduler) + val newDAGScheduler = new DAGScheduler(sc, spyScheduler) + scheduler.taskResultGetter = resultGetter + sc.dagScheduler = newDAGScheduler + sc.taskScheduler = spyScheduler + sc.taskScheduler.setDAGScheduler(newDAGScheduler) + + // Just run 1 task and capture the corresponding DirectTaskResult + sc.parallelize(1 to 1, 1).count() + val captor = ArgumentCaptor.forClass(classOf[DirectTaskResult[_]]) + verify(spyScheduler, times(1)).handleSuccessfulTask(any(), anyLong(), captor.capture()) + + // When a task finishes, the executor sends a serialized DirectTaskResult to the driver + // without setting the result size so as to avoid serializing the result again. Instead, + // the result size is set later in TaskResultGetter on the driver before passing the + // DirectTaskResult on to TaskSchedulerImpl. In this test, we capture the DirectTaskResult + // before and after the result size is set. + assert(resultGetter.taskResults.size === 1) + val resBefore = resultGetter.taskResults.head + val resAfter = captor.getValue + val resSizeBefore = resBefore.accumUpdates.find(_.name == Some(RESULT_SIZE)).map(_.value) + val resSizeAfter = resAfter.accumUpdates.find(_.name == Some(RESULT_SIZE)).map(_.value) + assert(resSizeBefore.exists(_ == 0L)) + assert(resSizeAfter.exists(_.toString.toLong > 0L)) + } + + test("failed task is handled when error occurs deserializing the reason") { + sc = new SparkContext("local", "test", conf) + val rdd = sc.parallelize(Seq(1), 1).map { _ => + throw new UndeserializableException + } + val message = intercept[SparkException] { + rdd.collect() + }.getMessage + // Job failed, even though the failure reason is unknown. + val unknownFailure = """(?s).*Lost task.*: UnknownReason.*""".r + assert(unknownFailure.findFirstMatchIn(message).isDefined) + } + +} + +private class UndeserializableException extends Exception { + private def readObject(in: ObjectInputStream): Unit = { + throw new NoClassDefFoundError() + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 2afb595e6f10..b8626bf77759 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -17,7 +17,19 @@ package org.apache.spark.scheduler +import java.nio.ByteBuffer + +import scala.collection.mutable.HashMap + +import org.mockito.Matchers.{anyInt, anyObject, anyString, eq => meq} +import org.mockito.Mockito.{atLeast, atMost, never, spy, times, verify, when} +import org.scalatest.BeforeAndAfterEach +import org.scalatest.mockito.MockitoSugar + import org.apache.spark._ +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config +import org.apache.spark.util.ManualClock class FakeSchedulerBackend extends SchedulerBackend { def start() {} @@ -26,20 +38,96 @@ class FakeSchedulerBackend extends SchedulerBackend { def defaultParallelism(): Int = 1 } -class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with Logging { +class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfterEach + with Logging with MockitoSugar { - test("Scheduler does not always schedule tasks on the same workers") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) + var failedTaskSetException: Option[Throwable] = None + var failedTaskSetReason: String = null + var failedTaskSet = false + + var blacklist: BlacklistTracker = null + var taskScheduler: TaskSchedulerImpl = null + var dagScheduler: DAGScheduler = null + + val stageToMockTaskSetBlacklist = new HashMap[Int, TaskSetBlacklist]() + val stageToMockTaskSetManager = new HashMap[Int, TaskSetManager]() + + override def beforeEach(): Unit = { + super.beforeEach() + failedTaskSet = false + failedTaskSetException = None + failedTaskSetReason = null + stageToMockTaskSetBlacklist.clear() + stageToMockTaskSetManager.clear() + } + + override def afterEach(): Unit = { + super.afterEach() + if (taskScheduler != null) { + taskScheduler.stop() + taskScheduler = null + } + if (dagScheduler != null) { + dagScheduler.stop() + dagScheduler = null + } + } + + def setupScheduler(confs: (String, String)*): TaskSchedulerImpl = { + val conf = new SparkConf().setMaster("local").setAppName("TaskSchedulerImplSuite") + confs.foreach { case (k, v) => conf.set(k, v) } + sc = new SparkContext(conf) + taskScheduler = new TaskSchedulerImpl(sc) + setupHelper() + } + + def setupSchedulerWithMockTaskSetBlacklist(): TaskSchedulerImpl = { + blacklist = mock[BlacklistTracker] + val conf = new SparkConf().setMaster("local").setAppName("TaskSchedulerImplSuite") + conf.set(config.BLACKLIST_ENABLED, true) + sc = new SparkContext(conf) + taskScheduler = + new TaskSchedulerImpl(sc, sc.conf.getInt("spark.task.maxFailures", 4)) { + override def createTaskSetManager(taskSet: TaskSet, maxFailures: Int): TaskSetManager = { + val tsm = super.createTaskSetManager(taskSet, maxFailures) + // we need to create a spied tsm just so we can set the TaskSetBlacklist + val tsmSpy = spy(tsm) + val taskSetBlacklist = mock[TaskSetBlacklist] + when(tsmSpy.taskSetBlacklistHelperOpt).thenReturn(Some(taskSetBlacklist)) + stageToMockTaskSetManager(taskSet.stageId) = tsmSpy + stageToMockTaskSetBlacklist(taskSet.stageId) = taskSetBlacklist + tsmSpy + } + + override private[scheduler] lazy val blacklistTrackerOpt = Some(blacklist) + } + setupHelper() + } + + def setupHelper(): TaskSchedulerImpl = { taskScheduler.initialize(new FakeSchedulerBackend) // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. - new DAGScheduler(sc, taskScheduler) { - override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} - override def executorAdded(execId: String, host: String) {} + dagScheduler = new DAGScheduler(sc, taskScheduler) { + override def taskStarted(task: Task[_], taskInfo: TaskInfo): Unit = {} + override def executorAdded(execId: String, host: String): Unit = {} + override def taskSetFailed( + taskSet: TaskSet, + reason: String, + exception: Option[Throwable]): Unit = { + // Normally the DAGScheduler puts this in the event loop, which will eventually fail + // dependent jobs + failedTaskSet = true + failedTaskSetReason = reason + failedTaskSetException = exception + } } + taskScheduler + } + test("Scheduler does not always schedule tasks on the same workers") { + val taskScheduler = setupScheduler() val numFreeCores = 1 - val workerOffers = Seq(new WorkerOffer("executor0", "host0", numFreeCores), + val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores), new WorkerOffer("executor1", "host1", numFreeCores)) // Repeatedly try to schedule a 1-task job, and make sure that it doesn't always // get scheduled on the same executor. While there is a chance this test will fail @@ -57,22 +145,14 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with L val count = selectedExecutorIds.count(_ == workerOffers(0).executorId) assert(count > 0) assert(count < numTrials) + assert(!failedTaskSet) } test("Scheduler correctly accounts for multiple CPUs per task") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") val taskCpus = 2 - - sc.conf.set("spark.task.cpus", taskCpus.toString) - val taskScheduler = new TaskSchedulerImpl(sc) - taskScheduler.initialize(new FakeSchedulerBackend) - // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. - new DAGScheduler(sc, taskScheduler) { - override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} - override def executorAdded(execId: String, host: String) {} - } + val taskScheduler = setupScheduler("spark.task.cpus" -> taskCpus.toString) // Give zero core offers. Should not generate any tasks - val zeroCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", 0), + val zeroCoreWorkerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 0), new WorkerOffer("executor1", "host1", 0)) val taskSet = FakeTask.createTaskSet(1) taskScheduler.submitTasks(taskSet) @@ -81,7 +161,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with L // No tasks should run as we only have 1 core free. val numFreeCores = 1 - val singleCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", numFreeCores), + val singleCoreWorkerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores), new WorkerOffer("executor1", "host1", numFreeCores)) taskScheduler.submitTasks(taskSet) taskDescriptions = taskScheduler.resourceOffers(singleCoreWorkerOffers).flatten @@ -89,55 +169,40 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with L // Now change the offers to have 2 cores in one executor and verify if it // is chosen. - val multiCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", taskCpus), + val multiCoreWorkerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", taskCpus), new WorkerOffer("executor1", "host1", numFreeCores)) taskScheduler.submitTasks(taskSet) taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten assert(1 === taskDescriptions.length) assert("executor0" === taskDescriptions(0).executorId) + assert(!failedTaskSet) } test("Scheduler does not crash when tasks are not serializable") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") val taskCpus = 2 - - sc.conf.set("spark.task.cpus", taskCpus.toString) - val taskScheduler = new TaskSchedulerImpl(sc) - taskScheduler.initialize(new FakeSchedulerBackend) - // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. - val dagScheduler = new DAGScheduler(sc, taskScheduler) { - override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} - override def executorAdded(execId: String, host: String) {} - } + val taskScheduler = setupScheduler("spark.task.cpus" -> taskCpus.toString) val numFreeCores = 1 - taskScheduler.setDAGScheduler(dagScheduler) val taskSet = new TaskSet( Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) - val multiCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", taskCpus), + val multiCoreWorkerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", taskCpus), new WorkerOffer("executor1", "host1", numFreeCores)) taskScheduler.submitTasks(taskSet) var taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten assert(0 === taskDescriptions.length) + assert(failedTaskSet) + assert(failedTaskSetReason.contains("Failed to serialize task")) // Now check that we can still submit tasks - // Even if one of the tasks has not-serializable tasks, the other task set should + // Even if one of the task sets has not-serializable tasks, the other task set should // still be processed without error - taskScheduler.submitTasks(taskSet) taskScheduler.submitTasks(FakeTask.createTaskSet(1)) + taskScheduler.submitTasks(taskSet) taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten assert(taskDescriptions.map(_.executorId) === Seq("executor0")) } test("refuse to schedule concurrent attempts for the same stage (SPARK-8103)") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) - taskScheduler.initialize(new FakeSchedulerBackend) - // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. - val dagScheduler = new DAGScheduler(sc, taskScheduler) { - override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} - override def executorAdded(execId: String, host: String) {} - } - taskScheduler.setDAGScheduler(dagScheduler) + val taskScheduler = setupScheduler() val attempt1 = FakeTask.createTaskSet(1, 0) val attempt2 = FakeTask.createTaskSet(1, 1) taskScheduler.submitTasks(attempt1) @@ -152,20 +217,14 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with L taskScheduler.taskSetManagerForAttempt(attempt2.stageId, attempt2.stageAttemptId) .get.isZombie = true taskScheduler.submitTasks(attempt3) + assert(!failedTaskSet) } test("don't schedule more tasks after a taskset is zombie") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) - taskScheduler.initialize(new FakeSchedulerBackend) - // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. - new DAGScheduler(sc, taskScheduler) { - override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} - override def executorAdded(execId: String, host: String) {} - } + val taskScheduler = setupScheduler() val numFreeCores = 1 - val workerOffers = Seq(new WorkerOffer("executor0", "host0", numFreeCores)) + val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores)) val attempt1 = FakeTask.createTaskSet(10) // submit attempt 1, offer some resources, some tasks get scheduled @@ -190,20 +249,14 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with L assert(1 === taskDescriptions3.length) val mgr = taskScheduler.taskIdToTaskSetManager.get(taskDescriptions3(0).taskId).get assert(mgr.taskSet.stageAttemptId === 1) + assert(!failedTaskSet) } test("if a zombie attempt finishes, continue scheduling tasks for non-zombie attempts") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) - taskScheduler.initialize(new FakeSchedulerBackend) - // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. - new DAGScheduler(sc, taskScheduler) { - override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} - override def executorAdded(execId: String, host: String) {} - } + val taskScheduler = setupScheduler() val numFreeCores = 10 - val workerOffers = Seq(new WorkerOffer("executor0", "host0", numFreeCores)) + val workerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores)) val attempt1 = FakeTask.createTaskSet(10) // submit attempt 1, offer some resources, some tasks get scheduled @@ -235,20 +288,14 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with L val mgr = taskScheduler.taskIdToTaskSetManager.get(task.taskId).get assert(mgr.taskSet.stageAttemptId === 1) } + assert(!failedTaskSet) } test("tasks are not re-scheduled while executor loss reason is pending") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) - taskScheduler.initialize(new FakeSchedulerBackend) - // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. - new DAGScheduler(sc, taskScheduler) { - override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} - override def executorAdded(execId: String, host: String) {} - } + val taskScheduler = setupScheduler() - val e0Offers = Seq(new WorkerOffer("executor0", "host0", 1)) - val e1Offers = Seq(new WorkerOffer("executor1", "host0", 1)) + val e0Offers = IndexedSeq(new WorkerOffer("executor0", "host0", 1)) + val e1Offers = IndexedSeq(new WorkerOffer("executor1", "host0", 1)) val attempt1 = FakeTask.createTaskSet(1) // submit attempt 1, offer resources, task gets scheduled @@ -271,6 +318,598 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with L val taskDescriptions3 = taskScheduler.resourceOffers(e1Offers).flatten assert(1 === taskDescriptions3.length) assert("executor1" === taskDescriptions3(0).executorId) + assert(!failedTaskSet) + } + + test("scheduled tasks obey task and stage blacklists") { + taskScheduler = setupSchedulerWithMockTaskSetBlacklist() + (0 to 2).foreach {stageId => + val taskSet = FakeTask.createTaskSet(numTasks = 2, stageId = stageId, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet) + } + + // Setup our mock blacklist: + // * stage 0 is blacklisted on node "host1" + // * stage 1 is blacklisted on executor "executor3" + // * stage 0, partition 0 is blacklisted on executor 0 + // (mocked methods default to returning false, ie. no blacklisting) + when(stageToMockTaskSetBlacklist(0).isNodeBlacklistedForTaskSet("host1")).thenReturn(true) + when(stageToMockTaskSetBlacklist(1).isExecutorBlacklistedForTaskSet("executor3")) + .thenReturn(true) + when(stageToMockTaskSetBlacklist(0).isExecutorBlacklistedForTask("executor0", 0)) + .thenReturn(true) + + val offers = IndexedSeq( + new WorkerOffer("executor0", "host0", 1), + new WorkerOffer("executor1", "host1", 1), + new WorkerOffer("executor2", "host1", 1), + new WorkerOffer("executor3", "host2", 10) + ) + val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten + // We should schedule all tasks. + assert(firstTaskAttempts.size === 6) + // Whenever we schedule a task, we must consult the node and executor blacklist. (The test + // doesn't check exactly what checks are made because the offers get shuffled.) + (0 to 2).foreach { stageId => + verify(stageToMockTaskSetBlacklist(stageId), atLeast(1)) + .isNodeBlacklistedForTaskSet(anyString()) + verify(stageToMockTaskSetBlacklist(stageId), atLeast(1)) + .isExecutorBlacklistedForTaskSet(anyString()) + } + + def tasksForStage(stageId: Int): Seq[TaskDescription] = { + firstTaskAttempts.filter{_.name.contains(s"stage $stageId")} + } + tasksForStage(0).foreach { task => + // executors 1 & 2 blacklisted for node + // executor 0 blacklisted just for partition 0 + if (task.index == 0) { + assert(task.executorId === "executor3") + } else { + assert(Set("executor0", "executor3").contains(task.executorId)) + } + } + tasksForStage(1).foreach { task => + // executor 3 blacklisted + assert("executor3" != task.executorId) + } + // no restrictions on stage 2 + + // Finally, just make sure that we can still complete tasks as usual with blacklisting + // in effect. Finish each of the tasksets -- taskset 0 & 1 complete successfully, taskset 2 + // fails. + (0 to 2).foreach { stageId => + val tasks = tasksForStage(stageId) + val tsm = taskScheduler.taskSetManagerForAttempt(stageId, 0).get + val valueSer = SparkEnv.get.serializer.newInstance() + if (stageId == 2) { + // Just need to make one task fail 4 times. + var task = tasks(0) + val taskIndex = task.index + (0 until 4).foreach { attempt => + assert(task.attemptNumber === attempt) + tsm.handleFailedTask(task.taskId, TaskState.FAILED, TaskResultLost) + val nextAttempts = + taskScheduler.resourceOffers(IndexedSeq(WorkerOffer("executor4", "host4", 1))).flatten + if (attempt < 3) { + assert(nextAttempts.size === 1) + task = nextAttempts(0) + assert(task.index === taskIndex) + } else { + assert(nextAttempts.size === 0) + } + } + // End the other task of the taskset, doesn't matter whether it succeeds or fails. + val otherTask = tasks(1) + val result = new DirectTaskResult[Int](valueSer.serialize(otherTask.taskId), Seq()) + tsm.handleSuccessfulTask(otherTask.taskId, result) + } else { + tasks.foreach { task => + val result = new DirectTaskResult[Int](valueSer.serialize(task.taskId), Seq()) + tsm.handleSuccessfulTask(task.taskId, result) + } + } + assert(tsm.isZombie) + } + + // the tasksSets complete, so the tracker should be notified of the successful ones + verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet( + stageId = 0, + stageAttemptId = 0, + failuresByExec = stageToMockTaskSetBlacklist(0).execToFailures) + verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet( + stageId = 1, + stageAttemptId = 0, + failuresByExec = stageToMockTaskSetBlacklist(1).execToFailures) + // but we shouldn't update for the failed taskset + verify(blacklist, never).updateBlacklistForSuccessfulTaskSet( + stageId = meq(2), + stageAttemptId = anyInt(), + failuresByExec = anyObject()) } + test("scheduled tasks obey node and executor blacklists") { + taskScheduler = setupSchedulerWithMockTaskSetBlacklist() + (0 to 2).foreach { stageId => + val taskSet = FakeTask.createTaskSet(numTasks = 2, stageId = stageId, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet) + } + + val offers = IndexedSeq( + new WorkerOffer("executor0", "host0", 1), + new WorkerOffer("executor1", "host1", 1), + new WorkerOffer("executor2", "host1", 1), + new WorkerOffer("executor3", "host2", 10), + new WorkerOffer("executor4", "host3", 1) + ) + + // setup our mock blacklist: + // host1, executor0 & executor3 are completely blacklisted + // This covers everything *except* one core on executor4 / host3, so that everything is still + // schedulable. + when(blacklist.isNodeBlacklisted("host1")).thenReturn(true) + when(blacklist.isExecutorBlacklisted("executor0")).thenReturn(true) + when(blacklist.isExecutorBlacklisted("executor3")).thenReturn(true) + + val stageToTsm = (0 to 2).map { stageId => + val tsm = taskScheduler.taskSetManagerForAttempt(stageId, 0).get + stageId -> tsm + }.toMap + + val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten + firstTaskAttempts.foreach { task => logInfo(s"scheduled $task on ${task.executorId}") } + assert(firstTaskAttempts.size === 1) + assert(firstTaskAttempts.head.executorId === "executor4") + ('0' until '2').foreach { hostNum => + verify(blacklist, atLeast(1)).isNodeBlacklisted("host" + hostNum) + } + } + + test("abort stage when all executors are blacklisted") { + taskScheduler = setupSchedulerWithMockTaskSetBlacklist() + val taskSet = FakeTask.createTaskSet(numTasks = 10, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet) + val tsm = stageToMockTaskSetManager(0) + + // first just submit some offers so the scheduler knows about all the executors + taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 2), + WorkerOffer("executor1", "host0", 2), + WorkerOffer("executor2", "host0", 2), + WorkerOffer("executor3", "host1", 2) + )) + + // now say our blacklist updates to blacklist a bunch of resources, but *not* everything + when(blacklist.isNodeBlacklisted("host1")).thenReturn(true) + when(blacklist.isExecutorBlacklisted("executor0")).thenReturn(true) + + // make an offer on the blacklisted resources. We won't schedule anything, but also won't + // abort yet, since we know of other resources that work + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 2), + WorkerOffer("executor3", "host1", 2) + )).flatten.size === 0) + assert(!tsm.isZombie) + + // now update the blacklist so that everything really is blacklisted + when(blacklist.isExecutorBlacklisted("executor1")).thenReturn(true) + when(blacklist.isExecutorBlacklisted("executor2")).thenReturn(true) + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 2), + WorkerOffer("executor3", "host1", 2) + )).flatten.size === 0) + assert(tsm.isZombie) + verify(tsm).abort(anyString(), anyObject()) + } + + /** + * Helper for performance tests. Takes the explicitly blacklisted nodes and executors; verifies + * that the blacklists are used efficiently to ensure scheduling is not O(numPendingTasks). + * Creates 1 offer on executor[1-3]. Executor1 & 2 are on host1, executor3 is on host2. Passed + * in nodes and executors should be on that list. + */ + private def testBlacklistPerformance( + testName: String, + nodeBlacklist: Seq[String], + execBlacklist: Seq[String]): Unit = { + // Because scheduling involves shuffling the order of offers around, we run this test a few + // times to cover more possibilities. There are only 3 offers, which means 6 permutations, + // so 10 iterations is pretty good. + (0 until 10).foreach { testItr => + test(s"$testName: iteration $testItr") { + // When an executor or node is blacklisted, we want to make sure that we don't try + // scheduling each pending task, one by one, to discover they are all blacklisted. This is + // important for performance -- if we did check each task one-by-one, then responding to a + // resource offer (which is usually O(1)-ish) would become O(numPendingTasks), which would + // slow down scheduler throughput and slow down scheduling even on healthy executors. + // Here, we check a proxy for the runtime -- we make sure the scheduling is short-circuited + // at the node or executor blacklist, so we never check the per-task blacklist. We also + // make sure we don't check the node & executor blacklist for the entire taskset + // O(numPendingTasks) times. + + taskScheduler = setupSchedulerWithMockTaskSetBlacklist() + // we schedule 500 tasks so we can clearly distinguish anything that is O(numPendingTasks) + val taskSet = FakeTask.createTaskSet(numTasks = 500, stageId = 0, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet) + + val offers = IndexedSeq( + new WorkerOffer("executor1", "host1", 1), + new WorkerOffer("executor2", "host1", 1), + new WorkerOffer("executor3", "host2", 1) + ) + // We should check the node & exec blacklists, but only O(numOffers), not O(numPendingTasks) + // times. In the worst case, after shuffling, we offer our blacklisted resource first, and + // then offer other resources which do get used. The taskset blacklist is consulted + // repeatedly as we offer resources to the taskset -- each iteration either schedules + // something, or it terminates that locality level, so the maximum number of checks is + // numCores + numLocalityLevels + val numCoresOnAllOffers = offers.map(_.cores).sum + val numLocalityLevels = TaskLocality.values.size + val maxBlacklistChecks = numCoresOnAllOffers + numLocalityLevels + + // Setup the blacklist + nodeBlacklist.foreach { node => + when(stageToMockTaskSetBlacklist(0).isNodeBlacklistedForTaskSet(node)).thenReturn(true) + } + execBlacklist.foreach { exec => + when(stageToMockTaskSetBlacklist(0).isExecutorBlacklistedForTaskSet(exec)) + .thenReturn(true) + } + + // Figure out which nodes have any effective blacklisting on them. This means all nodes + // that are explicitly blacklisted, plus those that have *any* executors blacklisted. + val nodesForBlacklistedExecutors = offers.filter { offer => + execBlacklist.contains(offer.executorId) + }.map(_.host).toSet.toSeq + val nodesWithAnyBlacklisting = (nodeBlacklist ++ nodesForBlacklistedExecutors).toSet + // Similarly, figure out which executors have any blacklisting. This means all executors + // that are explicitly blacklisted, plus all executors on nodes that are blacklisted. + val execsForBlacklistedNodes = offers.filter { offer => + nodeBlacklist.contains(offer.host) + }.map(_.executorId).toSeq + val executorsWithAnyBlacklisting = (execBlacklist ++ execsForBlacklistedNodes).toSet + + // Schedule a taskset, and make sure our test setup is correct -- we are able to schedule + // a task on all executors that aren't blacklisted (whether that executor is a explicitly + // blacklisted, or implicitly blacklisted via the node blacklist). + val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten + assert(firstTaskAttempts.size === offers.size - executorsWithAnyBlacklisting.size) + + // Now check that we haven't made too many calls to any of the blacklist methods. + // We should be checking our node blacklist, but it should be within the bound we defined + // above. + verify(stageToMockTaskSetBlacklist(0), atMost(maxBlacklistChecks)) + .isNodeBlacklistedForTaskSet(anyString()) + // We shouldn't ever consult the per-task blacklist for the nodes that have been blacklisted + // for the entire taskset, since the taskset level blacklisting should prevent scheduling + // from ever looking at specific tasks. + nodesWithAnyBlacklisting.foreach { node => + verify(stageToMockTaskSetBlacklist(0), never) + .isNodeBlacklistedForTask(meq(node), anyInt()) + } + executorsWithAnyBlacklisting.foreach { exec => + // We should be checking our executor blacklist, but it should be within the bound defined + // above. Its possible that this will be significantly fewer calls, maybe even 0, if + // there is also a node-blacklist which takes effect first. But this assert is all we + // need to avoid an O(numPendingTask) slowdown. + verify(stageToMockTaskSetBlacklist(0), atMost(maxBlacklistChecks)) + .isExecutorBlacklistedForTaskSet(exec) + // We shouldn't ever consult the per-task blacklist for executors that have been + // blacklisted for the entire taskset, since the taskset level blacklisting should prevent + // scheduling from ever looking at specific tasks. + verify(stageToMockTaskSetBlacklist(0), never) + .isExecutorBlacklistedForTask(meq(exec), anyInt()) + } + } + } + } + + testBlacklistPerformance( + testName = "Blacklisted node for entire task set prevents per-task blacklist checks", + nodeBlacklist = Seq("host1"), + execBlacklist = Seq()) + + testBlacklistPerformance( + testName = "Blacklisted executor for entire task set prevents per-task blacklist checks", + nodeBlacklist = Seq(), + execBlacklist = Seq("executor3") + ) + + test("abort stage if executor loss results in unschedulability from previously failed tasks") { + // Make sure we can detect when a taskset becomes unschedulable from a blacklisting. This + // test explores a particular corner case -- you may have one task fail, but still be + // schedulable on another executor. However, that executor may fail later on, leaving the + // first task with no place to run. + val taskScheduler = setupScheduler( + config.BLACKLIST_ENABLED.key -> "true" + ) + + val taskSet = FakeTask.createTaskSet(2) + taskScheduler.submitTasks(taskSet) + val tsm = taskScheduler.taskSetManagerForAttempt(taskSet.stageId, taskSet.stageAttemptId).get + + val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq( + new WorkerOffer("executor0", "host0", 1), + new WorkerOffer("executor1", "host1", 1) + )).flatten + assert(Set("executor0", "executor1") === firstTaskAttempts.map(_.executorId).toSet) + + // Fail one of the tasks, but leave the other running. + val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get + taskScheduler.handleFailedTask(tsm, failedTask.taskId, TaskState.FAILED, TaskResultLost) + // At this point, our failed task could run on the other executor, so don't give up the task + // set yet. + assert(!failedTaskSet) + + // Now we fail our second executor. The other task can still run on executor1, so make an offer + // on that executor, and make sure that the other task (not the failed one) is assigned there. + taskScheduler.executorLost("executor1", SlaveLost("oops")) + val nextTaskAttempts = + taskScheduler.resourceOffers(IndexedSeq(new WorkerOffer("executor0", "host0", 1))).flatten + // Note: Its OK if some future change makes this already realize the taskset has become + // unschedulable at this point (though in the current implementation, we're sure it will not). + assert(nextTaskAttempts.size === 1) + assert(nextTaskAttempts.head.executorId === "executor0") + assert(nextTaskAttempts.head.attemptNumber === 1) + assert(nextTaskAttempts.head.index != failedTask.index) + + // Now we should definitely realize that our task set is unschedulable, because the only + // task left can't be scheduled on any executors due to the blacklist. + taskScheduler.resourceOffers(IndexedSeq(new WorkerOffer("executor0", "host0", 1))) + sc.listenerBus.waitUntilEmpty(100000) + assert(tsm.isZombie) + assert(failedTaskSet) + val idx = failedTask.index + assert(failedTaskSetReason === s"Aborting TaskSet 0.0 because task $idx (partition $idx) " + + s"cannot run anywhere due to node and executor blacklist. Blacklisting behavior can be " + + s"configured via spark.blacklist.*.") + } + + test("don't abort if there is an executor available, though it hasn't had scheduled tasks yet") { + // interaction of SPARK-15865 & SPARK-16106 + // if we have a small number of tasks, we might be able to schedule them all on the first + // executor. But if those tasks fail, we should still realize there is another executor + // available and not bail on the job + + val taskScheduler = setupScheduler( + config.BLACKLIST_ENABLED.key -> "true" + ) + + val taskSet = FakeTask.createTaskSet(2, (0 until 2).map { _ => Seq(TaskLocation("host0")) }: _*) + taskScheduler.submitTasks(taskSet) + val tsm = taskScheduler.taskSetManagerForAttempt(taskSet.stageId, taskSet.stageAttemptId).get + + val offers = IndexedSeq( + // each offer has more than enough free cores for the entire task set, so when combined + // with the locality preferences, we schedule all tasks on one executor + new WorkerOffer("executor0", "host0", 4), + new WorkerOffer("executor1", "host1", 4) + ) + val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten + assert(firstTaskAttempts.size == 2) + firstTaskAttempts.foreach { taskAttempt => assert("executor0" === taskAttempt.executorId) } + + // fail all the tasks on the bad executor + firstTaskAttempts.foreach { taskAttempt => + taskScheduler.handleFailedTask(tsm, taskAttempt.taskId, TaskState.FAILED, TaskResultLost) + } + + // Here is the main check of this test -- we have the same offers again, and we schedule it + // successfully. Because the scheduler first tries to schedule with locality in mind, at first + // it won't schedule anything on executor1. But despite that, we don't abort the job. Then the + // scheduler tries for ANY locality, and successfully schedules tasks on executor1. + val secondTaskAttempts = taskScheduler.resourceOffers(offers).flatten + assert(secondTaskAttempts.size == 2) + secondTaskAttempts.foreach { taskAttempt => assert("executor1" === taskAttempt.executorId) } + assert(!failedTaskSet) + } + + test("SPARK-16106 locality levels updated if executor added to existing host") { + val taskScheduler = setupScheduler() + + taskScheduler.submitTasks(FakeTask.createTaskSet(2, 0, + (0 until 2).map { _ => Seq(TaskLocation("host0", "executor2")) }: _* + )) + + val taskDescs = taskScheduler.resourceOffers(IndexedSeq( + new WorkerOffer("executor0", "host0", 1), + new WorkerOffer("executor1", "host1", 1) + )).flatten + // only schedule one task because of locality + assert(taskDescs.size === 1) + + val mgr = taskScheduler.taskIdToTaskSetManager.get(taskDescs(0).taskId).get + assert(mgr.myLocalityLevels.toSet === Set(TaskLocality.NODE_LOCAL, TaskLocality.ANY)) + // we should know about both executors, even though we only scheduled tasks on one of them + assert(taskScheduler.getExecutorsAliveOnHost("host0") === Some(Set("executor0"))) + assert(taskScheduler.getExecutorsAliveOnHost("host1") === Some(Set("executor1"))) + + // when executor2 is added, we should realize that we can run process-local tasks. + // And we should know its alive on the host. + val secondTaskDescs = taskScheduler.resourceOffers( + IndexedSeq(new WorkerOffer("executor2", "host0", 1))).flatten + assert(secondTaskDescs.size === 1) + assert(mgr.myLocalityLevels.toSet === + Set(TaskLocality.PROCESS_LOCAL, TaskLocality.NODE_LOCAL, TaskLocality.ANY)) + assert(taskScheduler.getExecutorsAliveOnHost("host0") === Some(Set("executor0", "executor2"))) + assert(taskScheduler.getExecutorsAliveOnHost("host1") === Some(Set("executor1"))) + + // And even if we don't have anything left to schedule, another resource offer on yet another + // executor should also update the set of live executors + val thirdTaskDescs = taskScheduler.resourceOffers( + IndexedSeq(new WorkerOffer("executor3", "host1", 1))).flatten + assert(thirdTaskDescs.size === 0) + assert(taskScheduler.getExecutorsAliveOnHost("host1") === Some(Set("executor1", "executor3"))) + } + + test("scheduler checks for executors that can be expired from blacklist") { + taskScheduler = setupScheduler() + + taskScheduler.submitTasks(FakeTask.createTaskSet(1, 0)) + taskScheduler.resourceOffers(IndexedSeq( + new WorkerOffer("executor0", "host0", 1) + )).flatten + + verify(blacklist).applyBlacklistTimeout() + } + + test("if an executor is lost then the state for its running tasks is cleaned up (SPARK-18553)") { + sc = new SparkContext("local", "TaskSchedulerImplSuite") + val taskScheduler = new TaskSchedulerImpl(sc) + taskScheduler.initialize(new FakeSchedulerBackend) + // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. + new DAGScheduler(sc, taskScheduler) { + override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} + override def executorAdded(execId: String, host: String) {} + } + + val e0Offers = IndexedSeq(WorkerOffer("executor0", "host0", 1)) + val attempt1 = FakeTask.createTaskSet(1) + + // submit attempt 1, offer resources, task gets scheduled + taskScheduler.submitTasks(attempt1) + val taskDescriptions = taskScheduler.resourceOffers(e0Offers).flatten + assert(1 === taskDescriptions.length) + + // mark executor0 as dead + taskScheduler.executorLost("executor0", SlaveLost()) + assert(!taskScheduler.isExecutorAlive("executor0")) + assert(!taskScheduler.hasExecutorsAliveOnHost("host0")) + assert(taskScheduler.getExecutorsAliveOnHost("host0").isEmpty) + + + // Check that state associated with the lost task attempt is cleaned up: + assert(taskScheduler.taskIdToExecutorId.isEmpty) + assert(taskScheduler.taskIdToTaskSetManager.isEmpty) + assert(taskScheduler.runningTasksByExecutors.get("executor0").isEmpty) + } + + test("if a task finishes with TaskState.LOST its executor is marked as dead") { + sc = new SparkContext("local", "TaskSchedulerImplSuite") + val taskScheduler = new TaskSchedulerImpl(sc) + taskScheduler.initialize(new FakeSchedulerBackend) + // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. + new DAGScheduler(sc, taskScheduler) { + override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} + override def executorAdded(execId: String, host: String) {} + } + + val e0Offers = IndexedSeq(WorkerOffer("executor0", "host0", 1)) + val attempt1 = FakeTask.createTaskSet(1) + + // submit attempt 1, offer resources, task gets scheduled + taskScheduler.submitTasks(attempt1) + val taskDescriptions = taskScheduler.resourceOffers(e0Offers).flatten + assert(1 === taskDescriptions.length) + + // Report the task as failed with TaskState.LOST + taskScheduler.statusUpdate( + tid = taskDescriptions.head.taskId, + state = TaskState.LOST, + serializedData = ByteBuffer.allocate(0) + ) + + // Check that state associated with the lost task attempt is cleaned up: + assert(taskScheduler.taskIdToExecutorId.isEmpty) + assert(taskScheduler.taskIdToTaskSetManager.isEmpty) + assert(taskScheduler.runningTasksByExecutors.get("executor0").isEmpty) + + // Check that the executor has been marked as dead + assert(!taskScheduler.isExecutorAlive("executor0")) + assert(!taskScheduler.hasExecutorsAliveOnHost("host0")) + assert(taskScheduler.getExecutorsAliveOnHost("host0").isEmpty) + } + + test("Locality should be used for bulk offers even with delay scheduling off") { + val conf = new SparkConf() + .set("spark.locality.wait", "0") + sc = new SparkContext("local", "TaskSchedulerImplSuite", conf) + // we create a manual clock just so we can be sure the clock doesn't advance at all in this test + val clock = new ManualClock() + + // We customize the task scheduler just to let us control the way offers are shuffled, so we + // can be sure we try both permutations, and to control the clock on the tasksetmanager. + val taskScheduler = new TaskSchedulerImpl(sc) { + override def shuffleOffers(offers: IndexedSeq[WorkerOffer]): IndexedSeq[WorkerOffer] = { + // Don't shuffle the offers around for this test. Instead, we'll just pass in all + // the permutations we care about directly. + offers + } + override def createTaskSetManager(taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { + new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt, clock) + } + } + // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. + new DAGScheduler(sc, taskScheduler) { + override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} + override def executorAdded(execId: String, host: String) {} + } + taskScheduler.initialize(new FakeSchedulerBackend) + + // Make two different offers -- one in the preferred location, one that is not. + val offers = IndexedSeq( + WorkerOffer("exec1", "host1", 1), + WorkerOffer("exec2", "host2", 1) + ) + Seq(false, true).foreach { swapOrder => + // Submit a taskset with locality preferences. + val taskSet = FakeTask.createTaskSet( + 1, stageId = 1, stageAttemptId = 0, Seq(TaskLocation("host1", "exec1"))) + taskScheduler.submitTasks(taskSet) + val shuffledOffers = if (swapOrder) offers.reverse else offers + // Regardless of the order of the offers (after the task scheduler shuffles them), we should + // always take advantage of the local offer. + val taskDescs = taskScheduler.resourceOffers(shuffledOffers).flatten + withClue(s"swapOrder = $swapOrder") { + assert(taskDescs.size === 1) + assert(taskDescs.head.executorId === "exec1") + } + } + } + + test("With delay scheduling off, tasks can be run at any locality level immediately") { + val conf = new SparkConf() + .set("spark.locality.wait", "0") + sc = new SparkContext("local", "TaskSchedulerImplSuite", conf) + + // we create a manual clock just so we can be sure the clock doesn't advance at all in this test + val clock = new ManualClock() + val taskScheduler = new TaskSchedulerImpl(sc) { + override def createTaskSetManager(taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { + new TaskSetManager(this, taskSet, maxTaskFailures, blacklistTrackerOpt, clock) + } + } + // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. + new DAGScheduler(sc, taskScheduler) { + override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} + override def executorAdded(execId: String, host: String) {} + } + taskScheduler.initialize(new FakeSchedulerBackend) + // make an offer on the preferred host so the scheduler knows its alive. This is necessary + // so that the taskset knows that it *could* take advantage of locality. + taskScheduler.resourceOffers(IndexedSeq(WorkerOffer("exec1", "host1", 1))) + + // Submit a taskset with locality preferences. + val taskSet = FakeTask.createTaskSet( + 1, stageId = 1, stageAttemptId = 0, Seq(TaskLocation("host1", "exec1"))) + taskScheduler.submitTasks(taskSet) + val tsm = taskScheduler.taskSetManagerForAttempt(1, 0).get + // make sure we've setup our test correctly, so that the taskset knows it *could* use local + // offers. + assert(tsm.myLocalityLevels.contains(TaskLocality.NODE_LOCAL)) + // make an offer on a non-preferred location. Since the delay is 0, we should still schedule + // immediately. + val taskDescs = + taskScheduler.resourceOffers(IndexedSeq(WorkerOffer("exec2", "host2", 1))).flatten + assert(taskDescs.size === 1) + assert(taskDescs.head.executorId === "exec2") + } + + test("TaskScheduler should throw IllegalArgumentException when schedulingMode is not supported") { + intercept[IllegalArgumentException] { + val taskScheduler = setupScheduler( + TaskSchedulerImpl.SCHEDULER_MODE_PROPERTY -> SchedulingMode.NONE.toString) + taskScheduler.initialize(new FakeSchedulerBackend) + } + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala new file mode 100644 index 000000000000..f1392e9db6bf --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.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.scheduler + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config +import org.apache.spark.util.{ManualClock, SystemClock} + +class TaskSetBlacklistSuite extends SparkFunSuite { + + test("Blacklisting tasks, executors, and nodes") { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set(config.BLACKLIST_ENABLED.key, "true") + val clock = new ManualClock + + val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, clock = clock) + clock.setTime(0) + // We will mark task 0 & 1 failed on both executor 1 & 2. + // We should blacklist all executors on that host, for all tasks for the stage. Note the API + // will return false for isExecutorBacklistedForTaskSet even when the node is blacklisted, so + // the executor is implicitly blacklisted (this makes sense with how the scheduler uses the + // blacklist) + + // First, mark task 0 as failed on exec1. + // task 0 should be blacklisted on exec1, and nowhere else + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 0) + for { + executor <- (1 to 4).map(_.toString) + index <- 0 until 10 + } { + val shouldBeBlacklisted = (executor == "exec1" && index == 0) + assert(taskSetBlacklist.isExecutorBlacklistedForTask(executor, index) === shouldBeBlacklisted) + } + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // Mark task 1 failed on exec1 -- this pushes the executor into the blacklist + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + // Mark one task as failed on exec2 -- not enough for any further blacklisting yet. + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec2", index = 0) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + // Mark another task as failed on exec2 -- now we blacklist exec2, which also leads to + // blacklisting the entire node. + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec2", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1")) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2")) + assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + // Make sure the blacklist has the correct per-task && per-executor responses, over a wider + // range of inputs. + for { + executor <- (1 to 4).map(e => s"exec$e") + index <- 0 until 10 + } { + withClue(s"exec = $executor; index = $index") { + val badExec = (executor == "exec1" || executor == "exec2") + val badIndex = (index == 0 || index == 1) + assert( + // this ignores whether the executor is blacklisted entirely for the taskset -- that is + // intentional, it keeps it fast and is sufficient for usage in the scheduler. + taskSetBlacklist.isExecutorBlacklistedForTask(executor, index) === (badExec && badIndex)) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet(executor) === badExec) + } + } + assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + val execToFailures = taskSetBlacklist.execToFailures + assert(execToFailures.keySet === Set("exec1", "exec2")) + + Seq("exec1", "exec2").foreach { exec => + assert( + execToFailures(exec).taskToFailureCountAndFailureTime === Map( + 0 -> ((1, 0)), + 1 -> ((1, 0)) + ) + ) + } + } + + test("multiple attempts for the same task count once") { + // Make sure that for blacklisting tasks, the node counts task attempts, not executors. But for + // stage-level blacklisting, we count unique tasks. The reason for this difference is, with + // task-attempt blacklisting, we want to make it easy to configure so that you ensure a node + // is blacklisted before the taskset is completely aborted because of spark.task.maxFailures. + // But with stage-blacklisting, we want to make sure we're not just counting one bad task + // that has failed many times. + + val conf = new SparkConf().setMaster("local").setAppName("test") + .set(config.MAX_TASK_ATTEMPTS_PER_EXECUTOR, 2) + .set(config.MAX_TASK_ATTEMPTS_PER_NODE, 3) + .set(config.MAX_FAILURES_PER_EXEC_STAGE, 2) + .set(config.MAX_FAILED_EXEC_PER_NODE_STAGE, 3) + val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, new SystemClock()) + // Fail a task twice on hostA, exec:1 + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + assert(taskSetBlacklist.isExecutorBlacklistedForTask("1", 0)) + assert(!taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0)) + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // Fail the same task once more on hostA, exec:2 + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "2", index = 0) + assert(taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0)) + assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // Fail another task on hostA, exec:1. Now that executor has failures on two different tasks, + // so its blacklisted + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // Fail a third task on hostA, exec:2, so that exec is blacklisted for the whole task set + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "2", index = 2) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + // Fail a fourth & fifth task on hostA, exec:3. Now we've got three executors that are + // blacklisted for the taskset, so blacklist the whole node. + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "3", index = 3) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "3", index = 4) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("3")) + assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + } + + test("only blacklist nodes for the task set when all the blacklisted executors are all on " + + "same host") { + // we blacklist executors on two different hosts within one taskSet -- make sure that doesn't + // lead to any node blacklisting + val conf = new SparkConf().setAppName("test").setMaster("local") + .set(config.BLACKLIST_ENABLED.key, "true") + val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, new SystemClock()) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0) + taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + + taskSetBlacklist.updateBlacklistForFailedTask("hostB", exec = "2", index = 0) + taskSetBlacklist.updateBlacklistForFailedTask("hostB", exec = "2", index = 1) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1")) + assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA")) + assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostB")) + } + +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index ecc18fc6e15b..ae43f4cadc03 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -17,15 +17,22 @@ package org.apache.spark.scheduler -import java.util.Random +import java.util.{Properties, Random} -import scala.collection.Map import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.mockito.Matchers.{any, anyInt, anyString} +import org.mockito.Mockito.{mock, never, spy, times, verify, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer + import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.ManualClock +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config +import org.apache.spark.serializer.SerializerInstance +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.{AccumulatorV2, ManualClock, Utils} class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) extends DAGScheduler(sc) { @@ -38,15 +45,14 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) task: Task[_], reason: TaskEndReason, result: Any, - accumUpdates: Map[Long, Any], - taskInfo: TaskInfo, - taskMetrics: TaskMetrics) { + accumUpdates: Seq[AccumulatorV2[_, _]], + taskInfo: TaskInfo) { taskScheduler.endedTasks(taskInfo.index) = reason } override def executorAdded(execId: String, host: String) {} - override def executorLost(execId: String) {} + override def executorLost(execId: String, reason: ExecutorLossReason) {} override def taskSetFailed( taskSet: TaskSet, @@ -54,6 +60,10 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) exception: Option[Throwable]): Unit = { taskScheduler.taskSetsFailed += taskSet.id } + + override def speculativeTaskSubmitted(task: Task[_]): Unit = { + taskScheduler.speculativeTasks += task.partitionId + } } // Get the rack for a given host @@ -86,6 +96,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val endedTasks = new mutable.HashMap[Long, TaskEndReason] val finishedManagers = new ArrayBuffer[TaskSetManager] val taskSetsFailed = new ArrayBuffer[String] + val speculativeTasks = new ArrayBuffer[Int] val executors = new mutable.HashMap[String, String] for ((execId, host) <- liveExecutors) { @@ -103,7 +114,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val host = executorIdToHost.get(execId) assert(host != None) val hostId = host.get - val executorsOnHost = executorsByHost(hostId) + val executorsOnHost = hostToExecutors(hostId) executorsOnHost -= execId for (rack <- getRackForHost(hostId); hosts <- hostsByRack.get(rack)) { hosts -= hostId @@ -125,7 +136,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex def addExecutor(execId: String, host: String) { executors.put(execId, host) - val executorsOnHost = executorsByHost.getOrElseUpdate(host, new mutable.HashSet[String]) + val executorsOnHost = hostToExecutors.getOrElseUpdate(host, new mutable.HashSet[String]) executorsOnHost += execId executorIdToHost += execId -> host for (rack <- getRackForHost(host)) { @@ -133,13 +144,15 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex } } + override def getRackForHost(value: String): Option[String] = FakeRackUtil.getRackForHost(value) } /** * A Task implementation that results in a large serialized task. */ -class LargeTask(stageId: Int) extends Task[Array[Byte]](stageId, 0, 0, Seq.empty) { +class LargeTask(stageId: Int) extends Task[Array[Byte]](stageId, 0, 0) { + val randomBuffer = new Array[Byte](TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024) val random = new Random(0) random.nextBytes(randomBuffer) @@ -156,38 +169,56 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val LOCALITY_WAIT_MS = conf.getTimeAsMs("spark.locality.wait", "3s") val MAX_TASK_FAILURES = 4 - override def beforeEach() { + var sched: FakeTaskScheduler = null + + override def beforeEach(): Unit = { super.beforeEach() FakeRackUtil.cleanUp() + sched = null } + override def afterEach(): Unit = { + super.afterEach() + if (sched != null) { + sched.dagScheduler.stop() + sched.stop() + sched = null + } + } + + test("TaskSet with no preferences") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val accumUpdates = taskSet.tasks.head.metrics.internalAccums // Offer a host with NO_PREF as the constraint, // we should get a nopref task immediately since that's what we only have - var taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption.isDefined) + clock.advance(1) // Tell it the task has finished - manager.handleSuccessfulTask(0, createTaskResult(0)) + manager.handleSuccessfulTask(0, createTaskResult(0, accumUpdates)) assert(sched.endedTasks(0) === Success) assert(sched.finishedManagers.contains(manager)) } test("multiple offers with no preferences") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(3) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) + val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => + task.metrics.internalAccums + } // First three offers should all find tasks for (i <- 0 until 3) { - var taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === "exec1") @@ -198,24 +229,24 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.resourceOffer("exec1", "host1", NO_PREF) === None) // Finish the first two tasks - manager.handleSuccessfulTask(0, createTaskResult(0)) - manager.handleSuccessfulTask(1, createTaskResult(1)) + manager.handleSuccessfulTask(0, createTaskResult(0, accumUpdatesByTask(0))) + manager.handleSuccessfulTask(1, createTaskResult(1, accumUpdatesByTask(1))) assert(sched.endedTasks(0) === Success) assert(sched.endedTasks(1) === Success) assert(!sched.finishedManagers.contains(manager)) // Finish the last task - manager.handleSuccessfulTask(2, createTaskResult(2)) + manager.handleSuccessfulTask(2, createTaskResult(2, accumUpdatesByTask(2))) assert(sched.endedTasks(2) === Success) assert(sched.finishedManagers.contains(manager)) } test("skip unsatisfiable locality levels") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execC", "host2")) + sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execC", "host2")) val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "execB"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // An executor that is not NODE_LOCAL should be rejected. assert(manager.resourceOffer("execC", "host2", ANY) === None) @@ -228,7 +259,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("basic delay scheduling") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) val taskSet = FakeTask.createTaskSet(4, Seq(TaskLocation("host1", "exec1")), Seq(TaskLocation("host2", "exec2")), @@ -236,20 +267,20 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq() // Last task has no locality prefs ) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1, exec1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) == None) clock.advance(LOCALITY_WAIT_MS) - // Offer host1, exec1 again, at NODE_LOCAL level: the node local (task 2) should + // Offer host1, exec1 again, at NODE_LOCAL level: the node local (task 3) should // get chosen before the noPref task assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).get.index == 2) - // Offer host2, exec3 again, at NODE_LOCAL level: we should choose task 2 + // Offer host2, exec2, at NODE_LOCAL level: we should choose task 2 assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL).get.index == 1) - // Offer host2, exec3 again, at NODE_LOCAL level: we should get noPref task + // Offer host2, exec2 again, at NODE_LOCAL level: we should get noPref task // after failing to find a node_Local task assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL) == None) clock.advance(LOCALITY_WAIT_MS) @@ -258,14 +289,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("we do not need to delay scheduling when we only have noPref tasks in the queue") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec3", "host2")) + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec3", "host2")) val taskSet = FakeTask.createTaskSet(3, Seq(TaskLocation("host1", "exec1")), Seq(TaskLocation("host2", "exec3")), Seq() // Last task has no locality prefs ) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1, exec1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).get.index === 0) assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL).get.index === 1) @@ -275,7 +306,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("delay scheduling with fallback") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"), ("exec3", "host3")) val taskSet = FakeTask.createTaskSet(5, Seq(TaskLocation("host1")), @@ -285,7 +316,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host2")) ) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) @@ -315,7 +346,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("delay scheduling with failed hosts") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"), + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"), ("exec3", "host3")) val taskSet = FakeTask.createTaskSet(3, Seq(TaskLocation("host1")), @@ -323,7 +354,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(TaskLocation("host3")) ) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) @@ -352,10 +383,11 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("task result lost") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + clock.advance(1) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) @@ -369,10 +401,11 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("repeated failures lead to task set abortion") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + clock.advance(1) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted // after the last failure. @@ -393,18 +426,24 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("executors should be blacklisted after task failure, in spite of locality preferences") { val rescheduleDelay = 300L val conf = new SparkConf(). - set("spark.scheduler.executorTaskBlacklistTime", rescheduleDelay.toString). - // dont wait to jump locality levels in this test + set(config.BLACKLIST_ENABLED, true). + set(config.BLACKLIST_TIMEOUT_CONF, rescheduleDelay). + // don't wait to jump locality levels in this test set("spark.locality.wait", "0") sc = new SparkContext("local", "test", conf) // two executors on same host, one on different. - val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec1.1", "host1"), ("exec2", "host2")) // affinity to exec1 on host1 - which we will fail. val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "exec1"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, 4, clock) + clock.advance(1) + // We don't directly use the application blacklist, but its presence triggers blacklisting + // within the taskset. + val mockListenerBus = mock(classOf[LiveListenerBus]) + val blacklistTrackerOpt = Some(new BlacklistTracker(mockListenerBus, conf, None, clock)) + val manager = new TaskSetManager(sched, taskSet, 4, blacklistTrackerOpt, clock) { val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) @@ -457,19 +496,24 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.resourceOffer("exec2", "host2", ANY).isEmpty) } - // After reschedule delay, scheduling on exec1 should be possible. + // Despite advancing beyond the time for expiring executors from within the blacklist, + // we *never* expire from *within* the stage blacklist clock.advance(rescheduleDelay) { val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) - assert(offerResult.isDefined, "Expect resource offer to return a task") + assert(offerResult.isEmpty) + } + { + val offerResult = manager.resourceOffer("exec3", "host3", ANY) + assert(offerResult.isDefined) assert(offerResult.get.index === 0) - assert(offerResult.get.executorId === "exec1") + assert(offerResult.get.executorId === "exec3") - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).isEmpty) + assert(manager.resourceOffer("exec3", "host3", ANY).isEmpty) - // Cause exec1 to fail : failure 4 + // Cause exec3 to fail : failure 4 manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) } @@ -481,14 +525,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Assign host2 to rack2 FakeRackUtil.assignHostToRack("host2", "rack2") sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc) + sched = new FakeTaskScheduler(sc) val taskSet = FakeTask.createTaskSet(4, Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host1", "execB")), Seq(TaskLocation("host2", "execC")), Seq()) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // Only ANY is valid assert(manager.myLocalityLevels.sameElements(Array(NO_PREF, ANY))) // Add a new executor @@ -513,13 +557,15 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("Executors exit for reason unrelated to currently running tasks") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc) + sched = new FakeTaskScheduler(sc) val taskSet = FakeTask.createTaskSet(4, Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host1", "execB")), Seq(TaskLocation("host2", "execC")), Seq()) - val manager = new TaskSetManager(sched, taskSet, 1, new ManualClock) + val clock = new ManualClock() + clock.advance(1) + val manager = new TaskSetManager(sched, taskSet, 1, clock = clock) sched.addExecutor("execA", "host1") manager.executorAdded() sched.addExecutor("execC", "host2") @@ -546,13 +592,13 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Assign host3 to rack2 FakeRackUtil.assignHostToRack("host3", "rack2") sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, + sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) val taskSet = FakeTask.createTaskSet(2, Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host1", "execA"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY))) // Set allowed locality to ANY @@ -569,7 +615,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("do not emit warning when serialized task is small") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) @@ -582,7 +628,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("emit warning when serialized task is large") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = new TaskSet(Array(new LargeTask(0)), 0, 0, 0, null) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) @@ -596,7 +642,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("Not serializable exception thrown if the task cannot be serialized") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = new TaskSet( Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) @@ -620,7 +666,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // multiple 1k result val r = sc.makeRDD(0 until 10, 10).map(genBytes(1024)).collect() - assert(10 === r.size ) + assert(10 === r.size) // single 10M result val thrown = intercept[SparkException] {sc.makeRDD(genBytes(10 << 20)(0), 1).collect()} @@ -633,9 +679,74 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(thrown2.getMessage().contains("bigger than spark.driver.maxResultSize")) } + test("[SPARK-13931] taskSetManager should not send Resubmitted tasks after being a zombie") { + val conf = new SparkConf().set("spark.speculation", "true") + sc = new SparkContext("local", "test", conf) + + val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2")) + sched.initialize(new FakeSchedulerBackend() { + override def killTask( + taskId: Long, + executorId: String, + interruptThread: Boolean, + reason: String): Unit = {} + }) + + // Keep track of the number of tasks that are resubmitted, + // so that the test can check that no tasks were resubmitted. + var resubmittedTasks = 0 + val dagScheduler = new FakeDAGScheduler(sc, sched) { + override def taskEnded( + task: Task[_], + reason: TaskEndReason, + result: Any, + accumUpdates: Seq[AccumulatorV2[_, _]], + taskInfo: TaskInfo): Unit = { + super.taskEnded(task, reason, result, accumUpdates, taskInfo) + reason match { + case Resubmitted => resubmittedTasks += 1 + case _ => + } + } + } + sched.setDAGScheduler(dagScheduler) + + val singleTask = new ShuffleMapTask(0, 0, null, new Partition { + override def index: Int = 0 + }, Seq(TaskLocation("host1", "execA")), new Properties, null) + val taskSet = new TaskSet(Array(singleTask), 0, 0, 0, null) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) + + // Offer host1, which should be accepted as a PROCESS_LOCAL location + // by the one task in the task set + val task1 = manager.resourceOffer("execA", "host1", TaskLocality.PROCESS_LOCAL).get + + // Mark the task as available for speculation, and then offer another resource, + // which should be used to launch a speculative copy of the task. + manager.speculatableTasks += singleTask.partitionId + val task2 = manager.resourceOffer("execB", "host2", TaskLocality.ANY).get + + assert(manager.runningTasks === 2) + assert(manager.isZombie === false) + + val directTaskResult = new DirectTaskResult[String](null, Seq()) { + override def value(resultSer: SerializerInstance): String = "" + } + // Complete one copy of the task, which should result in the task set manager + // being marked as a zombie, because at least one copy of its only task has completed. + manager.handleSuccessfulTask(task1.taskId, directTaskResult) + assert(manager.isZombie === true) + assert(resubmittedTasks === 0) + assert(manager.runningTasks === 1) + + manager.executorLost("execB", "host2", new SlaveLost()) + assert(manager.runningTasks === 0) + assert(resubmittedTasks === 0) + } + test("speculative and noPref task should be scheduled after node-local") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler( + sched = new FakeTaskScheduler( sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) val taskSet = FakeTask.createTaskSet(4, Seq(TaskLocation("host1", "execA")), @@ -643,7 +754,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(), Seq(TaskLocation("host3", "execC"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 0) assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) @@ -663,7 +774,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("node-local tasks should be scheduled right away " + "when there are only node-local and no-preference tasks") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler( + sched = new FakeTaskScheduler( sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) val taskSet = FakeTask.createTaskSet(4, Seq(TaskLocation("host1")), @@ -671,7 +782,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg Seq(), Seq(TaskLocation("host3"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // node-local tasks are scheduled without delay assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0) @@ -686,14 +797,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("SPARK-4939: node-local tasks should be scheduled right after process-local tasks finished") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2")) + sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2")) val taskSet = FakeTask.createTaskSet(4, Seq(TaskLocation("host1")), Seq(TaskLocation("host2")), Seq(ExecutorCacheTaskLocation("host1", "execA")), Seq(ExecutorCacheTaskLocation("host2", "execB"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // process-local tasks are scheduled first assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 2) @@ -707,13 +818,13 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("SPARK-4939: no-pref tasks should be scheduled after process-local tasks finished") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2")) + sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2")) val taskSet = FakeTask.createTaskSet(3, Seq(), Seq(ExecutorCacheTaskLocation("host1", "execA")), Seq(ExecutorCacheTaskLocation("host2", "execB"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // process-local tasks are scheduled first assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 1) @@ -728,12 +839,12 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("Ensure TaskSetManager is usable after addition of levels") { // Regression test for SPARK-2931 sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc) + sched = new FakeTaskScheduler(sc) val taskSet = FakeTask.createTaskSet(2, Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host2", "execB.1"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // Only ANY is valid assert(manager.myLocalityLevels.sameElements(Array(ANY))) // Add a new executor @@ -760,14 +871,14 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("Test that locations with HDFSCacheTaskLocation are treated as PROCESS_LOCAL.") { // Regression test for SPARK-2931 sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, - ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + sched = new FakeTaskScheduler(sc, + ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) val taskSet = FakeTask.createTaskSet(3, Seq(TaskLocation("host1")), Seq(TaskLocation("host2")), Seq(TaskLocation("hdfs_cache_host3"))) val clock = new ManualClock - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) sched.removeExecutor("execA") manager.executorAdded() @@ -784,10 +895,370 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(TaskLocation("host1") === HostTaskLocation("host1")) assert(TaskLocation("hdfs_cache_host1") === HDFSCacheTaskLocation("host1")) assert(TaskLocation("executor_host1_3") === ExecutorCacheTaskLocation("host1", "3")) + assert(TaskLocation("executor_some.host1_executor_task_3") === + ExecutorCacheTaskLocation("some.host1", "executor_task_3")) + } + + test("Kill other task attempts when one attempt belonging to the same task succeeds") { + sc = new SparkContext("local", "test") + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + val taskSet = FakeTask.createTaskSet(4) + // Set the speculation multiplier to be 0 so speculative tasks are launched immediately + sc.conf.set("spark.speculation.multiplier", "0.0") + sc.conf.set("spark.speculation", "true") + val clock = new ManualClock() + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => + task.metrics.internalAccums + } + // Offer resources for 4 tasks to start + for ((k, v) <- List( + "exec1" -> "host1", + "exec1" -> "host1", + "exec2" -> "host2", + "exec2" -> "host2")) { + val taskOption = manager.resourceOffer(k, v, NO_PREF) + assert(taskOption.isDefined) + val task = taskOption.get + assert(task.executorId === k) + } + assert(sched.startedTasks.toSet === Set(0, 1, 2, 3)) + clock.advance(1) + // Complete the 3 tasks and leave 1 task in running + for (id <- Set(0, 1, 2)) { + manager.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id))) + assert(sched.endedTasks(id) === Success) + } + + // checkSpeculatableTasks checks that the task runtime is greater than the threshold for + // speculating. Since we use a threshold of 0 for speculation, tasks need to be running for + // > 0ms, so advance the clock by 1ms here. + clock.advance(1) + assert(manager.checkSpeculatableTasks(0)) + assert(sched.speculativeTasks.toSet === Set(3)) + + // Offer resource to start the speculative attempt for the running task + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) + assert(taskOption5.isDefined) + val task5 = taskOption5.get + assert(task5.index === 3) + assert(task5.taskId === 4) + assert(task5.executorId === "exec1") + assert(task5.attemptNumber === 1) + sched.backend = mock(classOf[SchedulerBackend]) + // Complete the speculative attempt for the running task + manager.handleSuccessfulTask(4, createTaskResult(3, accumUpdatesByTask(3))) + // Verify that it kills other running attempt + verify(sched.backend).killTask(3, "exec2", true, "another attempt succeeded") + // Because the SchedulerBackend was a mock, the 2nd copy of the task won't actually be + // killed, so the FakeTaskScheduler is only told about the successful completion + // of the speculated task. + assert(sched.endedTasks(3) === Success) + } + + test("Killing speculative tasks does not count towards aborting the taskset") { + sc = new SparkContext("local", "test") + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + val taskSet = FakeTask.createTaskSet(5) + // Set the speculation multiplier to be 0 so speculative tasks are launched immediately + sc.conf.set("spark.speculation.multiplier", "0.0") + sc.conf.set("spark.speculation.quantile", "0.6") + sc.conf.set("spark.speculation", "true") + val clock = new ManualClock() + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => + task.metrics.internalAccums + } + // Offer resources for 5 tasks to start + val tasks = new ArrayBuffer[TaskDescription]() + for ((k, v) <- List( + "exec1" -> "host1", + "exec1" -> "host1", + "exec1" -> "host1", + "exec2" -> "host2", + "exec2" -> "host2")) { + val taskOption = manager.resourceOffer(k, v, NO_PREF) + assert(taskOption.isDefined) + val task = taskOption.get + assert(task.executorId === k) + tasks += task + } + assert(sched.startedTasks.toSet === (0 until 5).toSet) + clock.advance(1) + // Complete 3 tasks and leave 2 tasks in running + for (id <- Set(0, 1, 2)) { + manager.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id))) + assert(sched.endedTasks(id) === Success) + } + + def runningTaskForIndex(index: Int): TaskDescription = { + tasks.find { task => + task.index == index && !sched.endedTasks.contains(task.taskId) + }.getOrElse { + throw new RuntimeException(s"couldn't find index $index in " + + s"tasks: ${tasks.map { t => t.index -> t.taskId }} with endedTasks:" + + s" ${sched.endedTasks.keys}") + } + } + + // have each of the running tasks fail 3 times (not enough to abort the stage) + (0 until 3).foreach { attempt => + Seq(3, 4).foreach { index => + val task = runningTaskForIndex(index) + logInfo(s"failing task $task") + val endReason = ExceptionFailure("a", "b", Array(), "c", None) + manager.handleFailedTask(task.taskId, TaskState.FAILED, endReason) + sched.endedTasks(task.taskId) = endReason + assert(!manager.isZombie) + val nextTask = manager.resourceOffer(s"exec2", s"host2", NO_PREF) + assert(nextTask.isDefined, s"no offer for attempt $attempt of $index") + tasks += nextTask.get + } + } + + // we can't be sure which one of our running tasks will get another speculative copy + val originalTasks = Seq(3, 4).map { index => index -> runningTaskForIndex(index) }.toMap + + // checkSpeculatableTasks checks that the task runtime is greater than the threshold for + // speculating. Since we use a threshold of 0 for speculation, tasks need to be running for + // > 0ms, so advance the clock by 1ms here. + clock.advance(1) + assert(manager.checkSpeculatableTasks(0)) + assert(sched.speculativeTasks.toSet === Set(3, 4)) + // Offer resource to start the speculative attempt for the running task + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) + assert(taskOption5.isDefined) + val speculativeTask = taskOption5.get + assert(speculativeTask.index === 3 || speculativeTask.index === 4) + assert(speculativeTask.taskId === 11) + assert(speculativeTask.executorId === "exec1") + assert(speculativeTask.attemptNumber === 4) + sched.backend = mock(classOf[SchedulerBackend]) + // Complete the speculative attempt for the running task + manager.handleSuccessfulTask(speculativeTask.taskId, createTaskResult(3, accumUpdatesByTask(3))) + // Verify that it kills other running attempt + val origTask = originalTasks(speculativeTask.index) + verify(sched.backend).killTask(origTask.taskId, "exec2", true, "another attempt succeeded") + // Because the SchedulerBackend was a mock, the 2nd copy of the task won't actually be + // killed, so the FakeTaskScheduler is only told about the successful completion + // of the speculated task. + assert(sched.endedTasks(3) === Success) + // also because the scheduler is a mock, our manager isn't notified about the task killed event, + // so we do that manually + manager.handleFailedTask(origTask.taskId, TaskState.KILLED, TaskKilled("test")) + // this task has "failed" 4 times, but one of them doesn't count, so keep running the stage + assert(manager.tasksSuccessful === 4) + assert(!manager.isZombie) + + // now run another speculative task + val taskOpt6 = manager.resourceOffer("exec1", "host1", NO_PREF) + assert(taskOpt6.isDefined) + val speculativeTask2 = taskOpt6.get + assert(speculativeTask2.index === 3 || speculativeTask2.index === 4) + assert(speculativeTask2.index !== speculativeTask.index) + assert(speculativeTask2.attemptNumber === 4) + // Complete the speculative attempt for the running task + manager.handleSuccessfulTask(speculativeTask2.taskId, + createTaskResult(3, accumUpdatesByTask(3))) + // Verify that it kills other running attempt + val origTask2 = originalTasks(speculativeTask2.index) + verify(sched.backend).killTask(origTask2.taskId, "exec2", true, "another attempt succeeded") + assert(manager.tasksSuccessful === 5) + assert(manager.isZombie) + } + + + test("SPARK-19868: DagScheduler only notified of taskEnd when state is ready") { + // dagScheduler.taskEnded() is async, so it may *seem* ok to call it before we've set all + // appropriate state, eg. isZombie. However, this sets up a race that could go the wrong way. + // This is a super-focused regression test which checks the zombie state as soon as + // dagScheduler.taskEnded() is called, to ensure we haven't introduced a race. + sc = new SparkContext("local", "test") + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + val mockDAGScheduler = mock(classOf[DAGScheduler]) + sched.dagScheduler = mockDAGScheduler + val taskSet = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = new ManualClock(1)) + when(mockDAGScheduler.taskEnded(any(), any(), any(), any(), any())).thenAnswer( + new Answer[Unit] { + override def answer(invocationOnMock: InvocationOnMock): Unit = { + assert(manager.isZombie) + } + }) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) + assert(taskOption.isDefined) + // this would fail, inside our mock dag scheduler, if it calls dagScheduler.taskEnded() too soon + manager.handleSuccessfulTask(0, createTaskResult(0)) + } + + test("SPARK-17894: Verify TaskSetManagers for different stage attempts have unique names") { + sc = new SparkContext("local", "test") + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + val taskSet = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = new ManualClock) + assert(manager.name === "TaskSet_0.0") + + // Make sure a task set with the same stage ID but different attempt ID has a unique name + val taskSet2 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 1) + val manager2 = new TaskSetManager(sched, taskSet2, MAX_TASK_FAILURES, clock = new ManualClock) + assert(manager2.name === "TaskSet_0.1") + + // Make sure a task set with the same attempt ID but different stage ID also has a unique name + val taskSet3 = FakeTask.createTaskSet(numTasks = 1, stageId = 1, stageAttemptId = 1) + val manager3 = new TaskSetManager(sched, taskSet3, MAX_TASK_FAILURES, clock = new ManualClock) + assert(manager3.name === "TaskSet_1.1") + } + + test("don't update blacklist for shuffle-fetch failures, preemption, denied commits, " + + "or killed tasks") { + // Setup a taskset, and fail some tasks for a fetch failure, preemption, denied commit, + // and killed task. + val conf = new SparkConf(). + set(config.BLACKLIST_ENABLED, true) + sc = new SparkContext("local", "test", conf) + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + val taskSet = FakeTask.createTaskSet(4) + val tsm = new TaskSetManager(sched, taskSet, 4) + // we need a spy so we can attach our mock blacklist + val tsmSpy = spy(tsm) + val blacklist = mock(classOf[TaskSetBlacklist]) + when(tsmSpy.taskSetBlacklistHelperOpt).thenReturn(Some(blacklist)) + + // make some offers to our taskset, to get tasks we will fail + val taskDescs = Seq( + "exec1" -> "host1", + "exec2" -> "host1" + ).flatMap { case (exec, host) => + // offer each executor twice (simulating 2 cores per executor) + (0 until 2).flatMap{ _ => tsmSpy.resourceOffer(exec, host, TaskLocality.ANY)} + } + assert(taskDescs.size === 4) + + // now fail those tasks + tsmSpy.handleFailedTask(taskDescs(0).taskId, TaskState.FAILED, + FetchFailed(BlockManagerId(taskDescs(0).executorId, "host1", 12345), 0, 0, 0, "ignored")) + tsmSpy.handleFailedTask(taskDescs(1).taskId, TaskState.FAILED, + ExecutorLostFailure(taskDescs(1).executorId, exitCausedByApp = false, reason = None)) + tsmSpy.handleFailedTask(taskDescs(2).taskId, TaskState.FAILED, + TaskCommitDenied(0, 2, 0)) + tsmSpy.handleFailedTask(taskDescs(3).taskId, TaskState.KILLED, TaskKilled("test")) + + // Make sure that the blacklist ignored all of the task failures above, since they aren't + // the fault of the executor where the task was running. + verify(blacklist, never()) + .updateBlacklistForFailedTask(anyString(), anyString(), anyInt()) + } + + test("update application blacklist for shuffle-fetch") { + // Setup a taskset, and fail some one task for fetch failure. + val conf = new SparkConf() + .set(config.BLACKLIST_ENABLED, true) + .set(config.SHUFFLE_SERVICE_ENABLED, true) + .set(config.BLACKLIST_FETCH_FAILURE_ENABLED, true) + sc = new SparkContext("local", "test", conf) + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + val taskSet = FakeTask.createTaskSet(4) + val blacklistTracker = new BlacklistTracker(sc, None) + val tsm = new TaskSetManager(sched, taskSet, 4, Some(blacklistTracker)) + + // make some offers to our taskset, to get tasks we will fail + val taskDescs = Seq( + "exec1" -> "host1", + "exec2" -> "host2" + ).flatMap { case (exec, host) => + // offer each executor twice (simulating 2 cores per executor) + (0 until 2).flatMap{ _ => tsm.resourceOffer(exec, host, TaskLocality.ANY)} + } + assert(taskDescs.size === 4) + + assert(!blacklistTracker.isExecutorBlacklisted(taskDescs(0).executorId)) + assert(!blacklistTracker.isNodeBlacklisted("host1")) + + // Fail the task with fetch failure + tsm.handleFailedTask(taskDescs(0).taskId, TaskState.FAILED, + FetchFailed(BlockManagerId(taskDescs(0).executorId, "host1", 12345), 0, 0, 0, "ignored")) + + assert(blacklistTracker.isNodeBlacklisted("host1")) + } + + test("update blacklist before adding pending task to avoid race condition") { + // When a task fails, it should apply the blacklist policy prior to + // retrying the task otherwise there's a race condition where run on + // the same executor that it was intended to be black listed from. + val conf = new SparkConf(). + set(config.BLACKLIST_ENABLED, true) + + // Create a task with two executors. + sc = new SparkContext("local", "test", conf) + val exec = "executor1" + val host = "host1" + val exec2 = "executor2" + val host2 = "host2" + sched = new FakeTaskScheduler(sc, (exec, host), (exec2, host2)) + val taskSet = FakeTask.createTaskSet(1) + + val clock = new ManualClock + val mockListenerBus = mock(classOf[LiveListenerBus]) + val blacklistTracker = new BlacklistTracker(mockListenerBus, conf, None, clock) + val taskSetManager = new TaskSetManager(sched, taskSet, 1, Some(blacklistTracker)) + val taskSetManagerSpy = spy(taskSetManager) + + val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY) + + // Assert the task has been black listed on the executor it was last executed on. + when(taskSetManagerSpy.addPendingTask(anyInt())).thenAnswer( + new Answer[Unit] { + override def answer(invocationOnMock: InvocationOnMock): Unit = { + val task = invocationOnMock.getArgumentAt(0, classOf[Int]) + assert(taskSetManager.taskSetBlacklistHelperOpt.get. + isExecutorBlacklistedForTask(exec, task)) + } + } + ) + + // Simulate a fake exception + val e = new ExceptionFailure("a", "b", Array(), "c", None) + taskSetManagerSpy.handleFailedTask(taskDesc.get.taskId, TaskState.FAILED, e) + + verify(taskSetManagerSpy, times(1)).addPendingTask(anyInt()) + } + + test("SPARK-21563 context's added jars shouldn't change mid-TaskSet") { + sc = new SparkContext("local", "test") + val addedJarsPreTaskSet = Map[String, Long](sc.addedJars.toSeq: _*) + assert(addedJarsPreTaskSet.size === 0) + + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + val taskSet1 = FakeTask.createTaskSet(3) + val manager1 = new TaskSetManager(sched, taskSet1, MAX_TASK_FAILURES, clock = new ManualClock) + + // all tasks from the first taskset have the same jars + val taskOption1 = manager1.resourceOffer("exec1", "host1", NO_PREF) + assert(taskOption1.get.addedJars === addedJarsPreTaskSet) + val taskOption2 = manager1.resourceOffer("exec1", "host1", NO_PREF) + assert(taskOption2.get.addedJars === addedJarsPreTaskSet) + + // even with a jar added mid-TaskSet + val jarPath = Thread.currentThread().getContextClassLoader.getResource("TestUDTF.jar") + sc.addJar(jarPath.toString) + val addedJarsMidTaskSet = Map[String, Long](sc.addedJars.toSeq: _*) + assert(addedJarsPreTaskSet !== addedJarsMidTaskSet) + val taskOption3 = manager1.resourceOffer("exec1", "host1", NO_PREF) + // which should have the old version of the jars list + assert(taskOption3.get.addedJars === addedJarsPreTaskSet) + + // and then the jar does appear in the next TaskSet + val taskSet2 = FakeTask.createTaskSet(1) + val manager2 = new TaskSetManager(sched, taskSet2, MAX_TASK_FAILURES, clock = new ManualClock) + + val taskOption4 = manager2.resourceOffer("exec1", "host1", NO_PREF) + assert(taskOption4.get.addedJars === addedJarsMidTaskSet) } - def createTaskResult(id: Int): DirectTaskResult[Int] = { + private def createTaskResult( + id: Int, + accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() - new DirectTaskResult[Int](valueSer.serialize(id), mutable.Map.empty, new TaskMetrics) + new DirectTaskResult[Int](valueSer.serialize(id), accumUpdates) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala deleted file mode 100644 index 525ee0d3bdc5..000000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala +++ /dev/null @@ -1,187 +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.scheduler.cluster.mesos - -import java.util -import java.util.Collections - -import org.apache.mesos.Protos.Value.Scalar -import org.apache.mesos.Protos._ -import org.apache.mesos.{Protos, Scheduler, SchedulerDriver} -import org.mockito.Matchers._ -import org.mockito.Mockito._ -import org.mockito.Matchers -import org.scalatest.mock.MockitoSugar -import org.scalatest.BeforeAndAfter - -import org.apache.spark.scheduler.TaskSchedulerImpl -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SecurityManager, SparkFunSuite} - -class CoarseMesosSchedulerBackendSuite extends SparkFunSuite - with LocalSparkContext - with MockitoSugar - with BeforeAndAfter { - - private def createOffer(offerId: String, slaveId: String, mem: Int, cpu: Int): Offer = { - val builder = Offer.newBuilder() - builder.addResourcesBuilder() - .setName("mem") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(mem)) - builder.addResourcesBuilder() - .setName("cpus") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(cpu)) - builder.setId(OfferID.newBuilder() - .setValue(offerId).build()) - .setFrameworkId(FrameworkID.newBuilder() - .setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(slaveId)) - .setHostname(s"host${slaveId}") - .build() - } - - private def createSchedulerBackend( - taskScheduler: TaskSchedulerImpl, - driver: SchedulerDriver): CoarseMesosSchedulerBackend = { - val securityManager = mock[SecurityManager] - val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master", securityManager) { - override protected def createSchedulerDriver( - masterUrl: String, - scheduler: Scheduler, - sparkUser: String, - appName: String, - conf: SparkConf, - webuiUrl: Option[String] = None, - checkpoint: Option[Boolean] = None, - failoverTimeout: Option[Double] = None, - frameworkId: Option[String] = None): SchedulerDriver = driver - markRegistered() - } - backend.start() - backend - } - - var sparkConf: SparkConf = _ - - before { - sparkConf = (new SparkConf) - .setMaster("local[*]") - .setAppName("test-mesos-dynamic-alloc") - .setSparkHome("/path") - - sc = new SparkContext(sparkConf) - } - - test("mesos supports killing and limiting executors") { - val driver = mock[SchedulerDriver] - when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) - val taskScheduler = mock[TaskSchedulerImpl] - when(taskScheduler.sc).thenReturn(sc) - - sparkConf.set("spark.driver.host", "driverHost") - sparkConf.set("spark.driver.port", "1234") - - val backend = createSchedulerBackend(taskScheduler, driver) - val minMem = backend.calculateTotalMemory(sc) - val minCpu = 4 - - val mesosOffers = new java.util.ArrayList[Offer] - mesosOffers.add(createOffer("o1", "s1", minMem, minCpu)) - - val taskID0 = TaskID.newBuilder().setValue("0").build() - - backend.resourceOffers(driver, mesosOffers) - verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), - any[util.Collection[TaskInfo]], - any[Filters]) - - // simulate the allocation manager down-scaling executors - backend.doRequestTotalExecutors(0) - assert(backend.doKillExecutors(Seq("s1/0"))) - verify(driver, times(1)).killTask(taskID0) - - val mesosOffers2 = new java.util.ArrayList[Offer] - mesosOffers2.add(createOffer("o2", "s2", minMem, minCpu)) - backend.resourceOffers(driver, mesosOffers2) - - verify(driver, times(1)) - .declineOffer(OfferID.newBuilder().setValue("o2").build()) - - // Verify we didn't launch any new executor - assert(backend.slaveIdsWithExecutors.size === 1) - - backend.doRequestTotalExecutors(2) - backend.resourceOffers(driver, mesosOffers2) - verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(mesosOffers2.get(0).getId)), - any[util.Collection[TaskInfo]], - any[Filters]) - - assert(backend.slaveIdsWithExecutors.size === 2) - backend.slaveLost(driver, SlaveID.newBuilder().setValue("s1").build()) - assert(backend.slaveIdsWithExecutors.size === 1) - } - - test("mesos supports killing and relaunching tasks with executors") { - val driver = mock[SchedulerDriver] - when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) - val taskScheduler = mock[TaskSchedulerImpl] - when(taskScheduler.sc).thenReturn(sc) - - val backend = createSchedulerBackend(taskScheduler, driver) - val minMem = backend.calculateTotalMemory(sc) + 1024 - val minCpu = 4 - - val mesosOffers = new java.util.ArrayList[Offer] - val offer1 = createOffer("o1", "s1", minMem, minCpu) - mesosOffers.add(offer1) - - val offer2 = createOffer("o2", "s1", minMem, 1); - - backend.resourceOffers(driver, mesosOffers) - - verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(offer1.getId)), - anyObject(), - anyObject[Filters]) - - // Simulate task killed, executor no longer running - val status = TaskStatus.newBuilder() - .setTaskId(TaskID.newBuilder().setValue("0").build()) - .setSlaveId(SlaveID.newBuilder().setValue("s1").build()) - .setState(TaskState.TASK_KILLED) - .build - - backend.statusUpdate(driver, status) - assert(!backend.slaveIdsWithExecutors.contains("s1")) - - mesosOffers.clear() - mesosOffers.add(offer2) - backend.resourceOffers(driver, mesosOffers) - assert(backend.slaveIdsWithExecutors.contains("s1")) - - verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(offer2.getId)), - anyObject(), - anyObject[Filters]) - - verify(driver, times(1)).reviveOffers() - } -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala deleted file mode 100644 index c4dc56003120..000000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala +++ /dev/null @@ -1,347 +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.scheduler.cluster.mesos - -import java.nio.ByteBuffer -import java.util.Arrays -import java.util.Collection -import java.util.Collections - -import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer - -import org.apache.mesos.Protos.Value.Scalar -import org.apache.mesos.Protos._ -import org.apache.mesos.SchedulerDriver -import org.mockito.Matchers._ -import org.mockito.Mockito._ -import org.mockito.{ArgumentCaptor, Matchers} -import org.scalatest.mock.MockitoSugar - -import org.apache.spark.executor.MesosExecutorBackend -import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded, - TaskDescription, TaskSchedulerImpl, WorkerOffer} -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} - -class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar { - - test("Use configured mesosExecutor.cores for ExecutorInfo") { - val mesosExecutorCores = 3 - val conf = new SparkConf - conf.set("spark.mesos.mesosExecutor.cores", mesosExecutorCores.toString) - - 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") - - val resources = Arrays.asList( - mesosSchedulerBackend.createResource("cpus", 4), - mesosSchedulerBackend.createResource("mem", 1024)) - // uri is null. - val (executorInfo, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id") - val executorResources = executorInfo.getResourcesList - val cpus = executorResources.asScala.find(_.getName.equals("cpus")).get.getScalar.getValue - - assert(cpus === mesosExecutorCores) - } - - test("check spark-class location correctly") { - val conf = new SparkConf - conf.set("spark.mesos.executor.home" , "/mesos-home") - - 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") - - val resources = Arrays.asList( - mesosSchedulerBackend.createResource("cpus", 4), - mesosSchedulerBackend.createResource("mem", 1024)) - // uri is null. - val (executorInfo, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id") - assert(executorInfo.getCommand.getValue === - s" /mesos-home/bin/spark-class ${classOf[MesosExecutorBackend].getName}") - - // uri exists. - conf.set("spark.executor.uri", "hdfs:///test-app-1.0.0.tgz") - val (executorInfo1, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id") - assert(executorInfo1.getCommand.getValue === - s"cd test-app-1*; ./bin/spark-class ${classOf[MesosExecutorBackend].getName}") - } - - test("spark docker properties correctly populate the DockerInfo message") { - val taskScheduler = mock[TaskSchedulerImpl] - - val conf = new SparkConf() - .set("spark.mesos.executor.docker.image", "spark/mock") - .set("spark.mesos.executor.docker.volumes", "/a,/b:/b,/c:/c:rw,/d:ro,/e:/e:ro") - .set("spark.mesos.executor.docker.portmaps", "80:8080,53:53:tcp") - - val listenerBus = mock[LiveListenerBus] - listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) - - val sc = mock[SparkContext] - when(sc.executorMemory).thenReturn(100) - when(sc.getSparkHome()).thenReturn(Option("/spark-home")) - when(sc.executorEnvs).thenReturn(new mutable.HashMap[String, String]) - when(sc.conf).thenReturn(conf) - when(sc.listenerBus).thenReturn(listenerBus) - - val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") - - val (execInfo, _) = backend.createExecutorInfo( - Arrays.asList(backend.createResource("cpus", 4)), "mockExecutor") - assert(execInfo.getContainer.getDocker.getImage.equals("spark/mock")) - val portmaps = execInfo.getContainer.getDocker.getPortMappingsList - assert(portmaps.get(0).getHostPort.equals(80)) - assert(portmaps.get(0).getContainerPort.equals(8080)) - assert(portmaps.get(0).getProtocol.equals("tcp")) - assert(portmaps.get(1).getHostPort.equals(53)) - assert(portmaps.get(1).getContainerPort.equals(53)) - assert(portmaps.get(1).getProtocol.equals("tcp")) - val volumes = execInfo.getContainer.getVolumesList - assert(volumes.get(0).getContainerPath.equals("/a")) - assert(volumes.get(0).getMode.equals(Volume.Mode.RW)) - assert(volumes.get(1).getContainerPath.equals("/b")) - assert(volumes.get(1).getHostPath.equals("/b")) - assert(volumes.get(1).getMode.equals(Volume.Mode.RW)) - assert(volumes.get(2).getContainerPath.equals("/c")) - assert(volumes.get(2).getHostPath.equals("/c")) - assert(volumes.get(2).getMode.equals(Volume.Mode.RW)) - assert(volumes.get(3).getContainerPath.equals("/d")) - assert(volumes.get(3).getMode.equals(Volume.Mode.RO)) - assert(volumes.get(4).getContainerPath.equals("/e")) - assert(volumes.get(4).getHostPath.equals("/e")) - assert(volumes.get(4).getMode.equals(Volume.Mode.RO)) - } - - test("mesos resource offers result in launching tasks") { - def createOffer(id: Int, mem: Int, cpu: Int): Offer = { - val builder = Offer.newBuilder() - builder.addResourcesBuilder() - .setName("mem") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(mem)) - builder.addResourcesBuilder() - .setName("cpus") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(cpu)) - builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()) - .setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")) - .setHostname(s"host${id.toString}").build() - } - - val driver = mock[SchedulerDriver] - val taskScheduler = mock[TaskSchedulerImpl] - - val listenerBus = mock[LiveListenerBus] - listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) - - 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 backend = new MesosSchedulerBackend(taskScheduler, sc, "master") - - val minMem = backend.calculateTotalMemory(sc) - val minCpu = 4 - - val mesosOffers = new java.util.ArrayList[Offer] - mesosOffers.add(createOffer(1, minMem, minCpu)) - mesosOffers.add(createOffer(2, minMem - 1, minCpu)) - mesosOffers.add(createOffer(3, minMem, minCpu)) - - val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2) - expectedWorkerOffers.append(new WorkerOffer( - mesosOffers.get(0).getSlaveId.getValue, - mesosOffers.get(0).getHostname, - (minCpu - backend.mesosExecutorCores).toInt - )) - expectedWorkerOffers.append(new WorkerOffer( - mesosOffers.get(2).getSlaveId.getValue, - mesosOffers.get(2).getHostname, - (minCpu - backend.mesosExecutorCores).toInt - )) - val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) - when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) - when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - - val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) - when( - driver.launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), - capture.capture(), - any(classOf[Filters]) - ) - ).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) - - 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")) - val cpus = taskInfo.getResourcesList.get(0) - assert(cpus.getName.equals("cpus")) - assert(cpus.getScalar.getValue.equals(2.0)) - assert(taskInfo.getSlaveId.getValue.equals("s1")) - - // 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)) - 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) - verify(driver, times(1)).declineOffer(mesosOffers2.get(0).getId) - } - - test("can handle multiple roles") { - val driver = mock[SchedulerDriver] - val taskScheduler = mock[TaskSchedulerImpl] - - val listenerBus = mock[LiveListenerBus] - listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) - - 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 id = 1 - val builder = Offer.newBuilder() - builder.addResourcesBuilder() - .setName("mem") - .setType(Value.Type.SCALAR) - .setRole("prod") - .setScalar(Scalar.newBuilder().setValue(500)) - builder.addResourcesBuilder() - .setName("cpus") - .setRole("prod") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(1)) - builder.addResourcesBuilder() - .setName("mem") - .setRole("dev") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(600)) - builder.addResourcesBuilder() - .setName("cpus") - .setRole("dev") - .setType(Value.Type.SCALAR) - .setScalar(Scalar.newBuilder().setValue(2)) - val offer = builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()) - .setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")) - .setHostname(s"host${id.toString}").build() - - val mesosOffers = new java.util.ArrayList[Offer] - mesosOffers.add(offer) - - val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") - - val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](1) - expectedWorkerOffers.append(new WorkerOffer( - mesosOffers.get(0).getSlaveId.getValue, - mesosOffers.get(0).getHostname, - 2 // Deducting 1 for executor - )) - - val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) - when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) - when(taskScheduler.CPUS_PER_TASK).thenReturn(1) - - val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) - when( - driver.launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), - capture.capture(), - any(classOf[Filters]) - ) - ).thenReturn(Status.valueOf(1)) - - backend.resourceOffers(driver, mesosOffers) - - verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), - capture.capture(), - any(classOf[Filters]) - ) - - assert(capture.getValue.size() === 1) - val taskInfo = capture.getValue.iterator().next() - assert(taskInfo.getName.equals("n1")) - assert(taskInfo.getResourcesCount === 1) - val cpusDev = taskInfo.getResourcesList.get(0) - assert(cpusDev.getName.equals("cpus")) - assert(cpusDev.getScalar.getValue.equals(1.0)) - assert(cpusDev.getRole.equals("dev")) - val executorResources = taskInfo.getExecutor.getResourcesList.asScala - assert(executorResources.exists { r => - r.getName.equals("mem") && r.getScalar.getValue.equals(484.0) && r.getRole.equals("prod") - }) - assert(executorResources.exists { r => - r.getName.equals("cpus") && r.getScalar.getValue.equals(1.0) && r.getRole.equals("prod") - }) - } -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala deleted file mode 100644 index 2eb43b731338..000000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala +++ /dev/null @@ -1,143 +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.scheduler.cluster.mesos - -import scala.language.reflectiveCalls - -import org.apache.mesos.Protos.Value -import org.mockito.Mockito._ -import org.scalatest._ -import org.scalatest.mock.MockitoSugar - -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} - -class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoSugar { - - // scalastyle:off structural.type - // this is the documented way of generating fixtures in scalatest - def fixture: Object {val sc: SparkContext; val sparkConf: SparkConf} = new { - val sparkConf = new SparkConf - val sc = mock[SparkContext] - when(sc.conf).thenReturn(sparkConf) - } - val utils = new MesosSchedulerUtils { } - // scalastyle:on structural.type - - test("use at-least minimum overhead") { - val f = fixture - when(f.sc.executorMemory).thenReturn(512) - utils.calculateTotalMemory(f.sc) shouldBe 896 - } - - test("use overhead if it is greater than minimum value") { - val f = fixture - when(f.sc.executorMemory).thenReturn(4096) - utils.calculateTotalMemory(f.sc) shouldBe 4505 - } - - test("use spark.mesos.executor.memoryOverhead (if set)") { - val f = fixture - when(f.sc.executorMemory).thenReturn(1024) - f.sparkConf.set("spark.mesos.executor.memoryOverhead", "512") - utils.calculateTotalMemory(f.sc) shouldBe 1536 - } - - test("parse a non-empty constraint string correctly") { - val expectedMap = Map( - "tachyon" -> Set("true"), - "zone" -> Set("us-east-1a", "us-east-1b") - ) - utils.parseConstraintString("tachyon:true;zone:us-east-1a,us-east-1b") should be (expectedMap) - } - - test("parse an empty constraint string correctly") { - utils.parseConstraintString("") shouldBe Map() - } - - test("throw an exception when the input is malformed") { - an[IllegalArgumentException] should be thrownBy - utils.parseConstraintString("tachyon;zone:us-east") - } - - test("empty values for attributes' constraints matches all values") { - val constraintsStr = "tachyon:" - val parsedConstraints = utils.parseConstraintString(constraintsStr) - - parsedConstraints shouldBe Map("tachyon" -> Set()) - - val zoneSet = Value.Set.newBuilder().addItem("us-east-1a").addItem("us-east-1b").build() - val noTachyonOffer = Map("zone" -> zoneSet) - val tachyonTrueOffer = Map("tachyon" -> Value.Text.newBuilder().setValue("true").build()) - val tachyonFalseOffer = Map("tachyon" -> Value.Text.newBuilder().setValue("false").build()) - - utils.matchesAttributeRequirements(parsedConstraints, noTachyonOffer) shouldBe false - utils.matchesAttributeRequirements(parsedConstraints, tachyonTrueOffer) shouldBe true - utils.matchesAttributeRequirements(parsedConstraints, tachyonFalseOffer) shouldBe true - } - - test("subset match is performed for set attributes") { - val supersetConstraint = Map( - "tachyon" -> Value.Text.newBuilder().setValue("true").build(), - "zone" -> Value.Set.newBuilder() - .addItem("us-east-1a") - .addItem("us-east-1b") - .addItem("us-east-1c") - .build()) - - val zoneConstraintStr = "tachyon:;zone:us-east-1a,us-east-1c" - val parsedConstraints = utils.parseConstraintString(zoneConstraintStr) - - utils.matchesAttributeRequirements(parsedConstraints, supersetConstraint) shouldBe true - } - - test("less than equal match is performed on scalar attributes") { - val offerAttribs = Map("gpus" -> Value.Scalar.newBuilder().setValue(3).build()) - - val ltConstraint = utils.parseConstraintString("gpus:2") - val eqConstraint = utils.parseConstraintString("gpus:3") - val gtConstraint = utils.parseConstraintString("gpus:4") - - utils.matchesAttributeRequirements(ltConstraint, offerAttribs) shouldBe true - utils.matchesAttributeRequirements(eqConstraint, offerAttribs) shouldBe true - utils.matchesAttributeRequirements(gtConstraint, offerAttribs) shouldBe false - } - - test("contains match is performed for range attributes") { - val offerAttribs = Map("ports" -> Value.Range.newBuilder().setBegin(7000).setEnd(8000).build()) - val ltConstraint = utils.parseConstraintString("ports:6000") - val eqConstraint = utils.parseConstraintString("ports:7500") - val gtConstraint = utils.parseConstraintString("ports:8002") - val multiConstraint = utils.parseConstraintString("ports:5000,7500,8300") - - utils.matchesAttributeRequirements(ltConstraint, offerAttribs) shouldBe false - utils.matchesAttributeRequirements(eqConstraint, offerAttribs) shouldBe true - utils.matchesAttributeRequirements(gtConstraint, offerAttribs) shouldBe false - utils.matchesAttributeRequirements(multiConstraint, offerAttribs) shouldBe true - } - - test("equality match is performed for text attributes") { - val offerAttribs = Map("tachyon" -> Value.Text.newBuilder().setValue("true").build()) - - val trueConstraint = utils.parseConstraintString("tachyon:true") - val falseConstraint = utils.parseConstraintString("tachyon:false") - - utils.matchesAttributeRequirements(trueConstraint, offerAttribs) shouldBe true - utils.matchesAttributeRequirements(falseConstraint, offerAttribs) shouldBe false - } - -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala deleted file mode 100644 index 5a81bb335fdb..000000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchDataSuite.scala +++ /dev/null @@ -1,36 +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.scheduler.cluster.mesos - -import java.nio.ByteBuffer - -import org.apache.spark.SparkFunSuite - -class MesosTaskLaunchDataSuite extends SparkFunSuite { - test("serialize and deserialize data must be same") { - val serializedTask = ByteBuffer.allocate(40) - (Range(100, 110).map(serializedTask.putInt(_))) - serializedTask.rewind - val attemptNumber = 100 - val byteString = MesosTaskLaunchData(serializedTask, attemptNumber).toByteString - serializedTask.rewind - val mesosTaskLaunchData = MesosTaskLaunchData.fromByteString(byteString) - assert(mesosTaskLaunchData.attemptNumber == attemptNumber) - assert(mesosTaskLaunchData.serializedTask.equals(serializedTask)) - } -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala deleted file mode 100644 index f5cef1caaf1a..000000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosClusterSchedulerSuite.scala +++ /dev/null @@ -1,75 +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.scheduler.mesos - -import java.util.Date - -import org.scalatest.mock.MockitoSugar - -import org.apache.spark.deploy.Command -import org.apache.spark.deploy.mesos.MesosDriverDescription -import org.apache.spark.scheduler.cluster.mesos._ -import org.apache.spark.{LocalSparkContext, SparkConf, SparkFunSuite} - - -class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar { - - private val command = new Command("mainClass", Seq("arg"), null, null, null, null) - - test("can queue drivers") { - val conf = new SparkConf() - conf.setMaster("mesos://localhost:5050") - conf.setAppName("spark mesos") - val scheduler = new MesosClusterScheduler( - new BlackHoleMesosClusterPersistenceEngineFactory, conf) { - override def start(): Unit = { ready = true } - } - scheduler.start() - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", 1000, 1, true, - command, Map[String, String](), "s1", new Date())) - assert(response.success) - val response2 = - scheduler.submitDriver(new MesosDriverDescription( - "d1", "jar", 1000, 1, true, command, Map[String, String](), "s2", new Date())) - assert(response2.success) - val state = scheduler.getSchedulerState() - val queuedDrivers = state.queuedDrivers.toList - assert(queuedDrivers(0).submissionId == response.submissionId) - assert(queuedDrivers(1).submissionId == response2.submissionId) - } - - test("can kill queued drivers") { - val conf = new SparkConf() - conf.setMaster("mesos://localhost:5050") - conf.setAppName("spark mesos") - val scheduler = new MesosClusterScheduler( - new BlackHoleMesosClusterPersistenceEngineFactory, conf) { - override def start(): Unit = { ready = true } - } - scheduler.start() - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", 1000, 1, true, - command, Map[String, String](), "s1", new Date())) - assert(response.success) - val killResponse = scheduler.killDriver(response.submissionId) - assert(killResponse.success) - val state = scheduler.getSchedulerState() - assert(state.queuedDrivers.isEmpty) - } -} diff --git a/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala new file mode 100644 index 000000000000..78f618f8a216 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.security + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, FileInputStream, FileOutputStream} +import java.nio.channels.Channels +import java.nio.charset.StandardCharsets.UTF_8 +import java.nio.file.Files +import java.util.{Arrays, Random, UUID} + +import com.google.common.io.ByteStreams + +import org.apache.spark._ +import org.apache.spark.internal.config._ +import org.apache.spark.network.util.CryptoUtils +import org.apache.spark.security.CryptoStreamUtils._ +import org.apache.spark.serializer.{JavaSerializer, SerializerManager} +import org.apache.spark.storage.TempShuffleBlockId + +class CryptoStreamUtilsSuite extends SparkFunSuite { + + test("crypto configuration conversion") { + val sparkKey1 = s"${SPARK_IO_ENCRYPTION_COMMONS_CONFIG_PREFIX}a.b.c" + val sparkVal1 = "val1" + val cryptoKey1 = s"${CryptoUtils.COMMONS_CRYPTO_CONFIG_PREFIX}a.b.c" + + val sparkKey2 = SPARK_IO_ENCRYPTION_COMMONS_CONFIG_PREFIX.stripSuffix(".") + "A.b.c" + val sparkVal2 = "val2" + val cryptoKey2 = s"${CryptoUtils.COMMONS_CRYPTO_CONFIG_PREFIX}A.b.c" + val conf = new SparkConf() + conf.set(sparkKey1, sparkVal1) + conf.set(sparkKey2, sparkVal2) + val props = CryptoStreamUtils.toCryptoConf(conf) + assert(props.getProperty(cryptoKey1) === sparkVal1) + assert(!props.containsKey(cryptoKey2)) + } + + test("shuffle encryption key length should be 128 by default") { + val conf = createConf() + var key = CryptoStreamUtils.createKey(conf) + val actual = key.length * (java.lang.Byte.SIZE) + assert(actual === 128) + } + + test("create 256-bit key") { + val conf = createConf(IO_ENCRYPTION_KEY_SIZE_BITS.key -> "256") + var key = CryptoStreamUtils.createKey(conf) + val actual = key.length * (java.lang.Byte.SIZE) + assert(actual === 256) + } + + test("create key with invalid length") { + intercept[IllegalArgumentException] { + val conf = createConf(IO_ENCRYPTION_KEY_SIZE_BITS.key -> "328") + CryptoStreamUtils.createKey(conf) + } + } + + test("serializer manager integration") { + val conf = createConf() + .set("spark.shuffle.compress", "true") + .set("spark.shuffle.spill.compress", "true") + + val plainStr = "hello world" + val blockId = new TempShuffleBlockId(UUID.randomUUID()) + val key = Some(CryptoStreamUtils.createKey(conf)) + val serializerManager = new SerializerManager(new JavaSerializer(conf), conf, + encryptionKey = key) + + val outputStream = new ByteArrayOutputStream() + val wrappedOutputStream = serializerManager.wrapStream(blockId, outputStream) + wrappedOutputStream.write(plainStr.getBytes(UTF_8)) + wrappedOutputStream.close() + + val encryptedBytes = outputStream.toByteArray + val encryptedStr = new String(encryptedBytes, UTF_8) + assert(plainStr !== encryptedStr) + + val inputStream = new ByteArrayInputStream(encryptedBytes) + val wrappedInputStream = serializerManager.wrapStream(blockId, inputStream) + val decryptedBytes = ByteStreams.toByteArray(wrappedInputStream) + val decryptedStr = new String(decryptedBytes, UTF_8) + assert(decryptedStr === plainStr) + } + + test("encryption key propagation to executors") { + val conf = createConf().setAppName("Crypto Test").setMaster("local-cluster[1,1,1024]") + val sc = new SparkContext(conf) + try { + val content = "This is the content to be encrypted." + val encrypted = sc.parallelize(Seq(1)) + .map { str => + val bytes = new ByteArrayOutputStream() + val out = CryptoStreamUtils.createCryptoOutputStream(bytes, SparkEnv.get.conf, + SparkEnv.get.securityManager.getIOEncryptionKey().get) + out.write(content.getBytes(UTF_8)) + out.close() + bytes.toByteArray() + }.collect()(0) + + assert(content != encrypted) + + val in = CryptoStreamUtils.createCryptoInputStream(new ByteArrayInputStream(encrypted), + sc.conf, SparkEnv.get.securityManager.getIOEncryptionKey().get) + val decrypted = new String(ByteStreams.toByteArray(in), UTF_8) + assert(content === decrypted) + } finally { + sc.stop() + } + } + + test("crypto stream wrappers") { + val testData = new Array[Byte](128 * 1024) + new Random().nextBytes(testData) + + val conf = createConf() + val key = createKey(conf) + val file = Files.createTempFile("crypto", ".test").toFile() + file.deleteOnExit() + + val outStream = createCryptoOutputStream(new FileOutputStream(file), conf, key) + try { + ByteStreams.copy(new ByteArrayInputStream(testData), outStream) + } finally { + outStream.close() + } + + val inStream = createCryptoInputStream(new FileInputStream(file), conf, key) + try { + val inStreamData = ByteStreams.toByteArray(inStream) + assert(Arrays.equals(inStreamData, testData)) + } finally { + inStream.close() + } + + val outChannel = createWritableChannel(new FileOutputStream(file).getChannel(), conf, key) + try { + val inByteChannel = Channels.newChannel(new ByteArrayInputStream(testData)) + ByteStreams.copy(inByteChannel, outChannel) + } finally { + outChannel.close() + } + + val inChannel = createReadableChannel(new FileInputStream(file).getChannel(), conf, key) + try { + val inChannelData = ByteStreams.toByteArray(Channels.newInputStream(inChannel)) + assert(Arrays.equals(inChannelData, testData)) + } finally { + inChannel.close() + } + } + + private def createConf(extra: (String, String)*): SparkConf = { + val conf = new SparkConf() + extra.foreach { case (k, v) => conf.set(k, v) } + conf.set(IO_ENCRYPTION_ENABLED, true) + conf + } + +} diff --git a/core/src/test/scala/org/apache/spark/security/EncryptionFunSuite.scala b/core/src/test/scala/org/apache/spark/security/EncryptionFunSuite.scala new file mode 100644 index 000000000000..3f52dc41abf6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/security/EncryptionFunSuite.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.security + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config._ + +trait EncryptionFunSuite { + + this: SparkFunSuite => + + /** + * Runs a test twice, initializing a SparkConf object with encryption off, then on. It's ok + * for the test to modify the provided SparkConf. + */ + final protected def encryptionTest(name: String)(fn: SparkConf => Unit) { + Seq(false, true).foreach { encrypt => + test(s"$name (encryption = ${ if (encrypt) "on" else "off" })") { + val conf = new SparkConf().set(IO_ENCRYPTION_ENABLED, encrypt) + fn(conf) + } + } + } + +} diff --git a/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala index 87f25e7245e1..3734f1cb408f 100644 --- a/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala @@ -20,11 +20,11 @@ package org.apache.spark.serializer import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import java.nio.ByteBuffer -import com.esotericsoftware.kryo.io.{Output, Input} -import org.apache.avro.{SchemaBuilder, Schema} +import com.esotericsoftware.kryo.io.{Input, Output} +import org.apache.avro.{Schema, SchemaBuilder} import org.apache.avro.generic.GenericData.Record -import org.apache.spark.{SparkFunSuite, SharedSparkContext} +import org.apache.spark.{SharedSparkContext, SparkFunSuite} class GenericAvroSerializerSuite extends SparkFunSuite with SharedSparkContext { conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") diff --git a/core/src/test/scala/org/apache/spark/serializer/JavaSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/JavaSerializerSuite.scala index 20f45670bc2b..6a6ea42797fb 100644 --- a/core/src/test/scala/org/apache/spark/serializer/JavaSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/JavaSerializerSuite.scala @@ -23,13 +23,18 @@ class JavaSerializerSuite extends SparkFunSuite { test("JavaSerializer instances are serializable") { val serializer = new JavaSerializer(new SparkConf()) val instance = serializer.newInstance() - instance.deserialize[JavaSerializer](instance.serialize(serializer)) + val obj = instance.deserialize[JavaSerializer](instance.serialize(serializer)) + // enforce class cast + obj.getClass } test("Deserialize object containing a primitive Class as attribute") { val serializer = new JavaSerializer(new SparkConf()) val instance = serializer.newInstance() - instance.deserialize[JavaSerializer](instance.serialize(new ContainsPrimitiveClass())) + val obj = instance.deserialize[ContainsPrimitiveClass](instance.serialize( + new ContainsPrimitiveClass())) + // enforce class cast + obj.getClass } } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala new file mode 100644 index 000000000000..a1cf3570a7a6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.serializer + +import scala.reflect.ClassTag +import scala.util.Random + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.serializer.KryoTest._ +import org.apache.spark.util.Benchmark + +class KryoBenchmark extends SparkFunSuite { + val benchmark = new Benchmark("Benchmark Kryo Unsafe vs safe Serialization", 1024 * 1024 * 15, 10) + + ignore(s"Benchmark Kryo Unsafe vs safe Serialization") { + Seq (true, false).foreach (runBenchmark) + benchmark.run() + + // scalastyle:off + /* + Benchmark Kryo Unsafe vs safe Serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + basicTypes: Int with unsafe:true 151 / 170 104.2 9.6 1.0X + basicTypes: Long with unsafe:true 175 / 191 89.8 11.1 0.9X + basicTypes: Float with unsafe:true 177 / 184 88.8 11.3 0.9X + basicTypes: Double with unsafe:true 193 / 216 81.4 12.3 0.8X + Array: Int with unsafe:true 513 / 587 30.7 32.6 0.3X + Array: Long with unsafe:true 1211 / 1358 13.0 77.0 0.1X + Array: Float with unsafe:true 890 / 964 17.7 56.6 0.2X + Array: Double with unsafe:true 1335 / 1428 11.8 84.9 0.1X + Map of string->Double with unsafe:true 931 / 988 16.9 59.2 0.2X + basicTypes: Int with unsafe:false 197 / 217 79.9 12.5 0.8X + basicTypes: Long with unsafe:false 219 / 240 71.8 13.9 0.7X + basicTypes: Float with unsafe:false 208 / 217 75.7 13.2 0.7X + basicTypes: Double with unsafe:false 208 / 225 75.6 13.2 0.7X + Array: Int with unsafe:false 2559 / 2681 6.1 162.7 0.1X + Array: Long with unsafe:false 3425 / 3516 4.6 217.8 0.0X + Array: Float with unsafe:false 2025 / 2134 7.8 128.7 0.1X + Array: Double with unsafe:false 2241 / 2358 7.0 142.5 0.1X + Map of string->Double with unsafe:false 1044 / 1085 15.1 66.4 0.1X + */ + // scalastyle:on + } + + private def runBenchmark(useUnsafe: Boolean): Unit = { + def check[T: ClassTag](t: T, ser: SerializerInstance): Int = { + if (ser.deserialize[T](ser.serialize(t)) === t) 1 else 0 + } + + // Benchmark Primitives + val basicTypeCount = 1000000 + def basicTypes[T: ClassTag](name: String, gen: () => T): Unit = { + lazy val ser = createSerializer(useUnsafe) + val arrayOfBasicType: Array[T] = Array.fill(basicTypeCount)(gen()) + + benchmark.addCase(s"basicTypes: $name with unsafe:$useUnsafe") { _ => + var sum = 0L + var i = 0 + while (i < basicTypeCount) { + sum += check(arrayOfBasicType(i), ser) + i += 1 + } + sum + } + } + basicTypes("Int", () => Random.nextInt()) + basicTypes("Long", () => Random.nextLong()) + basicTypes("Float", () => Random.nextFloat()) + basicTypes("Double", () => Random.nextDouble()) + + // Benchmark Array of Primitives + val arrayCount = 10000 + def basicTypeArray[T: ClassTag](name: String, gen: () => T): Unit = { + lazy val ser = createSerializer(useUnsafe) + val arrayOfArrays: Array[Array[T]] = + Array.fill(arrayCount)(Array.fill[T](Random.nextInt(arrayCount))(gen())) + + benchmark.addCase(s"Array: $name with unsafe:$useUnsafe") { _ => + var sum = 0L + var i = 0 + while (i < arrayCount) { + val arr = arrayOfArrays(i) + sum += check(arr, ser) + i += 1 + } + sum + } + } + basicTypeArray("Int", () => Random.nextInt()) + basicTypeArray("Long", () => Random.nextLong()) + basicTypeArray("Float", () => Random.nextFloat()) + basicTypeArray("Double", () => Random.nextDouble()) + + // Benchmark Maps + val mapsCount = 1000 + lazy val ser = createSerializer(useUnsafe) + val arrayOfMaps: Array[Map[String, Double]] = Array.fill(mapsCount) { + Array.fill(Random.nextInt(mapsCount)) { + (Random.nextString(mapsCount / 10), Random.nextDouble()) + }.toMap + } + + benchmark.addCase(s"Map of string->Double with unsafe:$useUnsafe") { _ => + var sum = 0L + var i = 0 + while (i < mapsCount) { + val map = arrayOfMaps(i) + sum += check(map, ser) + i += 1 + } + sum + } + } + + def createSerializer(useUnsafe: Boolean): SerializerInstance = { + val conf = new SparkConf() + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) + conf.set("spark.kryo.unsafe", useUnsafe.toString) + + new KryoSerializer(conf).newInstance() + } + +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala index 935a091f14f9..46aa9c37986c 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -17,25 +17,26 @@ package org.apache.spark.serializer -import org.apache.spark.util.Utils - import com.esotericsoftware.kryo.Kryo import org.apache.spark._ +import org.apache.spark.internal.config import org.apache.spark.serializer.KryoDistributedTest._ +import org.apache.spark.util.Utils -class KryoSerializerDistributedSuite extends SparkFunSuite { +class KryoSerializerDistributedSuite extends SparkFunSuite with LocalSparkContext { test("kryo objects are serialised consistently in different processes") { val conf = new SparkConf(false) .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .set("spark.kryo.registrator", classOf[AppJarRegistrator].getName) - .set("spark.task.maxFailures", "1") + .set(config.MAX_TASK_FAILURES, 1) + .set(config.BLACKLIST_ENABLED, false) val jar = TestUtils.createJarWithClasses(List(AppJarRegistrator.customClassName)) conf.setJars(List(jar.getPath)) - val sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) + sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) val original = Thread.currentThread.getContextClassLoader val loader = new java.net.URLClassLoader(Array(jar), Utils.getContextOrSparkClassLoader) SparkEnv.get.serializer.setDefaultClassLoader(loader) @@ -48,8 +49,6 @@ class KryoSerializerDistributedSuite extends SparkFunSuite { // Join the two RDDs, and force evaluation assert(shuffledRDD.join(cachedRDD).collect().size == 1) - - LocalSparkContext.stop(sc) } } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala index a9b209ccfc76..cf01f79f4909 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala @@ -18,11 +18,10 @@ package org.apache.spark.serializer import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.LocalSparkContext._ import org.apache.spark.SparkContext -import org.apache.spark.LocalSparkContext import org.apache.spark.SparkException - class KryoSerializerResizableOutputSuite extends SparkFunSuite { // trial and error showed this will not serialize with 1mb buffer @@ -33,9 +32,9 @@ class KryoSerializerResizableOutputSuite extends SparkFunSuite { conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") conf.set("spark.kryoserializer.buffer", "1m") conf.set("spark.kryoserializer.buffer.max", "1m") - val sc = new SparkContext("local", "test", conf) - intercept[SparkException](sc.parallelize(x).collect()) - LocalSparkContext.stop(sc) + withSpark(new SparkContext("local", "test", conf)) { sc => + intercept[SparkException](sc.parallelize(x).collect()) + } } test("kryo with resizable output buffer should succeed on large array") { @@ -43,8 +42,8 @@ class KryoSerializerResizableOutputSuite extends SparkFunSuite { conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") conf.set("spark.kryoserializer.buffer", "1m") conf.set("spark.kryoserializer.buffer.max", "2m") - val sc = new SparkContext("local", "test", conf) - assert(sc.parallelize(x).collect() === x) - LocalSparkContext.stop(sc) + withSpark(new SparkContext("local", "test", conf)) { sc => + assert(sc.parallelize(x).collect() === x) + } } } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index afe2e80358ca..eaec098b8d78 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -17,22 +17,26 @@ package org.apache.spark.serializer -import java.io.{ByteArrayInputStream, ByteArrayOutputStream} +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, FileInputStream, FileOutputStream} import scala.collection.JavaConverters._ import scala.collection.mutable import scala.reflect.ClassTag -import com.esotericsoftware.kryo.Kryo +import com.esotericsoftware.kryo.{Kryo, KryoException} +import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} +import org.roaringbitmap.RoaringBitmap import org.apache.spark.{SharedSparkContext, SparkConf, SparkFunSuite} import org.apache.spark.scheduler.HighlyCompressedMapStatus import org.apache.spark.serializer.KryoTest._ import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.Utils class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) + conf.set("spark.kryo.unsafe", "false") test("SPARK-7392 configuration limits") { val kryoBufferProperty = "spark.kryoserializer.buffer" @@ -72,6 +76,9 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { } test("basic types") { + val conf = new SparkConf(false) + conf.set("spark.kryo.registrationRequired", "true") + val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) @@ -97,11 +104,14 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { check(Array("aaa", "bbb", null)) check(Array(true, false, true)) check(Array('a', 'b', 'c')) - check(Array[Int]()) + check(Array.empty[Int]) check(Array(Array("1", "2"), Array("1", "2", "3", "4"))) } test("pairs") { + val conf = new SparkConf(false) + conf.set("spark.kryo.registrationRequired", "true") + val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) @@ -126,12 +136,16 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { } test("Scala data structures") { + val conf = new SparkConf(false) + conf.set("spark.kryo.registrationRequired", "true") + val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } check(List[Int]()) check(List[Int](1, 2, 3)) + check(Seq[Int](1, 2, 3)) check(List[String]()) check(List[String]("x", "y", "z")) check(None) @@ -144,10 +158,10 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { check(mutable.Map("one" -> 1, "two" -> 2)) check(mutable.HashMap(1 -> "one", 2 -> "two")) check(mutable.HashMap("one" -> 1, "two" -> 2)) - check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) + check(List(Some(mutable.HashMap(1 -> 1, 2 -> 2)), None, Some(mutable.HashMap(3 -> 4)))) check(List( mutable.HashMap("one" -> 1, "two" -> 2), - mutable.HashMap(1->"one", 2->"two", 3->"three"))) + mutable.HashMap(1 -> "one", 2 -> "two", 3 -> "three"))) } test("Bug: SPARK-10251") { @@ -174,10 +188,10 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { check(mutable.Map("one" -> 1, "two" -> 2)) check(mutable.HashMap(1 -> "one", 2 -> "two")) check(mutable.HashMap("one" -> 1, "two" -> 2)) - check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) + check(List(Some(mutable.HashMap(1 -> 1, 2 -> 2)), None, Some(mutable.HashMap(3 -> 4)))) check(List( mutable.HashMap("one" -> 1, "two" -> 2), - mutable.HashMap(1->"one", 2->"two", 3->"three"))) + mutable.HashMap(1 -> "one", 2 -> "two", 3 -> "three"))) } test("ranges") { @@ -262,7 +276,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { } test("kryo with collect for specialized tuples") { - assert (sc.parallelize( Array((1, 11), (2, 22), (3, 33)) ).collect().head === (1, 11)) + assert (sc.parallelize( Array((1, 11), (2, 22), (3, 33)) ).collect().head === ((1, 11))) } test("kryo with SerializableHyperLogLog") { @@ -279,8 +293,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("kryo with fold") { val control = 1 :: 2 :: Nil // zeroValue must not be a ClassWithoutNoArgConstructor instance because it will be - // serialized by spark.closure.serializer but spark.closure.serializer only supports - // the default Java serializer. + // serialized by the Java serializer. val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)) .fold(null)((t1, t2) => { val t1x = if (t1 == null) 0 else t1.x @@ -322,6 +335,12 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val conf = new SparkConf(false) conf.set("spark.kryo.registrationRequired", "true") + // these cases require knowing the internals of RoaringBitmap a little. Blocks span 2^16 + // values, and they use a bitmap (dense) if they have more than 4096 values, and an + // array (sparse) if they use less. So we just create two cases, one sparse and one dense. + // and we use a roaring bitmap for the empty blocks, so we trigger the dense case w/ mostly + // empty blocks + val ser = new KryoSerializer(conf).newInstance() val denseBlockSizes = new Array[Long](5000) val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L) @@ -342,6 +361,45 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val ser = new KryoSerializer(conf).newInstance() val thrown = intercept[SparkException](ser.serialize(largeObject)) assert(thrown.getMessage.contains(kryoBufferMaxProperty)) + assert(thrown.getCause.isInstanceOf[KryoException]) + } + + test("SPARK-12222: deserialize RoaringBitmap throw Buffer underflow exception") { + val dir = Utils.createTempDir() + val tmpfile = dir.toString + "/RoaringBitmap" + val outStream = new FileOutputStream(tmpfile) + val output = new KryoOutput(outStream) + val bitmap = new RoaringBitmap + bitmap.add(1) + bitmap.add(3) + bitmap.add(5) + // Ignore Kryo because it doesn't use writeObject + bitmap.serialize(new KryoOutputObjectOutputBridge(null, output)) + output.flush() + output.close() + + val inStream = new FileInputStream(tmpfile) + val input = new KryoInput(inStream) + val ret = new RoaringBitmap + // Ignore Kryo because it doesn't use readObject + ret.deserialize(new KryoInputObjectInputBridge(null, input)) + input.close() + assert(ret == bitmap) + Utils.deleteRecursively(dir) + } + + test("KryoOutputObjectOutputBridge.writeObject and KryoInputObjectInputBridge.readObject") { + val kryo = new KryoSerializer(conf).newKryo() + + val bytesOutput = new ByteArrayOutputStream() + val objectOutput = new KryoOutputObjectOutputBridge(kryo, new KryoOutput(bytesOutput)) + objectOutput.writeObject("test") + objectOutput.close() + + val bytesInput = new ByteArrayInputStream(bytesOutput.toByteArray) + val objectInput = new KryoInputObjectInputBridge(kryo, new KryoInput(bytesInput)) + assert(objectInput.readObject() === "test") + objectInput.close() } test("getAutoReset") { @@ -417,7 +475,7 @@ class KryoSerializerAutoResetDisabledSuite extends SparkFunSuite with SharedSpar val deserializationStream = serInstance.deserializeStream(new ByteArrayInputStream(worldWorld)) assert(deserializationStream.readValue[Any]() === world) deserializationStream.close() - assert(serInstance.deserialize[Any](helloHello) === (hello, hello)) + assert(serInstance.deserialize[Any](helloHello) === ((hello, hello))) } } @@ -430,6 +488,9 @@ object KryoTest { class ClassWithNoArgConstructor { var x: Int = 0 + + override def hashCode(): Int = x + override def equals(other: Any): Boolean = other match { case c: ClassWithNoArgConstructor => x == c.x case _ => false @@ -437,6 +498,8 @@ object KryoTest { } class ClassWithoutNoArgConstructor(val x: Int) { + override def hashCode(): Int = x + override def equals(other: Any): Boolean = other match { case c: ClassWithoutNoArgConstructor => x == c.x case _ => false diff --git a/core/src/test/scala/org/apache/spark/serializer/SerializationDebuggerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/SerializationDebuggerSuite.scala index 2d5e9d66b2e1..912a516dff0f 100644 --- a/core/src/test/scala/org/apache/spark/serializer/SerializationDebuggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/SerializationDebuggerSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.serializer import java.io._ +import scala.annotation.meta.param + import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkFunSuite @@ -29,6 +31,7 @@ class SerializationDebuggerSuite extends SparkFunSuite with BeforeAndAfterEach { import SerializationDebugger.find override def beforeEach(): Unit = { + super.beforeEach() SerializationDebugger.enableDebugging = true } @@ -123,7 +126,11 @@ class SerializationDebuggerSuite extends SparkFunSuite with BeforeAndAfterEach { assert(find(new SerializableClassWithWriteReplace(new SerializableClass1)).isEmpty) } - test("object containing writeObject() and not serializable field") { + test("no infinite loop with writeReplace() which returns class of its own type") { + assert(find(new SerializableClassWithRecursiveWriteReplace).isEmpty) + } + + test("object containing writeObject() and not serializable field") { val s = find(new SerializableClassWithWriteObject(new NotSerializable)) assert(s.size === 3) assert(s(0).contains("NotSerializable")) @@ -190,7 +197,7 @@ class SerializationDebuggerSuite extends SparkFunSuite with BeforeAndAfterEach { } val originalException = new NotSerializableException("someClass") - // verify thaht original exception is returned on failure + // verify that original exception is returned on failure assert(SerializationDebugger.improveException(o, originalException).eq(originalException)) } } @@ -218,7 +225,7 @@ class SerializableClassWithWriteObject(val objectField: Object) extends Serializ } -class SerializableClassWithWriteReplace(@transient replacementFieldObject: Object) +class SerializableClassWithWriteReplace(@(transient @param) replacementFieldObject: Object) extends Serializable { private def writeReplace(): Object = { replacementFieldObject @@ -226,6 +233,13 @@ class SerializableClassWithWriteReplace(@transient replacementFieldObject: Objec } +class SerializableClassWithRecursiveWriteReplace extends Serializable { + private def writeReplace(): Object = { + new SerializableClassWithRecursiveWriteReplace + } +} + + class ExternalizableClass(objectField: Object) extends java.io.Externalizable { val serializableObjectField = new SerializableClass1 diff --git a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala index 4ce3b941bea5..99882bf76e29 100644 --- a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.serializer.KryoTest.RegistratorWithoutAutoReset /** * Tests to ensure that [[Serializer]] implementations obey the API contracts for methods that * describe properties of the serialized stream, such as - * [[Serializer.supportsRelocationOfSerializedObjects]]. + * `Serializer.supportsRelocationOfSerializedObjects`. */ class SerializerPropertiesSuite extends SparkFunSuite { diff --git a/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala b/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala index c1e0a29a34bb..17037870f7a1 100644 --- a/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala +++ b/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala @@ -17,12 +17,11 @@ package org.apache.spark.serializer -import java.io.{EOFException, OutputStream, InputStream} +import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer import scala.reflect.ClassTag - /** * A serializer implementation that always returns two elements in a deserialization stream. */ diff --git a/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala new file mode 100644 index 000000000000..d63a45ae4a6a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.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.serializer + +class UnsafeKryoSerializerSuite extends KryoSerializerSuite { + + // This test suite should run all tests in KryoSerializerSuite with kryo unsafe. + + override def beforeAll() { + conf.set("spark.kryo.unsafe", "true") + super.beforeAll() + } + + override def afterAll() { + conf.set("spark.kryo.unsafe", "false") + super.afterAll() + } +} diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 26a372d6a905..dba1172d5fdb 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -20,14 +20,11 @@ package org.apache.spark.shuffle import java.io.{ByteArrayOutputStream, InputStream} import java.nio.ByteBuffer -import org.mockito.Matchers.{eq => meq, _} import org.mockito.Mockito.{mock, when} -import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.apache.spark._ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} -import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.serializer.{JavaSerializer, SerializerManager} import org.apache.spark.storage.{BlockManager, BlockManagerId, ShuffleBlockId} /** @@ -77,13 +74,6 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // can ensure retain() and release() are properly called. val blockManager = mock(classOf[BlockManager]) - // Create a return function to use for the mocked wrapForCompression method that just returns - // the original input stream. - val dummyCompressionFunction = new Answer[InputStream] { - override def answer(invocation: InvocationOnMock): InputStream = - invocation.getArguments()(1).asInstanceOf[InputStream] - } - // Create a buffer with some randomly generated key-value pairs to use as the shuffle data // from each mappers (all mappers return the same shuffle data). val byteOutputStream = new ByteArrayOutputStream() @@ -105,9 +95,6 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // fetch shuffle data. val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId) when(blockManager.getBlockData(shuffleBlockId)).thenReturn(managedBuffer) - when(blockManager.wrapForCompression(meq(shuffleBlockId), isA(classOf[InputStream]))) - .thenAnswer(dummyCompressionFunction) - managedBuffer } @@ -127,17 +114,24 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // Create a mocked shuffle handle to pass into HashShuffleReader. val shuffleHandle = { val dependency = mock(classOf[ShuffleDependency[Int, Int, Int]]) - when(dependency.serializer).thenReturn(Some(serializer)) + when(dependency.serializer).thenReturn(serializer) when(dependency.aggregator).thenReturn(None) when(dependency.keyOrdering).thenReturn(None) new BaseShuffleHandle(shuffleId, numMaps, dependency) } + val serializerManager = new SerializerManager( + serializer, + new SparkConf() + .set("spark.shuffle.compress", "false") + .set("spark.shuffle.spill.compress", "false")) + val shuffleReader = new BlockStoreShuffleReader( shuffleHandle, reduceId, reduceId + 1, TaskContext.empty(), + serializerManager, blockManager, mapOutputTracker) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index b92a302806f7..85ccb3347104 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -23,8 +23,8 @@ import java.util.UUID import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.Matchers._ import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock @@ -32,9 +32,9 @@ import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfterEach import org.apache.spark._ -import org.apache.spark.executor.{TaskMetrics, ShuffleWriteMetrics} +import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} +import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.IndexShuffleBlockResolver -import org.apache.spark.serializer.{JavaSerializer, SerializerInstance} import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -55,6 +55,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte private var shuffleHandle: BypassMergeSortShuffleHandle[Int, Int] = _ override def beforeEach(): Unit = { + super.beforeEach() tempDir = Utils.createTempDir() outputFile = File.createTempFile("shuffle", null, tempDir) taskMetrics = new TaskMetrics @@ -65,9 +66,20 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte dependency = dependency ) when(dependency.partitioner).thenReturn(new HashPartitioner(7)) - when(dependency.serializer).thenReturn(Some(new JavaSerializer(conf))) + when(dependency.serializer).thenReturn(new JavaSerializer(conf)) when(taskContext.taskMetrics()).thenReturn(taskMetrics) when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) + doAnswer(new Answer[Void] { + def answer(invocationOnMock: InvocationOnMock): Void = { + val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] + if (tmp != null) { + outputFile.delete + tmp.renameTo(outputFile) + } + null + } + }).when(blockResolver) + .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) when(blockManager.diskBlockManager).thenReturn(diskBlockManager) when(blockManager.getDiskWriter( any[BlockId], @@ -78,13 +90,15 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte )).thenAnswer(new Answer[DiskBlockObjectWriter] { override def answer(invocation: InvocationOnMock): DiskBlockObjectWriter = { val args = invocation.getArguments + val manager = new SerializerManager(new JavaSerializer(conf), conf) new DiskBlockObjectWriter( args(1).asInstanceOf[File], + manager, args(2).asInstanceOf[SerializerInstance], args(3).asInstanceOf[Int], - compressStream = identity, syncWrites = false, - args(4).asInstanceOf[ShuffleWriteMetrics] + args(4).asInstanceOf[ShuffleWriteMetrics], + blockId = args(0).asInstanceOf[BlockId] ) } }) @@ -92,9 +106,9 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte new Answer[(TempShuffleBlockId, File)] { override def answer(invocation: InvocationOnMock): (TempShuffleBlockId, File) = { val blockId = new TempShuffleBlockId(UUID.randomUUID) - val file = File.createTempFile(blockId.toString, null, tempDir) + val file = new File(tempDir, blockId.name) blockIdToFileMap.put(blockId, file) - temporaryFilesCreated.append(file) + temporaryFilesCreated += file (blockId, file) } }) @@ -107,9 +121,13 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte } override def afterEach(): Unit = { - Utils.deleteRecursively(tempDir) - blockIdToFileMap.clear() - temporaryFilesCreated.clear() + try { + Utils.deleteRecursively(tempDir) + blockIdToFileMap.clear() + temporaryFilesCreated.clear() + } finally { + super.afterEach() + } } test("write empty iterator") { @@ -127,9 +145,9 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte assert(outputFile.exists()) assert(outputFile.length() === 0) assert(temporaryFilesCreated.isEmpty) - val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics.get - assert(shuffleWriteMetrics.shuffleBytesWritten === 0) - assert(shuffleWriteMetrics.shuffleRecordsWritten === 0) + val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics + assert(shuffleWriteMetrics.bytesWritten === 0) + assert(shuffleWriteMetrics.recordsWritten === 0) assert(taskMetrics.diskBytesSpilled === 0) assert(taskMetrics.memoryBytesSpilled === 0) } @@ -149,14 +167,50 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte writer.stop( /* success = */ true) assert(temporaryFilesCreated.nonEmpty) assert(writer.getPartitionLengths.sum === outputFile.length()) + assert(writer.getPartitionLengths.count(_ == 0L) === 4) // should be 4 zero length files assert(temporaryFilesCreated.count(_.exists()) === 0) // check that temporary files were deleted - val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics.get - assert(shuffleWriteMetrics.shuffleBytesWritten === outputFile.length()) - assert(shuffleWriteMetrics.shuffleRecordsWritten === records.length) + val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics + assert(shuffleWriteMetrics.bytesWritten === outputFile.length()) + assert(shuffleWriteMetrics.recordsWritten === records.length) assert(taskMetrics.diskBytesSpilled === 0) assert(taskMetrics.memoryBytesSpilled === 0) } + test("only generate temp shuffle file for non-empty partition") { + // Using exception to test whether only non-empty partition creates temp shuffle file, + // because temp shuffle file will only be cleaned after calling stop(false) in the failure + // case, so we could use it to validate the temp shuffle files. + def records: Iterator[(Int, Int)] = + Iterator((1, 1), (5, 5)) ++ + (0 until 100000).iterator.map { i => + if (i == 99990) { + throw new SparkException("intentional failure") + } else { + (2, 2) + } + } + + val writer = new BypassMergeSortShuffleWriter[Int, Int]( + blockManager, + blockResolver, + shuffleHandle, + 0, // MapId + taskContext, + conf + ) + + intercept[SparkException] { + writer.write(records) + } + + assert(temporaryFilesCreated.nonEmpty) + // Only 3 temp shuffle files will be created + assert(temporaryFilesCreated.count(_.exists()) === 3) + + writer.stop( /* success = */ false) + assert(temporaryFilesCreated.count(_.exists()) === 0) // check that temporary files were deleted + } + test("cleanup of intermediate files after errors") { val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala new file mode 100644 index 000000000000..d21ce73f4021 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle.sort + +import java.io.{File, FileInputStream, FileOutputStream} + +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Answers.RETURNS_SMART_NULLS +import org.mockito.Matchers._ +import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.shuffle.IndexShuffleBlockResolver +import org.apache.spark.storage._ +import org.apache.spark.util.Utils + + +class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEach { + + @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ + @Mock(answer = RETURNS_SMART_NULLS) private var diskBlockManager: DiskBlockManager = _ + + private var tempDir: File = _ + private val conf: SparkConf = new SparkConf(loadDefaults = false) + + override def beforeEach(): Unit = { + super.beforeEach() + tempDir = Utils.createTempDir() + MockitoAnnotations.initMocks(this) + + when(blockManager.diskBlockManager).thenReturn(diskBlockManager) + when(diskBlockManager.getFile(any[BlockId])).thenAnswer( + new Answer[File] { + override def answer(invocation: InvocationOnMock): File = { + new File(tempDir, invocation.getArguments.head.toString) + } + }) + } + + override def afterEach(): Unit = { + try { + Utils.deleteRecursively(tempDir) + } finally { + super.afterEach() + } + } + + test("commit shuffle files multiple times") { + val resolver = new IndexShuffleBlockResolver(conf, blockManager) + val lengths = Array[Long](10, 0, 20) + val dataTmp = File.createTempFile("shuffle", null, tempDir) + val out = new FileOutputStream(dataTmp) + Utils.tryWithSafeFinally { + out.write(new Array[Byte](30)) + } { + out.close() + } + resolver.writeIndexFileAndCommit(1, 2, lengths, dataTmp) + + val dataFile = resolver.getDataFile(1, 2) + assert(dataFile.exists()) + assert(dataFile.length() === 30) + assert(!dataTmp.exists()) + + val lengths2 = new Array[Long](3) + val dataTmp2 = File.createTempFile("shuffle", null, tempDir) + val out2 = new FileOutputStream(dataTmp2) + Utils.tryWithSafeFinally { + out2.write(Array[Byte](1)) + out2.write(new Array[Byte](29)) + } { + out2.close() + } + resolver.writeIndexFileAndCommit(1, 2, lengths2, dataTmp2) + assert(lengths2.toSeq === lengths.toSeq) + assert(dataFile.exists()) + assert(dataFile.length() === 30) + assert(!dataTmp2.exists()) + + // The dataFile should be the previous one + val firstByte = new Array[Byte](1) + val in = new FileInputStream(dataFile) + Utils.tryWithSafeFinally { + in.read(firstByte) + } { + in.close() + } + assert(firstByte(0) === 0) + + // remove data file + dataFile.delete() + + val lengths3 = Array[Long](10, 10, 15) + val dataTmp3 = File.createTempFile("shuffle", null, tempDir) + val out3 = new FileOutputStream(dataTmp3) + Utils.tryWithSafeFinally { + out3.write(Array[Byte](2)) + out3.write(new Array[Byte](34)) + } { + out3.close() + } + resolver.writeIndexFileAndCommit(1, 2, lengths3, dataTmp3) + assert(lengths3.toSeq != lengths.toSeq) + assert(dataFile.exists()) + assert(dataFile.length() === 35) + assert(!dataTmp2.exists()) + + // The dataFile should be the previous one + val firstByte2 = new Array[Byte](1) + val in2 = new FileInputStream(dataFile) + Utils.tryWithSafeFinally { + in2.read(firstByte2) + } { + in2.close() + } + assert(firstByte2(0) === 2) + } +} diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala index 8744a072cb3f..55cebe7c8b6a 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala @@ -41,7 +41,7 @@ class SortShuffleManagerSuite extends SparkFunSuite with Matchers { private def shuffleDep( partitioner: Partitioner, - serializer: Option[Serializer], + serializer: Serializer, keyOrdering: Option[Ordering[Any]], aggregator: Option[Aggregator[Any, Any, Any]], mapSideCombine: Boolean): ShuffleDependency[Any, Any, Any] = { @@ -56,7 +56,7 @@ class SortShuffleManagerSuite extends SparkFunSuite with Matchers { } test("supported shuffle dependencies for serialized shuffle") { - val kryo = Some(new KryoSerializer(new SparkConf())) + val kryo = new KryoSerializer(new SparkConf()) assert(canUseSerializedShuffle(shuffleDep( partitioner = new HashPartitioner(2), @@ -88,8 +88,8 @@ class SortShuffleManagerSuite extends SparkFunSuite with Matchers { } test("unsupported shuffle dependencies for serialized shuffle") { - val kryo = Some(new KryoSerializer(new SparkConf())) - val java = Some(new JavaSerializer(new SparkConf())) + val kryo = new KryoSerializer(new SparkConf()) + val java = new JavaSerializer(new SparkConf()) // We only support serializers that support object relocation assert(!canUseSerializedShuffle(shuffleDep( diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala new file mode 100644 index 000000000000..82bd7c4ff660 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.status.api.v1 + +import java.util.Date + +import scala.collection.mutable.LinkedHashMap + +import org.apache.spark.SparkFunSuite +import org.apache.spark.scheduler.{StageInfo, TaskInfo, TaskLocality} +import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData} + +class AllStagesResourceSuite extends SparkFunSuite { + + def getFirstTaskLaunchTime(taskLaunchTimes: Seq[Long]): Option[Date] = { + val tasks = new LinkedHashMap[Long, TaskUIData] + taskLaunchTimes.zipWithIndex.foreach { case (time, idx) => + tasks(idx.toLong) = TaskUIData( + new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false)) + } + + val stageUiData = new StageUIData() + stageUiData.taskData = tasks + val status = StageStatus.ACTIVE + val stageInfo = new StageInfo( + 1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc") + val stageData = AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, false) + + stageData.firstTaskLaunchedTime + } + + test("firstTaskLaunchedTime when there are no tasks") { + val result = getFirstTaskLaunchTime(Seq()) + assert(result == None) + } + + test("firstTaskLaunchedTime when there are tasks but none launched") { + val result = getFirstTaskLaunchTime(Seq(-100L, -200L, -300L)) + assert(result == None) + } + + test("firstTaskLaunchedTime when there are tasks and some launched") { + val result = getFirstTaskLaunchTime(Seq(-100L, 1449255596000L, 1449255597000L)) + assert(result == Some(new Date(1449255596000L))) + } + +} diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala index 63b0e77629dd..18baeb1cb9c7 100644 --- a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala @@ -26,7 +26,8 @@ class SimpleDateParamSuite extends SparkFunSuite with Matchers { test("date parsing") { new SimpleDateParam("2015-02-20T23:21:17.190GMT").timestamp should be (1424474477190L) - new SimpleDateParam("2015-02-20T17:21:17.190EST").timestamp should be (1424470877190L) + // don't use EST, it is ambiguous, use -0500 instead, see SPARK-15723 + new SimpleDateParam("2015-02-20T17:21:17.190-0500").timestamp should be (1424470877190L) new SimpleDateParam("2015-02-20").timestamp should be (1424390400000L) // GMT intercept[WebApplicationException] { new SimpleDateParam("invalid date") diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala index 89ed031b6fcd..f0c521b00b58 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.storage +import java.util.UUID + import org.apache.spark.SparkFunSuite class BlockIdSuite extends SparkFunSuite { @@ -67,6 +69,32 @@ class BlockIdSuite extends SparkFunSuite { assertSame(id, BlockId(id.toString)) } + test("shuffle data") { + val id = ShuffleDataBlockId(4, 5, 6) + assertSame(id, ShuffleDataBlockId(4, 5, 6)) + assertDifferent(id, ShuffleDataBlockId(6, 5, 6)) + assert(id.name === "shuffle_4_5_6.data") + assert(id.asRDDId === None) + assert(id.shuffleId === 4) + assert(id.mapId === 5) + assert(id.reduceId === 6) + assert(!id.isShuffle) + assertSame(id, BlockId(id.toString)) + } + + test("shuffle index") { + val id = ShuffleIndexBlockId(7, 8, 9) + assertSame(id, ShuffleIndexBlockId(7, 8, 9)) + assertDifferent(id, ShuffleIndexBlockId(9, 8, 9)) + assert(id.name === "shuffle_7_8_9.index") + assert(id.asRDDId === None) + assert(id.shuffleId === 7) + assert(id.mapId === 8) + assert(id.reduceId === 9) + assert(!id.isShuffle) + assertSame(id, BlockId(id.toString)) + } + test("broadcast") { val id = BroadcastBlockId(42) assertSame(id, BroadcastBlockId(42)) @@ -101,6 +129,30 @@ class BlockIdSuite extends SparkFunSuite { assertSame(id, BlockId(id.toString)) } + test("temp local") { + val id = TempLocalBlockId(new UUID(5, 2)) + assertSame(id, TempLocalBlockId(new UUID(5, 2))) + assertDifferent(id, TempLocalBlockId(new UUID(5, 3))) + assert(id.name === "temp_local_00000000-0000-0005-0000-000000000002") + assert(id.asRDDId === None) + assert(id.isBroadcast === false) + assert(id.id.getMostSignificantBits() === 5) + assert(id.id.getLeastSignificantBits() === 2) + assert(!id.isShuffle) + } + + test("temp shuffle") { + val id = TempShuffleBlockId(new UUID(1, 2)) + assertSame(id, TempShuffleBlockId(new UUID(1, 2))) + assertDifferent(id, TempShuffleBlockId(new UUID(1, 3))) + assert(id.name === "temp_shuffle_00000000-0000-0001-0000-000000000002") + assert(id.asRDDId === None) + assert(id.isBroadcast === false) + assert(id.id.getMostSignificantBits() === 1) + assert(id.id.getLeastSignificantBits() === 2) + assert(!id.isShuffle) + } + test("test") { val id = TestBlockId("abc") assertSame(id, TestBlockId("abc")) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala new file mode 100644 index 000000000000..917db766f7f1 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -0,0 +1,360 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.util.Properties + +import scala.concurrent.{Await, ExecutionContext, Future} +import scala.language.implicitConversions +import scala.reflect.ClassTag + +import org.scalatest.BeforeAndAfterEach +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.{SparkException, SparkFunSuite, TaskContext, TaskContextImpl} +import org.apache.spark.util.ThreadUtils + + +class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { + + private implicit val ec = ExecutionContext.global + private var blockInfoManager: BlockInfoManager = _ + + override protected def beforeEach(): Unit = { + super.beforeEach() + blockInfoManager = new BlockInfoManager() + for (t <- 0 to 4) { + blockInfoManager.registerTask(t) + } + } + + override protected def afterEach(): Unit = { + try { + blockInfoManager = null + } finally { + super.afterEach() + } + } + + private implicit def stringToBlockId(str: String): BlockId = { + TestBlockId(str) + } + + private def newBlockInfo(): BlockInfo = { + new BlockInfo(StorageLevel.MEMORY_ONLY, ClassTag.Any, tellMaster = false) + } + + private def withTaskId[T](taskAttemptId: Long)(block: => T): T = { + try { + TaskContext.setTaskContext( + new TaskContextImpl(0, 0, taskAttemptId, 0, null, new Properties, null)) + block + } finally { + TaskContext.unset() + } + } + + test("initial memory usage") { + assert(blockInfoManager.size === 0) + } + + test("get non-existent block") { + assert(blockInfoManager.get("non-existent-block").isEmpty) + assert(blockInfoManager.lockForReading("non-existent-block").isEmpty) + assert(blockInfoManager.lockForWriting("non-existent-block").isEmpty) + } + + test("basic lockNewBlockForWriting") { + val initialNumMapEntries = blockInfoManager.getNumberOfMapEntries + val blockInfo = newBlockInfo() + withTaskId(1) { + assert(blockInfoManager.lockNewBlockForWriting("block", blockInfo)) + assert(blockInfoManager.get("block").get eq blockInfo) + assert(blockInfo.readerCount === 0) + assert(blockInfo.writerTask === 1) + // Downgrade lock so that second call doesn't block: + blockInfoManager.downgradeLock("block") + assert(blockInfo.readerCount === 1) + assert(blockInfo.writerTask === BlockInfo.NO_WRITER) + assert(!blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + assert(blockInfo.readerCount === 2) + assert(blockInfoManager.get("block").get eq blockInfo) + assert(blockInfo.readerCount === 2) + assert(blockInfo.writerTask === BlockInfo.NO_WRITER) + blockInfoManager.unlock("block") + blockInfoManager.unlock("block") + assert(blockInfo.readerCount === 0) + assert(blockInfo.writerTask === BlockInfo.NO_WRITER) + } + assert(blockInfoManager.size === 1) + assert(blockInfoManager.getNumberOfMapEntries === initialNumMapEntries + 1) + } + + test("lockNewBlockForWriting blocks while write lock is held, then returns false after release") { + withTaskId(0) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + } + val lock1Future = Future { + withTaskId(1) { + blockInfoManager.lockNewBlockForWriting("block", newBlockInfo()) + } + } + val lock2Future = Future { + withTaskId(2) { + blockInfoManager.lockNewBlockForWriting("block", newBlockInfo()) + } + } + Thread.sleep(300) // Hack to try to ensure that both future tasks are waiting + withTaskId(0) { + blockInfoManager.downgradeLock("block") + } + // After downgrading to a read lock, both threads should wake up and acquire the shared + // read lock. + assert(!ThreadUtils.awaitResult(lock1Future, 1.seconds)) + assert(!ThreadUtils.awaitResult(lock2Future, 1.seconds)) + assert(blockInfoManager.get("block").get.readerCount === 3) + } + + test("lockNewBlockForWriting blocks while write lock is held, then returns true after removal") { + withTaskId(0) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + } + val lock1Future = Future { + withTaskId(1) { + blockInfoManager.lockNewBlockForWriting("block", newBlockInfo()) + } + } + val lock2Future = Future { + withTaskId(2) { + blockInfoManager.lockNewBlockForWriting("block", newBlockInfo()) + } + } + Thread.sleep(300) // Hack to try to ensure that both future tasks are waiting + withTaskId(0) { + blockInfoManager.removeBlock("block") + } + // After removing the block, the write lock is released. Both threads should wake up but only + // one should acquire the write lock. The second thread should block until the winner of the + // write race releases its lock. + val winningFuture: Future[Boolean] = + ThreadUtils.awaitReady(Future.firstCompletedOf(Seq(lock1Future, lock2Future)), 1.seconds) + assert(winningFuture.value.get.get) + val winningTID = blockInfoManager.get("block").get.writerTask + assert(winningTID === 1 || winningTID === 2) + val losingFuture: Future[Boolean] = if (winningTID == 1) lock2Future else lock1Future + assert(!losingFuture.isCompleted) + // Once the writer releases its lock, the blocked future should wake up again and complete. + withTaskId(winningTID) { + blockInfoManager.unlock("block") + } + assert(!ThreadUtils.awaitResult(losingFuture, 1.seconds)) + assert(blockInfoManager.get("block").get.readerCount === 1) + } + + test("read locks are reentrant") { + withTaskId(1) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.unlock("block") + assert(blockInfoManager.lockForReading("block").isDefined) + assert(blockInfoManager.lockForReading("block").isDefined) + assert(blockInfoManager.get("block").get.readerCount === 2) + assert(blockInfoManager.get("block").get.writerTask === BlockInfo.NO_WRITER) + blockInfoManager.unlock("block") + assert(blockInfoManager.get("block").get.readerCount === 1) + blockInfoManager.unlock("block") + assert(blockInfoManager.get("block").get.readerCount === 0) + } + } + + test("multiple tasks can hold read locks") { + withTaskId(0) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.unlock("block") + } + withTaskId(1) { assert(blockInfoManager.lockForReading("block").isDefined) } + withTaskId(2) { assert(blockInfoManager.lockForReading("block").isDefined) } + withTaskId(3) { assert(blockInfoManager.lockForReading("block").isDefined) } + withTaskId(4) { assert(blockInfoManager.lockForReading("block").isDefined) } + assert(blockInfoManager.get("block").get.readerCount === 4) + } + + test("single task can hold write lock") { + withTaskId(0) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.unlock("block") + } + withTaskId(1) { + assert(blockInfoManager.lockForWriting("block").isDefined) + assert(blockInfoManager.get("block").get.writerTask === 1) + } + withTaskId(2) { + assert(blockInfoManager.lockForWriting("block", blocking = false).isEmpty) + assert(blockInfoManager.get("block").get.writerTask === 1) + } + } + + test("cannot grab a writer lock while already holding a write lock") { + withTaskId(0) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.unlock("block") + } + withTaskId(1) { + assert(blockInfoManager.lockForWriting("block").isDefined) + assert(blockInfoManager.lockForWriting("block", false).isEmpty) + blockInfoManager.assertBlockIsLockedForWriting("block") + } + } + + test("assertBlockIsLockedForWriting throws exception if block is not locked") { + intercept[SparkException] { + blockInfoManager.assertBlockIsLockedForWriting("block") + } + withTaskId(BlockInfo.NON_TASK_WRITER) { + intercept[SparkException] { + blockInfoManager.assertBlockIsLockedForWriting("block") + } + } + } + + test("downgrade lock") { + withTaskId(0) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.downgradeLock("block") + } + withTaskId(1) { + assert(blockInfoManager.lockForReading("block").isDefined) + } + assert(blockInfoManager.get("block").get.readerCount === 2) + assert(blockInfoManager.get("block").get.writerTask === BlockInfo.NO_WRITER) + } + + test("write lock will block readers") { + withTaskId(0) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + } + val get1Future = Future { + withTaskId(1) { + blockInfoManager.lockForReading("block") + } + } + val get2Future = Future { + withTaskId(2) { + blockInfoManager.lockForReading("block") + } + } + Thread.sleep(300) // Hack to try to ensure that both future tasks are waiting + withTaskId(0) { + blockInfoManager.unlock("block") + } + assert(ThreadUtils.awaitResult(get1Future, 1.seconds).isDefined) + assert(ThreadUtils.awaitResult(get2Future, 1.seconds).isDefined) + assert(blockInfoManager.get("block").get.readerCount === 2) + } + + test("read locks will block writer") { + withTaskId(0) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.unlock("block") + blockInfoManager.lockForReading("block") + } + val write1Future = Future { + withTaskId(1) { + blockInfoManager.lockForWriting("block") + } + } + val write2Future = Future { + withTaskId(2) { + blockInfoManager.lockForWriting("block") + } + } + Thread.sleep(300) // Hack to try to ensure that both future tasks are waiting + withTaskId(0) { + blockInfoManager.unlock("block") + } + assert( + ThreadUtils.awaitResult( + Future.firstCompletedOf(Seq(write1Future, write2Future)), 1.seconds).isDefined) + val firstWriteWinner = if (write1Future.isCompleted) 1 else 2 + withTaskId(firstWriteWinner) { + blockInfoManager.unlock("block") + } + assert(ThreadUtils.awaitResult(write1Future, 1.seconds).isDefined) + assert(ThreadUtils.awaitResult(write2Future, 1.seconds).isDefined) + } + + test("removing a non-existent block throws IllegalArgumentException") { + withTaskId(0) { + intercept[IllegalArgumentException] { + blockInfoManager.removeBlock("non-existent-block") + } + } + } + + test("removing a block without holding any locks throws IllegalStateException") { + withTaskId(0) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.unlock("block") + intercept[IllegalStateException] { + blockInfoManager.removeBlock("block") + } + } + } + + test("removing a block while holding only a read lock throws IllegalStateException") { + withTaskId(0) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.unlock("block") + assert(blockInfoManager.lockForReading("block").isDefined) + intercept[IllegalStateException] { + blockInfoManager.removeBlock("block") + } + } + } + + test("removing a block causes blocked callers to receive None") { + withTaskId(0) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + } + val getFuture = Future { + withTaskId(1) { + blockInfoManager.lockForReading("block") + } + } + val writeFuture = Future { + withTaskId(2) { + blockInfoManager.lockForWriting("block") + } + } + Thread.sleep(300) // Hack to try to ensure that both future tasks are waiting + withTaskId(0) { + blockInfoManager.removeBlock("block") + } + assert(ThreadUtils.awaitResult(getFuture, 1.seconds).isEmpty) + assert(ThreadUtils.awaitResult(writeFuture, 1.seconds).isEmpty) + } + + test("releaseAllLocksForTask releases write locks") { + val initialNumMapEntries = blockInfoManager.getNumberOfMapEntries + withTaskId(0) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + } + assert(blockInfoManager.getNumberOfMapEntries === initialNumMapEntries + 3) + blockInfoManager.releaseAllLocksForTask(0) + assert(blockInfoManager.getNumberOfMapEntries === initialNumMapEntries) + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index 6e3f500e15dc..dd61dcd11bcd 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.storage +import java.util.Locale + import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.implicitConversions @@ -26,43 +28,52 @@ import org.mockito.Mockito.{mock, when} import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually._ -import org.apache.spark.network.netty.NettyBlockTransferService -import org.apache.spark.rpc.RpcEnv import org.apache.spark._ -import org.apache.spark.memory.StaticMemoryManager +import org.apache.spark.broadcast.BroadcastManager +import org.apache.spark.internal.Logging +import org.apache.spark.memory.UnifiedMemoryManager import org.apache.spark.network.BlockTransferService +import org.apache.spark.network.netty.NettyBlockTransferService +import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.LiveListenerBus -import org.apache.spark.serializer.KryoSerializer -import org.apache.spark.shuffle.hash.HashShuffleManager +import org.apache.spark.serializer.{KryoSerializer, SerializerManager} +import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.storage.StorageLevel._ +import org.apache.spark.util.Utils + +trait BlockManagerReplicationBehavior extends SparkFunSuite + with Matchers + with BeforeAndAfter + with LocalSparkContext { -/** Testsuite that tests block replication in BlockManager */ -class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with BeforeAndAfter { + val conf: SparkConf - private val conf = new SparkConf(false).set("spark.app.id", "test") - private var rpcEnv: RpcEnv = null - private var master: BlockManagerMaster = null - private val securityMgr = new SecurityManager(conf) - private val mapOutputTracker = new MapOutputTrackerMaster(conf) - private val shuffleManager = new HashShuffleManager(conf) + protected var rpcEnv: RpcEnv = null + protected var master: BlockManagerMaster = null + protected lazy val securityMgr = new SecurityManager(conf) + protected lazy val bcastManager = new BroadcastManager(true, conf, securityMgr) + protected lazy val mapOutputTracker = new MapOutputTrackerMaster(conf, bcastManager, true) + protected lazy val shuffleManager = new SortShuffleManager(conf) // List of block manager created during an unit test, so that all of the them can be stopped // after the unit test. - private val allStores = new ArrayBuffer[BlockManager] + protected val allStores = new ArrayBuffer[BlockManager] // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test - conf.set("spark.kryoserializer.buffer", "1m") - private val serializer = new KryoSerializer(conf) + protected lazy val serializer = new KryoSerializer(conf) // Implicitly convert strings to BlockIds for test clarity. - private implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) + protected implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) - private def makeBlockManager( + protected def makeBlockManager( maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { - val transfer = new NettyBlockTransferService(conf, securityMgr, numCores = 1) - val memManager = new StaticMemoryManager(conf, Long.MaxValue, maxMem, numCores = 1) - val store = new BlockManager(name, rpcEnv, master, serializer, conf, + conf.set("spark.testing.memory", maxMem.toString) + conf.set("spark.memory.offHeap.size", maxMem.toString) + val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1) + val memManager = UnifiedMemoryManager(conf, numCores = 1) + val serializerManager = new SerializerManager(serializer, conf) + val store = new BlockManager(name, rpcEnv, master, serializerManager, conf, memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) memManager.setMemoryStore(store.memoryStore) store.initialize("app-id") @@ -75,6 +86,9 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo conf.set("spark.authenticate", "false") conf.set("spark.driver.port", rpcEnv.address.port.toString) + conf.set("spark.testing", "true") + conf.set("spark.memory.fraction", "1") + conf.set("spark.memory.storageFraction", "1") conf.set("spark.storage.unrollFraction", "0.4") conf.set("spark.storage.unrollMemoryThreshold", "512") @@ -83,8 +97,10 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo // to make cached peers refresh frequently conf.set("spark.storage.cachedPeersTtl", "10") + sc = new SparkContext("local", "test", conf) master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", - new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus)), conf, true) + new BlockManagerMasterEndpoint(rpcEnv, true, conf, + new LiveListenerBus(conf))), conf, true) allStores.clear() } @@ -171,6 +187,10 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo testReplication(5, storageLevels) } + test("block replication - off-heap") { + testReplication(2, Seq(OFF_HEAP, StorageLevel(true, true, true, false, 2))) + } + test("block replication - 2x replication without peers") { intercept[org.scalatest.exceptions.TestFailedException] { testReplication(1, @@ -261,8 +281,10 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo val failableTransfer = mock(classOf[BlockTransferService]) // this wont actually work when(failableTransfer.hostName).thenReturn("some-hostname") when(failableTransfer.port).thenReturn(1000) - val memManager = new StaticMemoryManager(conf, Long.MaxValue, 10000, numCores = 1) - val failableStore = new BlockManager("failable-store", rpcEnv, master, serializer, conf, + conf.set("spark.testing.memory", "10000") + val memManager = UnifiedMemoryManager(conf, numCores = 1) + val serializerManager = new SerializerManager(serializer, conf) + val failableStore = new BlockManager("failable-store", rpcEnv, master, serializerManager, conf, memManager, mapOutputTracker, shuffleManager, failableTransfer, securityMgr, 0) memManager.setMemoryStore(failableStore.memoryStore) failableStore.initialize("app-id") @@ -327,6 +349,8 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo } } + + /** * Test replication of blocks with different storage levels (various combinations of * memory, disk & serialization). For each storage level, this function tests every store @@ -334,7 +358,7 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo * is correct. Then it also drops the block from memory of each store (using LRU) and * again checks whether the master's knowledge gets updated. */ - private def testReplication(maxReplication: Int, storageLevels: Seq[StorageLevel]) { + protected def testReplication(maxReplication: Int, storageLevels: Seq[StorageLevel]) { import org.apache.spark.storage.StorageLevel._ assert(maxReplication > 1, @@ -352,9 +376,10 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo storageLevels.foreach { storageLevel => // Put the block into one of the stores - val blockId = new TestBlockId( - "block-with-" + storageLevel.description.replace(" ", "-").toLowerCase) - stores(0).putSingle(blockId, new Array[Byte](blockSize), storageLevel) + val blockId = TestBlockId( + "block-with-" + storageLevel.description.replace(" ", "-").toLowerCase(Locale.ROOT)) + val testValue = Array.fill[Byte](blockSize)(1) + stores(0).putSingle(blockId, testValue, storageLevel) // Assert that master know two locations for the block val blockLocations = master.getLocations(blockId).map(_.executorId).toSet @@ -366,10 +391,23 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo testStore => blockLocations.contains(testStore.blockManagerId.executorId) }.foreach { testStore => val testStoreName = testStore.blockManagerId.executorId - assert(testStore.getLocal(blockId).isDefined, s"$blockId was not found in $testStoreName") + val blockResultOpt = testStore.getLocalValues(blockId) + assert(blockResultOpt.isDefined, s"$blockId was not found in $testStoreName") + val localValues = blockResultOpt.get.data.toSeq + assert(localValues.size == 1) + assert(localValues.head === testValue) assert(master.getLocations(blockId).map(_.executorId).toSet.contains(testStoreName), s"master does not have status for ${blockId.name} in $testStoreName") + val memoryStore = testStore.memoryStore + if (memoryStore.contains(blockId) && !storageLevel.deserialized) { + memoryStore.getBytes(blockId).get.chunks.foreach { byteBuffer => + assert(storageLevel.useOffHeap == byteBuffer.isDirect, + s"memory mode ${storageLevel.memoryMode} is not compatible with " + + byteBuffer.getClass.getSimpleName) + } + } + val blockStatus = master.getBlockStatus(blockId)(testStore.blockManagerId) // Assert that block status in the master for this store has expected storage level @@ -388,10 +426,14 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo // If the block is supposed to be in memory, then drop the copy of the block in // this store test whether master is updated with zero memory usage this store if (storageLevel.useMemory) { + val sl = if (storageLevel.useOffHeap) { + StorageLevel(false, true, true, false, 1) + } else { + MEMORY_ONLY_SER + } // Force the block to be dropped by adding a number of dummy blocks (1 to 10).foreach { - i => - testStore.putSingle(s"dummy-block-$i", new Array[Byte](1000), MEMORY_ONLY_SER) + i => testStore.putSingle(s"dummy-block-$i", new Array[Byte](1000), sl) } (1 to 10).foreach { i => testStore.removeBlock(s"dummy-block-$i") @@ -421,3 +463,95 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo } } } + +class BlockManagerReplicationSuite extends BlockManagerReplicationBehavior { + val conf = new SparkConf(false).set("spark.app.id", "test") + conf.set("spark.kryoserializer.buffer", "1m") +} + +class BlockManagerProactiveReplicationSuite extends BlockManagerReplicationBehavior { + val conf = new SparkConf(false).set("spark.app.id", "test") + conf.set("spark.kryoserializer.buffer", "1m") + conf.set("spark.storage.replication.proactive", "true") + conf.set("spark.storage.exceptionOnPinLeak", "true") + + (2 to 5).foreach { i => + test(s"proactive block replication - $i replicas - ${i - 1} block manager deletions") { + testProactiveReplication(i) + } + } + + def testProactiveReplication(replicationFactor: Int) { + val blockSize = 1000 + val storeSize = 10000 + val initialStores = (1 to 10).map { i => makeBlockManager(storeSize, s"store$i") } + + val blockId = "a1" + + val storageLevel = StorageLevel(true, true, false, true, replicationFactor) + initialStores.head.putSingle(blockId, new Array[Byte](blockSize), storageLevel) + + val blockLocations = master.getLocations(blockId) + logInfo(s"Initial locations : $blockLocations") + + assert(blockLocations.size === replicationFactor) + + // remove a random blockManager + val executorsToRemove = blockLocations.take(replicationFactor - 1).toSet + logInfo(s"Removing $executorsToRemove") + initialStores.filter(bm => executorsToRemove.contains(bm.blockManagerId)).foreach { bm => + master.removeExecutor(bm.blockManagerId.executorId) + bm.stop() + // giving enough time for replication to happen and new block be reported to master + eventually(timeout(5 seconds), interval(100 millis)) { + val newLocations = master.getLocations(blockId).toSet + assert(newLocations.size === replicationFactor) + } + } + + val newLocations = eventually(timeout(5 seconds), interval(100 millis)) { + val _newLocations = master.getLocations(blockId).toSet + assert(_newLocations.size === replicationFactor) + _newLocations + } + logInfo(s"New locations : $newLocations") + + // new locations should not contain stopped block managers + assert(newLocations.forall(bmId => !executorsToRemove.contains(bmId)), + "New locations contain stopped block managers.") + + // Make sure all locks have been released. + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + initialStores.filter(bm => newLocations.contains(bm.blockManagerId)).foreach { bm => + assert(bm.blockInfoManager.getTaskLockCount(BlockInfo.NON_TASK_WRITER) === 0) + } + } + } +} + +class DummyTopologyMapper(conf: SparkConf) extends TopologyMapper(conf) with Logging { + // number of racks to test with + val numRacks = 3 + + /** + * Gets the topology information given the host name + * + * @param hostname Hostname + * @return random topology + */ + override def getTopologyForHost(hostname: String): Option[String] = { + Some(s"/Rack-${Utils.random.nextInt(numRacks)}") + } +} + +class BlockManagerBasicStrategyReplicationSuite extends BlockManagerReplicationBehavior { + val conf: SparkConf = new SparkConf(false).set("spark.app.id", "test") + conf.set("spark.kryoserializer.buffer", "1m") + conf.set( + "spark.storage.replication.policy", + classOf[BasicBlockReplicationPolicy].getName) + conf.set( + "spark.storage.replication.topologyMapper", + classOf[DummyTopologyMapper].getName) +} + diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index d49015afcd59..cfe89fde63f8 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -17,48 +17,64 @@ package org.apache.spark.storage -import java.nio.{ByteBuffer, MappedByteBuffer} -import java.util.Arrays +import java.io.File +import java.nio.ByteBuffer +import scala.collection.JavaConverters._ +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.concurrent.Future import scala.concurrent.duration._ -import scala.language.implicitConversions -import scala.language.postfixOps +import scala.language.{implicitConversions, postfixOps} +import scala.reflect.ClassTag -import org.mockito.Mockito.{mock, when} +import org.apache.commons.lang3.RandomUtils +import org.mockito.{Matchers => mc} +import org.mockito.Mockito.{mock, times, verify, when} import org.scalatest._ import org.scalatest.concurrent.Eventually._ -import org.scalatest.concurrent.Timeouts._ +import org.scalatest.concurrent.TimeLimits._ -import org.apache.spark.network.netty.NettyBlockTransferService -import org.apache.spark.rpc.RpcEnv import org.apache.spark._ +import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.executor.DataReadMethod -import org.apache.spark.memory.StaticMemoryManager +import org.apache.spark.internal.config._ +import org.apache.spark.memory.UnifiedMemoryManager +import org.apache.spark.network.{BlockDataManager, BlockTransferService, TransportContext} +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} +import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} +import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, TransportServerBootstrap} +import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient, TempShuffleFileManager} +import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor} +import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.LiveListenerBus -import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} -import org.apache.spark.shuffle.hash.HashShuffleManager +import org.apache.spark.security.{CryptoStreamUtils, EncryptionFunSuite} +import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, SerializerManager} +import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat import org.apache.spark.util._ - +import org.apache.spark.util.io.ChunkedByteBuffer class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach - with PrivateMethodTester with ResetSystemProperties { + with PrivateMethodTester with LocalSparkContext with ResetSystemProperties + with EncryptionFunSuite { + + import BlockManagerSuite._ - private val conf = new SparkConf(false).set("spark.app.id", "test") + var conf: SparkConf = null var store: BlockManager = null var store2: BlockManager = null var store3: BlockManager = null var rpcEnv: RpcEnv = null var master: BlockManagerMaster = null - conf.set("spark.authenticate", "false") - val securityMgr = new SecurityManager(conf) - val mapOutputTracker = new MapOutputTrackerMaster(conf) - val shuffleManager = new HashShuffleManager(conf) + val securityMgr = new SecurityManager(new SparkConf(false)) + val bcastManager = new BroadcastManager(true, new SparkConf(false), securityMgr) + val mapOutputTracker = new MapOutputTrackerMaster(new SparkConf(false), bcastManager, true) + val shuffleManager = new SortShuffleManager(new SparkConf(false)) // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test - conf.set("spark.kryoserializer.buffer", "1m") - val serializer = new KryoSerializer(conf) + val serializer = new KryoSerializer(new SparkConf(false).set("spark.kryoserializer.buffer", "1m")) // Implicitly convert strings to BlockIds for test clarity. implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) @@ -66,59 +82,90 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE private def makeBlockManager( maxMem: Long, - name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { - val transfer = new NettyBlockTransferService(conf, securityMgr, numCores = 1) - val memManager = new StaticMemoryManager(conf, Long.MaxValue, maxMem, numCores = 1) - val blockManager = new BlockManager(name, rpcEnv, master, serializer, conf, - memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) + name: String = SparkContext.DRIVER_IDENTIFIER, + master: BlockManagerMaster = this.master, + transferService: Option[BlockTransferService] = Option.empty, + testConf: Option[SparkConf] = None): BlockManager = { + val bmConf = testConf.map(_.setAll(conf.getAll)).getOrElse(conf) + bmConf.set("spark.testing.memory", maxMem.toString) + bmConf.set("spark.memory.offHeap.size", maxMem.toString) + val serializer = new KryoSerializer(bmConf) + val encryptionKey = if (bmConf.get(IO_ENCRYPTION_ENABLED)) { + Some(CryptoStreamUtils.createKey(bmConf)) + } else { + None + } + val bmSecurityMgr = new SecurityManager(bmConf, encryptionKey) + val transfer = transferService + .getOrElse(new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1)) + val memManager = UnifiedMemoryManager(bmConf, numCores = 1) + val serializerManager = new SerializerManager(serializer, bmConf) + val blockManager = new BlockManager(name, rpcEnv, master, serializerManager, bmConf, + memManager, mapOutputTracker, shuffleManager, transfer, bmSecurityMgr, 0) memManager.setMemoryStore(blockManager.memoryStore) blockManager.initialize("app-id") blockManager } override def beforeEach(): Unit = { - rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) - + super.beforeEach() // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case System.setProperty("os.arch", "amd64") - conf.set("os.arch", "amd64") - conf.set("spark.test.useCompressedOops", "true") + conf = new SparkConf(false) + .set("spark.app.id", "test") + .set("spark.testing", "true") + .set("spark.memory.fraction", "1") + .set("spark.memory.storageFraction", "1") + .set("spark.kryoserializer.buffer", "1m") + .set("spark.test.useCompressedOops", "true") + .set("spark.storage.unrollFraction", "0.4") + .set("spark.storage.unrollMemoryThreshold", "512") + + rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) conf.set("spark.driver.port", rpcEnv.address.port.toString) - conf.set("spark.storage.unrollFraction", "0.4") - conf.set("spark.storage.unrollMemoryThreshold", "512") + // Mock SparkContext to reduce the memory usage of tests. It's fine since the only reason we + // need to create a SparkContext is to initialize LiveListenerBus. + sc = mock(classOf[SparkContext]) + when(sc.conf).thenReturn(conf) master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", - new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus)), conf, true) + new BlockManagerMasterEndpoint(rpcEnv, true, conf, + new LiveListenerBus(conf))), conf, true) val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() } override def afterEach(): Unit = { - if (store != null) { - store.stop() - store = null - } - if (store2 != null) { - store2.stop() - store2 = null - } - if (store3 != null) { - store3.stop() - store3 = null + try { + conf = null + if (store != null) { + store.stop() + store = null + } + if (store2 != null) { + store2.stop() + store2 = null + } + if (store3 != null) { + store3.stop() + store3 = null + } + rpcEnv.shutdown() + rpcEnv.awaitTermination() + rpcEnv = null + master = null + } finally { + super.afterEach() } - rpcEnv.shutdown() - rpcEnv.awaitTermination() - rpcEnv = null - master = null } test("StorageLevel object caching") { - val level1 = StorageLevel(false, false, false, false, 3) + val level1 = StorageLevel(false, false, false, 3) // this should return the same object as level1 - val level2 = StorageLevel(false, false, false, false, 3) + val level2 = StorageLevel(false, false, false, 3) // this should return a different object - val level3 = StorageLevel(false, false, false, false, 2) + val level3 = StorageLevel(false, false, false, 2) assert(level2 === level1, "level2 is not same as level1") assert(level2.eq(level1), "level2 is not the same object as level1") assert(level3 != level1, "level3 is same as level1") @@ -167,9 +214,9 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY, tellMaster = false) // Checking whether blocks are in memory - assert(store.getSingle("a1").isDefined, "a1 was not in store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3").isDefined, "a3 was not in store") + assert(store.getSingleAndReleaseLock("a1").isDefined, "a1 was not in store") + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") + assert(store.getSingleAndReleaseLock("a3").isDefined, "a3 was not in store") // Checking whether master knows about the blocks or not assert(master.getLocations("a1").size > 0, "master was not told about a1") @@ -177,10 +224,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations("a3").size === 0, "master was told about a3") // Drop a1 and a2 from memory; this should be reported back to the master - store.dropFromMemory("a1", null: Either[Array[Any], ByteBuffer]) - store.dropFromMemory("a2", null: Either[Array[Any], ByteBuffer]) - assert(store.getSingle("a1") === None, "a1 not removed from store") - assert(store.getSingle("a2") === None, "a2 not removed from store") + store.dropFromMemoryIfExists("a1", () => null: Either[Array[Any], ChunkedByteBuffer]) + store.dropFromMemoryIfExists("a2", () => null: Either[Array[Any], ChunkedByteBuffer]) + assert(store.getSingleAndReleaseLock("a1") === None, "a1 not removed from store") + assert(store.getSingleAndReleaseLock("a2") === None, "a2 not removed from store") assert(master.getLocations("a1").size === 0, "master did not remove a1") assert(master.getLocations("a2").size === 0, "master did not remove a2") } @@ -214,11 +261,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // Checking whether blocks are in memory and memory size val memStatus = master.getMemoryStatus.head._2 - assert(memStatus._1 == 20000L, "total memory " + memStatus._1 + " should equal 20000") - assert(memStatus._2 <= 12000L, "remaining memory " + memStatus._2 + " should <= 12000") - assert(store.getSingle("a1-to-remove").isDefined, "a1 was not in store") - assert(store.getSingle("a2-to-remove").isDefined, "a2 was not in store") - assert(store.getSingle("a3-to-remove").isDefined, "a3 was not in store") + assert(memStatus._1 == 40000L, "total memory " + memStatus._1 + " should equal 40000") + assert(memStatus._2 <= 32000L, "remaining memory " + memStatus._2 + " should <= 12000") + assert(store.getSingleAndReleaseLock("a1-to-remove").isDefined, "a1 was not in store") + assert(store.getSingleAndReleaseLock("a2-to-remove").isDefined, "a2 was not in store") + assert(store.getSingleAndReleaseLock("a3-to-remove").isDefined, "a3 was not in store") // Checking whether master knows about the blocks or not assert(master.getLocations("a1-to-remove").size > 0, "master was not told about a1") @@ -231,21 +278,21 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE master.removeBlock("a3-to-remove") eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - store.getSingle("a1-to-remove") should be (None) + assert(!store.hasLocalBlock("a1-to-remove")) master.getLocations("a1-to-remove") should have size 0 } eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - store.getSingle("a2-to-remove") should be (None) + assert(!store.hasLocalBlock("a2-to-remove")) master.getLocations("a2-to-remove") should have size 0 } eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - store.getSingle("a3-to-remove") should not be (None) + assert(store.hasLocalBlock("a3-to-remove")) master.getLocations("a3-to-remove") should have size 0 } eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { val memStatus = master.getMemoryStatus.head._2 - memStatus._1 should equal (20000L) - memStatus._2 should equal (20000L) + memStatus._1 should equal (40000L) + memStatus._2 should equal (40000L) } } @@ -261,24 +308,24 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE master.removeRdd(0, blocking = false) eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - store.getSingle(rdd(0, 0)) should be (None) + store.getSingleAndReleaseLock(rdd(0, 0)) should be (None) master.getLocations(rdd(0, 0)) should have size 0 } eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - store.getSingle(rdd(0, 1)) should be (None) + store.getSingleAndReleaseLock(rdd(0, 1)) should be (None) master.getLocations(rdd(0, 1)) should have size 0 } eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - store.getSingle("nonrddblock") should not be (None) + store.getSingleAndReleaseLock("nonrddblock") should not be (None) master.getLocations("nonrddblock") should have size (1) } store.putSingle(rdd(0, 0), a1, StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 1), a2, StorageLevel.MEMORY_ONLY) master.removeRdd(0, blocking = true) - store.getSingle(rdd(0, 0)) should be (None) + store.getSingleAndReleaseLock(rdd(0, 0)) should be (None) master.getLocations(rdd(0, 0)) should have size 0 - store.getSingle(rdd(0, 1)) should be (None) + store.getSingleAndReleaseLock(rdd(0, 1)) should be (None) master.getLocations(rdd(0, 1)) should have size 0 } @@ -306,46 +353,46 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // verify whether the blocks exist in both the stores Seq(driverStore, executorStore).foreach { case s => - s.getLocal(broadcast0BlockId) should not be (None) - s.getLocal(broadcast1BlockId) should not be (None) - s.getLocal(broadcast2BlockId) should not be (None) - s.getLocal(broadcast2BlockId2) should not be (None) + assert(s.hasLocalBlock(broadcast0BlockId)) + assert(s.hasLocalBlock(broadcast1BlockId)) + assert(s.hasLocalBlock(broadcast2BlockId)) + assert(s.hasLocalBlock(broadcast2BlockId2)) } // remove broadcast 0 block only from executors master.removeBroadcast(0, removeFromMaster = false, blocking = true) // only broadcast 0 block should be removed from the executor store - executorStore.getLocal(broadcast0BlockId) should be (None) - executorStore.getLocal(broadcast1BlockId) should not be (None) - executorStore.getLocal(broadcast2BlockId) should not be (None) + assert(!executorStore.hasLocalBlock(broadcast0BlockId)) + assert(executorStore.hasLocalBlock(broadcast1BlockId)) + assert(executorStore.hasLocalBlock(broadcast2BlockId)) // nothing should be removed from the driver store - driverStore.getLocal(broadcast0BlockId) should not be (None) - driverStore.getLocal(broadcast1BlockId) should not be (None) - driverStore.getLocal(broadcast2BlockId) should not be (None) + assert(driverStore.hasLocalBlock(broadcast0BlockId)) + assert(driverStore.hasLocalBlock(broadcast1BlockId)) + assert(driverStore.hasLocalBlock(broadcast2BlockId)) // remove broadcast 0 block from the driver as well master.removeBroadcast(0, removeFromMaster = true, blocking = true) - driverStore.getLocal(broadcast0BlockId) should be (None) - driverStore.getLocal(broadcast1BlockId) should not be (None) + assert(!driverStore.hasLocalBlock(broadcast0BlockId)) + assert(driverStore.hasLocalBlock(broadcast1BlockId)) // remove broadcast 1 block from both the stores asynchronously // and verify all broadcast 1 blocks have been removed master.removeBroadcast(1, removeFromMaster = true, blocking = false) eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - driverStore.getLocal(broadcast1BlockId) should be (None) - executorStore.getLocal(broadcast1BlockId) should be (None) + assert(!driverStore.hasLocalBlock(broadcast1BlockId)) + assert(!executorStore.hasLocalBlock(broadcast1BlockId)) } // remove broadcast 2 from both the stores asynchronously // and verify all broadcast 2 blocks have been removed master.removeBroadcast(2, removeFromMaster = true, blocking = false) eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - driverStore.getLocal(broadcast2BlockId) should be (None) - driverStore.getLocal(broadcast2BlockId2) should be (None) - executorStore.getLocal(broadcast2BlockId) should be (None) - executorStore.getLocal(broadcast2BlockId2) should be (None) + assert(!driverStore.hasLocalBlock(broadcast2BlockId)) + assert(!driverStore.hasLocalBlock(broadcast2BlockId2)) + assert(!executorStore.hasLocalBlock(broadcast2BlockId)) + assert(!executorStore.hasLocalBlock(broadcast2BlockId2)) } executorStore.stop() driverStore.stop() @@ -358,13 +405,13 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) - assert(store.getSingle("a1").isDefined, "a1 was not in store") + assert(store.getSingleAndReleaseLock("a1").isDefined, "a1 was not in store") assert(master.getLocations("a1").size > 0, "master was not told about a1") master.removeExecutor(store.blockManagerId.executorId) assert(master.getLocations("a1").size == 0, "a1 was not removed from master") - val reregister = !master.driverEndpoint.askWithRetry[Boolean]( + val reregister = !master.driverEndpoint.askSync[Boolean]( BlockManagerHeartbeat(store.blockManagerId)) assert(reregister == true) } @@ -397,7 +444,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE master.removeExecutor(store.blockManagerId.executorId) val t1 = new Thread { override def run() { - store.putIterator("a2", a2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator( + "a2", a2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } } val t2 = new Thread { @@ -418,8 +466,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE t2.join() t3.join() - store.dropFromMemory("a1", null: Either[Array[Any], ByteBuffer]) - store.dropFromMemory("a2", null: Either[Array[Any], ByteBuffer]) + store.dropFromMemoryIfExists("a1", () => null: Either[Array[Any], ChunkedByteBuffer]) + store.dropFromMemoryIfExists("a2", () => null: Either[Array[Any], ChunkedByteBuffer]) store.waitForAsyncReregister() } } @@ -430,9 +478,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val list2 = List(new Array[Byte](500), new Array[Byte](1000), new Array[Byte](1500)) val list1SizeEstimate = SizeEstimator.estimate(list1.iterator.toArray) val list2SizeEstimate = SizeEstimator.estimate(list2.iterator.toArray) - store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.putIterator("list2memory", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.putIterator("list2disk", list2.iterator, StorageLevel.DISK_ONLY, tellMaster = true) + store.putIterator( + "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator( + "list2memory", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator( + "list2disk", list2.iterator, StorageLevel.DISK_ONLY, tellMaster = true) val list1Get = store.get("list1") assert(list1Get.isDefined, "list1 expected to be in store") assert(list1Get.get.data.size === 2) @@ -451,74 +502,112 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(list2DiskGet.get.readMethod === DataReadMethod.Disk) } + test("optimize a location order of blocks without topology information") { + val localHost = "localhost" + val otherHost = "otherHost" + val bmMaster = mock(classOf[BlockManagerMaster]) + val bmId1 = BlockManagerId("id1", localHost, 1) + val bmId2 = BlockManagerId("id2", localHost, 2) + val bmId3 = BlockManagerId("id3", otherHost, 3) + when(bmMaster.getLocations(mc.any[BlockId])).thenReturn(Seq(bmId1, bmId2, bmId3)) + + val blockManager = makeBlockManager(128, "exec", bmMaster) + val getLocations = PrivateMethod[Seq[BlockManagerId]]('getLocations) + val locations = blockManager invokePrivate getLocations(BroadcastBlockId(0)) + assert(locations.map(_.host) === Seq(localHost, localHost, otherHost)) + } + + test("optimize a location order of blocks with topology information") { + val localHost = "localhost" + val otherHost = "otherHost" + val localRack = "localRack" + val otherRack = "otherRack" + + val bmMaster = mock(classOf[BlockManagerMaster]) + val bmId1 = BlockManagerId("id1", localHost, 1, Some(localRack)) + val bmId2 = BlockManagerId("id2", localHost, 2, Some(localRack)) + val bmId3 = BlockManagerId("id3", otherHost, 3, Some(otherRack)) + val bmId4 = BlockManagerId("id4", otherHost, 4, Some(otherRack)) + val bmId5 = BlockManagerId("id5", otherHost, 5, Some(localRack)) + when(bmMaster.getLocations(mc.any[BlockId])) + .thenReturn(Seq(bmId1, bmId2, bmId5, bmId3, bmId4)) + + val blockManager = makeBlockManager(128, "exec", bmMaster) + blockManager.blockManagerId = + BlockManagerId(SparkContext.DRIVER_IDENTIFIER, localHost, 1, Some(localRack)) + val getLocations = PrivateMethod[Seq[BlockManagerId]]('getLocations) + val locations = blockManager invokePrivate getLocations(BroadcastBlockId(0)) + assert(locations.map(_.host) === Seq(localHost, localHost, otherHost, otherHost, otherHost)) + assert(locations.flatMap(_.topologyInfo) + === Seq(localRack, localRack, localRack, otherRack, otherRack)) + } + test("SPARK-9591: getRemoteBytes from another location when Exception throw") { - val origTimeoutOpt = conf.getOption("spark.network.timeout") - try { - conf.set("spark.network.timeout", "2s") - store = makeBlockManager(8000, "executor1") - store2 = makeBlockManager(8000, "executor2") - store3 = makeBlockManager(8000, "executor3") - val list1 = List(new Array[Byte](4000)) - store2.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store3.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - var list1Get = store.getRemoteBytes("list1") - assert(list1Get.isDefined, "list1Get expected to be fetched") - // block manager exit - store2.stop() - store2 = null - list1Get = store.getRemoteBytes("list1") - // get `list1` block - assert(list1Get.isDefined, "list1Get expected to be fetched") - store3.stop() - store3 = null - // exception throw because there is no locations - intercept[BlockFetchException] { - list1Get = store.getRemoteBytes("list1") - } - } finally { - origTimeoutOpt match { - case Some(t) => conf.set("spark.network.timeout", t) - case None => conf.remove("spark.network.timeout") - } - } + conf.set("spark.shuffle.io.maxRetries", "0") + store = makeBlockManager(8000, "executor1") + store2 = makeBlockManager(8000, "executor2") + store3 = makeBlockManager(8000, "executor3") + val list1 = List(new Array[Byte](4000)) + store2.putIterator( + "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store3.putIterator( + "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") + store2.stop() + store2 = null + assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") + store3.stop() + store3 = null + // Should return None instead of throwing an exception: + assert(store.getRemoteBytes("list1").isEmpty) + } + + test("SPARK-14252: getOrElseUpdate should still read from remote storage") { + store = makeBlockManager(8000, "executor1") + store2 = makeBlockManager(8000, "executor2") + val list1 = List(new Array[Byte](4000)) + store2.putIterator( + "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(store.getOrElseUpdate( + "list1", + StorageLevel.MEMORY_ONLY, + ClassTag.Any, + () => throw new AssertionError("attempted to compute locally")).isLeft) } test("in-memory LRU storage") { - store = makeBlockManager(12000) - val a1 = new Array[Byte](4000) - val a2 = new Array[Byte](4000) - val a3 = new Array[Byte](4000) - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) - store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) - store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY) - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - assert(store.getSingle("a1") === None, "a1 was in store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - // At this point a2 was gotten last, so LRU will getSingle rid of a3 - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) - assert(store.getSingle("a1").isDefined, "a1 was not in store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3") === None, "a3 was in store") + testInMemoryLRUStorage(StorageLevel.MEMORY_ONLY) } test("in-memory LRU storage with serialization") { + testInMemoryLRUStorage(StorageLevel.MEMORY_ONLY_SER) + } + + test("in-memory LRU storage with off-heap") { + testInMemoryLRUStorage(StorageLevel( + useDisk = false, + useMemory = true, + useOffHeap = true, + deserialized = false, replication = 1)) + } + + private def testInMemoryLRUStorage(storageLevel: StorageLevel): Unit = { store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) - store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) - store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY_SER) - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - assert(store.getSingle("a1") === None, "a1 was in store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") + store.putSingle("a1", a1, storageLevel) + store.putSingle("a2", a2, storageLevel) + store.putSingle("a3", a3, storageLevel) + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") + assert(store.getSingleAndReleaseLock("a3").isDefined, "a3 was not in store") + assert(store.getSingleAndReleaseLock("a1") === None, "a1 was in store") + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") // At this point a2 was gotten last, so LRU will getSingle rid of a3 - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) - assert(store.getSingle("a1").isDefined, "a1 was not in store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3") === None, "a3 was in store") + store.putSingle("a1", a1, storageLevel) + assert(store.getSingleAndReleaseLock("a1").isDefined, "a1 was not in store") + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") + assert(store.getSingleAndReleaseLock("a3") === None, "a3 was in store") } test("in-memory LRU for partitions of same RDD") { @@ -531,13 +620,13 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle(rdd(0, 3), a3, StorageLevel.MEMORY_ONLY) // Even though we accessed rdd_0_3 last, it should not have replaced partitions 1 and 2 // from the same RDD - assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store") - assert(store.getSingle(rdd(0, 2)).isDefined, "rdd_0_2 was not in store") - assert(store.getSingle(rdd(0, 1)).isDefined, "rdd_0_1 was not in store") + assert(store.getSingleAndReleaseLock(rdd(0, 3)) === None, "rdd_0_3 was in store") + assert(store.getSingleAndReleaseLock(rdd(0, 2)).isDefined, "rdd_0_2 was not in store") + assert(store.getSingleAndReleaseLock(rdd(0, 1)).isDefined, "rdd_0_1 was not in store") // Check that rdd_0_3 doesn't replace them even after further accesses - assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store") - assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store") - assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store") + assert(store.getSingleAndReleaseLock(rdd(0, 3)) === None, "rdd_0_3 was in store") + assert(store.getSingleAndReleaseLock(rdd(0, 3)) === None, "rdd_0_3 was in store") + assert(store.getSingleAndReleaseLock(rdd(0, 3)) === None, "rdd_0_3 was in store") } test("in-memory LRU for partitions of multiple RDDs") { @@ -550,7 +639,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(!store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was in store") assert(store.memoryStore.contains(rdd(0, 2)), "rdd_0_2 was not in store") // Do a get() on rdd_0_2 so that it is the most recently used item - assert(store.getSingle(rdd(0, 2)).isDefined, "rdd_0_2 was not in store") + assert(store.getSingleAndReleaseLock(rdd(0, 2)).isDefined, "rdd_0_2 was not in store") // Put in more partitions from RDD 0; they should replace rdd_1_1 store.putSingle(rdd(0, 3), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 4), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) @@ -563,101 +652,77 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(store.memoryStore.contains(rdd(0, 3)), "rdd_0_3 was not in store") } - test("tachyon storage") { - // TODO Make the spark.test.tachyon.enable true after using tachyon 0.5.0 testing jar. - val tachyonUnitTestEnabled = conf.getBoolean("spark.test.tachyon.enable", false) - conf.set(ExternalBlockStore.BLOCK_MANAGER_NAME, ExternalBlockStore.DEFAULT_BLOCK_MANAGER_NAME) - if (tachyonUnitTestEnabled) { - store = makeBlockManager(1200) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) - store.putSingle("a1", a1, StorageLevel.OFF_HEAP) - store.putSingle("a2", a2, StorageLevel.OFF_HEAP) - store.putSingle("a3", a3, StorageLevel.OFF_HEAP) - assert(store.getSingle("a3").isDefined, "a3 was in store") - assert(store.getSingle("a2").isDefined, "a2 was in store") - assert(store.getSingle("a1").isDefined, "a1 was in store") - } else { - info("tachyon storage test disabled.") - } - } - - test("on-disk storage") { - store = makeBlockManager(1200) + encryptionTest("on-disk storage") { _conf => + store = makeBlockManager(1200, testConf = Some(_conf)) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.DISK_ONLY) store.putSingle("a2", a2, StorageLevel.DISK_ONLY) store.putSingle("a3", a3, StorageLevel.DISK_ONLY) - assert(store.getSingle("a2").isDefined, "a2 was in store") - assert(store.getSingle("a3").isDefined, "a3 was in store") - assert(store.getSingle("a1").isDefined, "a1 was in store") + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was in store") + assert(store.getSingleAndReleaseLock("a3").isDefined, "a3 was in store") + assert(store.getSingleAndReleaseLock("a1").isDefined, "a1 was in store") } - test("disk and memory storage") { - store = makeBlockManager(12000) - val a1 = new Array[Byte](4000) - val a2 = new Array[Byte](4000) - val a3 = new Array[Byte](4000) - store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK) - store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK) - store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK) - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") - assert(store.getSingle("a1").isDefined, "a1 was not in store") - assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") + encryptionTest("disk and memory storage") { _conf => + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK, getAsBytes = false, testConf = conf) } - test("disk and memory storage with getLocalBytes") { - store = makeBlockManager(12000) - val a1 = new Array[Byte](4000) - val a2 = new Array[Byte](4000) - val a3 = new Array[Byte](4000) - store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK) - store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK) - store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK) - assert(store.getLocalBytes("a2").isDefined, "a2 was not in store") - assert(store.getLocalBytes("a3").isDefined, "a3 was not in store") - assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") - assert(store.getLocalBytes("a1").isDefined, "a1 was not in store") - assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") + encryptionTest("disk and memory storage with getLocalBytes") { _conf => + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK, getAsBytes = true, testConf = conf) } - test("disk and memory storage with serialization") { - store = makeBlockManager(12000) - val a1 = new Array[Byte](4000) - val a2 = new Array[Byte](4000) - val a3 = new Array[Byte](4000) - store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK_SER) - store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK_SER) - store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK_SER) - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") - assert(store.getSingle("a1").isDefined, "a1 was not in store") - assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") + encryptionTest("disk and memory storage with serialization") { _conf => + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK_SER, getAsBytes = false, testConf = conf) } - test("disk and memory storage with serialization and getLocalBytes") { - store = makeBlockManager(12000) + encryptionTest("disk and memory storage with serialization and getLocalBytes") { _conf => + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK_SER, getAsBytes = true, testConf = conf) + } + + encryptionTest("disk and off-heap memory storage") { _conf => + testDiskAndMemoryStorage(StorageLevel.OFF_HEAP, getAsBytes = false, testConf = conf) + } + + encryptionTest("disk and off-heap memory storage with getLocalBytes") { _conf => + testDiskAndMemoryStorage(StorageLevel.OFF_HEAP, getAsBytes = true, testConf = conf) + } + + def testDiskAndMemoryStorage( + storageLevel: StorageLevel, + getAsBytes: Boolean, + testConf: SparkConf): Unit = { + store = makeBlockManager(12000, testConf = Some(testConf)) + val accessMethod = + if (getAsBytes) store.getLocalBytesAndReleaseLock else store.getSingleAndReleaseLock val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) - store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK_SER) - store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK_SER) - store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK_SER) - assert(store.getLocalBytes("a2").isDefined, "a2 was not in store") - assert(store.getLocalBytes("a3").isDefined, "a3 was not in store") - assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") - assert(store.getLocalBytes("a1").isDefined, "a1 was not in store") - assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") + store.putSingle("a1", a1, storageLevel) + store.putSingle("a2", a2, storageLevel) + store.putSingle("a3", a3, storageLevel) + assert(accessMethod("a2").isDefined, "a2 was not in store") + assert(accessMethod("a3").isDefined, "a3 was not in store") + assert(accessMethod("a1").isDefined, "a1 was not in store") + val dataShouldHaveBeenCachedBackIntoMemory = { + if (storageLevel.deserialized) { + !getAsBytes + } else { + // If the block's storage level is serialized, then always cache the bytes in memory, even + // if the caller requested values. + true + } + } + if (dataShouldHaveBeenCachedBackIntoMemory) { + assert(store.memoryStore.contains("a1"), "a1 was not in memory store") + } else { + assert(!store.memoryStore.contains("a1"), "a1 was in memory store") + } } - test("LRU with mixed storage levels") { - store = makeBlockManager(12000) + encryptionTest("LRU with mixed storage levels") { _conf => + store = makeBlockManager(12000, testConf = Some(_conf)) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -667,75 +732,83 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a3", a3, StorageLevel.DISK_ONLY) // At this point LRU should not kick in because a3 is only on disk - assert(store.getSingle("a1").isDefined, "a1 was not in store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3").isDefined, "a3 was not in store") + assert(store.getSingleAndReleaseLock("a1").isDefined, "a1 was not in store") + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") + assert(store.getSingleAndReleaseLock("a3").isDefined, "a3 was not in store") // Now let's add in a4, which uses both disk and memory; a1 should drop out store.putSingle("a4", a4, StorageLevel.MEMORY_AND_DISK_SER) - assert(store.getSingle("a1") == None, "a1 was in store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - assert(store.getSingle("a4").isDefined, "a4 was not in store") + assert(store.getSingleAndReleaseLock("a1") == None, "a1 was in store") + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") + assert(store.getSingleAndReleaseLock("a3").isDefined, "a3 was not in store") + assert(store.getSingleAndReleaseLock("a4").isDefined, "a4 was not in store") } - test("in-memory LRU with streams") { - store = makeBlockManager(12000) + encryptionTest("in-memory LRU with streams") { _conf => + store = makeBlockManager(12000, testConf = Some(_conf)) val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) - store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.putIterator("list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.putIterator("list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - assert(store.get("list2").isDefined, "list2 was not in store") + store.putIterator( + "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator( + "list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator( + "list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) - assert(store.get("list3").isDefined, "list3 was not in store") + assert(store.getAndReleaseLock("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) - assert(store.get("list1") === None, "list1 was in store") - assert(store.get("list2").isDefined, "list2 was not in store") + assert(store.getAndReleaseLock("list1") === None, "list1 was in store") + assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) // At this point list2 was gotten last, so LRU will getSingle rid of list3 - store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - assert(store.get("list1").isDefined, "list1 was not in store") + store.putIterator( + "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(store.getAndReleaseLock("list1").isDefined, "list1 was not in store") assert(store.get("list1").get.data.size === 2) - assert(store.get("list2").isDefined, "list2 was not in store") + assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) - assert(store.get("list3") === None, "list1 was in store") + assert(store.getAndReleaseLock("list3") === None, "list1 was in store") } - test("LRU with mixed storage levels and streams") { - store = makeBlockManager(12000) + encryptionTest("LRU with mixed storage levels and streams") { _conf => + store = makeBlockManager(12000, testConf = Some(_conf)) val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) val list4 = List(new Array[Byte](2000), new Array[Byte](2000)) // First store list1 and list2, both in memory, and list3, on disk only - store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) - store.putIterator("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) - store.putIterator("list3", list3.iterator, StorageLevel.DISK_ONLY, tellMaster = true) + store.putIterator( + "list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.putIterator( + "list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.putIterator( + "list3", list3.iterator, StorageLevel.DISK_ONLY, tellMaster = true) val listForSizeEstimate = new ArrayBuffer[Any] listForSizeEstimate ++= list1.iterator val listSize = SizeEstimator.estimate(listForSizeEstimate) // At this point LRU should not kick in because list3 is only on disk - assert(store.get("list1").isDefined, "list1 was not in store") + assert(store.getAndReleaseLock("list1").isDefined, "list1 was not in store") assert(store.get("list1").get.data.size === 2) - assert(store.get("list2").isDefined, "list2 was not in store") + assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) - assert(store.get("list3").isDefined, "list3 was not in store") + assert(store.getAndReleaseLock("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) - assert(store.get("list1").isDefined, "list1 was not in store") + assert(store.getAndReleaseLock("list1").isDefined, "list1 was not in store") assert(store.get("list1").get.data.size === 2) - assert(store.get("list2").isDefined, "list2 was not in store") + assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) - assert(store.get("list3").isDefined, "list3 was not in store") + assert(store.getAndReleaseLock("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) // Now let's add in list4, which uses both disk and memory; list1 should drop out - store.putIterator("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) - assert(store.get("list1") === None, "list1 was in store") - assert(store.get("list2").isDefined, "list2 was not in store") + store.putIterator( + "list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) + assert(store.getAndReleaseLock("list1") === None, "list1 was in store") + assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) - assert(store.get("list3").isDefined, "list3 was not in store") + assert(store.getAndReleaseLock("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) - assert(store.get("list4").isDefined, "list4 was not in store") + assert(store.getAndReleaseLock("list4").isDefined, "list4 was not in store") assert(store.get("list4").get.data.size === 2) } @@ -754,17 +827,18 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("overly large block") { store = makeBlockManager(5000) store.putSingle("a1", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) - assert(store.getSingle("a1") === None, "a1 was in store") + assert(store.getSingleAndReleaseLock("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](10000), StorageLevel.MEMORY_AND_DISK) - assert(store.memoryStore.getValues("a2") === None, "a2 was in memory store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") + assert(!store.memoryStore.contains("a2"), "a2 was in memory store") + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") } test("block compression") { try { conf.set("spark.shuffle.compress", "true") store = makeBlockManager(20000, "exec1") - store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) + store.putSingle( + ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") store.stop() @@ -772,7 +846,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE conf.set("spark.shuffle.compress", "false") store = makeBlockManager(20000, "exec2") - store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + store.putSingle( + ShuffleBlockId(0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 10000, "shuffle_0_0_0 was compressed") store.stop() @@ -780,7 +855,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE conf.set("spark.broadcast.compress", "true") store = makeBlockManager(20000, "exec3") - store.putSingle(BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + store.putSingle( + BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 1000, "broadcast_0 was not compressed") store.stop() @@ -788,7 +864,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE conf.set("spark.broadcast.compress", "false") store = makeBlockManager(20000, "exec4") - store.putSingle(BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + store.putSingle( + BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 10000, "broadcast_0 was compressed") store.stop() store = null @@ -822,16 +899,15 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("block store put failure") { // Use Java serializer so we can create an unserializable error. - val transfer = new NettyBlockTransferService(conf, securityMgr, numCores = 1) - val memoryManager = new StaticMemoryManager( - conf, - maxExecutionMemory = Long.MaxValue, - maxStorageMemory = 1200, - numCores = 1) + conf.set("spark.testing.memory", "1200") + val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1) + val memoryManager = UnifiedMemoryManager(conf, numCores = 1) + val serializerManager = new SerializerManager(new JavaSerializer(conf), conf) store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master, - new JavaSerializer(conf), conf, memoryManager, mapOutputTracker, + serializerManager, conf, memoryManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) memoryManager.setMemoryStore(store.memoryStore) + store.initialize("app-id") // The put should fail since a1 is not serializable. class UnserializableClass @@ -842,76 +918,80 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // Make sure get a1 doesn't hang and returns None. failAfter(1 second) { - assert(store.getSingle("a1").isEmpty, "a1 should not be in store") + assert(store.getSingleAndReleaseLock("a1").isEmpty, "a1 should not be in store") } } - test("reads of memory-mapped and non memory-mapped files are equivalent") { - val confKey = "spark.storage.memoryMapThreshold" - - // Create a non-trivial (not all zeros) byte array - var counter = 0.toByte - def incr: Byte = {counter = (counter + 1).toByte; counter;} - val bytes = Array.fill[Byte](1000)(incr) - val byteBuffer = ByteBuffer.wrap(bytes) - - val blockId = BlockId("rdd_1_2") - - // This sequence of mocks makes these tests fairly brittle. It would - // be nice to refactor classes involved in disk storage in a way that - // allows for easier testing. - val blockManager = mock(classOf[BlockManager]) - when(blockManager.conf).thenReturn(conf.clone.set(confKey, "0")) - val diskBlockManager = new DiskBlockManager(blockManager, conf) - - val diskStoreMapped = new DiskStore(blockManager, diskBlockManager) - diskStoreMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) - val mapped = diskStoreMapped.getBytes(blockId).get - - when(blockManager.conf).thenReturn(conf.clone.set(confKey, "1m")) - val diskStoreNotMapped = new DiskStore(blockManager, diskBlockManager) - diskStoreNotMapped.putBytes(blockId, byteBuffer, StorageLevel.DISK_ONLY) - val notMapped = diskStoreNotMapped.getBytes(blockId).get - - // Not possible to do isInstanceOf due to visibility of HeapByteBuffer - assert(notMapped.getClass.getName.endsWith("HeapByteBuffer"), - "Expected HeapByteBuffer for un-mapped read") - assert(mapped.isInstanceOf[MappedByteBuffer], "Expected MappedByteBuffer for mapped read") - - def arrayFromByteBuffer(in: ByteBuffer): Array[Byte] = { - val array = new Array[Byte](in.remaining()) - in.get(array) - array + test("turn off updated block statuses") { + val conf = new SparkConf() + conf.set(TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES, false) + store = makeBlockManager(12000, testConf = Some(conf)) + + store.registerTask(0) + val list = List.fill(2)(new Array[Byte](2000)) + + def getUpdatedBlocks(task: => Unit): Seq[(BlockId, BlockStatus)] = { + val context = TaskContext.empty() + try { + TaskContext.setTaskContext(context) + task + } finally { + TaskContext.unset() + } + context.taskMetrics.updatedBlockStatuses } - val mappedAsArray = arrayFromByteBuffer(mapped) - val notMappedAsArray = arrayFromByteBuffer(notMapped) - assert(Arrays.equals(mappedAsArray, bytes)) - assert(Arrays.equals(notMappedAsArray, bytes)) + // 1 updated block (i.e. list1) + val updatedBlocks1 = getUpdatedBlocks { + store.putIterator( + "list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + } + assert(updatedBlocks1.size === 0) } + test("updated block statuses") { - store = makeBlockManager(12000) + val conf = new SparkConf() + conf.set(TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES, true) + store = makeBlockManager(12000, testConf = Some(conf)) + store.registerTask(0) val list = List.fill(2)(new Array[Byte](2000)) val bigList = List.fill(8)(new Array[Byte](2000)) + def getUpdatedBlocks(task: => Unit): Seq[(BlockId, BlockStatus)] = { + val context = TaskContext.empty() + try { + TaskContext.setTaskContext(context) + task + } finally { + TaskContext.unset() + } + context.taskMetrics.updatedBlockStatuses + } + // 1 updated block (i.e. list1) - val updatedBlocks1 = - store.putIterator("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + val updatedBlocks1 = getUpdatedBlocks { + store.putIterator( + "list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + } assert(updatedBlocks1.size === 1) assert(updatedBlocks1.head._1 === TestBlockId("list1")) assert(updatedBlocks1.head._2.storageLevel === StorageLevel.MEMORY_ONLY) // 1 updated block (i.e. list2) - val updatedBlocks2 = - store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + val updatedBlocks2 = getUpdatedBlocks { + store.putIterator( + "list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + } assert(updatedBlocks2.size === 1) assert(updatedBlocks2.head._1 === TestBlockId("list2")) assert(updatedBlocks2.head._2.storageLevel === StorageLevel.MEMORY_ONLY) // 2 updated blocks - list1 is kicked out of memory while list3 is added - val updatedBlocks3 = - store.putIterator("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + val updatedBlocks3 = getUpdatedBlocks { + store.putIterator( + "list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + } assert(updatedBlocks3.size === 2) updatedBlocks3.foreach { case (id, status) => id match { @@ -923,8 +1003,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(store.memoryStore.contains("list3"), "list3 was not in memory store") // 2 updated blocks - list2 is kicked out of memory (but put on disk) while list4 is added - val updatedBlocks4 = - store.putIterator("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + val updatedBlocks4 = getUpdatedBlocks { + store.putIterator( + "list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + } assert(updatedBlocks4.size === 2) updatedBlocks4.foreach { case (id, status) => id match { @@ -937,8 +1019,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(store.memoryStore.contains("list4"), "list4 was not in memory store") // No updated blocks - list5 is too big to fit in store and nothing is kicked out - val updatedBlocks5 = - store.putIterator("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + val updatedBlocks5 = getUpdatedBlocks { + store.putIterator( + "list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + } assert(updatedBlocks5.size === 0) // memory store contains only list3 and list4 @@ -954,6 +1038,16 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(!store.diskStore.contains("list3"), "list3 was in disk store") assert(!store.diskStore.contains("list4"), "list4 was in disk store") assert(!store.diskStore.contains("list5"), "list5 was in disk store") + + // remove block - list2 should be removed from disk + val updatedBlocks6 = getUpdatedBlocks { + store.removeBlock( + "list2", tellMaster = true) + } + assert(updatedBlocks6.size === 1) + assert(updatedBlocks6.head._1 === TestBlockId("list2")) + assert(updatedBlocks6.head._2.storageLevel == StorageLevel.NONE) + assert(!store.diskStore.contains("list2"), "list2 was in disk store") } test("query block statuses") { @@ -961,9 +1055,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val list = List.fill(2)(new Array[Byte](2000)) // Tell master. By LRU, only list2 and list3 remains. - store.putIterator("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.putIterator("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator( + "list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator( + "list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator( + "list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) // getLocations and getBlockStatus should yield the same locations assert(store.master.getLocations("list1").size === 0) @@ -977,9 +1074,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(store.master.getBlockStatus("list3", askSlaves = true).size === 1) // This time don't tell master and see what happens. By LRU, only list5 and list6 remains. - store.putIterator("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) - store.putIterator("list5", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - store.putIterator("list6", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.putIterator( + "list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.putIterator( + "list5", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator( + "list6", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) // getLocations should return nothing because the master is not informed // getBlockStatus without asking slaves should have the same result @@ -1000,9 +1100,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val list = List.fill(2)(new Array[Byte](100)) // insert some blocks - store.putIterator("list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.putIterator("list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator( + "list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator( + "list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator( + "list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) // getLocations and getBlockStatus should yield the same locations assert(store.master.getMatchingBlockIds(_.toString.contains("list"), askSlaves = false).size @@ -1011,9 +1114,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE === 1) // insert some more blocks - store.putIterator("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.putIterator("newlist2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - store.putIterator("newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator( + "newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator( + "newlist2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator( + "newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) // getLocations and getBlockStatus should yield the same locations assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = false).size @@ -1023,7 +1129,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val blockIds = Seq(RDDBlockId(1, 0), RDDBlockId(1, 1), RDDBlockId(2, 0)) blockIds.foreach { blockId => - store.putIterator(blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator( + blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } val matchedBlockIds = store.master.getMatchingBlockIds(_ match { case RDDBlockId(1, _) => true @@ -1037,7 +1144,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle(rdd(0, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Access rdd_1_0 to ensure it's not least recently used. - assert(store.getSingle(rdd(1, 0)).isDefined, "rdd_1_0 was not in store") + assert(store.getSingleAndReleaseLock(rdd(1, 0)).isDefined, "rdd_1_0 was not in store") // According to the same-RDD rule, rdd_1_0 should be replaced here. store.putSingle(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // rdd_1_0 should have been replaced, even it's not least recently used. @@ -1046,158 +1153,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(!store.memoryStore.contains(rdd(1, 0)), "rdd_1_0 was in store") } - test("reserve/release unroll memory") { - store = makeBlockManager(12000) - val memoryStore = store.memoryStore - assert(memoryStore.currentUnrollMemory === 0) - assert(memoryStore.currentUnrollMemoryForThisTask === 0) - - def reserveUnrollMemoryForThisTask(memory: Long): Boolean = { - memoryStore.reserveUnrollMemoryForThisTask( - TestBlockId(""), memory, new ArrayBuffer[(BlockId, BlockStatus)]) - } - - // Reserve - assert(reserveUnrollMemoryForThisTask(100)) - assert(memoryStore.currentUnrollMemoryForThisTask === 100) - assert(reserveUnrollMemoryForThisTask(200)) - assert(memoryStore.currentUnrollMemoryForThisTask === 300) - assert(reserveUnrollMemoryForThisTask(500)) - assert(memoryStore.currentUnrollMemoryForThisTask === 800) - assert(!reserveUnrollMemoryForThisTask(1000000)) - assert(memoryStore.currentUnrollMemoryForThisTask === 800) // not granted - // Release - memoryStore.releaseUnrollMemoryForThisTask(100) - assert(memoryStore.currentUnrollMemoryForThisTask === 700) - memoryStore.releaseUnrollMemoryForThisTask(100) - assert(memoryStore.currentUnrollMemoryForThisTask === 600) - // Reserve again - assert(reserveUnrollMemoryForThisTask(4400)) - assert(memoryStore.currentUnrollMemoryForThisTask === 5000) - assert(!reserveUnrollMemoryForThisTask(20000)) - assert(memoryStore.currentUnrollMemoryForThisTask === 5000) // not granted - // Release again - memoryStore.releaseUnrollMemoryForThisTask(1000) - assert(memoryStore.currentUnrollMemoryForThisTask === 4000) - memoryStore.releaseUnrollMemoryForThisTask() // release all - assert(memoryStore.currentUnrollMemoryForThisTask === 0) - } - - /** - * Verify the result of MemoryStore#unrollSafely is as expected. - */ - private def verifyUnroll( - expected: Iterator[Any], - result: Either[Array[Any], Iterator[Any]], - shouldBeArray: Boolean): Unit = { - val actual: Iterator[Any] = result match { - case Left(arr: Array[Any]) => - assert(shouldBeArray, "expected iterator from unroll!") - arr.iterator - case Right(it: Iterator[Any]) => - assert(!shouldBeArray, "expected array from unroll!") - it - case _ => - fail("unroll returned neither an iterator nor an array...") - } - expected.zip(actual).foreach { case (e, a) => - assert(e === a, "unroll did not return original values!") - } - } - - test("safely unroll blocks") { - store = makeBlockManager(12000) - val smallList = List.fill(40)(new Array[Byte](100)) - val bigList = List.fill(40)(new Array[Byte](1000)) - val memoryStore = store.memoryStore - val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - assert(memoryStore.currentUnrollMemoryForThisTask === 0) - - // Unroll with all the space in the world. This should succeed and return an array. - var unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks) - verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) - assert(memoryStore.currentUnrollMemoryForThisTask === 0) - memoryStore.releasePendingUnrollMemoryForThisTask() - - // Unroll with not enough space. This should succeed after kicking out someBlock1. - store.putIterator("someBlock1", smallList.iterator, StorageLevel.MEMORY_ONLY) - store.putIterator("someBlock2", smallList.iterator, StorageLevel.MEMORY_ONLY) - unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks) - verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) - assert(memoryStore.currentUnrollMemoryForThisTask === 0) - assert(droppedBlocks.size === 1) - assert(droppedBlocks.head._1 === TestBlockId("someBlock1")) - droppedBlocks.clear() - memoryStore.releasePendingUnrollMemoryForThisTask() - - // Unroll huge block with not enough space. Even after ensuring free space of 12000 * 0.4 = - // 4800 bytes, there is still not enough room to unroll this block. This returns an iterator. - // In the mean time, however, we kicked out someBlock2 before giving up. - store.putIterator("someBlock3", smallList.iterator, StorageLevel.MEMORY_ONLY) - unrollResult = memoryStore.unrollSafely("unroll", bigList.iterator, droppedBlocks) - verifyUnroll(bigList.iterator, unrollResult, shouldBeArray = false) - assert(memoryStore.currentUnrollMemoryForThisTask > 0) // we returned an iterator - assert(droppedBlocks.size === 1) - assert(droppedBlocks.head._1 === TestBlockId("someBlock2")) - droppedBlocks.clear() - } - - test("safely unroll blocks through putIterator") { - store = makeBlockManager(12000) - val memOnly = StorageLevel.MEMORY_ONLY - val memoryStore = store.memoryStore - val smallList = List.fill(40)(new Array[Byte](100)) - val bigList = List.fill(40)(new Array[Byte](1000)) - def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] - def bigIterator: Iterator[Any] = bigList.iterator.asInstanceOf[Iterator[Any]] - assert(memoryStore.currentUnrollMemoryForThisTask === 0) - - // Unroll with plenty of space. This should succeed and cache both blocks. - val result1 = memoryStore.putIterator("b1", smallIterator, memOnly, returnValues = true) - val result2 = memoryStore.putIterator("b2", smallIterator, memOnly, returnValues = true) - assert(memoryStore.contains("b1")) - assert(memoryStore.contains("b2")) - assert(result1.size > 0) // unroll was successful - assert(result2.size > 0) - assert(result1.data.isLeft) // unroll did not drop this block to disk - assert(result2.data.isLeft) - assert(memoryStore.currentUnrollMemoryForThisTask === 0) - - // Re-put these two blocks so block manager knows about them too. Otherwise, block manager - // would not know how to drop them from memory later. - memoryStore.remove("b1") - memoryStore.remove("b2") - store.putIterator("b1", smallIterator, memOnly) - store.putIterator("b2", smallIterator, memOnly) - - // Unroll with not enough space. This should succeed but kick out b1 in the process. - val result3 = memoryStore.putIterator("b3", smallIterator, memOnly, returnValues = true) - assert(result3.size > 0) - assert(result3.data.isLeft) - assert(!memoryStore.contains("b1")) - assert(memoryStore.contains("b2")) - assert(memoryStore.contains("b3")) - assert(memoryStore.currentUnrollMemoryForThisTask === 0) - memoryStore.remove("b3") - store.putIterator("b3", smallIterator, memOnly) - - // Unroll huge block with not enough space. This should fail and kick out b2 in the process. - val result4 = memoryStore.putIterator("b4", bigIterator, memOnly, returnValues = true) - assert(result4.size === 0) // unroll was unsuccessful - assert(result4.data.isLeft) - assert(!memoryStore.contains("b1")) - assert(!memoryStore.contains("b2")) - assert(memoryStore.contains("b3")) - assert(!memoryStore.contains("b4")) - assert(memoryStore.currentUnrollMemoryForThisTask > 0) // we returned an iterator - } - - /** - * This test is essentially identical to the preceding one, except that it uses MEMORY_AND_DISK. - */ test("safely unroll blocks through putIterator (disk)") { store = makeBlockManager(12000) - val memAndDisk = StorageLevel.MEMORY_AND_DISK val memoryStore = store.memoryStore val diskStore = store.diskStore val smallList = List.fill(40)(new Array[Byte](100)) @@ -1206,13 +1163,13 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE def bigIterator: Iterator[Any] = bigList.iterator.asInstanceOf[Iterator[Any]] assert(memoryStore.currentUnrollMemoryForThisTask === 0) - store.putIterator("b1", smallIterator, memAndDisk) - store.putIterator("b2", smallIterator, memAndDisk) + store.putIterator("b1", smallIterator, StorageLevel.MEMORY_AND_DISK) + store.putIterator("b2", smallIterator, StorageLevel.MEMORY_AND_DISK) // Unroll with not enough space. This should succeed but kick out b1 in the process. // Memory store should contain b2 and b3, while disk store should contain only b1 - val result3 = memoryStore.putIterator("b3", smallIterator, memAndDisk, returnValues = true) - assert(result3.size > 0) + val result3 = memoryStore.putIteratorAsValues("b3", smallIterator, ClassTag.Any) + assert(result3.isRight) assert(!memoryStore.contains("b1")) assert(memoryStore.contains("b2")) assert(memoryStore.contains("b3")) @@ -1223,83 +1180,283 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putIterator("b3", smallIterator, StorageLevel.MEMORY_ONLY) assert(memoryStore.currentUnrollMemoryForThisTask === 0) - // Unroll huge block with not enough space. This should fail and drop the new block to disk - // directly in addition to kicking out b2 in the process. Memory store should contain only - // b3, while disk store should contain b1, b2 and b4. - val result4 = memoryStore.putIterator("b4", bigIterator, memAndDisk, returnValues = true) - assert(result4.size > 0) - assert(result4.data.isRight) // unroll returned bytes from disk + // Unroll huge block with not enough space. This should fail and return an iterator so that + // the block may be stored to disk. During the unrolling process, block "b2" should be kicked + // out, so the memory store should contain only b3, while the disk store should contain + // b1, b2 and b4. + val result4 = memoryStore.putIteratorAsValues("b4", bigIterator, ClassTag.Any) + assert(result4.isLeft) assert(!memoryStore.contains("b1")) assert(!memoryStore.contains("b2")) assert(memoryStore.contains("b3")) assert(!memoryStore.contains("b4")) - assert(diskStore.contains("b1")) - assert(diskStore.contains("b2")) - assert(!diskStore.contains("b3")) - assert(diskStore.contains("b4")) - assert(memoryStore.currentUnrollMemoryForThisTask > 0) // we returned an iterator } - test("multiple unrolls by the same thread") { + test("read-locked blocks cannot be evicted from memory") { store = makeBlockManager(12000) - val memOnly = StorageLevel.MEMORY_ONLY - val memoryStore = store.memoryStore - val smallList = List.fill(40)(new Array[Byte](100)) - def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] - assert(memoryStore.currentUnrollMemoryForThisTask === 0) + val arr = new Array[Byte](4000) + // First store a1 and a2, both in memory, and a3, on disk only + store.putSingle("a1", arr, StorageLevel.MEMORY_ONLY_SER) + store.putSingle("a2", arr, StorageLevel.MEMORY_ONLY_SER) + assert(store.getSingle("a1").isDefined, "a1 was not in store") + assert(store.getSingle("a2").isDefined, "a2 was not in store") + // This put should fail because both a1 and a2 should be read-locked: + store.putSingle("a3", arr, StorageLevel.MEMORY_ONLY_SER) + assert(store.getSingle("a3").isEmpty, "a3 was in store") + assert(store.getSingle("a1").isDefined, "a1 was not in store") + assert(store.getSingle("a2").isDefined, "a2 was not in store") + // Release both pins of block a2: + store.releaseLock("a2") + store.releaseLock("a2") + // Block a1 is the least-recently accessed, so an LRU eviction policy would evict it before + // block a2. However, a1 is still pinned so this put of a3 should evict a2 instead: + store.putSingle("a3", arr, StorageLevel.MEMORY_ONLY_SER) + assert(store.getSingle("a2").isEmpty, "a2 was in store") + assert(store.getSingle("a1").isDefined, "a1 was not in store") + assert(store.getSingle("a3").isDefined, "a3 was not in store") + } - // All unroll memory used is released because unrollSafely returned an array - memoryStore.putIterator("b1", smallIterator, memOnly, returnValues = true) - assert(memoryStore.currentUnrollMemoryForThisTask === 0) - memoryStore.putIterator("b2", smallIterator, memOnly, returnValues = true) - assert(memoryStore.currentUnrollMemoryForThisTask === 0) + private def testReadWithLossOfOnDiskFiles( + storageLevel: StorageLevel, + readMethod: BlockManager => Option[_]): Unit = { + store = makeBlockManager(12000) + assert(store.putSingle("blockId", new Array[Byte](4000), storageLevel)) + assert(store.getStatus("blockId").isDefined) + // Directly delete all files from the disk store, triggering failures when reading blocks: + store.diskBlockManager.getAllFiles().foreach(_.delete()) + // The BlockManager still thinks that these blocks exist: + assert(store.getStatus("blockId").isDefined) + // Because the BlockManager's metadata claims that the block exists (i.e. that it's present + // in at least one store), the read attempts to read it and fails when the on-disk file is + // missing. + intercept[SparkException] { + readMethod(store) + } + // Subsequent read attempts will succeed; the block isn't present but we return an expected + // "block not found" response rather than a fatal error: + assert(readMethod(store).isEmpty) + // The reason why this second read succeeded is because the metadata entry for the missing + // block was removed as a result of the read failure: + assert(store.getStatus("blockId").isEmpty) + } - // Unroll memory is not released because unrollSafely returned an iterator - // that still depends on the underlying vector used in the process - memoryStore.putIterator("b3", smallIterator, memOnly, returnValues = true) - val unrollMemoryAfterB3 = memoryStore.currentUnrollMemoryForThisTask - assert(unrollMemoryAfterB3 > 0) - - // The unroll memory owned by this thread builds on top of its value after the previous unrolls - memoryStore.putIterator("b4", smallIterator, memOnly, returnValues = true) - val unrollMemoryAfterB4 = memoryStore.currentUnrollMemoryForThisTask - assert(unrollMemoryAfterB4 > unrollMemoryAfterB3) - - // ... but only to a certain extent (until we run out of free space to grant new unroll memory) - memoryStore.putIterator("b5", smallIterator, memOnly, returnValues = true) - val unrollMemoryAfterB5 = memoryStore.currentUnrollMemoryForThisTask - memoryStore.putIterator("b6", smallIterator, memOnly, returnValues = true) - val unrollMemoryAfterB6 = memoryStore.currentUnrollMemoryForThisTask - memoryStore.putIterator("b7", smallIterator, memOnly, returnValues = true) - val unrollMemoryAfterB7 = memoryStore.currentUnrollMemoryForThisTask - assert(unrollMemoryAfterB5 === unrollMemoryAfterB4) - assert(unrollMemoryAfterB6 === unrollMemoryAfterB4) - assert(unrollMemoryAfterB7 === unrollMemoryAfterB4) + test("remove block if a read fails due to missing DiskStore files (SPARK-15736)") { + val storageLevels = Seq( + StorageLevel(useDisk = true, useMemory = false, deserialized = false), + StorageLevel(useDisk = true, useMemory = false, deserialized = true)) + val readMethods = Map[String, BlockManager => Option[_]]( + "getLocalBytes" -> ((m: BlockManager) => m.getLocalBytes("blockId")), + "getLocalValues" -> ((m: BlockManager) => m.getLocalValues("blockId")) + ) + testReadWithLossOfOnDiskFiles(StorageLevel.DISK_ONLY, _.getLocalBytes("blockId")) + for ((readMethodName, readMethod) <- readMethods; storageLevel <- storageLevels) { + withClue(s"$readMethodName $storageLevel") { + testReadWithLossOfOnDiskFiles(storageLevel, readMethod) + } + } } - test("lazily create a big ByteBuffer to avoid OOM if it cannot be put into MemoryStore") { - store = makeBlockManager(12000) - val memoryStore = store.memoryStore - val blockId = BlockId("rdd_3_10") - val result = memoryStore.putBytes(blockId, 13000, () => { - fail("A big ByteBuffer that cannot be put into MemoryStore should not be created") - }) - assert(result.size === 13000) - assert(result.data === null) - assert(result.droppedBlocks === Nil) + test("SPARK-13328: refresh block locations (fetch should fail after hitting a threshold)") { + val mockBlockTransferService = + new MockBlockTransferService(conf.getInt("spark.block.failures.beforeLocationRefresh", 5)) + store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) + store.putSingle("item", 999L, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(store.getRemoteBytes("item").isEmpty) } - test("put a small ByteBuffer to MemoryStore") { - store = makeBlockManager(12000) - val memoryStore = store.memoryStore - val blockId = BlockId("rdd_3_10") - var bytes: ByteBuffer = null - val result = memoryStore.putBytes(blockId, 10000, () => { - bytes = ByteBuffer.allocate(10000) - bytes - }) - assert(result.size === 10000) - assert(result.data === Right(bytes)) - assert(result.droppedBlocks === Nil) + test("SPARK-13328: refresh block locations (fetch should succeed after location refresh)") { + val maxFailuresBeforeLocationRefresh = + conf.getInt("spark.block.failures.beforeLocationRefresh", 5) + val mockBlockManagerMaster = mock(classOf[BlockManagerMaster]) + val mockBlockTransferService = + new MockBlockTransferService(maxFailuresBeforeLocationRefresh) + // make sure we have more than maxFailuresBeforeLocationRefresh locations + // so that we have a chance to do location refresh + val blockManagerIds = (0 to maxFailuresBeforeLocationRefresh) + .map { i => BlockManagerId(s"id-$i", s"host-$i", i + 1) } + when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn(blockManagerIds) + store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, + transferService = Option(mockBlockTransferService)) + val block = store.getRemoteBytes("item") + .asInstanceOf[Option[ByteBuffer]] + assert(block.isDefined) + verify(mockBlockManagerMaster, times(2)).getLocations("item") + } + + test("SPARK-17484: block status is properly updated following an exception in put()") { + val mockBlockTransferService = new MockBlockTransferService(maxFailures = 10) { + override def uploadBlock( + hostname: String, + port: Int, execId: String, + blockId: BlockId, + blockData: ManagedBuffer, + level: StorageLevel, + classTag: ClassTag[_]): Future[Unit] = { + throw new InterruptedException("Intentional interrupt") + } + } + store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) + store2 = makeBlockManager(8000, "executor2", transferService = Option(mockBlockTransferService)) + intercept[InterruptedException] { + store.putSingle("item", "value", StorageLevel.MEMORY_ONLY_2, tellMaster = true) + } + assert(store.getLocalBytes("item").isEmpty) + assert(master.getLocations("item").isEmpty) + assert(store2.getRemoteBytes("item").isEmpty) + } + + test("SPARK-17484: master block locations are updated following an invalid remote block fetch") { + store = makeBlockManager(8000, "executor1") + store2 = makeBlockManager(8000, "executor2") + store.putSingle("item", "value", StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(master.getLocations("item").nonEmpty) + store.removeBlock("item", tellMaster = false) + assert(master.getLocations("item").nonEmpty) + assert(store2.getRemoteBytes("item").isEmpty) + assert(master.getLocations("item").isEmpty) + } + + test("SPARK-20640: Shuffle registration timeout and maxAttempts conf are working") { + val tryAgainMsg = "test_spark_20640_try_again" + // a server which delays response 50ms and must try twice for success. + def newShuffleServer(port: Int): (TransportServer, Int) = { + val attempts = new mutable.HashMap[String, Int]() + val handler = new NoOpRpcHandler { + override def receive( + client: TransportClient, + message: ByteBuffer, + callback: RpcResponseCallback): Unit = { + val msgObj = BlockTransferMessage.Decoder.fromByteBuffer(message) + msgObj match { + case exec: RegisterExecutor => + Thread.sleep(50) + val attempt = attempts.getOrElse(exec.execId, 0) + 1 + attempts(exec.execId) = attempt + if (attempt < 2) { + callback.onFailure(new Exception(tryAgainMsg)) + return + } + callback.onSuccess(ByteBuffer.wrap(new Array[Byte](0))) + } + } + } + + val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores = 0) + val transCtx = new TransportContext(transConf, handler, true) + (transCtx.createServer(port, Seq.empty[TransportServerBootstrap].asJava), port) + } + val candidatePort = RandomUtils.nextInt(1024, 65536) + val (server, shufflePort) = Utils.startServiceOnPort(candidatePort, + newShuffleServer, conf, "ShuffleServer") + + conf.set("spark.shuffle.service.enabled", "true") + conf.set("spark.shuffle.service.port", shufflePort.toString) + conf.set(SHUFFLE_REGISTRATION_TIMEOUT.key, "40") + conf.set(SHUFFLE_REGISTRATION_MAX_ATTEMPTS.key, "1") + var e = intercept[SparkException]{ + makeBlockManager(8000, "executor1") + }.getMessage + assert(e.contains("TimeoutException")) + + conf.set(SHUFFLE_REGISTRATION_TIMEOUT.key, "1000") + conf.set(SHUFFLE_REGISTRATION_MAX_ATTEMPTS.key, "1") + e = intercept[SparkException]{ + makeBlockManager(8000, "executor2") + }.getMessage + assert(e.contains(tryAgainMsg)) + + conf.set(SHUFFLE_REGISTRATION_TIMEOUT.key, "1000") + conf.set(SHUFFLE_REGISTRATION_MAX_ATTEMPTS.key, "2") + makeBlockManager(8000, "executor3") + server.close() + } + + class MockBlockTransferService(val maxFailures: Int) extends BlockTransferService { + var numCalls = 0 + + override def init(blockDataManager: BlockDataManager): Unit = {} + + override def fetchBlocks( + host: String, + port: Int, + execId: String, + blockIds: Array[String], + listener: BlockFetchingListener, + tempShuffleFileManager: TempShuffleFileManager): Unit = { + listener.onBlockFetchSuccess("mockBlockId", new NioManagedBuffer(ByteBuffer.allocate(1))) + } + + override def close(): Unit = {} + + override def hostName: String = { "MockBlockTransferServiceHost" } + + override def port: Int = { 63332 } + + override def uploadBlock( + hostname: String, + port: Int, execId: String, + blockId: BlockId, + blockData: ManagedBuffer, + level: StorageLevel, + classTag: ClassTag[_]): Future[Unit] = { + import scala.concurrent.ExecutionContext.Implicits.global + Future {} + } + + override def fetchBlockSync( + host: String, + port: Int, + execId: String, + blockId: String): ManagedBuffer = { + numCalls += 1 + if (numCalls <= maxFailures) { + throw new RuntimeException("Failing block fetch in the mock block transfer service") + } + super.fetchBlockSync(host, port, execId, blockId) + } } } + +private object BlockManagerSuite { + + private implicit class BlockManagerTestUtils(store: BlockManager) { + + def dropFromMemoryIfExists( + blockId: BlockId, + data: () => Either[Array[Any], ChunkedByteBuffer]): Unit = { + store.blockInfoManager.lockForWriting(blockId).foreach { info => + val newEffectiveStorageLevel = store.dropFromMemory(blockId, data) + if (newEffectiveStorageLevel.isValid) { + // The block is still present in at least one store, so release the lock + // but don't delete the block info + store.releaseLock(blockId) + } else { + // The block isn't present in any store, so delete the block info so that the + // block can be stored again + store.blockInfoManager.removeBlock(blockId) + } + } + } + + private def wrapGet[T](f: BlockId => Option[T]): BlockId => Option[T] = (blockId: BlockId) => { + val result = f(blockId) + if (result.isDefined) { + store.releaseLock(blockId) + } + result + } + + def hasLocalBlock(blockId: BlockId): Boolean = { + getLocalAndReleaseLock(blockId).isDefined + } + + val getLocalAndReleaseLock: (BlockId) => Option[BlockResult] = wrapGet(store.getLocalValues) + val getAndReleaseLock: (BlockId) => Option[BlockResult] = wrapGet(store.get) + val getSingleAndReleaseLock: (BlockId) => Option[Any] = wrapGet(store.getSingle) + val getLocalBytesAndReleaseLock: (BlockId) => Option[ChunkedByteBuffer] = { + val allocator = ByteBuffer.allocate _ + wrapGet { bid => store.getLocalBytes(bid).map(_.toChunkedByteBuffer(allocator)) } + } + } + +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockReplicationPolicySuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockReplicationPolicySuite.scala new file mode 100644 index 000000000000..4000218e71a8 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockReplicationPolicySuite.scala @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import scala.collection.mutable +import scala.language.implicitConversions +import scala.util.Random + +import org.scalatest.{BeforeAndAfter, Matchers} + +import org.apache.spark.{LocalSparkContext, SparkFunSuite} + +class RandomBlockReplicationPolicyBehavior extends SparkFunSuite + with Matchers + with BeforeAndAfter + with LocalSparkContext { + + // Implicitly convert strings to BlockIds for test clarity. + protected implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) + + val replicationPolicy: BlockReplicationPolicy = new RandomBlockReplicationPolicy + + val blockId = "test-block" + /** + * Test if we get the required number of peers when using random sampling from + * BlockReplicationPolicy + */ + test("block replication - random block replication policy") { + val numBlockManagers = 10 + val storeSize = 1000 + val blockManagers = generateBlockManagerIds(numBlockManagers, Seq("/Rack-1")) + val candidateBlockManager = BlockManagerId("test-store", "localhost", 1000, None) + + (1 to 10).foreach { numReplicas => + logDebug(s"Num replicas : $numReplicas") + val randomPeers = replicationPolicy.prioritize( + candidateBlockManager, + blockManagers, + mutable.HashSet.empty[BlockManagerId], + blockId, + numReplicas + ) + logDebug(s"Random peers : ${randomPeers.mkString(", ")}") + assert(randomPeers.toSet.size === numReplicas) + + // choosing n peers out of n + val secondPass = replicationPolicy.prioritize( + candidateBlockManager, + randomPeers, + mutable.HashSet.empty[BlockManagerId], + blockId, + numReplicas + ) + logDebug(s"Random peers : ${secondPass.mkString(", ")}") + assert(secondPass.toSet.size === numReplicas) + } + } + + /** + * Returns a sequence of [[BlockManagerId]], whose rack is randomly picked from the given `racks`. + * Note that, each rack will be picked at least once from `racks`, if `count` is greater or equal + * to the number of `racks`. + */ + protected def generateBlockManagerIds(count: Int, racks: Seq[String]): Seq[BlockManagerId] = { + val randomizedRacks: Seq[String] = Random.shuffle( + racks ++ racks.length.until(count).map(_ => racks(Random.nextInt(racks.length))) + ) + + (0 until count).map { i => + BlockManagerId(s"Exec-$i", s"Host-$i", 10000 + i, Some(randomizedRacks(i))) + } + } +} + +class TopologyAwareBlockReplicationPolicyBehavior extends RandomBlockReplicationPolicyBehavior { + override val replicationPolicy = new BasicBlockReplicationPolicy + + test("All peers in the same rack") { + val racks = Seq("/default-rack") + val numBlockManager = 10 + (1 to 10).foreach {numReplicas => + val peers = generateBlockManagerIds(numBlockManager, racks) + val blockManager = BlockManagerId("Driver", "Host-driver", 10001, Some(racks.head)) + + val prioritizedPeers = replicationPolicy.prioritize( + blockManager, + peers, + mutable.HashSet.empty, + blockId, + numReplicas + ) + + assert(prioritizedPeers.toSet.size == numReplicas) + assert(prioritizedPeers.forall(p => p.host != blockManager.host)) + } + } + + test("Peers in 2 racks") { + val racks = Seq("/Rack-1", "/Rack-2") + (1 to 10).foreach {numReplicas => + val peers = generateBlockManagerIds(10, racks) + val blockManager = BlockManagerId("Driver", "Host-driver", 9001, Some(racks.head)) + + val prioritizedPeers = replicationPolicy.prioritize( + blockManager, + peers, + mutable.HashSet.empty, + blockId, + numReplicas + ) + + assert(prioritizedPeers.toSet.size == numReplicas) + val priorityPeers = prioritizedPeers.take(2) + assert(priorityPeers.forall(p => p.host != blockManager.host)) + if(numReplicas > 1) { + // both these conditions should be satisfied when numReplicas > 1 + assert(priorityPeers.exists(p => p.topologyInfo == blockManager.topologyInfo)) + assert(priorityPeers.exists(p => p.topologyInfo != blockManager.topologyInfo)) + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockStatusListenerSuite.scala index d7ffde1e7864..06acca3943c2 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockStatusListenerSuite.scala @@ -34,16 +34,14 @@ class BlockStatusListenerSuite extends SparkFunSuite { StreamBlockId(0, 100), StorageLevel.MEMORY_AND_DISK, memSize = 100, - diskSize = 100, - externalBlockStoreSize = 0))) + diskSize = 100))) // The new block status should be added to the listener val expectedBlock = BlockUIData( StreamBlockId(0, 100), "localhost:10000", StorageLevel.MEMORY_AND_DISK, memSize = 100, - diskSize = 100, - externalBlockStoreSize = 0 + diskSize = 100 ) val expectedExecutorStreamBlockStatus = Seq( ExecutorStreamBlockStatus("0", "localhost:10000", Seq(expectedBlock)) @@ -60,15 +58,13 @@ class BlockStatusListenerSuite extends SparkFunSuite { StreamBlockId(0, 100), StorageLevel.MEMORY_AND_DISK, memSize = 100, - diskSize = 100, - externalBlockStoreSize = 0))) + diskSize = 100))) val expectedBlock2 = BlockUIData( StreamBlockId(0, 100), "localhost:10001", StorageLevel.MEMORY_AND_DISK, memSize = 100, - diskSize = 100, - externalBlockStoreSize = 0 + diskSize = 100 ) // Each block manager should contain one block val expectedExecutorStreamBlockStatus2 = Set( @@ -84,8 +80,7 @@ class BlockStatusListenerSuite extends SparkFunSuite { StreamBlockId(0, 100), StorageLevel.NONE, // StorageLevel.NONE means removing it memSize = 0, - diskSize = 0, - externalBlockStoreSize = 0))) + diskSize = 0))) // Only the first block manager contains a block val expectedExecutorStreamBlockStatus3 = Set( ExecutorStreamBlockStatus("0", "localhost:10000", Seq(expectedBlock)), @@ -102,8 +97,7 @@ class BlockStatusListenerSuite extends SparkFunSuite { StreamBlockId(0, 100), StorageLevel.MEMORY_AND_DISK, memSize = 100, - diskSize = 100, - externalBlockStoreSize = 0))) + diskSize = 100))) // The second block manager is removed so we should not see the new block val expectedExecutorStreamBlockStatus4 = Seq( ExecutorStreamBlockStatus("0", "localhost:10000", Seq(expectedBlock)) diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 688f56f4665f..7859b0bba2b4 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -19,9 +19,6 @@ package org.apache.spark.storage import java.io.{File, FileWriter} -import scala.language.reflectiveCalls - -import org.mockito.Mockito.{mock, when} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} import org.apache.spark.{SparkConf, SparkFunSuite} @@ -33,8 +30,6 @@ class DiskBlockManagerSuite extends SparkFunSuite with BeforeAndAfterEach with B private var rootDir1: File = _ private var rootDirs: String = _ - val blockManager = mock(classOf[BlockManager]) - when(blockManager.conf).thenReturn(testConf) var diskBlockManager: DiskBlockManager = _ override def beforeAll() { @@ -45,19 +40,27 @@ class DiskBlockManagerSuite extends SparkFunSuite with BeforeAndAfterEach with B } override def afterAll() { - super.afterAll() - Utils.deleteRecursively(rootDir0) - Utils.deleteRecursively(rootDir1) + try { + Utils.deleteRecursively(rootDir0) + Utils.deleteRecursively(rootDir1) + } finally { + super.afterAll() + } } override def beforeEach() { + super.beforeEach() val conf = testConf.clone conf.set("spark.local.dir", rootDirs) - diskBlockManager = new DiskBlockManager(blockManager, conf) + diskBlockManager = new DiskBlockManager(conf, deleteFilesOnStop = true) } override def afterEach() { - diskBlockManager.stop() + try { + diskBlockManager.stop() + } finally { + super.afterEach() + } } test("basic block creation") { diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala index 7c19531c1880..cea55012c1de 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.serializer.{JavaSerializer, SerializerManager} import org.apache.spark.util.Utils class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { @@ -30,63 +30,71 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { var tempDir: File = _ override def beforeEach(): Unit = { + super.beforeEach() tempDir = Utils.createTempDir() } override def afterEach(): Unit = { - Utils.deleteRecursively(tempDir) + try { + Utils.deleteRecursively(tempDir) + } finally { + super.afterEach() + } } - test("verify write metrics") { + private def createWriter(): (DiskBlockObjectWriter, File, ShuffleWriteMetrics) = { val file = new File(tempDir, "somefile") + val conf = new SparkConf() + val serializerManager = new SerializerManager(new JavaSerializer(conf), conf) val writeMetrics = new ShuffleWriteMetrics() val writer = new DiskBlockObjectWriter( - file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) + file, serializerManager, new JavaSerializer(new SparkConf()).newInstance(), 1024, true, + writeMetrics) + (writer, file, writeMetrics) + } + + test("verify write metrics") { + val (writer, file, writeMetrics) = createWriter() writer.write(Long.box(20), Long.box(30)) // Record metrics update on every write - assert(writeMetrics.shuffleRecordsWritten === 1) + assert(writeMetrics.recordsWritten === 1) // Metrics don't update on every write - assert(writeMetrics.shuffleBytesWritten == 0) - // After 32 writes, metrics should update - for (i <- 0 until 32) { + assert(writeMetrics.bytesWritten == 0) + // After 16384 writes, metrics should update + for (i <- 0 until 16384) { writer.flush() writer.write(Long.box(i), Long.box(i)) } - assert(writeMetrics.shuffleBytesWritten > 0) - assert(writeMetrics.shuffleRecordsWritten === 33) - writer.commitAndClose() - assert(file.length() == writeMetrics.shuffleBytesWritten) + assert(writeMetrics.bytesWritten > 0) + assert(writeMetrics.recordsWritten === 16385) + writer.commitAndGet() + writer.close() + assert(file.length() == writeMetrics.bytesWritten) } test("verify write metrics on revert") { - val file = new File(tempDir, "somefile") - val writeMetrics = new ShuffleWriteMetrics() - val writer = new DiskBlockObjectWriter( - file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) + val (writer, _, writeMetrics) = createWriter() writer.write(Long.box(20), Long.box(30)) // Record metrics update on every write - assert(writeMetrics.shuffleRecordsWritten === 1) + assert(writeMetrics.recordsWritten === 1) // Metrics don't update on every write - assert(writeMetrics.shuffleBytesWritten == 0) - // After 32 writes, metrics should update - for (i <- 0 until 32) { + assert(writeMetrics.bytesWritten == 0) + // After 16384 writes, metrics should update + for (i <- 0 until 16384) { writer.flush() writer.write(Long.box(i), Long.box(i)) } - assert(writeMetrics.shuffleBytesWritten > 0) - assert(writeMetrics.shuffleRecordsWritten === 33) + assert(writeMetrics.bytesWritten > 0) + assert(writeMetrics.recordsWritten === 16385) writer.revertPartialWritesAndClose() - assert(writeMetrics.shuffleBytesWritten == 0) - assert(writeMetrics.shuffleRecordsWritten == 0) + assert(writeMetrics.bytesWritten == 0) + assert(writeMetrics.recordsWritten == 0) } test("Reopening a closed block writer") { - val file = new File(tempDir, "somefile") - val writeMetrics = new ShuffleWriteMetrics() - val writer = new DiskBlockObjectWriter( - file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) + val (writer, _, _) = createWriter() writer.open() writer.close() @@ -95,78 +103,85 @@ class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { } } + test("calling revertPartialWritesAndClose() on a partial write should truncate up to commit") { + val (writer, file, writeMetrics) = createWriter() + + writer.write(Long.box(20), Long.box(30)) + val firstSegment = writer.commitAndGet() + assert(firstSegment.length === file.length()) + assert(writeMetrics.bytesWritten === file.length()) + + writer.write(Long.box(40), Long.box(50)) + + writer.revertPartialWritesAndClose() + assert(firstSegment.length === file.length()) + assert(writeMetrics.bytesWritten === file.length()) + assert(writeMetrics.recordsWritten == 1) + } + + test("calling revertPartialWritesAndClose() after commit() should have no effect") { + val (writer, file, writeMetrics) = createWriter() + + writer.write(Long.box(20), Long.box(30)) + val firstSegment = writer.commitAndGet() + assert(firstSegment.length === file.length()) + assert(writeMetrics.bytesWritten === file.length()) + + writer.revertPartialWritesAndClose() + assert(firstSegment.length === file.length()) + assert(writeMetrics.bytesWritten === file.length()) + } + test("calling revertPartialWritesAndClose() on a closed block writer should have no effect") { - val file = new File(tempDir, "somefile") - val writeMetrics = new ShuffleWriteMetrics() - val writer = new DiskBlockObjectWriter( - file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) + val (writer, file, writeMetrics) = createWriter() for (i <- 1 to 1000) { writer.write(i, i) } - writer.commitAndClose() - val bytesWritten = writeMetrics.shuffleBytesWritten - assert(writeMetrics.shuffleRecordsWritten === 1000) + writer.commitAndGet() + writer.close() + val bytesWritten = writeMetrics.bytesWritten + assert(writeMetrics.recordsWritten === 1000) writer.revertPartialWritesAndClose() - assert(writeMetrics.shuffleRecordsWritten === 1000) - assert(writeMetrics.shuffleBytesWritten === bytesWritten) + assert(writeMetrics.recordsWritten === 1000) + assert(writeMetrics.bytesWritten === bytesWritten) } - test("commitAndClose() should be idempotent") { - val file = new File(tempDir, "somefile") - val writeMetrics = new ShuffleWriteMetrics() - val writer = new DiskBlockObjectWriter( - file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) + test("commit() and close() should be idempotent") { + val (writer, file, writeMetrics) = createWriter() for (i <- 1 to 1000) { writer.write(i, i) } - writer.commitAndClose() - val bytesWritten = writeMetrics.shuffleBytesWritten - val writeTime = writeMetrics.shuffleWriteTime - assert(writeMetrics.shuffleRecordsWritten === 1000) - writer.commitAndClose() - assert(writeMetrics.shuffleRecordsWritten === 1000) - assert(writeMetrics.shuffleBytesWritten === bytesWritten) - assert(writeMetrics.shuffleWriteTime === writeTime) + writer.commitAndGet() + writer.close() + val bytesWritten = writeMetrics.bytesWritten + val writeTime = writeMetrics.writeTime + assert(writeMetrics.recordsWritten === 1000) + writer.commitAndGet() + writer.close() + assert(writeMetrics.recordsWritten === 1000) + assert(writeMetrics.bytesWritten === bytesWritten) + assert(writeMetrics.writeTime === writeTime) } test("revertPartialWritesAndClose() should be idempotent") { - val file = new File(tempDir, "somefile") - val writeMetrics = new ShuffleWriteMetrics() - val writer = new DiskBlockObjectWriter( - file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) + val (writer, file, writeMetrics) = createWriter() for (i <- 1 to 1000) { writer.write(i, i) } writer.revertPartialWritesAndClose() - val bytesWritten = writeMetrics.shuffleBytesWritten - val writeTime = writeMetrics.shuffleWriteTime - assert(writeMetrics.shuffleRecordsWritten === 0) + val bytesWritten = writeMetrics.bytesWritten + val writeTime = writeMetrics.writeTime + assert(writeMetrics.recordsWritten === 0) writer.revertPartialWritesAndClose() - assert(writeMetrics.shuffleRecordsWritten === 0) - assert(writeMetrics.shuffleBytesWritten === bytesWritten) - assert(writeMetrics.shuffleWriteTime === writeTime) + assert(writeMetrics.recordsWritten === 0) + assert(writeMetrics.bytesWritten === bytesWritten) + assert(writeMetrics.writeTime === writeTime) } - test("fileSegment() can only be called after commitAndClose() has been called") { - val file = new File(tempDir, "somefile") - val writeMetrics = new ShuffleWriteMetrics() - val writer = new DiskBlockObjectWriter( - file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) - for (i <- 1 to 1000) { - writer.write(i, i) - } - intercept[IllegalStateException] { - writer.fileSegment() - } + test("commit() and close() without ever opening or writing") { + val (writer, _, _) = createWriter() + val segment = writer.commitAndGet() writer.close() - } - - test("commitAndClose() without ever opening or writing") { - val file = new File(tempDir, "somefile") - val writeMetrics = new ShuffleWriteMetrics() - val writer = new DiskBlockObjectWriter( - file, new JavaSerializer(new SparkConf()).newInstance(), 1024, os => os, true, writeMetrics) - writer.commitAndClose() - assert(writer.fileSegment().length === 0) + assert(segment.length === 0) } } diff --git a/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala new file mode 100644 index 000000000000..7258fdf5efc0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.nio.{ByteBuffer, MappedByteBuffer} +import java.util.{Arrays, Random} + +import com.google.common.io.{ByteStreams, Files} +import io.netty.channel.FileRegion + +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.network.util.{ByteArrayWritableChannel, JavaUtils} +import org.apache.spark.security.CryptoStreamUtils +import org.apache.spark.util.Utils +import org.apache.spark.util.io.ChunkedByteBuffer + +class DiskStoreSuite extends SparkFunSuite { + + test("reads of memory-mapped and non memory-mapped files are equivalent") { + val conf = new SparkConf() + val securityManager = new SecurityManager(conf) + + // It will cause error when we tried to re-open the filestore and the + // memory-mapped byte buffer tot he file has not been GC on Windows. + assume(!Utils.isWindows) + val confKey = "spark.storage.memoryMapThreshold" + + // Create a non-trivial (not all zeros) byte array + val bytes = Array.tabulate[Byte](1000)(_.toByte) + val byteBuffer = new ChunkedByteBuffer(ByteBuffer.wrap(bytes)) + + val blockId = BlockId("rdd_1_2") + val diskBlockManager = new DiskBlockManager(conf, deleteFilesOnStop = true) + + val diskStoreMapped = new DiskStore(conf.clone().set(confKey, "0"), diskBlockManager, + securityManager) + diskStoreMapped.putBytes(blockId, byteBuffer) + val mapped = diskStoreMapped.getBytes(blockId).toByteBuffer() + assert(diskStoreMapped.remove(blockId)) + + val diskStoreNotMapped = new DiskStore(conf.clone().set(confKey, "1m"), diskBlockManager, + securityManager) + diskStoreNotMapped.putBytes(blockId, byteBuffer) + val notMapped = diskStoreNotMapped.getBytes(blockId).toByteBuffer() + + // Not possible to do isInstanceOf due to visibility of HeapByteBuffer + assert(notMapped.getClass.getName.endsWith("HeapByteBuffer"), + "Expected HeapByteBuffer for un-mapped read") + assert(mapped.isInstanceOf[MappedByteBuffer], + "Expected MappedByteBuffer for mapped read") + + def arrayFromByteBuffer(in: ByteBuffer): Array[Byte] = { + val array = new Array[Byte](in.remaining()) + in.get(array) + array + } + + assert(Arrays.equals(new ChunkedByteBuffer(mapped).toArray, bytes)) + assert(Arrays.equals(new ChunkedByteBuffer(notMapped).toArray, bytes)) + } + + test("block size tracking") { + val conf = new SparkConf() + val diskBlockManager = new DiskBlockManager(conf, deleteFilesOnStop = true) + val diskStore = new DiskStore(conf, diskBlockManager, new SecurityManager(conf)) + + val blockId = BlockId("rdd_1_2") + diskStore.put(blockId) { chan => + val buf = ByteBuffer.wrap(new Array[Byte](32)) + while (buf.hasRemaining()) { + chan.write(buf) + } + } + + assert(diskStore.getSize(blockId) === 32L) + diskStore.remove(blockId) + assert(diskStore.getSize(blockId) === 0L) + } + + test("blocks larger than 2gb") { + val conf = new SparkConf() + .set("spark.storage.memoryMapLimitForTests", "10k" ) + val diskBlockManager = new DiskBlockManager(conf, deleteFilesOnStop = true) + val diskStore = new DiskStore(conf, diskBlockManager, new SecurityManager(conf)) + + val blockId = BlockId("rdd_1_2") + diskStore.put(blockId) { chan => + val arr = new Array[Byte](1024) + for { + _ <- 0 until 20 + } { + val buf = ByteBuffer.wrap(arr) + while (buf.hasRemaining()) { + chan.write(buf) + } + } + } + + val blockData = diskStore.getBytes(blockId) + assert(blockData.size == 20 * 1024) + + val chunkedByteBuffer = blockData.toChunkedByteBuffer(ByteBuffer.allocate) + val chunks = chunkedByteBuffer.chunks + assert(chunks.size === 2) + for (chunk <- chunks) { + assert(chunk.limit === 10 * 1024) + } + + val e = intercept[IllegalArgumentException]{ + blockData.toByteBuffer() + } + + assert(e.getMessage === + s"requirement failed: can't create a byte buffer of size ${blockData.size}" + + " since it exceeds 10.0 KB.") + } + + test("block data encryption") { + val testDir = Utils.createTempDir() + val testData = new Array[Byte](128 * 1024) + new Random().nextBytes(testData) + + val conf = new SparkConf() + val securityManager = new SecurityManager(conf, Some(CryptoStreamUtils.createKey(conf))) + val diskBlockManager = new DiskBlockManager(conf, deleteFilesOnStop = true) + val diskStore = new DiskStore(conf, diskBlockManager, securityManager) + + val blockId = BlockId("rdd_1_2") + diskStore.put(blockId) { chan => + val buf = ByteBuffer.wrap(testData) + while (buf.hasRemaining()) { + chan.write(buf) + } + } + + assert(diskStore.getSize(blockId) === testData.length) + + val diskData = Files.toByteArray(diskBlockManager.getFile(blockId.name)) + assert(!Arrays.equals(testData, diskData)) + + val blockData = diskStore.getBytes(blockId) + assert(blockData.isInstanceOf[EncryptedBlockData]) + assert(blockData.size === testData.length) + Map( + "input stream" -> readViaInputStream _, + "chunked byte buffer" -> readViaChunkedByteBuffer _, + "nio byte buffer" -> readViaNioBuffer _, + "managed buffer" -> readViaManagedBuffer _ + ).foreach { case (name, fn) => + val readData = fn(blockData) + assert(readData.length === blockData.size, s"Size of data read via $name did not match.") + assert(Arrays.equals(testData, readData), s"Data read via $name did not match.") + } + } + + private def readViaInputStream(data: BlockData): Array[Byte] = { + val is = data.toInputStream() + try { + ByteStreams.toByteArray(is) + } finally { + is.close() + } + } + + private def readViaChunkedByteBuffer(data: BlockData): Array[Byte] = { + val buf = data.toChunkedByteBuffer(ByteBuffer.allocate _) + try { + buf.toArray + } finally { + buf.dispose() + } + } + + private def readViaNioBuffer(data: BlockData): Array[Byte] = { + JavaUtils.bufferToArray(data.toByteBuffer()) + } + + private def readViaManagedBuffer(data: BlockData): Array[Byte] = { + val region = data.toNetty().asInstanceOf[FileRegion] + val byteChannel = new ByteArrayWritableChannel(data.size.toInt) + + while (region.transfered() < region.count()) { + region.transferTo(byteChannel, region.transfered()) + } + + byteChannel.close() + byteChannel.getData + } + +} diff --git a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala index cc50289c7b3e..6883eb211efd 100644 --- a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala @@ -17,13 +17,12 @@ package org.apache.spark.storage -import java.io.File +import java.io.{File, IOException} -import org.apache.spark.util.Utils import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.util.SparkConfWithEnv +import org.apache.spark.util.{SparkConfWithEnv, Utils} /** * Tests for the spark.local.dir and SPARK_LOCAL_DIRS configuration options. @@ -34,22 +33,66 @@ class LocalDirsSuite extends SparkFunSuite with BeforeAndAfter { Utils.clearLocalRootDirs() } + after { + Utils.clearLocalRootDirs() + } + + private def assumeNonExistentAndNotCreatable(f: File): Unit = { + try { + assume(!f.exists() && !f.mkdirs()) + } finally { + Utils.deleteRecursively(f) + } + } + test("Utils.getLocalDir() returns a valid directory, even if some local dirs are missing") { // Regression test for SPARK-2974 - assert(!new File("/NONEXISTENT_DIR").exists()) + val f = new File("/NONEXISTENT_PATH") + assumeNonExistentAndNotCreatable(f) + val conf = new SparkConf(false) .set("spark.local.dir", s"/NONEXISTENT_PATH,${System.getProperty("java.io.tmpdir")}") assert(new File(Utils.getLocalDir(conf)).exists()) + + // This directory should not be created. + assert(!f.exists()) } test("SPARK_LOCAL_DIRS override also affects driver") { - // Regression test for SPARK-2975 - assert(!new File("/NONEXISTENT_DIR").exists()) + // Regression test for SPARK-2974 + val f = new File("/NONEXISTENT_PATH") + assumeNonExistentAndNotCreatable(f) + // spark.local.dir only contains invalid directories, but that's not a problem since // SPARK_LOCAL_DIRS will override it on both the driver and workers: val conf = new SparkConfWithEnv(Map("SPARK_LOCAL_DIRS" -> System.getProperty("java.io.tmpdir"))) .set("spark.local.dir", "/NONEXISTENT_PATH") assert(new File(Utils.getLocalDir(conf)).exists()) + + // This directory should not be created. + assert(!f.exists()) } + test("Utils.getLocalDir() throws an exception if any temporary directory cannot be retrieved") { + val path1 = "/NONEXISTENT_PATH_ONE" + val path2 = "/NONEXISTENT_PATH_TWO" + val f1 = new File(path1) + val f2 = new File(path2) + assumeNonExistentAndNotCreatable(f1) + assumeNonExistentAndNotCreatable(f2) + + assert(!new File(path1).exists()) + assert(!new File(path2).exists()) + val conf = new SparkConf(false).set("spark.local.dir", s"$path1,$path2") + val message = intercept[IOException] { + Utils.getLocalDir(conf) + }.getMessage + // If any temporary directory could not be retrieved under the given paths above, it should + // throw an exception with the message that includes the paths. + assert(message.contains(s"$path1,$path2")) + + // These directories should not be created. + assert(!f1.exists()) + assert(!f2.exists()) + } } diff --git a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala new file mode 100644 index 000000000000..7274072e5049 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala @@ -0,0 +1,529 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.nio.ByteBuffer + +import scala.language.implicitConversions +import scala.language.reflectiveCalls +import scala.reflect.ClassTag + +import org.scalatest._ + +import org.apache.spark._ +import org.apache.spark.memory.{MemoryMode, StaticMemoryManager} +import org.apache.spark.serializer.{KryoSerializer, SerializerManager} +import org.apache.spark.storage.memory.{BlockEvictionHandler, MemoryStore, PartiallySerializedBlock, PartiallyUnrolledIterator} +import org.apache.spark.util._ +import org.apache.spark.util.io.ChunkedByteBuffer + +class MemoryStoreSuite + extends SparkFunSuite + with PrivateMethodTester + with BeforeAndAfterEach + with ResetSystemProperties { + + var conf: SparkConf = new SparkConf(false) + .set("spark.test.useCompressedOops", "true") + .set("spark.storage.unrollFraction", "0.4") + .set("spark.storage.unrollMemoryThreshold", "512") + + // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test + val serializer = new KryoSerializer(new SparkConf(false).set("spark.kryoserializer.buffer", "1m")) + + val serializerManager = new SerializerManager(serializer, conf) + + // Implicitly convert strings to BlockIds for test clarity. + implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) + def rdd(rddId: Int, splitId: Int): RDDBlockId = RDDBlockId(rddId, splitId) + + override def beforeEach(): Unit = { + super.beforeEach() + // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case + System.setProperty("os.arch", "amd64") + val initialize = PrivateMethod[Unit]('initialize) + SizeEstimator invokePrivate initialize() + } + + def makeMemoryStore(maxMem: Long): (MemoryStore, BlockInfoManager) = { + val memManager = new StaticMemoryManager(conf, Long.MaxValue, maxMem, numCores = 1) + val blockInfoManager = new BlockInfoManager + val blockEvictionHandler = new BlockEvictionHandler { + var memoryStore: MemoryStore = _ + override private[storage] def dropFromMemory[T: ClassTag]( + blockId: BlockId, + data: () => Either[Array[T], ChunkedByteBuffer]): StorageLevel = { + memoryStore.remove(blockId) + StorageLevel.NONE + } + } + val memoryStore = + new MemoryStore(conf, blockInfoManager, serializerManager, memManager, blockEvictionHandler) + memManager.setMemoryStore(memoryStore) + blockEvictionHandler.memoryStore = memoryStore + (memoryStore, blockInfoManager) + } + + private def assertSameContents[T](expected: Seq[T], actual: Seq[T], hint: String): Unit = { + assert(actual.length === expected.length, s"wrong number of values returned in $hint") + expected.iterator.zip(actual.iterator).foreach { case (e, a) => + assert(e === a, s"$hint did not return original values!") + } + } + + test("reserve/release unroll memory") { + val (memoryStore, _) = makeMemoryStore(12000) + assert(memoryStore.currentUnrollMemory === 0) + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + + def reserveUnrollMemoryForThisTask(memory: Long): Boolean = { + memoryStore.reserveUnrollMemoryForThisTask(TestBlockId(""), memory, MemoryMode.ON_HEAP) + } + + // Reserve + assert(reserveUnrollMemoryForThisTask(100)) + assert(memoryStore.currentUnrollMemoryForThisTask === 100) + assert(reserveUnrollMemoryForThisTask(200)) + assert(memoryStore.currentUnrollMemoryForThisTask === 300) + assert(reserveUnrollMemoryForThisTask(500)) + assert(memoryStore.currentUnrollMemoryForThisTask === 800) + assert(!reserveUnrollMemoryForThisTask(1000000)) + assert(memoryStore.currentUnrollMemoryForThisTask === 800) // not granted + // Release + memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, 100) + assert(memoryStore.currentUnrollMemoryForThisTask === 700) + memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, 100) + assert(memoryStore.currentUnrollMemoryForThisTask === 600) + // Reserve again + assert(reserveUnrollMemoryForThisTask(4400)) + assert(memoryStore.currentUnrollMemoryForThisTask === 5000) + assert(!reserveUnrollMemoryForThisTask(20000)) + assert(memoryStore.currentUnrollMemoryForThisTask === 5000) // not granted + // Release again + memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, 1000) + assert(memoryStore.currentUnrollMemoryForThisTask === 4000) + memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP) // release all + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + } + + test("safely unroll blocks") { + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + val ct = implicitly[ClassTag[Array[Byte]]] + val (memoryStore, blockInfoManager) = makeMemoryStore(12000) + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + + def putIteratorAsValues[T]( + blockId: BlockId, + iter: Iterator[T], + classTag: ClassTag[T]): Either[PartiallyUnrolledIterator[T], Long] = { + assert(blockInfoManager.lockNewBlockForWriting( + blockId, + new BlockInfo(StorageLevel.MEMORY_ONLY, classTag, tellMaster = false))) + val res = memoryStore.putIteratorAsValues(blockId, iter, classTag) + blockInfoManager.unlock(blockId) + res + } + + // Unroll with all the space in the world. This should succeed. + var putResult = putIteratorAsValues("unroll", smallList.iterator, ClassTag.Any) + assert(putResult.isRight) + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + assertSameContents(smallList, memoryStore.getValues("unroll").get.toSeq, "getValues") + blockInfoManager.lockForWriting("unroll") + assert(memoryStore.remove("unroll")) + blockInfoManager.removeBlock("unroll") + + // Unroll with not enough space. This should succeed after kicking out someBlock1. + assert(putIteratorAsValues("someBlock1", smallList.iterator, ct).isRight) + assert(putIteratorAsValues("someBlock2", smallList.iterator, ct).isRight) + putResult = putIteratorAsValues("unroll", smallList.iterator, ClassTag.Any) + assert(putResult.isRight) + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + assert(memoryStore.contains("someBlock2")) + assert(!memoryStore.contains("someBlock1")) + assertSameContents(smallList, memoryStore.getValues("unroll").get.toSeq, "getValues") + blockInfoManager.lockForWriting("unroll") + assert(memoryStore.remove("unroll")) + blockInfoManager.removeBlock("unroll") + + // Unroll huge block with not enough space. Even after ensuring free space of 12000 * 0.4 = + // 4800 bytes, there is still not enough room to unroll this block. This returns an iterator. + // In the meantime, however, we kicked out someBlock2 before giving up. + assert(putIteratorAsValues("someBlock3", smallList.iterator, ct).isRight) + putResult = putIteratorAsValues("unroll", bigList.iterator, ClassTag.Any) + assert(memoryStore.currentUnrollMemoryForThisTask > 0) // we returned an iterator + assert(!memoryStore.contains("someBlock2")) + assert(putResult.isLeft) + assertSameContents(bigList, putResult.left.get.toSeq, "putIterator") + // The unroll memory was freed once the iterator returned by putIterator() was fully traversed. + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + } + + test("safely unroll blocks through putIteratorAsValues") { + val (memoryStore, blockInfoManager) = makeMemoryStore(12000) + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator: Iterator[Any] = bigList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + + def putIteratorAsValues[T]( + blockId: BlockId, + iter: Iterator[T], + classTag: ClassTag[T]): Either[PartiallyUnrolledIterator[T], Long] = { + assert(blockInfoManager.lockNewBlockForWriting( + blockId, + new BlockInfo(StorageLevel.MEMORY_ONLY, classTag, tellMaster = false))) + val res = memoryStore.putIteratorAsValues(blockId, iter, classTag) + blockInfoManager.unlock(blockId) + res + } + + // Unroll with plenty of space. This should succeed and cache both blocks. + val result1 = putIteratorAsValues("b1", smallIterator, ClassTag.Any) + val result2 = putIteratorAsValues("b2", smallIterator, ClassTag.Any) + assert(memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(result1.isRight) // unroll was successful + assert(result2.isRight) + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + + // Re-put these two blocks so block manager knows about them too. Otherwise, block manager + // would not know how to drop them from memory later. + blockInfoManager.lockForWriting("b1") + memoryStore.remove("b1") + blockInfoManager.removeBlock("b1") + blockInfoManager.lockForWriting("b2") + memoryStore.remove("b2") + blockInfoManager.removeBlock("b2") + putIteratorAsValues("b1", smallIterator, ClassTag.Any) + putIteratorAsValues("b2", smallIterator, ClassTag.Any) + + // Unroll with not enough space. This should succeed but kick out b1 in the process. + val result3 = putIteratorAsValues("b3", smallIterator, ClassTag.Any) + assert(result3.isRight) + assert(!memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + blockInfoManager.lockForWriting("b3") + assert(memoryStore.remove("b3")) + blockInfoManager.removeBlock("b3") + putIteratorAsValues("b3", smallIterator, ClassTag.Any) + + // Unroll huge block with not enough space. This should fail and kick out b2 in the process. + val result4 = putIteratorAsValues("b4", bigIterator, ClassTag.Any) + assert(result4.isLeft) // unroll was unsuccessful + assert(!memoryStore.contains("b1")) + assert(!memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(!memoryStore.contains("b4")) + assert(memoryStore.currentUnrollMemoryForThisTask > 0) // we returned an iterator + result4.left.get.close() + assert(memoryStore.currentUnrollMemoryForThisTask === 0) // close released the unroll memory + } + + test("safely unroll blocks through putIteratorAsBytes") { + val (memoryStore, blockInfoManager) = makeMemoryStore(12000) + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator: Iterator[Any] = bigList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + + def putIteratorAsBytes[T]( + blockId: BlockId, + iter: Iterator[T], + classTag: ClassTag[T]): Either[PartiallySerializedBlock[T], Long] = { + assert(blockInfoManager.lockNewBlockForWriting( + blockId, + new BlockInfo(StorageLevel.MEMORY_ONLY_SER, classTag, tellMaster = false))) + val res = memoryStore.putIteratorAsBytes(blockId, iter, classTag, MemoryMode.ON_HEAP) + blockInfoManager.unlock(blockId) + res + } + + // Unroll with plenty of space. This should succeed and cache both blocks. + val result1 = putIteratorAsBytes("b1", smallIterator, ClassTag.Any) + val result2 = putIteratorAsBytes("b2", smallIterator, ClassTag.Any) + assert(memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(result1.isRight) // unroll was successful + assert(result2.isRight) + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + + // Re-put these two blocks so block manager knows about them too. Otherwise, block manager + // would not know how to drop them from memory later. + blockInfoManager.lockForWriting("b1") + memoryStore.remove("b1") + blockInfoManager.removeBlock("b1") + blockInfoManager.lockForWriting("b2") + memoryStore.remove("b2") + blockInfoManager.removeBlock("b2") + putIteratorAsBytes("b1", smallIterator, ClassTag.Any) + putIteratorAsBytes("b2", smallIterator, ClassTag.Any) + + // Unroll with not enough space. This should succeed but kick out b1 in the process. + val result3 = putIteratorAsBytes("b3", smallIterator, ClassTag.Any) + assert(result3.isRight) + assert(!memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + blockInfoManager.lockForWriting("b3") + assert(memoryStore.remove("b3")) + blockInfoManager.removeBlock("b3") + putIteratorAsBytes("b3", smallIterator, ClassTag.Any) + + // Unroll huge block with not enough space. This should fail and kick out b2 in the process. + val result4 = putIteratorAsBytes("b4", bigIterator, ClassTag.Any) + assert(result4.isLeft) // unroll was unsuccessful + assert(!memoryStore.contains("b1")) + assert(!memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(!memoryStore.contains("b4")) + assert(memoryStore.currentUnrollMemoryForThisTask > 0) // we returned an iterator + result4.left.get.discard() + assert(memoryStore.currentUnrollMemoryForThisTask === 0) // discard released the unroll memory + } + + test("PartiallySerializedBlock.valuesIterator") { + val (memoryStore, blockInfoManager) = makeMemoryStore(12000) + val bigList = List.fill(40)(new Array[Byte](1000)) + def bigIterator: Iterator[Any] = bigList.iterator.asInstanceOf[Iterator[Any]] + + // Unroll huge block with not enough space. This should fail. + assert(blockInfoManager.lockNewBlockForWriting( + "b1", + new BlockInfo(StorageLevel.MEMORY_ONLY_SER, ClassTag.Any, tellMaster = false))) + val res = memoryStore.putIteratorAsBytes("b1", bigIterator, ClassTag.Any, MemoryMode.ON_HEAP) + blockInfoManager.unlock("b1") + assert(res.isLeft) + assert(memoryStore.currentUnrollMemoryForThisTask > 0) + val valuesReturnedFromFailedPut = res.left.get.valuesIterator.toSeq // force materialization + assertSameContents( + bigList, valuesReturnedFromFailedPut, "PartiallySerializedBlock.valuesIterator()") + // The unroll memory was freed once the iterator was fully traversed. + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + } + + test("PartiallySerializedBlock.finishWritingToStream") { + val (memoryStore, blockInfoManager) = makeMemoryStore(12000) + val bigList = List.fill(40)(new Array[Byte](1000)) + def bigIterator: Iterator[Any] = bigList.iterator.asInstanceOf[Iterator[Any]] + + // Unroll huge block with not enough space. This should fail. + assert(blockInfoManager.lockNewBlockForWriting( + "b1", + new BlockInfo(StorageLevel.MEMORY_ONLY_SER, ClassTag.Any, tellMaster = false))) + val res = memoryStore.putIteratorAsBytes("b1", bigIterator, ClassTag.Any, MemoryMode.ON_HEAP) + blockInfoManager.unlock("b1") + assert(res.isLeft) + assert(memoryStore.currentUnrollMemoryForThisTask > 0) + val bos = new ByteBufferOutputStream() + res.left.get.finishWritingToStream(bos) + // The unroll memory was freed once the block was fully written. + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + val deserializedValues = serializerManager.dataDeserializeStream[Any]( + "b1", new ByteBufferInputStream(bos.toByteBuffer))(ClassTag.Any).toSeq + assertSameContents( + bigList, deserializedValues, "PartiallySerializedBlock.finishWritingToStream()") + } + + test("multiple unrolls by the same thread") { + val (memoryStore, _) = makeMemoryStore(12000) + val smallList = List.fill(40)(new Array[Byte](100)) + def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + + def putIteratorAsValues( + blockId: BlockId, + iter: Iterator[Any]): Either[PartiallyUnrolledIterator[Any], Long] = { + memoryStore.putIteratorAsValues(blockId, iter, ClassTag.Any) + } + + // All unroll memory used is released because putIterator did not return an iterator + assert(putIteratorAsValues("b1", smallIterator).isRight) + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + assert(putIteratorAsValues("b2", smallIterator).isRight) + assert(memoryStore.currentUnrollMemoryForThisTask === 0) + + // Unroll memory is not released because putIterator returned an iterator + // that still depends on the underlying vector used in the process + assert(putIteratorAsValues("b3", smallIterator).isLeft) + val unrollMemoryAfterB3 = memoryStore.currentUnrollMemoryForThisTask + assert(unrollMemoryAfterB3 > 0) + + // The unroll memory owned by this thread builds on top of its value after the previous unrolls + assert(putIteratorAsValues("b4", smallIterator).isLeft) + val unrollMemoryAfterB4 = memoryStore.currentUnrollMemoryForThisTask + assert(unrollMemoryAfterB4 > unrollMemoryAfterB3) + + // ... but only to a certain extent (until we run out of free space to grant new unroll memory) + assert(putIteratorAsValues("b5", smallIterator).isLeft) + val unrollMemoryAfterB5 = memoryStore.currentUnrollMemoryForThisTask + assert(putIteratorAsValues("b6", smallIterator).isLeft) + val unrollMemoryAfterB6 = memoryStore.currentUnrollMemoryForThisTask + assert(putIteratorAsValues("b7", smallIterator).isLeft) + val unrollMemoryAfterB7 = memoryStore.currentUnrollMemoryForThisTask + assert(unrollMemoryAfterB5 === unrollMemoryAfterB4) + assert(unrollMemoryAfterB6 === unrollMemoryAfterB4) + assert(unrollMemoryAfterB7 === unrollMemoryAfterB4) + } + + test("lazily create a big ByteBuffer to avoid OOM if it cannot be put into MemoryStore") { + val (memoryStore, blockInfoManager) = makeMemoryStore(12000) + val blockId = BlockId("rdd_3_10") + blockInfoManager.lockNewBlockForWriting( + blockId, new BlockInfo(StorageLevel.MEMORY_ONLY, ClassTag.Any, tellMaster = false)) + memoryStore.putBytes(blockId, 13000, MemoryMode.ON_HEAP, () => { + fail("A big ByteBuffer that cannot be put into MemoryStore should not be created") + }) + } + + test("put a small ByteBuffer to MemoryStore") { + val (memoryStore, _) = makeMemoryStore(12000) + val blockId = BlockId("rdd_3_10") + var bytes: ChunkedByteBuffer = null + memoryStore.putBytes(blockId, 10000, MemoryMode.ON_HEAP, () => { + bytes = new ChunkedByteBuffer(ByteBuffer.allocate(10000)) + bytes + }) + assert(memoryStore.getSize(blockId) === 10000) + } + + test("SPARK-22083: Release all locks in evictBlocksToFreeSpace") { + // Setup a memory store with many blocks cached, and then one request which leads to multiple + // blocks getting evicted. We'll make the eviction throw an exception, and make sure that + // all locks are released. + val ct = implicitly[ClassTag[Array[Byte]]] + val numInitialBlocks = 10 + val memStoreSize = 100 + val bytesPerSmallBlock = memStoreSize / numInitialBlocks + def testFailureOnNthDrop(numValidBlocks: Int, readLockAfterDrop: Boolean): Unit = { + val tc = TaskContext.empty() + val memManager = new StaticMemoryManager(conf, Long.MaxValue, memStoreSize, numCores = 1) + val blockInfoManager = new BlockInfoManager + blockInfoManager.registerTask(tc.taskAttemptId) + var droppedSoFar = 0 + val blockEvictionHandler = new BlockEvictionHandler { + var memoryStore: MemoryStore = _ + + override private[storage] def dropFromMemory[T: ClassTag]( + blockId: BlockId, + data: () => Either[Array[T], ChunkedByteBuffer]): StorageLevel = { + if (droppedSoFar < numValidBlocks) { + droppedSoFar += 1 + memoryStore.remove(blockId) + if (readLockAfterDrop) { + // for testing purposes, we act like another thread gets the read lock on the new + // block + StorageLevel.DISK_ONLY + } else { + StorageLevel.NONE + } + } else { + throw new RuntimeException(s"Mock error dropping block $droppedSoFar") + } + } + } + val memoryStore = new MemoryStore(conf, blockInfoManager, serializerManager, memManager, + blockEvictionHandler) { + override def afterDropAction(blockId: BlockId): Unit = { + if (readLockAfterDrop) { + // pretend that we get a read lock on the block (now on disk) in another thread + TaskContext.setTaskContext(tc) + blockInfoManager.lockForReading(blockId) + TaskContext.unset() + } + } + } + + blockEvictionHandler.memoryStore = memoryStore + memManager.setMemoryStore(memoryStore) + + // Put in some small blocks to fill up the memory store + val initialBlocks = (1 to numInitialBlocks).map { id => + val blockId = BlockId(s"rdd_1_$id") + val blockInfo = new BlockInfo(StorageLevel.MEMORY_ONLY, ct, tellMaster = false) + val initialWriteLock = blockInfoManager.lockNewBlockForWriting(blockId, blockInfo) + assert(initialWriteLock) + val success = memoryStore.putBytes(blockId, bytesPerSmallBlock, MemoryMode.ON_HEAP, () => { + new ChunkedByteBuffer(ByteBuffer.allocate(bytesPerSmallBlock)) + }) + assert(success) + blockInfoManager.unlock(blockId, None) + } + assert(blockInfoManager.size === numInitialBlocks) + + + // Add one big block, which will require evicting everything in the memorystore. However our + // mock BlockEvictionHandler will throw an exception -- make sure all locks are cleared. + val largeBlockId = BlockId(s"rdd_2_1") + val largeBlockInfo = new BlockInfo(StorageLevel.MEMORY_ONLY, ct, tellMaster = false) + val initialWriteLock = blockInfoManager.lockNewBlockForWriting(largeBlockId, largeBlockInfo) + assert(initialWriteLock) + if (numValidBlocks < numInitialBlocks) { + val exc = intercept[RuntimeException] { + memoryStore.putBytes(largeBlockId, memStoreSize, MemoryMode.ON_HEAP, () => { + new ChunkedByteBuffer(ByteBuffer.allocate(memStoreSize)) + }) + } + assert(exc.getMessage().startsWith("Mock error dropping block"), exc) + // BlockManager.doPut takes care of releasing the lock for the newly written block -- not + // testing that here, so do it manually + blockInfoManager.removeBlock(largeBlockId) + } else { + memoryStore.putBytes(largeBlockId, memStoreSize, MemoryMode.ON_HEAP, () => { + new ChunkedByteBuffer(ByteBuffer.allocate(memStoreSize)) + }) + // BlockManager.doPut takes care of releasing the lock for the newly written block -- not + // testing that here, so do it manually + blockInfoManager.unlock(largeBlockId) + } + + val largeBlockInMemory = if (numValidBlocks == numInitialBlocks) 1 else 0 + val expBlocks = numInitialBlocks + + (if (readLockAfterDrop) 0 else -numValidBlocks) + + largeBlockInMemory + assert(blockInfoManager.size === expBlocks) + + val blocksStillInMemory = blockInfoManager.entries.filter { case (id, info) => + assert(info.writerTask === BlockInfo.NO_WRITER, id) + // in this test, all the blocks in memory have no reader, but everything dropped to disk + // had another thread read the block. We shouldn't lose the other thread's reader lock. + if (memoryStore.contains(id)) { + assert(info.readerCount === 0, id) + true + } else { + assert(info.readerCount === 1, id) + false + } + } + assert(blocksStillInMemory.size === + (numInitialBlocks - numValidBlocks + largeBlockInMemory)) + } + + Seq(0, 3, numInitialBlocks).foreach { failAfterDropping => + Seq(true, false).foreach { readLockAfterDropping => + testFailureOnNthDrop(failAfterDropping, readLockAfterDropping) + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala b/core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala new file mode 100644 index 000000000000..535105379963 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.nio.ByteBuffer + +import scala.reflect.ClassTag + +import org.mockito.Mockito +import org.mockito.Mockito.atLeastOnce +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} + +import org.apache.spark.{SparkConf, SparkFunSuite, TaskContext, TaskContextImpl} +import org.apache.spark.memory.MemoryMode +import org.apache.spark.serializer.{JavaSerializer, SerializationStream, SerializerManager} +import org.apache.spark.storage.memory.{MemoryStore, PartiallySerializedBlock, RedirectableOutputStream} +import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream} +import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} + +class PartiallySerializedBlockSuite + extends SparkFunSuite + with BeforeAndAfterEach + with PrivateMethodTester { + + private val blockId = new TestBlockId("test") + private val conf = new SparkConf() + private val memoryStore = Mockito.mock(classOf[MemoryStore], Mockito.RETURNS_SMART_NULLS) + private val serializerManager = new SerializerManager(new JavaSerializer(conf), conf) + + private val getSerializationStream = PrivateMethod[SerializationStream]('serializationStream) + private val getRedirectableOutputStream = + PrivateMethod[RedirectableOutputStream]('redirectableOutputStream) + + override protected def beforeEach(): Unit = { + super.beforeEach() + Mockito.reset(memoryStore) + } + + private def partiallyUnroll[T: ClassTag]( + iter: Iterator[T], + numItemsToBuffer: Int): PartiallySerializedBlock[T] = { + + val bbos: ChunkedByteBufferOutputStream = { + val spy = Mockito.spy(new ChunkedByteBufferOutputStream(128, ByteBuffer.allocate)) + Mockito.doAnswer(new Answer[ChunkedByteBuffer] { + override def answer(invocationOnMock: InvocationOnMock): ChunkedByteBuffer = { + Mockito.spy(invocationOnMock.callRealMethod().asInstanceOf[ChunkedByteBuffer]) + } + }).when(spy).toChunkedByteBuffer + spy + } + + val serializer = serializerManager + .getSerializer(implicitly[ClassTag[T]], autoPick = true).newInstance() + val redirectableOutputStream = Mockito.spy(new RedirectableOutputStream) + redirectableOutputStream.setOutputStream(bbos) + val serializationStream = Mockito.spy(serializer.serializeStream(redirectableOutputStream)) + + (1 to numItemsToBuffer).foreach { _ => + assert(iter.hasNext) + serializationStream.writeObject[T](iter.next()) + } + + val unrollMemory = bbos.size + new PartiallySerializedBlock[T]( + memoryStore, + serializerManager, + blockId, + serializationStream = serializationStream, + redirectableOutputStream, + unrollMemory = unrollMemory, + memoryMode = MemoryMode.ON_HEAP, + bbos, + rest = iter, + classTag = implicitly[ClassTag[T]]) + } + + test("valuesIterator() and finishWritingToStream() cannot be called after discard() is called") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.discard() + intercept[IllegalStateException] { + partiallySerializedBlock.finishWritingToStream(null) + } + intercept[IllegalStateException] { + partiallySerializedBlock.valuesIterator + } + } + + test("discard() can be called more than once") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.discard() + partiallySerializedBlock.discard() + } + + test("cannot call valuesIterator() more than once") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.valuesIterator + intercept[IllegalStateException] { + partiallySerializedBlock.valuesIterator + } + } + + test("cannot call finishWritingToStream() more than once") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.finishWritingToStream(new ByteBufferOutputStream()) + intercept[IllegalStateException] { + partiallySerializedBlock.finishWritingToStream(new ByteBufferOutputStream()) + } + } + + test("cannot call finishWritingToStream() after valuesIterator()") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.valuesIterator + intercept[IllegalStateException] { + partiallySerializedBlock.finishWritingToStream(new ByteBufferOutputStream()) + } + } + + test("cannot call valuesIterator() after finishWritingToStream()") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.finishWritingToStream(new ByteBufferOutputStream()) + intercept[IllegalStateException] { + partiallySerializedBlock.valuesIterator + } + } + + test("buffers are deallocated in a TaskCompletionListener") { + try { + TaskContext.setTaskContext(TaskContext.empty()) + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + TaskContext.get().asInstanceOf[TaskContextImpl].markTaskCompleted(None) + Mockito.verify(partiallySerializedBlock.getUnrolledChunkedByteBuffer).dispose() + Mockito.verifyNoMoreInteractions(memoryStore) + } finally { + TaskContext.unset() + } + } + + private def testUnroll[T: ClassTag]( + testCaseName: String, + items: Seq[T], + numItemsToBuffer: Int): Unit = { + + test(s"$testCaseName with discard() and numBuffered = $numItemsToBuffer") { + val partiallySerializedBlock = partiallyUnroll(items.iterator, numItemsToBuffer) + partiallySerializedBlock.discard() + + Mockito.verify(memoryStore).releaseUnrollMemoryForThisTask( + MemoryMode.ON_HEAP, partiallySerializedBlock.unrollMemory) + Mockito.verify(partiallySerializedBlock.invokePrivate(getSerializationStream())).close() + Mockito.verify(partiallySerializedBlock.invokePrivate(getRedirectableOutputStream())).close() + Mockito.verifyNoMoreInteractions(memoryStore) + Mockito.verify(partiallySerializedBlock.getUnrolledChunkedByteBuffer, atLeastOnce).dispose() + } + + test(s"$testCaseName with finishWritingToStream() and numBuffered = $numItemsToBuffer") { + val partiallySerializedBlock = partiallyUnroll(items.iterator, numItemsToBuffer) + val bbos = Mockito.spy(new ByteBufferOutputStream()) + partiallySerializedBlock.finishWritingToStream(bbos) + + Mockito.verify(memoryStore).releaseUnrollMemoryForThisTask( + MemoryMode.ON_HEAP, partiallySerializedBlock.unrollMemory) + Mockito.verify(partiallySerializedBlock.invokePrivate(getSerializationStream())).close() + Mockito.verify(partiallySerializedBlock.invokePrivate(getRedirectableOutputStream())).close() + Mockito.verify(bbos).close() + Mockito.verifyNoMoreInteractions(memoryStore) + Mockito.verify(partiallySerializedBlock.getUnrolledChunkedByteBuffer, atLeastOnce).dispose() + + val serializer = serializerManager + .getSerializer(implicitly[ClassTag[T]], autoPick = true).newInstance() + val deserialized = + serializer.deserializeStream(new ByteBufferInputStream(bbos.toByteBuffer)).asIterator.toSeq + assert(deserialized === items) + } + + test(s"$testCaseName with valuesIterator() and numBuffered = $numItemsToBuffer") { + val partiallySerializedBlock = partiallyUnroll(items.iterator, numItemsToBuffer) + val valuesIterator = partiallySerializedBlock.valuesIterator + Mockito.verify(partiallySerializedBlock.invokePrivate(getSerializationStream())).close() + Mockito.verify(partiallySerializedBlock.invokePrivate(getRedirectableOutputStream())).close() + + val deserializedItems = valuesIterator.toArray.toSeq + Mockito.verify(memoryStore).releaseUnrollMemoryForThisTask( + MemoryMode.ON_HEAP, partiallySerializedBlock.unrollMemory) + Mockito.verifyNoMoreInteractions(memoryStore) + Mockito.verify(partiallySerializedBlock.getUnrolledChunkedByteBuffer, atLeastOnce).dispose() + assert(deserializedItems === items) + } + } + + testUnroll("basic numbers", 1 to 1000, numItemsToBuffer = 50) + testUnroll("basic numbers", 1 to 1000, numItemsToBuffer = 0) + testUnroll("basic numbers", 1 to 1000, numItemsToBuffer = 1000) + testUnroll("case classes", (1 to 1000).map(x => MyCaseClass(x.toString)), numItemsToBuffer = 50) + testUnroll("case classes", (1 to 1000).map(x => MyCaseClass(x.toString)), numItemsToBuffer = 0) + testUnroll("case classes", (1 to 1000).map(x => MyCaseClass(x.toString)), numItemsToBuffer = 1000) + testUnroll("empty iterator", Seq.empty[String], numItemsToBuffer = 0) +} + +private case class MyCaseClass(str: String) diff --git a/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala new file mode 100644 index 000000000000..cbc903f17ad7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import org.mockito.Matchers +import org.mockito.Mockito._ +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark.SparkFunSuite +import org.apache.spark.memory.MemoryMode.ON_HEAP +import org.apache.spark.storage.memory.{MemoryStore, PartiallyUnrolledIterator} + +class PartiallyUnrolledIteratorSuite extends SparkFunSuite with MockitoSugar { + test("join two iterators") { + val unrollSize = 1000 + val unroll = (0 until unrollSize).iterator + val restSize = 500 + val rest = (unrollSize until restSize + unrollSize).iterator + + val memoryStore = mock[MemoryStore] + val joinIterator = new PartiallyUnrolledIterator(memoryStore, ON_HEAP, unrollSize, unroll, rest) + + // Firstly iterate over unrolling memory iterator + (0 until unrollSize).foreach { value => + assert(joinIterator.hasNext) + assert(joinIterator.hasNext) + assert(joinIterator.next() == value) + } + + joinIterator.hasNext + joinIterator.hasNext + verify(memoryStore, times(1)) + .releaseUnrollMemoryForThisTask(Matchers.eq(ON_HEAP), Matchers.eq(unrollSize.toLong)) + + // Secondly, iterate over rest iterator + (unrollSize until unrollSize + restSize).foreach { value => + assert(joinIterator.hasNext) + assert(joinIterator.hasNext) + assert(joinIterator.next() == value) + } + + joinIterator.close() + // MemoryMode.releaseUnrollMemoryForThisTask is called only once + verifyNoMoreInteractions(memoryStore) + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index 828153bdbfc4..c371cbcf8dff 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -17,11 +17,12 @@ package org.apache.spark.storage -import java.io.InputStream +import java.io.{File, InputStream, IOException} +import java.util.UUID import java.util.concurrent.Semaphore import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent.future +import scala.concurrent.Future import org.mockito.Matchers.{any, eq => meq} import org.mockito.Mockito._ @@ -31,9 +32,11 @@ import org.scalatest.PrivateMethodTester import org.apache.spark.{SparkFunSuite, TaskContext} import org.apache.spark.network._ -import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.network.shuffle.BlockFetchingListener +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.shuffle.{BlockFetchingListener, TempShuffleFileManager} +import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.util.Utils class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodTester { @@ -43,7 +46,8 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT /** Creates a mock [[BlockTransferService]] that returns data from the given map. */ private def createMockTransfer(data: Map[BlockId, ManagedBuffer]): BlockTransferService = { val transfer = mock(classOf[BlockTransferService]) - when(transfer.fetchBlocks(any(), any(), any(), any(), any())).thenAnswer(new Answer[Unit] { + when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) + .thenAnswer(new Answer[Unit] { override def answer(invocation: InvocationOnMock): Unit = { val blocks = invocation.getArguments()(3).asInstanceOf[Array[String]] val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] @@ -63,7 +67,10 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT // Create a mock managed buffer for testing def createMockManagedBuffer(): ManagedBuffer = { val mockManagedBuffer = mock(classOf[ManagedBuffer]) - when(mockManagedBuffer.createInputStream()).thenReturn(mock(classOf[InputStream])) + val in = mock(classOf[InputStream]) + when(in.read(any())).thenReturn(1) + when(in.read(any(), any(), any())).thenReturn(1) + when(mockManagedBuffer.createInputStream()).thenReturn(in) mockManagedBuffer } @@ -99,7 +106,12 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT transfer, blockManager, blocksByAddress, - 48 * 1024 * 1024) + (_, in) => in, + 48 * 1024 * 1024, + Int.MaxValue, + Int.MaxValue, + Int.MaxValue, + true) // 3 local blocks fetched in initialization verify(blockManager, times(3)).getBlockData(any()) @@ -127,7 +139,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT // 3 local blocks, and 2 remote blocks // (but from the same block manager so one call to fetchBlocks) verify(blockManager, times(3)).getBlockData(any()) - verify(transfer, times(1)).fetchBlocks(any(), any(), any(), any(), any()) + verify(transfer, times(1)).fetchBlocks(any(), any(), any(), any(), any(), any()) } test("release current unexhausted buffer in case the task completes early") { @@ -146,10 +158,11 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val sem = new Semaphore(0) val transfer = mock(classOf[BlockTransferService]) - when(transfer.fetchBlocks(any(), any(), any(), any(), any())).thenAnswer(new Answer[Unit] { + when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) + .thenAnswer(new Answer[Unit] { override def answer(invocation: InvocationOnMock): Unit = { val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] - future { + Future { // Return the first two blocks, and wait till task completion before returning the 3rd one listener.onBlockFetchSuccess( ShuffleBlockId(0, 0, 0).toString, blocks(ShuffleBlockId(0, 0, 0))) @@ -171,7 +184,12 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT transfer, blockManager, blocksByAddress, - 48 * 1024 * 1024) + (_, in) => in, + 48 * 1024 * 1024, + Int.MaxValue, + Int.MaxValue, + Int.MaxValue, + true) verify(blocks(ShuffleBlockId(0, 0, 0)), times(0)).release() iterator.next()._2.close() // close() first block's input stream @@ -182,7 +200,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT // Complete the task; then the 2nd block buffer should be exhausted verify(blocks(ShuffleBlockId(0, 1, 0)), times(0)).release() - taskContext.markTaskCompleted() + taskContext.markTaskCompleted(None) verify(blocks(ShuffleBlockId(0, 1, 0)), times(1)).release() // The 3rd block should not be retained because the iterator is already in zombie state @@ -199,19 +217,20 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT // Make sure remote blocks would return val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) val blocks = Map[BlockId, ManagedBuffer]( - ShuffleBlockId(0, 0, 0) -> mock(classOf[ManagedBuffer]), - ShuffleBlockId(0, 1, 0) -> mock(classOf[ManagedBuffer]), - ShuffleBlockId(0, 2, 0) -> mock(classOf[ManagedBuffer]) + ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(), + ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(), + ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer() ) // Semaphore to coordinate event sequence in two different threads. val sem = new Semaphore(0) val transfer = mock(classOf[BlockTransferService]) - when(transfer.fetchBlocks(any(), any(), any(), any(), any())).thenAnswer(new Answer[Unit] { + when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) + .thenAnswer(new Answer[Unit] { override def answer(invocation: InvocationOnMock): Unit = { val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] - future { + Future { // Return the first block, and then fail. listener.onBlockFetchSuccess( ShuffleBlockId(0, 0, 0).toString, blocks(ShuffleBlockId(0, 0, 0))) @@ -233,7 +252,12 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT transfer, blockManager, blocksByAddress, - 48 * 1024 * 1024) + (_, in) => in, + 48 * 1024 * 1024, + Int.MaxValue, + Int.MaxValue, + Int.MaxValue, + true) // Continue only after the mock calls onBlockFetchFailure sem.acquire() @@ -244,4 +268,217 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT intercept[FetchFailedException] { iterator.next() } intercept[FetchFailedException] { iterator.next() } } + + test("retry corrupt blocks") { + val blockManager = mock(classOf[BlockManager]) + val localBmId = BlockManagerId("test-client", "test-client", 1) + doReturn(localBmId).when(blockManager).blockManagerId + + // Make sure remote blocks would return + val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) + val blocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(), + ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(), + ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer() + ) + + // Semaphore to coordinate event sequence in two different threads. + val sem = new Semaphore(0) + + val corruptStream = mock(classOf[InputStream]) + when(corruptStream.read(any(), any(), any())).thenThrow(new IOException("corrupt")) + val corruptBuffer = mock(classOf[ManagedBuffer]) + when(corruptBuffer.createInputStream()).thenReturn(corruptStream) + val corruptLocalBuffer = new FileSegmentManagedBuffer(null, new File("a"), 0, 100) + + val transfer = mock(classOf[BlockTransferService]) + when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) + .thenAnswer(new Answer[Unit] { + override def answer(invocation: InvocationOnMock): Unit = { + val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + Future { + // Return the first block, and then fail. + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 0, 0).toString, blocks(ShuffleBlockId(0, 0, 0))) + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 1, 0).toString, corruptBuffer) + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 2, 0).toString, corruptLocalBuffer) + sem.release() + } + } + }) + + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)) + + val taskContext = TaskContext.empty() + val iterator = new ShuffleBlockFetcherIterator( + taskContext, + transfer, + blockManager, + blocksByAddress, + (_, in) => new LimitedInputStream(in, 100), + 48 * 1024 * 1024, + Int.MaxValue, + Int.MaxValue, + Int.MaxValue, + true) + + // Continue only after the mock calls onBlockFetchFailure + sem.acquire() + + // The first block should be returned without an exception + val (id1, _) = iterator.next() + assert(id1 === ShuffleBlockId(0, 0, 0)) + + when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) + .thenAnswer(new Answer[Unit] { + override def answer(invocation: InvocationOnMock): Unit = { + val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + Future { + // Return the first block, and then fail. + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 1, 0).toString, corruptBuffer) + sem.release() + } + } + }) + + // The next block is corrupt local block (the second one is corrupt and retried) + intercept[FetchFailedException] { iterator.next() } + + sem.acquire() + intercept[FetchFailedException] { iterator.next() } + } + + test("retry corrupt blocks (disabled)") { + val blockManager = mock(classOf[BlockManager]) + val localBmId = BlockManagerId("test-client", "test-client", 1) + doReturn(localBmId).when(blockManager).blockManagerId + + // Make sure remote blocks would return + val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) + val blocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer(), + ShuffleBlockId(0, 1, 0) -> createMockManagedBuffer(), + ShuffleBlockId(0, 2, 0) -> createMockManagedBuffer() + ) + + // Semaphore to coordinate event sequence in two different threads. + val sem = new Semaphore(0) + + val corruptStream = mock(classOf[InputStream]) + when(corruptStream.read(any(), any(), any())).thenThrow(new IOException("corrupt")) + val corruptBuffer = mock(classOf[ManagedBuffer]) + when(corruptBuffer.createInputStream()).thenReturn(corruptStream) + + val transfer = mock(classOf[BlockTransferService]) + when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) + .thenAnswer(new Answer[Unit] { + override def answer(invocation: InvocationOnMock): Unit = { + val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + Future { + // Return the first block, and then fail. + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 0, 0).toString, blocks(ShuffleBlockId(0, 0, 0))) + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 1, 0).toString, corruptBuffer) + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 2, 0).toString, corruptBuffer) + sem.release() + } + } + }) + + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)) + + val taskContext = TaskContext.empty() + val iterator = new ShuffleBlockFetcherIterator( + taskContext, + transfer, + blockManager, + blocksByAddress, + (_, in) => new LimitedInputStream(in, 100), + 48 * 1024 * 1024, + Int.MaxValue, + Int.MaxValue, + Int.MaxValue, + false) + + // Continue only after the mock calls onBlockFetchFailure + sem.acquire() + + // The first block should be returned without an exception + val (id1, _) = iterator.next() + assert(id1 === ShuffleBlockId(0, 0, 0)) + val (id2, _) = iterator.next() + assert(id2 === ShuffleBlockId(0, 1, 0)) + val (id3, _) = iterator.next() + assert(id3 === ShuffleBlockId(0, 2, 0)) + } + + test("Blocks should be shuffled to disk when size of the request is above the" + + " threshold(maxReqSizeShuffleToMem).") { + val blockManager = mock(classOf[BlockManager]) + val localBmId = BlockManagerId("test-client", "test-client", 1) + doReturn(localBmId).when(blockManager).blockManagerId + + val diskBlockManager = mock(classOf[DiskBlockManager]) + val tmpDir = Utils.createTempDir() + doReturn{ + val blockId = TempLocalBlockId(UUID.randomUUID()) + (blockId, new File(tmpDir, blockId.name)) + }.when(diskBlockManager).createTempLocalBlock() + doReturn(diskBlockManager).when(blockManager).diskBlockManager + + val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) + val remoteBlocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer()) + val transfer = mock(classOf[BlockTransferService]) + var tempShuffleFileManager: TempShuffleFileManager = null + when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) + .thenAnswer(new Answer[Unit] { + override def answer(invocation: InvocationOnMock): Unit = { + val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + tempShuffleFileManager = invocation.getArguments()(5).asInstanceOf[TempShuffleFileManager] + Future { + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0))) + } + } + }) + + def fetchShuffleBlock(blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])]): Unit = { + // Set `maxBytesInFlight` and `maxReqsInFlight` to `Int.MaxValue`, so that during the + // construction of `ShuffleBlockFetcherIterator`, all requests to fetch remote shuffle blocks + // are issued. The `maxReqSizeShuffleToMem` is hard-coded as 200 here. + new ShuffleBlockFetcherIterator( + TaskContext.empty(), + transfer, + blockManager, + blocksByAddress, + (_, in) => in, + maxBytesInFlight = Int.MaxValue, + maxReqsInFlight = Int.MaxValue, + maxBlocksInFlightPerAddress = Int.MaxValue, + maxReqSizeShuffleToMem = 200, + detectCorrupt = true) + } + + val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 100L)).toSeq)) + fetchShuffleBlock(blocksByAddress1) + // `maxReqSizeShuffleToMem` is 200, which is greater than the block size 100, so don't fetch + // shuffle block to disk. + assert(tempShuffleFileManager == null) + + val blocksByAddress2 = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 300L)).toSeq)) + fetchShuffleBlock(blocksByAddress2) + // `maxReqSizeShuffleToMem` is 200, which is smaller than the block size 300, so fetch + // shuffle block to disk. + assert(tempShuffleFileManager != null) + } } diff --git a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala index 1a199beb3558..9835f11a2f7e 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import org.apache.spark.{SparkFunSuite, Success} +import org.apache.spark.{SparkConf, SparkFunSuite, Success} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ @@ -29,9 +29,11 @@ class StorageStatusListenerSuite extends SparkFunSuite { private val bm2 = BlockManagerId("fat", "duck", 2) private val taskInfo1 = new TaskInfo(0, 0, 0, 0, "big", "dog", TaskLocality.ANY, false) private val taskInfo2 = new TaskInfo(0, 0, 0, 0, "fat", "duck", TaskLocality.ANY, false) + private val conf = new SparkConf() test("block manager added/removed") { - val listener = new StorageStatusListener + conf.set("spark.ui.retainedDeadExecutors", "1") + val listener = new StorageStatusListener(conf) // Block manager add assert(listener.executorIdToStorageStatus.size === 0) @@ -53,14 +55,18 @@ class StorageStatusListenerSuite extends SparkFunSuite { assert(listener.executorIdToStorageStatus.size === 1) assert(!listener.executorIdToStorageStatus.get("big").isDefined) assert(listener.executorIdToStorageStatus.get("fat").isDefined) + assert(listener.deadExecutorStorageStatus.size === 1) + assert(listener.deadExecutorStorageStatus(0).blockManagerId.executorId.equals("big")) listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(1L, bm2)) assert(listener.executorIdToStorageStatus.size === 0) assert(!listener.executorIdToStorageStatus.get("big").isDefined) assert(!listener.executorIdToStorageStatus.get("fat").isDefined) + assert(listener.deadExecutorStorageStatus.size === 1) + assert(listener.deadExecutorStorageStatus(0).blockManagerId.executorId.equals("fat")) } test("task end without updated blocks") { - val listener = new StorageStatusListener + val listener = new StorageStatusListener(conf) listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm2, 2000L)) val taskMetrics = new TaskMetrics @@ -76,48 +82,51 @@ class StorageStatusListenerSuite extends SparkFunSuite { assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) } - test("task end with updated blocks") { - val listener = new StorageStatusListener + test("updated blocks") { + val listener = new StorageStatusListener(conf) listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm2, 2000L)) - val taskMetrics1 = new TaskMetrics - val taskMetrics2 = new TaskMetrics - val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L, 0L)) - val block2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.DISK_ONLY, 0L, 200L, 0L)) - val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L, 0L)) - taskMetrics1.updatedBlocks = Some(Seq(block1, block2)) - taskMetrics2.updatedBlocks = Some(Seq(block3)) - - // Task end with new blocks + + val blockUpdateInfos1 = Seq( + BlockUpdatedInfo(bm1, RDDBlockId(1, 1), StorageLevel.DISK_ONLY, 0L, 100L), + BlockUpdatedInfo(bm1, RDDBlockId(1, 2), StorageLevel.DISK_ONLY, 0L, 200L) + ) + val blockUpdateInfos2 = + Seq(BlockUpdatedInfo(bm2, RDDBlockId(4, 0), StorageLevel.DISK_ONLY, 0L, 300L)) + + // Add some new blocks assert(listener.executorIdToStorageStatus("big").numBlocks === 0) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) + postUpdateBlock(listener, blockUpdateInfos1) assert(listener.executorIdToStorageStatus("big").numBlocks === 2) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo2, taskMetrics2)) + postUpdateBlock(listener, blockUpdateInfos2) assert(listener.executorIdToStorageStatus("big").numBlocks === 2) assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0))) - // Task end with dropped blocks - val droppedBlock1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) - val droppedBlock2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) - val droppedBlock3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) - taskMetrics1.updatedBlocks = Some(Seq(droppedBlock1, droppedBlock3)) - taskMetrics2.updatedBlocks = Some(Seq(droppedBlock2, droppedBlock3)) + // Dropped the blocks + val droppedBlockInfo1 = Seq( + BlockUpdatedInfo(bm1, RDDBlockId(1, 1), StorageLevel.NONE, 0L, 0L), + BlockUpdatedInfo(bm1, RDDBlockId(4, 0), StorageLevel.NONE, 0L, 0L) + ) + val droppedBlockInfo2 = Seq( + BlockUpdatedInfo(bm2, RDDBlockId(1, 2), StorageLevel.NONE, 0L, 0L), + BlockUpdatedInfo(bm2, RDDBlockId(4, 0), StorageLevel.NONE, 0L, 0L) + ) - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) + postUpdateBlock(listener, droppedBlockInfo1) assert(listener.executorIdToStorageStatus("big").numBlocks === 1) assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0))) - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo2, taskMetrics2)) + postUpdateBlock(listener, droppedBlockInfo2) assert(listener.executorIdToStorageStatus("big").numBlocks === 1) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) @@ -126,17 +135,16 @@ class StorageStatusListenerSuite extends SparkFunSuite { } test("unpersist RDD") { - val listener = new StorageStatusListener + val listener = new StorageStatusListener(conf) listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) - val taskMetrics1 = new TaskMetrics - val taskMetrics2 = new TaskMetrics - val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L, 0L)) - val block2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.DISK_ONLY, 0L, 200L, 0L)) - val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L, 0L)) - taskMetrics1.updatedBlocks = Some(Seq(block1, block2)) - taskMetrics2.updatedBlocks = Some(Seq(block3)) - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) - listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics2)) + val blockUpdateInfos1 = Seq( + BlockUpdatedInfo(bm1, RDDBlockId(1, 1), StorageLevel.DISK_ONLY, 0L, 100L), + BlockUpdatedInfo(bm1, RDDBlockId(1, 2), StorageLevel.DISK_ONLY, 0L, 200L) + ) + val blockUpdateInfos2 = + Seq(BlockUpdatedInfo(bm1, RDDBlockId(4, 0), StorageLevel.DISK_ONLY, 0L, 300L)) + postUpdateBlock(listener, blockUpdateInfos1) + postUpdateBlock(listener, blockUpdateInfos2) assert(listener.executorIdToStorageStatus("big").numBlocks === 3) // Unpersist RDD @@ -149,4 +157,11 @@ class StorageStatusListenerSuite extends SparkFunSuite { listener.onUnpersistRDD(SparkListenerUnpersistRDD(1)) assert(listener.executorIdToStorageStatus("big").numBlocks === 0) } + + private def postUpdateBlock( + listener: StorageStatusListener, updateBlockInfos: Seq[BlockUpdatedInfo]): Unit = { + updateBlockInfos.foreach { updateBlockInfo => + listener.onBlockUpdated(SparkListenerBlockUpdated(updateBlockInfo)) + } + } } diff --git a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala index 1d5a813a4d33..da198f946fd6 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -27,16 +27,15 @@ class StorageSuite extends SparkFunSuite { // For testing add, update, and remove (for non-RDD blocks) private def storageStatus1: StorageStatus = { - val status = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L) + val status = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L, Some(1000L), Some(0L)) assert(status.blocks.isEmpty) assert(status.rddBlocks.isEmpty) assert(status.memUsed === 0L) assert(status.memRemaining === 1000L) assert(status.diskUsed === 0L) - assert(status.offHeapUsed === 0L) - status.addBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 10L, 20L, 1L)) - status.addBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 10L, 20L, 1L)) - status.addBlock(TestBlockId("faa"), BlockStatus(memAndDisk, 10L, 20L, 1L)) + status.addBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 10L, 20L)) + status.addBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 10L, 20L)) + status.addBlock(TestBlockId("faa"), BlockStatus(memAndDisk, 10L, 20L)) status } @@ -50,18 +49,16 @@ class StorageSuite extends SparkFunSuite { assert(status.memUsed === 30L) assert(status.memRemaining === 970L) assert(status.diskUsed === 60L) - assert(status.offHeapUsed === 3L) } test("storage status update non-RDD blocks") { val status = storageStatus1 - status.updateBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 50L, 100L, 1L)) - status.updateBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 100L, 20L, 0L)) + status.updateBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 50L, 100L)) + status.updateBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 100L, 20L)) assert(status.blocks.size === 3) assert(status.memUsed === 160L) assert(status.memRemaining === 840L) assert(status.diskUsed === 140L) - assert(status.offHeapUsed === 2L) } test("storage status remove non-RDD blocks") { @@ -73,20 +70,19 @@ class StorageSuite extends SparkFunSuite { assert(status.memUsed === 10L) assert(status.memRemaining === 990L) assert(status.diskUsed === 20L) - assert(status.offHeapUsed === 1L) } // For testing add, update, remove, get, and contains etc. for both RDD and non-RDD blocks private def storageStatus2: StorageStatus = { - val status = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L) + val status = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L, Some(1000L), Some(0L)) assert(status.rddBlocks.isEmpty) - status.addBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 10L, 20L, 0L)) - status.addBlock(TestBlockId("man"), BlockStatus(memAndDisk, 10L, 20L, 0L)) - status.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 10L, 20L, 1L)) - status.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 100L, 200L, 1L)) - status.addBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 10L, 20L, 1L)) - status.addBlock(RDDBlockId(2, 3), BlockStatus(memAndDisk, 10L, 20L, 0L)) - status.addBlock(RDDBlockId(2, 4), BlockStatus(memAndDisk, 10L, 40L, 0L)) + status.addBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 10L, 20L)) + status.addBlock(TestBlockId("man"), BlockStatus(memAndDisk, 10L, 20L)) + status.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 10L, 20L)) + status.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 100L, 200L)) + status.addBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 10L, 20L)) + status.addBlock(RDDBlockId(2, 3), BlockStatus(memAndDisk, 10L, 20L)) + status.addBlock(RDDBlockId(2, 4), BlockStatus(memAndDisk, 10L, 40L)) status } @@ -113,9 +109,6 @@ class StorageSuite extends SparkFunSuite { assert(status.diskUsedByRdd(0) === 20L) assert(status.diskUsedByRdd(1) === 200L) assert(status.diskUsedByRdd(2) === 80L) - assert(status.offHeapUsedByRdd(0) === 1L) - assert(status.offHeapUsedByRdd(1) === 1L) - assert(status.offHeapUsedByRdd(2) === 1L) assert(status.rddStorageLevel(0) === Some(memAndDisk)) assert(status.rddStorageLevel(1) === Some(memAndDisk)) assert(status.rddStorageLevel(2) === Some(memAndDisk)) @@ -124,15 +117,14 @@ class StorageSuite extends SparkFunSuite { assert(status.rddBlocksById(10).isEmpty) assert(status.memUsedByRdd(10) === 0L) assert(status.diskUsedByRdd(10) === 0L) - assert(status.offHeapUsedByRdd(10) === 0L) assert(status.rddStorageLevel(10) === None) } test("storage status update RDD blocks") { val status = storageStatus2 - status.updateBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 5000L, 0L, 0L)) - status.updateBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)) - status.updateBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 0L, 1000L, 0L)) + status.updateBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 5000L, 0L)) + status.updateBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L)) + status.updateBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 0L, 1000L)) assert(status.blocks.size === 7) assert(status.rddBlocks.size === 5) assert(status.rddBlocksById(0).size === 1) @@ -144,9 +136,6 @@ class StorageSuite extends SparkFunSuite { assert(status.diskUsedByRdd(0) === 0L) assert(status.diskUsedByRdd(1) === 200L) assert(status.diskUsedByRdd(2) === 1060L) - assert(status.offHeapUsedByRdd(0) === 0L) - assert(status.offHeapUsedByRdd(1) === 1L) - assert(status.offHeapUsedByRdd(2) === 0L) } test("storage status remove RDD blocks") { @@ -170,9 +159,6 @@ class StorageSuite extends SparkFunSuite { assert(status.diskUsedByRdd(0) === 20L) assert(status.diskUsedByRdd(1) === 0L) assert(status.diskUsedByRdd(2) === 20L) - assert(status.offHeapUsedByRdd(0) === 1L) - assert(status.offHeapUsedByRdd(1) === 0L) - assert(status.offHeapUsedByRdd(2) === 0L) } test("storage status containsBlock") { @@ -209,17 +195,17 @@ class StorageSuite extends SparkFunSuite { val status = storageStatus2 assert(status.blocks.size === status.numBlocks) assert(status.rddBlocks.size === status.numRddBlocks) - status.addBlock(TestBlockId("Foo"), BlockStatus(memAndDisk, 0L, 0L, 100L)) - status.addBlock(RDDBlockId(4, 4), BlockStatus(memAndDisk, 0L, 0L, 100L)) - status.addBlock(RDDBlockId(4, 8), BlockStatus(memAndDisk, 0L, 0L, 100L)) + status.addBlock(TestBlockId("Foo"), BlockStatus(memAndDisk, 0L, 0L)) + status.addBlock(RDDBlockId(4, 4), BlockStatus(memAndDisk, 0L, 0L)) + status.addBlock(RDDBlockId(4, 8), BlockStatus(memAndDisk, 0L, 0L)) assert(status.blocks.size === status.numBlocks) assert(status.rddBlocks.size === status.numRddBlocks) assert(status.rddBlocksById(4).size === status.numRddBlocksById(4)) assert(status.rddBlocksById(10).size === status.numRddBlocksById(10)) - status.updateBlock(TestBlockId("Foo"), BlockStatus(memAndDisk, 0L, 10L, 400L)) - status.updateBlock(RDDBlockId(4, 0), BlockStatus(memAndDisk, 0L, 0L, 100L)) - status.updateBlock(RDDBlockId(4, 8), BlockStatus(memAndDisk, 0L, 0L, 100L)) - status.updateBlock(RDDBlockId(10, 10), BlockStatus(memAndDisk, 0L, 0L, 100L)) + status.updateBlock(TestBlockId("Foo"), BlockStatus(memAndDisk, 0L, 10L)) + status.updateBlock(RDDBlockId(4, 0), BlockStatus(memAndDisk, 0L, 0L)) + status.updateBlock(RDDBlockId(4, 8), BlockStatus(memAndDisk, 0L, 0L)) + status.updateBlock(RDDBlockId(10, 10), BlockStatus(memAndDisk, 0L, 0L)) assert(status.blocks.size === status.numBlocks) assert(status.rddBlocks.size === status.numRddBlocks) assert(status.rddBlocksById(4).size === status.numRddBlocksById(4)) @@ -244,44 +230,39 @@ class StorageSuite extends SparkFunSuite { val status = storageStatus2 def actualMemUsed: Long = status.blocks.values.map(_.memSize).sum def actualDiskUsed: Long = status.blocks.values.map(_.diskSize).sum - def actualOffHeapUsed: Long = status.blocks.values.map(_.externalBlockStoreSize).sum assert(status.memUsed === actualMemUsed) assert(status.diskUsed === actualDiskUsed) - assert(status.offHeapUsed === actualOffHeapUsed) - status.addBlock(TestBlockId("fire"), BlockStatus(memAndDisk, 4000L, 5000L, 6000L)) - status.addBlock(TestBlockId("wire"), BlockStatus(memAndDisk, 400L, 500L, 600L)) - status.addBlock(RDDBlockId(25, 25), BlockStatus(memAndDisk, 40L, 50L, 60L)) + status.addBlock(TestBlockId("fire"), BlockStatus(memAndDisk, 4000L, 5000L)) + status.addBlock(TestBlockId("wire"), BlockStatus(memAndDisk, 400L, 500L)) + status.addBlock(RDDBlockId(25, 25), BlockStatus(memAndDisk, 40L, 50L)) assert(status.memUsed === actualMemUsed) assert(status.diskUsed === actualDiskUsed) - assert(status.offHeapUsed === actualOffHeapUsed) - status.updateBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 4L, 5L, 6L)) - status.updateBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 4L, 5L, 6L)) - status.updateBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 4L, 5L, 6L)) + status.updateBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 4L, 5L)) + status.updateBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 4L, 5L)) + status.updateBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 4L, 5L)) assert(status.memUsed === actualMemUsed) assert(status.diskUsed === actualDiskUsed) - assert(status.offHeapUsed === actualOffHeapUsed) status.removeBlock(TestBlockId("fire")) status.removeBlock(TestBlockId("man")) status.removeBlock(RDDBlockId(2, 2)) status.removeBlock(RDDBlockId(2, 3)) assert(status.memUsed === actualMemUsed) assert(status.diskUsed === actualDiskUsed) - assert(status.offHeapUsed === actualOffHeapUsed) } // For testing StorageUtils.updateRddInfo and StorageUtils.getRddBlockLocations private def stockStorageStatuses: Seq[StorageStatus] = { - val status1 = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L) - val status2 = new StorageStatus(BlockManagerId("fat", "duck", 2), 2000L) - val status3 = new StorageStatus(BlockManagerId("fat", "cat", 3), 3000L) - status1.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) - status1.addBlock(RDDBlockId(0, 1), BlockStatus(memAndDisk, 1L, 2L, 0L)) - status2.addBlock(RDDBlockId(0, 2), BlockStatus(memAndDisk, 1L, 2L, 0L)) - status2.addBlock(RDDBlockId(0, 3), BlockStatus(memAndDisk, 1L, 2L, 0L)) - status2.addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) - status2.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 1L, 2L, 0L)) - status3.addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L, 0L)) - status3.addBlock(RDDBlockId(1, 2), BlockStatus(memAndDisk, 1L, 2L, 0L)) + val status1 = new StorageStatus(BlockManagerId("big", "dog", 1), 1000L, Some(1000L), Some(0L)) + val status2 = new StorageStatus(BlockManagerId("fat", "duck", 2), 2000L, Some(2000L), Some(0L)) + val status3 = new StorageStatus(BlockManagerId("fat", "cat", 3), 3000L, Some(3000L), Some(0L)) + status1.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L)) + status1.addBlock(RDDBlockId(0, 1), BlockStatus(memAndDisk, 1L, 2L)) + status2.addBlock(RDDBlockId(0, 2), BlockStatus(memAndDisk, 1L, 2L)) + status2.addBlock(RDDBlockId(0, 3), BlockStatus(memAndDisk, 1L, 2L)) + status2.addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L)) + status2.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 1L, 2L)) + status3.addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L)) + status3.addBlock(RDDBlockId(1, 2), BlockStatus(memAndDisk, 1L, 2L)) Seq(status1, status2, status3) } @@ -334,9 +315,9 @@ class StorageSuite extends SparkFunSuite { test("StorageUtils.getRddBlockLocations with multiple locations") { val storageStatuses = stockStorageStatuses - storageStatuses(0).addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) - storageStatuses(0).addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L, 0L)) - storageStatuses(2).addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + storageStatuses(0).addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L)) + storageStatuses(0).addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L)) + storageStatuses(2).addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L)) val blockLocations0 = StorageUtils.getRddBlockLocations(0, storageStatuses) val blockLocations1 = StorageUtils.getRddBlockLocations(1, storageStatuses) assert(blockLocations0.size === 5) @@ -351,4 +332,81 @@ class StorageSuite extends SparkFunSuite { assert(blockLocations1(RDDBlockId(1, 2)) === Seq("cat:3")) } + private val offheap = StorageLevel.OFF_HEAP + // For testing add, update, remove, get, and contains etc. for both RDD and non-RDD onheap + // and offheap blocks + private def storageStatus3: StorageStatus = { + val status = new StorageStatus(BlockManagerId("big", "dog", 1), 2000L, Some(1000L), Some(1000L)) + assert(status.rddBlocks.isEmpty) + status.addBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 10L, 20L)) + status.addBlock(TestBlockId("man"), BlockStatus(offheap, 10L, 0L)) + status.addBlock(RDDBlockId(0, 0), BlockStatus(offheap, 10L, 0L)) + status.addBlock(RDDBlockId(1, 1), BlockStatus(offheap, 100L, 0L)) + status.addBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 10L, 20L)) + status.addBlock(RDDBlockId(2, 3), BlockStatus(memAndDisk, 10L, 20L)) + status.addBlock(RDDBlockId(2, 4), BlockStatus(memAndDisk, 10L, 40L)) + status + } + + test("storage memUsed, diskUsed with on-heap and off-heap blocks") { + val status = storageStatus3 + def actualMemUsed: Long = status.blocks.values.map(_.memSize).sum + def actualDiskUsed: Long = status.blocks.values.map(_.diskSize).sum + + def actualOnHeapMemUsed: Long = + status.blocks.values.filter(!_.storageLevel.useOffHeap).map(_.memSize).sum + def actualOffHeapMemUsed: Long = + status.blocks.values.filter(_.storageLevel.useOffHeap).map(_.memSize).sum + + assert(status.maxMem === status.maxOnHeapMem.get + status.maxOffHeapMem.get) + + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.onHeapMemUsed.get === actualOnHeapMemUsed) + assert(status.offHeapMemUsed.get === actualOffHeapMemUsed) + + assert(status.memRemaining === status.maxMem - actualMemUsed) + assert(status.onHeapMemRemaining.get === status.maxOnHeapMem.get - actualOnHeapMemUsed) + assert(status.offHeapMemRemaining.get === status.maxOffHeapMem.get - actualOffHeapMemUsed) + + status.addBlock(TestBlockId("wire"), BlockStatus(memAndDisk, 400L, 500L)) + status.addBlock(RDDBlockId(25, 25), BlockStatus(memAndDisk, 40L, 50L)) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + + status.updateBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 4L, 5L)) + status.updateBlock(RDDBlockId(0, 0), BlockStatus(offheap, 4L, 0L)) + status.updateBlock(RDDBlockId(1, 1), BlockStatus(offheap, 4L, 0L)) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.onHeapMemUsed.get === actualOnHeapMemUsed) + assert(status.offHeapMemUsed.get === actualOffHeapMemUsed) + + status.removeBlock(TestBlockId("fire")) + status.removeBlock(TestBlockId("man")) + status.removeBlock(RDDBlockId(2, 2)) + status.removeBlock(RDDBlockId(2, 3)) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + } + + private def storageStatus4: StorageStatus = { + val status = new StorageStatus(BlockManagerId("big", "dog", 1), 2000L, None, None) + status + } + test("old SparkListenerBlockManagerAdded event compatible") { + // This scenario will only be happened when replaying old event log. In this scenario there's + // no block add or remove event replayed, so only total amount of memory is valid. + val status = storageStatus4 + assert(status.maxMem === status.maxMemory) + + assert(status.memUsed === 0L) + assert(status.diskUsed === 0L) + assert(status.onHeapMemUsed === None) + assert(status.offHeapMemUsed === None) + + assert(status.memRemaining === status.maxMem) + assert(status.onHeapMemRemaining === None) + assert(status.offHeapMemRemaining === None) + } } diff --git a/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala b/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala new file mode 100644 index 000000000000..bbd252d7be7e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.io.{File, FileOutputStream} + +import org.scalatest.{BeforeAndAfter, Matchers} + +import org.apache.spark._ +import org.apache.spark.util.Utils + +class TopologyMapperSuite extends SparkFunSuite + with Matchers + with BeforeAndAfter + with LocalSparkContext { + + test("File based Topology Mapper") { + val numHosts = 100 + val numRacks = 4 + val props = (1 to numHosts).map{i => s"host-$i" -> s"rack-${i % numRacks}"}.toMap + val propsFile = createPropertiesFile(props) + + val sparkConf = (new SparkConf(false)) + sparkConf.set("spark.storage.replication.topologyFile", propsFile.getAbsolutePath) + val topologyMapper = new FileBasedTopologyMapper(sparkConf) + + props.foreach {case (host, topology) => + val obtainedTopology = topologyMapper.getTopologyForHost(host) + assert(obtainedTopology.isDefined) + assert(obtainedTopology.get === topology) + } + + // we get None for hosts not in the file + assert(topologyMapper.getTopologyForHost("host").isEmpty) + + cleanup(propsFile) + } + + def createPropertiesFile(props: Map[String, String]): File = { + val testFile = new File(Utils.createTempDir(), "TopologyMapperSuite-test").getAbsoluteFile + val fileOS = new FileOutputStream(testFile) + props.foreach{case (k, v) => fileOS.write(s"$k=$v\n".getBytes)} + fileOS.close + testFile + } + + def cleanup(testFile: File): Unit = { + testFile.getParentFile.listFiles.filter { file => + file.getName.startsWith(testFile.getName) + }.foreach { _.delete() } + } + +} diff --git a/core/src/test/scala/org/apache/spark/ui/PagedTableSuite.scala b/core/src/test/scala/org/apache/spark/ui/PagedTableSuite.scala index cc76c141c53c..74eeca282882 100644 --- a/core/src/test/scala/org/apache/spark/ui/PagedTableSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/PagedTableSuite.scala @@ -64,7 +64,13 @@ class PagedTableSuite extends SparkFunSuite { override def row(t: Int): Seq[Node] = Nil - override def goButtonJavascriptFunction: (String, String) = ("", "") + override def pageSizeFormField: String = "pageSize" + + override def prevPageSizeFormField: String = "prevPageSize" + + override def pageNumberFormField: String = "page" + + override def goButtonFormPath: String = "" } assert(pagedTable.pageNavigation(1, 10, 1) === Nil) diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index 86699e7f5695..499d47b13d70 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.ui +import java.util.Locale import javax.servlet.http.HttpServletRequest import scala.xml.Node @@ -26,33 +27,27 @@ import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ +import org.apache.spark.storage.StorageStatusListener +import org.apache.spark.ui.exec.ExecutorsListener import org.apache.spark.ui.jobs.{JobProgressListener, StagePage, StagesTab} import org.apache.spark.ui.scope.RDDOperationGraphListener class StagePageSuite extends SparkFunSuite with LocalSparkContext { - test("peak execution memory only displayed if unsafe is enabled") { - val unsafeConf = "spark.sql.unsafe.enabled" - val conf = new SparkConf(false).set(unsafeConf, "true") - val html = renderStagePage(conf).toString().toLowerCase + private val peakExecutionMemory = 10 + + test("peak execution memory should displayed") { + val conf = new SparkConf(false) + val html = renderStagePage(conf).toString().toLowerCase(Locale.ROOT) val targetString = "peak execution memory" assert(html.contains(targetString)) - // Disable unsafe and make sure it's not there - val conf2 = new SparkConf(false).set(unsafeConf, "false") - val html2 = renderStagePage(conf2).toString().toLowerCase - assert(!html2.contains(targetString)) - // Avoid setting anything; it should be displayed by default - val conf3 = new SparkConf(false) - val html3 = renderStagePage(conf3).toString().toLowerCase - assert(html3.contains(targetString)) } test("SPARK-10543: peak execution memory should be per-task rather than cumulative") { - val unsafeConf = "spark.sql.unsafe.enabled" - val conf = new SparkConf(false).set(unsafeConf, "true") - val html = renderStagePage(conf).toString().toLowerCase + val conf = new SparkConf(false) + val html = renderStagePage(conf).toString().toLowerCase(Locale.ROOT) // verify min/25/50/75/max show task value not cumulative values - assert(html.contains("10.0 b" * 5)) + assert(html.contains(s"$peakExecutionMemory.0 b" * 5)) } /** @@ -62,11 +57,13 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { private def renderStagePage(conf: SparkConf): Seq[Node] = { val jobListener = new JobProgressListener(conf) val graphListener = new RDDOperationGraphListener(conf) + val executorsListener = new ExecutorsListener(new StorageStatusListener(conf), conf) val tab = mock(classOf[StagesTab], RETURNS_SMART_NULLS) val request = mock(classOf[HttpServletRequest]) when(tab.conf).thenReturn(conf) when(tab.progressListener).thenReturn(jobListener) when(tab.operationGraphListener).thenReturn(graphListener) + when(tab.executorsListener).thenReturn(executorsListener) when(tab.appName).thenReturn("testing") when(tab.headerTabs).thenReturn(Seq.empty) when(request.getParameter("id")).thenReturn("0") @@ -79,14 +76,13 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { (1 to 2).foreach { taskId => val taskInfo = new TaskInfo(taskId, taskId, 0, 0, "0", "localhost", TaskLocality.ANY, false) - val peakExecutionMemory = 10 - taskInfo.accumulables += new AccumulableInfo(0, InternalAccumulator.PEAK_EXECUTION_MEMORY, - Some(peakExecutionMemory.toString), (peakExecutionMemory * taskId).toString, true) jobListener.onStageSubmitted(SparkListenerStageSubmitted(stageInfo)) jobListener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo)) - taskInfo.markSuccessful() + taskInfo.markFinished(TaskState.FINISHED, System.currentTimeMillis()) + val taskMetrics = TaskMetrics.empty + taskMetrics.incPeakExecutionMemory(peakExecutionMemory) jobListener.onTaskEnd( - SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, TaskMetrics.empty)) + SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, taskMetrics)) } jobListener.onStageCompleted(SparkListenerStageCompleted(stageInfo)) page.render(request) diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 18eec7da9763..267c8dc1bd75 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -18,7 +18,8 @@ package org.apache.spark.ui import java.net.{HttpURLConnection, URL} -import javax.servlet.http.{HttpServletResponse, HttpServletRequest} +import java.util.Locale +import javax.servlet.http.{HttpServletRequest, HttpServletResponse} import scala.io.Source import scala.xml.Node @@ -26,20 +27,20 @@ import scala.xml.Node import com.gargoylesoftware.htmlunit.DefaultCssErrorHandler import org.json4s._ import org.json4s.jackson.JsonMethods -import org.openqa.selenium.htmlunit.HtmlUnitDriver import org.openqa.selenium.{By, WebDriver} +import org.openqa.selenium.htmlunit.HtmlUnitDriver import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ import org.w3c.css.sac.CSSParseException -import org.apache.spark.LocalSparkContext._ import org.apache.spark._ +import org.apache.spark.LocalSparkContext._ import org.apache.spark.api.java.StorageLevels import org.apache.spark.deploy.history.HistoryServerSuite import org.apache.spark.shuffle.FetchFailedException -import org.apache.spark.status.api.v1.{JacksonMessageWriter, StageStatus} +import org.apache.spark.status.api.v1.{JacksonMessageWriter, RDDDataDistribution, StageStatus} private[spark] class SparkUICssErrorHandler extends DefaultCssErrorHandler { @@ -76,14 +77,19 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B override def beforeAll(): Unit = { + super.beforeAll() webDriver = new HtmlUnitDriver { getWebClient.setCssErrorHandler(new SparkUICssErrorHandler) } } override def afterAll(): Unit = { - if (webDriver != null) { - webDriver.quit() + try { + if (webDriver != null) { + webDriver.quit() + } + } finally { + super.afterAll() } } @@ -98,6 +104,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B .set("spark.ui.enabled", "true") .set("spark.ui.port", "0") .set("spark.ui.killEnabled", killEnabled.toString) + .set("spark.memory.offHeap.size", "64m") val sc = new SparkContext(conf) assert(sc.ui.isDefined) sc @@ -146,6 +153,39 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B val updatedRddJson = getJson(ui, "storage/rdd/0") (updatedRddJson \ "storageLevel").extract[String] should be ( StorageLevels.MEMORY_ONLY.description) + + val dataDistributions0 = + (updatedRddJson \ "dataDistribution").extract[Seq[RDDDataDistribution]] + dataDistributions0.length should be (1) + val dist0 = dataDistributions0.head + + dist0.onHeapMemoryUsed should not be (None) + dist0.memoryUsed should be (dist0.onHeapMemoryUsed.get) + dist0.onHeapMemoryRemaining should not be (None) + dist0.offHeapMemoryRemaining should not be (None) + dist0.memoryRemaining should be ( + dist0.onHeapMemoryRemaining.get + dist0.offHeapMemoryRemaining.get) + dist0.onHeapMemoryUsed should not be (Some(0L)) + dist0.offHeapMemoryUsed should be (Some(0L)) + + rdd.unpersist() + rdd.persist(StorageLevels.OFF_HEAP).count() + val updatedStorageJson1 = getJson(ui, "storage/rdd") + updatedStorageJson1.children.length should be (1) + val updatedRddJson1 = getJson(ui, "storage/rdd/0") + val dataDistributions1 = + (updatedRddJson1 \ "dataDistribution").extract[Seq[RDDDataDistribution]] + dataDistributions1.length should be (1) + val dist1 = dataDistributions1.head + + dist1.offHeapMemoryUsed should not be (None) + dist1.memoryUsed should be (dist1.offHeapMemoryUsed.get) + dist1.onHeapMemoryRemaining should not be (None) + dist1.offHeapMemoryRemaining should not be (None) + dist1.memoryRemaining should be ( + dist1.onHeapMemoryRemaining.get + dist1.offHeapMemoryRemaining.get) + dist1.onHeapMemoryUsed should be (Some(0L)) + dist1.offHeapMemoryUsed should not be (Some(0L)) } } @@ -189,6 +229,22 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() } + withSpark(newSparkContext(killEnabled = true)) { sc => + runSlowJob(sc) + eventually(timeout(5 seconds), interval(50 milliseconds)) { + goToUi(sc, "/jobs") + assert(hasKillLink) + } + } + + withSpark(newSparkContext(killEnabled = false)) { sc => + runSlowJob(sc) + eventually(timeout(5 seconds), interval(50 milliseconds)) { + goToUi(sc, "/jobs") + assert(!hasKillLink) + } + } + withSpark(newSparkContext(killEnabled = true)) { sc => runSlowJob(sc) eventually(timeout(5 seconds), interval(50 milliseconds)) { @@ -213,7 +269,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B eventually(timeout(5 seconds), interval(50 milliseconds)) { goToUi(sc, "/jobs") val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq - tableHeaders should not contain "Job Id (Job Group)" + tableHeaders(0) should not startWith "Job Id (Job Group)" } // Once at least one job has been run in a job group, then we should display the group name: sc.setJobGroup("my-job-group", "my-job-group-description") @@ -221,7 +277,8 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B eventually(timeout(5 seconds), interval(50 milliseconds)) { goToUi(sc, "/jobs") val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq - tableHeaders should contain ("Job Id (Job Group)") + // Can suffix up/down arrow in the header + tableHeaders(0) should startWith ("Job Id (Job Group)") } val jobJson = getJson(sc.ui.get, "jobs") @@ -263,12 +320,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B eventually(timeout(5 seconds), interval(50 milliseconds)) { goToUi(sc, "/jobs") find(cssSelector(".stage-progress-cell")).get.text should be ("2/2 (1 failed)") - // Ideally, the following test would pass, but currently we overcount completed tasks - // if task recomputations occur: - // find(cssSelector(".progress-cell .progress")).get.text should be ("2/2 (1 failed)") - // Instead, we guarantee that the total number of tasks is always correct, while the number - // of completed tasks may be higher: - find(cssSelector(".progress-cell .progress")).get.text should be ("3/2 (1 failed)") + find(cssSelector(".progress-cell .progress")).get.text should be ("2/2 (1 failed)") } val jobJson = getJson(sc.ui.get, "jobs") (jobJson \ "numTasks").extract[Int]should be (2) @@ -284,7 +336,11 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B JInt(stageId) <- stage \ "stageId" JInt(attemptId) <- stage \ "attemptId" } { - val exp = if (attemptId == 0 && stageId == 1) StageStatus.FAILED else StageStatus.COMPLETE + val exp = if (attemptId.toInt == 0 && stageId.toInt == 1) { + StageStatus.FAILED + } else { + StageStatus.COMPLETE + } status should be (exp.name()) } @@ -393,8 +449,8 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B eventually(timeout(10 seconds), interval(50 milliseconds)) { goToUi(sc, "/jobs") findAll(cssSelector("tbody tr a")).foreach { link => - link.text.toLowerCase should include ("count") - link.text.toLowerCase should not include "unknown" + link.text.toLowerCase(Locale.ROOT) should include ("count") + link.text.toLowerCase(Locale.ROOT) should not include "unknown" } } } @@ -443,23 +499,27 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B } test("kill stage POST/GET response is correct") { - def getResponseCode(url: URL, method: String): Int = { - val connection = url.openConnection().asInstanceOf[HttpURLConnection] - connection.setRequestMethod(method) - connection.connect() - val code = connection.getResponseCode() - connection.disconnect() - code + withSpark(newSparkContext(killEnabled = true)) { sc => + sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + val url = new URL( + sc.ui.get.webUrl.stripSuffix("/") + "/stages/stage/kill/?id=0") + // SPARK-6846: should be POST only but YARN AM doesn't proxy POST + TestUtils.httpResponseCode(url, "GET") should be (200) + TestUtils.httpResponseCode(url, "POST") should be (200) + } } + } + test("kill job POST/GET response is correct") { withSpark(newSparkContext(killEnabled = true)) { sc => sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() eventually(timeout(5 seconds), interval(50 milliseconds)) { val url = new URL( - sc.ui.get.appUIAddress.stripSuffix("/") + "/stages/stage/kill/?id=0&terminate=true") + sc.ui.get.webUrl.stripSuffix("/") + "/jobs/job/kill/?id=0") // SPARK-6846: should be POST only but YARN AM doesn't proxy POST - getResponseCode(url, "GET") should be (200) - getResponseCode(url, "POST") should be (200) + TestUtils.httpResponseCode(url, "GET") should be (200) + TestUtils.httpResponseCode(url, "POST") should be (200) } } } @@ -590,7 +650,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B test("live UI json application list") { withSpark(newSparkContext()) { sc => val appListRawJson = HistoryServerSuite.getUrl(new URL( - sc.ui.get.appUIAddress + "/api/v1/applications")) + sc.ui.get.webUrl + "/api/v1/applications")) val appListJsonAst = JsonMethods.parse(appListRawJson) appListJsonAst.children.length should be (1) val attempts = (appListJsonAst \ "attempts").children @@ -610,34 +670,34 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity) rdd.count() - val stage0 = Source.fromURL(sc.ui.get.appUIAddress + + val stage0 = Source.fromURL(sc.ui.get.webUrl + "/stages/stage/?id=0&attempt=0&expandDagViz=true").mkString assert(stage0.contains("digraph G {\n subgraph clusterstage_0 {\n " + "label="Stage 0";\n subgraph ")) assert(stage0.contains("{\n label="parallelize";\n " + - "0 [label="ParallelCollectionRDD [0]"];\n }")) + "0 [label="ParallelCollectionRDD [0]")) assert(stage0.contains("{\n label="map";\n " + - "1 [label="MapPartitionsRDD [1]"];\n }")) + "1 [label="MapPartitionsRDD [1]")) assert(stage0.contains("{\n label="groupBy";\n " + - "2 [label="MapPartitionsRDD [2]"];\n }")) + "2 [label="MapPartitionsRDD [2]")) - val stage1 = Source.fromURL(sc.ui.get.appUIAddress + + val stage1 = Source.fromURL(sc.ui.get.webUrl + "/stages/stage/?id=1&attempt=0&expandDagViz=true").mkString assert(stage1.contains("digraph G {\n subgraph clusterstage_1 {\n " + "label="Stage 1";\n subgraph ")) assert(stage1.contains("{\n label="groupBy";\n " + - "3 [label="ShuffledRDD [3]"];\n }")) + "3 [label="ShuffledRDD [3]")) assert(stage1.contains("{\n label="map";\n " + - "4 [label="MapPartitionsRDD [4]"];\n }")) + "4 [label="MapPartitionsRDD [4]")) assert(stage1.contains("{\n label="groupBy";\n " + - "5 [label="MapPartitionsRDD [5]"];\n }")) + "5 [label="MapPartitionsRDD [5]")) - val stage2 = Source.fromURL(sc.ui.get.appUIAddress + + val stage2 = Source.fromURL(sc.ui.get.webUrl + "/stages/stage/?id=2&attempt=0&expandDagViz=true").mkString assert(stage2.contains("digraph G {\n subgraph clusterstage_2 {\n " + "label="Stage 2";\n subgraph ")) assert(stage2.contains("{\n label="groupBy";\n " + - "6 [label="ShuffledRDD [6]"];\n }")) + "6 [label="ShuffledRDD [6]")) } } @@ -646,7 +706,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B } def goToUi(ui: SparkUI, path: String): Unit = { - go to (ui.appUIAddress.stripSuffix("/") + path) + go to (ui.webUrl.stripSuffix("/") + path) } def parseDate(json: JValue): Long = { @@ -658,6 +718,6 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B } def apiUrl(ui: SparkUI, path: String): URL = { - new URL(ui.appUIAddress + "/api/v1/applications/" + ui.sc.get.applicationId + "/" + path) + new URL(ui.webUrl + "/api/v1/applications/" + ui.sc.get.applicationId + "/" + path) } } diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 8f9502b5673d..36ea3799afdf 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -17,17 +17,21 @@ package org.apache.spark.ui -import java.net.ServerSocket +import java.net.{BindException, ServerSocket} +import java.net.{URI, URL} +import java.util.Locale +import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.io.Source -import scala.util.{Failure, Success, Try} -import org.eclipse.jetty.servlet.ServletContextHandler +import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} +import org.mockito.Mockito.{mock, when} import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ +import org.apache.spark._ import org.apache.spark.LocalSparkContext._ -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.util.Utils class UISuite extends SparkFunSuite { @@ -45,16 +49,34 @@ class UISuite extends SparkFunSuite { sc } + private def sslDisabledConf(): (SparkConf, SSLOptions) = { + val conf = new SparkConf + (conf, new SecurityManager(conf).getSSLOptions("ui")) + } + + private def sslEnabledConf(sslPort: Option[Int] = None): (SparkConf, SSLOptions) = { + val keyStoreFilePath = getTestResourcePath("spark.keystore") + val conf = new SparkConf() + .set("spark.ssl.ui.enabled", "true") + .set("spark.ssl.ui.keyStore", keyStoreFilePath) + .set("spark.ssl.ui.keyStorePassword", "123456") + .set("spark.ssl.ui.keyPassword", "123456") + sslPort.foreach { p => + conf.set("spark.ssl.ui.port", p.toString) + } + (conf, new SecurityManager(conf).getSSLOptions("ui")) + } + ignore("basic ui visibility") { withSpark(newSparkContext()) { sc => // test if the ui is visible, and all the expected tabs are visible eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(sc.ui.get.appUIAddress).mkString + val html = Source.fromURL(sc.ui.get.webUrl).mkString assert(!html.contains("random data that should not be present")) - assert(html.toLowerCase.contains("stages")) - assert(html.toLowerCase.contains("storage")) - assert(html.toLowerCase.contains("environment")) - assert(html.toLowerCase.contains("executors")) + assert(html.toLowerCase(Locale.ROOT).contains("stages")) + assert(html.toLowerCase(Locale.ROOT).contains("storage")) + assert(html.toLowerCase(Locale.ROOT).contains("environment")) + assert(html.toLowerCase(Locale.ROOT).contains("executors")) } } } @@ -64,57 +86,225 @@ class UISuite extends SparkFunSuite { // test if visible from http://localhost:4040 eventually(timeout(10 seconds), interval(50 milliseconds)) { val html = Source.fromURL("http://localhost:4040").mkString - assert(html.toLowerCase.contains("stages")) + assert(html.toLowerCase(Locale.ROOT).contains("stages")) } } } test("jetty selects different port under contention") { - val server = new ServerSocket(0) - val startPort = server.getLocalPort - val serverInfo1 = JettyUtils.startJettyServer( - "0.0.0.0", startPort, Seq[ServletContextHandler](), new SparkConf) - val serverInfo2 = JettyUtils.startJettyServer( - "0.0.0.0", startPort, Seq[ServletContextHandler](), new SparkConf) - // Allow some wiggle room in case ports on the machine are under contention - val boundPort1 = serverInfo1.boundPort - val boundPort2 = serverInfo2.boundPort - assert(boundPort1 != startPort) - assert(boundPort2 != startPort) - assert(boundPort1 != boundPort2) - serverInfo1.server.stop() - serverInfo2.server.stop() - server.close() + var server: ServerSocket = null + var serverInfo1: ServerInfo = null + var serverInfo2: ServerInfo = null + val (conf, sslOptions) = sslDisabledConf() + try { + server = new ServerSocket(0) + val startPort = server.getLocalPort + serverInfo1 = JettyUtils.startJettyServer( + "0.0.0.0", startPort, sslOptions, Seq[ServletContextHandler](), conf) + serverInfo2 = JettyUtils.startJettyServer( + "0.0.0.0", startPort, sslOptions, Seq[ServletContextHandler](), conf) + // Allow some wiggle room in case ports on the machine are under contention + val boundPort1 = serverInfo1.boundPort + val boundPort2 = serverInfo2.boundPort + assert(boundPort1 != startPort) + assert(boundPort2 != startPort) + assert(boundPort1 != boundPort2) + } finally { + stopServer(serverInfo1) + stopServer(serverInfo2) + closeSocket(server) + } + } + + test("jetty with https selects different port under contention") { + var server: ServerSocket = null + var serverInfo1: ServerInfo = null + var serverInfo2: ServerInfo = null + try { + server = new ServerSocket(0) + val startPort = server.getLocalPort + val (conf, sslOptions) = sslEnabledConf() + serverInfo1 = JettyUtils.startJettyServer( + "0.0.0.0", startPort, sslOptions, Seq[ServletContextHandler](), conf, "server1") + serverInfo2 = JettyUtils.startJettyServer( + "0.0.0.0", startPort, sslOptions, Seq[ServletContextHandler](), conf, "server2") + // Allow some wiggle room in case ports on the machine are under contention + val boundPort1 = serverInfo1.boundPort + val boundPort2 = serverInfo2.boundPort + assert(boundPort1 != startPort) + assert(boundPort2 != startPort) + assert(boundPort1 != boundPort2) + } finally { + stopServer(serverInfo1) + stopServer(serverInfo2) + closeSocket(server) + } } test("jetty binds to port 0 correctly") { - val serverInfo = JettyUtils.startJettyServer( - "0.0.0.0", 0, Seq[ServletContextHandler](), new SparkConf) - val server = serverInfo.server - val boundPort = serverInfo.boundPort - assert(server.getState === "STARTED") - assert(boundPort != 0) - Try { new ServerSocket(boundPort) } match { - case Success(s) => fail("Port %s doesn't seem used by jetty server".format(boundPort)) - case Failure(e) => + var socket: ServerSocket = null + var serverInfo: ServerInfo = null + val (conf, sslOptions) = sslDisabledConf() + try { + serverInfo = JettyUtils.startJettyServer( + "0.0.0.0", 0, sslOptions, Seq[ServletContextHandler](), conf) + val server = serverInfo.server + val boundPort = serverInfo.boundPort + assert(server.getState === "STARTED") + assert(boundPort != 0) + intercept[BindException] { + socket = new ServerSocket(boundPort) + } + } finally { + stopServer(serverInfo) + closeSocket(socket) } } - test("verify appUIAddress contains the scheme") { + test("jetty with https binds to port 0 correctly") { + var socket: ServerSocket = null + var serverInfo: ServerInfo = null + try { + val (conf, sslOptions) = sslEnabledConf() + serverInfo = JettyUtils.startJettyServer( + "0.0.0.0", 0, sslOptions, Seq[ServletContextHandler](), conf) + val server = serverInfo.server + val boundPort = serverInfo.boundPort + assert(server.getState === "STARTED") + assert(boundPort != 0) + assert(serverInfo.securePort.isDefined) + intercept[BindException] { + socket = new ServerSocket(boundPort) + } + } finally { + stopServer(serverInfo) + closeSocket(socket) + } + } + + test("verify webUrl contains the scheme") { withSpark(newSparkContext()) { sc => val ui = sc.ui.get - val uiAddress = ui.appUIAddress - val uiHostPort = ui.appUIHostPort - assert(uiAddress.equals("http://" + uiHostPort)) + val uiAddress = ui.webUrl + assert(uiAddress.startsWith("http://") || uiAddress.startsWith("https://")) } } - test("verify appUIAddress contains the port") { + test("verify webUrl contains the port") { withSpark(newSparkContext()) { sc => val ui = sc.ui.get - val splitUIAddress = ui.appUIAddress.split(':') + val splitUIAddress = ui.webUrl.split(':') val boundPort = ui.boundPort assert(splitUIAddress(2).toInt == boundPort) } } + + test("verify proxy rewrittenURI") { + val prefix = "/worker-id" + val target = "http://localhost:8081" + val path = "/worker-id/json" + var rewrittenURI = JettyUtils.createProxyURI(prefix, target, path, null) + assert(rewrittenURI.toString() === "http://localhost:8081/json") + rewrittenURI = JettyUtils.createProxyURI(prefix, target, path, "test=done") + assert(rewrittenURI.toString() === "http://localhost:8081/json?test=done") + rewrittenURI = JettyUtils.createProxyURI(prefix, target, "/worker-id", null) + assert(rewrittenURI.toString() === "http://localhost:8081") + rewrittenURI = JettyUtils.createProxyURI(prefix, target, "/worker-id/test%2F", null) + assert(rewrittenURI.toString() === "http://localhost:8081/test%2F") + rewrittenURI = JettyUtils.createProxyURI(prefix, target, "/worker-id/%F0%9F%98%84", null) + assert(rewrittenURI.toString() === "http://localhost:8081/%F0%9F%98%84") + rewrittenURI = JettyUtils.createProxyURI(prefix, target, "/worker-noid/json", null) + assert(rewrittenURI === null) + } + + test("verify rewriting location header for reverse proxy") { + val clientRequest = mock(classOf[HttpServletRequest]) + var headerValue = "http://localhost:4040/jobs" + val targetUri = URI.create("http://localhost:4040") + when(clientRequest.getScheme()).thenReturn("http") + when(clientRequest.getHeader("host")).thenReturn("localhost:8080") + when(clientRequest.getPathInfo()).thenReturn("/proxy/worker-id/jobs") + var newHeader = JettyUtils.createProxyLocationHeader(headerValue, clientRequest, targetUri) + assert(newHeader.toString() === "http://localhost:8080/proxy/worker-id/jobs") + headerValue = "http://localhost:4041/jobs" + newHeader = JettyUtils.createProxyLocationHeader(headerValue, clientRequest, targetUri) + assert(newHeader === null) + } + + test("http -> https redirect applies to all URIs") { + var serverInfo: ServerInfo = null + try { + val servlet = new HttpServlet() { + override def doGet(req: HttpServletRequest, res: HttpServletResponse): Unit = { + res.sendError(HttpServletResponse.SC_OK) + } + } + + def newContext(path: String): ServletContextHandler = { + val ctx = new ServletContextHandler() + ctx.setContextPath(path) + ctx.addServlet(new ServletHolder(servlet), "/root") + ctx + } + + val (conf, sslOptions) = sslEnabledConf() + serverInfo = JettyUtils.startJettyServer("0.0.0.0", 0, sslOptions, + Seq[ServletContextHandler](newContext("/"), newContext("/test1")), + conf) + assert(serverInfo.server.getState === "STARTED") + + val testContext = newContext("/test2") + serverInfo.addHandler(testContext) + testContext.start() + + val httpPort = serverInfo.boundPort + + val tests = Seq( + ("http", serverInfo.boundPort, HttpServletResponse.SC_FOUND), + ("https", serverInfo.securePort.get, HttpServletResponse.SC_OK)) + + tests.foreach { case (scheme, port, expected) => + val urls = Seq( + s"$scheme://localhost:$port/root", + s"$scheme://localhost:$port/test1/root", + s"$scheme://localhost:$port/test2/root") + urls.foreach { url => + val rc = TestUtils.httpResponseCode(new URL(url)) + assert(rc === expected, s"Unexpected status $rc for $url") + } + } + } finally { + stopServer(serverInfo) + } + } + + test("specify both http and https ports separately") { + var socket: ServerSocket = null + var serverInfo: ServerInfo = null + try { + socket = new ServerSocket(0) + + // Make sure the SSL port lies way outside the "http + 400" range used as the default. + val baseSslPort = Utils.userPort(socket.getLocalPort(), 10000) + val (conf, sslOptions) = sslEnabledConf(sslPort = Some(baseSslPort)) + + serverInfo = JettyUtils.startJettyServer("0.0.0.0", socket.getLocalPort() + 1, + sslOptions, Seq[ServletContextHandler](), conf, "server1") + + val notAllowed = Utils.userPort(serverInfo.boundPort, 400) + assert(serverInfo.securePort.isDefined) + assert(serverInfo.securePort.get != Utils.userPort(serverInfo.boundPort, 400)) + } finally { + stopServer(serverInfo) + closeSocket(socket) + } + } + + def stopServer(info: ServerInfo): Unit = { + if (info != null) info.stop() + } + + def closeSocket(socket: ServerSocket): Unit = { + if (socket != null) socket.close + } } diff --git a/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala b/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala index 2b693c165180..423daacc0f5a 100644 --- a/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala @@ -17,49 +17,168 @@ package org.apache.spark.ui -import scala.xml.Elem +import scala.xml.{Node, Text} import org.apache.spark.SparkFunSuite class UIUtilsSuite extends SparkFunSuite { import UIUtils._ - test("makeDescription") { + test("makeDescription(plainText = false)") { verify( """test text """, test text , - "Correctly formatted text with only anchors and relative links should generate HTML" + "Correctly formatted text with only anchors and relative links should generate HTML", + plainText = false ) verify( """test """, {"""test """}, - "Badly formatted text should make the description be treated as a streaming instead of HTML" + "Badly formatted text should make the description be treated as a string instead of HTML", + plainText = false ) verify( """test text """, {"""test text """}, - "Non-relative links should make the description be treated as a string instead of HTML" + "Non-relative links should make the description be treated as a string instead of HTML", + plainText = false ) verify( """test""", {"""test"""}, - "Non-anchor elements should make the description be treated as a string instead of HTML" + "Non-anchor elements should make the description be treated as a string instead of HTML", + plainText = false ) verify( """test text """, test text , baseUrl = "base", - errorMsg = "Base URL should be prepended to html links" + errorMsg = "Base URL should be prepended to html links", + plainText = false ) } + test("makeDescription(plainText = true)") { + verify( + """test text """, + Text("test text "), + "Correctly formatted text with only anchors and relative links should generate a string " + + "without any html tags", + plainText = true + ) + + verify( + """test text1 text2 """, + Text("test text1 text2 "), + "Correctly formatted text with multiple anchors and relative links should generate a " + + "string without any html tags", + plainText = true + ) + + verify( + """test text """, + Text("test text "), + "Correctly formatted text with nested anchors and relative links and/or spans should " + + "generate a string without any html tags", + plainText = true + ) + + verify( + """test """, + Text("""test """), + "Badly formatted text should make the description be as the same as the original text", + plainText = true + ) + + verify( + """test text """, + Text("""test text """), + "Non-relative links should make the description be as the same as the original text", + plainText = true + ) + + verify( + """test""", + Text("""test"""), + "Non-anchor elements should make the description be as the same as the original text", + plainText = true + ) + } + + test("SPARK-11906: Progress bar should not overflow because of speculative tasks") { + val generated = makeProgressBar(2, 3, 0, 0, Map.empty, 4).head.child.filter(_.label == "div") + val expected = Seq( +
    , +
    + ) + assert(generated.sameElements(expected), + s"\nRunning progress bar should round down\n\nExpected:\n$expected\nGenerated:\n$generated") + } + + test("decodeURLParameter (SPARK-12708: Sorting task error in Stages Page when yarn mode.)") { + val encoded1 = "%252F" + val decoded1 = "/" + val encoded2 = "%253Cdriver%253E" + val decoded2 = "" + + assert(decoded1 === decodeURLParameter(encoded1)) + assert(decoded2 === decodeURLParameter(encoded2)) + + // verify that no affect to decoded URL. + assert(decoded1 === decodeURLParameter(decoded1)) + assert(decoded2 === decodeURLParameter(decoded2)) + } + + test("SPARK-20393: Prevent newline characters in parameters.") { + val encoding = "Encoding:base64%0d%0a%0d%0aPGh0bWw%2bjcmlwdD48L2h0bWw%2b" + val stripEncoding = "Encoding:base64PGh0bWw%2bjcmlwdD48L2h0bWw%2b" + + assert(stripEncoding === stripXSS(encoding)) + } + + test("SPARK-20393: Prevent script from parameters running on page.") { + val scriptAlert = """>"'> diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 01718d98dffe..4d0d043a349b 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -20,14 +20,14 @@ if not (ENV['SKIP_API'] == '1') if not (ENV['SKIP_SCALADOC'] == '1') - # Build Scaladoc for Java/Scala + # Build Scaladoc for Scala and Javadoc for Java puts "Moving to project root and building API docs." curr_dir = pwd cd("..") puts "Running 'build/sbt -Pkinesis-asl clean compile unidoc' from " + pwd + "; this may take a few minutes..." - puts `build/sbt -Pkinesis-asl clean compile unidoc` + system("build/sbt -Pkinesis-asl clean compile unidoc") || raise("Unidoc generation failed") puts "Moving back into docs dir." cd("docs") @@ -37,7 +37,7 @@ # Copy over the unified ScalaDoc for all projects to api/scala. # This directory will be copied over to _site when `jekyll` command is run. - source = "../target/scala-2.10/unidoc" + source = "../target/scala-2.11/unidoc" dest = "api/scala" puts "Making directory " + dest @@ -113,33 +113,68 @@ File.open(css_file, 'a') { |f| f.write("\n" + css.join()) } end - # Build Sphinx docs for Python + if not (ENV['SKIP_PYTHONDOC'] == '1') + # Build Sphinx docs for Python - puts "Moving to python/docs directory and building sphinx." - cd("../python/docs") - puts `make html` + puts "Moving to python/docs directory and building sphinx." + cd("../python/docs") + system("make html") || raise("Python doc generation failed") - puts "Moving back into home dir." - cd("../../") + puts "Moving back into docs dir." + cd("../../docs") + + puts "Making directory api/python" + mkdir_p "api/python" + + puts "cp -r ../python/docs/_build/html/. api/python" + cp_r("../python/docs/_build/html/.", "api/python") + end - puts "Making directory api/python" - mkdir_p "docs/api/python" + if not (ENV['SKIP_RDOC'] == '1') + # Build SparkR API docs - puts "cp -r python/docs/_build/html/. docs/api/python" - cp_r("python/docs/_build/html/.", "docs/api/python") + puts "Moving to R directory and building roxygen docs." + cd("../R") + system("./create-docs.sh") || raise("R doc generation failed") - # Build SparkR API docs - puts "Moving to R directory and building roxygen docs." - cd("R") - puts `./create-docs.sh` + puts "Moving back into docs dir." + cd("../docs") - puts "Moving back into home dir." - cd("../") + puts "Making directory api/R" + mkdir_p "api/R" - puts "Making directory api/R" - mkdir_p "docs/api/R" + puts "cp -r ../R/pkg/html/. api/R" + cp_r("../R/pkg/html/.", "api/R") - puts "cp -r R/pkg/html/. docs/api/R" - cp_r("R/pkg/html/.", "docs/api/R") + puts "cp ../R/pkg/DESCRIPTION api" + cp("../R/pkg/DESCRIPTION", "api") + end + + if not (ENV['SKIP_SQLDOC'] == '1') + # Build SQL API docs + + puts "Moving to project root and building API docs." + curr_dir = pwd + cd("..") + + puts "Running 'build/sbt clean package' from " + pwd + "; this may take a few minutes..." + system("build/sbt clean package") || raise("SQL doc generation failed") + + puts "Moving back into docs dir." + cd("docs") + + puts "Moving to SQL directory and building docs." + cd("../sql") + system("./create-docs.sh") || raise("SQL doc generation failed") + + puts "Moving back into docs dir." + cd("../docs") + + puts "Making directory api/sql" + mkdir_p "api/sql" + + puts "cp -r ../sql/site/. api/sql" + cp_r("../sql/site/.", "api/sql") + end end diff --git a/docs/_plugins/include_example.rb b/docs/_plugins/include_example.rb index 6ee63a5ac69d..6ea1d438f529 100644 --- a/docs/_plugins/include_example.rb +++ b/docs/_plugins/include_example.rb @@ -20,27 +20,50 @@ module Jekyll class IncludeExampleTag < Liquid::Tag - + def initialize(tag_name, markup, tokens) @markup = markup super end - + def render(context) site = context.registers[:site] - config_dir = (site.config['code_dir'] || '../examples/src/main').sub(/^\//,'') + config_dir = '../examples/src/main' @code_dir = File.join(site.source, config_dir) clean_markup = @markup.strip - @file = File.join(@code_dir, clean_markup) - @lang = clean_markup.split('.').last - code = File.open(@file).read.encode("UTF-8") + parts = clean_markup.strip.split(' ') + if parts.length > 1 then + @snippet_label = ':' + parts[0] + snippet_file = parts[1] + else + @snippet_label = '' + snippet_file = parts[0] + end + + @file = File.join(@code_dir, snippet_file) + @lang = snippet_file.split('.').last + + begin + code = File.open(@file).read.encode("UTF-8") + rescue => e + # We need to explicitly exit on execptions here because Jekyll will silently swallow + # them, leading to silent build failures (see https://github.com/jekyll/jekyll/issues/5104) + puts(e) + puts(e.backtrace) + exit 1 + end code = select_lines(code) - - Pygments.highlight(code, :lexer => @lang) + + rendered_code = Pygments.highlight(code, :lexer => @lang) + + hint = "
    Find full example code at " \ + "\"examples/src/main/#{snippet_file}\" in the Spark repo.
    " + + rendered_code + hint end - + # Trim the code block so as to have the same indention, regardless of their positions in the # code file. def trim_codeblock(lines) @@ -61,19 +84,19 @@ def select_lines(code) # Select the array of start labels from code. startIndices = lines .each_with_index - .select { |l, i| l.include? "$example on$" } + .select { |l, i| l.include? "$example on#{@snippet_label}$" } .map { |l, i| i } # Select the array of end labels from code. endIndices = lines .each_with_index - .select { |l, i| l.include? "$example off$" } + .select { |l, i| l.include? "$example off#{@snippet_label}$" } .map { |l, i| i } - raise "Start indices amount is not equal to end indices amount, please check the code." \ + raise "Start indices amount is not equal to end indices amount, see #{@file}." \ unless startIndices.size == endIndices.size - raise "No code is selected by include_example, please check the code." \ + raise "No code is selected by include_example, see #{@file}." \ if startIndices.size == 0 # Select and join code blocks together, with a space line between each of two continuous @@ -81,11 +104,16 @@ def select_lines(code) lastIndex = -1 result = "" startIndices.zip(endIndices).each do |start, endline| - raise "Overlapping between two example code blocks are not allowed." if start <= lastIndex - raise "$example on$ should not be in the same line with $example off$." if start == endline + raise "Overlapping between two example code blocks are not allowed, see #{@file}." \ + if start <= lastIndex + raise "$example on$ should not be in the same line with $example off$, see #{@file}." \ + if start == endline lastIndex = endline range = Range.new(start + 1, endline - 1) - result += trim_codeblock(lines[range]).join + trimmed = trim_codeblock(lines[range]) + # Filter out possible example tags of overlapped labels. + taggs_filtered = trimmed.select { |l| !l.include? '$example ' } + result += taggs_filtered.join result += "\n" end result diff --git a/docs/api.md b/docs/api.md index ae7d51c2aefb..70484f02de78 100644 --- a/docs/api.md +++ b/docs/api.md @@ -9,3 +9,4 @@ Here you can read API docs for Spark and its submodules. - [Spark Java API (Javadoc)](api/java/index.html) - [Spark Python API (Sphinx)](api/python/index.html) - [Spark R API (Roxygen2)](api/R/index.html) +- [Spark SQL, Built-in Functions (MkDocs)](api/sql/index.html) diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md deleted file mode 100644 index 347ca4a7af98..000000000000 --- a/docs/bagel-programming-guide.md +++ /dev/null @@ -1,159 +0,0 @@ ---- -layout: global -displayTitle: Bagel Programming Guide -title: Bagel ---- - -**Bagel is deprecated, and superseded by [GraphX](graphx-programming-guide.html).** - -Bagel is a Spark implementation of Google's [Pregel](http://portal.acm.org/citation.cfm?id=1807184) graph processing framework. Bagel currently supports basic graph computation, combiners, and aggregators. - -In the Pregel programming model, jobs run as a sequence of iterations called _supersteps_. In each superstep, each vertex in the graph runs a user-specified function that can update state associated with the vertex and send messages to other vertices for use in the *next* iteration. - -This guide shows the programming model and features of Bagel by walking through an example implementation of PageRank on Bagel. - -# Linking with Bagel - -To use Bagel in your program, add the following SBT or Maven dependency: - - groupId = org.apache.spark - artifactId = spark-bagel_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION}} - -# Programming Model - -Bagel operates on a graph represented as a [distributed dataset](programming-guide.html) of (K, V) pairs, where keys are vertex IDs and values are vertices plus their associated state. In each superstep, Bagel runs a user-specified compute function on each vertex that takes as input the current vertex state and a list of messages sent to that vertex during the previous superstep, and returns the new vertex state and a list of outgoing messages. - -For example, we can use Bagel to implement PageRank. Here, vertices represent pages, edges represent links between pages, and messages represent shares of PageRank sent to the pages that a particular page links to. - -We first extend the default `Vertex` class to store a `Double` -representing the current PageRank of the vertex, and similarly extend -the `Message` and `Edge` classes. Note that these need to be marked `@serializable` to allow Spark to transfer them across machines. We also import the Bagel types and implicit conversions. - -{% highlight scala %} -import org.apache.spark.bagel._ -import org.apache.spark.bagel.Bagel._ - -@serializable class PREdge(val targetId: String) extends Edge - -@serializable class PRVertex( - val id: String, val rank: Double, val outEdges: Seq[Edge], - val active: Boolean) extends Vertex - -@serializable class PRMessage( - val targetId: String, val rankShare: Double) extends Message -{% endhighlight %} - -Next, we load a sample graph from a text file as a distributed dataset and package it into `PRVertex` objects. We also cache the distributed dataset because Bagel will use it multiple times and we'd like to avoid recomputing it. - -{% highlight scala %} -val input = sc.textFile("data/mllib/pagerank_data.txt") - -val numVerts = input.count() - -val verts = input.map(line => { - val fields = line.split('\t') - val (id, linksStr) = (fields(0), fields(1)) - val links = linksStr.split(',').map(new PREdge(_)) - (id, new PRVertex(id, 1.0 / numVerts, links, true)) -}).cache -{% endhighlight %} - -We run the Bagel job, passing in `verts`, an empty distributed dataset of messages, and a custom compute function that runs PageRank for 10 iterations. - -{% highlight scala %} -val emptyMsgs = sc.parallelize(List[(String, PRMessage)]()) - -def compute(self: PRVertex, msgs: Option[Seq[PRMessage]], superstep: Int) -: (PRVertex, Iterable[PRMessage]) = { - val msgSum = msgs.getOrElse(List()).map(_.rankShare).sum - val newRank = - if (msgSum != 0) - 0.15 / numVerts + 0.85 * msgSum - else - self.rank - val halt = superstep >= 10 - val msgsOut = - if (!halt) - self.outEdges.map(edge => - new PRMessage(edge.targetId, newRank / self.outEdges.size)) - else - List() - (new PRVertex(self.id, newRank, self.outEdges, !halt), msgsOut) -} -{% endhighlight %} - -val result = Bagel.run(sc, verts, emptyMsgs)()(compute) - -Finally, we print the results. - -{% highlight scala %} -println(result.map(v => "%s\t%s\n".format(v.id, v.rank)).collect.mkString) -{% endhighlight %} - -## Combiners - -Sending a message to another vertex generally involves expensive communication over the network. For certain algorithms, it's possible to reduce the amount of communication using _combiners_. For example, if the compute function receives integer messages and only uses their sum, it's possible for Bagel to combine multiple messages to the same vertex by summing them. - -For combiner support, Bagel can optionally take a set of combiner functions that convert messages to their combined form. - -_Example: PageRank with combiners_ - -## Aggregators - -Aggregators perform a reduce across all vertices after each superstep, and provide the result to each vertex in the next superstep. - -For aggregator support, Bagel can optionally take an aggregator function that reduces across each vertex. - -_Example_ - -## Operations - -Here are the actions and types in the Bagel API. See [Bagel.scala](https://github.com/apache/spark/blob/master/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala) for details. - -### Actions - -{% highlight scala %} -/*** Full form ***/ - -Bagel.run(sc, vertices, messages, combiner, aggregator, partitioner, numSplits)(compute) -// where compute takes (vertex: V, combinedMessages: Option[C], aggregated: Option[A], superstep: Int) -// and returns (newVertex: V, outMessages: Array[M]) - -/*** Abbreviated forms ***/ - -Bagel.run(sc, vertices, messages, combiner, partitioner, numSplits)(compute) -// where compute takes (vertex: V, combinedMessages: Option[C], superstep: Int) -// and returns (newVertex: V, outMessages: Array[M]) - -Bagel.run(sc, vertices, messages, combiner, numSplits)(compute) -// where compute takes (vertex: V, combinedMessages: Option[C], superstep: Int) -// and returns (newVertex: V, outMessages: Array[M]) - -Bagel.run(sc, vertices, messages, numSplits)(compute) -// where compute takes (vertex: V, messages: Option[Array[M]], superstep: Int) -// and returns (newVertex: V, outMessages: Array[M]) -{% endhighlight %} - -### Types - -{% highlight scala %} -trait Combiner[M, C] { - def createCombiner(msg: M): C - def mergeMsg(combiner: C, msg: M): C - def mergeCombiners(a: C, b: C): C -} - -trait Aggregator[V, A] { - def createAggregator(vert: V): A - def mergeAggregators(a: A, b: A): A -} - -trait Vertex { - def active: Boolean -} - -trait Message[K] { - def targetId: K -} -{% endhighlight %} diff --git a/docs/building-spark.md b/docs/building-spark.md index 4f73adb85446..57baa503259c 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -7,166 +7,120 @@ redirect_from: "building-with-maven.html" * This will become a table of contents (this text will be scraped). {:toc} -Building Spark using Maven requires Maven 3.3.3 or newer and Java 7+. -The Spark build can supply a suitable Maven binary; see below. +# Building Apache Spark -# Building with `build/mvn` +## Apache Maven -Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: - -{% highlight bash %} -build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package -{% endhighlight %} - -Other build examples can be found below. - -**Note:** When building on an encrypted filesystem (if your home directory is encrypted, for example), then the Spark build might fail with a "Filename too long" error. As a workaround, add the following in the configuration args of the `scala-maven-plugin` in the project `pom.xml`: - - -Xmax-classfile-name - 128 - -and in `project/SparkBuild.scala` add: - - scalacOptions in Compile ++= Seq("-Xmax-classfile-name", "128"), - -to the `sharedSettings` val. See also [this PR](https://github.com/apache/spark/pull/2883/files) if you are unsure of where to add these lines. - -# Building a Runnable Distribution - -To create a Spark distribution like those distributed by the -[Spark Downloads](http://spark.apache.org/downloads.html) page, and that is laid out so as -to be runnable, use `make-distribution.sh` in the project root directory. It can be configured -with Maven profile settings and so on like the direct Maven build. Example: - - ./make-distribution.sh --name custom-spark --tgz -Psparkr -Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn - -For more information on usage, run `./make-distribution.sh --help` - -# Setting up Maven's Memory Usage +The Maven-based build is the build of reference for Apache Spark. +Building Spark using Maven requires Maven 3.3.9 or newer and Java 8+. +Note that support for Java 7 was removed as of Spark 2.2.0. -You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`. We recommend the following settings: +### Setting up Maven's Memory Usage -{% highlight bash %} -export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" -{% endhighlight %} +You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`: -If you don't run this, you may see errors like the following: + export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m" - [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_BINARY_VERSION}}/classes... - [ERROR] PermGen space -> [Help 1] +(The `ReservedCodeCacheSize` setting is optional but recommended.) +If you don't add these parameters to `MAVEN_OPTS`, you may see errors and warnings like the following: [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_BINARY_VERSION}}/classes... [ERROR] Java heap space -> [Help 1] -You can fix this by setting the `MAVEN_OPTS` variable as discussed before. +You can fix these problems by setting the `MAVEN_OPTS` variable as discussed before. **Note:** -* For Java 8 and above this step is not required. -* If using `build/mvn` with no `MAVEN_OPTS` set, the script will automate this for you. +* If using `build/mvn` with no `MAVEN_OPTS` set, the script will automatically add the above options to the `MAVEN_OPTS` environment variable. +* The `test` phase of the Spark build will automatically add these options to `MAVEN_OPTS`, even when not using `build/mvn`. -# Specifying the Hadoop Version +### build/mvn -Because HDFS is not protocol-compatible across versions, if you want to read from HDFS, you'll need to build Spark against the specific HDFS version in your environment. You can do this through the `hadoop.version` property. If unset, Spark will build against Hadoop 2.2.0 by default. Note that certain build profiles are required for particular Hadoop versions: +Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: - - - - - - - - - - - -
    Hadoop versionProfile required
    1.x to 2.1.xhadoop-1
    2.2.xhadoop-2.2
    2.3.xhadoop-2.3
    2.4.xhadoop-2.4
    2.6.x and later 2.xhadoop-2.6
    + ./build/mvn -DskipTests clean package -For Apache Hadoop versions 1.x, Cloudera CDH "mr1" distributions, and other Hadoop versions without YARN, use: +Other build examples can be found below. -{% highlight bash %} -# Apache Hadoop 1.2.1 -mvn -Dhadoop.version=1.2.1 -Phadoop-1 -DskipTests clean package +## Building a Runnable Distribution -# Cloudera CDH 4.2.0 with MapReduce v1 -mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -Phadoop-1 -DskipTests clean package -{% endhighlight %} +To create a Spark distribution like those distributed by the +[Spark Downloads](http://spark.apache.org/downloads.html) page, and that is laid out so as +to be runnable, use `./dev/make-distribution.sh` in the project root directory. It can be configured +with Maven profile settings and so on like the direct Maven build. Example: -You can enable the `yarn` profile and optionally set the `yarn.version` property if it is different from `hadoop.version`. Spark only supports YARN versions 2.2.0 and later. + ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr -Phadoop-2.7 -Phive -Phive-thriftserver -Pmesos -Pyarn -Examples: +This will build Spark distribution along with Python pip and R packages. For more information on usage, run `./dev/make-distribution.sh --help` -{% highlight bash %} +## Specifying the Hadoop Version and Enabling YARN -# Apache Hadoop 2.2.X -mvn -Pyarn -Phadoop-2.2 -DskipTests clean package +You can specify the exact version of Hadoop to compile against through the `hadoop.version` property. +If unset, Spark will build against Hadoop 2.6.X by default. -# Apache Hadoop 2.3.X -mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -DskipTests clean package +You can enable the `yarn` profile and optionally set the `yarn.version` property if it is different +from `hadoop.version`. + +Examples: -# Apache Hadoop 2.4.X or 2.5.X -mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=VERSION -DskipTests clean package + # Apache Hadoop 2.6.X + ./build/mvn -Pyarn -DskipTests clean package -Versions of Hadoop after 2.5.X may or may not work with the -Phadoop-2.4 profile (they were -released after this version of Spark). + # Apache Hadoop 2.7.X and later + ./build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=2.7.3 -DskipTests clean package -# Different versions of HDFS and YARN. -mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=2.2.0 -DskipTests clean package -{% endhighlight %} +## Building With Hive and JDBC Support -# Building With Hive and JDBC Support To enable Hive integration for Spark SQL along with its JDBC server and CLI, add the `-Phive` and `Phive-thriftserver` profiles to your existing build options. -By default Spark will build with Hive 0.13.1 bindings. -{% highlight bash %} -# Apache Hadoop 2.4.X with Hive 13 support -mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-thriftserver -DskipTests clean package -{% endhighlight %} +By default Spark will build with Hive 1.2.1 bindings. -# Building for Scala 2.11 -To produce a Spark package compiled with Scala 2.11, use the `-Dscala-2.11` property: + # With Hive 1.2.1 support + ./build/mvn -Pyarn -Phive -Phive-thriftserver -DskipTests clean package - ./dev/change-scala-version.sh 2.11 - mvn -Pyarn -Phadoop-2.4 -Dscala-2.11 -DskipTests clean package +## Packaging without Hadoop Dependencies for YARN -Spark does not yet support its JDBC component for Scala 2.11. +The assembly directory produced by `mvn package` will, by default, include all of Spark's +dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this +causes multiple versions of these to appear on executor classpaths: the version packaged in +the Spark assembly and the version on each node, included with `yarn.application.classpath`. +The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, +like ZooKeeper and Hadoop itself. -# Spark Tests in Maven +## Building with Mesos support -Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). - -Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: + ./build/mvn -Pmesos -DskipTests clean package + +## Building with Kafka 0.8 support - mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive -Phive-thriftserver clean package - mvn -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test +Kafka 0.8 support must be explicitly enabled with the `kafka-0-8` profile. +Note: Kafka 0.8 support is deprecated as of Spark 2.3.0. -The ScalaTest plugin also supports running only a specific test suite as follows: + ./build/mvn -Pkafka-0-8 -DskipTests clean package - mvn -Dhadoop.version=... -DwildcardSuites=org.apache.spark.repl.ReplSuite test +Kafka 0.10 support is still automatically built. -# Building submodules individually +## Building submodules individually It's possible to build Spark sub-modules using the `mvn -pl` option. For instance, you can build the Spark Streaming module using: -{% highlight bash %} -mvn -pl :spark-streaming_2.10 clean install -{% endhighlight %} + ./build/mvn -pl :spark-streaming_2.11 clean install -where `spark-streaming_2.10` is the `artifactId` as defined in `streaming/pom.xml` file. +where `spark-streaming_2.11` is the `artifactId` as defined in `streaming/pom.xml` file. -# Continuous Compilation +## Continuous Compilation We use the scala-maven-plugin which supports incremental and continuous compilation. E.g. - mvn scala:cc + ./build/mvn scala:cc should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. A couple of gotchas to note: * it only scans the paths `src/main` and `src/test` (see -[docs](http://scala-tools.org/mvnsites/maven-scala-plugin/usage_cc.html)), so it will only work +[docs](http://davidb.github.io/scala-maven-plugin/example_cc.html)), so it will only work from within certain submodules that have that structure. * you'll typically need to run `mvn install` from the project root for compilation within @@ -175,77 +129,114 @@ the `spark-parent` module). Thus, the full flow for running continuous-compilation of the `core` submodule may look more like: - $ mvn install + $ ./build/mvn install $ cd core - $ mvn scala:cc + $ ../build/mvn scala:cc -# Building Spark with IntelliJ IDEA or Eclipse +## Building with SBT + +Maven is the official build tool recommended for packaging Spark, and is the *build of reference*. +But SBT is supported for day-to-day development since it can provide much faster iterative +compilation. More advanced developers may wish to use SBT. + +The SBT build is derived from the Maven POM files, and so the same Maven profiles and variables +can be set to control the SBT build. For example: + + ./build/sbt package + +To avoid the overhead of launching sbt each time you need to re-compile, you can launch sbt +in interactive mode by running `build/sbt`, and then run all build commands at the command +prompt. + +## Speeding up Compilation + +Developers who compile Spark frequently may want to speed up compilation; e.g., by using Zinc +(for developers who build with Maven) or by avoiding re-compilation of the assembly JAR (for +developers who build with SBT). For more information about how to do this, refer to the +[Useful Developer Tools page](http://spark.apache.org/developer-tools.html#reducing-build-times). + +## Encrypted Filesystems + +When building on an encrypted filesystem (if your home directory is encrypted, for example), then the Spark build might fail with a "Filename too long" error. As a workaround, add the following in the configuration args of the `scala-maven-plugin` in the project `pom.xml`: + + -Xmax-classfile-name + 128 + +and in `project/SparkBuild.scala` add: + + scalacOptions in Compile ++= Seq("-Xmax-classfile-name", "128"), + +to the `sharedSettings` val. See also [this PR](https://github.com/apache/spark/pull/2883/files) if you are unsure of where to add these lines. + +## IntelliJ IDEA or Eclipse 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/Useful+Developer+Tools#UsefulDeveloperTools-IDESetup). +[Useful Developer Tools page](http://spark.apache.org/developer-tools.html). -# Running Java 8 Test Suites -Running only Java 8 tests and nothing else. +# Running Tests - mvn install -DskipTests -Pjava8-tests +Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). +Note that tests should not be run as root or an admin user. -Java 8 tests are run when `-Pjava8-tests` profile is enabled, they will run in spite of `-DskipTests`. -For these tests to run your system must have a JDK 8 installation. -If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. +The following is an example of a command to run the tests: -# Building for PySpark on YARN + ./build/mvn test -PySpark on YARN is only supported if the jar is built with Maven. Further, there is a known problem -with building this assembly jar on Red Hat based operating systems (see [SPARK-1753](https://issues.apache.org/jira/browse/SPARK-1753)). If you wish to -run PySpark on a YARN cluster with Red Hat installed, we recommend that you build the jar elsewhere, -then ship it over to the cluster. We are investigating the exact cause for this. +## Testing with SBT -# Packaging without Hadoop Dependencies for YARN +The following is an example of a command to run the tests: -The assembly jar produced by `mvn package` will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with `yarn.application.classpath`. The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. + ./build/sbt test -# Building with SBT +## Running Individual Tests -Maven is the official build tool recommended for packaging Spark, and is the *build of reference*. -But SBT is supported for day-to-day development since it can provide much faster iterative -compilation. More advanced developers may wish to use SBT. +For information about how to run individual tests, refer to the +[Useful Developer Tools page](http://spark.apache.org/developer-tools.html#running-individual-tests). -The SBT build is derived from the Maven POM files, and so the same Maven profiles and variables -can be set to control the SBT build. For example: +## PySpark pip installable - build/sbt -Pyarn -Phadoop-2.3 assembly +If you are building Spark for use in a Python environment and you wish to pip install it, you will first need to build the Spark JARs as described above. Then you can construct an sdist package suitable for setup.py and pip installable package. -To avoid the overhead of launching sbt each time you need to re-compile, you can launch sbt -in interactive mode by running `build/sbt`, and then run all build commands at the command -prompt. For more recommendations on reducing build time, refer to the -[wiki page](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-ReducingBuildTimes). + cd python; python setup.py sdist + +**Note:** Due to packaging requirements you can not directly pip install from the Python directory, rather you must first build the sdist package as described above. + +Alternatively, you can also run make-distribution with the --pip option. + +## PySpark Tests with Maven + +If you are building PySpark and wish to run the PySpark tests you will need to build Spark with Hive support. + + ./build/mvn -DskipTests clean package -Phive + ./python/run-tests + +The run-tests script also can be limited to a specific Python version or a specific module + + ./python/run-tests --python-executables=python --modules=pyspark-sql + +**Note:** You can also run Python tests with an sbt build, provided you build Spark with Hive support. -# Testing with SBT +## Running R Tests -Some of the tests require Spark to be packaged first, so always run `build/sbt assembly` the first time. The following is an example of a correct (build, test) sequence: +To run the SparkR tests you will need to install the [knitr](https://cran.r-project.org/package=knitr), [rmarkdown](https://cran.r-project.org/package=rmarkdown), [testthat](https://cran.r-project.org/package=testthat), [e1071](https://cran.r-project.org/package=e1071) and [survival](https://cran.r-project.org/package=survival) packages first: - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver assembly - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test + R -e "install.packages(c('knitr', 'rmarkdown', 'testthat', 'e1071', 'survival'), repos='http://cran.us.r-project.org')" -To run only a specific test suite as follows: +You can run just the SparkR tests using the command: - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.ReplSuite" + ./R/run-tests.sh -To run test suites of a specific sub project as follows: +## Running Docker-based Integration Test Suites - build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver core/test +In order to run Docker integration tests, you have to install the `docker` engine on your box. +The instructions for installation can be found at [the Docker site](https://docs.docker.com/engine/installation/). +Once installed, the `docker` service needs to be started, if not already running. +On Linux, this can be done by `sudo service docker start`. -# Speeding up Compilation with Zinc + ./build/mvn install -DskipTests + ./build/mvn test -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.11 -[Zinc](https://github.com/typesafehub/zinc) is a long-running server version of SBT's incremental -compiler. When run locally as a background process, it speeds up builds of Scala-based projects -like Spark. Developers who regularly recompile Spark with Maven will be the most interested in -Zinc. The project site gives instructions for building and running `zinc`; OS X users can -install it using `brew install zinc`. +or -If using the `build/mvn` package `zinc` will automatically be downloaded and leveraged for all -builds. This process will auto-start after the first time `build/mvn` is called and bind to port -3030 unless the `ZINC_PORT` environment variable is set. The `zinc` process can subsequently be -shut down at any time by running `build/zinc-/bin/zinc -shutdown` and will automatically -restart whenever `build/mvn` is called. + ./build/sbt docker-integration-tests/test diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md new file mode 100644 index 000000000000..751a192da4ff --- /dev/null +++ b/docs/cloud-integration.md @@ -0,0 +1,200 @@ +--- +layout: global +displayTitle: Integration with Cloud Infrastructures +title: Integration with Cloud Infrastructures +description: Introduction to cloud storage support in Apache Spark SPARK_VERSION_SHORT +--- + + +* This will become a table of contents (this text will be scraped). +{:toc} + +## Introduction + + +All major cloud providers offer persistent data storage in *object stores*. +These are not classic "POSIX" file systems. +In order to store hundreds of petabytes of data without any single points of failure, +object stores replace the classic filesystem directory tree +with a simpler model of `object-name => data`. To enable remote access, operations +on objects are usually offered as (slow) HTTP REST operations. + +Spark can read and write data in object stores through filesystem connectors implemented +in Hadoop or provided by the infrastructure suppliers themselves. +These connectors make the object stores look *almost* like filesystems, with directories and files +and the classic operations on them such as list, delete and rename. + + +### Important: Cloud Object Stores are Not Real Filesystems + +While the stores appear to be filesystems, underneath +they are still object stores, [and the difference is significant](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/filesystem/introduction.html) + +They cannot be used as a direct replacement for a cluster filesystem such as HDFS +*except where this is explicitly stated*. + +Key differences are: + +* Changes to stored objects may not be immediately visible, both in directory listings and actual data access. +* The means by which directories are emulated may make working with them slow. +* Rename operations may be very slow and, on failure, leave the store in an unknown state. +* Seeking within a file may require new HTTP calls, hurting performance. + +How does this affect Spark? + +1. Reading and writing data can be significantly slower than working with a normal filesystem. +1. Some directory structures may be very inefficient to scan during query split calculation. +1. The output of work may not be immediately visible to a follow-on query. +1. The rename-based algorithm by which Spark normally commits work when saving an RDD, DataFrame or Dataset + is potentially both slow and unreliable. + +For these reasons, it is not always safe to use an object store as a direct destination of queries, or as +an intermediate store in a chain of queries. Consult the documentation of the object store and its +connector to determine which uses are considered safe. + +In particular: *without some form of consistency layer, Amazon S3 cannot +be safely used as the direct destination of work with the normal rename-based committer.* + +### Installation + +With the relevant libraries on the classpath and Spark configured with valid credentials, +objects can be can be read or written by using their URLs as the path to data. +For example `sparkContext.textFile("s3a://landsat-pds/scene_list.gz")` will create +an RDD of the file `scene_list.gz` stored in S3, using the s3a connector. + +To add the relevant libraries to an application's classpath, include the `hadoop-cloud` +module and its dependencies. + +In Maven, add the following to the `pom.xml` file, assuming `spark.version` +is set to the chosen version of Spark: + +{% highlight xml %} + + ... + + org.apache.spark + hadoop-cloud_2.11 + ${spark.version} + + ... + +{% endhighlight %} + +Commercial products based on Apache Spark generally directly set up the classpath +for talking to cloud infrastructures, in which case this module may not be needed. + +### Authenticating + +Spark jobs must authenticate with the object stores to access data within them. + +1. When Spark is running in a cloud infrastructure, the credentials are usually automatically set up. +1. `spark-submit` reads the `AWS_ACCESS_KEY`, `AWS_SECRET_KEY` +and `AWS_SESSION_TOKEN` environment variables and sets the associated authentication options +for the `s3n` and `s3a` connectors to Amazon S3. +1. In a Hadoop cluster, settings may be set in the `core-site.xml` file. +1. Authentication details may be manually added to the Spark configuration in `spark-default.conf` +1. Alternatively, they can be programmatically set in the `SparkConf` instance used to configure +the application's `SparkContext`. + +*Important: never check authentication secrets into source code repositories, +especially public ones* + +Consult [the Hadoop documentation](https://hadoop.apache.org/docs/current/) for the relevant +configuration and security options. + +## Configuring + +Each cloud connector has its own set of configuration parameters, again, +consult the relevant documentation. + +### Recommended settings for writing to object stores + +For object stores whose consistency model means that rename-based commits are safe +use the `FileOutputCommitter` v2 algorithm for performance: + +``` +spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version 2 +``` + +This does less renaming at the end of a job than the "version 1" algorithm. +As it still uses `rename()` to commit files, it is unsafe to use +when the object store does not have consistent metadata/listings. + +The committer can also be set to ignore failures when cleaning up temporary +files; this reduces the risk that a transient network problem is escalated into a +job failure: + +``` +spark.hadoop.mapreduce.fileoutputcommitter.cleanup-failures.ignored true +``` + +As storing temporary files can run up charges; delete +directories called `"_temporary"` on a regular basis to avoid this. + +### Parquet I/O Settings + +For optimal performance when working with Parquet data use the following settings: + +``` +spark.hadoop.parquet.enable.summary-metadata false +spark.sql.parquet.mergeSchema false +spark.sql.parquet.filterPushdown true +spark.sql.hive.metastorePartitionPruning true +``` + +These minimise the amount of data read during queries. + +### ORC I/O Settings + +For best performance when working with ORC data, use these settings: + +``` +spark.sql.orc.filterPushdown true +spark.sql.orc.splits.include.file.footer true +spark.sql.orc.cache.stripe.details.size 10000 +spark.sql.hive.metastorePartitionPruning true +``` + +Again, these minimise the amount of data read during queries. + +## Spark Streaming and Object Storage + +Spark Streaming can monitor files added to object stores, by +creating a `FileInputDStream` to monitor a path in the store through a call to +`StreamingContext.textFileStream()`. + +1. The time to scan for new files is proportional to the number of files +under the path, not the number of *new* files, so it can become a slow operation. +The size of the window needs to be set to handle this. + +1. Files only appear in an object store once they are completely written; there +is no need for a worklow of write-then-rename to ensure that files aren't picked up +while they are still being written. Applications can write straight to the monitored directory. + +1. Streams should only be checkpointed to an store implementing a fast and +atomic `rename()` operation Otherwise the checkpointing may be slow and potentially unreliable. + +## Further Reading + +Here is the documentation on the standard connectors both from Apache and the cloud providers. + +* [OpenStack Swift](https://hadoop.apache.org/docs/current/hadoop-openstack/index.html). Hadoop 2.6+ +* [Azure Blob Storage](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). Since Hadoop 2.7 +* [Azure Data Lake](https://hadoop.apache.org/docs/current/hadoop-azure-datalake/index.html). Since Hadoop 2.8 +* [Amazon S3 via S3A and S3N](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). Hadoop 2.6+ +* [Amazon EMR File System (EMRFS)](https://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-fs.html). From Amazon +* [Google Cloud Storage Connector for Spark and Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector). From Google + + diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index faaf154d243f..a2ad958959a5 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -35,7 +35,7 @@ There are several useful things to note about this architecture: processes, and these communicate with each other, it is relatively easy to run it even on a cluster manager that also supports other applications (e.g. Mesos/YARN). 3. The driver program must listen for and accept incoming connections from its executors throughout - its lifetime (e.g., see [spark.driver.port and spark.fileserver.port in the network config + its lifetime (e.g., see [spark.driver.port in the network config section](configuration.html#networking)). As such, the driver program must be network addressable from the worker nodes. 4. Because the driver schedules tasks on the cluster, it should be run close to the worker @@ -52,9 +52,11 @@ The system currently supports three cluster managers: * [Apache Mesos](running-on-mesos.html) -- a general cluster manager that can also run Hadoop MapReduce and service applications. * [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2. - -In addition, Spark's [EC2 launch scripts](ec2-scripts.html) make it easy to launch a standalone -cluster on Amazon EC2. +* [Kubernetes (experimental)](https://github.com/apache-spark-on-k8s/spark) -- In addition to the above, +there is experimental support for Kubernetes. Kubernetes is an open-source platform +for providing container-centric infrastructure. Kubernetes support is being actively +developed in an [apache-spark-on-k8s](https://github.com/apache-spark-on-k8s/) Github organization. +For documentation, refer to that project's README. # Submitting Applications diff --git a/docs/configuration.md b/docs/configuration.md index c276e8e90dec..6e9fe591b70a 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -35,7 +35,7 @@ val sc = new SparkContext(conf) {% endhighlight %} Note that we can have more than 1 thread in local mode, and in cases like Spark Streaming, we may -actually require one to prevent any sort of starvation issues. +actually require more than 1 thread to prevent any sort of starvation issues. Properties that specify some time duration should be configured with a unit of time. The following format is accepted: @@ -48,7 +48,7 @@ The following format is accepted: 1y (years) -Properties that specify a byte size should be configured with a unit of size. +Properties that specify a byte size should be configured with a unit of size. The following format is accepted: 1b (bytes) @@ -59,6 +59,7 @@ The following format is accepted: 1p or 1pb (pebibytes = 1024 tebibytes) ## Dynamically Loading Spark Properties + In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For instance, if you'd like to run the same application with different masters or different amounts of memory. Spark allows you to simply create an empty conf: @@ -94,6 +95,13 @@ in the `spark-defaults.conf` file. A few configuration keys have been renamed si versions of Spark; in such cases, the older key names are still accepted, but take lower precedence than any instance of the newer key. +Spark properties mainly can be divided into two kinds: one is related to deploy, like +"spark.driver.memory", "spark.executor.instances", this kind of properties may not be affected when +setting programmatically through `SparkConf` in runtime, or the behavior is depending on which +cluster manager and deploy mode you choose, so it would be suggested to set through configuration +file or `spark-submit` command line options; another is mainly related to Spark runtime control, +like "spark.task.maxFailures", this kind of properties can be set in either way. + ## Viewing Spark Properties The application web UI at `http://:4040` lists Spark properties in the "Environment" tab. @@ -106,7 +114,8 @@ line will appear. For all other configuration properties, you can assume the def Most of the properties that control internal settings have reasonable default values. Some of the most common options to set are: -#### Application Properties +### Application Properties + @@ -123,6 +132,7 @@ of the most common options to set are: Number of cores to use for the driver process, only in cluster mode. + @@ -192,11 +202,38 @@ of the most common options to set are: allowed master URL's. + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.driver.maxResultSize 1g @@ -173,7 +183,7 @@ of the most common options to set are: stored on disk. This should be on a fast, local disk in your system. It can also be a comma-separated list of multiple directories on different disks. - NOTE: In Spark 1.0 and later this will be overriden by SPARK_LOCAL_DIRS (Standalone, Mesos) or + NOTE: In Spark 1.0 and later this will be overridden by SPARK_LOCAL_DIRS (Standalone, Mesos) or LOCAL_DIRS (YARN) environment variables set by the cluster manager.
    spark.submit.deployMode(none) + The deploy mode of Spark driver program, either "client" or "cluster", + Which means to launch driver program locally ("client") + or remotely ("cluster") on one of the nodes inside the cluster. +
    spark.log.callerContext(none) + Application information that will be written into Yarn RM log/HDFS audit log when running on Yarn/HDFS. + Its length depends on the Hadoop configuration hadoop.caller.context.max.size. It should be concise, + and typically can have up to 50 characters. +
    spark.driver.supervisefalse + If true, restarts the driver automatically if it fails with a non-zero exit status. + Only has effect in Spark standalone mode or Mesos cluster deploy mode. +
    Apart from these, the following properties are also available, and may be useful in some situations: -#### Runtime Environment +### Runtime Environment + @@ -208,7 +245,7 @@ Apart from these, the following properties are also available, and may be useful
    Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. Instead, please set this through the --driver-class-path command line option or in - your default properties file. + your default properties file. @@ -216,11 +253,14 @@ Apart from these, the following properties are also available, and may be useful + your default properties file. @@ -232,7 +272,7 @@ Apart from these, the following properties are also available, and may be useful
    Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. Instead, please set this through the --driver-library-path command line option or in - your default properties file. + your default properties file. @@ -240,7 +280,7 @@ Apart from these, the following properties are also available, and may be useful @@ -280,11 +320,19 @@ Apart from these, the following properties are also available, and may be useful Older log files will be deleted. Disabled by default. + + + + + @@ -296,7 +344,7 @@ Apart from these, the following properties are also available, and may be useful Set the strategy of rolling of executor logs. By default it is disabled. It can be set to "time" (time-based rolling) or "size" (size-based rolling). For "time", use spark.executor.logs.rolling.time.interval to set the rolling interval. - For "size", use spark.executor.logs.rolling.size.maxBytes to set + For "size", use spark.executor.logs.rolling.maxSize to set the maximum file size for rolling. @@ -305,7 +353,7 @@ Apart from these, the following properties are also available, and may be useful @@ -326,17 +374,26 @@ Apart from these, the following properties are also available, and may be useful process. The user can specify multiple of these to set multiple environment variables. + + + + + @@ -364,13 +421,99 @@ Apart from these, the following properties are also available, and may be useful + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    (none) A string of extra JVM options to pass to the driver. For instance, GC settings or other logging. + Note that it is illegal to set maximum heap size (-Xmx) settings with this option. Maximum heap + size settings can be set with spark.driver.memory in the cluster mode and through + the --driver-memory command line option in the client mode.
    Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. Instead, please set this through the --driver-java-options command line option or in - your default properties file.
    false (Experimental) Whether to give user-added jars precedence over Spark's own jars when loading - classes in the the driver. This feature can be used to mitigate conflicts between Spark's + classes in the driver. This feature can be used to mitigate conflicts between Spark's dependencies and user dependencies. It is currently an experimental feature. This is used in cluster mode only. @@ -260,9 +300,9 @@ Apart from these, the following properties are also available, and may be useful (none) A string of extra JVM options to pass to executors. For instance, GC settings or other logging. - Note that it is illegal to set Spark properties or heap size settings with this option. Spark - properties should be set using a SparkConf object or the spark-defaults.conf file used with the - spark-submit script. Heap size settings can be set with spark.executor.memory. + Note that it is illegal to set Spark properties or maximum heap size (-Xmx) settings with this + option. Spark properties should be set using a SparkConf object or the spark-defaults.conf file + used with the spark-submit script. Maximum heap size settings can be set with spark.executor.memory.
    spark.executor.logs.rolling.enableCompressionfalse + Enable executor log compression. If it is enabled, the rolled executor logs will be compressed. + Disabled by default. +
    spark.executor.logs.rolling.maxSize (none) - Set the max size of the file by which the executor logs will be rolled over. + Set the max size of the file in bytes by which the executor logs will be rolled over. Rolling is disabled by default. See spark.executor.logs.rolling.maxRetainedFiles for automatic cleaning of old logs.
    daily Set the time interval by which the executor logs will be rolled over. - Rolling is disabled by default. Valid values are daily, hourly, minutely or + Rolling is disabled by default. Valid values are daily, hourly, minutely or any interval in seconds. See spark.executor.logs.rolling.maxRetainedFiles for automatic cleaning of old logs.
    spark.redaction.regex(?i)secret|password + Regex to decide which Spark configuration properties and environment variables in driver and + executor environments contain sensitive information. When this regex matches a property key or + value, the value is redacted from the environment UI and various logs like YARN and event logs. +
    spark.python.profile false - Enable profiling in Python worker, the profile result will show up by sc.show_profiles(), + Enable profiling in Python worker, the profile result will show up by sc.show_profiles(), or it will be displayed before the driver exiting. It also can be dumped into disk by - sc.dump_profiles(path). If some of the profile results had been displayed manually, + sc.dump_profiles(path). If some of the profile results had been displayed manually, they will not be displayed automatically before driver exiting. - By default the pyspark.profiler.BasicProfiler will be used, but this can be overridden by - passing a profiler class in as a parameter to the SparkContext constructor. + By default the pyspark.profiler.BasicProfiler will be used, but this can be overridden by + passing a profiler class in as a parameter to the SparkContext constructor.
    Reuse Python worker or not. If yes, it will use a fixed number of Python workers, does not need to fork() a Python process for every tasks. It will be very useful - if there is large broadcast, then the broadcast will not be needed to transfered + if there is large broadcast, then the broadcast will not be needed to transferred from JVM to Python worker for every task.
    spark.files + Comma-separated list of files to be placed in the working directory of each executor. Globs are allowed. +
    spark.submit.pyFiles + Comma-separated list of .zip, .egg, or .py files to place on the PYTHONPATH for Python apps. Globs are allowed. +
    spark.jars + Comma-separated list of jars to include on the driver and executor classpaths. Globs are allowed. +
    spark.jars.packages + Comma-separated list of Maven coordinates of jars to include on the driver and executor + classpaths. The coordinates should be groupId:artifactId:version. If spark.jars.ivySettings + is given artifacts will be resolved according to the configuration in the file, otherwise artifacts + will be searched for in the local maven repo, then maven central and finally any additional remote + repositories given by the command-line option --repositories. For more details, see + Advanced Dependency Management. +
    spark.jars.excludes + Comma-separated list of groupId:artifactId, to exclude while resolving the dependencies + provided in spark.jars.packages to avoid dependency conflicts. +
    spark.jars.ivy + Path to specify the Ivy user directory, used for the local Ivy cache and package files from + spark.jars.packages. This will override the Ivy property ivy.default.ivy.user.dir + which defaults to ~/.ivy2. +
    spark.jars.ivySettings + Path to an Ivy settings file to customize resolution of jars specified using spark.jars.packages + instead of the built-in defaults, such as maven central. Additional repositories given by the command-line + option --repositories or spark.jars.repositories will also be included. + Useful for allowing Spark to resolve artifacts from behind a firewall e.g. via an in-house + artifact server like Artifactory. Details on the settings file format can be + found at http://ant.apache.org/ivy/history/latest-milestone/settings.html +
    spark.jars.repositories + Comma-separated list of additional remote repositories to search for the maven coordinates + given with --packages or spark.jars.packages. +
    spark.pyspark.driver.python + Python binary executable to use for PySpark in driver. + (default is spark.pyspark.python) +
    spark.pyspark.python + Python binary executable to use for PySpark in both driver and executors. +
    -#### Shuffle Behavior +### Shuffle Behavior + @@ -382,6 +525,37 @@ Apart from these, the following properties are also available, and may be useful overhead per reduce task, so keep it small unless you have a large amount of memory. + + + + + + + + + + + + + + + @@ -433,15 +607,6 @@ Apart from these, the following properties are also available, and may be useful is 15 seconds by default, calculated as maxRetries * retryWait. - - - - - @@ -461,6 +626,24 @@ Apart from these, the following properties are also available, and may be useful Port on which the external shuffle service will run. + + + + + + + + + + @@ -477,9 +660,57 @@ Apart from these, the following properties are also available, and may be useful spark.io.compression.codec. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.reducer.maxReqsInFlightInt.MaxValue + This configuration limits the number of remote requests to fetch blocks at any given point. + When the number of hosts in the cluster increase, it might lead to very large number + of in-bound connections to one or more nodes, causing the workers to fail under load. + By allowing it to limit the number of fetch requests, this scenario can be mitigated. +
    spark.reducer.maxBlocksInFlightPerAddressInt.MaxValue + This configuration limits the number of remote blocks being fetched per reduce task from a + given host port. When a large number of blocks are being requested from a given address in a + single fetch or simultaneously, this could crash the serving executor or Node Manager. This + is especially useful to reduce the load on the Node Manager when external shuffle is enabled. + You can mitigate this issue by setting it to a lower value. +
    spark.reducer.maxReqSizeShuffleToMemLong.MaxValue + The blocks of a shuffle request will be fetched to disk when size of the request is above + this threshold. This is to avoid a giant request takes too much memory. We can enable this + config by setting a specific value(e.g. 200m). Note that this config can be enabled only when + the shuffle shuffle service is newer than Spark-2.2 or the shuffle service is disabled. +
    spark.shuffle.compress true
    spark.shuffle.managersort - Implementation to use for shuffling data. There are two implementations available: - sort and hash. - Sort-based shuffle is more memory-efficient and is the default option starting in 1.2. -
    spark.shuffle.service.enabled false
    spark.shuffle.service.index.cache.size100m + Cache entries limited to the specified memory footprint. +
    spark.shuffle.maxChunksBeingTransferredLong.MAX_VALUE + The max number of chunks allowed to be transferred at the same time on shuffle service. + Note that new incoming connections will be closed when the max number is hit. The client will + retry according to the shuffle retry configs (see spark.shuffle.io.maxRetries and + spark.shuffle.io.retryWait), if those limits are reached the task will fail with + fetch failure. +
    spark.shuffle.sort.bypassMergeThreshold 200
    spark.shuffle.accurateBlockThreshold100 * 1024 * 1024 + When we compress the size of shuffle blocks in HighlyCompressedMapStatus, we will record the + size accurately if it's above this config. This helps to prevent OOM by avoiding + underestimating shuffle block size when fetch shuffle blocks. +
    spark.shuffle.registration.timeout5000 + Timeout in milliseconds for registration to the external shuffle service. +
    spark.shuffle.registration.maxAttempts3 + When we fail to register to the external shuffle service, we will retry for maxAttempts times. +
    spark.io.encryption.enabledfalse + Enable IO encryption. Currently supported by all modes except Mesos. It's recommended that RPC encryption + be enabled when using this feature. +
    spark.io.encryption.keySizeBits128 + IO encryption key size in bits. Supported values are 128, 192 and 256. +
    spark.io.encryption.keygen.algorithmHmacSHA1 + The algorithm to use when generating the IO encryption key. The supported algorithms are + described in the KeyGenerator section of the Java Cryptography Architecture Standard Algorithm + Name Documentation. +
    -#### Spark UI +### Spark UI + @@ -487,6 +718,7 @@ Apart from these, the following properties are also available, and may be useful @@ -507,11 +739,18 @@ Apart from these, the following properties are also available, and may be useful finished. + + + + + @@ -525,16 +764,47 @@ Apart from these, the following properties are also available, and may be useful + + + + + + + + + + + + + + + + + + + + @@ -565,9 +835,17 @@ Apart from these, the following properties are also available, and may be useful How many finished batches the Spark UI and status APIs remember before garbage collecting. + + + + +
    Property NameDefaultMeaning
    false Whether to compress logged events, if spark.eventLog.enabled is true. + Compression will use spark.io.compression.codec.
    spark.ui.enabledtrue + Whether to run the web UI for the Spark application. +
    spark.ui.killEnabled true - Allows stages and corresponding jobs to be killed from the web ui. + Allows jobs and stages to be killed from the web UI.
    spark.ui.retainedJobs 1000 - How many jobs the Spark UI and status APIs remember before garbage - collecting. + How many jobs the Spark UI and status APIs remember before garbage collecting. + This is a target maximum, and fewer elements may be retained in some circumstances.
    spark.ui.retainedStages 1000 - How many stages the Spark UI and status APIs remember before garbage - collecting. + How many stages the Spark UI and status APIs remember before garbage collecting. + This is a target maximum, and fewer elements may be retained in some circumstances. +
    spark.ui.retainedTasks100000 + How many tasks the Spark UI and status APIs remember before garbage collecting. + This is a target maximum, and fewer elements may be retained in some circumstances. +
    spark.ui.reverseProxyfalse + Enable running Spark Master as reverse proxy for worker and application UIs. In this mode, Spark master will reverse proxy the worker and application UIs to enable access without requiring direct access to their hosts. Use it with caution, as worker and application UI will not be accessible directly, you will only be able to access them through spark master/proxy public URL. This setting affects all the workers and application UIs running in the cluster and must be set on all the workers, drivers and masters. +
    spark.ui.reverseProxyUrl + This is the URL where your proxy is running. This URL is for proxy which is running in front of Spark Master. This is useful when running proxy for authentication e.g. OAuth proxy. Make sure this is a complete URL including scheme (http/https) and port to reach your proxy. +
    spark.ui.showConsoleProgresstrue + Show the progress bar in the console. The progress bar shows the progress of stages + that run for longer than 500ms. If multiple stages run at the same time, multiple + progress bars will be displayed on the same line.
    spark.ui.retainedDeadExecutors100 + How many dead executors the Spark UI and status APIs remember before garbage collecting. +
    -#### Compression and Serialization +### Compression and Serialization + @@ -575,21 +853,15 @@ Apart from these, the following properties are also available, and may be useful - - - - - - + - + + + + + + @@ -679,15 +959,16 @@ Apart from these, the following properties are also available, and may be useful
    Property NameDefaultMeaning
    true Whether to compress broadcast variables before sending them. Generally a good idea. -
    spark.closure.serializerorg.apache.spark.serializer.
    JavaSerializer
    - Serializer class to use for closures. Currently only the Java serializer is supported. + Compression will use spark.io.compression.codec.
    spark.io.compression.codecsnappylz4 - The codec used to compress internal data such as RDD partitions, broadcast variables and - shuffle outputs. By default, Spark provides three codecs: lz4, lzf, + The codec used to compress internal data such as RDD partitions, event log, broadcast variables + and shuffle outputs. By default, Spark provides three codecs: lz4, lzf, and snappy. You can also use fully qualified class names to specify the codec, e.g. org.apache.spark.io.LZ4CompressionCodec, @@ -624,7 +896,7 @@ Apart from these, the following properties are also available, and may be useful
    spark.kryo.referenceTrackingtrue (false when using Spark SQL Thrift Server)true Whether to track references to the same object when serializing data with Kryo, which is necessary if your object graphs have loops and useful for efficiency if they contain multiple @@ -647,22 +919,30 @@ Apart from these, the following properties are also available, and may be useful spark.kryo.registrator (none) - If you use Kryo serialization, set this class to register your custom classes with Kryo. This + If you use Kryo serialization, give a comma-separated list of classes that register your custom classes with Kryo. This property is useful if you need to register your classes in a custom way, e.g. to specify a custom field serializer. Otherwise spark.kryo.classesToRegister is simpler. It should be - set to a class that extends + set to classes that extend KryoRegistrator. See the tuning guide for more details.
    spark.kryo.unsafefalse + Whether to use unsafe based Kryo serializer. Can be + substantially faster by using Unsafe Based IO. +
    spark.kryoserializer.buffer.max 64m Maximum allowable size of Kryo serialization buffer. This must be larger than any - object you attempt to serialize. Increase this if you get a "buffer limit exceeded" exception - inside Kryo. + object you attempt to serialize and must be less than 2048m. + Increase this if you get a "buffer limit exceeded" exception inside Kryo.
    false Whether to compress serialized RDD partitions (e.g. for - StorageLevel.MEMORY_ONLY_SER). Can save substantial space at the cost of some - extra CPU time. + StorageLevel.MEMORY_ONLY_SER in Java + and Scala or StorageLevel.MEMORY_ONLY in Python). + Can save substantial space at the cost of some extra CPU time. + Compression will use spark.io.compression.codec.
    spark.serializer - org.apache.spark.serializer.
    JavaSerializer (org.apache.spark.serializer.
    - KryoSerializer when using Spark SQL Thrift Server) + org.apache.spark.serializer.
    JavaSerializer
    Class to use for serializing objects that will be sent over the network or need to be cached @@ -712,27 +993,48 @@ Apart from these, the following properties are also available, and may be useful
    -#### Memory Management +### Memory Management + - + + + + + + + + + + + @@ -780,9 +1082,19 @@ Apart from these, the following properties are also available, and may be useful storage space to unroll the new block in its entirety. + + + + +
    Property NameDefaultMeaning
    spark.memory.fraction0.750.6 - Fraction of the heap space used for execution and storage. The lower this is, the more - frequently spills and cached data eviction occur. The purpose of this config is to set + Fraction of (heap space - 300MB) used for execution and storage. The lower this is, the + more frequently spills and cached data eviction occur. The purpose of this config is to set aside memory for internal metadata, user data structures, and imprecise size estimation - in the case of sparse, unusually large records. + in the case of sparse, unusually large records. Leaving this at the default value is + recommended. For more detail, including important information about correctly tuning JVM + garbage collection when increasing this value, see + this description.
    spark.memory.storageFraction 0.5 - T​he size of the storage region within the space set aside by - s​park.memory.fraction. This region is not statically reserved, but dynamically - allocated as cache requests come in. ​Cached data may be evicted only if total storage exceeds - this region. + Amount of storage memory immune to eviction, expressed as a fraction of the size of the + region set aside by s​park.memory.fraction. The higher this is, the less + working memory may be available to execution and tasks may spill to disk more often. + Leaving this at the default value is recommended. For more detail, see + this description. +
    spark.memory.offHeap.enabledfalse + If true, Spark will attempt to use off-heap memory for certain operations. If off-heap memory use is enabled, then spark.memory.offHeap.size must be positive. +
    spark.memory.offHeap.size0 + The absolute amount of memory in bytes which can be used for off-heap allocation. + This setting has no impact on heap memory usage, so if your executors' total memory consumption must fit within some hard limit then be sure to shrink your JVM heap size accordingly. + This must be set to a positive value when spark.memory.offHeap.enabled=true.
    spark.storage.replication.proactivefalse + Enables proactive block replication for RDD blocks. Cached RDD block replicas lost due to + executor failures are replenished if there are any existing available replicas. This tries + to get the replication level of the block to the initial number. +
    -#### Execution Behavior +### Execution Behavior + @@ -795,32 +1107,19 @@ Apart from these, the following properties are also available, and may be useful - - - - - - - + - - - - @@ -845,7 +1144,8 @@ Apart from these, the following properties are also available, and may be useful + tasks. spark.executor.heartbeatInterval should be significantly less than + spark.network.timeout @@ -875,6 +1175,22 @@ Apart from these, the following properties are also available, and may be useful its contents do not match those of the source. + + + + + + + + + + @@ -904,103 +1220,62 @@ Apart from these, the following properties are also available, and may be useful - - - - - - - - - - - - + +
    Property NameDefaultMeaning
    spark.broadcast.factoryorg.apache.spark.broadcast.
    TorrentBroadcastFactory
    - Which broadcast implementation to use. -
    spark.cleaner.ttl(infinite)spark.executor.cores - Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks - generated, etc.). Periodic cleanups will ensure that metadata older than this duration will be - forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in - case of Spark Streaming applications). Note that any RDD that persists in memory for more than - this duration will be cleared as well. + 1 in YARN mode, all the available cores on the worker in + standalone and Mesos coarse-grained modes.
    spark.executor.cores1 in YARN mode, all the available cores on the worker in standalone mode. - The number of cores to use on each executor. For YARN and standalone mode only. + The number of cores to use on each executor. - In standalone mode, setting this parameter allows an application to run multiple executors on - the same worker, provided that there are enough cores on that worker. Otherwise, only one - executor per application will run on each worker. + In standalone and Mesos coarse-grained modes, setting this + parameter allows an application to run multiple executors on the + same worker, provided that there are enough cores on that + worker. Otherwise, only one executor per application will run on + each worker.
    10s Interval between each executor's heartbeats to the driver. Heartbeats let the driver know that the executor is still alive and update it with metrics for in-progress - tasks.
    spark.files.fetchTimeout
    spark.files.maxPartitionBytes134217728 (128 MB) + The maximum number of bytes to pack into a single partition when reading files. +
    spark.files.openCostInBytes4194304 (4 MB) + The estimated cost to open a file, measured by the number of bytes could be scanned in the same + time. This is used when putting multiple files into a partition. It is better to over estimate, + then the partitions with small files will be faster than partitions with bigger files. +
    spark.hadoop.cloneConf false
    spark.externalBlockStore.blockManagerorg.apache.spark.storage.TachyonBlockManager - Implementation of external block manager (file system) that store RDDs. The file system's URL is set by - spark.externalBlockStore.url. -
    spark.externalBlockStore.baseDirSystem.getProperty("java.io.tmpdir") - Directories of the external block store that store RDDs. The file system's URL is set by - spark.externalBlockStore.url It can also be a comma-separated list of multiple - directories on Tachyon file system. -
    spark.externalBlockStore.urltachyon://localhost:19998 for Tachyonspark.hadoop.mapreduce.fileoutputcommitter.algorithm.version1 - The URL of the underlying external blocker file system in the external block store. + The file output committer algorithm version, valid algorithm version number: 1 or 2. + Version 2 may have better performance, but version 1 may handle failures better in certain situations, + as per MAPREDUCE-4815.
    -#### Networking +### Networking + - + - - - - - - - - - - - - - - - - - + + - - + + - - + + @@ -1012,26 +1287,12 @@ Apart from these, the following properties are also available, and may be useful This is used for communicating with the executors and the standalone Master. - - - - - - - - - - - - - - - @@ -1073,7 +1326,7 @@ Apart from these, the following properties are also available, and may be useful - + @@ -1082,12 +1335,13 @@ Apart from these, the following properties are also available, and may be useful
    Property NameDefaultMeaning
    spark.akka.frameSizespark.rpc.message.maxSize 128 Maximum message size (in MB) to allow in "control plane" communication; generally only applies to map output size information sent between executors and the driver. Increase this if you are running - jobs with many thousands of map and reduce tasks and see messages about the frame size. -
    spark.akka.heartbeat.interval1000s - This is set to a larger value to disable the transport failure detector that comes built in to - Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger - interval value reduces network overhead and a smaller value ( ~ 1 s) might be more - informative for Akka's failure detector. Tune this in combination of spark.akka.heartbeat.pauses - if you need to. A likely positive use case for using failure detector would be: a sensistive - failure detector can help evict rogue executors quickly. However this is usually not the case - as GC pauses and network lags are expected in a real Spark cluster. Apart from that enabling - this leads to a lot of exchanges of heart beats between nodes leading to flooding the network - with those. -
    spark.akka.heartbeat.pauses6000s - This is set to a larger value to disable the transport failure detector that comes built in to Akka. - It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart - beat pause for Akka. This can be used to control sensitivity to GC pauses. Tune - this along with spark.akka.heartbeat.interval if you need to. -
    spark.akka.threads4 - Number of actor threads to use for communication. Can be useful to increase on large clusters - when the driver has a lot of CPU cores. + jobs with many thousands of map and reduce tasks and see messages about the RPC message size.
    spark.akka.timeout100sspark.blockManager.port(random) - Communication timeout between Spark nodes. + Port for all block managers to listen on. These exist on both the driver and the executors.
    spark.blockManager.port(random)spark.driver.blockManager.port(value of spark.blockManager.port) - Port for all block managers to listen on. These exist on both the driver and the executors. + Driver-specific port for the block manager to listen on, for cases where it cannot use the same + configuration as executors.
    spark.broadcast.port(random)spark.driver.bindAddress(value of spark.driver.host) - Port for the driver's HTTP broadcast server to listen on. - This is not relevant for torrent broadcast. + Hostname or IP address where to bind listening sockets. This config overrides the SPARK_LOCAL_IP + environment variable (see below). + +
    It also allows a different address from the local one to be advertised to executors or external systems. + This is useful, for example, when running containers with bridged networking. For this to properly work, + the different ports used by the driver (RPC, block manager and UI) need to be forwarded from the + container's host.
    spark.driver.host (local hostname) - Hostname or IP address for the driver to listen on. + Hostname or IP address for the driver. This is used for communicating with the executors and the standalone Master.
    spark.executor.port(random) - Port for the executor to listen on. This is used for communicating with the driver. -
    spark.fileserver.port(random) - Port for the driver's HTTP file server to listen on. -
    spark.network.timeout 120s Default timeout for all network interactions. This config will be used in place of - spark.core.connection.ack.wait.timeout, spark.akka.timeout, + spark.core.connection.ack.wait.timeout, spark.storage.blockManagerSlaveTimeoutMs, spark.shuffle.io.connectionTimeout, spark.rpc.askTimeout or spark.rpc.lookupTimeout if they are not configured. @@ -1048,14 +1309,6 @@ Apart from these, the following properties are also available, and may be useful to port + maxRetries.
    spark.replClassServer.port(random) - Port for the driver's HTTP class server to listen on. - This is only relevant for the Spark shell. -
    spark.rpc.numRetries 3
    spark.rpc.askTimeout120sspark.network.timeout Duration for an RPC ask operation to wait before timing out. spark.rpc.lookupTimeout 120s - Duration for an RPC remote endpoint lookup operation to wait before timing out. + Duration for an RPC remote endpoint lookup operation to wait before timing out.
    -#### Scheduling +### Scheduling + @@ -1174,6 +1428,106 @@ Apart from these, the following properties are also available, and may be useful The interval length for the scheduler to revive the worker resource offers to run tasks. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @@ -1200,7 +1554,7 @@ Apart from these, the following properties are also available, and may be useful @@ -1214,13 +1568,65 @@ Apart from these, the following properties are also available, and may be useful + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.scheduler.listenerbus.eventqueue.capacity10000 + Capacity for event queue in Spark listener bus, must be greater than 0. Consider increasing + value (e.g. 20000) if listener events are dropped. Increasing this value may result in the + driver using more memory. +
    spark.blacklist.enabled + false + + If set to "true", prevent Spark from scheduling tasks on executors that have been blacklisted + due to too many task failures. The blacklisting algorithm can be further controlled by the + other "spark.blacklist" configuration options. +
    spark.blacklist.timeout1h + (Experimental) How long a node or executor is blacklisted for the entire application, before it + is unconditionally removed from the blacklist to attempt running new tasks. +
    spark.blacklist.task.maxTaskAttemptsPerExecutor1 + (Experimental) For a given task, how many times it can be retried on one executor before the + executor is blacklisted for that task. +
    spark.blacklist.task.maxTaskAttemptsPerNode2 + (Experimental) For a given task, how many times it can be retried on one node, before the entire + node is blacklisted for that task. +
    spark.blacklist.stage.maxFailedTasksPerExecutor2 + (Experimental) How many different tasks must fail on one executor, within one stage, before the + executor is blacklisted for that stage. +
    spark.blacklist.stage.maxFailedExecutorsPerNode2 + (Experimental) How many different executors are marked as blacklisted for a given stage, before + the entire node is marked as failed for the stage. +
    spark.blacklist.application.maxFailedTasksPerExecutor2 + (Experimental) How many different tasks must fail on one executor, in successful task sets, + before the executor is blacklisted for the entire application. Blacklisted executors will + be automatically added back to the pool of available resources after the timeout specified by + spark.blacklist.timeout. Note that with dynamic allocation, though, the executors + may get marked as idle and be reclaimed by the cluster manager. +
    spark.blacklist.application.maxFailedExecutorsPerNode2 + (Experimental) How many different executors must be blacklisted for the entire application, + before the node is blacklisted for the entire application. Blacklisted nodes will + be automatically added back to the pool of available resources after the timeout specified by + spark.blacklist.timeout. Note that with dynamic allocation, though, the executors + on the node may get marked as idle and be reclaimed by the cluster manager. +
    spark.blacklist.killBlacklistedExecutorsfalse + (Experimental) If set to "true", allow Spark to automatically kill, and attempt to re-create, + executors when they are blacklisted. Note that, when an entire node is added to the blacklist, + all of the executors on that node will be killed. +
    spark.blacklist.application.fetchFailure.enabledfalse + (Experimental) If set to "true", Spark will blacklist the executor immediately when a fetch + failure happenes. If external shuffle service is enabled, then the whole node will be + blacklisted. +
    spark.speculation falsespark.speculation.quantile 0.75 - Percentage of tasks which must be complete before speculation is enabled for a particular stage. + Fraction of tasks which must be complete before speculation is enabled for a particular stage.
    spark.task.maxFailures 4 - Number of individual task failures before giving up on the job. + Number of failures of any particular task before giving up on the job. + The total number of failures spread across different tasks will not cause the job + to fail; a particular task has to fail this number of attempts. Should be greater than or equal to 1. Number of allowed retries = this value - 1.
    spark.task.reaper.enabledfalse + Enables monitoring of killed / interrupted tasks. When set to true, any task which is killed + will be monitored by the executor until that task actually finishes executing. See the other + spark.task.reaper.* configurations for details on how to control the exact behavior + of this monitoring. When set to false (the default), task killing will use an older code + path which lacks such monitoring. +
    spark.task.reaper.pollingInterval10s + When spark.task.reaper.enabled = true, this setting controls the frequency at which + executors will poll the status of killed tasks. If a killed task is still running when polled + then a warning will be logged and, by default, a thread-dump of the task will be logged + (this thread dump can be disabled via the spark.task.reaper.threadDump setting, + which is documented below). +
    spark.task.reaper.threadDumptrue + When spark.task.reaper.enabled = true, this setting controls whether task thread + dumps are logged during periodic polling of killed tasks. Set this to false to disable + collection of thread dumps. +
    spark.task.reaper.killTimeout-1 + When spark.task.reaper.enabled = true, this setting specifies a timeout after + which the executor JVM will kill itself if a killed task has not stopped running. The default + value, -1, disables this mechanism and prevents the executor from self-destructing. The purpose + of this setting is to act as a safety-net to prevent runaway uncancellable tasks from rendering + an executor unusable. +
    spark.stage.maxConsecutiveAttempts4 + Number of consecutive stage attempts allowed before a stage is aborted. +
    -#### Dynamic Allocation +### Dynamic Allocation + @@ -1228,8 +1634,8 @@ Apart from these, the following properties are also available, and may be useful @@ -1298,14 +1707,15 @@ Apart from these, the following properties are also available, and may be useful
    Property NameDefaultMeaning
    false Whether to use dynamic resource allocation, which scales the number of executors registered - with this application up and down based on the workload. Note that this is currently only - available on YARN mode. For more detail, see the description + with this application up and down based on the workload. + For more detail, see the description here.

    This requires spark.shuffle.service.enabled to be set. @@ -1262,6 +1668,9 @@ Apart from these, the following properties are also available, and may be useful
    spark.dynamicAllocation.minExecutors Initial number of executors to run if dynamic allocation is enabled. +

    + If `--num-executors` (or `spark.executor.instances`) is set and larger than this value, it will + be used as the initial number of executors.
    -#### Security +### Security + + + + + + + + + + + @@ -1337,39 +1772,73 @@ Apart from these, the following properties are also available, and may be useful not running on YARN and authentication is enabled. + + + + + + + + + + + + + + + + + + + + + + + + + - + - - - - - @@ -1379,6 +1848,18 @@ Apart from these, the following properties are also available, and may be useful the list means any user can have access to modify it. + + + + + @@ -1402,9 +1883,21 @@ Apart from these, the following properties are also available, and may be useful have view access to this Spark job. + + + + +
    Property NameDefaultMeaning
    spark.acls.enable false - Whether Spark acls should are enabled. If enabled, this checks to see if the user has + Whether Spark acls should be enabled. If enabled, this checks to see if the user has access permissions to view or modify the job. Note this requires the user to be known, so if the user comes across as null no checks are done. Filters can be used with the UI to authenticate and set the user. @@ -1317,8 +1727,33 @@ Apart from these, the following properties are also available, and may be useful Comma separated list of users/administrators that have view and modify access to all Spark jobs. This can be used if you run on a shared cluster and have a set of administrators or devs who - help debug when things work. Putting a "*" in the list means any user can have the priviledge - of admin. + help debug when things do not work. Putting a "*" in the list means any user can have the + privilege of admin. +
    spark.admin.acls.groupsEmpty + Comma separated list of groups that have view and modify access to all Spark jobs. + This can be used if you have a set of administrators or developers who help maintain and debug + the underlying infrastructure. Putting a "*" in the list means any user in any group can have + the privilege of admin. The user groups are obtained from the instance of the groups mapping + provider specified by spark.user.groups.mapping. Check the entry + spark.user.groups.mapping for more details. +
    spark.user.groups.mappingorg.apache.spark.security.ShellBasedGroupsMappingProvider + The list of groups for a user are determined by a group mapping service defined by the trait + org.apache.spark.security.GroupMappingServiceProvider which can configured by this property. + A default unix shell based implementation is provided org.apache.spark.security.ShellBasedGroupsMappingProvider + which can be specified to resolve a list of groups for a user. + Note: This implementation supports only a Unix/Linux based environment. Windows environment is + currently not supported. However, a new platform/protocol can be supported by implementing + the trait org.apache.spark.security.GroupMappingServiceProvider.
    spark.network.crypto.enabledfalse + Enable encryption using the commons-crypto library for RPC and block transfer service. + Requires spark.authenticate to be enabled. +
    spark.network.crypto.keyLength128 + The length in bits of the encryption key to generate. Valid values are 128, 192 and 256. +
    spark.network.crypto.keyFactoryAlgorithmPBKDF2WithHmacSHA1 + The key factory algorithm to use when generating encryption keys. Should be one of the + algorithms supported by the javax.crypto.SecretKeyFactory class in the JRE being used. +
    spark.network.crypto.saslFallbacktrue + Whether to fall back to SASL authentication if authentication fails using Spark's internal + mechanism. This is useful when the application is connecting to old shuffle services that + do not support the internal Spark authentication protocol. On the server side, this can be + used to block older clients from authenticating against a new shuffle service. +
    spark.network.crypto.config.*None + Configuration values for the commons-crypto library, such as which cipher implementations to + use. The config name should be the name of commons-crypto configuration without the + "commons.crypto" prefix. +
    spark.authenticate.enableSaslEncryption false - Enable encrypted communication when authentication is enabled. This option is currently - only supported by the block transfer service. + Enable encrypted communication when authentication is + enabled. This is supported by the block transfer service and the + RPC endpoints.
    spark.network.sasl.serverAlwaysEncrypt false - Disable unencrypted connections for services that support SASL authentication. This is - currently supported by the external shuffle service. + Disable unencrypted connections for services that support SASL authentication.
    spark.core.connection.ack.wait.timeout60sspark.network.timeout How long for the connection to wait for ack to occur before timing out and giving up. To avoid unwilling timeout caused by long pause like GC, you can set larger value.
    spark.core.connection.auth.wait.timeout30s - How long for the connection to wait for authentication to occur before timing - out and giving up. -
    spark.modify.acls Empty
    spark.modify.acls.groupsEmpty + Comma separated list of groups that have modify access to the Spark job. This can be used if you + have a set of administrators or developers from the same team to have access to control the job. + Putting a "*" in the list means any user in any group has the access to modify the Spark job. + The user groups are obtained from the instance of the groups mapping provider specified by + spark.user.groups.mapping. Check the entry spark.user.groups.mapping + for more details. +
    spark.ui.filters None
    spark.ui.view.acls.groupsEmpty + Comma separated list of groups that have view access to the Spark web ui to view the Spark Job + details. This can be used if you have a set of administrators or developers or users who can + monitor the Spark job submitted. Putting a "*" in the list means any user in any group can view + the Spark job details on the Spark web ui. The user groups are obtained from the instance of the + groups mapping provider specified by spark.user.groups.mapping. Check the entry + spark.user.groups.mapping for more details. +
    -#### Encryption +### TLS / SSL @@ -1412,17 +1905,35 @@ Apart from these, the following properties are also available, and may be useful + + + + + @@ -1433,6 +1944,7 @@ Apart from these, the following properties are also available, and may be useful The reference list of protocols one can find on this page. + Note: If not set, it will use the default cipher suites of JVM. @@ -1457,6 +1969,13 @@ Apart from these, the following properties are also available, and may be useful A password to the key-store. + + + + + @@ -1466,6 +1985,13 @@ Apart from these, the following properties are also available, and may be useful page. + + + + + @@ -1481,10 +2007,61 @@ Apart from these, the following properties are also available, and may be useful A password to the trust-store. + + + + +
    Property NameDefaultMeaning
    spark.ssl.enabled false -

    Whether to enable SSL connections on all supported protocols.

    + Whether to enable SSL connections on all supported protocols. + +
    When spark.ssl.enabled is configured, spark.ssl.protocol + is required. -

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

    + the properties must be overwritten in the protocol-specific namespace. -

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

    +
    Use spark.ssl.YYY.XXX settings to overwrite the global configuration for + particular protocol denoted by YYY. Example values for YYY + include fs, ui, standalone, and + historyServer. See SSL + Configuration for details on hierarchical SSL configuration for services. +
    spark.ssl.[namespace].portNone + The port where the SSL service will listen on. + +
    The port must be defined within a namespace configuration; see + SSL Configuration for the available + namespaces. + +
    When not set, the SSL port will be derived from the non-SSL port for the + same service. A value of "0" will make the service bind to an ephemeral port.
    spark.ssl.keyStoreTypeJKS + The type of the key-store. +
    spark.ssl.protocol None
    spark.ssl.needClientAuthfalse + Set true if SSL needs client authentication. +
    spark.ssl.trustStore None
    spark.ssl.trustStoreTypeJKS + The type of the trust-store. +
    -#### Spark Streaming +### Spark SQL + +Running the SET -v command will show the entire list of the SQL configuration. + +
    +
    + +{% highlight scala %} +// spark is an existing SparkSession +spark.sql("SET -v").show(numRows = 200, truncate = false) +{% endhighlight %} + +
    + +
    + +{% highlight java %} +// spark is an existing SparkSession +spark.sql("SET -v").show(200, false); +{% endhighlight %} +
    + +
    + +{% highlight python %} +# spark is an existing SparkSession +spark.sql("SET -v").show(n=200, truncate=False) +{% endhighlight %} + +
    + +
    + +{% highlight r %} +sparkR.session() +properties <- sql("SET -v") +showDF(properties, numRows = 200, truncate = FALSE) +{% endhighlight %} + +
    +
    + + +### Spark Streaming + @@ -1500,6 +2077,14 @@ Apart from these, the following properties are also available, and may be useful if they are set (see below). + + + + + @@ -1546,7 +2131,7 @@ Apart from these, the following properties are also available, and may be useful @@ -1577,9 +2162,28 @@ Apart from these, the following properties are also available, and may be useful How many batches the Spark Streaming UI and status APIs remember before garbage collecting. + + + + + + + + + +
    Property NameDefaultMeaning
    spark.streaming.backpressure.initialRatenot set + This is the initial maximum receiving rate at which each receiver will receive data for the + first batch when the backpressure mechanism is enabled. +
    spark.streaming.blockInterval 200msspark.streaming.stopGracefullyOnShutdown false - If true, Spark shuts down the StreamingContext gracefully on JVM + If true, Spark shuts down the StreamingContext gracefully on JVM shutdown rather than immediately.
    spark.streaming.driver.writeAheadLog.closeFileAfterWritefalse + Whether to close the file after writing a write ahead log record on the driver. Set this to 'true' + when you want to use S3 (or any file system that does not support flushing) for the metadata WAL + on the driver. +
    spark.streaming.receiver.writeAheadLog.closeFileAfterWritefalse + Whether to close the file after writing a write ahead log record on the receivers. Set this to 'true' + when you want to use S3 (or any file system that does not support flushing) for the data WAL + on the receivers. +
    -#### SparkR +### SparkR + @@ -1603,17 +2207,78 @@ Apart from these, the following properties are also available, and may be useful Executable for executing R scripts in client modes for driver. Ignored in cluster modes. + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.r.shell.commandR + Executable for executing sparkR shell in client modes for driver. Ignored in cluster modes. It is the same as environment variable SPARKR_DRIVER_R, but take precedence over it. + spark.r.shell.command is used for sparkR shell while spark.r.driver.command is used for running R script. +
    spark.r.backendConnectionTimeout6000 + Connection timeout set by R process on its connection to RBackend in seconds. +
    spark.r.heartBeatInterval100 + Interval for heartbeats sent from SparkR backend to R process to prevent connection timeout. +
    -#### Cluster Managers +### GraphX + + + + + + + + +
    Property NameDefaultMeaning
    spark.graphx.pregel.checkpointInterval-1 + Checkpoint interval for graph and message in Pregel. It used to avoid stackOverflowError due to long lineage chains + after lots of iterations. The checkpoint is disabled by default. +
    + +### Deploy + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.deploy.recoveryModeNONEThe recovery mode setting to recover submitted Spark jobs with cluster mode when it failed and relaunches. + This is only applicable for cluster mode when running with Standalone or Mesos.
    spark.deploy.zookeeper.urlNoneWhen `spark.deploy.recoveryMode` is set to ZOOKEEPER, this configuration is used to set the zookeeper URL to connect to.
    spark.deploy.zookeeper.dirNoneWhen `spark.deploy.recoveryMode` is set to ZOOKEEPER, this configuration is used to set the zookeeper directory to store recovery state.
    + + +### Cluster Managers + Each cluster manager in Spark has additional configuration options. Configurations can be found on the pages for each mode: -##### [YARN](running-on-yarn.html#configuration) +#### [YARN](running-on-yarn.html#configuration) -##### [Mesos](running-on-mesos.html#configuration) +#### [Mesos](running-on-mesos.html#configuration) -##### [Standalone Mode](spark-standalone.html#cluster-launch-scripts) +#### [Standalone Mode](spark-standalone.html#cluster-launch-scripts) # Environment Variables @@ -1636,15 +2301,18 @@ The following variables can be set in `spark-env.sh`: PYSPARK_PYTHON - Python binary executable to use for PySpark in both driver and workers (default is python). + Python binary executable to use for PySpark in both driver and workers (default is python2.7 if available, otherwise python). + Property spark.pyspark.python take precedence if it is set PYSPARK_DRIVER_PYTHON - Python binary executable to use for PySpark in driver only (default is PYSPARK_PYTHON). + Python binary executable to use for PySpark in driver only (default is PYSPARK_PYTHON). + Property spark.pyspark.driver.python take precedence if it is set SPARKR_DRIVER_R - R binary executable to use for SparkR shell (default is R). + R binary executable to use for SparkR shell (default is R). + Property spark.r.shell.command take precedence if it is set SPARK_LOCAL_IP @@ -1663,6 +2331,8 @@ to use on each machine and maximum memory. Since `spark-env.sh` is a shell script, some of these can be set programmatically -- for example, you might compute `SPARK_LOCAL_IP` by looking up the IP of a specific network interface. +Note: When running Spark on YARN in `cluster` mode, environment variables need to be set using the `spark.yarn.appMasterEnv.[EnvironmentVariableName]` property in your `conf/spark-defaults.conf` file. Environment variables that are set in `spark-env.sh` will not be reflected in the YARN Application Master process in `cluster` mode. See the [YARN-related Spark Properties](running-on-yarn.html#spark-properties) for more information. + # Configuring Logging Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can configure it by adding a @@ -1672,7 +2342,7 @@ Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can config # Overriding configuration directory To specify a different configuration directory other than the default "SPARK_HOME/conf", -you can set SPARK_CONF_DIR. Spark will use the the configuration files (spark-defaults.conf, spark-env.sh, log4j.properties, etc) +you can set SPARK_CONF_DIR. Spark will use the configuration files (spark-defaults.conf, spark-env.sh, log4j.properties, etc) from this directory. # Inheriting Hadoop Cluster Configuration @@ -1683,9 +2353,41 @@ should be included on Spark's classpath: * `hdfs-site.xml`, which provides default behaviors for the HDFS client. * `core-site.xml`, which sets the default filesystem name. -The location of these configuration files varies across CDH and HDP versions, but -a common location is inside of `/etc/hadoop/conf`. Some tools, such as Cloudera Manager, create +The location of these configuration files varies across Hadoop versions, but +a common location is inside of `/etc/hadoop/conf`. Some tools create configurations on-the-fly, but offer a mechanisms to download copies of them. -To make these files visible to Spark, set `HADOOP_CONF_DIR` in `$SPARK_HOME/spark-env.sh` +To make these files visible to Spark, set `HADOOP_CONF_DIR` in `$SPARK_HOME/conf/spark-env.sh` to a location containing the configuration files. + +# Custom Hadoop/Hive Configuration + +If your Spark application is interacting with Hadoop, Hive, or both, there are probably Hadoop/Hive +configuration files in Spark's classpath. + +Multiple running applications might require different Hadoop/Hive client side configurations. +You can copy and modify `hdfs-site.xml`, `core-site.xml`, `yarn-site.xml`, `hive-site.xml` in +Spark's classpath for each application. In a Spark cluster running on YARN, these configuration +files are set cluster-wide, and cannot safely be changed by the application. + +The better choice is to use spark hadoop properties in the form of `spark.hadoop.*`. +They can be considered as same as normal spark properties which can be set in `$SPARK_HOME/conf/spark-defalut.conf` + +In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For +instance, Spark allows you to simply create an empty conf and set spark/spark hadoop properties. + +{% highlight scala %} +val conf = new SparkConf().set("spark.hadoop.abc.def","xyz") +val sc = new SparkContext(conf) +{% endhighlight %} + +Also, you can modify or add configurations at runtime: +{% highlight bash %} +./bin/spark-submit \ + --name "My app" \ + --master local[4] \ + --conf spark.eventLog.enabled=false \ + --conf "spark.executor.extraJavaOptions=-XX:+PrintGCDetails -XX:+PrintGCTimeStamps" \ + --conf spark.hadoop.abc.def=xyz \ + myApp.jar +{% endhighlight %} \ No newline at end of file diff --git a/docs/contributing-to-spark.md b/docs/contributing-to-spark.md index ef1b3ad6da57..9252545e4a12 100644 --- a/docs/contributing-to-spark.md +++ b/docs/contributing-to-spark.md @@ -5,4 +5,4 @@ title: Contributing to Spark The Spark team welcomes all forms of contributions, including bug reports, documentation or patches. For the newest information on how to contribute to the project, please read the -[wiki page on contributing to Spark](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark). +[Contributing to Spark guide](http://spark.apache.org/contributing.html). diff --git a/docs/css/main.css b/docs/css/main.css index d770173be101..175e8004fca0 100755 --- a/docs/css/main.css +++ b/docs/css/main.css @@ -39,8 +39,15 @@ margin-left: 10px; } -body #content { - line-height: 1.6; /* Inspired by Github's wiki style */ +body .container-wrapper { + background-color: #FFF; + color: #1D1F22; + max-width: 1024px; + margin-top: 10px; + margin-left: auto; + margin-right: auto; + border-radius: 15px; + position: relative; } .title { @@ -91,6 +98,24 @@ a:hover code { max-width: 914px; } +.content { + z-index: 1; + position: relative; + background-color: #FFF; + max-width: 914px; + line-height: 1.6; /* Inspired by Github's wiki style */ + padding-left: 15px; +} + +.content-with-sidebar { + z-index: 1; + position: relative; + background-color: #FFF; + max-width: 914px; + line-height: 1.6; /* Inspired by Github's wiki style */ + padding-left: 30px; +} + .dropdown-menu { /* Remove the default 2px top margin which causes a small gap between the hover trigger area and the popup menu */ @@ -155,3 +180,110 @@ ul.nav li.dropdown ul.dropdown-menu li.dropdown-submenu ul.dropdown-menu { * AnchorJS (anchor links when hovering over headers) */ a.anchorjs-link:hover { text-decoration: none; } + + +/** + * The left navigation bar. + */ +.left-menu-wrapper { + margin-left: 0px; + margin-right: 0px; + background-color: #F0F8FC; + border-top-width: 0px; + border-left-width: 0px; + border-bottom-width: 0px; + margin-top: 0px; + width: 210px; + float: left; + position: absolute; +} + +.left-menu { + padding: 0px; + width: 199px; +} + +.left-menu h3 { + margin-left: 10px; + line-height: 30px; +} + +/** + * The collapsing button for the navigation bar. + */ +.nav-trigger { + position: fixed; + clip: rect(0, 0, 0, 0); +} + +.nav-trigger + label:after { + content: '»'; +} + +label { + z-index: 10; +} + +label[for="nav-trigger"] { + position: fixed; + margin-left: 0px; + padding-top: 100px; + padding-left: 5px; + width: 10px; + height: 80%; + cursor: pointer; + background-size: contain; + background-color: #D4F0FF; +} + +label[for="nav-trigger"]:hover { + background-color: #BEE9FF; +} + +.nav-trigger:checked + label { + margin-left: 200px; +} + +.nav-trigger:checked + label:after { + content: '«'; +} + +.nav-trigger:checked ~ div.content-with-sidebar { + margin-left: 200px; +} + +.nav-trigger + label, div.content-with-sidebar { + transition: left 0.4s; +} + +/** + * Rules to collapse the menu automatically when the screen becomes too thin. + */ + +@media all and (max-width: 780px) { + + div.content-with-sidebar { + margin-left: 200px; + } + .nav-trigger + label:after { + content: '«'; + } + label[for="nav-trigger"] { + margin-left: 200px; + } + + .nav-trigger:checked + label { + margin-left: 0px; + } + .nav-trigger:checked + label:after { + content: '»'; + } + .nav-trigger:checked ~ div.content-with-sidebar { + margin-left: 0px; + } + + div.container-index { + margin-left: -215px; + } + +} \ No newline at end of file diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md deleted file mode 100644 index 7f60f82b966f..000000000000 --- a/docs/ec2-scripts.md +++ /dev/null @@ -1,192 +0,0 @@ ---- -layout: global -title: Running Spark on EC2 ---- - -The `spark-ec2` script, located in Spark's `ec2` directory, allows you -to launch, manage and shut down Spark clusters on Amazon EC2. It automatically -sets up Spark and HDFS on the cluster for you. This guide describes -how to use `spark-ec2` to launch clusters, how to run jobs on them, and how -to shut them down. It assumes you've already signed up for an EC2 account -on the [Amazon Web Services site](http://aws.amazon.com/). - -`spark-ec2` is designed to manage multiple named clusters. You can -launch a new cluster (telling the script its size and giving it a name), -shutdown an existing cluster, or log into a cluster. Each cluster is -identified by placing its machines into EC2 security groups whose names -are derived from the name of the cluster. For example, a cluster named -`test` will contain a master node in a security group called -`test-master`, and a number of slave nodes in a security group called -`test-slaves`. The `spark-ec2` script will create these security groups -for you based on the cluster name you request. You can also use them to -identify machines belonging to each cluster in the Amazon EC2 Console. - - -# Before You Start - -- Create an Amazon EC2 key pair for yourself. This can be done by - logging into your Amazon Web Services account through the [AWS - console](http://aws.amazon.com/console/), clicking Key Pairs on the - left sidebar, and creating and downloading a key. Make sure that you - set the permissions for the private key file to `600` (i.e. only you - can read and write it) so that `ssh` will work. -- Whenever you want to use the `spark-ec2` script, set the environment - variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` to your - Amazon EC2 access key ID and secret access key. These can be - obtained from the [AWS homepage](http://aws.amazon.com/) by clicking - Account \> Security Credentials \> Access Credentials. - -# Launching a Cluster - -- Go into the `ec2` directory in the release of Spark you downloaded. -- Run - `./spark-ec2 -k -i -s launch `, - where `` is the name of your EC2 key pair (that you gave it - when you created it), `` is the private key file for your - key pair, `` is the number of slave nodes to launch (try - 1 at first), and `` is the name to give to your - cluster. - - For example: - - ```bash - export AWS_SECRET_ACCESS_KEY=AaBbCcDdEeFGgHhIiJjKkLlMmNnOoPpQqRrSsTtU -export AWS_ACCESS_KEY_ID=ABCDEFG1234567890123 -./spark-ec2 --key-pair=awskey --identity-file=awskey.pem --region=us-west-1 --zone=us-west-1a launch my-spark-cluster - ``` - -- After everything launches, check that the cluster scheduler is up and sees - all the slaves by going to its web UI, which will be printed at the end of - the script (typically `http://:8080`). - -You can also run `./spark-ec2 --help` to see more usage options. The -following options are worth pointing out: - -- `--instance-type=` can be used to specify an EC2 -instance type to use. For now, the script only supports 64-bit instance -types, and the default type is `m1.large` (which has 2 cores and 7.5 GB -RAM). Refer to the Amazon pages about [EC2 instance -types](http://aws.amazon.com/ec2/instance-types) and [EC2 -pricing](http://aws.amazon.com/ec2/#pricing) for information about other -instance types. -- `--region=` specifies an EC2 region in which to launch -instances. The default region is `us-east-1`. -- `--zone=` can be used to specify an EC2 availability zone -to launch instances in. Sometimes, you will get an error because there -is not enough capacity in one zone, and you should try to launch in -another. -- `--ebs-vol-size=` will attach an EBS volume with a given amount - of space to each node so that you can have a persistent HDFS cluster - on your nodes across cluster restarts (see below). -- `--spot-price=` will launch the worker nodes as - [Spot Instances](http://aws.amazon.com/ec2/spot-instances/), - bidding for the given maximum price (in dollars). -- `--spark-version=` will pre-load the cluster with the - specified version of Spark. The `` can be a version number - (e.g. "0.7.3") or a specific git hash. By default, a recent - version will be used. -- `--spark-git-repo=` will let you run a custom version of - Spark that is built from the given git repository. By default, the - [Apache Github mirror](https://github.com/apache/spark) will be used. - When using a custom Spark version, `--spark-version` must be set to git - commit hash, such as 317e114, instead of a version number. -- If one of your launches fails due to e.g. not having the right -permissions on your private key file, you can run `launch` with the -`--resume` option to restart the setup process on an existing cluster. - -# Launching a Cluster in a VPC - -- Run - `./spark-ec2 -k -i -s --vpc-id= --subnet-id= launch `, - where `` is the name of your EC2 key pair (that you gave it - when you created it), `` is the private key file for your - key pair, `` is the number of slave nodes to launch (try - 1 at first), `` is the name of your VPC, `` is the - name of your subnet, and `` is the name to give to your - cluster. - - For example: - - ```bash - export AWS_SECRET_ACCESS_KEY=AaBbCcDdEeFGgHhIiJjKkLlMmNnOoPpQqRrSsTtU -export AWS_ACCESS_KEY_ID=ABCDEFG1234567890123 -./spark-ec2 --key-pair=awskey --identity-file=awskey.pem --region=us-west-1 --zone=us-west-1a --vpc-id=vpc-a28d24c7 --subnet-id=subnet-4eb27b39 --spark-version=1.1.0 launch my-spark-cluster - ``` - -# Running Applications - -- Go into the `ec2` directory in the release of Spark you downloaded. -- Run `./spark-ec2 -k -i login ` to - SSH into the cluster, where `` and `` are as - above. (This is just for convenience; you could also use - the EC2 console.) -- To deploy code or data within your cluster, you can log in and use the - provided script `~/spark-ec2/copy-dir`, which, - given a directory path, RSYNCs it to the same location on all the slaves. -- If your application needs to access large datasets, the fastest way to do - that is to load them from Amazon S3 or an Amazon EBS device into an - instance of the Hadoop Distributed File System (HDFS) on your nodes. - The `spark-ec2` script already sets up a HDFS instance for you. It's - installed in `/root/ephemeral-hdfs`, and can be accessed using the - `bin/hadoop` script in that directory. Note that the data in this - HDFS goes away when you stop and restart a machine. -- There is also a *persistent HDFS* instance in - `/root/persistent-hdfs` that will keep data across cluster restarts. - Typically each node has relatively little space of persistent data - (about 3 GB), but you can use the `--ebs-vol-size` option to - `spark-ec2` to attach a persistent EBS volume to each node for - storing the persistent HDFS. -- Finally, if you get errors while running your application, look at the slave's logs - for that application inside of the scheduler work directory (/root/spark/work). You can - also view the status of the cluster using the web UI: `http://:8080`. - -# Configuration - -You can edit `/root/spark/conf/spark-env.sh` on each machine to set Spark configuration options, such -as JVM options. This file needs to be copied to **every machine** to reflect the change. The easiest way to -do this is to use a script we provide called `copy-dir`. First edit your `spark-env.sh` file on the master, -then run `~/spark-ec2/copy-dir /root/spark/conf` to RSYNC it to all the workers. - -The [configuration guide](configuration.html) describes the available configuration options. - -# Terminating a Cluster - -***Note that there is no way to recover data on EC2 nodes after shutting -them down! Make sure you have copied everything important off the nodes -before stopping them.*** - -- Go into the `ec2` directory in the release of Spark you downloaded. -- Run `./spark-ec2 destroy `. - -# Pausing and Restarting Clusters - -The `spark-ec2` script also supports pausing a cluster. In this case, -the VMs are stopped but not terminated, so they -***lose all data on ephemeral disks*** but keep the data in their -root partitions and their `persistent-hdfs`. Stopped machines will not -cost you any EC2 cycles, but ***will*** continue to cost money for EBS -storage. - -- To stop one of your clusters, go into the `ec2` directory and run -`./spark-ec2 --region= stop `. -- To restart it later, run -`./spark-ec2 -i --region= start `. -- To ultimately destroy the cluster and stop consuming EBS space, run -`./spark-ec2 --region= destroy ` as described in the previous -section. - -# Limitations - -- Support for "cluster compute" nodes is limited -- there's no way to specify a - locality group. However, you can launch slave nodes in your - `-slaves` group manually and then use `spark-ec2 launch - --resume` to start a cluster with them. - -If you have a patch or suggestion for one of these limitations, feel free to -[contribute](contributing-to-spark.html) it! - -# Accessing Data in S3 - -Spark's file interface allows it to process data in Amazon S3 using the same URI formats that are supported for Hadoop. You can specify a path in S3 as input through a URI of the form `s3n:///path`. To provide AWS credentials for S3 access, launch the Spark cluster with the option `--copy-aws-credentials`. Full instructions on S3 access using the Hadoop input libraries can be found on the [Hadoop S3 page](http://wiki.apache.org/hadoop/AmazonS3). - -In addition to using a single input file, you can also use a directory of files as input by simply giving the path to the directory. diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 6a512ab234bb..46225dc598da 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -11,6 +11,7 @@ description: GraphX graph processing library guide for Spark SPARK_VERSION_SHORT [EdgeRDD]: api/scala/index.html#org.apache.spark.graphx.EdgeRDD +[VertexRDD]: api/scala/index.html#org.apache.spark.graphx.VertexRDD [Edge]: api/scala/index.html#org.apache.spark.graphx.Edge [EdgeTriplet]: api/scala/index.html#org.apache.spark.graphx.EdgeTriplet [Graph]: api/scala/index.html#org.apache.spark.graphx.Graph @@ -24,10 +25,9 @@ description: GraphX graph processing library guide for Spark SPARK_VERSION_SHORT [Graph.outerJoinVertices]: api/scala/index.html#org.apache.spark.graphx.Graph@outerJoinVertices[U,VD2](RDD[(VertexId,U)])((VertexId,VD,Option[U])⇒VD2)(ClassTag[U],ClassTag[VD2]):Graph[VD2,ED] [Graph.aggregateMessages]: api/scala/index.html#org.apache.spark.graphx.Graph@aggregateMessages[A]((EdgeContext[VD,ED,A])⇒Unit,(A,A)⇒A,TripletFields)(ClassTag[A]):VertexRDD[A] [EdgeContext]: api/scala/index.html#org.apache.spark.graphx.EdgeContext -[Graph.mapReduceTriplets]: api/scala/index.html#org.apache.spark.graphx.Graph@mapReduceTriplets[A](mapFunc:org.apache.spark.graphx.EdgeTriplet[VD,ED]=>Iterator[(org.apache.spark.graphx.VertexId,A)],reduceFunc:(A,A)=>A,activeSetOpt:Option[(org.apache.spark.graphx.VertexRDD[_],org.apache.spark.graphx.EdgeDirection)])(implicitevidence$10:scala.reflect.ClassTag[A]):org.apache.spark.graphx.VertexRDD[A] [GraphOps.collectNeighborIds]: api/scala/index.html#org.apache.spark.graphx.GraphOps@collectNeighborIds(EdgeDirection):VertexRDD[Array[VertexId]] [GraphOps.collectNeighbors]: api/scala/index.html#org.apache.spark.graphx.GraphOps@collectNeighbors(EdgeDirection):VertexRDD[Array[(VertexId,VD)]] -[RDD Persistence]: programming-guide.html#rdd-persistence +[RDD Persistence]: rdd-programming-guide.html#rdd-persistence [Graph.cache]: api/scala/index.html#org.apache.spark.graphx.Graph@cache():Graph[VD,ED] [GraphOps.pregel]: api/scala/index.html#org.apache.spark.graphx.GraphOps@pregel[A](A,Int,EdgeDirection)((VertexId,VD,A)⇒VD,(EdgeTriplet[VD,ED])⇒Iterator[(VertexId,A)],(A,A)⇒A)(ClassTag[A]):Graph[VD,ED] [PartitionStrategy]: api/scala/index.html#org.apache.spark.graphx.PartitionStrategy$ @@ -36,7 +36,6 @@ description: GraphX graph processing library guide for Spark SPARK_VERSION_SHORT [Graph.fromEdgeTuples]: api/scala/index.html#org.apache.spark.graphx.Graph$@fromEdgeTuples[VD](RDD[(VertexId,VertexId)],VD,Option[PartitionStrategy])(ClassTag[VD]):Graph[VD,Int] [Graph.fromEdges]: api/scala/index.html#org.apache.spark.graphx.Graph$@fromEdges[VD,ED](RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED] [PartitionStrategy]: api/scala/index.html#org.apache.spark.graphx.PartitionStrategy -[Graph.partitionBy]: api/scala/index.html#org.apache.spark.graphx.Graph$@partitionBy(partitionStrategy:org.apache.spark.graphx.PartitionStrategy):org.apache.spark.graphx.Graph[VD,ED] [PageRank]: api/scala/index.html#org.apache.spark.graphx.lib.PageRank$ [ConnectedComponents]: api/scala/index.html#org.apache.spark.graphx.lib.ConnectedComponents$ [TriangleCount]: api/scala/index.html#org.apache.spark.graphx.lib.TriangleCount$ @@ -67,23 +66,6 @@ operators (e.g., [subgraph](#structural_operators), [joinVertices](#join_operato [aggregateMessages](#aggregateMessages)) as well as an optimized variant of the [Pregel](#pregel) API. In addition, GraphX includes a growing collection of graph [algorithms](#graph_algorithms) and [builders](#graph_builders) to simplify graph analytics tasks. - -## Migrating from Spark 1.1 - -GraphX in Spark {{site.SPARK_VERSION}} contains a few user facing API changes: - -1. To improve performance we have introduced a new version of -[`mapReduceTriplets`][Graph.mapReduceTriplets] called -[`aggregateMessages`][Graph.aggregateMessages] which takes the messages previously returned from -[`mapReduceTriplets`][Graph.mapReduceTriplets] through a callback ([`EdgeContext`][EdgeContext]) -rather than by return value. -We are deprecating [`mapReduceTriplets`][Graph.mapReduceTriplets] and encourage users to consult -the [transition guide](#mrTripletsTransition). - -2. In Spark 1.0 and 1.1, the type signature of [`EdgeRDD`][EdgeRDD] switched from -`EdgeRDD[ED]` to `EdgeRDD[ED, VD]` to enable some caching optimizations. We have since discovered -a more elegant solution and have restored the type signature to the more natural `EdgeRDD[ED]` type. - # Getting Started To get started you first need to import Spark and GraphX into your project, as follows: @@ -107,7 +89,7 @@ with user defined objects attached to each vertex and edge. A directed multigra graph with potentially multiple parallel edges sharing the same source and destination vertex. The ability to support parallel edges simplifies modeling scenarios where there can be multiple relationships (e.g., co-worker and friend) between the same vertices. Each vertex is keyed by a -*unique* 64-bit long identifier (`VertexID`). GraphX does not impose any ordering constraints on +*unique* 64-bit long identifier (`VertexId`). GraphX does not impose any ordering constraints on the vertex identifiers. Similarly, edges have corresponding source and destination vertex identifiers. @@ -132,7 +114,7 @@ var graph: Graph[VertexProperty, String] = null Like RDDs, property graphs are immutable, distributed, and fault-tolerant. Changes to the values or structure of the graph are accomplished by producing a new graph with the desired changes. Note -that substantial parts of the original graph (i.e., unaffected structure, attributes, and indicies) +that substantial parts of the original graph (i.e., unaffected structure, attributes, and indices) are reused in the new graph reducing the cost of this inherently functional data structure. The graph is partitioned across the executors using a range of vertex partitioning heuristics. As with RDDs, each partition of the graph can be recreated on a different machine in the event of a failure. @@ -148,12 +130,12 @@ class Graph[VD, ED] { } {% endhighlight %} -The classes `VertexRDD[VD]` and `EdgeRDD[ED]` extend and are optimized versions of `RDD[(VertexID, +The classes `VertexRDD[VD]` and `EdgeRDD[ED]` extend and are optimized versions of `RDD[(VertexId, VD)]` and `RDD[Edge[ED]]` respectively. Both `VertexRDD[VD]` and `EdgeRDD[ED]` provide additional functionality built around graph computation and leverage internal optimizations. We discuss the -`VertexRDD` and `EdgeRDD` API in greater detail in the section on [vertex and edge +`VertexRDD`[VertexRDD] and `EdgeRDD`[EdgeRDD] API in greater detail in the section on [vertex and edge RDDs](#vertex_and_edge_rdds) but for now they can be thought of as simply RDDs of the form: -`RDD[(VertexID, VD)]` and `RDD[Edge[ED]]`. +`RDD[(VertexId, VD)]` and `RDD[Edge[ED]]`. ### Example Property Graph @@ -215,7 +197,7 @@ graph.edges.filter(e => e.srcId > e.dstId).count {% endhighlight %} > Note that `graph.vertices` returns an `VertexRDD[(String, String)]` which extends -> `RDD[(VertexID, (String, String))]` and so we use the scala `case` expression to deconstruct the +> `RDD[(VertexId, (String, String))]` and so we use the scala `case` expression to deconstruct the > tuple. On the other hand, `graph.edges` returns an `EdgeRDD` containing `Edge[String]` objects. > We could have also used the case class type constructor as in the following: > {% highlight scala %} @@ -305,7 +287,7 @@ class Graph[VD, ED] { // Change the partitioning heuristic ============================================================ def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] // Transform vertex and edge attributes ========================================================== - def mapVertices[VD2](map: (VertexID, VD) => VD2): Graph[VD2, ED] + def mapVertices[VD2](map: (VertexId, VD) => VD2): Graph[VD2, ED] def mapEdges[ED2](map: Edge[ED] => ED2): Graph[VD, ED2] def mapEdges[ED2](map: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] def mapTriplets[ED2](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] @@ -315,18 +297,18 @@ class Graph[VD, ED] { def reverse: Graph[VD, ED] def subgraph( epred: EdgeTriplet[VD,ED] => Boolean = (x => true), - vpred: (VertexID, VD) => Boolean = ((v, d) => true)) + vpred: (VertexId, VD) => Boolean = ((v, d) => true)) : Graph[VD, ED] def mask[VD2, ED2](other: Graph[VD2, ED2]): Graph[VD, ED] def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] // Join RDDs with the graph ====================================================================== - def joinVertices[U](table: RDD[(VertexID, U)])(mapFunc: (VertexID, VD, U) => VD): Graph[VD, ED] - def outerJoinVertices[U, VD2](other: RDD[(VertexID, U)]) - (mapFunc: (VertexID, VD, Option[U]) => VD2) + def joinVertices[U](table: RDD[(VertexId, U)])(mapFunc: (VertexId, VD, U) => VD): Graph[VD, ED] + def outerJoinVertices[U, VD2](other: RDD[(VertexId, U)]) + (mapFunc: (VertexId, VD, Option[U]) => VD2) : Graph[VD2, ED] // Aggregate information about adjacent triplets ================================================= - def collectNeighborIds(edgeDirection: EdgeDirection): VertexRDD[Array[VertexID]] - def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[Array[(VertexID, VD)]] + def collectNeighborIds(edgeDirection: EdgeDirection): VertexRDD[Array[VertexId]] + def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[Array[(VertexId, VD)]] def aggregateMessages[Msg: ClassTag]( sendMsg: EdgeContext[VD, ED, Msg] => Unit, mergeMsg: (Msg, Msg) => Msg, @@ -334,15 +316,15 @@ class Graph[VD, ED] { : VertexRDD[A] // Iterative graph-parallel computation ========================================================== def pregel[A](initialMsg: A, maxIterations: Int, activeDirection: EdgeDirection)( - vprog: (VertexID, VD, A) => VD, - sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)], + vprog: (VertexId, VD, A) => VD, + sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexId,A)], mergeMsg: (A, A) => A) : Graph[VD, ED] // Basic graph algorithms ======================================================================== def pageRank(tol: Double, resetProb: Double = 0.15): Graph[Double, Double] - def connectedComponents(): Graph[VertexID, ED] + def connectedComponents(): Graph[VertexId, ED] def triangleCount(): Graph[Int, ED] - def stronglyConnectedComponents(numIter: Int): Graph[VertexID, ED] + def stronglyConnectedComponents(numIter: Int): Graph[VertexId, ED] } {% endhighlight %} @@ -438,15 +420,15 @@ val graph = Graph(users, relationships, defaultUser) // Notice that there is a user 0 (for which we have no information) connected to users // 4 (peter) and 5 (franklin). graph.triplets.map( - triplet => triplet.srcAttr._1 + " is the " + triplet.attr + " of " + triplet.dstAttr._1 - ).collect.foreach(println(_)) + triplet => triplet.srcAttr._1 + " is the " + triplet.attr + " of " + triplet.dstAttr._1 +).collect.foreach(println(_)) // Remove missing vertices as well as the edges to connected to them val validGraph = graph.subgraph(vpred = (id, attr) => attr._2 != "Missing") // The valid subgraph will disconnect users 4 and 5 by removing user 0 validGraph.vertices.collect.foreach(println(_)) validGraph.triplets.map( - triplet => triplet.srcAttr._1 + " is the " + triplet.attr + " of " + triplet.dstAttr._1 - ).collect.foreach(println(_)) + triplet => triplet.srcAttr._1 + " is the " + triplet.attr + " of " + triplet.dstAttr._1 +).collect.foreach(println(_)) {% endhighlight %} > Note in the above example only the vertex predicate is provided. The `subgraph` operator defaults @@ -499,7 +481,7 @@ original value. > is therefore recommended that the input RDD be made unique using the following which will > also *pre-index* the resulting values to substantially accelerate the subsequent join. > {% highlight scala %} -val nonUniqueCosts: RDD[(VertexID, Double)] +val nonUniqueCosts: RDD[(VertexId, Double)] val uniqueCosts: VertexRDD[Double] = graph.vertices.aggregateUsingIndex(nonUnique, (a,b) => a + b) val joinedGraph = graph.joinVertices(uniqueCosts)( @@ -529,7 +511,7 @@ val degreeGraph = graph.outerJoinVertices(outDegrees) { (id, oldAttr, outDegOpt) > provide type annotation for the user defined function: > {% highlight scala %} val joinedGraph = graph.joinVertices(uniqueCosts, - (id: VertexID, oldCost: Double, extraCost: Double) => oldCost + extraCost) + (id: VertexId, oldCost: Double, extraCost: Double) => oldCost + extraCost) {% endhighlight %} > @@ -576,7 +558,7 @@ The user defined `mergeMsg` function takes two messages destined to the same ver yields a single message. Think of `mergeMsg` as the reduce function in map-reduce. The [`aggregateMessages`][Graph.aggregateMessages] operator returns a `VertexRDD[Msg]` containing the aggregate message (of type `Msg`) destined to each vertex. Vertices that did not -receive a message are not included in the returned `VertexRDD`. +receive a message are not included in the returned `VertexRDD`[VertexRDD]. + +More details on parameters can be found in the [Python API documentation](api/python/pyspark.ml.html#pyspark.ml.regression.LinearRegression). + +{% include_example python/ml/linear_regression_with_elastic_net.py %} + + + + +## Generalized linear regression + +Contrasted with linear regression where the output is assumed to follow a Gaussian +distribution, [generalized linear models](https://en.wikipedia.org/wiki/Generalized_linear_model) (GLMs) are specifications of linear models where the response variable $Y_i$ follows some +distribution from the [exponential family of distributions](https://en.wikipedia.org/wiki/Exponential_family). +Spark's `GeneralizedLinearRegression` interface +allows for flexible specification of GLMs which can be used for various types of +prediction problems including linear regression, Poisson regression, logistic regression, and others. +Currently in `spark.ml`, only a subset of the exponential family distributions are supported and they are listed +[below](#available-families). + +**NOTE**: Spark currently only supports up to 4096 features through its `GeneralizedLinearRegression` +interface, and will throw an exception if this constraint is exceeded. See the [advanced section](ml-advanced) for more details. + Still, for linear and logistic regression, models with an increased number of features can be trained + using the `LinearRegression` and `LogisticRegression` estimators. + +GLMs require exponential family distributions that can be written in their "canonical" or "natural" form, aka +[natural exponential family distributions](https://en.wikipedia.org/wiki/Natural_exponential_family). The form of a natural exponential family distribution is given as: + +$$ +f_Y(y|\theta, \tau) = h(y, \tau)\exp{\left( \frac{\theta \cdot y - A(\theta)}{d(\tau)} \right)} +$$ + +where $\theta$ is the parameter of interest and $\tau$ is a dispersion parameter. In a GLM the response variable $Y_i$ is assumed to be drawn from a natural exponential family distribution: + +$$ +Y_i \sim f\left(\cdot|\theta_i, \tau \right) +$$ + +where the parameter of interest $\theta_i$ is related to the expected value of the response variable $\mu_i$ by + +$$ +\mu_i = A'(\theta_i) +$$ + +Here, $A'(\theta_i)$ is defined by the form of the distribution selected. GLMs also allow specification +of a link function, which defines the relationship between the expected value of the response variable $\mu_i$ +and the so called _linear predictor_ $\eta_i$: + +$$ +g(\mu_i) = \eta_i = \vec{x_i}^T \cdot \vec{\beta} +$$ + +Often, the link function is chosen such that $A' = g^{-1}$, which yields a simplified relationship +between the parameter of interest $\theta$ and the linear predictor $\eta$. In this case, the link +function $g(\mu)$ is said to be the "canonical" link function. + +$$ +\theta_i = A'^{-1}(\mu_i) = g(g^{-1}(\eta_i)) = \eta_i +$$ + +A GLM finds the regression coefficients $\vec{\beta}$ which maximize the likelihood function. + +$$ +\max_{\vec{\beta}} \mathcal{L}(\vec{\theta}|\vec{y},X) = +\prod_{i=1}^{N} h(y_i, \tau) \exp{\left(\frac{y_i\theta_i - A(\theta_i)}{d(\tau)}\right)} +$$ + +where the parameter of interest $\theta_i$ is related to the regression coefficients $\vec{\beta}$ +by + +$$ +\theta_i = A'^{-1}(g^{-1}(\vec{x_i} \cdot \vec{\beta})) +$$ + +Spark's generalized linear regression interface also provides summary statistics for diagnosing the +fit of GLM models, including residuals, p-values, deviances, the Akaike information criterion, and +others. + +[See here](http://data.princeton.edu/wws509/notes/) for a more comprehensive review of GLMs and their applications. + +### Available families + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    FamilyResponse TypeSupported Links
    GaussianContinuousIdentity*, Log, Inverse
    BinomialBinaryLogit*, Probit, CLogLog
    PoissonCountLog*, Identity, Sqrt
    GammaContinuousInverse*, Idenity, Log
    TweedieZero-inflated continuousPower link function
    * Canonical Link
    + +**Examples** + +The following example demonstrates training a GLM with a Gaussian response and identity link +function and extracting model summary statistics. + +
    + +
    + +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.regression.GeneralizedLinearRegression) for more details. + +{% include_example scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala %} +
    + +
    + +Refer to the [Java API docs](api/java/org/apache/spark/ml/regression/GeneralizedLinearRegression.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java %} +
    + +
    + +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression.GeneralizedLinearRegression) for more details. + +{% include_example python/ml/generalized_linear_regression_example.py %} +
    + +
    + +Refer to the [R API docs](api/R/spark.glm.html) for more details. + +{% include_example r/ml/glm.R %} +
    + +
    + + +## Decision tree regression + +Decision trees are a popular family of classification and regression methods. +More information about the `spark.ml` implementation can be found further in the [section on decision trees](#decision-trees). + +**Examples** + +The following examples load a dataset in LibSVM format, split it into training and test sets, train on the first dataset, and then evaluate on the held-out test set. +We use a feature transformer to index categorical features, adding metadata to the `DataFrame` which the Decision Tree algorithm can recognize. + +
    +
    + +More details on parameters can be found in the [Scala API documentation](api/scala/index.html#org.apache.spark.ml.regression.DecisionTreeRegressor). + +{% include_example scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala %} +
    + +
    + +More details on parameters can be found in the [Java API documentation](api/java/org/apache/spark/ml/regression/DecisionTreeRegressor.html). + +{% include_example java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java %} +
    + +
    + +More details on parameters can be found in the [Python API documentation](api/python/pyspark.ml.html#pyspark.ml.regression.DecisionTreeRegressor). + +{% include_example python/ml/decision_tree_regression_example.py %} +
    + +
    + +Refer to the [R API docs](api/R/spark.decisionTree.html) for more details. + +{% include_example regression r/ml/decisionTree.R %} +
    + +
    + + +## Random forest regression + +Random forests are a popular family of classification and regression methods. +More information about the `spark.ml` implementation can be found further in the [section on random forests](#random-forests). + +**Examples** + +The following examples load a dataset in LibSVM format, split it into training and test sets, train on the first dataset, and then evaluate on the held-out test set. +We use a feature transformer to index categorical features, adding metadata to the `DataFrame` which the tree-based algorithms can recognize. + +
    +
    + +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.regression.RandomForestRegressor) for more details. + +{% include_example scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala %} +
    + +
    + +Refer to the [Java API docs](api/java/org/apache/spark/ml/regression/RandomForestRegressor.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java %} +
    + +
    + +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression.RandomForestRegressor) for more details. + +{% include_example python/ml/random_forest_regressor_example.py %} +
    + +
    + +Refer to the [R API docs](api/R/spark.randomForest.html) for more details. + +{% include_example regression r/ml/randomForest.R %} +
    + +
    + +## Gradient-boosted tree regression + +Gradient-boosted trees (GBTs) are a popular regression method using ensembles of decision trees. +More information about the `spark.ml` implementation can be found further in the [section on GBTs](#gradient-boosted-trees-gbts). + +**Examples** + +Note: For this example dataset, `GBTRegressor` actually only needs 1 iteration, but that will not +be true in general. + +
    +
    + +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.regression.GBTRegressor) for more details. + +{% include_example scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala %} +
    + +
    + +Refer to the [Java API docs](api/java/org/apache/spark/ml/regression/GBTRegressor.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java %} +
    + +
    + +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression.GBTRegressor) for more details. + +{% include_example python/ml/gradient_boosted_tree_regressor_example.py %} +
    + +
    + +Refer to the [R API docs](api/R/spark.gbt.html) for more details. + +{% include_example regression r/ml/gbt.R %} +
    + +
    + + +## Survival regression + + +In `spark.ml`, we implement the [Accelerated failure time (AFT)](https://en.wikipedia.org/wiki/Accelerated_failure_time_model) +model which is a parametric survival regression model for censored data. +It describes a model for the log of survival time, so it's often called a +log-linear model for survival analysis. Different from a +[Proportional hazards](https://en.wikipedia.org/wiki/Proportional_hazards_model) model +designed for the same purpose, the AFT model is easier to parallelize +because each instance contributes to the objective function independently. + +Given the values of the covariates $x^{'}$, for random lifetime $t_{i}$ of +subjects i = 1, ..., n, with possible right-censoring, +the likelihood function under the AFT model is given as: +`\[ +L(\beta,\sigma)=\prod_{i=1}^n[\frac{1}{\sigma}f_{0}(\frac{\log{t_{i}}-x^{'}\beta}{\sigma})]^{\delta_{i}}S_{0}(\frac{\log{t_{i}}-x^{'}\beta}{\sigma})^{1-\delta_{i}} +\]` +Where $\delta_{i}$ is the indicator of the event has occurred i.e. uncensored or not. +Using $\epsilon_{i}=\frac{\log{t_{i}}-x^{'}\beta}{\sigma}$, the log-likelihood function +assumes the form: +`\[ +\iota(\beta,\sigma)=\sum_{i=1}^{n}[-\delta_{i}\log\sigma+\delta_{i}\log{f_{0}}(\epsilon_{i})+(1-\delta_{i})\log{S_{0}(\epsilon_{i})}] +\]` +Where $S_{0}(\epsilon_{i})$ is the baseline survivor function, +and $f_{0}(\epsilon_{i})$ is the corresponding density function. + +The most commonly used AFT model is based on the Weibull distribution of the survival time. +The Weibull distribution for lifetime corresponds to the extreme value distribution for the +log of the lifetime, and the $S_{0}(\epsilon)$ function is: +`\[ +S_{0}(\epsilon_{i})=\exp(-e^{\epsilon_{i}}) +\]` +the $f_{0}(\epsilon_{i})$ function is: +`\[ +f_{0}(\epsilon_{i})=e^{\epsilon_{i}}\exp(-e^{\epsilon_{i}}) +\]` +The log-likelihood function for AFT model with a Weibull distribution of lifetime is: +`\[ +\iota(\beta,\sigma)= -\sum_{i=1}^n[\delta_{i}\log\sigma-\delta_{i}\epsilon_{i}+e^{\epsilon_{i}}] +\]` +Due to minimizing the negative log-likelihood equivalent to maximum a posteriori probability, +the loss function we use to optimize is $-\iota(\beta,\sigma)$. +The gradient functions for $\beta$ and $\log\sigma$ respectively are: +`\[ +\frac{\partial (-\iota)}{\partial \beta}=\sum_{1=1}^{n}[\delta_{i}-e^{\epsilon_{i}}]\frac{x_{i}}{\sigma} +\]` +`\[ +\frac{\partial (-\iota)}{\partial (\log\sigma)}=\sum_{i=1}^{n}[\delta_{i}+(\delta_{i}-e^{\epsilon_{i}})\epsilon_{i}] +\]` + +The AFT model can be formulated as a convex optimization problem, +i.e. the task of finding a minimizer of a convex function $-\iota(\beta,\sigma)$ +that depends on the coefficients vector $\beta$ and the log of scale parameter $\log\sigma$. +The optimization algorithm underlying the implementation is L-BFGS. +The implementation matches the result from R's survival function +[survreg](https://stat.ethz.ch/R-manual/R-devel/library/survival/html/survreg.html) + + > When fitting AFTSurvivalRegressionModel without intercept on dataset with constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero columns. This behavior is different from R survival::survreg. + +**Examples** + +
    + +
    + +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.regression.AFTSurvivalRegression) for more details. + +{% include_example scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala %} +
    + +
    + +Refer to the [Java API docs](api/java/org/apache/spark/ml/regression/AFTSurvivalRegression.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java %} +
    + +
    + +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression.AFTSurvivalRegression) for more details. + +{% include_example python/ml/aft_survival_regression.py %} +
    + +
    + +Refer to the [R API docs](api/R/spark.survreg.html) for more details. + +{% include_example r/ml/survreg.R %} +
    + +
    + + +## 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. + +We implement 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 DataFrame which contains three columns +label, features and weight. 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** + +
    +
    + +Refer to the [`IsotonicRegression` Scala docs](api/scala/index.html#org.apache.spark.ml.regression.IsotonicRegression) for details on the API. + +{% include_example scala/org/apache/spark/examples/ml/IsotonicRegressionExample.scala %} +
    +
    + +Refer to the [`IsotonicRegression` Java docs](api/java/org/apache/spark/ml/regression/IsotonicRegression.html) for details on the API. + +{% include_example java/org/apache/spark/examples/ml/JavaIsotonicRegressionExample.java %} +
    +
    + +Refer to the [`IsotonicRegression` Python docs](api/python/pyspark.ml.html#pyspark.ml.regression.IsotonicRegression) for more details on the API. + +{% include_example python/ml/isotonic_regression_example.py %} +
    + +
    + +Refer to the [`IsotonicRegression` R API docs](api/R/spark.isoreg.html) for more details on the API. + +{% include_example r/ml/isoreg.R %} +
    + +
    + +# Linear methods + +We implement popular linear methods such as logistic +regression and linear least squares with $L_1$ or $L_2$ regularization. +Refer to [the linear methods guide for the RDD-based API](mllib-linear-methods.html) for +details about implementation and tuning; this information is still relevant. + +We also include a DataFrame API for [Elastic +net](http://en.wikipedia.org/wiki/Elastic_net_regularization), a hybrid +of $L_1$ and $L_2$ regularization proposed in [Zou et al, Regularization +and variable selection via the elastic +net](http://users.stat.umn.edu/~zouxx019/Papers/elasticnet.pdf). +Mathematically, it is defined as a convex combination of the $L_1$ and +the $L_2$ regularization terms: +`\[ +\alpha \left( \lambda \|\wv\|_1 \right) + (1-\alpha) \left( \frac{\lambda}{2}\|\wv\|_2^2 \right) , \alpha \in [0, 1], \lambda \geq 0 +\]` +By setting $\alpha$ properly, elastic net contains both $L_1$ and $L_2$ +regularization as special cases. For example, if a [linear +regression](https://en.wikipedia.org/wiki/Linear_regression) model is +trained with the elastic net parameter $\alpha$ set to $1$, it is +equivalent to a +[Lasso](http://en.wikipedia.org/wiki/Least_squares#Lasso_method) model. +On the other hand, if $\alpha$ is set to $0$, the trained model reduces +to a [ridge +regression](http://en.wikipedia.org/wiki/Tikhonov_regularization) model. +We implement Pipelines API for both linear regression and logistic +regression with elastic net regularization. + +# Decision trees + +[Decision trees](http://en.wikipedia.org/wiki/Decision_tree_learning) +and their ensembles are popular methods for the machine learning tasks of +classification and regression. Decision trees are widely used since they are easy to interpret, +handle categorical features, extend to the multiclass classification setting, do not require +feature scaling, and are able to capture non-linearities and feature interactions. Tree ensemble +algorithms such as random forests and boosting are among the top performers for classification and +regression tasks. + +The `spark.ml` implementation supports decision trees for binary and multiclass classification and for regression, +using both continuous and categorical features. The implementation partitions data by rows, +allowing distributed training with millions or even billions of instances. + +Users can find more information about the decision tree algorithm in the [MLlib Decision Tree guide](mllib-decision-tree.html). +The main differences between this API and the [original MLlib Decision Tree API](mllib-decision-tree.html) are: + +* support for ML Pipelines +* separation of Decision Trees for classification vs. regression +* use of DataFrame metadata to distinguish continuous and categorical features + + +The Pipelines API for Decision Trees offers a bit more functionality than the original API. +In particular, for classification, users can get the predicted probability of each class (a.k.a. class conditional probabilities); +for regression, users can get the biased sample variance of prediction. + +Ensembles of trees (Random Forests and Gradient-Boosted Trees) are described below in the [Tree ensembles section](#tree-ensembles). + +## Inputs and Outputs + +We list the input and output (prediction) column types here. +All output columns are optional; to exclude an output column, set its corresponding Param to an empty string. + +### Input Columns + + + + + + + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescription
    labelColDouble"label"Label to predict
    featuresColVector"features"Feature vector
    + +### Output Columns + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescriptionNotes
    predictionColDouble"prediction"Predicted label
    rawPredictionColVector"rawPrediction"Vector of length # classes, with the counts of training instance labels at the tree node which makes the predictionClassification only
    probabilityColVector"probability"Vector of length # classes equal to rawPrediction normalized to a multinomial distributionClassification only
    varianceColDoubleThe biased sample variance of predictionRegression only
    + + +# Tree Ensembles + +The DataFrame API supports two major tree ensemble algorithms: [Random Forests](http://en.wikipedia.org/wiki/Random_forest) and [Gradient-Boosted Trees (GBTs)](http://en.wikipedia.org/wiki/Gradient_boosting). +Both use [`spark.ml` decision trees](ml-classification-regression.html#decision-trees) as their base models. + +Users can find more information about ensemble algorithms in the [MLlib Ensemble guide](mllib-ensembles.html). +In this section, we demonstrate the DataFrame API for ensembles. + +The main differences between this API and the [original MLlib ensembles API](mllib-ensembles.html) are: + +* support for DataFrames and ML Pipelines +* separation of classification vs. regression +* use of DataFrame metadata to distinguish continuous and categorical features +* more functionality for random forests: estimates of feature importance, as well as the predicted probability of each class (a.k.a. class conditional probabilities) for classification. + +## Random Forests + +[Random forests](http://en.wikipedia.org/wiki/Random_forest) +are ensembles of [decision trees](ml-classification-regression.html#decision-trees). +Random forests combine many decision trees in order to reduce the risk of overfitting. +The `spark.ml` implementation supports random forests for binary and multiclass classification and for regression, +using both continuous and categorical features. + +For more information on the algorithm itself, please see the [`spark.mllib` documentation on random forests](mllib-ensembles.html#random-forests). + +### Inputs and Outputs + +We list the input and output (prediction) column types here. +All output columns are optional; to exclude an output column, set its corresponding Param to an empty string. + +#### Input Columns + + + + + + + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescription
    labelColDouble"label"Label to predict
    featuresColVector"features"Feature vector
    + +#### Output Columns (Predictions) + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescriptionNotes
    predictionColDouble"prediction"Predicted label
    rawPredictionColVector"rawPrediction"Vector of length # classes, with the counts of training instance labels at the tree node which makes the predictionClassification only
    probabilityColVector"probability"Vector of length # classes equal to rawPrediction normalized to a multinomial distributionClassification only
    + + + +## Gradient-Boosted Trees (GBTs) + +[Gradient-Boosted Trees (GBTs)](http://en.wikipedia.org/wiki/Gradient_boosting) +are ensembles of [decision trees](ml-classification-regression.html#decision-trees). +GBTs iteratively train decision trees in order to minimize a loss function. +The `spark.ml` implementation supports GBTs for binary classification and for regression, +using both continuous and categorical features. + +For more information on the algorithm itself, please see the [`spark.mllib` documentation on GBTs](mllib-ensembles.html#gradient-boosted-trees-gbts). + +### Inputs and Outputs + +We list the input and output (prediction) column types here. +All output columns are optional; to exclude an output column, set its corresponding Param to an empty string. + +#### Input Columns + + + + + + + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescription
    labelColDouble"label"Label to predict
    featuresColVector"features"Feature vector
    + +Note that `GBTClassifier` currently only supports binary labels. + +#### Output Columns (Predictions) + + + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescriptionNotes
    predictionColDouble"prediction"Predicted label
    + +In the future, `GBTClassifier` will also output columns for `rawPrediction` and `probability`, just as `RandomForestClassifier` does. + diff --git a/docs/ml-clustering.md b/docs/ml-clustering.md new file mode 100644 index 000000000000..1186fb73d0fa --- /dev/null +++ b/docs/ml-clustering.md @@ -0,0 +1,267 @@ +--- +layout: global +title: Clustering +displayTitle: Clustering +--- + +This page describes clustering algorithms in MLlib. +The [guide for clustering in the RDD-based API](mllib-clustering.html) also has relevant information +about these algorithms. + +**Table of Contents** + +* This will become a table of contents (this text will be scraped). +{: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 +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). + +`KMeans` is implemented as an `Estimator` and generates a `KMeansModel` as the base model. + +### Input Columns + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescription
    featuresColVector"features"Feature vector
    + +### Output Columns + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescription
    predictionColInt"prediction"Predicted cluster center
    + +**Examples** + +
    + +
    +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.clustering.KMeans) for more details. + +{% include_example scala/org/apache/spark/examples/ml/KMeansExample.scala %} +
    + +
    +Refer to the [Java API docs](api/java/org/apache/spark/ml/clustering/KMeans.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaKMeansExample.java %} +
    + +
    +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering.KMeans) for more details. + +{% include_example python/ml/kmeans_example.py %} +
    + +
    + +Refer to the [R API docs](api/R/spark.kmeans.html) for more details. + +{% include_example r/ml/kmeans.R %} +
    + +
    + +## Latent Dirichlet allocation (LDA) + +`LDA` is implemented as an `Estimator` that supports both `EMLDAOptimizer` and `OnlineLDAOptimizer`, +and generates a `LDAModel` as the base model. Expert users may cast a `LDAModel` generated by +`EMLDAOptimizer` to a `DistributedLDAModel` if needed. + +**Examples** + +
    + +
    + +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.clustering.LDA) for more details. + +{% include_example scala/org/apache/spark/examples/ml/LDAExample.scala %} +
    + +
    + +Refer to the [Java API docs](api/java/org/apache/spark/ml/clustering/LDA.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaLDAExample.java %} +
    + +
    + +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering.LDA) for more details. + +{% include_example python/ml/lda_example.py %} +
    + +
    + +Refer to the [R API docs](api/R/spark.lda.html) for more details. + +{% include_example r/ml/lda.R %} +
    + +
    + +## Bisecting k-means + +Bisecting k-means is a kind of [hierarchical clustering](https://en.wikipedia.org/wiki/Hierarchical_clustering) using a +divisive (or "top-down") approach: all observations start in one cluster, and splits are performed recursively as one +moves down the hierarchy. + +Bisecting K-means can often be much faster than regular K-means, but it will generally produce a different clustering. + +`BisectingKMeans` is implemented as an `Estimator` and generates a `BisectingKMeansModel` as the base model. + +**Examples** + +
    + +
    +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.clustering.BisectingKMeans) for more details. + +{% include_example scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala %} +
    + +
    +Refer to the [Java API docs](api/java/org/apache/spark/ml/clustering/BisectingKMeans.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java %} +
    + +
    +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering.BisectingKMeans) for more details. + +{% include_example python/ml/bisecting_k_means_example.py %} +
    + +
    + +Refer to the [R API docs](api/R/spark.bisectingKmeans.html) for more details. + +{% include_example r/ml/bisectingKmeans.R %} +
    +
    + +## Gaussian Mixture Model (GMM) + +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 `spark.ml` 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. + +`GaussianMixture` is implemented as an `Estimator` and generates a `GaussianMixtureModel` as the base +model. + +### Input Columns + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescription
    featuresColVector"features"Feature vector
    + +### Output Columns + + + + + + + + + + + + + + + + + + + + + + + + +
    Param nameType(s)DefaultDescription
    predictionColInt"prediction"Predicted cluster center
    probabilityColVector"probability"Probability of each cluster
    + +**Examples** + +
    + +
    +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.clustering.GaussianMixture) for more details. + +{% include_example scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala %} +
    + +
    +Refer to the [Java API docs](api/java/org/apache/spark/ml/clustering/GaussianMixture.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java %} +
    + +
    +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering.GaussianMixture) for more details. + +{% include_example python/ml/gaussian_mixture_example.py %} +
    + +
    + +Refer to the [R API docs](api/R/spark.gaussianMixture.html) for more details. + +{% include_example r/ml/gaussianMixture.R %} +
    + +
    diff --git a/docs/ml-collaborative-filtering.md b/docs/ml-collaborative-filtering.md new file mode 100644 index 000000000000..58f2d4b531e7 --- /dev/null +++ b/docs/ml-collaborative-filtering.md @@ -0,0 +1,188 @@ +--- +layout: global +title: Collaborative Filtering +displayTitle: Collaborative Filtering +--- + +* Table of contents +{:toc} + +## Collaborative filtering + +[Collaborative filtering](http://en.wikipedia.org/wiki/Recommender_system#Collaborative_filtering) +is commonly used for recommender systems. These techniques aim to fill in the +missing entries of a user-item association matrix. `spark.ml` currently supports +model-based collaborative filtering, in which users and products are described +by a small set of latent factors that can be used to predict missing entries. +`spark.ml` uses the [alternating least squares +(ALS)](http://dl.acm.org/citation.cfm?id=1608614) +algorithm to learn these latent factors. The implementation in `spark.ml` has the +following parameters: + +* *numBlocks* is the number of blocks the users and items will be partitioned into in order to parallelize computation (defaults to 10). +* *rank* is the number of latent factors in the model (defaults to 10). +* *maxIter* is the maximum number of iterations to run (defaults to 10). +* *regParam* specifies the regularization parameter in ALS (defaults to 1.0). +* *implicitPrefs* specifies whether to use the *explicit feedback* ALS variant or one adapted for + *implicit feedback* data (defaults to `false` which means using *explicit feedback*). +* *alpha* is a parameter applicable to the implicit feedback variant of ALS that governs the + *baseline* confidence in preference observations (defaults to 1.0). +* *nonnegative* specifies whether or not to use nonnegative constraints for least squares (defaults to `false`). + +**Note:** The DataFrame-based API for ALS currently only supports integers for user and item ids. +Other numeric types are supported for the user and item id columns, +but the ids must be within the integer value range. + +### Explicit vs. implicit feedback + +The standard approach to matrix factorization based collaborative filtering treats +the entries in the user-item matrix as *explicit* preferences given by the user to the item, +for example, users giving ratings to movies. + +It is common in many real-world use cases to only have access to *implicit feedback* (e.g. views, +clicks, purchases, likes, shares etc.). The approach used in `spark.ml` to deal with such data is taken +from [Collaborative Filtering for Implicit Feedback Datasets](http://dx.doi.org/10.1109/ICDM.2008.22). +Essentially, instead of trying to model the matrix of ratings directly, this approach treats the data +as numbers representing the *strength* in observations of user actions (such as the number of clicks, +or the cumulative duration someone spent viewing a movie). Those numbers are then related to the level of +confidence in observed user preferences, rather than explicit ratings given to items. The model +then tries to find latent factors that can be used to predict the expected preference of a user for +an item. + +### Scaling of the regularization parameter + +We scale the regularization parameter `regParam` in solving each least squares problem by +the number of ratings the user generated in updating user factors, +or the number of ratings the product received in updating product factors. +This approach is named "ALS-WR" and discussed in the paper +"[Large-Scale Parallel Collaborative Filtering for the Netflix Prize](http://dx.doi.org/10.1007/978-3-540-68880-8_32)". +It makes `regParam` less dependent on the scale of the dataset, so we can apply the +best parameter learned from a sampled subset to the full dataset and expect similar performance. + +### Cold-start strategy + +When making predictions using an `ALSModel`, it is common to encounter users and/or items in the +test dataset that were not present during training the model. This typically occurs in two +scenarios: + +1. In production, for new users or items that have no rating history and on which the model has not +been trained (this is the "cold start problem"). +2. During cross-validation, the data is split between training and evaluation sets. When using +simple random splits as in Spark's `CrossValidator` or `TrainValidationSplit`, it is actually +very common to encounter users and/or items in the evaluation set that are not in the training set + +By default, Spark assigns `NaN` predictions during `ALSModel.transform` when a user and/or item +factor is not present in the model. This can be useful in a production system, since it indicates +a new user or item, and so the system can make a decision on some fallback to use as the prediction. + +However, this is undesirable during cross-validation, since any `NaN` predicted values will result +in `NaN` results for the evaluation metric (for example when using `RegressionEvaluator`). +This makes model selection impossible. + +Spark allows users to set the `coldStartStrategy` parameter +to "drop" in order to drop any rows in the `DataFrame` of predictions that contain `NaN` values. +The evaluation metric will then be computed over the non-`NaN` data and will be valid. +Usage of this parameter is illustrated in the example below. + +**Note:** currently the supported cold start strategies are "nan" (the default behavior mentioned +above) and "drop". Further strategies may be supported in future. + +**Examples** + +
    +
    + +In the following example, we load ratings data from the +[MovieLens dataset](http://grouplens.org/datasets/movielens/), each row +consisting of a user, a movie, a rating and a timestamp. +We then train an ALS model which assumes, by default, that the ratings are +explicit (`implicitPrefs` is `false`). +We evaluate the recommendation model by measuring the root-mean-square error of +rating prediction. + +Refer to the [`ALS` Scala docs](api/scala/index.html#org.apache.spark.ml.recommendation.ALS) +for more details on the API. + +{% include_example scala/org/apache/spark/examples/ml/ALSExample.scala %} + +If the rating matrix is derived from another source of information (i.e. it is +inferred from other signals), you can set `implicitPrefs` to `true` to get +better results: + +{% highlight scala %} +val als = new ALS() + .setMaxIter(5) + .setRegParam(0.01) + .setImplicitPrefs(true) + .setUserCol("userId") + .setItemCol("movieId") + .setRatingCol("rating") +{% endhighlight %} + +
    + +
    + +In the following example, we load ratings data from the +[MovieLens dataset](http://grouplens.org/datasets/movielens/), each row +consisting of a user, a movie, a rating and a timestamp. +We then train an ALS model which assumes, by default, that the ratings are +explicit (`implicitPrefs` is `false`). +We evaluate the recommendation model by measuring the root-mean-square error of +rating prediction. + +Refer to the [`ALS` Java docs](api/java/org/apache/spark/ml/recommendation/ALS.html) +for more details on the API. + +{% include_example java/org/apache/spark/examples/ml/JavaALSExample.java %} + +If the rating matrix is derived from another source of information (i.e. it is +inferred from other signals), you can set `implicitPrefs` to `true` to get +better results: + +{% highlight java %} +ALS als = new ALS() + .setMaxIter(5) + .setRegParam(0.01) + .setImplicitPrefs(true) + .setUserCol("userId") + .setItemCol("movieId") + .setRatingCol("rating"); +{% endhighlight %} + +
    + +
    + +In the following example, we load ratings data from the +[MovieLens dataset](http://grouplens.org/datasets/movielens/), each row +consisting of a user, a movie, a rating and a timestamp. +We then train an ALS model which assumes, by default, that the ratings are +explicit (`implicitPrefs` is `False`). +We evaluate the recommendation model by measuring the root-mean-square error of +rating prediction. + +Refer to the [`ALS` Python docs](api/python/pyspark.ml.html#pyspark.ml.recommendation.ALS) +for more details on the API. + +{% include_example python/ml/als_example.py %} + +If the rating matrix is derived from another source of information (i.e. it is +inferred from other signals), you can set `implicitPrefs` to `True` to get +better results: + +{% highlight python %} +als = ALS(maxIter=5, regParam=0.01, implicitPrefs=True, + userCol="userId", itemCol="movieId", ratingCol="rating") +{% endhighlight %} + +
    + +
    + +Refer to the [R API docs](api/R/spark.als.html) for more details. + +{% include_example r/ml/als.R %} +
    + +
    diff --git a/docs/ml-decision-tree.md b/docs/ml-decision-tree.md index 542819e93e6d..5e1eeb95e472 100644 --- a/docs/ml-decision-tree.md +++ b/docs/ml-decision-tree.md @@ -1,493 +1,8 @@ --- layout: global -title: Decision Trees - SparkML -displayTitle: ML - Decision Trees +title: Decision trees +displayTitle: Decision trees --- -**Table of Contents** - -* This will become a table of contents (this text will be scraped). -{:toc} - - -# Overview - -[Decision trees](http://en.wikipedia.org/wiki/Decision_tree_learning) -and their ensembles are popular methods for the machine learning tasks of -classification and regression. Decision trees are widely used since they are easy to interpret, -handle categorical features, extend to the multiclass classification setting, do not require -feature scaling, and are able to capture non-linearities and feature interactions. Tree ensemble -algorithms such as random forests and boosting are among the top performers for classification and -regression tasks. - -MLlib supports decision trees for binary and multiclass classification and for regression, -using both continuous and categorical features. The implementation partitions data by rows, -allowing distributed training with millions or even billions of instances. - -Users can find more information about the decision tree algorithm in the [MLlib Decision Tree guide](mllib-decision-tree.html). In this section, we demonstrate the Pipelines API for Decision Trees. - -The Pipelines API for Decision Trees offers a bit more functionality than the original API. In particular, for classification, users can get the predicted probability of each class (a.k.a. class conditional probabilities). - -Ensembles of trees (Random Forests and Gradient-Boosted Trees) are described in the [Ensembles guide](ml-ensembles.html). - -# Inputs and Outputs - -We list the input and output (prediction) column types here. -All output columns are optional; to exclude an output column, set its corresponding Param to an empty string. - -## Input Columns - - - - - - - - - - - - - - - - - - - - - - - - -
    Param nameType(s)DefaultDescription
    labelColDouble"label"Label to predict
    featuresColVector"features"Feature vector
    - -## Output Columns - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Param nameType(s)DefaultDescriptionNotes
    predictionColDouble"prediction"Predicted label
    rawPredictionColVector"rawPrediction"Vector of length # classes, with the counts of training instance labels at the tree node which makes the predictionClassification only
    probabilityColVector"probability"Vector of length # classes equal to rawPrediction normalized to a multinomial distributionClassification only
    - -# Examples - -The below examples demonstrate the Pipelines API for Decision Trees. The main differences between this API and the [original MLlib Decision Tree API](mllib-decision-tree.html) are: - -* support for ML Pipelines -* separation of Decision Trees for classification vs. regression -* use of DataFrame metadata to distinguish continuous and categorical features - - -## Classification - -The following examples load a dataset in LibSVM format, split it into training and test sets, train on the first dataset, and then evaluate on the held-out test set. -We use two feature transformers to prepare the data; these help index categories for the label and categorical features, adding metadata to the `DataFrame` which the Decision Tree algorithm can recognize. - -
    -
    - -More details on parameters can be found in the [Scala API documentation](api/scala/index.html#org.apache.spark.ml.classification.DecisionTreeClassifier). - -{% highlight scala %} -import org.apache.spark.ml.Pipeline -import org.apache.spark.ml.classification.DecisionTreeClassifier -import org.apache.spark.ml.classification.DecisionTreeClassificationModel -import org.apache.spark.ml.feature.{StringIndexer, IndexToString, VectorIndexer} -import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator -import org.apache.spark.mllib.util.MLUtils - -// Load and parse the data file, converting it to a DataFrame. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() - -// Index labels, adding metadata to the label column. -// Fit on whole dataset to include all labels in index. -val labelIndexer = new StringIndexer() - .setInputCol("label") - .setOutputCol("indexedLabel") - .fit(data) -// Automatically identify categorical features, and index them. -val featureIndexer = new VectorIndexer() - .setInputCol("features") - .setOutputCol("indexedFeatures") - .setMaxCategories(4) // features with > 4 distinct values are treated as continuous - .fit(data) - -// Split the data into training and test sets (30% held out for testing) -val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) - -// Train a DecisionTree model. -val dt = new DecisionTreeClassifier() - .setLabelCol("indexedLabel") - .setFeaturesCol("indexedFeatures") - -// Convert indexed labels back to original labels. -val labelConverter = new IndexToString() - .setInputCol("prediction") - .setOutputCol("predictedLabel") - .setLabels(labelIndexer.labels) - -// Chain indexers and tree in a Pipeline -val pipeline = new Pipeline() - .setStages(Array(labelIndexer, featureIndexer, dt, labelConverter)) - -// Train model. This also runs the indexers. -val model = pipeline.fit(trainingData) - -// Make predictions. -val predictions = model.transform(testData) - -// Select example rows to display. -predictions.select("predictedLabel", "label", "features").show(5) - -// Select (prediction, true label) and compute test error -val evaluator = new MulticlassClassificationEvaluator() - .setLabelCol("indexedLabel") - .setPredictionCol("prediction") - .setMetricName("precision") -val accuracy = evaluator.evaluate(predictions) -println("Test Error = " + (1.0 - accuracy)) - -val treeModel = model.stages(2).asInstanceOf[DecisionTreeClassificationModel] -println("Learned classification tree model:\n" + treeModel.toDebugString) -{% endhighlight %} -
    - -
    - -More details on parameters can be found in the [Java API documentation](api/java/org/apache/spark/ml/classification/DecisionTreeClassifier.html). - -{% highlight java %} -import org.apache.spark.ml.Pipeline; -import org.apache.spark.ml.PipelineModel; -import org.apache.spark.ml.PipelineStage; -import org.apache.spark.ml.classification.DecisionTreeClassifier; -import org.apache.spark.ml.classification.DecisionTreeClassificationModel; -import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; -import org.apache.spark.ml.feature.*; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.rdd.RDD; -import org.apache.spark.sql.DataFrame; - -// Load and parse the data file, converting it to a DataFrame. -RDD rdd = MLUtils.loadLibSVMFile(sc.sc(), "data/mllib/sample_libsvm_data.txt"); -DataFrame data = jsql.createDataFrame(rdd, LabeledPoint.class); - -// Index labels, adding metadata to the label column. -// Fit on whole dataset to include all labels in index. -StringIndexerModel labelIndexer = new StringIndexer() - .setInputCol("label") - .setOutputCol("indexedLabel") - .fit(data); -// Automatically identify categorical features, and index them. -VectorIndexerModel featureIndexer = new VectorIndexer() - .setInputCol("features") - .setOutputCol("indexedFeatures") - .setMaxCategories(4) // features with > 4 distinct values are treated as continuous - .fit(data); - -// Split the data into training and test sets (30% held out for testing) -DataFrame[] splits = data.randomSplit(new double[] {0.7, 0.3}); -DataFrame trainingData = splits[0]; -DataFrame testData = splits[1]; - -// Train a DecisionTree model. -DecisionTreeClassifier dt = new DecisionTreeClassifier() - .setLabelCol("indexedLabel") - .setFeaturesCol("indexedFeatures"); - -// Convert indexed labels back to original labels. -IndexToString labelConverter = new IndexToString() - .setInputCol("prediction") - .setOutputCol("predictedLabel") - .setLabels(labelIndexer.labels()); - -// Chain indexers and tree in a Pipeline -Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[] {labelIndexer, featureIndexer, dt, labelConverter}); - -// Train model. This also runs the indexers. -PipelineModel model = pipeline.fit(trainingData); - -// Make predictions. -DataFrame predictions = model.transform(testData); - -// Select example rows to display. -predictions.select("predictedLabel", "label", "features").show(5); - -// Select (prediction, true label) and compute test error -MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() - .setLabelCol("indexedLabel") - .setPredictionCol("prediction") - .setMetricName("precision"); -double accuracy = evaluator.evaluate(predictions); -System.out.println("Test Error = " + (1.0 - accuracy)); - -DecisionTreeClassificationModel treeModel = - (DecisionTreeClassificationModel)(model.stages()[2]); -System.out.println("Learned classification tree model:\n" + treeModel.toDebugString()); -{% endhighlight %} -
    - -
    - -More details on parameters can be found in the [Python API documentation](api/python/pyspark.ml.html#pyspark.ml.classification.DecisionTreeClassifier). - -{% highlight python %} -from pyspark.ml import Pipeline -from pyspark.ml.classification import DecisionTreeClassifier -from pyspark.ml.feature import StringIndexer, VectorIndexer -from pyspark.ml.evaluation import MulticlassClassificationEvaluator -from pyspark.mllib.util import MLUtils - -# Load and parse the data file, converting it to a DataFrame. -data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() - -# Index labels, adding metadata to the label column. -# Fit on whole dataset to include all labels in index. -labelIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel").fit(data) -# Automatically identify categorical features, and index them. -# We specify maxCategories so features with > 4 distinct values are treated as continuous. -featureIndexer =\ - VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) - -# Split the data into training and test sets (30% held out for testing) -(trainingData, testData) = data.randomSplit([0.7, 0.3]) - -# Train a DecisionTree model. -dt = DecisionTreeClassifier(labelCol="indexedLabel", featuresCol="indexedFeatures") - -# Chain indexers and tree in a Pipeline -pipeline = Pipeline(stages=[labelIndexer, featureIndexer, dt]) - -# Train model. This also runs the indexers. -model = pipeline.fit(trainingData) - -# Make predictions. -predictions = model.transform(testData) - -# Select example rows to display. -predictions.select("prediction", "indexedLabel", "features").show(5) - -# Select (prediction, true label) and compute test error -evaluator = MulticlassClassificationEvaluator( - labelCol="indexedLabel", predictionCol="prediction", metricName="precision") -accuracy = evaluator.evaluate(predictions) -print "Test Error = %g" % (1.0 - accuracy) - -treeModel = model.stages[2] -print treeModel # summary only -{% endhighlight %} -
    - -
    - - -## Regression - -The following examples load a dataset in LibSVM format, split it into training and test sets, train on the first dataset, and then evaluate on the held-out test set. -We use a feature transformer to index categorical features, adding metadata to the `DataFrame` which the Decision Tree algorithm can recognize. - -
    -
    - -More details on parameters can be found in the [Scala API documentation](api/scala/index.html#org.apache.spark.ml.regression.DecisionTreeRegressor). - -{% highlight scala %} -import org.apache.spark.ml.Pipeline -import org.apache.spark.ml.regression.DecisionTreeRegressor -import org.apache.spark.ml.regression.DecisionTreeRegressionModel -import org.apache.spark.ml.feature.VectorIndexer -import org.apache.spark.ml.evaluation.RegressionEvaluator -import org.apache.spark.mllib.util.MLUtils - -// Load and parse the data file, converting it to a DataFrame. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() - -// Automatically identify categorical features, and index them. -// Here, we treat features with > 4 distinct values as continuous. -val featureIndexer = new VectorIndexer() - .setInputCol("features") - .setOutputCol("indexedFeatures") - .setMaxCategories(4) - .fit(data) - -// Split the data into training and test sets (30% held out for testing) -val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) - -// Train a DecisionTree model. -val dt = new DecisionTreeRegressor() - .setLabelCol("label") - .setFeaturesCol("indexedFeatures") - -// Chain indexer and tree in a Pipeline -val pipeline = new Pipeline() - .setStages(Array(featureIndexer, dt)) - -// Train model. This also runs the indexer. -val model = pipeline.fit(trainingData) - -// Make predictions. -val predictions = model.transform(testData) - -// Select example rows to display. -predictions.select("prediction", "label", "features").show(5) - -// Select (prediction, true label) and compute test error -val evaluator = new RegressionEvaluator() - .setLabelCol("label") - .setPredictionCol("prediction") - .setMetricName("rmse") -val rmse = evaluator.evaluate(predictions) -println("Root Mean Squared Error (RMSE) on test data = " + rmse) - -val treeModel = model.stages(1).asInstanceOf[DecisionTreeRegressionModel] -println("Learned regression tree model:\n" + treeModel.toDebugString) -{% endhighlight %} -
    - -
    - -More details on parameters can be found in the [Java API documentation](api/java/org/apache/spark/ml/regression/DecisionTreeRegressor.html). - -{% highlight java %} -import org.apache.spark.ml.Pipeline; -import org.apache.spark.ml.PipelineModel; -import org.apache.spark.ml.PipelineStage; -import org.apache.spark.ml.evaluation.RegressionEvaluator; -import org.apache.spark.ml.feature.VectorIndexer; -import org.apache.spark.ml.feature.VectorIndexerModel; -import org.apache.spark.ml.regression.DecisionTreeRegressionModel; -import org.apache.spark.ml.regression.DecisionTreeRegressor; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.rdd.RDD; -import org.apache.spark.sql.DataFrame; - -// Load and parse the data file, converting it to a DataFrame. -RDD rdd = MLUtils.loadLibSVMFile(sc.sc(), "data/mllib/sample_libsvm_data.txt"); -DataFrame data = jsql.createDataFrame(rdd, LabeledPoint.class); - -// Automatically identify categorical features, and index them. -// Set maxCategories so features with > 4 distinct values are treated as continuous. -VectorIndexerModel featureIndexer = new VectorIndexer() - .setInputCol("features") - .setOutputCol("indexedFeatures") - .setMaxCategories(4) - .fit(data); - -// Split the data into training and test sets (30% held out for testing) -DataFrame[] splits = data.randomSplit(new double[] {0.7, 0.3}); -DataFrame trainingData = splits[0]; -DataFrame testData = splits[1]; - -// Train a DecisionTree model. -DecisionTreeRegressor dt = new DecisionTreeRegressor() - .setFeaturesCol("indexedFeatures"); - -// Chain indexer and tree in a Pipeline -Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[] {featureIndexer, dt}); - -// Train model. This also runs the indexer. -PipelineModel model = pipeline.fit(trainingData); - -// Make predictions. -DataFrame predictions = model.transform(testData); - -// Select example rows to display. -predictions.select("label", "features").show(5); - -// Select (prediction, true label) and compute test error -RegressionEvaluator evaluator = new RegressionEvaluator() - .setLabelCol("label") - .setPredictionCol("prediction") - .setMetricName("rmse"); -double rmse = evaluator.evaluate(predictions); -System.out.println("Root Mean Squared Error (RMSE) on test data = " + rmse); - -DecisionTreeRegressionModel treeModel = - (DecisionTreeRegressionModel)(model.stages()[1]); -System.out.println("Learned regression tree model:\n" + treeModel.toDebugString()); -{% endhighlight %} -
    - -
    - -More details on parameters can be found in the [Python API documentation](api/python/pyspark.ml.html#pyspark.ml.regression.DecisionTreeRegressor). - -{% highlight python %} -from pyspark.ml import Pipeline -from pyspark.ml.regression import DecisionTreeRegressor -from pyspark.ml.feature import VectorIndexer -from pyspark.ml.evaluation import RegressionEvaluator -from pyspark.mllib.util import MLUtils - -# Load and parse the data file, converting it to a DataFrame. -data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() - -# Automatically identify categorical features, and index them. -# We specify maxCategories so features with > 4 distinct values are treated as continuous. -featureIndexer =\ - VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) - -# Split the data into training and test sets (30% held out for testing) -(trainingData, testData) = data.randomSplit([0.7, 0.3]) - -# Train a DecisionTree model. -dt = DecisionTreeRegressor(featuresCol="indexedFeatures") - -# Chain indexer and tree in a Pipeline -pipeline = Pipeline(stages=[featureIndexer, dt]) - -# Train model. This also runs the indexer. -model = pipeline.fit(trainingData) - -# Make predictions. -predictions = model.transform(testData) - -# Select example rows to display. -predictions.select("prediction", "label", "features").show(5) - -# Select (prediction, true label) and compute test error -evaluator = RegressionEvaluator( - labelCol="label", predictionCol="prediction", metricName="rmse") -rmse = evaluator.evaluate(predictions) -print "Root Mean Squared Error (RMSE) on test data = %g" % rmse - -treeModel = model.stages[1] -print treeModel # summary only -{% endhighlight %} -
    - -
    + > This section has been moved into the + [classification and regression section](ml-classification-regression.html#decision-trees). diff --git a/docs/ml-ensembles.md b/docs/ml-ensembles.md index 58f566c9b4b5..97f1bdc803d0 100644 --- a/docs/ml-ensembles.md +++ b/docs/ml-ensembles.md @@ -1,1044 +1,8 @@ --- layout: global -title: Ensembles -displayTitle: ML - Ensembles +title: Tree ensemble methods +displayTitle: Tree ensemble methods --- -**Table of Contents** - -* This will become a table of contents (this text will be scraped). -{:toc} - -An [ensemble method](http://en.wikipedia.org/wiki/Ensemble_learning) -is a learning algorithm which creates a model composed of a set of other base models. - -## Tree Ensembles - -The Pipelines API supports two major tree ensemble algorithms: [Random Forests](http://en.wikipedia.org/wiki/Random_forest) and [Gradient-Boosted Trees (GBTs)](http://en.wikipedia.org/wiki/Gradient_boosting). -Both use [MLlib decision trees](ml-decision-tree.html) as their base models. - -Users can find more information about ensemble algorithms in the [MLlib Ensemble guide](mllib-ensembles.html). In this section, we demonstrate the Pipelines API for ensembles. - -The main differences between this API and the [original MLlib ensembles API](mllib-ensembles.html) are: -* support for ML Pipelines -* separation of classification vs. regression -* use of DataFrame metadata to distinguish continuous and categorical features -* a bit more functionality for random forests: estimates of feature importance, as well as the predicted probability of each class (a.k.a. class conditional probabilities) for classification. - -### Random Forests - -[Random forests](http://en.wikipedia.org/wiki/Random_forest) -are ensembles of [decision trees](ml-decision-tree.html). -Random forests combine many decision trees in order to reduce the risk of overfitting. -MLlib supports random forests for binary and multiclass classification and for regression, -using both continuous and categorical features. - -This section gives examples of using random forests with the Pipelines API. -For more information on the algorithm, please see the [main MLlib docs on random forests](mllib-ensembles.html). - -#### Inputs and Outputs - -We list the input and output (prediction) column types here. -All output columns are optional; to exclude an output column, set its corresponding Param to an empty string. - -##### Input Columns - - - - - - - - - - - - - - - - - - - - - - - - -
    Param nameType(s)DefaultDescription
    labelColDouble"label"Label to predict
    featuresColVector"features"Feature vector
    - -##### Output Columns (Predictions) - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Param nameType(s)DefaultDescriptionNotes
    predictionColDouble"prediction"Predicted label
    rawPredictionColVector"rawPrediction"Vector of length # classes, with the counts of training instance labels at the tree node which makes the predictionClassification only
    probabilityColVector"probability"Vector of length # classes equal to rawPrediction normalized to a multinomial distributionClassification only
    - -#### Example: Classification - -The following examples load a dataset in LibSVM format, split it into training and test sets, train on the first dataset, and then evaluate on the held-out test set. -We use two feature transformers to prepare the data; these help index categories for the label and categorical features, adding metadata to the `DataFrame` which the tree-based algorithms can recognize. - -
    -
    - -Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.classification.RandomForestClassifier) for more details. - -{% highlight scala %} -import org.apache.spark.ml.Pipeline -import org.apache.spark.ml.classification.RandomForestClassifier -import org.apache.spark.ml.classification.RandomForestClassificationModel -import org.apache.spark.ml.feature.{StringIndexer, IndexToString, VectorIndexer} -import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator - -// Load and parse the data file, converting it to a DataFrame. -val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - -// Index labels, adding metadata to the label column. -// Fit on whole dataset to include all labels in index. -val labelIndexer = new StringIndexer() - .setInputCol("label") - .setOutputCol("indexedLabel") - .fit(data) -// Automatically identify categorical features, and index them. -// Set maxCategories so features with > 4 distinct values are treated as continuous. -val featureIndexer = new VectorIndexer() - .setInputCol("features") - .setOutputCol("indexedFeatures") - .setMaxCategories(4) - .fit(data) - -// Split the data into training and test sets (30% held out for testing) -val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) - -// Train a RandomForest model. -val rf = new RandomForestClassifier() - .setLabelCol("indexedLabel") - .setFeaturesCol("indexedFeatures") - .setNumTrees(10) - -// Convert indexed labels back to original labels. -val labelConverter = new IndexToString() - .setInputCol("prediction") - .setOutputCol("predictedLabel") - .setLabels(labelIndexer.labels) - -// Chain indexers and forest in a Pipeline -val pipeline = new Pipeline() - .setStages(Array(labelIndexer, featureIndexer, rf, labelConverter)) - -// Train model. This also runs the indexers. -val model = pipeline.fit(trainingData) - -// Make predictions. -val predictions = model.transform(testData) - -// Select example rows to display. -predictions.select("predictedLabel", "label", "features").show(5) - -// Select (prediction, true label) and compute test error -val evaluator = new MulticlassClassificationEvaluator() - .setLabelCol("indexedLabel") - .setPredictionCol("prediction") - .setMetricName("precision") -val accuracy = evaluator.evaluate(predictions) -println("Test Error = " + (1.0 - accuracy)) - -val rfModel = model.stages(2).asInstanceOf[RandomForestClassificationModel] -println("Learned classification forest model:\n" + rfModel.toDebugString) -{% endhighlight %} -
    - -
    - -Refer to the [Java API docs](api/java/org/apache/spark/ml/classification/RandomForestClassifier.html) for more details. - -{% highlight java %} -import org.apache.spark.ml.Pipeline; -import org.apache.spark.ml.PipelineModel; -import org.apache.spark.ml.PipelineStage; -import org.apache.spark.ml.classification.RandomForestClassifier; -import org.apache.spark.ml.classification.RandomForestClassificationModel; -import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; -import org.apache.spark.ml.feature.*; -import org.apache.spark.sql.DataFrame; - -// Load and parse the data file, converting it to a DataFrame. -DataFrame data = sqlContext.read.format("libsvm") - .load("data/mllib/sample_libsvm_data.txt"); - -// Index labels, adding metadata to the label column. -// Fit on whole dataset to include all labels in index. -StringIndexerModel labelIndexer = new StringIndexer() - .setInputCol("label") - .setOutputCol("indexedLabel") - .fit(data); -// Automatically identify categorical features, and index them. -// Set maxCategories so features with > 4 distinct values are treated as continuous. -VectorIndexerModel featureIndexer = new VectorIndexer() - .setInputCol("features") - .setOutputCol("indexedFeatures") - .setMaxCategories(4) - .fit(data); - -// Split the data into training and test sets (30% held out for testing) -DataFrame[] splits = data.randomSplit(new double[] {0.7, 0.3}); -DataFrame trainingData = splits[0]; -DataFrame testData = splits[1]; - -// Train a RandomForest model. -RandomForestClassifier rf = new RandomForestClassifier() - .setLabelCol("indexedLabel") - .setFeaturesCol("indexedFeatures"); - -// Convert indexed labels back to original labels. -IndexToString labelConverter = new IndexToString() - .setInputCol("prediction") - .setOutputCol("predictedLabel") - .setLabels(labelIndexer.labels()); - -// Chain indexers and forest in a Pipeline -Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[] {labelIndexer, featureIndexer, rf, labelConverter}); - -// Train model. This also runs the indexers. -PipelineModel model = pipeline.fit(trainingData); - -// Make predictions. -DataFrame predictions = model.transform(testData); - -// Select example rows to display. -predictions.select("predictedLabel", "label", "features").show(5); - -// Select (prediction, true label) and compute test error -MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() - .setLabelCol("indexedLabel") - .setPredictionCol("prediction") - .setMetricName("precision"); -double accuracy = evaluator.evaluate(predictions); -System.out.println("Test Error = " + (1.0 - accuracy)); - -RandomForestClassificationModel rfModel = - (RandomForestClassificationModel)(model.stages()[2]); -System.out.println("Learned classification forest model:\n" + rfModel.toDebugString()); -{% endhighlight %} -
    - -
    - -Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classification.RandomForestClassifier) for more details. - -{% highlight python %} -from pyspark.ml import Pipeline -from pyspark.ml.classification import RandomForestClassifier -from pyspark.ml.feature import StringIndexer, VectorIndexer -from pyspark.ml.evaluation import MulticlassClassificationEvaluator - -# Load and parse the data file, converting it to a DataFrame. -data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - -# Index labels, adding metadata to the label column. -# Fit on whole dataset to include all labels in index. -labelIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel").fit(data) -# Automatically identify categorical features, and index them. -# Set maxCategories so features with > 4 distinct values are treated as continuous. -featureIndexer =\ - VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) - -# Split the data into training and test sets (30% held out for testing) -(trainingData, testData) = data.randomSplit([0.7, 0.3]) - -# Train a RandomForest model. -rf = RandomForestClassifier(labelCol="indexedLabel", featuresCol="indexedFeatures") - -# Chain indexers and forest in a Pipeline -pipeline = Pipeline(stages=[labelIndexer, featureIndexer, rf]) - -# Train model. This also runs the indexers. -model = pipeline.fit(trainingData) - -# Make predictions. -predictions = model.transform(testData) - -# Select example rows to display. -predictions.select("prediction", "indexedLabel", "features").show(5) - -# Select (prediction, true label) and compute test error -evaluator = MulticlassClassificationEvaluator( - labelCol="indexedLabel", predictionCol="prediction", metricName="precision") -accuracy = evaluator.evaluate(predictions) -print "Test Error = %g" % (1.0 - accuracy) - -rfModel = model.stages[2] -print rfModel # summary only -{% endhighlight %} -
    -
    - -#### Example: Regression - -The following examples load a dataset in LibSVM format, split it into training and test sets, train on the first dataset, and then evaluate on the held-out test set. -We use a feature transformer to index categorical features, adding metadata to the `DataFrame` which the tree-based algorithms can recognize. - -
    -
    - -Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.regression.RandomForestRegressor) for more details. - -{% highlight scala %} -import org.apache.spark.ml.Pipeline -import org.apache.spark.ml.regression.RandomForestRegressor -import org.apache.spark.ml.regression.RandomForestRegressionModel -import org.apache.spark.ml.feature.VectorIndexer -import org.apache.spark.ml.evaluation.RegressionEvaluator - -// Load and parse the data file, converting it to a DataFrame. -val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - -// Automatically identify categorical features, and index them. -// Set maxCategories so features with > 4 distinct values are treated as continuous. -val featureIndexer = new VectorIndexer() - .setInputCol("features") - .setOutputCol("indexedFeatures") - .setMaxCategories(4) - .fit(data) - -// Split the data into training and test sets (30% held out for testing) -val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) - -// Train a RandomForest model. -val rf = new RandomForestRegressor() - .setLabelCol("label") - .setFeaturesCol("indexedFeatures") - -// Chain indexer and forest in a Pipeline -val pipeline = new Pipeline() - .setStages(Array(featureIndexer, rf)) - -// Train model. This also runs the indexer. -val model = pipeline.fit(trainingData) - -// Make predictions. -val predictions = model.transform(testData) - -// Select example rows to display. -predictions.select("prediction", "label", "features").show(5) - -// Select (prediction, true label) and compute test error -val evaluator = new RegressionEvaluator() - .setLabelCol("label") - .setPredictionCol("prediction") - .setMetricName("rmse") -val rmse = evaluator.evaluate(predictions) -println("Root Mean Squared Error (RMSE) on test data = " + rmse) - -val rfModel = model.stages(1).asInstanceOf[RandomForestRegressionModel] -println("Learned regression forest model:\n" + rfModel.toDebugString) -{% endhighlight %} -
    - -
    - -Refer to the [Java API docs](api/java/org/apache/spark/ml/regression/RandomForestRegressor.html) for more details. - -{% highlight java %} -import org.apache.spark.ml.Pipeline; -import org.apache.spark.ml.PipelineModel; -import org.apache.spark.ml.PipelineStage; -import org.apache.spark.ml.evaluation.RegressionEvaluator; -import org.apache.spark.ml.feature.VectorIndexer; -import org.apache.spark.ml.feature.VectorIndexerModel; -import org.apache.spark.ml.regression.RandomForestRegressionModel; -import org.apache.spark.ml.regression.RandomForestRegressor; -import org.apache.spark.sql.DataFrame; - -// Load and parse the data file, converting it to a DataFrame. -DataFrame data = sqlContext.read.format("libsvm") - .load("data/mllib/sample_libsvm_data.txt"); - -// Automatically identify categorical features, and index them. -// Set maxCategories so features with > 4 distinct values are treated as continuous. -VectorIndexerModel featureIndexer = new VectorIndexer() - .setInputCol("features") - .setOutputCol("indexedFeatures") - .setMaxCategories(4) - .fit(data); - -// Split the data into training and test sets (30% held out for testing) -DataFrame[] splits = data.randomSplit(new double[] {0.7, 0.3}); -DataFrame trainingData = splits[0]; -DataFrame testData = splits[1]; - -// Train a RandomForest model. -RandomForestRegressor rf = new RandomForestRegressor() - .setLabelCol("label") - .setFeaturesCol("indexedFeatures"); - -// Chain indexer and forest in a Pipeline -Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[] {featureIndexer, rf}); - -// Train model. This also runs the indexer. -PipelineModel model = pipeline.fit(trainingData); - -// Make predictions. -DataFrame predictions = model.transform(testData); - -// Select example rows to display. -predictions.select("prediction", "label", "features").show(5); - -// Select (prediction, true label) and compute test error -RegressionEvaluator evaluator = new RegressionEvaluator() - .setLabelCol("label") - .setPredictionCol("prediction") - .setMetricName("rmse"); -double rmse = evaluator.evaluate(predictions); -System.out.println("Root Mean Squared Error (RMSE) on test data = " + rmse); - -RandomForestRegressionModel rfModel = - (RandomForestRegressionModel)(model.stages()[1]); -System.out.println("Learned regression forest model:\n" + rfModel.toDebugString()); -{% endhighlight %} -
    - -
    - -Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression.RandomForestRegressor) for more details. - -{% highlight python %} -from pyspark.ml import Pipeline -from pyspark.ml.regression import RandomForestRegressor -from pyspark.ml.feature import VectorIndexer -from pyspark.ml.evaluation import RegressionEvaluator - -# Load and parse the data file, converting it to a DataFrame. -data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - -# Automatically identify categorical features, and index them. -# Set maxCategories so features with > 4 distinct values are treated as continuous. -featureIndexer =\ - VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) - -# Split the data into training and test sets (30% held out for testing) -(trainingData, testData) = data.randomSplit([0.7, 0.3]) - -# Train a RandomForest model. -rf = RandomForestRegressor(featuresCol="indexedFeatures") - -# Chain indexer and forest in a Pipeline -pipeline = Pipeline(stages=[featureIndexer, rf]) - -# Train model. This also runs the indexer. -model = pipeline.fit(trainingData) - -# Make predictions. -predictions = model.transform(testData) - -# Select example rows to display. -predictions.select("prediction", "label", "features").show(5) - -# Select (prediction, true label) and compute test error -evaluator = RegressionEvaluator( - labelCol="label", predictionCol="prediction", metricName="rmse") -rmse = evaluator.evaluate(predictions) -print "Root Mean Squared Error (RMSE) on test data = %g" % rmse - -rfModel = model.stages[1] -print rfModel # summary only -{% endhighlight %} -
    -
    - -### Gradient-Boosted Trees (GBTs) - -[Gradient-Boosted Trees (GBTs)](http://en.wikipedia.org/wiki/Gradient_boosting) -are ensembles of [decision trees](ml-decision-tree.html). -GBTs iteratively train decision trees in order to minimize a loss function. -MLlib supports GBTs for binary classification and for regression, -using both continuous and categorical features. - -This section gives examples of using GBTs with the Pipelines API. -For more information on the algorithm, please see the [main MLlib docs on GBTs](mllib-ensembles.html). - -#### Inputs and Outputs - -We list the input and output (prediction) column types here. -All output columns are optional; to exclude an output column, set its corresponding Param to an empty string. - -##### Input Columns - - - - - - - - - - - - - - - - - - - - - - - - -
    Param nameType(s)DefaultDescription
    labelColDouble"label"Label to predict
    featuresColVector"features"Feature vector
    - -Note that `GBTClassifier` currently only supports binary labels. - -##### Output Columns (Predictions) - - - - - - - - - - - - - - - - - - - - -
    Param nameType(s)DefaultDescriptionNotes
    predictionColDouble"prediction"Predicted label
    - -In the future, `GBTClassifier` will also output columns for `rawPrediction` and `probability`, just as `RandomForestClassifier` does. - -#### Example: Classification - -The following examples load a dataset in LibSVM format, split it into training and test sets, train on the first dataset, and then evaluate on the held-out test set. -We use two feature transformers to prepare the data; these help index categories for the label and categorical features, adding metadata to the `DataFrame` which the tree-based algorithms can recognize. - -
    -
    - -Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.classification.GBTClassifier) for more details. - -{% highlight scala %} -import org.apache.spark.ml.Pipeline -import org.apache.spark.ml.classification.GBTClassifier -import org.apache.spark.ml.classification.GBTClassificationModel -import org.apache.spark.ml.feature.{StringIndexer, IndexToString, VectorIndexer} -import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator - -// Load and parse the data file, converting it to a DataFrame. -val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - -// Index labels, adding metadata to the label column. -// Fit on whole dataset to include all labels in index. -val labelIndexer = new StringIndexer() - .setInputCol("label") - .setOutputCol("indexedLabel") - .fit(data) -// Automatically identify categorical features, and index them. -// Set maxCategories so features with > 4 distinct values are treated as continuous. -val featureIndexer = new VectorIndexer() - .setInputCol("features") - .setOutputCol("indexedFeatures") - .setMaxCategories(4) - .fit(data) - -// Split the data into training and test sets (30% held out for testing) -val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) - -// Train a GBT model. -val gbt = new GBTClassifier() - .setLabelCol("indexedLabel") - .setFeaturesCol("indexedFeatures") - .setMaxIter(10) - -// Convert indexed labels back to original labels. -val labelConverter = new IndexToString() - .setInputCol("prediction") - .setOutputCol("predictedLabel") - .setLabels(labelIndexer.labels) - -// Chain indexers and GBT in a Pipeline -val pipeline = new Pipeline() - .setStages(Array(labelIndexer, featureIndexer, gbt, labelConverter)) - -// Train model. This also runs the indexers. -val model = pipeline.fit(trainingData) - -// Make predictions. -val predictions = model.transform(testData) - -// Select example rows to display. -predictions.select("predictedLabel", "label", "features").show(5) - -// Select (prediction, true label) and compute test error -val evaluator = new MulticlassClassificationEvaluator() - .setLabelCol("indexedLabel") - .setPredictionCol("prediction") - .setMetricName("precision") -val accuracy = evaluator.evaluate(predictions) -println("Test Error = " + (1.0 - accuracy)) - -val gbtModel = model.stages(2).asInstanceOf[GBTClassificationModel] -println("Learned classification GBT model:\n" + gbtModel.toDebugString) -{% endhighlight %} -
    - -
    - -Refer to the [Java API docs](api/java/org/apache/spark/ml/classification/GBTClassifier.html) for more details. - -{% highlight java %} -import org.apache.spark.ml.Pipeline; -import org.apache.spark.ml.PipelineModel; -import org.apache.spark.ml.PipelineStage; -import org.apache.spark.ml.classification.GBTClassifier; -import org.apache.spark.ml.classification.GBTClassificationModel; -import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; -import org.apache.spark.ml.feature.*; -import org.apache.spark.sql.DataFrame; - -// Load and parse the data file, converting it to a DataFrame. -DataFrame data sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt"); - -// Index labels, adding metadata to the label column. -// Fit on whole dataset to include all labels in index. -StringIndexerModel labelIndexer = new StringIndexer() - .setInputCol("label") - .setOutputCol("indexedLabel") - .fit(data); -// Automatically identify categorical features, and index them. -// Set maxCategories so features with > 4 distinct values are treated as continuous. -VectorIndexerModel featureIndexer = new VectorIndexer() - .setInputCol("features") - .setOutputCol("indexedFeatures") - .setMaxCategories(4) - .fit(data); - -// Split the data into training and test sets (30% held out for testing) -DataFrame[] splits = data.randomSplit(new double[] {0.7, 0.3}); -DataFrame trainingData = splits[0]; -DataFrame testData = splits[1]; - -// Train a GBT model. -GBTClassifier gbt = new GBTClassifier() - .setLabelCol("indexedLabel") - .setFeaturesCol("indexedFeatures") - .setMaxIter(10); - -// Convert indexed labels back to original labels. -IndexToString labelConverter = new IndexToString() - .setInputCol("prediction") - .setOutputCol("predictedLabel") - .setLabels(labelIndexer.labels()); - -// Chain indexers and GBT in a Pipeline -Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[] {labelIndexer, featureIndexer, gbt, labelConverter}); - -// Train model. This also runs the indexers. -PipelineModel model = pipeline.fit(trainingData); - -// Make predictions. -DataFrame predictions = model.transform(testData); - -// Select example rows to display. -predictions.select("predictedLabel", "label", "features").show(5); - -// Select (prediction, true label) and compute test error -MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() - .setLabelCol("indexedLabel") - .setPredictionCol("prediction") - .setMetricName("precision"); -double accuracy = evaluator.evaluate(predictions); -System.out.println("Test Error = " + (1.0 - accuracy)); - -GBTClassificationModel gbtModel = - (GBTClassificationModel)(model.stages()[2]); -System.out.println("Learned classification GBT model:\n" + gbtModel.toDebugString()); -{% endhighlight %} -
    - -
    - -Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classification.GBTClassifier) for more details. - -{% highlight python %} -from pyspark.ml import Pipeline -from pyspark.ml.classification import GBTClassifier -from pyspark.ml.feature import StringIndexer, VectorIndexer -from pyspark.ml.evaluation import MulticlassClassificationEvaluator - -# Load and parse the data file, converting it to a DataFrame. -data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - -# Index labels, adding metadata to the label column. -# Fit on whole dataset to include all labels in index. -labelIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel").fit(data) -# Automatically identify categorical features, and index them. -# Set maxCategories so features with > 4 distinct values are treated as continuous. -featureIndexer =\ - VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) - -# Split the data into training and test sets (30% held out for testing) -(trainingData, testData) = data.randomSplit([0.7, 0.3]) - -# Train a GBT model. -gbt = GBTClassifier(labelCol="indexedLabel", featuresCol="indexedFeatures", maxIter=10) - -# Chain indexers and GBT in a Pipeline -pipeline = Pipeline(stages=[labelIndexer, featureIndexer, gbt]) - -# Train model. This also runs the indexers. -model = pipeline.fit(trainingData) - -# Make predictions. -predictions = model.transform(testData) - -# Select example rows to display. -predictions.select("prediction", "indexedLabel", "features").show(5) - -# Select (prediction, true label) and compute test error -evaluator = MulticlassClassificationEvaluator( - labelCol="indexedLabel", predictionCol="prediction", metricName="precision") -accuracy = evaluator.evaluate(predictions) -print "Test Error = %g" % (1.0 - accuracy) - -gbtModel = model.stages[2] -print gbtModel # summary only -{% endhighlight %} -
    -
    - -#### Example: Regression - -Note: For this example dataset, `GBTRegressor` actually only needs 1 iteration, but that will not -be true in general. - -
    -
    - -Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.regression.GBTRegressor) for more details. - -{% highlight scala %} -import org.apache.spark.ml.Pipeline -import org.apache.spark.ml.regression.GBTRegressor -import org.apache.spark.ml.regression.GBTRegressionModel -import org.apache.spark.ml.feature.VectorIndexer -import org.apache.spark.ml.evaluation.RegressionEvaluator - -// Load and parse the data file, converting it to a DataFrame. -val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - -// Automatically identify categorical features, and index them. -// Set maxCategories so features with > 4 distinct values are treated as continuous. -val featureIndexer = new VectorIndexer() - .setInputCol("features") - .setOutputCol("indexedFeatures") - .setMaxCategories(4) - .fit(data) - -// Split the data into training and test sets (30% held out for testing) -val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) - -// Train a GBT model. -val gbt = new GBTRegressor() - .setLabelCol("label") - .setFeaturesCol("indexedFeatures") - .setMaxIter(10) - -// Chain indexer and GBT in a Pipeline -val pipeline = new Pipeline() - .setStages(Array(featureIndexer, gbt)) - -// Train model. This also runs the indexer. -val model = pipeline.fit(trainingData) - -// Make predictions. -val predictions = model.transform(testData) - -// Select example rows to display. -predictions.select("prediction", "label", "features").show(5) - -// Select (prediction, true label) and compute test error -val evaluator = new RegressionEvaluator() - .setLabelCol("label") - .setPredictionCol("prediction") - .setMetricName("rmse") -val rmse = evaluator.evaluate(predictions) -println("Root Mean Squared Error (RMSE) on test data = " + rmse) - -val gbtModel = model.stages(1).asInstanceOf[GBTRegressionModel] -println("Learned regression GBT model:\n" + gbtModel.toDebugString) -{% endhighlight %} -
    - -
    - -Refer to the [Java API docs](api/java/org/apache/spark/ml/regression/GBTRegressor.html) for more details. - -{% highlight java %} -import org.apache.spark.ml.Pipeline; -import org.apache.spark.ml.PipelineModel; -import org.apache.spark.ml.PipelineStage; -import org.apache.spark.ml.evaluation.RegressionEvaluator; -import org.apache.spark.ml.feature.VectorIndexer; -import org.apache.spark.ml.feature.VectorIndexerModel; -import org.apache.spark.ml.regression.GBTRegressionModel; -import org.apache.spark.ml.regression.GBTRegressor; -import org.apache.spark.sql.DataFrame; - -// Load and parse the data file, converting it to a DataFrame. -DataFrame data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt"); - -// Automatically identify categorical features, and index them. -// Set maxCategories so features with > 4 distinct values are treated as continuous. -VectorIndexerModel featureIndexer = new VectorIndexer() - .setInputCol("features") - .setOutputCol("indexedFeatures") - .setMaxCategories(4) - .fit(data); - -// Split the data into training and test sets (30% held out for testing) -DataFrame[] splits = data.randomSplit(new double[] {0.7, 0.3}); -DataFrame trainingData = splits[0]; -DataFrame testData = splits[1]; - -// Train a GBT model. -GBTRegressor gbt = new GBTRegressor() - .setLabelCol("label") - .setFeaturesCol("indexedFeatures") - .setMaxIter(10); - -// Chain indexer and GBT in a Pipeline -Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[] {featureIndexer, gbt}); - -// Train model. This also runs the indexer. -PipelineModel model = pipeline.fit(trainingData); - -// Make predictions. -DataFrame predictions = model.transform(testData); - -// Select example rows to display. -predictions.select("prediction", "label", "features").show(5); - -// Select (prediction, true label) and compute test error -RegressionEvaluator evaluator = new RegressionEvaluator() - .setLabelCol("label") - .setPredictionCol("prediction") - .setMetricName("rmse"); -double rmse = evaluator.evaluate(predictions); -System.out.println("Root Mean Squared Error (RMSE) on test data = " + rmse); - -GBTRegressionModel gbtModel = - (GBTRegressionModel)(model.stages()[1]); -System.out.println("Learned regression GBT model:\n" + gbtModel.toDebugString()); -{% endhighlight %} -
    - -
    - -Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression.GBTRegressor) for more details. - -{% highlight python %} -from pyspark.ml import Pipeline -from pyspark.ml.regression import GBTRegressor -from pyspark.ml.feature import VectorIndexer -from pyspark.ml.evaluation import RegressionEvaluator - -# Load and parse the data file, converting it to a DataFrame. -data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - -# Automatically identify categorical features, and index them. -# Set maxCategories so features with > 4 distinct values are treated as continuous. -featureIndexer =\ - VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) - -# Split the data into training and test sets (30% held out for testing) -(trainingData, testData) = data.randomSplit([0.7, 0.3]) - -# Train a GBT model. -gbt = GBTRegressor(featuresCol="indexedFeatures", maxIter=10) - -# Chain indexer and GBT in a Pipeline -pipeline = Pipeline(stages=[featureIndexer, gbt]) - -# Train model. This also runs the indexer. -model = pipeline.fit(trainingData) - -# Make predictions. -predictions = model.transform(testData) - -# Select example rows to display. -predictions.select("prediction", "label", "features").show(5) - -# Select (prediction, true label) and compute test error -evaluator = RegressionEvaluator( - labelCol="label", predictionCol="prediction", metricName="rmse") -rmse = evaluator.evaluate(predictions) -print "Root Mean Squared Error (RMSE) on test data = %g" % rmse - -gbtModel = model.stages[1] -print gbtModel # summary only -{% endhighlight %} -
    -
    - - -## One-vs-Rest (a.k.a. One-vs-All) - -[OneVsRest](http://en.wikipedia.org/wiki/Multiclass_classification#One-vs.-rest) is an example of a machine learning reduction for performing multiclass classification given a base classifier that can perform binary classification efficiently. It is also known as "One-vs-All." - -`OneVsRest` is implemented as an `Estimator`. For the base classifier it takes instances of `Classifier` and creates a binary classification problem for each of the k classes. The classifier for class i is trained to predict whether the label is i or not, distinguishing class i from all other classes. - -Predictions are done by evaluating each binary classifier and the index of the most confident classifier is output as label. - -### Example - -The example below demonstrates how to load the -[Iris dataset](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/multiclass/iris.scale), parse it as a DataFrame and perform multiclass classification using `OneVsRest`. The test error is calculated to measure the algorithm accuracy. - -
    -
    - -Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.classifier.OneVsRest) for more details. - -{% highlight scala %} -import org.apache.spark.ml.classification.{LogisticRegression, OneVsRest} -import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.sql.{Row, SQLContext} - -val sqlContext = new SQLContext(sc) - -// parse data into dataframe -val data = sqlContext.read.format("libsvm") - .load("data/mllib/sample_multiclass_classification_data.txt") -val Array(train, test) = data.randomSplit(Array(0.7, 0.3)) - -// instantiate multiclass learner and train -val ovr = new OneVsRest().setClassifier(new LogisticRegression) - -val ovrModel = ovr.fit(train) - -// score model on test data -val predictions = ovrModel.transform(test).select("prediction", "label") -val predictionsAndLabels = predictions.map {case Row(p: Double, l: Double) => (p, l)} - -// compute confusion matrix -val metrics = new MulticlassMetrics(predictionsAndLabels) -println(metrics.confusionMatrix) - -// the Iris DataSet has three classes -val numClasses = 3 - -println("label\tfpr\n") -(0 until numClasses).foreach { index => - val label = index.toDouble - println(label + "\t" + metrics.falsePositiveRate(label)) -} -{% endhighlight %} -
    - -
    - -Refer to the [Java API docs](api/java/org/apache/spark/ml/classification/OneVsRest.html) for more details. - -{% highlight java %} -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.classification.LogisticRegression; -import org.apache.spark.ml.classification.OneVsRest; -import org.apache.spark.ml.classification.OneVsRestModel; -import org.apache.spark.mllib.evaluation.MulticlassMetrics; -import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; - -SparkConf conf = new SparkConf().setAppName("JavaOneVsRestExample"); -JavaSparkContext jsc = new JavaSparkContext(conf); -SQLContext jsql = new SQLContext(jsc); - -DataFrame dataFrame = sqlContext.read.format("libsvm") - .load("data/mllib/sample_multiclass_classification_data.txt"); - -DataFrame[] splits = dataFrame.randomSplit(new double[] {0.7, 0.3}, 12345); -DataFrame train = splits[0]; -DataFrame test = splits[1]; - -// instantiate the One Vs Rest Classifier -OneVsRest ovr = new OneVsRest().setClassifier(new LogisticRegression()); - -// train the multiclass model -OneVsRestModel ovrModel = ovr.fit(train.cache()); - -// score the model on test data -DataFrame predictions = ovrModel - .transform(test) - .select("prediction", "label"); - -// obtain metrics -MulticlassMetrics metrics = new MulticlassMetrics(predictions); -Matrix confusionMatrix = metrics.confusionMatrix(); - -// output the Confusion Matrix -System.out.println("Confusion Matrix"); -System.out.println(confusionMatrix); - -// compute the false positive rate per label -System.out.println(); -System.out.println("label\tfpr\n"); - -// the Iris DataSet has three classes -int numClasses = 3; -for (int index = 0; index < numClasses; index++) { - double label = (double) index; - System.out.print(label); - System.out.print("\t"); - System.out.print(metrics.falsePositiveRate(label)); - System.out.println(); -} -{% endhighlight %} -
    -
    + > This section has been moved into the + [classification and regression section](ml-classification-regression.html#tree-ensembles). diff --git a/docs/ml-features.md b/docs/ml-features.md index 142afac2f3f9..86a0e09997b8 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -1,7 +1,7 @@ --- layout: global -title: Feature Extraction, Transformation, and Selection - SparkML -displayTitle: ML - Features +title: Extracting, transforming and selecting features +displayTitle: Extracting, transforming and selecting features --- This section covers algorithms for working with features, roughly divided into these groups: @@ -9,6 +9,7 @@ This section covers algorithms for working with features, roughly divided into t * Extraction: Extracting features from "raw" data * Transformation: Scaling, converting, or modifying features * Selection: Selecting a subset from a larger set of features +* Locality Sensitive Hashing (LSH): This class of algorithms combines aspects of feature transformation with other algorithms. **Table of Contents** @@ -18,18 +19,64 @@ This section covers algorithms for working with features, roughly divided into t # Feature Extractors -## TF-IDF (HashingTF and IDF) - -[Term Frequency-Inverse Document Frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) is a common text pre-processing step. In Spark ML, TF-IDF is separate into two parts: TF (+hashing) and IDF. - -**TF**: `HashingTF` is a `Transformer` which takes sets of terms and converts those sets into fixed-length feature vectors. In text processing, a "set of terms" might be a bag of words. -The algorithm combines Term Frequency (TF) counts with the [hashing trick](http://en.wikipedia.org/wiki/Feature_hashing) for dimensionality reduction. - -**IDF**: `IDF` is an `Estimator` which fits on a dataset and produces an `IDFModel`. The `IDFModel` takes feature vectors (generally created from `HashingTF`) and scales each column. Intuitively, it down-weights columns which appear frequently in a corpus. +## TF-IDF + +[Term frequency-inverse document frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) +is a feature vectorization method widely used in text mining to reflect the importance of a term +to a document in the corpus. Denote a term by `$t$`, a document by `$d$`, and the corpus by `$D$`. +Term frequency `$TF(t, d)$` is the number of times that term `$t$` appears in document `$d$`, while +document frequency `$DF(t, D)$` is the number of documents that contains term `$t$`. If we only use +term frequency to measure the importance, it is very easy to over-emphasize terms that appear very +often but carry little information about the document, e.g. "a", "the", and "of". If a term appears +very often across the corpus, it means it doesn't carry special information about a particular document. +Inverse document frequency is a numerical measure of how much information a term provides: +`\[ +IDF(t, D) = \log \frac{|D| + 1}{DF(t, D) + 1}, +\]` +where `$|D|$` is the total number of documents in the corpus. Since logarithm is used, if a term +appears in all documents, its IDF value becomes 0. Note that a smoothing term is applied to avoid +dividing by zero for terms outside the corpus. The TF-IDF measure is simply the product of TF and IDF: +`\[ +TFIDF(t, d, D) = TF(t, d) \cdot IDF(t, D). +\]` +There are several variants on the definition of term frequency and document frequency. +In MLlib, we separate TF and IDF to make them flexible. + +**TF**: Both `HashingTF` and `CountVectorizer` can be used to generate the term frequency vectors. + +`HashingTF` is a `Transformer` which takes sets of terms and converts those sets into +fixed-length feature vectors. In text processing, a "set of terms" might be a bag of words. +`HashingTF` utilizes the [hashing trick](http://en.wikipedia.org/wiki/Feature_hashing). +A raw feature is mapped into an index (term) by applying a hash function. The hash function +used here is [MurmurHash 3](https://en.wikipedia.org/wiki/MurmurHash). Then term frequencies +are calculated based on the mapped indices. This approach avoids the need to compute a global +term-to-index map, which can be expensive for a large corpus, but it suffers from potential hash +collisions, where different raw features may become the same term after hashing. To reduce the +chance of collision, we can increase the target feature dimension, i.e. the number of buckets +of the hash table. Since a simple modulo on the hashed value is used to determine the vector index, +it is advisable to use a power of two as the feature dimension, otherwise the features will not +be mapped evenly to the vector indices. The default feature dimension is `$2^{18} = 262,144$`. +An optional binary toggle parameter controls term frequency counts. When set to true all nonzero +frequency counts are set to 1. This is especially useful for discrete probabilistic models that +model binary, rather than integer, counts. + +`CountVectorizer` converts text documents to vectors of term counts. Refer to [CountVectorizer +](ml-features.html#countvectorizer) for more details. + +**IDF**: `IDF` is an `Estimator` which is fit on a dataset and produces an `IDFModel`. The +`IDFModel` takes feature vectors (generally created from `HashingTF` or `CountVectorizer`) and +scales each feature. Intuitively, it down-weights features which appear frequently in a corpus. + +**Note:** `spark.ml` doesn't provide tools for text segmentation. +We refer users to the [Stanford NLP Group](http://nlp.stanford.edu/) and +[scalanlp/chalk](https://github.com/scalanlp/chalk). -Please refer to the [MLlib user guide on TF-IDF](mllib-feature-extraction.html#tf-idf) for more details on Term Frequency and Inverse Document Frequency. +**Examples** -In the following code segment, we start with a set of sentences. We split each sentence into words using `Tokenizer`. For each sentence (bag of words), we use `HashingTF` to hash the sentence into a feature vector. We use `IDF` to rescale the feature vectors; this generally improves performance when using text as features. Our feature vectors could then be passed to a learning algorithm. +In the following code segment, we start with a set of sentences. We split each sentence into words +using `Tokenizer`. For each sentence (bag of words), we use `HashingTF` to hash the sentence into +a feature vector. We use `IDF` to rescale the feature vectors; this generally improves performance +when using text as features. Our feature vectors could then be passed to a learning algorithm.
    @@ -62,10 +109,12 @@ the [IDF Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.IDF) for mor `Word2Vec` is an `Estimator` which takes sequences of words representing documents and trains a `Word2VecModel`. The model maps each word to a unique fixed-size vector. The `Word2VecModel` transforms each document into a vector using the average of all words in the document; this vector -can then be used for as features for prediction, document similarity calculations, etc. -Please refer to the [MLlib user guide on Word2Vec](mllib-feature-extraction.html#Word2Vec) for more +can then be used as features for prediction, document similarity calculations, etc. +Please refer to the [MLlib user guide on Word2Vec](mllib-feature-extraction.html#word2vec) for more details. +**Examples** + In the following code segment, we start with a set of documents, each of which is represented as a sequence of words. For each document, we transform it into a feature vector. This feature vector could then be passed to a learning algorithm.
    @@ -98,14 +147,16 @@ for more details on the API. `CountVectorizer` and `CountVectorizerModel` aim to help convert a collection of text documents to vectors of token counts. When an a-priori dictionary is not available, `CountVectorizer` can - be used as an `Estimator` to extract the vocabulary and generates a `CountVectorizerModel`. The + be used as an `Estimator` to extract the vocabulary, and generates a `CountVectorizerModel`. The model produces sparse representations for the documents over the vocabulary, which can then be passed to other algorithms like LDA. During the fitting process, `CountVectorizer` will select the top `vocabSize` words ordered by - term frequency across the corpus. An optional parameter "minDF" also affect the fitting process + term frequency across the corpus. An optional parameter `minDF` also affects the fitting process by specifying the minimum number (or fraction if < 1.0) of documents a term must appear in to be - included in the vocabulary. + included in the vocabulary. Another optional binary toggle parameter controls the output vector. + If set to true all nonzero counts are set to 1. This is especially useful for discrete probabilistic + models that model binary, rather than integer, counts. **Examples** @@ -118,9 +169,9 @@ Assume that we have the following DataFrame with columns `id` and `texts`: 1 | Array("a", "b", "b", "c", "a") ~~~~ -each row in`texts` is a document of type Array[String]. -Invoking fit of `CountVectorizer` produces a `CountVectorizerModel` with vocabulary (a, b, c), -then the output column "vector" after transformation contains: +each row in `texts` is a document of type Array[String]. +Invoking fit of `CountVectorizer` produces a `CountVectorizerModel` with vocabulary (a, b, c). +Then the output column "vector" after transformation contains: ~~~~ id | texts | vector @@ -129,7 +180,7 @@ then the output column "vector" after transformation contains: 1 | Array("a", "b", "b", "c", "a") | (3,[0,1,2],[2.0,2.0,1.0]) ~~~~ -each vector represents the token counts of the document over the vocabulary. +Each vector represents the token counts of the document over the vocabulary.
    @@ -149,6 +200,98 @@ for more details on the API. {% include_example java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java %}
    + +
    + +Refer to the [CountVectorizer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.CountVectorizer) +and the [CountVectorizerModel Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.CountVectorizerModel) +for more details on the API. + +{% include_example python/ml/count_vectorizer_example.py %} +
    +
    + +## FeatureHasher + +Feature hashing projects a set of categorical or numerical features into a feature vector of +specified dimension (typically substantially smaller than that of the original feature +space). This is done using the [hashing trick](https://en.wikipedia.org/wiki/Feature_hashing) +to map features to indices in the feature vector. + +The `FeatureHasher` transformer operates on multiple columns. Each column may contain either +numeric or categorical features. Behavior and handling of column data types is as follows: + +- Numeric columns: For numeric features, the hash value of the column name is used to map the +feature value to its index in the feature vector. Numeric features are never treated as +categorical, even when they are integers. You must explicitly convert numeric columns containing +categorical features to strings first. +- String columns: For categorical features, the hash value of the string "column_name=value" +is used to map to the vector index, with an indicator value of `1.0`. Thus, categorical features +are "one-hot" encoded (similarly to using [OneHotEncoder](ml-features.html#onehotencoder) with +`dropLast=false`). +- Boolean columns: Boolean values are treated in the same way as string columns. That is, +boolean features are represented as "column_name=true" or "column_name=false", with an indicator +value of `1.0`. + +Null (missing) values are ignored (implicitly zero in the resulting feature vector). + +The hash function used here is also the [MurmurHash 3](https://en.wikipedia.org/wiki/MurmurHash) +used in [HashingTF](ml-features.html#tf-idf). Since a simple modulo on the hashed value is used to +determine the vector index, it is advisable to use a power of two as the numFeatures parameter; +otherwise the features will not be mapped evenly to the vector indices. + +**Examples** + +Assume that we have a DataFrame with 4 input columns `real`, `bool`, `stringNum`, and `string`. +These different data types as input will illustrate the behavior of the transform to produce a +column of feature vectors. + +~~~~ +real| bool|stringNum|string +----|-----|---------|------ + 2.2| true| 1| foo + 3.3|false| 2| bar + 4.4|false| 3| baz + 5.5|false| 4| foo +~~~~ + +Then the output of `FeatureHasher.transform` on this DataFrame is: + +~~~~ +real|bool |stringNum|string|features +----|-----|---------|------|------------------------------------------------------- +2.2 |true |1 |foo |(262144,[51871, 63643,174475,253195],[1.0,1.0,2.2,1.0]) +3.3 |false|2 |bar |(262144,[6031, 80619,140467,174475],[1.0,1.0,1.0,3.3]) +4.4 |false|3 |baz |(262144,[24279,140467,174475,196810],[1.0,1.0,4.4,1.0]) +5.5 |false|4 |foo |(262144,[63643,140467,168512,174475],[1.0,1.0,1.0,5.5]) +~~~~ + +The resulting feature vectors could then be passed to a learning algorithm. + +
    +
    + +Refer to the [FeatureHasher Scala docs](api/scala/index.html#org.apache.spark.ml.feature.FeatureHasher) +for more details on the API. + +{% include_example scala/org/apache/spark/examples/ml/FeatureHasherExample.scala %} +
    + +
    + +Refer to the [FeatureHasher Java docs](api/java/org/apache/spark/ml/feature/FeatureHasher.html) +for more details on the API. + +{% include_example java/org/apache/spark/examples/ml/JavaFeatureHasherExample.java %} +
    + +
    + +Refer to the [FeatureHasher Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.FeatureHasher) +for more details on the API. + +{% include_example python/ml/feature_hasher_example.py %} +
    # Feature Transformers @@ -159,36 +302,20 @@ for more details on the API. [RegexTokenizer](api/scala/index.html#org.apache.spark.ml.feature.RegexTokenizer) allows more advanced tokenization based on regular expression (regex) matching. - By default, the parameter "pattern" (regex, default: \\s+) is used as delimiters to split the input text. + By default, the parameter "pattern" (regex, default: `"\\s+"`) is used as delimiters to split the input text. Alternatively, users can set parameter "gaps" to false indicating the regex "pattern" denotes "tokens" rather than splitting gaps, and find all matching occurrences as the tokenization result. +**Examples** +
    Refer to the [Tokenizer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.Tokenizer) -and the [RegexTokenizer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.Tokenizer) +and the [RegexTokenizer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.RegexTokenizer) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.{Tokenizer, RegexTokenizer} - -val sentenceDataFrame = sqlContext.createDataFrame(Seq( - (0, "Hi I heard about Spark"), - (1, "I wish Java could use case classes"), - (2, "Logistic,regression,models,are,neat") -)).toDF("label", "sentence") -val tokenizer = new Tokenizer().setInputCol("sentence").setOutputCol("words") -val regexTokenizer = new RegexTokenizer() - .setInputCol("sentence") - .setOutputCol("words") - .setPattern("\\W") // alternatively .setPattern("\\w+").setGaps(false) - -val tokenized = tokenizer.transform(sentenceDataFrame) -tokenized.select("words", "label").take(3).foreach(println) -val regexTokenized = regexTokenizer.transform(sentenceDataFrame) -regexTokenized.select("words", "label").take(3).foreach(println) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/TokenizerExample.scala %}
    @@ -197,67 +324,16 @@ Refer to the [Tokenizer Java docs](api/java/org/apache/spark/ml/feature/Tokenize and the [RegexTokenizer Java docs](api/java/org/apache/spark/ml/feature/RegexTokenizer.html) for more details on the API. -{% highlight java %} -import java.util.Arrays; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.ml.feature.RegexTokenizer; -import org.apache.spark.ml.feature.Tokenizer; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -JavaRDD jrdd = jsc.parallelize(Arrays.asList( - RowFactory.create(0, "Hi I heard about Spark"), - RowFactory.create(1, "I wish Java could use case classes"), - RowFactory.create(2, "Logistic,regression,models,are,neat") -)); -StructType schema = new StructType(new StructField[]{ - new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), - new StructField("sentence", DataTypes.StringType, false, Metadata.empty()) -}); -DataFrame sentenceDataFrame = sqlContext.createDataFrame(jrdd, schema); -Tokenizer tokenizer = new Tokenizer().setInputCol("sentence").setOutputCol("words"); -DataFrame wordsDataFrame = tokenizer.transform(sentenceDataFrame); -for (Row r : wordsDataFrame.select("words", "label").take(3)) { - java.util.List words = r.getList(0); - for (String word : words) System.out.print(word + " "); - System.out.println(); -} - -RegexTokenizer regexTokenizer = new RegexTokenizer() - .setInputCol("sentence") - .setOutputCol("words") - .setPattern("\\W"); // alternatively .setPattern("\\w+").setGaps(false); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaTokenizerExample.java %}
    Refer to the [Tokenizer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.Tokenizer) and -the the [RegexTokenizer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.RegexTokenizer) +the [RegexTokenizer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.RegexTokenizer) for more details on the API. -{% highlight python %} -from pyspark.ml.feature import Tokenizer, RegexTokenizer - -sentenceDataFrame = sqlContext.createDataFrame([ - (0, "Hi I heard about Spark"), - (1, "I wish Java could use case classes"), - (2, "Logistic,regression,models,are,neat") -], ["label", "sentence"]) -tokenizer = Tokenizer(inputCol="sentence", outputCol="words") -wordsDataFrame = tokenizer.transform(sentenceDataFrame) -for words_label in wordsDataFrame.select("words", "label").take(3): - print(words_label) -regexTokenizer = RegexTokenizer(inputCol="sentence", outputCol="words", pattern="\\W") -# alternatively, pattern="\\w+", gaps(False) -{% endhighlight %} +{% include_example python/ml/tokenizer_example.py %}
    @@ -269,11 +345,12 @@ frequently and don't carry as much meaning. `StopWordsRemover` takes as input a sequence of strings (e.g. the output of a [Tokenizer](ml-features.html#tokenizer)) and drops all the stop words from the input sequences. The list of stopwords is specified by -the `stopWords` parameter. We provide [a list of stop -words](http://ir.dcs.gla.ac.uk/resources/linguistic_utils/stop_words) by -default, accessible by calling `getStopWords` on a newly instantiated -`StopWordsRemover` instance. A boolean parameter `caseSensitive` indicates -if the matches should be case sensitive (false by default). +the `stopWords` parameter. Default stop words for some languages are accessible +by calling `StopWordsRemover.loadDefaultStopWords(language)`, for which available +options are "danish", "dutch", "english", "finnish", "french", "german", "hungarian", +"italian", "norwegian", "portuguese", "russian", "spanish", "swedish" and "turkish". +A boolean parameter `caseSensitive` indicates if the matches should be case sensitive +(false by default). **Examples** @@ -306,19 +383,7 @@ filtered out. Refer to the [StopWordsRemover Scala docs](api/scala/index.html#org.apache.spark.ml.feature.StopWordsRemover) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.StopWordsRemover - -val remover = new StopWordsRemover() - .setInputCol("raw") - .setOutputCol("filtered") -val dataSet = sqlContext.createDataFrame(Seq( - (0, Seq("I", "saw", "the", "red", "baloon")), - (1, Seq("Mary", "had", "a", "little", "lamb")) -)).toDF("id", "raw") - -remover.transform(dataSet).show() -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala %}
    @@ -326,34 +391,7 @@ remover.transform(dataSet).show() Refer to the [StopWordsRemover Java docs](api/java/org/apache/spark/ml/feature/StopWordsRemover.html) for more details on the API. -{% highlight java %} -import java.util.Arrays; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.ml.feature.StopWordsRemover; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -StopWordsRemover remover = new StopWordsRemover() - .setInputCol("raw") - .setOutputCol("filtered"); - -JavaRDD rdd = jsc.parallelize(Arrays.asList( - RowFactory.create(Arrays.asList("I", "saw", "the", "red", "baloon")), - RowFactory.create(Arrays.asList("Mary", "had", "a", "little", "lamb")) -)); -StructType schema = new StructType(new StructField[] { - new StructField("raw", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty()) -}); -DataFrame dataset = jsql.createDataFrame(rdd, schema); - -remover.transform(dataset).show(); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java %}
    @@ -361,17 +399,7 @@ remover.transform(dataset).show(); Refer to the [StopWordsRemover Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.StopWordsRemover) for more details on the API. -{% highlight python %} -from pyspark.ml.feature import StopWordsRemover - -sentenceData = sqlContext.createDataFrame([ - (0, ["I", "saw", "the", "red", "baloon"]), - (1, ["Mary", "had", "a", "little", "lamb"]) -], ["label", "raw"]) - -remover = StopWordsRemover(inputCol="raw", outputCol="filtered") -remover.transform(sentenceData).show(truncate=False) -{% endhighlight %} +{% include_example python/ml/stopwords_remover_example.py %}
    @@ -381,6 +409,8 @@ An [n-gram](https://en.wikipedia.org/wiki/N-gram) is a sequence of $n$ tokens (t `NGram` takes as input a sequence of strings (e.g. the output of a [Tokenizer](ml-features.html#tokenizer)). The parameter `n` is used to determine the number of terms in each $n$-gram. The output will consist of a sequence of $n$-grams where each $n$-gram is represented by a space-delimited string of $n$ consecutive words. If the input sequence contains fewer than `n` strings, no output is produced. +**Examples** +
    @@ -388,19 +418,7 @@ An [n-gram](https://en.wikipedia.org/wiki/N-gram) is a sequence of $n$ tokens (t Refer to the [NGram Scala docs](api/scala/index.html#org.apache.spark.ml.feature.NGram) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.NGram - -val wordDataFrame = sqlContext.createDataFrame(Seq( - (0, Array("Hi", "I", "heard", "about", "Spark")), - (1, Array("I", "wish", "Java", "could", "use", "case", "classes")), - (2, Array("Logistic", "regression", "models", "are", "neat")) -)).toDF("label", "words") - -val ngram = new NGram().setInputCol("words").setOutputCol("ngrams") -val ngramDataFrame = ngram.transform(wordDataFrame) -ngramDataFrame.take(3).map(_.getAs[Stream[String]]("ngrams").toList).foreach(println) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/NGramExample.scala %}
    @@ -408,38 +426,7 @@ ngramDataFrame.take(3).map(_.getAs[Stream[String]]("ngrams").toList).foreach(pri Refer to the [NGram Java docs](api/java/org/apache/spark/ml/feature/NGram.html) for more details on the API. -{% highlight java %} -import java.util.Arrays; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.ml.feature.NGram; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -JavaRDD jrdd = jsc.parallelize(Arrays.asList( - RowFactory.create(0.0, Arrays.asList("Hi", "I", "heard", "about", "Spark")), - RowFactory.create(1.0, Arrays.asList("I", "wish", "Java", "could", "use", "case", "classes")), - RowFactory.create(2.0, Arrays.asList("Logistic", "regression", "models", "are", "neat")) -)); -StructType schema = new StructType(new StructField[]{ - new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), - new StructField("words", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty()) -}); -DataFrame wordDataFrame = sqlContext.createDataFrame(jrdd, schema); -NGram ngramTransformer = new NGram().setInputCol("words").setOutputCol("ngrams"); -DataFrame ngramDataFrame = ngramTransformer.transform(wordDataFrame); -for (Row r : ngramDataFrame.select("ngrams", "label").take(3)) { - java.util.List ngrams = r.getList(0); - for (String ngram : ngrams) System.out.print(ngram + " --- "); - System.out.println(); -} -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaNGramExample.java %}
    @@ -447,19 +434,7 @@ for (Row r : ngramDataFrame.select("ngrams", "label").take(3)) { Refer to the [NGram Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.NGram) for more details on the API. -{% highlight python %} -from pyspark.ml.feature import NGram - -wordDataFrame = sqlContext.createDataFrame([ - (0, ["Hi", "I", "heard", "about", "Spark"]), - (1, ["I", "wish", "Java", "could", "use", "case", "classes"]), - (2, ["Logistic", "regression", "models", "are", "neat"]) -], ["label", "words"]) -ngram = NGram(inputCol="words", outputCol="ngrams") -ngramDataFrame = ngram.transform(wordDataFrame) -for ngrams_label in ngramDataFrame.select("ngrams", "label").take(3): - print(ngrams_label) -{% endhighlight %} +{% include_example python/ml/n_gram_example.py %}
    @@ -468,7 +443,12 @@ for ngrams_label in ngramDataFrame.select("ngrams", "label").take(3): Binarization is the process of thresholding numerical features to binary (0/1) features. -`Binarizer` takes the common parameters `inputCol` and `outputCol`, as well as the `threshold` for binarization. Feature values greater than the threshold are binarized to 1.0; values equal to or less than the threshold are binarized to 0.0. +`Binarizer` takes the common parameters `inputCol` and `outputCol`, as well as the `threshold` +for binarization. Feature values greater than the threshold are binarized to 1.0; values equal +to or less than the threshold are binarized to 0.0. Both Vector and Double types are supported +for `inputCol`. + +**Examples**
    @@ -476,26 +456,7 @@ Binarization is the process of thresholding numerical features to binary (0/1) f Refer to the [Binarizer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.Binarizer) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.Binarizer -import org.apache.spark.sql.DataFrame - -val data = Array( - (0, 0.1), - (1, 0.8), - (2, 0.2) -) -val dataFrame: DataFrame = sqlContext.createDataFrame(data).toDF("label", "feature") - -val binarizer: Binarizer = new Binarizer() - .setInputCol("feature") - .setOutputCol("binarized_feature") - .setThreshold(0.5) - -val binarizedDataFrame = binarizer.transform(dataFrame) -val binarizedFeatures = binarizedDataFrame.select("binarized_feature") -binarizedFeatures.collect().foreach(println) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/BinarizerExample.scala %}
    @@ -503,40 +464,7 @@ binarizedFeatures.collect().foreach(println) Refer to the [Binarizer Java docs](api/java/org/apache/spark/ml/feature/Binarizer.html) for more details on the API. -{% highlight java %} -import java.util.Arrays; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.ml.feature.Binarizer; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -JavaRDD jrdd = jsc.parallelize(Arrays.asList( - RowFactory.create(0, 0.1), - RowFactory.create(1, 0.8), - RowFactory.create(2, 0.2) -)); -StructType schema = new StructType(new StructField[]{ - new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), - new StructField("feature", DataTypes.DoubleType, false, Metadata.empty()) -}); -DataFrame continuousDataFrame = jsql.createDataFrame(jrdd, schema); -Binarizer binarizer = new Binarizer() - .setInputCol("feature") - .setOutputCol("binarized_feature") - .setThreshold(0.5); -DataFrame binarizedDataFrame = binarizer.transform(continuousDataFrame); -DataFrame binarizedFeatures = binarizedDataFrame.select("binarized_feature"); -for (Row r : binarizedFeatures.collect()) { - Double binarized_value = r.getDouble(0); - System.out.println(binarized_value); -} -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaBinarizerExample.java %}
    @@ -544,20 +472,7 @@ for (Row r : binarizedFeatures.collect()) { Refer to the [Binarizer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.Binarizer) for more details on the API. -{% highlight python %} -from pyspark.ml.feature import Binarizer - -continuousDataFrame = sqlContext.createDataFrame([ - (0, 0.1), - (1, 0.8), - (2, 0.2) -], ["label", "feature"]) -binarizer = Binarizer(threshold=0.5, inputCol="feature", outputCol="binarized_feature") -binarizedDataFrame = binarizer.transform(continuousDataFrame) -binarizedFeatures = binarizedDataFrame.select("binarized_feature") -for binarized_feature, in binarizedFeatures.collect(): - print(binarized_feature) -{% endhighlight %} +{% include_example python/ml/binarizer_example.py %}
    @@ -565,31 +480,15 @@ for binarized_feature, in binarizedFeatures.collect(): [PCA](http://en.wikipedia.org/wiki/Principal_component_analysis) is a statistical procedure that uses an orthogonal transformation to convert a set of observations of possibly correlated variables into a set of values of linearly uncorrelated variables called principal components. A [PCA](api/scala/index.html#org.apache.spark.ml.feature.PCA) class trains a model to project vectors to a low-dimensional space using PCA. The example below shows how to project 5-dimensional feature vectors into 3-dimensional principal components. +**Examples** +
    Refer to the [PCA Scala docs](api/scala/index.html#org.apache.spark.ml.feature.PCA) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.PCA -import org.apache.spark.mllib.linalg.Vectors - -val data = Array( - Vectors.sparse(5, Seq((1, 1.0), (3, 7.0))), - Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0), - Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0) -) -val df = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") -val pca = new PCA() - .setInputCol("features") - .setOutputCol("pcaFeatures") - .setK(3) - .fit(df) -val pcaDF = pca.transform(df) -val result = pcaDF.select("pcaFeatures") -result.show() -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/PCAExample.scala %}
    @@ -597,42 +496,7 @@ result.show() Refer to the [PCA Java docs](api/java/org/apache/spark/ml/feature/PCA.html) for more details on the API. -{% highlight java %} -import java.util.Arrays; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.feature.PCA -import org.apache.spark.ml.feature.PCAModel -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -JavaSparkContext jsc = ... -SQLContext jsql = ... -JavaRDD data = jsc.parallelize(Arrays.asList( - RowFactory.create(Vectors.sparse(5, new int[]{1, 3}, new double[]{1.0, 7.0})), - RowFactory.create(Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0)), - RowFactory.create(Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0)) -)); -StructType schema = new StructType(new StructField[] { - new StructField("features", new VectorUDT(), false, Metadata.empty()), -}); -DataFrame df = jsql.createDataFrame(data, schema); -PCAModel pca = new PCA() - .setInputCol("features") - .setOutputCol("pcaFeatures") - .setK(3) - .fit(df); -DataFrame result = pca.transform(df).select("pcaFeatures"); -result.show(); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaPCAExample.java %}
    @@ -640,19 +504,7 @@ result.show(); Refer to the [PCA Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.PCA) for more details on the API. -{% highlight python %} -from pyspark.ml.feature import PCA -from pyspark.mllib.linalg import Vectors - -data = [(Vectors.sparse(5, [(1, 1.0), (3, 7.0)]),), - (Vectors.dense([2.0, 0.0, 3.0, 4.0, 5.0]),), - (Vectors.dense([4.0, 0.0, 0.0, 6.0, 7.0]),)] -df = sqlContext.createDataFrame(data,["features"]) -pca = PCA(k=3, inputCol="features", outputCol="pcaFeatures") -model = pca.fit(df) -result = model.transform(df).select("pcaFeatures") -result.show(truncate=False) -{% endhighlight %} +{% include_example python/ml/pca_example.py %}
    @@ -660,29 +512,15 @@ result.show(truncate=False) [Polynomial expansion](http://en.wikipedia.org/wiki/Polynomial_expansion) is the process of expanding your features into a polynomial space, which is formulated by an n-degree combination of original dimensions. A [PolynomialExpansion](api/scala/index.html#org.apache.spark.ml.feature.PolynomialExpansion) class provides this functionality. The example below shows how to expand your features into a 3-degree polynomial space. +**Examples** +
    Refer to the [PolynomialExpansion Scala docs](api/scala/index.html#org.apache.spark.ml.feature.PolynomialExpansion) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.PolynomialExpansion -import org.apache.spark.mllib.linalg.Vectors - -val data = Array( - Vectors.dense(-2.0, 2.3), - Vectors.dense(0.0, 0.0), - Vectors.dense(0.6, -1.1) -) -val df = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") -val polynomialExpansion = new PolynomialExpansion() - .setInputCol("features") - .setOutputCol("polyFeatures") - .setDegree(3) -val polyDF = polynomialExpansion.transform(df) -polyDF.select("polyFeatures").take(3).foreach(println) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala %}
    @@ -690,43 +528,7 @@ polyDF.select("polyFeatures").take(3).foreach(println) Refer to the [PolynomialExpansion Java docs](api/java/org/apache/spark/ml/feature/PolynomialExpansion.html) for more details on the API. -{% highlight java %} -import java.util.Arrays; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -JavaSparkContext jsc = ... -SQLContext jsql = ... -PolynomialExpansion polyExpansion = new PolynomialExpansion() - .setInputCol("features") - .setOutputCol("polyFeatures") - .setDegree(3); -JavaRDD data = jsc.parallelize(Arrays.asList( - RowFactory.create(Vectors.dense(-2.0, 2.3)), - RowFactory.create(Vectors.dense(0.0, 0.0)), - RowFactory.create(Vectors.dense(0.6, -1.1)) -)); -StructType schema = new StructType(new StructField[] { - new StructField("features", new VectorUDT(), false, Metadata.empty()), -}); -DataFrame df = jsql.createDataFrame(data, schema); -DataFrame polyDF = polyExpansion.transform(df); -Row[] row = polyDF.select("polyFeatures").take(3); -for (Row r : row) { - System.out.println(r.get(0)); -} -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java %}
    @@ -734,20 +536,7 @@ for (Row r : row) { Refer to the [PolynomialExpansion Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.PolynomialExpansion) for more details on the API. -{% highlight python %} -from pyspark.ml.feature import PolynomialExpansion -from pyspark.mllib.linalg import Vectors - -df = sqlContext.createDataFrame( - [(Vectors.dense([-2.0, 2.3]), ), - (Vectors.dense([0.0, 0.0]), ), - (Vectors.dense([0.6, -1.1]), )], - ["features"]) -px = PolynomialExpansion(degree=2, inputCol="features", outputCol="polyFeatures") -polyDF = px.transform(df) -for expanded in polyDF.select("polyFeatures").take(3): - print(expanded) -{% endhighlight %} +{% include_example python/ml/polynomial_expansion_example.py %}
    @@ -765,28 +554,15 @@ for the transform is unitary. No shift is applied to the transformed sequence (e.g. the $0$th element of the transformed sequence is the $0$th DCT coefficient and _not_ the $N/2$th). +**Examples** +
    Refer to the [DCT Scala docs](api/scala/index.html#org.apache.spark.ml.feature.DCT) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.DCT -import org.apache.spark.mllib.linalg.Vectors - -val data = Seq( - Vectors.dense(0.0, 1.0, -2.0, 3.0), - Vectors.dense(-1.0, 2.0, 4.0, -7.0), - Vectors.dense(14.0, -2.0, -5.0, 1.0)) -val df = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") -val dct = new DCT() - .setInputCol("features") - .setOutputCol("featuresDCT") - .setInverse(false) -val dctDf = dct.transform(df) -dctDf.select("featuresDCT").show(3) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/DCTExample.scala %}
    @@ -794,51 +570,27 @@ dctDf.select("featuresDCT").show(3) Refer to the [DCT Java docs](api/java/org/apache/spark/ml/feature/DCT.html) for more details on the API. -{% highlight java %} -import java.util.Arrays; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.feature.DCT; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -JavaRDD data = jsc.parallelize(Arrays.asList( - RowFactory.create(Vectors.dense(0.0, 1.0, -2.0, 3.0)), - RowFactory.create(Vectors.dense(-1.0, 2.0, 4.0, -7.0)), - RowFactory.create(Vectors.dense(14.0, -2.0, -5.0, 1.0)) -)); -StructType schema = new StructType(new StructField[] { - new StructField("features", new VectorUDT(), false, Metadata.empty()), -}); -DataFrame df = jsql.createDataFrame(data, schema); -DCT dct = new DCT() - .setInputCol("features") - .setOutputCol("featuresDCT") - .setInverse(false); -DataFrame dctDf = dct.transform(df); -dctDf.select("featuresDCT").show(3); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaDCTExample.java %} +
    + +
    + +Refer to the [DCT Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.DCT) +for more details on the API. + +{% include_example python/ml/dct_example.py %}
    ## StringIndexer `StringIndexer` encodes a string column of labels to a column of label indices. -The indices are in `[0, numLabels)`, ordered by label frequencies. -So the most frequent label gets index `0`. -If the input column is numeric, we cast it to string and index the string -values. When downstream pipeline components such as `Estimator` or -`Transformer` make use of this string-indexed label, you must set the input -column of the component to this string-indexed column name. In many cases, +The indices are in `[0, numLabels)`, ordered by label frequencies, so the most frequent label gets index `0`. +The unseen labels will be put at index numLabels if user chooses to keep them. +If the input column is numeric, we cast it to string and index the string +values. When downstream pipeline components such as `Estimator` or +`Transformer` make use of this string-indexed label, you must set the input +column of the component to this string-indexed column name. In many cases, you can set the input column with `setInputCol`. **Examples** @@ -874,6 +626,59 @@ column, we should get the following: "a" gets index `0` because it is the most frequent, followed by "c" with index `1` and "b" with index `2`. +Additionally, there are three strategies regarding how `StringIndexer` will handle +unseen labels when you have fit a `StringIndexer` on one dataset and then use it +to transform another: + +- throw an exception (which is the default) +- skip the row containing the unseen label entirely +- put unseen labels in a special additional bucket, at index numLabels + +**Examples** + +Let's go back to our previous example but this time reuse our previously defined +`StringIndexer` on the following dataset: + +~~~~ + id | category +----|---------- + 0 | a + 1 | b + 2 | c + 3 | d + 4 | e +~~~~ + +If you've not set how `StringIndexer` handles unseen labels or set it to +"error", an exception will be thrown. +However, if you had called `setHandleInvalid("skip")`, the following dataset +will be generated: + +~~~~ + id | category | categoryIndex +----|----------|--------------- + 0 | a | 0.0 + 1 | b | 2.0 + 2 | c | 1.0 +~~~~ + +Notice that the rows containing "d" or "e" do not appear. + +If you call `setHandleInvalid("keep")`, the following dataset +will be generated: + +~~~~ + id | category | categoryIndex +----|----------|--------------- + 0 | a | 0.0 + 1 | b | 2.0 + 2 | c | 1.0 + 3 | d | 3.0 + 4 | e | 3.0 +~~~~ + +Notice that the rows containing "d" or "e" are mapped to index "3.0" +
    @@ -881,18 +686,7 @@ index `2`. Refer to the [StringIndexer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.StringIndexer) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.StringIndexer - -val df = sqlContext.createDataFrame( - Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")) -).toDF("id", "category") -val indexer = new StringIndexer() - .setInputCol("category") - .setOutputCol("categoryIndex") -val indexed = indexer.fit(df).transform(df) -indexed.show() -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/StringIndexerExample.scala %}
    @@ -900,37 +694,7 @@ indexed.show() Refer to the [StringIndexer Java docs](api/java/org/apache/spark/ml/feature/StringIndexer.html) for more details on the API. -{% highlight java %} -import java.util.Arrays; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.ml.feature.StringIndexer; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; -import static org.apache.spark.sql.types.DataTypes.*; - -JavaRDD jrdd = jsc.parallelize(Arrays.asList( - RowFactory.create(0, "a"), - RowFactory.create(1, "b"), - RowFactory.create(2, "c"), - RowFactory.create(3, "a"), - RowFactory.create(4, "a"), - RowFactory.create(5, "c") -)); -StructType schema = new StructType(new StructField[] { - createStructField("id", DoubleType, false), - createStructField("category", StringType, false) -}); -DataFrame df = sqlContext.createDataFrame(jrdd, schema); -StringIndexer indexer = new StringIndexer() - .setInputCol("category") - .setOutputCol("categoryIndex"); -DataFrame indexed = indexer.fit(df).transform(df); -indexed.show(); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaStringIndexerExample.java %}
    @@ -938,52 +702,92 @@ indexed.show(); Refer to the [StringIndexer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.StringIndexer) for more details on the API. -{% highlight python %} -from pyspark.ml.feature import StringIndexer - -df = sqlContext.createDataFrame( - [(0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")], - ["id", "category"]) -indexer = StringIndexer(inputCol="category", outputCol="categoryIndex") -indexed = indexer.fit(df).transform(df) -indexed.show() -{% endhighlight %} +{% include_example python/ml/string_indexer_example.py %}
    -## OneHotEncoder -[One-hot encoding](http://en.wikipedia.org/wiki/One-hot) maps a column of label indices to a column of binary vectors, with at most a single one-value. This encoding allows algorithms which expect continuous features, such as Logistic Regression, to use categorical features +## IndexToString + +Symmetrically to `StringIndexer`, `IndexToString` maps a column of label indices +back to a column containing the original labels as strings. A common use case +is to produce indices from labels with `StringIndexer`, train a model with those +indices and retrieve the original labels from the column of predicted indices +with `IndexToString`. However, you are free to supply your own labels. + +**Examples** + +Building on the `StringIndexer` example, let's assume we have the following +DataFrame with columns `id` and `categoryIndex`: + +~~~~ + id | categoryIndex +----|--------------- + 0 | 0.0 + 1 | 2.0 + 2 | 1.0 + 3 | 0.0 + 4 | 0.0 + 5 | 1.0 +~~~~ + +Applying `IndexToString` with `categoryIndex` as the input column, +`originalCategory` as the output column, we are able to retrieve our original +labels (they will be inferred from the columns' metadata): + +~~~~ + id | categoryIndex | originalCategory +----|---------------|----------------- + 0 | 0.0 | a + 1 | 2.0 | b + 2 | 1.0 | c + 3 | 0.0 | a + 4 | 0.0 | a + 5 | 1.0 | c +~~~~
    -Refer to the [OneHotEncoder Scala docs](api/scala/index.html#org.apache.spark.ml.feature.OneHotEncoder) +Refer to the [IndexToString Scala docs](api/scala/index.html#org.apache.spark.ml.feature.IndexToString) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.{OneHotEncoder, StringIndexer} +{% include_example scala/org/apache/spark/examples/ml/IndexToStringExample.scala %} + +
    + +
    + +Refer to the [IndexToString Java docs](api/java/org/apache/spark/ml/feature/IndexToString.html) +for more details on the API. + +{% include_example java/org/apache/spark/examples/ml/JavaIndexToStringExample.java %} + +
    + +
    + +Refer to the [IndexToString Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.IndexToString) +for more details on the API. + +{% include_example python/ml/index_to_string_example.py %} + +
    +
    + +## OneHotEncoder + +[One-hot encoding](http://en.wikipedia.org/wiki/One-hot) maps a column of label indices to a column of binary vectors, with at most a single one-value. This encoding allows algorithms which expect continuous features, such as Logistic Regression, to use categorical features. -val df = sqlContext.createDataFrame(Seq( - (0, "a"), - (1, "b"), - (2, "c"), - (3, "a"), - (4, "a"), - (5, "c") -)).toDF("id", "category") +**Examples** + +
    +
    -val indexer = new StringIndexer() - .setInputCol("category") - .setOutputCol("categoryIndex") - .fit(df) -val indexed = indexer.transform(df) +Refer to the [OneHotEncoder Scala docs](api/scala/index.html#org.apache.spark.ml.feature.OneHotEncoder) +for more details on the API. -val encoder = new OneHotEncoder().setInputCol("categoryIndex"). - setOutputCol("categoryVec") -val encoded = encoder.transform(indexed) -encoded.select("id", "categoryVec").foreach(println) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala %}
    @@ -991,45 +795,7 @@ encoded.select("id", "categoryVec").foreach(println) Refer to the [OneHotEncoder Java docs](api/java/org/apache/spark/ml/feature/OneHotEncoder.html) for more details on the API. -{% highlight java %} -import java.util.Arrays; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.ml.feature.OneHotEncoder; -import org.apache.spark.ml.feature.StringIndexer; -import org.apache.spark.ml.feature.StringIndexerModel; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -JavaRDD jrdd = jsc.parallelize(Arrays.asList( - RowFactory.create(0, "a"), - RowFactory.create(1, "b"), - RowFactory.create(2, "c"), - RowFactory.create(3, "a"), - RowFactory.create(4, "a"), - RowFactory.create(5, "c") -)); -StructType schema = new StructType(new StructField[]{ - new StructField("id", DataTypes.DoubleType, false, Metadata.empty()), - new StructField("category", DataTypes.StringType, false, Metadata.empty()) -}); -DataFrame df = sqlContext.createDataFrame(jrdd, schema); -StringIndexerModel indexer = new StringIndexer() - .setInputCol("category") - .setOutputCol("categoryIndex") - .fit(df); -DataFrame indexed = indexer.transform(df); - -OneHotEncoder encoder = new OneHotEncoder() - .setInputCol("categoryIndex") - .setOutputCol("categoryVec"); -DataFrame encoded = encoder.transform(indexed); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java %}
    @@ -1037,24 +803,7 @@ DataFrame encoded = encoder.transform(indexed); Refer to the [OneHotEncoder Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.OneHotEncoder) for more details on the API. -{% highlight python %} -from pyspark.ml.feature import OneHotEncoder, StringIndexer - -df = sqlContext.createDataFrame([ - (0, "a"), - (1, "b"), - (2, "c"), - (3, "a"), - (4, "a"), - (5, "c") -], ["id", "category"]) - -stringIndexer = StringIndexer(inputCol="category", outputCol="categoryIndex") -model = stringIndexer.fit(df) -indexed = model.transform(df) -encoder = OneHotEncoder(includeFirst=False, inputCol="categoryIndex", outputCol="categoryVec") -encoded = encoder.transform(indexed) -{% endhighlight %} +{% include_example python/ml/onehot_encoder_example.py %}
    @@ -1063,13 +812,15 @@ encoded = encoder.transform(indexed) `VectorIndexer` helps index categorical features in datasets of `Vector`s. It can both automatically decide which features are categorical and convert original values to category indices. Specifically, it does the following: -1. Take an input column of type [Vector](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) and a parameter `maxCategories`. +1. Take an input column of type [Vector](api/scala/index.html#org.apache.spark.ml.linalg.Vector) and a parameter `maxCategories`. 2. Decide which features should be categorical based on the number of distinct values, where features with at most `maxCategories` are declared categorical. 3. Compute 0-based category indices for each categorical feature. 4. Index categorical features and transform original feature values to indices. Indexing categorical features allows algorithms such as Decision Trees and Tree Ensembles to treat categorical features appropriately, improving performance. +**Examples** + In the example below, we read in a dataset of labeled points and then use `VectorIndexer` to decide which features should be treated as categorical. We transform the categorical feature values to their indices. This transformed data could then be passed to algorithms such as `DecisionTreeRegressor` that handle categorical features.
    @@ -1078,23 +829,7 @@ In the example below, we read in a dataset of labeled points and then use `Vecto Refer to the [VectorIndexer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.VectorIndexer) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.VectorIndexer - -val data = sqlContext.read.format("libsvm") - .load("data/mllib/sample_libsvm_data.txt") -val indexer = new VectorIndexer() - .setInputCol("features") - .setOutputCol("indexed") - .setMaxCategories(10) -val indexerModel = indexer.fit(data) -val categoricalFeatures: Set[Int] = indexerModel.categoryMaps.keys.toSet -println(s"Chose ${categoricalFeatures.size} categorical features: " + - categoricalFeatures.mkString(", ")) - -// Create new column "indexed" with categorical values transformed to indices -val indexedData = indexerModel.transform(data) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/VectorIndexerExample.scala %}
    @@ -1102,30 +837,7 @@ val indexedData = indexerModel.transform(data) Refer to the [VectorIndexer Java docs](api/java/org/apache/spark/ml/feature/VectorIndexer.html) for more details on the API. -{% highlight java %} -import java.util.Map; - -import org.apache.spark.ml.feature.VectorIndexer; -import org.apache.spark.ml.feature.VectorIndexerModel; -import org.apache.spark.sql.DataFrame; - -DataFrame data = sqlContext.read.format("libsvm") - .load("data/mllib/sample_libsvm_data.txt"); -VectorIndexer indexer = new VectorIndexer() - .setInputCol("features") - .setOutputCol("indexed") - .setMaxCategories(10); -VectorIndexerModel indexerModel = indexer.fit(data); -Map> categoryMaps = indexerModel.javaCategoryMaps(); -System.out.print("Chose " + categoryMaps.size() + "categorical features:"); -for (Integer feature : categoryMaps.keySet()) { - System.out.print(" " + feature); -} -System.out.println(); - -// Create new column "indexed" with categorical values transformed to indices -DataFrame indexedData = indexerModel.transform(data); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java %}
    @@ -1133,94 +845,94 @@ DataFrame indexedData = indexerModel.transform(data); Refer to the [VectorIndexer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.VectorIndexer) for more details on the API. -{% highlight python %} -from pyspark.ml.feature import VectorIndexer - -data = sqlContext.read.format("libsvm") - .load("data/mllib/sample_libsvm_data.txt") -indexer = VectorIndexer(inputCol="features", outputCol="indexed", maxCategories=10) -indexerModel = indexer.fit(data) - -# Create new column "indexed" with categorical values transformed to indices -indexedData = indexerModel.transform(data) -{% endhighlight %} +{% include_example python/ml/vector_indexer_example.py %}
    +## Interaction -## Normalizer +`Interaction` is a `Transformer` which takes vector or double-valued columns, and generates a single vector column that contains the product of all combinations of one value from each input column. -`Normalizer` is a `Transformer` which transforms a dataset of `Vector` rows, normalizing each `Vector` to have unit norm. It takes parameter `p`, which specifies the [p-norm](http://en.wikipedia.org/wiki/Norm_%28mathematics%29#p-norm) used for normalization. ($p = 2$ by default.) This normalization can help standardize your input data and improve the behavior of learning algorithms. +For example, if you have 2 vector type columns each of which has 3 dimensions as input columns, then you'll get a 9-dimensional vector as the output column. -The following example demonstrates how to load a dataset in libsvm format and then normalize each row to have unit $L^2$ norm and unit $L^\infty$ norm. +**Examples** + +Assume that we have the following DataFrame with the columns "id1", "vec1", and "vec2": + +~~~~ + id1|vec1 |vec2 + ---|--------------|-------------- + 1 |[1.0,2.0,3.0] |[8.0,4.0,5.0] + 2 |[4.0,3.0,8.0] |[7.0,9.0,8.0] + 3 |[6.0,1.0,9.0] |[2.0,3.0,6.0] + 4 |[10.0,8.0,6.0]|[9.0,4.0,5.0] + 5 |[9.0,2.0,7.0] |[10.0,7.0,3.0] + 6 |[1.0,1.0,4.0] |[2.0,8.0,4.0] +~~~~ + +Applying `Interaction` with those input columns, +then `interactedCol` as the output column contains: + +~~~~ + id1|vec1 |vec2 |interactedCol + ---|--------------|--------------|------------------------------------------------------ + 1 |[1.0,2.0,3.0] |[8.0,4.0,5.0] |[8.0,4.0,5.0,16.0,8.0,10.0,24.0,12.0,15.0] + 2 |[4.0,3.0,8.0] |[7.0,9.0,8.0] |[56.0,72.0,64.0,42.0,54.0,48.0,112.0,144.0,128.0] + 3 |[6.0,1.0,9.0] |[2.0,3.0,6.0] |[36.0,54.0,108.0,6.0,9.0,18.0,54.0,81.0,162.0] + 4 |[10.0,8.0,6.0]|[9.0,4.0,5.0] |[360.0,160.0,200.0,288.0,128.0,160.0,216.0,96.0,120.0] + 5 |[9.0,2.0,7.0] |[10.0,7.0,3.0]|[450.0,315.0,135.0,100.0,70.0,30.0,350.0,245.0,105.0] + 6 |[1.0,1.0,4.0] |[2.0,8.0,4.0] |[12.0,48.0,24.0,12.0,48.0,24.0,48.0,192.0,96.0] +~~~~
    -
    +
    -Refer to the [Normalizer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.Normalizer) +Refer to the [Interaction Scala docs](api/scala/index.html#org.apache.spark.ml.feature.Interaction) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.Normalizer +{% include_example scala/org/apache/spark/examples/ml/InteractionExample.scala %} +
    -val dataFrame = sqlContext.read.format("libsvm") - .load("data/mllib/sample_libsvm_data.txt") +
    -// Normalize each Vector using $L^1$ norm. -val normalizer = new Normalizer() - .setInputCol("features") - .setOutputCol("normFeatures") - .setP(1.0) -val l1NormData = normalizer.transform(dataFrame) +Refer to the [Interaction Java docs](api/java/org/apache/spark/ml/feature/Interaction.html) +for more details on the API. -// Normalize each Vector using $L^\infty$ norm. -val lInfNormData = normalizer.transform(dataFrame, normalizer.p -> Double.PositiveInfinity) -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaInteractionExample.java %} +
    -
    +## Normalizer -Refer to the [Normalizer Java docs](api/java/org/apache/spark/ml/feature/Normalizer.html) -for more details on the API. +`Normalizer` is a `Transformer` which transforms a dataset of `Vector` rows, normalizing each `Vector` to have unit norm. It takes parameter `p`, which specifies the [p-norm](http://en.wikipedia.org/wiki/Norm_%28mathematics%29#p-norm) used for normalization. ($p = 2$ by default.) This normalization can help standardize your input data and improve the behavior of learning algorithms. -{% highlight java %} -import org.apache.spark.ml.feature.Normalizer; -import org.apache.spark.sql.DataFrame; +**Examples** -DataFrame dataFrame = sqlContext.read.format("libsvm") - .load("data/mllib/sample_libsvm_data.txt"); +The following example demonstrates how to load a dataset in libsvm format and then normalize each row to have unit $L^1$ norm and unit $L^\infty$ norm. -// Normalize each Vector using $L^1$ norm. -Normalizer normalizer = new Normalizer() - .setInputCol("features") - .setOutputCol("normFeatures") - .setP(1.0); -DataFrame l1NormData = normalizer.transform(dataFrame); +
    +
    -// Normalize each Vector using $L^\infty$ norm. -DataFrame lInfNormData = - normalizer.transform(dataFrame, normalizer.p().w(Double.POSITIVE_INFINITY)); -{% endhighlight %} +Refer to the [Normalizer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.Normalizer) +for more details on the API. + +{% include_example scala/org/apache/spark/examples/ml/NormalizerExample.scala %}
    -
    +
    -Refer to the [Normalizer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.Normalizer) +Refer to the [Normalizer Java docs](api/java/org/apache/spark/ml/feature/Normalizer.html) for more details on the API. -{% highlight python %} -from pyspark.ml.feature import Normalizer +{% include_example java/org/apache/spark/examples/ml/JavaNormalizerExample.java %} +
    -dataFrame = sqlContext.read.format("libsvm") - .load("data/mllib/sample_libsvm_data.txt") +
    -# Normalize each Vector using $L^1$ norm. -normalizer = Normalizer(inputCol="features", outputCol="normFeatures", p=1.0) -l1NormData = normalizer.transform(dataFrame) +Refer to the [Normalizer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.Normalizer) +for more details on the API. -# Normalize each Vector using $L^\infty$ norm. -lInfNormData = normalizer.transform(dataFrame, {normalizer.p: float("inf")}) -{% endhighlight %} +{% include_example python/ml/normalizer_example.py %}
    @@ -1230,84 +942,39 @@ lInfNormData = normalizer.transform(dataFrame, {normalizer.p: float("inf")}) `StandardScaler` transforms a dataset of `Vector` rows, normalizing each feature to have unit standard deviation and/or zero mean. It takes parameters: * `withStd`: True by default. Scales the data to unit standard deviation. -* `withMean`: False by default. Centers the data with mean before scaling. It will build a dense output, so this does not work on sparse input and will raise an exception. +* `withMean`: False by default. Centers the data with mean before scaling. It will build a dense output, so take care when applying to sparse input. -`StandardScaler` is a `Model` which can be `fit` on a dataset to produce a `StandardScalerModel`; this amounts to computing summary statistics. The model can then transform a `Vector` column in a dataset to have unit standard deviation and/or zero mean features. +`StandardScaler` is an `Estimator` which can be `fit` on a dataset to produce a `StandardScalerModel`; this amounts to computing summary statistics. The model can then transform a `Vector` column in a dataset to have unit standard deviation and/or zero mean features. Note that if the standard deviation of a feature is zero, it will return default `0.0` value in the `Vector` for that feature. +**Examples** + The following example demonstrates how to load a dataset in libsvm format and then normalize each feature to have unit standard deviation.
    -
    +
    Refer to the [StandardScaler Scala docs](api/scala/index.html#org.apache.spark.ml.feature.StandardScaler) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.StandardScaler - -val dataFrame = sqlContext.read.format("libsvm") - .load("data/mllib/sample_libsvm_data.txt") -val scaler = new StandardScaler() - .setInputCol("features") - .setOutputCol("scaledFeatures") - .setWithStd(true) - .setWithMean(false) - -// Compute summary statistics by fitting the StandardScaler -val scalerModel = scaler.fit(dataFrame) - -// Normalize each feature to have unit standard deviation. -val scaledData = scalerModel.transform(dataFrame) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/StandardScalerExample.scala %}
    -
    +
    Refer to the [StandardScaler Java docs](api/java/org/apache/spark/ml/feature/StandardScaler.html) for more details on the API. -{% highlight java %} -import org.apache.spark.ml.feature.StandardScaler; -import org.apache.spark.ml.feature.StandardScalerModel; -import org.apache.spark.sql.DataFrame; - -DataFrame dataFrame = sqlContext.read.format("libsvm") - .load("data/mllib/sample_libsvm_data.txt"); -StandardScaler scaler = new StandardScaler() - .setInputCol("features") - .setOutputCol("scaledFeatures") - .setWithStd(true) - .setWithMean(false); - -// Compute summary statistics by fitting the StandardScaler -StandardScalerModel scalerModel = scaler.fit(dataFrame); - -// Normalize each feature to have unit standard deviation. -DataFrame scaledData = scalerModel.transform(dataFrame); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaStandardScalerExample.java %}
    -
    +
    Refer to the [StandardScaler Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.StandardScaler) for more details on the API. -{% highlight python %} -from pyspark.ml.feature import StandardScaler - -dataFrame = sqlContext.read.format("libsvm") - .load("data/mllib/sample_libsvm_data.txt") -scaler = StandardScaler(inputCol="features", outputCol="scaledFeatures", - withStd=True, withMean=False) - -# Compute summary statistics by fitting the StandardScaler -scalerModel = scaler.fit(dataFrame) - -# Normalize each feature to have unit standard deviation. -scaledData = scalerModel.transform(dataFrame) -{% endhighlight %} +{% include_example python/ml/standard_scaler_example.py %}
    @@ -1324,9 +991,11 @@ The rescaled value for a feature E is calculated as, `\begin{equation} Rescaled(e_i) = \frac{e_i - E_{min}}{E_{max} - E_{min}} * (max - min) + min \end{equation}` -For the case `E_{max} == E_{min}`, `Rescaled(e_i) = 0.5 * (max + min)` +For the case `$E_{max} == E_{min}$`, `$Rescaled(e_i) = 0.5 * (max + min)$` + +Note that since zero values will probably be transformed to non-zero values, output of the transformer will be `DenseVector` even for sparse input. -Note that since zero values will probably be transformed to non-zero values, output of the transformer will be DenseVector even for sparse input. +**Examples** The following example demonstrates how to load a dataset in libsvm format and then rescale each feature to [0, 1]. @@ -1337,21 +1006,7 @@ Refer to the [MinMaxScaler Scala docs](api/scala/index.html#org.apache.spark.ml. and the [MinMaxScalerModel Scala docs](api/scala/index.html#org.apache.spark.ml.feature.MinMaxScalerModel) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.MinMaxScaler - -val dataFrame = sqlContext.read.format("libsvm") - .load("data/mllib/sample_libsvm_data.txt") -val scaler = new MinMaxScaler() - .setInputCol("features") - .setOutputCol("scaledFeatures") - -// Compute summary statistics and generate MinMaxScalerModel -val scalerModel = scaler.fit(dataFrame) - -// rescale each feature to range [min, max]. -val scaledData = scalerModel.transform(dataFrame) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala %}
    @@ -1360,123 +1015,101 @@ Refer to the [MinMaxScaler Java docs](api/java/org/apache/spark/ml/feature/MinMa and the [MinMaxScalerModel Java docs](api/java/org/apache/spark/ml/feature/MinMaxScalerModel.html) for more details on the API. -{% highlight java %} -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.ml.feature.MinMaxScaler; -import org.apache.spark.ml.feature.MinMaxScalerModel; -import org.apache.spark.sql.DataFrame; +{% include_example java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java %} +
    -DataFrame dataFrame = sqlContext.read.format("libsvm") - .load("data/mllib/sample_libsvm_data.txt"); -MinMaxScaler scaler = new MinMaxScaler() - .setInputCol("features") - .setOutputCol("scaledFeatures"); +
    -// Compute summary statistics and generate MinMaxScalerModel -MinMaxScalerModel scalerModel = scaler.fit(dataFrame); +Refer to the [MinMaxScaler Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.MinMaxScaler) +and the [MinMaxScalerModel Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.MinMaxScalerModel) +for more details on the API. -// rescale each feature to range [min, max]. -DataFrame scaledData = scalerModel.transform(dataFrame); -{% endhighlight %} +{% include_example python/ml/min_max_scaler_example.py %}
    -## Bucketizer -`Bucketizer` transforms a column of continuous features to a column of feature buckets, where the buckets are specified by users. It takes a parameter: - -* `splits`: Parameter for mapping continuous features into buckets. With n+1 splits, there are n buckets. A bucket defined by splits x,y holds values in the range [x,y) except the last bucket, which also includes y. Splits should be strictly increasing. Values at -inf, inf must be explicitly provided to cover all Double values; Otherwise, values outside the splits specified will be treated as errors. Two examples of `splits` are `Array(Double.NegativeInfinity, 0.0, 1.0, Double.PositiveInfinity)` and `Array(0.0, 1.0, 2.0)`. +## MaxAbsScaler -Note that if you have no idea of the upper bound and lower bound of the targeted column, you would better add the `Double.NegativeInfinity` and `Double.PositiveInfinity` as the bounds of your splits to prevent a potenial out of Bucketizer bounds exception. +`MaxAbsScaler` transforms a dataset of `Vector` rows, rescaling each feature to range [-1, 1] +by dividing through the maximum absolute value in each feature. It does not shift/center the +data, and thus does not destroy any sparsity. -Note also that the splits that you provided have to be in strictly increasing order, i.e. `s0 < s1 < s2 < ... < sn`. +`MaxAbsScaler` computes summary statistics on a data set and produces a `MaxAbsScalerModel`. The +model can then transform each feature individually to range [-1, 1]. -More details can be found in the API docs for [Bucketizer](api/scala/index.html#org.apache.spark.ml.feature.Bucketizer). +**Examples** -The following example demonstrates how to bucketize a column of `Double`s into another index-wised column. +The following example demonstrates how to load a dataset in libsvm format and then rescale each feature to [-1, 1].
    -
    +
    -Refer to the [Bucketizer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.Bucketizer) +Refer to the [MaxAbsScaler Scala docs](api/scala/index.html#org.apache.spark.ml.feature.MaxAbsScaler) +and the [MaxAbsScalerModel Scala docs](api/scala/index.html#org.apache.spark.ml.feature.MaxAbsScalerModel) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.Bucketizer -import org.apache.spark.sql.DataFrame - -val splits = Array(Double.NegativeInfinity, -0.5, 0.0, 0.5, Double.PositiveInfinity) +{% include_example scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala %} +
    -val data = Array(-0.5, -0.3, 0.0, 0.2) -val dataFrame = sqlContext.createDataFrame(data.map(Tuple1.apply)).toDF("features") +
    -val bucketizer = new Bucketizer() - .setInputCol("features") - .setOutputCol("bucketedFeatures") - .setSplits(splits) +Refer to the [MaxAbsScaler Java docs](api/java/org/apache/spark/ml/feature/MaxAbsScaler.html) +and the [MaxAbsScalerModel Java docs](api/java/org/apache/spark/ml/feature/MaxAbsScalerModel.html) +for more details on the API. -// Transform original data into its bucket index. -val bucketedData = bucketizer.transform(dataFrame) -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java %}
    -
    +
    -Refer to the [Bucketizer Java docs](api/java/org/apache/spark/ml/feature/Bucketizer.html) +Refer to the [MaxAbsScaler Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.MaxAbsScaler) +and the [MaxAbsScalerModel Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.MaxAbsScalerModel) for more details on the API. -{% highlight java %} -import java.util.Arrays; +{% include_example python/ml/max_abs_scaler_example.py %} +
    +
    + +## Bucketizer -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; +`Bucketizer` transforms a column of continuous features to a column of feature buckets, where the buckets are specified by users. It takes a parameter: -double[] splits = {Double.NEGATIVE_INFINITY, -0.5, 0.0, 0.5, Double.POSITIVE_INFINITY}; +* `splits`: Parameter for mapping continuous features into buckets. With n+1 splits, there are n buckets. A bucket defined by splits x,y holds values in the range [x,y) except the last bucket, which also includes y. Splits should be strictly increasing. Values at -inf, inf must be explicitly provided to cover all Double values; Otherwise, values outside the splits specified will be treated as errors. Two examples of `splits` are `Array(Double.NegativeInfinity, 0.0, 1.0, Double.PositiveInfinity)` and `Array(0.0, 1.0, 2.0)`. -JavaRDD data = jsc.parallelize(Arrays.asList( - RowFactory.create(-0.5), - RowFactory.create(-0.3), - RowFactory.create(0.0), - RowFactory.create(0.2) -)); -StructType schema = new StructType(new StructField[] { - new StructField("features", DataTypes.DoubleType, false, Metadata.empty()) -}); -DataFrame dataFrame = jsql.createDataFrame(data, schema); +Note that if you have no idea of the upper and lower bounds of the targeted column, you should add `Double.NegativeInfinity` and `Double.PositiveInfinity` as the bounds of your splits to prevent a potential out of Bucketizer bounds exception. -Bucketizer bucketizer = new Bucketizer() - .setInputCol("features") - .setOutputCol("bucketedFeatures") - .setSplits(splits); +Note also that the splits that you provided have to be in strictly increasing order, i.e. `s0 < s1 < s2 < ... < sn`. -// Transform original data into its bucket index. -DataFrame bucketedData = bucketizer.transform(dataFrame); -{% endhighlight %} -
    +More details can be found in the API docs for [Bucketizer](api/scala/index.html#org.apache.spark.ml.feature.Bucketizer). -
    +**Examples** -Refer to the [Bucketizer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.Bucketizer) +The following example demonstrates how to bucketize a column of `Double`s into another index-wised column. + +
    +
    + +Refer to the [Bucketizer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.Bucketizer) for more details on the API. -{% highlight python %} -from pyspark.ml.feature import Bucketizer +{% include_example scala/org/apache/spark/examples/ml/BucketizerExample.scala %} +
    + +
    -splits = [-float("inf"), -0.5, 0.0, 0.5, float("inf")] +Refer to the [Bucketizer Java docs](api/java/org/apache/spark/ml/feature/Bucketizer.html) +for more details on the API. -data = [(-0.5,), (-0.3,), (0.0,), (0.2,)] -dataFrame = sqlContext.createDataFrame(data, ["features"]) +{% include_example java/org/apache/spark/examples/ml/JavaBucketizerExample.java %} +
    -bucketizer = Bucketizer(splits=splits, inputCol="features", outputCol="bucketedFeatures") +
    -# Transform original data into its bucket index. -bucketedData = bucketizer.transform(dataFrame) -{% endhighlight %} +Refer to the [Bucketizer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.Bucketizer) +for more details on the API. + +{% include_example python/ml/bucketizer_example.py %}
    @@ -1500,6 +1133,8 @@ v_N \end{pmatrix} \]` +**Examples** + This example below demonstrates how to transform vectors using a transforming vector value.
    @@ -1508,25 +1143,7 @@ This example below demonstrates how to transform vectors using a transforming ve Refer to the [ElementwiseProduct Scala docs](api/scala/index.html#org.apache.spark.ml.feature.ElementwiseProduct) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.ElementwiseProduct -import org.apache.spark.mllib.linalg.Vectors - -// Create some vector data; also works for sparse vectors -val dataFrame = sqlContext.createDataFrame(Seq( - ("a", Vectors.dense(1.0, 2.0, 3.0)), - ("b", Vectors.dense(4.0, 5.0, 6.0)))).toDF("id", "vector") - -val transformingVector = Vectors.dense(0.0, 1.0, 2.0) -val transformer = new ElementwiseProduct() - .setScalingVec(transformingVector) - .setInputCol("vector") - .setOutputCol("transformedVector") - -// Batch transform the vectors to create new column: -transformer.transform(dataFrame).show() - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala %}
    @@ -1534,41 +1151,7 @@ transformer.transform(dataFrame).show() Refer to the [ElementwiseProduct Java docs](api/java/org/apache/spark/ml/feature/ElementwiseProduct.html) for more details on the API. -{% highlight java %} -import java.util.Arrays; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.ml.feature.ElementwiseProduct; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; - -// Create some vector data; also works for sparse vectors -JavaRDD jrdd = jsc.parallelize(Arrays.asList( - RowFactory.create("a", Vectors.dense(1.0, 2.0, 3.0)), - RowFactory.create("b", Vectors.dense(4.0, 5.0, 6.0)) -)); -List fields = new ArrayList(2); -fields.add(DataTypes.createStructField("id", DataTypes.StringType, false)); -fields.add(DataTypes.createStructField("vector", DataTypes.StringType, false)); -StructType schema = DataTypes.createStructType(fields); -DataFrame dataFrame = sqlContext.createDataFrame(jrdd, schema); -Vector transformingVector = Vectors.dense(0.0, 1.0, 2.0); -ElementwiseProduct transformer = new ElementwiseProduct() - .setScalingVec(transformingVector) - .setInputCol("vector") - .setOutputCol("transformedVector"); -// Batch transform the vectors to create new column: -transformer.transform(dataFrame).show(); - -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java %}
    @@ -1576,19 +1159,67 @@ transformer.transform(dataFrame).show(); Refer to the [ElementwiseProduct Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.ElementwiseProduct) for more details on the API. -{% highlight python %} -from pyspark.ml.feature import ElementwiseProduct -from pyspark.mllib.linalg import Vectors +{% include_example python/ml/elementwise_product_example.py %} +
    +
    + +## SQLTransformer + +`SQLTransformer` implements the transformations which are defined by SQL statement. +Currently we only support SQL syntax like `"SELECT ... FROM __THIS__ ..."` +where `"__THIS__"` represents the underlying table of the input dataset. +The select clause specifies the fields, constants, and expressions to display in +the output, and can be any select clause that Spark SQL supports. Users can also +use Spark SQL built-in function and UDFs to operate on these selected columns. +For example, `SQLTransformer` supports statements like: + +* `SELECT a, a + b AS a_b FROM __THIS__` +* `SELECT a, SQRT(b) AS b_sqrt FROM __THIS__ where a > 5` +* `SELECT a, b, SUM(c) AS c_sum FROM __THIS__ GROUP BY a, b` + +**Examples** + +Assume that we have the following DataFrame with columns `id`, `v1` and `v2`: + +~~~~ + id | v1 | v2 +----|-----|----- + 0 | 1.0 | 3.0 + 2 | 2.0 | 5.0 +~~~~ + +This is the output of the `SQLTransformer` with statement `"SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__"`: -data = [(Vectors.dense([1.0, 2.0, 3.0]),), (Vectors.dense([4.0, 5.0, 6.0]),)] -df = sqlContext.createDataFrame(data, ["vector"]) -transformer = ElementwiseProduct(scalingVec=Vectors.dense([0.0, 1.0, 2.0]), - inputCol="vector", outputCol="transformedVector") -transformer.transform(df).show() +~~~~ + id | v1 | v2 | v3 | v4 +----|-----|-----|-----|----- + 0 | 1.0 | 3.0 | 4.0 | 3.0 + 2 | 2.0 | 5.0 | 7.0 |10.0 +~~~~ -{% endhighlight %} +
    +
    + +Refer to the [SQLTransformer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.SQLTransformer) +for more details on the API. + +{% include_example scala/org/apache/spark/examples/ml/SQLTransformerExample.scala %}
    +
    + +Refer to the [SQLTransformer Java docs](api/java/org/apache/spark/ml/feature/SQLTransformer.html) +for more details on the API. + +{% include_example java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java %} +
    + +
    + +Refer to the [SQLTransformer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.SQLTransformer) for more details on the API. + +{% include_example python/ml/sql_transformer.py %} +
    ## VectorAssembler @@ -1632,19 +1263,7 @@ output column to `features`, after transformation we should get the following Da Refer to the [VectorAssembler Scala docs](api/scala/index.html#org.apache.spark.ml.feature.VectorAssembler) for more details on the API. -{% highlight scala %} -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.ml.feature.VectorAssembler - -val dataset = sqlContext.createDataFrame( - Seq((0, 18, 1.0, Vectors.dense(0.0, 10.0, 0.5), 1.0)) -).toDF("id", "hour", "mobile", "userFeatures", "clicked") -val assembler = new VectorAssembler() - .setInputCols(Array("hour", "mobile", "userFeatures")) - .setOutputCol("features") -val output = assembler.transform(dataset) -println(output.select("features", "clicked").first()) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala %}
    @@ -1652,56 +1271,165 @@ println(output.select("features", "clicked").first()) Refer to the [VectorAssembler Java docs](api/java/org/apache/spark/ml/feature/VectorAssembler.html) for more details on the API. -{% highlight java %} -import java.util.Arrays; +{% include_example java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java %} +
    + +
    + +Refer to the [VectorAssembler Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.VectorAssembler) +for more details on the API. + +{% include_example python/ml/vector_assembler_example.py %} +
    +
    + +## QuantileDiscretizer + +`QuantileDiscretizer` takes a column with continuous features and outputs a column with binned +categorical features. The number of bins is set by the `numBuckets` parameter. It is possible +that the number of buckets used will be smaller than this value, for example, if there are too few +distinct values of the input to create enough distinct quantiles. + +NaN values: +NaN values will be removed from the column during `QuantileDiscretizer` fitting. This will produce +a `Bucketizer` model for making predictions. During the transformation, `Bucketizer` +will raise an error when it finds NaN values in the dataset, but the user can also choose to either +keep or remove NaN values within the dataset by setting `handleInvalid`. If the user chooses to keep +NaN values, they will be handled specially and placed into their own bucket, for example, if 4 buckets +are used, then non-NaN data will be put into buckets[0-3], but NaNs will be counted in a special bucket[4]. + +Algorithm: The bin ranges are chosen using an approximate algorithm (see the documentation for +[approxQuantile](api/scala/index.html#org.apache.spark.sql.DataFrameStatFunctions) for a +detailed description). The precision of the approximation can be controlled with the +`relativeError` parameter. When set to zero, exact quantiles are calculated +(**Note:** Computing exact quantiles is an expensive operation). The lower and upper bin bounds +will be `-Infinity` and `+Infinity` covering all real values. + +**Examples** -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.types.*; -import static org.apache.spark.sql.types.DataTypes.*; +Assume that we have a DataFrame with the columns `id`, `hour`: -StructType schema = createStructType(new StructField[] { - createStructField("id", IntegerType, false), - createStructField("hour", IntegerType, false), - createStructField("mobile", DoubleType, false), - createStructField("userFeatures", new VectorUDT(), false), - createStructField("clicked", DoubleType, false) -}); -Row row = RowFactory.create(0, 18, 1.0, Vectors.dense(0.0, 10.0, 0.5), 1.0); -JavaRDD rdd = jsc.parallelize(Arrays.asList(row)); -DataFrame dataset = sqlContext.createDataFrame(rdd, schema); +~~~ + id | hour +----|------ + 0 | 18.0 +----|------ + 1 | 19.0 +----|------ + 2 | 8.0 +----|------ + 3 | 5.0 +----|------ + 4 | 2.2 +~~~ + +`hour` is a continuous feature with `Double` type. We want to turn the continuous feature into +a categorical one. Given `numBuckets = 3`, we should get the following DataFrame: + +~~~ + id | hour | result +----|------|------ + 0 | 18.0 | 2.0 +----|------|------ + 1 | 19.0 | 2.0 +----|------|------ + 2 | 8.0 | 1.0 +----|------|------ + 3 | 5.0 | 1.0 +----|------|------ + 4 | 2.2 | 0.0 +~~~ + +
    +
    + +Refer to the [QuantileDiscretizer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.QuantileDiscretizer) +for more details on the API. + +{% include_example scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala %} +
    + +
    -VectorAssembler assembler = new VectorAssembler() - .setInputCols(new String[] {"hour", "mobile", "userFeatures"}) - .setOutputCol("features"); +Refer to the [QuantileDiscretizer Java docs](api/java/org/apache/spark/ml/feature/QuantileDiscretizer.html) +for more details on the API. -DataFrame output = assembler.transform(dataset); -System.out.println(output.select("features", "clicked").first()); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java %}
    -Refer to the [VectorAssembler Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.VectorAssembler) +Refer to the [QuantileDiscretizer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.QuantileDiscretizer) +for more details on the API. + +{% include_example python/ml/quantile_discretizer_example.py %} +
    + +
    + + +## Imputer + +The `Imputer` transformer completes missing values in a dataset, either using the mean or the +median of the columns in which the missing values are located. The input columns should be of +`DoubleType` or `FloatType`. Currently `Imputer` does not support categorical features and possibly +creates incorrect values for columns containing categorical features. + +**Note** all `null` values in the input columns are treated as missing, and so are also imputed. + +**Examples** + +Suppose that we have a DataFrame with the columns `a` and `b`: + +~~~ + a | b +------------|----------- + 1.0 | Double.NaN + 2.0 | Double.NaN + Double.NaN | 3.0 + 4.0 | 4.0 + 5.0 | 5.0 +~~~ + +In this example, Imputer will replace all occurrences of `Double.NaN` (the default for the missing value) +with the mean (the default imputation strategy) computed from the other values in the corresponding columns. +In this example, the surrogate values for columns `a` and `b` are 3.0 and 4.0 respectively. After +transformation, the missing values in the output columns will be replaced by the surrogate value for +the relevant column. + +~~~ + a | b | out_a | out_b +------------|------------|-------|------- + 1.0 | Double.NaN | 1.0 | 4.0 + 2.0 | Double.NaN | 2.0 | 4.0 + Double.NaN | 3.0 | 3.0 | 3.0 + 4.0 | 4.0 | 4.0 | 4.0 + 5.0 | 5.0 | 5.0 | 5.0 +~~~ + +
    +
    + +Refer to the [Imputer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.Imputer) +for more details on the API. + +{% include_example scala/org/apache/spark/examples/ml/ImputerExample.scala %} +
    + +
    + +Refer to the [Imputer Java docs](api/java/org/apache/spark/ml/feature/Imputer.html) for more details on the API. -{% highlight python %} -from pyspark.mllib.linalg import Vectors -from pyspark.ml.feature import VectorAssembler +{% include_example java/org/apache/spark/examples/ml/JavaImputerExample.java %} +
    + +
    + +Refer to the [Imputer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.Imputer) +for more details on the API. -dataset = sqlContext.createDataFrame( - [(0, 18, 1.0, Vectors.dense([0.0, 10.0, 0.5]), 1.0)], - ["id", "hour", "mobile", "userFeatures", "clicked"]) -assembler = VectorAssembler( - inputCols=["hour", "mobile", "userFeatures"], - outputCol="features") -output = assembler.transform(dataset) -print(output.select("features", "clicked").first()) -{% endhighlight %} +{% include_example python/ml/imputer_example.py %}
    @@ -1712,19 +1440,19 @@ print(output.select("features", "clicked").first()) `VectorSlicer` is a transformer that takes a feature vector and outputs a new feature vector with a sub-array of the original features. It is useful for extracting features from a vector column. -`VectorSlicer` accepts a vector column with a specified indices, then outputs a new vector column -whose values are selected via those indices. There are two types of indices, +`VectorSlicer` accepts a vector column with specified indices, then outputs a new vector column +whose values are selected via those indices. There are two types of indices, - 1. Integer indices that represents the indices into the vector, `setIndices()`; + 1. Integer indices that represent the indices into the vector, `setIndices()`. - 2. String indices that represents the names of features into the vector, `setNames()`. + 2. String indices that represent the names of features into the vector, `setNames()`. *This requires the vector column to have an `AttributeGroup` since the implementation matches on the name field of an `Attribute`.* -Specification by integer and string are both acceptable. Moreover, you can use integer index and +Specification by integer and string are both acceptable. Moreover, you can use integer index and string name simultaneously. At least one feature must be selected. Duplicate features are not allowed, so there can be no overlap between selected indices and names. Note that if names of -features are selected, an exception will be threw out when encountering with empty input attributes. +features are selected, an exception will be thrown if empty input attributes are encountered. The output vector will order features with the selected indices first (in the order given), followed by the selected names (in the order given). @@ -1734,13 +1462,13 @@ followed by the selected names (in the order given). Suppose that we have a DataFrame with the column `userFeatures`: ~~~ - userFeatures + userFeatures ------------------ - [0.0, 10.0, 0.5] + [0.0, 10.0, 0.5] ~~~ -`userFeatures` is a vector column that contains three user features. Assuming that the first column -of `userFeatures` are all zeros, so we want to remove it and only the last two columns are selected. +`userFeatures` is a vector column that contains three user features. Assume that the first column +of `userFeatures` are all zeros, so we want to remove it and select only the last two columns. The `VectorSlicer` selects the last two elements with `setIndices(1, 2)` then produces a new vector column named `features`: @@ -1750,7 +1478,7 @@ column named `features`: [0.0, 10.0, 0.5] | [10.0, 0.5] ~~~ -Suppose also that we have a potential input attributes for the `userFeatures`, i.e. +Suppose also that we have potential input attributes for the `userFeatures`, i.e. `["f1", "f2", "f3"]`, then we can use `setNames("f2", "f3")` to select them. ~~~ @@ -1766,33 +1494,7 @@ Suppose also that we have a potential input attributes for the `userFeatures`, i Refer to the [VectorSlicer Scala docs](api/scala/index.html#org.apache.spark.ml.feature.VectorSlicer) for more details on the API. -{% highlight scala %} -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute} -import org.apache.spark.ml.feature.VectorSlicer -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{DataFrame, Row, SQLContext} - -val data = Array( - Vectors.sparse(3, Seq((0, -2.0), (1, 2.3))), - Vectors.dense(-2.0, 2.3, 0.0) -) - -val defaultAttr = NumericAttribute.defaultAttr -val attrs = Array("f1", "f2", "f3").map(defaultAttr.withName) -val attrGroup = new AttributeGroup("userFeatures", attrs.asInstanceOf[Array[Attribute]]) - -val dataRDD = sc.parallelize(data).map(Row.apply) -val dataset = sqlContext.createDataFrame(dataRDD, StructType(attrGroup.toStructField())) - -val slicer = new VectorSlicer().setInputCol("userFeatures").setOutputCol("features") - -slicer.setIndices(1).setNames("f3") -// or slicer.setIndices(Array(1, 2)), or slicer.setNames(Array("f2", "f3")) - -val output = slicer.transform(dataset) -println(output.select("userFeatures", "features").first()) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/VectorSlicerExample.scala %}
    @@ -1800,47 +1502,39 @@ println(output.select("userFeatures", "features").first()) Refer to the [VectorSlicer Java docs](api/java/org/apache/spark/ml/feature/VectorSlicer.html) for more details on the API. -{% highlight java %} -import java.util.Arrays; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.types.*; -import static org.apache.spark.sql.types.DataTypes.*; +{% include_example java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java %} +
    -Attribute[] attrs = new Attribute[]{ - NumericAttribute.defaultAttr().withName("f1"), - NumericAttribute.defaultAttr().withName("f2"), - NumericAttribute.defaultAttr().withName("f3") -}; -AttributeGroup group = new AttributeGroup("userFeatures", attrs); +
    -JavaRDD jrdd = jsc.parallelize(Lists.newArrayList( - RowFactory.create(Vectors.sparse(3, new int[]{0, 1}, new double[]{-2.0, 2.3})), - RowFactory.create(Vectors.dense(-2.0, 2.3, 0.0)) -)); +Refer to the [VectorSlicer Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.VectorSlicer) +for more details on the API. -DataFrame dataset = jsql.createDataFrame(jrdd, (new StructType()).add(group.toStructField())); +{% include_example python/ml/vector_slicer_example.py %} +
    +
    -VectorSlicer vectorSlicer = new VectorSlicer() - .setInputCol("userFeatures").setOutputCol("features"); +## RFormula -vectorSlicer.setIndices(new int[]{1}).setNames(new String[]{"f3"}); -// or slicer.setIndices(new int[]{1, 2}), or slicer.setNames(new String[]{"f2", "f3"}) +`RFormula` selects columns specified by an [R model formula](https://stat.ethz.ch/R-manual/R-devel/library/stats/html/formula.html). +Currently we support a limited subset of the R operators, including '~', '.', ':', '+', and '-'. +The basic operators are: -DataFrame output = vectorSlicer.transform(dataset); +* `~` separate target and terms +* `+` concat terms, "+ 0" means removing intercept +* `-` remove a term, "- 1" means removing intercept +* `:` interaction (multiplication for numeric values, or binarized categorical values) +* `.` all columns except target -System.out.println(output.select("userFeatures", "features").first()); -{% endhighlight %} -
    - +Suppose `a` and `b` are double columns, we use the following simple examples to illustrate the effect of `RFormula`: -## RFormula +* `y ~ a + b` means model `y ~ w0 + w1 * a + w2 * b` where `w0` is the intercept and `w1, w2` are coefficients. +* `y ~ a + b + a:b - 1` means model `y ~ w1 * a + w2 * b + w3 * a * b` where `w1, w2, w3` are coefficients. -`RFormula` selects columns specified by an [R model formula](https://stat.ethz.ch/R-manual/R-devel/library/stats/html/formula.html). It produces a vector column of features and a double column of labels. Like when formulas are used in R for linear regression, string input columns will be one-hot encoded, and numeric columns will be cast to doubles. If not already present in the DataFrame, the output label column will be created from the specified response variable in the formula. +`RFormula` produces a vector column of features and a double or string column of label. +Like when formulas are used in R for linear regression, string input columns will be one-hot encoded, and numeric columns will be cast to doubles. +If the label column is of type string, it will be first transformed to double with `StringIndexer`. +If the label column does not exist in the DataFrame, the output label column will be created from the specified response variable in the formula. **Examples** @@ -1871,21 +1565,7 @@ id | country | hour | clicked | features | label Refer to the [RFormula Scala docs](api/scala/index.html#org.apache.spark.ml.feature.RFormula) for more details on the API. -{% highlight scala %} -import org.apache.spark.ml.feature.RFormula - -val dataset = sqlContext.createDataFrame(Seq( - (7, "US", 18, 1.0), - (8, "CA", 12, 0.0), - (9, "NZ", 15, 0.0) -)).toDF("id", "country", "hour", "clicked") -val formula = new RFormula() - .setFormula("clicked ~ country + hour") - .setFeaturesCol("features") - .setLabelCol("label") -val output = formula.fit(dataset).transform(dataset) -output.select("features", "label").show() -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/ml/RFormulaExample.scala %}
    @@ -1893,38 +1573,7 @@ output.select("features", "label").show() Refer to the [RFormula Java docs](api/java/org/apache/spark/ml/feature/RFormula.html) for more details on the API. -{% highlight java %} -import java.util.Arrays; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.ml.feature.RFormula; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.types.*; -import static org.apache.spark.sql.types.DataTypes.*; - -StructType schema = createStructType(new StructField[] { - createStructField("id", IntegerType, false), - createStructField("country", StringType, false), - createStructField("hour", IntegerType, false), - createStructField("clicked", DoubleType, false) -}); -JavaRDD rdd = jsc.parallelize(Arrays.asList( - RowFactory.create(7, "US", 18, 1.0), - RowFactory.create(8, "CA", 12, 0.0), - RowFactory.create(9, "NZ", 15, 0.0) -)); -DataFrame dataset = sqlContext.createDataFrame(rdd, schema); - -RFormula formula = new RFormula() - .setFormula("clicked ~ country + hour") - .setFeaturesCol("features") - .setLabelCol("label"); - -DataFrame output = formula.fit(dataset).transform(dataset); -output.select("features", "label").show(); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/ml/JavaRFormulaExample.java %}
    @@ -1932,20 +1581,197 @@ output.select("features", "label").show(); Refer to the [RFormula Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.RFormula) for more details on the API. -{% highlight python %} -from pyspark.ml.feature import RFormula - -dataset = sqlContext.createDataFrame( - [(7, "US", 18, 1.0), - (8, "CA", 12, 0.0), - (9, "NZ", 15, 0.0)], - ["id", "country", "hour", "clicked"]) -formula = RFormula( - formula="clicked ~ country + hour", - featuresCol="features", - labelCol="label") -output = formula.fit(dataset).transform(dataset) -output.select("features", "label").show() -{% endhighlight %} +{% include_example python/ml/rformula_example.py %} +
    + + +## ChiSqSelector + +`ChiSqSelector` stands for Chi-Squared feature selection. It operates on labeled data with +categorical features. ChiSqSelector uses the +[Chi-Squared test of independence](https://en.wikipedia.org/wiki/Chi-squared_test) to decide which +features to choose. It supports five selection methods: `numTopFeatures`, `percentile`, `fpr`, `fdr`, `fwe`: +* `numTopFeatures` chooses a fixed number of top features according to a chi-squared test. This is akin to yielding the features with the most predictive power. +* `percentile` is similar to `numTopFeatures` but chooses a fraction of all features instead of a fixed number. +* `fpr` chooses all features whose p-values are below a threshold, thus controlling the false positive rate of selection. +* `fdr` uses the [Benjamini-Hochberg procedure](https://en.wikipedia.org/wiki/False_discovery_rate#Benjamini.E2.80.93Hochberg_procedure) to choose all features whose false discovery rate is below a threshold. +* `fwe` chooses all features whose p-values are below a threshold. The threshold is scaled by 1/numFeatures, thus controlling the family-wise error rate of selection. +By default, the selection method is `numTopFeatures`, with the default number of top features set to 50. +The user can choose a selection method using `setSelectorType`. + +**Examples** + +Assume that we have a DataFrame with the columns `id`, `features`, and `clicked`, which is used as +our target to be predicted: + +~~~ +id | features | clicked +---|-----------------------|--------- + 7 | [0.0, 0.0, 18.0, 1.0] | 1.0 + 8 | [0.0, 1.0, 12.0, 0.0] | 0.0 + 9 | [1.0, 0.0, 15.0, 0.1] | 0.0 +~~~ + +If we use `ChiSqSelector` with `numTopFeatures = 1`, then according to our label `clicked` the +last column in our `features` is chosen as the most useful feature: + +~~~ +id | features | clicked | selectedFeatures +---|-----------------------|---------|------------------ + 7 | [0.0, 0.0, 18.0, 1.0] | 1.0 | [1.0] + 8 | [0.0, 1.0, 12.0, 0.0] | 0.0 | [0.0] + 9 | [1.0, 0.0, 15.0, 0.1] | 0.0 | [0.1] +~~~ + +
    +
    + +Refer to the [ChiSqSelector Scala docs](api/scala/index.html#org.apache.spark.ml.feature.ChiSqSelector) +for more details on the API. + +{% include_example scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala %} +
    + +
    + +Refer to the [ChiSqSelector Java docs](api/java/org/apache/spark/ml/feature/ChiSqSelector.html) +for more details on the API. + +{% include_example java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java %} +
    + +
    + +Refer to the [ChiSqSelector Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.ChiSqSelector) +for more details on the API. + +{% include_example python/ml/chisq_selector_example.py %} +
    +
    + +# Locality Sensitive Hashing +[Locality Sensitive Hashing (LSH)](https://en.wikipedia.org/wiki/Locality-sensitive_hashing) is an important class of hashing techniques, which is commonly used in clustering, approximate nearest neighbor search and outlier detection with large datasets. + +The general idea of LSH is to use a family of functions ("LSH families") to hash data points into buckets, so that the data points which are close to each other are in the same buckets with high probability, while data points that are far away from each other are very likely in different buckets. An LSH family is formally defined as follows. + +In a metric space `(M, d)`, where `M` is a set and `d` is a distance function on `M`, an LSH family is a family of functions `h` that satisfy the following properties: +`\[ +\forall p, q \in M,\\ +d(p,q) \leq r1 \Rightarrow Pr(h(p)=h(q)) \geq p1\\ +d(p,q) \geq r2 \Rightarrow Pr(h(p)=h(q)) \leq p2 +\]` +This LSH family is called `(r1, r2, p1, p2)`-sensitive. + +In Spark, different LSH families are implemented in separate classes (e.g., `MinHash`), and APIs for feature transformation, approximate similarity join and approximate nearest neighbor are provided in each class. + +In LSH, we define a false positive as a pair of distant input features (with `$d(p,q) \geq r2$`) which are hashed into the same bucket, and we define a false negative as a pair of nearby features (with `$d(p,q) \leq r1$`) which are hashed into different buckets. + +## LSH Operations + +We describe the major types of operations which LSH can be used for. A fitted LSH model has methods for each of these operations. + +### Feature Transformation +Feature transformation is the basic functionality to add hashed values as a new column. This can be useful for dimensionality reduction. Users can specify input and output column names by setting `inputCol` and `outputCol`. + +LSH also supports multiple LSH hash tables. Users can specify the number of hash tables by setting `numHashTables`. This is also used for [OR-amplification](https://en.wikipedia.org/wiki/Locality-sensitive_hashing#Amplification) in approximate similarity join and approximate nearest neighbor. Increasing the number of hash tables will increase the accuracy but will also increase communication cost and running time. + +The type of `outputCol` is `Seq[Vector]` where the dimension of the array equals `numHashTables`, and the dimensions of the vectors are currently set to 1. In future releases, we will implement AND-amplification so that users can specify the dimensions of these vectors. + +### Approximate Similarity Join +Approximate similarity join takes two datasets and approximately returns pairs of rows in the datasets whose distance is smaller than a user-defined threshold. Approximate similarity join supports both joining two different datasets and self-joining. Self-joining will produce some duplicate pairs. + +Approximate similarity join accepts both transformed and untransformed datasets as input. If an untransformed dataset is used, it will be transformed automatically. In this case, the hash signature will be created as `outputCol`. + +In the joined dataset, the origin datasets can be queried in `datasetA` and `datasetB`. A distance column will be added to the output dataset to show the true distance between each pair of rows returned. + +### Approximate Nearest Neighbor Search +Approximate nearest neighbor search takes a dataset (of feature vectors) and a key (a single feature vector), and it approximately returns a specified number of rows in the dataset that are closest to the vector. + +Approximate nearest neighbor search accepts both transformed and untransformed datasets as input. If an untransformed dataset is used, it will be transformed automatically. In this case, the hash signature will be created as `outputCol`. + +A distance column will be added to the output dataset to show the true distance between each output row and the searched key. + +**Note:** Approximate nearest neighbor search will return fewer than `k` rows when there are not enough candidates in the hash bucket. + +## LSH Algorithms + +### Bucketed Random Projection for Euclidean Distance + +[Bucketed Random Projection](https://en.wikipedia.org/wiki/Locality-sensitive_hashing#Stable_distributions) is an LSH family for Euclidean distance. The Euclidean distance is defined as follows: +`\[ +d(\mathbf{x}, \mathbf{y}) = \sqrt{\sum_i (x_i - y_i)^2} +\]` +Its LSH family projects feature vectors `$\mathbf{x}$` onto a random unit vector `$\mathbf{v}$` and portions the projected results into hash buckets: +`\[ +h(\mathbf{x}) = \Big\lfloor \frac{\mathbf{x} \cdot \mathbf{v}}{r} \Big\rfloor +\]` +where `r` is a user-defined bucket length. The bucket length can be used to control the average size of hash buckets (and thus the number of buckets). A larger bucket length (i.e., fewer buckets) increases the probability of features being hashed to the same bucket (increasing the numbers of true and false positives). + +Bucketed Random Projection accepts arbitrary vectors as input features, and supports both sparse and dense vectors. + +
    +
    + +Refer to the [BucketedRandomProjectionLSH Scala docs](api/scala/index.html#org.apache.spark.ml.feature.BucketedRandomProjectionLSH) +for more details on the API. + +{% include_example scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala %} +
    + +
    + +Refer to the [BucketedRandomProjectionLSH Java docs](api/java/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.html) +for more details on the API. + +{% include_example java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java %} +
    + +
    + +Refer to the [BucketedRandomProjectionLSH Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.BucketedRandomProjectionLSH) +for more details on the API. + +{% include_example python/ml/bucketed_random_projection_lsh_example.py %} +
    + +
    + +### MinHash for Jaccard Distance +[MinHash](https://en.wikipedia.org/wiki/MinHash) is an LSH family for Jaccard distance where input features are sets of natural numbers. Jaccard distance of two sets is defined by the cardinality of their intersection and union: +`\[ +d(\mathbf{A}, \mathbf{B}) = 1 - \frac{|\mathbf{A} \cap \mathbf{B}|}{|\mathbf{A} \cup \mathbf{B}|} +\]` +MinHash applies a random hash function `g` to each element in the set and take the minimum of all hashed values: +`\[ +h(\mathbf{A}) = \min_{a \in \mathbf{A}}(g(a)) +\]` + +The input sets for MinHash are represented as binary vectors, where the vector indices represent the elements themselves and the non-zero values in the vector represent the presence of that element in the set. While both dense and sparse vectors are supported, typically sparse vectors are recommended for efficiency. For example, `Vectors.sparse(10, Array[(2, 1.0), (3, 1.0), (5, 1.0)])` means there are 10 elements in the space. This set contains elem 2, elem 3 and elem 5. All non-zero values are treated as binary "1" values. + +**Note:** Empty sets cannot be transformed by MinHash, which means any input vector must have at least 1 non-zero entry. + +
    +
    + +Refer to the [MinHashLSH Scala docs](api/scala/index.html#org.apache.spark.ml.feature.MinHashLSH) +for more details on the API. + +{% include_example scala/org/apache/spark/examples/ml/MinHashLSHExample.scala %} +
    + +
    + +Refer to the [MinHashLSH Java docs](api/java/org/apache/spark/ml/feature/MinHashLSH.html) +for more details on the API. + +{% include_example java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java %} +
    + +
    + +Refer to the [MinHashLSH Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.MinHashLSH) +for more details on the API. + +{% include_example python/ml/min_hash_lsh_example.py %}
    diff --git a/docs/ml-frequent-pattern-mining.md b/docs/ml-frequent-pattern-mining.md new file mode 100644 index 000000000000..81634de8aade --- /dev/null +++ b/docs/ml-frequent-pattern-mining.md @@ -0,0 +1,87 @@ +--- +layout: global +title: Frequent Pattern Mining +displayTitle: 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. + +**Table of Contents** + +* This will become a table of contents (this text will be scraped). +{:toc} + +## 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 `spark.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 suffixes of transactions, +and hence is more scalable than a single-machine implementation. +We refer users to the papers for more details. + +`spark.ml`'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. +* `minConfidence`: minimum confidence for generating Association Rule. Confidence is an indication of how often an + association rule has been found to be true. For example, if in the transactions itemset `X` appears 4 times, `X` + and `Y` co-occur only 2 times, the confidence for the rule `X => Y` is then 2/4 = 0.5. The parameter will not + affect the mining for frequent itemsets, but specify the minimum confidence for generating association rules + from frequent itemsets. +* `numPartitions`: the number of partitions used to distribute the work. By default the param is not set, and + number of partitions of the input dataset is used. + +The `FPGrowthModel` provides: + +* `freqItemsets`: frequent itemsets in the format of DataFrame("items"[Array], "freq"[Long]) +* `associationRules`: association rules generated with confidence above `minConfidence`, in the format of + DataFrame("antecedent"[Array], "consequent"[Array], "confidence"[Double]). +* `transform`: For each transaction in `itemsCol`, the `transform` method will compare its items against the antecedents + of each association rule. If the record contains all the antecedents of a specific association rule, the rule + will be considered as applicable and its consequents will be added to the prediction result. The transform + method will summarize the consequents from all the applicable rules as prediction. The prediction column has + the same data type as `itemsCol` and does not contain existing items in the `itemsCol`. + + +**Examples** + +
    + +
    +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.fpm.FPGrowth) for more details. + +{% include_example scala/org/apache/spark/examples/ml/FPGrowthExample.scala %} +
    + +
    +Refer to the [Java API docs](api/java/org/apache/spark/ml/fpm/FPGrowth.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaFPGrowthExample.java %} +
    + +
    +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.fpm.FPGrowth) for more details. + +{% include_example python/ml/fpgrowth_example.py %} +
    + +
    + +Refer to the [R API docs](api/R/spark.fpGrowth.html) for more details. + +{% include_example r/ml/fpm.R %} +
    + +
    diff --git a/docs/ml-guide.md b/docs/ml-guide.md index fd3a6167bc65..f6288e7c32d9 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -1,958 +1,153 @@ --- layout: global -title: Spark ML Programming Guide +title: "MLlib: Main Guide" +displayTitle: "Machine Learning Library (MLlib) Guide" --- -`\[ -\newcommand{\R}{\mathbb{R}} -\newcommand{\E}{\mathbb{E}} -\newcommand{\x}{\mathbf{x}} -\newcommand{\y}{\mathbf{y}} -\newcommand{\wv}{\mathbf{w}} -\newcommand{\av}{\mathbf{\alpha}} -\newcommand{\bv}{\mathbf{b}} -\newcommand{\N}{\mathbb{N}} -\newcommand{\id}{\mathbf{I}} -\newcommand{\ind}{\mathbf{1}} -\newcommand{\0}{\mathbf{0}} -\newcommand{\unit}{\mathbf{e}} -\newcommand{\one}{\mathbf{1}} -\newcommand{\zero}{\mathbf{0}} -\]` +MLlib is Spark's machine learning (ML) library. +Its goal is to make practical machine learning scalable and easy. +At a high level, it provides tools such as: +* ML Algorithms: common learning algorithms such as classification, regression, clustering, and collaborative filtering +* Featurization: feature extraction, transformation, dimensionality reduction, and selection +* Pipelines: tools for constructing, evaluating, and tuning ML Pipelines +* Persistence: saving and load algorithms, models, and Pipelines +* Utilities: linear algebra, statistics, data handling, etc. -The `spark.ml` package aims to provide a uniform set of high-level APIs built on top of -[DataFrames](sql-programming-guide.html#dataframes) that help users create and tune practical -machine learning pipelines. -See the [algorithm guides](#algorithm-guides) section below for guides on sub-packages of -`spark.ml`, including feature transformers unique to the Pipelines API, ensembles, and more. +# Announcement: DataFrame-based API is primary API -**Table of contents** +**The MLlib RDD-based API is now in maintenance mode.** -* This will become a table of contents (this text will be scraped). -{:toc} +As of Spark 2.0, the [RDD](rdd-programming-guide.html#resilient-distributed-datasets-rdds)-based APIs in the `spark.mllib` package have entered maintenance mode. +The primary Machine Learning API for Spark is now the [DataFrame](sql-programming-guide.html)-based API in the `spark.ml` package. -# Algorithm guides +*What are the implications?* -We provide several algorithm guides specific to the Pipelines API. -Several of these algorithms, such as certain feature transformers, are not in the `spark.mllib` API. -Also, some algorithms have additional capabilities in the `spark.ml` API; e.g., random forests -provide class probabilities, and linear models provide model summaries. +* MLlib will still support the RDD-based API in `spark.mllib` with bug fixes. +* MLlib will not add new features to the RDD-based API. +* In the Spark 2.x releases, MLlib will add features to the DataFrames-based API to reach feature parity with the RDD-based API. +* After reaching feature parity (roughly estimated for Spark 2.3), the RDD-based API will be deprecated. +* The RDD-based API is expected to be removed in Spark 3.0. -* [Feature extraction, transformation, and selection](ml-features.html) -* [Decision Trees for classification and regression](ml-decision-tree.html) -* [Ensembles](ml-ensembles.html) -* [Linear methods with elastic net regularization](ml-linear-methods.html) -* [Multilayer perceptron classifier](ml-ann.html) +*Why is MLlib switching to the DataFrame-based API?* +* DataFrames provide a more user-friendly API than RDDs. The many benefits of DataFrames include Spark Datasources, SQL/DataFrame queries, Tungsten and Catalyst optimizations, and uniform APIs across languages. +* The DataFrame-based API for MLlib provides a uniform API across ML algorithms and across multiple languages. +* DataFrames facilitate practical ML Pipelines, particularly feature transformations. See the [Pipelines guide](ml-pipeline.html) for details. -# Main concepts in Pipelines +*What is "Spark ML"?* -Spark ML standardizes APIs for machine learning algorithms to make it easier to combine multiple -algorithms into a single pipeline, or workflow. -This section covers the key concepts introduced by the Spark ML API, where the pipeline concept is -mostly inspired by the [scikit-learn](http://scikit-learn.org/) project. +* "Spark ML" is not an official name but occasionally used to refer to the MLlib DataFrame-based API. + This is majorly due to the `org.apache.spark.ml` Scala package name used by the DataFrame-based API, + and the "Spark ML Pipelines" term we used initially to emphasize the pipeline concept. + +*Is MLlib deprecated?* -* **[`DataFrame`](ml-guide.html#dataframe)**: Spark ML uses `DataFrame` from Spark SQL as an ML - dataset, which can hold a variety of data types. - E.g., a `DataFrame` could have different columns storing text, feature vectors, true labels, and predictions. +* No. MLlib includes both the RDD-based API and the DataFrame-based API. + The RDD-based API is now in maintenance mode. + But neither API is deprecated, nor MLlib as a whole. -* **[`Transformer`](ml-guide.html#transformers)**: A `Transformer` is an algorithm which can transform one `DataFrame` into another `DataFrame`. -E.g., an ML model is a `Transformer` which transforms `DataFrame` with features into a `DataFrame` with predictions. +# Dependencies -* **[`Estimator`](ml-guide.html#estimators)**: An `Estimator` is an algorithm which can be fit on a `DataFrame` to produce a `Transformer`. -E.g., a learning algorithm is an `Estimator` which trains on a `DataFrame` and produces a model. +MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), which depends on +[netlib-java](https://github.com/fommil/netlib-java) for optimised numerical processing. +If native libraries[^1] are not available at runtime, you will see a warning message and a pure JVM +implementation will be used instead. -* **[`Pipeline`](ml-guide.html#pipeline)**: A `Pipeline` chains multiple `Transformer`s and `Estimator`s together to specify an ML workflow. +Due to licensing issues with runtime proprietary binaries, we do not include `netlib-java`'s native +proxies by default. +To configure `netlib-java` / Breeze to use system optimised binaries, include +`com.github.fommil.netlib:all:1.1.2` (or build Spark with `-Pnetlib-lgpl`) as a dependency of your +project and read the [netlib-java](https://github.com/fommil/netlib-java) documentation for your +platform's additional installation instructions. -* **[`Parameter`](ml-guide.html#parameters)**: All `Transformer`s and `Estimator`s now share a common API for specifying parameters. +The most popular native BLAS such as [Intel MKL](https://software.intel.com/en-us/mkl), [OpenBLAS](http://www.openblas.net), can use multiple threads in a single operation, which can conflict with Spark's execution model. -## DataFrame +Configuring these BLAS implementations to use a single thread for operations may actually improve performance (see [SPARK-21305](https://issues.apache.org/jira/browse/SPARK-21305)). It is usually optimal to match this to the number of cores each Spark task is configured to use, which is 1 by default and typically left at 1. -Machine learning can be applied to a wide variety of data types, such as vectors, text, images, and structured data. -Spark ML adopts the `DataFrame` from Spark SQL in order to support a variety of data types. +Please refer to resources like the following to understand how to configure the number of threads these BLAS implementations use: [Intel MKL](https://software.intel.com/en-us/articles/recommended-settings-for-calling-intel-mkl-routines-from-multi-threaded-applications) and [OpenBLAS](https://github.com/xianyi/OpenBLAS/wiki/faq#multi-threaded). -`DataFrame` supports many basic and structured types; see the [Spark SQL datatype reference](sql-programming-guide.html#spark-sql-datatype-reference) for a list of supported types. -In addition to the types listed in the Spark SQL guide, `DataFrame` can use ML [`Vector`](mllib-data-types.html#local-vector) types. +To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. -A `DataFrame` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples. +[^1]: To learn more about the benefits and background of system optimised natives, you may wish to + watch Sam Halliday's ScalaX talk on [High Performance Linear Algebra in Scala](http://fommil.github.io/scalax14/#/). -Columns in a `DataFrame` are named. The code examples below use names such as "text," "features," and "label." +# Highlights in 2.2 -## Pipeline components +The list below highlights some of the new features and enhancements added to MLlib in the `2.2` +release of Spark: -### Transformers +* [`ALS`](ml-collaborative-filtering.html) methods for _top-k_ recommendations for all + users or items, matching the functionality in `mllib` + ([SPARK-19535](https://issues.apache.org/jira/browse/SPARK-19535)). + Performance was also improved for both `ml` and `mllib` + ([SPARK-11968](https://issues.apache.org/jira/browse/SPARK-11968) and + [SPARK-20587](https://issues.apache.org/jira/browse/SPARK-20587)) +* [`Correlation`](ml-statistics.html#correlation) and + [`ChiSquareTest`](ml-statistics.html#hypothesis-testing) stats functions for `DataFrames` + ([SPARK-19636](https://issues.apache.org/jira/browse/SPARK-19636) and + [SPARK-19635](https://issues.apache.org/jira/browse/SPARK-19635)) +* [`FPGrowth`](ml-frequent-pattern-mining.html#fp-growth) algorithm for frequent pattern mining + ([SPARK-14503](https://issues.apache.org/jira/browse/SPARK-14503)) +* `GLM` now supports the full `Tweedie` family + ([SPARK-18929](https://issues.apache.org/jira/browse/SPARK-18929)) +* [`Imputer`](ml-features.html#imputer) feature transformer to impute missing values in a dataset + ([SPARK-13568](https://issues.apache.org/jira/browse/SPARK-13568)) +* [`LinearSVC`](ml-classification-regression.html#linear-support-vector-machine) + for linear Support Vector Machine classification + ([SPARK-14709](https://issues.apache.org/jira/browse/SPARK-14709)) +* Logistic regression now supports constraints on the coefficients during training + ([SPARK-20047](https://issues.apache.org/jira/browse/SPARK-20047)) -A `Transformer` is an abstraction that includes feature transformers and learned models. -Technically, a `Transformer` implements a method `transform()`, which converts one `DataFrame` into -another, generally by appending one or more columns. -For example: +# Migration guide -* A feature transformer might take a `DataFrame`, read a column (e.g., text), map it into a new - column (e.g., feature vectors), and output a new `DataFrame` with the mapped column appended. -* A learning model might take a `DataFrame`, read the column containing feature vectors, predict the - label for each feature vector, and output a new `DataFrame` with predicted labels appended as a - column. +MLlib is under active development. +The APIs marked `Experimental`/`DeveloperApi` may change in future releases, +and the migration guide below will explain all changes between releases. -### Estimators +## From 2.2 to 2.3 -An `Estimator` abstracts the concept of a learning algorithm or any algorithm that fits or trains on -data. -Technically, an `Estimator` implements a method `fit()`, which accepts a `DataFrame` and produces a -`Model`, which is a `Transformer`. -For example, a learning algorithm such as `LogisticRegression` is an `Estimator`, and calling -`fit()` trains a `LogisticRegressionModel`, which is a `Model` and hence a `Transformer`. +### Breaking changes -### Properties of pipeline components +There are no breaking changes. -`Transformer.transform()`s and `Estimator.fit()`s are both stateless. In the future, stateful algorithms may be supported via alternative concepts. +### Deprecations and changes of behavior -Each instance of a `Transformer` or `Estimator` has a unique ID, which is useful in specifying parameters (discussed below). +**Deprecations** -## Pipeline +There are no deprecations. -In machine learning, it is common to run a sequence of algorithms to process and learn from data. -E.g., a simple text document processing workflow might include several stages: +**Changes of behavior** -* Split each document's text into words. -* Convert each document's words into a numerical feature vector. -* Learn a prediction model using the feature vectors and labels. +* [SPARK-21027](https://issues.apache.org/jira/browse/SPARK-21027): + We are now setting the default parallelism used in `OneVsRest` to be 1 (i.e. serial), in 2.2 and earlier version, + the `OneVsRest` parallelism would be parallelism of the default threadpool in scala. -Spark ML represents such a workflow as a `Pipeline`, which consists of a sequence of -`PipelineStage`s (`Transformer`s and `Estimator`s) to be run in a specific order. -We will use this simple workflow as a running example in this section. +## From 2.1 to 2.2 -### How it works +### Breaking changes -A `Pipeline` is specified as a sequence of stages, and each stage is either a `Transformer` or an `Estimator`. -These stages are run in order, and the input `DataFrame` is transformed as it passes through each stage. -For `Transformer` stages, the `transform()` method is called on the `DataFrame`. -For `Estimator` stages, the `fit()` method is called to produce a `Transformer` (which becomes part of the `PipelineModel`, or fitted `Pipeline`), and that `Transformer`'s `transform()` method is called on the `DataFrame`. +There are no breaking changes. -We illustrate this for the simple text document workflow. The figure below is for the *training time* usage of a `Pipeline`. +### Deprecations and changes of behavior -

    - Spark ML Pipeline Example -

    +**Deprecations** -Above, the top row represents a `Pipeline` with three stages. -The first two (`Tokenizer` and `HashingTF`) are `Transformer`s (blue), and the third (`LogisticRegression`) is an `Estimator` (red). -The bottom row represents data flowing through the pipeline, where cylinders indicate `DataFrame`s. -The `Pipeline.fit()` method is called on the original `DataFrame`, which has raw text documents and labels. -The `Tokenizer.transform()` method splits the raw text documents into words, adding a new column with words to the `DataFrame`. -The `HashingTF.transform()` method converts the words column into feature vectors, adding a new column with those vectors to the `DataFrame`. -Now, since `LogisticRegression` is an `Estimator`, the `Pipeline` first calls `LogisticRegression.fit()` to produce a `LogisticRegressionModel`. -If the `Pipeline` had more stages, it would call the `LogisticRegressionModel`'s `transform()` -method on the `DataFrame` before passing the `DataFrame` to the next stage. +There are no deprecations. -A `Pipeline` is an `Estimator`. -Thus, after a `Pipeline`'s `fit()` method runs, it produces a `PipelineModel`, which is a -`Transformer`. -This `PipelineModel` is used at *test time*; the figure below illustrates this usage. +**Changes of behavior** -

    - Spark ML PipelineModel Example -

    +* [SPARK-19787](https://issues.apache.org/jira/browse/SPARK-19787): + Default value of `regParam` changed from `1.0` to `0.1` for `ALS.train` method (marked `DeveloperApi`). + **Note** this does _not affect_ the `ALS` Estimator or Model, nor MLlib's `ALS` class. +* [SPARK-14772](https://issues.apache.org/jira/browse/SPARK-14772): + Fixed inconsistency between Python and Scala APIs for `Param.copy` method. +* [SPARK-11569](https://issues.apache.org/jira/browse/SPARK-11569): + `StringIndexer` now handles `NULL` values in the same way as unseen values. Previously an exception + would always be thrown regardless of the setting of the `handleInvalid` parameter. + +## Previous Spark versions -In the figure above, the `PipelineModel` has the same number of stages as the original `Pipeline`, but all `Estimator`s in the original `Pipeline` have become `Transformer`s. -When the `PipelineModel`'s `transform()` method is called on a test dataset, the data are passed -through the fitted pipeline in order. -Each stage's `transform()` method updates the dataset and passes it to the next stage. +Earlier migration guides are archived [on this page](ml-migration-guides.html). -`Pipeline`s and `PipelineModel`s help to ensure that training and test data go through identical feature processing steps. - -### Details - -*DAG `Pipeline`s*: A `Pipeline`'s stages are specified as an ordered array. The examples given here are all for linear `Pipeline`s, i.e., `Pipeline`s in which each stage uses data produced by the previous stage. It is possible to create non-linear `Pipeline`s as long as the data flow graph forms a Directed Acyclic Graph (DAG). This graph is currently specified implicitly based on the input and output column names of each stage (generally specified as parameters). If the `Pipeline` forms a DAG, then the stages must be specified in topological order. - -*Runtime checking*: Since `Pipeline`s can operate on `DataFrame`s with varied types, they cannot use -compile-time type checking. -`Pipeline`s and `PipelineModel`s instead do runtime checking before actually running the `Pipeline`. -This type checking is done using the `DataFrame` *schema*, a description of the data types of columns in the `DataFrame`. - -*Unique Pipeline stages*: A `Pipeline`'s stages should be unique instances. E.g., the same instance -`myHashingTF` should not be inserted into the `Pipeline` twice since `Pipeline` stages must have -unique IDs. However, different instances `myHashingTF1` and `myHashingTF2` (both of type `HashingTF`) -can be put into the same `Pipeline` since different instances will be created with different IDs. - -## Parameters - -Spark ML `Estimator`s and `Transformer`s use a uniform API for specifying parameters. - -A `Param` is a named parameter with self-contained documentation. -A `ParamMap` is a set of (parameter, value) pairs. - -There are two main ways to pass parameters to an algorithm: - -1. Set parameters for an instance. E.g., if `lr` is an instance of `LogisticRegression`, one could - call `lr.setMaxIter(10)` to make `lr.fit()` use at most 10 iterations. - This API resembles the API used in `spark.mllib` package. -2. Pass a `ParamMap` to `fit()` or `transform()`. Any parameters in the `ParamMap` will override parameters previously specified via setter methods. - -Parameters belong to specific instances of `Estimator`s and `Transformer`s. -For example, if we have two `LogisticRegression` instances `lr1` and `lr2`, then we can build a `ParamMap` with both `maxIter` parameters specified: `ParamMap(lr1.maxIter -> 10, lr2.maxIter -> 20)`. -This is useful if there are two algorithms with the `maxIter` parameter in a `Pipeline`. - -# Code examples - -This section gives code examples illustrating the functionality discussed above. -For more info, please refer to the API documentation -([Scala](api/scala/index.html#org.apache.spark.ml.package), -[Java](api/java/org/apache/spark/ml/package-summary.html), -and [Python](api/python/pyspark.ml.html)). -Some Spark ML algorithms are wrappers for `spark.mllib` algorithms, and the -[MLlib programming guide](mllib-guide.html) has details on specific algorithms. - -## Example: Estimator, Transformer, and Param - -This example covers the concepts of `Estimator`, `Transformer`, and `Param`. - -
    - -
    -{% highlight scala %} -import org.apache.spark.ml.classification.LogisticRegression -import org.apache.spark.ml.param.ParamMap -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.sql.Row - -// Prepare training data from a list of (label, features) tuples. -val training = sqlContext.createDataFrame(Seq( - (1.0, Vectors.dense(0.0, 1.1, 0.1)), - (0.0, Vectors.dense(2.0, 1.0, -1.0)), - (0.0, Vectors.dense(2.0, 1.3, 1.0)), - (1.0, Vectors.dense(0.0, 1.2, -0.5)) -)).toDF("label", "features") - -// Create a LogisticRegression instance. This instance is an Estimator. -val lr = new LogisticRegression() -// Print out the parameters, documentation, and any default values. -println("LogisticRegression parameters:\n" + lr.explainParams() + "\n") - -// We may set parameters using setter methods. -lr.setMaxIter(10) - .setRegParam(0.01) - -// Learn a LogisticRegression model. This uses the parameters stored in lr. -val model1 = lr.fit(training) -// 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 -// LogisticRegression instance. -println("Model 1 was fit using parameters: " + model1.parent.extractParamMap) - -// We may alternatively specify parameters using a ParamMap, -// which supports several methods for specifying parameters. -val paramMap = ParamMap(lr.maxIter -> 20) - .put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. - .put(lr.regParam -> 0.1, lr.threshold -> 0.55) // Specify multiple Params. - -// One can also combine ParamMaps. -val paramMap2 = ParamMap(lr.probabilityCol -> "myProbability") // Change output column name -val paramMapCombined = paramMap ++ paramMap2 - -// Now learn a new model using the paramMapCombined parameters. -// paramMapCombined overrides all parameters set earlier via lr.set* methods. -val model2 = lr.fit(training, paramMapCombined) -println("Model 2 was fit using parameters: " + model2.parent.extractParamMap) - -// Prepare test data. -val test = sqlContext.createDataFrame(Seq( - (1.0, Vectors.dense(-1.0, 1.5, 1.3)), - (0.0, Vectors.dense(3.0, 2.0, -0.1)), - (1.0, Vectors.dense(0.0, 2.2, -1.5)) -)).toDF("label", "features") - -// Make predictions on test data using the Transformer.transform() method. -// 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) - .select("features", "label", "myProbability", "prediction") - .collect() - .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) => - println(s"($features, $label) -> prob=$prob, prediction=$prediction") - } - -{% endhighlight %} -
    - -
    -{% highlight java %} -import java.util.Arrays; -import java.util.List; - -import org.apache.spark.ml.classification.LogisticRegressionModel; -import org.apache.spark.ml.param.ParamMap; -import org.apache.spark.ml.classification.LogisticRegression; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; - -// Prepare training data. -// We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans -// into DataFrames, where it uses the bean metadata to infer the schema. -DataFrame training = sqlContext.createDataFrame(Arrays.asList( - new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), - new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), - new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), - new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)) -), LabeledPoint.class); - -// Create a LogisticRegression instance. This instance is an Estimator. -LogisticRegression lr = new LogisticRegression(); -// Print out the parameters, documentation, and any default values. -System.out.println("LogisticRegression parameters:\n" + lr.explainParams() + "\n"); - -// We may set parameters using setter methods. -lr.setMaxIter(10) - .setRegParam(0.01); - -// Learn a LogisticRegression model. This uses the parameters stored in lr. -LogisticRegressionModel model1 = lr.fit(training); -// 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 -// LogisticRegression instance. -System.out.println("Model 1 was fit using parameters: " + model1.parent().extractParamMap()); - -// We may alternatively specify parameters using a ParamMap. -ParamMap paramMap = new ParamMap() - .put(lr.maxIter().w(20)) // Specify 1 Param. - .put(lr.maxIter(), 30) // This overwrites the original maxIter. - .put(lr.regParam().w(0.1), lr.threshold().w(0.55)); // Specify multiple Params. - -// One can also combine ParamMaps. -ParamMap paramMap2 = new ParamMap() - .put(lr.probabilityCol().w("myProbability")); // Change output column name -ParamMap paramMapCombined = paramMap.$plus$plus(paramMap2); - -// Now learn a new model using the paramMapCombined parameters. -// paramMapCombined overrides all parameters set earlier via lr.set* methods. -LogisticRegressionModel model2 = lr.fit(training, paramMapCombined); -System.out.println("Model 2 was fit using parameters: " + model2.parent().extractParamMap()); - -// Prepare test documents. -DataFrame test = sqlContext.createDataFrame(Arrays.asList( - 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)) -), LabeledPoint.class); - -// Make predictions on test documents using the Transformer.transform() method. -// LogisticRegression.transform will only use the 'features' column. -// Note that model2.transform() outputs a 'myProbability' column instead of the usual -// 'probability' column since we renamed the lr.probabilityCol parameter previously. -DataFrame results = model2.transform(test); -for (Row r: results.select("features", "label", "myProbability", "prediction").collect()) { - System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) - + ", prediction=" + r.get(3)); -} - -{% endhighlight %} -
    - -
    -{% highlight python %} -from pyspark.mllib.linalg import Vectors -from pyspark.ml.classification import LogisticRegression -from pyspark.ml.param import Param, Params - -# Prepare training data from a list of (label, features) tuples. -training = sqlContext.createDataFrame([ - (1.0, Vectors.dense([0.0, 1.1, 0.1])), - (0.0, Vectors.dense([2.0, 1.0, -1.0])), - (0.0, Vectors.dense([2.0, 1.3, 1.0])), - (1.0, Vectors.dense([0.0, 1.2, -0.5]))], ["label", "features"]) - -# Create a LogisticRegression instance. This instance is an Estimator. -lr = LogisticRegression(maxIter=10, regParam=0.01) -# Print out the parameters, documentation, and any default values. -print "LogisticRegression parameters:\n" + lr.explainParams() + "\n" - -# Learn a LogisticRegression model. This uses the parameters stored in lr. -model1 = lr.fit(training) - -# 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 -# LogisticRegression instance. -print "Model 1 was fit using parameters: " -print model1.extractParamMap() - -# We may alternatively specify parameters using a Python dictionary as a paramMap -paramMap = {lr.maxIter: 20} -paramMap[lr.maxIter] = 30 # Specify 1 Param, overwriting the original maxIter. -paramMap.update({lr.regParam: 0.1, lr.threshold: 0.55}) # Specify multiple Params. - -# You can combine paramMaps, which are python dictionaries. -paramMap2 = {lr.probabilityCol: "myProbability"} # Change output column name -paramMapCombined = paramMap.copy() -paramMapCombined.update(paramMap2) - -# Now learn a new model using the paramMapCombined parameters. -# paramMapCombined overrides all parameters set earlier via lr.set* methods. -model2 = lr.fit(training, paramMapCombined) -print "Model 2 was fit using parameters: " -print model2.extractParamMap() - -# Prepare test data -test = sqlContext.createDataFrame([ - (1.0, Vectors.dense([-1.0, 1.5, 1.3])), - (0.0, Vectors.dense([3.0, 2.0, -0.1])), - (1.0, Vectors.dense([0.0, 2.2, -1.5]))], ["label", "features"]) - -# Make predictions on test data using the Transformer.transform() method. -# 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. -prediction = model2.transform(test) -selected = prediction.select("features", "label", "myProbability", "prediction") -for row in selected.collect(): - print row - -{% endhighlight %} -
    - -
    - -## Example: Pipeline - -This example follows the simple text document `Pipeline` illustrated in the figures above. - -
    - -
    -{% highlight scala %} -import org.apache.spark.ml.Pipeline -import org.apache.spark.ml.classification.LogisticRegression -import org.apache.spark.ml.feature.{HashingTF, Tokenizer} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.sql.Row - -// Prepare training documents from a list of (id, text, label) tuples. -val training = sqlContext.createDataFrame(Seq( - (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) -)).toDF("id", "text", "label") - -// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. -val tokenizer = new Tokenizer() - .setInputCol("text") - .setOutputCol("words") -val hashingTF = new HashingTF() - .setNumFeatures(1000) - .setInputCol(tokenizer.getOutputCol) - .setOutputCol("features") -val lr = new LogisticRegression() - .setMaxIter(10) - .setRegParam(0.01) -val pipeline = new Pipeline() - .setStages(Array(tokenizer, hashingTF, lr)) - -// Fit the pipeline to training documents. -val model = pipeline.fit(training) - -// Prepare test documents, which are unlabeled (id, text) tuples. -val test = sqlContext.createDataFrame(Seq( - (4L, "spark i j k"), - (5L, "l m n"), - (6L, "mapreduce spark"), - (7L, "apache hadoop") -)).toDF("id", "text") - -// Make predictions on test documents. -model.transform(test) - .select("id", "text", "probability", "prediction") - .collect() - .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => - println(s"($id, $text) --> prob=$prob, prediction=$prediction") - } - -{% endhighlight %} -
    - -
    -{% highlight java %} -import java.util.Arrays; -import java.util.List; - -import org.apache.spark.ml.Pipeline; -import org.apache.spark.ml.PipelineModel; -import org.apache.spark.ml.PipelineStage; -import org.apache.spark.ml.classification.LogisticRegression; -import org.apache.spark.ml.feature.HashingTF; -import org.apache.spark.ml.feature.Tokenizer; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; - -// Labeled and unlabeled instance types. -// Spark SQL can infer schema from Java Beans. -public class Document implements Serializable { - private long id; - private String text; - - public Document(long id, String text) { - this.id = id; - this.text = text; - } - - public long getId() { return this.id; } - public void setId(long id) { this.id = id; } - - public String getText() { return this.text; } - public void setText(String text) { this.text = text; } -} - -public class LabeledDocument extends Document implements Serializable { - private double label; - - public LabeledDocument(long id, String text, double label) { - super(id, text); - this.label = label; - } - - public double getLabel() { return this.label; } - public void setLabel(double label) { this.label = label; } -} - -// Prepare training documents, which are labeled. -DataFrame training = sqlContext.createDataFrame(Arrays.asList( - new LabeledDocument(0L, "a b c d e spark", 1.0), - new LabeledDocument(1L, "b d", 0.0), - new LabeledDocument(2L, "spark f g h", 1.0), - new LabeledDocument(3L, "hadoop mapreduce", 0.0) -), LabeledDocument.class); - -// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. -Tokenizer tokenizer = new Tokenizer() - .setInputCol("text") - .setOutputCol("words"); -HashingTF hashingTF = new HashingTF() - .setNumFeatures(1000) - .setInputCol(tokenizer.getOutputCol()) - .setOutputCol("features"); -LogisticRegression lr = new LogisticRegression() - .setMaxIter(10) - .setRegParam(0.01); -Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[] {tokenizer, hashingTF, lr}); - -// Fit the pipeline to training documents. -PipelineModel model = pipeline.fit(training); - -// Prepare test documents, which are unlabeled. -DataFrame test = sqlContext.createDataFrame(Arrays.asList( - new Document(4L, "spark i j k"), - new Document(5L, "l m n"), - new Document(6L, "mapreduce spark"), - new Document(7L, "apache hadoop") -), Document.class); - -// Make predictions on test documents. -DataFrame predictions = model.transform(test); -for (Row r: predictions.select("id", "text", "probability", "prediction").collect()) { - System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) - + ", prediction=" + r.get(3)); -} - -{% endhighlight %} -
    - -
    -{% highlight python %} -from pyspark.ml import Pipeline -from pyspark.ml.classification import LogisticRegression -from pyspark.ml.feature import HashingTF, Tokenizer -from pyspark.sql import Row - -# Prepare training documents from a list of (id, text, label) tuples. -LabeledDocument = Row("id", "text", "label") -training = sqlContext.createDataFrame([ - (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)], ["id", "text", "label"]) - -# Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. -tokenizer = Tokenizer(inputCol="text", outputCol="words") -hashingTF = HashingTF(inputCol=tokenizer.getOutputCol(), outputCol="features") -lr = LogisticRegression(maxIter=10, regParam=0.01) -pipeline = Pipeline(stages=[tokenizer, hashingTF, lr]) - -# Fit the pipeline to training documents. -model = pipeline.fit(training) - -# Prepare test documents, which are unlabeled (id, text) tuples. -test = sqlContext.createDataFrame([ - (4L, "spark i j k"), - (5L, "l m n"), - (6L, "mapreduce spark"), - (7L, "apache hadoop")], ["id", "text"]) - -# Make predictions on test documents and print columns of interest. -prediction = model.transform(test) -selected = prediction.select("id", "text", "prediction") -for row in selected.collect(): - print(row) - -{% endhighlight %} -
    - -
    - -## Example: model selection via cross-validation - -An important task in ML is *model selection*, or using data to find the best model or parameters for a given task. This is also called *tuning*. -`Pipeline`s facilitate model selection by making it easy to tune an entire `Pipeline` at once, rather than tuning each element in the `Pipeline` separately. - -Currently, `spark.ml` supports model selection using the [`CrossValidator`](api/scala/index.html#org.apache.spark.ml.tuning.CrossValidator) class, which takes an `Estimator`, a set of `ParamMap`s, and an [`Evaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.Evaluator). -`CrossValidator` begins by splitting the dataset into a set of *folds* which are used as separate training and test datasets; e.g., with `$k=3$` folds, `CrossValidator` will generate 3 (training, test) dataset pairs, each of which uses 2/3 of the data for training and 1/3 for testing. -`CrossValidator` iterates through the set of `ParamMap`s. For each `ParamMap`, it trains the given `Estimator` and evaluates it using the given `Evaluator`. - -The `Evaluator` can be a [`RegressionEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.RegressionEvaluator) -for regression problems, a [`BinaryClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.BinaryClassificationEvaluator) -for binary data, or a [`MultiClassClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.MultiClassClassificationEvaluator) -for multiclass problems. The default metric used to choose the best `ParamMap` can be overriden by the `setMetric` -method in each of these evaluators. - -The `ParamMap` which produces the best evaluation metric (averaged over the `$k$` folds) is selected as the best model. -`CrossValidator` finally fits the `Estimator` using the best `ParamMap` and the entire dataset. - -The following example demonstrates using `CrossValidator` to select from a grid of parameters. -To help construct the parameter grid, we use the [`ParamGridBuilder`](api/scala/index.html#org.apache.spark.ml.tuning.ParamGridBuilder) utility. - -Note that cross-validation over a grid of parameters is expensive. -E.g., in the example below, the parameter grid has 3 values for `hashingTF.numFeatures` and 2 values for `lr.regParam`, and `CrossValidator` uses 2 folds. This multiplies out to `$(3 \times 2) \times 2 = 12$` different models being trained. -In realistic settings, it can be common to try many more parameters and use more folds (`$k=3$` and `$k=10$` are common). -In other words, using `CrossValidator` can be very expensive. -However, it is also a well-established method for choosing parameters which is more statistically sound than heuristic hand-tuning. - -
    - -
    -{% highlight scala %} -import org.apache.spark.ml.Pipeline -import org.apache.spark.ml.classification.LogisticRegression -import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator -import org.apache.spark.ml.feature.{HashingTF, Tokenizer} -import org.apache.spark.ml.tuning.{ParamGridBuilder, CrossValidator} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.sql.Row - -// Prepare training data from a list of (id, text, label) tuples. -val training = sqlContext.createDataFrame(Seq( - (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), - (4L, "b spark who", 1.0), - (5L, "g d a y", 0.0), - (6L, "spark fly", 1.0), - (7L, "was mapreduce", 0.0), - (8L, "e spark program", 1.0), - (9L, "a e c l", 0.0), - (10L, "spark compile", 1.0), - (11L, "hadoop software", 0.0) -)).toDF("id", "text", "label") - -// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. -val tokenizer = new Tokenizer() - .setInputCol("text") - .setOutputCol("words") -val hashingTF = new HashingTF() - .setInputCol(tokenizer.getOutputCol) - .setOutputCol("features") -val lr = new LogisticRegression() - .setMaxIter(10) -val pipeline = new Pipeline() - .setStages(Array(tokenizer, hashingTF, lr)) - -// We use a ParamGridBuilder to construct a grid of parameters to search over. -// With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, -// this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. -val paramGrid = new ParamGridBuilder() - .addGrid(hashingTF.numFeatures, Array(10, 100, 1000)) - .addGrid(lr.regParam, Array(0.1, 0.01)) - .build() - -// We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. -// This will allow us to jointly choose parameters for all Pipeline stages. -// A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. -// Note that the evaluator here is a BinaryClassificationEvaluator and its default metric -// is areaUnderROC. -val cv = new CrossValidator() - .setEstimator(pipeline) - .setEvaluator(new BinaryClassificationEvaluator) - .setEstimatorParamMaps(paramGrid) - .setNumFolds(2) // Use 3+ in practice - -// Run cross-validation, and choose the best set of parameters. -val cvModel = cv.fit(training) - -// Prepare test documents, which are unlabeled (id, text) tuples. -val test = sqlContext.createDataFrame(Seq( - (4L, "spark i j k"), - (5L, "l m n"), - (6L, "mapreduce spark"), - (7L, "apache hadoop") -)).toDF("id", "text") - -// Make predictions on test documents. cvModel uses the best model found (lrModel). -cvModel.transform(test) - .select("id", "text", "probability", "prediction") - .collect() - .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => - println(s"($id, $text) --> prob=$prob, prediction=$prediction") - } - -{% endhighlight %} -
    - -
    -{% highlight java %} -import java.util.Arrays; -import java.util.List; - -import org.apache.spark.ml.Pipeline; -import org.apache.spark.ml.PipelineStage; -import org.apache.spark.ml.classification.LogisticRegression; -import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator; -import org.apache.spark.ml.feature.HashingTF; -import org.apache.spark.ml.feature.Tokenizer; -import org.apache.spark.ml.param.ParamMap; -import org.apache.spark.ml.tuning.CrossValidator; -import org.apache.spark.ml.tuning.CrossValidatorModel; -import org.apache.spark.ml.tuning.ParamGridBuilder; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; - -// Labeled and unlabeled instance types. -// Spark SQL can infer schema from Java Beans. -public class Document implements Serializable { - private long id; - private String text; - - public Document(long id, String text) { - this.id = id; - this.text = text; - } - - public long getId() { return this.id; } - public void setId(long id) { this.id = id; } - - public String getText() { return this.text; } - public void setText(String text) { this.text = text; } -} - -public class LabeledDocument extends Document implements Serializable { - private double label; - - public LabeledDocument(long id, String text, double label) { - super(id, text); - this.label = label; - } - - public double getLabel() { return this.label; } - public void setLabel(double label) { this.label = label; } -} - - -// Prepare training documents, which are labeled. -DataFrame training = sqlContext.createDataFrame(Arrays.asList( - new LabeledDocument(0L, "a b c d e spark", 1.0), - new LabeledDocument(1L, "b d", 0.0), - new LabeledDocument(2L, "spark f g h", 1.0), - new LabeledDocument(3L, "hadoop mapreduce", 0.0), - new LabeledDocument(4L, "b spark who", 1.0), - new LabeledDocument(5L, "g d a y", 0.0), - new LabeledDocument(6L, "spark fly", 1.0), - new LabeledDocument(7L, "was mapreduce", 0.0), - new LabeledDocument(8L, "e spark program", 1.0), - new LabeledDocument(9L, "a e c l", 0.0), - new LabeledDocument(10L, "spark compile", 1.0), - new LabeledDocument(11L, "hadoop software", 0.0) -), LabeledDocument.class); - -// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. -Tokenizer tokenizer = new Tokenizer() - .setInputCol("text") - .setOutputCol("words"); -HashingTF hashingTF = new HashingTF() - .setNumFeatures(1000) - .setInputCol(tokenizer.getOutputCol()) - .setOutputCol("features"); -LogisticRegression lr = new LogisticRegression() - .setMaxIter(10) - .setRegParam(0.01); -Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[] {tokenizer, hashingTF, lr}); - -// We use a ParamGridBuilder to construct a grid of parameters to search over. -// With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, -// this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. -ParamMap[] paramGrid = new ParamGridBuilder() - .addGrid(hashingTF.numFeatures(), new int[]{10, 100, 1000}) - .addGrid(lr.regParam(), new double[]{0.1, 0.01}) - .build(); - -// We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. -// This will allow us to jointly choose parameters for all Pipeline stages. -// A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. -// Note that the evaluator here is a BinaryClassificationEvaluator and its default metric -// is areaUnderROC. -CrossValidator cv = new CrossValidator() - .setEstimator(pipeline) - .setEvaluator(new BinaryClassificationEvaluator()) - .setEstimatorParamMaps(paramGrid) - .setNumFolds(2); // Use 3+ in practice - -// Run cross-validation, and choose the best set of parameters. -CrossValidatorModel cvModel = cv.fit(training); - -// Prepare test documents, which are unlabeled. -DataFrame test = sqlContext.createDataFrame(Arrays.asList( - new Document(4L, "spark i j k"), - new Document(5L, "l m n"), - new Document(6L, "mapreduce spark"), - new Document(7L, "apache hadoop") -), Document.class); - -// Make predictions on test documents. cvModel uses the best model found (lrModel). -DataFrame predictions = cvModel.transform(test); -for (Row r: predictions.select("id", "text", "probability", "prediction").collect()) { - System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) - + ", prediction=" + r.get(3)); -} - -{% endhighlight %} -
    - -
    - -## Example: model selection via train validation split -In addition to `CrossValidator` Spark also offers `TrainValidationSplit` for hyper-parameter tuning. -`TrainValidationSplit` only evaluates each combination of parameters once as opposed to k times in - case of `CrossValidator`. It is therefore less expensive, - but will not produce as reliable results when the training dataset is not sufficiently large. - -`TrainValidationSplit` takes an `Estimator`, a set of `ParamMap`s provided in the `estimatorParamMaps` parameter, -and an `Evaluator`. -It begins by splitting the dataset into two parts using `trainRatio` parameter -which are used as separate training and test datasets. For example with `$trainRatio=0.75$` (default), -`TrainValidationSplit` will generate a training and test dataset pair where 75% of the data is used for training and 25% for validation. -Similar to `CrossValidator`, `TrainValidationSplit` also iterates through the set of `ParamMap`s. -For each combination of parameters, it trains the given `Estimator` and evaluates it using the given `Evaluator`. -The `ParamMap` which produces the best evaluation metric is selected as the best option. -`TrainValidationSplit` finally fits the `Estimator` using the best `ParamMap` and the entire dataset. - -
    - -
    -{% highlight scala %} -import org.apache.spark.ml.evaluation.RegressionEvaluator -import org.apache.spark.ml.regression.LinearRegression -import org.apache.spark.ml.tuning.{ParamGridBuilder, TrainValidationSplit} -import org.apache.spark.mllib.util.MLUtils - -// Prepare training and test data. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() -val Array(training, test) = data.randomSplit(Array(0.9, 0.1), seed = 12345) - -val lr = new LinearRegression() - -// We use a ParamGridBuilder to construct a grid of parameters to search over. -// TrainValidationSplit will try all combinations of values and determine best model using -// the evaluator. -val paramGrid = new ParamGridBuilder() - .addGrid(lr.regParam, Array(0.1, 0.01)) - .addGrid(lr.fitIntercept) - .addGrid(lr.elasticNetParam, Array(0.0, 0.5, 1.0)) - .build() - -// In this case the estimator is simply the linear regression. -// A TrainValidationSplit requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. -val trainValidationSplit = new TrainValidationSplit() - .setEstimator(lr) - .setEvaluator(new RegressionEvaluator) - .setEstimatorParamMaps(paramGrid) - // 80% of the data will be used for training and the remaining 20% for validation. - .setTrainRatio(0.8) - -// Run train validation split, and choose the best set of parameters. -val model = trainValidationSplit.fit(training) - -// Make predictions on test data. model is the model with combination of parameters -// that performed best. -model.transform(test) - .select("features", "label", "prediction") - .show() - -{% endhighlight %} -
    - -
    -{% highlight java %} -import org.apache.spark.ml.evaluation.RegressionEvaluator; -import org.apache.spark.ml.param.ParamMap; -import org.apache.spark.ml.regression.LinearRegression; -import org.apache.spark.ml.tuning.*; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.rdd.RDD; -import org.apache.spark.sql.DataFrame; - -DataFrame data = sqlContext.createDataFrame( - MLUtils.loadLibSVMFile(jsc.sc(), "data/mllib/sample_libsvm_data.txt"), - LabeledPoint.class); - -// Prepare training and test data. -DataFrame[] splits = data.randomSplit(new double[] {0.9, 0.1}, 12345); -DataFrame training = splits[0]; -DataFrame test = splits[1]; - -LinearRegression lr = new LinearRegression(); - -// We use a ParamGridBuilder to construct a grid of parameters to search over. -// TrainValidationSplit will try all combinations of values and determine best model using -// the evaluator. -ParamMap[] paramGrid = new ParamGridBuilder() - .addGrid(lr.regParam(), new double[] {0.1, 0.01}) - .addGrid(lr.fitIntercept()) - .addGrid(lr.elasticNetParam(), new double[] {0.0, 0.5, 1.0}) - .build(); - -// In this case the estimator is simply the linear regression. -// A TrainValidationSplit requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. -TrainValidationSplit trainValidationSplit = new TrainValidationSplit() - .setEstimator(lr) - .setEvaluator(new RegressionEvaluator()) - .setEstimatorParamMaps(paramGrid) - .setTrainRatio(0.8); // 80% for training and the remaining 20% for validation - -// Run train validation split, and choose the best set of parameters. -TrainValidationSplitModel model = trainValidationSplit.fit(training); - -// Make predictions on test data. model is the model with combination of parameters -// that performed best. -model.transform(test) - .select("features", "label", "prediction") - .show(); - -{% endhighlight %} -
    - -
    +--- diff --git a/docs/ml-linear-methods.md b/docs/ml-linear-methods.md index 16e2ee71293a..eb39173505ae 100644 --- a/docs/ml-linear-methods.md +++ b/docs/ml-linear-methods.md @@ -1,350 +1,8 @@ --- layout: global -title: Linear Methods - ML -displayTitle: ML - Linear Methods +title: Linear methods +displayTitle: Linear methods --- - -`\[ -\newcommand{\R}{\mathbb{R}} -\newcommand{\E}{\mathbb{E}} -\newcommand{\x}{\mathbf{x}} -\newcommand{\y}{\mathbf{y}} -\newcommand{\wv}{\mathbf{w}} -\newcommand{\av}{\mathbf{\alpha}} -\newcommand{\bv}{\mathbf{b}} -\newcommand{\N}{\mathbb{N}} -\newcommand{\id}{\mathbf{I}} -\newcommand{\ind}{\mathbf{1}} -\newcommand{\0}{\mathbf{0}} -\newcommand{\unit}{\mathbf{e}} -\newcommand{\one}{\mathbf{1}} -\newcommand{\zero}{\mathbf{0}} -\]` - - -In MLlib, we implement popular linear methods such as logistic -regression and linear least squares with $L_1$ or $L_2$ regularization. -Refer to [the linear methods in mllib](mllib-linear-methods.html) for -details. In `spark.ml`, we also include Pipelines API for [Elastic -net](http://en.wikipedia.org/wiki/Elastic_net_regularization), a hybrid -of $L_1$ and $L_2$ regularization proposed in [Zou et al, Regularization -and variable selection via the elastic -net](http://users.stat.umn.edu/~zouxx019/Papers/elasticnet.pdf). -Mathematically, it is defined as a convex combination of the $L_1$ and -the $L_2$ regularization terms: -`\[ -\alpha \left( \lambda \|\wv\|_1 \right) + (1-\alpha) \left( \frac{\lambda}{2}\|\wv\|_2^2 \right) , \alpha \in [0, 1], \lambda \geq 0 -\]` -By setting $\alpha$ properly, elastic net contains both $L_1$ and $L_2$ -regularization as special cases. For example, if a [linear -regression](https://en.wikipedia.org/wiki/Linear_regression) model is -trained with the elastic net parameter $\alpha$ set to $1$, it is -equivalent to a -[Lasso](http://en.wikipedia.org/wiki/Least_squares#Lasso_method) model. -On the other hand, if $\alpha$ is set to $0$, the trained model reduces -to a [ridge -regression](http://en.wikipedia.org/wiki/Tikhonov_regularization) model. -We implement Pipelines API for both linear regression and logistic -regression with elastic net regularization. - -## Example: Logistic Regression - -The following example shows how to train a logistic regression model -with elastic net regularization. `elasticNetParam` corresponds to -$\alpha$ and `regParam` corresponds to $\lambda$. - -
    - -
    -{% highlight scala %} -import org.apache.spark.ml.classification.LogisticRegression - -// Load training data -val training = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - -val lr = new LogisticRegression() - .setMaxIter(10) - .setRegParam(0.3) - .setElasticNetParam(0.8) - -// Fit the model -val lrModel = lr.fit(training) - -// Print the coefficients and intercept for logistic regression -println(s"Coefficients: ${lrModel.coefficients} Intercept: ${lrModel.intercept}") -{% endhighlight %} -
    - -
    -{% highlight java %} -import org.apache.spark.ml.classification.LogisticRegression; -import org.apache.spark.ml.classification.LogisticRegressionModel; -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; - -public class LogisticRegressionWithElasticNetExample { - public static void main(String[] args) { - SparkConf conf = new SparkConf() - .setAppName("Logistic Regression with Elastic Net Example"); - - SparkContext sc = new SparkContext(conf); - SQLContext sql = new SQLContext(sc); - String path = "data/mllib/sample_libsvm_data.txt"; - - // Load training data - DataFrame training = sqlContext.read.format("libsvm").load(path); - - LogisticRegression lr = new LogisticRegression() - .setMaxIter(10) - .setRegParam(0.3) - .setElasticNetParam(0.8); - - // Fit the model - LogisticRegressionModel lrModel = lr.fit(training); - - // Print the coefficients and intercept for logistic regression - System.out.println("Coefficients: " + lrModel.coefficients() + " Intercept: " + lrModel.intercept()); - } -} -{% endhighlight %} -
    - -
    -{% highlight python %} -from pyspark.ml.classification import LogisticRegression - -# Load training data -training = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - -lr = LogisticRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) - -# Fit the model -lrModel = lr.fit(training) - -# Print the coefficients and intercept for logistic regression -print("Coefficients: " + str(lrModel.coefficients)) -print("Intercept: " + str(lrModel.intercept)) -{% endhighlight %} -
    - -
    - -The `spark.ml` implementation of logistic regression also supports -extracting a summary of the model over the training set. Note that the -predictions and metrics which are stored as `Dataframe` in -`BinaryLogisticRegressionSummary` are annotated `@transient` and hence -only available on the driver. - -
    - -
    - -[`LogisticRegressionTrainingSummary`](api/scala/index.html#org.apache.spark.ml.classification.LogisticRegressionTrainingSummary) -provides a summary for a -[`LogisticRegressionModel`](api/scala/index.html#org.apache.spark.ml.classification.LogisticRegressionModel). -Currently, only binary classification is supported and the -summary must be explicitly cast to -[`BinaryLogisticRegressionTrainingSummary`](api/scala/index.html#org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary). -This will likely change when multiclass classification is supported. - -Continuing the earlier example: - -{% highlight scala %} -import org.apache.spark.ml.classification.BinaryLogisticRegressionSummary - -// Extract the summary from the returned LogisticRegressionModel instance trained in the earlier example -val trainingSummary = lrModel.summary - -// Obtain the objective per iteration. -val objectiveHistory = trainingSummary.objectiveHistory -objectiveHistory.foreach(loss => println(loss)) - -// Obtain the metrics useful to judge performance on test data. -// We cast the summary to a BinaryLogisticRegressionSummary since the problem is a -// binary classification problem. -val binarySummary = trainingSummary.asInstanceOf[BinaryLogisticRegressionSummary] - -// Obtain the receiver-operating characteristic as a dataframe and areaUnderROC. -val roc = binarySummary.roc -roc.show() -println(binarySummary.areaUnderROC) - -// Set the model threshold to maximize F-Measure -val fMeasure = binarySummary.fMeasureByThreshold -val maxFMeasure = fMeasure.select(max("F-Measure")).head().getDouble(0) -val bestThreshold = fMeasure.where($"F-Measure" === maxFMeasure). - select("threshold").head().getDouble(0) -lrModel.setThreshold(bestThreshold) -{% endhighlight %} -
    - -
    -[`LogisticRegressionTrainingSummary`](api/java/org/apache/spark/ml/classification/LogisticRegressionTrainingSummary.html) -provides a summary for a -[`LogisticRegressionModel`](api/java/org/apache/spark/ml/classification/LogisticRegressionModel.html). -Currently, only binary classification is supported and the -summary must be explicitly cast to -[`BinaryLogisticRegressionTrainingSummary`](api/java/org/apache/spark/ml/classification/BinaryLogisticRegressionTrainingSummary.html). -This will likely change when multiclass classification is supported. - -Continuing the earlier example: - -{% highlight java %} -import org.apache.spark.ml.classification.LogisticRegressionTrainingSummary; -import org.apache.spark.ml.classification.BinaryLogisticRegressionSummary; -import org.apache.spark.sql.functions; - -// Extract the summary from the returned LogisticRegressionModel instance trained in the earlier example -LogisticRegressionTrainingSummary trainingSummary = lrModel.summary(); - -// Obtain the loss per iteration. -double[] objectiveHistory = trainingSummary.objectiveHistory(); -for (double lossPerIteration : objectiveHistory) { - System.out.println(lossPerIteration); -} - -// Obtain the metrics useful to judge performance on test data. -// We cast the summary to a BinaryLogisticRegressionSummary since the problem is a -// binary classification problem. -BinaryLogisticRegressionSummary binarySummary = (BinaryLogisticRegressionSummary) trainingSummary; - -// Obtain the receiver-operating characteristic as a dataframe and areaUnderROC. -DataFrame roc = binarySummary.roc(); -roc.show(); -roc.select("FPR").show(); -System.out.println(binarySummary.areaUnderROC()); - -// Get the threshold corresponding to the maximum F-Measure and rerun LogisticRegression with -// this selected threshold. -DataFrame fMeasure = binarySummary.fMeasureByThreshold(); -double maxFMeasure = fMeasure.select(functions.max("F-Measure")).head().getDouble(0); -double bestThreshold = fMeasure.where(fMeasure.col("F-Measure").equalTo(maxFMeasure)). - select("threshold").head().getDouble(0); -lrModel.setThreshold(bestThreshold); -{% endhighlight %} -
    - - -
    -Logistic regression model summary is not yet supported in Python. -
    - -
    - -## Example: Linear Regression - -The interface for working with linear regression models and model -summaries is similar to the logistic regression case. The following -example demonstrates training an elastic net regularized linear -regression model and extracting model summary statistics. - -
    - -
    -{% highlight scala %} -import org.apache.spark.ml.regression.LinearRegression - -// Load training data -val training = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - -val lr = new LinearRegression() - .setMaxIter(10) - .setRegParam(0.3) - .setElasticNetParam(0.8) - -// Fit the model -val lrModel = lr.fit(training) - -// Print the coefficients and intercept for linear regression -println(s"Coefficients: ${lrModel.coefficients} Intercept: ${lrModel.intercept}") - -// Summarize the model over the training set and print out some metrics -val trainingSummary = lrModel.summary -println(s"numIterations: ${trainingSummary.totalIterations}") -println(s"objectiveHistory: ${trainingSummary.objectiveHistory.toList}") -trainingSummary.residuals.show() -println(s"RMSE: ${trainingSummary.rootMeanSquaredError}") -println(s"r2: ${trainingSummary.r2}") -{% endhighlight %} -
    - -
    -{% highlight java %} -import org.apache.spark.ml.regression.LinearRegression; -import org.apache.spark.ml.regression.LinearRegressionModel; -import org.apache.spark.ml.regression.LinearRegressionTrainingSummary; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; - -public class LinearRegressionWithElasticNetExample { - public static void main(String[] args) { - SparkConf conf = new SparkConf() - .setAppName("Linear Regression with Elastic Net Example"); - - SparkContext sc = new SparkContext(conf); - SQLContext sql = new SQLContext(sc); - String path = "data/mllib/sample_libsvm_data.txt"; - - // Load training data - DataFrame training = sqlContext.read.format("libsvm").load(path); - - LinearRegression lr = new LinearRegression() - .setMaxIter(10) - .setRegParam(0.3) - .setElasticNetParam(0.8); - - // Fit the model - LinearRegressionModel lrModel = lr.fit(training); - - // Print the coefficients and intercept for linear regression - System.out.println("Coefficients: " + lrModel.coefficients() + " Intercept: " + lrModel.intercept()); - - // Summarize the model over the training set and print out some metrics - LinearRegressionTrainingSummary trainingSummary = lrModel.summary(); - System.out.println("numIterations: " + trainingSummary.totalIterations()); - System.out.println("objectiveHistory: " + Vectors.dense(trainingSummary.objectiveHistory())); - trainingSummary.residuals().show(); - System.out.println("RMSE: " + trainingSummary.rootMeanSquaredError()); - System.out.println("r2: " + trainingSummary.r2()); - } -} -{% endhighlight %} -
    - -
    - -{% highlight python %} -from pyspark.ml.regression import LinearRegression - -# Load training data -training = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - -lr = LinearRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) - -# Fit the model -lrModel = lr.fit(training) - -# Print the coefficients and intercept for linear regression -print("Coefficients: " + str(lrModel.coefficients)) -print("Intercept: " + str(lrModel.intercept)) - -# Linear regression model summary is not yet supported in Python. -{% endhighlight %} -
    - -
    - -# Optimization - -The optimization algorithm underlying the implementation is called -[Orthant-Wise Limited-memory -QuasiNewton](http://research-srv.microsoft.com/en-us/um/people/jfgao/paper/icml07scalable.pdf) -(OWL-QN). It is an extension of L-BFGS that can effectively handle L1 -regularization and elastic net. - + > This section has been moved into the + [classification and regression section](ml-classification-regression.html). diff --git a/docs/ml-migration-guides.md b/docs/ml-migration-guides.md new file mode 100644 index 000000000000..687d7c893036 --- /dev/null +++ b/docs/ml-migration-guides.md @@ -0,0 +1,335 @@ +--- +layout: global +title: Old Migration Guides - MLlib +displayTitle: Old Migration Guides - MLlib +description: MLlib migration guides from before Spark SPARK_VERSION_SHORT +--- + +The migration guide for the current Spark version is kept on the [MLlib Guide main page](ml-guide.html#migration-guide). + +## From 2.0 to 2.1 + +### Breaking changes + +**Deprecated methods removed** + +* `setLabelCol` in `feature.ChiSqSelectorModel` +* `numTrees` in `classification.RandomForestClassificationModel` (This now refers to the Param called `numTrees`) +* `numTrees` in `regression.RandomForestRegressionModel` (This now refers to the Param called `numTrees`) +* `model` in `regression.LinearRegressionSummary` +* `validateParams` in `PipelineStage` +* `validateParams` in `Evaluator` + +### Deprecations and changes of behavior + +**Deprecations** + +* [SPARK-18592](https://issues.apache.org/jira/browse/SPARK-18592): + Deprecate all Param setter methods except for input/output column Params for `DecisionTreeClassificationModel`, `GBTClassificationModel`, `RandomForestClassificationModel`, `DecisionTreeRegressionModel`, `GBTRegressionModel` and `RandomForestRegressionModel` + +**Changes of behavior** + +* [SPARK-17870](https://issues.apache.org/jira/browse/SPARK-17870): + Fix a bug of `ChiSqSelector` which will likely change its result. Now `ChiSquareSelector` use pValue rather than raw statistic to select a fixed number of top features. +* [SPARK-3261](https://issues.apache.org/jira/browse/SPARK-3261): + `KMeans` returns potentially fewer than k cluster centers in cases where k distinct centroids aren't available or aren't selected. +* [SPARK-17389](https://issues.apache.org/jira/browse/SPARK-17389): + `KMeans` reduces the default number of steps from 5 to 2 for the k-means|| initialization mode. + +## From 1.6 to 2.0 + +### Breaking changes + +There were several breaking changes in Spark 2.0, which are outlined below. + +**Linear algebra classes for DataFrame-based APIs** + +Spark's linear algebra dependencies were moved to a new project, `mllib-local` +(see [SPARK-13944](https://issues.apache.org/jira/browse/SPARK-13944)). +As part of this change, the linear algebra classes were copied to a new package, `spark.ml.linalg`. +The DataFrame-based APIs in `spark.ml` now depend on the `spark.ml.linalg` classes, +leading to a few breaking changes, predominantly in various model classes +(see [SPARK-14810](https://issues.apache.org/jira/browse/SPARK-14810) for a full list). + +**Note:** the RDD-based APIs in `spark.mllib` continue to depend on the previous package `spark.mllib.linalg`. + +_Converting vectors and matrices_ + +While most pipeline components support backward compatibility for loading, +some existing `DataFrames` and pipelines in Spark versions prior to 2.0, that contain vector or matrix +columns, may need to be migrated to the new `spark.ml` vector and matrix types. +Utilities for converting `DataFrame` columns from `spark.mllib.linalg` to `spark.ml.linalg` types +(and vice versa) can be found in `spark.mllib.util.MLUtils`. + +There are also utility methods available for converting single instances of +vectors and matrices. Use the `asML` method on a `mllib.linalg.Vector` / `mllib.linalg.Matrix` +for converting to `ml.linalg` types, and +`mllib.linalg.Vectors.fromML` / `mllib.linalg.Matrices.fromML` +for converting to `mllib.linalg` types. + +
    +
    + +{% highlight scala %} +import org.apache.spark.mllib.util.MLUtils + +// convert DataFrame columns +val convertedVecDF = MLUtils.convertVectorColumnsToML(vecDF) +val convertedMatrixDF = MLUtils.convertMatrixColumnsToML(matrixDF) +// convert a single vector or matrix +val mlVec: org.apache.spark.ml.linalg.Vector = mllibVec.asML +val mlMat: org.apache.spark.ml.linalg.Matrix = mllibMat.asML +{% endhighlight %} + +Refer to the [`MLUtils` Scala docs](api/scala/index.html#org.apache.spark.mllib.util.MLUtils$) for further detail. +
    + +
    + +{% highlight java %} +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.sql.Dataset; + +// convert DataFrame columns +Dataset convertedVecDF = MLUtils.convertVectorColumnsToML(vecDF); +Dataset convertedMatrixDF = MLUtils.convertMatrixColumnsToML(matrixDF); +// convert a single vector or matrix +org.apache.spark.ml.linalg.Vector mlVec = mllibVec.asML(); +org.apache.spark.ml.linalg.Matrix mlMat = mllibMat.asML(); +{% endhighlight %} + +Refer to the [`MLUtils` Java docs](api/java/org/apache/spark/mllib/util/MLUtils.html) for further detail. +
    + +
    + +{% highlight python %} +from pyspark.mllib.util import MLUtils + +# convert DataFrame columns +convertedVecDF = MLUtils.convertVectorColumnsToML(vecDF) +convertedMatrixDF = MLUtils.convertMatrixColumnsToML(matrixDF) +# convert a single vector or matrix +mlVec = mllibVec.asML() +mlMat = mllibMat.asML() +{% endhighlight %} + +Refer to the [`MLUtils` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.util.MLUtils) for further detail. +
    +
    + +**Deprecated methods removed** + +Several deprecated methods were removed in the `spark.mllib` and `spark.ml` packages: + +* `setScoreCol` in `ml.evaluation.BinaryClassificationEvaluator` +* `weights` in `LinearRegression` and `LogisticRegression` in `spark.ml` +* `setMaxNumIterations` in `mllib.optimization.LBFGS` (marked as `DeveloperApi`) +* `treeReduce` and `treeAggregate` in `mllib.rdd.RDDFunctions` (these functions are available on `RDD`s directly, and were marked as `DeveloperApi`) +* `defaultStategy` in `mllib.tree.configuration.Strategy` +* `build` in `mllib.tree.Node` +* libsvm loaders for multiclass and load/save labeledData methods in `mllib.util.MLUtils` + +A full list of breaking changes can be found at [SPARK-14810](https://issues.apache.org/jira/browse/SPARK-14810). + +### Deprecations and changes of behavior + +**Deprecations** + +Deprecations in the `spark.mllib` and `spark.ml` packages include: + +* [SPARK-14984](https://issues.apache.org/jira/browse/SPARK-14984): + In `spark.ml.regression.LinearRegressionSummary`, the `model` field has been deprecated. +* [SPARK-13784](https://issues.apache.org/jira/browse/SPARK-13784): + In `spark.ml.regression.RandomForestRegressionModel` and `spark.ml.classification.RandomForestClassificationModel`, + the `numTrees` parameter has been deprecated in favor of `getNumTrees` method. +* [SPARK-13761](https://issues.apache.org/jira/browse/SPARK-13761): + In `spark.ml.param.Params`, the `validateParams` method has been deprecated. + We move all functionality in overridden methods to the corresponding `transformSchema`. +* [SPARK-14829](https://issues.apache.org/jira/browse/SPARK-14829): + In `spark.mllib` package, `LinearRegressionWithSGD`, `LassoWithSGD`, `RidgeRegressionWithSGD` and `LogisticRegressionWithSGD` have been deprecated. + We encourage users to use `spark.ml.regression.LinearRegresson` and `spark.ml.classification.LogisticRegresson`. +* [SPARK-14900](https://issues.apache.org/jira/browse/SPARK-14900): + In `spark.mllib.evaluation.MulticlassMetrics`, the parameters `precision`, `recall` and `fMeasure` have been deprecated in favor of `accuracy`. +* [SPARK-15644](https://issues.apache.org/jira/browse/SPARK-15644): + In `spark.ml.util.MLReader` and `spark.ml.util.MLWriter`, the `context` method has been deprecated in favor of `session`. +* In `spark.ml.feature.ChiSqSelectorModel`, the `setLabelCol` method has been deprecated since it was not used by `ChiSqSelectorModel`. + +**Changes of behavior** + +Changes of behavior in the `spark.mllib` and `spark.ml` packages include: + +* [SPARK-7780](https://issues.apache.org/jira/browse/SPARK-7780): + `spark.mllib.classification.LogisticRegressionWithLBFGS` directly calls `spark.ml.classification.LogisticRegresson` for binary classification now. + This will introduce the following behavior changes for `spark.mllib.classification.LogisticRegressionWithLBFGS`: + * The intercept will not be regularized when training binary classification model with L1/L2 Updater. + * If users set without regularization, training with or without feature scaling will return the same solution by the same convergence rate. +* [SPARK-13429](https://issues.apache.org/jira/browse/SPARK-13429): + In order to provide better and consistent result with `spark.ml.classification.LogisticRegresson`, + the default value of `spark.mllib.classification.LogisticRegressionWithLBFGS`: `convergenceTol` has been changed from 1E-4 to 1E-6. +* [SPARK-12363](https://issues.apache.org/jira/browse/SPARK-12363): + Fix a bug of `PowerIterationClustering` which will likely change its result. +* [SPARK-13048](https://issues.apache.org/jira/browse/SPARK-13048): + `LDA` using the `EM` optimizer will keep the last checkpoint by default, if checkpointing is being used. +* [SPARK-12153](https://issues.apache.org/jira/browse/SPARK-12153): + `Word2Vec` now respects sentence boundaries. Previously, it did not handle them correctly. +* [SPARK-10574](https://issues.apache.org/jira/browse/SPARK-10574): + `HashingTF` uses `MurmurHash3` as default hash algorithm in both `spark.ml` and `spark.mllib`. +* [SPARK-14768](https://issues.apache.org/jira/browse/SPARK-14768): + The `expectedType` argument for PySpark `Param` was removed. +* [SPARK-14931](https://issues.apache.org/jira/browse/SPARK-14931): + Some default `Param` values, which were mismatched between pipelines in Scala and Python, have been changed. +* [SPARK-13600](https://issues.apache.org/jira/browse/SPARK-13600): + `QuantileDiscretizer` now uses `spark.sql.DataFrameStatFunctions.approxQuantile` to find splits (previously used custom sampling logic). + The output buckets will differ for same input data and params. + +## From 1.5 to 1.6 + +There are no breaking API changes in the `spark.mllib` or `spark.ml` packages, but there are +deprecations and changes of behavior. + +Deprecations: + +* [SPARK-11358](https://issues.apache.org/jira/browse/SPARK-11358): + In `spark.mllib.clustering.KMeans`, the `runs` parameter has been deprecated. +* [SPARK-10592](https://issues.apache.org/jira/browse/SPARK-10592): + In `spark.ml.classification.LogisticRegressionModel` and + `spark.ml.regression.LinearRegressionModel`, the `weights` field has been deprecated in favor of + the new name `coefficients`. This helps disambiguate from instance (row) "weights" given to + algorithms. + +Changes of behavior: + +* [SPARK-7770](https://issues.apache.org/jira/browse/SPARK-7770): + `spark.mllib.tree.GradientBoostedTrees`: `validationTol` has changed semantics in 1.6. + Previously, it was a threshold for absolute change in error. Now, it resembles the behavior of + `GradientDescent`'s `convergenceTol`: For large errors, it uses relative error (relative to the + previous error); for small errors (`< 0.01`), it uses absolute error. +* [SPARK-11069](https://issues.apache.org/jira/browse/SPARK-11069): + `spark.ml.feature.RegexTokenizer`: Previously, it did not convert strings to lowercase before + tokenizing. Now, it converts to lowercase by default, with an option not to. This matches the + behavior of the simpler `Tokenizer` transformer. + +## From 1.4 to 1.5 + +In the `spark.mllib` package, there are no breaking API changes but several behavior changes: + +* [SPARK-9005](https://issues.apache.org/jira/browse/SPARK-9005): + `RegressionMetrics.explainedVariance` returns the average regression sum of squares. +* [SPARK-8600](https://issues.apache.org/jira/browse/SPARK-8600): `NaiveBayesModel.labels` become + sorted. +* [SPARK-3382](https://issues.apache.org/jira/browse/SPARK-3382): `GradientDescent` has a default + convergence tolerance `1e-3`, and hence iterations might end earlier than 1.4. + +In the `spark.ml` package, there exists one breaking API change and one behavior change: + +* [SPARK-9268](https://issues.apache.org/jira/browse/SPARK-9268): Java's varargs support is removed + from `Params.setDefault` due to a + [Scala compiler bug](https://issues.scala-lang.org/browse/SI-9013). +* [SPARK-10097](https://issues.apache.org/jira/browse/SPARK-10097): `Evaluator.isLargerBetter` is + added to indicate metric ordering. Metrics like RMSE no longer flip signs as in 1.4. + +## From 1.3 to 1.4 + +In the `spark.mllib` package, there were several breaking changes, but all in `DeveloperApi` or `Experimental` APIs: + +* Gradient-Boosted Trees + * *(Breaking change)* The signature of the [`Loss.gradient`](api/scala/index.html#org.apache.spark.mllib.tree.loss.Loss) method was changed. This is only an issues for users who wrote their own losses for GBTs. + * *(Breaking change)* The `apply` and `copy` methods for the case class [`BoostingStrategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.BoostingStrategy) have been changed because of a modification to the case class fields. This could be an issue for users who use `BoostingStrategy` to set GBT parameters. +* *(Breaking change)* The return value of [`LDA.run`](api/scala/index.html#org.apache.spark.mllib.clustering.LDA) has changed. It now returns an abstract class `LDAModel` instead of the concrete class `DistributedLDAModel`. The object of type `LDAModel` can still be cast to the appropriate concrete type, which depends on the optimization algorithm. + +In the `spark.ml` package, several major API changes occurred, including: + +* `Param` and other APIs for specifying parameters +* `uid` unique IDs for Pipeline components +* Reorganization of certain classes + +Since the `spark.ml` API was an alpha component in Spark 1.3, we do not list all changes here. +However, since 1.4 `spark.ml` is no longer an alpha component, we will provide details on any API +changes for future releases. + +## From 1.2 to 1.3 + +In the `spark.mllib` package, there were several breaking changes. The first change (in `ALS`) is the only one in a component not marked as Alpha or Experimental. + +* *(Breaking change)* In [`ALS`](api/scala/index.html#org.apache.spark.mllib.recommendation.ALS), the extraneous method `solveLeastSquares` has been removed. The `DeveloperApi` method `analyzeBlocks` was also removed. +* *(Breaking change)* [`StandardScalerModel`](api/scala/index.html#org.apache.spark.mllib.feature.StandardScalerModel) remains an Alpha component. In it, the `variance` method has been replaced with the `std` method. To compute the column variance values returned by the original `variance` method, simply square the standard deviation values returned by `std`. +* *(Breaking change)* [`StreamingLinearRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD) remains an Experimental component. In it, there were two changes: + * The constructor taking arguments was removed in favor of a builder pattern using the default constructor plus parameter setter methods. + * Variable `model` is no longer public. +* *(Breaking change)* [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) remains an Experimental component. In it and its associated classes, there were several changes: + * In `DecisionTree`, the deprecated class method `train` has been removed. (The object/static `train` methods remain.) + * In `Strategy`, the `checkpointDir` parameter has been removed. Checkpointing is still supported, but the checkpoint directory must be set before calling tree and tree ensemble training. +* `PythonMLlibAPI` (the interface between Scala/Java and Python for MLlib) was a public API but is now private, declared `private[python]`. This was never meant for external use. +* In linear regression (including Lasso and ridge regression), the squared loss is now divided by 2. + So in order to produce the same result as in 1.2, the regularization parameter needs to be divided by 2 and the step size needs to be multiplied by 2. + +In the `spark.ml` package, the main API changes are from Spark SQL. We list the most important changes here: + +* The old [SchemaRDD](http://spark.apache.org/docs/1.2.1/api/scala/index.html#org.apache.spark.sql.SchemaRDD) has been replaced with [DataFrame](api/scala/index.html#org.apache.spark.sql.DataFrame) with a somewhat modified API. All algorithms in `spark.ml` which used to use SchemaRDD now use DataFrame. +* In Spark 1.2, we used implicit conversions from `RDD`s of `LabeledPoint` into `SchemaRDD`s by calling `import sqlContext._` where `sqlContext` was an instance of `SQLContext`. These implicits have been moved, so we now call `import sqlContext.implicits._`. +* Java APIs for SQL have also changed accordingly. Please see the examples above and the [Spark SQL Programming Guide](sql-programming-guide.html) for details. + +Other changes were in `LogisticRegression`: + +* The `scoreCol` output column (with default value "score") was renamed to be `probabilityCol` (with default value "probability"). The type was originally `Double` (for the probability of class 1.0), but it is now `Vector` (for the probability of each class, to support multiclass classification in the future). +* In Spark 1.2, `LogisticRegressionModel` did not include an intercept. In Spark 1.3, it includes an intercept; however, it will always be 0.0 since it uses the default settings for [spark.mllib.LogisticRegressionWithLBFGS](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS). The option to use an intercept will be added in the future. + +## From 1.1 to 1.2 + +The only API changes in MLlib v1.2 are in +[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +which continues to be an experimental API in MLlib 1.2: + +1. *(Breaking change)* The Scala API for classification takes a named argument specifying the number +of classes. In MLlib v1.1, this argument was called `numClasses` in Python and +`numClassesForClassification` in Scala. In MLlib v1.2, the names are both set to `numClasses`. +This `numClasses` parameter is specified either via +[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) +or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) +static `trainClassifier` and `trainRegressor` methods. + +2. *(Breaking change)* The API for +[`Node`](api/scala/index.html#org.apache.spark.mllib.tree.model.Node) has changed. +This should generally not affect user code, unless the user manually constructs decision trees +(instead of using the `trainClassifier` or `trainRegressor` methods). +The tree `Node` now includes more information, including the probability of the predicted label +(for classification). + +3. Printing methods' output has changed. The `toString` (Scala/Java) and `__repr__` (Python) methods used to print the full model; they now print a summary. For the full model, use `toDebugString`. + +Examples in the Spark distribution and examples in the +[Decision Trees Guide](mllib-decision-tree.html#examples) have been updated accordingly. + +## From 1.0 to 1.1 + +The only API changes in MLlib v1.1 are in +[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +which continues to be an experimental API in MLlib 1.1: + +1. *(Breaking change)* The meaning of tree depth has been changed by 1 in order to match +the implementations of trees in +[scikit-learn](http://scikit-learn.org/stable/modules/classes.html#module-sklearn.tree) +and in [rpart](http://cran.r-project.org/web/packages/rpart/index.html). +In MLlib v1.0, a depth-1 tree had 1 leaf node, and a depth-2 tree had 1 root node and 2 leaf nodes. +In MLlib v1.1, a depth-0 tree has 1 leaf node, and a depth-1 tree has 1 root node and 2 leaf nodes. +This depth is specified by the `maxDepth` parameter in +[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) +or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) +static `trainClassifier` and `trainRegressor` methods. + +2. *(Non-breaking change)* We recommend using the newly added `trainClassifier` and `trainRegressor` +methods to build a [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +rather than using the old parameter class `Strategy`. These new training methods explicitly +separate classification and regression, and they replace specialized parameter types with +simple `String` types. + +Examples of the new, recommended `trainClassifier` and `trainRegressor` are given in the +[Decision Trees Guide](mllib-decision-tree.html#examples). + +## From 0.9 to 1.0 + +In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few +breaking changes. If your data is sparse, please store it in a sparse format instead of dense to +take advantage of sparsity in both storage and computation. Details are described below. + diff --git a/docs/ml-pipeline.md b/docs/ml-pipeline.md new file mode 100644 index 000000000000..aa92c0a37c0f --- /dev/null +++ b/docs/ml-pipeline.md @@ -0,0 +1,270 @@ +--- +layout: global +title: ML Pipelines +displayTitle: ML Pipelines +--- + +`\[ +\newcommand{\R}{\mathbb{R}} +\newcommand{\E}{\mathbb{E}} +\newcommand{\x}{\mathbf{x}} +\newcommand{\y}{\mathbf{y}} +\newcommand{\wv}{\mathbf{w}} +\newcommand{\av}{\mathbf{\alpha}} +\newcommand{\bv}{\mathbf{b}} +\newcommand{\N}{\mathbb{N}} +\newcommand{\id}{\mathbf{I}} +\newcommand{\ind}{\mathbf{1}} +\newcommand{\0}{\mathbf{0}} +\newcommand{\unit}{\mathbf{e}} +\newcommand{\one}{\mathbf{1}} +\newcommand{\zero}{\mathbf{0}} +\]` + +In this section, we introduce the concept of ***ML Pipelines***. +ML Pipelines provide a uniform set of high-level APIs built on top of +[DataFrames](sql-programming-guide.html) that help users create and tune practical +machine learning pipelines. + +**Table of Contents** + +* This will become a table of contents (this text will be scraped). +{:toc} + +# Main concepts in Pipelines + +MLlib standardizes APIs for machine learning algorithms to make it easier to combine multiple +algorithms into a single pipeline, or workflow. +This section covers the key concepts introduced by the Pipelines API, where the pipeline concept is +mostly inspired by the [scikit-learn](http://scikit-learn.org/) project. + +* **[`DataFrame`](ml-pipeline.html#dataframe)**: This ML API uses `DataFrame` from Spark SQL as an ML + dataset, which can hold a variety of data types. + E.g., a `DataFrame` could have different columns storing text, feature vectors, true labels, and predictions. + +* **[`Transformer`](ml-pipeline.html#transformers)**: A `Transformer` is an algorithm which can transform one `DataFrame` into another `DataFrame`. +E.g., an ML model is a `Transformer` which transforms a `DataFrame` with features into a `DataFrame` with predictions. + +* **[`Estimator`](ml-pipeline.html#estimators)**: An `Estimator` is an algorithm which can be fit on a `DataFrame` to produce a `Transformer`. +E.g., a learning algorithm is an `Estimator` which trains on a `DataFrame` and produces a model. + +* **[`Pipeline`](ml-pipeline.html#pipeline)**: A `Pipeline` chains multiple `Transformer`s and `Estimator`s together to specify an ML workflow. + +* **[`Parameter`](ml-pipeline.html#parameters)**: All `Transformer`s and `Estimator`s now share a common API for specifying parameters. + +## DataFrame + +Machine learning can be applied to a wide variety of data types, such as vectors, text, images, and structured data. +This API adopts the `DataFrame` from Spark SQL in order to support a variety of data types. + +`DataFrame` supports many basic and structured types; see the [Spark SQL datatype reference](sql-programming-guide.html#data-types) for a list of supported types. +In addition to the types listed in the Spark SQL guide, `DataFrame` can use ML [`Vector`](mllib-data-types.html#local-vector) types. + +A `DataFrame` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples. + +Columns in a `DataFrame` are named. The code examples below use names such as "text," "features," and "label." + +## Pipeline components + +### Transformers + +A `Transformer` is an abstraction that includes feature transformers and learned models. +Technically, a `Transformer` implements a method `transform()`, which converts one `DataFrame` into +another, generally by appending one or more columns. +For example: + +* A feature transformer might take a `DataFrame`, read a column (e.g., text), map it into a new + column (e.g., feature vectors), and output a new `DataFrame` with the mapped column appended. +* A learning model might take a `DataFrame`, read the column containing feature vectors, predict the + label for each feature vector, and output a new `DataFrame` with predicted labels appended as a + column. + +### Estimators + +An `Estimator` abstracts the concept of a learning algorithm or any algorithm that fits or trains on +data. +Technically, an `Estimator` implements a method `fit()`, which accepts a `DataFrame` and produces a +`Model`, which is a `Transformer`. +For example, a learning algorithm such as `LogisticRegression` is an `Estimator`, and calling +`fit()` trains a `LogisticRegressionModel`, which is a `Model` and hence a `Transformer`. + +### Properties of pipeline components + +`Transformer.transform()`s and `Estimator.fit()`s are both stateless. In the future, stateful algorithms may be supported via alternative concepts. + +Each instance of a `Transformer` or `Estimator` has a unique ID, which is useful in specifying parameters (discussed below). + +## Pipeline + +In machine learning, it is common to run a sequence of algorithms to process and learn from data. +E.g., a simple text document processing workflow might include several stages: + +* Split each document's text into words. +* Convert each document's words into a numerical feature vector. +* Learn a prediction model using the feature vectors and labels. + +MLlib represents such a workflow as a `Pipeline`, which consists of a sequence of +`PipelineStage`s (`Transformer`s and `Estimator`s) to be run in a specific order. +We will use this simple workflow as a running example in this section. + +### How it works + +A `Pipeline` is specified as a sequence of stages, and each stage is either a `Transformer` or an `Estimator`. +These stages are run in order, and the input `DataFrame` is transformed as it passes through each stage. +For `Transformer` stages, the `transform()` method is called on the `DataFrame`. +For `Estimator` stages, the `fit()` method is called to produce a `Transformer` (which becomes part of the `PipelineModel`, or fitted `Pipeline`), and that `Transformer`'s `transform()` method is called on the `DataFrame`. + +We illustrate this for the simple text document workflow. The figure below is for the *training time* usage of a `Pipeline`. + +

    + ML Pipeline Example +

    + +Above, the top row represents a `Pipeline` with three stages. +The first two (`Tokenizer` and `HashingTF`) are `Transformer`s (blue), and the third (`LogisticRegression`) is an `Estimator` (red). +The bottom row represents data flowing through the pipeline, where cylinders indicate `DataFrame`s. +The `Pipeline.fit()` method is called on the original `DataFrame`, which has raw text documents and labels. +The `Tokenizer.transform()` method splits the raw text documents into words, adding a new column with words to the `DataFrame`. +The `HashingTF.transform()` method converts the words column into feature vectors, adding a new column with those vectors to the `DataFrame`. +Now, since `LogisticRegression` is an `Estimator`, the `Pipeline` first calls `LogisticRegression.fit()` to produce a `LogisticRegressionModel`. +If the `Pipeline` had more `Estimator`s, it would call the `LogisticRegressionModel`'s `transform()` +method on the `DataFrame` before passing the `DataFrame` to the next stage. + +A `Pipeline` is an `Estimator`. +Thus, after a `Pipeline`'s `fit()` method runs, it produces a `PipelineModel`, which is a +`Transformer`. +This `PipelineModel` is used at *test time*; the figure below illustrates this usage. + +

    + ML PipelineModel Example +

    + +In the figure above, the `PipelineModel` has the same number of stages as the original `Pipeline`, but all `Estimator`s in the original `Pipeline` have become `Transformer`s. +When the `PipelineModel`'s `transform()` method is called on a test dataset, the data are passed +through the fitted pipeline in order. +Each stage's `transform()` method updates the dataset and passes it to the next stage. + +`Pipeline`s and `PipelineModel`s help to ensure that training and test data go through identical feature processing steps. + +### Details + +*DAG `Pipeline`s*: A `Pipeline`'s stages are specified as an ordered array. The examples given here are all for linear `Pipeline`s, i.e., `Pipeline`s in which each stage uses data produced by the previous stage. It is possible to create non-linear `Pipeline`s as long as the data flow graph forms a Directed Acyclic Graph (DAG). This graph is currently specified implicitly based on the input and output column names of each stage (generally specified as parameters). If the `Pipeline` forms a DAG, then the stages must be specified in topological order. + +*Runtime checking*: Since `Pipeline`s can operate on `DataFrame`s with varied types, they cannot use +compile-time type checking. +`Pipeline`s and `PipelineModel`s instead do runtime checking before actually running the `Pipeline`. +This type checking is done using the `DataFrame` *schema*, a description of the data types of columns in the `DataFrame`. + +*Unique Pipeline stages*: A `Pipeline`'s stages should be unique instances. E.g., the same instance +`myHashingTF` should not be inserted into the `Pipeline` twice since `Pipeline` stages must have +unique IDs. However, different instances `myHashingTF1` and `myHashingTF2` (both of type `HashingTF`) +can be put into the same `Pipeline` since different instances will be created with different IDs. + +## Parameters + +MLlib `Estimator`s and `Transformer`s use a uniform API for specifying parameters. + +A `Param` is a named parameter with self-contained documentation. +A `ParamMap` is a set of (parameter, value) pairs. + +There are two main ways to pass parameters to an algorithm: + +1. Set parameters for an instance. E.g., if `lr` is an instance of `LogisticRegression`, one could + call `lr.setMaxIter(10)` to make `lr.fit()` use at most 10 iterations. + This API resembles the API used in `spark.mllib` package. +2. Pass a `ParamMap` to `fit()` or `transform()`. Any parameters in the `ParamMap` will override parameters previously specified via setter methods. + +Parameters belong to specific instances of `Estimator`s and `Transformer`s. +For example, if we have two `LogisticRegression` instances `lr1` and `lr2`, then we can build a `ParamMap` with both `maxIter` parameters specified: `ParamMap(lr1.maxIter -> 10, lr2.maxIter -> 20)`. +This is useful if there are two algorithms with the `maxIter` parameter in a `Pipeline`. + +## Saving and Loading Pipelines + +Often times it is worth it to save a model or a pipeline to disk for later use. In Spark 1.6, a model import/export functionality was added to the Pipeline API. Most basic transformers are supported as well as some of the more basic ML models. Please refer to the algorithm's API documentation to see if saving and loading is supported. + +# Code examples + +This section gives code examples illustrating the functionality discussed above. +For more info, please refer to the API documentation +([Scala](api/scala/index.html#org.apache.spark.ml.package), +[Java](api/java/org/apache/spark/ml/package-summary.html), +and [Python](api/python/pyspark.ml.html)). + +## Example: Estimator, Transformer, and Param + +This example covers the concepts of `Estimator`, `Transformer`, and `Param`. + +
    + +
    + +Refer to the [`Estimator` Scala docs](api/scala/index.html#org.apache.spark.ml.Estimator), +the [`Transformer` Scala docs](api/scala/index.html#org.apache.spark.ml.Transformer) and +the [`Params` Scala docs](api/scala/index.html#org.apache.spark.ml.param.Params) for details on the API. + +{% include_example scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala %} +
    + +
    + +Refer to the [`Estimator` Java docs](api/java/org/apache/spark/ml/Estimator.html), +the [`Transformer` Java docs](api/java/org/apache/spark/ml/Transformer.html) and +the [`Params` Java docs](api/java/org/apache/spark/ml/param/Params.html) for details on the API. + +{% include_example java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java %} +
    + +
    + +Refer to the [`Estimator` Python docs](api/python/pyspark.ml.html#pyspark.ml.Estimator), +the [`Transformer` Python docs](api/python/pyspark.ml.html#pyspark.ml.Transformer) and +the [`Params` Python docs](api/python/pyspark.ml.html#pyspark.ml.param.Params) for more details on the API. + +{% include_example python/ml/estimator_transformer_param_example.py %} +
    + +
    + +## Example: Pipeline + +This example follows the simple text document `Pipeline` illustrated in the figures above. + +
    + +
    + +Refer to the [`Pipeline` Scala docs](api/scala/index.html#org.apache.spark.ml.Pipeline) for details on the API. + +{% include_example scala/org/apache/spark/examples/ml/PipelineExample.scala %} +
    + +
    + + +Refer to the [`Pipeline` Java docs](api/java/org/apache/spark/ml/Pipeline.html) for details on the API. + +{% include_example java/org/apache/spark/examples/ml/JavaPipelineExample.java %} +
    + +
    + +Refer to the [`Pipeline` Python docs](api/python/pyspark.ml.html#pyspark.ml.Pipeline) for more details on the API. + +{% include_example python/ml/pipeline_example.py %} +
    + +
    + +## Model selection (hyperparameter tuning) + +A big benefit of using ML Pipelines is hyperparameter optimization. See the [ML Tuning Guide](ml-tuning.html) for more information on automatic model selection. diff --git a/docs/ml-statistics.md b/docs/ml-statistics.md new file mode 100644 index 000000000000..abfb3cab1e56 --- /dev/null +++ b/docs/ml-statistics.md @@ -0,0 +1,92 @@ +--- +layout: global +title: Basic Statistics +displayTitle: Basic Statistics +--- + + +`\[ +\newcommand{\R}{\mathbb{R}} +\newcommand{\E}{\mathbb{E}} +\newcommand{\x}{\mathbf{x}} +\newcommand{\y}{\mathbf{y}} +\newcommand{\wv}{\mathbf{w}} +\newcommand{\av}{\mathbf{\alpha}} +\newcommand{\bv}{\mathbf{b}} +\newcommand{\N}{\mathbb{N}} +\newcommand{\id}{\mathbf{I}} +\newcommand{\ind}{\mathbf{1}} +\newcommand{\0}{\mathbf{0}} +\newcommand{\unit}{\mathbf{e}} +\newcommand{\one}{\mathbf{1}} +\newcommand{\zero}{\mathbf{0}} +\]` + +**Table of Contents** + +* This will become a table of contents (this text will be scraped). +{:toc} + +## Correlation + +Calculating the correlation between two series of data is a common operation in Statistics. In `spark.ml` +we provide the flexibility to calculate pairwise correlations among many series. The supported +correlation methods are currently Pearson's and Spearman's correlation. + +
    +
    +[`Correlation`](api/scala/index.html#org.apache.spark.ml.stat.Correlation$) +computes the correlation matrix for the input Dataset of Vectors using the specified method. +The output will be a DataFrame that contains the correlation matrix of the column of vectors. + +{% include_example scala/org/apache/spark/examples/ml/CorrelationExample.scala %} +
    + +
    +[`Correlation`](api/java/org/apache/spark/ml/stat/Correlation.html) +computes the correlation matrix for the input Dataset of Vectors using the specified method. +The output will be a DataFrame that contains the correlation matrix of the column of vectors. + +{% include_example java/org/apache/spark/examples/ml/JavaCorrelationExample.java %} +
    + +
    +[`Correlation`](api/python/pyspark.ml.html#pyspark.ml.stat.Correlation$) +computes the correlation matrix for the input Dataset of Vectors using the specified method. +The output will be a DataFrame that contains the correlation matrix of the column of vectors. + +{% include_example python/ml/correlation_example.py %} +
    + +
    + +## Hypothesis testing + +Hypothesis testing is a powerful tool in statistics to determine whether a result is statistically +significant, whether this result occurred by chance or not. `spark.ml` currently supports Pearson's +Chi-squared ( $\chi^2$) tests for independence. + +`ChiSquareTest` conducts Pearson's independence test for every feature against the label. +For each feature, the (feature, label) pairs are converted into a contingency matrix for which +the Chi-squared statistic is computed. All label and feature values must be categorical. + +
    +
    +Refer to the [`ChiSquareTest` Scala docs](api/scala/index.html#org.apache.spark.ml.stat.ChiSquareTest$) for details on the API. + +{% include_example scala/org/apache/spark/examples/ml/ChiSquareTestExample.scala %} +
    + +
    +Refer to the [`ChiSquareTest` Java docs](api/java/org/apache/spark/ml/stat/ChiSquareTest.html) for details on the API. + +{% include_example java/org/apache/spark/examples/ml/JavaChiSquareTestExample.java %} +
    + +
    +Refer to the [`ChiSquareTest` Python docs](api/python/index.html#pyspark.ml.stat.ChiSquareTest$) for details on the API. + +{% include_example python/ml/chi_square_test_example.py %} +
    + +
    \ No newline at end of file diff --git a/docs/ml-survival-regression.md b/docs/ml-survival-regression.md new file mode 100644 index 000000000000..efa3c21c7ca1 --- /dev/null +++ b/docs/ml-survival-regression.md @@ -0,0 +1,8 @@ +--- +layout: global +title: Survival Regression +displayTitle: Survival Regression +--- + + > This section has been moved into the + [classification and regression section](ml-classification-regression.html#survival-regression). diff --git a/docs/ml-tuning.md b/docs/ml-tuning.md new file mode 100644 index 000000000000..64dc46cf0c0e --- /dev/null +++ b/docs/ml-tuning.md @@ -0,0 +1,140 @@ +--- +layout: global +title: "ML Tuning" +displayTitle: "ML Tuning: model selection and hyperparameter tuning" +--- + +`\[ +\newcommand{\R}{\mathbb{R}} +\newcommand{\E}{\mathbb{E}} +\newcommand{\x}{\mathbf{x}} +\newcommand{\y}{\mathbf{y}} +\newcommand{\wv}{\mathbf{w}} +\newcommand{\av}{\mathbf{\alpha}} +\newcommand{\bv}{\mathbf{b}} +\newcommand{\N}{\mathbb{N}} +\newcommand{\id}{\mathbf{I}} +\newcommand{\ind}{\mathbf{1}} +\newcommand{\0}{\mathbf{0}} +\newcommand{\unit}{\mathbf{e}} +\newcommand{\one}{\mathbf{1}} +\newcommand{\zero}{\mathbf{0}} +\]` + +This section describes how to use MLlib's tooling for tuning ML algorithms and Pipelines. +Built-in Cross-Validation and other tooling allow users to optimize hyperparameters in algorithms and Pipelines. + +**Table of contents** + +* This will become a table of contents (this text will be scraped). +{:toc} + +# Model selection (a.k.a. hyperparameter tuning) + +An important task in ML is *model selection*, or using data to find the best model or parameters for a given task. This is also called *tuning*. +Tuning may be done for individual `Estimator`s such as `LogisticRegression`, or for entire `Pipeline`s which include multiple algorithms, featurization, and other steps. Users can tune an entire `Pipeline` at once, rather than tuning each element in the `Pipeline` separately. + +MLlib supports model selection using tools such as [`CrossValidator`](api/scala/index.html#org.apache.spark.ml.tuning.CrossValidator) and [`TrainValidationSplit`](api/scala/index.html#org.apache.spark.ml.tuning.TrainValidationSplit). +These tools require the following items: + +* [`Estimator`](api/scala/index.html#org.apache.spark.ml.Estimator): algorithm or `Pipeline` to tune +* Set of `ParamMap`s: parameters to choose from, sometimes called a "parameter grid" to search over +* [`Evaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.Evaluator): metric to measure how well a fitted `Model` does on held-out test data + +At a high level, these model selection tools work as follows: + +* They split the input data into separate training and test datasets. +* For each (training, test) pair, they iterate through the set of `ParamMap`s: + * For each `ParamMap`, they fit the `Estimator` using those parameters, get the fitted `Model`, and evaluate the `Model`'s performance using the `Evaluator`. +* They select the `Model` produced by the best-performing set of parameters. + +The `Evaluator` can be a [`RegressionEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.RegressionEvaluator) +for regression problems, a [`BinaryClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.BinaryClassificationEvaluator) +for binary data, or a [`MulticlassClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator) +for multiclass problems. The default metric used to choose the best `ParamMap` can be overridden by the `setMetricName` +method in each of these evaluators. + +To help construct the parameter grid, users can use the [`ParamGridBuilder`](api/scala/index.html#org.apache.spark.ml.tuning.ParamGridBuilder) utility. +By default, sets of parameters from the parameter grid are evaluated in serial. Parameter evaluation can be done in parallel by setting `parallelism` with a value of 2 or more (a value of 1 will be serial) before running model selection with `CrossValidator` or `TrainValidationSplit` (NOTE: this is not yet supported in Python). +The value of `parallelism` should be chosen carefully to maximize parallelism without exceeding cluster resources, and larger values may not always lead to improved performance. Generally speaking, a value up to 10 should be sufficient for most clusters. + +# Cross-Validation + +`CrossValidator` begins by splitting the dataset into a set of *folds* which are used as separate training and test datasets. E.g., with `$k=3$` folds, `CrossValidator` will generate 3 (training, test) dataset pairs, each of which uses 2/3 of the data for training and 1/3 for testing. To evaluate a particular `ParamMap`, `CrossValidator` computes the average evaluation metric for the 3 `Model`s produced by fitting the `Estimator` on the 3 different (training, test) dataset pairs. + +After identifying the best `ParamMap`, `CrossValidator` finally re-fits the `Estimator` using the best `ParamMap` and the entire dataset. + +**Examples: model selection via cross-validation** + +The following example demonstrates using `CrossValidator` to select from a grid of parameters. + +Note that cross-validation over a grid of parameters is expensive. +E.g., in the example below, the parameter grid has 3 values for `hashingTF.numFeatures` and 2 values for `lr.regParam`, and `CrossValidator` uses 2 folds. This multiplies out to `$(3 \times 2) \times 2 = 12$` different models being trained. +In realistic settings, it can be common to try many more parameters and use more folds (`$k=3$` and `$k=10$` are common). +In other words, using `CrossValidator` can be very expensive. +However, it is also a well-established method for choosing parameters which is more statistically sound than heuristic hand-tuning. + +
    + +
    + +Refer to the [`CrossValidator` Scala docs](api/scala/index.html#org.apache.spark.ml.tuning.CrossValidator) for details on the API. + +{% include_example scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala %} +
    + +
    + +Refer to the [`CrossValidator` Java docs](api/java/org/apache/spark/ml/tuning/CrossValidator.html) for details on the API. + +{% include_example java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java %} +
    + +
    + +Refer to the [`CrossValidator` Python docs](api/python/pyspark.ml.html#pyspark.ml.tuning.CrossValidator) for more details on the API. + +{% include_example python/ml/cross_validator.py %} +
    + +
    + +# Train-Validation Split + +In addition to `CrossValidator` Spark also offers `TrainValidationSplit` for hyper-parameter tuning. +`TrainValidationSplit` only evaluates each combination of parameters once, as opposed to k times in + the case of `CrossValidator`. It is therefore less expensive, + but will not produce as reliable results when the training dataset is not sufficiently large. + +Unlike `CrossValidator`, `TrainValidationSplit` creates a single (training, test) dataset pair. +It splits the dataset into these two parts using the `trainRatio` parameter. For example with `$trainRatio=0.75$`, +`TrainValidationSplit` will generate a training and test dataset pair where 75% of the data is used for training and 25% for validation. + +Like `CrossValidator`, `TrainValidationSplit` finally fits the `Estimator` using the best `ParamMap` and the entire dataset. + +**Examples: model selection via train validation split** + +
    + +
    + +Refer to the [`TrainValidationSplit` Scala docs](api/scala/index.html#org.apache.spark.ml.tuning.TrainValidationSplit) for details on the API. + +{% include_example scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala %} +
    + +
    + +Refer to the [`TrainValidationSplit` Java docs](api/java/org/apache/spark/ml/tuning/TrainValidationSplit.html) for details on the API. + +{% include_example java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java %} +
    + +
    + +Refer to the [`TrainValidationSplit` Python docs](api/python/pyspark.ml.html#pyspark.ml.tuning.TrainValidationSplit) for more details on the API. + +{% include_example python/ml/train_validation_split.py %} +
    + +
    diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md index 0210950b8990..a7b90de09369 100644 --- a/docs/mllib-classification-regression.md +++ b/docs/mllib-classification-regression.md @@ -1,10 +1,10 @@ --- layout: global -title: Classification and Regression - MLlib -displayTitle: MLlib - Classification and Regression +title: Classification and Regression - RDD-based API +displayTitle: Classification and Regression - RDD-based API --- -MLlib supports various methods for +The `spark.mllib` package supports various methods for [binary classification](http://en.wikipedia.org/wiki/Binary_classification), [multiclass classification](http://en.wikipedia.org/wiki/Multiclass_classification), and diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 8fbced6c87d9..8990e95796b6 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -1,7 +1,7 @@ --- layout: global -title: Clustering - MLlib -displayTitle: MLlib - Clustering +title: Clustering - RDD-based API +displayTitle: Clustering - RDD-based API --- [Clustering](https://en.wikipedia.org/wiki/Cluster_analysis) is an unsupervised learning problem whereby we aim to group subsets @@ -10,27 +10,25 @@ often used for exploratory analysis and/or as a component of a hierarchical [supervised learning](https://en.wikipedia.org/wiki/Supervised_learning) pipeline (in which distinct classifiers or regression models are trained for each cluster). -MLlib supports the following models: +The `spark.mllib` package supports the following models: * Table of contents {:toc} ## K-means -[k-means](http://en.wikipedia.org/wiki/K-means_clustering) is one of the +[K-means](http://en.wikipedia.org/wiki/K-means_clustering) is one of the most commonly used clustering algorithms that clusters the data points into a -predefined number of clusters. The MLlib implementation includes a parallelized +predefined number of clusters. The `spark.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 `spark.mllib` has the following parameters: -* *k* is the number of desired clusters. +* *k* is the number of desired clusters. Note that it is possible for fewer than k clusters to be returned, for example, if there are fewer than k distinct points to cluster. * *maxIterations* is the maximum number of iterations to run. * *initializationMode* specifies either random initialization or initialization via k-means\|\|. -* *runs* is the number of times to run the k-means algorithm (k-means is not -guaranteed to find a globally optimal solution, and when run multiple times on -a given dataset, the algorithm returns the best clustering result). +* *runs* This param has no effect since Spark 2.0.0. * *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. * *initialModel* is an optional set of cluster centers used for initialization. If this parameter is supplied, only one run is performed. @@ -49,27 +47,7 @@ optimal *k* is usually one where there is an "elbow" in the WSSSE graph. Refer to the [`KMeans` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.KMeans) and [`KMeansModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.KMeansModel) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.clustering.{KMeans, KMeansModel} -import org.apache.spark.mllib.linalg.Vectors - -// Load and parse the data -val data = sc.textFile("data/mllib/kmeans_data.txt") -val parsedData = data.map(s => Vectors.dense(s.split(' ').map(_.toDouble))).cache() - -// Cluster the data into two classes using KMeans -val numClusters = 2 -val numIterations = 20 -val clusters = KMeans.train(parsedData, numClusters, numIterations) - -// Evaluate clustering by computing Within Set Sum of Squared Errors -val WSSSE = clusters.computeCost(parsedData) -println("Within Set Sum of Squared Errors = " + WSSSE) - -// Save and load model -clusters.save(sc, "myModelPath") -val sameModel = KMeansModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/KMeansExample.scala %}
    @@ -81,51 +59,7 @@ that is equivalent to the provided example in Scala is given below: Refer to the [`KMeans` Java docs](api/java/org/apache/spark/mllib/clustering/KMeans.html) and [`KMeansModel` Java docs](api/java/org/apache/spark/mllib/clustering/KMeansModel.html) for details on the API. -{% highlight java %} -import org.apache.spark.api.java.*; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.clustering.KMeans; -import org.apache.spark.mllib.clustering.KMeansModel; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.SparkConf; - -public class KMeansExample { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("K-means Example"); - JavaSparkContext sc = new JavaSparkContext(conf); - - // Load and parse data - String path = "data/mllib/kmeans_data.txt"; - JavaRDD data = sc.textFile(path); - JavaRDD parsedData = data.map( - new Function() { - public Vector call(String s) { - String[] sarray = s.split(" "); - double[] values = new double[sarray.length]; - for (int i = 0; i < sarray.length; i++) - values[i] = Double.parseDouble(sarray[i]); - return Vectors.dense(values); - } - } - ); - parsedData.cache(); - - // Cluster the data into two classes using KMeans - int numClusters = 2; - int numIterations = 20; - KMeansModel clusters = KMeans.train(parsedData.rdd(), numClusters, numIterations); - - // Evaluate clustering by computing Within Set Sum of Squared Errors - double WSSSE = clusters.computeCost(parsedData.rdd()); - System.out.println("Within Set Sum of Squared Errors = " + WSSSE); - - // Save and load model - clusters.save(sc.sc(), "myModelPath"); - KMeansModel sameModel = KMeansModel.load(sc.sc(), "myModelPath"); - } -} -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaKMeansExample.java %}
    @@ -138,31 +72,7 @@ fact the optimal *k* is usually one where there is an "elbow" in the WSSSE graph Refer to the [`KMeans` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.clustering.KMeans) and [`KMeansModel` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.clustering.KMeansModel) for more details on the API. -{% highlight python %} -from pyspark.mllib.clustering import KMeans, KMeansModel -from numpy import array -from math import sqrt - -# Load and parse the data -data = sc.textFile("data/mllib/kmeans_data.txt") -parsedData = data.map(lambda line: array([float(x) for x in line.split(' ')])) - -# Build the model (cluster the data) -clusters = KMeans.train(parsedData, 2, maxIterations=10, - runs=10, initializationMode="random") - -# Evaluate clustering by computing Within Set Sum of Squared Errors -def error(point): - center = clusters.centers[clusters.predict(point)] - return sqrt(sum([x**2 for x in (point - center)])) - -WSSSE = parsedData.map(lambda point: error(point)).reduce(lambda x, y: x + y) -print("Within Set Sum of Squared Error = " + str(WSSSE)) - -# Save and load model -clusters.save(sc, "myModelPath") -sameModel = KMeansModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example python/mllib/k_means_example.py %}
    @@ -171,7 +81,7 @@ sameModel = KMeansModel.load(sc, "myModelPath") 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 +each with its own probability. The `spark.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: @@ -192,29 +102,7 @@ to the algorithm. We then output the parameters of the mixture model. Refer to the [`GaussianMixture` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.GaussianMixture) and [`GaussianMixtureModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.GaussianMixtureModel) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.clustering.GaussianMixture -import org.apache.spark.mllib.clustering.GaussianMixtureModel -import org.apache.spark.mllib.linalg.Vectors - -// Load and parse the data -val data = sc.textFile("data/mllib/gmm_data.txt") -val parsedData = data.map(s => Vectors.dense(s.trim.split(' ').map(_.toDouble))).cache() - -// Cluster the data into two classes using GaussianMixture -val gmm = new GaussianMixture().setK(2).run(parsedData) - -// Save and load model -gmm.save(sc, "myGMMModel") -val sameModel = GaussianMixtureModel.load(sc, "myGMMModel") - -// output parameters of max-likelihood model -for (i <- 0 until gmm.k) { - println("weight=%f\nmu=%s\nsigma=\n%s\n" format - (gmm.weights(i), gmm.gaussians(i).mu, gmm.gaussians(i).sigma)) -} - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/GaussianMixtureExample.scala %}
    @@ -226,50 +114,7 @@ that is equivalent to the provided example in Scala is given below: Refer to the [`GaussianMixture` Java docs](api/java/org/apache/spark/mllib/clustering/GaussianMixture.html) and [`GaussianMixtureModel` Java docs](api/java/org/apache/spark/mllib/clustering/GaussianMixtureModel.html) for details on the API. -{% highlight java %} -import org.apache.spark.api.java.*; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.clustering.GaussianMixture; -import org.apache.spark.mllib.clustering.GaussianMixtureModel; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.SparkConf; - -public class GaussianMixtureExample { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("GaussianMixture Example"); - JavaSparkContext sc = new JavaSparkContext(conf); - - // Load and parse data - String path = "data/mllib/gmm_data.txt"; - JavaRDD data = sc.textFile(path); - JavaRDD parsedData = data.map( - new Function() { - public Vector call(String s) { - String[] sarray = s.trim().split(" "); - double[] values = new double[sarray.length]; - for (int i = 0; i < sarray.length; i++) - values[i] = Double.parseDouble(sarray[i]); - return Vectors.dense(values); - } - } - ); - parsedData.cache(); - - // Cluster the data into two classes using GaussianMixture - GaussianMixtureModel gmm = new GaussianMixture().setK(2).run(parsedData.rdd()); - - // Save and load GaussianMixtureModel - gmm.save(sc.sc(), "myGMMModel"); - GaussianMixtureModel sameModel = GaussianMixtureModel.load(sc.sc(), "myGMMModel"); - // Output the parameters of the mixture model - for(int j=0; j
    @@ -280,23 +125,7 @@ to the algorithm. We then output the parameters of the mixture model. Refer to the [`GaussianMixture` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.clustering.GaussianMixture) and [`GaussianMixtureModel` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.clustering.GaussianMixtureModel) for more details on the API. -{% highlight python %} -from pyspark.mllib.clustering import GaussianMixture -from numpy import array - -# Load and parse the data -data = sc.textFile("data/mllib/gmm_data.txt") -parsedData = data.map(lambda line: array([float(x) for x in line.strip().split(' ')])) - -# Build the model (cluster the data) -gmm = GaussianMixture.train(parsedData, 2) - -# output parameters of model -for i in range(2): - print ("weight = ", gmm.weights[i], "mu = ", gmm.gaussians[i].mu, - "sigma = ", gmm.gaussians[i].sigma.toArray()) - -{% endhighlight %} +{% include_example python/mllib/gaussian_mixture_example.py %}
    @@ -304,17 +133,17 @@ for i in range(2): ## Power iteration clustering (PIC) Power iteration clustering (PIC) is a scalable and efficient algorithm for clustering vertices of a -graph given pairwise similarties as edge properties, +graph given pairwise similarities 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. +`spark.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: +`spark.mllib`'s PIC implementation takes the following (hyper-)parameters: * `k`: number of clusters * `maxIterations`: maximum number of power iterations @@ -323,7 +152,7 @@ MLlib's PIC implementation takes the following (hyper-)parameters: **Examples** -In the following, we show code snippets to demonstrate how to use PIC in MLlib. +In the following, we show code snippets to demonstrate how to use PIC in `spark.mllib`.
    @@ -338,35 +167,7 @@ which contains the computed clustering assignments. Refer to the [`PowerIterationClustering` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.PowerIterationClustering) and [`PowerIterationClusteringModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.PowerIterationClusteringModel) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.clustering.{PowerIterationClustering, PowerIterationClusteringModel} -import org.apache.spark.mllib.linalg.Vectors - -// Load and parse the data -val data = sc.textFile("data/mllib/pic_data.txt") -val similarities = data.map { line => - val parts = line.split(' ') - (parts(0).toLong, parts(1).toLong, parts(2).toDouble) -} - -// Cluster the data into two classes using PowerIterationClustering -val pic = new PowerIterationClustering() - .setK(2) - .setMaxIterations(10) -val model = pic.run(similarities) - -model.assignments.foreach { a => - println(s"${a.id} -> ${a.cluster}") -} - -// Save and load model -model.save(sc, "myModelPath") -val sameModel = PowerIterationClusteringModel.load(sc, "myModelPath") -{% 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). - +{% include_example scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala %}
    @@ -381,40 +182,7 @@ which contains the computed clustering assignments. Refer to the [`PowerIterationClustering` Java docs](api/java/org/apache/spark/mllib/clustering/PowerIterationClustering.html) and [`PowerIterationClusteringModel` Java docs](api/java/org/apache/spark/mllib/clustering/PowerIterationClusteringModel.html) for details on the API. -{% highlight java %} -import scala.Tuple2; -import scala.Tuple3; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.clustering.PowerIterationClustering; -import org.apache.spark.mllib.clustering.PowerIterationClusteringModel; - -// Load and parse the data -JavaRDD data = sc.textFile("data/mllib/pic_data.txt"); -JavaRDD> similarities = data.map( - new Function>() { - public Tuple3 call(String line) { - String[] parts = line.split(" "); - return new Tuple3<>(new Long(parts[0]), new Long(parts[1]), new Double(parts[2])); - } - } -); - -// Cluster the data into two classes using PowerIterationClustering -PowerIterationClustering pic = new PowerIterationClustering() - .setK(2) - .setMaxIterations(10); -PowerIterationClusteringModel model = pic.run(similarities); - -for (PowerIterationClustering.Assignment a: model.assignments().toJavaRDD().collect()) { - System.out.println(a.id() + " -> " + a.cluster()); -} - -// Save and load model -model.save(sc.sc(), "myModelPath"); -PowerIterationClusteringModel sameModel = PowerIterationClusteringModel.load(sc.sc(), "myModelPath"); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java %}
    @@ -429,23 +197,7 @@ which contains the computed clustering assignments. Refer to the [`PowerIterationClustering` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.clustering.PowerIterationClustering) and [`PowerIterationClusteringModel` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.clustering.PowerIterationClusteringModel) for more details on the API. -{% highlight python %} -from __future__ import print_function -from pyspark.mllib.clustering import PowerIterationClustering, PowerIterationClusteringModel - -# Load and parse the data -data = sc.textFile("data/mllib/pic_data.txt") -similarities = data.map(lambda line: tuple([float(x) for x in line.split(' ')])) - -# Cluster the data into two classes using PowerIterationClustering -model = PowerIterationClustering.train(similarities, 2, 10) - -model.assignments().foreach(lambda x: print(str(x.id) + " -> " + str(x.cluster))) - -# Save and load model -model.save(sc, "myModelPath") -sameModel = PowerIterationClusteringModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example python/mllib/power_iteration_clustering_example.py %}
    @@ -493,7 +245,7 @@ checkpointing can help reduce shuffle file sizes on disk and help with failure recovery. -All of MLlib's LDA models support: +All of `spark.mllib`'s LDA models support: * `describeTopics`: Returns topics as arrays of most important terms and term weights @@ -591,137 +343,68 @@ to the algorithm. We then output the topics, represented as probability distribu
    Refer to the [`LDA` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.LDA) and [`DistributedLDAModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.DistributedLDAModel) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.clustering.{LDA, DistributedLDAModel} -import org.apache.spark.mllib.linalg.Vectors - -// Load and parse the data -val data = sc.textFile("data/mllib/sample_lda_data.txt") -val parsedData = data.map(s => Vectors.dense(s.trim.split(' ').map(_.toDouble))) -// Index documents with unique IDs -val corpus = parsedData.zipWithIndex.map(_.swap).cache() - -// Cluster the documents into three topics using LDA -val ldaModel = new LDA().setK(3).run(corpus) - -// Output topics. Each is a distribution over words (matching word count vectors) -println("Learned topics (as distributions over vocab of " + ldaModel.vocabSize + " words):") -val topics = ldaModel.topicsMatrix -for (topic <- Range(0, 3)) { - print("Topic " + topic + ":") - for (word <- Range(0, ldaModel.vocabSize)) { print(" " + topics(word, topic)); } - println() -} - -// Save and load model. -ldaModel.save(sc, "myLDAModel") -val sameModel = DistributedLDAModel.load(sc, "myLDAModel") - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/LatentDirichletAllocationExample.scala %}
    Refer to the [`LDA` Java docs](api/java/org/apache/spark/mllib/clustering/LDA.html) and [`DistributedLDAModel` Java docs](api/java/org/apache/spark/mllib/clustering/DistributedLDAModel.html) for details on the API. -{% highlight java %} -import scala.Tuple2; - -import org.apache.spark.api.java.*; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.clustering.DistributedLDAModel; -import org.apache.spark.mllib.clustering.LDA; -import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.SparkConf; - -public class JavaLDAExample { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("LDA Example"); - JavaSparkContext sc = new JavaSparkContext(conf); - - // Load and parse the data - String path = "data/mllib/sample_lda_data.txt"; - JavaRDD data = sc.textFile(path); - JavaRDD parsedData = data.map( - new Function() { - public Vector call(String s) { - String[] sarray = s.trim().split(" "); - double[] values = new double[sarray.length]; - for (int i = 0; i < sarray.length; i++) - values[i] = Double.parseDouble(sarray[i]); - return Vectors.dense(values); - } - } - ); - // Index documents with unique IDs - JavaPairRDD corpus = JavaPairRDD.fromJavaRDD(parsedData.zipWithIndex().map( - new Function, Tuple2>() { - public Tuple2 call(Tuple2 doc_id) { - return doc_id.swap(); - } - } - )); - corpus.cache(); - - // Cluster the documents into three topics using LDA - DistributedLDAModel ldaModel = new LDA().setK(3).run(corpus); - - // Output topics. Each is a distribution over words (matching word count vectors) - System.out.println("Learned topics (as distributions over vocab of " + ldaModel.vocabSize() - + " words):"); - Matrix topics = ldaModel.topicsMatrix(); - for (int topic = 0; topic < 3; topic++) { - System.out.print("Topic " + topic + ":"); - for (int word = 0; word < ldaModel.vocabSize(); word++) { - System.out.print(" " + topics.apply(word, topic)); - } - System.out.println(); - } - - ldaModel.save(sc.sc(), "myLDAModel"); - DistributedLDAModel sameModel = DistributedLDAModel.load(sc.sc(), "myLDAModel"); - } -} -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaLatentDirichletAllocationExample.java %}
    Refer to the [`LDA` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.clustering.LDA) and [`LDAModel` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.clustering.LDAModel) for more details on the API. -{% highlight python %} -from pyspark.mllib.clustering import LDA, LDAModel -from pyspark.mllib.linalg import Vectors - -# Load and parse the data -data = sc.textFile("data/mllib/sample_lda_data.txt") -parsedData = data.map(lambda line: Vectors.dense([float(x) for x in line.strip().split(' ')])) -# Index documents with unique IDs -corpus = parsedData.zipWithIndex().map(lambda x: [x[1], x[0]]).cache() - -# Cluster the documents into three topics using LDA -ldaModel = LDA.train(corpus, k=3) - -# Output topics. Each is a distribution over words (matching word count vectors) -print("Learned topics (as distributions over vocab of " + str(ldaModel.vocabSize()) + " words):") -topics = ldaModel.topicsMatrix() -for topic in range(3): - print("Topic " + str(topic) + ":") - for word in range(0, ldaModel.vocabSize()): - print(" " + str(topics[word][topic])) - -# Save and load model -model.save(sc, "myModelPath") -sameModel = LDAModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example python/mllib/latent_dirichlet_allocation_example.py %} +
    + + + +## Bisecting k-means + +Bisecting K-means can often be much faster than regular K-means, but it will generally produce a different clustering. + +Bisecting k-means is a kind of [hierarchical clustering](https://en.wikipedia.org/wiki/Hierarchical_clustering). +Hierarchical clustering is one of the most commonly used method of cluster analysis which seeks to build a hierarchy of clusters. +Strategies for hierarchical clustering generally fall into two types: + +- Agglomerative: This is a "bottom up" approach: each observation starts in its own cluster, and pairs of clusters are merged as one moves up the hierarchy. +- Divisive: This is a "top down" approach: all observations start in one cluster, and splits are performed recursively as one moves down the hierarchy. + +Bisecting k-means algorithm is a kind of divisive algorithms. +The implementation in MLlib has the following parameters: + +* *k*: the desired number of leaf clusters (default: 4). The actual number could be smaller if there are no divisible leaf clusters. +* *maxIterations*: the max number of k-means iterations to split clusters (default: 20) +* *minDivisibleClusterSize*: the minimum number of points (if >= 1.0) or the minimum proportion of points (if < 1.0) of a divisible cluster (default: 1) +* *seed*: a random seed (default: hash value of the class name) + +**Examples** + +
    +
    +Refer to the [`BisectingKMeans` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.BisectingKMeans) and [`BisectingKMeansModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.BisectingKMeansModel) for details on the API. + +{% include_example scala/org/apache/spark/examples/mllib/BisectingKMeansExample.scala %} +
    + +
    +Refer to the [`BisectingKMeans` Java docs](api/java/org/apache/spark/mllib/clustering/BisectingKMeans.html) and [`BisectingKMeansModel` Java docs](api/java/org/apache/spark/mllib/clustering/BisectingKMeansModel.html) for details on the API. + +{% include_example java/org/apache/spark/examples/mllib/JavaBisectingKMeansExample.java %}
    +
    +Refer to the [`BisectingKMeans` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.clustering.BisectingKMeans) and [`BisectingKMeansModel` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.clustering.BisectingKMeansModel) for more details on the API. + +{% include_example python/mllib/bisecting_k_means_example.py %} +
    ## 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, +updating them as new data arrive. `spark.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: @@ -754,96 +437,16 @@ This example shows how to estimate clusters on streaming data.
    Refer to the [`StreamingKMeans` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.StreamingKMeans) for details on the API. +And Refer to [Spark Streaming Programming Guide](streaming-programming-guide.html#initializing) for details on StreamingContext. -First we import the neccessary classes. - -{% highlight scala %} - -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint -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. - -{% highlight scala %} - -val trainingData = ssc.textFileStream("/training/data/dir").map(Vectors.parse) -val testData = ssc.textFileStream("/testing/data/dir").map(LabeledPoint.parse) - -{% endhighlight %} - -We create a model with random clusters and specify the number of clusters to find - -{% highlight scala %} - -val numDimensions = 3 -val numClusters = 2 -val model = new StreamingKMeans() - .setK(numClusters) - .setDecayFactor(1.0) - .setRandomCenters(numDimensions, 0.0) - -{% endhighlight %} - -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.map(lp => (lp.label, lp.features))).print() - -ssc.start() -ssc.awaitTermination() - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala %}
    Refer to the [`StreamingKMeans` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.clustering.StreamingKMeans) for more details on the API. +And Refer to [Spark Streaming Programming Guide](streaming-programming-guide.html#initializing) for details on StreamingContext. -First we import the neccessary classes. - -{% highlight python %} -from pyspark.mllib.linalg import Vectors -from pyspark.mllib.regression import LabeledPoint -from pyspark.mllib.clustering import 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. - -{% highlight python %} -def parse(lp): - label = float(lp[lp.find('(') + 1: lp.find(',')]) - vec = Vectors.dense(lp[lp.find('[') + 1: lp.find(']')].split(',')) - return LabeledPoint(label, vec) - -trainingData = ssc.textFileStream("/training/data/dir").map(Vectors.parse) -testData = ssc.textFileStream("/testing/data/dir").map(parse) -{% endhighlight %} - -We create a model with random clusters and specify the number of clusters to find - -{% highlight python %} -model = StreamingKMeans(k=2, decayFactor=1.0).setRandomCenters(3, 1.0, 0) -{% endhighlight %} - -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 python %} -model.trainOn(trainingData) -print(model.predictOnValues(testData.map(lambda lp: (lp.label, lp.features)))) - -ssc.start() -ssc.awaitTermination() -{% endhighlight %} +{% include_example python/mllib/streaming_k_means_example.py %}
    diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index 1ad52123c74a..76a00f18b3b9 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -1,7 +1,7 @@ --- layout: global -title: Collaborative Filtering - MLlib -displayTitle: MLlib - Collaborative Filtering +title: Collaborative Filtering - RDD-based API +displayTitle: Collaborative Filtering - RDD-based API --- * Table of contents @@ -11,17 +11,18 @@ displayTitle: MLlib - Collaborative Filtering [Collaborative filtering](http://en.wikipedia.org/wiki/Recommender_system#Collaborative_filtering) is commonly used for recommender systems. These techniques aim to fill in the -missing entries of a user-item association matrix. MLlib currently supports +missing entries of a user-item association matrix. `spark.mllib` currently supports model-based collaborative filtering, in which users and products are described by a small set of latent factors that can be used to predict missing entries. -MLlib uses the [alternating least squares +`spark.mllib` uses the [alternating least squares (ALS)](http://dl.acm.org/citation.cfm?id=1608614) -algorithm to learn these latent factors. The implementation in MLlib has the +algorithm to learn these latent factors. The implementation in `spark.mllib` has the following parameters: * *numBlocks* is the number of blocks used to parallelize computation (set to -1 to auto-configure). -* *rank* is the number of latent factors in the model. -* *iterations* is the number of iterations to run. +* *rank* is the number of features to use (also referred to as the number of latent factors). +* *iterations* is the number of iterations of ALS to run. ALS typically converges to a reasonable + solution in 20 iterations or less. * *lambda* specifies the regularization parameter in ALS. * *implicitPrefs* specifies whether to use the *explicit feedback* ALS variant or one adapted for *implicit feedback* data. @@ -31,17 +32,18 @@ following parameters: ### Explicit vs. implicit feedback The standard approach to matrix factorization based collaborative filtering treats -the entries in the user-item matrix as *explicit* preferences given by the user to the item. +the entries in the user-item matrix as *explicit* preferences given by the user to the item, +for example, users giving ratings to movies. It is common in many real-world use cases to only have access to *implicit feedback* (e.g. views, -clicks, purchases, likes, shares etc.). The approach used in MLlib to deal with such data is taken -from -[Collaborative Filtering for Implicit Feedback Datasets](http://dx.doi.org/10.1109/ICDM.2008.22). -Essentially instead of trying to model the matrix of ratings directly, this approach treats the data -as a combination of binary preferences and *confidence values*. The ratings are then related to the -level of confidence in observed user preferences, rather than explicit ratings given to items. The -model then tries to find latent factors that can be used to predict the expected preference of a -user for an item. +clicks, purchases, likes, shares etc.). The approach used in `spark.mllib` to deal with such data is taken +from [Collaborative Filtering for Implicit Feedback Datasets](http://dx.doi.org/10.1109/ICDM.2008.22). +Essentially, instead of trying to model the matrix of ratings directly, this approach treats the data +as numbers representing the *strength* in observations of user actions (such as the number of clicks, +or the cumulative duration someone spent viewing a movie). Those numbers are then related to the level of +confidence in observed user preferences, rather than explicit ratings given to items. The model +then tries to find latent factors that can be used to predict the expected preference of a user for +an item. ### Scaling of the regularization parameter @@ -50,9 +52,8 @@ the number of ratings the user generated in updating user factors, or the number of ratings the product received in updating product factors. This approach is named "ALS-WR" and discussed in the paper "[Large-Scale Parallel Collaborative Filtering for the Netflix Prize](http://dx.doi.org/10.1007/978-3-540-68880-8_32)". -It makes `lambda` less dependent on the scale of the dataset. -So we can apply the best parameter learned from a sampled subset to the full dataset -and expect similar performance. +It makes `lambda` less dependent on the scale of the dataset, so we can apply the +best parameter learned from a sampled subset to the full dataset and expect similar performance. ## Examples @@ -64,47 +65,11 @@ We use the default [ALS.train()](api/scala/index.html#org.apache.spark.mllib.rec method which assumes ratings are explicit. We evaluate the recommendation model by measuring the Mean Squared Error of rating prediction. -Refer to the [`ALS` Scala docs](api/scala/index.html#org.apache.spark.mllib.recommendation.ALS) for details on the API. +Refer to the [`ALS` Scala docs](api/scala/index.html#org.apache.spark.mllib.recommendation.ALS) for more details on the API. -{% highlight scala %} -import org.apache.spark.mllib.recommendation.ALS -import org.apache.spark.mllib.recommendation.MatrixFactorizationModel -import org.apache.spark.mllib.recommendation.Rating - -// Load and parse the data -val data = sc.textFile("data/mllib/als/test.data") -val ratings = data.map(_.split(',') match { case Array(user, item, rate) => - Rating(user.toInt, item.toInt, rate.toDouble) - }) - -// Build the recommendation model using ALS -val rank = 10 -val numIterations = 10 -val model = ALS.train(ratings, rank, numIterations, 0.01) - -// Evaluate the model on rating data -val usersProducts = ratings.map { case Rating(user, product, rate) => - (user, product) -} -val predictions = - model.predict(usersProducts).map { case Rating(user, product, rate) => - ((user, product), rate) - } -val ratesAndPreds = ratings.map { case Rating(user, product, rate) => - ((user, product), rate) -}.join(predictions) -val MSE = ratesAndPreds.map { case ((user, product), (r1, r2)) => - val err = (r1 - r2) - err * err -}.mean() -println("Mean Squared Error = " + MSE) - -// Save and load model -model.save(sc, "myModelPath") -val sameModel = MatrixFactorizationModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/RecommendationExample.scala %} -If the rating matrix is derived from another source of information (e.g., it is inferred from +If the rating matrix is derived from another source of information (i.e. it is inferred from other signals), you can use the `trainImplicit` method to get better results. {% highlight scala %} @@ -121,83 +86,9 @@ Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a calling `.rdd()` on your `JavaRDD` object. A self-contained application example that is equivalent to the provided example in Scala is given below: -Refer to the [`ALS` Java docs](api/java/org/apache/spark/mllib/recommendation/ALS.html) for details on the API. - -{% highlight java %} -import scala.Tuple2; - -import org.apache.spark.api.java.*; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.recommendation.ALS; -import org.apache.spark.mllib.recommendation.MatrixFactorizationModel; -import org.apache.spark.mllib.recommendation.Rating; -import org.apache.spark.SparkConf; - -public class CollaborativeFiltering { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("Collaborative Filtering Example"); - JavaSparkContext sc = new JavaSparkContext(conf); - - // Load and parse the data - String path = "data/mllib/als/test.data"; - JavaRDD data = sc.textFile(path); - JavaRDD ratings = data.map( - new Function() { - public Rating call(String s) { - String[] sarray = s.split(","); - return new Rating(Integer.parseInt(sarray[0]), Integer.parseInt(sarray[1]), - Double.parseDouble(sarray[2])); - } - } - ); - - // Build the recommendation model using ALS - int rank = 10; - int numIterations = 10; - MatrixFactorizationModel model = ALS.train(JavaRDD.toRDD(ratings), rank, numIterations, 0.01); - - // Evaluate the model on rating data - JavaRDD> userProducts = ratings.map( - new Function>() { - public Tuple2 call(Rating r) { - return new Tuple2(r.user(), r.product()); - } - } - ); - JavaPairRDD, Double> predictions = JavaPairRDD.fromJavaRDD( - model.predict(JavaRDD.toRDD(userProducts)).toJavaRDD().map( - new Function, Double>>() { - public Tuple2, Double> call(Rating r){ - return new Tuple2, Double>( - new Tuple2(r.user(), r.product()), r.rating()); - } - } - )); - JavaRDD> ratesAndPreds = - JavaPairRDD.fromJavaRDD(ratings.map( - new Function, Double>>() { - public Tuple2, Double> call(Rating r){ - return new Tuple2, Double>( - new Tuple2(r.user(), r.product()), r.rating()); - } - } - )).join(predictions).values(); - double MSE = JavaDoubleRDD.fromRDD(ratesAndPreds.map( - new Function, Object>() { - public Object call(Tuple2 pair) { - Double err = pair._1() - pair._2(); - return err * err; - } - } - ).rdd()).mean(); - System.out.println("Mean Squared Error = " + MSE); - - // Save and load model - model.save(sc.sc(), "myModelPath"); - MatrixFactorizationModel sameModel = MatrixFactorizationModel.load(sc.sc(), "myModelPath"); - } -} -{% endhighlight %} +Refer to the [`ALS` Java docs](api/java/org/apache/spark/mllib/recommendation/ALS.html) for more details on the API. + +{% include_example java/org/apache/spark/examples/mllib/JavaRecommendationExample.java %}
    @@ -207,31 +98,9 @@ recommendation by measuring the Mean Squared Error of rating prediction. Refer to the [`ALS` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.recommendation.ALS) for more details on the API. -{% highlight python %} -from pyspark.mllib.recommendation import ALS, MatrixFactorizationModel, Rating - -# Load and parse the data -data = sc.textFile("data/mllib/als/test.data") -ratings = data.map(lambda l: l.split(',')).map(lambda l: Rating(int(l[0]), int(l[1]), float(l[2]))) - -# Build the recommendation model using Alternating Least Squares -rank = 10 -numIterations = 10 -model = ALS.train(ratings, rank, numIterations) - -# Evaluate the model on training data -testdata = ratings.map(lambda p: (p[0], p[1])) -predictions = model.predictAll(testdata).map(lambda r: ((r[0], r[1]), r[2])) -ratesAndPreds = ratings.map(lambda r: ((r[0], r[1]), r[2])).join(predictions) -MSE = ratesAndPreds.map(lambda r: (r[1][0] - r[1][1])**2).mean() -print("Mean Squared Error = " + str(MSE)) - -# Save and load model -model.save(sc, "myModelPath") -sameModel = MatrixFactorizationModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example python/mllib/recommendation_example.py %} -If the rating matrix is derived from other source of information (i.e., it is inferred from other +If the rating matrix is derived from other source of information (i.e. it is inferred from other signals), you can use the trainImplicit method to get better results. {% highlight python %} @@ -250,5 +119,5 @@ a dependency. ## Tutorial -The [training exercises](https://databricks-training.s3.amazonaws.com/index.html) from the Spark Summit 2014 include a hands-on tutorial for -[personalized movie recommendation with MLlib](https://databricks-training.s3.amazonaws.com/movie-recommendation-with-mllib.html). +The [training exercises](https://github.com/databricks/spark-training) from the Spark Summit 2014 include a hands-on tutorial for +[personalized movie recommendation with `spark.mllib`](https://github.com/databricks/spark-training/blob/master/website/movie-recommendation-with-mllib.md). diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md index 3c0c0479674d..35cee3275e3b 100644 --- a/docs/mllib-data-types.md +++ b/docs/mllib-data-types.md @@ -1,7 +1,7 @@ --- layout: global -title: Data Types - MLlib -displayTitle: MLlib - Data Types +title: Data Types - RDD-based API +displayTitle: Data Types - RDD-based API --- * Table of contents @@ -11,7 +11,7 @@ MLlib supports local vectors and matrices stored on a single machine, as well as distributed matrices backed by one or more RDDs. Local vectors and local matrices are simple data models that serve as public interfaces. The underlying linear algebra operations are provided by -[Breeze](http://www.scalanlp.org/) and [jblas](http://jblas.org/). +[Breeze](http://www.scalanlp.org/). A training example used in supervised learning is called a "labeled point" in MLlib. ## Local vector @@ -33,7 +33,7 @@ implementations: [`DenseVector`](api/scala/index.html#org.apache.spark.mllib.lin using the factory methods implemented in [`Vectors`](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) to create local vectors. -Refer to the [`Vector` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) and [`Vectors` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors) for details on the API. +Refer to the [`Vector` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) and [`Vectors` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) for details on the API. {% highlight scala %} import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -104,7 +104,7 @@ dv2 = [1.0, 0.0, 3.0] # Create a SparseVector. sv1 = Vectors.sparse(3, [0, 2], [1.0, 3.0]) # Use a single-column SciPy csc_matrix as a sparse vector. -sv2 = sps.csc_matrix((np.array([1.0, 3.0]), np.array([0, 2]), np.array([0, 2])), shape = (3, 1)) +sv2 = sps.csc_matrix((np.array([1.0, 3.0]), np.array([0, 2]), np.array([0, 2])), shape=(3, 1)) {% endhighlight %}
    @@ -199,7 +199,7 @@ After loading, the feature indices are converted to zero-based. [`MLUtils.loadLibSVMFile`](api/scala/index.html#org.apache.spark.mllib.util.MLUtils$) reads training examples stored in LIBSVM format. -Refer to the [`MLUtils` Scala docs](api/scala/index.html#org.apache.spark.mllib.util.MLUtils) for details on the API. +Refer to the [`MLUtils` Scala docs](api/scala/index.html#org.apache.spark.mllib.util.MLUtils$) for details on the API. {% highlight scala %} import org.apache.spark.mllib.regression.LabeledPoint @@ -264,7 +264,7 @@ We recommend using the factory methods implemented in [`Matrices`](api/scala/index.html#org.apache.spark.mllib.linalg.Matrices$) to create local matrices. Remember, local matrices in MLlib are stored in column-major order. -Refer to the [`Matrix` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Matrix) and [`Matrices` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Matrices) for details on the API. +Refer to the [`Matrix` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Matrix) and [`Matrices` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Matrices$) for details on the API. {% highlight scala %} import org.apache.spark.mllib.linalg.{Matrix, Matrices} @@ -314,12 +314,12 @@ matrices. Remember, local matrices in MLlib are stored in column-major order. Refer to the [`Matrix` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.linalg.Matrix) and [`Matrices` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.linalg.Matrices) for more details on the API. {% highlight python %} -import org.apache.spark.mllib.linalg.{Matrix, Matrices} +from pyspark.mllib.linalg import Matrix, Matrices -// Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) +# Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) dm2 = Matrices.dense(3, 2, [1, 2, 3, 4, 5, 6]) -// Create a sparse matrix ((9.0, 0.0), (0.0, 8.0), (0.0, 6.0)) +# Create a sparse matrix ((9.0, 0.0), (0.0, 8.0), (0.0, 6.0)) sm = Matrices.sparse(3, 2, [0, 1, 3], [0, 2, 1], [9, 6, 8]) {% endhighlight %} @@ -331,7 +331,7 @@ sm = Matrices.sparse(3, 2, [0, 1, 3], [0, 2, 1], [9, 6, 8]) A distributed matrix has long-typed row and column indices and double-typed values, stored distributively in one or more RDDs. It is very important to choose the right format to store large and distributed matrices. Converting a distributed matrix to a different format may require a -global shuffle, which is quite expensive. Three types of distributed matrices have been implemented +global shuffle, which is quite expensive. Four types of distributed matrices have been implemented so far. The basic type is called `RowMatrix`. A `RowMatrix` is a row-oriented distributed @@ -344,6 +344,8 @@ An `IndexedRowMatrix` is similar to a `RowMatrix` but with row indices, which can be used for identifying rows and executing joins. A `CoordinateMatrix` is a distributed matrix stored in [coordinate list (COO)](https://en.wikipedia.org/wiki/Sparse_matrix#Coordinate_list_.28COO.29) format, backed by an RDD of its entries. +A `BlockMatrix` is a distributed matrix backed by an RDD of `MatrixBlock` +which is a tuple of `(Int, Int, Matrix)`. ***Note*** @@ -515,12 +517,12 @@ from pyspark.mllib.linalg.distributed import IndexedRow, IndexedRowMatrix # Create an RDD of indexed rows. # - This can be done explicitly with the IndexedRow class: -indexedRows = sc.parallelize([IndexedRow(0, [1, 2, 3]), - IndexedRow(1, [4, 5, 6]), - IndexedRow(2, [7, 8, 9]), +indexedRows = sc.parallelize([IndexedRow(0, [1, 2, 3]), + IndexedRow(1, [4, 5, 6]), + IndexedRow(2, [7, 8, 9]), IndexedRow(3, [10, 11, 12])]) # - or by using (long, vector) tuples: -indexedRows = sc.parallelize([(0, [1, 2, 3]), (1, [4, 5, 6]), +indexedRows = sc.parallelize([(0, [1, 2, 3]), (1, [4, 5, 6]), (2, [7, 8, 9]), (3, [10, 11, 12])]) # Create an IndexedRowMatrix from an RDD of IndexedRows. @@ -535,12 +537,6 @@ rowsRDD = mat.rows # Convert to a RowMatrix by dropping the row indices. rowMat = mat.toRowMatrix() - -# Convert to a CoordinateMatrix. -coordinateMat = mat.toCoordinateMatrix() - -# Convert to a BlockMatrix. -blockMat = mat.toBlockMatrix() {% endhighlight %} @@ -735,15 +731,15 @@ from pyspark.mllib.linalg import Matrices from pyspark.mllib.linalg.distributed import BlockMatrix # Create an RDD of sub-matrix blocks. -blocks = sc.parallelize([((0, 0), Matrices.dense(3, 2, [1, 2, 3, 4, 5, 6])), +blocks = sc.parallelize([((0, 0), Matrices.dense(3, 2, [1, 2, 3, 4, 5, 6])), ((1, 0), Matrices.dense(3, 2, [7, 8, 9, 10, 11, 12]))]) # Create a BlockMatrix from an RDD of sub-matrix blocks. mat = BlockMatrix(blocks, 3, 2) # Get its size. -m = mat.numRows() # 6 -n = mat.numCols() # 2 +m = mat.numRows() # 6 +n = mat.numCols() # 2 # Get the blocks as an RDD of sub-matrix blocks. blocksRDD = mat.blocks diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index f31c4f88936b..0e753b8dd04a 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -1,7 +1,7 @@ --- layout: global -title: Decision Trees - MLlib -displayTitle: MLlib - Decision Trees +title: Decision Trees - RDD-based API +displayTitle: Decision Trees - RDD-based API --- * Table of contents @@ -15,7 +15,7 @@ feature scaling, and are able to capture non-linearities and feature interaction algorithms such as random forests and boosting are among the top performers for classification and regression tasks. -MLlib supports decision trees for binary and multiclass classification and for regression, +`spark.mllib` supports decision trees for binary and multiclass classification and for regression, using both continuous and categorical features. The implementation partitions data by rows, allowing distributed training with millions of instances. @@ -121,12 +121,12 @@ The parameters are listed below roughly in order of descending importance. New These parameters describe the problem you want to solve and your dataset. They should be specified and do not require tuning. -* **`algo`**: `Classification` or `Regression` +* **`algo`**: Type of decision tree, either `Classification` or `Regression`. -* **`numClasses`**: Number of classes (for `Classification` only) +* **`numClasses`**: Number of classes (for `Classification` only). * **`categoricalFeaturesInfo`**: Specifies which features are categorical and how many categorical values each of those features can take. This is given as a map from feature indices to feature arity (number of categories). Any features not in this map are treated as continuous. - * E.g., `Map(0 -> 2, 4 -> 10)` specifies that feature `0` is binary (taking values `0` or `1`) and that feature `4` has 10 categories (values `{0, 1, ..., 9}`). Note that feature indices are 0-based: features `0` and `4` are the 1st and 5th elements of an instance's feature vector. + * For example, `Map(0 -> 2, 4 -> 10)` specifies that feature `0` is binary (taking values `0` or `1`) and that feature `4` has 10 categories (values `{0, 1, ..., 9}`). Note that feature indices are 0-based: features `0` and `4` are the 1st and 5th elements of an instance's feature vector. * Note that you do not have to specify `categoricalFeaturesInfo`. The algorithm will still run and may get reasonable results. However, performance should be better if categorical features are properly designated. ### Stopping criteria @@ -136,7 +136,7 @@ When tuning these parameters, be careful to validate on held-out test data to av * **`maxDepth`**: Maximum depth of a tree. Deeper trees are more expressive (potentially allowing higher accuracy), but they are also more costly to train and are more likely to overfit. -* **`minInstancesPerNode`**: For a node to be split further, each of its children must receive at least this number of training instances. This is commonly used with [RandomForest](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) since those are often trained deeper than individual trees. +* **`minInstancesPerNode`**: For a node to be split further, each of its children must receive at least this number of training instances. This is commonly used with [RandomForest](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest$) since those are often trained deeper than individual trees. * **`minInfoGain`**: For a node to be split further, the split must improve at least this much (in terms of information gain). @@ -152,13 +152,13 @@ These parameters may be tuned. Be careful to validate on held-out test data whe * The default value is conservatively chosen to be 256 MB to allow the decision algorithm to work in most scenarios. Increasing `maxMemoryInMB` can lead to faster training (if the memory is available) by allowing fewer passes over the data. However, there may be decreasing returns as `maxMemoryInMB` grows since the amount of communication on each iteration can be proportional to `maxMemoryInMB`. * *Implementation details*: For faster processing, the decision tree algorithm collects statistics about groups of nodes to split (rather than 1 node at a time). The number of nodes which can be handled in one group is determined by the memory requirements (which vary per features). The `maxMemoryInMB` parameter specifies the memory limit in terms of megabytes which each worker can use for these statistics. -* **`subsamplingRate`**: Fraction of the training data used for learning the decision tree. This parameter is most relevant for training ensembles of trees (using [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) and [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees)), where it can be useful to subsample the original data. For training a single decision tree, this parameter is less useful since the number of training instances is generally not the main constraint. +* **`subsamplingRate`**: Fraction of the training data used for learning the decision tree. This parameter is most relevant for training ensembles of trees (using [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest$) and [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees)), where it can be useful to subsample the original data. For training a single decision tree, this parameter is less useful since the number of training instances is generally not the main constraint. * **`impurity`**: Impurity measure (discussed above) used to choose between candidate splits. This measure must match the `algo` parameter. ### Caching and checkpointing -MLlib 1.2 adds several features for scaling up to larger (deeper) trees and tree ensembles. When `maxDepth` is set to be large, it can be useful to turn on node ID caching and checkpointing. These parameters are also useful for [RandomForest](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) when `numTrees` is set to be large. +MLlib 1.2 adds several features for scaling up to larger (deeper) trees and tree ensembles. When `maxDepth` is set to be large, it can be useful to turn on node ID caching and checkpointing. These parameters are also useful for [RandomForest](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest$) when `numTrees` is set to be large. * **`useNodeIdCache`**: If this is set to true, the algorithm will avoid passing the current model (tree or trees) to executors on each iteration. * This can be useful with deep trees (speeding up computation on workers) and for large Random Forests (reducing communication on each iteration). @@ -194,137 +194,19 @@ maximum tree depth of 5. The test error is calculated to measure the algorithm a
    Refer to the [`DecisionTree` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) and [`DecisionTreeModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.model.DecisionTreeModel) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.tree.model.DecisionTreeModel -import org.apache.spark.mllib.util.MLUtils - -// Load and parse the data file. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") -// Split the data into training and test sets (30% held out for testing) -val splits = data.randomSplit(Array(0.7, 0.3)) -val (trainingData, testData) = (splits(0), splits(1)) - -// Train a DecisionTree model. -// Empty categoricalFeaturesInfo indicates all features are continuous. -val numClasses = 2 -val categoricalFeaturesInfo = Map[Int, Int]() -val impurity = "gini" -val maxDepth = 5 -val maxBins = 32 - -val model = DecisionTree.trainClassifier(trainingData, numClasses, categoricalFeaturesInfo, - impurity, maxDepth, maxBins) - -// Evaluate model on test instances and compute test error -val labelAndPreds = testData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count() -println("Test Error = " + testErr) -println("Learned classification tree model:\n" + model.toDebugString) - -// Save and load model -model.save(sc, "myModelPath") -val sameModel = DecisionTreeModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/DecisionTreeClassificationExample.scala %}
    Refer to the [`DecisionTree` Java docs](api/java/org/apache/spark/mllib/tree/DecisionTree.html) and [`DecisionTreeModel` Java docs](api/java/org/apache/spark/mllib/tree/model/DecisionTreeModel.html) for details on the API. -{% highlight java %} -import java.util.HashMap; -import scala.Tuple2; -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.LabeledPoint; -import org.apache.spark.mllib.tree.DecisionTree; -import org.apache.spark.mllib.tree.model.DecisionTreeModel; -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.SparkConf; - -SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); -JavaSparkContext sc = new JavaSparkContext(sparkConf); - -// Load and parse the data file. -String datapath = "data/mllib/sample_libsvm_data.txt"; -JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); -// Split the data into training and test sets (30% held out for testing) -JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); -JavaRDD trainingData = splits[0]; -JavaRDD testData = splits[1]; - -// Set parameters. -// Empty categoricalFeaturesInfo indicates all features are continuous. -Integer numClasses = 2; -Map categoricalFeaturesInfo = new HashMap(); -String impurity = "gini"; -Integer maxDepth = 5; -Integer maxBins = 32; - -// Train a DecisionTree model for classification. -final DecisionTreeModel model = DecisionTree.trainClassifier(trainingData, numClasses, - categoricalFeaturesInfo, impurity, maxDepth, maxBins); - -// Evaluate model on test instances and compute test error -JavaPairRDD predictionAndLabel = - testData.mapToPair(new PairFunction() { - @Override - public Tuple2 call(LabeledPoint p) { - return new Tuple2(model.predict(p.features()), p.label()); - } - }); -Double testErr = - 1.0 * predictionAndLabel.filter(new Function, Boolean>() { - @Override - public Boolean call(Tuple2 pl) { - return !pl._1().equals(pl._2()); - } - }).count() / testData.count(); -System.out.println("Test Error: " + testErr); -System.out.println("Learned classification tree model:\n" + model.toDebugString()); - -// Save and load model -model.save(sc.sc(), "myModelPath"); -DecisionTreeModel sameModel = DecisionTreeModel.load(sc.sc(), "myModelPath"); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaDecisionTreeClassificationExample.java %}
    Refer to the [`DecisionTree` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.tree.DecisionTree) and [`DecisionTreeModel` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.tree.DecisionTreeModel) for more details on the API. -{% highlight python %} -from pyspark.mllib.regression import LabeledPoint -from pyspark.mllib.tree import DecisionTree, DecisionTreeModel -from pyspark.mllib.util import MLUtils - -# Load and parse the data file into an RDD of LabeledPoint. -data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') -# Split the data into training and test sets (30% held out for testing) -(trainingData, testData) = data.randomSplit([0.7, 0.3]) - -# Train a DecisionTree model. -# Empty categoricalFeaturesInfo indicates all features are continuous. -model = DecisionTree.trainClassifier(trainingData, numClasses=2, categoricalFeaturesInfo={}, - impurity='gini', maxDepth=5, maxBins=32) - -# Evaluate model on test instances and compute test error -predictions = model.predict(testData.map(lambda x: x.features)) -labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) -testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(testData.count()) -print('Test Error = ' + str(testErr)) -print('Learned classification tree model:') -print(model.toDebugString()) - -# Save and load model -model.save(sc, "myModelPath") -sameModel = DecisionTreeModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example python/mllib/decision_tree_classification_example.py %}
    @@ -343,142 +225,19 @@ depth of 5. The Mean Squared Error (MSE) is computed at the end to evaluate
    Refer to the [`DecisionTree` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) and [`DecisionTreeModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.model.DecisionTreeModel) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.tree.model.DecisionTreeModel -import org.apache.spark.mllib.util.MLUtils - -// Load and parse the data file. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") -// Split the data into training and test sets (30% held out for testing) -val splits = data.randomSplit(Array(0.7, 0.3)) -val (trainingData, testData) = (splits(0), splits(1)) - -// Train a DecisionTree model. -// Empty categoricalFeaturesInfo indicates all features are continuous. -val categoricalFeaturesInfo = Map[Int, Int]() -val impurity = "variance" -val maxDepth = 5 -val maxBins = 32 - -val model = DecisionTree.trainRegressor(trainingData, categoricalFeaturesInfo, impurity, - maxDepth, maxBins) - -// Evaluate model on test instances and compute test error -val labelsAndPredictions = testData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() -println("Test Mean Squared Error = " + testMSE) -println("Learned regression tree model:\n" + model.toDebugString) - -// Save and load model -model.save(sc, "myModelPath") -val sameModel = DecisionTreeModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/DecisionTreeRegressionExample.scala %}
    Refer to the [`DecisionTree` Java docs](api/java/org/apache/spark/mllib/tree/DecisionTree.html) and [`DecisionTreeModel` Java docs](api/java/org/apache/spark/mllib/tree/model/DecisionTreeModel.html) for details on the API. -{% highlight java %} -import java.util.HashMap; -import scala.Tuple2; -import org.apache.spark.api.java.function.Function2; -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.LabeledPoint; -import org.apache.spark.mllib.tree.DecisionTree; -import org.apache.spark.mllib.tree.model.DecisionTreeModel; -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.SparkConf; - -SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); -JavaSparkContext sc = new JavaSparkContext(sparkConf); - -// Load and parse the data file. -String datapath = "data/mllib/sample_libsvm_data.txt"; -JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); -// Split the data into training and test sets (30% held out for testing) -JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); -JavaRDD trainingData = splits[0]; -JavaRDD testData = splits[1]; - -// Set parameters. -// Empty categoricalFeaturesInfo indicates all features are continuous. -Map categoricalFeaturesInfo = new HashMap(); -String impurity = "variance"; -Integer maxDepth = 5; -Integer maxBins = 32; - -// Train a DecisionTree model. -final DecisionTreeModel model = DecisionTree.trainRegressor(trainingData, - categoricalFeaturesInfo, impurity, maxDepth, maxBins); - -// Evaluate model on test instances and compute test error -JavaPairRDD predictionAndLabel = - testData.mapToPair(new PairFunction() { - @Override - public Tuple2 call(LabeledPoint p) { - return new Tuple2(model.predict(p.features()), p.label()); - } - }); -Double testMSE = - predictionAndLabel.map(new Function, Double>() { - @Override - public Double call(Tuple2 pl) { - Double diff = pl._1() - pl._2(); - return diff * diff; - } - }).reduce(new Function2() { - @Override - public Double call(Double a, Double b) { - return a + b; - } - }) / data.count(); -System.out.println("Test Mean Squared Error: " + testMSE); -System.out.println("Learned regression tree model:\n" + model.toDebugString()); - -// Save and load model -model.save(sc.sc(), "myModelPath"); -DecisionTreeModel sameModel = DecisionTreeModel.load(sc.sc(), "myModelPath"); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaDecisionTreeRegressionExample.java %}
    Refer to the [`DecisionTree` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.tree.DecisionTree) and [`DecisionTreeModel` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.tree.DecisionTreeModel) for more details on the API. -{% highlight python %} -from pyspark.mllib.regression import LabeledPoint -from pyspark.mllib.tree import DecisionTree, DecisionTreeModel -from pyspark.mllib.util import MLUtils - -# Load and parse the data file into an RDD of LabeledPoint. -data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') -# Split the data into training and test sets (30% held out for testing) -(trainingData, testData) = data.randomSplit([0.7, 0.3]) - -# Train a DecisionTree model. -# Empty categoricalFeaturesInfo indicates all features are continuous. -model = DecisionTree.trainRegressor(trainingData, categoricalFeaturesInfo={}, - impurity='variance', maxDepth=5, maxBins=32) - -# Evaluate model on test instances and compute test error -predictions = model.predict(testData.map(lambda x: x.features)) -labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) -testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / float(testData.count()) -print('Test Mean Squared Error = ' + str(testMSE)) -print('Learned regression tree model:') -print(model.toDebugString()) - -# Save and load model -model.save(sc, "myModelPath") -sameModel = DecisionTreeModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example python/mllib/decision_tree_regression_example.py %}
    diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index ac3526908a9f..a72680d52a26 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -1,7 +1,7 @@ --- layout: global -title: Dimensionality Reduction - MLlib -displayTitle: MLlib - Dimensionality Reduction +title: Dimensionality Reduction - RDD-based API +displayTitle: Dimensionality Reduction - RDD-based API --- * Table of contents @@ -11,7 +11,7 @@ displayTitle: MLlib - Dimensionality Reduction of reducing the number of variables under consideration. It can be used to extract latent features from raw and noisy features or compress data while maintaining the structure. -MLlib provides support for dimensionality reduction on the RowMatrix class. +`spark.mllib` provides support for dimensionality reduction on the RowMatrix class. ## Singular value decomposition (SVD) @@ -57,26 +57,14 @@ passes, $O(n)$ storage on each executor, and $O(n k)$ storage on the driver. ### SVD Example -MLlib provides SVD functionality to row-oriented matrices, provided in the +`spark.mllib` provides SVD functionality to row-oriented matrices, provided in the RowMatrix class.
    Refer to the [`SingularValueDecomposition` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.SingularValueDecomposition) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.linalg.Matrix -import org.apache.spark.mllib.linalg.distributed.RowMatrix -import org.apache.spark.mllib.linalg.SingularValueDecomposition - -val mat: RowMatrix = ... - -// Compute the top 20 singular values and corresponding singular vectors. -val svd: SingularValueDecomposition[RowMatrix, Matrix] = mat.computeSVD(20, computeU = true) -val U: RowMatrix = svd.U // The U factor is a RowMatrix. -val s: Vector = svd.s // The singular values are stored in a local dense vector. -val V: Matrix = svd.V // The V factor is a local dense matrix. -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/SVDExample.scala %} The same code applies to `IndexedRowMatrix` if `U` is defined as an `IndexedRowMatrix`. @@ -84,53 +72,18 @@ The same code applies to `IndexedRowMatrix` if `U` is defined as an
    Refer to the [`SingularValueDecomposition` Java docs](api/java/org/apache/spark/mllib/linalg/SingularValueDecomposition.html) for details on the API. -{% highlight java %} -import java.util.LinkedList; - -import org.apache.spark.api.java.*; -import org.apache.spark.mllib.linalg.distributed.RowMatrix; -import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.linalg.SingularValueDecomposition; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.rdd.RDD; -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; - -public class SVD { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("SVD Example"); - SparkContext sc = new SparkContext(conf); - - double[][] array = ... - LinkedList rowsList = new LinkedList(); - for (int i = 0; i < array.length; i++) { - Vector currentRow = Vectors.dense(array[i]); - rowsList.add(currentRow); - } - JavaRDD rows = JavaSparkContext.fromSparkContext(sc).parallelize(rowsList); - - // Create a RowMatrix from JavaRDD. - RowMatrix mat = new RowMatrix(rows.rdd()); - - // Compute the top 4 singular values and corresponding singular vectors. - SingularValueDecomposition svd = mat.computeSVD(4, true, 1.0E-9d); - RowMatrix U = svd.U(); - Vector s = svd.s(); - Matrix V = svd.V(); - } -} -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaSVDExample.java %} The same code applies to `IndexedRowMatrix` if `U` is defined as an `IndexedRowMatrix`. +
    +
    +Refer to the [`SingularValueDecomposition` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.linalg.distributed.SingularValueDecomposition) for details on the API. -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. +{% include_example python/mllib/svd_example.py %} +The same code applies to `IndexedRowMatrix` if `U` is defined as an +`IndexedRowMatrix`.
    @@ -141,7 +94,7 @@ statistical method to find a rotation such that the first coordinate has the lar possible, and each succeeding coordinate in turn has the largest variance possible. The columns of the rotation matrix are called principal components. PCA is used widely in dimensionality reduction. -MLlib supports PCA for tall-and-skinny matrices stored in row-oriented format and any Vectors. +`spark.mllib` supports PCA for tall-and-skinny matrices stored in row-oriented format and any Vectors.
    @@ -151,87 +104,36 @@ and use them to project the vectors into a low-dimensional space. Refer to the [`RowMatrix` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.linalg.Matrix -import org.apache.spark.mllib.linalg.distributed.RowMatrix - -val mat: RowMatrix = ... - -// Compute the top 10 principal components. -val pc: Matrix = mat.computePrincipalComponents(10) // Principal components are stored in a local dense matrix. - -// Project the rows to the linear space spanned by the top 10 principal components. -val projected: RowMatrix = mat.multiply(pc) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/PCAOnRowMatrixExample.scala %} The following code demonstrates how to compute principal components on source vectors and use them to project the vectors into a low-dimensional space while keeping associated labels: Refer to the [`PCA` Scala docs](api/scala/index.html#org.apache.spark.mllib.feature.PCA) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.feature.PCA +{% include_example scala/org/apache/spark/examples/mllib/PCAOnSourceVectorExample.scala %} + +
    -val data: RDD[LabeledPoint] = ... +
    -// Compute the top 10 principal components. -val pca = new PCA(10).fit(data.map(_.features)) +The following code demonstrates how to compute principal components on a `RowMatrix` +and use them to project the vectors into a low-dimensional space. -// Project vectors to the linear space spanned by the top 10 principal components, keeping the label -val projected = data.map(p => p.copy(features = pca.transform(p.features))) -{% endhighlight %} +Refer to the [`RowMatrix` Java docs](api/java/org/apache/spark/mllib/linalg/distributed/RowMatrix.html) for details on the API. + +{% include_example java/org/apache/spark/examples/mllib/JavaPCAExample.java %}
    -
    +
    The following code demonstrates how to compute principal components on a `RowMatrix` and use them to project the vectors into a low-dimensional space. -The number of columns should be small, e.g, less than 1000. -Refer to the [`RowMatrix` Java docs](api/java/org/apache/spark/mllib/linalg/distributed/RowMatrix.html) for details on the API. +Refer to the [`RowMatrix` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.linalg.distributed.RowMatrix) for details on the API. -{% highlight java %} -import java.util.LinkedList; - -import org.apache.spark.api.java.*; -import org.apache.spark.mllib.linalg.distributed.RowMatrix; -import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.rdd.RDD; -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; - -public class PCA { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("PCA Example"); - SparkContext sc = new SparkContext(conf); - - double[][] array = ... - LinkedList rowsList = new LinkedList(); - for (int i = 0; i < array.length; i++) { - Vector currentRow = Vectors.dense(array[i]); - rowsList.add(currentRow); - } - JavaRDD rows = JavaSparkContext.fromSparkContext(sc).parallelize(rowsList); - - // Create a RowMatrix from JavaRDD. - RowMatrix mat = new RowMatrix(rows.rdd()); - - // Compute the top 3 principal components. - Matrix pc = mat.computePrincipalComponents(3); - RowMatrix projected = mat.multiply(pc); - } -} -{% endhighlight %} +{% include_example python/mllib/pca_rowmatrix_example.py %}
    - -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. diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md index fc587298f7d2..e1984b6c8d5a 100644 --- a/docs/mllib-ensembles.md +++ b/docs/mllib-ensembles.md @@ -1,7 +1,7 @@ --- layout: global -title: Ensembles - MLlib -displayTitle: MLlib - Ensembles +title: Ensembles - RDD-based API +displayTitle: Ensembles - RDD-based API --- * Table of contents @@ -9,7 +9,7 @@ displayTitle: MLlib - Ensembles An [ensemble method](http://en.wikipedia.org/wiki/Ensemble_learning) is a learning algorithm which creates a model composed of a set of other base models. -MLlib supports two major ensemble algorithms: [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees) and [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest). +`spark.mllib` supports two major ensemble algorithms: [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees) and [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest$). Both use [decision trees](mllib-decision-tree.html) as their base models. ## Gradient-Boosted Trees vs. Random Forests @@ -33,9 +33,9 @@ Like decision trees, random forests handle categorical features, extend to the multiclass classification setting, do not require feature scaling, and are able to capture non-linearities and feature interactions. -MLlib supports random forests for binary and multiclass classification and for regression, +`spark.mllib` supports random forests for binary and multiclass classification and for regression, using both continuous and categorical features. -MLlib implements random forests using the existing [decision tree](mllib-decision-tree.html) +`spark.mllib` implements random forests using the existing [decision tree](mllib-decision-tree.html) implementation. Please see the decision tree guide for more information on trees. ### Basic algorithm @@ -96,146 +96,21 @@ The test error is calculated to measure the algorithm accuracy.
    -Refer to the [`RandomForest` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) and [`RandomForestModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.model.RandomForestModel) for details on the API. - -{% highlight scala %} -import org.apache.spark.mllib.tree.RandomForest -import org.apache.spark.mllib.tree.model.RandomForestModel -import org.apache.spark.mllib.util.MLUtils - -// Load and parse the data file. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") -// Split the data into training and test sets (30% held out for testing) -val splits = data.randomSplit(Array(0.7, 0.3)) -val (trainingData, testData) = (splits(0), splits(1)) - -// Train a RandomForest model. -// Empty categoricalFeaturesInfo indicates all features are continuous. -val numClasses = 2 -val categoricalFeaturesInfo = Map[Int, Int]() -val numTrees = 3 // Use more in practice. -val featureSubsetStrategy = "auto" // Let the algorithm choose. -val impurity = "gini" -val maxDepth = 4 -val maxBins = 32 - -val model = RandomForest.trainClassifier(trainingData, numClasses, categoricalFeaturesInfo, - numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins) - -// Evaluate model on test instances and compute test error -val labelAndPreds = testData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count() -println("Test Error = " + testErr) -println("Learned classification forest model:\n" + model.toDebugString) - -// Save and load model -model.save(sc, "myModelPath") -val sameModel = RandomForestModel.load(sc, "myModelPath") -{% endhighlight %} +Refer to the [`RandomForest` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest$) and [`RandomForestModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.model.RandomForestModel) for details on the API. + +{% include_example scala/org/apache/spark/examples/mllib/RandomForestClassificationExample.scala %}
    Refer to the [`RandomForest` Java docs](api/java/org/apache/spark/mllib/tree/RandomForest.html) and [`RandomForestModel` Java docs](api/java/org/apache/spark/mllib/tree/model/RandomForestModel.html) for details on the API. -{% highlight java %} -import scala.Tuple2; -import java.util.HashMap; -import org.apache.spark.SparkConf; -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.LabeledPoint; -import org.apache.spark.mllib.tree.RandomForest; -import org.apache.spark.mllib.tree.model.RandomForestModel; -import org.apache.spark.mllib.util.MLUtils; - -SparkConf sparkConf = new SparkConf().setAppName("JavaRandomForestClassification"); -JavaSparkContext sc = new JavaSparkContext(sparkConf); - -// Load and parse the data file. -String datapath = "data/mllib/sample_libsvm_data.txt"; -JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); -// Split the data into training and test sets (30% held out for testing) -JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); -JavaRDD trainingData = splits[0]; -JavaRDD testData = splits[1]; - -// Train a RandomForest model. -// Empty categoricalFeaturesInfo indicates all features are continuous. -Integer numClasses = 2; -HashMap categoricalFeaturesInfo = new HashMap(); -Integer numTrees = 3; // Use more in practice. -String featureSubsetStrategy = "auto"; // Let the algorithm choose. -String impurity = "gini"; -Integer maxDepth = 5; -Integer maxBins = 32; -Integer seed = 12345; - -final RandomForestModel model = RandomForest.trainClassifier(trainingData, numClasses, - categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, - seed); - -// Evaluate model on test instances and compute test error -JavaPairRDD predictionAndLabel = - testData.mapToPair(new PairFunction() { - @Override - public Tuple2 call(LabeledPoint p) { - return new Tuple2(model.predict(p.features()), p.label()); - } - }); -Double testErr = - 1.0 * predictionAndLabel.filter(new Function, Boolean>() { - @Override - public Boolean call(Tuple2 pl) { - return !pl._1().equals(pl._2()); - } - }).count() / testData.count(); -System.out.println("Test Error: " + testErr); -System.out.println("Learned classification forest model:\n" + model.toDebugString()); - -// Save and load model -model.save(sc.sc(), "myModelPath"); -RandomForestModel sameModel = RandomForestModel.load(sc.sc(), "myModelPath"); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java %}
    Refer to the [`RandomForest` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.tree.RandomForest) and [`RandomForest` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.tree.RandomForestModel) for more details on the API. -{% highlight python %} -from pyspark.mllib.tree import RandomForest, RandomForestModel -from pyspark.mllib.util import MLUtils - -# Load and parse the data file into an RDD of LabeledPoint. -data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') -# Split the data into training and test sets (30% held out for testing) -(trainingData, testData) = data.randomSplit([0.7, 0.3]) - -# Train a RandomForest model. -# Empty categoricalFeaturesInfo indicates all features are continuous. -# Note: Use larger numTrees in practice. -# Setting featureSubsetStrategy="auto" lets the algorithm choose. -model = RandomForest.trainClassifier(trainingData, numClasses=2, categoricalFeaturesInfo={}, - numTrees=3, featureSubsetStrategy="auto", - impurity='gini', maxDepth=4, maxBins=32) - -# Evaluate model on test instances and compute test error -predictions = model.predict(testData.map(lambda x: x.features)) -labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) -testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(testData.count()) -print('Test Error = ' + str(testErr)) -print('Learned classification forest model:') -print(model.toDebugString()) - -# Save and load model -model.save(sc, "myModelPath") -sameModel = RandomForestModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example python/mllib/random_forest_classification_example.py %}
    @@ -252,149 +127,21 @@ The Mean Squared Error (MSE) is computed at the end to evaluate
    -Refer to the [`RandomForest` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) and [`RandomForestModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.model.RandomForestModel) for details on the API. - -{% highlight scala %} -import org.apache.spark.mllib.tree.RandomForest -import org.apache.spark.mllib.tree.model.RandomForestModel -import org.apache.spark.mllib.util.MLUtils - -// Load and parse the data file. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") -// Split the data into training and test sets (30% held out for testing) -val splits = data.randomSplit(Array(0.7, 0.3)) -val (trainingData, testData) = (splits(0), splits(1)) - -// Train a RandomForest model. -// Empty categoricalFeaturesInfo indicates all features are continuous. -val numClasses = 2 -val categoricalFeaturesInfo = Map[Int, Int]() -val numTrees = 3 // Use more in practice. -val featureSubsetStrategy = "auto" // Let the algorithm choose. -val impurity = "variance" -val maxDepth = 4 -val maxBins = 32 - -val model = RandomForest.trainRegressor(trainingData, categoricalFeaturesInfo, - numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins) - -// Evaluate model on test instances and compute test error -val labelsAndPredictions = testData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() -println("Test Mean Squared Error = " + testMSE) -println("Learned regression forest model:\n" + model.toDebugString) - -// Save and load model -model.save(sc, "myModelPath") -val sameModel = RandomForestModel.load(sc, "myModelPath") -{% endhighlight %} +Refer to the [`RandomForest` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest$) and [`RandomForestModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.model.RandomForestModel) for details on the API. + +{% include_example scala/org/apache/spark/examples/mllib/RandomForestRegressionExample.scala %}
    Refer to the [`RandomForest` Java docs](api/java/org/apache/spark/mllib/tree/RandomForest.html) and [`RandomForestModel` Java docs](api/java/org/apache/spark/mllib/tree/model/RandomForestModel.html) for details on the API. -{% highlight java %} -import java.util.HashMap; -import scala.Tuple2; -import org.apache.spark.api.java.function.Function2; -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.LabeledPoint; -import org.apache.spark.mllib.tree.RandomForest; -import org.apache.spark.mllib.tree.model.RandomForestModel; -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.SparkConf; - -SparkConf sparkConf = new SparkConf().setAppName("JavaRandomForest"); -JavaSparkContext sc = new JavaSparkContext(sparkConf); - -// Load and parse the data file. -String datapath = "data/mllib/sample_libsvm_data.txt"; -JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); -// Split the data into training and test sets (30% held out for testing) -JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); -JavaRDD trainingData = splits[0]; -JavaRDD testData = splits[1]; - -// Set parameters. -// Empty categoricalFeaturesInfo indicates all features are continuous. -Map categoricalFeaturesInfo = new HashMap(); -String impurity = "variance"; -Integer maxDepth = 4; -Integer maxBins = 32; - -// Train a RandomForest model. -final RandomForestModel model = RandomForest.trainRegressor(trainingData, - categoricalFeaturesInfo, impurity, maxDepth, maxBins); - -// Evaluate model on test instances and compute test error -JavaPairRDD predictionAndLabel = - testData.mapToPair(new PairFunction() { - @Override - public Tuple2 call(LabeledPoint p) { - return new Tuple2(model.predict(p.features()), p.label()); - } - }); -Double testMSE = - predictionAndLabel.map(new Function, Double>() { - @Override - public Double call(Tuple2 pl) { - Double diff = pl._1() - pl._2(); - return diff * diff; - } - }).reduce(new Function2() { - @Override - public Double call(Double a, Double b) { - return a + b; - } - }) / testData.count(); -System.out.println("Test Mean Squared Error: " + testMSE); -System.out.println("Learned regression forest model:\n" + model.toDebugString()); - -// Save and load model -model.save(sc.sc(), "myModelPath"); -RandomForestModel sameModel = RandomForestModel.load(sc.sc(), "myModelPath"); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaRandomForestRegressionExample.java %}
    Refer to the [`RandomForest` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.tree.RandomForest) and [`RandomForest` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.tree.RandomForestModel) for more details on the API. -{% highlight python %} -from pyspark.mllib.tree import RandomForest, RandomForestModel -from pyspark.mllib.util import MLUtils - -# Load and parse the data file into an RDD of LabeledPoint. -data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') -# Split the data into training and test sets (30% held out for testing) -(trainingData, testData) = data.randomSplit([0.7, 0.3]) - -# Train a RandomForest model. -# Empty categoricalFeaturesInfo indicates all features are continuous. -# Note: Use larger numTrees in practice. -# Setting featureSubsetStrategy="auto" lets the algorithm choose. -model = RandomForest.trainRegressor(trainingData, categoricalFeaturesInfo={}, - numTrees=3, featureSubsetStrategy="auto", - impurity='variance', maxDepth=4, maxBins=32) - -# Evaluate model on test instances and compute test error -predictions = model.predict(testData.map(lambda x: x.features)) -labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) -testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / float(testData.count()) -print('Test Mean Squared Error = ' + str(testMSE)) -print('Learned regression forest model:') -print(model.toDebugString()) - -# Save and load model -model.save(sc, "myModelPath") -sameModel = RandomForestModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example python/mllib/random_forest_regression_example.py %}
    @@ -408,9 +155,9 @@ Like decision trees, GBTs handle categorical features, extend to the multiclass classification setting, do not require feature scaling, and are able to capture non-linearities and feature interactions. -MLlib supports GBTs for binary classification and for regression, +`spark.mllib` supports GBTs for binary classification and for regression, using both continuous and categorical features. -MLlib implements GBTs using the existing [decision tree](mllib-decision-tree.html) implementation. Please see the decision tree guide for more information on trees. +`spark.mllib` implements GBTs using the existing [decision tree](mllib-decision-tree.html) implementation. Please see the decision tree guide for more information on trees. *Note*: GBTs do not yet support multiclass classification. For multiclass problems, please use [decision trees](mllib-decision-tree.html) or [Random Forests](mllib-ensembles.html#Random-Forest). @@ -424,7 +171,7 @@ The specific mechanism for re-labeling instances is defined by a loss function ( #### Losses -The table below lists the losses currently supported by GBTs in MLlib. +The table below lists the losses currently supported by GBTs in `spark.mllib`. Note that each loss is applicable to one of classification or regression, not both. Notation: $N$ = number of instances. $y_i$ = label of instance $i$. $x_i$ = features of instance $i$. $F(x_i)$ = model's predicted label for instance $i$. @@ -492,141 +239,19 @@ The test error is calculated to measure the algorithm accuracy.
    Refer to the [`GradientBoostedTrees` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees) and [`GradientBoostedTreesModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.model.GradientBoostedTreesModel) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.tree.GradientBoostedTrees -import org.apache.spark.mllib.tree.configuration.BoostingStrategy -import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel -import org.apache.spark.mllib.util.MLUtils - -// Load and parse the data file. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") -// Split the data into training and test sets (30% held out for testing) -val splits = data.randomSplit(Array(0.7, 0.3)) -val (trainingData, testData) = (splits(0), splits(1)) - -// Train a GradientBoostedTrees model. -// The defaultParams for Classification use LogLoss by default. -val boostingStrategy = BoostingStrategy.defaultParams("Classification") -boostingStrategy.numIterations = 3 // Note: Use more iterations in practice. -boostingStrategy.treeStrategy.numClasses = 2 -boostingStrategy.treeStrategy.maxDepth = 5 -// Empty categoricalFeaturesInfo indicates all features are continuous. -boostingStrategy.treeStrategy.categoricalFeaturesInfo = Map[Int, Int]() - -val model = GradientBoostedTrees.train(trainingData, boostingStrategy) - -// Evaluate model on test instances and compute test error -val labelAndPreds = testData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count() -println("Test Error = " + testErr) -println("Learned classification GBT model:\n" + model.toDebugString) - -// Save and load model -model.save(sc, "myModelPath") -val sameModel = GradientBoostedTreesModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/GradientBoostingClassificationExample.scala %}
    Refer to the [`GradientBoostedTrees` Java docs](api/java/org/apache/spark/mllib/tree/GradientBoostedTrees.html) and [`GradientBoostedTreesModel` Java docs](api/java/org/apache/spark/mllib/tree/model/GradientBoostedTreesModel.html) for details on the API. -{% highlight java %} -import scala.Tuple2; -import java.util.HashMap; -import java.util.Map; -import org.apache.spark.SparkConf; -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.LabeledPoint; -import org.apache.spark.mllib.tree.GradientBoostedTrees; -import org.apache.spark.mllib.tree.configuration.BoostingStrategy; -import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel; -import org.apache.spark.mllib.util.MLUtils; - -SparkConf sparkConf = new SparkConf().setAppName("JavaGradientBoostedTrees"); -JavaSparkContext sc = new JavaSparkContext(sparkConf); - -// Load and parse the data file. -String datapath = "data/mllib/sample_libsvm_data.txt"; -JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); -// Split the data into training and test sets (30% held out for testing) -JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); -JavaRDD trainingData = splits[0]; -JavaRDD testData = splits[1]; - -// Train a GradientBoostedTrees model. -// The defaultParams for Classification use LogLoss by default. -BoostingStrategy boostingStrategy = BoostingStrategy.defaultParams("Classification"); -boostingStrategy.setNumIterations(3); // Note: Use more iterations in practice. -boostingStrategy.getTreeStrategy().setNumClassesForClassification(2); -boostingStrategy.getTreeStrategy().setMaxDepth(5); -// Empty categoricalFeaturesInfo indicates all features are continuous. -Map categoricalFeaturesInfo = new HashMap(); -boostingStrategy.treeStrategy().setCategoricalFeaturesInfo(categoricalFeaturesInfo); - -final GradientBoostedTreesModel model = - GradientBoostedTrees.train(trainingData, boostingStrategy); - -// Evaluate model on test instances and compute test error -JavaPairRDD predictionAndLabel = - testData.mapToPair(new PairFunction() { - @Override - public Tuple2 call(LabeledPoint p) { - return new Tuple2(model.predict(p.features()), p.label()); - } - }); -Double testErr = - 1.0 * predictionAndLabel.filter(new Function, Boolean>() { - @Override - public Boolean call(Tuple2 pl) { - return !pl._1().equals(pl._2()); - } - }).count() / testData.count(); -System.out.println("Test Error: " + testErr); -System.out.println("Learned classification GBT model:\n" + model.toDebugString()); - -// Save and load model -model.save(sc.sc(), "myModelPath"); -GradientBoostedTreesModel sameModel = GradientBoostedTreesModel.load(sc.sc(), "myModelPath"); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaGradientBoostingClassificationExample.java %}
    Refer to the [`GradientBoostedTrees` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.tree.GradientBoostedTrees) and [`GradientBoostedTreesModel` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.tree.GradientBoostedTreesModel) for more details on the API. -{% highlight python %} -from pyspark.mllib.tree import GradientBoostedTrees, GradientBoostedTreesModel -from pyspark.mllib.util import MLUtils - -# Load and parse the data file. -data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") -# Split the data into training and test sets (30% held out for testing) -(trainingData, testData) = data.randomSplit([0.7, 0.3]) - -# Train a GradientBoostedTrees model. -# Notes: (a) Empty categoricalFeaturesInfo indicates all features are continuous. -# (b) Use more iterations in practice. -model = GradientBoostedTrees.trainClassifier(trainingData, - categoricalFeaturesInfo={}, numIterations=3) - -# Evaluate model on test instances and compute test error -predictions = model.predict(testData.map(lambda x: x.features)) -labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) -testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(testData.count()) -print('Test Error = ' + str(testErr)) -print('Learned classification GBT model:') -print(model.toDebugString()) - -# Save and load model -model.save(sc, "myModelPath") -sameModel = GradientBoostedTreesModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example python/mllib/gradient_boosting_classification_example.py %}
    @@ -645,146 +270,19 @@ The Mean Squared Error (MSE) is computed at the end to evaluate
    Refer to the [`GradientBoostedTrees` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees) and [`GradientBoostedTreesModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.tree.model.GradientBoostedTreesModel) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.tree.GradientBoostedTrees -import org.apache.spark.mllib.tree.configuration.BoostingStrategy -import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel -import org.apache.spark.mllib.util.MLUtils - -// Load and parse the data file. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") -// Split the data into training and test sets (30% held out for testing) -val splits = data.randomSplit(Array(0.7, 0.3)) -val (trainingData, testData) = (splits(0), splits(1)) - -// Train a GradientBoostedTrees model. -// The defaultParams for Regression use SquaredError by default. -val boostingStrategy = BoostingStrategy.defaultParams("Regression") -boostingStrategy.numIterations = 3 // Note: Use more iterations in practice. -boostingStrategy.treeStrategy.maxDepth = 5 -// Empty categoricalFeaturesInfo indicates all features are continuous. -boostingStrategy.treeStrategy.categoricalFeaturesInfo = Map[Int, Int]() - -val model = GradientBoostedTrees.train(trainingData, boostingStrategy) - -// Evaluate model on test instances and compute test error -val labelsAndPredictions = testData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() -println("Test Mean Squared Error = " + testMSE) -println("Learned regression GBT model:\n" + model.toDebugString) - -// Save and load model -model.save(sc, "myModelPath") -val sameModel = GradientBoostedTreesModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/GradientBoostingRegressionExample.scala %}
    Refer to the [`GradientBoostedTrees` Java docs](api/java/org/apache/spark/mllib/tree/GradientBoostedTrees.html) and [`GradientBoostedTreesModel` Java docs](api/java/org/apache/spark/mllib/tree/model/GradientBoostedTreesModel.html) for details on the API. -{% highlight java %} -import scala.Tuple2; -import java.util.HashMap; -import java.util.Map; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.function.Function2; -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.LabeledPoint; -import org.apache.spark.mllib.tree.GradientBoostedTrees; -import org.apache.spark.mllib.tree.configuration.BoostingStrategy; -import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel; -import org.apache.spark.mllib.util.MLUtils; - -SparkConf sparkConf = new SparkConf().setAppName("JavaGradientBoostedTrees"); -JavaSparkContext sc = new JavaSparkContext(sparkConf); - -// Load and parse the data file. -String datapath = "data/mllib/sample_libsvm_data.txt"; -JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); -// Split the data into training and test sets (30% held out for testing) -JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); -JavaRDD trainingData = splits[0]; -JavaRDD testData = splits[1]; - -// Train a GradientBoostedTrees model. -// The defaultParams for Regression use SquaredError by default. -BoostingStrategy boostingStrategy = BoostingStrategy.defaultParams("Regression"); -boostingStrategy.setNumIterations(3); // Note: Use more iterations in practice. -boostingStrategy.getTreeStrategy().setMaxDepth(5); -// Empty categoricalFeaturesInfo indicates all features are continuous. -Map categoricalFeaturesInfo = new HashMap(); -boostingStrategy.treeStrategy().setCategoricalFeaturesInfo(categoricalFeaturesInfo); - -final GradientBoostedTreesModel model = - GradientBoostedTrees.train(trainingData, boostingStrategy); - -// Evaluate model on test instances and compute test error -JavaPairRDD predictionAndLabel = - testData.mapToPair(new PairFunction() { - @Override - public Tuple2 call(LabeledPoint p) { - return new Tuple2(model.predict(p.features()), p.label()); - } - }); -Double testMSE = - predictionAndLabel.map(new Function, Double>() { - @Override - public Double call(Tuple2 pl) { - Double diff = pl._1() - pl._2(); - return diff * diff; - } - }).reduce(new Function2() { - @Override - public Double call(Double a, Double b) { - return a + b; - } - }) / data.count(); -System.out.println("Test Mean Squared Error: " + testMSE); -System.out.println("Learned regression GBT model:\n" + model.toDebugString()); - -// Save and load model -model.save(sc.sc(), "myModelPath"); -GradientBoostedTreesModel sameModel = GradientBoostedTreesModel.load(sc.sc(), "myModelPath"); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaGradientBoostingRegressionExample.java %}
    Refer to the [`GradientBoostedTrees` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.tree.GradientBoostedTrees) and [`GradientBoostedTreesModel` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.tree.GradientBoostedTreesModel) for more details on the API. -{% highlight python %} -from pyspark.mllib.tree import GradientBoostedTrees, GradientBoostedTreesModel -from pyspark.mllib.util import MLUtils - -# Load and parse the data file. -data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") -# Split the data into training and test sets (30% held out for testing) -(trainingData, testData) = data.randomSplit([0.7, 0.3]) - -# Train a GradientBoostedTrees model. -# Notes: (a) Empty categoricalFeaturesInfo indicates all features are continuous. -# (b) Use more iterations in practice. -model = GradientBoostedTrees.trainRegressor(trainingData, - categoricalFeaturesInfo={}, numIterations=3) - -# Evaluate model on test instances and compute test error -predictions = model.predict(testData.map(lambda x: x.features)) -labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) -testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / float(testData.count()) -print('Test Mean Squared Error = ' + str(testMSE)) -print('Learned regression GBT model:') -print(model.toDebugString()) - -# Save and load model -model.save(sc, "myModelPath") -sameModel = GradientBoostedTreesModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example python/mllib/gradient_boosting_regression_example.py %}
    diff --git a/docs/mllib-evaluation-metrics.md b/docs/mllib-evaluation-metrics.md index f73eff637dc3..7f277543d2e9 100644 --- a/docs/mllib-evaluation-metrics.md +++ b/docs/mllib-evaluation-metrics.md @@ -1,20 +1,20 @@ --- layout: global -title: Evaluation Metrics - MLlib -displayTitle: MLlib - Evaluation Metrics +title: Evaluation Metrics - RDD-based API +displayTitle: Evaluation Metrics - RDD-based API --- * Table of contents {:toc} -Spark's MLlib comes with a number of machine learning algorithms that can be used to learn from and make predictions +`spark.mllib` comes with a number of machine learning algorithms that can be used to learn from and make predictions on data. When these algorithms are applied to build machine learning models, there is a need to evaluate the performance -of the model on some criteria, which depends on the application and its requirements. Spark's MLlib also provides a +of the model on some criteria, which depends on the application and its requirements. `spark.mllib` also provides a suite of metrics for the purpose of evaluating the performance of machine learning models. Specific machine learning algorithms fall under broader types of machine learning applications like classification, regression, clustering, etc. Each of these types have well established metrics for performance evaluation and those -metrics that are currently available in Spark's MLlib are detailed in this section. +metrics that are currently available in `spark.mllib` are detailed in this section. ## Classification model evaluation @@ -67,7 +67,7 @@ plots (recall, false positive rate) points. - Precision (Postive Predictive Value) + Precision (Positive Predictive Value) $PPV=\frac{TP}{TP + FP}$ @@ -104,214 +104,21 @@ data, and evaluate the performance of the algorithm by several binary evaluation
    Refer to the [`LogisticRegressionWithLBFGS` Scala docs](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS) and [`BinaryClassificationMetrics` Scala docs](api/scala/index.html#org.apache.spark.mllib.evaluation.BinaryClassificationMetrics) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.util.MLUtils - -// Load training data in LIBSVM format -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_binary_classification_data.txt") - -// Split data into training (60%) and test (40%) -val Array(training, test) = data.randomSplit(Array(0.6, 0.4), seed = 11L) -training.cache() - -// Run training algorithm to build the model -val model = new LogisticRegressionWithLBFGS() - .setNumClasses(2) - .run(training) - -// Clear the prediction threshold so the model will return probabilities -model.clearThreshold - -// Compute raw scores on the test set -val predictionAndLabels = test.map { case LabeledPoint(label, features) => - val prediction = model.predict(features) - (prediction, label) -} - -// Instantiate metrics object -val metrics = new BinaryClassificationMetrics(predictionAndLabels) - -// Precision by threshold -val precision = metrics.precisionByThreshold -precision.foreach { case (t, p) => - println(s"Threshold: $t, Precision: $p") -} - -// Recall by threshold -val recall = metrics.recallByThreshold -recall.foreach { case (t, r) => - println(s"Threshold: $t, Recall: $r") -} - -// Precision-Recall Curve -val PRC = metrics.pr - -// F-measure -val f1Score = metrics.fMeasureByThreshold -f1Score.foreach { case (t, f) => - println(s"Threshold: $t, F-score: $f, Beta = 1") -} - -val beta = 0.5 -val fScore = metrics.fMeasureByThreshold(beta) -f1Score.foreach { case (t, f) => - println(s"Threshold: $t, F-score: $f, Beta = 0.5") -} - -// AUPRC -val auPRC = metrics.areaUnderPR -println("Area under precision-recall curve = " + auPRC) - -// Compute thresholds used in ROC and PR curves -val thresholds = precision.map(_._1) - -// ROC Curve -val roc = metrics.roc - -// AUROC -val auROC = metrics.areaUnderROC -println("Area under ROC = " + auROC) - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala %}
    Refer to the [`LogisticRegressionModel` Java docs](api/java/org/apache/spark/mllib/classification/LogisticRegressionModel.html) and [`LogisticRegressionWithLBFGS` Java docs](api/java/org/apache/spark/mllib/classification/LogisticRegressionWithLBFGS.html) for details on the API. -{% highlight java %} -import scala.Tuple2; - -import org.apache.spark.api.java.*; -import org.apache.spark.rdd.RDD; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.classification.LogisticRegressionModel; -import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS; -import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; - -public class BinaryClassification { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("Binary Classification Metrics"); - SparkContext sc = new SparkContext(conf); - String path = "data/mllib/sample_binary_classification_data.txt"; - JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); - - // Split initial RDD into two... [60% training data, 40% testing data]. - JavaRDD[] splits = data.randomSplit(new double[] {0.6, 0.4}, 11L); - JavaRDD training = splits[0].cache(); - JavaRDD test = splits[1]; - - // Run training algorithm to build the model. - final LogisticRegressionModel model = new LogisticRegressionWithLBFGS() - .setNumClasses(2) - .run(training.rdd()); - - // Clear the prediction threshold so the model will return probabilities - model.clearThreshold(); - - // Compute raw scores on the test set. - JavaRDD> predictionAndLabels = test.map( - new Function>() { - public Tuple2 call(LabeledPoint p) { - Double prediction = model.predict(p.features()); - return new Tuple2(prediction, p.label()); - } - } - ); - - // Get evaluation metrics. - BinaryClassificationMetrics metrics = new BinaryClassificationMetrics(predictionAndLabels.rdd()); - - // Precision by threshold - JavaRDD> precision = metrics.precisionByThreshold().toJavaRDD(); - System.out.println("Precision by threshold: " + precision.toArray()); - - // Recall by threshold - JavaRDD> recall = metrics.recallByThreshold().toJavaRDD(); - System.out.println("Recall by threshold: " + recall.toArray()); - - // F Score by threshold - JavaRDD> f1Score = metrics.fMeasureByThreshold().toJavaRDD(); - System.out.println("F1 Score by threshold: " + f1Score.toArray()); - - JavaRDD> f2Score = metrics.fMeasureByThreshold(2.0).toJavaRDD(); - System.out.println("F2 Score by threshold: " + f2Score.toArray()); - - // Precision-recall curve - JavaRDD> prc = metrics.pr().toJavaRDD(); - System.out.println("Precision-recall curve: " + prc.toArray()); - - // Thresholds - JavaRDD thresholds = precision.map( - new Function, Double>() { - public Double call (Tuple2 t) { - return new Double(t._1().toString()); - } - } - ); - - // ROC Curve - JavaRDD> roc = metrics.roc().toJavaRDD(); - System.out.println("ROC curve: " + roc.toArray()); - - // AUPRC - System.out.println("Area under precision-recall curve = " + metrics.areaUnderPR()); - - // AUROC - System.out.println("Area under ROC = " + metrics.areaUnderROC()); - - // Save and load model - model.save(sc, "myModelPath"); - LogisticRegressionModel sameModel = LogisticRegressionModel.load(sc, "myModelPath"); - } -} - -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaBinaryClassificationMetricsExample.java %}
    Refer to the [`BinaryClassificationMetrics` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.evaluation.BinaryClassificationMetrics) and [`LogisticRegressionWithLBFGS` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.classification.LogisticRegressionWithLBFGS) for more details on the API. -{% highlight python %} -from pyspark.mllib.classification import LogisticRegressionWithLBFGS -from pyspark.mllib.evaluation import BinaryClassificationMetrics -from pyspark.mllib.regression import LabeledPoint -from pyspark.mllib.util import MLUtils - -# Several of the methods available in scala are currently missing from pyspark - -# Load training data in LIBSVM format -data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_binary_classification_data.txt") - -# Split data into training (60%) and test (40%) -training, test = data.randomSplit([0.6, 0.4], seed = 11L) -training.cache() - -# Run training algorithm to build the model -model = LogisticRegressionWithLBFGS.train(training) - -# Compute raw scores on the test set -predictionAndLabels = test.map(lambda lp: (float(model.predict(lp.features)), lp.label)) - -# Instantiate metrics object -metrics = BinaryClassificationMetrics(predictionAndLabels) - -# Area under precision-recall curve -print("Area under PR = %s" % metrics.areaUnderPR) - -# Area under ROC curve -print("Area under ROC = %s" % metrics.areaUnderROC) - -{% endhighlight %} - +{% include_example python/mllib/binary_classification_metrics_example.py %}
    @@ -333,7 +140,7 @@ definitions of positive and negative labels is straightforward. #### Label based metrics Opposed to binary classification where there are only two possible labels, multiclass classification problems have many -possible labels and so the concept of label-based metrics is introduced. Overall precision measures precision across all +possible labels and so the concept of label-based metrics is introduced. Accuracy measures precision across all labels - the number of times any class was predicted correctly (true positives) normalized by the number of data points. Precision by label considers only one class, and measures the number of time a specific label was predicted correctly normalized by the number of times that label appears in the output. @@ -375,20 +182,10 @@ $$\hat{\delta}(x) = \begin{cases}1 & \text{if $x = 0$}, \\ 0 & \text{otherwise}. - Overall Precision - $PPV = \frac{TP}{TP + FP} = \frac{1}{N}\sum_{i=0}^{N-1} \hat{\delta}\left(\hat{\mathbf{y}}_i - - \mathbf{y}_i\right)$ - - - Overall Recall - $TPR = \frac{TP}{TP + FN} = \frac{1}{N}\sum_{i=0}^{N-1} \hat{\delta}\left(\hat{\mathbf{y}}_i - + Accuracy + $ACC = \frac{TP}{TP + FP} = \frac{1}{N}\sum_{i=0}^{N-1} \hat{\delta}\left(\hat{\mathbf{y}}_i - \mathbf{y}_i\right)$ - - Overall F1-measure - $F1 = 2 \cdot \left(\frac{PPV \cdot TPR} - {PPV + TPR}\right)$ - Precision by label $PPV(\ell) = \frac{TP}{TP + FP} = @@ -433,204 +230,21 @@ the data, and evaluate the performance of the algorithm by several multiclass cl
    Refer to the [`MulticlassMetrics` Scala docs](api/scala/index.html#org.apache.spark.mllib.evaluation.MulticlassMetrics) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.util.MLUtils - -// Load training data in LIBSVM format -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_multiclass_classification_data.txt") - -// Split data into training (60%) and test (40%) -val Array(training, test) = data.randomSplit(Array(0.6, 0.4), seed = 11L) -training.cache() - -// Run training algorithm to build the model -val model = new LogisticRegressionWithLBFGS() - .setNumClasses(3) - .run(training) - -// Compute raw scores on the test set -val predictionAndLabels = test.map { case LabeledPoint(label, features) => - val prediction = model.predict(features) - (prediction, label) -} - -// Instantiate metrics object -val metrics = new MulticlassMetrics(predictionAndLabels) - -// Confusion matrix -println("Confusion matrix:") -println(metrics.confusionMatrix) - -// Overall Statistics -val precision = metrics.precision -val recall = metrics.recall // same as true positive rate -val f1Score = metrics.fMeasure -println("Summary Statistics") -println(s"Precision = $precision") -println(s"Recall = $recall") -println(s"F1 Score = $f1Score") - -// Precision by label -val labels = metrics.labels -labels.foreach { l => - println(s"Precision($l) = " + metrics.precision(l)) -} - -// Recall by label -labels.foreach { l => - println(s"Recall($l) = " + metrics.recall(l)) -} - -// False positive rate by label -labels.foreach { l => - println(s"FPR($l) = " + metrics.falsePositiveRate(l)) -} - -// F-measure by label -labels.foreach { l => - println(s"F1-Score($l) = " + metrics.fMeasure(l)) -} - -// Weighted stats -println(s"Weighted precision: ${metrics.weightedPrecision}") -println(s"Weighted recall: ${metrics.weightedRecall}") -println(s"Weighted F1 score: ${metrics.weightedFMeasure}") -println(s"Weighted false positive rate: ${metrics.weightedFalsePositiveRate}") - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/MulticlassMetricsExample.scala %}
    Refer to the [`MulticlassMetrics` Java docs](api/java/org/apache/spark/mllib/evaluation/MulticlassMetrics.html) for details on the API. -{% highlight java %} -import scala.Tuple2; - -import org.apache.spark.api.java.*; -import org.apache.spark.rdd.RDD; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.classification.LogisticRegressionModel; -import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS; -import org.apache.spark.mllib.evaluation.MulticlassMetrics; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; - -public class MulticlassClassification { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("Multiclass Classification Metrics"); - SparkContext sc = new SparkContext(conf); - String path = "data/mllib/sample_multiclass_classification_data.txt"; - JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); - - // Split initial RDD into two... [60% training data, 40% testing data]. - JavaRDD[] splits = data.randomSplit(new double[] {0.6, 0.4}, 11L); - JavaRDD training = splits[0].cache(); - JavaRDD test = splits[1]; - - // Run training algorithm to build the model. - final LogisticRegressionModel model = new LogisticRegressionWithLBFGS() - .setNumClasses(3) - .run(training.rdd()); - - // Compute raw scores on the test set. - JavaRDD> predictionAndLabels = test.map( - new Function>() { - public Tuple2 call(LabeledPoint p) { - Double prediction = model.predict(p.features()); - return new Tuple2(prediction, p.label()); - } - } - ); - - // Get evaluation metrics. - MulticlassMetrics metrics = new MulticlassMetrics(predictionAndLabels.rdd()); - - // Confusion matrix - Matrix confusion = metrics.confusionMatrix(); - System.out.println("Confusion matrix: \n" + confusion); - - // Overall statistics - System.out.println("Precision = " + metrics.precision()); - System.out.println("Recall = " + metrics.recall()); - System.out.println("F1 Score = " + metrics.fMeasure()); - - // Stats by labels - for (int i = 0; i < metrics.labels().length; i++) { - System.out.format("Class %f precision = %f\n", metrics.labels()[i], metrics.precision(metrics.labels()[i])); - System.out.format("Class %f recall = %f\n", metrics.labels()[i], metrics.recall(metrics.labels()[i])); - System.out.format("Class %f F1 score = %f\n", metrics.labels()[i], metrics.fMeasure(metrics.labels()[i])); - } - - //Weighted stats - System.out.format("Weighted precision = %f\n", metrics.weightedPrecision()); - System.out.format("Weighted recall = %f\n", metrics.weightedRecall()); - System.out.format("Weighted F1 score = %f\n", metrics.weightedFMeasure()); - System.out.format("Weighted false positive rate = %f\n", metrics.weightedFalsePositiveRate()); - - // Save and load model - model.save(sc, "myModelPath"); - LogisticRegressionModel sameModel = LogisticRegressionModel.load(sc, "myModelPath"); - } -} - -{% endhighlight %} + {% include_example java/org/apache/spark/examples/mllib/JavaMulticlassClassificationMetricsExample.java %}
    Refer to the [`MulticlassMetrics` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.evaluation.MulticlassMetrics) for more details on the API. -{% highlight python %} -from pyspark.mllib.classification import LogisticRegressionWithLBFGS -from pyspark.mllib.util import MLUtils -from pyspark.mllib.evaluation import MulticlassMetrics - -# Load training data in LIBSVM format -data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_multiclass_classification_data.txt") - -# Split data into training (60%) and test (40%) -training, test = data.randomSplit([0.6, 0.4], seed = 11L) -training.cache() - -# Run training algorithm to build the model -model = LogisticRegressionWithLBFGS.train(training, numClasses=3) - -# Compute raw scores on the test set -predictionAndLabels = test.map(lambda lp: (float(model.predict(lp.features)), lp.label)) - -# Instantiate metrics object -metrics = MulticlassMetrics(predictionAndLabels) - -# Overall statistics -precision = metrics.precision() -recall = metrics.recall() -f1Score = metrics.fMeasure() -print("Summary Stats") -print("Precision = %s" % precision) -print("Recall = %s" % recall) -print("F1 Score = %s" % f1Score) - -# Statistics by class -labels = data.map(lambda lp: lp.label).distinct().collect() -for label in sorted(labels): - print("Class %s precision = %s" % (label, metrics.precision(label))) - print("Class %s recall = %s" % (label, metrics.recall(label))) - print("Class %s F1 Measure = %s" % (label, metrics.fMeasure(label, beta=1.0))) - -# Weighted stats -print("Weighted recall = %s" % metrics.weightedRecall) -print("Weighted precision = %s" % metrics.weightedPrecision) -print("Weighted F(1) Score = %s" % metrics.weightedFMeasure()) -print("Weighted F(0.5) Score = %s" % metrics.weightedFMeasure(beta=0.5)) -print("Weighted false positive rate = %s" % metrics.weightedFalsePositiveRate) -{% endhighlight %} +{% include_example python/mllib/multi_class_metrics_example.py %}
    @@ -736,7 +350,7 @@ $$I_A(x) = \begin{cases}1 & \text{if $x \in A$}, \\ 0 & \text{otherwise}.\end{ca **Examples** -The following code snippets illustrate how to evaluate the performance of a multilabel classifer. The examples +The following code snippets illustrate how to evaluate the performance of a multilabel classifier. The examples use the fake prediction and label data for multilabel classification that is shown below. Document predictions: @@ -766,154 +380,21 @@ True classes:
    Refer to the [`MultilabelMetrics` Scala docs](api/scala/index.html#org.apache.spark.mllib.evaluation.MultilabelMetrics) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.evaluation.MultilabelMetrics -import org.apache.spark.rdd.RDD; - -val scoreAndLabels: RDD[(Array[Double], Array[Double])] = sc.parallelize( - Seq((Array(0.0, 1.0), Array(0.0, 2.0)), - (Array(0.0, 2.0), Array(0.0, 1.0)), - (Array(), Array(0.0)), - (Array(2.0), Array(2.0)), - (Array(2.0, 0.0), Array(2.0, 0.0)), - (Array(0.0, 1.0, 2.0), Array(0.0, 1.0)), - (Array(1.0), Array(1.0, 2.0))), 2) - -// Instantiate metrics object -val metrics = new MultilabelMetrics(scoreAndLabels) - -// Summary stats -println(s"Recall = ${metrics.recall}") -println(s"Precision = ${metrics.precision}") -println(s"F1 measure = ${metrics.f1Measure}") -println(s"Accuracy = ${metrics.accuracy}") - -// Individual label stats -metrics.labels.foreach(label => println(s"Class $label precision = ${metrics.precision(label)}")) -metrics.labels.foreach(label => println(s"Class $label recall = ${metrics.recall(label)}")) -metrics.labels.foreach(label => println(s"Class $label F1-score = ${metrics.f1Measure(label)}")) - -// Micro stats -println(s"Micro recall = ${metrics.microRecall}") -println(s"Micro precision = ${metrics.microPrecision}") -println(s"Micro F1 measure = ${metrics.microF1Measure}") - -// Hamming loss -println(s"Hamming loss = ${metrics.hammingLoss}") - -// Subset accuracy -println(s"Subset accuracy = ${metrics.subsetAccuracy}") - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/MultiLabelMetricsExample.scala %}
    Refer to the [`MultilabelMetrics` Java docs](api/java/org/apache/spark/mllib/evaluation/MultilabelMetrics.html) for details on the API. -{% highlight java %} -import scala.Tuple2; - -import org.apache.spark.api.java.*; -import org.apache.spark.rdd.RDD; -import org.apache.spark.mllib.evaluation.MultilabelMetrics; -import org.apache.spark.SparkConf; -import java.util.Arrays; -import java.util.List; - -public class MultilabelClassification { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("Multilabel Classification Metrics"); - JavaSparkContext sc = new JavaSparkContext(conf); - - List> data = Arrays.asList( - new Tuple2(new double[]{0.0, 1.0}, new double[]{0.0, 2.0}), - new Tuple2(new double[]{0.0, 2.0}, new double[]{0.0, 1.0}), - new Tuple2(new double[]{}, new double[]{0.0}), - new Tuple2(new double[]{2.0}, new double[]{2.0}), - new Tuple2(new double[]{2.0, 0.0}, new double[]{2.0, 0.0}), - new Tuple2(new double[]{0.0, 1.0, 2.0}, new double[]{0.0, 1.0}), - new Tuple2(new double[]{1.0}, new double[]{1.0, 2.0}) - ); - JavaRDD> scoreAndLabels = sc.parallelize(data); - - // Instantiate metrics object - MultilabelMetrics metrics = new MultilabelMetrics(scoreAndLabels.rdd()); - - // Summary stats - System.out.format("Recall = %f\n", metrics.recall()); - System.out.format("Precision = %f\n", metrics.precision()); - System.out.format("F1 measure = %f\n", metrics.f1Measure()); - System.out.format("Accuracy = %f\n", metrics.accuracy()); - - // Stats by labels - for (int i = 0; i < metrics.labels().length - 1; i++) { - System.out.format("Class %1.1f precision = %f\n", metrics.labels()[i], metrics.precision(metrics.labels()[i])); - System.out.format("Class %1.1f recall = %f\n", metrics.labels()[i], metrics.recall(metrics.labels()[i])); - System.out.format("Class %1.1f F1 score = %f\n", metrics.labels()[i], metrics.f1Measure(metrics.labels()[i])); - } - - // Micro stats - System.out.format("Micro recall = %f\n", metrics.microRecall()); - System.out.format("Micro precision = %f\n", metrics.microPrecision()); - System.out.format("Micro F1 measure = %f\n", metrics.microF1Measure()); - - // Hamming loss - System.out.format("Hamming loss = %f\n", metrics.hammingLoss()); - - // Subset accuracy - System.out.format("Subset accuracy = %f\n", metrics.subsetAccuracy()); - - } -} - -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaMultiLabelClassificationMetricsExample.java %}
    Refer to the [`MultilabelMetrics` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.evaluation.MultilabelMetrics) for more details on the API. -{% highlight python %} -from pyspark.mllib.evaluation import MultilabelMetrics - -scoreAndLabels = sc.parallelize([ - ([0.0, 1.0], [0.0, 2.0]), - ([0.0, 2.0], [0.0, 1.0]), - ([], [0.0]), - ([2.0], [2.0]), - ([2.0, 0.0], [2.0, 0.0]), - ([0.0, 1.0, 2.0], [0.0, 1.0]), - ([1.0], [1.0, 2.0])]) - -# Instantiate metrics object -metrics = MultilabelMetrics(scoreAndLabels) - -# Summary stats -print("Recall = %s" % metrics.recall()) -print("Precision = %s" % metrics.precision()) -print("F1 measure = %s" % metrics.f1Measure()) -print("Accuracy = %s" % metrics.accuracy) - -# Individual label stats -labels = scoreAndLabels.flatMap(lambda x: x[1]).distinct().collect() -for label in labels: - print("Class %s precision = %s" % (label, metrics.precision(label))) - print("Class %s recall = %s" % (label, metrics.recall(label))) - print("Class %s F1 Measure = %s" % (label, metrics.f1Measure(label))) - -# Micro stats -print("Micro precision = %s" % metrics.microPrecision) -print("Micro recall = %s" % metrics.microRecall) -print("Micro F1 measure = %s" % metrics.microF1Measure) - -# Hamming loss -print("Hamming loss = %s" % metrics.hammingLoss) - -# Subset accuracy -print("Subset accuracy = %s" % metrics.subsetAccuracy) - -{% endhighlight %} +{% include_example python/mllib/multi_label_metrics_example.py %}
    @@ -1027,280 +508,21 @@ expanded world of non-positive weights are "the same as never having interacted
    Refer to the [`RegressionMetrics` Scala docs](api/scala/index.html#org.apache.spark.mllib.evaluation.RegressionMetrics) and [`RankingMetrics` Scala docs](api/scala/index.html#org.apache.spark.mllib.evaluation.RankingMetrics) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.evaluation.{RegressionMetrics, RankingMetrics} -import org.apache.spark.mllib.recommendation.{ALS, Rating} - -// Read in the ratings data -val ratings = sc.textFile("data/mllib/sample_movielens_data.txt").map { line => - val fields = line.split("::") - Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble - 2.5) -}.cache() - -// Map ratings to 1 or 0, 1 indicating a movie that should be recommended -val binarizedRatings = ratings.map(r => Rating(r.user, r.product, if (r.rating > 0) 1.0 else 0.0)).cache() - -// Summarize ratings -val numRatings = ratings.count() -val numUsers = ratings.map(_.user).distinct().count() -val numMovies = ratings.map(_.product).distinct().count() -println(s"Got $numRatings ratings from $numUsers users on $numMovies movies.") - -// Build the model -val numIterations = 10 -val rank = 10 -val lambda = 0.01 -val model = ALS.train(ratings, rank, numIterations, lambda) - -// Define a function to scale ratings from 0 to 1 -def scaledRating(r: Rating): Rating = { - val scaledRating = math.max(math.min(r.rating, 1.0), 0.0) - Rating(r.user, r.product, scaledRating) -} - -// Get sorted top ten predictions for each user and then scale from [0, 1] -val userRecommended = model.recommendProductsForUsers(10).map{ case (user, recs) => - (user, recs.map(scaledRating)) -} - -// Assume that any movie a user rated 3 or higher (which maps to a 1) is a relevant document -// Compare with top ten most relevant documents -val userMovies = binarizedRatings.groupBy(_.user) -val relevantDocuments = userMovies.join(userRecommended).map{ case (user, (actual, predictions)) => - (predictions.map(_.product), actual.filter(_.rating > 0.0).map(_.product).toArray) -} - -// Instantiate metrics object -val metrics = new RankingMetrics(relevantDocuments) - -// Precision at K -Array(1, 3, 5).foreach{ k => - println(s"Precision at $k = ${metrics.precisionAt(k)}") -} - -// Mean average precision -println(s"Mean average precision = ${metrics.meanAveragePrecision}") - -// Normalized discounted cumulative gain -Array(1, 3, 5).foreach{ k => - println(s"NDCG at $k = ${metrics.ndcgAt(k)}") -} - -// Get predictions for each data point -val allPredictions = model.predict(ratings.map(r => (r.user, r.product))).map(r => ((r.user, r.product), r.rating)) -val allRatings = ratings.map(r => ((r.user, r.product), r.rating)) -val predictionsAndLabels = allPredictions.join(allRatings).map{ case ((user, product), (predicted, actual)) => - (predicted, actual) -} - -// Get the RMSE using regression metrics -val regressionMetrics = new RegressionMetrics(predictionsAndLabels) -println(s"RMSE = ${regressionMetrics.rootMeanSquaredError}") - -// R-squared -println(s"R-squared = ${regressionMetrics.r2}") - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala %}
    Refer to the [`RegressionMetrics` Java docs](api/java/org/apache/spark/mllib/evaluation/RegressionMetrics.html) and [`RankingMetrics` Java docs](api/java/org/apache/spark/mllib/evaluation/RankingMetrics.html) for details on the API. -{% highlight java %} -import scala.Tuple2; - -import org.apache.spark.api.java.*; -import org.apache.spark.rdd.RDD; -import org.apache.spark.mllib.recommendation.MatrixFactorizationModel; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.function.Function; -import java.util.*; -import org.apache.spark.mllib.evaluation.RegressionMetrics; -import org.apache.spark.mllib.evaluation.RankingMetrics; -import org.apache.spark.mllib.recommendation.ALS; -import org.apache.spark.mllib.recommendation.Rating; - -// Read in the ratings data -public class Ranking { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("Ranking Metrics"); - JavaSparkContext sc = new JavaSparkContext(conf); - String path = "data/mllib/sample_movielens_data.txt"; - JavaRDD data = sc.textFile(path); - JavaRDD ratings = data.map( - new Function() { - public Rating call(String line) { - String[] parts = line.split("::"); - return new Rating(Integer.parseInt(parts[0]), Integer.parseInt(parts[1]), Double.parseDouble(parts[2]) - 2.5); - } - } - ); - ratings.cache(); - - // Train an ALS model - final MatrixFactorizationModel model = ALS.train(JavaRDD.toRDD(ratings), 10, 10, 0.01); - - // Get top 10 recommendations for every user and scale ratings from 0 to 1 - JavaRDD> userRecs = model.recommendProductsForUsers(10).toJavaRDD(); - JavaRDD> userRecsScaled = userRecs.map( - new Function, Tuple2>() { - public Tuple2 call(Tuple2 t) { - Rating[] scaledRatings = new Rating[t._2().length]; - for (int i = 0; i < scaledRatings.length; i++) { - double newRating = Math.max(Math.min(t._2()[i].rating(), 1.0), 0.0); - scaledRatings[i] = new Rating(t._2()[i].user(), t._2()[i].product(), newRating); - } - return new Tuple2(t._1(), scaledRatings); - } - } - ); - JavaPairRDD userRecommended = JavaPairRDD.fromJavaRDD(userRecsScaled); - - // Map ratings to 1 or 0, 1 indicating a movie that should be recommended - JavaRDD binarizedRatings = ratings.map( - new Function() { - public Rating call(Rating r) { - double binaryRating; - if (r.rating() > 0.0) { - binaryRating = 1.0; - } - else { - binaryRating = 0.0; - } - return new Rating(r.user(), r.product(), binaryRating); - } - } - ); - - // Group ratings by common user - JavaPairRDD> userMovies = binarizedRatings.groupBy( - new Function() { - public Object call(Rating r) { - return r.user(); - } - } - ); - - // Get true relevant documents from all user ratings - JavaPairRDD> userMoviesList = userMovies.mapValues( - new Function, List>() { - public List call(Iterable docs) { - List products = new ArrayList(); - for (Rating r : docs) { - if (r.rating() > 0.0) { - products.add(r.product()); - } - } - return products; - } - } - ); - - // Extract the product id from each recommendation - JavaPairRDD> userRecommendedList = userRecommended.mapValues( - new Function>() { - public List call(Rating[] docs) { - List products = new ArrayList(); - for (Rating r : docs) { - products.add(r.product()); - } - return products; - } - } - ); - JavaRDD, List>> relevantDocs = userMoviesList.join(userRecommendedList).values(); - - // Instantiate the metrics object - RankingMetrics metrics = RankingMetrics.of(relevantDocs); - - // Precision and NDCG at k - Integer[] kVector = {1, 3, 5}; - for (Integer k : kVector) { - System.out.format("Precision at %d = %f\n", k, metrics.precisionAt(k)); - System.out.format("NDCG at %d = %f\n", k, metrics.ndcgAt(k)); - } - - // Mean average precision - System.out.format("Mean average precision = %f\n", metrics.meanAveragePrecision()); - - // Evaluate the model using numerical ratings and regression metrics - JavaRDD> userProducts = ratings.map( - new Function>() { - public Tuple2 call(Rating r) { - return new Tuple2(r.user(), r.product()); - } - } - ); - JavaPairRDD, Object> predictions = JavaPairRDD.fromJavaRDD( - model.predict(JavaRDD.toRDD(userProducts)).toJavaRDD().map( - new Function, Object>>() { - public Tuple2, Object> call(Rating r){ - return new Tuple2, Object>( - new Tuple2(r.user(), r.product()), r.rating()); - } - } - )); - JavaRDD> ratesAndPreds = - JavaPairRDD.fromJavaRDD(ratings.map( - new Function, Object>>() { - public Tuple2, Object> call(Rating r){ - return new Tuple2, Object>( - new Tuple2(r.user(), r.product()), r.rating()); - } - } - )).join(predictions).values(); - - // Create regression metrics object - RegressionMetrics regressionMetrics = new RegressionMetrics(ratesAndPreds.rdd()); - - // Root mean squared error - System.out.format("RMSE = %f\n", regressionMetrics.rootMeanSquaredError()); - - // R-squared - System.out.format("R-squared = %f\n", regressionMetrics.r2()); - } -} - -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java %}
    Refer to the [`RegressionMetrics` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.evaluation.RegressionMetrics) and [`RankingMetrics` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.evaluation.RankingMetrics) for more details on the API. -{% highlight python %} -from pyspark.mllib.recommendation import ALS, Rating -from pyspark.mllib.evaluation import RegressionMetrics, RankingMetrics - -# Read in the ratings data -lines = sc.textFile("data/mllib/sample_movielens_data.txt") - -def parseLine(line): - fields = line.split("::") - return Rating(int(fields[0]), int(fields[1]), float(fields[2]) - 2.5) -ratings = lines.map(lambda r: parseLine(r)) - -# Train a model on to predict user-product ratings -model = ALS.train(ratings, 10, 10, 0.01) - -# Get predicted ratings on all existing user-product pairs -testData = ratings.map(lambda p: (p.user, p.product)) -predictions = model.predictAll(testData).map(lambda r: ((r.user, r.product), r.rating)) - -ratingsTuple = ratings.map(lambda r: ((r.user, r.product), r.rating)) -scoreAndLabels = predictions.join(ratingsTuple).map(lambda tup: tup[1]) - -# Instantiate regression metrics to compare predicted and actual ratings -metrics = RegressionMetrics(scoreAndLabels) - -# Root mean sqaured error -print("RMSE = %s" % metrics.rootMeanSquaredError) - -# R-squared -print("R-squared = %s" % metrics.r2) - -{% endhighlight %} +{% include_example python/mllib/ranking_metrics_example.py %}
    @@ -1326,8 +548,8 @@ variable from a number of independent variables. $RMSE = \sqrt{\frac{\sum_{i=0}^{N-1} (\mathbf{y}_i - \hat{\mathbf{y}}_i)^2}{N}}$ - Mean Absoloute Error (MAE) - $MAE=\sum_{i=0}^{N-1} \left|\mathbf{y}_i - \hat{\mathbf{y}}_i\right|$ + Mean Absolute Error (MAE) + $MAE=\frac{1}{N}\sum_{i=0}^{N-1} \left|\mathbf{y}_i - \hat{\mathbf{y}}_i\right|$ Coefficient of Determination $(R^2)$ @@ -1350,163 +572,21 @@ and evaluate the performance of the algorithm by several regression metrics.
    Refer to the [`RegressionMetrics` Scala docs](api/scala/index.html#org.apache.spark.mllib.evaluation.RegressionMetrics) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.regression.LinearRegressionModel -import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.evaluation.RegressionMetrics -import org.apache.spark.mllib.util.MLUtils - -// Load the data -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_linear_regression_data.txt").cache() - -// Build the model -val numIterations = 100 -val model = LinearRegressionWithSGD.train(data, numIterations) - -// Get predictions -val valuesAndPreds = data.map{ point => - val prediction = model.predict(point.features) - (prediction, point.label) -} - -// Instantiate metrics object -val metrics = new RegressionMetrics(valuesAndPreds) - -// Squared error -println(s"MSE = ${metrics.meanSquaredError}") -println(s"RMSE = ${metrics.rootMeanSquaredError}") - -// R-squared -println(s"R-squared = ${metrics.r2}") - -// Mean absolute error -println(s"MAE = ${metrics.meanAbsoluteError}") - -// Explained variance -println(s"Explained variance = ${metrics.explainedVariance}") - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala %}
    Refer to the [`RegressionMetrics` Java docs](api/java/org/apache/spark/mllib/evaluation/RegressionMetrics.html) for details on the API. -{% highlight java %} -import scala.Tuple2; - -import org.apache.spark.api.java.*; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.regression.LinearRegressionModel; -import org.apache.spark.mllib.regression.LinearRegressionWithSGD; -import org.apache.spark.mllib.evaluation.RegressionMetrics; -import org.apache.spark.SparkConf; - -public class LinearRegression { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("Linear Regression Example"); - JavaSparkContext sc = new JavaSparkContext(conf); - - // Load and parse the data - String path = "data/mllib/sample_linear_regression_data.txt"; - JavaRDD data = sc.textFile(path); - JavaRDD parsedData = data.map( - new Function() { - public LabeledPoint call(String line) { - String[] parts = line.split(" "); - double[] v = new double[parts.length - 1]; - for (int i = 1; i < parts.length - 1; i++) - v[i - 1] = Double.parseDouble(parts[i].split(":")[1]); - return new LabeledPoint(Double.parseDouble(parts[0]), Vectors.dense(v)); - } - } - ); - parsedData.cache(); - - // Building the model - int numIterations = 100; - final LinearRegressionModel model = - LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations); - - // Evaluate model on training examples and compute training error - JavaRDD> valuesAndPreds = parsedData.map( - new Function>() { - public Tuple2 call(LabeledPoint point) { - double prediction = model.predict(point.features()); - return new Tuple2(prediction, point.label()); - } - } - ); - - // Instantiate metrics object - RegressionMetrics metrics = new RegressionMetrics(valuesAndPreds.rdd()); - - // Squared error - System.out.format("MSE = %f\n", metrics.meanSquaredError()); - System.out.format("RMSE = %f\n", metrics.rootMeanSquaredError()); - - // R-squared - System.out.format("R Squared = %f\n", metrics.r2()); - - // Mean absolute error - System.out.format("MAE = %f\n", metrics.meanAbsoluteError()); - - // Explained variance - System.out.format("Explained Variance = %f\n", metrics.explainedVariance()); - - // Save and load model - model.save(sc.sc(), "myModelPath"); - LinearRegressionModel sameModel = LinearRegressionModel.load(sc.sc(), "myModelPath"); - } -} - -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaRegressionMetricsExample.java %}
    Refer to the [`RegressionMetrics` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.evaluation.RegressionMetrics) for more details on the API. -{% highlight python %} -from pyspark.mllib.regression import LabeledPoint, LinearRegressionWithSGD -from pyspark.mllib.evaluation import RegressionMetrics -from pyspark.mllib.linalg import DenseVector - -# Load and parse the data -def parsePoint(line): - values = line.split() - return LabeledPoint(float(values[0]), DenseVector([float(x.split(':')[1]) for x in values[1:]])) - -data = sc.textFile("data/mllib/sample_linear_regression_data.txt") -parsedData = data.map(parsePoint) - -# Build the model -model = LinearRegressionWithSGD.train(parsedData) - -# Get predictions -valuesAndPreds = parsedData.map(lambda p: (float(model.predict(p.features)), p.label)) - -# Instantiate metrics object -metrics = RegressionMetrics(valuesAndPreds) - -# Squared Error -print("MSE = %s" % metrics.meanSquaredError) -print("RMSE = %s" % metrics.rootMeanSquaredError) - -# R-squared -print("R-squared = %s" % metrics.r2) - -# Mean absolute error -print("MAE = %s" % metrics.meanAbsoluteError) - -# Explained variance -print("Explained variance = %s" % metrics.explainedVariance) - -{% endhighlight %} +{% include_example python/mllib/regression_metrics_example.py %}
    diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 5bee170c61fe..75aea7060187 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -1,7 +1,7 @@ --- layout: global -title: Feature Extraction and Transformation - MLlib -displayTitle: MLlib - Feature Extraction and Transformation +title: Feature Extraction and Transformation - RDD-based API +displayTitle: Feature Extraction and Transformation - RDD-based API --- * Table of contents @@ -10,6 +10,9 @@ displayTitle: MLlib - Feature Extraction and Tran ## TF-IDF +**Note** We recommend using the DataFrame-based API, which is detailed in the [ML user guide on +TF-IDF](ml-features.html#tf-idf). + [Term frequency-inverse document frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) is a feature vectorization method widely used in text mining to reflect the importance of a term to a document in the corpus. Denote a term by `$t$`, a document by `$d$`, and the corpus by `$D$`. @@ -31,7 +34,7 @@ The TF-IDF measure is simply the product of TF and IDF: TFIDF(t, d, D) = TF(t, d) \cdot IDF(t, D). \]` There are several variants on the definition of term frequency and document frequency. -In MLlib, we separate TF and IDF to make them flexible. +In `spark.mllib`, we separate TF and IDF to make them flexible. Our implementation of term frequency utilizes the [hashing trick](http://en.wikipedia.org/wiki/Feature_hashing). @@ -44,7 +47,7 @@ To reduce the chance of collision, we can increase the target feature dimension, the number of buckets of the hash table. The default feature dimension is `$2^{20} = 1,048,576$`. -**Note:** MLlib doesn't provide tools for text segmentation. +**Note:** `spark.mllib` doesn't provide tools for text segmentation. We refer users to the [Stanford NLP Group](http://nlp.stanford.edu/) and [scalanlp/chalk](https://github.com/scalanlp/chalk). @@ -58,46 +61,7 @@ Each record could be an iterable of strings or other types. Refer to the [`HashingTF` Scala docs](api/scala/index.html#org.apache.spark.mllib.feature.HashingTF) for details on the API. - -{% highlight scala %} -import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext -import org.apache.spark.mllib.feature.HashingTF -import org.apache.spark.mllib.linalg.Vector - -val sc: SparkContext = ... - -// Load documents (one per line). -val documents: RDD[Seq[String]] = sc.textFile("...").map(_.split(" ").toSeq) - -val hashingTF = new HashingTF() -val tf: RDD[Vector] = hashingTF.transform(documents) -{% endhighlight %} - -While applying `HashingTF` only needs a single pass to the data, applying `IDF` needs two passes: -first to compute the IDF vector and second to scale the term frequencies by IDF. - -{% highlight scala %} -import org.apache.spark.mllib.feature.IDF - -// ... continue from the previous example -tf.cache() -val idf = new IDF().fit(tf) -val tfidf: RDD[Vector] = idf.transform(tf) -{% endhighlight %} - -MLlib's IDF implementation provides an option for ignoring terms which occur in less than a -minimum number of documents. In such cases, the IDF for these terms is set to 0. This feature -can be used by passing the `minDocFreq` value to the IDF constructor. - -{% highlight scala %} -import org.apache.spark.mllib.feature.IDF - -// ... continue from the previous example -tf.cache() -val idf = new IDF(minDocFreq = 2).fit(tf) -val tfidf: RDD[Vector] = idf.transform(tf) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/TFIDFExample.scala %}
    @@ -109,41 +73,7 @@ Each record could be an iterable of strings or other types. Refer to the [`HashingTF` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.feature.HashingTF) for details on the API. -{% highlight python %} -from pyspark import SparkContext -from pyspark.mllib.feature import HashingTF - -sc = SparkContext() - -# Load documents (one per line). -documents = sc.textFile("...").map(lambda line: line.split(" ")) - -hashingTF = HashingTF() -tf = hashingTF.transform(documents) -{% endhighlight %} - -While applying `HashingTF` only needs a single pass to the data, applying `IDF` needs two passes: -first to compute the IDF vector and second to scale the term frequencies by IDF. - -{% highlight python %} -from pyspark.mllib.feature import IDF - -# ... continue from the previous example -tf.cache() -idf = IDF().fit(tf) -tfidf = idf.transform(tf) -{% endhighlight %} - -MLLib's IDF implementation provides an option for ignoring terms which occur in less than a -minimum number of documents. In such cases, the IDF for these terms is set to 0. This feature -can be used by passing the `minDocFreq` value to the IDF constructor. - -{% highlight python %} -# ... continue from the previous example -tf.cache() -idf = IDF(minDocFreq=2).fit(tf) -tfidf = idf.transform(tf) -{% endhighlight %} +{% include_example python/mllib/tf_idf_example.py %}
    @@ -192,47 +122,12 @@ Here we assume the extracted file is `text8` and in same directory as you run th
    Refer to the [`Word2Vec` Scala docs](api/scala/index.html#org.apache.spark.mllib.feature.Word2Vec) for details on the API. -{% highlight scala %} -import org.apache.spark._ -import org.apache.spark.rdd._ -import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.feature.{Word2Vec, Word2VecModel} - -val input = sc.textFile("text8").map(line => line.split(" ").toSeq) - -val word2vec = new Word2Vec() - -val model = word2vec.fit(input) - -val synonyms = model.findSynonyms("china", 40) - -for((synonym, cosineSimilarity) <- synonyms) { - println(s"$synonym $cosineSimilarity") -} - -// Save and load model -model.save(sc, "myModelPath") -val sameModel = Word2VecModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/Word2VecExample.scala %}
    Refer to the [`Word2Vec` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.feature.Word2Vec) for more details on the API. -{% highlight python %} -from pyspark import SparkContext -from pyspark.mllib.feature import Word2Vec - -sc = SparkContext(appName='Word2Vec') -inp = sc.textFile("text8_lines").map(lambda row: row.split(" ")) - -word2vec = Word2Vec() -model = word2vec.fit(inp) - -synonyms = model.findSynonyms('china', 40) - -for word, cosine_distance in synonyms: - print("{}: {}".format(word, cosine_distance)) -{% endhighlight %} +{% include_example python/mllib/word2vec_example.py %}
    @@ -253,7 +148,7 @@ against features with very large variances exerting an overly large influence du following parameters in the constructor: * `withMean` False by default. Centers the data with mean before scaling. It will build a dense -output, so this does not work on sparse input and will raise an exception. +output, so take care when applying to sparse input. * `withStd` True by default. Scales the data to unit standard deviation. We provide a [`fit`](api/scala/index.html#org.apache.spark.mllib.feature.StandardScaler) method in @@ -277,55 +172,13 @@ so that the new features have unit standard deviation and/or zero mean.
    Refer to the [`StandardScaler` Scala docs](api/scala/index.html#org.apache.spark.mllib.feature.StandardScaler) for details on the API. -{% highlight scala %} -import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.feature.StandardScaler -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.MLUtils - -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") - -val scaler1 = new StandardScaler().fit(data.map(x => x.features)) -val scaler2 = new StandardScaler(withMean = true, withStd = true).fit(data.map(x => x.features)) -// scaler3 is an identical model to scaler2, and will produce identical transformations -val scaler3 = new StandardScalerModel(scaler2.std, scaler2.mean) - -// data1 will be unit variance. -val data1 = data.map(x => (x.label, scaler1.transform(x.features))) - -// Without converting the features into dense vectors, transformation with zero mean will raise -// exception on sparse vector. -// data2 will be unit variance and zero mean. -val data2 = data.map(x => (x.label, scaler2.transform(Vectors.dense(x.features.toArray)))) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/StandardScalerExample.scala %}
    Refer to the [`StandardScaler` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.feature.StandardScaler) for more details on the API. -{% highlight python %} -from pyspark.mllib.util import MLUtils -from pyspark.mllib.linalg import Vectors -from pyspark.mllib.feature import StandardScaler - -data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") -label = data.map(lambda x: x.label) -features = data.map(lambda x: x.features) - -scaler1 = StandardScaler().fit(features) -scaler2 = StandardScaler(withMean=True, withStd=True).fit(features) -# scaler3 is an identical model to scaler2, and will produce identical transformations -scaler3 = StandardScalerModel(scaler2.std, scaler2.mean) - - -# data1 will be unit variance. -data1 = label.zip(scaler1.transform(features)) - -# Without converting the features into dense vectors, transformation with zero mean will raise -# exception on sparse vector. -# data2 will be unit variance and zero mean. -data2 = label.zip(scaler1.transform(features.map(lambda x: Vectors.dense(x.toArray())))) -{% endhighlight %} +{% include_example python/mllib/standard_scaler_example.py %}
    @@ -355,46 +208,13 @@ with $L^2$ norm, and $L^\infty$ norm.
    Refer to the [`Normalizer` Scala docs](api/scala/index.html#org.apache.spark.mllib.feature.Normalizer) for details on the API. -{% highlight scala %} -import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.feature.Normalizer -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.MLUtils - -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") - -val normalizer1 = new Normalizer() -val normalizer2 = new Normalizer(p = Double.PositiveInfinity) - -// Each sample in data1 will be normalized using $L^2$ norm. -val data1 = data.map(x => (x.label, normalizer1.transform(x.features))) - -// Each sample in data2 will be normalized using $L^\infty$ norm. -val data2 = data.map(x => (x.label, normalizer2.transform(x.features))) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/NormalizerExample.scala %}
    Refer to the [`Normalizer` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.feature.Normalizer) for more details on the API. -{% highlight python %} -from pyspark.mllib.util import MLUtils -from pyspark.mllib.linalg import Vectors -from pyspark.mllib.feature import Normalizer - -data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") -labels = data.map(lambda x: x.label) -features = data.map(lambda x: x.features) - -normalizer1 = Normalizer() -normalizer2 = Normalizer(p=float("inf")) - -# Each sample in data1 will be normalized using $L^2$ norm. -data1 = labels.zip(normalizer1.transform(features)) - -# Each sample in data2 will be normalized using $L^\infty$ norm. -data2 = labels.zip(normalizer2.transform(features)) -{% endhighlight %} +{% include_example python/mllib/normalizer_example.py %}
    @@ -405,18 +225,23 @@ features for use in model construction. It reduces the size of the feature space both speed and statistical learning behavior. [`ChiSqSelector`](api/scala/index.html#org.apache.spark.mllib.feature.ChiSqSelector) implements -Chi-Squared feature selection. It operates on labeled data with categorical features. -`ChiSqSelector` orders features based on a Chi-Squared test of independence from the class, -and then filters (selects) the top features which the class label depends on the most. -This is akin to yielding the features with the most predictive power. +Chi-Squared feature selection. It operates on labeled data with categorical features. ChiSqSelector uses the +[Chi-Squared test of independence](https://en.wikipedia.org/wiki/Chi-squared_test) to decide which +features to choose. It supports five selection methods: `numTopFeatures`, `percentile`, `fpr`, `fdr`, `fwe`: + +* `numTopFeatures` chooses a fixed number of top features according to a chi-squared test. This is akin to yielding the features with the most predictive power. +* `percentile` is similar to `numTopFeatures` but chooses a fraction of all features instead of a fixed number. +* `fpr` chooses all features whose p-values are below a threshold, thus controlling the false positive rate of selection. +* `fdr` uses the [Benjamini-Hochberg procedure](https://en.wikipedia.org/wiki/False_discovery_rate#Benjamini.E2.80.93Hochberg_procedure) to choose all features whose false discovery rate is below a threshold. +* `fwe` chooses all features whose p-values are below a threshold. The threshold is scaled by 1/numFeatures, thus controlling the family-wise error rate of selection. + +By default, the selection method is `numTopFeatures`, with the default number of top features set to 50. +The user can choose a selection method using `setSelectorType`. The number of features to select can be tuned using a held-out validation set. ### Model Fitting -`ChiSqSelector` takes a `numTopFeatures` parameter specifying the number of top features that -the selector will select. - The [`fit`](api/scala/index.html#org.apache.spark.mllib.feature.ChiSqSelector) method takes an input of `RDD[LabeledPoint]` with categorical features, learns the summary statistics, and then returns a `ChiSqSelectorModel` which can transform an input dataset into the reduced feature space. @@ -435,29 +260,7 @@ The following example shows the basic use of ChiSqSelector. The data set used ha Refer to the [`ChiSqSelector` Scala docs](api/scala/index.html#org.apache.spark.mllib.feature.ChiSqSelector) for details on the API. -{% highlight scala %} -import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.feature.ChiSqSelector - -// Load some data in libsvm format -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") -// Discretize data in 16 equal bins since ChiSqSelector requires categorical features -// Even though features are doubles, the ChiSqSelector treats each unique value as a category -val discretizedData = data.map { lp => - LabeledPoint(lp.label, Vectors.dense(lp.features.toArray.map { x => (x / 16).floor } ) ) -} -// Create ChiSqSelector that will select top 50 of 692 features -val selector = new ChiSqSelector(50) -// Create ChiSqSelector model (selecting features) -val transformer = selector.fit(discretizedData) -// Filter the top 50 features from each feature vector -val filteredData = discretizedData.map { lp => - LabeledPoint(lp.label, transformer.transform(lp.features)) -} -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala %}
    @@ -465,52 +268,7 @@ val filteredData = discretizedData.map { lp => Refer to the [`ChiSqSelector` Java docs](api/java/org/apache/spark/mllib/feature/ChiSqSelector.html) for details on the API. -{% highlight java %} -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.feature.ChiSqSelector; -import org.apache.spark.mllib.feature.ChiSqSelectorModel; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.util.MLUtils; - -SparkConf sparkConf = new SparkConf().setAppName("JavaChiSqSelector"); -JavaSparkContext sc = new JavaSparkContext(sparkConf); -JavaRDD points = MLUtils.loadLibSVMFile(sc.sc(), - "data/mllib/sample_libsvm_data.txt").toJavaRDD().cache(); - -// Discretize data in 16 equal bins since ChiSqSelector requires categorical features -// Even though features are doubles, the ChiSqSelector treats each unique value as a category -JavaRDD discretizedData = points.map( - new Function() { - @Override - public LabeledPoint call(LabeledPoint lp) { - final double[] discretizedFeatures = new double[lp.features().size()]; - for (int i = 0; i < lp.features().size(); ++i) { - discretizedFeatures[i] = Math.floor(lp.features().apply(i) / 16); - } - return new LabeledPoint(lp.label(), Vectors.dense(discretizedFeatures)); - } - }); - -// Create ChiSqSelector that will select top 50 of 692 features -ChiSqSelector selector = new ChiSqSelector(50); -// Create ChiSqSelector model (selecting features) -final ChiSqSelectorModel transformer = selector.fit(discretizedData.rdd()); -// Filter the top 50 features from each feature vector -JavaRDD filteredData = discretizedData.map( - new Function() { - @Override - public LabeledPoint call(LabeledPoint lp) { - return new LabeledPoint(lp.label(), transformer.transform(lp.features())); - } - } -); - -sc.stop(); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaChiSqSelectorExample.java %}
    @@ -554,78 +312,19 @@ This example below demonstrates how to transform vectors using a transforming ve Refer to the [`ElementwiseProduct` Scala docs](api/scala/index.html#org.apache.spark.mllib.feature.ElementwiseProduct) for details on the API. -{% highlight scala %} -import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.feature.ElementwiseProduct -import org.apache.spark.mllib.linalg.Vectors - -// Create some vector data; also works for sparse vectors -val data = sc.parallelize(Array(Vectors.dense(1.0, 2.0, 3.0), Vectors.dense(4.0, 5.0, 6.0))) - -val transformingVector = Vectors.dense(0.0, 1.0, 2.0) -val transformer = new ElementwiseProduct(transformingVector) - -// Batch transform and per-row transform give the same results: -val transformedData = transformer.transform(data) -val transformedData2 = data.map(x => transformer.transform(x)) - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala %}
    Refer to the [`ElementwiseProduct` Java docs](api/java/org/apache/spark/mllib/feature/ElementwiseProduct.html) for details on the API. -{% highlight java %} -import java.util.Arrays; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.feature.ElementwiseProduct; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; - -// Create some vector data; also works for sparse vectors -JavaRDD data = sc.parallelize(Arrays.asList( - Vectors.dense(1.0, 2.0, 3.0), Vectors.dense(4.0, 5.0, 6.0))); -Vector transformingVector = Vectors.dense(0.0, 1.0, 2.0); -ElementwiseProduct transformer = new ElementwiseProduct(transformingVector); - -// Batch transform and per-row transform give the same results: -JavaRDD transformedData = transformer.transform(data); -JavaRDD transformedData2 = data.map( - new Function() { - @Override - public Vector call(Vector v) { - return transformer.transform(v); - } - } -); - -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaElementwiseProductExample.java %}
    Refer to the [`ElementwiseProduct` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.feature.ElementwiseProduct) for more details on the API. -{% highlight python %} -from pyspark import SparkContext -from pyspark.mllib.linalg import Vectors -from pyspark.mllib.feature import ElementwiseProduct - -# Load and parse the data -sc = SparkContext() -data = sc.textFile("data/mllib/kmeans_data.txt") -parsedData = data.map(lambda x: [float(t) for t in x.split(" ")]) - -# Create weight vector. -transformingVector = Vectors.dense([0.0, 1.0, 2.0]) -transformer = ElementwiseProduct(transformingVector) - -# Batch transform -transformedData = transformer.transform(parsedData) -# Single-row transform -transformedData2 = transformer.transform(parsedData.first()) - -{% endhighlight %} +{% include_example python/mllib/elementwise_product_example.py %}
    @@ -639,50 +338,12 @@ Details you can read at [dimensionality reduction](mllib-dimensionality-reductio The following code demonstrates how to compute principal components on a `Vector` and use them to project the vectors into a low-dimensional space while keeping associated labels -for calculation a [Linear Regression]((mllib-linear-methods.html)) +for calculation a [Linear Regression](mllib-linear-methods.html)
    Refer to the [`PCA` Scala docs](api/scala/index.html#org.apache.spark.mllib.feature.PCA) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.feature.PCA - -val data = sc.textFile("data/mllib/ridge-data/lpsa.data").map { line => - val parts = line.split(',') - LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) -}.cache() - -val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) -val training = splits(0).cache() -val test = splits(1) - -val pca = new PCA(training.first().features.size/2).fit(data.map(_.features)) -val training_pca = training.map(p => p.copy(features = pca.transform(p.features))) -val test_pca = test.map(p => p.copy(features = pca.transform(p.features))) - -val numIterations = 100 -val model = LinearRegressionWithSGD.train(training, numIterations) -val model_pca = LinearRegressionWithSGD.train(training_pca, numIterations) - -val valuesAndPreds = test.map { point => - val score = model.predict(point.features) - (score, point.label) -} - -val valuesAndPreds_pca = test_pca.map { point => - val score = model_pca.predict(point.features) - (score, point.label) -} - -val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.mean() -val MSE_pca = valuesAndPreds_pca.map{case(v, p) => math.pow((v - p), 2)}.mean() - -println("Mean Squared Error = " + MSE) -println("PCA Mean Squared Error = " + MSE_pca) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/PCAExample.scala %}
    diff --git a/docs/mllib-frequent-pattern-mining.md b/docs/mllib-frequent-pattern-mining.md index fe42896a05d8..c9cd7cc85e75 100644 --- a/docs/mllib-frequent-pattern-mining.md +++ b/docs/mllib-frequent-pattern-mining.md @@ -1,7 +1,7 @@ --- layout: global -title: Frequent Pattern Mining - MLlib -displayTitle: MLlib - Frequent Pattern Mining +title: Frequent Pattern Mining - RDD-based API +displayTitle: Frequent Pattern Mining - RDD-based API --- Mining frequent items, itemsets, subsequences, or other substructures is usually among the @@ -9,7 +9,7 @@ first steps to analyze a large-scale dataset, which has been an active research 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, +`spark.mllib` provides a parallel implementation of FP-growth, a popular algorithm to mining frequent itemsets. ## FP-growth @@ -22,13 +22,13 @@ Different from [Apriori-like](http://en.wikipedia.org/wiki/Apriori_algorithm) al 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, +In `spark.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, +PFP distributes the work of growing FP-trees based on the suffixes 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: +`spark.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. @@ -126,7 +126,7 @@ PrefixSpan Approach](http://dx.doi.org/10.1109%2FTKDE.2004.77). We refer the reader to the referenced paper for formalizing the sequential pattern mining problem. -MLlib's PrefixSpan implementation takes the following parameters: +`spark.mllib`'s PrefixSpan implementation takes the following parameters: * `minSupport`: the minimum support required to be considered a frequent sequential pattern. @@ -135,7 +135,7 @@ MLlib's PrefixSpan implementation takes the following parameters: included in the results. * `maxLocalProjDBSize`: the maximum number of items allowed in a prefix-projected database before local iterative processing of the - projected databse begins. This parameter should be tuned with respect + projected database begins. This parameter should be tuned with respect to the size of your executors. **Examples** diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 91e50ccfecec..30112c72c9c3 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -1,32 +1,12 @@ --- layout: global -title: MLlib -displayTitle: Machine Learning Library (MLlib) Guide -description: MLlib machine learning library overview for Spark SPARK_VERSION_SHORT +title: "MLlib: RDD-based API" +displayTitle: "MLlib: RDD-based API" --- -MLlib is Spark's machine learning (ML) library. -Its goal is to make practical machine learning scalable and easy. -It consists of common learning algorithms and utilities, including classification, regression, -clustering, collaborative filtering, dimensionality reduction, as well as lower-level optimization -primitives and higher-level pipeline APIs. - -It divides into two packages: - -* [`spark.mllib`](mllib-guide.html#data-types-algorithms-and-utilities) contains the original API - built on top of [RDDs](programming-guide.html#resilient-distributed-datasets-rdds). -* [`spark.ml`](ml-guide.html) provides higher-level API - built on top of [DataFrames](sql-programming-guide.html#dataframes) for constructing ML pipelines. - -Using `spark.ml` is recommended because with DataFrames the API is more versatile and flexible. -But we will keep supporting `spark.mllib` along with the development of `spark.ml`. -Users should be comfortable using `spark.mllib` features and expect more features coming. -Developers should contribute new algorithms to `spark.ml` if they fit the ML pipeline concept well, -e.g., feature extractors and transformers. - -We list major functionality from both below, with links to detailed guides. - -# spark.mllib: data types, algorithms, and utilities +This page documents sections of the MLlib guide for the RDD-based API (the `spark.mllib` package). +Please see the [MLlib Main Guide](ml-guide.html) for the DataFrame-based API (the `spark.ml` package), +which is now the primary API for MLlib. * [Data types](mllib-data-types.html) * [Basic statistics](mllib-statistics.html) @@ -34,6 +14,7 @@ We list major functionality from both below, with links to detailed guides. * [correlations](mllib-statistics.html#correlations) * [stratified sampling](mllib-statistics.html#stratified-sampling) * [hypothesis testing](mllib-statistics.html#hypothesis-testing) + * [streaming significance testing](mllib-statistics.html#streaming-significance-testing) * [random data generation](mllib-statistics.html#random-data-generation) * [Classification and regression](mllib-classification-regression.html) * [linear models (SVMs, logistic regression, linear regression)](mllib-linear-methods.html) @@ -48,6 +29,7 @@ We list major functionality from both below, with links to detailed guides. * [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) + * [bisecting k-means](mllib-clustering.html#bisecting-kmeans) * [streaming k-means](mllib-clustering.html#streaming-k-means) * [Dimensionality reduction](mllib-dimensionality-reduction.html) * [singular value decomposition (SVD)](mllib-dimensionality-reduction.html#singular-value-decomposition-svd) @@ -63,63 +45,3 @@ We list major functionality from both below, with links to detailed guides. * [stochastic gradient descent](mllib-optimization.html#stochastic-gradient-descent-sgd) * [limited-memory BFGS (L-BFGS)](mllib-optimization.html#limited-memory-bfgs-l-bfgs) -# spark.ml: high-level APIs for ML pipelines - -**[spark.ml programming guide](ml-guide.html)** provides an overview of the Pipelines API and major -concepts. It also contains sections on using algorithms within the Pipelines API, for example: - -* [Feature extraction, transformation, and selection](ml-features.html) -* [Decision trees for classification and regression](ml-decision-tree.html) -* [Ensembles](ml-ensembles.html) -* [Linear methods with elastic net regularization](ml-linear-methods.html) -* [Multilayer perceptron classifier](ml-ann.html) - -# Dependencies - -MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), which depends on -[netlib-java](https://github.com/fommil/netlib-java) for optimised numerical processing. -If natives libraries[^1] are not available at runtime, you will see a warning message and a pure JVM -implementation will be used instead. - -Due to licensing issues with runtime proprietary binaries, we do not include `netlib-java`'s native -proxies by default. -To configure `netlib-java` / Breeze to use system optimised binaries, include -`com.github.fommil.netlib:all:1.1.2` (or build Spark with `-Pnetlib-lgpl`) as a dependency of your -project and read the [netlib-java](https://github.com/fommil/netlib-java) documentation for your -platform's additional installation instructions. - -To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. - -[^1]: To learn more about the benefits and background of system optimised natives, you may wish to - watch Sam Halliday's ScalaX talk on [High Performance Linear Algebra in Scala](http://fommil.github.io/scalax14/#/). - -# Migration guide - -MLlib is under active development. -The APIs marked `Experimental`/`DeveloperApi` may change in future releases, -and the migration guide below will explain all changes between releases. - -## From 1.4 to 1.5 - -In the `spark.mllib` package, there are no break API changes but several behavior changes: - -* [SPARK-9005](https://issues.apache.org/jira/browse/SPARK-9005): - `RegressionMetrics.explainedVariance` returns the average regression sum of squares. -* [SPARK-8600](https://issues.apache.org/jira/browse/SPARK-8600): `NaiveBayesModel.labels` become - sorted. -* [SPARK-3382](https://issues.apache.org/jira/browse/SPARK-3382): `GradientDescent` has a default - convergence tolerance `1e-3`, and hence iterations might end earlier than 1.4. - -In the `spark.ml` package, there exists one break API change and one behavior change: - -* [SPARK-9268](https://issues.apache.org/jira/browse/SPARK-9268): Java's varargs support is removed - from `Params.setDefault` due to a - [Scala compiler bug](https://issues.scala-lang.org/browse/SI-9013). -* [SPARK-10097](https://issues.apache.org/jira/browse/SPARK-10097): `Evaluator.isLargerBetter` is - added to indicate metric ordering. Metrics like RMSE no longer flip signs as in 1.4. - -## Previous Spark versions - -Earlier migration guides are archived [on this page](mllib-migration-guides.html). - ---- diff --git a/docs/mllib-isotonic-regression.md b/docs/mllib-isotonic-regression.md index 85f9226b4341..ca84551506b2 100644 --- a/docs/mllib-isotonic-regression.md +++ b/docs/mllib-isotonic-regression.md @@ -1,7 +1,7 @@ --- layout: global -title: Isotonic regression - MLlib -displayTitle: MLlib - Regression +title: Isotonic regression - RDD-based API +displayTitle: Regression - RDD-based API --- ## Isotonic regression @@ -23,11 +23,11 @@ Essentially isotonic regression is a [monotonic function](http://en.wikipedia.org/wiki/Monotonic_function) best fitting the original data points. -MLlib supports a +`spark.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 +The training input is an 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 diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 0c76e6e99946..034e89e25000 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -1,7 +1,7 @@ --- layout: global -title: Linear Methods - MLlib -displayTitle: MLlib - Linear Methods +title: Linear Methods - RDD-based API +displayTitle: Linear Methods - RDD-based API --- * Table of contents @@ -41,7 +41,7 @@ the objective function is of the form Here the vectors `$\x_i\in\R^d$` are the training data examples, for `$1\le i\le n$`, and `$y_i\in\R$` are their corresponding labels, which we want to predict. We call the method *linear* if $L(\wv; \x, y)$ can be expressed as a function of $\wv^T x$ and $y$. -Several of MLlib's classification and regression algorithms fall into this category, +Several of `spark.mllib`'s classification and regression algorithms fall into this category, and are discussed here. The objective function `$f$` has two parts: @@ -55,7 +55,7 @@ training error) and minimizing model complexity (i.e., to avoid overfitting). ### Loss functions The following table summarizes the loss functions and their gradients or sub-gradients for the -methods MLlib supports: +methods `spark.mllib` supports: @@ -78,12 +78,17 @@ methods MLlib supports:
    +Note that, in the mathematical formulation above, a binary label $y$ is denoted as either +$+1$ (positive) or $-1$ (negative), which is convenient for the formulation. +*However*, the negative label is represented by $0$ in `spark.mllib` instead of $-1$, to be consistent with +multiclass labeling. + ### Regularizers The purpose of the [regularizer](http://en.wikipedia.org/wiki/Regularization_(mathematics)) is to encourage simple models and avoid overfitting. We support the following -regularizers in MLlib: +regularizers in `spark.mllib`: @@ -115,28 +120,27 @@ especially when the number of training examples is small. ### Optimization -Under the hood, linear methods use convex optimization methods to optimize the objective functions. MLlib uses two methods, SGD and L-BFGS, described in the [optimization section](mllib-optimization.html). Currently, most algorithm APIs support Stochastic Gradient Descent (SGD), and a few support L-BFGS. Refer to [this optimization section](mllib-optimization.html#Choosing-an-Optimization-Method) for guidelines on choosing between optimization methods. +Under the hood, linear methods use convex optimization methods to optimize the objective functions. +`spark.mllib` uses two methods, SGD and L-BFGS, described in the [optimization section](mllib-optimization.html). +Currently, most algorithm APIs support Stochastic Gradient Descent (SGD), and a few support L-BFGS. +Refer to [this optimization section](mllib-optimization.html#Choosing-an-Optimization-Method) for guidelines on choosing between optimization methods. ## Classification [Classification](http://en.wikipedia.org/wiki/Statistical_classification) aims to divide items into categories. The most common classification type is -[binary classificaion](http://en.wikipedia.org/wiki/Binary_classification), where there are two +[binary classification](http://en.wikipedia.org/wiki/Binary_classification), where there are two categories, usually named positive and negative. If there are more than two categories, it is called [multiclass classification](http://en.wikipedia.org/wiki/Multiclass_classification). -MLlib supports two linear methods for classification: linear Support Vector Machines (SVMs) +`spark.mllib` supports two linear methods for classification: linear Support Vector Machines (SVMs) and logistic regression. Linear SVMs supports only binary classification, while logistic regression supports both binary and multiclass classification problems. -For both methods, MLlib supports L1 and L2 regularized variants. -The training data set is represented by an RDD of [LabeledPoint](mllib-data-types.html) in MLlib, +For both methods, `spark.mllib` supports L1 and L2 regularized variants. +The training data set is represented by an RDD of [LabeledPoint](mllib-data-types.html#labeled-point) in MLlib, where labels are class indices starting from zero: $0, 1, 2, \ldots$. -Note that, in the mathematical formulation in this guide, a binary label $y$ is denoted as either -$+1$ (positive) or $-1$ (negative), which is convenient for the formulation. -*However*, the negative label is represented by $0$ in MLlib instead of $-1$, to be consistent with -multiclass labeling. ### Linear Support Vector Machines (SVMs) @@ -167,59 +171,25 @@ error. Refer to the [`SVMWithSGD` Scala docs](api/scala/index.html#org.apache.spark.mllib.classification.SVMWithSGD) and [`SVMModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.classification.SVMModel) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.classification.{SVMModel, SVMWithSGD} -import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics -import org.apache.spark.mllib.util.MLUtils - -// Load training data in LIBSVM format. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") - -// Split data into training (60%) and test (40%). -val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) -val training = splits(0).cache() -val test = splits(1) - -// Run training algorithm to build the model -val numIterations = 100 -val model = SVMWithSGD.train(training, numIterations) - -// Clear the default threshold. -model.clearThreshold() - -// Compute raw scores on the test set. -val scoreAndLabels = test.map { point => - val score = model.predict(point.features) - (score, point.label) -} - -// Get evaluation metrics. -val metrics = new BinaryClassificationMetrics(scoreAndLabels) -val auROC = metrics.areaUnderROC() - -println("Area under ROC = " + auROC) - -// Save and load model -model.save(sc, "myModelPath") -val sameModel = SVMModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/SVMWithSGDExample.scala %} The `SVMWithSGD.train()` method by default performs L2 regularization with the regularization parameter set to 1.0. If we want to configure this algorithm, we can customize `SVMWithSGD` further by creating a new object directly and -calling setter methods. All other MLlib algorithms support customization in +calling setter methods. All other `spark.mllib` algorithms support customization in this way as well. For example, the following code produces an L1 regularized variant of SVMs with regularization parameter set to 0.1, and runs the training algorithm for 200 iterations. {% highlight scala %} + import org.apache.spark.mllib.optimization.L1Updater val svmAlg = new SVMWithSGD() -svmAlg.optimizer. - setNumIterations(200). - setRegParam(0.1). - setUpdater(new L1Updater) +svmAlg.optimizer + .setNumIterations(200) + .setRegParam(0.1) + .setUpdater(new L1Updater) val modelL1 = svmAlg.run(training) {% endhighlight %} @@ -234,66 +204,12 @@ that is equivalent to the provided example in Scala is given below: Refer to the [`SVMWithSGD` Java docs](api/java/org/apache/spark/mllib/classification/SVMWithSGD.html) and [`SVMModel` Java docs](api/java/org/apache/spark/mllib/classification/SVMModel.html) for details on the API. -{% highlight java %} -import scala.Tuple2; - -import org.apache.spark.api.java.*; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.classification.*; -import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics; - -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; - -public class SVMClassifier { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("SVM Classifier Example"); - SparkContext sc = new SparkContext(conf); - String path = "data/mllib/sample_libsvm_data.txt"; - JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); - - // Split initial RDD into two... [60% training data, 40% testing data]. - JavaRDD training = data.sample(false, 0.6, 11L); - training.cache(); - JavaRDD test = data.subtract(training); - - // Run training algorithm to build the model. - int numIterations = 100; - final SVMModel model = SVMWithSGD.train(training.rdd(), numIterations); - - // Clear the default threshold. - model.clearThreshold(); - - // Compute raw scores on the test set. - JavaRDD> scoreAndLabels = test.map( - new Function>() { - public Tuple2 call(LabeledPoint p) { - Double score = model.predict(p.features()); - return new Tuple2(score, p.label()); - } - } - ); - - // Get evaluation metrics. - BinaryClassificationMetrics metrics = - new BinaryClassificationMetrics(JavaRDD.toRDD(scoreAndLabels)); - double auROC = metrics.areaUnderROC(); - - System.out.println("Area under ROC = " + auROC); - - // Save and load model - model.save(sc, "myModelPath"); - SVMModel sameModel = SVMModel.load(sc, "myModelPath"); - } -} -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaSVMWithSGDExample.java %} The `SVMWithSGD.train()` method by default performs L2 regularization with the regularization parameter set to 1.0. If we want to configure this algorithm, we can customize `SVMWithSGD` further by creating a new object directly and -calling setter methods. All other MLlib algorithms support customization in +calling setter methods. All other `spark.mllib` algorithms support customization in this way as well. For example, the following code produces an L1 regularized variant of SVMs with regularization parameter set to 0.1, and runs the training algorithm for 200 iterations. @@ -306,7 +222,7 @@ svmAlg.optimizer() .setNumIterations(200) .setRegParam(0.1) .setUpdater(new L1Updater()); -final SVMModel modelL1 = svmAlg.run(training.rdd()); +SVMModel modelL1 = svmAlg.run(training.rdd()); {% endhighlight %} In order to run the above application, follow the instructions @@ -322,30 +238,7 @@ and make predictions with the resulting model to compute the training error. Refer to the [`SVMWithSGD` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.classification.SVMWithSGD) and [`SVMModel` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.classification.SVMModel) for more details on the API. -{% highlight python %} -from pyspark.mllib.classification import SVMWithSGD, SVMModel -from pyspark.mllib.regression import LabeledPoint - -# Load and parse the data -def parsePoint(line): - values = [float(x) for x in line.split(' ')] - return LabeledPoint(values[0], values[1:]) - -data = sc.textFile("data/mllib/sample_svm_data.txt") -parsedData = data.map(parsePoint) - -# Build the model -model = SVMWithSGD.train(parsedData, iterations=100) - -# Evaluating the model on training data -labelsAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) -trainErr = labelsAndPreds.filter(lambda (v, p): v != p).count() / float(parsedData.count()) -print("Training Error = " + str(trainErr)) - -# Save and load model -model.save(sc, "myModelPath") -sameModel = SVMModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example python/mllib/svm_with_sgd_example.py %} @@ -375,7 +268,7 @@ Binary logistic regression can be generalized into train and predict multiclass classification problems. For example, for $K$ possible outcomes, one of the outcomes can be chosen as a "pivot", and the other $K - 1$ outcomes can be separately regressed against the pivot outcome. -In MLlib, the first class $0$ is chosen as the "pivot" class. +In `spark.mllib`, the first class $0$ is chosen as the "pivot" class. See Section 4.4 of [The Elements of Statistical Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for references. @@ -403,42 +296,7 @@ Then the model is evaluated against the test dataset and saved to disk. Refer to the [`LogisticRegressionWithLBFGS` Scala docs](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS) and [`LogisticRegressionModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionModel) for details on the API. -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.classification.{LogisticRegressionWithLBFGS, LogisticRegressionModel} -import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.MLUtils - -// Load training data in LIBSVM format. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") - -// Split data into training (60%) and test (40%). -val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) -val training = splits(0).cache() -val test = splits(1) - -// Run training algorithm to build the model -val model = new LogisticRegressionWithLBFGS() - .setNumClasses(10) - .run(training) - -// Compute raw scores on the test set. -val predictionAndLabels = test.map { case LabeledPoint(label, features) => - val prediction = model.predict(features) - (prediction, label) -} - -// Get evaluation metrics. -val metrics = new MulticlassMetrics(predictionAndLabels) -val precision = metrics.precision -println("Precision = " + precision) - -// Save and load model -model.save(sc, "myModelPath") -val sameModel = LogisticRegressionModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/LogisticRegressionWithLBFGSExample.scala %} @@ -451,57 +309,7 @@ Then the model is evaluated against the test dataset and saved to disk. Refer to the [`LogisticRegressionWithLBFGS` Java docs](api/java/org/apache/spark/mllib/classification/LogisticRegressionWithLBFGS.html) and [`LogisticRegressionModel` Java docs](api/java/org/apache/spark/mllib/classification/LogisticRegressionModel.html) for details on the API. -{% highlight java %} -import scala.Tuple2; - -import org.apache.spark.api.java.*; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.classification.LogisticRegressionModel; -import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS; -import org.apache.spark.mllib.evaluation.MulticlassMetrics; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; - -public class MultinomialLogisticRegressionExample { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("SVM Classifier Example"); - SparkContext sc = new SparkContext(conf); - String path = "data/mllib/sample_libsvm_data.txt"; - JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); - - // Split initial RDD into two... [60% training data, 40% testing data]. - JavaRDD[] splits = data.randomSplit(new double[] {0.6, 0.4}, 11L); - JavaRDD training = splits[0].cache(); - JavaRDD test = splits[1]; - - // Run training algorithm to build the model. - final LogisticRegressionModel model = new LogisticRegressionWithLBFGS() - .setNumClasses(10) - .run(training.rdd()); - - // Compute raw scores on the test set. - JavaRDD> predictionAndLabels = test.map( - new Function>() { - public Tuple2 call(LabeledPoint p) { - Double prediction = model.predict(p.features()); - return new Tuple2(prediction, p.label()); - } - } - ); - - // Get evaluation metrics. - MulticlassMetrics metrics = new MulticlassMetrics(predictionAndLabels.rdd()); - double precision = metrics.precision(); - System.out.println("Precision = " + precision); - - // Save and load model - model.save(sc, "myModelPath"); - LogisticRegressionModel sameModel = LogisticRegressionModel.load(sc, "myModelPath"); - } -} -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaLogisticRegressionWithLBFGSExample.java %}
    @@ -513,30 +321,7 @@ will in the future. Refer to the [`LogisticRegressionWithLBFGS` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.classification.LogisticRegressionWithLBFGS) and [`LogisticRegressionModel` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.classification.LogisticRegressionModel) for more details on the API. -{% highlight python %} -from pyspark.mllib.classification import LogisticRegressionWithLBFGS, LogisticRegressionModel -from pyspark.mllib.regression import LabeledPoint - -# Load and parse the data -def parsePoint(line): - values = [float(x) for x in line.split(' ')] - return LabeledPoint(values[0], values[1:]) - -data = sc.textFile("data/mllib/sample_svm_data.txt") -parsedData = data.map(parsePoint) - -# Build the model -model = LogisticRegressionWithLBFGS.train(parsedData) - -# Evaluating the model on training data -labelsAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) -trainErr = labelsAndPreds.filter(lambda (v, p): v != p).count() / float(parsedData.count()) -print("Training Error = " + str(trainErr)) - -# Save and load model -model.save(sc, "myModelPath") -sameModel = LogisticRegressionModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example python/mllib/logistic_regression_with_lbfgs_example.py %}
    @@ -572,35 +357,7 @@ values. We compute the mean squared error at the end to evaluate Refer to the [`LinearRegressionWithSGD` Scala docs](api/scala/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) and [`LinearRegressionModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.regression.LinearRegressionModel) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.regression.LinearRegressionModel -import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.mllib.linalg.Vectors - -// Load and parse the data -val data = sc.textFile("data/mllib/ridge-data/lpsa.data") -val parsedData = data.map { line => - val parts = line.split(',') - LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) -}.cache() - -// Building the model -val numIterations = 100 -val model = LinearRegressionWithSGD.train(parsedData, numIterations) - -// Evaluate model on training examples and compute training error -val valuesAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.mean() -println("training Mean Squared Error = " + MSE) - -// Save and load model -model.save(sc, "myModelPath") -val sameModel = LinearRegressionModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala %} [`RidgeRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) and [`LassoWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.LassoWithSGD) can be used in a similar fashion as `LinearRegressionWithSGD`. @@ -616,69 +373,7 @@ the Scala snippet provided, is presented below: Refer to the [`LinearRegressionWithSGD` Java docs](api/java/org/apache/spark/mllib/regression/LinearRegressionWithSGD.html) and [`LinearRegressionModel` Java docs](api/java/org/apache/spark/mllib/regression/LinearRegressionModel.html) for details on the API. -{% highlight java %} -import scala.Tuple2; - -import org.apache.spark.api.java.*; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.regression.LinearRegressionModel; -import org.apache.spark.mllib.regression.LinearRegressionWithSGD; -import org.apache.spark.SparkConf; - -public class LinearRegression { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("Linear Regression Example"); - JavaSparkContext sc = new JavaSparkContext(conf); - - // Load and parse the data - String path = "data/mllib/ridge-data/lpsa.data"; - JavaRDD data = sc.textFile(path); - JavaRDD parsedData = data.map( - new Function() { - public LabeledPoint call(String line) { - String[] parts = line.split(","); - String[] features = parts[1].split(" "); - double[] v = new double[features.length]; - for (int i = 0; i < features.length - 1; i++) - v[i] = Double.parseDouble(features[i]); - return new LabeledPoint(Double.parseDouble(parts[0]), Vectors.dense(v)); - } - } - ); - parsedData.cache(); - - // Building the model - int numIterations = 100; - final LinearRegressionModel model = - LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations); - - // Evaluate model on training examples and compute training error - JavaRDD> valuesAndPreds = parsedData.map( - new Function>() { - public Tuple2 call(LabeledPoint point) { - double prediction = model.predict(point.features()); - return new Tuple2(prediction, point.label()); - } - } - ); - double MSE = new JavaDoubleRDD(valuesAndPreds.map( - new Function, Object>() { - public Object call(Tuple2 pair) { - return Math.pow(pair._1() - pair._2(), 2.0); - } - } - ).rdd()).mean(); - System.out.println("training Mean Squared Error = " + MSE); - - // Save and load model - model.save(sc.sc(), "myModelPath"); - LinearRegressionModel sameModel = LinearRegressionModel.load(sc.sc(), "myModelPath"); - } -} -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaLinearRegressionWithSGDExample.java %}
    @@ -691,29 +386,7 @@ Note that the Python API does not yet support model save/load but will in the fu Refer to the [`LinearRegressionWithSGD` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.regression.LinearRegressionWithSGD) and [`LinearRegressionModel` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.regression.LinearRegressionModel) for more details on the API. -{% highlight python %} -from pyspark.mllib.regression import LabeledPoint, LinearRegressionWithSGD, LinearRegressionModel - -# Load and parse the data -def parsePoint(line): - values = [float(x) for x in line.replace(',', ' ').split(' ')] - return LabeledPoint(values[0], values[1:]) - -data = sc.textFile("data/mllib/ridge-data/lpsa.data") -parsedData = data.map(parsePoint) - -# Build the model -model = LinearRegressionWithSGD.train(parsedData) - -# Evaluate the model on training data -valuesAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) -MSE = valuesAndPreds.map(lambda (v, p): (v - p)**2).reduce(lambda x, y: x + y) / valuesAndPreds.count() -print("Mean Squared Error = " + str(MSE)) - -# Save and load model -model.save(sc, "myModelPath") -sameModel = LinearRegressionModel.load(sc, "myModelPath") -{% endhighlight %} +{% include_example python/mllib/linear_regression_with_sgd_example.py %}
    @@ -723,10 +396,10 @@ section of the Spark quick-start guide. Be sure to also include *spark-mllib* to your build file as a dependency. -###Streaming linear regression +### Streaming linear regression When data arrive in a streaming fashion, it is useful to fit regression models online, -updating the parameters of the model as new data arrives. MLlib currently supports +updating the parameters of the model as new data arrives. `spark.mllib` currently supports streaming linear regression using ordinary least squares. The fitting is similar to that performed offline, except fitting occurs on each batch of data, so that the model continually updates to reflect the data from the stream. @@ -743,108 +416,50 @@ online to the first stream, and make predictions on the second stream. First, we import the necessary classes for parsing our input data and creating the model. -{% highlight scala %} - -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD - -{% endhighlight %} - Then we make input streams for training and testing data. We assume a StreamingContext `ssc` has already been created, see [Spark Streaming Programming Guide](streaming-programming-guide.html#initializing) for more info. For this example, we use labeled points in training and testing streams, but in practice you will likely want to use unlabeled vectors for test data. -{% highlight scala %} +We create our model by initializing the weights to zero and register the streams for training and +testing then start the job. Printing predictions alongside true labels lets us easily see the +result. -val trainingData = ssc.textFileStream("/training/data/dir").map(LabeledPoint.parse).cache() -val testData = ssc.textFileStream("/testing/data/dir").map(LabeledPoint.parse) - -{% endhighlight %} - -We create our model by initializing the weights to 0 - -{% highlight scala %} - -val numFeatures = 3 -val model = new StreamingLinearRegressionWithSGD() - .setInitialWeights(Vectors.zeros(numFeatures)) - -{% endhighlight %} - -Now we register the streams for training and testing and start the job. -Printing predictions alongside true labels lets us easily see the result. - -{% highlight scala %} - -model.trainOn(trainingData) -model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print() - -ssc.start() -ssc.awaitTermination() - -{% endhighlight %} - -We can now save text files with data to the training or testing folders. +Finally we can save text files with data to the training or testing folders. Each line should be a data point formatted as `(y,[x1,x2,x3])` where `y` is the label -and `x1,x2,x3` are the features. 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. +and `x1,x2,x3` are the features. Anytime a text file is placed in `args(0)` +the model will update. Anytime a text file is placed in `args(1)` you will see predictions. As you feed more data to the training directory, the predictions will get better! +Here is a complete example: +{% include_example scala/org/apache/spark/examples/mllib/StreamingLinearRegressionExample.scala %} +
    First, we import the necessary classes for parsing our input data and creating the model. -{% highlight python %} -from pyspark.mllib.linalg import Vectors -from pyspark.mllib.regression import LabeledPoint -from pyspark.mllib.regression import StreamingLinearRegressionWithSGD -{% endhighlight %} - Then we make input streams for training and testing data. We assume a StreamingContext `ssc` has already been created, see [Spark Streaming Programming Guide](streaming-programming-guide.html#initializing) for more info. For this example, we use labeled points in training and testing streams, but in practice you will likely want to use unlabeled vectors for test data. -{% highlight python %} -def parse(lp): - label = float(lp[lp.find('(') + 1: lp.find(',')]) - vec = Vectors.dense(lp[lp.find('[') + 1: lp.find(']')].split(',')) - return LabeledPoint(label, vec) - -trainingData = ssc.textFileStream("/training/data/dir").map(parse).cache() -testData = ssc.textFileStream("/testing/data/dir").map(parse) -{% endhighlight %} - -We create our model by initializing the weights to 0 - -{% highlight python %} -numFeatures = 3 -model = StreamingLinearRegressionWithSGD() -model.setInitialWeights([0.0, 0.0, 0.0]) -{% endhighlight %} +We create our model by initializing the weights to 0. Now we register the streams for training and testing and start the job. -{% highlight python %} -model.trainOn(trainingData) -print(model.predictOnValues(testData.map(lambda lp: (lp.label, lp.features)))) - -ssc.start() -ssc.awaitTermination() -{% endhighlight %} - We can now save text files with data to the training or testing folders. Each line should be a data point formatted as `(y,[x1,x2,x3])` where `y` is the label -and `x1,x2,x3` are the features. 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. +and `x1,x2,x3` are the features. Anytime a text file is placed in `sys.argv[1]` +the model will update. Anytime a text file is placed in `sys.argv[2]` you will see predictions. As you feed more data to the training directory, the predictions will get better! +Here a complete example: +{% include_example python/mllib/streaming_linear_regression_example.py %} +
    @@ -852,7 +467,7 @@ will get better! # Implementation (developer) -Behind the scene, MLlib implements a simple distributed version of stochastic gradient descent +Behind the scene, `spark.mllib` implements a simple distributed version of stochastic gradient descent (SGD), building on the underlying gradient descent primitive (as described in the optimization section). All provided algorithms take as input a regularization parameter (`regParam`) along with various parameters associated with stochastic @@ -876,5 +491,3 @@ Algorithms are all implemented in Scala: * [RidgeRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) * [LassoWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.LassoWithSGD) -Python calls the Scala implementation via -[PythonMLLibAPI](api/scala/index.html#org.apache.spark.mllib.api.python.PythonMLLibAPI). diff --git a/docs/mllib-migration-guides.md b/docs/mllib-migration-guides.md index 774b85d1f773..ea6f93fcf67f 100644 --- a/docs/mllib-migration-guides.md +++ b/docs/mllib-migration-guides.md @@ -1,113 +1,9 @@ --- layout: global title: Old Migration Guides - MLlib -displayTitle: MLlib - Old Migration Guides -description: MLlib migration guides from before Spark SPARK_VERSION_SHORT +displayTitle: Old Migration Guides - MLlib --- -The migration guide for the current Spark version is kept on the [MLlib Programming Guide main page](mllib-guide.html#migration-guide). - -## From 1.3 to 1.4 - -In the `spark.mllib` package, there were several breaking changes, but all in `DeveloperApi` or `Experimental` APIs: - -* Gradient-Boosted Trees - * *(Breaking change)* The signature of the [`Loss.gradient`](api/scala/index.html#org.apache.spark.mllib.tree.loss.Loss) method was changed. This is only an issues for users who wrote their own losses for GBTs. - * *(Breaking change)* The `apply` and `copy` methods for the case class [`BoostingStrategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.BoostingStrategy) have been changed because of a modification to the case class fields. This could be an issue for users who use `BoostingStrategy` to set GBT parameters. -* *(Breaking change)* The return value of [`LDA.run`](api/scala/index.html#org.apache.spark.mllib.clustering.LDA) has changed. It now returns an abstract class `LDAModel` instead of the concrete class `DistributedLDAModel`. The object of type `LDAModel` can still be cast to the appropriate concrete type, which depends on the optimization algorithm. - -In the `spark.ml` package, several major API changes occurred, including: - -* `Param` and other APIs for specifying parameters -* `uid` unique IDs for Pipeline components -* Reorganization of certain classes - -Since the `spark.ml` API was an alpha component in Spark 1.3, we do not list all changes here. -However, since 1.4 `spark.ml` is no longer an alpha component, we will provide details on any API -changes for future releases. - -## From 1.2 to 1.3 - -In the `spark.mllib` package, there were several breaking changes. The first change (in `ALS`) is the only one in a component not marked as Alpha or Experimental. - -* *(Breaking change)* In [`ALS`](api/scala/index.html#org.apache.spark.mllib.recommendation.ALS), the extraneous method `solveLeastSquares` has been removed. The `DeveloperApi` method `analyzeBlocks` was also removed. -* *(Breaking change)* [`StandardScalerModel`](api/scala/index.html#org.apache.spark.mllib.feature.StandardScalerModel) remains an Alpha component. In it, the `variance` method has been replaced with the `std` method. To compute the column variance values returned by the original `variance` method, simply square the standard deviation values returned by `std`. -* *(Breaking change)* [`StreamingLinearRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD) remains an Experimental component. In it, there were two changes: - * The constructor taking arguments was removed in favor of a builder pattern using the default constructor plus parameter setter methods. - * Variable `model` is no longer public. -* *(Breaking change)* [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) remains an Experimental component. In it and its associated classes, there were several changes: - * In `DecisionTree`, the deprecated class method `train` has been removed. (The object/static `train` methods remain.) - * In `Strategy`, the `checkpointDir` parameter has been removed. Checkpointing is still supported, but the checkpoint directory must be set before calling tree and tree ensemble training. -* `PythonMLlibAPI` (the interface between Scala/Java and Python for MLlib) was a public API but is now private, declared `private[python]`. This was never meant for external use. -* In linear regression (including Lasso and ridge regression), the squared loss is now divided by 2. - So in order to produce the same result as in 1.2, the regularization parameter needs to be divided by 2 and the step size needs to be multiplied by 2. - -In the `spark.ml` package, the main API changes are from Spark SQL. We list the most important changes here: - -* The old [SchemaRDD](http://spark.apache.org/docs/1.2.1/api/scala/index.html#org.apache.spark.sql.SchemaRDD) has been replaced with [DataFrame](api/scala/index.html#org.apache.spark.sql.DataFrame) with a somewhat modified API. All algorithms in Spark ML which used to use SchemaRDD now use DataFrame. -* In Spark 1.2, we used implicit conversions from `RDD`s of `LabeledPoint` into `SchemaRDD`s by calling `import sqlContext._` where `sqlContext` was an instance of `SQLContext`. These implicits have been moved, so we now call `import sqlContext.implicits._`. -* Java APIs for SQL have also changed accordingly. Please see the examples above and the [Spark SQL Programming Guide](sql-programming-guide.html) for details. - -Other changes were in `LogisticRegression`: - -* The `scoreCol` output column (with default value "score") was renamed to be `probabilityCol` (with default value "probability"). The type was originally `Double` (for the probability of class 1.0), but it is now `Vector` (for the probability of each class, to support multiclass classification in the future). -* In Spark 1.2, `LogisticRegressionModel` did not include an intercept. In Spark 1.3, it includes an intercept; however, it will always be 0.0 since it uses the default settings for [spark.mllib.LogisticRegressionWithLBFGS](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS). The option to use an intercept will be added in the future. - -## From 1.1 to 1.2 - -The only API changes in MLlib v1.2 are in -[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), -which continues to be an experimental API in MLlib 1.2: - -1. *(Breaking change)* The Scala API for classification takes a named argument specifying the number -of classes. In MLlib v1.1, this argument was called `numClasses` in Python and -`numClassesForClassification` in Scala. In MLlib v1.2, the names are both set to `numClasses`. -This `numClasses` parameter is specified either via -[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) -or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) -static `trainClassifier` and `trainRegressor` methods. - -2. *(Breaking change)* The API for -[`Node`](api/scala/index.html#org.apache.spark.mllib.tree.model.Node) has changed. -This should generally not affect user code, unless the user manually constructs decision trees -(instead of using the `trainClassifier` or `trainRegressor` methods). -The tree `Node` now includes more information, including the probability of the predicted label -(for classification). - -3. Printing methods' output has changed. The `toString` (Scala/Java) and `__repr__` (Python) methods used to print the full model; they now print a summary. For the full model, use `toDebugString`. - -Examples in the Spark distribution and examples in the -[Decision Trees Guide](mllib-decision-tree.html#examples) have been updated accordingly. - -## From 1.0 to 1.1 - -The only API changes in MLlib v1.1 are in -[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), -which continues to be an experimental API in MLlib 1.1: - -1. *(Breaking change)* The meaning of tree depth has been changed by 1 in order to match -the implementations of trees in -[scikit-learn](http://scikit-learn.org/stable/modules/classes.html#module-sklearn.tree) -and in [rpart](http://cran.r-project.org/web/packages/rpart/index.html). -In MLlib v1.0, a depth-1 tree had 1 leaf node, and a depth-2 tree had 1 root node and 2 leaf nodes. -In MLlib v1.1, a depth-0 tree has 1 leaf node, and a depth-1 tree has 1 root node and 2 leaf nodes. -This depth is specified by the `maxDepth` parameter in -[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) -or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) -static `trainClassifier` and `trainRegressor` methods. - -2. *(Non-breaking change)* We recommend using the newly added `trainClassifier` and `trainRegressor` -methods to build a [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), -rather than using the old parameter class `Strategy`. These new training methods explicitly -separate classification and regression, and they replace specialized parameter types with -simple `String` types. - -Examples of the new, recommended `trainClassifier` and `trainRegressor` are given in the -[Decision Trees Guide](mllib-decision-tree.html#examples). - -## From 0.9 to 1.0 - -In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few -breaking changes. If your data is sparse, please store it in a sparse format instead of dense to -take advantage of sparsity in both storage and computation. Details are described below. +The migration guide for the current Spark version is kept on the [MLlib Guide main page](ml-guide.html#migration-guide). +Past migration guides are now stored at [ml-migration-guides.html](ml-migration-guides.html). diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index 60ac6c7e5bb1..7471d18a0ddd 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -1,7 +1,7 @@ --- layout: global -title: Naive Bayes - MLlib -displayTitle: MLlib - Naive Bayes +title: Naive Bayes - RDD-based API +displayTitle: Naive Bayes - RDD-based API --- [Naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier) is a simple @@ -12,7 +12,7 @@ distribution of each feature given label, and then it applies Bayes' theorem to compute the conditional probability distribution of label given an observation and use it for prediction. -MLlib supports [multinomial naive +`spark.mllib` supports [multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) and [Bernoulli naive Bayes](http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html). These models are typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index a3bd130ba077..14d76a6e41e2 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -1,7 +1,7 @@ --- layout: global -title: Optimization - MLlib -displayTitle: MLlib - Optimization +title: Optimization - RDD-based API +displayTitle: Optimization - RDD-based API --- * Table of contents @@ -87,7 +87,7 @@ in the `$t$`-th iteration, with the input parameter `$s=$ stepSize`. Note that s step-size for SGD methods can often be delicate in practice and is a topic of active research. **Gradients.** -A table of (sub)gradients of the machine learning methods implemented in MLlib, is available in +A table of (sub)gradients of the machine learning methods implemented in `spark.mllib`, is available in the classification and regression section. @@ -116,7 +116,7 @@ is a stochastic gradient. Here `$S$` is the sampled subset of size `$|S|=$ miniB $\cdot n$`. In each iteration, the sampling over the distributed dataset -([RDD](programming-guide.html#resilient-distributed-datasets-rdds)), as well as the +([RDD](rdd-programming-guide.html#resilient-distributed-datasets-rdds)), as well as the computation of the sum of the partial results from each worker machine is performed by the standard spark routines. @@ -140,7 +140,7 @@ other first-order optimization. ### Choosing an Optimization Method -[Linear methods](mllib-linear-methods.html) use optimization internally, and some linear methods in MLlib support both SGD and L-BFGS. +[Linear methods](mllib-linear-methods.html) use optimization internally, and some linear methods in `spark.mllib` support both SGD and L-BFGS. Different optimization methods can have different convergence guarantees depending on the properties of the objective function, and we cannot cover the literature here. In general, when L-BFGS is available, we recommend using it instead of SGD since L-BFGS tends to converge faster (in fewer iterations). @@ -220,154 +220,13 @@ L-BFGS optimizer.
    Refer to the [`LBFGS` Scala docs](api/scala/index.html#org.apache.spark.mllib.optimization.LBFGS) and [`SquaredL2Updater` Scala docs](api/scala/index.html#org.apache.spark.mllib.optimization.SquaredL2Updater) for details on the API. -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.classification.LogisticRegressionModel -import org.apache.spark.mllib.optimization.{LBFGS, LogisticGradient, SquaredL2Updater} - -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") -val numFeatures = data.take(1)(0).features.size - -// Split data into training (60%) and test (40%). -val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) - -// Append 1 into the training data as intercept. -val training = splits(0).map(x => (x.label, MLUtils.appendBias(x.features))).cache() - -val test = splits(1) - -// Run training algorithm to build the model -val numCorrections = 10 -val convergenceTol = 1e-4 -val maxNumIterations = 20 -val regParam = 0.1 -val initialWeightsWithIntercept = Vectors.dense(new Array[Double](numFeatures + 1)) - -val (weightsWithIntercept, loss) = LBFGS.runLBFGS( - training, - new LogisticGradient(), - new SquaredL2Updater(), - numCorrections, - convergenceTol, - maxNumIterations, - regParam, - initialWeightsWithIntercept) - -val model = new LogisticRegressionModel( - Vectors.dense(weightsWithIntercept.toArray.slice(0, weightsWithIntercept.size - 1)), - weightsWithIntercept(weightsWithIntercept.size - 1)) - -// Clear the default threshold. -model.clearThreshold() - -// Compute raw scores on the test set. -val scoreAndLabels = test.map { point => - val score = model.predict(point.features) - (score, point.label) -} - -// Get evaluation metrics. -val metrics = new BinaryClassificationMetrics(scoreAndLabels) -val auROC = metrics.areaUnderROC() - -println("Loss of each step in training process") -loss.foreach(println) -println("Area under ROC = " + auROC) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/LBFGSExample.scala %}
    Refer to the [`LBFGS` Java docs](api/java/org/apache/spark/mllib/optimization/LBFGS.html) and [`SquaredL2Updater` Java docs](api/java/org/apache/spark/mllib/optimization/SquaredL2Updater.html) for details on the API. -{% highlight java %} -import java.util.Arrays; -import java.util.Random; - -import scala.Tuple2; - -import org.apache.spark.api.java.*; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.classification.LogisticRegressionModel; -import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.optimization.*; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; - -public class LBFGSExample { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("L-BFGS Example"); - SparkContext sc = new SparkContext(conf); - String path = "data/mllib/sample_libsvm_data.txt"; - JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); - int numFeatures = data.take(1).get(0).features().size(); - - // Split initial RDD into two... [60% training data, 40% testing data]. - JavaRDD trainingInit = data.sample(false, 0.6, 11L); - JavaRDD test = data.subtract(trainingInit); - - // Append 1 into the training data as intercept. - JavaRDD> training = data.map( - new Function>() { - public Tuple2 call(LabeledPoint p) { - return new Tuple2(p.label(), MLUtils.appendBias(p.features())); - } - }); - training.cache(); - - // Run training algorithm to build the model. - int numCorrections = 10; - double convergenceTol = 1e-4; - int maxNumIterations = 20; - double regParam = 0.1; - Vector initialWeightsWithIntercept = Vectors.dense(new double[numFeatures + 1]); - - Tuple2 result = LBFGS.runLBFGS( - training.rdd(), - new LogisticGradient(), - new SquaredL2Updater(), - numCorrections, - convergenceTol, - maxNumIterations, - regParam, - initialWeightsWithIntercept); - Vector weightsWithIntercept = result._1(); - double[] loss = result._2(); - - final LogisticRegressionModel model = new LogisticRegressionModel( - Vectors.dense(Arrays.copyOf(weightsWithIntercept.toArray(), weightsWithIntercept.size() - 1)), - (weightsWithIntercept.toArray())[weightsWithIntercept.size() - 1]); - - // Clear the default threshold. - model.clearThreshold(); - - // Compute raw scores on the test set. - JavaRDD> scoreAndLabels = test.map( - new Function>() { - public Tuple2 call(LabeledPoint p) { - Double score = model.predict(p.features()); - return new Tuple2(score, p.label()); - } - }); - - // Get evaluation metrics. - BinaryClassificationMetrics metrics = - new BinaryClassificationMetrics(scoreAndLabels.rdd()); - double auROC = metrics.areaUnderROC(); - - System.out.println("Loss of each step in training process"); - for (double l : loss) - System.out.println(l); - System.out.println("Area under ROC = " + auROC); - } -} -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaLBFGSExample.java %}
    diff --git a/docs/mllib-pmml-model-export.md b/docs/mllib-pmml-model-export.md index 615287125c03..d3530908706d 100644 --- a/docs/mllib-pmml-model-export.md +++ b/docs/mllib-pmml-model-export.md @@ -1,21 +1,21 @@ --- layout: global -title: PMML model export - MLlib -displayTitle: MLlib - PMML model export +title: PMML model export - RDD-based API +displayTitle: PMML model export - RDD-based API --- * Table of contents {:toc} -## MLlib supported models +## `spark.mllib` supported models -MLlib supports model export to Predictive Model Markup Language ([PMML](http://en.wikipedia.org/wiki/Predictive_Model_Markup_Language)). +`spark.mllib` supports model export to Predictive Model Markup Language ([PMML](http://en.wikipedia.org/wiki/Predictive_Model_Markup_Language)). -The table below outlines the MLlib models that can be exported to PMML and their equivalent PMML model. +The table below outlines the `spark.mllib` models that can be exported to PMML and their equivalent PMML model.
    - + @@ -45,41 +45,12 @@ The table below outlines the MLlib models that can be exported to PMML and their
    To export a supported `model` (see table above) to PMML, simply call `model.toPMML`. -Refer to the [`KMeans` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.KMeans) and [`Vectors` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors) for details on the API. +As well as exporting the PMML model to a String (`model.toPMML` as in the example above), you can export the PMML model to other formats. -Here a complete example of building a KMeansModel and print it out in PMML format: -{% highlight scala %} -import org.apache.spark.mllib.clustering.KMeans -import org.apache.spark.mllib.linalg.Vectors - -// Load and parse the data -val data = sc.textFile("data/mllib/kmeans_data.txt") -val parsedData = data.map(s => Vectors.dense(s.split(' ').map(_.toDouble))).cache() - -// Cluster the data into two classes using KMeans -val numClusters = 2 -val numIterations = 20 -val clusters = KMeans.train(parsedData, numClusters, numIterations) - -// Export to PMML -println("PMML Model:\n" + clusters.toPMML) -{% endhighlight %} - -As well as exporting the PMML model to a String (`model.toPMML` as in the example above), you can export the PMML model to other formats: +Refer to the [`KMeans` Scala docs](api/scala/index.html#org.apache.spark.mllib.clustering.KMeans) and [`Vectors` Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) for details on the API. -{% highlight scala %} -// Export the model to a String in PMML format -clusters.toPMML - -// Export the model to a local file in PMML format -clusters.toPMML("/tmp/kmeans.xml") - -// Export the model to a directory on a distributed file system in PMML format -clusters.toPMML(sc,"/tmp/kmeans") - -// Export the model to the OutputStream in PMML format -clusters.toPMML(System.out) -{% endhighlight %} +Here a complete example of building a KMeansModel and print it out in PMML format: +{% include_example scala/org/apache/spark/examples/mllib/PMMLModelExportExample.scala %} For unsupported models, either you will not find a `.toPMML` method or an `IllegalArgumentException` will be thrown. diff --git a/docs/mllib-statistics.md b/docs/mllib-statistics.md index 2c7c9ed693fd..c29400af8505 100644 --- a/docs/mllib-statistics.md +++ b/docs/mllib-statistics.md @@ -1,7 +1,7 @@ --- layout: global -title: Basic Statistics - MLlib -displayTitle: MLlib - Basic Statistics +title: Basic Statistics - RDD-based API +displayTitle: Basic Statistics - RDD-based API --- * Table of contents @@ -10,24 +10,24 @@ displayTitle: MLlib - Basic Statistics `\[ \newcommand{\R}{\mathbb{R}} -\newcommand{\E}{\mathbb{E}} +\newcommand{\E}{\mathbb{E}} \newcommand{\x}{\mathbf{x}} \newcommand{\y}{\mathbf{y}} \newcommand{\wv}{\mathbf{w}} \newcommand{\av}{\mathbf{\alpha}} \newcommand{\bv}{\mathbf{b}} \newcommand{\N}{\mathbb{N}} -\newcommand{\id}{\mathbf{I}} -\newcommand{\ind}{\mathbf{1}} -\newcommand{\0}{\mathbf{0}} -\newcommand{\unit}{\mathbf{e}} -\newcommand{\one}{\mathbf{1}} +\newcommand{\id}{\mathbf{I}} +\newcommand{\ind}{\mathbf{1}} +\newcommand{\0}{\mathbf{0}} +\newcommand{\unit}{\mathbf{e}} +\newcommand{\one}{\mathbf{1}} \newcommand{\zero}{\mathbf{0}} \]` -## Summary statistics +## Summary statistics -We provide column summary statistics for `RDD[Vector]` through the function `colStats` +We provide column summary statistics for `RDD[Vector]` through the function `colStats` available in `Statistics`.
    @@ -40,19 +40,7 @@ total count. Refer to the [`MultivariateStatisticalSummary` Scala docs](api/scala/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} - -val observations: RDD[Vector] = ... // an RDD of Vectors - -// Compute column summary statistics. -val summary: MultivariateStatisticalSummary = Statistics.colStats(observations) -println(summary.mean) // a dense vector containing the mean value for each column -println(summary.variance) // column-wise variance -println(summary.numNonzeros) // number of nonzeros in each column - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/SummaryStatisticsExample.scala %}
    @@ -64,24 +52,7 @@ total count. Refer to the [`MultivariateStatisticalSummary` Java docs](api/java/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.html) for details on the API. -{% highlight java %} -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; -import org.apache.spark.mllib.stat.Statistics; - -JavaSparkContext jsc = ... - -JavaRDD mat = ... // an RDD of Vectors - -// Compute column summary statistics. -MultivariateStatisticalSummary summary = Statistics.colStats(mat.rdd()); -System.out.println(summary.mean()); // a dense vector containing the mean value for each column -System.out.println(summary.variance()); // column-wise variance -System.out.println(summary.numNonzeros()); // number of nonzeros in each column - -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaSummaryStatisticsExample.java %}
    @@ -92,306 +63,124 @@ total count. Refer to the [`MultivariateStatisticalSummary` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.stat.MultivariateStatisticalSummary) for more details on the API. -{% highlight python %} -from pyspark.mllib.stat import Statistics - -sc = ... # SparkContext - -mat = ... # an RDD of Vectors - -# Compute column summary statistics. -summary = Statistics.colStats(mat) -print(summary.mean()) -print(summary.variance()) -print(summary.numNonzeros()) - -{% endhighlight %} +{% include_example python/mllib/summary_statistics_example.py %}
    ## Correlations -Calculating the correlation between two series of data is a common operation in Statistics. In MLlib -we provide the flexibility to calculate pairwise correlations among many series. The supported +Calculating the correlation between two series of data is a common operation in Statistics. In `spark.mllib` +we provide the flexibility to calculate pairwise correlations among many series. The supported correlation methods are currently Pearson's and Spearman's correlation. - +
    -[`Statistics`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) provides methods to -calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or +[`Statistics`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) provides methods to +calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or an `RDD[Vector]`, the output will be a `Double` or the correlation `Matrix` respectively. -Refer to the [`Statistics` Scala docs](api/scala/index.html#org.apache.spark.mllib.stat.Statistics) for details on the API. - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.linalg._ -import org.apache.spark.mllib.stat.Statistics - -val sc: SparkContext = ... - -val seriesX: RDD[Double] = ... // a series -val seriesY: RDD[Double] = ... // must have the same number of partitions and cardinality as seriesX - -// compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a -// method is not specified, Pearson's method will be used by default. -val correlation: Double = Statistics.corr(seriesX, seriesY, "pearson") - -val data: RDD[Vector] = ... // note that each Vector is a row and not a column +Refer to the [`Statistics` Scala docs](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) for details on the API. -// calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. -// If a method is not specified, Pearson's method will be used by default. -val correlMatrix: Matrix = Statistics.corr(data, "pearson") - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/CorrelationsExample.scala %}
    -[`Statistics`](api/java/org/apache/spark/mllib/stat/Statistics.html) provides methods to -calculate correlations between series. Depending on the type of input, two `JavaDoubleRDD`s or +[`Statistics`](api/java/org/apache/spark/mllib/stat/Statistics.html) provides methods to +calculate correlations between series. Depending on the type of input, two `JavaDoubleRDD`s or a `JavaRDD`, the output will be a `Double` or the correlation `Matrix` respectively. Refer to the [`Statistics` Java docs](api/java/org/apache/spark/mllib/stat/Statistics.html) for details on the API. -{% highlight java %} -import org.apache.spark.api.java.JavaDoubleRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.*; -import org.apache.spark.mllib.stat.Statistics; - -JavaSparkContext jsc = ... - -JavaDoubleRDD seriesX = ... // a series -JavaDoubleRDD seriesY = ... // must have the same number of partitions and cardinality as seriesX - -// compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a -// method is not specified, Pearson's method will be used by default. -Double correlation = Statistics.corr(seriesX.srdd(), seriesY.srdd(), "pearson"); - -JavaRDD data = ... // note that each Vector is a row and not a column - -// calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. -// If a method is not specified, Pearson's method will be used by default. -Matrix correlMatrix = Statistics.corr(data.rdd(), "pearson"); - -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaCorrelationsExample.java %}
    -[`Statistics`](api/python/pyspark.mllib.html#pyspark.mllib.stat.Statistics) provides methods to -calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or +[`Statistics`](api/python/pyspark.mllib.html#pyspark.mllib.stat.Statistics) provides methods to +calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or an `RDD[Vector]`, the output will be a `Double` or the correlation `Matrix` respectively. Refer to the [`Statistics` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.stat.Statistics) for more details on the API. -{% highlight python %} -from pyspark.mllib.stat import Statistics - -sc = ... # SparkContext - -seriesX = ... # a series -seriesY = ... # must have the same number of partitions and cardinality as seriesX - -# Compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a -# method is not specified, Pearson's method will be used by default. -print(Statistics.corr(seriesX, seriesY, method="pearson")) - -data = ... # an RDD of Vectors -# calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. -# If a method is not specified, Pearson's method will be used by default. -print(Statistics.corr(data, method="pearson")) - -{% endhighlight %} +{% include_example python/mllib/correlations_example.py %}
    ## Stratified sampling -Unlike the other statistics functions, which reside in MLlib, stratified sampling methods, +Unlike the other statistics functions, which reside in `spark.mllib`, stratified sampling methods, `sampleByKey` and `sampleByKeyExact`, can be performed on RDD's of key-value pairs. For stratified -sampling, the keys can be thought of as a label and the value as a specific attribute. For example -the key can be man or woman, or document ids, and the respective values can be the list of ages -of the people in the population or the list of words in the documents. The `sampleByKey` method -will flip a coin to decide whether an observation will be sampled or not, therefore requires one -pass over the data, and provides an *expected* sample size. `sampleByKeyExact` requires significant +sampling, the keys can be thought of as a label and the value as a specific attribute. For example +the key can be man or woman, or document ids, and the respective values can be the list of ages +of the people in the population or the list of words in the documents. The `sampleByKey` method +will flip a coin to decide whether an observation will be sampled or not, therefore requires one +pass over the data, and provides an *expected* sample size. `sampleByKeyExact` requires significant more resources than the per-stratum simple random sampling used in `sampleByKey`, but will provide -the exact sampling size with 99.99% confidence. `sampleByKeyExact` is currently not supported in +the exact sampling size with 99.99% confidence. `sampleByKeyExact` is currently not supported in python.
    [`sampleByKeyExact()`](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions) allows users to -sample exactly $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the desired +sample exactly $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the desired fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the set of -keys. Sampling without replacement requires one additional pass over the RDD to guarantee sample +keys. Sampling without replacement requires one additional pass over the RDD to guarantee sample size, whereas sampling with replacement requires two additional passes. -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.PairRDDFunctions - -val sc: SparkContext = ... - -val data = ... // an RDD[(K, V)] of any key value pairs -val fractions: Map[K, Double] = ... // specify the exact fraction desired from each key - -// Get an exact sample from each stratum -val approxSample = data.sampleByKey(withReplacement = false, fractions) -val exactSample = data.sampleByKeyExact(withReplacement = false, fractions) - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/StratifiedSamplingExample.scala %}
    [`sampleByKeyExact()`](api/java/org/apache/spark/api/java/JavaPairRDD.html) allows users to -sample exactly $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the desired +sample exactly $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the desired fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the set of -keys. Sampling without replacement requires one additional pass over the RDD to guarantee sample +keys. Sampling without replacement requires one additional pass over the RDD to guarantee sample size, whereas sampling with replacement requires two additional passes. -{% highlight java %} -import java.util.Map; - -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaSparkContext; - -JavaSparkContext jsc = ... - -JavaPairRDD data = ... // an RDD of any key value pairs -Map fractions = ... // specify the exact fraction desired from each key - -// Get an exact sample from each stratum -JavaPairRDD approxSample = data.sampleByKey(false, fractions); -JavaPairRDD exactSample = data.sampleByKeyExact(false, fractions); - -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaStratifiedSamplingExample.java %}
    [`sampleByKey()`](api/python/pyspark.html#pyspark.RDD.sampleByKey) allows users to -sample approximately $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the -desired fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the +sample approximately $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the +desired fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the set of keys. *Note:* `sampleByKeyExact()` is currently not supported in Python. -{% highlight python %} - -sc = ... # SparkContext - -data = ... # an RDD of any key value pairs -fractions = ... # specify the exact fraction desired from each key as a dictionary - -approxSample = data.sampleByKey(False, fractions); - -{% endhighlight %} +{% include_example python/mllib/stratified_sampling_example.py %}
    ## Hypothesis testing -Hypothesis testing is a powerful tool in statistics to determine whether a result is statistically -significant, whether this result occurred by chance or not. MLlib currently supports Pearson's +Hypothesis testing is a powerful tool in statistics to determine whether a result is statistically +significant, whether this result occurred by chance or not. `spark.mllib` currently supports Pearson's chi-squared ( $\chi^2$) tests for goodness of fit and independence. The input data types determine -whether the goodness of fit or the independence test is conducted. The goodness of fit test requires +whether the goodness of fit or the independence test is conducted. The goodness of fit test requires an input type of `Vector`, whereas the independence test requires a `Matrix` as input. -MLlib also supports the input type `RDD[LabeledPoint]` to enable feature selection via chi-squared +`spark.mllib` also supports the input type `RDD[LabeledPoint]` to enable feature selection via chi-squared independence tests.
    -[`Statistics`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) provides methods to -run Pearson's chi-squared tests. The following example demonstrates how to run and interpret +[`Statistics`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) provides methods to +run Pearson's chi-squared tests. The following example demonstrates how to run and interpret hypothesis tests. -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.linalg._ -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.stat.Statistics._ - -val sc: SparkContext = ... - -val vec: Vector = ... // a vector composed of the frequencies of events - -// compute the goodness of fit. If a second vector to test against is not supplied as a parameter, -// the test runs against a uniform distribution. -val goodnessOfFitTestResult = Statistics.chiSqTest(vec) -println(goodnessOfFitTestResult) // summary of the test including the p-value, degrees of freedom, - // test statistic, the method used, and the null hypothesis. - -val mat: Matrix = ... // a contingency matrix - -// conduct Pearson's independence test on the input contingency matrix -val independenceTestResult = Statistics.chiSqTest(mat) -println(independenceTestResult) // summary of the test including the p-value, degrees of freedom... - -val obs: RDD[LabeledPoint] = ... // (feature, label) pairs. - -// The contingency table is constructed from the raw (feature, label) pairs and used to conduct -// the independence test. Returns an array containing the ChiSquaredTestResult for every feature -// against the label. -val featureTestResults: Array[ChiSqTestResult] = Statistics.chiSqTest(obs) -var i = 1 -featureTestResults.foreach { result => - println(s"Column $i:\n$result") - i += 1 -} // summary of the test - -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/HypothesisTestingExample.scala %}
    -[`Statistics`](api/java/org/apache/spark/mllib/stat/Statistics.html) provides methods to -run Pearson's chi-squared tests. The following example demonstrates how to run and interpret +[`Statistics`](api/java/org/apache/spark/mllib/stat/Statistics.html) provides methods to +run Pearson's chi-squared tests. The following example demonstrates how to run and interpret hypothesis tests. Refer to the [`ChiSqTestResult` Java docs](api/java/org/apache/spark/mllib/stat/test/ChiSqTestResult.html) for details on the API. -{% highlight java %} -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.*; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.stat.Statistics; -import org.apache.spark.mllib.stat.test.ChiSqTestResult; - -JavaSparkContext jsc = ... - -Vector vec = ... // a vector composed of the frequencies of events - -// compute the goodness of fit. If a second vector to test against is not supplied as a parameter, -// the test runs against a uniform distribution. -ChiSqTestResult goodnessOfFitTestResult = Statistics.chiSqTest(vec); -// summary of the test including the p-value, degrees of freedom, test statistic, the method used, -// and the null hypothesis. -System.out.println(goodnessOfFitTestResult); - -Matrix mat = ... // a contingency matrix - -// conduct Pearson's independence test on the input contingency matrix -ChiSqTestResult independenceTestResult = Statistics.chiSqTest(mat); -// summary of the test including the p-value, degrees of freedom... -System.out.println(independenceTestResult); - -JavaRDD obs = ... // an RDD of labeled points - -// The contingency table is constructed from the raw (feature, label) pairs and used to conduct -// the independence test. Returns an array containing the ChiSquaredTestResult for every feature -// against the label. -ChiSqTestResult[] featureTestResults = Statistics.chiSqTest(obs.rdd()); -int i = 1; -for (ChiSqTestResult result : featureTestResults) { - System.out.println("Column " + i + ":"); - System.out.println(result); // summary of the test - i++; -} - -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaHypothesisTestingExample.java %}
    @@ -401,50 +190,18 @@ hypothesis tests. Refer to the [`Statistics` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.stat.Statistics) for more details on the API. -{% highlight python %} -from pyspark import SparkContext -from pyspark.mllib.linalg import Vectors, Matrices -from pyspark.mllib.regresssion import LabeledPoint -from pyspark.mllib.stat import Statistics - -sc = SparkContext() - -vec = Vectors.dense(...) # a vector composed of the frequencies of events - -# compute the goodness of fit. If a second vector to test against is not supplied as a parameter, -# the test runs against a uniform distribution. -goodnessOfFitTestResult = Statistics.chiSqTest(vec) -print(goodnessOfFitTestResult) # summary of the test including the p-value, degrees of freedom, - # test statistic, the method used, and the null hypothesis. - -mat = Matrices.dense(...) # a contingency matrix - -# conduct Pearson's independence test on the input contingency matrix -independenceTestResult = Statistics.chiSqTest(mat) -print(independenceTestResult) # summary of the test including the p-value, degrees of freedom... - -obs = sc.parallelize(...) # LabeledPoint(feature, label) . - -# The contingency table is constructed from an RDD of LabeledPoint and used to conduct -# the independence test. Returns an array containing the ChiSquaredTestResult for every feature -# against the label. -featureTestResults = Statistics.chiSqTest(obs) - -for i, result in enumerate(featureTestResults): - print("Column $d:" % (i + 1)) - print(result) -{% endhighlight %} +{% include_example python/mllib/hypothesis_testing_example.py %}
    -Additionally, MLlib provides a 1-sample, 2-sided implementation of the Kolmogorov-Smirnov (KS) test +Additionally, `spark.mllib` provides a 1-sample, 2-sided implementation of the Kolmogorov-Smirnov (KS) test for equality of probability distributions. By providing the name of a theoretical distribution -(currently solely supported for the normal distribution) and its parameters, or a function to +(currently solely supported for the normal distribution) and its parameters, or a function to calculate the cumulative distribution according to a given theoretical distribution, the user can test the null hypothesis that their sample is drawn from that distribution. In the case that the user tests against the normal distribution (`distName="norm"`), but does not provide distribution -parameters, the test initializes to the standard normal distribution and logs an appropriate +parameters, the test initializes to the standard normal distribution and logs an appropriate message.
    @@ -453,23 +210,9 @@ message. run a 1-sample, 2-sided Kolmogorov-Smirnov test. The following example demonstrates how to run and interpret the hypothesis tests. -Refer to the [`Statistics` Scala docs](api/scala/index.html#org.apache.spark.mllib.stat.Statistics) for details on the API. - -{% highlight scala %} -import org.apache.spark.mllib.stat.Statistics - -val data: RDD[Double] = ... // an RDD of sample data +Refer to the [`Statistics` Scala docs](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) for details on the API. -// run a KS test for the sample versus a standard normal distribution -val testResult = Statistics.kolmogorovSmirnovTest(data, "norm", 0, 1) -println(testResult) // summary of the test including the p-value, test statistic, - // and null hypothesis - // if our p-value indicates significance, we can reject the null hypothesis - -// perform a KS test using a cumulative distribution function of our making -val myCDF: Double => Double = ... -val testResult2 = Statistics.kolmogorovSmirnovTest(data, myCDF) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/HypothesisTestingKolmogorovSmirnovTestExample.scala %}
    @@ -479,23 +222,7 @@ and interpret the hypothesis tests. Refer to the [`Statistics` Java docs](api/java/org/apache/spark/mllib/stat/Statistics.html) for details on the API. -{% highlight java %} -import java.util.Arrays; - -import org.apache.spark.api.java.JavaDoubleRDD; -import org.apache.spark.api.java.JavaSparkContext; - -import org.apache.spark.mllib.stat.Statistics; -import org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult; - -JavaSparkContext jsc = ... -JavaDoubleRDD data = jsc.parallelizeDoubles(Arrays.asList(0.2, 1.0, ...)); -KolmogorovSmirnovTestResult testResult = Statistics.kolmogorovSmirnovTest(data, "norm", 0.0, 1.0); -// summary of the test including the p-value, test statistic, -// and null hypothesis -// if our p-value indicates significance, we can reject the null hypothesis -System.out.println(testResult); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaHypothesisTestingKolmogorovSmirnovTestExample.java %}
    @@ -505,19 +232,39 @@ and interpret the hypothesis tests. Refer to the [`Statistics` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.stat.Statistics) for more details on the API. -{% highlight python %} -from pyspark.mllib.stat import Statistics +{% include_example python/mllib/hypothesis_testing_kolmogorov_smirnov_test_example.py %} +
    + -parallelData = sc.parallelize([1.0, 2.0, ... ]) +### Streaming Significance Testing +`spark.mllib` provides online implementations of some tests to support use cases +like A/B testing. These tests may be performed on a Spark Streaming +`DStream[(Boolean,Double)]` where the first element of each tuple +indicates control group (`false`) or treatment group (`true`) and the +second element is the value of an observation. -# run a KS test for the sample versus a standard normal distribution -testResult = Statistics.kolmogorovSmirnovTest(parallelData, "norm", 0, 1) -print(testResult) # summary of the test including the p-value, test statistic, - # and null hypothesis - # if our p-value indicates significance, we can reject the null hypothesis -# Note that the Scala functionality of calling Statistics.kolmogorovSmirnovTest with -# a lambda to calculate the CDF is not made available in the Python API -{% endhighlight %} +Streaming significance testing supports the following parameters: + +* `peacePeriod` - The number of initial data points from the stream to +ignore, used to mitigate novelty effects. +* `windowSize` - The number of past batches to perform hypothesis +testing over. Setting to `0` will perform cumulative processing using +all prior batches. + + +
    +
    +[`StreamingTest`](api/scala/index.html#org.apache.spark.mllib.stat.test.StreamingTest) +provides streaming hypothesis testing. + +{% include_example scala/org/apache/spark/examples/mllib/StreamingTestExample.scala %} +
    + +
    +[`StreamingTest`](api/java/index.html#org.apache.spark.mllib.stat.test.StreamingTest) +provides streaming hypothesis testing. + +{% include_example java/org/apache/spark/examples/mllib/JavaStreamingTestExample.java %}
    @@ -525,17 +272,17 @@ print(testResult) # summary of the test including the p-value, test statistic, ## Random data generation Random data generation is useful for randomized algorithms, prototyping, and performance testing. -MLlib supports generating random RDDs with i.i.d. values drawn from a given distribution: +`spark.mllib` supports generating random RDDs with i.i.d. values drawn from a given distribution: uniform, standard normal, or Poisson.
    -[`RandomRDDs`](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory +[`RandomRDDs`](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs$) provides factory methods to generate random double RDDs or vector RDDs. The following example generates a random double RDD, whose values follows the standard normal distribution `N(0, 1)`, and then map it to `N(1, 4)`. -Refer to the [`RandomRDDs` Scala docs](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs) for details on the API. +Refer to the [`RandomRDDs` Scala docs](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs$) for details on the API. {% highlight scala %} import org.apache.spark.SparkContext @@ -570,12 +317,7 @@ JavaSparkContext jsc = ... // standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. JavaDoubleRDD u = normalJavaRDD(jsc, 1000000L, 10); // Apply a transform to get a random double RDD following `N(1, 4)`. -JavaDoubleRDD v = u.map( - new Function() { - public Double call(Double x) { - return 1.0 + 2.0 * x; - } - }); +JavaDoubleRDD v = u.mapToDouble(x -> 1.0 + 2.0 * x); {% endhighlight %}
    @@ -594,7 +336,7 @@ sc = ... # SparkContext # Generate a random double RDD that contains 1 million i.i.d. values drawn from the # standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. -u = RandomRDDs.uniformRDD(sc, 1000000L, 10) +u = RandomRDDs.normalRDD(sc, 1000000L, 10) # Apply a transform to get a random double RDD following `N(1, 4)`. v = u.map(lambda x: 1.0 + 2.0 * x) {% endhighlight %} @@ -607,7 +349,7 @@ v = u.map(lambda x: 1.0 + 2.0 * x) useful for visualizing empirical probability distributions without requiring assumptions about the particular distribution that the observed samples are drawn from. It computes an estimate of the probability density function of a random variables, evaluated at a given set of points. It achieves -this estimate by expressing the PDF of the empirical distribution at a particular point as the the +this estimate by expressing the PDF of the empirical distribution at a particular point as the mean of PDFs of normal distributions centered around each of the samples.
    @@ -619,21 +361,7 @@ to do so. Refer to the [`KernelDensity` Scala docs](api/scala/index.html#org.apache.spark.mllib.stat.KernelDensity) for details on the API. -{% highlight scala %} -import org.apache.spark.mllib.stat.KernelDensity -import org.apache.spark.rdd.RDD - -val data: RDD[Double] = ... // an RDD of sample data - -// Construct the density estimator with the sample data and a standard deviation for the Gaussian -// kernels -val kd = new KernelDensity() - .setSample(data) - .setBandwidth(3.0) - -// Find density estimates for the given values -val densities = kd.estimate(Array(-1.0, 2.0, 5.0)) -{% endhighlight %} +{% include_example scala/org/apache/spark/examples/mllib/KernelDensityEstimationExample.scala %}
    @@ -643,21 +371,7 @@ to do so. Refer to the [`KernelDensity` Java docs](api/java/org/apache/spark/mllib/stat/KernelDensity.html) for details on the API. -{% highlight java %} -import org.apache.spark.mllib.stat.KernelDensity; -import org.apache.spark.rdd.RDD; - -RDD data = ... // an RDD of sample data - -// Construct the density estimator with the sample data and a standard deviation for the Gaussian -// kernels -KernelDensity kd = new KernelDensity() - .setSample(data) - .setBandwidth(3.0); - -// Find density estimates for the given values -double[] densities = kd.estimate(new double[] {-1.0, 2.0, 5.0}); -{% endhighlight %} +{% include_example java/org/apache/spark/examples/mllib/JavaKernelDensityEstimationExample.java %}
    @@ -667,20 +381,7 @@ to do so. Refer to the [`KernelDensity` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.stat.KernelDensity) for more details on the API. -{% highlight python %} -from pyspark.mllib.stat import KernelDensity - -data = ... # an RDD of sample data - -# Construct the density estimator with the sample data and a standard deviation for the Gaussian -# kernels -kd = KernelDensity() -kd.setSample(data) -kd.setBandwidth(3.0) - -# Find density estimates for the given values -densities = kd.estimate([-1.0, 2.0, 5.0]) -{% endhighlight %} +{% include_example python/mllib/kernel_density_estimation_example.py %}
    diff --git a/docs/monitoring.md b/docs/monitoring.md index cedceb295802..51084a25983e 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -8,7 +8,7 @@ There are several ways to monitor Spark applications: web UIs, metrics, and exte # Web Interfaces -Every SparkContext launches a web UI, by default on port 4040, that +Every SparkContext launches a web UI, by default on port 4040, that displays useful information about the application. This includes: * A list of scheduler stages and tasks @@ -27,22 +27,29 @@ in the UI to persisted storage. ## Viewing After the Fact -Spark's Standalone Mode cluster manager also has its own -[web UI](spark-standalone.html#monitoring-and-logging). If an application has logged events over -the course of its lifetime, then the Standalone master's web UI will automatically re-render the -application's UI after the application has finished. - -If Spark is run on Mesos or YARN, it is still possible to reconstruct the UI of a finished -application through Spark's history server, provided that the application's event logs exist. +It is still possible to construct the UI of an application through Spark's history server, +provided that the application's event logs exist. You can start the history server by executing: ./sbin/start-history-server.sh -When using the file-system provider class (see spark.history.provider below), the base logging -directory must be supplied in the spark.history.fs.logDirectory configuration option, -and should contain sub-directories that each represents an application's event logs. This creates a -web interface at `http://:18080` by default. The history server can be configured as -follows: +This creates a web interface at `http://:18080` by default, listing incomplete +and completed applications and attempts. + +When using the file-system provider class (see `spark.history.provider` below), the base logging +directory must be supplied in the `spark.history.fs.logDirectory` configuration option, +and should contain sub-directories that each represents an application's event logs. + +The spark jobs themselves must be configured to log events, and to log them to the same shared, +writable directory. For example, if the server was configured with a log directory of +`hdfs://namenode/shared/spark-logs`, then the client-side options would be: + + spark.eventLog.enabled true + spark.eventLog.dir hdfs://namenode/shared/spark-logs + +The history server can be configured as follows: + +### Environment Variables
    MLlib modelPMML model
    `spark.mllib` modelPMML model
    @@ -54,6 +61,10 @@ follows: + + + +
    Environment VariableMeaning
    SPARK_DAEMON_JAVA_OPTS JVM options for the history server (default: none).
    SPARK_DAEMON_CLASSPATHClasspath for the history server (default: none).
    SPARK_PUBLIC_DNS @@ -69,11 +80,13 @@ follows:
    +### Spark configuration options + - + @@ -82,23 +95,38 @@ follows: + + + + + @@ -112,10 +140,10 @@ follows: @@ -137,12 +165,34 @@ follows: + + + + + + + + + + @@ -156,15 +206,22 @@ follows: + + + + +
    Property NameDefaultMeaning
    spark.history.providerorg.apache.spark.deploy.history.FsHistoryProviderorg.apache.spark.deploy.history.FsHistoryProvider Name of the class implementing the application history backend. Currently there is only one implementation, provided by Spark, which looks for application logs stored in the file system.spark.history.fs.logDirectory file:/tmp/spark-events - Directory that contains application event logs to be loaded by the history server + For the filesystem history provider, the URL to the directory containing application event + logs to load. This can be a local file:// path, + an HDFS path hdfs://namenode/shared/spark-logs + or that of an alternative filesystem supported by the Hadoop APIs.
    spark.history.fs.update.interval 10s - The period at which information displayed by this history server is updated. - Each update checks for any changes made to the event logs in persisted storage. + The period at which the filesystem history provider checks for new or + updated logs in the log directory. A shorter interval detects new applications faster, + at the expense of more server load re-reading updated applications. + As soon as an update has completed, listings of the completed and incomplete applications + will reflect the changes.
    spark.history.retainedApplications 50 - The number of application UIs to retain. If this cap is exceeded, then the oldest - applications will be removed. + The number of applications to retain UI data for in the cache. If this cap is exceeded, then + the oldest applications will be removed from the cache. If an application is not in the cache, + it will have to be loaded from disk if its accessed from the UI. +
    spark.history.ui.maxApplicationsInt.MaxValue + The number of applications to display on the history summary page. Application UIs are still + available by accessing their URLs directly even if they are not displayed on the history summary page.
    spark.history.kerberos.enabled false - Indicates whether the history server should use kerberos to login. This is useful - if the history server is accessing HDFS files on a secure Hadoop cluster. If this is + Indicates whether the history server should use kerberos to login. This is required + if the history server is accessing HDFS files on a secure Hadoop cluster. If this is true, it uses the configs spark.history.kerberos.principal and - spark.history.kerberos.keytab. + spark.history.kerberos.keytab.
    false Specifies whether acls should be checked to authorize users viewing the applications. - If enabled, access control checks are made regardless of what the individual application had + If enabled, access control checks are made regardless of what the individual application had set for spark.ui.acls.enable when the application was run. The application owner - will always have authorization to view their own application and any users specified via - spark.ui.view.acls when the application was run will also have authorization - to view that application. - If disabled, no access control checks are made. + will always have authorization to view their own application and any users specified via + spark.ui.view.acls and groups specified via spark.ui.view.acls.groups + when the application was run will also have authorization to view that application. + If disabled, no access control checks are made. +
    spark.history.ui.admin.aclsempty + Comma separated list of users/administrators that have view access to all the Spark applications in + history server. By default only the users permitted to view the application at run-time could + access the related application history, with this, configured users/administrators could also + have the permission to access it. + Putting a "*" in the list means any user can have the privilege of admin. +
    spark.history.ui.admin.acls.groupsempty + Comma separated list of groups that have view access to all the Spark applications in + history server. By default only the groups permitted to view the application at run-time could + access the related application history, with this, configured groups could also + have the permission to access it. + Putting a "*" in the list means any group can have the privilege of admin.
    spark.history.fs.cleaner.interval 1d - How often the job history cleaner checks for files to delete. - Files are only deleted if they are older than spark.history.fs.cleaner.maxAge. + How often the filesystem job history cleaner checks for files to delete. + Files are only deleted if they are older than spark.history.fs.cleaner.maxAge
    spark.history.fs.cleaner.maxAge 7d - Job history files older than this will be deleted when the history cleaner runs. + Job history files older than this will be deleted when the filesystem history cleaner runs. +
    spark.history.fs.numReplayThreads25% of available cores + Number of threads that will be used by history server to process event logs.
    @@ -172,7 +229,25 @@ follows: Note that in all of these UIs, the tables are sortable by clicking their headers, making it easy to identify slow tasks, data skew, etc. -Note that the history server only displays completed Spark jobs. One way to signal the completion of a Spark job is to stop the Spark Context explicitly (`sc.stop()`), or in Python using the `with SparkContext() as sc:` to handle the Spark Context setup and tear down, and still show the job history on the UI. +Note + +1. The history server displays both completed and incomplete Spark jobs. If an application makes +multiple attempts after failures, the failed attempts will be displayed, as well as any ongoing +incomplete attempt or the final successful attempt. + +2. Incomplete applications are only updated intermittently. The time between updates is defined +by the interval between checks for changed files (`spark.history.fs.update.interval`). +On larger clusters the update interval may be set to large values. +The way to view a running application is actually to view its own web UI. + +3. Applications which exited without registering themselves as completed will be listed +as incomplete —even though they are no longer running. This can happen if an application +crashes. + +2. One way to signal the completion of a Spark job is to stop the Spark Context +explicitly (`sc.stop()`), or in Python using the `with SparkContext() as sc:` construct +to handle the Spark Context setup and tear down. + ## REST API @@ -182,64 +257,148 @@ both running applications, and in the history server. The endpoints are mounted for the history server, they would typically be accessible at `http://:18080/api/v1`, and for a running application, at `http://localhost:4040/api/v1`. +In the API, an application is referenced by its application ID, `[app-id]`. +When running on YARN, each application may have multiple attempts, but there are attempt IDs +only for applications in cluster mode, not applications in client mode. Applications in YARN cluster mode +can be identified by their `[attempt-id]`. In the API listed below, when running in YARN cluster mode, +`[app-id]` will actually be `[base-app-id]/[attempt-id]`, where `[base-app-id]` is the YARN application ID. + - + - + - + - + +
    ?status=[active|complete|pending|failed] list only stages in the state. - + - + - + - + - + + + + + - + - + + + + + - - + + - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    EndpointMeaning
    /applicationsA list of all applicationsA list of all applications. +
    + ?status=[completed|running] list only applications in the chosen state. +
    + ?minDate=[date] earliest start date/time to list. +
    + ?maxDate=[date] latest start date/time to list. +
    + ?minEndDate=[date] earliest end date/time to list. +
    + ?maxEndDate=[date] latest end date/time to list. +
    + ?limit=[limit] limits the number of applications listed. +
    Examples: +
    ?minDate=2015-02-10 +
    ?minDate=2015-02-03T16:42:40.000GMT +
    ?maxDate=2015-02-11T20:41:30.000GMT +
    ?minEndDate=2015-02-12 +
    ?minEndDate=2015-02-12T09:15:10.000GMT +
    ?maxEndDate=2015-02-14T16:30:45.000GMT +
    ?limit=10
    /applications/[app-id]/jobsA list of all jobs for a given application + A list of all jobs for a given application. +
    ?status=[running|succeeded|failed|unknown] list only jobs in the specific state. +
    /applications/[app-id]/jobs/[job-id]Details for the given jobDetails for the given job.
    /applications/[app-id]/stagesA list of all stages for a given applicationA list of all stages for a given application.
    /applications/[app-id]/stages/[stage-id]A list of all attempts for the given stage + A list of all attempts for the given stage. +
    /applications/[app-id]/stages/[stage-id]/[stage-attempt-id]Details for the given stage attemptDetails for the given stage attempt.
    /applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskSummarySummary metrics of all tasks in the given stage attempt + Summary metrics of all tasks in the given stage attempt. +
    ?quantiles summarize the metrics with the given quantiles. +
    Example: ?quantiles=0.01,0.5,0.99 +
    /applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskListA list of all tasks for the given stage attempt + A list of all tasks for the given stage attempt. +
    ?offset=[offset]&length=[len] list tasks in the given range. +
    ?sortBy=[runtime|-runtime] sort the tasks. +
    Example: ?offset=10&length=50&sortBy=runtime +
    /applications/[app-id]/executorsA list of all executors for the given applicationA list of all active executors for the given application.
    /applications/[app-id]/allexecutorsA list of all(active and dead) executors for the given application.
    /applications/[app-id]/storage/rddA list of stored RDDs for the given applicationA list of stored RDDs for the given application.
    /applications/[app-id]/storage/rdd/[rdd-id]Details for the storage status of a given RDDDetails for the storage status of a given RDD.
    /applications/[base-app-id]/logsDownload the event logs for all attempts of the given application as files within + a zip file. +
    /applications/[app-id]/logsDownload the event logs for all attempts of the given application as a zip file/applications/[base-app-id]/[attempt-id]/logsDownload the event logs for a specific application attempt as a zip file.
    /applications/[app-id]/[attempt-id]/logsDownload the event logs for the specified attempt of the given application as a zip file/applications/[app-id]/streaming/statisticsStatistics for the streaming context.
    /applications/[app-id]/streaming/receiversA list of all streaming receivers.
    /applications/[app-id]/streaming/receivers/[stream-id]Details of the given receiver.
    /applications/[app-id]/streaming/batchesA list of all retained batches.
    /applications/[app-id]/streaming/batches/[batch-id]Details of the given batch.
    /applications/[app-id]/streaming/batches/[batch-id]/operationsA list of all output operations of the given batch.
    /applications/[app-id]/streaming/batches/[batch-id]/operations/[outputOp-id]Details of the given operation and given batch.
    /applications/[app-id]/environmentEnvironment details of the given application.
    -When running on Yarn, each application has multiple attempts, so `[app-id]` is actually -`[app-id]/[attempt-id]` in all cases. +The number of jobs and stages which can retrieved is constrained by the same retention +mechanism of the standalone Spark UI; `"spark.ui.retainedJobs"` defines the threshold +value triggering garbage collection on jobs, and `spark.ui.retainedStages` that for stages. +Note that the garbage collection takes place on playback: it is possible to retrieve +more entries by increasing these values and restarting the history server. + +### API Versioning Policy These endpoints have been strongly versioned to make it easier to develop applications on top. In particular, Spark guarantees: @@ -249,7 +408,7 @@ These endpoints have been strongly versioned to make it easier to develop applic * New endpoints may be added * New fields may be added to existing endpoints * New versions of the api may be added in the future at a separate endpoint (eg., `api/v2`). New versions are *not* required to be backwards compatible. -* Api versions may be dropped, but only after at least one minor release of co-existing with a new api version +* Api versions may be dropped, but only after at least one minor release of co-existing with a new api version. Note that even when examining the UI of a running applications, the `applications/[app-id]` portion is still required, though there is only one application available. Eg. to see the list of jobs for the @@ -258,14 +417,26 @@ keep the paths consistent in both modes. # Metrics -Spark has a configurable metrics system based on the -[Coda Hale Metrics Library](http://metrics.codahale.com/). -This allows users to report Spark metrics to a variety of sinks including HTTP, JMX, and CSV -files. The metrics system is configured via a configuration file that Spark expects to be present -at `$SPARK_HOME/conf/metrics.properties`. A custom file location can be specified via the +Spark has a configurable metrics system based on the +[Dropwizard Metrics Library](http://metrics.dropwizard.io/). +This allows users to report Spark metrics to a variety of sinks including HTTP, JMX, and CSV +files. The metrics system is configured via a configuration file that Spark expects to be present +at `$SPARK_HOME/conf/metrics.properties`. A custom file location can be specified via the `spark.metrics.conf` [configuration property](configuration.html#spark-properties). -Spark's metrics are decoupled into different -_instances_ corresponding to Spark components. Within each instance, you can configure a +By default, the root namespace used for driver or executor metrics is +the value of `spark.app.id`. However, often times, users want to be able to track the metrics +across apps for driver and executors, which is hard to do with application ID +(i.e. `spark.app.id`) since it changes with every invocation of the app. For such use cases, +a custom namespace can be specified for metrics reporting using `spark.metrics.namespace` +configuration property. +If, say, users wanted to set the metrics namespace to the name of the application, they +can set the `spark.metrics.namespace` property to a value like `${spark.app.name}`. This value is +then expanded appropriately by Spark and is used as the root namespace of the metrics system. +Non driver and executor metrics are never prefixed with `spark.app.id`, nor does the +`spark.metrics.namespace` property have any such affect on such metrics. + +Spark's metrics are decoupled into different +_instances_ corresponding to Spark components. Within each instance, you can configure a set of sinks to which metrics are reported. The following instances are currently supported: * `master`: The Spark standalone master process. @@ -273,6 +444,7 @@ set of sinks to which metrics are reported. The following instances are currentl * `worker`: A Spark standalone worker process. * `executor`: A Spark executor. * `driver`: The Spark driver process (the process in which your SparkContext is created). +* `shuffleService`: The Spark shuffle service. Each instance can report to zero or more _sinks_. Sinks are contained in the `org.apache.spark.metrics.sink` package: @@ -283,6 +455,7 @@ Each instance can report to zero or more _sinks_. Sinks are contained in the * `MetricsServlet`: Adds a servlet within the existing Spark UI to serve metrics data as JSON data. * `GraphiteSink`: Sends metrics to a Graphite node. * `Slf4jSink`: Sends metrics to slf4j as log entries. +* `StatsdSink`: Sends metrics to a StatsD node. Spark also supports a Ganglia sink which is not included in the default build due to licensing restrictions: @@ -290,26 +463,26 @@ licensing restrictions: * `GangliaSink`: Sends metrics to a Ganglia node or multicast group. To install the `GangliaSink` you'll need to perform a custom build of Spark. _**Note that -by embedding this library you will include [LGPL](http://www.gnu.org/copyleft/lesser.html)-licensed -code in your Spark package**_. For sbt users, set the -`SPARK_GANGLIA_LGPL` environment variable before building. For Maven users, enable +by embedding this library you will include [LGPL](http://www.gnu.org/copyleft/lesser.html)-licensed +code in your Spark package**_. For sbt users, set the +`SPARK_GANGLIA_LGPL` environment variable before building. For Maven users, enable the `-Pspark-ganglia-lgpl` profile. In addition to modifying the cluster's Spark build user applications will need to link to the `spark-ganglia-lgpl` artifact. -The syntax of the metrics configuration file is defined in an example configuration file, +The syntax of the metrics configuration file is defined in an example configuration file, `$SPARK_HOME/conf/metrics.properties.template`. # Advanced Instrumentation Several external tools can be used to help profile the performance of Spark jobs: -* Cluster-wide monitoring tools, such as [Ganglia](http://ganglia.sourceforge.net/), can provide -insight into overall cluster utilization and resource bottlenecks. For instance, a Ganglia -dashboard can quickly reveal whether a particular workload is disk bound, network bound, or +* Cluster-wide monitoring tools, such as [Ganglia](http://ganglia.sourceforge.net/), can provide +insight into overall cluster utilization and resource bottlenecks. For instance, a Ganglia +dashboard can quickly reveal whether a particular workload is disk bound, network bound, or CPU bound. -* OS profiling tools such as [dstat](http://dag.wieers.com/home-made/dstat/), -[iostat](http://linux.die.net/man/1/iostat), and [iotop](http://linux.die.net/man/1/iotop) +* OS profiling tools such as [dstat](http://dag.wieers.com/home-made/dstat/), +[iostat](http://linux.die.net/man/1/iostat), and [iotop](http://linux.die.net/man/1/iotop) can provide fine-grained profiling on individual nodes. -* JVM utilities such as `jstack` for providing stack traces, `jmap` for creating heap-dumps, -`jstat` for reporting time-series statistics and `jconsole` for visually exploring various JVM +* JVM utilities such as `jstack` for providing stack traces, `jmap` for creating heap-dumps, +`jstat` for reporting time-series statistics and `jconsole` for visually exploring various JVM properties are useful for those comfortable with JVM internals. diff --git a/docs/programming-guide.md b/docs/programming-guide.md index f823b89a4b5e..f8b8f74c53b5 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1,1600 +1,7 @@ --- layout: global title: Spark Programming Guide -description: Spark SPARK_VERSION_SHORT programming guide in Java, Scala and Python +redirect: rdd-programming-guide.html --- -* This will become a table of contents (this text will be scraped). -{:toc} - - -# Overview - -At a high level, every Spark application consists of a *driver program* that runs the user's `main` function and executes various *parallel operations* on a cluster. The main abstraction Spark provides is a *resilient distributed dataset* (RDD), which is a collection of elements partitioned across the nodes of the cluster that can be operated on in parallel. RDDs are created by starting with a file in the Hadoop file system (or any other Hadoop-supported file system), or an existing Scala collection in the driver program, and transforming it. Users may also ask Spark to *persist* an RDD in memory, allowing it to be reused efficiently across parallel operations. Finally, RDDs automatically recover from node failures. - -A second abstraction in Spark is *shared variables* that can be used in parallel operations. By default, when Spark runs a function in parallel as a set of tasks on different nodes, it ships a copy of each variable used in the function to each task. Sometimes, a variable needs to be shared across tasks, or between tasks and the driver program. Spark supports two types of shared variables: *broadcast variables*, which can be used to cache a value in memory on all nodes, and *accumulators*, which are variables that are only "added" to, such as counters and sums. - -This guide shows each of these features in each of Spark's supported languages. It is easiest to follow -along with if you launch Spark's interactive shell -- either `bin/spark-shell` for the Scala shell or -`bin/pyspark` for the Python one. - -# Linking with Spark - -
    - -
    - -Spark {{site.SPARK_VERSION}} uses Scala {{site.SCALA_BINARY_VERSION}}. To write -applications in Scala, you will need to use a compatible Scala version (e.g. {{site.SCALA_BINARY_VERSION}}.X). - -To write a Spark application, you need to add a Maven dependency on Spark. Spark is available through Maven Central at: - - groupId = org.apache.spark - artifactId = spark-core_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION}} - -In addition, if you wish to access an HDFS cluster, you need to add a dependency on -`hadoop-client` for your version of HDFS. - - groupId = org.apache.hadoop - artifactId = hadoop-client - version = - -Finally, you need to import some Spark classes into your program. Add the following lines: - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.SparkConf -{% endhighlight %} - -(Before Spark 1.3.0, you need to explicitly `import org.apache.spark.SparkContext._` to enable essential implicit conversions.) - -
    - -
    - -Spark {{site.SPARK_VERSION}} works with Java 7 and higher. If you are using Java 8, Spark supports -[lambda expressions](http://docs.oracle.com/javase/tutorial/java/javaOO/lambdaexpressions.html) -for concisely writing functions, otherwise you can use the classes in the -[org.apache.spark.api.java.function](api/java/index.html?org/apache/spark/api/java/function/package-summary.html) package. - -To write a Spark application in Java, you need to add a dependency on Spark. Spark is available through Maven Central at: - - groupId = org.apache.spark - artifactId = spark-core_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION}} - -In addition, if you wish to access an HDFS cluster, you need to add a dependency on -`hadoop-client` for your version of HDFS. - - groupId = org.apache.hadoop - artifactId = hadoop-client - version = - -Finally, you need to import some Spark classes into your program. Add the following lines: - -{% highlight scala %} -import org.apache.spark.api.java.JavaSparkContext -import org.apache.spark.api.java.JavaRDD -import org.apache.spark.SparkConf -{% endhighlight %} - -
    - -
    - -Spark {{site.SPARK_VERSION}} works with Python 2.6+ or Python 3.4+. It can use the standard CPython interpreter, -so C libraries like NumPy can be used. It also works with PyPy 2.3+. - -To run Spark applications in Python, use the `bin/spark-submit` script located in the Spark directory. -This script will load Spark's Java/Scala libraries and allow you to submit applications to a cluster. -You can also use `bin/pyspark` to launch an interactive Python shell. - -If you wish to access HDFS data, you need to use a build of PySpark linking -to your version of HDFS. -[Prebuilt packages](http://spark.apache.org/downloads.html) are also available on the Spark homepage -for common HDFS versions. - -Finally, you need to import some Spark classes into your program. Add the following line: - -{% highlight python %} -from pyspark import SparkContext, SparkConf -{% endhighlight %} - -PySpark requires the same minor version of Python in both driver and workers. It uses the default python version in PATH, -you can specify which version of Python you want to use by `PYSPARK_PYTHON`, for example: - -{% highlight bash %} -$ PYSPARK_PYTHON=python3.4 bin/pyspark -$ PYSPARK_PYTHON=/opt/pypy-2.5/bin/pypy bin/spark-submit examples/src/main/python/pi.py -{% endhighlight %} - -
    - -
    - - -# Initializing Spark - -
    - -
    - -The first thing a Spark program must do is to create a [SparkContext](api/scala/index.html#org.apache.spark.SparkContext) object, which tells Spark -how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object -that contains information about your application. - -Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before creating a new one. - -{% highlight scala %} -val conf = new SparkConf().setAppName(appName).setMaster(master) -new SparkContext(conf) -{% endhighlight %} - -
    - -
    - -The first thing a Spark program must do is to create a [JavaSparkContext](api/java/index.html?org/apache/spark/api/java/JavaSparkContext.html) object, which tells Spark -how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/java/index.html?org/apache/spark/SparkConf.html) object -that contains information about your application. - -{% highlight java %} -SparkConf conf = new SparkConf().setAppName(appName).setMaster(master); -JavaSparkContext sc = new JavaSparkContext(conf); -{% endhighlight %} - -
    - -
    - -The first thing a Spark program must do is to create a [SparkContext](api/python/pyspark.html#pyspark.SparkContext) object, which tells Spark -how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/python/pyspark.html#pyspark.SparkConf) object -that contains information about your application. - -{% highlight python %} -conf = SparkConf().setAppName(appName).setMaster(master) -sc = SparkContext(conf=conf) -{% endhighlight %} - -
    - -
    - -The `appName` parameter is a name for your application to show on the cluster UI. -`master` is a [Spark, Mesos or YARN cluster URL](submitting-applications.html#master-urls), -or a special "local" string to run in local mode. -In practice, when running on a cluster, you will not want to hardcode `master` in the program, -but rather [launch the application with `spark-submit`](submitting-applications.html) and -receive it there. However, for local testing and unit tests, you can pass "local" to run Spark -in-process. - - -## Using the Shell - -
    - -
    - -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. 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] -{% endhighlight %} - -Or, to also add `code.jar` to its classpath, use: - -{% highlight bash %} -$ ./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). - -
    - -
    - -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`. 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 %} -$ ./bin/pyspark --master local[4] -{% endhighlight %} - -Or, to also add `code.py` to the search path (in order to later be able to `import code`), use: - -{% highlight bash %} -$ ./bin/pyspark --master local[4] --py-files code.py -{% endhighlight %} - -For a complete list of options, run `pyspark --help`. Behind the scenes, -`pyspark` invokes the more general [`spark-submit` script](submitting-applications.html). - -It is also possible to launch the PySpark shell in [IPython](http://ipython.org), the -enhanced Python interpreter. PySpark works with IPython 1.0.0 and later. To -use IPython, set the `PYSPARK_DRIVER_PYTHON` variable to `ipython` when running `bin/pyspark`: - -{% highlight bash %} -$ PYSPARK_DRIVER_PYTHON=ipython ./bin/pyspark -{% endhighlight %} - -You can customize the `ipython` command by setting `PYSPARK_DRIVER_PYTHON_OPTS`. For example, to launch -the [IPython Notebook](http://ipython.org/notebook.html) with PyLab plot support: - -{% highlight bash %} -$ PYSPARK_DRIVER_PYTHON=ipython PYSPARK_DRIVER_PYTHON_OPTS="notebook" ./bin/pyspark -{% endhighlight %} - -After the IPython Notebook server is launched, you can create a new "Python 2" notebook from -the "Files" tab. Inside the notebook, you can input the command `%pylab inline` as part of -your notebook before you start to try Spark from the IPython notebook. - -
    - -
    - -# Resilient Distributed Datasets (RDDs) - -Spark revolves around the concept of a _resilient distributed dataset_ (RDD), which is a fault-tolerant collection of elements that can be operated on in parallel. There are two ways to create RDDs: *parallelizing* -an existing collection in your driver program, or referencing a dataset in an external storage system, such as a -shared filesystem, HDFS, HBase, or any data source offering a Hadoop InputFormat. - -## Parallelized Collections - -
    - -
    - -Parallelized collections are created by calling `SparkContext`'s `parallelize` method on an existing collection in your driver program (a Scala `Seq`). The elements of the collection are copied to form a distributed dataset that can be operated on in parallel. For example, here is how to create a parallelized collection holding the numbers 1 to 5: - -{% highlight scala %} -val data = Array(1, 2, 3, 4, 5) -val distData = sc.parallelize(data) -{% endhighlight %} - -Once created, the distributed dataset (`distData`) can be operated on in parallel. For example, we might call `distData.reduce((a, b) => a + b)` to add up the elements of the array. We describe operations on distributed datasets later on. - -
    - -
    - -Parallelized collections are created by calling `JavaSparkContext`'s `parallelize` method on an existing `Collection` in your driver program. The elements of the collection are copied to form a distributed dataset that can be operated on in parallel. For example, here is how to create a parallelized collection holding the numbers 1 to 5: - -{% highlight java %} -List data = Arrays.asList(1, 2, 3, 4, 5); -JavaRDD distData = sc.parallelize(data); -{% endhighlight %} - -Once created, the distributed dataset (`distData`) can be operated on in parallel. For example, we might call `distData.reduce((a, b) -> a + b)` to add up the elements of the list. -We describe operations on distributed datasets later on. - -**Note:** *In this guide, we'll often use the concise Java 8 lambda syntax to specify Java functions, but -in older versions of Java you can implement the interfaces in the -[org.apache.spark.api.java.function](api/java/index.html?org/apache/spark/api/java/function/package-summary.html) package. -We describe [passing functions to Spark](#passing-functions-to-spark) in more detail below.* - -
    - -
    - -Parallelized collections are created by calling `SparkContext`'s `parallelize` method on an existing iterable or collection in your driver program. The elements of the collection are copied to form a distributed dataset that can be operated on in parallel. For example, here is how to create a parallelized collection holding the numbers 1 to 5: - -{% highlight python %} -data = [1, 2, 3, 4, 5] -distData = sc.parallelize(data) -{% endhighlight %} - -Once created, the distributed dataset (`distData`) can be operated on in parallel. For example, we can call `distData.reduce(lambda a, b: a + b)` to add up the elements of the list. -We describe operations on distributed datasets later on. - -
    - -
    - -One important parameter for parallel collections is the number of *partitions* to cut the dataset into. Spark will run one task for each partition of the cluster. Typically you want 2-4 partitions for each CPU in your cluster. Normally, Spark tries to set the number of partitions automatically based on your cluster. However, you can also set it manually by passing it as a second parameter to `parallelize` (e.g. `sc.parallelize(data, 10)`). Note: some places in the code use the term slices (a synonym for partitions) to maintain backward compatibility. - -## External Datasets - -
    - -
    - -Spark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). - -Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3n://`, etc URI) and reads it as a collection of lines. Here is an example invocation: - -{% highlight scala %} -scala> val distFile = sc.textFile("data.txt") -distFile: RDD[String] = MappedRDD@1d4cee08 -{% endhighlight %} - -Once created, `distFile` can be acted on by dataset operations. For example, we can add up the sizes of all the lines using the `map` and `reduce` operations as follows: `distFile.map(s => s.length).reduce((a, b) => a + b)`. - -Some notes on reading files with Spark: - -* If using a path on the local filesystem, the file must also be accessible at the same path on worker nodes. Either copy the file to all workers or use a network-mounted shared file system. - -* All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. - -* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. - -Apart from text files, Spark's Scala API also supports several other data formats: - -* `SparkContext.wholeTextFiles` lets you read a directory containing multiple small text files, and returns each of them as (filename, content) pairs. This is in contrast with `textFile`, which would return one record per line in each file. - -* For [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), use SparkContext's `sequenceFile[K, V]` method where `K` and `V` are the types of key and values in the file. These should be subclasses of Hadoop's [Writable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Writable.html) interface, like [IntWritable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/IntWritable.html) and [Text](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Text.html). In addition, Spark allows you to specify native types for a few common Writables; for example, `sequenceFile[Int, String]` will automatically read IntWritables and Texts. - -* For other Hadoop InputFormats, you can use the `SparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source. You can also use `SparkContext.newAPIHadoopRDD` for InputFormats based on the "new" MapReduce API (`org.apache.hadoop.mapreduce`). - -* `RDD.saveAsObjectFile` and `SparkContext.objectFile` support saving an RDD in a simple format consisting of serialized Java objects. While this is not as efficient as specialized formats like Avro, it offers an easy way to save any RDD. - -
    - -
    - -Spark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). - -Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3n://`, etc URI) and reads it as a collection of lines. Here is an example invocation: - -{% highlight java %} -JavaRDD distFile = sc.textFile("data.txt"); -{% endhighlight %} - -Once created, `distFile` can be acted on by dataset operations. For example, we can add up the sizes of all the lines using the `map` and `reduce` operations as follows: `distFile.map(s -> s.length()).reduce((a, b) -> a + b)`. - -Some notes on reading files with Spark: - -* If using a path on the local filesystem, the file must also be accessible at the same path on worker nodes. Either copy the file to all workers or use a network-mounted shared file system. - -* All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. - -* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. - -Apart from text files, Spark's Java API also supports several other data formats: - -* `JavaSparkContext.wholeTextFiles` lets you read a directory containing multiple small text files, and returns each of them as (filename, content) pairs. This is in contrast with `textFile`, which would return one record per line in each file. - -* For [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), use SparkContext's `sequenceFile[K, V]` method where `K` and `V` are the types of key and values in the file. These should be subclasses of Hadoop's [Writable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Writable.html) interface, like [IntWritable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/IntWritable.html) and [Text](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Text.html). - -* For other Hadoop InputFormats, you can use the `JavaSparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source. You can also use `JavaSparkContext.newAPIHadoopRDD` for InputFormats based on the "new" MapReduce API (`org.apache.hadoop.mapreduce`). - -* `JavaRDD.saveAsObjectFile` and `JavaSparkContext.objectFile` support saving an RDD in a simple format consisting of serialized Java objects. While this is not as efficient as specialized formats like Avro, it offers an easy way to save any RDD. - -
    - -
    - -PySpark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). - -Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3n://`, etc URI) and reads it as a collection of lines. Here is an example invocation: - -{% highlight python %} ->>> distFile = sc.textFile("data.txt") -{% endhighlight %} - -Once created, `distFile` can be acted on by dataset operations. For example, we can add up the sizes of all the lines using the `map` and `reduce` operations as follows: `distFile.map(lambda s: len(s)).reduce(lambda a, b: a + b)`. - -Some notes on reading files with Spark: - -* If using a path on the local filesystem, the file must also be accessible at the same path on worker nodes. Either copy the file to all workers or use a network-mounted shared file system. - -* All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. - -* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. - -Apart from text files, Spark's Python API also supports several other data formats: - -* `SparkContext.wholeTextFiles` lets you read a directory containing multiple small text files, and returns each of them as (filename, content) pairs. This is in contrast with `textFile`, which would return one record per line in each file. - -* `RDD.saveAsPickleFile` and `SparkContext.pickleFile` support saving an RDD in a simple format consisting of pickled Python objects. Batching is used on pickle serialization, with default batch size 10. - -* SequenceFile and Hadoop Input/Output Formats - -**Note** this feature is currently marked ```Experimental``` and is intended for advanced users. It may be replaced in future with read/write support based on Spark SQL, in which case Spark SQL is the preferred approach. - -**Writable Support** - -PySpark SequenceFile support loads an RDD of key-value pairs within Java, converts Writables to base Java types, and pickles the -resulting Java objects using [Pyrolite](https://github.com/irmen/Pyrolite/). When saving an RDD of key-value pairs to SequenceFile, -PySpark does the reverse. It unpickles Python objects into Java objects and then converts them to Writables. The following -Writables are automatically converted: - - - - - - - - - - - -
    Writable TypePython Type
    Textunicode str
    IntWritableint
    FloatWritablefloat
    DoubleWritablefloat
    BooleanWritablebool
    BytesWritablebytearray
    NullWritableNone
    MapWritabledict
    - -Arrays are not handled out-of-the-box. Users need to specify custom `ArrayWritable` subtypes when reading or writing. When writing, -users also need to specify custom converters that convert arrays to custom `ArrayWritable` subtypes. When reading, the default -converter will convert custom `ArrayWritable` subtypes to Java `Object[]`, which then get pickled to Python tuples. To get -Python `array.array` for arrays of primitive types, users need to specify custom converters. - -**Saving and Loading SequenceFiles** - -Similarly to text files, SequenceFiles can be saved and loaded by specifying the path. The key and value -classes can be specified, but for standard Writables this is not required. - -{% highlight python %} ->>> rdd = sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x )) ->>> rdd.saveAsSequenceFile("path/to/file") ->>> sorted(sc.sequenceFile("path/to/file").collect()) -[(1, u'a'), (2, u'aa'), (3, u'aaa')] -{% endhighlight %} - -**Saving and Loading Other Hadoop Input/Output Formats** - -PySpark can also read any Hadoop InputFormat or write any Hadoop OutputFormat, for both 'new' and 'old' Hadoop MapReduce APIs. -If required, a Hadoop configuration can be passed in as a Python dict. Here is an example using the -Elasticsearch ESInputFormat: - -{% highlight python %} -$ SPARK_CLASSPATH=/path/to/elasticsearch-hadoop.jar ./bin/pyspark ->>> conf = {"es.resource" : "index/type"} # assume Elasticsearch is running on localhost defaults ->>> rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat",\ - "org.apache.hadoop.io.NullWritable", "org.elasticsearch.hadoop.mr.LinkedMapWritable", conf=conf) ->>> rdd.first() # the result is a MapWritable that is converted to a Python dict -(u'Elasticsearch ID', - {u'field1': True, - u'field2': u'Some Text', - u'field3': 12345}) -{% endhighlight %} - -Note that, if the InputFormat simply depends on a Hadoop configuration and/or input path, and -the key and value classes can easily be converted according to the above table, -then this approach should work well for such cases. - -If you have custom serialized binary data (such as loading data from Cassandra / HBase), then you will first need to -transform that data on the Scala/Java side to something which can be handled by Pyrolite's pickler. -A [Converter](api/scala/index.html#org.apache.spark.api.python.Converter) trait is provided -for this. Simply extend this trait and implement your transformation code in the ```convert``` -method. Remember to ensure that this class, along with any dependencies required to access your ```InputFormat```, are packaged into your Spark job jar and included on the PySpark -classpath. - -See the [Python examples]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python) and -the [Converter examples]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/pythonconverters) -for examples of using Cassandra / HBase ```InputFormat``` and ```OutputFormat``` with custom converters. - -
    -
    - -## RDD Operations - -RDDs support two types of operations: *transformations*, which create a new dataset from an existing one, and *actions*, which return a value to the driver program after running a computation on the dataset. For example, `map` is a transformation that passes each dataset element through a function and returns a new RDD representing the results. On the other hand, `reduce` is an action that aggregates all the elements of the RDD using some function and returns the final result to the driver program (although there is also a parallel `reduceByKey` that returns a distributed dataset). - -All transformations in Spark are lazy, in that they do not compute their results right away. Instead, they just remember the transformations applied to some base dataset (e.g. a file). The transformations are only computed when an action requires a result to be returned to the driver program. This design enables Spark to run more efficiently -- for example, we can realize that a dataset created through `map` will be used in a `reduce` and return only the result of the `reduce` to the driver, rather than the larger mapped dataset. - -By default, each transformed RDD may be recomputed each time you run an action on it. However, you may also *persist* an RDD in memory using the `persist` (or `cache`) method, in which case Spark will keep the elements around on the cluster for much faster access the next time you query it. There is also support for persisting RDDs on disk, or replicated across multiple nodes. - -### Basics - -
    - -
    - -To illustrate RDD basics, consider the simple program below: - -{% highlight scala %} -val lines = sc.textFile("data.txt") -val lineLengths = lines.map(s => s.length) -val totalLength = lineLengths.reduce((a, b) => a + b) -{% endhighlight %} - -The first line defines a base RDD from an external file. This dataset is not loaded in memory or -otherwise acted on: `lines` is merely a pointer to the file. -The second line defines `lineLengths` as the result of a `map` transformation. Again, `lineLengths` -is *not* immediately computed, due to laziness. -Finally, we run `reduce`, which is an action. At this point Spark breaks the computation into tasks -to run on separate machines, and each machine runs both its part of the map and a local reduction, -returning only its answer to the driver program. - -If we also wanted to use `lineLengths` again later, we could add: - -{% highlight scala %} -lineLengths.persist() -{% endhighlight %} - -before the `reduce`, which would cause `lineLengths` to be saved in memory after the first time it is computed. - -
    - -
    - -To illustrate RDD basics, consider the simple program below: - -{% highlight java %} -JavaRDD lines = sc.textFile("data.txt"); -JavaRDD lineLengths = lines.map(s -> s.length()); -int totalLength = lineLengths.reduce((a, b) -> a + b); -{% endhighlight %} - -The first line defines a base RDD from an external file. This dataset is not loaded in memory or -otherwise acted on: `lines` is merely a pointer to the file. -The second line defines `lineLengths` as the result of a `map` transformation. Again, `lineLengths` -is *not* immediately computed, due to laziness. -Finally, we run `reduce`, which is an action. At this point Spark breaks the computation into tasks -to run on separate machines, and each machine runs both its part of the map and a local reduction, -returning only its answer to the driver program. - -If we also wanted to use `lineLengths` again later, we could add: - -{% highlight java %} -lineLengths.persist(StorageLevel.MEMORY_ONLY()); -{% endhighlight %} - -before the `reduce`, which would cause `lineLengths` to be saved in memory after the first time it is computed. - -
    - -
    - -To illustrate RDD basics, consider the simple program below: - -{% highlight python %} -lines = sc.textFile("data.txt") -lineLengths = lines.map(lambda s: len(s)) -totalLength = lineLengths.reduce(lambda a, b: a + b) -{% endhighlight %} - -The first line defines a base RDD from an external file. This dataset is not loaded in memory or -otherwise acted on: `lines` is merely a pointer to the file. -The second line defines `lineLengths` as the result of a `map` transformation. Again, `lineLengths` -is *not* immediately computed, due to laziness. -Finally, we run `reduce`, which is an action. At this point Spark breaks the computation into tasks -to run on separate machines, and each machine runs both its part of the map and a local reduction, -returning only its answer to the driver program. - -If we also wanted to use `lineLengths` again later, we could add: - -{% highlight python %} -lineLengths.persist() -{% endhighlight %} - -before the `reduce`, which would cause `lineLengths` to be saved in memory after the first time it is computed. - -
    - -
    - -### Passing Functions to Spark - -
    - -
    - -Spark's API relies heavily on passing functions in the driver program to run on the cluster. -There are two recommended ways to do this: - -* [Anonymous function syntax](http://docs.scala-lang.org/tutorials/tour/anonymous-function-syntax.html), - which can be used for short pieces of code. -* Static methods in a global singleton object. For example, you can define `object MyFunctions` and then - pass `MyFunctions.func1`, as follows: - -{% highlight scala %} -object MyFunctions { - def func1(s: String): String = { ... } -} - -myRdd.map(MyFunctions.func1) -{% endhighlight %} - -Note that while it is also possible to pass a reference to a method in a class instance (as opposed to -a singleton object), this requires sending the object that contains that class along with the method. -For example, consider: - -{% highlight scala %} -class MyClass { - def func1(s: String): String = { ... } - def doStuff(rdd: RDD[String]): RDD[String] = { rdd.map(func1) } -} -{% endhighlight %} - -Here, if we create a `new MyClass` and call `doStuff` on it, the `map` inside there references the -`func1` method *of that `MyClass` instance*, so the whole object needs to be sent to the cluster. It is -similar to writing `rdd.map(x => this.func1(x))`. - -In a similar way, accessing fields of the outer object will reference the whole object: - -{% highlight scala %} -class MyClass { - val field = "Hello" - def doStuff(rdd: RDD[String]): RDD[String] = { rdd.map(x => field + x) } -} -{% endhighlight %} - -is equilvalent to writing `rdd.map(x => this.field + x)`, which references all of `this`. To avoid this -issue, the simplest way is to copy `field` into a local variable instead of accessing it externally: - -{% highlight scala %} -def doStuff(rdd: RDD[String]): RDD[String] = { - val field_ = this.field - rdd.map(x => field_ + x) -} -{% endhighlight %} - -
    - -
    - -Spark's API relies heavily on passing functions in the driver program to run on the cluster. -In Java, functions are represented by classes implementing the interfaces in the -[org.apache.spark.api.java.function](api/java/index.html?org/apache/spark/api/java/function/package-summary.html) package. -There are two ways to create such functions: - -* Implement the Function interfaces in your own class, either as an anonymous inner class or a named one, - and pass an instance of it to Spark. -* In Java 8, use [lambda expressions](http://docs.oracle.com/javase/tutorial/java/javaOO/lambdaexpressions.html) - to concisely define an implementation. - -While much of this guide uses lambda syntax for conciseness, it is easy to use all the same APIs -in long-form. For example, we could have written our code above as follows: - -{% highlight java %} -JavaRDD lines = sc.textFile("data.txt"); -JavaRDD lineLengths = lines.map(new Function() { - public Integer call(String s) { return s.length(); } -}); -int totalLength = lineLengths.reduce(new Function2() { - public Integer call(Integer a, Integer b) { return a + b; } -}); -{% endhighlight %} - -Or, if writing the functions inline is unwieldy: - -{% highlight java %} -class GetLength implements Function { - public Integer call(String s) { return s.length(); } -} -class Sum implements Function2 { - public Integer call(Integer a, Integer b) { return a + b; } -} - -JavaRDD lines = sc.textFile("data.txt"); -JavaRDD lineLengths = lines.map(new GetLength()); -int totalLength = lineLengths.reduce(new Sum()); -{% endhighlight %} - -Note that anonymous inner classes in Java can also access variables in the enclosing scope as long -as they are marked `final`. Spark will ship copies of these variables to each worker node as it does -for other languages. - -
    - -
    - -Spark's API relies heavily on passing functions in the driver program to run on the cluster. -There are three recommended ways to do this: - -* [Lambda expressions](https://docs.python.org/2/tutorial/controlflow.html#lambda-expressions), - for simple functions that can be written as an expression. (Lambdas do not support multi-statement - functions or statements that do not return a value.) -* Local `def`s inside the function calling into Spark, for longer code. -* Top-level functions in a module. - -For example, to pass a longer function than can be supported using a `lambda`, consider -the code below: - -{% highlight python %} -"""MyScript.py""" -if __name__ == "__main__": - def myFunc(s): - words = s.split(" ") - return len(words) - - sc = SparkContext(...) - sc.textFile("file.txt").map(myFunc) -{% endhighlight %} - -Note that while it is also possible to pass a reference to a method in a class instance (as opposed to -a singleton object), this requires sending the object that contains that class along with the method. -For example, consider: - -{% highlight python %} -class MyClass(object): - def func(self, s): - return s - def doStuff(self, rdd): - return rdd.map(self.func) -{% endhighlight %} - -Here, if we create a `new MyClass` and call `doStuff` on it, the `map` inside there references the -`func` method *of that `MyClass` instance*, so the whole object needs to be sent to the cluster. - -In a similar way, accessing fields of the outer object will reference the whole object: - -{% highlight python %} -class MyClass(object): - def __init__(self): - self.field = "Hello" - def doStuff(self, rdd): - return rdd.map(lambda s: self.field + s) -{% endhighlight %} - -To avoid this issue, the simplest way is to copy `field` into a local variable instead -of accessing it externally: - -{% highlight python %} -def doStuff(self, rdd): - field = self.field - return rdd.map(lambda s: field + s) -{% endhighlight %} - -
    - -
    - -### Understanding closures -One of the harder things about Spark is understanding the scope and life cycle of variables and methods when executing code across a cluster. RDD operations that modify variables outside of their scope can be a frequent source of confusion. In the example below we'll look at code that uses `foreach()` to increment a counter, but similar issues can occur for other operations as well. - -#### Example - -Consider the naive RDD element sum below, which behaves completely differently depending on whether execution is happening within the same JVM. A common example of this is when running Spark in `local` mode (`--master = local[n]`) versus deploying a Spark application to a cluster (e.g. via spark-submit to YARN): - -
    - -
    -{% highlight scala %} -var counter = 0 -var rdd = sc.parallelize(data) - -// Wrong: Don't do this!! -rdd.foreach(x => counter += x) - -println("Counter value: " + counter) -{% endhighlight %} -
    - -
    -{% highlight java %} -int counter = 0; -JavaRDD rdd = sc.parallelize(data); - -// Wrong: Don't do this!! -rdd.foreach(x -> counter += x); - -println("Counter value: " + counter); -{% endhighlight %} -
    - -
    -{% highlight python %} -counter = 0 -rdd = sc.parallelize(data) - -# Wrong: Don't do this!! -rdd.foreach(lambda x: counter += x) - -print("Counter value: " + counter) - -{% endhighlight %} -
    - -
    - -#### Local vs. cluster modes - -The primary challenge is that the behavior of the above code is undefined. In local mode with a single JVM, the above code will sum the values within the RDD and store it in **counter**. This is because both the RDD and the variable **counter** are in the same memory space on the driver node. - -However, in `cluster` mode, what happens is more complicated, and the above may not work as intended. To execute jobs, Spark breaks up the processing of RDD operations into tasks - each of which is operated on by an executor. Prior to execution, Spark computes the **closure**. The closure is those variables and methods which must be visible for the executor to perform its computations on the RDD (in this case `foreach()`). This closure is serialized and sent to each executor. In `local` mode, there is only the one executors so everything shares the same closure. In other modes however, this is not the case and the executors running on separate worker nodes each have their own copy of the closure. - -What is happening here is that the variables within the closure sent to each executor are now copies and thus, when **counter** is referenced within the `foreach` function, it's no longer the **counter** on the driver node. There is still a **counter** in the memory of the driver node but this is no longer visible to the executors! The executors only see the copy from the serialized closure. Thus, the final value of **counter** will still be zero since all operations on **counter** were referencing the value within the serialized closure. - -To ensure well-defined behavior in these sorts of scenarios one should use an [`Accumulator`](#AccumLink). Accumulators in Spark are used specifically to provide a mechanism for safely updating a variable when execution is split up across worker nodes in a cluster. The Accumulators section of this guide discusses these in more detail. - -In general, closures - constructs like loops or locally defined methods, should not be used to mutate some global state. Spark does not define or guarantee the behavior of mutations to objects referenced from outside of closures. Some code that does this may work in local mode, but that's just by accident and such code will not behave as expected in distributed mode. Use an Accumulator instead if some global aggregation is needed. - -#### Printing elements of an RDD -Another common idiom is attempting to print out the elements of an RDD using `rdd.foreach(println)` or `rdd.map(println)`. On a single machine, this will generate the expected output and print all the RDD's elements. However, in `cluster` mode, the output to `stdout` being called by the executors is now writing to the executor's `stdout` instead, not the one on the driver, so `stdout` on the driver won't show these! To print all elements on the driver, one can use the `collect()` method to first bring the RDD to the driver node thus: `rdd.collect().foreach(println)`. This can cause the driver to run out of memory, though, because `collect()` fetches the entire RDD to a single machine; if you only need to print a few elements of the RDD, a safer approach is to use the `take()`: `rdd.take(100).foreach(println)`. - -### Working with Key-Value Pairs - -
    - -
    - -While most Spark operations work on RDDs containing any type of objects, a few special operations are -only available on RDDs of key-value pairs. -The most common ones are distributed "shuffle" operations, such as grouping or aggregating the elements -by a key. - -In Scala, these operations are automatically available on RDDs containing -[Tuple2](http://www.scala-lang.org/api/{{site.SCALA_VERSION}}/index.html#scala.Tuple2) objects -(the built-in tuples in the language, created by simply writing `(a, b)`). The key-value pair operations are available in the -[PairRDDFunctions](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions) class, -which automatically wraps around an RDD of tuples. - -For example, the following code uses the `reduceByKey` operation on key-value pairs to count how -many times each line of text occurs in a file: - -{% highlight scala %} -val lines = sc.textFile("data.txt") -val pairs = lines.map(s => (s, 1)) -val counts = pairs.reduceByKey((a, b) => a + b) -{% endhighlight %} - -We could also use `counts.sortByKey()`, for example, to sort the pairs alphabetically, and finally -`counts.collect()` to bring them back to the driver program as an array of objects. - -**Note:** when using custom objects as the key in key-value pair operations, you must be sure that a -custom `equals()` method is accompanied with a matching `hashCode()` method. For full details, see -the contract outlined in the [Object.hashCode() -documentation](http://docs.oracle.com/javase/7/docs/api/java/lang/Object.html#hashCode()). - -
    - -
    - -While most Spark operations work on RDDs containing any type of objects, a few special operations are -only available on RDDs of key-value pairs. -The most common ones are distributed "shuffle" operations, such as grouping or aggregating the elements -by a key. - -In Java, key-value pairs are represented using the -[scala.Tuple2](http://www.scala-lang.org/api/{{site.SCALA_VERSION}}/index.html#scala.Tuple2) class -from the Scala standard library. You can simply call `new Tuple2(a, b)` to create a tuple, and access -its fields later with `tuple._1()` and `tuple._2()`. - -RDDs of key-value pairs are represented by the -[JavaPairRDD](api/java/index.html?org/apache/spark/api/java/JavaPairRDD.html) class. You can construct -JavaPairRDDs from JavaRDDs using special versions of the `map` operations, like -`mapToPair` and `flatMapToPair`. The JavaPairRDD will have both standard RDD functions and special -key-value ones. - -For example, the following code uses the `reduceByKey` operation on key-value pairs to count how -many times each line of text occurs in a file: - -{% highlight scala %} -JavaRDD lines = sc.textFile("data.txt"); -JavaPairRDD pairs = lines.mapToPair(s -> new Tuple2(s, 1)); -JavaPairRDD counts = pairs.reduceByKey((a, b) -> a + b); -{% endhighlight %} - -We could also use `counts.sortByKey()`, for example, to sort the pairs alphabetically, and finally -`counts.collect()` to bring them back to the driver program as an array of objects. - -**Note:** when using custom objects as the key in key-value pair operations, you must be sure that a -custom `equals()` method is accompanied with a matching `hashCode()` method. For full details, see -the contract outlined in the [Object.hashCode() -documentation](http://docs.oracle.com/javase/7/docs/api/java/lang/Object.html#hashCode()). - -
    - -
    - -While most Spark operations work on RDDs containing any type of objects, a few special operations are -only available on RDDs of key-value pairs. -The most common ones are distributed "shuffle" operations, such as grouping or aggregating the elements -by a key. - -In Python, these operations work on RDDs containing built-in Python tuples such as `(1, 2)`. -Simply create such tuples and then call your desired operation. - -For example, the following code uses the `reduceByKey` operation on key-value pairs to count how -many times each line of text occurs in a file: - -{% highlight python %} -lines = sc.textFile("data.txt") -pairs = lines.map(lambda s: (s, 1)) -counts = pairs.reduceByKey(lambda a, b: a + b) -{% endhighlight %} - -We could also use `counts.sortByKey()`, for example, to sort the pairs alphabetically, and finally -`counts.collect()` to bring them back to the driver program as a list of objects. - -
    - -
    - - -### Transformations - -The following table lists some of the common transformations supported by Spark. Refer to the -RDD API doc -([Scala](api/scala/index.html#org.apache.spark.rdd.RDD), - [Java](api/java/index.html?org/apache/spark/api/java/JavaRDD.html), - [Python](api/python/pyspark.html#pyspark.RDD), - [R](api/R/index.html)) -and pair RDD functions doc -([Scala](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions), - [Java](api/java/index.html?org/apache/spark/api/java/JavaPairRDD.html)) -for details. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    TransformationMeaning
    map(func) Return a new distributed dataset formed by passing each element of the source through a function func.
    filter(func) Return a new dataset formed by selecting those elements of the source on which func returns true.
    flatMap(func) Similar to map, but each input item can be mapped to 0 or more output items (so func should return a Seq rather than a single item).
    mapPartitions(func) Similar to map, but runs separately on each partition (block) of the RDD, so func must be of type - Iterator<T> => Iterator<U> when running on an RDD of type T.
    mapPartitionsWithIndex(func) Similar to mapPartitions, but also provides func with an integer value representing the index of - the partition, so func must be of type (Int, Iterator<T>) => Iterator<U> when running on an RDD of type T. -
    sample(withReplacement, fraction, seed) Sample a fraction fraction of the data, with or without replacement, using a given random number generator seed.
    union(otherDataset) Return a new dataset that contains the union of the elements in the source dataset and the argument.
    intersection(otherDataset) Return a new RDD that contains the intersection of elements in the source dataset and the argument.
    distinct([numTasks])) Return a new dataset that contains the distinct elements of the source dataset.
    groupByKey([numTasks]) When called on a dataset of (K, V) pairs, returns a dataset of (K, Iterable<V>) pairs.
    - Note: If you are grouping in order to perform an aggregation (such as a sum or - average) over each key, using reduceByKey or aggregateByKey will yield much better - performance. -
    - Note: By default, the level of parallelism in the output depends on the number of partitions of the parent RDD. - You can pass an optional numTasks argument to set a different number of tasks. -
    reduceByKey(func, [numTasks]) When called on a dataset of (K, V) pairs, returns a dataset of (K, V) pairs where the values for each key are aggregated using the given reduce function func, which must be of type (V,V) => V. Like in groupByKey, the number of reduce tasks is configurable through an optional second argument.
    aggregateByKey(zeroValue)(seqOp, combOp, [numTasks]) When called on a dataset of (K, V) pairs, returns a dataset of (K, U) pairs where the values for each key are aggregated using the given combine functions and a neutral "zero" value. Allows an aggregated value type that is different than the input value type, while avoiding unnecessary allocations. Like in groupByKey, the number of reduce tasks is configurable through an optional second argument.
    sortByKey([ascending], [numTasks]) When called on a dataset of (K, V) pairs where K implements Ordered, returns a dataset of (K, V) pairs sorted by keys in ascending or descending order, as specified in the boolean ascending argument.
    join(otherDataset, [numTasks]) When called on datasets of type (K, V) and (K, W), returns a dataset of (K, (V, W)) pairs with all pairs of elements for each key. - Outer joins are supported through leftOuterJoin, rightOuterJoin, and fullOuterJoin. -
    cogroup(otherDataset, [numTasks]) When called on datasets of type (K, V) and (K, W), returns a dataset of (K, (Iterable<V>, Iterable<W>)) tuples. This operation is also called groupWith.
    cartesian(otherDataset) When called on datasets of types T and U, returns a dataset of (T, U) pairs (all pairs of elements).
    pipe(command, [envVars]) Pipe each partition of the RDD through a shell command, e.g. a Perl or bash script. RDD elements are written to the - process's stdin and lines output to its stdout are returned as an RDD of strings.
    coalesce(numPartitions) Decrease the number of partitions in the RDD to numPartitions. Useful for running operations more efficiently - after filtering down a large dataset.
    repartition(numPartitions) Reshuffle the data in the RDD randomly to create either more or fewer partitions and balance it across them. - This always shuffles all data over the network.
    repartitionAndSortWithinPartitions(partitioner) Repartition the RDD according to the given partitioner and, within each resulting partition, - sort records by their keys. This is more efficient than calling repartition and then sorting within - each partition because it can push the sorting down into the shuffle machinery.
    - -### Actions - -The following table lists some of the common actions supported by Spark. Refer to the -RDD API doc -([Scala](api/scala/index.html#org.apache.spark.rdd.RDD), - [Java](api/java/index.html?org/apache/spark/api/java/JavaRDD.html), - [Python](api/python/pyspark.html#pyspark.RDD), - [R](api/R/index.html)) - -and pair RDD functions doc -([Scala](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions), - [Java](api/java/index.html?org/apache/spark/api/java/JavaPairRDD.html)) -for details. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    ActionMeaning
    reduce(func) Aggregate the elements of the dataset using a function func (which takes two arguments and returns one). The function should be commutative and associative so that it can be computed correctly in parallel.
    collect() Return all the elements of the dataset as an array at the driver program. This is usually useful after a filter or other operation that returns a sufficiently small subset of the data.
    count() Return the number of elements in the dataset.
    first() Return the first element of the dataset (similar to take(1)).
    take(n) Return an array with the first n elements of the dataset.
    takeSample(withReplacement, num, [seed]) Return an array with a random sample of num elements of the dataset, with or without replacement, optionally pre-specifying a random number generator seed.
    takeOrdered(n, [ordering]) Return the first n elements of the RDD using either their natural order or a custom comparator.
    saveAsTextFile(path) Write the elements of the dataset as a text file (or set of text files) in a given directory in the local filesystem, HDFS or any other Hadoop-supported file system. Spark will call toString on each element to convert it to a line of text in the file.
    saveAsSequenceFile(path)
    (Java and Scala)
    Write the elements of the dataset as a Hadoop SequenceFile in a given path in the local filesystem, HDFS or any other Hadoop-supported file system. This is available on RDDs of key-value pairs that implement Hadoop's Writable interface. In Scala, it is also - available on types that are implicitly convertible to Writable (Spark includes conversions for basic types like Int, Double, String, etc).
    saveAsObjectFile(path)
    (Java and Scala)
    Write the elements of the dataset in a simple format using Java serialization, which can then be loaded using - SparkContext.objectFile().
    countByKey() Only available on RDDs of type (K, V). Returns a hashmap of (K, Int) pairs with the count of each key.
    foreach(func) Run a function func on each element of the dataset. This is usually done for side effects such as updating an Accumulator or interacting with external storage systems. -
    Note: modifying variables other than Accumulators outside of the foreach() may result in undefined behavior. See Understanding closures for more details.
    - -### Shuffle operations - -Certain operations within Spark trigger an event known as the shuffle. The shuffle is Spark's -mechanism for re-distributing data so that it's grouped differently across partitions. This typically -involves copying data across executors and machines, making the shuffle a complex and -costly operation. - -#### Background - -To understand what happens during the shuffle we can consider the example of the -[`reduceByKey`](#ReduceByLink) operation. The `reduceByKey` operation generates a new RDD where all -values for a single key are combined into a tuple - the key and the result of executing a reduce -function against all values associated with that key. The challenge is that not all values for a -single key necessarily reside on the same partition, or even the same machine, but they must be -co-located to compute the result. - -In Spark, data is generally not distributed across partitions to be in the necessary place for a -specific operation. During computations, a single task will operate on a single partition - thus, to -organize all the data for a single `reduceByKey` reduce task to execute, Spark needs to perform an -all-to-all operation. It must read from all partitions to find all the values for all keys, -and then bring together values across partitions to compute the final result for each key - -this is called the **shuffle**. - -Although the set of elements in each partition of newly shuffled data will be deterministic, and so -is the ordering of partitions themselves, the ordering of these elements is not. If one desires predictably -ordered data following shuffle then it's possible to use: - -* `mapPartitions` to sort each partition using, for example, `.sorted` -* `repartitionAndSortWithinPartitions` to efficiently sort partitions while simultaneously repartitioning -* `sortBy` to make a globally ordered RDD - -Operations which can cause a shuffle include **repartition** operations like -[`repartition`](#RepartitionLink) and [`coalesce`](#CoalesceLink), **'ByKey** operations -(except for counting) like [`groupByKey`](#GroupByLink) and [`reduceByKey`](#ReduceByLink), and -**join** operations like [`cogroup`](#CogroupLink) and [`join`](#JoinLink). - -#### Performance Impact -The **Shuffle** is an expensive operation since it involves disk I/O, data serialization, and -network I/O. To organize data for the shuffle, Spark generates sets of tasks - *map* tasks to -organize the data, and a set of *reduce* tasks to aggregate it. This nomenclature comes from -MapReduce and does not directly relate to Spark's `map` and `reduce` operations. - -Internally, results from individual map tasks are kept in memory until they can't fit. Then, these -are sorted based on the target partition and written to a single file. On the reduce side, tasks -read the relevant sorted blocks. - -Certain shuffle operations can consume significant amounts of heap memory since they employ -in-memory data structures to organize records before or after transferring them. Specifically, -`reduceByKey` and `aggregateByKey` create these structures on the map side, and `'ByKey` operations -generate these on the reduce side. When data does not fit in memory Spark will spill these tables -to disk, incurring the additional overhead of disk I/O and increased garbage collection. - -Shuffle also generates a large number of intermediate files on disk. As of Spark 1.3, these files -are preserved until the corresponding RDDs are no longer used and are garbage collected. -This is done so the shuffle files don't need to be re-created if the lineage is re-computed. -Garbage collection may happen only after a long period time, if the application retains references -to these RDDs or if GC does not kick in frequently. This means that long-running Spark jobs may -consume a large amount of disk space. The temporary storage directory is specified by the -`spark.local.dir` configuration parameter when configuring the Spark context. - -Shuffle behavior can be tuned by adjusting a variety of configuration parameters. See the -'Shuffle Behavior' section within the [Spark Configuration Guide](configuration.html). - -## RDD Persistence - -One of the most important capabilities in Spark is *persisting* (or *caching*) a dataset in memory -across operations. When you persist an RDD, each node stores any partitions of it that it computes in -memory and reuses them in other actions on that dataset (or datasets derived from it). This allows -future actions to be much faster (often by more than 10x). Caching is a key tool for -iterative algorithms and fast interactive use. - -You can mark an RDD to be persisted using the `persist()` or `cache()` methods on it. The first time -it is computed in an action, it will be kept in memory on the nodes. Spark's cache is fault-tolerant -- -if any partition of an RDD is lost, it will automatically be recomputed using the transformations -that originally created it. - -In addition, each persisted RDD can be stored using a different *storage level*, allowing you, for example, -to persist the dataset on disk, persist it in memory but as serialized Java objects (to save space), -replicate it across nodes, or store it off-heap in [Tachyon](http://tachyon-project.org/). -These levels are set by passing a -`StorageLevel` object ([Scala](api/scala/index.html#org.apache.spark.storage.StorageLevel), -[Java](api/java/index.html?org/apache/spark/storage/StorageLevel.html), -[Python](api/python/pyspark.html#pyspark.StorageLevel)) -to `persist()`. The `cache()` method is a shorthand for using the default storage level, -which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The full set of -storage levels is: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Storage LevelMeaning
    MEMORY_ONLY Store RDD as deserialized Java objects in the JVM. If the RDD does not fit in memory, some partitions will - not be cached and will be recomputed on the fly each time they're needed. This is the default level.
    MEMORY_AND_DISK Store RDD as deserialized Java objects in the JVM. If the RDD does not fit in memory, store the - partitions that don't fit on disk, and read them from there when they're needed.
    MEMORY_ONLY_SER Store RDD as serialized Java objects (one byte array per partition). - This is generally more space-efficient than deserialized objects, especially when using a - fast serializer, but more CPU-intensive to read. -
    MEMORY_AND_DISK_SER Similar to MEMORY_ONLY_SER, but spill partitions that don't fit in memory to disk instead of - recomputing them on the fly each time they're needed.
    DISK_ONLY Store the RDD partitions only on disk.
    MEMORY_ONLY_2, MEMORY_AND_DISK_2, etc. Same as the levels above, but replicate each partition on two cluster nodes.
    OFF_HEAP (experimental) Store RDD in serialized format in Tachyon. - Compared to MEMORY_ONLY_SER, OFF_HEAP reduces garbage collection overhead and allows executors - to be smaller and to share a pool of memory, making it attractive in environments with - large heaps or multiple concurrent applications. Furthermore, as the RDDs reside in Tachyon, - the crash of an executor does not lead to losing the in-memory cache. In this mode, the memory - in Tachyon is discardable. Thus, Tachyon does not attempt to reconstruct a block that it evicts - from memory. If you plan to use Tachyon as the off heap store, Spark is compatible with Tachyon - out-of-the-box. Please refer to this page - for the suggested version pairings. -
    - -**Note:** *In Python, stored objects will always be serialized with the [Pickle](https://docs.python.org/2/library/pickle.html) library, so it does not matter whether you choose a serialized level.* - -Spark also automatically persists some intermediate data in shuffle operations (e.g. `reduceByKey`), even without users calling `persist`. This is done to avoid recomputing the entire input if a node fails during the shuffle. We still recommend users call `persist` on the resulting RDD if they plan to reuse it. - -### Which Storage Level to Choose? - -Spark's storage levels are meant to provide different trade-offs between memory usage and CPU -efficiency. We recommend going through the following process to select one: - -* If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY`), leave them that way. - This is the most CPU-efficient option, allowing operations on the RDDs to run as fast as possible. - -* If not, try using `MEMORY_ONLY_SER` and [selecting a fast serialization library](tuning.html) to -make the objects much more space-efficient, but still reasonably fast to access. - -* Don't spill to disk unless the functions that computed your datasets are expensive, or they filter -a large amount of the data. Otherwise, recomputing a partition may be as fast as reading it from -disk. - -* Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve -requests from a web application). *All* the storage levels provide full fault tolerance by -recomputing lost data, but the replicated ones let you continue running tasks on the RDD without -waiting to recompute a lost partition. - -* In environments with high amounts of memory or multiple applications, the experimental `OFF_HEAP` -mode has several advantages: - * It allows multiple executors to share the same pool of memory in Tachyon. - * It significantly reduces garbage collection costs. - * Cached data is not lost if individual executors crash. - -### Removing Data - -Spark automatically monitors cache usage on each node and drops out old data partitions in a -least-recently-used (LRU) fashion. If you would like to manually remove an RDD instead of waiting for -it to fall out of the cache, use the `RDD.unpersist()` method. - -# Shared Variables - -Normally, when a function passed to a Spark operation (such as `map` or `reduce`) is executed on a -remote cluster node, it works on separate copies of all the variables used in the function. These -variables are copied to each machine, and no updates to the variables on the remote machine are -propagated back to the driver program. Supporting general, read-write shared variables across tasks -would be inefficient. However, Spark does provide two limited types of *shared variables* for two -common usage patterns: broadcast variables and accumulators. - -## Broadcast Variables - -Broadcast variables allow the programmer to keep a read-only variable cached on each machine rather -than shipping a copy of it with tasks. They can be used, for example, to give every node a copy of a -large input dataset in an efficient manner. Spark also attempts to distribute broadcast variables -using efficient broadcast algorithms to reduce communication cost. - -Spark actions are executed through a set of stages, separated by distributed "shuffle" operations. -Spark automatically broadcasts the common data needed by tasks within each stage. The data -broadcasted this way is cached in serialized form and deserialized before running each task. This -means that explicitly creating broadcast variables is only useful when tasks across multiple stages -need the same data or when caching the data in deserialized form is important. - -Broadcast variables are created from a variable `v` by calling `SparkContext.broadcast(v)`. The -broadcast variable is a wrapper around `v`, and its value can be accessed by calling the `value` -method. The code below shows this: - -
    - -
    - -{% highlight scala %} -scala> val broadcastVar = sc.broadcast(Array(1, 2, 3)) -broadcastVar: org.apache.spark.broadcast.Broadcast[Array[Int]] = Broadcast(0) - -scala> broadcastVar.value -res0: Array[Int] = Array(1, 2, 3) -{% endhighlight %} - -
    - -
    - -{% highlight java %} -Broadcast broadcastVar = sc.broadcast(new int[] {1, 2, 3}); - -broadcastVar.value(); -// returns [1, 2, 3] -{% endhighlight %} - -
    - -
    - -{% highlight python %} ->>> broadcastVar = sc.broadcast([1, 2, 3]) - - ->>> broadcastVar.value -[1, 2, 3] -{% endhighlight %} - -
    - -
    - -After the broadcast variable is created, it should be used instead of the value `v` in any functions -run on the cluster so that `v` is not shipped to the nodes more than once. In addition, the object -`v` should not be modified after it is broadcast in order to ensure that all nodes get the same -value of the broadcast variable (e.g. if the variable is shipped to a new node later). - -## Accumulators - -Accumulators are variables that are only "added" to through an associative operation and can -therefore be efficiently supported in parallel. They can be used to implement counters (as in -MapReduce) or sums. Spark natively supports accumulators of numeric types, and programmers -can add support for new types. If accumulators are created with a name, they will be -displayed in Spark's UI. This can be useful for understanding the progress of -running stages (NOTE: this is not yet supported in Python). - -An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks -running on the cluster can then add to it using the `add` method or the `+=` operator (in Scala and Python). -However, they cannot read its value. -Only the driver program can read the accumulator's value, using its `value` method. - -The code below shows an accumulator being used to add up the elements of an array: - -
    - -
    - -{% highlight scala %} -scala> val accum = sc.accumulator(0, "My Accumulator") -accum: spark.Accumulator[Int] = 0 - -scala> sc.parallelize(Array(1, 2, 3, 4)).foreach(x => accum += x) -... -10/09/29 18:41:08 INFO SparkContext: Tasks finished in 0.317106 s - -scala> accum.value -res2: Int = 10 -{% endhighlight %} - -While this code used the built-in support for accumulators of type Int, programmers can also -create their own types by subclassing [AccumulatorParam](api/scala/index.html#org.apache.spark.AccumulatorParam). -The AccumulatorParam interface has two methods: `zero` for providing a "zero value" for your data -type, and `addInPlace` for adding two values together. For example, supposing we had a `Vector` class -representing mathematical vectors, we could write: - -{% highlight scala %} -object VectorAccumulatorParam extends AccumulatorParam[Vector] { - def zero(initialValue: Vector): Vector = { - Vector.zeros(initialValue.size) - } - def addInPlace(v1: Vector, v2: Vector): Vector = { - v1 += v2 - } -} - -// Then, create an Accumulator of this type: -val vecAccum = sc.accumulator(new Vector(...))(VectorAccumulatorParam) -{% endhighlight %} - -In Scala, Spark also supports the more general [Accumulable](api/scala/index.html#org.apache.spark.Accumulable) -interface to accumulate data where the resulting type is not the same as the elements added (e.g. build -a list by collecting together elements), and the `SparkContext.accumulableCollection` method for accumulating -common Scala collection types. - -
    - -
    - -{% highlight java %} -Accumulator accum = sc.accumulator(0); - -sc.parallelize(Arrays.asList(1, 2, 3, 4)).foreach(x -> accum.add(x)); -// ... -// 10/09/29 18:41:08 INFO SparkContext: Tasks finished in 0.317106 s - -accum.value(); -// returns 10 -{% endhighlight %} - -While this code used the built-in support for accumulators of type Integer, programmers can also -create their own types by subclassing [AccumulatorParam](api/java/index.html?org/apache/spark/AccumulatorParam.html). -The AccumulatorParam interface has two methods: `zero` for providing a "zero value" for your data -type, and `addInPlace` for adding two values together. For example, supposing we had a `Vector` class -representing mathematical vectors, we could write: - -{% highlight java %} -class VectorAccumulatorParam implements AccumulatorParam { - public Vector zero(Vector initialValue) { - return Vector.zeros(initialValue.size()); - } - public Vector addInPlace(Vector v1, Vector v2) { - v1.addInPlace(v2); return v1; - } -} - -// Then, create an Accumulator of this type: -Accumulator vecAccum = sc.accumulator(new Vector(...), new VectorAccumulatorParam()); -{% endhighlight %} - -In Java, Spark also supports the more general [Accumulable](api/java/index.html?org/apache/spark/Accumulable.html) -interface to accumulate data where the resulting type is not the same as the elements added (e.g. build -a list by collecting together elements). - -
    - -
    - -{% highlight python %} ->>> accum = sc.accumulator(0) -Accumulator - ->>> sc.parallelize([1, 2, 3, 4]).foreach(lambda x: accum.add(x)) -... -10/09/29 18:41:08 INFO SparkContext: Tasks finished in 0.317106 s - -scala> accum.value -10 -{% endhighlight %} - -While this code used the built-in support for accumulators of type Int, programmers can also -create their own types by subclassing [AccumulatorParam](api/python/pyspark.html#pyspark.AccumulatorParam). -The AccumulatorParam interface has two methods: `zero` for providing a "zero value" for your data -type, and `addInPlace` for adding two values together. For example, supposing we had a `Vector` class -representing mathematical vectors, we could write: - -{% highlight python %} -class VectorAccumulatorParam(AccumulatorParam): - def zero(self, initialValue): - return Vector.zeros(initialValue.size) - - def addInPlace(self, v1, v2): - v1 += v2 - return v1 - -# Then, create an Accumulator of this type: -vecAccum = sc.accumulator(Vector(...), VectorAccumulatorParam()) -{% endhighlight %} - -
    - -
    - -For accumulator updates performed inside actions only, Spark guarantees that each task's update to the accumulator -will only be applied once, i.e. restarted tasks will not update the value. In transformations, users should be aware -of that each task's update may be applied more than once if tasks or job stages are re-executed. - -Accumulators do not change the lazy evaluation model of Spark. If they are being updated within an operation on an RDD, their value is only updated once that RDD is computed as part of an action. Consequently, accumulator updates are not guaranteed to be executed when made within a lazy transformation like `map()`. The below code fragment demonstrates this property: - -
    - -
    -{% highlight scala %} -val accum = sc.accumulator(0) -data.map { x => accum += x; f(x) } -// Here, accum is still 0 because no actions have caused the map to be computed. -{% endhighlight %} -
    - -
    -{% highlight java %} -Accumulator accum = sc.accumulator(0); -data.map(x -> { accum.add(x); return f(x); }); -// Here, accum is still 0 because no actions have caused the `map` to be computed. -{% endhighlight %} -
    - -
    -{% highlight python %} -accum = sc.accumulator(0) -def g(x): - accum.add(x) - return f(x) -data.map(g) -# Here, accum is still 0 because no actions have caused the `map` to be computed. -{% endhighlight %} -
    - -
    - -# Deploying to a Cluster - -The [application submission guide](submitting-applications.html) describes how to submit applications to a cluster. -In short, once you package your application into a JAR (for Java/Scala) or a set of `.py` or `.zip` files (for Python), -the `bin/spark-submit` script lets you submit it to any supported cluster manager. - -# Launching Spark jobs from Java / Scala - -The [org.apache.spark.launcher](api/java/index.html?org/apache/spark/launcher/package-summary.html) -package provides classes for launching Spark jobs as child processes using a simple Java API. - -# Unit Testing - -Spark is friendly to unit testing with any popular unit test framework. -Simply create a `SparkContext` in your test with the master URL set to `local`, run your operations, -and then call `SparkContext.stop()` to tear it down. -Make sure you stop the context within a `finally` block or the test framework's `tearDown` method, -as Spark does not support two contexts running concurrently in the same program. - -# Migrating from pre-1.0 Versions of Spark - -
    - -
    - -Spark 1.0 freezes the API of Spark Core for the 1.X series, in that any API available today that is -not marked "experimental" or "developer API" will be supported in future versions. -The only change for Scala users is that the grouping operations, e.g. `groupByKey`, `cogroup` and `join`, -have changed from returning `(Key, Seq[Value])` pairs to `(Key, Iterable[Value])`. - -
    - -
    - -Spark 1.0 freezes the API of Spark Core for the 1.X series, in that any API available today that is -not marked "experimental" or "developer API" will be supported in future versions. -Several changes were made to the Java API: - -* The Function classes in `org.apache.spark.api.java.function` became interfaces in 1.0, meaning that old - code that `extends Function` should `implement Function` instead. -* New variants of the `map` transformations, like `mapToPair` and `mapToDouble`, were added to create RDDs - of special data types. -* Grouping operations like `groupByKey`, `cogroup` and `join` have changed from returning - `(Key, List)` pairs to `(Key, Iterable)`. - -
    - -
    - -Spark 1.0 freezes the API of Spark Core for the 1.X series, in that any API available today that is -not marked "experimental" or "developer API" will be supported in future versions. -The only change for Python users is that the grouping operations, e.g. `groupByKey`, `cogroup` and `join`, -have changed from returning (key, list of values) pairs to (key, iterable of values). - -
    - -
    - -Migration guides are also available for [Spark Streaming](streaming-programming-guide.html#migration-guide-from-091-or-below-to-1x), -[MLlib](mllib-guide.html#migration-guide) and [GraphX](graphx-programming-guide.html#migrating-from-spark-091). - - -# Where to Go from Here - -You can see some [example Spark programs](http://spark.apache.org/examples.html) on the Spark website. -In addition, Spark includes several samples in the `examples` directory -([Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples), - [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples), - [Python]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python), - [R]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/r)). -You can run Java and Scala examples by passing the class name to Spark's `bin/run-example` script; for instance: - - ./bin/run-example SparkPi - -For Python examples, use `spark-submit` instead: - - ./bin/spark-submit examples/src/main/python/pi.py - -For R examples, use `spark-submit` instead: - - ./bin/spark-submit examples/src/main/r/dataframe.R - -For help on optimizing your programs, the [configuration](configuration.html) and -[tuning](tuning.html) guides provide information on best practices. They are especially important for -making sure that your data is stored in memory in an efficient format. -For help on deploying, the [cluster mode overview](cluster-overview.html) describes the components involved -in distributed operation and supported cluster managers. - -Finally, full API documentation is available in -[Scala](api/scala/#org.apache.spark.package), [Java](api/java/), [Python](api/python/) and [R](api/R/). +This document has moved [here](rdd-programming-guide.html). diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md deleted file mode 100644 index 68f04b50aaa0..000000000000 --- a/docs/python-programming-guide.md +++ /dev/null @@ -1,7 +0,0 @@ ---- -layout: global -title: Python Programming Guide -redirect: programming-guide.html ---- - -This document has been merged into the [Spark programming guide](programming-guide.html). diff --git a/docs/quick-start.md b/docs/quick-start.md index d481fe0ea6d7..200b97230e86 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -10,12 +10,13 @@ description: Quick start tutorial for Spark SPARK_VERSION_SHORT This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's interactive shell (in Python or Scala), then show how to write applications in Java, Scala, and Python. -See the [programming guide](programming-guide.html) for a more complete reference. To follow along with this guide, first download a packaged release of Spark from the [Spark website](http://spark.apache.org/downloads.html). Since we won't be using HDFS, you can download a package for any version of Hadoop. +Note that, before Spark 2.0, the main programming interface of Spark was the Resilient Distributed Dataset (RDD). After Spark 2.0, RDDs are replaced by Dataset, which is strongly-typed like an RDD, but with richer optimizations under the hood. The RDD interface is still supported, and you can get a more complete reference at the [RDD programming guide](rdd-programming-guide.html). However, we highly recommend you to switch to use Dataset, which has better performance than RDD. See the [SQL programming guide](sql-programming-guide.html) to get more information about Dataset. + # Interactive Analysis with the Spark Shell ## Basics @@ -29,28 +30,28 @@ or Python. Start it by running the following in the Spark directory: ./bin/spark-shell -Spark's primary abstraction is a distributed collection of items called a Resilient Distributed Dataset (RDD). RDDs can be created from Hadoop InputFormats (such as HDFS files) or by transforming other RDDs. Let's make a new RDD from the text of the README file in the Spark source directory: +Spark's primary abstraction is a distributed collection of items called a Dataset. Datasets can be created from Hadoop InputFormats (such as HDFS files) or by transforming other Datasets. Let's make a new Dataset from the text of the README file in the Spark source directory: {% highlight scala %} -scala> val textFile = sc.textFile("README.md") -textFile: spark.RDD[String] = spark.MappedRDD@2ee9b6e3 +scala> val textFile = spark.read.textFile("README.md") +textFile: org.apache.spark.sql.Dataset[String] = [value: string] {% endhighlight %} -RDDs have _[actions](programming-guide.html#actions)_, which return values, and _[transformations](programming-guide.html#transformations)_, which return pointers to new RDDs. Let's start with a few actions: +You can get values from Dataset directly, by calling some actions, or transform the Dataset to get a new one. For more details, please read the _[API doc](api/scala/index.html#org.apache.spark.sql.Dataset)_. {% highlight scala %} -scala> textFile.count() // Number of items in this RDD -res0: Long = 126 +scala> textFile.count() // Number of items in this Dataset +res0: Long = 126 // May be different from yours as README.md will change over time, similar to other outputs -scala> textFile.first() // First item in this RDD +scala> textFile.first() // First item in this Dataset res1: String = # Apache Spark {% endhighlight %} -Now let's use a transformation. We will use the [`filter`](programming-guide.html#transformations) transformation to return a new RDD with a subset of the items in the file. +Now let's transform this Dataset to a new one. We call `filter` to return a new Dataset with a subset of the items in the file. {% highlight scala %} scala> val linesWithSpark = textFile.filter(line => line.contains("Spark")) -linesWithSpark: spark.RDD[String] = spark.FilteredRDD@7dd4af09 +linesWithSpark: org.apache.spark.sql.Dataset[String] = [value: string] {% endhighlight %} We can chain together transformations and actions: @@ -65,32 +66,37 @@ res3: Long = 15 ./bin/pyspark -Spark's primary abstraction is a distributed collection of items called a Resilient Distributed Dataset (RDD). RDDs can be created from Hadoop InputFormats (such as HDFS files) or by transforming other RDDs. Let's make a new RDD from the text of the README file in the Spark source directory: + +Or if PySpark is installed with pip in your current enviroment: + + pyspark + +Spark's primary abstraction is a distributed collection of items called a Dataset. Datasets can be created from Hadoop InputFormats (such as HDFS files) or by transforming other Datasets. Due to Python's dynamic nature, we don't need the Dataset to be strongly-typed in Python. As a result, all Datasets in Python are Dataset[Row], and we call it `DataFrame` to be consistent with the data frame concept in Pandas and R. Let's make a new DataFrame from the text of the README file in the Spark source directory: {% highlight python %} ->>> textFile = sc.textFile("README.md") +>>> textFile = spark.read.text("README.md") {% endhighlight %} -RDDs have _[actions](programming-guide.html#actions)_, which return values, and _[transformations](programming-guide.html#transformations)_, which return pointers to new RDDs. Let's start with a few actions: +You can get values from DataFrame directly, by calling some actions, or transform the DataFrame to get a new one. For more details, please read the _[API doc](api/python/index.html#pyspark.sql.DataFrame)_. {% highlight python %} ->>> textFile.count() # Number of items in this RDD +>>> textFile.count() # Number of rows in this DataFrame 126 ->>> textFile.first() # First item in this RDD -u'# Apache Spark' +>>> textFile.first() # First row in this DataFrame +Row(value=u'# Apache Spark') {% endhighlight %} -Now let's use a transformation. We will use the [`filter`](programming-guide.html#transformations) transformation to return a new RDD with a subset of the items in the file. +Now let's transform this DataFrame to a new one. We call `filter` to return a new DataFrame with a subset of the lines in the file. {% highlight python %} ->>> linesWithSpark = textFile.filter(lambda line: "Spark" in line) +>>> linesWithSpark = textFile.filter(textFile.value.contains("Spark")) {% endhighlight %} We can chain together transformations and actions: {% highlight python %} ->>> textFile.filter(lambda line: "Spark" in line).count() # How many lines contain "Spark"? +>>> textFile.filter(textFile.value.contains("Spark")).count() # How many lines contain "Spark"? 15 {% endhighlight %} @@ -98,8 +104,8 @@ We can chain together transformations and actions: -## More on RDD Operations -RDD actions and transformations can be used for more complex computations. Let's say we want to find the line with the most words: +## More on Dataset Operations +Dataset actions and transformations can be used for more complex computations. Let's say we want to find the line with the most words:
    @@ -109,7 +115,7 @@ scala> textFile.map(line => line.split(" ").size).reduce((a, b) => if (a > b) a res4: Long = 15 {% endhighlight %} -This first maps a line to an integer value, creating a new RDD. `reduce` is called on that RDD to find the largest line count. The arguments to `map` and `reduce` are Scala function literals (closures), and can use any language feature or Scala/Java library. For example, we can easily call functions declared elsewhere. We'll use `Math.max()` function to make this code easier to understand: +This first maps a line to an integer value, creating a new Dataset. `reduce` is called on that Dataset to find the largest word count. The arguments to `map` and `reduce` are Scala function literals (closures), and can use any language feature or Scala/Java library. For example, we can easily call functions declared elsewhere. We'll use `Math.max()` function to make this code easier to understand: {% highlight scala %} scala> import java.lang.Math @@ -122,11 +128,11 @@ res5: Int = 15 One common data flow pattern is MapReduce, as popularized by Hadoop. Spark can implement MapReduce flows easily: {% highlight scala %} -scala> val wordCounts = textFile.flatMap(line => line.split(" ")).map(word => (word, 1)).reduceByKey((a, b) => a + b) -wordCounts: spark.RDD[(String, Int)] = spark.ShuffledAggregatedRDD@71f027b8 +scala> val wordCounts = textFile.flatMap(line => line.split(" ")).groupByKey(identity).count() +wordCounts: org.apache.spark.sql.Dataset[(String, Long)] = [value: string, count(1): bigint] {% endhighlight %} -Here, we combined the [`flatMap`](programming-guide.html#transformations), [`map`](programming-guide.html#transformations), and [`reduceByKey`](programming-guide.html#transformations) transformations to compute the per-word counts in the file as an RDD of (String, Int) pairs. To collect the word counts in our shell, we can use the [`collect`](programming-guide.html#actions) action: +Here, we call `flatMap` to transform a Dataset of lines to a Dataset of words, and then combine `groupByKey` and `count` to compute the per-word counts in the file as a Dataset of (String, Long) pairs. To collect the word counts in our shell, we can call `collect`: {% highlight scala %} scala> wordCounts.collect() @@ -137,37 +143,24 @@ res6: Array[(String, Int)] = Array((means,1), (under,2), (this,3), (Because,1),
    {% highlight python %} ->>> textFile.map(lambda line: len(line.split())).reduce(lambda a, b: a if (a > b) else b) -15 +>>> from pyspark.sql.functions import * +>>> textFile.select(size(split(textFile.value, "\s+")).name("numWords")).agg(max(col("numWords"))).collect() +[Row(max(numWords)=15)] {% endhighlight %} -This first maps a line to an integer value, creating a new RDD. `reduce` is called on that RDD to find the largest line count. The arguments to `map` and `reduce` are Python [anonymous functions (lambdas)](https://docs.python.org/2/reference/expressions.html#lambda), -but we can also pass any top-level Python function we want. -For example, we'll define a `max` function to make this code easier to understand: - -{% highlight python %} ->>> def max(a, b): -... if a > b: -... return a -... else: -... return b -... - ->>> textFile.map(lambda line: len(line.split())).reduce(max) -15 -{% endhighlight %} +This first maps a line to an integer value and aliases it as "numWords", creating a new DataFrame. `agg` is called on that DataFrame to find the largest word count. The arguments to `select` and `agg` are both _[Column](api/python/index.html#pyspark.sql.Column)_, we can use `df.colName` to get a column from a DataFrame. We can also import pyspark.sql.functions, which provides a lot of convenient functions to build a new Column from an old one. One common data flow pattern is MapReduce, as popularized by Hadoop. Spark can implement MapReduce flows easily: {% highlight python %} ->>> wordCounts = textFile.flatMap(lambda line: line.split()).map(lambda word: (word, 1)).reduceByKey(lambda a, b: a+b) +>>> wordCounts = textFile.select(explode(split(textFile.value, "\s+")).alias("word")).groupBy("word").count() {% endhighlight %} -Here, we combined the [`flatMap`](programming-guide.html#transformations), [`map`](programming-guide.html#transformations), and [`reduceByKey`](programming-guide.html#transformations) transformations to compute the per-word counts in the file as an RDD of (string, int) pairs. To collect the word counts in our shell, we can use the [`collect`](programming-guide.html#actions) action: +Here, we use the `explode` function in `select`, to transfrom a Dataset of lines to a Dataset of words, and then combine `groupBy` and `count` to compute the per-word counts in the file as a DataFrame of 2 columns: "word" and "count". To collect the word counts in our shell, we can call `collect`: {% highlight python %} >>> wordCounts.collect() -[(u'and', 9), (u'A', 1), (u'webpage', 1), (u'README', 1), (u'Note', 1), (u'"local"', 1), (u'variable', 1), ...] +[Row(word=u'online', count=1), Row(word=u'graphs', count=1), ...] {% endhighlight %}
    @@ -181,19 +174,19 @@ Spark also supports pulling data sets into a cluster-wide in-memory cache. This {% highlight scala %} scala> linesWithSpark.cache() -res7: spark.RDD[String] = spark.FilteredRDD@17e51082 +res7: linesWithSpark.type = [value: string] scala> linesWithSpark.count() -res8: Long = 19 +res8: Long = 15 scala> linesWithSpark.count() -res9: Long = 19 +res9: Long = 15 {% endhighlight %} It may seem silly to use Spark to explore and cache a 100-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to -a cluster, as described in the [programming guide](programming-guide.html#initializing-spark). +a cluster, as described in the [RDD programming guide](rdd-programming-guide.html#using-the-shell).
    @@ -202,23 +195,23 @@ a cluster, as described in the [programming guide](programming-guide.html#initia >>> linesWithSpark.cache() >>> linesWithSpark.count() -19 +15 >>> linesWithSpark.count() -19 +15 {% endhighlight %} It may seem silly to use Spark to explore and cache a 100-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `bin/pyspark` to -a cluster, as described in the [programming guide](programming-guide.html#initializing-spark). +a cluster, as described in the [RDD programming guide](rdd-programming-guide.html#using-the-shell).
    # Self-Contained Applications Suppose we wish to write a self-contained application using the Spark API. We will walk through a -simple application in Scala (with sbt), Java (with Maven), and Python. +simple application in Scala (with sbt), Java (with Maven), and Python (pip).
    @@ -228,19 +221,17 @@ named `SimpleApp.scala`: {% highlight scala %} /* SimpleApp.scala */ -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession object SimpleApp { def main(args: Array[String]) { val logFile = "YOUR_SPARK_HOME/README.md" // Should be some file on your system - val conf = new SparkConf().setAppName("Simple Application") - val sc = new SparkContext(conf) - val logData = sc.textFile(logFile, 2).cache() + val spark = SparkSession.builder.appName("Simple Application").getOrCreate() + val logData = spark.read.textFile(logFile).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() - println("Lines with a: %s, Lines with b: %s".format(numAs, numBs)) + println(s"Lines with a: $numAs, Lines with b: $numBs") + spark.stop() } } {% endhighlight %} @@ -250,16 +241,13 @@ Subclasses of `scala.App` may not work correctly. This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is -installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, -we initialize a SparkContext as part of the program. +installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkSession, +we initialize a SparkSession as part of the program. -We pass the SparkContext constructor a -[SparkConf](api/scala/index.html#org.apache.spark.SparkConf) -object which contains information about our -application. +We call `SparkSession.builder` to construct a [[SparkSession]], then set the application name, and finally call `getOrCreate` to get the [[SparkSession]] instance. -Our application depends on the Spark API, so we'll also include an sbt configuration file, -`simple.sbt`, which explains that Spark is a dependency. This file also adds a repository that +Our application depends on the Spark API, so we'll also include an sbt configuration file, +`build.sbt`, which explains that Spark is a dependency. This file also adds a repository that Spark depends on: {% highlight scala %} @@ -269,10 +257,10 @@ version := "1.0" scalaVersion := "{{site.SCALA_VERSION}}" -libraryDependencies += "org.apache.spark" %% "spark-core" % "{{site.SPARK_VERSION}}" +libraryDependencies += "org.apache.spark" %% "spark-sql" % "{{site.SPARK_VERSION}}" {% endhighlight %} -For sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` +For sbt to work correctly, we'll need to layout `SimpleApp.scala` and `build.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the application's code, then use the `spark-submit` script to run our program. @@ -280,7 +268,7 @@ containing the application's code, then use the `spark-submit` script to run our # Your directory layout should look like this $ find . . -./simple.sbt +./build.sbt ./src ./src/main ./src/main/scala @@ -289,13 +277,13 @@ $ find . # Package a jar containing your application $ sbt package ... -[info] Packaging {..}/{..}/target/scala-2.10/simple-project_2.10-1.0.jar +[info] Packaging {..}/{..}/target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar # Use spark-submit to run your application $ YOUR_SPARK_HOME/bin/spark-submit \ --class "SimpleApp" \ --master local[4] \ - target/scala-2.10/simple-project_2.10-1.0.jar + target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar ... Lines with a: 46, Lines with b: 23 {% endhighlight %} @@ -308,37 +296,29 @@ We'll create a very simple Spark application, `SimpleApp.java`: {% highlight java %} /* SimpleApp.java */ -import org.apache.spark.api.java.*; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.function.Function; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.Dataset; public class SimpleApp { public static void main(String[] args) { String logFile = "YOUR_SPARK_HOME/README.md"; // Should be some file on your system - SparkConf conf = new SparkConf().setAppName("Simple Application"); - JavaSparkContext sc = new JavaSparkContext(conf); - JavaRDD logData = sc.textFile(logFile).cache(); - - long numAs = logData.filter(new Function() { - public Boolean call(String s) { return s.contains("a"); } - }).count(); + SparkSession spark = SparkSession.builder().appName("Simple Application").getOrCreate(); + Dataset logData = spark.read().textFile(logFile).cache(); - long numBs = logData.filter(new Function() { - public Boolean call(String s) { return s.contains("b"); } - }).count(); + long numAs = logData.filter(s -> s.contains("a")).count(); + long numBs = logData.filter(s -> s.contains("b")).count(); System.out.println("Lines with a: " + numAs + ", lines with b: " + numBs); + + spark.stop(); } } {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in a text -file. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is installed. -As with the Scala example, we initialize a SparkContext, though we use the special -`JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by -`JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes -that extend `spark.api.java.function.Function`. The -[Spark programming guide](programming-guide.html) describes these differences in more detail. +This program just counts the number of lines containing 'a' and the number containing 'b' in the +Spark README. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is +installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkSession, +we initialize a SparkSession as part of the program. To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version. @@ -354,7 +334,7 @@ Note that Spark artifacts are tagged with a Scala version. org.apache.spark - spark-core_{{site.SCALA_BINARY_VERSION}} + spark-sql_{{site.SCALA_BINARY_VERSION}} {{site.SPARK_VERSION}} @@ -393,29 +373,39 @@ Lines with a: 46, Lines with b: 23 Now we will show how to write an application using the Python API (PySpark). + +If you are building a packaged PySpark application or library you can add it to your setup.py file as: + +{% highlight python %} + install_requires=[ + 'pyspark=={site.SPARK_VERSION}' + ] +{% endhighlight %} + + As an example, we'll create a simple Spark application, `SimpleApp.py`: {% highlight python %} """SimpleApp.py""" -from pyspark import SparkContext +from pyspark.sql import SparkSession logFile = "YOUR_SPARK_HOME/README.md" # Should be some file on your system -sc = SparkContext("local", "Simple App") -logData = sc.textFile(logFile).cache() +spark = SparkSession.builder.appName("SimpleApp").getOrCreate() +logData = spark.read.text(logFile).cache() -numAs = logData.filter(lambda s: 'a' in s).count() -numBs = logData.filter(lambda s: 'b' in s).count() +numAs = logData.filter(logData.value.contains('a')).count() +numBs = logData.filter(logData.value.contains('b')).count() print("Lines with a: %i, lines with b: %i" % (numAs, numBs)) + +spark.stop() {% endhighlight %} This program just counts the number of lines containing 'a' and the number containing 'b' in a text file. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is installed. -As with the Scala and Java examples, we use a SparkContext to create RDDs. -We can pass Python functions to Spark, which are automatically serialized along with any variables -that they reference. +As with the Scala and Java examples, we use a SparkSession to create Datasets. For applications that use custom classes or third-party libraries, we can also add code dependencies to `spark-submit` through its `--py-files` argument by packaging them into a .zip file (see `spark-submit --help` for details). @@ -432,14 +422,22 @@ $ YOUR_SPARK_HOME/bin/spark-submit \ Lines with a: 46, Lines with b: 23 {% endhighlight %} +If you have PySpark pip installed into your enviroment (e.g., `pip install pyspark`), you can run your application with the regular Python interpreter or use the provided 'spark-submit' as you prefer. + +{% highlight bash %} +# Use the Python interpreter to run your application +$ python SimpleApp.py +... +Lines with a: 46, Lines with b: 23 +{% endhighlight %} +
    # Where to Go from Here Congratulations on running your first Spark application! -* For an in-depth overview of the API, start with the [Spark programming guide](programming-guide.html), - or see "Programming Guides" menu for other components. +* For an in-depth overview of the API, start with the [RDD programming guide](rdd-programming-guide.html) and the [SQL programming guide](sql-programming-guide.html), or see "Programming Guides" menu for other components. * For running applications on a cluster, head to the [deployment overview](cluster-overview.html). * Finally, Spark includes several samples in the `examples` directory ([Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples), diff --git a/docs/rdd-programming-guide.md b/docs/rdd-programming-guide.md new file mode 100644 index 000000000000..29af159510e4 --- /dev/null +++ b/docs/rdd-programming-guide.md @@ -0,0 +1,1597 @@ +--- +layout: global +title: RDD Programming Guide +description: Spark SPARK_VERSION_SHORT programming guide in Java, Scala and Python +--- + +* This will become a table of contents (this text will be scraped). +{:toc} + + +# Overview + +At a high level, every Spark application consists of a *driver program* that runs the user's `main` function and executes various *parallel operations* on a cluster. The main abstraction Spark provides is a *resilient distributed dataset* (RDD), which is a collection of elements partitioned across the nodes of the cluster that can be operated on in parallel. RDDs are created by starting with a file in the Hadoop file system (or any other Hadoop-supported file system), or an existing Scala collection in the driver program, and transforming it. Users may also ask Spark to *persist* an RDD in memory, allowing it to be reused efficiently across parallel operations. Finally, RDDs automatically recover from node failures. + +A second abstraction in Spark is *shared variables* that can be used in parallel operations. By default, when Spark runs a function in parallel as a set of tasks on different nodes, it ships a copy of each variable used in the function to each task. Sometimes, a variable needs to be shared across tasks, or between tasks and the driver program. Spark supports two types of shared variables: *broadcast variables*, which can be used to cache a value in memory on all nodes, and *accumulators*, which are variables that are only "added" to, such as counters and sums. + +This guide shows each of these features in each of Spark's supported languages. It is easiest to follow +along with if you launch Spark's interactive shell -- either `bin/spark-shell` for the Scala shell or +`bin/pyspark` for the Python one. + +# Linking with Spark + +
    + +
    + +Spark {{site.SPARK_VERSION}} is built and distributed to work with Scala {{site.SCALA_BINARY_VERSION}} +by default. (Spark can be built to work with other versions of Scala, too.) To write +applications in Scala, you will need to use a compatible Scala version (e.g. {{site.SCALA_BINARY_VERSION}}.X). + +To write a Spark application, you need to add a Maven dependency on Spark. Spark is available through Maven Central at: + + groupId = org.apache.spark + artifactId = spark-core_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION}} + +In addition, if you wish to access an HDFS cluster, you need to add a dependency on +`hadoop-client` for your version of HDFS. + + groupId = org.apache.hadoop + artifactId = hadoop-client + version = + +Finally, you need to import some Spark classes into your program. Add the following lines: + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.SparkConf +{% endhighlight %} + +(Before Spark 1.3.0, you need to explicitly `import org.apache.spark.SparkContext._` to enable essential implicit conversions.) + +
    + +
    + +Spark {{site.SPARK_VERSION}} supports +[lambda expressions](http://docs.oracle.com/javase/tutorial/java/javaOO/lambdaexpressions.html) +for concisely writing functions, otherwise you can use the classes in the +[org.apache.spark.api.java.function](api/java/index.html?org/apache/spark/api/java/function/package-summary.html) package. + +Note that support for Java 7 was removed in Spark 2.2.0. + +To write a Spark application in Java, you need to add a dependency on Spark. Spark is available through Maven Central at: + + groupId = org.apache.spark + artifactId = spark-core_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION}} + +In addition, if you wish to access an HDFS cluster, you need to add a dependency on +`hadoop-client` for your version of HDFS. + + groupId = org.apache.hadoop + artifactId = hadoop-client + version = + +Finally, you need to import some Spark classes into your program. Add the following lines: + +{% highlight java %} +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.SparkConf; +{% endhighlight %} + +
    + +
    + +Spark {{site.SPARK_VERSION}} works with Python 2.7+ or Python 3.4+. It can use the standard CPython interpreter, +so C libraries like NumPy can be used. It also works with PyPy 2.3+. + +Python 2.6 support was removed in Spark 2.2.0. + +Spark applications in Python can either be run with the `bin/spark-submit` script which includes Spark at runtime, or by including including it in your setup.py as: + +{% highlight python %} + install_requires=[ + 'pyspark=={site.SPARK_VERSION}' + ] +{% endhighlight %} + + +To run Spark applications in Python without pip installing PySpark, use the `bin/spark-submit` script located in the Spark directory. +This script will load Spark's Java/Scala libraries and allow you to submit applications to a cluster. +You can also use `bin/pyspark` to launch an interactive Python shell. + +If you wish to access HDFS data, you need to use a build of PySpark linking +to your version of HDFS. +[Prebuilt packages](http://spark.apache.org/downloads.html) are also available on the Spark homepage +for common HDFS versions. + +Finally, you need to import some Spark classes into your program. Add the following line: + +{% highlight python %} +from pyspark import SparkContext, SparkConf +{% endhighlight %} + +PySpark requires the same minor version of Python in both driver and workers. It uses the default python version in PATH, +you can specify which version of Python you want to use by `PYSPARK_PYTHON`, for example: + +{% highlight bash %} +$ PYSPARK_PYTHON=python3.4 bin/pyspark +$ PYSPARK_PYTHON=/opt/pypy-2.5/bin/pypy bin/spark-submit examples/src/main/python/pi.py +{% endhighlight %} + +
    + +
    + + +# Initializing Spark + +
    + +
    + +The first thing a Spark program must do is to create a [SparkContext](api/scala/index.html#org.apache.spark.SparkContext) object, which tells Spark +how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object +that contains information about your application. + +Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before creating a new one. + +{% highlight scala %} +val conf = new SparkConf().setAppName(appName).setMaster(master) +new SparkContext(conf) +{% endhighlight %} + +
    + +
    + +The first thing a Spark program must do is to create a [JavaSparkContext](api/java/index.html?org/apache/spark/api/java/JavaSparkContext.html) object, which tells Spark +how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/java/index.html?org/apache/spark/SparkConf.html) object +that contains information about your application. + +{% highlight java %} +SparkConf conf = new SparkConf().setAppName(appName).setMaster(master); +JavaSparkContext sc = new JavaSparkContext(conf); +{% endhighlight %} + +
    + +
    + +The first thing a Spark program must do is to create a [SparkContext](api/python/pyspark.html#pyspark.SparkContext) object, which tells Spark +how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/python/pyspark.html#pyspark.SparkConf) object +that contains information about your application. + +{% highlight python %} +conf = SparkConf().setAppName(appName).setMaster(master) +sc = SparkContext(conf=conf) +{% endhighlight %} + +
    + +
    + +The `appName` parameter is a name for your application to show on the cluster UI. +`master` is a [Spark, Mesos or YARN cluster URL](submitting-applications.html#master-urls), +or a special "local" string to run in local mode. +In practice, when running on a cluster, you will not want to hardcode `master` in the program, +but rather [launch the application with `spark-submit`](submitting-applications.html) and +receive it there. However, for local testing and unit tests, you can pass "local" to run Spark +in-process. + + +## Using the Shell + +
    + +
    + +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. 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] +{% endhighlight %} + +Or, to also add `code.jar` to its classpath, use: + +{% highlight bash %} +$ ./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). + +
    + +
    + +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`. 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 %} +$ ./bin/pyspark --master local[4] +{% endhighlight %} + +Or, to also add `code.py` to the search path (in order to later be able to `import code`), use: + +{% highlight bash %} +$ ./bin/pyspark --master local[4] --py-files code.py +{% endhighlight %} + +For a complete list of options, run `pyspark --help`. Behind the scenes, +`pyspark` invokes the more general [`spark-submit` script](submitting-applications.html). + +It is also possible to launch the PySpark shell in [IPython](http://ipython.org), the +enhanced Python interpreter. PySpark works with IPython 1.0.0 and later. To +use IPython, set the `PYSPARK_DRIVER_PYTHON` variable to `ipython` when running `bin/pyspark`: + +{% highlight bash %} +$ PYSPARK_DRIVER_PYTHON=ipython ./bin/pyspark +{% endhighlight %} + +To use the Jupyter notebook (previously known as the IPython notebook), + +{% highlight bash %} +$ PYSPARK_DRIVER_PYTHON=jupyter PYSPARK_DRIVER_PYTHON_OPTS=notebook ./bin/pyspark +{% endhighlight %} + +You can customize the `ipython` or `jupyter` commands by setting `PYSPARK_DRIVER_PYTHON_OPTS`. + +After the Jupyter Notebook server is launched, you can create a new "Python 2" notebook from +the "Files" tab. Inside the notebook, you can input the command `%pylab inline` as part of +your notebook before you start to try Spark from the Jupyter notebook. + +
    + +
    + +# Resilient Distributed Datasets (RDDs) + +Spark revolves around the concept of a _resilient distributed dataset_ (RDD), which is a fault-tolerant collection of elements that can be operated on in parallel. There are two ways to create RDDs: *parallelizing* +an existing collection in your driver program, or referencing a dataset in an external storage system, such as a +shared filesystem, HDFS, HBase, or any data source offering a Hadoop InputFormat. + +## Parallelized Collections + +
    + +
    + +Parallelized collections are created by calling `SparkContext`'s `parallelize` method on an existing collection in your driver program (a Scala `Seq`). The elements of the collection are copied to form a distributed dataset that can be operated on in parallel. For example, here is how to create a parallelized collection holding the numbers 1 to 5: + +{% highlight scala %} +val data = Array(1, 2, 3, 4, 5) +val distData = sc.parallelize(data) +{% endhighlight %} + +Once created, the distributed dataset (`distData`) can be operated on in parallel. For example, we might call `distData.reduce((a, b) => a + b)` to add up the elements of the array. We describe operations on distributed datasets later on. + +
    + +
    + +Parallelized collections are created by calling `JavaSparkContext`'s `parallelize` method on an existing `Collection` in your driver program. The elements of the collection are copied to form a distributed dataset that can be operated on in parallel. For example, here is how to create a parallelized collection holding the numbers 1 to 5: + +{% highlight java %} +List data = Arrays.asList(1, 2, 3, 4, 5); +JavaRDD distData = sc.parallelize(data); +{% endhighlight %} + +Once created, the distributed dataset (`distData`) can be operated on in parallel. For example, we might call `distData.reduce((a, b) -> a + b)` to add up the elements of the list. +We describe operations on distributed datasets later on. + +
    + +
    + +Parallelized collections are created by calling `SparkContext`'s `parallelize` method on an existing iterable or collection in your driver program. The elements of the collection are copied to form a distributed dataset that can be operated on in parallel. For example, here is how to create a parallelized collection holding the numbers 1 to 5: + +{% highlight python %} +data = [1, 2, 3, 4, 5] +distData = sc.parallelize(data) +{% endhighlight %} + +Once created, the distributed dataset (`distData`) can be operated on in parallel. For example, we can call `distData.reduce(lambda a, b: a + b)` to add up the elements of the list. +We describe operations on distributed datasets later on. + +
    + +
    + +One important parameter for parallel collections is the number of *partitions* to cut the dataset into. Spark will run one task for each partition of the cluster. Typically you want 2-4 partitions for each CPU in your cluster. Normally, Spark tries to set the number of partitions automatically based on your cluster. However, you can also set it manually by passing it as a second parameter to `parallelize` (e.g. `sc.parallelize(data, 10)`). Note: some places in the code use the term slices (a synonym for partitions) to maintain backward compatibility. + +## External Datasets + +
    + +
    + +Spark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). + +Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: + +{% highlight scala %} +scala> val distFile = sc.textFile("data.txt") +distFile: org.apache.spark.rdd.RDD[String] = data.txt MapPartitionsRDD[10] at textFile at :26 +{% endhighlight %} + +Once created, `distFile` can be acted on by dataset operations. For example, we can add up the sizes of all the lines using the `map` and `reduce` operations as follows: `distFile.map(s => s.length).reduce((a, b) => a + b)`. + +Some notes on reading files with Spark: + +* If using a path on the local filesystem, the file must also be accessible at the same path on worker nodes. Either copy the file to all workers or use a network-mounted shared file system. + +* All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. + +* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 128MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. + +Apart from text files, Spark's Scala API also supports several other data formats: + +* `SparkContext.wholeTextFiles` lets you read a directory containing multiple small text files, and returns each of them as (filename, content) pairs. This is in contrast with `textFile`, which would return one record per line in each file. Partitioning is determined by data locality which, in some cases, may result in too few partitions. For those cases, `wholeTextFiles` provides an optional second argument for controlling the minimal number of partitions. + +* For [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), use SparkContext's `sequenceFile[K, V]` method where `K` and `V` are the types of key and values in the file. These should be subclasses of Hadoop's [Writable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Writable.html) interface, like [IntWritable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/IntWritable.html) and [Text](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Text.html). In addition, Spark allows you to specify native types for a few common Writables; for example, `sequenceFile[Int, String]` will automatically read IntWritables and Texts. + +* For other Hadoop InputFormats, you can use the `SparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source. You can also use `SparkContext.newAPIHadoopRDD` for InputFormats based on the "new" MapReduce API (`org.apache.hadoop.mapreduce`). + +* `RDD.saveAsObjectFile` and `SparkContext.objectFile` support saving an RDD in a simple format consisting of serialized Java objects. While this is not as efficient as specialized formats like Avro, it offers an easy way to save any RDD. + +
    + +
    + +Spark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). + +Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: + +{% highlight java %} +JavaRDD distFile = sc.textFile("data.txt"); +{% endhighlight %} + +Once created, `distFile` can be acted on by dataset operations. For example, we can add up the sizes of all the lines using the `map` and `reduce` operations as follows: `distFile.map(s -> s.length()).reduce((a, b) -> a + b)`. + +Some notes on reading files with Spark: + +* If using a path on the local filesystem, the file must also be accessible at the same path on worker nodes. Either copy the file to all workers or use a network-mounted shared file system. + +* All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. + +* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 128MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. + +Apart from text files, Spark's Java API also supports several other data formats: + +* `JavaSparkContext.wholeTextFiles` lets you read a directory containing multiple small text files, and returns each of them as (filename, content) pairs. This is in contrast with `textFile`, which would return one record per line in each file. + +* For [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), use SparkContext's `sequenceFile[K, V]` method where `K` and `V` are the types of key and values in the file. These should be subclasses of Hadoop's [Writable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Writable.html) interface, like [IntWritable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/IntWritable.html) and [Text](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Text.html). + +* For other Hadoop InputFormats, you can use the `JavaSparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source. You can also use `JavaSparkContext.newAPIHadoopRDD` for InputFormats based on the "new" MapReduce API (`org.apache.hadoop.mapreduce`). + +* `JavaRDD.saveAsObjectFile` and `JavaSparkContext.objectFile` support saving an RDD in a simple format consisting of serialized Java objects. While this is not as efficient as specialized formats like Avro, it offers an easy way to save any RDD. + +
    + +
    + +PySpark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). + +Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: + +{% highlight python %} +>>> distFile = sc.textFile("data.txt") +{% endhighlight %} + +Once created, `distFile` can be acted on by dataset operations. For example, we can add up the sizes of all the lines using the `map` and `reduce` operations as follows: `distFile.map(lambda s: len(s)).reduce(lambda a, b: a + b)`. + +Some notes on reading files with Spark: + +* If using a path on the local filesystem, the file must also be accessible at the same path on worker nodes. Either copy the file to all workers or use a network-mounted shared file system. + +* All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. + +* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 128MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. + +Apart from text files, Spark's Python API also supports several other data formats: + +* `SparkContext.wholeTextFiles` lets you read a directory containing multiple small text files, and returns each of them as (filename, content) pairs. This is in contrast with `textFile`, which would return one record per line in each file. + +* `RDD.saveAsPickleFile` and `SparkContext.pickleFile` support saving an RDD in a simple format consisting of pickled Python objects. Batching is used on pickle serialization, with default batch size 10. + +* SequenceFile and Hadoop Input/Output Formats + +**Note** this feature is currently marked ```Experimental``` and is intended for advanced users. It may be replaced in future with read/write support based on Spark SQL, in which case Spark SQL is the preferred approach. + +**Writable Support** + +PySpark SequenceFile support loads an RDD of key-value pairs within Java, converts Writables to base Java types, and pickles the +resulting Java objects using [Pyrolite](https://github.com/irmen/Pyrolite/). When saving an RDD of key-value pairs to SequenceFile, +PySpark does the reverse. It unpickles Python objects into Java objects and then converts them to Writables. The following +Writables are automatically converted: + + + + + + + + + + + +
    Writable TypePython Type
    Textunicode str
    IntWritableint
    FloatWritablefloat
    DoubleWritablefloat
    BooleanWritablebool
    BytesWritablebytearray
    NullWritableNone
    MapWritabledict
    + +Arrays are not handled out-of-the-box. Users need to specify custom `ArrayWritable` subtypes when reading or writing. When writing, +users also need to specify custom converters that convert arrays to custom `ArrayWritable` subtypes. When reading, the default +converter will convert custom `ArrayWritable` subtypes to Java `Object[]`, which then get pickled to Python tuples. To get +Python `array.array` for arrays of primitive types, users need to specify custom converters. + +**Saving and Loading SequenceFiles** + +Similarly to text files, SequenceFiles can be saved and loaded by specifying the path. The key and value +classes can be specified, but for standard Writables this is not required. + +{% highlight python %} +>>> rdd = sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x)) +>>> rdd.saveAsSequenceFile("path/to/file") +>>> sorted(sc.sequenceFile("path/to/file").collect()) +[(1, u'a'), (2, u'aa'), (3, u'aaa')] +{% endhighlight %} + +**Saving and Loading Other Hadoop Input/Output Formats** + +PySpark can also read any Hadoop InputFormat or write any Hadoop OutputFormat, for both 'new' and 'old' Hadoop MapReduce APIs. +If required, a Hadoop configuration can be passed in as a Python dict. Here is an example using the +Elasticsearch ESInputFormat: + +{% highlight python %} +$ ./bin/pyspark --jars /path/to/elasticsearch-hadoop.jar +>>> conf = {"es.resource" : "index/type"} # assume Elasticsearch is running on localhost defaults +>>> rdd = sc.newAPIHadoopRDD("org.elasticsearch.hadoop.mr.EsInputFormat", + "org.apache.hadoop.io.NullWritable", + "org.elasticsearch.hadoop.mr.LinkedMapWritable", + conf=conf) +>>> rdd.first() # the result is a MapWritable that is converted to a Python dict +(u'Elasticsearch ID', + {u'field1': True, + u'field2': u'Some Text', + u'field3': 12345}) +{% endhighlight %} + +Note that, if the InputFormat simply depends on a Hadoop configuration and/or input path, and +the key and value classes can easily be converted according to the above table, +then this approach should work well for such cases. + +If you have custom serialized binary data (such as loading data from Cassandra / HBase), then you will first need to +transform that data on the Scala/Java side to something which can be handled by Pyrolite's pickler. +A [Converter](api/scala/index.html#org.apache.spark.api.python.Converter) trait is provided +for this. Simply extend this trait and implement your transformation code in the ```convert``` +method. Remember to ensure that this class, along with any dependencies required to access your ```InputFormat```, are packaged into your Spark job jar and included on the PySpark +classpath. + +See the [Python examples]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python) and +the [Converter examples]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/pythonconverters) +for examples of using Cassandra / HBase ```InputFormat``` and ```OutputFormat``` with custom converters. + +
    +
    + +## RDD Operations + +RDDs support two types of operations: *transformations*, which create a new dataset from an existing one, and *actions*, which return a value to the driver program after running a computation on the dataset. For example, `map` is a transformation that passes each dataset element through a function and returns a new RDD representing the results. On the other hand, `reduce` is an action that aggregates all the elements of the RDD using some function and returns the final result to the driver program (although there is also a parallel `reduceByKey` that returns a distributed dataset). + +All transformations in Spark are lazy, in that they do not compute their results right away. Instead, they just remember the transformations applied to some base dataset (e.g. a file). The transformations are only computed when an action requires a result to be returned to the driver program. This design enables Spark to run more efficiently. For example, we can realize that a dataset created through `map` will be used in a `reduce` and return only the result of the `reduce` to the driver, rather than the larger mapped dataset. + +By default, each transformed RDD may be recomputed each time you run an action on it. However, you may also *persist* an RDD in memory using the `persist` (or `cache`) method, in which case Spark will keep the elements around on the cluster for much faster access the next time you query it. There is also support for persisting RDDs on disk, or replicated across multiple nodes. + +### Basics + +
    + +
    + +To illustrate RDD basics, consider the simple program below: + +{% highlight scala %} +val lines = sc.textFile("data.txt") +val lineLengths = lines.map(s => s.length) +val totalLength = lineLengths.reduce((a, b) => a + b) +{% endhighlight %} + +The first line defines a base RDD from an external file. This dataset is not loaded in memory or +otherwise acted on: `lines` is merely a pointer to the file. +The second line defines `lineLengths` as the result of a `map` transformation. Again, `lineLengths` +is *not* immediately computed, due to laziness. +Finally, we run `reduce`, which is an action. At this point Spark breaks the computation into tasks +to run on separate machines, and each machine runs both its part of the map and a local reduction, +returning only its answer to the driver program. + +If we also wanted to use `lineLengths` again later, we could add: + +{% highlight scala %} +lineLengths.persist() +{% endhighlight %} + +before the `reduce`, which would cause `lineLengths` to be saved in memory after the first time it is computed. + +
    + +
    + +To illustrate RDD basics, consider the simple program below: + +{% highlight java %} +JavaRDD lines = sc.textFile("data.txt"); +JavaRDD lineLengths = lines.map(s -> s.length()); +int totalLength = lineLengths.reduce((a, b) -> a + b); +{% endhighlight %} + +The first line defines a base RDD from an external file. This dataset is not loaded in memory or +otherwise acted on: `lines` is merely a pointer to the file. +The second line defines `lineLengths` as the result of a `map` transformation. Again, `lineLengths` +is *not* immediately computed, due to laziness. +Finally, we run `reduce`, which is an action. At this point Spark breaks the computation into tasks +to run on separate machines, and each machine runs both its part of the map and a local reduction, +returning only its answer to the driver program. + +If we also wanted to use `lineLengths` again later, we could add: + +{% highlight java %} +lineLengths.persist(StorageLevel.MEMORY_ONLY()); +{% endhighlight %} + +before the `reduce`, which would cause `lineLengths` to be saved in memory after the first time it is computed. + +
    + +
    + +To illustrate RDD basics, consider the simple program below: + +{% highlight python %} +lines = sc.textFile("data.txt") +lineLengths = lines.map(lambda s: len(s)) +totalLength = lineLengths.reduce(lambda a, b: a + b) +{% endhighlight %} + +The first line defines a base RDD from an external file. This dataset is not loaded in memory or +otherwise acted on: `lines` is merely a pointer to the file. +The second line defines `lineLengths` as the result of a `map` transformation. Again, `lineLengths` +is *not* immediately computed, due to laziness. +Finally, we run `reduce`, which is an action. At this point Spark breaks the computation into tasks +to run on separate machines, and each machine runs both its part of the map and a local reduction, +returning only its answer to the driver program. + +If we also wanted to use `lineLengths` again later, we could add: + +{% highlight python %} +lineLengths.persist() +{% endhighlight %} + +before the `reduce`, which would cause `lineLengths` to be saved in memory after the first time it is computed. + +
    + +
    + +### Passing Functions to Spark + +
    + +
    + +Spark's API relies heavily on passing functions in the driver program to run on the cluster. +There are two recommended ways to do this: + +* [Anonymous function syntax](http://docs.scala-lang.org/tour/basics.html#functions), + which can be used for short pieces of code. +* Static methods in a global singleton object. For example, you can define `object MyFunctions` and then + pass `MyFunctions.func1`, as follows: + +{% highlight scala %} +object MyFunctions { + def func1(s: String): String = { ... } +} + +myRdd.map(MyFunctions.func1) +{% endhighlight %} + +Note that while it is also possible to pass a reference to a method in a class instance (as opposed to +a singleton object), this requires sending the object that contains that class along with the method. +For example, consider: + +{% highlight scala %} +class MyClass { + def func1(s: String): String = { ... } + def doStuff(rdd: RDD[String]): RDD[String] = { rdd.map(func1) } +} +{% endhighlight %} + +Here, if we create a new `MyClass` instance and call `doStuff` on it, the `map` inside there references the +`func1` method *of that `MyClass` instance*, so the whole object needs to be sent to the cluster. It is +similar to writing `rdd.map(x => this.func1(x))`. + +In a similar way, accessing fields of the outer object will reference the whole object: + +{% highlight scala %} +class MyClass { + val field = "Hello" + def doStuff(rdd: RDD[String]): RDD[String] = { rdd.map(x => field + x) } +} +{% endhighlight %} + +is equivalent to writing `rdd.map(x => this.field + x)`, which references all of `this`. To avoid this +issue, the simplest way is to copy `field` into a local variable instead of accessing it externally: + +{% highlight scala %} +def doStuff(rdd: RDD[String]): RDD[String] = { + val field_ = this.field + rdd.map(x => field_ + x) +} +{% endhighlight %} + +
    + +
    + +Spark's API relies heavily on passing functions in the driver program to run on the cluster. +In Java, functions are represented by classes implementing the interfaces in the +[org.apache.spark.api.java.function](api/java/index.html?org/apache/spark/api/java/function/package-summary.html) package. +There are two ways to create such functions: + +* Implement the Function interfaces in your own class, either as an anonymous inner class or a named one, + and pass an instance of it to Spark. +* Use [lambda expressions](http://docs.oracle.com/javase/tutorial/java/javaOO/lambdaexpressions.html) + to concisely define an implementation. + +While much of this guide uses lambda syntax for conciseness, it is easy to use all the same APIs +in long-form. For example, we could have written our code above as follows: + +{% highlight java %} +JavaRDD lines = sc.textFile("data.txt"); +JavaRDD lineLengths = lines.map(new Function() { + public Integer call(String s) { return s.length(); } +}); +int totalLength = lineLengths.reduce(new Function2() { + public Integer call(Integer a, Integer b) { return a + b; } +}); +{% endhighlight %} + +Or, if writing the functions inline is unwieldy: + +{% highlight java %} +class GetLength implements Function { + public Integer call(String s) { return s.length(); } +} +class Sum implements Function2 { + public Integer call(Integer a, Integer b) { return a + b; } +} + +JavaRDD lines = sc.textFile("data.txt"); +JavaRDD lineLengths = lines.map(new GetLength()); +int totalLength = lineLengths.reduce(new Sum()); +{% endhighlight %} + +Note that anonymous inner classes in Java can also access variables in the enclosing scope as long +as they are marked `final`. Spark will ship copies of these variables to each worker node as it does +for other languages. + +
    + +
    + +Spark's API relies heavily on passing functions in the driver program to run on the cluster. +There are three recommended ways to do this: + +* [Lambda expressions](https://docs.python.org/2/tutorial/controlflow.html#lambda-expressions), + for simple functions that can be written as an expression. (Lambdas do not support multi-statement + functions or statements that do not return a value.) +* Local `def`s inside the function calling into Spark, for longer code. +* Top-level functions in a module. + +For example, to pass a longer function than can be supported using a `lambda`, consider +the code below: + +{% highlight python %} +"""MyScript.py""" +if __name__ == "__main__": + def myFunc(s): + words = s.split(" ") + return len(words) + + sc = SparkContext(...) + sc.textFile("file.txt").map(myFunc) +{% endhighlight %} + +Note that while it is also possible to pass a reference to a method in a class instance (as opposed to +a singleton object), this requires sending the object that contains that class along with the method. +For example, consider: + +{% highlight python %} +class MyClass(object): + def func(self, s): + return s + def doStuff(self, rdd): + return rdd.map(self.func) +{% endhighlight %} + +Here, if we create a `new MyClass` and call `doStuff` on it, the `map` inside there references the +`func` method *of that `MyClass` instance*, so the whole object needs to be sent to the cluster. + +In a similar way, accessing fields of the outer object will reference the whole object: + +{% highlight python %} +class MyClass(object): + def __init__(self): + self.field = "Hello" + def doStuff(self, rdd): + return rdd.map(lambda s: self.field + s) +{% endhighlight %} + +To avoid this issue, the simplest way is to copy `field` into a local variable instead +of accessing it externally: + +{% highlight python %} +def doStuff(self, rdd): + field = self.field + return rdd.map(lambda s: field + s) +{% endhighlight %} + +
    + +
    + +### Understanding closures +One of the harder things about Spark is understanding the scope and life cycle of variables and methods when executing code across a cluster. RDD operations that modify variables outside of their scope can be a frequent source of confusion. In the example below we'll look at code that uses `foreach()` to increment a counter, but similar issues can occur for other operations as well. + +#### Example + +Consider the naive RDD element sum below, which may behave differently depending on whether execution is happening within the same JVM. A common example of this is when running Spark in `local` mode (`--master = local[n]`) versus deploying a Spark application to a cluster (e.g. via spark-submit to YARN): + +
    + +
    +{% highlight scala %} +var counter = 0 +var rdd = sc.parallelize(data) + +// Wrong: Don't do this!! +rdd.foreach(x => counter += x) + +println("Counter value: " + counter) +{% endhighlight %} +
    + +
    +{% highlight java %} +int counter = 0; +JavaRDD rdd = sc.parallelize(data); + +// Wrong: Don't do this!! +rdd.foreach(x -> counter += x); + +println("Counter value: " + counter); +{% endhighlight %} +
    + +
    +{% highlight python %} +counter = 0 +rdd = sc.parallelize(data) + +# Wrong: Don't do this!! +def increment_counter(x): + global counter + counter += x +rdd.foreach(increment_counter) + +print("Counter value: ", counter) +{% endhighlight %} +
    + +
    + +#### Local vs. cluster modes + +The behavior of the above code is undefined, and may not work as intended. To execute jobs, Spark breaks up the processing of RDD operations into tasks, each of which is executed by an executor. Prior to execution, Spark computes the task's **closure**. The closure is those variables and methods which must be visible for the executor to perform its computations on the RDD (in this case `foreach()`). This closure is serialized and sent to each executor. + +The variables within the closure sent to each executor are now copies and thus, when **counter** is referenced within the `foreach` function, it's no longer the **counter** on the driver node. There is still a **counter** in the memory of the driver node but this is no longer visible to the executors! The executors only see the copy from the serialized closure. Thus, the final value of **counter** will still be zero since all operations on **counter** were referencing the value within the serialized closure. + +In local mode, in some circumstances the `foreach` function will actually execute within the same JVM as the driver and will reference the same original **counter**, and may actually update it. + +To ensure well-defined behavior in these sorts of scenarios one should use an [`Accumulator`](#accumulators). Accumulators in Spark are used specifically to provide a mechanism for safely updating a variable when execution is split up across worker nodes in a cluster. The Accumulators section of this guide discusses these in more detail. + +In general, closures - constructs like loops or locally defined methods, should not be used to mutate some global state. Spark does not define or guarantee the behavior of mutations to objects referenced from outside of closures. Some code that does this may work in local mode, but that's just by accident and such code will not behave as expected in distributed mode. Use an Accumulator instead if some global aggregation is needed. + +#### Printing elements of an RDD +Another common idiom is attempting to print out the elements of an RDD using `rdd.foreach(println)` or `rdd.map(println)`. On a single machine, this will generate the expected output and print all the RDD's elements. However, in `cluster` mode, the output to `stdout` being called by the executors is now writing to the executor's `stdout` instead, not the one on the driver, so `stdout` on the driver won't show these! To print all elements on the driver, one can use the `collect()` method to first bring the RDD to the driver node thus: `rdd.collect().foreach(println)`. This can cause the driver to run out of memory, though, because `collect()` fetches the entire RDD to a single machine; if you only need to print a few elements of the RDD, a safer approach is to use the `take()`: `rdd.take(100).foreach(println)`. + +### Working with Key-Value Pairs + +
    + +
    + +While most Spark operations work on RDDs containing any type of objects, a few special operations are +only available on RDDs of key-value pairs. +The most common ones are distributed "shuffle" operations, such as grouping or aggregating the elements +by a key. + +In Scala, these operations are automatically available on RDDs containing +[Tuple2](http://www.scala-lang.org/api/{{site.SCALA_VERSION}}/index.html#scala.Tuple2) objects +(the built-in tuples in the language, created by simply writing `(a, b)`). The key-value pair operations are available in the +[PairRDDFunctions](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions) class, +which automatically wraps around an RDD of tuples. + +For example, the following code uses the `reduceByKey` operation on key-value pairs to count how +many times each line of text occurs in a file: + +{% highlight scala %} +val lines = sc.textFile("data.txt") +val pairs = lines.map(s => (s, 1)) +val counts = pairs.reduceByKey((a, b) => a + b) +{% endhighlight %} + +We could also use `counts.sortByKey()`, for example, to sort the pairs alphabetically, and finally +`counts.collect()` to bring them back to the driver program as an array of objects. + +**Note:** when using custom objects as the key in key-value pair operations, you must be sure that a +custom `equals()` method is accompanied with a matching `hashCode()` method. For full details, see +the contract outlined in the [Object.hashCode() +documentation](http://docs.oracle.com/javase/7/docs/api/java/lang/Object.html#hashCode()). + +
    + +
    + +While most Spark operations work on RDDs containing any type of objects, a few special operations are +only available on RDDs of key-value pairs. +The most common ones are distributed "shuffle" operations, such as grouping or aggregating the elements +by a key. + +In Java, key-value pairs are represented using the +[scala.Tuple2](http://www.scala-lang.org/api/{{site.SCALA_VERSION}}/index.html#scala.Tuple2) class +from the Scala standard library. You can simply call `new Tuple2(a, b)` to create a tuple, and access +its fields later with `tuple._1()` and `tuple._2()`. + +RDDs of key-value pairs are represented by the +[JavaPairRDD](api/java/index.html?org/apache/spark/api/java/JavaPairRDD.html) class. You can construct +JavaPairRDDs from JavaRDDs using special versions of the `map` operations, like +`mapToPair` and `flatMapToPair`. The JavaPairRDD will have both standard RDD functions and special +key-value ones. + +For example, the following code uses the `reduceByKey` operation on key-value pairs to count how +many times each line of text occurs in a file: + +{% highlight scala %} +JavaRDD lines = sc.textFile("data.txt"); +JavaPairRDD pairs = lines.mapToPair(s -> new Tuple2(s, 1)); +JavaPairRDD counts = pairs.reduceByKey((a, b) -> a + b); +{% endhighlight %} + +We could also use `counts.sortByKey()`, for example, to sort the pairs alphabetically, and finally +`counts.collect()` to bring them back to the driver program as an array of objects. + +**Note:** when using custom objects as the key in key-value pair operations, you must be sure that a +custom `equals()` method is accompanied with a matching `hashCode()` method. For full details, see +the contract outlined in the [Object.hashCode() +documentation](http://docs.oracle.com/javase/7/docs/api/java/lang/Object.html#hashCode()). + +
    + +
    + +While most Spark operations work on RDDs containing any type of objects, a few special operations are +only available on RDDs of key-value pairs. +The most common ones are distributed "shuffle" operations, such as grouping or aggregating the elements +by a key. + +In Python, these operations work on RDDs containing built-in Python tuples such as `(1, 2)`. +Simply create such tuples and then call your desired operation. + +For example, the following code uses the `reduceByKey` operation on key-value pairs to count how +many times each line of text occurs in a file: + +{% highlight python %} +lines = sc.textFile("data.txt") +pairs = lines.map(lambda s: (s, 1)) +counts = pairs.reduceByKey(lambda a, b: a + b) +{% endhighlight %} + +We could also use `counts.sortByKey()`, for example, to sort the pairs alphabetically, and finally +`counts.collect()` to bring them back to the driver program as a list of objects. + +
    + +
    + + +### Transformations + +The following table lists some of the common transformations supported by Spark. Refer to the +RDD API doc +([Scala](api/scala/index.html#org.apache.spark.rdd.RDD), + [Java](api/java/index.html?org/apache/spark/api/java/JavaRDD.html), + [Python](api/python/pyspark.html#pyspark.RDD), + [R](api/R/index.html)) +and pair RDD functions doc +([Scala](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions), + [Java](api/java/index.html?org/apache/spark/api/java/JavaPairRDD.html)) +for details. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    TransformationMeaning
    map(func) Return a new distributed dataset formed by passing each element of the source through a function func.
    filter(func) Return a new dataset formed by selecting those elements of the source on which func returns true.
    flatMap(func) Similar to map, but each input item can be mapped to 0 or more output items (so func should return a Seq rather than a single item).
    mapPartitions(func) Similar to map, but runs separately on each partition (block) of the RDD, so func must be of type + Iterator<T> => Iterator<U> when running on an RDD of type T.
    mapPartitionsWithIndex(func) Similar to mapPartitions, but also provides func with an integer value representing the index of + the partition, so func must be of type (Int, Iterator<T>) => Iterator<U> when running on an RDD of type T. +
    sample(withReplacement, fraction, seed) Sample a fraction fraction of the data, with or without replacement, using a given random number generator seed.
    union(otherDataset) Return a new dataset that contains the union of the elements in the source dataset and the argument.
    intersection(otherDataset) Return a new RDD that contains the intersection of elements in the source dataset and the argument.
    distinct([numPartitions])) Return a new dataset that contains the distinct elements of the source dataset.
    groupByKey([numPartitions]) When called on a dataset of (K, V) pairs, returns a dataset of (K, Iterable<V>) pairs.
    + Note: If you are grouping in order to perform an aggregation (such as a sum or + average) over each key, using reduceByKey or aggregateByKey will yield much better + performance. +
    + Note: By default, the level of parallelism in the output depends on the number of partitions of the parent RDD. + You can pass an optional numPartitions argument to set a different number of tasks. +
    reduceByKey(func, [numPartitions]) When called on a dataset of (K, V) pairs, returns a dataset of (K, V) pairs where the values for each key are aggregated using the given reduce function func, which must be of type (V,V) => V. Like in groupByKey, the number of reduce tasks is configurable through an optional second argument.
    aggregateByKey(zeroValue)(seqOp, combOp, [numPartitions]) When called on a dataset of (K, V) pairs, returns a dataset of (K, U) pairs where the values for each key are aggregated using the given combine functions and a neutral "zero" value. Allows an aggregated value type that is different than the input value type, while avoiding unnecessary allocations. Like in groupByKey, the number of reduce tasks is configurable through an optional second argument.
    sortByKey([ascending], [numPartitions]) When called on a dataset of (K, V) pairs where K implements Ordered, returns a dataset of (K, V) pairs sorted by keys in ascending or descending order, as specified in the boolean ascending argument.
    join(otherDataset, [numPartitions]) When called on datasets of type (K, V) and (K, W), returns a dataset of (K, (V, W)) pairs with all pairs of elements for each key. + Outer joins are supported through leftOuterJoin, rightOuterJoin, and fullOuterJoin. +
    cogroup(otherDataset, [numPartitions]) When called on datasets of type (K, V) and (K, W), returns a dataset of (K, (Iterable<V>, Iterable<W>)) tuples. This operation is also called groupWith.
    cartesian(otherDataset) When called on datasets of types T and U, returns a dataset of (T, U) pairs (all pairs of elements).
    pipe(command, [envVars]) Pipe each partition of the RDD through a shell command, e.g. a Perl or bash script. RDD elements are written to the + process's stdin and lines output to its stdout are returned as an RDD of strings.
    coalesce(numPartitions) Decrease the number of partitions in the RDD to numPartitions. Useful for running operations more efficiently + after filtering down a large dataset.
    repartition(numPartitions) Reshuffle the data in the RDD randomly to create either more or fewer partitions and balance it across them. + This always shuffles all data over the network.
    repartitionAndSortWithinPartitions(partitioner) Repartition the RDD according to the given partitioner and, within each resulting partition, + sort records by their keys. This is more efficient than calling repartition and then sorting within + each partition because it can push the sorting down into the shuffle machinery.
    + +### Actions + +The following table lists some of the common actions supported by Spark. Refer to the +RDD API doc +([Scala](api/scala/index.html#org.apache.spark.rdd.RDD), + [Java](api/java/index.html?org/apache/spark/api/java/JavaRDD.html), + [Python](api/python/pyspark.html#pyspark.RDD), + [R](api/R/index.html)) + +and pair RDD functions doc +([Scala](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions), + [Java](api/java/index.html?org/apache/spark/api/java/JavaPairRDD.html)) +for details. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    ActionMeaning
    reduce(func) Aggregate the elements of the dataset using a function func (which takes two arguments and returns one). The function should be commutative and associative so that it can be computed correctly in parallel.
    collect() Return all the elements of the dataset as an array at the driver program. This is usually useful after a filter or other operation that returns a sufficiently small subset of the data.
    count() Return the number of elements in the dataset.
    first() Return the first element of the dataset (similar to take(1)).
    take(n) Return an array with the first n elements of the dataset.
    takeSample(withReplacement, num, [seed]) Return an array with a random sample of num elements of the dataset, with or without replacement, optionally pre-specifying a random number generator seed.
    takeOrdered(n, [ordering]) Return the first n elements of the RDD using either their natural order or a custom comparator.
    saveAsTextFile(path) Write the elements of the dataset as a text file (or set of text files) in a given directory in the local filesystem, HDFS or any other Hadoop-supported file system. Spark will call toString on each element to convert it to a line of text in the file.
    saveAsSequenceFile(path)
    (Java and Scala)
    Write the elements of the dataset as a Hadoop SequenceFile in a given path in the local filesystem, HDFS or any other Hadoop-supported file system. This is available on RDDs of key-value pairs that implement Hadoop's Writable interface. In Scala, it is also + available on types that are implicitly convertible to Writable (Spark includes conversions for basic types like Int, Double, String, etc).
    saveAsObjectFile(path)
    (Java and Scala)
    Write the elements of the dataset in a simple format using Java serialization, which can then be loaded using + SparkContext.objectFile().
    countByKey() Only available on RDDs of type (K, V). Returns a hashmap of (K, Int) pairs with the count of each key.
    foreach(func) Run a function func on each element of the dataset. This is usually done for side effects such as updating an Accumulator or interacting with external storage systems. +
    Note: modifying variables other than Accumulators outside of the foreach() may result in undefined behavior. See Understanding closures for more details.
    + +The Spark RDD API also exposes asynchronous versions of some actions, like `foreachAsync` for `foreach`, which immediately return a `FutureAction` to the caller instead of blocking on completion of the action. This can be used to manage or wait for the asynchronous execution of the action. + + +### Shuffle operations + +Certain operations within Spark trigger an event known as the shuffle. The shuffle is Spark's +mechanism for re-distributing data so that it's grouped differently across partitions. This typically +involves copying data across executors and machines, making the shuffle a complex and +costly operation. + +#### Background + +To understand what happens during the shuffle we can consider the example of the +[`reduceByKey`](#ReduceByLink) operation. The `reduceByKey` operation generates a new RDD where all +values for a single key are combined into a tuple - the key and the result of executing a reduce +function against all values associated with that key. The challenge is that not all values for a +single key necessarily reside on the same partition, or even the same machine, but they must be +co-located to compute the result. + +In Spark, data is generally not distributed across partitions to be in the necessary place for a +specific operation. During computations, a single task will operate on a single partition - thus, to +organize all the data for a single `reduceByKey` reduce task to execute, Spark needs to perform an +all-to-all operation. It must read from all partitions to find all the values for all keys, +and then bring together values across partitions to compute the final result for each key - +this is called the **shuffle**. + +Although the set of elements in each partition of newly shuffled data will be deterministic, and so +is the ordering of partitions themselves, the ordering of these elements is not. If one desires predictably +ordered data following shuffle then it's possible to use: + +* `mapPartitions` to sort each partition using, for example, `.sorted` +* `repartitionAndSortWithinPartitions` to efficiently sort partitions while simultaneously repartitioning +* `sortBy` to make a globally ordered RDD + +Operations which can cause a shuffle include **repartition** operations like +[`repartition`](#RepartitionLink) and [`coalesce`](#CoalesceLink), **'ByKey** operations +(except for counting) like [`groupByKey`](#GroupByLink) and [`reduceByKey`](#ReduceByLink), and +**join** operations like [`cogroup`](#CogroupLink) and [`join`](#JoinLink). + +#### Performance Impact +The **Shuffle** is an expensive operation since it involves disk I/O, data serialization, and +network I/O. To organize data for the shuffle, Spark generates sets of tasks - *map* tasks to +organize the data, and a set of *reduce* tasks to aggregate it. This nomenclature comes from +MapReduce and does not directly relate to Spark's `map` and `reduce` operations. + +Internally, results from individual map tasks are kept in memory until they can't fit. Then, these +are sorted based on the target partition and written to a single file. On the reduce side, tasks +read the relevant sorted blocks. + +Certain shuffle operations can consume significant amounts of heap memory since they employ +in-memory data structures to organize records before or after transferring them. Specifically, +`reduceByKey` and `aggregateByKey` create these structures on the map side, and `'ByKey` operations +generate these on the reduce side. When data does not fit in memory Spark will spill these tables +to disk, incurring the additional overhead of disk I/O and increased garbage collection. + +Shuffle also generates a large number of intermediate files on disk. As of Spark 1.3, these files +are preserved until the corresponding RDDs are no longer used and are garbage collected. +This is done so the shuffle files don't need to be re-created if the lineage is re-computed. +Garbage collection may happen only after a long period of time, if the application retains references +to these RDDs or if GC does not kick in frequently. This means that long-running Spark jobs may +consume a large amount of disk space. The temporary storage directory is specified by the +`spark.local.dir` configuration parameter when configuring the Spark context. + +Shuffle behavior can be tuned by adjusting a variety of configuration parameters. See the +'Shuffle Behavior' section within the [Spark Configuration Guide](configuration.html). + +## RDD Persistence + +One of the most important capabilities in Spark is *persisting* (or *caching*) a dataset in memory +across operations. When you persist an RDD, each node stores any partitions of it that it computes in +memory and reuses them in other actions on that dataset (or datasets derived from it). This allows +future actions to be much faster (often by more than 10x). Caching is a key tool for +iterative algorithms and fast interactive use. + +You can mark an RDD to be persisted using the `persist()` or `cache()` methods on it. The first time +it is computed in an action, it will be kept in memory on the nodes. Spark's cache is fault-tolerant -- +if any partition of an RDD is lost, it will automatically be recomputed using the transformations +that originally created it. + +In addition, each persisted RDD can be stored using a different *storage level*, allowing you, for example, +to persist the dataset on disk, persist it in memory but as serialized Java objects (to save space), +replicate it across nodes. +These levels are set by passing a +`StorageLevel` object ([Scala](api/scala/index.html#org.apache.spark.storage.StorageLevel), +[Java](api/java/index.html?org/apache/spark/storage/StorageLevel.html), +[Python](api/python/pyspark.html#pyspark.StorageLevel)) +to `persist()`. The `cache()` method is a shorthand for using the default storage level, +which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The full set of +storage levels is: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Storage LevelMeaning
    MEMORY_ONLY Store RDD as deserialized Java objects in the JVM. If the RDD does not fit in memory, some partitions will + not be cached and will be recomputed on the fly each time they're needed. This is the default level.
    MEMORY_AND_DISK Store RDD as deserialized Java objects in the JVM. If the RDD does not fit in memory, store the + partitions that don't fit on disk, and read them from there when they're needed.
    MEMORY_ONLY_SER
    (Java and Scala)
    Store RDD as serialized Java objects (one byte array per partition). + This is generally more space-efficient than deserialized objects, especially when using a + fast serializer, but more CPU-intensive to read. +
    MEMORY_AND_DISK_SER
    (Java and Scala)
    Similar to MEMORY_ONLY_SER, but spill partitions that don't fit in memory to disk instead of + recomputing them on the fly each time they're needed.
    DISK_ONLY Store the RDD partitions only on disk.
    MEMORY_ONLY_2, MEMORY_AND_DISK_2, etc. Same as the levels above, but replicate each partition on two cluster nodes.
    OFF_HEAP (experimental) Similar to MEMORY_ONLY_SER, but store the data in + off-heap memory. This requires off-heap memory to be enabled.
    + +**Note:** *In Python, stored objects will always be serialized with the [Pickle](https://docs.python.org/2/library/pickle.html) library, +so it does not matter whether you choose a serialized level. The available storage levels in Python include `MEMORY_ONLY`, `MEMORY_ONLY_2`, +`MEMORY_AND_DISK`, `MEMORY_AND_DISK_2`, `DISK_ONLY`, and `DISK_ONLY_2`.* + +Spark also automatically persists some intermediate data in shuffle operations (e.g. `reduceByKey`), even without users calling `persist`. This is done to avoid recomputing the entire input if a node fails during the shuffle. We still recommend users call `persist` on the resulting RDD if they plan to reuse it. + +### Which Storage Level to Choose? + +Spark's storage levels are meant to provide different trade-offs between memory usage and CPU +efficiency. We recommend going through the following process to select one: + +* If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY`), leave them that way. + This is the most CPU-efficient option, allowing operations on the RDDs to run as fast as possible. + +* If not, try using `MEMORY_ONLY_SER` and [selecting a fast serialization library](tuning.html) to +make the objects much more space-efficient, but still reasonably fast to access. (Java and Scala) + +* Don't spill to disk unless the functions that computed your datasets are expensive, or they filter +a large amount of the data. Otherwise, recomputing a partition may be as fast as reading it from +disk. + +* Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve +requests from a web application). *All* the storage levels provide full fault tolerance by +recomputing lost data, but the replicated ones let you continue running tasks on the RDD without +waiting to recompute a lost partition. + + +### Removing Data + +Spark automatically monitors cache usage on each node and drops out old data partitions in a +least-recently-used (LRU) fashion. If you would like to manually remove an RDD instead of waiting for +it to fall out of the cache, use the `RDD.unpersist()` method. + +# Shared Variables + +Normally, when a function passed to a Spark operation (such as `map` or `reduce`) is executed on a +remote cluster node, it works on separate copies of all the variables used in the function. These +variables are copied to each machine, and no updates to the variables on the remote machine are +propagated back to the driver program. Supporting general, read-write shared variables across tasks +would be inefficient. However, Spark does provide two limited types of *shared variables* for two +common usage patterns: broadcast variables and accumulators. + +## Broadcast Variables + +Broadcast variables allow the programmer to keep a read-only variable cached on each machine rather +than shipping a copy of it with tasks. They can be used, for example, to give every node a copy of a +large input dataset in an efficient manner. Spark also attempts to distribute broadcast variables +using efficient broadcast algorithms to reduce communication cost. + +Spark actions are executed through a set of stages, separated by distributed "shuffle" operations. +Spark automatically broadcasts the common data needed by tasks within each stage. The data +broadcasted this way is cached in serialized form and deserialized before running each task. This +means that explicitly creating broadcast variables is only useful when tasks across multiple stages +need the same data or when caching the data in deserialized form is important. + +Broadcast variables are created from a variable `v` by calling `SparkContext.broadcast(v)`. The +broadcast variable is a wrapper around `v`, and its value can be accessed by calling the `value` +method. The code below shows this: + +
    + +
    + +{% highlight scala %} +scala> val broadcastVar = sc.broadcast(Array(1, 2, 3)) +broadcastVar: org.apache.spark.broadcast.Broadcast[Array[Int]] = Broadcast(0) + +scala> broadcastVar.value +res0: Array[Int] = Array(1, 2, 3) +{% endhighlight %} + +
    + +
    + +{% highlight java %} +Broadcast broadcastVar = sc.broadcast(new int[] {1, 2, 3}); + +broadcastVar.value(); +// returns [1, 2, 3] +{% endhighlight %} + +
    + +
    + +{% highlight python %} +>>> broadcastVar = sc.broadcast([1, 2, 3]) + + +>>> broadcastVar.value +[1, 2, 3] +{% endhighlight %} + +
    + +
    + +After the broadcast variable is created, it should be used instead of the value `v` in any functions +run on the cluster so that `v` is not shipped to the nodes more than once. In addition, the object +`v` should not be modified after it is broadcast in order to ensure that all nodes get the same +value of the broadcast variable (e.g. if the variable is shipped to a new node later). + +## Accumulators + +Accumulators are variables that are only "added" to through an associative and commutative operation and can +therefore be efficiently supported in parallel. They can be used to implement counters (as in +MapReduce) or sums. Spark natively supports accumulators of numeric types, and programmers +can add support for new types. + +As a user, you can create named or unnamed accumulators. As seen in the image below, a named accumulator (in this instance `counter`) will display in the web UI for the stage that modifies that accumulator. Spark displays the value for each accumulator modified by a task in the "Tasks" table. + +

    + Accumulators in the Spark UI +

    + +Tracking accumulators in the UI can be useful for understanding the progress of +running stages (NOTE: this is not yet supported in Python). + +
    + +
    + +A numeric accumulator can be created by calling `SparkContext.longAccumulator()` or `SparkContext.doubleAccumulator()` +to accumulate values of type Long or Double, respectively. Tasks running on a cluster can then add to it using +the `add` method. However, they cannot read its value. Only the driver program can read the accumulator's value, +using its `value` method. + +The code below shows an accumulator being used to add up the elements of an array: + +{% highlight scala %} +scala> val accum = sc.longAccumulator("My Accumulator") +accum: org.apache.spark.util.LongAccumulator = LongAccumulator(id: 0, name: Some(My Accumulator), value: 0) + +scala> sc.parallelize(Array(1, 2, 3, 4)).foreach(x => accum.add(x)) +... +10/09/29 18:41:08 INFO SparkContext: Tasks finished in 0.317106 s + +scala> accum.value +res2: Long = 10 +{% endhighlight %} + +While this code used the built-in support for accumulators of type Long, programmers can also +create their own types by subclassing [AccumulatorV2](api/scala/index.html#org.apache.spark.util.AccumulatorV2). +The AccumulatorV2 abstract class has several methods which one has to override: `reset` for resetting +the accumulator to zero, `add` for adding another value into the accumulator, +`merge` for merging another same-type accumulator into this one. Other methods that must be overridden +are contained in the [API documentation](api/scala/index.html#org.apache.spark.util.AccumulatorV2). For example, supposing we had a `MyVector` class +representing mathematical vectors, we could write: + +{% highlight scala %} +class VectorAccumulatorV2 extends AccumulatorV2[MyVector, MyVector] { + + private val myVector: MyVector = MyVector.createZeroVector + + def reset(): Unit = { + myVector.reset() + } + + def add(v: MyVector): Unit = { + myVector.add(v) + } + ... +} + +// Then, create an Accumulator of this type: +val myVectorAcc = new VectorAccumulatorV2 +// Then, register it into spark context: +sc.register(myVectorAcc, "MyVectorAcc1") +{% endhighlight %} + +Note that, when programmers define their own type of AccumulatorV2, the resulting type can be different than that of the elements added. + +
    + +
    + +A numeric accumulator can be created by calling `SparkContext.longAccumulator()` or `SparkContext.doubleAccumulator()` +to accumulate values of type Long or Double, respectively. Tasks running on a cluster can then add to it using +the `add` method. However, they cannot read its value. Only the driver program can read the accumulator's value, +using its `value` method. + +The code below shows an accumulator being used to add up the elements of an array: + +{% highlight java %} +LongAccumulator accum = jsc.sc().longAccumulator(); + +sc.parallelize(Arrays.asList(1, 2, 3, 4)).foreach(x -> accum.add(x)); +// ... +// 10/09/29 18:41:08 INFO SparkContext: Tasks finished in 0.317106 s + +accum.value(); +// returns 10 +{% endhighlight %} + +While this code used the built-in support for accumulators of type Long, programmers can also +create their own types by subclassing [AccumulatorV2](api/scala/index.html#org.apache.spark.util.AccumulatorV2). +The AccumulatorV2 abstract class has several methods which one has to override: `reset` for resetting +the accumulator to zero, `add` for adding another value into the accumulator, +`merge` for merging another same-type accumulator into this one. Other methods that must be overridden +are contained in the [API documentation](api/scala/index.html#org.apache.spark.util.AccumulatorV2). For example, supposing we had a `MyVector` class +representing mathematical vectors, we could write: + +{% highlight java %} +class VectorAccumulatorV2 implements AccumulatorV2 { + + private MyVector myVector = MyVector.createZeroVector(); + + public void reset() { + myVector.reset(); + } + + public void add(MyVector v) { + myVector.add(v); + } + ... +} + +// Then, create an Accumulator of this type: +VectorAccumulatorV2 myVectorAcc = new VectorAccumulatorV2(); +// Then, register it into spark context: +jsc.sc().register(myVectorAcc, "MyVectorAcc1"); +{% endhighlight %} + +Note that, when programmers define their own type of AccumulatorV2, the resulting type can be different than that of the elements added. + +
    + +
    + +An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks +running on a cluster can then add to it using the `add` method or the `+=` operator. However, they cannot read its value. +Only the driver program can read the accumulator's value, using its `value` method. + +The code below shows an accumulator being used to add up the elements of an array: + +{% highlight python %} +>>> accum = sc.accumulator(0) +>>> accum +Accumulator + +>>> sc.parallelize([1, 2, 3, 4]).foreach(lambda x: accum.add(x)) +... +10/09/29 18:41:08 INFO SparkContext: Tasks finished in 0.317106 s + +>>> accum.value +10 +{% endhighlight %} + +While this code used the built-in support for accumulators of type Int, programmers can also +create their own types by subclassing [AccumulatorParam](api/python/pyspark.html#pyspark.AccumulatorParam). +The AccumulatorParam interface has two methods: `zero` for providing a "zero value" for your data +type, and `addInPlace` for adding two values together. For example, supposing we had a `Vector` class +representing mathematical vectors, we could write: + +{% highlight python %} +class VectorAccumulatorParam(AccumulatorParam): + def zero(self, initialValue): + return Vector.zeros(initialValue.size) + + def addInPlace(self, v1, v2): + v1 += v2 + return v1 + +# Then, create an Accumulator of this type: +vecAccum = sc.accumulator(Vector(...), VectorAccumulatorParam()) +{% endhighlight %} + +
    + +
    + +For accumulator updates performed inside actions only, Spark guarantees that each task's update to the accumulator +will only be applied once, i.e. restarted tasks will not update the value. In transformations, users should be aware +of that each task's update may be applied more than once if tasks or job stages are re-executed. + +Accumulators do not change the lazy evaluation model of Spark. If they are being updated within an operation on an RDD, their value is only updated once that RDD is computed as part of an action. Consequently, accumulator updates are not guaranteed to be executed when made within a lazy transformation like `map()`. The below code fragment demonstrates this property: + +
    + +
    +{% highlight scala %} +val accum = sc.longAccumulator +data.map { x => accum.add(x); x } +// Here, accum is still 0 because no actions have caused the map operation to be computed. +{% endhighlight %} +
    + +
    +{% highlight java %} +LongAccumulator accum = jsc.sc().longAccumulator(); +data.map(x -> { accum.add(x); return f(x); }); +// Here, accum is still 0 because no actions have caused the `map` to be computed. +{% endhighlight %} +
    + +
    +{% highlight python %} +accum = sc.accumulator(0) +def g(x): + accum.add(x) + return f(x) +data.map(g) +# Here, accum is still 0 because no actions have caused the `map` to be computed. +{% endhighlight %} +
    + +
    + +# Deploying to a Cluster + +The [application submission guide](submitting-applications.html) describes how to submit applications to a cluster. +In short, once you package your application into a JAR (for Java/Scala) or a set of `.py` or `.zip` files (for Python), +the `bin/spark-submit` script lets you submit it to any supported cluster manager. + +# Launching Spark jobs from Java / Scala + +The [org.apache.spark.launcher](api/java/index.html?org/apache/spark/launcher/package-summary.html) +package provides classes for launching Spark jobs as child processes using a simple Java API. + +# Unit Testing + +Spark is friendly to unit testing with any popular unit test framework. +Simply create a `SparkContext` in your test with the master URL set to `local`, run your operations, +and then call `SparkContext.stop()` to tear it down. +Make sure you stop the context within a `finally` block or the test framework's `tearDown` method, +as Spark does not support two contexts running concurrently in the same program. + +# Where to Go from Here + +You can see some [example Spark programs](http://spark.apache.org/examples.html) on the Spark website. +In addition, Spark includes several samples in the `examples` directory +([Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples), + [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples), + [Python]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python), + [R]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/r)). +You can run Java and Scala examples by passing the class name to Spark's `bin/run-example` script; for instance: + + ./bin/run-example SparkPi + +For Python examples, use `spark-submit` instead: + + ./bin/spark-submit examples/src/main/python/pi.py + +For R examples, use `spark-submit` instead: + + ./bin/spark-submit examples/src/main/r/dataframe.R + +For help on optimizing your programs, the [configuration](configuration.html) and +[tuning](tuning.html) guides provide information on best practices. They are especially important for +making sure that your data is stored in memory in an efficient format. +For help on deploying, the [cluster mode overview](cluster-overview.html) describes the components involved +in distributed operation and supported cluster managers. + +Finally, full API documentation is available in +[Scala](api/scala/#org.apache.spark.package), [Java](api/java/), [Python](api/python/) and [R](api/R/). diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index ec5a44d79212..e0944bc9f5f8 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -10,7 +10,7 @@ Spark can run on hardware clusters managed by [Apache Mesos](http://mesos.apache The advantages of deploying Spark with Mesos include: - dynamic partitioning between Spark and other - [frameworks](https://mesos.apache.org/documentation/latest/mesos-frameworks/) + [frameworks](https://mesos.apache.org/documentation/latest/frameworks/) - scalable partitioning between multiple instances of Spark # How it Works @@ -32,8 +32,9 @@ To get started, follow the steps below to install Mesos and deploy Spark jobs vi # Installing Mesos -Spark {{site.SPARK_VERSION}} is designed for use with Mesos {{site.MESOS_VERSION}} and does not -require any special patches of Mesos. +Spark {{site.SPARK_VERSION}} is designed for use with Mesos {{site.MESOS_VERSION}} or newer and does not +require any special patches of Mesos. File and environment-based secrets support requires Mesos 1.3.0 or +newer. If you already have a Mesos cluster running, you can skip this Mesos installation step. @@ -61,7 +62,7 @@ third party projects publish binary releases that may be helpful in setting Meso One of those is Mesosphere. To install Mesos using the binary releases provided by Mesosphere: -1. Download Mesos installation package from [downloads page](http://mesosphere.io/downloads/) +1. Download Mesos installation package from [downloads page](https://open.mesosphere.com/downloads/mesos/) 2. Follow their instructions for installation and configuration The Mesosphere installation documents suggest setting up ZooKeeper to handle Mesos master failover, @@ -98,17 +99,17 @@ To host on HDFS, use the Hadoop fs put command: `hadoop fs -put spark-{{site.SPA Or if you are using a custom-compiled version of Spark, you will need to create a package using -the `make-distribution.sh` script included in a Spark source tarball/checkout. +the `dev/make-distribution.sh` script included in a Spark source tarball/checkout. 1. Download and build Spark using the instructions [here](index.html) -2. Create a binary package using `make-distribution.sh --tgz`. +2. Create a binary package using `./dev/make-distribution.sh --tgz`. 3. Upload archive to http/s3/hdfs ## Using a Mesos Master URL The Master URLs for Mesos are in the form `mesos://host:5050` for a single-master Mesos -cluster, or `mesos://zk://host:2181` for a multi-master Mesos cluster using ZooKeeper. +cluster, or `mesos://zk://host1:2181,host2:2181,host3:2181/mesos` for a multi-master Mesos cluster using ZooKeeper. ## Client Mode @@ -150,49 +151,119 @@ it does not need to be redundantly passed in as a system property. Spark on Mesos also supports cluster mode, where the driver is launched in the cluster and the client can find the results of the driver from the Mesos Web UI. -To use cluster mode, you must start the MesosClusterDispatcher in your cluster via the `sbin/start-mesos-dispatcher.sh` script, -passing in the Mesos master url (e.g: mesos://host:5050). +To use cluster mode, you must start the `MesosClusterDispatcher` in your cluster via the `sbin/start-mesos-dispatcher.sh` script, +passing in the Mesos master URL (e.g: mesos://host:5050). This starts the `MesosClusterDispatcher` as a daemon running on the host. -From the client, you can submit a job to Mesos cluster by running `spark-submit` and specifying the master url -to the url of the MesosClusterDispatcher (e.g: mesos://dispatcher:7077). You can view driver statuses on the -Spark cluster Web UI. +By setting the Mesos proxy config property (requires mesos version >= 1.4), `--conf spark.mesos.proxy.baseURL=http://localhost:5050` when launching the dispacther, the mesos sandbox URI for each driver is added to the mesos dispatcher UI. -Note that jars or python files that are passed to spark-submit should be URIs reachable by Mesos slaves. +If you like to run the `MesosClusterDispatcher` with Marathon, you need to run the `MesosClusterDispatcher` in the foreground (i.e: `bin/spark-class org.apache.spark.deploy.mesos.MesosClusterDispatcher`). Note that the `MesosClusterDispatcher` not yet supports multiple instances for HA. -# Mesos Run Modes +The `MesosClusterDispatcher` also supports writing recovery state into Zookeeper. This will allow the `MesosClusterDispatcher` to be able to recover all submitted and running containers on relaunch. In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env by configuring `spark.deploy.recoveryMode` and related spark.deploy.zookeeper.* configurations. +For more information about these configurations please refer to the configurations [doc](configurations.html#deploy). + +You can also specify any additional jars required by the `MesosClusterDispatcher` in the classpath by setting the environment variable SPARK_DAEMON_CLASSPATH in spark-env. -Spark can run over Mesos in two modes: "fine-grained" (default) and "coarse-grained". +From the client, you can submit a job to Mesos cluster by running `spark-submit` and specifying the master URL +to the URL of the `MesosClusterDispatcher` (e.g: mesos://dispatcher:7077). You can view driver statuses on the +Spark cluster Web UI. -In "fine-grained" mode (default), each Spark task runs as a separate Mesos task. This allows -multiple instances of Spark (and other frameworks) to share machines at a very fine granularity, -where each application gets more or fewer machines as it ramps up and down, but it comes with an -additional overhead in launching each task. This mode may be inappropriate for low-latency -requirements like interactive queries or serving web requests. +For example: +{% highlight bash %} +./bin/spark-submit \ + --class org.apache.spark.examples.SparkPi \ + --master mesos://207.184.161.138:7077 \ + --deploy-mode cluster \ + --supervise \ + --executor-memory 20G \ + --total-executor-cores 100 \ + http://path/to/examples.jar \ + 1000 +{% endhighlight %} -The "coarse-grained" mode will instead launch only *one* long-running Spark task on each Mesos -machine, and dynamically schedule its own "mini-tasks" within it. The benefit is much lower startup -overhead, but at the cost of reserving the Mesos resources for the complete duration of the -application. -To run in coarse-grained mode, set the `spark.mesos.coarse` property in your +Note that jars or python files that are passed to spark-submit should be URIs reachable by Mesos slaves, as the Spark driver doesn't automatically upload local jars. + +# Mesos Run Modes + +Spark can run over Mesos in two modes: "coarse-grained" (default) and +"fine-grained" (deprecated). + +## Coarse-Grained + +In "coarse-grained" mode, each Spark executor runs as a single Mesos +task. Spark executors are sized according to the following +configuration variables: + +* Executor memory: `spark.executor.memory` +* Executor cores: `spark.executor.cores` +* Number of executors: `spark.cores.max`/`spark.executor.cores` + +Please see the [Spark Configuration](configuration.html) page for +details and default values. + +Executors are brought up eagerly when the application starts, until +`spark.cores.max` is reached. If you don't set `spark.cores.max`, the +Spark application will reserve all resources offered to it by Mesos, +so we of course urge you to set this variable in any sort of +multi-tenant cluster, including one which runs multiple concurrent +Spark applications. + +The scheduler will start executors round-robin on the offers Mesos +gives it, but there are no spread guarantees, as Mesos does not +provide such guarantees on the offer stream. + +In this mode spark executors will honor port allocation if such is +provided from the user. Specifically if the user defines +`spark.blockManager.port` in Spark configuration, +the mesos scheduler will check the available offers for a valid port +range containing the port numbers. If no such range is available it will +not launch any task. If no restriction is imposed on port numbers by the +user, ephemeral ports are used as usual. This port honouring implementation +implies one task per host if the user defines a port. In the future network +isolation shall be supported. + +The benefit of coarse-grained mode is much lower startup overhead, but +at the cost of reserving Mesos resources for the complete duration of +the application. To configure your job to dynamically adjust to its +resource requirements, look into +[Dynamic Allocation](#dynamic-resource-allocation-with-mesos). + +## Fine-Grained (deprecated) + +**NOTE:** Fine-grained mode is deprecated as of Spark 2.0.0. Consider + using [Dynamic Allocation](#dynamic-resource-allocation-with-mesos) + for some of the benefits. For a full explanation see + [SPARK-11857](https://issues.apache.org/jira/browse/SPARK-11857) + +In "fine-grained" mode, each Spark task inside the Spark executor runs +as a separate Mesos task. This allows multiple instances of Spark (and +other frameworks) to share cores at a very fine granularity, where +each application gets more or fewer cores as it ramps up and down, but +it comes with an additional overhead in launching each task. This mode +may be inappropriate for low-latency requirements like interactive +queries or serving web requests. + +Note that while Spark tasks in fine-grained will relinquish cores as +they terminate, they will not relinquish memory, as the JVM does not +give memory back to the Operating System. Neither will executors +terminate when they're idle. + +To run in fine-grained mode, set the `spark.mesos.coarse` property to false in your [SparkConf](configuration.html#spark-properties): {% highlight scala %} -conf.set("spark.mesos.coarse", "true") +conf.set("spark.mesos.coarse", "false") {% endhighlight %} -In addition, for coarse-grained mode, you can control the maximum number of resources Spark will -acquire. By default, it will acquire *all* cores in the cluster (that get offered by Mesos), which -only makes sense if you run just one application at a time. You can cap the maximum number of cores -using `conf.set("spark.cores.max", "10")` (for example). - -You may also make use of `spark.mesos.constraints` to set attribute based constraints on mesos resource offers. By default, all resource offers will be accepted. +You may also make use of `spark.mesos.constraints` to set +attribute-based constraints on Mesos resource offers. By default, all +resource offers will be accepted. {% highlight scala %} -conf.set("spark.mesos.constraints", "tachyon:true;us-east-1:false") +conf.set("spark.mesos.constraints", "os:centos7;us-east-1:false") {% endhighlight %} -For example, Let's say `spark.mesos.constraints` is set to `tachyon:true;us-east-1:false`, then the resource offers will be checked to see if they meet both these constraints and only then will be accepted to start new executors. +For example, Let's say `spark.mesos.constraints` is set to `os:centos7;us-east-1:false`, then the resource offers will be checked to see if they meet both these constraints and only then will be accepted to start new executors. # Mesos Docker Support @@ -204,6 +275,10 @@ have Mesos download Spark via the usual methods. Requires Mesos version 0.20.1 or later. +Note that by default Mesos agents will not pull the image if it already exists on the agent. If you use mutable image +tags you can set `spark.mesos.executor.docker.forcePullImage` to `true` in order to force the agent to always pull the +image before running the executor. Force pulling images is only available in Mesos version 0.22 and above. + # Running Alongside Hadoop You can run Spark and Mesos alongside your existing Hadoop cluster by just launching them as a @@ -220,18 +295,15 @@ In either case, HDFS runs separately from Hadoop MapReduce, without being schedu # Dynamic Resource Allocation with Mesos -Mesos supports dynamic allocation only with coarse grain mode, which can resize the number of executors based on statistics -of the application. While dynamic allocation supports both scaling up and scaling down the number of executors, the coarse grain scheduler only supports scaling down -since it is already designed to run one executor per slave with the configured amount of resources. However, after scaling down the number of executors the coarse grain scheduler -can scale back up to the same amount of executors when Spark signals more executors are needed. +Mesos supports dynamic allocation only with coarse-grained mode, which can resize the number of +executors based on statistics of the application. For general information, +see [Dynamic Resource Allocation](job-scheduling.html#dynamic-resource-allocation). -Users that like to utilize this feature should launch the Mesos Shuffle Service that -provides shuffle data cleanup functionality on top of the Shuffle Service since Mesos doesn't yet support notifying another framework's -termination. To launch/stop the Mesos Shuffle Service please use the provided sbin/start-mesos-shuffle-service.sh and sbin/stop-mesos-shuffle-service.sh -scripts accordingly. +The External Shuffle Service to use is the Mesos Shuffle Service. It provides shuffle data cleanup functionality +on top of the Shuffle Service since Mesos doesn't yet support notifying another framework's +termination. To launch it, run `$SPARK_HOME/sbin/start-mesos-shuffle-service.sh` on all slave nodes, with `spark.shuffle.service.enabled` set to `true`. -The Shuffle Service is expected to be running on each slave node that will run Spark executors. One way to easily achieve this with Mesos -is to launch the Shuffle Service with Marathon with a unique host constraint. +This can also be achieved through Marathon, using a unique host constraint, and the following command: `bin/spark-class org.apache.spark.deploy.mesos.MesosExternalShuffleService`. # Configuration @@ -243,22 +315,22 @@ See the [configuration page](configuration.html) for information on Spark config Property NameDefaultMeaning spark.mesos.coarse - false + true - If set to true, runs over Mesos clusters in - "coarse-grained" sharing mode, - where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per - Spark task. This gives lower-latency scheduling for short queries, but leaves resources in use - for the whole duration of the Spark job. + If set to true, runs over Mesos clusters in "coarse-grained" sharing mode, where Spark acquires one long-lived Mesos task on each machine. + If set to false, runs over Mesos cluster in "fine-grained" sharing mode, where one Mesos task is created per Spark task. + Detailed information in 'Mesos Run Modes'. spark.mesos.extra.cores 0 - Set the extra amount of cpus to request per task. This setting is only used for Mesos coarse grain mode. - The total amount of cores requested per task is the number of cores in the offer plus the extra cores configured. - Note that total amount of cores the executor will request in total will not exceed the spark.cores.max setting. + Set the extra number of cores for an executor to advertise. This + does not result in more cores allocated. It instead means that an + executor will "pretend" it has more cores, so that the driver will + send it more tasks. Use this to increase parallelism. This + setting is only used for Mesos coarse-grained mode. @@ -278,7 +350,25 @@ See the [configuration page](configuration.html) for information on Spark config Set the name of the docker image that the Spark executors will run in. The selected image must have Spark installed, as well as a compatible version of the Mesos library. The installed path of Spark in the image can be specified with spark.mesos.executor.home; - the installed path of the Mesos library can be specified with spark.executorEnv.MESOS_NATIVE_LIBRARY. + the installed path of the Mesos library can be specified with spark.executorEnv.MESOS_NATIVE_JAVA_LIBRARY. + + + + spark.mesos.executor.docker.forcePullImage + false + + Force Mesos agents to pull the image specified in spark.mesos.executor.docker.image. + By default Mesos agents will not pull images they already have cached. + + + + spark.mesos.executor.docker.parameters + (none) + + Set the list of custom parameters which will be passed into the docker run command when launching the Spark executor on Mesos using the docker containerizer. The format of this property is a comma-separated list of + key/value pairs. Example: + +
    key1=val1,key2=val2,key3=val3
    @@ -293,14 +383,13 @@ See the [configuration page](configuration.html) for information on Spark config - spark.mesos.executor.docker.portmaps + spark.mesos.task.labels (none) - Set the list of incoming ports exposed by the Docker image, which was set using - spark.mesos.executor.docker.image. The format of this property is a comma-separated list of - mappings which take the form: - -
    host_port:container_port[:tcp|:udp]
    + Set the Mesos labels to add to each task. Labels are free-form key-value pairs. + Key-value pairs should be separated by a colon, and commas used to + list more than one. If your label includes a colon or comma, you + can escape it with a backslash. Ex. key:value,key2:a\:b. @@ -326,8 +415,9 @@ See the [configuration page](configuration.html) for information on Spark config spark.mesos.uris (none) - A list of URIs to be downloaded to the sandbox when driver or executor is launched by Mesos. - This applies to both coarse-grain and fine-grain mode. + A comma-separated list of URIs to be downloaded to the sandbox + when driver or executor is launched by Mesos. This applies to + both coarse-grained and fine-grained mode. @@ -341,7 +431,8 @@ See the [configuration page](configuration.html) for information on Spark config spark.mesos.secret (none) - Set the secret with which Spark framework will use to authenticate with Mesos. + Set the secret with which Spark framework will use to authenticate with Mesos. Used, for example, when + authenticating with the registry. @@ -361,11 +452,167 @@ See the [configuration page](configuration.html) for information on Spark config
  • Scalar constraints are matched with "less than equal" semantics i.e. value in the constraint must be less than or equal to the value in the resource offer.
  • Range constraints are matched with "contains" semantics i.e. value in the constraint must be within the resource offer's value.
  • Set constraints are matched with "subset of" semantics i.e. value in the constraint must be a subset of the resource offer's value.
  • -
  • Text constraints are metched with "equality" semantics i.e. value in the constraint must be exactly equal to the resource offer's value.
  • +
  • Text constraints are matched with "equality" semantics i.e. value in the constraint must be exactly equal to the resource offer's value.
  • In case there is no value present as a part of the constraint any offer with the corresponding attribute will be accepted (without value check).
  • + + spark.mesos.containerizer + docker + + This only affects docker containers, and must be one of "docker" + or "mesos". Mesos supports two types of + containerizers for docker: the "docker" containerizer, and the preferred + "mesos" containerizer. Read more here: http://mesos.apache.org/documentation/latest/container-image/ + + + + spark.mesos.driver.webui.url + (none) + + Set the Spark Mesos driver webui_url for interacting with the framework. + If unset it will point to Spark's internal web UI. + + + + spark.mesos.driver.labels + (none) + + Mesos labels to add to the driver. See spark.mesos.task.labels + for formatting information. + + + + + spark.mesos.driver.secret.envkeys + (none) + + A comma-separated list that, if set, the contents of the secret referenced + by spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be + set to the provided environment variable in the driver's process. + + + +spark.mesos.driver.secret.filenames + (none) + + A comma-separated list that, if set, the contents of the secret referenced by + spark.mesos.driver.secret.names or spark.mesos.driver.secret.values will be + written to the provided file. Paths are relative to the container's work + directory. Absolute paths must already exist. Consult the Mesos Secret + protobuf for more information. + + + + spark.mesos.driver.secret.names + (none) + + A comma-separated list of secret references. Consult the Mesos Secret + protobuf for more information. + + + + spark.mesos.driver.secret.values + (none) + + A comma-separated list of secret values. Consult the Mesos Secret + protobuf for more information. + + + + + spark.mesos.driverEnv.[EnvironmentVariableName] + (none) + + This only affects drivers submitted in cluster mode. Add the + environment variable specified by EnvironmentVariableName to the + driver process. The user can specify multiple of these to set + multiple environment variables. + + + + spark.mesos.dispatcher.webui.url + (none) + + Set the Spark Mesos dispatcher webui_url for interacting with the framework. + If unset it will point to Spark's internal web UI. + + + + spark.mesos.dispatcher.driverDefault.[PropertyName] + (none) + + Set default properties for drivers submitted through the + dispatcher. For example, + spark.mesos.dispatcher.driverProperty.spark.executor.memory=32g + results in the executors for all drivers submitted in cluster mode + to run in 32g containers. + + + + spark.mesos.dispatcher.historyServer.url + (none) + + Set the URL of the history + server. The dispatcher will then link each driver to its entry + in the history server. + + + + spark.mesos.gpus.max + 0 + + Set the maximum number GPU resources to acquire for this job. Note that executors will still launch when no GPU resources are found + since this configuration is just a upper limit and not a guaranteed amount. + + + + spark.mesos.network.name + (none) + + Attach containers to the given named network. If this job is + launched in cluster mode, also launch the driver in the given named + network. See + the Mesos CNI docs + for more details. + + + + spark.mesos.network.labels + (none) + + Pass network labels to CNI plugins. This is a comma-separated list + of key-value pairs, where each key-value pair has the format key:value. + Example: + +
    key1:val1,key2:val2
    + See + the Mesos CNI docs + for more details. + + + + spark.mesos.fetcherCache.enable + false + + If set to `true`, all URIs (example: `spark.executor.uri`, + `spark.mesos.uris`) will be cached by the Mesos + Fetcher Cache + + + + spark.mesos.driver.failoverTimeout + 0.0 + + The amount of time (in seconds) that the master will wait for the + driver to reconnect, after being temporarily disconnected, before + it tears down the driver framework by killing all its + executors. The default value is zero, meaning no timeout: if the + driver disconnects, the master immediately tears down the framework. + + # Troubleshooting and Debugging diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index db6bfa69ee0f..432639588cc2 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -49,8 +49,8 @@ In `cluster` mode, the driver runs on a different machine than the client, so `S $ ./bin/spark-submit --class my.main.Class \ --master yarn \ --deploy-mode cluster \ - --jars my-other-jar.jar,my-other-other-jar.jar - my-main-jar.jar + --jars my-other-jar.jar,my-other-other-jar.jar \ + my-main-jar.jar \ app_arg1 app_arg2 @@ -60,6 +60,8 @@ Running Spark on YARN requires a binary distribution of Spark which is built wit Binary distributions can be downloaded from the [downloads page](http://spark.apache.org/downloads.html) of the project website. To build Spark yourself, refer to [Building Spark](building-spark.html). +To make Spark runtime jars accessible from YARN side, you can specify `spark.yarn.archive` or `spark.yarn.jars`. For details please refer to [Spark Properties](running-on-yarn.html#spark-properties). If neither `spark.yarn.archive` nor `spark.yarn.jars` is specified, Spark will create a zip file with all jars under `$SPARK_HOME/jars` and upload it to the distributed cache. + # Configuration Most of the configs are the same for Spark on YARN as for other deployment modes. See the [configuration page](configuration.html) for more information on those. These are configs that are specific to Spark on YARN. @@ -99,6 +101,8 @@ to the same log file). If you need a reference to the proper location to put log files in the YARN so that YARN can properly display and aggregate them, use `spark.yarn.app.container.log.dir` in your `log4j.properties`. For example, `log4j.appender.file_appender.File=${spark.yarn.app.container.log.dir}/spark.log`. For streaming applications, configuring `RollingFileAppender` and setting file location to YARN's log directory will avoid disk overflow caused by large log files, and logs can be accessed using YARN's log utility. +To use a custom metrics.properties for the application master and executors, update the `$SPARK_CONF_DIR/metrics.properties` file. It will automatically be uploaded with other configurations, so you don't need to specify it manually with `--files`. + #### Spark Properties @@ -113,15 +117,6 @@ If you need a reference to the proper location to put log files in the YARN so t Use lower-case suffixes, e.g. k, m, g, t, and p, for kibi-, mebi-, gibi-, tebi-, and pebibytes, respectively. - - - - - @@ -146,6 +141,13 @@ If you need a reference to the proper location to put log files in the YARN so t HDFS replication level for the files uploaded into HDFS for the application. These include things like the Spark jar, the app jar, and any distributed cache files/archives. + + + + + @@ -202,11 +204,27 @@ If you need a reference to the proper location to put log files in the YARN so t Comma-separated list of files to be placed in the working directory of each executor. + + + + + + + + + + @@ -230,13 +248,6 @@ If you need a reference to the proper location to put log files in the YARN so t Same as spark.yarn.driver.memoryOverhead, but for the YARN Application Master in client mode. - - - - - @@ -245,26 +256,38 @@ If you need a reference to the proper location to put log files in the YARN so t - + - + + + + + + @@ -290,7 +313,9 @@ If you need a reference to the proper location to put log files in the YARN so t @@ -314,7 +339,15 @@ If you need a reference to the proper location to put log files in the YARN so t + + + + + @@ -326,6 +359,15 @@ If you need a reference to the proper location to put log files in the YARN so t Otherwise, the client process will exit after submission. + + + + + @@ -349,14 +391,14 @@ If you need a reference to the proper location to put log files in the YARN so t @@ -386,15 +428,38 @@ If you need a reference to the proper location to put log files in the YARN so t - + + + + + + + + + + +
    spark.driver.cores1 - Number of cores used by the driver in YARN cluster mode. - Since the driver is run in the same JVM as the YARN Application Master in cluster mode, this also controls the cores used by the YARN Application Master. - In client mode, use spark.yarn.am.cores to control the number of cores used by the YARN Application Master instead. -
    spark.yarn.am.cores 1
    spark.yarn.stagingDirCurrent user's home directory in the filesystem + Staging directory used while submitting applications. +
    spark.yarn.preserve.staging.files false
    spark.yarn.dist.jars(none) + Comma-separated list of jars to be placed in the working directory of each executor. +
    spark.yarn.dist.forceDownloadSchemes(none) + Comma-separated list of schemes for which files will be downloaded to the local disk prior to + being added to YARN's distributed cache. For use in cases where the YARN service does not + support schemes that are supported by Spark, like http, https and ftp. +
    spark.executor.instances 2 - The number of executors. Note that this property is incompatible with spark.dynamicAllocation.enabled. If both spark.dynamicAllocation.enabled and spark.executor.instances are specified, dynamic allocation is turned off and the specified number of spark.executor.instances is used. + The number of executors for static allocation. With spark.dynamicAllocation.enabled, the initial set of executors will be at least this large.
    spark.yarn.am.port(random) - Port for the YARN Application Master to listen on. In YARN client mode, this is used to communicate between the Spark driver running on a gateway and the YARN Application Master running on YARN. In YARN cluster mode, this is used for the dynamic executor feature, where it handles the kill from the scheduler backend. -
    spark.yarn.queue default
    spark.yarn.jarspark.yarn.jars (none) - The location of the Spark jar file, in case overriding the default location is desired. - By default, Spark on YARN will use a Spark jar installed locally, but the Spark jar can also be + List of libraries containing Spark code to distribute to YARN containers. + By default, Spark on YARN will use Spark jars installed locally, but the Spark jars can also be in a world-readable location on HDFS. This allows YARN to cache it on nodes so that it doesn't - need to be distributed each time an application runs. To point to a jar on HDFS, for example, - set this configuration to hdfs:///some/path. + need to be distributed each time an application runs. To point to jars on HDFS, for example, + set this configuration to hdfs:///some/path. Globs are allowed.
    spark.yarn.access.namenodesspark.yarn.archive (none) - A comma-separated list of secure HDFS namenodes your Spark application is going to access. For - example, spark.yarn.access.namenodes=hdfs://nn1.com:8032,hdfs://nn2.com:8032. - The Spark application must have access to the namenodes listed and Kerberos must - be properly configured to be able to access them (either in the same realm or in - a trusted realm). Spark acquires security tokens for each of the namenodes so that - the Spark application can access those remote HDFS clusters. + An archive containing needed Spark jars for distribution to the YARN cache. If set, this + configuration replaces spark.yarn.jars and the archive is used in all the + application's containers. The archive should contain jar files in its root directory. + Like with the previous option, the archive can also be hosted on HDFS to speed up file + distribution. +
    spark.yarn.access.hadoopFileSystems(none) + A comma-separated list of secure Hadoop filesystems your Spark application is going to access. For + example, spark.yarn.access.hadoopFileSystems=hdfs://nn1.com:8032,hdfs://nn2.com:8032, + webhdfs://nn3.com:50070. The Spark application must have access to the filesystems listed + and Kerberos must be properly configured to be able to access them (either in the same realm + or in a trusted realm). Spark acquires security tokens for each of the filesystems so that + the Spark application can access those remote Hadoop filesystems. spark.yarn.access.namenodes + is deprecated, please use this instead.
    (none) A string of extra JVM options to pass to the YARN Application Master in client mode. - In cluster mode, use spark.driver.extraJavaOptions instead. + In cluster mode, use spark.driver.extraJavaOptions instead. Note that it is illegal + to set maximum heap size (-Xmx) settings with this option. Maximum heap size settings can be set + with spark.yarn.am.memory
    Defines the validity interval for AM failure tracking. If the AM has been running for at least the defined interval, the AM failure count will be reset. - This feature is not enabled if not configured, and only supported in Hadoop 2.6+. + This feature is not enabled if not configured. +
    spark.yarn.executor.failuresValidityInterval(none) + Defines the validity interval for executor failure tracking. + Executor failures which are older than the validity interval will be ignored.
    spark.yarn.am.nodeLabelExpression(none) + A YARN node label expression that restricts the set of nodes AM will be scheduled on. + Only versions of YARN greater than or equal to 2.6 support node label expressions, so when + running against earlier versions, this property will be ignored. +
    spark.yarn.executor.nodeLabelExpression (none) The full path to the file that contains the keytab for the principal specified above. This keytab will be copied to the node running the YARN Application Master via the Secure Distributed Cache, - for renewing the login tickets and the delegation tokens periodically. + for renewing the login tickets and the delegation tokens periodically. (Works also with the "local" master)
    spark.yarn.principal (none) - Principal to be used to login to KDC, while running on secure HDFS. + Principal to be used to login to KDC, while running on secure HDFS. (Works also with the "local" master)
    spark.yarn.security.tokens.${service}.enabledspark.security.credentials.${service}.enabled true - Controls whether to retrieve delegation tokens for non-HDFS services when security is enabled. - By default, delegation tokens for all supported services are retrieved when those services are + Controls whether to obtain credentials for services when security is enabled. + By default, credentials for all supported services are retrieved when those services are configured, but it's possible to disable that behavior if it somehow conflicts with the - application being run. -

    - Currently supported services are: hive, hbase + application being run. For further details please see + [Running in a Secure Cluster](running-on-yarn.html#running-in-a-secure-cluster) +

    spark.yarn.rolledLog.includePattern(none) + Java Regex to filter the log files which match the defined include pattern + and those log files will be aggregated in a rolling fashion. + This will be used with YARN's rolling log aggregation, to enable this feature in YARN side + yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds should be + configured in yarn-site.xml. + This feature can only be used with Hadoop 2.6.4+. The Spark log4j appender needs be changed to use + FileAppender or another appender that can handle the files being removed while its running. Based + on the file name configured in the log4j configuration (like spark.log), the user should set the + regex (spark*) to include all the log files that need to be aggregated. +
    spark.yarn.rolledLog.excludePattern(none) + Java Regex to filter the log files which match the defined exclude pattern + and those log files will not be aggregated in a rolling fashion. If the log file + name matches both the include and the exclude pattern, this file will be excluded eventually.
    @@ -405,3 +470,154 @@ If you need a reference to the proper location to put log files in the YARN so t - In `cluster` mode, the local directories used by the Spark executors and the Spark driver will be the local directories configured for YARN (Hadoop YARN config `yarn.nodemanager.local-dirs`). If the user specifies `spark.local.dir`, it will be ignored. In `client` mode, the Spark executors will use the local directories configured for YARN while the Spark driver will use those defined in `spark.local.dir`. This is because the Spark driver does not run on the YARN cluster in `client` mode, only the Spark executors do. - The `--files` and `--archives` options support specifying file names with the # similar to Hadoop. For example you can specify: `--files localtest.txt#appSees.txt` and this will upload the file you have locally named `localtest.txt` into HDFS but this will be linked to by the name `appSees.txt`, and your application should use the name as `appSees.txt` to reference it when running on YARN. - The `--jars` option allows the `SparkContext.addJar` function to work if you are using it with local files and running in `cluster` mode. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files. + +# Running in a Secure Cluster + +As covered in [security](security.html), Kerberos is used in a secure Hadoop cluster to +authenticate principals associated with services and clients. This allows clients to +make requests of these authenticated services; the services to grant rights +to the authenticated principals. + +Hadoop services issue *hadoop tokens* to grant access to the services and data. +Clients must first acquire tokens for the services they will access and pass them along with their +application as it is launched in the YARN cluster. + +For a Spark application to interact with any of the Hadoop filesystem (for example hdfs, webhdfs, etc), HBase and Hive, it must acquire the relevant tokens +using the Kerberos credentials of the user launching the application +—that is, the principal whose identity will become that of the launched Spark application. + +This is normally done at launch time: in a secure cluster Spark will automatically obtain a +token for the cluster's default Hadoop filesystem, and potentially for HBase and Hive. + +An HBase token will be obtained if HBase is in on classpath, the HBase configuration declares +the application is secure (i.e. `hbase-site.xml` sets `hbase.security.authentication` to `kerberos`), +and `spark.security.credentials.hbase.enabled` is not set to `false`. + +Similarly, a Hive token will be obtained if Hive is on the classpath, its configuration +includes a URI of the metadata store in `"hive.metastore.uris`, and +`spark.security.credentials.hive.enabled` is not set to `false`. + +If an application needs to interact with other secure Hadoop filesystems, then +the tokens needed to access these clusters must be explicitly requested at +launch time. This is done by listing them in the `spark.yarn.access.hadoopFileSystems` property. + +``` +spark.yarn.access.hadoopFileSystems hdfs://ireland.example.org:8020/,webhdfs://frankfurt.example.org:50070/ +``` + +Spark supports integrating with other security-aware services through Java Services mechanism (see +`java.util.ServiceLoader`). To do that, implementations of `org.apache.spark.deploy.yarn.security.ServiceCredentialProvider` +should be available to Spark by listing their names in the corresponding file in the jar's +`META-INF/services` directory. These plug-ins can be disabled by setting +`spark.security.credentials.{service}.enabled` to `false`, where `{service}` is the name of +credential provider. + +## Configuring the External Shuffle Service + +To start the Spark Shuffle Service on each `NodeManager` in your YARN cluster, follow these +instructions: + +1. Build Spark with the [YARN profile](building-spark.html). Skip this step if you are using a +pre-packaged distribution. +1. Locate the `spark--yarn-shuffle.jar`. This should be under +`$SPARK_HOME/common/network-yarn/target/scala-` if you are building Spark yourself, and under +`yarn` if you are using a distribution. +1. Add this jar to the classpath of all `NodeManager`s in your cluster. +1. In the `yarn-site.xml` on each node, add `spark_shuffle` to `yarn.nodemanager.aux-services`, +then set `yarn.nodemanager.aux-services.spark_shuffle.class` to +`org.apache.spark.network.yarn.YarnShuffleService`. +1. Increase `NodeManager's` heap size by setting `YARN_HEAPSIZE` (1000 by default) in `etc/hadoop/yarn-env.sh` +to avoid garbage collection issues during shuffle. +1. Restart all `NodeManager`s in your cluster. + +The following extra configuration options are available when the shuffle service is running on YARN: + + + + + + + + +
    Property NameDefaultMeaning
    spark.yarn.shuffle.stopOnFailurefalse + Whether to stop the NodeManager when there's a failure in the Spark Shuffle Service's + initialization. This prevents application failures caused by running containers on + NodeManagers where the Spark Shuffle Service is not running. +
    + +## Launching your application with Apache Oozie + +Apache Oozie can launch Spark applications as part of a workflow. +In a secure cluster, the launched application will need the relevant tokens to access the cluster's +services. If Spark is launched with a keytab, this is automatic. +However, if Spark is to be launched without a keytab, the responsibility for setting up security +must be handed over to Oozie. + +The details of configuring Oozie for secure clusters and obtaining +credentials for a job can be found on the [Oozie web site](http://oozie.apache.org/) +in the "Authentication" section of the specific release's documentation. + +For Spark applications, the Oozie workflow must be set up for Oozie to request all tokens which +the application needs, including: + +- The YARN resource manager. +- The local Hadoop filesystem. +- Any remote Hadoop filesystems used as a source or destination of I/O. +- Hive —if used. +- HBase —if used. +- The YARN timeline server, if the application interacts with this. + +To avoid Spark attempting —and then failing— to obtain Hive, HBase and remote HDFS tokens, +the Spark configuration must be set to disable token collection for the services. + +The Spark configuration must include the lines: + +``` +spark.security.credentials.hive.enabled false +spark.security.credentials.hbase.enabled false +``` + +The configuration option `spark.yarn.access.hadoopFileSystems` must be unset. + +## Troubleshooting Kerberos + +Debugging Hadoop/Kerberos problems can be "difficult". One useful technique is to +enable extra logging of Kerberos operations in Hadoop by setting the `HADOOP_JAAS_DEBUG` +environment variable. + +```bash +export HADOOP_JAAS_DEBUG=true +``` + +The JDK classes can be configured to enable extra logging of their Kerberos and +SPNEGO/REST authentication via the system properties `sun.security.krb5.debug` +and `sun.security.spnego.debug=true` + +``` +-Dsun.security.krb5.debug=true -Dsun.security.spnego.debug=true +``` + +All these options can be enabled in the Application Master: + +``` +spark.yarn.appMasterEnv.HADOOP_JAAS_DEBUG true +spark.yarn.am.extraJavaOptions -Dsun.security.krb5.debug=true -Dsun.security.spnego.debug=true +``` + +Finally, if the log level for `org.apache.spark.deploy.yarn.Client` is set to `DEBUG`, the log +will include a list of all tokens obtained, and their expiry details + +## Using the Spark History Server to replace the Spark Web UI + +It is possible to use the Spark History Server application page as the tracking URL for running +applications when the application UI is disabled. This may be desirable on secure clusters, or to +reduce the memory usage of the Spark driver. To set up tracking through the Spark History Server, +do the following: + +- On the application side, set spark.yarn.historyServer.allowTracking=true in Spark's + configuration. This will tell Spark to use the history server's URL as the tracking URL if + the application's UI is disabled. +- On the Spark History Server, add org.apache.spark.deploy.yarn.YarnProxyRedirectFilter + to the list of filters in the spark.ui.filters configuration. + +Be aware that the history server information may not be up-to-date with the application's state. diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md deleted file mode 100644 index 69ceb637dc8d..000000000000 --- a/docs/scala-programming-guide.md +++ /dev/null @@ -1,7 +0,0 @@ ---- -layout: global -title: Spark Programming Guide -redirect: programming-guide.html ---- - -This document has moved [here](programming-guide.html). diff --git a/docs/security.md b/docs/security.md index 177109415180..1d004003f9a3 100644 --- a/docs/security.md +++ b/docs/security.md @@ -6,16 +6,20 @@ title: Security Spark currently supports authentication via a shared secret. Authentication can be configured to be on via the `spark.authenticate` configuration parameter. This parameter controls whether the Spark communication protocols do authentication using the shared secret. This authentication is a basic handshake to make sure both sides have the same shared secret and are allowed to communicate. If the shared secret is not identical they will not be allowed to communicate. The shared secret is created as follows: -* For Spark on [YARN](running-on-yarn.html) deployments, configuring `spark.authenticate` to `true` will automatically handle generating and distributing the shared secret. Each application will use a unique shared secret. +* For Spark on [YARN](running-on-yarn.html) deployments, configuring `spark.authenticate` to `true` will automatically handle generating and distributing the shared secret. Each application will use a unique shared secret. * For other types of Spark deployments, the Spark parameter `spark.authenticate.secret` should be configured on each of the nodes. This secret will be used by all the Master/Workers and applications. ## Web UI -The Spark UI can also be secured by using [javax servlet filters](http://docs.oracle.com/javaee/6/api/javax/servlet/Filter.html) via the `spark.ui.filters` setting. A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view ACLs to make sure they are authorized to view the UI. The configs `spark.acls.enable` and `spark.ui.view.acls` control the behavior of the ACLs. Note that the user who started the application always has view access to the UI. On YARN, the Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. +The Spark UI can be secured by using [javax servlet filters](http://docs.oracle.com/javaee/6/api/javax/servlet/Filter.html) via the `spark.ui.filters` setting +and by using [https/SSL](http://en.wikipedia.org/wiki/HTTPS) via [SSL settings](security.html#ssl-configuration). -Spark also supports modify ACLs to control who has access to modify a running Spark application. This includes things like killing the application or a task. This is controlled by the configs `spark.acls.enable` and `spark.modify.acls`. Note that if you are authenticating the web UI, in order to use the kill button on the web UI it might be necessary to add the users in the modify acls to the view acls also. On YARN, the modify acls are passed in and control who has modify access via YARN interfaces. +### Authentication -Spark allows for a set of administrators to be specified in the acls who always have view and modify permissions to all the applications. is controlled by the config `spark.admin.acls`. This is useful on a shared cluster where you might have administrators or support staff who help users debug applications. +A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view ACLs to make sure they are authorized to view the UI. The configs `spark.acls.enable`, `spark.ui.view.acls` and `spark.ui.view.acls.groups` control the behavior of the ACLs. Note that the user who started the application always has view access to the UI. On YARN, the Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. + +Spark also supports modify ACLs to control who has access to modify a running Spark application. This includes things like killing the application or a task. This is controlled by the configs `spark.acls.enable`, `spark.modify.acls` and `spark.modify.acls.groups`. Note that if you are authenticating the web UI, in order to use the kill button on the web UI it might be necessary to add the users in the modify acls to the view acls also. On YARN, the modify acls are passed in and control who has modify access via YARN interfaces. +Spark allows for a set of administrators to be specified in the acls who always have view and modify permissions to all the applications. is controlled by the configs `spark.admin.acls` and `spark.admin.acls.groups`. This is useful on a shared cluster where you might have administrators or support staff who help users debug applications. ## Event Logging @@ -23,17 +27,42 @@ If your applications are using event logging, the directory where the event logs ## Encryption -Spark supports SSL for Akka and HTTP (for broadcast and file server) protocols. SASL encryption is -supported for the block transfer service. Encryption is not yet supported for the WebUI. - -Encryption is not yet supported for data stored by Spark in temporary local storage, such as shuffle -files, cached data, and other application files. If encrypting this data is desired, a workaround is -to configure your cluster manager to store application data on encrypted disks. +Spark supports SSL for HTTP protocols. SASL encryption is supported for the block transfer service +and the RPC endpoints. Shuffle files can also be encrypted if desired. ### SSL Configuration -Configuration for SSL is organized hierarchically. The user can configure the default SSL settings which will be used for all the supported communication protocols unless they are overwritten by protocol-specific settings. This way the user can easily provide the common settings for all the protocols without disabling the ability to configure each one individually. The common SSL settings are at `spark.ssl` namespace in Spark configuration, while Akka SSL configuration is at `spark.ssl.akka` and HTTP for broadcast and file server SSL configuration is at `spark.ssl.fs`. The full breakdown can be found on the [configuration page](configuration.html). +Configuration for SSL is organized hierarchically. The user can configure the default SSL settings +which will be used for all the supported communication protocols unless they are overwritten by +protocol-specific settings. This way the user can easily provide the common settings for all the +protocols without disabling the ability to configure each one individually. The common SSL settings +are at `spark.ssl` namespace in Spark configuration. The following table describes the +component-specific configuration namespaces used to override the default settings: + + + + + + + + + + + + + + + + + + + + + +
    Config NamespaceComponent
    spark.ssl.fsFile download client (used to download jars and files from HTTPS-enabled servers).
    spark.ssl.uiSpark application Web UI
    spark.ssl.standaloneStandalone Master / Worker Web UI
    spark.ssl.historyServerHistory Server Web UI
    + +The full breakdown of available SSL options can be found on the [configuration page](configuration.html). SSL must be configured on each node and configured for each component involved in communication using the particular protocol. ### YARN mode @@ -44,11 +73,15 @@ For long-running apps like Spark Streaming apps to be able to write to HDFS, it ### Standalone mode The user needs to provide key-stores and configuration options for master and workers. They have to be set by attaching appropriate Java system properties in `SPARK_MASTER_OPTS` and in `SPARK_WORKER_OPTS` environment variables, or just in `SPARK_DAEMON_JAVA_OPTS`. In this mode, the user may allow the executors to use the SSL settings inherited from the worker which spawned that executor. It can be accomplished by setting `spark.ssl.useNodeLocalConf` to `true`. If that parameter is set, the settings provided by user on the client side, are not used by the executors. +### Mesos mode +Mesos 1.3.0 and newer supports `Secrets` primitives as both file-based and environment based secrets. Spark allows the specification of file-based and environment variable based secrets with the `spark.mesos.driver.secret.filenames` and `spark.mesos.driver.secret.envkeys`, respectively. Depending on the secret store backend secrets can be passed by reference or by value with the `spark.mesos.driver.secret.names` and `spark.mesos.driver.secret.values` configuration properties, respectively. Reference type secrets are served by the secret store and referred to by name, for example `/mysecret`. Value type secrets are passed on the command line and translated into their appropriate files or environment variables. + ### Preparing the key-stores Key-stores can be generated by `keytool` program. The reference documentation for this tool is [here](https://docs.oracle.com/javase/7/docs/technotes/tools/solaris/keytool.html). The most basic steps to configure the key-stores and the trust-store for the standalone deployment mode is as follows: + * Generate a keys pair for each node * Export the public key of the key pair to a file on each node * Import all exported public keys into a single trust-store @@ -100,7 +133,7 @@ configure those ports. 7077 Submit job to cluster /
    Join cluster SPARK_MASTER_PORT - Akka-based. Set to "0" to choose a port randomly. Standalone mode only. + Set to "0" to choose a port randomly. Standalone mode only. Standalone Master @@ -108,7 +141,7 @@ configure those ports. (random) Schedule executors SPARK_WORKER_PORT - Akka-based. Set to "0" to choose a port randomly. Standalone mode only. + Set to "0" to choose a port randomly. Standalone mode only. @@ -141,40 +174,7 @@ configure those ports. (random) Connect to application /
    Notify executor state changes spark.driver.port - Akka-based. Set to "0" to choose a port randomly. - - - Driver - Executor - (random) - Schedule tasks - spark.executor.port - Akka-based. Set to "0" to choose a port randomly. - - - Executor - Driver - (random) - File server for files and jars - spark.fileserver.port - Jetty-based - - - Executor - Driver - (random) - HTTP Broadcast - spark.broadcast.port - Jetty-based. Not used by TorrentBroadcast, which sends data through the block manager - instead. - - - Executor - Driver - (random) - Class file server - spark.replClassServer.port - Jetty-based. Only used in Spark shells. + Set to "0" to choose a port randomly. Executor / Driver diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 2fe9ec3542b2..1095386c31ab 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -83,7 +83,7 @@ Once you've set up this file, you can launch or stop your cluster with the follo - `sbin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file. - `sbin/start-slave.sh` - Starts a slave instance on the machine the script is executed on. - `sbin/start-all.sh` - Starts both a master and a number of slaves as described above. -- `sbin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script. +- `sbin/stop-master.sh` - Stops the master that was started via the `sbin/start-master.sh` script. - `sbin/stop-slaves.sh` - Stops all slave instances on the machines specified in the `conf/slaves` file. - `sbin/stop-all.sh` - Stops both the master and the slaves as described above. @@ -94,8 +94,8 @@ You can optionally configure the cluster further by setting environment variable - - + + @@ -112,8 +112,8 @@ You can optionally configure the cluster further by setting environment variable @@ -133,15 +133,6 @@ You can optionally configure the cluster further by setting environment variable - - - - @@ -158,6 +149,10 @@ You can optionally configure the cluster further by setting environment variable + + + + @@ -204,6 +199,21 @@ SPARK_MASTER_OPTS supports the following system properties: the whole cluster by default.
    + + + + + @@ -236,7 +246,7 @@ SPARK_WORKER_OPTS supports the following system properties: - + + + + + +
    Environment VariableMeaning
    SPARK_MASTER_IPBind the master to a specific IP address, for example a public one.SPARK_MASTER_HOSTBind the master to a specific hostname or IP address, for example a public one.
    SPARK_MASTER_PORT
    SPARK_LOCAL_DIRS - Directory to use for "scratch" space in Spark, including map output files and RDDs that get - stored on disk. This should be on a fast, local disk in your system. It can also be a + Directory to use for "scratch" space in Spark, including map output files and RDDs that get + stored on disk. This should be on a fast, local disk in your system. It can also be a comma-separated list of multiple directories on different disks.
    SPARK_WORKER_WEBUI_PORT Port for the worker web UI (default: 8081).
    SPARK_WORKER_INSTANCES - Number of worker instances to run on each machine (default: 1). You can make this more than 1 if - you have have very large machines and would like multiple Spark worker processes. If you do set - this, make sure to also set SPARK_WORKER_CORES explicitly to limit the cores per worker, - or else each worker will try to use all the cores. -
    SPARK_WORKER_DIR Directory to run applications in, which will include both logs and scratch space (default: SPARK_HOME/work).SPARK_DAEMON_JAVA_OPTS JVM options for the Spark master and worker daemons themselves in the form "-Dx=y" (default: none).
    SPARK_DAEMON_CLASSPATHClasspath for the Spark master and worker daemons themselves (default: none).
    SPARK_PUBLIC_DNS The public DNS name of the Spark master and workers (default: none).
    spark.deploy.maxExecutorRetries10 + Limit on the maximum number of back-to-back executor failures that can occur before the + standalone cluster manager removes a faulty application. An application will never be removed + if it has any running executors. If an application experiences more than + spark.deploy.maxExecutorRetries failures in a row, no executors + successfully start running in between those failures, and the application has no running + executors then the standalone cluster manager will remove the application and mark it as failed. + To disable this automatic removal, set spark.deploy.maxExecutorRetries to + -1. +
    +
    spark.worker.timeout 60
    spark.worker.cleanup.appDataTtl7 * 24 * 3600 (7 days)604800 (7 days, 7 * 24 * 3600) The number of seconds to retain application work directories on each worker. This is a Time To Live and should depend on the amount of available disk space you have. Application logs and jars are @@ -244,12 +254,21 @@ SPARK_WORKER_OPTS supports the following system properties: especially if you run jobs very frequently.
    spark.worker.ui.compressedLogFileLengthCacheSize100 + For compressed log files, the uncompressed file can only be computed by uncompressing the files. + Spark caches the uncompressed file size of compressed log files. This property controls the cache + size. +
    # Connecting an Application to the Cluster To run an application on the Spark cluster, simply pass the `spark://IP:PORT` URL of the master as to the [`SparkContext` -constructor](programming-guide.html#initializing-spark). +constructor](rdd-programming-guide.html#initializing-spark). To run an interactive Spark shell against the cluster, run the following command: @@ -292,9 +311,9 @@ application at a time. You can cap the number of cores by setting `spark.cores.m {% highlight scala %} val conf = new SparkConf() - .setMaster(...) - .setAppName(...) - .set("spark.cores.max", "10") + .setMaster(...) + .setAppName(...) + .set("spark.cores.max", "10") val sc = new SparkContext(conf) {% endhighlight %} @@ -335,29 +354,14 @@ By default, standalone scheduling clusters are resilient to Worker failures (ins **Overview** -Utilizing ZooKeeper to provide leader election and some state storage, you can launch multiple Masters in your cluster connected to the same ZooKeeper instance. One will be elected "leader" and the others will remain in standby mode. If the current leader dies, another Master will be elected, recover the old Master's state, and then resume scheduling. The entire recovery process (from the time the the first leader goes down) should take between 1 and 2 minutes. Note that this delay only affects scheduling _new_ applications -- applications that were already running during Master failover are unaffected. +Utilizing ZooKeeper to provide leader election and some state storage, you can launch multiple Masters in your cluster connected to the same ZooKeeper instance. One will be elected "leader" and the others will remain in standby mode. If the current leader dies, another Master will be elected, recover the old Master's state, and then resume scheduling. The entire recovery process (from the time the first leader goes down) should take between 1 and 2 minutes. Note that this delay only affects scheduling _new_ applications -- applications that were already running during Master failover are unaffected. Learn more about getting started with ZooKeeper [here](http://zookeeper.apache.org/doc/trunk/zookeeperStarted.html). **Configuration** -In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env using this configuration: - - - - - - - - - - - - - - - -
    System propertyMeaning
    spark.deploy.recoveryModeSet to ZOOKEEPER to enable standby Master recovery mode (default: NONE).
    spark.deploy.zookeeper.urlThe ZooKeeper cluster url (e.g., 192.168.1.100:2181,192.168.1.101:2181).
    spark.deploy.zookeeper.dirThe directory in ZooKeeper to store recovery state (default: /spark).
    +In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env by configuring `spark.deploy.recoveryMode` and related spark.deploy.zookeeper.* configurations. +For more information about these configurations please refer to the [configuration doc](configuration.html#deploy) Possible gotcha: If you have multiple Masters in your cluster but fail to correctly configure the Masters to use ZooKeeper, the Masters will fail to discover each other and think they're all leaders. This will not lead to a healthy cluster state (as all Masters will schedule independently). diff --git a/docs/sparkr.md b/docs/sparkr.md index 437bd4756c27..a3254e765413 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -14,29 +14,24 @@ supports operations like selection, filtering, aggregation etc. (similar to R da [dplyr](https://github.com/hadley/dplyr)) but on large datasets. SparkR also supports distributed machine learning using MLlib. -# SparkR DataFrames +# SparkDataFrame -A DataFrame is a distributed collection of data organized into named columns. It is conceptually +A SparkDataFrame is a distributed collection of data organized into named columns. It is conceptually equivalent to a table in a relational database or a data frame in R, but with richer -optimizations under the hood. DataFrames can be constructed from a wide array of sources such as: +optimizations under the hood. SparkDataFrames can be constructed from a wide array of sources such as: structured data files, tables in Hive, external databases, or existing local R data frames. All of the examples on this page use sample data included in R or the Spark distribution and can be run using the `./bin/sparkR` shell. -## Starting Up: SparkContext, SQLContext +## Starting Up: SparkSession
    -The entry point into SparkR is the `SparkContext` which connects your R program to a Spark cluster. -You can create a `SparkContext` using `sparkR.init` and pass in options such as the application name -, any spark packages depended on, etc. Further, to work with DataFrames we will need a `SQLContext`, -which can be created from the SparkContext. If you are working from the `sparkR` shell, the -`SQLContext` and `SparkContext` should already be created for you, and you would not need to call -`sparkR.init`. +The entry point into SparkR is the `SparkSession` which connects your R program to a Spark cluster. +You can create a `SparkSession` using `sparkR.session` and pass in options such as the application name, any spark packages depended on, etc. Further, you can also work with SparkDataFrames via `SparkSession`. If you are working from the `sparkR` shell, the `SparkSession` should already be created for you, and you would not need to call `sparkR.session`.
    {% highlight r %} -sc <- sparkR.init() -sqlContext <- sparkRSQL.init(sc) +sparkR.session() {% endhighlight %}
    @@ -45,13 +40,15 @@ sqlContext <- sparkRSQL.init(sc) You can also start SparkR from RStudio. You can connect your R program to a Spark cluster from RStudio, R shell, Rscript or other R IDEs. To start, make sure SPARK_HOME is set in environment (you can check [Sys.getenv](https://stat.ethz.ch/R-manual/R-devel/library/base/html/Sys.getenv.html)), -load the SparkR package, and call `sparkR.init` as below. In addition to calling `sparkR.init`, you -could also specify certain Spark driver properties. Normally these +load the SparkR package, and call `sparkR.session` as below. It will check for the Spark installation, and, if not found, it will be downloaded and cached automatically. Alternatively, you can also run `install.spark` manually. + +In addition to calling `sparkR.session`, + you could also specify certain Spark driver properties. Normally these [Application properties](configuration.html#application-properties) and [Runtime Environment](configuration.html#runtime-environment) cannot be set programmatically, as the driver JVM process would have been started, in this case SparkR takes care of this for you. To set -them, pass them as you would other configuration properties in the `sparkEnvir` argument to -`sparkR.init()`. +them, pass them as you would other configuration properties in the `sparkConfig` argument to +`sparkR.session()`.
    {% highlight r %} @@ -59,14 +56,29 @@ if (nchar(Sys.getenv("SPARK_HOME")) < 1) { Sys.setenv(SPARK_HOME = "/home/spark") } library(SparkR, lib.loc = c(file.path(Sys.getenv("SPARK_HOME"), "R", "lib"))) -sc <- sparkR.init(master = "local[*]", sparkEnvir = list(spark.driver.memory="2g")) +sparkR.session(master = "local[*]", sparkConfig = list(spark.driver.memory = "2g")) {% endhighlight %}
    -The following options can be set in `sparkEnvir` with `sparkR.init` from RStudio: +The following Spark driver properties can be set in `sparkConfig` with `sparkR.session` from RStudio: + + + + + + + + + + + + + + + @@ -91,17 +103,17 @@ The following options can be set in `sparkEnvir` with `sparkR.init` from RStudio -## Creating DataFrames -With a `SQLContext`, applications can create `DataFrame`s from a local R data frame, from a [Hive table](sql-programming-guide.html#hive-tables), or from other [data sources](sql-programming-guide.html#data-sources). +## Creating SparkDataFrames +With a `SparkSession`, applications can create `SparkDataFrame`s from a local R data frame, from a [Hive table](sql-programming-guide.html#hive-tables), or from other [data sources](sql-programming-guide.html#data-sources). ### From local data frames -The simplest way to create a data frame is to convert a local R data frame into a SparkR DataFrame. Specifically we can use `createDataFrame` and pass in the local R data frame to create a SparkR DataFrame. As an example, the following creates a `DataFrame` based using the `faithful` dataset from R. +The simplest way to create a data frame is to convert a local R data frame into a SparkDataFrame. Specifically we can use `as.DataFrame` or `createDataFrame` and pass in the local R data frame to create a SparkDataFrame. As an example, the following creates a `SparkDataFrame` based using the `faithful` dataset from R.
    {% highlight r %} -df <- createDataFrame(sqlContext, faithful) +df <- as.DataFrame(faithful) -# Displays the content of the DataFrame to stdout +# Displays the first part of the SparkDataFrame head(df) ## eruptions waiting ##1 3.600 79 @@ -113,25 +125,23 @@ head(df) ### From Data Sources -SparkR supports operating on a variety of data sources through the `DataFrame` interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more [specific options](sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. +SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more [specific options](sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. -The general method for creating DataFrames from data sources is `read.df`. This method takes in the `SQLContext`, the path for the file to load and the type of data source. SparkR supports reading JSON and Parquet files natively and through [Spark Packages](http://spark-packages.org/) you can find data source connectors for popular file formats like [CSV](http://spark-packages.org/package/databricks/spark-csv) and [Avro](http://spark-packages.org/package/databricks/spark-avro). These packages can either be added by -specifying `--packages` with `spark-submit` or `sparkR` commands, or if creating context through `init` -you can specify the packages with the `packages` argument. +The general method for creating SparkDataFrames from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active SparkSession will be used automatically. +SparkR supports reading JSON, CSV and Parquet files natively, and through packages available from sources like [Third Party Projects](http://spark.apache.org/third-party-projects.html), you can find data source connectors for popular file formats like Avro. These packages can either be added by +specifying `--packages` with `spark-submit` or `sparkR` commands, or if initializing SparkSession with `sparkPackages` parameter when in an interactive R shell or from RStudio.
    {% highlight r %} -sc <- sparkR.init(sparkPackages="com.databricks:spark-csv_2.11:1.0.3") -sqlContext <- sparkRSQL.init(sc) +sparkR.session(sparkPackages = "com.databricks:spark-avro_2.11:3.0.0") {% endhighlight %}
    -We can see how to use data sources using an example JSON input file. Note that the file that is used here is _not_ a typical JSON file. Each line in the file must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail. +We can see how to use data sources using an example JSON input file. Note that the file that is used here is _not_ a typical JSON file. Each line in the file must contain a separate, self-contained valid JSON object. For more information, please see [JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). As a consequence, a regular multi-line JSON file will most often fail.
    - {% highlight r %} -people <- read.df(sqlContext, "./examples/src/main/resources/people.json", "json") +people <- read.df("./examples/src/main/resources/people.json", "json") head(people) ## age name ##1 NA Michael @@ -141,37 +151,48 @@ head(people) # SparkR automatically infers the schema from the JSON file printSchema(people) # root -# |-- age: integer (nullable = true) +# |-- age: long (nullable = true) # |-- name: string (nullable = true) +# Similarly, multiple files can be read with read.json +people <- read.json(c("./examples/src/main/resources/people.json", "./examples/src/main/resources/people2.json")) + +{% endhighlight %} +
    + +The data sources API natively supports CSV formatted input files. For more information please refer to SparkR [read.df](api/R/read.df.html) API documentation. + +
    +{% highlight r %} +df <- read.df(csvPath, "csv", header = "true", inferSchema = "true", na.strings = "NA") + {% endhighlight %}
    -The data sources API can also be used to save out DataFrames into multiple file formats. For example we can save the DataFrame from the previous example -to a Parquet file using `write.df` +The data sources API can also be used to save out SparkDataFrames into multiple file formats. For example we can save the SparkDataFrame from the previous example +to a Parquet file using `write.df`.
    {% highlight r %} -write.df(people, path="people.parquet", source="parquet", mode="overwrite") +write.df(people, path = "people.parquet", source = "parquet", mode = "overwrite") {% endhighlight %}
    ### From Hive tables -You can also create SparkR DataFrames from Hive tables. To do this we will need to create a HiveContext which can access tables in the Hive MetaStore. Note that Spark should have been built with [Hive support](building-spark.html#building-with-hive-and-jdbc-support) and more details on the difference between SQLContext and HiveContext can be found in the [SQL programming guide](sql-programming-guide.html#starting-point-sqlcontext). +You can also create SparkDataFrames from Hive tables. To do this we will need to create a SparkSession with Hive support which can access tables in the Hive MetaStore. Note that Spark should have been built with [Hive support](building-spark.html#building-with-hive-and-jdbc-support) and more details can be found in the [SQL programming guide](sql-programming-guide.html#starting-point-sparksession). In SparkR, by default it will attempt to create a SparkSession with Hive support enabled (`enableHiveSupport = TRUE`).
    {% highlight r %} -# sc is an existing SparkContext. -hiveContext <- sparkRHive.init(sc) +sparkR.session() -sql(hiveContext, "CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -sql(hiveContext, "LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") # Queries can be expressed in HiveQL. -results <- sql(hiveContext, "FROM src SELECT key, value") +results <- sql("FROM src SELECT key, value") -# results is now a DataFrame +# results is now a SparkDataFrame head(results) ## key value ## 1 238 val_238 @@ -181,21 +202,21 @@ head(results) {% endhighlight %}
    -## DataFrame Operations +## SparkDataFrame Operations -SparkR DataFrames support a number of functions to do structured data processing. +SparkDataFrames support a number of functions to do structured data processing. Here we include some basic examples and a complete list can be found in the [API](api/R/index.html) docs: ### Selecting rows, columns
    {% highlight r %} -# Create the DataFrame -df <- createDataFrame(sqlContext, faithful) +# Create the SparkDataFrame +df <- as.DataFrame(faithful) -# Get basic information about the DataFrame +# Get basic information about the SparkDataFrame df -## DataFrame[eruptions:double, waiting:double] +## SparkDataFrame[eruptions:double, waiting:double] # Select only the "eruptions" column head(select(df, df$eruptions)) @@ -207,7 +228,7 @@ head(select(df, df$eruptions)) # You can also pass in column name as strings head(select(df, "eruptions")) -# Filter the DataFrame to only retain rows with wait times shorter than 50 mins +# Filter the SparkDataFrame to only retain rows with wait times shorter than 50 mins head(filter(df, df$waiting < 50)) ## eruptions waiting ##1 1.750 47 @@ -228,14 +249,13 @@ SparkR data frames support a number of commonly used functions to aggregate data # We use the `n` operator to count the number of times each waiting time appears head(summarize(groupBy(df, df$waiting), count = n(df$waiting))) ## waiting count -##1 81 13 -##2 60 6 -##3 68 1 +##1 70 4 +##2 67 1 +##3 69 2 # We can also sort the output from the aggregation to get the most common waiting times waiting_counts <- summarize(groupBy(df, df$waiting), count = n(df$waiting)) head(arrange(waiting_counts, desc(waiting_counts$count))) - ## waiting count ##1 78 15 ##2 83 14 @@ -244,6 +264,36 @@ head(arrange(waiting_counts, desc(waiting_counts$count))) {% endhighlight %}
    +In addition to standard aggregations, SparkR supports [OLAP cube](https://en.wikipedia.org/wiki/OLAP_cube) operators `cube`: + +
    +{% highlight r %} +head(agg(cube(df, "cyl", "disp", "gear"), avg(df$mpg))) +## cyl disp gear avg(mpg) +##1 NA 140.8 4 22.8 +##2 4 75.7 4 30.4 +##3 8 400.0 3 19.2 +##4 8 318.0 3 15.5 +##5 NA 351.0 NA 15.8 +##6 NA 275.8 NA 16.3 +{% endhighlight %} +
    + +and `rollup`: + +
    +{% highlight r %} +head(agg(rollup(df, "cyl", "disp", "gear"), avg(df$mpg))) +## cyl disp gear avg(mpg) +##1 4 75.7 4 30.4 +##2 8 400.0 3 19.2 +##3 8 318.0 3 15.5 +##4 4 78.7 NA 32.4 +##5 8 304.0 3 15.2 +##6 4 79.0 NA 27.3 +{% endhighlight %} +
    + ### Operating on Columns SparkR also provides a number of functions that can directly applied to columns for data processing and during aggregation. The example below shows the use of basic arithmetic functions. @@ -252,7 +302,7 @@ SparkR also provides a number of functions that can directly applied to columns {% highlight r %} # Convert waiting time from hours to seconds. -# Note that we can assign this to a new column in the same DataFrame +# Note that we can assign this to a new column in the same SparkDataFrame df$waiting_secs <- df$waiting * 60 head(df) ## eruptions waiting waiting_secs @@ -263,20 +313,157 @@ head(df) {% endhighlight %}
    +### Applying User-Defined Function +In SparkR, we support several kinds of User-Defined Functions: + +#### Run a given function on a large dataset using `dapply` or `dapplyCollect` + +##### dapply +Apply a function to each partition of a `SparkDataFrame`. The function to be applied to each partition of the `SparkDataFrame` +and should have only one parameter, to which a `data.frame` corresponds to each partition will be passed. The output of function should be a `data.frame`. Schema specifies the row format of the resulting a `SparkDataFrame`. It must match to [data types](#data-type-mapping-between-r-and-spark) of returned value. + +
    +{% highlight r %} + +# Convert waiting time from hours to seconds. +# Note that we can apply UDF to DataFrame. +schema <- structType(structField("eruptions", "double"), structField("waiting", "double"), + structField("waiting_secs", "double")) +df1 <- dapply(df, function(x) { x <- cbind(x, x$waiting * 60) }, schema) +head(collect(df1)) +## eruptions waiting waiting_secs +##1 3.600 79 4740 +##2 1.800 54 3240 +##3 3.333 74 4440 +##4 2.283 62 3720 +##5 4.533 85 5100 +##6 2.883 55 3300 +{% endhighlight %} +
    + +##### dapplyCollect +Like `dapply`, apply a function to each partition of a `SparkDataFrame` and collect the result back. The output of function +should be a `data.frame`. But, Schema is not required to be passed. Note that `dapplyCollect` can fail if the output of UDF run on all the partition cannot be pulled to the driver and fit in driver memory. + +
    +{% highlight r %} + +# Convert waiting time from hours to seconds. +# Note that we can apply UDF to DataFrame and return a R's data.frame +ldf <- dapplyCollect( + df, + function(x) { + x <- cbind(x, "waiting_secs" = x$waiting * 60) + }) +head(ldf, 3) +## eruptions waiting waiting_secs +##1 3.600 79 4740 +##2 1.800 54 3240 +##3 3.333 74 4440 + +{% endhighlight %} +
    + +#### Run a given function on a large dataset grouping by input column(s) and using `gapply` or `gapplyCollect` + +##### gapply +Apply a function to each group of a `SparkDataFrame`. The function is to be applied to each group of the `SparkDataFrame` and should have only two parameters: grouping key and R `data.frame` corresponding to +that key. The groups are chosen from `SparkDataFrame`s column(s). +The output of function should be a `data.frame`. Schema specifies the row format of the resulting +`SparkDataFrame`. It must represent R function's output schema on the basis of Spark [data types](#data-type-mapping-between-r-and-spark). The column names of the returned `data.frame` are set by user. + +
    +{% highlight r %} + +# Determine six waiting times with the largest eruption time in minutes. +schema <- structType(structField("waiting", "double"), structField("max_eruption", "double")) +result <- gapply( + df, + "waiting", + function(key, x) { + y <- data.frame(key, max(x$eruptions)) + }, + schema) +head(collect(arrange(result, "max_eruption", decreasing = TRUE))) + +## waiting max_eruption +##1 64 5.100 +##2 69 5.067 +##3 71 5.033 +##4 87 5.000 +##5 63 4.933 +##6 89 4.900 +{% endhighlight %} +
    + +##### gapplyCollect +Like `gapply`, applies a function to each partition of a `SparkDataFrame` and collect the result back to R data.frame. The output of the function should be a `data.frame`. But, the schema is not required to be passed. Note that `gapplyCollect` can fail if the output of UDF run on all the partition cannot be pulled to the driver and fit in driver memory. + +
    +{% highlight r %} + +# Determine six waiting times with the largest eruption time in minutes. +result <- gapplyCollect( + df, + "waiting", + function(key, x) { + y <- data.frame(key, max(x$eruptions)) + colnames(y) <- c("waiting", "max_eruption") + y + }) +head(result[order(result$max_eruption, decreasing = TRUE), ]) + +## waiting max_eruption +##1 64 5.100 +##2 69 5.067 +##3 71 5.033 +##4 87 5.000 +##5 63 4.933 +##6 89 4.900 + +{% endhighlight %} +
    + +#### Run local R functions distributed using `spark.lapply` + +##### spark.lapply +Similar to `lapply` in native R, `spark.lapply` runs a function over a list of elements and distributes the computations with Spark. +Applies a function in a manner that is similar to `doParallel` or `lapply` to elements of a list. The results of all the computations +should fit in a single machine. If that is not the case they can do something like `df <- createDataFrame(list)` and then use +`dapply` + +
    +{% highlight r %} +# Perform distributed training of multiple models with spark.lapply. Here, we pass +# a read-only list of arguments which specifies family the generalized linear model should be. +families <- c("gaussian", "poisson") +train <- function(family) { + model <- glm(Sepal.Length ~ Sepal.Width + Species, iris, family = family) + summary(model) +} +# Return a list of model's summaries +model.summaries <- spark.lapply(families, train) + +# Print the summary of each model +print(model.summaries) + +{% endhighlight %} +
    + ## Running SQL Queries from SparkR -A SparkR DataFrame can also be registered as a temporary table in Spark SQL and registering a DataFrame as a table allows you to run SQL queries over its data. -The `sql` function enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. +A SparkDataFrame can also be registered as a temporary view in Spark SQL and that allows you to run SQL queries over its data. +The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`.
    {% highlight r %} # Load a JSON file -people <- read.df(sqlContext, "./examples/src/main/resources/people.json", "json") +people <- read.df("./examples/src/main/resources/people.json", "json") -# Register this DataFrame as a table. -registerTempTable(people, "people") +# Register this SparkDataFrame as a temporary view. +createOrReplaceTempView(people, "people") # SQL statements can be run by using the sql method -teenagers <- sql(sqlContext, "SELECT name FROM people WHERE age >= 13 AND age <= 19") +teenagers <- sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") head(teenagers) ## name ##1 Justin @@ -286,34 +473,187 @@ head(teenagers) # Machine Learning -SparkR allows the fitting of generalized linear models over DataFrames using the [glm()](api/R/glm.html) function. Under the hood, SparkR uses MLlib to train a model of the specified family. Currently the gaussian and binomial families are supported. We support a subset of the available R formula operators for model fitting, including '~', '.', '+', and '-'. The example below shows the use of building a gaussian GLM model using SparkR. +## Algorithms -
    -{% highlight r %} -# Create the DataFrame -df <- createDataFrame(sqlContext, iris) - -# Fit a linear model over the dataset. -model <- glm(Sepal_Length ~ Sepal_Width + Species, data = df, family = "gaussian") - -# Model coefficients are returned in a similar format to R's native glm(). -summary(model) -##$coefficients -## Estimate -##(Intercept) 2.2513930 -##Sepal_Width 0.8035609 -##Species_versicolor 1.4587432 -##Species_virginica 1.9468169 - -# Make predictions based on the model. -predictions <- predict(model, newData = df) -head(select(predictions, "Sepal_Length", "prediction")) -## Sepal_Length prediction -##1 5.1 5.063856 -##2 4.9 4.662076 -##3 4.7 4.822788 -##4 4.6 4.742432 -##5 5.0 5.144212 -##6 5.4 5.385281 -{% endhighlight %} -
    +SparkR supports the following machine learning algorithms currently: + +#### Classification + +* [`spark.logit`](api/R/spark.logit.html): [`Logistic Regression`](ml-classification-regression.html#logistic-regression) +* [`spark.mlp`](api/R/spark.mlp.html): [`Multilayer Perceptron (MLP)`](ml-classification-regression.html#multilayer-perceptron-classifier) +* [`spark.naiveBayes`](api/R/spark.naiveBayes.html): [`Naive Bayes`](ml-classification-regression.html#naive-bayes) +* [`spark.svmLinear`](api/R/spark.svmLinear.html): [`Linear Support Vector Machine`](ml-classification-regression.html#linear-support-vector-machine) + +#### Regression + +* [`spark.survreg`](api/R/spark.survreg.html): [`Accelerated Failure Time (AFT) Survival Model`](ml-classification-regression.html#survival-regression) +* [`spark.glm`](api/R/spark.glm.html) or [`glm`](api/R/glm.html): [`Generalized Linear Model (GLM)`](ml-classification-regression.html#generalized-linear-regression) +* [`spark.isoreg`](api/R/spark.isoreg.html): [`Isotonic Regression`](ml-classification-regression.html#isotonic-regression) + +#### Tree + +* [`spark.decisionTree`](api/R/spark.decisionTree.html): `Decision Tree for` [`Regression`](ml-classification-regression.html#decision-tree-regression) `and` [`Classification`](ml-classification-regression.html#decision-tree-classifier) +* [`spark.gbt`](api/R/spark.gbt.html): `Gradient Boosted Trees for` [`Regression`](ml-classification-regression.html#gradient-boosted-tree-regression) `and` [`Classification`](ml-classification-regression.html#gradient-boosted-tree-classifier) +* [`spark.randomForest`](api/R/spark.randomForest.html): `Random Forest for` [`Regression`](ml-classification-regression.html#random-forest-regression) `and` [`Classification`](ml-classification-regression.html#random-forest-classifier) + +#### Clustering + +* [`spark.bisectingKmeans`](api/R/spark.bisectingKmeans.html): [`Bisecting k-means`](ml-clustering.html#bisecting-k-means) +* [`spark.gaussianMixture`](api/R/spark.gaussianMixture.html): [`Gaussian Mixture Model (GMM)`](ml-clustering.html#gaussian-mixture-model-gmm) +* [`spark.kmeans`](api/R/spark.kmeans.html): [`K-Means`](ml-clustering.html#k-means) +* [`spark.lda`](api/R/spark.lda.html): [`Latent Dirichlet Allocation (LDA)`](ml-clustering.html#latent-dirichlet-allocation-lda) + +#### Collaborative Filtering + +* [`spark.als`](api/R/spark.als.html): [`Alternating Least Squares (ALS)`](ml-collaborative-filtering.html#collaborative-filtering) + +#### Frequent Pattern Mining + +* [`spark.fpGrowth`](api/R/spark.fpGrowth.html) : [`FP-growth`](ml-frequent-pattern-mining.html#fp-growth) + +#### Statistics + +* [`spark.kstest`](api/R/spark.kstest.html): `Kolmogorov-Smirnov Test` + +Under the hood, SparkR uses MLlib to train the model. Please refer to the corresponding section of MLlib user guide for example code. +Users can call `summary` to print a summary of the fitted model, [predict](api/R/predict.html) to make predictions on new data, and [write.ml](api/R/write.ml.html)/[read.ml](api/R/read.ml.html) to save/load fitted models. +SparkR supports a subset of the available R formula operators for model fitting, including ‘~’, ‘.’, ‘:’, ‘+’, and ‘-‘. + + +## Model persistence + +The following example shows how to save/load a MLlib model by SparkR. +{% include_example read_write r/ml/ml.R %} + +# Data type mapping between R and Spark +
    Property NameProperty groupspark-submit equivalent
    spark.masterApplication Properties--master
    spark.yarn.keytabApplication Properties--keytab
    spark.yarn.principalApplication Properties--principal
    spark.driver.memory Application Properties
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    RSpark
    bytebyte
    integerinteger
    floatfloat
    doubledouble
    numericdouble
    characterstring
    stringstring
    binarybinary
    rawbinary
    logicalboolean
    POSIXcttimestamp
    POSIXlttimestamp
    Datedate
    arrayarray
    listarray
    envmap
    + +# Structured Streaming + +SparkR supports the Structured Streaming API (experimental). Structured Streaming is a scalable and fault-tolerant stream processing engine built on the Spark SQL engine. For more information see the R API on the [Structured Streaming Programming Guide](structured-streaming-programming-guide.html) + +# R Function Name Conflicts + +When loading and attaching a new package in R, it is possible to have a name [conflict](https://stat.ethz.ch/R-manual/R-devel/library/base/html/library.html), where a +function is masking another function. + +The following functions are masked by the SparkR package: + + + + + + + + + + + + + + + +
    Masked functionHow to Access
    cov in package:stats
    stats::cov(x, y = NULL, use = "everything",
    +           method = c("pearson", "kendall", "spearman"))
    filter in package:stats
    stats::filter(x, filter, method = c("convolution", "recursive"),
    +              sides = 2, circular = FALSE, init)
    sample in package:basebase::sample(x, size, replace = FALSE, prob = NULL)
    + +Since part of SparkR is modeled on the `dplyr` package, certain functions in SparkR share the same names with those in `dplyr`. Depending on the load order of the two packages, some functions from the package loaded first are masked by those in the package loaded after. In such case, prefix such calls with the package name, for instance, `SparkR::cume_dist(x)` or `dplyr::cume_dist(x)`. + +You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-manual/R-devel/library/base/html/search.html) + + +# Migration Guide + +## Upgrading From SparkR 1.5.x to 1.6.x + + - Before Spark 1.6.0, the default mode for writes was `append`. It was changed in Spark 1.6.0 to `error` to match the Scala API. + - SparkSQL converts `NA` in R to `null` and vice-versa. + +## Upgrading From SparkR 1.6.x to 2.0 + + - The method `table` has been removed and replaced by `tableToDF`. + - The class `DataFrame` has been renamed to `SparkDataFrame` to avoid name conflicts. + - Spark's `SQLContext` and `HiveContext` have been deprecated to be replaced by `SparkSession`. Instead of `sparkR.init()`, call `sparkR.session()` in its place to instantiate the SparkSession. Once that is done, that currently active SparkSession will be used for SparkDataFrame operations. + - The parameter `sparkExecutorEnv` is not supported by `sparkR.session`. To set environment for the executors, set Spark config properties with the prefix "spark.executorEnv.VAR_NAME", for example, "spark.executorEnv.PATH" + - The `sqlContext` parameter is no longer required for these functions: `createDataFrame`, `as.DataFrame`, `read.json`, `jsonFile`, `read.parquet`, `parquetFile`, `read.text`, `sql`, `tables`, `tableNames`, `cacheTable`, `uncacheTable`, `clearCache`, `dropTempTable`, `read.df`, `loadDF`, `createExternalTable`. + - The method `registerTempTable` has been deprecated to be replaced by `createOrReplaceTempView`. + - The method `dropTempTable` has been deprecated to be replaced by `dropTempView`. + - The `sc` SparkContext parameter is no longer required for these functions: `setJobGroup`, `clearJobGroup`, `cancelJobGroup` + +## Upgrading to SparkR 2.1.0 + + - `join` no longer performs Cartesian Product by default, use `crossJoin` instead. + +## Upgrading to SparkR 2.2.0 + + - A `numPartitions` parameter has been added to `createDataFrame` and `as.DataFrame`. When splitting the data, the partition position calculation has been made to match the one in Scala. + - The method `createExternalTable` has been deprecated to be replaced by `createTable`. Either methods can be called to create external or managed table. Additional catalog methods have also been added. + - By default, derby.log is now saved to `tempdir()`. This will be created when instantiating the SparkSession with `enableHiveSupport` set to `TRUE`. + - `spark.lda` was not setting the optimizer correctly. It has been corrected. + - Several model summary outputs are updated to have `coefficients` as `matrix`. This includes `spark.logit`, `spark.kmeans`, `spark.glm`. Model summary outputs for `spark.gaussianMixture` have added log-likelihood as `loglik`. diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 2fe5c3633889..a095263bfa61 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1,6 +1,6 @@ --- layout: global -displayTitle: Spark SQL and DataFrame Guide +displayTitle: Spark SQL, DataFrames and Datasets Guide title: Spark SQL and DataFrames --- @@ -9,269 +9,159 @@ title: Spark SQL and DataFrames # Overview -Spark SQL is a Spark module for structured data processing. It provides a programming abstraction called DataFrames and can also act as distributed SQL query engine. - -Spark SQL can also be used to read data from an existing Hive installation. For more on how to configure this feature, please refer to the [Hive Tables](#hive-tables) section. - -# DataFrames - -A DataFrame is a distributed collection of data organized into named columns. It is conceptually equivalent to a table in a relational database or a data frame in R/Python, but with richer optimizations under the hood. DataFrames can be constructed from a wide array of sources such as: structured data files, tables in Hive, external databases, or existing RDDs. - -The DataFrame API is available in [Scala](api/scala/index.html#org.apache.spark.sql.DataFrame), [Java](api/java/index.html?org/apache/spark/sql/DataFrame.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame), and [R](api/R/index.html). - -All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell`, `pyspark` shell, or `sparkR` shell. - - -## Starting Point: SQLContext +Spark SQL is a Spark module for structured data processing. Unlike the basic Spark RDD API, the interfaces provided +by Spark SQL provide Spark with more information about the structure of both the data and the computation being performed. Internally, +Spark SQL uses this extra information to perform extra optimizations. There are several ways to +interact with Spark SQL including SQL and the Dataset API. When computing a result +the same execution engine is used, independent of which API/language you are using to express the +computation. This unification means that developers can easily switch back and forth between +different APIs based on which provides the most natural way to express a given transformation. + +All of the examples on this page use sample data included in the Spark distribution and can be run in +the `spark-shell`, `pyspark` shell, or `sparkR` shell. + +## SQL + +One use of Spark SQL is to execute SQL queries. +Spark SQL can also be used to read data from an existing Hive installation. For more on how to +configure this feature, please refer to the [Hive Tables](#hive-tables) section. When running +SQL from within another programming language the results will be returned as a [Dataset/DataFrame](#datasets-and-dataframes). +You can also interact with the SQL interface using the [command-line](#running-the-spark-sql-cli) +or over [JDBC/ODBC](#running-the-thrift-jdbcodbc-server). + +## Datasets and DataFrames + +A Dataset is a distributed collection of data. +Dataset is a new interface added in Spark 1.6 that provides the benefits of RDDs (strong +typing, ability to use powerful lambda functions) with the benefits of Spark SQL's optimized +execution engine. A Dataset can be [constructed](#creating-datasets) from JVM objects and then +manipulated using functional transformations (`map`, `flatMap`, `filter`, etc.). +The Dataset API is available in [Scala][scala-datasets] and +[Java][java-datasets]. Python does not have the support for the Dataset API. But due to Python's dynamic nature, +many of the benefits of the Dataset API are already available (i.e. you can access the field of a row by name naturally +`row.columnName`). The case for R is similar. + +A DataFrame is a *Dataset* organized into named columns. It is conceptually +equivalent to a table in a relational database or a data frame in R/Python, but with richer +optimizations under the hood. DataFrames can be constructed from a wide array of [sources](#data-sources) such +as: structured data files, tables in Hive, external databases, or existing RDDs. +The DataFrame API is available in Scala, +Java, [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame), and [R](api/R/index.html). +In Scala and Java, a DataFrame is represented by a Dataset of `Row`s. +In [the Scala API][scala-datasets], `DataFrame` is simply a type alias of `Dataset[Row]`. +While, in [Java API][java-datasets], users need to use `Dataset` to represent a `DataFrame`. + +[scala-datasets]: api/scala/index.html#org.apache.spark.sql.Dataset +[java-datasets]: api/java/index.html?org/apache/spark/sql/Dataset.html + +Throughout this document, we will often refer to Scala/Java Datasets of `Row`s as DataFrames. + +# Getting Started + +## Starting Point: SparkSession
    -The entry point into all functionality in Spark SQL is the -[`SQLContext`](api/scala/index.html#org.apache.spark.sql.SQLContext) class, or one of its -descendants. To create a basic `SQLContext`, all you need is a SparkContext. - -{% highlight scala %} -val sc: SparkContext // An existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) - -// this is used to implicitly convert an RDD to a DataFrame. -import sqlContext.implicits._ -{% endhighlight %} +The entry point into all functionality in Spark is the [`SparkSession`](api/scala/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: +{% include_example init_session scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    -The entry point into all functionality in Spark SQL is the -[`SQLContext`](api/java/index.html#org.apache.spark.sql.SQLContext) class, or one of its -descendants. To create a basic `SQLContext`, all you need is a SparkContext. - -{% highlight java %} -JavaSparkContext sc = ...; // An existing JavaSparkContext. -SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); -{% endhighlight %} +The entry point into all functionality in Spark is the [`SparkSession`](api/java/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: +{% include_example init_session java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    -The entry point into all relational functionality in Spark is the -[`SQLContext`](api/python/pyspark.sql.html#pyspark.sql.SQLContext) class, or one -of its decedents. To create a basic `SQLContext`, all you need is a SparkContext. - -{% highlight python %} -from pyspark.sql import SQLContext -sqlContext = SQLContext(sc) -{% endhighlight %} +The entry point into all functionality in Spark is the [`SparkSession`](api/python/pyspark.sql.html#pyspark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder`: +{% include_example init_session python/sql/basic.py %}
    -The entry point into all relational functionality in Spark is the -`SQLContext` class, or one of its decedents. To create a basic `SQLContext`, all you need is a SparkContext. +The entry point into all functionality in Spark is the [`SparkSession`](api/R/sparkR.session.html) class. To initialize a basic `SparkSession`, just call `sparkR.session()`: -{% highlight r %} -sqlContext <- sparkRSQL.init(sc) -{% endhighlight %} +{% include_example init_session r/RSparkSQLExample.R %} +Note that when invoked for the first time, `sparkR.session()` initializes a global `SparkSession` singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the `SparkSession` once, then SparkR functions like `read.df` will be able to access this global instance implicitly, and users don't need to pass the `SparkSession` instance around.
    -In addition to the basic `SQLContext`, you can also create a `HiveContext`, which provides a -superset of the functionality provided by the basic `SQLContext`. Additional features include -the ability to write queries using the more complete HiveQL parser, access to Hive UDFs, and the -ability to read data from Hive tables. To use a `HiveContext`, you do not need to have an -existing Hive setup, and all of the data sources available to a `SQLContext` are still available. -`HiveContext` is only packaged separately to avoid including all of Hive's dependencies in the default -Spark build. If these dependencies are not a problem for your application then using `HiveContext` -is recommended for the 1.3 release of Spark. Future releases will focus on bringing `SQLContext` up -to feature parity with a `HiveContext`. - -The specific variant of SQL that is used to parse queries can also be selected using the -`spark.sql.dialect` option. This parameter can be changed using either the `setConf` method on -a `SQLContext` or by using a `SET key=value` command in SQL. For a `SQLContext`, the only dialect -available is "sql" which uses a simple SQL parser provided by Spark SQL. In a `HiveContext`, the -default is "hiveql", though "sql" is also available. Since the HiveQL parser is much more complete, -this is recommended for most use cases. - +`SparkSession` in Spark 2.0 provides builtin support for Hive features including the ability to +write queries using HiveQL, access to Hive UDFs, and the ability to read data from Hive tables. +To use these features, you do not need to have an existing Hive setup. ## Creating DataFrames -With a `SQLContext`, applications can create `DataFrame`s from an existing `RDD`, from a Hive table, or from data sources. - -As an example, the following creates a `DataFrame` based on the content of a JSON file: -
    -{% highlight scala %} -val sc: SparkContext // An existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) - -val df = sqlContext.read.json("examples/src/main/resources/people.json") +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](#data-sources). -// Displays the content of the DataFrame to stdout -df.show() -{% endhighlight %} +As an example, the following creates a DataFrame based on the content of a JSON file: +{% include_example create_df scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    -{% highlight java %} -JavaSparkContext sc = ...; // An existing JavaSparkContext. -SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](#data-sources). -DataFrame df = sqlContext.read().json("examples/src/main/resources/people.json"); - -// Displays the content of the DataFrame to stdout -df.show(); -{% endhighlight %} +As an example, the following creates a DataFrame based on the content of a JSON file: +{% include_example create_df java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    -{% highlight python %} -from pyspark.sql import SQLContext -sqlContext = SQLContext(sc) +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](#data-sources). -df = sqlContext.read.json("examples/src/main/resources/people.json") - -# Displays the content of the DataFrame to stdout -df.show() -{% endhighlight %} +As an example, the following creates a DataFrame based on the content of a JSON file: +{% include_example create_df python/sql/basic.py %}
    -{% highlight r %} -sqlContext <- SQLContext(sc) +With a `SparkSession`, applications can create DataFrames from a local R data.frame, +from a Hive table, or from [Spark data sources](#data-sources). -df <- jsonFile(sqlContext, "examples/src/main/resources/people.json") +As an example, the following creates a DataFrame based on the content of a JSON file: -# Displays the content of the DataFrame to stdout -showDF(df) -{% endhighlight %} +{% include_example create_df r/RSparkSQLExample.R %}
    -
    -## DataFrame Operations +## Untyped Dataset Operations (aka DataFrame Operations) -DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.DataFrame), [Java](api/java/index.html?org/apache/spark/sql/DataFrame.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/DataFrame.html). +DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset), [Java](api/java/index.html?org/apache/spark/sql/Dataset.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/SparkDataFrame.html). -Here we include some basic examples of structured data processing using DataFrames: +As mentioned above, in Spark 2.0, DataFrames are just Dataset of `Row`s in Scala and Java API. These operations are also referred as "untyped transformations" in contrast to "typed transformations" come with strongly typed Scala/Java Datasets. + +Here we include some basic examples of structured data processing using Datasets:
    -{% highlight scala %} -val sc: SparkContext // An existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) - -// Create the DataFrame -val df = sqlContext.read.json("examples/src/main/resources/people.json") - -// Show the content of the DataFrame -df.show() -// age name -// null Michael -// 30 Andy -// 19 Justin - -// Print the schema in a tree format -df.printSchema() -// root -// |-- age: long (nullable = true) -// |-- name: string (nullable = true) - -// Select only the "name" column -df.select("name").show() -// name -// Michael -// Andy -// Justin - -// Select everybody, but increment the age by 1 -df.select(df("name"), df("age") + 1).show() -// name (age + 1) -// Michael null -// Andy 31 -// Justin 20 - -// Select people older than 21 -df.filter(df("age") > 21).show() -// age name -// 30 Andy - -// Count people by age -df.groupBy("age").count().show() -// age count -// null 1 -// 19 1 -// 30 1 -{% endhighlight %} - -For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.DataFrame). - -In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/scala/index.html#org.apache.spark.sql.functions$). +{% include_example untyped_ops scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.Dataset). +In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/scala/index.html#org.apache.spark.sql.functions$).
    -{% highlight java %} -JavaSparkContext sc // An existing SparkContext. -SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc) - -// Create the DataFrame -DataFrame df = sqlContext.read().json("examples/src/main/resources/people.json"); - -// Show the content of the DataFrame -df.show(); -// age name -// null Michael -// 30 Andy -// 19 Justin - -// Print the schema in a tree format -df.printSchema(); -// root -// |-- age: long (nullable = true) -// |-- name: string (nullable = true) - -// Select only the "name" column -df.select("name").show(); -// name -// Michael -// Andy -// Justin - -// Select everybody, but increment the age by 1 -df.select(df.col("name"), df.col("age").plus(1)).show(); -// name (age + 1) -// Michael null -// Andy 31 -// Justin 20 - -// Select people older than 21 -df.filter(df.col("age").gt(21)).show(); -// age name -// 30 Andy - -// Count people by age -df.groupBy("age").count().show(); -// age count -// null 1 -// 19 1 -// 30 1 -{% endhighlight %} -For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/java/org/apache/spark/sql/DataFrame.html). +{% include_example untyped_ops java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/java/org/apache/spark/sql/functions.html). +For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/java/org/apache/spark/sql/Dataset.html). +In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/java/org/apache/spark/sql/functions.html).
    @@ -281,111 +171,20 @@ interactive data exploration, users are highly encouraged to use the latter form, which is future proof and won't break with column names that are also attributes on the DataFrame class. -{% highlight python %} -from pyspark.sql import SQLContext -sqlContext = SQLContext(sc) - -# Create the DataFrame -df = sqlContext.read.json("examples/src/main/resources/people.json") - -# Show the content of the DataFrame -df.show() -## age name -## null Michael -## 30 Andy -## 19 Justin - -# Print the schema in a tree format -df.printSchema() -## root -## |-- age: long (nullable = true) -## |-- name: string (nullable = true) - -# Select only the "name" column -df.select("name").show() -## name -## Michael -## Andy -## Justin - -# Select everybody, but increment the age by 1 -df.select(df['name'], df['age'] + 1).show() -## name (age + 1) -## Michael null -## Andy 31 -## Justin 20 - -# Select people older than 21 -df.filter(df['age'] > 21).show() -## age name -## 30 Andy - -# Count people by age -df.groupBy("age").count().show() -## age count -## null 1 -## 19 1 -## 30 1 - -{% endhighlight %} - +{% include_example untyped_ops python/sql/basic.py %} For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/python/pyspark.sql.html#pyspark.sql.DataFrame). -In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/python/pyspark.sql.html#module-pyspark.sql.functions). +In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/python/pyspark.sql.html#module-pyspark.sql.functions).
    -{% highlight r %} -sqlContext <- sparkRSQL.init(sc) - -# Create the DataFrame -df <- jsonFile(sqlContext, "examples/src/main/resources/people.json") - -# Show the content of the DataFrame -showDF(df) -## age name -## null Michael -## 30 Andy -## 19 Justin - -# Print the schema in a tree format -printSchema(df) -## root -## |-- age: long (nullable = true) -## |-- name: string (nullable = true) - -# Select only the "name" column -showDF(select(df, "name")) -## name -## Michael -## Andy -## Justin - -# Select everybody, but increment the age by 1 -showDF(select(df, df$name, df$age + 1)) -## name (age + 1) -## Michael null -## Andy 31 -## Justin 20 - -# Select people older than 21 -showDF(where(df, df$age > 21)) -## age name -## 30 Andy - -# Count people by age -showDF(count(groupBy(df, "age"))) -## age count -## null 1 -## 19 1 -## 30 1 -{% endhighlight %} +{% include_example untyped_ops r/RSparkSQLExample.R %} For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/R/index.html). -In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/R/index.html). +In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/R/SparkDataFrame.html).
    @@ -393,202 +192,133 @@ In addition to simple column references and expressions, DataFrames also have a ## Running SQL Queries Programmatically -The `sql` function on a `SQLContext` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. -
    -{% highlight scala %} -val sqlContext = ... // An existing SQLContext -val df = sqlContext.sql("SELECT * FROM table") -{% endhighlight %} +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. + +{% include_example run_sql scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    -{% highlight java %} -SQLContext sqlContext = ... // An existing SQLContext -DataFrame df = sqlContext.sql("SELECT * FROM table") -{% endhighlight %} +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `Dataset`. + +{% include_example run_sql java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    -{% highlight python %} -from pyspark.sql import SQLContext -sqlContext = SQLContext(sc) -df = sqlContext.sql("SELECT * FROM table") -{% endhighlight %} +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. + +{% include_example run_sql python/sql/basic.py %}
    -{% highlight r %} -sqlContext <- sparkRSQL.init(sc) -df <- sql(sqlContext, "SELECT * FROM table") -{% endhighlight %} -
    +The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. -
    +{% include_example run_sql r/RSparkSQLExample.R %} +
    +
    -## Interoperating with RDDs -Spark SQL supports two different methods for converting existing RDDs into DataFrames. The first -method uses reflection to infer the schema of an RDD that contains specific types of objects. This -reflection based approach leads to more concise code and works well when you already know the schema -while writing your Spark application. +## Global Temporary View -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 DataFrames when the columns and their types are not known until runtime. +Temporary views in Spark SQL are session-scoped and will disappear if the session that creates it +terminates. If you want to have a temporary view that is shared among all sessions and keep alive +until the Spark application terminates, you can create a global temporary view. Global temporary +view is tied to a system preserved database `global_temp`, and we must use the qualified name to +refer it, e.g. `SELECT * FROM global_temp.view1`. -### Inferring the Schema Using Reflection
    -
    +{% include_example global_temp_view scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    -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 -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) -// this is used to implicitly convert an RDD to a DataFrame. -import sqlContext.implicits._ +
    +{% include_example global_temp_view java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    -// 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, -// you can use custom classes that implement the Product interface. -case class Person(name: String, age: Int) +
    +{% include_example global_temp_view python/sql/basic.py %} +
    -// Create an RDD of Person objects and register it as a table. -val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)).toDF() -people.registerTempTable("people") +
    -// SQL statements can be run by using the sql methods provided by sqlContext. -val teenagers = sqlContext.sql("SELECT name, age FROM people WHERE age >= 13 AND age <= 19") +{% highlight sql %} -// 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 field index: -teenagers.map(t => "Name: " + t(0)).collect().foreach(println) +CREATE GLOBAL TEMPORARY VIEW temp_view AS SELECT a + 1, b * 2 FROM tbl -// or by field name: -teenagers.map(t => "Name: " + t.getAs[String]("name")).collect().foreach(println) +SELECT * FROM global_temp.temp_view -// row.getValuesMap[T] retrieves multiple columns at once into a Map[String, T] -teenagers.map(_.getValuesMap[Any](List("name", "age"))).collect().foreach(println) -// Map("name" -> "Justin", "age" -> 19) {% endhighlight %}
    +
    -
    -Spark SQL supports automatically converting an RDD of [JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly) -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. +## Creating Datasets -{% highlight java %} +Datasets are similar to RDDs, however, instead of using Java serialization or Kryo they use +a specialized [Encoder](api/scala/index.html#org.apache.spark.sql.Encoder) to serialize the objects +for processing or transmitting over the network. While both encoders and standard serialization are +responsible for turning an object into bytes, encoders are code generated dynamically and use a format +that allows Spark to perform many operations like filtering, sorting and hashing without deserializing +the bytes back into an object. + +
    +
    +{% include_example create_ds scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    -public static class Person implements Serializable { - private String name; - private int age; +
    +{% include_example create_ds java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    +
    - public String getName() { - return name; - } +## Interoperating with RDDs - public void setName(String name) { - this.name = name; - } +Spark SQL supports two different methods for converting existing RDDs into Datasets. 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. - public int getAge() { - return age; - } +The second method for creating Datasets 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 Datasets when the columns and their types are not known until runtime. - public void setAge(int age) { - this.age = age; - } -} +### Inferring the Schema Using Reflection +
    -{% endhighlight %} +
    +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 +types such as `Seq`s or `Array`s. This RDD can be implicitly converted to a DataFrame and then be +registered as a table. Tables can be used in subsequent SQL statements. -A schema can be applied to an existing RDD by calling `createDataFrame` and providing the Class object -for the JavaBean. +{% include_example schema_inferring scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    -{% highlight java %} -// sc is an existing JavaSparkContext. -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( - new Function() { - public Person call(String line) throws Exception { - String[] parts = line.split(","); - - Person person = new Person(); - person.setName(parts[0]); - person.setAge(Integer.parseInt(parts[1].trim())); - - return person; - } - }); - -// Apply a schema to an RDD of JavaBeans and register it as a table. -DataFrame schemaPeople = sqlContext.createDataFrame(people, Person.class); -schemaPeople.registerTempTable("people"); - -// SQL can be run over RDDs that have been registered as tables. -DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") - -// The results of SQL queries are DataFrames and support all the normal RDD operations. -// The columns of a row in the result can be accessed by ordinal. -List teenagerNames = teenagers.javaRDD().map(new Function() { - public String call(Row row) { - return "Name: " + row.getString(0); - } -}).collect(); +
    -{% endhighlight %} +Spark SQL supports automatically converting an RDD of +[JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly) into a DataFrame. +The `BeanInfo`, obtained using reflection, defines the schema of the table. Currently, Spark SQL +does not support JavaBeans that contain `Map` field(s). Nested JavaBeans and `List` or `Array` +fields are supported though. You can create a JavaBean by creating a class that implements +Serializable and has getters and setters for all of its fields. +{% include_example schema_inferring java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    -Spark SQL can convert an RDD of Row objects to a DataFrame, 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 -plan to more completely infer the schema by looking at more data, similar to the inference that is -performed on JSON files. - -{% highlight python %} -# sc is an existing SparkContext. -from pyspark.sql import SQLContext, Row -sqlContext = SQLContext(sc) - -# Load a text file and convert each line to a Row. -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 DataFrame as a table. -schemaPeople = sqlContext.createDataFrame(people) -schemaPeople.registerTempTable("people") - -# 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. -teenNames = teenagers.map(lambda p: "Name: " + p.name) -for teenName in teenNames.collect(): - print(teenName) -{% endhighlight %} +and the types are inferred by sampling the whole dataset, similar to the inference that is performed on JSON files. +{% include_example schema_inferring python/sql/basic.py %}
    @@ -608,48 +338,11 @@ a `DataFrame` can be created programmatically with three steps. 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 `SQLContext`. +by `SparkSession`. For example: -{% highlight scala %} -// sc is an existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) - -// Create an RDD -val people = sc.textFile("examples/src/main/resources/people.txt") - -// The schema is encoded in a string -val schemaString = "name age" - -// Import Row. -import org.apache.spark.sql.Row; - -// Import Spark SQL data types -import org.apache.spark.sql.types.{StructType,StructField,StringType}; - -// Generate the schema based on the string of schema -val schema = - StructType( - schemaString.split(" ").map(fieldName => StructField(fieldName, StringType, true))) - -// Convert records of the RDD (people) to Rows. -val rowRDD = people.map(_.split(",")).map(p => Row(p(0), p(1).trim)) - -// Apply the schema to the RDD. -val peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema) - -// 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 DataFrames and support all the normal RDD operations. -// The columns of a row in the result can be accessed by field index or by field name. -results.map(t => "Name: " + t(0)).collect().foreach(println) -{% endhighlight %} - +{% include_example programmatic_schema scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    @@ -657,71 +350,17 @@ 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 `DataFrame` can be created programmatically with three steps. +a `Dataset` 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 `SQLContext`. +by `SparkSession`. For example: -{% highlight java %} -import org.apache.spark.api.java.function.Function; -// Import factory methods provided by DataTypes. -import org.apache.spark.sql.types.DataTypes; -// Import StructType and StructField -import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.types.StructField; -// Import Row. -import org.apache.spark.sql.Row; -// Import RowFactory. -import org.apache.spark.sql.RowFactory; - -// sc is an existing JavaSparkContext. -SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc); - -// Load a text file and convert each line to a JavaBean. -JavaRDD people = sc.textFile("examples/src/main/resources/people.txt"); - -// The schema is encoded in a string -String schemaString = "name age"; - -// Generate the schema based on the string of schema -List fields = new ArrayList(); -for (String fieldName: schemaString.split(" ")) { - fields.add(DataTypes.createStructField(fieldName, DataTypes.StringType, true)); -} -StructType schema = DataTypes.createStructType(fields); - -// Convert records of the RDD (people) to Rows. -JavaRDD rowRDD = people.map( - new Function() { - public Row call(String record) throws Exception { - String[] fields = record.split(","); - return RowFactory.create(fields[0], fields[1].trim()); - } - }); - -// Apply the schema to the RDD. -DataFrame peopleDataFrame = sqlContext.createDataFrame(rowRDD, schema); - -// Register the DataFrame as a table. -peopleDataFrame.registerTempTable("people"); - -// SQL can be run over RDDs that have been registered as tables. -DataFrame results = sqlContext.sql("SELECT name FROM people"); - -// The results of SQL queries are DataFrames and support all the normal RDD operations. -// The columns of a row in the result can be accessed by ordinal. -List names = results.javaRDD().map(new Function() { - public String call(Row row) { - return "Name: " + row.getString(0); - } -}).collect(); - -{% endhighlight %} +{% include_example programmatic_schema java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %}
    @@ -734,53 +373,57 @@ 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 tuples or lists in the RDD created in the step 1. -3. Apply the schema to the RDD via `createDataFrame` method provided by `SQLContext`. +3. Apply the schema to the RDD via `createDataFrame` method provided by `SparkSession`. For example: -{% highlight python %} -# Import SQLContext and data types -from pyspark.sql import SQLContext -from pyspark.sql.types import * -# sc is an existing SparkContext. -sqlContext = SQLContext(sc) +{% include_example programmatic_schema python/sql/basic.py %} +
    -# Load a text file and convert each line to a tuple. -lines = sc.textFile("examples/src/main/resources/people.txt") -parts = lines.map(lambda l: l.split(",")) -people = parts.map(lambda p: (p[0], p[1].strip())) + -# The schema is encoded in a string. -schemaString = "name age" +## Aggregations -fields = [StructField(field_name, StringType(), True) for field_name in schemaString.split()] -schema = StructType(fields) +The [built-in DataFrames functions](api/scala/index.html#org.apache.spark.sql.functions$) provide common +aggregations such as `count()`, `countDistinct()`, `avg()`, `max()`, `min()`, etc. +While those functions are designed for DataFrames, Spark SQL also has type-safe versions for some of them in +[Scala](api/scala/index.html#org.apache.spark.sql.expressions.scalalang.typed$) and +[Java](api/java/org/apache/spark/sql/expressions/javalang/typed.html) to work with strongly typed Datasets. +Moreover, users are not limited to the predefined aggregate functions and can create their own. -# Apply the schema to the RDD. -schemaPeople = sqlContext.createDataFrame(people, schema) +### Untyped User-Defined Aggregate Functions +Users have to extend the [UserDefinedAggregateFunction](api/scala/index.html#org.apache.spark.sql.expressions.UserDefinedAggregateFunction) +abstract class to implement a custom untyped aggregate function. For example, a user-defined average +can look like: -# Register the DataFrame as a table. -schemaPeople.registerTempTable("people") +
    +
    +{% include_example untyped_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala%} +
    +
    +{% include_example untyped_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java%} +
    +
    -# SQL can be run over DataFrames that have been registered as a table. -results = sqlContext.sql("SELECT name FROM people") +### Type-Safe User-Defined Aggregate Functions -# The results of SQL queries are RDDs and support all the normal RDD operations. -names = results.map(lambda p: "Name: " + p.name) -for name in names.collect(): - print(name) -{% endhighlight %} +User-defined aggregations for strongly typed Datasets revolve around the [Aggregator](api/scala/index.html#org.apache.spark.sql.expressions.Aggregator) abstract class. +For example, a type-safe user-defined average can look like: +
    +
    +{% include_example typed_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala%} +
    +
    +{% include_example typed_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java%}
    -
    - # Data Sources -Spark SQL supports operating on a variety of data sources through the `DataFrame` interface. -A DataFrame can be operated on as normal RDDs and can also be registered as a temporary table. -Registering a DataFrame as a table allows you to run SQL queries over its data. This section +Spark SQL supports operating on a variety of data sources through the DataFrame interface. +A DataFrame can be operated on using relational transformations and can also be used to create a temporary view. +Registering a DataFrame as a temporary view allows you to run SQL queries over its data. This section describes the general methods for loading and saving data using the Spark Data Sources and then goes into specific options that are available for the built-in data sources. @@ -791,42 +434,21 @@ In the simplest form, the default data source (`parquet` unless otherwise config
    - -{% highlight scala %} -val df = sqlContext.read.load("examples/src/main/resources/users.parquet") -df.select("name", "favorite_color").write.save("namesAndFavColors.parquet") -{% endhighlight %} - +{% include_example generic_load_save_functions scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    - -{% highlight java %} - -DataFrame df = sqlContext.read().load("examples/src/main/resources/users.parquet"); -df.select("name", "favorite_color").write().save("namesAndFavColors.parquet"); - -{% endhighlight %} - +{% include_example generic_load_save_functions java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    -{% highlight python %} - -df = sqlContext.read.load("examples/src/main/resources/users.parquet") -df.select("name", "favorite_color").write.save("namesAndFavColors.parquet") - -{% endhighlight %} - +{% include_example generic_load_save_functions python/sql/datasource.py %}
    -{% highlight r %} -df <- loadDF(sqlContext, "people.parquet") -saveDF(select(df, "name", "age"), "namesAndAges.parquet") -{% endhighlight %} +{% include_example generic_load_save_functions r/RSparkSQLExample.R %}
    @@ -834,51 +456,26 @@ saveDF(select(df, "name", "age"), "namesAndAges.parquet") ### Manually Specifying Options You can also manually specify the data source that will be used along with any extra options -that you would like to pass to the data source. Data sources are specified by their fully qualified +that you would like to pass to the data source. Data sources are specified by their fully qualified name (i.e., `org.apache.spark.sql.parquet`), but for built-in sources you can also use their short -names (`json`, `parquet`, `jdbc`). DataFrames of any type can be converted into other types -using this syntax. +names (`json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`). DataFrames loaded from any data +source type can be converted into other types using this syntax.
    - -{% highlight scala %} -val df = sqlContext.read.format("json").load("examples/src/main/resources/people.json") -df.select("name", "age").write.format("parquet").save("namesAndAges.parquet") -{% endhighlight %} - +{% include_example manual_load_options scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    - -{% highlight java %} - -DataFrame df = sqlContext.read().format("json").load("examples/src/main/resources/people.json"); -df.select("name", "age").write().format("parquet").save("namesAndAges.parquet"); - -{% endhighlight %} - +{% include_example manual_load_options java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    - -{% highlight python %} - -df = sqlContext.read.load("examples/src/main/resources/people.json", format="json") -df.select("name", "age").write.save("namesAndAges.parquet", format="parquet") - -{% endhighlight %} - +{% include_example manual_load_options python/sql/datasource.py %}
    -
    - -{% highlight r %} - -df <- loadDF(sqlContext, "people.json", "json") -saveDF(select(df, "name", "age"), "namesAndAges.parquet", "parquet") - -{% endhighlight %} +
    +{% include_example manual_load_options r/RSparkSQLExample.R %}
    @@ -889,33 +486,19 @@ file directly with SQL.
    - -{% highlight scala %} -val df = sqlContext.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") -{% endhighlight %} - +{% include_example direct_sql scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    - -{% highlight java %} -DataFrame df = sqlContext.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"); -{% endhighlight %} +{% include_example direct_sql java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    - -{% highlight python %} -df = sqlContext.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") -{% endhighlight %} - +{% include_example direct_sql python/sql/datasource.py %}
    - -{% highlight r %} -df <- sql(sqlContext, "SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") -{% endhighlight %} +{% include_example direct_sql r/RSparkSQLExample.R %}
    @@ -923,15 +506,15 @@ df <- sql(sqlContext, "SELECT * FROM parquet.`examples/src/main/resources/users. ### Save Modes Save operations can optionally take a `SaveMode`, that specifies how to handle existing data if -present. It is important to realize that these save modes do not utilize any locking and are not -atomic. Additionally, when performing a `Overwrite`, the data will be deleted before writing out the +present. It is important to realize that these save modes do not utilize any locking and are not +atomic. Additionally, when performing an `Overwrite`, the data will be deleted before writing out the new data. - +
    Scala/JavaAny LanguageMeaning
    SaveMode.ErrorIfExists (default)"error" (default)"error" or "errorifexists" (default) When saving a DataFrame to a data source, if data already exists, an exception is expected to be thrown. @@ -960,141 +543,171 @@ new data. Ignore mode means that when saving a DataFrame to a data source, if data already exists, the save operation is expected to not save the contents of the DataFrame and to not - change the existing data. This is similar to a CREATE TABLE IF NOT EXISTS in SQL. + change the existing data. This is similar to a CREATE TABLE IF NOT EXISTS in SQL.
    ### Saving to Persistent Tables -When working with a `HiveContext`, `DataFrames` can also be saved as persistent tables using the -`saveAsTable` command. Unlike the `registerTempTable` command, `saveAsTable` will materialize the -contents of the dataframe and create a pointer to the data in the HiveMetastore. Persistent tables -will still exist even after your Spark program has restarted, as long as you maintain your connection -to the same metastore. A DataFrame for a persistent table can be created by calling the `table` -method on a `SQLContext` with the name of the table. +`DataFrames` can also be saved as persistent tables into Hive metastore using the `saveAsTable` +command. Notice that an existing Hive deployment is not necessary to use this feature. Spark will create a +default local Hive metastore (using Derby) for you. Unlike the `createOrReplaceTempView` command, +`saveAsTable` will materialize the contents of the DataFrame and create a pointer to the data in the +Hive metastore. Persistent tables will still exist even after your Spark program has restarted, as +long as you maintain your connection to the same metastore. A DataFrame for a persistent table can +be created by calling the `table` method on a `SparkSession` with the name of the table. -By default `saveAsTable` will create a "managed table", meaning that the location of the data will -be controlled by the metastore. Managed tables will also have their data deleted automatically -when a table is dropped. +For file-based data source, e.g. text, parquet, json, etc. you can specify a custom table path via the +`path` option, e.g. `df.write.option("path", "/some/path").saveAsTable("t")`. When the table is dropped, +the custom table path will not be removed and the table data is still there. If no custom table path is +specified, Spark will write data to a default table path under the warehouse directory. When the table is +dropped, the default table path will be removed too. -## Parquet Files +Starting from Spark 2.1, persistent datasource tables have per-partition metadata stored in the Hive metastore. This brings several benefits: -[Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. -Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema -of the original data. +- Since the metastore can return only necessary partitions for a query, discovering all the partitions on the first query to the table is no longer needed. +- Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. -### Loading Data Programmatically +Note that partition information is not gathered by default when creating external datasource tables (those with a `path` option). To sync the partition information in the metastore, you can invoke `MSCK REPAIR TABLE`. -Using the data from the above example: +### Bucketing, Sorting and Partitioning + +For file-based data source, it is also possible to bucket and sort or partition the output. +Bucketing and sorting are applicable only to persistent tables:
    +{% include_example write_sorting_and_bucketing scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    -{% highlight scala %} -// sqlContext from the previous example is used in this example. -// This is used to implicitly convert an RDD to a DataFrame. -import sqlContext.implicits._ +
    +{% include_example write_sorting_and_bucketing java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    -val people: RDD[Person] = ... // An RDD of case class objects, from the previous example. +
    +{% include_example write_sorting_and_bucketing python/sql/datasource.py %} +
    -// The RDD is implicitly converted to a DataFrame by implicits, allowing it to be stored using Parquet. -people.write.parquet("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 DataFrame. -val parquetFile = sqlContext.read.parquet("people.parquet") +{% highlight sql %} + +CREATE TABLE users_bucketed_by_name( + name STRING, + favorite_color STRING, + favorite_numbers array +) USING parquet +CLUSTERED BY(name) INTO 42 BUCKETS; -//Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerTempTable("parquetFile") -val teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") -teenagers.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %}
    +
    + +while partitioning can be used with both `save` and `saveAsTable` when using the Dataset APIs. + + +
    + +
    +{% include_example write_partitioning scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    +
    +{% include_example write_partitioning java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    -{% highlight java %} -// sqlContext from the previous example is used in this example. +
    +{% include_example write_partitioning python/sql/datasource.py %} +
    -DataFrame schemaPeople = ... // The DataFrame from the previous example. +
    -// DataFrames can be saved as Parquet files, maintaining the schema information. -schemaPeople.write().parquet("people.parquet"); +{% highlight sql %} -// Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved. -// The result of loading a parquet file is also a DataFrame. -DataFrame parquetFile = sqlContext.read().parquet("people.parquet"); +CREATE TABLE users_by_favorite_color( + name STRING, + favorite_color STRING, + favorite_numbers array +) USING csv PARTITIONED BY(favorite_color); -// Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerTempTable("parquetFile"); -DataFrame teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); -List teenagerNames = teenagers.javaRDD().map(new Function() { - public String call(Row row) { - return "Name: " + row.getString(0); - } -}).collect(); {% endhighlight %}
    -
    +
    -{% highlight python %} -# sqlContext from the previous example is used in this example. +It is possible to use both partitioning and bucketing for a single table: + +
    + +
    +{% include_example write_partition_and_bucket scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    -schemaPeople # The DataFrame from the previous example. +
    +{% include_example write_partition_and_bucket java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    -# DataFrames can be saved as Parquet files, maintaining the schema information. -schemaPeople.write.parquet("people.parquet") +
    +{% include_example write_partition_and_bucket python/sql/datasource.py %} +
    -# Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved. -# The result of loading a parquet file is also a DataFrame. -parquetFile = sqlContext.read.parquet("people.parquet") +
    + +{% highlight sql %} + +CREATE TABLE users_bucketed_and_partitioned( + name STRING, + favorite_color STRING, + favorite_numbers array +) USING parquet +PARTITIONED BY (favorite_color) +CLUSTERED BY(name) SORTED BY (favorite_numbers) INTO 42 BUCKETS; -# Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerTempTable("parquetFile"); -teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") -teenNames = teenagers.map(lambda p: "Name: " + p.name) -for teenName in teenNames.collect(): - print(teenName) {% endhighlight %}
    -
    +
    -{% highlight r %} -# sqlContext from the previous example is used in this example. +`partitionBy` creates a directory structure as described in the [Partition Discovery](#partition-discovery) section. +Thus, it has limited applicability to columns with high cardinality. In contrast + `bucketBy` distributes +data across a fixed number of buckets and can be used when a number of unique values is unbounded. -schemaPeople # The DataFrame from the previous example. +## Parquet Files -# DataFrames can be saved as Parquet files, maintaining the schema information. -saveAsParquetFile(schemaPeople, "people.parquet") +[Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. +Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema +of the original data. When writing Parquet files, all columns are automatically converted to be nullable for +compatibility reasons. -# Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved. -# The result of loading a parquet file is also a DataFrame. -parquetFile <- parquetFile(sqlContext, "people.parquet") +### Loading Data Programmatically -# Parquet files can also be registered as tables and then used in SQL statements. -registerTempTable(parquetFile, "parquetFile"); -teenagers <- sql(sqlContext, "SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") -teenNames <- map(teenagers, function(p) { paste("Name:", p$name)}) -for (teenName in collect(teenNames)) { - cat(teenName, "\n") -} -{% endhighlight %} +Using the data from the above example: + +
    +
    +{% include_example basic_parquet_example scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example basic_parquet_example java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    -{% highlight python %} -# sqlContext is an existing HiveContext -sqlContext.sql("REFRESH TABLE my_table") -{% endhighlight %} +{% include_example basic_parquet_example python/sql/datasource.py %} +
    + +
    + +{% include_example basic_parquet_example r/RSparkSQLExample.R %}
    @@ -1102,7 +715,7 @@ sqlContext.sql("REFRESH TABLE my_table") {% highlight sql %} -CREATE TEMPORARY TABLE parquetTable +CREATE TEMPORARY VIEW parquetTable USING org.apache.spark.sql.parquet OPTIONS ( path "examples/src/main/resources/people.parquet" @@ -1118,10 +731,11 @@ SELECT * FROM parquetTable ### Partition Discovery -Table partitioning is a common optimization approach used in systems like Hive. In a partitioned +Table partitioning is a common optimization approach used in systems like Hive. In a partitioned table, data are usually stored in different directories, with partitioning column values encoded in -the path of each partition directory. The Parquet data source is now able to discover and infer -partitioning information automatically. For example, we can store all our previously used +the path of each partition directory. All built-in file sources (including Text/CSV/JSON/ORC/Parquet) +are able to discover and infer partitioning information automatically. +For example, we can store all our previously used population data into a partitioned table using the following directory structure, with two extra columns, `gender` and `country` as partitioning columns: @@ -1149,7 +763,7 @@ path {% endhighlight %} -By passing `path/to/table` to either `SQLContext.read.parquet` or `SQLContext.read.load`, Spark SQL +By passing `path/to/table` to either `SparkSession.read.parquet` or `SparkSession.read.load`, Spark SQL will automatically extract the partitioning information from the paths. Now the schema of the returned DataFrame becomes: @@ -1163,22 +777,29 @@ root {% endhighlight %} -Notice that the data types of the partitioning columns are automatically inferred. Currently, +Notice that the data types of the partitioning columns are automatically inferred. Currently, numeric data types and string type are supported. Sometimes users may not want to automatically infer the data types of the partitioning columns. For these use cases, the automatic type inference can be configured by `spark.sql.sources.partitionColumnTypeInference.enabled`, which is default to `true`. When type inference is disabled, string type will be used for the partitioning columns. +Starting from Spark 1.6.0, partition discovery only finds partitions under the given paths +by default. For the above example, if users pass `path/to/table/gender=male` to either +`SparkSession.read.parquet` or `SparkSession.read.load`, `gender` will not be considered as a +partitioning column. If users need to specify the base path that partition discovery +should start with, they can set `basePath` in the data source options. For example, +when `path/to/table/gender=male` is the path of the data and +users set `basePath` to `path/to/table/`, `gender` will be a partitioning column. ### Schema Merging -Like ProtocolBuffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with -a simple schema, and gradually add more columns to the schema as needed. In this way, users may end -up with multiple Parquet files with different but mutually compatible schemas. The Parquet data +Like ProtocolBuffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with +a simple schema, and gradually add more columns to the schema as needed. In this way, users may end +up with multiple Parquet files with different but mutually compatible schemas. The Parquet data source is now able to automatically detect this case and merge schemas of all these files. Since schema merging is a relatively expensive operation, and is not a necessity in most cases, we -turned it off by default starting from 1.5.0. You may enable it by +turned it off by default starting from 1.5.0. You may enable it by 1. setting data source option `mergeSchema` to `true` when reading Parquet files (as shown in the examples below), or @@ -1187,91 +808,21 @@ turned it off by default starting from 1.5.0. You may enable it by
    +{% include_example schema_merging scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    -{% highlight scala %} -// sqlContext from the previous example is used in this example. -// This is used to implicitly convert an RDD to a DataFrame. -import sqlContext.implicits._ - -// Create a simple DataFrame, stored into a partition directory -val df1 = sc.makeRDD(1 to 5).map(i => (i, i * 2)).toDF("single", "double") -df1.write.parquet("data/test_table/key=1") - -// Create another DataFrame in a new partition directory, -// adding a new column and dropping an existing column -val df2 = sc.makeRDD(6 to 10).map(i => (i, i * 3)).toDF("single", "triple") -df2.write.parquet("data/test_table/key=2") - -// Read the partitioned table -val df3 = sqlContext.read.option("mergeSchema", "true").parquet("data/test_table") -df3.printSchema() - -// The final schema consists of all 3 columns in the Parquet files together -// with the partitioning column appeared in the partition directory paths. -// root -// |-- single: int (nullable = true) -// |-- double: int (nullable = true) -// |-- triple: int (nullable = true) -// |-- key : int (nullable = true) -{% endhighlight %} - +
    +{% include_example schema_merging java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    -{% highlight python %} -# sqlContext from the previous example is used in this example. - -# Create a simple DataFrame, stored into a partition directory -df1 = sqlContext.createDataFrame(sc.parallelize(range(1, 6))\ - .map(lambda i: Row(single=i, double=i * 2))) -df1.write.parquet("data/test_table/key=1") - -# Create another DataFrame in a new partition directory, -# adding a new column and dropping an existing column -df2 = sqlContext.createDataFrame(sc.parallelize(range(6, 11)) - .map(lambda i: Row(single=i, triple=i * 3))) -df2.write.parquet("data/test_table/key=2") - -# Read the partitioned table -df3 = sqlContext.read.option("mergeSchema", "true").parquet("data/test_table") -df3.printSchema() - -# The final schema consists of all 3 columns in the Parquet files together -# with the partitioning column appeared in the partition directory paths. -# root -# |-- single: int (nullable = true) -# |-- double: int (nullable = true) -# |-- triple: int (nullable = true) -# |-- key : int (nullable = true) -{% endhighlight %} - +{% include_example schema_merging python/sql/datasource.py %}
    -{% highlight r %} -# sqlContext from the previous example is used in this example. - -# Create a simple DataFrame, stored into a partition directory -saveDF(df1, "data/test_table/key=1", "parquet", "overwrite") - -# Create another DataFrame in a new partition directory, -# adding a new column and dropping an existing column -saveDF(df2, "data/test_table/key=2", "parquet", "overwrite") - -# Read the partitioned table -df3 <- loadDF(sqlContext, "data/test_table", "parquet", mergeSchema="true") -printSchema(df3) - -# The final schema consists of all 3 columns in the Parquet files together -# with the partitioning column appeared in the partition directory paths. -# root -# |-- single: int (nullable = true) -# |-- double: int (nullable = true) -# |-- triple: int (nullable = true) -# |-- key : int (nullable = true) -{% endhighlight %} +{% include_example schema_merging r/RSparkSQLExample.R %}
    @@ -1292,22 +843,22 @@ processing. 1. Hive considers all columns nullable, while nullability in Parquet is significant Due to this reason, we must reconcile Hive metastore schema with Parquet schema when converting a -Hive metastore Parquet table to a Spark SQL Parquet table. The reconciliation rules are: +Hive metastore Parquet table to a Spark SQL Parquet table. The reconciliation rules are: 1. Fields that have the same name in both schema must have the same data type regardless of - nullability. The reconciled field should have the data type of the Parquet side, so that + nullability. The reconciled field should have the data type of the Parquet side, so that nullability is respected. 1. The reconciled schema contains exactly those fields defined in Hive metastore schema. - Any fields that only appear in the Parquet schema are dropped in the reconciled schema. - - Any fileds that only appear in the Hive metastore schema are added as nullable field in the + - Any fields that only appear in the Hive metastore schema are added as nullable field in the reconciled schema. #### Metadata Refreshing -Spark SQL caches Parquet metadata for better performance. When Hive metastore Parquet table -conversion is enabled, metadata of those converted tables are also cached. If these tables are +Spark SQL caches Parquet metadata for better performance. When Hive metastore Parquet table +conversion is enabled, metadata of those converted tables are also cached. If these tables are updated by Hive or other external tools, you need to refresh them manually to ensure consistent metadata. @@ -1316,8 +867,8 @@ metadata.
    {% highlight scala %} -// sqlContext is an existing HiveContext -sqlContext.refreshTable("my_table") +// spark is an existing SparkSession +spark.catalog.refreshTable("my_table") {% endhighlight %}
    @@ -1325,8 +876,8 @@ sqlContext.refreshTable("my_table")
    {% highlight java %} -// sqlContext is an existing HiveContext -sqlContext.refreshTable("my_table") +// spark is an existing SparkSession +spark.catalog().refreshTable("my_table"); {% endhighlight %}
    @@ -1334,8 +885,8 @@ sqlContext.refreshTable("my_table")
    {% highlight python %} -# sqlContext is an existing HiveContext -sqlContext.refreshTable("my_table") +# spark is an existing SparkSession +spark.catalog.refreshTable("my_table") {% endhighlight %}
    @@ -1352,7 +903,7 @@ REFRESH TABLE my_table; ### Configuration -Configuration of Parquet can be done using the `setConf` method on `SQLContext` or by running +Configuration of Parquet can be done using the `setConf` method on `SparkSession` or by running `SET key=value` commands using SQL. @@ -1370,20 +921,13 @@ Configuration of Parquet can be done using the `setConf` method on `SQLContext` - - - - - - + - - + + - - + + @@ -1449,157 +974,66 @@ Configuration of Parquet can be done using the `setConf` method on `SQLContext`
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using `SQLContext.read.json()` on either an RDD of String, +Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset[Row]`. +This conversion can be done using `SparkSession.read.json()` on either a `Dataset[String]`, or a JSON file. Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. As a consequence, -a regular multi-line JSON file will most often fail. +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). -{% highlight scala %} -// sc is an existing SparkContext. -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" -val people = sqlContext.read.json(path) - -// The inferred schema can be visualized using the printSchema() method. -people.printSchema() -// root -// |-- age: integer (nullable = true) -// |-- name: string (nullable = true) - -// 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 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) -val anotherPeople = sqlContext.read.json(anotherPeopleRDD) -{% endhighlight %} +For a regular multi-line JSON file, set the `multiLine` option to `true`. +{% include_example json_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using `SQLContext.read().json()` on either an RDD of String, +Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset`. +This conversion can be done using `SparkSession.read().json()` on either a `Dataset`, or a JSON file. Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. As a consequence, -a regular multi-line JSON file will most often fail. +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). -{% highlight java %} -// sc is an existing JavaSparkContext. -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. -DataFrame people = sqlContext.read().json("examples/src/main/resources/people.json"); - -// The inferred schema can be visualized using the printSchema() method. -people.printSchema(); -// root -// |-- age: integer (nullable = true) -// |-- name: string (nullable = true) - -// Register this DataFrame as a table. -people.registerTempTable("people"); - -// SQL statements can be run by using the sql methods provided by sqlContext. -DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); - -// Alternatively, a DataFrame can be created for a JSON dataset represented by -// 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); -DataFrame anotherPeople = sqlContext.read().json(anotherPeopleRDD); -{% endhighlight %} +For a regular multi-line JSON file, set the `multiLine` option to `true`. + +{% include_example json_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
    Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using `SQLContext.read.json` on a JSON file. +This conversion can be done using `SparkSession.read.json` on a JSON file. Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. As a consequence, -a regular multi-line JSON file will most often fail. +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). -{% highlight python %} -# sc is an existing SparkContext. -from pyspark.sql import SQLContext -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. -people = sqlContext.read.json("examples/src/main/resources/people.json") - -# The inferred schema can be visualized using the printSchema() method. -people.printSchema() -# root -# |-- age: integer (nullable = true) -# |-- name: string (nullable = true) - -# 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 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"}}']) -anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD) -{% endhighlight %} +For a regular multi-line JSON file, set the `multiLine` parameter to `True`. + +{% include_example json_dataset python/sql/datasource.py %}
    Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. using -the `jsonFile` function, which loads data from a directory of JSON files where each line of the +the `read.json()` function, which loads data from a directory of JSON files where each line of the files is a JSON object. Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. As a consequence, -a regular multi-line JSON file will most often fail. - -{% highlight r %} -# sc is an existing SparkContext. -sqlContext <- sparkRSQL.init(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 DataFrame from the file(s) pointed to by path -people <- jsonFile(sqlContext, path) - -# The inferred schema can be visualized using the printSchema() method. -printSchema(people) -# root -# |-- age: integer (nullable = true) -# |-- name: string (nullable = true) - -# Register this DataFrame as a table. -registerTempTable(people, "people") - -# SQL statements can be run by using the sql methods provided by `sqlContext`. -teenagers <- sql(sqlContext, "SELECT name FROM people WHERE age >= 13 AND age <= 19") -{% endhighlight %} +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). + +For a regular multi-line JSON file, set a named parameter `multiLine` to `TRUE`. + +{% include_example json_dataset r/RSparkSQLExample.R %} +
    {% highlight sql %} -CREATE TEMPORARY TABLE jsonTable +CREATE TEMPORARY VIEW jsonTable USING org.apache.spark.sql.json OPTIONS ( path "examples/src/main/resources/people.json" @@ -1616,100 +1050,95 @@ SELECT * FROM jsonTable ## Hive Tables Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). -However, since Hive has a large number of dependencies, it is not included in the default Spark assembly. -Hive support is enabled by adding the `-Phive` and `-Phive-thriftserver` flags to Spark's build. -This command builds a new assembly jar that includes Hive. Note that this Hive assembly jar must also be present -on all of the worker nodes, as they will need access to the Hive serialization and deserialization libraries -(SerDes) in order to access data stored in Hive. - -Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. Please note when running -the query on a YARN cluster (`cluster` mode), the `datanucleus` jars under the `lib_managed/jars` directory -and `hive-site.xml` under `conf/` directory need to be available on the driver and all executors launched by the -YARN cluster. The convenient way to do this is adding them through the `--jars` option and `--file` option of the -`spark-submit` command. - +However, since Hive has a large number of dependencies, these dependencies are not included in the +default Spark distribution. If Hive dependencies can be found on the classpath, Spark will load them +automatically. Note that these Hive dependencies must also be present on all of the worker nodes, as +they will need access to the Hive serialization and deserialization libraries (SerDes) in order to +access data stored in Hive. + +Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` (for security configuration), +and `hdfs-site.xml` (for HDFS configuration) file in `conf/`. + +When working with Hive, one must instantiate `SparkSession` with Hive support, including +connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. +Users who do not have an existing Hive deployment can still enable Hive support. When not configured +by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and +creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory +`spark-warehouse` in the current directory that the Spark application is started. Note that +the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. +Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. +You may need to grant write privilege to the user who starts the Spark application.
    - -When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and -adds support for finding tables in the MetaStore and writing queries using HiveQL. Users who do -not have an existing Hive deployment can still create a `HiveContext`. When not configured by the -hive-site.xml, the context automatically creates `metastore_db` and `warehouse` in the current -directory. - -{% highlight scala %} -// sc is an existing SparkContext. -val 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") - -// Queries are expressed in HiveQL -sqlContext.sql("FROM src SELECT key, value").collect().foreach(println) -{% endhighlight %} - +{% include_example spark_hive scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala %}
    - -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 `HiveContext` also provides an `hql` method, which allows queries to be -expressed in HiveQL. - -{% highlight java %} -// sc is an existing JavaSparkContext. -HiveContext sqlContext = new org.apache.spark.sql.hive.HiveContext(sc.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"); - -// Queries are expressed in HiveQL. -Row[] results = sqlContext.sql("FROM src SELECT key, value").collect(); - -{% endhighlight %} - +{% include_example spark_hive java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java %}
    +{% include_example spark_hive python/sql/hive.py %} +
    -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. -{% highlight python %} -# sc is an existing SparkContext. -from pyspark.sql import HiveContext -sqlContext = 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") +
    -# Queries can be expressed in HiveQL. -results = sqlContext.sql("FROM src SELECT key, value").collect() +When working with Hive one must instantiate `SparkSession` with Hive support. This +adds support for finding tables in the MetaStore and writing queries using HiveQL. -{% endhighlight %} +{% include_example spark_hive r/RSparkSQLExample.R %} +
    -
    +### Specifying storage format for Hive tables -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. -{% highlight r %} -# sc is an existing SparkContext. -sqlContext <- sparkRHive.init(sc) +When you create a Hive table, you need to define how this table should read/write data from/to file system, +i.e. the "input format" and "output format". You also need to define how this table should deserialize the data +to rows, or serialize rows to data, i.e. the "serde". The following options can be used to specify the storage +format("serde", "input format", "output format"), e.g. `CREATE TABLE src(id int) USING hive OPTIONS(fileFormat 'parquet')`. +By default, we will read the table files as plain text. Note that, Hive storage handler is not supported yet when +creating table, you can create a table using storage handler at Hive side, and use Spark SQL to read it. -sql(sqlContext, "CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -sql(sqlContext, "LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +
    spark.sql.parquet.int96AsTimestamp true - Some Parquet-producing systems, in particular Impala and Hive, store Timestamp into INT96. This + Some Parquet-producing systems, in particular Impala and Hive, store Timestamp into INT96. This flag tells Spark SQL to interpret INT96 data as a timestamp to provide compatibility with these systems.
    spark.sql.parquet.cacheMetadatatrue - Turns on caching of Parquet schema metadata. Can speed up querying of static data. -
    spark.sql.parquet.compression.codecgzipsnappy Sets the compression codec use when writing Parquet files. Acceptable values include: uncompressed, snappy, gzip, lzo. @@ -1403,43 +947,24 @@ Configuration of Parquet can be done using the `setConf` method on `SQLContext`
    spark.sql.parquet.output.committer.classorg.apache.parquet.hadoop.
    ParquetOutputCommitter
    spark.sql.parquet.mergeSchemafalse

    - The output committer class used by Parquet. The specified class needs to be a subclass of - org.apache.hadoop.
    mapreduce.OutputCommitter
    . Typically, it's also a - subclass of org.apache.parquet.hadoop.ParquetOutputCommitter. -

    -

    - Note: -

      -
    • - This option is automatically ignored if spark.speculation is turned on. -
    • -
    • - This option must be set via Hadoop Configuration rather than Spark - SQLConf. -
    • -
    • - This option overrides spark.sql.sources.
      outputCommitterClass
      . -
    • -
    -

    -

    - Spark SQL comes with a builtin - org.apache.spark.sql.
    parquet.DirectParquetOutputCommitter
    , which can be more - efficient then the default Parquet output committer when writing data to S3. + When true, the Parquet data source merges schemas collected from all data files, otherwise the + schema is picked from the summary file or a random data file if no summary file is available.

    spark.sql.parquet.mergeSchemafalsespark.sql.optimizer.metadataOnlytrue

    - When true, the Parquet data source merges schemas collected from all data files, otherwise the - schema is picked from the summary file or a random data file if no summary file is available. + When true, enable the metadata-only query optimization that use the table's metadata to + produce the partition columns instead of table scans. It applies when all the columns scanned + are partition columns and the query has an aggregate operator that satisfies distinct + semantics.

    + + + + + -# Queries can be expressed in HiveQL. -results <- collect(sql(sqlContext, "FROM src SELECT key, value")) + + + + -{% endhighlight %} + + + + - - + + + + +
    Property NameMeaning
    fileFormat + A fileFormat is kind of a package of storage format specifications, including "serde", "input format" and + "output format". Currently we support 6 fileFormats: 'sequencefile', 'rcfile', 'orc', 'parquet', 'textfile' and 'avro'. +
    inputFormat, outputFormat + These 2 options specify the name of a corresponding `InputFormat` and `OutputFormat` class as a string literal, + e.g. `org.apache.hadoop.hive.ql.io.orc.OrcInputFormat`. These 2 options must be appeared in pair, and you can not + specify them if you already specified the `fileFormat` option. +
    serde + This option specifies the name of a serde class. When the `fileFormat` option is specified, do not specify this option + if the given `fileFormat` already include the information of serde. Currently "sequencefile", "textfile" and "rcfile" + don't include the serde information and you can use this option with these 3 fileFormats. +
    fieldDelim, escapeDelim, collectionDelim, mapkeyDelim, lineDelim + These options can only be used with "textfile" fileFormat. They define how to read delimited files into rows. +
    + +All other properties defined with `OPTIONS` will be regarded as Hive serde properties. ### Interacting with Different Versions of Hive Metastore @@ -1739,16 +1168,16 @@ The following options can be used to configure the version of Hive that is used property can be one of three options:
    1. builtin
    2. - Use Hive 1.2.1, which is bundled with the Spark assembly jar when -Phive is + Use Hive 1.2.1, which is bundled with the Spark assembly when -Phive is enabled. When this option is chosen, spark.sql.hive.metastore.version must be either 1.2.1 or not defined.
    3. maven
    4. - Use Hive jars of specified version downloaded from Maven repositories. This configuration + Use Hive jars of specified version downloaded from Maven repositories. This configuration is not generally recommended for production deployments. -
    5. A classpath in the standard format for the JVM. This classpath must include all of Hive - and its dependencies, including the correct version of Hadoop. These jars only need to be +
    6. A classpath in the standard format for the JVM. This classpath must include all of Hive + and its dependencies, including the correct version of Hadoop. These jars only need to be present on the driver, but if you are running in yarn cluster mode then you must ensure - they are packaged with you application.
    7. + they are packaged with your application.
    @@ -1781,7 +1210,7 @@ The following options can be used to configure the version of Hive that is used ## JDBC To Other Databases -Spark SQL also includes a data source that can read data from other databases using JDBC. This +Spark SQL also includes a data source that can read data from other databases using JDBC. This functionality should be preferred over using [JdbcRDD](api/scala/index.html#org.apache.spark.rdd.JdbcRDD). This is because the results are returned as a DataFrame and they can easily be processed in Spark SQL or joined with other data sources. @@ -1791,29 +1220,33 @@ provide a ClassTag. run queries using Spark SQL). To get started you will need to include the JDBC driver for you particular database on the -spark classpath. For example, to connect to postgres from the Spark Shell you would run the +spark classpath. For example, to connect to postgres from the Spark Shell you would run the following command: {% highlight bash %} -SPARK_CLASSPATH=postgresql-9.3-1102-jdbc41.jar bin/spark-shell +bin/spark-shell --driver-class-path postgresql-9.4.1207.jar --jars postgresql-9.4.1207.jar {% endhighlight %} -Tables from the remote database can be loaded as a DataFrame or Spark SQL Temporary table using -the Data Sources API. The following options are supported: +Tables from the remote database can be loaded as a DataFrame or Spark SQL temporary view using +the Data Sources API. Users can specify the JDBC connection properties in the data source options. +user and password are normally provided as connection properties for +logging into the data sources. In addition to the connection properties, Spark also supports +the following case-insensitive options: + @@ -1821,81 +1254,123 @@ the Data Sources API. The following options are supported: + - + -
    Property NameMeaning
    url - The JDBC URL to connect to. + The JDBC URL to connect to. The source-specific connection properties may be specified in the URL. e.g., jdbc:postgresql://localhost/test?user=fred&password=secret
    dbtable - The JDBC table that should be read. Note that anything that is valid in a FROM clause of - a SQL query can be used. For example, instead of a full table you could also use a + The JDBC table that should be read. Note that anything that is valid in a FROM clause of + a SQL query can be used. For example, instead of a full table you could also use a subquery in parentheses.
    driver - The class name of the JDBC driver needed to connect to this URL. This class will be loaded - on the master and workers before running an JDBC commands to allow the driver to - register itself with the JDBC subsystem. + The class name of the JDBC driver to use to connect to this URL.
    partitionColumn, lowerBound, upperBound, numPartitionspartitionColumn, lowerBound, upperBound - These options must all be specified if any of them is specified. They describe how to - partition the table when reading in parallel from multiple workers. + These options must all be specified if any of them is specified. In addition, + numPartitions must be specified. They describe how to partition the table when + reading in parallel from multiple workers. partitionColumn must be a numeric column from the table in question. Notice that lowerBound and upperBound are just used to decide the partition stride, not for filtering the rows in table. So all rows in the table will be - partitioned and returned. + partitioned and returned. This option applies only to reading.
    - -
    -
    - -{% highlight scala %} -val jdbcDF = sqlContext.read.format("jdbc").options( - Map("url" -> "jdbc:postgresql:dbserver", - "dbtable" -> "schema.tablename")).load() -{% endhighlight %} + + numPartitions + + The maximum number of partitions that can be used for parallelism in table reading and + writing. This also determines the maximum number of concurrent JDBC connections. + If the number of partitions to write exceeds this limit, we decrease it to this limit by + calling coalesce(numPartitions) before writing. + + -
    + + fetchsize + + The JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (eg. Oracle with 10 rows). This option applies only to reading. + + -
    + + batchsize + + The JDBC batch size, which determines how many rows to insert per round trip. This can help performance on JDBC drivers. This option applies only to writing. It defaults to 1000. + + -{% highlight java %} + + isolationLevel + + The transaction isolation level, which applies to current connection. It can be one of NONE, READ_COMMITTED, READ_UNCOMMITTED, REPEATABLE_READ, or SERIALIZABLE, corresponding to standard transaction isolation levels defined by JDBC's Connection object, with default of READ_UNCOMMITTED. This option applies only to writing. Please refer the documentation in java.sql.Connection. + + -Map options = new HashMap(); -options.put("url", "jdbc:postgresql:dbserver"); -options.put("dbtable", "schema.tablename"); + + sessionInitStatement + + After each database session is opened to the remote DB and before starting to read data, this option executes a custom SQL statement (or a PL/SQL block). Use this to implement session initialization code. Example: option("sessionInitStatement", """BEGIN execute immediate 'alter session set "_serial_direct_read"=true'; END;""") + + -DataFrame jdbcDF = sqlContext.read().format("jdbc"). options(options).load(); -{% endhighlight %} + + truncate + + This is a JDBC writer related option. When SaveMode.Overwrite is enabled, this option causes Spark to truncate an existing table instead of dropping and recreating it. This can be more efficient, and prevents the table metadata (e.g., indices) from being removed. However, it will not work in some cases, such as when the new data has a different schema. It defaults to false. This option applies only to writing. + + + + createTableOptions + + This is a JDBC writer related option. If specified, this option allows setting of database-specific table and partition options when creating a table (e.g., CREATE TABLE t (name string) ENGINE=InnoDB.). This option applies only to writing. + + -
    + + createTableColumnTypes + + The database column data types to use instead of the defaults, when creating the table. Data type information should be specified in the same format as CREATE TABLE columns syntax (e.g: "name CHAR(64), comments VARCHAR(1024)"). The specified types should be valid spark sql data types. This option applies only to writing. + + -
    + + customSchema + + The custom schema to use for reading data from JDBC connectors. For example, "id DECIMAL(38, 0), name STRING". You can also specify partial fields, and the others use the default type mapping. For example, "id DECIMAL(38, 0)". The column names should be identical to the corresponding column names of JDBC table. Users can specify the corresponding data types of Spark SQL instead of using the defaults. This option applies only to reading. + + + -{% highlight python %} +
    -df = sqlContext.read.format('jdbc').options(url='jdbc:postgresql:dbserver', dbtable='schema.tablename').load() +
    +{% include_example jdbc_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    -{% endhighlight %} +
    +{% include_example jdbc_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    +
    +{% include_example jdbc_dataset python/sql/datasource.py %}
    - -{% highlight r %} - -df <- loadDF(sqlContext, source="jdbc", url="jdbc:postgresql:dbserver", dbtable="schema.tablename") - -{% endhighlight %} - +{% include_example jdbc_dataset r/RSparkSQLExample.R %}
    {% highlight sql %} -CREATE TEMPORARY TABLE jdbcTable +CREATE TEMPORARY VIEW jdbcTable USING org.apache.spark.sql.jdbc OPTIONS ( url "jdbc:postgresql:dbserver", - dbtable "schema.tablename" + dbtable "schema.tablename", + user 'username', + password 'password' ) +INSERT INTO TABLE jdbcTable +SELECT * FROM resultTable {% endhighlight %}
    @@ -1914,11 +1389,11 @@ 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 `dataFrame.cache()`. +Spark SQL can cache tables using an in-memory columnar format by calling `spark.catalog.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. +memory usage and GC pressure. You can call `spark.catalog.uncacheTable("tableName")` to remove the table from memory. -Configuration of in-memory caching can be done using the `setConf` method on `SQLContext` or by running +Configuration of in-memory caching can be done using the `setConf` method on `SparkSession` or by running `SET key=value` commands using SQL. @@ -1935,7 +1410,7 @@ Configuration of in-memory caching can be done using the `setConf` method on `SQ @@ -1944,29 +1419,47 @@ Configuration of in-memory caching can be done using the `setConf` method on `SQ ## Other Configuration Options -The following options can also be used to tune the performance of query execution. It is possible +The following options can also be used to tune the performance of query execution. It is possible that these options will be deprecated in future release as more optimizations are performed automatically.
    spark.sql.inMemoryColumnarStorage.batchSize 10000 - Controls the size of batches for columnar caching. Larger batch sizes can improve memory utilization + Controls the size of batches for columnar caching. Larger batch sizes can improve memory utilization and compression, but risk OOMs when caching data.
    + + + + + + + + + + + + + + + - - - - - @@ -1992,8 +1485,8 @@ To start the JDBC/ODBC server, run the following in the Spark directory: ./sbin/start-thriftserver.sh This script accepts all `bin/spark-submit` command line options, plus a `--hiveconf` option to -specify Hive properties. You may run `./sbin/start-thriftserver.sh --help` for a complete list of -all available options. By default, the server listens on localhost:10000. You may override this +specify Hive properties. You may run `./sbin/start-thriftserver.sh --help` for a complete list of +all available options. By default, the server listens on localhost:10000. You may override this behaviour via either environment variables, i.e.: {% highlight bash %} @@ -2026,7 +1519,7 @@ Beeline will ask you for a username and password. In non-secure mode, simply ent your machine and a blank password. For secure mode, please follow the instructions given in the [beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients). -Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. +Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `conf/`. You may also use the beeline script that comes with Hive. @@ -2034,7 +1527,7 @@ Thrift JDBC server also supports sending thrift RPC messages over HTTP transport Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: hive.server2.transport.mode - Set this to value: http - hive.server2.thrift.http.port - HTTP port number fo listen on; default is 10001 + hive.server2.thrift.http.port - HTTP port number to listen on; default is 10001 hive.server2.http.endpoint - HTTP endpoint; default is cliservice To test, use beeline to connect to the JDBC/ODBC server in http mode with: @@ -2051,39 +1544,105 @@ To start the Spark SQL CLI, run the following in the Spark directory: ./bin/spark-sql -Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. +Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `conf/`. You may run `./bin/spark-sql --help` for a complete list of all available options. # Migration Guide +## Upgrading From Spark SQL 2.2 to 2.3 + + - Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the referenced columns only include the internal corrupt record column (named `_corrupt_record` by default). For example, `spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()` and `spark.read.schema(schema).json(file).select("_corrupt_record").show()`. Instead, you can cache or save the parsed results and then send the same query. For example, `val df = spark.read.schema(schema).json(file).cache()` and then `df.filter($"_corrupt_record".isNotNull).count()`. + - The `percentile_approx` function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles. + +## Upgrading From Spark SQL 2.1 to 2.2 + + - Spark 2.1.1 introduced a new configuration key: `spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of `NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting's default value to `INFER_AND_SAVE` to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the `INFER_AND_SAVE` configuration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to avoid the initial overhead of schema inference. Note that with the new default `INFER_AND_SAVE` setting, the results of the schema inference are saved as a metastore key for future use. Therefore, the initial schema inference occurs only at a table's first access. + +## Upgrading From Spark SQL 2.0 to 2.1 + + - Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. + - Legacy datasource tables can be migrated to this format via the `MSCK REPAIR TABLE` command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance. + - To determine if a table has been migrated, look for the `PartitionProvider: Catalog` attribute when issuing `DESCRIBE FORMATTED` on the table. + - Changes to `INSERT OVERWRITE TABLE ... PARTITION ...` behavior for Datasource tables. + - In prior Spark versions `INSERT OVERWRITE` overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten. + - Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data. + +## Upgrading From Spark SQL 1.6 to 2.0 + + - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and + `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new `catalog` interface is accessible from `SparkSession` - existing API on databases and tables access such as `listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved here. + + - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for + `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed + transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g., + `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in + Python and R is not a language feature, the concept of Dataset does not apply to these languages’ + APIs. Instead, `DataFrame` remains the primary programing abstraction, which is analogous to the + single-node data frame notion in these languages. + + - Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union` + - Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap` + - Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView` + + - Changes to `CREATE TABLE ... LOCATION` behavior for Hive tables. + - From Spark 2.0, `CREATE TABLE ... LOCATION` is equivalent to `CREATE EXTERNAL TABLE ... LOCATION` + in order to prevent accidental dropping the existing data in the user-provided locations. + That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table. + Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables. + Note that this is different from the Hive behavior. + - As a result, `DROP TABLE` statements on those tables will not remove the data. + + - `spark.sql.parquet.cacheMetadata` is no longer used. + See [SPARK-13664](https://issues.apache.org/jira/browse/SPARK-13664) for details. + +## Upgrading From Spark SQL 1.5 to 1.6 + + - From Spark 1.6, by default the Thrift server runs in multi-session mode. Which means each JDBC/ODBC + connection owns a copy of their own SQL configuration and temporary function registry. Cached + tables are still shared though. If you prefer to run the Thrift server in the old single-session + mode, please set option `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add + this option to `spark-defaults.conf`, or pass it to `start-thriftserver.sh` via `--conf`: + + {% highlight bash %} + ./sbin/start-thriftserver.sh \ + --conf spark.sql.hive.thriftServer.singleSession=true \ + ... + {% endhighlight %} + - Since 1.6.1, withColumn method in sparkR supports adding a new column to or replacing existing columns + of the same name of a DataFrame. + + - From Spark 1.6, LongType casts to TimestampType expect seconds instead of microseconds. This + change was made to match the behavior of Hive 1.2 for more consistent type casting to TimestampType + from numeric types. See [SPARK-11724](https://issues.apache.org/jira/browse/SPARK-11724) for + details. + ## Upgrading From Spark SQL 1.4 to 1.5 - Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with - code generation for expression evaluation. These features can both be disabled by setting + code generation for expression evaluation. These features can both be disabled by setting `spark.sql.tungsten.enabled` to `false`. - - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting + - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting `spark.sql.parquet.mergeSchema` to `true`. - Resolution of strings to columns in python now supports using dots (`.`) to qualify the column or - access nested values. For example `df['table.column.nestedField']`. However, this means that if - your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). + access nested values. For example `df['table.column.nestedField']`. However, this means that if + your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). - In-memory columnar storage partition pruning is on by default. It can be disabled by setting `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`. - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum - precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now + precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`. - Timestamps are now stored at a precision of 1us, rather than 1ns - - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains + - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains unchanged. - - The canonical name of SQL/DataFrame functions are now lower case (e.g. sum vs SUM). - - It has been determined that using the DirectOutputCommitter when speculation is enabled is unsafe - and thus this output committer will not be used when speculation is on, independent of configuration. + - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM). - JSON data source will not automatically load new files that are created by other applications (i.e. files that are not inserted to the dataset through Spark SQL). For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` method to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate the DataFrame and the new DataFrame will include new files. + - DataFrame.withColumn method in pySpark supports adding a new column or replacing existing columns of the same name. ## Upgrading from Spark SQL 1.3 to 1.4 @@ -2091,7 +1650,7 @@ options. Based on user feedback, we created a new, more fluid API for reading data in (`SQLContext.read`) and writing data out (`DataFrame.write`), -and deprecated the old APIs (e.g. `SQLContext.parquetFile`, `SQLContext.jsonFile`). +and deprecated the old APIs (e.g., `SQLContext.parquetFile`, `SQLContext.jsonFile`). See the API docs for `SQLContext.read` ( Scala, @@ -2149,7 +1708,7 @@ import pyspark.sql.functions as func # In 1.3.x, in order for the grouping column "department" to show up, # it must be included explicitly as part of the agg function call. -df.groupBy("department").agg("department"), func.max("age"), func.sum("expense")) +df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense")) # In 1.4+, grouping column "department" is included automatically. df.groupBy("department").agg(func.max("age"), func.sum("expense")) @@ -2163,41 +1722,51 @@ sqlContext.setConf("spark.sql.retainGroupColumns", "false") +#### Behavior change on DataFrame.withColumn + +Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column will always be added +as a new column with its specified name in the result DataFrame even if there may be any existing +columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a column of a different +name from names of all existing columns or replacing existing columns of the same name. + +Note that this change is only for Scala API, not for PySpark and SparkR. + + ## Upgrading from Spark SQL 1.0-1.2 to 1.3 In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the -available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other -releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked +available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other +releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked as unstable (i.e., DeveloperAPI or Experimental). #### Rename of SchemaRDD to DataFrame The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has -been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD +been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD directly, but instead provide most of the functionality that RDDs provide though their own -implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method. +implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method. In Scala there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for -some use cases. It is still recommended that users update their code to use `DataFrame` instead. +some use cases. It is still recommended that users update their code to use `DataFrame` instead. Java and Python users will need to update their code. #### Unification of the Java and Scala APIs Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`) -that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users -of either language should use `SQLContext` and `DataFrame`. In general theses classes try to +that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users +of either language should use `SQLContext` and `DataFrame`. In general theses classes try to use types that are usable from both languages (i.e. `Array` instead of language specific collections). In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading is used instead. -Additionally the Java specific types API has been removed. Users of both Scala and Java should +Additionally the Java specific types API has been removed. Users of both Scala and Java should use the classes present in `org.apache.spark.sql.types` to describe schema programmatically. #### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only) Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought -all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit +all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`. Users should now write `import sqlContext.implicits._`. @@ -2205,7 +1774,7 @@ Additionally, the implicit conversions now only augment RDDs that are composed o case classes or tuples) with a method `toDF`, instead of applying automatically. When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import -`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used: +`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used: `import org.apache.spark.sql.functions._`. #### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only) @@ -2244,57 +1813,12 @@ Python UDF registration is unchanged. When using DataTypes in Python you will need to construct them (i.e. `StringType()`) instead of referencing a singleton. -## Migration Guide for Shark Users - -### Scheduling -To set a [Fair Scheduler](job-scheduling.html#fair-scheduler-pools) pool for a JDBC client session, -users can set the `spark.sql.thriftserver.scheduler.pool` variable: - - SET spark.sql.thriftserver.scheduler.pool=accounting; - -### Reducer number - -In Shark, default reducer number is 1 and is controlled by the property `mapred.reduce.tasks`. Spark -SQL deprecates this property in favor of `spark.sql.shuffle.partitions`, whose default value -is 200. Users may customize this property via `SET`: - - SET spark.sql.shuffle.partitions=10; - SELECT page, count(*) c - FROM logs_last_month_cached - GROUP BY page ORDER BY c DESC LIMIT 10; - -You may also put this property in `hive-site.xml` to override the default value. - -For now, the `mapred.reduce.tasks` property is still recognized, and is converted to -`spark.sql.shuffle.partitions` automatically. - -### Caching - -The `shark.cache` table property no longer exists, and tables whose name end with `_cached` are no -longer automatically cached. Instead, we provide `CACHE TABLE` and `UNCACHE TABLE` statements to -let user control table caching explicitly: - - CACHE TABLE logs_last_month; - UNCACHE TABLE logs_last_month; - -**NOTE:** `CACHE TABLE tbl` is now __eager__ by default not __lazy__. Don’t need to trigger cache materialization manually anymore. - -Spark SQL newly introduced a statement to let user control table caching whether or not lazy since Spark 1.2.0: - - CACHE [LAZY] TABLE [AS SELECT] ... - -Several caching related features are not supported yet: - -* User defined partition level cache eviction policy -* RDD reloading -* In-memory cache write through policy - ## Compatibility with Apache Hive Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. Currently Hive SerDes and UDFs are based on Hive 1.2.1, and Spark SQL can be connected to different versions of Hive Metastore -(from 0.12.0 to 1.2.1. Also see http://spark.apache.org/docs/latest/sql-programming-guide.html#interacting-with-different-versions-of-hive-metastore). +(from 0.12.0 to 2.1.1. Also see [Interacting with Different Versions of Hive Metastore] (#interacting-with-different-versions-of-hive-metastore)). #### Deploying in Existing Hive Warehouses @@ -2395,6 +1919,23 @@ releases of Spark SQL. Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS metadata. Spark SQL does not support that. +**Hive UDF/UDTF/UDAF** + +Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below are the unsupported APIs: + +* `getRequiredJars` and `getRequiredFiles` (`UDF` and `GenericUDF`) are functions to automatically + include additional resources required by this UDF. +* `initialize(StructObjectInspector)` in `GenericUDTF` is not supported yet. Spark SQL currently uses + a deprecated interface `initialize(ObjectInspector[])` only. +* `configure` (`GenericUDF`, `GenericUDTF`, and `GenericUDAFEvaluator`) is a function to initialize + functions with `MapredContext`, which is inapplicable to Spark. +* `close` (`GenericUDF` and `GenericUDAFEvaluator`) is a function to release associated resources. + Spark SQL does not call this function when tasks finish. +* `reset` (`GenericUDAFEvaluator`) is a function to re-initialize aggregation for reusing the same aggregation. + Spark SQL currently does not support the reuse of aggregation. +* `getWindowingEvaluator` (`GenericUDAFEvaluator`) is a function to optimize aggregation by evaluating + an aggregate over a fixed window. + # Reference ## Data Types @@ -2444,9 +1985,8 @@ Spark SQL and DataFrames support the following data types: All data types of Spark SQL are located in the package `org.apache.spark.sql.types`. You can access them by doing -{% highlight scala %} -import org.apache.spark.sql.types._ -{% endhighlight %} + +{% include_example data_types scala/org/apache/spark/examples/sql/SparkSQLExample.scala %}
    Property NameDefaultMeaning
    spark.sql.files.maxPartitionBytes134217728 (128 MB) + The maximum number of bytes to pack into a single partition when reading files. +
    spark.sql.files.openCostInBytes4194304 (4 MB) + The estimated cost to open a file, measured by the number of bytes could be scanned in the same + time. This is used when putting multiple files into a partition. It is better to over estimated, + then the partitions with small files will be faster than partitions with bigger files (which is + scheduled first). +
    spark.sql.broadcastTimeout300 +

    + Timeout in seconds for the broadcast wait time in broadcast joins +

    +
    spark.sql.autoBroadcastJoinThreshold 10485760 (10 MB) Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when - performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently + performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently statistics are only supported for Hive Metastore tables where the command ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan has been run.
    spark.sql.tungsten.enabledtrue - When true, use the optimized Tungsten physical execution backend which explicitly manages memory - and dynamically generates bytecode for expression evaluation. -
    spark.sql.shuffle.partitions 200
    @@ -2567,7 +2107,8 @@ import org.apache.spark.sql.types._
    The value type in Scala of the data type of this field (For example, Int for a StructField with the data type IntegerType) - StructField(name, dataType, nullable) + StructField(name, dataType, [nullable])
    + Note: The default value of nullable is true.
    @@ -2855,7 +2396,8 @@ from pyspark.sql.types import * The value type in Python of the data type of this field (For example, Int for a StructField with the data type IntegerType) - StructField(name, dataType, nullable) + StructField(name, dataType, [nullable])
    + Note: The default value of nullable is True. @@ -2976,7 +2518,7 @@ from pyspark.sql.types import * vector or list list(type="array", elementType=elementType, containsNull=[containsNull])
    - Note: The default value of containsNull is True. + Note: The default value of containsNull is TRUE. @@ -2984,7 +2526,7 @@ from pyspark.sql.types import * environment list(type="map", keyType=keyType, valueType=valueType, valueContainsNull=[valueContainsNull])
    - Note: The default value of valueContainsNull is True. + Note: The default value of valueContainsNull is TRUE. @@ -3001,7 +2543,8 @@ from pyspark.sql.types import * The value type in R of the data type of this field (For example, integer for a StructField with the data type IntegerType) - list(name=name, type=dataType, nullable=nullable) + list(name=name, type=dataType, nullable=[nullable])
    + Note: The default value of nullable is TRUE. diff --git a/docs/storage-openstack-swift.md b/docs/storage-openstack-swift.md index c39ef1ce59e1..f4bb2353e3c4 100644 --- a/docs/storage-openstack-swift.md +++ b/docs/storage-openstack-swift.md @@ -8,7 +8,8 @@ same URI formats as in Hadoop. You can specify a path in Swift as input through URI of the form swift://container.PROVIDER/path. You will also need to set your Swift security credentials, through core-site.xml or via SparkContext.hadoopConfiguration. -Current Swift driver requires Swift to use Keystone authentication method. +The current Swift driver requires Swift to use the Keystone authentication method, or +its Rackspace-specific predecessor. # Configuring Swift for Better Data Locality @@ -19,41 +20,30 @@ Although not mandatory, it is recommended to configure the proxy server of Swift # Dependencies -The Spark application should include hadoop-openstack dependency. +The Spark application should include hadoop-openstack dependency, which can +be done by including the `hadoop-cloud` module for the specific version of spark used. For example, for Maven support, add the following to the pom.xml file: {% highlight xml %} ... - org.apache.hadoop - hadoop-openstack - 2.3.0 + org.apache.spark + hadoop-cloud_2.11 + ${spark.version} ... {% endhighlight %} - # Configuration Parameters Create core-site.xml and place it inside Spark's conf directory. -There are two main categories of parameters that should to be configured: declaration of the -Swift driver and the parameters that are required by Keystone. +The main category of parameters that should be configured are the authentication parameters +required by Keystone. -Configuration of Hadoop to use Swift File system achieved via - - - - - - - -
    Property NameValue
    fs.swift.implorg.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem
    - -Additional parameters required by Keystone (v2.0) and should be provided to the Swift driver. Those -parameters will be used to perform authentication in Keystone to access Swift. The following table -contains a list of Keystone mandatory parameters. PROVIDER can be any name. +The following table contains a list of Keystone mandatory parameters. PROVIDER can be +any (alphanumeric) name. @@ -94,7 +84,7 @@ contains a list of Keystone mandatory parameters. PROVIDER can be a - +
    Property NameMeaningRequired
    fs.swift.service.PROVIDER.publicIndicates if all URLs are publicIndicates whether to use the public (off cloud) or private (in cloud; no transfer fees) endpoints Mandatory
    @@ -104,10 +94,6 @@ defined for tenant test. Then core-site.xml should inc {% highlight xml %} - - fs.swift.impl - org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem - fs.swift.service.SparkTest.auth.url http://127.0.0.1:5000/v2.0/tokens diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index a75587a92adc..44ae52e81cd6 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -36,7 +36,7 @@ Any exception in the receiving threads should be caught and handled properly to failures of the receiver. `restart()` will restart the receiver by asynchronously calling `onStop()` and then calling `onStart()` after a delay. `stop()` will call `onStop()` and terminate the receiver. Also, `reportError()` -reports a error message to the driver (visible in the logs and UI) without stopping / restarting +reports an error message to the driver (visible in the logs and UI) without stopping / restarting the receiver. The following is a custom receiver that receives a stream of text over a socket. It treats @@ -59,8 +59,8 @@ class CustomReceiver(host: String, port: Int) } def onStop() { - // There is nothing much to do as the thread calling receive() - // is designed to stop by itself if isStopped() returns false + // There is nothing much to do as the thread calling receive() + // is designed to stop by itself if isStopped() returns false } /** Create a socket connection and receive data until receiver is stopped */ @@ -68,28 +68,29 @@ class CustomReceiver(host: String, port: Int) var socket: Socket = null var userInput: String = null try { - // Connect to host:port - socket = new Socket(host, port) - - // Until stopped or connection broken continue reading - val reader = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8")) - userInput = reader.readLine() - while(!isStopped && userInput != null) { - store(userInput) - userInput = reader.readLine() - } - reader.close() - socket.close() - - // Restart in an attempt to connect again when server is active again - restart("Trying to connect again") + // Connect to host:port + socket = new Socket(host, port) + + // Until stopped or connection broken continue reading + val reader = new BufferedReader( + new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8)) + userInput = reader.readLine() + while(!isStopped && userInput != null) { + store(userInput) + userInput = reader.readLine() + } + reader.close() + socket.close() + + // Restart in an attempt to connect again when server is active again + restart("Trying to connect again") } catch { - case e: java.net.ConnectException => - // restart if could not connect to server - restart("Error connecting to " + host + ":" + port, e) - case t: Throwable => - // restart if there is any other error - restart("Error receiving data", t) + case e: java.net.ConnectException => + // restart if could not connect to server + restart("Error connecting to " + host + ":" + port, e) + case t: Throwable => + // restart if there is any other error + restart("Error receiving data", t) } } } @@ -112,15 +113,13 @@ public class JavaCustomReceiver extends Receiver { port = port_; } + @Override public void onStart() { // Start the thread that receives data over a connection - new Thread() { - @Override public void run() { - receive(); - } - }.start(); + new Thread(this::receive).start(); } + @Override public void onStop() { // There is nothing much to do as the thread calling receive() // is designed to stop by itself if isStopped() returns false @@ -135,7 +134,8 @@ public class JavaCustomReceiver extends Receiver { // connect to the server socket = new Socket(host, port); - BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream())); + BufferedReader reader = new BufferedReader( + new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8)); // Until stopped or connection broken continue reading while (!isStopped() && (userInput = reader.readLine()) != null) { @@ -175,11 +175,11 @@ an input DStream using data received by the instance of custom receiver, as show {% highlight scala %} // Assuming ssc is the StreamingContext val customReceiverStream = ssc.receiverStream(new CustomReceiver(host, port)) -val words = lines.flatMap(_.split(" ")) +val words = customReceiverStream.flatMap(_.split(" ")) ... {% endhighlight %} -The full source code is in the example [CustomReceiver.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala). +The full source code is in the example [CustomReceiver.scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala).
    @@ -187,11 +187,11 @@ The full source code is in the example [CustomReceiver.scala](https://github.com {% highlight java %} // Assuming ssc is the JavaStreamingContext JavaDStream customReceiverStream = ssc.receiverStream(new JavaCustomReceiver(host, port)); -JavaDStream words = lines.flatMap(new FlatMapFunction() { ... }); +JavaDStream words = customReceiverStream.flatMap(s -> ...); ... {% endhighlight %} -The full source code is in the example [JavaCustomReceiver.java](https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java). +The full source code is in the example [JavaCustomReceiver.java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java).
    @@ -254,28 +254,3 @@ The following table summarizes the characteristics of both types of receivers - -## Implementing and Using a Custom Actor-based Receiver - -Custom [Akka Actors](http://doc.akka.io/docs/akka/2.2.4/scala/actors.html) can also be used to -receive data. The [`ActorHelper`](api/scala/index.html#org.apache.spark.streaming.receiver.ActorHelper) -trait can be applied on any Akka actor, which allows received data to be stored in Spark using - `store(...)` methods. The supervisor strategy of this actor can be configured to handle failures, etc. - -{% highlight scala %} -class CustomActor extends Actor with ActorHelper { - def receive = { - case data: String => store(data) - } -} -{% endhighlight %} - -And a new input stream can be created with this custom actor as - -{% highlight scala %} -// Assuming ssc is the StreamingContext -val lines = ssc.actorStream[String](Props(new CustomActor()), "CustomReceiver") -{% endhighlight %} - -See [ActorWordCount.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala) -for an end-to-end example. diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md index 383d954409ce..a5d36da5b6de 100644 --- a/docs/streaming-flume-integration.md +++ b/docs/streaming-flume-integration.md @@ -30,7 +30,7 @@ See the [Flume's documentation](https://flume.apache.org/documentation.html) for configuring Flume agents. #### Configuring Spark Streaming Application -1. **Linking:** In your SBT/Maven projrect definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). +1. **Linking:** In your SBT/Maven project definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). groupId = org.apache.spark artifactId = spark-streaming-flume_{{site.SCALA_BINARY_VERSION}} @@ -63,7 +63,7 @@ configuring Flume agents. By default, the Python API will decode Flume event body as UTF8 encoded strings. You can specify your custom decoding function to decode the body byte arrays in Flume events to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.flume.FlumeUtils) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/flume_wordcount.py). + and the [example]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/streaming/flume_wordcount.py).
    @@ -71,7 +71,16 @@ configuring Flume agents. cluster (Mesos, YARN or Spark Standalone), so that resource allocation can match the names and launch the receiver in the right machine. -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). +3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications. + + For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + + For Python applications which lack SBT/Maven project management, `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is, + + ./bin/spark-submit --packages org.apache.spark:spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... + + Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-flume-assembly` from the + [Maven repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-flume-assembly_{{site.SCALA_BINARY_VERSION}}%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`. ## 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. @@ -106,11 +115,11 @@ Configuring Flume on the chosen machine requires the following two steps. artifactId = scala-library version = {{site.SCALA_VERSION}} - (iii) *Commons Lang 3 JAR*: Download the Commons Lang 3 JAR. It can be found with the following artifact detail (or, [direct link](http://search.maven.org/remotecontent?filepath=org/apache/commons/commons-lang3/3.3.2/commons-lang3-3.3.2.jar)). + (iii) *Commons Lang 3 JAR*: Download the Commons Lang 3 JAR. It can be found with the following artifact detail (or, [direct link](http://search.maven.org/remotecontent?filepath=org/apache/commons/commons-lang3/3.5/commons-lang3-3.5.jar)). groupId = org.apache.commons artifactId = commons-lang3 - version = 3.3.2 + version = 3.5 2. **Configuration file**: On that machine, configure Flume agent to send data to an Avro sink by having the following in the configuration file. @@ -157,7 +166,7 @@ configuring Flume agents. Note that each input DStream can be configured to receive data from multiple sinks. -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). +3. **Deploying:** This is same as the first approach. diff --git a/docs/streaming-kafka-0-10-integration.md b/docs/streaming-kafka-0-10-integration.md new file mode 100644 index 000000000000..386066a85749 --- /dev/null +++ b/docs/streaming-kafka-0-10-integration.md @@ -0,0 +1,314 @@ +--- +layout: global +title: Spark Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher) +--- + +The Spark Streaming integration for Kafka 0.10 is similar in design to the 0.8 [Direct Stream approach](streaming-kafka-0-8-integration.html#approach-2-direct-approach-no-receivers). It provides simple parallelism, 1:1 correspondence between Kafka partitions and Spark partitions, and access to offsets and metadata. However, because the newer integration uses the [new Kafka consumer API](http://kafka.apache.org/documentation.html#newconsumerapi) instead of the simple API, there are notable differences in usage. This version of the integration is marked as experimental, so the API is potentially subject to change. + +### Linking +For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +**Do not** manually add dependencies on `org.apache.kafka` artifacts (e.g. `kafka-clients`). The `spark-streaming-kafka-0-10` artifact has the appropriate transitive dependencies already, and different versions may be incompatible in hard to diagnose ways. + +### Creating a Direct Stream + Note that the namespace for the import includes the version, org.apache.spark.streaming.kafka010 + +
    +
    +{% highlight scala %} +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.common.serialization.StringDeserializer +import org.apache.spark.streaming.kafka010._ +import org.apache.spark.streaming.kafka010.LocationStrategies.PreferConsistent +import org.apache.spark.streaming.kafka010.ConsumerStrategies.Subscribe + +val kafkaParams = Map[String, Object]( + "bootstrap.servers" -> "localhost:9092,anotherhost:9092", + "key.deserializer" -> classOf[StringDeserializer], + "value.deserializer" -> classOf[StringDeserializer], + "group.id" -> "use_a_separate_group_id_for_each_stream", + "auto.offset.reset" -> "latest", + "enable.auto.commit" -> (false: java.lang.Boolean) +) + +val topics = Array("topicA", "topicB") +val stream = KafkaUtils.createDirectStream[String, String]( + streamingContext, + PreferConsistent, + Subscribe[String, String](topics, kafkaParams) +) + +stream.map(record => (record.key, record.value)) +{% endhighlight %} +Each item in the stream is a [ConsumerRecord](http://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/ConsumerRecord.html) +
    +
    +{% highlight java %} +import java.util.*; +import org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.*; +import org.apache.spark.streaming.api.java.*; +import org.apache.spark.streaming.kafka010.*; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.StringDeserializer; +import scala.Tuple2; + +Map kafkaParams = new HashMap<>(); +kafkaParams.put("bootstrap.servers", "localhost:9092,anotherhost:9092"); +kafkaParams.put("key.deserializer", StringDeserializer.class); +kafkaParams.put("value.deserializer", StringDeserializer.class); +kafkaParams.put("group.id", "use_a_separate_group_id_for_each_stream"); +kafkaParams.put("auto.offset.reset", "latest"); +kafkaParams.put("enable.auto.commit", false); + +Collection topics = Arrays.asList("topicA", "topicB"); + +JavaInputDStream> stream = + KafkaUtils.createDirectStream( + streamingContext, + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Subscribe(topics, kafkaParams) + ); + +stream.mapToPair(record -> new Tuple2<>(record.key(), record.value())); +{% endhighlight %} +
    +
    + +For possible kafkaParams, see [Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs). +If your Spark batch duration is larger than the default Kafka heartbeat session timeout (30 seconds), increase heartbeat.interval.ms and session.timeout.ms appropriately. For batches larger than 5 minutes, this will require changing group.max.session.timeout.ms on the broker. +Note that the example sets enable.auto.commit to false, for discussion see [Storing Offsets](streaming-kafka-0-10-integration.html#storing-offsets) below. + +### LocationStrategies +The new Kafka consumer API will pre-fetch messages into buffers. Therefore it is important for performance reasons that the Spark integration keep cached consumers on executors (rather than recreating them for each batch), and prefer to schedule partitions on the host locations that have the appropriate consumers. + +In most cases, you should use `LocationStrategies.PreferConsistent` as shown above. This will distribute partitions evenly across available executors. If your executors are on the same hosts as your Kafka brokers, use `PreferBrokers`, which will prefer to schedule partitions on the Kafka leader for that partition. Finally, if you have a significant skew in load among partitions, use `PreferFixed`. This allows you to specify an explicit mapping of partitions to hosts (any unspecified partitions will use a consistent location). + +The cache for consumers has a default maximum size of 64. If you expect to be handling more than (64 * number of executors) Kafka partitions, you can change this setting via `spark.streaming.kafka.consumer.cache.maxCapacity`. + +If you would like to disable the caching for Kafka consumers, you can set `spark.streaming.kafka.consumer.cache.enabled` to `false`. Disabling the cache may be needed to workaround the problem described in SPARK-19185. This property may be removed in later versions of Spark, once SPARK-19185 is resolved. + +The cache is keyed by topicpartition and group.id, so use a **separate** `group.id` for each call to `createDirectStream`. + + +### ConsumerStrategies +The new Kafka consumer API has a number of different ways to specify topics, some of which require considerable post-object-instantiation setup. `ConsumerStrategies` provides an abstraction that allows Spark to obtain properly configured consumers even after restart from checkpoint. + +`ConsumerStrategies.Subscribe`, as shown above, allows you to subscribe to a fixed collection of topics. `SubscribePattern` allows you to use a regex to specify topics of interest. Note that unlike the 0.8 integration, using `Subscribe` or `SubscribePattern` should respond to adding partitions during a running stream. Finally, `Assign` allows you to specify a fixed collection of partitions. All three strategies have overloaded constructors that allow you to specify the starting offset for a particular partition. + +If you have specific consumer setup needs that are not met by the options above, `ConsumerStrategy` is a public class that you can extend. + +### Creating an RDD +If you have a use case that is better suited to batch processing, you can create an RDD for a defined range of offsets. + +
    +
    +{% highlight scala %} +// Import dependencies and create kafka params as in Create Direct Stream above + +val offsetRanges = Array( + // topic, partition, inclusive starting offset, exclusive ending offset + OffsetRange("test", 0, 0, 100), + OffsetRange("test", 1, 0, 100) +) + +val rdd = KafkaUtils.createRDD[String, String](sparkContext, kafkaParams, offsetRanges, PreferConsistent) +{% endhighlight %} +
    +
    +{% highlight java %} +// Import dependencies and create kafka params as in Create Direct Stream above + +OffsetRange[] offsetRanges = { + // topic, partition, inclusive starting offset, exclusive ending offset + OffsetRange.create("test", 0, 0, 100), + OffsetRange.create("test", 1, 0, 100) +}; + +JavaRDD> rdd = KafkaUtils.createRDD( + sparkContext, + kafkaParams, + offsetRanges, + LocationStrategies.PreferConsistent() +); +{% endhighlight %} +
    +
    + +Note that you cannot use `PreferBrokers`, because without the stream there is not a driver-side consumer to automatically look up broker metadata for you. Use `PreferFixed` with your own metadata lookups if necessary. + +### Obtaining Offsets + +
    +
    +{% highlight scala %} +stream.foreachRDD { rdd => + val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + rdd.foreachPartition { iter => + val o: OffsetRange = offsetRanges(TaskContext.get.partitionId) + println(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") + } +} +{% endhighlight %} +
    +
    +{% highlight java %} +stream.foreachRDD(rdd -> { + OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + rdd.foreachPartition(consumerRecords -> { + OffsetRange o = offsetRanges[TaskContext.get().partitionId()]; + System.out.println( + o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset()); + }); +}); +{% endhighlight %} +
    +
    + +Note that the typecast to `HasOffsetRanges` will only succeed if it is done in the first method called on the result of `createDirectStream`, not later down a chain of methods. Be aware that the one-to-one mapping between RDD partition and Kafka partition does not remain after any methods that shuffle or repartition, e.g. reduceByKey() or window(). + +### Storing Offsets +Kafka delivery semantics in the case of failure depend on how and when offsets are stored. Spark output operations are [at-least-once](streaming-programming-guide.html#semantics-of-output-operations). So if you want the equivalent of exactly-once semantics, you must either store offsets after an idempotent output, or store offsets in an atomic transaction alongside output. With this integration, you have 3 options, in order of increasing reliability (and code complexity), for how to store offsets. + +#### Checkpoints +If you enable Spark [checkpointing](streaming-programming-guide.html#checkpointing), offsets will be stored in the checkpoint. This is easy to enable, but there are drawbacks. Your output operation must be idempotent, since you will get repeated outputs; transactions are not an option. Furthermore, you cannot recover from a checkpoint if your application code has changed. For planned upgrades, you can mitigate this by running the new code at the same time as the old code (since outputs need to be idempotent anyway, they should not clash). But for unplanned failures that require code changes, you will lose data unless you have another way to identify known good starting offsets. + +#### Kafka itself +Kafka has an offset commit API that stores offsets in a special Kafka topic. By default, the new consumer will periodically auto-commit offsets. This is almost certainly not what you want, because messages successfully polled by the consumer may not yet have resulted in a Spark output operation, resulting in undefined semantics. This is why the stream example above sets "enable.auto.commit" to false. However, you can commit offsets to Kafka after you know your output has been stored, using the `commitAsync` API. The benefit as compared to checkpoints is that Kafka is a durable store regardless of changes to your application code. However, Kafka is not transactional, so your outputs must still be idempotent. + +
    +
    +{% highlight scala %} +stream.foreachRDD { rdd => + val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + + // some time later, after outputs have completed + stream.asInstanceOf[CanCommitOffsets].commitAsync(offsetRanges) +} +{% endhighlight %} +As with HasOffsetRanges, the cast to CanCommitOffsets will only succeed if called on the result of createDirectStream, not after transformations. The commitAsync call is threadsafe, but must occur after outputs if you want meaningful semantics. +
    +
    +{% highlight java %} +stream.foreachRDD(rdd -> { + OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + + // some time later, after outputs have completed + ((CanCommitOffsets) stream.inputDStream()).commitAsync(offsetRanges); +}); +{% endhighlight %} +
    +
    + +#### Your own data store +For data stores that support transactions, saving offsets in the same transaction as the results can keep the two in sync, even in failure situations. If you're careful about detecting repeated or skipped offset ranges, rolling back the transaction prevents duplicated or lost messages from affecting results. This gives the equivalent of exactly-once semantics. It is also possible to use this tactic even for outputs that result from aggregations, which are typically hard to make idempotent. + +
    +
    +{% highlight scala %} +// The details depend on your data store, but the general idea looks like this + +// begin from the the offsets committed to the database +val fromOffsets = selectOffsetsFromYourDatabase.map { resultSet => + new TopicPartition(resultSet.string("topic"), resultSet.int("partition")) -> resultSet.long("offset") +}.toMap + +val stream = KafkaUtils.createDirectStream[String, String]( + streamingContext, + PreferConsistent, + Assign[String, String](fromOffsets.keys.toList, kafkaParams, fromOffsets) +) + +stream.foreachRDD { rdd => + val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + + val results = yourCalculation(rdd) + + // begin your transaction + + // update results + // update offsets where the end of existing offsets matches the beginning of this batch of offsets + // assert that offsets were updated correctly + + // end your transaction +} +{% endhighlight %} +
    +
    +{% highlight java %} +// The details depend on your data store, but the general idea looks like this + +// begin from the the offsets committed to the database +Map fromOffsets = new HashMap<>(); +for (resultSet : selectOffsetsFromYourDatabase) + fromOffsets.put(new TopicPartition(resultSet.string("topic"), resultSet.int("partition")), resultSet.long("offset")); +} + +JavaInputDStream> stream = KafkaUtils.createDirectStream( + streamingContext, + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Assign(fromOffsets.keySet(), kafkaParams, fromOffsets) +); + +stream.foreachRDD(rdd -> { + OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + + Object results = yourCalculation(rdd); + + // begin your transaction + + // update results + // update offsets where the end of existing offsets matches the beginning of this batch of offsets + // assert that offsets were updated correctly + + // end your transaction +}); +{% endhighlight %} +
    +
    + +### SSL / TLS +The new Kafka consumer [supports SSL](http://kafka.apache.org/documentation.html#security_ssl). To enable it, set kafkaParams appropriately before passing to `createDirectStream` / `createRDD`. Note that this only applies to communication between Spark and Kafka brokers; you are still responsible for separately [securing](security.html) Spark inter-node communication. + + +
    +
    +{% highlight scala %} +val kafkaParams = Map[String, Object]( + // the usual params, make sure to change the port in bootstrap.servers if 9092 is not TLS + "security.protocol" -> "SSL", + "ssl.truststore.location" -> "/some-directory/kafka.client.truststore.jks", + "ssl.truststore.password" -> "test1234", + "ssl.keystore.location" -> "/some-directory/kafka.client.keystore.jks", + "ssl.keystore.password" -> "test1234", + "ssl.key.password" -> "test1234" +) +{% endhighlight %} +
    +
    +{% highlight java %} +Map kafkaParams = new HashMap(); +// the usual params, make sure to change the port in bootstrap.servers if 9092 is not TLS +kafkaParams.put("security.protocol", "SSL"); +kafkaParams.put("ssl.truststore.location", "/some-directory/kafka.client.truststore.jks"); +kafkaParams.put("ssl.truststore.password", "test1234"); +kafkaParams.put("ssl.keystore.location", "/some-directory/kafka.client.keystore.jks"); +kafkaParams.put("ssl.keystore.password", "test1234"); +kafkaParams.put("ssl.key.password", "test1234"); +{% endhighlight %} +
    +
    + +### Deploying + +As with any Spark applications, `spark-submit` is used to launch your application. + +For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + diff --git a/docs/streaming-kafka-0-8-integration.md b/docs/streaming-kafka-0-8-integration.md new file mode 100644 index 000000000000..9f0671da2ee3 --- /dev/null +++ b/docs/streaming-kafka-0-8-integration.md @@ -0,0 +1,196 @@ +--- +layout: global +title: Spark Streaming + Kafka Integration Guide (Kafka broker version 0.8.2.1 or higher) +--- + +**Note: Kafka 0.8 support is deprecated as of Spark 2.3.0.** + +Here we explain how to configure Spark Streaming to receive data from Kafka. There are two approaches to this - the old approach using Receivers and Kafka's high-level API, and a new approach (introduced in Spark 1.3) without using Receivers. They have different programming models, performance characteristics, and semantics guarantees, so read on for more details. Both approaches are considered stable APIs as of the current version of Spark. + +## Approach 1: Receiver-based Approach +This approach uses a Receiver to receive the data. The Receiver is implemented using the Kafka high-level consumer API. As with all receivers, the data received from Kafka through a Receiver is stored in Spark executors, and then jobs launched by Spark Streaming processes the data. + +However, under default configuration, this approach can lose data under failures (see [receiver reliability](streaming-programming-guide.html#receiver-reliability). To ensure zero-data loss, you have to additionally enable Write Ahead Logs in Spark Streaming (introduced in Spark 1.2). This synchronously saves all the received Kafka data into write ahead logs on a distributed file system (e.g HDFS), so that all the data can be recovered on failure. See [Deploying section](streaming-programming-guide.html#deploying-applications) in the streaming programming guide for more details on Write Ahead Logs. + +Next, we discuss how to use this approach in your streaming application. + +1. **Linking:** For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + + For Python applications, you will have to add this above library and its dependencies when deploying your application. See the *Deploying* subsection below. + +2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. + +
    +
    + import org.apache.spark.streaming.kafka._ + + val kafkaStream = KafkaUtils.createStream(streamingContext, + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) + + You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$). +
    +
    + import org.apache.spark.streaming.kafka.*; + + JavaPairReceiverInputDStream kafkaStream = + KafkaUtils.createStream(streamingContext, + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]); + + You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html). + +
    +
    + from pyspark.streaming.kafka import KafkaUtils + + kafkaStream = KafkaUtils.createStream(streamingContext, \ + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) + + By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils). +
    +
    + + **Points to remember:** + + - Topic partitions in Kafka does not correlate to partitions of RDDs generated in Spark Streaming. So increasing the number of topic-specific partitions in the `KafkaUtils.createStream()` only increases the number of threads using which topics that are consumed within a single receiver. It does not increase the parallelism of Spark in processing the data. Refer to the main document for more information on that. + + - Multiple Kafka input DStreams can be created with different groups and topics for parallel receiving of data using multiple receivers. + + - If you have enabled Write Ahead Logs with a replicated file system like HDFS, the received data is already being replicated in the log. Hence, the storage level in storage level for the input stream to `StorageLevel.MEMORY_AND_DISK_SER` (that is, use +`KafkaUtils.createStream(..., StorageLevel.MEMORY_AND_DISK_SER)`). + +3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications. + + For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + + For Python applications which lack SBT/Maven project management, `spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is, + + ./bin/spark-submit --packages org.apache.spark:spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... + + Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-kafka-0-8-assembly` from the + [Maven repository](https://search.maven.org/#search|ga|1|a%3A%22spark-streaming-kafka-0-8-assembly_{{site.SCALA_BINARY_VERSION}}%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`. + +## Approach 2: Direct Approach (No Receivers) +This new receiver-less "direct" approach has been introduced in Spark 1.3 to ensure stronger end-to-end guarantees. Instead of using receivers to receive data, this approach periodically queries Kafka for the latest offsets in each topic+partition, and accordingly defines the offset ranges to process in each batch. When the jobs to process the data are launched, Kafka's simple consumer API is used to read the defined ranges of offsets from Kafka (similar to read files from a file system). Note that this feature was introduced in Spark 1.3 for the Scala and Java API, in Spark 1.4 for the Python API. + +This approach has the following advantages over the receiver-based approach (i.e. Approach 1). + +- *Simplified Parallelism:* No need to create multiple input Kafka streams and union them. With `directStream`, Spark Streaming will create as many RDD partitions as there are Kafka partitions to consume, which will all read data from Kafka in parallel. So there is a one-to-one mapping between Kafka and RDD partitions, which is easier to understand and tune. + +- *Efficiency:* Achieving zero-data loss in the first approach required the data to be stored in a Write Ahead Log, which further replicated the data. This is actually inefficient as the data effectively gets replicated twice - once by Kafka, and a second time by the Write Ahead Log. This second approach eliminates the problem as there is no receiver, and hence no need for Write Ahead Logs. As long as you have sufficient Kafka retention, messages can be recovered from Kafka. + +- *Exactly-once semantics:* The first approach uses Kafka's high level API to store consumed offsets in Zookeeper. This is traditionally the way to consume data from Kafka. While this approach (in combination with write ahead logs) can ensure zero data loss (i.e. at-least once semantics), there is a small chance some records may get consumed twice under some failures. This occurs because of inconsistencies between data reliably received by Spark Streaming and offsets tracked by Zookeeper. Hence, in this second approach, we use simple Kafka API that does not use Zookeeper. Offsets are tracked by Spark Streaming within its checkpoints. This eliminates inconsistencies between Spark Streaming and Zookeeper/Kafka, and so each record is received by Spark Streaming effectively exactly once despite failures. In order to achieve exactly-once semantics for output of your results, your output operation that saves the data to an external data store must be either idempotent, or an atomic transaction that saves results and offsets (see [Semantics of output operations](streaming-programming-guide.html#semantics-of-output-operations) in the main programming guide for further information). + +Note that one disadvantage of this approach is that it does not update offsets in Zookeeper, hence Zookeeper-based Kafka monitoring tools will not show progress. However, you can access the offsets processed by this approach in each batch and update Zookeeper yourself (see below). + +Next, we discuss how to use this approach in your streaming application. + +1. **Linking:** This approach is supported only in Scala/Java application. Link your SBT/Maven project with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. + +
    +
    + import org.apache.spark.streaming.kafka._ + + val directKafkaStream = KafkaUtils.createDirectStream[ + [key class], [value class], [key decoder class], [value decoder class] ]( + streamingContext, [map of Kafka parameters], [set of topics to consume]) + + You can also pass a `messageHandler` to `createDirectStream` to access `MessageAndMetadata` that contains metadata about the current message and transform it to any desired type. + See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$). +
    +
    + import org.apache.spark.streaming.kafka.*; + + JavaPairInputDStream directKafkaStream = + KafkaUtils.createDirectStream(streamingContext, + [key class], [value class], [key decoder class], [value decoder class], + [map of Kafka parameters], [set of topics to consume]); + + You can also pass a `messageHandler` to `createDirectStream` to access `MessageAndMetadata` that contains metadata about the current message and transform it to any desired type. + See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html). + +
    +
    + from pyspark.streaming.kafka import KafkaUtils + directKafkaStream = KafkaUtils.createDirectStream(ssc, [topic], {"metadata.broker.list": brokers}) + + You can also pass a `messageHandler` to `createDirectStream` to access `KafkaMessageAndMetadata` that contains metadata about the current message and transform it to any desired type. + By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils). +
    +
    + + In the Kafka parameters, you must specify either `metadata.broker.list` or `bootstrap.servers`. + By default, it will start consuming from the latest offset of each Kafka partition. If you set configuration `auto.offset.reset` in Kafka parameters to `smallest`, then it will start consuming from the smallest offset. + + You can also start consuming from any arbitrary offset using other variations of `KafkaUtils.createDirectStream`. Furthermore, if you want to access the Kafka offsets consumed in each batch, you can do the following. + +
    +
    + // Hold a reference to the current offset ranges, so it can be used downstream + var offsetRanges = Array.empty[OffsetRange] + + directKafkaStream.transform { rdd => + offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + rdd + }.map { + ... + }.foreachRDD { rdd => + for (o <- offsetRanges) { + println(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") + } + ... + } +
    +
    + // Hold a reference to the current offset ranges, so it can be used downstream + AtomicReference offsetRanges = new AtomicReference<>(); + + directKafkaStream.transformToPair(rdd -> { + OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + offsetRanges.set(offsets); + return rdd; + }).map( + ... + ).foreachRDD(rdd -> { + for (OffsetRange o : offsetRanges.get()) { + System.out.println( + o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset() + ); + } + ... + }); +
    +
    + offsetRanges = [] + + def storeOffsetRanges(rdd): + global offsetRanges + offsetRanges = rdd.offsetRanges() + return rdd + + def printOffsetRanges(rdd): + for o in offsetRanges: + print "%s %s %s %s" % (o.topic, o.partition, o.fromOffset, o.untilOffset) + + directKafkaStream \ + .transform(storeOffsetRanges) \ + .foreachRDD(printOffsetRanges) +
    +
    + + You can use this to update Zookeeper yourself if you want Zookeeper-based Kafka monitoring tools to show progress of the streaming application. + + Note that the typecast to HasOffsetRanges will only succeed if it is done in the first method called on the directKafkaStream, not later down a chain of methods. You can use transform() instead of foreachRDD() as your first method call in order to access offsets, then call further Spark methods. However, be aware that the one-to-one mapping between RDD partition and Kafka partition does not remain after any methods that shuffle or repartition, e.g. reduceByKey() or window(). + + Another thing to note is that since this approach does not use Receivers, the standard receiver-related (that is, [configurations](configuration.html) of the form `spark.streaming.receiver.*` ) will not apply to the input DStreams created by this approach (will apply to other input DStreams though). Instead, use the [configurations](configuration.html) `spark.streaming.kafka.*`. An important one is `spark.streaming.kafka.maxRatePerPartition` which is the maximum rate (in messages per second) at which each Kafka partition will be read by this direct API. + +3. **Deploying:** This is same as the first approach. diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md index ab7f0117c0b7..4aca391e4ba1 100644 --- a/docs/streaming-kafka-integration.md +++ b/docs/streaming-kafka-integration.md @@ -2,193 +2,53 @@ layout: global title: Spark Streaming + Kafka Integration Guide --- -[Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka. There are two approaches to this - the old approach using Receivers and Kafka's high-level API, and a new experimental approach (introduced in Spark 1.3) without using Receivers. They have different programming models, performance characteristics, and semantics guarantees, so read on for more details. -## Approach 1: Receiver-based Approach -This approach uses a Receiver to receive the data. The Receiver is implemented using the Kafka high-level consumer API. As with all receivers, the data received from Kafka through a Receiver is stored in Spark executors, and then jobs launched by Spark Streaming processes the data. - -However, under default configuration, this approach can lose data under failures (see [receiver reliability](streaming-programming-guide.html#receiver-reliability). To ensure zero-data loss, you have to additionally enable Write Ahead Logs in Spark Streaming (introduced in Spark 1.2). This synchronously saves all the received Kafka data into write ahead logs on a distributed file system (e.g HDFS), so that all the data can be recovered on failure. See [Deploying section](streaming-programming-guide.html#deploying-applications) in the streaming programming guide for more details on Write Ahead Logs. - -Next, we discuss how to use this approach in your streaming application. - -1. **Linking:** For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). - - groupId = org.apache.spark - artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION_SHORT}} - - For Python applications, you will have to add this above library and its dependencies when deploying your application. See the *Deploying* subsection below. - -2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. - -
    -
    - import org.apache.spark.streaming.kafka._ - - val kafkaStream = KafkaUtils.createStream(streamingContext, - [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) - - You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala). -
    -
    - import org.apache.spark.streaming.kafka.*; - - JavaPairReceiverInputDStream kafkaStream = - KafkaUtils.createStream(streamingContext, - [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]); - - You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java). - -
    -
    - from pyspark.streaming.kafka import KafkaUtils - - kafkaStream = KafkaUtils.createStream(streamingContext, \ - [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) - - By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/kafka_wordcount.py). -
    -
    - - **Points to remember:** - - - Topic partitions in Kafka does not correlate to partitions of RDDs generated in Spark Streaming. So increasing the number of topic-specific partitions in the `KafkaUtils.createStream()` only increases the number of threads using which topics that are consumed within a single receiver. It does not increase the parallelism of Spark in processing the data. Refer to the main document for more information on that. - - - Multiple Kafka input DStreams can be created with different groups and topics for parallel receiving of data using multiple receivers. - - - If you have enabled Write Ahead Logs with a replicated file system like HDFS, the received data is already being replicated in the log. Hence, the storage level in storage level for the input stream to `StorageLevel.MEMORY_AND_DISK_SER` (that is, use -`KafkaUtils.createStream(..., StorageLevel.MEMORY_AND_DISK_SER)`). - -3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications. - - For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). - - For Python applications which lack SBT/Maven project management, `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is, - - ./bin/spark-submit --packages org.apache.spark:spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... - - Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-kafka-assembly` from the - [Maven repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-kafka-assembly_2.10%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`. - -## Approach 2: Direct Approach (No Receivers) -This new receiver-less "direct" approach has been introduced in Spark 1.3 to ensure stronger end-to-end guarantees. Instead of using receivers to receive data, this approach periodically queries Kafka for the latest offsets in each topic+partition, and accordingly defines the offset ranges to process in each batch. When the jobs to process the data are launched, Kafka's simple consumer API is used to read the defined ranges of offsets from Kafka (similar to read files from a file system). Note that this is an experimental feature introduced in Spark 1.3 for the Scala and Java API. Spark 1.4 added a Python API, but it is not yet at full feature parity. - -This approach has the following advantages over the receiver-based approach (i.e. Approach 1). - -- *Simplified Parallelism:* No need to create multiple input Kafka streams and union them. With `directStream`, Spark Streaming will create as many RDD partitions as there are Kafka partitions to consume, which will all read data from Kafka in parallel. So there is a one-to-one mapping between Kafka and RDD partitions, which is easier to understand and tune. - -- *Efficiency:* Achieving zero-data loss in the first approach required the data to be stored in a Write Ahead Log, which further replicated the data. This is actually inefficient as the data effectively gets replicated twice - once by Kafka, and a second time by the Write Ahead Log. This second approach eliminates the problem as there is no receiver, and hence no need for Write Ahead Logs. As long as you have sufficient Kafka retention, messages can be recovered from Kafka. - -- *Exactly-once semantics:* The first approach uses Kafka's high level API to store consumed offsets in Zookeeper. This is traditionally the way to consume data from Kafka. While this approach (in combination with write ahead logs) can ensure zero data loss (i.e. at-least once semantics), there is a small chance some records may get consumed twice under some failures. This occurs because of inconsistencies between data reliably received by Spark Streaming and offsets tracked by Zookeeper. Hence, in this second approach, we use simple Kafka API that does not use Zookeeper. Offsets are tracked by Spark Streaming within its checkpoints. This eliminates inconsistencies between Spark Streaming and Zookeeper/Kafka, and so each record is received by Spark Streaming effectively exactly once despite failures. In order to achieve exactly-once semantics for output of your results, your output operation that saves the data to an external data store must be either idempotent, or an atomic transaction that saves results and offsets (see [Semantics of output operations](streaming-programming-guide.html#semantics-of-output-operations) in the main programming guide for further information). - -Note that one disadvantage of this approach is that it does not update offsets in Zookeeper, hence Zookeeper-based Kafka monitoring tools will not show progress. However, you can access the offsets processed by this approach in each batch and update Zookeeper yourself (see below). - -Next, we discuss how to use this approach in your streaming application. - -1. **Linking:** This approach is supported only in Scala/Java application. Link your SBT/Maven project with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). - - groupId = org.apache.spark - artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION_SHORT}} - -2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. - -
    -
    - import org.apache.spark.streaming.kafka._ - - val directKafkaStream = KafkaUtils.createDirectStream[ - [key class], [value class], [key decoder class], [value decoder class] ]( - streamingContext, [map of Kafka parameters], [set of topics to consume]) - - See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala). -
    -
    - import org.apache.spark.streaming.kafka.*; - - JavaPairReceiverInputDStream directKafkaStream = - KafkaUtils.createDirectStream(streamingContext, - [key class], [value class], [key decoder class], [value decoder class], - [map of Kafka parameters], [set of topics to consume]); - - See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java). - -
    -
    - from pyspark.streaming.kafka import KafkaUtils - directKafkaStream = KafkaUtils.createDirectStream(ssc, [topic], {"metadata.broker.list": brokers}) - - By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils) - and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/direct_kafka_wordcount.py). -
    -
    - - In the Kafka parameters, you must specify either `metadata.broker.list` or `bootstrap.servers`. - By default, it will start consuming from the latest offset of each Kafka partition. If you set configuration `auto.offset.reset` in Kafka parameters to `smallest`, then it will start consuming from the smallest offset. - - You can also start consuming from any arbitrary offset using other variations of `KafkaUtils.createDirectStream`. Furthermore, if you want to access the Kafka offsets consumed in each batch, you can do the following. - -
    -
    - // Hold a reference to the current offset ranges, so it can be used downstream - var offsetRanges = Array[OffsetRange]() - - directKafkaStream.transform { rdd => - offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges - rdd - }.map { - ... - }.foreachRDD { rdd => - for (o <- offsetRanges) { - println(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") - } - ... - } -
    -
    - // Hold a reference to the current offset ranges, so it can be used downstream - final AtomicReference offsetRanges = new AtomicReference<>(); - - directKafkaStream.transformToPair( - new Function, JavaPairRDD>() { - @Override - public JavaPairRDD call(JavaPairRDD rdd) throws Exception { - OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); - offsetRanges.set(offsets); - return rdd; - } - } - ).map( - ... - ).foreachRDD( - new Function, Void>() { - @Override - public Void call(JavaPairRDD rdd) throws IOException { - for (OffsetRange o : offsetRanges.get()) { - System.out.println( - o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset() - ); - } - ... - return null; - } - } - ); -
    -
    - Not supported yet -
    -
    - - You can use this to update Zookeeper yourself if you want Zookeeper-based Kafka monitoring tools to show progress of the streaming application. - - Note that the typecast to HasOffsetRanges will only succeed if it is done in the first method called on the directKafkaStream, not later down a chain of methods. You can use transform() instead of foreachRDD() as your first method call in order to access offsets, then call further Spark methods. However, be aware that the one-to-one mapping between RDD partition and Kafka partition does not remain after any methods that shuffle or repartition, e.g. reduceByKey() or window(). - - Another thing to note is that since this approach does not use Receivers, the standard receiver-related (that is, [configurations](configuration.html) of the form `spark.streaming.receiver.*` ) will not apply to the input DStreams created by this approach (will apply to other input DStreams though). Instead, use the [configurations](configuration.html) `spark.streaming.kafka.*`. An important one is `spark.streaming.kafka.maxRatePerPartition` which is the maximum rate (in messages per second) at which each Kafka partition will be read by this direct API. - -3. **Deploying:** This is same as the first approach, for Scala, Java and Python. +[Apache Kafka](https://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Please read the [Kafka documentation](https://kafka.apache.org/documentation.html) thoroughly before starting an integration using Spark. + +The Kafka project introduced a new consumer API between versions 0.8 and 0.10, so there are 2 separate corresponding Spark Streaming packages available. Please choose the correct package for your brokers and desired features; note that the 0.8 integration is compatible with later 0.9 and 0.10 brokers, but the 0.10 integration is not compatible with earlier brokers. + +**Note: Kafka 0.8 support is deprecated as of Spark 2.3.0.** + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    spark-streaming-kafka-0-8spark-streaming-kafka-0-10
    Broker Version0.8.2.1 or higher0.10.0 or higher
    API MaturityDeprecatedStable
    Language SupportScala, Java, PythonScala, Java
    Receiver DStreamYesNo
    Direct DStreamYesYes
    SSL / TLS SupportNoYes
    Offset Commit APINoYes
    Dynamic Topic SubscriptionNoYes
    diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md index 238a911a9199..678b0643fd70 100644 --- a/docs/streaming-kinesis-integration.md +++ b/docs/streaming-kinesis-integration.md @@ -15,63 +15,123 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m #### Configuring Spark Streaming Application -1. **Linking:** In your SBT/Maven project definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). +1. **Linking:** For Scala/Java applications using SBT/Maven project definitions, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). groupId = org.apache.spark artifactId = spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION_SHORT}} + For Python applications, you will have to add this above library and its dependencies when deploying your application. See the *Deploying* subsection below. **Note that by linking to this library, you will include [ASL](https://aws.amazon.com/asl/)-licensed code in your application.** -2. **Programming:** In the streaming application code, import `KinesisUtils` and create the input DStream as follows: +2. **Programming:** In the streaming application code, import `KinesisInputDStream` and create the input DStream of byte array as follows:
    - import org.apache.spark.streaming.Duration - import org.apache.spark.streaming.kinesis._ - import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream - - val kinesisStream = KinesisUtils.createStream( - streamingContext, [Kinesis app name], [Kinesis stream name], [endpoint URL], - [region name], [initial position], [checkpoint interval], StorageLevel.MEMORY_AND_DISK_2) - - See the [API docs](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisUtils$) - and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala). Refer to the Running the Example section for instructions on how to run the example. + import org.apache.spark.storage.StorageLevel + import org.apache.spark.streaming.kinesis.KinesisInputDStream + import org.apache.spark.streaming.{Seconds, StreamingContext} + import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream + + val kinesisStream = KinesisInputDStream.builder + .streamingContext(streamingContext) + .endpointUrl([endpoint URL]) + .regionName([region name]) + .streamName([streamName]) + .initialPositionInStream([initial position]) + .checkpointAppName([Kinesis app name]) + .checkpointInterval([checkpoint interval]) + .storageLevel(StorageLevel.MEMORY_AND_DISK_2) + .build() + + See the [API docs](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisInputDStream) + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala). Refer to the [Running the Example](#running-the-example) subsection for instructions on how to run the example.
    - import org.apache.spark.streaming.Duration; - import org.apache.spark.streaming.kinesis.*; - import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; - - JavaReceiverInputDStream kinesisStream = KinesisUtils.createStream( - streamingContext, [Kinesis app name], [Kinesis stream name], [endpoint URL], - [region name], [initial position], [checkpoint interval], StorageLevel.MEMORY_AND_DISK_2); + import org.apache.spark.storage.StorageLevel + import org.apache.spark.streaming.kinesis.KinesisInputDStream + import org.apache.spark.streaming.Seconds + import org.apache.spark.streaming.StreamingContext + import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream + + KinesisInputDStream kinesisStream = KinesisInputDStream.builder + .streamingContext(streamingContext) + .endpointUrl([endpoint URL]) + .regionName([region name]) + .streamName([streamName]) + .initialPositionInStream([initial position]) + .checkpointAppName([Kinesis app name]) + .checkpointInterval([checkpoint interval]) + .storageLevel(StorageLevel.MEMORY_AND_DISK_2) + .build(); See the [API docs](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html) - and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java). Refer to the next subsection for instructions to run the example. + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java). Refer to the [Running the Example](#running-the-example) subsection for instructions to run the example.
    - from pyspark.streaming.kinesis import KinesisUtils, InitialPositionInStream + from pyspark.streaming.kinesis import KinesisUtils, InitialPositionInStream - kinesisStream = KinesisUtils.createStream( - streamingContext, [Kinesis app name], [Kinesis stream name], [endpoint URL], - [region name], [initial position], [checkpoint interval], StorageLevel.MEMORY_AND_DISK_2) + kinesisStream = KinesisUtils.createStream( + streamingContext, [Kinesis app name], [Kinesis stream name], [endpoint URL], + [region name], [initial position], [checkpoint interval], StorageLevel.MEMORY_AND_DISK_2) See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kinesis.KinesisUtils) - and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/extras/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py). Refer to the next subsection for instructions to run the example. + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py). Refer to the [Running the Example](#running-the-example) subsection for instructions to run the example. + +
    +
    + + You may also provide a "message handler function" that takes a Kinesis `Record` and returns a generic object `T`, in case you would like to use other data included in a `Record` such as partition key. This is currently only supported in Scala and Java. + +
    +
    + import org.apache.spark.storage.StorageLevel + import org.apache.spark.streaming.kinesis.KinesisInputDStream + import org.apache.spark.streaming.{Seconds, StreamingContext} + import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream + + val kinesisStream = KinesisInputDStream.builder + .streamingContext(streamingContext) + .endpointUrl([endpoint URL]) + .regionName([region name]) + .streamName([streamName]) + .initialPositionInStream([initial position]) + .checkpointAppName([Kinesis app name]) + .checkpointInterval([checkpoint interval]) + .storageLevel(StorageLevel.MEMORY_AND_DISK_2) + .buildWithMessageHandler([message handler]) + +
    +
    + import org.apache.spark.storage.StorageLevel + import org.apache.spark.streaming.kinesis.KinesisInputDStream + import org.apache.spark.streaming.Seconds + import org.apache.spark.streaming.StreamingContext + import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream + + KinesisInputDStream kinesisStream = KinesisInputDStream.builder + .streamingContext(streamingContext) + .endpointUrl([endpoint URL]) + .regionName([region name]) + .streamName([streamName]) + .initialPositionInStream([initial position]) + .checkpointAppName([Kinesis app name]) + .checkpointInterval([checkpoint interval]) + .storageLevel(StorageLevel.MEMORY_AND_DISK_2) + .buildWithMessageHandler([message handler]);
    - - `streamingContext`: StreamingContext containg an application name used by Kinesis to tie this Kinesis application to the Kinesis stream + - `streamingContext`: StreamingContext containing an application name used by Kinesis to tie this Kinesis application to the Kinesis stream - - `[Kineiss app name]`: The application name that will be used to checkpoint the Kinesis + - `[Kinesis app name]`: The application name that will be used to checkpoint the Kinesis sequence numbers in DynamoDB table. - The application name must be unique for a given account and region. - If the table exists but has incorrect checkpoint information (for a different stream, or - old expired sequenced numbers), then there may be temporary errors. + old expired sequenced numbers), then there may be temporary errors. - `[Kinesis stream name]`: The Kinesis stream that this streaming application will pull data from. @@ -81,29 +141,40 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m - `[checkpoint interval]`: The interval (e.g., Duration(2000) = 2 seconds) at which the Kinesis Client Library saves its position in the stream. For starters, set it to the same as the batch interval of the streaming application. - - `[initial position]`: Can be either `InitialPositionInStream.TRIM_HORIZON` or `InitialPositionInStream.LATEST` (see Kinesis Checkpointing section and Amazon Kinesis API documentation for more details). + - `[initial position]`: Can be either `InitialPositionInStream.TRIM_HORIZON` or `InitialPositionInStream.LATEST` (see [`Kinesis Checkpointing`](#kinesis-checkpointing) section and [`Amazon Kinesis API documentation`](http://docs.aws.amazon.com/streams/latest/dev/developing-consumers-with-sdk.html) for more details). + + - `[message handler]`: A function that takes a Kinesis `Record` and outputs generic `T`. In other versions of the API, you can also specify the AWS access key and secret key directly. -3. **Deploying:** Package `spark-streaming-kinesis-asl_{{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). +3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications. - *Points to remember at runtime:* + For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). - - Kinesis data processing is ordered per partition and occurs at-least once per message. + For Python applications which lack SBT/Maven project management, `spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is, - - Multiple applications can read from the same Kinesis stream. Kinesis will maintain the application-specific shard and checkpoint info in DynamoDB. + ./bin/spark-submit --packages org.apache.spark:spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... - - A single Kinesis stream shard is processed by one input DStream at a time. + Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-kinesis-asl-assembly` from the + [Maven repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-kinesis-asl-assembly_{{site.SCALA_BINARY_VERSION}}%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`.

    Spark Streaming Kinesis Architecture

    + *Points to remember at runtime:* + + - Kinesis data processing is ordered per partition and occurs at-least once per message. + + - Multiple applications can read from the same Kinesis stream. Kinesis will maintain the application-specific shard and checkpoint info in DynamoDB. + + - A single Kinesis stream shard is processed by one input DStream at a time. + - A single Kinesis input DStream can read from multiple shards of a Kinesis stream by creating multiple KinesisRecordProcessor threads. - Multiple input DStreams running in separate processes/instances can read from a Kinesis stream. @@ -125,33 +196,30 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m #### Running the Example To run the example, -- Download Spark source and follow the [instructions](building-spark.html) to build Spark with profile *-Pkinesis-asl*. - - mvn -Pkinesis-asl -DskipTests clean package - +- Download a Spark binary from the [download site](http://spark.apache.org/downloads.html). - Set up Kinesis stream (see earlier section) within AWS. Note the name of the Kinesis stream and the endpoint URL corresponding to the region where the stream was created. -- Set up the environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_KEY with your AWS credentials. +- Set up the environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_KEY` with your AWS credentials. - In the Spark root directory, run the example as
    - bin/run-example streaming.KinesisWordCountASL [Kinesis app name] [Kinesis stream name] [endpoint URL] + bin/run-example --packages org.apache.spark:spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} streaming.KinesisWordCountASL [Kinesis app name] [Kinesis stream name] [endpoint URL]
    - bin/run-example streaming.JavaKinesisWordCountASL [Kinesis app name] [Kinesis stream name] [endpoint URL] + bin/run-example --packages org.apache.spark:spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} streaming.JavaKinesisWordCountASL [Kinesis app name] [Kinesis stream name] [endpoint URL]
    - bin/spark-submit --jars extras/kinesis-asl/target/scala-*/\ + bin/spark-submit --jars external/kinesis-asl/target/scala-*/\ spark-streaming-kinesis-asl-assembly_*.jar \ - extras/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py \ + external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py \ [Kinesis app name] [Kinesis stream name] [endpoint URL] [region name]
    @@ -165,11 +233,20 @@ To run the example, This will push 1000 lines per second of 10 random numbers per line to the Kinesis stream. This data should then be received and processed by the running example. +#### Record De-aggregation + +When data is generated using the [Kinesis Producer Library (KPL)](http://docs.aws.amazon.com/kinesis/latest/dev/developing-producers-with-kpl.html), messages may be aggregated for cost savings. Spark Streaming will automatically +de-aggregate records during consumption. + #### Kinesis Checkpointing - Each Kinesis input DStream periodically stores the current position of the stream in the backing DynamoDB table. This allows the system to recover from failures and continue processing where the DStream left off. - Checkpointing too frequently will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling. The provided example handles this throttling with a random-backoff-retry strategy. -- If no Kinesis checkpoint info exists when the input DStream starts, it will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) or from the latest tip (InitialPostitionInStream.LATEST). This is configurable. -- InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no input DStreams are running (and no checkpoint info is being stored). -- InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records where the impact is dependent on checkpoint frequency and processing idempotency. +- If no Kinesis checkpoint info exists when the input DStream starts, it will start either from the oldest record available (`InitialPositionInStream.TRIM_HORIZON`) or from the latest tip (`InitialPositionInStream.LATEST`). This is configurable. + - `InitialPositionInStream.LATEST` could lead to missed records if data is added to the stream while no input DStreams are running (and no checkpoint info is being stored). + - `InitialPositionInStream.TRIM_HORIZON` may lead to duplicate processing of records where the impact is dependent on checkpoint frequency and processing idempotency. + +#### Kinesis retry configuration + - `spark.streaming.kinesis.retry.waitTime` : Wait time between Kinesis retries as a duration string. When reading from Amazon Kinesis, users may hit `ProvisionedThroughputExceededException`'s, when consuming faster than 5 transactions/second or, exceeding the maximum read rate of 2 MB/second. This configuration can be tweaked to increase the sleep between fetches when a fetch fails to reduce these exceptions. Default is "100ms". + - `spark.streaming.kinesis.retry.maxAttempts` : Max number of retries for Kinesis fetches. This config can also be used to tackle the Kinesis `ProvisionedThroughputExceededException`'s in scenarios mentioned above. It can be increased to have more number of retries for Kinesis reads. Default is 3. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index c751dbb41785..868acc41226d 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -11,11 +11,11 @@ description: Spark Streaming programming guide and tutorial for Spark SPARK_VERS # Overview Spark Streaming is an extension of the core Spark API that enables scalable, high-throughput, fault-tolerant stream processing of live data streams. Data can be ingested from many sources -like Kafka, Flume, Twitter, ZeroMQ, Kinesis, or TCP sockets, and can be processed using complex +like Kafka, Flume, Kinesis, or TCP sockets, and can be processed using complex algorithms expressed with high-level functions like `map`, `reduce`, `join` and `window`. Finally, processed data can be pushed out to filesystems, databases, and live dashboards. In fact, you can apply Spark's -[machine learning](mllib-guide.html) and +[machine learning](ml-guide.html) and [graph processing](graphx-programming-guide.html) algorithms on data streams.

    @@ -126,7 +126,7 @@ ssc.awaitTermination() // Wait for the computation to terminate {% endhighlight %} The complete code can be found in the Spark Streaming example -[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala). +[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala).

    @@ -145,8 +145,8 @@ import org.apache.spark.streaming.api.java.*; import scala.Tuple2; // Create a local StreamingContext with two working thread and batch interval of 1 second -SparkConf conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount") -JavaStreamingContext jssc = new JavaStreamingContext(conf, Durations.seconds(1)) +SparkConf conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount"); +JavaStreamingContext jssc = new JavaStreamingContext(conf, Durations.seconds(1)); {% endhighlight %} Using this context, we can create a DStream that represents streaming data from a TCP @@ -158,17 +158,12 @@ JavaReceiverInputDStream lines = jssc.socketTextStream("localhost", 9999 {% endhighlight %} This `lines` DStream represents the stream of data that will be received from the data -server. Each record in this stream is a line of text. Then, we want to split the the lines by +server. Each record in this stream is a line of text. Then, we want to split the lines by space into words. {% highlight java %} // Split each line into words -JavaDStream words = lines.flatMap( - new FlatMapFunction() { - @Override public Iterable call(String x) { - return Arrays.asList(x.split(" ")); - } - }); +JavaDStream words = lines.flatMap(x -> Arrays.asList(x.split(" ")).iterator()); {% endhighlight %} `flatMap` is a DStream operation that creates a new DStream by @@ -183,18 +178,8 @@ Next, we want to count these words. {% highlight java %} // Count each word in each batch -JavaPairDStream pairs = words.mapToPair( - new PairFunction() { - @Override public Tuple2 call(String s) { - return new Tuple2(s, 1); - } - }); -JavaPairDStream wordCounts = pairs.reduceByKey( - new Function2() { - @Override public Integer call(Integer i1, Integer i2) { - return i1 + i2; - } - }); +JavaPairDStream pairs = words.mapToPair(s -> new Tuple2<>(s, 1)); +JavaPairDStream wordCounts = pairs.reduceByKey((i1, i2) -> i1 + i2); // Print the first ten elements of each RDD generated in this DStream to the console wordCounts.print(); @@ -216,7 +201,7 @@ jssc.awaitTermination(); // Wait for the computation to terminate {% endhighlight %} The complete code can be found in the Spark Streaming example -[JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java). +[JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java).
    @@ -277,7 +262,7 @@ ssc.awaitTermination() # Wait for the computation to terminate {% endhighlight %} The complete code can be found in the Spark Streaming example -[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/network_wordcount.py). +[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/streaming/network_wordcount.py).
    @@ -416,17 +401,14 @@ some of the common ones are as follows. - + - - -
    SourceArtifact
    Kafka spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}
    Kafka spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}}
    Flume spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}
    Kinesis
    spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} [Amazon Software License]
    Twitter spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}
    ZeroMQ spark-streaming-zeromq_{{site.SCALA_BINARY_VERSION}}
    MQTT spark-streaming-mqtt_{{site.SCALA_BINARY_VERSION}}
    For an up-to-date list, please refer to the -[Maven repository](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) +[Maven repository](https://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) for the full list of supported sources and artifacts. *** @@ -480,7 +462,7 @@ import org.apache.spark.*; import org.apache.spark.streaming.api.java.*; SparkConf conf = new SparkConf().setAppName(appName).setMaster(master); -JavaStreamingContext ssc = new JavaStreamingContext(conf, Duration(1000)); +JavaStreamingContext ssc = new JavaStreamingContext(conf, new Duration(1000)); {% endhighlight %} The `appName` parameter is a name for your application to show on the cluster UI. @@ -553,7 +535,7 @@ After a context is defined, you have to do the following. It represents a continuous stream of data, either the input data stream received from source, or the processed data stream generated by transforming the input stream. Internally, a DStream is represented by a continuous series of RDDs, which is Spark's abstraction of an immutable, -distributed dataset (see [Spark Programming Guide](programming-guide.html#resilient-distributed-datasets-rdds) for more details). Each RDD in a DStream contains data from a certain interval, +distributed dataset (see [Spark Programming Guide](rdd-programming-guide.html#resilient-distributed-datasets-rdds) for more details). Each RDD in a DStream contains data from a certain interval, as shown in the following figure.

    @@ -594,8 +576,8 @@ data from a source and stores it in Spark's memory for processing. Spark Streaming provides two categories of built-in streaming sources. - *Basic sources*: Sources directly available in the StreamingContext API. - Examples: file systems, socket connections, and Akka actors. -- *Advanced sources*: Sources like Kafka, Flume, Kinesis, Twitter, etc. are available through + Examples: file systems, and socket connections. +- *Advanced sources*: Sources like Kafka, Flume, Kinesis, etc. are available through extra utility classes. These require linking against extra dependencies as discussed in the [linking](#linking) section. @@ -615,7 +597,7 @@ as well as to run the receiver(s). - When running a Spark Streaming program locally, do not use "local" or "local[1]" as the master URL. Either of these means that only one thread will be used for running tasks locally. If you are using - a input DStream based on a receiver (e.g. sockets, Kafka, Flume, etc.), then the single thread will + an input DStream based on a receiver (e.g. sockets, Kafka, Flume, etc.), then the single thread will be used to run the receiver, leaving no thread for processing the received data. Hence, when running locally, always use "local[*n*]" as the master URL, where *n* > number of receivers to run (see [Spark Properties](configuration.html#spark-properties) for information on how to set @@ -631,44 +613,117 @@ as well as to run the receiver(s). We have already taken a look at the `ssc.socketTextStream(...)` in the [quick example](#a-quick-example) which creates a DStream from text data received over a TCP socket connection. Besides sockets, the StreamingContext API provides -methods for creating DStreams from files and Akka actors as input sources. +methods for creating DStreams from files as input sources. -- **File Streams:** For reading data from files on any file system compatible with the HDFS API (that is, HDFS, S3, NFS, etc.), a DStream can be created as: +#### File Streams +{:.no_toc} + +For reading data from files on any file system compatible with the HDFS API (that is, HDFS, S3, NFS, etc.), a DStream can be created as +via `StreamingContext.fileStream[KeyClass, ValueClass, InputFormatClass]`. -

    -
    - streamingContext.fileStream[KeyClass, ValueClass, InputFormatClass](dataDirectory) -
    -
    - streamingContext.fileStream(dataDirectory); -
    -
    - streamingContext.textFileStream(dataDirectory) -
    -
    +File streams do not require running a receiver so there is no need to allocate any cores for receiving file data. - Spark Streaming will monitor the directory `dataDirectory` and process any files created in that directory (files written in nested directories not supported). Note that +For simple text files, the easiest method is `StreamingContext.textFileStream(dataDirectory)`. - + The files must have the same data format. - + The files must be created in the `dataDirectory` by atomically *moving* or *renaming* them into - the data directory. - + Once moved, the files must not be changed. So if the files are being continuously appended, the new data will not be read. +
    +
    - For simple text files, there is an easier method `streamingContext.textFileStream(dataDirectory)`. And file streams do not require running a receiver, hence does not require allocating cores. +{% highlight scala %} +streamingContext.fileStream[KeyClass, ValueClass, InputFormatClass](dataDirectory) +{% endhighlight %} +For text files - Python API `fileStream` is not available in the Python API, only `textFileStream` is available. +{% highlight scala %} +streamingContext.textFileStream(dataDirectory) +{% endhighlight %} +
    + +
    +{% highlight java %} +streamingContext.fileStream(dataDirectory); +{% endhighlight %} +For text files + +{% highlight java %} +streamingContext.textFileStream(dataDirectory); +{% endhighlight %} +
    -- **Streams based on Custom Actors:** DStreams can be created with data streams received through Akka - actors by using `streamingContext.actorStream(actorProps, actor-name)`. See the [Custom Receiver +
    +`fileStream` is not available in the Python API; only `textFileStream` is available. +{% highlight python %} +streamingContext.textFileStream(dataDirectory) +{% endhighlight %} +
    + +
    + +##### How Directories are Monitored +{:.no_toc} + +Spark Streaming will monitor the directory `dataDirectory` and process any files created in that directory. + + * A simple directory can be monitored, such as `"hdfs://namenode:8040/logs/"`. + All files directly under such a path will be processed as they are discovered. + + A [POSIX glob pattern](http://pubs.opengroup.org/onlinepubs/009695399/utilities/xcu_chap02.html#tag_02_13_02) can be supplied, such as + `"hdfs://namenode:8040/logs/2017/*"`. + Here, the DStream will consist of all files in the directories + matching the pattern. + That is: it is a pattern of directories, not of files in directories. + + All files must be in the same data format. + * A file is considered part of a time period based on its modification time, + not its creation time. + + Once processed, changes to a file within the current window will not cause the file to be reread. + That is: *updates are ignored*. + + The more files under a directory, the longer it will take to + scan for changes — even if no files have been modified. + * If a wildcard is used to identify directories, such as `"hdfs://namenode:8040/logs/2016-*"`, + renaming an entire directory to match the path will add the directory to the list of + monitored directories. Only the files in the directory whose modification time is + within the current window will be included in the stream. + + Calling [`FileSystem.setTimes()`](https://hadoop.apache.org/docs/current/api/org/apache/hadoop/fs/FileSystem.html#setTimes-org.apache.hadoop.fs.Path-long-long-) + to fix the timestamp is a way to have the file picked up in a later window, even if its contents have not changed. + + +##### Using Object Stores as a source of data +{:.no_toc} + +"Full" Filesystems such as HDFS tend to set the modification time on their files as soon +as the output stream is created. +When a file is opened, even before data has been completely written, +it may be included in the `DStream` - after which updates to the file within the same window +will be ignored. That is: changes may be missed, and data omitted from the stream. + +To guarantee that changes are picked up in a window, write the file +to an unmonitored directory, then, immediately after the output stream is closed, +rename it into the destination directory. +Provided the renamed file appears in the scanned destination directory during the window +of its creation, the new data will be picked up. + +In contrast, Object Stores such as Amazon S3 and Azure Storage usually have slow rename operations, as the +data is actually copied. +Furthermore, renamed object may have the time of the `rename()` operation as its modification time, so +may not be considered part of the window which the original create time implied they were. + +Careful testing is needed against the target object store to verify that the timestamp behavior +of the store is consistent with that expected by Spark Streaming. It may be +that writing directly into a destination directory is the appropriate strategy for +streaming data via the chosen object store. + +For more details on this topic, consult the [Hadoop Filesystem Specification](https://hadoop.apache.org/docs/stable2/hadoop-project-dist/hadoop-common/filesystem/introduction.html). + +#### Streams based on Custom Receivers +{:.no_toc} + +DStreams can be created with data streams received through custom receivers. See the [Custom Receiver Guide](streaming-custom-receivers.html) for more details. - Python API Since actors are available only in the Java and Scala - libraries, `actorStream` is not available in the Python API. +#### Queue of RDDs as a Stream +{:.no_toc} -- **Queue of RDDs as a Stream:** For testing a Spark Streaming application with test data, one can also create a DStream based on a queue of RDDs, using `streamingContext.queueStream(queueOfRDDs)`. Each RDD pushed into the queue will be treated as a batch of data in the DStream, and processed like a stream. +For testing a Spark Streaming application with test data, one can also create a DStream based on a queue of RDDs, using `streamingContext.queueStream(queueOfRDDs)`. Each RDD pushed into the queue will be treated as a batch of data in the DStream, and processed like a stream. -For more details on streams from sockets, files, and actors, -see the API documentations of the relevant functions in +For more details on streams from sockets and files, see the API documentations of the relevant functions in [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) for Scala, [JavaStreamingContext](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html) for Java, and [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) for Python. @@ -677,38 +732,12 @@ for Java, and [StreamingContext](api/python/pyspark.streaming.html#pyspark.strea {:.no_toc} Python API As of Spark {{site.SPARK_VERSION_SHORT}}, -out of these sources, Kafka, Kinesis, Flume and MQTT are available in the Python API. +out of these sources, Kafka, Kinesis and Flume are available in the Python API. This category of sources require interfacing with external non-Spark libraries, some of them with complex dependencies (e.g., Kafka and Flume). Hence, to minimize issues related to version conflicts of dependencies, the functionality to create DStreams from these sources has been moved to separate -libraries that can be [linked](#linking) to explicitly when necessary. For example, if you want to -create a DStream using data from Twitter's stream of tweets, you have to do the following: - -1. *Linking*: Add the artifact `spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}` to the - SBT/Maven project dependencies. -1. *Programming*: Import the `TwitterUtils` class and create a DStream with - `TwitterUtils.createStream` as shown below. -1. *Deploying*: Generate an uber JAR with all the dependencies (including the dependency - `spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}` and its transitive dependencies) and - then deploy the application. This is further explained in the [Deploying section](#deploying-applications). - -
    -
    -{% highlight scala %} -import org.apache.spark.streaming.twitter._ - -TwitterUtils.createStream(ssc, None) -{% endhighlight %} -
    -
    -{% highlight java %} -import org.apache.spark.streaming.twitter.*; - -TwitterUtils.createStream(jssc); -{% endhighlight %} -
    -
    +libraries that can be [linked](#linking) to explicitly when necessary. Note that these advanced sources are not available in the Spark shell, hence applications based on these advanced sources cannot be tested in the shell. If you really want to use them in the Spark @@ -717,21 +746,12 @@ and add it to the classpath. Some of these advanced sources are as follows. -- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka 0.8.2.1. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details. +- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka broker versions 0.8.2.1 or higher. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details. - **Flume:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Flume 1.6.0. See the [Flume Integration Guide](streaming-flume-integration.html) for more details. - **Kinesis:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kinesis Client Library 1.2.1. See the [Kinesis Integration Guide](streaming-kinesis-integration.html) for more details. -- **Twitter:** Spark Streaming's TwitterUtils uses Twitter4j 3.0.3 to get the public stream of tweets using - [Twitter's Streaming API](https://dev.twitter.com/docs/streaming-apis). Authentication information - can be provided by any of the [methods](http://twitter4j.org/en/configuration.html) supported by - Twitter4J library. You can either get the public stream, or get the filtered stream based on a - keywords. See the API documentation ([Scala](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), - [Java](api/java/index.html?org/apache/spark/streaming/twitter/TwitterUtils.html)) and examples - ([TwitterPopularTags]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala) - and [TwitterAlgebirdCMS]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala)). - ### Custom Sources {:.no_toc} @@ -798,7 +818,7 @@ Some of the common ones are as follows. reduce(func) Return a new DStream of single-element RDDs by aggregating the elements in each RDD of the source DStream using a function func (which takes two arguments and returns one). - The function should be associative so that it can be computed in parallel. + The function should be associative and commutative so that it can be computed in parallel. countByValue() @@ -872,22 +892,16 @@ 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). +the `(word, 1)` pairs) and the `runningCount` having the previous count.
    {% highlight java %} -import com.google.common.base.Optional; Function2, Optional, Optional> updateFunction = - new Function2, Optional, Optional>() { - @Override public Optional call(List values, Optional state) { - Integer newSum = ... // add the new values with the previous running count to get the new count - return Optional.of(newSum); - } + (values, state) -> { + Integer newSum = ... // add the new values with the previous running count to get the new count + return Optional.of(newSum); }; {% endhighlight %} @@ -901,7 +915,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/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/streaming /JavaStatefulNetworkWordCount.java).
    @@ -910,7 +924,7 @@ Java code, take a look at the example {% highlight python %} def updateFunction(newValues, runningCount): if runningCount is None: - runningCount = 0 + runningCount = 0 return sum(newValues, runningCount) # add the new values with the previous running count to get the new count {% endhighlight %} @@ -924,7 +938,7 @@ runningCounts = pairs.updateStateByKey(updateFunction) 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 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). +[stateful_network_wordcount.py]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/streaming/stateful_network_wordcount.py).
    @@ -950,10 +964,10 @@ spam information (maybe generated with Spark as well) and then filtering based o {% highlight scala %} val spamInfoRDD = ssc.sparkContext.newAPIHadoopRDD(...) // RDD containing spam information -val cleanedDStream = wordCounts.transform(rdd => { +val cleanedDStream = wordCounts.transform { rdd => rdd.join(spamInfoRDD).filter(...) // join data stream with spam information to do data cleaning ... -}) +} {% endhighlight %} @@ -962,22 +976,19 @@ val cleanedDStream = wordCounts.transform(rdd => { {% highlight java %} import org.apache.spark.streaming.api.java.*; // RDD containing spam information -final JavaPairRDD spamInfoRDD = jssc.sparkContext().newAPIHadoopRDD(...); +JavaPairRDD spamInfoRDD = jssc.sparkContext().newAPIHadoopRDD(...); -JavaPairDStream cleanedDStream = wordCounts.transform( - new Function, JavaPairRDD>() { - @Override public JavaPairRDD call(JavaPairRDD rdd) throws Exception { - rdd.join(spamInfoRDD).filter(...); // join data stream with spam information to do data cleaning - ... - } - }); +JavaPairDStream cleanedDStream = wordCounts.transform(rdd -> { + rdd.join(spamInfoRDD).filter(...); // join data stream with spam information to do data cleaning + ... +}); {% endhighlight %}
    {% highlight python %} -spamInfoRDD = sc.pickleFile(...) # RDD containing spam information +spamInfoRDD = sc.pickleFile(...) # RDD containing spam information # join data stream with spam information to do data cleaning cleanedDStream = wordCounts.transform(lambda rdd: rdd.join(spamInfoRDD).filter(...)) @@ -1033,15 +1044,8 @@ val windowedWordCounts = pairs.reduceByKeyAndWindow((a:Int,b:Int) => (a + b), Se
    {% highlight java %} -// Reduce function adding two integers, defined separately for clarity -Function2 reduceFunc = new Function2() { - @Override public Integer call(Integer i1, Integer i2) { - return i1 + i2; - } -}; - // Reduce last 30 seconds of data, every 10 seconds -JavaPairDStream windowedWordCounts = pairs.reduceByKeyAndWindow(reduceFunc, Durations.seconds(30), Durations.seconds(10)); +JavaPairDStream windowedWordCounts = pairs.reduceByKeyAndWindow((i1, i2) -> i1 + i2, Durations.seconds(30), Durations.seconds(10)); {% endhighlight %}
    @@ -1073,7 +1077,7 @@ said two parameters - windowLength and slideInterval. reduceByWindow(func, windowLength, slideInterval) Return a new single-element stream, created by aggregating elements in the stream over a - sliding interval using func. The function should be associative so that it can be computed + sliding interval using func. The function should be associative and commutative so that it can be computed correctly in parallel. @@ -1188,14 +1192,7 @@ val joinedStream = windowedStream.transform { rdd => rdd.join(dataset) } {% highlight java %} JavaPairRDD dataset = ... JavaPairDStream windowedStream = stream.window(Durations.seconds(20)); -JavaPairDStream joinedStream = windowedStream.transform( - new Function>, JavaRDD>>() { - @Override - public JavaRDD> call(JavaRDD> rdd) { - return rdd.join(dataset); - } - } -); +JavaPairDStream joinedStream = windowedStream.transform(rdd -> rdd.join(dataset)); {% endhighlight %}
    @@ -1293,6 +1290,16 @@ dstream.foreachRDD { rdd => } {% endhighlight %}
    +
    +{% highlight java %} +dstream.foreachRDD(rdd -> { + Connection connection = createNewConnection(); // executed at the driver + rdd.foreach(record -> { + connection.send(record); // executed at the worker + }); +}); +{% endhighlight %} +
    {% highlight python %} def sendRecord(rdd): @@ -1306,7 +1313,7 @@ dstream.foreachRDD(sendRecord)
    This is incorrect as this requires the connection object to be serialized and sent from the -driver to the worker. Such connection objects are rarely transferrable across machines. This +driver to the worker. Such connection objects are rarely transferable across machines. This error may manifest as serialization errors (connection object not serializable), initialization errors (connection object needs to be initialized at the workers), etc. The correct solution is to create the connection object at the worker. @@ -1326,6 +1333,17 @@ dstream.foreachRDD { rdd => } {% endhighlight %} +
    +{% highlight java %} +dstream.foreachRDD(rdd -> { + rdd.foreach(record -> { + Connection connection = createNewConnection(); + connection.send(record); + connection.close(); + }); +}); +{% endhighlight %} +
    {% highlight python %} def sendRecord(record): @@ -1356,6 +1374,19 @@ dstream.foreachRDD { rdd => } {% endhighlight %}
    +
    +{% highlight java %} +dstream.foreachRDD(rdd -> { + rdd.foreachPartition(partitionOfRecords -> { + Connection connection = createNewConnection(); + while (partitionOfRecords.hasNext()) { + connection.send(partitionOfRecords.next()); + } + connection.close(); + }); +}); +{% endhighlight %} +
    {% highlight python %} def sendPartition(iter): @@ -1389,6 +1420,20 @@ dstream.foreachRDD { rdd => {% endhighlight %}
    +
    +{% highlight java %} +dstream.foreachRDD(rdd -> { + rdd.foreachPartition(partitionOfRecords -> { + // ConnectionPool is a static, lazily initialized pool of connections + Connection connection = ConnectionPool.getConnection(); + while (partitionOfRecords.hasNext()) { + connection.send(partitionOfRecords.next()); + } + ConnectionPool.returnConnection(connection); // return to the pool for future reuse + }); +}); +{% endhighlight %} +
    {% highlight python %} def sendPartition(iter): @@ -1416,7 +1461,7 @@ Note that the connections in the pool should be lazily created on demand and tim *** ## DataFrame and SQL Operations -You can easily use [DataFrames and SQL](sql-programming-guide.html) operations on streaming data. You have to create a SQLContext using the SparkContext that the StreamingContext is using. Furthermore this has to done such that it can be restarted on driver failures. This is done by creating a lazily instantiated singleton instance of SQLContext. This is shown in the following example. It modifies the earlier [word count example](#a-quick-example) to generate word counts using DataFrames and SQL. Each RDD is converted to a DataFrame, registered as a temporary table and then queried using SQL. +You can easily use [DataFrames and SQL](sql-programming-guide.html) operations on streaming data. You have to create a SparkSession using the SparkContext that the StreamingContext is using. Furthermore this has to done such that it can be restarted on driver failures. This is done by creating a lazily instantiated singleton instance of SparkSession. This is shown in the following example. It modifies the earlier [word count example](#a-quick-example) to generate word counts using DataFrames and SQL. Each RDD is converted to a DataFrame, registered as a temporary table and then queried using SQL.
    @@ -1428,25 +1473,25 @@ val words: DStream[String] = ... words.foreachRDD { rdd => - // Get the singleton instance of SQLContext - val sqlContext = SQLContext.getOrCreate(rdd.sparkContext) - import sqlContext.implicits._ + // Get the singleton instance of SparkSession + val spark = SparkSession.builder.config(rdd.sparkContext.getConf).getOrCreate() + import spark.implicits._ // Convert RDD[String] to DataFrame val wordsDataFrame = rdd.toDF("word") - // Register as table - wordsDataFrame.registerTempTable("words") + // Create a temporary view + wordsDataFrame.createOrReplaceTempView("words") // Do word count on DataFrame using SQL and print it val wordCountsDataFrame = - sqlContext.sql("select word, count(*) as total from words group by word") + spark.sql("select word, count(*) as total from words group by word") wordCountsDataFrame.show() } {% endhighlight %} -See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala). +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala).
    {% highlight java %} @@ -1470,47 +1515,41 @@ public class JavaRow implements java.io.Serializable { JavaDStream words = ... -words.foreachRDD( - new Function2, Time, Void>() { - @Override - public Void call(JavaRDD rdd, Time time) { +words.foreachRDD((rdd, time) -> { + // Get the singleton instance of SparkSession + SparkSession spark = SparkSession.builder().config(rdd.sparkContext().getConf()).getOrCreate(); - // Get the singleton instance of SQLContext - SQLContext sqlContext = SQLContext.getOrCreate(rdd.context()); - - // Convert RDD[String] to RDD[case class] to DataFrame - JavaRDD rowRDD = rdd.map(new Function() { - public JavaRow call(String word) { - JavaRow record = new JavaRow(); - record.setWord(word); - return record; - } - }); - DataFrame wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRow.class); + // Convert RDD[String] to RDD[case class] to DataFrame + JavaRDD rowRDD = rdd.map(word -> { + JavaRow record = new JavaRow(); + record.setWord(word); + return record; + }); + DataFrame wordsDataFrame = spark.createDataFrame(rowRDD, JavaRow.class); - // Register as table - wordsDataFrame.registerTempTable("words"); + // Creates a temporary view using the DataFrame + wordsDataFrame.createOrReplaceTempView("words"); - // Do word count on table using SQL and print it - DataFrame wordCountsDataFrame = - sqlContext.sql("select word, count(*) as total from words group by word"); - wordCountsDataFrame.show(); - return null; - } - } -); + // Do word count on table using SQL and print it + DataFrame wordCountsDataFrame = + spark.sql("select word, count(*) as total from words group by word"); + wordCountsDataFrame.show(); +}); {% endhighlight %} -See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java). +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java).
    {% highlight python %} -# Lazily instantiated global instance of SQLContext -def getSqlContextInstance(sparkContext): - if ('sqlContextSingletonInstance' not in globals()): - globals()['sqlContextSingletonInstance'] = SQLContext(sparkContext) - return globals()['sqlContextSingletonInstance'] +# Lazily instantiated global instance of SparkSession +def getSparkSessionInstance(sparkConf): + if ("sparkSessionSingletonInstance" not in globals()): + globals()["sparkSessionSingletonInstance"] = SparkSession \ + .builder \ + .config(conf=sparkConf) \ + .getOrCreate() + return globals()["sparkSessionSingletonInstance"] ... @@ -1521,18 +1560,18 @@ words = ... # DStream of strings def process(time, rdd): print("========= %s =========" % str(time)) try: - # Get the singleton instance of SQLContext - sqlContext = getSqlContextInstance(rdd.context) + # Get the singleton instance of SparkSession + spark = getSparkSessionInstance(rdd.context.getConf()) # Convert RDD[String] to RDD[Row] to DataFrame rowRdd = rdd.map(lambda w: Row(word=w)) - wordsDataFrame = sqlContext.createDataFrame(rowRdd) + wordsDataFrame = spark.createDataFrame(rowRdd) - # Register as table - wordsDataFrame.registerTempTable("words") + # Creates a temporary view using the DataFrame + wordsDataFrame.createOrReplaceTempView("words") # Do word count on table using SQL and print it - wordCountsDataFrame = sqlContext.sql("select word, count(*) as total from words group by word") + wordCountsDataFrame = spark.sql("select word, count(*) as total from words group by word") wordCountsDataFrame.show() except: pass @@ -1540,7 +1579,7 @@ def process(time, rdd): words.foreachRDD(process) {% endhighlight %} -See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/sql_network_wordcount.py). +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/streaming/sql_network_wordcount.py).
    @@ -1552,7 +1591,7 @@ See the [DataFrames and SQL](sql-programming-guide.html) guide to learn more abo *** ## MLlib Operations -You can also easily use machine learning algorithms provided by [MLlib](mllib-guide.html). First of all, there are streaming machine learning algorithms (e.g. [Streaming Linear Regression](mllib-linear-methods.html#streaming-linear-regression), [Streaming KMeans](mllib-clustering.html#streaming-k-means), etc.) which can simultaneously learn from the streaming data as well as apply the model on the streaming data. Beyond these, for a much larger class of machine learning algorithms, you can learn a learning model offline (i.e. using historical data) and then apply the model online on streaming data. See the [MLlib](mllib-guide.html) guide for more details. +You can also easily use machine learning algorithms provided by [MLlib](ml-guide.html). First of all, there are streaming machine learning algorithms (e.g. [Streaming Linear Regression](mllib-linear-methods.html#streaming-linear-regression), [Streaming KMeans](mllib-clustering.html#streaming-k-means), etc.) which can simultaneously learn from the streaming data as well as apply the model on the streaming data. Beyond these, for a much larger class of machine learning algorithms, you can learn a learning model offline (i.e. using historical data) and then apply the model online on streaming data. See the [MLlib](ml-guide.html) guide for more details. *** @@ -1570,7 +1609,7 @@ default persistence level is set to replicate the data to two nodes for fault-to Note that, unlike RDDs, the default persistence level of DStreams keeps the data serialized in memory. This is further discussed in the [Performance Tuning](#memory-tuning) section. More -information on different persistence levels can be found in the [Spark Programming Guide](programming-guide.html#rdd-persistence). +information on different persistence levels can be found in the [Spark Programming Guide](rdd-programming-guide.html#rdd-persistence). *** @@ -1638,11 +1677,11 @@ This behavior is made simple by using `StreamingContext.getOrCreate`. This is us {% highlight scala %} // Function to create and setup a new StreamingContext def functionToCreateContext(): StreamingContext = { - val ssc = new StreamingContext(...) // new context - val lines = ssc.socketTextStream(...) // create DStreams - ... - ssc.checkpoint(checkpointDirectory) // set checkpoint directory - ssc + val ssc = new StreamingContext(...) // new context + val lines = ssc.socketTextStream(...) // create DStreams + ... + ssc.checkpoint(checkpointDirectory) // set checkpoint directory + ssc } // Get StreamingContext from checkpoint data or create a new one @@ -1670,7 +1709,7 @@ This example appends the word counts of network data into a file. This behavior is made simple by using `JavaStreamingContext.getOrCreate`. This is used as follows. {% highlight java %} -// Create a factory object that can create a and setup a new JavaStreamingContext +// Create a factory object that can create and setup a new JavaStreamingContext JavaStreamingContextFactory contextFactory = new JavaStreamingContextFactory() { @Override public JavaStreamingContext create() { JavaStreamingContext jssc = new JavaStreamingContext(...); // new context @@ -1708,11 +1747,11 @@ This behavior is made simple by using `StreamingContext.getOrCreate`. This is us {% highlight python %} # Function to create and setup a new StreamingContext def functionToCreateContext(): - sc = SparkContext(...) # new context - ssc = new StreamingContext(...) - lines = ssc.socketTextStream(...) # create DStreams + sc = SparkContext(...) # new context + ssc = StreamingContext(...) + lines = ssc.socketTextStream(...) # create DStreams ... - ssc.checkpoint(checkpointDirectory) # set checkpoint directory + ssc.checkpoint(checkpointDirectory) # set checkpoint directory return ssc # Get StreamingContext from checkpoint data or create a new one @@ -1757,6 +1796,170 @@ batch interval that is at least 10 seconds. It can be set by using *** +## Accumulators, Broadcast Variables, and Checkpoints + +[Accumulators](rdd-programming-guide.html#accumulators) and [Broadcast variables](rdd-programming-guide.html#broadcast-variables) +cannot be recovered from checkpoint in Spark Streaming. If you enable checkpointing and use +[Accumulators](rdd-programming-guide.html#accumulators) or [Broadcast variables](rdd-programming-guide.html#broadcast-variables) +as well, you'll have to create lazily instantiated singleton instances for +[Accumulators](rdd-programming-guide.html#accumulators) and [Broadcast variables](rdd-programming-guide.html#broadcast-variables) +so that they can be re-instantiated after the driver restarts on failure. +This is shown in the following example. + +
    +
    +{% highlight scala %} + +object WordBlacklist { + + @volatile private var instance: Broadcast[Seq[String]] = null + + def getInstance(sc: SparkContext): Broadcast[Seq[String]] = { + if (instance == null) { + synchronized { + if (instance == null) { + val wordBlacklist = Seq("a", "b", "c") + instance = sc.broadcast(wordBlacklist) + } + } + } + instance + } +} + +object DroppedWordsCounter { + + @volatile private var instance: LongAccumulator = null + + def getInstance(sc: SparkContext): LongAccumulator = { + if (instance == null) { + synchronized { + if (instance == null) { + instance = sc.longAccumulator("WordsInBlacklistCounter") + } + } + } + instance + } +} + +wordCounts.foreachRDD { (rdd: RDD[(String, Int)], time: Time) => + // Get or register the blacklist Broadcast + val blacklist = WordBlacklist.getInstance(rdd.sparkContext) + // Get or register the droppedWordsCounter Accumulator + val droppedWordsCounter = DroppedWordsCounter.getInstance(rdd.sparkContext) + // Use blacklist to drop words and use droppedWordsCounter to count them + val counts = rdd.filter { case (word, count) => + if (blacklist.value.contains(word)) { + droppedWordsCounter.add(count) + false + } else { + true + } + }.collect().mkString("[", ", ", "]") + val output = "Counts at time " + time + " " + counts +}) + +{% endhighlight %} + +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala). +
    +
    +{% highlight java %} + +class JavaWordBlacklist { + + private static volatile Broadcast> instance = null; + + public static Broadcast> getInstance(JavaSparkContext jsc) { + if (instance == null) { + synchronized (JavaWordBlacklist.class) { + if (instance == null) { + List wordBlacklist = Arrays.asList("a", "b", "c"); + instance = jsc.broadcast(wordBlacklist); + } + } + } + return instance; + } +} + +class JavaDroppedWordsCounter { + + private static volatile LongAccumulator instance = null; + + public static LongAccumulator getInstance(JavaSparkContext jsc) { + if (instance == null) { + synchronized (JavaDroppedWordsCounter.class) { + if (instance == null) { + instance = jsc.sc().longAccumulator("WordsInBlacklistCounter"); + } + } + } + return instance; + } +} + +wordCounts.foreachRDD((rdd, time) -> { + // Get or register the blacklist Broadcast + Broadcast> blacklist = JavaWordBlacklist.getInstance(new JavaSparkContext(rdd.context())); + // Get or register the droppedWordsCounter Accumulator + LongAccumulator droppedWordsCounter = JavaDroppedWordsCounter.getInstance(new JavaSparkContext(rdd.context())); + // Use blacklist to drop words and use droppedWordsCounter to count them + String counts = rdd.filter(wordCount -> { + if (blacklist.value().contains(wordCount._1())) { + droppedWordsCounter.add(wordCount._2()); + return false; + } else { + return true; + } + }).collect().toString(); + String output = "Counts at time " + time + " " + counts; +} + +{% endhighlight %} + +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java). +
    +
    +{% highlight python %} +def getWordBlacklist(sparkContext): + if ("wordBlacklist" not in globals()): + globals()["wordBlacklist"] = sparkContext.broadcast(["a", "b", "c"]) + return globals()["wordBlacklist"] + +def getDroppedWordsCounter(sparkContext): + if ("droppedWordsCounter" not in globals()): + globals()["droppedWordsCounter"] = sparkContext.accumulator(0) + return globals()["droppedWordsCounter"] + +def echo(time, rdd): + # Get or register the blacklist Broadcast + blacklist = getWordBlacklist(rdd.context) + # Get or register the droppedWordsCounter Accumulator + droppedWordsCounter = getDroppedWordsCounter(rdd.context) + + # Use blacklist to drop words and use droppedWordsCounter to count them + def filterFunc(wordCount): + if wordCount[0] in blacklist.value: + droppedWordsCounter.add(wordCount[1]) + False + else: + True + + counts = "Counts at time %s %s" % (time, rdd.filter(filterFunc).collect()) + +wordCounts.foreachRDD(echo) + +{% endhighlight %} + +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/streaming/recoverable_network_wordcount.py). + +
    +
    + +*** + ## Deploying Applications This section discusses the steps to deploy a Spark Streaming application. @@ -1771,10 +1974,10 @@ To run a Spark Streaming applications, you need to have the following. - *Package the application JAR* - You have to compile your streaming application into a JAR. If you are using [`spark-submit`](submitting-applications.html) to start the application, then you will not need to provide Spark and Spark Streaming in the JAR. However, - if your application uses [advanced sources](#advanced-sources) (e.g. Kafka, Flume, Twitter), + if your application uses [advanced sources](#advanced-sources) (e.g. Kafka, Flume), then you will have to package the extra artifact they link to, along with their dependencies, - in the JAR that is used to deploy the application. For example, an application using `TwitterUtils` - will have to include `spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}` and all its + in the JAR that is used to deploy the application. For example, an application using `KafkaUtils` + will have to include `spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` and all its transitive dependencies in the application JAR. - *Configuring sufficient memory for the executors* - Since the received data must be stored in @@ -1820,7 +2023,14 @@ To run a Spark Streaming applications, you need to have the following. to increase aggregate throughput. Additionally, it is recommended that the replication of the received data within Spark be disabled when the write ahead log is enabled as the log is already stored in a replicated storage system. This can be done by setting the storage level for the - input stream to `StorageLevel.MEMORY_AND_DISK_SER`. + input stream to `StorageLevel.MEMORY_AND_DISK_SER`. While using S3 (or any file system that + does not support flushing) for _write ahead logs_, please remember to enable + `spark.streaming.driver.writeAheadLog.closeFileAfterWrite` and + `spark.streaming.receiver.writeAheadLog.closeFileAfterWrite`. See + [Spark Streaming Configuration](configuration.html#spark-streaming) for more details. + Note that Spark will not encrypt data written to the write ahead log when I/O encryption is + enabled. If encryption of the write ahead log data is desired, it should be stored in a file + system that supports encryption natively. - *Setting the max receiving rate* - If the cluster resources is not large enough for the streaming application to process data as fast as it is being received, the receivers can be rate limited @@ -1858,12 +2068,6 @@ contains serialized Scala/Java/Python objects and trying to deserialize objects modified classes may lead to errors. In this case, either start the upgraded app with a different checkpoint directory, or delete the previous checkpoint directory. -### Other Considerations -{:.no_toc} -If the data is being received by the receivers faster than what can be processed, -you can limit the rate by setting the [configuration parameter](configuration.html#spark-streaming) -`spark.streaming.receiver.maxRate`. - *** ## Monitoring Applications @@ -1936,7 +2140,7 @@ unifiedStream.print()
    {% highlight java %} int numStreams = 5; -List> kafkaStreams = new ArrayList>(numStreams); +List> kafkaStreams = new ArrayList<>(numStreams); for (int i = 0; i < numStreams; i++) { kafkaStreams.add(KafkaUtils.createStream(...)); } @@ -1948,13 +2152,13 @@ unifiedStream.print(); {% highlight python %} numStreams = 5 kafkaStreams = [KafkaUtils.createStream(...) for _ in range (numStreams)] -unifiedStream = streamingContext.union(kafkaStreams) -unifiedStream.print() +unifiedStream = streamingContext.union(*kafkaStreams) +unifiedStream.pprint() {% endhighlight %}
    -Another parameter that should be considered is the receiver's blocking interval, +Another parameter that should be considered is the receiver's block interval, which is determined by the [configuration parameter](configuration.html#spark-streaming) `spark.streaming.blockInterval`. For most receivers, the received data is coalesced together into blocks of data before storing inside Spark's memory. The number of blocks in each batch @@ -2001,9 +2205,6 @@ If the number of tasks launched per second is high (say, 50 or more per second), of sending out tasks to the slaves may be significant and will make it hard to achieve sub-second latencies. The overhead can be reduced by the following changes: -* **Task Serialization**: Using Kryo serialization for serializing tasks can reduce the task - sizes, and therefore reduce the time taken to send them to the slaves. - * **Execution mode**: Running Spark in Standalone mode or coarse-grained Mesos mode leads to better task launch times than the fine-grained Mesos mode. Please refer to the [Running on Mesos guide](running-on-mesos.html) for more details. @@ -2065,9 +2266,28 @@ overall processing throughput of the system, its use is still recommended to ach consistent batch processing times. Make sure you set the CMS GC on both the driver (using `--driver-java-options` in `spark-submit`) and the executors (using [Spark configuration](configuration.html#runtime-environment) `spark.executor.extraJavaOptions`). * **Other tips**: To further reduce GC overheads, here are some more tips to try. - - Use Tachyon for off-heap storage of persisted RDDs. See more detail in the [Spark Programming Guide](programming-guide.html#rdd-persistence). + - Persist RDDs using the `OFF_HEAP` storage level. See more detail in the [Spark Programming Guide](rdd-programming-guide.html#rdd-persistence). - Use more executors with smaller heap sizes. This will reduce the GC pressure within each JVM heap. +*** + +##### Important points to remember: +{:.no_toc} +- A DStream is associated with a single receiver. For attaining read parallelism multiple receivers i.e. multiple DStreams need to be created. A receiver is run within an executor. It occupies one core. Ensure that there are enough cores for processing after receiver slots are booked i.e. `spark.cores.max` should take the receiver slots into account. The receivers are allocated to executors in a round robin fashion. + +- When data is received from a stream source, receiver creates blocks of data. A new block of data is generated every blockInterval milliseconds. N blocks of data are created during the batchInterval where N = batchInterval/blockInterval. These blocks are distributed by the BlockManager of the current executor to the block managers of other executors. After that, the Network Input Tracker running on the driver is informed about the block locations for further processing. + +- An RDD is created on the driver for the blocks created during the batchInterval. The blocks generated during the batchInterval are partitions of the RDD. Each partition is a task in spark. blockInterval== batchinterval would mean that a single partition is created and probably it is processed locally. + +- The map tasks on the blocks are processed in the executors (one that received the block, and another where the block was replicated) that has the blocks irrespective of block interval, unless non-local scheduling kicks in. +Having bigger blockinterval means bigger blocks. A high value of `spark.locality.wait` increases the chance of processing a block on the local node. A balance needs to be found out between these two parameters to ensure that the bigger blocks are processed locally. + +- Instead of relying on batchInterval and blockInterval, you can define the number of partitions by calling `inputDstream.repartition(n)`. This reshuffles the data in RDD randomly to create n number of partitions. Yes, for greater parallelism. Though comes at the cost of a shuffle. An RDD's processing is scheduled by driver's jobscheduler as a job. At a given point of time only one job is active. So, if one job is executing the other jobs are queued. + +- If you have two dstreams there will be two RDDs formed and there will be two jobs created which will be scheduled one after the another. To avoid this, you can union two dstreams. This will ensure that a single unionRDD is formed for the two RDDs of the dstreams. This unionRDD is then considered as a single job. However the partitioning of the RDDs is not impacted. + +- If the batch processing time is more than batchinterval then obviously the receiver's memory will start filling up and will end up in throwing exceptions (most probably BlockNotFoundException). Currently there is no way to pause the receiver. Using SparkConf configuration `spark.streaming.receiver.maxRate`, rate of receiver can be limited. + *************************************************************************************************** *************************************************************************************************** @@ -2215,7 +2435,7 @@ The following table summarizes the semantics under failures: ### With Kafka Direct API {:.no_toc} -In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that all the Kafka data is received by Spark Streaming exactly once. Along with this, if you implement exactly-once output operation, you can achieve end-to-end exactly-once guarantees. This approach (experimental as of Spark {{site.SPARK_VERSION_SHORT}}) is further discussed in the [Kafka Integration Guide](streaming-kafka-integration.html). +In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that all the Kafka data is received by Spark Streaming exactly once. Along with this, if you implement exactly-once output operation, you can achieve end-to-end exactly-once guarantees. This approach is further discussed in the [Kafka Integration Guide](streaming-kafka-integration.html). ## Semantics of output operations {:.no_toc} @@ -2243,63 +2463,12 @@ additional effort may be necessary to achieve exactly-once semantics. There are *************************************************************************************************** *************************************************************************************************** -# Migration Guide from 0.9.1 or below to 1.x -Between Spark 0.9.1 and Spark 1.0, there were a few API changes made to ensure future API stability. -This section elaborates the steps required to migrate your existing code to 1.0. - -**Input DStreams**: All operations that create an input stream (e.g., `StreamingContext.socketStream`, -`FlumeUtils.createStream`, etc.) now returns -[InputDStream](api/scala/index.html#org.apache.spark.streaming.dstream.InputDStream) / -[ReceiverInputDStream](api/scala/index.html#org.apache.spark.streaming.dstream.ReceiverInputDStream) -(instead of DStream) for Scala, and [JavaInputDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaInputDStream.html) / -[JavaPairInputDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairInputDStream.html) / -[JavaReceiverInputDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaReceiverInputDStream.html) / -[JavaPairReceiverInputDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.html) -(instead of JavaDStream) for Java. This ensures that functionality specific to input streams can -be added to these classes in the future without breaking binary compatibility. -Note that your existing Spark Streaming applications should not require any change -(as these new classes are subclasses of DStream/JavaDStream) but may require recompilation with Spark 1.0. - -**Custom Network Receivers**: Since the release to Spark Streaming, custom network receivers could be defined -in Scala using the class NetworkReceiver. However, the API was limited in terms of error handling -and reporting, and could not be used from Java. Starting Spark 1.0, this class has been -replaced by [Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) which has -the following advantages. - -* Methods like `stop` and `restart` have been added to for better control of the lifecycle of a receiver. See -the [custom receiver guide](streaming-custom-receivers.html) for more details. -* Custom receivers can be implemented using both Scala and Java. - -To migrate your existing custom receivers from the earlier NetworkReceiver to the new Receiver, you have -to do the following. - -* Make your custom receiver class extend -[`org.apache.spark.streaming.receiver.Receiver`](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) -instead of `org.apache.spark.streaming.dstream.NetworkReceiver`. -* Earlier, a BlockGenerator object had to be created by the custom receiver, to which received data was -added for being stored in Spark. It had to be explicitly started and stopped from `onStart()` and `onStop()` -methods. The new Receiver class makes this unnecessary as it adds a set of methods named `store()` -that can be called to store the data in Spark. So, to migrate your custom network receiver, remove any -BlockGenerator object (does not exist any more in Spark 1.0 anyway), and use `store(...)` methods on -received data. - -**Actor-based Receivers**: Data could have been received using any Akka Actors by extending the actor class with -`org.apache.spark.streaming.receivers.Receiver` trait. This has been renamed to -[`org.apache.spark.streaming.receiver.ActorHelper`](api/scala/index.html#org.apache.spark.streaming.receiver.ActorHelper) -and the `pushBlock(...)` methods to store received data has been renamed to `store(...)`. Other helper classes in -the `org.apache.spark.streaming.receivers` package were also moved -to [`org.apache.spark.streaming.receiver`](api/scala/index.html#org.apache.spark.streaming.receiver.package) -package and renamed for better clarity. - -*************************************************************************************************** -*************************************************************************************************** - # Where to Go from Here * Additional guides - [Kafka Integration Guide](streaming-kafka-integration.html) - - [Flume Integration Guide](streaming-flume-integration.html) - [Kinesis Integration Guide](streaming-kinesis-integration.html) - [Custom Receiver Guide](streaming-custom-receivers.html) +* Third-party DStream data sources can be found in [Third Party Projects](http://spark.apache.org/third-party-projects.html) * API documentation - Scala docs * [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) and @@ -2307,9 +2476,6 @@ package and renamed for better clarity. * [KafkaUtils](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), [FlumeUtils](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$), [KinesisUtils](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisUtils$), - [TwitterUtils](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), - [ZeroMQUtils](api/scala/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and - [MQTTUtils](api/scala/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$) - Java docs * [JavaStreamingContext](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html), [JavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaDStream.html) and @@ -2317,9 +2483,6 @@ package and renamed for better clarity. * [KafkaUtils](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html), [FlumeUtils](api/java/index.html?org/apache/spark/streaming/flume/FlumeUtils.html), [KinesisUtils](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html) - [TwitterUtils](api/java/index.html?org/apache/spark/streaming/twitter/TwitterUtils.html), - [ZeroMQUtils](api/java/index.html?org/apache/spark/streaming/zeromq/ZeroMQUtils.html), and - [MQTTUtils](api/java/index.html?org/apache/spark/streaming/mqtt/MQTTUtils.html) - Python docs * [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) and [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream) * [KafkaUtils](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md new file mode 100644 index 000000000000..bab0be8ddeb9 --- /dev/null +++ b/docs/structured-streaming-kafka-integration.md @@ -0,0 +1,617 @@ +--- +layout: global +title: Structured Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher) +--- + +Structured Streaming integration for Kafka 0.10 to read data from and write data to Kafka. + +## Linking +For Scala/Java applications using SBT/Maven project definitions, link your application with the following artifact: + + groupId = org.apache.spark + artifactId = spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +For Python applications, you need to add this above library and its dependencies when deploying your +application. See the [Deploying](#deploying) subsection below. + +For experimenting on `spark-shell`, you need to add this above library and its dependencies too when invoking `spark-shell`. Also see the [Deploying](#deploying) subsection below. + +## Reading Data from Kafka + +### Creating a Kafka Source for Streaming Queries + +
    +
    +{% highlight scala %} + +// Subscribe to 1 topic +val df = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + +// Subscribe to multiple topics +val df = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1,topic2") + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + +// Subscribe to a pattern +val df = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + +{% endhighlight %} +
    +
    +{% highlight java %} + +// Subscribe to 1 topic +DataFrame df = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +// Subscribe to multiple topics +DataFrame df = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1,topic2") + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +// Subscribe to a pattern +DataFrame df = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +{% endhighlight %} +
    +
    +{% highlight python %} + +# Subscribe to 1 topic +df = spark \ + .readStream \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("subscribe", "topic1") \ + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +# Subscribe to multiple topics +df = spark \ + .readStream \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("subscribe", "topic1,topic2") \ + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +# Subscribe to a pattern +df = spark \ + .readStream \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("subscribePattern", "topic.*") \ + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +{% endhighlight %} +
    +
    + +### Creating a Kafka Source for Batch Queries +If you have a use case that is better suited to batch processing, +you can create an Dataset/DataFrame for a defined range of offsets. + +
    +
    +{% highlight scala %} + +// Subscribe to 1 topic defaults to the earliest and latest offsets +val df = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + +// Subscribe to multiple topics, specifying explicit Kafka offsets +val df = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1,topic2") + .option("startingOffsets", """{"topic1":{"0":23,"1":-2},"topic2":{"0":-2}}""") + .option("endingOffsets", """{"topic1":{"0":50,"1":-1},"topic2":{"0":-1}}""") + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + +// Subscribe to a pattern, at the earliest and latest offsets +val df = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + +{% endhighlight %} +
    +
    +{% highlight java %} + +// Subscribe to 1 topic defaults to the earliest and latest offsets +DataFrame df = spark + .read() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load(); +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); + +// Subscribe to multiple topics, specifying explicit Kafka offsets +DataFrame df = spark + .read() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1,topic2") + .option("startingOffsets", "{\"topic1\":{\"0\":23,\"1\":-2},\"topic2\":{\"0\":-2}}") + .option("endingOffsets", "{\"topic1\":{\"0\":50,\"1\":-1},\"topic2\":{\"0\":-1}}") + .load(); +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); + +// Subscribe to a pattern, at the earliest and latest offsets +DataFrame df = spark + .read() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + .load(); +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); + +{% endhighlight %} +
    +
    +{% highlight python %} + +# Subscribe to 1 topic defaults to the earliest and latest offsets +df = spark \ + .read \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("subscribe", "topic1") \ + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +# Subscribe to multiple topics, specifying explicit Kafka offsets +df = spark \ + .read \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("subscribe", "topic1,topic2") \ + .option("startingOffsets", """{"topic1":{"0":23,"1":-2},"topic2":{"0":-2}}""") \ + .option("endingOffsets", """{"topic1":{"0":50,"1":-1},"topic2":{"0":-1}}""") \ + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +# Subscribe to a pattern, at the earliest and latest offsets +df = spark \ + .read \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("subscribePattern", "topic.*") \ + .option("startingOffsets", "earliest") \ + .option("endingOffsets", "latest") \ + .load() +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") +{% endhighlight %} +
    +
    + +Each row in the source has the following schema: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    ColumnType
    keybinary
    valuebinary
    topicstring
    partitionint
    offsetlong
    timestamplong
    timestampTypeint
    + +The following options must be set for the Kafka source +for both batch and streaming queries. + + + + + + + + + + + + + + + + + + + + + + + +
    Optionvaluemeaning
    assignjson string {"topicA":[0,1],"topicB":[2,4]}Specific TopicPartitions to consume. + Only one of "assign", "subscribe" or "subscribePattern" + options can be specified for Kafka source.
    subscribeA comma-separated list of topicsThe topic list to subscribe. + Only one of "assign", "subscribe" or "subscribePattern" + options can be specified for Kafka source.
    subscribePatternJava regex stringThe pattern used to subscribe to topic(s). + Only one of "assign, "subscribe" or "subscribePattern" + options can be specified for Kafka source.
    kafka.bootstrap.serversA comma-separated list of host:portThe Kafka "bootstrap.servers" configuration.
    + +The following configurations are optional: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Optionvaluedefaultquery typemeaning
    startingOffsets"earliest", "latest" (streaming only), or json string + """ {"topicA":{"0":23,"1":-1},"topicB":{"0":-2}} """ + "latest" for streaming, "earliest" for batchstreaming and batchThe start point when a query is started, either "earliest" which is from the earliest offsets, + "latest" which is just from the latest offsets, or a json string specifying a starting offset for + each TopicPartition. In the json, -2 as an offset can be used to refer to earliest, -1 to latest. + Note: For batch queries, latest (either implicitly or by using -1 in json) is not allowed. + For streaming queries, this only applies when a new query is started, and that resuming will + always pick up from where the query left off. Newly discovered partitions during a query will start at + earliest.
    endingOffsetslatest or json string + {"topicA":{"0":23,"1":-1},"topicB":{"0":-1}} + latestbatch queryThe end point when a batch query is ended, either "latest" which is just referred to the + latest, or a json string specifying an ending offset for each TopicPartition. In the json, -1 + as an offset can be used to refer to latest, and -2 (earliest) as an offset is not allowed.
    failOnDataLosstrue or falsetruestreaming queryWhether to fail the query when it's possible that data is lost (e.g., topics are deleted, or + offsets are out of range). This may be a false alarm. You can disable it when it doesn't work + as you expected. Batch queries will always fail if it fails to read any data from the provided + offsets due to lost data.
    kafkaConsumer.pollTimeoutMslong512streaming and batchThe timeout in milliseconds to poll data from Kafka in executors.
    fetchOffset.numRetriesint3streaming and batchNumber of times to retry before giving up fetching Kafka offsets.
    fetchOffset.retryIntervalMslong10streaming and batchmilliseconds to wait before retrying to fetch Kafka offsets
    maxOffsetsPerTriggerlongnonestreaming and batchRate limit on maximum number of offsets processed per trigger interval. The specified total number of offsets will be proportionally split across topicPartitions of different volume.
    + +## Writing Data to Kafka + +Here, we describe the support for writing Streaming Queries and Batch Queries to Apache Kafka. Take note that +Apache Kafka only supports at least once write semantics. Consequently, when writing---either Streaming Queries +or Batch Queries---to Kafka, some records may be duplicated; this can happen, for example, if Kafka needs +to retry a message that was not acknowledged by a Broker, even though that Broker received and wrote the message record. +Structured Streaming cannot prevent such duplicates from occurring due to these Kafka write semantics. However, +if writing the query is successful, then you can assume that the query output was written at least once. A possible +solution to remove duplicates when reading the written data could be to introduce a primary (unique) key +that can be used to perform de-duplication when reading. + +The Dataframe being written to Kafka should have the following columns in schema: + + + + + + + + + + + + + + +
    ColumnType
    key (optional)string or binary
    value (required)string or binary
    topic (*optional)string
    +\* The topic column is required if the "topic" configuration option is not specified.
    + +The value column is the only required option. If a key column is not specified then +a ```null``` valued key column will be automatically added (see Kafka semantics on +how ```null``` valued key values are handled). If a topic column exists then its value +is used as the topic when writing the given row to Kafka, unless the "topic" configuration +option is set i.e., the "topic" configuration option overrides the topic column. + +The following options must be set for the Kafka sink +for both batch and streaming queries. + + + + + + + + +
    Optionvaluemeaning
    kafka.bootstrap.serversA comma-separated list of host:portThe Kafka "bootstrap.servers" configuration.
    + +The following configurations are optional: + + + + + + + + + + +
    Optionvaluedefaultquery typemeaning
    topicstringnonestreaming and batchSets the topic that all rows will be written to in Kafka. This option overrides any + topic column that may exist in the data.
    + +### Creating a Kafka Sink for Streaming Queries + +
    +
    +{% highlight scala %} + +// Write key-value data from a DataFrame to a specific Kafka topic specified in an option +val ds = df + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .writeStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("topic", "topic1") + .start() + +// Write key-value data from a DataFrame to Kafka using a topic specified in the data +val ds = df + .selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)") + .writeStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .start() + +{% endhighlight %} +
    +
    +{% highlight java %} + +// Write key-value data from a DataFrame to a specific Kafka topic specified in an option +StreamingQuery ds = df + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .writeStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("topic", "topic1") + .start() + +// Write key-value data from a DataFrame to Kafka using a topic specified in the data +StreamingQuery ds = df + .selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)") + .writeStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .start() + +{% endhighlight %} +
    +
    +{% highlight python %} + +# Write key-value data from a DataFrame to a specific Kafka topic specified in an option +ds = df \ + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") \ + .writeStream \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("topic", "topic1") \ + .start() + +# Write key-value data from a DataFrame to Kafka using a topic specified in the data +ds = df \ + .selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)") \ + .writeStream \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .start() + +{% endhighlight %} +
    +
    + +### Writing the output of Batch Queries to Kafka + +
    +
    +{% highlight scala %} + +// Write key-value data from a DataFrame to a specific Kafka topic specified in an option +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .write + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("topic", "topic1") + .save() + +// Write key-value data from a DataFrame to Kafka using a topic specified in the data +df.selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)") + .write + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .save() + +{% endhighlight %} +
    +
    +{% highlight java %} + +// Write key-value data from a DataFrame to a specific Kafka topic specified in an option +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .write() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("topic", "topic1") + .save() + +// Write key-value data from a DataFrame to Kafka using a topic specified in the data +df.selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)") + .write() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .save() + +{% endhighlight %} +
    +
    +{% highlight python %} + +# Write key-value data from a DataFrame to a specific Kafka topic specified in an option +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") \ + .write \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("topic", "topic1") \ + .save() + +# Write key-value data from a DataFrame to Kafka using a topic specified in the data +df.selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)") \ + .write \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .save() + +{% endhighlight %} +
    +
    + + +## Kafka Specific Configurations + +Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, +`stream.option("kafka.bootstrap.servers", "host:port")`. For possible kafka parameters, see +[Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs) for +parameters related to reading data, and [Kafka producer config docs](http://kafka.apache.org/documentation/#producerconfigs) +for parameters related to writing data. + +Note that the following Kafka params cannot be set and the Kafka source or sink will throw an exception: + +- **group.id**: Kafka source will create a unique group id for each query automatically. +- **auto.offset.reset**: Set the source option `startingOffsets` to specify + where to start instead. Structured Streaming manages which offsets are consumed internally, rather + than rely on the kafka Consumer to do it. This will ensure that no data is missed when new + topics/partitions are dynamically subscribed. Note that `startingOffsets` only applies when a new + streaming query is started, and that resuming will always pick up from where the query left off. +- **key.deserializer**: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use + DataFrame operations to explicitly deserialize the keys. +- **value.deserializer**: Values are always deserialized as byte arrays with ByteArrayDeserializer. + Use DataFrame operations to explicitly deserialize the values. +- **key.serializer**: Keys are always serialized with ByteArraySerializer or StringSerializer. Use +DataFrame operations to explicitly serialize the keys into either strings or byte arrays. +- **value.serializer**: values are always serialized with ByteArraySerializer or StringSerializer. Use +DataFrame oeprations to explicitly serialize the values into either strings or byte arrays. +- **enable.auto.commit**: Kafka source doesn't commit any offset. +- **interceptor.classes**: Kafka source always read keys and values as byte arrays. It's not safe to + use ConsumerInterceptor as it may break the query. + +## Deploying + +As with any Spark applications, `spark-submit` is used to launch your application. `spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` +and its dependencies can be directly added to `spark-submit` using `--packages`, such as, + + ./bin/spark-submit --packages org.apache.spark:spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... + +For experimenting on `spark-shell`, you can also use `--packages` to add `spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` and its dependencies directly, + + ./bin/spark-shell --packages org.apache.spark:spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... + +See [Application Submission Guide](submitting-applications.html) for more details about submitting +applications with external dependencies. diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md new file mode 100644 index 000000000000..93bef8d5bb7e --- /dev/null +++ b/docs/structured-streaming-programming-guide.md @@ -0,0 +1,2119 @@ +--- +layout: global +displayTitle: Structured Streaming Programming Guide +title: Structured Streaming Programming Guide +--- + +* This will become a table of contents (this text will be scraped). +{:toc} + +# Overview +Structured Streaming is a scalable and fault-tolerant stream processing engine built on the Spark SQL engine. You can express your streaming computation the same way you would express a batch computation on static data. The Spark SQL engine will take care of running it incrementally and continuously and updating the final result as streaming data continues to arrive. You can use the [Dataset/DataFrame API](sql-programming-guide.html) in Scala, Java, Python or R to express streaming aggregations, event-time windows, stream-to-batch joins, etc. The computation is executed on the same optimized Spark SQL engine. Finally, the system ensures end-to-end exactly-once fault-tolerance guarantees through checkpointing and Write Ahead Logs. In short, *Structured Streaming provides fast, scalable, fault-tolerant, end-to-end exactly-once stream processing without the user having to reason about streaming.* + +In this guide, we are going to walk you through the programming model and the APIs. First, let's start with a simple example - a streaming word count. + +# Quick Example +Let’s say you want to maintain a running word count of text data received from a data server listening on a TCP socket. Let’s see how you can express this using Structured Streaming. You can see the full code in +[Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala)/[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java)/[Python]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/sql/streaming/structured_network_wordcount.py)/[R]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/r/streaming/structured_network_wordcount.R). +And if you [download Spark](http://spark.apache.org/downloads.html), you can directly [run the example](index.html#running-the-examples-and-shell). In any case, let’s walk through the example step-by-step and understand how it works. First, we have to import the necessary classes and create a local SparkSession, the starting point of all functionalities related to Spark. + +
    +
    + +{% highlight scala %} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.SparkSession + +val spark = SparkSession + .builder + .appName("StructuredNetworkWordCount") + .getOrCreate() + +import spark.implicits._ +{% endhighlight %} + +
    +
    + +{% highlight java %} +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.sql.*; +import org.apache.spark.sql.streaming.StreamingQuery; + +import java.util.Arrays; +import java.util.Iterator; + +SparkSession spark = SparkSession + .builder() + .appName("JavaStructuredNetworkWordCount") + .getOrCreate(); +{% endhighlight %} + +
    +
    + +{% highlight python %} +from pyspark.sql import SparkSession +from pyspark.sql.functions import explode +from pyspark.sql.functions import split + +spark = SparkSession \ + .builder \ + .appName("StructuredNetworkWordCount") \ + .getOrCreate() +{% endhighlight %} + +
    +
    + +{% highlight r %} +sparkR.session(appName = "StructuredNetworkWordCount") +{% endhighlight %} + +
    +
    + +Next, let’s create a streaming DataFrame that represents text data received from a server listening on localhost:9999, and transform the DataFrame to calculate word counts. + +
    +
    + +{% highlight scala %} +// Create DataFrame representing the stream of input lines from connection to localhost:9999 +val lines = spark.readStream + .format("socket") + .option("host", "localhost") + .option("port", 9999) + .load() + +// Split the lines into words +val words = lines.as[String].flatMap(_.split(" ")) + +// Generate running word count +val wordCounts = words.groupBy("value").count() +{% endhighlight %} + +This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named "value", and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have converted the DataFrame to a Dataset of String using `.as[String]`, so that we can apply the `flatMap` operation to split each line into multiple words. The resultant `words` Dataset contains all the words. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream. + +
    +
    + +{% highlight java %} +// Create DataFrame representing the stream of input lines from connection to localhost:9999 +Dataset lines = spark + .readStream() + .format("socket") + .option("host", "localhost") + .option("port", 9999) + .load(); + +// Split the lines into words +Dataset words = lines + .as(Encoders.STRING()) + .flatMap((FlatMapFunction) x -> Arrays.asList(x.split(" ")).iterator(), Encoders.STRING()); + +// Generate running word count +Dataset wordCounts = words.groupBy("value").count(); +{% endhighlight %} + +This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named "value", and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have converted the DataFrame to a Dataset of String using `.as(Encoders.STRING())`, so that we can apply the `flatMap` operation to split each line into multiple words. The resultant `words` Dataset contains all the words. Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream. + +
    +
    + +{% highlight python %} +# Create DataFrame representing the stream of input lines from connection to localhost:9999 +lines = spark \ + .readStream \ + .format("socket") \ + .option("host", "localhost") \ + .option("port", 9999) \ + .load() + +# Split the lines into words +words = lines.select( + explode( + split(lines.value, " ") + ).alias("word") +) + +# Generate running word count +wordCounts = words.groupBy("word").count() +{% endhighlight %} + +This `lines` DataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named "value", and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have used two built-in SQL functions - split and explode, to split each line into multiple rows with a word each. In addition, we use the function `alias` to name the new column as "word". Finally, we have defined the `wordCounts` DataFrame by grouping by the unique values in the Dataset and counting them. Note that this is a streaming DataFrame which represents the running word counts of the stream. + +
    +
    + +{% highlight r %} +# Create DataFrame representing the stream of input lines from connection to localhost:9999 +lines <- read.stream("socket", host = "localhost", port = 9999) + +# Split the lines into words +words <- selectExpr(lines, "explode(split(value, ' ')) as word") + +# Generate running word count +wordCounts <- count(group_by(words, "word")) +{% endhighlight %} + +This `lines` SparkDataFrame represents an unbounded table containing the streaming text data. This table contains one column of strings named "value", and each line in the streaming text data becomes a row in the table. Note, that this is not currently receiving any data as we are just setting up the transformation, and have not yet started it. Next, we have a SQL expression with two SQL functions - split and explode, to split each line into multiple rows with a word each. In addition, we name the new column as "word". Finally, we have defined the `wordCounts` SparkDataFrame by grouping by the unique values in the SparkDataFrame and counting them. Note that this is a streaming SparkDataFrame which represents the running word counts of the stream. + +
    +
    + +We have now set up the query on the streaming data. All that is left is to actually start receiving data and computing the counts. To do this, we set it up to print the complete set of counts (specified by `outputMode("complete")`) to the console every time they are updated. And then start the streaming computation using `start()`. + +
    +
    + +{% highlight scala %} +// Start running the query that prints the running counts to the console +val query = wordCounts.writeStream + .outputMode("complete") + .format("console") + .start() + +query.awaitTermination() +{% endhighlight %} + +
    +
    + +{% highlight java %} +// Start running the query that prints the running counts to the console +StreamingQuery query = wordCounts.writeStream() + .outputMode("complete") + .format("console") + .start(); + +query.awaitTermination(); +{% endhighlight %} + +
    +
    + +{% highlight python %} + # Start running the query that prints the running counts to the console +query = wordCounts \ + .writeStream \ + .outputMode("complete") \ + .format("console") \ + .start() + +query.awaitTermination() +{% endhighlight %} + +
    +
    + +{% highlight r %} +# Start running the query that prints the running counts to the console +query <- write.stream(wordCounts, "console", outputMode = "complete") + +awaitTermination(query) +{% endhighlight %} + +
    +
    + +After this code is executed, the streaming computation will have started in the background. The `query` object is a handle to that active streaming query, and we have decided to wait for the termination of the query using `awaitTermination()` to prevent the process from exiting while the query is active. + +To actually execute this example code, you can either compile the code in your own +[Spark application](quick-start.html#self-contained-applications), or simply +[run the example](index.html#running-the-examples-and-shell) once you have downloaded Spark. We are showing the latter. You will first need to run Netcat (a small utility found in most Unix-like systems) as a data server by using + + + $ nc -lk 9999 + +Then, in a different terminal, you can start the example by using + +
    +
    +{% highlight bash %} +$ ./bin/run-example org.apache.spark.examples.sql.streaming.StructuredNetworkWordCount localhost 9999 +{% endhighlight %} +
    +
    +{% highlight bash %} +$ ./bin/run-example org.apache.spark.examples.sql.streaming.JavaStructuredNetworkWordCount localhost 9999 +{% endhighlight %} +
    +
    +{% highlight bash %} +$ ./bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py localhost 9999 +{% endhighlight %} +
    +
    +{% highlight bash %} +$ ./bin/spark-submit examples/src/main/r/streaming/structured_network_wordcount.R localhost 9999 +{% endhighlight %} +
    +
    + +Then, any lines typed in the terminal running the netcat server will be counted and printed on screen every second. It will look something like the following. + + + + + +
    +{% highlight bash %} +# TERMINAL 1: +# Running Netcat + +$ nc -lk 9999 +apache spark +apache hadoop + + + + + + + + + + + + + + + + + + + +... +{% endhighlight %} + +
    + +
    +{% highlight bash %} +# TERMINAL 2: RUNNING StructuredNetworkWordCount + +$ ./bin/run-example org.apache.spark.examples.sql.streaming.StructuredNetworkWordCount localhost 9999 + +------------------------------------------- +Batch: 0 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 1| +| spark| 1| ++------+-----+ + +------------------------------------------- +Batch: 1 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 2| +| spark| 1| +|hadoop| 1| ++------+-----+ +... +{% endhighlight %} +
    + +
    +{% highlight bash %} +# TERMINAL 2: RUNNING JavaStructuredNetworkWordCount + +$ ./bin/run-example org.apache.spark.examples.sql.streaming.JavaStructuredNetworkWordCount localhost 9999 + +------------------------------------------- +Batch: 0 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 1| +| spark| 1| ++------+-----+ + +------------------------------------------- +Batch: 1 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 2| +| spark| 1| +|hadoop| 1| ++------+-----+ +... +{% endhighlight %} +
    +
    +{% highlight bash %} +# TERMINAL 2: RUNNING structured_network_wordcount.py + +$ ./bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py localhost 9999 + +------------------------------------------- +Batch: 0 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 1| +| spark| 1| ++------+-----+ + +------------------------------------------- +Batch: 1 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 2| +| spark| 1| +|hadoop| 1| ++------+-----+ +... +{% endhighlight %} +
    +
    +{% highlight bash %} +# TERMINAL 2: RUNNING structured_network_wordcount.R + +$ ./bin/spark-submit examples/src/main/r/streaming/structured_network_wordcount.R localhost 9999 + +------------------------------------------- +Batch: 0 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 1| +| spark| 1| ++------+-----+ + +------------------------------------------- +Batch: 1 +------------------------------------------- ++------+-----+ +| value|count| ++------+-----+ +|apache| 2| +| spark| 1| +|hadoop| 1| ++------+-----+ +... +{% endhighlight %} +
    +
    +
    + + +# Programming Model + +The key idea in Structured Streaming is to treat a live data stream as a +table that is being continuously appended. This leads to a new stream +processing model that is very similar to a batch processing model. You will +express your streaming computation as standard batch-like query as on a static +table, and Spark runs it as an *incremental* query on the *unbounded* input +table. Let’s understand this model in more detail. + +## Basic Concepts +Consider the input data stream as the "Input Table". Every data item that is +arriving on the stream is like a new row being appended to the Input Table. + +![Stream as a Table](img/structured-streaming-stream-as-a-table.png "Stream as a Table") + +A query on the input will generate the "Result Table". Every trigger interval (say, every 1 second), new rows get appended to the Input Table, which eventually updates the Result Table. Whenever the result table gets updated, we would want to write the changed result rows to an external sink. + +![Model](img/structured-streaming-model.png) + +The "Output" is defined as what gets written out to the external storage. The output can be defined in a different mode: + + - *Complete Mode* - The entire updated Result Table will be written to the external storage. It is up to the storage connector to decide how to handle writing of the entire table. + + - *Append Mode* - Only the new rows appended in the Result Table since the last trigger will be written to the external storage. This is applicable only on the queries where existing rows in the Result Table are not expected to change. + + - *Update Mode* - Only the rows that were updated in the Result Table since the last trigger will be written to the external storage (available since Spark 2.1.1). Note that this is different from the Complete Mode in that this mode only outputs the rows that have changed since the last trigger. If the query doesn't contain aggregations, it will be equivalent to Append mode. + +Note that each mode is applicable on certain types of queries. This is discussed in detail [later](#output-modes). + +To illustrate the use of this model, let’s understand the model in context of +the [Quick Example](#quick-example) above. The first `lines` DataFrame is the input table, and +the final `wordCounts` DataFrame is the result table. Note that the query on +streaming `lines` DataFrame to generate `wordCounts` is *exactly the same* as +it would be a static DataFrame. However, when this query is started, Spark +will continuously check for new data from the socket connection. If there is +new data, Spark will run an "incremental" query that combines the previous +running counts with the new data to compute updated counts, as shown below. + +![Model](img/structured-streaming-example-model.png) + +**Note that Structured Streaming does not materialize the entire table**. It reads the latest +available data from the streaming data source, processes it incrementally to update the result, +and then discards the source data. It only keeps around the minimal intermediate *state* data as +required to update the result (e.g. intermediate counts in the earlier example). + +This model is significantly different from many other stream processing +engines. Many streaming systems require the user to maintain running +aggregations themselves, thus having to reason about fault-tolerance, and +data consistency (at-least-once, or at-most-once, or exactly-once). In this +model, Spark is responsible for updating the Result Table when there is new +data, thus relieving the users from reasoning about it. As an example, let’s +see how this model handles event-time based processing and late arriving data. + +## Handling Event-time and Late Data +Event-time is the time embedded in the data itself. For many applications, you may want to operate on this event-time. For example, if you want to get the number of events generated by IoT devices every minute, then you probably want to use the time when the data was generated (that is, event-time in the data), rather than the time Spark receives them. This event-time is very naturally expressed in this model -- each event from the devices is a row in the table, and event-time is a column value in the row. This allows window-based aggregations (e.g. number of events every minute) to be just a special type of grouping and aggregation on the event-time column -- each time window is a group and each row can belong to multiple windows/groups. Therefore, such event-time-window-based aggregation queries can be defined consistently on both a static dataset (e.g. from collected device events logs) as well as on a data stream, making the life of the user much easier. + +Furthermore, this model naturally handles data that has arrived later than +expected based on its event-time. Since Spark is updating the Result Table, +it has full control over updating old aggregates when there is late data, +as well as cleaning up old aggregates to limit the size of intermediate +state data. Since Spark 2.1, we have support for watermarking which +allows the user to specify the threshold of late data, and allows the engine +to accordingly clean up old state. These are explained later in more +detail in the [Window Operations](#window-operations-on-event-time) section. + +## Fault Tolerance Semantics +Delivering end-to-end exactly-once semantics was one of key goals behind the design of Structured Streaming. To achieve that, we have designed the Structured Streaming sources, the sinks and the execution engine to reliably track the exact progress of the processing so that it can handle any kind of failure by restarting and/or reprocessing. Every streaming source is assumed to have offsets (similar to Kafka offsets, or Kinesis sequence numbers) +to track the read position in the stream. The engine uses checkpointing and write ahead logs to record the offset range of the data being processed in each trigger. The streaming sinks are designed to be idempotent for handling reprocessing. Together, using replayable sources and idempotent sinks, Structured Streaming can ensure **end-to-end exactly-once semantics** under any failure. + +# API using Datasets and DataFrames +Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as well as streaming, unbounded data. Similar to static Datasets/DataFrames, you can use the common entry point `SparkSession` +([Scala](api/scala/index.html#org.apache.spark.sql.SparkSession)/[Java](api/java/org/apache/spark/sql/SparkSession.html)/[Python](api/python/pyspark.sql.html#pyspark.sql.SparkSession)/[R](api/R/sparkR.session.html) docs) +to create streaming DataFrames/Datasets from streaming sources, and apply the same operations on them as static DataFrames/Datasets. If you are not familiar with Datasets/DataFrames, you are strongly advised to familiarize yourself with them using the +[DataFrame/Dataset Programming Guide](sql-programming-guide.html). + +## Creating streaming DataFrames and streaming Datasets +Streaming DataFrames can be created through the `DataStreamReader` interface +([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamReader)/[Java](api/java/org/apache/spark/sql/streaming/DataStreamReader.html)/[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) +returned by `SparkSession.readStream()`. In [R](api/R/read.stream.html), with the `read.stream()` method. Similar to the read interface for creating static DataFrame, you can specify the details of the source – data format, schema, options, etc. + +#### Input Sources +There are a few built-in sources. + + - **File source** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations. + + - **Kafka source** - Reads data from Kafka. It's compatible with Kafka broker versions 0.10.0 or higher. See the [Kafka Integration Guide](structured-streaming-kafka-integration.html) for more details. + + - **Socket source (for testing)** - Reads UTF8 text data from a socket connection. The listening server socket is at the driver. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees. + + - **Rate source (for testing)** - Generates data at the specified number of rows per second, each output row contains a `timestamp` and `value`. Where `timestamp` is a `Timestamp` type containing the time of message dispatch, and `value` is of `Long` type containing the message count, starting from 0 as the first row. This source is intended for testing and benchmarking. + +Some sources are not fault-tolerant because they do not guarantee that data can be replayed using +checkpointed offsets after a failure. See the earlier section on +[fault-tolerance semantics](#fault-tolerance-semantics). +Here are the details of all the sources in Spark. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    SourceOptionsFault-tolerantNotes
    File source + path: path to the input directory, and common to all file formats. +
    + maxFilesPerTrigger: maximum number of new files to be considered in every trigger (default: no max) +
    + latestFirst: whether to processs the latest new files first, useful when there is a large backlog of files (default: false) +
    + fileNameOnly: whether to check new files based on only the filename instead of on the full path (default: false). With this set to `true`, the following files would be considered as the same file, because their filenames, "dataset.txt", are the same: +
    + "file:///dataset.txt"
    + "s3://a/dataset.txt"
    + "s3n://a/b/dataset.txt"
    + "s3a://a/b/c/dataset.txt"
    +

    + For file-format-specific options, see the related methods in DataStreamReader + (Scala/Java/Python/R). + E.g. for "parquet" format options see DataStreamReader.parquet(). +

    + In addition, there are session configurations that affect certain file-formats. See the SQL Programming Guide for more details. E.g., for "parquet", see Parquet configuration section. +
    YesSupports glob paths, but does not support multiple comma-separated paths/globs.
    Socket Source + host: host to connect to, must be specified
    + port: port to connect to, must be specified +
    No
    Rate Source + rowsPerSecond (e.g. 100, default: 1): How many rows should be generated per second.

    + rampUpTime (e.g. 5s, default: 0s): How long to ramp up before the generating speed becomes rowsPerSecond. Using finer granularities than seconds will be truncated to integer seconds.

    + numPartitions (e.g. 10, default: Spark's default parallelism): The partition number for the generated rows.

    + + The source will try its best to reach rowsPerSecond, but the query may be resource constrained, and numPartitions can be tweaked to help reach the desired speed. +
    Yes
    Kafka Source + See the Kafka Integration Guide. + Yes
    + +Here are some examples. + +
    +
    + +{% highlight scala %} +val spark: SparkSession = ... + +// Read text from socket +val socketDF = spark + .readStream + .format("socket") + .option("host", "localhost") + .option("port", 9999) + .load() + +socketDF.isStreaming // Returns True for DataFrames that have streaming sources + +socketDF.printSchema + +// Read all the csv files written atomically in a directory +val userSchema = new StructType().add("name", "string").add("age", "integer") +val csvDF = spark + .readStream + .option("sep", ";") + .schema(userSchema) // Specify schema of the csv files + .csv("/path/to/directory") // Equivalent to format("csv").load("/path/to/directory") +{% endhighlight %} + +
    +
    + +{% highlight java %} +SparkSession spark = ... + +// Read text from socket +Dataset socketDF = spark + .readStream() + .format("socket") + .option("host", "localhost") + .option("port", 9999) + .load(); + +socketDF.isStreaming(); // Returns True for DataFrames that have streaming sources + +socketDF.printSchema(); + +// Read all the csv files written atomically in a directory +StructType userSchema = new StructType().add("name", "string").add("age", "integer"); +Dataset csvDF = spark + .readStream() + .option("sep", ";") + .schema(userSchema) // Specify schema of the csv files + .csv("/path/to/directory"); // Equivalent to format("csv").load("/path/to/directory") +{% endhighlight %} + +
    +
    + +{% highlight python %} +spark = SparkSession. ... + +# Read text from socket +socketDF = spark \ + .readStream \ + .format("socket") \ + .option("host", "localhost") \ + .option("port", 9999) \ + .load() + +socketDF.isStreaming() # Returns True for DataFrames that have streaming sources + +socketDF.printSchema() + +# Read all the csv files written atomically in a directory +userSchema = StructType().add("name", "string").add("age", "integer") +csvDF = spark \ + .readStream \ + .option("sep", ";") \ + .schema(userSchema) \ + .csv("/path/to/directory") # Equivalent to format("csv").load("/path/to/directory") +{% endhighlight %} + +
    +
    + +{% highlight r %} +sparkR.session(...) + +# Read text from socket +socketDF <- read.stream("socket", host = hostname, port = port) + +isStreaming(socketDF) # Returns TRUE for SparkDataFrames that have streaming sources + +printSchema(socketDF) + +# Read all the csv files written atomically in a directory +schema <- structType(structField("name", "string"), + structField("age", "integer")) +csvDF <- read.stream("csv", path = "/path/to/directory", schema = schema, sep = ";") +{% endhighlight %} + +
    +
    + +These examples generate streaming DataFrames that are untyped, meaning that the schema of the DataFrame is not checked at compile time, only checked at runtime when the query is submitted. Some operations like `map`, `flatMap`, etc. need the type to be known at compile time. To do those, you can convert these untyped streaming DataFrames to typed streaming Datasets using the same methods as static DataFrame. See the [SQL Programming Guide](sql-programming-guide.html) for more details. Additionally, more details on the supported streaming sources are discussed later in the document. + +### Schema inference and partition of streaming DataFrames/Datasets + +By default, Structured Streaming from file based sources requires you to specify the schema, rather than rely on Spark to infer it automatically. This restriction ensures a consistent schema will be used for the streaming query, even in the case of failures. For ad-hoc use cases, you can reenable schema inference by setting `spark.sql.streaming.schemaInference` to `true`. + +Partition discovery does occur when subdirectories that are named `/key=value/` are present and listing will automatically recurse into these directories. If these columns appear in the user provided schema, they will be filled in by Spark based on the path of the file being read. The directories that make up the partitioning scheme must be present when the query starts and must remain static. For example, it is okay to add `/data/year=2016/` when `/data/year=2015/` was present, but it is invalid to change the partitioning column (i.e. by creating the directory `/data/date=2016-04-17/`). + +## Operations on streaming DataFrames/Datasets +You can apply all kinds of operations on streaming DataFrames/Datasets – ranging from untyped, SQL-like operations (e.g. `select`, `where`, `groupBy`), to typed RDD-like operations (e.g. `map`, `filter`, `flatMap`). See the [SQL programming guide](sql-programming-guide.html) for more details. Let’s take a look at a few example operations that you can use. + +### Basic Operations - Selection, Projection, Aggregation +Most of the common operations on DataFrame/Dataset are supported for streaming. The few operations that are not supported are [discussed later](#unsupported-operations) in this section. + +
    +
    + +{% highlight scala %} +case class DeviceData(device: String, deviceType: String, signal: Double, time: DateTime) + +val df: DataFrame = ... // streaming DataFrame with IOT device data with schema { device: string, deviceType: string, signal: double, time: string } +val ds: Dataset[DeviceData] = df.as[DeviceData] // streaming Dataset with IOT device data + +// Select the devices which have signal more than 10 +df.select("device").where("signal > 10") // using untyped APIs +ds.filter(_.signal > 10).map(_.device) // using typed APIs + +// Running count of the number of updates for each device type +df.groupBy("deviceType").count() // using untyped API + +// Running average signal for each device type +import org.apache.spark.sql.expressions.scalalang.typed +ds.groupByKey(_.deviceType).agg(typed.avg(_.signal)) // using typed API +{% endhighlight %} + +
    +
    + +{% highlight java %} +import org.apache.spark.api.java.function.*; +import org.apache.spark.sql.*; +import org.apache.spark.sql.expressions.javalang.typed; +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; + +public class DeviceData { + private String device; + private String deviceType; + private Double signal; + private java.sql.Date time; + ... + // Getter and setter methods for each field +} + +Dataset df = ...; // streaming DataFrame with IOT device data with schema { device: string, type: string, signal: double, time: DateType } +Dataset ds = df.as(ExpressionEncoder.javaBean(DeviceData.class)); // streaming Dataset with IOT device data + +// Select the devices which have signal more than 10 +df.select("device").where("signal > 10"); // using untyped APIs +ds.filter((FilterFunction) value -> value.getSignal() > 10) + .map((MapFunction) value -> value.getDevice(), Encoders.STRING()); + +// Running count of the number of updates for each device type +df.groupBy("deviceType").count(); // using untyped API + +// Running average signal for each device type +ds.groupByKey((MapFunction) value -> value.getDeviceType(), Encoders.STRING()) + .agg(typed.avg((MapFunction) value -> value.getSignal())); +{% endhighlight %} + + +
    +
    + +{% highlight python %} +df = ... # streaming DataFrame with IOT device data with schema { device: string, deviceType: string, signal: double, time: DateType } + +# Select the devices which have signal more than 10 +df.select("device").where("signal > 10") + +# Running count of the number of updates for each device type +df.groupBy("deviceType").count() +{% endhighlight %} +
    +
    + +{% highlight r %} +df <- ... # streaming DataFrame with IOT device data with schema { device: string, deviceType: string, signal: double, time: DateType } + +# Select the devices which have signal more than 10 +select(where(df, "signal > 10"), "device") + +# Running count of the number of updates for each device type +count(groupBy(df, "deviceType")) +{% endhighlight %} +
    +
    + +You can also register a streaming DataFrame/Dataset as a temporary view and then apply SQL commands on it. + +
    +
    +{% highlight scala %} +df.createOrReplaceTempView("updates") +spark.sql("select count(*) from updates") // returns another streaming DF +{% endhighlight %} +
    +
    +{% highlight java %} +df.createOrReplaceTempView("updates"); +spark.sql("select count(*) from updates"); // returns another streaming DF +{% endhighlight %} +
    +
    +{% highlight python %} +df.createOrReplaceTempView("updates") +spark.sql("select count(*) from updates") # returns another streaming DF +{% endhighlight %} +
    +
    +{% highlight r %} +createOrReplaceTempView(df, "updates") +sql("select count(*) from updates") +{% endhighlight %} +
    +
    + +Note, you can identify whether a DataFrame/Dataset has streaming data or not by using `df.isStreaming`. + +
    +
    +{% highlight scala %} +df.isStreaming +{% endhighlight %} +
    +
    +{% highlight java %} +df.isStreaming() +{% endhighlight %} +
    +
    +{% highlight python %} +df.isStreaming() +{% endhighlight %} +
    +
    +{% highlight bash %} +Not available. +{% endhighlight %} +
    +
    + +### Window Operations on Event Time +Aggregations over a sliding event-time window are straightforward with Structured Streaming and are very similar to grouped aggregations. In a grouped aggregation, aggregate values (e.g. counts) are maintained for each unique value in the user-specified grouping column. In case of window-based aggregations, aggregate values are maintained for each window the event-time of a row falls into. Let's understand this with an illustration. + +Imagine our [quick example](#quick-example) is modified and the stream now contains lines along with the time when the line was generated. Instead of running word counts, we want to count words within 10 minute windows, updating every 5 minutes. That is, word counts in words received between 10 minute windows 12:00 - 12:10, 12:05 - 12:15, 12:10 - 12:20, etc. Note that 12:00 - 12:10 means data that arrived after 12:00 but before 12:10. Now, consider a word that was received at 12:07. This word should increment the counts corresponding to two windows 12:00 - 12:10 and 12:05 - 12:15. So the counts will be indexed by both, the grouping key (i.e. the word) and the window (can be calculated from the event-time). + +The result tables would look something like the following. + +![Window Operations](img/structured-streaming-window.png) + +Since this windowing is similar to grouping, in code, you can use `groupBy()` and `window()` operations to express windowed aggregations. You can see the full code for the below examples in +[Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala)/[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCountWindowed.java)/[Python]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py). + +
    +
    + +{% highlight scala %} +import spark.implicits._ + +val words = ... // streaming DataFrame of schema { timestamp: Timestamp, word: String } + +// Group the data by window and word and compute the count of each group +val windowedCounts = words.groupBy( + window($"timestamp", "10 minutes", "5 minutes"), + $"word" +).count() +{% endhighlight %} + +
    +
    + +{% highlight java %} +Dataset words = ... // streaming DataFrame of schema { timestamp: Timestamp, word: String } + +// Group the data by window and word and compute the count of each group +Dataset windowedCounts = words.groupBy( + functions.window(words.col("timestamp"), "10 minutes", "5 minutes"), + words.col("word") +).count(); +{% endhighlight %} + +
    +
    +{% highlight python %} +words = ... # streaming DataFrame of schema { timestamp: Timestamp, word: String } + +# Group the data by window and word and compute the count of each group +windowedCounts = words.groupBy( + window(words.timestamp, "10 minutes", "5 minutes"), + words.word +).count() +{% endhighlight %} + +
    +
    + + +### Handling Late Data and Watermarking +Now consider what happens if one of the events arrives late to the application. +For example, say, a word generated at 12:04 (i.e. event time) could be received by +the application at 12:11. The application should use the time 12:04 instead of 12:11 +to update the older counts for the window `12:00 - 12:10`. This occurs +naturally in our window-based grouping – Structured Streaming can maintain the intermediate state +for partial aggregates for a long period of time such that late data can update aggregates of +old windows correctly, as illustrated below. + +![Handling Late Data](img/structured-streaming-late-data.png) + +However, to run this query for days, it's necessary for the system to bound the amount of +intermediate in-memory state it accumulates. This means the system needs to know when an old +aggregate can be dropped from the in-memory state because the application is not going to receive +late data for that aggregate any more. To enable this, in Spark 2.1, we have introduced +**watermarking**, which lets the engine automatically track the current event time in the data +and attempt to clean up old state accordingly. You can define the watermark of a query by +specifying the event time column and the threshold on how late the data is expected to be in terms of +event time. For a specific window starting at time `T`, the engine will maintain state and allow late +data to update the state until `(max event time seen by the engine - late threshold > T)`. +In other words, late data within the threshold will be aggregated, +but data later than the threshold will be dropped. Let's understand this with an example. We can +easily define watermarking on the previous example using `withWatermark()` as shown below. + +
    +
    + +{% highlight scala %} +import spark.implicits._ + +val words = ... // streaming DataFrame of schema { timestamp: Timestamp, word: String } + +// Group the data by window and word and compute the count of each group +val windowedCounts = words + .withWatermark("timestamp", "10 minutes") + .groupBy( + window($"timestamp", "10 minutes", "5 minutes"), + $"word") + .count() +{% endhighlight %} + +
    +
    + +{% highlight java %} +Dataset words = ... // streaming DataFrame of schema { timestamp: Timestamp, word: String } + +// Group the data by window and word and compute the count of each group +Dataset windowedCounts = words + .withWatermark("timestamp", "10 minutes") + .groupBy( + functions.window(words.col("timestamp"), "10 minutes", "5 minutes"), + words.col("word")) + .count(); +{% endhighlight %} + +
    +
    +{% highlight python %} +words = ... # streaming DataFrame of schema { timestamp: Timestamp, word: String } + +# Group the data by window and word and compute the count of each group +windowedCounts = words \ + .withWatermark("timestamp", "10 minutes") \ + .groupBy( + window(words.timestamp, "10 minutes", "5 minutes"), + words.word) \ + .count() +{% endhighlight %} + +
    +
    + +In this example, we are defining the watermark of the query on the value of the column "timestamp", +and also defining "10 minutes" as the threshold of how late is the data allowed to be. If this query +is run in Update output mode (discussed later in [Output Modes](#output-modes) section), +the engine will keep updating counts of a window in the Result Table until the window is older +than the watermark, which lags behind the current event time in column "timestamp" by 10 minutes. +Here is an illustration. + +![Watermarking in Update Mode](img/structured-streaming-watermark-update-mode.png) + +As shown in the illustration, the maximum event time tracked by the engine is the +*blue dashed line*, and the watermark set as `(max event time - '10 mins')` +at the beginning of every trigger is the red line For example, when the engine observes the data +`(12:14, dog)`, it sets the watermark for the next trigger as `12:04`. +This watermark lets the engine maintain intermediate state for additional 10 minutes to allow late +data to be counted. For example, the data `(12:09, cat)` is out of order and late, and it falls in +windows `12:00 - 12:10` and `12:05 - 12:15`. Since, it is still ahead of the watermark `12:04` in +the trigger, the engine still maintains the intermediate counts as state and correctly updates the +counts of the related windows. However, when the watermark is updated to `12:11`, the intermediate +state for window `(12:00 - 12:10)` is cleared, and all subsequent data (e.g. `(12:04, donkey)`) +is considered "too late" and therefore ignored. Note that after every trigger, +the updated counts (i.e. purple rows) are written to sink as the trigger output, as dictated by +the Update mode. + +Some sinks (e.g. files) may not supported fine-grained updates that Update Mode requires. To work +with them, we have also support Append Mode, where only the *final counts* are written to sink. +This is illustrated below. + +Note that using `withWatermark` on a non-streaming Dataset is no-op. As the watermark should not affect +any batch query in any way, we will ignore it directly. + +![Watermarking in Append Mode](img/structured-streaming-watermark-append-mode.png) + +Similar to the Update Mode earlier, the engine maintains intermediate counts for each window. +However, the partial counts are not updated to the Result Table and not written to sink. The engine +waits for "10 mins" for late date to be counted, +then drops intermediate state of a window < watermark, and appends the final +counts to the Result Table/sink. For example, the final counts of window `12:00 - 12:10` is +appended to the Result Table only after the watermark is updated to `12:11`. + +**Conditions for watermarking to clean aggregation state** +It is important to note that the following conditions must be satisfied for the watermarking to +clean the state in aggregation queries *(as of Spark 2.1.1, subject to change in the future)*. + +- **Output mode must be Append or Update.** Complete mode requires all aggregate data to be preserved, +and hence cannot use watermarking to drop intermediate state. See the [Output Modes](#output-modes) +section for detailed explanation of the semantics of each output mode. + +- The aggregation must have either the event-time column, or a `window` on the event-time column. + +- `withWatermark` must be called on the +same column as the timestamp column used in the aggregate. For example, +`df.withWatermark("time", "1 min").groupBy("time2").count()` is invalid +in Append output mode, as watermark is defined on a different column +from the aggregation column. + +- `withWatermark` must be called before the aggregation for the watermark details to be used. +For example, `df.groupBy("time").count().withWatermark("time", "1 min")` is invalid in Append +output mode. + + +### Join Operations +Streaming DataFrames can be joined with static DataFrames to create new streaming DataFrames. Here are a few examples. + +
    +
    + +{% highlight scala %} +val staticDf = spark.read. ... +val streamingDf = spark.readStream. ... + +streamingDf.join(staticDf, "type") // inner equi-join with a static DF +streamingDf.join(staticDf, "type", "right_join") // right outer join with a static DF + +{% endhighlight %} + +
    +
    + +{% highlight java %} +Dataset staticDf = spark.read(). ...; +Dataset streamingDf = spark.readStream(). ...; +streamingDf.join(staticDf, "type"); // inner equi-join with a static DF +streamingDf.join(staticDf, "type", "right_join"); // right outer join with a static DF +{% endhighlight %} + + +
    +
    + +{% highlight python %} +staticDf = spark.read. ... +streamingDf = spark.readStream. ... +streamingDf.join(staticDf, "type") # inner equi-join with a static DF +streamingDf.join(staticDf, "type", "right_join") # right outer join with a static DF +{% endhighlight %} + +
    +
    + +### Streaming Deduplication +You can deduplicate records in data streams using a unique identifier in the events. This is exactly same as deduplication on static using a unique identifier column. The query will store the necessary amount of data from previous records such that it can filter duplicate records. Similar to aggregations, you can use deduplication with or without watermarking. + +- *With watermark* - If there is a upper bound on how late a duplicate record may arrive, then you can define a watermark on a event time column and deduplicate using both the guid and the event time columns. The query will use the watermark to remove old state data from past records that are not expected to get any duplicates any more. This bounds the amount of the state the query has to maintain. + +- *Without watermark* - Since there are no bounds on when a duplicate record may arrive, the query stores the data from all the past records as state. + +
    +
    + +{% highlight scala %} +val streamingDf = spark.readStream. ... // columns: guid, eventTime, ... + +// Without watermark using guid column +streamingDf.dropDuplicates("guid") + +// With watermark using guid and eventTime columns +streamingDf + .withWatermark("eventTime", "10 seconds") + .dropDuplicates("guid", "eventTime") +{% endhighlight %} + +
    +
    + +{% highlight java %} +Dataset streamingDf = spark.readStream(). ...; // columns: guid, eventTime, ... + +// Without watermark using guid column +streamingDf.dropDuplicates("guid"); + +// With watermark using guid and eventTime columns +streamingDf + .withWatermark("eventTime", "10 seconds") + .dropDuplicates("guid", "eventTime"); +{% endhighlight %} + + +
    +
    + +{% highlight python %} +streamingDf = spark.readStream. ... + +// Without watermark using guid column +streamingDf.dropDuplicates("guid") + +// With watermark using guid and eventTime columns +streamingDf \ + .withWatermark("eventTime", "10 seconds") \ + .dropDuplicates("guid", "eventTime") +{% endhighlight %} + +
    +
    + +### Arbitrary Stateful Operations +Many usecases require more advanced stateful operations than aggregations. For example, in many usecases, you have to track sessions from data streams of events. For doing such sessionization, you will have to save arbitrary types of data as state, and perform arbitrary operations on the state using the data stream events in every trigger. Since Spark 2.2, this can be done using the operation `mapGroupsWithState` and the more powerful operation `flatMapGroupsWithState`. Both operations allow you to apply user-defined code on grouped Datasets to update user-defined state. For more concrete details, take a look at the API documentation ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.GroupState)/[Java](api/java/org/apache/spark/sql/streaming/GroupState.html)) and the examples ([Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala)/[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java)). + +### Unsupported Operations +There are a few DataFrame/Dataset operations that are not supported with streaming DataFrames/Datasets. +Some of them are as follows. + +- Multiple streaming aggregations (i.e. a chain of aggregations on a streaming DF) are not yet supported on streaming Datasets. + +- Limit and take first N rows are not supported on streaming Datasets. + +- Distinct operations on streaming Datasets are not supported. + +- Sorting operations are supported on streaming Datasets only after an aggregation and in Complete Output Mode. + +- Outer joins between a streaming and a static Datasets are conditionally supported. + + + Full outer join with a streaming Dataset is not supported + + + Left outer join with a streaming Dataset on the right is not supported + + + Right outer join with a streaming Dataset on the left is not supported + +- Any kind of joins between two streaming Datasets is not yet supported. + +In addition, there are some Dataset methods that will not work on streaming Datasets. They are actions that will immediately run queries and return results, which does not make sense on a streaming Dataset. Rather, those functionalities can be done by explicitly starting a streaming query (see the next section regarding that). + +- `count()` - Cannot return a single count from a streaming Dataset. Instead, use `ds.groupBy().count()` which returns a streaming Dataset containing a running count. + +- `foreach()` - Instead use `ds.writeStream.foreach(...)` (see next section). + +- `show()` - Instead use the console sink (see next section). + +If you try any of these operations, you will see an `AnalysisException` like "operation XYZ is not supported with streaming DataFrames/Datasets". +While some of them may be supported in future releases of Spark, +there are others which are fundamentally hard to implement on streaming data efficiently. +For example, sorting on the input stream is not supported, as it requires keeping +track of all the data received in the stream. This is therefore fundamentally hard to execute +efficiently. + +## Starting Streaming Queries +Once you have defined the final result DataFrame/Dataset, all that is left is for you to start the streaming computation. To do that, you have to use the `DataStreamWriter` +([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamWriter)/[Java](api/java/org/apache/spark/sql/streaming/DataStreamWriter.html)/[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamWriter) docs) +returned through `Dataset.writeStream()`. You will have to specify one or more of the following in this interface. + +- *Details of the output sink:* Data format, location, etc. + +- *Output mode:* Specify what gets written to the output sink. + +- *Query name:* Optionally, specify a unique name of the query for identification. + +- *Trigger interval:* Optionally, specify the trigger interval. If it is not specified, the system will check for availability of new data as soon as the previous processing has completed. If a trigger time is missed because the previous processing has not completed, then the system will trigger processing immediately. + +- *Checkpoint location:* For some output sinks where the end-to-end fault-tolerance can be guaranteed, specify the location where the system will write all the checkpoint information. This should be a directory in an HDFS-compatible fault-tolerant file system. The semantics of checkpointing is discussed in more detail in the next section. + +#### Output Modes +There are a few types of output modes. + +- **Append mode (default)** - This is the default mode, where only the +new rows added to the Result Table since the last trigger will be +outputted to the sink. This is supported for only those queries where +rows added to the Result Table is never going to change. Hence, this mode +guarantees that each row will be output only once (assuming +fault-tolerant sink). For example, queries with only `select`, +`where`, `map`, `flatMap`, `filter`, `join`, etc. will support Append mode. + +- **Complete mode** - The whole Result Table will be outputted to the sink after every trigger. + This is supported for aggregation queries. + +- **Update mode** - (*Available since Spark 2.1.1*) Only the rows in the Result Table that were +updated since the last trigger will be outputted to the sink. +More information to be added in future releases. + +Different types of streaming queries support different output modes. +Here is the compatibility matrix. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Query TypeSupported Output ModesNotes
    Queries with aggregationAggregation on event-time with watermarkAppend, Update, Complete + Append mode uses watermark to drop old aggregation state. But the output of a + windowed aggregation is delayed the late threshold specified in `withWatermark()` as by + the modes semantics, rows can be added to the Result Table only once after they are + finalized (i.e. after watermark is crossed). See the + Late Data section for more details. +

    + Update mode uses watermark to drop old aggregation state. +

    + Complete mode does not drop old aggregation state since by definition this mode + preserves all data in the Result Table. +
    Other aggregationsComplete, Update + Since no watermark is defined (only defined in other category), + old aggregation state is not dropped. +

    + Append mode is not supported as aggregates can update thus violating the semantics of + this mode. +
    Queries with mapGroupsWithStateUpdate
    Queries with flatMapGroupsWithStateAppend operation modeAppend + Aggregations are allowed after flatMapGroupsWithState. +
    Update operation modeUpdate + Aggregations not allowed after flatMapGroupsWithState. +
    Other queriesAppend, Update + Complete mode not supported as it is infeasible to keep all unaggregated data in the Result Table. +
    + + +#### Output Sinks +There are a few types of built-in output sinks. + +- **File sink** - Stores the output to a directory. + +{% highlight scala %} +writeStream + .format("parquet") // can be "orc", "json", "csv", etc. + .option("path", "path/to/destination/dir") + .start() +{% endhighlight %} + +- **Kafka sink** - Stores the output to one or more topics in Kafka. + +{% highlight scala %} +writeStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("topic", "updates") + .start() +{% endhighlight %} + +- **Foreach sink** - Runs arbitrary computation on the records in the output. See later in the section for more details. + +{% highlight scala %} +writeStream + .foreach(...) + .start() +{% endhighlight %} + +- **Console sink (for debugging)** - Prints the output to the console/stdout every time there is a trigger. Both, Append and Complete output modes, are supported. This should be used for debugging purposes on low data volumes as the entire output is collected and stored in the driver's memory after every trigger. + +{% highlight scala %} +writeStream + .format("console") + .start() +{% endhighlight %} + +- **Memory sink (for debugging)** - The output is stored in memory as an in-memory table. +Both, Append and Complete output modes, are supported. This should be used for debugging purposes +on low data volumes as the entire output is collected and stored in the driver's memory. +Hence, use it with caution. + +{% highlight scala %} +writeStream + .format("memory") + .queryName("tableName") + .start() +{% endhighlight %} + +Some sinks are not fault-tolerant because they do not guarantee persistence of the output and are +meant for debugging purposes only. See the earlier section on +[fault-tolerance semantics](#fault-tolerance-semantics). +Here are the details of all the sinks in Spark. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    SinkSupported Output ModesOptionsFault-tolerantNotes
    File SinkAppend + path: path to the output directory, must be specified. +

    + For file-format-specific options, see the related methods in DataFrameWriter + (Scala/Java/Python/R). + E.g. for "parquet" format options see DataFrameWriter.parquet() +
    Yes (exactly-once)Supports writes to partitioned tables. Partitioning by time may be useful.
    Kafka SinkAppend, Update, CompleteSee the Kafka Integration GuideYes (at-least-once)More details in the Kafka Integration Guide
    Foreach SinkAppend, Update, CompleteNoneDepends on ForeachWriter implementationMore details in the next section
    Console SinkAppend, Update, Complete + numRows: Number of rows to print every trigger (default: 20) +
    + truncate: Whether to truncate the output if too long (default: true) +
    No
    Memory SinkAppend, CompleteNoneNo. But in Complete Mode, restarted query will recreate the full table.Table name is the query name.
    + +Note that you have to call `start()` to actually start the execution of the query. This returns a StreamingQuery object which is a handle to the continuously running execution. You can use this object to manage the query, which we will discuss in the next subsection. For now, let’s understand all this with a few examples. + + +
    +
    + +{% highlight scala %} +// ========== DF with no aggregations ========== +val noAggDF = deviceDataDf.select("device").where("signal > 10") + +// Print new data to console +noAggDF + .writeStream + .format("console") + .start() + +// Write new data to Parquet files +noAggDF + .writeStream + .format("parquet") + .option("checkpointLocation", "path/to/checkpoint/dir") + .option("path", "path/to/destination/dir") + .start() + +// ========== DF with aggregation ========== +val aggDF = df.groupBy("device").count() + +// Print updated aggregations to console +aggDF + .writeStream + .outputMode("complete") + .format("console") + .start() + +// Have all the aggregates in an in-memory table +aggDF + .writeStream + .queryName("aggregates") // this query name will be the table name + .outputMode("complete") + .format("memory") + .start() + +spark.sql("select * from aggregates").show() // interactively query in-memory table +{% endhighlight %} + +
    +
    + +{% highlight java %} +// ========== DF with no aggregations ========== +Dataset noAggDF = deviceDataDf.select("device").where("signal > 10"); + +// Print new data to console +noAggDF + .writeStream() + .format("console") + .start(); + +// Write new data to Parquet files +noAggDF + .writeStream() + .format("parquet") + .option("checkpointLocation", "path/to/checkpoint/dir") + .option("path", "path/to/destination/dir") + .start(); + +// ========== DF with aggregation ========== +Dataset aggDF = df.groupBy("device").count(); + +// Print updated aggregations to console +aggDF + .writeStream() + .outputMode("complete") + .format("console") + .start(); + +// Have all the aggregates in an in-memory table +aggDF + .writeStream() + .queryName("aggregates") // this query name will be the table name + .outputMode("complete") + .format("memory") + .start(); + +spark.sql("select * from aggregates").show(); // interactively query in-memory table +{% endhighlight %} + +
    +
    + +{% highlight python %} +# ========== DF with no aggregations ========== +noAggDF = deviceDataDf.select("device").where("signal > 10") + +# Print new data to console +noAggDF \ + .writeStream \ + .format("console") \ + .start() + +# Write new data to Parquet files +noAggDF \ + .writeStream \ + .format("parquet") \ + .option("checkpointLocation", "path/to/checkpoint/dir") \ + .option("path", "path/to/destination/dir") \ + .start() + +# ========== DF with aggregation ========== +aggDF = df.groupBy("device").count() + +# Print updated aggregations to console +aggDF \ + .writeStream \ + .outputMode("complete") \ + .format("console") \ + .start() + +# Have all the aggregates in an in memory table. The query name will be the table name +aggDF \ + .writeStream \ + .queryName("aggregates") \ + .outputMode("complete") \ + .format("memory") \ + .start() + +spark.sql("select * from aggregates").show() # interactively query in-memory table +{% endhighlight %} + +
    +
    + +{% highlight r %} +# ========== DF with no aggregations ========== +noAggDF <- select(where(deviceDataDf, "signal > 10"), "device") + +# Print new data to console +write.stream(noAggDF, "console") + +# Write new data to Parquet files +write.stream(noAggDF, + "parquet", + path = "path/to/destination/dir", + checkpointLocation = "path/to/checkpoint/dir") + +# ========== DF with aggregation ========== +aggDF <- count(groupBy(df, "device")) + +# Print updated aggregations to console +write.stream(aggDF, "console", outputMode = "complete") + +# Have all the aggregates in an in memory table. The query name will be the table name +write.stream(aggDF, "memory", queryName = "aggregates", outputMode = "complete") + +# Interactively query in-memory table +head(sql("select * from aggregates")) +{% endhighlight %} + +
    +
    + +#### Using Foreach +The `foreach` operation allows arbitrary operations to be computed on the output data. As of Spark 2.1, this is available only for Scala and Java. To use this, you will have to implement the interface `ForeachWriter` +([Scala](api/scala/index.html#org.apache.spark.sql.ForeachWriter)/[Java](api/java/org/apache/spark/sql/ForeachWriter.html) docs), +which has methods that get called whenever there is a sequence of rows generated as output after a trigger. Note the following important points. + +- The writer must be serializable, as it will be serialized and sent to the executors for execution. + +- All the three methods, `open`, `process` and `close` will be called on the executors. + +- The writer must do all the initialization (e.g. opening connections, starting a transaction, etc.) only when the `open` method is called. Be aware that, if there is any initialization in the class as soon as the object is created, then that initialization will happen in the driver (because that is where the instance is being created), which may not be what you intend. + +- `version` and `partition` are two parameters in `open` that uniquely represent a set of rows that needs to be pushed out. `version` is a monotonically increasing id that increases with every trigger. `partition` is an id that represents a partition of the output, since the output is distributed and will be processed on multiple executors. + +- `open` can use the `version` and `partition` to choose whether it needs to write the sequence of rows. Accordingly, it can return `true` (proceed with writing), or `false` (no need to write). If `false` is returned, then `process` will not be called on any row. For example, after a partial failure, some of the output partitions of the failed trigger may have already been committed to a database. Based on metadata stored in the database, the writer can identify partitions that have already been committed and accordingly return false to skip committing them again. + +- Whenever `open` is called, `close` will also be called (unless the JVM exits due to some error). This is true even if `open` returns false. If there is any error in processing and writing the data, `close` will be called with the error. It is your responsibility to clean up state (e.g. connections, transactions, etc.) that have been created in `open` such that there are no resource leaks. + +## Managing Streaming Queries +The `StreamingQuery` object created when a query is started can be used to monitor and manage the query. + +
    +
    + +{% highlight scala %} +val query = df.writeStream.format("console").start() // get the query object + +query.id // get the unique identifier of the running query that persists across restarts from checkpoint data + +query.runId // get the unique id of this run of the query, which will be generated at every start/restart + +query.name // get the name of the auto-generated or user-specified name + +query.explain() // print detailed explanations of the query + +query.stop() // stop the query + +query.awaitTermination() // block until query is terminated, with stop() or with error + +query.exception // the exception if the query has been terminated with error + +query.recentProgress // an array of the most recent progress updates for this query + +query.lastProgress // the most recent progress update of this streaming query +{% endhighlight %} + + +
    +
    + +{% highlight java %} +StreamingQuery query = df.writeStream().format("console").start(); // get the query object + +query.id(); // get the unique identifier of the running query that persists across restarts from checkpoint data + +query.runId(); // get the unique id of this run of the query, which will be generated at every start/restart + +query.name(); // get the name of the auto-generated or user-specified name + +query.explain(); // print detailed explanations of the query + +query.stop(); // stop the query + +query.awaitTermination(); // block until query is terminated, with stop() or with error + +query.exception(); // the exception if the query has been terminated with error + +query.recentProgress(); // an array of the most recent progress updates for this query + +query.lastProgress(); // the most recent progress update of this streaming query + +{% endhighlight %} + +
    +
    + +{% highlight python %} +query = df.writeStream.format("console").start() # get the query object + +query.id() # get the unique identifier of the running query that persists across restarts from checkpoint data + +query.runId() # get the unique id of this run of the query, which will be generated at every start/restart + +query.name() # get the name of the auto-generated or user-specified name + +query.explain() # print detailed explanations of the query + +query.stop() # stop the query + +query.awaitTermination() # block until query is terminated, with stop() or with error + +query.exception() # the exception if the query has been terminated with error + +query.recentProgress() # an array of the most recent progress updates for this query + +query.lastProgress() # the most recent progress update of this streaming query + +{% endhighlight %} + +
    +
    + +{% highlight r %} +query <- write.stream(df, "console") # get the query object + +queryName(query) # get the name of the auto-generated or user-specified name + +explain(query) # print detailed explanations of the query + +stopQuery(query) # stop the query + +awaitTermination(query) # block until query is terminated, with stop() or with error + +lastProgress(query) # the most recent progress update of this streaming query + +{% endhighlight %} + +
    +
    + +You can start any number of queries in a single SparkSession. They will all be running concurrently sharing the cluster resources. You can use `sparkSession.streams()` to get the `StreamingQueryManager` +([Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryManager)/[Java](api/java/org/apache/spark/sql/streaming/StreamingQueryManager.html)/[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.StreamingQueryManager) docs) +that can be used to manage the currently active queries. + +
    +
    + +{% highlight scala %} +val spark: SparkSession = ... + +spark.streams.active // get the list of currently active streaming queries + +spark.streams.get(id) // get a query object by its unique id + +spark.streams.awaitAnyTermination() // block until any one of them terminates +{% endhighlight %} + +
    +
    + +{% highlight java %} +SparkSession spark = ... + +spark.streams().active(); // get the list of currently active streaming queries + +spark.streams().get(id); // get a query object by its unique id + +spark.streams().awaitAnyTermination(); // block until any one of them terminates +{% endhighlight %} + +
    +
    + +{% highlight python %} +spark = ... # spark session + +spark.streams().active # get the list of currently active streaming queries + +spark.streams().get(id) # get a query object by its unique id + +spark.streams().awaitAnyTermination() # block until any one of them terminates +{% endhighlight %} + +
    +
    +{% highlight bash %} +Not available in R. +{% endhighlight %} + +
    +
    + + +## Monitoring Streaming Queries +There are multiple ways to monitor active streaming queries. You can either push metrics to external systems using Spark's Dropwizard Metrics support, or access them programmatically. + +### Reading Metrics Interactively + +You can directly get the current status and metrics of an active query using +`streamingQuery.lastProgress()` and `streamingQuery.status()`. +`lastProgress()` returns a `StreamingQueryProgress` object +in [Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryProgress) +and [Java](api/java/org/apache/spark/sql/streaming/StreamingQueryProgress.html) +and a dictionary with the same fields in Python. It has all the information about +the progress made in the last trigger of the stream - what data was processed, +what were the processing rates, latencies, etc. There is also +`streamingQuery.recentProgress` which returns an array of last few progresses. + +In addition, `streamingQuery.status()` returns a `StreamingQueryStatus` object +in [Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryStatus) +and [Java](api/java/org/apache/spark/sql/streaming/StreamingQueryStatus.html) +and a dictionary with the same fields in Python. It gives information about +what the query is immediately doing - is a trigger active, is data being processed, etc. + +Here are a few examples. + +
    +
    + +{% highlight scala %} +val query: StreamingQuery = ... + +println(query.lastProgress) + +/* Will print something like the following. + +{ + "id" : "ce011fdc-8762-4dcb-84eb-a77333e28109", + "runId" : "88e2ff94-ede0-45a8-b687-6316fbef529a", + "name" : "MyQuery", + "timestamp" : "2016-12-14T18:45:24.873Z", + "numInputRows" : 10, + "inputRowsPerSecond" : 120.0, + "processedRowsPerSecond" : 200.0, + "durationMs" : { + "triggerExecution" : 3, + "getOffset" : 2 + }, + "eventTime" : { + "watermark" : "2016-12-14T18:45:24.873Z" + }, + "stateOperators" : [ ], + "sources" : [ { + "description" : "KafkaSource[Subscribe[topic-0]]", + "startOffset" : { + "topic-0" : { + "2" : 0, + "4" : 1, + "1" : 1, + "3" : 1, + "0" : 1 + } + }, + "endOffset" : { + "topic-0" : { + "2" : 0, + "4" : 115, + "1" : 134, + "3" : 21, + "0" : 534 + } + }, + "numInputRows" : 10, + "inputRowsPerSecond" : 120.0, + "processedRowsPerSecond" : 200.0 + } ], + "sink" : { + "description" : "MemorySink" + } +} +*/ + + +println(query.status) + +/* Will print something like the following. +{ + "message" : "Waiting for data to arrive", + "isDataAvailable" : false, + "isTriggerActive" : false +} +*/ +{% endhighlight %} + +
    +
    + +{% highlight java %} +StreamingQuery query = ... + +System.out.println(query.lastProgress()); +/* Will print something like the following. + +{ + "id" : "ce011fdc-8762-4dcb-84eb-a77333e28109", + "runId" : "88e2ff94-ede0-45a8-b687-6316fbef529a", + "name" : "MyQuery", + "timestamp" : "2016-12-14T18:45:24.873Z", + "numInputRows" : 10, + "inputRowsPerSecond" : 120.0, + "processedRowsPerSecond" : 200.0, + "durationMs" : { + "triggerExecution" : 3, + "getOffset" : 2 + }, + "eventTime" : { + "watermark" : "2016-12-14T18:45:24.873Z" + }, + "stateOperators" : [ ], + "sources" : [ { + "description" : "KafkaSource[Subscribe[topic-0]]", + "startOffset" : { + "topic-0" : { + "2" : 0, + "4" : 1, + "1" : 1, + "3" : 1, + "0" : 1 + } + }, + "endOffset" : { + "topic-0" : { + "2" : 0, + "4" : 115, + "1" : 134, + "3" : 21, + "0" : 534 + } + }, + "numInputRows" : 10, + "inputRowsPerSecond" : 120.0, + "processedRowsPerSecond" : 200.0 + } ], + "sink" : { + "description" : "MemorySink" + } +} +*/ + + +System.out.println(query.status()); +/* Will print something like the following. +{ + "message" : "Waiting for data to arrive", + "isDataAvailable" : false, + "isTriggerActive" : false +} +*/ +{% endhighlight %} + +
    +
    + +{% highlight python %} +query = ... # a StreamingQuery +print(query.lastProgress) + +''' +Will print something like the following. + +{u'stateOperators': [], u'eventTime': {u'watermark': u'2016-12-14T18:45:24.873Z'}, u'name': u'MyQuery', u'timestamp': u'2016-12-14T18:45:24.873Z', u'processedRowsPerSecond': 200.0, u'inputRowsPerSecond': 120.0, u'numInputRows': 10, u'sources': [{u'description': u'KafkaSource[Subscribe[topic-0]]', u'endOffset': {u'topic-0': {u'1': 134, u'0': 534, u'3': 21, u'2': 0, u'4': 115}}, u'processedRowsPerSecond': 200.0, u'inputRowsPerSecond': 120.0, u'numInputRows': 10, u'startOffset': {u'topic-0': {u'1': 1, u'0': 1, u'3': 1, u'2': 0, u'4': 1}}}], u'durationMs': {u'getOffset': 2, u'triggerExecution': 3}, u'runId': u'88e2ff94-ede0-45a8-b687-6316fbef529a', u'id': u'ce011fdc-8762-4dcb-84eb-a77333e28109', u'sink': {u'description': u'MemorySink'}} +''' + +print(query.status) +''' +Will print something like the following. + +{u'message': u'Waiting for data to arrive', u'isTriggerActive': False, u'isDataAvailable': False} +''' +{% endhighlight %} + +
    +
    + +{% highlight r %} +query <- ... # a StreamingQuery +lastProgress(query) + +''' +Will print something like the following. + +{ + "id" : "8c57e1ec-94b5-4c99-b100-f694162df0b9", + "runId" : "ae505c5a-a64e-4896-8c28-c7cbaf926f16", + "name" : null, + "timestamp" : "2017-04-26T08:27:28.835Z", + "numInputRows" : 0, + "inputRowsPerSecond" : 0.0, + "processedRowsPerSecond" : 0.0, + "durationMs" : { + "getOffset" : 0, + "triggerExecution" : 1 + }, + "stateOperators" : [ { + "numRowsTotal" : 4, + "numRowsUpdated" : 0 + } ], + "sources" : [ { + "description" : "TextSocketSource[host: localhost, port: 9999]", + "startOffset" : 1, + "endOffset" : 1, + "numInputRows" : 0, + "inputRowsPerSecond" : 0.0, + "processedRowsPerSecond" : 0.0 + } ], + "sink" : { + "description" : "org.apache.spark.sql.execution.streaming.ConsoleSink@76b37531" + } +} +''' + +status(query) +''' +Will print something like the following. + +{ + "message" : "Waiting for data to arrive", + "isDataAvailable" : false, + "isTriggerActive" : false +} +''' +{% endhighlight %} + +
    +
    + +### Reporting Metrics programmatically using Asynchronous APIs + +You can also asynchronously monitor all queries associated with a +`SparkSession` by attaching a `StreamingQueryListener` +([Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryListener)/[Java](api/java/org/apache/spark/sql/streaming/StreamingQueryListener.html) docs). +Once you attach your custom `StreamingQueryListener` object with +`sparkSession.streams.attachListener()`, you will get callbacks when a query is started and +stopped and when there is progress made in an active query. Here is an example, + +
    +
    + +{% highlight scala %} +val spark: SparkSession = ... + +spark.streams.addListener(new StreamingQueryListener() { + override def onQueryStarted(queryStarted: QueryStartedEvent): Unit = { + println("Query started: " + queryStarted.id) + } + override def onQueryTerminated(queryTerminated: QueryTerminatedEvent): Unit = { + println("Query terminated: " + queryTerminated.id) + } + override def onQueryProgress(queryProgress: QueryProgressEvent): Unit = { + println("Query made progress: " + queryProgress.progress) + } +}) +{% endhighlight %} + +
    +
    + +{% highlight java %} +SparkSession spark = ... + +spark.streams().addListener(new StreamingQueryListener() { + @Override + public void onQueryStarted(QueryStartedEvent queryStarted) { + System.out.println("Query started: " + queryStarted.id()); + } + @Override + public void onQueryTerminated(QueryTerminatedEvent queryTerminated) { + System.out.println("Query terminated: " + queryTerminated.id()); + } + @Override + public void onQueryProgress(QueryProgressEvent queryProgress) { + System.out.println("Query made progress: " + queryProgress.progress()); + } +}); +{% endhighlight %} + +
    +
    +{% highlight bash %} +Not available in Python. +{% endhighlight %} + +
    +
    +{% highlight bash %} +Not available in R. +{% endhighlight %} + +
    +
    + +### Reporting Metrics using Dropwizard +Spark supports reporting metrics using the [Dropwizard Library](monitoring.html#metrics). To enable metrics of Structured Streaming queries to be reported as well, you have to explicitly enable the configuration `spark.sql.streaming.metricsEnabled` in the SparkSession. + +
    +
    +{% highlight scala %} +spark.conf.set("spark.sql.streaming.metricsEnabled", "true") +// or +spark.sql("SET spark.sql.streaming.metricsEnabled=true") +{% endhighlight %} +
    +
    +{% highlight java %} +spark.conf().set("spark.sql.streaming.metricsEnabled", "true"); +// or +spark.sql("SET spark.sql.streaming.metricsEnabled=true"); +{% endhighlight %} +
    +
    +{% highlight python %} +spark.conf.set("spark.sql.streaming.metricsEnabled", "true") +# or +spark.sql("SET spark.sql.streaming.metricsEnabled=true") +{% endhighlight %} +
    +
    +{% highlight r %} +sql("SET spark.sql.streaming.metricsEnabled=true") +{% endhighlight %} +
    +
    + + +All queries started in the SparkSession after this configuration has been enabled will report metrics through Dropwizard to whatever [sinks](monitoring.html#metrics) have been configured (e.g. Ganglia, Graphite, JMX, etc.). + +## Recovering from Failures with Checkpointing +In case of a failure or intentional shutdown, you can recover the previous progress and state of a previous query, and continue where it left off. This is done using checkpointing and write ahead logs. You can configure a query with a checkpoint location, and the query will save all the progress information (i.e. range of offsets processed in each trigger) and the running aggregates (e.g. word counts in the [quick example](#quick-example)) to the checkpoint location. This checkpoint location has to be a path in an HDFS compatible file system, and can be set as an option in the DataStreamWriter when [starting a query](#starting-streaming-queries). + +
    +
    + +{% highlight scala %} +aggDF + .writeStream + .outputMode("complete") + .option("checkpointLocation", "path/to/HDFS/dir") + .format("memory") + .start() +{% endhighlight %} + +
    +
    + +{% highlight java %} +aggDF + .writeStream() + .outputMode("complete") + .option("checkpointLocation", "path/to/HDFS/dir") + .format("memory") + .start(); +{% endhighlight %} + +
    +
    + +{% highlight python %} +aggDF \ + .writeStream \ + .outputMode("complete") \ + .option("checkpointLocation", "path/to/HDFS/dir") \ + .format("memory") \ + .start() +{% endhighlight %} + +
    +
    + +{% highlight r %} +write.stream(aggDF, "memory", outputMode = "complete", checkpointLocation = "path/to/HDFS/dir") +{% endhighlight %} + +
    +
    + +# Additional Information + +**Further Reading** + +- See and run the + [Scala]({{site.SPARK_GITHUB_URL}}/tree/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming)/[Java]({{site.SPARK_GITHUB_URL}}/tree/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming)/[Python]({{site.SPARK_GITHUB_URL}}/tree/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/sql/streaming)/[R]({{site.SPARK_GITHUB_URL}}/tree/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/r/streaming) + examples. + - [Instructions](index.html#running-the-examples-and-shell) on how to run Spark examples +- Read about integrating with Kafka in the [Structured Streaming Kafka Integration Guide](structured-streaming-kafka-integration.html) +- Read more details about using DataFrames/Datasets in the [Spark SQL Programming Guide](sql-programming-guide.html) +- Third-party Blog Posts + - [Real-time Streaming ETL with Structured Streaming in Apache Spark 2.1 (Databricks Blog)](https://databricks.com/blog/2017/01/19/real-time-streaming-etl-structured-streaming-apache-spark-2-1.html) + - [Real-Time End-to-End Integration with Apache Kafka in Apache Spark’s Structured Streaming (Databricks Blog)](https://databricks.com/blog/2017/04/04/real-time-end-to-end-integration-with-apache-kafka-in-apache-sparks-structured-streaming.html) + - [Event-time Aggregation and Watermarking in Apache Spark’s Structured Streaming (Databricks Blog)](https://databricks.com/blog/2017/05/08/event-time-aggregation-watermarking-apache-sparks-structured-streaming.html) + +**Talks** + +- Spark Summit 2017 Talk - [Easy, Scalable, Fault-tolerant Stream Processing with Structured Streaming in Apache Spark](https://spark-summit.org/2017/events/easy-scalable-fault-tolerant-stream-processing-with-structured-streaming-in-apache-spark/) +- Spark Summit 2016 Talk - [A Deep Dive into Structured Streaming](https://spark-summit.org/2016/events/a-deep-dive-into-structured-streaming/) + diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index ac2a14eb56fe..866d6e527549 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -30,7 +30,7 @@ dependencies, and can support different cluster managers and deploy modes that S {% highlight bash %} ./bin/spark-submit \ - --class + --class \ --master \ --deploy-mode \ --conf = \ @@ -58,8 +58,8 @@ for applications that involve the REPL (e.g. Spark shell). Alternatively, if your application is submitted from a machine far from the worker machines (e.g. locally on your laptop), it is common to use `cluster` mode to minimize network latency between -the drivers and the executors. Note that `cluster` mode is currently not supported for -Mesos clusters. Currently only YARN supports cluster mode for Python applications. +the drivers and the executors. Currently, standalone mode does not support cluster mode for Python +applications. For Python applications, simply pass a `.py` file in the place of `` instead of a JAR, and add Python `.zip`, `.egg` or `.py` files to the search path with `--py-files`. @@ -92,8 +92,8 @@ run it with `--help`. Here are a few examples of common options: ./bin/spark-submit \ --class org.apache.spark.examples.SparkPi \ --master spark://207.184.161.138:7077 \ - --deploy-mode cluster - --supervise + --deploy-mode cluster \ + --supervise \ --executor-memory 20G \ --total-executor-cores 100 \ /path/to/examples.jar \ @@ -115,6 +115,18 @@ export HADOOP_CONF_DIR=XXX --master spark://207.184.161.138:7077 \ examples/src/main/python/pi.py \ 1000 + +# Run on a Mesos cluster in cluster deploy mode with supervise +./bin/spark-submit \ + --class org.apache.spark.examples.SparkPi \ + --master mesos://207.184.161.138:7077 \ + --deploy-mode cluster \ + --supervise \ + --executor-memory 20G \ + --total-executor-cores 100 \ + http://path/to/examples.jar \ + 1000 + {% endhighlight %} # Master URLs @@ -125,24 +137,24 @@ The master URL passed to Spark can be in one of the following formats: Master URLMeaning local Run Spark locally with one worker thread (i.e. no parallelism at all). local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine). + local[K,F] Run Spark locally with K worker threads and F maxFailures (see spark.task.maxFailures for an explanation of this variable) local[*] Run Spark locally with as many worker threads as logical cores on your machine. + local[*,F] Run Spark locally with as many worker threads as logical cores on your machine and F maxFailures. spark://HOST:PORT Connect to the given Spark standalone cluster master. The port must be whichever one your master is configured to use, which is 7077 by default. + spark://HOST1:PORT1,HOST2:PORT2 Connect to the given Spark standalone + cluster with standby masters with Zookeeper. The list must have all the master hosts in the high availability cluster set up with Zookeeper. The port must be whichever each master is configured to use, which is 7077 by default. + mesos://HOST:PORT Connect to the given Mesos cluster. The port must be whichever one your is configured to use, which is 5050 by default. Or, for a Mesos cluster using ZooKeeper, use mesos://zk://.... + To submit with --deploy-mode cluster, the HOST:PORT should be configured to connect to the MesosClusterDispatcher. yarn Connect to a YARN cluster in - client or cluster mode depending on the value of --deploy-mode. + client or cluster mode depending on the value of --deploy-mode. The cluster location will be found based on the HADOOP_CONF_DIR or YARN_CONF_DIR variable. - yarn-client Equivalent to yarn with --deploy-mode client, - which is preferred to `yarn-client` - - yarn-cluster Equivalent to yarn with --deploy-mode cluster, - which is preferred to `yarn-cluster` - @@ -164,8 +176,9 @@ debugging information by running `spark-submit` with the `--verbose` option. # Advanced Dependency Management When using `spark-submit`, the application jar along with any jars included with the `--jars` option -will be automatically transferred to the cluster. Spark uses the following URL scheme to allow -different strategies for disseminating jars: +will be automatically transferred to the cluster. URLs supplied after `--jars` must be separated by commas. That list is included on the driver and executor classpaths. Directory expansion does not work with `--jars`. + +Spark uses the following URL scheme to allow different strategies for disseminating jars: - **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and every executor pulls the file from the driver HTTP server. @@ -179,9 +192,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 +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`. +(Note that credentials for password-protected repositories can be supplied in some cases in the repository URI, +such as in `https://user:password@host/...`. Be careful when supplying credentials this way.) 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 diff --git a/docs/tuning.md b/docs/tuning.md index 6936912a6be5..7d5f97a02fe6 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -12,7 +12,7 @@ Because of the in-memory nature of most Spark computations, Spark programs can b by any resource in the cluster: CPU, network bandwidth, or memory. Most often, if the data fits in memory, the bottleneck is network bandwidth, but sometimes, you also need to do some tuning, such as -[storing RDDs in serialized form](programming-guide.html#rdd-persistence), to +[storing RDDs in serialized form](rdd-programming-guide.html#rdd-persistence), to decrease memory usage. This guide will cover two main topics: data serialization, which is crucial for good network performance and can also reduce memory use, and memory tuning. We also sketch several smaller topics. @@ -45,6 +45,7 @@ and calling `conf.set("spark.serializer", "org.apache.spark.serializer.KryoSeria This setting configures the serializer used for not only shuffling data between worker nodes but also when serializing RDDs to disk. The only reason Kryo is not the default is because of the custom registration requirement, but we recommend trying it in any network-intensive application. +Since Spark 2.0.0, we internally use Kryo serializer when shuffling RDDs with simple types, arrays of simple types, or string type. Spark automatically includes Kryo serializers for the many commonly-used core Scala classes covered in the AllScalaRegistrar from the [Twitter chill](https://github.com/twitter/chill) library. @@ -61,8 +62,8 @@ The [Kryo documentation](https://github.com/EsotericSoftware/kryo) describes mor registration options, such as adding custom serialization code. If your objects are large, you may also need to increase the `spark.kryoserializer.buffer` -config property. The default is 2, but this value needs to be large enough to hold the *largest* -object you will serialize. +[config](configuration.html#compression-and-serialization). This value needs to be large enough +to hold the *largest* object you will serialize. Finally, if you don't register your custom classes, Kryo will still work, but it will have to store the full class name with each object, which is wasteful. @@ -88,9 +89,42 @@ than the "raw" data inside their fields. This is due to several reasons: but also pointers (typically 8 bytes each) to the next object in the list. * Collections of primitive types often store them as "boxed" objects such as `java.lang.Integer`. -This section will discuss how to determine the memory usage of your objects, and how to improve -it -- either by changing your data structures, or by storing data in a serialized format. -We will then cover tuning Spark's cache size and the Java garbage collector. +This section will start with an overview of memory management in Spark, then discuss specific +strategies the user can take to make more efficient use of memory in his/her application. In +particular, we will describe how to determine the memory usage of your objects, and how to +improve it -- either by changing your data structures, or by storing data in a serialized +format. We will then cover tuning Spark's cache size and the Java garbage collector. + +## Memory Management Overview + +Memory usage in Spark largely falls under one of two categories: execution and storage. +Execution memory refers to that used for computation in shuffles, joins, sorts and aggregations, +while storage memory refers to that used for caching and propagating internal data across the +cluster. In Spark, execution and storage share a unified region (M). When no execution memory is +used, storage can acquire all the available memory and vice versa. Execution may evict storage +if necessary, but only until total storage memory usage falls under a certain threshold (R). +In other words, `R` describes a subregion within `M` where cached blocks are never evicted. +Storage may not evict execution due to complexities in implementation. + +This design ensures several desirable properties. First, applications that do not use caching +can use the entire space for execution, obviating unnecessary disk spills. Second, applications +that do use caching can reserve a minimum storage space (R) where their data blocks are immune +to being evicted. Lastly, this approach provides reasonable out-of-the-box performance for a +variety of workloads without requiring user expertise of how memory is divided internally. + +Although there are two relevant configurations, the typical user should not need to adjust them +as the default values are applicable to most workloads: + +* `spark.memory.fraction` expresses the size of `M` as a fraction of the (JVM heap space - 300MB) +(default 0.6). The rest of the space (40%) is reserved for user data structures, internal +metadata in Spark, and safeguarding against OOM errors in the case of sparse and unusually +large records. +* `spark.memory.storageFraction` expresses the size of `R` as a fraction of `M` (default 0.5). +`R` is the storage space within `M` where cached blocks immune to being evicted by execution. + +The value of `spark.memory.fraction` should be set in order to fit this amount of heap space +comfortably within the JVM's old or "tenured" generation. See the discussion of advanced GC +tuning below for details. ## Determining Memory Consumption @@ -121,7 +155,7 @@ pointer-based data structures and wrapper objects. There are several ways to do When your objects are still too large to efficiently store despite this tuning, a much simpler way to reduce memory usage is to store them in *serialized* form, using the serialized StorageLevels in -the [RDD persistence API](programming-guide.html#rdd-persistence), such as `MEMORY_ONLY_SER`. +the [RDD persistence API](rdd-programming-guide.html#rdd-persistence), such as `MEMORY_ONLY_SER`. Spark will then store each RDD partition as one large byte array. The only downside of storing data in serialized form is slower access times, due to having to deserialize each object on the fly. @@ -151,18 +185,6 @@ time spent GC. This can be done by adding `-verbose:gc -XX:+PrintGCDetails -XX:+ each time a garbage collection occurs. Note these logs will be on your cluster's worker nodes (in the `stdout` files in their work directories), *not* on your driver program. -**Cache Size Tuning** - -One important configuration parameter for GC is the amount of memory that should be used for caching RDDs. -By default, Spark uses 60% of the configured executor memory (`spark.executor.memory`) to -cache RDDs. This means that 40% of memory is available for any objects created during task execution. - -In case your tasks slow down and you find that your JVM is garbage-collecting frequently or running out of -memory, lowering this value will help reduce the memory consumption. To change this to, say, 50%, you can call -`conf.set("spark.storage.memoryFraction", "0.5")` on your SparkConf. Combined with the use of serialized caching, -using a smaller cache should be sufficient to mitigate most of the garbage collection problems. -In case you are interested in further tuning the Java GC, continue reading below. - **Advanced GC Tuning** To further tune garbage collection, we first need to understand some basic information about memory management in the JVM: @@ -183,19 +205,27 @@ temporary objects created during task execution. Some steps which may be useful * Check if there are too many garbage collections by collecting GC stats. If a full GC is invoked multiple times for before a task completes, it means that there isn't enough memory available for executing tasks. -* In the GC stats that are printed, if the OldGen is close to being full, reduce the amount of memory used for caching. - This can be done using the `spark.storage.memoryFraction` property. It is better to cache fewer objects than to slow - down task execution! - * If there are too many minor collections but not many major GCs, allocating more memory for Eden would help. You can set the size of the Eden to be an over-estimate of how much memory each task will need. If the size of Eden is determined to be `E`, then you can set the size of the Young generation using the option `-Xmn=4/3*E`. (The scaling up by 4/3 is to account for space used by survivor regions as well.) + +* In the GC stats that are printed, if the OldGen is close to being full, reduce the amount of + memory used for caching by lowering `spark.memory.fraction`; it is better to cache fewer + objects than to slow down task execution. Alternatively, consider decreasing the size of + the Young generation. This means lowering `-Xmn` if you've set it as above. If not, try changing the + value of the JVM's `NewRatio` parameter. Many JVMs default this to 2, meaning that the Old generation + occupies 2/3 of the heap. It should be large enough such that this fraction exceeds `spark.memory.fraction`. + +* Try the G1GC garbage collector with `-XX:+UseG1GC`. It can improve performance in some situations where + garbage collection is a bottleneck. Note that with large executor heap sizes, it may be important to + increase the [G1 region size](https://blogs.oracle.com/g1gc/entry/g1_gc_tuning_a_case) + with `-XX:G1HeapRegionSize` * As an example, if your task is reading data from HDFS, the amount of memory used by the task can be estimated using the size of the data block read from HDFS. Note that the size of a decompressed block is often 2 or 3 times the - size of the block. So if we wish to have 3 or 4 tasks' worth of working space, and the HDFS block size is 64 MB, - we can estimate size of Eden to be `4*3*64MB`. + size of the block. So if we wish to have 3 or 4 tasks' worth of working space, and the HDFS block size is 128 MB, + we can estimate size of Eden to be `4*3*128MB`. * Monitor how the frequency and time taken by garbage collection changes with the new settings. @@ -203,6 +233,9 @@ Our experience suggests that the effect of GC tuning depends on your application There are [many more tuning options](http://www.oracle.com/technetwork/java/javase/gc-tuning-6-140523.html) described online, but at a high level, managing how frequently full GC takes place can help in reducing the overhead. +GC tuning flags for executors can be specified by setting `spark.executor.extraJavaOptions` in +a job's configuration. + # Other Considerations ## Level of Parallelism @@ -229,7 +262,7 @@ number of cores in your clusters. ## Broadcasting Large Variables -Using the [broadcast functionality](programming-guide.html#broadcast-variables) +Using the [broadcast functionality](rdd-programming-guide.html#broadcast-variables) available in `SparkContext` can greatly reduce the size of each serialized task, and the cost of launching a job over a cluster. If your tasks use any large object from the driver program inside of them (e.g. a static lookup table), consider turning it into a broadcast variable. diff --git a/ec2/README b/ec2/README deleted file mode 100644 index 72434f24bf98..000000000000 --- a/ec2/README +++ /dev/null @@ -1,4 +0,0 @@ -This folder contains a script, spark-ec2, for launching Spark clusters on -Amazon EC2. Usage instructions are available online at: - -http://spark.apache.org/docs/latest/ec2-scripts.html diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh deleted file mode 100644 index 4f3e8da809f7..000000000000 --- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh +++ /dev/null @@ -1,34 +0,0 @@ -#!/usr/bin/env bash - -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# These variables are automatically filled in by the spark-ec2 script. -export MASTERS="{{master_list}}" -export SLAVES="{{slave_list}}" -export HDFS_DATA_DIRS="{{hdfs_data_dirs}}" -export MAPRED_LOCAL_DIRS="{{mapred_local_dirs}}" -export SPARK_LOCAL_DIRS="{{spark_local_dirs}}" -export MODULES="{{modules}}" -export SPARK_VERSION="{{spark_version}}" -export TACHYON_VERSION="{{tachyon_version}}" -export HADOOP_MAJOR_VERSION="{{hadoop_major_version}}" -export SWAP_MB="{{swap}}" -export SPARK_WORKER_INSTANCES="{{spark_worker_instances}}" -export SPARK_MASTER_OPTS="{{spark_master_opts}}" -export AWS_ACCESS_KEY_ID="{{aws_access_key_id}}" -export AWS_SECRET_ACCESS_KEY="{{aws_secret_access_key}}" diff --git a/ec2/spark-ec2 b/ec2/spark-ec2 deleted file mode 100755 index 26e7d2265569..000000000000 --- a/ec2/spark-ec2 +++ /dev/null @@ -1,25 +0,0 @@ -#!/bin/sh - -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Preserve the user's CWD so that relative paths are passed correctly to -#+ the underlying Python script. -SPARK_EC2_DIR="$(dirname "$0")" - -python -Wdefault "${SPARK_EC2_DIR}/spark_ec2.py" "$@" diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py deleted file mode 100755 index 9327e21e43db..000000000000 --- a/ec2/spark_ec2.py +++ /dev/null @@ -1,1520 +0,0 @@ -#!/usr/bin/env python -# -*- coding: utf-8 -*- - -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT 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 __future__ import division, print_function, with_statement - -import codecs -import hashlib -import itertools -import logging -import os -import os.path -import pipes -import random -import shutil -import string -from stat import S_IRUSR -import subprocess -import sys -import tarfile -import tempfile -import textwrap -import time -import warnings -from datetime import datetime -from optparse import OptionParser -from sys import stderr - -if sys.version < "3": - from urllib2 import urlopen, Request, HTTPError -else: - from urllib.request import urlopen, Request - from urllib.error import HTTPError - raw_input = input - xrange = range - -SPARK_EC2_VERSION = "1.5.0" -SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) - -VALID_SPARK_VERSIONS = set([ - "0.7.3", - "0.8.0", - "0.8.1", - "0.9.0", - "0.9.1", - "0.9.2", - "1.0.0", - "1.0.1", - "1.0.2", - "1.1.0", - "1.1.1", - "1.2.0", - "1.2.1", - "1.3.0", - "1.3.1", - "1.4.0", - "1.4.1", - "1.5.0" -]) - -SPARK_TACHYON_MAP = { - "1.0.0": "0.4.1", - "1.0.1": "0.4.1", - "1.0.2": "0.4.1", - "1.1.0": "0.5.0", - "1.1.1": "0.5.0", - "1.2.0": "0.5.0", - "1.2.1": "0.5.0", - "1.3.0": "0.5.0", - "1.3.1": "0.5.0", - "1.4.0": "0.6.4", - "1.4.1": "0.6.4", - "1.5.0": "0.7.1" -} - -DEFAULT_SPARK_VERSION = SPARK_EC2_VERSION -DEFAULT_SPARK_GITHUB_REPO = "https://github.com/apache/spark" - -# Default location to get the spark-ec2 scripts (and ami-list) from -DEFAULT_SPARK_EC2_GITHUB_REPO = "https://github.com/amplab/spark-ec2" -DEFAULT_SPARK_EC2_BRANCH = "branch-1.5" - - -def setup_external_libs(libs): - """ - Download external libraries from PyPI to SPARK_EC2_DIR/lib/ and prepend them to our PATH. - """ - PYPI_URL_PREFIX = "https://pypi.python.org/packages/source" - SPARK_EC2_LIB_DIR = os.path.join(SPARK_EC2_DIR, "lib") - - if not os.path.exists(SPARK_EC2_LIB_DIR): - print("Downloading external libraries that spark-ec2 needs from PyPI to {path}...".format( - path=SPARK_EC2_LIB_DIR - )) - print("This should be a one-time operation.") - os.mkdir(SPARK_EC2_LIB_DIR) - - for lib in libs: - versioned_lib_name = "{n}-{v}".format(n=lib["name"], v=lib["version"]) - lib_dir = os.path.join(SPARK_EC2_LIB_DIR, versioned_lib_name) - - if not os.path.isdir(lib_dir): - tgz_file_path = os.path.join(SPARK_EC2_LIB_DIR, versioned_lib_name + ".tar.gz") - print(" - Downloading {lib}...".format(lib=lib["name"])) - download_stream = urlopen( - "{prefix}/{first_letter}/{lib_name}/{lib_name}-{lib_version}.tar.gz".format( - prefix=PYPI_URL_PREFIX, - first_letter=lib["name"][:1], - lib_name=lib["name"], - lib_version=lib["version"] - ) - ) - with open(tgz_file_path, "wb") as tgz_file: - tgz_file.write(download_stream.read()) - with open(tgz_file_path, "rb") as tar: - if hashlib.md5(tar.read()).hexdigest() != lib["md5"]: - print("ERROR: Got wrong md5sum for {lib}.".format(lib=lib["name"]), file=stderr) - sys.exit(1) - tar = tarfile.open(tgz_file_path) - tar.extractall(path=SPARK_EC2_LIB_DIR) - tar.close() - os.remove(tgz_file_path) - print(" - Finished downloading {lib}.".format(lib=lib["name"])) - sys.path.insert(1, lib_dir) - - -# Only PyPI libraries are supported. -external_libs = [ - { - "name": "boto", - "version": "2.34.0", - "md5": "5556223d2d0cc4d06dd4829e671dcecd" - } -] - -setup_external_libs(external_libs) - -import boto -from boto.ec2.blockdevicemapping import BlockDeviceMapping, BlockDeviceType, EBSBlockDeviceType -from boto import ec2 - - -class UsageError(Exception): - pass - - -# Configure and parse our command-line arguments -def parse_args(): - parser = OptionParser( - prog="spark-ec2", - version="%prog {v}".format(v=SPARK_EC2_VERSION), - usage="%prog [options] \n\n" - + " can be: launch, destroy, login, stop, start, get-master, reboot-slaves") - - parser.add_option( - "-s", "--slaves", type="int", default=1, - help="Number of slaves to launch (default: %default)") - parser.add_option( - "-w", "--wait", type="int", - help="DEPRECATED (no longer necessary) - Seconds to wait for nodes to start") - parser.add_option( - "-k", "--key-pair", - help="Key pair to use on instances") - parser.add_option( - "-i", "--identity-file", - help="SSH private key file to use for logging into instances") - parser.add_option( - "-p", "--profile", default=None, - help="If you have multiple profiles (AWS or boto config), you can configure " + - "additional, named profiles by using this option (default: %default)") - parser.add_option( - "-t", "--instance-type", default="m1.large", - help="Type of instance to launch (default: %default). " + - "WARNING: must be 64-bit; small instances won't work") - parser.add_option( - "-m", "--master-instance-type", default="", - help="Master instance type (leave empty for same as instance-type)") - parser.add_option( - "-r", "--region", default="us-east-1", - help="EC2 region used to launch instances in, or to find them in (default: %default)") - parser.add_option( - "-z", "--zone", default="", - help="Availability zone to launch instances in, or 'all' to spread " + - "slaves across multiple (an additional $0.01/Gb for bandwidth" + - "between zones applies) (default: a single zone chosen at random)") - parser.add_option( - "-a", "--ami", - help="Amazon Machine Image ID to use") - parser.add_option( - "-v", "--spark-version", default=DEFAULT_SPARK_VERSION, - help="Version of Spark to use: 'X.Y.Z' or a specific git hash (default: %default)") - parser.add_option( - "--spark-git-repo", - default=DEFAULT_SPARK_GITHUB_REPO, - help="Github repo from which to checkout supplied commit hash (default: %default)") - parser.add_option( - "--spark-ec2-git-repo", - default=DEFAULT_SPARK_EC2_GITHUB_REPO, - help="Github repo from which to checkout spark-ec2 (default: %default)") - parser.add_option( - "--spark-ec2-git-branch", - default=DEFAULT_SPARK_EC2_BRANCH, - help="Github repo branch of spark-ec2 to use (default: %default)") - parser.add_option( - "--deploy-root-dir", - default=None, - help="A directory to copy into / on the first master. " + - "Must be absolute. Note that a trailing slash is handled as per rsync: " + - "If you omit it, the last directory of the --deploy-root-dir path will be created " + - "in / before copying its contents. If you append the trailing slash, " + - "the directory is not created and its contents are copied directly into /. " + - "(default: %default).") - parser.add_option( - "--hadoop-major-version", default="1", - help="Major version of Hadoop. Valid options are 1 (Hadoop 1.0.4), 2 (CDH 4.2.0), yarn " + - "(Hadoop 2.4.0) (default: %default)") - parser.add_option( - "-D", metavar="[ADDRESS:]PORT", dest="proxy_port", - help="Use SSH dynamic port forwarding to create a SOCKS proxy at " + - "the given local address (for use with login)") - parser.add_option( - "--resume", action="store_true", default=False, - help="Resume installation on a previously launched cluster " + - "(for debugging)") - parser.add_option( - "--ebs-vol-size", metavar="SIZE", type="int", default=0, - help="Size (in GB) of each EBS volume.") - parser.add_option( - "--ebs-vol-type", default="standard", - help="EBS volume type (e.g. 'gp2', 'standard').") - parser.add_option( - "--ebs-vol-num", type="int", default=1, - help="Number of EBS volumes to attach to each node as /vol[x]. " + - "The volumes will be deleted when the instances terminate. " + - "Only possible on EBS-backed AMIs. " + - "EBS volumes are only attached if --ebs-vol-size > 0. " + - "Only support up to 8 EBS volumes.") - parser.add_option( - "--placement-group", type="string", default=None, - help="Which placement group to try and launch " + - "instances into. Assumes placement group is already " + - "created.") - parser.add_option( - "--swap", metavar="SWAP", type="int", default=1024, - help="Swap space to set up per node, in MB (default: %default)") - parser.add_option( - "--spot-price", metavar="PRICE", type="float", - help="If specified, launch slaves as spot instances with the given " + - "maximum price (in dollars)") - parser.add_option( - "--ganglia", action="store_true", default=True, - help="Setup Ganglia monitoring on cluster (default: %default). NOTE: " + - "the Ganglia page will be publicly accessible") - parser.add_option( - "--no-ganglia", action="store_false", dest="ganglia", - help="Disable Ganglia monitoring for the cluster") - parser.add_option( - "-u", "--user", default="root", - help="The SSH user you want to connect as (default: %default)") - parser.add_option( - "--delete-groups", action="store_true", default=False, - help="When destroying a cluster, delete the security groups that were created") - parser.add_option( - "--use-existing-master", action="store_true", default=False, - help="Launch fresh slaves, but use an existing stopped master if possible") - parser.add_option( - "--worker-instances", type="int", default=1, - help="Number of instances per worker: variable SPARK_WORKER_INSTANCES. Not used if YARN " + - "is used as Hadoop major version (default: %default)") - parser.add_option( - "--master-opts", type="string", default="", - help="Extra options to give to master through SPARK_MASTER_OPTS variable " + - "(e.g -Dspark.worker.timeout=180)") - parser.add_option( - "--user-data", type="string", default="", - help="Path to a user-data file (most AMIs interpret this as an initialization script)") - parser.add_option( - "--authorized-address", type="string", default="0.0.0.0/0", - help="Address to authorize on created security groups (default: %default)") - parser.add_option( - "--additional-security-group", type="string", default="", - help="Additional security group to place the machines in") - parser.add_option( - "--additional-tags", type="string", default="", - help="Additional tags to set on the machines; tags are comma-separated, while name and " + - "value are colon separated; ex: \"Task:MySparkProject,Env:production\"") - parser.add_option( - "--copy-aws-credentials", action="store_true", default=False, - help="Add AWS credentials to hadoop configuration to allow Spark to access S3") - parser.add_option( - "--subnet-id", default=None, - help="VPC subnet to launch instances in") - parser.add_option( - "--vpc-id", default=None, - help="VPC to launch instances in") - parser.add_option( - "--private-ips", action="store_true", default=False, - help="Use private IPs for instances rather than public if VPC/subnet " + - "requires that.") - parser.add_option( - "--instance-initiated-shutdown-behavior", default="stop", - choices=["stop", "terminate"], - help="Whether instances should terminate when shut down or just stop") - parser.add_option( - "--instance-profile-name", default=None, - help="IAM profile name to launch instances under") - - (opts, args) = parser.parse_args() - if len(args) != 2: - parser.print_help() - sys.exit(1) - (action, cluster_name) = args - - # Boto config check - # http://boto.cloudhackers.com/en/latest/boto_config_tut.html - home_dir = os.getenv('HOME') - if home_dir is None or not os.path.isfile(home_dir + '/.boto'): - if not os.path.isfile('/etc/boto.cfg'): - # If there is no boto config, check aws credentials - if not os.path.isfile(home_dir + '/.aws/credentials'): - if os.getenv('AWS_ACCESS_KEY_ID') is None: - print("ERROR: The environment variable AWS_ACCESS_KEY_ID must be set", - file=stderr) - sys.exit(1) - if os.getenv('AWS_SECRET_ACCESS_KEY') is None: - print("ERROR: The environment variable AWS_SECRET_ACCESS_KEY must be set", - file=stderr) - sys.exit(1) - return (opts, action, cluster_name) - - -# Get the EC2 security group of the given name, creating it if it doesn't exist -def get_or_make_group(conn, name, vpc_id): - groups = conn.get_all_security_groups() - group = [g for g in groups if g.name == name] - if len(group) > 0: - return group[0] - else: - print("Creating security group " + name) - return conn.create_security_group(name, "Spark EC2 group", vpc_id) - - -def get_validate_spark_version(version, repo): - if "." in version: - version = version.replace("v", "") - if version not in VALID_SPARK_VERSIONS: - print("Don't know about Spark version: {v}".format(v=version), file=stderr) - sys.exit(1) - return version - else: - github_commit_url = "{repo}/commit/{commit_hash}".format(repo=repo, commit_hash=version) - request = Request(github_commit_url) - request.get_method = lambda: 'HEAD' - try: - response = urlopen(request) - except HTTPError as e: - print("Couldn't validate Spark commit: {url}".format(url=github_commit_url), - file=stderr) - print("Received HTTP response code of {code}.".format(code=e.code), file=stderr) - sys.exit(1) - return version - - -# Source: http://aws.amazon.com/amazon-linux-ami/instance-type-matrix/ -# Last Updated: 2015-06-19 -# For easy maintainability, please keep this manually-inputted dictionary sorted by key. -EC2_INSTANCE_TYPES = { - "c1.medium": "pvm", - "c1.xlarge": "pvm", - "c3.large": "pvm", - "c3.xlarge": "pvm", - "c3.2xlarge": "pvm", - "c3.4xlarge": "pvm", - "c3.8xlarge": "pvm", - "c4.large": "hvm", - "c4.xlarge": "hvm", - "c4.2xlarge": "hvm", - "c4.4xlarge": "hvm", - "c4.8xlarge": "hvm", - "cc1.4xlarge": "hvm", - "cc2.8xlarge": "hvm", - "cg1.4xlarge": "hvm", - "cr1.8xlarge": "hvm", - "d2.xlarge": "hvm", - "d2.2xlarge": "hvm", - "d2.4xlarge": "hvm", - "d2.8xlarge": "hvm", - "g2.2xlarge": "hvm", - "g2.8xlarge": "hvm", - "hi1.4xlarge": "pvm", - "hs1.8xlarge": "pvm", - "i2.xlarge": "hvm", - "i2.2xlarge": "hvm", - "i2.4xlarge": "hvm", - "i2.8xlarge": "hvm", - "m1.small": "pvm", - "m1.medium": "pvm", - "m1.large": "pvm", - "m1.xlarge": "pvm", - "m2.xlarge": "pvm", - "m2.2xlarge": "pvm", - "m2.4xlarge": "pvm", - "m3.medium": "hvm", - "m3.large": "hvm", - "m3.xlarge": "hvm", - "m3.2xlarge": "hvm", - "m4.large": "hvm", - "m4.xlarge": "hvm", - "m4.2xlarge": "hvm", - "m4.4xlarge": "hvm", - "m4.10xlarge": "hvm", - "r3.large": "hvm", - "r3.xlarge": "hvm", - "r3.2xlarge": "hvm", - "r3.4xlarge": "hvm", - "r3.8xlarge": "hvm", - "t1.micro": "pvm", - "t2.micro": "hvm", - "t2.small": "hvm", - "t2.medium": "hvm", - "t2.large": "hvm", -} - - -def get_tachyon_version(spark_version): - return SPARK_TACHYON_MAP.get(spark_version, "") - - -# Attempt to resolve an appropriate AMI given the architecture and region of the request. -def get_spark_ami(opts): - if opts.instance_type in EC2_INSTANCE_TYPES: - instance_type = EC2_INSTANCE_TYPES[opts.instance_type] - else: - instance_type = "pvm" - print("Don't recognize %s, assuming type is pvm" % opts.instance_type, file=stderr) - - # URL prefix from which to fetch AMI information - ami_prefix = "{r}/{b}/ami-list".format( - r=opts.spark_ec2_git_repo.replace("https://github.com", "https://raw.github.com", 1), - b=opts.spark_ec2_git_branch) - - ami_path = "%s/%s/%s" % (ami_prefix, opts.region, instance_type) - reader = codecs.getreader("ascii") - try: - ami = reader(urlopen(ami_path)).read().strip() - except: - print("Could not resolve AMI at: " + ami_path, file=stderr) - sys.exit(1) - - print("Spark AMI: " + ami) - return ami - - -# Launch a cluster of the given name, by setting up its security groups, -# and then starting new instances in them. -# Returns a tuple of EC2 reservation objects for the master and slaves -# Fails if there already instances running in the cluster's groups. -def launch_cluster(conn, opts, cluster_name): - if opts.identity_file is None: - print("ERROR: Must provide an identity file (-i) for ssh connections.", file=stderr) - sys.exit(1) - - if opts.key_pair is None: - print("ERROR: Must provide a key pair name (-k) to use on instances.", file=stderr) - sys.exit(1) - - user_data_content = None - if opts.user_data: - with open(opts.user_data) as user_data_file: - user_data_content = user_data_file.read() - - print("Setting up security groups...") - master_group = get_or_make_group(conn, cluster_name + "-master", opts.vpc_id) - slave_group = get_or_make_group(conn, cluster_name + "-slaves", opts.vpc_id) - authorized_address = opts.authorized_address - if master_group.rules == []: # Group was just now created - if opts.vpc_id is None: - master_group.authorize(src_group=master_group) - master_group.authorize(src_group=slave_group) - else: - master_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, - src_group=master_group) - master_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, - src_group=master_group) - master_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, - src_group=master_group) - master_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, - src_group=slave_group) - master_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, - src_group=slave_group) - master_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, - src_group=slave_group) - master_group.authorize('tcp', 22, 22, authorized_address) - master_group.authorize('tcp', 8080, 8081, authorized_address) - master_group.authorize('tcp', 18080, 18080, authorized_address) - master_group.authorize('tcp', 19999, 19999, authorized_address) - master_group.authorize('tcp', 50030, 50030, authorized_address) - master_group.authorize('tcp', 50070, 50070, authorized_address) - master_group.authorize('tcp', 60070, 60070, authorized_address) - master_group.authorize('tcp', 4040, 4045, authorized_address) - # Rstudio (GUI for R) needs port 8787 for web access - master_group.authorize('tcp', 8787, 8787, authorized_address) - # HDFS NFS gateway requires 111,2049,4242 for tcp & udp - master_group.authorize('tcp', 111, 111, authorized_address) - master_group.authorize('udp', 111, 111, authorized_address) - master_group.authorize('tcp', 2049, 2049, authorized_address) - master_group.authorize('udp', 2049, 2049, authorized_address) - master_group.authorize('tcp', 4242, 4242, authorized_address) - master_group.authorize('udp', 4242, 4242, authorized_address) - # RM in YARN mode uses 8088 - master_group.authorize('tcp', 8088, 8088, authorized_address) - if opts.ganglia: - master_group.authorize('tcp', 5080, 5080, authorized_address) - if slave_group.rules == []: # Group was just now created - if opts.vpc_id is None: - slave_group.authorize(src_group=master_group) - slave_group.authorize(src_group=slave_group) - else: - slave_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, - src_group=master_group) - slave_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, - src_group=master_group) - slave_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, - src_group=master_group) - slave_group.authorize(ip_protocol='icmp', from_port=-1, to_port=-1, - src_group=slave_group) - slave_group.authorize(ip_protocol='tcp', from_port=0, to_port=65535, - src_group=slave_group) - slave_group.authorize(ip_protocol='udp', from_port=0, to_port=65535, - src_group=slave_group) - slave_group.authorize('tcp', 22, 22, authorized_address) - slave_group.authorize('tcp', 8080, 8081, authorized_address) - slave_group.authorize('tcp', 50060, 50060, authorized_address) - slave_group.authorize('tcp', 50075, 50075, authorized_address) - slave_group.authorize('tcp', 60060, 60060, authorized_address) - slave_group.authorize('tcp', 60075, 60075, authorized_address) - - # Check if instances are already running in our groups - existing_masters, existing_slaves = get_existing_cluster(conn, opts, cluster_name, - die_on_error=False) - if existing_slaves or (existing_masters and not opts.use_existing_master): - print("ERROR: There are already instances running in group %s or %s" % - (master_group.name, slave_group.name), file=stderr) - sys.exit(1) - - # Figure out Spark AMI - if opts.ami is None: - opts.ami = get_spark_ami(opts) - - # we use group ids to work around https://github.com/boto/boto/issues/350 - additional_group_ids = [] - if opts.additional_security_group: - additional_group_ids = [sg.id - for sg in conn.get_all_security_groups() - if opts.additional_security_group in (sg.name, sg.id)] - print("Launching instances...") - - try: - image = conn.get_all_images(image_ids=[opts.ami])[0] - except: - print("Could not find AMI " + opts.ami, file=stderr) - sys.exit(1) - - # Create block device mapping so that we can add EBS volumes if asked to. - # The first drive is attached as /dev/sds, 2nd as /dev/sdt, ... /dev/sdz - block_map = BlockDeviceMapping() - if opts.ebs_vol_size > 0: - for i in range(opts.ebs_vol_num): - device = EBSBlockDeviceType() - device.size = opts.ebs_vol_size - device.volume_type = opts.ebs_vol_type - device.delete_on_termination = True - block_map["/dev/sd" + chr(ord('s') + i)] = device - - # AWS ignores the AMI-specified block device mapping for M3 (see SPARK-3342). - if opts.instance_type.startswith('m3.'): - for i in range(get_num_disks(opts.instance_type)): - dev = BlockDeviceType() - dev.ephemeral_name = 'ephemeral%d' % i - # The first ephemeral drive is /dev/sdb. - name = '/dev/sd' + string.letters[i + 1] - block_map[name] = dev - - # Launch slaves - if opts.spot_price is not None: - # Launch spot instances with the requested price - print("Requesting %d slaves as spot instances with price $%.3f" % - (opts.slaves, opts.spot_price)) - zones = get_zones(conn, opts) - num_zones = len(zones) - i = 0 - my_req_ids = [] - for zone in zones: - num_slaves_this_zone = get_partition(opts.slaves, num_zones, i) - slave_reqs = conn.request_spot_instances( - price=opts.spot_price, - image_id=opts.ami, - launch_group="launch-group-%s" % cluster_name, - placement=zone, - count=num_slaves_this_zone, - key_name=opts.key_pair, - security_group_ids=[slave_group.id] + additional_group_ids, - instance_type=opts.instance_type, - block_device_map=block_map, - subnet_id=opts.subnet_id, - placement_group=opts.placement_group, - user_data=user_data_content, - instance_profile_name=opts.instance_profile_name) - my_req_ids += [req.id for req in slave_reqs] - i += 1 - - print("Waiting for spot instances to be granted...") - try: - while True: - time.sleep(10) - reqs = conn.get_all_spot_instance_requests() - id_to_req = {} - for r in reqs: - id_to_req[r.id] = r - active_instance_ids = [] - for i in my_req_ids: - if i in id_to_req and id_to_req[i].state == "active": - active_instance_ids.append(id_to_req[i].instance_id) - if len(active_instance_ids) == opts.slaves: - print("All %d slaves granted" % opts.slaves) - reservations = conn.get_all_reservations(active_instance_ids) - slave_nodes = [] - for r in reservations: - slave_nodes += r.instances - break - else: - print("%d of %d slaves granted, waiting longer" % ( - len(active_instance_ids), opts.slaves)) - except: - print("Canceling spot instance requests") - conn.cancel_spot_instance_requests(my_req_ids) - # Log a warning if any of these requests actually launched instances: - (master_nodes, slave_nodes) = get_existing_cluster( - conn, opts, cluster_name, die_on_error=False) - running = len(master_nodes) + len(slave_nodes) - if running: - print(("WARNING: %d instances are still running" % running), file=stderr) - sys.exit(0) - else: - # Launch non-spot instances - zones = get_zones(conn, opts) - num_zones = len(zones) - i = 0 - slave_nodes = [] - for zone in zones: - num_slaves_this_zone = get_partition(opts.slaves, num_zones, i) - if num_slaves_this_zone > 0: - slave_res = image.run( - key_name=opts.key_pair, - security_group_ids=[slave_group.id] + additional_group_ids, - instance_type=opts.instance_type, - placement=zone, - min_count=num_slaves_this_zone, - max_count=num_slaves_this_zone, - block_device_map=block_map, - subnet_id=opts.subnet_id, - placement_group=opts.placement_group, - user_data=user_data_content, - instance_initiated_shutdown_behavior=opts.instance_initiated_shutdown_behavior, - instance_profile_name=opts.instance_profile_name) - slave_nodes += slave_res.instances - print("Launched {s} slave{plural_s} in {z}, regid = {r}".format( - s=num_slaves_this_zone, - plural_s=('' if num_slaves_this_zone == 1 else 's'), - z=zone, - r=slave_res.id)) - i += 1 - - # Launch or resume masters - if existing_masters: - print("Starting master...") - for inst in existing_masters: - if inst.state not in ["shutting-down", "terminated"]: - inst.start() - master_nodes = existing_masters - else: - master_type = opts.master_instance_type - if master_type == "": - master_type = opts.instance_type - if opts.zone == 'all': - opts.zone = random.choice(conn.get_all_zones()).name - master_res = image.run( - key_name=opts.key_pair, - security_group_ids=[master_group.id] + additional_group_ids, - instance_type=master_type, - placement=opts.zone, - min_count=1, - max_count=1, - block_device_map=block_map, - subnet_id=opts.subnet_id, - placement_group=opts.placement_group, - user_data=user_data_content, - instance_initiated_shutdown_behavior=opts.instance_initiated_shutdown_behavior, - instance_profile_name=opts.instance_profile_name) - - master_nodes = master_res.instances - print("Launched master in %s, regid = %s" % (zone, master_res.id)) - - # This wait time corresponds to SPARK-4983 - print("Waiting for AWS to propagate instance metadata...") - time.sleep(15) - - # Give the instances descriptive names and set additional tags - additional_tags = {} - if opts.additional_tags.strip(): - additional_tags = dict( - map(str.strip, tag.split(':', 1)) for tag in opts.additional_tags.split(',') - ) - - for master in master_nodes: - master.add_tags( - dict(additional_tags, Name='{cn}-master-{iid}'.format(cn=cluster_name, iid=master.id)) - ) - - for slave in slave_nodes: - slave.add_tags( - dict(additional_tags, Name='{cn}-slave-{iid}'.format(cn=cluster_name, iid=slave.id)) - ) - - # Return all the instances - return (master_nodes, slave_nodes) - - -def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): - """ - Get the EC2 instances in an existing cluster if available. - Returns a tuple of lists of EC2 instance objects for the masters and slaves. - """ - print("Searching for existing cluster {c} in region {r}...".format( - c=cluster_name, r=opts.region)) - - def get_instances(group_names): - """ - Get all non-terminated instances that belong to any of the provided security groups. - - EC2 reservation filters and instance states are documented here: - http://docs.aws.amazon.com/cli/latest/reference/ec2/describe-instances.html#options - """ - reservations = conn.get_all_reservations( - filters={"instance.group-name": group_names}) - instances = itertools.chain.from_iterable(r.instances for r in reservations) - return [i for i in instances if i.state not in ["shutting-down", "terminated"]] - - master_instances = get_instances([cluster_name + "-master"]) - slave_instances = get_instances([cluster_name + "-slaves"]) - - if any((master_instances, slave_instances)): - print("Found {m} master{plural_m}, {s} slave{plural_s}.".format( - m=len(master_instances), - plural_m=('' if len(master_instances) == 1 else 's'), - s=len(slave_instances), - plural_s=('' if len(slave_instances) == 1 else 's'))) - - if not master_instances and die_on_error: - print("ERROR: Could not find a master for cluster {c} in region {r}.".format( - c=cluster_name, r=opts.region), file=sys.stderr) - sys.exit(1) - - return (master_instances, slave_instances) - - -# Deploy configuration files and run setup scripts on a newly launched -# or started EC2 cluster. -def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): - master = get_dns_name(master_nodes[0], opts.private_ips) - if deploy_ssh_key: - print("Generating cluster's SSH key on master...") - key_setup = """ - [ -f ~/.ssh/id_rsa ] || - (ssh-keygen -q -t rsa -N '' -f ~/.ssh/id_rsa && - cat ~/.ssh/id_rsa.pub >> ~/.ssh/authorized_keys) - """ - ssh(master, opts, key_setup) - dot_ssh_tar = ssh_read(master, opts, ['tar', 'c', '.ssh']) - print("Transferring cluster's SSH key to slaves...") - for slave in slave_nodes: - slave_address = get_dns_name(slave, opts.private_ips) - print(slave_address) - ssh_write(slave_address, opts, ['tar', 'x'], dot_ssh_tar) - - modules = ['spark', 'ephemeral-hdfs', 'persistent-hdfs', - 'mapreduce', 'spark-standalone', 'tachyon', 'rstudio'] - - if opts.hadoop_major_version == "1": - modules = list(filter(lambda x: x != "mapreduce", modules)) - - if opts.ganglia: - modules.append('ganglia') - - # Clear SPARK_WORKER_INSTANCES if running on YARN - if opts.hadoop_major_version == "yarn": - opts.worker_instances = "" - - # NOTE: We should clone the repository before running deploy_files to - # prevent ec2-variables.sh from being overwritten - print("Cloning spark-ec2 scripts from {r}/tree/{b} on master...".format( - r=opts.spark_ec2_git_repo, b=opts.spark_ec2_git_branch)) - ssh( - host=master, - opts=opts, - command="rm -rf spark-ec2" - + " && " - + "git clone {r} -b {b} spark-ec2".format(r=opts.spark_ec2_git_repo, - b=opts.spark_ec2_git_branch) - ) - - print("Deploying files to master...") - deploy_files( - conn=conn, - root_dir=SPARK_EC2_DIR + "/" + "deploy.generic", - opts=opts, - master_nodes=master_nodes, - slave_nodes=slave_nodes, - modules=modules - ) - - if opts.deploy_root_dir is not None: - print("Deploying {s} to master...".format(s=opts.deploy_root_dir)) - deploy_user_files( - root_dir=opts.deploy_root_dir, - opts=opts, - master_nodes=master_nodes - ) - - print("Running setup on master...") - setup_spark_cluster(master, opts) - print("Done!") - - -def setup_spark_cluster(master, opts): - ssh(master, opts, "chmod u+x spark-ec2/setup.sh") - ssh(master, opts, "spark-ec2/setup.sh") - print("Spark standalone cluster started at http://%s:8080" % master) - - if opts.ganglia: - print("Ganglia started at http://%s:5080/ganglia" % master) - - -def is_ssh_available(host, opts, print_ssh_output=True): - """ - Check if SSH is available on a host. - """ - s = subprocess.Popen( - ssh_command(opts) + ['-t', '-t', '-o', 'ConnectTimeout=3', - '%s@%s' % (opts.user, host), stringify_command('true')], - stdout=subprocess.PIPE, - stderr=subprocess.STDOUT # we pipe stderr through stdout to preserve output order - ) - cmd_output = s.communicate()[0] # [1] is stderr, which we redirected to stdout - - if s.returncode != 0 and print_ssh_output: - # extra leading newline is for spacing in wait_for_cluster_state() - print(textwrap.dedent("""\n - Warning: SSH connection error. (This could be temporary.) - Host: {h} - SSH return code: {r} - SSH output: {o} - """).format( - h=host, - r=s.returncode, - o=cmd_output.strip() - )) - - return s.returncode == 0 - - -def is_cluster_ssh_available(cluster_instances, opts): - """ - Check if SSH is available on all the instances in a cluster. - """ - for i in cluster_instances: - dns_name = get_dns_name(i, opts.private_ips) - if not is_ssh_available(host=dns_name, opts=opts): - return False - else: - return True - - -def wait_for_cluster_state(conn, opts, cluster_instances, cluster_state): - """ - Wait for all the instances in the cluster to reach a designated state. - - cluster_instances: a list of boto.ec2.instance.Instance - cluster_state: a string representing the desired state of all the instances in the cluster - value can be 'ssh-ready' or a valid value from boto.ec2.instance.InstanceState such as - 'running', 'terminated', etc. - (would be nice to replace this with a proper enum: http://stackoverflow.com/a/1695250) - """ - sys.stdout.write( - "Waiting for cluster to enter '{s}' state.".format(s=cluster_state) - ) - sys.stdout.flush() - - start_time = datetime.now() - num_attempts = 0 - - while True: - time.sleep(5 * num_attempts) # seconds - - for i in cluster_instances: - i.update() - - max_batch = 100 - statuses = [] - for j in xrange(0, len(cluster_instances), max_batch): - batch = [i.id for i in cluster_instances[j:j + max_batch]] - statuses.extend(conn.get_all_instance_status(instance_ids=batch)) - - if cluster_state == 'ssh-ready': - if all(i.state == 'running' for i in cluster_instances) and \ - all(s.system_status.status == 'ok' for s in statuses) and \ - all(s.instance_status.status == 'ok' for s in statuses) and \ - is_cluster_ssh_available(cluster_instances, opts): - break - else: - if all(i.state == cluster_state for i in cluster_instances): - break - - num_attempts += 1 - - sys.stdout.write(".") - sys.stdout.flush() - - sys.stdout.write("\n") - - end_time = datetime.now() - print("Cluster is now in '{s}' state. Waited {t} seconds.".format( - s=cluster_state, - t=(end_time - start_time).seconds - )) - - -# Get number of local disks available for a given EC2 instance type. -def get_num_disks(instance_type): - # Source: http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/InstanceStorage.html - # Last Updated: 2015-06-19 - # For easy maintainability, please keep this manually-inputted dictionary sorted by key. - disks_by_instance = { - "c1.medium": 1, - "c1.xlarge": 4, - "c3.large": 2, - "c3.xlarge": 2, - "c3.2xlarge": 2, - "c3.4xlarge": 2, - "c3.8xlarge": 2, - "c4.large": 0, - "c4.xlarge": 0, - "c4.2xlarge": 0, - "c4.4xlarge": 0, - "c4.8xlarge": 0, - "cc1.4xlarge": 2, - "cc2.8xlarge": 4, - "cg1.4xlarge": 2, - "cr1.8xlarge": 2, - "d2.xlarge": 3, - "d2.2xlarge": 6, - "d2.4xlarge": 12, - "d2.8xlarge": 24, - "g2.2xlarge": 1, - "g2.8xlarge": 2, - "hi1.4xlarge": 2, - "hs1.8xlarge": 24, - "i2.xlarge": 1, - "i2.2xlarge": 2, - "i2.4xlarge": 4, - "i2.8xlarge": 8, - "m1.small": 1, - "m1.medium": 1, - "m1.large": 2, - "m1.xlarge": 4, - "m2.xlarge": 1, - "m2.2xlarge": 1, - "m2.4xlarge": 2, - "m3.medium": 1, - "m3.large": 1, - "m3.xlarge": 2, - "m3.2xlarge": 2, - "m4.large": 0, - "m4.xlarge": 0, - "m4.2xlarge": 0, - "m4.4xlarge": 0, - "m4.10xlarge": 0, - "r3.large": 1, - "r3.xlarge": 1, - "r3.2xlarge": 1, - "r3.4xlarge": 1, - "r3.8xlarge": 2, - "t1.micro": 0, - "t2.micro": 0, - "t2.small": 0, - "t2.medium": 0, - "t2.large": 0, - } - if instance_type in disks_by_instance: - return disks_by_instance[instance_type] - else: - print("WARNING: Don't know number of disks on instance type %s; assuming 1" - % instance_type, file=stderr) - return 1 - - -# Deploy the configuration file templates in a given local directory to -# a cluster, filling in any template parameters with information about the -# cluster (e.g. lists of masters and slaves). Files are only deployed to -# the first master instance in the cluster, and we expect the setup -# script to be run on that instance to copy them to other nodes. -# -# root_dir should be an absolute path to the directory with the files we want to deploy. -def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): - active_master = get_dns_name(master_nodes[0], opts.private_ips) - - num_disks = get_num_disks(opts.instance_type) - hdfs_data_dirs = "/mnt/ephemeral-hdfs/data" - mapred_local_dirs = "/mnt/hadoop/mrlocal" - spark_local_dirs = "/mnt/spark" - if num_disks > 1: - for i in range(2, num_disks + 1): - hdfs_data_dirs += ",/mnt%d/ephemeral-hdfs/data" % i - mapred_local_dirs += ",/mnt%d/hadoop/mrlocal" % i - spark_local_dirs += ",/mnt%d/spark" % i - - cluster_url = "%s:7077" % active_master - - if "." in opts.spark_version: - # Pre-built Spark deploy - spark_v = get_validate_spark_version(opts.spark_version, opts.spark_git_repo) - tachyon_v = get_tachyon_version(spark_v) - else: - # Spark-only custom deploy - spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version) - tachyon_v = "" - print("Deploying Spark via git hash; Tachyon won't be set up") - modules = filter(lambda x: x != "tachyon", modules) - - master_addresses = [get_dns_name(i, opts.private_ips) for i in master_nodes] - slave_addresses = [get_dns_name(i, opts.private_ips) for i in slave_nodes] - worker_instances_str = "%d" % opts.worker_instances if opts.worker_instances else "" - template_vars = { - "master_list": '\n'.join(master_addresses), - "active_master": active_master, - "slave_list": '\n'.join(slave_addresses), - "cluster_url": cluster_url, - "hdfs_data_dirs": hdfs_data_dirs, - "mapred_local_dirs": mapred_local_dirs, - "spark_local_dirs": spark_local_dirs, - "swap": str(opts.swap), - "modules": '\n'.join(modules), - "spark_version": spark_v, - "tachyon_version": tachyon_v, - "hadoop_major_version": opts.hadoop_major_version, - "spark_worker_instances": worker_instances_str, - "spark_master_opts": opts.master_opts - } - - if opts.copy_aws_credentials: - template_vars["aws_access_key_id"] = conn.aws_access_key_id - template_vars["aws_secret_access_key"] = conn.aws_secret_access_key - else: - template_vars["aws_access_key_id"] = "" - template_vars["aws_secret_access_key"] = "" - - # Create a temp directory in which we will place all the files to be - # deployed after we substitue template parameters in them - tmp_dir = tempfile.mkdtemp() - for path, dirs, files in os.walk(root_dir): - if path.find(".svn") == -1: - dest_dir = os.path.join('/', path[len(root_dir):]) - local_dir = tmp_dir + dest_dir - if not os.path.exists(local_dir): - os.makedirs(local_dir) - for filename in files: - if filename[0] not in '#.~' and filename[-1] != '~': - dest_file = os.path.join(dest_dir, filename) - local_file = tmp_dir + dest_file - with open(os.path.join(path, filename)) as src: - with open(local_file, "w") as dest: - text = src.read() - for key in template_vars: - text = text.replace("{{" + key + "}}", template_vars[key]) - dest.write(text) - dest.close() - # rsync the whole directory over to the master machine - command = [ - 'rsync', '-rv', - '-e', stringify_command(ssh_command(opts)), - "%s/" % tmp_dir, - "%s@%s:/" % (opts.user, active_master) - ] - subprocess.check_call(command) - # Remove the temp directory we created above - shutil.rmtree(tmp_dir) - - -# Deploy a given local directory to a cluster, WITHOUT parameter substitution. -# Note that unlike deploy_files, this works for binary files. -# Also, it is up to the user to add (or not) the trailing slash in root_dir. -# Files are only deployed to the first master instance in the cluster. -# -# root_dir should be an absolute path. -def deploy_user_files(root_dir, opts, master_nodes): - active_master = get_dns_name(master_nodes[0], opts.private_ips) - command = [ - 'rsync', '-rv', - '-e', stringify_command(ssh_command(opts)), - "%s" % root_dir, - "%s@%s:/" % (opts.user, active_master) - ] - subprocess.check_call(command) - - -def stringify_command(parts): - if isinstance(parts, str): - return parts - else: - return ' '.join(map(pipes.quote, parts)) - - -def ssh_args(opts): - parts = ['-o', 'StrictHostKeyChecking=no'] - parts += ['-o', 'UserKnownHostsFile=/dev/null'] - if opts.identity_file is not None: - parts += ['-i', opts.identity_file] - return parts - - -def ssh_command(opts): - return ['ssh'] + ssh_args(opts) - - -# Run a command on a host through ssh, retrying up to five times -# and then throwing an exception if ssh continues to fail. -def ssh(host, opts, command): - tries = 0 - while True: - try: - return subprocess.check_call( - ssh_command(opts) + ['-t', '-t', '%s@%s' % (opts.user, host), - stringify_command(command)]) - except subprocess.CalledProcessError as e: - if tries > 5: - # If this was an ssh failure, provide the user with hints. - if e.returncode == 255: - raise UsageError( - "Failed to SSH to remote host {0}.\n" - "Please check that you have provided the correct --identity-file and " - "--key-pair parameters and try again.".format(host)) - else: - raise e - print("Error executing remote command, retrying after 30 seconds: {0}".format(e), - file=stderr) - time.sleep(30) - tries = tries + 1 - - -# Backported from Python 2.7 for compatiblity with 2.6 (See SPARK-1990) -def _check_output(*popenargs, **kwargs): - if 'stdout' in kwargs: - raise ValueError('stdout argument not allowed, it will be overridden.') - process = subprocess.Popen(stdout=subprocess.PIPE, *popenargs, **kwargs) - output, unused_err = process.communicate() - retcode = process.poll() - if retcode: - cmd = kwargs.get("args") - if cmd is None: - cmd = popenargs[0] - raise subprocess.CalledProcessError(retcode, cmd, output=output) - return output - - -def ssh_read(host, opts, command): - return _check_output( - ssh_command(opts) + ['%s@%s' % (opts.user, host), stringify_command(command)]) - - -def ssh_write(host, opts, command, arguments): - tries = 0 - while True: - proc = subprocess.Popen( - ssh_command(opts) + ['%s@%s' % (opts.user, host), stringify_command(command)], - stdin=subprocess.PIPE) - proc.stdin.write(arguments) - proc.stdin.close() - status = proc.wait() - if status == 0: - break - elif tries > 5: - raise RuntimeError("ssh_write failed with error %s" % proc.returncode) - else: - print("Error {0} while executing remote command, retrying after 30 seconds". - format(status), file=stderr) - time.sleep(30) - tries = tries + 1 - - -# Gets a list of zones to launch instances in -def get_zones(conn, opts): - if opts.zone == 'all': - zones = [z.name for z in conn.get_all_zones()] - else: - zones = [opts.zone] - return zones - - -# Gets the number of items in a partition -def get_partition(total, num_partitions, current_partitions): - num_slaves_this_zone = total // num_partitions - if (total % num_partitions) - current_partitions > 0: - num_slaves_this_zone += 1 - return num_slaves_this_zone - - -# Gets the IP address, taking into account the --private-ips flag -def get_ip_address(instance, private_ips=False): - ip = instance.ip_address if not private_ips else \ - instance.private_ip_address - return ip - - -# Gets the DNS name, taking into account the --private-ips flag -def get_dns_name(instance, private_ips=False): - dns = instance.public_dns_name if not private_ips else \ - instance.private_ip_address - return dns - - -def real_main(): - (opts, action, cluster_name) = parse_args() - - # Input parameter validation - get_validate_spark_version(opts.spark_version, opts.spark_git_repo) - - if opts.wait is not None: - # NOTE: DeprecationWarnings are silent in 2.7+ by default. - # To show them, run Python with the -Wdefault switch. - # See: https://docs.python.org/3.5/whatsnew/2.7.html - warnings.warn( - "This option is deprecated and has no effect. " - "spark-ec2 automatically waits as long as necessary for clusters to start up.", - DeprecationWarning - ) - - if opts.identity_file is not None: - if not os.path.exists(opts.identity_file): - print("ERROR: The identity file '{f}' doesn't exist.".format(f=opts.identity_file), - file=stderr) - sys.exit(1) - - file_mode = os.stat(opts.identity_file).st_mode - if not (file_mode & S_IRUSR) or not oct(file_mode)[-2:] == '00': - print("ERROR: The identity file must be accessible only by you.", file=stderr) - print('You can fix this with: chmod 400 "{f}"'.format(f=opts.identity_file), - file=stderr) - sys.exit(1) - - if opts.instance_type not in EC2_INSTANCE_TYPES: - print("Warning: Unrecognized EC2 instance type for instance-type: {t}".format( - t=opts.instance_type), file=stderr) - - if opts.master_instance_type != "": - if opts.master_instance_type not in EC2_INSTANCE_TYPES: - print("Warning: Unrecognized EC2 instance type for master-instance-type: {t}".format( - t=opts.master_instance_type), file=stderr) - # Since we try instance types even if we can't resolve them, we check if they resolve first - # and, if they do, see if they resolve to the same virtualization type. - if opts.instance_type in EC2_INSTANCE_TYPES and \ - opts.master_instance_type in EC2_INSTANCE_TYPES: - if EC2_INSTANCE_TYPES[opts.instance_type] != \ - EC2_INSTANCE_TYPES[opts.master_instance_type]: - print("Error: spark-ec2 currently does not support having a master and slaves " - "with different AMI virtualization types.", file=stderr) - print("master instance virtualization type: {t}".format( - t=EC2_INSTANCE_TYPES[opts.master_instance_type]), file=stderr) - print("slave instance virtualization type: {t}".format( - t=EC2_INSTANCE_TYPES[opts.instance_type]), file=stderr) - sys.exit(1) - - if opts.ebs_vol_num > 8: - print("ebs-vol-num cannot be greater than 8", file=stderr) - sys.exit(1) - - # Prevent breaking ami_prefix (/, .git and startswith checks) - # Prevent forks with non spark-ec2 names for now. - if opts.spark_ec2_git_repo.endswith("/") or \ - opts.spark_ec2_git_repo.endswith(".git") or \ - not opts.spark_ec2_git_repo.startswith("https://github.com") or \ - not opts.spark_ec2_git_repo.endswith("spark-ec2"): - print("spark-ec2-git-repo must be a github repo and it must not have a trailing / or .git. " - "Furthermore, we currently only support forks named spark-ec2.", file=stderr) - sys.exit(1) - - if not (opts.deploy_root_dir is None or - (os.path.isabs(opts.deploy_root_dir) and - os.path.isdir(opts.deploy_root_dir) and - os.path.exists(opts.deploy_root_dir))): - print("--deploy-root-dir must be an absolute path to a directory that exists " - "on the local file system", file=stderr) - sys.exit(1) - - try: - if opts.profile is None: - conn = ec2.connect_to_region(opts.region) - else: - conn = ec2.connect_to_region(opts.region, profile_name=opts.profile) - except Exception as e: - print((e), file=stderr) - sys.exit(1) - - # Select an AZ at random if it was not specified. - if opts.zone == "": - opts.zone = random.choice(conn.get_all_zones()).name - - if action == "launch": - if opts.slaves <= 0: - print("ERROR: You have to start at least 1 slave", file=sys.stderr) - sys.exit(1) - if opts.resume: - (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - else: - (master_nodes, slave_nodes) = launch_cluster(conn, opts, cluster_name) - wait_for_cluster_state( - conn=conn, - opts=opts, - cluster_instances=(master_nodes + slave_nodes), - cluster_state='ssh-ready' - ) - setup_cluster(conn, master_nodes, slave_nodes, opts, True) - - elif action == "destroy": - (master_nodes, slave_nodes) = get_existing_cluster( - conn, opts, cluster_name, die_on_error=False) - - if any(master_nodes + slave_nodes): - print("The following instances will be terminated:") - for inst in master_nodes + slave_nodes: - print("> %s" % get_dns_name(inst, opts.private_ips)) - print("ALL DATA ON ALL NODES WILL BE LOST!!") - - msg = "Are you sure you want to destroy the cluster {c}? (y/N) ".format(c=cluster_name) - response = raw_input(msg) - if response == "y": - print("Terminating master...") - for inst in master_nodes: - inst.terminate() - print("Terminating slaves...") - for inst in slave_nodes: - inst.terminate() - - # Delete security groups as well - if opts.delete_groups: - group_names = [cluster_name + "-master", cluster_name + "-slaves"] - wait_for_cluster_state( - conn=conn, - opts=opts, - cluster_instances=(master_nodes + slave_nodes), - cluster_state='terminated' - ) - print("Deleting security groups (this will take some time)...") - attempt = 1 - while attempt <= 3: - print("Attempt %d" % attempt) - groups = [g for g in conn.get_all_security_groups() if g.name in group_names] - success = True - # Delete individual rules in all groups before deleting groups to - # remove dependencies between them - for group in groups: - print("Deleting rules in security group " + group.name) - for rule in group.rules: - for grant in rule.grants: - success &= group.revoke(ip_protocol=rule.ip_protocol, - from_port=rule.from_port, - to_port=rule.to_port, - src_group=grant) - - # Sleep for AWS eventual-consistency to catch up, and for instances - # to terminate - time.sleep(30) # Yes, it does have to be this long :-( - for group in groups: - try: - # It is needed to use group_id to make it work with VPC - conn.delete_security_group(group_id=group.id) - print("Deleted security group %s" % group.name) - except boto.exception.EC2ResponseError: - success = False - 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 - if success: - break - - attempt += 1 - - if not success: - print("Failed to delete all security groups after 3 tries.") - print("Try re-running in a few minutes.") - - elif action == "login": - (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - if not master_nodes[0].public_dns_name and not opts.private_ips: - print("Master has no public DNS name. Maybe you meant to specify --private-ips?") - else: - master = get_dns_name(master_nodes[0], opts.private_ips) - print("Logging into master " + master + "...") - proxy_opt = [] - if opts.proxy_port is not None: - proxy_opt = ['-D', opts.proxy_port] - subprocess.check_call( - ssh_command(opts) + proxy_opt + ['-t', '-t', "%s@%s" % (opts.user, master)]) - - elif action == "reboot-slaves": - response = raw_input( - "Are you sure you want to reboot the cluster " + - cluster_name + " slaves?\n" + - "Reboot cluster slaves " + cluster_name + " (y/N): ") - if response == "y": - (master_nodes, slave_nodes) = get_existing_cluster( - conn, opts, cluster_name, die_on_error=False) - print("Rebooting slaves...") - for inst in slave_nodes: - if inst.state not in ["shutting-down", "terminated"]: - print("Rebooting " + inst.id) - inst.reboot() - - elif action == "get-master": - (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - if not master_nodes[0].public_dns_name and not opts.private_ips: - print("Master has no public DNS name. Maybe you meant to specify --private-ips?") - else: - print(get_dns_name(master_nodes[0], opts.private_ips)) - - elif action == "stop": - response = raw_input( - "Are you sure you want to stop the cluster " + - cluster_name + "?\nDATA ON EPHEMERAL DISKS WILL BE LOST, " + - "BUT THE CLUSTER WILL KEEP USING SPACE ON\n" + - "AMAZON EBS IF IT IS EBS-BACKED!!\n" + - "All data on spot-instance slaves will be lost.\n" + - "Stop cluster " + cluster_name + " (y/N): ") - if response == "y": - (master_nodes, slave_nodes) = get_existing_cluster( - conn, opts, cluster_name, die_on_error=False) - print("Stopping master...") - for inst in master_nodes: - if inst.state not in ["shutting-down", "terminated"]: - inst.stop() - print("Stopping slaves...") - for inst in slave_nodes: - if inst.state not in ["shutting-down", "terminated"]: - if inst.spot_instance_request_id: - inst.terminate() - else: - inst.stop() - - elif action == "start": - (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - print("Starting slaves...") - for inst in slave_nodes: - if inst.state not in ["shutting-down", "terminated"]: - inst.start() - print("Starting master...") - for inst in master_nodes: - if inst.state not in ["shutting-down", "terminated"]: - inst.start() - wait_for_cluster_state( - conn=conn, - opts=opts, - cluster_instances=(master_nodes + slave_nodes), - cluster_state='ssh-ready' - ) - - # Determine types of running instances - existing_master_type = master_nodes[0].instance_type - existing_slave_type = slave_nodes[0].instance_type - # Setting opts.master_instance_type to the empty string indicates we - # have the same instance type for the master and the slaves - if existing_master_type == existing_slave_type: - existing_master_type = "" - opts.master_instance_type = existing_master_type - opts.instance_type = existing_slave_type - - setup_cluster(conn, master_nodes, slave_nodes, opts, False) - - else: - print("Invalid action: %s" % action, file=stderr) - sys.exit(1) - - -def main(): - try: - real_main() - except UsageError as e: - print("\nError:\n", e, file=stderr) - sys.exit(1) - - -if __name__ == "__main__": - logging.basicConfig() - main() diff --git a/examples/pom.xml b/examples/pom.xml index f5ab2a7fdc09..52a6764ae26a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -20,42 +20,49 @@ 4.0.0 org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT + spark-parent_2.11 + 2.3.0-SNAPSHOT ../pom.xml - org.apache.spark - spark-examples_2.10 - - examples - + spark-examples_2.11 jar Spark Project Examples http://spark.apache.org/ + + examples + none + package + provided + provided + provided + provided + + + - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} + org.spark-project.spark + unused + 1.0.0 provided org.apache.spark - spark-streaming_${scala.binary.version} + spark-core_${scala.binary.version} ${project.version} provided org.apache.spark - spark-mllib_${scala.binary.version} + spark-streaming_${scala.binary.version} ${project.version} provided org.apache.spark - spark-bagel_${scala.binary.version} + spark-mllib_${scala.binary.version} ${project.version} provided @@ -71,267 +78,49 @@ ${project.version} provided - - org.apache.spark - spark-streaming-twitter_${scala.binary.version} - ${project.version} - org.apache.spark spark-streaming-flume_${scala.binary.version} ${project.version} + provided org.apache.spark - spark-streaming-mqtt_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-streaming-zeromq_${scala.binary.version} + spark-streaming-kafka-0-10_${scala.binary.version} ${project.version} - - - org.spark-project.protobuf - protobuf-java - - + provided org.apache.spark - spark-streaming-kafka_${scala.binary.version} + spark-sql-kafka-0-10_${scala.binary.version} ${project.version} - - - org.apache.hbase - hbase-testing-util - ${hbase.version} - ${hbase.deps.scope} - - - - org.apache.hbase - hbase-annotations - - - org.jruby - jruby-complete - - - - - org.apache.hbase - hbase-protocol - ${hbase.version} - ${hbase.deps.scope} - - - org.apache.hbase - hbase-common - ${hbase.version} - ${hbase.deps.scope} - - - - org.apache.hbase - hbase-annotations - - - - - org.apache.hbase - hbase-client - ${hbase.version} - ${hbase.deps.scope} - - - - org.apache.hbase - hbase-annotations - - - io.netty - netty - - - - - org.apache.hbase - hbase-server - ${hbase.version} - ${hbase.deps.scope} - - - - org.apache.hbase - hbase-annotations - - - org.apache.hadoop - hadoop-core - - - org.apache.hadoop - hadoop-client - - - org.apache.hadoop - hadoop-mapreduce-client-jobclient - - - org.apache.hadoop - hadoop-mapreduce-client-core - - - org.apache.hadoop - hadoop-auth - - - org.apache.hadoop - hadoop-annotations - - - org.apache.hadoop - hadoop-hdfs - - - org.apache.hbase - hbase-hadoop1-compat - - - org.apache.commons - commons-math - - - com.sun.jersey - jersey-core - - - org.slf4j - slf4j-api - - - com.sun.jersey - jersey-server - - - com.sun.jersey - jersey-core - - - com.sun.jersey - jersey-json - - - - commons-io - commons-io - - - - - org.apache.hbase - hbase-hadoop-compat - ${hbase.version} - ${hbase.deps.scope} - - - org.apache.hbase - hbase-hadoop-compat - ${hbase.version} - test-jar - test + provided org.apache.commons commons-math3 provided - - com.twitter - algebird-core_${scala.binary.version} - 0.9.0 - org.scalacheck scalacheck_${scala.binary.version} test - org.apache.cassandra - cassandra-all - 1.2.6 - - - com.google.guava - guava - - - com.googlecode.concurrentlinkedhashmap - concurrentlinkedhashmap-lru - - - com.ning - compress-lzf - - - commons-cli - commons-cli - - - commons-codec - commons-codec - - - commons-lang - commons-lang - - - commons-logging - commons-logging - - - io.netty - netty - - - jline - jline - - - net.jpountz.lz4 - lz4 - - - org.apache.cassandra.deps - avro - - - org.apache.commons - commons-math3 - - - org.apache.thrift - libthrift - - + org.scala-lang + scala-library + provided com.github.scopt scopt_${scala.binary.version} - 3.2.0 + 3.7.0 - - - org.scala-lang - scala-library + com.twitter + parquet-hadoop-bundle provided - @@ -354,34 +143,9 @@ org.apache.maven.plugins - maven-shade-plugin + maven-jar-plugin - false - ${project.build.directory}/scala-${scala.binary.version}/spark-examples-${project.version}-hadoop${hadoop.version}.jar - - - *:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - reference.conf - - - log4j.properties - - + ${jars.target.dir} @@ -394,40 +158,9 @@ org.apache.spark spark-streaming-kinesis-asl_${scala.binary.version} ${project.version} + provided - - - - flume-provided - - provided - - - - hadoop-provided - - provided - - - - hbase-provided - - provided - - - - hive-provided - - provided - - - - parquet-provided - - provided - - diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index 31a79ddd3fff..362bd4435ecb 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -17,11 +17,10 @@ package org.apache.spark.examples; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.Function2; +import org.apache.spark.sql.SparkSession; import java.io.Serializable; import java.util.Arrays; @@ -32,8 +31,7 @@ * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ public final class JavaHdfsLR { @@ -43,8 +41,7 @@ public final class JavaHdfsLR { static void showWarning() { String warning = "WARN: This is a naive implementation of Logistic Regression " + "and is given as an example!\n" + - "Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD " + - "or org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS " + + "Please use org.apache.spark.ml.classification.LogisticRegression " + "for more conventional use."; System.err.println(warning); } @@ -124,9 +121,12 @@ public static void main(String[] args) { showWarning(); - SparkConf sparkConf = new SparkConf().setAppName("JavaHdfsLR"); - JavaSparkContext sc = new JavaSparkContext(sparkConf); - JavaRDD lines = sc.textFile(args[0]); + SparkSession spark = SparkSession + .builder() + .appName("JavaHdfsLR") + .getOrCreate(); + + JavaRDD lines = spark.read().textFile(args[0]).javaRDD(); JavaRDD points = lines.map(new ParsePoint()).cache(); int ITERATIONS = Integer.parseInt(args[1]); @@ -154,6 +154,6 @@ public static void main(String[] args) { System.out.print("Final w: "); printWeights(w); - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index 812e9d5580cb..cf12de390f60 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -17,30 +17,28 @@ package org.apache.spark.examples; -import com.google.common.collect.Lists; import scala.Tuple2; import scala.Tuple3; -import org.apache.spark.SparkConf; + 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.Function2; -import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.SparkSession; import java.io.Serializable; -import java.util.Collections; +import java.util.Arrays; import java.util.List; import java.util.regex.Matcher; import java.util.regex.Pattern; /** * Executes a roll up-style query against Apache logs. - * + * * Usage: JavaLogQuery [logFile] */ public final class JavaLogQuery { - public static final List exampleApacheLogs = Lists.newArrayList( + public static final List exampleApacheLogs = Arrays.asList( "10.10.10.10 - \"FRED\" [18/Jan/2013:17:56:07 +1100] \"GET http://images.com/2013/Generic.jpg " + "HTTP/1.1\" 304 315 \"http://referall.com/\" \"Mozilla/4.0 (compatible; MSIE 7.0; " + "Windows NT 5.1; GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; " + @@ -83,10 +81,10 @@ public static Tuple3 extractKey(String line) { String user = m.group(3); String query = m.group(5); if (!user.equalsIgnoreCase("-")) { - return new Tuple3(ip, user, query); + return new Tuple3<>(ip, user, query); } } - return new Tuple3(null, null, null); + return new Tuple3<>(null, null, null); } public static Stats extractStats(String line) { @@ -100,30 +98,24 @@ public static Stats extractStats(String line) { } public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaLogQuery") + .getOrCreate(); - SparkConf sparkConf = new SparkConf().setAppName("JavaLogQuery"); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); + JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); JavaRDD dataSet = (args.length == 1) ? jsc.textFile(args[0]) : jsc.parallelize(exampleApacheLogs); - JavaPairRDD, Stats> extracted = dataSet.mapToPair(new PairFunction, Stats>() { - @Override - public Tuple2, Stats> call(String s) { - return new Tuple2, Stats>(extractKey(s), extractStats(s)); - } - }); + JavaPairRDD, Stats> extracted = + dataSet.mapToPair(s -> new Tuple2<>(extractKey(s), extractStats(s))); - JavaPairRDD, Stats> counts = extracted.reduceByKey(new Function2() { - @Override - public Stats call(Stats stats, Stats stats2) { - return stats.merge(stats2); - } - }); + JavaPairRDD, Stats> counts = extracted.reduceByKey(Stats::merge); List, Stats>> output = counts.collect(); for (Tuple2 t : output) { System.out.println(t._1() + "\t" + t._2()); } - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index a5db8accdf13..b5b4703932f0 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -17,25 +17,18 @@ package org.apache.spark.examples; - +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Pattern; import scala.Tuple2; import com.google.common.collect.Iterables; -import org.apache.spark.SparkConf; 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.Function2; -import org.apache.spark.api.java.function.PairFlatMapFunction; -import org.apache.spark.api.java.function.PairFunction; - -import java.util.ArrayList; -import java.util.List; -import java.util.Iterator; -import java.util.regex.Pattern; +import org.apache.spark.sql.SparkSession; /** * Computes the PageRank of URLs from an input file. Input file should @@ -48,6 +41,11 @@ * * This is an example implementation for learning how to use Spark. For more conventional use, * please refer to org.apache.spark.graphx.lib.PageRank + * + * Example Usage: + *
    + * bin/run-example JavaPageRank data/mllib/pagerank_data.txt 10
    + * 
    */ public final class JavaPageRank { private static final Pattern SPACES = Pattern.compile("\\s+"); @@ -75,64 +73,50 @@ public static void main(String[] args) throws Exception { showWarning(); - SparkConf sparkConf = new SparkConf().setAppName("JavaPageRank"); - JavaSparkContext ctx = new JavaSparkContext(sparkConf); + SparkSession spark = SparkSession + .builder() + .appName("JavaPageRank") + .getOrCreate(); // Loads in input file. It should be in format of: // URL neighbor URL // URL neighbor URL // URL neighbor URL // ... - JavaRDD lines = ctx.textFile(args[0], 1); + JavaRDD lines = spark.read().textFile(args[0]).javaRDD(); // Loads all URLs from input file and initialize their neighbors. - JavaPairRDD> links = lines.mapToPair(new PairFunction() { - @Override - public Tuple2 call(String s) { - String[] parts = SPACES.split(s); - return new Tuple2(parts[0], parts[1]); - } + JavaPairRDD> links = lines.mapToPair(s -> { + String[] parts = SPACES.split(s); + return new Tuple2<>(parts[0], parts[1]); }).distinct().groupByKey().cache(); // Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. - JavaPairRDD ranks = links.mapValues(new Function, Double>() { - @Override - public Double call(Iterable rs) { - return 1.0; - } - }); + JavaPairRDD ranks = links.mapValues(rs -> 1.0); // Calculates and updates URL ranks continuously using PageRank algorithm. for (int current = 0; current < Integer.parseInt(args[1]); current++) { // Calculates URL contributions to the rank of other URLs. JavaPairRDD contribs = links.join(ranks).values() - .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { - @Override - public Iterable> call(Tuple2, Double> s) { - int urlCount = Iterables.size(s._1); - List> results = new ArrayList>(); - for (String n : s._1) { - results.add(new Tuple2(n, s._2() / urlCount)); - } - return results; + .flatMapToPair(s -> { + int urlCount = Iterables.size(s._1()); + List> results = new ArrayList<>(); + for (String n : s._1) { + results.add(new Tuple2<>(n, s._2() / urlCount)); } - }); + return results.iterator(); + }); // Re-calculates URL ranks based on neighbor contributions. - ranks = contribs.reduceByKey(new Sum()).mapValues(new Function() { - @Override - public Double call(Double sum) { - return 0.15 + sum * 0.85; - } - }); + ranks = contribs.reduceByKey(new Sum()).mapValues(sum -> 0.15 + sum * 0.85); } // Collects all URL ranks and dump them to console. List> output = ranks.collect(); for (Tuple2 tuple : output) { - System.out.println(tuple._1() + " has rank: " + tuple._2() + "."); + System.out.println(tuple._1() + " has rank: " + tuple._2() + "."); } - ctx.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index 0f07cb409832..37bd8fffbe45 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -17,50 +17,44 @@ package org.apache.spark.examples; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.Function2; +import org.apache.spark.sql.SparkSession; import java.util.ArrayList; import java.util.List; -/** +/** * Computes an approximation to pi - * Usage: JavaSparkPi [slices] + * Usage: JavaSparkPi [partitions] */ public final class JavaSparkPi { public static void main(String[] args) throws Exception { - SparkConf sparkConf = new SparkConf().setAppName("JavaSparkPi"); - JavaSparkContext jsc = new JavaSparkContext(sparkConf); + SparkSession spark = SparkSession + .builder() + .appName("JavaSparkPi") + .getOrCreate(); + + JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); int slices = (args.length == 1) ? Integer.parseInt(args[0]) : 2; int n = 100000 * slices; - List l = new ArrayList(n); + List l = new ArrayList<>(n); for (int i = 0; i < n; i++) { l.add(i); } JavaRDD dataSet = jsc.parallelize(l, slices); - int count = dataSet.map(new Function() { - @Override - public Integer call(Integer integer) { - double x = Math.random() * 2 - 1; - double y = Math.random() * 2 - 1; - return (x * x + y * y < 1) ? 1 : 0; - } - }).reduce(new Function2() { - @Override - public Integer call(Integer integer, Integer integer2) { - return integer + integer2; - } - }); + int count = dataSet.map(integer -> { + double x = Math.random() * 2 - 1; + double y = Math.random() * 2 - 1; + return (x * x + y * y <= 1) ? 1 : 0; + }).reduce((integer, integer2) -> integer + integer2); System.out.println("Pi is roughly " + 4.0 * count / n); - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java b/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java index e68ec74c3ed5..b0ebedfed6a8 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaStatusTrackerDemo.java @@ -17,13 +17,13 @@ package org.apache.spark.examples; -import org.apache.spark.SparkConf; import org.apache.spark.SparkJobInfo; import org.apache.spark.SparkStageInfo; import org.apache.spark.api.java.JavaFutureAction; 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.sql.SparkSession; import java.util.Arrays; import java.util.List; @@ -44,12 +44,16 @@ public T call(T x) throws Exception { } public static void main(String[] args) throws Exception { - SparkConf sparkConf = new SparkConf().setAppName(APP_NAME); - final JavaSparkContext sc = new JavaSparkContext(sparkConf); + SparkSession spark = SparkSession + .builder() + .appName(APP_NAME) + .getOrCreate(); + + JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); // Example of implementing a progress reporter for a simple job. - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 5).map( - new IdentityWithDelay()); + JavaRDD rdd = jsc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 5).map( + new IdentityWithDelay<>()); JavaFutureAction> jobFuture = rdd.collectAsync(); while (!jobFuture.isDone()) { Thread.sleep(1000); // 1 second @@ -58,13 +62,13 @@ public static void main(String[] args) throws Exception { continue; } int currentJobId = jobIds.get(jobIds.size() - 1); - SparkJobInfo jobInfo = sc.statusTracker().getJobInfo(currentJobId); - SparkStageInfo stageInfo = sc.statusTracker().getStageInfo(jobInfo.stageIds()[0]); + SparkJobInfo jobInfo = jsc.statusTracker().getJobInfo(currentJobId); + SparkStageInfo stageInfo = jsc.statusTracker().getStageInfo(jobInfo.stageIds()[0]); System.out.println(stageInfo.numTasks() + " tasks total: " + stageInfo.numActiveTasks() + " active, " + stageInfo.numCompletedTasks() + " complete"); } System.out.println("Job results are: " + jobFuture.get()); - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 2563fcdd234b..c9ca9c9b3a41 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -25,14 +25,14 @@ import scala.Tuple2; -import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.SparkSession; /** * Transitive closure on a graph, implemented in Java. - * Usage: JavaTC [slices] + * Usage: JavaTC [partitions] */ public final class JavaTC { @@ -41,16 +41,16 @@ public final class JavaTC { private static final Random rand = new Random(42); static List> generateGraph() { - Set> edges = new HashSet>(numEdges); + Set> edges = new HashSet<>(numEdges); while (edges.size() < numEdges) { int from = rand.nextInt(numVertices); int to = rand.nextInt(numVertices); - Tuple2 e = new Tuple2(from, to); + Tuple2 e = new Tuple2<>(from, to); if (from != to) { edges.add(e); } } - return new ArrayList>(edges); + return new ArrayList<>(edges); } static class ProjectFn implements PairFunction>, @@ -59,15 +59,20 @@ static class ProjectFn implements PairFunction call(Tuple2> triple) { - return new Tuple2(triple._2()._2(), triple._2()._1()); + return new Tuple2<>(triple._2()._2(), triple._2()._1()); } } public static void main(String[] args) { - SparkConf sparkConf = new SparkConf().setAppName("JavaHdfsLR"); - JavaSparkContext sc = new JavaSparkContext(sparkConf); + SparkSession spark = SparkSession + .builder() + .appName("JavaTC") + .getOrCreate(); + + JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); + Integer slices = (args.length > 0) ? Integer.parseInt(args[0]): 2; - JavaPairRDD tc = sc.parallelizePairs(generateGraph(), slices).cache(); + JavaPairRDD tc = jsc.parallelizePairs(generateGraph(), slices).cache(); // Linear transitive closure: each round grows paths by one edge, // by joining the graph's edges with the already-discovered paths. @@ -75,13 +80,7 @@ public static void main(String[] args) { // the graph to obtain the path (x, z). // Because join() joins on keys, the edges are stored in reversed order. - JavaPairRDD edges = tc.mapToPair( - new PairFunction, Integer, Integer>() { - @Override - public Tuple2 call(Tuple2 e) { - return new Tuple2(e._2(), e._1()); - } - }); + JavaPairRDD edges = tc.mapToPair(e -> new Tuple2<>(e._2(), e._1())); long oldCount; long nextCount = tc.count(); @@ -94,6 +93,6 @@ public Tuple2 call(Tuple2 e) { } while (nextCount != oldCount); System.out.println("TC has " + tc.count() + " edges."); - sc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index 9a6a944f7ede..f1ce1e958580 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -18,13 +18,10 @@ package org.apache.spark.examples; import scala.Tuple2; -import org.apache.spark.SparkConf; + 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.FlatMapFunction; -import org.apache.spark.api.java.function.Function2; -import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.SparkSession; import java.util.Arrays; import java.util.List; @@ -40,35 +37,23 @@ public static void main(String[] args) throws Exception { System.exit(1); } - SparkConf sparkConf = new SparkConf().setAppName("JavaWordCount"); - JavaSparkContext ctx = new JavaSparkContext(sparkConf); - JavaRDD lines = ctx.textFile(args[0], 1); + SparkSession spark = SparkSession + .builder() + .appName("JavaWordCount") + .getOrCreate(); + + JavaRDD lines = spark.read().textFile(args[0]).javaRDD(); - JavaRDD words = lines.flatMap(new FlatMapFunction() { - @Override - public Iterable call(String s) { - return Arrays.asList(SPACE.split(s)); - } - }); + JavaRDD words = lines.flatMap(s -> Arrays.asList(SPACE.split(s)).iterator()); - JavaPairRDD ones = words.mapToPair(new PairFunction() { - @Override - public Tuple2 call(String s) { - return new Tuple2(s, 1); - } - }); + JavaPairRDD ones = words.mapToPair(s -> new Tuple2<>(s, 1)); - JavaPairRDD counts = ones.reduceByKey(new Function2() { - @Override - public Integer call(Integer i1, Integer i2) { - return i1 + i2; - } - }); + JavaPairRDD counts = ones.reduceByKey((i1, i2) -> i1 + i2); List> output = counts.collect(); for (Tuple2 tuple : output) { System.out.println(tuple._1() + ": " + tuple._2()); } - ctx.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java new file mode 100644 index 000000000000..7c741ff56eaf --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaAFTSurvivalRegressionExample.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.regression.AFTSurvivalRegression; +import org.apache.spark.ml.regression.AFTSurvivalRegressionModel; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +/** + * An example demonstrating AFTSurvivalRegression. + * Run with + *
    + * bin/run-example ml.JavaAFTSurvivalRegressionExample
    + * 
    + */ +public class JavaAFTSurvivalRegressionExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaAFTSurvivalRegressionExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(1.218, 1.0, Vectors.dense(1.560, -0.605)), + RowFactory.create(2.949, 0.0, Vectors.dense(0.346, 2.158)), + RowFactory.create(3.627, 0.0, Vectors.dense(1.380, 0.231)), + RowFactory.create(0.273, 1.0, Vectors.dense(0.520, 1.151)), + RowFactory.create(4.199, 0.0, Vectors.dense(0.795, -0.226)) + ); + StructType schema = new StructType(new StructField[]{ + new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("censor", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("features", new VectorUDT(), false, Metadata.empty()) + }); + Dataset training = spark.createDataFrame(data, schema); + double[] quantileProbabilities = new double[]{0.3, 0.6}; + AFTSurvivalRegression aft = new AFTSurvivalRegression() + .setQuantileProbabilities(quantileProbabilities) + .setQuantilesCol("quantiles"); + + AFTSurvivalRegressionModel model = aft.fit(training); + + // Print the coefficients, intercept and scale parameter for AFT survival regression + System.out.println("Coefficients: " + model.coefficients()); + System.out.println("Intercept: " + model.intercept()); + System.out.println("Scale: " + model.scale()); + model.transform(training).show(false); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java new file mode 100644 index 000000000000..fe4d6bc83f04 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.io.Serializable; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.ml.evaluation.RegressionEvaluator; +import org.apache.spark.ml.recommendation.ALS; +import org.apache.spark.ml.recommendation.ALSModel; +// $example off$ + +public class JavaALSExample { + + // $example on$ + public static class Rating implements Serializable { + private int userId; + private int movieId; + private float rating; + private long timestamp; + + public Rating() {} + + public Rating(int userId, int movieId, float rating, long timestamp) { + this.userId = userId; + this.movieId = movieId; + this.rating = rating; + this.timestamp = timestamp; + } + + public int getUserId() { + return userId; + } + + public int getMovieId() { + return movieId; + } + + public float getRating() { + return rating; + } + + public long getTimestamp() { + return timestamp; + } + + public static Rating parseRating(String str) { + String[] fields = str.split("::"); + if (fields.length != 4) { + throw new IllegalArgumentException("Each line must contain 4 fields"); + } + int userId = Integer.parseInt(fields[0]); + int movieId = Integer.parseInt(fields[1]); + float rating = Float.parseFloat(fields[2]); + long timestamp = Long.parseLong(fields[3]); + return new Rating(userId, movieId, rating, timestamp); + } + } + // $example off$ + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaALSExample") + .getOrCreate(); + + // $example on$ + JavaRDD ratingsRDD = spark + .read().textFile("data/mllib/als/sample_movielens_ratings.txt").javaRDD() + .map(Rating::parseRating); + Dataset ratings = spark.createDataFrame(ratingsRDD, Rating.class); + Dataset[] splits = ratings.randomSplit(new double[]{0.8, 0.2}); + Dataset training = splits[0]; + Dataset test = splits[1]; + + // Build the recommendation model using ALS on the training data + ALS als = new ALS() + .setMaxIter(5) + .setRegParam(0.01) + .setUserCol("userId") + .setItemCol("movieId") + .setRatingCol("rating"); + ALSModel model = als.fit(training); + + // Evaluate the model by computing the RMSE on the test data + // Note we set cold start strategy to 'drop' to ensure we don't get NaN evaluation metrics + model.setColdStartStrategy("drop"); + Dataset predictions = model.transform(test); + + RegressionEvaluator evaluator = new RegressionEvaluator() + .setMetricName("rmse") + .setLabelCol("rating") + .setPredictionCol("prediction"); + Double rmse = evaluator.evaluate(predictions); + System.out.println("Root-mean-square error = " + rmse); + + // Generate top 10 movie recommendations for each user + Dataset userRecs = model.recommendForAllUsers(10); + // Generate top 10 user recommendations for each movie + Dataset movieRecs = model.recommendForAllItems(10); + // $example off$ + userRecs.show(); + movieRecs.show(); + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java new file mode 100644 index 000000000000..3090d8fd1452 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.Binarizer; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaBinarizerExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaBinarizerExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(0, 0.1), + RowFactory.create(1, 0.8), + RowFactory.create(2, 0.2) + ); + StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("feature", DataTypes.DoubleType, false, Metadata.empty()) + }); + Dataset continuousDataFrame = spark.createDataFrame(data, schema); + + Binarizer binarizer = new Binarizer() + .setInputCol("feature") + .setOutputCol("binarized_feature") + .setThreshold(0.5); + + Dataset binarizedDataFrame = binarizer.transform(continuousDataFrame); + + System.out.println("Binarizer output with Threshold = " + binarizer.getThreshold()); + binarizedDataFrame.show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java new file mode 100644 index 000000000000..8c82aaaacca3 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import org.apache.spark.ml.clustering.BisectingKMeans; +import org.apache.spark.ml.clustering.BisectingKMeansModel; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + + +/** + * An example demonstrating bisecting k-means clustering. + * Run with + *
    + * bin/run-example ml.JavaBisectingKMeansExample
    + * 
    + */ +public class JavaBisectingKMeansExample { + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaBisectingKMeansExample") + .getOrCreate(); + + // $example on$ + // Loads data. + Dataset dataset = spark.read().format("libsvm").load("data/mllib/sample_kmeans_data.txt"); + + // Trains a bisecting k-means model. + BisectingKMeans bkm = new BisectingKMeans().setK(2).setSeed(1); + BisectingKMeansModel model = bkm.fit(dataset); + + // Evaluate clustering. + double cost = model.computeCost(dataset); + System.out.println("Within Set Sum of Squared Errors = " + cost); + + // Shows the result. + System.out.println("Cluster Centers: "); + Vector[] centers = model.clusterCenters(); + for (Vector center : centers) { + System.out.println(center); + } + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java new file mode 100644 index 000000000000..ff917b720c8b --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketedRandomProjectionLSHExample.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.BucketedRandomProjectionLSH; +import org.apache.spark.ml.feature.BucketedRandomProjectionLSHModel; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import static org.apache.spark.sql.functions.col; +// $example off$ + +/** + * An example demonstrating BucketedRandomProjectionLSH. + * Run with: + * bin/run-example ml.JavaBucketedRandomProjectionLSHExample + */ +public class JavaBucketedRandomProjectionLSHExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaBucketedRandomProjectionLSHExample") + .getOrCreate(); + + // $example on$ + List dataA = Arrays.asList( + RowFactory.create(0, Vectors.dense(1.0, 1.0)), + RowFactory.create(1, Vectors.dense(1.0, -1.0)), + RowFactory.create(2, Vectors.dense(-1.0, -1.0)), + RowFactory.create(3, Vectors.dense(-1.0, 1.0)) + ); + + List dataB = Arrays.asList( + RowFactory.create(4, Vectors.dense(1.0, 0.0)), + RowFactory.create(5, Vectors.dense(-1.0, 0.0)), + RowFactory.create(6, Vectors.dense(0.0, 1.0)), + RowFactory.create(7, Vectors.dense(0.0, -1.0)) + ); + + StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("features", new VectorUDT(), false, Metadata.empty()) + }); + Dataset dfA = spark.createDataFrame(dataA, schema); + Dataset dfB = spark.createDataFrame(dataB, schema); + + Vector key = Vectors.dense(1.0, 0.0); + + BucketedRandomProjectionLSH mh = new BucketedRandomProjectionLSH() + .setBucketLength(2.0) + .setNumHashTables(3) + .setInputCol("features") + .setOutputCol("hashes"); + + BucketedRandomProjectionLSHModel model = mh.fit(dfA); + + // Feature Transformation + System.out.println("The hashed dataset where hashed values are stored in the column 'hashes':"); + model.transform(dfA).show(); + + // Compute the locality sensitive hashes for the input rows, then perform approximate + // similarity join. + // We could avoid computing hashes by passing in the already-transformed dataset, e.g. + // `model.approxSimilarityJoin(transformedA, transformedB, 1.5)` + System.out.println("Approximately joining dfA and dfB on distance smaller than 1.5:"); + model.approxSimilarityJoin(dfA, dfB, 1.5, "EuclideanDistance") + .select(col("datasetA.id").alias("idA"), + col("datasetB.id").alias("idB"), + col("EuclideanDistance")).show(); + + // Compute the locality sensitive hashes for the input rows, then perform approximate nearest + // neighbor search. + // We could avoid computing hashes by passing in the already-transformed dataset, e.g. + // `model.approxNearestNeighbors(transformedA, key, 2)` + System.out.println("Approximately searching dfA for 2 nearest neighbors of the key:"); + model.approxNearestNeighbors(dfA, key, 2).show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java new file mode 100644 index 000000000000..f00993833321 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBucketizerExample.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.Bucketizer; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaBucketizerExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaBucketizerExample") + .getOrCreate(); + + // $example on$ + double[] splits = {Double.NEGATIVE_INFINITY, -0.5, 0.0, 0.5, Double.POSITIVE_INFINITY}; + + List data = Arrays.asList( + RowFactory.create(-999.9), + RowFactory.create(-0.5), + RowFactory.create(-0.3), + RowFactory.create(0.0), + RowFactory.create(0.2), + RowFactory.create(999.9) + ); + StructType schema = new StructType(new StructField[]{ + new StructField("features", DataTypes.DoubleType, false, Metadata.empty()) + }); + Dataset dataFrame = spark.createDataFrame(data, schema); + + Bucketizer bucketizer = new Bucketizer() + .setInputCol("features") + .setOutputCol("bucketedFeatures") + .setSplits(splits); + + // Transform original data into its bucket index. + Dataset bucketedData = bucketizer.transform(dataFrame); + + System.out.println("Bucketizer output with " + (bucketizer.getSplits().length-1) + " buckets"); + bucketedData.show(); + // $example off$ + + spark.stop(); + } +} + + diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java new file mode 100644 index 000000000000..73738966b118 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSqSelectorExample.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.ChiSqSelector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaChiSqSelectorExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaChiSqSelectorExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(7, Vectors.dense(0.0, 0.0, 18.0, 1.0), 1.0), + RowFactory.create(8, Vectors.dense(0.0, 1.0, 12.0, 0.0), 0.0), + RowFactory.create(9, Vectors.dense(1.0, 0.0, 15.0, 0.1), 0.0) + ); + StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("features", new VectorUDT(), false, Metadata.empty()), + new StructField("clicked", DataTypes.DoubleType, false, Metadata.empty()) + }); + + Dataset df = spark.createDataFrame(data, schema); + + ChiSqSelector selector = new ChiSqSelector() + .setNumTopFeatures(1) + .setFeaturesCol("features") + .setLabelCol("clicked") + .setOutputCol("selectedFeatures"); + + Dataset result = selector.fit(df).transform(df); + + System.out.println("ChiSqSelector output with top " + selector.getNumTopFeatures() + + " features selected"); + result.show(); + + // $example off$ + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSquareTestExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSquareTestExample.java new file mode 100644 index 000000000000..4b39350fab9b --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaChiSquareTestExample.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.stat.ChiSquareTest; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.*; +// $example off$ + +/** + * An example for Chi-square hypothesis testing. + * Run with + *
    + * bin/run-example ml.JavaChiSquareTestExample
    + * 
    + */ +public class JavaChiSquareTestExample { + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaChiSquareTestExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(0.0, Vectors.dense(0.5, 10.0)), + RowFactory.create(0.0, Vectors.dense(1.5, 20.0)), + RowFactory.create(1.0, Vectors.dense(1.5, 30.0)), + RowFactory.create(0.0, Vectors.dense(3.5, 30.0)), + RowFactory.create(0.0, Vectors.dense(3.5, 40.0)), + RowFactory.create(1.0, Vectors.dense(3.5, 40.0)) + ); + + StructType schema = new StructType(new StructField[]{ + new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("features", new VectorUDT(), false, Metadata.empty()), + }); + + Dataset df = spark.createDataFrame(data, schema); + Row r = ChiSquareTest.test(df, "features", "label").head(); + System.out.println("pValues: " + r.get(0).toString()); + System.out.println("degreesOfFreedom: " + r.getList(1).toString()); + System.out.println("statistics: " + r.get(2).toString()); + + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCorrelationExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCorrelationExample.java new file mode 100644 index 000000000000..2a6d62ab3fb7 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCorrelationExample.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.stat.Correlation; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.*; +// $example off$ + +/** + * An example for computing correlation matrix. + * Run with + *
    + * bin/run-example ml.JavaCorrelationExample
    + * 
    + */ +public class JavaCorrelationExample { + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaCorrelationExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(Vectors.sparse(4, new int[]{0, 3}, new double[]{1.0, -2.0})), + RowFactory.create(Vectors.dense(4.0, 5.0, 0.0, 3.0)), + RowFactory.create(Vectors.dense(6.0, 7.0, 0.0, 8.0)), + RowFactory.create(Vectors.sparse(4, new int[]{0, 3}, new double[]{9.0, 1.0})) + ); + + StructType schema = new StructType(new StructField[]{ + new StructField("features", new VectorUDT(), false, Metadata.empty()), + }); + + Dataset df = spark.createDataFrame(data, schema); + Row r1 = Correlation.corr(df, "features").head(); + System.out.println("Pearson correlation matrix:\n" + r1.get(0).toString()); + + Row r2 = Correlation.corr(df, "features", "spearman").head(); + System.out.println("Spearman correlation matrix:\n" + r2.get(0).toString()); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java index ac33adb65292..ac2a86c30b0b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCountVectorizerExample.java @@ -19,36 +19,34 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.CountVectorizer; import org.apache.spark.ml.feature.CountVectorizerModel; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; // $example off$ public class JavaCountVectorizerExample { public static void main(String[] args) { - - SparkConf conf = new SparkConf().setAppName("JavaCountVectorizerExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaCountVectorizerExample") + .getOrCreate(); // $example on$ // Input data: Each row is a bag of words from a sentence or document. - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Arrays.asList("a", "b", "c")), RowFactory.create(Arrays.asList("a", "b", "b", "c", "a")) - )); + ); StructType schema = new StructType(new StructField [] { new StructField("text", new ArrayType(DataTypes.StringType, true), false, Metadata.empty()) }); - DataFrame df = sqlContext.createDataFrame(jrdd, schema); + Dataset df = spark.createDataFrame(data, schema); // fit a CountVectorizerModel from the corpus CountVectorizerModel cvModel = new CountVectorizer() @@ -63,7 +61,9 @@ public static void main(String[] args) { .setInputCol("text") .setOutputCol("feature"); - cvModel.transform(df).show(); + cvModel.transform(df).show(false); // $example off$ + + spark.stop(); } } 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 deleted file mode 100644 index 9bbc14ea4087..000000000000 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java +++ /dev/null @@ -1,126 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.examples.ml; - -import java.util.List; - -import com.google.common.collect.Lists; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.Pipeline; -import org.apache.spark.ml.PipelineStage; -import org.apache.spark.ml.classification.LogisticRegression; -import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator; -import org.apache.spark.ml.feature.HashingTF; -import org.apache.spark.ml.feature.Tokenizer; -import org.apache.spark.ml.param.ParamMap; -import org.apache.spark.ml.tuning.CrossValidator; -import org.apache.spark.ml.tuning.CrossValidatorModel; -import org.apache.spark.ml.tuning.ParamGridBuilder; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; - -/** - * A simple example demonstrating model selection using CrossValidator. - * This example also demonstrates how Pipelines are Estimators. - * - * This example uses the Java bean classes {@link org.apache.spark.examples.ml.LabeledDocument} and - * {@link org.apache.spark.examples.ml.Document} defined in the Scala example - * {@link org.apache.spark.examples.ml.SimpleTextClassificationPipeline}. - * - * Run with - *
    - * bin/run-example ml.JavaCrossValidatorExample
    - * 
    - */ -public class JavaCrossValidatorExample { - - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaCrossValidatorExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); - - // Prepare training documents, which are labeled. - List localTraining = Lists.newArrayList( - new LabeledDocument(0L, "a b c d e spark", 1.0), - new LabeledDocument(1L, "b d", 0.0), - new LabeledDocument(2L, "spark f g h", 1.0), - new LabeledDocument(3L, "hadoop mapreduce", 0.0), - new LabeledDocument(4L, "b spark who", 1.0), - new LabeledDocument(5L, "g d a y", 0.0), - new LabeledDocument(6L, "spark fly", 1.0), - new LabeledDocument(7L, "was mapreduce", 0.0), - new LabeledDocument(8L, "e spark program", 1.0), - 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.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); - - // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. - Tokenizer tokenizer = new Tokenizer() - .setInputCol("text") - .setOutputCol("words"); - HashingTF hashingTF = new HashingTF() - .setNumFeatures(1000) - .setInputCol(tokenizer.getOutputCol()) - .setOutputCol("features"); - LogisticRegression lr = new LogisticRegression() - .setMaxIter(10) - .setRegParam(0.01); - Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[] {tokenizer, hashingTF, lr}); - - // We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. - // This will allow us to jointly choose parameters for all Pipeline stages. - // A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. - CrossValidator crossval = new CrossValidator() - .setEstimator(pipeline) - .setEvaluator(new BinaryClassificationEvaluator()); - // We use a ParamGridBuilder to construct a grid of parameters to search over. - // With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, - // this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. - ParamMap[] paramGrid = new ParamGridBuilder() - .addGrid(hashingTF.numFeatures(), new int[]{10, 100, 1000}) - .addGrid(lr.regParam(), new double[]{0.1, 0.01}) - .build(); - crossval.setEstimatorParamMaps(paramGrid); - crossval.setNumFolds(2); // Use 3+ in practice - - // Run cross-validation, and choose the best set of parameters. - CrossValidatorModel cvModel = crossval.fit(training); - - // Prepare test documents, which are unlabeled. - List localTest = Lists.newArrayList( - new Document(4L, "spark i j k"), - new Document(5L, "l m n"), - new Document(6L, "mapreduce spark"), - new Document(7L, "apache hadoop")); - DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); - - // Make predictions on test documents. cvModel uses the best model found (lrModel). - DataFrame predictions = cvModel.transform(test); - for (Row r: predictions.select("id", "text", "probability", "prediction").collect()) { - System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) - + ", prediction=" + r.get(3)); - } - - jsc.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java new file mode 100644 index 000000000000..04546d29fadd --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDCTExample.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.DCT; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaDCTExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaDCTExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(Vectors.dense(0.0, 1.0, -2.0, 3.0)), + RowFactory.create(Vectors.dense(-1.0, 2.0, 4.0, -7.0)), + RowFactory.create(Vectors.dense(14.0, -2.0, -5.0, 1.0)) + ); + StructType schema = new StructType(new StructField[]{ + new StructField("features", new VectorUDT(), false, Metadata.empty()), + }); + Dataset df = spark.createDataFrame(data, schema); + + DCT dct = new DCT() + .setInputCol("features") + .setOutputCol("featuresDCT") + .setInverse(false); + + Dataset dctDf = dct.transform(df); + + dctDf.select("featuresDCT").show(false); + // $example off$ + + spark.stop(); + } +} + diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java new file mode 100644 index 000000000000..a9c6e7f0bf6c --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +// scalastyle:off println +package org.apache.spark.examples.ml; +// $example on$ +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.DecisionTreeClassifier; +import org.apache.spark.ml.classification.DecisionTreeClassificationModel; +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; +import org.apache.spark.ml.feature.*; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off$ + +public class JavaDecisionTreeClassificationExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaDecisionTreeClassificationExample") + .getOrCreate(); + + // $example on$ + // Load the data stored in LIBSVM format as a DataFrame. + Dataset data = spark + .read() + .format("libsvm") + .load("data/mllib/sample_libsvm_data.txt"); + + // Index labels, adding metadata to the label column. + // Fit on whole dataset to include all labels in index. + StringIndexerModel labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data); + + // Automatically identify categorical features, and index them. + VectorIndexerModel featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) // features with > 4 distinct values are treated as continuous. + .fit(data); + + // Split the data into training and test sets (30% held out for testing). + Dataset[] splits = data.randomSplit(new double[]{0.7, 0.3}); + Dataset trainingData = splits[0]; + Dataset testData = splits[1]; + + // Train a DecisionTree model. + DecisionTreeClassifier dt = new DecisionTreeClassifier() + .setLabelCol("indexedLabel") + .setFeaturesCol("indexedFeatures"); + + // Convert indexed labels back to original labels. + IndexToString labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labels()); + + // Chain indexers and tree in a Pipeline. + Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[]{labelIndexer, featureIndexer, dt, labelConverter}); + + // Train model. This also runs the indexers. + PipelineModel model = pipeline.fit(trainingData); + + // Make predictions. + Dataset predictions = model.transform(testData); + + // Select example rows to display. + predictions.select("predictedLabel", "label", "features").show(5); + + // Select (prediction, true label) and compute test error. + MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("accuracy"); + double accuracy = evaluator.evaluate(predictions); + System.out.println("Test Error = " + (1.0 - accuracy)); + + DecisionTreeClassificationModel treeModel = + (DecisionTreeClassificationModel) (model.stages()[2]); + System.out.println("Learned classification tree model:\n" + treeModel.toDebugString()); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java new file mode 100644 index 000000000000..cffb7139edcc --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeRegressionExample.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +// scalastyle:off println +package org.apache.spark.examples.ml; +// $example on$ +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.evaluation.RegressionEvaluator; +import org.apache.spark.ml.feature.VectorIndexer; +import org.apache.spark.ml.feature.VectorIndexerModel; +import org.apache.spark.ml.regression.DecisionTreeRegressionModel; +import org.apache.spark.ml.regression.DecisionTreeRegressor; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off$ + +public class JavaDecisionTreeRegressionExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaDecisionTreeRegressionExample") + .getOrCreate(); + // $example on$ + // Load the data stored in LIBSVM format as a DataFrame. + Dataset data = spark.read().format("libsvm") + .load("data/mllib/sample_libsvm_data.txt"); + + // Automatically identify categorical features, and index them. + // Set maxCategories so features with > 4 distinct values are treated as continuous. + VectorIndexerModel featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data); + + // Split the data into training and test sets (30% held out for testing). + Dataset[] splits = data.randomSplit(new double[]{0.7, 0.3}); + Dataset trainingData = splits[0]; + Dataset testData = splits[1]; + + // Train a DecisionTree model. + DecisionTreeRegressor dt = new DecisionTreeRegressor() + .setFeaturesCol("indexedFeatures"); + + // Chain indexer and tree in a Pipeline. + Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[]{featureIndexer, dt}); + + // Train model. This also runs the indexer. + PipelineModel model = pipeline.fit(trainingData); + + // Make predictions. + Dataset predictions = model.transform(testData); + + // Select example rows to display. + predictions.select("label", "features").show(5); + + // Select (prediction, true label) and compute test error. + RegressionEvaluator evaluator = new RegressionEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("rmse"); + double rmse = evaluator.evaluate(predictions); + System.out.println("Root Mean Squared Error (RMSE) on test data = " + rmse); + + DecisionTreeRegressionModel treeModel = + (DecisionTreeRegressionModel) (model.stages()[1]); + System.out.println("Learned regression tree model:\n" + treeModel.toDebugString()); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java deleted file mode 100644 index 0b4c0d9ba9f8..000000000000 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java +++ /dev/null @@ -1,241 +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.examples.ml; - -import java.util.List; - -import com.google.common.collect.Lists; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.classification.Classifier; -import org.apache.spark.ml.classification.ClassificationModel; -import org.apache.spark.ml.param.IntParam; -import org.apache.spark.ml.param.ParamMap; -import org.apache.spark.ml.util.Identifiable$; -import org.apache.spark.mllib.linalg.BLAS; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; - - -/** - * A simple example demonstrating how to write your own learning algorithm using Estimator, - * Transformer, and other abstractions. - * This mimics {@link org.apache.spark.ml.classification.LogisticRegression}. - * - * Run with - *
    - * bin/run-example ml.JavaDeveloperApiExample
    - * 
    - */ -public class JavaDeveloperApiExample { - - public static void main(String[] args) throws Exception { - SparkConf conf = new SparkConf().setAppName("JavaDeveloperApiExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); - - // Prepare training data. - List localTraining = Lists.newArrayList( - new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), - new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), - new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), - new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); - DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledPoint.class); - - // Create a LogisticRegression instance. This instance is an Estimator. - MyJavaLogisticRegression lr = new MyJavaLogisticRegression(); - // Print out the parameters, documentation, and any default values. - System.out.println("MyJavaLogisticRegression parameters:\n" + lr.explainParams() + "\n"); - - // We may set parameters using setter methods. - lr.setMaxIter(10); - - // Learn a LogisticRegression model. This uses the parameters stored in lr. - MyJavaLogisticRegressionModel model = lr.fit(training); - - // Prepare test data. - List localTest = Lists.newArrayList( - new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), - new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), - new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); - DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class); - - // Make predictions on test documents. cvModel uses the best model found (lrModel). - DataFrame results = model.transform(test); - double sumPredictions = 0; - for (Row r : results.select("features", "label", "prediction").collect()) { - sumPredictions += r.getDouble(2); - } - if (sumPredictions != 0.0) { - throw new Exception("MyJavaLogisticRegression predicted something other than 0," + - " even though all weights are 0!"); - } - - jsc.stop(); - } -} - -/** - * Example of defining a type of {@link Classifier}. - * - * Note: Some IDEs (e.g., IntelliJ) will complain that this will not compile due to - * {@link org.apache.spark.ml.param.Params#set} using incompatible return types. - * However, this should still compile and run successfully. - */ -class MyJavaLogisticRegression - extends Classifier { - - public MyJavaLogisticRegression() { - init(); - } - - public MyJavaLogisticRegression(String uid) { - this.uid_ = uid; - init(); - } - - private String uid_ = Identifiable$.MODULE$.randomUID("myJavaLogReg"); - - @Override - public String uid() { - return uid_; - } - - /** - * Param for max number of iterations - *

    - * NOTE: The usual way to add a parameter to a model or algorithm is to include: - * - val myParamName: ParamType - * - def getMyParamName - * - def setMyParamName - */ - IntParam maxIter = new IntParam(this, "maxIter", "max number of iterations"); - - int getMaxIter() { return (Integer) getOrDefault(maxIter); } - - private void init() { - setMaxIter(100); - } - - // The parameter setter is in this class since it should return type MyJavaLogisticRegression. - MyJavaLogisticRegression setMaxIter(int value) { - return (MyJavaLogisticRegression) set(maxIter, value); - } - - // This method is used by fit(). - // In Java, we have to make it public since Java does not understand Scala's protected modifier. - public MyJavaLogisticRegressionModel train(DataFrame dataset) { - // Extract columns from data using helper method. - JavaRDD oldDataset = extractLabeledPoints(dataset).toJavaRDD(); - - // Do learning to estimate the weight vector. - int numFeatures = oldDataset.take(1).get(0).features().size(); - Vector weights = Vectors.zeros(numFeatures); // Learning would happen here. - - // Create a model, and return it. - return new MyJavaLogisticRegressionModel(uid(), weights).setParent(this); - } - - @Override - public MyJavaLogisticRegression copy(ParamMap extra) { - return defaultCopy(extra); - } -} - -/** - * Example of defining a type of {@link ClassificationModel}. - * - * Note: Some IDEs (e.g., IntelliJ) will complain that this will not compile due to - * {@link org.apache.spark.ml.param.Params#set} using incompatible return types. - * However, this should still compile and run successfully. - */ -class MyJavaLogisticRegressionModel - extends ClassificationModel { - - private Vector weights_; - public Vector weights() { return weights_; } - - public MyJavaLogisticRegressionModel(String uid, Vector weights) { - this.uid_ = uid; - this.weights_ = weights; - } - - private String uid_ = Identifiable$.MODULE$.randomUID("myJavaLogReg"); - - @Override - public String uid() { - return uid_; - } - - // This uses the default implementation of transform(), which reads column "features" and outputs - // columns "prediction" and "rawPrediction." - - // This uses the default implementation of predict(), which chooses the label corresponding to - // the maximum value returned by [[predictRaw()]]. - - /** - * Raw prediction for each possible label. - * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives - * a measure of confidence in each possible label (where larger = more confident). - * This internal method is used to implement [[transform()]] and output [[rawPredictionCol]]. - * - * @return vector where element i is the raw prediction for label i. - * This raw prediction may be any real number, where a larger value indicates greater - * confidence for that label. - * - * In Java, we have to make this method public since Java does not understand Scala's protected - * modifier. - */ - public Vector predictRaw(Vector features) { - double margin = BLAS.dot(features, weights_); - // There are 2 classes (binary classification), so we return a length-2 vector, - // where index i corresponds to class i (i = 0, 1). - return Vectors.dense(-margin, margin); - } - - /** - * Number of classes the label can take. 2 indicates binary classification. - */ - public int numClasses() { return 2; } - - /** - * Number of features the model was trained on. - */ - public int numFeatures() { return weights_.size(); } - - /** - * Create a copy of the model. - * The copy is shallow, except for the embedded paramMap, which gets a deep copy. - *

    - * This is used for the defaul implementation of [[transform()]]. - * - * In Java, we have to make this method public since Java does not understand Scala's protected - * modifier. - */ - @Override - public MyJavaLogisticRegressionModel copy(ParamMap extra) { - return copyValues(new MyJavaLogisticRegressionModel(uid(), weights_), extra) - .setParent(parent()); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDocument.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDocument.java new file mode 100644 index 000000000000..6459dabc0698 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDocument.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import java.io.Serializable; + +/** + * Unlabeled instance type, Spark SQL can infer schema from Java Beans. + */ +@SuppressWarnings("serial") +public class JavaDocument implements Serializable { + + private long id; + private String text; + + public JavaDocument(long id, String text) { + this.id = id; + this.text = text; + } + + public long getId() { + return this.id; + } + + public String getText() { + return this.text; + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java new file mode 100644 index 000000000000..d2e70c23babc --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaElementwiseProductExample.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.ElementwiseProduct; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaElementwiseProductExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaElementwiseProductExample") + .getOrCreate(); + + // $example on$ + // Create some vector data; also works for sparse vectors + List data = Arrays.asList( + RowFactory.create("a", Vectors.dense(1.0, 2.0, 3.0)), + RowFactory.create("b", Vectors.dense(4.0, 5.0, 6.0)) + ); + + List fields = new ArrayList<>(2); + fields.add(DataTypes.createStructField("id", DataTypes.StringType, false)); + fields.add(DataTypes.createStructField("vector", new VectorUDT(), false)); + + StructType schema = DataTypes.createStructType(fields); + + Dataset dataFrame = spark.createDataFrame(data, schema); + + Vector transformingVector = Vectors.dense(0.0, 1.0, 2.0); + + ElementwiseProduct transformer = new ElementwiseProduct() + .setScalingVec(transformingVector) + .setInputCol("vector") + .setOutputCol("transformedVector"); + + // Batch transform the vectors to create new column: + transformer.transform(dataFrame).show(); + // $example off$ + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java new file mode 100644 index 000000000000..9e07a0c2f899 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.ml.classification.LogisticRegressionModel; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ +import org.apache.spark.sql.SparkSession; + +/** + * Java example for Estimator, Transformer, and Param. + */ +public class JavaEstimatorTransformerParamExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaEstimatorTransformerParamExample") + .getOrCreate(); + + // $example on$ + // Prepare training data. + List dataTraining = Arrays.asList( + RowFactory.create(1.0, Vectors.dense(0.0, 1.1, 0.1)), + RowFactory.create(0.0, Vectors.dense(2.0, 1.0, -1.0)), + RowFactory.create(0.0, Vectors.dense(2.0, 1.3, 1.0)), + RowFactory.create(1.0, Vectors.dense(0.0, 1.2, -0.5)) + ); + StructType schema = new StructType(new StructField[]{ + new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("features", new VectorUDT(), false, Metadata.empty()) + }); + Dataset training = spark.createDataFrame(dataTraining, schema); + + // Create a LogisticRegression instance. This instance is an Estimator. + LogisticRegression lr = new LogisticRegression(); + // Print out the parameters, documentation, and any default values. + System.out.println("LogisticRegression parameters:\n" + lr.explainParams() + "\n"); + + // We may set parameters using setter methods. + lr.setMaxIter(10).setRegParam(0.01); + + // Learn a LogisticRegression model. This uses the parameters stored in lr. + LogisticRegressionModel model1 = lr.fit(training); + // 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 + // LogisticRegression instance. + System.out.println("Model 1 was fit using parameters: " + model1.parent().extractParamMap()); + + // We may alternatively specify parameters using a ParamMap. + ParamMap paramMap = new ParamMap() + .put(lr.maxIter().w(20)) // Specify 1 Param. + .put(lr.maxIter(), 30) // This overwrites the original maxIter. + .put(lr.regParam().w(0.1), lr.threshold().w(0.55)); // Specify multiple Params. + + // One can also combine ParamMaps. + ParamMap paramMap2 = new ParamMap() + .put(lr.probabilityCol().w("myProbability")); // Change output column name + ParamMap paramMapCombined = paramMap.$plus$plus(paramMap2); + + // Now learn a new model using the paramMapCombined parameters. + // paramMapCombined overrides all parameters set earlier via lr.set* methods. + LogisticRegressionModel model2 = lr.fit(training, paramMapCombined); + System.out.println("Model 2 was fit using parameters: " + model2.parent().extractParamMap()); + + // Prepare test documents. + List dataTest = Arrays.asList( + RowFactory.create(1.0, Vectors.dense(-1.0, 1.5, 1.3)), + RowFactory.create(0.0, Vectors.dense(3.0, 2.0, -0.1)), + RowFactory.create(1.0, Vectors.dense(0.0, 2.2, -1.5)) + ); + Dataset test = spark.createDataFrame(dataTest, schema); + + // Make predictions on test documents using the Transformer.transform() method. + // 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. + Dataset results = model2.transform(test); + Dataset rows = results.select("features", "label", "myProbability", "prediction"); + for (Row r: rows.collectAsList()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) + + ", prediction=" + r.get(3)); + } + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaFPGrowthExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaFPGrowthExample.java new file mode 100644 index 000000000000..717ec21c8b20 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaFPGrowthExample.java @@ -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.examples.ml; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.fpm.FPGrowth; +import org.apache.spark.ml.fpm.FPGrowthModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.*; +// $example off$ + +/** + * An example demonstrating FPGrowth. + * Run with + *

    + * bin/run-example ml.JavaFPGrowthExample
    + * 
    + */ +public class JavaFPGrowthExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaFPGrowthExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(Arrays.asList("1 2 5".split(" "))), + RowFactory.create(Arrays.asList("1 2 3 5".split(" "))), + RowFactory.create(Arrays.asList("1 2".split(" "))) + ); + StructType schema = new StructType(new StructField[]{ new StructField( + "items", new ArrayType(DataTypes.StringType, true), false, Metadata.empty()) + }); + Dataset itemsDF = spark.createDataFrame(data, schema); + + FPGrowthModel model = new FPGrowth() + .setItemsCol("items") + .setMinSupport(0.5) + .setMinConfidence(0.6) + .fit(itemsDF); + + // Display frequent itemsets. + model.freqItemsets().show(); + + // Display generated association rules. + model.associationRules().show(); + + // transform examines the input items against all the association rules and summarize the + // consequents as prediction + model.transform(itemsDF).show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaFeatureHasherExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaFeatureHasherExample.java new file mode 100644 index 000000000000..9730d42e6db8 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaFeatureHasherExample.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.FeatureHasher; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaFeatureHasherExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaFeatureHasherExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(2.2, true, "1", "foo"), + RowFactory.create(3.3, false, "2", "bar"), + RowFactory.create(4.4, false, "3", "baz"), + RowFactory.create(5.5, false, "4", "foo") + ); + StructType schema = new StructType(new StructField[]{ + new StructField("real", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("bool", DataTypes.BooleanType, false, Metadata.empty()), + new StructField("stringNum", DataTypes.StringType, false, Metadata.empty()), + new StructField("string", DataTypes.StringType, false, Metadata.empty()) + }); + Dataset dataset = spark.createDataFrame(data, schema); + + FeatureHasher hasher = new FeatureHasher() + .setInputCols(new String[]{"real", "bool", "stringNum", "string"}) + .setOutputCol("features"); + + Dataset featurized = hasher.transform(dataset); + + featurized.show(false); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java new file mode 100644 index 000000000000..72bd5d0395ee --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGaussianMixtureExample.java @@ -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.examples.ml; + +// $example on$ +import org.apache.spark.ml.clustering.GaussianMixture; +import org.apache.spark.ml.clustering.GaussianMixtureModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + + +/** + * An example demonstrating Gaussian Mixture Model. + * Run with + *
    + * bin/run-example ml.JavaGaussianMixtureExample
    + * 
    + */ +public class JavaGaussianMixtureExample { + + public static void main(String[] args) { + + // Creates a SparkSession + SparkSession spark = SparkSession + .builder() + .appName("JavaGaussianMixtureExample") + .getOrCreate(); + + // $example on$ + // Loads data + Dataset dataset = spark.read().format("libsvm").load("data/mllib/sample_kmeans_data.txt"); + + // Trains a GaussianMixture model + GaussianMixture gmm = new GaussianMixture() + .setK(2); + GaussianMixtureModel model = gmm.fit(dataset); + + // Output the parameters of the mixture model + for (int i = 0; i < model.getK(); i++) { + System.out.printf("Gaussian %d:\nweight=%f\nmu=%s\nsigma=\n%s\n\n", + i, model.weights()[i], model.gaussians()[i].mean(), model.gaussians()[i].cov()); + } + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java new file mode 100644 index 000000000000..3f072d1e50eb --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGeneralizedLinearRegressionExample.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import java.util.Arrays; + +import org.apache.spark.ml.regression.GeneralizedLinearRegression; +import org.apache.spark.ml.regression.GeneralizedLinearRegressionModel; +import org.apache.spark.ml.regression.GeneralizedLinearRegressionTrainingSummary; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + +/** + * An example demonstrating generalized linear regression. + * Run with + *
    + * bin/run-example ml.JavaGeneralizedLinearRegressionExample
    + * 
    + */ + +public class JavaGeneralizedLinearRegressionExample { + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaGeneralizedLinearRegressionExample") + .getOrCreate(); + + // $example on$ + // Load training data + Dataset dataset = spark.read().format("libsvm") + .load("data/mllib/sample_linear_regression_data.txt"); + + GeneralizedLinearRegression glr = new GeneralizedLinearRegression() + .setFamily("gaussian") + .setLink("identity") + .setMaxIter(10) + .setRegParam(0.3); + + // Fit the model + GeneralizedLinearRegressionModel model = glr.fit(dataset); + + // Print the coefficients and intercept for generalized linear regression model + System.out.println("Coefficients: " + model.coefficients()); + System.out.println("Intercept: " + model.intercept()); + + // Summarize the model over the training set and print out some metrics + GeneralizedLinearRegressionTrainingSummary summary = model.summary(); + System.out.println("Coefficient Standard Errors: " + + Arrays.toString(summary.coefficientStandardErrors())); + System.out.println("T Values: " + Arrays.toString(summary.tValues())); + System.out.println("P Values: " + Arrays.toString(summary.pValues())); + System.out.println("Dispersion: " + summary.dispersion()); + System.out.println("Null Deviance: " + summary.nullDeviance()); + System.out.println("Residual Degree Of Freedom Null: " + summary.residualDegreeOfFreedomNull()); + System.out.println("Deviance: " + summary.deviance()); + System.out.println("Residual Degree Of Freedom: " + summary.residualDegreeOfFreedom()); + System.out.println("AIC: " + summary.aic()); + System.out.println("Deviance Residuals: "); + summary.residuals().show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java new file mode 100644 index 000000000000..3e9eb998c8e1 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.GBTClassificationModel; +import org.apache.spark.ml.classification.GBTClassifier; +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; +import org.apache.spark.ml.feature.*; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off$ + +public class JavaGradientBoostedTreeClassifierExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaGradientBoostedTreeClassifierExample") + .getOrCreate(); + + // $example on$ + // Load and parse the data file, converting it to a DataFrame. + Dataset data = spark + .read() + .format("libsvm") + .load("data/mllib/sample_libsvm_data.txt"); + + // Index labels, adding metadata to the label column. + // Fit on whole dataset to include all labels in index. + StringIndexerModel labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data); + // Automatically identify categorical features, and index them. + // Set maxCategories so features with > 4 distinct values are treated as continuous. + VectorIndexerModel featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data); + + // Split the data into training and test sets (30% held out for testing) + Dataset[] splits = data.randomSplit(new double[] {0.7, 0.3}); + Dataset trainingData = splits[0]; + Dataset testData = splits[1]; + + // Train a GBT model. + GBTClassifier gbt = new GBTClassifier() + .setLabelCol("indexedLabel") + .setFeaturesCol("indexedFeatures") + .setMaxIter(10); + + // Convert indexed labels back to original labels. + IndexToString labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labels()); + + // Chain indexers and GBT in a Pipeline. + Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {labelIndexer, featureIndexer, gbt, labelConverter}); + + // Train model. This also runs the indexers. + PipelineModel model = pipeline.fit(trainingData); + + // Make predictions. + Dataset predictions = model.transform(testData); + + // Select example rows to display. + predictions.select("predictedLabel", "label", "features").show(5); + + // Select (prediction, true label) and compute test error. + MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("accuracy"); + double accuracy = evaluator.evaluate(predictions); + System.out.println("Test Error = " + (1.0 - accuracy)); + + GBTClassificationModel gbtModel = (GBTClassificationModel)(model.stages()[2]); + System.out.println("Learned classification GBT model:\n" + gbtModel.toDebugString()); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java new file mode 100644 index 000000000000..769b5c3e8525 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeRegressorExample.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.evaluation.RegressionEvaluator; +import org.apache.spark.ml.feature.VectorIndexer; +import org.apache.spark.ml.feature.VectorIndexerModel; +import org.apache.spark.ml.regression.GBTRegressionModel; +import org.apache.spark.ml.regression.GBTRegressor; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off$ + +public class JavaGradientBoostedTreeRegressorExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaGradientBoostedTreeRegressorExample") + .getOrCreate(); + + // $example on$ + // Load and parse the data file, converting it to a DataFrame. + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + + // Automatically identify categorical features, and index them. + // Set maxCategories so features with > 4 distinct values are treated as continuous. + VectorIndexerModel featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data); + + // Split the data into training and test sets (30% held out for testing). + Dataset[] splits = data.randomSplit(new double[] {0.7, 0.3}); + Dataset trainingData = splits[0]; + Dataset testData = splits[1]; + + // Train a GBT model. + GBTRegressor gbt = new GBTRegressor() + .setLabelCol("label") + .setFeaturesCol("indexedFeatures") + .setMaxIter(10); + + // Chain indexer and GBT in a Pipeline. + Pipeline pipeline = new Pipeline().setStages(new PipelineStage[] {featureIndexer, gbt}); + + // Train model. This also runs the indexer. + PipelineModel model = pipeline.fit(trainingData); + + // Make predictions. + Dataset predictions = model.transform(testData); + + // Select example rows to display. + predictions.select("prediction", "label", "features").show(5); + + // Select (prediction, true label) and compute test error. + RegressionEvaluator evaluator = new RegressionEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("rmse"); + double rmse = evaluator.evaluate(predictions); + System.out.println("Root Mean Squared Error (RMSE) on test data = " + rmse); + + GBTRegressionModel gbtModel = (GBTRegressionModel)(model.stages()[1]); + System.out.println("Learned regression GBT model:\n" + gbtModel.toDebugString()); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaImputerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaImputerExample.java new file mode 100644 index 000000000000..ac40ccd9dbd7 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaImputerExample.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.Imputer; +import org.apache.spark.ml.feature.ImputerModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.*; +// $example off$ + +import static org.apache.spark.sql.types.DataTypes.*; + +/** + * An example demonstrating Imputer. + * Run with: + * bin/run-example ml.JavaImputerExample + */ +public class JavaImputerExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaImputerExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(1.0, Double.NaN), + RowFactory.create(2.0, Double.NaN), + RowFactory.create(Double.NaN, 3.0), + RowFactory.create(4.0, 4.0), + RowFactory.create(5.0, 5.0) + ); + StructType schema = new StructType(new StructField[]{ + createStructField("a", DoubleType, false), + createStructField("b", DoubleType, false) + }); + Dataset df = spark.createDataFrame(data, schema); + + Imputer imputer = new Imputer() + .setInputCols(new String[]{"a", "b"}) + .setOutputCols(new String[]{"out_a", "out_b"}); + + ImputerModel model = imputer.fit(df); + model.transform(df).show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java new file mode 100644 index 000000000000..6965512f9372 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaIndexToStringExample.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.attribute.Attribute; +import org.apache.spark.ml.feature.IndexToString; +import org.apache.spark.ml.feature.StringIndexer; +import org.apache.spark.ml.feature.StringIndexerModel; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaIndexToStringExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaIndexToStringExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(0, "a"), + RowFactory.create(1, "b"), + RowFactory.create(2, "c"), + RowFactory.create(3, "a"), + RowFactory.create(4, "a"), + RowFactory.create(5, "c") + ); + StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("category", DataTypes.StringType, false, Metadata.empty()) + }); + Dataset df = spark.createDataFrame(data, schema); + + StringIndexerModel indexer = new StringIndexer() + .setInputCol("category") + .setOutputCol("categoryIndex") + .fit(df); + Dataset indexed = indexer.transform(df); + + System.out.println("Transformed string column '" + indexer.getInputCol() + "' " + + "to indexed column '" + indexer.getOutputCol() + "'"); + indexed.show(); + + StructField inputColSchema = indexed.schema().apply(indexer.getOutputCol()); + System.out.println("StringIndexer will store labels in output column metadata: " + + Attribute.fromStructField(inputColSchema).toString() + "\n"); + + IndexToString converter = new IndexToString() + .setInputCol("categoryIndex") + .setOutputCol("originalCategory"); + Dataset converted = converter.transform(indexed); + + System.out.println("Transformed indexed column '" + converter.getInputCol() + "' back to " + + "original string column '" + converter.getOutputCol() + "' using labels in metadata"); + converted.select("id", "categoryIndex", "originalCategory").show(); + + // $example off$ + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java new file mode 100644 index 000000000000..3684a87e22e7 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.ml.feature.Interaction; +import org.apache.spark.ml.feature.VectorAssembler; +import org.apache.spark.sql.*; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import java.util.Arrays; +import java.util.List; + +// $example on$ +// $example off$ + +public class JavaInteractionExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaInteractionExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(1, 1, 2, 3, 8, 4, 5), + RowFactory.create(2, 4, 3, 8, 7, 9, 8), + RowFactory.create(3, 6, 1, 9, 2, 3, 6), + RowFactory.create(4, 10, 8, 6, 9, 4, 5), + RowFactory.create(5, 9, 2, 7, 10, 7, 3), + RowFactory.create(6, 1, 1, 4, 2, 8, 4) + ); + + StructType schema = new StructType(new StructField[]{ + new StructField("id1", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("id2", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("id3", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("id4", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("id5", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("id6", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("id7", DataTypes.IntegerType, false, Metadata.empty()) + }); + + Dataset df = spark.createDataFrame(data, schema); + + VectorAssembler assembler1 = new VectorAssembler() + .setInputCols(new String[]{"id2", "id3", "id4"}) + .setOutputCol("vec1"); + + Dataset assembled1 = assembler1.transform(df); + + VectorAssembler assembler2 = new VectorAssembler() + .setInputCols(new String[]{"id5", "id6", "id7"}) + .setOutputCol("vec2"); + + Dataset assembled2 = assembler2.transform(assembled1).select("id1", "vec1", "vec2"); + + Interaction interaction = new Interaction() + .setInputCols(new String[]{"id1","vec1","vec2"}) + .setOutputCol("interactedCol"); + + Dataset interacted = interaction.transform(assembled2); + + interacted.show(false); + // $example off$ + + spark.stop(); + } +} + diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaIsotonicRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaIsotonicRegressionExample.java new file mode 100644 index 000000000000..a7de8e699c40 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaIsotonicRegressionExample.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.ml; + +// $example on$ + +import org.apache.spark.ml.regression.IsotonicRegression; +import org.apache.spark.ml.regression.IsotonicRegressionModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + +/** + * An example demonstrating IsotonicRegression. + * Run with + *
    + * bin/run-example ml.JavaIsotonicRegressionExample
    + * 
    + */ +public class JavaIsotonicRegressionExample { + + public static void main(String[] args) { + // Create a SparkSession. + SparkSession spark = SparkSession + .builder() + .appName("JavaIsotonicRegressionExample") + .getOrCreate(); + + // $example on$ + // Loads data. + Dataset dataset = spark.read().format("libsvm") + .load("data/mllib/sample_isotonic_regression_libsvm_data.txt"); + + // Trains an isotonic regression model. + IsotonicRegression ir = new IsotonicRegression(); + IsotonicRegressionModel model = ir.fit(dataset); + + System.out.println("Boundaries in increasing order: " + model.boundaries() + "\n"); + System.out.println("Predictions associated with the boundaries: " + model.predictions() + "\n"); + + // Makes predictions. + model.transform(dataset).show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java index be2bf0c7b465..d8f948ae38cb 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaKMeansExample.java @@ -17,81 +17,52 @@ package org.apache.spark.examples.ml; -import java.util.regex.Pattern; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; +// $example on$ import org.apache.spark.ml.clustering.KMeansModel; import org.apache.spark.ml.clustering.KMeans; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.catalyst.expressions.GenericRow; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; +// $example off$ +import org.apache.spark.sql.SparkSession; /** - * An example demonstrating a k-means clustering. + * An example demonstrating k-means clustering. * Run with *
    - * bin/run-example ml.JavaSimpleParamsExample  
    + * bin/run-example ml.JavaKMeansExample
      * 
    */ public class JavaKMeansExample { - private static class ParsePoint implements Function { - private static final Pattern separator = Pattern.compile(" "); - - @Override - public Row call(String line) { - String[] tok = separator.split(line); - double[] point = new double[tok.length]; - for (int i = 0; i < tok.length; ++i) { - point[i] = Double.parseDouble(tok[i]); - } - Vector[] points = {Vectors.dense(point)}; - return new GenericRow(points); - } - } - public static void main(String[] args) { - if (args.length != 2) { - System.err.println("Usage: ml.JavaKMeansExample "); - System.exit(1); - } - String inputFile = args[0]; - int k = Integer.parseInt(args[1]); + // Create a SparkSession. + SparkSession spark = SparkSession + .builder() + .appName("JavaKMeansExample") + .getOrCreate(); - // Parses the arguments - SparkConf conf = new SparkConf().setAppName("JavaKMeansExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + // $example on$ + // Loads data. + Dataset dataset = spark.read().format("libsvm").load("data/mllib/sample_kmeans_data.txt"); - // Loads data - JavaRDD points = jsc.textFile(inputFile).map(new ParsePoint()); - StructField[] fields = {new StructField("features", new VectorUDT(), false, Metadata.empty())}; - StructType schema = new StructType(fields); - DataFrame dataset = sqlContext.createDataFrame(points, schema); - - // Trains a k-means model - KMeans kmeans = new KMeans() - .setK(k); + // Trains a k-means model. + KMeans kmeans = new KMeans().setK(2).setSeed(1L); KMeansModel model = kmeans.fit(dataset); - // Shows the result + // Evaluate clustering by computing Within Set Sum of Squared Errors. + double WSSSE = model.computeCost(dataset); + System.out.println("Within Set Sum of Squared Errors = " + WSSSE); + + // Shows the result. Vector[] centers = model.clusterCenters(); System.out.println("Cluster Centers: "); for (Vector center: centers) { System.out.println(center); } + // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java new file mode 100644 index 000000000000..0e5d00565b71 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLDAExample.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; +// $example on$ +import org.apache.spark.ml.clustering.LDA; +import org.apache.spark.ml.clustering.LDAModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off$ + +/** + * An example demonstrating LDA. + * Run with + *
    + * bin/run-example ml.JavaLDAExample
    + * 
    + */ +public class JavaLDAExample { + + public static void main(String[] args) { + // Creates a SparkSession + SparkSession spark = SparkSession + .builder() + .appName("JavaLDAExample") + .getOrCreate(); + + // $example on$ + // Loads data. + Dataset dataset = spark.read().format("libsvm") + .load("data/mllib/sample_lda_libsvm_data.txt"); + + // Trains a LDA model. + LDA lda = new LDA().setK(10).setMaxIter(10); + LDAModel model = lda.fit(dataset); + + double ll = model.logLikelihood(dataset); + double lp = model.logPerplexity(dataset); + System.out.println("The lower bound on the log likelihood of the entire corpus: " + ll); + System.out.println("The upper bound on perplexity: " + lp); + + // Describe topics. + Dataset topics = model.describeTopics(3); + System.out.println("The topics described by their top-weighted terms:"); + topics.show(false); + + // Shows the result. + Dataset transformed = model.transform(dataset); + transformed.show(false); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLabeledDocument.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLabeledDocument.java new file mode 100644 index 000000000000..68d1caf6ad3f --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLabeledDocument.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import java.io.Serializable; + +/** + * Labeled instance type, Spark SQL can infer schema from Java Beans. + */ +@SuppressWarnings("serial") +public class JavaLabeledDocument extends JavaDocument implements Serializable { + + private double label; + + public JavaLabeledDocument(long id, String text, double label) { + super(id, text); + this.label = label; + } + + public double getLabel() { + return this.label; + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java new file mode 100644 index 000000000000..a561b6d39ba8 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearRegressionWithElasticNetExample.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import org.apache.spark.ml.regression.LinearRegression; +import org.apache.spark.ml.regression.LinearRegressionModel; +import org.apache.spark.ml.regression.LinearRegressionTrainingSummary; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off$ + +public class JavaLinearRegressionWithElasticNetExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaLinearRegressionWithElasticNetExample") + .getOrCreate(); + + // $example on$ + // Load training data. + Dataset training = spark.read().format("libsvm") + .load("data/mllib/sample_linear_regression_data.txt"); + + LinearRegression lr = new LinearRegression() + .setMaxIter(10) + .setRegParam(0.3) + .setElasticNetParam(0.8); + + // Fit the model. + LinearRegressionModel lrModel = lr.fit(training); + + // Print the coefficients and intercept for linear regression. + System.out.println("Coefficients: " + + lrModel.coefficients() + " Intercept: " + lrModel.intercept()); + + // Summarize the model over the training set and print out some metrics. + LinearRegressionTrainingSummary trainingSummary = lrModel.summary(); + System.out.println("numIterations: " + trainingSummary.totalIterations()); + System.out.println("objectiveHistory: " + Vectors.dense(trainingSummary.objectiveHistory())); + trainingSummary.residuals().show(); + System.out.println("RMSE: " + trainingSummary.rootMeanSquaredError()); + System.out.println("r2: " + trainingSummary.r2()); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearSVCExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearSVCExample.java new file mode 100644 index 000000000000..a18ed1d0b48f --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLinearSVCExample.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import org.apache.spark.ml.classification.LinearSVC; +import org.apache.spark.ml.classification.LinearSVCModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off$ + +public class JavaLinearSVCExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaLinearSVCExample") + .getOrCreate(); + + // $example on$ + // Load training data + Dataset training = spark.read().format("libsvm") + .load("data/mllib/sample_libsvm_data.txt"); + + LinearSVC lsvc = new LinearSVC() + .setMaxIter(10) + .setRegParam(0.1); + + // Fit the model + LinearSVCModel lsvcModel = lsvc.fit(training); + + // Print the coefficients and intercept for LinearSVC + System.out.println("Coefficients: " + + lsvcModel.coefficients() + " Intercept: " + lsvcModel.intercept()); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java new file mode 100644 index 000000000000..dee56799d8ae --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionSummaryExample.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import org.apache.spark.ml.classification.BinaryLogisticRegressionSummary; +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.ml.classification.LogisticRegressionModel; +import org.apache.spark.ml.classification.LogisticRegressionTrainingSummary; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +// $example off$ + +public class JavaLogisticRegressionSummaryExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaLogisticRegressionSummaryExample") + .getOrCreate(); + + // Load training data + Dataset training = spark.read().format("libsvm") + .load("data/mllib/sample_libsvm_data.txt"); + + LogisticRegression lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.3) + .setElasticNetParam(0.8); + + // Fit the model + LogisticRegressionModel lrModel = lr.fit(training); + + // $example on$ + // Extract the summary from the returned LogisticRegressionModel instance trained in the earlier + // example + LogisticRegressionTrainingSummary trainingSummary = lrModel.summary(); + + // Obtain the loss per iteration. + double[] objectiveHistory = trainingSummary.objectiveHistory(); + for (double lossPerIteration : objectiveHistory) { + System.out.println(lossPerIteration); + } + + // Obtain the metrics useful to judge performance on test data. + // We cast the summary to a BinaryLogisticRegressionSummary since the problem is a binary + // classification problem. + BinaryLogisticRegressionSummary binarySummary = + (BinaryLogisticRegressionSummary) trainingSummary; + + // Obtain the receiver-operating characteristic as a dataframe and areaUnderROC. + Dataset roc = binarySummary.roc(); + roc.show(); + roc.select("FPR").show(); + System.out.println(binarySummary.areaUnderROC()); + + // Get the threshold corresponding to the maximum F-Measure and rerun LogisticRegression with + // this selected threshold. + Dataset fMeasure = binarySummary.fMeasureByThreshold(); + double maxFMeasure = fMeasure.select(functions.max("F-Measure")).head().getDouble(0); + double bestThreshold = fMeasure.where(fMeasure.col("F-Measure").equalTo(maxFMeasure)) + .select("threshold").head().getDouble(0); + lrModel.setThreshold(bestThreshold); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java new file mode 100644 index 000000000000..4cdec21d2302 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.ml.classification.LogisticRegressionModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off$ + +public class JavaLogisticRegressionWithElasticNetExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaLogisticRegressionWithElasticNetExample") + .getOrCreate(); + + // $example on$ + // Load training data + Dataset training = spark.read().format("libsvm") + .load("data/mllib/sample_libsvm_data.txt"); + + LogisticRegression lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.3) + .setElasticNetParam(0.8); + + // Fit the model + LogisticRegressionModel lrModel = lr.fit(training); + + // Print the coefficients and intercept for logistic regression + System.out.println("Coefficients: " + + lrModel.coefficients() + " Intercept: " + lrModel.intercept()); + + // We can also use the multinomial family for binary classification + LogisticRegression mlr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.3) + .setElasticNetParam(0.8) + .setFamily("multinomial"); + + // Fit the model + LogisticRegressionModel mlrModel = mlr.fit(training); + + // Print the coefficients and intercepts for logistic regression with multinomial family + System.out.println("Multinomial coefficients: " + lrModel.coefficientMatrix() + + "\nMultinomial intercepts: " + mlrModel.interceptVector()); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java new file mode 100644 index 000000000000..9f1ce463cf30 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMaxAbsScalerExample.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.MaxAbsScaler; +import org.apache.spark.ml.feature.MaxAbsScalerModel; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ +import org.apache.spark.sql.SparkSession; + +public class JavaMaxAbsScalerExample { + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaMaxAbsScalerExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(0, Vectors.dense(1.0, 0.1, -8.0)), + RowFactory.create(1, Vectors.dense(2.0, 1.0, -4.0)), + RowFactory.create(2, Vectors.dense(4.0, 10.0, 8.0)) + ); + StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("features", new VectorUDT(), false, Metadata.empty()) + }); + Dataset dataFrame = spark.createDataFrame(data, schema); + + MaxAbsScaler scaler = new MaxAbsScaler() + .setInputCol("features") + .setOutputCol("scaledFeatures"); + + // Compute summary statistics and generate MaxAbsScalerModel + MaxAbsScalerModel scalerModel = scaler.fit(dataFrame); + + // rescale each feature to range [-1, 1]. + Dataset scaledData = scalerModel.transform(dataFrame); + scaledData.select("features", "scaledFeatures").show(); + // $example off$ + + spark.stop(); + } + +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java new file mode 100644 index 000000000000..e164598e3ef8 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinHashLSHExample.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.MinHashLSH; +import org.apache.spark.ml.feature.MinHashLSHModel; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import static org.apache.spark.sql.functions.col; +// $example off$ + +/** + * An example demonstrating MinHashLSH. + * Run with: + * bin/run-example ml.JavaMinHashLSHExample + */ +public class JavaMinHashLSHExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaMinHashLSHExample") + .getOrCreate(); + + // $example on$ + List dataA = Arrays.asList( + RowFactory.create(0, Vectors.sparse(6, new int[]{0, 1, 2}, new double[]{1.0, 1.0, 1.0})), + RowFactory.create(1, Vectors.sparse(6, new int[]{2, 3, 4}, new double[]{1.0, 1.0, 1.0})), + RowFactory.create(2, Vectors.sparse(6, new int[]{0, 2, 4}, new double[]{1.0, 1.0, 1.0})) + ); + + List dataB = Arrays.asList( + RowFactory.create(0, Vectors.sparse(6, new int[]{1, 3, 5}, new double[]{1.0, 1.0, 1.0})), + RowFactory.create(1, Vectors.sparse(6, new int[]{2, 3, 5}, new double[]{1.0, 1.0, 1.0})), + RowFactory.create(2, Vectors.sparse(6, new int[]{1, 2, 4}, new double[]{1.0, 1.0, 1.0})) + ); + + StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("features", new VectorUDT(), false, Metadata.empty()) + }); + Dataset dfA = spark.createDataFrame(dataA, schema); + Dataset dfB = spark.createDataFrame(dataB, schema); + + int[] indices = {1, 3}; + double[] values = {1.0, 1.0}; + Vector key = Vectors.sparse(6, indices, values); + + MinHashLSH mh = new MinHashLSH() + .setNumHashTables(5) + .setInputCol("features") + .setOutputCol("hashes"); + + MinHashLSHModel model = mh.fit(dfA); + + // Feature Transformation + System.out.println("The hashed dataset where hashed values are stored in the column 'hashes':"); + model.transform(dfA).show(); + + // Compute the locality sensitive hashes for the input rows, then perform approximate + // similarity join. + // We could avoid computing hashes by passing in the already-transformed dataset, e.g. + // `model.approxSimilarityJoin(transformedA, transformedB, 0.6)` + System.out.println("Approximately joining dfA and dfB on Jaccard distance smaller than 0.6:"); + model.approxSimilarityJoin(dfA, dfB, 0.6, "JaccardDistance") + .select(col("datasetA.id").alias("idA"), + col("datasetB.id").alias("idB"), + col("JaccardDistance")).show(); + + // Compute the locality sensitive hashes for the input rows, then perform approximate nearest + // neighbor search. + // We could avoid computing hashes by passing in the already-transformed dataset, e.g. + // `model.approxNearestNeighbors(transformedA, key, 2)` + // It may return less than 2 rows when not enough approximate near-neighbor candidates are + // found. + System.out.println("Approximately searching dfA for 2 nearest neighbors of the key:"); + model.approxNearestNeighbors(dfA, key, 2).show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java new file mode 100644 index 000000000000..2757af8d245d --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMinMaxScalerExample.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.MinMaxScaler; +import org.apache.spark.ml.feature.MinMaxScalerModel; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaMinMaxScalerExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaMinMaxScalerExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(0, Vectors.dense(1.0, 0.1, -1.0)), + RowFactory.create(1, Vectors.dense(2.0, 1.1, 1.0)), + RowFactory.create(2, Vectors.dense(3.0, 10.1, 3.0)) + ); + StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("features", new VectorUDT(), false, Metadata.empty()) + }); + Dataset dataFrame = spark.createDataFrame(data, schema); + + MinMaxScaler scaler = new MinMaxScaler() + .setInputCol("features") + .setOutputCol("scaledFeatures"); + + // Compute summary statistics and generate MinMaxScalerModel + MinMaxScalerModel scalerModel = scaler.fit(dataFrame); + + // rescale each feature to range [min, max]. + Dataset scaledData = scalerModel.transform(dataFrame); + System.out.println("Features scaled to range: [" + scaler.getMin() + ", " + + scaler.getMax() + "]"); + scaledData.select("features", "scaledFeatures").show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java new file mode 100644 index 000000000000..d97327969ab2 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import java.util.Arrays; +// $example off$ + +// $example on$ +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator; +import org.apache.spark.ml.feature.HashingTF; +import org.apache.spark.ml.feature.Tokenizer; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.ml.tuning.CrossValidator; +import org.apache.spark.ml.tuning.CrossValidatorModel; +import org.apache.spark.ml.tuning.ParamGridBuilder; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + +/** + * Java example for Model Selection via Cross Validation. + */ +public class JavaModelSelectionViaCrossValidationExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaModelSelectionViaCrossValidationExample") + .getOrCreate(); + + // $example on$ + // Prepare training documents, which are labeled. + Dataset training = spark.createDataFrame(Arrays.asList( + new JavaLabeledDocument(0L, "a b c d e spark", 1.0), + new JavaLabeledDocument(1L, "b d", 0.0), + new JavaLabeledDocument(2L,"spark f g h", 1.0), + new JavaLabeledDocument(3L, "hadoop mapreduce", 0.0), + new JavaLabeledDocument(4L, "b spark who", 1.0), + new JavaLabeledDocument(5L, "g d a y", 0.0), + new JavaLabeledDocument(6L, "spark fly", 1.0), + new JavaLabeledDocument(7L, "was mapreduce", 0.0), + new JavaLabeledDocument(8L, "e spark program", 1.0), + new JavaLabeledDocument(9L, "a e c l", 0.0), + new JavaLabeledDocument(10L, "spark compile", 1.0), + new JavaLabeledDocument(11L, "hadoop software", 0.0) + ), JavaLabeledDocument.class); + + // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. + Tokenizer tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words"); + HashingTF hashingTF = new HashingTF() + .setNumFeatures(1000) + .setInputCol(tokenizer.getOutputCol()) + .setOutputCol("features"); + LogisticRegression lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.01); + Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {tokenizer, hashingTF, lr}); + + // We use a ParamGridBuilder to construct a grid of parameters to search over. + // With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, + // this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. + ParamMap[] paramGrid = new ParamGridBuilder() + .addGrid(hashingTF.numFeatures(), new int[] {10, 100, 1000}) + .addGrid(lr.regParam(), new double[] {0.1, 0.01}) + .build(); + + // We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. + // This will allow us to jointly choose parameters for all Pipeline stages. + // A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. + // Note that the evaluator here is a BinaryClassificationEvaluator and its default metric + // is areaUnderROC. + CrossValidator cv = new CrossValidator() + .setEstimator(pipeline) + .setEvaluator(new BinaryClassificationEvaluator()) + .setEstimatorParamMaps(paramGrid) + .setNumFolds(2) // Use 3+ in practice + .setParallelism(2); // Evaluate up to 2 parameter settings in parallel + + // Run cross-validation, and choose the best set of parameters. + CrossValidatorModel cvModel = cv.fit(training); + + // Prepare test documents, which are unlabeled. + Dataset test = spark.createDataFrame(Arrays.asList( + new JavaDocument(4L, "spark i j k"), + new JavaDocument(5L, "l m n"), + new JavaDocument(6L, "mapreduce spark"), + new JavaDocument(7L, "apache hadoop") + ), JavaDocument.class); + + // Make predictions on test documents. cvModel uses the best model found (lrModel). + Dataset predictions = cvModel.transform(test); + for (Row r : predictions.select("id", "text", "probability", "prediction").collectAsList()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + + ", prediction=" + r.get(3)); + } + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java new file mode 100644 index 000000000000..2ef8bea0b2a2 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaTrainValidationSplitExample.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import org.apache.spark.ml.evaluation.RegressionEvaluator; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.ml.regression.LinearRegression; +import org.apache.spark.ml.tuning.ParamGridBuilder; +import org.apache.spark.ml.tuning.TrainValidationSplit; +import org.apache.spark.ml.tuning.TrainValidationSplitModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + +/** + * Java example demonstrating model selection using TrainValidationSplit. + * + * Run with + * {{{ + * bin/run-example ml.JavaModelSelectionViaTrainValidationSplitExample + * }}} + */ +public class JavaModelSelectionViaTrainValidationSplitExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaModelSelectionViaTrainValidationSplitExample") + .getOrCreate(); + + // $example on$ + Dataset data = spark.read().format("libsvm") + .load("data/mllib/sample_linear_regression_data.txt"); + + // Prepare training and test data. + Dataset[] splits = data.randomSplit(new double[] {0.9, 0.1}, 12345); + Dataset training = splits[0]; + Dataset test = splits[1]; + + LinearRegression lr = new LinearRegression(); + + // We use a ParamGridBuilder to construct a grid of parameters to search over. + // TrainValidationSplit will try all combinations of values and determine best model using + // the evaluator. + ParamMap[] paramGrid = new ParamGridBuilder() + .addGrid(lr.regParam(), new double[] {0.1, 0.01}) + .addGrid(lr.fitIntercept()) + .addGrid(lr.elasticNetParam(), new double[] {0.0, 0.5, 1.0}) + .build(); + + // In this case the estimator is simply the linear regression. + // A TrainValidationSplit requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. + TrainValidationSplit trainValidationSplit = new TrainValidationSplit() + .setEstimator(lr) + .setEvaluator(new RegressionEvaluator()) + .setEstimatorParamMaps(paramGrid) + .setTrainRatio(0.8) // 80% for training and the remaining 20% for validation + .setParallelism(2); // Evaluate up to 2 parameter settings in parallel + + // Run train validation split, and choose the best set of parameters. + TrainValidationSplitModel model = trainValidationSplit.fit(training); + + // Make predictions on test data. model is the model with combination of parameters + // that performed best. + model.transform(test) + .select("features", "label", "prediction") + .show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMulticlassLogisticRegressionWithElasticNetExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMulticlassLogisticRegressionWithElasticNetExample.java new file mode 100644 index 000000000000..da410cba2b3f --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMulticlassLogisticRegressionWithElasticNetExample.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.ml.classification.LogisticRegressionModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off$ + +public class JavaMulticlassLogisticRegressionWithElasticNetExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaMulticlassLogisticRegressionWithElasticNetExample") + .getOrCreate(); + + // $example on$ + // Load training data + Dataset training = spark.read().format("libsvm") + .load("data/mllib/sample_multiclass_classification_data.txt"); + + LogisticRegression lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.3) + .setElasticNetParam(0.8); + + // Fit the model + LogisticRegressionModel lrModel = lr.fit(training); + + // Print the coefficients and intercept for multinomial logistic regression + System.out.println("Coefficients: \n" + + lrModel.coefficientMatrix() + " \nIntercept: " + lrModel.interceptVector()); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java new file mode 100644 index 000000000000..43db41ce1746 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaMultilayerPerceptronClassifierExample.java @@ -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.examples.ml; + +// $example on$ +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel; +import org.apache.spark.ml.classification.MultilayerPerceptronClassifier; +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; +// $example off$ + +/** + * An example for Multilayer Perceptron Classification. + */ +public class JavaMultilayerPerceptronClassifierExample { + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaMultilayerPerceptronClassifierExample") + .getOrCreate(); + + // $example on$ + // Load training data + String path = "data/mllib/sample_multiclass_classification_data.txt"; + Dataset dataFrame = spark.read().format("libsvm").load(path); + + // Split the data into train and test + Dataset[] splits = dataFrame.randomSplit(new double[]{0.6, 0.4}, 1234L); + Dataset train = splits[0]; + Dataset test = splits[1]; + + // specify layers for the neural network: + // input layer of size 4 (features), two intermediate of size 5 and 4 + // and output of size 3 (classes) + int[] layers = new int[] {4, 5, 4, 3}; + + // create the trainer and set its parameters + MultilayerPerceptronClassifier trainer = new MultilayerPerceptronClassifier() + .setLayers(layers) + .setBlockSize(128) + .setSeed(1234L) + .setMaxIter(100); + + // train the model + MultilayerPerceptronClassificationModel model = trainer.fit(train); + + // compute accuracy on the test set + Dataset result = model.transform(test); + Dataset predictionAndLabels = result.select("prediction", "label"); + MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() + .setMetricName("accuracy"); + + System.out.println("Test set accuracy = " + evaluator.evaluate(predictionAndLabels)); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java new file mode 100644 index 000000000000..5427e466656a --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.NGram; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaNGramExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaNGramExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(0, Arrays.asList("Hi", "I", "heard", "about", "Spark")), + RowFactory.create(1, Arrays.asList("I", "wish", "Java", "could", "use", "case", "classes")), + RowFactory.create(2, Arrays.asList("Logistic", "regression", "models", "are", "neat")) + ); + + StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField( + "words", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty()) + }); + + Dataset wordDataFrame = spark.createDataFrame(data, schema); + + NGram ngramTransformer = new NGram().setN(2).setInputCol("words").setOutputCol("ngrams"); + + Dataset ngramDataFrame = ngramTransformer.transform(wordDataFrame); + ngramDataFrame.select("ngrams").show(false); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java new file mode 100644 index 000000000000..be578dc8110e --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNaiveBayesExample.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import org.apache.spark.ml.classification.NaiveBayes; +import org.apache.spark.ml.classification.NaiveBayesModel; +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off$ + +/** + * An example for Naive Bayes Classification. + */ +public class JavaNaiveBayesExample { + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaNaiveBayesExample") + .getOrCreate(); + + // $example on$ + // Load training data + Dataset dataFrame = + spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + // Split the data into train and test + Dataset[] splits = dataFrame.randomSplit(new double[]{0.6, 0.4}, 1234L); + Dataset train = splits[0]; + Dataset test = splits[1]; + + // create the trainer and set its parameters + NaiveBayes nb = new NaiveBayes(); + + // train the model + NaiveBayesModel model = nb.fit(train); + + // Select example rows to display. + Dataset predictions = model.transform(test); + predictions.show(); + + // compute accuracy on the test set + MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("accuracy"); + double accuracy = evaluator.evaluate(predictions); + System.out.println("Test set accuracy = " + accuracy); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java new file mode 100644 index 000000000000..f878c420d823 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNormalizerExample.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.Normalizer; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaNormalizerExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaNormalizerExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(0, Vectors.dense(1.0, 0.1, -8.0)), + RowFactory.create(1, Vectors.dense(2.0, 1.0, -4.0)), + RowFactory.create(2, Vectors.dense(4.0, 10.0, 8.0)) + ); + StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("features", new VectorUDT(), false, Metadata.empty()) + }); + Dataset dataFrame = spark.createDataFrame(data, schema); + + // Normalize each Vector using $L^1$ norm. + Normalizer normalizer = new Normalizer() + .setInputCol("features") + .setOutputCol("normFeatures") + .setP(1.0); + + Dataset l1NormData = normalizer.transform(dataFrame); + l1NormData.show(); + + // Normalize each Vector using $L^\infty$ norm. + Dataset lInfNormData = + normalizer.transform(dataFrame, normalizer.p().w(Double.POSITIVE_INFINITY)); + lInfNormData.show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java new file mode 100644 index 000000000000..99af37676ba9 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.OneHotEncoder; +import org.apache.spark.ml.feature.StringIndexer; +import org.apache.spark.ml.feature.StringIndexerModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaOneHotEncoderExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaOneHotEncoderExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(0, "a"), + RowFactory.create(1, "b"), + RowFactory.create(2, "c"), + RowFactory.create(3, "a"), + RowFactory.create(4, "a"), + RowFactory.create(5, "c") + ); + + StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("category", DataTypes.StringType, false, Metadata.empty()) + }); + + Dataset df = spark.createDataFrame(data, schema); + + StringIndexerModel indexer = new StringIndexer() + .setInputCol("category") + .setOutputCol("categoryIndex") + .fit(df); + Dataset indexed = indexer.transform(df); + + OneHotEncoder encoder = new OneHotEncoder() + .setInputCol("categoryIndex") + .setOutputCol("categoryVec"); + + Dataset encoded = encoder.transform(indexed); + encoded.show(); + // $example off$ + + spark.stop(); + } +} + diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java index e7f2f6f61507..82fb54095019 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneVsRestExample.java @@ -17,221 +17,68 @@ package org.apache.spark.examples.ml; -import org.apache.commons.cli.*; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; +// $example on$ import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.ml.classification.OneVsRest; import org.apache.spark.ml.classification.OneVsRestModel; -import org.apache.spark.ml.util.MetadataUtils; -import org.apache.spark.mllib.evaluation.MulticlassMetrics; -import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.rdd.RDD; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.types.StructField; +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + /** - * An example runner for Multiclass to Binary Reduction with One Vs Rest. - * The example uses Logistic Regression as the base classifier. All parameters that - * can be specified on the base classifier can be passed in to the runner options. + * An example of Multiclass to Binary Reduction with One Vs Rest, + * using Logistic Regression as the base classifier. * Run with *
    - * bin/run-example ml.JavaOneVsRestExample [options]
    + * bin/run-example ml.JavaOneVsRestExample
      * 
    */ public class JavaOneVsRestExample { - - private static class Params { - String input; - String testInput = null; - Integer maxIter = 100; - double tol = 1E-6; - boolean fitIntercept = true; - Double regParam = null; - Double elasticNetParam = null; - double fracTest = 0.2; - } - public static void main(String[] args) { - // parse the arguments - Params params = parse(args); - SparkConf conf = new SparkConf().setAppName("JavaOneVsRestExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); - - // configure the base classifier + SparkSession spark = SparkSession + .builder() + .appName("JavaOneVsRestExample") + .getOrCreate(); + + // $example on$ + // load data file. + Dataset inputData = spark.read().format("libsvm") + .load("data/mllib/sample_multiclass_classification_data.txt"); + + // generate the train/test split. + Dataset[] tmp = inputData.randomSplit(new double[]{0.8, 0.2}); + Dataset train = tmp[0]; + Dataset test = tmp[1]; + + // configure the base classifier. LogisticRegression classifier = new LogisticRegression() - .setMaxIter(params.maxIter) - .setTol(params.tol) - .setFitIntercept(params.fitIntercept); - - if (params.regParam != null) { - classifier.setRegParam(params.regParam); - } - if (params.elasticNetParam != null) { - classifier.setElasticNetParam(params.elasticNetParam); - } + .setMaxIter(10) + .setTol(1E-6) + .setFitIntercept(true); - // instantiate the One Vs Rest Classifier + // instantiate the One Vs Rest Classifier. OneVsRest ovr = new OneVsRest().setClassifier(classifier); - String input = params.input; - RDD inputData = MLUtils.loadLibSVMFile(jsc.sc(), input); - RDD train; - RDD test; - - // compute the train/ test split: if testInput is not provided use part of input - String testInput = params.testInput; - if (testInput != null) { - train = inputData; - // compute the number of features in the training set. - int numFeatures = inputData.first().features().size(); - test = MLUtils.loadLibSVMFile(jsc.sc(), testInput, numFeatures); - } else { - double f = params.fracTest; - RDD[] tmp = inputData.randomSplit(new double[]{1 - f, f}, 12345); - train = tmp[0]; - test = tmp[1]; - } + // train the multiclass model. + OneVsRestModel ovrModel = ovr.fit(train); - // train the multiclass model - DataFrame trainingDataFrame = jsql.createDataFrame(train, LabeledPoint.class); - OneVsRestModel ovrModel = ovr.fit(trainingDataFrame.cache()); - - // score the model on test data - DataFrame testDataFrame = jsql.createDataFrame(test, LabeledPoint.class); - DataFrame predictions = ovrModel.transform(testDataFrame.cache()) + // score the model on test data. + Dataset predictions = ovrModel.transform(test) .select("prediction", "label"); - // obtain metrics - MulticlassMetrics metrics = new MulticlassMetrics(predictions); - StructField predictionColSchema = predictions.schema().apply("prediction"); - Integer numClasses = (Integer) MetadataUtils.getNumClasses(predictionColSchema).get(); - - // compute the false positive rate per label - StringBuilder results = new StringBuilder(); - results.append("label\tfpr\n"); - for (int label = 0; label < numClasses; label++) { - results.append(label); - results.append("\t"); - results.append(metrics.falsePositiveRate((double) label)); - results.append("\n"); - } + // obtain evaluator. + MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() + .setMetricName("accuracy"); - Matrix confusionMatrix = metrics.confusionMatrix(); - // output the Confusion Matrix - System.out.println("Confusion Matrix"); - System.out.println(confusionMatrix); - System.out.println(); - System.out.println(results); + // compute the classification error on test data. + double accuracy = evaluator.evaluate(predictions); + System.out.println("Test Error = " + (1 - accuracy)); + // $example off$ - jsc.stop(); + spark.stop(); } - private static Params parse(String[] args) { - Options options = generateCommandlineOptions(); - CommandLineParser parser = new PosixParser(); - Params params = new Params(); - - try { - CommandLine cmd = parser.parse(options, args); - String value; - if (cmd.hasOption("input")) { - params.input = cmd.getOptionValue("input"); - } - if (cmd.hasOption("maxIter")) { - value = cmd.getOptionValue("maxIter"); - params.maxIter = Integer.parseInt(value); - } - if (cmd.hasOption("tol")) { - value = cmd.getOptionValue("tol"); - params.tol = Double.parseDouble(value); - } - if (cmd.hasOption("fitIntercept")) { - value = cmd.getOptionValue("fitIntercept"); - params.fitIntercept = Boolean.parseBoolean(value); - } - if (cmd.hasOption("regParam")) { - value = cmd.getOptionValue("regParam"); - params.regParam = Double.parseDouble(value); - } - if (cmd.hasOption("elasticNetParam")) { - value = cmd.getOptionValue("elasticNetParam"); - params.elasticNetParam = Double.parseDouble(value); - } - if (cmd.hasOption("testInput")) { - value = cmd.getOptionValue("testInput"); - params.testInput = value; - } - if (cmd.hasOption("fracTest")) { - value = cmd.getOptionValue("fracTest"); - params.fracTest = Double.parseDouble(value); - } - - } catch (ParseException e) { - printHelpAndQuit(options); - } - return params; - } - - @SuppressWarnings("static") - private static Options generateCommandlineOptions() { - Option input = OptionBuilder.withArgName("input") - .hasArg() - .isRequired() - .withDescription("input path to labeled examples. This path must be specified") - .create("input"); - Option testInput = OptionBuilder.withArgName("testInput") - .hasArg() - .withDescription("input path to test examples") - .create("testInput"); - Option fracTest = OptionBuilder.withArgName("testInput") - .hasArg() - .withDescription("fraction of data to hold out for testing." + - " If given option testInput, this option is ignored. default: 0.2") - .create("fracTest"); - Option maxIter = OptionBuilder.withArgName("maxIter") - .hasArg() - .withDescription("maximum number of iterations for Logistic Regression. default:100") - .create("maxIter"); - Option tol = OptionBuilder.withArgName("tol") - .hasArg() - .withDescription("the convergence tolerance of iterations " + - "for Logistic Regression. default: 1E-6") - .create("tol"); - Option fitIntercept = OptionBuilder.withArgName("fitIntercept") - .hasArg() - .withDescription("fit intercept for logistic regression. default true") - .create("fitIntercept"); - Option regParam = OptionBuilder.withArgName( "regParam" ) - .hasArg() - .withDescription("the regularization parameter for Logistic Regression.") - .create("regParam"); - Option elasticNetParam = OptionBuilder.withArgName("elasticNetParam" ) - .hasArg() - .withDescription("the ElasticNet mixing parameter for Logistic Regression.") - .create("elasticNetParam"); - - Options options = new Options() - .addOption(input) - .addOption(testInput) - .addOption(fracTest) - .addOption(maxIter) - .addOption(tol) - .addOption(fitIntercept) - .addOption(regParam) - .addOption(elasticNetParam); - - return options; - } - - private static void printHelpAndQuit(Options options) { - HelpFormatter formatter = new HelpFormatter(); - formatter.printHelp("JavaOneVsRestExample", options); - System.exit(-1); - } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java new file mode 100644 index 000000000000..6951a65553e5 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPCAExample.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.PCA; +import org.apache.spark.ml.feature.PCAModel; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaPCAExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaPCAExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(Vectors.sparse(5, new int[]{1, 3}, new double[]{1.0, 7.0})), + RowFactory.create(Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0)), + RowFactory.create(Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0)) + ); + + StructType schema = new StructType(new StructField[]{ + new StructField("features", new VectorUDT(), false, Metadata.empty()), + }); + + Dataset df = spark.createDataFrame(data, schema); + + PCAModel pca = new PCA() + .setInputCol("features") + .setOutputCol("pcaFeatures") + .setK(3) + .fit(df); + + Dataset result = pca.transform(df).select("pcaFeatures"); + result.show(false); + // $example off$ + spark.stop(); + } +} + diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java new file mode 100644 index 000000000000..4ccd8f6ce265 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import java.util.Arrays; + +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.ml.feature.HashingTF; +import org.apache.spark.ml.feature.Tokenizer; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ +import org.apache.spark.sql.SparkSession; + +/** + * Java example for simple text document 'Pipeline'. + */ +public class JavaPipelineExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaPipelineExample") + .getOrCreate(); + + // $example on$ + // Prepare training documents, which are labeled. + Dataset training = spark.createDataFrame(Arrays.asList( + new JavaLabeledDocument(0L, "a b c d e spark", 1.0), + new JavaLabeledDocument(1L, "b d", 0.0), + new JavaLabeledDocument(2L, "spark f g h", 1.0), + new JavaLabeledDocument(3L, "hadoop mapreduce", 0.0) + ), JavaLabeledDocument.class); + + // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. + Tokenizer tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words"); + HashingTF hashingTF = new HashingTF() + .setNumFeatures(1000) + .setInputCol(tokenizer.getOutputCol()) + .setOutputCol("features"); + LogisticRegression lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.001); + Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {tokenizer, hashingTF, lr}); + + // Fit the pipeline to training documents. + PipelineModel model = pipeline.fit(training); + + // Prepare test documents, which are unlabeled. + Dataset test = spark.createDataFrame(Arrays.asList( + new JavaDocument(4L, "spark i j k"), + new JavaDocument(5L, "l m n"), + new JavaDocument(6L, "spark hadoop spark"), + new JavaDocument(7L, "apache hadoop") + ), JavaDocument.class); + + // Make predictions on test documents. + Dataset predictions = model.transform(test); + for (Row r : predictions.select("id", "text", "probability", "prediction").collectAsList()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + + ", prediction=" + r.get(3)); + } + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java new file mode 100644 index 000000000000..43c636c53403 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.PolynomialExpansion; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaPolynomialExpansionExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaPolynomialExpansionExample") + .getOrCreate(); + + // $example on$ + PolynomialExpansion polyExpansion = new PolynomialExpansion() + .setInputCol("features") + .setOutputCol("polyFeatures") + .setDegree(3); + + List data = Arrays.asList( + RowFactory.create(Vectors.dense(2.0, 1.0)), + RowFactory.create(Vectors.dense(0.0, 0.0)), + RowFactory.create(Vectors.dense(3.0, -1.0)) + ); + StructType schema = new StructType(new StructField[]{ + new StructField("features", new VectorUDT(), false, Metadata.empty()), + }); + Dataset df = spark.createDataFrame(data, schema); + + Dataset polyDF = polyExpansion.transform(df); + polyDF.show(false); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java new file mode 100644 index 000000000000..dd20cac62110 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaQuantileDiscretizerExample.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.QuantileDiscretizer; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaQuantileDiscretizerExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaQuantileDiscretizerExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(0, 18.0), + RowFactory.create(1, 19.0), + RowFactory.create(2, 8.0), + RowFactory.create(3, 5.0), + RowFactory.create(4, 2.2) + ); + + StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("hour", DataTypes.DoubleType, false, Metadata.empty()) + }); + + Dataset df = spark.createDataFrame(data, schema); + // $example off$ + // Output of QuantileDiscretizer for such small datasets can depend on the number of + // partitions. Here we force a single partition to ensure consistent results. + // Note this is not necessary for normal use cases + df = df.repartition(1); + // $example on$ + QuantileDiscretizer discretizer = new QuantileDiscretizer() + .setInputCol("hour") + .setOutputCol("result") + .setNumBuckets(3); + + Dataset result = discretizer.fit(df).transform(df); + result.show(); + // $example off$ + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java new file mode 100644 index 000000000000..428067e0f7ef --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRFormulaExample.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.RFormula; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import static org.apache.spark.sql.types.DataTypes.*; +// $example off$ + +public class JavaRFormulaExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaRFormulaExample") + .getOrCreate(); + + // $example on$ + StructType schema = createStructType(new StructField[]{ + createStructField("id", IntegerType, false), + createStructField("country", StringType, false), + createStructField("hour", IntegerType, false), + createStructField("clicked", DoubleType, false) + }); + + List data = Arrays.asList( + RowFactory.create(7, "US", 18, 1.0), + RowFactory.create(8, "CA", 12, 0.0), + RowFactory.create(9, "NZ", 15, 0.0) + ); + + Dataset dataset = spark.createDataFrame(data, schema); + RFormula formula = new RFormula() + .setFormula("clicked ~ country + hour") + .setFeaturesCol("features") + .setLabelCol("label"); + Dataset output = formula.fit(dataset).transform(dataset); + output.select("features", "label").show(); + // $example off$ + spark.stop(); + } +} + diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java new file mode 100644 index 000000000000..da2633e8860a --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestClassifierExample.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.RandomForestClassificationModel; +import org.apache.spark.ml.classification.RandomForestClassifier; +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; +import org.apache.spark.ml.feature.*; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off$ + +public class JavaRandomForestClassifierExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaRandomForestClassifierExample") + .getOrCreate(); + + // $example on$ + // Load and parse the data file, converting it to a DataFrame. + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + + // Index labels, adding metadata to the label column. + // Fit on whole dataset to include all labels in index. + StringIndexerModel labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data); + // Automatically identify categorical features, and index them. + // Set maxCategories so features with > 4 distinct values are treated as continuous. + VectorIndexerModel featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data); + + // Split the data into training and test sets (30% held out for testing) + Dataset[] splits = data.randomSplit(new double[] {0.7, 0.3}); + Dataset trainingData = splits[0]; + Dataset testData = splits[1]; + + // Train a RandomForest model. + RandomForestClassifier rf = new RandomForestClassifier() + .setLabelCol("indexedLabel") + .setFeaturesCol("indexedFeatures"); + + // Convert indexed labels back to original labels. + IndexToString labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labels()); + + // Chain indexers and forest in a Pipeline + Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {labelIndexer, featureIndexer, rf, labelConverter}); + + // Train model. This also runs the indexers. + PipelineModel model = pipeline.fit(trainingData); + + // Make predictions. + Dataset predictions = model.transform(testData); + + // Select example rows to display. + predictions.select("predictedLabel", "label", "features").show(5); + + // Select (prediction, true label) and compute test error + MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("accuracy"); + double accuracy = evaluator.evaluate(predictions); + System.out.println("Test Error = " + (1.0 - accuracy)); + + RandomForestClassificationModel rfModel = (RandomForestClassificationModel)(model.stages()[2]); + System.out.println("Learned classification forest model:\n" + rfModel.toDebugString()); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java new file mode 100644 index 000000000000..a7078453deb8 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaRandomForestRegressorExample.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.evaluation.RegressionEvaluator; +import org.apache.spark.ml.feature.VectorIndexer; +import org.apache.spark.ml.feature.VectorIndexerModel; +import org.apache.spark.ml.regression.RandomForestRegressionModel; +import org.apache.spark.ml.regression.RandomForestRegressor; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off$ + +public class JavaRandomForestRegressorExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaRandomForestRegressorExample") + .getOrCreate(); + + // $example on$ + // Load and parse the data file, converting it to a DataFrame. + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + + // Automatically identify categorical features, and index them. + // Set maxCategories so features with > 4 distinct values are treated as continuous. + VectorIndexerModel featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data); + + // Split the data into training and test sets (30% held out for testing) + Dataset[] splits = data.randomSplit(new double[] {0.7, 0.3}); + Dataset trainingData = splits[0]; + Dataset testData = splits[1]; + + // Train a RandomForest model. + RandomForestRegressor rf = new RandomForestRegressor() + .setLabelCol("label") + .setFeaturesCol("indexedFeatures"); + + // Chain indexer and forest in a Pipeline + Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {featureIndexer, rf}); + + // Train model. This also runs the indexer. + PipelineModel model = pipeline.fit(trainingData); + + // Make predictions. + Dataset predictions = model.transform(testData); + + // Select example rows to display. + predictions.select("prediction", "label", "features").show(5); + + // Select (prediction, true label) and compute test error + RegressionEvaluator evaluator = new RegressionEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("rmse"); + double rmse = evaluator.evaluate(predictions); + System.out.println("Root Mean Squared Error (RMSE) on test data = " + rmse); + + RandomForestRegressionModel rfModel = (RandomForestRegressionModel)(model.stages()[1]); + System.out.println("Learned regression forest model:\n" + rfModel.toDebugString()); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java new file mode 100644 index 000000000000..2a3d62de41ab --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSQLTransformerExample.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.SQLTransformer; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.*; +// $example off$ + +public class JavaSQLTransformerExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaSQLTransformerExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(0, 1.0, 3.0), + RowFactory.create(2, 2.0, 5.0) + ); + StructType schema = new StructType(new StructField [] { + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("v1", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("v2", DataTypes.DoubleType, false, Metadata.empty()) + }); + Dataset df = spark.createDataFrame(data, schema); + + SQLTransformer sqlTrans = new SQLTransformer().setStatement( + "SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__"); + + sqlTrans.transform(df).show(); + // $example off$ + + spark.stop(); + } +} 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 deleted file mode 100644 index 94beeced3d47..000000000000 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ /dev/null @@ -1,112 +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.examples.ml; - -import java.util.List; - -import com.google.common.collect.Lists; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.classification.LogisticRegressionModel; -import org.apache.spark.ml.param.ParamMap; -import org.apache.spark.ml.classification.LogisticRegression; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; - -/** - * A simple example demonstrating ways to specify parameters for Estimators and Transformers. - * Run with - * {{{ - * bin/run-example ml.JavaSimpleParamsExample - * }}} - */ -public class JavaSimpleParamsExample { - - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaSimpleParamsExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); - - // Prepare training data. - // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans - // into DataFrames, where it uses the bean metadata to infer the schema. - List localTraining = Lists.newArrayList( - new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), - new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), - new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), - new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); - DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledPoint.class); - - // Create a LogisticRegression instance. This instance is an Estimator. - LogisticRegression lr = new LogisticRegression(); - // Print out the parameters, documentation, and any default values. - System.out.println("LogisticRegression parameters:\n" + lr.explainParams() + "\n"); - - // We may set parameters using setter methods. - lr.setMaxIter(10) - .setRegParam(0.01); - - // Learn a LogisticRegression model. This uses the parameters stored in lr. - LogisticRegressionModel model1 = lr.fit(training); - // 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 - // LogisticRegression instance. - System.out.println("Model 1 was fit using parameters: " + model1.parent().extractParamMap()); - - // We may alternatively specify parameters using a ParamMap. - ParamMap paramMap = new ParamMap(); - paramMap.put(lr.maxIter().w(20)); // Specify 1 Param. - paramMap.put(lr.maxIter(), 30); // This overwrites the original maxIter. - double thresholds[] = {0.45, 0.55}; - paramMap.put(lr.regParam().w(0.1), lr.thresholds().w(thresholds)); // Specify multiple Params. - - // One can also combine ParamMaps. - ParamMap paramMap2 = new ParamMap(); - paramMap2.put(lr.probabilityCol().w("myProbability")); // Change output column name - ParamMap paramMapCombined = paramMap.$plus$plus(paramMap2); - - // Now learn a new model using the paramMapCombined parameters. - // paramMapCombined overrides all parameters set earlier via lr.set* methods. - LogisticRegressionModel model2 = lr.fit(training, paramMapCombined); - System.out.println("Model 2 was fit using parameters: " + model2.parent().extractParamMap()); - - // Prepare test documents. - List localTest = Lists.newArrayList( - new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), - new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), - new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); - DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class); - - // Make predictions on test documents using the Transformer.transform() method. - // LogisticRegressionModel.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. - DataFrame results = model2.transform(test); - for (Row r: results.select("features", "label", "myProbability", "prediction").collect()) { - System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) - + ", prediction=" + r.get(3)); - } - - jsc.stop(); - } -} 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 deleted file mode 100644 index 54738813d001..000000000000 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java +++ /dev/null @@ -1,93 +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.examples.ml; - -import java.util.List; - -import com.google.common.collect.Lists; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.Pipeline; -import org.apache.spark.ml.PipelineModel; -import org.apache.spark.ml.PipelineStage; -import org.apache.spark.ml.classification.LogisticRegression; -import org.apache.spark.ml.feature.HashingTF; -import org.apache.spark.ml.feature.Tokenizer; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; - -/** - * A simple text classification pipeline that recognizes "spark" from input text. It uses the Java - * bean classes {@link LabeledDocument} and {@link Document} defined in the Scala counterpart of - * this example {@link SimpleTextClassificationPipeline}. Run with - *
    - * bin/run-example ml.JavaSimpleTextClassificationPipeline
    - * 
    - */ -public class JavaSimpleTextClassificationPipeline { - - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaSimpleTextClassificationPipeline"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); - - // Prepare training documents, which are labeled. - List localTraining = Lists.newArrayList( - new LabeledDocument(0L, "a b c d e spark", 1.0), - 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.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); - - // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. - Tokenizer tokenizer = new Tokenizer() - .setInputCol("text") - .setOutputCol("words"); - HashingTF hashingTF = new HashingTF() - .setNumFeatures(1000) - .setInputCol(tokenizer.getOutputCol()) - .setOutputCol("features"); - LogisticRegression lr = new LogisticRegression() - .setMaxIter(10) - .setRegParam(0.001); - Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[] {tokenizer, hashingTF, lr}); - - // Fit the pipeline to training documents. - PipelineModel model = pipeline.fit(training); - - // Prepare test documents, which are unlabeled. - List localTest = Lists.newArrayList( - new Document(4L, "spark i j k"), - new Document(5L, "l m n"), - new Document(6L, "spark hadoop spark"), - new Document(7L, "apache hadoop")); - DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); - - // Make predictions on test documents. - DataFrame predictions = model.transform(test); - for (Row r: predictions.select("id", "text", "probability", "prediction").collect()) { - System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) - + ", prediction=" + r.get(3)); - } - - jsc.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java new file mode 100644 index 000000000000..08ea285a0d53 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStandardScalerExample.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import org.apache.spark.ml.feature.StandardScaler; +import org.apache.spark.ml.feature.StandardScalerModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ + +public class JavaStandardScalerExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaStandardScalerExample") + .getOrCreate(); + + // $example on$ + Dataset dataFrame = + spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + + StandardScaler scaler = new StandardScaler() + .setInputCol("features") + .setOutputCol("scaledFeatures") + .setWithStd(true) + .setWithMean(false); + + // Compute summary statistics by fitting the StandardScaler + StandardScalerModel scalerModel = scaler.fit(dataFrame); + + // Normalize each feature to have unit standard deviation. + Dataset scaledData = scalerModel.transform(dataFrame); + scaledData.show(); + // $example off$ + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java new file mode 100644 index 000000000000..94ead625b474 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStopWordsRemoverExample.java @@ -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.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.StopWordsRemover; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off$ + +public class JavaStopWordsRemoverExample { + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaStopWordsRemoverExample") + .getOrCreate(); + + // $example on$ + StopWordsRemover remover = new StopWordsRemover() + .setInputCol("raw") + .setOutputCol("filtered"); + + List data = Arrays.asList( + RowFactory.create(Arrays.asList("I", "saw", "the", "red", "balloon")), + RowFactory.create(Arrays.asList("Mary", "had", "a", "little", "lamb")) + ); + + StructType schema = new StructType(new StructField[]{ + new StructField( + "raw", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty()) + }); + + Dataset dataset = spark.createDataFrame(data, schema); + remover.transform(dataset).show(false); + // $example off$ + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java new file mode 100644 index 000000000000..cf9747a99469 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaStringIndexerExample.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.feature.StringIndexer; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import static org.apache.spark.sql.types.DataTypes.*; +// $example off$ + +public class JavaStringIndexerExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaStringIndexerExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(0, "a"), + RowFactory.create(1, "b"), + RowFactory.create(2, "c"), + RowFactory.create(3, "a"), + RowFactory.create(4, "a"), + RowFactory.create(5, "c") + ); + StructType schema = new StructType(new StructField[]{ + createStructField("id", IntegerType, false), + createStructField("category", StringType, false) + }); + Dataset df = spark.createDataFrame(data, schema); + + StringIndexer indexer = new StringIndexer() + .setInputCol("category") + .setOutputCol("categoryIndex"); + + Dataset indexed = indexer.fit(df).transform(df); + indexed.show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java index a41a5ec9bff0..b740cd097a9b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java @@ -19,19 +19,16 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.HashingTF; import org.apache.spark.ml.feature.IDF; import org.apache.spark.ml.feature.IDFModel; import org.apache.spark.ml.feature.Tokenizer; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; @@ -40,40 +37,42 @@ public class JavaTfIdfExample { public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaTfIdfExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaTfIdfExample") + .getOrCreate(); // $example on$ - JavaRDD jrdd = jsc.parallelize(Arrays.asList( - RowFactory.create(0, "Hi I heard about Spark"), - RowFactory.create(0, "I wish Java could use case classes"), - RowFactory.create(1, "Logistic regression models are neat") - )); + List data = Arrays.asList( + RowFactory.create(0.0, "Hi I heard about Spark"), + RowFactory.create(0.0, "I wish Java could use case classes"), + RowFactory.create(1.0, "Logistic regression models are neat") + ); StructType schema = new StructType(new StructField[]{ new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), new StructField("sentence", DataTypes.StringType, false, Metadata.empty()) }); - DataFrame sentenceData = sqlContext.createDataFrame(jrdd, schema); + Dataset sentenceData = spark.createDataFrame(data, schema); + Tokenizer tokenizer = new Tokenizer().setInputCol("sentence").setOutputCol("words"); - DataFrame wordsData = tokenizer.transform(sentenceData); + Dataset wordsData = tokenizer.transform(sentenceData); + int numFeatures = 20; HashingTF hashingTF = new HashingTF() .setInputCol("words") .setOutputCol("rawFeatures") .setNumFeatures(numFeatures); - DataFrame featurizedData = hashingTF.transform(wordsData); + + Dataset featurizedData = hashingTF.transform(wordsData); + // alternatively, CountVectorizer can also be used to get term frequency vectors + IDF idf = new IDF().setInputCol("rawFeatures").setOutputCol("features"); IDFModel idfModel = idf.fit(featurizedData); - DataFrame rescaledData = idfModel.transform(featurizedData); - for (Row r : rescaledData.select("features", "label").take(3)) { - Vector features = r.getAs(0); - Double label = r.getDouble(1); - System.out.println(features); - System.out.println(label); - } + + Dataset rescaledData = idfModel.transform(featurizedData); + rescaledData.select("label", "features").show(); // $example off$ - jsc.stop(); + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java new file mode 100644 index 000000000000..a0979aa2d24e --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import scala.collection.mutable.WrappedArray; + +import org.apache.spark.ml.feature.RegexTokenizer; +import org.apache.spark.ml.feature.Tokenizer; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +// col("...") is preferable to df.col("...") +import static org.apache.spark.sql.functions.callUDF; +import static org.apache.spark.sql.functions.col; +// $example off$ + +public class JavaTokenizerExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaTokenizerExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(0, "Hi I heard about Spark"), + RowFactory.create(1, "I wish Java could use case classes"), + RowFactory.create(2, "Logistic,regression,models,are,neat") + ); + + StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("sentence", DataTypes.StringType, false, Metadata.empty()) + }); + + Dataset sentenceDataFrame = spark.createDataFrame(data, schema); + + Tokenizer tokenizer = new Tokenizer().setInputCol("sentence").setOutputCol("words"); + + RegexTokenizer regexTokenizer = new RegexTokenizer() + .setInputCol("sentence") + .setOutputCol("words") + .setPattern("\\W"); // alternatively .setPattern("\\w+").setGaps(false); + + spark.udf().register( + "countTokens", (WrappedArray words) -> words.size(), DataTypes.IntegerType); + + Dataset tokenized = tokenizer.transform(sentenceDataFrame); + tokenized.select("sentence", "words") + .withColumn("tokens", callUDF("countTokens", col("words"))) + .show(false); + + Dataset regexTokenized = regexTokenizer.transform(sentenceDataFrame); + regexTokenized.select("sentence", "words") + .withColumn("tokens", callUDF("countTokens", col("words"))) + .show(false); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTrainValidationSplitExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTrainValidationSplitExample.java deleted file mode 100644 index 23f834ab4332..000000000000 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTrainValidationSplitExample.java +++ /dev/null @@ -1,90 +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.examples.ml; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.evaluation.RegressionEvaluator; -import org.apache.spark.ml.param.ParamMap; -import org.apache.spark.ml.regression.LinearRegression; -import org.apache.spark.ml.tuning.*; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; - -/** - * A simple example demonstrating model selection using TrainValidationSplit. - * - * The example is based on {@link org.apache.spark.examples.ml.JavaSimpleParamsExample} - * using linear regression. - * - * Run with - * {{{ - * bin/run-example ml.JavaTrainValidationSplitExample - * }}} - */ -public class JavaTrainValidationSplitExample { - - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("JavaTrainValidationSplitExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext jsql = new SQLContext(jsc); - - DataFrame data = jsql.createDataFrame( - MLUtils.loadLibSVMFile(jsc.sc(), "data/mllib/sample_libsvm_data.txt"), - LabeledPoint.class); - - // Prepare training and test data. - DataFrame[] splits = data.randomSplit(new double [] {0.9, 0.1}, 12345); - DataFrame training = splits[0]; - DataFrame test = splits[1]; - - LinearRegression lr = new LinearRegression(); - - // We use a ParamGridBuilder to construct a grid of parameters to search over. - // TrainValidationSplit will try all combinations of values and determine best model using - // the evaluator. - ParamMap[] paramGrid = new ParamGridBuilder() - .addGrid(lr.regParam(), new double[] {0.1, 0.01}) - .addGrid(lr.fitIntercept()) - .addGrid(lr.elasticNetParam(), new double[] {0.0, 0.5, 1.0}) - .build(); - - // In this case the estimator is simply the linear regression. - // A TrainValidationSplit requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. - TrainValidationSplit trainValidationSplit = new TrainValidationSplit() - .setEstimator(lr) - .setEvaluator(new RegressionEvaluator()) - .setEstimatorParamMaps(paramGrid); - - // 80% of the data will be used for training and the remaining 20% for validation. - trainValidationSplit.setTrainRatio(0.8); - - // Run train validation split, and choose the best set of parameters. - TrainValidationSplitModel model = trainValidationSplit.fit(training); - - // Make predictions on test data. model is the model with combination of parameters - // that performed best. - model.transform(test) - .select("features", "label", "prediction") - .show(); - - jsc.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java new file mode 100644 index 000000000000..384e09c73bed --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorAssemblerExample.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; + +import org.apache.spark.ml.feature.VectorAssembler; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.*; +import static org.apache.spark.sql.types.DataTypes.*; +// $example off$ + +public class JavaVectorAssemblerExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaVectorAssemblerExample") + .getOrCreate(); + + // $example on$ + StructType schema = createStructType(new StructField[]{ + createStructField("id", IntegerType, false), + createStructField("hour", IntegerType, false), + createStructField("mobile", DoubleType, false), + createStructField("userFeatures", new VectorUDT(), false), + createStructField("clicked", DoubleType, false) + }); + Row row = RowFactory.create(0, 18, 1.0, Vectors.dense(0.0, 10.0, 0.5), 1.0); + Dataset dataset = spark.createDataFrame(Arrays.asList(row), schema); + + VectorAssembler assembler = new VectorAssembler() + .setInputCols(new String[]{"hour", "mobile", "userFeatures"}) + .setOutputCol("features"); + + Dataset output = assembler.transform(dataset); + System.out.println("Assembled columns 'hour', 'mobile', 'userFeatures' to vector column " + + "'features'"); + output.select("features", "clicked").show(false); + // $example off$ + + spark.stop(); + } +} + diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java new file mode 100644 index 000000000000..dd9d757dd683 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorIndexerExample.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Map; + +import org.apache.spark.ml.feature.VectorIndexer; +import org.apache.spark.ml.feature.VectorIndexerModel; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off$ + +public class JavaVectorIndexerExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaVectorIndexerExample") + .getOrCreate(); + + // $example on$ + Dataset data = spark.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + + VectorIndexer indexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexed") + .setMaxCategories(10); + VectorIndexerModel indexerModel = indexer.fit(data); + + Map> categoryMaps = indexerModel.javaCategoryMaps(); + System.out.print("Chose " + categoryMaps.size() + " categorical features:"); + + for (Integer feature : categoryMaps.keySet()) { + System.out.print(" " + feature); + } + System.out.println(); + + // Create new column "indexed" with categorical values transformed to indices + Dataset indexedData = indexerModel.transform(data); + indexedData.show(); + // $example off$ + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java new file mode 100644 index 000000000000..1ae48be2660b --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaVectorSlicerExample.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import org.apache.spark.sql.SparkSession; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.attribute.Attribute; +import org.apache.spark.ml.attribute.AttributeGroup; +import org.apache.spark.ml.attribute.NumericAttribute; +import org.apache.spark.ml.feature.VectorSlicer; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.*; +// $example off$ + +public class JavaVectorSlicerExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaVectorSlicerExample") + .getOrCreate(); + + // $example on$ + Attribute[] attrs = { + NumericAttribute.defaultAttr().withName("f1"), + NumericAttribute.defaultAttr().withName("f2"), + NumericAttribute.defaultAttr().withName("f3") + }; + AttributeGroup group = new AttributeGroup("userFeatures", attrs); + + List data = Arrays.asList( + RowFactory.create(Vectors.sparse(3, new int[]{0, 1}, new double[]{-2.0, 2.3})), + RowFactory.create(Vectors.dense(-2.0, 2.3, 0.0)) + ); + + Dataset dataset = + spark.createDataFrame(data, (new StructType()).add(group.toStructField())); + + VectorSlicer vectorSlicer = new VectorSlicer() + .setInputCol("userFeatures").setOutputCol("features"); + + vectorSlicer.setIndices(new int[]{1}).setNames(new String[]{"f3"}); + // or slicer.setIndices(new int[]{1, 2}), or slicer.setNames(new String[]{"f2", "f3"}) + + Dataset output = vectorSlicer.transform(dataset); + output.show(false); + // $example off$ + + spark.stop(); + } +} + diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java index d472375ca982..fc9b45968874 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java @@ -19,37 +19,36 @@ // $example on$ import java.util.Arrays; +import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.Word2Vec; import org.apache.spark.ml.feature.Word2VecModel; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.*; // $example off$ public class JavaWord2VecExample { public static void main(String[] args) { - - SparkConf conf = new SparkConf().setAppName("JavaWord2VecExample"); - JavaSparkContext jsc = new JavaSparkContext(conf); - SQLContext sqlContext = new SQLContext(jsc); + SparkSession spark = SparkSession + .builder() + .appName("JavaWord2VecExample") + .getOrCreate(); // $example on$ // Input data: Each row is a bag of words from a sentence or document. - JavaRDD jrdd = jsc.parallelize(Arrays.asList( + List data = Arrays.asList( RowFactory.create(Arrays.asList("Hi I heard about Spark".split(" "))), RowFactory.create(Arrays.asList("I wish Java could use case classes".split(" "))), RowFactory.create(Arrays.asList("Logistic regression models are neat".split(" "))) - )); + ); StructType schema = new StructType(new StructField[]{ new StructField("text", new ArrayType(DataTypes.StringType, true), false, Metadata.empty()) }); - DataFrame documentDF = sqlContext.createDataFrame(jrdd, schema); + Dataset documentDF = spark.createDataFrame(data, schema); // Learn a mapping from words to Vectors. Word2Vec word2Vec = new Word2Vec() @@ -57,11 +56,17 @@ public static void main(String[] args) { .setOutputCol("result") .setVectorSize(3) .setMinCount(0); + Word2VecModel model = word2Vec.fit(documentDF); - DataFrame result = model.transform(documentDF); - for (Row r : result.select("result").take(3)) { - System.out.println(r); + Dataset result = model.transform(documentDF); + + for (Row row : result.collectAsList()) { + List text = row.getList(0); + Vector vector = (Vector) row.get(1); + System.out.println("Text: " + text + " => \nVector: " + vector + "\n"); } // $example off$ + + spark.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaAssociationRulesExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaAssociationRulesExample.java index 4d0f989819ac..5f43603f4ff5 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaAssociationRulesExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaAssociationRulesExample.java @@ -38,9 +38,9 @@ public static void main(String[] args) { // $example on$ JavaRDD> freqItemsets = sc.parallelize(Arrays.asList( - new FreqItemset(new String[] {"a"}, 15L), - new FreqItemset(new String[] {"b"}, 35L), - new FreqItemset(new String[] {"a", "b"}, 12L) + new FreqItemset<>(new String[] {"a"}, 15L), + new FreqItemset<>(new String[] {"b"}, 35L), + new FreqItemset<>(new String[] {"a", "b"}, 12L) )); AssociationRules arules = new AssociationRules() @@ -52,5 +52,7 @@ public static void main(String[] args) { rule.javaAntecedent() + " => " + rule.javaConsequent() + ", " + rule.confidence()); } // $example off$ + + sc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaBinaryClassificationMetricsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaBinaryClassificationMetricsExample.java new file mode 100644 index 000000000000..b9d0313c6bb5 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaBinaryClassificationMetricsExample.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +// $example on$ +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.mllib.classification.LogisticRegressionModel; +import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS; +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +// $example off$ +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; + +public class JavaBinaryClassificationMetricsExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("Java Binary Classification Metrics Example"); + SparkContext sc = new SparkContext(conf); + // $example on$ + String path = "data/mllib/sample_binary_classification_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); + + // Split initial RDD into two... [60% training data, 40% testing data]. + JavaRDD[] splits = + data.randomSplit(new double[]{0.6, 0.4}, 11L); + JavaRDD training = splits[0].cache(); + JavaRDD test = splits[1]; + + // Run training algorithm to build the model. + LogisticRegressionModel model = new LogisticRegressionWithLBFGS() + .setNumClasses(2) + .run(training.rdd()); + + // Clear the prediction threshold so the model will return probabilities + model.clearThreshold(); + + // Compute raw scores on the test set. + JavaPairRDD predictionAndLabels = test.mapToPair(p -> + new Tuple2<>(model.predict(p.features()), p.label())); + + // Get evaluation metrics. + BinaryClassificationMetrics metrics = + new BinaryClassificationMetrics(predictionAndLabels.rdd()); + + // Precision by threshold + JavaRDD> precision = metrics.precisionByThreshold().toJavaRDD(); + System.out.println("Precision by threshold: " + precision.collect()); + + // Recall by threshold + JavaRDD recall = metrics.recallByThreshold().toJavaRDD(); + System.out.println("Recall by threshold: " + recall.collect()); + + // F Score by threshold + JavaRDD f1Score = metrics.fMeasureByThreshold().toJavaRDD(); + System.out.println("F1 Score by threshold: " + f1Score.collect()); + + JavaRDD f2Score = metrics.fMeasureByThreshold(2.0).toJavaRDD(); + System.out.println("F2 Score by threshold: " + f2Score.collect()); + + // Precision-recall curve + JavaRDD prc = metrics.pr().toJavaRDD(); + System.out.println("Precision-recall curve: " + prc.collect()); + + // Thresholds + JavaRDD thresholds = precision.map(t -> Double.parseDouble(t._1().toString())); + + // ROC Curve + JavaRDD roc = metrics.roc().toJavaRDD(); + System.out.println("ROC curve: " + roc.collect()); + + // AUPRC + System.out.println("Area under precision-recall curve = " + metrics.areaUnderPR()); + + // AUROC + System.out.println("Area under ROC = " + metrics.areaUnderROC()); + + // Save and load model + model.save(sc, "target/tmp/LogisticRegressionModel"); + LogisticRegressionModel.load(sc, "target/tmp/LogisticRegressionModel"); + // $example off$ + + sc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaBisectingKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaBisectingKMeansExample.java new file mode 100644 index 000000000000..f878b55a98ad --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaBisectingKMeansExample.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +// $example on$ +import java.util.Arrays; +import java.util.List; +// $example off$ +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +// $example on$ +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.clustering.BisectingKMeans; +import org.apache.spark.mllib.clustering.BisectingKMeansModel; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +// $example off$ + +/** + * Java example for bisecting k-means clustering. + */ +public class JavaBisectingKMeansExample { + public static void main(String[] args) { + SparkConf sparkConf = new SparkConf().setAppName("JavaBisectingKMeansExample"); + JavaSparkContext sc = new JavaSparkContext(sparkConf); + + // $example on$ + List localData = Arrays.asList( + Vectors.dense(0.1, 0.1), Vectors.dense(0.3, 0.3), + Vectors.dense(10.1, 10.1), Vectors.dense(10.3, 10.3), + Vectors.dense(20.1, 20.1), Vectors.dense(20.3, 20.3), + Vectors.dense(30.1, 30.1), Vectors.dense(30.3, 30.3) + ); + JavaRDD data = sc.parallelize(localData, 2); + + BisectingKMeans bkm = new BisectingKMeans() + .setK(4); + BisectingKMeansModel model = bkm.run(data); + + System.out.println("Compute Cost: " + model.computeCost(data)); + + Vector[] clusterCenters = model.clusterCenters(); + for (int i = 0; i < clusterCenters.length; i++) { + Vector clusterCenter = clusterCenters[i]; + System.out.println("Cluster Center " + i + ": " + clusterCenter); + } + // $example off$ + + sc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaChiSqSelectorExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaChiSqSelectorExample.java new file mode 100644 index 000000000000..ce354af2b579 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaChiSqSelectorExample.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +// $example on$ +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.feature.ChiSqSelector; +import org.apache.spark.mllib.feature.ChiSqSelectorModel; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +// $example off$ + +public class JavaChiSqSelectorExample { + public static void main(String[] args) { + + SparkConf conf = new SparkConf().setAppName("JavaChiSqSelectorExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + + // $example on$ + JavaRDD points = MLUtils.loadLibSVMFile(jsc.sc(), + "data/mllib/sample_libsvm_data.txt").toJavaRDD().cache(); + + // Discretize data in 16 equal bins since ChiSqSelector requires categorical features + // Although features are doubles, the ChiSqSelector treats each unique value as a category + JavaRDD discretizedData = points.map(lp -> { + double[] discretizedFeatures = new double[lp.features().size()]; + for (int i = 0; i < lp.features().size(); ++i) { + discretizedFeatures[i] = Math.floor(lp.features().apply(i) / 16); + } + return new LabeledPoint(lp.label(), Vectors.dense(discretizedFeatures)); + }); + + // Create ChiSqSelector that will select top 50 of 692 features + ChiSqSelector selector = new ChiSqSelector(50); + // Create ChiSqSelector model (selecting features) + ChiSqSelectorModel transformer = selector.fit(discretizedData.rdd()); + // Filter the top 50 features from each feature vector + JavaRDD filteredData = discretizedData.map(lp -> + new LabeledPoint(lp.label(), transformer.transform(lp.features()))); + // $example off$ + + System.out.println("filtered data: "); + filteredData.foreach(System.out::println); + + jsc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaCorrelationsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaCorrelationsExample.java new file mode 100644 index 000000000000..c0fa0b3cac1e --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaCorrelationsExample.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +// $example on$ +import java.util.Arrays; + +import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.stat.Statistics; +// $example off$ + +public class JavaCorrelationsExample { + public static void main(String[] args) { + + SparkConf conf = new SparkConf().setAppName("JavaCorrelationsExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + + // $example on$ + JavaDoubleRDD seriesX = jsc.parallelizeDoubles( + Arrays.asList(1.0, 2.0, 3.0, 3.0, 5.0)); // a series + + // must have the same number of partitions and cardinality as seriesX + JavaDoubleRDD seriesY = jsc.parallelizeDoubles( + Arrays.asList(11.0, 22.0, 33.0, 33.0, 555.0)); + + // compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. + // If a method is not specified, Pearson's method will be used by default. + Double correlation = Statistics.corr(seriesX.srdd(), seriesY.srdd(), "pearson"); + System.out.println("Correlation is: " + correlation); + + // note that each Vector is a row and not a column + JavaRDD data = jsc.parallelize( + Arrays.asList( + Vectors.dense(1.0, 10.0, 100.0), + Vectors.dense(2.0, 20.0, 200.0), + Vectors.dense(5.0, 33.0, 366.0) + ) + ); + + // calculate the correlation matrix using Pearson's method. + // Use "spearman" for Spearman's method. + // If a method is not specified, Pearson's method will be used by default. + Matrix correlMatrix = Statistics.corr(data.rdd(), "pearson"); + System.out.println(correlMatrix.toString()); + // $example off$ + + jsc.stop(); + } +} + diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java deleted file mode 100644 index 1f82e3f4cb18..000000000000 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java +++ /dev/null @@ -1,116 +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.examples.mllib; - -import java.util.HashMap; - -import scala.Tuple2; - -import org.apache.spark.api.java.function.Function2; -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.LabeledPoint; -import org.apache.spark.mllib.tree.DecisionTree; -import org.apache.spark.mllib.tree.model.DecisionTreeModel; -import org.apache.spark.mllib.util.MLUtils; -import org.apache.spark.SparkConf; - -/** - * Classification and regression using decision trees. - */ -public final class JavaDecisionTree { - - public static void main(String[] args) { - String datapath = "data/mllib/sample_libsvm_data.txt"; - if (args.length == 1) { - datapath = args[0]; - } else if (args.length > 1) { - System.err.println("Usage: JavaDecisionTree "); - System.exit(1); - } - SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); - JavaSparkContext sc = new JavaSparkContext(sparkConf); - - JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); - - // Compute the number of classes from the data. - Integer numClasses = data.map(new Function() { - @Override public Double call(LabeledPoint p) { - return p.label(); - } - }).countByValue().size(); - - // Set parameters. - // Empty categoricalFeaturesInfo indicates all features are continuous. - HashMap categoricalFeaturesInfo = new HashMap(); - String impurity = "gini"; - Integer maxDepth = 5; - Integer maxBins = 32; - - // Train a DecisionTree model for classification. - final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, - categoricalFeaturesInfo, impurity, maxDepth, maxBins); - - // Evaluate model on training instances and compute training error - JavaPairRDD predictionAndLabel = - data.mapToPair(new PairFunction() { - @Override public Tuple2 call(LabeledPoint p) { - return new Tuple2(model.predict(p.features()), p.label()); - } - }); - Double trainErr = - 1.0 * predictionAndLabel.filter(new Function, Boolean>() { - @Override public Boolean call(Tuple2 pl) { - return !pl._1().equals(pl._2()); - } - }).count() / data.count(); - System.out.println("Training error: " + trainErr); - System.out.println("Learned classification tree model:\n" + model); - - // Train a DecisionTree model for regression. - impurity = "variance"; - final DecisionTreeModel regressionModel = DecisionTree.trainRegressor(data, - categoricalFeaturesInfo, impurity, maxDepth, maxBins); - - // Evaluate model on training instances and compute training error - JavaPairRDD regressorPredictionAndLabel = - data.mapToPair(new PairFunction() { - @Override public Tuple2 call(LabeledPoint p) { - return new Tuple2(regressionModel.predict(p.features()), p.label()); - } - }); - Double trainMSE = - regressorPredictionAndLabel.map(new Function, Double>() { - @Override public Double call(Tuple2 pl) { - Double diff = pl._1() - pl._2(); - return diff * diff; - } - }).reduce(new Function2() { - @Override public Double call(Double a, Double b) { - return a + b; - } - }) / data.count(); - System.out.println("Training Mean Squared Error: " + trainMSE); - System.out.println("Learned regression tree model:\n" + regressionModel); - - sc.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTreeClassificationExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTreeClassificationExample.java new file mode 100644 index 000000000000..032c168b946d --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTreeClassificationExample.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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; + +// $example on$ +import java.util.HashMap; +import java.util.Map; + +import scala.Tuple2; + +import org.apache.spark.SparkConf; +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.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.DecisionTree; +import org.apache.spark.mllib.tree.model.DecisionTreeModel; +import org.apache.spark.mllib.util.MLUtils; +// $example off$ + +class JavaDecisionTreeClassificationExample { + + public static void main(String[] args) { + + // $example on$ + SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTreeClassificationExample"); + JavaSparkContext jsc = new JavaSparkContext(sparkConf); + + // Load and parse the data file. + String datapath = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(jsc.sc(), datapath).toJavaRDD(); + // Split the data into training and test sets (30% held out for testing) + JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); + JavaRDD trainingData = splits[0]; + JavaRDD testData = splits[1]; + + // Set parameters. + // Empty categoricalFeaturesInfo indicates all features are continuous. + int numClasses = 2; + Map categoricalFeaturesInfo = new HashMap<>(); + String impurity = "gini"; + int maxDepth = 5; + int maxBins = 32; + + // Train a DecisionTree model for classification. + DecisionTreeModel model = DecisionTree.trainClassifier(trainingData, numClasses, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + + // Evaluate model on test instances and compute test error + JavaPairRDD predictionAndLabel = + testData.mapToPair(p -> new Tuple2<>(model.predict(p.features()), p.label())); + double testErr = + predictionAndLabel.filter(pl -> !pl._1().equals(pl._2())).count() / (double) testData.count(); + + System.out.println("Test Error: " + testErr); + System.out.println("Learned classification tree model:\n" + model.toDebugString()); + + // Save and load model + model.save(jsc.sc(), "target/tmp/myDecisionTreeClassificationModel"); + DecisionTreeModel sameModel = DecisionTreeModel + .load(jsc.sc(), "target/tmp/myDecisionTreeClassificationModel"); + // $example off$ + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTreeRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTreeRegressionExample.java new file mode 100644 index 000000000000..f222c38fc82b --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTreeRegressionExample.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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; + +// $example on$ +import java.util.HashMap; +import java.util.Map; + +import scala.Tuple2; + +import org.apache.spark.SparkConf; +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.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.DecisionTree; +import org.apache.spark.mllib.tree.model.DecisionTreeModel; +import org.apache.spark.mllib.util.MLUtils; +// $example off$ + +class JavaDecisionTreeRegressionExample { + + public static void main(String[] args) { + + // $example on$ + SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTreeRegressionExample"); + JavaSparkContext jsc = new JavaSparkContext(sparkConf); + + // Load and parse the data file. + String datapath = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(jsc.sc(), datapath).toJavaRDD(); + // Split the data into training and test sets (30% held out for testing) + JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); + JavaRDD trainingData = splits[0]; + JavaRDD testData = splits[1]; + + // Set parameters. + // Empty categoricalFeaturesInfo indicates all features are continuous. + Map categoricalFeaturesInfo = new HashMap<>(); + String impurity = "variance"; + int maxDepth = 5; + int maxBins = 32; + + // Train a DecisionTree model. + DecisionTreeModel model = DecisionTree.trainRegressor(trainingData, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + + // Evaluate model on test instances and compute test error + JavaPairRDD predictionAndLabel = + testData.mapToPair(p -> new Tuple2<>(model.predict(p.features()), p.label())); + double testMSE = predictionAndLabel.mapToDouble(pl -> { + double diff = pl._1() - pl._2(); + return diff * diff; + }).mean(); + System.out.println("Test Mean Squared Error: " + testMSE); + System.out.println("Learned regression tree model:\n" + model.toDebugString()); + + // Save and load model + model.save(jsc.sc(), "target/tmp/myDecisionTreeRegressionModel"); + DecisionTreeModel sameModel = DecisionTreeModel + .load(jsc.sc(), "target/tmp/myDecisionTreeRegressionModel"); + // $example off$ + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaElementwiseProductExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaElementwiseProductExample.java new file mode 100644 index 000000000000..2d45c6166fee --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaElementwiseProductExample.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +// $example on$ +import java.util.Arrays; +// $example off$ + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +// $example on$ +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.feature.ElementwiseProduct; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +// $example off$ + +public class JavaElementwiseProductExample { + public static void main(String[] args) { + + SparkConf conf = new SparkConf().setAppName("JavaElementwiseProductExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + + // $example on$ + // Create some vector data; also works for sparse vectors + JavaRDD data = jsc.parallelize(Arrays.asList( + Vectors.dense(1.0, 2.0, 3.0), Vectors.dense(4.0, 5.0, 6.0))); + Vector transformingVector = Vectors.dense(0.0, 1.0, 2.0); + ElementwiseProduct transformer = new ElementwiseProduct(transformingVector); + + // Batch transform and per-row transform give the same results: + JavaRDD transformedData = transformer.transform(data); + JavaRDD transformedData2 = data.map(transformer::transform); + // $example off$ + + System.out.println("transformedData: "); + transformedData.foreach(System.out::println); + + System.out.println("transformedData2: "); + transformedData2.foreach(System.out::println); + + jsc.stop(); + } +} 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 deleted file mode 100644 index 36baf5868736..000000000000 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java +++ /dev/null @@ -1,78 +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.examples.mllib; - -import java.util.ArrayList; - -import com.google.common.base.Joiner; -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.api.java.function.Function; -import org.apache.spark.mllib.fpm.FPGrowth; -import org.apache.spark.mllib.fpm.FPGrowthModel; - -/** - * Java example for mining frequent itemsets using FP-growth. - * Example usage: ./bin/run-example mllib.JavaFPGrowthExample ./data/mllib/sample_fpgrowth.txt - */ -public class JavaFPGrowthExample { - - public static void main(String[] args) { - String inputFile; - double minSupport = 0.3; - int numPartition = -1; - if (args.length < 1) { - System.err.println( - "Usage: JavaFPGrowth [minSupport] [numPartition]"); - System.exit(1); - } - inputFile = args[0]; - if (args.length >= 2) { - minSupport = Double.parseDouble(args[1]); - } - if (args.length >= 3) { - numPartition = Integer.parseInt(args[2]); - } - - SparkConf sparkConf = new SparkConf().setAppName("JavaFPGrowthExample"); - JavaSparkContext sc = new JavaSparkContext(sparkConf); - - JavaRDD> transactions = sc.textFile(inputFile).map( - new Function>() { - @Override - public ArrayList call(String s) { - return Lists.newArrayList(s.split(" ")); - } - } - ); - - FPGrowthModel model = new FPGrowth() - .setMinSupport(minSupport) - .setNumPartitions(numPartition) - .run(transactions); - - for (FPGrowth.FreqItemset s: model.freqItemsets().toJavaRDD().collect()) { - System.out.println("[" + Joiner.on(",").join(s.javaItems()) + "], " + s.freq()); - } - - sc.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaGaussianMixtureExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGaussianMixtureExample.java new file mode 100644 index 000000000000..5792e5a71cb0 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGaussianMixtureExample.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; + +// $example on$ +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.clustering.GaussianMixture; +import org.apache.spark.mllib.clustering.GaussianMixtureModel; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +// $example off$ + +public class JavaGaussianMixtureExample { + public static void main(String[] args) { + + SparkConf conf = new SparkConf().setAppName("JavaGaussianMixtureExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + + // $example on$ + // Load and parse data + String path = "data/mllib/gmm_data.txt"; + JavaRDD data = jsc.textFile(path); + JavaRDD parsedData = data.map(s -> { + String[] sarray = s.trim().split(" "); + double[] values = new double[sarray.length]; + for (int i = 0; i < sarray.length; i++) { + values[i] = Double.parseDouble(sarray[i]); + } + return Vectors.dense(values); + }); + parsedData.cache(); + + // Cluster the data into two classes using GaussianMixture + GaussianMixtureModel gmm = new GaussianMixture().setK(2).run(parsedData.rdd()); + + // Save and load GaussianMixtureModel + gmm.save(jsc.sc(), "target/org/apache/spark/JavaGaussianMixtureExample/GaussianMixtureModel"); + GaussianMixtureModel sameModel = GaussianMixtureModel.load(jsc.sc(), + "target/org.apache.spark.JavaGaussianMixtureExample/GaussianMixtureModel"); + + // Output the parameters of the mixture model + for (int j = 0; j < gmm.k(); j++) { + System.out.printf("weight=%f\nmu=%s\nsigma=\n%s\n", + gmm.weights()[j], gmm.gaussians()[j].mu(), gmm.gaussians()[j].sigma()); + } + // $example off$ + + jsc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTreesRunner.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTreesRunner.java deleted file mode 100644 index a1844d5d07ad..000000000000 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTreesRunner.java +++ /dev/null @@ -1,126 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.examples.mllib; - -import scala.Tuple2; - -import org.apache.spark.SparkConf; -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.Function2; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.tree.GradientBoostedTrees; -import org.apache.spark.mllib.tree.configuration.BoostingStrategy; -import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel; -import org.apache.spark.mllib.util.MLUtils; - -/** - * Classification and regression using gradient-boosted decision trees. - */ -public final class JavaGradientBoostedTreesRunner { - - private static void usage() { - System.err.println("Usage: JavaGradientBoostedTreesRunner " + - " "); - System.exit(-1); - } - - public static void main(String[] args) { - String datapath = "data/mllib/sample_libsvm_data.txt"; - String algo = "Classification"; - if (args.length >= 1) { - datapath = args[0]; - } - if (args.length >= 2) { - algo = args[1]; - } - if (args.length > 2) { - usage(); - } - SparkConf sparkConf = new SparkConf().setAppName("JavaGradientBoostedTreesRunner"); - JavaSparkContext sc = new JavaSparkContext(sparkConf); - - JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); - - // Set parameters. - // Note: All features are treated as continuous. - BoostingStrategy boostingStrategy = BoostingStrategy.defaultParams(algo); - boostingStrategy.setNumIterations(10); - boostingStrategy.treeStrategy().setMaxDepth(5); - - if (algo.equals("Classification")) { - // Compute the number of classes from the data. - Integer numClasses = data.map(new Function() { - @Override public Double call(LabeledPoint p) { - return p.label(); - } - }).countByValue().size(); - boostingStrategy.treeStrategy().setNumClasses(numClasses); - - // Train a GradientBoosting model for classification. - final GradientBoostedTreesModel model = GradientBoostedTrees.train(data, boostingStrategy); - - // Evaluate model on training instances and compute training error - JavaPairRDD predictionAndLabel = - data.mapToPair(new PairFunction() { - @Override public Tuple2 call(LabeledPoint p) { - return new Tuple2(model.predict(p.features()), p.label()); - } - }); - Double trainErr = - 1.0 * predictionAndLabel.filter(new Function, Boolean>() { - @Override public Boolean call(Tuple2 pl) { - return !pl._1().equals(pl._2()); - } - }).count() / data.count(); - System.out.println("Training error: " + trainErr); - System.out.println("Learned classification tree model:\n" + model); - } else if (algo.equals("Regression")) { - // Train a GradientBoosting model for classification. - final GradientBoostedTreesModel model = GradientBoostedTrees.train(data, boostingStrategy); - - // Evaluate model on training instances and compute training error - JavaPairRDD predictionAndLabel = - data.mapToPair(new PairFunction() { - @Override public Tuple2 call(LabeledPoint p) { - return new Tuple2(model.predict(p.features()), p.label()); - } - }); - Double trainMSE = - predictionAndLabel.map(new Function, Double>() { - @Override public Double call(Tuple2 pl) { - Double diff = pl._1() - pl._2(); - return diff * diff; - } - }).reduce(new Function2() { - @Override public Double call(Double a, Double b) { - return a + b; - } - }) / data.count(); - System.out.println("Training Mean Squared Error: " + trainMSE); - System.out.println("Learned regression tree model:\n" + model); - } else { - usage(); - } - - sc.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostingClassificationExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostingClassificationExample.java new file mode 100644 index 000000000000..521ee96fbdf4 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostingClassificationExample.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +// $example on$ +import java.util.HashMap; +import java.util.Map; + +import scala.Tuple2; + +import org.apache.spark.SparkConf; +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.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.GradientBoostedTrees; +import org.apache.spark.mllib.tree.configuration.BoostingStrategy; +import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel; +import org.apache.spark.mllib.util.MLUtils; +// $example off$ + +public class JavaGradientBoostingClassificationExample { + public static void main(String[] args) { + // $example on$ + SparkConf sparkConf = new SparkConf() + .setAppName("JavaGradientBoostedTreesClassificationExample"); + JavaSparkContext jsc = new JavaSparkContext(sparkConf); + + // Load and parse the data file. + String datapath = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(jsc.sc(), datapath).toJavaRDD(); + // Split the data into training and test sets (30% held out for testing) + JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); + JavaRDD trainingData = splits[0]; + JavaRDD testData = splits[1]; + + // Train a GradientBoostedTrees model. + // The defaultParams for Classification use LogLoss by default. + BoostingStrategy boostingStrategy = BoostingStrategy.defaultParams("Classification"); + boostingStrategy.setNumIterations(3); // Note: Use more iterations in practice. + boostingStrategy.getTreeStrategy().setNumClasses(2); + boostingStrategy.getTreeStrategy().setMaxDepth(5); + // Empty categoricalFeaturesInfo indicates all features are continuous. + Map categoricalFeaturesInfo = new HashMap<>(); + boostingStrategy.treeStrategy().setCategoricalFeaturesInfo(categoricalFeaturesInfo); + + GradientBoostedTreesModel model = GradientBoostedTrees.train(trainingData, boostingStrategy); + + // Evaluate model on test instances and compute test error + JavaPairRDD predictionAndLabel = + testData.mapToPair(p -> new Tuple2<>(model.predict(p.features()), p.label())); + double testErr = + predictionAndLabel.filter(pl -> !pl._1().equals(pl._2())).count() / (double) testData.count(); + System.out.println("Test Error: " + testErr); + System.out.println("Learned classification GBT model:\n" + model.toDebugString()); + + // Save and load model + model.save(jsc.sc(), "target/tmp/myGradientBoostingClassificationModel"); + GradientBoostedTreesModel sameModel = GradientBoostedTreesModel.load(jsc.sc(), + "target/tmp/myGradientBoostingClassificationModel"); + // $example off$ + + jsc.stop(); + } + +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostingRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostingRegressionExample.java new file mode 100644 index 000000000000..b345d19f59ab --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostingRegressionExample.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +// $example on$ +import java.util.HashMap; +import java.util.Map; + +import scala.Tuple2; + +import org.apache.spark.SparkConf; +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.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.GradientBoostedTrees; +import org.apache.spark.mllib.tree.configuration.BoostingStrategy; +import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel; +import org.apache.spark.mllib.util.MLUtils; +// $example off$ + +public class JavaGradientBoostingRegressionExample { + public static void main(String[] args) { + // $example on$ + SparkConf sparkConf = new SparkConf() + .setAppName("JavaGradientBoostedTreesRegressionExample"); + JavaSparkContext jsc = new JavaSparkContext(sparkConf); + // Load and parse the data file. + String datapath = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(jsc.sc(), datapath).toJavaRDD(); + // Split the data into training and test sets (30% held out for testing) + JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); + JavaRDD trainingData = splits[0]; + JavaRDD testData = splits[1]; + + // Train a GradientBoostedTrees model. + // The defaultParams for Regression use SquaredError by default. + BoostingStrategy boostingStrategy = BoostingStrategy.defaultParams("Regression"); + boostingStrategy.setNumIterations(3); // Note: Use more iterations in practice. + boostingStrategy.getTreeStrategy().setMaxDepth(5); + // Empty categoricalFeaturesInfo indicates all features are continuous. + Map categoricalFeaturesInfo = new HashMap<>(); + boostingStrategy.treeStrategy().setCategoricalFeaturesInfo(categoricalFeaturesInfo); + + GradientBoostedTreesModel model = GradientBoostedTrees.train(trainingData, boostingStrategy); + + // Evaluate model on test instances and compute test error + JavaPairRDD predictionAndLabel = + testData.mapToPair(p -> new Tuple2<>(model.predict(p.features()), p.label())); + double testMSE = predictionAndLabel.mapToDouble(pl -> { + double diff = pl._1() - pl._2(); + return diff * diff; + }).mean(); + System.out.println("Test Mean Squared Error: " + testMSE); + System.out.println("Learned regression GBT model:\n" + model.toDebugString()); + + // Save and load model + model.save(jsc.sc(), "target/tmp/myGradientBoostingRegressionModel"); + GradientBoostedTreesModel sameModel = GradientBoostedTreesModel.load(jsc.sc(), + "target/tmp/myGradientBoostingRegressionModel"); + // $example off$ + + jsc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaHypothesisTestingExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaHypothesisTestingExample.java new file mode 100644 index 000000000000..b48b95ff1d2a --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaHypothesisTestingExample.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; + +// $example on$ +import java.util.Arrays; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.linalg.Matrices; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.stat.Statistics; +import org.apache.spark.mllib.stat.test.ChiSqTestResult; +// $example off$ + +public class JavaHypothesisTestingExample { + public static void main(String[] args) { + + SparkConf conf = new SparkConf().setAppName("JavaHypothesisTestingExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + + // $example on$ + // a vector composed of the frequencies of events + Vector vec = Vectors.dense(0.1, 0.15, 0.2, 0.3, 0.25); + + // compute the goodness of fit. If a second vector to test against is not supplied + // as a parameter, the test runs against a uniform distribution. + ChiSqTestResult goodnessOfFitTestResult = Statistics.chiSqTest(vec); + // summary of the test including the p-value, degrees of freedom, test statistic, + // the method used, and the null hypothesis. + System.out.println(goodnessOfFitTestResult + "\n"); + + // Create a contingency matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) + Matrix mat = Matrices.dense(3, 2, new double[]{1.0, 3.0, 5.0, 2.0, 4.0, 6.0}); + + // conduct Pearson's independence test on the input contingency matrix + ChiSqTestResult independenceTestResult = Statistics.chiSqTest(mat); + // summary of the test including the p-value, degrees of freedom... + System.out.println(independenceTestResult + "\n"); + + // an RDD of labeled points + JavaRDD obs = jsc.parallelize( + Arrays.asList( + new LabeledPoint(1.0, Vectors.dense(1.0, 0.0, 3.0)), + new LabeledPoint(1.0, Vectors.dense(1.0, 2.0, 0.0)), + new LabeledPoint(-1.0, Vectors.dense(-1.0, 0.0, -0.5)) + ) + ); + + // The contingency table is constructed from the raw (feature, label) pairs and used to conduct + // the independence test. Returns an array containing the ChiSquaredTestResult for every feature + // against the label. + ChiSqTestResult[] featureTestResults = Statistics.chiSqTest(obs.rdd()); + int i = 1; + for (ChiSqTestResult result : featureTestResults) { + System.out.println("Column " + i + ":"); + System.out.println(result + "\n"); // summary of the test + i++; + } + // $example off$ + + jsc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaHypothesisTestingKolmogorovSmirnovTestExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaHypothesisTestingKolmogorovSmirnovTestExample.java new file mode 100644 index 000000000000..fe611c9ae67c --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaHypothesisTestingKolmogorovSmirnovTestExample.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +// $example on$ +import java.util.Arrays; + +import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.mllib.stat.Statistics; +import org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult; +// $example off$ + +public class JavaHypothesisTestingKolmogorovSmirnovTestExample { + public static void main(String[] args) { + + SparkConf conf = + new SparkConf().setAppName("JavaHypothesisTestingKolmogorovSmirnovTestExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + + // $example on$ + JavaDoubleRDD data = jsc.parallelizeDoubles(Arrays.asList(0.1, 0.15, 0.2, 0.3, 0.25)); + KolmogorovSmirnovTestResult testResult = + Statistics.kolmogorovSmirnovTest(data, "norm", 0.0, 1.0); + // summary of the test including the p-value, test statistic, and null hypothesis + // if our p-value indicates significance, we can reject the null hypothesis + System.out.println(testResult); + // $example off$ + + jsc.stop(); + } +} + diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaIsotonicRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaIsotonicRegressionExample.java index 37e709b4cbc0..adebafe4b89d 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaIsotonicRegressionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaIsotonicRegressionExample.java @@ -17,16 +17,16 @@ package org.apache.spark.examples.mllib; // $example on$ + import scala.Tuple2; import scala.Tuple3; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.api.java.JavaDoubleRDD; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.mllib.regression.IsotonicRegression; import org.apache.spark.mllib.regression.IsotonicRegressionModel; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; // $example off$ import org.apache.spark.SparkConf; @@ -35,52 +35,40 @@ public static void main(String[] args) { SparkConf sparkConf = new SparkConf().setAppName("JavaIsotonicRegressionExample"); JavaSparkContext jsc = new JavaSparkContext(sparkConf); // $example on$ - JavaRDD data = jsc.textFile("data/mllib/sample_isotonic_regression_data.txt"); + JavaRDD data = MLUtils.loadLibSVMFile( + jsc.sc(), "data/mllib/sample_isotonic_regression_libsvm_data.txt").toJavaRDD(); // 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); - } - } - ); + JavaRDD> parsedData = data.map(point -> + new Tuple3<>(point.label(), point.features().apply(0), 1.0)); // Split data into training (60%) and test (40%) sets. - JavaRDD>[] splits = parsedData.randomSplit(new double[]{0.6, 0.4}, 11L); + 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 - final IsotonicRegressionModel model = new IsotonicRegression().setIsotonic(true).run(training); + 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()); - } - } - ); + JavaPairRDD predictionAndLabel = test.mapToPair(point -> + new Tuple2<>(model.predict(point._2()), 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(); + double meanSquaredError = predictionAndLabel.mapToDouble(pl -> { + double diff = pl._1() - pl._2(); + return diff * diff; + }).mean(); System.out.println("Mean Squared Error = " + meanSquaredError); // Save and load model model.save(jsc.sc(), "target/tmp/myIsotonicRegressionModel"); - IsotonicRegressionModel sameModel = IsotonicRegressionModel.load(jsc.sc(), "target/tmp/myIsotonicRegressionModel"); + IsotonicRegressionModel sameModel = + IsotonicRegressionModel.load(jsc.sc(), "target/tmp/myIsotonicRegressionModel"); // $example off$ + + jsc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java deleted file mode 100644 index e575eedeb465..000000000000 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java +++ /dev/null @@ -1,82 +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.examples.mllib; - -import java.util.regex.Pattern; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; - -import org.apache.spark.mllib.clustering.KMeans; -import org.apache.spark.mllib.clustering.KMeansModel; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; - -/** - * Example using MLlib KMeans from Java. - */ -public final class JavaKMeans { - - private static class ParsePoint implements Function { - private static final Pattern SPACE = Pattern.compile(" "); - - @Override - public Vector call(String line) { - String[] tok = SPACE.split(line); - double[] point = new double[tok.length]; - for (int i = 0; i < tok.length; ++i) { - point[i] = Double.parseDouble(tok[i]); - } - return Vectors.dense(point); - } - } - - public static void main(String[] args) { - if (args.length < 3) { - System.err.println( - "Usage: JavaKMeans []"); - System.exit(1); - } - String inputFile = args[0]; - int k = Integer.parseInt(args[1]); - int iterations = Integer.parseInt(args[2]); - int runs = 1; - - if (args.length >= 4) { - runs = Integer.parseInt(args[3]); - } - SparkConf sparkConf = new SparkConf().setAppName("JavaKMeans"); - JavaSparkContext sc = new JavaSparkContext(sparkConf); - JavaRDD lines = sc.textFile(inputFile); - - JavaRDD points = lines.map(new ParsePoint()); - - KMeansModel model = KMeans.train(points.rdd(), k, iterations, runs, KMeans.K_MEANS_PARALLEL()); - - System.out.println("Cluster centers:"); - for (Vector center : model.clusterCenters()) { - System.out.println(" " + center); - } - double cost = model.computeCost(points.rdd()); - System.out.println("Cost: " + cost); - - sc.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeansExample.java new file mode 100644 index 000000000000..f17275617ad5 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeansExample.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; + +// $example on$ +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.clustering.KMeans; +import org.apache.spark.mllib.clustering.KMeansModel; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +// $example off$ + +public class JavaKMeansExample { + public static void main(String[] args) { + + SparkConf conf = new SparkConf().setAppName("JavaKMeansExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + + // $example on$ + // Load and parse data + String path = "data/mllib/kmeans_data.txt"; + JavaRDD data = jsc.textFile(path); + JavaRDD parsedData = data.map(s -> { + String[] sarray = s.split(" "); + double[] values = new double[sarray.length]; + for (int i = 0; i < sarray.length; i++) { + values[i] = Double.parseDouble(sarray[i]); + } + return Vectors.dense(values); + }); + parsedData.cache(); + + // Cluster the data into two classes using KMeans + int numClusters = 2; + int numIterations = 20; + KMeansModel clusters = KMeans.train(parsedData.rdd(), numClusters, numIterations); + + System.out.println("Cluster centers:"); + for (Vector center: clusters.clusterCenters()) { + System.out.println(" " + center); + } + double cost = clusters.computeCost(parsedData.rdd()); + System.out.println("Cost: " + cost); + + // Evaluate clustering by computing Within Set Sum of Squared Errors + double WSSSE = clusters.computeCost(parsedData.rdd()); + System.out.println("Within Set Sum of Squared Errors = " + WSSSE); + + // Save and load model + clusters.save(jsc.sc(), "target/org/apache/spark/JavaKMeansExample/KMeansModel"); + KMeansModel sameModel = KMeansModel.load(jsc.sc(), + "target/org/apache/spark/JavaKMeansExample/KMeansModel"); + // $example off$ + + jsc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaKernelDensityEstimationExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKernelDensityEstimationExample.java new file mode 100644 index 000000000000..41de0d90eccd --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKernelDensityEstimationExample.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +// $example on$ +import java.util.Arrays; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.stat.KernelDensity; +// $example off$ + +public class JavaKernelDensityEstimationExample { + public static void main(String[] args) { + + SparkConf conf = new SparkConf().setAppName("JavaKernelDensityEstimationExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + + // $example on$ + // an RDD of sample data + JavaRDD data = jsc.parallelize( + Arrays.asList(1.0, 1.0, 1.0, 2.0, 3.0, 4.0, 5.0, 5.0, 6.0, 7.0, 8.0, 9.0, 9.0)); + + // Construct the density estimator with the sample data + // and a standard deviation for the Gaussian kernels + KernelDensity kd = new KernelDensity().setSample(data).setBandwidth(3.0); + + // Find density estimates for the given values + double[] densities = kd.estimate(new double[]{-1.0, 2.0, 5.0}); + + System.out.println(Arrays.toString(densities)); + // $example off$ + + jsc.stop(); + } +} + diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLBFGSExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLBFGSExample.java new file mode 100644 index 000000000000..3fdc03a92ad7 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLBFGSExample.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +// $example on$ +import java.util.Arrays; + +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.mllib.classification.LogisticRegressionModel; +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.optimization.*; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +// $example off$ + +public class JavaLBFGSExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("L-BFGS Example"); + SparkContext sc = new SparkContext(conf); + + // $example on$ + String path = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); + int numFeatures = data.take(1).get(0).features().size(); + + // Split initial RDD into two... [60% training data, 40% testing data]. + JavaRDD trainingInit = data.sample(false, 0.6, 11L); + JavaRDD test = data.subtract(trainingInit); + + // Append 1 into the training data as intercept. + JavaPairRDD training = data.mapToPair(p -> + new Tuple2<>(p.label(), MLUtils.appendBias(p.features()))); + training.cache(); + + // Run training algorithm to build the model. + int numCorrections = 10; + double convergenceTol = 1e-4; + int maxNumIterations = 20; + double regParam = 0.1; + Vector initialWeightsWithIntercept = Vectors.dense(new double[numFeatures + 1]); + + Tuple2 result = LBFGS.runLBFGS( + training.rdd(), + new LogisticGradient(), + new SquaredL2Updater(), + numCorrections, + convergenceTol, + maxNumIterations, + regParam, + initialWeightsWithIntercept); + Vector weightsWithIntercept = result._1(); + double[] loss = result._2(); + + LogisticRegressionModel model = new LogisticRegressionModel( + Vectors.dense(Arrays.copyOf(weightsWithIntercept.toArray(), weightsWithIntercept.size() - 1)), + (weightsWithIntercept.toArray())[weightsWithIntercept.size() - 1]); + + // Clear the default threshold. + model.clearThreshold(); + + // Compute raw scores on the test set. + JavaPairRDD scoreAndLabels = test.mapToPair(p -> + new Tuple2<>(model.predict(p.features()), p.label())); + + // Get evaluation metrics. + BinaryClassificationMetrics metrics = + new BinaryClassificationMetrics(scoreAndLabels.rdd()); + double auROC = metrics.areaUnderROC(); + + System.out.println("Loss of each step in training process"); + for (double l : loss) { + System.out.println(l); + } + System.out.println("Area under ROC = " + auROC); + // $example off$ + + sc.stop(); + } +} + diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java deleted file mode 100644 index fd53c81cc497..000000000000 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLDAExample.java +++ /dev/null @@ -1,76 +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.examples.mllib; - -import scala.Tuple2; - -import org.apache.spark.api.java.*; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.clustering.DistributedLDAModel; -import org.apache.spark.mllib.clustering.LDA; -import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.SparkConf; - -public class JavaLDAExample { - public static void main(String[] args) { - SparkConf conf = new SparkConf().setAppName("LDA Example"); - JavaSparkContext sc = new JavaSparkContext(conf); - - // Load and parse the data - String path = "data/mllib/sample_lda_data.txt"; - JavaRDD data = sc.textFile(path); - JavaRDD parsedData = data.map( - new Function() { - public Vector call(String s) { - String[] sarray = s.trim().split(" "); - double[] values = new double[sarray.length]; - for (int i = 0; i < sarray.length; i++) - values[i] = Double.parseDouble(sarray[i]); - return Vectors.dense(values); - } - } - ); - // Index documents with unique IDs - JavaPairRDD corpus = JavaPairRDD.fromJavaRDD(parsedData.zipWithIndex().map( - new Function, Tuple2>() { - public Tuple2 call(Tuple2 doc_id) { - return doc_id.swap(); - } - } - )); - corpus.cache(); - - // Cluster the documents into three topics using LDA - DistributedLDAModel ldaModel = (DistributedLDAModel)new LDA().setK(3).run(corpus); - - // Output topics. Each is a distribution over words (matching word count vectors) - System.out.println("Learned topics (as distributions over vocab of " + ldaModel.vocabSize() - + " words):"); - Matrix topics = ldaModel.topicsMatrix(); - for (int topic = 0; topic < 3; topic++) { - System.out.print("Topic " + topic + ":"); - for (int word = 0; word < ldaModel.vocabSize(); word++) { - System.out.print(" " + topics.apply(word, topic)); - } - System.out.println(); - } - sc.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java deleted file mode 100644 index eceb6927d555..000000000000 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java +++ /dev/null @@ -1,82 +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.examples.mllib; - -import java.util.regex.Pattern; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; - -import org.apache.spark.mllib.classification.LogisticRegressionWithSGD; -import org.apache.spark.mllib.classification.LogisticRegressionModel; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; - -/** - * Logistic regression based classification using ML Lib. - */ -public final class JavaLR { - - static class ParsePoint implements Function { - private static final Pattern COMMA = Pattern.compile(","); - private static final Pattern SPACE = Pattern.compile(" "); - - @Override - public LabeledPoint call(String line) { - String[] parts = COMMA.split(line); - double y = Double.parseDouble(parts[0]); - String[] tok = SPACE.split(parts[1]); - double[] x = new double[tok.length]; - for (int i = 0; i < tok.length; ++i) { - x[i] = Double.parseDouble(tok[i]); - } - return new LabeledPoint(y, Vectors.dense(x)); - } - } - - public static void main(String[] args) { - if (args.length != 3) { - System.err.println("Usage: JavaLR "); - System.exit(1); - } - SparkConf sparkConf = new SparkConf().setAppName("JavaLR"); - JavaSparkContext sc = new JavaSparkContext(sparkConf); - JavaRDD lines = sc.textFile(args[0]); - JavaRDD points = lines.map(new ParsePoint()).cache(); - double stepSize = Double.parseDouble(args[1]); - int iterations = Integer.parseInt(args[2]); - - // Another way to configure LogisticRegression - // - // LogisticRegressionWithSGD lr = new LogisticRegressionWithSGD(); - // lr.optimizer().setNumIterations(iterations) - // .setStepSize(stepSize) - // .setMiniBatchFraction(1.0); - // lr.setIntercept(true); - // LogisticRegressionModel model = lr.train(points.rdd()); - - LogisticRegressionModel model = LogisticRegressionWithSGD.train(points.rdd(), - iterations, stepSize); - - System.out.print("Final w: " + model.weights()); - - sc.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLatentDirichletAllocationExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLatentDirichletAllocationExample.java new file mode 100644 index 000000000000..887edf8c2121 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLatentDirichletAllocationExample.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; + +// $example on$ +import scala.Tuple2; + +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.clustering.DistributedLDAModel; +import org.apache.spark.mllib.clustering.LDA; +import org.apache.spark.mllib.clustering.LDAModel; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +// $example off$ + +public class JavaLatentDirichletAllocationExample { + public static void main(String[] args) { + + SparkConf conf = new SparkConf().setAppName("JavaKLatentDirichletAllocationExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + + // $example on$ + // Load and parse the data + String path = "data/mllib/sample_lda_data.txt"; + JavaRDD data = jsc.textFile(path); + JavaRDD parsedData = data.map(s -> { + String[] sarray = s.trim().split(" "); + double[] values = new double[sarray.length]; + for (int i = 0; i < sarray.length; i++) { + values[i] = Double.parseDouble(sarray[i]); + } + return Vectors.dense(values); + }); + // Index documents with unique IDs + JavaPairRDD corpus = + JavaPairRDD.fromJavaRDD(parsedData.zipWithIndex().map(Tuple2::swap)); + corpus.cache(); + + // Cluster the documents into three topics using LDA + LDAModel ldaModel = new LDA().setK(3).run(corpus); + + // Output topics. Each is a distribution over words (matching word count vectors) + System.out.println("Learned topics (as distributions over vocab of " + ldaModel.vocabSize() + + " words):"); + Matrix topics = ldaModel.topicsMatrix(); + for (int topic = 0; topic < 3; topic++) { + System.out.print("Topic " + topic + ":"); + for (int word = 0; word < ldaModel.vocabSize(); word++) { + System.out.print(" " + topics.apply(word, topic)); + } + System.out.println(); + } + + ldaModel.save(jsc.sc(), + "target/org/apache/spark/JavaLatentDirichletAllocationExample/LDAModel"); + DistributedLDAModel sameModel = DistributedLDAModel.load(jsc.sc(), + "target/org/apache/spark/JavaLatentDirichletAllocationExample/LDAModel"); + // $example off$ + + jsc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLinearRegressionWithSGDExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLinearRegressionWithSGDExample.java new file mode 100644 index 000000000000..324a781c1a44 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLinearRegressionWithSGDExample.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; + +// $example on$ +import scala.Tuple2; + +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.regression.LinearRegressionModel; +import org.apache.spark.mllib.regression.LinearRegressionWithSGD; +// $example off$ + +/** + * Example for LinearRegressionWithSGD. + */ +public class JavaLinearRegressionWithSGDExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("JavaLinearRegressionWithSGDExample"); + JavaSparkContext sc = new JavaSparkContext(conf); + + // $example on$ + // Load and parse the data + String path = "data/mllib/ridge-data/lpsa.data"; + JavaRDD data = sc.textFile(path); + JavaRDD parsedData = data.map(line -> { + String[] parts = line.split(","); + String[] features = parts[1].split(" "); + double[] v = new double[features.length]; + for (int i = 0; i < features.length - 1; i++) { + v[i] = Double.parseDouble(features[i]); + } + return new LabeledPoint(Double.parseDouble(parts[0]), Vectors.dense(v)); + }); + parsedData.cache(); + + // Building the model + int numIterations = 100; + double stepSize = 0.00000001; + LinearRegressionModel model = + LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations, stepSize); + + // Evaluate model on training examples and compute training error + JavaPairRDD valuesAndPreds = parsedData.mapToPair(point -> + new Tuple2<>(model.predict(point.features()), point.label())); + + double MSE = valuesAndPreds.mapToDouble(pair -> { + double diff = pair._1() - pair._2(); + return diff * diff; + }).mean(); + System.out.println("training Mean Squared Error = " + MSE); + + // Save and load model + model.save(sc.sc(), "target/tmp/javaLinearRegressionWithSGDModel"); + LinearRegressionModel sameModel = LinearRegressionModel.load(sc.sc(), + "target/tmp/javaLinearRegressionWithSGDModel"); + // $example off$ + + sc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLogisticRegressionWithLBFGSExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLogisticRegressionWithLBFGSExample.java new file mode 100644 index 000000000000..26b8a6e9fa3a --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLogisticRegressionWithLBFGSExample.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; + +// $example on$ +import scala.Tuple2; + +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.classification.LogisticRegressionModel; +import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS; +import org.apache.spark.mllib.evaluation.MulticlassMetrics; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +// $example off$ + +/** + * Example for LogisticRegressionWithLBFGS. + */ +public class JavaLogisticRegressionWithLBFGSExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("JavaLogisticRegressionWithLBFGSExample"); + SparkContext sc = new SparkContext(conf); + // $example on$ + String path = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); + + // Split initial RDD into two... [60% training data, 40% testing data]. + JavaRDD[] splits = data.randomSplit(new double[] {0.6, 0.4}, 11L); + JavaRDD training = splits[0].cache(); + JavaRDD test = splits[1]; + + // Run training algorithm to build the model. + LogisticRegressionModel model = new LogisticRegressionWithLBFGS() + .setNumClasses(10) + .run(training.rdd()); + + // Compute raw scores on the test set. + JavaPairRDD predictionAndLabels = test.mapToPair(p -> + new Tuple2<>(model.predict(p.features()), p.label())); + + // Get evaluation metrics. + MulticlassMetrics metrics = new MulticlassMetrics(predictionAndLabels.rdd()); + double accuracy = metrics.accuracy(); + System.out.println("Accuracy = " + accuracy); + + // Save and load model + model.save(sc, "target/tmp/javaLogisticRegressionWithLBFGSModel"); + LogisticRegressionModel sameModel = LogisticRegressionModel.load(sc, + "target/tmp/javaLogisticRegressionWithLBFGSModel"); + // $example off$ + + sc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaMultiLabelClassificationMetricsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaMultiLabelClassificationMetricsExample.java new file mode 100644 index 000000000000..bc99dc023fa7 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaMultiLabelClassificationMetricsExample.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.mllib.evaluation.MultilabelMetrics; +import org.apache.spark.SparkConf; +// $example off$ + +public class JavaMultiLabelClassificationMetricsExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("Multilabel Classification Metrics Example"); + JavaSparkContext sc = new JavaSparkContext(conf); + // $example on$ + List> data = Arrays.asList( + new Tuple2<>(new double[]{0.0, 1.0}, new double[]{0.0, 2.0}), + new Tuple2<>(new double[]{0.0, 2.0}, new double[]{0.0, 1.0}), + new Tuple2<>(new double[]{}, new double[]{0.0}), + new Tuple2<>(new double[]{2.0}, new double[]{2.0}), + new Tuple2<>(new double[]{2.0, 0.0}, new double[]{2.0, 0.0}), + new Tuple2<>(new double[]{0.0, 1.0, 2.0}, new double[]{0.0, 1.0}), + new Tuple2<>(new double[]{1.0}, new double[]{1.0, 2.0}) + ); + JavaRDD> scoreAndLabels = sc.parallelize(data); + + // Instantiate metrics object + MultilabelMetrics metrics = new MultilabelMetrics(scoreAndLabels.rdd()); + + // Summary stats + System.out.format("Recall = %f\n", metrics.recall()); + System.out.format("Precision = %f\n", metrics.precision()); + System.out.format("F1 measure = %f\n", metrics.f1Measure()); + System.out.format("Accuracy = %f\n", metrics.accuracy()); + + // Stats by labels + for (int i = 0; i < metrics.labels().length - 1; i++) { + System.out.format("Class %1.1f precision = %f\n", metrics.labels()[i], metrics.precision( + metrics.labels()[i])); + System.out.format("Class %1.1f recall = %f\n", metrics.labels()[i], metrics.recall( + metrics.labels()[i])); + System.out.format("Class %1.1f F1 score = %f\n", metrics.labels()[i], metrics.f1Measure( + metrics.labels()[i])); + } + + // Micro stats + System.out.format("Micro recall = %f\n", metrics.microRecall()); + System.out.format("Micro precision = %f\n", metrics.microPrecision()); + System.out.format("Micro F1 measure = %f\n", metrics.microF1Measure()); + + // Hamming loss + System.out.format("Hamming loss = %f\n", metrics.hammingLoss()); + + // Subset accuracy + System.out.format("Subset accuracy = %f\n", metrics.subsetAccuracy()); + // $example off$ + + sc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaMulticlassClassificationMetricsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaMulticlassClassificationMetricsExample.java new file mode 100644 index 000000000000..03670383b794 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaMulticlassClassificationMetricsExample.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +// $example on$ +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.mllib.classification.LogisticRegressionModel; +import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS; +import org.apache.spark.mllib.evaluation.MulticlassMetrics; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.mllib.linalg.Matrix; +// $example off$ +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; + +public class JavaMulticlassClassificationMetricsExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("Multi class Classification Metrics Example"); + SparkContext sc = new SparkContext(conf); + // $example on$ + String path = "data/mllib/sample_multiclass_classification_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); + + // Split initial RDD into two... [60% training data, 40% testing data]. + JavaRDD[] splits = data.randomSplit(new double[]{0.6, 0.4}, 11L); + JavaRDD training = splits[0].cache(); + JavaRDD test = splits[1]; + + // Run training algorithm to build the model. + LogisticRegressionModel model = new LogisticRegressionWithLBFGS() + .setNumClasses(3) + .run(training.rdd()); + + // Compute raw scores on the test set. + JavaPairRDD predictionAndLabels = test.mapToPair(p -> + new Tuple2<>(model.predict(p.features()), p.label())); + + // Get evaluation metrics. + MulticlassMetrics metrics = new MulticlassMetrics(predictionAndLabels.rdd()); + + // Confusion matrix + Matrix confusion = metrics.confusionMatrix(); + System.out.println("Confusion matrix: \n" + confusion); + + // Overall statistics + System.out.println("Accuracy = " + metrics.accuracy()); + + // Stats by labels + for (int i = 0; i < metrics.labels().length; i++) { + System.out.format("Class %f precision = %f\n", metrics.labels()[i],metrics.precision( + metrics.labels()[i])); + System.out.format("Class %f recall = %f\n", metrics.labels()[i], metrics.recall( + metrics.labels()[i])); + System.out.format("Class %f F1 score = %f\n", metrics.labels()[i], metrics.fMeasure( + metrics.labels()[i])); + } + + //Weighted stats + System.out.format("Weighted precision = %f\n", metrics.weightedPrecision()); + System.out.format("Weighted recall = %f\n", metrics.weightedRecall()); + System.out.format("Weighted F1 score = %f\n", metrics.weightedFMeasure()); + System.out.format("Weighted false positive rate = %f\n", metrics.weightedFalsePositiveRate()); + + // Save and load model + model.save(sc, "target/tmp/LogisticRegressionModel"); + LogisticRegressionModel sameModel = LogisticRegressionModel.load(sc, + "target/tmp/LogisticRegressionModel"); + // $example off$ + + sc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java index e6a5904bd71f..d80dbe80000b 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java @@ -19,8 +19,6 @@ // $example on$ import scala.Tuple2; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -36,29 +34,22 @@ public static void main(String[] args) { SparkConf sparkConf = new SparkConf().setAppName("JavaNaiveBayesExample"); JavaSparkContext jsc = new JavaSparkContext(sparkConf); // $example on$ - String path = "data/mllib/sample_naive_bayes_data.txt"; + String path = "data/mllib/sample_libsvm_data.txt"; JavaRDD inputData = MLUtils.loadLibSVMFile(jsc.sc(), path).toJavaRDD(); - JavaRDD[] tmp = inputData.randomSplit(new double[]{0.6, 0.4}, 12345); + JavaRDD[] tmp = inputData.randomSplit(new double[]{0.6, 0.4}); JavaRDD training = tmp[0]; // training set JavaRDD test = tmp[1]; // test set - final NaiveBayesModel model = NaiveBayes.train(training.rdd(), 1.0); + NaiveBayesModel model = NaiveBayes.train(training.rdd(), 1.0); JavaPairRDD predictionAndLabel = - test.mapToPair(new PairFunction() { - @Override - public Tuple2 call(LabeledPoint p) { - return new Tuple2(model.predict(p.features()), p.label()); - } - }); - double accuracy = predictionAndLabel.filter(new Function, Boolean>() { - @Override - public Boolean call(Tuple2 pl) { - return pl._1().equals(pl._2()); - } - }).count() / (double) test.count(); + test.mapToPair(p -> new Tuple2<>(model.predict(p.features()), p.label())); + double accuracy = + predictionAndLabel.filter(pl -> pl._1().equals(pl._2())).count() / (double) test.count(); // Save and load model model.save(jsc.sc(), "target/tmp/myNaiveBayesModel"); NaiveBayesModel sameModel = NaiveBayesModel.load(jsc.sc(), "target/tmp/myNaiveBayesModel"); // $example off$ + + jsc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaPCAExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaPCAExample.java new file mode 100644 index 000000000000..0a7dc621e111 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaPCAExample.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +// $example on$ +import java.util.Arrays; +import java.util.List; +// $example off$ + +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +// $example on$ +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; +// $example off$ + +/** + * Example for compute principal components on a 'RowMatrix'. + */ +public class JavaPCAExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("PCA Example"); + SparkContext sc = new SparkContext(conf); + JavaSparkContext jsc = JavaSparkContext.fromSparkContext(sc); + + // $example on$ + List data = Arrays.asList( + Vectors.sparse(5, new int[] {1, 3}, new double[] {1.0, 7.0}), + Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0), + Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0) + ); + + JavaRDD rows = jsc.parallelize(data); + + // Create a RowMatrix from JavaRDD. + RowMatrix mat = new RowMatrix(rows.rdd()); + + // Compute the top 4 principal components. + // Principal components are stored in a local dense matrix. + Matrix pc = mat.computePrincipalComponents(4); + + // Project the rows to the linear space spanned by the top 4 principal components. + RowMatrix projected = mat.multiply(pc); + // $example off$ + Vector[] collectPartitions = (Vector[])projected.rows().collect(); + System.out.println("Projected vector of principal component:"); + for (Vector vector : collectPartitions) { + System.out.println("\t" + vector); + } + jsc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java index 6c6f9768f015..5155f182ba20 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaPowerIterationClusteringExample.java @@ -17,15 +17,17 @@ package org.apache.spark.examples.mllib; -import scala.Tuple3; +import java.util.Arrays; -import com.google.common.collect.Lists; +import scala.Tuple3; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +// $example on$ import org.apache.spark.mllib.clustering.PowerIterationClustering; import org.apache.spark.mllib.clustering.PowerIterationClusteringModel; +// $example off$ /** * Java example for graph clustering using power iteration clustering (PIC). @@ -36,12 +38,13 @@ public static void main(String[] args) { 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))); + // $example on$ + JavaRDD> similarities = sc.parallelize(Arrays.asList( + 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) @@ -51,6 +54,7 @@ public static void main(String[] args) { for (PowerIterationClustering.Assignment a: model.assignments().toJavaRDD().collect()) { System.out.println(a.id() + " -> " + a.cluster()); } + // $example off$ sc.stop(); } diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaPrefixSpanExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaPrefixSpanExample.java index 68ec7c1e6ebe..163407594129 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaPrefixSpanExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaPrefixSpanExample.java @@ -51,5 +51,7 @@ public static void main(String[] args) { System.out.println(freqSeq.javaSequence() + ", " + freqSeq.freq()); } // $example off$ + + sc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java new file mode 100644 index 000000000000..6998ce2156c2 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +// $example on$ +import java.util.HashMap; +import java.util.Map; + +import scala.Tuple2; + +import org.apache.spark.SparkConf; +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.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.RandomForest; +import org.apache.spark.mllib.tree.model.RandomForestModel; +import org.apache.spark.mllib.util.MLUtils; +// $example off$ + +public class JavaRandomForestClassificationExample { + public static void main(String[] args) { + // $example on$ + SparkConf sparkConf = new SparkConf().setAppName("JavaRandomForestClassificationExample"); + JavaSparkContext jsc = new JavaSparkContext(sparkConf); + // Load and parse the data file. + String datapath = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(jsc.sc(), datapath).toJavaRDD(); + // Split the data into training and test sets (30% held out for testing) + JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); + JavaRDD trainingData = splits[0]; + JavaRDD testData = splits[1]; + + // Train a RandomForest model. + // Empty categoricalFeaturesInfo indicates all features are continuous. + Integer numClasses = 2; + Map categoricalFeaturesInfo = new HashMap<>(); + Integer numTrees = 3; // Use more in practice. + String featureSubsetStrategy = "auto"; // Let the algorithm choose. + String impurity = "gini"; + Integer maxDepth = 5; + Integer maxBins = 32; + Integer seed = 12345; + + RandomForestModel model = RandomForest.trainClassifier(trainingData, numClasses, + categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, + seed); + + // Evaluate model on test instances and compute test error + JavaPairRDD predictionAndLabel = + testData.mapToPair(p -> new Tuple2<>(model.predict(p.features()), p.label())); + double testErr = + predictionAndLabel.filter(pl -> !pl._1().equals(pl._2())).count() / (double) testData.count(); + System.out.println("Test Error: " + testErr); + System.out.println("Learned classification forest model:\n" + model.toDebugString()); + + // Save and load model + model.save(jsc.sc(), "target/tmp/myRandomForestClassificationModel"); + RandomForestModel sameModel = RandomForestModel.load(jsc.sc(), + "target/tmp/myRandomForestClassificationModel"); + // $example off$ + + jsc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestExample.java deleted file mode 100644 index 89a4e092a5af..000000000000 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestExample.java +++ /dev/null @@ -1,139 +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.examples.mllib; - -import scala.Tuple2; - -import java.util.HashMap; - -import org.apache.spark.SparkConf; -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.Function2; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.mllib.tree.RandomForest; -import org.apache.spark.mllib.tree.model.RandomForestModel; -import org.apache.spark.mllib.util.MLUtils; - -public final class JavaRandomForestExample { - - /** - * Note: This example illustrates binary classification. - * For information on multiclass classification, please refer to the JavaDecisionTree.java - * example. - */ - private static void testClassification(JavaRDD trainingData, - JavaRDD testData) { - // Train a RandomForest model. - // Empty categoricalFeaturesInfo indicates all features are continuous. - Integer numClasses = 2; - HashMap categoricalFeaturesInfo = new HashMap(); - Integer numTrees = 3; // Use more in practice. - String featureSubsetStrategy = "auto"; // Let the algorithm choose. - String impurity = "gini"; - Integer maxDepth = 4; - Integer maxBins = 32; - Integer seed = 12345; - - final RandomForestModel model = RandomForest.trainClassifier(trainingData, numClasses, - categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, - seed); - - // Evaluate model on test instances and compute test error - JavaPairRDD predictionAndLabel = - testData.mapToPair(new PairFunction() { - @Override - public Tuple2 call(LabeledPoint p) { - return new Tuple2(model.predict(p.features()), p.label()); - } - }); - Double testErr = - 1.0 * predictionAndLabel.filter(new Function, Boolean>() { - @Override - public Boolean call(Tuple2 pl) { - return !pl._1().equals(pl._2()); - } - }).count() / testData.count(); - System.out.println("Test Error: " + testErr); - System.out.println("Learned classification forest model:\n" + model.toDebugString()); - } - - private static void testRegression(JavaRDD trainingData, - JavaRDD testData) { - // Train a RandomForest model. - // Empty categoricalFeaturesInfo indicates all features are continuous. - HashMap categoricalFeaturesInfo = new HashMap(); - Integer numTrees = 3; // Use more in practice. - String featureSubsetStrategy = "auto"; // Let the algorithm choose. - String impurity = "variance"; - Integer maxDepth = 4; - Integer maxBins = 32; - Integer seed = 12345; - - final RandomForestModel model = RandomForest.trainRegressor(trainingData, - categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, - seed); - - // Evaluate model on test instances and compute test error - JavaPairRDD predictionAndLabel = - testData.mapToPair(new PairFunction() { - @Override - public Tuple2 call(LabeledPoint p) { - return new Tuple2(model.predict(p.features()), p.label()); - } - }); - Double testMSE = - predictionAndLabel.map(new Function, Double>() { - @Override - public Double call(Tuple2 pl) { - Double diff = pl._1() - pl._2(); - return diff * diff; - } - }).reduce(new Function2() { - @Override - public Double call(Double a, Double b) { - return a + b; - } - }) / testData.count(); - System.out.println("Test Mean Squared Error: " + testMSE); - System.out.println("Learned regression forest model:\n" + model.toDebugString()); - } - - public static void main(String[] args) { - SparkConf sparkConf = new SparkConf().setAppName("JavaRandomForestExample"); - JavaSparkContext sc = new JavaSparkContext(sparkConf); - - // Load and parse the data file. - String datapath = "data/mllib/sample_libsvm_data.txt"; - JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); - // Split the data into training and test sets (30% held out for testing) - JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); - JavaRDD trainingData = splits[0]; - JavaRDD testData = splits[1]; - - System.out.println("\nRunning example of classification using RandomForest\n"); - testClassification(trainingData, testData); - - System.out.println("\nRunning example of regression using RandomForest\n"); - testRegression(trainingData, testData); - sc.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestRegressionExample.java new file mode 100644 index 000000000000..4a0f55f52980 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestRegressionExample.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +// $example on$ +import java.util.HashMap; +import java.util.Map; + +import scala.Tuple2; + +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.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.RandomForest; +import org.apache.spark.mllib.tree.model.RandomForestModel; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; +// $example off$ + +public class JavaRandomForestRegressionExample { + public static void main(String[] args) { + // $example on$ + SparkConf sparkConf = new SparkConf().setAppName("JavaRandomForestRegressionExample"); + JavaSparkContext jsc = new JavaSparkContext(sparkConf); + // Load and parse the data file. + String datapath = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(jsc.sc(), datapath).toJavaRDD(); + // Split the data into training and test sets (30% held out for testing) + JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); + JavaRDD trainingData = splits[0]; + JavaRDD testData = splits[1]; + + // Set parameters. + // Empty categoricalFeaturesInfo indicates all features are continuous. + Map categoricalFeaturesInfo = new HashMap<>(); + int numTrees = 3; // Use more in practice. + String featureSubsetStrategy = "auto"; // Let the algorithm choose. + String impurity = "variance"; + int maxDepth = 4; + int maxBins = 32; + int seed = 12345; + // Train a RandomForest model. + RandomForestModel model = RandomForest.trainRegressor(trainingData, + categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed); + + // Evaluate model on test instances and compute test error + JavaPairRDD predictionAndLabel = + testData.mapToPair(p -> new Tuple2<>(model.predict(p.features()), p.label())); + double testMSE = predictionAndLabel.mapToDouble(pl -> { + double diff = pl._1() - pl._2(); + return diff * diff; + }).mean(); + System.out.println("Test Mean Squared Error: " + testMSE); + System.out.println("Learned regression forest model:\n" + model.toDebugString()); + + // Save and load model + model.save(jsc.sc(), "target/tmp/myRandomForestRegressionModel"); + RandomForestModel sameModel = RandomForestModel.load(jsc.sc(), + "target/tmp/myRandomForestRegressionModel"); + // $example off$ + + jsc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java new file mode 100644 index 000000000000..dc9970d88527 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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; + +// $example on$ +import java.util.*; + +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.mllib.evaluation.RegressionMetrics; +import org.apache.spark.mllib.evaluation.RankingMetrics; +import org.apache.spark.mllib.recommendation.ALS; +import org.apache.spark.mllib.recommendation.MatrixFactorizationModel; +import org.apache.spark.mllib.recommendation.Rating; +// $example off$ +import org.apache.spark.SparkConf; + +public class JavaRankingMetricsExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("Java Ranking Metrics Example"); + JavaSparkContext sc = new JavaSparkContext(conf); + // $example on$ + String path = "data/mllib/sample_movielens_data.txt"; + JavaRDD data = sc.textFile(path); + JavaRDD ratings = data.map(line -> { + String[] parts = line.split("::"); + return new Rating(Integer.parseInt(parts[0]), Integer.parseInt(parts[1]), Double + .parseDouble(parts[2]) - 2.5); + }); + ratings.cache(); + + // Train an ALS model + MatrixFactorizationModel model = ALS.train(JavaRDD.toRDD(ratings), 10, 10, 0.01); + + // Get top 10 recommendations for every user and scale ratings from 0 to 1 + JavaRDD> userRecs = model.recommendProductsForUsers(10).toJavaRDD(); + JavaRDD> userRecsScaled = userRecs.map(t -> { + Rating[] scaledRatings = new Rating[t._2().length]; + for (int i = 0; i < scaledRatings.length; i++) { + double newRating = Math.max(Math.min(t._2()[i].rating(), 1.0), 0.0); + scaledRatings[i] = new Rating(t._2()[i].user(), t._2()[i].product(), newRating); + } + return new Tuple2<>(t._1(), scaledRatings); + }); + JavaPairRDD userRecommended = JavaPairRDD.fromJavaRDD(userRecsScaled); + + // Map ratings to 1 or 0, 1 indicating a movie that should be recommended + JavaRDD binarizedRatings = ratings.map(r -> { + double binaryRating; + if (r.rating() > 0.0) { + binaryRating = 1.0; + } else { + binaryRating = 0.0; + } + return new Rating(r.user(), r.product(), binaryRating); + }); + + // Group ratings by common user + JavaPairRDD> userMovies = binarizedRatings.groupBy(Rating::user); + + // Get true relevant documents from all user ratings + JavaPairRDD> userMoviesList = userMovies.mapValues(docs -> { + List products = new ArrayList<>(); + for (Rating r : docs) { + if (r.rating() > 0.0) { + products.add(r.product()); + } + } + return products; + }); + + // Extract the product id from each recommendation + JavaPairRDD> userRecommendedList = userRecommended.mapValues(docs -> { + List products = new ArrayList<>(); + for (Rating r : docs) { + products.add(r.product()); + } + return products; + }); + JavaRDD, List>> relevantDocs = userMoviesList.join( + userRecommendedList).values(); + + // Instantiate the metrics object + RankingMetrics metrics = RankingMetrics.of(relevantDocs); + + // Precision and NDCG at k + Integer[] kVector = {1, 3, 5}; + for (Integer k : kVector) { + System.out.format("Precision at %d = %f\n", k, metrics.precisionAt(k)); + System.out.format("NDCG at %d = %f\n", k, metrics.ndcgAt(k)); + } + + // Mean average precision + System.out.format("Mean average precision = %f\n", metrics.meanAveragePrecision()); + + // Evaluate the model using numerical ratings and regression metrics + JavaRDD> userProducts = + ratings.map(r -> new Tuple2<>(r.user(), r.product())); + + JavaPairRDD, Object> predictions = JavaPairRDD.fromJavaRDD( + model.predict(JavaRDD.toRDD(userProducts)).toJavaRDD().map(r -> + new Tuple2<>(new Tuple2<>(r.user(), r.product()), r.rating()))); + JavaRDD> ratesAndPreds = + JavaPairRDD.fromJavaRDD(ratings.map(r -> + new Tuple2, Object>( + new Tuple2<>(r.user(), r.product()), + r.rating()) + )).join(predictions).values(); + + // Create regression metrics object + RegressionMetrics regressionMetrics = new RegressionMetrics(ratesAndPreds.rdd()); + + // Root mean squared error + System.out.format("RMSE = %f\n", regressionMetrics.rootMeanSquaredError()); + + // R-squared + System.out.format("R-squared = %f\n", regressionMetrics.r2()); + // $example off$ + + sc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRecommendationExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRecommendationExample.java new file mode 100644 index 000000000000..1ee68da35e81 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRecommendationExample.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +// $example on$ +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.mllib.recommendation.ALS; +import org.apache.spark.mllib.recommendation.MatrixFactorizationModel; +import org.apache.spark.mllib.recommendation.Rating; +import org.apache.spark.SparkConf; +// $example off$ + +public class JavaRecommendationExample { + public static void main(String[] args) { + // $example on$ + SparkConf conf = new SparkConf().setAppName("Java Collaborative Filtering Example"); + JavaSparkContext jsc = new JavaSparkContext(conf); + + // Load and parse the data + String path = "data/mllib/als/test.data"; + JavaRDD data = jsc.textFile(path); + JavaRDD ratings = data.map(s -> { + String[] sarray = s.split(","); + return new Rating(Integer.parseInt(sarray[0]), + Integer.parseInt(sarray[1]), + Double.parseDouble(sarray[2])); + }); + + // Build the recommendation model using ALS + int rank = 10; + int numIterations = 10; + MatrixFactorizationModel model = ALS.train(JavaRDD.toRDD(ratings), rank, numIterations, 0.01); + + // Evaluate the model on rating data + JavaRDD> userProducts = + ratings.map(r -> new Tuple2<>(r.user(), r.product())); + JavaPairRDD, Double> predictions = JavaPairRDD.fromJavaRDD( + model.predict(JavaRDD.toRDD(userProducts)).toJavaRDD() + .map(r -> new Tuple2<>(new Tuple2<>(r.user(), r.product()), r.rating())) + ); + JavaRDD> ratesAndPreds = JavaPairRDD.fromJavaRDD( + ratings.map(r -> new Tuple2<>(new Tuple2<>(r.user(), r.product()), r.rating()))) + .join(predictions).values(); + double MSE = ratesAndPreds.mapToDouble(pair -> { + double err = pair._1() - pair._2(); + return err * err; + }).mean(); + System.out.println("Mean Squared Error = " + MSE); + + // Save and load model + model.save(jsc.sc(), "target/tmp/myCollaborativeFilter"); + MatrixFactorizationModel sameModel = MatrixFactorizationModel.load(jsc.sc(), + "target/tmp/myCollaborativeFilter"); + // $example off$ + + jsc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRegressionMetricsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRegressionMetricsExample.java new file mode 100644 index 000000000000..00033b5730a3 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRegressionMetricsExample.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +// $example on$ +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.regression.LinearRegressionModel; +import org.apache.spark.mllib.regression.LinearRegressionWithSGD; +import org.apache.spark.mllib.evaluation.RegressionMetrics; +import org.apache.spark.SparkConf; +// $example off$ + +public class JavaRegressionMetricsExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("Java Regression Metrics Example"); + JavaSparkContext sc = new JavaSparkContext(conf); + // $example on$ + // Load and parse the data + String path = "data/mllib/sample_linear_regression_data.txt"; + JavaRDD data = sc.textFile(path); + JavaRDD parsedData = data.map(line -> { + String[] parts = line.split(" "); + double[] v = new double[parts.length - 1]; + for (int i = 1; i < parts.length; i++) { + v[i - 1] = Double.parseDouble(parts[i].split(":")[1]); + } + return new LabeledPoint(Double.parseDouble(parts[0]), Vectors.dense(v)); + }); + parsedData.cache(); + + // Building the model + int numIterations = 100; + LinearRegressionModel model = LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), + numIterations); + + // Evaluate model on training examples and compute training error + JavaPairRDD valuesAndPreds = parsedData.mapToPair(point -> + new Tuple2<>(model.predict(point.features()), point.label())); + + // Instantiate metrics object + RegressionMetrics metrics = new RegressionMetrics(valuesAndPreds.rdd()); + + // Squared error + System.out.format("MSE = %f\n", metrics.meanSquaredError()); + System.out.format("RMSE = %f\n", metrics.rootMeanSquaredError()); + + // R-squared + System.out.format("R Squared = %f\n", metrics.r2()); + + // Mean absolute error + System.out.format("MAE = %f\n", metrics.meanAbsoluteError()); + + // Explained variance + System.out.format("Explained Variance = %f\n", metrics.explainedVariance()); + + // Save and load model + model.save(sc.sc(), "target/tmp/LogisticRegressionModel"); + LinearRegressionModel sameModel = LinearRegressionModel.load(sc.sc(), + "target/tmp/LogisticRegressionModel"); + // $example off$ + + sc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaSVDExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaSVDExample.java new file mode 100644 index 000000000000..802be3960a33 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaSVDExample.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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; + +// $example on$ +import java.util.Arrays; +import java.util.List; +// $example off$ + +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +// $example on$ +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.SingularValueDecomposition; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; +// $example off$ + +/** + * Example for SingularValueDecomposition. + */ +public class JavaSVDExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("SVD Example"); + SparkContext sc = new SparkContext(conf); + JavaSparkContext jsc = JavaSparkContext.fromSparkContext(sc); + + // $example on$ + List data = Arrays.asList( + Vectors.sparse(5, new int[] {1, 3}, new double[] {1.0, 7.0}), + Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0), + Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0) + ); + + JavaRDD rows = jsc.parallelize(data); + + // Create a RowMatrix from JavaRDD. + RowMatrix mat = new RowMatrix(rows.rdd()); + + // Compute the top 5 singular values and corresponding singular vectors. + SingularValueDecomposition svd = mat.computeSVD(5, true, 1.0E-9d); + RowMatrix U = svd.U(); // The U factor is a RowMatrix. + Vector s = svd.s(); // The singular values are stored in a local dense vector. + Matrix V = svd.V(); // The V factor is a local dense matrix. + // $example off$ + Vector[] collectPartitions = (Vector[]) U.rows().collect(); + System.out.println("U factor is:"); + for (Vector vector : collectPartitions) { + System.out.println("\t" + vector); + } + System.out.println("Singular values are: " + s); + System.out.println("V factor is:\n" + V); + + jsc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaSVMWithSGDExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaSVMWithSGDExample.java new file mode 100644 index 000000000000..866a221fdb59 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaSVMWithSGDExample.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; + +// $example on$ +import scala.Tuple2; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.classification.SVMModel; +import org.apache.spark.mllib.classification.SVMWithSGD; +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +// $example off$ + +/** + * Example for SVMWithSGD. + */ +public class JavaSVMWithSGDExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("JavaSVMWithSGDExample"); + SparkContext sc = new SparkContext(conf); + // $example on$ + String path = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); + + // Split initial RDD into two... [60% training data, 40% testing data]. + JavaRDD training = data.sample(false, 0.6, 11L); + training.cache(); + JavaRDD test = data.subtract(training); + + // Run training algorithm to build the model. + int numIterations = 100; + SVMModel model = SVMWithSGD.train(training.rdd(), numIterations); + + // Clear the default threshold. + model.clearThreshold(); + + // Compute raw scores on the test set. + JavaRDD> scoreAndLabels = test.map(p -> + new Tuple2<>(model.predict(p.features()), p.label())); + + // Get evaluation metrics. + BinaryClassificationMetrics metrics = + new BinaryClassificationMetrics(JavaRDD.toRDD(scoreAndLabels)); + double auROC = metrics.areaUnderROC(); + + System.out.println("Area under ROC = " + auROC); + + // Save and load model + model.save(sc, "target/tmp/javaSVMWithSGDModel"); + SVMModel sameModel = SVMModel.load(sc, "target/tmp/javaSVMWithSGDModel"); + // $example off$ + + sc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaSimpleFPGrowth.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaSimpleFPGrowth.java index 72edaca5e95b..f9198e75c2ff 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaSimpleFPGrowth.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaSimpleFPGrowth.java @@ -23,9 +23,6 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -// $example off$ -import org.apache.spark.api.java.function.Function; -// $example on$ import org.apache.spark.mllib.fpm.AssociationRules; import org.apache.spark.mllib.fpm.FPGrowth; import org.apache.spark.mllib.fpm.FPGrowthModel; @@ -42,14 +39,7 @@ public static void main(String[] args) { // $example on$ JavaRDD data = sc.textFile("data/mllib/sample_fpgrowth.txt"); - JavaRDD> transactions = data.map( - new Function>() { - public List call(String line) { - String[] parts = line.split(" "); - return Arrays.asList(parts); - } - } - ); + JavaRDD> transactions = data.map(line -> Arrays.asList(line.split(" "))); FPGrowth fpg = new FPGrowth() .setMinSupport(0.2) @@ -67,5 +57,7 @@ public List call(String line) { rule.javaAntecedent() + " => " + rule.javaConsequent() + ", " + rule.confidence()); } // $example off$ + + sc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaStratifiedSamplingExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaStratifiedSamplingExample.java new file mode 100644 index 000000000000..286b95cfbc33 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaStratifiedSamplingExample.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 com.google.common.collect.ImmutableMap; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; + +// $example on$ +import java.util.*; + +import scala.Tuple2; + +import org.apache.spark.api.java.JavaPairRDD; +// $example off$ + +public class JavaStratifiedSamplingExample { + public static void main(String[] args) { + + SparkConf conf = new SparkConf().setAppName("JavaStratifiedSamplingExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + + @SuppressWarnings("unchecked") + // $example on$ + List> list = Arrays.asList( + new Tuple2<>(1, 'a'), + new Tuple2<>(1, 'b'), + new Tuple2<>(2, 'c'), + new Tuple2<>(2, 'd'), + new Tuple2<>(2, 'e'), + new Tuple2<>(3, 'f') + ); + + JavaPairRDD data = jsc.parallelizePairs(list); + + // specify the exact fraction desired from each key Map + ImmutableMap fractions = ImmutableMap.of(1, 0.1, 2, 0.6, 3, 0.3); + + // Get an approximate sample from each stratum + JavaPairRDD approxSample = data.sampleByKey(false, fractions); + // Get an exact sample from each stratum + JavaPairRDD exactSample = data.sampleByKeyExact(false, fractions); + // $example off$ + + System.out.println("approxSample size is " + approxSample.collect().size()); + for (Tuple2 t : approxSample.collect()) { + System.out.println(t._1() + " " + t._2()); + } + + System.out.println("exactSample size is " + exactSample.collect().size()); + for (Tuple2 t : exactSample.collect()) { + System.out.println(t._1() + " " + t._2()); + } + + jsc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaStreamingTestExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaStreamingTestExample.java new file mode 100644 index 000000000000..4be702c2ba6a --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaStreamingTestExample.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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; + +// $example on$ +import org.apache.spark.mllib.stat.test.BinarySample; +import org.apache.spark.mllib.stat.test.StreamingTest; +import org.apache.spark.mllib.stat.test.StreamingTestResult; +// $example off$ +import org.apache.spark.SparkConf; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.Seconds; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.util.Utils; + + +/** + * Perform streaming testing using Welch's 2-sample t-test on a stream of data, where the data + * stream arrives as text files in a directory. Stops when the two groups are statistically + * significant (p-value < 0.05) or after a user-specified timeout in number of batches is exceeded. + * + * The rows of the text files must be in the form `Boolean, Double`. For example: + * false, -3.92 + * true, 99.32 + * + * Usage: + * JavaStreamingTestExample + * + * To run on your local machine using the directory `dataDir` with 5 seconds between each batch and + * a timeout after 100 insignificant batches, call: + * $ bin/run-example mllib.JavaStreamingTestExample dataDir 5 100 + * + * As you add text files to `dataDir` the significance test wil continually update every + * `batchDuration` seconds until the test becomes significant (p-value < 0.05) or the number of + * batches processed exceeds `numBatchesTimeout`. + */ +public class JavaStreamingTestExample { + + private static int timeoutCounter = 0; + + public static void main(String[] args) throws Exception { + if (args.length != 3) { + System.err.println("Usage: JavaStreamingTestExample " + + " "); + System.exit(1); + } + + String dataDir = args[0]; + Duration batchDuration = Seconds.apply(Long.parseLong(args[1])); + int numBatchesTimeout = Integer.parseInt(args[2]); + + SparkConf conf = new SparkConf().setMaster("local").setAppName("StreamingTestExample"); + JavaStreamingContext ssc = new JavaStreamingContext(conf, batchDuration); + + ssc.checkpoint(Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark").toString()); + + // $example on$ + JavaDStream data = ssc.textFileStream(dataDir).map(line -> { + String[] ts = line.split(","); + boolean label = Boolean.parseBoolean(ts[0]); + double value = Double.parseDouble(ts[1]); + return new BinarySample(label, value); + }); + + StreamingTest streamingTest = new StreamingTest() + .setPeacePeriod(0) + .setWindowSize(0) + .setTestMethod("welch"); + + JavaDStream out = streamingTest.registerStream(data); + out.print(); + // $example off$ + + // Stop processing if test becomes significant or we time out + timeoutCounter = numBatchesTimeout; + + out.foreachRDD(rdd -> { + timeoutCounter -= 1; + boolean anySignificant = !rdd.filter(v -> v.pValue() < 0.05).isEmpty(); + if (timeoutCounter <= 0 || anySignificant) { + rdd.context().stop(); + } + }); + + ssc.start(); + ssc.awaitTermination(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaSummaryStatisticsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaSummaryStatisticsExample.java new file mode 100644 index 000000000000..278706bc8f6e --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaSummaryStatisticsExample.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +// $example on$ +import java.util.Arrays; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; +import org.apache.spark.mllib.stat.Statistics; +// $example off$ + +public class JavaSummaryStatisticsExample { + public static void main(String[] args) { + + SparkConf conf = new SparkConf().setAppName("JavaSummaryStatisticsExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + + // $example on$ + JavaRDD mat = jsc.parallelize( + Arrays.asList( + Vectors.dense(1.0, 10.0, 100.0), + Vectors.dense(2.0, 20.0, 200.0), + Vectors.dense(3.0, 30.0, 300.0) + ) + ); // an RDD of Vectors + + // Compute column summary statistics. + MultivariateStatisticalSummary summary = Statistics.colStats(mat.rdd()); + System.out.println(summary.mean()); // a dense vector containing the mean value for each column + System.out.println(summary.variance()); // column-wise variance + System.out.println(summary.numNonzeros()); // number of nonzeros in each column + // $example off$ + + jsc.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java new file mode 100644 index 000000000000..95859c52c2ae --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java @@ -0,0 +1,288 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.sql; + +// $example on:schema_merging$ +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +// $example off:schema_merging$ +import java.util.Properties; + +// $example on:basic_parquet_example$ +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Encoders; +// $example on:schema_merging$ +// $example on:json_dataset$ +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off:json_dataset$ +// $example off:schema_merging$ +// $example off:basic_parquet_example$ +import org.apache.spark.sql.SparkSession; + +public class JavaSQLDataSourceExample { + + // $example on:schema_merging$ + public static class Square implements Serializable { + private int value; + private int square; + + // Getters and setters... + // $example off:schema_merging$ + public int getValue() { + return value; + } + + public void setValue(int value) { + this.value = value; + } + + public int getSquare() { + return square; + } + + public void setSquare(int square) { + this.square = square; + } + // $example on:schema_merging$ + } + // $example off:schema_merging$ + + // $example on:schema_merging$ + public static class Cube implements Serializable { + private int value; + private int cube; + + // Getters and setters... + // $example off:schema_merging$ + public int getValue() { + return value; + } + + public void setValue(int value) { + this.value = value; + } + + public int getCube() { + return cube; + } + + public void setCube(int cube) { + this.cube = cube; + } + // $example on:schema_merging$ + } + // $example off:schema_merging$ + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("Java Spark SQL data sources example") + .config("spark.some.config.option", "some-value") + .getOrCreate(); + + runBasicDataSourceExample(spark); + runBasicParquetExample(spark); + runParquetSchemaMergingExample(spark); + runJsonDatasetExample(spark); + runJdbcDatasetExample(spark); + + spark.stop(); + } + + private static void runBasicDataSourceExample(SparkSession spark) { + // $example on:generic_load_save_functions$ + Dataset usersDF = spark.read().load("examples/src/main/resources/users.parquet"); + usersDF.select("name", "favorite_color").write().save("namesAndFavColors.parquet"); + // $example off:generic_load_save_functions$ + // $example on:manual_load_options$ + Dataset peopleDF = + spark.read().format("json").load("examples/src/main/resources/people.json"); + peopleDF.select("name", "age").write().format("parquet").save("namesAndAges.parquet"); + // $example off:manual_load_options$ + // $example on:direct_sql$ + Dataset sqlDF = + spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"); + // $example off:direct_sql$ + // $example on:write_sorting_and_bucketing$ + peopleDF.write().bucketBy(42, "name").sortBy("age").saveAsTable("people_bucketed"); + // $example off:write_sorting_and_bucketing$ + // $example on:write_partitioning$ + usersDF + .write() + .partitionBy("favorite_color") + .format("parquet") + .save("namesPartByColor.parquet"); + // $example off:write_partitioning$ + // $example on:write_partition_and_bucket$ + peopleDF + .write() + .partitionBy("favorite_color") + .bucketBy(42, "name") + .saveAsTable("people_partitioned_bucketed"); + // $example off:write_partition_and_bucket$ + + spark.sql("DROP TABLE IF EXISTS people_bucketed"); + spark.sql("DROP TABLE IF EXISTS people_partitioned_bucketed"); + } + + private static void runBasicParquetExample(SparkSession spark) { + // $example on:basic_parquet_example$ + Dataset peopleDF = spark.read().json("examples/src/main/resources/people.json"); + + // DataFrames can be saved as Parquet files, maintaining the schema information + peopleDF.write().parquet("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 DataFrame + Dataset parquetFileDF = spark.read().parquet("people.parquet"); + + // Parquet files can also be used to create a temporary view and then used in SQL statements + parquetFileDF.createOrReplaceTempView("parquetFile"); + Dataset namesDF = spark.sql("SELECT name FROM parquetFile WHERE age BETWEEN 13 AND 19"); + Dataset namesDS = namesDF.map( + (MapFunction) row -> "Name: " + row.getString(0), + Encoders.STRING()); + namesDS.show(); + // +------------+ + // | value| + // +------------+ + // |Name: Justin| + // +------------+ + // $example off:basic_parquet_example$ + } + + private static void runParquetSchemaMergingExample(SparkSession spark) { + // $example on:schema_merging$ + List squares = new ArrayList<>(); + for (int value = 1; value <= 5; value++) { + Square square = new Square(); + square.setValue(value); + square.setSquare(value * value); + squares.add(square); + } + + // Create a simple DataFrame, store into a partition directory + Dataset squaresDF = spark.createDataFrame(squares, Square.class); + squaresDF.write().parquet("data/test_table/key=1"); + + List cubes = new ArrayList<>(); + for (int value = 6; value <= 10; value++) { + Cube cube = new Cube(); + cube.setValue(value); + cube.setCube(value * value * value); + cubes.add(cube); + } + + // Create another DataFrame in a new partition directory, + // adding a new column and dropping an existing column + Dataset cubesDF = spark.createDataFrame(cubes, Cube.class); + cubesDF.write().parquet("data/test_table/key=2"); + + // Read the partitioned table + Dataset mergedDF = spark.read().option("mergeSchema", true).parquet("data/test_table"); + mergedDF.printSchema(); + + // The final schema consists of all 3 columns in the Parquet files together + // with the partitioning column appeared in the partition directory paths + // root + // |-- value: int (nullable = true) + // |-- square: int (nullable = true) + // |-- cube: int (nullable = true) + // |-- key: int (nullable = true) + // $example off:schema_merging$ + } + + private static void runJsonDatasetExample(SparkSession spark) { + // $example on:json_dataset$ + // A JSON dataset is pointed to by path. + // The path can be either a single text file or a directory storing text files + Dataset people = spark.read().json("examples/src/main/resources/people.json"); + + // The inferred schema can be visualized using the printSchema() method + people.printSchema(); + // root + // |-- age: long (nullable = true) + // |-- name: string (nullable = true) + + // Creates a temporary view using the DataFrame + people.createOrReplaceTempView("people"); + + // SQL statements can be run by using the sql methods provided by spark + Dataset namesDF = spark.sql("SELECT name FROM people WHERE age BETWEEN 13 AND 19"); + namesDF.show(); + // +------+ + // | name| + // +------+ + // |Justin| + // +------+ + + // Alternatively, a DataFrame can be created for a JSON dataset represented by + // a Dataset storing one JSON object per string. + List jsonData = Arrays.asList( + "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); + Dataset anotherPeopleDataset = spark.createDataset(jsonData, Encoders.STRING()); + Dataset anotherPeople = spark.read().json(anotherPeopleDataset); + anotherPeople.show(); + // +---------------+----+ + // | address|name| + // +---------------+----+ + // |[Columbus,Ohio]| Yin| + // +---------------+----+ + // $example off:json_dataset$ + } + + private static void runJdbcDatasetExample(SparkSession spark) { + // $example on:jdbc_dataset$ + // Note: JDBC loading and saving can be achieved via either the load/save or jdbc methods + // Loading data from a JDBC source + Dataset jdbcDF = spark.read() + .format("jdbc") + .option("url", "jdbc:postgresql:dbserver") + .option("dbtable", "schema.tablename") + .option("user", "username") + .option("password", "password") + .load(); + + Properties connectionProperties = new Properties(); + connectionProperties.put("user", "username"); + connectionProperties.put("password", "password"); + Dataset jdbcDF2 = spark.read() + .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties); + + // Saving data to a JDBC source + jdbcDF.write() + .format("jdbc") + .option("url", "jdbc:postgresql:dbserver") + .option("dbtable", "schema.tablename") + .option("user", "username") + .option("password", "password") + .save(); + + jdbcDF2.write() + .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties); + + // Specifying create table column data types on write + jdbcDF.write() + .option("createTableColumnTypes", "name CHAR(64), comments VARCHAR(1024)") + .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties); + // $example off:jdbc_dataset$ + } +} 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 deleted file mode 100644 index afee279ec32b..000000000000 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ /dev/null @@ -1,180 +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.examples.sql; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; - -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; - -public class JavaSparkSQL { - public static class Person implements Serializable { - private String name; - private int age; - - public String getName() { - return name; - } - - public void setName(String name) { - this.name = name; - } - - public int getAge() { - return age; - } - - public void setAge(int age) { - this.age = age; - } - } - - public static void main(String[] args) throws Exception { - SparkConf sparkConf = new SparkConf().setAppName("JavaSparkSQL"); - JavaSparkContext ctx = new JavaSparkContext(sparkConf); - SQLContext sqlContext = new SQLContext(ctx); - - System.out.println("=== Data source: RDD ==="); - // Load a text file and convert each line to a Java Bean. - JavaRDD people = ctx.textFile("examples/src/main/resources/people.txt").map( - new Function() { - @Override - public Person call(String line) { - String[] parts = line.split(","); - - Person person = new Person(); - person.setName(parts[0]); - person.setAge(Integer.parseInt(parts[1].trim())); - - return person; - } - }); - - // Apply a schema to an RDD of Java Beans and register it as a table. - DataFrame schemaPeople = sqlContext.createDataFrame(people, Person.class); - schemaPeople.registerTempTable("people"); - - // SQL can be run over RDDs that have been registered as tables. - DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); - - // The results of SQL queries are DataFrames and support all the normal RDD operations. - // The columns of a row in the result can be accessed by ordinal. - List teenagerNames = teenagers.toJavaRDD().map(new Function() { - @Override - public String call(Row row) { - return "Name: " + row.getString(0); - } - }).collect(); - for (String name: teenagerNames) { - System.out.println(name); - } - - System.out.println("=== Data source: Parquet File ==="); - // DataFrames can be saved as parquet files, maintaining the schema information. - schemaPeople.write().parquet("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 DataFrame. - DataFrame parquetFile = sqlContext.read().parquet("people.parquet"); - - //Parquet files can also be registered as tables and then used in SQL statements. - parquetFile.registerTempTable("parquetFile"); - DataFrame teenagers2 = - sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); - teenagerNames = teenagers2.toJavaRDD().map(new Function() { - @Override - public String call(Row row) { - return "Name: " + row.getString(0); - } - }).collect(); - for (String name: teenagerNames) { - System.out.println(name); - } - - System.out.println("=== Data source: JSON Dataset ==="); - // A JSON dataset is pointed 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 DataFrame from the file(s) pointed by path - DataFrame peopleFromJsonFile = sqlContext.read().json(path); - - // Because the schema of a JSON dataset is automatically inferred, to write queries, - // it is better to take a look at what is the schema. - peopleFromJsonFile.printSchema(); - // The schema of people is ... - // root - // |-- age: IntegerType - // |-- name: StringType - - // Register this DataFrame as a table. - peopleFromJsonFile.registerTempTable("people"); - - // SQL statements can be run by using the sql methods provided by sqlContext. - DataFrame teenagers3 = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); - - // The results of SQL queries are DataFrame and support all the normal RDD operations. - // The columns of a row in the result can be accessed by ordinal. - teenagerNames = teenagers3.toJavaRDD().map(new Function() { - @Override - public String call(Row row) { return "Name: " + row.getString(0); } - }).collect(); - for (String name: teenagerNames) { - System.out.println(name); - } - - // Alternatively, a DataFrame can be created for a JSON dataset represented by - // a RDD[String] storing one JSON object per string. - List jsonData = Arrays.asList( - "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); - JavaRDD anotherPeopleRDD = ctx.parallelize(jsonData); - DataFrame peopleFromJsonRDD = sqlContext.read().json(anotherPeopleRDD.rdd()); - - // Take a look at the schema of this new DataFrame. - peopleFromJsonRDD.printSchema(); - // The schema of anotherPeople is ... - // root - // |-- address: StructType - // | |-- city: StringType - // | |-- state: StringType - // |-- name: StringType - - peopleFromJsonRDD.registerTempTable("people2"); - - DataFrame peopleWithCity = sqlContext.sql("SELECT name, address.city FROM people2"); - List nameAndCity = peopleWithCity.toJavaRDD().map(new Function() { - @Override - public String call(Row row) { - return "Name: " + row.getString(0) + ", City: " + row.getString(1); - } - }).collect(); - for (String name: nameAndCity) { - System.out.println(name); - } - - ctx.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java new file mode 100644 index 000000000000..8605852d0881 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.sql; + +// $example on:programmatic_schema$ +import java.util.ArrayList; +import java.util.List; +// $example off:programmatic_schema$ +// $example on:create_ds$ +import java.util.Arrays; +import java.util.Collections; +import java.io.Serializable; +// $example off:create_ds$ + +// $example on:schema_inferring$ +// $example on:programmatic_schema$ +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.Function; +// $example off:programmatic_schema$ +// $example on:create_ds$ +import org.apache.spark.api.java.function.MapFunction; +// $example on:create_df$ +// $example on:run_sql$ +// $example on:programmatic_schema$ +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +// $example off:programmatic_schema$ +// $example off:create_df$ +// $example off:run_sql$ +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +// $example off:create_ds$ +// $example off:schema_inferring$ +import org.apache.spark.sql.RowFactory; +// $example on:init_session$ +import org.apache.spark.sql.SparkSession; +// $example off:init_session$ +// $example on:programmatic_schema$ +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off:programmatic_schema$ +import org.apache.spark.sql.AnalysisException; + +// $example on:untyped_ops$ +// col("...") is preferable to df.col("...") +import static org.apache.spark.sql.functions.col; +// $example off:untyped_ops$ + +public class JavaSparkSQLExample { + // $example on:create_ds$ + public static class Person implements Serializable { + private String name; + private int age; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public int getAge() { + return age; + } + + public void setAge(int age) { + this.age = age; + } + } + // $example off:create_ds$ + + public static void main(String[] args) throws AnalysisException { + // $example on:init_session$ + SparkSession spark = SparkSession + .builder() + .appName("Java Spark SQL basic example") + .config("spark.some.config.option", "some-value") + .getOrCreate(); + // $example off:init_session$ + + runBasicDataFrameExample(spark); + runDatasetCreationExample(spark); + runInferSchemaExample(spark); + runProgrammaticSchemaExample(spark); + + spark.stop(); + } + + private static void runBasicDataFrameExample(SparkSession spark) throws AnalysisException { + // $example on:create_df$ + Dataset df = spark.read().json("examples/src/main/resources/people.json"); + + // Displays the content of the DataFrame to stdout + df.show(); + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:create_df$ + + // $example on:untyped_ops$ + // Print the schema in a tree format + df.printSchema(); + // root + // |-- age: long (nullable = true) + // |-- name: string (nullable = true) + + // Select only the "name" column + df.select("name").show(); + // +-------+ + // | name| + // +-------+ + // |Michael| + // | Andy| + // | Justin| + // +-------+ + + // Select everybody, but increment the age by 1 + df.select(col("name"), col("age").plus(1)).show(); + // +-------+---------+ + // | name|(age + 1)| + // +-------+---------+ + // |Michael| null| + // | Andy| 31| + // | Justin| 20| + // +-------+---------+ + + // Select people older than 21 + df.filter(col("age").gt(21)).show(); + // +---+----+ + // |age|name| + // +---+----+ + // | 30|Andy| + // +---+----+ + + // Count people by age + df.groupBy("age").count().show(); + // +----+-----+ + // | age|count| + // +----+-----+ + // | 19| 1| + // |null| 1| + // | 30| 1| + // +----+-----+ + // $example off:untyped_ops$ + + // $example on:run_sql$ + // Register the DataFrame as a SQL temporary view + df.createOrReplaceTempView("people"); + + Dataset sqlDF = spark.sql("SELECT * FROM people"); + sqlDF.show(); + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:run_sql$ + + // $example on:global_temp_view$ + // Register the DataFrame as a global temporary view + df.createGlobalTempView("people"); + + // Global temporary view is tied to a system preserved database `global_temp` + spark.sql("SELECT * FROM global_temp.people").show(); + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + + // Global temporary view is cross-session + spark.newSession().sql("SELECT * FROM global_temp.people").show(); + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:global_temp_view$ + } + + private static void runDatasetCreationExample(SparkSession spark) { + // $example on:create_ds$ + // Create an instance of a Bean class + Person person = new Person(); + person.setName("Andy"); + person.setAge(32); + + // Encoders are created for Java beans + Encoder personEncoder = Encoders.bean(Person.class); + Dataset javaBeanDS = spark.createDataset( + Collections.singletonList(person), + personEncoder + ); + javaBeanDS.show(); + // +---+----+ + // |age|name| + // +---+----+ + // | 32|Andy| + // +---+----+ + + // Encoders for most common types are provided in class Encoders + Encoder integerEncoder = Encoders.INT(); + Dataset primitiveDS = spark.createDataset(Arrays.asList(1, 2, 3), integerEncoder); + Dataset transformedDS = primitiveDS.map( + (MapFunction) value -> value + 1, + integerEncoder); + transformedDS.collect(); // Returns [2, 3, 4] + + // DataFrames can be converted to a Dataset by providing a class. Mapping based on name + String path = "examples/src/main/resources/people.json"; + Dataset peopleDS = spark.read().json(path).as(personEncoder); + peopleDS.show(); + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:create_ds$ + } + + private static void runInferSchemaExample(SparkSession spark) { + // $example on:schema_inferring$ + // Create an RDD of Person objects from a text file + JavaRDD peopleRDD = spark.read() + .textFile("examples/src/main/resources/people.txt") + .javaRDD() + .map(line -> { + String[] parts = line.split(","); + Person person = new Person(); + person.setName(parts[0]); + person.setAge(Integer.parseInt(parts[1].trim())); + return person; + }); + + // Apply a schema to an RDD of JavaBeans to get a DataFrame + Dataset peopleDF = spark.createDataFrame(peopleRDD, Person.class); + // Register the DataFrame as a temporary view + peopleDF.createOrReplaceTempView("people"); + + // SQL statements can be run by using the sql methods provided by spark + Dataset teenagersDF = spark.sql("SELECT name FROM people WHERE age BETWEEN 13 AND 19"); + + // The columns of a row in the result can be accessed by field index + Encoder stringEncoder = Encoders.STRING(); + Dataset teenagerNamesByIndexDF = teenagersDF.map( + (MapFunction) row -> "Name: " + row.getString(0), + stringEncoder); + teenagerNamesByIndexDF.show(); + // +------------+ + // | value| + // +------------+ + // |Name: Justin| + // +------------+ + + // or by field name + Dataset teenagerNamesByFieldDF = teenagersDF.map( + (MapFunction) row -> "Name: " + row.getAs("name"), + stringEncoder); + teenagerNamesByFieldDF.show(); + // +------------+ + // | value| + // +------------+ + // |Name: Justin| + // +------------+ + // $example off:schema_inferring$ + } + + private static void runProgrammaticSchemaExample(SparkSession spark) { + // $example on:programmatic_schema$ + // Create an RDD + JavaRDD peopleRDD = spark.sparkContext() + .textFile("examples/src/main/resources/people.txt", 1) + .toJavaRDD(); + + // The schema is encoded in a string + String schemaString = "name age"; + + // Generate the schema based on the string of schema + List fields = new ArrayList<>(); + for (String fieldName : schemaString.split(" ")) { + StructField field = DataTypes.createStructField(fieldName, DataTypes.StringType, true); + fields.add(field); + } + StructType schema = DataTypes.createStructType(fields); + + // Convert records of the RDD (people) to Rows + JavaRDD rowRDD = peopleRDD.map((Function) record -> { + String[] attributes = record.split(","); + return RowFactory.create(attributes[0], attributes[1].trim()); + }); + + // Apply the schema to the RDD + Dataset peopleDataFrame = spark.createDataFrame(rowRDD, schema); + + // Creates a temporary view using the DataFrame + peopleDataFrame.createOrReplaceTempView("people"); + + // SQL can be run over a temporary view created using DataFrames + Dataset results = spark.sql("SELECT name FROM people"); + + // The results of SQL queries are DataFrames and support all the normal RDD operations + // The columns of a row in the result can be accessed by field index or by field name + Dataset namesDS = results.map( + (MapFunction) row -> "Name: " + row.getString(0), + Encoders.STRING()); + namesDS.show(); + // +-------------+ + // | value| + // +-------------+ + // |Name: Michael| + // | Name: Andy| + // | Name: Justin| + // +-------------+ + // $example off:programmatic_schema$ + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java new file mode 100644 index 000000000000..78e9011be470 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java @@ -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.sql; + +// $example on:typed_custom_aggregation$ +import java.io.Serializable; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.TypedColumn; +import org.apache.spark.sql.expressions.Aggregator; +// $example off:typed_custom_aggregation$ + +public class JavaUserDefinedTypedAggregation { + + // $example on:typed_custom_aggregation$ + public static class Employee implements Serializable { + private String name; + private long salary; + + // Constructors, getters, setters... + // $example off:typed_custom_aggregation$ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public long getSalary() { + return salary; + } + + public void setSalary(long salary) { + this.salary = salary; + } + // $example on:typed_custom_aggregation$ + } + + public static class Average implements Serializable { + private long sum; + private long count; + + // Constructors, getters, setters... + // $example off:typed_custom_aggregation$ + public Average() { + } + + public Average(long sum, long count) { + this.sum = sum; + this.count = count; + } + + public long getSum() { + return sum; + } + + public void setSum(long sum) { + this.sum = sum; + } + + public long getCount() { + return count; + } + + public void setCount(long count) { + this.count = count; + } + // $example on:typed_custom_aggregation$ + } + + public static class MyAverage extends Aggregator { + // A zero value for this aggregation. Should satisfy the property that any b + zero = b + public Average zero() { + return new Average(0L, 0L); + } + // Combine two values to produce a new value. For performance, the function may modify `buffer` + // and return it instead of constructing a new object + public Average reduce(Average buffer, Employee employee) { + long newSum = buffer.getSum() + employee.getSalary(); + long newCount = buffer.getCount() + 1; + buffer.setSum(newSum); + buffer.setCount(newCount); + return buffer; + } + // Merge two intermediate values + public Average merge(Average b1, Average b2) { + long mergedSum = b1.getSum() + b2.getSum(); + long mergedCount = b1.getCount() + b2.getCount(); + b1.setSum(mergedSum); + b1.setCount(mergedCount); + return b1; + } + // Transform the output of the reduction + public Double finish(Average reduction) { + return ((double) reduction.getSum()) / reduction.getCount(); + } + // Specifies the Encoder for the intermediate value type + public Encoder bufferEncoder() { + return Encoders.bean(Average.class); + } + // Specifies the Encoder for the final output value type + public Encoder outputEncoder() { + return Encoders.DOUBLE(); + } + } + // $example off:typed_custom_aggregation$ + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("Java Spark SQL user-defined Datasets aggregation example") + .getOrCreate(); + + // $example on:typed_custom_aggregation$ + Encoder employeeEncoder = Encoders.bean(Employee.class); + String path = "examples/src/main/resources/employees.json"; + Dataset ds = spark.read().json(path).as(employeeEncoder); + ds.show(); + // +-------+------+ + // | name|salary| + // +-------+------+ + // |Michael| 3000| + // | Andy| 4500| + // | Justin| 3500| + // | Berta| 4000| + // +-------+------+ + + MyAverage myAverage = new MyAverage(); + // Convert the function to a `TypedColumn` and give it a name + TypedColumn averageSalary = myAverage.toColumn().name("average_salary"); + Dataset result = ds.select(averageSalary); + result.show(); + // +--------------+ + // |average_salary| + // +--------------+ + // | 3750.0| + // +--------------+ + // $example off:typed_custom_aggregation$ + spark.stop(); + } + +} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java new file mode 100644 index 000000000000..6da60a1fc6b8 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql; + +// $example on:untyped_custom_aggregation$ +import java.util.ArrayList; +import java.util.List; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.expressions.MutableAggregationBuffer; +import org.apache.spark.sql.expressions.UserDefinedAggregateFunction; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +// $example off:untyped_custom_aggregation$ + +public class JavaUserDefinedUntypedAggregation { + + // $example on:untyped_custom_aggregation$ + public static class MyAverage extends UserDefinedAggregateFunction { + + private StructType inputSchema; + private StructType bufferSchema; + + public MyAverage() { + List inputFields = new ArrayList<>(); + inputFields.add(DataTypes.createStructField("inputColumn", DataTypes.LongType, true)); + inputSchema = DataTypes.createStructType(inputFields); + + List bufferFields = new ArrayList<>(); + bufferFields.add(DataTypes.createStructField("sum", DataTypes.LongType, true)); + bufferFields.add(DataTypes.createStructField("count", DataTypes.LongType, true)); + bufferSchema = DataTypes.createStructType(bufferFields); + } + // Data types of input arguments of this aggregate function + public StructType inputSchema() { + return inputSchema; + } + // Data types of values in the aggregation buffer + public StructType bufferSchema() { + return bufferSchema; + } + // The data type of the returned value + public DataType dataType() { + return DataTypes.DoubleType; + } + // Whether this function always returns the same output on the identical input + public boolean deterministic() { + return true; + } + // Initializes the given aggregation buffer. The buffer itself is a `Row` that in addition to + // standard methods like retrieving a value at an index (e.g., get(), getBoolean()), provides + // the opportunity to update its values. Note that arrays and maps inside the buffer are still + // immutable. + public void initialize(MutableAggregationBuffer buffer) { + buffer.update(0, 0L); + buffer.update(1, 0L); + } + // Updates the given aggregation buffer `buffer` with new input data from `input` + public void update(MutableAggregationBuffer buffer, Row input) { + if (!input.isNullAt(0)) { + long updatedSum = buffer.getLong(0) + input.getLong(0); + long updatedCount = buffer.getLong(1) + 1; + buffer.update(0, updatedSum); + buffer.update(1, updatedCount); + } + } + // Merges two aggregation buffers and stores the updated buffer values back to `buffer1` + public void merge(MutableAggregationBuffer buffer1, Row buffer2) { + long mergedSum = buffer1.getLong(0) + buffer2.getLong(0); + long mergedCount = buffer1.getLong(1) + buffer2.getLong(1); + buffer1.update(0, mergedSum); + buffer1.update(1, mergedCount); + } + // Calculates the final result + public Double evaluate(Row buffer) { + return ((double) buffer.getLong(0)) / buffer.getLong(1); + } + } + // $example off:untyped_custom_aggregation$ + + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("Java Spark SQL user-defined DataFrames aggregation example") + .getOrCreate(); + + // $example on:untyped_custom_aggregation$ + // Register the function to access it + spark.udf().register("myAverage", new MyAverage()); + + Dataset df = spark.read().json("examples/src/main/resources/employees.json"); + df.createOrReplaceTempView("employees"); + df.show(); + // +-------+------+ + // | name|salary| + // +-------+------+ + // |Michael| 3000| + // | Andy| 4500| + // | Justin| 3500| + // | Berta| 4000| + // +-------+------+ + + Dataset result = spark.sql("SELECT myAverage(salary) as average_salary FROM employees"); + result.show(); + // +--------------+ + // |average_salary| + // +--------------+ + // | 3750.0| + // +--------------+ + // $example off:untyped_custom_aggregation$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java b/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java new file mode 100644 index 000000000000..575a463e8725 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql.hive; + +// $example on:spark_hive$ +import java.io.File; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +// $example off:spark_hive$ + +public class JavaSparkHiveExample { + + // $example on:spark_hive$ + public static class Record implements Serializable { + private int key; + private String value; + + public int getKey() { + return key; + } + + public void setKey(int key) { + this.key = key; + } + + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + } + // $example off:spark_hive$ + + public static void main(String[] args) { + // $example on:spark_hive$ + // warehouseLocation points to the default location for managed databases and tables + String warehouseLocation = new File("spark-warehouse").getAbsolutePath(); + SparkSession spark = SparkSession + .builder() + .appName("Java Spark Hive Example") + .config("spark.sql.warehouse.dir", warehouseLocation) + .enableHiveSupport() + .getOrCreate(); + + spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING) USING hive"); + spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); + + // Queries are expressed in HiveQL + spark.sql("SELECT * FROM src").show(); + // +---+-------+ + // |key| value| + // +---+-------+ + // |238|val_238| + // | 86| val_86| + // |311|val_311| + // ... + + // Aggregation queries are also supported. + spark.sql("SELECT COUNT(*) FROM src").show(); + // +--------+ + // |count(1)| + // +--------+ + // | 500 | + // +--------+ + + // The results of SQL queries are themselves DataFrames and support all normal functions. + Dataset sqlDF = spark.sql("SELECT key, value FROM src WHERE key < 10 ORDER BY key"); + + // The items in DataFrames are of type Row, which lets you to access each column by ordinal. + Dataset stringsDS = sqlDF.map( + (MapFunction) row -> "Key: " + row.get(0) + ", Value: " + row.get(1), + Encoders.STRING()); + stringsDS.show(); + // +--------------------+ + // | value| + // +--------------------+ + // |Key: 0, Value: val_0| + // |Key: 0, Value: val_0| + // |Key: 0, Value: val_0| + // ... + + // You can also use DataFrames to create temporary views within a SparkSession. + List records = new ArrayList<>(); + for (int key = 1; key < 100; key++) { + Record record = new Record(); + record.setKey(key); + record.setValue("val_" + key); + records.add(record); + } + Dataset recordsDF = spark.createDataFrame(records, Record.class); + recordsDF.createOrReplaceTempView("records"); + + // Queries can then join DataFrames data with data stored in Hive. + spark.sql("SELECT * FROM records r JOIN src s ON r.key = s.key").show(); + // +---+------+---+------+ + // |key| value|key| value| + // +---+------+---+------+ + // | 2| val_2| 2| val_2| + // | 2| val_2| 2| val_2| + // | 4| val_4| 4| val_4| + // ... + // $example off:spark_hive$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredKafkaWordCount.java b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredKafkaWordCount.java new file mode 100644 index 000000000000..4e02719e043a --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredKafkaWordCount.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.sql.streaming; + +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.streaming.StreamingQuery; + +import java.util.Arrays; + +/** + * Consumes messages from one or more topics in Kafka and does wordcount. + * Usage: JavaStructuredKafkaWordCount + * The Kafka "bootstrap.servers" configuration. A + * comma-separated list of host:port. + * There are three kinds of type, i.e. 'assign', 'subscribe', + * 'subscribePattern'. + * |- Specific TopicPartitions to consume. Json string + * | {"topicA":[0,1],"topicB":[2,4]}. + * |- The topic list to subscribe. A comma-separated list of + * | topics. + * |- The pattern used to subscribe to topic(s). + * | Java regex string. + * |- Only one of "assign, "subscribe" or "subscribePattern" options can be + * | specified for Kafka source. + * Different value format depends on the value of 'subscribe-type'. + * + * Example: + * `$ bin/run-example \ + * sql.streaming.JavaStructuredKafkaWordCount host1:port1,host2:port2 \ + * subscribe topic1,topic2` + */ +public final class JavaStructuredKafkaWordCount { + + public static void main(String[] args) throws Exception { + if (args.length < 3) { + System.err.println("Usage: JavaStructuredKafkaWordCount " + + " "); + System.exit(1); + } + + String bootstrapServers = args[0]; + String subscribeType = args[1]; + String topics = args[2]; + + SparkSession spark = SparkSession + .builder() + .appName("JavaStructuredKafkaWordCount") + .getOrCreate(); + + // Create DataSet representing the stream of input lines from kafka + Dataset lines = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", bootstrapServers) + .option(subscribeType, topics) + .load() + .selectExpr("CAST(value AS STRING)") + .as(Encoders.STRING()); + + // Generate running word count + Dataset wordCounts = lines.flatMap( + (FlatMapFunction) x -> Arrays.asList(x.split(" ")).iterator(), + Encoders.STRING()).groupBy("value").count(); + + // Start running the query that prints the running counts to the console + StreamingQuery query = wordCounts.writeStream() + .outputMode("complete") + .format("console") + .start(); + + query.awaitTermination(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java new file mode 100644 index 000000000000..3af786978b16 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql.streaming; + +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.sql.*; +import org.apache.spark.sql.streaming.StreamingQuery; + +import java.util.Arrays; + +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network. + * + * Usage: JavaStructuredNetworkWordCount + * and describe the TCP server that Structured 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 sql.streaming.JavaStructuredNetworkWordCount + * localhost 9999` + */ +public final class JavaStructuredNetworkWordCount { + + public static void main(String[] args) throws Exception { + if (args.length < 2) { + System.err.println("Usage: JavaStructuredNetworkWordCount "); + System.exit(1); + } + + String host = args[0]; + int port = Integer.parseInt(args[1]); + + SparkSession spark = SparkSession + .builder() + .appName("JavaStructuredNetworkWordCount") + .getOrCreate(); + + // Create DataFrame representing the stream of input lines from connection to host:port + Dataset lines = spark + .readStream() + .format("socket") + .option("host", host) + .option("port", port) + .load(); + + // Split the lines into words + Dataset words = lines.as(Encoders.STRING()).flatMap( + (FlatMapFunction) x -> Arrays.asList(x.split(" ")).iterator(), + Encoders.STRING()); + + // Generate running word count + Dataset wordCounts = words.groupBy("value").count(); + + // Start running the query that prints the running counts to the console + StreamingQuery query = wordCounts.writeStream() + .outputMode("complete") + .format("console") + .start(); + + query.awaitTermination(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCountWindowed.java b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCountWindowed.java new file mode 100644 index 000000000000..93ec5e269515 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCountWindowed.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.sql.streaming; + +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.sql.*; +import org.apache.spark.sql.streaming.StreamingQuery; +import scala.Tuple2; + +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.List; + +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network over a + * sliding window of configurable duration. Each line from the network is tagged + * with a timestamp that is used to determine the windows into which it falls. + * + * Usage: JavaStructuredNetworkWordCountWindowed + * [] + * and describe the TCP server that Structured Streaming + * would connect to receive data. + * gives the size of window, specified as integer number of seconds + * gives the amount of time successive windows are offset from one another, + * given in the same units as above. should be less than or equal to + * . If the two are equal, successive windows have no overlap. If + * is not provided, it defaults to . + * + * 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 sql.streaming.JavaStructuredNetworkWordCountWindowed + * localhost 9999 []` + * + * One recommended , pair is 10, 5 + */ +public final class JavaStructuredNetworkWordCountWindowed { + + public static void main(String[] args) throws Exception { + if (args.length < 3) { + System.err.println("Usage: JavaStructuredNetworkWordCountWindowed " + + " []"); + System.exit(1); + } + + String host = args[0]; + int port = Integer.parseInt(args[1]); + int windowSize = Integer.parseInt(args[2]); + int slideSize = (args.length == 3) ? windowSize : Integer.parseInt(args[3]); + if (slideSize > windowSize) { + System.err.println(" must be less than or equal to "); + } + String windowDuration = windowSize + " seconds"; + String slideDuration = slideSize + " seconds"; + + SparkSession spark = SparkSession + .builder() + .appName("JavaStructuredNetworkWordCountWindowed") + .getOrCreate(); + + // Create DataFrame representing the stream of input lines from connection to host:port + Dataset lines = spark + .readStream() + .format("socket") + .option("host", host) + .option("port", port) + .option("includeTimestamp", true) + .load(); + + // Split the lines into words, retaining timestamps + Dataset words = lines + .as(Encoders.tuple(Encoders.STRING(), Encoders.TIMESTAMP())) + .flatMap((FlatMapFunction, Tuple2>) t -> { + List> result = new ArrayList<>(); + for (String word : t._1.split(" ")) { + result.add(new Tuple2<>(word, t._2)); + } + return result.iterator(); + }, + Encoders.tuple(Encoders.STRING(), Encoders.TIMESTAMP()) + ).toDF("word", "timestamp"); + + // Group the data by window and word and compute the count of each group + Dataset windowedCounts = words.groupBy( + functions.window(words.col("timestamp"), windowDuration, slideDuration), + words.col("word") + ).count().orderBy("window"); + + // Start running the query that prints the windowed word counts to the console + StreamingQuery query = windowedCounts.writeStream() + .outputMode("complete") + .format("console") + .option("truncate", "false") + .start(); + + query.awaitTermination(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java new file mode 100644 index 000000000000..943e3d82f30f --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java @@ -0,0 +1,251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.sql.streaming; + +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.MapGroupsWithStateFunction; +import org.apache.spark.sql.*; +import org.apache.spark.sql.streaming.GroupState; +import org.apache.spark.sql.streaming.GroupStateTimeout; +import org.apache.spark.sql.streaming.StreamingQuery; + +import java.io.Serializable; +import java.sql.Timestamp; +import java.util.*; + +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network. + *

    + * Usage: JavaStructuredNetworkWordCount + * and describe the TCP server that Structured 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 sql.streaming.JavaStructuredSessionization + * localhost 9999` + */ +public final class JavaStructuredSessionization { + + public static void main(String[] args) throws Exception { + if (args.length < 2) { + System.err.println("Usage: JavaStructuredSessionization "); + System.exit(1); + } + + String host = args[0]; + int port = Integer.parseInt(args[1]); + + SparkSession spark = SparkSession + .builder() + .appName("JavaStructuredSessionization") + .getOrCreate(); + + // Create DataFrame representing the stream of input lines from connection to host:port + Dataset lines = spark + .readStream() + .format("socket") + .option("host", host) + .option("port", port) + .option("includeTimestamp", true) + .load(); + + FlatMapFunction linesToEvents = + new FlatMapFunction() { + @Override + public Iterator call(LineWithTimestamp lineWithTimestamp) { + ArrayList eventList = new ArrayList(); + for (String word : lineWithTimestamp.getLine().split(" ")) { + eventList.add(new Event(word, lineWithTimestamp.getTimestamp())); + } + return eventList.iterator(); + } + }; + + // Split the lines into words, treat words as sessionId of events + Dataset events = lines + .withColumnRenamed("value", "line") + .as(Encoders.bean(LineWithTimestamp.class)) + .flatMap(linesToEvents, Encoders.bean(Event.class)); + + // Sessionize the events. Track number of events, start and end timestamps of session, and + // and report session updates. + // + // Step 1: Define the state update function + MapGroupsWithStateFunction stateUpdateFunc = + new MapGroupsWithStateFunction() { + @Override public SessionUpdate call( + String sessionId, Iterator events, GroupState state) { + // If timed out, then remove session and send final update + if (state.hasTimedOut()) { + SessionUpdate finalUpdate = new SessionUpdate( + sessionId, state.get().calculateDuration(), state.get().getNumEvents(), true); + state.remove(); + return finalUpdate; + + } else { + // Find max and min timestamps in events + long maxTimestampMs = Long.MIN_VALUE; + long minTimestampMs = Long.MAX_VALUE; + int numNewEvents = 0; + while (events.hasNext()) { + Event e = events.next(); + long timestampMs = e.getTimestamp().getTime(); + maxTimestampMs = Math.max(timestampMs, maxTimestampMs); + minTimestampMs = Math.min(timestampMs, minTimestampMs); + numNewEvents += 1; + } + SessionInfo updatedSession = new SessionInfo(); + + // Update start and end timestamps in session + if (state.exists()) { + SessionInfo oldSession = state.get(); + updatedSession.setNumEvents(oldSession.numEvents + numNewEvents); + updatedSession.setStartTimestampMs(oldSession.startTimestampMs); + updatedSession.setEndTimestampMs(Math.max(oldSession.endTimestampMs, maxTimestampMs)); + } else { + updatedSession.setNumEvents(numNewEvents); + updatedSession.setStartTimestampMs(minTimestampMs); + updatedSession.setEndTimestampMs(maxTimestampMs); + } + state.update(updatedSession); + // Set timeout such that the session will be expired if no data received for 10 seconds + state.setTimeoutDuration("10 seconds"); + return new SessionUpdate( + sessionId, state.get().calculateDuration(), state.get().getNumEvents(), false); + } + } + }; + + // Step 2: Apply the state update function to the events streaming Dataset grouped by sessionId + Dataset sessionUpdates = events + .groupByKey( + new MapFunction() { + @Override public String call(Event event) { + return event.getSessionId(); + } + }, Encoders.STRING()) + .mapGroupsWithState( + stateUpdateFunc, + Encoders.bean(SessionInfo.class), + Encoders.bean(SessionUpdate.class), + GroupStateTimeout.ProcessingTimeTimeout()); + + // Start running the query that prints the session updates to the console + StreamingQuery query = sessionUpdates + .writeStream() + .outputMode("update") + .format("console") + .start(); + + query.awaitTermination(); + } + + /** + * User-defined data type representing the raw lines with timestamps. + */ + public static class LineWithTimestamp implements Serializable { + private String line; + private Timestamp timestamp; + + public Timestamp getTimestamp() { return timestamp; } + public void setTimestamp(Timestamp timestamp) { this.timestamp = timestamp; } + + public String getLine() { return line; } + public void setLine(String sessionId) { this.line = sessionId; } + } + + /** + * User-defined data type representing the input events + */ + public static class Event implements Serializable { + private String sessionId; + private Timestamp timestamp; + + public Event() { } + public Event(String sessionId, Timestamp timestamp) { + this.sessionId = sessionId; + this.timestamp = timestamp; + } + + public Timestamp getTimestamp() { return timestamp; } + public void setTimestamp(Timestamp timestamp) { this.timestamp = timestamp; } + + public String getSessionId() { return sessionId; } + public void setSessionId(String sessionId) { this.sessionId = sessionId; } + } + + /** + * User-defined data type for storing a session information as state in mapGroupsWithState. + */ + public static class SessionInfo implements Serializable { + private int numEvents = 0; + private long startTimestampMs = -1; + private long endTimestampMs = -1; + + public int getNumEvents() { return numEvents; } + public void setNumEvents(int numEvents) { this.numEvents = numEvents; } + + public long getStartTimestampMs() { return startTimestampMs; } + public void setStartTimestampMs(long startTimestampMs) { + this.startTimestampMs = startTimestampMs; + } + + public long getEndTimestampMs() { return endTimestampMs; } + public void setEndTimestampMs(long endTimestampMs) { this.endTimestampMs = endTimestampMs; } + + public long calculateDuration() { return endTimestampMs - startTimestampMs; } + + @Override public String toString() { + return "SessionInfo(numEvents = " + numEvents + + ", timestamps = " + startTimestampMs + " to " + endTimestampMs + ")"; + } + } + + /** + * User-defined data type representing the update information returned by mapGroupsWithState. + */ + public static class SessionUpdate implements Serializable { + private String id; + private long durationMs; + private int numEvents; + private boolean expired; + + public SessionUpdate() { } + + public SessionUpdate(String id, long durationMs, int numEvents, boolean expired) { + this.id = id; + this.durationMs = durationMs; + this.numEvents = numEvents; + this.expired = expired; + } + + public String getId() { return id; } + public void setId(String id) { this.id = id; } + + public long getDurationMs() { return durationMs; } + public void setDurationMs(long durationMs) { this.durationMs = durationMs; } + + public int getNumEvents() { return numEvents; } + public void setNumEvents(int numEvents) { this.numEvents = numEvents; } + + public boolean isExpired() { return expired; } + public void setExpired(boolean expired) { this.expired = expired; } + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java index 99df259b4e8e..47692ec98289 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java @@ -17,12 +17,9 @@ package org.apache.spark.examples.streaming; -import com.google.common.collect.Lists; +import com.google.common.io.Closeables; import org.apache.spark.SparkConf; -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.storage.StorageLevel; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; @@ -36,6 +33,8 @@ import java.io.InputStreamReader; import java.net.ConnectException; import java.net.Socket; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.regex.Pattern; /** @@ -55,7 +54,7 @@ public class JavaCustomReceiver extends Receiver { private static final Pattern SPACE = Pattern.compile(" "); - public static void main(String[] args) { + public static void main(String[] args) throws Exception { if (args.length < 2) { System.err.println("Usage: JavaCustomReceiver "); System.exit(1); @@ -67,27 +66,13 @@ public static void main(String[] args) { SparkConf sparkConf = new SparkConf().setAppName("JavaCustomReceiver"); JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000)); - // Create a input stream with the custom receiver on target ip:port and count the + // Create an input stream with the custom receiver on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') JavaReceiverInputDStream lines = ssc.receiverStream( new JavaCustomReceiver(args[0], Integer.parseInt(args[1]))); - JavaDStream words = lines.flatMap(new FlatMapFunction() { - @Override - public Iterable call(String x) { - return Lists.newArrayList(SPACE.split(x)); - } - }); - JavaPairDStream wordCounts = words.mapToPair( - new PairFunction() { - @Override public Tuple2 call(String s) { - return new Tuple2(s, 1); - } - }).reduceByKey(new Function2() { - @Override - public Integer call(Integer i1, Integer i2) { - return i1 + i2; - } - }); + JavaDStream words = lines.flatMap(x -> Arrays.asList(SPACE.split(x)).iterator()); + JavaPairDStream wordCounts = words.mapToPair(s -> new Tuple2<>(s, 1)) + .reduceByKey((i1, i2) -> i1 + i2); wordCounts.print(); ssc.start(); @@ -105,15 +90,13 @@ public JavaCustomReceiver(String host_ , int port_) { port = port_; } + @Override public void onStart() { // Start the thread that receives data over a connection - new Thread() { - @Override public void run() { - receive(); - } - }.start(); + new Thread(this::receive).start(); } + @Override public void onStop() { // There is nothing much to do as the thread calling receive() // is designed to stop by itself isStopped() returns false @@ -121,23 +104,24 @@ public void onStop() { /** Create a socket connection and receive data until receiver is stopped */ private void receive() { - Socket socket = null; - String userInput = null; - try { - // connect to the server - socket = new Socket(host, port); - - BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream())); - - // Until stopped or connection broken continue reading - while (!isStopped() && (userInput = reader.readLine()) != null) { - System.out.println("Received data '" + userInput + "'"); - store(userInput); + Socket socket = null; + BufferedReader reader = null; + try { + // connect to the server + socket = new Socket(host, port); + reader = new BufferedReader( + new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8)); + // Until stopped or connection broken continue reading + String userInput; + while (!isStopped() && (userInput = reader.readLine()) != null) { + System.out.println("Received data '" + userInput + "'"); + store(userInput); + } + } finally { + Closeables.close(reader, /* swallowIOException = */ true); + Closeables.close(socket, /* swallowIOException = */ true); } - reader.close(); - socket.close(); - // Restart in an attempt to connect again when server is active again restart("Trying to connect again"); } catch(ConnectException ce) { diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java index bab9f2478e77..b6b163fa8b2c 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java @@ -20,35 +20,38 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Arrays; +import java.util.Map; +import java.util.Set; import java.util.regex.Pattern; import scala.Tuple2; -import com.google.common.collect.Lists; -import kafka.serializer.StringDecoder; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.function.*; import org.apache.spark.streaming.api.java.*; -import org.apache.spark.streaming.kafka.KafkaUtils; +import org.apache.spark.streaming.kafka010.ConsumerStrategies; +import org.apache.spark.streaming.kafka010.KafkaUtils; +import org.apache.spark.streaming.kafka010.LocationStrategies; import org.apache.spark.streaming.Durations; /** * Consumes messages from one or more topics in Kafka and does wordcount. - * Usage: DirectKafkaWordCount + * Usage: JavaDirectKafkaWordCount * is a list of one or more Kafka brokers * is a list of one or more kafka topics to consume from * * Example: - * $ bin/run-example streaming.KafkaWordCount broker1-host:port,broker2-host:port topic1,topic2 + * $ bin/run-example streaming.JavaDirectKafkaWordCount broker1-host:port,broker2-host:port \ + * topic1,topic2 */ public final class JavaDirectKafkaWordCount { private static final Pattern SPACE = Pattern.compile(" "); - public static void main(String[] args) { + public static void main(String[] args) throws Exception { if (args.length < 2) { - System.err.println("Usage: DirectKafkaWordCount \n" + + System.err.println("Usage: JavaDirectKafkaWordCount \n" + " is a list of one or more Kafka brokers\n" + " is a list of one or more kafka topics to consume from\n\n"); System.exit(1); @@ -59,51 +62,25 @@ public static void main(String[] args) { String brokers = args[0]; String topics = args[1]; - // Create context with 2 second batch interval + // Create context with a 2 seconds batch interval SparkConf sparkConf = new SparkConf().setAppName("JavaDirectKafkaWordCount"); JavaStreamingContext jssc = new JavaStreamingContext(sparkConf, Durations.seconds(2)); - HashSet topicsSet = new HashSet(Arrays.asList(topics.split(","))); - HashMap kafkaParams = new HashMap(); + Set topicsSet = new HashSet<>(Arrays.asList(topics.split(","))); + Map kafkaParams = new HashMap<>(); kafkaParams.put("metadata.broker.list", brokers); // Create direct kafka stream with brokers and topics - JavaPairInputDStream messages = KafkaUtils.createDirectStream( + JavaInputDStream> messages = KafkaUtils.createDirectStream( jssc, - String.class, - String.class, - StringDecoder.class, - StringDecoder.class, - kafkaParams, - topicsSet - ); + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Subscribe(topicsSet, kafkaParams)); // Get the lines, split them into words, count the words and print - JavaDStream lines = messages.map(new Function, String>() { - @Override - public String call(Tuple2 tuple2) { - return tuple2._2(); - } - }); - JavaDStream words = lines.flatMap(new FlatMapFunction() { - @Override - public Iterable call(String x) { - return Lists.newArrayList(SPACE.split(x)); - } - }); - JavaPairDStream wordCounts = words.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(String s) { - return new Tuple2(s, 1); - } - }).reduceByKey( - new Function2() { - @Override - public Integer call(Integer i1, Integer i2) { - return i1 + i2; - } - }); + JavaDStream lines = messages.map(ConsumerRecord::value); + JavaDStream words = lines.flatMap(x -> Arrays.asList(SPACE.split(x)).iterator()); + JavaPairDStream wordCounts = words.mapToPair(s -> new Tuple2<>(s, 1)) + .reduceByKey((i1, i2) -> i1 + i2); wordCounts.print(); // Start the computation diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java index da56637fe891..0c651049d0ff 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java @@ -18,8 +18,6 @@ package org.apache.spark.examples.streaming; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.examples.streaming.StreamingExamples; import org.apache.spark.streaming.*; import org.apache.spark.streaming.api.java.*; import org.apache.spark.streaming.flume.FlumeUtils; @@ -44,7 +42,7 @@ public final class JavaFlumeEventCount { private JavaFlumeEventCount() { } - public static void main(String[] args) { + public static void main(String[] args) throws Exception { if (args.length != 2) { System.err.println("Usage: JavaFlumeEventCount "); System.exit(1); @@ -58,16 +56,12 @@ public static void main(String[] args) { Duration batchInterval = new Duration(2000); SparkConf sparkConf = new SparkConf().setAppName("JavaFlumeEventCount"); JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, batchInterval); - JavaReceiverInputDStream flumeStream = FlumeUtils.createStream(ssc, host, port); + JavaReceiverInputDStream flumeStream = + FlumeUtils.createStream(ssc, host, port); flumeStream.count(); - flumeStream.count().map(new Function() { - @Override - public String call(Long in) { - return "Received " + in + " flume events."; - } - }).print(); + flumeStream.count().map(in -> "Received " + in + " flume events.").print(); ssc.start(); ssc.awaitTermination(); diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java deleted file mode 100644 index 16ae9a3319ee..000000000000 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java +++ /dev/null @@ -1,113 +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.examples.streaming; - -import java.util.Map; -import java.util.HashMap; -import java.util.regex.Pattern; - - -import scala.Tuple2; - -import com.google.common.collect.Lists; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.function.FlatMapFunction; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.Function2; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.examples.streaming.StreamingExamples; -import org.apache.spark.streaming.Duration; -import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.JavaPairDStream; -import org.apache.spark.streaming.api.java.JavaPairReceiverInputDStream; -import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.apache.spark.streaming.kafka.KafkaUtils; - -/** - * Consumes messages from one or more topics in Kafka and does wordcount. - * - * Usage: JavaKafkaWordCount - * is a list of one or more zookeeper servers that make quorum - * is the name of kafka consumer group - * is a list of one or more kafka topics to consume from - * is the number of threads the kafka consumer should use - * - * To run this example: - * `$ bin/run-example org.apache.spark.examples.streaming.JavaKafkaWordCount zoo01,zoo02, \ - * zoo03 my-consumer-group topic1,topic2 1` - */ - -public final class JavaKafkaWordCount { - private static final Pattern SPACE = Pattern.compile(" "); - - private JavaKafkaWordCount() { - } - - public static void main(String[] args) { - if (args.length < 4) { - System.err.println("Usage: JavaKafkaWordCount "); - System.exit(1); - } - - StreamingExamples.setStreamingLogLevels(); - SparkConf sparkConf = new SparkConf().setAppName("JavaKafkaWordCount"); - // Create the context with a 1 second batch size - JavaStreamingContext jssc = new JavaStreamingContext(sparkConf, new Duration(2000)); - - int numThreads = Integer.parseInt(args[3]); - Map topicMap = new HashMap(); - String[] topics = args[2].split(","); - for (String topic: topics) { - topicMap.put(topic, numThreads); - } - - JavaPairReceiverInputDStream messages = - KafkaUtils.createStream(jssc, args[0], args[1], topicMap); - - JavaDStream lines = messages.map(new Function, String>() { - @Override - public String call(Tuple2 tuple2) { - return tuple2._2(); - } - }); - - JavaDStream words = lines.flatMap(new FlatMapFunction() { - @Override - public Iterable call(String x) { - return Lists.newArrayList(SPACE.split(x)); - } - }); - - JavaPairDStream wordCounts = words.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(String s) { - return new Tuple2(s, 1); - } - }).reduceByKey(new Function2() { - @Override - public Integer call(Integer i1, Integer i2) { - return i1 + i2; - } - }); - - wordCounts.print(); - jssc.start(); - jssc.awaitTermination(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java index 3e9f0f4b8f12..b217672def88 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java @@ -17,13 +17,12 @@ package org.apache.spark.examples.streaming; +import java.util.Arrays; +import java.util.regex.Pattern; + import scala.Tuple2; -import com.google.common.collect.Lists; import org.apache.spark.SparkConf; -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.api.java.StorageLevels; import org.apache.spark.streaming.Durations; import org.apache.spark.streaming.api.java.JavaDStream; @@ -31,8 +30,6 @@ import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import java.util.regex.Pattern; - /** * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. * @@ -47,7 +44,7 @@ public final class JavaNetworkWordCount { private static final Pattern SPACE = Pattern.compile(" "); - public static void main(String[] args) { + public static void main(String[] args) throws Exception { if (args.length < 2) { System.err.println("Usage: JavaNetworkWordCount "); System.exit(1); @@ -65,24 +62,9 @@ public static void main(String[] args) { // Replication necessary in distributed scenario for fault tolerance. JavaReceiverInputDStream lines = ssc.socketTextStream( args[0], Integer.parseInt(args[1]), StorageLevels.MEMORY_AND_DISK_SER); - JavaDStream words = lines.flatMap(new FlatMapFunction() { - @Override - public Iterable call(String x) { - return Lists.newArrayList(SPACE.split(x)); - } - }); - JavaPairDStream wordCounts = words.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(String s) { - return new Tuple2(s, 1); - } - }).reduceByKey(new Function2() { - @Override - public Integer call(Integer i1, Integer i2) { - return i1 + i2; - } - }); + JavaDStream words = lines.flatMap(x -> Arrays.asList(SPACE.split(x)).iterator()); + JavaPairDStream wordCounts = words.mapToPair(s -> new Tuple2<>(s, 1)) + .reduceByKey((i1, i2) -> i1 + i2); wordCounts.print(); ssc.start(); diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java index 4ce8437f8270..e86f8ab38a74 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java @@ -17,20 +17,15 @@ package org.apache.spark.examples.streaming; - +import java.util.ArrayList; import java.util.LinkedList; import java.util.List; import java.util.Queue; 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.function.Function2; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.examples.streaming.StreamingExamples; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; @@ -50,14 +45,14 @@ public static void main(String[] args) throws Exception { // Create the queue through which RDDs can be pushed to // a QueueInputDStream - Queue> rddQueue = new LinkedList>(); // Create and push some RDDs into the queue - List list = Lists.newArrayList(); + List list = new ArrayList<>(); for (int i = 0; i < 1000; i++) { list.add(i); } + Queue> rddQueue = new LinkedList<>(); for (int i = 0; i < 30; i++) { rddQueue.add(ssc.sparkContext().parallelize(list)); } @@ -65,19 +60,9 @@ public static void main(String[] args) throws Exception { // Create the QueueInputDStream and use it do some processing JavaDStream inputStream = ssc.queueStream(rddQueue); JavaPairDStream mappedStream = inputStream.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(Integer i) { - return new Tuple2(i % 10, 1); - } - }); + i -> new Tuple2<>(i % 10, 1)); JavaPairDStream reducedStream = mappedStream.reduceByKey( - new Function2() { - @Override - public Integer call(Integer i1, Integer i2) { - return i1 + i2; - } - }); + (i1, i2) -> i1 + i2); reducedStream.print(); ssc.start(); diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java index bceda97f058e..45a876decff8 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java @@ -18,30 +18,69 @@ package org.apache.spark.examples.streaming; import java.io.File; -import java.io.IOException; import java.nio.charset.Charset; import java.util.Arrays; +import java.util.List; import java.util.regex.Pattern; import scala.Tuple2; -import com.google.common.collect.Lists; + import com.google.common.io.Files; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -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.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.*; +import org.apache.spark.broadcast.Broadcast; import org.apache.spark.streaming.Durations; -import org.apache.spark.streaming.Time; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaStreamingContextFactory; +import org.apache.spark.util.LongAccumulator; /** - * Counts words in text encoded with UTF8 received from the network every second. + * Use this singleton to get or register a Broadcast variable. + */ +class JavaWordBlacklist { + + private static volatile Broadcast> instance = null; + + public static Broadcast> getInstance(JavaSparkContext jsc) { + if (instance == null) { + synchronized (JavaWordBlacklist.class) { + if (instance == null) { + List wordBlacklist = Arrays.asList("a", "b", "c"); + instance = jsc.broadcast(wordBlacklist); + } + } + } + return instance; + } +} + +/** + * Use this singleton to get or register an Accumulator. + */ +class JavaDroppedWordsCounter { + + private static volatile LongAccumulator instance = null; + + public static LongAccumulator getInstance(JavaSparkContext jsc) { + if (instance == null) { + synchronized (JavaDroppedWordsCounter.class) { + if (instance == null) { + instance = jsc.sc().longAccumulator("WordsInBlacklistCounter"); + } + } + } + return instance; + } +} + +/** + * Counts words in text encoded with UTF8 received from the network every second. This example also + * shows how to use lazily instantiated singleton instances for Accumulator and Broadcast so that + * they can be registered on driver failures. * * Usage: JavaRecoverableNetworkWordCount * and describe the TCP server that Spark Streaming would connect to receive @@ -77,7 +116,7 @@ private static JavaStreamingContext createContext(String ip, // If you do not see this printed, that means the StreamingContext has been loaded // from the new checkpoint System.out.println("Creating new context"); - final File outputFile = new File(outputPath); + File outputFile = new File(outputPath); if (outputFile.exists()) { outputFile.delete(); } @@ -89,40 +128,37 @@ private static JavaStreamingContext createContext(String ip, // Create a socket stream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') JavaReceiverInputDStream lines = ssc.socketTextStream(ip, port); - JavaDStream words = lines.flatMap(new FlatMapFunction() { - @Override - public Iterable call(String x) { - return Lists.newArrayList(SPACE.split(x)); - } - }); - JavaPairDStream wordCounts = words.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(String s) { - return new Tuple2(s, 1); - } - }).reduceByKey(new Function2() { - @Override - public Integer call(Integer i1, Integer i2) { - return i1 + i2; + JavaDStream words = lines.flatMap(x -> Arrays.asList(SPACE.split(x)).iterator()); + JavaPairDStream wordCounts = words.mapToPair(s -> new Tuple2<>(s, 1)) + .reduceByKey((i1, i2) -> i1 + i2); + + wordCounts.foreachRDD((rdd, time) -> { + // Get or register the blacklist Broadcast + Broadcast> blacklist = + JavaWordBlacklist.getInstance(new JavaSparkContext(rdd.context())); + // Get or register the droppedWordsCounter Accumulator + LongAccumulator droppedWordsCounter = + JavaDroppedWordsCounter.getInstance(new JavaSparkContext(rdd.context())); + // Use blacklist to drop words and use droppedWordsCounter to count them + String counts = rdd.filter(wordCount -> { + if (blacklist.value().contains(wordCount._1())) { + droppedWordsCounter.add(wordCount._2()); + return false; + } else { + return true; } - }); - - wordCounts.foreachRDD(new Function2, Time, Void>() { - @Override - public Void call(JavaPairRDD rdd, Time time) throws IOException { - String counts = "Counts at time " + time + " " + rdd.collect(); - System.out.println(counts); - System.out.println("Appending to " + outputFile.getAbsolutePath()); - Files.append(counts + "\n", outputFile, Charset.defaultCharset()); - return null; - } + }).collect().toString(); + String output = "Counts at time " + time + " " + counts; + System.out.println(output); + System.out.println("Dropped " + droppedWordsCounter.value() + " word(s) totally"); + System.out.println("Appending to " + outputFile.getAbsolutePath()); + Files.append(output + "\n", outputFile, Charset.defaultCharset()); }); return ssc; } - public static void main(String[] args) { + public static void main(String[] args) throws Exception { if (args.length != 4) { System.err.println("You arguments were " + Arrays.asList(args)); System.err.println( @@ -137,17 +173,18 @@ public static void main(String[] args) { System.exit(1); } - final String ip = args[0]; - final int port = Integer.parseInt(args[1]); - final String checkpointDirectory = args[2]; - final String outputPath = args[3]; - JavaStreamingContextFactory factory = new JavaStreamingContextFactory() { - @Override - public JavaStreamingContext create() { - return createContext(ip, port, checkpointDirectory, outputPath); - } - }; - JavaStreamingContext ssc = JavaStreamingContext.getOrCreate(checkpointDirectory, factory); + String ip = args[0]; + int port = Integer.parseInt(args[1]); + String checkpointDirectory = args[2]; + String outputPath = args[3]; + + // Function to create JavaStreamingContext without any output operations + // (used to detect the new context) + Function0 createContextFunc = + () -> createContext(ip, port, checkpointDirectory, outputPath); + + JavaStreamingContext ssc = + JavaStreamingContext.getOrCreate(checkpointDirectory, createContextFunc); ssc.start(); ssc.awaitTermination(); } diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java index 46562ddbbcb5..948d1a211178 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java @@ -17,21 +17,16 @@ package org.apache.spark.examples.streaming; +import java.util.Arrays; import java.util.regex.Pattern; -import com.google.common.collect.Lists; - import org.apache.spark.SparkConf; -import org.apache.spark.SparkContext; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.FlatMapFunction; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.Function2; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; import org.apache.spark.api.java.StorageLevels; import org.apache.spark.streaming.Durations; -import org.apache.spark.streaming.Time; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; @@ -48,11 +43,10 @@ * and then run the example * `$ bin/run-example org.apache.spark.examples.streaming.JavaSqlNetworkWordCount localhost 9999` */ - public final class JavaSqlNetworkWordCount { private static final Pattern SPACE = Pattern.compile(" "); - public static void main(String[] args) { + public static void main(String[] args) throws Exception { if (args.length < 2) { System.err.println("Usage: JavaNetworkWordCount "); System.exit(1); @@ -70,39 +64,28 @@ public static void main(String[] args) { // Replication necessary in distributed scenario for fault tolerance. JavaReceiverInputDStream lines = ssc.socketTextStream( args[0], Integer.parseInt(args[1]), StorageLevels.MEMORY_AND_DISK_SER); - JavaDStream words = lines.flatMap(new FlatMapFunction() { - @Override - public Iterable call(String x) { - return Lists.newArrayList(SPACE.split(x)); - } - }); + JavaDStream words = lines.flatMap(x -> Arrays.asList(SPACE.split(x)).iterator()); // Convert RDDs of the words DStream to DataFrame and run SQL query - words.foreachRDD(new Function2, Time, Void>() { - @Override - public Void call(JavaRDD rdd, Time time) { - SQLContext sqlContext = JavaSQLContextSingleton.getInstance(rdd.context()); - - // Convert JavaRDD[String] to JavaRDD[bean class] to DataFrame - JavaRDD rowRDD = rdd.map(new Function() { - public JavaRecord call(String word) { - JavaRecord record = new JavaRecord(); - record.setWord(word); - return record; - } - }); - DataFrame wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRecord.class); - - // Register as table - wordsDataFrame.registerTempTable("words"); - - // Do word count on table using SQL and print it - DataFrame wordCountsDataFrame = - sqlContext.sql("select word, count(*) as total from words group by word"); - System.out.println("========= " + time + "========="); - wordCountsDataFrame.show(); - return null; - } + words.foreachRDD((rdd, time) -> { + SparkSession spark = JavaSparkSessionSingleton.getInstance(rdd.context().getConf()); + + // Convert JavaRDD[String] to JavaRDD[bean class] to DataFrame + JavaRDD rowRDD = rdd.map(word -> { + JavaRecord record = new JavaRecord(); + record.setWord(word); + return record; + }); + Dataset wordsDataFrame = spark.createDataFrame(rowRDD, JavaRecord.class); + + // Creates a temporary view using the DataFrame + wordsDataFrame.createOrReplaceTempView("words"); + + // Do word count on table using SQL and print it + Dataset wordCountsDataFrame = + spark.sql("select word, count(*) as total from words group by word"); + System.out.println("========= " + time + "========="); + wordCountsDataFrame.show(); }); ssc.start(); @@ -110,12 +93,15 @@ public JavaRecord call(String word) { } } -/** Lazily instantiated singleton instance of SQLContext */ -class JavaSQLContextSingleton { - static private transient SQLContext instance = null; - static public SQLContext getInstance(SparkContext sparkContext) { +/** Lazily instantiated singleton instance of SparkSession */ +class JavaSparkSessionSingleton { + private static transient SparkSession instance = null; + public static SparkSession getInstance(SparkConf sparkConf) { if (instance == null) { - instance = new SQLContext(sparkContext); + instance = SparkSession + .builder() + .config(sparkConf) + .getOrCreate(); } return instance; } 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 99b63a2590ae..9d8bd7fd11eb 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 @@ -23,21 +23,15 @@ 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.function.*; import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.Optional; 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; +import org.apache.spark.streaming.State; +import org.apache.spark.streaming.StateSpec; +import org.apache.spark.streaming.api.java.*; /** * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every @@ -55,7 +49,7 @@ public class JavaStatefulNetworkWordCount { private static final Pattern SPACE = Pattern.compile(" "); - public static void main(String[] args) { + public static void main(String[] args) throws Exception { if (args.length < 2) { System.err.println("Usage: JavaStatefulNetworkWordCount "); System.exit(1); @@ -63,51 +57,36 @@ 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); - } - }; - // 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 + // Initial state RDD input to mapWithState @SuppressWarnings("unchecked") - List> tuples = Arrays.asList(new Tuple2("hello", 1), - new Tuple2("world", 1)); + List> tuples = + Arrays.asList(new Tuple2<>("hello", 1), new Tuple2<>("world", 1)); JavaPairRDD initialRDD = ssc.sparkContext().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)); - } - }); + JavaDStream words = lines.flatMap(x -> Arrays.asList(SPACE.split(x)).iterator()); - JavaPairDStream wordsDstream = words.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(String s) { - return new Tuple2(s, 1); - } - }); + JavaPairDStream wordsDstream = words.mapToPair(s -> new Tuple2<>(s, 1)); + + // Update the cumulative count function + Function3, State, Tuple2> mappingFunc = + (word, one, state) -> { + int sum = one.orElse(0) + (state.exists() ? state.get() : 0); + Tuple2 output = new Tuple2<>(word, sum); + state.update(sum); + return output; + }; - // This will give a Dstream made of state (which is the cumulative count of the words) - JavaPairDStream stateDstream = wordsDstream.updateStateByKey(updateFunction, - new HashPartitioner(ssc.sparkContext().defaultParallelism()), initialRDD); + // DStream made of get cumulative counts that get updated in every batch + JavaMapWithStateDStream> stateDstream = + wordsDstream.mapWithState(StateSpec.function(mappingFunc).initialState(initialRDD)); stateDstream.print(); ssc.start(); diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 1c3a787bd0e9..6d3241876ad5 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -17,7 +17,7 @@ """ This is an example implementation of ALS for learning how to use Spark. Please refer to -ALS in pyspark.mllib.recommendation for more conventional use. +pyspark.ml.recommendation.ALS for more conventional use. This example requires numpy (http://www.numpy.org/) """ @@ -28,7 +28,7 @@ import numpy as np from numpy.random import rand from numpy import matrix -from pyspark import SparkContext +from pyspark.sql import SparkSession LAMBDA = 0.01 # regularization np.random.seed(42) @@ -36,10 +36,10 @@ def rmse(R, ms, us): diff = R - ms * us.T - return np.sqrt(np.sum(np.power(diff, 2)) / M * U) + return np.sqrt(np.sum(np.power(diff, 2)) / (M * U)) -def update(i, vec, mat, ratings): +def update(i, mat, ratings): uu = mat.shape[0] ff = mat.shape[1] @@ -59,10 +59,16 @@ def update(i, vec, mat, ratings): """ print("""WARN: This is a naive implementation of ALS and is given as an - example. Please use the ALS method found in pyspark.mllib.recommendation for more + example. Please use pyspark.ml.recommendation.ALS for more conventional use.""", file=sys.stderr) - sc = SparkContext(appName="PythonALS") + spark = SparkSession\ + .builder\ + .appName("PythonALS")\ + .getOrCreate() + + sc = spark.sparkContext + M = int(sys.argv[1]) if len(sys.argv) > 1 else 100 U = int(sys.argv[2]) if len(sys.argv) > 2 else 500 F = int(sys.argv[3]) if len(sys.argv) > 3 else 10 @@ -82,7 +88,7 @@ def update(i, vec, mat, ratings): for i in range(ITERATIONS): ms = sc.parallelize(range(M), partitions) \ - .map(lambda x: update(x, msb.value[x, :], usb.value, Rb.value)) \ + .map(lambda x: update(x, usb.value, Rb.value)) \ .collect() # collect() returns a list, so array ends up being # a 3-d array, we take the first 2 dims for the matrix @@ -90,7 +96,7 @@ def update(i, vec, mat, ratings): msb = sc.broadcast(ms) us = sc.parallelize(range(U), partitions) \ - .map(lambda x: update(x, usb.value[x, :], msb.value, Rb.value.T)) \ + .map(lambda x: update(x, msb.value, Rb.value.T)) \ .collect() us = matrix(np.array(us)[:, :, 0]) usb = sc.broadcast(us) @@ -99,4 +105,4 @@ def update(i, vec, mat, ratings): print("Iteration %d:" % i) print("\nRMSE: %5.4f\n" % error) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py index da368ac628a4..4422f9e7a958 100644 --- a/examples/src/main/python/avro_inputformat.py +++ b/examples/src/main/python/avro_inputformat.py @@ -19,8 +19,8 @@ import sys -from pyspark import SparkContext from functools import reduce +from pyspark.sql import SparkSession """ Read data file users.avro in local Spark distro: @@ -64,7 +64,13 @@ exit(-1) path = sys.argv[1] - sc = SparkContext(appName="AvroKeyInputFormat") + + spark = SparkSession\ + .builder\ + .appName("AvroKeyInputFormat")\ + .getOrCreate() + + sc = spark.sparkContext conf = None if len(sys.argv) == 3: @@ -82,4 +88,4 @@ for k in output: print(k) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/cassandra_inputformat.py b/examples/src/main/python/cassandra_inputformat.py deleted file mode 100644 index 93ca0cfcc930..000000000000 --- a/examples/src/main/python/cassandra_inputformat.py +++ /dev/null @@ -1,84 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -from __future__ import print_function - -import sys - -from pyspark import SparkContext - -""" -Create data in Cassandra fist -(following: https://wiki.apache.org/cassandra/GettingStarted) - -cqlsh> CREATE KEYSPACE test - ... WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }; -cqlsh> use test; -cqlsh:test> CREATE TABLE users ( - ... user_id int PRIMARY KEY, - ... fname text, - ... lname text - ... ); -cqlsh:test> INSERT INTO users (user_id, fname, lname) - ... VALUES (1745, 'john', 'smith'); -cqlsh:test> INSERT INTO users (user_id, fname, lname) - ... VALUES (1744, 'john', 'doe'); -cqlsh:test> INSERT INTO users (user_id, fname, lname) - ... VALUES (1746, 'john', 'smith'); -cqlsh:test> SELECT * FROM users; - - user_id | fname | lname ----------+-------+------- - 1745 | john | smith - 1744 | john | doe - 1746 | john | smith -""" -if __name__ == "__main__": - if len(sys.argv) != 4: - print(""" - Usage: cassandra_inputformat - - Run with example jar: - ./bin/spark-submit --driver-class-path /path/to/example/jar \ - /path/to/examples/cassandra_inputformat.py - Assumes you have some data in Cassandra already, running on , in and - """, file=sys.stderr) - exit(-1) - - host = sys.argv[1] - keyspace = sys.argv[2] - cf = sys.argv[3] - sc = SparkContext(appName="CassandraInputFormat") - - conf = {"cassandra.input.thrift.address": host, - "cassandra.input.thrift.port": "9160", - "cassandra.input.keyspace": keyspace, - "cassandra.input.columnfamily": cf, - "cassandra.input.partitioner.class": "Murmur3Partitioner", - "cassandra.input.page.row.size": "3"} - cass_rdd = sc.newAPIHadoopRDD( - "org.apache.cassandra.hadoop.cql3.CqlPagingInputFormat", - "java.util.Map", - "java.util.Map", - keyConverter="org.apache.spark.examples.pythonconverters.CassandraCQLKeyConverter", - valueConverter="org.apache.spark.examples.pythonconverters.CassandraCQLValueConverter", - conf=conf) - output = cass_rdd.collect() - for (k, v) in output: - print((k, v)) - - sc.stop() diff --git a/examples/src/main/python/cassandra_outputformat.py b/examples/src/main/python/cassandra_outputformat.py deleted file mode 100644 index 5d643eac92f9..000000000000 --- a/examples/src/main/python/cassandra_outputformat.py +++ /dev/null @@ -1,88 +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. -# - -from __future__ import print_function - -import sys - -from pyspark import SparkContext - -""" -Create data in Cassandra fist -(following: https://wiki.apache.org/cassandra/GettingStarted) - -cqlsh> CREATE KEYSPACE test - ... WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }; -cqlsh> use test; -cqlsh:test> CREATE TABLE users ( - ... user_id int PRIMARY KEY, - ... fname text, - ... lname text - ... ); - -> cassandra_outputformat test users 1745 john smith -> cassandra_outputformat test users 1744 john doe -> cassandra_outputformat test users 1746 john smith - -cqlsh:test> SELECT * FROM users; - - user_id | fname | lname ----------+-------+------- - 1745 | john | smith - 1744 | john | doe - 1746 | john | smith -""" -if __name__ == "__main__": - if len(sys.argv) != 7: - print(""" - Usage: cassandra_outputformat - - Run with example jar: - ./bin/spark-submit --driver-class-path /path/to/example/jar \ - /path/to/examples/cassandra_outputformat.py - Assumes you have created the following table in Cassandra already, - running on , in . - - cqlsh:> CREATE TABLE ( - ... user_id int PRIMARY KEY, - ... fname text, - ... lname text - ... ); - """, file=sys.stderr) - exit(-1) - - host = sys.argv[1] - keyspace = sys.argv[2] - cf = sys.argv[3] - sc = SparkContext(appName="CassandraOutputFormat") - - conf = {"cassandra.output.thrift.address": host, - "cassandra.output.thrift.port": "9160", - "cassandra.output.keyspace": keyspace, - "cassandra.output.partitioner.class": "Murmur3Partitioner", - "cassandra.output.cql": "UPDATE " + keyspace + "." + cf + " SET fname = ?, lname = ?", - "mapreduce.output.basename": cf, - "mapreduce.outputformat.class": "org.apache.cassandra.hadoop.cql3.CqlOutputFormat", - "mapreduce.job.output.key.class": "java.util.Map", - "mapreduce.job.output.value.class": "java.util.List"} - key = {"user_id": int(sys.argv[4])} - sc.parallelize([(key, sys.argv[5:])]).saveAsNewAPIHadoopDataset( - conf=conf, - keyConverter="org.apache.spark.examples.pythonconverters.ToCassandraCQLKeyConverter", - valueConverter="org.apache.spark.examples.pythonconverters.ToCassandraCQLValueConverter") - - sc.stop() diff --git a/examples/src/main/python/hbase_inputformat.py b/examples/src/main/python/hbase_inputformat.py deleted file mode 100644 index c5ae5d043b8e..000000000000 --- a/examples/src/main/python/hbase_inputformat.py +++ /dev/null @@ -1,90 +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. -# - -from __future__ import print_function - -import sys -import json - -from pyspark import SparkContext - -""" -Create test data in HBase first: - -hbase(main):016:0> create 'test', 'f1' -0 row(s) in 1.0430 seconds - -hbase(main):017:0> put 'test', 'row1', 'f1:a', 'value1' -0 row(s) in 0.0130 seconds - -hbase(main):018:0> put 'test', 'row1', 'f1:b', 'value2' -0 row(s) in 0.0030 seconds - -hbase(main):019:0> put 'test', 'row2', 'f1', 'value3' -0 row(s) in 0.0050 seconds - -hbase(main):020:0> put 'test', 'row3', 'f1', 'value4' -0 row(s) in 0.0110 seconds - -hbase(main):021:0> scan 'test' -ROW COLUMN+CELL - row1 column=f1:a, timestamp=1401883411986, value=value1 - row1 column=f1:b, timestamp=1401883415212, value=value2 - row2 column=f1:, timestamp=1401883417858, value=value3 - row3 column=f1:, timestamp=1401883420805, value=value4 -4 row(s) in 0.0240 seconds -""" -if __name__ == "__main__": - if len(sys.argv) != 3: - print(""" - Usage: hbase_inputformat - - Run with example jar: - ./bin/spark-submit --driver-class-path /path/to/example/jar \ - /path/to/examples/hbase_inputformat.py
    [] - Assumes you have some data in HBase already, running on , in
    - optionally, you can specify parent znode for your hbase cluster - - """, file=sys.stderr) - exit(-1) - - host = sys.argv[1] - table = sys.argv[2] - sc = SparkContext(appName="HBaseInputFormat") - - # Other options for configuring scan behavior are available. More information available at - # https://github.com/apache/hbase/blob/master/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java - conf = {"hbase.zookeeper.quorum": host, "hbase.mapreduce.inputtable": table} - if len(sys.argv) > 3: - conf = {"hbase.zookeeper.quorum": host, "zookeeper.znode.parent": sys.argv[3], - "hbase.mapreduce.inputtable": table} - keyConv = "org.apache.spark.examples.pythonconverters.ImmutableBytesWritableToStringConverter" - valueConv = "org.apache.spark.examples.pythonconverters.HBaseResultToStringConverter" - - hbase_rdd = sc.newAPIHadoopRDD( - "org.apache.hadoop.hbase.mapreduce.TableInputFormat", - "org.apache.hadoop.hbase.io.ImmutableBytesWritable", - "org.apache.hadoop.hbase.client.Result", - keyConverter=keyConv, - valueConverter=valueConv, - conf=conf) - hbase_rdd = hbase_rdd.flatMapValues(lambda v: v.split("\n")).mapValues(json.loads) - - output = hbase_rdd.collect() - for (k, v) in output: - print((k, v)) - - sc.stop() diff --git a/examples/src/main/python/hbase_outputformat.py b/examples/src/main/python/hbase_outputformat.py deleted file mode 100644 index 9e5641789a97..000000000000 --- a/examples/src/main/python/hbase_outputformat.py +++ /dev/null @@ -1,73 +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. -# - -from __future__ import print_function - -import sys - -from pyspark import SparkContext - -""" -Create test table in HBase first: - -hbase(main):001:0> create 'test', 'f1' -0 row(s) in 0.7840 seconds - -> hbase_outputformat test row1 f1 q1 value1 -> hbase_outputformat test row2 f1 q1 value2 -> hbase_outputformat test row3 f1 q1 value3 -> hbase_outputformat test row4 f1 q1 value4 - -hbase(main):002:0> scan 'test' -ROW COLUMN+CELL - row1 column=f1:q1, timestamp=1405659615726, value=value1 - row2 column=f1:q1, timestamp=1405659626803, value=value2 - row3 column=f1:q1, timestamp=1405659640106, value=value3 - row4 column=f1:q1, timestamp=1405659650292, value=value4 -4 row(s) in 0.0780 seconds -""" -if __name__ == "__main__": - if len(sys.argv) != 7: - print(""" - Usage: hbase_outputformat
    - - Run with example jar: - ./bin/spark-submit --driver-class-path /path/to/example/jar \ - /path/to/examples/hbase_outputformat.py - Assumes you have created
    with column family in HBase - running on already - """, file=sys.stderr) - exit(-1) - - host = sys.argv[1] - table = sys.argv[2] - sc = SparkContext(appName="HBaseOutputFormat") - - conf = {"hbase.zookeeper.quorum": host, - "hbase.mapred.outputtable": table, - "mapreduce.outputformat.class": "org.apache.hadoop.hbase.mapreduce.TableOutputFormat", - "mapreduce.job.output.key.class": "org.apache.hadoop.hbase.io.ImmutableBytesWritable", - "mapreduce.job.output.value.class": "org.apache.hadoop.io.Writable"} - keyConv = "org.apache.spark.examples.pythonconverters.StringToImmutableBytesWritableConverter" - valueConv = "org.apache.spark.examples.pythonconverters.StringListToPutConverter" - - sc.parallelize([sys.argv[3:]]).map(lambda x: (x[0], x)).saveAsNewAPIHadoopDataset( - conf=conf, - keyConverter=keyConv, - valueConverter=valueConv) - - sc.stop() diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index 0ea7cfb7025a..92e0a3ae2ee6 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -17,8 +17,8 @@ """ The K-means algorithm written from scratch against PySpark. In practice, -one may prefer to use the KMeans algorithm in MLlib, as shown in -examples/src/main/python/mllib/kmeans.py. +one may prefer to use the KMeans algorithm in ML, as shown in +examples/src/main/python/ml/kmeans_example.py. This example requires NumPy (http://www.numpy.org/). """ @@ -27,7 +27,7 @@ import sys import numpy as np -from pyspark import SparkContext +from pyspark.sql import SparkSession def parseVector(line): @@ -52,11 +52,15 @@ def closestPoint(p, centers): exit(-1) print("""WARN: This is a naive implementation of KMeans Clustering and is given - as an example! Please refer to examples/src/main/python/mllib/kmeans.py for an example on - how to use MLlib's KMeans implementation.""", file=sys.stderr) + as an example! Please refer to examples/src/main/python/ml/kmeans_example.py for an + example on how to use ML's KMeans implementation.""", file=sys.stderr) - sc = SparkContext(appName="PythonKMeans") - lines = sc.textFile(sys.argv[1]) + spark = SparkSession\ + .builder\ + .appName("PythonKMeans")\ + .getOrCreate() + + lines = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0]) data = lines.map(parseVector).cache() K = int(sys.argv[2]) convergeDist = float(sys.argv[3]) @@ -79,4 +83,4 @@ def closestPoint(p, centers): print("Final centers: " + str(kPoints)) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index b318b7d87bfd..01c938454b10 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -20,14 +20,14 @@ to act on batches of input data using efficient matrix operations. In practice, one may prefer to use the LogisticRegression algorithm in -MLlib, as shown in examples/src/main/python/mllib/logistic_regression.py. +ML, as shown in examples/src/main/python/ml/logistic_regression_with_elastic_net.py. """ from __future__ import print_function import sys import numpy as np -from pyspark import SparkContext +from pyspark.sql import SparkSession D = 10 # Number of dimensions @@ -51,11 +51,17 @@ def readPointBatch(iterator): exit(-1) print("""WARN: This is a naive implementation of Logistic Regression and is - given as an example! Please refer to examples/src/main/python/mllib/logistic_regression.py - to see how MLlib's implementation is used.""", file=sys.stderr) + given as an example! + Please refer to examples/src/main/python/ml/logistic_regression_with_elastic_net.py + to see how ML's implementation is used.""", file=sys.stderr) - sc = SparkContext(appName="PythonLR") - points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache() + spark = SparkSession\ + .builder\ + .appName("PythonLR")\ + .getOrCreate() + + points = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0])\ + .mapPartitions(readPointBatch).cache() iterations = int(sys.argv[2]) # Initialize w to a random value @@ -79,4 +85,4 @@ def add(x, y): print("Final w: " + str(w)) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/aft_survival_regression.py b/examples/src/main/python/ml/aft_survival_regression.py new file mode 100644 index 000000000000..2f0ca995e55c --- /dev/null +++ b/examples/src/main/python/ml/aft_survival_regression.py @@ -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. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.regression import AFTSurvivalRegression +from pyspark.ml.linalg import Vectors +# $example off$ +from pyspark.sql import SparkSession + +""" +An example demonstrating aft survival regression. +Run with: + bin/spark-submit examples/src/main/python/ml/aft_survival_regression.py +""" + +if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("AFTSurvivalRegressionExample") \ + .getOrCreate() + + # $example on$ + training = spark.createDataFrame([ + (1.218, 1.0, Vectors.dense(1.560, -0.605)), + (2.949, 0.0, Vectors.dense(0.346, 2.158)), + (3.627, 0.0, Vectors.dense(1.380, 0.231)), + (0.273, 1.0, Vectors.dense(0.520, 1.151)), + (4.199, 0.0, Vectors.dense(0.795, -0.226))], ["label", "censor", "features"]) + quantileProbabilities = [0.3, 0.6] + aft = AFTSurvivalRegression(quantileProbabilities=quantileProbabilities, + quantilesCol="quantiles") + + model = aft.fit(training) + + # Print the coefficients, intercept and scale parameter for AFT survival regression + print("Coefficients: " + str(model.coefficients)) + print("Intercept: " + str(model.intercept)) + print("Scale: " + str(model.scale)) + model.transform(training).show(truncate=False) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/als_example.py b/examples/src/main/python/ml/als_example.py new file mode 100644 index 000000000000..1672d552eb1d --- /dev/null +++ b/examples/src/main/python/ml/als_example.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. +# + +from __future__ import print_function + +import sys +if sys.version >= '3': + long = int + +from pyspark.sql import SparkSession + +# $example on$ +from pyspark.ml.evaluation import RegressionEvaluator +from pyspark.ml.recommendation import ALS +from pyspark.sql import Row +# $example off$ + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("ALSExample")\ + .getOrCreate() + + # $example on$ + lines = spark.read.text("data/mllib/als/sample_movielens_ratings.txt").rdd + parts = lines.map(lambda row: row.value.split("::")) + ratingsRDD = parts.map(lambda p: Row(userId=int(p[0]), movieId=int(p[1]), + rating=float(p[2]), timestamp=long(p[3]))) + ratings = spark.createDataFrame(ratingsRDD) + (training, test) = ratings.randomSplit([0.8, 0.2]) + + # Build the recommendation model using ALS on the training data + # Note we set cold start strategy to 'drop' to ensure we don't get NaN evaluation metrics + als = ALS(maxIter=5, regParam=0.01, userCol="userId", itemCol="movieId", ratingCol="rating", + coldStartStrategy="drop") + model = als.fit(training) + + # Evaluate the model by computing the RMSE on the test data + predictions = model.transform(test) + evaluator = RegressionEvaluator(metricName="rmse", labelCol="rating", + predictionCol="prediction") + rmse = evaluator.evaluate(predictions) + print("Root-mean-square error = " + str(rmse)) + + # Generate top 10 movie recommendations for each user + userRecs = model.recommendForAllUsers(10) + # Generate top 10 user recommendations for each movie + movieRecs = model.recommendForAllItems(10) + # $example off$ + userRecs.show() + movieRecs.show() + + spark.stop() diff --git a/examples/src/main/python/ml/binarizer_example.py b/examples/src/main/python/ml/binarizer_example.py new file mode 100644 index 000000000000..669bb2aeabec --- /dev/null +++ b/examples/src/main/python/ml/binarizer_example.py @@ -0,0 +1,46 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark.sql import SparkSession +# $example on$ +from pyspark.ml.feature import Binarizer +# $example off$ + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("BinarizerExample")\ + .getOrCreate() + + # $example on$ + continuousDataFrame = spark.createDataFrame([ + (0, 0.1), + (1, 0.8), + (2, 0.2) + ], ["id", "feature"]) + + binarizer = Binarizer(threshold=0.5, inputCol="feature", outputCol="binarized_feature") + + binarizedDataFrame = binarizer.transform(continuousDataFrame) + + print("Binarizer output with Threshold = %f" % binarizer.getThreshold()) + binarizedDataFrame.show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/bisecting_k_means_example.py b/examples/src/main/python/ml/bisecting_k_means_example.py new file mode 100644 index 000000000000..1263cb5d177a --- /dev/null +++ b/examples/src/main/python/ml/bisecting_k_means_example.py @@ -0,0 +1,56 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.clustering import BisectingKMeans +# $example off$ +from pyspark.sql import SparkSession + +""" +An example demonstrating bisecting k-means clustering. +Run with: + bin/spark-submit examples/src/main/python/ml/bisecting_k_means_example.py +""" + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("BisectingKMeansExample")\ + .getOrCreate() + + # $example on$ + # Loads data. + dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") + + # Trains a bisecting k-means model. + bkm = BisectingKMeans().setK(2).setSeed(1) + model = bkm.fit(dataset) + + # Evaluate clustering. + cost = model.computeCost(dataset) + print("Within Set Sum of Squared Errors = " + str(cost)) + + # Shows the result. + print("Cluster Centers: ") + centers = model.clusterCenters() + for center in centers: + print(center) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py new file mode 100644 index 000000000000..1b7a458125ce --- /dev/null +++ b/examples/src/main/python/ml/bucketed_random_projection_lsh_example.py @@ -0,0 +1,81 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import BucketedRandomProjectionLSH +from pyspark.ml.linalg import Vectors +from pyspark.sql.functions import col +# $example off$ +from pyspark.sql import SparkSession + +""" +An example demonstrating BucketedRandomProjectionLSH. +Run with: + bin/spark-submit examples/src/main/python/ml/bucketed_random_projection_lsh_example.py +""" + +if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("BucketedRandomProjectionLSHExample") \ + .getOrCreate() + + # $example on$ + dataA = [(0, Vectors.dense([1.0, 1.0]),), + (1, Vectors.dense([1.0, -1.0]),), + (2, Vectors.dense([-1.0, -1.0]),), + (3, Vectors.dense([-1.0, 1.0]),)] + dfA = spark.createDataFrame(dataA, ["id", "features"]) + + dataB = [(4, Vectors.dense([1.0, 0.0]),), + (5, Vectors.dense([-1.0, 0.0]),), + (6, Vectors.dense([0.0, 1.0]),), + (7, Vectors.dense([0.0, -1.0]),)] + dfB = spark.createDataFrame(dataB, ["id", "features"]) + + key = Vectors.dense([1.0, 0.0]) + + brp = BucketedRandomProjectionLSH(inputCol="features", outputCol="hashes", bucketLength=2.0, + numHashTables=3) + model = brp.fit(dfA) + + # Feature Transformation + print("The hashed dataset where hashed values are stored in the column 'hashes':") + model.transform(dfA).show() + + # Compute the locality sensitive hashes for the input rows, then perform approximate + # similarity join. + # We could avoid computing hashes by passing in the already-transformed dataset, e.g. + # `model.approxSimilarityJoin(transformedA, transformedB, 1.5)` + print("Approximately joining dfA and dfB on Euclidean distance smaller than 1.5:") + model.approxSimilarityJoin(dfA, dfB, 1.5, distCol="EuclideanDistance")\ + .select(col("datasetA.id").alias("idA"), + col("datasetB.id").alias("idB"), + col("EuclideanDistance")).show() + + # Compute the locality sensitive hashes for the input rows, then perform approximate nearest + # neighbor search. + # We could avoid computing hashes by passing in the already-transformed dataset, e.g. + # `model.approxNearestNeighbors(transformedA, key, 2)` + print("Approximately searching dfA for 2 nearest neighbors of the key:") + model.approxNearestNeighbors(dfA, key, 2).show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/bucketizer_example.py b/examples/src/main/python/ml/bucketizer_example.py new file mode 100644 index 000000000000..742f35093b9d --- /dev/null +++ b/examples/src/main/python/ml/bucketizer_example.py @@ -0,0 +1,46 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark.sql import SparkSession +# $example on$ +from pyspark.ml.feature import Bucketizer +# $example off$ + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("BucketizerExample")\ + .getOrCreate() + + # $example on$ + splits = [-float("inf"), -0.5, 0.0, 0.5, float("inf")] + + data = [(-999.9,), (-0.5,), (-0.3,), (0.0,), (0.2,), (999.9,)] + dataFrame = spark.createDataFrame(data, ["features"]) + + bucketizer = Bucketizer(splits=splits, inputCol="features", outputCol="bucketedFeatures") + + # Transform original data into its bucket index. + bucketedData = bucketizer.transform(dataFrame) + + print("Bucketizer output with %d buckets" % (len(bucketizer.getSplits())-1)) + bucketedData.show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/chi_square_test_example.py b/examples/src/main/python/ml/chi_square_test_example.py new file mode 100644 index 000000000000..8f25318ded00 --- /dev/null +++ b/examples/src/main/python/ml/chi_square_test_example.py @@ -0,0 +1,52 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark.sql import SparkSession +# $example on$ +from pyspark.ml.linalg import Vectors +from pyspark.ml.stat import ChiSquareTest +# $example off$ + +""" +An example for Chi-square hypothesis testing. +Run with: + bin/spark-submit examples/src/main/python/ml/chi_square_test_example.py +""" +if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("ChiSquareTestExample") \ + .getOrCreate() + + # $example on$ + data = [(0.0, Vectors.dense(0.5, 10.0)), + (0.0, Vectors.dense(1.5, 20.0)), + (1.0, Vectors.dense(1.5, 30.0)), + (0.0, Vectors.dense(3.5, 30.0)), + (0.0, Vectors.dense(3.5, 40.0)), + (1.0, Vectors.dense(3.5, 40.0))] + df = spark.createDataFrame(data, ["label", "features"]) + + r = ChiSquareTest.test(df, "features", "label").head() + print("pValues: " + str(r.pValues)) + print("degreesOfFreedom: " + str(r.degreesOfFreedom)) + print("statistics: " + str(r.statistics)) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/chisq_selector_example.py b/examples/src/main/python/ml/chisq_selector_example.py new file mode 100644 index 000000000000..028a9ea9d67b --- /dev/null +++ b/examples/src/main/python/ml/chisq_selector_example.py @@ -0,0 +1,47 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark.sql import SparkSession +# $example on$ +from pyspark.ml.feature import ChiSqSelector +from pyspark.ml.linalg import Vectors +# $example off$ + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("ChiSqSelectorExample")\ + .getOrCreate() + + # $example on$ + df = spark.createDataFrame([ + (7, Vectors.dense([0.0, 0.0, 18.0, 1.0]), 1.0,), + (8, Vectors.dense([0.0, 1.0, 12.0, 0.0]), 0.0,), + (9, Vectors.dense([1.0, 0.0, 15.0, 0.1]), 0.0,)], ["id", "features", "clicked"]) + + selector = ChiSqSelector(numTopFeatures=1, featuresCol="features", + outputCol="selectedFeatures", labelCol="clicked") + + result = selector.fit(df).transform(df) + + print("ChiSqSelector output with top %d features selected" % selector.getNumTopFeatures()) + result.show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/correlation_example.py b/examples/src/main/python/ml/correlation_example.py new file mode 100644 index 000000000000..0a9d30da5a42 --- /dev/null +++ b/examples/src/main/python/ml/correlation_example.py @@ -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. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.linalg import Vectors +from pyspark.ml.stat import Correlation +# $example off$ +from pyspark.sql import SparkSession + +""" +An example for computing correlation matrix. +Run with: + bin/spark-submit examples/src/main/python/ml/correlation_example.py +""" +if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("CorrelationExample") \ + .getOrCreate() + + # $example on$ + data = [(Vectors.sparse(4, [(0, 1.0), (3, -2.0)]),), + (Vectors.dense([4.0, 5.0, 0.0, 3.0]),), + (Vectors.dense([6.0, 7.0, 0.0, 8.0]),), + (Vectors.sparse(4, [(0, 9.0), (3, 1.0)]),)] + df = spark.createDataFrame(data, ["features"]) + + r1 = Correlation.corr(df, "features").head() + print("Pearson correlation matrix:\n" + str(r1[0])) + + r2 = Correlation.corr(df, "features", "spearman").head() + print("Spearman correlation matrix:\n" + str(r2[0])) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/count_vectorizer_example.py b/examples/src/main/python/ml/count_vectorizer_example.py new file mode 100644 index 000000000000..f2e41db77d89 --- /dev/null +++ b/examples/src/main/python/ml/count_vectorizer_example.py @@ -0,0 +1,47 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark.sql import SparkSession +# $example on$ +from pyspark.ml.feature import CountVectorizer +# $example off$ + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("CountVectorizerExample")\ + .getOrCreate() + + # $example on$ + # Input data: Each row is a bag of words with a ID. + df = spark.createDataFrame([ + (0, "a b c".split(" ")), + (1, "a b b c a".split(" ")) + ], ["id", "words"]) + + # fit a CountVectorizerModel from the corpus. + cv = CountVectorizer(inputCol="words", outputCol="features", vocabSize=3, minDF=2.0) + + model = cv.fit(df) + + result = model.transform(df) + result.show(truncate=False) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/cross_validator.py b/examples/src/main/python/ml/cross_validator.py index f0ca97c72494..db7054307c2e 100644 --- a/examples/src/main/python/ml/cross_validator.py +++ b/examples/src/main/python/ml/cross_validator.py @@ -17,13 +17,14 @@ from __future__ import print_function -from pyspark import SparkContext +# $example on$ from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression from pyspark.ml.evaluation import BinaryClassificationEvaluator from pyspark.ml.feature import HashingTF, Tokenizer from pyspark.ml.tuning import CrossValidator, ParamGridBuilder -from pyspark.sql import Row, SQLContext +# $example off$ +from pyspark.sql import SparkSession """ A simple example demonstrating model selection using CrossValidator. @@ -34,25 +35,27 @@ """ if __name__ == "__main__": - sc = SparkContext(appName="CrossValidatorExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("CrossValidatorExample")\ + .getOrCreate() + # $example on$ # Prepare training documents, which are labeled. - LabeledDocument = Row("id", "text", "label") - training = sc.parallelize([(0, "a b c d e spark", 1.0), - (1, "b d", 0.0), - (2, "spark f g h", 1.0), - (3, "hadoop mapreduce", 0.0), - (4, "b spark who", 1.0), - (5, "g d a y", 0.0), - (6, "spark fly", 1.0), - (7, "was mapreduce", 0.0), - (8, "e spark program", 1.0), - (9, "a e c l", 0.0), - (10, "spark compile", 1.0), - (11, "hadoop software", 0.0) - ]) \ - .map(lambda x: LabeledDocument(*x)).toDF() + training = spark.createDataFrame([ + (0, "a b c d e spark", 1.0), + (1, "b d", 0.0), + (2, "spark f g h", 1.0), + (3, "hadoop mapreduce", 0.0), + (4, "b spark who", 1.0), + (5, "g d a y", 0.0), + (6, "spark fly", 1.0), + (7, "was mapreduce", 0.0), + (8, "e spark program", 1.0), + (9, "a e c l", 0.0), + (10, "spark compile", 1.0), + (11, "hadoop software", 0.0) + ], ["id", "text", "label"]) # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. tokenizer = Tokenizer(inputCol="text", outputCol="words") @@ -80,17 +83,18 @@ cvModel = crossval.fit(training) # Prepare test documents, which are unlabeled. - Document = Row("id", "text") - test = sc.parallelize([(4L, "spark i j k"), - (5L, "l m n"), - (6L, "mapreduce spark"), - (7L, "apache hadoop")]) \ - .map(lambda x: Document(*x)).toDF() + test = spark.createDataFrame([ + (4, "spark i j k"), + (5, "l m n"), + (6, "mapreduce spark"), + (7, "apache hadoop") + ], ["id", "text"]) # Make predictions on test documents. cvModel uses the best model found (lrModel). prediction = cvModel.transform(test) selected = prediction.select("id", "text", "probability", "prediction") for row in selected.collect(): print(row) + # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/dataframe_example.py b/examples/src/main/python/ml/dataframe_example.py new file mode 100644 index 000000000000..109f901012c9 --- /dev/null +++ b/examples/src/main/python/ml/dataframe_example.py @@ -0,0 +1,79 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +An example of how to use DataFrame for ML. Run with:: + bin/spark-submit examples/src/main/python/ml/dataframe_example.py +""" +from __future__ import print_function + +import os +import sys +import tempfile +import shutil + +from pyspark.sql import SparkSession +from pyspark.mllib.stat import Statistics +from pyspark.mllib.util import MLUtils + +if __name__ == "__main__": + if len(sys.argv) > 2: + print("Usage: dataframe_example.py ", file=sys.stderr) + exit(-1) + elif len(sys.argv) == 2: + input = sys.argv[1] + else: + input = "data/mllib/sample_libsvm_data.txt" + + spark = SparkSession \ + .builder \ + .appName("DataFrameExample") \ + .getOrCreate() + + # Load input data + print("Loading LIBSVM file with UDT from " + input + ".") + df = spark.read.format("libsvm").load(input).cache() + print("Schema from LIBSVM:") + df.printSchema() + print("Loaded training data as a DataFrame with " + + str(df.count()) + " records.") + + # Show statistical summary of labels. + labelSummary = df.describe("label") + labelSummary.show() + + # Convert features column to an RDD of vectors. + features = MLUtils.convertVectorColumnsFromML(df, "features") \ + .select("features").rdd.map(lambda r: r.features) + summary = Statistics.colStats(features) + print("Selected features column with average values:\n" + + str(summary.mean())) + + # Save the records in a parquet file. + tempdir = tempfile.NamedTemporaryFile(delete=False).name + os.unlink(tempdir) + print("Saving to " + tempdir + " as Parquet file.") + df.write.parquet(tempdir) + + # Load the records back. + print("Loading Parquet file with UDT from " + tempdir) + newDF = spark.read.parquet(tempdir) + print("Schema from Parquet:") + newDF.printSchema() + shutil.rmtree(tempdir) + + spark.stop() diff --git a/examples/src/main/python/ml/dct_example.py b/examples/src/main/python/ml/dct_example.py new file mode 100644 index 000000000000..c0457f8d0f43 --- /dev/null +++ b/examples/src/main/python/ml/dct_example.py @@ -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. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import DCT +from pyspark.ml.linalg import Vectors +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("DCTExample")\ + .getOrCreate() + + # $example on$ + df = spark.createDataFrame([ + (Vectors.dense([0.0, 1.0, -2.0, 3.0]),), + (Vectors.dense([-1.0, 2.0, 4.0, -7.0]),), + (Vectors.dense([14.0, -2.0, -5.0, 1.0]),)], ["features"]) + + dct = DCT(inverse=False, inputCol="features", outputCol="featuresDCT") + + dctDf = dct.transform(df) + + dctDf.select("featuresDCT").show(truncate=False) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/decision_tree_classification_example.py b/examples/src/main/python/ml/decision_tree_classification_example.py new file mode 100644 index 000000000000..d6e2977de008 --- /dev/null +++ b/examples/src/main/python/ml/decision_tree_classification_example.py @@ -0,0 +1,78 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Decision Tree Classification Example. +""" +from __future__ import print_function + +# $example on$ +from pyspark.ml import Pipeline +from pyspark.ml.classification import DecisionTreeClassifier +from pyspark.ml.feature import StringIndexer, VectorIndexer +from pyspark.ml.evaluation import MulticlassClassificationEvaluator +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("DecisionTreeClassificationExample")\ + .getOrCreate() + + # $example on$ + # Load the data stored in LIBSVM format as a DataFrame. + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + # Index labels, adding metadata to the label column. + # Fit on whole dataset to include all labels in index. + labelIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel").fit(data) + # Automatically identify categorical features, and index them. + # We specify maxCategories so features with > 4 distinct values are treated as continuous. + featureIndexer =\ + VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) + + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + # Train a DecisionTree model. + dt = DecisionTreeClassifier(labelCol="indexedLabel", featuresCol="indexedFeatures") + + # Chain indexers and tree in a Pipeline + pipeline = Pipeline(stages=[labelIndexer, featureIndexer, dt]) + + # Train model. This also runs the indexers. + model = pipeline.fit(trainingData) + + # Make predictions. + predictions = model.transform(testData) + + # Select example rows to display. + predictions.select("prediction", "indexedLabel", "features").show(5) + + # Select (prediction, true label) and compute test error + evaluator = MulticlassClassificationEvaluator( + labelCol="indexedLabel", predictionCol="prediction", metricName="accuracy") + accuracy = evaluator.evaluate(predictions) + print("Test Error = %g " % (1.0 - accuracy)) + + treeModel = model.stages[2] + # summary only + print(treeModel) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/decision_tree_regression_example.py b/examples/src/main/python/ml/decision_tree_regression_example.py new file mode 100644 index 000000000000..58d7ad921d8e --- /dev/null +++ b/examples/src/main/python/ml/decision_tree_regression_example.py @@ -0,0 +1,75 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Decision Tree Regression Example. +""" +from __future__ import print_function + +# $example on$ +from pyspark.ml import Pipeline +from pyspark.ml.regression import DecisionTreeRegressor +from pyspark.ml.feature import VectorIndexer +from pyspark.ml.evaluation import RegressionEvaluator +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("DecisionTreeRegressionExample")\ + .getOrCreate() + + # $example on$ + # Load the data stored in LIBSVM format as a DataFrame. + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + # Automatically identify categorical features, and index them. + # We specify maxCategories so features with > 4 distinct values are treated as continuous. + featureIndexer =\ + VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) + + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + # Train a DecisionTree model. + dt = DecisionTreeRegressor(featuresCol="indexedFeatures") + + # Chain indexer and tree in a Pipeline + pipeline = Pipeline(stages=[featureIndexer, dt]) + + # Train model. This also runs the indexer. + model = pipeline.fit(trainingData) + + # Make predictions. + predictions = model.transform(testData) + + # Select example rows to display. + predictions.select("prediction", "label", "features").show(5) + + # Select (prediction, true label) and compute test error + evaluator = RegressionEvaluator( + labelCol="label", predictionCol="prediction", metricName="rmse") + rmse = evaluator.evaluate(predictions) + print("Root Mean Squared Error (RMSE) on test data = %g" % rmse) + + treeModel = model.stages[1] + # summary only + print(treeModel) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/elementwise_product_example.py b/examples/src/main/python/ml/elementwise_product_example.py new file mode 100644 index 000000000000..590053998bcc --- /dev/null +++ b/examples/src/main/python/ml/elementwise_product_example.py @@ -0,0 +1,42 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import ElementwiseProduct +from pyspark.ml.linalg import Vectors +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("ElementwiseProductExample")\ + .getOrCreate() + + # $example on$ + # Create some vector data; also works for sparse vectors + data = [(Vectors.dense([1.0, 2.0, 3.0]),), (Vectors.dense([4.0, 5.0, 6.0]),)] + df = spark.createDataFrame(data, ["vector"]) + transformer = ElementwiseProduct(scalingVec=Vectors.dense([0.0, 1.0, 2.0]), + inputCol="vector", outputCol="transformedVector") + # Batch transform the vectors to create new column: + transformer.transform(df).show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/estimator_transformer_param_example.py b/examples/src/main/python/ml/estimator_transformer_param_example.py new file mode 100644 index 000000000000..eb2105143539 --- /dev/null +++ b/examples/src/main/python/ml/estimator_transformer_param_example.py @@ -0,0 +1,93 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Estimator Transformer Param Example. +""" +from __future__ import print_function + +# $example on$ +from pyspark.ml.linalg import Vectors +from pyspark.ml.classification import LogisticRegression +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("EstimatorTransformerParamExample")\ + .getOrCreate() + + # $example on$ + # Prepare training data from a list of (label, features) tuples. + training = spark.createDataFrame([ + (1.0, Vectors.dense([0.0, 1.1, 0.1])), + (0.0, Vectors.dense([2.0, 1.0, -1.0])), + (0.0, Vectors.dense([2.0, 1.3, 1.0])), + (1.0, Vectors.dense([0.0, 1.2, -0.5]))], ["label", "features"]) + + # Create a LogisticRegression instance. This instance is an Estimator. + lr = LogisticRegression(maxIter=10, regParam=0.01) + # Print out the parameters, documentation, and any default values. + print("LogisticRegression parameters:\n" + lr.explainParams() + "\n") + + # Learn a LogisticRegression model. This uses the parameters stored in lr. + model1 = lr.fit(training) + + # 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 + # LogisticRegression instance. + print("Model 1 was fit using parameters: ") + print(model1.extractParamMap()) + + # We may alternatively specify parameters using a Python dictionary as a paramMap + paramMap = {lr.maxIter: 20} + paramMap[lr.maxIter] = 30 # Specify 1 Param, overwriting the original maxIter. + paramMap.update({lr.regParam: 0.1, lr.threshold: 0.55}) # Specify multiple Params. + + # You can combine paramMaps, which are python dictionaries. + paramMap2 = {lr.probabilityCol: "myProbability"} # Change output column name + paramMapCombined = paramMap.copy() + paramMapCombined.update(paramMap2) + + # Now learn a new model using the paramMapCombined parameters. + # paramMapCombined overrides all parameters set earlier via lr.set* methods. + model2 = lr.fit(training, paramMapCombined) + print("Model 2 was fit using parameters: ") + print(model2.extractParamMap()) + + # Prepare test data + test = spark.createDataFrame([ + (1.0, Vectors.dense([-1.0, 1.5, 1.3])), + (0.0, Vectors.dense([3.0, 2.0, -0.1])), + (1.0, Vectors.dense([0.0, 2.2, -1.5]))], ["label", "features"]) + + # Make predictions on test data using the Transformer.transform() method. + # 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. + prediction = model2.transform(test) + result = prediction.select("features", "label", "myProbability", "prediction") \ + .collect() + + for row in result: + print("features=%s, label=%s -> prob=%s, prediction=%s" + % (row.features, row.label, row.myProbability, row.prediction)) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/feature_hasher_example.py b/examples/src/main/python/ml/feature_hasher_example.py new file mode 100644 index 000000000000..6cf9ecc39640 --- /dev/null +++ b/examples/src/main/python/ml/feature_hasher_example.py @@ -0,0 +1,46 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark.sql import SparkSession +# $example on$ +from pyspark.ml.feature import FeatureHasher +# $example off$ + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("FeatureHasherExample")\ + .getOrCreate() + + # $example on$ + dataset = spark.createDataFrame([ + (2.2, True, "1", "foo"), + (3.3, False, "2", "bar"), + (4.4, False, "3", "baz"), + (5.5, False, "4", "foo") + ], ["real", "bool", "stringNum", "string"]) + + hasher = FeatureHasher(inputCols=["real", "bool", "stringNum", "string"], + outputCol="features") + + featurized = hasher.transform(dataset) + featurized.show(truncate=False) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/fpgrowth_example.py b/examples/src/main/python/ml/fpgrowth_example.py new file mode 100644 index 000000000000..c92c3c27abb2 --- /dev/null +++ b/examples/src/main/python/ml/fpgrowth_example.py @@ -0,0 +1,56 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# $example on$ +from pyspark.ml.fpm import FPGrowth +# $example off$ +from pyspark.sql import SparkSession + +""" +An example demonstrating FPGrowth. +Run with: + bin/spark-submit examples/src/main/python/ml/fpgrowth_example.py +""" + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("FPGrowthExample")\ + .getOrCreate() + + # $example on$ + df = spark.createDataFrame([ + (0, [1, 2, 5]), + (1, [1, 2, 3, 5]), + (2, [1, 2]) + ], ["id", "items"]) + + fpGrowth = FPGrowth(itemsCol="items", minSupport=0.5, minConfidence=0.6) + model = fpGrowth.fit(df) + + # Display frequent itemsets. + model.freqItemsets.show() + + # Display generated association rules. + model.associationRules.show() + + # transform examines the input items against all the association rules and summarize the + # consequents as prediction + model.transform(df).show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/gaussian_mixture_example.py b/examples/src/main/python/ml/gaussian_mixture_example.py new file mode 100644 index 000000000000..e4a0d314e9d9 --- /dev/null +++ b/examples/src/main/python/ml/gaussian_mixture_example.py @@ -0,0 +1,48 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.clustering import GaussianMixture +# $example off$ +from pyspark.sql import SparkSession + +""" +A simple example demonstrating Gaussian Mixture Model (GMM). +Run with: + bin/spark-submit examples/src/main/python/ml/gaussian_mixture_example.py +""" + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("GaussianMixtureExample")\ + .getOrCreate() + + # $example on$ + # loads data + dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") + + gmm = GaussianMixture().setK(2).setSeed(538009335) + model = gmm.fit(dataset) + + print("Gaussians shown as a DataFrame: ") + model.gaussiansDF.show(truncate=False) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/generalized_linear_regression_example.py b/examples/src/main/python/ml/generalized_linear_regression_example.py new file mode 100644 index 000000000000..796752a60f3a --- /dev/null +++ b/examples/src/main/python/ml/generalized_linear_regression_example.py @@ -0,0 +1,66 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark.sql import SparkSession +# $example on$ +from pyspark.ml.regression import GeneralizedLinearRegression +# $example off$ + +""" +An example demonstrating generalized linear regression. +Run with: + bin/spark-submit examples/src/main/python/ml/generalized_linear_regression_example.py +""" + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("GeneralizedLinearRegressionExample")\ + .getOrCreate() + + # $example on$ + # Load training data + dataset = spark.read.format("libsvm")\ + .load("data/mllib/sample_linear_regression_data.txt") + + glr = GeneralizedLinearRegression(family="gaussian", link="identity", maxIter=10, regParam=0.3) + + # Fit the model + model = glr.fit(dataset) + + # Print the coefficients and intercept for generalized linear regression model + print("Coefficients: " + str(model.coefficients)) + print("Intercept: " + str(model.intercept)) + + # Summarize the model over the training set and print out some metrics + summary = model.summary + print("Coefficient Standard Errors: " + str(summary.coefficientStandardErrors)) + print("T Values: " + str(summary.tValues)) + print("P Values: " + str(summary.pValues)) + print("Dispersion: " + str(summary.dispersion)) + print("Null Deviance: " + str(summary.nullDeviance)) + print("Residual Degree Of Freedom Null: " + str(summary.residualDegreeOfFreedomNull)) + print("Deviance: " + str(summary.deviance)) + print("Residual Degree Of Freedom: " + str(summary.residualDegreeOfFreedom)) + print("AIC: " + str(summary.aic)) + print("Deviance Residuals: ") + summary.residuals().show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py new file mode 100644 index 000000000000..c2042fd7b7b0 --- /dev/null +++ b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py @@ -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. +# + +""" +Gradient Boosted Tree Classifier Example. +""" +from __future__ import print_function + +# $example on$ +from pyspark.ml import Pipeline +from pyspark.ml.classification import GBTClassifier +from pyspark.ml.feature import StringIndexer, VectorIndexer +from pyspark.ml.evaluation import MulticlassClassificationEvaluator +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("GradientBoostedTreeClassifierExample")\ + .getOrCreate() + + # $example on$ + # Load and parse the data file, converting it to a DataFrame. + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + # Index labels, adding metadata to the label column. + # Fit on whole dataset to include all labels in index. + labelIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel").fit(data) + # Automatically identify categorical features, and index them. + # Set maxCategories so features with > 4 distinct values are treated as continuous. + featureIndexer =\ + VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) + + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + # Train a GBT model. + gbt = GBTClassifier(labelCol="indexedLabel", featuresCol="indexedFeatures", maxIter=10) + + # Chain indexers and GBT in a Pipeline + pipeline = Pipeline(stages=[labelIndexer, featureIndexer, gbt]) + + # Train model. This also runs the indexers. + model = pipeline.fit(trainingData) + + # Make predictions. + predictions = model.transform(testData) + + # Select example rows to display. + predictions.select("prediction", "indexedLabel", "features").show(5) + + # Select (prediction, true label) and compute test error + evaluator = MulticlassClassificationEvaluator( + labelCol="indexedLabel", predictionCol="prediction", metricName="accuracy") + accuracy = evaluator.evaluate(predictions) + print("Test Error = %g" % (1.0 - accuracy)) + + gbtModel = model.stages[2] + print(gbtModel) # summary only + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py new file mode 100644 index 000000000000..cc96c973e4b2 --- /dev/null +++ b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py @@ -0,0 +1,74 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Gradient Boosted Tree Regressor Example. +""" +from __future__ import print_function + +# $example on$ +from pyspark.ml import Pipeline +from pyspark.ml.regression import GBTRegressor +from pyspark.ml.feature import VectorIndexer +from pyspark.ml.evaluation import RegressionEvaluator +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("GradientBoostedTreeRegressorExample")\ + .getOrCreate() + + # $example on$ + # Load and parse the data file, converting it to a DataFrame. + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + # Automatically identify categorical features, and index them. + # Set maxCategories so features with > 4 distinct values are treated as continuous. + featureIndexer =\ + VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) + + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + # Train a GBT model. + gbt = GBTRegressor(featuresCol="indexedFeatures", maxIter=10) + + # Chain indexer and GBT in a Pipeline + pipeline = Pipeline(stages=[featureIndexer, gbt]) + + # Train model. This also runs the indexer. + model = pipeline.fit(trainingData) + + # Make predictions. + predictions = model.transform(testData) + + # Select example rows to display. + predictions.select("prediction", "label", "features").show(5) + + # Select (prediction, true label) and compute test error + evaluator = RegressionEvaluator( + labelCol="label", predictionCol="prediction", metricName="rmse") + rmse = evaluator.evaluate(predictions) + print("Root Mean Squared Error (RMSE) on test data = %g" % rmse) + + gbtModel = model.stages[1] + print(gbtModel) # summary only + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/gradient_boosted_trees.py b/examples/src/main/python/ml/gradient_boosted_trees.py deleted file mode 100644 index 6446f0fe5eea..000000000000 --- a/examples/src/main/python/ml/gradient_boosted_trees.py +++ /dev/null @@ -1,83 +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. -# - -from __future__ import print_function - -import sys - -from pyspark import SparkContext -from pyspark.ml.classification import GBTClassifier -from pyspark.ml.feature import StringIndexer -from pyspark.ml.regression import GBTRegressor -from pyspark.mllib.evaluation import BinaryClassificationMetrics, RegressionMetrics -from pyspark.mllib.util import MLUtils -from pyspark.sql import Row, SQLContext - -""" -A simple example demonstrating a Gradient Boosted Trees Classification/Regression Pipeline. -Note: GBTClassifier only supports binary classification currently -Run with: - bin/spark-submit examples/src/main/python/ml/gradient_boosted_trees.py -""" - - -def testClassification(train, test): - # Train a GradientBoostedTrees model. - - rf = GBTClassifier(maxIter=30, maxDepth=4, labelCol="indexedLabel") - - model = rf.fit(train) - predictionAndLabels = model.transform(test).select("prediction", "indexedLabel") \ - .map(lambda x: (x.prediction, x.indexedLabel)) - - metrics = BinaryClassificationMetrics(predictionAndLabels) - print("AUC %.3f" % metrics.areaUnderROC) - - -def testRegression(train, test): - # Train a GradientBoostedTrees model. - - rf = GBTRegressor(maxIter=30, maxDepth=4, labelCol="indexedLabel") - - model = rf.fit(train) - predictionAndLabels = model.transform(test).select("prediction", "indexedLabel") \ - .map(lambda x: (x.prediction, x.indexedLabel)) - - metrics = RegressionMetrics(predictionAndLabels) - print("rmse %.3f" % metrics.rootMeanSquaredError) - print("r2 %.3f" % metrics.r2) - print("mae %.3f" % metrics.meanAbsoluteError) - - -if __name__ == "__main__": - if len(sys.argv) > 1: - print("Usage: gradient_boosted_trees", file=sys.stderr) - exit(1) - sc = SparkContext(appName="PythonGBTExample") - sqlContext = SQLContext(sc) - - # Load and parse the data file into a dataframe. - df = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() - - # Map labels into an indexed column of labels in [0, numLabels) - stringIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel") - si_model = stringIndexer.fit(df) - td = si_model.transform(df) - [train, test] = td.randomSplit([0.7, 0.3]) - testClassification(train, test) - testRegression(train, test) - sc.stop() diff --git a/examples/src/main/python/ml/imputer_example.py b/examples/src/main/python/ml/imputer_example.py new file mode 100644 index 000000000000..b8437f827e56 --- /dev/null +++ b/examples/src/main/python/ml/imputer_example.py @@ -0,0 +1,50 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# $example on$ +from pyspark.ml.feature import Imputer +# $example off$ +from pyspark.sql import SparkSession + +""" +An example demonstrating Imputer. +Run with: + bin/spark-submit examples/src/main/python/ml/imputer_example.py +""" + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("ImputerExample")\ + .getOrCreate() + + # $example on$ + df = spark.createDataFrame([ + (1.0, float("nan")), + (2.0, float("nan")), + (float("nan"), 3.0), + (4.0, 4.0), + (5.0, 5.0) + ], ["a", "b"]) + + imputer = Imputer(inputCols=["a", "b"], outputCols=["out_a", "out_b"]) + model = imputer.fit(df) + + model.transform(df).show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/index_to_string_example.py b/examples/src/main/python/ml/index_to_string_example.py new file mode 100644 index 000000000000..33d104e8e3f4 --- /dev/null +++ b/examples/src/main/python/ml/index_to_string_example.py @@ -0,0 +1,54 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import IndexToString, StringIndexer +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("IndexToStringExample")\ + .getOrCreate() + + # $example on$ + df = spark.createDataFrame( + [(0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")], + ["id", "category"]) + + indexer = StringIndexer(inputCol="category", outputCol="categoryIndex") + model = indexer.fit(df) + indexed = model.transform(df) + + print("Transformed string column '%s' to indexed column '%s'" + % (indexer.getInputCol(), indexer.getOutputCol())) + indexed.show() + + print("StringIndexer will store labels in output column metadata\n") + + converter = IndexToString(inputCol="categoryIndex", outputCol="originalCategory") + converted = converter.transform(indexed) + + print("Transformed indexed column '%s' back to original string column '%s' using " + "labels in metadata" % (converter.getInputCol(), converter.getOutputCol())) + converted.select("id", "categoryIndex", "originalCategory").show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/isotonic_regression_example.py b/examples/src/main/python/ml/isotonic_regression_example.py new file mode 100644 index 000000000000..6ae15f1b4b0d --- /dev/null +++ b/examples/src/main/python/ml/isotonic_regression_example.py @@ -0,0 +1,54 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Isotonic Regression Example. +""" +from __future__ import print_function + +# $example on$ +from pyspark.ml.regression import IsotonicRegression +# $example off$ +from pyspark.sql import SparkSession + +""" +An example demonstrating isotonic regression. +Run with: + bin/spark-submit examples/src/main/python/ml/isotonic_regression_example.py +""" + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("IsotonicRegressionExample")\ + .getOrCreate() + + # $example on$ + # Loads data. + dataset = spark.read.format("libsvm")\ + .load("data/mllib/sample_isotonic_regression_libsvm_data.txt") + + # Trains an isotonic regression model. + model = IsotonicRegression().fit(dataset) + print("Boundaries in increasing order: %s\n" % str(model.boundaries)) + print("Predictions associated with the boundaries: %s\n" % str(model.predictions)) + + # Makes predictions. + model.transform(dataset).show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/kmeans_example.py b/examples/src/main/python/ml/kmeans_example.py index 150dadd42f33..6846ec459971 100644 --- a/examples/src/main/python/ml/kmeans_example.py +++ b/examples/src/main/python/ml/kmeans_example.py @@ -17,55 +17,43 @@ from __future__ import print_function -import sys -import re +# $example on$ +from pyspark.ml.clustering import KMeans +# $example off$ -import numpy as np -from pyspark import SparkContext -from pyspark.ml.clustering import KMeans, KMeansModel -from pyspark.mllib.linalg import VectorUDT, _convert_to_vector -from pyspark.sql import SQLContext -from pyspark.sql.types import Row, StructField, StructType +from pyspark.sql import SparkSession """ -A simple example demonstrating a k-means clustering. +An example demonstrating k-means clustering. Run with: - bin/spark-submit examples/src/main/python/ml/kmeans_example.py + bin/spark-submit examples/src/main/python/ml/kmeans_example.py This example requires NumPy (http://www.numpy.org/). """ - -def parseVector(line): - array = np.array([float(x) for x in line.split(' ')]) - return _convert_to_vector(array) - - if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("KMeansExample")\ + .getOrCreate() - FEATURES_COL = "features" + # $example on$ + # Loads data. + dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") - if len(sys.argv) != 3: - print("Usage: kmeans_example.py ", file=sys.stderr) - exit(-1) - path = sys.argv[1] - k = sys.argv[2] + # Trains a k-means model. + kmeans = KMeans().setK(2).setSeed(1) + model = kmeans.fit(dataset) - sc = SparkContext(appName="PythonKMeansExample") - sqlContext = SQLContext(sc) + # Evaluate clustering by computing Within Set Sum of Squared Errors. + wssse = model.computeCost(dataset) + print("Within Set Sum of Squared Errors = " + str(wssse)) - lines = sc.textFile(path) - data = lines.map(parseVector) - row_rdd = data.map(lambda x: Row(x)) - schema = StructType([StructField(FEATURES_COL, VectorUDT(), False)]) - df = sqlContext.createDataFrame(row_rdd, schema) - - kmeans = KMeans().setK(2).setSeed(1).setFeaturesCol(FEATURES_COL) - model = kmeans.fit(df) + # Shows the result. centers = model.clusterCenters() - print("Cluster Centers: ") for center in centers: print(center) + # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/lda_example.py b/examples/src/main/python/ml/lda_example.py new file mode 100644 index 000000000000..a8b346f72cd6 --- /dev/null +++ b/examples/src/main/python/ml/lda_example.py @@ -0,0 +1,61 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + + +from __future__ import print_function + +# $example on$ +from pyspark.ml.clustering import LDA +# $example off$ +from pyspark.sql import SparkSession + +""" +An example demonstrating LDA. +Run with: + bin/spark-submit examples/src/main/python/ml/lda_example.py +""" + +if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("LDAExample") \ + .getOrCreate() + + # $example on$ + # Loads data. + dataset = spark.read.format("libsvm").load("data/mllib/sample_lda_libsvm_data.txt") + + # Trains a LDA model. + lda = LDA(k=10, maxIter=10) + model = lda.fit(dataset) + + ll = model.logLikelihood(dataset) + lp = model.logPerplexity(dataset) + print("The lower bound on the log likelihood of the entire corpus: " + str(ll)) + print("The upper bound on perplexity: " + str(lp)) + + # Describe topics. + topics = model.describeTopics(3) + print("The topics described by their top-weighted terms:") + topics.show(truncate=False) + + # Shows the result + transformed = model.transform(dataset) + transformed.show(truncate=False) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/linear_regression_with_elastic_net.py b/examples/src/main/python/ml/linear_regression_with_elastic_net.py new file mode 100644 index 000000000000..6639e9160ab7 --- /dev/null +++ b/examples/src/main/python/ml/linear_regression_with_elastic_net.py @@ -0,0 +1,54 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.regression import LinearRegression +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("LinearRegressionWithElasticNet")\ + .getOrCreate() + + # $example on$ + # Load training data + training = spark.read.format("libsvm")\ + .load("data/mllib/sample_linear_regression_data.txt") + + lr = LinearRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) + + # Fit the model + lrModel = lr.fit(training) + + # Print the coefficients and intercept for linear regression + print("Coefficients: %s" % str(lrModel.coefficients)) + print("Intercept: %s" % str(lrModel.intercept)) + + # Summarize the model over the training set and print out some metrics + trainingSummary = lrModel.summary + print("numIterations: %d" % trainingSummary.totalIterations) + print("objectiveHistory: %s" % str(trainingSummary.objectiveHistory)) + trainingSummary.residuals.show() + print("RMSE: %f" % trainingSummary.rootMeanSquaredError) + print("r2: %f" % trainingSummary.r2) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/linearsvc.py b/examples/src/main/python/ml/linearsvc.py new file mode 100644 index 000000000000..18cbf87a1069 --- /dev/null +++ b/examples/src/main/python/ml/linearsvc.py @@ -0,0 +1,46 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.classification import LinearSVC +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("linearSVC Example")\ + .getOrCreate() + + # $example on$ + # Load training data + training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + lsvc = LinearSVC(maxIter=10, regParam=0.1) + + # Fit the model + lsvcModel = lsvc.fit(training) + + # Print the coefficients and intercept for linearsSVC + print("Coefficients: " + str(lsvcModel.coefficients)) + print("Intercept: " + str(lsvcModel.intercept)) + + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/logistic_regression.py b/examples/src/main/python/ml/logistic_regression.py deleted file mode 100644 index 55afe1b207fe..000000000000 --- a/examples/src/main/python/ml/logistic_regression.py +++ /dev/null @@ -1,67 +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. -# - -from __future__ import print_function - -import sys - -from pyspark import SparkContext -from pyspark.ml.classification import LogisticRegression -from pyspark.mllib.evaluation import MulticlassMetrics -from pyspark.ml.feature import StringIndexer -from pyspark.mllib.util import MLUtils -from pyspark.sql import SQLContext - -""" -A simple example demonstrating a logistic regression with elastic net regularization Pipeline. -Run with: - bin/spark-submit examples/src/main/python/ml/logistic_regression.py -""" - -if __name__ == "__main__": - - if len(sys.argv) > 1: - print("Usage: logistic_regression", file=sys.stderr) - exit(-1) - - sc = SparkContext(appName="PythonLogisticRegressionExample") - sqlContext = SQLContext(sc) - - # Load and parse the data file into a dataframe. - df = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() - - # Map labels into an indexed column of labels in [0, numLabels) - stringIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel") - si_model = stringIndexer.fit(df) - td = si_model.transform(df) - [training, test] = td.randomSplit([0.7, 0.3]) - - lr = LogisticRegression(maxIter=100, regParam=0.3).setLabelCol("indexedLabel") - lr.setElasticNetParam(0.8) - - # Fit the model - lrModel = lr.fit(training) - - predictionAndLabels = lrModel.transform(test).select("prediction", "indexedLabel") \ - .map(lambda x: (x.prediction, x.indexedLabel)) - - metrics = MulticlassMetrics(predictionAndLabels) - print("weighted f-measure %.3f" % metrics.weightedFMeasure()) - print("precision %s" % metrics.precision()) - print("recall %s" % metrics.recall()) - - sc.stop() diff --git a/examples/src/main/python/ml/logistic_regression_summary_example.py b/examples/src/main/python/ml/logistic_regression_summary_example.py new file mode 100644 index 000000000000..bd440a1fbe8d --- /dev/null +++ b/examples/src/main/python/ml/logistic_regression_summary_example.py @@ -0,0 +1,68 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.classification import LogisticRegression +# $example off$ +from pyspark.sql import SparkSession + +""" +An example demonstrating Logistic Regression Summary. +Run with: + bin/spark-submit examples/src/main/python/ml/logistic_regression_summary_example.py +""" + +if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("LogisticRegressionSummary") \ + .getOrCreate() + + # Load training data + training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + lr = LogisticRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) + + # Fit the model + lrModel = lr.fit(training) + + # $example on$ + # Extract the summary from the returned LogisticRegressionModel instance trained + # in the earlier example + trainingSummary = lrModel.summary + + # Obtain the objective per iteration + objectiveHistory = trainingSummary.objectiveHistory + print("objectiveHistory:") + for objective in objectiveHistory: + print(objective) + + # Obtain the receiver-operating characteristic as a dataframe and areaUnderROC. + trainingSummary.roc.show() + print("areaUnderROC: " + str(trainingSummary.areaUnderROC)) + + # Set the model threshold to maximize F-Measure + fMeasure = trainingSummary.fMeasureByThreshold + maxFMeasure = fMeasure.groupBy().max('F-Measure').select('max(F-Measure)').head() + bestThreshold = fMeasure.where(fMeasure['F-Measure'] == maxFMeasure['max(F-Measure)']) \ + .select('threshold').head()['threshold'] + lr.setThreshold(bestThreshold) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/logistic_regression_with_elastic_net.py b/examples/src/main/python/ml/logistic_regression_with_elastic_net.py new file mode 100644 index 000000000000..d095fbd37340 --- /dev/null +++ b/examples/src/main/python/ml/logistic_regression_with_elastic_net.py @@ -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. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.classification import LogisticRegression +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("LogisticRegressionWithElasticNet")\ + .getOrCreate() + + # $example on$ + # Load training data + training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + lr = LogisticRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) + + # Fit the model + lrModel = lr.fit(training) + + # Print the coefficients and intercept for logistic regression + print("Coefficients: " + str(lrModel.coefficients)) + print("Intercept: " + str(lrModel.intercept)) + + # We can also use the multinomial family for binary classification + mlr = LogisticRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8, family="multinomial") + + # Fit the model + mlrModel = mlr.fit(training) + + # Print the coefficients and intercepts for logistic regression with multinomial family + print("Multinomial coefficients: " + str(mlrModel.coefficientMatrix)) + print("Multinomial intercepts: " + str(mlrModel.interceptVector)) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/max_abs_scaler_example.py b/examples/src/main/python/ml/max_abs_scaler_example.py new file mode 100644 index 000000000000..45eda3cdadde --- /dev/null +++ b/examples/src/main/python/ml/max_abs_scaler_example.py @@ -0,0 +1,50 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import MaxAbsScaler +from pyspark.ml.linalg import Vectors +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("MaxAbsScalerExample")\ + .getOrCreate() + + # $example on$ + dataFrame = spark.createDataFrame([ + (0, Vectors.dense([1.0, 0.1, -8.0]),), + (1, Vectors.dense([2.0, 1.0, -4.0]),), + (2, Vectors.dense([4.0, 10.0, 8.0]),) + ], ["id", "features"]) + + scaler = MaxAbsScaler(inputCol="features", outputCol="scaledFeatures") + + # Compute summary statistics and generate MaxAbsScalerModel + scalerModel = scaler.fit(dataFrame) + + # rescale each feature to range [-1, 1]. + scaledData = scalerModel.transform(dataFrame) + + scaledData.select("features", "scaledFeatures").show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/min_hash_lsh_example.py b/examples/src/main/python/ml/min_hash_lsh_example.py new file mode 100644 index 000000000000..7b1dd611a865 --- /dev/null +++ b/examples/src/main/python/ml/min_hash_lsh_example.py @@ -0,0 +1,81 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import MinHashLSH +from pyspark.ml.linalg import Vectors +from pyspark.sql.functions import col +# $example off$ +from pyspark.sql import SparkSession + +""" +An example demonstrating MinHashLSH. +Run with: + bin/spark-submit examples/src/main/python/ml/min_hash_lsh_example.py +""" + +if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("MinHashLSHExample") \ + .getOrCreate() + + # $example on$ + dataA = [(0, Vectors.sparse(6, [0, 1, 2], [1.0, 1.0, 1.0]),), + (1, Vectors.sparse(6, [2, 3, 4], [1.0, 1.0, 1.0]),), + (2, Vectors.sparse(6, [0, 2, 4], [1.0, 1.0, 1.0]),)] + dfA = spark.createDataFrame(dataA, ["id", "features"]) + + dataB = [(3, Vectors.sparse(6, [1, 3, 5], [1.0, 1.0, 1.0]),), + (4, Vectors.sparse(6, [2, 3, 5], [1.0, 1.0, 1.0]),), + (5, Vectors.sparse(6, [1, 2, 4], [1.0, 1.0, 1.0]),)] + dfB = spark.createDataFrame(dataB, ["id", "features"]) + + key = Vectors.sparse(6, [1, 3], [1.0, 1.0]) + + mh = MinHashLSH(inputCol="features", outputCol="hashes", numHashTables=5) + model = mh.fit(dfA) + + # Feature Transformation + print("The hashed dataset where hashed values are stored in the column 'hashes':") + model.transform(dfA).show() + + # Compute the locality sensitive hashes for the input rows, then perform approximate + # similarity join. + # We could avoid computing hashes by passing in the already-transformed dataset, e.g. + # `model.approxSimilarityJoin(transformedA, transformedB, 0.6)` + print("Approximately joining dfA and dfB on distance smaller than 0.6:") + model.approxSimilarityJoin(dfA, dfB, 0.6, distCol="JaccardDistance")\ + .select(col("datasetA.id").alias("idA"), + col("datasetB.id").alias("idB"), + col("JaccardDistance")).show() + + # Compute the locality sensitive hashes for the input rows, then perform approximate nearest + # neighbor search. + # We could avoid computing hashes by passing in the already-transformed dataset, e.g. + # `model.approxNearestNeighbors(transformedA, key, 2)` + # It may return less than 2 rows when not enough approximate near-neighbor candidates are + # found. + print("Approximately searching dfA for 2 nearest neighbors of the key:") + model.approxNearestNeighbors(dfA, key, 2).show() + + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/min_max_scaler_example.py b/examples/src/main/python/ml/min_max_scaler_example.py new file mode 100644 index 000000000000..b5f272e59bc3 --- /dev/null +++ b/examples/src/main/python/ml/min_max_scaler_example.py @@ -0,0 +1,50 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import MinMaxScaler +from pyspark.ml.linalg import Vectors +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("MinMaxScalerExample")\ + .getOrCreate() + + # $example on$ + dataFrame = spark.createDataFrame([ + (0, Vectors.dense([1.0, 0.1, -1.0]),), + (1, Vectors.dense([2.0, 1.1, 1.0]),), + (2, Vectors.dense([3.0, 10.1, 3.0]),) + ], ["id", "features"]) + + scaler = MinMaxScaler(inputCol="features", outputCol="scaledFeatures") + + # Compute summary statistics and generate MinMaxScalerModel + scalerModel = scaler.fit(dataFrame) + + # rescale each feature to range [min, max]. + scaledData = scalerModel.transform(dataFrame) + print("Features scaled to range: [%f, %f]" % (scaler.getMin(), scaler.getMax())) + scaledData.select("features", "scaledFeatures").show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/multiclass_logistic_regression_with_elastic_net.py b/examples/src/main/python/ml/multiclass_logistic_regression_with_elastic_net.py new file mode 100644 index 000000000000..bb9cd82d6ba2 --- /dev/null +++ b/examples/src/main/python/ml/multiclass_logistic_regression_with_elastic_net.py @@ -0,0 +1,48 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.classification import LogisticRegression +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("MulticlassLogisticRegressionWithElasticNet") \ + .getOrCreate() + + # $example on$ + # Load training data + training = spark \ + .read \ + .format("libsvm") \ + .load("data/mllib/sample_multiclass_classification_data.txt") + + lr = LogisticRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) + + # Fit the model + lrModel = lr.fit(training) + + # Print the coefficients and intercept for multinomial logistic regression + print("Coefficients: \n" + str(lrModel.coefficientMatrix)) + print("Intercept: " + str(lrModel.interceptVector)) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/multilayer_perceptron_classification.py b/examples/src/main/python/ml/multilayer_perceptron_classification.py new file mode 100644 index 000000000000..88fc69f75395 --- /dev/null +++ b/examples/src/main/python/ml/multilayer_perceptron_classification.py @@ -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. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.classification import MultilayerPerceptronClassifier +from pyspark.ml.evaluation import MulticlassClassificationEvaluator +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder.appName("multilayer_perceptron_classification_example").getOrCreate() + + # $example on$ + # Load training data + data = spark.read.format("libsvm")\ + .load("data/mllib/sample_multiclass_classification_data.txt") + + # Split the data into train and test + splits = data.randomSplit([0.6, 0.4], 1234) + train = splits[0] + test = splits[1] + + # specify layers for the neural network: + # input layer of size 4 (features), two intermediate of size 5 and 4 + # and output of size 3 (classes) + layers = [4, 5, 4, 3] + + # create the trainer and set its parameters + trainer = MultilayerPerceptronClassifier(maxIter=100, layers=layers, blockSize=128, seed=1234) + + # train the model + model = trainer.fit(train) + + # compute accuracy on the test set + result = model.transform(test) + predictionAndLabels = result.select("prediction", "label") + evaluator = MulticlassClassificationEvaluator(metricName="accuracy") + print("Test set accuracy = " + str(evaluator.evaluate(predictionAndLabels))) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/n_gram_example.py b/examples/src/main/python/ml/n_gram_example.py new file mode 100644 index 000000000000..31676e076a11 --- /dev/null +++ b/examples/src/main/python/ml/n_gram_example.py @@ -0,0 +1,44 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 __future__ import print_function + +# $example on$ +from pyspark.ml.feature import NGram +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("NGramExample")\ + .getOrCreate() + + # $example on$ + wordDataFrame = spark.createDataFrame([ + (0, ["Hi", "I", "heard", "about", "Spark"]), + (1, ["I", "wish", "Java", "could", "use", "case", "classes"]), + (2, ["Logistic", "regression", "models", "are", "neat"]) + ], ["id", "words"]) + + ngram = NGram(n=2, inputCol="words", outputCol="ngrams") + + ngramDataFrame = ngram.transform(wordDataFrame) + ngramDataFrame.select("ngrams").show(truncate=False) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/naive_bayes_example.py b/examples/src/main/python/ml/naive_bayes_example.py new file mode 100644 index 000000000000..7290ab81cd0e --- /dev/null +++ b/examples/src/main/python/ml/naive_bayes_example.py @@ -0,0 +1,59 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.classification import NaiveBayes +from pyspark.ml.evaluation import MulticlassClassificationEvaluator +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("NaiveBayesExample")\ + .getOrCreate() + + # $example on$ + # Load training data + data = spark.read.format("libsvm") \ + .load("data/mllib/sample_libsvm_data.txt") + + # Split the data into train and test + splits = data.randomSplit([0.6, 0.4], 1234) + train = splits[0] + test = splits[1] + + # create the trainer and set its parameters + nb = NaiveBayes(smoothing=1.0, modelType="multinomial") + + # train the model + model = nb.fit(train) + + # select example rows to display. + predictions = model.transform(test) + predictions.show() + + # compute accuracy on the test set + evaluator = MulticlassClassificationEvaluator(labelCol="label", predictionCol="prediction", + metricName="accuracy") + accuracy = evaluator.evaluate(predictions) + print("Test set accuracy = " + str(accuracy)) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/normalizer_example.py b/examples/src/main/python/ml/normalizer_example.py new file mode 100644 index 000000000000..510bd825fd28 --- /dev/null +++ b/examples/src/main/python/ml/normalizer_example.py @@ -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. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import Normalizer +from pyspark.ml.linalg import Vectors +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("NormalizerExample")\ + .getOrCreate() + + # $example on$ + dataFrame = spark.createDataFrame([ + (0, Vectors.dense([1.0, 0.5, -1.0]),), + (1, Vectors.dense([2.0, 1.0, 1.0]),), + (2, Vectors.dense([4.0, 10.0, 2.0]),) + ], ["id", "features"]) + + # Normalize each Vector using $L^1$ norm. + normalizer = Normalizer(inputCol="features", outputCol="normFeatures", p=1.0) + l1NormData = normalizer.transform(dataFrame) + print("Normalized using L^1 norm") + l1NormData.show() + + # Normalize each Vector using $L^\infty$ norm. + lInfNormData = normalizer.transform(dataFrame, {normalizer.p: float("inf")}) + print("Normalized using L^inf norm") + lInfNormData.show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/one_vs_rest_example.py b/examples/src/main/python/ml/one_vs_rest_example.py new file mode 100644 index 000000000000..8e00c25d9342 --- /dev/null +++ b/examples/src/main/python/ml/one_vs_rest_example.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. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.classification import LogisticRegression, OneVsRest +from pyspark.ml.evaluation import MulticlassClassificationEvaluator +# $example off$ +from pyspark.sql import SparkSession + +""" +An example of Multiclass to Binary Reduction with One Vs Rest, +using Logistic Regression as the base classifier. +Run with: + bin/spark-submit examples/src/main/python/ml/one_vs_rest_example.py +""" + +if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("OneVsRestExample") \ + .getOrCreate() + + # $example on$ + # load data file. + inputData = spark.read.format("libsvm") \ + .load("data/mllib/sample_multiclass_classification_data.txt") + + # generate the train/test split. + (train, test) = inputData.randomSplit([0.8, 0.2]) + + # instantiate the base classifier. + lr = LogisticRegression(maxIter=10, tol=1E-6, fitIntercept=True) + + # instantiate the One Vs Rest Classifier. + ovr = OneVsRest(classifier=lr) + + # train the multiclass model. + ovrModel = ovr.fit(train) + + # score the model on test data. + predictions = ovrModel.transform(test) + + # obtain evaluator. + evaluator = MulticlassClassificationEvaluator(metricName="accuracy") + + # compute the classification error on test data. + accuracy = evaluator.evaluate(predictions) + print("Test Error = %g" % (1.0 - accuracy)) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/onehot_encoder_example.py b/examples/src/main/python/ml/onehot_encoder_example.py new file mode 100644 index 000000000000..e1996c7f0a55 --- /dev/null +++ b/examples/src/main/python/ml/onehot_encoder_example.py @@ -0,0 +1,50 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import OneHotEncoder, StringIndexer +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("OneHotEncoderExample")\ + .getOrCreate() + + # $example on$ + df = spark.createDataFrame([ + (0, "a"), + (1, "b"), + (2, "c"), + (3, "a"), + (4, "a"), + (5, "c") + ], ["id", "category"]) + + stringIndexer = StringIndexer(inputCol="category", outputCol="categoryIndex") + model = stringIndexer.fit(df) + indexed = model.transform(df) + + encoder = OneHotEncoder(inputCol="categoryIndex", outputCol="categoryVec") + encoded = encoder.transform(indexed) + encoded.show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/pca_example.py b/examples/src/main/python/ml/pca_example.py new file mode 100644 index 000000000000..38746aced096 --- /dev/null +++ b/examples/src/main/python/ml/pca_example.py @@ -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. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import PCA +from pyspark.ml.linalg import Vectors +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("PCAExample")\ + .getOrCreate() + + # $example on$ + data = [(Vectors.sparse(5, [(1, 1.0), (3, 7.0)]),), + (Vectors.dense([2.0, 0.0, 3.0, 4.0, 5.0]),), + (Vectors.dense([4.0, 0.0, 0.0, 6.0, 7.0]),)] + df = spark.createDataFrame(data, ["features"]) + + pca = PCA(k=3, inputCol="features", outputCol="pcaFeatures") + model = pca.fit(df) + + result = model.transform(df).select("pcaFeatures") + result.show(truncate=False) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/pipeline_example.py b/examples/src/main/python/ml/pipeline_example.py new file mode 100644 index 000000000000..e1fab7cbe6d8 --- /dev/null +++ b/examples/src/main/python/ml/pipeline_example.py @@ -0,0 +1,69 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Pipeline Example. +""" + +# $example on$ +from pyspark.ml import Pipeline +from pyspark.ml.classification import LogisticRegression +from pyspark.ml.feature import HashingTF, Tokenizer +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("PipelineExample")\ + .getOrCreate() + + # $example on$ + # Prepare training documents from a list of (id, text, label) tuples. + training = spark.createDataFrame([ + (0, "a b c d e spark", 1.0), + (1, "b d", 0.0), + (2, "spark f g h", 1.0), + (3, "hadoop mapreduce", 0.0) + ], ["id", "text", "label"]) + + # Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. + tokenizer = Tokenizer(inputCol="text", outputCol="words") + hashingTF = HashingTF(inputCol=tokenizer.getOutputCol(), outputCol="features") + lr = LogisticRegression(maxIter=10, regParam=0.001) + pipeline = Pipeline(stages=[tokenizer, hashingTF, lr]) + + # Fit the pipeline to training documents. + model = pipeline.fit(training) + + # Prepare test documents, which are unlabeled (id, text) tuples. + test = spark.createDataFrame([ + (4, "spark i j k"), + (5, "l m n"), + (6, "spark hadoop spark"), + (7, "apache hadoop") + ], ["id", "text"]) + + # Make predictions on test documents and print columns of interest. + prediction = model.transform(test) + selected = prediction.select("id", "text", "probability", "prediction") + for row in selected.collect(): + rid, text, prob, prediction = row + print("(%d, %s) --> prob=%s, prediction=%f" % (rid, text, str(prob), prediction)) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/polynomial_expansion_example.py b/examples/src/main/python/ml/polynomial_expansion_example.py new file mode 100644 index 000000000000..40bcb7b13a3d --- /dev/null +++ b/examples/src/main/python/ml/polynomial_expansion_example.py @@ -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. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import PolynomialExpansion +from pyspark.ml.linalg import Vectors +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("PolynomialExpansionExample")\ + .getOrCreate() + + # $example on$ + df = spark.createDataFrame([ + (Vectors.dense([2.0, 1.0]),), + (Vectors.dense([0.0, 0.0]),), + (Vectors.dense([3.0, -1.0]),) + ], ["features"]) + + polyExpansion = PolynomialExpansion(degree=3, inputCol="features", outputCol="polyFeatures") + polyDF = polyExpansion.transform(df) + + polyDF.show(truncate=False) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/quantile_discretizer_example.py b/examples/src/main/python/ml/quantile_discretizer_example.py new file mode 100644 index 000000000000..0fc1d1949a77 --- /dev/null +++ b/examples/src/main/python/ml/quantile_discretizer_example.py @@ -0,0 +1,48 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import QuantileDiscretizer +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("QuantileDiscretizerExample")\ + .getOrCreate() + + # $example on$ + data = [(0, 18.0), (1, 19.0), (2, 8.0), (3, 5.0), (4, 2.2)] + df = spark.createDataFrame(data, ["id", "hour"]) + # $example off$ + + # Output of QuantileDiscretizer for such small datasets can depend on the number of + # partitions. Here we force a single partition to ensure consistent results. + # Note this is not necessary for normal use cases + df = df.repartition(1) + + # $example on$ + discretizer = QuantileDiscretizer(numBuckets=3, inputCol="hour", outputCol="result") + + result = discretizer.fit(df).transform(df) + result.show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/random_forest_classifier_example.py b/examples/src/main/python/ml/random_forest_classifier_example.py new file mode 100644 index 000000000000..4eaa94dd7f48 --- /dev/null +++ b/examples/src/main/python/ml/random_forest_classifier_example.py @@ -0,0 +1,82 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Random Forest Classifier Example. +""" +from __future__ import print_function + +# $example on$ +from pyspark.ml import Pipeline +from pyspark.ml.classification import RandomForestClassifier +from pyspark.ml.feature import IndexToString, StringIndexer, VectorIndexer +from pyspark.ml.evaluation import MulticlassClassificationEvaluator +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("RandomForestClassifierExample")\ + .getOrCreate() + + # $example on$ + # Load and parse the data file, converting it to a DataFrame. + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + # Index labels, adding metadata to the label column. + # Fit on whole dataset to include all labels in index. + labelIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel").fit(data) + + # Automatically identify categorical features, and index them. + # Set maxCategories so features with > 4 distinct values are treated as continuous. + featureIndexer =\ + VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) + + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + # Train a RandomForest model. + rf = RandomForestClassifier(labelCol="indexedLabel", featuresCol="indexedFeatures", numTrees=10) + + # Convert indexed labels back to original labels. + labelConverter = IndexToString(inputCol="prediction", outputCol="predictedLabel", + labels=labelIndexer.labels) + + # Chain indexers and forest in a Pipeline + pipeline = Pipeline(stages=[labelIndexer, featureIndexer, rf, labelConverter]) + + # Train model. This also runs the indexers. + model = pipeline.fit(trainingData) + + # Make predictions. + predictions = model.transform(testData) + + # Select example rows to display. + predictions.select("predictedLabel", "label", "features").show(5) + + # Select (prediction, true label) and compute test error + evaluator = MulticlassClassificationEvaluator( + labelCol="indexedLabel", predictionCol="prediction", metricName="accuracy") + accuracy = evaluator.evaluate(predictions) + print("Test Error = %g" % (1.0 - accuracy)) + + rfModel = model.stages[2] + print(rfModel) # summary only + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/random_forest_example.py b/examples/src/main/python/ml/random_forest_example.py deleted file mode 100644 index c7730e1bfacd..000000000000 --- a/examples/src/main/python/ml/random_forest_example.py +++ /dev/null @@ -1,87 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -from __future__ import print_function - -import sys - -from pyspark import SparkContext -from pyspark.ml.classification import RandomForestClassifier -from pyspark.ml.feature import StringIndexer -from pyspark.ml.regression import RandomForestRegressor -from pyspark.mllib.evaluation import MulticlassMetrics, RegressionMetrics -from pyspark.mllib.util import MLUtils -from pyspark.sql import Row, SQLContext - -""" -A simple example demonstrating a RandomForest Classification/Regression Pipeline. -Run with: - bin/spark-submit examples/src/main/python/ml/random_forest_example.py -""" - - -def testClassification(train, test): - # Train a RandomForest model. - # Setting featureSubsetStrategy="auto" lets the algorithm choose. - # Note: Use larger numTrees in practice. - - rf = RandomForestClassifier(labelCol="indexedLabel", numTrees=3, maxDepth=4) - - model = rf.fit(train) - predictionAndLabels = model.transform(test).select("prediction", "indexedLabel") \ - .map(lambda x: (x.prediction, x.indexedLabel)) - - metrics = MulticlassMetrics(predictionAndLabels) - print("weighted f-measure %.3f" % metrics.weightedFMeasure()) - print("precision %s" % metrics.precision()) - print("recall %s" % metrics.recall()) - - -def testRegression(train, test): - # Train a RandomForest model. - # Note: Use larger numTrees in practice. - - rf = RandomForestRegressor(labelCol="indexedLabel", numTrees=3, maxDepth=4) - - model = rf.fit(train) - predictionAndLabels = model.transform(test).select("prediction", "indexedLabel") \ - .map(lambda x: (x.prediction, x.indexedLabel)) - - metrics = RegressionMetrics(predictionAndLabels) - print("rmse %.3f" % metrics.rootMeanSquaredError) - print("r2 %.3f" % metrics.r2) - print("mae %.3f" % metrics.meanAbsoluteError) - - -if __name__ == "__main__": - if len(sys.argv) > 1: - print("Usage: random_forest_example", file=sys.stderr) - exit(1) - sc = SparkContext(appName="PythonRandomForestExample") - sqlContext = SQLContext(sc) - - # Load and parse the data file into a dataframe. - df = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() - - # Map labels into an indexed column of labels in [0, numLabels) - stringIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel") - si_model = stringIndexer.fit(df) - td = si_model.transform(df) - [train, test] = td.randomSplit([0.7, 0.3]) - testClassification(train, test) - testRegression(train, test) - sc.stop() diff --git a/examples/src/main/python/ml/random_forest_regressor_example.py b/examples/src/main/python/ml/random_forest_regressor_example.py new file mode 100644 index 000000000000..a34edff2ecaa --- /dev/null +++ b/examples/src/main/python/ml/random_forest_regressor_example.py @@ -0,0 +1,74 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Random Forest Regressor Example. +""" +from __future__ import print_function + +# $example on$ +from pyspark.ml import Pipeline +from pyspark.ml.regression import RandomForestRegressor +from pyspark.ml.feature import VectorIndexer +from pyspark.ml.evaluation import RegressionEvaluator +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("RandomForestRegressorExample")\ + .getOrCreate() + + # $example on$ + # Load and parse the data file, converting it to a DataFrame. + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + # Automatically identify categorical features, and index them. + # Set maxCategories so features with > 4 distinct values are treated as continuous. + featureIndexer =\ + VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) + + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + # Train a RandomForest model. + rf = RandomForestRegressor(featuresCol="indexedFeatures") + + # Chain indexer and forest in a Pipeline + pipeline = Pipeline(stages=[featureIndexer, rf]) + + # Train model. This also runs the indexer. + model = pipeline.fit(trainingData) + + # Make predictions. + predictions = model.transform(testData) + + # Select example rows to display. + predictions.select("prediction", "label", "features").show(5) + + # Select (prediction, true label) and compute test error + evaluator = RegressionEvaluator( + labelCol="label", predictionCol="prediction", metricName="rmse") + rmse = evaluator.evaluate(predictions) + print("Root Mean Squared Error (RMSE) on test data = %g" % rmse) + + rfModel = model.stages[1] + print(rfModel) # summary only + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/rformula_example.py b/examples/src/main/python/ml/rformula_example.py new file mode 100644 index 000000000000..6629239db29e --- /dev/null +++ b/examples/src/main/python/ml/rformula_example.py @@ -0,0 +1,47 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import RFormula +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("RFormulaExample")\ + .getOrCreate() + + # $example on$ + dataset = spark.createDataFrame( + [(7, "US", 18, 1.0), + (8, "CA", 12, 0.0), + (9, "NZ", 15, 0.0)], + ["id", "country", "hour", "clicked"]) + + formula = RFormula( + formula="clicked ~ country + hour", + featuresCol="features", + labelCol="label") + + output = formula.fit(dataset).transform(dataset) + output.select("features", "label").show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/simple_params_example.py b/examples/src/main/python/ml/simple_params_example.py deleted file mode 100644 index 2d6d115d54d0..000000000000 --- a/examples/src/main/python/ml/simple_params_example.py +++ /dev/null @@ -1,98 +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. -# - -from __future__ import print_function - -import pprint -import sys - -from pyspark import SparkContext -from pyspark.ml.classification import LogisticRegression -from pyspark.mllib.linalg import DenseVector -from pyspark.mllib.regression import LabeledPoint -from pyspark.sql import SQLContext - -""" -A simple example demonstrating ways to specify parameters for Estimators and Transformers. -Run with: - bin/spark-submit examples/src/main/python/ml/simple_params_example.py -""" - -if __name__ == "__main__": - if len(sys.argv) > 1: - print("Usage: simple_params_example", file=sys.stderr) - exit(1) - sc = SparkContext(appName="PythonSimpleParamsExample") - sqlContext = SQLContext(sc) - - # prepare training data. - # We create an RDD of LabeledPoints and convert them into a DataFrame. - # A LabeledPoint is an Object with two fields named label and features - # and Spark SQL identifies these fields and creates the schema appropriately. - training = sc.parallelize([ - LabeledPoint(1.0, DenseVector([0.0, 1.1, 0.1])), - LabeledPoint(0.0, DenseVector([2.0, 1.0, -1.0])), - LabeledPoint(0.0, DenseVector([2.0, 1.3, 1.0])), - LabeledPoint(1.0, DenseVector([0.0, 1.2, -0.5]))]).toDF() - - # Create a LogisticRegression instance with maxIter = 10. - # This instance is an Estimator. - lr = LogisticRegression(maxIter=10) - # Print out the parameters, documentation, and any default values. - print("LogisticRegression parameters:\n" + lr.explainParams() + "\n") - - # We may also set parameters using setter methods. - lr.setRegParam(0.01) - - # Learn a LogisticRegression model. This uses the parameters stored in lr. - model1 = lr.fit(training) - - # 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 - # LogisticRegression instance. - print("Model 1 was fit using parameters:\n") - pprint.pprint(model1.extractParamMap()) - - # We may alternatively specify parameters using a parameter map. - # paramMap overrides all lr parameters set earlier. - paramMap = {lr.maxIter: 20, lr.thresholds: [0.45, 0.55], lr.probabilityCol: "myProbability"} - - # Now learn a new model using the new parameters. - model2 = lr.fit(training, paramMap) - print("Model 2 was fit using parameters:\n") - pprint.pprint(model2.extractParamMap()) - - # prepare test data. - test = sc.parallelize([ - LabeledPoint(1.0, DenseVector([-1.0, 1.5, 1.3])), - LabeledPoint(0.0, DenseVector([3.0, 2.0, -0.1])), - LabeledPoint(0.0, DenseVector([0.0, 2.2, -1.5]))]).toDF() - - # Make predictions on test data using the Transformer.transform() method. - # LogisticRegressionModel.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. - result = model2.transform(test) \ - .select("features", "label", "myProbability", "prediction") \ - .collect() - - for row in result: - print("features=%s,label=%s -> prob=%s, prediction=%s" - % (row.features, row.label, row.myProbability, row.prediction)) - - sc.stop() diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py deleted file mode 100644 index b4f06bf88874..000000000000 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ /dev/null @@ -1,71 +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. -# - -from __future__ import print_function - -from pyspark import SparkContext -from pyspark.ml import Pipeline -from pyspark.ml.classification import LogisticRegression -from pyspark.ml.feature import HashingTF, Tokenizer -from pyspark.sql import Row, SQLContext - - -""" -A simple text classification pipeline that recognizes "spark" from -input text. This is to show how to create and configure a Spark ML -pipeline in Python. Run with: - - bin/spark-submit examples/src/main/python/ml/simple_text_classification_pipeline.py -""" - - -if __name__ == "__main__": - sc = SparkContext(appName="SimpleTextClassificationPipeline") - sqlContext = SQLContext(sc) - - # Prepare training documents, which are labeled. - LabeledDocument = Row("id", "text", "label") - training = sc.parallelize([(0, "a b c d e spark", 1.0), - (1, "b d", 0.0), - (2, "spark f g h", 1.0), - (3, "hadoop mapreduce", 0.0)]) \ - .map(lambda x: LabeledDocument(*x)).toDF() - - # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. - tokenizer = Tokenizer(inputCol="text", outputCol="words") - hashingTF = HashingTF(inputCol=tokenizer.getOutputCol(), outputCol="features") - lr = LogisticRegression(maxIter=10, regParam=0.001) - pipeline = Pipeline(stages=[tokenizer, hashingTF, lr]) - - # Fit the pipeline to training documents. - model = pipeline.fit(training) - - # Prepare test documents, which are unlabeled. - Document = Row("id", "text") - test = sc.parallelize([(4, "spark i j k"), - (5, "l m n"), - (6, "spark hadoop spark"), - (7, "apache hadoop")]) \ - .map(lambda x: Document(*x)).toDF() - - # Make predictions on test documents and print columns of interest. - prediction = model.transform(test) - selected = prediction.select("id", "text", "prediction") - for row in selected.collect(): - print(row) - - sc.stop() diff --git a/examples/src/main/python/ml/sql_transformer.py b/examples/src/main/python/ml/sql_transformer.py new file mode 100644 index 000000000000..0bf8f35720c9 --- /dev/null +++ b/examples/src/main/python/ml/sql_transformer.py @@ -0,0 +1,41 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import SQLTransformer +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("SQLTransformerExample")\ + .getOrCreate() + + # $example on$ + df = spark.createDataFrame([ + (0, 1.0, 3.0), + (2, 2.0, 5.0) + ], ["id", "v1", "v2"]) + sqlTrans = SQLTransformer( + statement="SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__") + sqlTrans.transform(df).show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/standard_scaler_example.py b/examples/src/main/python/ml/standard_scaler_example.py new file mode 100644 index 000000000000..c0027480e69b --- /dev/null +++ b/examples/src/main/python/ml/standard_scaler_example.py @@ -0,0 +1,44 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 __future__ import print_function + +# $example on$ +from pyspark.ml.feature import StandardScaler +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("StandardScalerExample")\ + .getOrCreate() + + # $example on$ + dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + scaler = StandardScaler(inputCol="features", outputCol="scaledFeatures", + withStd=True, withMean=False) + + # Compute summary statistics by fitting the StandardScaler + scalerModel = scaler.fit(dataFrame) + + # Normalize each feature to have unit standard deviation. + scaledData = scalerModel.transform(dataFrame) + scaledData.show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/stopwords_remover_example.py b/examples/src/main/python/ml/stopwords_remover_example.py new file mode 100644 index 000000000000..3b8e7855e3e7 --- /dev/null +++ b/examples/src/main/python/ml/stopwords_remover_example.py @@ -0,0 +1,41 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import StopWordsRemover +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("StopWordsRemoverExample")\ + .getOrCreate() + + # $example on$ + sentenceData = spark.createDataFrame([ + (0, ["I", "saw", "the", "red", "balloon"]), + (1, ["Mary", "had", "a", "little", "lamb"]) + ], ["id", "raw"]) + + remover = StopWordsRemover(inputCol="raw", outputCol="filtered") + remover.transform(sentenceData).show(truncate=False) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/string_indexer_example.py b/examples/src/main/python/ml/string_indexer_example.py new file mode 100644 index 000000000000..2255bfb9c1a6 --- /dev/null +++ b/examples/src/main/python/ml/string_indexer_example.py @@ -0,0 +1,41 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import StringIndexer +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("StringIndexerExample")\ + .getOrCreate() + + # $example on$ + df = spark.createDataFrame( + [(0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")], + ["id", "category"]) + + indexer = StringIndexer(inputCol="category", outputCol="categoryIndex") + indexed = indexer.fit(df).transform(df) + indexed.show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/tf_idf_example.py b/examples/src/main/python/ml/tf_idf_example.py index c92313378eec..d43244fa68e9 100644 --- a/examples/src/main/python/ml/tf_idf_example.py +++ b/examples/src/main/python/ml/tf_idf_example.py @@ -17,31 +17,36 @@ from __future__ import print_function -from pyspark import SparkContext # $example on$ from pyspark.ml.feature import HashingTF, IDF, Tokenizer # $example off$ -from pyspark.sql import SQLContext +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="TfIdfExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("TfIdfExample")\ + .getOrCreate() # $example on$ - sentenceData = sqlContext.createDataFrame([ - (0, "Hi I heard about Spark"), - (0, "I wish Java could use case classes"), - (1, "Logistic regression models are neat") + sentenceData = spark.createDataFrame([ + (0.0, "Hi I heard about Spark"), + (0.0, "I wish Java could use case classes"), + (1.0, "Logistic regression models are neat") ], ["label", "sentence"]) + tokenizer = Tokenizer(inputCol="sentence", outputCol="words") wordsData = tokenizer.transform(sentenceData) + hashingTF = HashingTF(inputCol="words", outputCol="rawFeatures", numFeatures=20) featurizedData = hashingTF.transform(wordsData) + # alternatively, CountVectorizer can also be used to get term frequency vectors + idf = IDF(inputCol="rawFeatures", outputCol="features") idfModel = idf.fit(featurizedData) rescaledData = idfModel.transform(featurizedData) - for features_label in rescaledData.select("features", "label").take(3): - print(features_label) + + rescaledData.select("label", "features").show() # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/ml/tokenizer_example.py b/examples/src/main/python/ml/tokenizer_example.py new file mode 100644 index 000000000000..5c65c5c9f826 --- /dev/null +++ b/examples/src/main/python/ml/tokenizer_example.py @@ -0,0 +1,56 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import Tokenizer, RegexTokenizer +from pyspark.sql.functions import col, udf +from pyspark.sql.types import IntegerType +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("TokenizerExample")\ + .getOrCreate() + + # $example on$ + sentenceDataFrame = spark.createDataFrame([ + (0, "Hi I heard about Spark"), + (1, "I wish Java could use case classes"), + (2, "Logistic,regression,models,are,neat") + ], ["id", "sentence"]) + + tokenizer = Tokenizer(inputCol="sentence", outputCol="words") + + regexTokenizer = RegexTokenizer(inputCol="sentence", outputCol="words", pattern="\\W") + # alternatively, pattern="\\w+", gaps(False) + + countTokens = udf(lambda words: len(words), IntegerType()) + + tokenized = tokenizer.transform(sentenceDataFrame) + tokenized.select("sentence", "words")\ + .withColumn("tokens", countTokens(col("words"))).show(truncate=False) + + regexTokenized = regexTokenizer.transform(sentenceDataFrame) + regexTokenized.select("sentence", "words") \ + .withColumn("tokens", countTokens(col("words"))).show(truncate=False) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/train_validation_split.py b/examples/src/main/python/ml/train_validation_split.py new file mode 100644 index 000000000000..d104f7d30a1b --- /dev/null +++ b/examples/src/main/python/ml/train_validation_split.py @@ -0,0 +1,74 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# $example on$ +from pyspark.ml.evaluation import RegressionEvaluator +from pyspark.ml.regression import LinearRegression +from pyspark.ml.tuning import ParamGridBuilder, TrainValidationSplit +# $example off$ +from pyspark.sql import SparkSession + +""" +This example demonstrates applying TrainValidationSplit to split data +and preform model selection. +Run with: + + bin/spark-submit examples/src/main/python/ml/train_validation_split.py +""" + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("TrainValidationSplit")\ + .getOrCreate() + + # $example on$ + # Prepare training and test data. + data = spark.read.format("libsvm")\ + .load("data/mllib/sample_linear_regression_data.txt") + train, test = data.randomSplit([0.9, 0.1], seed=12345) + + lr = LinearRegression(maxIter=10) + + # We use a ParamGridBuilder to construct a grid of parameters to search over. + # TrainValidationSplit will try all combinations of values and determine best model using + # the evaluator. + paramGrid = ParamGridBuilder()\ + .addGrid(lr.regParam, [0.1, 0.01]) \ + .addGrid(lr.fitIntercept, [False, True])\ + .addGrid(lr.elasticNetParam, [0.0, 0.5, 1.0])\ + .build() + + # In this case the estimator is simply the linear regression. + # A TrainValidationSplit requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. + tvs = TrainValidationSplit(estimator=lr, + estimatorParamMaps=paramGrid, + evaluator=RegressionEvaluator(), + # 80% of the data will be used for training, 20% for validation. + trainRatio=0.8) + + # Run TrainValidationSplit, and choose the best set of parameters. + model = tvs.fit(train) + + # Make predictions on test data. model is the model with combination of parameters + # that performed best. + model.transform(test)\ + .select("features", "label", "prediction")\ + .show() + + # $example off$ + spark.stop() diff --git a/examples/src/main/python/ml/vector_assembler_example.py b/examples/src/main/python/ml/vector_assembler_example.py new file mode 100644 index 000000000000..98de1d5ea7da --- /dev/null +++ b/examples/src/main/python/ml/vector_assembler_example.py @@ -0,0 +1,46 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.linalg import Vectors +from pyspark.ml.feature import VectorAssembler +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("VectorAssemblerExample")\ + .getOrCreate() + + # $example on$ + dataset = spark.createDataFrame( + [(0, 18, 1.0, Vectors.dense([0.0, 10.0, 0.5]), 1.0)], + ["id", "hour", "mobile", "userFeatures", "clicked"]) + + assembler = VectorAssembler( + inputCols=["hour", "mobile", "userFeatures"], + outputCol="features") + + output = assembler.transform(dataset) + print("Assembled columns 'hour', 'mobile', 'userFeatures' to vector column 'features'") + output.select("features", "clicked").show(truncate=False) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/vector_indexer_example.py b/examples/src/main/python/ml/vector_indexer_example.py new file mode 100644 index 000000000000..5c2956077d6c --- /dev/null +++ b/examples/src/main/python/ml/vector_indexer_example.py @@ -0,0 +1,46 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import VectorIndexer +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("VectorIndexerExample")\ + .getOrCreate() + + # $example on$ + data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + indexer = VectorIndexer(inputCol="features", outputCol="indexed", maxCategories=10) + indexerModel = indexer.fit(data) + + categoricalFeatures = indexerModel.categoryMaps + print("Chose %d categorical features: %s" % + (len(categoricalFeatures), ", ".join(str(k) for k in categoricalFeatures.keys()))) + + # Create new column "indexed" with categorical values transformed to indices + indexedData = indexerModel.transform(data) + indexedData.show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/vector_slicer_example.py b/examples/src/main/python/ml/vector_slicer_example.py new file mode 100644 index 000000000000..68c8cfe27e37 --- /dev/null +++ b/examples/src/main/python/ml/vector_slicer_example.py @@ -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. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import VectorSlicer +from pyspark.ml.linalg import Vectors +from pyspark.sql.types import Row +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("VectorSlicerExample")\ + .getOrCreate() + + # $example on$ + df = spark.createDataFrame([ + Row(userFeatures=Vectors.sparse(3, {0: -2.0, 1: 2.3})), + Row(userFeatures=Vectors.dense([-2.0, 2.3, 0.0]))]) + + slicer = VectorSlicer(inputCol="userFeatures", outputCol="features", indices=[1]) + + output = slicer.transform(df) + + output.select("userFeatures", "features").show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/word2vec_example.py b/examples/src/main/python/ml/word2vec_example.py index 53c77feb1014..77f8951df088 100644 --- a/examples/src/main/python/ml/word2vec_example.py +++ b/examples/src/main/python/ml/word2vec_example.py @@ -17,29 +17,33 @@ from __future__ import print_function -from pyspark import SparkContext -from pyspark.sql import SQLContext # $example on$ from pyspark.ml.feature import Word2Vec # $example off$ +from pyspark.sql import SparkSession if __name__ == "__main__": - sc = SparkContext(appName="Word2VecExample") - sqlContext = SQLContext(sc) + spark = SparkSession\ + .builder\ + .appName("Word2VecExample")\ + .getOrCreate() # $example on$ # Input data: Each row is a bag of words from a sentence or document. - documentDF = sqlContext.createDataFrame([ + documentDF = spark.createDataFrame([ ("Hi I heard about Spark".split(" "), ), ("I wish Java could use case classes".split(" "), ), ("Logistic regression models are neat".split(" "), ) ], ["text"]) + # Learn a mapping from words to Vectors. word2Vec = Word2Vec(vectorSize=3, minCount=0, inputCol="text", outputCol="result") model = word2Vec.fit(documentDF) + result = model.transform(documentDF) - for feature in result.select("result").take(3): - print(feature) + for row in result.collect(): + text, vector = row + print("Text: [%s] => \nVector: %s\n" % (", ".join(text), str(vector))) # $example off$ - sc.stop() + spark.stop() diff --git a/examples/src/main/python/mllib/binary_classification_metrics_example.py b/examples/src/main/python/mllib/binary_classification_metrics_example.py new file mode 100644 index 000000000000..d14ce7982e24 --- /dev/null +++ b/examples/src/main/python/mllib/binary_classification_metrics_example.py @@ -0,0 +1,56 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +""" +Binary Classification Metrics Example. +""" +from __future__ import print_function +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.classification import LogisticRegressionWithLBFGS +from pyspark.mllib.evaluation import BinaryClassificationMetrics +from pyspark.mllib.util import MLUtils +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="BinaryClassificationMetricsExample") + + # $example on$ + # Several of the methods available in scala are currently missing from pyspark + # Load training data in LIBSVM format + data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_binary_classification_data.txt") + + # Split data into training (60%) and test (40%) + training, test = data.randomSplit([0.6, 0.4], seed=11) + training.cache() + + # Run training algorithm to build the model + model = LogisticRegressionWithLBFGS.train(training) + + # Compute raw scores on the test set + predictionAndLabels = test.map(lambda lp: (float(model.predict(lp.features)), lp.label)) + + # Instantiate metrics object + metrics = BinaryClassificationMetrics(predictionAndLabels) + + # Area under precision-recall curve + print("Area under PR = %s" % metrics.areaUnderPR) + + # Area under ROC curve + print("Area under ROC = %s" % metrics.areaUnderROC) + # $example off$ + + sc.stop() diff --git a/examples/src/main/python/mllib/bisecting_k_means_example.py b/examples/src/main/python/mllib/bisecting_k_means_example.py new file mode 100644 index 000000000000..31f3e72d7ff1 --- /dev/null +++ b/examples/src/main/python/mllib/bisecting_k_means_example.py @@ -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. +# + +from __future__ import print_function + +# $example on$ +from numpy import array +# $example off$ + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.clustering import BisectingKMeans, BisectingKMeansModel +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="PythonBisectingKMeansExample") # SparkContext + + # $example on$ + # Load and parse the data + data = sc.textFile("data/mllib/kmeans_data.txt") + parsedData = data.map(lambda line: array([float(x) for x in line.split(' ')])) + + # Build the model (cluster the data) + model = BisectingKMeans.train(parsedData, 2, maxIterations=5) + + # Evaluate clustering + cost = model.computeCost(parsedData) + print("Bisecting K-means Cost = " + str(cost)) + # $example off$ + + sc.stop() diff --git a/examples/src/main/python/mllib/correlations_example.py b/examples/src/main/python/mllib/correlations_example.py new file mode 100644 index 000000000000..66d18f6e5df1 --- /dev/null +++ b/examples/src/main/python/mllib/correlations_example.py @@ -0,0 +1,48 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +import numpy as np + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.stat import Statistics +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="CorrelationsExample") # SparkContext + + # $example on$ + seriesX = sc.parallelize([1.0, 2.0, 3.0, 3.0, 5.0]) # a series + # seriesY must have the same number of partitions and cardinality as seriesX + seriesY = sc.parallelize([11.0, 22.0, 33.0, 33.0, 555.0]) + + # Compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. + # If a method is not specified, Pearson's method will be used by default. + print("Correlation is: " + str(Statistics.corr(seriesX, seriesY, method="pearson"))) + + data = sc.parallelize( + [np.array([1.0, 10.0, 100.0]), np.array([2.0, 20.0, 200.0]), np.array([5.0, 33.0, 366.0])] + ) # an RDD of Vectors + + # calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. + # If a method is not specified, Pearson's method will be used by default. + print(Statistics.corr(data, method="pearson")) + # $example off$ + + sc.stop() diff --git a/examples/src/main/python/mllib/dataset_example.py b/examples/src/main/python/mllib/dataset_example.py deleted file mode 100644 index e23ecc0c5d30..000000000000 --- a/examples/src/main/python/mllib/dataset_example.py +++ /dev/null @@ -1,63 +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. -# - -""" -An example of how to use DataFrame as a dataset for ML. Run with:: - bin/spark-submit examples/src/main/python/mllib/dataset_example.py -""" -from __future__ import print_function - -import os -import sys -import tempfile -import shutil - -from pyspark import SparkContext -from pyspark.sql import SQLContext -from pyspark.mllib.util import MLUtils -from pyspark.mllib.stat import Statistics - - -def summarize(dataset): - print("schema: %s" % dataset.schema().json()) - labels = dataset.map(lambda r: r.label) - print("label average: %f" % labels.mean()) - features = dataset.map(lambda r: r.features) - summary = Statistics.colStats(features) - print("features average: %r" % summary.mean()) - -if __name__ == "__main__": - if len(sys.argv) > 2: - print("Usage: dataset_example.py ", file=sys.stderr) - exit(-1) - sc = SparkContext(appName="DatasetExample") - sqlContext = SQLContext(sc) - if len(sys.argv) == 2: - input = sys.argv[1] - else: - input = "data/mllib/sample_libsvm_data.txt" - points = MLUtils.loadLibSVMFile(sc, input) - dataset0 = sqlContext.inferSchema(points).setName("dataset0").cache() - summarize(dataset0) - tempdir = tempfile.NamedTemporaryFile(delete=False).name - os.unlink(tempdir) - print("Save dataset as a Parquet file to %s." % tempdir) - dataset0.saveAsParquetFile(tempdir) - print("Load it back and summarize it again.") - dataset1 = sqlContext.parquetFile(tempdir).setName("dataset1").cache() - summarize(dataset1) - shutil.rmtree(tempdir) diff --git a/examples/src/main/python/mllib/decision_tree_classification_example.py b/examples/src/main/python/mllib/decision_tree_classification_example.py new file mode 100644 index 000000000000..7eecf500584a --- /dev/null +++ b/examples/src/main/python/mllib/decision_tree_classification_example.py @@ -0,0 +1,56 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Decision Tree Classification Example. +""" +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.tree import DecisionTree, DecisionTreeModel +from pyspark.mllib.util import MLUtils +# $example off$ + +if __name__ == "__main__": + + sc = SparkContext(appName="PythonDecisionTreeClassificationExample") + + # $example on$ + # Load and parse the data file into an RDD of LabeledPoint. + data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + # Train a DecisionTree model. + # Empty categoricalFeaturesInfo indicates all features are continuous. + model = DecisionTree.trainClassifier(trainingData, numClasses=2, categoricalFeaturesInfo={}, + impurity='gini', maxDepth=5, maxBins=32) + + # Evaluate model on test instances and compute test error + predictions = model.predict(testData.map(lambda x: x.features)) + labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) + testErr = labelsAndPredictions.filter( + lambda lp: lp[0] != lp[1]).count() / float(testData.count()) + print('Test Error = ' + str(testErr)) + print('Learned classification tree model:') + print(model.toDebugString()) + + # Save and load model + model.save(sc, "target/tmp/myDecisionTreeClassificationModel") + sameModel = DecisionTreeModel.load(sc, "target/tmp/myDecisionTreeClassificationModel") + # $example off$ diff --git a/examples/src/main/python/mllib/decision_tree_regression_example.py b/examples/src/main/python/mllib/decision_tree_regression_example.py new file mode 100644 index 000000000000..acf9e25fdf31 --- /dev/null +++ b/examples/src/main/python/mllib/decision_tree_regression_example.py @@ -0,0 +1,56 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Decision Tree Regression Example. +""" +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.tree import DecisionTree, DecisionTreeModel +from pyspark.mllib.util import MLUtils +# $example off$ + +if __name__ == "__main__": + + sc = SparkContext(appName="PythonDecisionTreeRegressionExample") + + # $example on$ + # Load and parse the data file into an RDD of LabeledPoint. + data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + # Train a DecisionTree model. + # Empty categoricalFeaturesInfo indicates all features are continuous. + model = DecisionTree.trainRegressor(trainingData, categoricalFeaturesInfo={}, + impurity='variance', maxDepth=5, maxBins=32) + + # Evaluate model on test instances and compute test error + predictions = model.predict(testData.map(lambda x: x.features)) + labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) + testMSE = labelsAndPredictions.map(lambda lp: (lp[0] - lp[1]) * (lp[0] - lp[1])).sum() /\ + float(testData.count()) + print('Test Mean Squared Error = ' + str(testMSE)) + print('Learned regression tree model:') + print(model.toDebugString()) + + # Save and load model + model.save(sc, "target/tmp/myDecisionTreeRegressionModel") + sameModel = DecisionTreeModel.load(sc, "target/tmp/myDecisionTreeRegressionModel") + # $example off$ diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py deleted file mode 100755 index 513ed8fd5145..000000000000 --- a/examples/src/main/python/mllib/decision_tree_runner.py +++ /dev/null @@ -1,144 +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. -# - -""" -Decision tree classification and regression using MLlib. - -This example requires NumPy (http://www.numpy.org/). -""" -from __future__ import print_function - -import numpy -import os -import sys - -from operator import add - -from pyspark import SparkContext -from pyspark.mllib.regression import LabeledPoint -from pyspark.mllib.tree import DecisionTree -from pyspark.mllib.util import MLUtils - - -def getAccuracy(dtModel, data): - """ - Return accuracy of DecisionTreeModel on the given RDD[LabeledPoint]. - """ - seqOp = (lambda acc, x: acc + (x[0] == x[1])) - predictions = dtModel.predict(data.map(lambda x: x.features)) - truth = data.map(lambda p: p.label) - trainCorrect = predictions.zip(truth).aggregate(0, seqOp, add) - if data.count() == 0: - return 0 - return trainCorrect / (0.0 + data.count()) - - -def getMSE(dtModel, data): - """ - Return mean squared error (MSE) of DecisionTreeModel on the given - RDD[LabeledPoint]. - """ - seqOp = (lambda acc, x: acc + numpy.square(x[0] - x[1])) - predictions = dtModel.predict(data.map(lambda x: x.features)) - truth = data.map(lambda p: p.label) - trainMSE = predictions.zip(truth).aggregate(0, seqOp, add) - if data.count() == 0: - return 0 - return trainMSE / (0.0 + data.count()) - - -def reindexClassLabels(data): - """ - Re-index class labels in a dataset to the range {0,...,numClasses-1}. - If all labels in that range already appear at least once, - then the returned RDD is the same one (without a mapping). - Note: If a label simply does not appear in the data, - the index will not include it. - Be aware of this when reindexing subsampled data. - :param data: RDD of LabeledPoint where labels are integer values - denoting labels for a classification problem. - :return: Pair (reindexedData, origToNewLabels) where - reindexedData is an RDD of LabeledPoint with labels in - the range {0,...,numClasses-1}, and - origToNewLabels is a dictionary mapping original labels - to new labels. - """ - # classCounts: class --> # examples in class - classCounts = data.map(lambda x: x.label).countByValue() - numExamples = sum(classCounts.values()) - sortedClasses = sorted(classCounts.keys()) - numClasses = len(classCounts) - # origToNewLabels: class --> index in 0,...,numClasses-1 - if (numClasses < 2): - print("Dataset for classification should have at least 2 classes." - " The given dataset had only %d classes." % numClasses, file=sys.stderr) - exit(1) - origToNewLabels = dict([(sortedClasses[i], i) for i in range(0, numClasses)]) - - print("numClasses = %d" % numClasses) - print("Per-class example fractions, counts:") - print("Class\tFrac\tCount") - for c in sortedClasses: - frac = classCounts[c] / (numExamples + 0.0) - print("%g\t%g\t%d" % (c, frac, classCounts[c])) - - if (sortedClasses[0] == 0 and sortedClasses[-1] == numClasses - 1): - return (data, origToNewLabels) - else: - reindexedData = \ - data.map(lambda x: LabeledPoint(origToNewLabels[x.label], x.features)) - return (reindexedData, origToNewLabels) - - -def usage(): - print("Usage: decision_tree_runner [libsvm format data filepath]", file=sys.stderr) - exit(1) - - -if __name__ == "__main__": - if len(sys.argv) > 2: - usage() - sc = SparkContext(appName="PythonDT") - - # Load data. - dataPath = 'data/mllib/sample_libsvm_data.txt' - if len(sys.argv) == 2: - dataPath = sys.argv[1] - if not os.path.isfile(dataPath): - sc.stop() - usage() - points = MLUtils.loadLibSVMFile(sc, dataPath) - - # Re-index class labels if needed. - (reindexedData, origToNewLabels) = reindexClassLabels(points) - numClasses = len(origToNewLabels) - - # Train a classifier. - categoricalFeaturesInfo = {} # no categorical features - model = DecisionTree.trainClassifier(reindexedData, numClasses=numClasses, - categoricalFeaturesInfo=categoricalFeaturesInfo) - # Print learned tree and stats. - print("Trained DecisionTree for classification:") - print(" Model numNodes: %d" % model.numNodes()) - print(" Model depth: %d" % model.depth()) - print(" Training accuracy: %g" % getAccuracy(model, reindexedData)) - if model.numNodes() < 20: - print(model.toDebugString()) - else: - print(model) - - sc.stop() diff --git a/examples/src/main/python/mllib/elementwise_product_example.py b/examples/src/main/python/mllib/elementwise_product_example.py new file mode 100644 index 000000000000..8ae9afb1dc47 --- /dev/null +++ b/examples/src/main/python/mllib/elementwise_product_example.py @@ -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. +# + +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.feature import ElementwiseProduct +from pyspark.mllib.linalg import Vectors +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="ElementwiseProductExample") # SparkContext + + # $example on$ + data = sc.textFile("data/mllib/kmeans_data.txt") + parsedData = data.map(lambda x: [float(t) for t in x.split(" ")]) + + # Create weight vector. + transformingVector = Vectors.dense([0.0, 1.0, 2.0]) + transformer = ElementwiseProduct(transformingVector) + + # Batch transform + transformedData = transformer.transform(parsedData) + # Single-row transform + transformedData2 = transformer.transform(parsedData.first()) + # $example off$ + + print("transformedData:") + for each in transformedData.collect(): + print(each) + + print("transformedData2:") + for each in transformedData2: + print(each) + + sc.stop() diff --git a/examples/src/main/python/mllib/gaussian_mixture_example.py b/examples/src/main/python/mllib/gaussian_mixture_example.py new file mode 100644 index 000000000000..a60e799d62eb --- /dev/null +++ b/examples/src/main/python/mllib/gaussian_mixture_example.py @@ -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. +# + +from __future__ import print_function + +# $example on$ +from numpy import array +# $example off$ + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.clustering import GaussianMixture, GaussianMixtureModel +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="GaussianMixtureExample") # SparkContext + + # $example on$ + # Load and parse the data + data = sc.textFile("data/mllib/gmm_data.txt") + parsedData = data.map(lambda line: array([float(x) for x in line.strip().split(' ')])) + + # Build the model (cluster the data) + gmm = GaussianMixture.train(parsedData, 2) + + # Save and load model + gmm.save(sc, "target/org/apache/spark/PythonGaussianMixtureExample/GaussianMixtureModel") + sameModel = GaussianMixtureModel\ + .load(sc, "target/org/apache/spark/PythonGaussianMixtureExample/GaussianMixtureModel") + + # output parameters of model + for i in range(2): + print("weight = ", gmm.weights[i], "mu = ", gmm.gaussians[i].mu, + "sigma = ", gmm.gaussians[i].sigma.toArray()) + # $example off$ + + sc.stop() diff --git a/examples/src/main/python/mllib/gaussian_mixture_model.py b/examples/src/main/python/mllib/gaussian_mixture_model.py index 2cb8010cdc07..6b46e27ddaaa 100644 --- a/examples/src/main/python/mllib/gaussian_mixture_model.py +++ b/examples/src/main/python/mllib/gaussian_mixture_model.py @@ -20,6 +20,10 @@ """ from __future__ import print_function +import sys +if sys.version >= '3': + long = int + import random import argparse import numpy as np @@ -62,5 +66,9 @@ def parseVector(line): for i in range(args.k): print(("weight = ", model.weights[i], "mu = ", model.gaussians[i].mu, "sigma = ", model.gaussians[i].sigma.toArray())) + print("\n") + print(("The membership value of each vector to all mixture components (first 100): ", + model.predictSoft(data).take(100))) + print("\n") print(("Cluster labels (first 100): ", model.predict(data).take(100))) sc.stop() diff --git a/examples/src/main/python/mllib/gradient_boosted_trees.py b/examples/src/main/python/mllib/gradient_boosted_trees.py deleted file mode 100644 index 781bd61c9d2b..000000000000 --- a/examples/src/main/python/mllib/gradient_boosted_trees.py +++ /dev/null @@ -1,77 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -""" -Gradient boosted Trees classification and regression using MLlib. -""" -from __future__ import print_function - -import sys - -from pyspark.context import SparkContext -from pyspark.mllib.tree import GradientBoostedTrees -from pyspark.mllib.util import MLUtils - - -def testClassification(trainingData, testData): - # Train a GradientBoostedTrees model. - # Empty categoricalFeaturesInfo indicates all features are continuous. - model = GradientBoostedTrees.trainClassifier(trainingData, categoricalFeaturesInfo={}, - numIterations=30, maxDepth=4) - # Evaluate model on test instances and compute test error - predictions = model.predict(testData.map(lambda x: x.features)) - labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testErr = labelsAndPredictions.filter(lambda v_p: v_p[0] != v_p[1]).count() \ - / float(testData.count()) - print('Test Error = ' + str(testErr)) - print('Learned classification ensemble model:') - print(model.toDebugString()) - - -def testRegression(trainingData, testData): - # Train a GradientBoostedTrees model. - # Empty categoricalFeaturesInfo indicates all features are continuous. - model = GradientBoostedTrees.trainRegressor(trainingData, categoricalFeaturesInfo={}, - numIterations=30, maxDepth=4) - # Evaluate model on test instances and compute test error - predictions = model.predict(testData.map(lambda x: x.features)) - labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testMSE = labelsAndPredictions.map(lambda vp: (vp[0] - vp[1]) * (vp[0] - vp[1])).sum() \ - / float(testData.count()) - print('Test Mean Squared Error = ' + str(testMSE)) - print('Learned regression ensemble model:') - print(model.toDebugString()) - - -if __name__ == "__main__": - if len(sys.argv) > 1: - print("Usage: gradient_boosted_trees", file=sys.stderr) - exit(1) - sc = SparkContext(appName="PythonGradientBoostedTrees") - - # Load and parse the data file into an RDD of LabeledPoint. - data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') - # Split the data into training and test sets (30% held out for testing) - (trainingData, testData) = data.randomSplit([0.7, 0.3]) - - print('\nRunning example of classification using GradientBoostedTrees\n') - testClassification(trainingData, testData) - - print('\nRunning example of regression using GradientBoostedTrees\n') - testRegression(trainingData, testData) - - sc.stop() diff --git a/examples/src/main/python/mllib/gradient_boosting_classification_example.py b/examples/src/main/python/mllib/gradient_boosting_classification_example.py new file mode 100644 index 000000000000..65a03572be9b --- /dev/null +++ b/examples/src/main/python/mllib/gradient_boosting_classification_example.py @@ -0,0 +1,56 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Gradient Boosted Trees Classification Example. +""" +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.tree import GradientBoostedTrees, GradientBoostedTreesModel +from pyspark.mllib.util import MLUtils +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="PythonGradientBoostedTreesClassificationExample") + # $example on$ + # Load and parse the data file. + data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + # Train a GradientBoostedTrees model. + # Notes: (a) Empty categoricalFeaturesInfo indicates all features are continuous. + # (b) Use more iterations in practice. + model = GradientBoostedTrees.trainClassifier(trainingData, + categoricalFeaturesInfo={}, numIterations=3) + + # Evaluate model on test instances and compute test error + predictions = model.predict(testData.map(lambda x: x.features)) + labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) + testErr = labelsAndPredictions.filter( + lambda lp: lp[0] != lp[1]).count() / float(testData.count()) + print('Test Error = ' + str(testErr)) + print('Learned classification GBT model:') + print(model.toDebugString()) + + # Save and load model + model.save(sc, "target/tmp/myGradientBoostingClassificationModel") + sameModel = GradientBoostedTreesModel.load(sc, + "target/tmp/myGradientBoostingClassificationModel") + # $example off$ diff --git a/examples/src/main/python/mllib/gradient_boosting_regression_example.py b/examples/src/main/python/mllib/gradient_boosting_regression_example.py new file mode 100644 index 000000000000..877f8ab461cc --- /dev/null +++ b/examples/src/main/python/mllib/gradient_boosting_regression_example.py @@ -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. +# + +""" +Gradient Boosted Trees Regression Example. +""" +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.tree import GradientBoostedTrees, GradientBoostedTreesModel +from pyspark.mllib.util import MLUtils +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="PythonGradientBoostedTreesRegressionExample") + # $example on$ + # Load and parse the data file. + data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + # Train a GradientBoostedTrees model. + # Notes: (a) Empty categoricalFeaturesInfo indicates all features are continuous. + # (b) Use more iterations in practice. + model = GradientBoostedTrees.trainRegressor(trainingData, + categoricalFeaturesInfo={}, numIterations=3) + + # Evaluate model on test instances and compute test error + predictions = model.predict(testData.map(lambda x: x.features)) + labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) + testMSE = labelsAndPredictions.map(lambda lp: (lp[0] - lp[1]) * (lp[0] - lp[1])).sum() /\ + float(testData.count()) + print('Test Mean Squared Error = ' + str(testMSE)) + print('Learned regression GBT model:') + print(model.toDebugString()) + + # Save and load model + model.save(sc, "target/tmp/myGradientBoostingRegressionModel") + sameModel = GradientBoostedTreesModel.load(sc, "target/tmp/myGradientBoostingRegressionModel") + # $example off$ diff --git a/examples/src/main/python/mllib/hypothesis_testing_example.py b/examples/src/main/python/mllib/hypothesis_testing_example.py new file mode 100644 index 000000000000..e566ead0d318 --- /dev/null +++ b/examples/src/main/python/mllib/hypothesis_testing_example.py @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.linalg import Matrices, Vectors +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.stat import Statistics +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="HypothesisTestingExample") + + # $example on$ + vec = Vectors.dense(0.1, 0.15, 0.2, 0.3, 0.25) # a vector composed of the frequencies of events + + # compute the goodness of fit. If a second vector to test against + # is not supplied as a parameter, the test runs against a uniform distribution. + goodnessOfFitTestResult = Statistics.chiSqTest(vec) + + # summary of the test including the p-value, degrees of freedom, + # test statistic, the method used, and the null hypothesis. + print("%s\n" % goodnessOfFitTestResult) + + mat = Matrices.dense(3, 2, [1.0, 3.0, 5.0, 2.0, 4.0, 6.0]) # a contingency matrix + + # conduct Pearson's independence test on the input contingency matrix + independenceTestResult = Statistics.chiSqTest(mat) + + # summary of the test including the p-value, degrees of freedom, + # test statistic, the method used, and the null hypothesis. + print("%s\n" % independenceTestResult) + + obs = sc.parallelize( + [LabeledPoint(1.0, [1.0, 0.0, 3.0]), + LabeledPoint(1.0, [1.0, 2.0, 0.0]), + LabeledPoint(1.0, [-1.0, 0.0, -0.5])] + ) # LabeledPoint(feature, label) + + # The contingency table is constructed from an RDD of LabeledPoint and used to conduct + # the independence test. Returns an array containing the ChiSquaredTestResult for every feature + # against the label. + featureTestResults = Statistics.chiSqTest(obs) + + for i, result in enumerate(featureTestResults): + print("Column %d:\n%s" % (i + 1, result)) + # $example off$ + + sc.stop() diff --git a/examples/src/main/python/mllib/hypothesis_testing_kolmogorov_smirnov_test_example.py b/examples/src/main/python/mllib/hypothesis_testing_kolmogorov_smirnov_test_example.py new file mode 100644 index 000000000000..ef380dee79d3 --- /dev/null +++ b/examples/src/main/python/mllib/hypothesis_testing_kolmogorov_smirnov_test_example.py @@ -0,0 +1,40 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.stat import Statistics +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="HypothesisTestingKolmogorovSmirnovTestExample") + + # $example on$ + parallelData = sc.parallelize([0.1, 0.15, 0.2, 0.3, 0.25]) + + # run a KS test for the sample versus a standard normal distribution + testResult = Statistics.kolmogorovSmirnovTest(parallelData, "norm", 0, 1) + # summary of the test including the p-value, test statistic, and null hypothesis + # if our p-value indicates significance, we can reject the null hypothesis + # Note that the Scala functionality of calling Statistics.kolmogorovSmirnovTest with + # a lambda to calculate the CDF is not made available in the Python API + print(testResult) + # $example off$ + + sc.stop() diff --git a/examples/src/main/python/mllib/isotonic_regression_example.py b/examples/src/main/python/mllib/isotonic_regression_example.py index 89dc9f4b6611..33d618ab48ea 100644 --- a/examples/src/main/python/mllib/isotonic_regression_example.py +++ b/examples/src/main/python/mllib/isotonic_regression_example.py @@ -23,7 +23,8 @@ from pyspark import SparkContext # $example on$ import math -from pyspark.mllib.regression import IsotonicRegression, IsotonicRegressionModel +from pyspark.mllib.regression import LabeledPoint, IsotonicRegression, IsotonicRegressionModel +from pyspark.mllib.util import MLUtils # $example off$ if __name__ == "__main__": @@ -31,10 +32,14 @@ sc = SparkContext(appName="PythonIsotonicRegressionExample") # $example on$ - data = sc.textFile("data/mllib/sample_isotonic_regression_data.txt") + # Load and parse the data + def parsePoint(labeledData): + return (labeledData.label, labeledData.features[0], 1.0) + + data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_isotonic_regression_libsvm_data.txt") # Create label, feature, weight tuples from input data with weight set to default value 1.0. - parsedData = data.map(lambda line: tuple([float(x) for x in line.split(',')]) + (1.0,)) + parsedData = data.map(parsePoint) # Split data into training (60%) and test (40%) sets. training, test = parsedData.randomSplit([0.6, 0.4], 11) diff --git a/examples/src/main/python/mllib/k_means_example.py b/examples/src/main/python/mllib/k_means_example.py new file mode 100644 index 000000000000..d6058f45020c --- /dev/null +++ b/examples/src/main/python/mllib/k_means_example.py @@ -0,0 +1,54 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from numpy import array +from math import sqrt +# $example off$ + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.clustering import KMeans, KMeansModel +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="KMeansExample") # SparkContext + + # $example on$ + # Load and parse the data + data = sc.textFile("data/mllib/kmeans_data.txt") + parsedData = data.map(lambda line: array([float(x) for x in line.split(' ')])) + + # Build the model (cluster the data) + clusters = KMeans.train(parsedData, 2, maxIterations=10, initializationMode="random") + + # Evaluate clustering by computing Within Set Sum of Squared Errors + def error(point): + center = clusters.centers[clusters.predict(point)] + return sqrt(sum([x**2 for x in (point - center)])) + + WSSSE = parsedData.map(lambda point: error(point)).reduce(lambda x, y: x + y) + print("Within Set Sum of Squared Error = " + str(WSSSE)) + + # Save and load model + clusters.save(sc, "target/org/apache/spark/PythonKMeansExample/KMeansModel") + sameModel = KMeansModel.load(sc, "target/org/apache/spark/PythonKMeansExample/KMeansModel") + # $example off$ + + sc.stop() diff --git a/examples/src/main/python/mllib/kernel_density_estimation_example.py b/examples/src/main/python/mllib/kernel_density_estimation_example.py new file mode 100644 index 000000000000..3e8f7241a4a1 --- /dev/null +++ b/examples/src/main/python/mllib/kernel_density_estimation_example.py @@ -0,0 +1,44 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.stat import KernelDensity +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="KernelDensityEstimationExample") # SparkContext + + # $example on$ + # an RDD of sample data + data = sc.parallelize([1.0, 1.0, 1.0, 2.0, 3.0, 4.0, 5.0, 5.0, 6.0, 7.0, 8.0, 9.0, 9.0]) + + # Construct the density estimator with the sample data and a standard deviation for the Gaussian + # kernels + kd = KernelDensity() + kd.setSample(data) + kd.setBandwidth(3.0) + + # Find density estimates for the given values + densities = kd.estimate([-1.0, 2.0, 5.0]) + # $example off$ + + print(densities) + + sc.stop() diff --git a/examples/src/main/python/mllib/latent_dirichlet_allocation_example.py b/examples/src/main/python/mllib/latent_dirichlet_allocation_example.py new file mode 100644 index 000000000000..2a1bef5f207b --- /dev/null +++ b/examples/src/main/python/mllib/latent_dirichlet_allocation_example.py @@ -0,0 +1,54 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.clustering import LDA, LDAModel +from pyspark.mllib.linalg import Vectors +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="LatentDirichletAllocationExample") # SparkContext + + # $example on$ + # Load and parse the data + data = sc.textFile("data/mllib/sample_lda_data.txt") + parsedData = data.map(lambda line: Vectors.dense([float(x) for x in line.strip().split(' ')])) + # Index documents with unique IDs + corpus = parsedData.zipWithIndex().map(lambda x: [x[1], x[0]]).cache() + + # Cluster the documents into three topics using LDA + ldaModel = LDA.train(corpus, k=3) + + # Output topics. Each is a distribution over words (matching word count vectors) + print("Learned topics (as distributions over vocab of " + str(ldaModel.vocabSize()) + + " words):") + topics = ldaModel.topicsMatrix() + for topic in range(3): + print("Topic " + str(topic) + ":") + for word in range(0, ldaModel.vocabSize()): + print(" " + str(topics[word][topic])) + + # Save and load model + ldaModel.save(sc, "target/org/apache/spark/PythonLatentDirichletAllocationExample/LDAModel") + sameModel = LDAModel\ + .load(sc, "target/org/apache/spark/PythonLatentDirichletAllocationExample/LDAModel") + # $example off$ + + sc.stop() diff --git a/examples/src/main/python/mllib/linear_regression_with_sgd_example.py b/examples/src/main/python/mllib/linear_regression_with_sgd_example.py new file mode 100644 index 000000000000..6744463d40ef --- /dev/null +++ b/examples/src/main/python/mllib/linear_regression_with_sgd_example.py @@ -0,0 +1,54 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Linear Regression With SGD Example. +""" +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.regression import LabeledPoint, LinearRegressionWithSGD, LinearRegressionModel +# $example off$ + +if __name__ == "__main__": + + sc = SparkContext(appName="PythonLinearRegressionWithSGDExample") + + # $example on$ + # Load and parse the data + def parsePoint(line): + values = [float(x) for x in line.replace(',', ' ').split(' ')] + return LabeledPoint(values[0], values[1:]) + + data = sc.textFile("data/mllib/ridge-data/lpsa.data") + parsedData = data.map(parsePoint) + + # Build the model + model = LinearRegressionWithSGD.train(parsedData, iterations=100, step=0.00000001) + + # Evaluate the model on training data + valuesAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) + MSE = valuesAndPreds \ + .map(lambda vp: (vp[0] - vp[1])**2) \ + .reduce(lambda x, y: x + y) / valuesAndPreds.count() + print("Mean Squared Error = " + str(MSE)) + + # Save and load model + model.save(sc, "target/tmp/pythonLinearRegressionWithSGDModel") + sameModel = LinearRegressionModel.load(sc, "target/tmp/pythonLinearRegressionWithSGDModel") + # $example off$ diff --git a/examples/src/main/python/mllib/logistic_regression_with_lbfgs_example.py b/examples/src/main/python/mllib/logistic_regression_with_lbfgs_example.py new file mode 100644 index 000000000000..c9b768b3147d --- /dev/null +++ b/examples/src/main/python/mllib/logistic_regression_with_lbfgs_example.py @@ -0,0 +1,54 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Logistic Regression With LBFGS Example. +""" +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.classification import LogisticRegressionWithLBFGS, LogisticRegressionModel +from pyspark.mllib.regression import LabeledPoint +# $example off$ + +if __name__ == "__main__": + + sc = SparkContext(appName="PythonLogisticRegressionWithLBFGSExample") + + # $example on$ + # Load and parse the data + def parsePoint(line): + values = [float(x) for x in line.split(' ')] + return LabeledPoint(values[0], values[1:]) + + data = sc.textFile("data/mllib/sample_svm_data.txt") + parsedData = data.map(parsePoint) + + # Build the model + model = LogisticRegressionWithLBFGS.train(parsedData) + + # Evaluating the model on training data + labelsAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) + trainErr = labelsAndPreds.filter(lambda lp: lp[0] != lp[1]).count() / float(parsedData.count()) + print("Training Error = " + str(trainErr)) + + # Save and load model + model.save(sc, "target/tmp/pythonLogisticRegressionWithLBFGSModel") + sameModel = LogisticRegressionModel.load(sc, + "target/tmp/pythonLogisticRegressionWithLBFGSModel") + # $example off$ diff --git a/examples/src/main/python/mllib/multi_class_metrics_example.py b/examples/src/main/python/mllib/multi_class_metrics_example.py new file mode 100644 index 000000000000..7dc5fb4f9127 --- /dev/null +++ b/examples/src/main/python/mllib/multi_class_metrics_example.py @@ -0,0 +1,69 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# $example on$ +from pyspark.mllib.classification import LogisticRegressionWithLBFGS +from pyspark.mllib.util import MLUtils +from pyspark.mllib.evaluation import MulticlassMetrics +# $example off$ + +from pyspark import SparkContext + +if __name__ == "__main__": + sc = SparkContext(appName="MultiClassMetricsExample") + + # Several of the methods available in scala are currently missing from pyspark + # $example on$ + # Load training data in LIBSVM format + data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_multiclass_classification_data.txt") + + # Split data into training (60%) and test (40%) + training, test = data.randomSplit([0.6, 0.4], seed=11) + training.cache() + + # Run training algorithm to build the model + model = LogisticRegressionWithLBFGS.train(training, numClasses=3) + + # Compute raw scores on the test set + predictionAndLabels = test.map(lambda lp: (float(model.predict(lp.features)), lp.label)) + + # Instantiate metrics object + metrics = MulticlassMetrics(predictionAndLabels) + + # Overall statistics + precision = metrics.precision() + recall = metrics.recall() + f1Score = metrics.fMeasure() + print("Summary Stats") + print("Precision = %s" % precision) + print("Recall = %s" % recall) + print("F1 Score = %s" % f1Score) + + # Statistics by class + labels = data.map(lambda lp: lp.label).distinct().collect() + for label in sorted(labels): + print("Class %s precision = %s" % (label, metrics.precision(label))) + print("Class %s recall = %s" % (label, metrics.recall(label))) + print("Class %s F1 Measure = %s" % (label, metrics.fMeasure(label, beta=1.0))) + + # Weighted stats + print("Weighted recall = %s" % metrics.weightedRecall) + print("Weighted precision = %s" % metrics.weightedPrecision) + print("Weighted F(1) Score = %s" % metrics.weightedFMeasure()) + print("Weighted F(0.5) Score = %s" % metrics.weightedFMeasure(beta=0.5)) + print("Weighted false positive rate = %s" % metrics.weightedFalsePositiveRate) + # $example off$ diff --git a/examples/src/main/python/mllib/multi_label_metrics_example.py b/examples/src/main/python/mllib/multi_label_metrics_example.py new file mode 100644 index 000000000000..960ade659737 --- /dev/null +++ b/examples/src/main/python/mllib/multi_label_metrics_example.py @@ -0,0 +1,61 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# $example on$ +from pyspark.mllib.evaluation import MultilabelMetrics +# $example off$ +from pyspark import SparkContext + +if __name__ == "__main__": + sc = SparkContext(appName="MultiLabelMetricsExample") + # $example on$ + scoreAndLabels = sc.parallelize([ + ([0.0, 1.0], [0.0, 2.0]), + ([0.0, 2.0], [0.0, 1.0]), + ([], [0.0]), + ([2.0], [2.0]), + ([2.0, 0.0], [2.0, 0.0]), + ([0.0, 1.0, 2.0], [0.0, 1.0]), + ([1.0], [1.0, 2.0])]) + + # Instantiate metrics object + metrics = MultilabelMetrics(scoreAndLabels) + + # Summary stats + print("Recall = %s" % metrics.recall()) + print("Precision = %s" % metrics.precision()) + print("F1 measure = %s" % metrics.f1Measure()) + print("Accuracy = %s" % metrics.accuracy) + + # Individual label stats + labels = scoreAndLabels.flatMap(lambda x: x[1]).distinct().collect() + for label in labels: + print("Class %s precision = %s" % (label, metrics.precision(label))) + print("Class %s recall = %s" % (label, metrics.recall(label))) + print("Class %s F1 Measure = %s" % (label, metrics.f1Measure(label))) + + # Micro stats + print("Micro precision = %s" % metrics.microPrecision) + print("Micro recall = %s" % metrics.microRecall) + print("Micro F1 measure = %s" % metrics.microF1Measure) + + # Hamming loss + print("Hamming loss = %s" % metrics.hammingLoss) + + # Subset accuracy + print("Subset accuracy = %s" % metrics.subsetAccuracy) + # $example off$ diff --git a/examples/src/main/python/mllib/naive_bayes_example.py b/examples/src/main/python/mllib/naive_bayes_example.py index a2e7dacf2549..a29fcccac5bf 100644 --- a/examples/src/main/python/mllib/naive_bayes_example.py +++ b/examples/src/main/python/mllib/naive_bayes_example.py @@ -17,20 +17,21 @@ """ NaiveBayes Example. + +Usage: + `spark-submit --master local[4] examples/src/main/python/mllib/naive_bayes_example.py` """ + from __future__ import print_function +import shutil + +from pyspark import SparkContext # $example on$ from pyspark.mllib.classification import NaiveBayes, NaiveBayesModel -from pyspark.mllib.linalg import Vectors -from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.util import MLUtils -def parseLine(line): - parts = line.split(',') - label = float(parts[0]) - features = Vectors.dense([float(x) for x in parts[1].split(' ')]) - return LabeledPoint(label, features) # $example off$ if __name__ == "__main__": @@ -38,19 +39,27 @@ def parseLine(line): sc = SparkContext(appName="PythonNaiveBayesExample") # $example on$ - data = sc.textFile('data/mllib/sample_naive_bayes_data.txt').map(parseLine) + # Load and parse the data file. + data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") - # Split data aproximately into training (60%) and test (40%) - training, test = data.randomSplit([0.6, 0.4], seed=0) + # Split data approximately into training (60%) and test (40%) + training, test = data.randomSplit([0.6, 0.4]) # Train a naive Bayes model. model = NaiveBayes.train(training, 1.0) # Make prediction and test accuracy. predictionAndLabel = test.map(lambda p: (model.predict(p.features), p.label)) - accuracy = 1.0 * predictionAndLabel.filter(lambda (x, v): x == v).count() / test.count() + accuracy = 1.0 * predictionAndLabel.filter(lambda pl: pl[0] == pl[1]).count() / test.count() + print('model accuracy {}'.format(accuracy)) # Save and load model - model.save(sc, "target/tmp/myNaiveBayesModel") - sameModel = NaiveBayesModel.load(sc, "target/tmp/myNaiveBayesModel") + output_dir = 'target/tmp/myNaiveBayesModel' + shutil.rmtree(output_dir, ignore_errors=True) + model.save(sc, output_dir) + sameModel = NaiveBayesModel.load(sc, output_dir) + predictionAndLabel = test.map(lambda p: (sameModel.predict(p.features), p.label)) + accuracy = 1.0 * predictionAndLabel.filter(lambda pl: pl[0] == pl[1]).count() / test.count() + print('sameModel accuracy {}'.format(accuracy)) + # $example off$ diff --git a/examples/src/main/python/mllib/normalizer_example.py b/examples/src/main/python/mllib/normalizer_example.py new file mode 100644 index 000000000000..a4e028ca9af8 --- /dev/null +++ b/examples/src/main/python/mllib/normalizer_example.py @@ -0,0 +1,52 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.feature import Normalizer +from pyspark.mllib.util import MLUtils +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="NormalizerExample") # SparkContext + + # $example on$ + data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + labels = data.map(lambda x: x.label) + features = data.map(lambda x: x.features) + + normalizer1 = Normalizer() + normalizer2 = Normalizer(p=float("inf")) + + # Each sample in data1 will be normalized using $L^2$ norm. + data1 = labels.zip(normalizer1.transform(features)) + + # Each sample in data2 will be normalized using $L^\infty$ norm. + data2 = labels.zip(normalizer2.transform(features)) + # $example off$ + + print("data1:") + for each in data1.collect(): + print(each) + + print("data2:") + for each in data2.collect(): + print(each) + + sc.stop() diff --git a/examples/src/main/python/mllib/pca_rowmatrix_example.py b/examples/src/main/python/mllib/pca_rowmatrix_example.py new file mode 100644 index 000000000000..49b9b1bbe08e --- /dev/null +++ b/examples/src/main/python/mllib/pca_rowmatrix_example.py @@ -0,0 +1,46 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.linalg import Vectors +from pyspark.mllib.linalg.distributed import RowMatrix +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="PythonPCAOnRowMatrixExample") + + # $example on$ + rows = sc.parallelize([ + Vectors.sparse(5, {1: 1.0, 3: 7.0}), + Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0), + Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0) + ]) + + mat = RowMatrix(rows) + # Compute the top 4 principal components. + # Principal components are stored in a local dense matrix. + pc = mat.computePrincipalComponents(4) + + # Project the rows to the linear space spanned by the top 4 principal components. + projected = mat.multiply(pc) + # $example off$ + collected = projected.rows.collect() + print("Projected Row Matrix of principal component:") + for vector in collected: + print(vector) + sc.stop() diff --git a/examples/src/main/python/mllib/power_iteration_clustering_example.py b/examples/src/main/python/mllib/power_iteration_clustering_example.py new file mode 100644 index 000000000000..ca19c0ccb60c --- /dev/null +++ b/examples/src/main/python/mllib/power_iteration_clustering_example.py @@ -0,0 +1,44 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.clustering import PowerIterationClustering, PowerIterationClusteringModel +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="PowerIterationClusteringExample") # SparkContext + + # $example on$ + # Load and parse the data + data = sc.textFile("data/mllib/pic_data.txt") + similarities = data.map(lambda line: tuple([float(x) for x in line.split(' ')])) + + # Cluster the data into two classes using PowerIterationClustering + model = PowerIterationClustering.train(similarities, 2, 10) + + model.assignments().foreach(lambda x: print(str(x.id) + " -> " + str(x.cluster))) + + # Save and load model + model.save(sc, "target/org/apache/spark/PythonPowerIterationClusteringExample/PICModel") + sameModel = PowerIterationClusteringModel\ + .load(sc, "target/org/apache/spark/PythonPowerIterationClusteringExample/PICModel") + # $example off$ + + sc.stop() diff --git a/examples/src/main/python/mllib/random_forest_classification_example.py b/examples/src/main/python/mllib/random_forest_classification_example.py new file mode 100644 index 000000000000..5ac67520daee --- /dev/null +++ b/examples/src/main/python/mllib/random_forest_classification_example.py @@ -0,0 +1,57 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Random Forest Classification Example. +""" +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.tree import RandomForest, RandomForestModel +from pyspark.mllib.util import MLUtils +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="PythonRandomForestClassificationExample") + # $example on$ + # Load and parse the data file into an RDD of LabeledPoint. + data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + # Train a RandomForest model. + # Empty categoricalFeaturesInfo indicates all features are continuous. + # Note: Use larger numTrees in practice. + # Setting featureSubsetStrategy="auto" lets the algorithm choose. + model = RandomForest.trainClassifier(trainingData, numClasses=2, categoricalFeaturesInfo={}, + numTrees=3, featureSubsetStrategy="auto", + impurity='gini', maxDepth=4, maxBins=32) + + # Evaluate model on test instances and compute test error + predictions = model.predict(testData.map(lambda x: x.features)) + labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) + testErr = labelsAndPredictions.filter( + lambda lp: lp[0] != lp[1]).count() / float(testData.count()) + print('Test Error = ' + str(testErr)) + print('Learned classification forest model:') + print(model.toDebugString()) + + # Save and load model + model.save(sc, "target/tmp/myRandomForestClassificationModel") + sameModel = RandomForestModel.load(sc, "target/tmp/myRandomForestClassificationModel") + # $example off$ diff --git a/examples/src/main/python/mllib/random_forest_example.py b/examples/src/main/python/mllib/random_forest_example.py deleted file mode 100755 index 4cfdad868c66..000000000000 --- a/examples/src/main/python/mllib/random_forest_example.py +++ /dev/null @@ -1,90 +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. -# - -""" -Random Forest classification and regression using MLlib. - -Note: This example illustrates binary classification. - For information on multiclass classification, please refer to the decision_tree_runner.py - example. -""" -from __future__ import print_function - -import sys - -from pyspark.context import SparkContext -from pyspark.mllib.tree import RandomForest -from pyspark.mllib.util import MLUtils - - -def testClassification(trainingData, testData): - # Train a RandomForest model. - # Empty categoricalFeaturesInfo indicates all features are continuous. - # Note: Use larger numTrees in practice. - # Setting featureSubsetStrategy="auto" lets the algorithm choose. - model = RandomForest.trainClassifier(trainingData, numClasses=2, - categoricalFeaturesInfo={}, - numTrees=3, featureSubsetStrategy="auto", - impurity='gini', maxDepth=4, maxBins=32) - - # Evaluate model on test instances and compute test error - predictions = model.predict(testData.map(lambda x: x.features)) - labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testErr = labelsAndPredictions.filter(lambda v_p: v_p[0] != v_p[1]).count()\ - / float(testData.count()) - print('Test Error = ' + str(testErr)) - print('Learned classification forest model:') - print(model.toDebugString()) - - -def testRegression(trainingData, testData): - # Train a RandomForest model. - # Empty categoricalFeaturesInfo indicates all features are continuous. - # Note: Use larger numTrees in practice. - # Setting featureSubsetStrategy="auto" lets the algorithm choose. - model = RandomForest.trainRegressor(trainingData, categoricalFeaturesInfo={}, - numTrees=3, featureSubsetStrategy="auto", - impurity='variance', maxDepth=4, maxBins=32) - - # Evaluate model on test instances and compute test error - predictions = model.predict(testData.map(lambda x: x.features)) - labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) - testMSE = labelsAndPredictions.map(lambda v_p1: (v_p1[0] - v_p1[1]) * (v_p1[0] - v_p1[1]))\ - .sum() / float(testData.count()) - print('Test Mean Squared Error = ' + str(testMSE)) - print('Learned regression forest model:') - print(model.toDebugString()) - - -if __name__ == "__main__": - if len(sys.argv) > 1: - print("Usage: random_forest_example", file=sys.stderr) - exit(1) - sc = SparkContext(appName="PythonRandomForestExample") - - # Load and parse the data file into an RDD of LabeledPoint. - data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') - # Split the data into training and test sets (30% held out for testing) - (trainingData, testData) = data.randomSplit([0.7, 0.3]) - - print('\nRunning example of classification using RandomForest\n') - testClassification(trainingData, testData) - - print('\nRunning example of regression using RandomForest\n') - testRegression(trainingData, testData) - - sc.stop() diff --git a/examples/src/main/python/mllib/random_forest_regression_example.py b/examples/src/main/python/mllib/random_forest_regression_example.py new file mode 100644 index 000000000000..7e986a0d307f --- /dev/null +++ b/examples/src/main/python/mllib/random_forest_regression_example.py @@ -0,0 +1,57 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Random Forest Regression Example. +""" +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.tree import RandomForest, RandomForestModel +from pyspark.mllib.util import MLUtils +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="PythonRandomForestRegressionExample") + # $example on$ + # Load and parse the data file into an RDD of LabeledPoint. + data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + # Train a RandomForest model. + # Empty categoricalFeaturesInfo indicates all features are continuous. + # Note: Use larger numTrees in practice. + # Setting featureSubsetStrategy="auto" lets the algorithm choose. + model = RandomForest.trainRegressor(trainingData, categoricalFeaturesInfo={}, + numTrees=3, featureSubsetStrategy="auto", + impurity='variance', maxDepth=4, maxBins=32) + + # Evaluate model on test instances and compute test error + predictions = model.predict(testData.map(lambda x: x.features)) + labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) + testMSE = labelsAndPredictions.map(lambda lp: (lp[0] - lp[1]) * (lp[0] - lp[1])).sum() /\ + float(testData.count()) + print('Test Mean Squared Error = ' + str(testMSE)) + print('Learned regression forest model:') + print(model.toDebugString()) + + # Save and load model + model.save(sc, "target/tmp/myRandomForestRegressionModel") + sameModel = RandomForestModel.load(sc, "target/tmp/myRandomForestRegressionModel") + # $example off$ diff --git a/examples/src/main/python/mllib/ranking_metrics_example.py b/examples/src/main/python/mllib/ranking_metrics_example.py new file mode 100644 index 000000000000..21333deded35 --- /dev/null +++ b/examples/src/main/python/mllib/ranking_metrics_example.py @@ -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. +# + +# $example on$ +from pyspark.mllib.recommendation import ALS, Rating +from pyspark.mllib.evaluation import RegressionMetrics, RankingMetrics +# $example off$ +from pyspark import SparkContext + +if __name__ == "__main__": + sc = SparkContext(appName="Ranking Metrics Example") + + # Several of the methods available in scala are currently missing from pyspark + # $example on$ + # Read in the ratings data + lines = sc.textFile("data/mllib/sample_movielens_data.txt") + + def parseLine(line): + fields = line.split("::") + return Rating(int(fields[0]), int(fields[1]), float(fields[2]) - 2.5) + ratings = lines.map(lambda r: parseLine(r)) + + # Train a model on to predict user-product ratings + model = ALS.train(ratings, 10, 10, 0.01) + + # Get predicted ratings on all existing user-product pairs + testData = ratings.map(lambda p: (p.user, p.product)) + predictions = model.predictAll(testData).map(lambda r: ((r.user, r.product), r.rating)) + + ratingsTuple = ratings.map(lambda r: ((r.user, r.product), r.rating)) + scoreAndLabels = predictions.join(ratingsTuple).map(lambda tup: tup[1]) + + # Instantiate regression metrics to compare predicted and actual ratings + metrics = RegressionMetrics(scoreAndLabels) + + # Root mean squared error + print("RMSE = %s" % metrics.rootMeanSquaredError) + + # R-squared + print("R-squared = %s" % metrics.r2) + # $example off$ diff --git a/examples/src/main/python/mllib/recommendation_example.py b/examples/src/main/python/mllib/recommendation_example.py new file mode 100644 index 000000000000..00e683c3ae93 --- /dev/null +++ b/examples/src/main/python/mllib/recommendation_example.py @@ -0,0 +1,52 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Collaborative Filtering Classification Example. +""" +from __future__ import print_function + +from pyspark import SparkContext + +# $example on$ +from pyspark.mllib.recommendation import ALS, MatrixFactorizationModel, Rating +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="PythonCollaborativeFilteringExample") + # $example on$ + # Load and parse the data + data = sc.textFile("data/mllib/als/test.data") + ratings = data.map(lambda l: l.split(','))\ + .map(lambda l: Rating(int(l[0]), int(l[1]), float(l[2]))) + + # Build the recommendation model using Alternating Least Squares + rank = 10 + numIterations = 10 + model = ALS.train(ratings, rank, numIterations) + + # Evaluate the model on training data + testdata = ratings.map(lambda p: (p[0], p[1])) + predictions = model.predictAll(testdata).map(lambda r: ((r[0], r[1]), r[2])) + ratesAndPreds = ratings.map(lambda r: ((r[0], r[1]), r[2])).join(predictions) + MSE = ratesAndPreds.map(lambda r: (r[1][0] - r[1][1])**2).mean() + print("Mean Squared Error = " + str(MSE)) + + # Save and load model + model.save(sc, "target/tmp/myCollaborativeFilter") + sameModel = MatrixFactorizationModel.load(sc, "target/tmp/myCollaborativeFilter") + # $example off$ diff --git a/examples/src/main/python/mllib/regression_metrics_example.py b/examples/src/main/python/mllib/regression_metrics_example.py new file mode 100644 index 000000000000..a3a83aafd7a1 --- /dev/null +++ b/examples/src/main/python/mllib/regression_metrics_example.py @@ -0,0 +1,59 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# $example on$ +from pyspark.mllib.regression import LabeledPoint, LinearRegressionWithSGD +from pyspark.mllib.evaluation import RegressionMetrics +from pyspark.mllib.linalg import DenseVector +# $example off$ + +from pyspark import SparkContext + +if __name__ == "__main__": + sc = SparkContext(appName="Regression Metrics Example") + + # $example on$ + # Load and parse the data + def parsePoint(line): + values = line.split() + return LabeledPoint(float(values[0]), + DenseVector([float(x.split(':')[1]) for x in values[1:]])) + + data = sc.textFile("data/mllib/sample_linear_regression_data.txt") + parsedData = data.map(parsePoint) + + # Build the model + model = LinearRegressionWithSGD.train(parsedData) + + # Get predictions + valuesAndPreds = parsedData.map(lambda p: (float(model.predict(p.features)), p.label)) + + # Instantiate metrics object + metrics = RegressionMetrics(valuesAndPreds) + + # Squared Error + print("MSE = %s" % metrics.meanSquaredError) + print("RMSE = %s" % metrics.rootMeanSquaredError) + + # R-squared + print("R-squared = %s" % metrics.r2) + + # Mean absolute error + print("MAE = %s" % metrics.meanAbsoluteError) + + # Explained variance + print("Explained variance = %s" % metrics.explainedVariance) + # $example off$ diff --git a/examples/src/main/python/mllib/standard_scaler_example.py b/examples/src/main/python/mllib/standard_scaler_example.py new file mode 100644 index 000000000000..442094e1bf36 --- /dev/null +++ b/examples/src/main/python/mllib/standard_scaler_example.py @@ -0,0 +1,53 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.feature import StandardScaler, StandardScalerModel +from pyspark.mllib.linalg import Vectors +from pyspark.mllib.util import MLUtils +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="StandardScalerExample") # SparkContext + + # $example on$ + data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + label = data.map(lambda x: x.label) + features = data.map(lambda x: x.features) + + scaler1 = StandardScaler().fit(features) + scaler2 = StandardScaler(withMean=True, withStd=True).fit(features) + + # data1 will be unit variance. + data1 = label.zip(scaler1.transform(features)) + + # data2 will be unit variance and zero mean. + data2 = label.zip(scaler2.transform(features.map(lambda x: Vectors.dense(x.toArray())))) + # $example off$ + + print("data1:") + for each in data1.collect(): + print(each) + + print("data2:") + for each in data2.collect(): + print(each) + + sc.stop() diff --git a/examples/src/main/python/mllib/stratified_sampling_example.py b/examples/src/main/python/mllib/stratified_sampling_example.py new file mode 100644 index 000000000000..a13f8f08dd68 --- /dev/null +++ b/examples/src/main/python/mllib/stratified_sampling_example.py @@ -0,0 +1,38 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 __future__ import print_function + +from pyspark import SparkContext + +if __name__ == "__main__": + sc = SparkContext(appName="StratifiedSamplingExample") # SparkContext + + # $example on$ + # an RDD of any key value pairs + data = sc.parallelize([(1, 'a'), (1, 'b'), (2, 'c'), (2, 'd'), (2, 'e'), (3, 'f')]) + + # specify the exact fraction desired from each key as a dictionary + fractions = {1: 0.1, 2: 0.6, 3: 0.3} + + approxSample = data.sampleByKey(False, fractions) + # $example off$ + + for each in approxSample.collect(): + print(each) + + sc.stop() diff --git a/examples/src/main/python/mllib/streaming_k_means_example.py b/examples/src/main/python/mllib/streaming_k_means_example.py new file mode 100644 index 000000000000..e82509ad3ffb --- /dev/null +++ b/examples/src/main/python/mllib/streaming_k_means_example.py @@ -0,0 +1,66 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext +# $example on$ +from pyspark.mllib.linalg import Vectors +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.clustering import StreamingKMeans +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="StreamingKMeansExample") # SparkContext + ssc = StreamingContext(sc, 1) + + # $example on$ + # we make an input stream of vectors for training, + # as well as a stream of vectors for testing + def parse(lp): + label = float(lp[lp.find('(') + 1: lp.find(')')]) + vec = Vectors.dense(lp[lp.find('[') + 1: lp.find(']')].split(',')) + + return LabeledPoint(label, vec) + + trainingData = sc.textFile("data/mllib/kmeans_data.txt")\ + .map(lambda line: Vectors.dense([float(x) for x in line.strip().split(' ')])) + + testingData = sc.textFile("data/mllib/streaming_kmeans_data_test.txt").map(parse) + + trainingQueue = [trainingData] + testingQueue = [testingData] + + trainingStream = ssc.queueStream(trainingQueue) + testingStream = ssc.queueStream(testingQueue) + + # We create a model with random clusters and specify the number of clusters to find + model = StreamingKMeans(k=2, decayFactor=1.0).setRandomCenters(3, 1.0, 0) + + # Now register the streams for training and testing and start the job, + # printing the predicted cluster assignments on new data points as they arrive. + model.trainOn(trainingStream) + + result = model.predictOnValues(testingStream.map(lambda lp: (lp.label, lp.features))) + result.pprint() + + ssc.start() + ssc.stop(stopSparkContext=True, stopGraceFully=True) + # $example off$ + + print("Final centers: " + str(model.latestModel().centers)) diff --git a/examples/src/main/python/mllib/streaming_linear_regression_example.py b/examples/src/main/python/mllib/streaming_linear_regression_example.py new file mode 100644 index 000000000000..f600496867c1 --- /dev/null +++ b/examples/src/main/python/mllib/streaming_linear_regression_example.py @@ -0,0 +1,62 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Streaming Linear Regression Example. +""" +from __future__ import print_function + +# $example on$ +import sys +# $example off$ + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext +# $example on$ +from pyspark.mllib.linalg import Vectors +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.regression import StreamingLinearRegressionWithSGD +# $example off$ + +if __name__ == "__main__": + if len(sys.argv) != 3: + print("Usage: streaming_linear_regression_example.py ", + file=sys.stderr) + exit(-1) + + sc = SparkContext(appName="PythonLogisticRegressionWithLBFGSExample") + ssc = StreamingContext(sc, 1) + + # $example on$ + def parse(lp): + label = float(lp[lp.find('(') + 1: lp.find(',')]) + vec = Vectors.dense(lp[lp.find('[') + 1: lp.find(']')].split(',')) + return LabeledPoint(label, vec) + + trainingData = ssc.textFileStream(sys.argv[1]).map(parse).cache() + testData = ssc.textFileStream(sys.argv[2]).map(parse) + + numFeatures = 3 + model = StreamingLinearRegressionWithSGD() + model.setInitialWeights([0.0, 0.0, 0.0]) + + model.trainOn(trainingData) + print(model.predictOnValues(testData.map(lambda lp: (lp.label, lp.features)))) + + ssc.start() + ssc.awaitTermination() + # $example off$ diff --git a/examples/src/main/python/mllib/summary_statistics_example.py b/examples/src/main/python/mllib/summary_statistics_example.py new file mode 100644 index 000000000000..d55d1a2c2d0e --- /dev/null +++ b/examples/src/main/python/mllib/summary_statistics_example.py @@ -0,0 +1,42 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +import numpy as np + +from pyspark.mllib.stat import Statistics +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="SummaryStatisticsExample") # SparkContext + + # $example on$ + mat = sc.parallelize( + [np.array([1.0, 10.0, 100.0]), np.array([2.0, 20.0, 200.0]), np.array([3.0, 30.0, 300.0])] + ) # an RDD of Vectors + + # Compute column summary statistics. + summary = Statistics.colStats(mat) + print(summary.mean()) # a dense vector containing the mean value for each column + print(summary.variance()) # column-wise variance + print(summary.numNonzeros()) # number of nonzeros in each column + # $example off$ + + sc.stop() diff --git a/examples/src/main/python/mllib/svd_example.py b/examples/src/main/python/mllib/svd_example.py new file mode 100644 index 000000000000..5b220fdb3fd6 --- /dev/null +++ b/examples/src/main/python/mllib/svd_example.py @@ -0,0 +1,48 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.linalg import Vectors +from pyspark.mllib.linalg.distributed import RowMatrix +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="PythonSVDExample") + + # $example on$ + rows = sc.parallelize([ + Vectors.sparse(5, {1: 1.0, 3: 7.0}), + Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0), + Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0) + ]) + + mat = RowMatrix(rows) + + # Compute the top 5 singular values and corresponding singular vectors. + svd = mat.computeSVD(5, computeU=True) + U = svd.U # The U factor is a RowMatrix. + s = svd.s # The singular values are stored in a local dense vector. + V = svd.V # The V factor is a local dense matrix. + # $example off$ + collected = U.rows.collect() + print("U factor is:") + for vector in collected: + print(vector) + print("Singular values are: %s" % s) + print("V factor is:\n%s" % V) + sc.stop() diff --git a/examples/src/main/python/mllib/svm_with_sgd_example.py b/examples/src/main/python/mllib/svm_with_sgd_example.py new file mode 100644 index 000000000000..24b8f431e059 --- /dev/null +++ b/examples/src/main/python/mllib/svm_with_sgd_example.py @@ -0,0 +1,47 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.classification import SVMWithSGD, SVMModel +from pyspark.mllib.regression import LabeledPoint +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="PythonSVMWithSGDExample") + + # $example on$ + # Load and parse the data + def parsePoint(line): + values = [float(x) for x in line.split(' ')] + return LabeledPoint(values[0], values[1:]) + + data = sc.textFile("data/mllib/sample_svm_data.txt") + parsedData = data.map(parsePoint) + + # Build the model + model = SVMWithSGD.train(parsedData, iterations=100) + + # Evaluating the model on training data + labelsAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) + trainErr = labelsAndPreds.filter(lambda lp: lp[0] != lp[1]).count() / float(parsedData.count()) + print("Training Error = " + str(trainErr)) + + # Save and load model + model.save(sc, "target/tmp/pythonSVMWithSGDModel") + sameModel = SVMModel.load(sc, "target/tmp/pythonSVMWithSGDModel") + # $example off$ diff --git a/examples/src/main/python/mllib/tf_idf_example.py b/examples/src/main/python/mllib/tf_idf_example.py new file mode 100644 index 000000000000..b66412b2334e --- /dev/null +++ b/examples/src/main/python/mllib/tf_idf_example.py @@ -0,0 +1,57 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.feature import HashingTF, IDF +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="TFIDFExample") # SparkContext + + # $example on$ + # Load documents (one per line). + documents = sc.textFile("data/mllib/kmeans_data.txt").map(lambda line: line.split(" ")) + + hashingTF = HashingTF() + tf = hashingTF.transform(documents) + + # While applying HashingTF only needs a single pass to the data, applying IDF needs two passes: + # First to compute the IDF vector and second to scale the term frequencies by IDF. + tf.cache() + idf = IDF().fit(tf) + tfidf = idf.transform(tf) + + # spark.mllib's IDF implementation provides an option for ignoring terms + # which occur in less than a minimum number of documents. + # In such cases, the IDF for these terms is set to 0. + # This feature can be used by passing the minDocFreq value to the IDF constructor. + idfIgnore = IDF(minDocFreq=2).fit(tf) + tfidfIgnore = idfIgnore.transform(tf) + # $example off$ + + print("tfidf:") + for each in tfidf.collect(): + print(each) + + print("tfidfIgnore:") + for each in tfidfIgnore.collect(): + print(each) + + sc.stop() diff --git a/examples/src/main/python/mllib/word2vec.py b/examples/src/main/python/mllib/word2vec.py index 40d1b887927e..4e7d4f7610c2 100644 --- a/examples/src/main/python/mllib/word2vec.py +++ b/examples/src/main/python/mllib/word2vec.py @@ -16,7 +16,7 @@ # # This example uses text8 file from http://mattmahoney.net/dc/text8.zip -# The file was downloadded, unziped and split into multiple lines using +# The file was downloaded, unzipped and split into multiple lines using # # wget http://mattmahoney.net/dc/text8.zip # unzip text8.zip diff --git a/examples/src/main/python/mllib/word2vec_example.py b/examples/src/main/python/mllib/word2vec_example.py new file mode 100644 index 000000000000..ad1090c77ee1 --- /dev/null +++ b/examples/src/main/python/mllib/word2vec_example.py @@ -0,0 +1,40 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark import SparkContext +# $example on$ +from pyspark.mllib.feature import Word2Vec +# $example off$ + +if __name__ == "__main__": + sc = SparkContext(appName="Word2VecExample") # SparkContext + + # $example on$ + inp = sc.textFile("data/mllib/sample_lda_data.txt").map(lambda row: row.split(" ")) + + word2vec = Word2Vec() + model = word2vec.fit(inp) + + synonyms = model.findSynonyms('1', 5) + + for word, cosine_distance in synonyms: + print("{}: {}".format(word, cosine_distance)) + # $example off$ + + sc.stop() diff --git a/examples/src/main/python/pagerank.py b/examples/src/main/python/pagerank.py index 2fdc9773d4eb..0d6c253d397a 100755 --- a/examples/src/main/python/pagerank.py +++ b/examples/src/main/python/pagerank.py @@ -18,6 +18,9 @@ """ This is an example implementation of PageRank. For more conventional use, Please refer to PageRank implementation provided by graphx + +Example Usage: +bin/spark-submit examples/src/main/python/pagerank.py data/mllib/pagerank_data.txt 10 """ from __future__ import print_function @@ -25,7 +28,7 @@ import sys from operator import add -from pyspark import SparkContext +from pyspark.sql import SparkSession def computeContribs(urls, rank): @@ -46,19 +49,22 @@ def parseNeighbors(urls): print("Usage: pagerank ", file=sys.stderr) exit(-1) - print("""WARN: This is a naive implementation of PageRank and is - given as an example! Please refer to PageRank implementation provided by graphx""", + print("WARN: This is a naive implementation of PageRank and is given as an example!\n" + + "Please refer to PageRank implementation provided by graphx", file=sys.stderr) # Initialize the spark context. - sc = SparkContext(appName="PythonPageRank") + spark = SparkSession\ + .builder\ + .appName("PythonPageRank")\ + .getOrCreate() # Loads in input file. It should be in format of: # URL neighbor URL # URL neighbor URL # URL neighbor URL # ... - lines = sc.textFile(sys.argv[1], 1) + lines = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0]) # Loads all URLs from input file and initialize their neighbors. links = lines.map(lambda urls: parseNeighbors(urls)).distinct().groupByKey().cache() @@ -79,4 +85,4 @@ def parseNeighbors(urls): for (link, rank) in ranks.collect(): print("%s has rank: %s." % (link, rank)) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/parquet_inputformat.py b/examples/src/main/python/parquet_inputformat.py index e1fd85b082c0..52e9662d528d 100644 --- a/examples/src/main/python/parquet_inputformat.py +++ b/examples/src/main/python/parquet_inputformat.py @@ -1,4 +1,3 @@ -from __future__ import print_function # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -16,9 +15,11 @@ # limitations under the License. # +from __future__ import print_function + import sys -from pyspark import SparkContext +from pyspark.sql import SparkSession """ Read data file users.parquet in local Spark distro: @@ -47,7 +48,13 @@ exit(-1) path = sys.argv[1] - sc = SparkContext(appName="ParquetInputFormat") + + spark = SparkSession\ + .builder\ + .appName("ParquetInputFormat")\ + .getOrCreate() + + sc = spark.sparkContext parquet_rdd = sc.newAPIHadoopFile( path, @@ -59,4 +66,4 @@ for k in output: print(k) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index 92e5cf45abc8..5839cc287495 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -1,4 +1,3 @@ -from __future__ import print_function # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -16,27 +15,33 @@ # limitations under the License. # +from __future__ import print_function + import sys from random import random from operator import add -from pyspark import SparkContext +from pyspark.sql import SparkSession if __name__ == "__main__": """ Usage: pi [partitions] """ - sc = SparkContext(appName="PythonPi") + spark = SparkSession\ + .builder\ + .appName("PythonPi")\ + .getOrCreate() + partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 n = 100000 * partitions def f(_): x = random() * 2 - 1 y = random() * 2 - 1 - return 1 if x ** 2 + y ** 2 < 1 else 0 + return 1 if x ** 2 + y ** 2 <= 1 else 0 - count = sc.parallelize(range(1, n + 1), partitions).map(f).reduce(add) + count = spark.sparkContext.parallelize(range(1, n + 1), partitions).map(f).reduce(add) print("Pi is roughly %f" % (4.0 * count / n)) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py index f6b0ecb02c10..81898cf6d5ce 100755 --- a/examples/src/main/python/sort.py +++ b/examples/src/main/python/sort.py @@ -19,22 +19,27 @@ import sys -from pyspark import SparkContext +from pyspark.sql import SparkSession if __name__ == "__main__": if len(sys.argv) != 2: print("Usage: sort ", file=sys.stderr) exit(-1) - sc = SparkContext(appName="PythonSort") - lines = sc.textFile(sys.argv[1], 1) + + spark = SparkSession\ + .builder\ + .appName("PythonSort")\ + .getOrCreate() + + lines = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0]) sortedCount = lines.flatMap(lambda x: x.split(' ')) \ .map(lambda x: (int(x), 1)) \ - .sortByKey(lambda x: x) + .sortByKey() # This is just a demo on how to bring all the sorted data back to a single node. # In reality, we wouldn't want to collect all the data to the driver node. output = sortedCount.collect() for (num, unitcount) in output: print(num) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py deleted file mode 100644 index 2c188759328f..000000000000 --- a/examples/src/main/python/sql.py +++ /dev/null @@ -1,80 +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. -# - -from __future__ import print_function - -import os -import sys - -from pyspark import SparkContext -from pyspark.sql import SQLContext -from pyspark.sql.types import Row, StructField, StructType, StringType, IntegerType - - -if __name__ == "__main__": - sc = SparkContext(appName="PythonSQL") - sqlContext = SQLContext(sc) - - # RDD is created from a list of rows - some_rdd = sc.parallelize([Row(name="John", age=19), - 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.createDataFrame(some_rdd) - some_df.printSchema() - - # Another RDD is created from a list of tuples - another_rdd = sc.parallelize([("John", 19), ("Smith", 23), ("Sarah", 18)]) - # Schema with two fields - person_name and person_age - 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.createDataFrame(another_rdd, schema) - another_df.printSchema() - # root - # |-- age: integer (nullable = true) - # |-- name: string (nullable = true) - - # A JSON dataset is pointed to by path. - # The path can be either a single text file or a directory storing text files. - if len(sys.argv) < 2: - path = "file://" + \ - os.path.join(os.environ['SPARK_HOME'], "examples/src/main/resources/people.json") - else: - path = sys.argv[1] - # Create a DataFrame from the file(s) pointed to by path - people = sqlContext.jsonFile(path) - # root - # |-- person_name: string (nullable = false) - # |-- person_age: integer (nullable = false) - - # The inferred schema can be visualized using the printSchema() method. - people.printSchema() - # root - # |-- age: IntegerType - # |-- name: StringType - - # Register this DataFrame as a table. - people.registerAsTable("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") - - for each in teenagers.collect(): - print(each[0]) - - sc.stop() diff --git a/examples/src/main/python/sql/basic.py b/examples/src/main/python/sql/basic.py new file mode 100644 index 000000000000..c07fa8f2752b --- /dev/null +++ b/examples/src/main/python/sql/basic.py @@ -0,0 +1,216 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 __future__ import print_function + +# $example on:init_session$ +from pyspark.sql import SparkSession +# $example off:init_session$ + +# $example on:schema_inferring$ +from pyspark.sql import Row +# $example off:schema_inferring$ + +# $example on:programmatic_schema$ +# Import data types +from pyspark.sql.types import * +# $example off:programmatic_schema$ + +""" +A simple example demonstrating basic Spark SQL features. +Run with: + ./bin/spark-submit examples/src/main/python/sql/basic.py +""" + + +def basic_df_example(spark): + # $example on:create_df$ + # spark is an existing SparkSession + df = spark.read.json("examples/src/main/resources/people.json") + # Displays the content of the DataFrame to stdout + df.show() + # +----+-------+ + # | age| name| + # +----+-------+ + # |null|Michael| + # | 30| Andy| + # | 19| Justin| + # +----+-------+ + # $example off:create_df$ + + # $example on:untyped_ops$ + # spark, df are from the previous example + # Print the schema in a tree format + df.printSchema() + # root + # |-- age: long (nullable = true) + # |-- name: string (nullable = true) + + # Select only the "name" column + df.select("name").show() + # +-------+ + # | name| + # +-------+ + # |Michael| + # | Andy| + # | Justin| + # +-------+ + + # Select everybody, but increment the age by 1 + df.select(df['name'], df['age'] + 1).show() + # +-------+---------+ + # | name|(age + 1)| + # +-------+---------+ + # |Michael| null| + # | Andy| 31| + # | Justin| 20| + # +-------+---------+ + + # Select people older than 21 + df.filter(df['age'] > 21).show() + # +---+----+ + # |age|name| + # +---+----+ + # | 30|Andy| + # +---+----+ + + # Count people by age + df.groupBy("age").count().show() + # +----+-----+ + # | age|count| + # +----+-----+ + # | 19| 1| + # |null| 1| + # | 30| 1| + # +----+-----+ + # $example off:untyped_ops$ + + # $example on:run_sql$ + # Register the DataFrame as a SQL temporary view + df.createOrReplaceTempView("people") + + sqlDF = spark.sql("SELECT * FROM people") + sqlDF.show() + # +----+-------+ + # | age| name| + # +----+-------+ + # |null|Michael| + # | 30| Andy| + # | 19| Justin| + # +----+-------+ + # $example off:run_sql$ + + # $example on:global_temp_view$ + # Register the DataFrame as a global temporary view + df.createGlobalTempView("people") + + # Global temporary view is tied to a system preserved database `global_temp` + spark.sql("SELECT * FROM global_temp.people").show() + # +----+-------+ + # | age| name| + # +----+-------+ + # |null|Michael| + # | 30| Andy| + # | 19| Justin| + # +----+-------+ + + # Global temporary view is cross-session + spark.newSession().sql("SELECT * FROM global_temp.people").show() + # +----+-------+ + # | age| name| + # +----+-------+ + # |null|Michael| + # | 30| Andy| + # | 19| Justin| + # +----+-------+ + # $example off:global_temp_view$ + + +def schema_inference_example(spark): + # $example on:schema_inferring$ + sc = spark.sparkContext + + # Load a text file and convert each line to a Row. + 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 DataFrame as a table. + schemaPeople = spark.createDataFrame(people) + schemaPeople.createOrReplaceTempView("people") + + # SQL can be run over DataFrames that have been registered as a table. + teenagers = spark.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + + # The results of SQL queries are Dataframe objects. + # rdd returns the content as an :class:`pyspark.RDD` of :class:`Row`. + teenNames = teenagers.rdd.map(lambda p: "Name: " + p.name).collect() + for name in teenNames: + print(name) + # Name: Justin + # $example off:schema_inferring$ + + +def programmatic_schema_example(spark): + # $example on:programmatic_schema$ + sc = spark.sparkContext + + # Load a text file and convert each line to a Row. + lines = sc.textFile("examples/src/main/resources/people.txt") + parts = lines.map(lambda l: l.split(",")) + # Each line is converted to a tuple. + people = parts.map(lambda p: (p[0], p[1].strip())) + + # The schema is encoded in a string. + schemaString = "name age" + + fields = [StructField(field_name, StringType(), True) for field_name in schemaString.split()] + schema = StructType(fields) + + # Apply the schema to the RDD. + schemaPeople = spark.createDataFrame(people, schema) + + # Creates a temporary view using the DataFrame + schemaPeople.createOrReplaceTempView("people") + + # SQL can be run over DataFrames that have been registered as a table. + results = spark.sql("SELECT name FROM people") + + results.show() + # +-------+ + # | name| + # +-------+ + # |Michael| + # | Andy| + # | Justin| + # +-------+ + # $example off:programmatic_schema$ + +if __name__ == "__main__": + # $example on:init_session$ + spark = SparkSession \ + .builder \ + .appName("Python Spark SQL basic example") \ + .config("spark.some.config.option", "some-value") \ + .getOrCreate() + # $example off:init_session$ + + basic_df_example(spark) + schema_inference_example(spark) + programmatic_schema_example(spark) + + spark.stop() diff --git a/examples/src/main/python/sql/datasource.py b/examples/src/main/python/sql/datasource.py new file mode 100644 index 000000000000..f86012ea382e --- /dev/null +++ b/examples/src/main/python/sql/datasource.py @@ -0,0 +1,223 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +from pyspark.sql import SparkSession +# $example on:schema_merging$ +from pyspark.sql import Row +# $example off:schema_merging$ + +""" +A simple example demonstrating Spark SQL data sources. +Run with: + ./bin/spark-submit examples/src/main/python/sql/datasource.py +""" + + +def basic_datasource_example(spark): + # $example on:generic_load_save_functions$ + df = spark.read.load("examples/src/main/resources/users.parquet") + df.select("name", "favorite_color").write.save("namesAndFavColors.parquet") + # $example off:generic_load_save_functions$ + + # $example on:write_partitioning$ + df.write.partitionBy("favorite_color").format("parquet").save("namesPartByColor.parquet") + # $example off:write_partitioning$ + + # $example on:write_partition_and_bucket$ + df = spark.read.parquet("examples/src/main/resources/users.parquet") + (df + .write + .partitionBy("favorite_color") + .bucketBy(42, "name") + .saveAsTable("people_partitioned_bucketed")) + # $example off:write_partition_and_bucket$ + + # $example on:manual_load_options$ + df = spark.read.load("examples/src/main/resources/people.json", format="json") + df.select("name", "age").write.save("namesAndAges.parquet", format="parquet") + # $example off:manual_load_options$ + + # $example on:write_sorting_and_bucketing$ + df.write.bucketBy(42, "name").sortBy("age").saveAsTable("people_bucketed") + # $example off:write_sorting_and_bucketing$ + + # $example on:direct_sql$ + df = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") + # $example off:direct_sql$ + + spark.sql("DROP TABLE IF EXISTS people_bucketed") + spark.sql("DROP TABLE IF EXISTS people_partitioned_bucketed") + + +def parquet_example(spark): + # $example on:basic_parquet_example$ + peopleDF = spark.read.json("examples/src/main/resources/people.json") + + # DataFrames can be saved as Parquet files, maintaining the schema information. + peopleDF.write.parquet("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 DataFrame. + parquetFile = spark.read.parquet("people.parquet") + + # Parquet files can also be used to create a temporary view and then used in SQL statements. + parquetFile.createOrReplaceTempView("parquetFile") + teenagers = spark.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") + teenagers.show() + # +------+ + # | name| + # +------+ + # |Justin| + # +------+ + # $example off:basic_parquet_example$ + + +def parquet_schema_merging_example(spark): + # $example on:schema_merging$ + # spark is from the previous example. + # Create a simple DataFrame, stored into a partition directory + sc = spark.sparkContext + + squaresDF = spark.createDataFrame(sc.parallelize(range(1, 6)) + .map(lambda i: Row(single=i, double=i ** 2))) + squaresDF.write.parquet("data/test_table/key=1") + + # Create another DataFrame in a new partition directory, + # adding a new column and dropping an existing column + cubesDF = spark.createDataFrame(sc.parallelize(range(6, 11)) + .map(lambda i: Row(single=i, triple=i ** 3))) + cubesDF.write.parquet("data/test_table/key=2") + + # Read the partitioned table + mergedDF = spark.read.option("mergeSchema", "true").parquet("data/test_table") + mergedDF.printSchema() + + # The final schema consists of all 3 columns in the Parquet files together + # with the partitioning column appeared in the partition directory paths. + # root + # |-- double: long (nullable = true) + # |-- single: long (nullable = true) + # |-- triple: long (nullable = true) + # |-- key: integer (nullable = true) + # $example off:schema_merging$ + + +def json_dataset_example(spark): + # $example on:json_dataset$ + # spark is from the previous example. + sc = spark.sparkContext + + # 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" + peopleDF = spark.read.json(path) + + # The inferred schema can be visualized using the printSchema() method + peopleDF.printSchema() + # root + # |-- age: long (nullable = true) + # |-- name: string (nullable = true) + + # Creates a temporary view using the DataFrame + peopleDF.createOrReplaceTempView("people") + + # SQL statements can be run by using the sql methods provided by spark + teenagerNamesDF = spark.sql("SELECT name FROM people WHERE age BETWEEN 13 AND 19") + teenagerNamesDF.show() + # +------+ + # | name| + # +------+ + # |Justin| + # +------+ + + # Alternatively, a DataFrame can be created for a JSON dataset represented by + # an RDD[String] storing one JSON object per string + jsonStrings = ['{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}'] + otherPeopleRDD = sc.parallelize(jsonStrings) + otherPeople = spark.read.json(otherPeopleRDD) + otherPeople.show() + # +---------------+----+ + # | address|name| + # +---------------+----+ + # |[Columbus,Ohio]| Yin| + # +---------------+----+ + # $example off:json_dataset$ + + +def jdbc_dataset_example(spark): + # $example on:jdbc_dataset$ + # Note: JDBC loading and saving can be achieved via either the load/save or jdbc methods + # Loading data from a JDBC source + jdbcDF = spark.read \ + .format("jdbc") \ + .option("url", "jdbc:postgresql:dbserver") \ + .option("dbtable", "schema.tablename") \ + .option("user", "username") \ + .option("password", "password") \ + .load() + + jdbcDF2 = spark.read \ + .jdbc("jdbc:postgresql:dbserver", "schema.tablename", + properties={"user": "username", "password": "password"}) + + # Specifying dataframe column data types on read + jdbcDF3 = spark.read \ + .format("jdbc") \ + .option("url", "jdbc:postgresql:dbserver") \ + .option("dbtable", "schema.tablename") \ + .option("user", "username") \ + .option("password", "password") \ + .option("customSchema", "id DECIMAL(38, 0), name STRING") \ + .load() + + # Saving data to a JDBC source + jdbcDF.write \ + .format("jdbc") \ + .option("url", "jdbc:postgresql:dbserver") \ + .option("dbtable", "schema.tablename") \ + .option("user", "username") \ + .option("password", "password") \ + .save() + + jdbcDF2.write \ + .jdbc("jdbc:postgresql:dbserver", "schema.tablename", + properties={"user": "username", "password": "password"}) + + # Specifying create table column data types on write + jdbcDF.write \ + .option("createTableColumnTypes", "name CHAR(64), comments VARCHAR(1024)") \ + .jdbc("jdbc:postgresql:dbserver", "schema.tablename", + properties={"user": "username", "password": "password"}) + # $example off:jdbc_dataset$ + + +if __name__ == "__main__": + spark = SparkSession \ + .builder \ + .appName("Python Spark SQL data source example") \ + .getOrCreate() + + basic_datasource_example(spark) + parquet_example(spark) + parquet_schema_merging_example(spark) + json_dataset_example(spark) + jdbc_dataset_example(spark) + + spark.stop() diff --git a/examples/src/main/python/sql/hive.py b/examples/src/main/python/sql/hive.py new file mode 100644 index 000000000000..1f83a6fb48b9 --- /dev/null +++ b/examples/src/main/python/sql/hive.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 __future__ import print_function + +# $example on:spark_hive$ +from os.path import expanduser, join, abspath + +from pyspark.sql import SparkSession +from pyspark.sql import Row +# $example off:spark_hive$ + +""" +A simple example demonstrating Spark SQL Hive integration. +Run with: + ./bin/spark-submit examples/src/main/python/sql/hive.py +""" + + +if __name__ == "__main__": + # $example on:spark_hive$ + # warehouse_location points to the default location for managed databases and tables + warehouse_location = abspath('spark-warehouse') + + spark = SparkSession \ + .builder \ + .appName("Python Spark SQL Hive integration example") \ + .config("spark.sql.warehouse.dir", warehouse_location) \ + .enableHiveSupport() \ + .getOrCreate() + + # spark is an existing SparkSession + spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING) USING hive") + spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") + + # Queries are expressed in HiveQL + spark.sql("SELECT * FROM src").show() + # +---+-------+ + # |key| value| + # +---+-------+ + # |238|val_238| + # | 86| val_86| + # |311|val_311| + # ... + + # Aggregation queries are also supported. + spark.sql("SELECT COUNT(*) FROM src").show() + # +--------+ + # |count(1)| + # +--------+ + # | 500 | + # +--------+ + + # The results of SQL queries are themselves DataFrames and support all normal functions. + sqlDF = spark.sql("SELECT key, value FROM src WHERE key < 10 ORDER BY key") + + # The items in DataFrames are of type Row, which allows you to access each column by ordinal. + stringsDS = sqlDF.rdd.map(lambda row: "Key: %d, Value: %s" % (row.key, row.value)) + for record in stringsDS.collect(): + print(record) + # Key: 0, Value: val_0 + # Key: 0, Value: val_0 + # Key: 0, Value: val_0 + # ... + + # You can also use DataFrames to create temporary views within a SparkSession. + Record = Row("key", "value") + recordsDF = spark.createDataFrame([Record(i, "val_" + str(i)) for i in range(1, 101)]) + recordsDF.createOrReplaceTempView("records") + + # Queries can then join DataFrame data with data stored in Hive. + spark.sql("SELECT * FROM records r JOIN src s ON r.key = s.key").show() + # +---+------+---+------+ + # |key| value|key| value| + # +---+------+---+------+ + # | 2| val_2| 2| val_2| + # | 4| val_4| 4| val_4| + # | 5| val_5| 5| val_5| + # ... + # $example off:spark_hive$ + + spark.stop() diff --git a/examples/src/main/python/sql/streaming/structured_kafka_wordcount.py b/examples/src/main/python/sql/streaming/structured_kafka_wordcount.py new file mode 100644 index 000000000000..9e8a552b3b10 --- /dev/null +++ b/examples/src/main/python/sql/streaming/structured_kafka_wordcount.py @@ -0,0 +1,90 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" + Consumes messages from one or more topics in Kafka and does wordcount. + Usage: structured_kafka_wordcount.py + The Kafka "bootstrap.servers" configuration. A + comma-separated list of host:port. + There are three kinds of type, i.e. 'assign', 'subscribe', + 'subscribePattern'. + |- Specific TopicPartitions to consume. Json string + | {"topicA":[0,1],"topicB":[2,4]}. + |- The topic list to subscribe. A comma-separated list of + | topics. + |- The pattern used to subscribe to topic(s). + | Java regex string. + |- Only one of "assign, "subscribe" or "subscribePattern" options can be + | specified for Kafka source. + Different value format depends on the value of 'subscribe-type'. + + Run the example + `$ bin/spark-submit examples/src/main/python/sql/streaming/structured_kafka_wordcount.py \ + host1:port1,host2:port2 subscribe topic1,topic2` +""" +from __future__ import print_function + +import sys + +from pyspark.sql import SparkSession +from pyspark.sql.functions import explode +from pyspark.sql.functions import split + +if __name__ == "__main__": + if len(sys.argv) != 4: + print(""" + Usage: structured_kafka_wordcount.py + """, file=sys.stderr) + exit(-1) + + bootstrapServers = sys.argv[1] + subscribeType = sys.argv[2] + topics = sys.argv[3] + + spark = SparkSession\ + .builder\ + .appName("StructuredKafkaWordCount")\ + .getOrCreate() + + # Create DataSet representing the stream of input lines from kafka + lines = spark\ + .readStream\ + .format("kafka")\ + .option("kafka.bootstrap.servers", bootstrapServers)\ + .option(subscribeType, topics)\ + .load()\ + .selectExpr("CAST(value AS STRING)") + + # Split the lines into words + words = lines.select( + # explode turns each item in an array into a separate row + explode( + split(lines.value, ' ') + ).alias('word') + ) + + # Generate running word count + wordCounts = words.groupBy('word').count() + + # Start running the query that prints the running counts to the console + query = wordCounts\ + .writeStream\ + .outputMode('complete')\ + .format('console')\ + .start() + + query.awaitTermination() diff --git a/examples/src/main/python/sql/streaming/structured_network_wordcount.py b/examples/src/main/python/sql/streaming/structured_network_wordcount.py new file mode 100644 index 000000000000..afde2550587c --- /dev/null +++ b/examples/src/main/python/sql/streaming/structured_network_wordcount.py @@ -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. +# + +""" + Counts words in UTF8 encoded, '\n' delimited text received from the network. + Usage: structured_network_wordcount.py + and describe the TCP server that Structured Streaming + would connect to receive data. + + To run this on your local machine, you need to first run a Netcat server + `$ nc -lk 9999` + and then run the example + `$ bin/spark-submit examples/src/main/python/sql/streaming/structured_network_wordcount.py + localhost 9999` +""" +from __future__ import print_function + +import sys + +from pyspark.sql import SparkSession +from pyspark.sql.functions import explode +from pyspark.sql.functions import split + +if __name__ == "__main__": + if len(sys.argv) != 3: + print("Usage: structured_network_wordcount.py ", file=sys.stderr) + exit(-1) + + host = sys.argv[1] + port = int(sys.argv[2]) + + spark = SparkSession\ + .builder\ + .appName("StructuredNetworkWordCount")\ + .getOrCreate() + + # Create DataFrame representing the stream of input lines from connection to host:port + lines = spark\ + .readStream\ + .format('socket')\ + .option('host', host)\ + .option('port', port)\ + .load() + + # Split the lines into words + words = lines.select( + # explode turns each item in an array into a separate row + explode( + split(lines.value, ' ') + ).alias('word') + ) + + # Generate running word count + wordCounts = words.groupBy('word').count() + + # Start running the query that prints the running counts to the console + query = wordCounts\ + .writeStream\ + .outputMode('complete')\ + .format('console')\ + .start() + + query.awaitTermination() diff --git a/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py b/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py new file mode 100644 index 000000000000..02a7d3363d78 --- /dev/null +++ b/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py @@ -0,0 +1,102 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" + Counts words in UTF8 encoded, '\n' delimited text received from the network over a + sliding window of configurable duration. Each line from the network is tagged + with a timestamp that is used to determine the windows into which it falls. + + Usage: structured_network_wordcount_windowed.py + [] + and describe the TCP server that Structured Streaming + would connect to receive data. + gives the size of window, specified as integer number of seconds + gives the amount of time successive windows are offset from one another, + given in the same units as above. should be less than or equal to + . If the two are equal, successive windows have no overlap. If + is not provided, it defaults to . + + To run this on your local machine, you need to first run a Netcat server + `$ nc -lk 9999` + and then run the example + `$ bin/spark-submit + examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py + localhost 9999 []` + + One recommended , pair is 10, 5 +""" +from __future__ import print_function + +import sys + +from pyspark.sql import SparkSession +from pyspark.sql.functions import explode +from pyspark.sql.functions import split +from pyspark.sql.functions import window + +if __name__ == "__main__": + if len(sys.argv) != 5 and len(sys.argv) != 4: + msg = ("Usage: structured_network_wordcount_windowed.py " + " []") + print(msg, file=sys.stderr) + exit(-1) + + host = sys.argv[1] + port = int(sys.argv[2]) + windowSize = int(sys.argv[3]) + slideSize = int(sys.argv[4]) if (len(sys.argv) == 5) else windowSize + if slideSize > windowSize: + print(" must be less than or equal to ", file=sys.stderr) + windowDuration = '{} seconds'.format(windowSize) + slideDuration = '{} seconds'.format(slideSize) + + spark = SparkSession\ + .builder\ + .appName("StructuredNetworkWordCountWindowed")\ + .getOrCreate() + + # Create DataFrame representing the stream of input lines from connection to host:port + lines = spark\ + .readStream\ + .format('socket')\ + .option('host', host)\ + .option('port', port)\ + .option('includeTimestamp', 'true')\ + .load() + + # Split the lines into words, retaining timestamps + # split() splits each line into an array, and explode() turns the array into multiple rows + words = lines.select( + explode(split(lines.value, ' ')).alias('word'), + lines.timestamp + ) + + # Group the data by window and word and compute the count of each group + windowedCounts = words.groupBy( + window(words.timestamp, windowDuration, slideDuration), + words.word + ).count().orderBy('window') + + # Start running the query that prints the windowed word counts to the console + query = windowedCounts\ + .writeStream\ + .outputMode('complete')\ + .format('console')\ + .option('truncate', 'false')\ + .start() + + query.awaitTermination() diff --git a/examples/src/main/python/status_api_demo.py b/examples/src/main/python/status_api_demo.py index 49b7902185aa..8cc8cc820cfc 100644 --- a/examples/src/main/python/status_api_demo.py +++ b/examples/src/main/python/status_api_demo.py @@ -19,7 +19,11 @@ import time import threading -import Queue +import sys +if sys.version >= '3': + import queue as Queue +else: + import Queue from pyspark import SparkConf, SparkContext diff --git a/examples/src/main/python/streaming/direct_kafka_wordcount.py b/examples/src/main/python/streaming/direct_kafka_wordcount.py index ea20678b9aca..7097f7f4502b 100644 --- a/examples/src/main/python/streaming/direct_kafka_wordcount.py +++ b/examples/src/main/python/streaming/direct_kafka_wordcount.py @@ -28,6 +28,7 @@ examples/src/main/python/streaming/direct_kafka_wordcount.py \ localhost:9092 test` """ +from __future__ import print_function import sys diff --git a/examples/src/main/python/streaming/mqtt_wordcount.py b/examples/src/main/python/streaming/mqtt_wordcount.py deleted file mode 100644 index abf9c0e21d30..000000000000 --- a/examples/src/main/python/streaming/mqtt_wordcount.py +++ /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. -# - -""" - A sample wordcount with MqttStream stream - Usage: mqtt_wordcount.py - - To run this in your local machine, you need to setup a MQTT broker and publisher first, - Mosquitto is one of the open source MQTT Brokers, see - http://mosquitto.org/ - Eclipse paho project provides number of clients and utilities for working with MQTT, see - http://www.eclipse.org/paho/#getting-started - - and then run the example - `$ bin/spark-submit --jars \ - external/mqtt-assembly/target/scala-*/spark-streaming-mqtt-assembly-*.jar \ - examples/src/main/python/streaming/mqtt_wordcount.py \ - tcp://localhost:1883 foo` -""" - -import sys - -from pyspark import SparkContext -from pyspark.streaming import StreamingContext -from pyspark.streaming.mqtt import MQTTUtils - -if __name__ == "__main__": - if len(sys.argv) != 3: - print >> sys.stderr, "Usage: mqtt_wordcount.py " - exit(-1) - - sc = SparkContext(appName="PythonStreamingMQTTWordCount") - ssc = StreamingContext(sc, 1) - - brokerUrl = sys.argv[1] - topic = sys.argv[2] - - lines = MQTTUtils.createStream(ssc, brokerUrl, topic) - counts = lines.flatMap(lambda line: line.split(" ")) \ - .map(lambda word: (word, 1)) \ - .reduceByKey(lambda a, b: a+b) - counts.pprint() - - ssc.start() - ssc.awaitTermination() diff --git a/examples/src/main/python/streaming/network_wordjoinsentiments.py b/examples/src/main/python/streaming/network_wordjoinsentiments.py new file mode 100644 index 000000000000..b309d9fad33f --- /dev/null +++ b/examples/src/main/python/streaming/network_wordjoinsentiments.py @@ -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. +# + +""" + Shows the most positive words in UTF8 encoded, '\n' delimited text directly received the network + every 5 seconds. The streaming data is joined with a static RDD of the AFINN word list + (http://neuro.imm.dtu.dk/wiki/AFINN) + + Usage: network_wordjoinsentiments.py + and describe the TCP server that Spark Streaming would connect to receive data. + + To run this on your local machine, you need to first run a Netcat server + `$ nc -lk 9999` + and then run the example + `$ bin/spark-submit examples/src/main/python/streaming/network_wordjoinsentiments.py \ + localhost 9999` +""" + +from __future__ import print_function + +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext + + +def print_happiest_words(rdd): + top_list = rdd.take(5) + print("Happiest topics in the last 5 seconds (%d total):" % rdd.count()) + for tuple in top_list: + print("%s (%d happiness)" % (tuple[1], tuple[0])) + +if __name__ == "__main__": + if len(sys.argv) != 3: + print("Usage: network_wordjoinsentiments.py ", file=sys.stderr) + exit(-1) + + sc = SparkContext(appName="PythonStreamingNetworkWordJoinSentiments") + ssc = StreamingContext(sc, 5) + + # Read in the word-sentiment list and create a static RDD from it + word_sentiments_file_path = "data/streaming/AFINN-111.txt" + word_sentiments = ssc.sparkContext.textFile(word_sentiments_file_path) \ + .map(lambda line: tuple(line.split("\t"))) + + lines = ssc.socketTextStream(sys.argv[1], int(sys.argv[2])) + + word_counts = lines.flatMap(lambda line: line.split(" ")) \ + .map(lambda word: (word, 1)) \ + .reduceByKey(lambda a, b: a + b) + + # Determine the words with the highest sentiment values by joining the streaming RDD + # with the static RDD inside the transform() method and then multiplying + # the frequency of the words by its sentiment value + happiest_words = word_counts.transform(lambda rdd: word_sentiments.join(rdd)) \ + .map(lambda word_tuples: (word_tuples[0], float(word_tuples[1][0]) * word_tuples[1][1])) \ + .map(lambda word_happiness: (word_happiness[1], word_happiness[0])) \ + .transform(lambda rdd: rdd.sortByKey(False)) + + happiest_words.foreachRDD(print_happiest_words) + + ssc.start() + ssc.awaitTermination() diff --git a/examples/src/main/python/streaming/queue_stream.py b/examples/src/main/python/streaming/queue_stream.py index b3808907f74a..bdd2d4851949 100644 --- a/examples/src/main/python/streaming/queue_stream.py +++ b/examples/src/main/python/streaming/queue_stream.py @@ -22,7 +22,6 @@ To run this example use `$ bin/spark-submit examples/src/main/python/streaming/queue_stream.py """ -import sys import time from pyspark import SparkContext diff --git a/examples/src/main/python/streaming/recoverable_network_wordcount.py b/examples/src/main/python/streaming/recoverable_network_wordcount.py index ac91f0a06b17..52b2639cdf55 100644 --- a/examples/src/main/python/streaming/recoverable_network_wordcount.py +++ b/examples/src/main/python/streaming/recoverable_network_wordcount.py @@ -44,6 +44,20 @@ from pyspark.streaming import StreamingContext +# Get or register a Broadcast variable +def getWordBlacklist(sparkContext): + if ('wordBlacklist' not in globals()): + globals()['wordBlacklist'] = sparkContext.broadcast(["a", "b", "c"]) + return globals()['wordBlacklist'] + + +# Get or register an Accumulator +def getDroppedWordsCounter(sparkContext): + if ('droppedWordsCounter' not in globals()): + globals()['droppedWordsCounter'] = sparkContext.accumulator(0) + return globals()['droppedWordsCounter'] + + def createContext(host, port, outputPath): # If you do not see this printed, that means the StreamingContext has been loaded # from the new checkpoint @@ -60,8 +74,22 @@ def createContext(host, port, outputPath): wordCounts = words.map(lambda x: (x, 1)).reduceByKey(lambda x, y: x + y) def echo(time, rdd): - counts = "Counts at time %s %s" % (time, rdd.collect()) + # Get or register the blacklist Broadcast + blacklist = getWordBlacklist(rdd.context) + # Get or register the droppedWordsCounter Accumulator + droppedWordsCounter = getDroppedWordsCounter(rdd.context) + + # Use blacklist to drop words and use droppedWordsCounter to count them + def filterFunc(wordCount): + if wordCount[0] in blacklist.value: + droppedWordsCounter.add(wordCount[1]) + False + else: + True + + counts = "Counts at time %s %s" % (time, rdd.filter(filterFunc).collect()) print(counts) + print("Dropped %d word(s) totally" % droppedWordsCounter.value) print("Appending to " + os.path.abspath(outputPath)) with open(outputPath, 'a') as f: f.write(counts + "\n") diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py index da90c07dbd82..398ac8d2d8f5 100644 --- a/examples/src/main/python/streaming/sql_network_wordcount.py +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -29,18 +29,20 @@ """ from __future__ import print_function -import os import sys from pyspark import SparkContext from pyspark.streaming import StreamingContext -from pyspark.sql import SQLContext, Row +from pyspark.sql import Row, SparkSession -def getSqlContextInstance(sparkContext): - if ('sqlContextSingletonInstance' not in globals()): - globals()['sqlContextSingletonInstance'] = SQLContext(sparkContext) - return globals()['sqlContextSingletonInstance'] +def getSparkSessionInstance(sparkConf): + if ('sparkSessionSingletonInstance' not in globals()): + globals()['sparkSessionSingletonInstance'] = SparkSession\ + .builder\ + .config(conf=sparkConf)\ + .getOrCreate() + return globals()['sparkSessionSingletonInstance'] if __name__ == "__main__": @@ -61,19 +63,19 @@ def process(time, rdd): print("========= %s =========" % str(time)) try: - # Get the singleton instance of SQLContext - sqlContext = getSqlContextInstance(rdd.context) + # Get the singleton instance of SparkSession + spark = getSparkSessionInstance(rdd.context.getConf()) # Convert RDD[String] to RDD[Row] to DataFrame rowRdd = rdd.map(lambda w: Row(word=w)) - wordsDataFrame = sqlContext.createDataFrame(rowRdd) + wordsDataFrame = spark.createDataFrame(rowRdd) - # Register as table - wordsDataFrame.registerTempTable("words") + # Creates a temporary view using the DataFrame. + wordsDataFrame.createOrReplaceTempView("words") # Do word count on table using SQL and print it wordCountsDataFrame = \ - sqlContext.sql("select word, count(*) as total from words group by word") + spark.sql("select word, count(*) as total from words group by word") wordCountsDataFrame.show() except: pass diff --git a/examples/src/main/python/streaming/stateful_network_wordcount.py b/examples/src/main/python/streaming/stateful_network_wordcount.py index 16ef646b7c42..f8bbc659c2ea 100644 --- a/examples/src/main/python/streaming/stateful_network_wordcount.py +++ b/examples/src/main/python/streaming/stateful_network_wordcount.py @@ -44,13 +44,16 @@ ssc = StreamingContext(sc, 1) ssc.checkpoint("checkpoint") + # RDD with initial state (key, value) pairs + initialStateRDD = sc.parallelize([(u'hello', 1), (u'world', 1)]) + def updateFunc(new_values, last_sum): return sum(new_values) + (last_sum or 0) lines = ssc.socketTextStream(sys.argv[1], int(sys.argv[2])) running_counts = lines.flatMap(lambda line: line.split(" "))\ .map(lambda word: (word, 1))\ - .updateStateByKey(updateFunc) + .updateStateByKey(updateFunc, initialRDD=initialStateRDD) running_counts.pprint() diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index 7bf5fb6ddfe2..49551d40851c 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -20,7 +20,7 @@ import sys from random import Random -from pyspark import SparkContext +from pyspark.sql import SparkSession numEdges = 200 numVertices = 100 @@ -30,8 +30,8 @@ def generateGraph(): edges = set() while len(edges) < numEdges: - src = rand.randrange(0, numEdges) - dst = rand.randrange(0, numEdges) + src = rand.randrange(0, numVertices) + dst = rand.randrange(0, numVertices) if src != dst: edges.add((src, dst)) return edges @@ -41,9 +41,13 @@ def generateGraph(): """ Usage: transitive_closure [partitions] """ - sc = SparkContext(appName="PythonTransitiveClosure") + spark = SparkSession\ + .builder\ + .appName("PythonTransitiveClosure")\ + .getOrCreate() + partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 - tc = sc.parallelize(generateGraph(), partitions).cache() + tc = spark.sparkContext.parallelize(generateGraph(), partitions).cache() # Linear transitive closure: each round grows paths by one edge, # by joining the graph's edges with the already-discovered paths. @@ -67,4 +71,4 @@ def generateGraph(): print("TC has %i edges" % tc.count()) - sc.stop() + spark.stop() diff --git a/examples/src/main/python/wordcount.py b/examples/src/main/python/wordcount.py index 7c0143607b61..3d5e44d5b2df 100755 --- a/examples/src/main/python/wordcount.py +++ b/examples/src/main/python/wordcount.py @@ -20,15 +20,20 @@ import sys from operator import add -from pyspark import SparkContext +from pyspark.sql import SparkSession if __name__ == "__main__": if len(sys.argv) != 2: print("Usage: wordcount ", file=sys.stderr) exit(-1) - sc = SparkContext(appName="PythonWordCount") - lines = sc.textFile(sys.argv[1], 1) + + spark = SparkSession\ + .builder\ + .appName("PythonWordCount")\ + .getOrCreate() + + lines = spark.read.text(sys.argv[1]).rdd.map(lambda r: r[0]) counts = lines.flatMap(lambda x: x.split(' ')) \ .map(lambda x: (x, 1)) \ .reduceByKey(add) @@ -36,4 +41,4 @@ for (word, count) in output: print("%s: %i" % (word, count)) - sc.stop() + spark.stop() diff --git a/examples/src/main/r/RSparkSQLExample.R b/examples/src/main/r/RSparkSQLExample.R new file mode 100644 index 000000000000..3734568d872d --- /dev/null +++ b/examples/src/main/r/RSparkSQLExample.R @@ -0,0 +1,218 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/RSparkSQLExample.R + +library(SparkR) + +# $example on:init_session$ +sparkR.session(appName = "R Spark SQL basic example", sparkConfig = list(spark.some.config.option = "some-value")) +# $example off:init_session$ + + +# $example on:create_df$ +df <- read.json("examples/src/main/resources/people.json") + +# Displays the content of the DataFrame +head(df) +## age name +## 1 NA Michael +## 2 30 Andy +## 3 19 Justin + +# Another method to print the first few rows and optionally truncate the printing of long values +showDF(df) +## +----+-------+ +## | age| name| +## +----+-------+ +## |null|Michael| +## | 30| Andy| +## | 19| Justin| +## +----+-------+ +## $example off:create_df$ + + +# $example on:untyped_ops$ +# Create the DataFrame +df <- read.json("examples/src/main/resources/people.json") + +# Show the content of the DataFrame +head(df) +## age name +## 1 NA Michael +## 2 30 Andy +## 3 19 Justin + + +# Print the schema in a tree format +printSchema(df) +## root +## |-- age: long (nullable = true) +## |-- name: string (nullable = true) + +# Select only the "name" column +head(select(df, "name")) +## name +## 1 Michael +## 2 Andy +## 3 Justin + +# Select everybody, but increment the age by 1 +head(select(df, df$name, df$age + 1)) +## name (age + 1.0) +## 1 Michael NA +## 2 Andy 31 +## 3 Justin 20 + +# Select people older than 21 +head(where(df, df$age > 21)) +## age name +## 1 30 Andy + +# Count people by age +head(count(groupBy(df, "age"))) +## age count +## 1 19 1 +## 2 NA 1 +## 3 30 1 +# $example off:untyped_ops$ + + +# Register this DataFrame as a table. +createOrReplaceTempView(df, "table") +# $example on:run_sql$ +df <- sql("SELECT * FROM table") +# $example off:run_sql$ + + +# $example on:generic_load_save_functions$ +df <- read.df("examples/src/main/resources/users.parquet") +write.df(select(df, "name", "favorite_color"), "namesAndFavColors.parquet") +# $example off:generic_load_save_functions$ + + +# $example on:manual_load_options$ +df <- read.df("examples/src/main/resources/people.json", "json") +namesAndAges <- select(df, "name", "age") +write.df(namesAndAges, "namesAndAges.parquet", "parquet") +# $example off:manual_load_options$ + + +# $example on:direct_sql$ +df <- sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") +# $example off:direct_sql$ + + +# $example on:basic_parquet_example$ +df <- read.df("examples/src/main/resources/people.json", "json") + +# SparkDataFrame can be saved as Parquet files, maintaining the schema information. +write.parquet(df, "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 DataFrame. +parquetFile <- read.parquet("people.parquet") + +# Parquet files can also be used to create a temporary view and then used in SQL statements. +createOrReplaceTempView(parquetFile, "parquetFile") +teenagers <- sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") +head(teenagers) +## name +## 1 Justin + +# We can also run custom R-UDFs on Spark DataFrames. Here we prefix all the names with "Name:" +schema <- structType(structField("name", "string")) +teenNames <- dapply(df, function(p) { cbind(paste("Name:", p$name)) }, schema) +for (teenName in collect(teenNames)$name) { + cat(teenName, "\n") +} +## Name: Michael +## Name: Andy +## Name: Justin +# $example off:basic_parquet_example$ + + +# $example on:schema_merging$ +df1 <- createDataFrame(data.frame(single=c(12, 29), double=c(19, 23))) +df2 <- createDataFrame(data.frame(double=c(19, 23), triple=c(23, 18))) + +# Create a simple DataFrame, stored into a partition directory +write.df(df1, "data/test_table/key=1", "parquet", "overwrite") + +# Create another DataFrame in a new partition directory, +# adding a new column and dropping an existing column +write.df(df2, "data/test_table/key=2", "parquet", "overwrite") + +# Read the partitioned table +df3 <- read.df("data/test_table", "parquet", mergeSchema = "true") +printSchema(df3) +# The final schema consists of all 3 columns in the Parquet files together +# with the partitioning column appeared in the partition directory paths +## root +## |-- single: double (nullable = true) +## |-- double: double (nullable = true) +## |-- triple: double (nullable = true) +## |-- key: integer (nullable = true) +# $example off:schema_merging$ + + +# $example on:json_dataset$ +# 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 DataFrame from the file(s) pointed to by path +people <- read.json(path) + +# The inferred schema can be visualized using the printSchema() method. +printSchema(people) +## root +## |-- age: long (nullable = true) +## |-- name: string (nullable = true) + +# Register this DataFrame as a table. +createOrReplaceTempView(people, "people") + +# SQL statements can be run by using the sql methods. +teenagers <- sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +head(teenagers) +## name +## 1 Justin +# $example off:json_dataset$ + + +# $example on:spark_hive$ +# enableHiveSupport defaults to TRUE +sparkR.session(enableHiveSupport = TRUE) +sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING) USING hive") +sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") + +# Queries can be expressed in HiveQL. +results <- collect(sql("FROM src SELECT key, value")) +# $example off:spark_hive$ + + +# $example on:jdbc_dataset$ +# Loading data from a JDBC source +df <- read.jdbc("jdbc:postgresql:dbserver", "schema.tablename", user = "username", password = "password") + +# Saving data to a JDBC source +write.jdbc(df, "jdbc:postgresql:dbserver", "schema.tablename", user = "username", password = "password") +# $example off:jdbc_dataset$ + +# Stop the SparkSession now +sparkR.session.stop() diff --git a/examples/src/main/r/data-manipulation.R b/examples/src/main/r/data-manipulation.R index aa2336e300a9..371335a62e92 100644 --- a/examples/src/main/r/data-manipulation.R +++ b/examples/src/main/r/data-manipulation.R @@ -17,11 +17,10 @@ # For this example, we shall use the "flights" dataset # The dataset consists of every flight departing Houston in 2011. -# The data set is made up of 227,496 rows x 14 columns. +# The data set is made up of 227,496 rows x 14 columns. # To run this example use -# ./bin/sparkR --packages com.databricks:spark-csv_2.10:1.0.3 -# examples/src/main/r/data-manipulation.R +# ./bin/spark-submit examples/src/main/r/data-manipulation.R # Load SparkR library into your R session library(SparkR) @@ -29,16 +28,13 @@ library(SparkR) args <- commandArgs(trailing = TRUE) if (length(args) != 1) { - print("Usage: data-manipulation.R ") + print("The data can be downloaded from: http://s3-us-west-2.amazonaws.com/sparkr-data/flights.csv") q("no") } -## Initialize SparkContext -sc <- sparkR.init(appName = "SparkR-data-manipulation-example") - -## Initialize SQLContext -sqlContext <- sparkRSQL.init(sc) +## Initialize SparkSession +sparkR.session(appName = "SparkR-data-manipulation-example") flightsCsvPath <- args[[1]] @@ -47,37 +43,37 @@ flights_df <- read.csv(flightsCsvPath, header = TRUE) flights_df$date <- as.Date(flights_df$date) ## Filter flights whose destination is San Francisco and write to a local data frame -SFO_df <- flights_df[flights_df$dest == "SFO", ] +SFO_df <- flights_df[flights_df$dest == "SFO", ] -# Convert the local data frame into a SparkR DataFrame -SFO_DF <- createDataFrame(sqlContext, SFO_df) +# Convert the local data frame into a SparkDataFrame +SFO_DF <- createDataFrame(SFO_df) -# Directly create a SparkR DataFrame from the source data -flightsDF <- read.df(sqlContext, flightsCsvPath, source = "com.databricks.spark.csv", header = "true") +# Directly create a SparkDataFrame from the source data +flightsDF <- read.df(flightsCsvPath, source = "csv", header = "true") -# Print the schema of this Spark DataFrame +# Print the schema of this SparkDataFrame printSchema(flightsDF) -# Cache the DataFrame +# Cache the SparkDataFrame cache(flightsDF) -# Print the first 6 rows of the DataFrame +# Print the first 6 rows of the SparkDataFrame showDF(flightsDF, numRows = 6) ## Or head(flightsDF) -# Show the column names in the DataFrame +# Show the column names in the SparkDataFrame columns(flightsDF) -# Show the number of rows in the DataFrame +# Show the number of rows in the SparkDataFrame count(flightsDF) # Select specific columns destDF <- select(flightsDF, "dest", "cancelled") # Using SQL to select columns of data -# First, register the flights DataFrame as a table -registerTempTable(flightsDF, "flightsTable") -destDF <- sql(sqlContext, "SELECT dest, cancelled FROM flightsTable") +# First, register the flights SparkDataFrame as a table +createOrReplaceTempView(flightsDF, "flightsTable") +destDF <- sql("SELECT dest, cancelled FROM flightsTable") # Use collect to create a local R data frame local_df <- collect(destDF) @@ -95,13 +91,13 @@ if("magrittr" %in% rownames(installed.packages())) { library(magrittr) # Group the flights by date and then find the average daily delay - # Write the result into a DataFrame + # Write the result into a SparkDataFrame groupBy(flightsDF, flightsDF$date) %>% summarize(avg(flightsDF$dep_delay), avg(flightsDF$arr_delay)) -> dailyDelayDF - # Print the computed data frame + # Print the computed SparkDataFrame head(dailyDelayDF) } -# Stop the SparkContext now -sparkR.stop() +# Stop the SparkSession now +sparkR.session.stop() diff --git a/examples/src/main/r/dataframe.R b/examples/src/main/r/dataframe.R index 53b817144f6a..311350497f87 100644 --- a/examples/src/main/r/dataframe.R +++ b/examples/src/main/r/dataframe.R @@ -15,17 +15,19 @@ # limitations under the License. # +# To run this example use +# ./bin/spark-submit examples/src/main/r/dataframe.R + library(SparkR) -# Initialize SparkContext and SQLContext -sc <- sparkR.init(appName="SparkR-DataFrame-example") -sqlContext <- sparkRSQL.init(sc) +# Initialize SparkSession +sparkR.session(appName = "SparkR-DataFrame-example") # Create a simple local data.frame localDF <- data.frame(name=c("John", "Smith", "Sarah"), age=c(19, 23, 18)) -# Convert local data frame to a SparkR DataFrame -df <- createDataFrame(sqlContext, localDF) +# Convert local data frame to a SparkDataFrame +df <- createDataFrame(localDF) # Print its schema printSchema(df) @@ -35,20 +37,23 @@ printSchema(df) # Create a DataFrame from a JSON file path <- file.path(Sys.getenv("SPARK_HOME"), "examples/src/main/resources/people.json") -peopleDF <- jsonFile(sqlContext, path) +peopleDF <- read.json(path) printSchema(peopleDF) +# root +# |-- age: long (nullable = true) +# |-- name: string (nullable = true) # Register this DataFrame as a table. -registerTempTable(peopleDF, "people") +createOrReplaceTempView(peopleDF, "people") -# SQL statements can be run by using the sql methods provided by sqlContext -teenagers <- sql(sqlContext, "SELECT name FROM people WHERE age >= 13 AND age <= 19") +# SQL statements can be run by using the sql methods +teenagers <- sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") # Call collect to get a local data.frame teenagersLocalDF <- collect(teenagers) -# Print the teenagers in our dataset +# Print the teenagers in our dataset print(teenagersLocalDF) -# Stop the SparkContext now -sparkR.stop() +# Stop the SparkSession now +sparkR.session.stop() diff --git a/examples/src/main/r/ml/als.R b/examples/src/main/r/ml/als.R new file mode 100644 index 000000000000..4d1c91add54e --- /dev/null +++ b/examples/src/main/r/ml/als.R @@ -0,0 +1,47 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/als.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-als-example") + +# $example on$ +# Load training data +data <- list(list(0, 0, 4.0), list(0, 1, 2.0), list(1, 1, 3.0), + list(1, 2, 4.0), list(2, 1, 1.0), list(2, 2, 5.0)) +df <- createDataFrame(data, c("userId", "movieId", "rating")) +training <- df +test <- df + +# Fit a recommendation model using ALS with spark.als +model <- spark.als(training, maxIter = 5, regParam = 0.01, userCol = "userId", + itemCol = "movieId", ratingCol = "rating") + +# Model summary +summary(model) + +# Prediction +predictions <- predict(model, test) +head(predictions) +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/bisectingKmeans.R b/examples/src/main/r/ml/bisectingKmeans.R new file mode 100644 index 000000000000..b3eaa6dd86d7 --- /dev/null +++ b/examples/src/main/r/ml/bisectingKmeans.R @@ -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. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/bisectingKmeans.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-bisectingKmeans-example") + +# $example on$ +t <- as.data.frame(Titanic) +training <- createDataFrame(t) + +# Fit bisecting k-means model with four centers +model <- spark.bisectingKmeans(training, Class ~ Survived, k = 4) + +# get fitted result from a bisecting k-means model +fitted.model <- fitted(model, "centers") + +# Model summary +head(summary(fitted.model)) + +# fitted values on training data +fitted <- predict(model, training) +head(select(fitted, "Class", "prediction")) +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/decisionTree.R b/examples/src/main/r/ml/decisionTree.R new file mode 100644 index 000000000000..9e10ae5519cd --- /dev/null +++ b/examples/src/main/r/ml/decisionTree.R @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/decisionTree.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-decisionTree-example") + +# DecisionTree classification model + +# $example on:classification$ +# Load training data +df <- read.df("data/mllib/sample_libsvm_data.txt", source = "libsvm") +training <- df +test <- df + +# Fit a DecisionTree classification model with spark.decisionTree +model <- spark.decisionTree(training, label ~ features, "classification") + +# Model summary +summary(model) + +# Prediction +predictions <- predict(model, test) +head(predictions) +# $example off:classification$ + +# DecisionTree regression model + +# $example on:regression$ +# Load training data +df <- read.df("data/mllib/sample_linear_regression_data.txt", source = "libsvm") +training <- df +test <- df + +# Fit a DecisionTree regression model with spark.decisionTree +model <- spark.decisionTree(training, label ~ features, "regression") + +# Model summary +summary(model) + +# Prediction +predictions <- predict(model, test) +head(predictions) +# $example off:regression$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/fpm.R b/examples/src/main/r/ml/fpm.R new file mode 100644 index 000000000000..89c4564457d9 --- /dev/null +++ b/examples/src/main/r/ml/fpm.R @@ -0,0 +1,50 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/fpm.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-fpm-example") + +# $example on$ +# Load training data + +df <- selectExpr(createDataFrame(data.frame(rawItems = c( + "1,2,5", "1,2,3,5", "1,2" +))), "split(rawItems, ',') AS items") + +fpm <- spark.fpGrowth(df, itemsCol="items", minSupport=0.5, minConfidence=0.6) + +# Extracting frequent itemsets + +spark.freqItemsets(fpm) + +# Extracting association rules + +spark.associationRules(fpm) + +# Predict uses association rules to and combines possible consequents + +predict(fpm, df) + +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/gaussianMixture.R b/examples/src/main/r/ml/gaussianMixture.R new file mode 100644 index 000000000000..558e44cc112e --- /dev/null +++ b/examples/src/main/r/ml/gaussianMixture.R @@ -0,0 +1,44 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/gaussianMixture.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-gaussianMixture-example") + +# $example on$ +# Load training data +df <- read.df("data/mllib/sample_kmeans_data.txt", source = "libsvm") +training <- df +test <- df + +# Fit a gaussian mixture clustering model with spark.gaussianMixture +model <- spark.gaussianMixture(training, ~ features, k = 2) + +# Model summary +summary(model) + +# Prediction +predictions <- predict(model, test) +head(predictions) +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/gbt.R b/examples/src/main/r/ml/gbt.R new file mode 100644 index 000000000000..bc654f1df7ab --- /dev/null +++ b/examples/src/main/r/ml/gbt.R @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/gbt.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-gbt-example") + +# GBT classification model + +# $example on:classification$ +# Load training data +df <- read.df("data/mllib/sample_libsvm_data.txt", source = "libsvm") +training <- df +test <- df + +# Fit a GBT classification model with spark.gbt +model <- spark.gbt(training, label ~ features, "classification", maxIter = 10) + +# Model summary +summary(model) + +# Prediction +predictions <- predict(model, test) +head(predictions) +# $example off:classification$ + +# GBT regression model + +# $example on:regression$ +# Load training data +df <- read.df("data/mllib/sample_linear_regression_data.txt", source = "libsvm") +training <- df +test <- df + +# Fit a GBT regression model with spark.gbt +model <- spark.gbt(training, label ~ features, "regression", maxIter = 10) + +# Model summary +summary(model) + +# Prediction +predictions <- predict(model, test) +head(predictions) +# $example off:regression$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/glm.R b/examples/src/main/r/ml/glm.R new file mode 100644 index 000000000000..68787f9aa9dc --- /dev/null +++ b/examples/src/main/r/ml/glm.R @@ -0,0 +1,71 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/glm.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-glm-example") + +# $example on$ +training <- read.df("data/mllib/sample_multiclass_classification_data.txt", source = "libsvm") +# Fit a generalized linear model of family "gaussian" with spark.glm +df_list <- randomSplit(training, c(7, 3), 2) +gaussianDF <- df_list[[1]] +gaussianTestDF <- df_list[[2]] +gaussianGLM <- spark.glm(gaussianDF, label ~ features, family = "gaussian") + +# Model summary +summary(gaussianGLM) + +# Prediction +gaussianPredictions <- predict(gaussianGLM, gaussianTestDF) +head(gaussianPredictions) + +# Fit a generalized linear model with glm (R-compliant) +gaussianGLM2 <- glm(label ~ features, gaussianDF, family = "gaussian") +summary(gaussianGLM2) + +# Fit a generalized linear model of family "binomial" with spark.glm +training2 <- read.df("data/mllib/sample_multiclass_classification_data.txt", source = "libsvm") +training2 <- transform(training2, label = cast(training2$label > 1, "integer")) +df_list2 <- randomSplit(training2, c(7, 3), 2) +binomialDF <- df_list2[[1]] +binomialTestDF <- df_list2[[2]] +binomialGLM <- spark.glm(binomialDF, label ~ features, family = "binomial") + +# Model summary +summary(binomialGLM) + +# Prediction +binomialPredictions <- predict(binomialGLM, binomialTestDF) +head(binomialPredictions) + +# Fit a generalized linear model of family "tweedie" with spark.glm +training3 <- read.df("data/mllib/sample_multiclass_classification_data.txt", source = "libsvm") +tweedieDF <- transform(training3, label = training3$label * exp(randn(10))) +tweedieGLM <- spark.glm(tweedieDF, label ~ features, family = "tweedie", + var.power = 1.2, link.power = 0) + +# Model summary +summary(tweedieGLM) +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/isoreg.R b/examples/src/main/r/ml/isoreg.R new file mode 100644 index 000000000000..a53c83eac430 --- /dev/null +++ b/examples/src/main/r/ml/isoreg.R @@ -0,0 +1,44 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/isoreg.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-isoreg-example") + +# $example on$ +# Load training data +df <- read.df("data/mllib/sample_isotonic_regression_libsvm_data.txt", source = "libsvm") +training <- df +test <- df + +# Fit an isotonic regression model with spark.isoreg +model <- spark.isoreg(training, label ~ features, isotonic = FALSE) + +# Model summary +summary(model) + +# Prediction +predictions <- predict(model, test) +head(predictions) +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/kmeans.R b/examples/src/main/r/ml/kmeans.R new file mode 100644 index 000000000000..824df20644fa --- /dev/null +++ b/examples/src/main/r/ml/kmeans.R @@ -0,0 +1,48 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/kmeans.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-kmeans-example") + +# $example on$ +# Fit a k-means model with spark.kmeans +t <- as.data.frame(Titanic) +training <- createDataFrame(t) +df_list <- randomSplit(training, c(7,3), 2) +kmeansDF <- df_list[[1]] +kmeansTestDF <- df_list[[2]] +kmeansModel <- spark.kmeans(kmeansDF, ~ Class + Sex + Age + Freq, + k = 3) + +# Model summary +summary(kmeansModel) + +# Get fitted result from the k-means model +head(fitted(kmeansModel)) + +# Prediction +kmeansPredictions <- predict(kmeansModel, kmeansTestDF) +head(kmeansPredictions) +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/kstest.R b/examples/src/main/r/ml/kstest.R new file mode 100644 index 000000000000..e2b07702b6f3 --- /dev/null +++ b/examples/src/main/r/ml/kstest.R @@ -0,0 +1,41 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/kstest.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-kstest-example") + +# $example on$ +# Load training data +data <- data.frame(test = c(0.1, 0.15, 0.2, 0.3, 0.25, -1, -0.5)) +df <- createDataFrame(data) +training <- df +test <- df + +# Conduct the two-sided Kolmogorov-Smirnov (KS) test with spark.kstest +model <- spark.kstest(df, "test", "norm") + +# Model summary +summary(model) +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/lda.R b/examples/src/main/r/ml/lda.R new file mode 100644 index 000000000000..769be0a78dfb --- /dev/null +++ b/examples/src/main/r/ml/lda.R @@ -0,0 +1,48 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/lda.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-lda-example") + +# $example on$ +# Load training data +df <- read.df("data/mllib/sample_lda_libsvm_data.txt", source = "libsvm") +training <- df +test <- df + +# Fit a latent dirichlet allocation model with spark.lda +model <- spark.lda(training, k = 10, maxIter = 10) + +# Model summary +summary(model) + +# Posterior probabilities +posterior <- spark.posterior(model, test) +head(posterior) + +# The log perplexity of the LDA model +logPerplexity <- spark.perplexity(model, test) +print(paste0("The upper bound bound on perplexity: ", logPerplexity)) +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/logit.R b/examples/src/main/r/ml/logit.R new file mode 100644 index 000000000000..4c8fd428d385 --- /dev/null +++ b/examples/src/main/r/ml/logit.R @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/logit.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-logit-example") + +# Binomial logistic regression + +# $example on:binomial$ +# Load training data +df <- read.df("data/mllib/sample_libsvm_data.txt", source = "libsvm") +training <- df +test <- df + +# Fit an binomial logistic regression model with spark.logit +model <- spark.logit(training, label ~ features, maxIter = 10, regParam = 0.3, elasticNetParam = 0.8) + +# Model summary +summary(model) + +# Prediction +predictions <- predict(model, test) +head(predictions) +# $example off:binomial$ + +# Multinomial logistic regression + +# $example on:multinomial$ +# Load training data +df <- read.df("data/mllib/sample_multiclass_classification_data.txt", source = "libsvm") +training <- df +test <- df + +# Fit a multinomial logistic regression model with spark.logit +model <- spark.logit(training, label ~ features, maxIter = 10, regParam = 0.3, elasticNetParam = 0.8) + +# Model summary +summary(model) + +# Prediction +predictions <- predict(model, test) +head(predictions) +# $example off:multinomial$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/ml.R b/examples/src/main/r/ml/ml.R new file mode 100644 index 000000000000..41b7867f64e3 --- /dev/null +++ b/examples/src/main/r/ml/ml.R @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/ml.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-example") + +############################ model read/write ############################################## +# $example on:read_write$ +training <- read.df("data/mllib/sample_multiclass_classification_data.txt", source = "libsvm") +# Fit a generalized linear model of family "gaussian" with spark.glm +df_list <- randomSplit(training, c(7,3), 2) +gaussianDF <- df_list[[1]] +gaussianTestDF <- df_list[[2]] +gaussianGLM <- spark.glm(gaussianDF, label ~ features, family = "gaussian") + +# Save and then load a fitted MLlib model +modelPath <- tempfile(pattern = "ml", fileext = ".tmp") +write.ml(gaussianGLM, modelPath) +gaussianGLM2 <- read.ml(modelPath) + +# Check model summary +summary(gaussianGLM2) + +# Check model prediction +gaussianPredictions <- predict(gaussianGLM2, gaussianTestDF) +head(gaussianPredictions) + +unlink(modelPath) +# $example off:read_write$ + +############################ fit models with spark.lapply ##################################### +# Perform distributed training of multiple models with spark.lapply +algorithms <- c("Hartigan-Wong", "Lloyd", "MacQueen") +train <- function(algorithm) { + model <- kmeans(x = iris[1:4], centers = 3, algorithm = algorithm) + model$withinss +} + +model.withinss <- spark.lapply(algorithms, train) + +# Print the within-cluster sum of squares for each model +print(model.withinss) + +# Stop the SparkSession now +sparkR.session.stop() diff --git a/examples/src/main/r/ml/mlp.R b/examples/src/main/r/ml/mlp.R new file mode 100644 index 000000000000..b69ac845f2db --- /dev/null +++ b/examples/src/main/r/ml/mlp.R @@ -0,0 +1,50 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/mlp.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-mlp-example") + +# $example on$ +# Load training data +df <- read.df("data/mllib/sample_multiclass_classification_data.txt", source = "libsvm") +training <- df +test <- df + +# specify layers for the neural network: +# input layer of size 4 (features), two intermediate of size 5 and 4 +# and output of size 3 (classes) +layers = c(4, 5, 4, 3) + +# Fit a multi-layer perceptron neural network model with spark.mlp +model <- spark.mlp(training, label ~ features, maxIter = 100, + layers = layers, blockSize = 128, seed = 1234) + +# Model summary +summary(model) + +# Prediction +predictions <- predict(model, test) +head(predictions) +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/naiveBayes.R b/examples/src/main/r/ml/naiveBayes.R new file mode 100644 index 000000000000..da69e93ef294 --- /dev/null +++ b/examples/src/main/r/ml/naiveBayes.R @@ -0,0 +1,43 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/naiveBayes.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-naiveBayes-example") + +# $example on$ +# Fit a Bernoulli naive Bayes model with spark.naiveBayes +titanic <- as.data.frame(Titanic) +titanicDF <- createDataFrame(titanic[titanic$Freq > 0, -5]) +nbDF <- titanicDF +nbTestDF <- titanicDF +nbModel <- spark.naiveBayes(nbDF, Survived ~ Class + Sex + Age) + +# Model summary +summary(nbModel) + +# Prediction +nbPredictions <- predict(nbModel, nbTestDF) +head(nbPredictions) +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/randomForest.R b/examples/src/main/r/ml/randomForest.R new file mode 100644 index 000000000000..5d99502cd971 --- /dev/null +++ b/examples/src/main/r/ml/randomForest.R @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/randomForest.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-randomForest-example") + +# Random forest classification model + +# $example on:classification$ +# Load training data +df <- read.df("data/mllib/sample_libsvm_data.txt", source = "libsvm") +training <- df +test <- df + +# Fit a random forest classification model with spark.randomForest +model <- spark.randomForest(training, label ~ features, "classification", numTrees = 10) + +# Model summary +summary(model) + +# Prediction +predictions <- predict(model, test) +head(predictions) +# $example off:classification$ + +# Random forest regression model + +# $example on:regression$ +# Load training data +df <- read.df("data/mllib/sample_linear_regression_data.txt", source = "libsvm") +training <- df +test <- df + +# Fit a random forest regression model with spark.randomForest +model <- spark.randomForest(training, label ~ features, "regression", numTrees = 10) + +# Model summary +summary(model) + +# Prediction +predictions <- predict(model, test) +head(predictions) +# $example off:regression$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/survreg.R b/examples/src/main/r/ml/survreg.R new file mode 100644 index 000000000000..e4eadfca86f6 --- /dev/null +++ b/examples/src/main/r/ml/survreg.R @@ -0,0 +1,46 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/survreg.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-survreg-example") + +# $example on$ +# Use the ovarian dataset available in R survival package +library(survival) + +# Fit an accelerated failure time (AFT) survival regression model with spark.survreg +ovarianDF <- suppressWarnings(createDataFrame(ovarian)) +aftDF <- ovarianDF +aftTestDF <- ovarianDF +aftModel <- spark.survreg(aftDF, Surv(futime, fustat) ~ ecog_ps + rx) + +# Model summary +summary(aftModel) + +# Prediction +aftPredictions <- predict(aftModel, aftTestDF) +head(aftPredictions) +# $example off$ + +sparkR.session.stop() + diff --git a/examples/src/main/r/ml/svmLinear.R b/examples/src/main/r/ml/svmLinear.R new file mode 100644 index 000000000000..c632f1282ea7 --- /dev/null +++ b/examples/src/main/r/ml/svmLinear.R @@ -0,0 +1,42 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# To run this example use +# ./bin/spark-submit examples/src/main/r/ml/svmLinear.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-svmLinear-example") + +# $example on$ +# load training data +t <- as.data.frame(Titanic) +training <- createDataFrame(t) + +# fit Linear SVM model +model <- spark.svmLinear(training, Survived ~ ., regParam = 0.01, maxIter = 10) + +# Model summary +summary(model) + +# Prediction +prediction <- predict(model, training) +showDF(prediction) +# $example off$ +sparkR.session.stop() diff --git a/examples/src/main/r/streaming/structured_network_wordcount.R b/examples/src/main/r/streaming/structured_network_wordcount.R new file mode 100644 index 000000000000..cda18ebc072e --- /dev/null +++ b/examples/src/main/r/streaming/structured_network_wordcount.R @@ -0,0 +1,57 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Counts words in UTF8 encoded, '\n' delimited text received from the network. + +# To run this on your local machine, you need to first run a Netcat server +# $ nc -lk 9999 +# and then run the example +# ./bin/spark-submit examples/src/main/r/streaming/structured_network_wordcount.R localhost 9999 + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-Streaming-structured-network-wordcount-example") + +args <- commandArgs(trailing = TRUE) + +if (length(args) != 2) { + print("Usage: structured_network_wordcount.R ") + print(" and describe the TCP server that Structured Streaming") + print("would connect to receive data.") + q("no") +} + +hostname <- args[[1]] +port <- as.integer(args[[2]]) + +# Create DataFrame representing the stream of input lines from connection to localhost:9999 +lines <- read.stream("socket", host = hostname, port = port) + +# Split the lines into words +words <- selectExpr(lines, "explode(split(value, ' ')) as word") + +# Generate running word count +wordCounts <- count(groupBy(words, "word")) + +# Start running the query that prints the running counts to the console +query <- write.stream(wordCounts, "console", outputMode = "complete") + +awaitTermination(query) + +sparkR.session.stop() diff --git a/examples/src/main/resources/employees.json b/examples/src/main/resources/employees.json new file mode 100644 index 000000000000..6b2e6329a1cb --- /dev/null +++ b/examples/src/main/resources/employees.json @@ -0,0 +1,4 @@ +{"name":"Michael", "salary":3000} +{"name":"Andy", "salary":4500} +{"name":"Justin", "salary":3500} +{"name":"Berta", "salary":4000} diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index d812262fd87d..25718f904cc4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -18,21 +18,23 @@ // scalastyle:off println package org.apache.spark.examples -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** - * Usage: BroadcastTest [slices] [numElem] [broadcastAlgo] [blockSize] - */ + * Usage: BroadcastTest [partitions] [numElem] [blockSize] + */ object BroadcastTest { def main(args: Array[String]) { - val bcName = if (args.length > 2) args(2) else "Http" - val blockSize = if (args.length > 3) args(3) else "4096" + val blockSize = if (args.length > 2) args(2) else "4096" + + val spark = SparkSession + .builder() + .appName("Broadcast Test") + .config("spark.broadcast.blockSize", blockSize) + .getOrCreate() - val sparkConf = new SparkConf().setAppName("Broadcast Test") - .set("spark.broadcast.factory", s"org.apache.spark.broadcast.${bcName}BroadcastFactory") - .set("spark.broadcast.blockSize", blockSize) - val sc = new SparkContext(sparkConf) + val sc = spark.sparkContext val slices = if (args.length > 0) args(0).toInt else 2 val num = if (args.length > 1) args(1).toInt else 1000000 @@ -44,13 +46,13 @@ object BroadcastTest { println("===========") val startTime = System.nanoTime val barr1 = sc.broadcast(arr1) - val observedSizes = sc.parallelize(1 to 10, slices).map(_ => barr1.value.size) + val observedSizes = sc.parallelize(1 to 10, slices).map(_ => barr1.value.length) // Collect the small RDD so we can print the observed sizes locally. observedSizes.collect().foreach(i => println(i)) println("Iteration %d took %.0f milliseconds".format(i, (System.nanoTime - startTime) / 1E6)) } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala deleted file mode 100644 index d1b9b8d398dd..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala +++ /dev/null @@ -1,140 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - // scalastyle:off println - // scalastyle:off jobcontext -package org.apache.spark.examples - -import java.nio.ByteBuffer -import java.util.Collections - -import org.apache.cassandra.hadoop.ConfigHelper -import org.apache.cassandra.hadoop.cql3.CqlPagingInputFormat -import org.apache.cassandra.hadoop.cql3.CqlConfigHelper -import org.apache.cassandra.hadoop.cql3.CqlOutputFormat -import org.apache.cassandra.utils.ByteBufferUtil -import org.apache.hadoop.mapreduce.Job - -import org.apache.spark.{SparkConf, SparkContext} - - -/* - Need to create following keyspace and column family in cassandra before running this example - Start CQL shell using ./bin/cqlsh and execute following commands - CREATE KEYSPACE retail WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; - use retail; - CREATE TABLE salecount (prod_id text, sale_count int, PRIMARY KEY (prod_id)); - CREATE TABLE ordercf (user_id text, - time timestamp, - prod_id text, - quantity int, - PRIMARY KEY (user_id, time)); - INSERT INTO ordercf (user_id, - time, - prod_id, - quantity) VALUES ('bob', 1385983646000, 'iphone', 1); - INSERT INTO ordercf (user_id, - time, - prod_id, - quantity) VALUES ('tom', 1385983647000, 'samsung', 4); - INSERT INTO ordercf (user_id, - time, - prod_id, - quantity) VALUES ('dora', 1385983648000, 'nokia', 2); - INSERT INTO ordercf (user_id, - time, - prod_id, - quantity) VALUES ('charlie', 1385983649000, 'iphone', 2); -*/ - -/** - * This example demonstrates how to read and write to cassandra column family created using CQL3 - * using Spark. - * Parameters : - * Usage: ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.CassandraCQLTest localhost 9160 - */ -object CassandraCQLTest { - - def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("CQLTestApp") - - val sc = new SparkContext(sparkConf) - val cHost: String = args(0) - val cPort: String = args(1) - val KeySpace = "retail" - val InputColumnFamily = "ordercf" - val OutputColumnFamily = "salecount" - - val job = new Job() - job.setInputFormatClass(classOf[CqlPagingInputFormat]) - val configuration = job.getConfiguration - ConfigHelper.setInputInitialAddress(job.getConfiguration(), cHost) - ConfigHelper.setInputRpcPort(job.getConfiguration(), cPort) - ConfigHelper.setInputColumnFamily(job.getConfiguration(), KeySpace, InputColumnFamily) - ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner") - CqlConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), "3") - - /** CqlConfigHelper.setInputWhereClauses(job.getConfiguration(), "user_id='bob'") */ - - /** An UPDATE writes one or more columns to a record in a Cassandra column family */ - val query = "UPDATE " + KeySpace + "." + OutputColumnFamily + " SET sale_count = ? " - CqlConfigHelper.setOutputCql(job.getConfiguration(), query) - - job.setOutputFormatClass(classOf[CqlOutputFormat]) - ConfigHelper.setOutputColumnFamily(job.getConfiguration(), KeySpace, OutputColumnFamily) - ConfigHelper.setOutputInitialAddress(job.getConfiguration(), cHost) - ConfigHelper.setOutputRpcPort(job.getConfiguration(), cPort) - ConfigHelper.setOutputPartitioner(job.getConfiguration(), "Murmur3Partitioner") - - val casRdd = sc.newAPIHadoopRDD(job.getConfiguration(), - classOf[CqlPagingInputFormat], - classOf[java.util.Map[String, ByteBuffer]], - classOf[java.util.Map[String, ByteBuffer]]) - - println("Count: " + casRdd.count) - val productSaleRDD = casRdd.map { - case (key, value) => { - (ByteBufferUtil.string(value.get("prod_id")), ByteBufferUtil.toInt(value.get("quantity"))) - } - } - val aggregatedRDD = productSaleRDD.reduceByKey(_ + _) - aggregatedRDD.collect().foreach { - case (productId, saleCount) => println(productId + ":" + saleCount) - } - - val casoutputCF = aggregatedRDD.map { - case (productId, saleCount) => { - val outKey = Collections.singletonMap("prod_id", ByteBufferUtil.bytes(productId)) - val outVal = Collections.singletonList(ByteBufferUtil.bytes(saleCount)) - (outKey, outVal) - } - } - - casoutputCF.saveAsNewAPIHadoopFile( - KeySpace, - classOf[java.util.Map[String, ByteBuffer]], - classOf[java.util.List[ByteBuffer]], - classOf[CqlOutputFormat], - job.getConfiguration() - ) - - sc.stop() - } -} -// scalastyle:on println -// scalastyle:on jobcontext diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala deleted file mode 100644 index 1e679bfb5534..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala +++ /dev/null @@ -1,219 +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. - */ - -// scalastyle:off println -// scalastyle:off jobcontext -package org.apache.spark.examples - -import java.nio.ByteBuffer -import java.util.Arrays -import java.util.SortedMap - -import org.apache.cassandra.db.IColumn -import org.apache.cassandra.hadoop.ColumnFamilyOutputFormat -import org.apache.cassandra.hadoop.ConfigHelper -import org.apache.cassandra.hadoop.ColumnFamilyInputFormat -import org.apache.cassandra.thrift._ -import org.apache.cassandra.utils.ByteBufferUtil -import org.apache.hadoop.mapreduce.Job - -import org.apache.spark.{SparkConf, SparkContext} - -/* - * This example demonstrates using Spark with Cassandra with the New Hadoop API and Cassandra - * support for Hadoop. - * - * To run this example, run this file with the following command params - - * - * - * So if you want to run this on localhost this will be, - * localhost 9160 - * - * The example makes some assumptions: - * 1. You have already created a keyspace called casDemo and it has a column family named Words - * 2. There are column family has a column named "para" which has test content. - * - * You can create the content by running the following script at the bottom of this file with - * cassandra-cli. - * - */ -object CassandraTest { - - def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("casDemo") - // Get a SparkContext - val sc = new SparkContext(sparkConf) - - // Build the job configuration with ConfigHelper provided by Cassandra - val job = new Job() - job.setInputFormatClass(classOf[ColumnFamilyInputFormat]) - - val host: String = args(1) - val port: String = args(2) - - ConfigHelper.setInputInitialAddress(job.getConfiguration(), host) - ConfigHelper.setInputRpcPort(job.getConfiguration(), port) - ConfigHelper.setOutputInitialAddress(job.getConfiguration(), host) - ConfigHelper.setOutputRpcPort(job.getConfiguration(), port) - ConfigHelper.setInputColumnFamily(job.getConfiguration(), "casDemo", "Words") - ConfigHelper.setOutputColumnFamily(job.getConfiguration(), "casDemo", "WordCount") - - val predicate = new SlicePredicate() - val sliceRange = new SliceRange() - sliceRange.setStart(Array.empty[Byte]) - sliceRange.setFinish(Array.empty[Byte]) - predicate.setSlice_range(sliceRange) - ConfigHelper.setInputSlicePredicate(job.getConfiguration(), predicate) - - ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner") - ConfigHelper.setOutputPartitioner(job.getConfiguration(), "Murmur3Partitioner") - - // Make a new Hadoop RDD - val casRdd = sc.newAPIHadoopRDD( - job.getConfiguration(), - classOf[ColumnFamilyInputFormat], - classOf[ByteBuffer], - classOf[SortedMap[ByteBuffer, IColumn]]) - - // Let us first get all the paragraphs from the retrieved rows - val paraRdd = casRdd.map { - case (key, value) => { - ByteBufferUtil.string(value.get(ByteBufferUtil.bytes("para")).value()) - } - } - - // Lets get the word count in paras - val counts = paraRdd.flatMap(p => p.split(" ")).map(word => (word, 1)).reduceByKey(_ + _) - - counts.collect().foreach { - case (word, count) => println(word + ":" + count) - } - - counts.map { - case (word, count) => { - val colWord = new org.apache.cassandra.thrift.Column() - colWord.setName(ByteBufferUtil.bytes("word")) - colWord.setValue(ByteBufferUtil.bytes(word)) - colWord.setTimestamp(System.currentTimeMillis) - - val colCount = new org.apache.cassandra.thrift.Column() - colCount.setName(ByteBufferUtil.bytes("wcount")) - colCount.setValue(ByteBufferUtil.bytes(count.toLong)) - colCount.setTimestamp(System.currentTimeMillis) - - val outputkey = ByteBufferUtil.bytes(word + "-COUNT-" + System.currentTimeMillis) - - val mutations = Arrays.asList(new Mutation(), new Mutation()) - mutations.get(0).setColumn_or_supercolumn(new ColumnOrSuperColumn()) - mutations.get(0).column_or_supercolumn.setColumn(colWord) - mutations.get(1).setColumn_or_supercolumn(new ColumnOrSuperColumn()) - mutations.get(1).column_or_supercolumn.setColumn(colCount) - (outputkey, mutations) - } - }.saveAsNewAPIHadoopFile("casDemo", classOf[ByteBuffer], classOf[List[Mutation]], - classOf[ColumnFamilyOutputFormat], job.getConfiguration) - - sc.stop() - } -} -// scalastyle:on println -// scalastyle:on jobcontext - -/* -create keyspace casDemo; -use casDemo; - -create column family WordCount with comparator = UTF8Type; -update column family WordCount with column_metadata = - [{column_name: word, validation_class: UTF8Type}, - {column_name: wcount, validation_class: LongType}]; - -create column family Words with comparator = UTF8Type; -update column family Words with column_metadata = - [{column_name: book, validation_class: UTF8Type}, - {column_name: para, validation_class: UTF8Type}]; - -assume Words keys as utf8; - -set Words['3musk001']['book'] = 'The Three Musketeers'; -set Words['3musk001']['para'] = 'On the first Monday of the month of April, 1625, the market - town of Meung, in which the author of ROMANCE OF THE ROSE was born, appeared to - be in as perfect a state of revolution as if the Huguenots had just made - a second La Rochelle of it. Many citizens, seeing the women flying - toward the High Street, leaving their children crying at the open doors, - hastened to don the cuirass, and supporting their somewhat uncertain - courage with a musket or a partisan, directed their steps toward the - hostelry of the Jolly Miller, before which was gathered, increasing - every minute, a compact group, vociferous and full of curiosity.'; - -set Words['3musk002']['book'] = 'The Three Musketeers'; -set Words['3musk002']['para'] = 'In those times panics were common, and few days passed without - some city or other registering in its archives an event of this kind. There were - nobles, who made war against each other; there was the king, who made - war against the cardinal; there was Spain, which made war against the - king. Then, in addition to these concealed or public, secret or open - wars, there were robbers, mendicants, Huguenots, wolves, and scoundrels, - who made war upon everybody. The citizens always took up arms readily - against thieves, wolves or scoundrels, often against nobles or - Huguenots, sometimes against the king, but never against cardinal or - Spain. It resulted, then, from this habit that on the said first Monday - of April, 1625, the citizens, on hearing the clamor, and seeing neither - the red-and-yellow standard nor the livery of the Duc de Richelieu, - rushed toward the hostel of the Jolly Miller. When arrived there, the - cause of the hubbub was apparent to all'; - -set Words['3musk003']['book'] = 'The Three Musketeers'; -set Words['3musk003']['para'] = 'You ought, I say, then, to husband the means you have, however - large the sum may be; but you ought also to endeavor to perfect yourself in - the exercises becoming a gentleman. I will write a letter today to the - Director of the Royal Academy, and tomorrow he will admit you without - any expense to yourself. Do not refuse this little service. Our - best-born and richest gentlemen sometimes solicit it without being able - to obtain it. You will learn horsemanship, swordsmanship in all its - branches, and dancing. You will make some desirable acquaintances; and - from time to time you can call upon me, just to tell me how you are - getting on, and to say whether I can be of further service to you.'; - - -set Words['thelostworld001']['book'] = 'The Lost World'; -set Words['thelostworld001']['para'] = 'She sat with that proud, delicate profile of hers outlined - against the red curtain. How beautiful she was! And yet how aloof! We had been - friends, quite good friends; but never could I get beyond the same - comradeship which I might have established with one of my - fellow-reporters upon the Gazette,--perfectly frank, perfectly kindly, - and perfectly unsexual. My instincts are all against a woman being too - frank and at her ease with me. It is no compliment to a man. Where - the real sex feeling begins, timidity and distrust are its companions, - heritage from old wicked days when love and violence went often hand in - hand. The bent head, the averted eye, the faltering voice, the wincing - figure--these, and not the unshrinking gaze and frank reply, are the - true signals of passion. Even in my short life I had learned as much - as that--or had inherited it in that race memory which we call instinct.'; - -set Words['thelostworld002']['book'] = 'The Lost World'; -set Words['thelostworld002']['para'] = 'I always liked McArdle, the crabbed, old, round-backed, - red-headed news editor, and I rather hoped that he liked me. Of course, Beaumont was - the real boss; but he lived in the rarefied atmosphere of some Olympian - height from which he could distinguish nothing smaller than an - international crisis or a split in the Cabinet. Sometimes we saw him - passing in lonely majesty to his inner sanctum, with his eyes staring - vaguely and his mind hovering over the Balkans or the Persian Gulf. He - was above and beyond us. But McArdle was his first lieutenant, and it - was he that we knew. The old man nodded as I entered the room, and he - pushed his spectacles far up on his bald forehead.'; - -*/ diff --git a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala index d651fe4d6ee7..3bff7ce736d0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala @@ -22,20 +22,19 @@ import java.io.File import scala.io.Source._ -import org.apache.spark.{SparkContext, SparkConf} -import org.apache.spark.SparkContext._ +import org.apache.spark.sql.SparkSession /** - * Simple test for reading and writing to a distributed - * file system. This example does the following: - * - * 1. Reads local file - * 2. Computes word count on local file - * 3. Writes local file to a DFS - * 4. Reads the file back from the DFS - * 5. Computes word count on the file using Spark - * 6. Compares the word count results - */ + * Simple test for reading and writing to a distributed + * file system. This example does the following: + * + * 1. Reads local file + * 2. Computes word count on local file + * 3. Writes local file to a DFS + * 4. Reads the file back from the DFS + * 5. Computes word count on the file using Spark + * 6. Compares the word count results + */ object DFSReadWriteTest { private var localFilePath: File = new File(".") @@ -88,7 +87,7 @@ object DFSReadWriteTest { def runLocalWordCount(fileContents: List[String]): Int = { fileContents.flatMap(_.split(" ")) .flatMap(_.split("\t")) - .filter(_.size > 0) + .filter(_.nonEmpty) .groupBy(w => w) .mapValues(_.size) .values @@ -102,30 +101,30 @@ object DFSReadWriteTest { val fileContents = readFile(localFilePath.toString()) val localWordCount = runLocalWordCount(fileContents) - println("Creating SparkConf") - val conf = new SparkConf().setAppName("DFS Read Write Test") - - println("Creating SparkContext") - val sc = new SparkContext(conf) + println("Creating SparkSession") + val spark = SparkSession + .builder + .appName("DFS Read Write Test") + .getOrCreate() println("Writing local file to DFS") val dfsFilename = dfsDirPath + "/dfs_read_write_test" - val fileRDD = sc.parallelize(fileContents) + val fileRDD = spark.sparkContext.parallelize(fileContents) fileRDD.saveAsTextFile(dfsFilename) println("Reading file from DFS and running Word Count") - val readFileRDD = sc.textFile(dfsFilename) + val readFileRDD = spark.sparkContext.textFile(dfsFilename) val dfsWordCount = readFileRDD .flatMap(_.split(" ")) .flatMap(_.split("\t")) - .filter(_.size > 0) + .filter(_.nonEmpty) .map(w => (w, 1)) .countByKey() .values .sum - sc.stop() + spark.stop() if (localWordCount == dfsWordCount) { println(s"Success! Local Word Count ($localWordCount) " + diff --git a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala index bec61f3cd429..d12ef642bd2c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala @@ -22,11 +22,13 @@ import scala.collection.JavaConverters._ import org.apache.spark.util.Utils -/** Prints out environmental information, sleeps, and then exits. Made to - * test driver submission in the standalone scheduler. */ +/** + * Prints out environmental information, sleeps, and then exits. Made to + * test driver submission in the standalone scheduler. + */ object DriverSubmissionTest { def main(args: Array[String]) { - if (args.size < 1) { + if (args.length < 1) { println("Usage: DriverSubmissionTest ") System.exit(0) } diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index d42f63e87052..45c4953a84be 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -17,18 +17,21 @@ package org.apache.spark.examples -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession object ExceptionHandlingTest { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("ExceptionHandlingTest") - val sc = new SparkContext(sparkConf) - sc.parallelize(0 until sc.defaultParallelism).foreach { i => + val spark = SparkSession + .builder + .appName("ExceptionHandlingTest") + .getOrCreate() + + spark.sparkContext.parallelize(0 until spark.sparkContext.defaultParallelism).foreach { i => if (math.random > 0.75) { throw new Exception("Testing exception handling") } } - sc.stop() + spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index fa4a3afeecd1..2f2bbb127543 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -20,25 +20,26 @@ package org.apache.spark.examples import java.util.Random -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ +import org.apache.spark.sql.SparkSession /** - * Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers] - */ + * Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers] + */ object GroupByTest { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("GroupBy Test") - var numMappers = if (args.length > 0) args(0).toInt else 2 - var numKVPairs = if (args.length > 1) args(1).toInt else 1000 - var valSize = if (args.length > 2) args(2).toInt else 1000 - var numReducers = if (args.length > 3) args(3).toInt else numMappers + val spark = SparkSession + .builder + .appName("GroupBy Test") + .getOrCreate() - val sc = new SparkContext(sparkConf) + val numMappers = if (args.length > 0) args(0).toInt else 2 + val numKVPairs = if (args.length > 1) args(1).toInt else 1000 + val valSize = if (args.length > 2) args(2).toInt else 1000 + val numReducers = if (args.length > 3) args(3).toInt else numMappers - val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => + val pairs1 = spark.sparkContext.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random - var arr1 = new Array[(Int, Array[Byte])](numKVPairs) + val arr1 = new Array[(Int, Array[Byte])](numKVPairs) for (i <- 0 until numKVPairs) { val byteArr = new Array[Byte](valSize) ranGen.nextBytes(byteArr) @@ -51,7 +52,7 @@ object GroupByTest { println(pairs1.groupByKey(numReducers).count()) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala deleted file mode 100644 index 244742327a90..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -// scalastyle:off println -package org.apache.spark.examples - -import org.apache.hadoop.hbase.client.HBaseAdmin -import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor, TableName} -import org.apache.hadoop.hbase.mapreduce.TableInputFormat - -import org.apache.spark._ - - -object HBaseTest { - def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("HBaseTest") - val sc = new SparkContext(sparkConf) - - // please ensure HBASE_CONF_DIR is on classpath of spark driver - // e.g: set it through spark.driver.extraClassPath property - // in spark-defaults.conf or through --driver-class-path - // command line option of spark-submit - - val conf = HBaseConfiguration.create() - - if (args.length < 1) { - System.err.println("Usage: HBaseTest ") - System.exit(1) - } - - // Other options for configuring scan behavior are available. More information available at - // http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html - conf.set(TableInputFormat.INPUT_TABLE, args(0)) - - // Initialize hBase table if necessary - val admin = new HBaseAdmin(conf) - if (!admin.isTableAvailable(args(0))) { - val tableDesc = new HTableDescriptor(TableName.valueOf(args(0))) - admin.createTable(tableDesc) - } - - val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], - classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], - classOf[org.apache.hadoop.hbase.client.Result]) - - hBaseRDD.count() - - sc.stop() - admin.close() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index 124dc9af6390..aa8de69839e2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -18,7 +18,7 @@ // scalastyle:off println package org.apache.spark.examples -import org.apache.spark._ +import org.apache.spark.sql.SparkSession object HdfsTest { @@ -29,9 +29,11 @@ object HdfsTest { System.err.println("Usage: HdfsTest ") System.exit(1) } - val sparkConf = new SparkConf().setAppName("HdfsTest") - val sc = new SparkContext(sparkConf) - val file = sc.textFile(args(0)) + val spark = SparkSession + .builder + .appName("HdfsTest") + .getOrCreate() + val file = spark.read.text(args(0)).rdd val mapped = file.map(s => s.length).cache() for (iter <- 1 to 10) { val start = System.currentTimeMillis() @@ -39,7 +41,7 @@ object HdfsTest { val end = System.currentTimeMillis() println("Iteration " + iter + " took " + (end-start) + " ms") } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index af5f216f28ba..97aefac025e5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -24,7 +24,7 @@ import org.apache.commons.math3.linear._ * Alternating least squares matrix factorization. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.recommendation.ALS + * please refer to org.apache.spark.ml.recommendation.ALS. */ object LocalALS { @@ -96,7 +96,7 @@ object LocalALS { def showWarning() { System.err.println( """WARN: This is a naive implementation of ALS and is given as an example! - |Please use the ALS method found in org.apache.spark.mllib.recommendation + |Please use org.apache.spark.ml.recommendation.ALS |for more conventional use. """.stripMargin) } @@ -104,16 +104,14 @@ object LocalALS { def main(args: Array[String]) { args match { - case Array(m, u, f, iters) => { + case Array(m, u, f, iters) => M = m.toInt U = u.toInt F = f.toInt ITERATIONS = iters.toInt - } - case _ => { + case _ => System.err.println("Usage: LocalALS ") System.exit(1) - } } showWarning() diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala index 9c8aae53cf48..8dbb7ee4e530 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala @@ -20,17 +20,16 @@ package org.apache.spark.examples import java.util.Random -import breeze.linalg.{Vector, DenseVector} +import breeze.linalg.{DenseVector, Vector} /** * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object LocalFileLR { - val D = 10 // Numer of dimensions + val D = 10 // Number of dimensions val rand = new Random(42) case class DataPoint(x: Vector[Double], y: Double) @@ -43,8 +42,7 @@ object LocalFileLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } @@ -53,17 +51,18 @@ object LocalFileLR { showWarning() - val lines = scala.io.Source.fromFile(args(0)).getLines().toArray - val points = lines.map(parsePoint _) + val fileSrc = scala.io.Source.fromFile(args(0)) + val lines = fileSrc.getLines().toArray + val points = lines.map(parsePoint) val ITERATIONS = args(1).toInt // Initialize w to a random value - var w = DenseVector.fill(D){2 * rand.nextDouble - 1} + val w = DenseVector.fill(D) {2 * rand.nextDouble - 1} println("Initial w: " + w) for (i <- 1 to ITERATIONS) { println("On iteration " + i) - var gradient = DenseVector.zeros[Double](D) + val gradient = DenseVector.zeros[Double](D) for (p <- points) { val scale = (1 / (1 + math.exp(-p.y * (w.dot(p.x)))) - 1) * p.y gradient += p.x * scale @@ -71,6 +70,7 @@ object LocalFileLR { w -= gradient } + fileSrc.close() println("Final w: " + w) } } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala index e7b28d38bdfc..963c9a56d6ca 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala @@ -23,15 +23,13 @@ import java.util.Random import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet -import breeze.linalg.{Vector, DenseVector, squaredDistance} - -import org.apache.spark.SparkContext._ +import breeze.linalg.{squaredDistance, DenseVector, Vector} /** * K-means clustering. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.clustering.KMeans + * please refer to org.apache.spark.ml.clustering.KMeans. */ object LocalKMeans { val N = 1000 @@ -43,18 +41,17 @@ object LocalKMeans { def generateData: Array[DenseVector[Double]] = { def generatePoint(i: Int): DenseVector[Double] = { - DenseVector.fill(D){rand.nextDouble * R} + DenseVector.fill(D) {rand.nextDouble * R} } Array.tabulate(N)(generatePoint) } def closestPoint(p: Vector[Double], centers: HashMap[Int, Vector[Double]]): Int = { - var index = 0 var bestIndex = 0 var closest = Double.PositiveInfinity for (i <- 1 to centers.size) { - val vCurr = centers.get(i).get + val vCurr = centers(i) val tempDist = squaredDistance(p, vCurr) if (tempDist < closest) { closest = tempDist @@ -68,7 +65,7 @@ object LocalKMeans { def showWarning() { System.err.println( """WARN: This is a naive implementation of KMeans Clustering and is given as an example! - |Please use the KMeans method found in org.apache.spark.mllib.clustering + |Please use org.apache.spark.ml.clustering.KMeans |for more conventional use. """.stripMargin) } @@ -78,8 +75,8 @@ object LocalKMeans { showWarning() val data = generateData - var points = new HashSet[Vector[Double]] - var kPoints = new HashMap[Int, Vector[Double]] + val points = new HashSet[Vector[Double]] + val kPoints = new HashMap[Int, Vector[Double]] var tempDist = 1.0 while (points.size < K) { @@ -94,11 +91,11 @@ object LocalKMeans { println("Initial centers: " + kPoints) while(tempDist > convergeDist) { - var closest = data.map (p => (closestPoint(p, kPoints), (p, 1))) + val closest = data.map (p => (closestPoint(p, kPoints), (p, 1))) - var mappings = closest.groupBy[Int] (x => x._1) + val mappings = closest.groupBy[Int] (x => x._1) - var pointStats = mappings.map { pair => + val pointStats = mappings.map { pair => pair._2.reduceLeft [(Int, (Vector[Double], Int))] { case ((id1, (p1, c1)), (id2, (p2, c2))) => (id1, (p1 + p2, c1 + c2)) } @@ -109,7 +106,7 @@ object LocalKMeans { tempDist = 0.0 for (mapping <- newPoints) { - tempDist += squaredDistance(kPoints.get(mapping._1).get, mapping._2) + tempDist += squaredDistance(kPoints(mapping._1), mapping._2) } for (newP <- newPoints) { diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala index 4f6b092a59ca..eb5221f08593 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala @@ -20,14 +20,13 @@ package org.apache.spark.examples import java.util.Random -import breeze.linalg.{Vector, DenseVector} +import breeze.linalg.{DenseVector, Vector} /** * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object LocalLR { val N = 10000 // Number of data points @@ -41,7 +40,7 @@ object LocalLR { def generateData: Array[DataPoint] = { def generatePoint(i: Int): DataPoint = { val y = if (i % 2 == 0) -1 else 1 - val x = DenseVector.fill(D){rand.nextGaussian + y * R} + val x = DenseVector.fill(D) {rand.nextGaussian + y * R} DataPoint(x, y) } Array.tabulate(N)(generatePoint) @@ -50,8 +49,7 @@ object LocalLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } @@ -62,12 +60,12 @@ object LocalLR { val data = generateData // Initialize w to a random value - var w = DenseVector.fill(D){2 * rand.nextDouble - 1} + val w = DenseVector.fill(D) {2 * rand.nextDouble - 1} println("Initial w: " + w) for (i <- 1 to ITERATIONS) { println("On iteration " + i) - var gradient = DenseVector.zeros[Double](D) + val gradient = DenseVector.zeros[Double](D) for (p <- data) { val scale = (1 / (1 + math.exp(-p.y * (w.dot(p.x)))) - 1) * p.y gradient += p.x * scale diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala index 3d923625f11b..121b768e4198 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala @@ -20,16 +20,13 @@ package org.apache.spark.examples import scala.math.random -import org.apache.spark._ -import org.apache.spark.SparkContext._ - object LocalPi { def main(args: Array[String]) { var count = 0 for (i <- 1 to 100000) { val x = random * 2 - 1 val y = random * 2 - 1 - if (x*x + y*y < 1) count += 1 + if (x*x + y*y <= 1) count += 1 } println("Pi is roughly " + 4 * count / 100000.0) } diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index a80de10f4610..c55b68e03396 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -19,7 +19,6 @@ package org.apache.spark.examples import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ /** * Executes a roll up-style query against Apache logs. diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index 61ce9db914f9..e6f33b7adf5d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -19,16 +19,19 @@ package org.apache.spark.examples import org.apache.spark.rdd.RDD -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession + /** - * Usage: MultiBroadcastTest [slices] [numElem] - */ + * Usage: MultiBroadcastTest [partitions] [numElem] + */ object MultiBroadcastTest { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("Multi-Broadcast Test") - val sc = new SparkContext(sparkConf) + val spark = SparkSession + .builder + .appName("Multi-Broadcast Test") + .getOrCreate() val slices = if (args.length > 0) args(0).toInt else 2 val num = if (args.length > 1) args(1).toInt else 1000000 @@ -43,15 +46,15 @@ object MultiBroadcastTest { arr2(i) = i } - val barr1 = sc.broadcast(arr1) - val barr2 = sc.broadcast(arr2) - val observedSizes: RDD[(Int, Int)] = sc.parallelize(1 to 10, slices).map { _ => - (barr1.value.size, barr2.value.size) + val barr1 = spark.sparkContext.broadcast(arr1) + val barr2 = spark.sparkContext.broadcast(arr2) + val observedSizes: RDD[(Int, Int)] = spark.sparkContext.parallelize(1 to 10, slices).map { _ => + (barr1.value.length, barr2.value.length) } // Collect the small RDD so we can print the observed sizes locally. observedSizes.collect().foreach(i => println(i)) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 3b0b00fe4dd0..8e1a574c9222 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -20,27 +20,27 @@ package org.apache.spark.examples import java.util.Random -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ +import org.apache.spark.sql.SparkSession /** - * Usage: SimpleSkewedGroupByTest [numMappers] [numKVPairs] [valSize] [numReducers] [ratio] - */ + * Usage: SimpleSkewedGroupByTest [numMappers] [numKVPairs] [valSize] [numReducers] [ratio] + */ object SimpleSkewedGroupByTest { def main(args: Array[String]) { + val spark = SparkSession + .builder + .appName("SimpleSkewedGroupByTest") + .getOrCreate() - val sparkConf = new SparkConf().setAppName("SimpleSkewedGroupByTest") - var numMappers = if (args.length > 0) args(0).toInt else 2 - var numKVPairs = if (args.length > 1) args(1).toInt else 1000 - var valSize = if (args.length > 2) args(2).toInt else 1000 - var numReducers = if (args.length > 3) args(3).toInt else numMappers - var ratio = if (args.length > 4) args(4).toInt else 5.0 - - val sc = new SparkContext(sparkConf) + val numMappers = if (args.length > 0) args(0).toInt else 2 + val numKVPairs = if (args.length > 1) args(1).toInt else 1000 + val valSize = if (args.length > 2) args(2).toInt else 1000 + val numReducers = if (args.length > 3) args(3).toInt else numMappers + val ratio = if (args.length > 4) args(4).toInt else 5.0 - val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => + val pairs1 = spark.sparkContext.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random - var result = new Array[(Int, Array[Byte])](numKVPairs) + val result = new Array[(Int, Array[Byte])](numKVPairs) for (i <- 0 until numKVPairs) { val byteArr = new Array[Byte](valSize) ranGen.nextBytes(byteArr) @@ -65,7 +65,7 @@ object SimpleSkewedGroupByTest { // .map{case (k,v) => (k, v.size)} // .collectAsMap) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 719e2176fed3..4d3c34041bc1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -20,29 +20,30 @@ package org.apache.spark.examples import java.util.Random -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ +import org.apache.spark.sql.SparkSession /** - * Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers] - */ + * Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers] + */ object SkewedGroupByTest { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("GroupBy Test") - var numMappers = if (args.length > 0) args(0).toInt else 2 - var numKVPairs = if (args.length > 1) args(1).toInt else 1000 - var valSize = if (args.length > 2) args(2).toInt else 1000 - var numReducers = if (args.length > 3) args(3).toInt else numMappers + val spark = SparkSession + .builder + .appName("GroupBy Test") + .getOrCreate() - val sc = new SparkContext(sparkConf) + val numMappers = if (args.length > 0) args(0).toInt else 2 + var numKVPairs = if (args.length > 1) args(1).toInt else 1000 + val valSize = if (args.length > 2) args(2).toInt else 1000 + val numReducers = if (args.length > 3) args(3).toInt else numMappers - val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => + val pairs1 = spark.sparkContext.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random - // map output sizes lineraly increase from the 1st to the last + // map output sizes linearly increase from the 1st to the last numKVPairs = (1.0 * (p + 1) / numMappers * numKVPairs).toInt - var arr1 = new Array[(Int, Array[Byte])](numKVPairs) + val arr1 = new Array[(Int, Array[Byte])](numKVPairs) for (i <- 0 until numKVPairs) { val byteArr = new Array[Byte](valSize) ranGen.nextBytes(byteArr) @@ -55,7 +56,7 @@ object SkewedGroupByTest { println(pairs1.groupByKey(numReducers).count()) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index 69799b7c2bb3..a99ddd9fd37d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -20,13 +20,13 @@ package org.apache.spark.examples import org.apache.commons.math3.linear._ -import org.apache.spark._ +import org.apache.spark.sql.SparkSession /** * Alternating least squares matrix factorization. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.recommendation.ALS + * please refer to org.apache.spark.ml.recommendation.ALS. */ object SparkALS { @@ -58,7 +58,7 @@ object SparkALS { } def update(i: Int, m: RealVector, us: Array[RealVector], R: RealMatrix) : RealVector = { - val U = us.size + val U = us.length val F = us(0).getDimension var XtX: RealMatrix = new Array2DRowRealMatrix(F, F) var Xty: RealVector = new ArrayRealVector(F) @@ -81,7 +81,7 @@ object SparkALS { def showWarning() { System.err.println( """WARN: This is a naive implementation of ALS and is given as an example! - |Please use the ALS method found in org.apache.spark.mllib.recommendation + |Please use org.apache.spark.ml.recommendation.ALS |for more conventional use. """.stripMargin) } @@ -100,7 +100,7 @@ object SparkALS { ITERATIONS = iters.getOrElse("5").toInt slices = slices_.getOrElse("2").toInt case _ => - System.err.println("Usage: SparkALS [M] [U] [F] [iters] [slices]") + System.err.println("Usage: SparkALS [M] [U] [F] [iters] [partitions]") System.exit(1) } @@ -108,8 +108,12 @@ object SparkALS { println(s"Running with M=$M, U=$U, F=$F, iters=$ITERATIONS") - val sparkConf = new SparkConf().setAppName("SparkALS") - val sc = new SparkContext(sparkConf) + val spark = SparkSession + .builder + .appName("SparkALS") + .getOrCreate() + + val sc = spark.sparkContext val R = generateR() @@ -135,7 +139,7 @@ object SparkALS { println() } - sc.stop() + spark.stop() } private def randomVector(n: Int): RealVector = diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 505ea5a4c7a8..9d675bbc18f3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -22,30 +22,26 @@ import java.util.Random import scala.math.exp -import breeze.linalg.{Vector, DenseVector} -import org.apache.hadoop.conf.Configuration - -import org.apache.spark._ -import org.apache.spark.scheduler.InputFormatInfo +import breeze.linalg.{DenseVector, Vector} +import org.apache.spark.sql.SparkSession /** * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object SparkHdfsLR { - val D = 10 // Numer of dimensions + val D = 10 // Number of dimensions val rand = new Random(42) case class DataPoint(x: Vector[Double], y: Double) def parsePoint(line: String): DataPoint = { val tok = new java.util.StringTokenizer(line, " ") - var y = tok.nextToken.toDouble - var x = new Array[Double](D) + val y = tok.nextToken.toDouble + val x = new Array[Double](D) var i = 0 while (i < D) { x(i) = tok.nextToken.toDouble; i += 1 @@ -56,8 +52,7 @@ object SparkHdfsLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } @@ -71,19 +66,19 @@ object SparkHdfsLR { showWarning() - val sparkConf = new SparkConf().setAppName("SparkHdfsLR") + val spark = SparkSession + .builder + .appName("SparkHdfsLR") + .getOrCreate() + val inputPath = args(0) - val conf = new Configuration() - val sc = new SparkContext(sparkConf, - InputFormatInfo.computePreferredLocations( - Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) - )) - val lines = sc.textFile(inputPath) - val points = lines.map(parsePoint _).cache() + val lines = spark.read.textFile(inputPath).rdd + + val points = lines.map(parsePoint).cache() val ITERATIONS = args(1).toInt // Initialize w to a random value - var w = DenseVector.fill(D){2 * rand.nextDouble - 1} + val w = DenseVector.fill(D) {2 * rand.nextDouble - 1} println("Initial w: " + w) for (i <- 1 to ITERATIONS) { @@ -95,7 +90,7 @@ object SparkHdfsLR { } println("Final w: " + w) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index c56e1124ad41..fec3160e9f37 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -18,16 +18,15 @@ // scalastyle:off println package org.apache.spark.examples -import breeze.linalg.{Vector, DenseVector, squaredDistance} +import breeze.linalg.{squaredDistance, DenseVector, Vector} -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ +import org.apache.spark.sql.SparkSession /** * K-means clustering. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.clustering.KMeans + * please refer to org.apache.spark.ml.clustering.KMeans. */ object SparkKMeans { @@ -53,7 +52,7 @@ object SparkKMeans { def showWarning() { System.err.println( """WARN: This is a naive implementation of KMeans Clustering and is given as an example! - |Please use the KMeans method found in org.apache.spark.mllib.clustering + |Please use org.apache.spark.ml.clustering.KMeans |for more conventional use. """.stripMargin) } @@ -67,14 +66,17 @@ object SparkKMeans { showWarning() - val sparkConf = new SparkConf().setAppName("SparkKMeans") - val sc = new SparkContext(sparkConf) - val lines = sc.textFile(args(0)) + val spark = SparkSession + .builder + .appName("SparkKMeans") + .getOrCreate() + + val lines = spark.read.textFile(args(0)).rdd val data = lines.map(parseVector _).cache() val K = args(1).toInt val convergeDist = args(2).toDouble - val kPoints = data.takeSample(withReplacement = false, K, 42).toArray + val kPoints = data.takeSample(withReplacement = false, K, 42) var tempDist = 1.0 while(tempDist > convergeDist) { @@ -98,7 +100,7 @@ object SparkKMeans { println("Final centers:") kPoints.foreach(println) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index d265c227f4ed..c18e3d31f149 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -22,21 +22,20 @@ import java.util.Random import scala.math.exp -import breeze.linalg.{Vector, DenseVector} +import breeze.linalg.{DenseVector, Vector} -import org.apache.spark._ +import org.apache.spark.sql.SparkSession /** * Logistic regression based classification. - * Usage: SparkLR [slices] + * Usage: SparkLR [partitions] * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. + * please refer to org.apache.spark.ml.classification.LogisticRegression. */ object SparkLR { val N = 10000 // Number of data points - val D = 10 // Numer of dimensions + val D = 10 // Number of dimensions val R = 0.7 // Scaling factor val ITERATIONS = 5 val rand = new Random(42) @@ -46,7 +45,7 @@ object SparkLR { def generateData: Array[DataPoint] = { def generatePoint(i: Int): DataPoint = { val y = if (i % 2 == 0) -1 else 1 - val x = DenseVector.fill(D){rand.nextGaussian + y * R} + val x = DenseVector.fill(D) {rand.nextGaussian + y * R} DataPoint(x, y) } Array.tabulate(N)(generatePoint) @@ -55,8 +54,7 @@ object SparkLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |Please use org.apache.spark.ml.classification.LogisticRegression |for more conventional use. """.stripMargin) } @@ -65,13 +63,16 @@ object SparkLR { showWarning() - val sparkConf = new SparkConf().setAppName("SparkLR") - val sc = new SparkContext(sparkConf) + val spark = SparkSession + .builder + .appName("SparkLR") + .getOrCreate() + val numSlices = if (args.length > 0) args(0).toInt else 2 - val points = sc.parallelize(generateData, numSlices).cache() + val points = spark.sparkContext.parallelize(generateData, numSlices).cache() // Initialize w to a random value - var w = DenseVector.fill(D){2 * rand.nextDouble - 1} + val w = DenseVector.fill(D) {2 * rand.nextDouble - 1} println("Initial w: " + w) for (i <- 1 to ITERATIONS) { @@ -84,7 +85,7 @@ object SparkLR { println("Final w: " + w) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index 0fd79660dd19..5d8831265e4a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -18,8 +18,7 @@ // scalastyle:off println package org.apache.spark.examples -import org.apache.spark.SparkContext._ -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession /** * Computes the PageRank of URLs from an input file. Input file should @@ -32,6 +31,11 @@ import org.apache.spark.{SparkConf, SparkContext} * * This is an example implementation for learning how to use Spark. For more conventional use, * please refer to org.apache.spark.graphx.lib.PageRank + * + * Example Usage: + * {{{ + * bin/run-example SparkPageRank data/mllib/pagerank_data.txt 10 + * }}} */ object SparkPageRank { @@ -51,10 +55,13 @@ object SparkPageRank { showWarning() - val sparkConf = new SparkConf().setAppName("PageRank") + val spark = SparkSession + .builder + .appName("SparkPageRank") + .getOrCreate() + val iters = if (args.length > 1) args(1).toInt else 10 - val ctx = new SparkContext(sparkConf) - val lines = ctx.textFile(args(0), 1) + val lines = spark.read.textFile(args(0)).rdd val links = lines.map{ s => val parts = s.split("\\s+") (parts(0), parts(1)) @@ -72,7 +79,7 @@ object SparkPageRank { val output = ranks.collect() output.foreach(tup => println(tup._1 + " has rank: " + tup._2 + ".")) - ctx.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index 818d4f2b81f8..a5cacf17a5cc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -20,21 +20,23 @@ package org.apache.spark.examples import scala.math.random -import org.apache.spark._ +import org.apache.spark.sql.SparkSession /** Computes an approximation to pi */ object SparkPi { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("Spark Pi") - val spark = new SparkContext(conf) + val spark = SparkSession + .builder + .appName("Spark Pi") + .getOrCreate() val slices = if (args.length > 0) args(0).toInt else 2 val n = math.min(100000L * slices, Int.MaxValue).toInt // avoid overflow - val count = spark.parallelize(1 until n, slices).map { i => + val count = spark.sparkContext.parallelize(1 until n, slices).map { i => val x = random * 2 - 1 val y = random * 2 - 1 - if (x*x + y*y < 1) 1 else 0 + if (x*x + y*y <= 1) 1 else 0 }.reduce(_ + _) - println("Pi is roughly " + 4.0 * count / n) + println("Pi is roughly " + 4.0 * count / (n - 1)) spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 95072071ccdd..558295ab928a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -18,11 +18,10 @@ // scalastyle:off println package org.apache.spark.examples -import scala.util.Random import scala.collection.mutable +import scala.util.Random -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ +import org.apache.spark.sql.SparkSession /** * Transitive closure on a graph. @@ -43,10 +42,12 @@ object SparkTC { } def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("SparkTC") - val spark = new SparkContext(sparkConf) + val spark = SparkSession + .builder + .appName("SparkTC") + .getOrCreate() val slices = if (args.length > 0) args(0).toInt else 2 - var tc = spark.parallelize(generateGraph, slices).cache() + var tc = spark.sparkContext.parallelize(generateGraph, slices).cache() // Linear transitive closure: each round grows paths by one edge, // by joining the graph's edges with the already-discovered paths. diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala deleted file mode 100644 index cfbdae02212a..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala +++ /dev/null @@ -1,98 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples - -import java.util.Random - -import scala.math.exp - -import breeze.linalg.{Vector, DenseVector} -import org.apache.hadoop.conf.Configuration - -import org.apache.spark._ -import org.apache.spark.scheduler.InputFormatInfo -import org.apache.spark.storage.StorageLevel - - -/** - * Logistic regression based classification. - * This example uses Tachyon to persist rdds during computation. - * - * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. - */ -object SparkTachyonHdfsLR { - val D = 10 // Numer of dimensions - val rand = new Random(42) - - def showWarning() { - System.err.println( - """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or - |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS - |for more conventional use. - """.stripMargin) - } - - case class DataPoint(x: Vector[Double], y: Double) - - def parsePoint(line: String): DataPoint = { - val tok = new java.util.StringTokenizer(line, " ") - var y = tok.nextToken.toDouble - var x = new Array[Double](D) - var i = 0 - while (i < D) { - x(i) = tok.nextToken.toDouble; i += 1 - } - DataPoint(new DenseVector(x), y) - } - - def main(args: Array[String]) { - - showWarning() - - val inputPath = args(0) - val sparkConf = new SparkConf().setAppName("SparkTachyonHdfsLR") - val conf = new Configuration() - val sc = new SparkContext(sparkConf, - InputFormatInfo.computePreferredLocations( - Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) - )) - val lines = sc.textFile(inputPath) - val points = lines.map(parsePoint _).persist(StorageLevel.OFF_HEAP) - val ITERATIONS = args(1).toInt - - // Initialize w to a random value - var w = DenseVector.fill(D){2 * rand.nextDouble - 1} - println("Initial w: " + w) - - for (i <- 1 to ITERATIONS) { - println("On iteration " + i) - val gradient = points.map { p => - p.x * (1 / (1 + exp(-p.y * (w.dot(p.x)))) - 1) * p.y - }.reduce(_ + _) - w -= gradient - } - - println("Final w: " + w) - sc.stop() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala deleted file mode 100644 index e46ac655beb5..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala +++ /dev/null @@ -1,50 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples - -import scala.math.random - -import org.apache.spark._ -import org.apache.spark.storage.StorageLevel - -/** - * Computes an approximation to pi - * This example uses Tachyon to persist rdds during computation. - */ -object SparkTachyonPi { - def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("SparkTachyonPi") - val spark = new SparkContext(sparkConf) - - val slices = if (args.length > 0) args(0).toInt else 2 - val n = 100000 * slices - - val rdd = spark.parallelize(1 to n, slices) - rdd.persist(StorageLevel.OFF_HEAP) - val count = rdd.map { i => - val x = random * 2 - 1 - val y = random * 2 - 1 - if (x * x + y * y < 1) 1 else 0 - }.reduce(_ + _) - println("Pi is roughly " + 4.0 * count / n) - - spark.stop() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala new file mode 100644 index 000000000000..8441b5a9dd8a --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.graphx + +// $example on$ +import org.apache.spark.graphx.{Graph, VertexRDD} +import org.apache.spark.graphx.util.GraphGenerators +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example use the [`aggregateMessages`][Graph.aggregateMessages] operator to + * compute the average age of the more senior followers of each user + * Run with + * {{{ + * bin/run-example graphx.AggregateMessagesExample + * }}} + */ +object AggregateMessagesExample { + + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + val sc = spark.sparkContext + + // $example on$ + // Create a graph with "age" as the vertex property. + // Here we use a random graph for simplicity. + val graph: Graph[Double, Int] = + GraphGenerators.logNormalGraph(sc, numVertices = 100).mapVertices( (id, _) => id.toDouble ) + // Compute the number of older followers and their total age + val olderFollowers: VertexRDD[(Int, Double)] = graph.aggregateMessages[(Int, Double)]( + triplet => { // Map Function + if (triplet.srcAttr > triplet.dstAttr) { + // Send message to destination vertex containing counter and age + triplet.sendToDst((1, triplet.srcAttr)) + } + }, + // Add counter and age + (a, b) => (a._1 + b._1, a._2 + b._2) // Reduce Function + ) + // Divide total age by number of older followers to get average age of older followers + val avgAgeOfOlderFollowers: VertexRDD[Double] = + olderFollowers.mapValues( (id, value) => + value match { case (count, totalAge) => totalAge / count } ) + // Display the results + avgAgeOfOlderFollowers.collect.foreach(println(_)) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala index 8dd6c9706e7d..619e585b6ca1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala @@ -19,11 +19,13 @@ package org.apache.spark.examples.graphx import scala.collection.mutable + import org.apache.spark._ -import org.apache.spark.storage.StorageLevel import org.apache.spark.graphx._ -import org.apache.spark.graphx.lib._ import org.apache.spark.graphx.PartitionStrategy._ +import org.apache.spark.graphx.lib._ +import org.apache.spark.internal.Logging +import org.apache.spark.storage.StorageLevel /** * Driver program for running graph algorithms. diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/ComprehensiveExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/ComprehensiveExample.scala new file mode 100644 index 000000000000..6598863bd2ea --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/ComprehensiveExample.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.graphx + +// $example on$ +import org.apache.spark.graphx.GraphLoader +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * Suppose I want to build a graph from some text files, restrict the graph + * to important relationships and users, run page-rank on the sub-graph, and + * then finally return attributes associated with the top users. + * This example do all of this in just a few lines with GraphX. + * + * Run with + * {{{ + * bin/run-example graphx.ComprehensiveExample + * }}} + */ +object ComprehensiveExample { + + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + val sc = spark.sparkContext + + // $example on$ + // Load my user data and parse into tuples of user id and attribute list + val users = (sc.textFile("data/graphx/users.txt") + .map(line => line.split(",")).map( parts => (parts.head.toLong, parts.tail) )) + + // Parse the edge data which is already in userId -> userId format + val followerGraph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt") + + // Attach the user attributes + val graph = followerGraph.outerJoinVertices(users) { + case (uid, deg, Some(attrList)) => attrList + // Some users may not have attributes so we set them as empty + case (uid, deg, None) => Array.empty[String] + } + + // Restrict the graph to users with usernames and names + val subgraph = graph.subgraph(vpred = (vid, attr) => attr.size == 2) + + // Compute the PageRank + val pagerankGraph = subgraph.pageRank(0.001) + + // Get the attributes of the top pagerank users + val userInfoWithPageRank = subgraph.outerJoinVertices(pagerankGraph.vertices) { + case (uid, attrList, Some(pr)) => (pr, attrList.toList) + case (uid, attrList, None) => (0.0, attrList.toList) + } + + println(userInfoWithPageRank.vertices.top(5)(Ordering.by(_._2._1)).mkString("\n")) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/ConnectedComponentsExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/ConnectedComponentsExample.scala new file mode 100644 index 000000000000..5377ddb3594b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/ConnectedComponentsExample.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.graphx + +// $example on$ +import org.apache.spark.graphx.GraphLoader +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * A connected components algorithm example. + * The connected components algorithm labels each connected component of the graph + * with the ID of its lowest-numbered vertex. + * For example, in a social network, connected components can approximate clusters. + * GraphX contains an implementation of the algorithm in the + * [`ConnectedComponents` object][ConnectedComponents], + * and we compute the connected components of the example social network dataset. + * + * Run with + * {{{ + * bin/run-example graphx.ConnectedComponentsExample + * }}} + */ +object ConnectedComponentsExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + val sc = spark.sparkContext + + // $example on$ + // Load the graph as in the PageRank example + val graph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt") + // Find the connected components + val cc = graph.connectedComponents().vertices + // Join the connected components with the usernames + val users = sc.textFile("data/graphx/users.txt").map { line => + val fields = line.split(",") + (fields(0).toLong, fields(1)) + } + val ccByUsername = users.join(cc).map { + case (id, (username, cc)) => (username, cc) + } + // Print the result + println(ccByUsername.collect().mkString("\n")) + // $example off$ + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/PageRankExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/PageRankExample.scala new file mode 100644 index 000000000000..9e9affca07a1 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/PageRankExample.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.graphx + +// $example on$ +import org.apache.spark.graphx.GraphLoader +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * A PageRank example on social network dataset + * Run with + * {{{ + * bin/run-example graphx.PageRankExample + * }}} + */ +object PageRankExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + val sc = spark.sparkContext + + // $example on$ + // Load the edges as a graph + val graph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt") + // Run PageRank + val ranks = graph.pageRank(0.0001).vertices + // Join the ranks with the usernames + val users = sc.textFile("data/graphx/users.txt").map { line => + val fields = line.split(",") + (fields(0).toLong, fields(1)) + } + val ranksByUsername = users.join(ranks).map { + case (id, (username, rank)) => (username, rank) + } + // Print the result + println(ranksByUsername.collect().mkString("\n")) + // $example off$ + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SSSPExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SSSPExample.scala new file mode 100644 index 000000000000..5e8b19671de7 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SSSPExample.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.graphx + +// $example on$ +import org.apache.spark.graphx.{Graph, VertexId} +import org.apache.spark.graphx.util.GraphGenerators +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example use the Pregel operator to express computation + * such as single source shortest path + * Run with + * {{{ + * bin/run-example graphx.SSSPExample + * }}} + */ +object SSSPExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + val sc = spark.sparkContext + + // $example on$ + // A graph with edge attributes containing distances + val graph: Graph[Long, Double] = + GraphGenerators.logNormalGraph(sc, numVertices = 100).mapEdges(e => e.attr.toDouble) + val sourceId: VertexId = 42 // The ultimate source + // Initialize the graph such that all vertices except the root have distance infinity. + val initialGraph = graph.mapVertices((id, _) => + if (id == sourceId) 0.0 else Double.PositiveInfinity) + val sssp = initialGraph.pregel(Double.PositiveInfinity)( + (id, dist, newDist) => math.min(dist, newDist), // Vertex Program + triplet => { // Send Message + if (triplet.srcAttr + triplet.attr < triplet.dstAttr) { + Iterator((triplet.dstId, triplet.srcAttr + triplet.attr)) + } else { + Iterator.empty + } + }, + (a, b) => math.min(a, b) // Merge Message + ) + println(sssp.vertices.collect.mkString("\n")) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala index 46e52aacd90b..6d2228c8742a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala @@ -18,11 +18,11 @@ // scalastyle:off println package org.apache.spark.examples.graphx -import org.apache.spark.SparkContext._ +import java.io.{FileOutputStream, PrintWriter} + +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.graphx.{GraphXUtils, PartitionStrategy} -import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.graphx.util.GraphGenerators -import java.io.{PrintWriter, FileOutputStream} /** * The SynthBenchmark application can be used to run various GraphX algorithms on diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/TriangleCountingExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/TriangleCountingExample.scala new file mode 100644 index 000000000000..b9bff69086cc --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/TriangleCountingExample.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.graphx + +// $example on$ +import org.apache.spark.graphx.{GraphLoader, PartitionStrategy} +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * A vertex is part of a triangle when it has two adjacent vertices with an edge between them. + * GraphX implements a triangle counting algorithm in the [`TriangleCount` object][TriangleCount] + * that determines the number of triangles passing through each vertex, + * providing a measure of clustering. + * We compute the triangle count of the social network dataset. + * + * Note that `TriangleCount` requires the edges to be in canonical orientation (`srcId < dstId`) + * and the graph to be partitioned using [`Graph.partitionBy`][Graph.partitionBy]. + * + * Run with + * {{{ + * bin/run-example graphx.TriangleCountingExample + * }}} + */ +object TriangleCountingExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + val sc = spark.sparkContext + + // $example on$ + // Load the edges in canonical order and partition the graph for triangle count + val graph = GraphLoader.edgeListFile(sc, "data/graphx/followers.txt", true) + .partitionBy(PartitionStrategy.RandomVertexCut) + // Find the triangle count for each vertex + val triCounts = graph.triangleCount().vertices + // Join the triangle counts with the usernames + val users = sc.textFile("data/graphx/users.txt").map { line => + val fields = line.split(",") + (fields(0).toLong, fields(1)) + } + val triCountByUsername = users.join(triCounts).map { case (id, (username, tc)) => + (username, tc) + } + // Print the result + println(triCountByUsername.collect().mkString("\n")) + // $example off$ + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala new file mode 100644 index 000000000000..cdb33f4d6d21 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/AFTSurvivalRegressionExample.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.regression.AFTSurvivalRegression +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating AFTSurvivalRegression. + * Run with + * {{{ + * bin/run-example ml.AFTSurvivalRegressionExample + * }}} + */ +object AFTSurvivalRegressionExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("AFTSurvivalRegressionExample") + .getOrCreate() + + // $example on$ + val training = spark.createDataFrame(Seq( + (1.218, 1.0, Vectors.dense(1.560, -0.605)), + (2.949, 0.0, Vectors.dense(0.346, 2.158)), + (3.627, 0.0, Vectors.dense(1.380, 0.231)), + (0.273, 1.0, Vectors.dense(0.520, 1.151)), + (4.199, 0.0, Vectors.dense(0.795, -0.226)) + )).toDF("label", "censor", "features") + val quantileProbabilities = Array(0.3, 0.6) + val aft = new AFTSurvivalRegression() + .setQuantileProbabilities(quantileProbabilities) + .setQuantilesCol("quantiles") + + val model = aft.fit(training) + + // Print the coefficients, intercept and scale parameter for AFT survival regression + println(s"Coefficients: ${model.coefficients}") + println(s"Intercept: ${model.intercept}") + println(s"Scale: ${model.scale}") + model.transform(training).show(false) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala new file mode 100644 index 000000000000..07b15dfa178f --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ALSExample.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.evaluation.RegressionEvaluator +import org.apache.spark.ml.recommendation.ALS +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating ALS. + * Run with + * {{{ + * bin/run-example ml.ALSExample + * }}} + */ +object ALSExample { + + // $example on$ + case class Rating(userId: Int, movieId: Int, rating: Float, timestamp: Long) + def parseRating(str: String): Rating = { + val fields = str.split("::") + assert(fields.size == 4) + Rating(fields(0).toInt, fields(1).toInt, fields(2).toFloat, fields(3).toLong) + } + // $example off$ + + def main(args: Array[String]) { + val spark = SparkSession + .builder + .appName("ALSExample") + .getOrCreate() + import spark.implicits._ + + // $example on$ + val ratings = spark.read.textFile("data/mllib/als/sample_movielens_ratings.txt") + .map(parseRating) + .toDF() + val Array(training, test) = ratings.randomSplit(Array(0.8, 0.2)) + + // Build the recommendation model using ALS on the training data + val als = new ALS() + .setMaxIter(5) + .setRegParam(0.01) + .setUserCol("userId") + .setItemCol("movieId") + .setRatingCol("rating") + val model = als.fit(training) + + // Evaluate the model by computing the RMSE on the test data + // Note we set cold start strategy to 'drop' to ensure we don't get NaN evaluation metrics + model.setColdStartStrategy("drop") + val predictions = model.transform(test) + + val evaluator = new RegressionEvaluator() + .setMetricName("rmse") + .setLabelCol("rating") + .setPredictionCol("prediction") + val rmse = evaluator.evaluate(predictions) + println(s"Root-mean-square error = $rmse") + + // Generate top 10 movie recommendations for each user + val userRecs = model.recommendForAllUsers(10) + // Generate top 10 user recommendations for each movie + val movieRecs = model.recommendForAllItems(10) + // $example off$ + userRecs.show() + movieRecs.show() + + spark.stop() + } +} +// scalastyle:on println + diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.scala new file mode 100644 index 000000000000..c2852aacb05d --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BinarizerExample.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. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.Binarizer +// $example off$ +import org.apache.spark.sql.SparkSession + +object BinarizerExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("BinarizerExample") + .getOrCreate() + + // $example on$ + val data = Array((0, 0.1), (1, 0.8), (2, 0.2)) + val dataFrame = spark.createDataFrame(data).toDF("id", "feature") + + val binarizer: Binarizer = new Binarizer() + .setInputCol("feature") + .setOutputCol("binarized_feature") + .setThreshold(0.5) + + val binarizedDataFrame = binarizer.transform(dataFrame) + + println(s"Binarizer output with Threshold = ${binarizer.getThreshold}") + binarizedDataFrame.show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala new file mode 100644 index 000000000000..5f8f2c99cbaf --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml + +// scalastyle:off println + +// $example on$ +import org.apache.spark.ml.clustering.BisectingKMeans +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating bisecting k-means clustering. + * Run with + * {{{ + * bin/run-example ml.BisectingKMeansExample + * }}} + */ +object BisectingKMeansExample { + + def main(args: Array[String]): Unit = { + // Creates a SparkSession + val spark = SparkSession + .builder + .appName("BisectingKMeansExample") + .getOrCreate() + + // $example on$ + // Loads data. + val dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") + + // Trains a bisecting k-means model. + val bkm = new BisectingKMeans().setK(2).setSeed(1) + val model = bkm.fit(dataset) + + // Evaluate clustering. + val cost = model.computeCost(dataset) + println(s"Within Set Sum of Squared Errors = $cost") + + // Shows the result. + println("Cluster Centers: ") + val centers = model.clusterCenters + centers.foreach(println) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println + diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala new file mode 100644 index 000000000000..58f9fb30078d --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketedRandomProjectionLSHExample.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.BucketedRandomProjectionLSH +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.functions.col +// $example off$ + +/** + * An example demonstrating BucketedRandomProjectionLSH. + * Run with: + * bin/run-example ml.BucketedRandomProjectionLSHExample + */ +object BucketedRandomProjectionLSHExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession + val spark = SparkSession + .builder + .appName("BucketedRandomProjectionLSHExample") + .getOrCreate() + + // $example on$ + val dfA = spark.createDataFrame(Seq( + (0, Vectors.dense(1.0, 1.0)), + (1, Vectors.dense(1.0, -1.0)), + (2, Vectors.dense(-1.0, -1.0)), + (3, Vectors.dense(-1.0, 1.0)) + )).toDF("id", "features") + + val dfB = spark.createDataFrame(Seq( + (4, Vectors.dense(1.0, 0.0)), + (5, Vectors.dense(-1.0, 0.0)), + (6, Vectors.dense(0.0, 1.0)), + (7, Vectors.dense(0.0, -1.0)) + )).toDF("id", "features") + + val key = Vectors.dense(1.0, 0.0) + + val brp = new BucketedRandomProjectionLSH() + .setBucketLength(2.0) + .setNumHashTables(3) + .setInputCol("features") + .setOutputCol("hashes") + + val model = brp.fit(dfA) + + // Feature Transformation + println("The hashed dataset where hashed values are stored in the column 'hashes':") + model.transform(dfA).show() + + // Compute the locality sensitive hashes for the input rows, then perform approximate + // similarity join. + // We could avoid computing hashes by passing in the already-transformed dataset, e.g. + // `model.approxSimilarityJoin(transformedA, transformedB, 1.5)` + println("Approximately joining dfA and dfB on Euclidean distance smaller than 1.5:") + model.approxSimilarityJoin(dfA, dfB, 1.5, "EuclideanDistance") + .select(col("datasetA.id").alias("idA"), + col("datasetB.id").alias("idB"), + col("EuclideanDistance")).show() + + // Compute the locality sensitive hashes for the input rows, then perform approximate nearest + // neighbor search. + // We could avoid computing hashes by passing in the already-transformed dataset, e.g. + // `model.approxNearestNeighbors(transformedA, key, 2)` + println("Approximately searching dfA for 2 nearest neighbors of the key:") + model.approxNearestNeighbors(dfA, key, 2).show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.scala new file mode 100644 index 000000000000..04e4eccd436e --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BucketizerExample.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. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.Bucketizer +// $example off$ +import org.apache.spark.sql.SparkSession + +object BucketizerExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("BucketizerExample") + .getOrCreate() + + // $example on$ + val splits = Array(Double.NegativeInfinity, -0.5, 0.0, 0.5, Double.PositiveInfinity) + + val data = Array(-999.9, -0.5, -0.3, 0.0, 0.2, 999.9) + val dataFrame = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") + + val bucketizer = new Bucketizer() + .setInputCol("features") + .setOutputCol("bucketedFeatures") + .setSplits(splits) + + // Transform original data into its bucket index. + val bucketedData = bucketizer.transform(dataFrame) + + println(s"Bucketizer output with ${bucketizer.getSplits.length-1} buckets") + bucketedData.show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println + diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala new file mode 100644 index 000000000000..5638e66b8792 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSqSelectorExample.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.ChiSqSelector +import org.apache.spark.ml.linalg.Vectors +// $example off$ +import org.apache.spark.sql.SparkSession + +object ChiSqSelectorExample { + def main(args: Array[String]) { + val spark = SparkSession + .builder + .appName("ChiSqSelectorExample") + .getOrCreate() + import spark.implicits._ + + // $example on$ + val data = Seq( + (7, Vectors.dense(0.0, 0.0, 18.0, 1.0), 1.0), + (8, Vectors.dense(0.0, 1.0, 12.0, 0.0), 0.0), + (9, Vectors.dense(1.0, 0.0, 15.0, 0.1), 0.0) + ) + + val df = spark.createDataset(data).toDF("id", "features", "clicked") + + val selector = new ChiSqSelector() + .setNumTopFeatures(1) + .setFeaturesCol("features") + .setLabelCol("clicked") + .setOutputCol("selectedFeatures") + + val result = selector.fit(df).transform(df) + + println(s"ChiSqSelector output with top ${selector.getNumTopFeatures} features selected") + result.show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ChiSquareTestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSquareTestExample.scala new file mode 100644 index 000000000000..dcee1e427ce5 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ChiSquareTestExample.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.stat.ChiSquareTest +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example for Chi-square hypothesis testing. + * Run with + * {{{ + * bin/run-example ml.ChiSquareTestExample + * }}} + */ +object ChiSquareTestExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("ChiSquareTestExample") + .getOrCreate() + import spark.implicits._ + + // $example on$ + val data = Seq( + (0.0, Vectors.dense(0.5, 10.0)), + (0.0, Vectors.dense(1.5, 20.0)), + (1.0, Vectors.dense(1.5, 30.0)), + (0.0, Vectors.dense(3.5, 30.0)), + (0.0, Vectors.dense(3.5, 40.0)), + (1.0, Vectors.dense(3.5, 40.0)) + ) + + val df = data.toDF("label", "features") + val chi = ChiSquareTest.test(df, "features", "label").head + println("pValues = " + chi.getAs[Vector](0)) + println("degreesOfFreedom = " + chi.getSeq[Int](1).mkString("[", ",", "]")) + println("statistics = " + chi.getAs[Vector](2)) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CorrelationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CorrelationExample.scala new file mode 100644 index 000000000000..3f57dc342eb0 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CorrelationExample.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.linalg.{Matrix, Vectors} +import org.apache.spark.ml.stat.Correlation +import org.apache.spark.sql.Row +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example for computing correlation matrix. + * Run with + * {{{ + * bin/run-example ml.CorrelationExample + * }}} + */ +object CorrelationExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("CorrelationExample") + .getOrCreate() + import spark.implicits._ + + // $example on$ + val data = Seq( + Vectors.sparse(4, Seq((0, 1.0), (3, -2.0))), + Vectors.dense(4.0, 5.0, 0.0, 3.0), + Vectors.dense(6.0, 7.0, 0.0, 8.0), + Vectors.sparse(4, Seq((0, 9.0), (3, 1.0))) + ) + + val df = data.map(Tuple1.apply).toDF("features") + val Row(coeff1: Matrix) = Correlation.corr(df, "features").head + println("Pearson correlation matrix:\n" + coeff1.toString) + + val Row(coeff2: Matrix) = Correlation.corr(df, "features", "spearman").head + println("Spearman correlation matrix:\n" + coeff2.toString) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala index ba916f66c4c0..91d861dd4380 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CountVectorizerExample.scala @@ -21,18 +21,17 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.{CountVectorizer, CountVectorizerModel} // $example off$ -import org.apache.spark.sql.SQLContext -import org.apache.spark.{SparkConf, SparkContext} - +import org.apache.spark.sql.SparkSession object CountVectorizerExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("CounterVectorizerExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("CountVectorizerExample") + .getOrCreate() // $example on$ - val df = sqlContext.createDataFrame(Seq( + val df = spark.createDataFrame(Seq( (0, Array("a", "b", "c")), (1, Array("a", "b", "b", "c", "a")) )).toDF("id", "words") @@ -50,8 +49,10 @@ object CountVectorizerExample { .setInputCol("words") .setOutputCol("features") - cvModel.transform(df).select("features").show() + cvModel.transform(df).show(false) // $example off$ + + spark.stop() } } // scalastyle:on println 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 deleted file mode 100644 index 14b358d46f6a..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala +++ /dev/null @@ -1,114 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.ml - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.ml.Pipeline -import org.apache.spark.ml.classification.LogisticRegression -import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator -import org.apache.spark.ml.feature.{HashingTF, Tokenizer} -import org.apache.spark.ml.tuning.{ParamGridBuilder, CrossValidator} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.sql.{Row, SQLContext} - -/** - * A simple example demonstrating model selection using CrossValidator. - * This example also demonstrates how Pipelines are Estimators. - * - * This example uses the [[LabeledDocument]] and [[Document]] case classes from - * [[SimpleTextClassificationPipeline]]. - * - * Run with - * {{{ - * bin/run-example ml.CrossValidatorExample - * }}} - */ -object CrossValidatorExample { - - def main(args: Array[String]) { - val conf = new SparkConf().setAppName("CrossValidatorExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ - - // Prepare training documents, which are labeled. - val training = sc.parallelize(Seq( - LabeledDocument(0L, "a b c d e spark", 1.0), - LabeledDocument(1L, "b d", 0.0), - LabeledDocument(2L, "spark f g h", 1.0), - LabeledDocument(3L, "hadoop mapreduce", 0.0), - LabeledDocument(4L, "b spark who", 1.0), - LabeledDocument(5L, "g d a y", 0.0), - LabeledDocument(6L, "spark fly", 1.0), - LabeledDocument(7L, "was mapreduce", 0.0), - LabeledDocument(8L, "e spark program", 1.0), - LabeledDocument(9L, "a e c l", 0.0), - LabeledDocument(10L, "spark compile", 1.0), - LabeledDocument(11L, "hadoop software", 0.0))) - - // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. - val tokenizer = new Tokenizer() - .setInputCol("text") - .setOutputCol("words") - val hashingTF = new HashingTF() - .setInputCol(tokenizer.getOutputCol) - .setOutputCol("features") - val lr = new LogisticRegression() - .setMaxIter(10) - val pipeline = new Pipeline() - .setStages(Array(tokenizer, hashingTF, lr)) - - // We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. - // This will allow us to jointly choose parameters for all Pipeline stages. - // A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. - val crossval = new CrossValidator() - .setEstimator(pipeline) - .setEvaluator(new BinaryClassificationEvaluator) - // We use a ParamGridBuilder to construct a grid of parameters to search over. - // With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, - // this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. - val paramGrid = new ParamGridBuilder() - .addGrid(hashingTF.numFeatures, Array(10, 100, 1000)) - .addGrid(lr.regParam, Array(0.1, 0.01)) - .build() - crossval.setEstimatorParamMaps(paramGrid) - crossval.setNumFolds(2) // Use 3+ in practice - - // Run cross-validation, and choose the best set of parameters. - val cvModel = crossval.fit(training.toDF()) - - // Prepare test documents, which are unlabeled. - val test = sc.parallelize(Seq( - Document(4L, "spark i j k"), - Document(5L, "l m n"), - Document(6L, "mapreduce spark"), - Document(7L, "apache hadoop"))) - - // Make predictions on test documents. cvModel uses the best model found (lrModel). - cvModel.transform(test.toDF()) - .select("id", "text", "probability", "prediction") - .collect() - .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => - println(s"($id, $text) --> prob=$prob, prediction=$prediction") - } - - sc.stop() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.scala new file mode 100644 index 000000000000..3383171303ec --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DCTExample.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. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.DCT +import org.apache.spark.ml.linalg.Vectors +// $example off$ +import org.apache.spark.sql.SparkSession + +object DCTExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("DCTExample") + .getOrCreate() + + // $example on$ + val data = Seq( + Vectors.dense(0.0, 1.0, -2.0, 3.0), + Vectors.dense(-1.0, 2.0, 4.0, -7.0), + Vectors.dense(14.0, -2.0, -5.0, 1.0)) + + val df = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") + + val dct = new DCT() + .setInputCol("features") + .setOutputCol("featuresDCT") + .setInverse(false) + + val dctDf = dct.transform(df) + dctDf.select("featuresDCT").show(false) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println + diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala new file mode 100644 index 000000000000..0658bddf1696 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +import java.io.File + +import scopt.OptionParser + +import org.apache.spark.examples.mllib.AbstractParams +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer +import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.util.Utils + +/** + * An example of how to use [[DataFrame]] for ML. Run with + * {{{ + * ./bin/run-example ml.DataFrameExample [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object DataFrameExample { + + case class Params(input: String = "data/mllib/sample_libsvm_data.txt") + extends AbstractParams[Params] + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("DataFrameExample") { + head("DataFrameExample: an example app using DataFrame for ML.") + opt[String]("input") + .text(s"input path to dataframe") + .action((x, c) => c.copy(input = x)) + checkConfig { params => + success + } + } + + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) + } + } + + def run(params: Params): Unit = { + val spark = SparkSession + .builder + .appName(s"DataFrameExample with $params") + .getOrCreate() + + // Load input data + println(s"Loading LIBSVM file with UDT from ${params.input}.") + val df: DataFrame = spark.read.format("libsvm").load(params.input).cache() + println("Schema from LIBSVM:") + df.printSchema() + println(s"Loaded training data as a DataFrame with ${df.count()} records.") + + // Show statistical summary of labels. + val labelSummary = df.describe("label") + labelSummary.show() + + // Convert features column to an RDD of vectors. + val features = df.select("features").rdd.map { case Row(v: Vector) => v } + val featureSummary = features.aggregate(new MultivariateOnlineSummarizer())( + (summary, feat) => summary.add(Vectors.fromML(feat)), + (sum1, sum2) => sum1.merge(sum2)) + println(s"Selected features column with average values:\n ${featureSummary.mean.toString}") + + // Save the records in a parquet file. + val tmpDir = Utils.createTempDir() + val outputDir = new File(tmpDir, "dataframe").toString + println(s"Saving to $outputDir as Parquet file.") + df.write.parquet(outputDir) + + // Load the records back. + println(s"Loading Parquet file with UDT from $outputDir.") + val newDF = spark.read.parquet(outputDir) + println(s"Schema from Parquet:") + newDF.printSchema() + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala new file mode 100644 index 000000000000..bc6d3275933e --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeClassificationExample.scala @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.classification.DecisionTreeClassificationModel +import org.apache.spark.ml.classification.DecisionTreeClassifier +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator +import org.apache.spark.ml.feature.{IndexToString, StringIndexer, VectorIndexer} +// $example off$ +import org.apache.spark.sql.SparkSession + +object DecisionTreeClassificationExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("DecisionTreeClassificationExample") + .getOrCreate() + // $example on$ + // Load the data stored in LIBSVM format as a DataFrame. + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + // Index labels, adding metadata to the label column. + // Fit on whole dataset to include all labels in index. + val labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data) + // Automatically identify categorical features, and index them. + val featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) // features with > 4 distinct values are treated as continuous. + .fit(data) + + // Split the data into training and test sets (30% held out for testing). + val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + + // Train a DecisionTree model. + val dt = new DecisionTreeClassifier() + .setLabelCol("indexedLabel") + .setFeaturesCol("indexedFeatures") + + // Convert indexed labels back to original labels. + val labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labels) + + // Chain indexers and tree in a Pipeline. + val pipeline = new Pipeline() + .setStages(Array(labelIndexer, featureIndexer, dt, labelConverter)) + + // Train model. This also runs the indexers. + val model = pipeline.fit(trainingData) + + // Make predictions. + val predictions = model.transform(testData) + + // Select example rows to display. + predictions.select("predictedLabel", "label", "features").show(5) + + // Select (prediction, true label) and compute test error. + val evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("accuracy") + val accuracy = evaluator.evaluate(predictions) + println("Test Error = " + (1.0 - accuracy)) + + val treeModel = model.stages(2).asInstanceOf[DecisionTreeClassificationModel] + println("Learned classification tree model:\n" + treeModel.toDebugString) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index f28671f7869f..19f2d7751bc5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -18,33 +18,29 @@ // scalastyle:off println package org.apache.spark.examples.ml +import java.util.Locale + import scala.collection.mutable -import scala.language.reflectiveCalls import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.{Pipeline, PipelineStage, Transformer} import org.apache.spark.ml.classification.{DecisionTreeClassificationModel, DecisionTreeClassifier} -import org.apache.spark.ml.feature.{VectorIndexer, StringIndexer} +import org.apache.spark.ml.feature.{StringIndexer, VectorIndexer} +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} import org.apache.spark.ml.util.MetadataUtils -import org.apache.spark.mllib.evaluation.{RegressionMetrics, MulticlassMetrics} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.evaluation.{MulticlassMetrics, RegressionMetrics} import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.types.StringType -import org.apache.spark.sql.{SQLContext, DataFrame} - +import org.apache.spark.sql.{DataFrame, SparkSession} /** * An example runner for decision trees. Run with * {{{ * ./bin/run-example ml.DecisionTreeExample [options] * }}} - * Note that Decision Trees can take a large amount of memory. If the run-example command above + * Note that Decision Trees can take a large amount of memory. If the run-example command above * fails, try running via spark-submit and specifying the amount of memory as at least 1g. * For local mode, run * {{{ @@ -91,7 +87,7 @@ object DecisionTreeExample { .text(s"min info gain required to create a split, default: ${defaultParams.minInfoGain}") .action((x, c) => c.copy(minInfoGain = x)) opt[Double]("fracTest") - .text(s"fraction of data to hold out for testing. If given option testInput, " + + .text(s"fraction of data to hold out for testing. If given option testInput, " + s"this option is ignored. default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) opt[Boolean]("cacheNodeIds") @@ -110,7 +106,7 @@ object DecisionTreeExample { s"default: ${defaultParams.checkpointInterval}") .action((x, c) => c.copy(checkpointInterval = x)) opt[String]("testInput") - .text(s"input path to test dataset. If given, option fracTest is ignored." + + .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) opt[String]("dataFormat") @@ -129,24 +125,26 @@ object DecisionTreeExample { } } - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } /** Load a dataset from the given path, using the given format */ private[ml] def loadData( - sc: SparkContext, + spark: SparkSession, path: String, format: String, - expectedNumFeatures: Option[Int] = None): RDD[LabeledPoint] = { + expectedNumFeatures: Option[Int] = None): DataFrame = { + import spark.implicits._ + format match { - case "dense" => MLUtils.loadLabeledPoints(sc, path) + case "dense" => MLUtils.loadLabeledPoints(spark.sparkContext, path).toDF() case "libsvm" => expectedNumFeatures match { - case Some(numFeatures) => MLUtils.loadLibSVMFile(sc, path, numFeatures) - case None => MLUtils.loadLibSVMFile(sc, path) + case Some(numFeatures) => spark.read.option("numFeatures", numFeatures.toString) + .format("libsvm").load(path) + case None => spark.read.format("libsvm").load(path) } case _ => throw new IllegalArgumentException(s"Bad data format: $format") } @@ -158,47 +156,34 @@ object DecisionTreeExample { * @param dataFormat "libsvm" or "dense" * @param testInput Path to test dataset. * @param algo Classification or Regression - * @param fracTest Fraction of input data to hold out for testing. Ignored if testInput given. + * @param fracTest Fraction of input data to hold out for testing. Ignored if testInput given. * @return (training dataset, test dataset) */ private[ml] def loadDatasets( - sc: SparkContext, input: String, dataFormat: String, testInput: String, algo: String, fracTest: Double): (DataFrame, DataFrame) = { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession + .builder + .getOrCreate() // Load training data - val origExamples: RDD[LabeledPoint] = loadData(sc, input, dataFormat) + val origExamples: DataFrame = loadData(spark, input, dataFormat) // Load or create test set - val splits: Array[RDD[LabeledPoint]] = if (testInput != "") { + val dataframes: Array[DataFrame] = if (testInput != "") { // Load testInput. - val numFeatures = origExamples.take(1)(0).features.size - val origTestExamples: RDD[LabeledPoint] = - loadData(sc, testInput, dataFormat, Some(numFeatures)) + val numFeatures = origExamples.first().getAs[Vector](1).size + val origTestExamples: DataFrame = + loadData(spark, testInput, dataFormat, Some(numFeatures)) Array(origExamples, origTestExamples) } else { // Split input into training, test. origExamples.randomSplit(Array(1.0 - fracTest, fracTest), seed = 12345) } - // For classification, convert labels to Strings since we will index them later with - // StringIndexer. - def labelsToStrings(data: DataFrame): DataFrame = { - algo.toLowerCase match { - case "classification" => - data.withColumn("labelString", data("label").cast(StringType)) - case "regression" => - data - case _ => - throw new IllegalArgumentException("Algo ${params.algo} not supported.") - } - } - val dataframes = splits.map(_.toDF()).map(labelsToStrings) val training = dataframes(0).cache() val test = dataframes(1).cache() @@ -212,25 +197,28 @@ object DecisionTreeExample { (training, test) } - def run(params: Params) { - val conf = new SparkConf().setAppName(s"DecisionTreeExample with $params") - val sc = new SparkContext(conf) - params.checkpointDir.foreach(sc.setCheckpointDir) - val algo = params.algo.toLowerCase + def run(params: Params): Unit = { + val spark = SparkSession + .builder + .appName(s"DecisionTreeExample with $params") + .getOrCreate() + + params.checkpointDir.foreach(spark.sparkContext.setCheckpointDir) + val algo = params.algo.toLowerCase(Locale.ROOT) println(s"DecisionTreeExample with parameters:\n$params") // Load training and test data and cache it. val (training: DataFrame, test: DataFrame) = - loadDatasets(sc, params.input, params.dataFormat, params.testInput, algo, params.fracTest) + loadDatasets(params.input, params.dataFormat, params.testInput, algo, params.fracTest) - // Set up Pipeline + // Set up Pipeline. val stages = new mutable.ArrayBuffer[PipelineStage]() // (1) For classification, re-index classes. val labelColName = if (algo == "classification") "indexedLabel" else "label" if (algo == "classification") { val labelIndexer = new StringIndexer() - .setInputCol("labelString") + .setInputCol("label") .setOutputCol(labelColName) stages += labelIndexer } @@ -241,7 +229,7 @@ object DecisionTreeExample { .setOutputCol("indexedFeatures") .setMaxCategories(10) stages += featuresIndexer - // (3) Learn Decision Tree + // (3) Learn Decision Tree. val dt = algo match { case "classification" => new DecisionTreeClassifier() @@ -263,18 +251,18 @@ object DecisionTreeExample { .setMinInfoGain(params.minInfoGain) .setCacheNodeIds(params.cacheNodeIds) .setCheckpointInterval(params.checkpointInterval) - case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + case _ => throw new IllegalArgumentException(s"Algo ${params.algo} not supported.") } stages += dt val pipeline = new Pipeline().setStages(stages.toArray) - // Fit the Pipeline + // Fit the Pipeline. val startTime = System.nanoTime() val pipelineModel = pipeline.fit(training) val elapsedTime = (System.nanoTime() - startTime) / 1e9 println(s"Training time: $elapsedTime seconds") - // Get the trained Decision Tree from the fitted PipelineModel + // Get the trained Decision Tree from the fitted PipelineModel. algo match { case "classification" => val treeModel = pipelineModel.stages.last.asInstanceOf[DecisionTreeClassificationModel] @@ -290,10 +278,10 @@ object DecisionTreeExample { } else { println(treeModel) // Print model summary. } - case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + case _ => throw new IllegalArgumentException(s"Algo ${params.algo} not supported.") } - // Evaluate model on training, test data + // Evaluate model on training, test data. algo match { case "classification" => println("Training data results:") @@ -306,14 +294,14 @@ object DecisionTreeExample { println("Test data results:") evaluateRegressionModel(pipelineModel, test, labelColName) case _ => - throw new IllegalArgumentException("Algo ${params.algo} not supported.") + throw new IllegalArgumentException(s"Algo ${params.algo} not supported.") } - sc.stop() + spark.stop() } /** - * Evaluate the given ClassificationModel on data. Print the results. + * Evaluate the given ClassificationModel on data. Print the results. * @param model Must fit ClassificationModel abstraction * @param data DataFrame with "prediction" and labelColName columns * @param labelColName Name of the labelCol parameter for the model @@ -325,20 +313,20 @@ object DecisionTreeExample { data: DataFrame, labelColName: String): Unit = { val fullPredictions = model.transform(data).cache() - val predictions = fullPredictions.select("prediction").map(_.getDouble(0)) - val labels = fullPredictions.select(labelColName).map(_.getDouble(0)) - // Print number of classes for reference + val predictions = fullPredictions.select("prediction").rdd.map(_.getDouble(0)) + val labels = fullPredictions.select(labelColName).rdd.map(_.getDouble(0)) + // Print number of classes for reference. val numClasses = MetadataUtils.getNumClasses(fullPredictions.schema(labelColName)) match { case Some(n) => n case None => throw new RuntimeException( "Unknown failure when indexing labels for classification.") } - val accuracy = new MulticlassMetrics(predictions.zip(labels)).precision + val accuracy = new MulticlassMetrics(predictions.zip(labels)).accuracy println(s" Accuracy ($numClasses classes): $accuracy") } /** - * Evaluate the given RegressionModel on data. Print the results. + * Evaluate the given RegressionModel on data. Print the results. * @param model Must fit RegressionModel abstraction * @param data DataFrame with "prediction" and labelColName columns * @param labelColName Name of the labelCol parameter for the model @@ -350,8 +338,8 @@ object DecisionTreeExample { data: DataFrame, labelColName: String): Unit = { val fullPredictions = model.transform(data).cache() - val predictions = fullPredictions.select("prediction").map(_.getDouble(0)) - val labels = fullPredictions.select(labelColName).map(_.getDouble(0)) + val predictions = fullPredictions.select("prediction").rdd.map(_.getDouble(0)) + val labels = fullPredictions.select(labelColName).rdd.map(_.getDouble(0)) val RMSE = new RegressionMetrics(predictions.zip(labels)).rootMeanSquaredError println(s" Root mean squared error (RMSE): $RMSE") } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala new file mode 100644 index 000000000000..ee61200ad1d0 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeRegressionExample.scala @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.evaluation.RegressionEvaluator +import org.apache.spark.ml.feature.VectorIndexer +import org.apache.spark.ml.regression.DecisionTreeRegressionModel +import org.apache.spark.ml.regression.DecisionTreeRegressor +// $example off$ +import org.apache.spark.sql.SparkSession + +object DecisionTreeRegressionExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("DecisionTreeRegressionExample") + .getOrCreate() + + // $example on$ + // Load the data stored in LIBSVM format as a DataFrame. + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + // Automatically identify categorical features, and index them. + // Here, we treat features with > 4 distinct values as continuous. + val featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data) + + // Split the data into training and test sets (30% held out for testing). + val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + + // Train a DecisionTree model. + val dt = new DecisionTreeRegressor() + .setLabelCol("label") + .setFeaturesCol("indexedFeatures") + + // Chain indexer and tree in a Pipeline. + val pipeline = new Pipeline() + .setStages(Array(featureIndexer, dt)) + + // Train model. This also runs the indexer. + val model = pipeline.fit(trainingData) + + // Make predictions. + val predictions = model.transform(testData) + + // Select example rows to display. + predictions.select("prediction", "label", "features").show(5) + + // Select (prediction, true label) and compute test error. + val evaluator = new RegressionEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("rmse") + val rmse = evaluator.evaluate(predictions) + println("Root Mean Squared Error (RMSE) on test data = " + rmse) + + val treeModel = model.stages(1).asInstanceOf[DecisionTreeRegressionModel] + println("Learned regression tree model:\n" + treeModel.toDebugString) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println 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 3758edc56198..d94d837d10e9 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 @@ -18,13 +18,12 @@ // scalastyle:off println package org.apache.spark.examples.ml -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.classification.{ClassificationModel, Classifier, ClassifierParams} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} import org.apache.spark.ml.param.{IntParam, ParamMap} import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{Dataset, Row, SparkSession} /** * A simple example demonstrating how to write your own learning algorithm using Estimator, @@ -38,19 +37,20 @@ import org.apache.spark.sql.{DataFrame, Row, SQLContext} object DeveloperApiExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("DeveloperApiExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession + .builder + .appName("DeveloperApiExample") + .getOrCreate() + import spark.implicits._ // Prepare training data. - val training = sc.parallelize(Seq( + val training = spark.createDataFrame(Seq( LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)))) - // Create a LogisticRegression instance. This instance is an Estimator. + // Create a LogisticRegression instance. This instance is an Estimator. val lr = new MyLogisticRegression() // Print out the parameters, documentation, and any default values. println("MyLogisticRegression parameters:\n" + lr.explainParams() + "\n") @@ -58,33 +58,33 @@ object DeveloperApiExample { // We may set parameters using setter methods. lr.setMaxIter(10) - // Learn a LogisticRegression model. This uses the parameters stored in lr. + // Learn a LogisticRegression model. This uses the parameters stored in lr. val model = lr.fit(training.toDF()) // Prepare test data. - val test = sc.parallelize(Seq( + val test = spark.createDataFrame(Seq( LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) // Make predictions on test data. - val sumPredictions: Double = model.transform(test.toDF()) + val sumPredictions: Double = model.transform(test) .select("features", "label", "prediction") .collect() .map { case Row(features: Vector, label: Double, prediction: Double) => prediction }.sum assert(sumPredictions == 0.0, - "MyLogisticRegression predicted something other than 0, even though all weights are 0!") + "MyLogisticRegression predicted something other than 0, even though all coefficients are 0!") - sc.stop() + spark.stop() } } /** * Example of defining a parameter trait for a user-defined type of [[Classifier]]. * - * NOTE: This is private since it is an example. In practice, you may not want it to be private. + * NOTE: This is private since it is an example. In practice, you may not want it to be private. */ private trait MyLogisticRegressionParams extends ClassifierParams { @@ -96,7 +96,7 @@ private trait MyLogisticRegressionParams extends ClassifierParams { * - def getMyParamName * - def setMyParamName * Here, we have a trait to be mixed in with the Estimator and Model (MyLogisticRegression - * and MyLogisticRegressionModel). We place the setter (setMaxIter) method in the Estimator + * and MyLogisticRegressionModel). We place the setter (setMaxIter) method in the Estimator * class since the maxIter parameter is only used during training (not in the Model). */ val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") @@ -106,7 +106,7 @@ private trait MyLogisticRegressionParams extends ClassifierParams { /** * Example of defining a type of [[Classifier]]. * - * NOTE: This is private since it is an example. In practice, you may not want it to be private. + * NOTE: This is private since it is an example. In practice, you may not want it to be private. */ private class MyLogisticRegression(override val uid: String) extends Classifier[Vector, MyLogisticRegression, MyLogisticRegressionModel] @@ -120,16 +120,16 @@ private class MyLogisticRegression(override val uid: String) def setMaxIter(value: Int): this.type = set(maxIter, value) // This method is used by fit() - override protected def train(dataset: DataFrame): MyLogisticRegressionModel = { + override protected def train(dataset: Dataset[_]): MyLogisticRegressionModel = { // Extract columns from data using helper method. val oldDataset = extractLabeledPoints(dataset) - // Do learning to estimate the weight vector. + // Do learning to estimate the coefficients vector. val numFeatures = oldDataset.take(1)(0).features.size - val weights = Vectors.zeros(numFeatures) // Learning would happen here. + val coefficients = Vectors.zeros(numFeatures) // Learning would happen here. // Create a model, and return it. - new MyLogisticRegressionModel(uid, weights).setParent(this) + new MyLogisticRegressionModel(uid, coefficients).setParent(this) } override def copy(extra: ParamMap): MyLogisticRegression = defaultCopy(extra) @@ -138,11 +138,11 @@ private class MyLogisticRegression(override val uid: String) /** * Example of defining a type of [[ClassificationModel]]. * - * NOTE: This is private since it is an example. In practice, you may not want it to be private. + * NOTE: This is private since it is an example. In practice, you may not want it to be private. */ private class MyLogisticRegressionModel( override val uid: String, - val weights: Vector) + val coefficients: Vector) extends ClassificationModel[Vector, MyLogisticRegressionModel] with MyLogisticRegressionParams { @@ -163,17 +163,17 @@ private class MyLogisticRegressionModel( * confidence for that label. */ override protected def predictRaw(features: Vector): Vector = { - val margin = BLAS.dot(features, weights) + val margin = BLAS.dot(features, coefficients) // There are 2 classes (binary classification), so we return a length-2 vector, // where index i corresponds to class i (i = 0, 1). Vectors.dense(-margin, margin) } - /** Number of classes the label can take. 2 indicates binary classification. */ + /** Number of classes the label can take. 2 indicates binary classification. */ override val numClasses: Int = 2 /** Number of features the model was trained on. */ - override val numFeatures: Int = weights.size + override val numFeatures: Int = coefficients.size /** * Create a copy of the model. @@ -182,7 +182,7 @@ private class MyLogisticRegressionModel( * This is used for the default implementation of [[transform()]]. */ override def copy(extra: ParamMap): MyLogisticRegressionModel = { - copyValues(new MyLogisticRegressionModel(uid, weights), extra).setParent(parent) + copyValues(new MyLogisticRegressionModel(uid, coefficients), extra).setParent(parent) } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala new file mode 100644 index 000000000000..c0ffc01934b6 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ElementwiseProductExample.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.ElementwiseProduct +import org.apache.spark.ml.linalg.Vectors +// $example off$ +import org.apache.spark.sql.SparkSession + +object ElementwiseProductExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("ElementwiseProductExample") + .getOrCreate() + + // $example on$ + // Create some vector data; also works for sparse vectors + val dataFrame = spark.createDataFrame(Seq( + ("a", Vectors.dense(1.0, 2.0, 3.0)), + ("b", Vectors.dense(4.0, 5.0, 6.0)))).toDF("id", "vector") + + val transformingVector = Vectors.dense(0.0, 1.0, 2.0) + val transformer = new ElementwiseProduct() + .setScalingVec(transformingVector) + .setInputCol("vector") + .setOutputCol("transformedVector") + + // Batch transform the vectors to create new column: + transformer.transform(dataFrame).show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala new file mode 100644 index 000000000000..f18d86e1a692 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/EstimatorTransformerParamExample.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.sql.Row +// $example off$ +import org.apache.spark.sql.SparkSession + +object EstimatorTransformerParamExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("EstimatorTransformerParamExample") + .getOrCreate() + + // $example on$ + // Prepare training data from a list of (label, features) tuples. + val training = spark.createDataFrame(Seq( + (1.0, Vectors.dense(0.0, 1.1, 0.1)), + (0.0, Vectors.dense(2.0, 1.0, -1.0)), + (0.0, Vectors.dense(2.0, 1.3, 1.0)), + (1.0, Vectors.dense(0.0, 1.2, -0.5)) + )).toDF("label", "features") + + // Create a LogisticRegression instance. This instance is an Estimator. + val lr = new LogisticRegression() + // Print out the parameters, documentation, and any default values. + println("LogisticRegression parameters:\n" + lr.explainParams() + "\n") + + // We may set parameters using setter methods. + lr.setMaxIter(10) + .setRegParam(0.01) + + // Learn a LogisticRegression model. This uses the parameters stored in lr. + val model1 = lr.fit(training) + // 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 + // LogisticRegression instance. + println("Model 1 was fit using parameters: " + model1.parent.extractParamMap) + + // We may alternatively specify parameters using a ParamMap, + // which supports several methods for specifying parameters. + val paramMap = ParamMap(lr.maxIter -> 20) + .put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. + .put(lr.regParam -> 0.1, lr.threshold -> 0.55) // Specify multiple Params. + + // One can also combine ParamMaps. + val paramMap2 = ParamMap(lr.probabilityCol -> "myProbability") // Change output column name. + val paramMapCombined = paramMap ++ paramMap2 + + // Now learn a new model using the paramMapCombined parameters. + // paramMapCombined overrides all parameters set earlier via lr.set* methods. + val model2 = lr.fit(training, paramMapCombined) + println("Model 2 was fit using parameters: " + model2.parent.extractParamMap) + + // Prepare test data. + val test = spark.createDataFrame(Seq( + (1.0, Vectors.dense(-1.0, 1.5, 1.3)), + (0.0, Vectors.dense(3.0, 2.0, -0.1)), + (1.0, Vectors.dense(0.0, 2.2, -1.5)) + )).toDF("label", "features") + + // Make predictions on test data using the Transformer.transform() method. + // 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) + .select("features", "label", "myProbability", "prediction") + .collect() + .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) => + println(s"($features, $label) -> prob=$prob, prediction=$prediction") + } + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala new file mode 100644 index 000000000000..59110d70de55 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml + +// scalastyle:off println + +// $example on$ +import org.apache.spark.ml.fpm.FPGrowth +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating FP-Growth. + * Run with + * {{{ + * bin/run-example ml.FPGrowthExample + * }}} + */ +object FPGrowthExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + import spark.implicits._ + + // $example on$ + val dataset = spark.createDataset(Seq( + "1 2 5", + "1 2 3 5", + "1 2") + ).map(t => t.split(" ")).toDF("items") + + val fpgrowth = new FPGrowth().setItemsCol("items").setMinSupport(0.5).setMinConfidence(0.6) + val model = fpgrowth.fit(dataset) + + // Display frequent itemsets. + model.freqItemsets.show() + + // Display generated association rules. + model.associationRules.show() + + // transform examines the input items against all the association rules and summarize the + // consequents as prediction + model.transform(dataset).show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/FeatureHasherExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/FeatureHasherExample.scala new file mode 100644 index 000000000000..1aed10bfb2d3 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/FeatureHasherExample.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.FeatureHasher +// $example off$ +import org.apache.spark.sql.SparkSession + +object FeatureHasherExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("FeatureHasherExample") + .getOrCreate() + + // $example on$ + val dataset = spark.createDataFrame(Seq( + (2.2, true, "1", "foo"), + (3.3, false, "2", "bar"), + (4.4, false, "3", "baz"), + (5.5, false, "4", "foo") + )).toDF("real", "bool", "stringNum", "string") + + val hasher = new FeatureHasher() + .setInputCols("real", "bool", "stringNum", "string") + .setOutputCol("features") + + val featurized = hasher.transform(dataset) + featurized.show(false) + // $example off$ + + spark.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala index f4a15f806ea8..8f3ce4b315bd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala @@ -18,18 +18,18 @@ // scalastyle:off println package org.apache.spark.examples.ml +import java.util.Locale + import scala.collection.mutable -import scala.language.reflectiveCalls import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.{Pipeline, PipelineStage} import org.apache.spark.ml.classification.{GBTClassificationModel, GBTClassifier} import org.apache.spark.ml.feature.{StringIndexer, VectorIndexer} import org.apache.spark.ml.regression.{GBTRegressionModel, GBTRegressor} -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, SparkSession} /** @@ -37,7 +37,7 @@ import org.apache.spark.sql.DataFrame * {{{ * ./bin/run-example ml.GBTExample [options] * }}} - * Decision Trees and ensembles can take a large amount of memory. If the run-example command + * Decision Trees and ensembles can take a large amount of memory. If the run-example command * above fails, try running via spark-submit and specifying the amount of memory as at least 1g. * For local mode, run * {{{ @@ -88,7 +88,7 @@ object GBTExample { .text(s"number of trees in ensemble, default: ${defaultParams.maxIter}") .action((x, c) => c.copy(maxIter = x)) opt[Double]("fracTest") - .text(s"fraction of data to hold out for testing. If given option testInput, " + + .text(s"fraction of data to hold out for testing. If given option testInput, " + s"this option is ignored. default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) opt[Boolean]("cacheNodeIds") @@ -109,7 +109,7 @@ object GBTExample { s"default: ${defaultParams.checkpointInterval}") .action((x, c) => c.copy(checkpointInterval = x)) opt[String]("testInput") - .text(s"input path to test dataset. If given, option fracTest is ignored." + + .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) opt[String]("dataFormat") @@ -128,23 +128,25 @@ object GBTExample { } } - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { - val conf = new SparkConf().setAppName(s"GBTExample with $params") - val sc = new SparkContext(conf) - params.checkpointDir.foreach(sc.setCheckpointDir) - val algo = params.algo.toLowerCase + def run(params: Params): Unit = { + val spark = SparkSession + .builder + .appName(s"GBTExample with $params") + .getOrCreate() + + params.checkpointDir.foreach(spark.sparkContext.setCheckpointDir) + val algo = params.algo.toLowerCase(Locale.ROOT) println(s"GBTExample with parameters:\n$params") // Load training and test data and cache it. - val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(sc, params.input, + val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(params.input, params.dataFormat, params.testInput, algo, params.fracTest) // Set up Pipeline @@ -153,7 +155,7 @@ object GBTExample { val labelColName = if (algo == "classification") "indexedLabel" else "label" if (algo == "classification") { val labelIndexer = new StringIndexer() - .setInputCol("labelString") + .setInputCol("label") .setOutputCol(labelColName) stages += labelIndexer } @@ -164,7 +166,7 @@ object GBTExample { .setOutputCol("indexedFeatures") .setMaxCategories(10) stages += featuresIndexer - // (3) Learn GBT + // (3) Learn GBT. val dt = algo match { case "classification" => new GBTClassifier() @@ -188,18 +190,18 @@ object GBTExample { .setCacheNodeIds(params.cacheNodeIds) .setCheckpointInterval(params.checkpointInterval) .setMaxIter(params.maxIter) - case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + case _ => throw new IllegalArgumentException(s"Algo ${params.algo} not supported.") } stages += dt val pipeline = new Pipeline().setStages(stages.toArray) - // Fit the Pipeline + // Fit the Pipeline. val startTime = System.nanoTime() val pipelineModel = pipeline.fit(training) val elapsedTime = (System.nanoTime() - startTime) / 1e9 println(s"Training time: $elapsedTime seconds") - // Get the trained GBT from the fitted PipelineModel + // Get the trained GBT from the fitted PipelineModel. algo match { case "classification" => val rfModel = pipelineModel.stages.last.asInstanceOf[GBTClassificationModel] @@ -215,10 +217,10 @@ object GBTExample { } else { println(rfModel) // Print model summary. } - case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + case _ => throw new IllegalArgumentException(s"Algo ${params.algo} not supported.") } - // Evaluate model on training, test data + // Evaluate model on training, test data. algo match { case "classification" => println("Training data results:") @@ -231,10 +233,10 @@ object GBTExample { println("Test data results:") DecisionTreeExample.evaluateRegressionModel(pipelineModel, test, labelColName) case _ => - throw new IllegalArgumentException("Algo ${params.algo} not supported.") + throw new IllegalArgumentException(s"Algo ${params.algo} not supported.") } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala new file mode 100644 index 000000000000..5e4bea4c4fb6 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GaussianMixtureExample.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml + +// scalastyle:off println + +// $example on$ +import org.apache.spark.ml.clustering.GaussianMixture +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating Gaussian Mixture Model (GMM). + * Run with + * {{{ + * bin/run-example ml.GaussianMixtureExample + * }}} + */ +object GaussianMixtureExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + + // $example on$ + // Loads data + val dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") + + // Trains Gaussian Mixture Model + val gmm = new GaussianMixture() + .setK(2) + val model = gmm.fit(dataset) + + // output parameters of mixture model model + for (i <- 0 until model.getK) { + println(s"Gaussian $i:\nweight=${model.weights(i)}\n" + + s"mu=${model.gaussians(i).mean}\nsigma=\n${model.gaussians(i).cov}\n") + } + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala new file mode 100644 index 000000000000..1b86d7cad0b3 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GeneralizedLinearRegressionExample.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.regression.GeneralizedLinearRegression +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating generalized linear regression. + * Run with + * {{{ + * bin/run-example ml.GeneralizedLinearRegressionExample + * }}} + */ + +object GeneralizedLinearRegressionExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("GeneralizedLinearRegressionExample") + .getOrCreate() + + // $example on$ + // Load training data + val dataset = spark.read.format("libsvm") + .load("data/mllib/sample_linear_regression_data.txt") + + val glr = new GeneralizedLinearRegression() + .setFamily("gaussian") + .setLink("identity") + .setMaxIter(10) + .setRegParam(0.3) + + // Fit the model + val model = glr.fit(dataset) + + // Print the coefficients and intercept for generalized linear regression model + println(s"Coefficients: ${model.coefficients}") + println(s"Intercept: ${model.intercept}") + + // Summarize the model over the training set and print out some metrics + val summary = model.summary + println(s"Coefficient Standard Errors: ${summary.coefficientStandardErrors.mkString(",")}") + println(s"T Values: ${summary.tValues.mkString(",")}") + println(s"P Values: ${summary.pValues.mkString(",")}") + println(s"Dispersion: ${summary.dispersion}") + println(s"Null Deviance: ${summary.nullDeviance}") + println(s"Residual Degree Of Freedom Null: ${summary.residualDegreeOfFreedomNull}") + println(s"Deviance: ${summary.deviance}") + println(s"Residual Degree Of Freedom: ${summary.residualDegreeOfFreedom}") + println(s"AIC: ${summary.aic}") + println("Deviance Residuals: ") + summary.residuals().show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala new file mode 100644 index 000000000000..9a39acfbf37e --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeClassifierExample.scala @@ -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. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.classification.{GBTClassificationModel, GBTClassifier} +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator +import org.apache.spark.ml.feature.{IndexToString, StringIndexer, VectorIndexer} +// $example off$ +import org.apache.spark.sql.SparkSession + +object GradientBoostedTreeClassifierExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("GradientBoostedTreeClassifierExample") + .getOrCreate() + + // $example on$ + // Load and parse the data file, converting it to a DataFrame. + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + // Index labels, adding metadata to the label column. + // Fit on whole dataset to include all labels in index. + val labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data) + // Automatically identify categorical features, and index them. + // Set maxCategories so features with > 4 distinct values are treated as continuous. + val featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data) + + // Split the data into training and test sets (30% held out for testing). + val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + + // Train a GBT model. + val gbt = new GBTClassifier() + .setLabelCol("indexedLabel") + .setFeaturesCol("indexedFeatures") + .setMaxIter(10) + + // Convert indexed labels back to original labels. + val labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labels) + + // Chain indexers and GBT in a Pipeline. + val pipeline = new Pipeline() + .setStages(Array(labelIndexer, featureIndexer, gbt, labelConverter)) + + // Train model. This also runs the indexers. + val model = pipeline.fit(trainingData) + + // Make predictions. + val predictions = model.transform(testData) + + // Select example rows to display. + predictions.select("predictedLabel", "label", "features").show(5) + + // Select (prediction, true label) and compute test error. + val evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("accuracy") + val accuracy = evaluator.evaluate(predictions) + println("Test Error = " + (1.0 - accuracy)) + + val gbtModel = model.stages(2).asInstanceOf[GBTClassificationModel] + println("Learned classification GBT model:\n" + gbtModel.toDebugString) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala new file mode 100644 index 000000000000..e53aab7f326d --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GradientBoostedTreeRegressorExample.scala @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.evaluation.RegressionEvaluator +import org.apache.spark.ml.feature.VectorIndexer +import org.apache.spark.ml.regression.{GBTRegressionModel, GBTRegressor} +// $example off$ +import org.apache.spark.sql.SparkSession + +object GradientBoostedTreeRegressorExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("GradientBoostedTreeRegressorExample") + .getOrCreate() + + // $example on$ + // Load and parse the data file, converting it to a DataFrame. + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + // Automatically identify categorical features, and index them. + // Set maxCategories so features with > 4 distinct values are treated as continuous. + val featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data) + + // Split the data into training and test sets (30% held out for testing). + val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + + // Train a GBT model. + val gbt = new GBTRegressor() + .setLabelCol("label") + .setFeaturesCol("indexedFeatures") + .setMaxIter(10) + + // Chain indexer and GBT in a Pipeline. + val pipeline = new Pipeline() + .setStages(Array(featureIndexer, gbt)) + + // Train model. This also runs the indexer. + val model = pipeline.fit(trainingData) + + // Make predictions. + val predictions = model.transform(testData) + + // Select example rows to display. + predictions.select("prediction", "label", "features").show(5) + + // Select (prediction, true label) and compute test error. + val evaluator = new RegressionEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("rmse") + val rmse = evaluator.evaluate(predictions) + println("Root Mean Squared Error (RMSE) on test data = " + rmse) + + val gbtModel = model.stages(1).asInstanceOf[GBTRegressionModel] + println("Learned regression GBT model:\n" + gbtModel.toDebugString) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ImputerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ImputerExample.scala new file mode 100644 index 000000000000..49e98d0c622c --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ImputerExample.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.Imputer +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating Imputer. + * Run with: + * bin/run-example ml.ImputerExample + */ +object ImputerExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession.builder + .appName("ImputerExample") + .getOrCreate() + + // $example on$ + val df = spark.createDataFrame(Seq( + (1.0, Double.NaN), + (2.0, Double.NaN), + (Double.NaN, 3.0), + (4.0, 4.0), + (5.0, 5.0) + )).toDF("a", "b") + + val imputer = new Imputer() + .setInputCols(Array("a", "b")) + .setOutputCols(Array("out_a", "out_b")) + + val model = imputer.fit(df) + model.transform(df).show() + // $example off$ + + spark.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala new file mode 100644 index 000000000000..2940682c3280 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/IndexToStringExample.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.attribute.Attribute +import org.apache.spark.ml.feature.{IndexToString, StringIndexer} +// $example off$ +import org.apache.spark.sql.SparkSession + +object IndexToStringExample { + def main(args: Array[String]) { + val spark = SparkSession + .builder + .appName("IndexToStringExample") + .getOrCreate() + + // $example on$ + val df = spark.createDataFrame(Seq( + (0, "a"), + (1, "b"), + (2, "c"), + (3, "a"), + (4, "a"), + (5, "c") + )).toDF("id", "category") + + val indexer = new StringIndexer() + .setInputCol("category") + .setOutputCol("categoryIndex") + .fit(df) + val indexed = indexer.transform(df) + + println(s"Transformed string column '${indexer.getInputCol}' " + + s"to indexed column '${indexer.getOutputCol}'") + indexed.show() + + val inputColSchema = indexed.schema(indexer.getOutputCol) + println(s"StringIndexer will store labels in output column metadata: " + + s"${Attribute.fromStructField(inputColSchema).toString}\n") + + val converter = new IndexToString() + .setInputCol("categoryIndex") + .setOutputCol("originalCategory") + + val converted = converter.transform(indexed) + + println(s"Transformed indexed column '${converter.getInputCol}' back to original string " + + s"column '${converter.getOutputCol}' using labels in metadata") + converted.select("id", "categoryIndex", "originalCategory").show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/InteractionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/InteractionExample.scala new file mode 100644 index 000000000000..8113c992b1d6 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/InteractionExample.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.Interaction +import org.apache.spark.ml.feature.VectorAssembler +// $example off$ +import org.apache.spark.sql.SparkSession + +object InteractionExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("InteractionExample") + .getOrCreate() + + // $example on$ + val df = spark.createDataFrame(Seq( + (1, 1, 2, 3, 8, 4, 5), + (2, 4, 3, 8, 7, 9, 8), + (3, 6, 1, 9, 2, 3, 6), + (4, 10, 8, 6, 9, 4, 5), + (5, 9, 2, 7, 10, 7, 3), + (6, 1, 1, 4, 2, 8, 4) + )).toDF("id1", "id2", "id3", "id4", "id5", "id6", "id7") + + val assembler1 = new VectorAssembler(). + setInputCols(Array("id2", "id3", "id4")). + setOutputCol("vec1") + + val assembled1 = assembler1.transform(df) + + val assembler2 = new VectorAssembler(). + setInputCols(Array("id5", "id6", "id7")). + setOutputCol("vec2") + + val assembled2 = assembler2.transform(assembled1).select("id1", "vec1", "vec2") + + val interaction = new Interaction() + .setInputCols(Array("id1", "vec1", "vec2")) + .setOutputCol("interactedCol") + + val interacted = interaction.transform(assembled2) + + interacted.show(truncate = false) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/IsotonicRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/IsotonicRegressionExample.scala new file mode 100644 index 000000000000..9bac16ec769a --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/IsotonicRegressionExample.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.regression.IsotonicRegression +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating Isotonic Regression. + * Run with + * {{{ + * bin/run-example ml.IsotonicRegressionExample + * }}} + */ +object IsotonicRegressionExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + + // $example on$ + // Loads data. + val dataset = spark.read.format("libsvm") + .load("data/mllib/sample_isotonic_regression_libsvm_data.txt") + + // Trains an isotonic regression model. + val ir = new IsotonicRegression() + val model = ir.fit(dataset) + + println(s"Boundaries in increasing order: ${model.boundaries}\n") + println(s"Predictions associated with the boundaries: ${model.predictions}\n") + + // Makes predictions. + model.transform(dataset).show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala index 5ce38462d118..a1d19e138ded 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/KMeansExample.scala @@ -17,57 +17,46 @@ package org.apache.spark.examples.ml -import org.apache.spark.{SparkContext, SparkConf} -import org.apache.spark.mllib.linalg.{VectorUDT, Vectors} -import org.apache.spark.ml.clustering.KMeans -import org.apache.spark.sql.{Row, SQLContext} -import org.apache.spark.sql.types.{StructField, StructType} +// scalastyle:off println +// $example on$ +import org.apache.spark.ml.clustering.KMeans +// $example off$ +import org.apache.spark.sql.SparkSession /** - * An example demonstrating a k-means clustering. + * An example demonstrating k-means clustering. * Run with * {{{ - * bin/run-example ml.KMeansExample + * bin/run-example ml.KMeansExample * }}} */ object KMeansExample { - final val FEATURES_COL = "features" - def main(args: Array[String]): Unit = { - if (args.length != 2) { - // scalastyle:off println - System.err.println("Usage: ml.KMeansExample ") - // scalastyle:on println - System.exit(1) - } - val input = args(0) - val k = args(1).toInt + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() - // Creates a Spark context and a SQL context - val conf = new SparkConf().setAppName(s"${this.getClass.getSimpleName}") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + // $example on$ + // Loads data. + val dataset = spark.read.format("libsvm").load("data/mllib/sample_kmeans_data.txt") - // Loads data - val rowRDD = sc.textFile(input).filter(_.nonEmpty) - .map(_.split(" ").map(_.toDouble)).map(Vectors.dense).map(Row(_)) - val schema = StructType(Array(StructField(FEATURES_COL, new VectorUDT, false))) - val dataset = sqlContext.createDataFrame(rowRDD, schema) - - // Trains a k-means model - val kmeans = new KMeans() - .setK(k) - .setFeaturesCol(FEATURES_COL) + // Trains a k-means model. + val kmeans = new KMeans().setK(2).setSeed(1L) val model = kmeans.fit(dataset) - // Shows the result - // scalastyle:off println - println("Final Centers: ") + // Evaluate clustering by computing Within Set Sum of Squared Errors. + val WSSSE = model.computeCost(dataset) + println(s"Within Set Sum of Squared Errors = $WSSSE") + + // Shows the result. + println("Cluster Centers: ") model.clusterCenters.foreach(println) - // scalastyle:on println + // $example off$ - sc.stop() + spark.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala new file mode 100644 index 000000000000..4215d37cb59d --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LDAExample.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml + +// scalastyle:off println +// $example on$ +import org.apache.spark.ml.clustering.LDA +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating LDA. + * Run with + * {{{ + * bin/run-example ml.LDAExample + * }}} + */ +object LDAExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + + // $example on$ + // Loads data. + val dataset = spark.read.format("libsvm") + .load("data/mllib/sample_lda_libsvm_data.txt") + + // Trains a LDA model. + val lda = new LDA().setK(10).setMaxIter(10) + val model = lda.fit(dataset) + + val ll = model.logLikelihood(dataset) + val lp = model.logPerplexity(dataset) + println(s"The lower bound on the log likelihood of the entire corpus: $ll") + println(s"The upper bound on perplexity: $lp") + + // Describe topics. + val topics = model.describeTopics(3) + println("The topics described by their top-weighted terms:") + topics.show(false) + + // Shows the result. + val transformed = model.transform(dataset) + transformed.show(false) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala index b73299fb12d3..6903a1c298ce 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala @@ -18,16 +18,11 @@ // scalastyle:off println package org.apache.spark.examples.ml -import scala.collection.mutable -import scala.language.reflectiveCalls - import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.examples.mllib.AbstractParams -import org.apache.spark.ml.{Pipeline, PipelineStage} -import org.apache.spark.ml.regression.{LinearRegression, LinearRegressionModel} -import org.apache.spark.sql.DataFrame +import org.apache.spark.ml.regression.LinearRegression +import org.apache.spark.sql.{DataFrame, SparkSession} /** * An example runner for linear regression with elastic-net (mixing L1/L2) regularization. @@ -76,11 +71,11 @@ object LinearRegressionExample { s"to higher accuracy with the cost of more iterations, default: ${defaultParams.tol}") .action((x, c) => c.copy(tol = x)) opt[Double]("fracTest") - .text(s"fraction of data to hold out for testing. If given option testInput, " + + .text(s"fraction of data to hold out for testing. If given option testInput, " + s"this option is ignored. default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) opt[String]("testInput") - .text(s"input path to test dataset. If given, option fracTest is ignored." + + .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) opt[String]("dataFormat") @@ -99,21 +94,22 @@ object LinearRegressionExample { } } - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { - val conf = new SparkConf().setAppName(s"LinearRegressionExample with $params") - val sc = new SparkContext(conf) + def run(params: Params): Unit = { + val spark = SparkSession + .builder + .appName(s"LinearRegressionExample with $params") + .getOrCreate() println(s"LinearRegressionExample with parameters:\n$params") // Load training and test data and cache it. - val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(sc, params.input, + val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(params.input, params.dataFormat, params.testInput, "regression", params.fracTest) val lir = new LinearRegression() @@ -131,14 +127,14 @@ object LinearRegressionExample { println(s"Training time: $elapsedTime seconds") // Print the weights and intercept for linear regression. - println(s"Weights: ${lirModel.weights} Intercept: ${lirModel.intercept}") + println(s"Weights: ${lirModel.coefficients} Intercept: ${lirModel.intercept}") println("Training data results:") DecisionTreeExample.evaluateRegressionModel(lirModel, training, "label") println("Test data results:") DecisionTreeExample.evaluateRegressionModel(lirModel, test, "label") - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala new file mode 100644 index 000000000000..4540a8d72812 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionWithElasticNetExample.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.regression.LinearRegression +// $example off$ +import org.apache.spark.sql.SparkSession + +object LinearRegressionWithElasticNetExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("LinearRegressionWithElasticNetExample") + .getOrCreate() + + // $example on$ + // Load training data + val training = spark.read.format("libsvm") + .load("data/mllib/sample_linear_regression_data.txt") + + val lr = new LinearRegression() + .setMaxIter(10) + .setRegParam(0.3) + .setElasticNetParam(0.8) + + // Fit the model + val lrModel = lr.fit(training) + + // Print the coefficients and intercept for linear regression + println(s"Coefficients: ${lrModel.coefficients} Intercept: ${lrModel.intercept}") + + // Summarize the model over the training set and print out some metrics + val trainingSummary = lrModel.summary + println(s"numIterations: ${trainingSummary.totalIterations}") + println(s"objectiveHistory: [${trainingSummary.objectiveHistory.mkString(",")}]") + trainingSummary.residuals.show() + println(s"RMSE: ${trainingSummary.rootMeanSquaredError}") + println(s"r2: ${trainingSummary.r2}") + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearSVCExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearSVCExample.scala new file mode 100644 index 000000000000..5f43e65712b5 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearSVCExample.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.classification.LinearSVC +// $example off$ +import org.apache.spark.sql.SparkSession + +object LinearSVCExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("LinearSVCExample") + .getOrCreate() + + // $example on$ + // Load training data + val training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + val lsvc = new LinearSVC() + .setMaxIter(10) + .setRegParam(0.1) + + // Fit the model + val lsvcModel = lsvc.fit(training) + + // Print the coefficients and intercept for linear svc + println(s"Coefficients: ${lsvcModel.coefficients} Intercept: ${lsvcModel.intercept}") + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala index 8e3760ddb50a..bd6cc8cff234 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala @@ -19,16 +19,14 @@ package org.apache.spark.examples.ml import scala.collection.mutable -import scala.language.reflectiveCalls import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.{Pipeline, PipelineStage} import org.apache.spark.ml.classification.{LogisticRegression, LogisticRegressionModel} import org.apache.spark.ml.feature.StringIndexer -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, SparkSession} /** * An example runner for logistic regression with elastic-net (mixing L1/L2) regularization. @@ -81,11 +79,11 @@ object LogisticRegressionExample { s"to higher accuracy with the cost of more iterations, default: ${defaultParams.tol}") .action((x, c) => c.copy(tol = x)) opt[Double]("fracTest") - .text(s"fraction of data to hold out for testing. If given option testInput, " + + .text(s"fraction of data to hold out for testing. If given option testInput, " + s"this option is ignored. default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) opt[String]("testInput") - .text(s"input path to test dataset. If given, option fracTest is ignored." + + .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) opt[String]("dataFormat") @@ -104,28 +102,29 @@ object LogisticRegressionExample { } } - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { - val conf = new SparkConf().setAppName(s"LogisticRegressionExample with $params") - val sc = new SparkContext(conf) + def run(params: Params): Unit = { + val spark = SparkSession + .builder + .appName(s"LogisticRegressionExample with $params") + .getOrCreate() println(s"LogisticRegressionExample with parameters:\n$params") // Load training and test data and cache it. - val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(sc, params.input, + val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(params.input, params.dataFormat, params.testInput, "classification", params.fracTest) - // Set up Pipeline + // Set up Pipeline. val stages = new mutable.ArrayBuffer[PipelineStage]() val labelIndexer = new StringIndexer() - .setInputCol("labelString") + .setInputCol("label") .setOutputCol("indexedLabel") stages += labelIndexer @@ -141,7 +140,7 @@ object LogisticRegressionExample { stages += lor val pipeline = new Pipeline().setStages(stages.toArray) - // Fit the Pipeline + // Fit the Pipeline. val startTime = System.nanoTime() val pipelineModel = pipeline.fit(training) val elapsedTime = (System.nanoTime() - startTime) / 1e9 @@ -149,14 +148,14 @@ object LogisticRegressionExample { val lorModel = pipelineModel.stages.last.asInstanceOf[LogisticRegressionModel] // Print the weights and intercept for logistic regression. - println(s"Weights: ${lorModel.weights} Intercept: ${lorModel.intercept}") + println(s"Weights: ${lorModel.coefficients} Intercept: ${lorModel.intercept}") println("Training data results:") DecisionTreeExample.evaluateClassificationModel(pipelineModel, training, "indexedLabel") println("Test data results:") DecisionTreeExample.evaluateClassificationModel(pipelineModel, test, "indexedLabel") - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala new file mode 100644 index 000000000000..1740a0d3f9d1 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionSummaryExample.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.classification.{BinaryLogisticRegressionSummary, LogisticRegression} +// $example off$ +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.functions.max + +object LogisticRegressionSummaryExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("LogisticRegressionSummaryExample") + .getOrCreate() + import spark.implicits._ + + // Load training data + val training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + val lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.3) + .setElasticNetParam(0.8) + + // Fit the model + val lrModel = lr.fit(training) + + // $example on$ + // Extract the summary from the returned LogisticRegressionModel instance trained in the earlier + // example + val trainingSummary = lrModel.summary + + // Obtain the objective per iteration. + val objectiveHistory = trainingSummary.objectiveHistory + println("objectiveHistory:") + objectiveHistory.foreach(loss => println(loss)) + + // Obtain the metrics useful to judge performance on test data. + // We cast the summary to a BinaryLogisticRegressionSummary since the problem is a + // binary classification problem. + val binarySummary = trainingSummary.asInstanceOf[BinaryLogisticRegressionSummary] + + // Obtain the receiver-operating characteristic as a dataframe and areaUnderROC. + val roc = binarySummary.roc + roc.show() + println(s"areaUnderROC: ${binarySummary.areaUnderROC}") + + // Set the model threshold to maximize F-Measure + val fMeasure = binarySummary.fMeasureByThreshold + val maxFMeasure = fMeasure.select(max("F-Measure")).head().getDouble(0) + val bestThreshold = fMeasure.where($"F-Measure" === maxFMeasure) + .select("threshold").head().getDouble(0) + lrModel.setThreshold(bestThreshold) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala new file mode 100644 index 000000000000..18471049087d --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionWithElasticNetExample.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.classification.LogisticRegression +// $example off$ +import org.apache.spark.sql.SparkSession + +object LogisticRegressionWithElasticNetExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("LogisticRegressionWithElasticNetExample") + .getOrCreate() + + // $example on$ + // Load training data + val training = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + val lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.3) + .setElasticNetParam(0.8) + + // Fit the model + val lrModel = lr.fit(training) + + // Print the coefficients and intercept for logistic regression + println(s"Coefficients: ${lrModel.coefficients} Intercept: ${lrModel.intercept}") + + // We can also use the multinomial family for binary classification + val mlr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.3) + .setElasticNetParam(0.8) + .setFamily("multinomial") + + val mlrModel = mlr.fit(training) + + // Print the coefficients and intercepts for logistic regression with multinomial family + println(s"Multinomial coefficients: ${mlrModel.coefficientMatrix}") + println(s"Multinomial intercepts: ${mlrModel.interceptVector}") + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala new file mode 100644 index 000000000000..85d071369d9c --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MaxAbsScalerExample.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.MaxAbsScaler +import org.apache.spark.ml.linalg.Vectors +// $example off$ +import org.apache.spark.sql.SparkSession + +object MaxAbsScalerExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("MaxAbsScalerExample") + .getOrCreate() + + // $example on$ + val dataFrame = spark.createDataFrame(Seq( + (0, Vectors.dense(1.0, 0.1, -8.0)), + (1, Vectors.dense(2.0, 1.0, -4.0)), + (2, Vectors.dense(4.0, 10.0, 8.0)) + )).toDF("id", "features") + + val scaler = new MaxAbsScaler() + .setInputCol("features") + .setOutputCol("scaledFeatures") + + // Compute summary statistics and generate MaxAbsScalerModel + val scalerModel = scaler.fit(dataFrame) + + // rescale each feature to range [-1, 1] + val scaledData = scalerModel.transform(dataFrame) + scaledData.select("features", "scaledFeatures").show() + // $example off$ + + spark.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala new file mode 100644 index 000000000000..8515821a9413 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinHashLSHExample.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.MinHashLSH +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.functions.col +// $example off$ + +/** + * An example demonstrating MinHashLSH. + * Run with: + * bin/run-example ml.MinHashLSHExample + */ +object MinHashLSHExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession + val spark = SparkSession + .builder + .appName("MinHashLSHExample") + .getOrCreate() + + // $example on$ + val dfA = spark.createDataFrame(Seq( + (0, Vectors.sparse(6, Seq((0, 1.0), (1, 1.0), (2, 1.0)))), + (1, Vectors.sparse(6, Seq((2, 1.0), (3, 1.0), (4, 1.0)))), + (2, Vectors.sparse(6, Seq((0, 1.0), (2, 1.0), (4, 1.0)))) + )).toDF("id", "features") + + val dfB = spark.createDataFrame(Seq( + (3, Vectors.sparse(6, Seq((1, 1.0), (3, 1.0), (5, 1.0)))), + (4, Vectors.sparse(6, Seq((2, 1.0), (3, 1.0), (5, 1.0)))), + (5, Vectors.sparse(6, Seq((1, 1.0), (2, 1.0), (4, 1.0)))) + )).toDF("id", "features") + + val key = Vectors.sparse(6, Seq((1, 1.0), (3, 1.0))) + + val mh = new MinHashLSH() + .setNumHashTables(5) + .setInputCol("features") + .setOutputCol("hashes") + + val model = mh.fit(dfA) + + // Feature Transformation + println("The hashed dataset where hashed values are stored in the column 'hashes':") + model.transform(dfA).show() + + // Compute the locality sensitive hashes for the input rows, then perform approximate + // similarity join. + // We could avoid computing hashes by passing in the already-transformed dataset, e.g. + // `model.approxSimilarityJoin(transformedA, transformedB, 0.6)` + println("Approximately joining dfA and dfB on Jaccard distance smaller than 0.6:") + model.approxSimilarityJoin(dfA, dfB, 0.6, "JaccardDistance") + .select(col("datasetA.id").alias("idA"), + col("datasetB.id").alias("idB"), + col("JaccardDistance")).show() + + // Compute the locality sensitive hashes for the input rows, then perform approximate nearest + // neighbor search. + // We could avoid computing hashes by passing in the already-transformed dataset, e.g. + // `model.approxNearestNeighbors(transformedA, key, 2)` + // It may return less than 2 rows when not enough approximate near-neighbor candidates are + // found. + println("Approximately searching dfA for 2 nearest neighbors of the key:") + model.approxNearestNeighbors(dfA, key, 2).show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala new file mode 100644 index 000000000000..9ee6d9b44934 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MinMaxScalerExample.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.MinMaxScaler +import org.apache.spark.ml.linalg.Vectors +// $example off$ +import org.apache.spark.sql.SparkSession + +object MinMaxScalerExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("MinMaxScalerExample") + .getOrCreate() + + // $example on$ + val dataFrame = spark.createDataFrame(Seq( + (0, Vectors.dense(1.0, 0.1, -1.0)), + (1, Vectors.dense(2.0, 1.1, 1.0)), + (2, Vectors.dense(3.0, 10.1, 3.0)) + )).toDF("id", "features") + + val scaler = new MinMaxScaler() + .setInputCol("features") + .setOutputCol("scaledFeatures") + + // Compute summary statistics and generate MinMaxScalerModel + val scalerModel = scaler.fit(dataFrame) + + // rescale each feature to range [min, max]. + val scaledData = scalerModel.transform(dataFrame) + println(s"Features scaled to range: [${scaler.getMin}, ${scaler.getMax}]") + scaledData.select("features", "scaledFeatures").show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala new file mode 100644 index 000000000000..87d96dd51eb9 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator +import org.apache.spark.ml.feature.{HashingTF, Tokenizer} +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.tuning.{CrossValidator, ParamGridBuilder} +import org.apache.spark.sql.Row +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * A simple example demonstrating model selection using CrossValidator. + * This example also demonstrates how Pipelines are Estimators. + * + * Run with + * {{{ + * bin/run-example ml.ModelSelectionViaCrossValidationExample + * }}} + */ +object ModelSelectionViaCrossValidationExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("ModelSelectionViaCrossValidationExample") + .getOrCreate() + + // $example on$ + // Prepare training data from a list of (id, text, label) tuples. + val training = spark.createDataFrame(Seq( + (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), + (4L, "b spark who", 1.0), + (5L, "g d a y", 0.0), + (6L, "spark fly", 1.0), + (7L, "was mapreduce", 0.0), + (8L, "e spark program", 1.0), + (9L, "a e c l", 0.0), + (10L, "spark compile", 1.0), + (11L, "hadoop software", 0.0) + )).toDF("id", "text", "label") + + // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. + val tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words") + val hashingTF = new HashingTF() + .setInputCol(tokenizer.getOutputCol) + .setOutputCol("features") + val lr = new LogisticRegression() + .setMaxIter(10) + val pipeline = new Pipeline() + .setStages(Array(tokenizer, hashingTF, lr)) + + // We use a ParamGridBuilder to construct a grid of parameters to search over. + // With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, + // this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. + val paramGrid = new ParamGridBuilder() + .addGrid(hashingTF.numFeatures, Array(10, 100, 1000)) + .addGrid(lr.regParam, Array(0.1, 0.01)) + .build() + + // We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. + // This will allow us to jointly choose parameters for all Pipeline stages. + // A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. + // Note that the evaluator here is a BinaryClassificationEvaluator and its default metric + // is areaUnderROC. + val cv = new CrossValidator() + .setEstimator(pipeline) + .setEvaluator(new BinaryClassificationEvaluator) + .setEstimatorParamMaps(paramGrid) + .setNumFolds(2) // Use 3+ in practice + .setParallelism(2) // Evaluate up to 2 parameter settings in parallel + + // Run cross-validation, and choose the best set of parameters. + val cvModel = cv.fit(training) + + // Prepare test documents, which are unlabeled (id, text) tuples. + val test = spark.createDataFrame(Seq( + (4L, "spark i j k"), + (5L, "l m n"), + (6L, "mapreduce spark"), + (7L, "apache hadoop") + )).toDF("id", "text") + + // Make predictions on test documents. cvModel uses the best model found (lrModel). + cvModel.transform(test) + .select("id", "text", "probability", "prediction") + .collect() + .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => + println(s"($id, $text) --> prob=$prob, prediction=$prediction") + } + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala new file mode 100644 index 000000000000..71e41e7298c7 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.evaluation.RegressionEvaluator +import org.apache.spark.ml.regression.LinearRegression +import org.apache.spark.ml.tuning.{ParamGridBuilder, TrainValidationSplit} +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * A simple example demonstrating model selection using TrainValidationSplit. + * + * Run with + * {{{ + * bin/run-example ml.ModelSelectionViaTrainValidationSplitExample + * }}} + */ +object ModelSelectionViaTrainValidationSplitExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("ModelSelectionViaTrainValidationSplitExample") + .getOrCreate() + + // $example on$ + // Prepare training and test data. + val data = spark.read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") + val Array(training, test) = data.randomSplit(Array(0.9, 0.1), seed = 12345) + + val lr = new LinearRegression() + .setMaxIter(10) + + // We use a ParamGridBuilder to construct a grid of parameters to search over. + // TrainValidationSplit will try all combinations of values and determine best model using + // the evaluator. + val paramGrid = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.1, 0.01)) + .addGrid(lr.fitIntercept) + .addGrid(lr.elasticNetParam, Array(0.0, 0.5, 1.0)) + .build() + + // In this case the estimator is simply the linear regression. + // A TrainValidationSplit requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. + val trainValidationSplit = new TrainValidationSplit() + .setEstimator(lr) + .setEvaluator(new RegressionEvaluator) + .setEstimatorParamMaps(paramGrid) + // 80% of the data will be used for training and the remaining 20% for validation. + .setTrainRatio(0.8) + // Evaluate up to 2 parameter settings in parallel + .setParallelism(2) + + // Run train validation split, and choose the best set of parameters. + val model = trainValidationSplit.fit(training) + + // Make predictions on test data. model is the model with combination of parameters + // that performed best. + model.transform(test) + .select("features", "label", "prediction") + .show() + // $example off$ + + spark.stop() + } +} 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 deleted file mode 100644 index 3ae53e57dbdb..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala +++ /dev/null @@ -1,182 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.ml - -import scopt.OptionParser - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.examples.mllib.AbstractParams -import org.apache.spark.ml.recommendation.ALS -import org.apache.spark.sql.{Row, SQLContext} - -/** - * An example app for ALS on MovieLens data (http://grouplens.org/datasets/movielens/). - * Run with - * {{{ - * bin/run-example ml.MovieLensALS - * }}} - */ -object MovieLensALS { - - case class Rating(userId: Int, movieId: Int, rating: Float, timestamp: Long) - - object Rating { - def parseRating(str: String): Rating = { - val fields = str.split("::") - assert(fields.size == 4) - Rating(fields(0).toInt, fields(1).toInt, fields(2).toFloat, fields(3).toLong) - } - } - - case class Movie(movieId: Int, title: String, genres: Seq[String]) - - object Movie { - def parseMovie(str: String): Movie = { - val fields = str.split("::") - assert(fields.size == 3) - Movie(fields(0).toInt, fields(1), fields(2).split("|")) - } - } - - case class Params( - ratings: String = null, - movies: String = null, - maxIter: Int = 10, - regParam: Double = 0.1, - rank: Int = 10, - numBlocks: Int = 10) extends AbstractParams[Params] - - def main(args: Array[String]) { - val defaultParams = Params() - - val parser = new OptionParser[Params]("MovieLensALS") { - head("MovieLensALS: an example app for ALS on MovieLens data.") - opt[String]("ratings") - .required() - .text("path to a MovieLens dataset of ratings") - .action((x, c) => c.copy(ratings = x)) - opt[String]("movies") - .required() - .text("path to a MovieLens dataset of movies") - .action((x, c) => c.copy(movies = x)) - opt[Int]("rank") - .text(s"rank, default: ${defaultParams.rank}") - .action((x, c) => c.copy(rank = x)) - opt[Int]("maxIter") - .text(s"max number of iterations, default: ${defaultParams.maxIter}") - .action((x, c) => c.copy(maxIter = x)) - opt[Double]("regParam") - .text(s"regularization parameter, default: ${defaultParams.regParam}") - .action((x, c) => c.copy(regParam = x)) - opt[Int]("numBlocks") - .text(s"number of blocks, default: ${defaultParams.numBlocks}") - .action((x, c) => c.copy(numBlocks = x)) - note( - """ - |Example command line to run this app: - | - | bin/spark-submit --class org.apache.spark.examples.ml.MovieLensALS \ - | examples/target/scala-*/spark-examples-*.jar \ - | --rank 10 --maxIter 15 --regParam 0.1 \ - | --movies data/mllib/als/sample_movielens_movies.txt \ - | --ratings data/mllib/als/sample_movielens_ratings.txt - """.stripMargin) - } - - parser.parse(args, defaultParams).map { params => - run(params) - } getOrElse { - System.exit(1) - } - } - - def run(params: Params) { - val conf = new SparkConf().setAppName(s"MovieLensALS with $params") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ - - val ratings = sc.textFile(params.ratings).map(Rating.parseRating).cache() - - val numRatings = ratings.count() - val numUsers = ratings.map(_.userId).distinct().count() - val numMovies = ratings.map(_.movieId).distinct().count() - - println(s"Got $numRatings ratings from $numUsers users on $numMovies movies.") - - val splits = ratings.randomSplit(Array(0.8, 0.2), 0L) - val training = splits(0).cache() - val test = splits(1).cache() - - val numTraining = training.count() - val numTest = test.count() - println(s"Training: $numTraining, test: $numTest.") - - ratings.unpersist(blocking = false) - - val als = new ALS() - .setUserCol("userId") - .setItemCol("movieId") - .setRank(params.rank) - .setMaxIter(params.maxIter) - .setRegParam(params.regParam) - .setNumBlocks(params.numBlocks) - - val model = als.fit(training.toDF()) - - val predictions = model.transform(test.toDF()).cache() - - // Evaluate the model. - // TODO: Create an evaluator to compute RMSE. - val mse = predictions.select("rating", "prediction").rdd - .flatMap { case Row(rating: Float, prediction: Float) => - val err = rating.toDouble - prediction - val err2 = err * err - if (err2.isNaN) { - None - } else { - Some(err2) - } - }.mean() - val rmse = math.sqrt(mse) - println(s"Test RMSE = $rmse.") - - // Inspect false positives. - // Note: We reference columns in 2 ways: - // (1) predictions("movieId") lets us specify the movieId column in the predictions - // DataFrame, rather than the movieId column in the movies DataFrame. - // (2) $"userId" specifies the userId column in the predictions DataFrame. - // We could also write predictions("userId") but do not have to since - // the movies DataFrame does not have a column "userId." - val movies = sc.textFile(params.movies).map(Movie.parseMovie).toDF() - val falsePositives = predictions.join(movies) - .where((predictions("movieId") === movies("movieId")) - && ($"rating" <= 1) && ($"prediction" >= 4)) - .select($"userId", predictions("movieId"), $"title", $"rating", $"prediction") - val numFalsePositives = falsePositives.count() - println(s"Found $numFalsePositives false positives") - if (numFalsePositives > 0) { - println(s"Example false positives:") - falsePositives.limit(100).collect().foreach(println) - } - - sc.stop() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MulticlassLogisticRegressionWithElasticNetExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MulticlassLogisticRegressionWithElasticNetExample.scala new file mode 100644 index 000000000000..42f0ace7a353 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MulticlassLogisticRegressionWithElasticNetExample.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.classification.LogisticRegression +// $example off$ +import org.apache.spark.sql.SparkSession + +object MulticlassLogisticRegressionWithElasticNetExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("MulticlassLogisticRegressionWithElasticNetExample") + .getOrCreate() + + // $example on$ + // Load training data + val training = spark + .read + .format("libsvm") + .load("data/mllib/sample_multiclass_classification_data.txt") + + val lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.3) + .setElasticNetParam(0.8) + + // Fit the model + val lrModel = lr.fit(training) + + // Print the coefficients and intercept for multinomial logistic regression + println(s"Coefficients: \n${lrModel.coefficientMatrix}") + println(s"Intercepts: ${lrModel.interceptVector}") + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala new file mode 100644 index 000000000000..6fce82d294f8 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MultilayerPerceptronClassifierExample.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.classification.MultilayerPerceptronClassifier +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example for Multilayer Perceptron Classification. + */ +object MultilayerPerceptronClassifierExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("MultilayerPerceptronClassifierExample") + .getOrCreate() + + // $example on$ + // Load the data stored in LIBSVM format as a DataFrame. + val data = spark.read.format("libsvm") + .load("data/mllib/sample_multiclass_classification_data.txt") + + // Split the data into train and test + val splits = data.randomSplit(Array(0.6, 0.4), seed = 1234L) + val train = splits(0) + val test = splits(1) + + // specify layers for the neural network: + // input layer of size 4 (features), two intermediate of size 5 and 4 + // and output of size 3 (classes) + val layers = Array[Int](4, 5, 4, 3) + + // create the trainer and set its parameters + val trainer = new MultilayerPerceptronClassifier() + .setLayers(layers) + .setBlockSize(128) + .setSeed(1234L) + .setMaxIter(100) + + // train the model + val model = trainer.fit(train) + + // compute accuracy on the test set + val result = model.transform(test) + val predictionAndLabels = result.select("prediction", "label") + val evaluator = new MulticlassClassificationEvaluator() + .setMetricName("accuracy") + + println("Test set accuracy = " + evaluator.evaluate(predictionAndLabels)) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala new file mode 100644 index 000000000000..d2183d6b4956 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NGramExample.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.NGram +// $example off$ +import org.apache.spark.sql.SparkSession + +object NGramExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("NGramExample") + .getOrCreate() + + // $example on$ + val wordDataFrame = spark.createDataFrame(Seq( + (0, Array("Hi", "I", "heard", "about", "Spark")), + (1, Array("I", "wish", "Java", "could", "use", "case", "classes")), + (2, Array("Logistic", "regression", "models", "are", "neat")) + )).toDF("id", "words") + + val ngram = new NGram().setN(2).setInputCol("words").setOutputCol("ngrams") + + val ngramDataFrame = ngram.transform(wordDataFrame) + ngramDataFrame.select("ngrams").show(false) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala new file mode 100644 index 000000000000..bd9fcc420a66 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NaiveBayesExample.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.classification.NaiveBayes +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator +// $example off$ +import org.apache.spark.sql.SparkSession + +object NaiveBayesExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("NaiveBayesExample") + .getOrCreate() + + // $example on$ + // Load the data stored in LIBSVM format as a DataFrame. + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + // Split the data into training and test sets (30% held out for testing) + val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3), seed = 1234L) + + // Train a NaiveBayes model. + val model = new NaiveBayes() + .fit(trainingData) + + // Select example rows to display. + val predictions = model.transform(testData) + predictions.show() + + // Select (prediction, true label) and compute test error + val evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("accuracy") + val accuracy = evaluator.evaluate(predictions) + println("Test set accuracy = " + accuracy) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala new file mode 100644 index 000000000000..989d250c1771 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/NormalizerExample.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.Normalizer +import org.apache.spark.ml.linalg.Vectors +// $example off$ +import org.apache.spark.sql.SparkSession + +object NormalizerExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("NormalizerExample") + .getOrCreate() + + // $example on$ + val dataFrame = spark.createDataFrame(Seq( + (0, Vectors.dense(1.0, 0.5, -1.0)), + (1, Vectors.dense(2.0, 1.0, 1.0)), + (2, Vectors.dense(4.0, 10.0, 2.0)) + )).toDF("id", "features") + + // Normalize each Vector using $L^1$ norm. + val normalizer = new Normalizer() + .setInputCol("features") + .setOutputCol("normFeatures") + .setP(1.0) + + val l1NormData = normalizer.transform(dataFrame) + println("Normalized using L^1 norm") + l1NormData.show() + + // Normalize each Vector using $L^\infty$ norm. + val lInfNormData = normalizer.transform(dataFrame, normalizer.p -> Double.PositiveInfinity) + println("Normalized using L^inf norm") + lInfNormData.show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala new file mode 100644 index 000000000000..274cc1268f4d --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.{OneHotEncoder, StringIndexer} +// $example off$ +import org.apache.spark.sql.SparkSession + +object OneHotEncoderExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("OneHotEncoderExample") + .getOrCreate() + + // $example on$ + val df = spark.createDataFrame(Seq( + (0, "a"), + (1, "b"), + (2, "c"), + (3, "a"), + (4, "a"), + (5, "c") + )).toDF("id", "category") + + val indexer = new StringIndexer() + .setInputCol("category") + .setOutputCol("categoryIndex") + .fit(df) + val indexed = indexer.transform(df) + + val encoder = new OneHotEncoder() + .setInputCol("categoryIndex") + .setOutputCol("categoryVec") + + val encoded = encoder.transform(indexed) + encoded.show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala index bab31f585b0e..4ad6c7c3ef20 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala @@ -18,170 +18,62 @@ // scalastyle:off println package org.apache.spark.examples.ml -import java.util.concurrent.TimeUnit.{NANOSECONDS => NANO} - -import scopt.OptionParser - -import org.apache.spark.{SparkContext, SparkConf} -import org.apache.spark.examples.mllib.AbstractParams -import org.apache.spark.ml.classification.{OneVsRest, LogisticRegression} -import org.apache.spark.ml.util.MetadataUtils -import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext +// $example on$ +import org.apache.spark.ml.classification.{LogisticRegression, OneVsRest} +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator +// $example off$ +import org.apache.spark.sql.SparkSession /** - * An example runner for Multiclass to Binary Reduction with One Vs Rest. - * The example uses Logistic Regression as the base classifier. All parameters that - * can be specified on the base classifier can be passed in to the runner options. + * An example of Multiclass to Binary Reduction with One Vs Rest, + * using Logistic Regression as the base classifier. * Run with * {{{ - * ./bin/run-example ml.OneVsRestExample [options] + * ./bin/run-example ml.OneVsRestExample * }}} - * For local mode, run - * {{{ - * ./bin/spark-submit --class org.apache.spark.examples.ml.OneVsRestExample --driver-memory 1g - * [examples JAR path] [options] - * }}} - * If you use it as a template to create your own app, please use `spark-submit` to submit your app. */ -object OneVsRestExample { - - case class Params private[ml] ( - input: String = null, - testInput: Option[String] = None, - maxIter: Int = 100, - tol: Double = 1E-6, - fitIntercept: Boolean = true, - regParam: Option[Double] = None, - elasticNetParam: Option[Double] = None, - fracTest: Double = 0.2) extends AbstractParams[Params] +object OneVsRestExample { def main(args: Array[String]) { - val defaultParams = Params() + val spark = SparkSession + .builder + .appName(s"OneVsRestExample") + .getOrCreate() - val parser = new OptionParser[Params]("OneVsRest Example") { - head("OneVsRest Example: multiclass to binary reduction using OneVsRest") - opt[String]("input") - .text("input path to labeled examples. This path must be specified") - .required() - .action((x, c) => c.copy(input = x)) - opt[Double]("fracTest") - .text(s"fraction of data to hold out for testing. If given option testInput, " + - s"this option is ignored. default: ${defaultParams.fracTest}") - .action((x, c) => c.copy(fracTest = x)) - opt[String]("testInput") - .text("input path to test dataset. If given, option fracTest is ignored") - .action((x, c) => c.copy(testInput = Some(x))) - opt[Int]("maxIter") - .text(s"maximum number of iterations for Logistic Regression." + - s" default: ${defaultParams.maxIter}") - .action((x, c) => c.copy(maxIter = x)) - opt[Double]("tol") - .text(s"the convergence tolerance of iterations for Logistic Regression." + - s" default: ${defaultParams.tol}") - .action((x, c) => c.copy(tol = x)) - opt[Boolean]("fitIntercept") - .text(s"fit intercept for Logistic Regression." + - s" default: ${defaultParams.fitIntercept}") - .action((x, c) => c.copy(fitIntercept = x)) - opt[Double]("regParam") - .text(s"the regularization parameter for Logistic Regression.") - .action((x, c) => c.copy(regParam = Some(x))) - opt[Double]("elasticNetParam") - .text(s"the ElasticNet mixing parameter for Logistic Regression.") - .action((x, c) => c.copy(elasticNetParam = Some(x))) - checkConfig { params => - if (params.fracTest < 0 || params.fracTest >= 1) { - failure(s"fracTest ${params.fracTest} value incorrect; should be in [0,1).") - } else { - success - } - } - } - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - sys.exit(1) - } - } + // $example on$ + // load data file. + val inputData = spark.read.format("libsvm") + .load("data/mllib/sample_multiclass_classification_data.txt") - private def run(params: Params) { - val conf = new SparkConf().setAppName(s"OneVsRestExample with $params") - val sc = new SparkContext(conf) - val inputData = MLUtils.loadLibSVMFile(sc, params.input) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ - - // compute the train/test split: if testInput is not provided use part of input. - val data = params.testInput match { - case Some(t) => { - // compute the number of features in the training set. - val numFeatures = inputData.first().features.size - val testData = MLUtils.loadLibSVMFile(sc, t, numFeatures) - Array[RDD[LabeledPoint]](inputData, testData) - } - case None => { - val f = params.fracTest - inputData.randomSplit(Array(1 - f, f), seed = 12345) - } - } - val Array(train, test) = data.map(_.toDF().cache()) + // generate the train/test split. + val Array(train, test) = inputData.randomSplit(Array(0.8, 0.2)) // instantiate the base classifier val classifier = new LogisticRegression() - .setMaxIter(params.maxIter) - .setTol(params.tol) - .setFitIntercept(params.fitIntercept) - - // Set regParam, elasticNetParam if specified in params - params.regParam.foreach(classifier.setRegParam) - params.elasticNetParam.foreach(classifier.setElasticNetParam) + .setMaxIter(10) + .setTol(1E-6) + .setFitIntercept(true) // instantiate the One Vs Rest Classifier. - - val ovr = new OneVsRest() - ovr.setClassifier(classifier) + val ovr = new OneVsRest().setClassifier(classifier) // train the multiclass model. - val (trainingDuration, ovrModel) = time(ovr.fit(train)) + val ovrModel = ovr.fit(train) // score the model on test data. - val (predictionDuration, predictions) = time(ovrModel.transform(test)) - - // evaluate the model - val predictionsAndLabels = predictions.select("prediction", "label") - .map(row => (row.getDouble(0), row.getDouble(1))) - - val metrics = new MulticlassMetrics(predictionsAndLabels) - - val confusionMatrix = metrics.confusionMatrix - - // compute the false positive rate per label - val predictionColSchema = predictions.schema("prediction") - val numClasses = MetadataUtils.getNumClasses(predictionColSchema).get - val fprs = Range(0, numClasses).map(p => (p, metrics.falsePositiveRate(p.toDouble))) + val predictions = ovrModel.transform(test) - println(s" Training Time ${trainingDuration} sec\n") + // obtain evaluator. + val evaluator = new MulticlassClassificationEvaluator() + .setMetricName("accuracy") - println(s" Prediction Time ${predictionDuration} sec\n") + // compute the classification error on test data. + val accuracy = evaluator.evaluate(predictions) + println(s"Test Error = ${1 - accuracy}") + // $example off$ - println(s" Confusion Matrix\n ${confusionMatrix.toString}\n") - - println("label\tfpr") - - println(fprs.map {case (label, fpr) => label + "\t" + fpr}.mkString("\n")) - - sc.stop() + spark.stop() } - private def time[R](block: => R): (Long, R) = { - val t0 = System.nanoTime() - val result = block // call-by-name - val t1 = System.nanoTime() - (NANO.toSeconds(t1 - t0), result) - } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.scala new file mode 100644 index 000000000000..4e1d7cdbabdb --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PCAExample.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. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.PCA +import org.apache.spark.ml.linalg.Vectors +// $example off$ +import org.apache.spark.sql.SparkSession + +object PCAExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("PCAExample") + .getOrCreate() + + // $example on$ + val data = Array( + Vectors.sparse(5, Seq((1, 1.0), (3, 7.0))), + Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0), + Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0) + ) + val df = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") + + val pca = new PCA() + .setInputCol("features") + .setOutputCol("pcaFeatures") + .setK(3) + .fit(df) + + val result = pca.transform(df).select("pcaFeatures") + result.show(false) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala new file mode 100644 index 000000000000..12f8663b9ce5 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PipelineExample.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.feature.{HashingTF, Tokenizer} +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.sql.Row +// $example off$ +import org.apache.spark.sql.SparkSession + +object PipelineExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("PipelineExample") + .getOrCreate() + + // $example on$ + // Prepare training documents from a list of (id, text, label) tuples. + val training = spark.createDataFrame(Seq( + (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) + )).toDF("id", "text", "label") + + // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. + val tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words") + val hashingTF = new HashingTF() + .setNumFeatures(1000) + .setInputCol(tokenizer.getOutputCol) + .setOutputCol("features") + val lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.001) + val pipeline = new Pipeline() + .setStages(Array(tokenizer, hashingTF, lr)) + + // Fit the pipeline to training documents. + val model = pipeline.fit(training) + + // Now we can optionally save the fitted pipeline to disk + model.write.overwrite().save("/tmp/spark-logistic-regression-model") + + // We can also save this unfit pipeline to disk + pipeline.write.overwrite().save("/tmp/unfit-lr-model") + + // And load it back in during production + val sameModel = PipelineModel.load("/tmp/spark-logistic-regression-model") + + // Prepare test documents, which are unlabeled (id, text) tuples. + val test = spark.createDataFrame(Seq( + (4L, "spark i j k"), + (5L, "l m n"), + (6L, "spark hadoop spark"), + (7L, "apache hadoop") + )).toDF("id", "text") + + // Make predictions on test documents. + model.transform(test) + .select("id", "text", "probability", "prediction") + .collect() + .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => + println(s"($id, $text) --> prob=$prob, prediction=$prediction") + } + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala new file mode 100644 index 000000000000..f117b03ab217 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PolynomialExpansionExample.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.PolynomialExpansion +import org.apache.spark.ml.linalg.Vectors +// $example off$ +import org.apache.spark.sql.SparkSession + +object PolynomialExpansionExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("PolynomialExpansionExample") + .getOrCreate() + + // $example on$ + val data = Array( + Vectors.dense(2.0, 1.0), + Vectors.dense(0.0, 0.0), + Vectors.dense(3.0, -1.0) + ) + val df = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") + + val polyExpansion = new PolynomialExpansion() + .setInputCol("features") + .setOutputCol("polyFeatures") + .setDegree(3) + + val polyDF = polyExpansion.transform(df) + polyDF.show(false) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala new file mode 100644 index 000000000000..aedb9e7d3bb7 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/QuantileDiscretizerExample.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.QuantileDiscretizer +// $example off$ +import org.apache.spark.sql.SparkSession + +object QuantileDiscretizerExample { + def main(args: Array[String]) { + val spark = SparkSession + .builder + .appName("QuantileDiscretizerExample") + .getOrCreate() + + // $example on$ + val data = Array((0, 18.0), (1, 19.0), (2, 8.0), (3, 5.0), (4, 2.2)) + val df = spark.createDataFrame(data).toDF("id", "hour") + // $example off$ + // Output of QuantileDiscretizer for such small datasets can depend on the number of + // partitions. Here we force a single partition to ensure consistent results. + // Note this is not necessary for normal use cases + .repartition(1) + + // $example on$ + val discretizer = new QuantileDiscretizer() + .setInputCol("hour") + .setOutputCol("result") + .setNumBuckets(3) + + val result = discretizer.fit(df).transform(df) + result.show() + // $example off$ + + spark.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala new file mode 100644 index 000000000000..3498fa8a50c6 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RFormulaExample.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.RFormula +// $example off$ +import org.apache.spark.sql.SparkSession + +object RFormulaExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("RFormulaExample") + .getOrCreate() + + // $example on$ + val dataset = spark.createDataFrame(Seq( + (7, "US", 18, 1.0), + (8, "CA", 12, 0.0), + (9, "NZ", 15, 0.0) + )).toDF("id", "country", "hour", "clicked") + + val formula = new RFormula() + .setFormula("clicked ~ country + hour") + .setFeaturesCol("features") + .setLabelCol("label") + + val output = formula.fit(dataset).transform(dataset) + output.select("features", "label").show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala new file mode 100644 index 000000000000..5eafda8ce428 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestClassifierExample.scala @@ -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. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.classification.{RandomForestClassificationModel, RandomForestClassifier} +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator +import org.apache.spark.ml.feature.{IndexToString, StringIndexer, VectorIndexer} +// $example off$ +import org.apache.spark.sql.SparkSession + +object RandomForestClassifierExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("RandomForestClassifierExample") + .getOrCreate() + + // $example on$ + // Load and parse the data file, converting it to a DataFrame. + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + // Index labels, adding metadata to the label column. + // Fit on whole dataset to include all labels in index. + val labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data) + // Automatically identify categorical features, and index them. + // Set maxCategories so features with > 4 distinct values are treated as continuous. + val featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data) + + // Split the data into training and test sets (30% held out for testing). + val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + + // Train a RandomForest model. + val rf = new RandomForestClassifier() + .setLabelCol("indexedLabel") + .setFeaturesCol("indexedFeatures") + .setNumTrees(10) + + // Convert indexed labels back to original labels. + val labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labels) + + // Chain indexers and forest in a Pipeline. + val pipeline = new Pipeline() + .setStages(Array(labelIndexer, featureIndexer, rf, labelConverter)) + + // Train model. This also runs the indexers. + val model = pipeline.fit(trainingData) + + // Make predictions. + val predictions = model.transform(testData) + + // Select example rows to display. + predictions.select("predictedLabel", "label", "features").show(5) + + // Select (prediction, true label) and compute test error. + val evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("accuracy") + val accuracy = evaluator.evaluate(predictions) + println("Test Error = " + (1.0 - accuracy)) + + val rfModel = model.stages(2).asInstanceOf[RandomForestClassificationModel] + println("Learned classification forest model:\n" + rfModel.toDebugString) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala index 109178f4137b..3c127a46e1f1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala @@ -18,18 +18,18 @@ // scalastyle:off println package org.apache.spark.examples.ml +import java.util.Locale + import scala.collection.mutable -import scala.language.reflectiveCalls import scopt.OptionParser -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.examples.mllib.AbstractParams import org.apache.spark.ml.{Pipeline, PipelineStage} import org.apache.spark.ml.classification.{RandomForestClassificationModel, RandomForestClassifier} import org.apache.spark.ml.feature.{StringIndexer, VectorIndexer} import org.apache.spark.ml.regression.{RandomForestRegressionModel, RandomForestRegressor} -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, SparkSession} /** @@ -37,7 +37,7 @@ import org.apache.spark.sql.DataFrame * {{{ * ./bin/run-example ml.RandomForestExample [options] * }}} - * Decision Trees and ensembles can take a large amount of memory. If the run-example command + * Decision Trees and ensembles can take a large amount of memory. If the run-example command * above fails, try running via spark-submit and specifying the amount of memory as at least 1g. * For local mode, run * {{{ @@ -94,7 +94,7 @@ object RandomForestExample { s" default: ${defaultParams.numTrees}") .action((x, c) => c.copy(featureSubsetStrategy = x)) opt[Double]("fracTest") - .text(s"fraction of data to hold out for testing. If given option testInput, " + + .text(s"fraction of data to hold out for testing. If given option testInput, " + s"this option is ignored. default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) opt[Boolean]("cacheNodeIds") @@ -115,7 +115,7 @@ object RandomForestExample { s"default: ${defaultParams.checkpointInterval}") .action((x, c) => c.copy(checkpointInterval = x)) opt[String]("testInput") - .text(s"input path to test dataset. If given, option fracTest is ignored." + + .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") .action((x, c) => c.copy(testInput = x)) opt[String]("dataFormat") @@ -134,32 +134,34 @@ object RandomForestExample { } } - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { - val conf = new SparkConf().setAppName(s"RandomForestExample with $params") - val sc = new SparkContext(conf) - params.checkpointDir.foreach(sc.setCheckpointDir) - val algo = params.algo.toLowerCase + def run(params: Params): Unit = { + val spark = SparkSession + .builder + .appName(s"RandomForestExample with $params") + .getOrCreate() + + params.checkpointDir.foreach(spark.sparkContext.setCheckpointDir) + val algo = params.algo.toLowerCase(Locale.ROOT) println(s"RandomForestExample with parameters:\n$params") // Load training and test data and cache it. - val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(sc, params.input, + val (training: DataFrame, test: DataFrame) = DecisionTreeExample.loadDatasets(params.input, params.dataFormat, params.testInput, algo, params.fracTest) - // Set up Pipeline + // Set up Pipeline. val stages = new mutable.ArrayBuffer[PipelineStage]() // (1) For classification, re-index classes. val labelColName = if (algo == "classification") "indexedLabel" else "label" if (algo == "classification") { val labelIndexer = new StringIndexer() - .setInputCol("labelString") + .setInputCol("label") .setOutputCol(labelColName) stages += labelIndexer } @@ -170,7 +172,7 @@ object RandomForestExample { .setOutputCol("indexedFeatures") .setMaxCategories(10) stages += featuresIndexer - // (3) Learn Random Forest + // (3) Learn Random Forest. val dt = algo match { case "classification" => new RandomForestClassifier() @@ -196,18 +198,18 @@ object RandomForestExample { .setCheckpointInterval(params.checkpointInterval) .setFeatureSubsetStrategy(params.featureSubsetStrategy) .setNumTrees(params.numTrees) - case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + case _ => throw new IllegalArgumentException(s"Algo ${params.algo} not supported.") } stages += dt val pipeline = new Pipeline().setStages(stages.toArray) - // Fit the Pipeline + // Fit the Pipeline. val startTime = System.nanoTime() val pipelineModel = pipeline.fit(training) val elapsedTime = (System.nanoTime() - startTime) / 1e9 println(s"Training time: $elapsedTime seconds") - // Get the trained Random Forest from the fitted PipelineModel + // Get the trained Random Forest from the fitted PipelineModel. algo match { case "classification" => val rfModel = pipelineModel.stages.last.asInstanceOf[RandomForestClassificationModel] @@ -223,10 +225,10 @@ object RandomForestExample { } else { println(rfModel) // Print model summary. } - case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") + case _ => throw new IllegalArgumentException(s"Algo ${params.algo} not supported.") } - // Evaluate model on training, test data + // Evaluate model on training, test data. algo match { case "classification" => println("Training data results:") @@ -239,10 +241,10 @@ object RandomForestExample { println("Test data results:") DecisionTreeExample.evaluateRegressionModel(pipelineModel, test, labelColName) case _ => - throw new IllegalArgumentException("Algo ${params.algo} not supported.") + throw new IllegalArgumentException(s"Algo ${params.algo} not supported.") } - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala new file mode 100644 index 000000000000..9a0a001c26ef --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestRegressorExample.scala @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.evaluation.RegressionEvaluator +import org.apache.spark.ml.feature.VectorIndexer +import org.apache.spark.ml.regression.{RandomForestRegressionModel, RandomForestRegressor} +// $example off$ +import org.apache.spark.sql.SparkSession + +object RandomForestRegressorExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("RandomForestRegressorExample") + .getOrCreate() + + // $example on$ + // Load and parse the data file, converting it to a DataFrame. + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + // Automatically identify categorical features, and index them. + // Set maxCategories so features with > 4 distinct values are treated as continuous. + val featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data) + + // Split the data into training and test sets (30% held out for testing). + val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + + // Train a RandomForest model. + val rf = new RandomForestRegressor() + .setLabelCol("label") + .setFeaturesCol("indexedFeatures") + + // Chain indexer and forest in a Pipeline. + val pipeline = new Pipeline() + .setStages(Array(featureIndexer, rf)) + + // Train model. This also runs the indexer. + val model = pipeline.fit(trainingData) + + // Make predictions. + val predictions = model.transform(testData) + + // Select example rows to display. + predictions.select("prediction", "label", "features").show(5) + + // Select (prediction, true label) and compute test error. + val evaluator = new RegressionEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("rmse") + val rmse = evaluator.evaluate(predictions) + println("Root Mean Squared Error (RMSE) on test data = " + rmse) + + val rfModel = model.stages(1).asInstanceOf[RandomForestRegressionModel] + println("Learned regression forest model:\n" + rfModel.toDebugString) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala new file mode 100644 index 000000000000..bb4587b82cb3 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SQLTransformerExample.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.SQLTransformer +// $example off$ +import org.apache.spark.sql.SparkSession + +object SQLTransformerExample { + def main(args: Array[String]) { + val spark = SparkSession + .builder + .appName("SQLTransformerExample") + .getOrCreate() + + // $example on$ + val df = spark.createDataFrame( + Seq((0, 1.0, 3.0), (2, 2.0, 5.0))).toDF("id", "v1", "v2") + + val sqlTrans = new SQLTransformer().setStatement( + "SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__") + + sqlTrans.transform(df).show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println 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 deleted file mode 100644 index f4d1fe57856a..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ /dev/null @@ -1,104 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.ml - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.ml.classification.LogisticRegression -import org.apache.spark.ml.param.ParamMap -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.sql.{Row, SQLContext} - -/** - * A simple example demonstrating ways to specify parameters for Estimators and Transformers. - * Run with - * {{{ - * bin/run-example ml.SimpleParamsExample - * }}} - */ -object SimpleParamsExample { - - def main(args: Array[String]) { - val conf = new SparkConf().setAppName("SimpleParamsExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ - - // Prepare training data. - // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes - // into DataFrames, where it uses the case class metadata to infer the schema. - val training = sc.parallelize(Seq( - LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), - LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), - LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), - LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)))) - - // Create a LogisticRegression instance. This instance is an Estimator. - val lr = new LogisticRegression() - // Print out the parameters, documentation, and any default values. - println("LogisticRegression parameters:\n" + lr.explainParams() + "\n") - - // We may set parameters using setter methods. - lr.setMaxIter(10) - .setRegParam(0.01) - - // Learn a LogisticRegression model. This uses the parameters stored in lr. - 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 - // LogisticRegression instance. - println("Model 1 was fit using parameters: " + model1.parent.extractParamMap()) - - // We may alternatively specify parameters using a ParamMap, - // which supports several methods for specifying parameters. - val paramMap = ParamMap(lr.maxIter -> 20) - paramMap.put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. - paramMap.put(lr.regParam -> 0.1, lr.thresholds -> Array(0.45, 0.55)) // Specify multiple Params. - - // One can also combine ParamMaps. - val paramMap2 = ParamMap(lr.probabilityCol -> "myProbability") // Change output column name - val paramMapCombined = paramMap ++ paramMap2 - - // Now learn a new model using the paramMapCombined parameters. - // paramMapCombined overrides all parameters set earlier via lr.set* methods. - val model2 = lr.fit(training.toDF(), paramMapCombined) - println("Model 2 was fit using parameters: " + model2.parent.extractParamMap()) - - // Prepare test data. - val test = sc.parallelize(Seq( - LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), - LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), - LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) - - // Make predictions on test data using the Transformer.transform() method. - // LogisticRegressionModel.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()) - .select("features", "label", "myProbability", "prediction") - .collect() - .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) => - println(s"($features, $label) -> prob=$prob, prediction=$prediction") - } - - sc.stop() - } -} -// scalastyle:on println 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 deleted file mode 100644 index 960280137cbf..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ /dev/null @@ -1,93 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.ml - -import scala.beans.BeanInfo - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.ml.Pipeline -import org.apache.spark.ml.classification.LogisticRegression -import org.apache.spark.ml.feature.{HashingTF, Tokenizer} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.sql.{Row, SQLContext} - -@BeanInfo -case class LabeledDocument(id: Long, text: String, label: Double) - -@BeanInfo -case class Document(id: Long, text: String) - -/** - * A simple text classification pipeline that recognizes "spark" from input text. This is to show - * how to create and configure an ML pipeline. Run with - * {{{ - * bin/run-example ml.SimpleTextClassificationPipeline - * }}} - */ -object SimpleTextClassificationPipeline { - - def main(args: Array[String]) { - val conf = new SparkConf().setAppName("SimpleTextClassificationPipeline") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ - - // Prepare training documents, which are labeled. - val training = sc.parallelize(Seq( - LabeledDocument(0L, "a b c d e spark", 1.0), - LabeledDocument(1L, "b d", 0.0), - LabeledDocument(2L, "spark f g h", 1.0), - LabeledDocument(3L, "hadoop mapreduce", 0.0))) - - // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. - val tokenizer = new Tokenizer() - .setInputCol("text") - .setOutputCol("words") - val hashingTF = new HashingTF() - .setNumFeatures(1000) - .setInputCol(tokenizer.getOutputCol) - .setOutputCol("features") - val lr = new LogisticRegression() - .setMaxIter(10) - .setRegParam(0.001) - val pipeline = new Pipeline() - .setStages(Array(tokenizer, hashingTF, lr)) - - // Fit the pipeline to training documents. - val model = pipeline.fit(training.toDF()) - - // Prepare test documents, which are unlabeled. - val test = sc.parallelize(Seq( - Document(4L, "spark i j k"), - Document(5L, "l m n"), - Document(6L, "spark hadoop spark"), - Document(7L, "apache hadoop"))) - - // Make predictions on test documents. - model.transform(test.toDF()) - .select("id", "text", "probability", "prediction") - .collect() - .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => - println(s"($id, $text) --> prob=$prob, prediction=$prediction") - } - - sc.stop() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala new file mode 100644 index 000000000000..4d668e8ab967 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StandardScalerExample.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.StandardScaler +// $example off$ +import org.apache.spark.sql.SparkSession + +object StandardScalerExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("StandardScalerExample") + .getOrCreate() + + // $example on$ + val dataFrame = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + val scaler = new StandardScaler() + .setInputCol("features") + .setOutputCol("scaledFeatures") + .setWithStd(true) + .setWithMean(false) + + // Compute summary statistics by fitting the StandardScaler. + val scalerModel = scaler.fit(dataFrame) + + // Normalize each feature to have unit standard deviation. + val scaledData = scalerModel.transform(dataFrame) + scaledData.show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala new file mode 100644 index 000000000000..369a6fffd79b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StopWordsRemoverExample.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.StopWordsRemover +// $example off$ +import org.apache.spark.sql.SparkSession + +object StopWordsRemoverExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("StopWordsRemoverExample") + .getOrCreate() + + // $example on$ + val remover = new StopWordsRemover() + .setInputCol("raw") + .setOutputCol("filtered") + + val dataSet = spark.createDataFrame(Seq( + (0, Seq("I", "saw", "the", "red", "balloon")), + (1, Seq("Mary", "had", "a", "little", "lamb")) + )).toDF("id", "raw") + + remover.transform(dataSet).show(false) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala new file mode 100644 index 000000000000..63f273e87a20 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/StringIndexerExample.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.StringIndexer +// $example off$ +import org.apache.spark.sql.SparkSession + +object StringIndexerExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("StringIndexerExample") + .getOrCreate() + + // $example on$ + val df = spark.createDataFrame( + Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")) + ).toDF("id", "category") + + val indexer = new StringIndexer() + .setInputCol("category") + .setOutputCol("categoryIndex") + + val indexed = indexer.fit(df).transform(df) + indexed.show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala index 40c33e4e7d44..ec2df2ef876b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala @@ -21,33 +21,40 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.{HashingTF, IDF, Tokenizer} // $example off$ -import org.apache.spark.sql.SQLContext -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession object TfIdfExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("TfIdfExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("TfIdfExample") + .getOrCreate() // $example on$ - val sentenceData = sqlContext.createDataFrame(Seq( - (0, "Hi I heard about Spark"), - (0, "I wish Java could use case classes"), - (1, "Logistic regression models are neat") + val sentenceData = spark.createDataFrame(Seq( + (0.0, "Hi I heard about Spark"), + (0.0, "I wish Java could use case classes"), + (1.0, "Logistic regression models are neat") )).toDF("label", "sentence") val tokenizer = new Tokenizer().setInputCol("sentence").setOutputCol("words") val wordsData = tokenizer.transform(sentenceData) + val hashingTF = new HashingTF() .setInputCol("words").setOutputCol("rawFeatures").setNumFeatures(20) + val featurizedData = hashingTF.transform(wordsData) + // alternatively, CountVectorizer can also be used to get term frequency vectors + val idf = new IDF().setInputCol("rawFeatures").setOutputCol("features") val idfModel = idf.fit(featurizedData) + val rescaledData = idfModel.transform(featurizedData) - rescaledData.select("features", "label").take(3).foreach(println) + rescaledData.select("label", "features").show() // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala new file mode 100644 index 000000000000..154777690f00 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TokenizerExample.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.{RegexTokenizer, Tokenizer} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.functions._ +// $example off$ + +object TokenizerExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("TokenizerExample") + .getOrCreate() + + // $example on$ + val sentenceDataFrame = spark.createDataFrame(Seq( + (0, "Hi I heard about Spark"), + (1, "I wish Java could use case classes"), + (2, "Logistic,regression,models,are,neat") + )).toDF("id", "sentence") + + val tokenizer = new Tokenizer().setInputCol("sentence").setOutputCol("words") + val regexTokenizer = new RegexTokenizer() + .setInputCol("sentence") + .setOutputCol("words") + .setPattern("\\W") // alternatively .setPattern("\\w+").setGaps(false) + + val countTokens = udf { (words: Seq[String]) => words.length } + + val tokenized = tokenizer.transform(sentenceDataFrame) + tokenized.select("sentence", "words") + .withColumn("tokens", countTokens(col("words"))).show(false) + + val regexTokenized = regexTokenizer.transform(sentenceDataFrame) + regexTokenized.select("sentence", "words") + .withColumn("tokens", countTokens(col("words"))).show(false) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TrainValidationSplitExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TrainValidationSplitExample.scala deleted file mode 100644 index 1abdf219b1c0..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TrainValidationSplitExample.scala +++ /dev/null @@ -1,80 +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.examples.ml - -import org.apache.spark.ml.evaluation.RegressionEvaluator -import org.apache.spark.ml.regression.LinearRegression -import org.apache.spark.ml.tuning.{ParamGridBuilder, TrainValidationSplit} -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.sql.SQLContext -import org.apache.spark.{SparkConf, SparkContext} - -/** - * A simple example demonstrating model selection using TrainValidationSplit. - * - * The example is based on [[SimpleParamsExample]] using linear regression. - * Run with - * {{{ - * bin/run-example ml.TrainValidationSplitExample - * }}} - */ -object TrainValidationSplitExample { - - def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("TrainValidationSplitExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ - - // Prepare training and test data. - val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() - val Array(training, test) = data.randomSplit(Array(0.9, 0.1), seed = 12345) - - val lr = new LinearRegression() - - // We use a ParamGridBuilder to construct a grid of parameters to search over. - // TrainValidationSplit will try all combinations of values and determine best model using - // the evaluator. - val paramGrid = new ParamGridBuilder() - .addGrid(lr.regParam, Array(0.1, 0.01)) - .addGrid(lr.fitIntercept, Array(true, false)) - .addGrid(lr.elasticNetParam, Array(0.0, 0.5, 1.0)) - .build() - - // In this case the estimator is simply the linear regression. - // A TrainValidationSplit requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. - val trainValidationSplit = new TrainValidationSplit() - .setEstimator(lr) - .setEvaluator(new RegressionEvaluator) - .setEstimatorParamMaps(paramGrid) - - // 80% of the data will be used for training and the remaining 20% for validation. - trainValidationSplit.setTrainRatio(0.8) - - // Run train validation split, and choose the best set of parameters. - val model = trainValidationSplit.fit(training) - - // Make predictions on test data. model is the model with combination of parameters - // that performed best. - model.transform(test) - .select("features", "label", "prediction") - .show() - - sc.stop() - } -} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/UnaryTransformerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/UnaryTransformerExample.scala new file mode 100644 index 000000000000..b4179ecc1e56 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/UnaryTransformerExample.scala @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.param.DoubleParam +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types.{DataType, DataTypes} +import org.apache.spark.util.Utils +// $example off$ + +/** + * An example demonstrating creating a custom [[org.apache.spark.ml.Transformer]] using + * the [[UnaryTransformer]] abstraction. + * + * Run with + * {{{ + * bin/run-example ml.UnaryTransformerExample + * }}} + */ +object UnaryTransformerExample { + + // $example on$ + /** + * Simple Transformer which adds a constant value to input Doubles. + * + * [[UnaryTransformer]] can be used to create a stage usable within Pipelines. + * It defines parameters for specifying input and output columns: + * [[UnaryTransformer.inputCol]] and [[UnaryTransformer.outputCol]]. + * It can optionally handle schema validation. + * + * [[DefaultParamsWritable]] provides a default implementation for persisting instances + * of this Transformer. + */ + class MyTransformer(override val uid: String) + extends UnaryTransformer[Double, Double, MyTransformer] with DefaultParamsWritable { + + final val shift: DoubleParam = new DoubleParam(this, "shift", "Value added to input") + + def getShift: Double = $(shift) + + def setShift(value: Double): this.type = set(shift, value) + + def this() = this(Identifiable.randomUID("myT")) + + override protected def createTransformFunc: Double => Double = (input: Double) => { + input + $(shift) + } + + override protected def outputDataType: DataType = DataTypes.DoubleType + + override protected def validateInputType(inputType: DataType): Unit = { + require(inputType == DataTypes.DoubleType, s"Bad input type: $inputType. Requires Double.") + } + } + + /** + * Companion object for our simple Transformer. + * + * [[DefaultParamsReadable]] provides a default implementation for loading instances + * of this Transformer which were persisted using [[DefaultParamsWritable]]. + */ + object MyTransformer extends DefaultParamsReadable[MyTransformer] + // $example off$ + + def main(args: Array[String]) { + val spark = SparkSession + .builder() + .appName("UnaryTransformerExample") + .getOrCreate() + + // $example on$ + val myTransformer = new MyTransformer() + .setShift(0.5) + .setInputCol("input") + .setOutputCol("output") + + // Create data, transform, and display it. + val data = spark.range(0, 5).toDF("input") + .select(col("input").cast("double").as("input")) + val result = myTransformer.transform(data) + println("Transformed by adding constant value") + result.show() + + // Save and load the Transformer. + val tmpDir = Utils.createTempDir() + val dirName = tmpDir.getCanonicalPath + myTransformer.write.overwrite().save(dirName) + val sameTransformer = MyTransformer.load(dirName) + + // Transform the data to show the results are identical. + println("Same transform applied from loaded model") + val sameResult = sameTransformer.transform(data) + sameResult.show() + + Utils.deleteRecursively(tmpDir) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println + diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.scala new file mode 100644 index 000000000000..3d5c7efb2053 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorAssemblerExample.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. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.VectorAssembler +import org.apache.spark.ml.linalg.Vectors +// $example off$ +import org.apache.spark.sql.SparkSession + +object VectorAssemblerExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("VectorAssemblerExample") + .getOrCreate() + + // $example on$ + val dataset = spark.createDataFrame( + Seq((0, 18, 1.0, Vectors.dense(0.0, 10.0, 0.5), 1.0)) + ).toDF("id", "hour", "mobile", "userFeatures", "clicked") + + val assembler = new VectorAssembler() + .setInputCols(Array("hour", "mobile", "userFeatures")) + .setOutputCol("features") + + val output = assembler.transform(dataset) + println("Assembled columns 'hour', 'mobile', 'userFeatures' to vector column 'features'") + output.select("features", "clicked").show(false) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.scala new file mode 100644 index 000000000000..afa761aee0b9 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorIndexerExample.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. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import org.apache.spark.ml.feature.VectorIndexer +// $example off$ +import org.apache.spark.sql.SparkSession + +object VectorIndexerExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("VectorIndexerExample") + .getOrCreate() + + // $example on$ + val data = spark.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") + + val indexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexed") + .setMaxCategories(10) + + val indexerModel = indexer.fit(data) + + val categoricalFeatures: Set[Int] = indexerModel.categoryMaps.keys.toSet + println(s"Chose ${categoricalFeatures.size} categorical features: " + + categoricalFeatures.mkString(", ")) + + // Create new column "indexed" with categorical values transformed to indices + val indexedData = indexerModel.transform(data) + indexedData.show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala new file mode 100644 index 000000000000..9a0af5d7b358 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/VectorSlicerExample.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.ml + +// $example on$ +import java.util.Arrays + +import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute} +import org.apache.spark.ml.feature.VectorSlicer +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.types.StructType +// $example off$ + +object VectorSlicerExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("VectorSlicerExample") + .getOrCreate() + + // $example on$ + val data = Arrays.asList( + Row(Vectors.sparse(3, Seq((0, -2.0), (1, 2.3)))), + Row(Vectors.dense(-2.0, 2.3, 0.0)) + ) + + val defaultAttr = NumericAttribute.defaultAttr + val attrs = Array("f1", "f2", "f3").map(defaultAttr.withName) + val attrGroup = new AttributeGroup("userFeatures", attrs.asInstanceOf[Array[Attribute]]) + + val dataset = spark.createDataFrame(data, StructType(Array(attrGroup.toStructField()))) + + val slicer = new VectorSlicer().setInputCol("userFeatures").setOutputCol("features") + + slicer.setIndices(Array(1)).setNames(Array("f3")) + // or slicer.setIndices(Array(1, 2)), or slicer.setNames(Array("f2", "f3")) + + val output = slicer.transform(dataset) + output.show(false) + // $example off$ + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala index 631ab4c8efa0..4bcc6ac6a01f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/Word2VecExample.scala @@ -20,19 +20,21 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.feature.Word2Vec +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.sql.Row // $example off$ -import org.apache.spark.sql.SQLContext -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SparkSession object Word2VecExample { def main(args: Array[String]) { - val conf = new SparkConf().setAppName("Word2Vec example") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) + val spark = SparkSession + .builder + .appName("Word2Vec example") + .getOrCreate() // $example on$ // Input data: Each row is a bag of words from a sentence or document. - val documentDF = sqlContext.createDataFrame(Seq( + val documentDF = spark.createDataFrame(Seq( "Hi I heard about Spark".split(" "), "I wish Java could use case classes".split(" "), "Logistic regression models are neat".split(" ") @@ -45,9 +47,13 @@ object Word2VecExample { .setVectorSize(3) .setMinCount(0) val model = word2Vec.fit(documentDF) + val result = model.transform(documentDF) - result.select("result").take(3).foreach(println) + result.collect().foreach { case Row(text: Seq[_], features: Vector) => + println(s"Text: [${text.mkString(", ")}] => \nVector: $features\n") } // $example off$ + + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/AbstractParams.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/AbstractParams.scala index ae6057758d6f..8985c8565c53 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/AbstractParams.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/AbstractParams.scala @@ -38,7 +38,7 @@ abstract class AbstractParams[T: TypeTag] { */ override def toString: String = { val tpe = tag.tpe - val allAccessors = tpe.declarations.collect { + val allAccessors = tpe.decls.collect { case m: MethodSymbol if m.isCaseAccessor => m } val mirror = runtimeMirror(getClass.getClassLoader) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/AssociationRulesExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/AssociationRulesExample.scala index ca22ddafc3c4..ff44de56839e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/AssociationRulesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/AssociationRulesExample.scala @@ -18,13 +18,12 @@ // scalastyle:off println package org.apache.spark.examples.mllib +import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.mllib.fpm.AssociationRules import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset // $example off$ -import org.apache.spark.{SparkConf, SparkContext} - object AssociationRulesExample { def main(args: Array[String]) { @@ -48,6 +47,8 @@ object AssociationRulesExample { + rule.consequent.mkString(",") + "]," + rule.confidence) } // $example off$ + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala index 1a4016f76c2a..a1a5b5915264 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala @@ -24,8 +24,8 @@ import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.classification.{LogisticRegressionWithLBFGS, SVMWithSGD} import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics +import org.apache.spark.mllib.optimization.{L1Updater, SquaredL2Updater} import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.optimization.{SquaredL2Updater, L1Updater} /** * An example app for binary classification. Run with @@ -95,14 +95,13 @@ object BinaryClassification { """.stripMargin) } - parser.parse(args, defaultParams).map { params => - run(params) - } getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { + def run(params: Params): Unit = { val conf = new SparkConf().setAppName(s"BinaryClassification with $params") val sc = new SparkContext(conf) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala new file mode 100644 index 000000000000..b9263ac6fcff --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassificationMetricsExample.scala @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLUtils +// $example off$ + +object BinaryClassificationMetricsExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("BinaryClassificationMetricsExample") + val sc = new SparkContext(conf) + // $example on$ + // Load training data in LIBSVM format + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_binary_classification_data.txt") + + // Split data into training (60%) and test (40%) + val Array(training, test) = data.randomSplit(Array(0.6, 0.4), seed = 11L) + training.cache() + + // Run training algorithm to build the model + val model = new LogisticRegressionWithLBFGS() + .setNumClasses(2) + .run(training) + + // Clear the prediction threshold so the model will return probabilities + model.clearThreshold + + // Compute raw scores on the test set + val predictionAndLabels = test.map { case LabeledPoint(label, features) => + val prediction = model.predict(features) + (prediction, label) + } + + // Instantiate metrics object + val metrics = new BinaryClassificationMetrics(predictionAndLabels) + + // Precision by threshold + val precision = metrics.precisionByThreshold + precision.foreach { case (t, p) => + println(s"Threshold: $t, Precision: $p") + } + + // Recall by threshold + val recall = metrics.recallByThreshold + recall.foreach { case (t, r) => + println(s"Threshold: $t, Recall: $r") + } + + // Precision-Recall Curve + val PRC = metrics.pr + + // F-measure + val f1Score = metrics.fMeasureByThreshold + f1Score.foreach { case (t, f) => + println(s"Threshold: $t, F-score: $f, Beta = 1") + } + + val beta = 0.5 + val fScore = metrics.fMeasureByThreshold(beta) + f1Score.foreach { case (t, f) => + println(s"Threshold: $t, F-score: $f, Beta = 0.5") + } + + // AUPRC + val auPRC = metrics.areaUnderPR + println("Area under precision-recall curve = " + auPRC) + + // Compute thresholds used in ROC and PR curves + val thresholds = precision.map(_._1) + + // ROC Curve + val roc = metrics.roc + + // AUROC + val auROC = metrics.areaUnderROC + println("Area under ROC = " + auROC) + // $example off$ + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BisectingKMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BisectingKMeansExample.scala new file mode 100644 index 000000000000..53d0b8fc208e --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BisectingKMeansExample.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +// scalastyle:off println +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.clustering.BisectingKMeans +import org.apache.spark.mllib.linalg.{Vector, Vectors} +// $example off$ + +/** + * An example demonstrating a bisecting k-means clustering in spark.mllib. + * + * Run with + * {{{ + * bin/run-example mllib.BisectingKMeansExample + * }}} + */ +object BisectingKMeansExample { + + def main(args: Array[String]) { + val sparkConf = new SparkConf().setAppName("mllib.BisectingKMeansExample") + val sc = new SparkContext(sparkConf) + + // $example on$ + // Loads and parses data + def parse(line: String): Vector = Vectors.dense(line.split(" ").map(_.toDouble)) + val data = sc.textFile("data/mllib/kmeans_data.txt").map(parse).cache() + + // Clustering the data into 6 clusters by BisectingKMeans. + val bkm = new BisectingKMeans().setK(6) + val model = bkm.run(data) + + // Show the compute cost and the cluster centers + println(s"Compute Cost: ${model.computeCost(data)}") + model.clusterCenters.zipWithIndex.foreach { case (center, idx) => + println(s"Cluster Center ${idx}: ${center}") + } + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala new file mode 100644 index 000000000000..5e400b7d715b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/ChiSqSelectorExample.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +// $example on$ +import org.apache.spark.mllib.feature.ChiSqSelector +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLUtils +// $example off$ + +object ChiSqSelectorExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("ChiSqSelectorExample") + val sc = new SparkContext(conf) + + // $example on$ + // Load some data in libsvm format + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + // Discretize data in 16 equal bins since ChiSqSelector requires categorical features + // Even though features are doubles, the ChiSqSelector treats each unique value as a category + val discretizedData = data.map { lp => + LabeledPoint(lp.label, Vectors.dense(lp.features.toArray.map { x => (x / 16).floor })) + } + // Create ChiSqSelector that will select top 50 of 692 features + val selector = new ChiSqSelector(50) + // Create ChiSqSelector model (selecting features) + val transformer = selector.fit(discretizedData) + // Filter the top 50 features from each feature vector + val filteredData = discretizedData.map { lp => + LabeledPoint(lp.label, transformer.transform(lp.features)) + } + // $example off$ + + println("filtered data: ") + filteredData.foreach(x => println(x)) + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala index 026d4ecc6d10..0b44c339ef13 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala @@ -20,10 +20,9 @@ package org.apache.spark.examples.mllib import scopt.OptionParser +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.stat.Statistics import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.{SparkConf, SparkContext} - /** * An example app for summarizing multivariate data from a file. Run with @@ -57,14 +56,13 @@ object Correlations { """.stripMargin) } - parser.parse(args, defaultParams).map { params => - run(params) - } getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { + def run(params: Params): Unit = { val conf = new SparkConf().setAppName(s"Correlations with $params") val sc = new SparkContext(conf) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/CorrelationsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/CorrelationsExample.scala new file mode 100644 index 000000000000..1202caf534e9 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/CorrelationsExample.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.rdd.RDD +// $example off$ + +object CorrelationsExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("CorrelationsExample") + val sc = new SparkContext(conf) + + // $example on$ + val seriesX: RDD[Double] = sc.parallelize(Array(1, 2, 3, 3, 5)) // a series + // must have the same number of partitions and cardinality as seriesX + val seriesY: RDD[Double] = sc.parallelize(Array(11, 22, 33, 33, 555)) + + // compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a + // method is not specified, Pearson's method will be used by default. + val correlation: Double = Statistics.corr(seriesX, seriesY, "pearson") + println(s"Correlation is: $correlation") + + val data: RDD[Vector] = sc.parallelize( + Seq( + Vectors.dense(1.0, 10.0, 100.0), + Vectors.dense(2.0, 20.0, 200.0), + Vectors.dense(5.0, 33.0, 366.0)) + ) // note that each Vector is a row and not a column + + // calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method + // If a method is not specified, Pearson's method will be used by default. + val correlMatrix: Matrix = Statistics.corr(data, "pearson") + println(correlMatrix.toString) + // $example off$ + + sc.stop() + } +} +// scalastyle:on println + diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala index 69988cc1b933..681465d2176d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala @@ -20,10 +20,9 @@ package org.apache.spark.examples.mllib import scopt.OptionParser -import org.apache.spark.SparkContext._ +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.linalg.distributed.{MatrixEntry, RowMatrix} -import org.apache.spark.{SparkConf, SparkContext} /** * Compute the similar columns of a matrix, using cosine similarity. @@ -69,14 +68,13 @@ object CosineSimilarity { """.stripMargin) } - parser.parse(args, defaultParams).map { params => - run(params) - } getOrElse { - System.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { + def run(params: Params): Unit = { val conf = new SparkConf().setAppName("CosineSimilarity") val sc = new SparkContext(conf) 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 deleted file mode 100644 index dc13f82488af..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala +++ /dev/null @@ -1,123 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.mllib - -import java.io.File - -import com.google.common.io.Files -import scopt.OptionParser - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext, DataFrame} - -/** - * An example of how to use [[org.apache.spark.sql.DataFrame]] as a Dataset for ML. Run with - * {{{ - * ./bin/run-example org.apache.spark.examples.mllib.DatasetExample [options] - * }}} - * If you use it as a template to create your own app, please use `spark-submit` to submit your app. - */ -object DatasetExample { - - case class Params( - input: String = "data/mllib/sample_libsvm_data.txt", - dataFormat: String = "libsvm") extends AbstractParams[Params] - - def main(args: Array[String]) { - val defaultParams = Params() - - val parser = new OptionParser[Params]("DatasetExample") { - head("Dataset: an example app using DataFrame as a Dataset for ML.") - opt[String]("input") - .text(s"input path to dataset") - .action((x, c) => c.copy(input = x)) - opt[String]("dataFormat") - .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") - .action((x, c) => c.copy(input = x)) - checkConfig { params => - success - } - } - - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - sys.exit(1) - } - } - - def run(params: Params) { - - val conf = new SparkConf().setAppName(s"DatasetExample with $params") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ // for implicit conversions - - // Load input data - val origData: RDD[LabeledPoint] = params.dataFormat match { - case "dense" => MLUtils.loadLabeledPoints(sc, params.input) - case "libsvm" => MLUtils.loadLibSVMFile(sc, params.input) - } - println(s"Loaded ${origData.count()} instances from file: ${params.input}") - - // Convert input data to DataFrame explicitly. - val df: DataFrame = origData.toDF() - println(s"Inferred schema:\n${df.schema.prettyJson}") - println(s"Converted to DataFrame with ${df.count()} records") - - // 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 = 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), - (sum1, sum2) => sum1.merge(sum2)) - println(s"Selected features column with average values:\n ${featureSummary.mean.toString}") - - val tmpDir = Files.createTempDir() - tmpDir.deleteOnExit() - val outputDir = new File(tmpDir, "dataset").toString - println(s"Saving to $outputDir as Parquet file.") - df.write.parquet(outputDir) - - println(s"Loading Parquet file with UDT from $outputDir.") - val newDataset = sqlContext.read.parquet(outputDir) - - println(s"Schema from Parquet: ${newDataset.schema.prettyJson}") - val newFeatures = newDataset.select("features").map { case Row(v: Vector) => v } - val newFeaturesSummary = newFeatures.aggregate(new MultivariateOnlineSummarizer())( - (summary, feat) => summary.add(feat), - (sum1, sum2) => sum1.merge(sum2)) - println(s"Selected features column with average values:\n ${newFeaturesSummary.mean.toString}") - - sc.stop() - } - -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeClassificationExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeClassificationExample.scala new file mode 100644 index 000000000000..b50b4592777c --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeClassificationExample.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.tree.DecisionTree +import org.apache.spark.mllib.tree.model.DecisionTreeModel +import org.apache.spark.mllib.util.MLUtils +// $example off$ + +object DecisionTreeClassificationExample { + + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("DecisionTreeClassificationExample") + val sc = new SparkContext(conf) + + // $example on$ + // Load and parse the data file. + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + // Split the data into training and test sets (30% held out for testing) + val splits = data.randomSplit(Array(0.7, 0.3)) + val (trainingData, testData) = (splits(0), splits(1)) + + // Train a DecisionTree model. + // Empty categoricalFeaturesInfo indicates all features are continuous. + val numClasses = 2 + val categoricalFeaturesInfo = Map[Int, Int]() + val impurity = "gini" + val maxDepth = 5 + val maxBins = 32 + + val model = DecisionTree.trainClassifier(trainingData, numClasses, categoricalFeaturesInfo, + impurity, maxDepth, maxBins) + + // Evaluate model on test instances and compute test error + val labelAndPreds = testData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) + } + val testErr = labelAndPreds.filter(r => r._1 != r._2).count().toDouble / testData.count() + println("Test Error = " + testErr) + println("Learned classification tree model:\n" + model.toDebugString) + + // Save and load model + model.save(sc, "target/tmp/myDecisionTreeClassificationModel") + val sameModel = DecisionTreeModel.load(sc, "target/tmp/myDecisionTreeClassificationModel") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRegressionExample.scala new file mode 100644 index 000000000000..2af45afae3d5 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRegressionExample.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.tree.DecisionTree +import org.apache.spark.mllib.tree.model.DecisionTreeModel +import org.apache.spark.mllib.util.MLUtils +// $example off$ + +object DecisionTreeRegressionExample { + + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("DecisionTreeRegressionExample") + val sc = new SparkContext(conf) + + // $example on$ + // Load and parse the data file. + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + // Split the data into training and test sets (30% held out for testing) + val splits = data.randomSplit(Array(0.7, 0.3)) + val (trainingData, testData) = (splits(0), splits(1)) + + // Train a DecisionTree model. + // Empty categoricalFeaturesInfo indicates all features are continuous. + val categoricalFeaturesInfo = Map[Int, Int]() + val impurity = "variance" + val maxDepth = 5 + val maxBins = 32 + + val model = DecisionTree.trainRegressor(trainingData, categoricalFeaturesInfo, impurity, + maxDepth, maxBins) + + // Evaluate model on test instances and compute test error + val labelsAndPredictions = testData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) + } + val testMSE = labelsAndPredictions.map{ case (v, p) => math.pow(v - p, 2) }.mean() + println("Test Mean Squared Error = " + testMSE) + println("Learned regression tree model:\n" + model.toDebugString) + + // Save and load model + model.save(sc, "target/tmp/myDecisionTreeRegressionModel") + val sameModel = DecisionTreeModel.load(sc, "target/tmp/myDecisionTreeRegressionModel") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index cc6bce3cb7c9..fa47e12857f0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -26,7 +26,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.evaluation.MulticlassMetrics import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.{DecisionTree, RandomForest, impurity} +import org.apache.spark.mllib.tree.{impurity, DecisionTree, RandomForest} import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.util.MLUtils @@ -149,10 +149,9 @@ object DecisionTreeRunner { } } - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } @@ -180,7 +179,7 @@ object DecisionTreeRunner { } // For classification, re-index classes if needed. val (examples, classIndexMap, numClasses) = algo match { - case Classification => { + case Classification => // classCounts: class --> # examples in class val classCounts = origExamples.map(_.label).countByValue() val sortedClasses = classCounts.keys.toList.sorted @@ -209,11 +208,10 @@ object DecisionTreeRunner { println(s"$c\t$frac\t${classCounts(c)}") } (examples, classIndexMap, numClasses) - } case Regression => (origExamples, null, 0) case _ => - throw new IllegalArgumentException("Algo ${params.algo} not supported.") + throw new IllegalArgumentException(s"Algo $algo not supported.") } // Create training, test sets. @@ -225,7 +223,7 @@ object DecisionTreeRunner { case "libsvm" => MLUtils.loadLibSVMFile(sc, testInput, numFeatures) } algo match { - case Classification => { + case Classification => // classCounts: class --> # examples in class val testExamples = { if (classIndexMap.isEmpty) { @@ -235,7 +233,6 @@ object DecisionTreeRunner { } } Array(examples, testExamples) - } case Regression => Array(examples, origTestExamples) } @@ -255,7 +252,7 @@ object DecisionTreeRunner { (training, test, numClasses) } - def run(params: Params) { + def run(params: Params): Unit = { val conf = new SparkConf().setAppName(s"DecisionTreeRunner with $params") val sc = new SparkContext(conf) @@ -297,11 +294,10 @@ object DecisionTreeRunner { } if (params.algo == Classification) { val trainAccuracy = - new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))) - .precision + new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))).accuracy println(s"Train accuracy = $trainAccuracy") val testAccuracy = - new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).precision + new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).accuracy println(s"Test accuracy = $testAccuracy") } if (params.algo == Regression) { @@ -324,11 +320,10 @@ object DecisionTreeRunner { println(model) // Print model summary. } val trainAccuracy = - new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))) - .precision + new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))).accuracy println(s"Train accuracy = $trainAccuracy") val testAccuracy = - new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).precision + new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).accuracy println(s"Test accuracy = $testAccuracy") } if (params.algo == Regression) { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala deleted file mode 100644 index 1fce4ba7efd6..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala +++ /dev/null @@ -1,69 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.mllib - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.mllib.clustering.GaussianMixture -import org.apache.spark.mllib.linalg.Vectors - -/** - * An example Gaussian Mixture Model EM app. Run with - * {{{ - * ./bin/run-example mllib.DenseGaussianMixture - * }}} - * If you use it as a template to create your own app, please use `spark-submit` to submit your app. - */ -object DenseGaussianMixture { - def main(args: Array[String]): Unit = { - if (args.length < 3) { - println("usage: DenseGmmEM [maxIterations]") - } else { - val maxIterations = if (args.length > 3) args(3).toInt else 100 - run(args(0), args(1).toInt, args(2).toDouble, maxIterations) - } - } - - private def run(inputFile: String, k: Int, convergenceTol: Double, maxIterations: Int) { - val conf = new SparkConf().setAppName("Gaussian Mixture Model EM example") - val ctx = new SparkContext(conf) - - val data = ctx.textFile(inputFile).map { line => - Vectors.dense(line.trim.split(' ').map(_.toDouble)) - }.cache() - - val clusters = new GaussianMixture() - .setK(k) - .setConvergenceTol(convergenceTol) - .setMaxIterations(maxIterations) - .run(data) - - for (i <- 0 until clusters.k) { - println("weight=%f\nmu=%s\nsigma=\n%s\n" format - (clusters.weights(i), clusters.gaussians(i).mu, clusters.gaussians(i).sigma)) - } - - println("Cluster labels (first <= 100):") - val clusterLabels = clusters.predict(data) - clusterLabels.take(100).foreach { x => - print(" " + x) - } - println() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala index 380d85d60e7b..b228827e5886 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala @@ -69,14 +69,13 @@ object DenseKMeans { .action((x, c) => c.copy(input = x)) } - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { + def run(params: Params): Unit = { val conf = new SparkConf().setAppName(s"DenseKMeans with $params") val sc = new SparkContext(conf) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala new file mode 100644 index 000000000000..1e4e3543194e --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/ElementwiseProductExample.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +// $example on$ +import org.apache.spark.mllib.feature.ElementwiseProduct +import org.apache.spark.mllib.linalg.Vectors +// $example off$ + +object ElementwiseProductExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("ElementwiseProductExample") + val sc = new SparkContext(conf) + + // $example on$ + // Create some vector data; also works for sparse vectors + val data = sc.parallelize(Array(Vectors.dense(1.0, 2.0, 3.0), Vectors.dense(4.0, 5.0, 6.0))) + + val transformingVector = Vectors.dense(0.0, 1.0, 2.0) + val transformer = new ElementwiseProduct(transformingVector) + + // Batch transform and per-row transform give the same results: + val transformedData = transformer.transform(data) + val transformedData2 = data.map(x => transformer.transform(x)) + // $example off$ + + println("transformedData: ") + transformedData.foreach(x => println(x)) + + println("transformedData2: ") + transformedData2.foreach(x => println(x)) + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala index 14b930550d55..6435abc12775 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala @@ -20,8 +20,8 @@ package org.apache.spark.examples.mllib import scopt.OptionParser -import org.apache.spark.mllib.fpm.FPGrowth import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.fpm.FPGrowth /** * Example for mining frequent itemsets using FP-growth. @@ -53,14 +53,13 @@ object FPGrowthExample { .action((x, c) => c.copy(input = x)) } - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { + def run(params: Params): Unit = { val conf = new SparkConf().setAppName(s"FPGrowthExample with $params") val sc = new SparkContext(conf) val transactions = sc.textFile(params.input).map(_.split(" ")).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GaussianMixtureExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GaussianMixtureExample.scala new file mode 100644 index 000000000000..b1b3a79d87ae --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GaussianMixtureExample.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.clustering.{GaussianMixture, GaussianMixtureModel} +import org.apache.spark.mllib.linalg.Vectors +// $example off$ + +object GaussianMixtureExample { + + def main(args: Array[String]) { + + val conf = new SparkConf().setAppName("GaussianMixtureExample") + val sc = new SparkContext(conf) + + // $example on$ + // Load and parse the data + val data = sc.textFile("data/mllib/gmm_data.txt") + val parsedData = data.map(s => Vectors.dense(s.trim.split(' ').map(_.toDouble))).cache() + + // Cluster the data into two classes using GaussianMixture + val gmm = new GaussianMixture().setK(2).run(parsedData) + + // Save and load model + gmm.save(sc, "target/org/apache/spark/GaussianMixtureExample/GaussianMixtureModel") + val sameModel = GaussianMixtureModel.load(sc, + "target/org/apache/spark/GaussianMixtureExample/GaussianMixtureModel") + + // output parameters of max-likelihood model + for (i <- 0 until gmm.k) { + println("weight=%f\nmu=%s\nsigma=\n%s\n" format + (gmm.weights(i), gmm.gaussians(i).mu, gmm.gaussians(i).sigma)) + } + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala index e16a6bf03357..4020c6b6bca7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala @@ -23,10 +23,9 @@ import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.evaluation.MulticlassMetrics import org.apache.spark.mllib.tree.GradientBoostedTrees -import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Algo} +import org.apache.spark.mllib.tree.configuration.{Algo, BoostingStrategy} import org.apache.spark.util.Utils - /** * An example runner for Gradient Boosting using decision trees as weak learners. Run with * {{{ @@ -86,14 +85,13 @@ object GradientBoostedTreesRunner { } } - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { + def run(params: Params): Unit = { val conf = new SparkConf().setAppName(s"GradientBoostedTreesRunner with $params") val sc = new SparkContext(conf) @@ -121,11 +119,10 @@ object GradientBoostedTreesRunner { println(model) // Print model summary. } val trainAccuracy = - new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))) - .precision + new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))).accuracy println(s"Train accuracy = $trainAccuracy") val testAccuracy = - new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).precision + new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).accuracy println(s"Test accuracy = $testAccuracy") } else if (params.algo == "Regression") { val startTime = System.nanoTime() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostingClassificationExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostingClassificationExample.scala new file mode 100644 index 000000000000..00bb3348d2a3 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostingClassificationExample.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.tree.GradientBoostedTrees +import org.apache.spark.mllib.tree.configuration.BoostingStrategy +import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel +import org.apache.spark.mllib.util.MLUtils +// $example off$ + +object GradientBoostingClassificationExample { + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("GradientBoostedTreesClassificationExample") + val sc = new SparkContext(conf) + // $example on$ + // Load and parse the data file. + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + // Split the data into training and test sets (30% held out for testing) + val splits = data.randomSplit(Array(0.7, 0.3)) + val (trainingData, testData) = (splits(0), splits(1)) + + // Train a GradientBoostedTrees model. + // The defaultParams for Classification use LogLoss by default. + val boostingStrategy = BoostingStrategy.defaultParams("Classification") + boostingStrategy.numIterations = 3 // Note: Use more iterations in practice. + boostingStrategy.treeStrategy.numClasses = 2 + boostingStrategy.treeStrategy.maxDepth = 5 + // Empty categoricalFeaturesInfo indicates all features are continuous. + boostingStrategy.treeStrategy.categoricalFeaturesInfo = Map[Int, Int]() + + val model = GradientBoostedTrees.train(trainingData, boostingStrategy) + + // Evaluate model on test instances and compute test error + val labelAndPreds = testData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) + } + val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count() + println("Test Error = " + testErr) + println("Learned classification GBT model:\n" + model.toDebugString) + + // Save and load model + model.save(sc, "target/tmp/myGradientBoostingClassificationModel") + val sameModel = GradientBoostedTreesModel.load(sc, + "target/tmp/myGradientBoostingClassificationModel") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println + + diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostingRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostingRegressionExample.scala new file mode 100644 index 000000000000..d8c263460839 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostingRegressionExample.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.tree.GradientBoostedTrees +import org.apache.spark.mllib.tree.configuration.BoostingStrategy +import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel +import org.apache.spark.mllib.util.MLUtils +// $example off$ + +object GradientBoostingRegressionExample { + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("GradientBoostedTreesRegressionExample") + val sc = new SparkContext(conf) + // $example on$ + // Load and parse the data file. + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + // Split the data into training and test sets (30% held out for testing) + val splits = data.randomSplit(Array(0.7, 0.3)) + val (trainingData, testData) = (splits(0), splits(1)) + + // Train a GradientBoostedTrees model. + // The defaultParams for Regression use SquaredError by default. + val boostingStrategy = BoostingStrategy.defaultParams("Regression") + boostingStrategy.numIterations = 3 // Note: Use more iterations in practice. + boostingStrategy.treeStrategy.maxDepth = 5 + // Empty categoricalFeaturesInfo indicates all features are continuous. + boostingStrategy.treeStrategy.categoricalFeaturesInfo = Map[Int, Int]() + + val model = GradientBoostedTrees.train(trainingData, boostingStrategy) + + // Evaluate model on test instances and compute test error + val labelsAndPredictions = testData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) + } + val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() + println("Test Mean Squared Error = " + testMSE) + println("Learned regression GBT model:\n" + model.toDebugString) + + // Save and load model + model.save(sc, "target/tmp/myGradientBoostingRegressionModel") + val sameModel = GradientBoostedTreesModel.load(sc, + "target/tmp/myGradientBoostingRegressionModel") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/HypothesisTestingExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/HypothesisTestingExample.scala new file mode 100644 index 000000000000..0d391a3637c0 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/HypothesisTestingExample.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.mllib.stat.test.ChiSqTestResult +import org.apache.spark.rdd.RDD +// $example off$ + +object HypothesisTestingExample { + + def main(args: Array[String]) { + + val conf = new SparkConf().setAppName("HypothesisTestingExample") + val sc = new SparkContext(conf) + + // $example on$ + // a vector composed of the frequencies of events + val vec: Vector = Vectors.dense(0.1, 0.15, 0.2, 0.3, 0.25) + + // compute the goodness of fit. If a second vector to test against is not supplied + // as a parameter, the test runs against a uniform distribution. + val goodnessOfFitTestResult = Statistics.chiSqTest(vec) + // summary of the test including the p-value, degrees of freedom, test statistic, the method + // used, and the null hypothesis. + println(s"$goodnessOfFitTestResult\n") + + // a contingency matrix. Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) + val mat: Matrix = Matrices.dense(3, 2, Array(1.0, 3.0, 5.0, 2.0, 4.0, 6.0)) + + // conduct Pearson's independence test on the input contingency matrix + val independenceTestResult = Statistics.chiSqTest(mat) + // summary of the test including the p-value, degrees of freedom + println(s"$independenceTestResult\n") + + val obs: RDD[LabeledPoint] = + sc.parallelize( + Seq( + LabeledPoint(1.0, Vectors.dense(1.0, 0.0, 3.0)), + LabeledPoint(1.0, Vectors.dense(1.0, 2.0, 0.0)), + LabeledPoint(-1.0, Vectors.dense(-1.0, 0.0, -0.5) + ) + ) + ) // (feature, label) pairs. + + // The contingency table is constructed from the raw (feature, label) pairs and used to conduct + // the independence test. Returns an array containing the ChiSquaredTestResult for every feature + // against the label. + val featureTestResults: Array[ChiSqTestResult] = Statistics.chiSqTest(obs) + featureTestResults.zipWithIndex.foreach { case (k, v) => + println("Column " + (v + 1).toString + ":") + println(k) + } // summary of the test + // $example off$ + + sc.stop() + } +} +// scalastyle:on println + diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/HypothesisTestingKolmogorovSmirnovTestExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/HypothesisTestingKolmogorovSmirnovTestExample.scala new file mode 100644 index 000000000000..840874cf3c2f --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/HypothesisTestingKolmogorovSmirnovTestExample.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.rdd.RDD +// $example off$ + +object HypothesisTestingKolmogorovSmirnovTestExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("HypothesisTestingKolmogorovSmirnovTestExample") + val sc = new SparkContext(conf) + + // $example on$ + val data: RDD[Double] = sc.parallelize(Seq(0.1, 0.15, 0.2, 0.3, 0.25)) // an RDD of sample data + + // run a KS test for the sample versus a standard normal distribution + val testResult = Statistics.kolmogorovSmirnovTest(data, "norm", 0, 1) + // summary of the test including the p-value, test statistic, and null hypothesis if our p-value + // indicates significance, we can reject the null hypothesis. + println(testResult) + println() + + // perform a KS test using a cumulative distribution function of our making + val myCDF = Map(0.1 -> 0.2, 0.15 -> 0.6, 0.2 -> 0.05, 0.3 -> 0.05, 0.25 -> 0.1) + val testResult2 = Statistics.kolmogorovSmirnovTest(data, myCDF) + println(testResult2) + // $example off$ + + sc.stop() + } +} +// scalastyle:on println + diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/IsotonicRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/IsotonicRegressionExample.scala index 52ac9ae7dd2d..4aee951f5b04 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/IsotonicRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/IsotonicRegressionExample.scala @@ -18,24 +18,25 @@ // scalastyle:off println package org.apache.spark.examples.mllib +import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.mllib.regression.{IsotonicRegression, IsotonicRegressionModel} +import org.apache.spark.mllib.util.MLUtils // $example off$ -import org.apache.spark.{SparkConf, SparkContext} object IsotonicRegressionExample { - def main(args: Array[String]) : Unit = { + def main(args: Array[String]): Unit = { val conf = new SparkConf().setAppName("IsotonicRegressionExample") val sc = new SparkContext(conf) // $example on$ - val data = sc.textFile("data/mllib/sample_isotonic_regression_data.txt") + val data = MLUtils.loadLibSVMFile(sc, + "data/mllib/sample_isotonic_regression_libsvm_data.txt").cache() // 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) + val parsedData = data.map { labeledPoint => + (labeledPoint.label, labeledPoint.features(0), 1.0) } // Split data into training (60%) and test (40%) sets. @@ -61,6 +62,8 @@ object IsotonicRegressionExample { model.save(sc, "target/tmp/myIsotonicRegressionModel") val sameModel = IsotonicRegressionModel.load(sc, "target/tmp/myIsotonicRegressionModel") // $example off$ + + sc.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/KMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/KMeansExample.scala new file mode 100644 index 000000000000..c4d71d862f37 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/KMeansExample.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.clustering.{KMeans, KMeansModel} +import org.apache.spark.mllib.linalg.Vectors +// $example off$ + +object KMeansExample { + + def main(args: Array[String]) { + + val conf = new SparkConf().setAppName("KMeansExample") + val sc = new SparkContext(conf) + + // $example on$ + // Load and parse the data + val data = sc.textFile("data/mllib/kmeans_data.txt") + val parsedData = data.map(s => Vectors.dense(s.split(' ').map(_.toDouble))).cache() + + // Cluster the data into two classes using KMeans + val numClusters = 2 + val numIterations = 20 + val clusters = KMeans.train(parsedData, numClusters, numIterations) + + // Evaluate clustering by computing Within Set Sum of Squared Errors + val WSSSE = clusters.computeCost(parsedData) + println("Within Set Sum of Squared Errors = " + WSSSE) + + // Save and load model + clusters.save(sc, "target/org/apache/spark/KMeansExample/KMeansModel") + val sameModel = KMeansModel.load(sc, "target/org/apache/spark/KMeansExample/KMeansModel") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/KernelDensityEstimationExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/KernelDensityEstimationExample.scala new file mode 100644 index 000000000000..cc5d159b36cc --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/KernelDensityEstimationExample.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.stat.KernelDensity +import org.apache.spark.rdd.RDD +// $example off$ + +object KernelDensityEstimationExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("KernelDensityEstimationExample") + val sc = new SparkContext(conf) + + // $example on$ + // an RDD of sample data + val data: RDD[Double] = sc.parallelize(Seq(1, 1, 1, 2, 3, 4, 5, 5, 6, 7, 8, 9, 9)) + + // Construct the density estimator with the sample data and a standard deviation + // for the Gaussian kernels + val kd = new KernelDensity() + .setSample(data) + .setBandwidth(3.0) + + // Find density estimates for the given values + val densities = kd.estimate(Array(-1.0, 2.0, 5.0)) + // $example off$ + + densities.foreach(println) + + sc.stop() + } +} +// scalastyle:on println + diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LBFGSExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LBFGSExample.scala new file mode 100644 index 000000000000..fedcefa09838 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LBFGSExample.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.classification.LogisticRegressionModel +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.optimization.{LBFGS, LogisticGradient, SquaredL2Updater} +import org.apache.spark.mllib.util.MLUtils +// $example off$ + +object LBFGSExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("LBFGSExample") + val sc = new SparkContext(conf) + + // $example on$ + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + val numFeatures = data.take(1)(0).features.size + + // Split data into training (60%) and test (40%). + val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) + + // Append 1 into the training data as intercept. + val training = splits(0).map(x => (x.label, MLUtils.appendBias(x.features))).cache() + + val test = splits(1) + + // Run training algorithm to build the model + val numCorrections = 10 + val convergenceTol = 1e-4 + val maxNumIterations = 20 + val regParam = 0.1 + val initialWeightsWithIntercept = Vectors.dense(new Array[Double](numFeatures + 1)) + + val (weightsWithIntercept, loss) = LBFGS.runLBFGS( + training, + new LogisticGradient(), + new SquaredL2Updater(), + numCorrections, + convergenceTol, + maxNumIterations, + regParam, + initialWeightsWithIntercept) + + val model = new LogisticRegressionModel( + Vectors.dense(weightsWithIntercept.toArray.slice(0, weightsWithIntercept.size - 1)), + weightsWithIntercept(weightsWithIntercept.size - 1)) + + // Clear the default threshold. + model.clearThreshold() + + // Compute raw scores on the test set. + val scoreAndLabels = test.map { point => + val score = model.predict(point.features) + (score, point.label) + } + + // Get evaluation metrics. + val metrics = new BinaryClassificationMetrics(scoreAndLabels) + val auROC = metrics.areaUnderROC() + + println("Loss of each step in training process") + loss.foreach(println) + println("Area under ROC = " + auROC) + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index 75b0f69cf91a..cd77ecf990b3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -18,19 +18,19 @@ // scalastyle:off println package org.apache.spark.examples.mllib -import java.text.BreakIterator - -import scala.collection.mutable - -import scopt.OptionParser +import java.util.Locale import org.apache.log4j.{Level, Logger} +import scopt.OptionParser -import org.apache.spark.{SparkContext, SparkConf} -import org.apache.spark.mllib.clustering.{EMLDAOptimizer, OnlineLDAOptimizer, DistributedLDAModel, LDA} +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.feature.{CountVectorizer, CountVectorizerModel, RegexTokenizer, StopWordsRemover} +import org.apache.spark.ml.linalg.{Vector => MLVector} +import org.apache.spark.mllib.clustering.{DistributedLDAModel, EMLDAOptimizer, LDA, OnlineLDAOptimizer} import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.rdd.RDD - +import org.apache.spark.sql.{Row, SparkSession} /** * An example Latent Dirichlet Allocation (LDA) app. Run with @@ -101,15 +101,13 @@ object LDAExample { .action((x, c) => c.copy(input = c.input :+ x)) } - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - parser.showUsageAsError - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - private def run(params: Params) { + private def run(params: Params): Unit = { val conf = new SparkConf().setAppName(s"LDAExample with $params") val sc = new SparkContext(conf) @@ -121,7 +119,7 @@ object LDAExample { preprocess(sc, params.input, params.vocabSize, params.stopwordFile) corpus.cache() val actualCorpusSize = corpus.count() - val actualVocabSize = vocabArray.size + val actualVocabSize = vocabArray.length val preprocessElapsed = (System.nanoTime() - preprocessStart) / 1e9 println() @@ -135,7 +133,7 @@ object LDAExample { // Run LDA. val lda = new LDA() - val optimizer = params.algorithm.toLowerCase match { + val optimizer = params.algorithm.toLowerCase(Locale.ROOT) match { case "em" => new EMLDAOptimizer // add (1.0 / actualCorpusSize) to MiniBatchFraction be more robust on tiny datasets. case "online" => new OnlineLDAOptimizer().setMiniBatchFraction(0.05 + 1.0 / actualCorpusSize) @@ -192,115 +190,49 @@ object LDAExample { vocabSize: Int, stopwordFile: String): (RDD[(Long, Vector)], Array[String], Long) = { + val spark = SparkSession + .builder + .sparkContext(sc) + .getOrCreate() + import spark.implicits._ + // Get dataset of document texts // One document per line in each text file. If the input consists of many small files, // this can result in a large number of small partitions, which can degrade performance. // In this case, consider using coalesce() to create fewer, larger partitions. - val textRDD: RDD[String] = sc.textFile(paths.mkString(",")) - - // Split text into words - val tokenizer = new SimpleTokenizer(sc, stopwordFile) - val tokenized: RDD[(Long, IndexedSeq[String])] = textRDD.zipWithIndex().map { case (text, id) => - id -> tokenizer.getWords(text) - } - tokenized.cache() - - // Counts words: RDD[(word, wordCount)] - val wordCounts: RDD[(String, Long)] = tokenized - .flatMap { case (_, tokens) => tokens.map(_ -> 1L) } - .reduceByKey(_ + _) - wordCounts.cache() - val fullVocabSize = wordCounts.count() - // Select vocab - // (vocab: Map[word -> id], total tokens after selecting vocab) - val (vocab: Map[String, Int], selectedTokenCount: Long) = { - val tmpSortedWC: Array[(String, Long)] = if (vocabSize == -1 || fullVocabSize <= vocabSize) { - // Use all terms - wordCounts.collect().sortBy(-_._2) - } else { - // Sort terms to select vocab - wordCounts.sortBy(_._2, ascending = false).take(vocabSize) - } - (tmpSortedWC.map(_._1).zipWithIndex.toMap, tmpSortedWC.map(_._2).sum) - } - - val documents = tokenized.map { case (id, tokens) => - // Filter tokens by vocabulary, and create word count vector representation of document. - val wc = new mutable.HashMap[Int, Int]() - tokens.foreach { term => - if (vocab.contains(term)) { - val termIndex = vocab(term) - wc(termIndex) = wc.getOrElse(termIndex, 0) + 1 - } - } - val indices = wc.keys.toArray.sorted - val values = indices.map(i => wc(i).toDouble) - - val sb = Vectors.sparse(vocab.size, indices, values) - (id, sb) + val df = sc.textFile(paths.mkString(",")).toDF("docs") + val customizedStopWords: Array[String] = if (stopwordFile.isEmpty) { + Array.empty[String] + } else { + val stopWordText = sc.textFile(stopwordFile).collect() + stopWordText.flatMap(_.stripMargin.split("\\s+")) } - - val vocabArray = new Array[String](vocab.size) - vocab.foreach { case (term, i) => vocabArray(i) = term } - - (documents, vocabArray, selectedTokenCount) + val tokenizer = new RegexTokenizer() + .setInputCol("docs") + .setOutputCol("rawTokens") + val stopWordsRemover = new StopWordsRemover() + .setInputCol("rawTokens") + .setOutputCol("tokens") + stopWordsRemover.setStopWords(stopWordsRemover.getStopWords ++ customizedStopWords) + val countVectorizer = new CountVectorizer() + .setVocabSize(vocabSize) + .setInputCol("tokens") + .setOutputCol("features") + + val pipeline = new Pipeline() + .setStages(Array(tokenizer, stopWordsRemover, countVectorizer)) + + val model = pipeline.fit(df) + val documents = model.transform(df) + .select("features") + .rdd + .map { case Row(features: MLVector) => Vectors.fromML(features) } + .zipWithIndex() + .map(_.swap) + + (documents, + model.stages(2).asInstanceOf[CountVectorizerModel].vocabulary, // vocabulary + documents.map(_._2.numActives).sum().toLong) // total token count } } - -/** - * Simple Tokenizer. - * - * TODO: Formalize the interface, and make this a public class in mllib.feature - */ -private class SimpleTokenizer(sc: SparkContext, stopwordFile: String) extends Serializable { - - private val stopwords: Set[String] = if (stopwordFile.isEmpty) { - Set.empty[String] - } else { - val stopwordText = sc.textFile(stopwordFile).collect() - stopwordText.flatMap(_.stripMargin.split("\\s+")).toSet - } - - // Matches sequences of Unicode letters - private val allWordRegex = "^(\\p{L}*)$".r - - // Ignore words shorter than this length. - private val minWordLength = 3 - - def getWords(text: String): IndexedSeq[String] = { - - val words = new mutable.ArrayBuffer[String]() - - // Use Java BreakIterator to tokenize text into words. - val wb = BreakIterator.getWordInstance - wb.setText(text) - - // current,end index start,end of each word - var current = wb.first() - var end = wb.next() - while (end != BreakIterator.DONE) { - // Convert to lowercase - val word: String = text.substring(current, end).toLowerCase - // Remove short words and strings that aren't only letters - word match { - case allWordRegex(w) if w.length >= minWordLength && !stopwords.contains(w) => - words += w - case _ => - } - - current = end - try { - end = wb.next() - } catch { - case e: Exception => - // Ignore remaining text in line. - // This is a known bug in BreakIterator (for some Java versions), - // which fails when it sees certain characters. - end = BreakIterator.DONE - } - } - words - } - -} // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LatentDirichletAllocationExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LatentDirichletAllocationExample.scala new file mode 100644 index 000000000000..f2c8ec01439f --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LatentDirichletAllocationExample.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.clustering.{DistributedLDAModel, LDA} +import org.apache.spark.mllib.linalg.Vectors +// $example off$ + +object LatentDirichletAllocationExample { + + def main(args: Array[String]) { + + val conf = new SparkConf().setAppName("LatentDirichletAllocationExample") + val sc = new SparkContext(conf) + + // $example on$ + // Load and parse the data + val data = sc.textFile("data/mllib/sample_lda_data.txt") + val parsedData = data.map(s => Vectors.dense(s.trim.split(' ').map(_.toDouble))) + // Index documents with unique IDs + val corpus = parsedData.zipWithIndex.map(_.swap).cache() + + // Cluster the documents into three topics using LDA + val ldaModel = new LDA().setK(3).run(corpus) + + // Output topics. Each is a distribution over words (matching word count vectors) + println("Learned topics (as distributions over vocab of " + ldaModel.vocabSize + " words):") + val topics = ldaModel.topicsMatrix + for (topic <- Range(0, 3)) { + print("Topic " + topic + ":") + for (word <- Range(0, ldaModel.vocabSize)) { print(" " + topics(word, topic)); } + println() + } + + // Save and load model. + ldaModel.save(sc, "target/org/apache/spark/LatentDirichletAllocationExample/LDAModel") + val sameModel = DistributedLDAModel.load(sc, + "target/org/apache/spark/LatentDirichletAllocationExample/LDAModel") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala index 8878061a0970..86aec363ea42 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala @@ -22,9 +22,9 @@ import org.apache.log4j.{Level, Logger} import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.optimization.{L1Updater, SimpleUpdater, SquaredL2Updater} import org.apache.spark.mllib.regression.LinearRegressionWithSGD import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.optimization.{SimpleUpdater, SquaredL2Updater, L1Updater} /** * An example app for linear regression. Run with @@ -34,6 +34,7 @@ import org.apache.spark.mllib.optimization.{SimpleUpdater, SquaredL2Updater, L1U * A synthetic dataset can be found at `data/mllib/sample_linear_regression_data.txt`. * If you use it as a template to create your own app, please use `spark-submit` to submit your app. */ +@deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0") object LinearRegression { object RegType extends Enumeration { @@ -81,14 +82,13 @@ object LinearRegression { """.stripMargin) } - parser.parse(args, defaultParams).map { params => - run(params) - } getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { + def run(params: Params): Unit = { val conf = new SparkConf().setAppName(s"LinearRegression with $params") val sc = new SparkContext(conf) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala new file mode 100644 index 000000000000..d39961809448 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.LinearRegressionModel +import org.apache.spark.mllib.regression.LinearRegressionWithSGD +// $example off$ + +@deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0") +object LinearRegressionWithSGDExample { + + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("LinearRegressionWithSGDExample") + val sc = new SparkContext(conf) + + // $example on$ + // Load and parse the data + val data = sc.textFile("data/mllib/ridge-data/lpsa.data") + val parsedData = data.map { line => + val parts = line.split(',') + LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) + }.cache() + + // Building the model + val numIterations = 100 + val stepSize = 0.00000001 + val model = LinearRegressionWithSGD.train(parsedData, numIterations, stepSize) + + // Evaluate model on training examples and compute training error + val valuesAndPreds = parsedData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) + } + val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2) }.mean() + println("training Mean Squared Error = " + MSE) + + // Save and load model + model.save(sc, "target/tmp/scalaLinearRegressionWithSGDModel") + val sameModel = LinearRegressionModel.load(sc, "target/tmp/scalaLinearRegressionWithSGDModel") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LogisticRegressionWithLBFGSExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LogisticRegressionWithLBFGSExample.scala new file mode 100644 index 000000000000..31ba740ad4af --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LogisticRegressionWithLBFGSExample.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.classification.{LogisticRegressionModel, LogisticRegressionWithLBFGS} +import org.apache.spark.mllib.evaluation.MulticlassMetrics +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLUtils +// $example off$ + +object LogisticRegressionWithLBFGSExample { + + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("LogisticRegressionWithLBFGSExample") + val sc = new SparkContext(conf) + + // $example on$ + // Load training data in LIBSVM format. + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + + // Split data into training (60%) and test (40%). + val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) + val training = splits(0).cache() + val test = splits(1) + + // Run training algorithm to build the model + val model = new LogisticRegressionWithLBFGS() + .setNumClasses(10) + .run(training) + + // Compute raw scores on the test set. + val predictionAndLabels = test.map { case LabeledPoint(label, features) => + val prediction = model.predict(features) + (prediction, label) + } + + // Get evaluation metrics. + val metrics = new MulticlassMetrics(predictionAndLabels) + val accuracy = metrics.accuracy + println(s"Accuracy = $accuracy") + + // Save and load model + model.save(sc, "target/tmp/scalaLogisticRegressionWithLBFGSModel") + val sameModel = LogisticRegressionModel.load(sc, + "target/tmp/scalaLogisticRegressionWithLBFGSModel") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala index 69691ae297f6..9bd6927fb7fc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -24,7 +24,6 @@ import org.apache.log4j.{Level, Logger} import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ import org.apache.spark.mllib.recommendation.{ALS, MatrixFactorizationModel, Rating} import org.apache.spark.rdd.RDD @@ -90,14 +89,13 @@ object MovieLensALS { """.stripMargin) } - parser.parse(args, defaultParams).map { params => - run(params) - } getOrElse { - System.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { + def run(params: Params): Unit = { val conf = new SparkConf().setAppName(s"MovieLensALS with $params") if (params.kryo) { conf.registerKryoClasses(Array(classOf[mutable.BitSet], classOf[Rating])) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MultiLabelMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MultiLabelMetricsExample.scala new file mode 100644 index 000000000000..ebab81b334a5 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MultiLabelMetricsExample.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.evaluation.MultilabelMetrics +import org.apache.spark.rdd.RDD +// $example off$ + +object MultiLabelMetricsExample { + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("MultiLabelMetricsExample") + val sc = new SparkContext(conf) + // $example on$ + val scoreAndLabels: RDD[(Array[Double], Array[Double])] = sc.parallelize( + Seq((Array(0.0, 1.0), Array(0.0, 2.0)), + (Array(0.0, 2.0), Array(0.0, 1.0)), + (Array.empty[Double], Array(0.0)), + (Array(2.0), Array(2.0)), + (Array(2.0, 0.0), Array(2.0, 0.0)), + (Array(0.0, 1.0, 2.0), Array(0.0, 1.0)), + (Array(1.0), Array(1.0, 2.0))), 2) + + // Instantiate metrics object + val metrics = new MultilabelMetrics(scoreAndLabels) + + // Summary stats + println(s"Recall = ${metrics.recall}") + println(s"Precision = ${metrics.precision}") + println(s"F1 measure = ${metrics.f1Measure}") + println(s"Accuracy = ${metrics.accuracy}") + + // Individual label stats + metrics.labels.foreach(label => + println(s"Class $label precision = ${metrics.precision(label)}")) + metrics.labels.foreach(label => println(s"Class $label recall = ${metrics.recall(label)}")) + metrics.labels.foreach(label => println(s"Class $label F1-score = ${metrics.f1Measure(label)}")) + + // Micro stats + println(s"Micro recall = ${metrics.microRecall}") + println(s"Micro precision = ${metrics.microPrecision}") + println(s"Micro F1 measure = ${metrics.microF1Measure}") + + // Hamming loss + println(s"Hamming loss = ${metrics.hammingLoss}") + + // Subset accuracy + println(s"Subset accuracy = ${metrics.subsetAccuracy}") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MulticlassMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MulticlassMetricsExample.scala new file mode 100644 index 000000000000..e0b98eeb446b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MulticlassMetricsExample.scala @@ -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. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS +import org.apache.spark.mllib.evaluation.MulticlassMetrics +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLUtils +// $example off$ + +object MulticlassMetricsExample { + + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("MulticlassMetricsExample") + val sc = new SparkContext(conf) + + // $example on$ + // Load training data in LIBSVM format + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_multiclass_classification_data.txt") + + // Split data into training (60%) and test (40%) + val Array(training, test) = data.randomSplit(Array(0.6, 0.4), seed = 11L) + training.cache() + + // Run training algorithm to build the model + val model = new LogisticRegressionWithLBFGS() + .setNumClasses(3) + .run(training) + + // Compute raw scores on the test set + val predictionAndLabels = test.map { case LabeledPoint(label, features) => + val prediction = model.predict(features) + (prediction, label) + } + + // Instantiate metrics object + val metrics = new MulticlassMetrics(predictionAndLabels) + + // Confusion matrix + println("Confusion matrix:") + println(metrics.confusionMatrix) + + // Overall Statistics + val accuracy = metrics.accuracy + println("Summary Statistics") + println(s"Accuracy = $accuracy") + + // Precision by label + val labels = metrics.labels + labels.foreach { l => + println(s"Precision($l) = " + metrics.precision(l)) + } + + // Recall by label + labels.foreach { l => + println(s"Recall($l) = " + metrics.recall(l)) + } + + // False positive rate by label + labels.foreach { l => + println(s"FPR($l) = " + metrics.falsePositiveRate(l)) + } + + // F-measure by label + labels.foreach { l => + println(s"F1-Score($l) = " + metrics.fMeasure(l)) + } + + // Weighted stats + println(s"Weighted precision: ${metrics.weightedPrecision}") + println(s"Weighted recall: ${metrics.weightedRecall}") + println(s"Weighted F1 score: ${metrics.weightedFMeasure}") + println(s"Weighted false positive rate: ${metrics.weightedFalsePositiveRate}") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala index 5f839c75dd58..f9e47e485e72 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala @@ -20,11 +20,10 @@ package org.apache.spark.examples.mllib import scopt.OptionParser +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.{SparkConf, SparkContext} - /** * An example app for summarizing multivariate data from a file. Run with @@ -58,14 +57,13 @@ object MultivariateSummarizer { """.stripMargin) } - parser.parse(args, defaultParams).map { params => - run(params) - } getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { + def run(params: Params): Unit = { val conf = new SparkConf().setAppName(s"MultivariateSummarizer with $params") val sc = new SparkContext(conf) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/NaiveBayesExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/NaiveBayesExample.scala index a7a47c2a3556..24c8e3445e53 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/NaiveBayesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/NaiveBayesExample.scala @@ -18,29 +18,23 @@ // scalastyle:off println package org.apache.spark.examples.mllib +import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.mllib.classification.{NaiveBayes, NaiveBayesModel} -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLUtils // $example off$ -import org.apache.spark.{SparkConf, SparkContext} object NaiveBayesExample { - def main(args: Array[String]) : Unit = { + def main(args: Array[String]): Unit = { val conf = new SparkConf().setAppName("NaiveBayesExample") val sc = new SparkContext(conf) // $example on$ - val data = sc.textFile("data/mllib/sample_naive_bayes_data.txt") - val parsedData = data.map { line => - val parts = line.split(',') - LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) - } + // Load and parse the data file. + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") // Split data into training (60%) and test (40%). - val splits = parsedData.randomSplit(Array(0.6, 0.4), seed = 11L) - val training = splits(0) - val test = splits(1) + val Array(training, test) = data.randomSplit(Array(0.6, 0.4)) val model = NaiveBayes.train(training, lambda = 1.0, modelType = "multinomial") @@ -51,6 +45,8 @@ object NaiveBayesExample { model.save(sc, "target/tmp/myNaiveBayesModel") val sameModel = NaiveBayesModel.load(sc, "target/tmp/myNaiveBayesModel") // $example off$ + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala new file mode 100644 index 000000000000..b3a9604c2be3 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/NormalizerExample.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +// $example on$ +import org.apache.spark.mllib.feature.Normalizer +import org.apache.spark.mllib.util.MLUtils +// $example off$ + +object NormalizerExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("NormalizerExample") + val sc = new SparkContext(conf) + + // $example on$ + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + + val normalizer1 = new Normalizer() + val normalizer2 = new Normalizer(p = Double.PositiveInfinity) + + // Each sample in data1 will be normalized using $L^2$ norm. + val data1 = data.map(x => (x.label, normalizer1.transform(x.features))) + + // Each sample in data2 will be normalized using $L^\infty$ norm. + val data2 = data.map(x => (x.label, normalizer2.transform(x.features))) + // $example off$ + + println("data1: ") + data1.foreach(x => println(x)) + + println("data2: ") + data2.foreach(x => println(x)) + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PCAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PCAExample.scala new file mode 100644 index 000000000000..eb36697d94ba --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PCAExample.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +// $example on$ +import org.apache.spark.mllib.feature.PCA +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.{LabeledPoint, LinearRegressionWithSGD} +// $example off$ + +@deprecated("Deprecated since LinearRegressionWithSGD is deprecated. Use ml.feature.PCA", "2.0.0") +object PCAExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("PCAExample") + val sc = new SparkContext(conf) + + // $example on$ + val data = sc.textFile("data/mllib/ridge-data/lpsa.data").map { line => + val parts = line.split(',') + LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) + }.cache() + + val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) + val training = splits(0).cache() + val test = splits(1) + + val pca = new PCA(training.first().features.size / 2).fit(data.map(_.features)) + val training_pca = training.map(p => p.copy(features = pca.transform(p.features))) + val test_pca = test.map(p => p.copy(features = pca.transform(p.features))) + + val numIterations = 100 + val model = LinearRegressionWithSGD.train(training, numIterations) + val model_pca = LinearRegressionWithSGD.train(training_pca, numIterations) + + val valuesAndPreds = test.map { point => + val score = model.predict(point.features) + (score, point.label) + } + + val valuesAndPreds_pca = test_pca.map { point => + val score = model_pca.predict(point.features) + (score, point.label) + } + + val MSE = valuesAndPreds.map { case (v, p) => math.pow((v - p), 2) }.mean() + val MSE_pca = valuesAndPreds_pca.map { case (v, p) => math.pow((v - p), 2) }.mean() + + println("Mean Squared Error = " + MSE) + println("PCA Mean Squared Error = " + MSE_pca) + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PCAOnRowMatrixExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PCAOnRowMatrixExample.scala new file mode 100644 index 000000000000..da43a8d9c7e8 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PCAOnRowMatrixExample.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +// $example on$ +import org.apache.spark.mllib.linalg.Matrix +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.distributed.RowMatrix +// $example off$ + +object PCAOnRowMatrixExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("PCAOnRowMatrixExample") + val sc = new SparkContext(conf) + + // $example on$ + val data = Array( + Vectors.sparse(5, Seq((1, 1.0), (3, 7.0))), + Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0), + Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0)) + + val rows = sc.parallelize(data) + + val mat: RowMatrix = new RowMatrix(rows) + + // Compute the top 4 principal components. + // Principal components are stored in a local dense matrix. + val pc: Matrix = mat.computePrincipalComponents(4) + + // Project the rows to the linear space spanned by the top 4 principal components. + val projected: RowMatrix = mat.multiply(pc) + // $example off$ + val collect = projected.rows.collect() + println("Projected Row Matrix of principal component:") + collect.foreach { vector => println(vector) } + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PCAOnSourceVectorExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PCAOnSourceVectorExample.scala new file mode 100644 index 000000000000..cef5402581f5 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PCAOnSourceVectorExample.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +// $example on$ +import org.apache.spark.mllib.feature.PCA +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.rdd.RDD +// $example off$ + +object PCAOnSourceVectorExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("PCAOnSourceVectorExample") + val sc = new SparkContext(conf) + + // $example on$ + val data: RDD[LabeledPoint] = sc.parallelize(Seq( + new LabeledPoint(0, Vectors.dense(1, 0, 0, 0, 1)), + new LabeledPoint(1, Vectors.dense(1, 1, 0, 1, 0)), + new LabeledPoint(1, Vectors.dense(1, 1, 0, 0, 0)), + new LabeledPoint(0, Vectors.dense(1, 0, 0, 0, 0)), + new LabeledPoint(1, Vectors.dense(1, 1, 0, 0, 0)))) + + // Compute the top 5 principal components. + val pca = new PCA(5).fit(data.map(_.features)) + + // Project vectors to the linear space spanned by the top 5 principal + // components, keeping the label + val projected = data.map(p => p.copy(features = pca.transform(p.features))) + // $example off$ + val collect = projected.collect() + println("Projected vector of principal component:") + collect.foreach { vector => println(vector) } + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PMMLModelExportExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PMMLModelExportExample.scala new file mode 100644 index 000000000000..d74d74a37fb1 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PMMLModelExportExample.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.clustering.KMeans +import org.apache.spark.mllib.linalg.Vectors +// $example off$ + +object PMMLModelExportExample { + + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("PMMLModelExportExample") + val sc = new SparkContext(conf) + + // $example on$ + // Load and parse the data + val data = sc.textFile("data/mllib/kmeans_data.txt") + val parsedData = data.map(s => Vectors.dense(s.split(' ').map(_.toDouble))).cache() + + // Cluster the data into two classes using KMeans + val numClusters = 2 + val numIterations = 20 + val clusters = KMeans.train(parsedData, numClusters, numIterations) + + // Export to PMML to a String in PMML format + println("PMML Model:\n" + clusters.toPMML) + + // Export the model to a local file in PMML format + clusters.toPMML("/tmp/kmeans.xml") + + // Export the model to a directory on a distributed file system in PMML format + clusters.toPMML(sc, "/tmp/kmeans") + + // Export the model to the OutputStream in PMML format + clusters.toPMML(System.out) + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala index 072322395461..986496c0d943 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala @@ -21,9 +21,11 @@ package org.apache.spark.examples.mllib import org.apache.log4j.{Level, Logger} import scopt.OptionParser +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ import org.apache.spark.mllib.clustering.PowerIterationClustering +// $example off$ 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. @@ -40,27 +42,23 @@ import org.apache.spark.{SparkConf, SparkContext} * 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] + * ./bin/run-example mllib.PowerIterationClusteringExample -k 2 --n 10 --maxIterations 15 * + * Cluster assignments: 1 -> [0,1,2,3,4,5,6,7,8,9], + * 0 -> [10,11,12,13,14,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 + k: Int = 2, + numPoints: Int = 10, + maxIterations: Int = 15 ) extends AbstractParams[Params] def main(args: Array[String]) { @@ -69,7 +67,7 @@ object PowerIterationClusteringExample { val parser = new OptionParser[Params]("PowerIterationClusteringExample") { head("PowerIterationClusteringExample: an example PIC app using concentric circles.") opt[Int]('k', "k") - .text(s"number of circles (/clusters), default: ${defaultParams.k}") + .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}") @@ -77,19 +75,15 @@ object PowerIterationClusteringExample { opt[Int]("maxIterations") .text(s"number of iterations, default: ${defaultParams.maxIterations}") .action((x, c) => c.copy(maxIterations = x)) - opt[Double]('r', "r") - .text(s"radius of outermost circle, default: ${defaultParams.outerRadius}") - .action((x, c) => c.copy(outerRadius = x)) } - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { + def run(params: Params): Unit = { val conf = new SparkConf() .setMaster("local") .setAppName(s"PowerIterationClustering with $params") @@ -97,22 +91,25 @@ object PowerIterationClusteringExample { Logger.getRootLogger.setLevel(Level.WARN) - val circlesRdd = generateCirclesRdd(sc, params.k, params.numPoints, params.outerRadius) + // $example on$ + val circlesRdd = generateCirclesRdd(sc, params.k, params.numPoints) val model = new PowerIterationClustering() .setK(params.k) .setMaxIterations(params.maxIterations) + .setInitializationMode("degree") .run(circlesRdd) val clusters = model.assignments.collect().groupBy(_.cluster).mapValues(_.map(_.id)) - val assignments = clusters.toList.sortBy { case (k, v) => v.length} + val assignments = clusters.toList.sortBy { case (k, v) => v.length } val assignmentsStr = assignments .map { case (k, v) => - s"$k -> ${v.sorted.mkString("[", ",", "]")}" - }.mkString(",") + s"$k -> ${v.sorted.mkString("[", ",", "]")}" + }.mkString(", ") val sizesStr = assignments.map { - _._2.size + _._2.length }.sorted.mkString("(", ",", ")") println(s"Cluster assignments: $assignmentsStr\ncluster sizes: $sizesStr") + // $example off$ sc.stop() } @@ -124,20 +121,17 @@ object PowerIterationClusteringExample { } } - 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)) + def generateCirclesRdd( + sc: SparkContext, + nCircles: Int, + nPoints: Int): RDD[(Long, Long, Double)] = { + val points = (1 to nCircles).flatMap { i => + generateCircle(i, i * nPoints) }.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))) + Some((i0.toLong, i1.toLong, gaussianSimilarity((x0, y0), (x1, y1)))) } else { None } @@ -148,11 +142,9 @@ object PowerIterationClusteringExample { /** * Gaussian Similarity: http://en.wikipedia.org/wiki/Radial_basis_function_kernel */ - def gaussianSimilarity(p1: (Double, Double), p2: (Double, Double), sigma: Double): Double = { - val coeff = 1.0 / (math.sqrt(2.0 * math.Pi) * sigma) - val expCoeff = -1.0 / 2.0 * math.pow(sigma, 2.0) + def gaussianSimilarity(p1: (Double, Double), p2: (Double, Double)): Double = { 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(-ssquares / 2.0) } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PrefixSpanExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PrefixSpanExample.scala index d237232c430c..69c72c433657 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/PrefixSpanExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PrefixSpanExample.scala @@ -18,12 +18,11 @@ // scalastyle:off println package org.apache.spark.examples.mllib +import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.mllib.fpm.PrefixSpan // $example off$ -import org.apache.spark.{SparkConf, SparkContext} - object PrefixSpanExample { def main(args: Array[String]) { @@ -47,6 +46,8 @@ object PrefixSpanExample { ", " + freqSequence.freq) } // $example off$ + + sc.stop() } } // scalastyle:off println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RandomForestClassificationExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomForestClassificationExample.scala new file mode 100644 index 000000000000..f1ebdf1a733e --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomForestClassificationExample.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.tree.RandomForest +import org.apache.spark.mllib.tree.model.RandomForestModel +import org.apache.spark.mllib.util.MLUtils +// $example off$ + +object RandomForestClassificationExample { + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("RandomForestClassificationExample") + val sc = new SparkContext(conf) + // $example on$ + // Load and parse the data file. + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + // Split the data into training and test sets (30% held out for testing) + val splits = data.randomSplit(Array(0.7, 0.3)) + val (trainingData, testData) = (splits(0), splits(1)) + + // Train a RandomForest model. + // Empty categoricalFeaturesInfo indicates all features are continuous. + val numClasses = 2 + val categoricalFeaturesInfo = Map[Int, Int]() + val numTrees = 3 // Use more in practice. + val featureSubsetStrategy = "auto" // Let the algorithm choose. + val impurity = "gini" + val maxDepth = 4 + val maxBins = 32 + + val model = RandomForest.trainClassifier(trainingData, numClasses, categoricalFeaturesInfo, + numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins) + + // Evaluate model on test instances and compute test error + val labelAndPreds = testData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) + } + val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count() + println("Test Error = " + testErr) + println("Learned classification forest model:\n" + model.toDebugString) + + // Save and load model + model.save(sc, "target/tmp/myRandomForestClassificationModel") + val sameModel = RandomForestModel.load(sc, "target/tmp/myRandomForestClassificationModel") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RandomForestRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomForestRegressionExample.scala new file mode 100644 index 000000000000..11d612e651b4 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomForestRegressionExample.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.tree.RandomForest +import org.apache.spark.mllib.tree.model.RandomForestModel +import org.apache.spark.mllib.util.MLUtils +// $example off$ + +object RandomForestRegressionExample { + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("RandomForestRegressionExample") + val sc = new SparkContext(conf) + // $example on$ + // Load and parse the data file. + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + // Split the data into training and test sets (30% held out for testing) + val splits = data.randomSplit(Array(0.7, 0.3)) + val (trainingData, testData) = (splits(0), splits(1)) + + // Train a RandomForest model. + // Empty categoricalFeaturesInfo indicates all features are continuous. + val numClasses = 2 + val categoricalFeaturesInfo = Map[Int, Int]() + val numTrees = 3 // Use more in practice. + val featureSubsetStrategy = "auto" // Let the algorithm choose. + val impurity = "variance" + val maxDepth = 4 + val maxBins = 32 + + val model = RandomForest.trainRegressor(trainingData, categoricalFeaturesInfo, + numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins) + + // Evaluate model on test instances and compute test error + val labelsAndPredictions = testData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) + } + val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() + println("Test Mean Squared Error = " + testMSE) + println("Learned regression forest model:\n" + model.toDebugString) + + // Save and load model + model.save(sc, "target/tmp/myRandomForestRegressionModel") + val sameModel = RandomForestModel.load(sc, "target/tmp/myRandomForestRegressionModel") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println + diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala index bee85ba0f996..7ccbb5a0640c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala @@ -18,11 +18,10 @@ // scalastyle:off println package org.apache.spark.examples.mllib +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.random.RandomRDDs import org.apache.spark.rdd.RDD -import org.apache.spark.{SparkConf, SparkContext} - /** * An example app for randomly generated RDDs. Run with * {{{ diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala new file mode 100644 index 000000000000..d514891da78f --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +// $example on$ +import org.apache.spark.mllib.evaluation.{RankingMetrics, RegressionMetrics} +import org.apache.spark.mllib.recommendation.{ALS, Rating} +// $example off$ +import org.apache.spark.sql.SparkSession + +object RankingMetricsExample { + def main(args: Array[String]) { + val spark = SparkSession + .builder + .appName("RankingMetricsExample") + .getOrCreate() + import spark.implicits._ + // $example on$ + // Read in the ratings data + val ratings = spark.read.textFile("data/mllib/sample_movielens_data.txt").rdd.map { line => + val fields = line.split("::") + Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble - 2.5) + }.cache() + + // Map ratings to 1 or 0, 1 indicating a movie that should be recommended + val binarizedRatings = ratings.map(r => Rating(r.user, r.product, + if (r.rating > 0) 1.0 else 0.0)).cache() + + // Summarize ratings + val numRatings = ratings.count() + val numUsers = ratings.map(_.user).distinct().count() + val numMovies = ratings.map(_.product).distinct().count() + println(s"Got $numRatings ratings from $numUsers users on $numMovies movies.") + + // Build the model + val numIterations = 10 + val rank = 10 + val lambda = 0.01 + val model = ALS.train(ratings, rank, numIterations, lambda) + + // Define a function to scale ratings from 0 to 1 + def scaledRating(r: Rating): Rating = { + val scaledRating = math.max(math.min(r.rating, 1.0), 0.0) + Rating(r.user, r.product, scaledRating) + } + + // Get sorted top ten predictions for each user and then scale from [0, 1] + val userRecommended = model.recommendProductsForUsers(10).map { case (user, recs) => + (user, recs.map(scaledRating)) + } + + // Assume that any movie a user rated 3 or higher (which maps to a 1) is a relevant document + // Compare with top ten most relevant documents + val userMovies = binarizedRatings.groupBy(_.user) + val relevantDocuments = userMovies.join(userRecommended).map { case (user, (actual, + predictions)) => + (predictions.map(_.product), actual.filter(_.rating > 0.0).map(_.product).toArray) + } + + // Instantiate metrics object + val metrics = new RankingMetrics(relevantDocuments) + + // Precision at K + Array(1, 3, 5).foreach { k => + println(s"Precision at $k = ${metrics.precisionAt(k)}") + } + + // Mean average precision + println(s"Mean average precision = ${metrics.meanAveragePrecision}") + + // Normalized discounted cumulative gain + Array(1, 3, 5).foreach { k => + println(s"NDCG at $k = ${metrics.ndcgAt(k)}") + } + + // Get predictions for each data point + val allPredictions = model.predict(ratings.map(r => (r.user, r.product))).map(r => ((r.user, + r.product), r.rating)) + val allRatings = ratings.map(r => ((r.user, r.product), r.rating)) + val predictionsAndLabels = allPredictions.join(allRatings).map { case ((user, product), + (predicted, actual)) => + (predicted, actual) + } + + // Get the RMSE using regression metrics + val regressionMetrics = new RegressionMetrics(predictionsAndLabels) + println(s"RMSE = ${regressionMetrics.rootMeanSquaredError}") + + // R-squared + println(s"R-squared = ${regressionMetrics.r2}") + // $example off$ + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RecommendationExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RecommendationExample.scala new file mode 100644 index 000000000000..6df742d737e7 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RecommendationExample.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.recommendation.ALS +import org.apache.spark.mllib.recommendation.MatrixFactorizationModel +import org.apache.spark.mllib.recommendation.Rating +// $example off$ + +object RecommendationExample { + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("CollaborativeFilteringExample") + val sc = new SparkContext(conf) + // $example on$ + // Load and parse the data + val data = sc.textFile("data/mllib/als/test.data") + val ratings = data.map(_.split(',') match { case Array(user, item, rate) => + Rating(user.toInt, item.toInt, rate.toDouble) + }) + + // Build the recommendation model using ALS + val rank = 10 + val numIterations = 10 + val model = ALS.train(ratings, rank, numIterations, 0.01) + + // Evaluate the model on rating data + val usersProducts = ratings.map { case Rating(user, product, rate) => + (user, product) + } + val predictions = + model.predict(usersProducts).map { case Rating(user, product, rate) => + ((user, product), rate) + } + val ratesAndPreds = ratings.map { case Rating(user, product, rate) => + ((user, product), rate) + }.join(predictions) + val MSE = ratesAndPreds.map { case ((user, product), (r1, r2)) => + val err = (r1 - r2) + err * err + }.mean() + println("Mean Squared Error = " + MSE) + + // Save and load model + model.save(sc, "target/tmp/myCollaborativeFilter") + val sameModel = MatrixFactorizationModel.load(sc, "target/tmp/myCollaborativeFilter") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala new file mode 100644 index 000000000000..76cfb804e18f --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +// scalastyle:off println + +package org.apache.spark.examples.mllib + +// $example on$ +import org.apache.spark.mllib.evaluation.RegressionMetrics +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.{LabeledPoint, LinearRegressionWithSGD} +// $example off$ +import org.apache.spark.sql.SparkSession + +@deprecated("Use ml.regression.LinearRegression and the resulting model summary for metrics", + "2.0.0") +object RegressionMetricsExample { + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("RegressionMetricsExample") + .getOrCreate() + // $example on$ + // Load the data + val data = spark + .read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") + .rdd.map(row => LabeledPoint(row.getDouble(0), row.get(1).asInstanceOf[Vector])) + .cache() + + // Build the model + val numIterations = 100 + val model = LinearRegressionWithSGD.train(data, numIterations) + + // Get predictions + val valuesAndPreds = data.map{ point => + val prediction = model.predict(point.features) + (prediction, point.label) + } + + // Instantiate metrics object + val metrics = new RegressionMetrics(valuesAndPreds) + + // Squared error + println(s"MSE = ${metrics.meanSquaredError}") + println(s"RMSE = ${metrics.rootMeanSquaredError}") + + // R-squared + println(s"R-squared = ${metrics.r2}") + + // Mean absolute error + println(s"MAE = ${metrics.meanAbsoluteError}") + + // Explained variance + println(s"Explained variance = ${metrics.explainedVariance}") + // $example off$ + + spark.stop() + } +} +// scalastyle:on println + diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SVDExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SVDExample.scala new file mode 100644 index 000000000000..769ae2a3a88b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SVDExample.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +// $example on$ +import org.apache.spark.mllib.linalg.Matrix +import org.apache.spark.mllib.linalg.SingularValueDecomposition +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.distributed.RowMatrix +// $example off$ + +/** + * Example for SingularValueDecomposition. + */ +object SVDExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("SVDExample") + val sc = new SparkContext(conf) + + // $example on$ + val data = Array( + Vectors.sparse(5, Seq((1, 1.0), (3, 7.0))), + Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0), + Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0)) + + val rows = sc.parallelize(data) + + val mat: RowMatrix = new RowMatrix(rows) + + // Compute the top 5 singular values and corresponding singular vectors. + val svd: SingularValueDecomposition[RowMatrix, Matrix] = mat.computeSVD(5, computeU = true) + val U: RowMatrix = svd.U // The U factor is a RowMatrix. + val s: Vector = svd.s // The singular values are stored in a local dense vector. + val V: Matrix = svd.V // The V factor is a local dense matrix. + // $example off$ + val collect = U.rows.collect() + println("U factor is:") + collect.foreach { vector => println(vector) } + println(s"Singular values are: $s") + println(s"V factor is:\n$V") + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SVMWithSGDExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SVMWithSGDExample.scala new file mode 100644 index 000000000000..b73fe9b2b3fa --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SVMWithSGDExample.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.classification.{SVMModel, SVMWithSGD} +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics +import org.apache.spark.mllib.util.MLUtils +// $example off$ + +object SVMWithSGDExample { + + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("SVMWithSGDExample") + val sc = new SparkContext(conf) + + // $example on$ + // Load training data in LIBSVM format. + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + + // Split data into training (60%) and test (40%). + val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) + val training = splits(0).cache() + val test = splits(1) + + // Run training algorithm to build the model + val numIterations = 100 + val model = SVMWithSGD.train(training, numIterations) + + // Clear the default threshold. + model.clearThreshold() + + // Compute raw scores on the test set. + val scoreAndLabels = test.map { point => + val score = model.predict(point.features) + (score, point.label) + } + + // Get evaluation metrics. + val metrics = new BinaryClassificationMetrics(scoreAndLabels) + val auROC = metrics.areaUnderROC() + + println("Area under ROC = " + auROC) + + // Save and load model + model.save(sc, "target/tmp/scalaSVMWithSGDModel") + val sameModel = SVMModel.load(sc, "target/tmp/scalaSVMWithSGDModel") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala index 6963f43e082c..ba3deae5d688 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala @@ -18,11 +18,10 @@ // scalastyle:off println package org.apache.spark.examples.mllib -import org.apache.spark.mllib.util.MLUtils import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.util.MLUtils /** * An example app for randomly generated and sampled RDDs. Run with @@ -53,14 +52,13 @@ object SampledRDDs { """.stripMargin) } - parser.parse(args, defaultParams).map { params => - run(params) - } getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { + def run(params: Params): Unit = { val conf = new SparkConf().setAppName(s"SampledRDDs with $params") val sc = new SparkContext(conf) @@ -79,7 +77,7 @@ object SampledRDDs { val sampledRDD = examples.sample(withReplacement = true, fraction = fraction) println(s" RDD.sample(): sample has ${sampledRDD.count()} examples") val sampledArray = examples.takeSample(withReplacement = true, num = expectedSampleSize) - println(s" RDD.takeSample(): sample has ${sampledArray.size} examples") + println(s" RDD.takeSample(): sample has ${sampledArray.length} examples") println() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SimpleFPGrowth.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SimpleFPGrowth.scala index b4e06afa7410..b5c3033bcba0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SimpleFPGrowth.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SimpleFPGrowth.scala @@ -18,13 +18,12 @@ // scalastyle:off println package org.apache.spark.examples.mllib +import org.apache.spark.{SparkConf, SparkContext} // $example on$ import org.apache.spark.mllib.fpm.FPGrowth import org.apache.spark.rdd.RDD // $example off$ -import org.apache.spark.{SparkContext, SparkConf} - object SimpleFPGrowth { def main(args: Array[String]) { @@ -54,6 +53,8 @@ object SimpleFPGrowth { + ", " + rule.confidence) } // $example off$ + + sc.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala index f81fc292a3bd..b76add2f9bc9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala @@ -60,14 +60,13 @@ object SparseNaiveBayes { .action((x, c) => c.copy(input = x)) } - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - sys.exit(1) + parser.parse(args, defaultParams) match { + case Some(params) => run(params) + case _ => sys.exit(1) } } - def run(params: Params) { + def run(params: Params): Unit = { val conf = new SparkConf().setAppName(s"SparseNaiveBayes with $params") val sc = new SparkContext(conf) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala new file mode 100644 index 000000000000..769fc17b3dc6 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StandardScalerExample.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +// $example on$ +import org.apache.spark.mllib.feature.{StandardScaler, StandardScalerModel} +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils +// $example off$ + +object StandardScalerExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("StandardScalerExample") + val sc = new SparkContext(conf) + + // $example on$ + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + + val scaler1 = new StandardScaler().fit(data.map(x => x.features)) + val scaler2 = new StandardScaler(withMean = true, withStd = true).fit(data.map(x => x.features)) + // scaler3 is an identical model to scaler2, and will produce identical transformations + val scaler3 = new StandardScalerModel(scaler2.std, scaler2.mean) + + // data1 will be unit variance. + val data1 = data.map(x => (x.label, scaler1.transform(x.features))) + + // data2 will be unit variance and zero mean. + val data2 = data.map(x => (x.label, scaler2.transform(Vectors.dense(x.features.toArray)))) + // $example off$ + + println("data1: ") + data1.foreach(x => println(x)) + + println("data2: ") + data2.foreach(x => println(x)) + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StratifiedSamplingExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StratifiedSamplingExample.scala new file mode 100644 index 000000000000..16b074ef6069 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StratifiedSamplingExample.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} + +object StratifiedSamplingExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("StratifiedSamplingExample") + val sc = new SparkContext(conf) + + // $example on$ + // an RDD[(K, V)] of any key value pairs + val data = sc.parallelize( + Seq((1, 'a'), (1, 'b'), (2, 'c'), (2, 'd'), (2, 'e'), (3, 'f'))) + + // specify the exact fraction desired from each key + val fractions = Map(1 -> 0.1, 2 -> 0.6, 3 -> 0.3) + + // Get an approximate sample from each stratum + val approxSample = data.sampleByKey(withReplacement = false, fractions = fractions) + // Get an exact sample from each stratum + val exactSample = data.sampleByKeyExact(withReplacement = false, fractions = fractions) + // $example off$ + + println("approxSample size is " + approxSample.collect().size.toString) + approxSample.collect().foreach(println) + + println("exactSample its size is " + exactSample.collect().size.toString) + exactSample.collect().foreach(println) + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala index af03724a8ac6..7888af79f87f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala @@ -19,10 +19,12 @@ package org.apache.spark.examples.mllib import org.apache.spark.SparkConf +// $example on$ import org.apache.spark.mllib.clustering.StreamingKMeans import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.streaming.{Seconds, StreamingContext} +// $example off$ /** * Estimate clusters on one stream of data and make predictions @@ -58,7 +60,8 @@ object StreamingKMeansExample { System.exit(1) } - val conf = new SparkConf().setMaster("local").setAppName("StreamingKMeansExample") + // $example on$ + val conf = new SparkConf().setAppName("StreamingKMeansExample") val ssc = new StreamingContext(conf, Seconds(args(2).toLong)) val trainingData = ssc.textFileStream(args(0)).map(Vectors.parse) @@ -74,6 +77,7 @@ object StreamingKMeansExample { ssc.start() ssc.awaitTermination() + // $example off$ } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala deleted file mode 100644 index b4a5dca031ab..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala +++ /dev/null @@ -1,73 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.mllib - -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.{LabeledPoint, StreamingLinearRegressionWithSGD} -import org.apache.spark.SparkConf -import org.apache.spark.streaming.{Seconds, StreamingContext} - -/** - * Train a linear regression model on one stream of data and make predictions - * on another stream, where the data streams arrive as text files - * into two different directories. - * - * The rows of the text files must be labeled data points in the form - * `(y,[x1,x2,x3,...,xn])` - * Where n is the number of features. n must be the same for train and test. - * - * Usage: StreamingLinearRegression - * - * To run on your local machine using the two directories `trainingDir` and `testDir`, - * with updates every 5 seconds, and 2 features per data point, call: - * $ bin/run-example mllib.StreamingLinearRegression trainingDir testDir 5 2 - * - * As you add text files to `trainingDir` the model will continuously update. - * Anytime you add text files to `testDir`, you'll see predictions from the current model. - * - */ -object StreamingLinearRegression { - - def main(args: Array[String]) { - - if (args.length != 4) { - System.err.println( - "Usage: StreamingLinearRegression ") - System.exit(1) - } - - val conf = new SparkConf().setMaster("local").setAppName("StreamingLinearRegression") - val ssc = new StreamingContext(conf, Seconds(args(2).toLong)) - - val trainingData = ssc.textFileStream(args(0)).map(LabeledPoint.parse) - val testData = ssc.textFileStream(args(1)).map(LabeledPoint.parse) - - val model = new StreamingLinearRegressionWithSGD() - .setInitialWeights(Vectors.zeros(args(3).toInt)) - - model.trainOn(trainingData) - model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print() - - ssc.start() - ssc.awaitTermination() - - } - -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegressionExample.scala new file mode 100644 index 000000000000..2ba1a62e450e --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegressionExample.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. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.SparkConf +// $example on$ +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD +// $example off$ +import org.apache.spark.streaming._ + +/** + * Train a linear regression model on one stream of data and make predictions + * on another stream, where the data streams arrive as text files + * into two different directories. + * + * The rows of the text files must be labeled data points in the form + * `(y,[x1,x2,x3,...,xn])` + * Where n is the number of features. n must be the same for train and test. + * + * Usage: StreamingLinearRegressionExample + * + * To run on your local machine using the two directories `trainingDir` and `testDir`, + * with updates every 5 seconds, and 2 features per data point, call: + * $ bin/run-example mllib.StreamingLinearRegressionExample trainingDir testDir + * + * As you add text files to `trainingDir` the model will continuously update. + * Anytime you add text files to `testDir`, you'll see predictions from the current model. + * + */ +object StreamingLinearRegressionExample { + + def main(args: Array[String]): Unit = { + if (args.length != 2) { + System.err.println("Usage: StreamingLinearRegressionExample ") + System.exit(1) + } + + val conf = new SparkConf().setAppName("StreamingLinearRegressionExample") + val ssc = new StreamingContext(conf, Seconds(1)) + + // $example on$ + val trainingData = ssc.textFileStream(args(0)).map(LabeledPoint.parse).cache() + val testData = ssc.textFileStream(args(1)).map(LabeledPoint.parse) + + val numFeatures = 3 + val model = new StreamingLinearRegressionWithSGD() + .setInitialWeights(Vectors.zeros(numFeatures)) + + model.trainOn(trainingData) + model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print() + + ssc.start() + ssc.awaitTermination() + // $example off$ + + ssc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala index b42f4cb5f933..a8b144a19722 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala @@ -18,10 +18,10 @@ // scalastyle:off println package org.apache.spark.examples.mllib +import org.apache.spark.SparkConf +import org.apache.spark.mllib.classification.StreamingLogisticRegressionWithSGD import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.classification.StreamingLogisticRegressionWithSGD -import org.apache.spark.SparkConf import org.apache.spark.streaming.{Seconds, StreamingContext} /** diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingTestExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingTestExample.scala index ab29f90254d3..ae4dee24c647 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingTestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingTestExample.scala @@ -18,7 +18,7 @@ package org.apache.spark.examples.mllib import org.apache.spark.SparkConf -import org.apache.spark.mllib.stat.test.StreamingTest +import org.apache.spark.mllib.stat.test.{BinarySample, StreamingTest} import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.util.Utils @@ -59,13 +59,14 @@ object StreamingTestExample { val conf = new SparkConf().setMaster("local").setAppName("StreamingTestExample") val ssc = new StreamingContext(conf, batchDuration) - ssc.checkpoint({ + ssc.checkpoint { val dir = Utils.createTempDir() dir.toString - }) + } + // $example on$ val data = ssc.textFileStream(dataDir).map(line => line.split(",") match { - case Array(label, value) => (label.toBoolean, value.toDouble) + case Array(label, value) => BinarySample(label.toBoolean, value.toDouble) }) val streamingTest = new StreamingTest() @@ -75,6 +76,7 @@ object StreamingTestExample { val out = streamingTest.registerStream(data) out.print() + // $example off$ // Stop processing if test becomes significant or we time out var timeoutCounter = numBatchesTimeout diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SummaryStatisticsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SummaryStatisticsExample.scala new file mode 100644 index 000000000000..948b443c0a75 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SummaryStatisticsExample.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +// $example on$ +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} +// $example off$ + +object SummaryStatisticsExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("SummaryStatisticsExample") + val sc = new SparkContext(conf) + + // $example on$ + val observations = sc.parallelize( + Seq( + Vectors.dense(1.0, 10.0, 100.0), + Vectors.dense(2.0, 20.0, 200.0), + Vectors.dense(3.0, 30.0, 300.0) + ) + ) + + // Compute column summary statistics. + val summary: MultivariateStatisticalSummary = Statistics.colStats(observations) + println(summary.mean) // a dense vector containing the mean value for each column + println(summary.variance) // column-wise variance + println(summary.numNonzeros) // number of nonzeros in each column + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala new file mode 100644 index 000000000000..a5bdcd8f2ed3 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TFIDFExample.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +// $example on$ +import org.apache.spark.mllib.feature.{HashingTF, IDF} +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.rdd.RDD +// $example off$ + +object TFIDFExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("TFIDFExample") + val sc = new SparkContext(conf) + + // $example on$ + // Load documents (one per line). + val documents: RDD[Seq[String]] = sc.textFile("data/mllib/kmeans_data.txt") + .map(_.split(" ").toSeq) + + val hashingTF = new HashingTF() + val tf: RDD[Vector] = hashingTF.transform(documents) + + // While applying HashingTF only needs a single pass to the data, applying IDF needs two passes: + // First to compute the IDF vector and second to scale the term frequencies by IDF. + tf.cache() + val idf = new IDF().fit(tf) + val tfidf: RDD[Vector] = idf.transform(tf) + + // spark.mllib IDF implementation provides an option for ignoring terms which occur in less than + // a minimum number of documents. In such cases, the IDF for these terms is set to 0. + // This feature can be used by passing the minDocFreq value to the IDF constructor. + val idfIgnore = new IDF(minDocFreq = 2).fit(tf) + val tfidfIgnore: RDD[Vector] = idfIgnore.transform(tf) + // $example off$ + + println("tfidf: ") + tfidf.foreach(x => println(x)) + + println("tfidfIgnore: ") + tfidfIgnore.foreach(x => println(x)) + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala index 464fbd385ab5..03bc675299c5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -19,8 +19,8 @@ package org.apache.spark.examples.mllib import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.distributed.RowMatrix /** * Compute the principal components of a tall-and-skinny matrix, whose rows are observations. diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala index 65b4bc46f026..067e49b9599e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -19,8 +19,8 @@ package org.apache.spark.examples.mllib import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.distributed.RowMatrix /** * Compute the singular value decomposition (SVD) of a tall-and-skinny matrix. diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/Word2VecExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/Word2VecExample.scala new file mode 100644 index 000000000000..ea794c700ae7 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/Word2VecExample.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. + */ + +// scalastyle:off println +package org.apache.spark.examples.mllib + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +// $example on$ +import org.apache.spark.mllib.feature.{Word2Vec, Word2VecModel} +// $example off$ + +object Word2VecExample { + + def main(args: Array[String]): Unit = { + + val conf = new SparkConf().setAppName("Word2VecExample") + val sc = new SparkContext(conf) + + // $example on$ + val input = sc.textFile("data/mllib/sample_lda_data.txt").map(line => line.split(" ").toSeq) + + val word2vec = new Word2Vec() + + val model = word2vec.fit(input) + + val synonyms = model.findSynonyms("1", 5) + + for((synonym, cosineSimilarity) <- synonyms) { + println(s"$synonym $cosineSimilarity") + } + + // Save and load model + model.save(sc, "myModelPath") + val sameModel = Word2VecModel.load(sc, "myModelPath") + // $example off$ + + sc.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala index 805184e740f0..6bd96346ca23 100644 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala @@ -21,13 +21,13 @@ import java.util.{Collection => JCollection, Map => JMap} import scala.collection.JavaConverters._ -import org.apache.avro.generic.{GenericFixed, IndexedRecord} -import org.apache.avro.mapred.AvroWrapper import org.apache.avro.Schema import org.apache.avro.Schema.Type._ +import org.apache.avro.generic.{GenericFixed, IndexedRecord} +import org.apache.avro.mapred.AvroWrapper -import org.apache.spark.api.python.Converter import org.apache.spark.SparkException +import org.apache.spark.api.python.Converter object AvroConversionUtil extends Serializable { @@ -79,7 +79,10 @@ object AvroConversionUtil extends Serializable { def unpackBytes(obj: Any): Array[Byte] = { val bytes: Array[Byte] = obj match { - case buf: java.nio.ByteBuffer => buf.array() + case buf: java.nio.ByteBuffer => + val arr = new Array[Byte](buf.remaining()) + buf.get(arr) + arr case arr: Array[Byte] => arr case other => throw new SparkException( s"Unknown BYTES type ${other.getClass.getName}") diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala deleted file mode 100644 index 00ce47af4813..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala +++ /dev/null @@ -1,70 +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.examples.pythonconverters - -import java.nio.ByteBuffer - -import scala.collection.JavaConverters._ - -import org.apache.cassandra.utils.ByteBufferUtil - -import org.apache.spark.api.python.Converter - -/** - * Implementation of [[org.apache.spark.api.python.Converter]] that converts Cassandra - * output to a Map[String, Int] - */ -class CassandraCQLKeyConverter extends Converter[Any, java.util.Map[String, Int]] { - override def convert(obj: Any): java.util.Map[String, Int] = { - val result = obj.asInstanceOf[java.util.Map[String, ByteBuffer]] - result.asScala.mapValues(ByteBufferUtil.toInt).asJava - } -} - -/** - * Implementation of [[org.apache.spark.api.python.Converter]] that converts Cassandra - * output to a Map[String, String] - */ -class CassandraCQLValueConverter extends Converter[Any, java.util.Map[String, String]] { - override def convert(obj: Any): java.util.Map[String, String] = { - val result = obj.asInstanceOf[java.util.Map[String, ByteBuffer]] - result.asScala.mapValues(ByteBufferUtil.string).asJava - } -} - -/** - * Implementation of [[org.apache.spark.api.python.Converter]] that converts a - * Map[String, Int] to Cassandra key - */ -class ToCassandraCQLKeyConverter extends Converter[Any, java.util.Map[String, ByteBuffer]] { - override def convert(obj: Any): java.util.Map[String, ByteBuffer] = { - val input = obj.asInstanceOf[java.util.Map[String, Int]] - input.asScala.mapValues(ByteBufferUtil.bytes).asJava - } -} - -/** - * Implementation of [[org.apache.spark.api.python.Converter]] that converts a - * List[String] to Cassandra value - */ -class ToCassandraCQLValueConverter extends Converter[Any, java.util.List[ByteBuffer]] { - override def convert(obj: Any): java.util.List[ByteBuffer] = { - val input = obj.asInstanceOf[java.util.List[String]] - input.asScala.map(ByteBufferUtil.bytes).asJava - } -} diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala deleted file mode 100644 index 0a25ee7ae56f..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala +++ /dev/null @@ -1,83 +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.examples.pythonconverters - -import scala.collection.JavaConverters._ -import scala.util.parsing.json.JSONObject - -import org.apache.spark.api.python.Converter -import org.apache.hadoop.hbase.client.{Put, Result} -import org.apache.hadoop.hbase.io.ImmutableBytesWritable -import org.apache.hadoop.hbase.util.Bytes -import org.apache.hadoop.hbase.KeyValue.Type -import org.apache.hadoop.hbase.CellUtil - -/** - * Implementation of [[org.apache.spark.api.python.Converter]] that converts all - * the records in an HBase Result to a String - */ -class HBaseResultToStringConverter extends Converter[Any, String] { - override def convert(obj: Any): String = { - val result = obj.asInstanceOf[Result] - val output = result.listCells.asScala.map(cell => - Map( - "row" -> Bytes.toStringBinary(CellUtil.cloneRow(cell)), - "columnFamily" -> Bytes.toStringBinary(CellUtil.cloneFamily(cell)), - "qualifier" -> Bytes.toStringBinary(CellUtil.cloneQualifier(cell)), - "timestamp" -> cell.getTimestamp.toString, - "type" -> Type.codeToType(cell.getTypeByte).toString, - "value" -> Bytes.toStringBinary(CellUtil.cloneValue(cell)) - ) - ) - output.map(JSONObject(_).toString()).mkString("\n") - } -} - -/** - * Implementation of [[org.apache.spark.api.python.Converter]] that converts an - * ImmutableBytesWritable to a String - */ -class ImmutableBytesWritableToStringConverter extends Converter[Any, String] { - override def convert(obj: Any): String = { - val key = obj.asInstanceOf[ImmutableBytesWritable] - Bytes.toStringBinary(key.get()) - } -} - -/** - * Implementation of [[org.apache.spark.api.python.Converter]] that converts a - * String to an ImmutableBytesWritable - */ -class StringToImmutableBytesWritableConverter extends Converter[Any, ImmutableBytesWritable] { - override def convert(obj: Any): ImmutableBytesWritable = { - val bytes = Bytes.toBytes(obj.asInstanceOf[String]) - new ImmutableBytesWritable(bytes) - } -} - -/** - * Implementation of [[org.apache.spark.api.python.Converter]] that converts a - * list of Strings to HBase Put - */ -class StringListToPutConverter extends Converter[Any, Put] { - override def convert(obj: Any): Put = { - val output = obj.asInstanceOf[java.util.ArrayList[String]].asScala.map(Bytes.toBytes).toArray - val put = new Put(output(0)) - put.add(output(1), output(2), output(3)) - } -} 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 2cc56f04e5c1..deaa9f252b9b 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 @@ -18,9 +18,10 @@ // scalastyle:off println package org.apache.spark.examples.sql -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.functions._ +import org.apache.spark.sql.SaveMode +// $example on:init_session$ +import org.apache.spark.sql.SparkSession +// $example off:init_session$ // One method for defining the schema of an RDD is to make a case class with the desired column // names and types. @@ -28,50 +29,54 @@ case class Record(key: Int, value: String) object RDDRelation { def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("RDDRelation") - val sc = new SparkContext(sparkConf) - val sqlContext = new SQLContext(sc) - - // 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() - // 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") + // $example on:init_session$ + val spark = SparkSession + .builder + .appName("Spark Examples") + .config("spark.some.config.option", "some-value") + .getOrCreate() + + // Importing the SparkSession gives access to all the SQL functions and implicit conversions. + import spark.implicits._ + // $example off:init_session$ + + val df = spark.createDataFrame((1 to 100).map(i => Record(i, s"val_$i"))) + // Any RDD containing case classes can be used to create a temporary view. The schema of the + // view is automatically inferred using scala reflection. + df.createOrReplaceTempView("records") // Once tables have been registered, you can run SQL queries over them. println("Result of SELECT *:") - sqlContext.sql("SELECT * FROM records").collect().foreach(println) + spark.sql("SELECT * FROM records").collect().foreach(println) // Aggregation queries are also supported. - val count = sqlContext.sql("SELECT COUNT(*) FROM records").collect().head.getLong(0) + val count = spark.sql("SELECT COUNT(*) FROM records").collect().head.getLong(0) println(s"COUNT(*): $count") - // The results of SQL queries are themselves RDDs and support all normal RDD functions. The + // The results of SQL queries are themselves RDDs and support all normal RDD functions. The // items in the RDD are of type Row, which allows you to access each column by ordinal. - val rddFromSql = sqlContext.sql("SELECT key, value FROM records WHERE key < 10") + val rddFromSql = spark.sql("SELECT key, value FROM records WHERE key < 10") println("Result of RDD.map:") - rddFromSql.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect().foreach(println) + rddFromSql.rdd.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect().foreach(println) // Queries can also be written using a LINQ-like Scala DSL. df.where($"key" === 1).orderBy($"value".asc).select($"key").collect().foreach(println) - // Write out an RDD as a parquet file. - df.write.parquet("pair.parquet") + // Write out an RDD as a parquet file with overwrite mode. + df.write.mode(SaveMode.Overwrite).parquet("pair.parquet") - // Read in parquet file. Parquet files are self-describing so the schmema is preserved. - val parquetFile = sqlContext.read.parquet("pair.parquet") + // Read in parquet file. Parquet files are self-describing so the schema is preserved. + val parquetFile = spark.read.parquet("pair.parquet") - // Queries can be run using the DSL on parequet files just like the original RDD. + // Queries can be run using the DSL on parquet files just like the original RDD. parquetFile.where($"key" === 1).select($"value".as("a")).collect().foreach(println) - // These files can also be registered as tables. - parquetFile.registerTempTable("parquetFile") - sqlContext.sql("SELECT * FROM parquetFile").collect().foreach(println) + // These files can also be used to create a temporary view. + parquetFile.createOrReplaceTempView("parquetFile") + spark.sql("SELECT * FROM parquetFile").collect().foreach(println) - sc.stop() + spark.stop() } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala new file mode 100644 index 000000000000..86b3dc4a84f5 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala @@ -0,0 +1,211 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.sql + +import java.util.Properties + +import org.apache.spark.sql.SparkSession + +object SQLDataSourceExample { + + case class Person(name: String, age: Long) + + def main(args: Array[String]) { + val spark = SparkSession + .builder() + .appName("Spark SQL data sources example") + .config("spark.some.config.option", "some-value") + .getOrCreate() + + runBasicDataSourceExample(spark) + runBasicParquetExample(spark) + runParquetSchemaMergingExample(spark) + runJsonDatasetExample(spark) + runJdbcDatasetExample(spark) + + spark.stop() + } + + private def runBasicDataSourceExample(spark: SparkSession): Unit = { + // $example on:generic_load_save_functions$ + val usersDF = spark.read.load("examples/src/main/resources/users.parquet") + usersDF.select("name", "favorite_color").write.save("namesAndFavColors.parquet") + // $example off:generic_load_save_functions$ + // $example on:manual_load_options$ + val peopleDF = spark.read.format("json").load("examples/src/main/resources/people.json") + peopleDF.select("name", "age").write.format("parquet").save("namesAndAges.parquet") + // $example off:manual_load_options$ + // $example on:direct_sql$ + val sqlDF = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") + // $example off:direct_sql$ + // $example on:write_sorting_and_bucketing$ + peopleDF.write.bucketBy(42, "name").sortBy("age").saveAsTable("people_bucketed") + // $example off:write_sorting_and_bucketing$ + // $example on:write_partitioning$ + usersDF.write.partitionBy("favorite_color").format("parquet").save("namesPartByColor.parquet") + // $example off:write_partitioning$ + // $example on:write_partition_and_bucket$ + peopleDF + .write + .partitionBy("favorite_color") + .bucketBy(42, "name") + .saveAsTable("people_partitioned_bucketed") + // $example off:write_partition_and_bucket$ + + spark.sql("DROP TABLE IF EXISTS people_bucketed") + spark.sql("DROP TABLE IF EXISTS people_partitioned_bucketed") + } + + private def runBasicParquetExample(spark: SparkSession): Unit = { + // $example on:basic_parquet_example$ + // Encoders for most common types are automatically provided by importing spark.implicits._ + import spark.implicits._ + + val peopleDF = spark.read.json("examples/src/main/resources/people.json") + + // DataFrames can be saved as Parquet files, maintaining the schema information + peopleDF.write.parquet("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 DataFrame + val parquetFileDF = spark.read.parquet("people.parquet") + + // Parquet files can also be used to create a temporary view and then used in SQL statements + parquetFileDF.createOrReplaceTempView("parquetFile") + val namesDF = spark.sql("SELECT name FROM parquetFile WHERE age BETWEEN 13 AND 19") + namesDF.map(attributes => "Name: " + attributes(0)).show() + // +------------+ + // | value| + // +------------+ + // |Name: Justin| + // +------------+ + // $example off:basic_parquet_example$ + } + + private def runParquetSchemaMergingExample(spark: SparkSession): Unit = { + // $example on:schema_merging$ + // This is used to implicitly convert an RDD to a DataFrame. + import spark.implicits._ + + // Create a simple DataFrame, store into a partition directory + val squaresDF = spark.sparkContext.makeRDD(1 to 5).map(i => (i, i * i)).toDF("value", "square") + squaresDF.write.parquet("data/test_table/key=1") + + // Create another DataFrame in a new partition directory, + // adding a new column and dropping an existing column + val cubesDF = spark.sparkContext.makeRDD(6 to 10).map(i => (i, i * i * i)).toDF("value", "cube") + cubesDF.write.parquet("data/test_table/key=2") + + // Read the partitioned table + val mergedDF = spark.read.option("mergeSchema", "true").parquet("data/test_table") + mergedDF.printSchema() + + // The final schema consists of all 3 columns in the Parquet files together + // with the partitioning column appeared in the partition directory paths + // root + // |-- value: int (nullable = true) + // |-- square: int (nullable = true) + // |-- cube: int (nullable = true) + // |-- key: int (nullable = true) + // $example off:schema_merging$ + } + + private def runJsonDatasetExample(spark: SparkSession): Unit = { + // $example on:json_dataset$ + // Primitive types (Int, String, etc) and Product types (case classes) encoders are + // supported by importing this when creating a Dataset. + import spark.implicits._ + + // 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" + val peopleDF = spark.read.json(path) + + // The inferred schema can be visualized using the printSchema() method + peopleDF.printSchema() + // root + // |-- age: long (nullable = true) + // |-- name: string (nullable = true) + + // Creates a temporary view using the DataFrame + peopleDF.createOrReplaceTempView("people") + + // SQL statements can be run by using the sql methods provided by spark + val teenagerNamesDF = spark.sql("SELECT name FROM people WHERE age BETWEEN 13 AND 19") + teenagerNamesDF.show() + // +------+ + // | name| + // +------+ + // |Justin| + // +------+ + + // Alternatively, a DataFrame can be created for a JSON dataset represented by + // a Dataset[String] storing one JSON object per string + val otherPeopleDataset = spark.createDataset( + """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) + val otherPeople = spark.read.json(otherPeopleDataset) + otherPeople.show() + // +---------------+----+ + // | address|name| + // +---------------+----+ + // |[Columbus,Ohio]| Yin| + // +---------------+----+ + // $example off:json_dataset$ + } + + private def runJdbcDatasetExample(spark: SparkSession): Unit = { + // $example on:jdbc_dataset$ + // Note: JDBC loading and saving can be achieved via either the load/save or jdbc methods + // Loading data from a JDBC source + val jdbcDF = spark.read + .format("jdbc") + .option("url", "jdbc:postgresql:dbserver") + .option("dbtable", "schema.tablename") + .option("user", "username") + .option("password", "password") + .load() + + val connectionProperties = new Properties() + connectionProperties.put("user", "username") + connectionProperties.put("password", "password") + val jdbcDF2 = spark.read + .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties) + // Specifying the custom data types of the read schema + connectionProperties.put("customSchema", "id DECIMAL(38, 0), name STRING") + val jdbcDF3 = spark.read + .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties) + + // Saving data to a JDBC source + jdbcDF.write + .format("jdbc") + .option("url", "jdbc:postgresql:dbserver") + .option("dbtable", "schema.tablename") + .option("user", "username") + .option("password", "password") + .save() + + jdbcDF2.write + .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties) + + // Specifying create table column data types on write + jdbcDF.write + .option("createTableColumnTypes", "name CHAR(64), comments VARCHAR(1024)") + .jdbc("jdbc:postgresql:dbserver", "schema.tablename", connectionProperties) + // $example off:jdbc_dataset$ + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala new file mode 100644 index 000000000000..958361a6684c --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala @@ -0,0 +1,273 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.sql + +import org.apache.spark.sql.Row +// $example on:init_session$ +import org.apache.spark.sql.SparkSession +// $example off:init_session$ +// $example on:programmatic_schema$ +// $example on:data_types$ +import org.apache.spark.sql.types._ +// $example off:data_types$ +// $example off:programmatic_schema$ + +object SparkSQLExample { + + // $example on:create_ds$ + case class Person(name: String, age: Long) + // $example off:create_ds$ + + def main(args: Array[String]) { + // $example on:init_session$ + val spark = SparkSession + .builder() + .appName("Spark SQL basic example") + .config("spark.some.config.option", "some-value") + .getOrCreate() + + // For implicit conversions like converting RDDs to DataFrames + import spark.implicits._ + // $example off:init_session$ + + runBasicDataFrameExample(spark) + runDatasetCreationExample(spark) + runInferSchemaExample(spark) + runProgrammaticSchemaExample(spark) + + spark.stop() + } + + private def runBasicDataFrameExample(spark: SparkSession): Unit = { + // $example on:create_df$ + val df = spark.read.json("examples/src/main/resources/people.json") + + // Displays the content of the DataFrame to stdout + df.show() + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:create_df$ + + // $example on:untyped_ops$ + // This import is needed to use the $-notation + import spark.implicits._ + // Print the schema in a tree format + df.printSchema() + // root + // |-- age: long (nullable = true) + // |-- name: string (nullable = true) + + // Select only the "name" column + df.select("name").show() + // +-------+ + // | name| + // +-------+ + // |Michael| + // | Andy| + // | Justin| + // +-------+ + + // Select everybody, but increment the age by 1 + df.select($"name", $"age" + 1).show() + // +-------+---------+ + // | name|(age + 1)| + // +-------+---------+ + // |Michael| null| + // | Andy| 31| + // | Justin| 20| + // +-------+---------+ + + // Select people older than 21 + df.filter($"age" > 21).show() + // +---+----+ + // |age|name| + // +---+----+ + // | 30|Andy| + // +---+----+ + + // Count people by age + df.groupBy("age").count().show() + // +----+-----+ + // | age|count| + // +----+-----+ + // | 19| 1| + // |null| 1| + // | 30| 1| + // +----+-----+ + // $example off:untyped_ops$ + + // $example on:run_sql$ + // Register the DataFrame as a SQL temporary view + df.createOrReplaceTempView("people") + + val sqlDF = spark.sql("SELECT * FROM people") + sqlDF.show() + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:run_sql$ + + // $example on:global_temp_view$ + // Register the DataFrame as a global temporary view + df.createGlobalTempView("people") + + // Global temporary view is tied to a system preserved database `global_temp` + spark.sql("SELECT * FROM global_temp.people").show() + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + + // Global temporary view is cross-session + spark.newSession().sql("SELECT * FROM global_temp.people").show() + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:global_temp_view$ + } + + private def runDatasetCreationExample(spark: SparkSession): Unit = { + import spark.implicits._ + // $example on:create_ds$ + // Encoders are created for case classes + val caseClassDS = Seq(Person("Andy", 32)).toDS() + caseClassDS.show() + // +----+---+ + // |name|age| + // +----+---+ + // |Andy| 32| + // +----+---+ + + // Encoders for most common types are automatically provided by importing spark.implicits._ + val primitiveDS = Seq(1, 2, 3).toDS() + primitiveDS.map(_ + 1).collect() // Returns: Array(2, 3, 4) + + // DataFrames can be converted to a Dataset by providing a class. Mapping will be done by name + val path = "examples/src/main/resources/people.json" + val peopleDS = spark.read.json(path).as[Person] + peopleDS.show() + // +----+-------+ + // | age| name| + // +----+-------+ + // |null|Michael| + // | 30| Andy| + // | 19| Justin| + // +----+-------+ + // $example off:create_ds$ + } + + private def runInferSchemaExample(spark: SparkSession): Unit = { + // $example on:schema_inferring$ + // For implicit conversions from RDDs to DataFrames + import spark.implicits._ + + // Create an RDD of Person objects from a text file, convert it to a Dataframe + val peopleDF = spark.sparkContext + .textFile("examples/src/main/resources/people.txt") + .map(_.split(",")) + .map(attributes => Person(attributes(0), attributes(1).trim.toInt)) + .toDF() + // Register the DataFrame as a temporary view + peopleDF.createOrReplaceTempView("people") + + // SQL statements can be run by using the sql methods provided by Spark + val teenagersDF = spark.sql("SELECT name, age FROM people WHERE age BETWEEN 13 AND 19") + + // The columns of a row in the result can be accessed by field index + teenagersDF.map(teenager => "Name: " + teenager(0)).show() + // +------------+ + // | value| + // +------------+ + // |Name: Justin| + // +------------+ + + // or by field name + teenagersDF.map(teenager => "Name: " + teenager.getAs[String]("name")).show() + // +------------+ + // | value| + // +------------+ + // |Name: Justin| + // +------------+ + + // No pre-defined encoders for Dataset[Map[K,V]], define explicitly + implicit val mapEncoder = org.apache.spark.sql.Encoders.kryo[Map[String, Any]] + // Primitive types and case classes can be also defined as + // implicit val stringIntMapEncoder: Encoder[Map[String, Any]] = ExpressionEncoder() + + // row.getValuesMap[T] retrieves multiple columns at once into a Map[String, T] + teenagersDF.map(teenager => teenager.getValuesMap[Any](List("name", "age"))).collect() + // Array(Map("name" -> "Justin", "age" -> 19)) + // $example off:schema_inferring$ + } + + private def runProgrammaticSchemaExample(spark: SparkSession): Unit = { + import spark.implicits._ + // $example on:programmatic_schema$ + // Create an RDD + val peopleRDD = spark.sparkContext.textFile("examples/src/main/resources/people.txt") + + // The schema is encoded in a string + val schemaString = "name age" + + // Generate the schema based on the string of schema + val fields = schemaString.split(" ") + .map(fieldName => StructField(fieldName, StringType, nullable = true)) + val schema = StructType(fields) + + // Convert records of the RDD (people) to Rows + val rowRDD = peopleRDD + .map(_.split(",")) + .map(attributes => Row(attributes(0), attributes(1).trim)) + + // Apply the schema to the RDD + val peopleDF = spark.createDataFrame(rowRDD, schema) + + // Creates a temporary view using the DataFrame + peopleDF.createOrReplaceTempView("people") + + // SQL can be run over a temporary view created using DataFrames + val results = spark.sql("SELECT name FROM people") + + // The results of SQL queries are DataFrames and support all the normal RDD operations + // The columns of a row in the result can be accessed by field index or by field name + results.map(attributes => "Name: " + attributes(0)).show() + // +-------------+ + // | value| + // +-------------+ + // |Name: Michael| + // | Name: Andy| + // | Name: Justin| + // +-------------+ + // $example off:programmatic_schema$ + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala new file mode 100644 index 000000000000..f04a831487b5 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql + +// $example on:typed_custom_aggregation$ +import org.apache.spark.sql.{Encoder, Encoders, SparkSession} +import org.apache.spark.sql.expressions.Aggregator +// $example off:typed_custom_aggregation$ + +object UserDefinedTypedAggregation { + + // $example on:typed_custom_aggregation$ + case class Employee(name: String, salary: Long) + case class Average(var sum: Long, var count: Long) + + object MyAverage extends Aggregator[Employee, Average, Double] { + // A zero value for this aggregation. Should satisfy the property that any b + zero = b + def zero: Average = Average(0L, 0L) + // Combine two values to produce a new value. For performance, the function may modify `buffer` + // and return it instead of constructing a new object + def reduce(buffer: Average, employee: Employee): Average = { + buffer.sum += employee.salary + buffer.count += 1 + buffer + } + // Merge two intermediate values + def merge(b1: Average, b2: Average): Average = { + b1.sum += b2.sum + b1.count += b2.count + b1 + } + // Transform the output of the reduction + def finish(reduction: Average): Double = reduction.sum.toDouble / reduction.count + // Specifies the Encoder for the intermediate value type + def bufferEncoder: Encoder[Average] = Encoders.product + // Specifies the Encoder for the final output value type + def outputEncoder: Encoder[Double] = Encoders.scalaDouble + } + // $example off:typed_custom_aggregation$ + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder() + .appName("Spark SQL user-defined Datasets aggregation example") + .getOrCreate() + + import spark.implicits._ + + // $example on:typed_custom_aggregation$ + val ds = spark.read.json("examples/src/main/resources/employees.json").as[Employee] + ds.show() + // +-------+------+ + // | name|salary| + // +-------+------+ + // |Michael| 3000| + // | Andy| 4500| + // | Justin| 3500| + // | Berta| 4000| + // +-------+------+ + + // Convert the function to a `TypedColumn` and give it a name + val averageSalary = MyAverage.toColumn.name("average_salary") + val result = ds.select(averageSalary) + result.show() + // +--------------+ + // |average_salary| + // +--------------+ + // | 3750.0| + // +--------------+ + // $example off:typed_custom_aggregation$ + + spark.stop() + } + +} diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala new file mode 100644 index 000000000000..3656a84c571d --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql + +// $example on:untyped_custom_aggregation$ +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.expressions.MutableAggregationBuffer +import org.apache.spark.sql.expressions.UserDefinedAggregateFunction +import org.apache.spark.sql.types._ +// $example off:untyped_custom_aggregation$ + +object UserDefinedUntypedAggregation { + + // $example on:untyped_custom_aggregation$ + object MyAverage extends UserDefinedAggregateFunction { + // Data types of input arguments of this aggregate function + def inputSchema: StructType = StructType(StructField("inputColumn", LongType) :: Nil) + // Data types of values in the aggregation buffer + def bufferSchema: StructType = { + StructType(StructField("sum", LongType) :: StructField("count", LongType) :: Nil) + } + // The data type of the returned value + def dataType: DataType = DoubleType + // Whether this function always returns the same output on the identical input + def deterministic: Boolean = true + // Initializes the given aggregation buffer. The buffer itself is a `Row` that in addition to + // standard methods like retrieving a value at an index (e.g., get(), getBoolean()), provides + // the opportunity to update its values. Note that arrays and maps inside the buffer are still + // immutable. + def initialize(buffer: MutableAggregationBuffer): Unit = { + buffer(0) = 0L + buffer(1) = 0L + } + // Updates the given aggregation buffer `buffer` with new input data from `input` + def update(buffer: MutableAggregationBuffer, input: Row): Unit = { + if (!input.isNullAt(0)) { + buffer(0) = buffer.getLong(0) + input.getLong(0) + buffer(1) = buffer.getLong(1) + 1 + } + } + // Merges two aggregation buffers and stores the updated buffer values back to `buffer1` + def merge(buffer1: MutableAggregationBuffer, buffer2: Row): Unit = { + buffer1(0) = buffer1.getLong(0) + buffer2.getLong(0) + buffer1(1) = buffer1.getLong(1) + buffer2.getLong(1) + } + // Calculates the final result + def evaluate(buffer: Row): Double = buffer.getLong(0).toDouble / buffer.getLong(1) + } + // $example off:untyped_custom_aggregation$ + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder() + .appName("Spark SQL user-defined DataFrames aggregation example") + .getOrCreate() + + // $example on:untyped_custom_aggregation$ + // Register the function to access it + spark.udf.register("myAverage", MyAverage) + + val df = spark.read.json("examples/src/main/resources/employees.json") + df.createOrReplaceTempView("employees") + df.show() + // +-------+------+ + // | name|salary| + // +-------+------+ + // |Michael| 3000| + // | Andy| 4500| + // | Justin| 3500| + // | Berta| 4000| + // +-------+------+ + + val result = spark.sql("SELECT myAverage(salary) as average_salary FROM employees") + result.show() + // +--------------+ + // |average_salary| + // +--------------+ + // | 3750.0| + // +--------------+ + // $example off:untyped_custom_aggregation$ + + spark.stop() + } + +} diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala deleted file mode 100644 index bf40bd1ef13d..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ /dev/null @@ -1,81 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.sql.hive - -import com.google.common.io.{ByteStreams, Files} - -import java.io.File - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql._ -import org.apache.spark.sql.hive.HiveContext - -object HiveFromSpark { - case class Record(key: Int, value: String) - - // Copy kv1.txt file from classpath to temporary directory - val kv1Stream = HiveFromSpark.getClass.getResourceAsStream("/kv1.txt") - val kv1File = File.createTempFile("kv1", "txt") - kv1File.deleteOnExit() - ByteStreams.copy(kv1Stream, Files.newOutputStreamSupplier(kv1File)) - - def main(args: Array[String]) { - val sparkConf = new SparkConf().setAppName("HiveFromSpark") - val sc = new SparkContext(sparkConf) - - // A hive context adds support for finding tables in the MetaStore and writing queries - // using HiveQL. Users who do not have an existing Hive deployment can still create a - // HiveContext. When not configured by the hive-site.xml, the context automatically - // creates metastore_db and warehouse in the current directory. - val hiveContext = new HiveContext(sc) - import hiveContext.implicits._ - import hiveContext.sql - - sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - sql(s"LOAD DATA LOCAL INPATH '${kv1File.getAbsolutePath}' INTO TABLE src") - - // Queries are expressed in HiveQL - println("Result of 'SELECT *': ") - sql("SELECT * FROM src").collect().foreach(println) - - // Aggregation queries are also supported. - val count = sql("SELECT COUNT(*) FROM src").collect().head.getLong(0) - println(s"COUNT(*): $count") - - // The results of SQL queries are themselves RDDs and support all normal RDD functions. The - // items in the RDD are of type Row, which allows you to access each column by ordinal. - val rddFromSql = sql("SELECT key, value FROM src WHERE key < 10 ORDER BY key") - - println("Result of RDD.map:") - val rddAsStrings = rddFromSql.map { - case Row(key: Int, value: String) => s"Key: $key, Value: $value" - } - - // 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") - - // Queries can then join RDD data with data stored in Hive. - println("Result of SELECT *:") - sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) - - sc.stop() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala new file mode 100644 index 000000000000..e5f75d53edc8 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.sql.hive + +// $example on:spark_hive$ +import java.io.File + +import org.apache.spark.sql.Row +import org.apache.spark.sql.SparkSession +// $example off:spark_hive$ + +object SparkHiveExample { + + // $example on:spark_hive$ + case class Record(key: Int, value: String) + // $example off:spark_hive$ + + def main(args: Array[String]) { + // When working with Hive, one must instantiate `SparkSession` with Hive support, including + // connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined + // functions. Users who do not have an existing Hive deployment can still enable Hive support. + // When not configured by the hive-site.xml, the context automatically creates `metastore_db` + // in the current directory and creates a directory configured by `spark.sql.warehouse.dir`, + // which defaults to the directory `spark-warehouse` in the current directory that the spark + // application is started. + + // $example on:spark_hive$ + // warehouseLocation points to the default location for managed databases and tables + val warehouseLocation = new File("spark-warehouse").getAbsolutePath + + val spark = SparkSession + .builder() + .appName("Spark Hive Example") + .config("spark.sql.warehouse.dir", warehouseLocation) + .enableHiveSupport() + .getOrCreate() + + import spark.implicits._ + import spark.sql + + sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING) USING hive") + sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") + + // Queries are expressed in HiveQL + sql("SELECT * FROM src").show() + // +---+-------+ + // |key| value| + // +---+-------+ + // |238|val_238| + // | 86| val_86| + // |311|val_311| + // ... + + // Aggregation queries are also supported. + sql("SELECT COUNT(*) FROM src").show() + // +--------+ + // |count(1)| + // +--------+ + // | 500 | + // +--------+ + + // The results of SQL queries are themselves DataFrames and support all normal functions. + val sqlDF = sql("SELECT key, value FROM src WHERE key < 10 ORDER BY key") + + // The items in DataFrames are of type Row, which allows you to access each column by ordinal. + val stringsDS = sqlDF.map { + case Row(key: Int, value: String) => s"Key: $key, Value: $value" + } + stringsDS.show() + // +--------------------+ + // | value| + // +--------------------+ + // |Key: 0, Value: val_0| + // |Key: 0, Value: val_0| + // |Key: 0, Value: val_0| + // ... + + // You can also use DataFrames to create temporary views within a SparkSession. + val recordsDF = spark.createDataFrame((1 to 100).map(i => Record(i, s"val_$i"))) + recordsDF.createOrReplaceTempView("records") + + // Queries can then join DataFrame data with data stored in Hive. + sql("SELECT * FROM records r JOIN src s ON r.key = s.key").show() + // +---+------+---+------+ + // |key| value|key| value| + // +---+------+---+------+ + // | 2| val_2| 2| val_2| + // | 4| val_4| 4| val_4| + // | 5| val_5| 5| val_5| + // ... + // $example off:spark_hive$ + + spark.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredKafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredKafkaWordCount.scala new file mode 100644 index 000000000000..c26f73e78881 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredKafkaWordCount.scala @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.sql.streaming + +import org.apache.spark.sql.SparkSession + +/** + * Consumes messages from one or more topics in Kafka and does wordcount. + * Usage: StructuredKafkaWordCount + * The Kafka "bootstrap.servers" configuration. A + * comma-separated list of host:port. + * There are three kinds of type, i.e. 'assign', 'subscribe', + * 'subscribePattern'. + * |- Specific TopicPartitions to consume. Json string + * | {"topicA":[0,1],"topicB":[2,4]}. + * |- The topic list to subscribe. A comma-separated list of + * | topics. + * |- The pattern used to subscribe to topic(s). + * | Java regex string. + * |- Only one of "assign, "subscribe" or "subscribePattern" options can be + * | specified for Kafka source. + * Different value format depends on the value of 'subscribe-type'. + * + * Example: + * `$ bin/run-example \ + * sql.streaming.StructuredKafkaWordCount host1:port1,host2:port2 \ + * subscribe topic1,topic2` + */ +object StructuredKafkaWordCount { + def main(args: Array[String]): Unit = { + if (args.length < 3) { + System.err.println("Usage: StructuredKafkaWordCount " + + " ") + System.exit(1) + } + + val Array(bootstrapServers, subscribeType, topics) = args + + val spark = SparkSession + .builder + .appName("StructuredKafkaWordCount") + .getOrCreate() + + import spark.implicits._ + + // Create DataSet representing the stream of input lines from kafka + val lines = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", bootstrapServers) + .option(subscribeType, topics) + .load() + .selectExpr("CAST(value AS STRING)") + .as[String] + + // Generate running word count + val wordCounts = lines.flatMap(_.split(" ")).groupBy("value").count() + + // Start running the query that prints the running counts to the console + val query = wordCounts.writeStream + .outputMode("complete") + .format("console") + .start() + + query.awaitTermination() + } + +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala new file mode 100644 index 000000000000..de477c5ce816 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.sql.streaming + +import org.apache.spark.sql.SparkSession + +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network. + * + * Usage: StructuredNetworkWordCount + * and describe the TCP server that Structured 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 sql.streaming.StructuredNetworkWordCount + * localhost 9999` + */ +object StructuredNetworkWordCount { + def main(args: Array[String]) { + if (args.length < 2) { + System.err.println("Usage: StructuredNetworkWordCount ") + System.exit(1) + } + + val host = args(0) + val port = args(1).toInt + + val spark = SparkSession + .builder + .appName("StructuredNetworkWordCount") + .getOrCreate() + + import spark.implicits._ + + // Create DataFrame representing the stream of input lines from connection to host:port + val lines = spark.readStream + .format("socket") + .option("host", host) + .option("port", port) + .load() + + // Split the lines into words + val words = lines.as[String].flatMap(_.split(" ")) + + // Generate running word count + val wordCounts = words.groupBy("value").count() + + // Start running the query that prints the running counts to the console + val query = wordCounts.writeStream + .outputMode("complete") + .format("console") + .start() + + query.awaitTermination() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala new file mode 100644 index 000000000000..b4dad21dd75b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.sql.streaming + +import java.sql.Timestamp + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.functions._ + +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network over a + * sliding window of configurable duration. Each line from the network is tagged + * with a timestamp that is used to determine the windows into which it falls. + * + * Usage: StructuredNetworkWordCountWindowed + * [] + * and describe the TCP server that Structured Streaming + * would connect to receive data. + * gives the size of window, specified as integer number of seconds + * gives the amount of time successive windows are offset from one another, + * given in the same units as above. should be less than or equal to + * . If the two are equal, successive windows have no overlap. If + * is not provided, it defaults to . + * + * 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 sql.streaming.StructuredNetworkWordCountWindowed + * localhost 9999 []` + * + * One recommended , pair is 10, 5 + */ +object StructuredNetworkWordCountWindowed { + + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: StructuredNetworkWordCountWindowed " + + " []") + System.exit(1) + } + + val host = args(0) + val port = args(1).toInt + val windowSize = args(2).toInt + val slideSize = if (args.length == 3) windowSize else args(3).toInt + if (slideSize > windowSize) { + System.err.println(" must be less than or equal to ") + } + val windowDuration = s"$windowSize seconds" + val slideDuration = s"$slideSize seconds" + + val spark = SparkSession + .builder + .appName("StructuredNetworkWordCountWindowed") + .getOrCreate() + + import spark.implicits._ + + // Create DataFrame representing the stream of input lines from connection to host:port + val lines = spark.readStream + .format("socket") + .option("host", host) + .option("port", port) + .option("includeTimestamp", true) + .load() + + // Split the lines into words, retaining timestamps + val words = lines.as[(String, Timestamp)].flatMap(line => + line._1.split(" ").map(word => (word, line._2)) + ).toDF("word", "timestamp") + + // Group the data by window and word and compute the count of each group + val windowedCounts = words.groupBy( + window($"timestamp", windowDuration, slideDuration), $"word" + ).count().orderBy("window") + + // Start running the query that prints the windowed word counts to the console + val query = windowedCounts.writeStream + .outputMode("complete") + .format("console") + .option("truncate", "false") + .start() + + query.awaitTermination() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala new file mode 100644 index 000000000000..ed63fb677b9e --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.sql.streaming + +import java.sql.Timestamp + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.streaming._ + + +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network. + * + * Usage: MapGroupsWithState + * and describe the TCP server that Structured 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 sql.streaming.StructuredSessionization + * localhost 9999` + */ +object StructuredSessionization { + + def main(args: Array[String]): Unit = { + if (args.length < 2) { + System.err.println("Usage: StructuredSessionization ") + System.exit(1) + } + + val host = args(0) + val port = args(1).toInt + + val spark = SparkSession + .builder + .appName("StructuredSessionization") + .getOrCreate() + + import spark.implicits._ + + // Create DataFrame representing the stream of input lines from connection to host:port + val lines = spark.readStream + .format("socket") + .option("host", host) + .option("port", port) + .option("includeTimestamp", true) + .load() + + // Split the lines into words, treat words as sessionId of events + val events = lines + .as[(String, Timestamp)] + .flatMap { case (line, timestamp) => + line.split(" ").map(word => Event(sessionId = word, timestamp)) + } + + // Sessionize the events. Track number of events, start and end timestamps of session, and + // and report session updates. + val sessionUpdates = events + .groupByKey(event => event.sessionId) + .mapGroupsWithState[SessionInfo, SessionUpdate](GroupStateTimeout.ProcessingTimeTimeout) { + + case (sessionId: String, events: Iterator[Event], state: GroupState[SessionInfo]) => + + // If timed out, then remove session and send final update + if (state.hasTimedOut) { + val finalUpdate = + SessionUpdate(sessionId, state.get.durationMs, state.get.numEvents, expired = true) + state.remove() + finalUpdate + } else { + // Update start and end timestamps in session + val timestamps = events.map(_.timestamp.getTime).toSeq + val updatedSession = if (state.exists) { + val oldSession = state.get + SessionInfo( + oldSession.numEvents + timestamps.size, + oldSession.startTimestampMs, + math.max(oldSession.endTimestampMs, timestamps.max)) + } else { + SessionInfo(timestamps.size, timestamps.min, timestamps.max) + } + state.update(updatedSession) + + // Set timeout such that the session will be expired if no data received for 10 seconds + state.setTimeoutDuration("10 seconds") + SessionUpdate(sessionId, state.get.durationMs, state.get.numEvents, expired = false) + } + } + + // Start running the query that prints the session updates to the console + val query = sessionUpdates + .writeStream + .outputMode("update") + .format("console") + .start() + + query.awaitTermination() + } +} +/** User-defined data type representing the input events */ +case class Event(sessionId: String, timestamp: Timestamp) + +/** + * User-defined data type for storing a session information as state in mapGroupsWithState. + * + * @param numEvents total number of events received in the session + * @param startTimestampMs timestamp of first event received in the session when it started + * @param endTimestampMs timestamp of last event received in the session before it expired + */ +case class SessionInfo( + numEvents: Int, + startTimestampMs: Long, + endTimestampMs: Long) { + + /** Duration of the session, between the first and last events */ + def durationMs: Long = endTimestampMs - startTimestampMs +} + +/** + * User-defined data type representing the update information returned by mapGroupsWithState. + * + * @param id Id of the session + * @param durationMs Duration the session was active, that is, from first event to its expiry + * @param numEvents Number of events received by the session while it was active + * @param expired Is the session active or expired + */ +case class SessionUpdate( + id: String, + durationMs: Long, + numEvents: Int, + expired: Boolean) + +// scalastyle:on println + diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala deleted file mode 100644 index e9c990719876..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala +++ /dev/null @@ -1,174 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import scala.collection.mutable.LinkedList -import scala.reflect.ClassTag -import scala.util.Random - -import akka.actor.{Actor, ActorRef, Props, actorRef2Scala} - -import org.apache.spark.{SparkConf, SecurityManager} -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions -import org.apache.spark.util.AkkaUtils -import org.apache.spark.streaming.receiver.ActorHelper - -case class SubscribeReceiver(receiverActor: ActorRef) -case class UnsubscribeReceiver(receiverActor: ActorRef) - -/** - * Sends the random content to every receiver subscribed with 1/2 - * second delay. - */ -class FeederActor extends Actor { - - val rand = new Random() - var receivers: LinkedList[ActorRef] = new LinkedList[ActorRef]() - - val strings: Array[String] = Array("words ", "may ", "count ") - - def makeMessage(): String = { - val x = rand.nextInt(3) - strings(x) + strings(2 - x) - } - - /* - * A thread to generate random messages - */ - new Thread() { - override def run() { - while (true) { - Thread.sleep(500) - receivers.foreach(_ ! makeMessage) - } - } - }.start() - - def receive: Receive = { - - case SubscribeReceiver(receiverActor: ActorRef) => - println("received subscribe from %s".format(receiverActor.toString)) - receivers = LinkedList(receiverActor) ++ receivers - - case UnsubscribeReceiver(receiverActor: ActorRef) => - println("received unsubscribe from %s".format(receiverActor.toString)) - receivers = receivers.dropWhile(x => x eq receiverActor) - - } -} - -/** - * A sample actor as receiver, is also simplest. This receiver actor - * goes and subscribe to a typical publisher/feeder actor and receives - * data. - * - * @see [[org.apache.spark.examples.streaming.FeederActor]] - */ -class SampleActorReceiver[T: ClassTag](urlOfPublisher: String) -extends Actor with ActorHelper { - - lazy private val remotePublisher = context.actorSelection(urlOfPublisher) - - override def preStart(): Unit = remotePublisher ! SubscribeReceiver(context.self) - - def receive: PartialFunction[Any, Unit] = { - case msg => store(msg.asInstanceOf[T]) - } - - override def postStop(): Unit = remotePublisher ! UnsubscribeReceiver(context.self) - -} - -/** - * A sample feeder actor - * - * Usage: FeederActor - * and describe the AkkaSystem that Spark Sample feeder would start on. - */ -object FeederActor { - - def main(args: Array[String]) { - if (args.length < 2){ - System.err.println("Usage: FeederActor \n") - System.exit(1) - } - val Seq(host, port) = args.toSeq - - val conf = new SparkConf - val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt, conf = conf, - securityManager = new SecurityManager(conf))._1 - val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor") - - println("Feeder started as:" + feeder) - - actorSystem.awaitTermination() - } -} - -/** - * A sample word count program demonstrating the use of plugging in - * Actor as Receiver - * Usage: ActorWordCount - * and describe the AkkaSystem that Spark Sample feeder is running on. - * - * To run this example locally, you may run Feeder Actor as - * `$ bin/run-example org.apache.spark.examples.streaming.FeederActor 127.0.1.1 9999` - * and then run the example - * `$ bin/run-example org.apache.spark.examples.streaming.ActorWordCount 127.0.1.1 9999` - */ -object ActorWordCount { - def main(args: Array[String]) { - if (args.length < 2) { - System.err.println( - "Usage: ActorWordCount ") - System.exit(1) - } - - StreamingExamples.setStreamingLogLevels() - - val Seq(host, port) = args.toSeq - val sparkConf = new SparkConf().setAppName("ActorWordCount") - // Create the context and set the batch size - val ssc = new StreamingContext(sparkConf, Seconds(2)) - - /* - * Following is the use of actorStream to plug in custom actor as receiver - * - * An important point to note: - * Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e type of data received and InputDstream - * should be same. - * - * For example: Both actorStream and SampleActorReceiver are parameterized - * to same type to ensure type safety. - */ - - val lines = ssc.actorStream[String]( - Props(new SampleActorReceiver[String]("akka.tcp://test@%s:%s/user/FeederActor".format( - host, port.toInt))), "SampleReceiver") - - // compute wordcount - lines.flatMap(_.split("\\s+")).map(x => (x, 1)).reduceByKey(_ + _).print() - - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala index 28e9bf520e56..43044d01b120 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala @@ -18,16 +18,18 @@ // scalastyle:off println package org.apache.spark.examples.streaming -import java.io.{InputStreamReader, BufferedReader, InputStream} +import java.io.{BufferedReader, InputStreamReader} import java.net.Socket +import java.nio.charset.StandardCharsets -import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.receiver.Receiver /** - * Custom Receiver that receives data over a socket. Received bytes is interpreted as + * Custom Receiver that receives data over a socket. Received bytes are interpreted as * text and \n delimited lines are considered as records. They are then counted and printed. * * To run this on your local machine, you need to first run a Netcat server @@ -48,7 +50,7 @@ object CustomReceiver { val sparkConf = new SparkConf().setAppName("CustomReceiver") val ssc = new StreamingContext(sparkConf, Seconds(1)) - // Create a input stream with the custom receiver on target ip:port and count the + // Create an input stream with the custom receiver on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') val lines = ssc.receiverStream(new CustomReceiver(args(0), args(1).toInt)) val words = lines.flatMap(_.split(" ")) @@ -83,7 +85,8 @@ class CustomReceiver(host: String, port: Int) logInfo("Connecting to " + host + ":" + port) socket = new Socket(host, port) logInfo("Connected to " + host + ":" + port) - val reader = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8")) + val reader = new BufferedReader( + new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8)) userInput = reader.readLine() while(!isStopped && userInput != null) { store(userInput) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala index bd78526f8c29..def06026bde9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala @@ -18,11 +18,9 @@ // scalastyle:off println package org.apache.spark.examples.streaming -import kafka.serializer.StringDecoder - -import org.apache.spark.streaming._ -import org.apache.spark.streaming.kafka._ import org.apache.spark.SparkConf +import org.apache.spark.streaming._ +import org.apache.spark.streaming.kafka010._ /** * Consumes messages from one or more topics in Kafka and does wordcount. @@ -57,11 +55,13 @@ object DirectKafkaWordCount { // Create direct kafka stream with brokers and topics val topicsSet = topics.split(",").toSet val kafkaParams = Map[String, String]("metadata.broker.list" -> brokers) - val messages = KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, topicsSet) + val messages = KafkaUtils.createDirectStream[String, String]( + ssc, + LocationStrategies.PreferConsistent, + ConsumerStrategies.Subscribe[String, String](topicsSet, kafkaParams)) // Get the lines, split them into words, count the words and print - val lines = messages.map(_._2) + val lines = messages.map(_.value) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1L)).reduceByKey(_ + _) wordCounts.print() diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala index 2bdbc37e2a28..dd725d72c23e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala @@ -19,11 +19,9 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf -import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.flume._ import org.apache.spark.util.IntParam -import java.net.InetSocketAddress /** * Produces a count of events received from Flume. diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala deleted file mode 100644 index b40d17e9c2fa..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ /dev/null @@ -1,105 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import java.util.HashMap - -import org.apache.kafka.clients.producer.{ProducerConfig, KafkaProducer, ProducerRecord} - -import org.apache.spark.streaming._ -import org.apache.spark.streaming.kafka._ -import org.apache.spark.SparkConf - -/** - * Consumes messages from one or more topics in Kafka and does wordcount. - * Usage: KafkaWordCount - * is a list of one or more zookeeper servers that make quorum - * is the name of kafka consumer group - * is a list of one or more kafka topics to consume from - * is the number of threads the kafka consumer should use - * - * Example: - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.KafkaWordCount zoo01,zoo02,zoo03 \ - * my-consumer-group topic1,topic2 1` - */ -object KafkaWordCount { - def main(args: Array[String]) { - if (args.length < 4) { - System.err.println("Usage: KafkaWordCount ") - System.exit(1) - } - - StreamingExamples.setStreamingLogLevels() - - val Array(zkQuorum, group, topics, numThreads) = args - val sparkConf = new SparkConf().setAppName("KafkaWordCount") - val ssc = new StreamingContext(sparkConf, Seconds(2)) - ssc.checkpoint("checkpoint") - - val topicMap = topics.split(",").map((_, numThreads.toInt)).toMap - val lines = KafkaUtils.createStream(ssc, zkQuorum, group, topicMap).map(_._2) - val words = lines.flatMap(_.split(" ")) - val wordCounts = words.map(x => (x, 1L)) - .reduceByKeyAndWindow(_ + _, _ - _, Minutes(10), Seconds(2), 2) - wordCounts.print() - - ssc.start() - ssc.awaitTermination() - } -} - -// Produces some random words between 1 and 100. -object KafkaWordCountProducer { - - def main(args: Array[String]) { - if (args.length < 4) { - System.err.println("Usage: KafkaWordCountProducer " + - " ") - System.exit(1) - } - - val Array(brokers, topic, messagesPerSec, wordsPerMessage) = args - - // Zookeeper connection properties - val props = new HashMap[String, Object]() - props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers) - props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, - "org.apache.kafka.common.serialization.StringSerializer") - props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, - "org.apache.kafka.common.serialization.StringSerializer") - - val producer = new KafkaProducer[String, String](props) - - // Send some messages - while(true) { - (1 to messagesPerSec.toInt).foreach { messageNum => - val str = (1 to wordsPerMessage.toInt).map(x => scala.util.Random.nextInt(10).toString) - .mkString(" ") - - val message = new ProducerRecord[String, String](topic, null, str) - producer.send(message) - } - - Thread.sleep(1000) - } - } - -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala deleted file mode 100644 index d772ae309f40..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala +++ /dev/null @@ -1,119 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import org.eclipse.paho.client.mqttv3._ -import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.mqtt._ -import org.apache.spark.SparkConf - -/** - * A simple Mqtt publisher for demonstration purposes, repeatedly publishes - * Space separated String Message "hello mqtt demo for spark streaming" - */ -object MQTTPublisher { - - def main(args: Array[String]) { - if (args.length < 2) { - System.err.println("Usage: MQTTPublisher ") - System.exit(1) - } - - StreamingExamples.setStreamingLogLevels() - - val Seq(brokerUrl, topic) = args.toSeq - - var client: MqttClient = null - - try { - val persistence = new MemoryPersistence() - client = new MqttClient(brokerUrl, MqttClient.generateClientId(), persistence) - - client.connect() - - val msgtopic = client.getTopic(topic) - val msgContent = "hello mqtt demo for spark streaming" - val message = new MqttMessage(msgContent.getBytes("utf-8")) - - while (true) { - try { - msgtopic.publish(message) - println(s"Published data. topic: ${msgtopic.getName()}; Message: $message") - } catch { - case e: MqttException if e.getReasonCode == MqttException.REASON_CODE_MAX_INFLIGHT => - Thread.sleep(10) - println("Queue is full, wait for to consume data from the message queue") - } - } - } catch { - case e: MqttException => println("Exception Caught: " + e) - } finally { - if (client != null) { - client.disconnect() - } - } - } -} - -/** - * A sample wordcount with MqttStream stream - * - * To work with Mqtt, Mqtt Message broker/server required. - * Mosquitto (http://mosquitto.org/) is an open source Mqtt Broker - * In ubuntu mosquitto can be installed using the command `$ sudo apt-get install mosquitto` - * Eclipse paho project provides Java library for Mqtt Client http://www.eclipse.org/paho/ - * Example Java code for Mqtt Publisher and Subscriber can be found here - * https://bitbucket.org/mkjinesh/mqttclient - * Usage: MQTTWordCount - * and describe where Mqtt publisher is running. - * - * To run this example locally, you may run publisher as - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.MQTTPublisher tcp://localhost:1883 foo` - * and run the example as - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.MQTTWordCount tcp://localhost:1883 foo` - */ -object MQTTWordCount { - - def main(args: Array[String]) { - if (args.length < 2) { - // scalastyle:off println - System.err.println( - "Usage: MQTTWordCount ") - // scalastyle:on println - System.exit(1) - } - - val Seq(brokerUrl, topic) = args.toSeq - val sparkConf = new SparkConf().setAppName("MQTTWordCount") - val ssc = new StreamingContext(sparkConf, Seconds(2)) - val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_ONLY_SER_2) - val words = lines.flatMap(x => x.split(" ")) - val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) - - wordCounts.print() - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala index 9a57fe286d1a..15b57fccb407 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala @@ -19,8 +19,8 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf -import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Seconds, StreamingContext} /** * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala index 13ba9a43ec3c..19bacd449787 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.examples.streaming -import scala.collection.mutable.SynchronizedQueue +import scala.collection.mutable.Queue import org.apache.spark.SparkConf import org.apache.spark.rdd.RDD @@ -34,7 +34,7 @@ object QueueStream { // Create the queue through which RDDs can be pushed to // a QueueInputDStream - val rddQueue = new SynchronizedQueue[RDD[Int]]() + val rddQueue = new Queue[RDD[Int]]() // Create the QueueInputDStream and use it do some processing val inputStream = ssc.queueStream(rddQueue) @@ -43,9 +43,11 @@ object QueueStream { reducedStream.print() ssc.start() - // Create and push some RDDs into + // Create and push some RDDs into rddQueue for (i <- 1 to 30) { - rddQueue += ssc.sparkContext.makeRDD(1 to 1000, 10) + rddQueue.synchronized { + rddQueue += ssc.sparkContext.makeRDD(1 to 1000, 10) + } Thread.sleep(1000) } ssc.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index 9916882e4f94..49c042732113 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -23,13 +23,55 @@ import java.nio.charset.Charset import com.google.common.io.Files -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.{Time, Seconds, StreamingContext} -import org.apache.spark.util.IntParam +import org.apache.spark.streaming.{Seconds, StreamingContext, Time} +import org.apache.spark.util.{IntParam, LongAccumulator} /** - * Counts words in text encoded with UTF8 received from the network every second. + * Use this singleton to get or register a Broadcast variable. + */ +object WordBlacklist { + + @volatile private var instance: Broadcast[Seq[String]] = null + + def getInstance(sc: SparkContext): Broadcast[Seq[String]] = { + if (instance == null) { + synchronized { + if (instance == null) { + val wordBlacklist = Seq("a", "b", "c") + instance = sc.broadcast(wordBlacklist) + } + } + } + instance + } +} + +/** + * Use this singleton to get or register an Accumulator. + */ +object DroppedWordsCounter { + + @volatile private var instance: LongAccumulator = null + + def getInstance(sc: SparkContext): LongAccumulator = { + if (instance == null) { + synchronized { + if (instance == null) { + instance = sc.longAccumulator("WordsInBlacklistCounter") + } + } + } + instance + } +} + +/** + * Counts words in text encoded with UTF8 received from the network every second. This example also + * shows how to use lazily instantiated singleton instances for Accumulator and Broadcast so that + * they can be registered on driver failures. * * Usage: RecoverableNetworkWordCount * and describe the TCP server that Spark Streaming would connect to receive @@ -73,19 +115,33 @@ object RecoverableNetworkWordCount { // words in input stream of \n delimited text (eg. generated by 'nc') val lines = ssc.socketTextStream(ip, port) val words = lines.flatMap(_.split(" ")) - val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) - wordCounts.foreachRDD((rdd: RDD[(String, Int)], time: Time) => { - val counts = "Counts at time " + time + " " + rdd.collect().mkString("[", ", ", "]") - println(counts) + val wordCounts = words.map((_, 1)).reduceByKey(_ + _) + wordCounts.foreachRDD { (rdd: RDD[(String, Int)], time: Time) => + // Get or register the blacklist Broadcast + val blacklist = WordBlacklist.getInstance(rdd.sparkContext) + // Get or register the droppedWordsCounter Accumulator + val droppedWordsCounter = DroppedWordsCounter.getInstance(rdd.sparkContext) + // Use blacklist to drop words and use droppedWordsCounter to count them + val counts = rdd.filter { case (word, count) => + if (blacklist.value.contains(word)) { + droppedWordsCounter.add(count) + false + } else { + true + } + }.collect().mkString("[", ", ", "]") + val output = "Counts at time " + time + " " + counts + println(output) + println("Dropped " + droppedWordsCounter.value + " word(s) totally") println("Appending to " + outputFile.getAbsolutePath) - Files.append(counts + "\n", outputFile, Charset.defaultCharset()) - }) + Files.append(output + "\n", outputFile, Charset.defaultCharset()) + } ssc } def main(args: Array[String]) { if (args.length != 4) { - System.err.println("You arguments were " + args.mkString("[", ", ", "]")) + System.err.println("Your arguments were " + args.mkString("[", ", ", "]")) System.err.println( """ |Usage: RecoverableNetworkWordCount @@ -102,9 +158,7 @@ object RecoverableNetworkWordCount { } val Array(ip, IntParam(port), checkpointDirectory, outputPath) = args val ssc = StreamingContext.getOrCreate(checkpointDirectory, - () => { - createContext(ip, port, outputPath, checkpointDirectory) - }) + () => createContext(ip, port, outputPath, checkpointDirectory)) ssc.start() ssc.awaitTermination() } diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala index ed617754cbf1..787bbec73b28 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala @@ -19,12 +19,10 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf -import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.{Time, Seconds, StreamingContext} -import org.apache.spark.util.IntParam -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Seconds, StreamingContext, Time} /** * Use DataFrames and SQL to count words in UTF8 encoded, '\n' delimited text received from the @@ -60,23 +58,23 @@ object SqlNetworkWordCount { val words = lines.flatMap(_.split(" ")) // Convert RDDs of the words DStream to DataFrame and run SQL query - words.foreachRDD((rdd: RDD[String], time: Time) => { - // Get the singleton instance of SQLContext - val sqlContext = SQLContextSingleton.getInstance(rdd.sparkContext) - import sqlContext.implicits._ + words.foreachRDD { (rdd: RDD[String], time: Time) => + // Get the singleton instance of SparkSession + val spark = SparkSessionSingleton.getInstance(rdd.sparkContext.getConf) + import spark.implicits._ // Convert RDD[String] to RDD[case class] to DataFrame val wordsDataFrame = rdd.map(w => Record(w)).toDF() - // Register as table - wordsDataFrame.registerTempTable("words") + // Creates a temporary view using the DataFrame + wordsDataFrame.createOrReplaceTempView("words") // Do word count on table using SQL and print it val wordCountsDataFrame = - sqlContext.sql("select word, count(*) as total from words group by word") + spark.sql("select word, count(*) as total from words group by word") println(s"========= $time =========") wordCountsDataFrame.show() - }) + } ssc.start() ssc.awaitTermination() @@ -88,14 +86,17 @@ object SqlNetworkWordCount { case class Record(word: String) -/** Lazily instantiated singleton instance of SQLContext */ -object SQLContextSingleton { +/** Lazily instantiated singleton instance of SparkSession */ +object SparkSessionSingleton { - @transient private var instance: SQLContext = _ + @transient private var instance: SparkSession = _ - def getInstance(sparkContext: SparkContext): SQLContext = { + def getInstance(sparkConf: SparkConf): SparkSession = { if (instance == null) { - instance = new SQLContext(sparkContext) + instance = SparkSession + .builder + .config(sparkConf) + .getOrCreate() } instance } diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala index 02ba1c2eed0f..2811e67009fb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala @@ -19,7 +19,6 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf -import org.apache.spark.HashPartitioner import org.apache.spark.streaming._ /** @@ -44,24 +43,12 @@ object StatefulNetworkWordCount { StreamingExamples.setStreamingLogLevels() - val updateFunc = (values: Seq[Int], state: Option[Int]) => { - val currentCount = values.sum - - val previousCount = state.getOrElse(0) - - Some(currentCount + previousCount) - } - - val newUpdateFunc = (iterator: Iterator[(String, Seq[Int], Option[Int])]) => { - iterator.flatMap(t => updateFunc(t._2, t._3).map(s => (t._1, s))) - } - val sparkConf = new SparkConf().setAppName("StatefulNetworkWordCount") // Create the context with a 1 second batch size val ssc = new StreamingContext(sparkConf, Seconds(1)) ssc.checkpoint(".") - // Initial RDD input to updateStateByKey + // Initial state RDD for mapWithState operation val initialRDD = ssc.sparkContext.parallelize(List(("hello", 1), ("world", 1))) // Create a ReceiverInputDStream on target ip:port and count the @@ -70,10 +57,17 @@ object StatefulNetworkWordCount { val words = lines.flatMap(_.split(" ")) val wordDstream = words.map(x => (x, 1)) - // Update the cumulative count using updateStateByKey - // This will give a Dstream made of state (which is the cumulative count of the words) - val stateDstream = wordDstream.updateStateByKey[Int](newUpdateFunc, - new HashPartitioner (ssc.sparkContext.defaultParallelism), true, initialRDD) + // Update the cumulative count using mapWithState + // This will give a DStream made of state (which is the cumulative count of the words) + val mappingFunc = (word: String, one: Option[Int], state: State[Int]) => { + val sum = one.getOrElse(0) + state.getOption.getOrElse(0) + val output = (word, sum) + state.update(sum) + output + } + + val stateDstream = wordDstream.mapWithState( + StateSpec.function(mappingFunc).initialState(initialRDD)) stateDstream.print() ssc.start() ssc.awaitTermination() diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala index 8396e65d0d58..b00f32fb2524 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala @@ -17,10 +17,10 @@ package org.apache.spark.examples.streaming -import org.apache.spark.Logging - import org.apache.log4j.{Level, Logger} +import org.apache.spark.internal.Logging + /** Utility functions for Spark Streaming examples. */ object StreamingExamples extends Logging { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala deleted file mode 100644 index 825c671a929b..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala +++ /dev/null @@ -1,117 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import com.twitter.algebird._ -import com.twitter.algebird.CMSHasherImplicits._ - -import org.apache.spark.SparkConf -import org.apache.spark.SparkContext._ -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.twitter._ - -// scalastyle:off -/** - * Illustrates the use of the Count-Min Sketch, from Twitter's Algebird library, to compute - * windowed and global Top-K estimates of user IDs occurring in a Twitter stream. - *
    - * Note that since Algebird's implementation currently only supports Long inputs, - * the example operates on Long IDs. Once the implementation supports other inputs (such as String), - * the same approach could be used for computing popular topics for example. - *

    - *

    - * - * This blog post has a good overview of the Count-Min Sketch (CMS). The CMS is a data - * structure for approximate frequency estimation in data streams (e.g. Top-K elements, frequency - * of any given element, etc), that uses space sub-linear in the number of elements in the - * stream. Once elements are added to the CMS, the estimated count of an element can be computed, - * as well as "heavy-hitters" that occur more than a threshold percentage of the overall total - * count. - *

    - * Algebird's implementation is a monoid, so we can succinctly merge two CMS instances in the - * reduce operation. - */ -// scalastyle:on -object TwitterAlgebirdCMS { - def main(args: Array[String]) { - StreamingExamples.setStreamingLogLevels() - - // CMS parameters - val DELTA = 1E-3 - val EPS = 0.01 - val SEED = 1 - val PERC = 0.001 - // K highest frequency elements to take - val TOPK = 10 - - val filters = args - val sparkConf = new SparkConf().setAppName("TwitterAlgebirdCMS") - val ssc = new StreamingContext(sparkConf, Seconds(10)) - val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER_2) - - val users = stream.map(status => status.getUser.getId) - - // val cms = new CountMinSketchMonoid(EPS, DELTA, SEED, PERC) - val cms = TopPctCMS.monoid[Long](EPS, DELTA, SEED, PERC) - var globalCMS = cms.zero - val mm = new MapMonoid[Long, Int]() - var globalExact = Map[Long, Int]() - - val approxTopUsers = users.mapPartitions(ids => { - ids.map(id => cms.create(id)) - }).reduce(_ ++ _) - - val exactTopUsers = users.map(id => (id, 1)) - .reduceByKey((a, b) => a + b) - - approxTopUsers.foreachRDD(rdd => { - if (rdd.count() != 0) { - val partial = rdd.first() - val partialTopK = partial.heavyHitters.map(id => - (id, partial.frequency(id).estimate)).toSeq.sortBy(_._2).reverse.slice(0, TOPK) - globalCMS ++= partial - val globalTopK = globalCMS.heavyHitters.map(id => - (id, globalCMS.frequency(id).estimate)).toSeq.sortBy(_._2).reverse.slice(0, TOPK) - println("Approx heavy hitters at %2.2f%% threshold this batch: %s".format(PERC, - partialTopK.mkString("[", ",", "]"))) - println("Approx heavy hitters at %2.2f%% threshold overall: %s".format(PERC, - globalTopK.mkString("[", ",", "]"))) - } - }) - - exactTopUsers.foreachRDD(rdd => { - if (rdd.count() != 0) { - val partialMap = rdd.collect().toMap - val partialTopK = rdd.map( - {case (id, count) => (count, id)}) - .sortByKey(ascending = false).take(TOPK) - globalExact = mm.plus(globalExact.toMap, partialMap) - val globalTopK = globalExact.toSeq.sortBy(_._2).reverse.slice(0, TOPK) - println("Exact heavy hitters this batch: %s".format(partialTopK.mkString("[", ",", "]"))) - println("Exact heavy hitters overall: %s".format(globalTopK.mkString("[", ",", "]"))) - } - }) - - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala deleted file mode 100644 index 49826ede7041..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala +++ /dev/null @@ -1,94 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import com.twitter.algebird.HyperLogLogMonoid -import com.twitter.algebird.HyperLogLog._ - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.twitter._ -import org.apache.spark.SparkConf - -// scalastyle:off -/** - * Illustrates the use of the HyperLogLog algorithm, from Twitter's Algebird library, to compute - * a windowed and global estimate of the unique user IDs occurring in a Twitter stream. - *

    - *

    - * This - * blog post and this - * - * blog post - * have good overviews of HyperLogLog (HLL). HLL is a memory-efficient datastructure for - * estimating the cardinality of a data stream, i.e. the number of unique elements. - *

    - * Algebird's implementation is a monoid, so we can succinctly merge two HLL instances in the - * reduce operation. - */ -// scalastyle:on -object TwitterAlgebirdHLL { - def main(args: Array[String]) { - - StreamingExamples.setStreamingLogLevels() - - /** Bit size parameter for HyperLogLog, trades off accuracy vs size */ - val BIT_SIZE = 12 - val filters = args - val sparkConf = new SparkConf().setAppName("TwitterAlgebirdHLL") - val ssc = new StreamingContext(sparkConf, Seconds(5)) - val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER) - - val users = stream.map(status => status.getUser.getId) - - val hll = new HyperLogLogMonoid(BIT_SIZE) - var globalHll = hll.zero - var userSet: Set[Long] = Set() - - val approxUsers = users.mapPartitions(ids => { - ids.map(id => hll(id)) - }).reduce(_ + _) - - val exactUsers = users.map(id => Set(id)).reduce(_ ++ _) - - approxUsers.foreachRDD(rdd => { - if (rdd.count() != 0) { - val partial = rdd.first() - globalHll += partial - println("Approx distinct users this batch: %d".format(partial.estimatedSize.toInt)) - println("Approx distinct users overall: %d".format(globalHll.estimatedSize.toInt)) - } - }) - - exactUsers.foreachRDD(rdd => { - if (rdd.count() != 0) { - val partial = rdd.first() - userSet ++= partial - println("Exact distinct users this batch: %d".format(partial.size)) - println("Exact distinct users overall: %d".format(userSet.size)) - println("Error rate: %2.5f%%".format(((globalHll.estimatedSize / userSet.size.toDouble) - 1 - ) * 100)) - } - }) - - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala deleted file mode 100644 index 49cee1b43c2d..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala +++ /dev/null @@ -1,86 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.SparkContext._ -import org.apache.spark.streaming.twitter._ -import org.apache.spark.SparkConf - -/** - * Calculates popular hashtags (topics) over sliding 10 and 60 second windows from a Twitter - * stream. The stream is instantiated with credentials and optionally filters supplied by the - * command line arguments. - * - * Run this on your local machine as - * - */ -object TwitterPopularTags { - def main(args: Array[String]) { - if (args.length < 4) { - System.err.println("Usage: TwitterPopularTags " + - " []") - System.exit(1) - } - - StreamingExamples.setStreamingLogLevels() - - val Array(consumerKey, consumerSecret, accessToken, accessTokenSecret) = args.take(4) - val filters = args.takeRight(args.length - 4) - - // Set the system properties so that Twitter4j library used by twitter stream - // can use them to generat OAuth credentials - System.setProperty("twitter4j.oauth.consumerKey", consumerKey) - System.setProperty("twitter4j.oauth.consumerSecret", consumerSecret) - System.setProperty("twitter4j.oauth.accessToken", accessToken) - System.setProperty("twitter4j.oauth.accessTokenSecret", accessTokenSecret) - - val sparkConf = new SparkConf().setAppName("TwitterPopularTags") - val ssc = new StreamingContext(sparkConf, Seconds(2)) - val stream = TwitterUtils.createStream(ssc, None, filters) - - val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) - - val topCounts60 = hashTags.map((_, 1)).reduceByKeyAndWindow(_ + _, Seconds(60)) - .map{case (topic, count) => (count, topic)} - .transform(_.sortByKey(false)) - - val topCounts10 = hashTags.map((_, 1)).reduceByKeyAndWindow(_ + _, Seconds(10)) - .map{case (topic, count) => (count, topic)} - .transform(_.sortByKey(false)) - - - // Print popular hashtags - topCounts60.foreachRDD(rdd => { - val topList = rdd.take(10) - println("\nPopular topics in last 60 seconds (%s total):".format(rdd.count())) - topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))} - }) - - topCounts10.foreachRDD(rdd => { - val topList = rdd.take(10) - println("\nPopular topics in last 10 seconds (%s total):".format(rdd.count())) - topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))} - }) - - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala deleted file mode 100644 index 6ac9a72c3794..000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala +++ /dev/null @@ -1,101 +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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import akka.actor.ActorSystem -import akka.actor.actorRef2Scala -import akka.zeromq._ -import akka.zeromq.Subscribe -import akka.util.ByteString - -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.zeromq._ - -import scala.language.implicitConversions -import org.apache.spark.SparkConf - -/** - * A simple publisher for demonstration purposes, repeatedly publishes random Messages - * every one second. - */ -object SimpleZeroMQPublisher { - - def main(args: Array[String]): Unit = { - if (args.length < 2) { - System.err.println("Usage: SimpleZeroMQPublisher ") - System.exit(1) - } - - val Seq(url, topic) = args.toSeq - val acs: ActorSystem = ActorSystem() - - val pubSocket = ZeroMQExtension(acs).newSocket(SocketType.Pub, Bind(url)) - implicit def stringToByteString(x: String): ByteString = ByteString(x) - val messages: List[ByteString] = List("words ", "may ", "count ") - while (true) { - Thread.sleep(1000) - pubSocket ! ZMQMessage(ByteString(topic) :: messages) - } - acs.awaitTermination() - } -} - -// scalastyle:off -/** - * A sample wordcount with ZeroMQStream stream - * - * To work with zeroMQ, some native libraries have to be installed. - * Install zeroMQ (release 2.1) core libraries. [ZeroMQ Install guide] - * (http://www.zeromq.org/intro:get-the-software) - * - * Usage: ZeroMQWordCount - * and describe where zeroMq publisher is running. - * - * To run this example locally, you may run publisher as - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` - * and run the example as - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.ZeroMQWordCount tcp://127.0.1.1:1234 foo` - */ -// scalastyle:on -object ZeroMQWordCount { - def main(args: Array[String]) { - if (args.length < 2) { - System.err.println("Usage: ZeroMQWordCount ") - System.exit(1) - } - StreamingExamples.setStreamingLogLevels() - val Seq(url, topic) = args.toSeq - val sparkConf = new SparkConf().setAppName("ZeroMQWordCount") - // Create the context and set the batch size - val ssc = new StreamingContext(sparkConf, Seconds(2)) - - def bytesToStringIterator(x: Seq[ByteString]): Iterator[String] = x.map(_.utf8String).iterator - - // For this stream, a zeroMQ publisher should be running. - val lines = ZeroMQUtils.createStream(ssc, url, Subscribe(topic), bytesToStringIterator _) - val words = lines.flatMap(_.split(" ")) - val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) - wordCounts.print() - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala index bea7a47cb285..0ddd065f0db2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala @@ -18,49 +18,50 @@ // scalastyle:off println package org.apache.spark.examples.streaming.clickstream -import java.net.ServerSocket import java.io.PrintWriter -import util.Random +import java.net.ServerSocket +import java.util.Random /** Represents a page view on a website with associated dimension data. */ -class PageView(val url : String, val status : Int, val zipCode : Int, val userID : Int) +class PageView(val url: String, val status: Int, val zipCode: Int, val userID: Int) extends Serializable { - override def toString() : String = { + override def toString(): String = { "%s\t%s\t%s\t%s\n".format(url, status, zipCode, userID) } } object PageView extends Serializable { - def fromString(in : String) : PageView = { + def fromString(in: String): PageView = { val parts = in.split("\t") new PageView(parts(0), parts(1).toInt, parts(2).toInt, parts(3).toInt) } } // scalastyle:off -/** Generates streaming events to simulate page views on a website. - * - * This should be used in tandem with PageViewStream.scala. Example: - * - * To run the generator - * `$ bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10` - * To process the generated stream - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444` - * - */ +/** + * Generates streaming events to simulate page views on a website. + * + * This should be used in tandem with PageViewStream.scala. Example: + * + * To run the generator + * `$ bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10` + * To process the generated stream + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444` + * + */ // scalastyle:on object PageViewGenerator { - val pages = Map("http://foo.com/" -> .7, - "http://foo.com/news" -> 0.2, + val pages = Map("http://foo.com/" -> .7, + "http://foo.com/news" -> 0.2, "http://foo.com/contact" -> .1) val httpStatus = Map(200 -> .95, 404 -> .05) val userZipCode = Map(94709 -> .5, 94117 -> .5) - val userID = Map((1 to 100).map(_ -> .01) : _*) + val userID = Map((1 to 100).map(_ -> .01): _*) - def pickFromDistribution[T](inputMap : Map[T, Double]) : T = { + def pickFromDistribution[T](inputMap: Map[T, Double]): T = { val rand = new Random().nextDouble() var total = 0.0 for ((item, prob) <- inputMap) { @@ -72,7 +73,7 @@ object PageViewGenerator { inputMap.take(1).head._1 // Shouldn't get here if probabilities add up to 1.0 } - def getNextClickEvent() : String = { + def getNextClickEvent(): String = { val id = pickFromDistribution(userID) val page = pickFromDistribution(pages) val status = pickFromDistribution(httpStatus) @@ -80,7 +81,7 @@ object PageViewGenerator { new PageView(page, status, zipCode, id).toString() } - def main(args : Array[String]) { + def main(args: Array[String]) { if (args.length != 2) { System.err.println("Usage: PageViewGenerator ") System.exit(1) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala index ec7d39da8b2e..1ba093f57b32 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala @@ -18,20 +18,21 @@ // scalastyle:off println package org.apache.spark.examples.streaming.clickstream -import org.apache.spark.SparkContext._ -import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.examples.streaming.StreamingExamples +import org.apache.spark.streaming.{Seconds, StreamingContext} + // scalastyle:off -/** Analyses a streaming dataset of web page views. This class demonstrates several types of - * operators available in Spark streaming. - * - * This should be used in tandem with PageViewStream.scala. Example: - * To run the generator - * `$ bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10` - * To process the generated stream - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444` - */ +/** + * Analyses a streaming dataset of web page views. This class demonstrates several types of + * operators available in Spark streaming. + * + * This should be used in tandem with PageViewStream.scala. Example: + * To run the generator + * `$ bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10` + * To process the generated stream + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444` + */ // scalastyle:on object PageViewStream { def main(args: Array[String]) { @@ -69,7 +70,7 @@ object PageViewStream { .groupByKey() val errorRatePerZipCode = statusesPerZipCode.map{ case(zip, statuses) => - val normalCount = statuses.filter(_ == 200).size + val normalCount = statuses.count(_ == 200) val errorCount = statuses.size - normalCount val errorRatio = errorCount.toFloat / statuses.size if (errorRatio > 0.05) { @@ -87,8 +88,10 @@ object PageViewStream { .map("Unique active users: " + _) // An external dataset we want to join to this stream - val userList = ssc.sparkContext.parallelize( - Map(1 -> "Patrick Wendell", 2->"Reynold Xin", 3->"Matei Zaharia").toSeq) + val userList = ssc.sparkContext.parallelize(Seq( + 1 -> "Patrick Wendell", + 2 -> "Reynold Xin", + 3 -> "Matei Zaharia")) metric match { case "pageCounts" => pageCounts.print() @@ -106,6 +109,7 @@ object PageViewStream { } ssc.start() + ssc.awaitTermination() } } // scalastyle:on println diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml new file mode 100644 index 000000000000..485b562dce99 --- /dev/null +++ b/external/docker-integration-tests/pom.xml @@ -0,0 +1,154 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-docker-integration-tests_2.11 + jar + Spark Project Docker Integration Tests + http://spark.apache.org/ + + docker-integration-tests + + + + + db + https://app.camunda.com/nexus/content/repositories/public/ + + true + warn + + + + + + + com.spotify + docker-client + test + + + org.apache.httpcomponents + httpclient + test + + + org.apache.httpcomponents + httpcore + test + + + + com.google.guava + guava + 18.0 + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + test + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + mysql + mysql-connector-java + test + + + org.postgresql + postgresql + test + + + + com.oracle + ojdbc6 + 11.2.0.1.0 + test + + + + + com.ibm.db2.jcc + db2jcc4 + 10.5.0.5 + jar + + + diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala new file mode 100644 index 000000000000..f5930bc281e8 --- /dev/null +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.math.BigDecimal +import java.sql.{Connection, Date, Timestamp} +import java.util.Properties + +import org.scalatest.Ignore + +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.{BooleanType, ByteType, ShortType, StructType} +import org.apache.spark.tags.DockerTest + + +@DockerTest +@Ignore // AMPLab Jenkins needs to be updated before shared memory works on docker +class DB2IntegrationSuite extends DockerJDBCIntegrationSuite { + override val db = new DatabaseOnDocker { + override val imageName = "lresende/db2express-c:10.5.0.5-3.10.0" + override val env = Map( + "DB2INST1_PASSWORD" -> "rootpass", + "LICENSE" -> "accept" + ) + override val usesIpc = false + override val jdbcPort: Int = 50000 + override def getJdbcUrl(ip: String, port: Int): String = + s"jdbc:db2://$ip:$port/foo:user=db2inst1;password=rootpass;retrieveMessagesFromServerOnGetMessage=true;" //scalastyle:ignore + override def getStartupProcessName: Option[String] = Some("db2start") + } + + override def dataPreparation(conn: Connection): Unit = { + conn.prepareStatement("CREATE TABLE tbl (x INTEGER, y VARCHAR(8))").executeUpdate() + conn.prepareStatement("INSERT INTO tbl VALUES (42,'fred')").executeUpdate() + conn.prepareStatement("INSERT INTO tbl VALUES (17,'dave')").executeUpdate() + + conn.prepareStatement("CREATE TABLE numbers ( small SMALLINT, med INTEGER, big BIGINT, " + + "deci DECIMAL(31,20), flt FLOAT, dbl DOUBLE, real REAL, " + + "decflt DECFLOAT, decflt16 DECFLOAT(16), decflt34 DECFLOAT(34))").executeUpdate() + conn.prepareStatement("INSERT INTO numbers VALUES (17, 77777, 922337203685477580, " + + "123456745.56789012345000000000, 42.75, 5.4E-70, " + + "3.4028234663852886e+38, 4.2999, DECFLOAT('9.999999999999999E19', 16), " + + "DECFLOAT('1234567891234567.123456789123456789', 34))").executeUpdate() + + conn.prepareStatement("CREATE TABLE dates (d DATE, t TIME, ts TIMESTAMP )").executeUpdate() + conn.prepareStatement("INSERT INTO dates VALUES ('1991-11-09', '13:31:24', " + + "'2009-02-13 23:31:30')").executeUpdate() + + // TODO: Test locale conversion for strings. + conn.prepareStatement("CREATE TABLE strings (a CHAR(10), b VARCHAR(10), c CLOB, d BLOB, e XML)") + .executeUpdate() + conn.prepareStatement("INSERT INTO strings VALUES ('the', 'quick', 'brown', BLOB('fox')," + + "'Kathy')").executeUpdate() + } + + test("Basic test") { + val df = sqlContext.read.jdbc(jdbcUrl, "tbl", new Properties) + val rows = df.collect() + assert(rows.length == 2) + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types.length == 2) + assert(types(0).equals("class java.lang.Integer")) + assert(types(1).equals("class java.lang.String")) + } + + test("Numeric types") { + val df = sqlContext.read.jdbc(jdbcUrl, "numbers", new Properties) + val rows = df.collect() + assert(rows.length == 1) + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types.length == 10) + assert(types(0).equals("class java.lang.Integer")) + assert(types(1).equals("class java.lang.Integer")) + assert(types(2).equals("class java.lang.Long")) + assert(types(3).equals("class java.math.BigDecimal")) + assert(types(4).equals("class java.lang.Double")) + assert(types(5).equals("class java.lang.Double")) + assert(types(6).equals("class java.lang.Float")) + assert(types(7).equals("class java.math.BigDecimal")) + assert(types(8).equals("class java.math.BigDecimal")) + assert(types(9).equals("class java.math.BigDecimal")) + assert(rows(0).getInt(0) == 17) + assert(rows(0).getInt(1) == 77777) + assert(rows(0).getLong(2) == 922337203685477580L) + val bd = new BigDecimal("123456745.56789012345000000000") + assert(rows(0).getAs[BigDecimal](3).equals(bd)) + assert(rows(0).getDouble(4) == 42.75) + assert(rows(0).getDouble(5) == 5.4E-70) + assert(rows(0).getFloat(6) == 3.4028234663852886e+38) + assert(rows(0).getDecimal(7) == new BigDecimal("4.299900000000000000")) + assert(rows(0).getDecimal(8) == new BigDecimal("99999999999999990000.000000000000000000")) + assert(rows(0).getDecimal(9) == new BigDecimal("1234567891234567.123456789123456789")) + } + + test("Date types") { + val df = sqlContext.read.jdbc(jdbcUrl, "dates", new Properties) + val rows = df.collect() + assert(rows.length == 1) + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types.length == 3) + assert(types(0).equals("class java.sql.Date")) + assert(types(1).equals("class java.sql.Timestamp")) + assert(types(2).equals("class java.sql.Timestamp")) + assert(rows(0).getAs[Date](0).equals(Date.valueOf("1991-11-09"))) + assert(rows(0).getAs[Timestamp](1).equals(Timestamp.valueOf("1970-01-01 13:31:24"))) + assert(rows(0).getAs[Timestamp](2).equals(Timestamp.valueOf("2009-02-13 23:31:30"))) + } + + test("String types") { + val df = sqlContext.read.jdbc(jdbcUrl, "strings", new Properties) + val rows = df.collect() + assert(rows.length == 1) + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types.length == 5) + assert(types(0).equals("class java.lang.String")) + assert(types(1).equals("class java.lang.String")) + assert(types(2).equals("class java.lang.String")) + assert(types(3).equals("class [B")) + assert(rows(0).getString(0).equals("the ")) + assert(rows(0).getString(1).equals("quick")) + assert(rows(0).getString(2).equals("brown")) + assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](3), Array[Byte](102, 111, 120))) + assert(rows(0).getString(4).equals("""Kathy""")) + } + + test("Basic write test") { + // cast decflt column with precision value of 38 to DB2 max decimal precision value of 31. + val df1 = sqlContext.read.jdbc(jdbcUrl, "numbers", new Properties) + .selectExpr("small", "med", "big", "deci", "flt", "dbl", "real", + "cast(decflt as decimal(31, 5)) as decflt") + val df2 = sqlContext.read.jdbc(jdbcUrl, "dates", new Properties) + val df3 = sqlContext.read.jdbc(jdbcUrl, "strings", new Properties) + df1.write.jdbc(jdbcUrl, "numberscopy", new Properties) + df2.write.jdbc(jdbcUrl, "datescopy", new Properties) + df3.write.jdbc(jdbcUrl, "stringscopy", new Properties) + // spark types that does not have exact matching db2 table types. + val df4 = sqlContext.createDataFrame( + sparkContext.parallelize(Seq(Row("1".toShort, "20".toByte, true))), + new StructType().add("c1", ShortType).add("b", ByteType).add("c3", BooleanType)) + df4.write.jdbc(jdbcUrl, "otherscopy", new Properties) + val rows = sqlContext.read.jdbc(jdbcUrl, "otherscopy", new Properties).collect() + assert(rows(0).getInt(0) == 1) + assert(rows(0).getInt(1) == 20) + assert(rows(0).getString(2) == "1") + } +} diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala new file mode 100644 index 000000000000..609696bc8a2c --- /dev/null +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerJDBCIntegrationSuite.scala @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.net.ServerSocket +import java.sql.Connection + +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +import com.spotify.docker.client._ +import com.spotify.docker.client.exceptions.ImageNotFoundException +import com.spotify.docker.client.messages.{ContainerConfig, HostConfig, PortBinding} +import org.scalatest.BeforeAndAfterAll +import org.scalatest.concurrent.Eventually +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.util.DockerUtils + +abstract class DatabaseOnDocker { + /** + * The docker image to be pulled. + */ + val imageName: String + + /** + * Environment variables to set inside of the Docker container while launching it. + */ + val env: Map[String, String] + + /** + * Wheather or not to use ipc mode for shared memory when starting docker image + */ + val usesIpc: Boolean + + /** + * The container-internal JDBC port that the database listens on. + */ + val jdbcPort: Int + + /** + * Return a JDBC URL that connects to the database running at the given IP address and port. + */ + def getJdbcUrl(ip: String, port: Int): String + + /** + * Optional process to run when container starts + */ + def getStartupProcessName: Option[String] +} + +abstract class DockerJDBCIntegrationSuite + extends SparkFunSuite + with BeforeAndAfterAll + with Eventually + with SharedSQLContext { + + val db: DatabaseOnDocker + + private var docker: DockerClient = _ + private var containerId: String = _ + protected var jdbcUrl: String = _ + + override def beforeAll() { + super.beforeAll() + try { + docker = DefaultDockerClient.fromEnv.build() + // Check that Docker is actually up + try { + docker.ping() + } catch { + case NonFatal(e) => + log.error("Exception while connecting to Docker. Check whether Docker is running.") + throw e + } + // Ensure that the Docker image is installed: + try { + docker.inspectImage(db.imageName) + } catch { + case e: ImageNotFoundException => + log.warn(s"Docker image ${db.imageName} not found; pulling image from registry") + docker.pull(db.imageName) + } + // Configure networking (necessary for boot2docker / Docker Machine) + val externalPort: Int = { + val sock = new ServerSocket(0) + val port = sock.getLocalPort + sock.close() + port + } + val dockerIp = DockerUtils.getDockerIp() + val hostConfig: HostConfig = HostConfig.builder() + .networkMode("bridge") + .ipcMode(if (db.usesIpc) "host" else "") + .portBindings( + Map(s"${db.jdbcPort}/tcp" -> List(PortBinding.of(dockerIp, externalPort)).asJava).asJava) + .build() + // Create the database container: + val containerConfigBuilder = ContainerConfig.builder() + .image(db.imageName) + .networkDisabled(false) + .env(db.env.map { case (k, v) => s"$k=$v" }.toSeq.asJava) + .hostConfig(hostConfig) + .exposedPorts(s"${db.jdbcPort}/tcp") + if(db.getStartupProcessName.isDefined) { + containerConfigBuilder + .cmd(db.getStartupProcessName.get) + } + val config = containerConfigBuilder.build() + // Create the database container: + containerId = docker.createContainer(config).id + // Start the container and wait until the database can accept JDBC connections: + docker.startContainer(containerId) + jdbcUrl = db.getJdbcUrl(dockerIp, externalPort) + eventually(timeout(60.seconds), interval(1.seconds)) { + val conn = java.sql.DriverManager.getConnection(jdbcUrl) + conn.close() + } + // Run any setup queries: + val conn: Connection = java.sql.DriverManager.getConnection(jdbcUrl) + try { + dataPreparation(conn) + } finally { + conn.close() + } + } catch { + case NonFatal(e) => + try { + afterAll() + } finally { + throw e + } + } + } + + override def afterAll() { + try { + if (docker != null) { + try { + if (containerId != null) { + docker.killContainer(containerId) + docker.removeContainer(containerId) + } + } catch { + case NonFatal(e) => + logWarning(s"Could not stop container $containerId", e) + } finally { + docker.close() + } + } + } finally { + super.afterAll() + } + } + + /** + * Prepare databases and tables for testing. + */ + def dataPreparation(connection: Connection): Unit +} diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala new file mode 100644 index 000000000000..a70ed98b52d5 --- /dev/null +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.math.BigDecimal +import java.sql.{Connection, Date, Timestamp} +import java.util.Properties + +import org.apache.spark.tags.DockerTest + +@DockerTest +class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { + override val db = new DatabaseOnDocker { + override val imageName = "mysql:5.7.9" + override val env = Map( + "MYSQL_ROOT_PASSWORD" -> "rootpass" + ) + override val usesIpc = false + override val jdbcPort: Int = 3306 + override def getJdbcUrl(ip: String, port: Int): String = + s"jdbc:mysql://$ip:$port/mysql?user=root&password=rootpass" + override def getStartupProcessName: Option[String] = None + } + + override def dataPreparation(conn: Connection): Unit = { + conn.prepareStatement("CREATE DATABASE foo").executeUpdate() + conn.prepareStatement("CREATE TABLE tbl (x INTEGER, y TEXT(8))").executeUpdate() + conn.prepareStatement("INSERT INTO tbl VALUES (42,'fred')").executeUpdate() + conn.prepareStatement("INSERT INTO tbl VALUES (17,'dave')").executeUpdate() + + conn.prepareStatement("CREATE TABLE numbers (onebit BIT(1), tenbits BIT(10), " + + "small SMALLINT, med MEDIUMINT, nor INT, big BIGINT, deci DECIMAL(40,20), flt FLOAT, " + + "dbl DOUBLE)").executeUpdate() + conn.prepareStatement("INSERT INTO numbers VALUES (b'0', b'1000100101', " + + "17, 77777, 123456789, 123456789012345, 123456789012345.123456789012345, " + + "42.75, 1.0000000000000002)").executeUpdate() + + conn.prepareStatement("CREATE TABLE dates (d DATE, t TIME, dt DATETIME, ts TIMESTAMP, " + + "yr YEAR)").executeUpdate() + conn.prepareStatement("INSERT INTO dates VALUES ('1991-11-09', '13:31:24', " + + "'1996-01-01 01:23:45', '2009-02-13 23:31:30', '2001')").executeUpdate() + + // TODO: Test locale conversion for strings. + conn.prepareStatement("CREATE TABLE strings (a CHAR(10), b VARCHAR(10), c TINYTEXT, " + + "d TEXT, e MEDIUMTEXT, f LONGTEXT, g BINARY(4), h VARBINARY(10), i BLOB)" + ).executeUpdate() + conn.prepareStatement("INSERT INTO strings VALUES ('the', 'quick', 'brown', 'fox', " + + "'jumps', 'over', 'the', 'lazy', 'dog')").executeUpdate() + } + + test("Basic test") { + val df = sqlContext.read.jdbc(jdbcUrl, "tbl", new Properties) + val rows = df.collect() + assert(rows.length == 2) + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types.length == 2) + assert(types(0).equals("class java.lang.Integer")) + assert(types(1).equals("class java.lang.String")) + } + + test("Numeric types") { + val df = sqlContext.read.jdbc(jdbcUrl, "numbers", new Properties) + val rows = df.collect() + assert(rows.length == 1) + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types.length == 9) + assert(types(0).equals("class java.lang.Boolean")) + assert(types(1).equals("class java.lang.Long")) + assert(types(2).equals("class java.lang.Integer")) + assert(types(3).equals("class java.lang.Integer")) + assert(types(4).equals("class java.lang.Integer")) + assert(types(5).equals("class java.lang.Long")) + assert(types(6).equals("class java.math.BigDecimal")) + assert(types(7).equals("class java.lang.Double")) + assert(types(8).equals("class java.lang.Double")) + assert(rows(0).getBoolean(0) == false) + assert(rows(0).getLong(1) == 0x225) + assert(rows(0).getInt(2) == 17) + assert(rows(0).getInt(3) == 77777) + assert(rows(0).getInt(4) == 123456789) + assert(rows(0).getLong(5) == 123456789012345L) + val bd = new BigDecimal("123456789012345.12345678901234500000") + assert(rows(0).getAs[BigDecimal](6).equals(bd)) + assert(rows(0).getDouble(7) == 42.75) + assert(rows(0).getDouble(8) == 1.0000000000000002) + } + + test("Date types") { + val df = sqlContext.read.jdbc(jdbcUrl, "dates", new Properties) + val rows = df.collect() + assert(rows.length == 1) + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types.length == 5) + assert(types(0).equals("class java.sql.Date")) + assert(types(1).equals("class java.sql.Timestamp")) + assert(types(2).equals("class java.sql.Timestamp")) + assert(types(3).equals("class java.sql.Timestamp")) + assert(types(4).equals("class java.sql.Date")) + assert(rows(0).getAs[Date](0).equals(Date.valueOf("1991-11-09"))) + assert(rows(0).getAs[Timestamp](1).equals(Timestamp.valueOf("1970-01-01 13:31:24"))) + assert(rows(0).getAs[Timestamp](2).equals(Timestamp.valueOf("1996-01-01 01:23:45"))) + assert(rows(0).getAs[Timestamp](3).equals(Timestamp.valueOf("2009-02-13 23:31:30"))) + assert(rows(0).getAs[Date](4).equals(Date.valueOf("2001-01-01"))) + } + + test("String types") { + val df = sqlContext.read.jdbc(jdbcUrl, "strings", new Properties) + val rows = df.collect() + assert(rows.length == 1) + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types.length == 9) + assert(types(0).equals("class java.lang.String")) + assert(types(1).equals("class java.lang.String")) + assert(types(2).equals("class java.lang.String")) + assert(types(3).equals("class java.lang.String")) + assert(types(4).equals("class java.lang.String")) + assert(types(5).equals("class java.lang.String")) + assert(types(6).equals("class [B")) + assert(types(7).equals("class [B")) + assert(types(8).equals("class [B")) + assert(rows(0).getString(0).equals("the")) + assert(rows(0).getString(1).equals("quick")) + assert(rows(0).getString(2).equals("brown")) + assert(rows(0).getString(3).equals("fox")) + assert(rows(0).getString(4).equals("jumps")) + assert(rows(0).getString(5).equals("over")) + assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](6), Array[Byte](116, 104, 101, 0))) + assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](7), Array[Byte](108, 97, 122, 121))) + assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](8), Array[Byte](100, 111, 103))) + } + + test("Basic write test") { + val df1 = sqlContext.read.jdbc(jdbcUrl, "numbers", new Properties) + val df2 = sqlContext.read.jdbc(jdbcUrl, "dates", new Properties) + val df3 = sqlContext.read.jdbc(jdbcUrl, "strings", new Properties) + df1.write.jdbc(jdbcUrl, "numberscopy", new Properties) + df2.write.jdbc(jdbcUrl, "datescopy", new Properties) + df3.write.jdbc(jdbcUrl, "stringscopy", new Properties) + } +} diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala new file mode 100644 index 000000000000..7680ae383513 --- /dev/null +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -0,0 +1,310 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.sql.{Connection, Date, Timestamp} +import java.util.Properties +import java.math.BigDecimal + +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.execution.{WholeStageCodegenExec, RowDataSourceScanExec} +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ +import org.apache.spark.tags.DockerTest + +/** + * This patch was tested using the Oracle docker. Created this integration suite for the same. + * The ojdbc6-11.2.0.2.0.jar was to be downloaded from the maven repository. Since there was + * no jdbc jar available in the maven repository, the jar was downloaded from oracle site + * manually and installed in the local; thus tested. So, for SparkQA test case run, the + * ojdbc jar might be manually placed in the local maven repository(com/oracle/ojdbc6/11.2.0.2.0) + * while Spark QA test run. + * + * The following would be the steps to test this + * 1. Pull oracle 11g image - docker pull wnameless/oracle-xe-11g + * 2. Start docker - sudo service docker start + * 3. Download oracle 11g driver jar and put it in maven local repo: + * (com/oracle/ojdbc6/11.2.0.2.0/ojdbc6-11.2.0.2.0.jar) + * 4. The timeout and interval parameter to be increased from 60,1 to a high value for oracle test + * in DockerJDBCIntegrationSuite.scala (Locally tested with 200,200 and executed successfully). + * 5. Run spark test - ./build/sbt "test-only org.apache.spark.sql.jdbc.OracleIntegrationSuite" + * + * All tests in this suite are ignored because of the dependency with the oracle jar from maven + * repository. + */ +@DockerTest +class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLContext { + import testImplicits._ + + override val db = new DatabaseOnDocker { + override val imageName = "wnameless/oracle-xe-11g:14.04.4" + override val env = Map( + "ORACLE_ROOT_PASSWORD" -> "oracle" + ) + override val usesIpc = false + override val jdbcPort: Int = 1521 + override def getJdbcUrl(ip: String, port: Int): String = + s"jdbc:oracle:thin:system/oracle@//$ip:$port/xe" + override def getStartupProcessName: Option[String] = None + } + + override def dataPreparation(conn: Connection): Unit = { + conn.prepareStatement("CREATE TABLE datetime (id NUMBER(10), d DATE, t TIMESTAMP)") + .executeUpdate() + conn.prepareStatement( + """INSERT INTO datetime VALUES + |(1, {d '1991-11-09'}, {ts '1996-01-01 01:23:45'}) + """.stripMargin.replaceAll("\n", " ")).executeUpdate() + conn.commit() + + conn.prepareStatement( + "CREATE TABLE ts_with_timezone (id NUMBER(10), t TIMESTAMP WITH TIME ZONE)").executeUpdate() + conn.prepareStatement( + "INSERT INTO ts_with_timezone VALUES " + + "(1, to_timestamp_tz('1999-12-01 11:00:00 UTC','YYYY-MM-DD HH:MI:SS TZR'))").executeUpdate() + conn.commit() + + conn.prepareStatement( + "CREATE TABLE tableWithCustomSchema (id NUMBER, n1 NUMBER(1), n2 NUMBER(1))").executeUpdate() + conn.prepareStatement( + "INSERT INTO tableWithCustomSchema values(12312321321321312312312312123, 1, 0)").executeUpdate() + conn.commit() + + sql( + s""" + |CREATE TEMPORARY VIEW datetime + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$jdbcUrl', dbTable 'datetime', oracle.jdbc.mapDateToTimestamp 'false') + """.stripMargin.replaceAll("\n", " ")) + + conn.prepareStatement("CREATE TABLE datetime1 (id NUMBER(10), d DATE, t TIMESTAMP)") + .executeUpdate() + conn.commit() + + sql( + s""" + |CREATE TEMPORARY VIEW datetime1 + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$jdbcUrl', dbTable 'datetime1', oracle.jdbc.mapDateToTimestamp 'false') + """.stripMargin.replaceAll("\n", " ")) + + + conn.prepareStatement("CREATE TABLE numerics (b DECIMAL(1), f DECIMAL(3, 2), i DECIMAL(10))").executeUpdate(); + conn.prepareStatement( + "INSERT INTO numerics VALUES (4, 1.23, 9999999999)").executeUpdate(); + conn.commit(); + } + + + test("SPARK-16625 : Importing Oracle numeric types") { + val df = sqlContext.read.jdbc(jdbcUrl, "numerics", new Properties); + val rows = df.collect() + assert(rows.size == 1) + val row = rows(0) + // The main point of the below assertions is not to make sure that these Oracle types are + // mapped to decimal types, but to make sure that the returned values are correct. + // A value > 1 from DECIMAL(1) is correct: + assert(row.getDecimal(0).compareTo(BigDecimal.valueOf(4)) == 0) + // A value with fractions from DECIMAL(3, 2) is correct: + assert(row.getDecimal(1).compareTo(BigDecimal.valueOf(1.23)) == 0) + // A value > Int.MaxValue from DECIMAL(10) is correct: + assert(row.getDecimal(2).compareTo(BigDecimal.valueOf(9999999999l)) == 0) + } + + + test("SPARK-12941: String datatypes to be mapped to Varchar in Oracle") { + // create a sample dataframe with string type + val df1 = sparkContext.parallelize(Seq(("foo"))).toDF("x") + // write the dataframe to the oracle table tbl + df1.write.jdbc(jdbcUrl, "tbl2", new Properties) + // read the table from the oracle + val dfRead = sqlContext.read.jdbc(jdbcUrl, "tbl2", new Properties) + // get the rows + val rows = dfRead.collect() + // verify the data type is inserted + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types(0).equals("class java.lang.String")) + // verify the value is the inserted correct or not + assert(rows(0).getString(0).equals("foo")) + } + + test("SPARK-16625: General data types to be mapped to Oracle") { + val props = new Properties() + props.put("oracle.jdbc.mapDateToTimestamp", "false") + + val schema = StructType(Seq( + StructField("boolean_type", BooleanType, true), + StructField("integer_type", IntegerType, true), + StructField("long_type", LongType, true), + StructField("float_Type", FloatType, true), + StructField("double_type", DoubleType, true), + StructField("byte_type", ByteType, true), + StructField("short_type", ShortType, true), + StructField("string_type", StringType, true), + StructField("binary_type", BinaryType, true), + StructField("date_type", DateType, true), + StructField("timestamp_type", TimestampType, true) + )) + + val tableName = "test_oracle_general_types" + val booleanVal = true + val integerVal = 1 + val longVal = 2L + val floatVal = 3.0f + val doubleVal = 4.0 + val byteVal = 2.toByte + val shortVal = 5.toShort + val stringVal = "string" + val binaryVal = Array[Byte](6, 7, 8) + val dateVal = Date.valueOf("2016-07-26") + val timestampVal = Timestamp.valueOf("2016-07-26 11:49:45") + + val data = spark.sparkContext.parallelize(Seq( + Row( + booleanVal, integerVal, longVal, floatVal, doubleVal, byteVal, shortVal, stringVal, + binaryVal, dateVal, timestampVal + ))) + + val dfWrite = spark.createDataFrame(data, schema) + dfWrite.write.jdbc(jdbcUrl, tableName, props) + + val dfRead = spark.read.jdbc(jdbcUrl, tableName, props) + val rows = dfRead.collect() + // verify the data type is inserted + val types = dfRead.schema.map(field => field.dataType) + assert(types(0).equals(DecimalType(1, 0))) + assert(types(1).equals(DecimalType(10, 0))) + assert(types(2).equals(DecimalType(19, 0))) + assert(types(3).equals(DecimalType(19, 4))) + assert(types(4).equals(DecimalType(19, 4))) + assert(types(5).equals(DecimalType(3, 0))) + assert(types(6).equals(DecimalType(5, 0))) + assert(types(7).equals(StringType)) + assert(types(8).equals(BinaryType)) + assert(types(9).equals(DateType)) + assert(types(10).equals(TimestampType)) + + // verify the value is the inserted correct or not + val values = rows(0) + assert(values.getDecimal(0).compareTo(BigDecimal.valueOf(1)) == 0) + assert(values.getDecimal(1).compareTo(BigDecimal.valueOf(integerVal)) == 0) + assert(values.getDecimal(2).compareTo(BigDecimal.valueOf(longVal)) == 0) + assert(values.getDecimal(3).compareTo(BigDecimal.valueOf(floatVal)) == 0) + assert(values.getDecimal(4).compareTo(BigDecimal.valueOf(doubleVal)) == 0) + assert(values.getDecimal(5).compareTo(BigDecimal.valueOf(byteVal)) == 0) + assert(values.getDecimal(6).compareTo(BigDecimal.valueOf(shortVal)) == 0) + assert(values.getString(7).equals(stringVal)) + assert(values.getAs[Array[Byte]](8).mkString.equals("678")) + assert(values.getDate(9).equals(dateVal)) + assert(values.getTimestamp(10).equals(timestampVal)) + } + + test("SPARK-19318: connection property keys should be case-sensitive") { + def checkRow(row: Row): Unit = { + assert(row.getDecimal(0).equals(BigDecimal.valueOf(1))) + assert(row.getDate(1).equals(Date.valueOf("1991-11-09"))) + assert(row.getTimestamp(2).equals(Timestamp.valueOf("1996-01-01 01:23:45"))) + } + checkRow(sql("SELECT * FROM datetime where id = 1").head()) + sql("INSERT INTO TABLE datetime1 SELECT * FROM datetime where id = 1") + checkRow(sql("SELECT * FROM datetime1 where id = 1").head()) + } + + test("SPARK-20557: column type TIMESTAMP with TIME ZONE should be recognized") { + val dfRead = sqlContext.read.jdbc(jdbcUrl, "ts_with_timezone", new Properties) + val rows = dfRead.collect() + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types(1).equals("class java.sql.Timestamp")) + } + + test("SPARK-18004: Make sure date or timestamp related predicate is pushed down correctly") { + val props = new Properties() + props.put("oracle.jdbc.mapDateToTimestamp", "false") + + val schema = StructType(Seq( + StructField("date_type", DateType, true), + StructField("timestamp_type", TimestampType, true) + )) + + val tableName = "test_date_timestamp_pushdown" + val dateVal = Date.valueOf("2017-06-22") + val timestampVal = Timestamp.valueOf("2017-06-22 21:30:07") + + val data = spark.sparkContext.parallelize(Seq( + Row(dateVal, timestampVal) + )) + + val dfWrite = spark.createDataFrame(data, schema) + dfWrite.write.jdbc(jdbcUrl, tableName, props) + + val dfRead = spark.read.jdbc(jdbcUrl, tableName, props) + + val millis = System.currentTimeMillis() + val dt = new java.sql.Date(millis) + val ts = new java.sql.Timestamp(millis) + + // Query Oracle table with date and timestamp predicates + // which should be pushed down to Oracle. + val df = dfRead.filter(dfRead.col("date_type").lt(dt)) + .filter(dfRead.col("timestamp_type").lt(ts)) + + val parentPlan = df.queryExecution.executedPlan + assert(parentPlan.isInstanceOf[WholeStageCodegenExec]) + val node = parentPlan.asInstanceOf[WholeStageCodegenExec] + val metadata = node.child.asInstanceOf[RowDataSourceScanExec].metadata + // The "PushedFilters" part should exist in Dataframe's + // physical plan and the existence of right literals in + // "PushedFilters" is used to prove that the predicates + // pushing down have been effective. + assert(metadata.get("PushedFilters").isDefined) + assert(metadata("PushedFilters").contains(dt.toString)) + assert(metadata("PushedFilters").contains(ts.toString)) + + val row = df.collect()(0) + assert(row.getDate(0).equals(dateVal)) + assert(row.getTimestamp(1).equals(timestampVal)) + } + + test("SPARK-20427/SPARK-20921: read table use custom schema by jdbc api") { + // default will throw IllegalArgumentException + val e = intercept[org.apache.spark.SparkException] { + spark.read.jdbc(jdbcUrl, "tableWithCustomSchema", new Properties()).collect() + } + assert(e.getMessage.contains( + "requirement failed: Decimal precision 39 exceeds max precision 38")) + + // custom schema can read data + val props = new Properties() + props.put("customSchema", + s"ID DECIMAL(${DecimalType.MAX_PRECISION}, 0), N1 INT, N2 BOOLEAN") + val dfRead = spark.read.jdbc(jdbcUrl, "tableWithCustomSchema", props) + + val rows = dfRead.collect() + // verify the data type + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types(0).equals("class java.math.BigDecimal")) + assert(types(1).equals("class java.lang.Integer")) + assert(types(2).equals("class java.lang.Boolean")) + + // verify the value + val values = rows(0) + assert(values.getDecimal(0).equals(new java.math.BigDecimal("12312321321321312312312312123"))) + assert(values.getInt(1).equals(1)) + assert(values.getBoolean(2).equals(false)) + } +} diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala new file mode 100644 index 000000000000..eb3c458360e7 --- /dev/null +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.sql.Connection +import java.util.Properties + +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.types.{ArrayType, DecimalType, FloatType, ShortType} +import org.apache.spark.tags.DockerTest + +@DockerTest +class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { + override val db = new DatabaseOnDocker { + override val imageName = "postgres:9.4.5" + override val env = Map( + "POSTGRES_PASSWORD" -> "rootpass" + ) + override val usesIpc = false + override val jdbcPort = 5432 + override def getJdbcUrl(ip: String, port: Int): String = + s"jdbc:postgresql://$ip:$port/postgres?user=postgres&password=rootpass" + override def getStartupProcessName: Option[String] = None + } + + override def dataPreparation(conn: Connection): Unit = { + conn.prepareStatement("CREATE DATABASE foo").executeUpdate() + conn.setCatalog("foo") + conn.prepareStatement("CREATE TYPE enum_type AS ENUM ('d1', 'd2')").executeUpdate() + conn.prepareStatement("CREATE TABLE bar (c0 text, c1 integer, c2 double precision, c3 bigint, " + + "c4 bit(1), c5 bit(10), c6 bytea, c7 boolean, c8 inet, c9 cidr, " + + "c10 integer[], c11 text[], c12 real[], c13 numeric(2,2)[], c14 enum_type, " + + "c15 float4, c16 smallint)").executeUpdate() + conn.prepareStatement("INSERT INTO bar VALUES ('hello', 42, 1.25, 123456789012345, B'0', " + + "B'1000100101', E'\\\\xDEADBEEF', true, '172.16.0.42', '192.168.0.0/16', " + + """'{1, 2}', '{"a", null, "b"}', '{0.11, 0.22}', '{0.11, 0.22}', 'd1', 1.01, 1)""" + ).executeUpdate() + conn.prepareStatement("INSERT INTO bar VALUES (null, null, null, null, null, " + + "null, null, null, null, null, " + + "null, null, null, null, null, null, null)" + ).executeUpdate() + + conn.prepareStatement("CREATE TABLE ts_with_timezone " + + "(id integer, tstz TIMESTAMP WITH TIME ZONE, ttz TIME WITH TIME ZONE)") + .executeUpdate() + conn.prepareStatement("INSERT INTO ts_with_timezone VALUES " + + "(1, TIMESTAMP WITH TIME ZONE '2016-08-12 10:22:31.949271-07', TIME WITH TIME ZONE '17:22:31.949271+00')") + .executeUpdate() + } + + test("Type mapping for various types") { + val df = sqlContext.read.jdbc(jdbcUrl, "bar", new Properties) + val rows = df.collect().sortBy(_.toString()) + assert(rows.length == 2) + // Test the types, and values using the first row. + val types = rows(0).toSeq.map(x => x.getClass) + assert(types.length == 17) + assert(classOf[String].isAssignableFrom(types(0))) + assert(classOf[java.lang.Integer].isAssignableFrom(types(1))) + assert(classOf[java.lang.Double].isAssignableFrom(types(2))) + assert(classOf[java.lang.Long].isAssignableFrom(types(3))) + assert(classOf[java.lang.Boolean].isAssignableFrom(types(4))) + assert(classOf[Array[Byte]].isAssignableFrom(types(5))) + assert(classOf[Array[Byte]].isAssignableFrom(types(6))) + assert(classOf[java.lang.Boolean].isAssignableFrom(types(7))) + assert(classOf[String].isAssignableFrom(types(8))) + assert(classOf[String].isAssignableFrom(types(9))) + assert(classOf[Seq[Int]].isAssignableFrom(types(10))) + assert(classOf[Seq[String]].isAssignableFrom(types(11))) + assert(classOf[Seq[Double]].isAssignableFrom(types(12))) + assert(classOf[Seq[BigDecimal]].isAssignableFrom(types(13))) + assert(classOf[String].isAssignableFrom(types(14))) + assert(classOf[java.lang.Float].isAssignableFrom(types(15))) + assert(classOf[java.lang.Short].isAssignableFrom(types(16))) + assert(rows(0).getString(0).equals("hello")) + assert(rows(0).getInt(1) == 42) + assert(rows(0).getDouble(2) == 1.25) + assert(rows(0).getLong(3) == 123456789012345L) + assert(!rows(0).getBoolean(4)) + // BIT(10)'s come back as ASCII strings of ten ASCII 0's and 1's... + assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](5), + Array[Byte](49, 48, 48, 48, 49, 48, 48, 49, 48, 49))) + assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](6), + Array[Byte](0xDE.toByte, 0xAD.toByte, 0xBE.toByte, 0xEF.toByte))) + assert(rows(0).getBoolean(7)) + assert(rows(0).getString(8) == "172.16.0.42") + assert(rows(0).getString(9) == "192.168.0.0/16") + assert(rows(0).getSeq(10) == Seq(1, 2)) + assert(rows(0).getSeq(11) == Seq("a", null, "b")) + assert(rows(0).getSeq(12).toSeq == Seq(0.11f, 0.22f)) + assert(rows(0).getSeq(13) == Seq("0.11", "0.22").map(BigDecimal(_).bigDecimal)) + assert(rows(0).getString(14) == "d1") + assert(rows(0).getFloat(15) == 1.01f) + assert(rows(0).getShort(16) == 1) + + // Test reading null values using the second row. + assert(0.until(16).forall(rows(1).isNullAt(_))) + } + + test("Basic write test") { + val df = sqlContext.read.jdbc(jdbcUrl, "bar", new Properties) + // Test only that it doesn't crash. + df.write.jdbc(jdbcUrl, "public.barcopy", new Properties) + // Test that written numeric type has same DataType as input + assert(sqlContext.read.jdbc(jdbcUrl, "public.barcopy", new Properties).schema(13).dataType == + ArrayType(DecimalType(2, 2), true)) + // Test write null values. + df.select(df.queryExecution.analyzed.output.map { a => + Column(Literal.create(null, a.dataType)).as(a.name) + }: _*).write.jdbc(jdbcUrl, "public.barcopy2", new Properties) + } + + test("Creating a table with shorts and floats") { + sqlContext.createDataFrame(Seq((1.0f, 1.toShort))) + .write.jdbc(jdbcUrl, "shortfloat", new Properties) + val schema = sqlContext.read.jdbc(jdbcUrl, "shortfloat", new Properties).schema + assert(schema(0).dataType == FloatType) + assert(schema(1).dataType == ShortType) + } + + test("SPARK-20557: column type TIMESTAMP with TIME ZONE and TIME with TIME ZONE should be recognized") { + val dfRead = sqlContext.read.jdbc(jdbcUrl, "ts_with_timezone", new Properties) + val rows = dfRead.collect() + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types(1).equals("class java.sql.Timestamp")) + assert(types(2).equals("class java.sql.Timestamp")) + } +} diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/util/DockerUtils.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/util/DockerUtils.scala new file mode 100644 index 000000000000..fda377e03235 --- /dev/null +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/util/DockerUtils.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.net.{Inet4Address, InetAddress, NetworkInterface} + +import scala.collection.JavaConverters._ +import scala.sys.process._ +import scala.util.Try + +private[spark] object DockerUtils { + + def getDockerIp(): String = { + /** If docker-machine is setup on this box, attempts to find the ip from it. */ + def findFromDockerMachine(): Option[String] = { + sys.env.get("DOCKER_MACHINE_NAME").flatMap { name => + Try(Seq("/bin/bash", "-c", s"docker-machine ip $name 2>/dev/null").!!.trim).toOption + } + } + sys.env.get("DOCKER_IP") + .orElse(findFromDockerMachine()) + .orElse(Try(Seq("/bin/bash", "-c", "boot2docker ip 2>/dev/null").!!.trim).toOption) + .getOrElse { + // This block of code is based on Utils.findLocalInetAddress(), but is modified to blacklist + // certain interfaces. + val address = InetAddress.getLocalHost + // Address resolves to something like 127.0.1.1, which happens on Debian; try to find + // a better address using the local network interfaces + // getNetworkInterfaces returns ifs in reverse order compared to ifconfig output order + // on unix-like system. On windows, it returns in index order. + // It's more proper to pick ip address following system output order. + val blackListedIFs = Seq( + "vboxnet0", // Mac + "docker0" // Linux + ) + val activeNetworkIFs = NetworkInterface.getNetworkInterfaces.asScala.toSeq.filter { i => + !blackListedIFs.contains(i.getName) + } + val reOrderedNetworkIFs = activeNetworkIFs.reverse + for (ni <- reOrderedNetworkIFs) { + val addresses = ni.getInetAddresses.asScala + .filterNot(addr => addr.isLinkLocalAddress || addr.isLoopbackAddress).toSeq + if (addresses.nonEmpty) { + val addr = addresses.find(_.isInstanceOf[Inet4Address]).getOrElse(addresses.head) + // because of Inet6Address.toHostName may add interface at the end if it knows about it + val strippedAddress = InetAddress.getByAddress(addr.getAddress) + return strippedAddress.getHostAddress + } + } + address.getHostAddress + } + } +} diff --git a/docker/README.md b/external/docker/README.md similarity index 100% rename from docker/README.md rename to external/docker/README.md diff --git a/docker/build b/external/docker/build similarity index 100% rename from docker/build rename to external/docker/build diff --git a/docker/spark-mesos/Dockerfile b/external/docker/spark-mesos/Dockerfile similarity index 100% rename from docker/spark-mesos/Dockerfile rename to external/docker/spark-mesos/Dockerfile diff --git a/docker/spark-test/README.md b/external/docker/spark-test/README.md similarity index 100% rename from docker/spark-test/README.md rename to external/docker/spark-test/README.md diff --git a/docker/spark-test/base/Dockerfile b/external/docker/spark-test/base/Dockerfile similarity index 98% rename from docker/spark-test/base/Dockerfile rename to external/docker/spark-test/base/Dockerfile index 7ba0de603dc7..5a95a9387c31 100644 --- a/docker/spark-test/base/Dockerfile +++ b/external/docker/spark-test/base/Dockerfile @@ -25,7 +25,7 @@ RUN apt-get update && \ apt-get install -y less openjdk-7-jre-headless net-tools vim-tiny sudo openssh-server && \ rm -rf /var/lib/apt/lists/* -ENV SCALA_VERSION 2.10.5 +ENV SCALA_VERSION 2.11.8 ENV CDH_VERSION cdh4 ENV SCALA_HOME /opt/scala-$SCALA_VERSION ENV SPARK_HOME /opt/spark diff --git a/docker/spark-test/build b/external/docker/spark-test/build similarity index 100% rename from docker/spark-test/build rename to external/docker/spark-test/build diff --git a/docker/spark-test/master/Dockerfile b/external/docker/spark-test/master/Dockerfile similarity index 100% rename from docker/spark-test/master/Dockerfile rename to external/docker/spark-test/master/Dockerfile diff --git a/docker/spark-test/master/default_cmd b/external/docker/spark-test/master/default_cmd similarity index 100% rename from docker/spark-test/master/default_cmd rename to external/docker/spark-test/master/default_cmd diff --git a/docker/spark-test/worker/Dockerfile b/external/docker/spark-test/worker/Dockerfile similarity index 100% rename from docker/spark-test/worker/Dockerfile rename to external/docker/spark-test/worker/Dockerfile diff --git a/docker/spark-test/worker/default_cmd b/external/docker/spark-test/worker/default_cmd similarity index 100% rename from docker/spark-test/worker/default_cmd rename to external/docker/spark-test/worker/default_cmd diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index dceedcf23ed5..71016bc645ca 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -20,13 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT + spark-parent_2.11 + 2.3.0-SNAPSHOT ../../pom.xml - org.apache.spark - spark-streaming-flume-assembly_2.10 + spark-streaming-flume-assembly_2.11 jar Spark Project External Flume Assembly http://spark.apache.org/ diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 75113ff753e7..12630840e79d 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -20,13 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT + spark-parent_2.11 + 2.3.0-SNAPSHOT ../../pom.xml - org.apache.spark - spark-streaming-flume-sink_2.10 + spark-streaming-flume-sink_2.11 streaming-flume-sink @@ -92,8 +91,20 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + target/scala-${scala.binary.version}/classes diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala index d87b86932dd4..09d3fe91e42c 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala @@ -26,20 +26,20 @@ import org.slf4j.{Logger, LoggerFactory} private[sink] trait Logging { // Make the log field transient so that objects with Logging can // be serialized and used on another machine - @transient private var log_ : Logger = null + @transient private var _log: Logger = null // Method to get or create the logger for this object protected def log: Logger = { - if (log_ == null) { + if (_log == null) { initializeIfNecessary() var className = this.getClass.getName // Ignore trailing $'s in the class names for Scala objects if (className.endsWith("$")) { className = className.substring(0, className.length - 1) } - log_ = LoggerFactory.getLogger(className) + _log = LoggerFactory.getLogger(className) } - log_ + _log } // Log methods that take only a String @@ -101,7 +101,7 @@ private[sink] trait Logging { private def initializeLogging() { Logging.initialized = true - // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads + // Force a call into slf4j to initialize it. Avoids this happening from multiple threads // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html log } diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala index 719fca0938b3..8050ec357e26 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala @@ -129,9 +129,9 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha * @param success Whether the batch was successful or not. */ private def completeTransaction(sequenceNumber: CharSequence, success: Boolean) { - removeAndGetProcessor(sequenceNumber).foreach(processor => { + removeAndGetProcessor(sequenceNumber).foreach { processor => processor.batchProcessed(success) - }) + } } /** diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala index 14dffb15fef9..e5b63aa1a77e 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala @@ -45,7 +45,7 @@ import org.apache.flume.sink.AbstractSink * the thread itself is blocked and a reference to it saved off. * * When the ack for that batch is received, - * the thread which created the transaction is is retrieved and it commits the transaction with the + * the thread which created the transaction is retrieved and it commits the transaction with the * channel from the same thread it was originally created in (since Flume transactions are * thread local). If a nack is received instead, the sink rolls back the transaction. If no ack * is received within the specified timeout, the transaction is rolled back too. If an ack comes @@ -88,23 +88,23 @@ class SparkSink extends AbstractSink with Logging with Configurable { // dependencies which are being excluded in the build. In practice, // Netty dependencies are already available on the JVM as Flume would have pulled them in. serverOpt = Option(new NettyServer(responder, new InetSocketAddress(hostname, port))) - serverOpt.foreach(server => { + serverOpt.foreach { server => logInfo("Starting Avro server for sink: " + getName) server.start() - }) + } super.start() } override def stop() { logInfo("Stopping Spark Sink: " + getName) - handler.foreach(callbackHandler => { + handler.foreach { callbackHandler => callbackHandler.shutdown() - }) - serverOpt.foreach(server => { + } + serverOpt.foreach { server => logInfo("Stopping Avro Server for sink: " + getName) server.close() server.join() - }) + } blockingLatch.countDown() super.stop() } diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala index 7ad43b1d7b0a..19e736f01697 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala @@ -22,7 +22,7 @@ import java.util.concurrent.{Callable, CountDownLatch, TimeUnit} import scala.util.control.Breaks -import org.apache.flume.{Transaction, Channel} +import org.apache.flume.{Channel, Transaction} // Flume forces transactions to be thread-local (horrible, I know!) // So the sink basically spawns a new thread to pull the events out within a transaction. @@ -110,7 +110,7 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, eventBatch.setErrorMsg("Something went wrong. Channel was " + "unable to create a transaction!") } - txOpt.foreach(tx => { + txOpt.foreach { tx => tx.begin() val events = new util.ArrayList[SparkSinkEvent](maxBatchSize) val loop = new Breaks @@ -145,7 +145,7 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, // At this point, the events are available, so fill them into the event batch eventBatch = new EventBatch("", seqNum, events) } - }) + } } catch { case interrupted: InterruptedException => // Don't pollute logs if the InterruptedException came from this being stopped @@ -156,9 +156,9 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, logWarning("Error while processing transaction.", e) eventBatch.setErrorMsg(e.getMessage) try { - txOpt.foreach(tx => { + txOpt.foreach { tx => rollbackAndClose(tx, close = true) - }) + } } finally { txOpt = None } @@ -174,7 +174,7 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, */ private def processAckOrNack() { batchAckLatch.await(transactionTimeout, TimeUnit.SECONDS) - txOpt.foreach(tx => { + txOpt.foreach { tx => if (batchSuccess) { try { logDebug("Committing transaction") @@ -197,7 +197,7 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, // cause issues. This is required to ensure the TransactionProcessor instance is not leaked parent.removeAndGetProcessor(seqNum) } - }) + } } /** diff --git a/external/flume-sink/src/test/resources/log4j.properties b/external/flume-sink/src/test/resources/log4j.properties index 42df8792f147..1e3f163f95c0 100644 --- a/external/flume-sink/src/test/resources/log4j.properties +++ b/external/flume-sink/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN +log4j.logger.org.spark_project.jetty=WARN diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala index d2654700ea72..e8ca1e716394 100644 --- a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala +++ b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala @@ -17,8 +17,9 @@ package org.apache.spark.streaming.flume.sink import java.net.InetSocketAddress +import java.nio.charset.StandardCharsets +import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.{TimeUnit, CountDownLatch, Executors} import scala.collection.JavaConverters._ import scala.concurrent.{ExecutionContext, Future} @@ -36,11 +37,11 @@ import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory // Spark core main, which has too many dependencies to require here manually. // For this reason, we continue to use FunSuite and ignore the scalastyle checks // that fail if this is detected. -//scalastyle:off +// scalastyle:off import org.scalatest.FunSuite class SparkSinkSuite extends FunSuite { -//scalastyle:on +// scalastyle:on val eventsPerBatch = 1000 val channelCapacity = 5000 @@ -184,7 +185,8 @@ class SparkSinkSuite extends FunSuite { private def putEvents(ch: MemoryChannel, count: Int): Unit = { val tx = ch.getTransaction tx.begin() - (1 to count).foreach(x => ch.put(EventBuilder.withBody(x.toString.getBytes))) + (1 to count).foreach(x => + ch.put(EventBuilder.withBody(x.toString.getBytes(StandardCharsets.UTF_8)))) tx.commit() tx.close() } diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 57f83607365d..87a09642405a 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -20,13 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT + spark-parent_2.11 + 2.3.0-SNAPSHOT ../../pom.xml - org.apache.spark - spark-streaming-flume_2.10 + spark-streaming-flume_2.11 streaming-flume @@ -68,8 +67,20 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + target/scala-${scala.binary.version}/classes diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala index 48df27b26867..07c528647773 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala @@ -17,12 +17,12 @@ package org.apache.spark.streaming.flume -import java.io.{ObjectOutput, ObjectInput} +import java.io.{ObjectInput, ObjectOutput} import scala.collection.JavaConverters._ +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils -import org.apache.spark.Logging /** * A simple object that provides the implementation of readExternal and writeExternal for both diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala index b9d4e762ca05..8af7c2343106 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala @@ -20,13 +20,13 @@ import scala.collection.mutable.ArrayBuffer import com.google.common.base.Throwables -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.streaming.flume.sink._ /** * This class implements the core functionality of [[FlumePollingReceiver]]. When started it * pulls data from Flume, stores it to Spark and then sends an Ack or Nack. This class should be - * run via an [[java.util.concurrent.Executor]] as this implements [[Runnable]] + * run via a [[java.util.concurrent.Executor]] as this implements [[Runnable]] * * @param receiver The receiver that owns this instance. */ @@ -77,7 +77,7 @@ private[flume] class FlumeBatchFetcher(receiver: FlumePollingReceiver) extends R /** * Gets a batch of events from the specified client. This method does not handle any exceptions - * which will be propogated to the caller. + * which will be propagated to the caller. * @param client Client to get events from * @return [[Some]] which contains the event batch if Flume sent any events back, else [[None]] */ @@ -96,8 +96,8 @@ private[flume] class FlumeBatchFetcher(receiver: FlumePollingReceiver) extends R } /** - * Store the events in the buffer to Spark. This method will not propogate any exceptions, - * but will propogate any other errors. + * Store the events in the buffer to Spark. This method will not propagate any exceptions, + * but will propagate any other errors. * @param buffer The buffer to store * @return true if the data was stored without any exception being thrown, else false */ diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index c8780aa83bdb..13aa817492f7 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -17,38 +17,36 @@ package org.apache.spark.streaming.flume +import java.io.{Externalizable, ObjectInput, ObjectOutput} import java.net.InetSocketAddress -import java.io.{ObjectInput, ObjectOutput, Externalizable} import java.nio.ByteBuffer import java.util.concurrent.Executors import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import org.apache.flume.source.avro.AvroSourceProtocol -import org.apache.flume.source.avro.AvroFlumeEvent -import org.apache.flume.source.avro.Status -import org.apache.avro.ipc.specific.SpecificResponder import org.apache.avro.ipc.NettyServer -import org.apache.spark.Logging -import org.apache.spark.util.Utils -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.receiver.Receiver - +import org.apache.avro.ipc.specific.SpecificResponder +import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol, Status} import org.jboss.netty.channel.{ChannelPipeline, ChannelPipelineFactory, Channels} import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory import org.jboss.netty.handler.codec.compression._ +import org.apache.spark.internal.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.util.Utils + private[streaming] class FlumeInputDStream[T: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, host: String, port: Int, storageLevel: StorageLevel, enableDecompression: Boolean -) extends ReceiverInputDStream[SparkFlumeEvent](ssc_) { +) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) { override def getReceiver(): Receiver[SparkFlumeEvent] = { new FlumeReceiver(host, port, storageLevel, enableDecompression) @@ -62,7 +60,7 @@ class FlumeInputDStream[T: ClassTag]( * which are not serializable. */ class SparkFlumeEvent() extends Externalizable { - var event : AvroFlumeEvent = new AvroFlumeEvent() + var event: AvroFlumeEvent = new AvroFlumeEvent() /* De-serialize from bytes. */ def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -77,12 +75,12 @@ class SparkFlumeEvent() extends Externalizable { val keyLength = in.readInt() val keyBuff = new Array[Byte](keyLength) in.readFully(keyBuff) - val key : String = Utils.deserialize(keyBuff) + val key: String = Utils.deserialize(keyBuff) val valLength = in.readInt() val valBuff = new Array[Byte](valLength) in.readFully(valBuff) - val value : String = Utils.deserialize(valBuff) + val value: String = Utils.deserialize(valBuff) headers.put(key, value) } @@ -93,9 +91,9 @@ class SparkFlumeEvent() extends Externalizable { /* Serialize to bytes. */ def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { - val body = event.getBody.array() - out.writeInt(body.length) - out.write(body) + val body = event.getBody + out.writeInt(body.remaining()) + Utils.writeByteBuffer(body, out) val numHeaders = event.getHeaders.size() out.writeInt(numHeaders) @@ -111,7 +109,7 @@ class SparkFlumeEvent() extends Externalizable { } private[streaming] object SparkFlumeEvent { - def fromAvroFlumeEvent(in : AvroFlumeEvent) : SparkFlumeEvent = { + def fromAvroFlumeEvent(in: AvroFlumeEvent): SparkFlumeEvent = { val event = new SparkFlumeEvent event.event = in event @@ -120,20 +118,22 @@ private[streaming] object SparkFlumeEvent { /** A simple server that implements Flume's Avro protocol. */ private[streaming] -class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol { - override def append(event : AvroFlumeEvent) : Status = { +class FlumeEventServer(receiver: FlumeReceiver) extends AvroSourceProtocol { + override def append(event: AvroFlumeEvent): Status = { receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event)) Status.OK } - override def appendBatch(events : java.util.List[AvroFlumeEvent]) : Status = { + override def appendBatch(events: java.util.List[AvroFlumeEvent]): Status = { events.asScala.foreach(event => receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event))) Status.OK } } -/** A NetworkReceiver which listens for events using the - * Flume Avro interface. */ +/** + * A NetworkReceiver which listens for events using the + * Flume Avro interface. + */ private[streaming] class FlumeReceiver( host: String, @@ -187,13 +187,14 @@ class FlumeReceiver( override def preferredLocation: Option[String] = Option(host) - /** A Netty Pipeline factory that will decompress incoming data from - * and the Netty client and compress data going back to the client. - * - * The compression on the return is required because Flume requires - * a successful response to indicate it can remove the event/batch - * from the configured channel - */ + /** + * A Netty Pipeline factory that will decompress incoming data from + * and the Netty client and compress data going back to the client. + * + * The compression on the return is required because Flume requires + * a successful response to indicate it can remove the event/batch + * from the configured channel + */ private[streaming] class CompressionChannelPipelineFactory extends ChannelPipelineFactory { def getPipeline(): ChannelPipeline = { diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index 6737750c3d63..d84e289272c6 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -28,15 +28,15 @@ import org.apache.avro.ipc.NettyTransceiver import org.apache.avro.ipc.specific.SpecificRequestor import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream.ReceiverInputDStream -import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.streaming.flume.sink._ +import org.apache.spark.streaming.receiver.Receiver /** - * A [[ReceiverInputDStream]] that can be used to read data from several Flume agents running + * A `ReceiverInputDStream` that can be used to read data from several Flume agents running * [[org.apache.spark.streaming.flume.sink.SparkSink]]s. * @param _ssc Streaming context that will execute this input stream * @param addresses List of addresses at which SparkSinks are listening @@ -79,11 +79,11 @@ private[streaming] class FlumePollingReceiver( override def onStart(): Unit = { // Create the connections to each Flume agent. - addresses.foreach(host => { + addresses.foreach { host => val transceiver = new NettyTransceiver(host, channelFactory) val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) connections.add(new FlumeConnection(transceiver, client)) - }) + } for (i <- 0 until parallelism) { logInfo("Starting Flume Polling Receiver worker threads..") // Threads that pull data from Flume. diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala index 70018c86f92b..e8623b4766ae 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala @@ -19,23 +19,24 @@ package org.apache.spark.streaming.flume import java.net.{InetSocketAddress, ServerSocket} import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets +import java.util.{List => JList} import java.util.Collections import scala.collection.JavaConverters._ -import com.google.common.base.Charsets.UTF_8 import org.apache.avro.ipc.NettyTransceiver import org.apache.avro.ipc.specific.SpecificRequestor import org.apache.commons.lang3.RandomUtils import org.apache.flume.source.avro -import org.apache.flume.source.avro.{AvroSourceProtocol, AvroFlumeEvent} +import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol} import org.jboss.netty.channel.ChannelPipeline import org.jboss.netty.channel.socket.SocketChannel import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory import org.jboss.netty.handler.codec.compression.{ZlibDecoder, ZlibEncoder} -import org.apache.spark.util.Utils import org.apache.spark.SparkConf +import org.apache.spark.util.Utils /** * Share codes for Scala and Python unit tests @@ -59,12 +60,12 @@ private[flume] class FlumeTestUtils { } /** Send data to the flume receiver */ - def writeInput(input: Seq[String], enableCompression: Boolean): Unit = { + def writeInput(input: JList[String], enableCompression: Boolean): Unit = { val testAddress = new InetSocketAddress("localhost", testPort) - val inputEvents = input.map { item => + val inputEvents = input.asScala.map { item => val event = new AvroFlumeEvent - event.setBody(ByteBuffer.wrap(item.getBytes(UTF_8))) + event.setBody(ByteBuffer.wrap(item.getBytes(StandardCharsets.UTF_8))) event.setHeaders(Collections.singletonMap("test", "header")) event } 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 c719b80aca7e..3e3ed712f0db 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 @@ -17,8 +17,8 @@ package org.apache.spark.streaming.flume +import java.io.{ByteArrayOutputStream, DataOutputStream} import java.net.InetSocketAddress -import java.io.{DataOutputStream, ByteArrayOutputStream} import java.util.{List => JList, Map => JMap} import scala.collection.JavaConverters._ @@ -30,7 +30,6 @@ import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaReceiverInputDStream, JavaStreamingContext} import org.apache.spark.streaming.dstream.ReceiverInputDStream - object FlumeUtils { private val DEFAULT_POLLING_PARALLELISM = 5 private val DEFAULT_POLLING_BATCH_SIZE = 1000 diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala index a2ab320957db..a3e784a4f32e 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala @@ -17,18 +17,18 @@ package org.apache.spark.streaming.flume +import java.nio.charset.StandardCharsets +import java.util.{Collections, List => JList, Map => JMap} import java.util.concurrent._ -import java.util.{Map => JMap, Collections} import scala.collection.mutable.ArrayBuffer -import com.google.common.base.Charsets.UTF_8 -import org.apache.flume.event.EventBuilder import org.apache.flume.Context import org.apache.flume.channel.MemoryChannel import org.apache.flume.conf.Configurables +import org.apache.flume.event.EventBuilder -import org.apache.spark.streaming.flume.sink.{SparkSinkConfig, SparkSink} +import org.apache.spark.streaming.flume.sink.{SparkSink, SparkSinkConfig} /** * Share codes for Scala and Python unit tests @@ -116,16 +116,16 @@ private[flume] class PollingFlumeTestUtils { /** * Send data and wait until all data has been received */ - def sendDatAndEnsureAllDataHasBeenReceived(): Unit = { + def sendDataAndEnsureAllDataHasBeenReceived(): Unit = { val executor = Executors.newCachedThreadPool() val executorCompletion = new ExecutorCompletionService[Void](executor) val latch = new CountDownLatch(batchCount * channels.size) sinks.foreach(_.countdownWhenBatchReceived(latch)) - channels.foreach(channel => { + channels.foreach { channel => executorCompletion.submit(new TxnSubmitter(channel)) - }) + } for (i <- 0 until channels.size) { executorCompletion.take() @@ -137,7 +137,8 @@ private[flume] class PollingFlumeTestUtils { /** * A Python-friendly method to assert the output */ - def assertOutput(outputHeaders: Seq[JMap[String, String]], outputBodies: Seq[String]): Unit = { + def assertOutput( + outputHeaders: JList[JMap[String, String]], outputBodies: JList[String]): Unit = { require(outputHeaders.size == outputBodies.size) val eventSize = outputHeaders.size if (eventSize != totalEventsPerChannel * channels.size) { @@ -151,8 +152,8 @@ private[flume] class PollingFlumeTestUtils { var found = false var j = 0 while (j < eventSize && !found) { - if (eventBodyToVerify == outputBodies(j) && - eventHeaderToVerify == outputHeaders(j)) { + if (eventBodyToVerify == outputBodies.get(j) && + eventHeaderToVerify == outputHeaders.get(j)) { found = true counter += 1 } @@ -173,7 +174,7 @@ private[flume] class PollingFlumeTestUtils { val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") queueRemaining.setAccessible(true) val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") - if (m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] != 5000) { + if (m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] != channelCapacity) { throw new AssertionError(s"Channel ${channel.getName} is not empty") } } @@ -192,7 +193,8 @@ private[flume] class PollingFlumeTestUtils { val tx = channel.getTransaction tx.begin() for (j <- 0 until eventsPerBatch) { - channel.put(EventBuilder.withBody(s"${channel.getName}-$t".getBytes(UTF_8), + channel.put(EventBuilder.withBody( + s"${channel.getName}-$t".getBytes(StandardCharsets.UTF_8), Collections.singletonMap(s"test-$t", "header"))) t += 1 } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java index d31aa5f5c096..4a5da226aded 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java @@ -18,4 +18,4 @@ /** * Spark streaming receiver for Flume. */ -package org.apache.spark.streaming.flume; \ No newline at end of file +package org.apache.spark.streaming.flume; diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java index 3b5e0c7746b2..ada05f203b6a 100644 --- a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java @@ -27,10 +27,11 @@ public class JavaFlumeStreamSuite extends LocalJavaStreamingContext { @Test public void testFlumeStream() { // tests the API, does not actually test data receiving - JavaReceiverInputDStream test1 = FlumeUtils.createStream(ssc, "localhost", 12345); - JavaReceiverInputDStream test2 = FlumeUtils.createStream(ssc, "localhost", 12345, - StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaReceiverInputDStream test3 = FlumeUtils.createStream(ssc, "localhost", 12345, - StorageLevel.MEMORY_AND_DISK_SER_2(), false); + JavaReceiverInputDStream test1 = FlumeUtils.createStream(ssc, "localhost", + 12345); + JavaReceiverInputDStream test2 = FlumeUtils.createStream(ssc, "localhost", + 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaReceiverInputDStream test3 = FlumeUtils.createStream(ssc, "localhost", + 12345, StorageLevel.MEMORY_AND_DISK_SER_2(), false); } } diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties index 75e3b53a093f..fd51f8faf56b 100644 --- a/external/flume/src/test/resources/log4j.properties +++ b/external/flume/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN +log4j.logger.org.spark_project.jetty=WARN diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala b/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala index 1a900007b696..c97a27ca7c7a 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala @@ -18,14 +18,14 @@ package org.apache.spark.streaming import java.io.{IOException, ObjectInputStream} +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.reflect.ClassTag import org.apache.spark.rdd.RDD import org.apache.spark.streaming.dstream.{DStream, ForEachDStream} import org.apache.spark.util.Utils -import scala.collection.mutable.ArrayBuffer -import scala.reflect.ClassTag - /** * This is a output stream just for the testsuites. All the output is collected into a * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint. @@ -33,11 +33,11 @@ import scala.reflect.ClassTag * The buffer contains a sequence of RDD's, each containing a sequence of items */ class TestOutputStream[T: ClassTag](parent: DStream[T], - val output: ArrayBuffer[Seq[T]] = ArrayBuffer[Seq[T]]()) + val output: ConcurrentLinkedQueue[Seq[T]] = new ConcurrentLinkedQueue[Seq[T]]()) extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => { val collected = rdd.collect() - output += collected - }) { + output.add(collected) + }, false) { // This is to clear the output buffer every it is read from a checkpoint @throws(classOf[IOException]) diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index ff2fb8eed204..4324cc6d0f80 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -18,27 +18,30 @@ package org.apache.spark.streaming.flume import java.net.InetSocketAddress +import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.JavaConverters._ -import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} import scala.concurrent.duration._ import scala.language.postfixOps -import com.google.common.base.Charsets.UTF_8 -import org.scalatest.BeforeAndAfter +import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ -import org.apache.spark.{Logging, SparkConf, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.Logging +import org.apache.spark.network.util.JavaUtils import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Seconds, StreamingContext, TestOutputStream} import org.apache.spark.streaming.dstream.ReceiverInputDStream -import org.apache.spark.streaming.{Seconds, TestOutputStream, StreamingContext} import org.apache.spark.util.{ManualClock, Utils} -class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfter with Logging { +class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { val maxAttempts = 5 val batchDuration = Seconds(1) + @transient private var _sc: SparkContext = _ + val conf = new SparkConf() .setMaster("local[2]") .setAppName(this.getClass.getSimpleName) @@ -46,12 +49,23 @@ class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfter with Log val utils = new PollingFlumeTestUtils + override def beforeAll(): Unit = { + _sc = new SparkContext(conf) + } + + override def afterAll(): Unit = { + if (_sc != null) { + _sc.stop() + _sc = null + } + } + test("flume polling test") { - testMultipleTimes(testFlumePolling) + testMultipleTimes(() => testFlumePolling()) } test("flume polling test multiple hosts") { - testMultipleTimes(testFlumePollingMultipleHost) + testMultipleTimes(() => testFlumePollingMultipleHost()) } /** @@ -97,33 +111,33 @@ class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfter with Log def writeAndVerify(sinkPorts: Seq[Int]): Unit = { // Set up the streaming context and input streams - val ssc = new StreamingContext(conf, batchDuration) + val ssc = new StreamingContext(_sc, batchDuration) val addresses = sinkPorts.map(port => new InetSocketAddress("localhost", port)) val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, utils.eventsPerBatch, 5) - val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] - with SynchronizedBuffer[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream, outputBuffer) + val outputQueue = new ConcurrentLinkedQueue[Seq[SparkFlumeEvent]] + val outputStream = new TestOutputStream(flumeStream, outputQueue) outputStream.register() ssc.start() try { - utils.sendDatAndEnsureAllDataHasBeenReceived() + utils.sendDataAndEnsureAllDataHasBeenReceived() val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] clock.advance(batchDuration.milliseconds) // The eventually is required to ensure that all data in the batch has been processed. eventually(timeout(10 seconds), interval(100 milliseconds)) { - val flattenOutputBuffer = outputBuffer.flatten - val headers = flattenOutputBuffer.map(_.event.getHeaders.asScala.map { + val flattenOutput = outputQueue.asScala.toSeq.flatten + val headers = flattenOutput.map(_.event.getHeaders.asScala.map { case (key, value) => (key.toString, value.toString) }).map(_.asJava) - val bodies = flattenOutputBuffer.map(e => new String(e.event.getBody.array(), UTF_8)) - utils.assertOutput(headers, bodies) + val bodies = flattenOutput.map(e => JavaUtils.bytesToString(e.event.getBody)) + utils.assertOutput(headers.asJava, bodies.asJava) } } finally { - ssc.stop() + // here stop ssc only, but not underlying sparkcontext + ssc.stop(false) } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 5ffb60bd602f..7bac1cc4b0ae 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -17,12 +17,12 @@ package org.apache.spark.streaming.flume +import java.util.concurrent.ConcurrentLinkedQueue + import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import scala.concurrent.duration._ import scala.language.postfixOps -import com.google.common.base.Charsets import org.jboss.netty.channel.ChannelPipeline import org.jboss.netty.channel.socket.SocketChannel import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory @@ -30,7 +30,9 @@ import org.jboss.netty.handler.codec.compression._ import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually._ -import org.apache.spark.{Logging, SparkConf, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.Logging +import org.apache.spark.network.util.JavaUtils import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Milliseconds, StreamingContext, TestOutputStream} @@ -51,19 +53,19 @@ class FlumeStreamSuite extends SparkFunSuite with BeforeAndAfter with Matchers w val input = (1 to 100).map { _.toString } val utils = new FlumeTestUtils try { - val outputBuffer = startContext(utils.getTestPort(), testCompression) + val outputQueue = startContext(utils.getTestPort(), testCompression) eventually(timeout(10 seconds), interval(100 milliseconds)) { - utils.writeInput(input, testCompression) + utils.writeInput(input.asJava, testCompression) } eventually(timeout(10 seconds), interval(100 milliseconds)) { - val outputEvents = outputBuffer.flatten.map { _.event } + val outputEvents = outputQueue.asScala.toSeq.flatten.map { _.event } outputEvents.foreach { event => event.getHeaders.get("test") should be("header") } - val output = outputEvents.map(event => new String(event.getBody.array(), Charsets.UTF_8)) + val output = outputEvents.map(event => JavaUtils.bytesToString(event.getBody)) output should be (input) } } finally { @@ -76,16 +78,15 @@ class FlumeStreamSuite extends SparkFunSuite with BeforeAndAfter with Matchers w /** Setup and start the streaming context */ private def startContext( - testPort: Int, testCompression: Boolean): (ArrayBuffer[Seq[SparkFlumeEvent]]) = { + testPort: Int, testCompression: Boolean): (ConcurrentLinkedQueue[Seq[SparkFlumeEvent]]) = { ssc = new StreamingContext(conf, Milliseconds(200)) val flumeStream = FlumeUtils.createStream( ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK, testCompression) - val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] - with SynchronizedBuffer[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream, outputBuffer) + val outputQueue = new ConcurrentLinkedQueue[Seq[SparkFlumeEvent]] + val outputStream = new TestOutputStream(flumeStream, outputQueue) outputStream.register() ssc.start() - outputBuffer + outputQueue } /** Class to create socket channel with compression */ diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml new file mode 100644 index 000000000000..d6f97316b326 --- /dev/null +++ b/external/kafka-0-10-assembly/pom.xml @@ -0,0 +1,175 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-streaming-kafka-0-10-assembly_2.11 + jar + Spark Integration for Kafka 0.10 Assembly + http://spark.apache.org/ + + + streaming-kafka-0-10-assembly + + + + + org.apache.spark + spark-streaming-kafka-0-10_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + + commons-codec + commons-codec + provided + + + commons-lang + commons-lang + provided + + + com.google.protobuf + protobuf-java + provided + + + org.lz4 + lz4-java + provided + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.avro + avro-mapred + ${avro.mapred.classifier} + provided + + + org.apache.curator + curator-recipes + provided + + + org.apache.zookeeper + zookeeper + provided + + + log4j + log4j + provided + + + net.java.dev.jets3t + jets3t + provided + + + org.scala-lang + scala-library + provided + + + org.slf4j + slf4j-api + provided + + + org.slf4j + slf4j-log4j12 + provided + + + org.xerial.snappy + snappy-java + provided + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + log4j.properties + + + + + + + + + + + + diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml new file mode 100644 index 000000000000..0c9f0aa765a3 --- /dev/null +++ b/external/kafka-0-10-sql/pom.xml @@ -0,0 +1,120 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-sql-kafka-0-10_2.11 + + sql-kafka-0-10 + 0.10.0.1 + + jar + Kafka 0.10 Source for Structured Streaming + http://spark.apache.org/ + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.kafka + kafka-clients + ${kafka.version} + + + org.apache.kafka + kafka_${scala.binary.version} + ${kafka.version} + test + + + net.sf.jopt-simple + jopt-simple + 3.2 + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + + + scala-2.12 + + 0.10.1.1 + + + + + diff --git a/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister new file mode 100644 index 000000000000..2f9e9fc0396d --- /dev/null +++ b/external/kafka-0-10-sql/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -0,0 +1 @@ +org.apache.spark.sql.kafka010.KafkaSourceProvider diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala new file mode 100644 index 000000000000..90ed7b1fba2f --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala @@ -0,0 +1,438 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.{util => ju} +import java.util.concurrent.TimeoutException + +import scala.collection.JavaConverters._ + +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer, OffsetOutOfRangeException} +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.{SparkEnv, SparkException, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.kafka010.KafkaSource._ +import org.apache.spark.util.UninterruptibleThread + + +/** + * Consumer of single topicpartition, intended for cached reuse. + * Underlying consumer is not threadsafe, so neither is this, + * but processing the same topicpartition and group id in multiple threads is usually bad anyway. + */ +private[kafka010] case class CachedKafkaConsumer private( + topicPartition: TopicPartition, + kafkaParams: ju.Map[String, Object]) extends Logging { + import CachedKafkaConsumer._ + + private val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + + private var consumer = createConsumer + + /** indicates whether this consumer is in use or not */ + private var inuse = true + + /** Iterator to the already fetch data */ + private var fetchedData = ju.Collections.emptyIterator[ConsumerRecord[Array[Byte], Array[Byte]]] + private var nextOffsetInFetchedData = UNKNOWN_OFFSET + + /** Create a KafkaConsumer to fetch records for `topicPartition` */ + private def createConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = { + val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + val tps = new ju.ArrayList[TopicPartition]() + tps.add(topicPartition) + c.assign(tps) + c + } + + case class AvailableOffsetRange(earliest: Long, latest: Long) + + private def runUninterruptiblyIfPossible[T](body: => T): T = Thread.currentThread match { + case ut: UninterruptibleThread => + ut.runUninterruptibly(body) + case _ => + logWarning("CachedKafkaConsumer is not running in UninterruptibleThread. " + + "It may hang when CachedKafkaConsumer's methods are interrupted because of KAFKA-1894") + body + } + + /** + * Return the available offset range of the current partition. It's a pair of the earliest offset + * and the latest offset. + */ + def getAvailableOffsetRange(): AvailableOffsetRange = runUninterruptiblyIfPossible { + consumer.seekToBeginning(Set(topicPartition).asJava) + val earliestOffset = consumer.position(topicPartition) + consumer.seekToEnd(Set(topicPartition).asJava) + val latestOffset = consumer.position(topicPartition) + AvailableOffsetRange(earliestOffset, latestOffset) + } + + /** + * Get the record for the given offset if available. Otherwise it will either throw error + * (if failOnDataLoss = true), or return the next available offset within [offset, untilOffset), + * or null. + * + * @param offset the offset to fetch. + * @param untilOffset the max offset to fetch. Exclusive. + * @param pollTimeoutMs timeout in milliseconds to poll data from Kafka. + * @param failOnDataLoss When `failOnDataLoss` is `true`, this method will either return record at + * offset if available, or throw exception.when `failOnDataLoss` is `false`, + * this method will either return record at offset if available, or return + * the next earliest available record less than untilOffset, or null. It + * will not throw any exception. + */ + def get( + offset: Long, + untilOffset: Long, + pollTimeoutMs: Long, + failOnDataLoss: Boolean): + ConsumerRecord[Array[Byte], Array[Byte]] = runUninterruptiblyIfPossible { + require(offset < untilOffset, + s"offset must always be less than untilOffset [offset: $offset, untilOffset: $untilOffset]") + logDebug(s"Get $groupId $topicPartition nextOffset $nextOffsetInFetchedData requested $offset") + // The following loop is basically for `failOnDataLoss = false`. When `failOnDataLoss` is + // `false`, first, we will try to fetch the record at `offset`. If no such record exists, then + // we will move to the next available offset within `[offset, untilOffset)` and retry. + // If `failOnDataLoss` is `true`, the loop body will be executed only once. + var toFetchOffset = offset + var consumerRecord: ConsumerRecord[Array[Byte], Array[Byte]] = null + // We want to break out of the while loop on a successful fetch to avoid using "return" + // which may causes a NonLocalReturnControl exception when this method is used as a function. + var isFetchComplete = false + + while (toFetchOffset != UNKNOWN_OFFSET && !isFetchComplete) { + try { + consumerRecord = fetchData(toFetchOffset, untilOffset, pollTimeoutMs, failOnDataLoss) + isFetchComplete = true + } catch { + case e: OffsetOutOfRangeException => + // When there is some error thrown, it's better to use a new consumer to drop all cached + // states in the old consumer. We don't need to worry about the performance because this + // is not a common path. + resetConsumer() + reportDataLoss(failOnDataLoss, s"Cannot fetch offset $toFetchOffset", e) + toFetchOffset = getEarliestAvailableOffsetBetween(toFetchOffset, untilOffset) + } + } + + if (isFetchComplete) { + consumerRecord + } else { + resetFetchedData() + null + } + } + + /** + * Return the next earliest available offset in [offset, untilOffset). If all offsets in + * [offset, untilOffset) are invalid (e.g., the topic is deleted and recreated), it will return + * `UNKNOWN_OFFSET`. + */ + private def getEarliestAvailableOffsetBetween(offset: Long, untilOffset: Long): Long = { + val range = getAvailableOffsetRange() + logWarning(s"Some data may be lost. Recovering from the earliest offset: ${range.earliest}") + if (offset >= range.latest || range.earliest >= untilOffset) { + // [offset, untilOffset) and [earliestOffset, latestOffset) have no overlap, + // either + // -------------------------------------------------------- + // ^ ^ ^ ^ + // | | | | + // earliestOffset latestOffset offset untilOffset + // + // or + // -------------------------------------------------------- + // ^ ^ ^ ^ + // | | | | + // offset untilOffset earliestOffset latestOffset + val warningMessage = + s""" + |The current available offset range is $range. + | Offset ${offset} is out of range, and records in [$offset, $untilOffset) will be + | skipped ${additionalMessage(failOnDataLoss = false)} + """.stripMargin + logWarning(warningMessage) + UNKNOWN_OFFSET + } else if (offset >= range.earliest) { + // ----------------------------------------------------------------------------- + // ^ ^ ^ ^ + // | | | | + // earliestOffset offset min(untilOffset,latestOffset) max(untilOffset, latestOffset) + // + // This will happen when a topic is deleted and recreated, and new data are pushed very fast, + // then we will see `offset` disappears first then appears again. Although the parameters + // are same, the state in Kafka cluster is changed, so the outer loop won't be endless. + logWarning(s"Found a disappeared offset $offset. " + + s"Some data may be lost ${additionalMessage(failOnDataLoss = false)}") + offset + } else { + // ------------------------------------------------------------------------------ + // ^ ^ ^ ^ + // | | | | + // offset earliestOffset min(untilOffset,latestOffset) max(untilOffset, latestOffset) + val warningMessage = + s""" + |The current available offset range is $range. + | Offset ${offset} is out of range, and records in [$offset, ${range.earliest}) will be + | skipped ${additionalMessage(failOnDataLoss = false)} + """.stripMargin + logWarning(warningMessage) + range.earliest + } + } + + /** + * Get the record for the given offset if available. Otherwise it will either throw error + * (if failOnDataLoss = true), or return the next available offset within [offset, untilOffset), + * or null. + * + * @throws OffsetOutOfRangeException if `offset` is out of range + * @throws TimeoutException if cannot fetch the record in `pollTimeoutMs` milliseconds. + */ + private def fetchData( + offset: Long, + untilOffset: Long, + pollTimeoutMs: Long, + failOnDataLoss: Boolean): ConsumerRecord[Array[Byte], Array[Byte]] = { + if (offset != nextOffsetInFetchedData || !fetchedData.hasNext()) { + // This is the first fetch, or the last pre-fetched data has been drained. + // Seek to the offset because we may call seekToBeginning or seekToEnd before this. + seek(offset) + poll(pollTimeoutMs) + } + + if (!fetchedData.hasNext()) { + // We cannot fetch anything after `poll`. Two possible cases: + // - `offset` is out of range so that Kafka returns nothing. Just throw + // `OffsetOutOfRangeException` to let the caller handle it. + // - Cannot fetch any data before timeout. TimeoutException will be thrown. + val range = getAvailableOffsetRange() + if (offset < range.earliest || offset >= range.latest) { + throw new OffsetOutOfRangeException( + Map(topicPartition -> java.lang.Long.valueOf(offset)).asJava) + } else { + throw new TimeoutException( + s"Cannot fetch record for offset $offset in $pollTimeoutMs milliseconds") + } + } else { + val record = fetchedData.next() + nextOffsetInFetchedData = record.offset + 1 + // In general, Kafka uses the specified offset as the start point, and tries to fetch the next + // available offset. Hence we need to handle offset mismatch. + if (record.offset > offset) { + // This may happen when some records aged out but their offsets already got verified + if (failOnDataLoss) { + reportDataLoss(true, s"Cannot fetch records in [$offset, ${record.offset})") + // Never happen as "reportDataLoss" will throw an exception + null + } else { + if (record.offset >= untilOffset) { + reportDataLoss(false, s"Skip missing records in [$offset, $untilOffset)") + null + } else { + reportDataLoss(false, s"Skip missing records in [$offset, ${record.offset})") + record + } + } + } else if (record.offset < offset) { + // This should not happen. If it does happen, then we probably misunderstand Kafka internal + // mechanism. + throw new IllegalStateException( + s"Tried to fetch $offset but the returned record offset was ${record.offset}") + } else { + record + } + } + } + + /** Create a new consumer and reset cached states */ + private def resetConsumer(): Unit = { + consumer.close() + consumer = createConsumer + resetFetchedData() + } + + /** Reset the internal pre-fetched data. */ + private def resetFetchedData(): Unit = { + nextOffsetInFetchedData = UNKNOWN_OFFSET + fetchedData = ju.Collections.emptyIterator[ConsumerRecord[Array[Byte], Array[Byte]]] + } + + /** + * Return an addition message including useful message and instruction. + */ + private def additionalMessage(failOnDataLoss: Boolean): String = { + if (failOnDataLoss) { + s"(GroupId: $groupId, TopicPartition: $topicPartition). " + + s"$INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE" + } else { + s"(GroupId: $groupId, TopicPartition: $topicPartition). " + + s"$INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE" + } + } + + /** + * Throw an exception or log a warning as per `failOnDataLoss`. + */ + private def reportDataLoss( + failOnDataLoss: Boolean, + message: String, + cause: Throwable = null): Unit = { + val finalMessage = s"$message ${additionalMessage(failOnDataLoss)}" + reportDataLoss0(failOnDataLoss, finalMessage, cause) + } + + def close(): Unit = consumer.close() + + private def seek(offset: Long): Unit = { + logDebug(s"Seeking to $groupId $topicPartition $offset") + consumer.seek(topicPartition, offset) + } + + private def poll(pollTimeoutMs: Long): Unit = { + val p = consumer.poll(pollTimeoutMs) + val r = p.records(topicPartition) + logDebug(s"Polled $groupId ${p.partitions()} ${r.size}") + fetchedData = r.iterator + } +} + +private[kafka010] object CachedKafkaConsumer extends Logging { + + private val UNKNOWN_OFFSET = -2L + + private case class CacheKey(groupId: String, topicPartition: TopicPartition) + + private lazy val cache = { + val conf = SparkEnv.get.conf + val capacity = conf.getInt("spark.sql.kafkaConsumerCache.capacity", 64) + new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer](capacity, 0.75f, true) { + override def removeEldestEntry( + entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer]): Boolean = { + if (entry.getValue.inuse == false && this.size > capacity) { + logWarning(s"KafkaConsumer cache hitting max capacity of $capacity, " + + s"removing consumer for ${entry.getKey}") + try { + entry.getValue.close() + } catch { + case e: SparkException => + logError(s"Error closing earliest Kafka consumer for ${entry.getKey}", e) + } + true + } else { + false + } + } + } + } + + def releaseKafkaConsumer( + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): Unit = { + val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + val topicPartition = new TopicPartition(topic, partition) + val key = CacheKey(groupId, topicPartition) + + synchronized { + val consumer = cache.get(key) + if (consumer != null) { + consumer.inuse = false + } else { + logWarning(s"Attempting to release consumer that does not exist") + } + } + } + + /** + * Removes (and closes) the Kafka Consumer for the given topic, partition and group id. + */ + def removeKafkaConsumer( + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): Unit = { + val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + val topicPartition = new TopicPartition(topic, partition) + val key = CacheKey(groupId, topicPartition) + + synchronized { + val removedConsumer = cache.remove(key) + if (removedConsumer != null) { + removedConsumer.close() + } + } + } + + /** + * Get a cached consumer for groupId, assigned to topic and partition. + * If matching consumer doesn't already exist, will be created using kafkaParams. + */ + def getOrCreate( + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer = synchronized { + val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + val topicPartition = new TopicPartition(topic, partition) + val key = CacheKey(groupId, topicPartition) + + // If this is reattempt at running the task, then invalidate cache and start with + // a new consumer + if (TaskContext.get != null && TaskContext.get.attemptNumber >= 1) { + removeKafkaConsumer(topic, partition, kafkaParams) + val consumer = new CachedKafkaConsumer(topicPartition, kafkaParams) + consumer.inuse = true + cache.put(key, consumer) + consumer + } else { + if (!cache.containsKey(key)) { + cache.put(key, new CachedKafkaConsumer(topicPartition, kafkaParams)) + } + val consumer = cache.get(key) + consumer.inuse = true + consumer + } + } + + /** Create an [[CachedKafkaConsumer]] but don't put it into cache. */ + def createUncached( + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer = { + new CachedKafkaConsumer(new TopicPartition(topic, partition), kafkaParams) + } + + private def reportDataLoss0( + failOnDataLoss: Boolean, + finalMessage: String, + cause: Throwable = null): Unit = { + if (failOnDataLoss) { + if (cause != null) { + throw new IllegalStateException(finalMessage, cause) + } else { + throw new IllegalStateException(finalMessage) + } + } else { + if (cause != null) { + logWarning(finalMessage, cause) + } else { + logWarning(finalMessage) + } + } + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala new file mode 100644 index 000000000000..571140b0afbc --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.{util => ju} +import java.util.concurrent.{ConcurrentMap, ExecutionException, TimeUnit} + +import com.google.common.cache._ +import com.google.common.util.concurrent.{ExecutionError, UncheckedExecutionException} +import org.apache.kafka.clients.producer.KafkaProducer +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +import org.apache.spark.SparkEnv +import org.apache.spark.internal.Logging + +private[kafka010] object CachedKafkaProducer extends Logging { + + private type Producer = KafkaProducer[Array[Byte], Array[Byte]] + + private lazy val cacheExpireTimeout: Long = + SparkEnv.get.conf.getTimeAsMs("spark.kafka.producer.cache.timeout", "10m") + + private val cacheLoader = new CacheLoader[Seq[(String, Object)], Producer] { + override def load(config: Seq[(String, Object)]): Producer = { + val configMap = config.map(x => x._1 -> x._2).toMap.asJava + createKafkaProducer(configMap) + } + } + + private val removalListener = new RemovalListener[Seq[(String, Object)], Producer]() { + override def onRemoval( + notification: RemovalNotification[Seq[(String, Object)], Producer]): Unit = { + val paramsSeq: Seq[(String, Object)] = notification.getKey + val producer: Producer = notification.getValue + logDebug( + s"Evicting kafka producer $producer params: $paramsSeq, due to ${notification.getCause}") + close(paramsSeq, producer) + } + } + + private lazy val guavaCache: LoadingCache[Seq[(String, Object)], Producer] = + CacheBuilder.newBuilder().expireAfterAccess(cacheExpireTimeout, TimeUnit.MILLISECONDS) + .removalListener(removalListener) + .build[Seq[(String, Object)], Producer](cacheLoader) + + private def createKafkaProducer(producerConfiguration: ju.Map[String, Object]): Producer = { + val kafkaProducer: Producer = new Producer(producerConfiguration) + logDebug(s"Created a new instance of KafkaProducer for $producerConfiguration.") + kafkaProducer + } + + /** + * Get a cached KafkaProducer for a given configuration. If matching KafkaProducer doesn't + * exist, a new KafkaProducer will be created. KafkaProducer is thread safe, it is best to keep + * one instance per specified kafkaParams. + */ + private[kafka010] def getOrCreate(kafkaParams: ju.Map[String, Object]): Producer = { + val paramsSeq: Seq[(String, Object)] = paramsToSeq(kafkaParams) + try { + guavaCache.get(paramsSeq) + } catch { + case e @ (_: ExecutionException | _: UncheckedExecutionException | _: ExecutionError) + if e.getCause != null => + throw e.getCause + } + } + + private def paramsToSeq(kafkaParams: ju.Map[String, Object]): Seq[(String, Object)] = { + val paramsSeq: Seq[(String, Object)] = kafkaParams.asScala.toSeq.sortBy(x => x._1) + paramsSeq + } + + /** For explicitly closing kafka producer */ + private[kafka010] def close(kafkaParams: ju.Map[String, Object]): Unit = { + val paramsSeq = paramsToSeq(kafkaParams) + guavaCache.invalidate(paramsSeq) + } + + /** Auto close on cache evict */ + private def close(paramsSeq: Seq[(String, Object)], producer: Producer): Unit = { + try { + logInfo(s"Closing the KafkaProducer with params: ${paramsSeq.mkString("\n")}.") + producer.close() + } catch { + case NonFatal(e) => logWarning("Error while closing kafka producer.", e) + } + } + + private def clear(): Unit = { + logInfo("Cleaning up guava cache.") + guavaCache.invalidateAll() + } + + // Intended for testing purpose only. + private def getAsMap: ConcurrentMap[Seq[(String, Object)], Producer] = guavaCache.asMap() +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala new file mode 100644 index 000000000000..66511b306541 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.{util => ju} + +import scala.collection.JavaConverters._ + +import org.apache.kafka.clients.consumer.{Consumer, KafkaConsumer} +import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener +import org.apache.kafka.common.TopicPartition + +/** + * Subscribe allows you to subscribe to a fixed collection of topics. + * SubscribePattern allows you to use a regex to specify topics of interest. + * Note that unlike the 0.8 integration, using Subscribe or SubscribePattern + * should respond to adding partitions during a running stream. + * Finally, Assign allows you to specify a fixed collection of partitions. + * All three strategies have overloaded constructors that allow you to specify + * the starting offset for a particular partition. + */ +sealed trait ConsumerStrategy { + /** Create a [[KafkaConsumer]] and subscribe to topics according to a desired strategy */ + def createConsumer(kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] +} + +/** + * Specify a fixed collection of partitions. + */ +case class AssignStrategy(partitions: Array[TopicPartition]) extends ConsumerStrategy { + override def createConsumer( + kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.assign(ju.Arrays.asList(partitions: _*)) + consumer + } + + override def toString: String = s"Assign[${partitions.mkString(", ")}]" +} + +/** + * Subscribe to a fixed collection of topics. + */ +case class SubscribeStrategy(topics: Seq[String]) extends ConsumerStrategy { + override def createConsumer( + kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.subscribe(topics.asJava) + consumer + } + + override def toString: String = s"Subscribe[${topics.mkString(", ")}]" +} + +/** + * Use a regex to specify topics of interest. + */ +case class SubscribePatternStrategy(topicPattern: String) extends ConsumerStrategy { + override def createConsumer( + kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + consumer.subscribe( + ju.regex.Pattern.compile(topicPattern), + new NoOpConsumerRebalanceListener()) + consumer + } + + override def toString: String = s"SubscribePattern[$topicPattern]" +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala new file mode 100644 index 000000000000..868edb5dcdc0 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/JsonUtils.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import scala.collection.mutable.HashMap +import scala.util.control.NonFatal + +import org.apache.kafka.common.TopicPartition +import org.json4s.NoTypeHints +import org.json4s.jackson.Serialization + +/** + * Utilities for converting Kafka related objects to and from json. + */ +private object JsonUtils { + private implicit val formats = Serialization.formats(NoTypeHints) + + /** + * Read TopicPartitions from json string + */ + def partitions(str: String): Array[TopicPartition] = { + try { + Serialization.read[Map[String, Seq[Int]]](str).flatMap { case (topic, parts) => + parts.map { part => + new TopicPartition(topic, part) + } + }.toArray + } catch { + case NonFatal(x) => + throw new IllegalArgumentException( + s"""Expected e.g. {"topicA":[0,1],"topicB":[0,1]}, got $str""") + } + } + + /** + * Write TopicPartitions as json string + */ + def partitions(partitions: Iterable[TopicPartition]): String = { + val result = new HashMap[String, List[Int]] + partitions.foreach { tp => + val parts: List[Int] = result.getOrElse(tp.topic, Nil) + result += tp.topic -> (tp.partition::parts) + } + Serialization.write(result) + } + + /** + * Read per-TopicPartition offsets from json string + */ + def partitionOffsets(str: String): Map[TopicPartition, Long] = { + try { + Serialization.read[Map[String, Map[Int, Long]]](str).flatMap { case (topic, partOffsets) => + partOffsets.map { case (part, offset) => + new TopicPartition(topic, part) -> offset + } + }.toMap + } catch { + case NonFatal(x) => + throw new IllegalArgumentException( + s"""Expected e.g. {"topicA":{"0":23,"1":-1},"topicB":{"0":-2}}, got $str""") + } + } + + /** + * Write per-TopicPartition offsets as json string + */ + def partitionOffsets(partitionOffsets: Map[TopicPartition, Long]): String = { + val result = new HashMap[String, HashMap[Int, Long]]() + implicit val ordering = new Ordering[TopicPartition] { + override def compare(x: TopicPartition, y: TopicPartition): Int = { + Ordering.Tuple2[String, Int].compare((x.topic, x.partition), (y.topic, y.partition)) + } + } + val partitions = partitionOffsets.keySet.toSeq.sorted // sort for more determinism + partitions.foreach { tp => + val off = partitionOffsets(tp) + val parts = result.getOrElse(tp.topic, new HashMap[Int, Long]) + parts += tp.partition -> off + result += tp.topic -> parts + } + Serialization.write(result) + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala new file mode 100644 index 000000000000..80a026f4f5d7 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeLimit.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import org.apache.kafka.common.TopicPartition + +/** + * Objects that represent desired offset range limits for starting, + * ending, and specific offsets. + */ +private[kafka010] sealed trait KafkaOffsetRangeLimit + +/** + * Represents the desire to bind to the earliest offsets in Kafka + */ +private[kafka010] case object EarliestOffsetRangeLimit extends KafkaOffsetRangeLimit + +/** + * Represents the desire to bind to the latest offsets in Kafka + */ +private[kafka010] case object LatestOffsetRangeLimit extends KafkaOffsetRangeLimit + +/** + * Represents the desire to bind to specific offsets. A offset == -1 binds to the + * latest offset, and offset == -2 binds to the earliest offset. + */ +private[kafka010] case class SpecificOffsetRangeLimit( + partitionOffsets: Map[TopicPartition, Long]) extends KafkaOffsetRangeLimit + +private[kafka010] object KafkaOffsetRangeLimit { + /** + * Used to denote offset range limits that are resolved via Kafka + */ + val LATEST = -1L // indicates resolution to the latest offset + val EARLIEST = -2L // indicates resolution to the earliest offset +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala new file mode 100644 index 000000000000..3e65949a6fd1 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.{util => ju} +import java.util.concurrent.{Executors, ThreadFactory} + +import scala.collection.JavaConverters._ +import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration.Duration +import scala.util.control.NonFatal + +import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer} +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.types._ +import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} + +/** + * This class uses Kafka's own [[KafkaConsumer]] API to read data offsets from Kafka. + * The [[ConsumerStrategy]] class defines which Kafka topics and partitions should be read + * by this source. These strategies directly correspond to the different consumption options + * in. This class is designed to return a configured [[KafkaConsumer]] that is used by the + * [[KafkaSource]] to query for the offsets. See the docs on + * [[org.apache.spark.sql.kafka010.ConsumerStrategy]] + * for more details. + * + * Note: This class is not ThreadSafe + */ +private[kafka010] class KafkaOffsetReader( + consumerStrategy: ConsumerStrategy, + driverKafkaParams: ju.Map[String, Object], + readerOptions: Map[String, String], + driverGroupIdPrefix: String) extends Logging { + /** + * Used to ensure execute fetch operations execute in an UninterruptibleThread + */ + val kafkaReaderThread = Executors.newSingleThreadExecutor(new ThreadFactory { + override def newThread(r: Runnable): Thread = { + val t = new UninterruptibleThread("Kafka Offset Reader") { + override def run(): Unit = { + r.run() + } + } + t.setDaemon(true) + t + } + }) + val execContext = ExecutionContext.fromExecutorService(kafkaReaderThread) + + /** + * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is + * created -- see SPARK-19564. + */ + private var groupId: String = null + private var nextId = 0 + + /** + * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the + * offsets and never commits them. + */ + protected var consumer = createConsumer() + + private val maxOffsetFetchAttempts = + readerOptions.getOrElse("fetchOffset.numRetries", "3").toInt + + private val offsetFetchAttemptIntervalMs = + readerOptions.getOrElse("fetchOffset.retryIntervalMs", "1000").toLong + + private def nextGroupId(): String = { + groupId = driverGroupIdPrefix + "-" + nextId + nextId += 1 + groupId + } + + override def toString(): String = consumerStrategy.toString + + /** + * Closes the connection to Kafka, and cleans up state. + */ + def close(): Unit = { + runUninterruptibly { + consumer.close() + } + kafkaReaderThread.shutdown() + } + + /** + * @return The Set of TopicPartitions for a given topic + */ + def fetchTopicPartitions(): Set[TopicPartition] = runUninterruptibly { + assert(Thread.currentThread().isInstanceOf[UninterruptibleThread]) + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + partitions.asScala.toSet + } + + /** + * Resolves the specific offsets based on Kafka seek positions. + * This method resolves offset value -1 to the latest and -2 to the + * earliest Kafka seek position. + */ + def fetchSpecificOffsets( + partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = + runUninterruptibly { + withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + assert(partitions.asScala == partitionOffsets.keySet, + "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + + "Use -1 for latest, -2 for earliest, if you don't care.\n" + + s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions.asScala}") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") + + partitionOffsets.foreach { + case (tp, KafkaOffsetRangeLimit.LATEST) => + consumer.seekToEnd(ju.Arrays.asList(tp)) + case (tp, KafkaOffsetRangeLimit.EARLIEST) => + consumer.seekToBeginning(ju.Arrays.asList(tp)) + case (tp, off) => consumer.seek(tp, off) + } + partitionOffsets.map { + case (tp, _) => tp -> consumer.position(tp) + } + } + } + + /** + * Fetch the earliest offsets for the topic partitions that are indicated + * in the [[ConsumerStrategy]]. + */ + def fetchEarliestOffsets(): Map[TopicPartition, Long] = runUninterruptibly { + withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the beginning") + + consumer.seekToBeginning(partitions) + val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got earliest offsets for partition : $partitionOffsets") + partitionOffsets + } + } + + /** + * Fetch the latest offsets for the topic partitions that are indicated + * in the [[ConsumerStrategy]]. + */ + def fetchLatestOffsets(): Map[TopicPartition, Long] = runUninterruptibly { + withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the end.") + + consumer.seekToEnd(partitions) + val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got latest offsets for partition : $partitionOffsets") + partitionOffsets + } + } + + /** + * Fetch the earliest offsets for specific topic partitions. + * The return result may not contain some partitions if they are deleted. + */ + def fetchEarliestOffsets( + newPartitions: Seq[TopicPartition]): Map[TopicPartition, Long] = { + if (newPartitions.isEmpty) { + Map.empty[TopicPartition, Long] + } else { + runUninterruptibly { + withRetriesWithoutInterrupt { + // Poll to get the latest assigned partitions + consumer.poll(0) + val partitions = consumer.assignment() + consumer.pause(partitions) + logDebug(s"\tPartitions assigned to consumer: $partitions") + + // Get the earliest offset of each partition + consumer.seekToBeginning(partitions) + val partitionOffsets = newPartitions.filter { p => + // When deleting topics happen at the same time, some partitions may not be in + // `partitions`. So we need to ignore them + partitions.contains(p) + }.map(p => p -> consumer.position(p)).toMap + logDebug(s"Got earliest offsets for new partitions: $partitionOffsets") + partitionOffsets + } + } + } + } + + /** + * This method ensures that the closure is called in an [[UninterruptibleThread]]. + * This is required when communicating with the [[KafkaConsumer]]. In the case + * of streaming queries, we are already running in an [[UninterruptibleThread]], + * however for batch mode this is not the case. + */ + private def runUninterruptibly[T](body: => T): T = { + if (!Thread.currentThread.isInstanceOf[UninterruptibleThread]) { + val future = Future { + body + }(execContext) + ThreadUtils.awaitResult(future, Duration.Inf) + } else { + body + } + } + + /** + * Helper function that does multiple retries on a body of code that returns offsets. + * Retries are needed to handle transient failures. For e.g. race conditions between getting + * assignment and getting position while topics/partitions are deleted can cause NPEs. + * + * This method also makes sure `body` won't be interrupted to workaround a potential issue in + * `KafkaConsumer.poll`. (KAFKA-1894) + */ + private def withRetriesWithoutInterrupt( + body: => Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + // Make sure `KafkaConsumer.poll` won't be interrupted (KAFKA-1894) + assert(Thread.currentThread().isInstanceOf[UninterruptibleThread]) + + synchronized { + var result: Option[Map[TopicPartition, Long]] = None + var attempt = 1 + var lastException: Throwable = null + while (result.isEmpty && attempt <= maxOffsetFetchAttempts + && !Thread.currentThread().isInterrupted) { + Thread.currentThread match { + case ut: UninterruptibleThread => + // "KafkaConsumer.poll" may hang forever if the thread is interrupted (E.g., the query + // is stopped)(KAFKA-1894). Hence, we just make sure we don't interrupt it. + // + // If the broker addresses are wrong, or Kafka cluster is down, "KafkaConsumer.poll" may + // hang forever as well. This cannot be resolved in KafkaSource until Kafka fixes the + // issue. + ut.runUninterruptibly { + try { + result = Some(body) + } catch { + case NonFatal(e) => + lastException = e + logWarning(s"Error in attempt $attempt getting Kafka offsets: ", e) + attempt += 1 + Thread.sleep(offsetFetchAttemptIntervalMs) + resetConsumer() + } + } + case _ => + throw new IllegalStateException( + "Kafka APIs must be executed on a o.a.spark.util.UninterruptibleThread") + } + } + if (Thread.interrupted()) { + throw new InterruptedException() + } + if (result.isEmpty) { + assert(attempt > maxOffsetFetchAttempts) + assert(lastException != null) + throw lastException + } + result.get + } + } + + /** + * Create a consumer using the new generated group id. We always use a new consumer to avoid + * just using a broken consumer to retry on Kafka errors, which likely will fail again. + */ + private def createConsumer(): Consumer[Array[Byte], Array[Byte]] = synchronized { + val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams) + newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId()) + consumerStrategy.createConsumer(newKafkaParams) + } + + private def resetConsumer(): Unit = synchronized { + consumer.close() + consumer = createConsumer() + } +} + +private[kafka010] object KafkaOffsetReader { + + def kafkaSchema: StructType = StructType(Seq( + StructField("key", BinaryType), + StructField("value", BinaryType), + StructField("topic", StringType), + StructField("partition", IntegerType), + StructField("offset", LongType), + StructField("timestamp", TimestampType), + StructField("timestampType", IntegerType) + )) +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala new file mode 100644 index 000000000000..7103709969c1 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.{util => ju} +import java.util.UUID + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.sources.{BaseRelation, TableScan} +import org.apache.spark.sql.types.StructType +import org.apache.spark.unsafe.types.UTF8String + + +private[kafka010] class KafkaRelation( + override val sqlContext: SQLContext, + strategy: ConsumerStrategy, + sourceOptions: Map[String, String], + specifiedKafkaParams: Map[String, String], + failOnDataLoss: Boolean, + startingOffsets: KafkaOffsetRangeLimit, + endingOffsets: KafkaOffsetRangeLimit) + extends BaseRelation with TableScan with Logging { + assert(startingOffsets != LatestOffsetRangeLimit, + "Starting offset not allowed to be set to latest offsets.") + assert(endingOffsets != EarliestOffsetRangeLimit, + "Ending offset not allowed to be set to earliest offsets.") + + private val pollTimeoutMs = sourceOptions.getOrElse( + "kafkaConsumer.pollTimeoutMs", + sqlContext.sparkContext.conf.getTimeAsMs("spark.network.timeout", "120s").toString + ).toLong + + override def schema: StructType = KafkaOffsetReader.kafkaSchema + + override def buildScan(): RDD[Row] = { + // Each running query should use its own group id. Otherwise, the query may be only assigned + // partial data since Kafka will assign partitions to multiple consumers having the same group + // id. Hence, we should generate a unique id for each query. + val uniqueGroupId = s"spark-kafka-relation-${UUID.randomUUID}" + + val kafkaOffsetReader = new KafkaOffsetReader( + strategy, + KafkaSourceProvider.kafkaParamsForDriver(specifiedKafkaParams), + sourceOptions, + driverGroupIdPrefix = s"$uniqueGroupId-driver") + + // Leverage the KafkaReader to obtain the relevant partition offsets + val (fromPartitionOffsets, untilPartitionOffsets) = { + try { + (getPartitionOffsets(kafkaOffsetReader, startingOffsets), + getPartitionOffsets(kafkaOffsetReader, endingOffsets)) + } finally { + kafkaOffsetReader.close() + } + } + + // Obtain topicPartitions in both from and until partition offset, ignoring + // topic partitions that were added and/or deleted between the two above calls. + if (fromPartitionOffsets.keySet != untilPartitionOffsets.keySet) { + implicit val topicOrdering: Ordering[TopicPartition] = Ordering.by(t => t.topic()) + val fromTopics = fromPartitionOffsets.keySet.toList.sorted.mkString(",") + val untilTopics = untilPartitionOffsets.keySet.toList.sorted.mkString(",") + throw new IllegalStateException("different topic partitions " + + s"for starting offsets topics[${fromTopics}] and " + + s"ending offsets topics[${untilTopics}]") + } + + // Calculate offset ranges + val offsetRanges = untilPartitionOffsets.keySet.map { tp => + val fromOffset = fromPartitionOffsets.get(tp).getOrElse { + // This should not happen since topicPartitions contains all partitions not in + // fromPartitionOffsets + throw new IllegalStateException(s"$tp doesn't have a from offset") + } + val untilOffset = untilPartitionOffsets(tp) + KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, None) + }.toArray + + logInfo("GetBatch generating RDD of offset range: " + + offsetRanges.sortBy(_.topicPartition.toString).mkString(", ")) + + // Create an RDD that reads from Kafka and get the (key, value) pair as byte arrays. + val executorKafkaParams = + KafkaSourceProvider.kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId) + val rdd = new KafkaSourceRDD( + sqlContext.sparkContext, executorKafkaParams, offsetRanges, + pollTimeoutMs, failOnDataLoss, reuseKafkaConsumer = false).map { cr => + InternalRow( + cr.key, + cr.value, + UTF8String.fromString(cr.topic), + cr.partition, + cr.offset, + DateTimeUtils.fromJavaTimestamp(new java.sql.Timestamp(cr.timestamp)), + cr.timestampType.id) + } + sqlContext.internalCreateDataFrame(rdd, schema).rdd + } + + private def getPartitionOffsets( + kafkaReader: KafkaOffsetReader, + kafkaOffsets: KafkaOffsetRangeLimit): Map[TopicPartition, Long] = { + def validateTopicPartitions(partitions: Set[TopicPartition], + partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + assert(partitions == partitionOffsets.keySet, + "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + + "Use -1 for latest, -2 for earliest, if you don't care.\n" + + s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions}") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") + partitionOffsets + } + val partitions = kafkaReader.fetchTopicPartitions() + // Obtain TopicPartition offsets with late binding support + kafkaOffsets match { + case EarliestOffsetRangeLimit => partitions.map { + case tp => tp -> KafkaOffsetRangeLimit.EARLIEST + }.toMap + case LatestOffsetRangeLimit => partitions.map { + case tp => tp -> KafkaOffsetRangeLimit.LATEST + }.toMap + case SpecificOffsetRangeLimit(partitionOffsets) => + validateTopicPartitions(partitions, partitionOffsets) + } + } + + override def toString: String = + s"KafkaRelation(strategy=$strategy, start=$startingOffsets, end=$endingOffsets)" +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala new file mode 100644 index 000000000000..08914d82fffd --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSink.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.{util => ju} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.execution.streaming.Sink + +private[kafka010] class KafkaSink( + sqlContext: SQLContext, + executorKafkaParams: ju.Map[String, Object], + topic: Option[String]) extends Sink with Logging { + @volatile private var latestBatchId = -1L + + override def toString(): String = "KafkaSink" + + override def addBatch(batchId: Long, data: DataFrame): Unit = { + if (batchId <= latestBatchId) { + logInfo(s"Skipping already committed batch $batchId") + } else { + KafkaWriter.write(sqlContext.sparkSession, + data.queryExecution, executorKafkaParams, topic) + latestBatchId = batchId + } + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala new file mode 100644 index 000000000000..e9cff04ba5f2 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -0,0 +1,368 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.{util => ju} +import java.io._ +import java.nio.charset.StandardCharsets + +import org.apache.commons.io.IOUtils +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.ExecutorCacheTaskLocation +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.kafka010.KafkaSource._ +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +/** + * A [[Source]] that reads data from Kafka using the following design. + * + * - The [[KafkaSourceOffset]] is the custom [[Offset]] defined for this source that contains + * a map of TopicPartition -> offset. Note that this offset is 1 + (available offset). For + * example if the last record in a Kafka topic "t", partition 2 is offset 5, then + * KafkaSourceOffset will contain TopicPartition("t", 2) -> 6. This is done keep it consistent + * with the semantics of `KafkaConsumer.position()`. + * + * - The [[KafkaSource]] written to do the following. + * + * - As soon as the source is created, the pre-configured [[KafkaOffsetReader]] + * is used to query the initial offsets that this source should + * start reading from. This is used to create the first batch. + * + * - `getOffset()` uses the [[KafkaOffsetReader]] to query the latest + * available offsets, which are returned as a [[KafkaSourceOffset]]. + * + * - `getBatch()` returns a DF that reads from the 'start offset' until the 'end offset' in + * for each partition. The end offset is excluded to be consistent with the semantics of + * [[KafkaSourceOffset]] and `KafkaConsumer.position()`. + * + * - The DF returned is based on [[KafkaSourceRDD]] which is constructed such that the + * data from Kafka topic + partition is consistently read by the same executors across + * batches, and cached KafkaConsumers in the executors can be reused efficiently. See the + * docs on [[KafkaSourceRDD]] for more details. + * + * Zero data lost is not guaranteed when topics are deleted. If zero data lost is critical, the user + * must make sure all messages in a topic have been processed when deleting a topic. + * + * There is a known issue caused by KAFKA-1894: the query using KafkaSource maybe cannot be stopped. + * To avoid this issue, you should make sure stopping the query before stopping the Kafka brokers + * and not use wrong broker addresses. + */ +private[kafka010] class KafkaSource( + sqlContext: SQLContext, + kafkaReader: KafkaOffsetReader, + executorKafkaParams: ju.Map[String, Object], + sourceOptions: Map[String, String], + metadataPath: String, + startingOffsets: KafkaOffsetRangeLimit, + failOnDataLoss: Boolean) + extends Source with Logging { + + private val sc = sqlContext.sparkContext + + private val pollTimeoutMs = sourceOptions.getOrElse( + "kafkaConsumer.pollTimeoutMs", + sc.conf.getTimeAsMs("spark.network.timeout", "120s").toString + ).toLong + + private val maxOffsetsPerTrigger = + sourceOptions.get("maxOffsetsPerTrigger").map(_.toLong) + + /** + * Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only + * called in StreamExecutionThread. Otherwise, interrupting a thread while running + * `KafkaConsumer.poll` may hang forever (KAFKA-1894). + */ + private lazy val initialPartitionOffsets = { + val metadataLog = + new HDFSMetadataLog[KafkaSourceOffset](sqlContext.sparkSession, metadataPath) { + override def serialize(metadata: KafkaSourceOffset, out: OutputStream): Unit = { + out.write(0) // A zero byte is written to support Spark 2.1.0 (SPARK-19517) + val writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8)) + writer.write("v" + VERSION + "\n") + writer.write(metadata.json) + writer.flush + } + + override def deserialize(in: InputStream): KafkaSourceOffset = { + in.read() // A zero byte is read to support Spark 2.1.0 (SPARK-19517) + val content = IOUtils.toString(new InputStreamReader(in, StandardCharsets.UTF_8)) + // HDFSMetadataLog guarantees that it never creates a partial file. + assert(content.length != 0) + if (content(0) == 'v') { + val indexOfNewLine = content.indexOf("\n") + if (indexOfNewLine > 0) { + val version = parseVersion(content.substring(0, indexOfNewLine), VERSION) + KafkaSourceOffset(SerializedOffset(content.substring(indexOfNewLine + 1))) + } else { + throw new IllegalStateException( + s"Log file was malformed: failed to detect the log file version line.") + } + } else { + // The log was generated by Spark 2.1.0 + KafkaSourceOffset(SerializedOffset(content)) + } + } + } + + metadataLog.get(0).getOrElse { + val offsets = startingOffsets match { + case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchEarliestOffsets()) + case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets()) + case SpecificOffsetRangeLimit(p) => fetchAndVerify(p) + } + metadataLog.add(0, offsets) + logInfo(s"Initial offsets: $offsets") + offsets + }.partitionToOffsets + } + + private def fetchAndVerify(specificOffsets: Map[TopicPartition, Long]) = { + val result = kafkaReader.fetchSpecificOffsets(specificOffsets) + specificOffsets.foreach { + case (tp, off) if off != KafkaOffsetRangeLimit.LATEST && + off != KafkaOffsetRangeLimit.EARLIEST => + if (result(tp) != off) { + reportDataLoss( + s"startingOffsets for $tp was $off but consumer reset to ${result(tp)}") + } + case _ => + // no real way to check that beginning or end is reasonable + } + KafkaSourceOffset(result) + } + + private var currentPartitionOffsets: Option[Map[TopicPartition, Long]] = None + + override def schema: StructType = KafkaOffsetReader.kafkaSchema + + /** Returns the maximum available offset for this source. */ + override def getOffset: Option[Offset] = { + // Make sure initialPartitionOffsets is initialized + initialPartitionOffsets + + val latest = kafkaReader.fetchLatestOffsets() + val offsets = maxOffsetsPerTrigger match { + case None => + latest + case Some(limit) if currentPartitionOffsets.isEmpty => + rateLimit(limit, initialPartitionOffsets, latest) + case Some(limit) => + rateLimit(limit, currentPartitionOffsets.get, latest) + } + + currentPartitionOffsets = Some(offsets) + logDebug(s"GetOffset: ${offsets.toSeq.map(_.toString).sorted}") + Some(KafkaSourceOffset(offsets)) + } + + /** Proportionally distribute limit number of offsets among topicpartitions */ + private def rateLimit( + limit: Long, + from: Map[TopicPartition, Long], + until: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + val fromNew = kafkaReader.fetchEarliestOffsets(until.keySet.diff(from.keySet).toSeq) + val sizes = until.flatMap { + case (tp, end) => + // If begin isn't defined, something's wrong, but let alert logic in getBatch handle it + from.get(tp).orElse(fromNew.get(tp)).flatMap { begin => + val size = end - begin + logDebug(s"rateLimit $tp size is $size") + if (size > 0) Some(tp -> size) else None + } + } + val total = sizes.values.sum.toDouble + if (total < 1) { + until + } else { + until.map { + case (tp, end) => + tp -> sizes.get(tp).map { size => + val begin = from.get(tp).getOrElse(fromNew(tp)) + val prorate = limit * (size / total) + logDebug(s"rateLimit $tp prorated amount is $prorate") + // Don't completely starve small topicpartitions + val off = begin + (if (prorate < 1) Math.ceil(prorate) else Math.floor(prorate)).toLong + logDebug(s"rateLimit $tp new offset is $off") + // Paranoia, make sure not to return an offset that's past end + Math.min(end, off) + }.getOrElse(end) + } + } + } + + /** + * Returns the data that is between the offsets + * [`start.get.partitionToOffsets`, `end.partitionToOffsets`), i.e. end.partitionToOffsets is + * exclusive. + */ + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + // Make sure initialPartitionOffsets is initialized + initialPartitionOffsets + + logInfo(s"GetBatch called with start = $start, end = $end") + val untilPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(end) + val fromPartitionOffsets = start match { + case Some(prevBatchEndOffset) => + KafkaSourceOffset.getPartitionOffsets(prevBatchEndOffset) + case None => + initialPartitionOffsets + } + + // Find the new partitions, and get their earliest offsets + val newPartitions = untilPartitionOffsets.keySet.diff(fromPartitionOffsets.keySet) + val newPartitionOffsets = kafkaReader.fetchEarliestOffsets(newPartitions.toSeq) + if (newPartitionOffsets.keySet != newPartitions) { + // We cannot get from offsets for some partitions. It means they got deleted. + val deletedPartitions = newPartitions.diff(newPartitionOffsets.keySet) + reportDataLoss( + s"Cannot find earliest offsets of ${deletedPartitions}. Some data may have been missed") + } + logInfo(s"Partitions added: $newPartitionOffsets") + newPartitionOffsets.filter(_._2 != 0).foreach { case (p, o) => + reportDataLoss( + s"Added partition $p starts from $o instead of 0. Some data may have been missed") + } + + val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet) + if (deletedPartitions.nonEmpty) { + reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed") + } + + // Use the until partitions to calculate offset ranges to ignore partitions that have + // been deleted + val topicPartitions = untilPartitionOffsets.keySet.filter { tp => + // Ignore partitions that we don't know the from offsets. + newPartitionOffsets.contains(tp) || fromPartitionOffsets.contains(tp) + }.toSeq + logDebug("TopicPartitions: " + topicPartitions.mkString(", ")) + + val sortedExecutors = getSortedExecutorList(sc) + val numExecutors = sortedExecutors.length + logDebug("Sorted executors: " + sortedExecutors.mkString(", ")) + + // Calculate offset ranges + val offsetRanges = topicPartitions.map { tp => + val fromOffset = fromPartitionOffsets.get(tp).getOrElse { + newPartitionOffsets.getOrElse(tp, { + // This should not happen since newPartitionOffsets contains all partitions not in + // fromPartitionOffsets + throw new IllegalStateException(s"$tp doesn't have a from offset") + }) + } + val untilOffset = untilPartitionOffsets(tp) + val preferredLoc = if (numExecutors > 0) { + // This allows cached KafkaConsumers in the executors to be re-used to read the same + // partition in every batch. + Some(sortedExecutors(Math.floorMod(tp.hashCode, numExecutors))) + } else None + KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, preferredLoc) + }.filter { range => + if (range.untilOffset < range.fromOffset) { + reportDataLoss(s"Partition ${range.topicPartition}'s offset was changed from " + + s"${range.fromOffset} to ${range.untilOffset}, some data may have been missed") + false + } else { + true + } + }.toArray + + // Create an RDD that reads from Kafka and get the (key, value) pair as byte arrays. + val rdd = new KafkaSourceRDD( + sc, executorKafkaParams, offsetRanges, pollTimeoutMs, failOnDataLoss, + reuseKafkaConsumer = true).map { cr => + InternalRow( + cr.key, + cr.value, + UTF8String.fromString(cr.topic), + cr.partition, + cr.offset, + DateTimeUtils.fromJavaTimestamp(new java.sql.Timestamp(cr.timestamp)), + cr.timestampType.id) + } + + logInfo("GetBatch generating RDD of offset range: " + + offsetRanges.sortBy(_.topicPartition.toString).mkString(", ")) + + // On recovery, getBatch will get called before getOffset + if (currentPartitionOffsets.isEmpty) { + currentPartitionOffsets = Some(untilPartitionOffsets) + } + + sqlContext.internalCreateDataFrame(rdd, schema, isStreaming = true) + } + + /** Stop this source and free any resources it has allocated. */ + override def stop(): Unit = synchronized { + kafkaReader.close() + } + + override def toString(): String = s"KafkaSource[$kafkaReader]" + + /** + * If `failOnDataLoss` is true, this method will throw an `IllegalStateException`. + * Otherwise, just log a warning. + */ + private def reportDataLoss(message: String): Unit = { + if (failOnDataLoss) { + throw new IllegalStateException(message + s". $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE") + } else { + logWarning(message + s". $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE") + } + } +} + +/** Companion object for the [[KafkaSource]]. */ +private[kafka010] object KafkaSource { + val INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE = + """ + |Some data may have been lost because they are not available in Kafka any more; either the + | data was aged out by Kafka or the topic may have been deleted before all the data in the + | topic was processed. If you want your streaming query to fail on such cases, set the source + | option "failOnDataLoss" to "true". + """.stripMargin + + val INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE = + """ + |Some data may have been lost because they are not available in Kafka any more; either the + | data was aged out by Kafka or the topic may have been deleted before all the data in the + | topic was processed. If you don't want your streaming query to fail on such cases, set the + | source option "failOnDataLoss" to "false". + """.stripMargin + + private[kafka010] val VERSION = 1 + + def getSortedExecutorList(sc: SparkContext): Array[String] = { + val bm = sc.env.blockManager + bm.master.getPeers(bm.blockManagerId).toArray + .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) + .sortWith(compare) + .map(_.toString) + } + + private def compare(a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = { + if (a.host == b.host) { a.executorId > b.executorId } else { a.host > b.host } + } + +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala new file mode 100644 index 000000000000..b5da415b3097 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset} + +/** + * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and + * their offsets. + */ +private[kafka010] +case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset { + + override val json = JsonUtils.partitionOffsets(partitionToOffsets) +} + +/** Companion object of the [[KafkaSourceOffset]] */ +private[kafka010] object KafkaSourceOffset { + + def getPartitionOffsets(offset: Offset): Map[TopicPartition, Long] = { + offset match { + case o: KafkaSourceOffset => o.partitionToOffsets + case so: SerializedOffset => KafkaSourceOffset(so).partitionToOffsets + case _ => + throw new IllegalArgumentException( + s"Invalid conversion from offset of ${offset.getClass} to KafkaSourceOffset") + } + } + + /** + * Returns [[KafkaSourceOffset]] from a variable sequence of (topic, partitionId, offset) + * tuples. + */ + def apply(offsetTuples: (String, Int, Long)*): KafkaSourceOffset = { + KafkaSourceOffset(offsetTuples.map { case(t, p, o) => (new TopicPartition(t, p), o) }.toMap) + } + + /** + * Returns [[KafkaSourceOffset]] from a JSON [[SerializedOffset]] + */ + def apply(offset: SerializedOffset): KafkaSourceOffset = + KafkaSourceOffset(JsonUtils.partitionOffsets(offset.json)) +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala new file mode 100644 index 000000000000..3cb4d8cad12c --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -0,0 +1,453 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.{util => ju} +import java.util.{Locale, UUID} + +import scala.collection.JavaConverters._ + +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} +import org.apache.spark.sql.execution.streaming.{Sink, Source} +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.streaming.OutputMode +import org.apache.spark.sql.types.StructType + +/** + * The provider class for the [[KafkaSource]]. This provider is designed such that it throws + * IllegalArgumentException when the Kafka Dataset is created, so that it can catch + * missing options even before the query is started. + */ +private[kafka010] class KafkaSourceProvider extends DataSourceRegister + with StreamSourceProvider + with StreamSinkProvider + with RelationProvider + with CreatableRelationProvider + with Logging { + import KafkaSourceProvider._ + + override def shortName(): String = "kafka" + + /** + * Returns the name and schema of the source. In addition, it also verifies whether the options + * are correct and sufficient to create the [[KafkaSource]] when the query is started. + */ + override def sourceSchema( + sqlContext: SQLContext, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): (String, StructType) = { + validateStreamOptions(parameters) + require(schema.isEmpty, "Kafka source has a fixed schema and cannot be set with a custom one") + (shortName(), KafkaOffsetReader.kafkaSchema) + } + + override def createSource( + sqlContext: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source = { + validateStreamOptions(parameters) + // Each running query should use its own group id. Otherwise, the query may be only assigned + // partial data since Kafka will assign partitions to multiple consumers having the same group + // id. Hence, we should generate a unique id for each query. + val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}" + + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } + val specifiedKafkaParams = + parameters + .keySet + .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) + .map { k => k.drop(6).toString -> parameters(k) } + .toMap + + val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(caseInsensitiveParams, + STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + + val kafkaOffsetReader = new KafkaOffsetReader( + strategy(caseInsensitiveParams), + kafkaParamsForDriver(specifiedKafkaParams), + parameters, + driverGroupIdPrefix = s"$uniqueGroupId-driver") + + new KafkaSource( + sqlContext, + kafkaOffsetReader, + kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId), + parameters, + metadataPath, + startingStreamOffsets, + failOnDataLoss(caseInsensitiveParams)) + } + + /** + * Returns a new base relation with the given parameters. + * + * @note The parameters' keywords are case insensitive and this insensitivity is enforced + * by the Map that is passed to the function. + */ + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + validateBatchOptions(parameters) + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } + val specifiedKafkaParams = + parameters + .keySet + .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) + .map { k => k.drop(6).toString -> parameters(k) } + .toMap + + val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( + caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) + assert(startingRelationOffsets != LatestOffsetRangeLimit) + + val endingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(caseInsensitiveParams, + ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + assert(endingRelationOffsets != EarliestOffsetRangeLimit) + + new KafkaRelation( + sqlContext, + strategy(caseInsensitiveParams), + sourceOptions = parameters, + specifiedKafkaParams = specifiedKafkaParams, + failOnDataLoss = failOnDataLoss(caseInsensitiveParams), + startingOffsets = startingRelationOffsets, + endingOffsets = endingRelationOffsets) + } + + override def createSink( + sqlContext: SQLContext, + parameters: Map[String, String], + partitionColumns: Seq[String], + outputMode: OutputMode): Sink = { + val defaultTopic = parameters.get(TOPIC_OPTION_KEY).map(_.trim) + val specifiedKafkaParams = kafkaParamsForProducer(parameters) + new KafkaSink(sqlContext, + new ju.HashMap[String, Object](specifiedKafkaParams.asJava), defaultTopic) + } + + override def createRelation( + outerSQLContext: SQLContext, + mode: SaveMode, + parameters: Map[String, String], + data: DataFrame): BaseRelation = { + mode match { + case SaveMode.Overwrite | SaveMode.Ignore => + throw new AnalysisException(s"Save mode $mode not allowed for Kafka. " + + s"Allowed save modes are ${SaveMode.Append} and " + + s"${SaveMode.ErrorIfExists} (default).") + case _ => // good + } + val topic = parameters.get(TOPIC_OPTION_KEY).map(_.trim) + val specifiedKafkaParams = kafkaParamsForProducer(parameters) + KafkaWriter.write(outerSQLContext.sparkSession, data.queryExecution, + new ju.HashMap[String, Object](specifiedKafkaParams.asJava), topic) + + /* This method is suppose to return a relation that reads the data that was written. + * We cannot support this for Kafka. Therefore, in order to make things consistent, + * we return an empty base relation. + */ + new BaseRelation { + override def sqlContext: SQLContext = unsupportedException + override def schema: StructType = unsupportedException + override def needConversion: Boolean = unsupportedException + override def sizeInBytes: Long = unsupportedException + override def unhandledFilters(filters: Array[Filter]): Array[Filter] = unsupportedException + private def unsupportedException = + throw new UnsupportedOperationException("BaseRelation from Kafka write " + + "operation is not usable.") + } + } + + private def kafkaParamsForProducer(parameters: Map[String, String]): Map[String, String] = { + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } + if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}")) { + throw new IllegalArgumentException( + s"Kafka option '${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}' is not supported as keys " + + "are serialized with ByteArraySerializer.") + } + + if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}")) + { + throw new IllegalArgumentException( + s"Kafka option '${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}' is not supported as " + + "value are serialized with ByteArraySerializer.") + } + parameters + .keySet + .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) + .map { k => k.drop(6).toString -> parameters(k) } + .toMap + (ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> classOf[ByteArraySerializer].getName, + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> classOf[ByteArraySerializer].getName) + } + + private def strategy(caseInsensitiveParams: Map[String, String]) = + caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + case ("assign", value) => + AssignStrategy(JsonUtils.partitions(value)) + case ("subscribe", value) => + SubscribeStrategy(value.split(",").map(_.trim()).filter(_.nonEmpty)) + case ("subscribepattern", value) => + SubscribePatternStrategy(value.trim()) + case _ => + // Should never reach here as we are already matching on + // matched strategy names + throw new IllegalArgumentException("Unknown option") + } + + private def failOnDataLoss(caseInsensitiveParams: Map[String, String]) = + caseInsensitiveParams.getOrElse(FAIL_ON_DATA_LOSS_OPTION_KEY, "true").toBoolean + + private def validateGeneralOptions(parameters: Map[String, String]): Unit = { + // Validate source options + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } + val specifiedStrategies = + caseInsensitiveParams.filter { case (k, _) => STRATEGY_OPTION_KEYS.contains(k) }.toSeq + + if (specifiedStrategies.isEmpty) { + throw new IllegalArgumentException( + "One of the following options must be specified for Kafka source: " + + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.") + } else if (specifiedStrategies.size > 1) { + throw new IllegalArgumentException( + "Only one of the following options can be specified for Kafka source: " + + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.") + } + + val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + case ("assign", value) => + if (!value.trim.startsWith("{")) { + throw new IllegalArgumentException( + "No topicpartitions to assign as specified value for option " + + s"'assign' is '$value'") + } + + case ("subscribe", value) => + val topics = value.split(",").map(_.trim).filter(_.nonEmpty) + if (topics.isEmpty) { + throw new IllegalArgumentException( + "No topics to subscribe to as specified value for option " + + s"'subscribe' is '$value'") + } + case ("subscribepattern", value) => + val pattern = caseInsensitiveParams("subscribepattern").trim() + if (pattern.isEmpty) { + throw new IllegalArgumentException( + "Pattern to subscribe is empty as specified value for option " + + s"'subscribePattern' is '$value'") + } + case _ => + // Should never reach here as we are already matching on + // matched strategy names + throw new IllegalArgumentException("Unknown option") + } + + // Validate user-specified Kafka options + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as " + + s"user-specified consumer groups is not used to track offsets.") + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) { + throw new IllegalArgumentException( + s""" + |Kafka option '${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}' is not supported. + |Instead set the source option '$STARTING_OFFSETS_OPTION_KEY' to 'earliest' or 'latest' + |to specify where to start. Structured Streaming manages which offsets are consumed + |internally, rather than relying on the kafkaConsumer to do it. This will ensure that no + |data is missed when new topics/partitions are dynamically subscribed. Note that + |'$STARTING_OFFSETS_OPTION_KEY' only applies when a new Streaming query is started, and + |that resuming will always pick up from where the query left off. See the docs for more + |details. + """.stripMargin) + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}")) { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG}' is not supported as keys " + + "are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations " + + "to explicitly deserialize the keys.") + } + + if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}")) + { + throw new IllegalArgumentException( + s"Kafka option '${ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG}' is not supported as " + + "value are deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame " + + "operations to explicitly deserialize the values.") + } + + val otherUnsupportedConfigs = Seq( + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, // committing correctly requires new APIs in Source + ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG) // interceptors can modify payload, so not safe + + otherUnsupportedConfigs.foreach { c => + if (caseInsensitiveParams.contains(s"kafka.$c")) { + throw new IllegalArgumentException(s"Kafka option '$c' is not supported") + } + } + + if (!caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}")) { + throw new IllegalArgumentException( + s"Option 'kafka.${ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}' must be specified for " + + s"configuring Kafka consumer") + } + } + + private def validateStreamOptions(caseInsensitiveParams: Map[String, String]) = { + // Stream specific options + caseInsensitiveParams.get(ENDING_OFFSETS_OPTION_KEY).map(_ => + throw new IllegalArgumentException("ending offset not valid in streaming queries")) + validateGeneralOptions(caseInsensitiveParams) + } + + private def validateBatchOptions(caseInsensitiveParams: Map[String, String]) = { + // Batch specific options + KafkaSourceProvider.getKafkaOffsetRangeLimit( + caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) match { + case EarliestOffsetRangeLimit => // good to go + case LatestOffsetRangeLimit => + throw new IllegalArgumentException("starting offset can't be latest " + + "for batch queries on Kafka") + case SpecificOffsetRangeLimit(partitionOffsets) => + partitionOffsets.foreach { + case (tp, off) if off == KafkaOffsetRangeLimit.LATEST => + throw new IllegalArgumentException(s"startingOffsets for $tp can't " + + "be latest for batch queries on Kafka") + case _ => // ignore + } + } + + KafkaSourceProvider.getKafkaOffsetRangeLimit( + caseInsensitiveParams, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) match { + case EarliestOffsetRangeLimit => + throw new IllegalArgumentException("ending offset can't be earliest " + + "for batch queries on Kafka") + case LatestOffsetRangeLimit => // good to go + case SpecificOffsetRangeLimit(partitionOffsets) => + partitionOffsets.foreach { + case (tp, off) if off == KafkaOffsetRangeLimit.EARLIEST => + throw new IllegalArgumentException(s"ending offset for $tp can't be " + + "earliest for batch queries on Kafka") + case _ => // ignore + } + } + + validateGeneralOptions(caseInsensitiveParams) + + // Don't want to throw an error, but at least log a warning. + if (caseInsensitiveParams.get("maxoffsetspertrigger").isDefined) { + logWarning("maxOffsetsPerTrigger option ignored in batch queries") + } + } +} + +private[kafka010] object KafkaSourceProvider extends Logging { + private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign") + private[kafka010] val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" + private[kafka010] val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" + private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" + val TOPIC_OPTION_KEY = "topic" + + private val deserClassName = classOf[ByteArrayDeserializer].getName + + def getKafkaOffsetRangeLimit( + params: Map[String, String], + offsetOptionKey: String, + defaultOffsets: KafkaOffsetRangeLimit): KafkaOffsetRangeLimit = { + params.get(offsetOptionKey).map(_.trim) match { + case Some(offset) if offset.toLowerCase(Locale.ROOT) == "latest" => + LatestOffsetRangeLimit + case Some(offset) if offset.toLowerCase(Locale.ROOT) == "earliest" => + EarliestOffsetRangeLimit + case Some(json) => SpecificOffsetRangeLimit(JsonUtils.partitionOffsets(json)) + case None => defaultOffsets + } + } + + def kafkaParamsForDriver(specifiedKafkaParams: Map[String, String]): ju.Map[String, Object] = + ConfigUpdater("source", specifiedKafkaParams) + .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) + .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) + + // Set to "earliest" to avoid exceptions. However, KafkaSource will fetch the initial + // offsets by itself instead of counting on KafkaConsumer. + .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + + // So that consumers in the driver does not commit offsets unnecessarily + .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + + // So that the driver does not pull too much data + .set(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, new java.lang.Integer(1)) + + // If buffer config is not set, set it to reasonable value to work around + // buffer issues (see KAFKA-3135) + .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + .build() + + def kafkaParamsForExecutors( + specifiedKafkaParams: Map[String, String], + uniqueGroupId: String): ju.Map[String, Object] = + ConfigUpdater("executor", specifiedKafkaParams) + .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) + .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) + + // Make sure executors do only what the driver tells them. + .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + + // So that consumers in executors do not mess with any existing group id + .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-executor") + + // So that consumers in executors does not commit offsets unnecessarily + .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + + // If buffer config is not set, set it to reasonable value to work around + // buffer issues (see KAFKA-3135) + .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + .build() + + /** Class to conveniently update Kafka config params, while logging the changes */ + private case class ConfigUpdater(module: String, kafkaParams: Map[String, String]) { + private val map = new ju.HashMap[String, Object](kafkaParams.asJava) + + def set(key: String, value: Object): this.type = { + map.put(key, value) + logDebug(s"$module: Set $key to $value, earlier value: ${kafkaParams.getOrElse(key, "")}") + this + } + + def setIfUnset(key: String, value: Object): ConfigUpdater = { + if (!map.containsKey(key)) { + map.put(key, value) + logDebug(s"$module: Set $key to $value") + } + this + } + + def build(): ju.Map[String, Object] = map + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala new file mode 100644 index 000000000000..9d9e2aaba807 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceRDD.scala @@ -0,0 +1,211 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.{util => ju} + +import scala.collection.mutable.ArrayBuffer + +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord} +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.NextIterator + + +/** Offset range that one partition of the KafkaSourceRDD has to read */ +private[kafka010] case class KafkaSourceRDDOffsetRange( + topicPartition: TopicPartition, + fromOffset: Long, + untilOffset: Long, + preferredLoc: Option[String]) { + def topic: String = topicPartition.topic + def partition: Int = topicPartition.partition + def size: Long = untilOffset - fromOffset +} + + +/** Partition of the KafkaSourceRDD */ +private[kafka010] case class KafkaSourceRDDPartition( + index: Int, offsetRange: KafkaSourceRDDOffsetRange) extends Partition + + +/** + * An RDD that reads data from Kafka based on offset ranges across multiple partitions. + * Additionally, it allows preferred locations to be set for each topic + partition, so that + * the [[KafkaSource]] can ensure the same executor always reads the same topic + partition + * and cached KafkaConsuemrs (see [[CachedKafkaConsumer]] can be used read data efficiently. + * + * @param sc the [[SparkContext]] + * @param executorKafkaParams Kafka configuration for creating KafkaConsumer on the executors + * @param offsetRanges Offset ranges that define the Kafka data belonging to this RDD + */ +private[kafka010] class KafkaSourceRDD( + sc: SparkContext, + executorKafkaParams: ju.Map[String, Object], + offsetRanges: Seq[KafkaSourceRDDOffsetRange], + pollTimeoutMs: Long, + failOnDataLoss: Boolean, + reuseKafkaConsumer: Boolean) + extends RDD[ConsumerRecord[Array[Byte], Array[Byte]]](sc, Nil) { + + override def persist(newLevel: StorageLevel): this.type = { + logError("Kafka ConsumerRecord is not serializable. " + + "Use .map to extract fields before calling .persist or .window") + super.persist(newLevel) + } + + override def getPartitions: Array[Partition] = { + offsetRanges.zipWithIndex.map { case (o, i) => new KafkaSourceRDDPartition(i, o) }.toArray + } + + override def count(): Long = offsetRanges.map(_.size).sum + + override def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = { + val c = count + new PartialResult(new BoundedDouble(c, 1.0, c, c), true) + } + + override def isEmpty(): Boolean = count == 0L + + override def take(num: Int): Array[ConsumerRecord[Array[Byte], Array[Byte]]] = { + val nonEmptyPartitions = + this.partitions.map(_.asInstanceOf[KafkaSourceRDDPartition]).filter(_.offsetRange.size > 0) + + if (num < 1 || nonEmptyPartitions.isEmpty) { + return new Array[ConsumerRecord[Array[Byte], Array[Byte]]](0) + } + + // Determine in advance how many messages need to be taken from each partition + val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) => + val remain = num - result.values.sum + if (remain > 0) { + val taken = Math.min(remain, part.offsetRange.size) + result + (part.index -> taken.toInt) + } else { + result + } + } + + val buf = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]] + val res = context.runJob( + this, + (tc: TaskContext, it: Iterator[ConsumerRecord[Array[Byte], Array[Byte]]]) => + it.take(parts(tc.partitionId)).toArray, parts.keys.toArray + ) + res.foreach(buf ++= _) + buf.toArray + } + + override def getPreferredLocations(split: Partition): Seq[String] = { + val part = split.asInstanceOf[KafkaSourceRDDPartition] + part.offsetRange.preferredLoc.map(Seq(_)).getOrElse(Seq.empty) + } + + override def compute( + thePart: Partition, + context: TaskContext): Iterator[ConsumerRecord[Array[Byte], Array[Byte]]] = { + val sourcePartition = thePart.asInstanceOf[KafkaSourceRDDPartition] + val topic = sourcePartition.offsetRange.topic + val kafkaPartition = sourcePartition.offsetRange.partition + val consumer = + if (!reuseKafkaConsumer) { + // If we can't reuse CachedKafkaConsumers, creating a new CachedKafkaConsumer. As here we + // uses `assign`, we don't need to worry about the "group.id" conflicts. + CachedKafkaConsumer.createUncached(topic, kafkaPartition, executorKafkaParams) + } else { + CachedKafkaConsumer.getOrCreate(topic, kafkaPartition, executorKafkaParams) + } + val range = resolveRange(consumer, sourcePartition.offsetRange) + assert( + range.fromOffset <= range.untilOffset, + s"Beginning offset ${range.fromOffset} is after the ending offset ${range.untilOffset} " + + s"for topic ${range.topic} partition ${range.partition}. " + + "You either provided an invalid fromOffset, or the Kafka topic has been damaged") + if (range.fromOffset == range.untilOffset) { + logInfo(s"Beginning offset ${range.fromOffset} is the same as ending offset " + + s"skipping ${range.topic} ${range.partition}") + Iterator.empty + } else { + val underlying = new NextIterator[ConsumerRecord[Array[Byte], Array[Byte]]]() { + var requestOffset = range.fromOffset + + override def getNext(): ConsumerRecord[Array[Byte], Array[Byte]] = { + if (requestOffset >= range.untilOffset) { + // Processed all offsets in this partition. + finished = true + null + } else { + val r = consumer.get(requestOffset, range.untilOffset, pollTimeoutMs, failOnDataLoss) + if (r == null) { + // Losing some data. Skip the rest offsets in this partition. + finished = true + null + } else { + requestOffset = r.offset + 1 + r + } + } + } + + override protected def close(): Unit = { + if (!reuseKafkaConsumer) { + // Don't forget to close non-reuse KafkaConsumers. You may take down your cluster! + consumer.close() + } else { + // Indicate that we're no longer using this consumer + CachedKafkaConsumer.releaseKafkaConsumer(topic, kafkaPartition, executorKafkaParams) + } + } + } + // Release consumer, either by removing it or indicating we're no longer using it + context.addTaskCompletionListener { _ => + underlying.closeIfNeeded() + } + underlying + } + } + + private def resolveRange(consumer: CachedKafkaConsumer, range: KafkaSourceRDDOffsetRange) = { + if (range.fromOffset < 0 || range.untilOffset < 0) { + // Late bind the offset range + val availableOffsetRange = consumer.getAvailableOffsetRange() + val fromOffset = if (range.fromOffset < 0) { + assert(range.fromOffset == KafkaOffsetRangeLimit.EARLIEST, + s"earliest offset ${range.fromOffset} does not equal ${KafkaOffsetRangeLimit.EARLIEST}") + availableOffsetRange.earliest + } else { + range.fromOffset + } + val untilOffset = if (range.untilOffset < 0) { + assert(range.untilOffset == KafkaOffsetRangeLimit.LATEST, + s"latest offset ${range.untilOffset} does not equal ${KafkaOffsetRangeLimit.LATEST}") + availableOffsetRange.latest + } else { + range.untilOffset + } + KafkaSourceRDDOffsetRange(range.topicPartition, + fromOffset, untilOffset, range.preferredLoc) + } else { + range + } + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala new file mode 100644 index 000000000000..6fd333e2f43b --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriteTask.scala @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.{util => ju} + +import org.apache.kafka.clients.producer.{Callback, KafkaProducer, ProducerRecord, RecordMetadata} + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, UnsafeProjection} +import org.apache.spark.sql.types.{BinaryType, StringType} + +/** + * A simple trait for writing out data in a single Spark task, without any concerns about how + * to commit or abort tasks. Exceptions thrown by the implementation of this class will + * automatically trigger task aborts. + */ +private[kafka010] class KafkaWriteTask( + producerConfiguration: ju.Map[String, Object], + inputSchema: Seq[Attribute], + topic: Option[String]) { + // used to synchronize with Kafka callbacks + @volatile private var failedWrite: Exception = null + private val projection = createProjection + private var producer: KafkaProducer[Array[Byte], Array[Byte]] = _ + + /** + * Writes key value data out to topics. + */ + def execute(iterator: Iterator[InternalRow]): Unit = { + producer = CachedKafkaProducer.getOrCreate(producerConfiguration) + while (iterator.hasNext && failedWrite == null) { + val currentRow = iterator.next() + val projectedRow = projection(currentRow) + val topic = projectedRow.getUTF8String(0) + val key = projectedRow.getBinary(1) + val value = projectedRow.getBinary(2) + if (topic == null) { + throw new NullPointerException(s"null topic present in the data. Use the " + + s"${KafkaSourceProvider.TOPIC_OPTION_KEY} option for setting a default topic.") + } + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic.toString, key, value) + val callback = new Callback() { + override def onCompletion(recordMetadata: RecordMetadata, e: Exception): Unit = { + if (failedWrite == null && e != null) { + failedWrite = e + } + } + } + producer.send(record, callback) + } + } + + def close(): Unit = { + checkForErrors() + if (producer != null) { + producer.flush() + checkForErrors() + producer = null + } + } + + private def createProjection: UnsafeProjection = { + val topicExpression = topic.map(Literal(_)).orElse { + inputSchema.find(_.name == KafkaWriter.TOPIC_ATTRIBUTE_NAME) + }.getOrElse { + throw new IllegalStateException(s"topic option required when no " + + s"'${KafkaWriter.TOPIC_ATTRIBUTE_NAME}' attribute is present") + } + topicExpression.dataType match { + case StringType => // good + case t => + throw new IllegalStateException(s"${KafkaWriter.TOPIC_ATTRIBUTE_NAME} " + + s"attribute unsupported type $t. ${KafkaWriter.TOPIC_ATTRIBUTE_NAME} " + + "must be a StringType") + } + val keyExpression = inputSchema.find(_.name == KafkaWriter.KEY_ATTRIBUTE_NAME) + .getOrElse(Literal(null, BinaryType)) + keyExpression.dataType match { + case StringType | BinaryType => // good + case t => + throw new IllegalStateException(s"${KafkaWriter.KEY_ATTRIBUTE_NAME} " + + s"attribute unsupported type $t") + } + val valueExpression = inputSchema + .find(_.name == KafkaWriter.VALUE_ATTRIBUTE_NAME).getOrElse( + throw new IllegalStateException("Required attribute " + + s"'${KafkaWriter.VALUE_ATTRIBUTE_NAME}' not found") + ) + valueExpression.dataType match { + case StringType | BinaryType => // good + case t => + throw new IllegalStateException(s"${KafkaWriter.VALUE_ATTRIBUTE_NAME} " + + s"attribute unsupported type $t") + } + UnsafeProjection.create( + Seq(topicExpression, Cast(keyExpression, BinaryType), + Cast(valueExpression, BinaryType)), inputSchema) + } + + private def checkForErrors(): Unit = { + if (failedWrite != null) { + throw failedWrite + } + } +} + diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala new file mode 100644 index 000000000000..5e9ae35b3f00 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.{util => ju} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} +import org.apache.spark.sql.types.{BinaryType, StringType} +import org.apache.spark.util.Utils + +/** + * The [[KafkaWriter]] class is used to write data from a batch query + * or structured streaming query, given by a [[QueryExecution]], to Kafka. + * The data is assumed to have a value column, and an optional topic and key + * columns. If the topic column is missing, then the topic must come from + * the 'topic' configuration option. If the key column is missing, then a + * null valued key field will be added to the + * [[org.apache.kafka.clients.producer.ProducerRecord]]. + */ +private[kafka010] object KafkaWriter extends Logging { + val TOPIC_ATTRIBUTE_NAME: String = "topic" + val KEY_ATTRIBUTE_NAME: String = "key" + val VALUE_ATTRIBUTE_NAME: String = "value" + + override def toString: String = "KafkaWriter" + + def validateQuery( + queryExecution: QueryExecution, + kafkaParameters: ju.Map[String, Object], + topic: Option[String] = None): Unit = { + val schema = queryExecution.analyzed.output + schema.find(_.name == TOPIC_ATTRIBUTE_NAME).getOrElse( + if (topic.isEmpty) { + throw new AnalysisException(s"topic option required when no " + + s"'$TOPIC_ATTRIBUTE_NAME' attribute is present. Use the " + + s"${KafkaSourceProvider.TOPIC_OPTION_KEY} option for setting a topic.") + } else { + Literal(topic.get, StringType) + } + ).dataType match { + case StringType => // good + case _ => + throw new AnalysisException(s"Topic type must be a String") + } + schema.find(_.name == KEY_ATTRIBUTE_NAME).getOrElse( + Literal(null, StringType) + ).dataType match { + case StringType | BinaryType => // good + case _ => + throw new AnalysisException(s"$KEY_ATTRIBUTE_NAME attribute type " + + s"must be a String or BinaryType") + } + schema.find(_.name == VALUE_ATTRIBUTE_NAME).getOrElse( + throw new AnalysisException(s"Required attribute '$VALUE_ATTRIBUTE_NAME' not found") + ).dataType match { + case StringType | BinaryType => // good + case _ => + throw new AnalysisException(s"$VALUE_ATTRIBUTE_NAME attribute type " + + s"must be a String or BinaryType") + } + } + + def write( + sparkSession: SparkSession, + queryExecution: QueryExecution, + kafkaParameters: ju.Map[String, Object], + topic: Option[String] = None): Unit = { + val schema = queryExecution.analyzed.output + validateQuery(queryExecution, kafkaParameters, topic) + queryExecution.toRdd.foreachPartition { iter => + val writeTask = new KafkaWriteTask(kafkaParameters, schema, topic) + Utils.tryWithSafeFinally(block = writeTask.execute(iter))( + finallyBlock = writeTask.close()) + } + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java new file mode 100644 index 000000000000..596f775c56db --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Structured Streaming Data Source for Kafka 0.10 + */ +package org.apache.spark.sql.kafka010; diff --git a/external/kafka-0-10-sql/src/test/resources/kafka-source-initial-offset-version-2.1.0.bin b/external/kafka-0-10-sql/src/test/resources/kafka-source-initial-offset-version-2.1.0.bin new file mode 100644 index 000000000000..ae928e724967 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/resources/kafka-source-initial-offset-version-2.1.0.bin @@ -0,0 +1 @@ +2{"kafka-initial-offset-2-1-0":{"2":0,"1":0,"0":0}} \ No newline at end of file diff --git a/external/kafka-0-10-sql/src/test/resources/kafka-source-offset-version-2.1.0.txt b/external/kafka-0-10-sql/src/test/resources/kafka-source-offset-version-2.1.0.txt new file mode 100644 index 000000000000..6410031743d2 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/resources/kafka-source-offset-version-2.1.0.txt @@ -0,0 +1 @@ +{"topic1":{"0":456,"1":789},"topic2":{"0":0}} diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka-0-10-sql/src/test/resources/log4j.properties similarity index 100% rename from external/kafka/src/test/resources/log4j.properties rename to external/kafka-0-10-sql/src/test/resources/log4j.properties diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumerSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumerSuite.scala new file mode 100644 index 000000000000..7aa7dd096c07 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumerSuite.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.kafka010 + +import org.scalatest.PrivateMethodTester + +import org.apache.spark.sql.test.SharedSQLContext + +class CachedKafkaConsumerSuite extends SharedSQLContext with PrivateMethodTester { + + test("SPARK-19886: Report error cause correctly in reportDataLoss") { + val cause = new Exception("D'oh!") + val reportDataLoss = PrivateMethod[Unit]('reportDataLoss0) + val e = intercept[IllegalStateException] { + CachedKafkaConsumer.invokePrivate(reportDataLoss(true, "message", cause)) + } + assert(e.getCause === cause) + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala new file mode 100644 index 000000000000..789bffa9da12 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.{util => ju} +import java.util.concurrent.ConcurrentMap + +import org.apache.kafka.clients.producer.KafkaProducer +import org.apache.kafka.common.serialization.ByteArraySerializer +import org.scalatest.PrivateMethodTester + +import org.apache.spark.sql.test.SharedSQLContext + +class CachedKafkaProducerSuite extends SharedSQLContext with PrivateMethodTester { + + type KP = KafkaProducer[Array[Byte], Array[Byte]] + + protected override def beforeEach(): Unit = { + super.beforeEach() + val clear = PrivateMethod[Unit]('clear) + CachedKafkaProducer.invokePrivate(clear()) + } + + test("Should return the cached instance on calling getOrCreate with same params.") { + val kafkaParams = new ju.HashMap[String, Object]() + kafkaParams.put("acks", "0") + // Here only host should be resolvable, it does not need a running instance of kafka server. + kafkaParams.put("bootstrap.servers", "127.0.0.1:9022") + kafkaParams.put("key.serializer", classOf[ByteArraySerializer].getName) + kafkaParams.put("value.serializer", classOf[ByteArraySerializer].getName) + val producer = CachedKafkaProducer.getOrCreate(kafkaParams) + val producer2 = CachedKafkaProducer.getOrCreate(kafkaParams) + assert(producer == producer2) + + val cacheMap = PrivateMethod[ConcurrentMap[Seq[(String, Object)], KP]]('getAsMap) + val map = CachedKafkaProducer.invokePrivate(cacheMap()) + assert(map.size == 1) + } + + test("Should close the correct kafka producer for the given kafkaPrams.") { + val kafkaParams = new ju.HashMap[String, Object]() + kafkaParams.put("acks", "0") + kafkaParams.put("bootstrap.servers", "127.0.0.1:9022") + kafkaParams.put("key.serializer", classOf[ByteArraySerializer].getName) + kafkaParams.put("value.serializer", classOf[ByteArraySerializer].getName) + val producer: KP = CachedKafkaProducer.getOrCreate(kafkaParams) + kafkaParams.put("acks", "1") + val producer2: KP = CachedKafkaProducer.getOrCreate(kafkaParams) + // With updated conf, a new producer instance should be created. + assert(producer != producer2) + + val cacheMap = PrivateMethod[ConcurrentMap[Seq[(String, Object)], KP]]('getAsMap) + val map = CachedKafkaProducer.invokePrivate(cacheMap()) + assert(map.size == 2) + + CachedKafkaProducer.close(kafkaParams) + val map2 = CachedKafkaProducer.invokePrivate(cacheMap()) + assert(map2.size == 1) + import scala.collection.JavaConverters._ + val (seq: Seq[(String, Object)], _producer: KP) = map2.asScala.toArray.apply(0) + assert(_producer == producer) + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/JsonUtilsSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/JsonUtilsSuite.scala new file mode 100644 index 000000000000..54b980049d1a --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/JsonUtilsSuite.scala @@ -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.kafka010 + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.SparkFunSuite + +class JsonUtilsSuite extends SparkFunSuite { + + test("parsing partitions") { + val parsed = JsonUtils.partitions("""{"topicA":[0,1],"topicB":[4,6]}""") + val expected = Array( + new TopicPartition("topicA", 0), + new TopicPartition("topicA", 1), + new TopicPartition("topicB", 4), + new TopicPartition("topicB", 6) + ) + assert(parsed.toSeq === expected.toSeq) + } + + test("parsing partitionOffsets") { + val parsed = JsonUtils.partitionOffsets( + """{"topicA":{"0":23,"1":-1},"topicB":{"0":-2}}""") + + assert(parsed(new TopicPartition("topicA", 0)) === 23) + assert(parsed(new TopicPartition("topicA", 1)) === -1) + assert(parsed(new TopicPartition("topicB", 0)) === -2) + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala new file mode 100644 index 000000000000..91893df4ec32 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.util.Locale +import java.util.concurrent.atomic.AtomicInteger + +import org.apache.kafka.common.TopicPartition +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.util.Utils + +class KafkaRelationSuite extends QueryTest with BeforeAndAfter with SharedSQLContext { + + import testImplicits._ + + private val topicId = new AtomicInteger(0) + + private var testUtils: KafkaTestUtils = _ + + private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" + + private def assignString(topic: String, partitions: Iterable[Int]): String = { + JsonUtils.partitions(partitions.map(p => new TopicPartition(topic, p))) + } + + override def beforeAll(): Unit = { + super.beforeAll() + testUtils = new KafkaTestUtils + testUtils.setup() + } + + override def afterAll(): Unit = { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + super.afterAll() + } + } + + private def createDF( + topic: String, + withOptions: Map[String, String] = Map.empty[String, String], + brokerAddress: Option[String] = None) = { + val df = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", + brokerAddress.getOrElse(testUtils.brokerAddress)) + .option("subscribe", topic) + withOptions.foreach { + case (key, value) => df.option(key, value) + } + df.load().selectExpr("CAST(value AS STRING)") + } + + + test("explicit earliest to latest offsets") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, Array("20"), Some(2)) + + // Specify explicit earliest and latest offset values + val df = createDF(topic, + withOptions = Map("startingOffsets" -> "earliest", "endingOffsets" -> "latest")) + checkAnswer(df, (0 to 20).map(_.toString).toDF) + + // "latest" should late bind to the current (latest) offset in the df + testUtils.sendMessages(topic, (21 to 29).map(_.toString).toArray, Some(2)) + checkAnswer(df, (0 to 29).map(_.toString).toDF) + } + + test("default starting and ending offsets") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, Array("20"), Some(2)) + + // Implicit offset values, should default to earliest and latest + val df = createDF(topic) + // Test that we default to "earliest" and "latest" + checkAnswer(df, (0 to 20).map(_.toString).toDF) + } + + test("explicit offsets") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, Array("20"), Some(2)) + + // Test explicitly specified offsets + val startPartitionOffsets = Map( + new TopicPartition(topic, 0) -> -2L, // -2 => earliest + new TopicPartition(topic, 1) -> -2L, + new TopicPartition(topic, 2) -> 0L // explicit earliest + ) + val startingOffsets = JsonUtils.partitionOffsets(startPartitionOffsets) + + val endPartitionOffsets = Map( + new TopicPartition(topic, 0) -> -1L, // -1 => latest + new TopicPartition(topic, 1) -> -1L, + new TopicPartition(topic, 2) -> 1L // explicit offset happens to = the latest + ) + val endingOffsets = JsonUtils.partitionOffsets(endPartitionOffsets) + val df = createDF(topic, + withOptions = Map("startingOffsets" -> startingOffsets, "endingOffsets" -> endingOffsets)) + checkAnswer(df, (0 to 20).map(_.toString).toDF) + + // static offset partition 2, nothing should change + testUtils.sendMessages(topic, (31 to 39).map(_.toString).toArray, Some(2)) + checkAnswer(df, (0 to 20).map(_.toString).toDF) + + // latest offset partition 1, should change + testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray, Some(1)) + checkAnswer(df, (0 to 30).map(_.toString).toDF) + } + + test("reuse same dataframe in query") { + // This test ensures that we do not cache the Kafka Consumer in KafkaRelation + val topic = newTopic() + testUtils.createTopic(topic, partitions = 1) + testUtils.sendMessages(topic, (0 to 10).map(_.toString).toArray, Some(0)) + + // Specify explicit earliest and latest offset values + val df = createDF(topic, + withOptions = Map("startingOffsets" -> "earliest", "endingOffsets" -> "latest")) + checkAnswer(df.union(df), ((0 to 10) ++ (0 to 10)).map(_.toString).toDF) + } + + test("test late binding start offsets") { + // Kafka fails to remove the logs on Windows. See KAFKA-1194. + assume(!Utils.isWindows) + + var kafkaUtils: KafkaTestUtils = null + try { + /** + * The following settings will ensure that all log entries + * are removed following a call to cleanupLogs + */ + val brokerProps = Map[String, Object]( + "log.retention.bytes" -> 1.asInstanceOf[AnyRef], // retain nothing + "log.retention.ms" -> 1.asInstanceOf[AnyRef] // no wait time + ) + kafkaUtils = new KafkaTestUtils(withBrokerProps = brokerProps) + kafkaUtils.setup() + + val topic = newTopic() + kafkaUtils.createTopic(topic, partitions = 1) + kafkaUtils.sendMessages(topic, (0 to 9).map(_.toString).toArray, Some(0)) + // Specify explicit earliest and latest offset values + val df = createDF(topic, + withOptions = Map("startingOffsets" -> "earliest", "endingOffsets" -> "latest"), + Some(kafkaUtils.brokerAddress)) + checkAnswer(df, (0 to 9).map(_.toString).toDF) + // Blow away current set of messages. + kafkaUtils.cleanupLogs() + // Add some more data, but do not call cleanup + kafkaUtils.sendMessages(topic, (10 to 19).map(_.toString).toArray, Some(0)) + // Ensure that we late bind to the new starting position + checkAnswer(df, (10 to 19).map(_.toString).toDF) + } finally { + if (kafkaUtils != null) { + kafkaUtils.teardown() + } + } + } + + test("bad batch query options") { + def testBadOptions(options: (String, String)*)(expectedMsgs: String*): Unit = { + val ex = intercept[IllegalArgumentException] { + val reader = spark + .read + .format("kafka") + options.foreach { case (k, v) => reader.option(k, v) } + reader.load() + } + expectedMsgs.foreach { m => + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains(m.toLowerCase(Locale.ROOT))) + } + } + + // Specifying an ending offset as the starting point + testBadOptions("startingOffsets" -> "latest")("starting offset can't be latest " + + "for batch queries on Kafka") + + // Now do it with an explicit json start offset indicating latest + val startPartitionOffsets = Map( new TopicPartition("t", 0) -> -1L) + val startingOffsets = JsonUtils.partitionOffsets(startPartitionOffsets) + testBadOptions("subscribe" -> "t", "startingOffsets" -> startingOffsets)( + "startingOffsets for t-0 can't be latest for batch queries on Kafka") + + + // Make sure we catch ending offsets that indicate earliest + testBadOptions("endingOffsets" -> "earliest")("ending offset can't be earliest " + + "for batch queries on Kafka") + + // Make sure we catch ending offsets that indicating earliest + val endPartitionOffsets = Map(new TopicPartition("t", 0) -> -2L) + val endingOffsets = JsonUtils.partitionOffsets(endPartitionOffsets) + testBadOptions("subscribe" -> "t", "endingOffsets" -> endingOffsets)( + "ending offset for t-0 can't be earliest for batch queries on Kafka") + + // No strategy specified + testBadOptions()("options must be specified", "subscribe", "subscribePattern") + + // Multiple strategies specified + testBadOptions("subscribe" -> "t", "subscribePattern" -> "t.*")( + "only one", "options can be specified") + + testBadOptions("subscribe" -> "t", "assign" -> """{"a":[0]}""")( + "only one", "options can be specified") + + testBadOptions("assign" -> "")("no topicpartitions to assign") + testBadOptions("subscribe" -> "")("no topics to subscribe") + testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty") + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala new file mode 100644 index 000000000000..2ab336c7ac47 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -0,0 +1,430 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.util.Locale +import java.util.concurrent.atomic.AtomicInteger + +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.common.serialization.ByteArraySerializer +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.SparkException +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, SpecificInternalRow, UnsafeProjection} +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.streaming._ +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{BinaryType, DataType} + +class KafkaSinkSuite extends StreamTest with SharedSQLContext { + import testImplicits._ + + protected var testUtils: KafkaTestUtils = _ + + override val streamingTimeout = 30.seconds + + override def beforeAll(): Unit = { + super.beforeAll() + testUtils = new KafkaTestUtils( + withBrokerProps = Map("auto.create.topics.enable" -> "false")) + testUtils.setup() + } + + override def afterAll(): Unit = { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + super.afterAll() + } + } + + test("batch - write to kafka") { + val topic = newTopic() + testUtils.createTopic(topic) + val df = Seq("1", "2", "3", "4", "5").map(v => (topic, v)).toDF("topic", "value") + df.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("topic", topic) + .save() + checkAnswer( + createKafkaReader(topic).selectExpr("CAST(value as STRING) value"), + Row("1") :: Row("2") :: Row("3") :: Row("4") :: Row("5") :: Nil) + } + + test("batch - null topic field value, and no topic option") { + val df = Seq[(String, String)](null.asInstanceOf[String] -> "1").toDF("topic", "value") + val ex = intercept[SparkException] { + df.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .save() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "null topic present in the data")) + } + + test("batch - unsupported save modes") { + val topic = newTopic() + testUtils.createTopic(topic) + val df = Seq[(String, String)](null.asInstanceOf[String] -> "1").toDF("topic", "value") + + // Test bad save mode Ignore + var ex = intercept[AnalysisException] { + df.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .mode(SaveMode.Ignore) + .save() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + s"save mode ignore not allowed for kafka")) + + // Test bad save mode Overwrite + ex = intercept[AnalysisException] { + df.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .mode(SaveMode.Overwrite) + .save() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + s"save mode overwrite not allowed for kafka")) + } + + test("SPARK-20496: batch - enforce analyzed plans") { + val inputEvents = + spark.range(1, 1000) + .select(to_json(struct("*")) as 'value) + + val topic = newTopic() + testUtils.createTopic(topic) + // used to throw UnresolvedException + inputEvents.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("topic", topic) + .save() + } + + test("streaming - write to kafka with topic field") { + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) + + val writer = createKafkaWriter( + input.toDF(), + withTopic = None, + withOutputMode = Some(OutputMode.Append))( + withSelectExpr = s"'$topic' as topic", "value") + + val reader = createKafkaReader(topic) + .selectExpr("CAST(key as STRING) key", "CAST(value as STRING) value") + .selectExpr("CAST(key as INT) key", "CAST(value as INT) value") + .as[(Int, Int)] + .map(_._2) + + try { + input.addData("1", "2", "3", "4", "5") + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, 1, 2, 3, 4, 5) + input.addData("6", "7", "8", "9", "10") + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10) + } finally { + writer.stop() + } + } + + test("streaming - write aggregation w/o topic field, with topic option") { + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) + + val writer = createKafkaWriter( + input.toDF().groupBy("value").count(), + withTopic = Some(topic), + withOutputMode = Some(OutputMode.Update()))( + withSelectExpr = "CAST(value as STRING) key", "CAST(count as STRING) value") + + val reader = createKafkaReader(topic) + .selectExpr("CAST(key as STRING) key", "CAST(value as STRING) value") + .selectExpr("CAST(key as INT) key", "CAST(value as INT) value") + .as[(Int, Int)] + + try { + input.addData("1", "2", "2", "3", "3", "3") + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, (1, 1), (2, 2), (3, 3)) + input.addData("1", "2", "3") + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, (1, 1), (2, 2), (3, 3), (1, 2), (2, 3), (3, 4)) + } finally { + writer.stop() + } + } + + test("streaming - aggregation with topic field and topic option") { + /* The purpose of this test is to ensure that the topic option + * overrides the topic field. We begin by writing some data that + * includes a topic field and value (e.g., 'foo') along with a topic + * option. Then when we read from the topic specified in the option + * we should see the data i.e., the data was written to the topic + * option, and not to the topic in the data e.g., foo + */ + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) + + val writer = createKafkaWriter( + input.toDF().groupBy("value").count(), + withTopic = Some(topic), + withOutputMode = Some(OutputMode.Update()))( + withSelectExpr = "'foo' as topic", + "CAST(value as STRING) key", "CAST(count as STRING) value") + + val reader = createKafkaReader(topic) + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .selectExpr("CAST(key AS INT)", "CAST(value AS INT)") + .as[(Int, Int)] + + try { + input.addData("1", "2", "2", "3", "3", "3") + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, (1, 1), (2, 2), (3, 3)) + input.addData("1", "2", "3") + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + checkDatasetUnorderly(reader, (1, 1), (2, 2), (3, 3), (1, 2), (2, 3), (3, 4)) + } finally { + writer.stop() + } + } + + + test("streaming - write data with bad schema") { + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) + + /* No topic field or topic option */ + var writer: StreamingQuery = null + var ex: Exception = null + try { + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter(input.toDF())( + withSelectExpr = "value as key", "value" + ) + input.addData("1", "2", "3", "4", "5") + writer.processAllAvailable() + } + } finally { + writer.stop() + } + assert(ex.getMessage + .toLowerCase(Locale.ROOT) + .contains("topic option required when no 'topic' attribute is present")) + + try { + /* No value field */ + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter(input.toDF())( + withSelectExpr = s"'$topic' as topic", "value as key" + ) + input.addData("1", "2", "3", "4", "5") + writer.processAllAvailable() + } + } finally { + writer.stop() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "required attribute 'value' not found")) + } + + test("streaming - write data with valid schema but wrong types") { + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) + + var writer: StreamingQuery = null + var ex: Exception = null + try { + /* topic field wrong type */ + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter(input.toDF())( + withSelectExpr = s"CAST('1' as INT) as topic", "value" + ) + input.addData("1", "2", "3", "4", "5") + writer.processAllAvailable() + } + } finally { + writer.stop() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("topic type must be a string")) + + try { + /* value field wrong type */ + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter(input.toDF())( + withSelectExpr = s"'$topic' as topic", "CAST(value as INT) as value" + ) + input.addData("1", "2", "3", "4", "5") + writer.processAllAvailable() + } + } finally { + writer.stop() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "value attribute type must be a string or binarytype")) + + try { + ex = intercept[StreamingQueryException] { + /* key field wrong type */ + writer = createKafkaWriter(input.toDF())( + withSelectExpr = s"'$topic' as topic", "CAST(value as INT) as key", "value" + ) + input.addData("1", "2", "3", "4", "5") + writer.processAllAvailable() + } + } finally { + writer.stop() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "key attribute type must be a string or binarytype")) + } + + test("streaming - write to non-existing topic") { + val input = MemoryStream[String] + val topic = newTopic() + + var writer: StreamingQuery = null + var ex: Exception = null + try { + ex = intercept[StreamingQueryException] { + writer = createKafkaWriter(input.toDF(), withTopic = Some(topic))() + input.addData("1", "2", "3", "4", "5") + writer.processAllAvailable() + } + } finally { + writer.stop() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("job aborted")) + } + + test("streaming - exception on config serializer") { + val input = MemoryStream[String] + var writer: StreamingQuery = null + var ex: Exception = null + ex = intercept[IllegalArgumentException] { + writer = createKafkaWriter( + input.toDF(), + withOptions = Map("kafka.key.serializer" -> "foo"))() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "kafka option 'key.serializer' is not supported")) + + ex = intercept[IllegalArgumentException] { + writer = createKafkaWriter( + input.toDF(), + withOptions = Map("kafka.value.serializer" -> "foo"))() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "kafka option 'value.serializer' is not supported")) + } + + test("generic - write big data with small producer buffer") { + /* This test ensures that we understand the semantics of Kafka when + * is comes to blocking on a call to send when the send buffer is full. + * This test will configure the smallest possible producer buffer and + * indicate that we should block when it is full. Thus, no exception should + * be thrown in the case of a full buffer. + */ + val topic = newTopic() + testUtils.createTopic(topic, 1) + val options = new java.util.HashMap[String, Object] + options.put("bootstrap.servers", testUtils.brokerAddress) + options.put("buffer.memory", "16384") // min buffer size + options.put("block.on.buffer.full", "true") + options.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[ByteArraySerializer].getName) + options.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[ByteArraySerializer].getName) + val inputSchema = Seq(AttributeReference("value", BinaryType)()) + val data = new Array[Byte](15000) // large value + val writeTask = new KafkaWriteTask(options, inputSchema, Some(topic)) + try { + val fieldTypes: Array[DataType] = Array(BinaryType) + val converter = UnsafeProjection.create(fieldTypes) + val row = new SpecificInternalRow(fieldTypes) + row.update(0, data) + val iter = Seq.fill(1000)(converter.apply(row)).iterator + writeTask.execute(iter) + } finally { + writeTask.close() + } + } + + private val topicId = new AtomicInteger(0) + + private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" + + private def createKafkaReader(topic: String): DataFrame = { + spark.read + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + .option("subscribe", topic) + .load() + } + + private def createKafkaWriter( + input: DataFrame, + withTopic: Option[String] = None, + withOutputMode: Option[OutputMode] = None, + withOptions: Map[String, String] = Map[String, String]()) + (withSelectExpr: String*): StreamingQuery = { + var stream: DataStreamWriter[Row] = null + withTempDir { checkpointDir => + var df = input.toDF() + if (withSelectExpr.length > 0) { + df = df.selectExpr(withSelectExpr: _*) + } + stream = df.writeStream + .format("kafka") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .queryName("kafkaStream") + withTopic.foreach(stream.option("topic", _)) + withOutputMode.foreach(stream.outputMode(_)) + withOptions.foreach(opt => stream.option(opt._1, opt._2)) + } + stream.start() + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala new file mode 100644 index 000000000000..efec51d09745 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.io.File + +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.streaming.OffsetSuite +import org.apache.spark.sql.test.SharedSQLContext + +class KafkaSourceOffsetSuite extends OffsetSuite with SharedSQLContext { + + compare( + one = KafkaSourceOffset(("t", 0, 1L)), + two = KafkaSourceOffset(("t", 0, 2L))) + + compare( + one = KafkaSourceOffset(("t", 0, 1L), ("t", 1, 0L)), + two = KafkaSourceOffset(("t", 0, 2L), ("t", 1, 1L))) + + compare( + one = KafkaSourceOffset(("t", 0, 1L), ("T", 0, 0L)), + two = KafkaSourceOffset(("t", 0, 2L), ("T", 0, 1L))) + + compare( + one = KafkaSourceOffset(("t", 0, 1L)), + two = KafkaSourceOffset(("t", 0, 2L), ("t", 1, 1L))) + + + val kso1 = KafkaSourceOffset(("t", 0, 1L)) + val kso2 = KafkaSourceOffset(("t", 0, 2L), ("t", 1, 3L)) + val kso3 = KafkaSourceOffset(("t", 0, 2L), ("t", 1, 3L), ("t", 1, 4L)) + + compare(KafkaSourceOffset(SerializedOffset(kso1.json)), + KafkaSourceOffset(SerializedOffset(kso2.json))) + + test("basic serialization - deserialization") { + assert(KafkaSourceOffset.getPartitionOffsets(kso1) == + KafkaSourceOffset.getPartitionOffsets(SerializedOffset(kso1.json))) + } + + + test("OffsetSeqLog serialization - deserialization") { + withTempDir { temp => + // use non-existent directory to test whether log make the dir + val dir = new File(temp, "dir") + val metadataLog = new OffsetSeqLog(spark, dir.getAbsolutePath) + val batch0 = OffsetSeq.fill(kso1) + val batch1 = OffsetSeq.fill(kso2, kso3) + + val batch0Serialized = OffsetSeq.fill(batch0.offsets.flatMap(_.map(o => + SerializedOffset(o.json))): _*) + + val batch1Serialized = OffsetSeq.fill(batch1.offsets.flatMap(_.map(o => + SerializedOffset(o.json))): _*) + + assert(metadataLog.add(0, batch0)) + assert(metadataLog.getLatest() === Some(0 -> batch0Serialized)) + assert(metadataLog.get(0) === Some(batch0Serialized)) + + assert(metadataLog.add(1, batch1)) + assert(metadataLog.get(0) === Some(batch0Serialized)) + assert(metadataLog.get(1) === Some(batch1Serialized)) + assert(metadataLog.getLatest() === Some(1 -> batch1Serialized)) + assert(metadataLog.get(None, Some(1)) === + Array(0 -> batch0Serialized, 1 -> batch1Serialized)) + + // Adding the same batch does nothing + metadataLog.add(1, OffsetSeq.fill(LongOffset(3))) + assert(metadataLog.get(0) === Some(batch0Serialized)) + assert(metadataLog.get(1) === Some(batch1Serialized)) + assert(metadataLog.getLatest() === Some(1 -> batch1Serialized)) + assert(metadataLog.get(None, Some(1)) === + Array(0 -> batch0Serialized, 1 -> batch1Serialized)) + } + } + + test("read Spark 2.1.0 offset format") { + val offset = readFromResource("kafka-source-offset-version-2.1.0.txt") + assert(KafkaSourceOffset(offset) === + KafkaSourceOffset(("topic1", 0, 456L), ("topic1", 1, 789L), ("topic2", 0, 0L))) + } + + private def readFromResource(file: String): SerializedOffset = { + import scala.io.Source + val input = getClass.getResource(s"/$file").toURI + val str = Source.fromFile(input).mkString + SerializedOffset(str) + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala new file mode 100644 index 000000000000..2034b9be07f2 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -0,0 +1,1060 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.io._ +import java.nio.charset.StandardCharsets.UTF_8 +import java.nio.file.{Files, Paths} +import java.util.{Locale, Properties} +import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.mutable +import scala.util.Random + +import org.apache.kafka.clients.producer.RecordMetadata +import org.apache.kafka.common.TopicPartition +import org.scalatest.concurrent.Eventually._ +import org.scalatest.concurrent.PatienceConfiguration.Timeout +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.SparkContext +import org.apache.spark.sql.ForeachWriter +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.functions.{count, window} +import org.apache.spark.sql.kafka010.KafkaSourceProvider._ +import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest} +import org.apache.spark.sql.streaming.util.StreamManualClock +import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} +import org.apache.spark.util.Utils + +abstract class KafkaSourceTest extends StreamTest with SharedSQLContext { + + protected var testUtils: KafkaTestUtils = _ + + override val streamingTimeout = 30.seconds + + override def beforeAll(): Unit = { + super.beforeAll() + testUtils = new KafkaTestUtils + testUtils.setup() + } + + override def afterAll(): Unit = { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + super.afterAll() + } + } + + protected def makeSureGetOffsetCalled = AssertOnQuery { q => + // Because KafkaSource's initialPartitionOffsets is set lazily, we need to make sure + // its "getOffset" is called before pushing any data. Otherwise, because of the race contion, + // we don't know which data should be fetched when `startingOffsets` is latest. + q.processAllAvailable() + true + } + + /** + * Add data to Kafka. + * + * `topicAction` can be used to run actions for each topic before inserting data. + */ + case class AddKafkaData(topics: Set[String], data: Int*) + (implicit ensureDataInMultiplePartition: Boolean = false, + concurrent: Boolean = false, + message: String = "", + topicAction: (String, Option[Int]) => Unit = (_, _) => {}) extends AddData { + + override def addData(query: Option[StreamExecution]): (Source, Offset) = { + if (query.get.isActive) { + // Make sure no Spark job is running when deleting a topic + query.get.processAllAvailable() + } + + val existingTopics = testUtils.getAllTopicsAndPartitionSize().toMap + val newTopics = topics.diff(existingTopics.keySet) + for (newTopic <- newTopics) { + topicAction(newTopic, None) + } + for (existingTopicPartitions <- existingTopics) { + topicAction(existingTopicPartitions._1, Some(existingTopicPartitions._2)) + } + + // Read all topics again in case some topics are delete. + val allTopics = testUtils.getAllTopicsAndPartitionSize().toMap.keys + require( + query.nonEmpty, + "Cannot add data when there is no query for finding the active kafka source") + + val sources = query.get.logicalPlan.collect { + case StreamingExecutionRelation(source, _) if source.isInstanceOf[KafkaSource] => + source.asInstanceOf[KafkaSource] + } + if (sources.isEmpty) { + throw new Exception( + "Could not find Kafka source in the StreamExecution logical plan to add data to") + } else if (sources.size > 1) { + throw new Exception( + "Could not select the Kafka source in the StreamExecution logical plan as there" + + "are multiple Kafka sources:\n\t" + sources.mkString("\n\t")) + } + val kafkaSource = sources.head + val topic = topics.toSeq(Random.nextInt(topics.size)) + val sentMetadata = testUtils.sendMessages(topic, data.map { _.toString }.toArray) + + def metadataToStr(m: (String, RecordMetadata)): String = { + s"Sent ${m._1} to partition ${m._2.partition()}, offset ${m._2.offset()}" + } + // Verify that the test data gets inserted into multiple partitions + if (ensureDataInMultiplePartition) { + require( + sentMetadata.groupBy(_._2.partition).size > 1, + s"Added data does not test multiple partitions: ${sentMetadata.map(metadataToStr)}") + } + + val offset = KafkaSourceOffset(testUtils.getLatestOffsets(topics)) + logInfo(s"Added data, expected offset $offset") + (kafkaSource, offset) + } + + override def toString: String = + s"AddKafkaData(topics = $topics, data = $data, message = $message)" + } +} + + +class KafkaSourceSuite extends KafkaSourceTest { + + import testImplicits._ + + private val topicId = new AtomicInteger(0) + + testWithUninterruptibleThread( + "deserialization of initial offset with Spark 2.1.0") { + withTempDir { metadataPath => + val topic = newTopic + testUtils.createTopic(topic, partitions = 3) + + val provider = new KafkaSourceProvider + val parameters = Map( + "kafka.bootstrap.servers" -> testUtils.brokerAddress, + "subscribe" -> topic + ) + val source = provider.createSource(spark.sqlContext, metadataPath.getAbsolutePath, None, + "", parameters) + source.getOffset.get // Write initial offset + + // Make sure Spark 2.1.0 will throw an exception when reading the new log + intercept[java.lang.IllegalArgumentException] { + // Simulate how Spark 2.1.0 reads the log + Utils.tryWithResource(new FileInputStream(metadataPath.getAbsolutePath + "/0")) { in => + val length = in.read() + val bytes = new Array[Byte](length) + in.read(bytes) + KafkaSourceOffset(SerializedOffset(new String(bytes, UTF_8))) + } + } + } + } + + testWithUninterruptibleThread("deserialization of initial offset written by Spark 2.1.0") { + withTempDir { metadataPath => + val topic = "kafka-initial-offset-2-1-0" + testUtils.createTopic(topic, partitions = 3) + + val provider = new KafkaSourceProvider + val parameters = Map( + "kafka.bootstrap.servers" -> testUtils.brokerAddress, + "subscribe" -> topic + ) + + val from = new File( + getClass.getResource("/kafka-source-initial-offset-version-2.1.0.bin").toURI).toPath + val to = Paths.get(s"${metadataPath.getAbsolutePath}/0") + Files.copy(from, to) + + val source = provider.createSource( + spark.sqlContext, metadataPath.toURI.toString, None, "", parameters) + val deserializedOffset = source.getOffset.get + val referenceOffset = KafkaSourceOffset((topic, 0, 0L), (topic, 1, 0L), (topic, 2, 0L)) + assert(referenceOffset == deserializedOffset) + } + } + + testWithUninterruptibleThread("deserialization of initial offset written by future version") { + withTempDir { metadataPath => + val futureMetadataLog = + new HDFSMetadataLog[KafkaSourceOffset](sqlContext.sparkSession, + metadataPath.getAbsolutePath) { + override def serialize(metadata: KafkaSourceOffset, out: OutputStream): Unit = { + out.write(0) + val writer = new BufferedWriter(new OutputStreamWriter(out, UTF_8)) + writer.write(s"v99999\n${metadata.json}") + writer.flush + } + } + + val topic = newTopic + testUtils.createTopic(topic, partitions = 3) + val offset = KafkaSourceOffset((topic, 0, 0L), (topic, 1, 0L), (topic, 2, 0L)) + futureMetadataLog.add(0, offset) + + val provider = new KafkaSourceProvider + val parameters = Map( + "kafka.bootstrap.servers" -> testUtils.brokerAddress, + "subscribe" -> topic + ) + val source = provider.createSource(spark.sqlContext, metadataPath.getAbsolutePath, None, + "", parameters) + + val e = intercept[java.lang.IllegalStateException] { + source.getOffset.get // Read initial offset + } + + Seq( + s"maximum supported log version is v${KafkaSource.VERSION}, but encountered v99999", + "produced by a newer version of Spark and cannot be read by this version" + ).foreach { message => + assert(e.getMessage.contains(message)) + } + } + } + + test("(de)serialization of initial offsets") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 64) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + + testStream(reader.load)( + makeSureGetOffsetCalled, + StopStream, + StartStream(), + StopStream) + } + + test("maxOffsetsPerTrigger") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (100 to 200).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (10 to 20).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, Array("1"), Some(2)) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("maxOffsetsPerTrigger", 10) + .option("subscribe", topic) + .option("startingOffsets", "earliest") + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped: org.apache.spark.sql.Dataset[_] = kafka.map(kv => kv._2.toInt) + + val clock = new StreamManualClock + + val waitUntilBatchProcessed = AssertOnQuery { q => + eventually(Timeout(streamingTimeout)) { + if (!q.exception.isDefined) { + assert(clock.isStreamWaitingAt(clock.getTimeMillis())) + } + } + if (q.exception.isDefined) { + throw q.exception.get + } + true + } + + testStream(mapped)( + StartStream(ProcessingTime(100), clock), + waitUntilBatchProcessed, + // 1 from smallest, 1 from middle, 8 from biggest + CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107), + AdvanceManualClock(100), + waitUntilBatchProcessed, + // smallest now empty, 1 more from middle, 9 more from biggest + CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107, + 11, 108, 109, 110, 111, 112, 113, 114, 115, 116 + ), + StopStream, + StartStream(ProcessingTime(100), clock), + waitUntilBatchProcessed, + // smallest now empty, 1 more from middle, 9 more from biggest + CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107, + 11, 108, 109, 110, 111, 112, 113, 114, 115, 116, + 12, 117, 118, 119, 120, 121, 122, 123, 124, 125 + ), + AdvanceManualClock(100), + waitUntilBatchProcessed, + // smallest now empty, 1 more from middle, 9 more from biggest + CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107, + 11, 108, 109, 110, 111, 112, 113, 114, 115, 116, + 12, 117, 118, 119, 120, 121, 122, 123, 124, 125, + 13, 126, 127, 128, 129, 130, 131, 132, 133, 134 + ) + ) + } + + test("cannot stop Kafka stream") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", s"topic-.*") + + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) + + testStream(mapped)( + makeSureGetOffsetCalled, + StopStream + ) + } + + for (failOnDataLoss <- Seq(true, false)) { + test(s"assign from latest offsets (failOnDataLoss: $failOnDataLoss)") { + val topic = newTopic() + testFromLatestOffsets( + topic, + addPartitions = false, + failOnDataLoss = failOnDataLoss, + "assign" -> assignString(topic, 0 to 4)) + } + + test(s"assign from earliest offsets (failOnDataLoss: $failOnDataLoss)") { + val topic = newTopic() + testFromEarliestOffsets( + topic, + addPartitions = false, + failOnDataLoss = failOnDataLoss, + "assign" -> assignString(topic, 0 to 4)) + } + + test(s"assign from specific offsets (failOnDataLoss: $failOnDataLoss)") { + val topic = newTopic() + testFromSpecificOffsets( + topic, + failOnDataLoss = failOnDataLoss, + "assign" -> assignString(topic, 0 to 4), + "failOnDataLoss" -> failOnDataLoss.toString) + } + + test(s"subscribing topic by name from latest offsets (failOnDataLoss: $failOnDataLoss)") { + val topic = newTopic() + testFromLatestOffsets( + topic, + addPartitions = true, + failOnDataLoss = failOnDataLoss, + "subscribe" -> topic) + } + + test(s"subscribing topic by name from earliest offsets (failOnDataLoss: $failOnDataLoss)") { + val topic = newTopic() + testFromEarliestOffsets( + topic, + addPartitions = true, + failOnDataLoss = failOnDataLoss, + "subscribe" -> topic) + } + + test(s"subscribing topic by name from specific offsets (failOnDataLoss: $failOnDataLoss)") { + val topic = newTopic() + testFromSpecificOffsets(topic, failOnDataLoss = failOnDataLoss, "subscribe" -> topic) + } + + test(s"subscribing topic by pattern from latest offsets (failOnDataLoss: $failOnDataLoss)") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-suffix" + testFromLatestOffsets( + topic, + addPartitions = true, + failOnDataLoss = failOnDataLoss, + "subscribePattern" -> s"$topicPrefix-.*") + } + + test(s"subscribing topic by pattern from earliest offsets (failOnDataLoss: $failOnDataLoss)") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-suffix" + testFromEarliestOffsets( + topic, + addPartitions = true, + failOnDataLoss = failOnDataLoss, + "subscribePattern" -> s"$topicPrefix-.*") + } + + test(s"subscribing topic by pattern from specific offsets (failOnDataLoss: $failOnDataLoss)") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-suffix" + testFromSpecificOffsets( + topic, + failOnDataLoss = failOnDataLoss, + "subscribePattern" -> s"$topicPrefix-.*") + } + } + + test("subscribing topic by pattern with topic deletions") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-seems" + val topic2 = topicPrefix + "-bad" + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", s"$topicPrefix-.*") + .option("failOnDataLoss", "false") + + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) + + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + Assert { + testUtils.deleteTopic(topic) + testUtils.createTopic(topic2, partitions = 5) + true + }, + AddKafkaData(Set(topic2), 4, 5, 6), + CheckAnswer(2, 3, 4, 5, 6, 7) + ) + } + + test("starting offset is latest by default") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("0")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + + val kafka = reader.load() + .selectExpr("CAST(value AS STRING)") + .as[String] + val mapped = kafka.map(_.toInt) + + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(1, 2, 3) // should not have 0 + ) + } + + test("bad source options") { + def testBadOptions(options: (String, String)*)(expectedMsgs: String*): Unit = { + val ex = intercept[IllegalArgumentException] { + val reader = spark + .readStream + .format("kafka") + options.foreach { case (k, v) => reader.option(k, v) } + reader.load() + } + expectedMsgs.foreach { m => + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains(m.toLowerCase(Locale.ROOT))) + } + } + + // Specifying an ending offset + testBadOptions("endingOffsets" -> "latest")("Ending offset not valid in streaming queries") + + // No strategy specified + testBadOptions()("options must be specified", "subscribe", "subscribePattern") + + // Multiple strategies specified + testBadOptions("subscribe" -> "t", "subscribePattern" -> "t.*")( + "only one", "options can be specified") + + testBadOptions("subscribe" -> "t", "assign" -> """{"a":[0]}""")( + "only one", "options can be specified") + + testBadOptions("assign" -> "")("no topicpartitions to assign") + testBadOptions("subscribe" -> "")("no topics to subscribe") + testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty") + } + + test("unsupported kafka configs") { + def testUnsupportedConfig(key: String, value: String = "someValue"): Unit = { + val ex = intercept[IllegalArgumentException] { + val reader = spark + .readStream + .format("kafka") + .option("subscribe", "topic") + .option("kafka.bootstrap.servers", "somehost") + .option(s"$key", value) + reader.load() + } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("not supported")) + } + + testUnsupportedConfig("kafka.group.id") + testUnsupportedConfig("kafka.auto.offset.reset") + testUnsupportedConfig("kafka.enable.auto.commit") + testUnsupportedConfig("kafka.interceptor.classes") + testUnsupportedConfig("kafka.key.deserializer") + testUnsupportedConfig("kafka.value.deserializer") + + testUnsupportedConfig("kafka.auto.offset.reset", "none") + testUnsupportedConfig("kafka.auto.offset.reset", "someValue") + testUnsupportedConfig("kafka.auto.offset.reset", "earliest") + testUnsupportedConfig("kafka.auto.offset.reset", "latest") + } + + test("input row metrics") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val kafka = spark + .readStream + .format("kafka") + .option("subscribe", topic) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + + val mapped = kafka.map(kv => kv._2.toInt + 1) + testStream(mapped)( + StartStream(trigger = ProcessingTime(1)), + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + AssertOnQuery { query => + val recordsRead = query.recentProgress.map(_.numInputRows).sum + recordsRead == 3 + } + ) + } + + test("delete a topic when a Spark job is running") { + KafkaSourceSuite.collectedData.clear() + + val topic = newTopic() + testUtils.createTopic(topic, partitions = 1) + testUtils.sendMessages(topic, (1 to 10).map(_.toString).toArray) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribe", topic) + // If a topic is deleted and we try to poll data starting from offset 0, + // the Kafka consumer will just block until timeout and return an empty result. + // So set the timeout to 1 second to make this test fast. + .option("kafkaConsumer.pollTimeoutMs", "1000") + .option("startingOffsets", "earliest") + .option("failOnDataLoss", "false") + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + KafkaSourceSuite.globalTestUtils = testUtils + // The following ForeachWriter will delete the topic before fetching data from Kafka + // in executors. + val query = kafka.map(kv => kv._2.toInt).writeStream.foreach(new ForeachWriter[Int] { + override def open(partitionId: Long, version: Long): Boolean = { + KafkaSourceSuite.globalTestUtils.deleteTopic(topic) + true + } + + override def process(value: Int): Unit = { + KafkaSourceSuite.collectedData.add(value) + } + + override def close(errorOrNull: Throwable): Unit = {} + }).start() + query.processAllAvailable() + query.stop() + // `failOnDataLoss` is `false`, we should not fail the query + assert(query.exception.isEmpty) + } + + test("get offsets from case insensitive parameters") { + for ((optionKey, optionValue, answer) <- Seq( + (STARTING_OFFSETS_OPTION_KEY, "earLiEst", EarliestOffsetRangeLimit), + (ENDING_OFFSETS_OPTION_KEY, "laTest", LatestOffsetRangeLimit), + (STARTING_OFFSETS_OPTION_KEY, """{"topic-A":{"0":23}}""", + SpecificOffsetRangeLimit(Map(new TopicPartition("topic-A", 0) -> 23))))) { + val offset = getKafkaOffsetRangeLimit(Map(optionKey -> optionValue), optionKey, answer) + assert(offset === answer) + } + + for ((optionKey, answer) <- Seq( + (STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit), + (ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit))) { + val offset = getKafkaOffsetRangeLimit(Map.empty, optionKey, answer) + assert(offset === answer) + } + } + + private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" + + private def assignString(topic: String, partitions: Iterable[Int]): String = { + JsonUtils.partitions(partitions.map(p => new TopicPartition(topic, p))) + } + + private def testFromSpecificOffsets( + topic: String, + failOnDataLoss: Boolean, + options: (String, String)*): Unit = { + val partitionOffsets = Map( + new TopicPartition(topic, 0) -> -2L, + new TopicPartition(topic, 1) -> -1L, + new TopicPartition(topic, 2) -> 0L, + new TopicPartition(topic, 3) -> 1L, + new TopicPartition(topic, 4) -> 2L + ) + val startingOffsets = JsonUtils.partitionOffsets(partitionOffsets) + + testUtils.createTopic(topic, partitions = 5) + // part 0 starts at earliest, these should all be seen + testUtils.sendMessages(topic, Array(-20, -21, -22).map(_.toString), Some(0)) + // part 1 starts at latest, these should all be skipped + testUtils.sendMessages(topic, Array(-10, -11, -12).map(_.toString), Some(1)) + // part 2 starts at 0, these should all be seen + testUtils.sendMessages(topic, Array(0, 1, 2).map(_.toString), Some(2)) + // part 3 starts at 1, first should be skipped + testUtils.sendMessages(topic, Array(10, 11, 12).map(_.toString), Some(3)) + // part 4 starts at 2, first and second should be skipped + testUtils.sendMessages(topic, Array(20, 21, 22).map(_.toString), Some(4)) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark + .readStream + .format("kafka") + .option("startingOffsets", startingOffsets) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("failOnDataLoss", failOnDataLoss.toString) + options.foreach { case (k, v) => reader.option(k, v) } + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped: org.apache.spark.sql.Dataset[_] = kafka.map(kv => kv._2.toInt) + + testStream(mapped)( + makeSureGetOffsetCalled, + CheckAnswer(-20, -21, -22, 0, 1, 2, 11, 12, 22), + StopStream, + StartStream(), + CheckAnswer(-20, -21, -22, 0, 1, 2, 11, 12, 22), // Should get the data back on recovery + AddKafkaData(Set(topic), 30, 31, 32, 33, 34)(ensureDataInMultiplePartition = true), + CheckAnswer(-20, -21, -22, 0, 1, 2, 11, 12, 22, 30, 31, 32, 33, 34), + StopStream + ) + } + + test("Kafka column types") { + val now = System.currentTimeMillis() + val topic = newTopic() + testUtils.createTopic(newTopic(), partitions = 1) + testUtils.sendMessages(topic, Array(1).map(_.toString)) + + val kafka = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("startingOffsets", s"earliest") + .option("subscribe", topic) + .load() + + val query = kafka + .writeStream + .format("memory") + .outputMode("append") + .queryName("kafkaColumnTypes") + .start() + query.processAllAvailable() + val rows = spark.table("kafkaColumnTypes").collect() + assert(rows.length === 1, s"Unexpected results: ${rows.toList}") + val row = rows(0) + assert(row.getAs[Array[Byte]]("key") === null, s"Unexpected results: $row") + assert(row.getAs[Array[Byte]]("value") === "1".getBytes(UTF_8), s"Unexpected results: $row") + assert(row.getAs[String]("topic") === topic, s"Unexpected results: $row") + assert(row.getAs[Int]("partition") === 0, s"Unexpected results: $row") + assert(row.getAs[Long]("offset") === 0L, s"Unexpected results: $row") + // We cannot check the exact timestamp as it's the time that messages were inserted by the + // producer. So here we just use a low bound to make sure the internal conversion works. + assert(row.getAs[java.sql.Timestamp]("timestamp").getTime >= now, s"Unexpected results: $row") + assert(row.getAs[Int]("timestampType") === 0, s"Unexpected results: $row") + query.stop() + } + + test("KafkaSource with watermark") { + val now = System.currentTimeMillis() + val topic = newTopic() + testUtils.createTopic(newTopic(), partitions = 1) + testUtils.sendMessages(topic, Array(1).map(_.toString)) + + val kafka = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("startingOffsets", s"earliest") + .option("subscribe", topic) + .load() + + val windowedAggregation = kafka + .withWatermark("timestamp", "10 seconds") + .groupBy(window($"timestamp", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"window".getField("start") as 'window, $"count") + + val query = windowedAggregation + .writeStream + .format("memory") + .outputMode("complete") + .queryName("kafkaWatermark") + .start() + query.processAllAvailable() + val rows = spark.table("kafkaWatermark").collect() + assert(rows.length === 1, s"Unexpected results: ${rows.toList}") + val row = rows(0) + // We cannot check the exact window start time as it depands on the time that messages were + // inserted by the producer. So here we just use a low bound to make sure the internal + // conversion works. + assert( + row.getAs[java.sql.Timestamp]("window").getTime >= now - 5 * 1000, + s"Unexpected results: $row") + assert(row.getAs[Int]("count") === 1, s"Unexpected results: $row") + query.stop() + } + + private def testFromLatestOffsets( + topic: String, + addPartitions: Boolean, + failOnDataLoss: Boolean, + options: (String, String)*): Unit = { + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark + .readStream + .format("kafka") + .option("startingOffsets", s"latest") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("failOnDataLoss", failOnDataLoss.toString) + options.foreach { case (k, v) => reader.option(k, v) } + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) + + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + StopStream, + StartStream(), + CheckAnswer(2, 3, 4), // Should get the data back on recovery + StopStream, + AddKafkaData(Set(topic), 4, 5, 6), // Add data when stream is stopped + StartStream(), + CheckAnswer(2, 3, 4, 5, 6, 7), // Should get the added data + AddKafkaData(Set(topic), 7, 8), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), + AssertOnQuery("Add partitions") { query: StreamExecution => + if (addPartitions) { + testUtils.addPartitions(topic, 10) + } + true + }, + AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) + ) + } + + private def testFromEarliestOffsets( + topic: String, + addPartitions: Boolean, + failOnDataLoss: Boolean, + options: (String, String)*): Unit = { + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, (1 to 3).map { _.toString }.toArray) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val reader = spark.readStream + reader + .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) + .option("startingOffsets", s"earliest") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("failOnDataLoss", failOnDataLoss.toString) + options.foreach { case (k, v) => reader.option(k, v) } + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped = kafka.map(kv => kv._2.toInt + 1) + + testStream(mapped)( + AddKafkaData(Set(topic), 4, 5, 6), // Add data when stream is stopped + CheckAnswer(2, 3, 4, 5, 6, 7), + StopStream, + StartStream(), + CheckAnswer(2, 3, 4, 5, 6, 7), + StopStream, + AddKafkaData(Set(topic), 7, 8), + StartStream(), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9), + AssertOnQuery("Add partitions") { query: StreamExecution => + if (addPartitions) { + testUtils.addPartitions(topic, 10) + } + true + }, + AddKafkaData(Set(topic), 9, 10, 11, 12, 13, 14, 15, 16), + CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) + ) + } +} + +object KafkaSourceSuite { + @volatile var globalTestUtils: KafkaTestUtils = _ + val collectedData = new ConcurrentLinkedQueue[Any]() +} + + +class KafkaSourceStressSuite extends KafkaSourceTest { + + import testImplicits._ + + val topicId = new AtomicInteger(1) + + @volatile var topics: Seq[String] = (1 to 5).map(_ => newStressTopic) + + def newStressTopic: String = s"stress${topicId.getAndIncrement()}" + + private def nextInt(start: Int, end: Int): Int = { + start + Random.nextInt(start + end - 1) + } + + test("stress test with multiple topics and partitions") { + topics.foreach { topic => + testUtils.createTopic(topic, partitions = nextInt(1, 6)) + testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray) + } + + // Create Kafka source that reads from latest offset + val kafka = + spark.readStream + .format(classOf[KafkaSourceProvider].getCanonicalName.stripSuffix("$")) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", "stress.*") + .option("failOnDataLoss", "false") + .load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + + val mapped = kafka.map(kv => kv._2.toInt + 1) + + runStressTest( + mapped, + Seq(makeSureGetOffsetCalled), + (d, running) => { + Random.nextInt(5) match { + case 0 => // Add a new topic + topics = topics ++ Seq(newStressTopic) + AddKafkaData(topics.toSet, d: _*)(message = s"Add topic $newStressTopic", + topicAction = (topic, partition) => { + if (partition.isEmpty) { + testUtils.createTopic(topic, partitions = nextInt(1, 6)) + } + }) + case 1 if running => + // Only delete a topic when the query is running. Otherwise, we may lost data and + // cannot check the correctness. + val deletedTopic = topics(Random.nextInt(topics.size)) + if (deletedTopic != topics.head) { + topics = topics.filterNot(_ == deletedTopic) + } + AddKafkaData(topics.toSet, d: _*)(message = s"Delete topic $deletedTopic", + topicAction = (topic, partition) => { + // Never remove the first topic to make sure we have at least one topic + if (topic == deletedTopic && deletedTopic != topics.head) { + testUtils.deleteTopic(deletedTopic) + } + }) + case 2 => // Add new partitions + AddKafkaData(topics.toSet, d: _*)(message = "Add partition", + topicAction = (topic, partition) => { + testUtils.addPartitions(topic, partition.get + nextInt(1, 6)) + }) + case _ => // Just add new data + AddKafkaData(topics.toSet, d: _*) + } + }, + iterations = 50) + } +} + +class KafkaSourceStressForDontFailOnDataLossSuite extends StreamTest with SharedSQLContext { + + import testImplicits._ + + private var testUtils: KafkaTestUtils = _ + + private val topicId = new AtomicInteger(0) + + private def newTopic(): String = s"failOnDataLoss-${topicId.getAndIncrement()}" + + override def createSparkSession(): TestSparkSession = { + // Set maxRetries to 3 to handle NPE from `poll` when deleting a topic + new TestSparkSession(new SparkContext("local[2,3]", "test-sql-context", sparkConf)) + } + + override def beforeAll(): Unit = { + super.beforeAll() + testUtils = new KafkaTestUtils { + override def brokerConfiguration: Properties = { + val props = super.brokerConfiguration + // Try to make Kafka clean up messages as fast as possible. However, there is a hard-code + // 30 seconds delay (kafka.log.LogManager.InitialTaskDelayMs) so this test should run at + // least 30 seconds. + props.put("log.cleaner.backoff.ms", "100") + props.put("log.segment.bytes", "40") + props.put("log.retention.bytes", "40") + props.put("log.retention.check.interval.ms", "100") + props.put("delete.retention.ms", "10") + props.put("log.flush.scheduler.interval.ms", "10") + props + } + } + testUtils.setup() + } + + override def afterAll(): Unit = { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + super.afterAll() + } + } + + test("stress test for failOnDataLoss=false") { + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("subscribePattern", "failOnDataLoss.*") + .option("startingOffsets", "earliest") + .option("failOnDataLoss", "false") + .option("fetchOffset.retryIntervalMs", "3000") + val kafka = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val query = kafka.map(kv => kv._2.toInt).writeStream.foreach(new ForeachWriter[Int] { + + override def open(partitionId: Long, version: Long): Boolean = { + true + } + + override def process(value: Int): Unit = { + // Slow down the processing speed so that messages may be aged out. + Thread.sleep(Random.nextInt(500)) + } + + override def close(errorOrNull: Throwable): Unit = { + } + }).start() + + val testTime = 1.minutes + val startTime = System.currentTimeMillis() + // Track the current existing topics + val topics = mutable.ArrayBuffer[String]() + // Track topics that have been deleted + val deletedTopics = mutable.Set[String]() + while (System.currentTimeMillis() - testTime.toMillis < startTime) { + Random.nextInt(10) match { + case 0 => // Create a new topic + val topic = newTopic() + topics += topic + // As pushing messages into Kafka updates Zookeeper asynchronously, there is a small + // chance that a topic will be recreated after deletion due to the asynchronous update. + // Hence, always overwrite to handle this race condition. + testUtils.createTopic(topic, partitions = 1, overwrite = true) + logInfo(s"Create topic $topic") + case 1 if topics.nonEmpty => // Delete an existing topic + val topic = topics.remove(Random.nextInt(topics.size)) + testUtils.deleteTopic(topic) + logInfo(s"Delete topic $topic") + deletedTopics += topic + case 2 if deletedTopics.nonEmpty => // Recreate a topic that was deleted. + val topic = deletedTopics.toSeq(Random.nextInt(deletedTopics.size)) + deletedTopics -= topic + topics += topic + // As pushing messages into Kafka updates Zookeeper asynchronously, there is a small + // chance that a topic will be recreated after deletion due to the asynchronous update. + // Hence, always overwrite to handle this race condition. + testUtils.createTopic(topic, partitions = 1, overwrite = true) + logInfo(s"Create topic $topic") + case 3 => + Thread.sleep(1000) + case _ => // Push random messages + for (topic <- topics) { + val size = Random.nextInt(10) + for (_ <- 0 until size) { + testUtils.sendMessages(topic, Array(Random.nextInt(10).toString)) + } + } + } + // `failOnDataLoss` is `false`, we should not fail the query + if (query.exception.nonEmpty) { + throw query.exception.get + } + } + + query.stop() + // `failOnDataLoss` is `false`, we should not fail the query + if (query.exception.nonEmpty) { + throw query.exception.get + } + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala new file mode 100644 index 000000000000..2df8352f4866 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -0,0 +1,429 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.kafka010 + +import java.io.{File, IOException} +import java.lang.{Integer => JInt} +import java.net.InetSocketAddress +import java.util.{Map => JMap, Properties} +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.language.postfixOps +import scala.util.Random + +import kafka.admin.AdminUtils +import kafka.api.Request +import kafka.common.TopicAndPartition +import kafka.server.{KafkaConfig, KafkaServer, OffsetCheckpoint} +import kafka.utils.ZkUtils +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.producer._ +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer} +import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +/** + * This is a helper class for Kafka test suites. This has the functionality to set up + * and tear down local Kafka servers, and to push data using Kafka producers. + * + * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. + */ +class KafkaTestUtils(withBrokerProps: Map[String, Object] = Map.empty) extends Logging { + + // Zookeeper related configurations + private val zkHost = "localhost" + private var zkPort: Int = 0 + private val zkConnectionTimeout = 60000 + private val zkSessionTimeout = 6000 + + private var zookeeper: EmbeddedZookeeper = _ + + private var zkUtils: ZkUtils = _ + + // Kafka broker related configurations + private val brokerHost = "localhost" + private var brokerPort = 0 + private var brokerConf: KafkaConfig = _ + + // Kafka broker server + private var server: KafkaServer = _ + + // Kafka producer + private var producer: Producer[String, String] = _ + + // Flag to test whether the system is correctly started + private var zkReady = false + private var brokerReady = false + + def zkAddress: String = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") + s"$zkHost:$zkPort" + } + + def brokerAddress: String = { + assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") + s"$brokerHost:$brokerPort" + } + + def zookeeperClient: ZkUtils = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") + Option(zkUtils).getOrElse( + throw new IllegalStateException("Zookeeper client is not yet initialized")) + } + + // Set up the Embedded Zookeeper server and get the proper Zookeeper port + private def setupEmbeddedZookeeper(): Unit = { + // Zookeeper server startup + zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") + // Get the actual zookeeper binding port + zkPort = zookeeper.actualPort + zkUtils = ZkUtils(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, false) + zkReady = true + } + + // Set up the Embedded Kafka server + private def setupEmbeddedKafkaServer(): Unit = { + assert(zkReady, "Zookeeper should be set up beforehand") + + // Kafka broker startup + Utils.startServiceOnPort(brokerPort, port => { + brokerPort = port + brokerConf = new KafkaConfig(brokerConfiguration, doLog = false) + server = new KafkaServer(brokerConf) + server.startup() + brokerPort = server.boundPort() + (server, brokerPort) + }, new SparkConf(), "KafkaBroker") + + brokerReady = true + } + + /** setup the whole embedded servers, including Zookeeper and Kafka brokers */ + def setup(): Unit = { + setupEmbeddedZookeeper() + setupEmbeddedKafkaServer() + } + + /** Teardown the whole servers, including Kafka broker and Zookeeper */ + def teardown(): Unit = { + brokerReady = false + zkReady = false + + if (producer != null) { + producer.close() + producer = null + } + + if (server != null) { + server.shutdown() + server.awaitShutdown() + server = null + } + + // On Windows, `logDirs` is left open even after Kafka server above is completely shut down + // in some cases. It leads to test failures on Windows if the directory deletion failure + // throws an exception. + brokerConf.logDirs.foreach { f => + try { + Utils.deleteRecursively(new File(f)) + } catch { + case e: IOException if Utils.isWindows => + logWarning(e.getMessage) + } + } + + if (zkUtils != null) { + zkUtils.close() + zkUtils = null + } + + if (zookeeper != null) { + zookeeper.shutdown() + zookeeper = null + } + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String, partitions: Int, overwrite: Boolean = false): Unit = { + var created = false + while (!created) { + try { + AdminUtils.createTopic(zkUtils, topic, partitions, 1) + created = true + } catch { + // Workaround fact that TopicExistsException is in kafka.common in 0.10.0 and + // org.apache.kafka.common.errors in 0.10.1 (!) + case e: Exception if (e.getClass.getSimpleName == "TopicExistsException") && overwrite => + deleteTopic(topic) + } + } + // wait until metadata is propagated + (0 until partitions).foreach { p => + waitUntilMetadataIsPropagated(topic, p) + } + } + + def getAllTopicsAndPartitionSize(): Seq[(String, Int)] = { + zkUtils.getPartitionsForTopics(zkUtils.getAllTopics()).mapValues(_.size).toSeq + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String): Unit = { + createTopic(topic, 1) + } + + /** Delete a Kafka topic and wait until it is propagated to the whole cluster */ + def deleteTopic(topic: String): Unit = { + val partitions = zkUtils.getPartitionsForTopics(Seq(topic))(topic).size + AdminUtils.deleteTopic(zkUtils, topic) + verifyTopicDeletionWithRetries(zkUtils, topic, partitions, List(this.server)) + } + + /** Add new paritions to a Kafka topic */ + def addPartitions(topic: String, partitions: Int): Unit = { + AdminUtils.addPartitions(zkUtils, topic, partitions) + // wait until metadata is propagated + (0 until partitions).foreach { p => + waitUntilMetadataIsPropagated(topic, p) + } + } + + /** Java-friendly function for sending messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { + sendMessages(topic, Map(messageToFreq.asScala.mapValues(_.intValue()).toSeq: _*)) + } + + /** Send the messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = { + val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray + sendMessages(topic, messages) + } + + /** Send the array of messages to the Kafka broker */ + def sendMessages(topic: String, messages: Array[String]): Seq[(String, RecordMetadata)] = { + sendMessages(topic, messages, None) + } + + /** Send the array of messages to the Kafka broker using specified partition */ + def sendMessages( + topic: String, + messages: Array[String], + partition: Option[Int]): Seq[(String, RecordMetadata)] = { + producer = new KafkaProducer[String, String](producerConfiguration) + val offsets = try { + messages.map { m => + val record = partition match { + case Some(p) => new ProducerRecord[String, String](topic, p, null, m) + case None => new ProducerRecord[String, String](topic, m) + } + val metadata = + producer.send(record).get(10, TimeUnit.SECONDS) + logInfo(s"\tSent $m to partition ${metadata.partition}, offset ${metadata.offset}") + (m, metadata) + } + } finally { + if (producer != null) { + producer.close() + producer = null + } + } + offsets + } + + def cleanupLogs(): Unit = { + server.logManager.cleanupLogs() + } + + def getEarliestOffsets(topics: Set[String]): Map[TopicPartition, Long] = { + val kc = new KafkaConsumer[String, String](consumerConfiguration) + logInfo("Created consumer to get earliest offsets") + kc.subscribe(topics.asJavaCollection) + kc.poll(0) + val partitions = kc.assignment() + kc.pause(partitions) + kc.seekToBeginning(partitions) + val offsets = partitions.asScala.map(p => p -> kc.position(p)).toMap + kc.close() + logInfo("Closed consumer to get earliest offsets") + offsets + } + + def getLatestOffsets(topics: Set[String]): Map[TopicPartition, Long] = { + val kc = new KafkaConsumer[String, String](consumerConfiguration) + logInfo("Created consumer to get latest offsets") + kc.subscribe(topics.asJavaCollection) + kc.poll(0) + val partitions = kc.assignment() + kc.pause(partitions) + kc.seekToEnd(partitions) + val offsets = partitions.asScala.map(p => p -> kc.position(p)).toMap + kc.close() + logInfo("Closed consumer to get latest offsets") + offsets + } + + protected def brokerConfiguration: Properties = { + val props = new Properties() + props.put("broker.id", "0") + props.put("host.name", "localhost") + props.put("advertised.host.name", "localhost") + props.put("port", brokerPort.toString) + props.put("log.dir", Utils.createTempDir().getAbsolutePath) + props.put("zookeeper.connect", zkAddress) + props.put("log.flush.interval.messages", "1") + props.put("replica.socket.timeout.ms", "1500") + props.put("delete.topic.enable", "true") + props.put("offsets.topic.num.partitions", "1") + props.putAll(withBrokerProps.asJava) + props + } + + private def producerConfiguration: Properties = { + val props = new Properties() + props.put("bootstrap.servers", brokerAddress) + props.put("value.serializer", classOf[StringSerializer].getName) + props.put("key.serializer", classOf[StringSerializer].getName) + // wait for all in-sync replicas to ack sends + props.put("acks", "all") + props + } + + private def consumerConfiguration: Properties = { + val props = new Properties() + props.put("bootstrap.servers", brokerAddress) + props.put("group.id", "group-KafkaTestUtils-" + Random.nextInt) + props.put("value.deserializer", classOf[StringDeserializer].getName) + props.put("key.deserializer", classOf[StringDeserializer].getName) + props.put("enable.auto.commit", "false") + props + } + + /** Verify topic is deleted in all places, e.g, brokers, zookeeper. */ + private def verifyTopicDeletion( + topic: String, + numPartitions: Int, + servers: Seq[KafkaServer]): Unit = { + val topicAndPartitions = (0 until numPartitions).map(TopicAndPartition(topic, _)) + + import ZkUtils._ + // wait until admin path for delete topic is deleted, signaling completion of topic deletion + assert( + !zkUtils.pathExists(getDeleteTopicPath(topic)), + s"${getDeleteTopicPath(topic)} still exists") + assert(!zkUtils.pathExists(getTopicPath(topic)), s"${getTopicPath(topic)} still exists") + // ensure that the topic-partition has been deleted from all brokers' replica managers + assert(servers.forall(server => topicAndPartitions.forall(tp => + server.replicaManager.getPartition(tp.topic, tp.partition) == None)), + s"topic $topic still exists in the replica manager") + // ensure that logs from all replicas are deleted if delete topic is marked successful + assert(servers.forall(server => topicAndPartitions.forall(tp => + server.getLogManager().getLog(tp).isEmpty)), + s"topic $topic still exists in log mananger") + // ensure that topic is removed from all cleaner offsets + assert(servers.forall(server => topicAndPartitions.forall { tp => + val checkpoints = server.getLogManager().logDirs.map { logDir => + new OffsetCheckpoint(new File(logDir, "cleaner-offset-checkpoint")).read() + } + checkpoints.forall(checkpointsPerLogDir => !checkpointsPerLogDir.contains(tp)) + }), s"checkpoint for topic $topic still exists") + // ensure the topic is gone + assert( + !zkUtils.getAllTopics().contains(topic), + s"topic $topic still exists on zookeeper") + } + + /** Verify topic is deleted. Retry to delete the topic if not. */ + private def verifyTopicDeletionWithRetries( + zkUtils: ZkUtils, + topic: String, + numPartitions: Int, + servers: Seq[KafkaServer]) { + eventually(timeout(60.seconds), interval(200.millis)) { + try { + verifyTopicDeletion(topic, numPartitions, servers) + } catch { + case e: Throwable => + // As pushing messages into Kafka updates Zookeeper asynchronously, there is a small + // chance that a topic will be recreated after deletion due to the asynchronous update. + // Hence, delete the topic and retry. + AdminUtils.deleteTopic(zkUtils, topic) + throw e + } + } + } + + private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { + def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { + case Some(partitionState) => + val leaderAndInSyncReplicas = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr + + zkUtils.getLeaderForPartition(topic, partition).isDefined && + Request.isValidBrokerId(leaderAndInSyncReplicas.leader) && + leaderAndInSyncReplicas.isr.nonEmpty + + case _ => + false + } + eventually(timeout(60.seconds)) { + assert(isPropagated, s"Partition [$topic, $partition] metadata not propagated after timeout") + } + } + + private class EmbeddedZookeeper(val zkConnect: String) { + val snapshotDir = Utils.createTempDir() + val logDir = Utils.createTempDir() + + val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) + val (ip, port) = { + val splits = zkConnect.split(":") + (splits(0), splits(1).toInt) + } + val factory = new NIOServerCnxnFactory() + factory.configure(new InetSocketAddress(ip, port), 16) + factory.startup(zookeeper) + + val actualPort = factory.getLocalPort + + def shutdown() { + factory.shutdown() + // The directories are not closed even if the ZooKeeper server is shut down. + // Please see ZOOKEEPER-1844, which is fixed in 3.4.6+. It leads to test failures + // on Windows if the directory deletion failure throws an exception. + try { + Utils.deleteRecursively(snapshotDir) + } catch { + case e: IOException if Utils.isWindows => + logWarning(e.getMessage) + } + try { + Utils.deleteRecursively(logDir) + } catch { + case e: IOException if Utils.isWindows => + logWarning(e.getMessage) + } + } + } +} + diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml new file mode 100644 index 000000000000..6eb7ba5f0092 --- /dev/null +++ b/external/kafka-0-10/pom.xml @@ -0,0 +1,105 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-streaming-kafka-0-10_2.11 + + streaming-kafka-0-10 + 0.10.0.1 + + jar + Spark Integration for Kafka 0.10 + http://spark.apache.org/ + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.kafka + kafka-clients + ${kafka.version} + + + org.apache.kafka + kafka_${scala.binary.version} + ${kafka.version} + test + + + net.sf.jopt-simple + jopt-simple + 3.2 + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + + + scala-2.12 + + 0.10.1.1 + + + + + diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/CachedKafkaConsumer.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/CachedKafkaConsumer.scala new file mode 100644 index 000000000000..fa3ea6131a50 --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/CachedKafkaConsumer.scala @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.{ util => ju } + +import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord, KafkaConsumer } +import org.apache.kafka.common.{ KafkaException, TopicPartition } + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging + + +/** + * Consumer of single topicpartition, intended for cached reuse. + * Underlying consumer is not threadsafe, so neither is this, + * but processing the same topicpartition and group id in multiple threads is usually bad anyway. + */ +private[kafka010] +class CachedKafkaConsumer[K, V] private( + val groupId: String, + val topic: String, + val partition: Int, + val kafkaParams: ju.Map[String, Object]) extends Logging { + + assert(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG), + "groupId used for cache key must match the groupId in kafkaParams") + + val topicPartition = new TopicPartition(topic, partition) + + protected val consumer = { + val c = new KafkaConsumer[K, V](kafkaParams) + val tps = new ju.ArrayList[TopicPartition]() + tps.add(topicPartition) + c.assign(tps) + c + } + + // TODO if the buffer was kept around as a random-access structure, + // could possibly optimize re-calculating of an RDD in the same batch + protected var buffer = ju.Collections.emptyList[ConsumerRecord[K, V]]().iterator + protected var nextOffset = -2L + + def close(): Unit = consumer.close() + + /** + * Get the record for the given offset, waiting up to timeout ms if IO is necessary. + * Sequential forward access will use buffers, but random access will be horribly inefficient. + */ + def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = { + logDebug(s"Get $groupId $topic $partition nextOffset $nextOffset requested $offset") + if (offset != nextOffset) { + logInfo(s"Initial fetch for $groupId $topic $partition $offset") + seek(offset) + poll(timeout) + } + + if (!buffer.hasNext()) { poll(timeout) } + assert(buffer.hasNext(), + s"Failed to get records for $groupId $topic $partition $offset after polling for $timeout") + var record = buffer.next() + + if (record.offset != offset) { + logInfo(s"Buffer miss for $groupId $topic $partition $offset") + seek(offset) + poll(timeout) + assert(buffer.hasNext(), + s"Failed to get records for $groupId $topic $partition $offset after polling for $timeout") + record = buffer.next() + assert(record.offset == offset, + s"Got wrong record for $groupId $topic $partition even after seeking to offset $offset") + } + + nextOffset = offset + 1 + record + } + + private def seek(offset: Long): Unit = { + logDebug(s"Seeking to $topicPartition $offset") + consumer.seek(topicPartition, offset) + } + + private def poll(timeout: Long): Unit = { + val p = consumer.poll(timeout) + val r = p.records(topicPartition) + logDebug(s"Polled ${p.partitions()} ${r.size}") + buffer = r.iterator + } + +} + +private[kafka010] +object CachedKafkaConsumer extends Logging { + + private case class CacheKey(groupId: String, topic: String, partition: Int) + + // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap + private var cache: ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]] = null + + /** Must be called before get, once per JVM, to configure the cache. Further calls are ignored */ + def init( + initialCapacity: Int, + maxCapacity: Int, + loadFactor: Float): Unit = CachedKafkaConsumer.synchronized { + if (null == cache) { + logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor") + cache = new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]]( + initialCapacity, loadFactor, true) { + override def removeEldestEntry( + entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer[_, _]]): Boolean = { + if (this.size > maxCapacity) { + try { + entry.getValue.consumer.close() + } catch { + case x: KafkaException => + logError("Error closing oldest Kafka consumer", x) + } + true + } else { + false + } + } + } + } + } + + /** + * Get a cached consumer for groupId, assigned to topic and partition. + * If matching consumer doesn't already exist, will be created using kafkaParams. + */ + def get[K, V]( + groupId: String, + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = + CachedKafkaConsumer.synchronized { + val k = CacheKey(groupId, topic, partition) + val v = cache.get(k) + if (null == v) { + logInfo(s"Cache miss for $k") + logDebug(cache.keySet.toString) + val c = new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) + cache.put(k, c) + c + } else { + // any given topicpartition should have a consistent key and value type + v.asInstanceOf[CachedKafkaConsumer[K, V]] + } + } + + /** + * Get a fresh new instance, unassociated with the global cache. + * Caller is responsible for closing + */ + def getUncached[K, V]( + groupId: String, + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = + new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) + + /** remove consumer for given groupId, topic, and partition, if it exists */ + def remove(groupId: String, topic: String, partition: Int): Unit = { + val k = CacheKey(groupId, topic, partition) + logInfo(s"Removing $k from cache") + val v = CachedKafkaConsumer.synchronized { + cache.remove(k) + } + if (null != v) { + v.close() + logInfo(s"Removed $k from cache") + } + } +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala new file mode 100644 index 000000000000..d2100fc5a4ab --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala @@ -0,0 +1,480 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.{lang => jl, util => ju} +import java.util.Locale + +import scala.collection.JavaConverters._ + +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.annotation.Experimental +import org.apache.spark.internal.Logging + +/** + * :: Experimental :: + * Choice of how to create and configure underlying Kafka Consumers on driver and executors. + * See [[ConsumerStrategies]] to obtain instances. + * Kafka 0.10 consumers can require additional, sometimes complex, setup after object + * instantiation. This interface encapsulates that process, and allows it to be checkpointed. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ +@Experimental +abstract class ConsumerStrategy[K, V] { + /** + * Kafka + * configuration parameters to be used on executors. Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + def executorKafkaParams: ju.Map[String, Object] + + /** + * Must return a fully configured Kafka Consumer, including subscribed or assigned topics. + * See Kafka docs. + * This consumer will be used on the driver to query for offsets only, not messages. + * The consumer must be returned in a state that it is safe to call poll(0) on. + * @param currentOffsets A map from TopicPartition to offset, indicating how far the driver + * has successfully read. Will be empty on initial start, possibly non-empty on restart from + * checkpoint. + */ + def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] +} + +/** + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ +private case class Subscribe[K, V]( + topics: ju.Collection[jl.String], + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, jl.Long] + ) extends ConsumerStrategy[K, V] with Logging { + + def executorKafkaParams: ju.Map[String, Object] = kafkaParams + + def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] = { + val consumer = new KafkaConsumer[K, V](kafkaParams) + consumer.subscribe(topics) + val toSeek = if (currentOffsets.isEmpty) { + offsets + } else { + currentOffsets + } + if (!toSeek.isEmpty) { + // work around KAFKA-3370 when reset is none + // poll will throw if no position, i.e. auto offset reset none and no explicit position + // but cant seek to a position before poll, because poll is what gets subscription partitions + // So, poll, suppress the first exception, then seek + val aor = kafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG) + val shouldSuppress = + aor != null && aor.asInstanceOf[String].toUpperCase(Locale.ROOT) == "NONE" + try { + consumer.poll(0) + } catch { + case x: NoOffsetForPartitionException if shouldSuppress => + logWarning("Catching NoOffsetForPartitionException since " + + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + " is none. See KAFKA-3370") + } + toSeek.asScala.foreach { case (topicPartition, offset) => + consumer.seek(topicPartition, offset) + } + // we've called poll, we must pause or next poll may consume messages and set position + consumer.pause(consumer.assignment()) + } + + consumer + } +} + +/** + * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. + * The pattern matching will be done periodically against topics existing at the time of check. + * @param pattern pattern to subscribe to + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ +private case class SubscribePattern[K, V]( + pattern: ju.regex.Pattern, + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, jl.Long] + ) extends ConsumerStrategy[K, V] with Logging { + + def executorKafkaParams: ju.Map[String, Object] = kafkaParams + + def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] = { + val consumer = new KafkaConsumer[K, V](kafkaParams) + consumer.subscribe(pattern, new NoOpConsumerRebalanceListener()) + val toSeek = if (currentOffsets.isEmpty) { + offsets + } else { + currentOffsets + } + if (!toSeek.isEmpty) { + // work around KAFKA-3370 when reset is none, see explanation in Subscribe above + val aor = kafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG) + val shouldSuppress = + aor != null && aor.asInstanceOf[String].toUpperCase(Locale.ROOT) == "NONE" + try { + consumer.poll(0) + } catch { + case x: NoOffsetForPartitionException if shouldSuppress => + logWarning("Catching NoOffsetForPartitionException since " + + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + " is none. See KAFKA-3370") + } + toSeek.asScala.foreach { case (topicPartition, offset) => + consumer.seek(topicPartition, offset) + } + // we've called poll, we must pause or next poll may consume messages and set position + consumer.pause(consumer.assignment()) + } + + consumer + } +} + +/** + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ +private case class Assign[K, V]( + topicPartitions: ju.Collection[TopicPartition], + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, jl.Long] + ) extends ConsumerStrategy[K, V] { + + def executorKafkaParams: ju.Map[String, Object] = kafkaParams + + def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] = { + val consumer = new KafkaConsumer[K, V](kafkaParams) + consumer.assign(topicPartitions) + val toSeek = if (currentOffsets.isEmpty) { + offsets + } else { + currentOffsets + } + if (!toSeek.isEmpty) { + // this doesn't need a KAFKA-3370 workaround, because partitions are known, no poll needed + toSeek.asScala.foreach { case (topicPartition, offset) => + consumer.seek(topicPartition, offset) + } + } + + consumer + } +} + +/** + * :: Experimental :: + * object for obtaining instances of [[ConsumerStrategy]] + */ +@Experimental +object ConsumerStrategies { + /** + * :: Experimental :: + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + @Experimental + def Subscribe[K, V]( + topics: Iterable[jl.String], + kafkaParams: collection.Map[String, Object], + offsets: collection.Map[TopicPartition, Long]): ConsumerStrategy[K, V] = { + new Subscribe[K, V]( + new ju.ArrayList(topics.asJavaCollection), + new ju.HashMap[String, Object](kafkaParams.asJava), + new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(l => new jl.Long(l)).asJava)) + } + + /** + * :: Experimental :: + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + @Experimental + def Subscribe[K, V]( + topics: Iterable[jl.String], + kafkaParams: collection.Map[String, Object]): ConsumerStrategy[K, V] = { + new Subscribe[K, V]( + new ju.ArrayList(topics.asJavaCollection), + new ju.HashMap[String, Object](kafkaParams.asJava), + ju.Collections.emptyMap[TopicPartition, jl.Long]()) + } + + /** + * :: Experimental :: + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + @Experimental + def Subscribe[K, V]( + topics: ju.Collection[jl.String], + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, jl.Long]): ConsumerStrategy[K, V] = { + new Subscribe[K, V](topics, kafkaParams, offsets) + } + + /** + * :: Experimental :: + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + @Experimental + def Subscribe[K, V]( + topics: ju.Collection[jl.String], + kafkaParams: ju.Map[String, Object]): ConsumerStrategy[K, V] = { + new Subscribe[K, V](topics, kafkaParams, ju.Collections.emptyMap[TopicPartition, jl.Long]()) + } + + /** :: Experimental :: + * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. + * The pattern matching will be done periodically against topics existing at the time of check. + * @param pattern pattern to subscribe to + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + @Experimental + def SubscribePattern[K, V]( + pattern: ju.regex.Pattern, + kafkaParams: collection.Map[String, Object], + offsets: collection.Map[TopicPartition, Long]): ConsumerStrategy[K, V] = { + new SubscribePattern[K, V]( + pattern, + new ju.HashMap[String, Object](kafkaParams.asJava), + new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(l => new jl.Long(l)).asJava)) + } + + /** :: Experimental :: + * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. + * The pattern matching will be done periodically against topics existing at the time of check. + * @param pattern pattern to subscribe to + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + @Experimental + def SubscribePattern[K, V]( + pattern: ju.regex.Pattern, + kafkaParams: collection.Map[String, Object]): ConsumerStrategy[K, V] = { + new SubscribePattern[K, V]( + pattern, + new ju.HashMap[String, Object](kafkaParams.asJava), + ju.Collections.emptyMap[TopicPartition, jl.Long]()) + } + + /** :: Experimental :: + * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. + * The pattern matching will be done periodically against topics existing at the time of check. + * @param pattern pattern to subscribe to + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + @Experimental + def SubscribePattern[K, V]( + pattern: ju.regex.Pattern, + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, jl.Long]): ConsumerStrategy[K, V] = { + new SubscribePattern[K, V](pattern, kafkaParams, offsets) + } + + /** :: Experimental :: + * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. + * The pattern matching will be done periodically against topics existing at the time of check. + * @param pattern pattern to subscribe to + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + @Experimental + def SubscribePattern[K, V]( + pattern: ju.regex.Pattern, + kafkaParams: ju.Map[String, Object]): ConsumerStrategy[K, V] = { + new SubscribePattern[K, V]( + pattern, + kafkaParams, + ju.Collections.emptyMap[TopicPartition, jl.Long]()) + } + + /** + * :: Experimental :: + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + @Experimental + def Assign[K, V]( + topicPartitions: Iterable[TopicPartition], + kafkaParams: collection.Map[String, Object], + offsets: collection.Map[TopicPartition, Long]): ConsumerStrategy[K, V] = { + new Assign[K, V]( + new ju.ArrayList(topicPartitions.asJavaCollection), + new ju.HashMap[String, Object](kafkaParams.asJava), + new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(l => new jl.Long(l)).asJava)) + } + + /** + * :: Experimental :: + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + @Experimental + def Assign[K, V]( + topicPartitions: Iterable[TopicPartition], + kafkaParams: collection.Map[String, Object]): ConsumerStrategy[K, V] = { + new Assign[K, V]( + new ju.ArrayList(topicPartitions.asJavaCollection), + new ju.HashMap[String, Object](kafkaParams.asJava), + ju.Collections.emptyMap[TopicPartition, jl.Long]()) + } + + /** + * :: Experimental :: + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + @Experimental + def Assign[K, V]( + topicPartitions: ju.Collection[TopicPartition], + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, jl.Long]): ConsumerStrategy[K, V] = { + new Assign[K, V](topicPartitions, kafkaParams, offsets) + } + + /** + * :: Experimental :: + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + @Experimental + def Assign[K, V]( + topicPartitions: ju.Collection[TopicPartition], + kafkaParams: ju.Map[String, Object]): ConsumerStrategy[K, V] = { + new Assign[K, V]( + topicPartitions, + kafkaParams, + ju.Collections.emptyMap[TopicPartition, jl.Long]()) + } + +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala new file mode 100644 index 000000000000..0fa3287f36db --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -0,0 +1,333 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.{ util => ju } +import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.atomic.AtomicReference + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.internal.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{StreamingContext, Time} +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.scheduler.{RateController, StreamInputInfo} +import org.apache.spark.streaming.scheduler.rate.RateEstimator + +/** + * A DStream where + * each given Kafka topic/partition corresponds to an RDD partition. + * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number + * of messages + * per second that each '''partition''' will accept. + * @param locationStrategy In most cases, pass in [[LocationStrategies.PreferConsistent]], + * see [[LocationStrategy]] for more details. + * @param consumerStrategy In most cases, pass in [[ConsumerStrategies.Subscribe]], + * see [[ConsumerStrategy]] for more details + * @param ppc configuration of settings such as max rate on a per-partition basis. + * see [[PerPartitionConfig]] for more details. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ +private[spark] class DirectKafkaInputDStream[K, V]( + _ssc: StreamingContext, + locationStrategy: LocationStrategy, + consumerStrategy: ConsumerStrategy[K, V], + ppc: PerPartitionConfig + ) extends InputDStream[ConsumerRecord[K, V]](_ssc) with Logging with CanCommitOffsets { + + val executorKafkaParams = { + val ekp = new ju.HashMap[String, Object](consumerStrategy.executorKafkaParams) + KafkaUtils.fixKafkaParams(ekp) + ekp + } + + protected var currentOffsets = Map[TopicPartition, Long]() + + @transient private var kc: Consumer[K, V] = null + def consumer(): Consumer[K, V] = this.synchronized { + if (null == kc) { + kc = consumerStrategy.onStart(currentOffsets.mapValues(l => new java.lang.Long(l)).asJava) + } + kc + } + + override def persist(newLevel: StorageLevel): DStream[ConsumerRecord[K, V]] = { + logError("Kafka ConsumerRecord is not serializable. " + + "Use .map to extract fields before calling .persist or .window") + super.persist(newLevel) + } + + protected def getBrokers = { + val c = consumer + val result = new ju.HashMap[TopicPartition, String]() + val hosts = new ju.HashMap[TopicPartition, String]() + val assignments = c.assignment().iterator() + while (assignments.hasNext()) { + val tp: TopicPartition = assignments.next() + if (null == hosts.get(tp)) { + val infos = c.partitionsFor(tp.topic).iterator() + while (infos.hasNext()) { + val i = infos.next() + hosts.put(new TopicPartition(i.topic(), i.partition()), i.leader.host()) + } + } + result.put(tp, hosts.get(tp)) + } + result + } + + protected def getPreferredHosts: ju.Map[TopicPartition, String] = { + locationStrategy match { + case PreferBrokers => getBrokers + case PreferConsistent => ju.Collections.emptyMap[TopicPartition, String]() + case PreferFixed(hostMap) => hostMap + } + } + + // Keep this consistent with how other streams are named (e.g. "Flume polling stream [2]") + private[streaming] override def name: String = s"Kafka 0.10 direct stream [$id]" + + protected[streaming] override val checkpointData = + new DirectKafkaInputDStreamCheckpointData + + + /** + * Asynchronously maintains & sends new rate limits to the receiver through the receiver tracker. + */ + override protected[streaming] val rateController: Option[RateController] = { + if (RateController.isBackPressureEnabled(ssc.conf)) { + Some(new DirectKafkaRateController(id, + RateEstimator.create(ssc.conf, context.graph.batchDuration))) + } else { + None + } + } + + protected[streaming] def maxMessagesPerPartition( + offsets: Map[TopicPartition, Long]): Option[Map[TopicPartition, Long]] = { + val estimatedRateLimit = rateController.map(_.getLatestRate()) + + // calculate a per-partition rate limit based on current lag + val effectiveRateLimitPerPartition = estimatedRateLimit.filter(_ > 0) match { + case Some(rate) => + val lagPerPartition = offsets.map { case (tp, offset) => + tp -> Math.max(offset - currentOffsets(tp), 0) + } + val totalLag = lagPerPartition.values.sum + + lagPerPartition.map { case (tp, lag) => + val maxRateLimitPerPartition = ppc.maxRatePerPartition(tp) + val backpressureRate = Math.round(lag / totalLag.toFloat * rate) + tp -> (if (maxRateLimitPerPartition > 0) { + Math.min(backpressureRate, maxRateLimitPerPartition)} else backpressureRate) + } + case None => offsets.map { case (tp, offset) => tp -> ppc.maxRatePerPartition(tp) } + } + + if (effectiveRateLimitPerPartition.values.sum > 0) { + val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000 + Some(effectiveRateLimitPerPartition.map { + case (tp, limit) => tp -> (secsPerBatch * limit).toLong + }) + } else { + None + } + } + + /** + * The concern here is that poll might consume messages despite being paused, + * which would throw off consumer position. Fix position if this happens. + */ + private def paranoidPoll(c: Consumer[K, V]): Unit = { + val msgs = c.poll(0) + if (!msgs.isEmpty) { + // position should be minimum offset per topicpartition + msgs.asScala.foldLeft(Map[TopicPartition, Long]()) { (acc, m) => + val tp = new TopicPartition(m.topic, m.partition) + val off = acc.get(tp).map(o => Math.min(o, m.offset)).getOrElse(m.offset) + acc + (tp -> off) + }.foreach { case (tp, off) => + logInfo(s"poll(0) returned messages, seeking $tp to $off to compensate") + c.seek(tp, off) + } + } + } + + /** + * Returns the latest (highest) available offsets, taking new partitions into account. + */ + protected def latestOffsets(): Map[TopicPartition, Long] = { + val c = consumer + paranoidPoll(c) + val parts = c.assignment().asScala + + // make sure new partitions are reflected in currentOffsets + val newPartitions = parts.diff(currentOffsets.keySet) + // position for new partitions determined by auto.offset.reset if no commit + currentOffsets = currentOffsets ++ newPartitions.map(tp => tp -> c.position(tp)).toMap + // don't want to consume messages, so pause + c.pause(newPartitions.asJava) + // find latest available offsets + c.seekToEnd(currentOffsets.keySet.asJava) + parts.map(tp => tp -> c.position(tp)).toMap + } + + // limits the maximum number of messages per partition + protected def clamp( + offsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + + maxMessagesPerPartition(offsets).map { mmp => + mmp.map { case (tp, messages) => + val uo = offsets(tp) + tp -> Math.min(currentOffsets(tp) + messages, uo) + } + }.getOrElse(offsets) + } + + override def compute(validTime: Time): Option[KafkaRDD[K, V]] = { + val untilOffsets = clamp(latestOffsets()) + val offsetRanges = untilOffsets.map { case (tp, uo) => + val fo = currentOffsets(tp) + OffsetRange(tp.topic, tp.partition, fo, uo) + } + val useConsumerCache = context.conf.getBoolean("spark.streaming.kafka.consumer.cache.enabled", + true) + val rdd = new KafkaRDD[K, V](context.sparkContext, executorKafkaParams, offsetRanges.toArray, + getPreferredHosts, useConsumerCache) + + // Report the record number and metadata of this batch interval to InputInfoTracker. + val description = offsetRanges.filter { offsetRange => + // Don't display empty ranges. + offsetRange.fromOffset != offsetRange.untilOffset + }.map { offsetRange => + s"topic: ${offsetRange.topic}\tpartition: ${offsetRange.partition}\t" + + s"offsets: ${offsetRange.fromOffset} to ${offsetRange.untilOffset}" + }.mkString("\n") + // Copy offsetRanges to immutable.List to prevent from being modified by the user + val metadata = Map( + "offsets" -> offsetRanges.toList, + StreamInputInfo.METADATA_KEY_DESCRIPTION -> description) + val inputInfo = StreamInputInfo(id, rdd.count, metadata) + ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) + + currentOffsets = untilOffsets + commitAll() + Some(rdd) + } + + override def start(): Unit = { + val c = consumer + paranoidPoll(c) + if (currentOffsets.isEmpty) { + currentOffsets = c.assignment().asScala.map { tp => + tp -> c.position(tp) + }.toMap + } + + // don't actually want to consume any messages, so pause all partitions + c.pause(currentOffsets.keySet.asJava) + } + + override def stop(): Unit = this.synchronized { + if (kc != null) { + kc.close() + } + } + + protected val commitQueue = new ConcurrentLinkedQueue[OffsetRange] + protected val commitCallback = new AtomicReference[OffsetCommitCallback] + + /** + * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. + * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. + */ + def commitAsync(offsetRanges: Array[OffsetRange]): Unit = { + commitAsync(offsetRanges, null) + } + + /** + * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. + * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. + * @param callback Only the most recently provided callback will be used at commit. + */ + def commitAsync(offsetRanges: Array[OffsetRange], callback: OffsetCommitCallback): Unit = { + commitCallback.set(callback) + commitQueue.addAll(ju.Arrays.asList(offsetRanges: _*)) + } + + protected def commitAll(): Unit = { + val m = new ju.HashMap[TopicPartition, OffsetAndMetadata]() + var osr = commitQueue.poll() + while (null != osr) { + val tp = osr.topicPartition + val x = m.get(tp) + val offset = if (null == x) { osr.untilOffset } else { Math.max(x.offset, osr.untilOffset) } + m.put(tp, new OffsetAndMetadata(offset)) + osr = commitQueue.poll() + } + if (!m.isEmpty) { + consumer.commitAsync(m, commitCallback.get) + } + } + + private[streaming] + class DirectKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) { + def batchForTime: mutable.HashMap[Time, Array[(String, Int, Long, Long)]] = { + data.asInstanceOf[mutable.HashMap[Time, Array[OffsetRange.OffsetRangeTuple]]] + } + + override def update(time: Time): Unit = { + batchForTime.clear() + generatedRDDs.foreach { kv => + val a = kv._2.asInstanceOf[KafkaRDD[K, V]].offsetRanges.map(_.toTuple).toArray + batchForTime += kv._1 -> a + } + } + + override def cleanup(time: Time): Unit = { } + + override def restore(): Unit = { + batchForTime.toSeq.sortBy(_._1)(Time.ordering).foreach { case (t, b) => + logInfo(s"Restoring KafkaRDD for time $t ${b.mkString("[", ", ", "]")}") + generatedRDDs += t -> new KafkaRDD[K, V]( + context.sparkContext, + executorKafkaParams, + b.map(OffsetRange(_)), + getPreferredHosts, + // during restore, it's possible same partition will be consumed from multiple + // threads, so do not use cache. + false + ) + } + } + } + + /** + * A RateController to retrieve the rate from RateEstimator. + */ + private[streaming] class DirectKafkaRateController(id: Int, estimator: RateEstimator) + extends RateController(id, estimator) { + override def publish(rate: Long): Unit = () + } +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala new file mode 100644 index 000000000000..d9fc9cc20664 --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala @@ -0,0 +1,228 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka010 + +import java.{ util => ju } + +import scala.collection.mutable.ArrayBuffer + +import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord } +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.ExecutorCacheTaskLocation +import org.apache.spark.storage.StorageLevel + +/** + * A batch-oriented interface for consuming from Kafka. + * Starting and ending offsets are specified in advance, + * so that you can control exactly-once semantics. + * @param kafkaParams Kafka + * + * configuration parameters. Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD + * @param preferredHosts map from TopicPartition to preferred host for processing that partition. + * In most cases, use [[LocationStrategies.PreferConsistent]] + * Use [[LocationStrategies.PreferBrokers]] if your executors are on same nodes as brokers. + * @param useConsumerCache whether to use a consumer from a per-jvm cache + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ +private[spark] class KafkaRDD[K, V]( + sc: SparkContext, + val kafkaParams: ju.Map[String, Object], + val offsetRanges: Array[OffsetRange], + val preferredHosts: ju.Map[TopicPartition, String], + useConsumerCache: Boolean +) extends RDD[ConsumerRecord[K, V]](sc, Nil) with Logging with HasOffsetRanges { + + assert("none" == + kafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).asInstanceOf[String], + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + + " must be set to none for executor kafka params, else messages may not match offsetRange") + + assert(false == + kafkaParams.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).asInstanceOf[Boolean], + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG + + " must be set to false for executor kafka params, else offsets may commit before processing") + + // TODO is it necessary to have separate configs for initial poll time vs ongoing poll time? + private val pollTimeout = conf.getLong("spark.streaming.kafka.consumer.poll.ms", + conf.getTimeAsMs("spark.network.timeout", "120s")) + private val cacheInitialCapacity = + conf.getInt("spark.streaming.kafka.consumer.cache.initialCapacity", 16) + private val cacheMaxCapacity = + conf.getInt("spark.streaming.kafka.consumer.cache.maxCapacity", 64) + private val cacheLoadFactor = + conf.getDouble("spark.streaming.kafka.consumer.cache.loadFactor", 0.75).toFloat + + override def persist(newLevel: StorageLevel): this.type = { + logError("Kafka ConsumerRecord is not serializable. " + + "Use .map to extract fields before calling .persist or .window") + super.persist(newLevel) + } + + override def getPartitions: Array[Partition] = { + offsetRanges.zipWithIndex.map { case (o, i) => + new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset) + }.toArray + } + + override def count(): Long = offsetRanges.map(_.count).sum + + override def countApprox( + timeout: Long, + confidence: Double = 0.95 + ): PartialResult[BoundedDouble] = { + val c = count + new PartialResult(new BoundedDouble(c, 1.0, c, c), true) + } + + override def isEmpty(): Boolean = count == 0L + + override def take(num: Int): Array[ConsumerRecord[K, V]] = { + val nonEmptyPartitions = this.partitions + .map(_.asInstanceOf[KafkaRDDPartition]) + .filter(_.count > 0) + + if (num < 1 || nonEmptyPartitions.isEmpty) { + return new Array[ConsumerRecord[K, V]](0) + } + + // Determine in advance how many messages need to be taken from each partition + val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) => + val remain = num - result.values.sum + if (remain > 0) { + val taken = Math.min(remain, part.count) + result + (part.index -> taken.toInt) + } else { + result + } + } + + val buf = new ArrayBuffer[ConsumerRecord[K, V]] + val res = context.runJob( + this, + (tc: TaskContext, it: Iterator[ConsumerRecord[K, V]]) => + it.take(parts(tc.partitionId)).toArray, parts.keys.toArray + ) + res.foreach(buf ++= _) + buf.toArray + } + + private def executors(): Array[ExecutorCacheTaskLocation] = { + val bm = sparkContext.env.blockManager + bm.master.getPeers(bm.blockManagerId).toArray + .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) + .sortWith(compareExecutors) + } + + protected[kafka010] def compareExecutors( + a: ExecutorCacheTaskLocation, + b: ExecutorCacheTaskLocation): Boolean = + if (a.host == b.host) { + a.executorId > b.executorId + } else { + a.host > b.host + } + + override def getPreferredLocations(thePart: Partition): Seq[String] = { + // The intention is best-effort consistent executor for a given topicpartition, + // so that caching consumers can be effective. + // TODO what about hosts specified by ip vs name + val part = thePart.asInstanceOf[KafkaRDDPartition] + val allExecs = executors() + val tp = part.topicPartition + val prefHost = preferredHosts.get(tp) + val prefExecs = if (null == prefHost) allExecs else allExecs.filter(_.host == prefHost) + val execs = if (prefExecs.isEmpty) allExecs else prefExecs + if (execs.isEmpty) { + Seq.empty + } else { + // execs is sorted, tp.hashCode depends only on topic and partition, so consistent index + val index = Math.floorMod(tp.hashCode, execs.length) + val chosen = execs(index) + Seq(chosen.toString) + } + } + + private def errBeginAfterEnd(part: KafkaRDDPartition): String = + s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " + + s"for topic ${part.topic} partition ${part.partition}. " + + "You either provided an invalid fromOffset, or the Kafka topic has been damaged" + + override def compute(thePart: Partition, context: TaskContext): Iterator[ConsumerRecord[K, V]] = { + val part = thePart.asInstanceOf[KafkaRDDPartition] + assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part)) + if (part.fromOffset == part.untilOffset) { + logInfo(s"Beginning offset ${part.fromOffset} is the same as ending offset " + + s"skipping ${part.topic} ${part.partition}") + Iterator.empty + } else { + new KafkaRDDIterator(part, context) + } + } + + /** + * An iterator that fetches messages directly from Kafka for the offsets in partition. + * Uses a cached consumer where possible to take advantage of prefetching + */ + private class KafkaRDDIterator( + part: KafkaRDDPartition, + context: TaskContext) extends Iterator[ConsumerRecord[K, V]] { + + logInfo(s"Computing topic ${part.topic}, partition ${part.partition} " + + s"offsets ${part.fromOffset} -> ${part.untilOffset}") + + val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] + + context.addTaskCompletionListener{ context => closeIfNeeded() } + + val consumer = if (useConsumerCache) { + CachedKafkaConsumer.init(cacheInitialCapacity, cacheMaxCapacity, cacheLoadFactor) + if (context.attemptNumber >= 1) { + // just in case the prior attempt failures were cache related + CachedKafkaConsumer.remove(groupId, part.topic, part.partition) + } + CachedKafkaConsumer.get[K, V](groupId, part.topic, part.partition, kafkaParams) + } else { + CachedKafkaConsumer.getUncached[K, V](groupId, part.topic, part.partition, kafkaParams) + } + + var requestOffset = part.fromOffset + + def closeIfNeeded(): Unit = { + if (!useConsumerCache && consumer != null) { + consumer.close + } + } + + override def hasNext(): Boolean = requestOffset < part.untilOffset + + override def next(): ConsumerRecord[K, V] = { + assert(hasNext(), "Can't call getNext() once untilOffset has been reached") + val r = consumer.get(requestOffset, pollTimeout) + requestOffset += 1 + r + } + } +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDDPartition.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDDPartition.scala new file mode 100644 index 000000000000..95569b109f30 --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDDPartition.scala @@ -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.streaming.kafka010 + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.Partition + + +/** + * @param topic kafka topic name + * @param partition kafka partition id + * @param fromOffset inclusive starting offset + * @param untilOffset exclusive ending offset + */ +private[kafka010] +class KafkaRDDPartition( + val index: Int, + val topic: String, + val partition: Int, + val fromOffset: Long, + val untilOffset: Long +) extends Partition { + /** Number of messages this partition refers to */ + def count(): Long = untilOffset - fromOffset + + /** Kafka TopicPartition object, for convenience */ + def topicPartition(): TopicPartition = new TopicPartition(topic, partition) + +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala new file mode 100644 index 000000000000..e6bdef04512d --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala @@ -0,0 +1,223 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka010 + +import java.{ util => ju } + +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.{ JavaRDD, JavaSparkContext } +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.api.java.{ JavaInputDStream, JavaStreamingContext } +import org.apache.spark.streaming.dstream._ + +/** + * :: Experimental :: + * object for constructing Kafka streams and RDDs + */ +@Experimental +object KafkaUtils extends Logging { + /** + * :: Experimental :: + * Scala constructor for a batch-oriented interface for consuming from Kafka. + * Starting and ending offsets are specified in advance, + * so that you can control exactly-once semantics. + * @param kafkaParams Kafka + * + * configuration parameters. Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD + * @param locationStrategy In most cases, pass in [[LocationStrategies.PreferConsistent]], + * see [[LocationStrategies]] for more details. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ + @Experimental + def createRDD[K, V]( + sc: SparkContext, + kafkaParams: ju.Map[String, Object], + offsetRanges: Array[OffsetRange], + locationStrategy: LocationStrategy + ): RDD[ConsumerRecord[K, V]] = { + val preferredHosts = locationStrategy match { + case PreferBrokers => + throw new AssertionError( + "If you want to prefer brokers, you must provide a mapping using PreferFixed " + + "A single KafkaRDD does not have a driver consumer and cannot look up brokers for you.") + case PreferConsistent => ju.Collections.emptyMap[TopicPartition, String]() + case PreferFixed(hostMap) => hostMap + } + val kp = new ju.HashMap[String, Object](kafkaParams) + fixKafkaParams(kp) + val osr = offsetRanges.clone() + + new KafkaRDD[K, V](sc, kp, osr, preferredHosts, true) + } + + /** + * :: Experimental :: + * Java constructor for a batch-oriented interface for consuming from Kafka. + * Starting and ending offsets are specified in advance, + * so that you can control exactly-once semantics. + * @param kafkaParams Kafka + * + * configuration parameters. Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD + * @param locationStrategy In most cases, pass in [[LocationStrategies.PreferConsistent]], + * see [[LocationStrategies]] for more details. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ + @Experimental + def createRDD[K, V]( + jsc: JavaSparkContext, + kafkaParams: ju.Map[String, Object], + offsetRanges: Array[OffsetRange], + locationStrategy: LocationStrategy + ): JavaRDD[ConsumerRecord[K, V]] = { + + new JavaRDD(createRDD[K, V](jsc.sc, kafkaParams, offsetRanges, locationStrategy)) + } + + /** + * :: Experimental :: + * Scala constructor for a DStream where + * each given Kafka topic/partition corresponds to an RDD partition. + * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number + * of messages + * per second that each '''partition''' will accept. + * @param locationStrategy In most cases, pass in [[LocationStrategies.PreferConsistent]], + * see [[LocationStrategies]] for more details. + * @param consumerStrategy In most cases, pass in [[ConsumerStrategies.Subscribe]], + * see [[ConsumerStrategies]] for more details + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ + @Experimental + def createDirectStream[K, V]( + ssc: StreamingContext, + locationStrategy: LocationStrategy, + consumerStrategy: ConsumerStrategy[K, V] + ): InputDStream[ConsumerRecord[K, V]] = { + val ppc = new DefaultPerPartitionConfig(ssc.sparkContext.getConf) + createDirectStream[K, V](ssc, locationStrategy, consumerStrategy, ppc) + } + + /** + * :: Experimental :: + * Scala constructor for a DStream where + * each given Kafka topic/partition corresponds to an RDD partition. + * @param locationStrategy In most cases, pass in [[LocationStrategies.PreferConsistent]], + * see [[LocationStrategies]] for more details. + * @param consumerStrategy In most cases, pass in [[ConsumerStrategies.Subscribe]], + * see [[ConsumerStrategies]] for more details. + * @param perPartitionConfig configuration of settings such as max rate on a per-partition basis. + * see [[PerPartitionConfig]] for more details. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ + @Experimental + def createDirectStream[K, V]( + ssc: StreamingContext, + locationStrategy: LocationStrategy, + consumerStrategy: ConsumerStrategy[K, V], + perPartitionConfig: PerPartitionConfig + ): InputDStream[ConsumerRecord[K, V]] = { + new DirectKafkaInputDStream[K, V](ssc, locationStrategy, consumerStrategy, perPartitionConfig) + } + + /** + * :: Experimental :: + * Java constructor for a DStream where + * each given Kafka topic/partition corresponds to an RDD partition. + * @param locationStrategy In most cases, pass in [[LocationStrategies.PreferConsistent]], + * see [[LocationStrategies]] for more details. + * @param consumerStrategy In most cases, pass in [[ConsumerStrategies.Subscribe]], + * see [[ConsumerStrategies]] for more details + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ + @Experimental + def createDirectStream[K, V]( + jssc: JavaStreamingContext, + locationStrategy: LocationStrategy, + consumerStrategy: ConsumerStrategy[K, V] + ): JavaInputDStream[ConsumerRecord[K, V]] = { + new JavaInputDStream( + createDirectStream[K, V]( + jssc.ssc, locationStrategy, consumerStrategy)) + } + + /** + * :: Experimental :: + * Java constructor for a DStream where + * each given Kafka topic/partition corresponds to an RDD partition. + * @param locationStrategy In most cases, pass in [[LocationStrategies.PreferConsistent]], + * see [[LocationStrategies]] for more details. + * @param consumerStrategy In most cases, pass in [[ConsumerStrategies.Subscribe]], + * see [[ConsumerStrategies]] for more details + * @param perPartitionConfig configuration of settings such as max rate on a per-partition basis. + * see [[PerPartitionConfig]] for more details. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ + @Experimental + def createDirectStream[K, V]( + jssc: JavaStreamingContext, + locationStrategy: LocationStrategy, + consumerStrategy: ConsumerStrategy[K, V], + perPartitionConfig: PerPartitionConfig + ): JavaInputDStream[ConsumerRecord[K, V]] = { + new JavaInputDStream( + createDirectStream[K, V]( + jssc.ssc, locationStrategy, consumerStrategy, perPartitionConfig)) + } + + /** + * Tweak kafka params to prevent issues on executors + */ + private[kafka010] def fixKafkaParams(kafkaParams: ju.HashMap[String, Object]): Unit = { + logWarning(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for executor") + kafkaParams.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) + + logWarning(s"overriding ${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} to none for executor") + kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + + // driver and executor should be in different consumer groups + val originalGroupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) + if (null == originalGroupId) { + logError(s"${ConsumerConfig.GROUP_ID_CONFIG} is null, you should probably set it") + } + val groupId = "spark-executor-" + originalGroupId + logWarning(s"overriding executor ${ConsumerConfig.GROUP_ID_CONFIG} to ${groupId}") + kafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) + + // possible workaround for KAFKA-3135 + val rbb = kafkaParams.get(ConsumerConfig.RECEIVE_BUFFER_CONFIG) + if (null == rbb || rbb.asInstanceOf[java.lang.Integer] < 65536) { + logWarning(s"overriding ${ConsumerConfig.RECEIVE_BUFFER_CONFIG} to 65536 see KAFKA-3135") + kafkaParams.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + } + } +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala new file mode 100644 index 000000000000..c9a8a13f51c3 --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka010 + +import java.{ util => ju } + +import scala.collection.JavaConverters._ + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.annotation.Experimental + + +/** + * :: Experimental :: + * Choice of how to schedule consumers for a given TopicPartition on an executor. + * See [[LocationStrategies]] to obtain instances. + * Kafka 0.10 consumers prefetch messages, so it's important for performance + * to keep cached consumers on appropriate executors, not recreate them for every partition. + * Choice of location is only a preference, not an absolute; partitions may be scheduled elsewhere. + */ +@Experimental +sealed abstract class LocationStrategy + +private case object PreferBrokers extends LocationStrategy + +private case object PreferConsistent extends LocationStrategy + +private case class PreferFixed(hostMap: ju.Map[TopicPartition, String]) extends LocationStrategy + +/** + * :: Experimental :: object to obtain instances of [[LocationStrategy]] + * + */ +@Experimental +object LocationStrategies { + /** + * :: Experimental :: + * Use this only if your executors are on the same nodes as your Kafka brokers. + */ + @Experimental + def PreferBrokers: LocationStrategy = + org.apache.spark.streaming.kafka010.PreferBrokers + + /** + * :: Experimental :: + * Use this in most cases, it will consistently distribute partitions across all executors. + */ + @Experimental + def PreferConsistent: LocationStrategy = + org.apache.spark.streaming.kafka010.PreferConsistent + + /** + * :: Experimental :: + * Use this to place particular TopicPartitions on particular hosts if your load is uneven. + * Any TopicPartition not specified in the map will use a consistent location. + */ + @Experimental + def PreferFixed(hostMap: collection.Map[TopicPartition, String]): LocationStrategy = + new PreferFixed(new ju.HashMap[TopicPartition, String](hostMap.asJava)) + + /** + * :: Experimental :: + * Use this to place particular TopicPartitions on particular hosts if your load is uneven. + * Any TopicPartition not specified in the map will use a consistent location. + */ + @Experimental + def PreferFixed(hostMap: ju.Map[TopicPartition, String]): LocationStrategy = + new PreferFixed(hostMap) +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala new file mode 100644 index 000000000000..c66d3c9b8d22 --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka010 + +import org.apache.kafka.clients.consumer.OffsetCommitCallback +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.annotation.Experimental + +/** + * Represents any object that has a collection of [[OffsetRange]]s. This can be used to access the + * offset ranges in RDDs generated by the direct Kafka DStream (see + * [[KafkaUtils.createDirectStream]]). + * {{{ + * KafkaUtils.createDirectStream(...).foreachRDD { rdd => + * val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + * ... + * } + * }}} + */ +trait HasOffsetRanges { + def offsetRanges: Array[OffsetRange] +} + +/** + * :: Experimental :: + * Represents any object that can commit a collection of [[OffsetRange]]s. + * The direct Kafka DStream implements this interface (see + * [[KafkaUtils.createDirectStream]]). + * {{{ + * val stream = KafkaUtils.createDirectStream(...) + * ... + * stream.asInstanceOf[CanCommitOffsets].commitAsync(offsets, new OffsetCommitCallback() { + * def onComplete(m: java.util.Map[TopicPartition, OffsetAndMetadata], e: Exception) { + * if (null != e) { + * // error + * } else { + * // success + * } + * } + * }) + * }}} + */ +@Experimental +trait CanCommitOffsets { + /** + * :: Experimental :: + * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. + * This is only needed if you intend to store offsets in Kafka, instead of your own store. + * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. + */ + @Experimental + def commitAsync(offsetRanges: Array[OffsetRange]): Unit + + /** + * :: Experimental :: + * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. + * This is only needed if you intend to store offsets in Kafka, instead of your own store. + * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. + * @param callback Only the most recently provided callback will be used at commit. + */ + @Experimental + def commitAsync(offsetRanges: Array[OffsetRange], callback: OffsetCommitCallback): Unit +} + +/** + * Represents a range of offsets from a single Kafka TopicPartition. Instances of this class + * can be created with `OffsetRange.create()`. + * @param topic Kafka topic name + * @param partition Kafka partition id + * @param fromOffset Inclusive starting offset + * @param untilOffset Exclusive ending offset + */ +final class OffsetRange private( + val topic: String, + val partition: Int, + val fromOffset: Long, + val untilOffset: Long) extends Serializable { + import OffsetRange.OffsetRangeTuple + + /** Kafka TopicPartition object, for convenience */ + def topicPartition(): TopicPartition = new TopicPartition(topic, partition) + + /** Number of messages this OffsetRange refers to */ + def count(): Long = untilOffset - fromOffset + + override def equals(obj: Any): Boolean = obj match { + case that: OffsetRange => + this.topic == that.topic && + this.partition == that.partition && + this.fromOffset == that.fromOffset && + this.untilOffset == that.untilOffset + case _ => false + } + + override def hashCode(): Int = { + toTuple.hashCode() + } + + override def toString(): String = { + s"OffsetRange(topic: '$topic', partition: $partition, range: [$fromOffset -> $untilOffset])" + } + + /** this is to avoid ClassNotFoundException during checkpoint restore */ + private[streaming] + def toTuple: OffsetRangeTuple = (topic, partition, fromOffset, untilOffset) +} + +/** + * Companion object the provides methods to create instances of [[OffsetRange]]. + */ +object OffsetRange { + def create(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = + new OffsetRange(topic, partition, fromOffset, untilOffset) + + def create( + topicPartition: TopicPartition, + fromOffset: Long, + untilOffset: Long): OffsetRange = + new OffsetRange(topicPartition.topic, topicPartition.partition, fromOffset, untilOffset) + + def apply(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = + new OffsetRange(topic, partition, fromOffset, untilOffset) + + def apply( + topicPartition: TopicPartition, + fromOffset: Long, + untilOffset: Long): OffsetRange = + new OffsetRange(topicPartition.topic, topicPartition.partition, fromOffset, untilOffset) + + /** this is to avoid ClassNotFoundException during checkpoint restore */ + private[kafka010] + type OffsetRangeTuple = (String, Int, Long, Long) + + private[kafka010] + def apply(t: OffsetRangeTuple) = + new OffsetRange(t._1, t._2, t._3, t._4) +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/PerPartitionConfig.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/PerPartitionConfig.scala new file mode 100644 index 000000000000..4792f2a95511 --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/PerPartitionConfig.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka010 + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.SparkConf +import org.apache.spark.annotation.Experimental + +/** + * :: Experimental :: + * Interface for user-supplied configurations that can't otherwise be set via Spark properties, + * because they need tweaking on a per-partition basis, + */ +@Experimental +abstract class PerPartitionConfig extends Serializable { + /** + * Maximum rate (number of records per second) at which data will be read + * from each Kafka partition. + */ + def maxRatePerPartition(topicPartition: TopicPartition): Long +} + +/** + * Default per-partition configuration + */ +private class DefaultPerPartitionConfig(conf: SparkConf) + extends PerPartitionConfig { + val maxRate = conf.getLong("spark.streaming.kafka.maxRatePerPartition", 0) + + def maxRatePerPartition(topicPartition: TopicPartition): Long = maxRate +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package-info.java b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package-info.java new file mode 100644 index 000000000000..ebfcf8764a32 --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark Integration for Kafka 0.10 + */ +package org.apache.spark.streaming.kafka010; diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala new file mode 100644 index 000000000000..09db6d6062d8 --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/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.streaming + +/** + * Spark Integration for Kafka 0.10 + */ +package object kafka010 //scalastyle:ignore diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java new file mode 100644 index 000000000000..938cc8ddfb5d --- /dev/null +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka010; + +import java.io.Serializable; +import java.util.*; +import java.util.regex.Pattern; + +import scala.collection.JavaConverters; + +import org.apache.kafka.common.TopicPartition; + +import org.junit.Assert; +import org.junit.Test; + +public class JavaConsumerStrategySuite implements Serializable { + + @Test + public void testConsumerStrategyConstructors() { + final String topic1 = "topic1"; + final Pattern pat = Pattern.compile("top.*"); + final Collection topics = Arrays.asList(topic1); + final scala.collection.Iterable sTopics = + JavaConverters.collectionAsScalaIterableConverter(topics).asScala(); + final TopicPartition tp1 = new TopicPartition(topic1, 0); + final TopicPartition tp2 = new TopicPartition(topic1, 1); + final Collection parts = Arrays.asList(tp1, tp2); + final scala.collection.Iterable sParts = + JavaConverters.collectionAsScalaIterableConverter(parts).asScala(); + final Map kafkaParams = new HashMap(); + kafkaParams.put("bootstrap.servers", "not used"); + final scala.collection.Map sKafkaParams = + JavaConverters.mapAsScalaMapConverter(kafkaParams).asScala(); + final Map offsets = new HashMap<>(); + offsets.put(tp1, 23L); + final scala.collection.Map sOffsets = + JavaConverters.mapAsScalaMapConverter(offsets).asScala().mapValues( + new scala.runtime.AbstractFunction1() { + @Override + public Object apply(Long x) { + return (Object) x; + } + } + ); + + final ConsumerStrategy sub1 = + ConsumerStrategies.Subscribe(sTopics, sKafkaParams, sOffsets); + final ConsumerStrategy sub2 = + ConsumerStrategies.Subscribe(sTopics, sKafkaParams); + final ConsumerStrategy sub3 = + ConsumerStrategies.Subscribe(topics, kafkaParams, offsets); + final ConsumerStrategy sub4 = + ConsumerStrategies.Subscribe(topics, kafkaParams); + + Assert.assertEquals( + sub1.executorKafkaParams().get("bootstrap.servers"), + sub3.executorKafkaParams().get("bootstrap.servers")); + + final ConsumerStrategy psub1 = + ConsumerStrategies.SubscribePattern(pat, sKafkaParams, sOffsets); + final ConsumerStrategy psub2 = + ConsumerStrategies.SubscribePattern(pat, sKafkaParams); + final ConsumerStrategy psub3 = + ConsumerStrategies.SubscribePattern(pat, kafkaParams, offsets); + final ConsumerStrategy psub4 = + ConsumerStrategies.SubscribePattern(pat, kafkaParams); + + Assert.assertEquals( + psub1.executorKafkaParams().get("bootstrap.servers"), + psub3.executorKafkaParams().get("bootstrap.servers")); + + final ConsumerStrategy asn1 = + ConsumerStrategies.Assign(sParts, sKafkaParams, sOffsets); + final ConsumerStrategy asn2 = + ConsumerStrategies.Assign(sParts, sKafkaParams); + final ConsumerStrategy asn3 = + ConsumerStrategies.Assign(parts, kafkaParams, offsets); + final ConsumerStrategy asn4 = + ConsumerStrategies.Assign(parts, kafkaParams); + + Assert.assertEquals( + asn1.executorKafkaParams().get("bootstrap.servers"), + asn3.executorKafkaParams().get("bootstrap.servers")); + } + +} diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java new file mode 100644 index 000000000000..dc9c13ba863f --- /dev/null +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010; + +import java.io.Serializable; +import java.util.*; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.VoidFunction; +import org.apache.spark.streaming.Durations; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaInputDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; + +public class JavaDirectKafkaStreamSuite implements Serializable { + private transient JavaStreamingContext ssc = null; + private transient KafkaTestUtils kafkaTestUtils = null; + + @Before + public void setUp() { + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200)); + } + + @After + public void tearDown() { + if (ssc != null) { + ssc.stop(); + ssc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } + } + + @Test + public void testKafkaStream() throws InterruptedException { + final String topic1 = "topic1"; + final String topic2 = "topic2"; + // hold a reference to the current offset ranges, so it can be used downstream + final AtomicReference offsetRanges = new AtomicReference<>(); + + String[] topic1data = createTopicAndSendData(topic1); + String[] topic2data = createTopicAndSendData(topic2); + + Set sent = new HashSet<>(); + sent.addAll(Arrays.asList(topic1data)); + sent.addAll(Arrays.asList(topic2data)); + + Random random = new Random(); + + final Map kafkaParams = new HashMap<>(); + kafkaParams.put("bootstrap.servers", kafkaTestUtils.brokerAddress()); + kafkaParams.put("key.deserializer", StringDeserializer.class); + kafkaParams.put("value.deserializer", StringDeserializer.class); + kafkaParams.put("auto.offset.reset", "earliest"); + kafkaParams.put("group.id", "java-test-consumer-" + random.nextInt() + + "-" + System.currentTimeMillis()); + + JavaInputDStream> istream1 = KafkaUtils.createDirectStream( + ssc, + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Subscribe(Arrays.asList(topic1), kafkaParams) + ); + + JavaDStream stream1 = istream1.transform( + // Make sure you can get offset ranges from the rdd + new Function>, + JavaRDD>>() { + @Override + public JavaRDD> call( + JavaRDD> rdd + ) { + OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + offsetRanges.set(offsets); + Assert.assertEquals(topic1, offsets[0].topic()); + return rdd; + } + } + ).map( + new Function, String>() { + @Override + public String call(ConsumerRecord r) { + return r.value(); + } + } + ); + + final Map kafkaParams2 = new HashMap<>(kafkaParams); + kafkaParams2.put("group.id", "java-test-consumer-" + random.nextInt() + + "-" + System.currentTimeMillis()); + + JavaInputDStream> istream2 = KafkaUtils.createDirectStream( + ssc, + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Subscribe(Arrays.asList(topic2), kafkaParams2) + ); + + JavaDStream stream2 = istream2.transform( + // Make sure you can get offset ranges from the rdd + new Function>, + JavaRDD>>() { + @Override + public JavaRDD> call( + JavaRDD> rdd + ) { + OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + offsetRanges.set(offsets); + Assert.assertEquals(topic2, offsets[0].topic()); + return rdd; + } + } + ).map( + new Function, String>() { + @Override + public String call(ConsumerRecord r) { + return r.value(); + } + } + ); + + JavaDStream unifiedStream = stream1.union(stream2); + + final Set result = Collections.synchronizedSet(new HashSet()); + unifiedStream.foreachRDD(new VoidFunction>() { + @Override + public void call(JavaRDD rdd) { + result.addAll(rdd.collect()); + } + } + ); + ssc.start(); + long startTime = System.currentTimeMillis(); + boolean matches = false; + while (!matches && System.currentTimeMillis() - startTime < 20000) { + matches = sent.size() == result.size(); + Thread.sleep(50); + } + Assert.assertEquals(sent, result); + ssc.stop(); + } + + private String[] createTopicAndSendData(String topic) { + String[] data = { topic + "-1", topic + "-2", topic + "-3"}; + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, data); + return data; + } +} diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaKafkaRDDSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaKafkaRDDSuite.java new file mode 100644 index 000000000000..87bfe1514e33 --- /dev/null +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaKafkaRDDSuite.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka010; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; + +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; + +public class JavaKafkaRDDSuite implements Serializable { + private transient JavaSparkContext sc = null; + private transient KafkaTestUtils kafkaTestUtils = null; + + @Before + public void setUp() { + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + sc = new JavaSparkContext(sparkConf); + } + + @After + public void tearDown() { + if (sc != null) { + sc.stop(); + sc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } + } + + @Test + public void testKafkaRDD() throws InterruptedException { + String topic1 = "topic1"; + String topic2 = "topic2"; + + Random random = new Random(); + + createTopicAndSendData(topic1); + createTopicAndSendData(topic2); + + Map kafkaParams = new HashMap<>(); + kafkaParams.put("bootstrap.servers", kafkaTestUtils.brokerAddress()); + kafkaParams.put("key.deserializer", StringDeserializer.class); + kafkaParams.put("value.deserializer", StringDeserializer.class); + kafkaParams.put("group.id", "java-test-consumer-" + random.nextInt() + + "-" + System.currentTimeMillis()); + + OffsetRange[] offsetRanges = { + OffsetRange.create(topic1, 0, 0, 1), + OffsetRange.create(topic2, 0, 0, 1) + }; + + Map leaders = new HashMap<>(); + String[] hostAndPort = kafkaTestUtils.brokerAddress().split(":"); + String broker = hostAndPort[0]; + leaders.put(offsetRanges[0].topicPartition(), broker); + leaders.put(offsetRanges[1].topicPartition(), broker); + + Function, String> handler = + new Function, String>() { + @Override + public String call(ConsumerRecord r) { + return r.value(); + } + }; + + JavaRDD rdd1 = KafkaUtils.createRDD( + sc, + kafkaParams, + offsetRanges, + LocationStrategies.PreferFixed(leaders) + ).map(handler); + + JavaRDD rdd2 = KafkaUtils.createRDD( + sc, + kafkaParams, + offsetRanges, + LocationStrategies.PreferConsistent() + ).map(handler); + + // just making sure the java user apis work; the scala tests handle logic corner cases + long count1 = rdd1.count(); + long count2 = rdd2.count(); + Assert.assertTrue(count1 > 0); + Assert.assertEquals(count1, count2); + } + + private String[] createTopicAndSendData(String topic) { + String[] data = { topic + "-1", topic + "-2", topic + "-3"}; + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, data); + return data; + } +} diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaLocationStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaLocationStrategySuite.java new file mode 100644 index 000000000000..41ccb0ebe7bf --- /dev/null +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaLocationStrategySuite.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka010; + +import java.io.Serializable; +import java.util.*; + +import scala.collection.JavaConverters; + +import org.apache.kafka.common.TopicPartition; + +import org.junit.Assert; +import org.junit.Test; + +public class JavaLocationStrategySuite implements Serializable { + + @Test + public void testLocationStrategyConstructors() { + final String topic1 = "topic1"; + final TopicPartition tp1 = new TopicPartition(topic1, 0); + final TopicPartition tp2 = new TopicPartition(topic1, 1); + final Map hosts = new HashMap<>(); + hosts.put(tp1, "node1"); + hosts.put(tp2, "node2"); + final scala.collection.Map sHosts = + JavaConverters.mapAsScalaMapConverter(hosts).asScala(); + + // make sure constructors can be called from java + final LocationStrategy c1 = LocationStrategies.PreferConsistent(); + final LocationStrategy c2 = LocationStrategies.PreferConsistent(); + Assert.assertSame(c1, c2); + + final LocationStrategy c3 = LocationStrategies.PreferBrokers(); + final LocationStrategy c4 = LocationStrategies.PreferBrokers(); + Assert.assertSame(c3, c4); + + Assert.assertNotSame(c1, c3); + + final LocationStrategy c5 = LocationStrategies.PreferFixed(hosts); + final LocationStrategy c6 = LocationStrategies.PreferFixed(sHosts); + Assert.assertEquals(c5, c6); + } + +} diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/kafka-0-10/src/test/resources/log4j.properties similarity index 100% rename from external/mqtt/src/test/resources/log4j.properties rename to external/kafka-0-10/src/test/resources/log4j.properties diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala new file mode 100644 index 000000000000..453b5e5ab20d --- /dev/null +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -0,0 +1,709 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.io.File +import java.lang.{ Long => JLong } +import java.util.{ Arrays, HashMap => JHashMap, Map => JMap } +import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.atomic.AtomicLong + +import scala.collection.JavaConverters._ +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.util.Random + +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.serialization.StringDeserializer +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.concurrent.Eventually + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.streaming.scheduler.rate.RateEstimator +import org.apache.spark.util.Utils + +class DirectKafkaStreamSuite + extends SparkFunSuite + with BeforeAndAfter + with BeforeAndAfterAll + with Eventually + with Logging { + val sparkConf = new SparkConf() + .setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + + private var ssc: StreamingContext = _ + private var testDir: File = _ + + private var kafkaTestUtils: KafkaTestUtils = _ + + override def beforeAll { + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + } + + override def afterAll { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + after { + if (ssc != null) { + ssc.stop(stopSparkContext = true) + } + if (testDir != null) { + Utils.deleteRecursively(testDir) + } + } + + def getKafkaParams(extra: (String, Object)*): JHashMap[String, Object] = { + val kp = new JHashMap[String, Object]() + kp.put("bootstrap.servers", kafkaTestUtils.brokerAddress) + kp.put("key.deserializer", classOf[StringDeserializer]) + kp.put("value.deserializer", classOf[StringDeserializer]) + kp.put("group.id", s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}") + extra.foreach(e => kp.put(e._1, e._2)) + kp + } + + val preferredHosts = LocationStrategies.PreferConsistent + + test("basic stream receiving with multiple topics and smallest starting offset") { + val topics = List("basic1", "basic2", "basic3") + val data = Map("a" -> 7, "b" -> 9) + topics.foreach { t => + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) + } + val offsets = Map(new TopicPartition("basic3", 0) -> 2L) + // one topic is starting 2 messages later + val expectedTotal = (data.values.sum * topics.size) - 2 + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") + + ssc = new StreamingContext(sparkConf, Milliseconds(1000)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String]( + ssc, + preferredHosts, + ConsumerStrategies.Subscribe[String, String](topics, kafkaParams.asScala, offsets)) + } + val allReceived = new ConcurrentLinkedQueue[(String, String)]() + + // hold a reference to the current offset ranges, so it can be used downstream + var offsetRanges = Array[OffsetRange]() + val tf = stream.transform { rdd => + // Get the offset ranges in the RDD + offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + rdd.map(r => (r.key, r.value)) + } + + tf.foreachRDD { rdd => + for (o <- offsetRanges) { + logInfo(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") + } + val collected = rdd.mapPartitionsWithIndex { (i, iter) => + // For each partition, get size of the range in the partition, + // and the number of items in the partition + val off = offsetRanges(i) + val all = iter.toSeq + val partSize = all.size + val rangeSize = off.untilOffset - off.fromOffset + Iterator((partSize, rangeSize)) + }.collect + + // Verify whether number of elements in each partition + // matches with the corresponding offset range + collected.foreach { case (partSize, rangeSize) => + assert(partSize === rangeSize, "offset ranges are wrong") + } + } + + stream.foreachRDD { rdd => + allReceived.addAll(Arrays.asList(rdd.map(r => (r.key, r.value)).collect(): _*)) + } + ssc.start() + eventually(timeout(100000.milliseconds), interval(1000.milliseconds)) { + assert(allReceived.size === expectedTotal, + "didn't get expected number of messages, messages:\n" + + allReceived.asScala.mkString("\n")) + } + ssc.stop() + } + + test("pattern based subscription") { + val topics = List("pat1", "pat2", "pat3", "advanced3") + // Should match 3 out of 4 topics + val pat = """pat\d""".r.pattern + val data = Map("a" -> 7, "b" -> 9) + topics.foreach { t => + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) + } + val offsets = Map( + new TopicPartition("pat2", 0) -> 3L, + new TopicPartition("pat3", 0) -> 4L) + // 3 matching topics, two of which start a total of 7 messages later + val expectedTotal = (data.values.sum * 3) - 7 + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") + + ssc = new StreamingContext(sparkConf, Milliseconds(1000)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String]( + ssc, + preferredHosts, + ConsumerStrategies.SubscribePattern[String, String](pat, kafkaParams.asScala, offsets)) + } + val allReceived = new ConcurrentLinkedQueue[(String, String)]() + + // hold a reference to the current offset ranges, so it can be used downstream + var offsetRanges = Array[OffsetRange]() + val tf = stream.transform { rdd => + // Get the offset ranges in the RDD + offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + rdd.map(r => (r.key, r.value)) + } + + tf.foreachRDD { rdd => + for (o <- offsetRanges) { + logInfo(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") + } + val collected = rdd.mapPartitionsWithIndex { (i, iter) => + // For each partition, get size of the range in the partition, + // and the number of items in the partition + val off = offsetRanges(i) + val all = iter.toSeq + val partSize = all.size + val rangeSize = off.untilOffset - off.fromOffset + Iterator((partSize, rangeSize)) + }.collect + + // Verify whether number of elements in each partition + // matches with the corresponding offset range + collected.foreach { case (partSize, rangeSize) => + assert(partSize === rangeSize, "offset ranges are wrong") + } + } + + stream.foreachRDD { rdd => + allReceived.addAll(Arrays.asList(rdd.map(r => (r.key, r.value)).collect(): _*)) + } + ssc.start() + eventually(timeout(100000.milliseconds), interval(1000.milliseconds)) { + assert(allReceived.size === expectedTotal, + "didn't get expected number of messages, messages:\n" + + allReceived.asScala.mkString("\n")) + } + ssc.stop() + } + + + test("receiving from largest starting offset") { + val topic = "latest" + val topicPartition = new TopicPartition(topic, 0) + val data = Map("a" -> 10) + kafkaTestUtils.createTopic(topic) + val kafkaParams = getKafkaParams("auto.offset.reset" -> "latest") + val kc = new KafkaConsumer(kafkaParams) + kc.assign(Arrays.asList(topicPartition)) + def getLatestOffset(): Long = { + kc.seekToEnd(Arrays.asList(topicPartition)) + kc.position(topicPartition) + } + + // Send some initial messages before starting context + kafkaTestUtils.sendMessages(topic, data) + eventually(timeout(10 seconds), interval(20 milliseconds)) { + assert(getLatestOffset() > 3) + } + val offsetBeforeStart = getLatestOffset() + kc.close() + + // Setup context and kafka stream with largest offset + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + val s = new DirectKafkaInputDStream[String, String]( + ssc, + preferredHosts, + ConsumerStrategies.Subscribe[String, String](List(topic), kafkaParams.asScala), + new DefaultPerPartitionConfig(sparkConf)) + s.consumer.poll(0) + assert( + s.consumer.position(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + s + } + + val collectedData = new ConcurrentLinkedQueue[String]() + stream.map { _.value }.foreachRDD { rdd => + collectedData.addAll(Arrays.asList(rdd.collect(): _*)) + } + ssc.start() + val newData = Map("b" -> 10) + kafkaTestUtils.sendMessages(topic, newData) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + collectedData.contains("b") + } + assert(!collectedData.contains("a")) + ssc.stop() + } + + + test("creating stream by offset") { + val topic = "offset" + val topicPartition = new TopicPartition(topic, 0) + val data = Map("a" -> 10) + kafkaTestUtils.createTopic(topic) + val kafkaParams = getKafkaParams("auto.offset.reset" -> "latest") + val kc = new KafkaConsumer(kafkaParams) + kc.assign(Arrays.asList(topicPartition)) + def getLatestOffset(): Long = { + kc.seekToEnd(Arrays.asList(topicPartition)) + kc.position(topicPartition) + } + + // Send some initial messages before starting context + kafkaTestUtils.sendMessages(topic, data) + eventually(timeout(10 seconds), interval(20 milliseconds)) { + assert(getLatestOffset() >= 10) + } + val offsetBeforeStart = getLatestOffset() + kc.close() + + // Setup context and kafka stream with largest offset + kafkaParams.put("auto.offset.reset", "none") + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + val s = new DirectKafkaInputDStream[String, String]( + ssc, + preferredHosts, + ConsumerStrategies.Assign[String, String]( + List(topicPartition), + kafkaParams.asScala, + Map(topicPartition -> 11L)), + new DefaultPerPartitionConfig(sparkConf)) + s.consumer.poll(0) + assert( + s.consumer.position(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + s + } + + val collectedData = new ConcurrentLinkedQueue[String]() + stream.map(_.value).foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } + ssc.start() + val newData = Map("b" -> 10) + kafkaTestUtils.sendMessages(topic, newData) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + collectedData.contains("b") + } + assert(!collectedData.contains("a")) + ssc.stop() + } + + // Test to verify the offset ranges can be recovered from the checkpoints + test("offset recovery") { + val topic = "recovery" + kafkaTestUtils.createTopic(topic) + testDir = Utils.createTempDir() + + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") + + // Send data to Kafka + def sendData(data: Seq[Int]) { + val strings = data.map { _.toString} + kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) + } + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(100)) + val kafkaStream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String]( + ssc, + preferredHosts, + ConsumerStrategies.Subscribe[String, String](List(topic), kafkaParams.asScala)) + } + val keyedStream = kafkaStream.map { r => "key" -> r.value.toInt } + val stateStream = keyedStream.updateStateByKey { (values: Seq[Int], state: Option[Int]) => + Some(values.sum + state.getOrElse(0)) + } + ssc.checkpoint(testDir.getAbsolutePath) + + // This is ensure all the data is eventually receiving only once + stateStream.foreachRDD { (rdd: RDD[(String, Int)]) => + rdd.collect().headOption.foreach { x => + DirectKafkaStreamSuite.total.set(x._2) + } + } + + ssc.start() + + // Send some data + for (i <- (1 to 10).grouped(4)) { + sendData(i) + } + + eventually(timeout(20 seconds), interval(50 milliseconds)) { + assert(DirectKafkaStreamSuite.total.get === (1 to 10).sum) + } + + ssc.stop() + + // Verify that offset ranges were generated + val offsetRangesBeforeStop = getOffsetRanges(kafkaStream) + assert(offsetRangesBeforeStop.size >= 1, "No offset ranges generated") + assert( + offsetRangesBeforeStop.head._2.forall { _.fromOffset === 0 }, + "starting offset not zero" + ) + + logInfo("====== RESTARTING ========") + + // Recover context from checkpoints + ssc = new StreamingContext(testDir.getAbsolutePath) + val recoveredStream = + ssc.graph.getInputStreams().head.asInstanceOf[DStream[ConsumerRecord[String, String]]] + + // Verify offset ranges have been recovered + val recoveredOffsetRanges = getOffsetRanges(recoveredStream).map { x => (x._1, x._2.toSet) } + assert(recoveredOffsetRanges.size > 0, "No offset ranges recovered") + val earlierOffsetRanges = offsetRangesBeforeStop.map { x => (x._1, x._2.toSet) } + assert( + recoveredOffsetRanges.forall { or => + earlierOffsetRanges.contains((or._1, or._2)) + }, + "Recovered ranges are not the same as the ones generated\n" + + earlierOffsetRanges + "\n" + recoveredOffsetRanges + ) + // Restart context, give more data and verify the total at the end + // If the total is write that means each records has been received only once + ssc.start() + for (i <- (11 to 20).grouped(4)) { + sendData(i) + } + + eventually(timeout(20 seconds), interval(50 milliseconds)) { + assert(DirectKafkaStreamSuite.total.get === (1 to 20).sum) + } + ssc.stop() + } + + // Test to verify the offsets can be recovered from Kafka + test("offset recovery from kafka") { + val topic = "recoveryfromkafka" + kafkaTestUtils.createTopic(topic) + + val kafkaParams = getKafkaParams( + "auto.offset.reset" -> "earliest", + ("enable.auto.commit", false: java.lang.Boolean) + ) + + val collectedData = new ConcurrentLinkedQueue[String]() + val committed = new JHashMap[TopicPartition, OffsetAndMetadata]() + + // Send data to Kafka and wait for it to be received + def sendDataAndWaitForReceive(data: Seq[Int]) { + val strings = data.map { _.toString} + kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + assert(strings.forall { collectedData.contains }) + } + } + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(100)) + withClue("Error creating direct stream") { + val kafkaStream = KafkaUtils.createDirectStream[String, String]( + ssc, + preferredHosts, + ConsumerStrategies.Subscribe[String, String](List(topic), kafkaParams.asScala)) + kafkaStream.foreachRDD { (rdd: RDD[ConsumerRecord[String, String]], time: Time) => + val offsets = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + val data = rdd.map(_.value).collect() + collectedData.addAll(Arrays.asList(data: _*)) + kafkaStream.asInstanceOf[CanCommitOffsets] + .commitAsync(offsets, new OffsetCommitCallback() { + def onComplete(m: JMap[TopicPartition, OffsetAndMetadata], e: Exception) { + if (null != e) { + logError("commit failed", e) + } else { + committed.putAll(m) + } + } + }) + } + } + ssc.start() + // Send some data and wait for them to be received + for (i <- (1 to 10).grouped(4)) { + sendDataAndWaitForReceive(i) + } + ssc.stop() + assert(! committed.isEmpty) + val consumer = new KafkaConsumer[String, String](kafkaParams) + consumer.subscribe(Arrays.asList(topic)) + consumer.poll(0) + committed.asScala.foreach { + case (k, v) => + // commits are async, not exactly once + assert(v.offset > 0) + assert(consumer.position(k) >= v.offset) + } + } + + + test("Direct Kafka stream report input information") { + val topic = "report-test" + val data = Map("a" -> 7, "b" -> 9) + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, data) + + val totalSent = data.values.sum + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") + + import DirectKafkaStreamSuite._ + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val collector = new InputInfoCollector + ssc.addStreamingListener(collector) + + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String]( + ssc, + preferredHosts, + ConsumerStrategies.Subscribe[String, String](List(topic), kafkaParams.asScala)) + } + + val allReceived = new ConcurrentLinkedQueue[(String, String)] + + stream.map(r => (r.key, r.value)) + .foreachRDD { rdd => allReceived.addAll(Arrays.asList(rdd.collect(): _*)) } + ssc.start() + eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { + assert(allReceived.size === totalSent, + "didn't get expected number of messages, messages:\n" + + allReceived.asScala.mkString("\n")) + + // Calculate all the record number collected in the StreamingListener. + assert(collector.numRecordsSubmitted.get() === totalSent) + assert(collector.numRecordsStarted.get() === totalSent) + assert(collector.numRecordsCompleted.get() === totalSent) + } + ssc.stop() + } + + test("maxMessagesPerPartition with backpressure disabled") { + val topic = "maxMessagesPerPartition" + val kafkaStream = getDirectKafkaStream(topic, None, None) + + val input = Map(new TopicPartition(topic, 0) -> 50L, new TopicPartition(topic, 1) -> 50L) + assert(kafkaStream.maxMessagesPerPartition(input).get == + Map(new TopicPartition(topic, 0) -> 10L, new TopicPartition(topic, 1) -> 10L)) + } + + test("maxMessagesPerPartition with no lag") { + val topic = "maxMessagesPerPartition" + val rateController = Some(new ConstantRateController(0, new ConstantEstimator(100), 100)) + val kafkaStream = getDirectKafkaStream(topic, rateController, None) + + val input = Map(new TopicPartition(topic, 0) -> 0L, new TopicPartition(topic, 1) -> 0L) + assert(kafkaStream.maxMessagesPerPartition(input).isEmpty) + } + + test("maxMessagesPerPartition respects max rate") { + val topic = "maxMessagesPerPartition" + val rateController = Some(new ConstantRateController(0, new ConstantEstimator(100), 1000)) + val ppc = Some(new PerPartitionConfig { + def maxRatePerPartition(tp: TopicPartition) = + if (tp.topic == topic && tp.partition == 0) { + 50 + } else { + 100 + } + }) + val kafkaStream = getDirectKafkaStream(topic, rateController, ppc) + + val input = Map(new TopicPartition(topic, 0) -> 1000L, new TopicPartition(topic, 1) -> 1000L) + assert(kafkaStream.maxMessagesPerPartition(input).get == + Map(new TopicPartition(topic, 0) -> 5L, new TopicPartition(topic, 1) -> 10L)) + } + + test("using rate controller") { + val topic = "backpressure" + kafkaTestUtils.createTopic(topic, 1) + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") + val executorKafkaParams = new JHashMap[String, Object](kafkaParams) + KafkaUtils.fixKafkaParams(executorKafkaParams) + + val batchIntervalMilliseconds = 500 + val estimator = new ConstantEstimator(100) + val messages = Map("foo" -> 5000) + kafkaTestUtils.sendMessages(topic, messages) + + val sparkConf = new SparkConf() + // Safe, even with streaming, because we're using the direct API. + // Using 1 core is useful to make the test more predictable. + .setMaster("local[1]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.kafka.maxRatePerPartition", "100") + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) + + val kafkaStream = withClue("Error creating direct stream") { + new DirectKafkaInputDStream[String, String]( + ssc, + preferredHosts, + ConsumerStrategies.Subscribe[String, String](List(topic), kafkaParams.asScala), + new DefaultPerPartitionConfig(sparkConf) + ) { + override protected[streaming] val rateController = + Some(new DirectKafkaRateController(id, estimator)) + }.map(r => (r.key, r.value)) + } + + val collectedData = new ConcurrentLinkedQueue[Array[String]]() + + // Used for assertion failure messages. + def dataToString: String = + collectedData.asScala.map(_.mkString("[", ",", "]")).mkString("{", ", ", "}") + + // This is to collect the raw data received from Kafka + kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) => + val data = rdd.map { _._2 }.collect() + collectedData.add(data) + } + + ssc.start() + + // Try different rate limits. + // Wait for arrays of data to appear matching the rate. + Seq(100, 50, 20).foreach { rate => + collectedData.clear() // Empty this buffer on each pass. + estimator.updateRate(rate) // Set a new rate. + // Expect blocks of data equal to "rate", scaled by the interval length in secs. + val expectedSize = Math.round(rate * batchIntervalMilliseconds * 0.001) + eventually(timeout(5.seconds), interval(10 milliseconds)) { + // Assert that rate estimator values are used to determine maxMessagesPerPartition. + // Funky "-" in message makes the complete assertion message read better. + assert(collectedData.asScala.exists(_.size == expectedSize), + s" - No arrays of size $expectedSize for rate $rate found in $dataToString") + } + } + + ssc.stop() + } + + /** Get the generated offset ranges from the DirectKafkaStream */ + private def getOffsetRanges[K, V]( + kafkaStream: DStream[ConsumerRecord[K, V]]): Seq[(Time, Array[OffsetRange])] = { + kafkaStream.generatedRDDs.mapValues { rdd => + rdd.asInstanceOf[HasOffsetRanges].offsetRanges + }.toSeq.sortBy { _._1 } + } + + private def getDirectKafkaStream( + topic: String, + mockRateController: Option[RateController], + ppc: Option[PerPartitionConfig]) = { + val batchIntervalMilliseconds = 100 + + val sparkConf = new SparkConf() + .setMaster("local[1]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.kafka.maxRatePerPartition", "100") + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) + + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") + val ekp = new JHashMap[String, Object](kafkaParams) + KafkaUtils.fixKafkaParams(ekp) + + val s = new DirectKafkaInputDStream[String, String]( + ssc, + preferredHosts, + new ConsumerStrategy[String, String] { + def executorKafkaParams = ekp + def onStart(currentOffsets: JMap[TopicPartition, JLong]): Consumer[String, String] = { + val consumer = new KafkaConsumer[String, String](kafkaParams) + val tps = List(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) + consumer.assign(Arrays.asList(tps: _*)) + tps.foreach(tp => consumer.seek(tp, 0)) + consumer + } + }, + ppc.getOrElse(new DefaultPerPartitionConfig(sparkConf)) + ) { + override protected[streaming] val rateController = mockRateController + } + // manual start necessary because we arent consuming the stream, just checking its state + s.start() + s + } +} + +object DirectKafkaStreamSuite { + val total = new AtomicLong(-1L) + + class InputInfoCollector extends StreamingListener { + val numRecordsSubmitted = new AtomicLong(0L) + val numRecordsStarted = new AtomicLong(0L) + val numRecordsCompleted = new AtomicLong(0L) + + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit = { + numRecordsSubmitted.addAndGet(batchSubmitted.batchInfo.numRecords) + } + + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = { + numRecordsStarted.addAndGet(batchStarted.batchInfo.numRecords) + } + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { + numRecordsCompleted.addAndGet(batchCompleted.batchInfo.numRecords) + } + } +} + +private[streaming] class ConstantEstimator(@volatile private var rate: Long) + extends RateEstimator { + + def updateRate(newRate: Long): Unit = { + rate = newRate + } + + def compute( + time: Long, + elements: Long, + processingDelay: Long, + schedulingDelay: Long): Option[Double] = Some(rate) +} + +private[streaming] class ConstantRateController(id: Int, estimator: RateEstimator, rate: Long) + extends RateController(id, estimator) { + override def publish(rate: Long): Unit = () + override def getLatestRate(): Long = rate +} diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala new file mode 100644 index 000000000000..be373af0599c --- /dev/null +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka010 + +import java.{ util => ju } + +import scala.collection.JavaConverters._ +import scala.util.Random + +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.serialization.StringDeserializer +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark._ +import org.apache.spark.scheduler.ExecutorCacheTaskLocation + +class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { + + private var kafkaTestUtils: KafkaTestUtils = _ + + private val sparkConf = new SparkConf().setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + private var sc: SparkContext = _ + + override def beforeAll { + sc = new SparkContext(sparkConf) + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + } + + override def afterAll { + if (sc != null) { + sc.stop + sc = null + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + private def getKafkaParams() = Map[String, Object]( + "bootstrap.servers" -> kafkaTestUtils.brokerAddress, + "key.deserializer" -> classOf[StringDeserializer], + "value.deserializer" -> classOf[StringDeserializer], + "group.id" -> s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}" + ).asJava + + private val preferredHosts = LocationStrategies.PreferConsistent + + test("basic usage") { + val topic = s"topicbasic-${Random.nextInt}-${System.currentTimeMillis}" + kafkaTestUtils.createTopic(topic) + val messages = Array("the", "quick", "brown", "fox") + kafkaTestUtils.sendMessages(topic, messages) + + val kafkaParams = getKafkaParams() + + val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) + + val rdd = KafkaUtils.createRDD[String, String](sc, kafkaParams, offsetRanges, preferredHosts) + .map(_.value) + + val received = rdd.collect.toSet + assert(received === messages.toSet) + + // size-related method optimizations return sane results + assert(rdd.count === messages.size) + assert(rdd.countApprox(0).getFinalValue.mean === messages.size) + assert(!rdd.isEmpty) + assert(rdd.take(1).size === 1) + assert(rdd.take(1).head === messages.head) + assert(rdd.take(messages.size + 10).size === messages.size) + + val emptyRdd = KafkaUtils.createRDD[String, String]( + sc, kafkaParams, Array(OffsetRange(topic, 0, 0, 0)), preferredHosts) + + assert(emptyRdd.isEmpty) + + // invalid offset ranges throw exceptions + val badRanges = Array(OffsetRange(topic, 0, 0, messages.size + 1)) + intercept[SparkException] { + val result = KafkaUtils.createRDD[String, String](sc, kafkaParams, badRanges, preferredHosts) + .map(_.value) + .collect() + } + } + + 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 = s"topicboundary-${Random.nextInt}-${System.currentTimeMillis}" + val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) + kafkaTestUtils.createTopic(topic) + + val kafkaParams = getKafkaParams() + + // this is the "lots of messages" case + kafkaTestUtils.sendMessages(topic, sent) + var sentCount = sent.values.sum + + val rdd = KafkaUtils.createRDD[String, String](sc, kafkaParams, + Array(OffsetRange(topic, 0, 0, sentCount)), preferredHosts) + + val ranges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + val rangeCount = ranges.map(o => o.untilOffset - o.fromOffset).sum + + assert(rangeCount === sentCount, "offset range didn't include all sent messages") + assert(rdd.map(_.offset).collect.sorted === (0 until sentCount).toArray, + "didn't get all sent messages") + + // this is the "0 messages" case + val rdd2 = KafkaUtils.createRDD[String, String](sc, kafkaParams, + Array(OffsetRange(topic, 0, sentCount, sentCount)), preferredHosts) + + // shouldn't get anything, since message is sent after rdd was defined + val sentOnlyOne = Map("d" -> 1) + + kafkaTestUtils.sendMessages(topic, sentOnlyOne) + + assert(rdd2.map(_.value).collect.size === 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 = KafkaUtils.createRDD[String, String](sc, kafkaParams, + Array(OffsetRange(topic, 0, sentCount, sentCount + 1)), preferredHosts) + + // send lots of messages after rdd was defined, they shouldn't show up + kafkaTestUtils.sendMessages(topic, Map("extra" -> 22)) + + assert(rdd3.map(_.value).collect.head === sentOnlyOne.keys.head, + "didn't get exactly one message") + } + + test("executor sorting") { + val kafkaParams = new ju.HashMap[String, Object](getKafkaParams()) + kafkaParams.put("auto.offset.reset", "none") + val rdd = new KafkaRDD[String, String]( + sc, + kafkaParams, + Array(OffsetRange("unused", 0, 1, 2)), + ju.Collections.emptyMap[TopicPartition, String](), + true) + val a3 = ExecutorCacheTaskLocation("a", "3") + val a4 = ExecutorCacheTaskLocation("a", "4") + val b1 = ExecutorCacheTaskLocation("b", "1") + val b2 = ExecutorCacheTaskLocation("b", "2") + + val correct = Array(b2, b1, a4, a3) + + correct.permutations.foreach { p => + assert(p.sortWith(rdd.compareExecutors) === correct) + } + } +} diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala new file mode 100644 index 000000000000..6c7024ea4b5a --- /dev/null +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala @@ -0,0 +1,304 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kafka010 + +import java.io.{File, IOException} +import java.lang.{Integer => JInt} +import java.net.InetSocketAddress +import java.util.{Map => JMap, Properties} +import java.util.concurrent.TimeoutException + +import scala.annotation.tailrec +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +import kafka.admin.AdminUtils +import kafka.api.Request +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.utils.ZkUtils +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} +import org.apache.kafka.common.serialization.StringSerializer +import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.streaming.Time +import org.apache.spark.util.Utils + +/** + * This is a helper class for Kafka test suites. This has the functionality to set up + * and tear down local Kafka servers, and to push data using Kafka producers. + * + * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. + */ +private[kafka010] class KafkaTestUtils extends Logging { + + // Zookeeper related configurations + private val zkHost = "localhost" + private var zkPort: Int = 0 + private val zkConnectionTimeout = 60000 + private val zkSessionTimeout = 6000 + + private var zookeeper: EmbeddedZookeeper = _ + + private var zkUtils: ZkUtils = _ + + // Kafka broker related configurations + private val brokerHost = "localhost" + private var brokerPort = 0 + private var brokerConf: KafkaConfig = _ + + // Kafka broker server + private var server: KafkaServer = _ + + // Kafka producer + private var producer: KafkaProducer[String, String] = _ + + // Flag to test whether the system is correctly started + private var zkReady = false + private var brokerReady = false + + def zkAddress: String = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") + s"$zkHost:$zkPort" + } + + def brokerAddress: String = { + assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") + s"$brokerHost:$brokerPort" + } + + def zookeeperClient: ZkUtils = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") + Option(zkUtils).getOrElse( + throw new IllegalStateException("Zookeeper client is not yet initialized")) + } + + // Set up the Embedded Zookeeper server and get the proper Zookeeper port + private def setupEmbeddedZookeeper(): Unit = { + // Zookeeper server startup + zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") + // Get the actual zookeeper binding port + zkPort = zookeeper.actualPort + zkUtils = ZkUtils(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, false) + zkReady = true + } + + // Set up the Embedded Kafka server + private def setupEmbeddedKafkaServer(): Unit = { + assert(zkReady, "Zookeeper should be set up beforehand") + + // Kafka broker startup + Utils.startServiceOnPort(brokerPort, port => { + brokerPort = port + brokerConf = new KafkaConfig(brokerConfiguration, doLog = false) + server = new KafkaServer(brokerConf) + server.startup() + brokerPort = server.boundPort() + (server, brokerPort) + }, new SparkConf(), "KafkaBroker") + + brokerReady = true + } + + /** setup the whole embedded servers, including Zookeeper and Kafka brokers */ + def setup(): Unit = { + setupEmbeddedZookeeper() + setupEmbeddedKafkaServer() + } + + /** Teardown the whole servers, including Kafka broker and Zookeeper */ + def teardown(): Unit = { + brokerReady = false + zkReady = false + + if (producer != null) { + producer.close() + producer = null + } + + if (server != null) { + server.shutdown() + server.awaitShutdown() + server = null + } + + // On Windows, `logDirs` is left open even after Kafka server above is completely shut down + // in some cases. It leads to test failures on Windows if the directory deletion failure + // throws an exception. + brokerConf.logDirs.foreach { f => + try { + Utils.deleteRecursively(new File(f)) + } catch { + case e: IOException if Utils.isWindows => + logWarning(e.getMessage) + } + } + + if (zkUtils != null) { + zkUtils.close() + zkUtils = null + } + + if (zookeeper != null) { + zookeeper.shutdown() + zookeeper = null + } + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String, partitions: Int): Unit = { + AdminUtils.createTopic(zkUtils, topic, partitions, 1) + // wait until metadata is propagated + (0 until partitions).foreach { p => + waitUntilMetadataIsPropagated(topic, p) + } + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String): Unit = { + createTopic(topic, 1) + } + + /** Java-friendly function for sending messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { + sendMessages(topic, Map(messageToFreq.asScala.mapValues(_.intValue()).toSeq: _*)) + } + + /** Send the messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = { + val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray + sendMessages(topic, messages) + } + + /** Send the array of messages to the Kafka broker */ + def sendMessages(topic: String, messages: Array[String]): Unit = { + producer = new KafkaProducer[String, String](producerConfiguration) + messages.foreach { message => + producer.send(new ProducerRecord[String, String](topic, message)) + } + producer.close() + producer = null + } + + private def brokerConfiguration: Properties = { + val props = new Properties() + props.put("broker.id", "0") + props.put("host.name", "localhost") + props.put("port", brokerPort.toString) + props.put("log.dir", Utils.createTempDir().getAbsolutePath) + props.put("zookeeper.connect", zkAddress) + props.put("log.flush.interval.messages", "1") + props.put("replica.socket.timeout.ms", "1500") + props + } + + private def producerConfiguration: Properties = { + val props = new Properties() + props.put("bootstrap.servers", brokerAddress) + props.put("value.serializer", classOf[StringSerializer].getName) + // Key serializer is required. + props.put("key.serializer", classOf[StringSerializer].getName) + // wait for all in-sync replicas to ack sends + props.put("acks", "all") + props + } + + // A simplified version of scalatest eventually, rewritten here to avoid adding extra test + // dependency + def eventually[T](timeout: Time, interval: Time)(func: => T): T = { + def makeAttempt(): Either[Throwable, T] = { + try { + Right(func) + } catch { + case e if NonFatal(e) => Left(e) + } + } + + val startTime = System.currentTimeMillis() + @tailrec + def tryAgain(attempt: Int): T = { + makeAttempt() match { + case Right(result) => result + case Left(e) => + val duration = System.currentTimeMillis() - startTime + if (duration < timeout.milliseconds) { + Thread.sleep(interval.milliseconds) + } else { + throw new TimeoutException(e.getMessage) + } + + tryAgain(attempt + 1) + } + } + + tryAgain(1) + } + + private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { + def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { + case Some(partitionState) => + val leaderAndInSyncReplicas = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr + + zkUtils.getLeaderForPartition(topic, partition).isDefined && + Request.isValidBrokerId(leaderAndInSyncReplicas.leader) && + leaderAndInSyncReplicas.isr.nonEmpty + + case _ => + false + } + eventually(Time(10000), Time(100)) { + assert(isPropagated, s"Partition [$topic, $partition] metadata not propagated after timeout") + } + } + + private class EmbeddedZookeeper(val zkConnect: String) { + val snapshotDir = Utils.createTempDir() + val logDir = Utils.createTempDir() + + val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) + val (ip, port) = { + val splits = zkConnect.split(":") + (splits(0), splits(1).toInt) + } + val factory = new NIOServerCnxnFactory() + factory.configure(new InetSocketAddress(ip, port), 16) + factory.startup(zookeeper) + + val actualPort = factory.getLocalPort + + def shutdown() { + factory.shutdown() + // The directories are not closed even if the ZooKeeper server is shut down. + // Please see ZOOKEEPER-1844, which is fixed in 3.4.6+. It leads to test failures + // on Windows if the directory deletion failure throws an exception. + try { + Utils.deleteRecursively(snapshotDir) + } catch { + case e: IOException if Utils.isWindows => + logWarning(e.getMessage) + } + try { + Utils.deleteRecursively(logDir) + } catch { + case e: IOException if Utils.isWindows => + logWarning(e.getMessage) + } + } + } +} diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml new file mode 100644 index 000000000000..786349474389 --- /dev/null +++ b/external/kafka-0-8-assembly/pom.xml @@ -0,0 +1,175 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-streaming-kafka-0-8-assembly_2.11 + jar + Spark Project External Kafka Assembly + http://spark.apache.org/ + + + streaming-kafka-0-8-assembly + + + + + org.apache.spark + spark-streaming-kafka-0-8_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + + commons-codec + commons-codec + provided + + + commons-lang + commons-lang + provided + + + com.google.protobuf + protobuf-java + provided + + + org.lz4 + lz4-java + provided + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.avro + avro-mapred + ${avro.mapred.classifier} + provided + + + org.apache.curator + curator-recipes + provided + + + org.apache.zookeeper + zookeeper + provided + + + log4j + log4j + provided + + + net.java.dev.jets3t + jets3t + provided + + + org.scala-lang + scala-library + provided + + + org.slf4j + slf4j-api + provided + + + org.slf4j + slf4j-log4j12 + provided + + + org.xerial.snappy + snappy-java + provided + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + log4j.properties + + + + + + + + + + + + diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml new file mode 100644 index 000000000000..849c8b465f99 --- /dev/null +++ b/external/kafka-0-8/pom.xml @@ -0,0 +1,109 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-streaming-kafka-0-8_2.11 + + streaming-kafka-0-8 + + jar + Spark Integration for Kafka 0.8 + http://spark.apache.org/ + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.kafka + kafka_${scala.binary.version} + 0.8.2.1 + + + com.sun.jmx + jmxri + + + com.sun.jdmk + jmxtools + + + net.sf.jopt-simple + jopt-simple + + + org.slf4j + slf4j-simple + + + org.apache.zookeeper + zookeeper + + + + + net.sf.jopt-simple + jopt-simple + 3.2 + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala similarity index 94% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala index 9159051ba06e..89ccbe219cec 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala @@ -23,6 +23,7 @@ import org.apache.spark.annotation.Experimental * Represents the host and port info for a Kafka broker. * Differs from the Kafka project's internal kafka.cluster.Broker, which contains a server ID. */ +@deprecated("Update to Kafka 0.10 integration", "2.3.0") final class Broker private( /** Broker's hostname */ val host: String, @@ -49,6 +50,7 @@ final class Broker private( * Companion object that provides methods to create instances of [[Broker]]. */ @Experimental +@deprecated("Update to Kafka 0.10 integration", "2.3.0") object Broker { def create(host: String, port: Int): Broker = new Broker(host, port) diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala new file mode 100644 index 000000000000..d52c230eb784 --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -0,0 +1,227 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import scala.annotation.tailrec +import scala.collection.mutable +import scala.reflect.ClassTag + +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata +import kafka.serializer.Decoder + +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging +import org.apache.spark.streaming.{StreamingContext, Time} +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset +import org.apache.spark.streaming.scheduler.{RateController, StreamInputInfo} +import org.apache.spark.streaming.scheduler.rate.RateEstimator + +/** + * A stream of [[KafkaRDD]] where + * each given Kafka topic/partition corresponds to an RDD partition. + * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number + * of messages + * per second that each '''partition''' will accept. + * Starting offsets are specified in advance, + * and this DStream is not responsible for committing offsets, + * so that you can control exactly-once semantics. + * For an easy interface to Kafka-managed offsets, + * see [[KafkaCluster]] + * @param kafkaParams Kafka + * configuration parameters. + * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the stream + * @param messageHandler function for translating each message into the desired type + */ +private[streaming] +class DirectKafkaInputDStream[ + K: ClassTag, + V: ClassTag, + U <: Decoder[K]: ClassTag, + T <: Decoder[V]: ClassTag, + R: ClassTag]( + _ssc: StreamingContext, + val kafkaParams: Map[String, String], + val fromOffsets: Map[TopicAndPartition, Long], + messageHandler: MessageAndMetadata[K, V] => R + ) extends InputDStream[R](_ssc) with Logging { + val maxRetries = context.sparkContext.getConf.getInt( + "spark.streaming.kafka.maxRetries", 1) + + // Keep this consistent with how other streams are named (e.g. "Flume polling stream [2]") + private[streaming] override def name: String = s"Kafka direct stream [$id]" + + protected[streaming] override val checkpointData = + new DirectKafkaInputDStreamCheckpointData + + + /** + * Asynchronously maintains & sends new rate limits to the receiver through the receiver tracker. + */ + override protected[streaming] val rateController: Option[RateController] = { + if (RateController.isBackPressureEnabled(ssc.conf)) { + Some(new DirectKafkaRateController(id, + RateEstimator.create(ssc.conf, context.graph.batchDuration))) + } else { + None + } + } + + protected val kc = new KafkaCluster(kafkaParams) + + private val maxRateLimitPerPartition: Long = context.sparkContext.getConf.getLong( + "spark.streaming.kafka.maxRatePerPartition", 0) + + protected[streaming] def maxMessagesPerPartition( + offsets: Map[TopicAndPartition, Long]): Option[Map[TopicAndPartition, Long]] = { + val estimatedRateLimit = rateController.map(_.getLatestRate()) + + // calculate a per-partition rate limit based on current lag + val effectiveRateLimitPerPartition = estimatedRateLimit.filter(_ > 0) match { + case Some(rate) => + val lagPerPartition = offsets.map { case (tp, offset) => + tp -> Math.max(offset - currentOffsets(tp), 0) + } + val totalLag = lagPerPartition.values.sum + + lagPerPartition.map { case (tp, lag) => + val backpressureRate = Math.round(lag / totalLag.toFloat * rate) + tp -> (if (maxRateLimitPerPartition > 0) { + Math.min(backpressureRate, maxRateLimitPerPartition)} else backpressureRate) + } + case None => offsets.map { case (tp, offset) => tp -> maxRateLimitPerPartition } + } + + if (effectiveRateLimitPerPartition.values.sum > 0) { + val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000 + Some(effectiveRateLimitPerPartition.map { + case (tp, limit) => tp -> (secsPerBatch * limit).toLong + }) + } else { + None + } + } + + protected var currentOffsets = fromOffsets + + @tailrec + protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = { + val o = kc.getLatestLeaderOffsets(currentOffsets.keySet) + // Either.fold would confuse @tailrec, do it manually + if (o.isLeft) { + val err = o.left.get.toString + if (retries <= 0) { + throw new SparkException(err) + } else { + logError(err) + Thread.sleep(kc.config.refreshLeaderBackoffMs) + latestLeaderOffsets(retries - 1) + } + } else { + o.right.get + } + } + + // limits the maximum number of messages per partition + protected def clamp( + leaderOffsets: Map[TopicAndPartition, LeaderOffset]): Map[TopicAndPartition, LeaderOffset] = { + val offsets = leaderOffsets.mapValues(lo => lo.offset) + + maxMessagesPerPartition(offsets).map { mmp => + mmp.map { case (tp, messages) => + val lo = leaderOffsets(tp) + tp -> lo.copy(offset = Math.min(currentOffsets(tp) + messages, lo.offset)) + } + }.getOrElse(leaderOffsets) + } + + override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = { + val untilOffsets = clamp(latestLeaderOffsets(maxRetries)) + val rdd = KafkaRDD[K, V, U, T, R]( + context.sparkContext, kafkaParams, currentOffsets, untilOffsets, messageHandler) + + // Report the record number and metadata of this batch interval to InputInfoTracker. + val offsetRanges = currentOffsets.map { case (tp, fo) => + val uo = untilOffsets(tp) + OffsetRange(tp.topic, tp.partition, fo, uo.offset) + } + val description = offsetRanges.filter { offsetRange => + // Don't display empty ranges. + offsetRange.fromOffset != offsetRange.untilOffset + }.map { offsetRange => + s"topic: ${offsetRange.topic}\tpartition: ${offsetRange.partition}\t" + + s"offsets: ${offsetRange.fromOffset} to ${offsetRange.untilOffset}" + }.mkString("\n") + // Copy offsetRanges to immutable.List to prevent from being modified by the user + val metadata = Map( + "offsets" -> offsetRanges.toList, + StreamInputInfo.METADATA_KEY_DESCRIPTION -> description) + val inputInfo = StreamInputInfo(id, rdd.count, metadata) + ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) + + currentOffsets = untilOffsets.map(kv => kv._1 -> kv._2.offset) + Some(rdd) + } + + override def start(): Unit = { + } + + def stop(): Unit = { + } + + private[streaming] + class DirectKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) { + def batchForTime: mutable.HashMap[Time, Array[(String, Int, Long, Long)]] = { + data.asInstanceOf[mutable.HashMap[Time, Array[OffsetRange.OffsetRangeTuple]]] + } + + override def update(time: Time): Unit = { + batchForTime.clear() + generatedRDDs.foreach { kv => + val a = kv._2.asInstanceOf[KafkaRDD[K, V, U, T, R]].offsetRanges.map(_.toTuple).toArray + batchForTime += kv._1 -> a + } + } + + override def cleanup(time: Time): Unit = { } + + override def restore(): Unit = { + // this is assuming that the topics don't change during execution, which is true currently + val topics = fromOffsets.keySet + val leaders = KafkaCluster.checkErrors(kc.findLeaders(topics)) + + batchForTime.toSeq.sortBy(_._1)(Time.ordering).foreach { case (t, b) => + logInfo(s"Restoring KafkaRDD for time $t ${b.mkString("[", ", ", "]")}") + generatedRDDs += t -> new KafkaRDD[K, V, U, T, R]( + context.sparkContext, kafkaParams, b.map(OffsetRange(_)), leaders, messageHandler) + } + } + } + + /** + * A RateController to retrieve the rate from RateEstimator. + */ + private[streaming] class DirectKafkaRateController(id: Int, estimator: RateEstimator) + extends RateController(id, estimator) { + override def publish(rate: Long): Unit = () + } +} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala similarity index 90% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala index 8465432c5850..570affab1185 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala @@ -17,24 +17,32 @@ package org.apache.spark.streaming.kafka -import scala.util.control.NonFatal -import scala.util.Random -import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConverters._ import java.util.Properties + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.util.Random +import scala.util.control.NonFatal + import kafka.api._ import kafka.common.{ErrorMapping, OffsetAndMetadata, OffsetMetadataAndError, TopicAndPartition} import kafka.consumer.{ConsumerConfig, SimpleConsumer} + import org.apache.spark.SparkException +import org.apache.spark.annotation.DeveloperApi /** + * :: DeveloperApi :: * Convenience methods for interacting with a Kafka cluster. + * See + * A Guide To The Kafka Protocol for more details on individual api calls. * @param kafkaParams Kafka * configuration parameters. * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), * NOT zookeeper servers, specified in host1:port1,host2:port2 form */ -private[spark] +@DeveloperApi +@deprecated("Update to Kafka 0.10 integration", "2.3.0") class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { import KafkaCluster.{Err, LeaderOffset, SimpleConsumerConfig} @@ -101,7 +109,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { } else { val missing = topicAndPartitions.diff(leaderMap.keySet) val err = new Err - err.append(new SparkException(s"Couldn't find leaders for ${missing}")) + err += new SparkException(s"Couldn't find leaders for ${missing}") Left(err) } } @@ -132,7 +140,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { respErrs.foreach { m => val cause = ErrorMapping.exceptionFor(m.errorCode) val msg = s"Error getting partition metadata for '${m.topic}'. Does the topic exist?" - errs.append(new SparkException(msg, cause)) + errs += new SparkException(msg, cause) } } } @@ -160,7 +168,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = { getLeaderOffsets(topicAndPartitions, before, 1).right.map { r => r.map { kv => - // mapValues isnt serializable, see SI-7005 + // mapValues isn't serializable, see SI-7005 kv._1 -> kv._2.head } } @@ -198,11 +206,11 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { LeaderOffset(consumer.host, consumer.port, off) } } else { - errs.append(new SparkException( - s"Empty offsets for ${tp}, is ${before} before log beginning?")) + errs += new SparkException( + s"Empty offsets for ${tp}, is ${before} before log beginning?") } } else { - errs.append(ErrorMapping.exceptionFor(por.error)) + errs += ErrorMapping.exceptionFor(por.error) } } } @@ -211,7 +219,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { } } val missing = topicAndPartitions.diff(result.keySet) - errs.append(new SparkException(s"Couldn't find leader offsets for ${missing}")) + errs += new SparkException(s"Couldn't find leader offsets for ${missing}") Left(errs) } } @@ -224,7 +232,10 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { // this 0 here indicates api version, in this case the original ZK backed api. private def defaultConsumerApiVersion: Short = 0 - /** Requires Kafka >= 0.8.1.1 */ + /** + * Requires Kafka 0.8.1.1 or later. + * Defaults to the original ZooKeeper backed API version. + */ def getConsumerOffsets( groupId: String, topicAndPartitions: Set[TopicAndPartition] @@ -243,7 +254,10 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { } } - /** Requires Kafka >= 0.8.1.1 */ + /** + * Requires Kafka 0.8.1.1 or later. + * Defaults to the original ZooKeeper backed API version. + */ def getConsumerOffsetMetadata( groupId: String, topicAndPartitions: Set[TopicAndPartition] @@ -267,7 +281,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { if (ome.error == ErrorMapping.NoError) { result += tp -> ome } else { - errs.append(ErrorMapping.exceptionFor(ome.error)) + errs += ErrorMapping.exceptionFor(ome.error) } } } @@ -276,11 +290,14 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { } } val missing = topicAndPartitions.diff(result.keySet) - errs.append(new SparkException(s"Couldn't find consumer offsets for ${missing}")) + errs += new SparkException(s"Couldn't find consumer offsets for ${missing}") Left(errs) } - /** Requires Kafka >= 0.8.1.1 */ + /** + * Requires Kafka 0.8.1.1 or later. + * Defaults to the original ZooKeeper backed API version. + */ def setConsumerOffsets( groupId: String, offsets: Map[TopicAndPartition, Long] @@ -298,7 +315,10 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { setConsumerOffsetMetadata(groupId, meta, consumerApiVersion) } - /** Requires Kafka >= 0.8.1.1 */ + /** + * Requires Kafka 0.8.1.1 or later. + * Defaults to the original ZooKeeper backed API version. + */ def setConsumerOffsetMetadata( groupId: String, metadata: Map[TopicAndPartition, OffsetAndMetadata] @@ -323,7 +343,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { if (err == ErrorMapping.NoError) { result += tp -> err } else { - errs.append(ErrorMapping.exceptionFor(err)) + errs += ErrorMapping.exceptionFor(err) } } } @@ -332,7 +352,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { } } val missing = topicAndPartitions.diff(result.keySet) - errs.append(new SparkException(s"Couldn't set offsets for ${missing}")) + errs += new SparkException(s"Couldn't set offsets for ${missing}") Left(errs) } @@ -346,7 +366,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { fn(consumer) } catch { case NonFatal(e) => - errs.append(e) + errs += e } finally { if (consumer != null) { consumer.close() @@ -356,7 +376,8 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { } } -private[spark] +@DeveloperApi +@deprecated("Update to Kafka 0.10 integration", "2.3.0") object KafkaCluster { type Err = ArrayBuffer[Throwable] @@ -368,7 +389,6 @@ object KafkaCluster { ) } - private[spark] case class LeaderOffset(host: String, port: Int, offset: Long) /** @@ -376,19 +396,17 @@ object KafkaCluster { * Simple consumers connect directly to brokers, but need many of the same configs. * This subclass won't warn about missing ZK params, or presence of broker params. */ - private[spark] class SimpleConsumerConfig private(brokers: String, originalProps: Properties) extends ConsumerConfig(originalProps) { val seedBrokers: Array[(String, Int)] = brokers.split(",").map { hp => val hpa = hp.split(":") if (hpa.size == 1) { - throw new SparkException(s"Broker not the in correct format of : [$brokers]") + throw new SparkException(s"Broker not in the correct format of : [$brokers]") } (hpa(0), hpa(1).toInt) } } - private[spark] object SimpleConsumerConfig { /** * Make a consumer config without requiring group.id or zookeeper.connect, diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala similarity index 94% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index 38730fecf332..7ff3a98ca52c 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -22,11 +22,11 @@ import java.util.Properties import scala.collection.Map import scala.reflect.{classTag, ClassTag} -import kafka.consumer.{KafkaStream, Consumer, ConsumerConfig, ConsumerConnector} +import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector, KafkaStream} import kafka.serializer.Decoder import kafka.utils.VerifiableProperties -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ @@ -38,7 +38,7 @@ import org.apache.spark.util.ThreadUtils * * @param kafkaParams Map of kafka configuration parameters. * See: http://kafka.apache.org/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed * in its own thread. * @param storageLevel RDD storage level. */ @@ -48,12 +48,12 @@ class KafkaInputDStream[ V: ClassTag, U <: Decoder[_]: ClassTag, T <: Decoder[_]: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, kafkaParams: Map[String, String], topics: Map[String, Int], useReliableReceiver: Boolean, storageLevel: StorageLevel - ) extends ReceiverInputDStream[(K, V)](ssc_) with Logging { + ) extends ReceiverInputDStream[(K, V)](_ssc) with Logging { def getReceiver(): Receiver[(K, V)] = { if (!useReliableReceiver) { diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala new file mode 100644 index 000000000000..5ea52b6ad36a --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -0,0 +1,272 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import scala.collection.mutable.ArrayBuffer +import scala.reflect.{classTag, ClassTag} + +import kafka.api.{FetchRequestBuilder, FetchResponse} +import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.consumer.SimpleConsumer +import kafka.message.{MessageAndMetadata, MessageAndOffset} +import kafka.serializer.Decoder +import kafka.utils.VerifiableProperties + +import org.apache.spark.{Partition, SparkContext, SparkException, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.NextIterator + +/** + * A batch-oriented interface for consuming from Kafka. + * Starting and ending offsets are specified in advance, + * so that you can control exactly-once semantics. + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD + * @param messageHandler function for translating each message into the desired type + */ +private[kafka] +class KafkaRDD[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag, + R: ClassTag] private[spark] ( + sc: SparkContext, + kafkaParams: Map[String, String], + val offsetRanges: Array[OffsetRange], + leaders: Map[TopicAndPartition, (String, Int)], + messageHandler: MessageAndMetadata[K, V] => R + ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges { + override def getPartitions: Array[Partition] = { + offsetRanges.zipWithIndex.map { case (o, i) => + val (host, port) = leaders(TopicAndPartition(o.topic, o.partition)) + new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port) + }.toArray + } + + override def count(): Long = offsetRanges.map(_.count).sum + + override def countApprox( + timeout: Long, + confidence: Double = 0.95 + ): PartialResult[BoundedDouble] = { + val c = count + new PartialResult(new BoundedDouble(c, 1.0, c, c), true) + } + + override def isEmpty(): Boolean = count == 0L + + override def take(num: Int): Array[R] = { + val nonEmptyPartitions = this.partitions + .map(_.asInstanceOf[KafkaRDDPartition]) + .filter(_.count > 0) + + if (num < 1 || nonEmptyPartitions.isEmpty) { + return new Array[R](0) + } + + // Determine in advance how many messages need to be taken from each partition + val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) => + val remain = num - result.values.sum + if (remain > 0) { + val taken = Math.min(remain, part.count) + result + (part.index -> taken.toInt) + } else { + result + } + } + + val buf = new ArrayBuffer[R] + val res = context.runJob( + this, + (tc: TaskContext, it: Iterator[R]) => it.take(parts(tc.partitionId)).toArray, + parts.keys.toArray) + res.foreach(buf ++= _) + buf.toArray + } + + override def getPreferredLocations(thePart: Partition): Seq[String] = { + val part = thePart.asInstanceOf[KafkaRDDPartition] + // TODO is additional hostname resolution necessary here + Seq(part.host) + } + + private def errBeginAfterEnd(part: KafkaRDDPartition): String = + s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " + + s"for topic ${part.topic} partition ${part.partition}. " + + "You either provided an invalid fromOffset, or the Kafka topic has been damaged" + + private def errRanOutBeforeEnd(part: KafkaRDDPartition): String = + s"Ran out of messages before reaching ending offset ${part.untilOffset} " + + s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." + + " This should not happen, and indicates that messages may have been lost" + + private def errOvershotEnd(itemOffset: Long, part: KafkaRDDPartition): String = + s"Got ${itemOffset} > ending offset ${part.untilOffset} " + + s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." + + " This should not happen, and indicates a message may have been skipped" + + override def compute(thePart: Partition, context: TaskContext): Iterator[R] = { + val part = thePart.asInstanceOf[KafkaRDDPartition] + assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part)) + if (part.fromOffset == part.untilOffset) { + logInfo(s"Beginning offset ${part.fromOffset} is the same as ending offset " + + s"skipping ${part.topic} ${part.partition}") + Iterator.empty + } else { + new KafkaRDDIterator(part, context) + } + } + + /** + * An iterator that fetches messages directly from Kafka for the offsets in partition. + */ + private class KafkaRDDIterator( + part: KafkaRDDPartition, + context: TaskContext) extends NextIterator[R] { + + context.addTaskCompletionListener{ context => closeIfNeeded() } + + logInfo(s"Computing topic ${part.topic}, partition ${part.partition} " + + s"offsets ${part.fromOffset} -> ${part.untilOffset}") + + val kc = new KafkaCluster(kafkaParams) + val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(kc.config.props) + .asInstanceOf[Decoder[K]] + val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(kc.config.props) + .asInstanceOf[Decoder[V]] + val consumer = connectLeader + var requestOffset = part.fromOffset + var iter: Iterator[MessageAndOffset] = null + + // The idea is to use the provided preferred host, except on task retry attempts, + // to minimize number of kafka metadata requests + private def connectLeader: SimpleConsumer = { + if (context.attemptNumber > 0) { + kc.connectLeader(part.topic, part.partition).fold( + errs => throw new SparkException( + s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " + + errs.mkString("\n")), + consumer => consumer + ) + } else { + kc.connect(part.host, part.port) + } + } + + private def handleFetchErr(resp: FetchResponse) { + if (resp.hasError) { + val err = resp.errorCode(part.topic, part.partition) + if (err == ErrorMapping.LeaderNotAvailableCode || + err == ErrorMapping.NotLeaderForPartitionCode) { + logError(s"Lost leader for topic ${part.topic} partition ${part.partition}, " + + s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms") + Thread.sleep(kc.config.refreshLeaderBackoffMs) + } + // Let normal rdd retry sort out reconnect attempts + throw ErrorMapping.exceptionFor(err) + } + } + + private def fetchBatch: Iterator[MessageAndOffset] = { + val req = new FetchRequestBuilder() + .addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes) + .build() + val resp = consumer.fetch(req) + handleFetchErr(resp) + // kafka may return a batch that starts before the requested offset + resp.messageSet(part.topic, part.partition) + .iterator + .dropWhile(_.offset < requestOffset) + } + + override def close(): Unit = { + if (consumer != null) { + consumer.close() + } + } + + override def getNext(): R = { + if (iter == null || !iter.hasNext) { + iter = fetchBatch + } + if (!iter.hasNext) { + assert(requestOffset == part.untilOffset, errRanOutBeforeEnd(part)) + finished = true + null.asInstanceOf[R] + } else { + val item = iter.next() + if (item.offset >= part.untilOffset) { + assert(item.offset == part.untilOffset, errOvershotEnd(item.offset, part)) + finished = true + null.asInstanceOf[R] + } else { + requestOffset = item.nextOffset + messageHandler(new MessageAndMetadata( + part.topic, part.partition, item.message, item.offset, keyDecoder, valueDecoder)) + } + } + } + } +} + +private[kafka] +object KafkaRDD { + import KafkaCluster.LeaderOffset + + /** + * @param kafkaParams Kafka + * configuration parameters. + * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the batch + * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive) + * ending point of the batch + * @param messageHandler function for translating each message into the desired type + */ + def apply[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag, + R: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + fromOffsets: Map[TopicAndPartition, Long], + untilOffsets: Map[TopicAndPartition, LeaderOffset], + messageHandler: MessageAndMetadata[K, V] => R + ): KafkaRDD[K, V, U, T, R] = { + val leaders = untilOffsets.map { case (tp, lo) => + tp -> ((lo.host, lo.port)) + } + + val offsetRanges = fromOffsets.map { case (tp, fo) => + val uo = untilOffsets(tp) + OffsetRange(tp.topic, tp.partition, fo, uo.offset) + }.toArray + + new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaders, messageHandler) + } +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala new file mode 100644 index 000000000000..02917becf0ff --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import org.apache.spark.Partition + +/** + * @param topic kafka topic name + * @param partition kafka partition id + * @param fromOffset inclusive starting offset + * @param untilOffset exclusive ending offset + * @param host preferred kafka host, i.e. the leader at the time the rdd was created + * @param port preferred kafka host's port + */ +private[kafka] +class KafkaRDDPartition( + val index: Int, + val topic: String, + val partition: Int, + val fromOffset: Long, + val untilOffset: Long, + val host: String, + val port: Int +) extends Partition { + /** Number of messages this partition refers to */ + def count(): Long = untilOffset - fromOffset +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala new file mode 100644 index 000000000000..ef1968585be6 --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala @@ -0,0 +1,299 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.io.{File, IOException} +import java.lang.{Integer => JInt} +import java.net.InetSocketAddress +import java.util.{Map => JMap, Properties} +import java.util.concurrent.TimeoutException + +import scala.annotation.tailrec +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +import kafka.admin.AdminUtils +import kafka.api.Request +import kafka.producer.{KeyedMessage, Producer, ProducerConfig} +import kafka.serializer.StringEncoder +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.utils.{ZKStringSerializer, ZkUtils} +import org.I0Itec.zkclient.ZkClient +import org.apache.commons.lang3.RandomUtils +import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.streaming.Time +import org.apache.spark.util.Utils + +/** + * This is a helper class for Kafka test suites. This has the functionality to set up + * and tear down local Kafka servers, and to push data using Kafka producers. + * + * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. + */ +private[kafka] class KafkaTestUtils extends Logging { + + // Zookeeper related configurations + private val zkHost = "localhost" + private var zkPort: Int = 0 + private val zkConnectionTimeout = 60000 + private val zkSessionTimeout = 6000 + + private var zookeeper: EmbeddedZookeeper = _ + + private var zkClient: ZkClient = _ + + // Kafka broker related configurations + private val brokerHost = "localhost" + // 0.8.2 server doesn't have a boundPort method, so can't use 0 for a random port + private var brokerPort = RandomUtils.nextInt(1024, 65536) + private var brokerConf: KafkaConfig = _ + + // Kafka broker server + private var server: KafkaServer = _ + + // Kafka producer + private var producer: Producer[String, String] = _ + + // Flag to test whether the system is correctly started + private var zkReady = false + private var brokerReady = false + + def zkAddress: String = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") + s"$zkHost:$zkPort" + } + + def brokerAddress: String = { + assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") + s"$brokerHost:$brokerPort" + } + + def zookeeperClient: ZkClient = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") + Option(zkClient).getOrElse( + throw new IllegalStateException("Zookeeper client is not yet initialized")) + } + + // Set up the Embedded Zookeeper server and get the proper Zookeeper port + private def setupEmbeddedZookeeper(): Unit = { + // Zookeeper server startup + zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") + // Get the actual zookeeper binding port + zkPort = zookeeper.actualPort + zkClient = new ZkClient(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, + ZKStringSerializer) + zkReady = true + } + + // Set up the Embedded Kafka server + private def setupEmbeddedKafkaServer(): Unit = { + assert(zkReady, "Zookeeper should be set up beforehand") + + // Kafka broker startup + Utils.startServiceOnPort(brokerPort, port => { + brokerPort = port + brokerConf = new KafkaConfig(brokerConfiguration) + server = new KafkaServer(brokerConf) + server.startup() + (server, brokerPort) + }, new SparkConf(), "KafkaBroker") + + brokerReady = true + } + + /** setup the whole embedded servers, including Zookeeper and Kafka brokers */ + def setup(): Unit = { + setupEmbeddedZookeeper() + setupEmbeddedKafkaServer() + } + + /** Teardown the whole servers, including Kafka broker and Zookeeper */ + def teardown(): Unit = { + brokerReady = false + zkReady = false + + if (producer != null) { + producer.close() + producer = null + } + + if (server != null) { + server.shutdown() + server.awaitShutdown() + server = null + } + + // On Windows, `logDirs` is left open even after Kafka server above is completely shut down + // in some cases. It leads to test failures on Windows if the directory deletion failure + // throws an exception. + brokerConf.logDirs.foreach { f => + try { + Utils.deleteRecursively(new File(f)) + } catch { + case e: IOException if Utils.isWindows => + logWarning(e.getMessage) + } + } + + if (zkClient != null) { + zkClient.close() + zkClient = null + } + + if (zookeeper != null) { + zookeeper.shutdown() + zookeeper = null + } + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String, partitions: Int): Unit = { + AdminUtils.createTopic(zkClient, topic, partitions, 1) + // wait until metadata is propagated + (0 until partitions).foreach { p => waitUntilMetadataIsPropagated(topic, p) } + } + + /** Single-argument version for backwards compatibility */ + def createTopic(topic: String): Unit = createTopic(topic, 1) + + /** Java-friendly function for sending messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { + sendMessages(topic, Map(messageToFreq.asScala.mapValues(_.intValue()).toSeq: _*)) + } + + /** Send the messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = { + val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray + sendMessages(topic, messages) + } + + /** Send the array of messages to the Kafka broker */ + def sendMessages(topic: String, messages: Array[String]): Unit = { + producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) + producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) + producer.close() + producer = null + } + + private def brokerConfiguration: Properties = { + val props = new Properties() + props.put("broker.id", "0") + props.put("host.name", "localhost") + props.put("port", brokerPort.toString) + props.put("log.dir", Utils.createTempDir().getAbsolutePath) + props.put("zookeeper.connect", zkAddress) + props.put("log.flush.interval.messages", "1") + props.put("replica.socket.timeout.ms", "1500") + props + } + + private def producerConfiguration: Properties = { + val props = new Properties() + props.put("metadata.broker.list", brokerAddress) + props.put("serializer.class", classOf[StringEncoder].getName) + // wait for all in-sync replicas to ack sends + props.put("request.required.acks", "-1") + props + } + + // A simplified version of scalatest eventually, rewritten here to avoid adding extra test + // dependency + def eventually[T](timeout: Time, interval: Time)(func: => T): T = { + def makeAttempt(): Either[Throwable, T] = { + try { + Right(func) + } catch { + case e if NonFatal(e) => Left(e) + } + } + + val startTime = System.currentTimeMillis() + @tailrec + def tryAgain(attempt: Int): T = { + makeAttempt() match { + case Right(result) => result + case Left(e) => + val duration = System.currentTimeMillis() - startTime + if (duration < timeout.milliseconds) { + Thread.sleep(interval.milliseconds) + } else { + throw new TimeoutException(e.getMessage) + } + + tryAgain(attempt + 1) + } + } + + tryAgain(1) + } + + private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { + def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { + case Some(partitionState) => + val leaderAndInSyncReplicas = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr + + ZkUtils.getLeaderForPartition(zkClient, topic, partition).isDefined && + Request.isValidBrokerId(leaderAndInSyncReplicas.leader) && + leaderAndInSyncReplicas.isr.size >= 1 + + case _ => + false + } + eventually(Time(10000), Time(100)) { + assert(isPropagated, s"Partition [$topic, $partition] metadata not propagated after timeout") + } + } + + private class EmbeddedZookeeper(val zkConnect: String) { + val snapshotDir = Utils.createTempDir() + val logDir = Utils.createTempDir() + + val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) + val (ip, port) = { + val splits = zkConnect.split(":") + (splits(0), splits(1).toInt) + } + val factory = new NIOServerCnxnFactory() + factory.configure(new InetSocketAddress(ip, port), 16) + factory.startup(zookeeper) + + val actualPort = factory.getLocalPort + + def shutdown() { + factory.shutdown() + // The directories are not closed even if the ZooKeeper server is shut down. + // Please see ZOOKEEPER-1844, which is fixed in 3.4.6+. It leads to test failures + // on Windows if the directory deletion failure throws an exception. + try { + Utils.deleteRecursively(snapshotDir) + } catch { + case e: IOException if Utils.isWindows => + logWarning(e.getMessage) + } + try { + Utils.deleteRecursively(logDir) + } catch { + case e: IOException if Utils.isWindows => + logWarning(e.getMessage) + } + } + } +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala new file mode 100644 index 000000000000..36082e93707b --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -0,0 +1,806 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.OutputStream +import java.lang.{Integer => JInt, Long => JLong, Number => JNumber} +import java.nio.charset.StandardCharsets +import java.util.{List => JList, Locale, Map => JMap, Set => JSet} + +import scala.collection.JavaConverters._ +import scala.reflect.ClassTag + +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata +import kafka.serializer.{Decoder, DefaultDecoder, StringDecoder} +import net.razorvine.pickle.{IObjectPickler, Opcodes, Pickler} + +import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} +import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.api.python.SerDeUtil +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.api.java._ +import org.apache.spark.streaming.dstream.{DStream, InputDStream, ReceiverInputDStream} +import org.apache.spark.streaming.util.WriteAheadLogUtils + +@deprecated("Update to Kafka 0.10 integration", "2.3.0") +object KafkaUtils { + /** + * Create an input stream that pulls messages from Kafka Brokers. + * @param ssc StreamingContext object + * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) + * @param groupId The group id for this consumer + * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed + * in its own thread + * @param storageLevel Storage level to use for storing the received objects + * (default: StorageLevel.MEMORY_AND_DISK_SER_2) + * @return DStream of (Kafka message key, Kafka message value) + */ + def createStream( + ssc: StreamingContext, + zkQuorum: String, + groupId: String, + topics: Map[String, Int], + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): ReceiverInputDStream[(String, String)] = { + val kafkaParams = Map[String, String]( + "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, + "zookeeper.connection.timeout.ms" -> "10000") + createStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, topics, storageLevel) + } + + /** + * Create an input stream that pulls messages from Kafka Brokers. + * @param ssc StreamingContext object + * @param kafkaParams Map of kafka configuration parameters, + * see http://kafka.apache.org/08/configuration.html + * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel Storage level to use for storing the received objects + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam U type of Kafka message key decoder + * @tparam T type of Kafka message value decoder + * @return DStream of (Kafka message key, Kafka message value) + */ + def createStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: ClassTag, T <: Decoder[_]: ClassTag]( + ssc: StreamingContext, + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel + ): ReceiverInputDStream[(K, V)] = { + val walEnabled = WriteAheadLogUtils.enableReceiverLog(ssc.conf) + new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, walEnabled, storageLevel) + } + + /** + * Create an input stream that pulls messages from Kafka Brokers. + * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. + * @param jssc JavaStreamingContext object + * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) + * @param groupId The group id for this consumer + * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed + * in its own thread + * @return DStream of (Kafka message key, Kafka message value) + */ + def createStream( + jssc: JavaStreamingContext, + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt] + ): JavaPairReceiverInputDStream[String, String] = { + createStream(jssc.ssc, zkQuorum, groupId, Map(topics.asScala.mapValues(_.intValue()).toSeq: _*)) + } + + /** + * Create an input stream that pulls messages from Kafka Brokers. + * @param jssc JavaStreamingContext object + * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..). + * @param groupId The group id for this consumer. + * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel RDD storage level. + * @return DStream of (Kafka message key, Kafka message value) + */ + def createStream( + jssc: JavaStreamingContext, + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairReceiverInputDStream[String, String] = { + createStream(jssc.ssc, zkQuorum, groupId, Map(topics.asScala.mapValues(_.intValue()).toSeq: _*), + storageLevel) + } + + /** + * Create an input stream that pulls messages from Kafka Brokers. + * @param jssc JavaStreamingContext object + * @param keyTypeClass Key type of DStream + * @param valueTypeClass value type of Dstream + * @param keyDecoderClass Type of kafka key decoder + * @param valueDecoderClass Type of kafka value decoder + * @param kafkaParams Map of kafka configuration parameters, + * see http://kafka.apache.org/08/configuration.html + * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed + * in its own thread + * @param storageLevel RDD storage level. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam U type of Kafka message key decoder + * @tparam T type of Kafka message value decoder + * @return DStream of (Kafka message key, Kafka message value) + */ + def createStream[K, V, U <: Decoder[_], T <: Decoder[_]]( + jssc: JavaStreamingContext, + keyTypeClass: Class[K], + valueTypeClass: Class[V], + keyDecoderClass: Class[U], + valueDecoderClass: Class[T], + kafkaParams: JMap[String, String], + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairReceiverInputDStream[K, V] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyTypeClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueTypeClass) + + implicit val keyCmd: ClassTag[U] = ClassTag(keyDecoderClass) + implicit val valueCmd: ClassTag[T] = ClassTag(valueDecoderClass) + + createStream[K, V, U, T]( + jssc.ssc, + kafkaParams.asScala.toMap, + Map(topics.asScala.mapValues(_.intValue()).toSeq: _*), + storageLevel) + } + + /** get leaders for the given offset ranges, or throw an exception */ + private def leadersForRanges( + kc: KafkaCluster, + offsetRanges: Array[OffsetRange]): Map[TopicAndPartition, (String, Int)] = { + val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet + val leaders = kc.findLeaders(topics) + KafkaCluster.checkErrors(leaders) + } + + /** Make sure offsets are available in kafka, or throw an exception */ + private def checkOffsets( + kc: KafkaCluster, + offsetRanges: Array[OffsetRange]): Unit = { + val topics = offsetRanges.map(_.topicAndPartition).toSet + val result = for { + low <- kc.getEarliestLeaderOffsets(topics).right + high <- kc.getLatestLeaderOffsets(topics).right + } yield { + offsetRanges.filterNot { o => + low(o.topicAndPartition).offset <= o.fromOffset && + o.untilOffset <= high(o.topicAndPartition).offset + } + } + val badRanges = KafkaCluster.checkErrors(result) + if (!badRanges.isEmpty) { + throw new SparkException("Offsets not available on leader: " + badRanges.mkString(",")) + } + } + + private[kafka] def getFromOffsets( + kc: KafkaCluster, + kafkaParams: Map[String, String], + topics: Set[String] + ): Map[TopicAndPartition, Long] = { + val reset = kafkaParams.get("auto.offset.reset").map(_.toLowerCase(Locale.ROOT)) + val result = for { + topicPartitions <- kc.getPartitions(topics).right + leaderOffsets <- (if (reset == Some("smallest")) { + kc.getEarliestLeaderOffsets(topicPartitions) + } else { + kc.getLatestLeaderOffsets(topicPartitions) + }).right + } yield { + leaderOffsets.map { case (tp, lo) => + (tp, lo.offset) + } + } + KafkaCluster.checkErrors(result) + } + + /** + * Create an RDD from Kafka using offset ranges for each topic and partition. + * + * @param sc SparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @return RDD of (Kafka message key, Kafka message value) + */ + def createRDD[ + K: ClassTag, + V: ClassTag, + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + offsetRanges: Array[OffsetRange] + ): RDD[(K, V)] = sc.withScope { + val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) + val kc = new KafkaCluster(kafkaParams) + val leaders = leadersForRanges(kc, offsetRanges) + checkOffsets(kc, offsetRanges) + new KafkaRDD[K, V, KD, VD, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler) + } + + /** + * Create an RDD from Kafka using offset ranges for each topic and partition. This allows you + * specify the Kafka leader to connect to (to optimize fetching) and access the message as well + * as the metadata. + * + * @param sc SparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + * @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 + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @tparam R type returned by messageHandler + * @return RDD of R + */ + def createRDD[ + K: ClassTag, + V: ClassTag, + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag, + R: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + offsetRanges: Array[OffsetRange], + leaders: Map[TopicAndPartition, Broker], + messageHandler: MessageAndMetadata[K, V] => R + ): RDD[R] = sc.withScope { + val kc = new KafkaCluster(kafkaParams) + val leaderMap = if (leaders.isEmpty) { + leadersForRanges(kc, 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)) + } + } + val cleanedHandler = sc.clean(messageHandler) + checkOffsets(kc, offsetRanges) + new KafkaRDD[K, V, KD, VD, R](sc, kafkaParams, offsetRanges, leaderMap, cleanedHandler) + } + + /** + * Create an RDD from Kafka using offset ranges for each topic and partition. + * + * @param jsc JavaSparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + * @param keyClass type of Kafka message key + * @param valueClass type of Kafka message value + * @param keyDecoderClass type of Kafka message key decoder + * @param valueDecoderClass type of Kafka message value decoder + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @return RDD of (Kafka message key, Kafka message value) + */ + def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V]]( + jsc: JavaSparkContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + kafkaParams: JMap[String, String], + offsetRanges: Array[OffsetRange] + ): JavaPairRDD[K, V] = jsc.sc.withScope { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + new JavaPairRDD(createRDD[K, V, KD, VD]( + jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges)) + } + + /** + * Create an RDD from Kafka using offset ranges for each topic and partition. This allows you + * specify the Kafka leader to connect to (to optimize fetching) and access the message as well + * as the metadata. + * + * @param jsc JavaSparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + * @param leaders Kafka 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 + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @tparam R type returned by messageHandler + * @return RDD of R + */ + def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( + jsc: JavaSparkContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + recordClass: Class[R], + kafkaParams: JMap[String, String], + offsetRanges: Array[OffsetRange], + leaders: JMap[TopicAndPartition, Broker], + messageHandler: JFunction[MessageAndMetadata[K, V], R] + ): JavaRDD[R] = jsc.sc.withScope { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) + val leaderMap = Map(leaders.asScala.toSeq: _*) + createRDD[K, V, KD, VD, R]( + jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges, leaderMap, messageHandler.call(_)) + } + + /** + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the `StreamingContext`. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param ssc StreamingContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the stream + * @param messageHandler Function for translating each message and metadata into the desired type + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @tparam R type returned by messageHandler + * @return DStream of R + */ + def createDirectStream[ + K: ClassTag, + V: ClassTag, + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag, + R: ClassTag] ( + ssc: StreamingContext, + kafkaParams: Map[String, String], + fromOffsets: Map[TopicAndPartition, Long], + messageHandler: MessageAndMetadata[K, V] => R + ): InputDStream[R] = { + val cleanedHandler = ssc.sc.clean(messageHandler) + new DirectKafkaInputDStream[K, V, KD, VD, R]( + ssc, kafkaParams, fromOffsets, cleanedHandler) + } + + /** + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the `StreamingContext`. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param ssc StreamingContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers), specified in + * host1:port1,host2:port2 form. + * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" + * to determine where the stream starts (defaults to "largest") + * @param topics Names of the topics to consume + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @return DStream of (Kafka message key, Kafka message value) + */ + def createDirectStream[ + K: ClassTag, + V: ClassTag, + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag] ( + ssc: StreamingContext, + kafkaParams: Map[String, String], + topics: Set[String] + ): InputDStream[(K, V)] = { + val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) + val kc = new KafkaCluster(kafkaParams) + val fromOffsets = getFromOffsets(kc, kafkaParams, topics) + new DirectKafkaInputDStream[K, V, KD, VD, (K, V)]( + ssc, kafkaParams, fromOffsets, messageHandler) + } + + /** + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the `StreamingContext`. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param jssc JavaStreamingContext object + * @param keyClass Class of the keys in the Kafka records + * @param valueClass Class of the values in the Kafka records + * @param keyDecoderClass Class of the key decoder + * @param valueDecoderClass Class of the value decoder + * @param recordClass Class of the records in DStream + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers), specified in + * host1:port1,host2:port2 form. + * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the stream + * @param messageHandler Function for translating each message and metadata into the desired type + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @tparam R type returned by messageHandler + * @return DStream of R + */ + def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( + jssc: JavaStreamingContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + recordClass: Class[R], + kafkaParams: JMap[String, String], + fromOffsets: JMap[TopicAndPartition, JLong], + messageHandler: JFunction[MessageAndMetadata[K, V], R] + ): JavaInputDStream[R] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) + val cleanedHandler = jssc.sparkContext.clean(messageHandler.call _) + createDirectStream[K, V, KD, VD, R]( + jssc.ssc, + Map(kafkaParams.asScala.toSeq: _*), + Map(fromOffsets.asScala.mapValues(_.longValue()).toSeq: _*), + cleanedHandler + ) + } + + /** + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the `StreamingContext`. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param jssc JavaStreamingContext object + * @param keyClass Class of the keys in the Kafka records + * @param valueClass Class of the values in the Kafka records + * @param keyDecoderClass Class of the key decoder + * @param valueDecoderClass Class type of the value decoder + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers), specified in + * host1:port1,host2:port2 form. + * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" + * to determine where the stream starts (defaults to "largest") + * @param topics Names of the topics to consume + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @return DStream of (Kafka message key, Kafka message value) + */ + def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V]]( + jssc: JavaStreamingContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + kafkaParams: JMap[String, String], + topics: JSet[String] + ): JavaPairInputDStream[K, V] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + createDirectStream[K, V, KD, VD]( + jssc.ssc, + Map(kafkaParams.asScala.toSeq: _*), + Set(topics.asScala.toSeq: _*) + ) + } +} + +/** + * This is a helper class that wraps the KafkaUtils.createStream() into more + * Python-friendly class and function so that it can be easily + * instantiated and called from Python's KafkaUtils. + * + * The zero-arg constructor helps instantiate this class from the Class object + * classOf[KafkaUtilsPythonHelper].newInstance(), and the createStream() + * takes care of known parameters instead of passing them from Python + */ +private[kafka] class KafkaUtilsPythonHelper { + import KafkaUtilsPythonHelper._ + + def createStream( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JMap[String, JInt], + storageLevel: StorageLevel): JavaPairReceiverInputDStream[Array[Byte], Array[Byte]] = { + KafkaUtils.createStream[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder]( + jssc, + classOf[Array[Byte]], + classOf[Array[Byte]], + classOf[DefaultDecoder], + classOf[DefaultDecoder], + kafkaParams, + topics, + storageLevel) + } + + def createRDDWithoutMessageHandler( + jsc: JavaSparkContext, + kafkaParams: JMap[String, String], + offsetRanges: JList[OffsetRange], + leaders: JMap[TopicAndPartition, Broker]): JavaRDD[(Array[Byte], Array[Byte])] = { + val messageHandler = + (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => (mmd.key, mmd.message) + new JavaRDD(createRDD(jsc, kafkaParams, offsetRanges, leaders, messageHandler)) + } + + def createRDDWithMessageHandler( + jsc: JavaSparkContext, + kafkaParams: JMap[String, String], + offsetRanges: JList[OffsetRange], + leaders: JMap[TopicAndPartition, Broker]): JavaRDD[Array[Byte]] = { + val messageHandler = (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => + new PythonMessageAndMetadata( + mmd.topic, mmd.partition, mmd.offset, mmd.key(), mmd.message()) + val rdd = createRDD(jsc, kafkaParams, offsetRanges, leaders, messageHandler). + mapPartitions(picklerIterator) + new JavaRDD(rdd) + } + + private def createRDD[V: ClassTag]( + jsc: JavaSparkContext, + kafkaParams: JMap[String, String], + offsetRanges: JList[OffsetRange], + leaders: JMap[TopicAndPartition, Broker], + messageHandler: MessageAndMetadata[Array[Byte], Array[Byte]] => V): RDD[V] = { + KafkaUtils.createRDD[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder, V]( + jsc.sc, + kafkaParams.asScala.toMap, + offsetRanges.toArray(new Array[OffsetRange](offsetRanges.size())), + leaders.asScala.toMap, + messageHandler + ) + } + + def createDirectStreamWithoutMessageHandler( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JSet[String], + fromOffsets: JMap[TopicAndPartition, JNumber]): JavaDStream[(Array[Byte], Array[Byte])] = { + val messageHandler = + (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => (mmd.key, mmd.message) + new JavaDStream(createDirectStream(jssc, kafkaParams, topics, fromOffsets, messageHandler)) + } + + def createDirectStreamWithMessageHandler( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JSet[String], + fromOffsets: JMap[TopicAndPartition, JNumber]): JavaDStream[Array[Byte]] = { + val messageHandler = (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => + new PythonMessageAndMetadata(mmd.topic, mmd.partition, mmd.offset, mmd.key(), mmd.message()) + val stream = createDirectStream(jssc, kafkaParams, topics, fromOffsets, messageHandler). + mapPartitions(picklerIterator) + new JavaDStream(stream) + } + + private def createDirectStream[V: ClassTag]( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JSet[String], + fromOffsets: JMap[TopicAndPartition, JNumber], + messageHandler: MessageAndMetadata[Array[Byte], Array[Byte]] => V): DStream[V] = { + + val currentFromOffsets = if (!fromOffsets.isEmpty) { + val topicsFromOffsets = fromOffsets.keySet().asScala.map(_.topic) + if (topicsFromOffsets != topics.asScala.toSet) { + throw new IllegalStateException( + s"The specified topics: ${topics.asScala.toSet.mkString(" ")} " + + s"do not equal to the topic from offsets: ${topicsFromOffsets.mkString(" ")}") + } + Map(fromOffsets.asScala.mapValues { _.longValue() }.toSeq: _*) + } else { + val kc = new KafkaCluster(Map(kafkaParams.asScala.toSeq: _*)) + KafkaUtils.getFromOffsets( + kc, Map(kafkaParams.asScala.toSeq: _*), Set(topics.asScala.toSeq: _*)) + } + + KafkaUtils.createDirectStream[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder, V]( + jssc.ssc, + Map(kafkaParams.asScala.toSeq: _*), + Map(currentFromOffsets.toSeq: _*), + messageHandler) + } + + def createOffsetRange(topic: String, partition: JInt, fromOffset: JLong, untilOffset: JLong + ): OffsetRange = OffsetRange.create(topic, partition, fromOffset, untilOffset) + + def createTopicAndPartition(topic: String, partition: JInt): TopicAndPartition = + TopicAndPartition(topic, partition) + + def createBroker(host: String, port: JInt): Broker = Broker(host, port) + + def offsetRangesOfKafkaRDD(rdd: RDD[_]): JList[OffsetRange] = { + val parentRDDs = rdd.getNarrowAncestors + val kafkaRDDs = parentRDDs.filter(rdd => rdd.isInstanceOf[KafkaRDD[_, _, _, _, _]]) + + require( + kafkaRDDs.length == 1, + "Cannot get offset ranges, as there may be multiple Kafka RDDs or no Kafka RDD associated" + + "with this RDD, please call this method only on a Kafka RDD.") + + val kafkaRDD = kafkaRDDs.head.asInstanceOf[KafkaRDD[_, _, _, _, _]] + kafkaRDD.offsetRanges.toSeq.asJava + } +} + +private object KafkaUtilsPythonHelper { + private var initialized = false + + def initialize(): Unit = { + SerDeUtil.initialize() + synchronized { + if (!initialized) { + new PythonMessageAndMetadataPickler().register() + initialized = true + } + } + } + + initialize() + + def picklerIterator(iter: Iterator[Any]): Iterator[Array[Byte]] = { + new SerDeUtil.AutoBatchedPickler(iter) + } + + case class PythonMessageAndMetadata( + topic: String, + partition: JInt, + offset: JLong, + key: Array[Byte], + message: Array[Byte]) + + class PythonMessageAndMetadataPickler extends IObjectPickler { + private val module = "pyspark.streaming.kafka" + + def register(): Unit = { + Pickler.registerCustomPickler(classOf[PythonMessageAndMetadata], this) + Pickler.registerCustomPickler(this.getClass, this) + } + + def pickle(obj: Object, out: OutputStream, pickler: Pickler) { + if (obj == this) { + out.write(Opcodes.GLOBAL) + out.write(s"$module\nKafkaMessageAndMetadata\n".getBytes(StandardCharsets.UTF_8)) + } else { + pickler.save(this) + val msgAndMetaData = obj.asInstanceOf[PythonMessageAndMetadata] + out.write(Opcodes.MARK) + pickler.save(msgAndMetaData.topic) + pickler.save(msgAndMetaData.partition) + pickler.save(msgAndMetaData.offset) + pickler.save(msgAndMetaData.key) + pickler.save(msgAndMetaData.message) + out.write(Opcodes.TUPLE) + out.write(Opcodes.REDUCE) + } + } + } +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala new file mode 100644 index 000000000000..6dab5f950d4c --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import kafka.common.TopicAndPartition + +/** + * Represents any object that has a collection of [[OffsetRange]]s. This can be used to access the + * offset ranges in RDDs generated by the direct Kafka DStream (see + * `KafkaUtils.createDirectStream()`). + * {{{ + * KafkaUtils.createDirectStream(...).foreachRDD { rdd => + * val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + * ... + * } + * }}} + */ +@deprecated("Update to Kafka 0.10 integration", "2.3.0") +trait HasOffsetRanges { + def offsetRanges: Array[OffsetRange] +} + +/** + * Represents a range of offsets from a single Kafka TopicAndPartition. Instances of this class + * can be created with `OffsetRange.create()`. + * @param topic Kafka topic name + * @param partition Kafka partition id + * @param fromOffset Inclusive starting offset + * @param untilOffset Exclusive ending offset + */ +@deprecated("Update to Kafka 0.10 integration", "2.3.0") +final class OffsetRange private( + val topic: String, + val partition: Int, + val fromOffset: Long, + val untilOffset: Long) extends Serializable { + import OffsetRange.OffsetRangeTuple + + /** Kafka TopicAndPartition object, for convenience */ + def topicAndPartition(): TopicAndPartition = TopicAndPartition(topic, partition) + + /** Number of messages this OffsetRange refers to */ + def count(): Long = untilOffset - fromOffset + + override def equals(obj: Any): Boolean = obj match { + case that: OffsetRange => + this.topic == that.topic && + this.partition == that.partition && + this.fromOffset == that.fromOffset && + this.untilOffset == that.untilOffset + case _ => false + } + + override def hashCode(): Int = { + toTuple.hashCode() + } + + override def toString(): String = { + s"OffsetRange(topic: '$topic', partition: $partition, range: [$fromOffset -> $untilOffset])" + } + + /** this is to avoid ClassNotFoundException during checkpoint restore */ + private[streaming] + def toTuple: OffsetRangeTuple = (topic, partition, fromOffset, untilOffset) +} + +/** + * Companion object the provides methods to create instances of [[OffsetRange]]. + */ +@deprecated("Update to Kafka 0.10 integration", "2.3.0") +object OffsetRange { + def create(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = + new OffsetRange(topic, partition, fromOffset, untilOffset) + + def create( + topicAndPartition: TopicAndPartition, + fromOffset: Long, + untilOffset: Long): OffsetRange = + new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) + + def apply(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = + new OffsetRange(topic, partition, fromOffset, untilOffset) + + def apply( + topicAndPartition: TopicAndPartition, + fromOffset: Long, + untilOffset: Long): OffsetRange = + new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) + + /** this is to avoid ClassNotFoundException during checkpoint restore */ + private[kafka] + type OffsetRangeTuple = (String, Int, Long, Long) + + private[kafka] + def apply(t: OffsetRangeTuple) = + new OffsetRange(t._1, t._2, t._3, t._4) +} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala similarity index 98% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index 764d170934aa..39abe3c3e29d 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -18,10 +18,10 @@ package org.apache.spark.streaming.kafka import java.util.Properties -import java.util.concurrent.{ThreadPoolExecutor, ConcurrentHashMap} +import java.util.concurrent.{ConcurrentHashMap, ThreadPoolExecutor} -import scala.collection.{Map, mutable} -import scala.reflect.{ClassTag, classTag} +import scala.collection.{mutable, Map} +import scala.reflect.{classTag, ClassTag} import kafka.common.TopicAndPartition import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector, KafkaStream} @@ -30,7 +30,8 @@ import kafka.serializer.Decoder import kafka.utils.{VerifiableProperties, ZKGroupTopicDirs, ZKStringSerializer, ZkUtils} import org.I0Itec.zkclient.ZkClient -import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.SparkEnv +import org.apache.spark.internal.Logging import org.apache.spark.storage.{StorageLevel, StreamBlockId} import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver} import org.apache.spark.util.ThreadUtils diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package-info.java b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package-info.java new file mode 100644 index 000000000000..2e5ab0fb3bef --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Kafka receiver for spark streaming. + */ +package org.apache.spark.streaming.kafka; diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package.scala similarity index 100% rename from external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package.scala diff --git a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java new file mode 100644 index 000000000000..71404a7331ec --- /dev/null +++ b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -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. + */ + +package org.apache.spark.streaming.kafka; + +import java.io.Serializable; +import java.util.*; +import java.util.concurrent.atomic.AtomicReference; + +import scala.Tuple2; + +import kafka.common.TopicAndPartition; +import kafka.message.MessageAndMetadata; +import kafka.serializer.StringDecoder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.VoidFunction; +import org.apache.spark.streaming.Durations; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; + +public class JavaDirectKafkaStreamSuite implements Serializable { + private transient JavaStreamingContext ssc = null; + private transient KafkaTestUtils kafkaTestUtils = null; + + @Before + public void setUp() { + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200)); + } + + @After + public void tearDown() { + if (ssc != null) { + ssc.stop(); + ssc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } + } + + @Test + public void testKafkaStream() throws InterruptedException { + final String topic1 = "topic1"; + final String topic2 = "topic2"; + // hold a reference to the current offset ranges, so it can be used downstream + final AtomicReference offsetRanges = new AtomicReference<>(); + + String[] topic1data = createTopicAndSendData(topic1); + String[] topic2data = createTopicAndSendData(topic2); + + Set sent = new HashSet<>(); + sent.addAll(Arrays.asList(topic1data)); + sent.addAll(Arrays.asList(topic2data)); + + Map kafkaParams = new HashMap<>(); + kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); + kafkaParams.put("auto.offset.reset", "smallest"); + + JavaDStream stream1 = KafkaUtils.createDirectStream( + ssc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + kafkaParams, + topicToSet(topic1) + ).transformToPair( + // Make sure you can get offset ranges from the rdd + new Function, JavaPairRDD>() { + @Override + public JavaPairRDD call(JavaPairRDD rdd) { + OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + offsetRanges.set(offsets); + Assert.assertEquals(topic1, offsets[0].topic()); + return rdd; + } + } + ).map( + new Function, String>() { + @Override + public String call(Tuple2 kv) { + return kv._2(); + } + } + ); + + JavaDStream stream2 = KafkaUtils.createDirectStream( + ssc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + String.class, + kafkaParams, + topicOffsetToMap(topic2, 0L), + new Function, String>() { + @Override + public String call(MessageAndMetadata msgAndMd) { + return msgAndMd.message(); + } + } + ); + JavaDStream unifiedStream = stream1.union(stream2); + + final Set result = Collections.synchronizedSet(new HashSet()); + unifiedStream.foreachRDD(new VoidFunction>() { + @Override + public void call(JavaRDD rdd) { + result.addAll(rdd.collect()); + } + } + ); + ssc.start(); + long startTime = System.currentTimeMillis(); + boolean matches = false; + while (!matches && System.currentTimeMillis() - startTime < 20000) { + matches = sent.size() == result.size(); + Thread.sleep(50); + } + Assert.assertEquals(sent, result); + ssc.stop(); + } + + private static Set topicToSet(String topic) { + Set topicSet = new HashSet<>(); + topicSet.add(topic); + return topicSet; + } + + private static Map topicOffsetToMap(String topic, Long offsetToStart) { + Map topicMap = new HashMap<>(); + topicMap.put(new TopicAndPartition(topic, 0), offsetToStart); + return topicMap; + } + + private String[] createTopicAndSendData(String topic) { + String[] data = { topic + "-1", topic + "-2", topic + "-3"}; + kafkaTestUtils.createTopic(topic, 1); + kafkaTestUtils.sendMessages(topic, data); + return data; + } +} diff --git a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java new file mode 100644 index 000000000000..c41b6297b048 --- /dev/null +++ b/external/kafka-0-8/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.Map; + +import scala.Tuple2; + +import kafka.common.TopicAndPartition; +import kafka.message.MessageAndMetadata; +import kafka.serializer.StringDecoder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; + +public class JavaKafkaRDDSuite implements Serializable { + private transient JavaSparkContext sc = null; + private transient KafkaTestUtils kafkaTestUtils = null; + + @Before + public void setUp() { + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + sc = new JavaSparkContext(sparkConf); + } + + @After + public void tearDown() { + if (sc != null) { + sc.stop(); + sc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } + } + + @Test + public void testKafkaRDD() throws InterruptedException { + String topic1 = "topic1"; + String topic2 = "topic2"; + + createTopicAndSendData(topic1); + createTopicAndSendData(topic2); + + Map kafkaParams = new HashMap<>(); + kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); + + OffsetRange[] offsetRanges = { + OffsetRange.create(topic1, 0, 0, 1), + OffsetRange.create(topic2, 0, 0, 1) + }; + + Map emptyLeaders = new HashMap<>(); + Map leaders = new HashMap<>(); + String[] hostAndPort = kafkaTestUtils.brokerAddress().split(":"); + Broker broker = Broker.create(hostAndPort[0], Integer.parseInt(hostAndPort[1])); + leaders.put(new TopicAndPartition(topic1, 0), broker); + leaders.put(new TopicAndPartition(topic2, 0), broker); + + JavaRDD rdd1 = KafkaUtils.createRDD( + sc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + kafkaParams, + offsetRanges + ).map( + new Function, String>() { + @Override + public String call(Tuple2 kv) { + 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) { + 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) { + 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"}; + kafkaTestUtils.createTopic(topic, 1); + kafkaTestUtils.sendMessages(topic, data); + return data; + } +} diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java similarity index 83% rename from external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java rename to external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 1e69de46cd35..98fe38e826af 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -31,6 +31,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; @@ -75,7 +76,7 @@ public void testKafkaStream() throws InterruptedException { sent.put("b", 3); sent.put("c", 10); - kafkaTestUtils.createTopic(topic); + kafkaTestUtils.createTopic(topic, 1); kafkaTestUtils.sendMessages(topic, sent); Map kafkaParams = new HashMap<>(); @@ -103,10 +104,9 @@ public String call(Tuple2 tuple2) { } ); - words.countByValue().foreachRDD( - new Function, Void>() { + words.countByValue().foreachRDD(new VoidFunction>() { @Override - public Void call(JavaPairRDD rdd) { + public void call(JavaPairRDD rdd) { List> ret = rdd.collect(); for (Tuple2 r : ret) { if (result.containsKey(r._1())) { @@ -115,8 +115,6 @@ public Void call(JavaPairRDD rdd) { result.put(r._1(), r._2()); } } - - return null; } } ); @@ -124,14 +122,23 @@ public Void call(JavaPairRDD rdd) { ssc.start(); long startTime = System.currentTimeMillis(); - boolean sizeMatches = false; - while (!sizeMatches && System.currentTimeMillis() - startTime < 20000) { - sizeMatches = sent.size() == result.size(); + AssertionError lastError = null; + while (System.currentTimeMillis() - startTime < 20000) { + try { + Assert.assertEquals(sent.size(), result.size()); + for (Map.Entry e : sent.entrySet()) { + Assert.assertEquals(e.getValue().intValue(), result.get(e.getKey()).intValue()); + } + return; + } catch (AssertionError e) { + lastError = e; + } Thread.sleep(200); } - Assert.assertEquals(sent.size(), result.size()); - for (Map.Entry e : sent.entrySet()) { - Assert.assertEquals(e.getValue().intValue(), result.get(e.getKey()).intValue()); + if (lastError != null) { + throw lastError; + } else { + Assert.fail("timeout"); } } } diff --git a/external/kafka-0-8/src/test/resources/log4j.properties b/external/kafka-0-8/src/test/resources/log4j.properties new file mode 100644 index 000000000000..fd51f8faf56b --- /dev/null +++ b/external/kafka-0-8/src/test/resources/log4j.properties @@ -0,0 +1,28 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.spark_project.jetty=WARN + diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala new file mode 100644 index 000000000000..06ef5bc3f8bd --- /dev/null +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -0,0 +1,527 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.io.File +import java.util.Arrays +import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.atomic.AtomicLong + +import scala.collection.JavaConverters._ +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 + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.streaming.scheduler.rate.RateEstimator +import org.apache.spark.util.Utils + +class DirectKafkaStreamSuite + extends SparkFunSuite + with BeforeAndAfter + with BeforeAndAfterAll + with Eventually + with Logging { + val sparkConf = new SparkConf() + .setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + + private var ssc: StreamingContext = _ + private var testDir: File = _ + + private var kafkaTestUtils: KafkaTestUtils = _ + + override def beforeAll { + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + } + + override def afterAll { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + after { + if (ssc != null) { + ssc.stop(stopSparkContext = true) + } + if (testDir != null) { + Utils.deleteRecursively(testDir) + } + } + + + test("basic stream receiving with multiple topics and smallest starting offset") { + val topics = Set("basic1", "basic2", "basic3") + val data = Map("a" -> 7, "b" -> 9) + topics.foreach { t => + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) + } + val totalSent = data.values.sum * topics.size + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, topics) + } + + val allReceived = new ConcurrentLinkedQueue[(String, String)]() + + // hold a reference to the current offset ranges, so it can be used downstream + var offsetRanges = Array[OffsetRange]() + + stream.transform { rdd => + // Get the offset ranges in the RDD + offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + rdd + }.foreachRDD { rdd => + for (o <- offsetRanges) { + logInfo(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") + } + val collected = rdd.mapPartitionsWithIndex { (i, iter) => + // For each partition, get size of the range in the partition, + // and the number of items in the partition + val off = offsetRanges(i) + val all = iter.toSeq + val partSize = all.size + val rangeSize = off.untilOffset - off.fromOffset + Iterator((partSize, rangeSize)) + }.collect + + // Verify whether number of elements in each partition + // matches with the corresponding offset range + collected.foreach { case (partSize, rangeSize) => + assert(partSize === rangeSize, "offset ranges are wrong") + } + } + stream.foreachRDD { rdd => allReceived.addAll(Arrays.asList(rdd.collect(): _*)) } + ssc.start() + eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { + assert(allReceived.size === totalSent, + "didn't get expected number of messages, messages:\n" + + allReceived.asScala.mkString("\n")) + } + ssc.stop() + } + + test("receiving from largest starting offset") { + val topic = "largest" + val topicPartition = TopicAndPartition(topic, 0) + val data = Map("a" -> 10) + kafkaTestUtils.createTopic(topic) + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "largest" + ) + val kc = new KafkaCluster(kafkaParams) + def getLatestOffset(): Long = { + kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset + } + + // Send some initial messages before starting context + kafkaTestUtils.sendMessages(topic, data) + eventually(timeout(10 seconds), interval(20 milliseconds)) { + assert(getLatestOffset() > 3) + } + val offsetBeforeStart = getLatestOffset() + + // Setup context and kafka stream with largest offset + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, Set(topic)) + } + assert( + stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]] + .fromOffsets(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + + val collectedData = new ConcurrentLinkedQueue[String]() + stream.map { _._2 }.foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } + ssc.start() + val newData = Map("b" -> 10) + kafkaTestUtils.sendMessages(topic, newData) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + collectedData.contains("b") + } + assert(!collectedData.contains("a")) + ssc.stop() + } + + + test("creating stream by offset") { + val topic = "offset" + val topicPartition = TopicAndPartition(topic, 0) + val data = Map("a" -> 10) + kafkaTestUtils.createTopic(topic) + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "largest" + ) + val kc = new KafkaCluster(kafkaParams) + def getLatestOffset(): Long = { + kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset + } + + // Send some initial messages before starting context + kafkaTestUtils.sendMessages(topic, data) + eventually(timeout(10 seconds), interval(20 milliseconds)) { + assert(getLatestOffset() >= 10) + } + val offsetBeforeStart = getLatestOffset() + + // Setup context and kafka stream with largest offset + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder, String]( + ssc, kafkaParams, Map(topicPartition -> 11L), + (m: MessageAndMetadata[String, String]) => m.message()) + } + assert( + stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]] + .fromOffsets(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + + val collectedData = new ConcurrentLinkedQueue[String]() + stream.foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } + ssc.start() + val newData = Map("b" -> 10) + kafkaTestUtils.sendMessages(topic, newData) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + collectedData.contains("b") + } + assert(!collectedData.contains("a")) + ssc.stop() + } + + // Test to verify the offset ranges can be recovered from the checkpoints + test("offset recovery") { + val topic = "recovery" + kafkaTestUtils.createTopic(topic) + testDir = Utils.createTempDir() + + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + // Send data to Kafka and wait for it to be received + def sendData(data: Seq[Int]) { + val strings = data.map { _.toString} + kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) + } + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(100)) + val kafkaStream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, Set(topic)) + } + val keyedStream = kafkaStream.map { v => "key" -> v._2.toInt } + val stateStream = keyedStream.updateStateByKey { (values: Seq[Int], state: Option[Int]) => + Some(values.sum + state.getOrElse(0)) + } + ssc.checkpoint(testDir.getAbsolutePath) + + // This is ensure all the data is eventually receiving only once + stateStream.foreachRDD { (rdd: RDD[(String, Int)]) => + rdd.collect().headOption.foreach { x => + DirectKafkaStreamSuite.total.set(x._2) + } + } + ssc.start() + + // Send some data + for (i <- (1 to 10).grouped(4)) { + sendData(i) + } + + eventually(timeout(20 seconds), interval(50 milliseconds)) { + assert(DirectKafkaStreamSuite.total.get === (1 to 10).sum) + } + + ssc.stop() + + // Verify that offset ranges were generated + // Since "offsetRangesAfterStop" will be used to compare with "recoveredOffsetRanges", we should + // collect offset ranges after stopping. Otherwise, because new RDDs keep being generated before + // stopping, we may not be able to get the latest RDDs, then "recoveredOffsetRanges" will + // contain something not in "offsetRangesAfterStop". + val offsetRangesAfterStop = getOffsetRanges(kafkaStream) + assert(offsetRangesAfterStop.size >= 1, "No offset ranges generated") + assert( + offsetRangesAfterStop.head._2.forall { _.fromOffset === 0 }, + "starting offset not zero" + ) + + logInfo("====== RESTARTING ========") + + // Recover context from checkpoints + ssc = new StreamingContext(testDir.getAbsolutePath) + val recoveredStream = ssc.graph.getInputStreams().head.asInstanceOf[DStream[(String, String)]] + + // Verify offset ranges have been recovered + val recoveredOffsetRanges = getOffsetRanges(recoveredStream).map { x => (x._1, x._2.toSet) } + assert(recoveredOffsetRanges.size > 0, "No offset ranges recovered") + val earlierOffsetRanges = offsetRangesAfterStop.map { x => (x._1, x._2.toSet) } + assert( + recoveredOffsetRanges.forall { or => + earlierOffsetRanges.contains((or._1, or._2)) + }, + "Recovered ranges are not the same as the ones generated\n" + + s"recoveredOffsetRanges: $recoveredOffsetRanges\n" + + s"earlierOffsetRanges: $earlierOffsetRanges" + ) + // Restart context, give more data and verify the total at the end + // If the total is write that means each records has been received only once + ssc.start() + for (i <- (11 to 20).grouped(4)) { + sendData(i) + } + + eventually(timeout(20 seconds), interval(50 milliseconds)) { + assert(DirectKafkaStreamSuite.total.get === (1 to 20).sum) + } + ssc.stop() + } + + test("Direct Kafka stream report input information") { + val topic = "report-test" + val data = Map("a" -> 7, "b" -> 9) + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, data) + + val totalSent = data.values.sum + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + import DirectKafkaStreamSuite._ + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val collector = new InputInfoCollector + ssc.addStreamingListener(collector) + + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, Set(topic)) + } + + val allReceived = new ConcurrentLinkedQueue[(String, String)] + + stream.foreachRDD { rdd => allReceived.addAll(Arrays.asList(rdd.collect(): _*)) } + ssc.start() + eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { + assert(allReceived.size === totalSent, + "didn't get expected number of messages, messages:\n" + + allReceived.asScala.mkString("\n")) + + // Calculate all the record number collected in the StreamingListener. + assert(collector.numRecordsSubmitted.get() === totalSent) + assert(collector.numRecordsStarted.get() === totalSent) + assert(collector.numRecordsCompleted.get() === totalSent) + } + ssc.stop() + } + + test("maxMessagesPerPartition with backpressure disabled") { + val topic = "maxMessagesPerPartition" + val kafkaStream = getDirectKafkaStream(topic, None) + + val input = Map(TopicAndPartition(topic, 0) -> 50L, TopicAndPartition(topic, 1) -> 50L) + assert(kafkaStream.maxMessagesPerPartition(input).get == + Map(TopicAndPartition(topic, 0) -> 10L, TopicAndPartition(topic, 1) -> 10L)) + } + + test("maxMessagesPerPartition with no lag") { + val topic = "maxMessagesPerPartition" + val rateController = Some(new ConstantRateController(0, new ConstantEstimator(100), 100)) + val kafkaStream = getDirectKafkaStream(topic, rateController) + + val input = Map(TopicAndPartition(topic, 0) -> 0L, TopicAndPartition(topic, 1) -> 0L) + assert(kafkaStream.maxMessagesPerPartition(input).isEmpty) + } + + test("maxMessagesPerPartition respects max rate") { + val topic = "maxMessagesPerPartition" + val rateController = Some(new ConstantRateController(0, new ConstantEstimator(100), 1000)) + val kafkaStream = getDirectKafkaStream(topic, rateController) + + val input = Map(TopicAndPartition(topic, 0) -> 1000L, TopicAndPartition(topic, 1) -> 1000L) + assert(kafkaStream.maxMessagesPerPartition(input).get == + Map(TopicAndPartition(topic, 0) -> 10L, TopicAndPartition(topic, 1) -> 10L)) + } + + test("using rate controller") { + val topic = "backpressure" + val topicPartitions = Set(TopicAndPartition(topic, 0), TopicAndPartition(topic, 1)) + kafkaTestUtils.createTopic(topic, 2) + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + val batchIntervalMilliseconds = 100 + val estimator = new ConstantEstimator(100) + val messages = Map("foo" -> 200) + kafkaTestUtils.sendMessages(topic, messages) + + val sparkConf = new SparkConf() + // Safe, even with streaming, because we're using the direct API. + // Using 1 core is useful to make the test more predictable. + .setMaster("local[1]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.kafka.maxRatePerPartition", "100") + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) + + val kafkaStream = withClue("Error creating direct stream") { + val kc = new KafkaCluster(kafkaParams) + val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) + val m = kc.getEarliestLeaderOffsets(topicPartitions) + .fold(e => Map.empty[TopicAndPartition, Long], m => m.mapValues(lo => lo.offset)) + + new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( + ssc, kafkaParams, m, messageHandler) { + override protected[streaming] val rateController = + Some(new DirectKafkaRateController(id, estimator)) + } + } + + val collectedData = new ConcurrentLinkedQueue[Array[String]]() + + // Used for assertion failure messages. + def dataToString: String = + collectedData.asScala.map(_.mkString("[", ",", "]")).mkString("{", ", ", "}") + + // This is to collect the raw data received from Kafka + kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) => + val data = rdd.map { _._2 }.collect() + collectedData.add(data) + } + + ssc.start() + + // Try different rate limits. + // Wait for arrays of data to appear matching the rate. + Seq(100, 50, 20).foreach { rate => + collectedData.clear() // Empty this buffer on each pass. + estimator.updateRate(rate) // Set a new rate. + // Expect blocks of data equal to "rate", scaled by the interval length in secs. + val expectedSize = Math.round(rate * batchIntervalMilliseconds * 0.001) + eventually(timeout(5.seconds), interval(batchIntervalMilliseconds.milliseconds)) { + // Assert that rate estimator values are used to determine maxMessagesPerPartition. + // Funky "-" in message makes the complete assertion message read better. + assert(collectedData.asScala.exists(_.size == expectedSize), + s" - No arrays of size $expectedSize for rate $rate found in $dataToString") + } + } + + ssc.stop() + } + + /** Get the generated offset ranges from the DirectKafkaStream */ + private def getOffsetRanges[K, V]( + kafkaStream: DStream[(K, V)]): Seq[(Time, Array[OffsetRange])] = { + kafkaStream.generatedRDDs.mapValues { rdd => + rdd.asInstanceOf[KafkaRDD[K, V, _, _, (K, V)]].offsetRanges + }.toSeq.sortBy { _._1 } + } + + private def getDirectKafkaStream(topic: String, mockRateController: Option[RateController]) = { + val batchIntervalMilliseconds = 100 + + val sparkConf = new SparkConf() + .setMaster("local[1]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.kafka.maxRatePerPartition", "100") + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) + + val earliestOffsets = Map(TopicAndPartition(topic, 0) -> 0L, TopicAndPartition(topic, 1) -> 0L) + val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) + new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( + ssc, Map[String, String](), earliestOffsets, messageHandler) { + override protected[streaming] val rateController = mockRateController + } + } +} + +object DirectKafkaStreamSuite { + val total = new AtomicLong(-1L) + + class InputInfoCollector extends StreamingListener { + val numRecordsSubmitted = new AtomicLong(0L) + val numRecordsStarted = new AtomicLong(0L) + val numRecordsCompleted = new AtomicLong(0L) + + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit = { + numRecordsSubmitted.addAndGet(batchSubmitted.batchInfo.numRecords) + } + + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = { + numRecordsStarted.addAndGet(batchStarted.batchInfo.numRecords) + } + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { + numRecordsCompleted.addAndGet(batchCompleted.batchInfo.numRecords) + } + } +} + +private[streaming] class ConstantEstimator(@volatile private var rate: Long) + extends RateEstimator { + + def updateRate(newRate: Long): Unit = { + rate = newRate + } + + def compute( + time: Long, + elements: Long, + processingDelay: Long, + schedulingDelay: Long): Option[Double] = Some(rate) +} + +private[streaming] class ConstantRateController(id: Int, estimator: RateEstimator, rate: Long) + extends RateController(id, estimator) { + override def publish(rate: Long): Unit = () + override def getLatestRate(): Long = rate +} diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala similarity index 100% rename from external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala rename to external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala new file mode 100644 index 000000000000..809699a73996 --- /dev/null +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import scala.util.Random + +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata +import kafka.serializer.StringDecoder +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark._ + +class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { + + private var kafkaTestUtils: KafkaTestUtils = _ + + private val sparkConf = new SparkConf().setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + private var sc: SparkContext = _ + + override def beforeAll { + sc = new SparkContext(sparkConf) + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + } + + override def afterAll { + if (sc != null) { + sc.stop + sc = null + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + test("basic usage") { + val topic = s"topicbasic-${Random.nextInt}-${System.currentTimeMillis}" + kafkaTestUtils.createTopic(topic) + val messages = Array("the", "quick", "brown", "fox") + kafkaTestUtils.sendMessages(topic, messages) + + val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "group.id" -> s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}") + + 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.toSet) + + // size-related method optimizations return sane results + assert(rdd.count === messages.size) + assert(rdd.countApprox(0).getFinalValue.mean === messages.size) + assert(!rdd.isEmpty) + assert(rdd.take(1).size === 1) + assert(rdd.take(1).head._2 === messages.head) + assert(rdd.take(messages.size + 10).size === messages.size) + + val emptyRdd = KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( + sc, kafkaParams, Array(OffsetRange(topic, 0, 0, 0))) + + assert(emptyRdd.isEmpty) + + // invalid offset ranges throw exceptions + val badRanges = Array(OffsetRange(topic, 0, 0, messages.size + 1)) + intercept[SparkException] { + KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( + sc, kafkaParams, badRanges) + } + } + + 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 = s"topicboundary-${Random.nextInt}-${System.currentTimeMillis}" + val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) + kafkaTestUtils.createTopic(topic) + + val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "group.id" -> s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}") + + val kc = new KafkaCluster(kafkaParams) + + // this is the "lots of messages" case + kafkaTestUtils.sendMessages(topic, sent) + val sentCount = sent.values.sum + + // rdd defined from leaders after sending messages, should get the number sent + val rdd = getRdd(kc, Set(topic)) + + assert(rdd.isDefined) + + val ranges = rdd.get.asInstanceOf[HasOffsetRanges].offsetRanges + val rangeCount = ranges.map(o => o.untilOffset - o.fromOffset).sum + + assert(rangeCount === sentCount, "offset range didn't include all sent messages") + assert(rdd.get.count === sentCount, "didn't get all sent messages") + + val rangesMap = ranges.map(o => TopicAndPartition(o.topic, o.partition) -> o.untilOffset).toMap + + // make sure consumer offsets are committed before the next getRdd call + kc.setConsumerOffsets(kafkaParams("group.id"), rangesMap).fold( + err => throw new Exception(err.mkString("\n")), + _ => () + ) + + // this is the "0 messages" case + val rdd2 = getRdd(kc, Set(topic)) + // shouldn't get anything, since message is sent after rdd was defined + val sentOnlyOne = Map("d" -> 1) + + kafkaTestUtils.sendMessages(topic, sentOnlyOne) + + assert(rdd2.isDefined) + assert(rdd2.get.count === 0, "got messages when there shouldn't be any") + + // this is the "exactly 1 message" case, namely the single message from sentOnlyOne above + val rdd3 = getRdd(kc, Set(topic)) + // send lots of messages after rdd was defined, they shouldn't show up + kafkaTestUtils.sendMessages(topic, Map("extra" -> 22)) + + assert(rdd3.isDefined) + 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") + 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) + } + ) + } + 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}") + } + } + } + } +} diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala similarity index 89% rename from external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala rename to external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index 797b07f80d8e..426cd83b4ddf 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -65,19 +65,21 @@ class KafkaStreamSuite extends SparkFunSuite with Eventually with BeforeAndAfter val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY) - val result = new mutable.HashMap[String, Long]() with mutable.SynchronizedMap[String, Long] + val result = new mutable.HashMap[String, Long]() stream.map(_._2).countByValue().foreachRDD { r => - val ret = r.collect() - ret.toMap.foreach { kv => - val count = result.getOrElseUpdate(kv._1, 0) + kv._2 - result.put(kv._1, count) + r.collect().foreach { kv => + result.synchronized { + val count = result.getOrElseUpdate(kv._1, 0) + kv._2 + result.put(kv._1, count) + } } } ssc.start() eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { - assert(sent === result) + assert(result.synchronized { sent === result }) } + ssc.stop() } } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala similarity index 98% rename from external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala rename to external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 80e2df62de3f..57f89cc7dbc6 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -50,7 +50,7 @@ class ReliableKafkaStreamSuite extends SparkFunSuite private var ssc: StreamingContext = _ private var tempDirectory: File = null - override def beforeAll() : Unit = { + override def beforeAll(): Unit = { kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() @@ -80,7 +80,7 @@ class ReliableKafkaStreamSuite extends SparkFunSuite after { if (ssc != null) { - ssc.stop() + ssc.stop(stopSparkContext = true) ssc = null } } diff --git a/external/kafka-assembly/pom.xml b/external/kafka-assembly/pom.xml deleted file mode 100644 index a9ed39ef8c9a..000000000000 --- a/external/kafka-assembly/pom.xml +++ /dev/null @@ -1,186 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-kafka-assembly_2.10 - jar - Spark Project External Kafka Assembly - http://spark.apache.org/ - - - streaming-kafka-assembly - - - - - org.apache.spark - spark-streaming-kafka_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - - commons-codec - commons-codec - provided - - - commons-lang - commons-lang - provided - - - com.google.protobuf - protobuf-java - provided - - - com.sun.jersey - jersey-server - provided - - - com.sun.jersey - jersey-core - provided - - - net.jpountz.lz4 - lz4 - provided - - - org.apache.hadoop - hadoop-client - provided - - - org.apache.avro - avro-mapred - ${avro.mapred.classifier} - provided - - - org.apache.curator - curator-recipes - provided - - - org.apache.zookeeper - zookeeper - provided - - - log4j - log4j - provided - - - net.java.dev.jets3t - jets3t - provided - - - org.scala-lang - scala-library - provided - - - org.slf4j - slf4j-api - provided - - - org.slf4j - slf4j-log4j12 - provided - - - org.xerial.snappy - snappy-java - provided - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-shade-plugin - - false - - - *:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - package - - shade - - - - - - reference.conf - - - log4j.properties - - - - - - - - - - - - diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml deleted file mode 100644 index 79258c126e04..000000000000 --- a/external/kafka/pom.xml +++ /dev/null @@ -1,98 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-kafka_2.10 - - streaming-kafka - - jar - Spark Project External Kafka - http://spark.apache.org/ - - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - org.apache.kafka - kafka_${scala.binary.version} - 0.8.2.1 - - - com.sun.jmx - jmxri - - - com.sun.jdmk - jmxtools - - - net.sf.jopt-simple - jopt-simple - - - org.slf4j - slf4j-simple - - - org.apache.zookeeper - zookeeper - - - - - net.sf.jopt-simple - jopt-simple - 3.2 - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala deleted file mode 100644 index 8a087474d316..000000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ /dev/null @@ -1,214 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import scala.annotation.tailrec -import scala.collection.mutable -import scala.reflect.ClassTag - -import kafka.common.TopicAndPartition -import kafka.message.MessageAndMetadata -import kafka.serializer.Decoder - -import org.apache.spark.{Logging, SparkException} -import org.apache.spark.streaming.{StreamingContext, Time} -import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset -import org.apache.spark.streaming.scheduler.{RateController, StreamInputInfo} -import org.apache.spark.streaming.scheduler.rate.RateEstimator - -/** - * A stream of {@link org.apache.spark.streaming.kafka.KafkaRDD} where - * each given Kafka topic/partition corresponds to an RDD partition. - * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number - * of messages - * per second that each '''partition''' will accept. - * Starting offsets are specified in advance, - * and this DStream is not responsible for committing offsets, - * so that you can control exactly-once semantics. - * For an easy interface to Kafka-managed offsets, - * see {@link org.apache.spark.streaming.kafka.KafkaCluster} - * @param kafkaParams Kafka - * configuration parameters. - * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. - * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) - * starting point of the stream - * @param messageHandler function for translating each message into the desired type - */ -private[streaming] -class DirectKafkaInputDStream[ - K: ClassTag, - V: ClassTag, - U <: Decoder[K]: ClassTag, - T <: Decoder[V]: ClassTag, - R: ClassTag]( - ssc_ : StreamingContext, - val kafkaParams: Map[String, String], - val fromOffsets: Map[TopicAndPartition, Long], - messageHandler: MessageAndMetadata[K, V] => R - ) extends InputDStream[R](ssc_) with Logging { - val maxRetries = context.sparkContext.getConf.getInt( - "spark.streaming.kafka.maxRetries", 1) - - // Keep this consistent with how other streams are named (e.g. "Flume polling stream [2]") - private[streaming] override def name: String = s"Kafka direct stream [$id]" - - protected[streaming] override val checkpointData = - new DirectKafkaInputDStreamCheckpointData - - - /** - * Asynchronously maintains & sends new rate limits to the receiver through the receiver tracker. - */ - override protected[streaming] val rateController: Option[RateController] = { - if (RateController.isBackPressureEnabled(ssc.conf)) { - Some(new DirectKafkaRateController(id, - RateEstimator.create(ssc.conf, context.graph.batchDuration))) - } else { - None - } - } - - protected val kc = new KafkaCluster(kafkaParams) - - private val maxRateLimitPerPartition: Int = context.sparkContext.getConf.getInt( - "spark.streaming.kafka.maxRatePerPartition", 0) - protected def maxMessagesPerPartition: Option[Long] = { - val estimatedRateLimit = rateController.map(_.getLatestRate().toInt) - val numPartitions = currentOffsets.keys.size - - val effectiveRateLimitPerPartition = estimatedRateLimit - .filter(_ > 0) - .map { limit => - if (maxRateLimitPerPartition > 0) { - Math.min(maxRateLimitPerPartition, (limit / numPartitions)) - } else { - limit / numPartitions - } - }.getOrElse(maxRateLimitPerPartition) - - if (effectiveRateLimitPerPartition > 0) { - val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000 - Some((secsPerBatch * effectiveRateLimitPerPartition).toLong) - } else { - None - } - } - - protected var currentOffsets = fromOffsets - - @tailrec - protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = { - val o = kc.getLatestLeaderOffsets(currentOffsets.keySet) - // Either.fold would confuse @tailrec, do it manually - if (o.isLeft) { - val err = o.left.get.toString - if (retries <= 0) { - throw new SparkException(err) - } else { - log.error(err) - Thread.sleep(kc.config.refreshLeaderBackoffMs) - latestLeaderOffsets(retries - 1) - } - } else { - o.right.get - } - } - - // limits the maximum number of messages per partition - protected def clamp( - leaderOffsets: Map[TopicAndPartition, LeaderOffset]): Map[TopicAndPartition, LeaderOffset] = { - maxMessagesPerPartition.map { mmp => - leaderOffsets.map { case (tp, lo) => - tp -> lo.copy(offset = Math.min(currentOffsets(tp) + mmp, lo.offset)) - } - }.getOrElse(leaderOffsets) - } - - override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = { - val untilOffsets = clamp(latestLeaderOffsets(maxRetries)) - val rdd = KafkaRDD[K, V, U, T, R]( - context.sparkContext, kafkaParams, currentOffsets, untilOffsets, messageHandler) - - // Report the record number and metadata of this batch interval to InputInfoTracker. - val offsetRanges = currentOffsets.map { case (tp, fo) => - val uo = untilOffsets(tp) - OffsetRange(tp.topic, tp.partition, fo, uo.offset) - } - val description = offsetRanges.filter { offsetRange => - // Don't display empty ranges. - offsetRange.fromOffset != offsetRange.untilOffset - }.map { offsetRange => - s"topic: ${offsetRange.topic}\tpartition: ${offsetRange.partition}\t" + - s"offsets: ${offsetRange.fromOffset} to ${offsetRange.untilOffset}" - }.mkString("\n") - // Copy offsetRanges to immutable.List to prevent from being modified by the user - val metadata = Map( - "offsets" -> offsetRanges.toList, - StreamInputInfo.METADATA_KEY_DESCRIPTION -> description) - val inputInfo = StreamInputInfo(id, rdd.count, metadata) - ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) - - currentOffsets = untilOffsets.map(kv => kv._1 -> kv._2.offset) - Some(rdd) - } - - override def start(): Unit = { - } - - def stop(): Unit = { - } - - private[streaming] - class DirectKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) { - def batchForTime: mutable.HashMap[Time, Array[(String, Int, Long, Long)]] = { - data.asInstanceOf[mutable.HashMap[Time, Array[OffsetRange.OffsetRangeTuple]]] - } - - override def update(time: Time) { - batchForTime.clear() - generatedRDDs.foreach { kv => - val a = kv._2.asInstanceOf[KafkaRDD[K, V, U, T, R]].offsetRanges.map(_.toTuple).toArray - batchForTime += kv._1 -> a - } - } - - override def cleanup(time: Time) { } - - override def restore() { - // this is assuming that the topics don't change during execution, which is true currently - val topics = fromOffsets.keySet - val leaders = KafkaCluster.checkErrors(kc.findLeaders(topics)) - - batchForTime.toSeq.sortBy(_._1)(Time.ordering).foreach { case (t, b) => - logInfo(s"Restoring KafkaRDD for time $t ${b.mkString("[", ", ", "]")}") - generatedRDDs += t -> new KafkaRDD[K, V, U, T, R]( - context.sparkContext, kafkaParams, b.map(OffsetRange(_)), leaders, messageHandler) - } - } - } - - /** - * A RateController to retrieve the rate from RateEstimator. - */ - private[streaming] class DirectKafkaRateController(id: Int, estimator: RateEstimator) - extends RateController(id, estimator) { - override def publish(rate: Long): Unit = () - } -} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala deleted file mode 100644 index ea5f842c6caf..000000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ /dev/null @@ -1,268 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import scala.collection.mutable.ArrayBuffer -import scala.reflect.{classTag, ClassTag} - -import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext} -import org.apache.spark.partial.{PartialResult, BoundedDouble} -import org.apache.spark.rdd.RDD -import org.apache.spark.util.NextIterator - -import kafka.api.{FetchRequestBuilder, FetchResponse} -import kafka.common.{ErrorMapping, TopicAndPartition} -import kafka.consumer.SimpleConsumer -import kafka.message.{MessageAndMetadata, MessageAndOffset} -import kafka.serializer.Decoder -import kafka.utils.VerifiableProperties - -/** - * A batch-oriented interface for consuming from Kafka. - * Starting and ending offsets are specified in advance, - * so that you can control exactly-once semantics. - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" to be set - * with Kafka broker(s) specified in host1:port1,host2:port2 form. - * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD - * @param messageHandler function for translating each message into the desired type - */ -private[kafka] -class KafkaRDD[ - K: ClassTag, - V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag, - R: ClassTag] private[spark] ( - sc: SparkContext, - kafkaParams: Map[String, String], - val offsetRanges: Array[OffsetRange], - leaders: Map[TopicAndPartition, (String, Int)], - messageHandler: MessageAndMetadata[K, V] => R - ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges { - override def getPartitions: Array[Partition] = { - offsetRanges.zipWithIndex.map { case (o, i) => - val (host, port) = leaders(TopicAndPartition(o.topic, o.partition)) - new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port) - }.toArray - } - - override def count(): Long = offsetRanges.map(_.count).sum - - override def countApprox( - timeout: Long, - confidence: Double = 0.95 - ): PartialResult[BoundedDouble] = { - val c = count - new PartialResult(new BoundedDouble(c, 1.0, c, c), true) - } - - override def isEmpty(): Boolean = count == 0L - - override def take(num: Int): Array[R] = { - val nonEmptyPartitions = this.partitions - .map(_.asInstanceOf[KafkaRDDPartition]) - .filter(_.count > 0) - - if (num < 1 || nonEmptyPartitions.size < 1) { - return new Array[R](0) - } - - // Determine in advance how many messages need to be taken from each partition - val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) => - val remain = num - result.values.sum - if (remain > 0) { - val taken = Math.min(remain, part.count) - result + (part.index -> taken.toInt) - } else { - result - } - } - - val buf = new ArrayBuffer[R] - val res = context.runJob( - this, - (tc: TaskContext, it: Iterator[R]) => it.take(parts(tc.partitionId)).toArray, - parts.keys.toArray) - res.foreach(buf ++= _) - buf.toArray - } - - override def getPreferredLocations(thePart: Partition): Seq[String] = { - val part = thePart.asInstanceOf[KafkaRDDPartition] - // TODO is additional hostname resolution necessary here - Seq(part.host) - } - - private def errBeginAfterEnd(part: KafkaRDDPartition): String = - s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " + - s"for topic ${part.topic} partition ${part.partition}. " + - "You either provided an invalid fromOffset, or the Kafka topic has been damaged" - - private def errRanOutBeforeEnd(part: KafkaRDDPartition): String = - s"Ran out of messages before reaching ending offset ${part.untilOffset} " + - s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." + - " This should not happen, and indicates that messages may have been lost" - - private def errOvershotEnd(itemOffset: Long, part: KafkaRDDPartition): String = - s"Got ${itemOffset} > ending offset ${part.untilOffset} " + - s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." + - " This should not happen, and indicates a message may have been skipped" - - override def compute(thePart: Partition, context: TaskContext): Iterator[R] = { - val part = thePart.asInstanceOf[KafkaRDDPartition] - assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part)) - if (part.fromOffset == part.untilOffset) { - log.info(s"Beginning offset ${part.fromOffset} is the same as ending offset " + - s"skipping ${part.topic} ${part.partition}") - Iterator.empty - } else { - new KafkaRDDIterator(part, context) - } - } - - private class KafkaRDDIterator( - part: KafkaRDDPartition, - context: TaskContext) extends NextIterator[R] { - - context.addTaskCompletionListener{ context => closeIfNeeded() } - - log.info(s"Computing topic ${part.topic}, partition ${part.partition} " + - s"offsets ${part.fromOffset} -> ${part.untilOffset}") - - val kc = new KafkaCluster(kafkaParams) - val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) - .newInstance(kc.config.props) - .asInstanceOf[Decoder[K]] - val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) - .newInstance(kc.config.props) - .asInstanceOf[Decoder[V]] - val consumer = connectLeader - var requestOffset = part.fromOffset - var iter: Iterator[MessageAndOffset] = null - - // The idea is to use the provided preferred host, except on task retry atttempts, - // to minimize number of kafka metadata requests - private def connectLeader: SimpleConsumer = { - if (context.attemptNumber > 0) { - kc.connectLeader(part.topic, part.partition).fold( - errs => throw new SparkException( - s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " + - errs.mkString("\n")), - consumer => consumer - ) - } else { - kc.connect(part.host, part.port) - } - } - - private def handleFetchErr(resp: FetchResponse) { - if (resp.hasError) { - val err = resp.errorCode(part.topic, part.partition) - if (err == ErrorMapping.LeaderNotAvailableCode || - err == ErrorMapping.NotLeaderForPartitionCode) { - log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " + - s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms") - Thread.sleep(kc.config.refreshLeaderBackoffMs) - } - // Let normal rdd retry sort out reconnect attempts - throw ErrorMapping.exceptionFor(err) - } - } - - private def fetchBatch: Iterator[MessageAndOffset] = { - val req = new FetchRequestBuilder() - .addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes) - .build() - val resp = consumer.fetch(req) - handleFetchErr(resp) - // kafka may return a batch that starts before the requested offset - resp.messageSet(part.topic, part.partition) - .iterator - .dropWhile(_.offset < requestOffset) - } - - override def close(): Unit = { - if (consumer != null) { - consumer.close() - } - } - - override def getNext(): R = { - if (iter == null || !iter.hasNext) { - iter = fetchBatch - } - if (!iter.hasNext) { - assert(requestOffset == part.untilOffset, errRanOutBeforeEnd(part)) - finished = true - null.asInstanceOf[R] - } else { - val item = iter.next() - if (item.offset >= part.untilOffset) { - assert(item.offset == part.untilOffset, errOvershotEnd(item.offset, part)) - finished = true - null.asInstanceOf[R] - } else { - requestOffset = item.nextOffset - messageHandler(new MessageAndMetadata( - part.topic, part.partition, item.message, item.offset, keyDecoder, valueDecoder)) - } - } - } - } -} - -private[kafka] -object KafkaRDD { - import KafkaCluster.LeaderOffset - - /** - * @param kafkaParams Kafka - * configuration parameters. - * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. - * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) - * starting point of the batch - * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive) - * ending point of the batch - * @param messageHandler function for translating each message into the desired type - */ - def apply[ - K: ClassTag, - V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag, - R: ClassTag]( - sc: SparkContext, - kafkaParams: Map[String, String], - fromOffsets: Map[TopicAndPartition, Long], - untilOffsets: Map[TopicAndPartition, LeaderOffset], - messageHandler: MessageAndMetadata[K, V] => R - ): KafkaRDD[K, V, U, T, R] = { - val leaders = untilOffsets.map { case (tp, lo) => - tp -> (lo.host, lo.port) - }.toMap - - val offsetRanges = fromOffsets.map { case (tp, fo) => - val uo = untilOffsets(tp) - OffsetRange(tp.topic, tp.partition, fo, uo.offset) - }.toArray - - new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaders, messageHandler) - } -} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala deleted file mode 100644 index a660d2a00c35..000000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import org.apache.spark.Partition - -/** @param topic kafka topic name - * @param partition kafka partition id - * @param fromOffset inclusive starting offset - * @param untilOffset exclusive ending offset - * @param host preferred kafka host, i.e. the leader at the time the rdd was created - * @param port preferred kafka host's port - */ -private[kafka] -class KafkaRDDPartition( - val index: Int, - val topic: String, - val partition: Int, - val fromOffset: Long, - val untilOffset: Long, - val host: String, - val port: Int -) extends Partition { - /** Number of messages this partition refers to */ - def count(): Long = untilOffset - fromOffset -} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala deleted file mode 100644 index c9fd715d3d55..000000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala +++ /dev/null @@ -1,271 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import java.io.File -import java.lang.{Integer => JInt} -import java.net.InetSocketAddress -import java.util.concurrent.TimeoutException -import java.util.{Map => JMap, Properties} - -import scala.annotation.tailrec -import scala.collection.JavaConverters._ -import scala.language.postfixOps -import scala.util.control.NonFatal - -import kafka.admin.AdminUtils -import kafka.api.Request -import kafka.producer.{KeyedMessage, Producer, ProducerConfig} -import kafka.serializer.StringEncoder -import kafka.server.{KafkaConfig, KafkaServer} -import kafka.utils.{ZKStringSerializer, ZkUtils} -import org.I0Itec.zkclient.ZkClient -import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} - -import org.apache.spark.streaming.Time -import org.apache.spark.util.Utils -import org.apache.spark.{Logging, SparkConf} - -/** - * This is a helper class for Kafka test suites. This has the functionality to set up - * and tear down local Kafka servers, and to push data using Kafka producers. - * - * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. - */ -private[kafka] class KafkaTestUtils extends Logging { - - // Zookeeper related configurations - private val zkHost = "localhost" - private var zkPort: Int = 0 - private val zkConnectionTimeout = 6000 - private val zkSessionTimeout = 6000 - - private var zookeeper: EmbeddedZookeeper = _ - - private var zkClient: ZkClient = _ - - // Kafka broker related configurations - private val brokerHost = "localhost" - private var brokerPort = 9092 - private var brokerConf: KafkaConfig = _ - - // Kafka broker server - private var server: KafkaServer = _ - - // Kafka producer - private var producer: Producer[String, String] = _ - - // Flag to test whether the system is correctly started - private var zkReady = false - private var brokerReady = false - - def zkAddress: String = { - assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") - s"$zkHost:$zkPort" - } - - def brokerAddress: String = { - assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") - s"$brokerHost:$brokerPort" - } - - def zookeeperClient: ZkClient = { - assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") - Option(zkClient).getOrElse( - throw new IllegalStateException("Zookeeper client is not yet initialized")) - } - - // Set up the Embedded Zookeeper server and get the proper Zookeeper port - private def setupEmbeddedZookeeper(): Unit = { - // Zookeeper server startup - zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") - // Get the actual zookeeper binding port - zkPort = zookeeper.actualPort - zkClient = new ZkClient(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, - ZKStringSerializer) - zkReady = true - } - - // Set up the Embedded Kafka server - private def setupEmbeddedKafkaServer(): Unit = { - assert(zkReady, "Zookeeper should be set up beforehand") - - // Kafka broker startup - Utils.startServiceOnPort(brokerPort, port => { - brokerPort = port - brokerConf = new KafkaConfig(brokerConfiguration) - server = new KafkaServer(brokerConf) - server.startup() - (server, port) - }, new SparkConf(), "KafkaBroker") - - brokerReady = true - } - - /** setup the whole embedded servers, including Zookeeper and Kafka brokers */ - def setup(): Unit = { - setupEmbeddedZookeeper() - setupEmbeddedKafkaServer() - } - - /** Teardown the whole servers, including Kafka broker and Zookeeper */ - def teardown(): Unit = { - brokerReady = false - zkReady = false - - if (producer != null) { - producer.close() - producer = null - } - - if (server != null) { - server.shutdown() - server = null - } - - brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) } - - if (zkClient != null) { - zkClient.close() - zkClient = null - } - - if (zookeeper != null) { - zookeeper.shutdown() - zookeeper = null - } - } - - /** Create a Kafka topic and wait until it propagated to the whole cluster */ - def createTopic(topic: String): Unit = { - AdminUtils.createTopic(zkClient, topic, 1, 1) - // wait until metadata is propagated - waitUntilMetadataIsPropagated(topic, 0) - } - - /** Java-friendly function for sending messages to the Kafka broker */ - def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { - sendMessages(topic, Map(messageToFreq.asScala.mapValues(_.intValue()).toSeq: _*)) - } - - /** Send the messages to the Kafka broker */ - def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = { - val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray - sendMessages(topic, messages) - } - - /** Send the array of messages to the Kafka broker */ - def sendMessages(topic: String, messages: Array[String]): Unit = { - producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) - producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) - producer.close() - producer = null - } - - private def brokerConfiguration: Properties = { - val props = new Properties() - props.put("broker.id", "0") - props.put("host.name", "localhost") - props.put("port", brokerPort.toString) - props.put("log.dir", Utils.createTempDir().getAbsolutePath) - props.put("zookeeper.connect", zkAddress) - props.put("log.flush.interval.messages", "1") - props.put("replica.socket.timeout.ms", "1500") - props - } - - private def producerConfiguration: Properties = { - val props = new Properties() - props.put("metadata.broker.list", brokerAddress) - props.put("serializer.class", classOf[StringEncoder].getName) - // wait for all in-sync replicas to ack sends - props.put("request.required.acks", "-1") - props - } - - // A simplified version of scalatest eventually, rewritten here to avoid adding extra test - // dependency - def eventually[T](timeout: Time, interval: Time)(func: => T): T = { - def makeAttempt(): Either[Throwable, T] = { - try { - Right(func) - } catch { - case e if NonFatal(e) => Left(e) - } - } - - val startTime = System.currentTimeMillis() - @tailrec - def tryAgain(attempt: Int): T = { - makeAttempt() match { - case Right(result) => result - case Left(e) => - val duration = System.currentTimeMillis() - startTime - if (duration < timeout.milliseconds) { - Thread.sleep(interval.milliseconds) - } else { - throw new TimeoutException(e.getMessage) - } - - tryAgain(attempt + 1) - } - } - - tryAgain(1) - } - - private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { - def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { - case Some(partitionState) => - val leaderAndInSyncReplicas = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr - - ZkUtils.getLeaderForPartition(zkClient, topic, partition).isDefined && - Request.isValidBrokerId(leaderAndInSyncReplicas.leader) && - leaderAndInSyncReplicas.isr.size >= 1 - - case _ => - false - } - eventually(Time(10000), Time(100)) { - assert(isPropagated, s"Partition [$topic, $partition] metadata not propagated after timeout") - } - } - - private class EmbeddedZookeeper(val zkConnect: String) { - val snapshotDir = Utils.createTempDir() - val logDir = Utils.createTempDir() - - val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) - val (ip, port) = { - val splits = zkConnect.split(":") - (splits(0), splits(1).toInt) - } - val factory = new NIOServerCnxnFactory() - factory.configure(new InetSocketAddress(ip, port), 16) - factory.startup(zookeeper) - - val actualPort = factory.getLocalPort - - def shutdown() { - factory.shutdown() - Utils.deleteRecursively(snapshotDir) - Utils.deleteRecursively(logDir) - } - } -} - diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala deleted file mode 100644 index 312822207753..000000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ /dev/null @@ -1,671 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import java.lang.{Integer => JInt, Long => JLong} -import java.util.{List => JList, Map => JMap, Set => JSet} - -import scala.collection.JavaConverters._ -import scala.reflect.ClassTag - -import kafka.common.TopicAndPartition -import kafka.message.MessageAndMetadata -import kafka.serializer.{Decoder, DefaultDecoder, StringDecoder} - -import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaInputDStream, JavaPairInputDStream, JavaPairReceiverInputDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.{InputDStream, ReceiverInputDStream} -import org.apache.spark.streaming.util.WriteAheadLogUtils -import org.apache.spark.{SparkContext, SparkException} - -object KafkaUtils { - /** - * Create an input stream that pulls messages from Kafka Brokers. - * @param ssc StreamingContext object - * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) - * @param groupId The group id for this consumer - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread - * @param storageLevel Storage level to use for storing the received objects - * (default: StorageLevel.MEMORY_AND_DISK_SER_2) - */ - def createStream( - ssc: StreamingContext, - zkQuorum: String, - groupId: String, - topics: Map[String, Int], - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): ReceiverInputDStream[(String, String)] = { - val kafkaParams = Map[String, String]( - "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, - "zookeeper.connection.timeout.ms" -> "10000") - createStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, topics, storageLevel) - } - - /** - * Create an input stream that pulls messages from Kafka Brokers. - * @param ssc StreamingContext object - * @param kafkaParams Map of kafka configuration parameters, - * see http://kafka.apache.org/08/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel Storage level to use for storing the received objects - */ - def createStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: ClassTag, T <: Decoder[_]: ClassTag]( - ssc: StreamingContext, - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel - ): ReceiverInputDStream[(K, V)] = { - val walEnabled = WriteAheadLogUtils.enableReceiverLog(ssc.conf) - new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, walEnabled, storageLevel) - } - - /** - * Create an input stream that pulls messages from Kafka Brokers. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param jssc JavaStreamingContext object - * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) - * @param groupId The group id for this consumer - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread - */ - def createStream( - jssc: JavaStreamingContext, - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt] - ): JavaPairReceiverInputDStream[String, String] = { - createStream(jssc.ssc, zkQuorum, groupId, Map(topics.asScala.mapValues(_.intValue()).toSeq: _*)) - } - - /** - * Create an input stream that pulls messages from Kafka Brokers. - * @param jssc JavaStreamingContext object - * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..). - * @param groupId The group id for this consumer. - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel RDD storage level. - */ - def createStream( - jssc: JavaStreamingContext, - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt], - storageLevel: StorageLevel - ): JavaPairReceiverInputDStream[String, String] = { - createStream(jssc.ssc, zkQuorum, groupId, Map(topics.asScala.mapValues(_.intValue()).toSeq: _*), - storageLevel) - } - - /** - * Create an input stream that pulls messages from Kafka Brokers. - * @param jssc JavaStreamingContext object - * @param keyTypeClass Key type of DStream - * @param valueTypeClass value type of Dstream - * @param keyDecoderClass Type of kafka key decoder - * @param valueDecoderClass Type of kafka value decoder - * @param kafkaParams Map of kafka configuration parameters, - * see http://kafka.apache.org/08/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread - * @param storageLevel RDD storage level. - */ - def createStream[K, V, U <: Decoder[_], T <: Decoder[_]]( - jssc: JavaStreamingContext, - keyTypeClass: Class[K], - valueTypeClass: Class[V], - keyDecoderClass: Class[U], - valueDecoderClass: Class[T], - kafkaParams: JMap[String, String], - topics: JMap[String, JInt], - storageLevel: StorageLevel - ): JavaPairReceiverInputDStream[K, V] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyTypeClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueTypeClass) - - implicit val keyCmd: ClassTag[U] = ClassTag(keyDecoderClass) - implicit val valueCmd: ClassTag[T] = ClassTag(valueDecoderClass) - - createStream[K, V, U, T]( - jssc.ssc, - kafkaParams.asScala.toMap, - Map(topics.asScala.mapValues(_.intValue()).toSeq: _*), - storageLevel) - } - - /** get leaders for the given offset ranges, or throw an exception */ - private def leadersForRanges( - kc: KafkaCluster, - offsetRanges: Array[OffsetRange]): Map[TopicAndPartition, (String, Int)] = { - val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet - val leaders = kc.findLeaders(topics) - KafkaCluster.checkErrors(leaders) - } - - /** Make sure offsets are available in kafka, or throw an exception */ - private def checkOffsets( - kc: KafkaCluster, - offsetRanges: Array[OffsetRange]): Unit = { - val topics = offsetRanges.map(_.topicAndPartition).toSet - val result = for { - low <- kc.getEarliestLeaderOffsets(topics).right - high <- kc.getLatestLeaderOffsets(topics).right - } yield { - offsetRanges.filterNot { o => - low(o.topicAndPartition).offset <= o.fromOffset && - o.untilOffset <= high(o.topicAndPartition).offset - } - } - val badRanges = KafkaCluster.checkErrors(result) - if (!badRanges.isEmpty) { - throw new SparkException("Offsets not available on leader: " + badRanges.mkString(",")) - } - } - - /** - * Create a RDD from Kafka using offset ranges for each topic and partition. - * - * @param sc SparkContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers) specified in - * host1:port1,host2:port2 form. - * @param offsetRanges Each OffsetRange in the batch corresponds to a - * range of offsets for a given Kafka topic/partition - */ - def createRDD[ - K: ClassTag, - V: ClassTag, - KD <: Decoder[K]: ClassTag, - VD <: Decoder[V]: ClassTag]( - sc: SparkContext, - kafkaParams: Map[String, String], - offsetRanges: Array[OffsetRange] - ): RDD[(K, V)] = sc.withScope { - val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) - val kc = new KafkaCluster(kafkaParams) - val leaders = leadersForRanges(kc, offsetRanges) - checkOffsets(kc, offsetRanges) - new KafkaRDD[K, V, KD, VD, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler) - } - - /** - * Create a RDD from Kafka using offset ranges for each topic and partition. This allows you - * specify the Kafka leader to connect to (to optimize fetching) and access the message as well - * as the metadata. - * - * @param sc SparkContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers) specified in - * host1:port1,host2:port2 form. - * @param offsetRanges Each OffsetRange in the batch corresponds to a - * range of offsets for a given Kafka topic/partition - * @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 - */ - def createRDD[ - K: ClassTag, - V: ClassTag, - KD <: Decoder[K]: ClassTag, - VD <: Decoder[V]: ClassTag, - R: ClassTag]( - sc: SparkContext, - kafkaParams: Map[String, String], - offsetRanges: Array[OffsetRange], - leaders: Map[TopicAndPartition, Broker], - messageHandler: MessageAndMetadata[K, V] => R - ): RDD[R] = sc.withScope { - val kc = new KafkaCluster(kafkaParams) - val leaderMap = if (leaders.isEmpty) { - leadersForRanges(kc, 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 - } - val cleanedHandler = sc.clean(messageHandler) - checkOffsets(kc, offsetRanges) - new KafkaRDD[K, V, KD, VD, R](sc, kafkaParams, offsetRanges, leaderMap, cleanedHandler) - } - - /** - * Create a RDD from Kafka using offset ranges for each topic and partition. - * - * @param jsc JavaSparkContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers) specified in - * host1:port1,host2:port2 form. - * @param offsetRanges Each OffsetRange in the batch corresponds to a - * range of offsets for a given Kafka topic/partition - */ - def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V]]( - jsc: JavaSparkContext, - keyClass: Class[K], - valueClass: Class[V], - keyDecoderClass: Class[KD], - valueDecoderClass: Class[VD], - kafkaParams: JMap[String, String], - offsetRanges: Array[OffsetRange] - ): JavaPairRDD[K, V] = jsc.sc.withScope { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) - implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) - new JavaPairRDD(createRDD[K, V, KD, VD]( - jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges)) - } - - /** - * Create a RDD from Kafka using offset ranges for each topic and partition. This allows you - * specify the Kafka leader to connect to (to optimize fetching) and access the message as well - * as the metadata. - * - * @param jsc JavaSparkContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers) specified in - * host1:port1,host2:port2 form. - * @param offsetRanges Each OffsetRange in the batch corresponds to a - * range of offsets for a given Kafka topic/partition - * @param leaders Kafka 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 - */ - def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( - jsc: JavaSparkContext, - keyClass: Class[K], - valueClass: Class[V], - keyDecoderClass: Class[KD], - valueDecoderClass: Class[VD], - recordClass: Class[R], - kafkaParams: JMap[String, String], - offsetRanges: Array[OffsetRange], - leaders: JMap[TopicAndPartition, Broker], - messageHandler: JFunction[MessageAndMetadata[K, V], R] - ): JavaRDD[R] = jsc.sc.withScope { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) - implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) - implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) - val leaderMap = Map(leaders.asScala.toSeq: _*) - createRDD[K, V, KD, VD, R]( - jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges, leaderMap, messageHandler.call(_)) - } - - /** - * Create an input stream that directly pulls messages from Kafka Brokers - * without using any receiver. This stream can guarantee that each message - * from Kafka is included in transformations exactly once (see points below). - * - * Points to note: - * - No receivers: This stream does not use any receiver. It directly queries Kafka - * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked - * by the stream itself. For interoperability with Kafka monitoring tools that depend on - * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. - * You can access the offsets used in each batch from the generated RDDs (see - * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). - * - Failure Recovery: To recover from driver failures, you have to enable checkpointing - * in the [[StreamingContext]]. The information on consumed offset can be - * recovered from the checkpoint. See the programming guide for details (constraints, etc.). - * - End-to-end semantics: This stream ensures that every records is effectively received and - * transformed exactly once, but gives no guarantees on whether the transformed data are - * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure - * that the output operation is idempotent, or use transactions to output records atomically. - * See the programming guide for more details. - * - * @param ssc StreamingContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers) specified in - * host1:port1,host2:port2 form. - * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) - * starting point of the stream - * @param messageHandler Function for translating each message and metadata into the desired type - */ - def createDirectStream[ - K: ClassTag, - V: ClassTag, - KD <: Decoder[K]: ClassTag, - VD <: Decoder[V]: ClassTag, - R: ClassTag] ( - ssc: StreamingContext, - kafkaParams: Map[String, String], - fromOffsets: Map[TopicAndPartition, Long], - messageHandler: MessageAndMetadata[K, V] => R - ): InputDStream[R] = { - val cleanedHandler = ssc.sc.clean(messageHandler) - new DirectKafkaInputDStream[K, V, KD, VD, R]( - ssc, kafkaParams, fromOffsets, cleanedHandler) - } - - /** - * Create an input stream that directly pulls messages from Kafka Brokers - * without using any receiver. This stream can guarantee that each message - * from Kafka is included in transformations exactly once (see points below). - * - * Points to note: - * - No receivers: This stream does not use any receiver. It directly queries Kafka - * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked - * by the stream itself. For interoperability with Kafka monitoring tools that depend on - * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. - * You can access the offsets used in each batch from the generated RDDs (see - * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). - * - Failure Recovery: To recover from driver failures, you have to enable checkpointing - * in the [[StreamingContext]]. The information on consumed offset can be - * recovered from the checkpoint. See the programming guide for details (constraints, etc.). - * - End-to-end semantics: This stream ensures that every records is effectively received and - * transformed exactly once, but gives no guarantees on whether the transformed data are - * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure - * that the output operation is idempotent, or use transactions to output records atomically. - * See the programming guide for more details. - * - * @param ssc StreamingContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers), specified in - * host1:port1,host2:port2 form. - * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" - * to determine where the stream starts (defaults to "largest") - * @param topics Names of the topics to consume - */ - def createDirectStream[ - K: ClassTag, - V: ClassTag, - KD <: Decoder[K]: ClassTag, - VD <: Decoder[V]: ClassTag] ( - ssc: StreamingContext, - kafkaParams: Map[String, String], - topics: Set[String] - ): InputDStream[(K, V)] = { - val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) - val kc = new KafkaCluster(kafkaParams) - val reset = kafkaParams.get("auto.offset.reset").map(_.toLowerCase) - - val result = for { - topicPartitions <- kc.getPartitions(topics).right - leaderOffsets <- (if (reset == Some("smallest")) { - kc.getEarliestLeaderOffsets(topicPartitions) - } else { - kc.getLatestLeaderOffsets(topicPartitions) - }).right - } yield { - val fromOffsets = leaderOffsets.map { case (tp, lo) => - (tp, lo.offset) - } - new DirectKafkaInputDStream[K, V, KD, VD, (K, V)]( - ssc, kafkaParams, fromOffsets, messageHandler) - } - KafkaCluster.checkErrors(result) - } - - /** - * Create an input stream that directly pulls messages from Kafka Brokers - * without using any receiver. This stream can guarantee that each message - * from Kafka is included in transformations exactly once (see points below). - * - * Points to note: - * - No receivers: This stream does not use any receiver. It directly queries Kafka - * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked - * by the stream itself. For interoperability with Kafka monitoring tools that depend on - * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. - * You can access the offsets used in each batch from the generated RDDs (see - * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). - * - Failure Recovery: To recover from driver failures, you have to enable checkpointing - * in the [[StreamingContext]]. The information on consumed offset can be - * recovered from the checkpoint. See the programming guide for details (constraints, etc.). - * - End-to-end semantics: This stream ensures that every records is effectively received and - * transformed exactly once, but gives no guarantees on whether the transformed data are - * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure - * that the output operation is idempotent, or use transactions to output records atomically. - * See the programming guide for more details. - * - * @param jssc JavaStreamingContext object - * @param keyClass Class of the keys in the Kafka records - * @param valueClass Class of the values in the Kafka records - * @param keyDecoderClass Class of the key decoder - * @param valueDecoderClass Class of the value decoder - * @param recordClass Class of the records in DStream - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers), specified in - * host1:port1,host2:port2 form. - * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) - * starting point of the stream - * @param messageHandler Function for translating each message and metadata into the desired type - */ - def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( - jssc: JavaStreamingContext, - keyClass: Class[K], - valueClass: Class[V], - keyDecoderClass: Class[KD], - valueDecoderClass: Class[VD], - recordClass: Class[R], - kafkaParams: JMap[String, String], - fromOffsets: JMap[TopicAndPartition, JLong], - messageHandler: JFunction[MessageAndMetadata[K, V], R] - ): JavaInputDStream[R] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) - implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) - implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) - val cleanedHandler = jssc.sparkContext.clean(messageHandler.call _) - createDirectStream[K, V, KD, VD, R]( - jssc.ssc, - Map(kafkaParams.asScala.toSeq: _*), - Map(fromOffsets.asScala.mapValues(_.longValue()).toSeq: _*), - cleanedHandler - ) - } - - /** - * Create an input stream that directly pulls messages from Kafka Brokers - * without using any receiver. This stream can guarantee that each message - * from Kafka is included in transformations exactly once (see points below). - * - * Points to note: - * - No receivers: This stream does not use any receiver. It directly queries Kafka - * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked - * by the stream itself. For interoperability with Kafka monitoring tools that depend on - * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. - * You can access the offsets used in each batch from the generated RDDs (see - * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). - * - Failure Recovery: To recover from driver failures, you have to enable checkpointing - * in the [[StreamingContext]]. The information on consumed offset can be - * recovered from the checkpoint. See the programming guide for details (constraints, etc.). - * - End-to-end semantics: This stream ensures that every records is effectively received and - * transformed exactly once, but gives no guarantees on whether the transformed data are - * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure - * that the output operation is idempotent, or use transactions to output records atomically. - * See the programming guide for more details. - * - * @param jssc JavaStreamingContext object - * @param keyClass Class of the keys in the Kafka records - * @param valueClass Class of the values in the Kafka records - * @param keyDecoderClass Class of the key decoder - * @param valueDecoderClass Class type of the value decoder - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers), specified in - * host1:port1,host2:port2 form. - * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" - * to determine where the stream starts (defaults to "largest") - * @param topics Names of the topics to consume - */ - def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V]]( - jssc: JavaStreamingContext, - keyClass: Class[K], - valueClass: Class[V], - keyDecoderClass: Class[KD], - valueDecoderClass: Class[VD], - kafkaParams: JMap[String, String], - topics: JSet[String] - ): JavaPairInputDStream[K, V] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) - implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) - createDirectStream[K, V, KD, VD]( - jssc.ssc, - Map(kafkaParams.asScala.toSeq: _*), - Set(topics.asScala.toSeq: _*) - ) - } -} - -/** - * This is a helper class that wraps the KafkaUtils.createStream() into more - * Python-friendly class and function so that it can be easily - * instantiated and called from Python's KafkaUtils (see SPARK-6027). - * - * The zero-arg constructor helps instantiate this class from the Class object - * classOf[KafkaUtilsPythonHelper].newInstance(), and the createStream() - * takes care of known parameters instead of passing them from Python - */ -private[kafka] class KafkaUtilsPythonHelper { - def createStream( - jssc: JavaStreamingContext, - kafkaParams: JMap[String, String], - topics: JMap[String, JInt], - storageLevel: StorageLevel): JavaPairReceiverInputDStream[Array[Byte], Array[Byte]] = { - KafkaUtils.createStream[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder]( - jssc, - classOf[Array[Byte]], - classOf[Array[Byte]], - classOf[DefaultDecoder], - classOf[DefaultDecoder], - kafkaParams, - topics, - storageLevel) - } - - def createRDD( - jsc: JavaSparkContext, - kafkaParams: JMap[String, String], - offsetRanges: JList[OffsetRange], - leaders: JMap[TopicAndPartition, Broker]): JavaPairRDD[Array[Byte], Array[Byte]] = { - val messageHandler = new JFunction[MessageAndMetadata[Array[Byte], Array[Byte]], - (Array[Byte], Array[Byte])] { - def call(t1: MessageAndMetadata[Array[Byte], Array[Byte]]): (Array[Byte], Array[Byte]) = - (t1.key(), t1.message()) - } - - val jrdd = KafkaUtils.createRDD[ - Array[Byte], - Array[Byte], - DefaultDecoder, - DefaultDecoder, - (Array[Byte], Array[Byte])]( - jsc, - classOf[Array[Byte]], - classOf[Array[Byte]], - classOf[DefaultDecoder], - classOf[DefaultDecoder], - classOf[(Array[Byte], Array[Byte])], - kafkaParams, - offsetRanges.toArray(new Array[OffsetRange](offsetRanges.size())), - leaders, - messageHandler - ) - new JavaPairRDD(jrdd.rdd) - } - - def createDirectStream( - jssc: JavaStreamingContext, - kafkaParams: JMap[String, String], - topics: JSet[String], - fromOffsets: JMap[TopicAndPartition, JLong] - ): JavaPairInputDStream[Array[Byte], Array[Byte]] = { - - if (!fromOffsets.isEmpty) { - val topicsFromOffsets = fromOffsets.keySet().asScala.map(_.topic) - if (topicsFromOffsets != topics.asScala.toSet) { - throw new IllegalStateException( - s"The specified topics: ${topics.asScala.toSet.mkString(" ")} " + - s"do not equal to the topic from offsets: ${topicsFromOffsets.mkString(" ")}") - } - } - - if (fromOffsets.isEmpty) { - KafkaUtils.createDirectStream[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder]( - jssc, - classOf[Array[Byte]], - classOf[Array[Byte]], - classOf[DefaultDecoder], - classOf[DefaultDecoder], - kafkaParams, - topics) - } else { - val messageHandler = new JFunction[MessageAndMetadata[Array[Byte], Array[Byte]], - (Array[Byte], Array[Byte])] { - def call(t1: MessageAndMetadata[Array[Byte], Array[Byte]]): (Array[Byte], Array[Byte]) = - (t1.key(), t1.message()) - } - - val jstream = KafkaUtils.createDirectStream[ - Array[Byte], - Array[Byte], - DefaultDecoder, - DefaultDecoder, - (Array[Byte], Array[Byte])]( - jssc, - classOf[Array[Byte]], - classOf[Array[Byte]], - classOf[DefaultDecoder], - classOf[DefaultDecoder], - classOf[(Array[Byte], Array[Byte])], - kafkaParams, - fromOffsets, - messageHandler) - new JavaPairInputDStream(jstream.inputDStream) - } - } - - def createOffsetRange(topic: String, partition: JInt, fromOffset: JLong, untilOffset: JLong - ): OffsetRange = OffsetRange.create(topic, partition, fromOffset, untilOffset) - - def createTopicAndPartition(topic: String, partition: JInt): TopicAndPartition = - TopicAndPartition(topic, partition) - - def createBroker(host: String, port: JInt): Broker = Broker(host, port) - - def offsetRangesOfKafkaRDD(rdd: RDD[_]): JList[OffsetRange] = { - val parentRDDs = rdd.getNarrowAncestors - val kafkaRDDs = parentRDDs.filter(rdd => rdd.isInstanceOf[KafkaRDD[_, _, _, _, _]]) - - require( - kafkaRDDs.length == 1, - "Cannot get offset ranges, as there may be multiple Kafka RDDs or no Kafka RDD associated" + - "with this RDD, please call this method only on a Kafka RDD.") - - val kafkaRDD = kafkaRDDs.head.asInstanceOf[KafkaRDD[_, _, _, _, _]] - kafkaRDD.offsetRanges.toSeq.asJava - } -} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala deleted file mode 100644 index 8a5f37149451..000000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import kafka.common.TopicAndPartition - -/** - * Represents any object that has a collection of [[OffsetRange]]s. This can be used access the - * offset ranges in RDDs generated by the direct Kafka DStream (see - * [[KafkaUtils.createDirectStream()]]). - * {{{ - * KafkaUtils.createDirectStream(...).foreachRDD { rdd => - * val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges - * ... - * } - * }}} - */ -trait HasOffsetRanges { - def offsetRanges: Array[OffsetRange] -} - -/** - * Represents a range of offsets from a single Kafka TopicAndPartition. Instances of this class - * can be created with `OffsetRange.create()`. - * @param topic Kafka topic name - * @param partition Kafka partition id - * @param fromOffset Inclusive starting offset - * @param untilOffset Exclusive ending offset - */ -final class OffsetRange private( - val topic: String, - val partition: Int, - val fromOffset: Long, - val untilOffset: Long) extends Serializable { - import OffsetRange.OffsetRangeTuple - - /** Kafka TopicAndPartition object, for convenience */ - def topicAndPartition(): TopicAndPartition = TopicAndPartition(topic, partition) - - /** Number of messages this OffsetRange refers to */ - def count(): Long = untilOffset - fromOffset - - override def equals(obj: Any): Boolean = obj match { - case that: OffsetRange => - this.topic == that.topic && - this.partition == that.partition && - this.fromOffset == that.fromOffset && - this.untilOffset == that.untilOffset - case _ => false - } - - override def hashCode(): Int = { - toTuple.hashCode() - } - - override def toString(): String = { - s"OffsetRange(topic: '$topic', partition: $partition, range: [$fromOffset -> $untilOffset])" - } - - /** this is to avoid ClassNotFoundException during checkpoint restore */ - private[streaming] - def toTuple: OffsetRangeTuple = (topic, partition, fromOffset, untilOffset) -} - -/** - * Companion object the provides methods to create instances of [[OffsetRange]]. - */ -object OffsetRange { - def create(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = - new OffsetRange(topic, partition, fromOffset, untilOffset) - - def create( - topicAndPartition: TopicAndPartition, - fromOffset: Long, - untilOffset: Long): OffsetRange = - new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) - - def apply(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = - new OffsetRange(topic, partition, fromOffset, untilOffset) - - def apply( - topicAndPartition: TopicAndPartition, - fromOffset: Long, - untilOffset: Long): OffsetRange = - new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) - - /** this is to avoid ClassNotFoundException during checkpoint restore */ - private[kafka] - type OffsetRangeTuple = (String, Int, Long, Long) - - private[kafka] - def apply(t: OffsetRangeTuple) = - new OffsetRange(t._1, t._2, t._3, t._4) -} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package-info.java b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package-info.java deleted file mode 100644 index 947bae115a62..000000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package-info.java +++ /dev/null @@ -1,21 +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. - */ - -/** - * Kafka receiver for spark streaming. - */ -package org.apache.spark.streaming.kafka; \ No newline at end of file diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java deleted file mode 100644 index fbdfbf7e509b..000000000000 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ /dev/null @@ -1,176 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka; - -import java.io.Serializable; -import java.util.*; -import java.util.concurrent.atomic.AtomicReference; - -import scala.Tuple2; - -import kafka.common.TopicAndPartition; -import kafka.message.MessageAndMetadata; -import kafka.serializer.StringDecoder; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.streaming.Durations; -import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.JavaStreamingContext; - -public class JavaDirectKafkaStreamSuite implements Serializable { - private transient JavaStreamingContext ssc = null; - private transient KafkaTestUtils kafkaTestUtils = null; - - @Before - public void setUp() { - kafkaTestUtils = new KafkaTestUtils(); - kafkaTestUtils.setup(); - SparkConf sparkConf = new SparkConf() - .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); - ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200)); - } - - @After - public void tearDown() { - if (ssc != null) { - ssc.stop(); - ssc = null; - } - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown(); - kafkaTestUtils = null; - } - } - - @Test - public void testKafkaStream() throws InterruptedException { - final String topic1 = "topic1"; - final String topic2 = "topic2"; - // hold a reference to the current offset ranges, so it can be used downstream - final AtomicReference offsetRanges = new AtomicReference<>(); - - String[] topic1data = createTopicAndSendData(topic1); - String[] topic2data = createTopicAndSendData(topic2); - - Set sent = new HashSet<>(); - sent.addAll(Arrays.asList(topic1data)); - sent.addAll(Arrays.asList(topic2data)); - - Map kafkaParams = new HashMap<>(); - kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); - kafkaParams.put("auto.offset.reset", "smallest"); - - JavaDStream stream1 = KafkaUtils.createDirectStream( - ssc, - String.class, - String.class, - StringDecoder.class, - StringDecoder.class, - kafkaParams, - topicToSet(topic1) - ).transformToPair( - // Make sure you can get offset ranges from the rdd - new Function, JavaPairRDD>() { - @Override - public JavaPairRDD call(JavaPairRDD rdd) { - OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); - offsetRanges.set(offsets); - Assert.assertEquals(topic1, offsets[0].topic()); - return rdd; - } - } - ).map( - new Function, String>() { - @Override - public String call(Tuple2 kv) { - return kv._2(); - } - } - ); - - JavaDStream stream2 = KafkaUtils.createDirectStream( - ssc, - String.class, - String.class, - StringDecoder.class, - StringDecoder.class, - String.class, - kafkaParams, - topicOffsetToMap(topic2, 0L), - new Function, String>() { - @Override - public String call(MessageAndMetadata msgAndMd) { - return msgAndMd.message(); - } - } - ); - JavaDStream unifiedStream = stream1.union(stream2); - - final Set result = Collections.synchronizedSet(new HashSet()); - unifiedStream.foreachRDD( - new Function, Void>() { - @Override - public Void call(JavaRDD rdd) { - result.addAll(rdd.collect()); - for (OffsetRange o : offsetRanges.get()) { - System.out.println( - o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset() - ); - } - return null; - } - } - ); - ssc.start(); - long startTime = System.currentTimeMillis(); - boolean matches = false; - while (!matches && System.currentTimeMillis() - startTime < 20000) { - matches = sent.size() == result.size(); - Thread.sleep(50); - } - Assert.assertEquals(sent, result); - ssc.stop(); - } - - private static Set topicToSet(String topic) { - Set topicSet = new HashSet<>(); - topicSet.add(topic); - return topicSet; - } - - private static Map topicOffsetToMap(String topic, Long offsetToStart) { - Map topicMap = new HashMap<>(); - topicMap.put(new TopicAndPartition(topic, 0), offsetToStart); - return topicMap; - } - - private String[] createTopicAndSendData(String topic) { - String[] data = { topic + "-1", topic + "-2", topic + "-3"}; - kafkaTestUtils.createTopic(topic); - kafkaTestUtils.sendMessages(topic, data); - return data; - } -} diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java deleted file mode 100644 index afcc6cfccd39..000000000000 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.Map; - -import scala.Tuple2; - -import kafka.common.TopicAndPartition; -import kafka.message.MessageAndMetadata; -import kafka.serializer.StringDecoder; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; - -public class JavaKafkaRDDSuite implements Serializable { - private transient JavaSparkContext sc = null; - private transient KafkaTestUtils kafkaTestUtils = null; - - @Before - public void setUp() { - kafkaTestUtils = new KafkaTestUtils(); - kafkaTestUtils.setup(); - SparkConf sparkConf = new SparkConf() - .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); - sc = new JavaSparkContext(sparkConf); - } - - @After - public void tearDown() { - if (sc != null) { - sc.stop(); - sc = null; - } - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown(); - kafkaTestUtils = null; - } - } - - @Test - public void testKafkaRDD() throws InterruptedException { - String topic1 = "topic1"; - String topic2 = "topic2"; - - createTopicAndSendData(topic1); - createTopicAndSendData(topic2); - - Map kafkaParams = new HashMap<>(); - kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); - - OffsetRange[] offsetRanges = { - OffsetRange.create(topic1, 0, 0, 1), - OffsetRange.create(topic2, 0, 0, 1) - }; - - Map emptyLeaders = new HashMap<>(); - Map leaders = new HashMap<>(); - String[] hostAndPort = kafkaTestUtils.brokerAddress().split(":"); - Broker broker = Broker.create(hostAndPort[0], Integer.parseInt(hostAndPort[1])); - leaders.put(new TopicAndPartition(topic1, 0), broker); - leaders.put(new TopicAndPartition(topic2, 0), broker); - - JavaRDD rdd1 = KafkaUtils.createRDD( - sc, - String.class, - String.class, - StringDecoder.class, - StringDecoder.class, - kafkaParams, - offsetRanges - ).map( - new Function, String>() { - @Override - public String call(Tuple2 kv) { - 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) { - 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) { - 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"}; - kafkaTestUtils.createTopic(topic); - kafkaTestUtils.sendMessages(topic, data); - return data; - } -} 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 deleted file mode 100644 index 02225d5aa7cc..000000000000 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ /dev/null @@ -1,472 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import java.io.File -import java.util.concurrent.atomic.AtomicLong - -import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset -import org.apache.spark.streaming.scheduler.rate.RateEstimator - -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 - -import org.apache.spark.{Logging, SparkConf, SparkContext, SparkFunSuite} -import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} -import org.apache.spark.streaming.dstream.DStream -import org.apache.spark.streaming.scheduler._ -import org.apache.spark.util.Utils - -class DirectKafkaStreamSuite - extends SparkFunSuite - with BeforeAndAfter - with BeforeAndAfterAll - with Eventually - with Logging { - val sparkConf = new SparkConf() - .setMaster("local[4]") - .setAppName(this.getClass.getSimpleName) - - private var sc: SparkContext = _ - private var ssc: StreamingContext = _ - private var testDir: File = _ - - private var kafkaTestUtils: KafkaTestUtils = _ - - override def beforeAll { - kafkaTestUtils = new KafkaTestUtils - kafkaTestUtils.setup() - } - - override def afterAll { - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null - } - } - - after { - if (ssc != null) { - ssc.stop() - sc = null - } - if (sc != null) { - sc.stop() - } - if (testDir != null) { - Utils.deleteRecursively(testDir) - } - } - - - test("basic stream receiving with multiple topics and smallest starting offset") { - val topics = Set("basic1", "basic2", "basic3") - val data = Map("a" -> 7, "b" -> 9) - topics.foreach { t => - kafkaTestUtils.createTopic(t) - kafkaTestUtils.sendMessages(t, data) - } - val totalSent = data.values.sum * topics.size - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "smallest" - ) - - ssc = new StreamingContext(sparkConf, Milliseconds(200)) - val stream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, topics) - } - - val allReceived = - new ArrayBuffer[(String, String)] with mutable.SynchronizedBuffer[(String, String)] - - // hold a reference to the current offset ranges, so it can be used downstream - var offsetRanges = Array[OffsetRange]() - - stream.transform { rdd => - // Get the offset ranges in the RDD - offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges - rdd - }.foreachRDD { rdd => - for (o <- offsetRanges) { - logInfo(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") - } - val collected = rdd.mapPartitionsWithIndex { (i, iter) => - // For each partition, get size of the range in the partition, - // and the number of items in the partition - val off = offsetRanges(i) - val all = iter.toSeq - val partSize = all.size - val rangeSize = off.untilOffset - off.fromOffset - Iterator((partSize, rangeSize)) - }.collect - - // Verify whether number of elements in each partition - // matches with the corresponding offset range - collected.foreach { case (partSize, rangeSize) => - assert(partSize === rangeSize, "offset ranges are wrong") - } - } - stream.foreachRDD { rdd => allReceived ++= rdd.collect() } - ssc.start() - eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { - assert(allReceived.size === totalSent, - "didn't get expected number of messages, messages:\n" + allReceived.mkString("\n")) - } - ssc.stop() - } - - test("receiving from largest starting offset") { - val topic = "largest" - val topicPartition = TopicAndPartition(topic, 0) - val data = Map("a" -> 10) - kafkaTestUtils.createTopic(topic) - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "largest" - ) - val kc = new KafkaCluster(kafkaParams) - def getLatestOffset(): Long = { - kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset - } - - // Send some initial messages before starting context - kafkaTestUtils.sendMessages(topic, data) - eventually(timeout(10 seconds), interval(20 milliseconds)) { - assert(getLatestOffset() > 3) - } - val offsetBeforeStart = getLatestOffset() - - // Setup context and kafka stream with largest offset - ssc = new StreamingContext(sparkConf, Milliseconds(200)) - val stream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, Set(topic)) - } - assert( - stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]] - .fromOffsets(topicPartition) >= offsetBeforeStart, - "Start offset not from latest" - ) - - val collectedData = new mutable.ArrayBuffer[String]() with mutable.SynchronizedBuffer[String] - stream.map { _._2 }.foreachRDD { rdd => collectedData ++= rdd.collect() } - ssc.start() - val newData = Map("b" -> 10) - kafkaTestUtils.sendMessages(topic, newData) - eventually(timeout(10 seconds), interval(50 milliseconds)) { - collectedData.contains("b") - } - assert(!collectedData.contains("a")) - } - - - test("creating stream by offset") { - val topic = "offset" - val topicPartition = TopicAndPartition(topic, 0) - val data = Map("a" -> 10) - kafkaTestUtils.createTopic(topic) - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "largest" - ) - val kc = new KafkaCluster(kafkaParams) - def getLatestOffset(): Long = { - kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset - } - - // Send some initial messages before starting context - kafkaTestUtils.sendMessages(topic, data) - eventually(timeout(10 seconds), interval(20 milliseconds)) { - assert(getLatestOffset() >= 10) - } - val offsetBeforeStart = getLatestOffset() - - // Setup context and kafka stream with largest offset - ssc = new StreamingContext(sparkConf, Milliseconds(200)) - val stream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder, String]( - ssc, kafkaParams, Map(topicPartition -> 11L), - (m: MessageAndMetadata[String, String]) => m.message()) - } - assert( - stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]] - .fromOffsets(topicPartition) >= offsetBeforeStart, - "Start offset not from latest" - ) - - val collectedData = new mutable.ArrayBuffer[String]() with mutable.SynchronizedBuffer[String] - stream.foreachRDD { rdd => collectedData ++= rdd.collect() } - ssc.start() - val newData = Map("b" -> 10) - kafkaTestUtils.sendMessages(topic, newData) - eventually(timeout(10 seconds), interval(50 milliseconds)) { - collectedData.contains("b") - } - assert(!collectedData.contains("a")) - } - - // Test to verify the offset ranges can be recovered from the checkpoints - test("offset recovery") { - val topic = "recovery" - kafkaTestUtils.createTopic(topic) - testDir = Utils.createTempDir() - - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "smallest" - ) - - // Send data to Kafka and wait for it to be received - def sendDataAndWaitForReceive(data: Seq[Int]) { - val strings = data.map { _.toString} - kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) - eventually(timeout(10 seconds), interval(50 milliseconds)) { - assert(strings.forall { DirectKafkaStreamSuite.collectedData.contains }) - } - } - - // Setup the streaming context - ssc = new StreamingContext(sparkConf, Milliseconds(100)) - val kafkaStream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, Set(topic)) - } - val keyedStream = kafkaStream.map { v => "key" -> v._2.toInt } - val stateStream = keyedStream.updateStateByKey { (values: Seq[Int], state: Option[Int]) => - Some(values.sum + state.getOrElse(0)) - } - ssc.checkpoint(testDir.getAbsolutePath) - - // This is to collect the raw data received from Kafka - kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) => - val data = rdd.map { _._2 }.collect() - DirectKafkaStreamSuite.collectedData.appendAll(data) - } - - // This is ensure all the data is eventually receiving only once - stateStream.foreachRDD { (rdd: RDD[(String, Int)]) => - rdd.collect().headOption.foreach { x => DirectKafkaStreamSuite.total = x._2 } - } - ssc.start() - - // Send some data and wait for them to be received - for (i <- (1 to 10).grouped(4)) { - sendDataAndWaitForReceive(i) - } - - // Verify that offset ranges were generated - val offsetRangesBeforeStop = getOffsetRanges(kafkaStream) - assert(offsetRangesBeforeStop.size >= 1, "No offset ranges generated") - assert( - offsetRangesBeforeStop.head._2.forall { _.fromOffset === 0 }, - "starting offset not zero" - ) - ssc.stop() - logInfo("====== RESTARTING ========") - - // Recover context from checkpoints - ssc = new StreamingContext(testDir.getAbsolutePath) - val recoveredStream = ssc.graph.getInputStreams().head.asInstanceOf[DStream[(String, String)]] - - // Verify offset ranges have been recovered - val recoveredOffsetRanges = getOffsetRanges(recoveredStream) - assert(recoveredOffsetRanges.size > 0, "No offset ranges recovered") - val earlierOffsetRangesAsSets = offsetRangesBeforeStop.map { x => (x._1, x._2.toSet) } - assert( - recoveredOffsetRanges.forall { or => - earlierOffsetRangesAsSets.contains((or._1, or._2.toSet)) - }, - "Recovered ranges are not the same as the ones generated" - ) - // Restart context, give more data and verify the total at the end - // If the total is write that means each records has been received only once - ssc.start() - sendDataAndWaitForReceive(11 to 20) - eventually(timeout(10 seconds), interval(50 milliseconds)) { - assert(DirectKafkaStreamSuite.total === (1 to 20).sum) - } - ssc.stop() - } - - test("Direct Kafka stream report input information") { - val topic = "report-test" - val data = Map("a" -> 7, "b" -> 9) - kafkaTestUtils.createTopic(topic) - kafkaTestUtils.sendMessages(topic, data) - - val totalSent = data.values.sum - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "smallest" - ) - - import DirectKafkaStreamSuite._ - ssc = new StreamingContext(sparkConf, Milliseconds(200)) - val collector = new InputInfoCollector - ssc.addStreamingListener(collector) - - val stream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, Set(topic)) - } - - val allReceived = - new ArrayBuffer[(String, String)] with mutable.SynchronizedBuffer[(String, String)] - - stream.foreachRDD { rdd => allReceived ++= rdd.collect() } - ssc.start() - eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { - assert(allReceived.size === totalSent, - "didn't get expected number of messages, messages:\n" + allReceived.mkString("\n")) - - // Calculate all the record number collected in the StreamingListener. - assert(collector.numRecordsSubmitted.get() === totalSent) - assert(collector.numRecordsStarted.get() === totalSent) - assert(collector.numRecordsCompleted.get() === totalSent) - } - ssc.stop() - } - - test("using rate controller") { - val topic = "backpressure" - val topicPartition = TopicAndPartition(topic, 0) - kafkaTestUtils.createTopic(topic) - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "smallest" - ) - - val batchIntervalMilliseconds = 100 - val estimator = new ConstantEstimator(100) - val messageKeys = (1 to 200).map(_.toString) - val messages = messageKeys.map((_, 1)).toMap - - val sparkConf = new SparkConf() - // Safe, even with streaming, because we're using the direct API. - // Using 1 core is useful to make the test more predictable. - .setMaster("local[1]") - .setAppName(this.getClass.getSimpleName) - .set("spark.streaming.kafka.maxRatePerPartition", "100") - - // Setup the streaming context - ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) - - val kafkaStream = withClue("Error creating direct stream") { - val kc = new KafkaCluster(kafkaParams) - val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) - val m = kc.getEarliestLeaderOffsets(Set(topicPartition)) - .fold(e => Map.empty[TopicAndPartition, Long], m => m.mapValues(lo => lo.offset)) - - new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( - ssc, kafkaParams, m, messageHandler) { - override protected[streaming] val rateController = - Some(new DirectKafkaRateController(id, estimator)) - } - } - - val collectedData = - new mutable.ArrayBuffer[Array[String]]() with mutable.SynchronizedBuffer[Array[String]] - - // Used for assertion failure messages. - def dataToString: String = - collectedData.map(_.mkString("[", ",", "]")).mkString("{", ", ", "}") - - // This is to collect the raw data received from Kafka - kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) => - val data = rdd.map { _._2 }.collect() - collectedData += data - } - - ssc.start() - - // Try different rate limits. - // Send data to Kafka and wait for arrays of data to appear matching the rate. - Seq(100, 50, 20).foreach { rate => - collectedData.clear() // Empty this buffer on each pass. - estimator.updateRate(rate) // Set a new rate. - // Expect blocks of data equal to "rate", scaled by the interval length in secs. - val expectedSize = Math.round(rate * batchIntervalMilliseconds * 0.001) - kafkaTestUtils.sendMessages(topic, messages) - eventually(timeout(5.seconds), interval(batchIntervalMilliseconds.milliseconds)) { - // Assert that rate estimator values are used to determine maxMessagesPerPartition. - // Funky "-" in message makes the complete assertion message read better. - assert(collectedData.exists(_.size == expectedSize), - s" - No arrays of size $expectedSize for rate $rate found in $dataToString") - } - } - - ssc.stop() - } - - /** Get the generated offset ranges from the DirectKafkaStream */ - private def getOffsetRanges[K, V]( - kafkaStream: DStream[(K, V)]): Seq[(Time, Array[OffsetRange])] = { - kafkaStream.generatedRDDs.mapValues { rdd => - rdd.asInstanceOf[KafkaRDD[K, V, _, _, (K, V)]].offsetRanges - }.toSeq.sortBy { _._1 } - } -} - -object DirectKafkaStreamSuite { - val collectedData = new mutable.ArrayBuffer[String]() with mutable.SynchronizedBuffer[String] - @volatile var total = -1L - - class InputInfoCollector extends StreamingListener { - val numRecordsSubmitted = new AtomicLong(0L) - val numRecordsStarted = new AtomicLong(0L) - val numRecordsCompleted = new AtomicLong(0L) - - override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit = { - numRecordsSubmitted.addAndGet(batchSubmitted.batchInfo.numRecords) - } - - override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = { - numRecordsStarted.addAndGet(batchStarted.batchInfo.numRecords) - } - - override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { - numRecordsCompleted.addAndGet(batchCompleted.batchInfo.numRecords) - } - } -} - -private[streaming] class ConstantEstimator(@volatile private var rate: Long) - extends RateEstimator { - - def updateRate(newRate: Long): Unit = { - rate = newRate - } - - def compute( - time: Long, - elements: Long, - processingDelay: Long, - schedulingDelay: Long): Option[Double] = Some(rate) -} - 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 deleted file mode 100644 index f52a738afd65..000000000000 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ /dev/null @@ -1,175 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import scala.util.Random - -import kafka.serializer.StringDecoder -import kafka.common.TopicAndPartition -import kafka.message.MessageAndMetadata -import org.scalatest.BeforeAndAfterAll - -import org.apache.spark._ - -class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { - - private var kafkaTestUtils: KafkaTestUtils = _ - - private val sparkConf = new SparkConf().setMaster("local[4]") - .setAppName(this.getClass.getSimpleName) - private var sc: SparkContext = _ - - override def beforeAll { - sc = new SparkContext(sparkConf) - kafkaTestUtils = new KafkaTestUtils - kafkaTestUtils.setup() - } - - override def afterAll { - if (sc != null) { - sc.stop - sc = null - } - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null - } - } - - test("basic usage") { - val topic = s"topicbasic-${Random.nextInt}" - kafkaTestUtils.createTopic(topic) - val messages = Array("the", "quick", "brown", "fox") - kafkaTestUtils.sendMessages(topic, messages) - - val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "group.id" -> s"test-consumer-${Random.nextInt}") - - 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.toSet) - - // size-related method optimizations return sane results - assert(rdd.count === messages.size) - assert(rdd.countApprox(0).getFinalValue.mean === messages.size) - assert(!rdd.isEmpty) - assert(rdd.take(1).size === 1) - assert(rdd.take(1).head._2 === messages.head) - assert(rdd.take(messages.size + 10).size === messages.size) - - val emptyRdd = KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( - sc, kafkaParams, Array(OffsetRange(topic, 0, 0, 0))) - - assert(emptyRdd.isEmpty) - - // invalid offset ranges throw exceptions - val badRanges = Array(OffsetRange(topic, 0, 0, messages.size + 1)) - intercept[SparkException] { - KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( - sc, kafkaParams, badRanges) - } - } - - 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 = s"topicboundary-${Random.nextInt}" - val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) - kafkaTestUtils.createTopic(topic) - - val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "group.id" -> s"test-consumer-${Random.nextInt}") - - val kc = new KafkaCluster(kafkaParams) - - // this is the "lots of messages" case - kafkaTestUtils.sendMessages(topic, sent) - val sentCount = sent.values.sum - - // rdd defined from leaders after sending messages, should get the number sent - val rdd = getRdd(kc, Set(topic)) - - assert(rdd.isDefined) - - val ranges = rdd.get.asInstanceOf[HasOffsetRanges].offsetRanges - val rangeCount = ranges.map(o => o.untilOffset - o.fromOffset).sum - - assert(rangeCount === sentCount, "offset range didn't include all sent messages") - assert(rdd.get.count === sentCount, "didn't get all sent messages") - - val rangesMap = ranges.map(o => TopicAndPartition(o.topic, o.partition) -> o.untilOffset).toMap - - // make sure consumer offsets are committed before the next getRdd call - kc.setConsumerOffsets(kafkaParams("group.id"), rangesMap).fold( - err => throw new Exception(err.mkString("\n")), - _ => () - ) - - // this is the "0 messages" case - val rdd2 = getRdd(kc, Set(topic)) - // shouldn't get anything, since message is sent after rdd was defined - val sentOnlyOne = Map("d" -> 1) - - kafkaTestUtils.sendMessages(topic, sentOnlyOne) - - assert(rdd2.isDefined) - assert(rdd2.get.count === 0, "got messages when there shouldn't be any") - - // this is the "exactly 1 message" case, namely the single message from sentOnlyOne above - val rdd3 = getRdd(kc, Set(topic)) - // send lots of messages after rdd was defined, they shouldn't show up - kafkaTestUtils.sendMessages(topic, Map("extra" -> 22)) - - assert(rdd3.isDefined) - 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") - 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) - } - ) - } - 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}") - } - } - } - } -} diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml new file mode 100644 index 000000000000..48783d65826a --- /dev/null +++ b/external/kinesis-asl-assembly/pom.xml @@ -0,0 +1,214 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + spark-streaming-kinesis-asl-assembly_2.11 + jar + Spark Project Kinesis Assembly + http://spark.apache.org/ + + + streaming-kinesis-asl-assembly + + + + + org.apache.spark + spark-streaming-kinesis-asl_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + + com.fasterxml.jackson.core + jackson-databind + provided + + + commons-lang + commons-lang + provided + + + com.google.protobuf + protobuf-java + 2.6.1 + + + + org.glassfish.jersey.core + jersey-client + provided + + + org.glassfish.jersey.core + jersey-common + provided + + + org.glassfish.jersey.core + jersey-server + provided + + + log4j + log4j + provided + + + net.java.dev.jets3t + jets3t + provided + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.avro + avro-ipc + provided + + + org.apache.avro + avro-mapred + ${avro.mapred.classifier} + provided + + + org.apache.curator + curator-recipes + provided + + + org.apache.zookeeper + zookeeper + provided + + + org.slf4j + slf4j-api + provided + + + org.slf4j + slf4j-log4j12 + provided + + + org.xerial.snappy + snappy-java + provided + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + *:* + + + + + com.google.protobuf + kinesis.protobuf + + com.google.protobuf.** + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + log4j.properties + + + + + + + + + + + + diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml new file mode 100644 index 000000000000..40a751a652fa --- /dev/null +++ b/external/kinesis-asl/pom.xml @@ -0,0 +1,103 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + + spark-streaming-kinesis-asl_2.11 + jar + Spark Kinesis Integration + + + streaming-kinesis-asl + + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + test + + + com.amazonaws + amazon-kinesis-client + ${aws.kinesis.client.version} + + + com.amazonaws + aws-java-sdk-sts + ${aws.java.sdk.version} + + + com.amazonaws + amazon-kinesis-producer + ${aws.kinesis.producer.version} + test + + + org.mockito + mockito-core + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java b/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java similarity index 92% rename from extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java rename to external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java index 06e0ff28afd9..626bde48e1a8 100644 --- a/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java +++ b/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java @@ -16,12 +16,13 @@ */ package org.apache.spark.examples.streaming; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; import java.util.List; import java.util.regex.Pattern; -import com.amazonaws.regions.RegionUtils; -import org.apache.log4j.Logger; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function2; @@ -38,7 +39,6 @@ import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; import com.amazonaws.services.kinesis.AmazonKinesisClient; import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; -import com.google.common.collect.Lists; /** * Consumes messages from a Amazon Kinesis streams and does wordcount. @@ -79,9 +79,8 @@ */ public final class JavaKinesisWordCountASL { // needs to be public for access from run-example private static final Pattern WORD_SEPARATOR = Pattern.compile(" "); - private static final Logger logger = Logger.getLogger(JavaKinesisWordCountASL.class); - public static void main(String[] args) { + public static void main(String[] args) throws Exception { // Check that all required args were passed in. if (args.length != 3) { System.err.println( @@ -127,18 +126,19 @@ public static void main(String[] args) { // Get the region name from the endpoint URL to save Kinesis Client Library metadata in // DynamoDB of the same region as the Kinesis stream - String regionName = RegionUtils.getRegionByEndpoint(endpointUrl).getName(); + String regionName = KinesisExampleUtils.getRegionNameByEndpoint(endpointUrl); // Setup the Spark config and StreamingContext SparkConf sparkConfig = new SparkConf().setAppName("JavaKinesisWordCountASL"); JavaStreamingContext jssc = new JavaStreamingContext(sparkConfig, batchInterval); // Create the Kinesis DStreams - List> streamsList = new ArrayList>(numStreams); + List> streamsList = new ArrayList<>(numStreams); for (int i = 0; i < numStreams; i++) { streamsList.add( KinesisUtils.createStream(jssc, kinesisAppName, streamName, endpointUrl, regionName, - InitialPositionInStream.LATEST, kinesisCheckpointInterval, StorageLevel.MEMORY_AND_DISK_2()) + InitialPositionInStream.LATEST, kinesisCheckpointInterval, + StorageLevel.MEMORY_AND_DISK_2()) ); } @@ -154,8 +154,9 @@ public static void main(String[] args) { // Convert each line of Array[Byte] to String, and split into words JavaDStream words = unionStreams.flatMap(new FlatMapFunction() { @Override - public Iterable call(byte[] line) { - return Lists.newArrayList(WORD_SEPARATOR.split(new String(line))); + public Iterator call(byte[] line) { + String s = new String(line, StandardCharsets.UTF_8); + return Arrays.asList(WORD_SEPARATOR.split(s)).iterator(); } }); @@ -164,7 +165,7 @@ public Iterable call(byte[] line) { new PairFunction() { @Override public Tuple2 call(String s) { - return new Tuple2(s, 1); + return new Tuple2<>(s, 1); } } ).reduceByKey( diff --git a/extras/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py b/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py similarity index 94% rename from extras/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py rename to external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py index f428f64da3c4..4d7fc9a549bf 100644 --- a/extras/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py +++ b/external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py @@ -34,9 +34,9 @@ $ export AWS_SECRET_KEY= # run the example - $ bin/spark-submit -jar extras/kinesis-asl/target/scala-*/\ + $ bin/spark-submit -jar external/kinesis-asl/target/scala-*/\ spark-streaming-kinesis-asl-assembly_*.jar \ - extras/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py \ + external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py \ myAppName mySparkStream https://kinesis.us-east-1.amazonaws.com There is a companion helper class called KinesisWordProducerASL which puts dummy data @@ -54,6 +54,8 @@ See http://spark.apache.org/docs/latest/streaming-kinesis-integration.html for more details on the Kinesis Spark Streaming integration. """ +from __future__ import print_function + import sys from pyspark import SparkContext diff --git a/external/kinesis-asl/src/main/resources/log4j.properties b/external/kinesis-asl/src/main/resources/log4j.properties new file mode 100644 index 000000000000..4f5ea7bafe48 --- /dev/null +++ b/external/kinesis-asl/src/main/resources/log4j.properties @@ -0,0 +1,37 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +log4j.rootCategory=WARN, console + +# File appender +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Console appender +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.out +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + +# Settings to quiet third party logs that are too verbose +log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO +log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisExampleUtils.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisExampleUtils.scala new file mode 100644 index 000000000000..2eebd6130d4d --- /dev/null +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisExampleUtils.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.examples.streaming + +import scala.collection.JavaConverters._ + +import com.amazonaws.regions.RegionUtils +import com.amazonaws.services.kinesis.AmazonKinesis + +private[streaming] object KinesisExampleUtils { + def getRegionNameByEndpoint(endpoint: String): String = { + val uri = new java.net.URI(endpoint) + RegionUtils.getRegionsForService(AmazonKinesis.ENDPOINT_PREFIX) + .asScala + .find(_.getAvailableEndpoints.asScala.toSeq.contains(uri.getHost)) + .map(_.getName) + .getOrElse( + throw new IllegalArgumentException(s"Could not resolve region for endpoint: $endpoint")) + } +} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala similarity index 94% rename from extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala rename to external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala index de749626ec09..cde2c4b04c0c 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala @@ -22,18 +22,18 @@ import java.nio.ByteBuffer import scala.util.Random -import com.amazonaws.auth.{DefaultAWSCredentialsProviderChain, BasicAWSCredentials} -import com.amazonaws.regions.RegionUtils +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain import com.amazonaws.services.kinesis.AmazonKinesisClient import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream import com.amazonaws.services.kinesis.model.PutRecordRequest import org.apache.log4j.{Level, Logger} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.streaming.dstream.DStream.toPairDStreamFunctions -import org.apache.spark.streaming.kinesis.KinesisUtils +import org.apache.spark.streaming.kinesis.KinesisInputDStream /** @@ -126,7 +126,7 @@ object KinesisWordCountASL extends Logging { // Get the region name from the endpoint URL to save Kinesis Client Library metadata in // DynamoDB of the same region as the Kinesis stream - val regionName = RegionUtils.getRegionByEndpoint(endpointUrl).getName() + val regionName = KinesisExampleUtils.getRegionNameByEndpoint(endpointUrl) // Setup the SparkConfig and StreamingContext val sparkConfig = new SparkConf().setAppName("KinesisWordCountASL") @@ -134,8 +134,16 @@ object KinesisWordCountASL extends Logging { // Create the Kinesis DStreams val kinesisStreams = (0 until numStreams).map { i => - KinesisUtils.createStream(ssc, appName, streamName, endpointUrl, regionName, - InitialPositionInStream.LATEST, kinesisCheckpointInterval, StorageLevel.MEMORY_AND_DISK_2) + KinesisInputDStream.builder + .streamingContext(ssc) + .streamName(streamName) + .endpointUrl(endpointUrl) + .regionName(regionName) + .initialPositionInStream(InitialPositionInStream.LATEST) + .checkpointAppName(appName) + .checkpointInterval(kinesisCheckpointInterval) + .storageLevel(StorageLevel.MEMORY_AND_DISK_2) + .build() } // Union all the streams diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala similarity index 77% rename from extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala rename to external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala index 000897a4e729..88b294246bb3 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala @@ -21,11 +21,13 @@ import scala.collection.JavaConverters._ import scala.reflect.ClassTag import scala.util.control.NonFatal -import com.amazonaws.auth.{AWSCredentials, DefaultAWSCredentialsProviderChain} +import com.amazonaws.auth.AWSCredentials import com.amazonaws.services.kinesis.AmazonKinesisClient +import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord import com.amazonaws.services.kinesis.model._ import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.rdd.{BlockRDD, BlockRDDPartition} import org.apache.spark.storage.BlockId import org.apache.spark.util.NextIterator @@ -34,7 +36,11 @@ import org.apache.spark.util.NextIterator /** Class representing a range of Kinesis sequence numbers. Both sequence numbers are inclusive. */ private[kinesis] case class SequenceNumberRange( - streamName: String, shardId: String, fromSeqNumber: String, toSeqNumber: String) + streamName: String, + shardId: String, + fromSeqNumber: String, + toSeqNumber: String, + recordCount: Int) /** Class representing an array of Kinesis sequence number ranges */ private[kinesis] @@ -69,26 +75,26 @@ class KinesisBackedBlockRDDPartition( */ private[kinesis] class KinesisBackedBlockRDD[T: ClassTag]( - @transient sc: SparkContext, + sc: SparkContext, val regionName: String, val endpointUrl: String, - @transient blockIds: Array[BlockId], + @transient private val _blockIds: Array[BlockId], @transient val arrayOfseqNumberRanges: Array[SequenceNumberRanges], - @transient isBlockIdValid: Array[Boolean] = Array.empty, - val retryTimeoutMs: Int = 10000, - val messageHandler: Record => T = KinesisUtils.defaultMessageHandler _, - val awsCredentialsOption: Option[SerializableAWSCredentials] = None - ) extends BlockRDD[T](sc, blockIds) { + @transient private val isBlockIdValid: Array[Boolean] = Array.empty, + val messageHandler: Record => T = KinesisInputDStream.defaultMessageHandler _, + val kinesisCreds: SparkAWSCredentials = DefaultCredentials, + val kinesisReadConfigs: KinesisReadConfigurations = KinesisReadConfigurations() + ) extends BlockRDD[T](sc, _blockIds) { - require(blockIds.length == arrayOfseqNumberRanges.length, + require(_blockIds.length == arrayOfseqNumberRanges.length, "Number of blockIds is not equal to the number of sequence number ranges") override def isValid(): Boolean = true override def getPartitions: Array[Partition] = { - Array.tabulate(blockIds.length) { i => + Array.tabulate(_blockIds.length) { i => val isValid = if (isBlockIdValid.length == 0) true else isBlockIdValid(i) - new KinesisBackedBlockRDDPartition(i, blockIds(i), isValid, arrayOfseqNumberRanges(i)) + new KinesisBackedBlockRDDPartition(i, _blockIds(i), isValid, arrayOfseqNumberRanges(i)) } } @@ -103,12 +109,10 @@ class KinesisBackedBlockRDD[T: ClassTag]( } def getBlockFromKinesis(): Iterator[T] = { - val credentials = awsCredentialsOption.getOrElse { - new DefaultAWSCredentialsProviderChain().getCredentials() - } + val credentials = kinesisCreds.provider.getCredentials partition.seqNumberRanges.ranges.iterator.flatMap { range => new KinesisSequenceRangeIterator(credentials, endpointUrl, regionName, - range, retryTimeoutMs).map(messageHandler) + range, kinesisReadConfigs).map(messageHandler) } } if (partition.isBlockIdValid) { @@ -131,17 +135,19 @@ class KinesisSequenceRangeIterator( endpointUrl: String, regionId: String, range: SequenceNumberRange, - retryTimeoutMs: Int) extends NextIterator[Record] with Logging { + kinesisReadConfigs: KinesisReadConfigurations) extends NextIterator[Record] with Logging { private val client = new AmazonKinesisClient(credentials) private val streamName = range.streamName private val shardId = range.shardId + // AWS limits to maximum of 10k records per get call + private val maxGetRecordsLimit = 10000 private var toSeqNumberReceived = false private var lastSeqNumber: String = null private var internalIterator: Iterator[Record] = null - client.setEndpoint(endpointUrl, "kinesis", regionId) + client.setEndpoint(endpointUrl) override protected def getNext(): Record = { var nextRecord: Record = null @@ -153,12 +159,14 @@ class KinesisSequenceRangeIterator( // If the internal iterator has not been initialized, // then fetch records from starting sequence number - internalIterator = getRecords(ShardIteratorType.AT_SEQUENCE_NUMBER, range.fromSeqNumber) + internalIterator = getRecords(ShardIteratorType.AT_SEQUENCE_NUMBER, range.fromSeqNumber, + range.recordCount) } else if (!internalIterator.hasNext) { // If the internal iterator does not have any more records, // then fetch more records after the last consumed sequence number - internalIterator = getRecords(ShardIteratorType.AFTER_SEQUENCE_NUMBER, lastSeqNumber) + internalIterator = getRecords(ShardIteratorType.AFTER_SEQUENCE_NUMBER, lastSeqNumber, + range.recordCount) } if (!internalIterator.hasNext) { @@ -191,9 +199,12 @@ class KinesisSequenceRangeIterator( /** * Get records starting from or after the given sequence number. */ - private def getRecords(iteratorType: ShardIteratorType, seqNum: String): Iterator[Record] = { + private def getRecords( + iteratorType: ShardIteratorType, + seqNum: String, + recordCount: Int): Iterator[Record] = { val shardIterator = getKinesisIterator(iteratorType, seqNum) - val result = getRecordsAndNextKinesisIterator(shardIterator) + val result = getRecordsAndNextKinesisIterator(shardIterator, recordCount) result._1 } @@ -202,15 +213,20 @@ class KinesisSequenceRangeIterator( * to get records from Kinesis), and get the next shard iterator for next consumption. */ private def getRecordsAndNextKinesisIterator( - shardIterator: String): (Iterator[Record], String) = { + shardIterator: String, + recordCount: Int): (Iterator[Record], String) = { val getRecordsRequest = new GetRecordsRequest getRecordsRequest.setRequestCredentials(credentials) getRecordsRequest.setShardIterator(shardIterator) + getRecordsRequest.setLimit(Math.min(recordCount, this.maxGetRecordsLimit)) val getRecordsResult = retryOrTimeout[GetRecordsResult]( s"getting records using shard iterator") { client.getRecords(getRecordsRequest) } - (getRecordsResult.getRecords.iterator().asScala, getRecordsResult.getNextShardIterator) + // De-aggregate records, if KPL was used in producing the records. The KCL automatically + // handles de-aggregation during regular operation. This code path is used during recovery + val recordIterator = UserRecord.deaggregate(getRecordsResult.getRecords) + (recordIterator.iterator().asScala, getRecordsResult.getNextShardIterator) } /** @@ -235,21 +251,19 @@ class KinesisSequenceRangeIterator( /** Helper method to retry Kinesis API request with exponential backoff and timeouts */ private def retryOrTimeout[T](message: String)(body: => T): T = { - import KinesisSequenceRangeIterator._ - - var startTimeMs = System.currentTimeMillis() + val startTimeMs = System.currentTimeMillis() var retryCount = 0 - var waitTimeMs = MIN_RETRY_WAIT_TIME_MS var result: Option[T] = None var lastError: Throwable = null + var waitTimeInterval = kinesisReadConfigs.retryWaitTimeMs - def isTimedOut = (System.currentTimeMillis() - startTimeMs) >= retryTimeoutMs - def isMaxRetryDone = retryCount >= MAX_RETRIES + def isTimedOut = (System.currentTimeMillis() - startTimeMs) >= kinesisReadConfigs.retryTimeoutMs + def isMaxRetryDone = retryCount >= kinesisReadConfigs.maxRetries while (result.isEmpty && !isTimedOut && !isMaxRetryDone) { if (retryCount > 0) { // wait only if this is a retry - Thread.sleep(waitTimeMs) - waitTimeMs *= 2 // if you have waited, then double wait time for next round + Thread.sleep(waitTimeInterval) + waitTimeInterval *= 2 // if you have waited, then double wait time for next round } try { result = Some(body) @@ -268,7 +282,8 @@ class KinesisSequenceRangeIterator( result.getOrElse { if (isTimedOut) { throw new SparkException( - s"Timed out after $retryTimeoutMs ms while $message, last exception: ", lastError) + s"Timed out after ${kinesisReadConfigs.retryTimeoutMs} ms while " + + s"$message, last exception: ", lastError) } else { throw new SparkException( s"Gave up after $retryCount retries while $message, last exception: ", lastError) @@ -276,9 +291,3 @@ class KinesisSequenceRangeIterator( } } } - -private[streaming] -object KinesisSequenceRangeIterator { - val MAX_RETRIES = 3 - val MIN_RETRY_WAIT_TIME_MS = 100 -} diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala new file mode 100644 index 000000000000..5fb83b26f838 --- /dev/null +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointer.scala @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.kinesis + +import java.util.concurrent._ + +import scala.util.control.NonFatal + +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason + +import org.apache.spark.internal.Logging +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.util.RecurringTimer +import org.apache.spark.util.{Clock, SystemClock} + +/** + * This is a helper class for managing Kinesis checkpointing. + * + * @param receiver The receiver that keeps track of which sequence numbers we can checkpoint + * @param checkpointInterval How frequently we will checkpoint to DynamoDB + * @param workerId Worker Id of KCL worker for logging purposes + * @param clock In order to use ManualClocks for the purpose of testing + */ +private[kinesis] class KinesisCheckpointer( + receiver: KinesisReceiver[_], + checkpointInterval: Duration, + workerId: String, + clock: Clock = new SystemClock) extends Logging { + + // a map from shardId's to checkpointers + private val checkpointers = new ConcurrentHashMap[String, IRecordProcessorCheckpointer]() + + private val lastCheckpointedSeqNums = new ConcurrentHashMap[String, String]() + + private val checkpointerThread: RecurringTimer = startCheckpointerThread() + + /** Update the checkpointer instance to the most recent one for the given shardId. */ + def setCheckpointer(shardId: String, checkpointer: IRecordProcessorCheckpointer): Unit = { + checkpointers.put(shardId, checkpointer) + } + + /** + * Stop tracking the specified shardId. + * + * If a checkpointer is provided, e.g. on IRecordProcessor.shutdown [[ShutdownReason.TERMINATE]], + * we will use that to make the final checkpoint. If `null` is provided, we will not make the + * checkpoint, e.g. in case of [[ShutdownReason.ZOMBIE]]. + */ + def removeCheckpointer(shardId: String, checkpointer: IRecordProcessorCheckpointer): Unit = { + synchronized { + checkpointers.remove(shardId) + } + if (checkpointer != null) { + try { + // We must call `checkpoint()` with no parameter to finish reading shards. + // See an URL below for details: + // https://forums.aws.amazon.com/thread.jspa?threadID=244218 + KinesisRecordProcessor.retryRandom(checkpointer.checkpoint(), 4, 100) + } catch { + case NonFatal(e) => + logError(s"Exception: WorkerId $workerId encountered an exception while checkpointing" + + s"to finish reading a shard of $shardId.", e) + // Rethrow the exception to the Kinesis Worker that is managing this RecordProcessor + throw e + } + } + } + + /** Perform the checkpoint. */ + private def checkpoint(shardId: String, checkpointer: IRecordProcessorCheckpointer): Unit = { + try { + if (checkpointer != null) { + receiver.getLatestSeqNumToCheckpoint(shardId).foreach { latestSeqNum => + val lastSeqNum = lastCheckpointedSeqNums.get(shardId) + // Kinesis sequence numbers are monotonically increasing strings, therefore we can do + // safely do the string comparison + if (lastSeqNum == null || latestSeqNum > lastSeqNum) { + /* Perform the checkpoint */ + KinesisRecordProcessor.retryRandom(checkpointer.checkpoint(latestSeqNum), 4, 100) + logDebug(s"Checkpoint: WorkerId $workerId completed checkpoint at sequence number" + + s" $latestSeqNum for shardId $shardId") + lastCheckpointedSeqNums.put(shardId, latestSeqNum) + } + } + } else { + logDebug(s"Checkpointing skipped for shardId $shardId. Checkpointer not set.") + } + } catch { + case NonFatal(e) => + logWarning(s"Failed to checkpoint shardId $shardId to DynamoDB.", e) + } + } + + /** Checkpoint the latest saved sequence numbers for all active shardId's. */ + private def checkpointAll(): Unit = synchronized { + // if this method throws an exception, then the scheduled task will not run again + try { + val shardIds = checkpointers.keys() + while (shardIds.hasMoreElements) { + val shardId = shardIds.nextElement() + checkpoint(shardId, checkpointers.get(shardId)) + } + } catch { + case NonFatal(e) => + logWarning("Failed to checkpoint to DynamoDB.", e) + } + } + + /** + * Start the checkpointer thread with the given checkpoint duration. + */ + private def startCheckpointerThread(): RecurringTimer = { + val period = checkpointInterval.milliseconds + val threadName = s"Kinesis Checkpointer - Worker $workerId" + val timer = new RecurringTimer(clock, period, _ => checkpointAll(), threadName) + timer.start() + logDebug(s"Started checkpointer thread: $threadName") + timer + } + + /** + * Shutdown the checkpointer. Should be called on the onStop of the Receiver. + */ + def shutdown(): Unit = { + // the recurring timer checkpoints for us one last time. + checkpointerThread.stop(interruptTimer = false) + checkpointers.clear() + lastCheckpointedSeqNums.clear() + logInfo("Successfully shutdown Kinesis Checkpointer.") + } +} diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala new file mode 100644 index 000000000000..f61e398e7bdd --- /dev/null +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala @@ -0,0 +1,315 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis + +import scala.reflect.ClassTag + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream +import com.amazonaws.services.kinesis.model.Record + +import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.streaming.{Duration, StreamingContext, Time} +import org.apache.spark.streaming.api.java.JavaStreamingContext +import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.scheduler.ReceivedBlockInfo + +private[kinesis] class KinesisInputDStream[T: ClassTag]( + _ssc: StreamingContext, + val streamName: String, + val endpointUrl: String, + val regionName: String, + val initialPositionInStream: InitialPositionInStream, + val checkpointAppName: String, + val checkpointInterval: Duration, + val _storageLevel: StorageLevel, + val messageHandler: Record => T, + val kinesisCreds: SparkAWSCredentials, + val dynamoDBCreds: Option[SparkAWSCredentials], + val cloudWatchCreds: Option[SparkAWSCredentials] + ) extends ReceiverInputDStream[T](_ssc) { + + import KinesisReadConfigurations._ + + private[streaming] + override def createBlockRDD(time: Time, blockInfos: Seq[ReceivedBlockInfo]): RDD[T] = { + + // This returns true even for when blockInfos is empty + val allBlocksHaveRanges = blockInfos.map { _.metadataOption }.forall(_.nonEmpty) + + if (allBlocksHaveRanges) { + // Create a KinesisBackedBlockRDD, even when there are no blocks + val blockIds = blockInfos.map { _.blockId.asInstanceOf[BlockId] }.toArray + val seqNumRanges = blockInfos.map { + _.metadataOption.get.asInstanceOf[SequenceNumberRanges] }.toArray + val isBlockIdValid = blockInfos.map { _.isBlockIdValid() }.toArray + logDebug(s"Creating KinesisBackedBlockRDD for $time with ${seqNumRanges.length} " + + s"seq number ranges: ${seqNumRanges.mkString(", ")} ") + + new KinesisBackedBlockRDD( + context.sc, regionName, endpointUrl, blockIds, seqNumRanges, + isBlockIdValid = isBlockIdValid, + messageHandler = messageHandler, + kinesisCreds = kinesisCreds, + kinesisReadConfigs = KinesisReadConfigurations(ssc)) + } else { + logWarning("Kinesis sequence number information was not present with some block metadata," + + " it may not be possible to recover from failures") + super.createBlockRDD(time, blockInfos) + } + } + + override def getReceiver(): Receiver[T] = { + new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream, + checkpointAppName, checkpointInterval, _storageLevel, messageHandler, + kinesisCreds, dynamoDBCreds, cloudWatchCreds) + } +} + +@InterfaceStability.Evolving +object KinesisInputDStream { + /** + * Builder for [[KinesisInputDStream]] instances. + * + * @since 2.2.0 + */ + @InterfaceStability.Evolving + class Builder { + // Required params + private var streamingContext: Option[StreamingContext] = None + private var streamName: Option[String] = None + private var checkpointAppName: Option[String] = None + + // Params with defaults + private var endpointUrl: Option[String] = None + private var regionName: Option[String] = None + private var initialPositionInStream: Option[InitialPositionInStream] = None + private var checkpointInterval: Option[Duration] = None + private var storageLevel: Option[StorageLevel] = None + private var kinesisCredsProvider: Option[SparkAWSCredentials] = None + private var dynamoDBCredsProvider: Option[SparkAWSCredentials] = None + private var cloudWatchCredsProvider: Option[SparkAWSCredentials] = None + + /** + * Sets the StreamingContext that will be used to construct the Kinesis DStream. This is a + * required parameter. + * + * @param ssc [[StreamingContext]] used to construct Kinesis DStreams + * @return Reference to this [[KinesisInputDStream.Builder]] + */ + def streamingContext(ssc: StreamingContext): Builder = { + streamingContext = Option(ssc) + this + } + + /** + * Sets the StreamingContext that will be used to construct the Kinesis DStream. This is a + * required parameter. + * + * @param jssc [[JavaStreamingContext]] used to construct Kinesis DStreams + * @return Reference to this [[KinesisInputDStream.Builder]] + */ + def streamingContext(jssc: JavaStreamingContext): Builder = { + streamingContext = Option(jssc.ssc) + this + } + + /** + * Sets the name of the Kinesis stream that the DStream will read from. This is a required + * parameter. + * + * @param streamName Name of Kinesis stream that the DStream will read from + * @return Reference to this [[KinesisInputDStream.Builder]] + */ + def streamName(streamName: String): Builder = { + this.streamName = Option(streamName) + this + } + + /** + * Sets the KCL application name to use when checkpointing state to DynamoDB. This is a + * required parameter. + * + * @param appName Value to use for the KCL app name (used when creating the DynamoDB checkpoint + * table and when writing metrics to CloudWatch) + * @return Reference to this [[KinesisInputDStream.Builder]] + */ + def checkpointAppName(appName: String): Builder = { + checkpointAppName = Option(appName) + this + } + + /** + * Sets the AWS Kinesis endpoint URL. Defaults to "https://kinesis.us-east-1.amazonaws.com" if + * no custom value is specified + * + * @param url Kinesis endpoint URL to use + * @return Reference to this [[KinesisInputDStream.Builder]] + */ + def endpointUrl(url: String): Builder = { + endpointUrl = Option(url) + this + } + + /** + * Sets the AWS region to construct clients for. Defaults to "us-east-1" if no custom value + * is specified. + * + * @param regionName Name of AWS region to use (e.g. "us-west-2") + * @return Reference to this [[KinesisInputDStream.Builder]] + */ + def regionName(regionName: String): Builder = { + this.regionName = Option(regionName) + this + } + + /** + * Sets the initial position data is read from in the Kinesis stream. Defaults to + * [[InitialPositionInStream.LATEST]] if no custom value is specified. + * + * @param initialPosition InitialPositionInStream value specifying where Spark Streaming + * will start reading records in the Kinesis stream from + * @return Reference to this [[KinesisInputDStream.Builder]] + */ + def initialPositionInStream(initialPosition: InitialPositionInStream): Builder = { + initialPositionInStream = Option(initialPosition) + this + } + + /** + * Sets how often the KCL application state is checkpointed to DynamoDB. Defaults to the Spark + * Streaming batch interval if no custom value is specified. + * + * @param interval [[Duration]] specifying how often the KCL state should be checkpointed to + * DynamoDB. + * @return Reference to this [[KinesisInputDStream.Builder]] + */ + def checkpointInterval(interval: Duration): Builder = { + checkpointInterval = Option(interval) + this + } + + /** + * Sets the storage level of the blocks for the DStream created. Defaults to + * [[StorageLevel.MEMORY_AND_DISK_2]] if no custom value is specified. + * + * @param storageLevel [[StorageLevel]] to use for the DStream data blocks + * @return Reference to this [[KinesisInputDStream.Builder]] + */ + def storageLevel(storageLevel: StorageLevel): Builder = { + this.storageLevel = Option(storageLevel) + this + } + + /** + * Sets the [[SparkAWSCredentials]] to use for authenticating to the AWS Kinesis + * endpoint. Defaults to [[DefaultCredentialsProvider]] if no custom value is specified. + * + * @param credentials [[SparkAWSCredentials]] to use for Kinesis authentication + */ + def kinesisCredentials(credentials: SparkAWSCredentials): Builder = { + kinesisCredsProvider = Option(credentials) + this + } + + /** + * Sets the [[SparkAWSCredentials]] to use for authenticating to the AWS DynamoDB + * endpoint. Will use the same credentials used for AWS Kinesis if no custom value is set. + * + * @param credentials [[SparkAWSCredentials]] to use for DynamoDB authentication + */ + def dynamoDBCredentials(credentials: SparkAWSCredentials): Builder = { + dynamoDBCredsProvider = Option(credentials) + this + } + + /** + * Sets the [[SparkAWSCredentials]] to use for authenticating to the AWS CloudWatch + * endpoint. Will use the same credentials used for AWS Kinesis if no custom value is set. + * + * @param credentials [[SparkAWSCredentials]] to use for CloudWatch authentication + */ + def cloudWatchCredentials(credentials: SparkAWSCredentials): Builder = { + cloudWatchCredsProvider = Option(credentials) + this + } + + /** + * Create a new instance of [[KinesisInputDStream]] with configured parameters and the provided + * message handler. + * + * @param handler Function converting [[Record]] instances read by the KCL to DStream type [[T]] + * @return Instance of [[KinesisInputDStream]] constructed with configured parameters + */ + def buildWithMessageHandler[T: ClassTag]( + handler: Record => T): KinesisInputDStream[T] = { + val ssc = getRequiredParam(streamingContext, "streamingContext") + new KinesisInputDStream( + ssc, + getRequiredParam(streamName, "streamName"), + endpointUrl.getOrElse(DEFAULT_KINESIS_ENDPOINT_URL), + regionName.getOrElse(DEFAULT_KINESIS_REGION_NAME), + initialPositionInStream.getOrElse(DEFAULT_INITIAL_POSITION_IN_STREAM), + getRequiredParam(checkpointAppName, "checkpointAppName"), + checkpointInterval.getOrElse(ssc.graph.batchDuration), + storageLevel.getOrElse(DEFAULT_STORAGE_LEVEL), + ssc.sc.clean(handler), + kinesisCredsProvider.getOrElse(DefaultCredentials), + dynamoDBCredsProvider, + cloudWatchCredsProvider) + } + + /** + * Create a new instance of [[KinesisInputDStream]] with configured parameters and using the + * default message handler, which returns [[Array[Byte]]]. + * + * @return Instance of [[KinesisInputDStream]] constructed with configured parameters + */ + def build(): KinesisInputDStream[Array[Byte]] = buildWithMessageHandler(defaultMessageHandler) + + private def getRequiredParam[T](param: Option[T], paramName: String): T = param.getOrElse { + throw new IllegalArgumentException(s"No value provided for required parameter $paramName") + } + } + + /** + * Creates a [[KinesisInputDStream.Builder]] for constructing [[KinesisInputDStream]] instances. + * + * @since 2.2.0 + * + * @return [[KinesisInputDStream.Builder]] instance + */ + def builder: Builder = new Builder + + private[kinesis] def defaultMessageHandler(record: Record): Array[Byte] = { + if (record == null) return null + val byteBuffer = record.getData() + val byteArray = new Array[Byte](byteBuffer.remaining()) + byteBuffer.get(byteArray) + byteArray + } + + private[kinesis] val DEFAULT_KINESIS_ENDPOINT_URL: String = + "https://kinesis.us-east-1.amazonaws.com" + private[kinesis] val DEFAULT_KINESIS_REGION_NAME: String = "us-east-1" + private[kinesis] val DEFAULT_INITIAL_POSITION_IN_STREAM: InitialPositionInStream = + InitialPositionInStream.LATEST + private[kinesis] val DEFAULT_STORAGE_LEVEL: StorageLevel = StorageLevel.MEMORY_AND_DISK_2 +} diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReadConfigurations.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReadConfigurations.scala new file mode 100644 index 000000000000..871071e4677e --- /dev/null +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReadConfigurations.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kinesis + +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.streaming.StreamingContext + +/** + * Configurations to pass to the [[KinesisBackedBlockRDD]]. + * + * @param maxRetries: The maximum number of attempts to be made to Kinesis. Defaults to 3. + * @param retryWaitTimeMs: The interval between consequent Kinesis retries. + * Defaults to 100ms. + * @param retryTimeoutMs: The timeout in milliseconds for a Kinesis request. + * Defaults to batch duration provided for streaming, + * else uses 10000 if invoked directly. + */ +private[kinesis] case class KinesisReadConfigurations( + maxRetries: Int, + retryWaitTimeMs: Long, + retryTimeoutMs: Long) + +private[kinesis] object KinesisReadConfigurations { + def apply(): KinesisReadConfigurations = { + KinesisReadConfigurations(maxRetries = DEFAULT_MAX_RETRIES, + retryWaitTimeMs = JavaUtils.timeStringAsMs(DEFAULT_RETRY_WAIT_TIME), + retryTimeoutMs = DEFAULT_RETRY_TIMEOUT) + } + + def apply(ssc: StreamingContext): KinesisReadConfigurations = { + KinesisReadConfigurations( + maxRetries = ssc.sc.getConf.getInt(RETRY_MAX_ATTEMPTS_KEY, DEFAULT_MAX_RETRIES), + retryWaitTimeMs = JavaUtils.timeStringAsMs( + ssc.sc.getConf.get(RETRY_WAIT_TIME_KEY, DEFAULT_RETRY_WAIT_TIME)), + retryTimeoutMs = ssc.graph.batchDuration.milliseconds) + } + + /** + * SparkConf key for configuring the maximum number of retries used when attempting a Kinesis + * request. + */ + val RETRY_MAX_ATTEMPTS_KEY = "spark.streaming.kinesis.retry.maxAttempts" + + /** + * SparkConf key for configuring the wait time to use before retrying a Kinesis attempt. + */ + val RETRY_WAIT_TIME_KEY = "spark.streaming.kinesis.retry.waitTime" + + /** + * Default value for the RETRY_MAX_ATTEMPTS_KEY + */ + val DEFAULT_MAX_RETRIES = 3 + + /** + * Default value for the RETRY_WAIT_TIME_KEY + */ + val DEFAULT_RETRY_WAIT_TIME = "100ms" + + /** + * Default value for the retry timeout + */ + val DEFAULT_RETRY_TIMEOUT = 10000 +} diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala new file mode 100644 index 000000000000..1026d0fcb59b --- /dev/null +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala @@ -0,0 +1,353 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis + +import java.util.UUID +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util.control.NonFatal + +import com.amazonaws.services.kinesis.clientlibrary.interfaces.{IRecordProcessor, IRecordProcessorCheckpointer, IRecordProcessorFactory} +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.{InitialPositionInStream, KinesisClientLibConfiguration, Worker} +import com.amazonaws.services.kinesis.model.Record + +import org.apache.spark.internal.Logging +import org.apache.spark.storage.{StorageLevel, StreamBlockId} +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver} +import org.apache.spark.util.Utils + +/** + * Custom AWS Kinesis-specific implementation of Spark Streaming's Receiver. + * This implementation relies on the Kinesis Client Library (KCL) Worker as described here: + * https://github.com/awslabs/amazon-kinesis-client + * + * The way this Receiver works is as follows: + * + * - The receiver starts a KCL Worker, which is essentially runs a threadpool of multiple + * KinesisRecordProcessor + * - Each KinesisRecordProcessor receives data from a Kinesis shard in batches. Each batch is + * inserted into a Block Generator, and the corresponding range of sequence numbers is recorded. + * - When the block generator defines a block, then the recorded sequence number ranges that were + * inserted into the block are recorded separately for being used later. + * - When the block is ready to be pushed, the block is pushed and the ranges are reported as + * metadata of the block. In addition, the ranges are used to find out the latest sequence + * number for each shard that can be checkpointed through the DynamoDB. + * - Periodically, each KinesisRecordProcessor checkpoints the latest successfully stored sequence + * number for it own shard. + * + * @param streamName Kinesis stream name + * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) + * @param regionName Region name used by the Kinesis Client Library for + * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics) + * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the + * worker's initial starting position in the stream. + * The values are either the beginning of the stream + * per Kinesis' limit of 24 hours + * (InitialPositionInStream.TRIM_HORIZON) or + * the tip of the stream (InitialPositionInStream.LATEST). + * @param checkpointAppName Kinesis application name. Kinesis Apps are mapped to Kinesis Streams + * by the Kinesis Client Library. If you change the App name or Stream name, + * the KCL will throw errors. This usually requires deleting the backing + * DynamoDB table with the same name this Kinesis application. + * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. + * See the Kinesis Spark Streaming documentation for more + * details on the different types of checkpoints. + * @param storageLevel Storage level to use for storing the received objects + * @param kinesisCreds SparkAWSCredentials instance that will be used to generate the + * AWSCredentialsProvider passed to the KCL to authorize Kinesis API calls. + * @param cloudWatchCreds Optional SparkAWSCredentials instance that will be used to generate the + * AWSCredentialsProvider passed to the KCL to authorize CloudWatch API + * calls. Will use kinesisCreds if value is None. + * @param dynamoDBCreds Optional SparkAWSCredentials instance that will be used to generate the + * AWSCredentialsProvider passed to the KCL to authorize DynamoDB API calls. + * Will use kinesisCreds if value is None. + */ +private[kinesis] class KinesisReceiver[T]( + val streamName: String, + endpointUrl: String, + regionName: String, + initialPositionInStream: InitialPositionInStream, + checkpointAppName: String, + checkpointInterval: Duration, + storageLevel: StorageLevel, + messageHandler: Record => T, + kinesisCreds: SparkAWSCredentials, + dynamoDBCreds: Option[SparkAWSCredentials], + cloudWatchCreds: Option[SparkAWSCredentials]) + extends Receiver[T](storageLevel) with Logging { receiver => + + /* + * ================================================================================= + * The following vars are initialize in the onStart() method which executes in the + * Spark worker after this Receiver is serialized and shipped to the worker. + * ================================================================================= + */ + + /** + * workerId is used by the KCL should be based on the ip address of the actual Spark Worker + * where this code runs (not the driver's IP address.) + */ + @volatile private var workerId: String = null + + /** + * Worker is the core client abstraction from the Kinesis Client Library (KCL). + * A worker can process more than one shards from the given stream. + * Each shard is assigned its own IRecordProcessor and the worker run multiple such + * processors. + */ + @volatile private var worker: Worker = null + @volatile private var workerThread: Thread = null + + /** BlockGenerator used to generates blocks out of Kinesis data */ + @volatile private var blockGenerator: BlockGenerator = null + + /** + * Sequence number ranges added to the current block being generated. + * Accessing and updating of this map is synchronized by locks in BlockGenerator. + */ + private val seqNumRangesInCurrentBlock = new mutable.ArrayBuffer[SequenceNumberRange] + + /** Sequence number ranges of data added to each generated block */ + private val blockIdToSeqNumRanges = new ConcurrentHashMap[StreamBlockId, SequenceNumberRanges] + + /** + * The centralized kinesisCheckpointer that checkpoints based on the given checkpointInterval. + */ + @volatile private var kinesisCheckpointer: KinesisCheckpointer = null + + /** + * Latest sequence number ranges that have been stored successfully. + * This is used for checkpointing through KCL */ + private val shardIdToLatestStoredSeqNum = new ConcurrentHashMap[String, String] + + /** + * This is called when the KinesisReceiver starts and must be non-blocking. + * The KCL creates and manages the receiving/processing thread pool through Worker.run(). + */ + override def onStart() { + blockGenerator = supervisor.createBlockGenerator(new GeneratedBlockHandler) + + workerId = Utils.localHostName() + ":" + UUID.randomUUID() + + kinesisCheckpointer = new KinesisCheckpointer(receiver, checkpointInterval, workerId) + val kinesisProvider = kinesisCreds.provider + val kinesisClientLibConfiguration = new KinesisClientLibConfiguration( + checkpointAppName, + streamName, + kinesisProvider, + dynamoDBCreds.map(_.provider).getOrElse(kinesisProvider), + cloudWatchCreds.map(_.provider).getOrElse(kinesisProvider), + workerId) + .withKinesisEndpoint(endpointUrl) + .withInitialPositionInStream(initialPositionInStream) + .withTaskBackoffTimeMillis(500) + .withRegionName(regionName) + + /* + * RecordProcessorFactory creates impls of IRecordProcessor. + * IRecordProcessor adapts the KCL to our Spark KinesisReceiver via the + * IRecordProcessor.processRecords() method. + * We're using our custom KinesisRecordProcessor in this case. + */ + val recordProcessorFactory = new IRecordProcessorFactory { + override def createProcessor: IRecordProcessor = + new KinesisRecordProcessor(receiver, workerId) + } + + worker = new Worker(recordProcessorFactory, kinesisClientLibConfiguration) + workerThread = new Thread() { + override def run(): Unit = { + try { + worker.run() + } catch { + case NonFatal(e) => + restart("Error running the KCL worker in Receiver", e) + } + } + } + + blockIdToSeqNumRanges.clear() + blockGenerator.start() + + workerThread.setName(s"Kinesis Receiver ${streamId}") + workerThread.setDaemon(true) + workerThread.start() + + logInfo(s"Started receiver with workerId $workerId") + } + + /** + * This is called when the KinesisReceiver stops. + * The KCL worker.shutdown() method stops the receiving/processing threads. + * The KCL will do its best to drain and checkpoint any in-flight records upon shutdown. + */ + override def onStop() { + if (workerThread != null) { + if (worker != null) { + worker.shutdown() + worker = null + } + workerThread.join() + workerThread = null + logInfo(s"Stopped receiver for workerId $workerId") + } + workerId = null + if (kinesisCheckpointer != null) { + kinesisCheckpointer.shutdown() + kinesisCheckpointer = null + } + } + + /** Add records of the given shard to the current block being generated */ + private[kinesis] def addRecords(shardId: String, records: java.util.List[Record]): Unit = { + if (records.size > 0) { + val dataIterator = records.iterator().asScala.map(messageHandler) + val metadata = SequenceNumberRange(streamName, shardId, + records.get(0).getSequenceNumber(), records.get(records.size() - 1).getSequenceNumber(), + records.size()) + blockGenerator.addMultipleDataWithCallback(dataIterator, metadata) + } + } + + /** Return the current rate limit defined in [[BlockGenerator]]. */ + private[kinesis] def getCurrentLimit: Int = { + assert(blockGenerator != null) + math.min(blockGenerator.getCurrentLimit, Int.MaxValue).toInt + } + + /** Get the latest sequence number for the given shard that can be checkpointed through KCL */ + private[kinesis] def getLatestSeqNumToCheckpoint(shardId: String): Option[String] = { + Option(shardIdToLatestStoredSeqNum.get(shardId)) + } + + /** + * Set the checkpointer that will be used to checkpoint sequence numbers to DynamoDB for the + * given shardId. + */ + def setCheckpointer(shardId: String, checkpointer: IRecordProcessorCheckpointer): Unit = { + assert(kinesisCheckpointer != null, "Kinesis Checkpointer not initialized!") + kinesisCheckpointer.setCheckpointer(shardId, checkpointer) + } + + /** + * Remove the checkpointer for the given shardId. The provided checkpointer will be used to + * checkpoint one last time for the given shard. If `checkpointer` is `null`, then we will not + * checkpoint. + */ + def removeCheckpointer(shardId: String, checkpointer: IRecordProcessorCheckpointer): Unit = { + assert(kinesisCheckpointer != null, "Kinesis Checkpointer not initialized!") + kinesisCheckpointer.removeCheckpointer(shardId, checkpointer) + } + + /** + * Remember the range of sequence numbers that was added to the currently active block. + * Internally, this is synchronized with `finalizeRangesForCurrentBlock()`. + */ + private def rememberAddedRange(range: SequenceNumberRange): Unit = { + seqNumRangesInCurrentBlock += range + } + + /** + * Finalize the ranges added to the block that was active and prepare the ranges buffer + * for next block. Internally, this is synchronized with `rememberAddedRange()`. + */ + private def finalizeRangesForCurrentBlock(blockId: StreamBlockId): Unit = { + blockIdToSeqNumRanges.put(blockId, SequenceNumberRanges(seqNumRangesInCurrentBlock.toArray)) + seqNumRangesInCurrentBlock.clear() + logDebug(s"Generated block $blockId has $blockIdToSeqNumRanges") + } + + /** Store the block along with its associated ranges */ + private def storeBlockWithRanges( + blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[T]): Unit = { + val rangesToReportOption = Option(blockIdToSeqNumRanges.remove(blockId)) + if (rangesToReportOption.isEmpty) { + stop("Error while storing block into Spark, could not find sequence number ranges " + + s"for block $blockId") + return + } + + val rangesToReport = rangesToReportOption.get + var attempt = 0 + var stored = false + var throwable: Throwable = null + while (!stored && attempt <= 3) { + try { + store(arrayBuffer, rangesToReport) + stored = true + } catch { + case NonFatal(th) => + attempt += 1 + throwable = th + } + } + if (!stored) { + stop("Error while storing block into Spark", throwable) + } + + // Update the latest sequence number that have been successfully stored for each shard + // Note that we are doing this sequentially because the array of sequence number ranges + // is assumed to be + rangesToReport.ranges.foreach { range => + shardIdToLatestStoredSeqNum.put(range.shardId, range.toSeqNumber) + } + } + + /** + * Class to handle blocks generated by this receiver's block generator. Specifically, in + * the context of the Kinesis Receiver, this handler does the following. + * + * - When an array of records is added to the current active block in the block generator, + * this handler keeps track of the corresponding sequence number range. + * - When the currently active block is ready to sealed (not more records), this handler + * keep track of the list of ranges added into this block in another H + */ + private class GeneratedBlockHandler extends BlockGeneratorListener { + + /** + * Callback method called after a data item is added into the BlockGenerator. + * The data addition, block generation, and calls to onAddData and onGenerateBlock + * are all synchronized through the same lock. + */ + def onAddData(data: Any, metadata: Any): Unit = { + rememberAddedRange(metadata.asInstanceOf[SequenceNumberRange]) + } + + /** + * Callback method called after a block has been generated. + * The data addition, block generation, and calls to onAddData and onGenerateBlock + * are all synchronized through the same lock. + */ + def onGenerateBlock(blockId: StreamBlockId): Unit = { + finalizeRangesForCurrentBlock(blockId) + } + + /** Callback method called when a block is ready to be pushed / stored. */ + def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { + storeBlockWithRanges(blockId, + arrayBuffer.asInstanceOf[mutable.ArrayBuffer[T]]) + } + + /** Callback called in case of any error in internal of the BlockGenerator */ + def onError(message: String, throwable: Throwable): Unit = { + reportError(message, throwable) + } + } +} diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala new file mode 100644 index 000000000000..8c6a399dd763 --- /dev/null +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis + +import java.util.List + +import scala.util.Random +import scala.util.control.NonFatal + +import com.amazonaws.services.kinesis.clientlibrary.exceptions.{InvalidStateException, KinesisClientLibDependencyException, ShutdownException, ThrottlingException} +import com.amazonaws.services.kinesis.clientlibrary.interfaces.{IRecordProcessor, IRecordProcessorCheckpointer} +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason +import com.amazonaws.services.kinesis.model.Record + +import org.apache.spark.internal.Logging + +/** + * Kinesis-specific implementation of the Kinesis Client Library (KCL) IRecordProcessor. + * This implementation operates on the Array[Byte] from the KinesisReceiver. + * The Kinesis Worker creates an instance of this KinesisRecordProcessor for each + * shard in the Kinesis stream upon startup. This is normally done in separate threads, + * but the KCLs within the KinesisReceivers will balance themselves out if you create + * multiple Receivers. + * + * @param receiver Kinesis receiver + * @param workerId for logging purposes + */ +private[kinesis] class KinesisRecordProcessor[T](receiver: KinesisReceiver[T], workerId: String) + extends IRecordProcessor with Logging { + + // shardId populated during initialize() + @volatile + private var shardId: String = _ + + /** + * The Kinesis Client Library calls this method during IRecordProcessor initialization. + * + * @param shardId assigned by the KCL to this particular RecordProcessor. + */ + override def initialize(shardId: String) { + this.shardId = shardId + logInfo(s"Initialized workerId $workerId with shardId $shardId") + } + + /** + * This method is called by the KCL when a batch of records is pulled from the Kinesis stream. + * This is the record-processing bridge between the KCL's IRecordProcessor.processRecords() + * and Spark Streaming's Receiver.store(). + * + * @param batch list of records from the Kinesis stream shard + * @param checkpointer used to update Kinesis when this batch has been processed/stored + * in the DStream + */ + override def processRecords(batch: List[Record], checkpointer: IRecordProcessorCheckpointer) { + if (!receiver.isStopped()) { + try { + // Limit the number of processed records from Kinesis stream. This is because the KCL cannot + // control the number of aggregated records to be fetched even if we set `MaxRecords` + // in `KinesisClientLibConfiguration`. For example, if we set 10 to the number of max + // records in a worker and a producer aggregates two records into one message, the worker + // possibly 20 records every callback function called. + val maxRecords = receiver.getCurrentLimit + for (start <- 0 until batch.size by maxRecords) { + val miniBatch = batch.subList(start, math.min(start + maxRecords, batch.size)) + receiver.addRecords(shardId, miniBatch) + logDebug(s"Stored: Worker $workerId stored ${miniBatch.size} records " + + s"for shardId $shardId") + } + receiver.setCheckpointer(shardId, checkpointer) + } catch { + case NonFatal(e) => + /* + * If there is a failure within the batch, the batch will not be checkpointed. + * This will potentially cause records since the last checkpoint to be processed + * more than once. + */ + logError(s"Exception: WorkerId $workerId encountered and exception while storing " + + s" or checkpointing a batch for workerId $workerId and shardId $shardId.", e) + + /* Rethrow the exception to the Kinesis Worker that is managing this RecordProcessor. */ + throw e + } + } else { + /* RecordProcessor has been stopped. */ + logInfo(s"Stopped: KinesisReceiver has stopped for workerId $workerId" + + s" and shardId $shardId. No more records will be processed.") + } + } + + /** + * Kinesis Client Library is shutting down this Worker for 1 of 2 reasons: + * 1) the stream is resharding by splitting or merging adjacent shards + * (ShutdownReason.TERMINATE) + * 2) the failed or latent Worker has stopped sending heartbeats for whatever reason + * (ShutdownReason.ZOMBIE) + * + * @param checkpointer used to perform a Kinesis checkpoint for ShutdownReason.TERMINATE + * @param reason for shutdown (ShutdownReason.TERMINATE or ShutdownReason.ZOMBIE) + */ + override def shutdown( + checkpointer: IRecordProcessorCheckpointer, + reason: ShutdownReason): Unit = { + logInfo(s"Shutdown: Shutting down workerId $workerId with reason $reason") + // null if not initialized before shutdown: + if (shardId == null) { + logWarning(s"No shardId for workerId $workerId?") + } else { + reason match { + /* + * TERMINATE Use Case. Checkpoint. + * Checkpoint to indicate that all records from the shard have been drained and processed. + * It's now OK to read from the new shards that resulted from a resharding event. + */ + case ShutdownReason.TERMINATE => receiver.removeCheckpointer(shardId, checkpointer) + + /* + * ZOMBIE Use Case or Unknown reason. NoOp. + * No checkpoint because other workers may have taken over and already started processing + * the same records. + * This may lead to records being processed more than once. + * Return null so that we don't checkpoint + */ + case _ => receiver.removeCheckpointer(shardId, null) + } + } + } +} + +private[kinesis] object KinesisRecordProcessor extends Logging { + /** + * Retry the given amount of times with a random backoff time (millis) less than the + * given maxBackOffMillis + * + * @param expression expression to evaluate + * @param numRetriesLeft number of retries left + * @param maxBackOffMillis: max millis between retries + * + * @return evaluation of the given expression + * @throws Unretryable exception, unexpected exception, + * or any exception that persists after numRetriesLeft reaches 0 + */ + @annotation.tailrec + def retryRandom[T](expression: => T, numRetriesLeft: Int, maxBackOffMillis: Int): T = { + util.Try { expression } match { + /* If the function succeeded, evaluate to x. */ + case util.Success(x) => x + /* If the function failed, either retry or throw the exception */ + case util.Failure(e) => e match { + /* Retry: Throttling or other Retryable exception has occurred */ + case _: ThrottlingException | _: KinesisClientLibDependencyException + if numRetriesLeft > 1 => + val backOffMillis = Random.nextInt(maxBackOffMillis) + Thread.sleep(backOffMillis) + logError(s"Retryable Exception: Random backOffMillis=${backOffMillis}", e) + retryRandom(expression, numRetriesLeft - 1, maxBackOffMillis) + /* Throw: Shutdown has been requested by the Kinesis Client Library. */ + case _: ShutdownException => + logError(s"ShutdownException: Caught shutdown exception, skipping checkpoint.", e) + throw e + /* Throw: Non-retryable exception has occurred with the Kinesis Client Library */ + case _: InvalidStateException => + logError(s"InvalidStateException: Cannot save checkpoint to the DynamoDB table used" + + s" by the Amazon Kinesis Client Library. Table likely doesn't exist.", e) + throw e + /* Throw: Unexpected exception has occurred */ + case _ => + logError(s"Unexpected, non-retryable exception.", e) + throw e + } + } + } +} diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala new file mode 100644 index 000000000000..73ac7a3cd235 --- /dev/null +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala @@ -0,0 +1,295 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kinesis + +import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.util.{Failure, Random, Success, Try} + +import com.amazonaws.auth.{AWSCredentials, DefaultAWSCredentialsProviderChain} +import com.amazonaws.regions.RegionUtils +import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient +import com.amazonaws.services.dynamodbv2.document.DynamoDB +import com.amazonaws.services.kinesis.{AmazonKinesis, AmazonKinesisClient} +import com.amazonaws.services.kinesis.model._ + +import org.apache.spark.internal.Logging + +/** + * Shared utility methods for performing Kinesis tests that actually transfer data. + * + * PLEASE KEEP THIS FILE UNDER src/main AS PYTHON TESTS NEED ACCESS TO THIS FILE! + */ +private[kinesis] class KinesisTestUtils(streamShardCount: Int = 2) extends Logging { + + val endpointUrl = KinesisTestUtils.endpointUrl + val regionName = KinesisTestUtils.getRegionNameByEndpoint(endpointUrl) + + private val createStreamTimeoutSeconds = 300 + private val describeStreamPollTimeSeconds = 1 + + @volatile + private var streamCreated = false + + @volatile + private var _streamName: String = _ + + protected lazy val kinesisClient = { + val client = new AmazonKinesisClient(KinesisTestUtils.getAWSCredentials()) + client.setEndpoint(endpointUrl) + client + } + + private lazy val dynamoDB = { + val dynamoDBClient = new AmazonDynamoDBClient(new DefaultAWSCredentialsProviderChain()) + dynamoDBClient.setRegion(RegionUtils.getRegion(regionName)) + new DynamoDB(dynamoDBClient) + } + + protected def getProducer(aggregate: Boolean): KinesisDataGenerator = { + if (!aggregate) { + new SimpleDataGenerator(kinesisClient) + } else { + throw new UnsupportedOperationException("Aggregation is not supported through this code path") + } + } + + def streamName: String = { + require(streamCreated, "Stream not yet created, call createStream() to create one") + _streamName + } + + def createStream(): Unit = { + require(!streamCreated, "Stream already created") + _streamName = findNonExistentStreamName() + + // Create a stream. The number of shards determines the provisioned throughput. + logInfo(s"Creating stream ${_streamName}") + val createStreamRequest = new CreateStreamRequest() + createStreamRequest.setStreamName(_streamName) + createStreamRequest.setShardCount(streamShardCount) + kinesisClient.createStream(createStreamRequest) + + // The stream is now being created. Wait for it to become active. + waitForStreamToBeActive(_streamName) + streamCreated = true + logInfo(s"Created stream ${_streamName}") + } + + def getShards(): Seq[Shard] = { + kinesisClient.describeStream(_streamName).getStreamDescription.getShards.asScala + } + + def splitShard(shardId: String): Unit = { + val splitShardRequest = new SplitShardRequest() + splitShardRequest.withStreamName(_streamName) + splitShardRequest.withShardToSplit(shardId) + // Set a half of the max hash value + splitShardRequest.withNewStartingHashKey("170141183460469231731687303715884105728") + kinesisClient.splitShard(splitShardRequest) + // Wait for the shards to become active + waitForStreamToBeActive(_streamName) + } + + def mergeShard(shardToMerge: String, adjacentShardToMerge: String): Unit = { + val mergeShardRequest = new MergeShardsRequest + mergeShardRequest.withStreamName(_streamName) + mergeShardRequest.withShardToMerge(shardToMerge) + mergeShardRequest.withAdjacentShardToMerge(adjacentShardToMerge) + kinesisClient.mergeShards(mergeShardRequest) + // Wait for the shards to become active + waitForStreamToBeActive(_streamName) + } + + /** + * Push data to Kinesis stream and return a map of + * shardId -> seq of (data, seq number) pushed to corresponding shard + */ + def pushData(testData: Seq[Int], aggregate: Boolean): Map[String, Seq[(Int, String)]] = { + require(streamCreated, "Stream not yet created, call createStream() to create one") + val producer = getProducer(aggregate) + val shardIdToSeqNumbers = producer.sendData(streamName, testData) + logInfo(s"Pushed $testData:\n\t ${shardIdToSeqNumbers.mkString("\n\t")}") + shardIdToSeqNumbers.toMap + } + + /** + * Expose a Python friendly API. + */ + def pushData(testData: java.util.List[Int]): Unit = { + pushData(testData.asScala, aggregate = false) + } + + def deleteStream(): Unit = { + try { + if (streamCreated) { + kinesisClient.deleteStream(streamName) + } + } catch { + case e: Exception => + logWarning(s"Could not delete stream $streamName") + } + } + + def deleteDynamoDBTable(tableName: String): Unit = { + try { + val table = dynamoDB.getTable(tableName) + table.delete() + table.waitForDelete() + } catch { + case e: Exception => + logWarning(s"Could not delete DynamoDB table $tableName") + } + } + + private def describeStream(streamNameToDescribe: String): Option[StreamDescription] = { + try { + val describeStreamRequest = new DescribeStreamRequest().withStreamName(streamNameToDescribe) + val desc = kinesisClient.describeStream(describeStreamRequest).getStreamDescription() + Some(desc) + } catch { + case rnfe: ResourceNotFoundException => + None + } + } + + private def findNonExistentStreamName(): String = { + var testStreamName: String = null + do { + Thread.sleep(TimeUnit.SECONDS.toMillis(describeStreamPollTimeSeconds)) + testStreamName = s"KinesisTestUtils-${math.abs(Random.nextLong())}" + } while (describeStream(testStreamName).nonEmpty) + testStreamName + } + + private def waitForStreamToBeActive(streamNameToWaitFor: String): Unit = { + val startTime = System.currentTimeMillis() + val endTime = startTime + TimeUnit.SECONDS.toMillis(createStreamTimeoutSeconds) + while (System.currentTimeMillis() < endTime) { + Thread.sleep(TimeUnit.SECONDS.toMillis(describeStreamPollTimeSeconds)) + describeStream(streamNameToWaitFor).foreach { description => + val streamStatus = description.getStreamStatus() + logDebug(s"\t- current state: $streamStatus\n") + if ("ACTIVE".equals(streamStatus)) { + return + } + } + } + require(false, s"Stream $streamName never became active") + } +} + +private[kinesis] object KinesisTestUtils { + + val envVarNameForEnablingTests = "ENABLE_KINESIS_TESTS" + val endVarNameForEndpoint = "KINESIS_TEST_ENDPOINT_URL" + val defaultEndpointUrl = "https://kinesis.us-west-2.amazonaws.com" + + def getRegionNameByEndpoint(endpoint: String): String = { + val uri = new java.net.URI(endpoint) + RegionUtils.getRegionsForService(AmazonKinesis.ENDPOINT_PREFIX) + .asScala + .find(_.getAvailableEndpoints.asScala.toSeq.contains(uri.getHost)) + .map(_.getName) + .getOrElse( + throw new IllegalArgumentException(s"Could not resolve region for endpoint: $endpoint")) + } + + lazy val shouldRunTests = { + val isEnvSet = sys.env.get(envVarNameForEnablingTests) == Some("1") + if (isEnvSet) { + // scalastyle:off println + // Print this so that they are easily visible on the console and not hidden in the log4j logs. + println( + s""" + |Kinesis tests that actually send data has been enabled by setting the environment + |variable $envVarNameForEnablingTests to 1. This will create Kinesis Streams and + |DynamoDB tables in AWS. Please be aware that this may incur some AWS costs. + |By default, the tests use the endpoint URL $defaultEndpointUrl to create Kinesis streams. + |To change this endpoint URL to a different region, you can set the environment variable + |$endVarNameForEndpoint to the desired endpoint URL + |(e.g. $endVarNameForEndpoint="https://kinesis.us-west-2.amazonaws.com"). + """.stripMargin) + // scalastyle:on println + } + isEnvSet + } + + lazy val endpointUrl = { + val url = sys.env.getOrElse(endVarNameForEndpoint, defaultEndpointUrl) + // scalastyle:off println + // Print this so that they are easily visible on the console and not hidden in the log4j logs. + println(s"Using endpoint URL $url for creating Kinesis streams for tests.") + // scalastyle:on println + url + } + + def isAWSCredentialsPresent: Boolean = { + Try { new DefaultAWSCredentialsProviderChain().getCredentials() }.isSuccess + } + + def getAWSCredentials(): AWSCredentials = { + assert(shouldRunTests, + "Kinesis test not enabled, should not attempt to get AWS credentials") + Try { new DefaultAWSCredentialsProviderChain().getCredentials() } match { + case Success(cred) => cred + case Failure(e) => + throw new Exception( + s""" + |Kinesis tests enabled using environment variable $envVarNameForEnablingTests + |but could not find AWS credentials. Please follow instructions in AWS documentation + |to set the credentials in your system such that the DefaultAWSCredentialsProviderChain + |can find the credentials. + """.stripMargin) + } + } +} + +/** A wrapper interface that will allow us to consolidate the code for synthetic data generation. */ +private[kinesis] trait KinesisDataGenerator { + /** Sends the data to Kinesis and returns the metadata for everything that has been sent. */ + def sendData(streamName: String, data: Seq[Int]): Map[String, Seq[(Int, String)]] +} + +private[kinesis] class SimpleDataGenerator( + client: AmazonKinesisClient) extends KinesisDataGenerator { + override def sendData(streamName: String, data: Seq[Int]): Map[String, Seq[(Int, String)]] = { + val shardIdToSeqNumbers = new mutable.HashMap[String, ArrayBuffer[(Int, String)]]() + data.foreach { num => + val str = num.toString + val data = ByteBuffer.wrap(str.getBytes(StandardCharsets.UTF_8)) + val putRecordRequest = new PutRecordRequest().withStreamName(streamName) + .withData(data) + .withPartitionKey(str) + + val putRecordResult = client.putRecord(putRecordRequest) + val shardId = putRecordResult.getShardId + val seqNumber = putRecordResult.getSequenceNumber() + val sentSeqNumbers = shardIdToSeqNumbers.getOrElseUpdate(shardId, + new ArrayBuffer[(Int, String)]()) + sentSeqNumbers += ((num, seqNumber)) + } + + shardIdToSeqNumbers.toMap + } +} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala similarity index 75% rename from extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala rename to external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala index 2849fd8a8210..1298463bfba1 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala @@ -24,19 +24,15 @@ import com.amazonaws.services.kinesis.model.Record import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Duration, StreamingContext} import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} import org.apache.spark.streaming.dstream.ReceiverInputDStream -import org.apache.spark.streaming.{Duration, StreamingContext} object KinesisUtils { /** * Create an input stream that pulls messages from a Kinesis stream. * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. * - * Note: The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain - * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain - * gets the AWS credentials. - * * @param ssc StreamingContext object * @param kinesisAppName Kinesis application name used by the Kinesis Client Library * (KCL) to update DynamoDB @@ -57,7 +53,12 @@ object KinesisUtils { * StorageLevel.MEMORY_AND_DISK_2 is recommended. * @param messageHandler A custom message handler that can generate a generic output from a * Kinesis `Record`, which contains both message data, and metadata. + * + * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain + * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain + * gets the AWS credentials. */ + @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") def createStream[T: ClassTag]( ssc: StreamingContext, kinesisAppName: String, @@ -73,7 +74,7 @@ object KinesisUtils { ssc.withNamedScope("kinesis stream") { new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName), initialPositionInStream, kinesisAppName, checkpointInterval, storageLevel, - cleanedHandler, None) + cleanedHandler, DefaultCredentials, None, None) } } @@ -81,10 +82,6 @@ object KinesisUtils { * Create an input stream that pulls messages from a Kinesis stream. * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. * - * Note: - * The given AWS credentials will get saved in DStream checkpoints if checkpointing - * is enabled. Make sure that your checkpoint directory is secure. - * * @param ssc StreamingContext object * @param kinesisAppName Kinesis application name used by the Kinesis Client Library * (KCL) to update DynamoDB @@ -107,8 +104,12 @@ object KinesisUtils { * Kinesis `Record`, which contains both message data, and metadata. * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain) * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain) + * + * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing + * is enabled. Make sure that your checkpoint directory is secure. */ // scalastyle:off + @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") def createStream[T: ClassTag]( ssc: StreamingContext, kinesisAppName: String, @@ -124,9 +125,12 @@ object KinesisUtils { // scalastyle:on val cleanedHandler = ssc.sc.clean(messageHandler) ssc.withNamedScope("kinesis stream") { + val kinesisCredsProvider = BasicCredentials( + awsAccessKeyId = awsAccessKeyId, + awsSecretKey = awsSecretKey) new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName), initialPositionInStream, kinesisAppName, checkpointInterval, storageLevel, - cleanedHandler, Some(SerializableAWSCredentials(awsAccessKeyId, awsSecretKey))) + cleanedHandler, kinesisCredsProvider, None, None) } } @@ -134,10 +138,6 @@ object KinesisUtils { * Create an input stream that pulls messages from a Kinesis stream. * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. * - * Note: The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain - * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain - * gets the AWS credentials. - * * @param ssc StreamingContext object * @param kinesisAppName Kinesis application name used by the Kinesis Client Library * (KCL) to update DynamoDB @@ -156,8 +156,23 @@ object KinesisUtils { * details on the different types of checkpoints. * @param storageLevel Storage level to use for storing the received objects. * StorageLevel.MEMORY_AND_DISK_2 is recommended. + * @param messageHandler A custom message handler that can generate a generic output from a + * Kinesis `Record`, which contains both message data, and metadata. + * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain) + * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain) + * @param stsAssumeRoleArn ARN of IAM role to assume when using STS sessions to read from + * Kinesis stream. + * @param stsSessionName Name to uniquely identify STS sessions if multiple princples assume + * the same role. + * @param stsExternalId External ID that can be used to validate against the assumed IAM role's + * trust policy. + * + * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing + * is enabled. Make sure that your checkpoint directory is secure. */ - def createStream( + // scalastyle:off + @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") + def createStream[T: ClassTag]( ssc: StreamingContext, kinesisAppName: String, streamName: String, @@ -165,12 +180,26 @@ object KinesisUtils { regionName: String, initialPositionInStream: InitialPositionInStream, checkpointInterval: Duration, - storageLevel: StorageLevel): ReceiverInputDStream[Array[Byte]] = { - // Setting scope to override receiver stream's scope of "receiver stream" + storageLevel: StorageLevel, + messageHandler: Record => T, + awsAccessKeyId: String, + awsSecretKey: String, + stsAssumeRoleArn: String, + stsSessionName: String, + stsExternalId: String): ReceiverInputDStream[T] = { + // scalastyle:on + val cleanedHandler = ssc.sc.clean(messageHandler) ssc.withNamedScope("kinesis stream") { - new KinesisInputDStream[Array[Byte]](ssc, streamName, endpointUrl, validateRegion(regionName), + val kinesisCredsProvider = STSCredentials( + stsRoleArn = stsAssumeRoleArn, + stsSessionName = stsSessionName, + stsExternalId = Option(stsExternalId), + longLivedCreds = BasicCredentials( + awsAccessKeyId = awsAccessKeyId, + awsSecretKey = awsSecretKey)) + new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName), initialPositionInStream, kinesisAppName, checkpointInterval, storageLevel, - defaultMessageHandler, None) + cleanedHandler, kinesisCredsProvider, None, None) } } @@ -178,10 +207,6 @@ object KinesisUtils { * Create an input stream that pulls messages from a Kinesis stream. * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. * - * Note: - * The given AWS credentials will get saved in DStream checkpoints if checkpointing - * is enabled. Make sure that your checkpoint directory is secure. - * * @param ssc StreamingContext object * @param kinesisAppName Kinesis application name used by the Kinesis Client Library * (KCL) to update DynamoDB @@ -200,9 +225,12 @@ object KinesisUtils { * details on the different types of checkpoints. * @param storageLevel Storage level to use for storing the received objects. * StorageLevel.MEMORY_AND_DISK_2 is recommended. - * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain) - * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain) + * + * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain + * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain + * gets the AWS credentials. */ + @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") def createStream( ssc: StreamingContext, kinesisAppName: String, @@ -211,13 +239,12 @@ object KinesisUtils { regionName: String, initialPositionInStream: InitialPositionInStream, checkpointInterval: Duration, - storageLevel: StorageLevel, - awsAccessKeyId: String, - awsSecretKey: String): ReceiverInputDStream[Array[Byte]] = { + storageLevel: StorageLevel): ReceiverInputDStream[Array[Byte]] = { + // Setting scope to override receiver stream's scope of "receiver stream" ssc.withNamedScope("kinesis stream") { new KinesisInputDStream[Array[Byte]](ssc, streamName, endpointUrl, validateRegion(regionName), initialPositionInStream, kinesisAppName, checkpointInterval, storageLevel, - defaultMessageHandler, Some(SerializableAWSCredentials(awsAccessKeyId, awsSecretKey))) + KinesisInputDStream.defaultMessageHandler, DefaultCredentials, None, None) } } @@ -225,43 +252,49 @@ object KinesisUtils { * Create an input stream that pulls messages from a Kinesis stream. * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. * - * Note: - * - The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain - * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain - * gets AWS credentials. - * - The region of the `endpointUrl` will be used for DynamoDB and CloudWatch. - * - The Kinesis application name used by the Kinesis Client Library (KCL) will be the app name in - * [[org.apache.spark.SparkConf]]. - * * @param ssc StreamingContext object + * @param kinesisAppName Kinesis application name used by the Kinesis Client Library + * (KCL) to update DynamoDB * @param streamName Kinesis stream name - * @param endpointUrl Endpoint url of Kinesis service - * (e.g., https://kinesis.us-east-1.amazonaws.com) - * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. - * See the Kinesis Spark Streaming documentation for more - * details on the different types of checkpoints. + * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) + * @param regionName Name of region used by the Kinesis Client Library (KCL) to update + * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics) * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the * worker's initial starting position in the stream. * The values are either the beginning of the stream * per Kinesis' limit of 24 hours * (InitialPositionInStream.TRIM_HORIZON) or * the tip of the stream (InitialPositionInStream.LATEST). - * @param storageLevel Storage level to use for storing the received objects + * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. + * See the Kinesis Spark Streaming documentation for more + * details on the different types of checkpoints. + * @param storageLevel Storage level to use for storing the received objects. * StorageLevel.MEMORY_AND_DISK_2 is recommended. + * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain) + * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain) + * + * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing + * is enabled. Make sure that your checkpoint directory is secure. */ - @deprecated("use other forms of createStream", "1.4.0") + @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") def createStream( ssc: StreamingContext, + kinesisAppName: String, streamName: String, endpointUrl: String, - checkpointInterval: Duration, + regionName: String, initialPositionInStream: InitialPositionInStream, - storageLevel: StorageLevel - ): ReceiverInputDStream[Array[Byte]] = { + checkpointInterval: Duration, + storageLevel: StorageLevel, + awsAccessKeyId: String, + awsSecretKey: String): ReceiverInputDStream[Array[Byte]] = { ssc.withNamedScope("kinesis stream") { - new KinesisInputDStream[Array[Byte]](ssc, streamName, endpointUrl, - getRegionByEndpoint(endpointUrl), initialPositionInStream, ssc.sc.appName, - checkpointInterval, storageLevel, defaultMessageHandler, None) + val kinesisCredsProvider = BasicCredentials( + awsAccessKeyId = awsAccessKeyId, + awsSecretKey = awsSecretKey) + new KinesisInputDStream[Array[Byte]](ssc, streamName, endpointUrl, validateRegion(regionName), + initialPositionInStream, kinesisAppName, checkpointInterval, storageLevel, + KinesisInputDStream.defaultMessageHandler, kinesisCredsProvider, None, None) } } @@ -269,10 +302,6 @@ object KinesisUtils { * Create an input stream that pulls messages from a Kinesis stream. * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. * - * Note: The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain - * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain - * gets the AWS credentials. - * * @param jssc Java StreamingContext object * @param kinesisAppName Kinesis application name used by the Kinesis Client Library * (KCL) to update DynamoDB @@ -294,7 +323,12 @@ object KinesisUtils { * @param messageHandler A custom message handler that can generate a generic output from a * Kinesis `Record`, which contains both message data, and metadata. * @param recordClass Class of the records in DStream + * + * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain + * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain + * gets the AWS credentials. */ + @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") def createStream[T]( jssc: JavaStreamingContext, kinesisAppName: String, @@ -316,10 +350,6 @@ object KinesisUtils { * Create an input stream that pulls messages from a Kinesis stream. * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. * - * Note: - * The given AWS credentials will get saved in DStream checkpoints if checkpointing - * is enabled. Make sure that your checkpoint directory is secure. - * * @param jssc Java StreamingContext object * @param kinesisAppName Kinesis application name used by the Kinesis Client Library * (KCL) to update DynamoDB @@ -343,8 +373,12 @@ object KinesisUtils { * @param recordClass Class of the records in DStream * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain) * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain) + * + * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing + * is enabled. Make sure that your checkpoint directory is secure. */ // scalastyle:off + @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") def createStream[T]( jssc: JavaStreamingContext, kinesisAppName: String, @@ -370,10 +404,6 @@ object KinesisUtils { * Create an input stream that pulls messages from a Kinesis stream. * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. * - * Note: The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain - * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain - * gets the AWS credentials. - * * @param jssc Java StreamingContext object * @param kinesisAppName Kinesis application name used by the Kinesis Client Library * (KCL) to update DynamoDB @@ -392,8 +422,24 @@ object KinesisUtils { * details on the different types of checkpoints. * @param storageLevel Storage level to use for storing the received objects. * StorageLevel.MEMORY_AND_DISK_2 is recommended. + * @param messageHandler A custom message handler that can generate a generic output from a + * Kinesis `Record`, which contains both message data, and metadata. + * @param recordClass Class of the records in DStream + * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain) + * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain) + * @param stsAssumeRoleArn ARN of IAM role to assume when using STS sessions to read from + * Kinesis stream. + * @param stsSessionName Name to uniquely identify STS sessions if multiple princples assume + * the same role. + * @param stsExternalId External ID that can be used to validate against the assumed IAM role's + * trust policy. + * + * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing + * is enabled. Make sure that your checkpoint directory is secure. */ - def createStream( + // scalastyle:off + @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") + def createStream[T]( jssc: JavaStreamingContext, kinesisAppName: String, streamName: String, @@ -401,20 +447,26 @@ object KinesisUtils { regionName: String, initialPositionInStream: InitialPositionInStream, checkpointInterval: Duration, - storageLevel: StorageLevel - ): JavaReceiverInputDStream[Array[Byte]] = { - createStream[Array[Byte]](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName, - initialPositionInStream, checkpointInterval, storageLevel, defaultMessageHandler(_)) + storageLevel: StorageLevel, + messageHandler: JFunction[Record, T], + recordClass: Class[T], + awsAccessKeyId: String, + awsSecretKey: String, + stsAssumeRoleArn: String, + stsSessionName: String, + stsExternalId: String): JavaReceiverInputDStream[T] = { + // scalastyle:on + implicit val recordCmt: ClassTag[T] = ClassTag(recordClass) + val cleanedHandler = jssc.sparkContext.clean(messageHandler.call(_)) + createStream[T](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName, + initialPositionInStream, checkpointInterval, storageLevel, cleanedHandler, + awsAccessKeyId, awsSecretKey, stsAssumeRoleArn, stsSessionName, stsExternalId) } /** * Create an input stream that pulls messages from a Kinesis stream. * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. * - * Note: - * The given AWS credentials will get saved in DStream checkpoints if checkpointing - * is enabled. Make sure that your checkpoint directory is secure. - * * @param jssc Java StreamingContext object * @param kinesisAppName Kinesis application name used by the Kinesis Client Library * (KCL) to update DynamoDB @@ -433,9 +485,12 @@ object KinesisUtils { * details on the different types of checkpoints. * @param storageLevel Storage level to use for storing the received objects. * StorageLevel.MEMORY_AND_DISK_2 is recommended. - * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain) - * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain) + * + * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain + * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain + * gets the AWS credentials. */ + @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") def createStream( jssc: JavaStreamingContext, kinesisAppName: String, @@ -444,57 +499,56 @@ object KinesisUtils { regionName: String, initialPositionInStream: InitialPositionInStream, checkpointInterval: Duration, - storageLevel: StorageLevel, - awsAccessKeyId: String, - awsSecretKey: String): JavaReceiverInputDStream[Array[Byte]] = { + storageLevel: StorageLevel + ): JavaReceiverInputDStream[Array[Byte]] = { createStream[Array[Byte]](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName, initialPositionInStream, checkpointInterval, storageLevel, - defaultMessageHandler(_), awsAccessKeyId, awsSecretKey) + KinesisInputDStream.defaultMessageHandler(_)) } /** * Create an input stream that pulls messages from a Kinesis stream. * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis. * - * Note: - * - The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain - * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain - * gets AWS credentials. - * - The region of the `endpointUrl` will be used for DynamoDB and CloudWatch. - * - The Kinesis application name used by the Kinesis Client Library (KCL) will be the app name in - * [[org.apache.spark.SparkConf]]. - * * @param jssc Java StreamingContext object + * @param kinesisAppName Kinesis application name used by the Kinesis Client Library + * (KCL) to update DynamoDB * @param streamName Kinesis stream name - * @param endpointUrl Endpoint url of Kinesis service - * (e.g., https://kinesis.us-east-1.amazonaws.com) - * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. - * See the Kinesis Spark Streaming documentation for more - * details on the different types of checkpoints. + * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) + * @param regionName Name of region used by the Kinesis Client Library (KCL) to update + * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics) * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the * worker's initial starting position in the stream. * The values are either the beginning of the stream * per Kinesis' limit of 24 hours * (InitialPositionInStream.TRIM_HORIZON) or * the tip of the stream (InitialPositionInStream.LATEST). - * @param storageLevel Storage level to use for storing the received objects + * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. + * See the Kinesis Spark Streaming documentation for more + * details on the different types of checkpoints. + * @param storageLevel Storage level to use for storing the received objects. * StorageLevel.MEMORY_AND_DISK_2 is recommended. + * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain) + * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain) + * + * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing + * is enabled. Make sure that your checkpoint directory is secure. */ - @deprecated("use other forms of createStream", "1.4.0") + @deprecated("Use KinesisInputDStream.builder instead", "2.2.0") def createStream( jssc: JavaStreamingContext, + kinesisAppName: String, streamName: String, endpointUrl: String, - checkpointInterval: Duration, + regionName: String, initialPositionInStream: InitialPositionInStream, - storageLevel: StorageLevel - ): JavaReceiverInputDStream[Array[Byte]] = { - createStream( - jssc.ssc, streamName, endpointUrl, checkpointInterval, initialPositionInStream, storageLevel) - } - - private def getRegionByEndpoint(endpointUrl: String): String = { - RegionUtils.getRegionByEndpoint(endpointUrl).getName() + checkpointInterval: Duration, + storageLevel: StorageLevel, + awsAccessKeyId: String, + awsSecretKey: String): JavaReceiverInputDStream[Array[Byte]] = { + createStream[Array[Byte]](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName, + initialPositionInStream, checkpointInterval, storageLevel, + KinesisInputDStream.defaultMessageHandler(_), awsAccessKeyId, awsSecretKey) } private def validateRegion(regionName: String): String = { @@ -502,14 +556,6 @@ object KinesisUtils { throw new IllegalArgumentException(s"Region name '$regionName' is not valid") } } - - private[kinesis] def defaultMessageHandler(record: Record): Array[Byte] = { - if (record == null) return null - val byteBuffer = record.getData() - val byteArray = new Array[Byte](byteBuffer.remaining()) - byteBuffer.get(byteArray) - byteArray - } } /** @@ -528,6 +574,7 @@ private class KinesisUtilsPythonHelper { } } + // scalastyle:off def createStream( jssc: JavaStreamingContext, kinesisAppName: String, @@ -538,22 +585,43 @@ private class KinesisUtilsPythonHelper { checkpointInterval: Duration, storageLevel: StorageLevel, awsAccessKeyId: String, - awsSecretKey: String - ): JavaReceiverInputDStream[Array[Byte]] = { + awsSecretKey: String, + stsAssumeRoleArn: String, + stsSessionName: String, + stsExternalId: String): JavaReceiverInputDStream[Array[Byte]] = { + // scalastyle:on + if (!(stsAssumeRoleArn != null && stsSessionName != null && stsExternalId != null) + && !(stsAssumeRoleArn == null && stsSessionName == null && stsExternalId == null)) { + throw new IllegalArgumentException("stsAssumeRoleArn, stsSessionName, and stsExtenalId " + + "must all be defined or all be null") + } + + if (stsAssumeRoleArn != null && stsSessionName != null && stsExternalId != null) { + validateAwsCreds(awsAccessKeyId, awsSecretKey) + KinesisUtils.createStream(jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName, + getInitialPositionInStream(initialPositionInStream), checkpointInterval, storageLevel, + KinesisInputDStream.defaultMessageHandler(_), awsAccessKeyId, awsSecretKey, + stsAssumeRoleArn, stsSessionName, stsExternalId) + } else { + validateAwsCreds(awsAccessKeyId, awsSecretKey) + if (awsAccessKeyId == null && awsSecretKey == null) { + KinesisUtils.createStream(jssc, kinesisAppName, streamName, endpointUrl, regionName, + getInitialPositionInStream(initialPositionInStream), checkpointInterval, storageLevel) + } else { + KinesisUtils.createStream(jssc, kinesisAppName, streamName, endpointUrl, regionName, + getInitialPositionInStream(initialPositionInStream), checkpointInterval, storageLevel, + awsAccessKeyId, awsSecretKey) + } + } + } + + // Throw IllegalArgumentException unless both values are null or neither are. + private def validateAwsCreds(awsAccessKeyId: String, awsSecretKey: String) { if (awsAccessKeyId == null && awsSecretKey != null) { throw new IllegalArgumentException("awsSecretKey is set but awsAccessKeyId is null") } if (awsAccessKeyId != null && awsSecretKey == null) { throw new IllegalArgumentException("awsAccessKeyId is set but awsSecretKey is null") } - if (awsAccessKeyId == null && awsSecretKey == null) { - KinesisUtils.createStream(jssc, kinesisAppName, streamName, endpointUrl, regionName, - getInitialPositionInStream(initialPositionInStream), checkpointInterval, storageLevel) - } else { - KinesisUtils.createStream(jssc, kinesisAppName, streamName, endpointUrl, regionName, - getInitialPositionInStream(initialPositionInStream), checkpointInterval, storageLevel, - awsAccessKeyId, awsSecretKey) - } } - } diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala new file mode 100644 index 000000000000..9facfe8ff2b0 --- /dev/null +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis + +import scala.collection.JavaConverters._ + +import com.amazonaws.auth._ + +import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.internal.Logging + +/** + * Serializable interface providing a method executors can call to obtain an + * AWSCredentialsProvider instance for authenticating to AWS services. + */ +private[kinesis] sealed trait SparkAWSCredentials extends Serializable { + /** + * Return an AWSCredentialProvider instance that can be used by the Kinesis Client + * Library to authenticate to AWS services (Kinesis, CloudWatch and DynamoDB). + */ + def provider: AWSCredentialsProvider +} + +/** Returns DefaultAWSCredentialsProviderChain for authentication. */ +private[kinesis] final case object DefaultCredentials extends SparkAWSCredentials { + + def provider: AWSCredentialsProvider = new DefaultAWSCredentialsProviderChain +} + +/** + * Returns AWSStaticCredentialsProvider constructed using basic AWS keypair. Falls back to using + * DefaultCredentialsProviderChain if unable to construct a AWSCredentialsProviderChain + * instance with the provided arguments (e.g. if they are null). + */ +private[kinesis] final case class BasicCredentials( + awsAccessKeyId: String, + awsSecretKey: String) extends SparkAWSCredentials with Logging { + + def provider: AWSCredentialsProvider = try { + new AWSStaticCredentialsProvider(new BasicAWSCredentials(awsAccessKeyId, awsSecretKey)) + } catch { + case e: IllegalArgumentException => + logWarning("Unable to construct AWSStaticCredentialsProvider with provided keypair; " + + "falling back to DefaultCredentialsProviderChain.", e) + new DefaultAWSCredentialsProviderChain + } +} + +/** + * Returns an STSAssumeRoleSessionCredentialsProvider instance which assumes an IAM + * role in order to authenticate against resources in an external account. + */ +private[kinesis] final case class STSCredentials( + stsRoleArn: String, + stsSessionName: String, + stsExternalId: Option[String] = None, + longLivedCreds: SparkAWSCredentials = DefaultCredentials) + extends SparkAWSCredentials { + + def provider: AWSCredentialsProvider = { + val builder = new STSAssumeRoleSessionCredentialsProvider.Builder(stsRoleArn, stsSessionName) + .withLongLivedCredentialsProvider(longLivedCreds.provider) + stsExternalId match { + case Some(stsExternalId) => + builder.withExternalId(stsExternalId) + .build() + case None => + builder.build() + } + } +} + +@InterfaceStability.Evolving +object SparkAWSCredentials { + /** + * Builder for [[SparkAWSCredentials]] instances. + * + * @since 2.2.0 + */ + @InterfaceStability.Evolving + class Builder { + private var basicCreds: Option[BasicCredentials] = None + private var stsCreds: Option[STSCredentials] = None + + // scalastyle:off + /** + * Use a basic AWS keypair for long-lived authorization. + * + * @note The given AWS keypair will be saved in DStream checkpoints if checkpointing is + * enabled. Make sure that your checkpoint directory is secure. Prefer using the + * [[http://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default default provider chain]] + * instead if possible. + * + * @param accessKeyId AWS access key ID + * @param secretKey AWS secret key + * @return Reference to this [[SparkAWSCredentials.Builder]] + */ + // scalastyle:on + def basicCredentials(accessKeyId: String, secretKey: String): Builder = { + basicCreds = Option(BasicCredentials( + awsAccessKeyId = accessKeyId, + awsSecretKey = secretKey)) + this + } + + /** + * Use STS to assume an IAM role for temporary session-based authentication. Will use configured + * long-lived credentials for authorizing to STS itself (either the default provider chain + * or a configured keypair). + * + * @param roleArn ARN of IAM role to assume via STS + * @param sessionName Name to use for the STS session + * @return Reference to this [[SparkAWSCredentials.Builder]] + */ + def stsCredentials(roleArn: String, sessionName: String): Builder = { + stsCreds = Option(STSCredentials(stsRoleArn = roleArn, stsSessionName = sessionName)) + this + } + + /** + * Use STS to assume an IAM role for temporary session-based authentication. Will use configured + * long-lived credentials for authorizing to STS itself (either the default provider chain + * or a configured keypair). STS will validate the provided external ID with the one defined + * in the trust policy of the IAM role to be assumed (if one is present). + * + * @param roleArn ARN of IAM role to assume via STS + * @param sessionName Name to use for the STS session + * @param externalId External ID to validate against assumed IAM role's trust policy + * @return Reference to this [[SparkAWSCredentials.Builder]] + */ + def stsCredentials(roleArn: String, sessionName: String, externalId: String): Builder = { + stsCreds = Option(STSCredentials( + stsRoleArn = roleArn, + stsSessionName = sessionName, + stsExternalId = Option(externalId))) + this + } + + /** + * Returns the appropriate instance of [[SparkAWSCredentials]] given the configured + * parameters. + * + * - The long-lived credentials will either be [[DefaultCredentials]] or [[BasicCredentials]] + * if they were provided. + * + * - If STS credentials were provided, the configured long-lived credentials will be added to + * them and the result will be returned. + * + * - The long-lived credentials will be returned otherwise. + * + * @return [[SparkAWSCredentials]] to use for configured parameters + */ + def build(): SparkAWSCredentials = + stsCreds.map(_.copy(longLivedCreds = longLivedCreds)).getOrElse(longLivedCreds) + + private def longLivedCreds: SparkAWSCredentials = basicCreds.getOrElse(DefaultCredentials) + } + + /** + * Creates a [[SparkAWSCredentials.Builder]] for constructing + * [[SparkAWSCredentials]] instances. + * + * @since 2.2.0 + * + * @return [[SparkAWSCredentials.Builder]] instance + */ + def builder: Builder = new Builder +} diff --git a/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java b/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java new file mode 100644 index 000000000000..be6d549b1e42 --- /dev/null +++ b/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kinesis; + +import org.junit.Test; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.Seconds; +import org.apache.spark.streaming.LocalJavaStreamingContext; + +public class JavaKinesisInputDStreamBuilderSuite extends LocalJavaStreamingContext { + /** + * Basic test to ensure that the KinesisDStream.Builder interface is accessible from Java. + */ + @Test + public void testJavaKinesisDStreamBuilder() { + String streamName = "a-very-nice-stream-name"; + String endpointUrl = "https://kinesis.us-west-2.amazonaws.com"; + String region = "us-west-2"; + InitialPositionInStream initialPosition = InitialPositionInStream.TRIM_HORIZON; + String appName = "a-very-nice-kinesis-app"; + Duration checkpointInterval = Seconds.apply(30); + StorageLevel storageLevel = StorageLevel.MEMORY_ONLY(); + + KinesisInputDStream kinesisDStream = KinesisInputDStream.builder() + .streamingContext(ssc) + .streamName(streamName) + .endpointUrl(endpointUrl) + .regionName(region) + .initialPositionInStream(initialPosition) + .checkpointAppName(appName) + .checkpointInterval(checkpointInterval) + .storageLevel(storageLevel) + .build(); + assert(kinesisDStream.streamName() == streamName); + assert(kinesisDStream.endpointUrl() == endpointUrl); + assert(kinesisDStream.regionName() == region); + assert(kinesisDStream.initialPositionInStream() == initialPosition); + assert(kinesisDStream.checkpointAppName() == appName); + assert(kinesisDStream.checkpointInterval() == checkpointInterval); + assert(kinesisDStream._storageLevel() == storageLevel); + ssc.stop(); + } +} diff --git a/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java b/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java new file mode 100644 index 000000000000..b37b08746792 --- /dev/null +++ b/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kinesis; + +import com.amazonaws.services.kinesis.model.Record; +import org.junit.Test; + +import org.apache.spark.api.java.function.Function; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.LocalJavaStreamingContext; +import org.apache.spark.streaming.api.java.JavaDStream; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; + +/** + * Demonstrate the use of the KinesisUtils Java API + */ +public class JavaKinesisStreamSuite extends LocalJavaStreamingContext { + @Test + public void testKinesisStream() { + String dummyEndpointUrl = KinesisTestUtils.defaultEndpointUrl(); + String dummyRegionName = KinesisTestUtils.getRegionNameByEndpoint(dummyEndpointUrl); + + // Tests the API, does not actually test data receiving + JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream", + dummyEndpointUrl, dummyRegionName, InitialPositionInStream.LATEST, new Duration(2000), + StorageLevel.MEMORY_AND_DISK_2()); + ssc.stop(); + } + + @Test + public void testAwsCreds() { + String dummyEndpointUrl = KinesisTestUtils.defaultEndpointUrl(); + String dummyRegionName = KinesisTestUtils.getRegionNameByEndpoint(dummyEndpointUrl); + + // Tests the API, does not actually test data receiving + JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream", + dummyEndpointUrl, dummyRegionName, InitialPositionInStream.LATEST, new Duration(2000), + StorageLevel.MEMORY_AND_DISK_2(), "fakeAccessKey", "fakeSecretKey"); + ssc.stop(); + } + + private static Function handler = new Function() { + @Override + public String call(Record record) { + return record.getPartitionKey() + "-" + record.getSequenceNumber(); + } + }; + + @Test + public void testCustomHandler() { + // Tests the API, does not actually test data receiving + JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "testApp", "mySparkStream", + "https://kinesis.us-west-2.amazonaws.com", "us-west-2", InitialPositionInStream.LATEST, + new Duration(2000), StorageLevel.MEMORY_AND_DISK_2(), handler, String.class); + + ssc.stop(); + } + + @Test + public void testCustomHandlerAwsCreds() { + // Tests the API, does not actually test data receiving + JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "testApp", "mySparkStream", + "https://kinesis.us-west-2.amazonaws.com", "us-west-2", InitialPositionInStream.LATEST, + new Duration(2000), StorageLevel.MEMORY_AND_DISK_2(), handler, String.class, + "fakeAccessKey", "fakeSecretKey"); + + ssc.stop(); + } + + @Test + public void testCustomHandlerAwsStsCreds() { + // Tests the API, does not actually test data receiving + JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "testApp", "mySparkStream", + "https://kinesis.us-west-2.amazonaws.com", "us-west-2", InitialPositionInStream.LATEST, + new Duration(2000), StorageLevel.MEMORY_AND_DISK_2(), handler, String.class, + "fakeAccessKey", "fakeSecretKey", "fakeSTSRoleArn", "fakeSTSSessionName", + "fakeSTSExternalId"); + + ssc.stop(); + } +} diff --git a/external/kinesis-asl/src/test/resources/log4j.properties b/external/kinesis-asl/src/test/resources/log4j.properties new file mode 100644 index 000000000000..3706a6e36130 --- /dev/null +++ b/external/kinesis-asl/src/test/resources/log4j.properties @@ -0,0 +1,27 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.spark_project.jetty=WARN diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala new file mode 100644 index 000000000000..2ee3224b3c28 --- /dev/null +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis + +import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import com.amazonaws.services.kinesis.producer.{KinesisProducer => KPLProducer, KinesisProducerConfiguration, UserRecordResult} +import com.google.common.util.concurrent.{FutureCallback, Futures} + +private[kinesis] class KPLBasedKinesisTestUtils(streamShardCount: Int = 2) + extends KinesisTestUtils(streamShardCount) { + override protected def getProducer(aggregate: Boolean): KinesisDataGenerator = { + if (!aggregate) { + new SimpleDataGenerator(kinesisClient) + } else { + new KPLDataGenerator(regionName) + } + } +} + +/** A wrapper for the KinesisProducer provided in the KPL. */ +private[kinesis] class KPLDataGenerator(regionName: String) extends KinesisDataGenerator { + + private lazy val producer: KPLProducer = { + val conf = new KinesisProducerConfiguration() + .setRecordMaxBufferedTime(1000) + .setMaxConnections(1) + .setRegion(regionName) + .setMetricsLevel("none") + + new KPLProducer(conf) + } + + override def sendData(streamName: String, data: Seq[Int]): Map[String, Seq[(Int, String)]] = { + val shardIdToSeqNumbers = new mutable.HashMap[String, ArrayBuffer[(Int, String)]]() + data.foreach { num => + val str = num.toString + val data = ByteBuffer.wrap(str.getBytes(StandardCharsets.UTF_8)) + val future = producer.addUserRecord(streamName, str, data) + val kinesisCallBack = new FutureCallback[UserRecordResult]() { + override def onFailure(t: Throwable): Unit = {} // do nothing + + override def onSuccess(result: UserRecordResult): Unit = { + val shardId = result.getShardId + val seqNumber = result.getSequenceNumber() + val sentSeqNumbers = shardIdToSeqNumbers.getOrElseUpdate(shardId, + new ArrayBuffer[(Int, String)]()) + sentSeqNumbers += ((num, seqNumber)) + } + } + Futures.addCallback(future, kinesisCallBack) + } + producer.flushSync() + shardIdToSeqNumbers.toMap + } +} diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala similarity index 87% rename from extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala rename to external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala index 9f9e146a08d4..2c7b9c58e6fa 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala @@ -17,12 +17,13 @@ package org.apache.spark.streaming.kinesis -import org.scalatest.BeforeAndAfterAll +import org.scalatest.BeforeAndAfterEach +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException} import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId} -import org.apache.spark.{SparkConf, SparkContext, SparkException} -class KinesisBackedBlockRDDSuite extends KinesisFunSuite with BeforeAndAfterAll { +abstract class KinesisBackedBlockRDDTests(aggregateTestData: Boolean) + extends KinesisFunSuite with BeforeAndAfterEach with LocalSparkContext { private val testData = 1 to 8 @@ -34,16 +35,15 @@ class KinesisBackedBlockRDDSuite extends KinesisFunSuite with BeforeAndAfterAll private var shardIdToRange: Map[String, SequenceNumberRange] = null private var allRanges: Seq[SequenceNumberRange] = null - private var sc: SparkContext = null private var blockManager: BlockManager = null - override def beforeAll(): Unit = { + super.beforeAll() runIfTestsEnabled("Prepare KinesisTestUtils") { - testUtils = new KinesisTestUtils() + testUtils = new KPLBasedKinesisTestUtils() testUtils.createStream() - shardIdToDataAndSeqNumbers = testUtils.pushData(testData) + shardIdToDataAndSeqNumbers = testUtils.pushData(testData, aggregate = aggregateTestData) require(shardIdToDataAndSeqNumbers.size > 1, "Need data to be sent to multiple shards") shardIds = shardIdToDataAndSeqNumbers.keySet.toSeq @@ -51,23 +51,27 @@ class KinesisBackedBlockRDDSuite extends KinesisFunSuite with BeforeAndAfterAll shardIdToSeqNumbers = shardIdToDataAndSeqNumbers.mapValues { _.map { _._2 }} shardIdToRange = shardIdToSeqNumbers.map { case (shardId, seqNumbers) => val seqNumRange = SequenceNumberRange( - testUtils.streamName, shardId, seqNumbers.head, seqNumbers.last) + testUtils.streamName, shardId, seqNumbers.head, seqNumbers.last, seqNumbers.size) (shardId, seqNumRange) } allRanges = shardIdToRange.values.toSeq - - val conf = new SparkConf().setMaster("local[4]").setAppName("KinesisBackedBlockRDDSuite") - sc = new SparkContext(conf) - blockManager = sc.env.blockManager } } + override def beforeEach(): Unit = { + super.beforeEach() + val conf = new SparkConf().setMaster("local[4]").setAppName("KinesisBackedBlockRDDSuite") + sc = new SparkContext(conf) + blockManager = sc.env.blockManager + } + override def afterAll(): Unit = { - if (testUtils != null) { - testUtils.deleteStream() - } - if (sc != null) { - sc.stop() + try { + if (testUtils != null) { + testUtils.deleteStream() + } + } finally { + super.afterAll() } } @@ -118,14 +122,14 @@ class KinesisBackedBlockRDDSuite extends KinesisFunSuite with BeforeAndAfterAll testIsBlockValid = true) } - testIfEnabled("Test whether RDD is valid after removing blocks from block anager") { + testIfEnabled("Test whether RDD is valid after removing blocks from block manager") { testRDD(numPartitions = 2, numPartitionsInBM = 2, numPartitionsInKinesis = 2, testBlockRemove = true) } /** * Test the WriteAheadLogBackedRDD, by writing some partitions of the data to block manager - * and the rest to a write ahead log, and then reading reading it all back using the RDD. + * and the rest to a write ahead log, and then reading it all back using the RDD. * It can also test if the partitions that were read from the log were again stored in * block manager. * @@ -158,9 +162,9 @@ class KinesisBackedBlockRDDSuite extends KinesisFunSuite with BeforeAndAfterAll testBlockRemove: Boolean = false ): Unit = { require(shardIds.size > 1, "Need at least 2 shards to test") - require(numPartitionsInBM <= shardIds.size , + require(numPartitionsInBM <= shardIds.size, "Number of partitions in BlockManager cannot be more than the Kinesis test shards available") - require(numPartitionsInKinesis <= shardIds.size , + require(numPartitionsInKinesis <= shardIds.size, "Number of partitions in Kinesis cannot be more than the Kinesis test shards available") require(numPartitionsInBM <= numPartitions, "Number of partitions in BlockManager cannot be more than that in RDD") @@ -177,7 +181,7 @@ class KinesisBackedBlockRDDSuite extends KinesisFunSuite with BeforeAndAfterAll // Create the necessary ranges to use in the RDD val fakeRanges = Array.fill(numPartitions - numPartitionsInKinesis)( - SequenceNumberRanges(SequenceNumberRange("fakeStream", "fakeShardId", "xxx", "yyy"))) + SequenceNumberRanges(SequenceNumberRange("fakeStream", "fakeShardId", "xxx", "yyy", 1))) val realRanges = Array.tabulate(numPartitionsInKinesis) { i => val range = shardIdToRange(shardIds(i + (numPartitions - numPartitionsInKinesis))) SequenceNumberRanges(Array(range)) @@ -217,7 +221,7 @@ class KinesisBackedBlockRDDSuite extends KinesisFunSuite with BeforeAndAfterAll assert(collectedData.toSet === testData.toSet) // Verify that the block fetching is skipped when isBlockValid is set to false. - // This is done by using a RDD whose data is only in memory but is set to skip block fetching + // This is done by using an RDD whose data is only in memory but is set to skip block fetching // Using that RDD will throw exception, as it skips block fetching even if the blocks are in // in BlockManager. if (testIsBlockValid) { @@ -247,3 +251,9 @@ class KinesisBackedBlockRDDSuite extends KinesisFunSuite with BeforeAndAfterAll Array.tabulate(num) { i => new StreamBlockId(0, i) } } } + +class WithAggregationKinesisBackedBlockRDDSuite + extends KinesisBackedBlockRDDTests(aggregateTestData = true) + +class WithoutAggregationKinesisBackedBlockRDDSuite + extends KinesisBackedBlockRDDTests(aggregateTestData = false) diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala new file mode 100644 index 000000000000..e26f4477d1d7 --- /dev/null +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis + +import java.util.concurrent.TimeoutException + +import scala.concurrent.{Await, ExecutionContext, Future} +import scala.concurrent.duration._ +import scala.language.postfixOps + +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer +import org.mockito.Matchers._ +import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} +import org.scalatest.concurrent.Eventually +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark.streaming.{Duration, TestSuiteBase} +import org.apache.spark.util.ManualClock + +class KinesisCheckpointerSuite extends TestSuiteBase + with MockitoSugar + with BeforeAndAfterEach + with PrivateMethodTester + with Eventually { + + private val workerId = "dummyWorkerId" + private val shardId = "dummyShardId" + private val seqNum = "123" + private val otherSeqNum = "245" + private val checkpointInterval = Duration(10) + private val someSeqNum = Some(seqNum) + private val someOtherSeqNum = Some(otherSeqNum) + + private var receiverMock: KinesisReceiver[Array[Byte]] = _ + private var checkpointerMock: IRecordProcessorCheckpointer = _ + private var kinesisCheckpointer: KinesisCheckpointer = _ + private var clock: ManualClock = _ + + private val checkpoint = PrivateMethod[Unit]('checkpoint) + + override def beforeEach(): Unit = { + receiverMock = mock[KinesisReceiver[Array[Byte]]] + checkpointerMock = mock[IRecordProcessorCheckpointer] + clock = new ManualClock() + kinesisCheckpointer = new KinesisCheckpointer(receiverMock, checkpointInterval, workerId, clock) + } + + test("checkpoint is not called twice for the same sequence number") { + when(receiverMock.getLatestSeqNumToCheckpoint(shardId)).thenReturn(someSeqNum) + kinesisCheckpointer.invokePrivate(checkpoint(shardId, checkpointerMock)) + kinesisCheckpointer.invokePrivate(checkpoint(shardId, checkpointerMock)) + + verify(checkpointerMock, times(1)).checkpoint(anyString()) + } + + test("checkpoint is called after sequence number increases") { + when(receiverMock.getLatestSeqNumToCheckpoint(shardId)) + .thenReturn(someSeqNum).thenReturn(someOtherSeqNum) + kinesisCheckpointer.invokePrivate(checkpoint(shardId, checkpointerMock)) + kinesisCheckpointer.invokePrivate(checkpoint(shardId, checkpointerMock)) + + verify(checkpointerMock, times(1)).checkpoint(seqNum) + verify(checkpointerMock, times(1)).checkpoint(otherSeqNum) + } + + test("should checkpoint if we have exceeded the checkpoint interval") { + when(receiverMock.getLatestSeqNumToCheckpoint(shardId)) + .thenReturn(someSeqNum).thenReturn(someOtherSeqNum) + + kinesisCheckpointer.setCheckpointer(shardId, checkpointerMock) + clock.advance(5 * checkpointInterval.milliseconds) + + eventually(timeout(1 second)) { + verify(checkpointerMock, times(1)).checkpoint(seqNum) + verify(checkpointerMock, times(1)).checkpoint(otherSeqNum) + } + } + + test("shouldn't checkpoint if we have not exceeded the checkpoint interval") { + when(receiverMock.getLatestSeqNumToCheckpoint(shardId)).thenReturn(someSeqNum) + + kinesisCheckpointer.setCheckpointer(shardId, checkpointerMock) + clock.advance(checkpointInterval.milliseconds / 2) + + verify(checkpointerMock, never()).checkpoint(anyString()) + } + + test("should not checkpoint for the same sequence number") { + when(receiverMock.getLatestSeqNumToCheckpoint(shardId)).thenReturn(someSeqNum) + + kinesisCheckpointer.setCheckpointer(shardId, checkpointerMock) + + clock.advance(checkpointInterval.milliseconds * 5) + eventually(timeout(1 second)) { + verify(checkpointerMock, atMost(1)).checkpoint(anyString()) + } + } + + test("removing checkpointer checkpoints one last time") { + when(receiverMock.getLatestSeqNumToCheckpoint(shardId)).thenReturn(someSeqNum) + + kinesisCheckpointer.removeCheckpointer(shardId, checkpointerMock) + verify(checkpointerMock, times(1)).checkpoint() + } + + test("if checkpointing is going on, wait until finished before removing and checkpointing") { + when(receiverMock.getLatestSeqNumToCheckpoint(shardId)) + .thenReturn(someSeqNum).thenReturn(someOtherSeqNum) + when(checkpointerMock.checkpoint(anyString)).thenAnswer(new Answer[Unit] { + override def answer(invocations: InvocationOnMock): Unit = { + clock.waitTillTime(clock.getTimeMillis() + checkpointInterval.milliseconds / 2) + } + }) + + kinesisCheckpointer.setCheckpointer(shardId, checkpointerMock) + clock.advance(checkpointInterval.milliseconds) + eventually(timeout(1 second)) { + verify(checkpointerMock, times(1)).checkpoint(anyString()) + } + // don't block test thread + val f = Future(kinesisCheckpointer.removeCheckpointer(shardId, checkpointerMock))( + ExecutionContext.global) + + intercept[TimeoutException] { + // scalastyle:off awaitready + Await.ready(f, 50 millis) + // scalastyle:on awaitready + } + + clock.advance(checkpointInterval.milliseconds / 2) + eventually(timeout(1 second)) { + verify(checkpointerMock, times(1)).checkpoint(anyString) + verify(checkpointerMock, times(1)).checkpoint() + } + } +} diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala similarity index 98% rename from extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala rename to external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala index ee428f31d6ce..1c81298a7c20 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisFunSuite.scala @@ -40,7 +40,7 @@ trait KinesisFunSuite extends SparkFunSuite { if (shouldRunTests) { body } else { - ignore(s"$message [enable by setting env var $envVarNameForEnablingTests=1]")() + ignore(s"$message [enable by setting env var $envVarNameForEnablingTests=1]")(()) } } } diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala new file mode 100644 index 000000000000..afa1a7f8ca66 --- /dev/null +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream +import org.scalatest.BeforeAndAfterEach +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Seconds, StreamingContext, TestSuiteBase} + +class KinesisInputDStreamBuilderSuite extends TestSuiteBase with BeforeAndAfterEach + with MockitoSugar { + import KinesisInputDStream._ + + private val ssc = new StreamingContext(conf, batchDuration) + private val streamName = "a-very-nice-kinesis-stream-name" + private val checkpointAppName = "a-very-nice-kcl-app-name" + private def baseBuilder = KinesisInputDStream.builder + private def builder = baseBuilder.streamingContext(ssc) + .streamName(streamName) + .checkpointAppName(checkpointAppName) + + override def afterAll(): Unit = { + ssc.stop() + } + + test("should raise an exception if the StreamingContext is missing") { + intercept[IllegalArgumentException] { + baseBuilder.streamName(streamName).checkpointAppName(checkpointAppName).build() + } + } + + test("should raise an exception if the stream name is missing") { + intercept[IllegalArgumentException] { + baseBuilder.streamingContext(ssc).checkpointAppName(checkpointAppName).build() + } + } + + test("should raise an exception if the checkpoint app name is missing") { + intercept[IllegalArgumentException] { + baseBuilder.streamingContext(ssc).streamName(streamName).build() + } + } + + test("should propagate required values to KinesisInputDStream") { + val dstream = builder.build() + assert(dstream.context == ssc) + assert(dstream.streamName == streamName) + assert(dstream.checkpointAppName == checkpointAppName) + } + + test("should propagate default values to KinesisInputDStream") { + val dstream = builder.build() + assert(dstream.endpointUrl == DEFAULT_KINESIS_ENDPOINT_URL) + assert(dstream.regionName == DEFAULT_KINESIS_REGION_NAME) + assert(dstream.initialPositionInStream == DEFAULT_INITIAL_POSITION_IN_STREAM) + assert(dstream.checkpointInterval == batchDuration) + assert(dstream._storageLevel == DEFAULT_STORAGE_LEVEL) + assert(dstream.kinesisCreds == DefaultCredentials) + assert(dstream.dynamoDBCreds == None) + assert(dstream.cloudWatchCreds == None) + } + + test("should propagate custom non-auth values to KinesisInputDStream") { + val customEndpointUrl = "https://kinesis.us-west-2.amazonaws.com" + val customRegion = "us-west-2" + val customInitialPosition = InitialPositionInStream.TRIM_HORIZON + val customAppName = "a-very-nice-kinesis-app" + val customCheckpointInterval = Seconds(30) + val customStorageLevel = StorageLevel.MEMORY_ONLY + val customKinesisCreds = mock[SparkAWSCredentials] + val customDynamoDBCreds = mock[SparkAWSCredentials] + val customCloudWatchCreds = mock[SparkAWSCredentials] + + val dstream = builder + .endpointUrl(customEndpointUrl) + .regionName(customRegion) + .initialPositionInStream(customInitialPosition) + .checkpointAppName(customAppName) + .checkpointInterval(customCheckpointInterval) + .storageLevel(customStorageLevel) + .kinesisCredentials(customKinesisCreds) + .dynamoDBCredentials(customDynamoDBCreds) + .cloudWatchCredentials(customCloudWatchCreds) + .build() + assert(dstream.endpointUrl == customEndpointUrl) + assert(dstream.regionName == customRegion) + assert(dstream.initialPositionInStream == customInitialPosition) + assert(dstream.checkpointAppName == customAppName) + assert(dstream.checkpointInterval == customCheckpointInterval) + assert(dstream._storageLevel == customStorageLevel) + assert(dstream.kinesisCreds == customKinesisCreds) + assert(dstream.dynamoDBCreds == Option(customDynamoDBCreds)) + assert(dstream.cloudWatchCreds == Option(customCloudWatchCreds)) + } +} diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala new file mode 100644 index 000000000000..2fadda271ea2 --- /dev/null +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis + +import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets +import java.util.Arrays + +import com.amazonaws.services.kinesis.clientlibrary.exceptions._ +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason +import com.amazonaws.services.kinesis.model.Record +import org.mockito.Matchers._ +import org.mockito.Matchers.{eq => meq} +import org.mockito.Mockito._ +import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark.streaming.{Duration, TestSuiteBase} + +/** + * Suite of Kinesis streaming receiver tests focusing mostly on the KinesisRecordProcessor + */ +class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAfter + with MockitoSugar { + + val app = "TestKinesisReceiver" + val stream = "mySparkStream" + val endpoint = "endpoint-url" + val workerId = "dummyWorkerId" + val shardId = "dummyShardId" + val seqNum = "dummySeqNum" + val checkpointInterval = Duration(10) + val someSeqNum = Some(seqNum) + + val record1 = new Record() + record1.setData(ByteBuffer.wrap("Spark In Action".getBytes(StandardCharsets.UTF_8))) + val record2 = new Record() + record2.setData(ByteBuffer.wrap("Learning Spark".getBytes(StandardCharsets.UTF_8))) + val batch = Arrays.asList(record1, record2) + + var receiverMock: KinesisReceiver[Array[Byte]] = _ + var checkpointerMock: IRecordProcessorCheckpointer = _ + + override def beforeFunction(): Unit = { + receiverMock = mock[KinesisReceiver[Array[Byte]]] + checkpointerMock = mock[IRecordProcessorCheckpointer] + } + + test("process records including store and set checkpointer") { + when(receiverMock.isStopped()).thenReturn(false) + when(receiverMock.getCurrentLimit).thenReturn(Int.MaxValue) + + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId) + recordProcessor.initialize(shardId) + recordProcessor.processRecords(batch, checkpointerMock) + + verify(receiverMock, times(1)).isStopped() + verify(receiverMock, times(1)).addRecords(shardId, batch) + verify(receiverMock, times(1)).setCheckpointer(shardId, checkpointerMock) + } + + test("split into multiple processes if a limitation is set") { + when(receiverMock.isStopped()).thenReturn(false) + when(receiverMock.getCurrentLimit).thenReturn(1) + + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId) + recordProcessor.initialize(shardId) + recordProcessor.processRecords(batch, checkpointerMock) + + verify(receiverMock, times(1)).isStopped() + verify(receiverMock, times(1)).addRecords(shardId, batch.subList(0, 1)) + verify(receiverMock, times(1)).addRecords(shardId, batch.subList(1, 2)) + verify(receiverMock, times(1)).setCheckpointer(shardId, checkpointerMock) + } + + test("shouldn't store and update checkpointer when receiver is stopped") { + when(receiverMock.isStopped()).thenReturn(true) + when(receiverMock.getCurrentLimit).thenReturn(Int.MaxValue) + + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId) + recordProcessor.processRecords(batch, checkpointerMock) + + verify(receiverMock, times(1)).isStopped() + verify(receiverMock, never).addRecords(anyString, anyListOf(classOf[Record])) + verify(receiverMock, never).setCheckpointer(anyString, meq(checkpointerMock)) + } + + test("shouldn't update checkpointer when exception occurs during store") { + when(receiverMock.isStopped()).thenReturn(false) + when(receiverMock.getCurrentLimit).thenReturn(Int.MaxValue) + when( + receiverMock.addRecords(anyString, anyListOf(classOf[Record])) + ).thenThrow(new RuntimeException()) + + intercept[RuntimeException] { + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId) + recordProcessor.initialize(shardId) + recordProcessor.processRecords(batch, checkpointerMock) + } + + verify(receiverMock, times(1)).isStopped() + verify(receiverMock, times(1)).addRecords(shardId, batch) + verify(receiverMock, never).setCheckpointer(anyString, meq(checkpointerMock)) + } + + test("shutdown should checkpoint if the reason is TERMINATE") { + when(receiverMock.getLatestSeqNumToCheckpoint(shardId)).thenReturn(someSeqNum) + + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId) + recordProcessor.initialize(shardId) + recordProcessor.shutdown(checkpointerMock, ShutdownReason.TERMINATE) + + verify(receiverMock, times(1)).removeCheckpointer(meq(shardId), meq(checkpointerMock)) + } + + + test("shutdown should not checkpoint if the reason is something other than TERMINATE") { + when(receiverMock.getLatestSeqNumToCheckpoint(shardId)).thenReturn(someSeqNum) + + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId) + recordProcessor.initialize(shardId) + recordProcessor.shutdown(checkpointerMock, ShutdownReason.ZOMBIE) + recordProcessor.shutdown(checkpointerMock, null) + + verify(receiverMock, times(2)).removeCheckpointer(meq(shardId), + meq[IRecordProcessorCheckpointer](null)) + } + + test("retry success on first attempt") { + val expectedIsStopped = false + 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 + 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 + 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") { + 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") { + 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") { + 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 exhausting all retries") { + val expectedErrorMessage = "final try error message" + 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/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala new file mode 100644 index 000000000000..7e5bda923f63 --- /dev/null +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala @@ -0,0 +1,435 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis + +import scala.collection.mutable +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.util.Random + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream +import com.amazonaws.services.kinesis.model.Record +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.Matchers._ +import org.scalatest.concurrent.Eventually + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.{StorageLevel, StreamBlockId} +import org.apache.spark.streaming._ +import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.streaming.kinesis.KinesisReadConfigurations._ +import org.apache.spark.streaming.kinesis.KinesisTestUtils._ +import org.apache.spark.streaming.receiver.BlockManagerBasedStoreResult +import org.apache.spark.streaming.scheduler.ReceivedBlockInfo +import org.apache.spark.util.Utils + +abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFunSuite + with Eventually with BeforeAndAfter with BeforeAndAfterAll { + + // This is the name that KCL will use to save metadata to DynamoDB + private val appName = s"KinesisStreamSuite-${math.abs(Random.nextLong())}" + private val batchDuration = Seconds(1) + + // Dummy parameters for API testing + private val dummyEndpointUrl = defaultEndpointUrl + private val dummyRegionName = KinesisTestUtils.getRegionNameByEndpoint(dummyEndpointUrl) + private val dummyAWSAccessKey = "dummyAccessKey" + private val dummyAWSSecretKey = "dummySecretKey" + + private var testUtils: KinesisTestUtils = null + private var ssc: StreamingContext = null + private var sc: SparkContext = null + + override def beforeAll(): Unit = { + val conf = new SparkConf() + .setMaster("local[4]") + .setAppName("KinesisStreamSuite") // Setting Spark app name to Kinesis app name + sc = new SparkContext(conf) + + runIfTestsEnabled("Prepare KinesisTestUtils") { + testUtils = new KPLBasedKinesisTestUtils() + testUtils.createStream() + } + } + + override def afterAll(): Unit = { + if (ssc != null) { + ssc.stop() + } + if (sc != null) { + sc.stop() + } + if (testUtils != null) { + // Delete the Kinesis stream as well as the DynamoDB table generated by + // Kinesis Client Library when consuming the stream + testUtils.deleteStream() + testUtils.deleteDynamoDBTable(appName) + } + } + + before { + ssc = new StreamingContext(sc, batchDuration) + } + + after { + if (ssc != null) { + ssc.stop(stopSparkContext = false) + ssc = null + } + if (testUtils != null) { + testUtils.deleteDynamoDBTable(appName) + } + } + + test("KinesisUtils API") { + val kinesisStream1 = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream", + dummyEndpointUrl, dummyRegionName, + InitialPositionInStream.LATEST, Seconds(2), StorageLevel.MEMORY_AND_DISK_2) + val kinesisStream2 = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream", + dummyEndpointUrl, dummyRegionName, + InitialPositionInStream.LATEST, Seconds(2), StorageLevel.MEMORY_AND_DISK_2, + dummyAWSAccessKey, dummyAWSSecretKey) + } + + test("RDD generation") { + val inputStream = KinesisUtils.createStream(ssc, appName, "dummyStream", + dummyEndpointUrl, dummyRegionName, InitialPositionInStream.LATEST, Seconds(2), + StorageLevel.MEMORY_AND_DISK_2, dummyAWSAccessKey, dummyAWSSecretKey) + assert(inputStream.isInstanceOf[KinesisInputDStream[Array[Byte]]]) + + val kinesisStream = inputStream.asInstanceOf[KinesisInputDStream[Array[Byte]]] + val time = Time(1000) + + // Generate block info data for testing + val seqNumRanges1 = SequenceNumberRanges( + SequenceNumberRange("fakeStream", "fakeShardId", "xxx", "yyy", 67)) + val blockId1 = StreamBlockId(kinesisStream.id, 123) + val blockInfo1 = ReceivedBlockInfo( + 0, None, Some(seqNumRanges1), new BlockManagerBasedStoreResult(blockId1, None)) + + val seqNumRanges2 = SequenceNumberRanges( + SequenceNumberRange("fakeStream", "fakeShardId", "aaa", "bbb", 89)) + val blockId2 = StreamBlockId(kinesisStream.id, 345) + val blockInfo2 = ReceivedBlockInfo( + 0, None, Some(seqNumRanges2), new BlockManagerBasedStoreResult(blockId2, None)) + + // Verify that the generated KinesisBackedBlockRDD has the all the right information + val blockInfos = Seq(blockInfo1, blockInfo2) + val nonEmptyRDD = kinesisStream.createBlockRDD(time, blockInfos) + nonEmptyRDD shouldBe a [KinesisBackedBlockRDD[_]] + val kinesisRDD = nonEmptyRDD.asInstanceOf[KinesisBackedBlockRDD[_]] + assert(kinesisRDD.regionName === dummyRegionName) + assert(kinesisRDD.endpointUrl === dummyEndpointUrl) + assert(kinesisRDD.kinesisReadConfigs.retryTimeoutMs === batchDuration.milliseconds) + assert(kinesisRDD.kinesisCreds === BasicCredentials( + awsAccessKeyId = dummyAWSAccessKey, + awsSecretKey = dummyAWSSecretKey)) + assert(nonEmptyRDD.partitions.size === blockInfos.size) + nonEmptyRDD.partitions.foreach { _ shouldBe a [KinesisBackedBlockRDDPartition] } + val partitions = nonEmptyRDD.partitions.map { + _.asInstanceOf[KinesisBackedBlockRDDPartition] }.toSeq + assert(partitions.map { _.seqNumberRanges } === Seq(seqNumRanges1, seqNumRanges2)) + assert(partitions.map { _.blockId } === Seq(blockId1, blockId2)) + assert(partitions.forall { _.isBlockIdValid === true }) + + // Verify that KinesisBackedBlockRDD is generated even when there are no blocks + val emptyRDD = kinesisStream.createBlockRDD(time, Seq.empty) + // Verify it's KinesisBackedBlockRDD[_] rather than KinesisBackedBlockRDD[Array[Byte]], because + // the type parameter will be erased at runtime + emptyRDD shouldBe a [KinesisBackedBlockRDD[_]] + emptyRDD.partitions shouldBe empty + + // Verify that the KinesisBackedBlockRDD has isBlockValid = false when blocks are invalid + blockInfos.foreach { _.setBlockIdInvalid() } + kinesisStream.createBlockRDD(time, blockInfos).partitions.foreach { partition => + assert(partition.asInstanceOf[KinesisBackedBlockRDDPartition].isBlockIdValid === false) + } + } + + + /** + * Test the stream by sending data to a Kinesis stream and receiving from it. + * This test is not run by default as it requires AWS credentials that the test + * environment may not have. Even if there is AWS credentials available, the user + * may not want to run these tests to avoid the Kinesis costs. To enable this test, + * you must have AWS credentials available through the default AWS provider chain, + * and you have to set the system environment variable RUN_KINESIS_TESTS=1 . + */ + testIfEnabled("basic operation") { + val stream = KinesisInputDStream.builder.streamingContext(ssc) + .checkpointAppName(appName) + .streamName(testUtils.streamName) + .endpointUrl(testUtils.endpointUrl) + .regionName(testUtils.regionName) + .initialPositionInStream(InitialPositionInStream.LATEST) + .checkpointInterval(Seconds(10)) + .storageLevel(StorageLevel.MEMORY_ONLY) + .build() + + val collected = new mutable.HashSet[Int] + stream.map { bytes => new String(bytes).toInt }.foreachRDD { rdd => + collected.synchronized { + collected ++= rdd.collect() + logInfo("Collected = " + collected.mkString(", ")) + } + } + ssc.start() + + val testData = 1 to 10 + eventually(timeout(120 seconds), interval(10 second)) { + testUtils.pushData(testData, aggregateTestData) + assert(collected.synchronized { collected === testData.toSet }, + "\nData received does not match data sent") + } + ssc.stop(stopSparkContext = false) + } + + testIfEnabled("custom message handling") { + def addFive(r: Record): Int = JavaUtils.bytesToString(r.getData).toInt + 5 + + val stream = KinesisInputDStream.builder.streamingContext(ssc) + .checkpointAppName(appName) + .streamName(testUtils.streamName) + .endpointUrl(testUtils.endpointUrl) + .regionName(testUtils.regionName) + .initialPositionInStream(InitialPositionInStream.LATEST) + .checkpointInterval(Seconds(10)) + .storageLevel(StorageLevel.MEMORY_ONLY) + .buildWithMessageHandler(addFive(_)) + + stream shouldBe a [ReceiverInputDStream[_]] + + val collected = new mutable.HashSet[Int] + stream.foreachRDD { rdd => + collected.synchronized { + collected ++= rdd.collect() + logInfo("Collected = " + collected.mkString(", ")) + } + } + ssc.start() + + val testData = 1 to 10 + eventually(timeout(120 seconds), interval(10 second)) { + testUtils.pushData(testData, aggregateTestData) + val modData = testData.map(_ + 5) + assert(collected.synchronized { collected === modData.toSet }, + "\nData received does not match data sent") + } + ssc.stop(stopSparkContext = false) + } + + test("Kinesis read with custom configurations") { + try { + ssc.sc.conf.set(RETRY_WAIT_TIME_KEY, "2000ms") + ssc.sc.conf.set(RETRY_MAX_ATTEMPTS_KEY, "5") + + val kinesisStream = KinesisInputDStream.builder.streamingContext(ssc) + .checkpointAppName(appName) + .streamName("dummyStream") + .endpointUrl(dummyEndpointUrl) + .regionName(dummyRegionName) + .initialPositionInStream(InitialPositionInStream.LATEST) + .checkpointInterval(Seconds(10)) + .storageLevel(StorageLevel.MEMORY_ONLY) + .build() + .asInstanceOf[KinesisInputDStream[Array[Byte]]] + + val time = Time(1000) + // Generate block info data for testing + val seqNumRanges1 = SequenceNumberRanges( + SequenceNumberRange("fakeStream", "fakeShardId", "xxx", "yyy", 67)) + val blockId1 = StreamBlockId(kinesisStream.id, 123) + val blockInfo1 = ReceivedBlockInfo( + 0, None, Some(seqNumRanges1), new BlockManagerBasedStoreResult(blockId1, None)) + + val seqNumRanges2 = SequenceNumberRanges( + SequenceNumberRange("fakeStream", "fakeShardId", "aaa", "bbb", 89)) + val blockId2 = StreamBlockId(kinesisStream.id, 345) + val blockInfo2 = ReceivedBlockInfo( + 0, None, Some(seqNumRanges2), new BlockManagerBasedStoreResult(blockId2, None)) + + // Verify that the generated KinesisBackedBlockRDD has the all the right information + val blockInfos = Seq(blockInfo1, blockInfo2) + + val kinesisRDD = + kinesisStream.createBlockRDD(time, blockInfos).asInstanceOf[KinesisBackedBlockRDD[_]] + + assert(kinesisRDD.kinesisReadConfigs.retryWaitTimeMs === 2000) + assert(kinesisRDD.kinesisReadConfigs.maxRetries === 5) + assert(kinesisRDD.kinesisReadConfigs.retryTimeoutMs === batchDuration.milliseconds) + } finally { + ssc.sc.conf.remove(RETRY_WAIT_TIME_KEY) + ssc.sc.conf.remove(RETRY_MAX_ATTEMPTS_KEY) + ssc.stop(stopSparkContext = false) + } + } + + testIfEnabled("split and merge shards in a stream") { + // Since this test tries to split and merge shards in a stream, we create another + // temporary stream and then remove it when finished. + val localAppName = s"KinesisStreamSuite-${math.abs(Random.nextLong())}" + val localTestUtils = new KPLBasedKinesisTestUtils(1) + localTestUtils.createStream() + try { + val stream = KinesisInputDStream.builder.streamingContext(ssc) + .checkpointAppName(localAppName) + .streamName(localTestUtils.streamName) + .endpointUrl(localTestUtils.endpointUrl) + .regionName(localTestUtils.regionName) + .initialPositionInStream(InitialPositionInStream.LATEST) + .checkpointInterval(Seconds(10)) + .storageLevel(StorageLevel.MEMORY_ONLY) + .build() + + val collected = new mutable.HashSet[Int] + stream.map { bytes => new String(bytes).toInt }.foreachRDD { rdd => + collected.synchronized { + collected ++= rdd.collect() + logInfo("Collected = " + collected.mkString(", ")) + } + } + ssc.start() + + val testData1 = 1 to 10 + val testData2 = 11 to 20 + val testData3 = 21 to 30 + + eventually(timeout(60 seconds), interval(10 second)) { + localTestUtils.pushData(testData1, aggregateTestData) + assert(collected.synchronized { collected === testData1.toSet }, + "\nData received does not match data sent") + } + + val shardToSplit = localTestUtils.getShards().head + localTestUtils.splitShard(shardToSplit.getShardId) + val (splitOpenShards, splitCloseShards) = localTestUtils.getShards().partition { shard => + shard.getSequenceNumberRange.getEndingSequenceNumber == null + } + + // We should have one closed shard and two open shards + assert(splitCloseShards.size == 1) + assert(splitOpenShards.size == 2) + + eventually(timeout(60 seconds), interval(10 second)) { + localTestUtils.pushData(testData2, aggregateTestData) + assert(collected.synchronized { collected === (testData1 ++ testData2).toSet }, + "\nData received does not match data sent after splitting a shard") + } + + val Seq(shardToMerge, adjShard) = splitOpenShards + localTestUtils.mergeShard(shardToMerge.getShardId, adjShard.getShardId) + val (mergedOpenShards, mergedCloseShards) = localTestUtils.getShards().partition { shard => + shard.getSequenceNumberRange.getEndingSequenceNumber == null + } + + // We should have three closed shards and one open shard + assert(mergedCloseShards.size == 3) + assert(mergedOpenShards.size == 1) + + eventually(timeout(60 seconds), interval(10 second)) { + localTestUtils.pushData(testData3, aggregateTestData) + assert(collected.synchronized { collected === (testData1 ++ testData2 ++ testData3).toSet }, + "\nData received does not match data sent after merging shards") + } + } finally { + ssc.stop(stopSparkContext = false) + localTestUtils.deleteStream() + localTestUtils.deleteDynamoDBTable(localAppName) + } + } + + testIfEnabled("failure recovery") { + val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) + val checkpointDir = Utils.createTempDir().getAbsolutePath + + ssc = new StreamingContext(sc, Milliseconds(1000)) + ssc.checkpoint(checkpointDir) + + val collectedData = new mutable.HashMap[Time, (Array[SequenceNumberRanges], Seq[Int])] + + val kinesisStream = KinesisInputDStream.builder.streamingContext(ssc) + .checkpointAppName(appName) + .streamName(testUtils.streamName) + .endpointUrl(testUtils.endpointUrl) + .regionName(testUtils.regionName) + .initialPositionInStream(InitialPositionInStream.LATEST) + .checkpointInterval(Seconds(10)) + .storageLevel(StorageLevel.MEMORY_ONLY) + .build() + + // Verify that the generated RDDs are KinesisBackedBlockRDDs, and collect the data in each batch + kinesisStream.foreachRDD((rdd: RDD[Array[Byte]], time: Time) => { + val kRdd = rdd.asInstanceOf[KinesisBackedBlockRDD[Array[Byte]]] + val data = rdd.map { bytes => new String(bytes).toInt }.collect().toSeq + collectedData.synchronized { + collectedData(time) = (kRdd.arrayOfseqNumberRanges, data) + } + }) + + ssc.remember(Minutes(60)) // remember all the batches so that they are all saved in checkpoint + ssc.start() + + def numBatchesWithData: Int = + collectedData.synchronized { collectedData.count(_._2._2.nonEmpty) } + + def isCheckpointPresent: Boolean = Checkpoint.getCheckpointFiles(checkpointDir).nonEmpty + + // Run until there are at least 10 batches with some data in them + // If this times out because numBatchesWithData is empty, then its likely that foreachRDD + // function failed with exceptions, and nothing got added to `collectedData` + eventually(timeout(2 minutes), interval(1 seconds)) { + testUtils.pushData(1 to 5, aggregateTestData) + assert(isCheckpointPresent && numBatchesWithData > 10) + } + ssc.stop(stopSparkContext = true) // stop the SparkContext so that the blocks are not reused + + // Restart the context from checkpoint and verify whether the + logInfo("Restarting from checkpoint") + ssc = new StreamingContext(checkpointDir) + ssc.start() + val recoveredKinesisStream = ssc.graph.getInputStreams().head + + // Verify that the recomputed RDDs are KinesisBackedBlockRDDs with the same sequence ranges + // and return the same data + collectedData.synchronized { + val times = collectedData.keySet + times.foreach { time => + val (arrayOfSeqNumRanges, data) = collectedData(time) + val rdd = recoveredKinesisStream.getOrCompute(time).get.asInstanceOf[RDD[Array[Byte]]] + rdd shouldBe a[KinesisBackedBlockRDD[_]] + + // Verify the recovered sequence ranges + val kRdd = rdd.asInstanceOf[KinesisBackedBlockRDD[Array[Byte]]] + assert(kRdd.arrayOfseqNumberRanges.size === arrayOfSeqNumRanges.size) + arrayOfSeqNumRanges.zip(kRdd.arrayOfseqNumberRanges).foreach { case (expected, found) => + assert(expected.ranges.toSeq === found.ranges.toSeq) + } + + // Verify the recovered data + assert(rdd.map { bytes => new String(bytes).toInt }.collect().toSeq === data) + } + } + ssc.stop() + } +} + +class WithAggregationKinesisStreamSuite extends KinesisStreamTests(aggregateTestData = true) + +class WithoutAggregationKinesisStreamSuite extends KinesisStreamTests(aggregateTestData = false) diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentialsBuilderSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentialsBuilderSuite.scala new file mode 100644 index 000000000000..f579c2c3a679 --- /dev/null +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentialsBuilderSuite.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.kinesis + +import org.apache.spark.streaming.TestSuiteBase +import org.apache.spark.util.Utils + +class SparkAWSCredentialsBuilderSuite extends TestSuiteBase { + private def builder = SparkAWSCredentials.builder + + private val basicCreds = BasicCredentials( + awsAccessKeyId = "a-very-nice-access-key", + awsSecretKey = "a-very-nice-secret-key") + + private val stsCreds = STSCredentials( + stsRoleArn = "a-very-nice-role-arn", + stsSessionName = "a-very-nice-secret-key", + stsExternalId = Option("a-very-nice-external-id"), + longLivedCreds = basicCreds) + + test("should build DefaultCredentials when given no params") { + assert(builder.build() == DefaultCredentials) + } + + test("should build BasicCredentials") { + assertResult(basicCreds) { + builder.basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey) + .build() + } + } + + test("should build STSCredentials") { + // No external ID, default long-lived creds + assertResult(stsCreds.copy(stsExternalId = None, longLivedCreds = DefaultCredentials)) { + builder.stsCredentials(stsCreds.stsRoleArn, stsCreds.stsSessionName) + .build() + } + // Default long-lived creds + assertResult(stsCreds.copy(longLivedCreds = DefaultCredentials)) { + builder.stsCredentials( + stsCreds.stsRoleArn, + stsCreds.stsSessionName, + stsCreds.stsExternalId.get) + .build() + } + // No external ID, basic keypair for long-lived creds + assertResult(stsCreds.copy(stsExternalId = None)) { + builder.stsCredentials(stsCreds.stsRoleArn, stsCreds.stsSessionName) + .basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey) + .build() + } + // Basic keypair for long-lived creds + assertResult(stsCreds) { + builder.stsCredentials( + stsCreds.stsRoleArn, + stsCreds.stsSessionName, + stsCreds.stsExternalId.get) + .basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey) + .build() + } + // Order shouldn't matter + assertResult(stsCreds) { + builder.basicCredentials(basicCreds.awsAccessKeyId, basicCreds.awsSecretKey) + .stsCredentials( + stsCreds.stsRoleArn, + stsCreds.stsSessionName, + stsCreds.stsExternalId.get) + .build() + } + } + + test("SparkAWSCredentials classes should be serializable") { + assertResult(basicCreds) { + Utils.deserialize[BasicCredentials](Utils.serialize(basicCreds)) + } + assertResult(stsCreds) { + Utils.deserialize[STSCredentials](Utils.serialize(stsCreds)) + } + // Will also test if DefaultCredentials can be serialized + val stsDefaultCreds = stsCreds.copy(longLivedCreds = DefaultCredentials) + assertResult(stsDefaultCreds) { + Utils.deserialize[STSCredentials](Utils.serialize(stsDefaultCreds)) + } + } +} diff --git a/external/mqtt-assembly/pom.xml b/external/mqtt-assembly/pom.xml deleted file mode 100644 index 89713a28ca6a..000000000000 --- a/external/mqtt-assembly/pom.xml +++ /dev/null @@ -1,175 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-mqtt-assembly_2.10 - jar - Spark Project External MQTT Assembly - http://spark.apache.org/ - - - streaming-mqtt-assembly - - - - - org.apache.spark - spark-streaming-mqtt_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - - commons-lang - commons-lang - provided - - - com.google.protobuf - protobuf-java - provided - - - com.sun.jersey - jersey-server - provided - - - com.sun.jersey - jersey-core - provided - - - org.apache.hadoop - hadoop-client - provided - - - org.apache.avro - avro-mapred - ${avro.mapred.classifier} - provided - - - org.apache.curator - curator-recipes - provided - - - org.apache.zookeeper - zookeeper - provided - - - log4j - log4j - provided - - - net.java.dev.jets3t - jets3t - provided - - - org.scala-lang - scala-library - provided - - - org.slf4j - slf4j-api - provided - - - org.slf4j - slf4j-log4j12 - provided - - - org.xerial.snappy - snappy-java - provided - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-shade-plugin - - false - - - *:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - package - - shade - - - - - - reference.conf - - - log4j.properties - - - - - - - - - - - diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml deleted file mode 100644 index 59fba8b826b4..000000000000 --- a/external/mqtt/pom.xml +++ /dev/null @@ -1,104 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-mqtt_2.10 - - streaming-mqtt - - jar - Spark Project External MQTT - http://spark.apache.org/ - - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - org.eclipse.paho - org.eclipse.paho.client.mqttv3 - 1.0.1 - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - org.apache.activemq - activemq-core - 5.7.0 - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - - - org.apache.maven.plugins - maven-assembly-plugin - - - test-jar-with-dependencies - package - - single - - - - spark-streaming-mqtt-test-${project.version} - ${project.build.directory}/scala-${scala.binary.version}/ - false - - false - - src/main/assembly/assembly.xml - - - - - - - - diff --git a/external/mqtt/src/main/assembly/assembly.xml b/external/mqtt/src/main/assembly/assembly.xml deleted file mode 100644 index ecab5b360eb3..000000000000 --- a/external/mqtt/src/main/assembly/assembly.xml +++ /dev/null @@ -1,44 +0,0 @@ - - - test-jar-with-dependencies - - jar - - false - - - - ${project.build.directory}/scala-${scala.binary.version}/test-classes - / - - - - - - true - test - true - - org.apache.hadoop:*:jar - org.apache.zookeeper:*:jar - org.apache.avro:*:jar - - - - - diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala deleted file mode 100644 index 116c170489e9..000000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.mqtt - -import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken -import org.eclipse.paho.client.mqttv3.MqttCallback -import org.eclipse.paho.client.mqttv3.MqttClient -import org.eclipse.paho.client.mqttv3.MqttMessage -import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receiver.Receiver - -/** - * Input stream that subscribe messages from a Mqtt Broker. - * Uses eclipse paho as MqttClient http://www.eclipse.org/paho/ - * @param brokerUrl Url of remote mqtt publisher - * @param topic topic name to subscribe to - * @param storageLevel RDD storage level. - */ - -private[streaming] -class MQTTInputDStream( - ssc_ : StreamingContext, - brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ) extends ReceiverInputDStream[String](ssc_) { - - private[streaming] override def name: String = s"MQTT stream [$id]" - - def getReceiver(): Receiver[String] = { - new MQTTReceiver(brokerUrl, topic, storageLevel) - } -} - -private[streaming] -class MQTTReceiver( - brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ) extends Receiver[String](storageLevel) { - - def onStop() { - - } - - def onStart() { - - // Set up persistence for messages - val persistence = new MemoryPersistence() - - // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance - val client = new MqttClient(brokerUrl, MqttClient.generateClientId(), persistence) - - // Callback automatically triggers as and when new message arrives on specified topic - val callback = new MqttCallback() { - - // Handles Mqtt message - override def messageArrived(topic: String, message: MqttMessage) { - store(new String(message.getPayload(), "utf-8")) - } - - override def deliveryComplete(token: IMqttDeliveryToken) { - } - - override def connectionLost(cause: Throwable) { - restart("Connection lost ", cause) - } - } - - // Set up callback for MqttClient. This needs to happen before - // connecting or subscribing, otherwise messages may be lost - client.setCallback(callback) - - // Connect to MqttBroker - client.connect() - - // Subscribe to Mqtt topic - client.subscribe(topic) - - } -} diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala deleted file mode 100644 index 7b8d56d6faf2..000000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.mqtt - -import scala.reflect.ClassTag - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaDStream, JavaReceiverInputDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.ReceiverInputDStream - -object MQTTUtils { - /** - * Create an input stream that receives messages pushed by a MQTT publisher. - * @param ssc StreamingContext object - * @param brokerUrl Url of remote MQTT publisher - * @param topic Topic name to subscribe to - * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2. - */ - def createStream( - ssc: StreamingContext, - brokerUrl: String, - topic: String, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): ReceiverInputDStream[String] = { - new MQTTInputDStream(ssc, brokerUrl, topic, storageLevel) - } - - /** - * Create an input stream that receives messages pushed by a MQTT publisher. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param jssc JavaStreamingContext object - * @param brokerUrl Url of remote MQTT publisher - * @param topic Topic name to subscribe to - */ - def createStream( - jssc: JavaStreamingContext, - brokerUrl: String, - topic: String - ): JavaReceiverInputDStream[String] = { - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - createStream(jssc.ssc, brokerUrl, topic) - } - - /** - * Create an input stream that receives messages pushed by a MQTT publisher. - * @param jssc JavaStreamingContext object - * @param brokerUrl Url of remote MQTT publisher - * @param topic Topic name to subscribe to - * @param storageLevel RDD storage level. - */ - def createStream( - jssc: JavaStreamingContext, - brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ): JavaReceiverInputDStream[String] = { - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - createStream(jssc.ssc, brokerUrl, topic, storageLevel) - } -} - -/** - * This is a helper class that wraps the methods in MQTTUtils into more Python-friendly class and - * function so that it can be easily instantiated and called from Python's MQTTUtils. - */ -private[mqtt] class MQTTUtilsPythonHelper { - - def createStream( - jssc: JavaStreamingContext, - brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ): JavaDStream[String] = { - MQTTUtils.createStream(jssc, brokerUrl, topic, storageLevel) - } -} diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package-info.java b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package-info.java deleted file mode 100644 index 728e0d8663d0..000000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package-info.java +++ /dev/null @@ -1,21 +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. - */ - -/** - * MQTT receiver for Spark Streaming. - */ -package org.apache.spark.streaming.mqtt; \ No newline at end of file diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala deleted file mode 100644 index 63d0d138183a..000000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming - -/** - * MQTT receiver for Spark Streaming. - */ -package object mqtt diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java deleted file mode 100644 index cfedb5a042a3..000000000000 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming; - -import org.apache.spark.SparkConf; -import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.junit.After; -import org.junit.Before; - -public abstract class LocalJavaStreamingContext { - - protected transient JavaStreamingContext ssc; - - @Before - public void setUp() { - SparkConf conf = new SparkConf() - .setMaster("local[2]") - .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); - ssc = new JavaStreamingContext(conf, new Duration(1000)); - ssc.checkpoint("checkpoint"); - } - - @After - public void tearDown() { - ssc.stop(); - ssc = null; - } -} diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java deleted file mode 100644 index ce5aa1e0cdda..000000000000 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.mqtt; - -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; -import org.junit.Test; - -import org.apache.spark.streaming.LocalJavaStreamingContext; - -public class JavaMQTTStreamSuite extends LocalJavaStreamingContext { - @Test - public void testMQTTStream() { - String brokerUrl = "abc"; - String topic = "def"; - - // tests the API, does not actually test data receiving - JavaReceiverInputDStream test1 = MQTTUtils.createStream(ssc, brokerUrl, topic); - JavaReceiverInputDStream test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, - StorageLevel.MEMORY_AND_DISK_SER_2()); - } -} 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 deleted file mode 100644 index a6a9249db8ed..000000000000 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.mqtt - -import scala.concurrent.duration._ -import scala.language.postfixOps - -import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.Eventually - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Milliseconds, StreamingContext} - -class MQTTStreamSuite extends SparkFunSuite with Eventually with BeforeAndAfter { - - private val batchDuration = Milliseconds(500) - private val master = "local[2]" - private val framework = this.getClass.getSimpleName - private val topic = "def" - - private var ssc: StreamingContext = _ - private var mqttTestUtils: MQTTTestUtils = _ - - before { - ssc = new StreamingContext(master, framework, batchDuration) - mqttTestUtils = new MQTTTestUtils - mqttTestUtils.setup() - } - - after { - if (ssc != null) { - ssc.stop() - ssc = null - } - if (mqttTestUtils != null) { - mqttTestUtils.teardown() - mqttTestUtils = null - } - } - - test("mqtt input stream") { - val sendMessage = "MQTT demo for spark streaming" - val receiveStream = MQTTUtils.createStream(ssc, "tcp://" + mqttTestUtils.brokerUri, topic, - StorageLevel.MEMORY_ONLY) - - @volatile var receiveMessage: List[String] = List() - receiveStream.foreachRDD { rdd => - if (rdd.collect.length > 0) { - receiveMessage = receiveMessage ::: List(rdd.first) - receiveMessage - } - } - - ssc.start() - - // Retry it because we don't know when the receiver will start. - eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { - mqttTestUtils.publishData(topic, sendMessage) - assert(sendMessage.equals(receiveMessage(0))) - } - ssc.stop() - } -} diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTTestUtils.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTTestUtils.scala deleted file mode 100644 index 1618e2c088b7..000000000000 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTTestUtils.scala +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.mqtt - -import java.net.{ServerSocket, URI} - -import scala.language.postfixOps - -import com.google.common.base.Charsets.UTF_8 -import org.apache.activemq.broker.{BrokerService, TransportConnector} -import org.apache.commons.lang3.RandomUtils -import org.eclipse.paho.client.mqttv3._ -import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence - -import org.apache.spark.util.Utils -import org.apache.spark.{Logging, SparkConf} - -/** - * Share codes for Scala and Python unit tests - */ -private[mqtt] class MQTTTestUtils extends Logging { - - private val persistenceDir = Utils.createTempDir() - private val brokerHost = "localhost" - private val brokerPort = findFreePort() - - private var broker: BrokerService = _ - private var connector: TransportConnector = _ - - def brokerUri: String = { - s"$brokerHost:$brokerPort" - } - - def setup(): Unit = { - broker = new BrokerService() - broker.setDataDirectoryFile(Utils.createTempDir()) - connector = new TransportConnector() - connector.setName("mqtt") - connector.setUri(new URI("mqtt://" + brokerUri)) - broker.addConnector(connector) - broker.start() - } - - def teardown(): Unit = { - if (broker != null) { - broker.stop() - broker = null - } - if (connector != null) { - connector.stop() - connector = null - } - Utils.deleteRecursively(persistenceDir) - } - - private def findFreePort(): Int = { - val candidatePort = RandomUtils.nextInt(1024, 65536) - Utils.startServiceOnPort(candidatePort, (trialPort: Int) => { - val socket = new ServerSocket(trialPort) - socket.close() - (null, trialPort) - }, new SparkConf())._2 - } - - def publishData(topic: String, data: String): Unit = { - var client: MqttClient = null - try { - val persistence = new MqttDefaultFilePersistence(persistenceDir.getAbsolutePath) - client = new MqttClient("tcp://" + brokerUri, MqttClient.generateClientId(), persistence) - client.connect() - if (client.isConnected) { - val msgTopic = client.getTopic(topic) - val message = new MqttMessage(data.getBytes(UTF_8)) - message.setQos(1) - message.setRetained(true) - - for (i <- 0 to 10) { - try { - msgTopic.publish(message) - } catch { - case e: MqttException if e.getReasonCode == MqttException.REASON_CODE_MAX_INFLIGHT => - // wait for Spark streaming to consume something from the message queue - Thread.sleep(50) - } - } - } - } finally { - if (client != null) { - client.disconnect() - client.close() - client = null - } - } - } - -} diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml new file mode 100644 index 000000000000..36d555066b18 --- /dev/null +++ b/external/spark-ganglia-lgpl/pom.xml @@ -0,0 +1,48 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../../pom.xml + + + + spark-ganglia-lgpl_2.11 + jar + Spark Ganglia Integration + + + ganglia-lgpl + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + + io.dropwizard.metrics + metrics-ganglia + + + diff --git a/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/external/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala similarity index 94% rename from extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala rename to external/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala index 3b1880e14351..0cd795f63887 100644 --- a/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala +++ b/external/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala @@ -46,6 +46,9 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry, val GANGLIA_KEY_HOST = "host" val GANGLIA_KEY_PORT = "port" + val GANGLIA_KEY_DMAX = "dmax" + val GANGLIA_DEFAULT_DMAX = 0 + def propertyToOption(prop: String): Option[String] = Option(property.getProperty(prop)) if (!propertyToOption(GANGLIA_KEY_HOST).isDefined) { @@ -59,6 +62,7 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry, val host = propertyToOption(GANGLIA_KEY_HOST).get val port = propertyToOption(GANGLIA_KEY_PORT).get.toInt val ttl = propertyToOption(GANGLIA_KEY_TTL).map(_.toInt).getOrElse(GANGLIA_DEFAULT_TTL) + val dmax = propertyToOption(GANGLIA_KEY_DMAX).map(_.toInt).getOrElse(GANGLIA_DEFAULT_DMAX) val mode: UDPAddressingMode = propertyToOption(GANGLIA_KEY_MODE) .map(u => GMetric.UDPAddressingMode.valueOf(u.toUpperCase)).getOrElse(GANGLIA_DEFAULT_MODE) val pollPeriod = propertyToOption(GANGLIA_KEY_PERIOD).map(_.toInt) @@ -73,6 +77,7 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry, val reporter: GangliaReporter = GangliaReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) + .withDMax(dmax) .build(ganglia) override def start() { diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml deleted file mode 100644 index 087270de90b3..000000000000 --- a/external/twitter/pom.xml +++ /dev/null @@ -1,70 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-twitter_2.10 - - streaming-twitter - - jar - Spark Project External Twitter - http://spark.apache.org/ - - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - org.twitter4j - twitter4j-stream - 4.0.4 - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala deleted file mode 100644 index 9a85a6597c27..000000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ /dev/null @@ -1,115 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.twitter - -import twitter4j._ -import twitter4j.auth.Authorization -import twitter4j.conf.ConfigurationBuilder -import twitter4j.auth.OAuthAuthorization - -import org.apache.spark.streaming._ -import org.apache.spark.streaming.dstream._ -import org.apache.spark.storage.StorageLevel -import org.apache.spark.Logging -import org.apache.spark.streaming.receiver.Receiver - -/* A stream of Twitter statuses, potentially filtered by one or more keywords. -* -* @constructor create a new Twitter stream using the supplied Twitter4J authentication credentials. -* An optional set of string filters can be used to restrict the set of tweets. The Twitter API is -* such that this may return a sampled subset of all tweets during each interval. -* -* If no Authorization object is provided, initializes OAuth authorization using the system -* properties twitter4j.oauth.consumerKey, .consumerSecret, .accessToken and .accessTokenSecret. -*/ -private[streaming] -class TwitterInputDStream( - ssc_ : StreamingContext, - twitterAuth: Option[Authorization], - filters: Seq[String], - storageLevel: StorageLevel - ) extends ReceiverInputDStream[Status](ssc_) { - - private def createOAuthAuthorization(): Authorization = { - new OAuthAuthorization(new ConfigurationBuilder().build()) - } - - private val authorization = twitterAuth.getOrElse(createOAuthAuthorization()) - - override def getReceiver(): Receiver[Status] = { - new TwitterReceiver(authorization, filters, storageLevel) - } -} - -private[streaming] -class TwitterReceiver( - twitterAuth: Authorization, - filters: Seq[String], - storageLevel: StorageLevel - ) extends Receiver[Status](storageLevel) with Logging { - - @volatile private var twitterStream: TwitterStream = _ - @volatile private var stopped = false - - def onStart() { - try { - val newTwitterStream = new TwitterStreamFactory().getInstance(twitterAuth) - newTwitterStream.addListener(new StatusListener { - def onStatus(status: Status): Unit = { - store(status) - } - // Unimplemented - def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {} - def onTrackLimitationNotice(i: Int) {} - def onScrubGeo(l: Long, l1: Long) {} - def onStallWarning(stallWarning: StallWarning) {} - def onException(e: Exception) { - if (!stopped) { - restart("Error receiving tweets", e) - } - } - }) - - val query = new FilterQuery - if (filters.size > 0) { - query.track(filters.mkString(",")) - newTwitterStream.filter(query) - } else { - newTwitterStream.sample() - } - setTwitterStream(newTwitterStream) - logInfo("Twitter receiver started") - stopped = false - } catch { - case e: Exception => restart("Error starting Twitter stream", e) - } - } - - def onStop() { - stopped = true - setTwitterStream(null) - logInfo("Twitter receiver stopped") - } - - private def setTwitterStream(newTwitterStream: TwitterStream) = synchronized { - if (twitterStream != null) { - twitterStream.shutdown() - } - twitterStream = newTwitterStream - } -} diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala deleted file mode 100644 index c6a9a2b73714..000000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.twitter - -import twitter4j.Status -import twitter4j.auth.Authorization -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} - -object TwitterUtils { - /** - * Create a input stream that returns tweets received from Twitter. - * @param ssc StreamingContext object - * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth - * authorization; this uses the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def createStream( - ssc: StreamingContext, - twitterAuth: Option[Authorization], - filters: Seq[String] = Nil, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): ReceiverInputDStream[Status] = { - new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel) - } - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param jssc JavaStreamingContext object - */ - def createStream(jssc: JavaStreamingContext): JavaReceiverInputDStream[Status] = { - createStream(jssc.ssc, None) - } - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param jssc JavaStreamingContext object - * @param filters Set of filter strings to get only those tweets that match them - */ - def createStream(jssc: JavaStreamingContext, filters: Array[String] - ): JavaReceiverInputDStream[Status] = { - createStream(jssc.ssc, None, filters) - } - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret. - * @param jssc JavaStreamingContext object - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def createStream( - jssc: JavaStreamingContext, - filters: Array[String], - storageLevel: StorageLevel - ): JavaReceiverInputDStream[Status] = { - createStream(jssc.ssc, None, filters, storageLevel) - } - - /** - * Create a input stream that returns tweets received from Twitter. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param jssc JavaStreamingContext object - * @param twitterAuth Twitter4J Authorization - */ - def createStream(jssc: JavaStreamingContext, twitterAuth: Authorization - ): JavaReceiverInputDStream[Status] = { - createStream(jssc.ssc, Some(twitterAuth)) - } - - /** - * Create a input stream that returns tweets received from Twitter. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param jssc JavaStreamingContext object - * @param twitterAuth Twitter4J Authorization - * @param filters Set of filter strings to get only those tweets that match them - */ - def createStream( - jssc: JavaStreamingContext, - twitterAuth: Authorization, - filters: Array[String] - ): JavaReceiverInputDStream[Status] = { - createStream(jssc.ssc, Some(twitterAuth), filters) - } - - /** - * Create a input stream that returns tweets received from Twitter. - * @param jssc JavaStreamingContext object - * @param twitterAuth Twitter4J Authorization object - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def createStream( - jssc: JavaStreamingContext, - twitterAuth: Authorization, - filters: Array[String], - storageLevel: StorageLevel - ): JavaReceiverInputDStream[Status] = { - createStream(jssc.ssc, Some(twitterAuth), filters, storageLevel) - } -} diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package-info.java b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package-info.java deleted file mode 100644 index 258c0950a0aa..000000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package-info.java +++ /dev/null @@ -1,21 +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. - */ - -/** - * Twitter feed receiver for spark streaming. - */ -package org.apache.spark.streaming.twitter; \ No newline at end of file diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala deleted file mode 100644 index 580e37fa8f81..000000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming - -/** - * Twitter feed receiver for spark streaming. - */ -package object twitter diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java deleted file mode 100644 index cfedb5a042a3..000000000000 --- a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming; - -import org.apache.spark.SparkConf; -import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.junit.After; -import org.junit.Before; - -public abstract class LocalJavaStreamingContext { - - protected transient JavaStreamingContext ssc; - - @Before - public void setUp() { - SparkConf conf = new SparkConf() - .setMaster("local[2]") - .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); - ssc = new JavaStreamingContext(conf, new Duration(1000)); - ssc.checkpoint("checkpoint"); - } - - @After - public void tearDown() { - ssc.stop(); - ssc = null; - } -} diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java deleted file mode 100644 index 26ec8af455bc..000000000000 --- a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.twitter; - -import org.junit.Test; -import twitter4j.Status; -import twitter4j.auth.Authorization; -import twitter4j.auth.NullAuthorization; -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaDStream; - -public class JavaTwitterStreamSuite extends LocalJavaStreamingContext { - @Test - public void testTwitterStream() { - String[] filters = { "filter1", "filter2" }; - Authorization auth = NullAuthorization.getInstance(); - - // tests the API, does not actually test data receiving - JavaDStream test1 = TwitterUtils.createStream(ssc); - JavaDStream test2 = TwitterUtils.createStream(ssc, filters); - JavaDStream test3 = TwitterUtils.createStream( - ssc, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaDStream test4 = TwitterUtils.createStream(ssc, auth); - JavaDStream test5 = TwitterUtils.createStream(ssc, auth, filters); - JavaDStream test6 = TwitterUtils.createStream(ssc, - auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); - } -} diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties deleted file mode 100644 index 9a3569789d2e..000000000000 --- a/external/twitter/src/test/resources/log4j.properties +++ /dev/null @@ -1,28 +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. -# - -# Set everything to be logged to the filetarget/unit-tests.log -log4j.rootCategory=INFO, file -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=true -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN - diff --git a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala deleted file mode 100644 index d9acb568879f..000000000000 --- a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.twitter - - -import org.scalatest.BeforeAndAfter -import twitter4j.Status -import twitter4j.auth.{NullAuthorization, Authorization} - -import org.apache.spark.{Logging, SparkFunSuite} -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.dstream.ReceiverInputDStream - -class TwitterStreamSuite extends SparkFunSuite with BeforeAndAfter with Logging { - - val batchDuration = Seconds(1) - - private val master: String = "local[2]" - - private val framework: String = this.getClass.getSimpleName - - test("twitter input stream") { - val ssc = new StreamingContext(master, framework, batchDuration) - val filters = Seq("filter1", "filter2") - val authorization: Authorization = NullAuthorization.getInstance() - - // tests the API, does not actually test data receiving - val test1: ReceiverInputDStream[Status] = TwitterUtils.createStream(ssc, None) - val test2: ReceiverInputDStream[Status] = - TwitterUtils.createStream(ssc, None, filters) - val test3: ReceiverInputDStream[Status] = - TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_AND_DISK_SER_2) - val test4: ReceiverInputDStream[Status] = - TwitterUtils.createStream(ssc, Some(authorization)) - val test5: ReceiverInputDStream[Status] = - TwitterUtils.createStream(ssc, Some(authorization), filters) - val test6: ReceiverInputDStream[Status] = TwitterUtils.createStream( - ssc, Some(authorization), filters, StorageLevel.MEMORY_AND_DISK_SER_2) - - // Note that actually testing the data receiving is hard as authentication keys are - // necessary for accessing Twitter live stream - ssc.stop() - } -} diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml deleted file mode 100644 index 02d6b8128157..000000000000 --- a/external/zeromq/pom.xml +++ /dev/null @@ -1,69 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-zeromq_2.10 - - streaming-zeromq - - jar - Spark Project External ZeroMQ - http://spark.apache.org/ - - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - ${akka.group} - akka-zeromq_${scala.binary.version} - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala deleted file mode 100644 index 588e6bac7b14..000000000000 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.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.streaming.zeromq - -import scala.reflect.ClassTag - -import akka.actor.Actor -import akka.util.ByteString -import akka.zeromq._ - -import org.apache.spark.Logging -import org.apache.spark.streaming.receiver.ActorHelper - -/** - * A receiver to subscribe to ZeroMQ stream. - */ -private[streaming] class ZeroMQReceiver[T: ClassTag]( - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: Seq[ByteString] => Iterator[T]) - extends Actor with ActorHelper with Logging { - - override def preStart(): Unit = { - ZeroMQExtension(context.system) - .newSocket(SocketType.Sub, Listener(self), Connect(publisherUrl), subscribe) - } - - def receive: Receive = { - - case Connecting => logInfo("connecting ...") - - case m: ZMQMessage => - logDebug("Received message for:" + m.frame(0)) - - // We ignore first frame for processing as it is the topic - val bytes = m.frames.tail - store(bytesToObjects(bytes)) - - case Closed => logInfo("received closed ") - } -} diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala deleted file mode 100644 index 4ea218eaa4de..000000000000 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala +++ /dev/null @@ -1,133 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.zeromq - -import scala.reflect.ClassTag -import scala.collection.JavaConverters._ - -import akka.actor.{Props, SupervisorStrategy} -import akka.util.ByteString -import akka.zeromq.Subscribe - -import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.ReceiverInputDStream -import org.apache.spark.streaming.receiver.ActorSupervisorStrategy - -object ZeroMQUtils { - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param ssc StreamingContext object - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe Topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic - * and each frame has sequence of byte thus it needs the converter - * (which might be deserializer of bytes) to translate from sequence - * of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. - * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2. - */ - def createStream[T: ClassTag]( - ssc: StreamingContext, - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: Seq[ByteString] => Iterator[T], - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, - supervisorStrategy: SupervisorStrategy = ActorSupervisorStrategy.defaultStrategy - ): ReceiverInputDStream[T] = { - ssc.actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)), - "ZeroMQReceiver", storageLevel, supervisorStrategy) - } - - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param jssc JavaStreamingContext object - * @param publisherUrl Url of remote ZeroMQ publisher - * @param subscribe Topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each - * frame has sequence of byte thus it needs the converter(which might be - * deserializer of bytes) to translate from sequence of sequence of bytes, - * where sequence refer to a frame and sub sequence refer to its payload. - * @param storageLevel Storage level to use for storing the received objects - */ - def createStream[T]( - jssc: JavaStreamingContext, - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], - storageLevel: StorageLevel, - supervisorStrategy: SupervisorStrategy - ): JavaReceiverInputDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - val fn = - (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).iterator().asScala - createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel, supervisorStrategy) - } - - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param jssc JavaStreamingContext object - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe Topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each - * frame has sequence of byte thus it needs the converter(which might be - * deserializer of bytes) to translate from sequence of sequence of bytes, - * where sequence refer to a frame and sub sequence refer to its payload. - * @param storageLevel RDD storage level. - */ - def createStream[T]( - jssc: JavaStreamingContext, - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], - storageLevel: StorageLevel - ): JavaReceiverInputDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - val fn = - (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).iterator().asScala - createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel) - } - - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param jssc JavaStreamingContext object - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe Topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each - * frame has sequence of byte thus it needs the converter(which might - * be deserializer of bytes) to translate from sequence of sequence of - * bytes, where sequence refer to a frame and sub sequence refer to its - * payload. - */ - def createStream[T]( - jssc: JavaStreamingContext, - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]] - ): JavaReceiverInputDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - val fn = - (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).iterator().asScala - createStream[T](jssc.ssc, publisherUrl, subscribe, fn) - } -} diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package-info.java b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package-info.java deleted file mode 100644 index 587c524e2120..000000000000 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package-info.java +++ /dev/null @@ -1,21 +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. - */ - -/** - * Zeromq receiver for spark streaming. - */ -package org.apache.spark.streaming.zeromq; \ No newline at end of file diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala deleted file mode 100644 index 65e6e57f2c05..000000000000 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming - -/** - * Zeromq receiver for spark streaming. - */ -package object zeromq diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java deleted file mode 100644 index cfedb5a042a3..000000000000 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming; - -import org.apache.spark.SparkConf; -import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.junit.After; -import org.junit.Before; - -public abstract class LocalJavaStreamingContext { - - protected transient JavaStreamingContext ssc; - - @Before - public void setUp() { - SparkConf conf = new SparkConf() - .setMaster("local[2]") - .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); - ssc = new JavaStreamingContext(conf, new Duration(1000)); - ssc.checkpoint("checkpoint"); - } - - @After - public void tearDown() { - ssc.stop(); - ssc = null; - } -} diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java deleted file mode 100644 index 417b91eecb0e..000000000000 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.zeromq; - -import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; -import org.junit.Test; -import akka.actor.SupervisorStrategy; -import akka.util.ByteString; -import akka.zeromq.Subscribe; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.LocalJavaStreamingContext; - -public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext { - - @Test // tests the API, does not actually test data receiving - public void testZeroMQStream() { - String publishUrl = "abc"; - Subscribe subscribe = new Subscribe((ByteString)null); - Function> bytesToObjects = new Function>() { - @Override - public Iterable call(byte[][] bytes) throws Exception { - return null; - } - }; - - JavaReceiverInputDStream test1 = ZeroMQUtils.createStream( - ssc, publishUrl, subscribe, bytesToObjects); - JavaReceiverInputDStream test2 = ZeroMQUtils.createStream( - ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaReceiverInputDStream test3 = ZeroMQUtils.createStream( - ssc,publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), - SupervisorStrategy.defaultStrategy()); - } -} diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties deleted file mode 100644 index 75e3b53a093f..000000000000 --- a/external/zeromq/src/test/resources/log4j.properties +++ /dev/null @@ -1,28 +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. -# - -# Set everything to be logged to the file target/unit-tests.log -log4j.rootCategory=INFO, file -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=true -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN - diff --git a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala deleted file mode 100644 index 35d2e62c6848..000000000000 --- a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.zeromq - -import akka.actor.SupervisorStrategy -import akka.util.ByteString -import akka.zeromq.Subscribe - -import org.apache.spark.SparkFunSuite -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.dstream.ReceiverInputDStream - -class ZeroMQStreamSuite extends SparkFunSuite { - - val batchDuration = Seconds(1) - - private val master: String = "local[2]" - - private val framework: String = this.getClass.getSimpleName - - test("zeromq input stream") { - val ssc = new StreamingContext(master, framework, batchDuration) - val publishUrl = "abc" - val subscribe = new Subscribe(null.asInstanceOf[ByteString]) - val bytesToObjects = (bytes: Seq[ByteString]) => null.asInstanceOf[Iterator[String]] - - // tests the API, does not actually test data receiving - val test1: ReceiverInputDStream[String] = - ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects) - val test2: ReceiverInputDStream[String] = ZeroMQUtils.createStream( - ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2) - val test3: ReceiverInputDStream[String] = ZeroMQUtils.createStream( - ssc, publishUrl, subscribe, bytesToObjects, - StorageLevel.MEMORY_AND_DISK_SER_2, SupervisorStrategy.defaultStrategy) - - // TODO: Actually test data receiving - ssc.stop() - } -} diff --git a/extras/README.md b/extras/README.md deleted file mode 100644 index 1b4174b7d5cf..000000000000 --- a/extras/README.md +++ /dev/null @@ -1 +0,0 @@ -This directory contains build components not included by default in Spark's build. diff --git a/extras/java8-tests/README.md b/extras/java8-tests/README.md deleted file mode 100644 index dc9e87f2eeb9..000000000000 --- a/extras/java8-tests/README.md +++ /dev/null @@ -1,24 +0,0 @@ -# Java 8 Test Suites - -These tests require having Java 8 installed and are isolated from the main Spark build. -If Java 8 is not your system's default Java version, you will need to point Spark's build -to your Java location. The set-up depends a bit on the build system: - -* Sbt users can either set JAVA_HOME to the location of a Java 8 JDK or explicitly pass - `-java-home` to the sbt launch script. If a Java 8 JDK is detected sbt will automatically - include the Java 8 test project. - - `$ JAVA_HOME=/opt/jdk1.8.0/ build/sbt clean "test-only org.apache.spark.Java8APISuite"` - -* For Maven users, - - Maven users can also refer to their Java 8 directory using JAVA_HOME. However, Maven will not - automatically detect the presence of a Java 8 JDK, so a special build profile `-Pjava8-tests` - must be used. - - `$ JAVA_HOME=/opt/jdk1.8.0/ mvn clean install -DskipTests` - `$ JAVA_HOME=/opt/jdk1.8.0/ mvn test -Pjava8-tests -DwildcardSuites=org.apache.spark.Java8APISuite` - - Note that the above command can only be run from project root directory since this module - depends on core and the test-jars of core and streaming. This means an install step is - required to make the test dependencies visible to the Java 8 sub-project. diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml deleted file mode 100644 index 4ce90e75fd35..000000000000 --- a/extras/java8-tests/pom.xml +++ /dev/null @@ -1,161 +0,0 @@ - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - java8-tests_2.10 - pom - Spark Project Java8 Tests POM - - - java8-tests - - - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - test-jar - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - - - - java8-tests - - - - - - org.apache.maven.plugins - maven-deploy-plugin - - true - - - - org.apache.maven.plugins - maven-install-plugin - - true - - - - org.apache.maven.plugins - maven-surefire-plugin - - - test - - test - - - - - - - - file:src/test/resources/log4j.properties - - - false - - **/Suite*.java - **/*Suite.java - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - test-compile-first - process-test-resources - - testCompile - - - - - true - true - true - 1.8 - 1.8 - 1.8 - UTF-8 - 1024m - - - - - net.alchim31.maven - scala-maven-plugin - - - none - - - scala-compile-first - none - - - scala-test-compile-first - none - - - attach-scaladocs - none - - - - - - diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java deleted file mode 100644 index 14975265ab2c..000000000000 --- a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java +++ /dev/null @@ -1,393 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark; - -import java.io.File; -import java.io.Serializable; -import java.util.*; - -import scala.Tuple2; - -import com.google.common.collect.Iterables; -import com.google.common.base.Optional; -import com.google.common.io.Files; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.SequenceFileOutputFormat; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -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.*; -import org.apache.spark.util.Utils; - -/** - * Most of these tests replicate org.apache.spark.JavaAPISuite using java 8 - * lambda syntax. - */ -public class Java8APISuite implements Serializable { - static int foreachCalls = 0; - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaAPISuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } - - @Test - public void foreachWithAnonymousClass() { - foreachCalls = 0; - JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); - rdd.foreach(new VoidFunction() { - @Override - public void call(String s) { - foreachCalls++; - } - }); - Assert.assertEquals(2, foreachCalls); - } - - @Test - public void foreach() { - foreachCalls = 0; - JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); - rdd.foreach(x -> foreachCalls++); - Assert.assertEquals(2, foreachCalls); - } - - @Test - public void groupBy() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); - Function isOdd = x -> x % 2 == 0; - JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); - Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds - - oddsAndEvens = rdd.groupBy(isOdd, 1); - Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds - } - - @Test - public void leftOuterJoin() { - JavaPairRDD rdd1 = sc.parallelizePairs(Arrays.asList( - new Tuple2<>(1, 1), - new Tuple2<>(1, 2), - new Tuple2<>(2, 1), - new Tuple2<>(3, 1) - )); - JavaPairRDD rdd2 = sc.parallelizePairs(Arrays.asList( - new Tuple2<>(1, 'x'), - new Tuple2<>(2, 'y'), - new Tuple2<>(2, 'z'), - new Tuple2<>(4, 'w') - )); - List>>> joined = - rdd1.leftOuterJoin(rdd2).collect(); - Assert.assertEquals(5, joined.size()); - Tuple2>> firstUnmatched = - rdd1.leftOuterJoin(rdd2).filter(tup -> !tup._2()._2().isPresent()).first(); - Assert.assertEquals(3, firstUnmatched._1().intValue()); - } - - @Test - public void foldReduce() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); - Function2 add = (a, b) -> a + b; - - int sum = rdd.fold(0, add); - Assert.assertEquals(33, sum); - - sum = rdd.reduce(add); - Assert.assertEquals(33, sum); - } - - @Test - public void foldByKey() { - List> pairs = Arrays.asList( - new Tuple2<>(2, 1), - new Tuple2<>(2, 1), - new Tuple2<>(1, 1), - new Tuple2<>(3, 2), - new Tuple2<>(3, 1) - ); - JavaPairRDD rdd = sc.parallelizePairs(pairs); - JavaPairRDD sums = rdd.foldByKey(0, (a, b) -> a + b); - Assert.assertEquals(1, sums.lookup(1).get(0).intValue()); - Assert.assertEquals(2, sums.lookup(2).get(0).intValue()); - Assert.assertEquals(3, sums.lookup(3).get(0).intValue()); - } - - @Test - public void reduceByKey() { - List> pairs = Arrays.asList( - new Tuple2<>(2, 1), - new Tuple2<>(2, 1), - new Tuple2<>(1, 1), - new Tuple2<>(3, 2), - new Tuple2<>(3, 1) - ); - JavaPairRDD rdd = sc.parallelizePairs(pairs); - JavaPairRDD counts = rdd.reduceByKey((a, b) -> a + b); - Assert.assertEquals(1, counts.lookup(1).get(0).intValue()); - Assert.assertEquals(2, counts.lookup(2).get(0).intValue()); - Assert.assertEquals(3, counts.lookup(3).get(0).intValue()); - - Map localCounts = counts.collectAsMap(); - Assert.assertEquals(1, localCounts.get(1).intValue()); - Assert.assertEquals(2, localCounts.get(2).intValue()); - Assert.assertEquals(3, localCounts.get(3).intValue()); - - localCounts = rdd.reduceByKeyLocally((a, b) -> a + b); - Assert.assertEquals(1, localCounts.get(1).intValue()); - Assert.assertEquals(2, localCounts.get(2).intValue()); - Assert.assertEquals(3, localCounts.get(3).intValue()); - } - - @Test - public void map() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - JavaDoubleRDD doubles = rdd.mapToDouble(x -> 1.0 * x).cache(); - doubles.collect(); - JavaPairRDD pairs = rdd.mapToPair(x -> new Tuple2<>(x, x)) - .cache(); - pairs.collect(); - JavaRDD strings = rdd.map(Object::toString).cache(); - strings.collect(); - } - - @Test - public void flatMap() { - JavaRDD rdd = sc.parallelize(Arrays.asList("Hello World!", - "The quick brown fox jumps over the lazy dog.")); - JavaRDD words = rdd.flatMap(x -> Arrays.asList(x.split(" "))); - - Assert.assertEquals("Hello", words.first()); - Assert.assertEquals(11, words.count()); - - JavaPairRDD pairs = rdd.flatMapToPair(s -> { - List> pairs2 = new LinkedList<>(); - for (String word : s.split(" ")) { - pairs2.add(new Tuple2<>(word, word)); - } - return pairs2; - }); - - Assert.assertEquals(new Tuple2<>("Hello", "Hello"), pairs.first()); - Assert.assertEquals(11, pairs.count()); - - JavaDoubleRDD doubles = rdd.flatMapToDouble(s -> { - List lengths = new LinkedList<>(); - for (String word : s.split(" ")) { - lengths.add((double) word.length()); - } - return lengths; - }); - - Assert.assertEquals(5.0, doubles.first(), 0.01); - Assert.assertEquals(11, pairs.count()); - } - - @Test - public void mapsFromPairsToPairs() { - List> pairs = Arrays.asList( - new Tuple2<>(1, "a"), - new Tuple2<>(2, "aa"), - new Tuple2<>(3, "aaa") - ); - JavaPairRDD pairRDD = sc.parallelizePairs(pairs); - - // Regression test for SPARK-668: - JavaPairRDD swapped = - pairRDD.flatMapToPair(x -> Collections.singletonList(x.swap())); - swapped.collect(); - - // There was never a bug here, but it's worth testing: - pairRDD.map(Tuple2::swap).collect(); - } - - @Test - public void mapPartitions() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); - JavaRDD partitionSums = rdd.mapPartitions(iter -> { - int sum = 0; - while (iter.hasNext()) { - sum += iter.next(); - } - return Collections.singletonList(sum); - }); - - Assert.assertEquals("[3, 7]", partitionSums.collect().toString()); - } - - @Test - public void sequenceFile() { - File tempDir = Files.createTempDir(); - tempDir.deleteOnExit(); - String outputDir = new File(tempDir, "output").getAbsolutePath(); - List> pairs = Arrays.asList( - new Tuple2<>(1, "a"), - new Tuple2<>(2, "aa"), - new Tuple2<>(3, "aaa") - ); - JavaPairRDD rdd = sc.parallelizePairs(pairs); - - rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()))) - .saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); - - // Try reading the output back as an object file - JavaPairRDD readRDD = sc.sequenceFile(outputDir, IntWritable.class, Text.class) - .mapToPair(pair -> new Tuple2<>(pair._1().get(), pair._2().toString())); - Assert.assertEquals(pairs, readRDD.collect()); - Utils.deleteRecursively(tempDir); - } - - @Test - public void zip() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - JavaDoubleRDD doubles = rdd.mapToDouble(x -> 1.0 * x); - JavaPairRDD zipped = rdd.zip(doubles); - zipped.count(); - } - - @Test - public void zipPartitions() { - JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6), 2); - JavaRDD rdd2 = sc.parallelize(Arrays.asList("1", "2", "3", "4"), 2); - FlatMapFunction2, Iterator, Integer> sizesFn = - (Iterator i, Iterator s) -> { - int sizeI = 0; - while (i.hasNext()) { - sizeI += 1; - i.next(); - } - int sizeS = 0; - while (s.hasNext()) { - sizeS += 1; - s.next(); - } - return Arrays.asList(sizeI, sizeS); - }; - JavaRDD sizes = rdd1.zipPartitions(rdd2, sizesFn); - Assert.assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); - } - - @Test - public void accumulators() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - - Accumulator intAccum = sc.intAccumulator(10); - rdd.foreach(intAccum::add); - Assert.assertEquals((Integer) 25, intAccum.value()); - - Accumulator doubleAccum = sc.doubleAccumulator(10.0); - rdd.foreach(x -> doubleAccum.add((double) x)); - Assert.assertEquals((Double) 25.0, doubleAccum.value()); - - // Try a custom accumulator type - AccumulatorParam floatAccumulatorParam = new AccumulatorParam() { - @Override - public Float addInPlace(Float r, Float t) { - return r + t; - } - @Override - public Float addAccumulator(Float r, Float t) { - return r + t; - } - @Override - public Float zero(Float initialValue) { - return 0.0f; - } - }; - - Accumulator floatAccum = sc.accumulator(10.0f, floatAccumulatorParam); - rdd.foreach(x -> floatAccum.add((float) x)); - Assert.assertEquals((Float) 25.0f, floatAccum.value()); - - // Test the setValue method - floatAccum.setValue(5.0f); - Assert.assertEquals((Float) 5.0f, floatAccum.value()); - } - - @Test - public void keyBy() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2)); - List> s = rdd.keyBy(Object::toString).collect(); - Assert.assertEquals(new Tuple2<>("1", 1), s.get(0)); - Assert.assertEquals(new Tuple2<>("2", 2), s.get(1)); - } - - @Test - public void mapOnPairRDD() { - JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4)); - JavaPairRDD rdd2 = - rdd1.mapToPair(i -> new Tuple2<>(i, i % 2)); - JavaPairRDD rdd3 = - rdd2.mapToPair(in -> new Tuple2<>(in._2(), in._1())); - Assert.assertEquals(Arrays.asList( - new Tuple2<>(1, 1), - new Tuple2<>(0, 2), - new Tuple2<>(1, 3), - new Tuple2<>(0, 4)), rdd3.collect()); - } - - @Test - public void collectPartitions() { - JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); - - JavaPairRDD rdd2 = - rdd1.mapToPair(i -> new Tuple2<>(i, i % 2)); - List[] parts = rdd1.collectPartitions(new int[]{0}); - Assert.assertEquals(Arrays.asList(1, 2), parts[0]); - - parts = rdd1.collectPartitions(new int[]{1, 2}); - Assert.assertEquals(Arrays.asList(3, 4), parts[0]); - Assert.assertEquals(Arrays.asList(5, 6, 7), parts[1]); - - Assert.assertEquals(Arrays.asList(new Tuple2<>(1, 1), new Tuple2<>(2, 0)), - rdd2.collectPartitions(new int[]{0})[0]); - - List>[] parts2 = rdd2.collectPartitions(new int[]{1, 2}); - Assert.assertEquals(Arrays.asList(new Tuple2<>(3, 1), new Tuple2<>(4, 0)), parts2[0]); - Assert.assertEquals(Arrays.asList(new Tuple2<>(5, 1), new Tuple2<>(6, 0), new Tuple2<>(7, 1)), - parts2[1]); - } - - @Test - public void collectAsMapWithIntArrayValues() { - // Regression test for SPARK-1040 - JavaRDD rdd = sc.parallelize(Arrays.asList(1)); - JavaPairRDD pairRDD = - rdd.mapToPair(x -> new Tuple2<>(x, new int[]{x})); - pairRDD.collect(); // Works fine - pairRDD.collectAsMap(); // Used to crash with ClassCastException - } -} diff --git a/extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java deleted file mode 100644 index 73091cfe2c09..000000000000 --- a/extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java +++ /dev/null @@ -1,834 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming; - -import java.io.Serializable; -import java.util.*; - -import scala.Tuple2; - -import com.google.common.base.Optional; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import org.junit.Assert; -import org.junit.Test; - -import org.apache.spark.HashPartitioner; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.JavaPairDStream; - -/** - * Most of these tests replicate org.apache.spark.streaming.JavaAPISuite using java 8 - * lambda syntax. - */ -@SuppressWarnings("unchecked") -public class Java8APISuite extends LocalJavaStreamingContext implements Serializable { - - @Test - public void testMap() { - List> inputData = Arrays.asList( - Arrays.asList("hello", "world"), - Arrays.asList("goodnight", "moon")); - - List> expected = Arrays.asList( - Arrays.asList(5, 5), - Arrays.asList(9, 4)); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream letterCount = stream.map(String::length); - JavaTestUtils.attachTestOutputStream(letterCount); - List> result = JavaTestUtils.runStreams(ssc, 2, 2); - - assertOrderInvariantEquals(expected, result); - } - - @Test - public void testFilter() { - List> inputData = Arrays.asList( - Arrays.asList("giants", "dodgers"), - Arrays.asList("yankees", "red sox")); - - List> expected = Arrays.asList( - Arrays.asList("giants"), - Arrays.asList("yankees")); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream filtered = stream.filter(s -> s.contains("a")); - JavaTestUtils.attachTestOutputStream(filtered); - List> result = JavaTestUtils.runStreams(ssc, 2, 2); - - assertOrderInvariantEquals(expected, result); - } - - @Test - public void testMapPartitions() { - List> inputData = Arrays.asList( - Arrays.asList("giants", "dodgers"), - Arrays.asList("yankees", "red sox")); - - List> expected = Arrays.asList( - Arrays.asList("GIANTSDODGERS"), - Arrays.asList("YANKEESRED SOX")); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream mapped = stream.mapPartitions(in -> { - String out = ""; - while (in.hasNext()) { - out = out + in.next().toUpperCase(); - } - return Lists.newArrayList(out); - }); - JavaTestUtils.attachTestOutputStream(mapped); - List> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @Test - public void testReduce() { - List> inputData = Arrays.asList( - Arrays.asList(1, 2, 3), - Arrays.asList(4, 5, 6), - Arrays.asList(7, 8, 9)); - - List> expected = Arrays.asList( - Arrays.asList(6), - Arrays.asList(15), - Arrays.asList(24)); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream reduced = stream.reduce((x, y) -> x + y); - JavaTestUtils.attachTestOutputStream(reduced); - List> result = JavaTestUtils.runStreams(ssc, 3, 3); - - Assert.assertEquals(expected, result); - } - - @Test - public void testReduceByWindow() { - List> inputData = Arrays.asList( - Arrays.asList(1, 2, 3), - Arrays.asList(4, 5, 6), - Arrays.asList(7, 8, 9)); - - List> expected = Arrays.asList( - Arrays.asList(6), - Arrays.asList(21), - Arrays.asList(39), - Arrays.asList(24)); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream reducedWindowed = stream.reduceByWindow((x, y) -> x + y, - (x, y) -> x - y, new Duration(2000), new Duration(1000)); - JavaTestUtils.attachTestOutputStream(reducedWindowed); - List> result = JavaTestUtils.runStreams(ssc, 4, 4); - - Assert.assertEquals(expected, result); - } - - @Test - public void testTransform() { - List> inputData = Arrays.asList( - Arrays.asList(1, 2, 3), - Arrays.asList(4, 5, 6), - Arrays.asList(7, 8, 9)); - - List> expected = Arrays.asList( - Arrays.asList(3, 4, 5), - Arrays.asList(6, 7, 8), - Arrays.asList(9, 10, 11)); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream transformed = stream.transform(in -> in.map(i -> i + 2)); - - JavaTestUtils.attachTestOutputStream(transformed); - List> result = JavaTestUtils.runStreams(ssc, 3, 3); - - assertOrderInvariantEquals(expected, result); - } - - @Test - public void testVariousTransform() { - // tests whether all variations of transform can be called from Java - - List> inputData = Arrays.asList(Arrays.asList(1)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - - List>> pairInputData = - Arrays.asList(Arrays.asList(new Tuple2<>("x", 1))); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream( - JavaTestUtils.attachTestInputStream(ssc, pairInputData, 1)); - - JavaDStream transformed1 = stream.transform(in -> null); - JavaDStream transformed2 = stream.transform((x, time) -> null); - JavaPairDStream transformed3 = stream.transformToPair(x -> null); - JavaPairDStream transformed4 = stream.transformToPair((x, time) -> null); - JavaDStream pairTransformed1 = pairStream.transform(x -> null); - JavaDStream pairTransformed2 = pairStream.transform((x, time) -> null); - JavaPairDStream pairTransformed3 = pairStream.transformToPair(x -> null); - JavaPairDStream pairTransformed4 = - pairStream.transformToPair((x, time) -> null); - - } - - @Test - public void testTransformWith() { - List>> stringStringKVStream1 = Arrays.asList( - Arrays.asList( - new Tuple2<>("california", "dodgers"), - new Tuple2<>("new york", "yankees")), - Arrays.asList( - new Tuple2<>("california", "sharks"), - new Tuple2<>("new york", "rangers"))); - - List>> stringStringKVStream2 = Arrays.asList( - Arrays.asList( - new Tuple2<>("california", "giants"), - new Tuple2<>("new york", "mets")), - Arrays.asList( - new Tuple2<>("california", "ducks"), - new Tuple2<>("new york", "islanders"))); - - - List>>> expected = Arrays.asList( - Sets.newHashSet( - new Tuple2<>("california", - new Tuple2<>("dodgers", "giants")), - new Tuple2<>("new york", - new Tuple2<>("yankees", "mets"))), - Sets.newHashSet( - new Tuple2<>("california", - new Tuple2<>("sharks", "ducks")), - new Tuple2<>("new york", - new Tuple2<>("rangers", "islanders")))); - - JavaDStream> stream1 = JavaTestUtils.attachTestInputStream( - ssc, stringStringKVStream1, 1); - JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream(stream1); - - JavaDStream> stream2 = JavaTestUtils.attachTestInputStream( - ssc, stringStringKVStream2, 1); - JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); - - JavaPairDStream> joined = - pairStream1.transformWithToPair(pairStream2,(x, y, z) -> x.join(y)); - - JavaTestUtils.attachTestOutputStream(joined); - List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - List>>> unorderedResult = Lists.newArrayList(); - for (List>> res : result) { - unorderedResult.add(Sets.newHashSet(res)); - } - - Assert.assertEquals(expected, unorderedResult); - } - - - @Test - public void testVariousTransformWith() { - // tests whether all variations of transformWith can be called from Java - - List> inputData1 = Arrays.asList(Arrays.asList(1)); - List> inputData2 = Arrays.asList(Arrays.asList("x")); - JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 1); - JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 1); - - List>> pairInputData1 = - Arrays.asList(Arrays.asList(new Tuple2<>("x", 1))); - List>> pairInputData2 = - Arrays.asList(Arrays.asList(new Tuple2<>(1.0, 'x'))); - JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream( - JavaTestUtils.attachTestInputStream(ssc, pairInputData1, 1)); - JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream( - JavaTestUtils.attachTestInputStream(ssc, pairInputData2, 1)); - - JavaDStream transformed1 = stream1.transformWith(stream2, (x, y, z) -> null); - JavaDStream transformed2 = stream1.transformWith(pairStream1,(x, y, z) -> null); - - JavaPairDStream transformed3 = - stream1.transformWithToPair(stream2,(x, y, z) -> null); - - JavaPairDStream transformed4 = - stream1.transformWithToPair(pairStream1,(x, y, z) -> null); - - JavaDStream pairTransformed1 = pairStream1.transformWith(stream2,(x, y, z) -> null); - - JavaDStream pairTransformed2_ = - pairStream1.transformWith(pairStream1,(x, y, z) -> null); - - JavaPairDStream pairTransformed3 = - pairStream1.transformWithToPair(stream2,(x, y, z) -> null); - - JavaPairDStream pairTransformed4 = - pairStream1.transformWithToPair(pairStream2,(x, y, z) -> null); - } - - @Test - public void testStreamingContextTransform() { - List> stream1input = Arrays.asList( - Arrays.asList(1), - Arrays.asList(2) - ); - - List> stream2input = Arrays.asList( - Arrays.asList(3), - Arrays.asList(4) - ); - - List>> pairStream1input = Arrays.asList( - Arrays.asList(new Tuple2<>(1, "x")), - Arrays.asList(new Tuple2<>(2, "y")) - ); - - List>>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>(1, new Tuple2<>(1, "x"))), - Arrays.asList(new Tuple2<>(2, new Tuple2<>(2, "y"))) - ); - - JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, stream1input, 1); - JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, stream2input, 1); - JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream( - JavaTestUtils.attachTestInputStream(ssc, pairStream1input, 1)); - - List> listOfDStreams1 = Arrays.>asList(stream1, stream2); - - // This is just to test whether this transform to JavaStream compiles - JavaDStream transformed1 = ssc.transform( - listOfDStreams1, (List> listOfRDDs, Time time) -> { - Assert.assertEquals(2, listOfRDDs.size()); - return null; - }); - - List> listOfDStreams2 = - Arrays.>asList(stream1, stream2, pairStream1.toJavaDStream()); - - JavaPairDStream> transformed2 = ssc.transformToPair( - listOfDStreams2, (List> listOfRDDs, Time time) -> { - Assert.assertEquals(3, listOfRDDs.size()); - JavaRDD rdd1 = (JavaRDD) listOfRDDs.get(0); - JavaRDD rdd2 = (JavaRDD) listOfRDDs.get(1); - JavaRDD> rdd3 = (JavaRDD>) listOfRDDs.get(2); - JavaPairRDD prdd3 = JavaPairRDD.fromJavaRDD(rdd3); - PairFunction mapToTuple = - (Integer i) -> new Tuple2<>(i, i); - return rdd1.union(rdd2).mapToPair(mapToTuple).join(prdd3); - }); - JavaTestUtils.attachTestOutputStream(transformed2); - List>>> result = - JavaTestUtils.runStreams(ssc, 2, 2); - Assert.assertEquals(expected, result); - } - - @Test - public void testFlatMap() { - List> inputData = Arrays.asList( - Arrays.asList("go", "giants"), - Arrays.asList("boo", "dodgers"), - Arrays.asList("athletics")); - - List> expected = Arrays.asList( - Arrays.asList("g", "o", "g", "i", "a", "n", "t", "s"), - Arrays.asList("b", "o", "o", "d", "o", "d", "g", "e", "r", "s"), - Arrays.asList("a", "t", "h", "l", "e", "t", "i", "c", "s")); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream flatMapped = stream.flatMap(s -> Lists.newArrayList(s.split("(?!^)"))); - JavaTestUtils.attachTestOutputStream(flatMapped); - List> result = JavaTestUtils.runStreams(ssc, 3, 3); - - assertOrderInvariantEquals(expected, result); - } - - @Test - public void testPairFlatMap() { - List> inputData = Arrays.asList( - Arrays.asList("giants"), - Arrays.asList("dodgers"), - Arrays.asList("athletics")); - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>(6, "g"), - new Tuple2<>(6, "i"), - new Tuple2<>(6, "a"), - new Tuple2<>(6, "n"), - new Tuple2<>(6, "t"), - new Tuple2<>(6, "s")), - Arrays.asList( - new Tuple2<>(7, "d"), - new Tuple2<>(7, "o"), - new Tuple2<>(7, "d"), - new Tuple2<>(7, "g"), - new Tuple2<>(7, "e"), - new Tuple2<>(7, "r"), - new Tuple2<>(7, "s")), - Arrays.asList( - new Tuple2<>(9, "a"), - new Tuple2<>(9, "t"), - new Tuple2<>(9, "h"), - new Tuple2<>(9, "l"), - new Tuple2<>(9, "e"), - new Tuple2<>(9, "t"), - new Tuple2<>(9, "i"), - new Tuple2<>(9, "c"), - new Tuple2<>(9, "s"))); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream flatMapped = stream.flatMapToPair(s -> { - List> out = Lists.newArrayList(); - for (String letter : s.split("(?!^)")) { - out.add(new Tuple2<>(s.length(), letter)); - } - return out; - }); - - JavaTestUtils.attachTestOutputStream(flatMapped); - List>> result = JavaTestUtils.runStreams(ssc, 3, 3); - - Assert.assertEquals(expected, result); - } - - /* - * Performs an order-invariant comparison of lists representing two RDD streams. This allows - * us to account for ordering variation within individual RDD's which occurs during windowing. - */ - public static > void assertOrderInvariantEquals( - List> expected, List> actual) { - expected.forEach((List list) -> Collections.sort(list)); - actual.forEach((List list) -> Collections.sort(list)); - Assert.assertEquals(expected, actual); - } - - @Test - public void testPairFilter() { - List> inputData = Arrays.asList( - Arrays.asList("giants", "dodgers"), - Arrays.asList("yankees", "red sox")); - - List>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>("giants", 6)), - Arrays.asList(new Tuple2<>("yankees", 7))); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = - stream.mapToPair(x -> new Tuple2<>(x, x.length())); - JavaPairDStream filtered = pairStream.filter(x -> x._1().contains("a")); - JavaTestUtils.attachTestOutputStream(filtered); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - List>> stringStringKVStream = Arrays.asList( - Arrays.asList(new Tuple2<>("california", "dodgers"), - new Tuple2<>("california", "giants"), - new Tuple2<>("new york", "yankees"), - new Tuple2<>("new york", "mets")), - Arrays.asList(new Tuple2<>("california", "sharks"), - new Tuple2<>("california", "ducks"), - new Tuple2<>("new york", "rangers"), - new Tuple2<>("new york", "islanders"))); - - List>> stringIntKVStream = Arrays.asList( - Arrays.asList( - new Tuple2<>("california", 1), - new Tuple2<>("california", 3), - new Tuple2<>("new york", 4), - new Tuple2<>("new york", 1)), - Arrays.asList( - new Tuple2<>("california", 5), - new Tuple2<>("california", 5), - new Tuple2<>("new york", 3), - new Tuple2<>("new york", 1))); - - @Test - public void testPairMap() { // Maps pair -> pair of different type - List>> inputData = stringIntKVStream; - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>(1, "california"), - new Tuple2<>(3, "california"), - new Tuple2<>(4, "new york"), - new Tuple2<>(1, "new york")), - Arrays.asList( - new Tuple2<>(5, "california"), - new Tuple2<>(5, "california"), - new Tuple2<>(3, "new york"), - new Tuple2<>(1, "new york"))); - - JavaDStream> stream = - JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream reversed = pairStream.mapToPair(x -> x.swap()); - JavaTestUtils.attachTestOutputStream(reversed); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @Test - public void testPairMapPartitions() { // Maps pair -> pair of different type - List>> inputData = stringIntKVStream; - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>(1, "california"), - new Tuple2<>(3, "california"), - new Tuple2<>(4, "new york"), - new Tuple2<>(1, "new york")), - Arrays.asList( - new Tuple2<>(5, "california"), - new Tuple2<>(5, "california"), - new Tuple2<>(3, "new york"), - new Tuple2<>(1, "new york"))); - - JavaDStream> stream = - JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream reversed = pairStream.mapPartitionsToPair(in -> { - LinkedList> out = new LinkedList<>(); - while (in.hasNext()) { - Tuple2 next = in.next(); - out.add(next.swap()); - } - return out; - }); - - JavaTestUtils.attachTestOutputStream(reversed); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @Test - public void testPairMap2() { // Maps pair -> single - List>> inputData = stringIntKVStream; - - List> expected = Arrays.asList( - Arrays.asList(1, 3, 4, 1), - Arrays.asList(5, 5, 3, 1)); - - JavaDStream> stream = - JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaDStream reversed = pairStream.map(in -> in._2()); - JavaTestUtils.attachTestOutputStream(reversed); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @Test - public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair - List>> inputData = Arrays.asList( - Arrays.asList( - new Tuple2<>("hi", 1), - new Tuple2<>("ho", 2)), - Arrays.asList( - new Tuple2<>("hi", 1), - new Tuple2<>("ho", 2))); - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>(1, "h"), - new Tuple2<>(1, "i"), - new Tuple2<>(2, "h"), - new Tuple2<>(2, "o")), - Arrays.asList( - new Tuple2<>(1, "h"), - new Tuple2<>(1, "i"), - new Tuple2<>(2, "h"), - new Tuple2<>(2, "o"))); - - JavaDStream> stream = - JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream flatMapped = pairStream.flatMapToPair(in -> { - List> out = new LinkedList<>(); - for (Character s : in._1().toCharArray()) { - out.add(new Tuple2<>(in._2(), s.toString())); - } - return out; - }); - - JavaTestUtils.attachTestOutputStream(flatMapped); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @Test - public void testPairReduceByKey() { - List>> inputData = stringIntKVStream; - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>("california", 4), - new Tuple2<>("new york", 5)), - Arrays.asList( - new Tuple2<>("california", 10), - new Tuple2<>("new york", 4))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream( - ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream reduced = pairStream.reduceByKey((x, y) -> x + y); - - JavaTestUtils.attachTestOutputStream(reduced); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @Test - public void testCombineByKey() { - List>> inputData = stringIntKVStream; - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>("california", 4), - new Tuple2<>("new york", 5)), - Arrays.asList( - new Tuple2<>("california", 10), - new Tuple2<>("new york", 4))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream( - ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream combined = pairStream.combineByKey(i -> i, - (x, y) -> x + y, (x, y) -> x + y, new HashPartitioner(2)); - - JavaTestUtils.attachTestOutputStream(combined); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @Test - public void testReduceByKeyAndWindow() { - List>> inputData = stringIntKVStream; - - List>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>("california", 4), - new Tuple2<>("new york", 5)), - Arrays.asList(new Tuple2<>("california", 14), - new Tuple2<>("new york", 9)), - Arrays.asList(new Tuple2<>("california", 10), - new Tuple2<>("new york", 4))); - - JavaDStream> stream = - JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream reduceWindowed = - pairStream.reduceByKeyAndWindow((x, y) -> x + y, new Duration(2000), new Duration(1000)); - JavaTestUtils.attachTestOutputStream(reduceWindowed); - List>> result = JavaTestUtils.runStreams(ssc, 3, 3); - - Assert.assertEquals(expected, result); - } - - @Test - public void testUpdateStateByKey() { - List>> inputData = stringIntKVStream; - - List>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>("california", 4), - new Tuple2<>("new york", 5)), - Arrays.asList(new Tuple2<>("california", 14), - new Tuple2<>("new york", 9)), - Arrays.asList(new Tuple2<>("california", 14), - new Tuple2<>("new york", 9))); - - JavaDStream> stream = - JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream updated = pairStream.updateStateByKey((values, state) -> { - int out = 0; - if (state.isPresent()) { - out = out + state.get(); - } - for (Integer v : values) { - out = out + v; - } - return Optional.of(out); - }); - - JavaTestUtils.attachTestOutputStream(updated); - List>> result = JavaTestUtils.runStreams(ssc, 3, 3); - - Assert.assertEquals(expected, result); - } - - @Test - public void testReduceByKeyAndWindowWithInverse() { - List>> inputData = stringIntKVStream; - - List>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>("california", 4), - new Tuple2<>("new york", 5)), - Arrays.asList(new Tuple2<>("california", 14), - new Tuple2<>("new york", 9)), - Arrays.asList(new Tuple2<>("california", 10), - new Tuple2<>("new york", 4))); - - JavaDStream> stream = - JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream reduceWindowed = - pairStream.reduceByKeyAndWindow((x, y) -> x + y, (x, y) -> x - y, new Duration(2000), - new Duration(1000)); - JavaTestUtils.attachTestOutputStream(reduceWindowed); - List>> result = JavaTestUtils.runStreams(ssc, 3, 3); - - Assert.assertEquals(expected, result); - } - - @Test - public void testPairTransform() { - List>> inputData = Arrays.asList( - Arrays.asList( - new Tuple2<>(3, 5), - new Tuple2<>(1, 5), - new Tuple2<>(4, 5), - new Tuple2<>(2, 5)), - Arrays.asList( - new Tuple2<>(2, 5), - new Tuple2<>(3, 5), - new Tuple2<>(4, 5), - new Tuple2<>(1, 5))); - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>(1, 5), - new Tuple2<>(2, 5), - new Tuple2<>(3, 5), - new Tuple2<>(4, 5)), - Arrays.asList( - new Tuple2<>(1, 5), - new Tuple2<>(2, 5), - new Tuple2<>(3, 5), - new Tuple2<>(4, 5))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream( - ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream sorted = pairStream.transformToPair(in -> in.sortByKey()); - - JavaTestUtils.attachTestOutputStream(sorted); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @Test - public void testPairToNormalRDDTransform() { - List>> inputData = Arrays.asList( - Arrays.asList( - new Tuple2<>(3, 5), - new Tuple2<>(1, 5), - new Tuple2<>(4, 5), - new Tuple2<>(2, 5)), - Arrays.asList( - new Tuple2<>(2, 5), - new Tuple2<>(3, 5), - new Tuple2<>(4, 5), - new Tuple2<>(1, 5))); - - List> expected = Arrays.asList( - Arrays.asList(3, 1, 4, 2), - Arrays.asList(2, 3, 4, 1)); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream( - ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaDStream firstParts = pairStream.transform(in -> in.map(x -> x._1())); - JavaTestUtils.attachTestOutputStream(firstParts); - List> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @Test - public void testMapValues() { - List>> inputData = stringStringKVStream; - - List>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>("california", "DODGERS"), - new Tuple2<>("california", "GIANTS"), - new Tuple2<>("new york", "YANKEES"), - new Tuple2<>("new york", "METS")), - Arrays.asList(new Tuple2<>("california", "SHARKS"), - new Tuple2<>("california", "DUCKS"), - new Tuple2<>("new york", "RANGERS"), - new Tuple2<>("new york", "ISLANDERS"))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream( - ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream mapped = pairStream.mapValues(String::toUpperCase); - JavaTestUtils.attachTestOutputStream(mapped); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @Test - public void testFlatMapValues() { - List>> inputData = stringStringKVStream; - - List>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>("california", "dodgers1"), - new Tuple2<>("california", "dodgers2"), - new Tuple2<>("california", "giants1"), - new Tuple2<>("california", "giants2"), - new Tuple2<>("new york", "yankees1"), - new Tuple2<>("new york", "yankees2"), - new Tuple2<>("new york", "mets1"), - new Tuple2<>("new york", "mets2")), - Arrays.asList(new Tuple2<>("california", "sharks1"), - new Tuple2<>("california", "sharks2"), - new Tuple2<>("california", "ducks1"), - new Tuple2<>("california", "ducks2"), - new Tuple2<>("new york", "rangers1"), - new Tuple2<>("new york", "rangers2"), - new Tuple2<>("new york", "islanders1"), - new Tuple2<>("new york", "islanders2"))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream( - ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream flatMapped = - pairStream.flatMapValues(in -> Arrays.asList(in + "1", in + "2")); - JavaTestUtils.attachTestOutputStream(flatMapped); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - Assert.assertEquals(expected, result); - } - -} diff --git a/extras/java8-tests/src/test/resources/log4j.properties b/extras/java8-tests/src/test/resources/log4j.properties deleted file mode 100644 index eb3b1999eb99..000000000000 --- a/extras/java8-tests/src/test/resources/log4j.properties +++ /dev/null @@ -1,28 +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. -# - -# Set everything to be logged to the file target/unit-tests.log -log4j.rootCategory=INFO, file -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=true -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN -org.spark-project.jetty.LEVEL=WARN diff --git a/extras/kinesis-asl-assembly/pom.xml b/extras/kinesis-asl-assembly/pom.xml deleted file mode 100644 index 61ba4787fbf9..000000000000 --- a/extras/kinesis-asl-assembly/pom.xml +++ /dev/null @@ -1,181 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-kinesis-asl-assembly_2.10 - jar - Spark Project Kinesis Assembly - http://spark.apache.org/ - - - streaming-kinesis-asl-assembly - - - - - org.apache.spark - spark-streaming-kinesis-asl_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - - com.fasterxml.jackson.core - jackson-databind - provided - - - commons-lang - commons-lang - provided - - - com.google.protobuf - protobuf-java - provided - - - com.sun.jersey - jersey-server - provided - - - com.sun.jersey - jersey-core - provided - - - log4j - log4j - provided - - - net.java.dev.jets3t - jets3t - provided - - - org.apache.hadoop - hadoop-client - provided - - - org.apache.avro - avro-ipc - provided - - - org.apache.avro - avro-mapred - ${avro.mapred.classifier} - provided - - - org.apache.curator - curator-recipes - provided - - - org.apache.zookeeper - zookeeper - provided - - - org.slf4j - slf4j-api - provided - - - org.slf4j - slf4j-log4j12 - provided - - - org.xerial.snappy - snappy-java - provided - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-shade-plugin - - false - - - *:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - package - - shade - - - - - - reference.conf - - - log4j.properties - - - - - - - - - - - - diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml deleted file mode 100644 index ef72d97eae69..000000000000 --- a/extras/kinesis-asl/pom.xml +++ /dev/null @@ -1,86 +0,0 @@ - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../../pom.xml - - - - org.apache.spark - spark-streaming-kinesis-asl_2.10 - jar - Spark Kinesis Integration - - - streaming-kinesis-asl - - - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - test-jar - test - - - com.amazonaws - amazon-kinesis-client - ${aws.kinesis.client.version} - - - com.amazonaws - aws-java-sdk - ${aws.java.sdk.version} - - - org.mockito - mockito-core - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/extras/kinesis-asl/src/main/resources/log4j.properties b/extras/kinesis-asl/src/main/resources/log4j.properties deleted file mode 100644 index 6cdc9286c5d7..000000000000 --- a/extras/kinesis-asl/src/main/resources/log4j.properties +++ /dev/null @@ -1,37 +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. -# - -log4j.rootCategory=WARN, console - -# File appender -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=false -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n - -# Console appender -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.target=System.out -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n - -# Settings to quiet third party logs that are too verbose -log4j.logger.org.spark-project.jetty=WARN -log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR -log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO -log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO \ No newline at end of file diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala deleted file mode 100644 index 83a453755951..000000000000 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.streaming.kinesis - -import org.apache.spark.Logging -import org.apache.spark.streaming.Duration -import org.apache.spark.util.{Clock, ManualClock, SystemClock} - -/** - * This is a helper class for managing checkpoint clocks. - * - * @param checkpointInterval - * @param currentClock. Default to current SystemClock if none is passed in (mocking purposes) - */ -private[kinesis] class KinesisCheckpointState( - checkpointInterval: Duration, - currentClock: Clock = new SystemClock()) - extends Logging { - - /* Initialize the checkpoint clock using the given currentClock + checkpointInterval millis */ - val checkpointClock = new ManualClock() - checkpointClock.setTime(currentClock.getTimeMillis() + checkpointInterval.milliseconds) - - /** - * Check if it's time to checkpoint based on the current time and the derived time - * for the next checkpoint - * - * @return true if it's time to checkpoint - */ - def shouldCheckpoint(): Boolean = { - new SystemClock().getTimeMillis() > checkpointClock.getTimeMillis() - } - - /** - * Advance the checkpoint clock by the checkpoint interval. - */ - def advanceCheckpoint(): Unit = { - checkpointClock.advance(checkpointInterval.milliseconds) - } -} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala deleted file mode 100644 index 72ab6357a53b..000000000000 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kinesis - -import scala.reflect.ClassTag - -import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream -import com.amazonaws.services.kinesis.model.Record - -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.{BlockId, StorageLevel} -import org.apache.spark.streaming.dstream.ReceiverInputDStream -import org.apache.spark.streaming.receiver.Receiver -import org.apache.spark.streaming.scheduler.ReceivedBlockInfo -import org.apache.spark.streaming.{Duration, StreamingContext, Time} - -private[kinesis] class KinesisInputDStream[T: ClassTag]( - @transient _ssc: StreamingContext, - streamName: String, - endpointUrl: String, - regionName: String, - initialPositionInStream: InitialPositionInStream, - checkpointAppName: String, - checkpointInterval: Duration, - storageLevel: StorageLevel, - messageHandler: Record => T, - awsCredentialsOption: Option[SerializableAWSCredentials] - ) extends ReceiverInputDStream[T](_ssc) { - - private[streaming] - override def createBlockRDD(time: Time, blockInfos: Seq[ReceivedBlockInfo]): RDD[T] = { - - // This returns true even for when blockInfos is empty - val allBlocksHaveRanges = blockInfos.map { _.metadataOption }.forall(_.nonEmpty) - - if (allBlocksHaveRanges) { - // Create a KinesisBackedBlockRDD, even when there are no blocks - val blockIds = blockInfos.map { _.blockId.asInstanceOf[BlockId] }.toArray - val seqNumRanges = blockInfos.map { - _.metadataOption.get.asInstanceOf[SequenceNumberRanges] }.toArray - val isBlockIdValid = blockInfos.map { _.isBlockIdValid() }.toArray - logDebug(s"Creating KinesisBackedBlockRDD for $time with ${seqNumRanges.length} " + - s"seq number ranges: ${seqNumRanges.mkString(", ")} ") - new KinesisBackedBlockRDD( - context.sc, regionName, endpointUrl, blockIds, seqNumRanges, - isBlockIdValid = isBlockIdValid, - retryTimeoutMs = ssc.graph.batchDuration.milliseconds.toInt, - messageHandler = messageHandler, - awsCredentialsOption = awsCredentialsOption) - } else { - logWarning("Kinesis sequence number information was not present with some block metadata," + - " it may not be possible to recover from failures") - super.createBlockRDD(time, blockInfos) - } - } - - override def getReceiver(): Receiver[T] = { - new KinesisReceiver(streamName, endpointUrl, regionName, initialPositionInStream, - checkpointAppName, checkpointInterval, storageLevel, messageHandler, awsCredentialsOption) - } -} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala deleted file mode 100644 index 134d627cdaff..000000000000 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala +++ /dev/null @@ -1,332 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.streaming.kinesis - -import java.util.UUID - -import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.util.control.NonFatal - -import com.amazonaws.auth.{AWSCredentials, AWSCredentialsProvider, DefaultAWSCredentialsProviderChain} -import com.amazonaws.services.kinesis.clientlibrary.interfaces.{IRecordProcessor, IRecordProcessorFactory} -import com.amazonaws.services.kinesis.clientlibrary.lib.worker.{InitialPositionInStream, KinesisClientLibConfiguration, Worker} -import com.amazonaws.services.kinesis.model.Record - -import org.apache.spark.storage.{StorageLevel, StreamBlockId} -import org.apache.spark.streaming.Duration -import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver} -import org.apache.spark.util.Utils -import org.apache.spark.{Logging, SparkEnv} - - -private[kinesis] -case class SerializableAWSCredentials(accessKeyId: String, secretKey: String) - extends AWSCredentials { - override def getAWSAccessKeyId: String = accessKeyId - override def getAWSSecretKey: String = secretKey -} - -/** - * Custom AWS Kinesis-specific implementation of Spark Streaming's Receiver. - * This implementation relies on the Kinesis Client Library (KCL) Worker as described here: - * https://github.com/awslabs/amazon-kinesis-client - * - * The way this Receiver works is as follows: - * - The receiver starts a KCL Worker, which is essentially runs a threadpool of multiple - * KinesisRecordProcessor - * - Each KinesisRecordProcessor receives data from a Kinesis shard in batches. Each batch is - * inserted into a Block Generator, and the corresponding range of sequence numbers is recorded. - * - When the block generator defines a block, then the recorded sequence number ranges that were - * inserted into the block are recorded separately for being used later. - * - When the block is ready to be pushed, the block is pushed and the ranges are reported as - * metadata of the block. In addition, the ranges are used to find out the latest sequence - * number for each shard that can be checkpointed through the DynamoDB. - * - Periodically, each KinesisRecordProcessor checkpoints the latest successfully stored sequence - * number for it own shard. - * - * @param streamName Kinesis stream name - * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) - * @param regionName Region name used by the Kinesis Client Library for - * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics) - * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the - * worker's initial starting position in the stream. - * The values are either the beginning of the stream - * per Kinesis' limit of 24 hours - * (InitialPositionInStream.TRIM_HORIZON) or - * the tip of the stream (InitialPositionInStream.LATEST). - * @param checkpointAppName Kinesis application name. Kinesis Apps are mapped to Kinesis Streams - * by the Kinesis Client Library. If you change the App name or Stream name, - * the KCL will throw errors. This usually requires deleting the backing - * DynamoDB table with the same name this Kinesis application. - * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. - * See the Kinesis Spark Streaming documentation for more - * details on the different types of checkpoints. - * @param storageLevel Storage level to use for storing the received objects - * @param awsCredentialsOption Optional AWS credentials, used when user directly specifies - * the credentials - */ -private[kinesis] class KinesisReceiver[T]( - val streamName: String, - endpointUrl: String, - regionName: String, - initialPositionInStream: InitialPositionInStream, - checkpointAppName: String, - checkpointInterval: Duration, - storageLevel: StorageLevel, - messageHandler: Record => T, - awsCredentialsOption: Option[SerializableAWSCredentials]) - extends Receiver[T](storageLevel) with Logging { receiver => - - /* - * ================================================================================= - * The following vars are initialize in the onStart() method which executes in the - * Spark worker after this Receiver is serialized and shipped to the worker. - * ================================================================================= - */ - - /** - * workerId is used by the KCL should be based on the ip address of the actual Spark Worker - * where this code runs (not the driver's IP address.) - */ - @volatile private var workerId: String = null - - /** - * Worker is the core client abstraction from the Kinesis Client Library (KCL). - * A worker can process more than one shards from the given stream. - * Each shard is assigned its own IRecordProcessor and the worker run multiple such - * processors. - */ - @volatile private var worker: Worker = null - @volatile private var workerThread: Thread = null - - /** BlockGenerator used to generates blocks out of Kinesis data */ - @volatile private var blockGenerator: BlockGenerator = null - - /** - * Sequence number ranges added to the current block being generated. - * Accessing and updating of this map is synchronized by locks in BlockGenerator. - */ - private val seqNumRangesInCurrentBlock = new mutable.ArrayBuffer[SequenceNumberRange] - - /** Sequence number ranges of data added to each generated block */ - private val blockIdToSeqNumRanges = new mutable.HashMap[StreamBlockId, SequenceNumberRanges] - with mutable.SynchronizedMap[StreamBlockId, SequenceNumberRanges] - - /** - * Latest sequence number ranges that have been stored successfully. - * This is used for checkpointing through KCL */ - private val shardIdToLatestStoredSeqNum = new mutable.HashMap[String, String] - with mutable.SynchronizedMap[String, String] - /** - * This is called when the KinesisReceiver starts and must be non-blocking. - * The KCL creates and manages the receiving/processing thread pool through Worker.run(). - */ - override def onStart() { - blockGenerator = supervisor.createBlockGenerator(new GeneratedBlockHandler) - - workerId = Utils.localHostName() + ":" + UUID.randomUUID() - - // KCL config instance - val awsCredProvider = resolveAWSCredentialsProvider() - val kinesisClientLibConfiguration = - new KinesisClientLibConfiguration(checkpointAppName, streamName, awsCredProvider, workerId) - .withKinesisEndpoint(endpointUrl) - .withInitialPositionInStream(initialPositionInStream) - .withTaskBackoffTimeMillis(500) - .withRegionName(regionName) - - /* - * RecordProcessorFactory creates impls of IRecordProcessor. - * IRecordProcessor adapts the KCL to our Spark KinesisReceiver via the - * IRecordProcessor.processRecords() method. - * We're using our custom KinesisRecordProcessor in this case. - */ - val recordProcessorFactory = new IRecordProcessorFactory { - override def createProcessor: IRecordProcessor = new KinesisRecordProcessor(receiver, - workerId, new KinesisCheckpointState(checkpointInterval)) - } - - worker = new Worker(recordProcessorFactory, kinesisClientLibConfiguration) - workerThread = new Thread() { - override def run(): Unit = { - try { - worker.run() - } catch { - case NonFatal(e) => - restart("Error running the KCL worker in Receiver", e) - } - } - } - - blockIdToSeqNumRanges.clear() - blockGenerator.start() - - workerThread.setName(s"Kinesis Receiver ${streamId}") - workerThread.setDaemon(true) - workerThread.start() - logInfo(s"Started receiver with workerId $workerId") - } - - /** - * This is called when the KinesisReceiver stops. - * The KCL worker.shutdown() method stops the receiving/processing threads. - * The KCL will do its best to drain and checkpoint any in-flight records upon shutdown. - */ - override def onStop() { - if (workerThread != null) { - if (worker != null) { - worker.shutdown() - worker = null - } - workerThread.join() - workerThread = null - logInfo(s"Stopped receiver for workerId $workerId") - } - workerId = null - } - - /** Add records of the given shard to the current block being generated */ - private[kinesis] def addRecords(shardId: String, records: java.util.List[Record]): Unit = { - if (records.size > 0) { - val dataIterator = records.iterator().asScala.map(messageHandler) - val metadata = SequenceNumberRange(streamName, shardId, - records.get(0).getSequenceNumber(), records.get(records.size() - 1).getSequenceNumber()) - blockGenerator.addMultipleDataWithCallback(dataIterator, metadata) - - } - } - - /** Get the latest sequence number for the given shard that can be checkpointed through KCL */ - private[kinesis] def getLatestSeqNumToCheckpoint(shardId: String): Option[String] = { - shardIdToLatestStoredSeqNum.get(shardId) - } - - /** - * Remember the range of sequence numbers that was added to the currently active block. - * Internally, this is synchronized with `finalizeRangesForCurrentBlock()`. - */ - private def rememberAddedRange(range: SequenceNumberRange): Unit = { - seqNumRangesInCurrentBlock += range - } - - /** - * Finalize the ranges added to the block that was active and prepare the ranges buffer - * for next block. Internally, this is synchronized with `rememberAddedRange()`. - */ - private def finalizeRangesForCurrentBlock(blockId: StreamBlockId): Unit = { - blockIdToSeqNumRanges(blockId) = SequenceNumberRanges(seqNumRangesInCurrentBlock.toArray) - seqNumRangesInCurrentBlock.clear() - logDebug(s"Generated block $blockId has $blockIdToSeqNumRanges") - } - - /** Store the block along with its associated ranges */ - private def storeBlockWithRanges( - blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[T]): Unit = { - val rangesToReportOption = blockIdToSeqNumRanges.remove(blockId) - if (rangesToReportOption.isEmpty) { - stop("Error while storing block into Spark, could not find sequence number ranges " + - s"for block $blockId") - return - } - - val rangesToReport = rangesToReportOption.get - var attempt = 0 - var stored = false - var throwable: Throwable = null - while (!stored && attempt <= 3) { - try { - store(arrayBuffer, rangesToReport) - stored = true - } catch { - case NonFatal(th) => - attempt += 1 - throwable = th - } - } - if (!stored) { - stop("Error while storing block into Spark", throwable) - } - - // Update the latest sequence number that have been successfully stored for each shard - // Note that we are doing this sequentially because the array of sequence number ranges - // is assumed to be - rangesToReport.ranges.foreach { range => - shardIdToLatestStoredSeqNum(range.shardId) = range.toSeqNumber - } - } - - /** - * If AWS credential is provided, return a AWSCredentialProvider returning that credential. - * Otherwise, return the DefaultAWSCredentialsProviderChain. - */ - private def resolveAWSCredentialsProvider(): AWSCredentialsProvider = { - awsCredentialsOption match { - case Some(awsCredentials) => - logInfo("Using provided AWS credentials") - new AWSCredentialsProvider { - override def getCredentials: AWSCredentials = awsCredentials - override def refresh(): Unit = { } - } - case None => - logInfo("Using DefaultAWSCredentialsProviderChain") - new DefaultAWSCredentialsProviderChain() - } - } - - - /** - * Class to handle blocks generated by this receiver's block generator. Specifically, in - * the context of the Kinesis Receiver, this handler does the following. - * - * - When an array of records is added to the current active block in the block generator, - * this handler keeps track of the corresponding sequence number range. - * - When the currently active block is ready to sealed (not more records), this handler - * keep track of the list of ranges added into this block in another H - */ - private class GeneratedBlockHandler extends BlockGeneratorListener { - - /** - * Callback method called after a data item is added into the BlockGenerator. - * The data addition, block generation, and calls to onAddData and onGenerateBlock - * are all synchronized through the same lock. - */ - def onAddData(data: Any, metadata: Any): Unit = { - rememberAddedRange(metadata.asInstanceOf[SequenceNumberRange]) - } - - /** - * Callback method called after a block has been generated. - * The data addition, block generation, and calls to onAddData and onGenerateBlock - * are all synchronized through the same lock. - */ - def onGenerateBlock(blockId: StreamBlockId): Unit = { - finalizeRangesForCurrentBlock(blockId) - } - - /** Callback method called when a block is ready to be pushed / stored. */ - def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { - storeBlockWithRanges(blockId, - arrayBuffer.asInstanceOf[mutable.ArrayBuffer[T]]) - } - - /** Callback called in case of any error in internal of the BlockGenerator */ - def onError(message: String, throwable: Throwable): Unit = { - reportError(message, throwable) - } - } -} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala deleted file mode 100644 index 1d5178790ec4..000000000000 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala +++ /dev/null @@ -1,212 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.streaming.kinesis - -import java.util.List - -import scala.util.Random -import scala.util.control.NonFatal - -import com.amazonaws.services.kinesis.clientlibrary.exceptions.{InvalidStateException, KinesisClientLibDependencyException, ShutdownException, ThrottlingException} -import com.amazonaws.services.kinesis.clientlibrary.interfaces.{IRecordProcessor, IRecordProcessorCheckpointer} -import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason -import com.amazonaws.services.kinesis.model.Record - -import org.apache.spark.Logging - -/** - * Kinesis-specific implementation of the Kinesis Client Library (KCL) IRecordProcessor. - * This implementation operates on the Array[Byte] from the KinesisReceiver. - * The Kinesis Worker creates an instance of this KinesisRecordProcessor for each - * shard in the Kinesis stream upon startup. This is normally done in separate threads, - * but the KCLs within the KinesisReceivers will balance themselves out if you create - * multiple Receivers. - * - * @param receiver Kinesis receiver - * @param workerId for logging purposes - * @param checkpointState represents the checkpoint state including the next checkpoint time. - * It's injected here for mocking purposes. - */ -private[kinesis] class KinesisRecordProcessor[T]( - receiver: KinesisReceiver[T], - workerId: String, - checkpointState: KinesisCheckpointState) extends IRecordProcessor with Logging { - - // shardId to be populated during initialize() - @volatile - private var shardId: String = _ - - /** - * The Kinesis Client Library calls this method during IRecordProcessor initialization. - * - * @param shardId assigned by the KCL to this particular RecordProcessor. - */ - override def initialize(shardId: String) { - this.shardId = shardId - logInfo(s"Initialized workerId $workerId with shardId $shardId") - } - - /** - * This method is called by the KCL when a batch of records is pulled from the Kinesis stream. - * This is the record-processing bridge between the KCL's IRecordProcessor.processRecords() - * and Spark Streaming's Receiver.store(). - * - * @param batch list of records from the Kinesis stream shard - * @param checkpointer used to update Kinesis when this batch has been processed/stored - * in the DStream - */ - override def processRecords(batch: List[Record], checkpointer: IRecordProcessorCheckpointer) { - if (!receiver.isStopped()) { - try { - receiver.addRecords(shardId, batch) - logDebug(s"Stored: Worker $workerId stored ${batch.size} records for shardId $shardId") - - /* - * - * Checkpoint the sequence number of the last record successfully stored. - * Note that in this current implementation, the checkpointing occurs only when after - * checkpointIntervalMillis from the last checkpoint, AND when there is new record - * to process. This leads to the checkpointing lagging behind what records have been - * stored by the receiver. Ofcourse, this can lead records processed more than once, - * under failures and restarts. - * - * TODO: Instead of checkpointing here, run a separate timer task to perform - * checkpointing so that it checkpoints in a timely manner independent of whether - * new records are available or not. - */ - if (checkpointState.shouldCheckpoint()) { - receiver.getLatestSeqNumToCheckpoint(shardId).foreach { latestSeqNum => - /* Perform the checkpoint */ - KinesisRecordProcessor.retryRandom(checkpointer.checkpoint(latestSeqNum), 4, 100) - - /* Update the next checkpoint time */ - checkpointState.advanceCheckpoint() - - logDebug(s"Checkpoint: WorkerId $workerId completed checkpoint of ${batch.size}" + - s" records for shardId $shardId") - logDebug(s"Checkpoint: Next checkpoint is at " + - s" ${checkpointState.checkpointClock.getTimeMillis()} for shardId $shardId") - } - } - } catch { - case NonFatal(e) => { - /* - * If there is a failure within the batch, the batch will not be checkpointed. - * This will potentially cause records since the last checkpoint to be processed - * more than once. - */ - logError(s"Exception: WorkerId $workerId encountered and exception while storing " + - " or checkpointing a batch for workerId $workerId and shardId $shardId.", e) - - /* Rethrow the exception to the Kinesis Worker that is managing this RecordProcessor. */ - throw e - } - } - } else { - /* RecordProcessor has been stopped. */ - logInfo(s"Stopped: KinesisReceiver has stopped for workerId $workerId" + - s" and shardId $shardId. No more records will be processed.") - } - } - - /** - * Kinesis Client Library is shutting down this Worker for 1 of 2 reasons: - * 1) the stream is resharding by splitting or merging adjacent shards - * (ShutdownReason.TERMINATE) - * 2) the failed or latent Worker has stopped sending heartbeats for whatever reason - * (ShutdownReason.ZOMBIE) - * - * @param checkpointer used to perform a Kinesis checkpoint for ShutdownReason.TERMINATE - * @param reason for shutdown (ShutdownReason.TERMINATE or ShutdownReason.ZOMBIE) - */ - override def shutdown(checkpointer: IRecordProcessorCheckpointer, reason: ShutdownReason) { - logInfo(s"Shutdown: Shutting down workerId $workerId with reason $reason") - reason match { - /* - * TERMINATE Use Case. Checkpoint. - * Checkpoint to indicate that all records from the shard have been drained and processed. - * It's now OK to read from the new shards that resulted from a resharding event. - */ - case ShutdownReason.TERMINATE => - val latestSeqNumToCheckpointOption = receiver.getLatestSeqNumToCheckpoint(shardId) - if (latestSeqNumToCheckpointOption.nonEmpty) { - KinesisRecordProcessor.retryRandom( - checkpointer.checkpoint(latestSeqNumToCheckpointOption.get), 4, 100) - } - - /* - * ZOMBIE Use Case. NoOp. - * No checkpoint because other workers may have taken over and already started processing - * the same records. - * This may lead to records being processed more than once. - */ - case ShutdownReason.ZOMBIE => - - /* Unknown reason. NoOp */ - case _ => - } - } -} - -private[kinesis] object KinesisRecordProcessor extends Logging { - /** - * Retry the given amount of times with a random backoff time (millis) less than the - * given maxBackOffMillis - * - * @param expression expression to evalute - * @param numRetriesLeft number of retries left - * @param maxBackOffMillis: max millis between retries - * - * @return evaluation of the given expression - * @throws Unretryable exception, unexpected exception, - * or any exception that persists after numRetriesLeft reaches 0 - */ - @annotation.tailrec - def retryRandom[T](expression: => T, numRetriesLeft: Int, maxBackOffMillis: Int): T = { - util.Try { expression } match { - /* If the function succeeded, evaluate to x. */ - case util.Success(x) => x - /* If the function failed, either retry or throw the exception */ - case util.Failure(e) => e match { - /* Retry: Throttling or other Retryable exception has occurred */ - case _: ThrottlingException | _: KinesisClientLibDependencyException if numRetriesLeft > 1 - => { - val backOffMillis = Random.nextInt(maxBackOffMillis) - Thread.sleep(backOffMillis) - logError(s"Retryable Exception: Random backOffMillis=${backOffMillis}", e) - retryRandom(expression, numRetriesLeft - 1, maxBackOffMillis) - } - /* Throw: Shutdown has been requested by the Kinesis Client Library. */ - case _: ShutdownException => { - logError(s"ShutdownException: Caught shutdown exception, skipping checkpoint.", e) - throw e - } - /* Throw: Non-retryable exception has occurred with the Kinesis Client Library */ - case _: InvalidStateException => { - logError(s"InvalidStateException: Cannot save checkpoint to the DynamoDB table used" + - s" by the Amazon Kinesis Client Library. Table likely doesn't exist.", e) - throw e - } - /* Throw: Unexpected exception has occurred */ - case _ => { - logError(s"Unexpected, non-retryable exception.", e) - throw e - } - } - } - } -} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala deleted file mode 100644 index 634bf9452107..000000000000 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala +++ /dev/null @@ -1,235 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kinesis - -import java.nio.ByteBuffer -import java.util.concurrent.TimeUnit - -import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer -import scala.util.{Failure, Random, Success, Try} - -import com.amazonaws.auth.{AWSCredentials, DefaultAWSCredentialsProviderChain} -import com.amazonaws.regions.RegionUtils -import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient -import com.amazonaws.services.dynamodbv2.document.DynamoDB -import com.amazonaws.services.kinesis.AmazonKinesisClient -import com.amazonaws.services.kinesis.model._ - -import org.apache.spark.Logging - -/** - * Shared utility methods for performing Kinesis tests that actually transfer data - */ -private[kinesis] class KinesisTestUtils extends Logging { - - val endpointUrl = KinesisTestUtils.endpointUrl - val regionName = RegionUtils.getRegionByEndpoint(endpointUrl).getName() - val streamShardCount = 2 - - private val createStreamTimeoutSeconds = 300 - private val describeStreamPollTimeSeconds = 1 - - @volatile - private var streamCreated = false - - @volatile - private var _streamName: String = _ - - private lazy val kinesisClient = { - val client = new AmazonKinesisClient(KinesisTestUtils.getAWSCredentials()) - client.setEndpoint(endpointUrl) - client - } - - private lazy val dynamoDB = { - val dynamoDBClient = new AmazonDynamoDBClient(new DefaultAWSCredentialsProviderChain()) - dynamoDBClient.setRegion(RegionUtils.getRegion(regionName)) - new DynamoDB(dynamoDBClient) - } - - def streamName: String = { - require(streamCreated, "Stream not yet created, call createStream() to create one") - _streamName - } - - def createStream(): Unit = { - require(!streamCreated, "Stream already created") - _streamName = findNonExistentStreamName() - - // Create a stream. The number of shards determines the provisioned throughput. - logInfo(s"Creating stream ${_streamName}") - val createStreamRequest = new CreateStreamRequest() - createStreamRequest.setStreamName(_streamName) - createStreamRequest.setShardCount(2) - kinesisClient.createStream(createStreamRequest) - - // The stream is now being created. Wait for it to become active. - waitForStreamToBeActive(_streamName) - streamCreated = true - logInfo(s"Created stream ${_streamName}") - } - - /** - * Push data to Kinesis stream and return a map of - * shardId -> seq of (data, seq number) pushed to corresponding shard - */ - def pushData(testData: Seq[Int]): Map[String, Seq[(Int, String)]] = { - require(streamCreated, "Stream not yet created, call createStream() to create one") - val shardIdToSeqNumbers = new mutable.HashMap[String, ArrayBuffer[(Int, String)]]() - - testData.foreach { num => - val str = num.toString - val putRecordRequest = new PutRecordRequest().withStreamName(streamName) - .withData(ByteBuffer.wrap(str.getBytes())) - .withPartitionKey(str) - - val putRecordResult = kinesisClient.putRecord(putRecordRequest) - val shardId = putRecordResult.getShardId - val seqNumber = putRecordResult.getSequenceNumber() - val sentSeqNumbers = shardIdToSeqNumbers.getOrElseUpdate(shardId, - new ArrayBuffer[(Int, String)]()) - sentSeqNumbers += ((num, seqNumber)) - } - - logInfo(s"Pushed $testData:\n\t ${shardIdToSeqNumbers.mkString("\n\t")}") - shardIdToSeqNumbers.toMap - } - - /** - * Expose a Python friendly API. - */ - def pushData(testData: java.util.List[Int]): Unit = { - pushData(testData.asScala) - } - - def deleteStream(): Unit = { - try { - if (streamCreated) { - kinesisClient.deleteStream(streamName) - } - } catch { - case e: Exception => - logWarning(s"Could not delete stream $streamName") - } - } - - def deleteDynamoDBTable(tableName: String): Unit = { - try { - val table = dynamoDB.getTable(tableName) - table.delete() - table.waitForDelete() - } catch { - case e: Exception => - logWarning(s"Could not delete DynamoDB table $tableName") - } - } - - private def describeStream(streamNameToDescribe: String): Option[StreamDescription] = { - try { - val describeStreamRequest = new DescribeStreamRequest().withStreamName(streamNameToDescribe) - val desc = kinesisClient.describeStream(describeStreamRequest).getStreamDescription() - Some(desc) - } catch { - case rnfe: ResourceNotFoundException => - None - } - } - - private def findNonExistentStreamName(): String = { - var testStreamName: String = null - do { - Thread.sleep(TimeUnit.SECONDS.toMillis(describeStreamPollTimeSeconds)) - testStreamName = s"KinesisTestUtils-${math.abs(Random.nextLong())}" - } while (describeStream(testStreamName).nonEmpty) - testStreamName - } - - private def waitForStreamToBeActive(streamNameToWaitFor: String): Unit = { - val startTime = System.currentTimeMillis() - val endTime = startTime + TimeUnit.SECONDS.toMillis(createStreamTimeoutSeconds) - while (System.currentTimeMillis() < endTime) { - Thread.sleep(TimeUnit.SECONDS.toMillis(describeStreamPollTimeSeconds)) - describeStream(streamNameToWaitFor).foreach { description => - val streamStatus = description.getStreamStatus() - logDebug(s"\t- current state: $streamStatus\n") - if ("ACTIVE".equals(streamStatus)) { - return - } - } - } - require(false, s"Stream $streamName never became active") - } -} - -private[kinesis] object KinesisTestUtils { - - val envVarNameForEnablingTests = "ENABLE_KINESIS_TESTS" - val endVarNameForEndpoint = "KINESIS_TEST_ENDPOINT_URL" - val defaultEndpointUrl = "https://kinesis.us-west-2.amazonaws.com" - - lazy val shouldRunTests = { - val isEnvSet = sys.env.get(envVarNameForEnablingTests) == Some("1") - if (isEnvSet) { - // scalastyle:off println - // Print this so that they are easily visible on the console and not hidden in the log4j logs. - println( - s""" - |Kinesis tests that actually send data has been enabled by setting the environment - |variable $envVarNameForEnablingTests to 1. This will create Kinesis Streams and - |DynamoDB tables in AWS. Please be aware that this may incur some AWS costs. - |By default, the tests use the endpoint URL $defaultEndpointUrl to create Kinesis streams. - |To change this endpoint URL to a different region, you can set the environment variable - |$endVarNameForEndpoint to the desired endpoint URL - |(e.g. $endVarNameForEndpoint="https://kinesis.us-west-2.amazonaws.com"). - """.stripMargin) - // scalastyle:on println - } - isEnvSet - } - - lazy val endpointUrl = { - val url = sys.env.getOrElse(endVarNameForEndpoint, defaultEndpointUrl) - // scalastyle:off println - // Print this so that they are easily visible on the console and not hidden in the log4j logs. - println(s"Using endpoint URL $url for creating Kinesis streams for tests.") - // scalastyle:on println - url - } - - def isAWSCredentialsPresent: Boolean = { - Try { new DefaultAWSCredentialsProviderChain().getCredentials() }.isSuccess - } - - def getAWSCredentials(): AWSCredentials = { - assert(shouldRunTests, - "Kinesis test not enabled, should not attempt to get AWS credentials") - Try { new DefaultAWSCredentialsProviderChain().getCredentials() } match { - case Success(cred) => cred - case Failure(e) => - throw new Exception( - s""" - |Kinesis tests enabled using environment variable $envVarNameForEnablingTests - |but could not find AWS credentials. Please follow instructions in AWS documentation - |to set the credentials in your system such that the DefaultAWSCredentialsProviderChain - |can find the credentials. - """.stripMargin) - } - } -} diff --git a/extras/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java b/extras/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java deleted file mode 100644 index 3f0f6793d2d2..000000000000 --- a/extras/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kinesis; - -import com.amazonaws.services.kinesis.model.Record; -import org.junit.Test; - -import org.apache.spark.api.java.function.Function; -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.Duration; -import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaDStream; - -import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; - -import java.nio.ByteBuffer; - -/** - * Demonstrate the use of the KinesisUtils Java API - */ -public class JavaKinesisStreamSuite extends LocalJavaStreamingContext { - @Test - public void testKinesisStream() { - // Tests the API, does not actually test data receiving - JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "mySparkStream", - "https://kinesis.us-west-2.amazonaws.com", new Duration(2000), - InitialPositionInStream.LATEST, StorageLevel.MEMORY_AND_DISK_2()); - - ssc.stop(); - } - - - private static Function handler = new Function() { - @Override - public String call(Record record) { - return record.getPartitionKey() + "-" + record.getSequenceNumber(); - } - }; - - @Test - public void testCustomHandler() { - // Tests the API, does not actually test data receiving - JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "testApp", "mySparkStream", - "https://kinesis.us-west-2.amazonaws.com", "us-west-2", InitialPositionInStream.LATEST, - new Duration(2000), StorageLevel.MEMORY_AND_DISK_2(), handler, String.class); - - ssc.stop(); - } -} diff --git a/extras/kinesis-asl/src/test/resources/log4j.properties b/extras/kinesis-asl/src/test/resources/log4j.properties deleted file mode 100644 index edbecdae9209..000000000000 --- a/extras/kinesis-asl/src/test/resources/log4j.properties +++ /dev/null @@ -1,27 +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. -# - -# Set everything to be logged to the file target/unit-tests.log -log4j.rootCategory=INFO, file -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=true -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN 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 deleted file mode 100644 index 17ab444704f4..000000000000 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala +++ /dev/null @@ -1,268 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.streaming.kinesis - -import java.nio.ByteBuffer -import java.nio.charset.StandardCharsets -import java.util.Arrays - -import com.amazonaws.services.kinesis.clientlibrary.exceptions._ -import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer -import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason -import com.amazonaws.services.kinesis.model.Record -import org.mockito.Matchers._ -import org.mockito.Mockito._ -import org.scalatest.mock.MockitoSugar -import org.scalatest.{BeforeAndAfter, Matchers} - -import org.apache.spark.streaming.{Milliseconds, TestSuiteBase} -import org.apache.spark.util.{Clock, ManualClock, Utils} - -/** - * Suite of Kinesis streaming receiver tests focusing mostly on the KinesisRecordProcessor - */ -class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAfter - with MockitoSugar { - - val app = "TestKinesisReceiver" - val stream = "mySparkStream" - val endpoint = "endpoint-url" - val workerId = "dummyWorkerId" - val shardId = "dummyShardId" - val seqNum = "dummySeqNum" - val someSeqNum = Some(seqNum) - - val record1 = new Record() - record1.setData(ByteBuffer.wrap("Spark In Action".getBytes(StandardCharsets.UTF_8))) - val record2 = new Record() - record2.setData(ByteBuffer.wrap("Learning Spark".getBytes(StandardCharsets.UTF_8))) - val batch = Arrays.asList(record1, record2) - - var receiverMock: KinesisReceiver[Array[Byte]] = _ - var checkpointerMock: IRecordProcessorCheckpointer = _ - var checkpointClockMock: ManualClock = _ - var checkpointStateMock: KinesisCheckpointState = _ - var currentClockMock: Clock = _ - - override def beforeFunction(): Unit = { - receiverMock = mock[KinesisReceiver[Array[Byte]]] - checkpointerMock = mock[IRecordProcessorCheckpointer] - checkpointClockMock = mock[ManualClock] - checkpointStateMock = mock[KinesisCheckpointState] - 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("check serializability of SerializableAWSCredentials") { - Utils.deserialize[SerializableAWSCredentials]( - Utils.serialize(new SerializableAWSCredentials("x", "y"))) - } - - test("process records including store and checkpoint") { - when(receiverMock.isStopped()).thenReturn(false) - when(receiverMock.getLatestSeqNumToCheckpoint(shardId)).thenReturn(someSeqNum) - when(checkpointStateMock.shouldCheckpoint()).thenReturn(true) - - val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock) - recordProcessor.initialize(shardId) - recordProcessor.processRecords(batch, checkpointerMock) - - verify(receiverMock, times(1)).isStopped() - verify(receiverMock, times(1)).addRecords(shardId, batch) - verify(receiverMock, times(1)).getLatestSeqNumToCheckpoint(shardId) - verify(checkpointStateMock, times(1)).shouldCheckpoint() - verify(checkpointerMock, times(1)).checkpoint(anyString) - verify(checkpointStateMock, times(1)).advanceCheckpoint() - } - - test("shouldn't store and checkpoint when receiver is stopped") { - when(receiverMock.isStopped()).thenReturn(true) - - val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock) - recordProcessor.processRecords(batch, checkpointerMock) - - verify(receiverMock, times(1)).isStopped() - verify(receiverMock, never).addRecords(anyString, anyListOf(classOf[Record])) - verify(checkpointerMock, never).checkpoint(anyString) - } - - test("shouldn't checkpoint when exception occurs during store") { - when(receiverMock.isStopped()).thenReturn(false) - when( - receiverMock.addRecords(anyString, anyListOf(classOf[Record])) - ).thenThrow(new RuntimeException()) - - intercept[RuntimeException] { - val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock) - recordProcessor.initialize(shardId) - recordProcessor.processRecords(batch, checkpointerMock) - } - - verify(receiverMock, times(1)).isStopped() - verify(receiverMock, times(1)).addRecords(shardId, batch) - verify(checkpointerMock, never).checkpoint(anyString) - } - - test("should set checkpoint time to currentTime + checkpoint interval upon instantiation") { - when(currentClockMock.getTimeMillis()).thenReturn(0) - - val checkpointIntervalMillis = 10 - val checkpointState = - new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock) - assert(checkpointState.checkpointClock.getTimeMillis() == checkpointIntervalMillis) - - verify(currentClockMock, times(1)).getTimeMillis() - } - - test("should checkpoint if we have exceeded the checkpoint interval") { - when(currentClockMock.getTimeMillis()).thenReturn(0) - - val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MinValue), currentClockMock) - assert(checkpointState.shouldCheckpoint()) - - verify(currentClockMock, times(1)).getTimeMillis() - } - - test("shouldn't checkpoint if we have not exceeded the checkpoint interval") { - when(currentClockMock.getTimeMillis()).thenReturn(0) - - val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MaxValue), currentClockMock) - assert(!checkpointState.shouldCheckpoint()) - - verify(currentClockMock, times(1)).getTimeMillis() - } - - test("should add to time when advancing checkpoint") { - when(currentClockMock.getTimeMillis()).thenReturn(0) - - val checkpointIntervalMillis = 10 - val checkpointState = - new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock) - assert(checkpointState.checkpointClock.getTimeMillis() == checkpointIntervalMillis) - checkpointState.advanceCheckpoint() - assert(checkpointState.checkpointClock.getTimeMillis() == (2 * checkpointIntervalMillis)) - - verify(currentClockMock, times(1)).getTimeMillis() - } - - test("shutdown should checkpoint if the reason is TERMINATE") { - when(receiverMock.getLatestSeqNumToCheckpoint(shardId)).thenReturn(someSeqNum) - - val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock) - recordProcessor.initialize(shardId) - recordProcessor.shutdown(checkpointerMock, ShutdownReason.TERMINATE) - - verify(receiverMock, times(1)).getLatestSeqNumToCheckpoint(shardId) - verify(checkpointerMock, times(1)).checkpoint(anyString) - } - - test("shutdown should not checkpoint if the reason is something other than TERMINATE") { - when(receiverMock.getLatestSeqNumToCheckpoint(shardId)).thenReturn(someSeqNum) - - val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, checkpointStateMock) - recordProcessor.initialize(shardId) - recordProcessor.shutdown(checkpointerMock, ShutdownReason.ZOMBIE) - recordProcessor.shutdown(checkpointerMock, null) - - verify(checkpointerMock, never).checkpoint(anyString) - } - - test("retry success on first attempt") { - val expectedIsStopped = false - 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 - 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 - 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") { - 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") { - 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") { - 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" - 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/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala deleted file mode 100644 index ba84e557dfcc..000000000000 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala +++ /dev/null @@ -1,289 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kinesis - -import scala.collection.mutable -import scala.concurrent.duration._ -import scala.language.postfixOps -import scala.util.Random - -import com.amazonaws.regions.RegionUtils -import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream -import com.amazonaws.services.kinesis.model.Record -import org.scalatest.Matchers._ -import org.scalatest.concurrent.Eventually -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} - -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.{StorageLevel, StreamBlockId} -import org.apache.spark.streaming._ -import org.apache.spark.streaming.dstream.ReceiverInputDStream -import org.apache.spark.streaming.kinesis.KinesisTestUtils._ -import org.apache.spark.streaming.receiver.BlockManagerBasedStoreResult -import org.apache.spark.streaming.scheduler.ReceivedBlockInfo -import org.apache.spark.util.Utils -import org.apache.spark.{SparkConf, SparkContext} - -class KinesisStreamSuite extends KinesisFunSuite - with Eventually with BeforeAndAfter with BeforeAndAfterAll { - - // This is the name that KCL will use to save metadata to DynamoDB - private val appName = s"KinesisStreamSuite-${math.abs(Random.nextLong())}" - private val batchDuration = Seconds(1) - - // Dummy parameters for API testing - private val dummyEndpointUrl = defaultEndpointUrl - private val dummyRegionName = RegionUtils.getRegionByEndpoint(dummyEndpointUrl).getName() - private val dummyAWSAccessKey = "dummyAccessKey" - private val dummyAWSSecretKey = "dummySecretKey" - - private var testUtils: KinesisTestUtils = null - private var ssc: StreamingContext = null - private var sc: SparkContext = null - - override def beforeAll(): Unit = { - val conf = new SparkConf() - .setMaster("local[4]") - .setAppName("KinesisStreamSuite") // Setting Spark app name to Kinesis app name - sc = new SparkContext(conf) - - runIfTestsEnabled("Prepare KinesisTestUtils") { - testUtils = new KinesisTestUtils() - testUtils.createStream() - } - } - - override def afterAll(): Unit = { - if (ssc != null) { - ssc.stop() - } - if (sc != null) { - sc.stop() - } - if (testUtils != null) { - // Delete the Kinesis stream as well as the DynamoDB table generated by - // Kinesis Client Library when consuming the stream - testUtils.deleteStream() - testUtils.deleteDynamoDBTable(appName) - } - } - - before { - ssc = new StreamingContext(sc, batchDuration) - } - - after { - if (ssc != null) { - ssc.stop(stopSparkContext = false) - ssc = null - } - if (testUtils != null) { - testUtils.deleteDynamoDBTable(appName) - } - } - - test("KinesisUtils API") { - // Tests the API, does not actually test data receiving - val kinesisStream1 = KinesisUtils.createStream(ssc, "mySparkStream", - dummyEndpointUrl, Seconds(2), - InitialPositionInStream.LATEST, StorageLevel.MEMORY_AND_DISK_2) - val kinesisStream2 = KinesisUtils.createStream(ssc, "myAppNam", "mySparkStream", - dummyEndpointUrl, dummyRegionName, - InitialPositionInStream.LATEST, Seconds(2), StorageLevel.MEMORY_AND_DISK_2) - val kinesisStream3 = KinesisUtils.createStream(ssc, "myAppNam", "mySparkStream", - dummyEndpointUrl, dummyRegionName, - InitialPositionInStream.LATEST, Seconds(2), StorageLevel.MEMORY_AND_DISK_2, - dummyAWSAccessKey, dummyAWSSecretKey) - } - - test("RDD generation") { - val inputStream = KinesisUtils.createStream(ssc, appName, "dummyStream", - dummyEndpointUrl, dummyRegionName, InitialPositionInStream.LATEST, Seconds(2), - StorageLevel.MEMORY_AND_DISK_2, dummyAWSAccessKey, dummyAWSSecretKey) - assert(inputStream.isInstanceOf[KinesisInputDStream[Array[Byte]]]) - - val kinesisStream = inputStream.asInstanceOf[KinesisInputDStream[Array[Byte]]] - val time = Time(1000) - - // Generate block info data for testing - val seqNumRanges1 = SequenceNumberRanges( - SequenceNumberRange("fakeStream", "fakeShardId", "xxx", "yyy")) - val blockId1 = StreamBlockId(kinesisStream.id, 123) - val blockInfo1 = ReceivedBlockInfo( - 0, None, Some(seqNumRanges1), new BlockManagerBasedStoreResult(blockId1, None)) - - val seqNumRanges2 = SequenceNumberRanges( - SequenceNumberRange("fakeStream", "fakeShardId", "aaa", "bbb")) - val blockId2 = StreamBlockId(kinesisStream.id, 345) - val blockInfo2 = ReceivedBlockInfo( - 0, None, Some(seqNumRanges2), new BlockManagerBasedStoreResult(blockId2, None)) - - // Verify that the generated KinesisBackedBlockRDD has the all the right information - val blockInfos = Seq(blockInfo1, blockInfo2) - val nonEmptyRDD = kinesisStream.createBlockRDD(time, blockInfos) - nonEmptyRDD shouldBe a [KinesisBackedBlockRDD[Array[Byte]]] - val kinesisRDD = nonEmptyRDD.asInstanceOf[KinesisBackedBlockRDD[Array[Byte]]] - assert(kinesisRDD.regionName === dummyRegionName) - assert(kinesisRDD.endpointUrl === dummyEndpointUrl) - assert(kinesisRDD.retryTimeoutMs === batchDuration.milliseconds) - assert(kinesisRDD.awsCredentialsOption === - Some(SerializableAWSCredentials(dummyAWSAccessKey, dummyAWSSecretKey))) - assert(nonEmptyRDD.partitions.size === blockInfos.size) - nonEmptyRDD.partitions.foreach { _ shouldBe a [KinesisBackedBlockRDDPartition] } - val partitions = nonEmptyRDD.partitions.map { - _.asInstanceOf[KinesisBackedBlockRDDPartition] }.toSeq - assert(partitions.map { _.seqNumberRanges } === Seq(seqNumRanges1, seqNumRanges2)) - assert(partitions.map { _.blockId } === Seq(blockId1, blockId2)) - assert(partitions.forall { _.isBlockIdValid === true }) - - // Verify that KinesisBackedBlockRDD is generated even when there are no blocks - val emptyRDD = kinesisStream.createBlockRDD(time, Seq.empty) - emptyRDD shouldBe a [KinesisBackedBlockRDD[Array[Byte]]] - emptyRDD.partitions shouldBe empty - - // Verify that the KinesisBackedBlockRDD has isBlockValid = false when blocks are invalid - blockInfos.foreach { _.setBlockIdInvalid() } - kinesisStream.createBlockRDD(time, blockInfos).partitions.foreach { partition => - assert(partition.asInstanceOf[KinesisBackedBlockRDDPartition].isBlockIdValid === false) - } - } - - - /** - * Test the stream by sending data to a Kinesis stream and receiving from it. - * This test is not run by default as it requires AWS credentials that the test - * environment may not have. Even if there is AWS credentials available, the user - * may not want to run these tests to avoid the Kinesis costs. To enable this test, - * you must have AWS credentials available through the default AWS provider chain, - * and you have to set the system environment variable RUN_KINESIS_TESTS=1 . - */ - testIfEnabled("basic operation") { - val awsCredentials = KinesisTestUtils.getAWSCredentials() - val stream = KinesisUtils.createStream(ssc, appName, testUtils.streamName, - testUtils.endpointUrl, testUtils.regionName, InitialPositionInStream.LATEST, - Seconds(10), StorageLevel.MEMORY_ONLY, - awsCredentials.getAWSAccessKeyId, awsCredentials.getAWSSecretKey) - - val collected = new mutable.HashSet[Int] with mutable.SynchronizedSet[Int] - stream.map { bytes => new String(bytes).toInt }.foreachRDD { rdd => - collected ++= rdd.collect() - logInfo("Collected = " + rdd.collect().toSeq.mkString(", ")) - } - ssc.start() - - val testData = 1 to 10 - eventually(timeout(120 seconds), interval(10 second)) { - testUtils.pushData(testData) - assert(collected === testData.toSet, "\nData received does not match data sent") - } - ssc.stop(stopSparkContext = false) - } - - testIfEnabled("custom message handling") { - val awsCredentials = KinesisTestUtils.getAWSCredentials() - def addFive(r: Record): Int = new String(r.getData.array()).toInt + 5 - val stream = KinesisUtils.createStream(ssc, appName, testUtils.streamName, - testUtils.endpointUrl, testUtils.regionName, InitialPositionInStream.LATEST, - Seconds(10), StorageLevel.MEMORY_ONLY, addFive, - awsCredentials.getAWSAccessKeyId, awsCredentials.getAWSSecretKey) - - stream shouldBe a [ReceiverInputDStream[Int]] - - val collected = new mutable.HashSet[Int] with mutable.SynchronizedSet[Int] - stream.foreachRDD { rdd => - collected ++= rdd.collect() - logInfo("Collected = " + rdd.collect().toSeq.mkString(", ")) - } - ssc.start() - - val testData = 1 to 10 - eventually(timeout(120 seconds), interval(10 second)) { - testUtils.pushData(testData) - val modData = testData.map(_ + 5) - assert(collected === modData.toSet, "\nData received does not match data sent") - } - ssc.stop(stopSparkContext = false) - } - - testIfEnabled("failure recovery") { - val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) - val checkpointDir = Utils.createTempDir().getAbsolutePath - - ssc = new StreamingContext(sc, Milliseconds(1000)) - ssc.checkpoint(checkpointDir) - - val awsCredentials = KinesisTestUtils.getAWSCredentials() - val collectedData = new mutable.HashMap[Time, (Array[SequenceNumberRanges], Seq[Int])] - with mutable.SynchronizedMap[Time, (Array[SequenceNumberRanges], Seq[Int])] - - val kinesisStream = KinesisUtils.createStream(ssc, appName, testUtils.streamName, - testUtils.endpointUrl, testUtils.regionName, InitialPositionInStream.LATEST, - Seconds(10), StorageLevel.MEMORY_ONLY, - awsCredentials.getAWSAccessKeyId, awsCredentials.getAWSSecretKey) - - // Verify that the generated RDDs are KinesisBackedBlockRDDs, and collect the data in each batch - kinesisStream.foreachRDD((rdd: RDD[Array[Byte]], time: Time) => { - val kRdd = rdd.asInstanceOf[KinesisBackedBlockRDD[Array[Byte]]] - val data = rdd.map { bytes => new String(bytes).toInt }.collect().toSeq - collectedData(time) = (kRdd.arrayOfseqNumberRanges, data) - }) - - ssc.remember(Minutes(60)) // remember all the batches so that they are all saved in checkpoint - ssc.start() - - def numBatchesWithData: Int = collectedData.count(_._2._2.nonEmpty) - - def isCheckpointPresent: Boolean = Checkpoint.getCheckpointFiles(checkpointDir).nonEmpty - - // Run until there are at least 10 batches with some data in them - // If this times out because numBatchesWithData is empty, then its likely that foreachRDD - // function failed with exceptions, and nothing got added to `collectedData` - eventually(timeout(2 minutes), interval(1 seconds)) { - testUtils.pushData(1 to 5) - assert(isCheckpointPresent && numBatchesWithData > 10) - } - ssc.stop(stopSparkContext = true) // stop the SparkContext so that the blocks are not reused - - // Restart the context from checkpoint and verify whether the - logInfo("Restarting from checkpoint") - ssc = new StreamingContext(checkpointDir) - ssc.start() - val recoveredKinesisStream = ssc.graph.getInputStreams().head - - // Verify that the recomputed RDDs are KinesisBackedBlockRDDs with the same sequence ranges - // and return the same data - val times = collectedData.keySet - times.foreach { time => - val (arrayOfSeqNumRanges, data) = collectedData(time) - val rdd = recoveredKinesisStream.getOrCompute(time).get.asInstanceOf[RDD[Array[Byte]]] - rdd shouldBe a [KinesisBackedBlockRDD[Array[Byte]]] - - // Verify the recovered sequence ranges - val kRdd = rdd.asInstanceOf[KinesisBackedBlockRDD[Array[Byte]]] - assert(kRdd.arrayOfseqNumberRanges.size === arrayOfSeqNumRanges.size) - arrayOfSeqNumRanges.zip(kRdd.arrayOfseqNumberRanges).foreach { case (expected, found) => - assert(expected.ranges.toSeq === found.ranges.toSeq) - } - - // Verify the recovered data - assert(rdd.map { bytes => new String(bytes).toInt }.collect().toSeq === data) - } - ssc.stop() - } - -} diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml deleted file mode 100644 index 87a4f05a0596..000000000000 --- a/extras/spark-ganglia-lgpl/pom.xml +++ /dev/null @@ -1,49 +0,0 @@ - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../../pom.xml - - - - org.apache.spark - spark-ganglia-lgpl_2.10 - jar - Spark Ganglia Integration - - - ganglia-lgpl - - - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - - io.dropwizard.metrics - metrics-ganglia - - - diff --git a/graphx/pom.xml b/graphx/pom.xml index 987b831021a5..cb30e4a4af4b 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -20,13 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT + spark-parent_2.11 + 2.3.0-SNAPSHOT ../pom.xml - org.apache.spark - spark-graphx_2.10 + spark-graphx_2.11 graphx @@ -47,6 +46,15 @@ test-jar test + + org.apache.spark + spark-mllib-local_${scala.binary.version} + ${project.version} + + + org.apache.xbean + xbean-asm5-shaded + com.google.guava guava @@ -68,8 +76,20 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + target/scala-${scala.binary.version}/classes diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala index 23430179f12e..3b96a420b34a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala @@ -63,5 +63,5 @@ object EdgeContext { * }}} */ def unapply[VD, ED, A](edge: EdgeContext[VD, ED, A]): Some[(VertexId, VertexId, VD, VD, ED)] = - Some(edge.srcId, edge.dstId, edge.srcAttr, edge.dstAttr, edge.attr) + Some((edge.srcId, edge.dstId, edge.srcAttr, edge.dstAttr, edge.attr)) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index ee7302a1edbf..45526bf062fa 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -24,12 +24,11 @@ import org.apache.spark.Dependency import org.apache.spark.Partition import org.apache.spark.SparkContext import org.apache.spark.TaskContext -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel - import org.apache.spark.graphx.impl.EdgePartition import org.apache.spark.graphx.impl.EdgePartitionBuilder import org.apache.spark.graphx.impl.EdgeRDDImpl +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel /** * `EdgeRDD[ED, VD]` extends `RDD[Edge[ED]]` by storing the edges in columnar format on each diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 869caa340f52..b3a3420b8494 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -54,8 +54,8 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * * @return an RDD containing the edges in this graph * - * @see [[Edge]] for the edge type. - * @see [[Graph#triplets]] to get an RDD which contains all the edges + * @see `Edge` for the edge type. + * @see `Graph#triplets` to get an RDD which contains all the edges * along with their vertex data. * */ @@ -297,7 +297,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab /** * Restricts the graph to only the vertices and edges satisfying the predicates. The resulting - * subgraph satisifies + * subgraph satisfies * * {{{ * V' = {v : for all v in V where vpred(v)} @@ -331,7 +331,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab /** * Merges multiple edges between two vertices into a single edge. For correct results, the graph - * must have been partitioned using [[partitionBy]]. + * must have been partitioned using `partitionBy`. * * @param merge the user-supplied commutative associative function to merge edge attributes * for duplicate edges. @@ -340,55 +340,6 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab */ def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] - /** - * Aggregates values from the neighboring edges and vertices of each vertex. The user supplied - * `mapFunc` function is invoked on each edge of the graph, generating 0 or more "messages" to be - * "sent" to either vertex in the edge. The `reduceFunc` is then used to combine the output of - * the map phase destined to each vertex. - * - * This function is deprecated in 1.2.0 because of SPARK-3936. Use aggregateMessages instead. - * - * @tparam A the type of "message" to be sent to each vertex - * - * @param mapFunc the user defined map function which returns 0 or - * more messages to neighboring vertices - * - * @param reduceFunc the user defined reduce function which should - * be commutative and associative and is used to combine the output - * of the map phase - * - * @param activeSetOpt an efficient way to run the aggregation on a subset of the edges if - * desired. This is done by specifying a set of "active" vertices and an edge direction. The - * `sendMsg` function will then run only on edges connected to active vertices by edges in the - * specified direction. If the direction is `In`, `sendMsg` will only be run on edges with - * destination in the active set. If the direction is `Out`, `sendMsg` will only be run on edges - * originating from vertices in the active set. If the direction is `Either`, `sendMsg` will be - * run on edges with *either* vertex in the active set. If the direction is `Both`, `sendMsg` - * will be run on edges with *both* vertices in the active set. The active set must have the - * same index as the graph's vertices. - * - * @example We can use this function to compute the in-degree of each - * vertex - * {{{ - * val rawGraph: Graph[(),()] = Graph.textFile("twittergraph") - * val inDeg: RDD[(VertexId, Int)] = - * mapReduceTriplets[Int](et => Iterator((et.dst.id, 1)), _ + _) - * }}} - * - * @note By expressing computation at the edge level we achieve - * maximum parallelism. This is one of the core functions in the - * Graph API in that enables neighborhood level computation. For - * example this function can be used to count neighbors satisfying a - * predicate or implement PageRank. - * - */ - @deprecated("use aggregateMessages", "1.2.0") - def mapReduceTriplets[A: ClassTag]( - mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], - reduceFunc: (A, A) => A, - activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) - : VertexRDD[A] - /** * Aggregates values from the neighboring edges and vertices of each vertex. The user-supplied * `sendMsg` function is invoked on each edge of the graph, generating 0 or more messages to be @@ -414,7 +365,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * * @note By expressing computation at the edge level we achieve * maximum parallelism. This is one of the core functions in the - * Graph API in that enables neighborhood level computation. For + * Graph API that enables neighborhood level computation. For * example this function can be used to count neighbors satisfying a * predicate or implement PageRank. * diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala deleted file mode 100644 index 563c948957ec..000000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ /dev/null @@ -1,50 +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.graphx - -import com.esotericsoftware.kryo.Kryo - -import org.apache.spark.serializer.KryoRegistrator -import org.apache.spark.util.BoundedPriorityQueue -import org.apache.spark.util.collection.BitSet - -import org.apache.spark.graphx.impl._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap -import org.apache.spark.util.collection.OpenHashSet - -/** - * Registers GraphX classes with Kryo for improved performance. - */ -@deprecated("Register GraphX classes with Kryo using GraphXUtils.registerKryoClasses", "1.2.0") -class GraphKryoRegistrator extends KryoRegistrator { - - def registerClasses(kryo: Kryo) { - kryo.register(classOf[Edge[Object]]) - kryo.register(classOf[(VertexId, Object)]) - kryo.register(classOf[EdgePartition[Object, Object]]) - kryo.register(classOf[BitSet]) - kryo.register(classOf[VertexIdToIndexMap]) - kryo.register(classOf[VertexAttributeBlock[Object]]) - kryo.register(classOf[PartitionStrategy]) - kryo.register(classOf[BoundedPriorityQueue[Object]]) - kryo.register(classOf[EdgeDirection]) - kryo.register(classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]]) - kryo.register(classOf[OpenHashSet[Int]]) - kryo.register(classOf[OpenHashSet[Long]]) - } -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala index 21187be7678a..f665727ef90d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -17,9 +17,10 @@ package org.apache.spark.graphx -import org.apache.spark.storage.StorageLevel -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.graphx.impl.{EdgePartitionBuilder, GraphImpl} +import org.apache.spark.internal.Logging +import org.apache.spark.storage.StorageLevel /** * Provides utilities for loading [[Graph]]s from files. @@ -31,7 +32,7 @@ object GraphLoader extends Logging { * id and a target id. Skips lines that begin with `#`. * * If desired the edges can be automatically oriented in the positive - * direction (source Id < target Id) by setting `canonicalOrientation` to + * direction (source Id is less than target Id) by setting `canonicalOrientation` to * true. * * @example Loads a file in the following format: diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 9451ff1e5c0e..49e8487dbe93 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -21,10 +21,9 @@ import scala.reflect.ClassTag import scala.util.Random import org.apache.spark.SparkException -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - import org.apache.spark.graphx.lib._ +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.rdd.RDD /** * Contains additional functionality for [[Graph]]. All operations are expressed in terms of the @@ -185,6 +184,15 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali } } + /** + * Remove self edges. + * + * @return a graph with all self edges removed + */ + def removeSelfEdges(): Graph[VD, ED] = { + graph.subgraph(epred = e => e.srcId != e.dstId) + } + /** * Join the vertices with an RDD and then apply a function from the * vertex and RDD entry to a new vertex value. The input table @@ -229,11 +237,11 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * @param preprocess a function to compute new vertex and edge data before filtering * @param epred edge pred to filter on after preprocess, see more details under * [[org.apache.spark.graphx.Graph#subgraph]] - * @param vpred vertex pred to filter on after prerocess, see more details under + * @param vpred vertex pred to filter on after preprocess, see more details under * [[org.apache.spark.graphx.Graph#subgraph]] * @tparam VD2 vertex type the vpred operates on * @tparam ED2 edge type the epred operates on - * @return a subgraph of the orginal graph, with its data unchanged + * @return a subgraph of the original graph, with its data unchanged * * @example This function can be used to filter the graph based on some property, without * changing the vertex and edge values in your program. For example, we could remove the vertices @@ -269,10 +277,10 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali if (Random.nextDouble() < probability) { Some(vidVvals._1) } else { None } } - if (selectedVertices.count > 1) { + if (selectedVertices.count > 0) { found = true val collectedVertices = selectedVertices.collect() - retVal = collectedVertices(Random.nextInt(collectedVertices.size)) + retVal = collectedVertices(Random.nextInt(collectedVertices.length)) } } retVal @@ -282,7 +290,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * Convert bi-directional edges into uni-directional ones. * Some graph algorithms (e.g., TriangleCount) assume that an input graph * has its edges in canonical direction. - * This function rewrites the vertex ids of edges so that srcIds are bigger + * This function rewrites the vertex ids of edges so that srcIds are smaller * than dstIds, and merges the duplicated edges. * * @param mergeFunc the user defined reduce function which should @@ -380,10 +388,19 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * @see [[org.apache.spark.graphx.lib.PageRank$#runUntilConvergenceWithOptions]] */ def personalizedPageRank(src: VertexId, tol: Double, - resetProb: Double = 0.15) : Graph[Double, Double] = { + resetProb: Double = 0.15): Graph[Double, Double] = { PageRank.runUntilConvergenceWithOptions(graph, tol, resetProb, Some(src)) } + /** + * Run parallel personalized PageRank for a given array of source vertices, such + * that all random walks are started relative to the source vertices + */ + def staticParallelPersonalizedPageRank(sources: Array[VertexId], numIter: Int, + resetProb: Double = 0.15) : Graph[Vector, Double] = { + PageRank.runParallelPersonalizedPageRank(graph, numIter, resetProb, sources) + } + /** * Run Personalized PageRank for a fixed number of iterations with * with all iterations originating at the source node @@ -393,7 +410,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * @see [[org.apache.spark.graphx.lib.PageRank$#runWithOptions]] */ def staticPersonalizedPageRank(src: VertexId, numIter: Int, - resetProb: Double = 0.15) : Graph[Double, Double] = { + resetProb: Double = 0.15): Graph[Double, Double] = { PageRank.runWithOptions(graph, numIter, resetProb, Some(src)) } @@ -411,12 +428,22 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * Compute the connected component membership of each vertex and return a graph with the vertex * value containing the lowest vertex id in the connected component containing that vertex. * - * @see [[org.apache.spark.graphx.lib.ConnectedComponents$#run]] + * @see `org.apache.spark.graphx.lib.ConnectedComponents.run` */ def connectedComponents(): Graph[VertexId, ED] = { ConnectedComponents.run(graph) } + /** + * Compute the connected component membership of each vertex and return a graph with the vertex + * value containing the lowest vertex id in the connected component containing that vertex. + * + * @see `org.apache.spark.graphx.lib.ConnectedComponents.run` + */ + def connectedComponents(maxIterations: Int): Graph[VertexId, ED] = { + ConnectedComponents.run(graph, maxIterations) + } + /** * Compute the number of triangles passing through each vertex. * diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphXUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphXUtils.scala index 2cb07937eaa2..ef0b943fc3c3 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphXUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphXUtils.scala @@ -17,15 +17,16 @@ package org.apache.spark.graphx -import org.apache.spark.SparkConf +import scala.reflect.ClassTag +import org.apache.spark.SparkConf import org.apache.spark.graphx.impl._ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap - -import org.apache.spark.util.collection.{OpenHashSet, BitSet} import org.apache.spark.util.BoundedPriorityQueue +import org.apache.spark.util.collection.{BitSet, OpenHashSet} object GraphXUtils { + /** * Registers classes that GraphX uses with Kryo. */ @@ -44,4 +45,28 @@ object GraphXUtils { classOf[OpenHashSet[Int]], classOf[OpenHashSet[Long]])) } + + /** + * A proxy method to map the obsolete API to the new one. + */ + private[graphx] def mapReduceTriplets[VD: ClassTag, ED: ClassTag, A: ClassTag]( + g: Graph[VD, ED], + mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], + reduceFunc: (A, A) => A, + activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None): VertexRDD[A] = { + def sendMsg(ctx: EdgeContext[VD, ED, A]) { + mapFunc(ctx.toEdgeTriplet).foreach { kv => + val id = kv._1 + val msg = kv._2 + if (id == ctx.srcId) { + ctx.sendToSrc(msg) + } else { + assert(id == ctx.dstId) + ctx.sendToDst(msg) + } + } + } + g.aggregateMessagesWithActiveSet( + sendMsg, reduceFunc, TripletFields.All, activeSetOpt) + } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index 2ca60d51f833..755c6febc48e 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -18,8 +18,11 @@ package org.apache.spark.graphx import scala.reflect.ClassTag -import org.apache.spark.Logging +import org.apache.spark.graphx.util.PeriodicGraphCheckpointer +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.util.PeriodicRDDCheckpointer /** * Implements a Pregel-like bulk-synchronous message-passing API. @@ -119,27 +122,42 @@ object Pregel extends Logging { mergeMsg: (A, A) => A) : Graph[VD, ED] = { - var g = graph.mapVertices((vid, vdata) => vprog(vid, vdata, initialMsg)).cache() + require(maxIterations > 0, s"Maximum number of iterations must be greater than 0," + + s" but got ${maxIterations}") + + val checkpointInterval = graph.vertices.sparkContext.getConf + .getInt("spark.graphx.pregel.checkpointInterval", -1) + var g = graph.mapVertices((vid, vdata) => vprog(vid, vdata, initialMsg)) + val graphCheckpointer = new PeriodicGraphCheckpointer[VD, ED]( + checkpointInterval, graph.vertices.sparkContext) + graphCheckpointer.update(g) + // compute the messages - var messages = g.mapReduceTriplets(sendMsg, mergeMsg) + var messages = GraphXUtils.mapReduceTriplets(g, sendMsg, mergeMsg) + val messageCheckpointer = new PeriodicRDDCheckpointer[(VertexId, A)]( + checkpointInterval, graph.vertices.sparkContext) + messageCheckpointer.update(messages.asInstanceOf[RDD[(VertexId, A)]]) var activeMessages = messages.count() + // Loop var prevG: Graph[VD, ED] = null var i = 0 while (activeMessages > 0 && i < maxIterations) { // Receive the messages and update the vertices. prevG = g - g = g.joinVertices(messages)(vprog).cache() + g = g.joinVertices(messages)(vprog) + graphCheckpointer.update(g) val oldMessages = messages // Send new messages, skipping edges where neither side received a message. We must cache // messages so it can be materialized on the next line, allowing us to uncache the previous // iteration. - messages = g.mapReduceTriplets( - sendMsg, mergeMsg, Some((oldMessages, activeDirection))).cache() + messages = GraphXUtils.mapReduceTriplets( + g, sendMsg, mergeMsg, Some((oldMessages, activeDirection))) // The call to count() materializes `messages` and the vertices of `g`. This hides oldMessages // (depended on by the vertices of g) and the vertices of prevG (depended on by oldMessages // and the vertices of g). + messageCheckpointer.update(messages.asInstanceOf[RDD[(VertexId, A)]]) activeMessages = messages.count() logInfo("Pregel finished iteration " + i) @@ -151,7 +169,9 @@ object Pregel extends Logging { // count the iteration i += 1 } - + messageCheckpointer.unpersistDataSet() + graphCheckpointer.deleteAllCheckpoints() + messageCheckpointer.deleteAllCheckpoints() g } // end of apply diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index 1ef7a78fbcd0..35577d9e2fc6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -20,14 +20,12 @@ package org.apache.spark.graphx import scala.reflect.ClassTag import org.apache.spark._ -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd._ -import org.apache.spark.storage.StorageLevel - import org.apache.spark.graphx.impl.RoutingTablePartition import org.apache.spark.graphx.impl.ShippableVertexPartition import org.apache.spark.graphx.impl.VertexAttributeBlock import org.apache.spark.graphx.impl.VertexRDDImpl +import org.apache.spark.rdd._ +import org.apache.spark.storage.StorageLevel /** * Extends `RDD[(VertexId, VD)]` by ensuring that there is only one entry for each vertex and by @@ -277,7 +275,7 @@ object VertexRDD { def apply[VD: ClassTag](vertices: RDD[(VertexId, VD)]): VertexRDD[VD] = { val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match { case Some(p) => vertices - case None => vertices.partitionBy(new HashPartitioner(vertices.partitions.size)) + case None => vertices.partitionBy(new HashPartitioner(vertices.partitions.length)) } val vertexPartitions = vPartitioned.mapPartitions( iter => Iterator(ShippableVertexPartition(iter)), @@ -318,7 +316,7 @@ object VertexRDD { ): VertexRDD[VD] = { val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match { case Some(p) => vertices - case None => vertices.partitionBy(new HashPartitioner(vertices.partitions.size)) + case None => vertices.partitionBy(new HashPartitioner(vertices.partitions.length)) } val routingTables = createRoutingTables(edges, vPartitioned.partitioner.get) val vertexPartitions = vPartitioned.zipPartitions(routingTables, preservesPartitioning = true) { @@ -359,7 +357,7 @@ object VertexRDD { Function.tupled(RoutingTablePartition.edgePartitionToMsgs))) .setName("VertexRDD.createRoutingTables - vid2pid (aggregation)") - val numEdgePartitions = edges.partitions.size + val numEdgePartitions = edges.partitions.length vid2pid.partitionBy(vertexPartitioner).mapPartitions( iter => Iterator(RoutingTablePartition.fromMsgs(numEdgePartitions, iter)), preservesPartitioning = true) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index ab021a252eb8..0e6a340a680b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -17,7 +17,7 @@ package org.apache.spark.graphx.impl -import scala.reflect.{classTag, ClassTag} +import scala.reflect.ClassTag import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap @@ -151,9 +151,9 @@ class EdgePartition[ * applied to each edge */ def map[ED2: ClassTag](f: Edge[ED] => ED2): EdgePartition[ED2, VD] = { - val newData = new Array[ED2](data.size) + val newData = new Array[ED2](data.length) val edge = new Edge[ED]() - val size = data.size + val size = data.length var i = 0 while (i < size) { edge.srcId = srcIds(i) @@ -179,13 +179,13 @@ class EdgePartition[ */ def map[ED2: ClassTag](iter: Iterator[ED2]): EdgePartition[ED2, VD] = { // Faster than iter.toArray, because the expected size is known. - val newData = new Array[ED2](data.size) + val newData = new Array[ED2](data.length) var i = 0 while (iter.hasNext) { newData(i) = iter.next() i += 1 } - assert(newData.size == i) + assert(newData.length == i) this.withData(newData) } @@ -311,7 +311,7 @@ class EdgePartition[ * * @return size of the partition */ - val size: Int = localSrcIds.size + val size: Int = localSrcIds.length /** The number of unique source vertices in the partition. */ def indexSize: Int = index.size @@ -388,7 +388,7 @@ class EdgePartition[ val aggregates = new Array[A](vertexAttrs.length) val bitset = new BitSet(vertexAttrs.length) - var ctx = new AggregatingEdgeContext[VD, ED, A](mergeMsg, aggregates, bitset) + val ctx = new AggregatingEdgeContext[VD, ED, A](mergeMsg, aggregates, bitset) var i = 0 while (i < size) { val localSrcId = localSrcIds(i) @@ -433,7 +433,7 @@ class EdgePartition[ val aggregates = new Array[A](vertexAttrs.length) val bitset = new BitSet(vertexAttrs.length) - var ctx = new AggregatingEdgeContext[VD, ED, A](mergeMsg, aggregates, bitset) + val ctx = new AggregatingEdgeContext[VD, ED, A](mergeMsg, aggregates, bitset) index.iterator.foreach { cluster => val clusterSrcId = cluster._1 val clusterPos = cluster._2 diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index 906d42328fcb..da3db3c4dca0 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap -import org.apache.spark.util.collection.{SortDataFormat, Sorter, PrimitiveVector} +import org.apache.spark.util.collection.{PrimitiveVector, SortDataFormat, Sorter} /** Constructs an EdgePartition from scratch. */ private[graphx] @@ -38,9 +38,9 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla val edgeArray = edges.trim().array new Sorter(Edge.edgeArraySortDataFormat[ED]) .sort(edgeArray, 0, edgeArray.length, Edge.lexicographicOrdering) - val localSrcIds = new Array[Int](edgeArray.size) - val localDstIds = new Array[Int](edgeArray.size) - val data = new Array[ED](edgeArray.size) + val localSrcIds = new Array[Int](edgeArray.length) + val localDstIds = new Array[Int](edgeArray.length) + val data = new Array[ED](edgeArray.length) val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int] val global2local = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int] val local2global = new PrimitiveVector[VertexId] @@ -52,7 +52,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla var currSrcId: VertexId = edgeArray(0).srcId var currLocalId = -1 var i = 0 - while (i < edgeArray.size) { + while (i < edgeArray.length) { val srcId = edgeArray(i).srcId val dstId = edgeArray(i).dstId localSrcIds(i) = global2local.changeValue(srcId, @@ -98,9 +98,9 @@ class ExistingEdgePartitionBuilder[ val edgeArray = edges.trim().array new Sorter(EdgeWithLocalIds.edgeArraySortDataFormat[ED]) .sort(edgeArray, 0, edgeArray.length, EdgeWithLocalIds.lexicographicOrdering) - val localSrcIds = new Array[Int](edgeArray.size) - val localDstIds = new Array[Int](edgeArray.size) - val data = new Array[ED](edgeArray.size) + val localSrcIds = new Array[Int](edgeArray.length) + val localDstIds = new Array[Int](edgeArray.length) + val data = new Array[ED](edgeArray.length) val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int] // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and // adding them to the index @@ -108,7 +108,7 @@ class ExistingEdgePartitionBuilder[ index.update(edgeArray(0).srcId, 0) var currSrcId: VertexId = edgeArray(0).srcId var i = 0 - while (i < edgeArray.size) { + while (i < edgeArray.length) { localSrcIds(i) = edgeArray(i).localSrcId localDstIds(i) = edgeArray(i).localDstId data(i) = edgeArray(i).attr diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala index c88b2f65a86c..376c7b06f9d2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala @@ -19,12 +19,11 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} -import org.apache.spark.{OneToOneDependency, HashPartitioner} +import org.apache.spark.{HashPartitioner, OneToOneDependency} +import org.apache.spark.graphx._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.graphx._ - class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( @transient override val partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])], val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) @@ -42,11 +41,11 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( /** * If `partitionsRDD` already has a partitioner, use it. Otherwise assume that the - * [[PartitionID]]s in `partitionsRDD` correspond to the actual partitions and create a new + * `PartitionID`s in `partitionsRDD` correspond to the actual partitions and create a new * partitioner that allows co-partitioning with `partitionsRDD`. */ override val partitioner = - partitionsRDD.partitioner.orElse(Some(new HashPartitioner(partitions.size))) + partitionsRDD.partitioner.orElse(Some(new HashPartitioner(partitions.length))) override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() @@ -64,7 +63,9 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( this } - /** Persists the edge partitions using `targetStorageLevel`, which defaults to MEMORY_ONLY. */ + /** + * Persists the edge partitions using `targetStorageLevel`, which defaults to MEMORY_ONLY. + */ override def cache(): this.type = { partitionsRDD.persist(targetStorageLevel) this diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index da95314440d8..34e1253ff42a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -20,13 +20,10 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.HashPartitioner -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.{RDD, ShuffledRDD} -import org.apache.spark.storage.StorageLevel import org.apache.spark.graphx._ -import org.apache.spark.graphx.impl.GraphImpl._ import org.apache.spark.graphx.util.BytecodeUtils - +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel /** * An implementation of [[org.apache.spark.graphx.Graph]] to support computation on graphs. @@ -45,7 +42,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient override val edges: EdgeRDDImpl[ED, VD] = replicatedVertexView.edges - /** Return a RDD that brings edges together with their source and destination vertices. */ + /** Return an RDD that brings edges together with their source and destination vertices. */ @transient override lazy val triplets: RDD[EdgeTriplet[VD, ED]] = { replicatedVertexView.upgrade(vertices, true, true) replicatedVertexView.edges.partitionsRDD.mapPartitions(_.flatMap { @@ -77,7 +74,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def getCheckpointFiles: Seq[String] = { Seq(vertices.getCheckpointFile, replicatedVertexView.edges.getCheckpointFile).flatMap { case Some(path) => Seq(path) - case None => Seq() + case None => Seq.empty } } @@ -94,7 +91,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { - partitionBy(partitionStrategy, edges.partitions.size) + partitionBy(partitionStrategy, edges.partitions.length) } override def partitionBy( @@ -188,31 +185,6 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( // Lower level transformation methods // /////////////////////////////////////////////////////////////////////////////////////////////// - override def mapReduceTriplets[A: ClassTag]( - mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], - reduceFunc: (A, A) => A, - activeSetOpt: Option[(VertexRDD[_], EdgeDirection)]): VertexRDD[A] = { - - def sendMsg(ctx: EdgeContext[VD, ED, A]) { - mapFunc(ctx.toEdgeTriplet).foreach { kv => - val id = kv._1 - val msg = kv._2 - if (id == ctx.srcId) { - ctx.sendToSrc(msg) - } else { - assert(id == ctx.dstId) - ctx.sendToDst(msg) - } - } - } - - val mapUsesSrcAttr = accessesVertexAttr(mapFunc, "srcAttr") - val mapUsesDstAttr = accessesVertexAttr(mapFunc, "dstAttr") - val tripletFields = new TripletFields(mapUsesSrcAttr, mapUsesDstAttr, true) - - aggregateMessagesWithActiveSet(sendMsg, reduceFunc, tripletFields, activeSetOpt) - } - override def aggregateMessagesWithActiveSet[A: ClassTag]( sendMsg: EdgeContext[VD, ED, A] => Unit, mergeMsg: (A, A) => A, @@ -292,7 +264,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } } - /** Test whether the closure accesses the the attribute with name `attrName`. */ + /** Test whether the closure accesses the attribute with name `attrName`. */ private def accessesVertexAttr(closure: AnyRef, attrName: String): Boolean = { try { BytecodeUtils.invokedMethod(closure, classOf[EdgeTriplet[VD, ED]], attrName) @@ -305,7 +277,9 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( object GraphImpl { - /** Create a graph from edges, setting referenced vertices to `defaultVertexAttr`. */ + /** + * Create a graph from edges, setting referenced vertices to `defaultVertexAttr`. + */ def apply[VD: ClassTag, ED: ClassTag]( edges: RDD[Edge[ED]], defaultVertexAttr: VD, @@ -314,7 +288,9 @@ object GraphImpl { fromEdgeRDD(EdgeRDD.fromEdges(edges), defaultVertexAttr, edgeStorageLevel, vertexStorageLevel) } - /** Create a graph from EdgePartitions, setting referenced vertices to `defaultVertexAttr`. */ + /** + * Create a graph from EdgePartitions, setting referenced vertices to `defaultVertexAttr`. + */ def fromEdgePartitions[VD: ClassTag, ED: ClassTag]( edgePartitions: RDD[(PartitionID, EdgePartition[ED, VD])], defaultVertexAttr: VD, @@ -324,7 +300,9 @@ object GraphImpl { vertexStorageLevel) } - /** Create a graph from vertices and edges, setting missing vertices to `defaultVertexAttr`. */ + /** + * Create a graph from vertices and edges, setting missing vertices to `defaultVertexAttr`. + */ def apply[VD: ClassTag, ED: ClassTag]( vertices: RDD[(VertexId, VD)], edges: RDD[Edge[ED]], @@ -378,7 +356,8 @@ object GraphImpl { edgeStorageLevel: StorageLevel, vertexStorageLevel: StorageLevel): GraphImpl[VD, ED] = { val edgesCached = edges.withTargetStorageLevel(edgeStorageLevel).cache() - val vertices = VertexRDD.fromEdges(edgesCached, edgesCached.partitions.size, defaultVertexAttr) + val vertices = + VertexRDD.fromEdges(edgesCached, edgesCached.partitions.length, defaultVertexAttr) .withTargetStorageLevel(vertexStorageLevel) fromExistingRDDs(vertices, edgesCached) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala index 1df86449fa0c..d2194d85bf52 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -17,12 +17,10 @@ package org.apache.spark.graphx.impl -import scala.reflect.{classTag, ClassTag} - -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag import org.apache.spark.graphx._ +import org.apache.spark.rdd.RDD /** * Manages shipping vertex attributes to the edge partitions of an @@ -42,8 +40,8 @@ class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( * shipping level. */ def withEdges[VD2: ClassTag, ED2: ClassTag]( - edges_ : EdgeRDDImpl[ED2, VD2]): ReplicatedVertexView[VD2, ED2] = { - new ReplicatedVertexView(edges_, hasSrcId, hasDstId) + _edges: EdgeRDDImpl[ED2, VD2]): ReplicatedVertexView[VD2, ED2] = { + new ReplicatedVertexView(_edges, hasSrcId, hasDstId) } /** diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index 4f1260a5a67b..6453bbeae9f1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -17,17 +17,9 @@ package org.apache.spark.graphx.impl -import scala.reflect.ClassTag - -import org.apache.spark.Partitioner -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.ShuffledRDD -import org.apache.spark.util.collection.{BitSet, PrimitiveVector} - import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap - -import org.apache.spark.graphx.impl.RoutingTablePartition.RoutingTableMessage +import org.apache.spark.util.collection.{BitSet, PrimitiveVector} private[graphx] object RoutingTablePartition { @@ -110,10 +102,10 @@ private[graphx] class RoutingTablePartition( private val routingTable: Array[(Array[VertexId], BitSet, BitSet)]) extends Serializable { /** The maximum number of edge partitions this `RoutingTablePartition` is built to join with. */ - val numEdgePartitions: Int = routingTable.size + val numEdgePartitions: Int = routingTable.length /** Returns the number of vertices that will be sent to the specified edge partition. */ - def partitionSize(pid: PartitionID): Int = routingTable(pid)._1.size + def partitionSize(pid: PartitionID): Int = routingTable(pid)._1.length /** Returns an iterator over all vertex ids stored in this `RoutingTablePartition`. */ def iterator: Iterator[VertexId] = routingTable.iterator.flatMap(_._1.iterator) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index aa320088f208..a4e293d74a01 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -19,17 +19,16 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag -import org.apache.spark.util.collection.{BitSet, PrimitiveVector} - import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.{BitSet, PrimitiveVector} /** Stores vertex attributes to ship to an edge partition. */ private[graphx] class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexId], val attrs: Array[VD]) extends Serializable { def iterator: Iterator[(VertexId, VD)] = - (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } + (0 until vids.length).iterator.map { i => (vids(i), attrs(i)) } } private[graphx] @@ -50,7 +49,7 @@ object ShippableVertexPartition { /** * Construct a `ShippableVertexPartition` from the given vertices with the specified routing * table, filling in missing vertices mentioned in the routing table using `defaultVal`, - * and merging duplicate vertex atrribute with mergeFunc. + * and merging duplicate vertex attribute with mergeFunc. */ def apply[VD: ClassTag]( iter: Iterator[(VertexId, VD)], routingTable: RoutingTablePartition, defaultVal: VD, @@ -103,8 +102,8 @@ class ShippableVertexPartition[VD: ClassTag]( extends VertexPartitionBase[VD] { /** Return a new ShippableVertexPartition with the specified routing table. */ - def withRoutingTable(routingTable_ : RoutingTablePartition): ShippableVertexPartition[VD] = { - new ShippableVertexPartition(index, values, mask, routingTable_) + def withRoutingTable(_routingTable: RoutingTablePartition): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(index, values, mask, _routingTable) } /** diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index fbe53acfc32a..b4100bade073 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -19,10 +19,8 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag -import org.apache.spark.util.collection.BitSet - import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.BitSet private[graphx] object VertexPartition { /** Construct a `VertexPartition` from the given vertices. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala index 5ad6390a56c4..8da46db98be8 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -20,10 +20,9 @@ package org.apache.spark.graphx.impl import scala.language.higherKinds import scala.reflect.ClassTag -import org.apache.spark.util.collection.BitSet - import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.BitSet private[graphx] object VertexPartitionBase { /** @@ -58,7 +57,7 @@ private[graphx] object VertexPartitionBase { * concrete implementation. [[VertexPartitionBaseOps]] provides a variety of operations for * VertexPartitionBase and subclasses that provide implicit evidence of membership in the * `VertexPartitionBaseOpsConstructor` typeclass (for example, - * [[VertexPartition.VertexPartitionOpsConstructor]]). + * `VertexPartition.VertexPartitionOpsConstructor`). */ private[graphx] abstract class VertexPartitionBase[@specialized(Long, Int, Double) VD: ClassTag] extends Serializable { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index b90f9fa32705..a8ed59b09bbb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -21,19 +21,18 @@ import scala.language.higherKinds import scala.language.implicitConversions import scala.reflect.ClassTag -import org.apache.spark.Logging -import org.apache.spark.util.collection.BitSet - import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.internal.Logging +import org.apache.spark.util.collection.BitSet /** - * An class containing additional operations for subclasses of VertexPartitionBase that provide + * A class containing additional operations for subclasses of VertexPartitionBase that provide * implicit evidence of membership in the `VertexPartitionBaseOpsConstructor` typeclass (for - * example, [[VertexPartition.VertexPartitionOpsConstructor]]). + * example, `VertexPartition.VertexPartitionOpsConstructor`). */ private[graphx] abstract class VertexPartitionBaseOps - [VD: ClassTag, Self[X] <: VertexPartitionBase[X] : VertexPartitionBaseOpsConstructor] + [VD: ClassTag, Self[X] <: VertexPartitionBase[X]: VertexPartitionBaseOpsConstructor] (self: Self[VD]) extends Serializable with Logging { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala index 7f4e7e9d79d6..3c6f22d97360 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala @@ -20,12 +20,10 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark._ -import org.apache.spark.SparkContext._ +import org.apache.spark.graphx._ import org.apache.spark.rdd._ import org.apache.spark.storage.StorageLevel -import org.apache.spark.graphx._ - class VertexRDDImpl[VD] private[graphx] ( @transient val partitionsRDD: RDD[ShippableVertexPartition[VD]], val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) @@ -65,7 +63,9 @@ class VertexRDDImpl[VD] private[graphx] ( this } - /** Persists the vertex partitions at `targetStorageLevel`, which defaults to MEMORY_ONLY. */ + /** + * Persists the vertex partitions at `targetStorageLevel`, which defaults to MEMORY_ONLY. + */ override def cache(): this.type = { partitionsRDD.persist(targetStorageLevel) this diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala index 859f89603904..4e9b13162e5c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala @@ -29,13 +29,16 @@ object ConnectedComponents { * * @tparam VD the vertex attribute type (discarded in the computation) * @tparam ED the edge attribute type (preserved in the computation) - * * @param graph the graph for which to compute the connected components - * + * @param maxIterations the maximum number of iterations to run for * @return a graph with vertex attributes containing the smallest vertex in each * connected component */ - def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexId, ED] = { + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], + maxIterations: Int): Graph[VertexId, ED] = { + require(maxIterations > 0, s"Maximum of iterations must be greater than 0," + + s" but got ${maxIterations}") + val ccGraph = graph.mapVertices { case (vid, _) => vid } def sendMessage(edge: EdgeTriplet[VertexId, ED]): Iterator[(VertexId, VertexId)] = { if (edge.srcAttr < edge.dstAttr) { @@ -47,9 +50,26 @@ object ConnectedComponents { } } val initialMessage = Long.MaxValue - Pregel(ccGraph, initialMessage, activeDirection = EdgeDirection.Either)( + val pregelGraph = Pregel(ccGraph, initialMessage, + maxIterations, EdgeDirection.Either)( vprog = (id, attr, msg) => math.min(attr, msg), sendMsg = sendMessage, mergeMsg = (a, b) => math.min(a, b)) + ccGraph.unpersist() + pregelGraph } // end of connectedComponents + + /** + * Compute the connected component membership of each vertex and return a graph with the vertex + * value containing the lowest vertex id in the connected component containing that vertex. + * + * @tparam VD the vertex attribute type (discarded in the computation) + * @tparam ED the edge attribute type (preserved in the computation) + * @param graph the graph for which to compute the connected components + * @return a graph with vertex attributes containing the smallest vertex in each + * connected component + */ + def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexId, ED] = { + run(graph, Int.MaxValue) + } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala index a3ad6bed1c99..cb3025f8bef5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala @@ -18,6 +18,7 @@ package org.apache.spark.graphx.lib import scala.reflect.ClassTag + import org.apache.spark.graphx._ /** Label Propagation algorithm. */ @@ -42,6 +43,8 @@ object LabelPropagation { * @return a graph with vertex attributes containing the label of community affiliation */ def run[VD, ED: ClassTag](graph: Graph[VD, ED], maxSteps: Int): Graph[VertexId, ED] = { + require(maxSteps > 0, s"Maximum of steps must be greater than 0, but got ${maxSteps}") + val lpaGraph = graph.mapVertices { case (vid, _) => vid } def sendMessage(e: EdgeTriplet[VertexId, ED]): Iterator[(VertexId, Map[VertexId, Long])] = { Iterator((e.srcId, Map(e.dstAttr -> 1L)), (e.dstId, Map(e.srcAttr -> 1L))) @@ -52,7 +55,7 @@ object LabelPropagation { val count1Val = count1.getOrElse(i, 0L) val count2Val = count2.getOrElse(i, 0L) i -> (count1Val + count2Val) - }.toMap + }(collection.breakOut) // more efficient alternative to [[collection.Traversable.toMap]] } def vertexProgram(vid: VertexId, attr: Long, message: Map[VertexId, Long]): VertexId = { if (message.isEmpty) attr else message.maxBy(_._2)._1 diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index 52b237fc1509..fd7b7f7c1c48 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -18,15 +18,17 @@ package org.apache.spark.graphx.lib import scala.reflect.ClassTag -import scala.language.postfixOps -import org.apache.spark.Logging +import breeze.linalg.{Vector => BV} + import org.apache.spark.graphx._ +import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.{Vector, Vectors} /** * PageRank algorithm implementation. There are two implementations of PageRank implemented. * - * The first implementation uses the standalone [[Graph]] interface and runs PageRank + * The first implementation uses the standalone `Graph` interface and runs PageRank * for a fixed number of iterations: * {{{ * var PR = Array.fill(n)( 1.0 ) @@ -39,7 +41,7 @@ import org.apache.spark.graphx._ * } * }}} * - * The second implementation uses the [[Pregel]] interface and runs PageRank until + * The second implementation uses the `Pregel` interface and runs PageRank until * convergence: * * {{{ @@ -54,9 +56,9 @@ import org.apache.spark.graphx._ * }}} * * `alpha` is the random reset probability (typically 0.15), `inNbrs[i]` is the set of - * neighbors whick link to `i` and `outDeg[j]` is the out degree of vertex `j`. + * neighbors which link to `i` and `outDeg[j]` is the out degree of vertex `j`. * - * Note that this is not the "normalized" PageRank and as a consequence pages that have no + * @note This is not the "normalized" PageRank and as a consequence pages that have no * inlinks will have a PageRank of alpha. */ object PageRank extends Logging { @@ -104,13 +106,18 @@ object PageRank extends Logging { graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15, srcId: Option[VertexId] = None): Graph[Double, Double] = { - val personalized = srcId isDefined + require(numIter > 0, s"Number of iterations must be greater than 0," + + s" but got ${numIter}") + require(resetProb >= 0 && resetProb <= 1, s"Random reset probability must belong" + + s" to [0, 1], but got ${resetProb}") + + val personalized = srcId.isDefined val src: VertexId = srcId.getOrElse(-1L) // Initialize the PageRank graph with each edge attribute having - // weight 1/outDegree and each vertex with attribute resetProb. + // weight 1/outDegree and each vertex with attribute 1.0. // When running personalized pagerank, only the source vertex - // has an attribute resetProb. All others are set to 0. + // has an attribute 1.0. All others are set to 0. var rankGraph: Graph[Double, Double] = graph // Associate the degree with each vertex .outerJoinVertices(graph.outDegrees) { (vid, vdata, deg) => deg.getOrElse(0) } @@ -118,7 +125,7 @@ object PageRank extends Logging { .mapTriplets( e => 1.0 / e.srcAttr, TripletFields.Src ) // Set the vertex attributes to the initial pagerank values .mapVertices { (id, attr) => - if (!(id != src && personalized)) resetProb else 0.0 + if (!(id != src && personalized)) 1.0 else 0.0 } def delta(u: VertexId, v: VertexId): Double = { if (u == v) 1.0 else 0.0 } @@ -138,13 +145,13 @@ object PageRank extends Logging { // edge partitions. prevRankGraph = rankGraph val rPrb = if (personalized) { - (src: VertexId , id: VertexId) => resetProb * delta(src, id) + (src: VertexId, id: VertexId) => resetProb * delta(src, id) } else { (src: VertexId, id: VertexId) => resetProb } - rankGraph = rankGraph.joinVertices(rankUpdates) { - (id, oldRank, msgSum) => rPrb(src, id) + (1.0 - resetProb) * msgSum + rankGraph = rankGraph.outerJoinVertices(rankUpdates) { + (id, oldRank, msgSumOpt) => rPrb(src, id) + (1.0 - resetProb) * msgSumOpt.getOrElse(0.0) }.cache() rankGraph.edges.foreachPartition(x => {}) // also materializes rankGraph.vertices @@ -155,7 +162,98 @@ object PageRank extends Logging { iteration += 1 } - rankGraph + // SPARK-18847 If the graph has sinks (vertices with no outgoing edges) correct the sum of ranks + normalizeRankSum(rankGraph, personalized) + } + + /** + * Run Personalized PageRank for a fixed number of iterations, for a + * set of starting nodes in parallel. Returns a graph with vertex attributes + * containing the pagerank relative to all starting nodes (as a sparse vector) and + * edge attributes the normalized edge weight + * + * @tparam VD The original vertex attribute (not used) + * @tparam ED The original edge attribute (not used) + * + * @param graph The graph on which to compute personalized pagerank + * @param numIter The number of iterations to run + * @param resetProb The random reset probability + * @param sources The list of sources to compute personalized pagerank from + * @return the graph with vertex attributes + * containing the pagerank relative to all starting nodes (as a sparse vector + * indexed by the position of nodes in the sources list) and + * edge attributes the normalized edge weight + */ + def runParallelPersonalizedPageRank[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], + numIter: Int, resetProb: Double = 0.15, + sources: Array[VertexId]): Graph[Vector, Double] = { + require(numIter > 0, s"Number of iterations must be greater than 0," + + s" but got ${numIter}") + require(resetProb >= 0 && resetProb <= 1, s"Random reset probability must belong" + + s" to [0, 1], but got ${resetProb}") + require(sources.nonEmpty, s"The list of sources must be non-empty," + + s" but got ${sources.mkString("[", ",", "]")}") + + // TODO if one sources vertex id is outside of the int range + // we won't be able to store its activations in a sparse vector + require(sources.max <= Int.MaxValue.toLong, + s"This implementation currently only works for source vertex ids at most ${Int.MaxValue}") + val zero = Vectors.sparse(sources.size, List()).asBreeze + val sourcesInitMap = sources.zipWithIndex.map { case (vid, i) => + val v = Vectors.sparse(sources.size, Array(i), Array(1.0)).asBreeze + (vid, v) + }.toMap + val sc = graph.vertices.sparkContext + val sourcesInitMapBC = sc.broadcast(sourcesInitMap) + // Initialize the PageRank graph with each edge attribute having + // weight 1/outDegree and each source vertex with attribute 1.0. + var rankGraph = graph + // Associate the degree with each vertex + .outerJoinVertices(graph.outDegrees) { (vid, vdata, deg) => deg.getOrElse(0) } + // Set the weight on the edges based on the degree + .mapTriplets(e => 1.0 / e.srcAttr, TripletFields.Src) + .mapVertices { (vid, attr) => + if (sourcesInitMapBC.value contains vid) { + sourcesInitMapBC.value(vid) + } else { + zero + } + } + + var i = 0 + while (i < numIter) { + val prevRankGraph = rankGraph + // Propagates the message along outbound edges + // and adding start nodes back in with activation resetProb + val rankUpdates = rankGraph.aggregateMessages[BV[Double]]( + ctx => ctx.sendToDst(ctx.srcAttr *:* ctx.attr), + (a : BV[Double], b : BV[Double]) => a +:+ b, TripletFields.Src) + + rankGraph = rankGraph.outerJoinVertices(rankUpdates) { + (vid, oldRank, msgSumOpt) => + val popActivations: BV[Double] = msgSumOpt.getOrElse(zero) *:* (1.0 - resetProb) + val resetActivations = if (sourcesInitMapBC.value contains vid) { + sourcesInitMapBC.value(vid) *:* resetProb + } else { + zero + } + popActivations +:+ resetActivations + }.cache() + + rankGraph.edges.foreachPartition(x => {}) // also materializes rankGraph.vertices + prevRankGraph.vertices.unpersist(false) + prevRankGraph.edges.unpersist(false) + + logInfo(s"Parallel Personalized PageRank finished iteration $i.") + + i += 1 + } + + // SPARK-18847 If the graph has sinks (vertices with no outgoing edges) correct the sum of ranks + val rankSums = rankGraph.vertices.values.fold(zero)(_ +:+ _) + rankGraph.mapVertices { (vid, attr) => + Vectors.fromBreeze(attr /:/ rankSums) + } } /** @@ -197,6 +295,10 @@ object PageRank extends Logging { graph: Graph[VD, ED], tol: Double, resetProb: Double = 0.15, srcId: Option[VertexId] = None): Graph[Double, Double] = { + require(tol >= 0, s"Tolerance must be no less than 0, but got ${tol}") + require(resetProb >= 0 && resetProb <= 1, s"Random reset probability must belong" + + s" to [0, 1], but got ${resetProb}") + val personalized = srcId.isDefined val src: VertexId = srcId.getOrElse(-1L) @@ -209,9 +311,9 @@ object PageRank extends Logging { } // Set the weight on the edges based on the degree .mapTriplets( e => 1.0 / e.srcAttr ) - // Set the vertex attributes to (initalPR, delta = 0) + // Set the vertex attributes to (initialPR, delta = 0) .mapVertices { (id, attr) => - if (id == src) (resetProb, Double.NegativeInfinity) else (0.0, 0.0) + if (id == src) (0.0, Double.NegativeInfinity) else (0.0, 0.0) } .cache() @@ -226,13 +328,12 @@ object PageRank extends Logging { def personalizedVertexProgram(id: VertexId, attr: (Double, Double), msgSum: Double): (Double, Double) = { val (oldPR, lastDelta) = attr - var teleport = oldPR - val delta = if (src==id) 1.0 else 0.0 - teleport = oldPR*delta - - val newPR = teleport + (1.0 - resetProb) * msgSum - val newDelta = if (lastDelta == Double.NegativeInfinity) newPR else newPR - oldPR - (newPR, newDelta) + val newPR = if (lastDelta == Double.NegativeInfinity) { + 1.0 + } else { + oldPR + (1.0 - resetProb) * msgSum + } + (newPR, newPR - oldPR) } def sendMessage(edge: EdgeTriplet[(Double, Double), Double]) = { @@ -257,9 +358,23 @@ object PageRank extends Logging { vertexProgram(id, attr, msgSum) } - Pregel(pagerankGraph, initialMessage, activeDirection = EdgeDirection.Out)( + val rankGraph = Pregel(pagerankGraph, initialMessage, activeDirection = EdgeDirection.Out)( vp, sendMessage, messageCombiner) .mapVertices((vid, attr) => attr._1) - } // end of deltaPageRank + // SPARK-18847 If the graph has sinks (vertices with no outgoing edges) correct the sum of ranks + normalizeRankSum(rankGraph, personalized) + } + + // Normalizes the sum of ranks to n (or 1 if personalized) + private def normalizeRankSum(rankGraph: Graph[Double, Double], personalized: Boolean) = { + val rankSum = rankGraph.vertices.values.sum() + if (personalized) { + rankGraph.mapVertices((id, rank) => rank / rankSum) + } else { + val numVertices = rankGraph.numVertices + val correctionFactor = numVertices.toDouble / rankSum + rankGraph.mapVertices((id, rank) => rank * correctionFactor) + } + } } 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 9cb24ed080e1..59fdd855e6f3 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 @@ -21,8 +21,8 @@ import scala.util.Random import com.github.fommil.netlib.BLAS.{getInstance => blas} -import org.apache.spark.rdd._ import org.apache.spark.graphx._ +import org.apache.spark.rdd._ /** Implementation of SVD++ algorithm. */ object SVDPlusPlus { @@ -39,21 +39,11 @@ object SVDPlusPlus { var gamma7: Double) extends Serializable - /** - * 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]]. + * available at + * here. * * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^^-0.5^^*sum(y)), * see the details on page 6. @@ -67,6 +57,11 @@ object SVDPlusPlus { def run(edges: RDD[Edge[Double]], conf: Conf) : (Graph[(Array[Double], Array[Double], Double, Double), Double], Double) = { + require(conf.maxIters > 0, s"Maximum of iterations must be greater than 0," + + s" but got ${conf.maxIters}") + require(conf.maxVal > conf.minVal, s"MaxVal must be greater than MinVal," + + s" but got {maxVal: ${conf.maxVal}, minVal: ${conf.minVal}}") + // Generate default vertex attribute def defaultF(rank: Int): (Array[Double], Array[Double], Double, Double) = { // TODO: use a fixed random seed 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 179f2843818e..4cac633aed00 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 @@ -17,9 +17,10 @@ package org.apache.spark.graphx.lib -import org.apache.spark.graphx._ import scala.reflect.ClassTag +import org.apache.spark.graphx._ + /** * Computes shortest paths to the given set of landmark vertices, returning a graph where each * vertex attribute is a map containing the shortest-path distance to each reachable landmark. @@ -32,10 +33,11 @@ object ShortestPaths { private def incrementMap(spmap: SPMap): SPMap = spmap.map { case (v, d) => v -> (d + 1) } - private def addMaps(spmap1: SPMap, spmap2: SPMap): SPMap = + private def addMaps(spmap1: SPMap, spmap2: SPMap): SPMap = { (spmap1.keySet ++ spmap2.keySet).map { k => k -> math.min(spmap1.getOrElse(k, Int.MaxValue), spmap2.getOrElse(k, Int.MaxValue)) - }.toMap + }(collection.breakOut) // more efficient alternative to [[collection.Traversable.toMap]] + } /** * Computes shortest paths to the given set of landmark vertices. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala old mode 100644 new mode 100755 index 8dd958033b33..e4f80ffcb451 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala @@ -36,12 +36,17 @@ object StronglyConnectedComponents { * @return a graph with vertex attributes containing the smallest vertex id in each SCC */ def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], numIter: Int): Graph[VertexId, ED] = { + require(numIter > 0, s"Number of iterations must be greater than 0," + + s" but got ${numIter}") // the graph we update with final SCC ids, and the graph we return at the end var sccGraph = graph.mapVertices { case (vid, _) => vid } // graph we are going to work with in our iterations var sccWorkGraph = graph.mapVertices { case (vid, _) => (vid, false) }.cache() + // helper variables to unpersist cached graphs + var prevSccGraph = sccGraph + var numVertices = sccWorkGraph.numVertices var iter = 0 while (sccWorkGraph.numVertices > 0 && iter < numIter) { @@ -62,48 +67,59 @@ object StronglyConnectedComponents { // write values to sccGraph sccGraph = sccGraph.outerJoinVertices(finalVertices) { (vid, scc, opt) => opt.getOrElse(scc) - } + }.cache() + // materialize vertices and edges + sccGraph.vertices.count() + sccGraph.edges.count() + // sccGraph materialized so, unpersist can be done on previous + prevSccGraph.unpersist(blocking = false) + prevSccGraph = sccGraph + // only keep vertices that are not final sccWorkGraph = sccWorkGraph.subgraph(vpred = (vid, data) => !data._2).cache() } while (sccWorkGraph.numVertices < numVertices) - sccWorkGraph = sccWorkGraph.mapVertices{ case (vid, (color, isFinal)) => (vid, isFinal) } + // if iter < numIter at this point sccGraph that is returned + // will not be recomputed and pregel executions are pointless + if (iter < numIter) { + sccWorkGraph = sccWorkGraph.mapVertices { case (vid, (color, isFinal)) => (vid, isFinal) } - // collect min of all my neighbor's scc values, update if it's smaller than mine - // then notify any neighbors with scc values larger than mine - sccWorkGraph = Pregel[(VertexId, Boolean), ED, VertexId]( - sccWorkGraph, Long.MaxValue, activeDirection = EdgeDirection.Out)( - (vid, myScc, neighborScc) => (math.min(myScc._1, neighborScc), myScc._2), - e => { - if (e.srcAttr._1 < e.dstAttr._1) { - Iterator((e.dstId, e.srcAttr._1)) - } else { - Iterator() - } - }, - (vid1, vid2) => math.min(vid1, vid2)) + // collect min of all my neighbor's scc values, update if it's smaller than mine + // then notify any neighbors with scc values larger than mine + sccWorkGraph = Pregel[(VertexId, Boolean), ED, VertexId]( + sccWorkGraph, Long.MaxValue, activeDirection = EdgeDirection.Out)( + (vid, myScc, neighborScc) => (math.min(myScc._1, neighborScc), myScc._2), + e => { + if (e.srcAttr._1 < e.dstAttr._1) { + Iterator((e.dstId, e.srcAttr._1)) + } else { + Iterator() + } + }, + (vid1, vid2) => math.min(vid1, vid2)) - // start at root of SCCs. Traverse values in reverse, notify all my neighbors - // do not propagate if colors do not match! - sccWorkGraph = Pregel[(VertexId, Boolean), ED, Boolean]( - sccWorkGraph, false, activeDirection = EdgeDirection.In)( - // vertex is final if it is the root of a color - // or it has the same color as a neighbor that is final - (vid, myScc, existsSameColorFinalNeighbor) => { - val isColorRoot = vid == myScc._1 - (myScc._1, myScc._2 || isColorRoot || existsSameColorFinalNeighbor) - }, - // activate neighbor if they are not final, you are, and you have the same color - e => { - val sameColor = e.dstAttr._1 == e.srcAttr._1 - val onlyDstIsFinal = e.dstAttr._2 && !e.srcAttr._2 - if (sameColor && onlyDstIsFinal) { - Iterator((e.srcId, e.dstAttr._2)) - } else { - Iterator() - } - }, - (final1, final2) => final1 || final2) + // start at root of SCCs. Traverse values in reverse, notify all my neighbors + // do not propagate if colors do not match! + sccWorkGraph = Pregel[(VertexId, Boolean), ED, Boolean]( + sccWorkGraph, false, activeDirection = EdgeDirection.In)( + // vertex is final if it is the root of a color + // or it has the same color as a neighbor that is final + (vid, myScc, existsSameColorFinalNeighbor) => { + val isColorRoot = vid == myScc._1 + (myScc._1, myScc._2 || isColorRoot || existsSameColorFinalNeighbor) + }, + // activate neighbor if they are not final, you are, and you have the same color + e => { + val sameColor = e.dstAttr._1 == e.srcAttr._1 + val onlyDstIsFinal = e.dstAttr._2 && !e.srcAttr._2 + if (sameColor && onlyDstIsFinal) { + Iterator((e.srcId, e.dstAttr._2)) + } else { + Iterator() + } + }, + (final1, final2) => final1 || final2) + } } sccGraph } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala index a5d598053f9c..2715137d19eb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala @@ -27,27 +27,49 @@ import org.apache.spark.graphx._ * The algorithm is relatively straightforward and can be computed in three steps: * *

      - *
    • Compute the set of neighbors for each vertex - *
    • For each edge compute the intersection of the sets and send the count to both vertices. - *
    • Compute the sum at each vertex and divide by two since each triangle is counted twice. + *
    • Compute the set of neighbors for each vertex
    • + *
    • For each edge compute the intersection of the sets and send the count to both vertices.
    • + *
    • Compute the sum at each vertex and divide by two since each triangle is counted twice.
    • *
    * - * Note that the input graph should have its edges in canonical direction - * (i.e. the `sourceId` less than `destId`). Also the graph must have been partitioned - * using [[org.apache.spark.graphx.Graph#partitionBy]]. + * There are two implementations. The default `TriangleCount.run` implementation first removes + * self cycles and canonicalizes the graph to ensure that the following conditions hold: + *
      + *
    • There are no self edges
    • + *
    • All edges are oriented (src is greater than dst)
    • + *
    • There are no duplicate edges
    • + *
    + * However, the canonicalization procedure is costly as it requires repartitioning the graph. + * If the input data is already in "canonical form" with self cycles removed then the + * `TriangleCount.runPreCanonicalized` should be used instead. + * + * {{{ + * val canonicalGraph = graph.mapEdges(e => 1).removeSelfEdges().canonicalizeEdges() + * val counts = TriangleCount.runPreCanonicalized(canonicalGraph).vertices + * }}} + * */ object TriangleCount { def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[Int, ED] = { - // Remove redundant edges - val g = graph.groupEdges((a, b) => a).cache() + // Transform the edge data something cheap to shuffle and then canonicalize + val canonicalGraph = graph.mapEdges(e => true).removeSelfEdges().convertToCanonicalEdges() + // Get the triangle counts + val counters = runPreCanonicalized(canonicalGraph).vertices + // Join them bath with the original graph + graph.outerJoinVertices(counters) { (vid, _, optCounter: Option[Int]) => + optCounter.getOrElse(0) + } + } + + def runPreCanonicalized[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[Int, ED] = { // Construct set representations of the neighborhoods val nbrSets: VertexRDD[VertexSet] = - g.collectNeighborIds(EdgeDirection.Either).mapValues { (vid, nbrs) => - val set = new VertexSet(4) + graph.collectNeighborIds(EdgeDirection.Either).mapValues { (vid, nbrs) => + val set = new VertexSet(nbrs.length) var i = 0 - while (i < nbrs.size) { + while (i < nbrs.length) { // prevent self cycle if (nbrs(i) != vid) { set.add(nbrs(i)) @@ -56,14 +78,14 @@ object TriangleCount { } set } + // join the sets with the graph - val setGraph: Graph[VertexSet, ED] = g.outerJoinVertices(nbrSets) { + val setGraph: Graph[VertexSet, ED] = graph.outerJoinVertices(nbrSets) { (vid, _, optSet) => optSet.getOrElse(null) } + // Edge function computes intersection of smaller vertex with larger vertex def edgeFunc(ctx: EdgeContext[VertexSet, ED, Int]) { - assert(ctx.srcAttr != null) - assert(ctx.dstAttr != null) val (smallSet, largeSet) = if (ctx.srcAttr.size < ctx.dstAttr.size) { (ctx.srcAttr, ctx.dstAttr) } else { @@ -80,15 +102,15 @@ object TriangleCount { ctx.sendToSrc(counter) ctx.sendToDst(counter) } + // compute the intersection along edges val counters: VertexRDD[Int] = setGraph.aggregateMessages(edgeFunc, _ + _) // Merge counters with the graph and divide by two since each triangle is counted twice - g.outerJoinVertices(counters) { - (vid, _, optCounter: Option[Int]) => - val dblCount = optCounter.getOrElse(0) - // double count should be even (divisible by two) - assert((dblCount & 1) == 0) - dblCount / 2 + graph.outerJoinVertices(counters) { (_, _, optCounter: Option[Int]) => + val dblCount = optCounter.getOrElse(0) + // This algorithm double counts each triangle so the final count should be even + require(dblCount % 2 == 0, "Triangle count resulted in an invalid number of triangles.") + dblCount / 2 } - } // end of TriangleCount + } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package-info.java b/graphx/src/main/scala/org/apache/spark/graphx/package-info.java index f659cc518ebd..7c63447070fc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package-info.java +++ b/graphx/src/main/scala/org/apache/spark/graphx/package-info.java @@ -19,4 +19,4 @@ * ALPHA COMPONENT * GraphX is a graph processing framework built on top of Spark. */ -package org.apache.spark.graphx; \ No newline at end of file +package org.apache.spark.graphx; diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index 6aab28ff0535..dde25b96594b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -30,7 +30,7 @@ package object graphx { */ type VertexId = Long - /** Integer identifer of a graph partition. Must be less than 2^30. */ + /** Integer identifier of a graph partition. Must be less than 2^30. */ // TODO: Consider using Char. type PartitionID = Int diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala index 74a7de18d416..d76e84ed8c9e 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala @@ -22,11 +22,10 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.HashSet import scala.language.existentials -import org.apache.spark.util.Utils - -import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor} -import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ +import org.apache.xbean.asm5.{ClassReader, ClassVisitor, MethodVisitor} +import org.apache.xbean.asm5.Opcodes._ +import org.apache.spark.util.Utils /** * Includes an utility function to test whether a function accesses a specific attribute @@ -93,7 +92,7 @@ private[graphx] object BytecodeUtils { /** * Given the class name, return whether we should look into the class or not. This is used to - * skip examing a large quantity of Java or Scala classes that we know for sure wouldn't access + * skip examining a large quantity of Java or Scala classes that we know for sure wouldn't access * the closures. Note that the class name is expected in ASM style (i.e. use "/" instead of "."). */ private def skipClass(className: String): Boolean = { @@ -107,18 +106,19 @@ private[graphx] object BytecodeUtils { * MethodInvocationFinder("spark/graph/Foo", "test") * its methodsInvoked variable will contain the set of methods invoked directly by * Foo.test(). Interface invocations are not returned as part of the result set because we cannot - * determine the actual metod invoked by inspecting the bytecode. + * determine the actual method invoked by inspecting the bytecode. */ private class MethodInvocationFinder(className: String, methodName: String) - extends ClassVisitor(ASM4) { + extends ClassVisitor(ASM5) { val methodsInvoked = new HashSet[(Class[_], String)] override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { if (name == methodName) { - new MethodVisitor(ASM4) { - override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { + new MethodVisitor(ASM5) { + override def visitMethodInsn( + op: Int, owner: String, name: String, desc: String, itf: Boolean) { if (op == INVOKEVIRTUAL || op == INVOKESPECIAL || op == INVOKESTATIC) { if (!skipClass(owner)) { methodsInvoked.add((Utils.classForName(owner.replace("/", ".")), name)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index 989e22630526..419731146df7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -18,19 +18,14 @@ package org.apache.spark.graphx.util import scala.annotation.tailrec -import scala.math._ +import scala.collection.mutable import scala.reflect.ClassTag import scala.util._ import org.apache.spark._ -import org.apache.spark.serializer._ -import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ import org.apache.spark.graphx._ -import org.apache.spark.graphx.Graph -import org.apache.spark.graphx.Edge -import org.apache.spark.graphx.impl.GraphImpl +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD /** A collection of graph generating functions. */ object GraphGenerators extends Logging { @@ -125,7 +120,7 @@ object GraphGenerators extends Logging { * A random graph generator using the R-MAT model, proposed in * "R-MAT: A Recursive Model for Graph Mining" by Chakrabarti et al. * - * See [[http://www.cs.cmu.edu/~christos/PUBLICATIONS/siam04.pdf]]. + * See http://www.cs.cmu.edu/~christos/PUBLICATIONS/siam04.pdf. */ def rmatGraph(sc: SparkContext, requestedNumVertices: Int, numEdges: Int): Graph[Int, Int] = { // let N = requestedNumVertices @@ -139,7 +134,7 @@ object GraphGenerators extends Logging { throw new IllegalArgumentException( s"numEdges must be <= $numEdgesUpperBound but was $numEdges") } - var edges: Set[Edge[Int]] = Set() + var edges = mutable.Set.empty[Edge[Int]] while (edges.size < numEdges) { if (edges.size % 100 == 0) { logDebug(edges.size + " edges") @@ -169,7 +164,7 @@ object GraphGenerators extends Logging { } /** - * This method recursively subdivides the the adjacency matrix into quadrants + * This method recursively subdivides the adjacency matrix into quadrants * until it picks a single cell. The naming conventions in this paper match * those of the R-MAT paper. There are a power of 2 number of nodes in the graph. * The adjacency matrix looks like: @@ -215,7 +210,6 @@ object GraphGenerators extends Logging { } } - // TODO(crankshaw) turn result into an enum (or case class for pattern matching} private def pickQuadrant(a: Double, b: Double, c: Double, d: Double): Int = { if (a + b + c + d != 1.0) { throw new IllegalArgumentException("R-MAT probability parameters sum to " + (a + b + c + d) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointer.scala similarity index 86% rename from mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointer.scala index 11a059536c50..539b66f747cc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointer.scala @@ -15,11 +15,12 @@ * limitations under the License. */ -package org.apache.spark.mllib.impl +package org.apache.spark.graphx.util import org.apache.spark.SparkContext import org.apache.spark.graphx.Graph import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.PeriodicCheckpointer /** @@ -49,6 +50,7 @@ import org.apache.spark.storage.StorageLevel * {{{ * val (graph1, graph2, graph3, ...) = ... * val cp = new PeriodicGraphCheckpointer(2, sc) + * cp.updateGraph(graph1) * graph1.vertices.count(); graph1.edges.count() * // persisted: graph1 * cp.updateGraph(graph2) @@ -69,13 +71,13 @@ import org.apache.spark.storage.StorageLevel * // checkpointed: graph4 * }}} * - * @param checkpointInterval Graphs will be checkpointed at this interval + * @param checkpointInterval Graphs will be checkpointed at this interval. + * If this interval was set as -1, then checkpointing will be disabled. * @tparam VD Vertex descriptor type * @tparam ED Edge descriptor type * - * TODO: Move this out of MLlib? */ -private[mllib] class PeriodicGraphCheckpointer[VD, ED]( +private[spark] class PeriodicGraphCheckpointer[VD, ED]( checkpointInterval: Int, sc: SparkContext) extends PeriodicCheckpointer[Graph[VD, ED]](checkpointInterval, sc) { @@ -86,7 +88,13 @@ private[mllib] class PeriodicGraphCheckpointer[VD, ED]( override protected def persist(data: Graph[VD, ED]): Unit = { if (data.vertices.getStorageLevel == StorageLevel.NONE) { - data.persist() + /* We need to use cache because persist does not honor the default storage level requested + * when constructing the graph. Only cache does that. + */ + data.vertices.cache() + } + if (data.edges.getStorageLevel == StorageLevel.NONE) { + data.edges.cache() } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala index e2754ea699da..972237da1cb2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala @@ -17,10 +17,10 @@ package org.apache.spark.graphx.util.collection -import org.apache.spark.util.collection.OpenHashSet - import scala.reflect._ +import org.apache.spark.util.collection.OpenHashSet + /** * A fast hash map implementation for primitive, non-null keys. This hash map supports * insertions and updates, but not deletions. This map is about an order of magnitude diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/package-info.java b/graphx/src/main/scala/org/apache/spark/graphx/util/package-info.java index 90cd1d46db17..86b427e31d26 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/package-info.java +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/package-info.java @@ -18,4 +18,4 @@ /** * Collections of utilities used by graphx. */ -package org.apache.spark.graphx.util; \ No newline at end of file +package org.apache.spark.graphx.util; diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties index eb3b1999eb99..3706a6e36130 100644 --- a/graphx/src/test/resources/log4j.properties +++ b/graphx/src/test/resources/log4j.properties @@ -24,5 +24,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN -org.spark-project.jetty.LEVEL=WARN +log4j.logger.org.spark_project.jetty=WARN diff --git a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala index f1ecc9e2219d..7a24e320c3e0 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.graphx import org.apache.spark.SparkFunSuite import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class EdgeRDDSuite extends SparkFunSuite with LocalSparkContext { @@ -33,4 +34,30 @@ class EdgeRDDSuite extends SparkFunSuite with LocalSparkContext { } } + test("checkpointing") { + withSpark { sc => + val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) + val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) + sc.setCheckpointDir(Utils.createTempDir().getCanonicalPath) + edges.checkpoint() + + // EdgeRDD not yet checkpointed + assert(!edges.isCheckpointed) + assert(!edges.isCheckpointedAndMaterialized) + assert(!edges.partitionsRDD.isCheckpointed) + assert(!edges.partitionsRDD.isCheckpointedAndMaterialized) + + val data = edges.collect().toSeq // force checkpointing + + // EdgeRDD shows up as checkpointed, but internally it is not. + // Only internal partitionsRDD is checkpointed. + assert(edges.isCheckpointed) + assert(!edges.isCheckpointedAndMaterialized) + assert(edges.partitionsRDD.isCheckpointed) + assert(edges.partitionsRDD.isCheckpointedAndMaterialized) + + assert(edges.collect().toSeq === data) // test checkpointed RDD + } + } + } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/EdgeSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/EdgeSuite.scala index 094a63472eaa..4d6b899c83a0 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/EdgeSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/EdgeSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkFunSuite class EdgeSuite extends SparkFunSuite { test ("compare") { - // decending order + // descending order val testEdges: Array[Edge[Int]] = Array( Edge(0x7FEDCBA987654321L, -0x7FEDCBA987654321L, 1), Edge(0x2345L, 0x1234L, 1), diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphLoaderSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphLoaderSuite.scala new file mode 100644 index 000000000000..e55b05fa996a --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphLoaderSuite.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx + +import java.io.File +import java.io.FileOutputStream +import java.io.OutputStreamWriter +import java.nio.charset.StandardCharsets + +import org.apache.spark.SparkFunSuite +import org.apache.spark.util.Utils + +class GraphLoaderSuite extends SparkFunSuite with LocalSparkContext { + + test("GraphLoader.edgeListFile") { + withSpark { sc => + val tmpDir = Utils.createTempDir() + val graphFile = new File(tmpDir.getAbsolutePath, "graph.txt") + val writer = new OutputStreamWriter(new FileOutputStream(graphFile), StandardCharsets.UTF_8) + for (i <- (1 until 101)) writer.write(s"$i 0\n") + writer.close() + try { + val graph = GraphLoader.edgeListFile(sc, tmpDir.getAbsolutePath) + val neighborAttrSums = graph.aggregateMessages[Int]( + ctx => ctx.sendToDst(ctx.srcAttr), + _ + _) + assert(neighborAttrSums.collect.toSet === Set((0: VertexId, 100))) + } finally { + Utils.deleteRecursively(tmpDir) + } + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index 57a8b95dd12e..32981719499d 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.graphx import org.apache.spark.{SparkContext, SparkFunSuite} import org.apache.spark.graphx.Graph._ -import org.apache.spark.graphx.impl.EdgePartition -import org.apache.spark.rdd._ class GraphOpsSuite extends SparkFunSuite with LocalSparkContext { @@ -55,6 +53,21 @@ class GraphOpsSuite extends SparkFunSuite with LocalSparkContext { } } + test("removeSelfEdges") { + withSpark { sc => + val edgeArray = Array((1 -> 2), (2 -> 3), (3 -> 3), (4 -> 3), (1 -> 1)) + .map { + case (a, b) => (a.toLong, b.toLong) + } + val correctEdges = edgeArray.filter { case (a, b) => a != b }.toSet + val graph = Graph.fromEdgeTuples(sc.parallelize(edgeArray), 1) + val canonicalizedEdges = graph.removeSelfEdges().edges.map(e => (e.srcId, e.dstId)) + .collect + assert(canonicalizedEdges.toSet.size === canonicalizedEdges.size) + assert(canonicalizedEdges.toSet === correctEdges) + } + } + test ("filter") { withSpark { sc => val n = 5 diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 1f5e27d5508b..88b59a343a83 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -62,7 +62,7 @@ class GraphSuite extends SparkFunSuite with LocalSparkContext { assert( graph.edges.count() === rawEdges.size ) // Vertices not explicitly provided but referenced by edges should be created automatically assert( graph.vertices.count() === 100) - graph.triplets.collect().map { et => + graph.triplets.collect().foreach { et => assert((et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr)) assert((et.dstId < 10 && et.dstAttr) || (et.dstId >= 10 && !et.dstAttr)) } @@ -221,14 +221,15 @@ class GraphSuite extends SparkFunSuite with LocalSparkContext { val vertices: RDD[(VertexId, Int)] = sc.parallelize(Array((1L, 1), (2L, 2))) val edges: RDD[Edge[Int]] = sc.parallelize(Array(Edge(1L, 2L, 0))) val graph = Graph(vertices, edges).reverse - val result = graph.mapReduceTriplets[Int](et => Iterator((et.dstId, et.srcAttr)), _ + _) + val result = GraphXUtils.mapReduceTriplets[Int, Int, Int]( + graph, et => Iterator((et.dstId, et.srcAttr)), _ + _) assert(result.collect().toSet === Set((1L, 2))) } } test("subgraph") { withSpark { sc => - // Create a star graph of 10 veritces. + // Create a star graph of 10 vertices. val n = 10 val star = starGraph(sc, n) // Take only vertices whose vids are even @@ -281,49 +282,6 @@ class GraphSuite extends SparkFunSuite with LocalSparkContext { } } - test("mapReduceTriplets") { - withSpark { sc => - val n = 5 - val star = starGraph(sc, n).mapVertices { (_, _) => 0 }.cache() - val starDeg = star.joinVertices(star.degrees){ (vid, oldV, deg) => deg } - val neighborDegreeSums = starDeg.mapReduceTriplets( - edge => Iterator((edge.srcId, edge.dstAttr), (edge.dstId, edge.srcAttr)), - (a: Int, b: Int) => a + b) - assert(neighborDegreeSums.collect().toSet === (0 to n).map(x => (x, n)).toSet) - - // activeSetOpt - val allPairs = for (x <- 1 to n; y <- 1 to n) yield (x: VertexId, y: VertexId) - val complete = Graph.fromEdgeTuples(sc.parallelize(allPairs, 3), 0) - val vids = complete.mapVertices((vid, attr) => vid).cache() - val active = vids.vertices.filter { case (vid, attr) => attr % 2 == 0 } - val numEvenNeighbors = vids.mapReduceTriplets(et => { - // Map function should only run on edges with destination in the active set - if (et.dstId % 2 != 0) { - throw new Exception("map ran on edge with dst vid %d, which is odd".format(et.dstId)) - } - Iterator((et.srcId, 1)) - }, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect().toSet - assert(numEvenNeighbors === (1 to n).map(x => (x: VertexId, n / 2)).toSet) - - // outerJoinVertices followed by mapReduceTriplets(activeSetOpt) - val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexId, (x + 1) % n: VertexId)), 3) - val ring = Graph.fromEdgeTuples(ringEdges, 0) .mapVertices((vid, attr) => vid).cache() - val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_).cache() - val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => - newOpt.getOrElse(old) - } - val numOddNeighbors = changedGraph.mapReduceTriplets(et => { - // Map function should only run on edges with source in the active set - if (et.srcId % 2 != 1) { - throw new Exception("map ran on edge with src vid %d, which is even".format(et.dstId)) - } - Iterator((et.dstId, 1)) - }, (a: Int, b: Int) => a + b, Some(changed, EdgeDirection.Out)).collect().toSet - assert(numOddNeighbors === (2 to n by 2).map(x => (x: VertexId, 1)).toSet) - - } - } - test("aggregateMessages") { withSpark { sc => val n = 5 @@ -347,7 +305,8 @@ class GraphSuite extends SparkFunSuite with LocalSparkContext { val reverseStarDegrees = reverseStar.outerJoinVertices(reverseStar.outDegrees) { (vid, a, bOpt) => bOpt.getOrElse(0) } - val neighborDegreeSums = reverseStarDegrees.mapReduceTriplets( + val neighborDegreeSums = GraphXUtils.mapReduceTriplets[Int, Int, Int]( + reverseStarDegrees, et => Iterator((et.srcId, et.dstAttr), (et.dstId, et.srcAttr)), (a: Int, b: Int) => a + b).collect().toSet assert(neighborDegreeSums === Set((0: VertexId, n)) ++ (1 to n).map(x => (x: VertexId, 0))) @@ -420,7 +379,8 @@ class GraphSuite extends SparkFunSuite with LocalSparkContext { val edges = sc.parallelize((1 to n).map(x => (x: VertexId, 0: VertexId)), numEdgePartitions) val graph = Graph.fromEdgeTuples(edges, 1) - val neighborAttrSums = graph.mapReduceTriplets[Int]( + val neighborAttrSums = GraphXUtils.mapReduceTriplets[Int, Int, Int]( + graph, et => Iterator((et.dstId, et.srcAttr)), _ + _) assert(neighborAttrSums.collect().toSet === Set((0: VertexId, n))) } finally { @@ -428,4 +388,29 @@ class GraphSuite extends SparkFunSuite with LocalSparkContext { } } + test("unpersist graph RDD") { + withSpark { sc => + val vert = sc.parallelize(List((1L, "a"), (2L, "b"), (3L, "c")), 1) + val edges = sc.parallelize(List(Edge[Long](1L, 2L), Edge[Long](1L, 3L)), 1) + val g0 = Graph(vert, edges) + val g = g0.partitionBy(PartitionStrategy.EdgePartition2D, 2) + val cc = g.connectedComponents() + assert(sc.getPersistentRDDs.nonEmpty) + cc.unpersist() + g.unpersist() + g0.unpersist() + vert.unpersist() + edges.unpersist() + assert(sc.getPersistentRDDs.isEmpty) + } + } + + test("SPARK-14219: pickRandomVertex") { + withSpark { sc => + val vert = sc.parallelize(List((1L, "a")), 1) + val edges = sc.parallelize(List(Edge[Long](1L, 1L)), 1) + val g0 = Graph(vert, edges) + assert(g0.pickRandomVertex() === 1L) + } + } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala index d2ad9be55577..66c4747fec26 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkConf import org.apache.spark.SparkContext /** - * Provides a method to run tests against a {@link SparkContext} variable that is correctly stopped + * Provides a method to run tests against a `SparkContext` variable that is correctly stopped * after each test. */ trait LocalSparkContext { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala index 8afa2d403b53..90a9ac613ef9 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala @@ -17,8 +17,7 @@ package org.apache.spark.graphx -import org.apache.spark.{SparkContext, SparkFunSuite} -import org.apache.spark.rdd._ +import org.apache.spark.SparkFunSuite class PregelSuite extends SparkFunSuite with LocalSparkContext { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index f1aa685a79c9..8e630435279d 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.graphx import org.apache.spark.{HashPartitioner, SparkContext, SparkFunSuite} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class VertexRDDSuite extends SparkFunSuite with LocalSparkContext { @@ -32,7 +33,7 @@ class VertexRDDSuite extends SparkFunSuite with LocalSparkContext { val n = 100 val verts = vertices(sc, n) val evens = verts.filter(q => ((q._2 % 2) == 0)) - assert(evens.count === (0 to n).filter(_ % 2 == 0).size) + assert(evens.count === (0 to n).count(_ % 2 == 0)) } } @@ -197,4 +198,29 @@ class VertexRDDSuite extends SparkFunSuite with LocalSparkContext { } } + test("checkpoint") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n) + sc.setCheckpointDir(Utils.createTempDir().getCanonicalPath) + verts.checkpoint() + + // VertexRDD not yet checkpointed + assert(!verts.isCheckpointed) + assert(!verts.isCheckpointedAndMaterialized) + assert(!verts.partitionsRDD.isCheckpointed) + assert(!verts.partitionsRDD.isCheckpointedAndMaterialized) + + val data = verts.collect().toSeq // force checkpointing + + // VertexRDD shows up as checkpointed, but internally it is not. + // Only internal partitionsRDD is checkpointed. + assert(verts.isCheckpointed) + assert(!verts.isCheckpointedAndMaterialized) + assert(verts.partitionsRDD.isCheckpointed) + assert(verts.partitionsRDD.isCheckpointedAndMaterialized) + + assert(verts.collect().toSeq === data) // test checkpointed RDD + } + } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index 7435647c6d9e..e4678b3578d9 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -18,14 +18,12 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag -import scala.util.Random import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.graphx._ import org.apache.spark.serializer.JavaSerializer import org.apache.spark.serializer.KryoSerializer -import org.apache.spark.graphx._ - class EdgePartitionSuite extends SparkFunSuite { def makeEdgePartition[A: ClassTag](xs: Iterable[(Int, Int, A)]): EdgePartition[A, Int] = { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala index 1203f8959f50..0fb8451fdcab 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala @@ -18,11 +18,10 @@ package org.apache.spark.graphx.impl import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.graphx._ import org.apache.spark.serializer.JavaSerializer import org.apache.spark.serializer.KryoSerializer -import org.apache.spark.graphx._ - class VertexPartitionSuite extends SparkFunSuite { test("isDefined, filter") { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala index c965a6eb8df1..1b8142356337 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala @@ -17,8 +17,7 @@ package org.apache.spark.graphx.lib -import org.apache.spark.{SparkContext, SparkFunSuite} -import org.apache.spark.SparkContext._ +import org.apache.spark.SparkFunSuite import org.apache.spark.graphx._ import org.apache.spark.graphx.util.GraphGenerators import org.apache.spark.rdd._ diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala index bdff31446f8e..9779553ce85d 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala @@ -41,7 +41,7 @@ object GridPageRank { } } // compute the pagerank - var pr = Array.fill(nRows * nCols)(resetProb) + var pr = Array.fill(nRows * nCols)(1.0) for (iter <- 0 until nIter) { val oldPr = pr pr = new Array[Double](nRows * nCols) @@ -50,7 +50,8 @@ object GridPageRank { inNbrs(ind).map( nbr => oldPr(nbr) / outDegree(nbr)).sum } } - (0L until (nRows * nCols)).zip(pr) + val prSum = pr.sum + (0L until (nRows * nCols)).zip(pr.map(_ * pr.length / prSum)) } } @@ -68,26 +69,34 @@ class PageRankSuite extends SparkFunSuite with LocalSparkContext { val nVertices = 100 val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() val resetProb = 0.15 + val tol = 0.0001 + val numIter = 2 val errorTol = 1.0e-5 - val staticRanks1 = starGraph.staticPageRank(numIter = 1, resetProb).vertices - val staticRanks2 = starGraph.staticPageRank(numIter = 2, resetProb).vertices.cache() + val staticRanks = starGraph.staticPageRank(numIter, resetProb).vertices.cache() + val staticRanks2 = starGraph.staticPageRank(numIter + 1, resetProb).vertices // Static PageRank should only take 2 iterations to converge - val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => + val notMatching = staticRanks.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => if (pr1 != pr2) 1 else 0 }.map { case (vid, test) => test }.sum() assert(notMatching === 0) - val staticErrors = staticRanks2.map { case (vid, pr) => - val p = math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) - val correct = (vid > 0 && pr == resetProb) || (vid == 0L && p < 1.0E-5) - if (!correct) 1 else 0 - } - assert(staticErrors.sum === 0) + val dynamicRanks = starGraph.pageRank(tol, resetProb).vertices.cache() + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + + // Computed in igraph 1.0 w/ R bindings: + // > page_rank(make_star(100, mode = "in")) + // Alternatively in NetworkX 1.11: + // > nx.pagerank(nx.DiGraph([(x, 0) for x in range(1,100)])) + // We multiply by the number of vertices to account for difference in normalization + val centerRank = 0.462394787 * nVertices + val othersRank = 0.005430356 * nVertices + val igraphPR = centerRank +: Seq.fill(nVertices - 1)(othersRank) + val ranks = VertexRDD(sc.parallelize(0L until nVertices zip igraphPR)) + assert(compareRanks(staticRanks, ranks) < errorTol) + assert(compareRanks(dynamicRanks, ranks) < errorTol) - val dynamicRanks = starGraph.pageRank(0, resetProb).vertices.cache() - assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) } } // end of test Star PageRank @@ -96,33 +105,62 @@ class PageRankSuite extends SparkFunSuite with LocalSparkContext { val nVertices = 100 val starGraph = GraphGenerators.starGraph(sc, nVertices).cache() val resetProb = 0.15 + val tol = 0.0001 + val numIter = 2 val errorTol = 1.0e-5 - val staticRanks1 = starGraph.staticPersonalizedPageRank(0, numIter = 1, resetProb).vertices - val staticRanks2 = starGraph.staticPersonalizedPageRank(0, numIter = 2, resetProb) - .vertices.cache() + val staticRanks = starGraph.staticPersonalizedPageRank(0, numIter, resetProb).vertices.cache() - // Static PageRank should only take 2 iterations to converge - val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => - if (pr1 != pr2) 1 else 0 - }.map { case (vid, test) => test }.sum - assert(notMatching === 0) + val dynamicRanks = starGraph.personalizedPageRank(0, tol, resetProb).vertices.cache() + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) - val staticErrors = staticRanks2.map { case (vid, pr) => - val correct = (vid > 0 && pr == 0.0) || - (vid == 0 && pr == resetProb) - if (!correct) 1 else 0 - } - assert(staticErrors.sum === 0) + val parallelStaticRanks = starGraph + .staticParallelPersonalizedPageRank(Array(0), numIter, resetProb).mapVertices { + case (vertexId, vector) => vector(0) + }.vertices.cache() + assert(compareRanks(staticRanks, parallelStaticRanks) < errorTol) + + // Computed in igraph 1.0 w/ R bindings: + // > page_rank(make_star(100, mode = "in"), personalized = c(1, rep(0, 99)), algo = "arpack") + // NOTE: We use the arpack algorithm as prpack (the default) redistributes rank to all + // vertices uniformly instead of just to the personalization source. + // Alternatively in NetworkX 1.11: + // > nx.pagerank(nx.DiGraph([(x, 0) for x in range(1,100)]), + // personalization=dict([(x, 1 if x == 0 else 0) for x in range(0,100)])) + // We multiply by the number of vertices to account for difference in normalization + val igraphPR0 = 1.0 +: Seq.fill(nVertices - 1)(0.0) + val ranks0 = VertexRDD(sc.parallelize(0L until nVertices zip igraphPR0)) + assert(compareRanks(staticRanks, ranks0) < errorTol) + assert(compareRanks(dynamicRanks, ranks0) < errorTol) - val dynamicRanks = starGraph.personalizedPageRank(0, 0, resetProb).vertices.cache() - assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) // We have one outbound edge from 1 to 0 - val otherStaticRanks2 = starGraph.staticPersonalizedPageRank(1, numIter = 2, resetProb) + val otherStaticRanks = starGraph.staticPersonalizedPageRank(1, numIter, resetProb) .vertices.cache() - val otherDynamicRanks = starGraph.personalizedPageRank(1, 0, resetProb).vertices.cache() - assert(compareRanks(otherDynamicRanks, otherStaticRanks2) < errorTol) + val otherDynamicRanks = starGraph.personalizedPageRank(1, tol, resetProb).vertices.cache() + val otherParallelStaticRanks = starGraph + .staticParallelPersonalizedPageRank(Array(0, 1), numIter, resetProb).mapVertices { + case (vertexId, vector) => vector(1) + }.vertices.cache() + assert(compareRanks(otherDynamicRanks, otherStaticRanks) < errorTol) + assert(compareRanks(otherStaticRanks, otherParallelStaticRanks) < errorTol) + assert(compareRanks(otherDynamicRanks, otherParallelStaticRanks) < errorTol) + + // Computed in igraph 1.0 w/ R bindings: + // > page_rank(make_star(100, mode = "in"), + // personalized = c(0, 1, rep(0, 98)), algo = "arpack") + // NOTE: We use the arpack algorithm as prpack (the default) redistributes rank to all + // vertices uniformly instead of just to the personalization source. + // Alternatively in NetworkX 1.11: + // > nx.pagerank(nx.DiGraph([(x, 0) for x in range(1,100)]), + // personalization=dict([(x, 1 if x == 1 else 0) for x in range(0,100)])) + val centerRank = 0.4594595 + val sourceRank = 0.5405405 + val igraphPR1 = centerRank +: sourceRank +: Seq.fill(nVertices - 2)(0.0) + val ranks1 = VertexRDD(sc.parallelize(0L until nVertices zip igraphPR1)) + assert(compareRanks(otherStaticRanks, ranks1) < errorTol) + assert(compareRanks(otherDynamicRanks, ranks1) < errorTol) + assert(compareRanks(otherParallelStaticRanks, ranks1) < errorTol) } } // end of test Star PersonalPageRank @@ -177,6 +215,84 @@ class PageRankSuite extends SparkFunSuite with LocalSparkContext { val dynamicRanks = chain.personalizedPageRank(4, tol, resetProb).vertices assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + + val parallelStaticRanks = chain + .staticParallelPersonalizedPageRank(Array(4), numIter, resetProb).mapVertices { + case (vertexId, vector) => vector(0) + }.vertices.cache() + assert(compareRanks(staticRanks, parallelStaticRanks) < errorTol) + } + } + + test("Loop with source PageRank") { + withSpark { sc => + val edges = sc.parallelize((1L, 2L) :: (2L, 3L) :: (3L, 4L) :: (4L, 2L) :: Nil) + val g = Graph.fromEdgeTuples(edges, 1) + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 50 + val errorTol = 1.0e-5 + + val staticRanks = g.staticPageRank(numIter, resetProb).vertices + val dynamicRanks = g.pageRank(tol, resetProb).vertices + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + + // Computed in igraph 1.0 w/ R bindings: + // > page_rank(graph_from_literal( A -+ B -+ C -+ D -+ B)) + // Alternatively in NetworkX 1.11: + // > nx.pagerank(nx.DiGraph([(1,2),(2,3),(3,4),(4,2)])) + // We multiply by the number of vertices to account for difference in normalization + val igraphPR = Seq(0.0375000, 0.3326045, 0.3202138, 0.3096817).map(_ * 4) + val ranks = VertexRDD(sc.parallelize(1L to 4L zip igraphPR)) + assert(compareRanks(staticRanks, ranks) < errorTol) + assert(compareRanks(dynamicRanks, ranks) < errorTol) + + } + } + + test("Loop with sink PageRank") { + withSpark { sc => + val edges = sc.parallelize((1L, 2L) :: (2L, 3L) :: (3L, 1L) :: (1L, 4L) :: Nil) + val g = Graph.fromEdgeTuples(edges, 1) + val resetProb = 0.15 + val tol = 0.0001 + val numIter = 20 + val errorTol = 1.0e-5 + + val staticRanks = g.staticPageRank(numIter, resetProb).vertices.cache() + val dynamicRanks = g.pageRank(tol, resetProb).vertices.cache() + + assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + + // Computed in igraph 1.0 w/ R bindings: + // > page_rank(graph_from_literal( A -+ B -+ C -+ A -+ D)) + // Alternatively in NetworkX 1.11: + // > nx.pagerank(nx.DiGraph([(1,2),(2,3),(3,1),(1,4)])) + // We multiply by the number of vertices to account for difference in normalization + val igraphPR = Seq(0.3078534, 0.2137622, 0.2646223, 0.2137622).map(_ * 4) + val ranks = VertexRDD(sc.parallelize(1L to 4L zip igraphPR)) + assert(compareRanks(staticRanks, ranks) < errorTol) + assert(compareRanks(dynamicRanks, ranks) < errorTol) + + val p1staticRanks = g.staticPersonalizedPageRank(1, numIter, resetProb).vertices.cache() + val p1dynamicRanks = g.personalizedPageRank(1, tol, resetProb).vertices.cache() + val p1parallelDynamicRanks = + g.staticParallelPersonalizedPageRank(Array(1, 2, 3, 4), numIter, resetProb) + .vertices.mapValues(v => v(0)).cache() + + // Computed in igraph 1.0 w/ R bindings: + // > page_rank(graph_from_literal( A -+ B -+ C -+ A -+ D), personalized = c(1, 0, 0, 0), + // algo = "arpack") + // NOTE: We use the arpack algorithm as prpack (the default) redistributes rank to all + // vertices uniformly instead of just to the personalization source. + // Alternatively in NetworkX 1.11: + // > nx.pagerank(nx.DiGraph([(1,2),(2,3),(3,1),(1,4)]), personalization={1:1, 2:0, 3:0, 4:0}) + val igraphPR2 = Seq(0.4522329, 0.1921990, 0.1633691, 0.1921990) + val ranks2 = VertexRDD(sc.parallelize(1L to 4L zip igraphPR2)) + assert(compareRanks(p1staticRanks, ranks2) < errorTol) + assert(compareRanks(p1dynamicRanks, ranks2) < errorTol) + assert(compareRanks(p1parallelDynamicRanks, ranks2) < errorTol) + } } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala index d7eaa70ce640..994395bbffa5 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala @@ -17,12 +17,8 @@ package org.apache.spark.graphx.lib -import org.apache.spark.{SparkContext, SparkFunSuite} -import org.apache.spark.SparkContext._ +import org.apache.spark.SparkFunSuite import org.apache.spark.graphx._ -import org.apache.spark.graphx.lib._ -import org.apache.spark.graphx.util.GraphGenerators -import org.apache.spark.rdd._ class ShortestPathsSuite extends SparkFunSuite with LocalSparkContext { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala index d6b03208180d..2c57e8927e4d 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala @@ -17,11 +17,8 @@ package org.apache.spark.graphx.lib -import org.apache.spark.{SparkContext, SparkFunSuite} -import org.apache.spark.SparkContext._ +import org.apache.spark.SparkFunSuite import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.GraphGenerators -import org.apache.spark.rdd._ class StronglyConnectedComponentsSuite extends SparkFunSuite with LocalSparkContext { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala index c47552cf3a3b..f19c3acdc85c 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala @@ -26,7 +26,7 @@ class TriangleCountSuite extends SparkFunSuite with LocalSparkContext { test("Count a single triangle") { withSpark { sc => - val rawEdges = sc.parallelize(Array( 0L->1L, 1L->2L, 2L->0L ), 2) + val rawEdges = sc.parallelize(Array( 0L -> 1L, 1L -> 2L, 2L -> 0L ), 2) val graph = Graph.fromEdgeTuples(rawEdges, true).cache() val triangleCount = graph.triangleCount() val verts = triangleCount.vertices @@ -64,9 +64,9 @@ class TriangleCountSuite extends SparkFunSuite with LocalSparkContext { val verts = triangleCount.vertices verts.collect().foreach { case (vid, count) => if (vid == 0) { - assert(count === 4) - } else { assert(count === 2) + } else { + assert(count === 1) } } } @@ -75,7 +75,8 @@ class TriangleCountSuite extends SparkFunSuite with LocalSparkContext { test("Count a single triangle with duplicate edges") { withSpark { sc => val rawEdges = sc.parallelize(Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ - Array(0L -> 1L, 1L -> 2L, 2L -> 0L), 2) + Array(0L -> 1L, 1L -> 2L, 2L -> 0L) ++ + Array(1L -> 0L, 1L -> 1L), 2) val graph = Graph.fromEdgeTuples(rawEdges, true, uniqueEdges = Some(RandomVertexCut)).cache() val triangleCount = graph.triangleCount() val verts = triangleCount.vertices diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointerSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointerSuite.scala new file mode 100644 index 000000000000..e0c65e6940f6 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointerSuite.scala @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx.util + +import org.apache.hadoop.fs.Path + +import org.apache.spark.{SparkContext, SparkFunSuite} +import org.apache.spark.graphx.{Edge, Graph, LocalSparkContext} +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils + + +class PeriodicGraphCheckpointerSuite extends SparkFunSuite with LocalSparkContext { + + import PeriodicGraphCheckpointerSuite._ + + test("Persisting") { + var graphsToCheck = Seq.empty[GraphToCheck] + + withSpark { sc => + val graph1 = createGraph(sc) + val checkpointer = + new PeriodicGraphCheckpointer[Double, Double](10, graph1.vertices.sparkContext) + checkpointer.update(graph1) + graphsToCheck = graphsToCheck :+ GraphToCheck(graph1, 1) + checkPersistence(graphsToCheck, 1) + + var iteration = 2 + while (iteration < 9) { + val graph = createGraph(sc) + checkpointer.update(graph) + graphsToCheck = graphsToCheck :+ GraphToCheck(graph, iteration) + checkPersistence(graphsToCheck, iteration) + iteration += 1 + } + } + } + + test("Checkpointing") { + withSpark { sc => + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + val checkpointInterval = 2 + var graphsToCheck = Seq.empty[GraphToCheck] + sc.setCheckpointDir(path) + val graph1 = createGraph(sc) + val checkpointer = new PeriodicGraphCheckpointer[Double, Double]( + checkpointInterval, graph1.vertices.sparkContext) + checkpointer.update(graph1) + graph1.edges.count() + graph1.vertices.count() + graphsToCheck = graphsToCheck :+ GraphToCheck(graph1, 1) + checkCheckpoint(graphsToCheck, 1, checkpointInterval) + + var iteration = 2 + while (iteration < 9) { + val graph = createGraph(sc) + checkpointer.update(graph) + graph.vertices.count() + graph.edges.count() + graphsToCheck = graphsToCheck :+ GraphToCheck(graph, iteration) + checkCheckpoint(graphsToCheck, iteration, checkpointInterval) + iteration += 1 + } + + checkpointer.deleteAllCheckpoints() + graphsToCheck.foreach { graph => + confirmCheckpointRemoved(graph.graph) + } + + Utils.deleteRecursively(tempDir) + } + } +} + +private object PeriodicGraphCheckpointerSuite { + private val defaultStorageLevel = StorageLevel.MEMORY_ONLY_SER + + case class GraphToCheck(graph: Graph[Double, Double], gIndex: Int) + + val edges = Seq( + Edge[Double](0, 1, 0), + Edge[Double](1, 2, 0), + Edge[Double](2, 3, 0), + Edge[Double](3, 4, 0)) + + def createGraph(sc: SparkContext): Graph[Double, Double] = { + Graph.fromEdges[Double, Double]( + sc.parallelize(edges), 0, defaultStorageLevel, defaultStorageLevel) + } + + def checkPersistence(graphs: Seq[GraphToCheck], iteration: Int): Unit = { + graphs.foreach { g => + checkPersistence(g.graph, g.gIndex, iteration) + } + } + + /** + * Check storage level of graph. + * @param gIndex Index of graph in order inserted into checkpointer (from 1). + * @param iteration Total number of graphs inserted into checkpointer. + */ + def checkPersistence(graph: Graph[_, _], gIndex: Int, iteration: Int): Unit = { + try { + if (gIndex + 2 < iteration) { + assert(graph.vertices.getStorageLevel == StorageLevel.NONE) + assert(graph.edges.getStorageLevel == StorageLevel.NONE) + } else { + assert(graph.vertices.getStorageLevel == defaultStorageLevel) + assert(graph.edges.getStorageLevel == defaultStorageLevel) + } + } catch { + case _: AssertionError => + throw new Exception(s"PeriodicGraphCheckpointerSuite.checkPersistence failed with:\n" + + s"\t gIndex = $gIndex\n" + + s"\t iteration = $iteration\n" + + s"\t graph.vertices.getStorageLevel = ${graph.vertices.getStorageLevel}\n" + + s"\t graph.edges.getStorageLevel = ${graph.edges.getStorageLevel}\n") + } + } + + def checkCheckpoint(graphs: Seq[GraphToCheck], iteration: Int, checkpointInterval: Int): Unit = { + graphs.reverse.foreach { g => + checkCheckpoint(g.graph, g.gIndex, iteration, checkpointInterval) + } + } + + def confirmCheckpointRemoved(graph: Graph[_, _]): Unit = { + // Note: We cannot check graph.isCheckpointed since that value is never updated. + // Instead, we check for the presence of the checkpoint files. + // This test should continue to work even after this graph.isCheckpointed issue + // is fixed (though it can then be simplified and not look for the files). + val hadoopConf = graph.vertices.sparkContext.hadoopConfiguration + graph.getCheckpointFiles.foreach { checkpointFile => + val path = new Path(checkpointFile) + val fs = path.getFileSystem(hadoopConf) + assert(!fs.exists(path), + "Graph checkpoint file should have been removed") + } + } + + /** + * Check checkpointed status of graph. + * @param gIndex Index of graph in order inserted into checkpointer (from 1). + * @param iteration Total number of graphs inserted into checkpointer. + */ + def checkCheckpoint( + graph: Graph[_, _], + gIndex: Int, + iteration: Int, + checkpointInterval: Int): Unit = { + try { + if (gIndex % checkpointInterval == 0) { + // We allow 2 checkpoint intervals since we perform an action (checkpointing a second graph) + // only AFTER PeriodicGraphCheckpointer decides whether to remove the previous checkpoint. + if (iteration - 2 * checkpointInterval < gIndex && gIndex <= iteration) { + assert(graph.isCheckpointed, "Graph should be checkpointed") + assert(graph.getCheckpointFiles.length == 2, "Graph should have 2 checkpoint files") + } else { + confirmCheckpointRemoved(graph) + } + } else { + // Graph should never be checkpointed + assert(!graph.isCheckpointed, "Graph should never have been checkpointed") + assert(graph.getCheckpointFiles.isEmpty, "Graph should not have any checkpoint files") + } + } catch { + case e: AssertionError => + throw new Exception(s"PeriodicGraphCheckpointerSuite.checkCheckpoint failed with:\n" + + s"\t gIndex = $gIndex\n" + + s"\t iteration = $iteration\n" + + s"\t checkpointInterval = $checkpointInterval\n" + + s"\t graph.isCheckpointed = ${graph.isCheckpointed}\n" + + s"\t graph.getCheckpointFiles = ${graph.getCheckpointFiles.mkString(", ")}\n" + + s" AssertionError message: ${e.getMessage}") + } + } + +} diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml new file mode 100644 index 000000000000..aa36dd4774d8 --- /dev/null +++ b/hadoop-cloud/pom.xml @@ -0,0 +1,185 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../pom.xml + + + spark-hadoop-cloud_2.11 + jar + Spark Project Cloud Integration through Hadoop Libraries + + Contains support for cloud infrastructures, specifically the Hadoop JARs and + transitive dependencies needed to interact with the infrastructures, + making everything consistent with Spark's other dependencies. + + + hadoop-cloud + + + + + + org.apache.hadoop + hadoop-aws + ${hadoop.version} + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-common + + + commons-logging + commons-logging + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-core-asl + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + + + org.apache.hadoop + hadoop-openstack + ${hadoop.version} + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-common + + + commons-logging + commons-logging + + + junit + junit + + + org.mockito + mockito-all + + + + + + + joda-time + joda-time + ${hadoop.deps.scope} + + + + com.fasterxml.jackson.core + jackson-databind + ${hadoop.deps.scope} + + + com.fasterxml.jackson.core + jackson-annotations + ${hadoop.deps.scope} + + + com.fasterxml.jackson.dataformat + jackson-dataformat-cbor + ${fasterxml.jackson.version} + + + + org.apache.httpcomponents + httpclient + ${hadoop.deps.scope} + + + + org.apache.httpcomponents + httpcore + ${hadoop.deps.scope} + + + + + + + hadoop-2.7 + + + + + + org.apache.hadoop + hadoop-azure + ${hadoop.version} + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-common + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.jackson.core + jackson-core + + + com.google.guava + guava + + + + + + + + + diff --git a/launcher/pom.xml b/launcher/pom.xml index 5739bfc16958..e9b46c4cf0ff 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -21,13 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT + spark-parent_2.11 + 2.3.0-SNAPSHOT ../pom.xml - org.apache.spark - spark-launcher_2.10 + spark-launcher_2.11 jar Spark Project Launcher http://spark.apache.org/ @@ -65,7 +64,18 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index 3ee6bd92e47f..ce24400f557c 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -19,18 +19,18 @@ import java.io.BufferedReader; import java.io.File; -import java.io.FileFilter; import java.io.FileInputStream; import java.io.InputStreamReader; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Properties; -import java.util.jar.JarFile; +import java.util.Set; import java.util.regex.Pattern; import static org.apache.spark.launcher.CommandBuilderUtils.*; @@ -59,13 +59,13 @@ abstract class AbstractCommandBuilder { // properties files multiple times. private Map effectiveConfig; - public AbstractCommandBuilder() { - this.appArgs = new ArrayList(); - this.childEnv = new HashMap(); - this.conf = new HashMap(); - this.files = new ArrayList(); - this.jars = new ArrayList(); - this.pyFiles = new ArrayList(); + AbstractCommandBuilder() { + this.appArgs = new ArrayList<>(); + this.childEnv = new HashMap<>(); + this.conf = new HashMap<>(); + this.files = new ArrayList<>(); + this.jars = new ArrayList<>(); + this.pyFiles = new ArrayList<>(); } /** @@ -76,7 +76,8 @@ public AbstractCommandBuilder() { * SparkLauncher constructor that takes an environment), and may be modified to * include other variables needed by the process to be executed. */ - abstract List buildCommand(Map env) throws IOException; + abstract List buildCommand(Map env) + throws IOException, IllegalArgumentException; /** * Builds a list of arguments to run java. @@ -89,7 +90,7 @@ public AbstractCommandBuilder() { * class. */ List buildJavaCommand(String extraClassPath) throws IOException { - List cmd = new ArrayList(); + List cmd = new ArrayList<>(); String envJavaHome; if (javaHome != null) { @@ -103,15 +104,12 @@ List buildJavaCommand(String extraClassPath) throws IOException { // Load extra JAVA_OPTS from conf/java-opts, if it exists. File javaOpts = new File(join(File.separator, getConfDir(), "java-opts")); if (javaOpts.isFile()) { - BufferedReader br = new BufferedReader(new InputStreamReader( - new FileInputStream(javaOpts), "UTF-8")); - try { + try (BufferedReader br = new BufferedReader(new InputStreamReader( + new FileInputStream(javaOpts), StandardCharsets.UTF_8))) { String line; while ((line = br.readLine()) != null) { addOptionString(cmd, line); } - } finally { - br.close(); } } @@ -136,8 +134,7 @@ void addOptionString(List cmd, String options) { List buildClassPath(String appClassPath) throws IOException { String sparkHome = getSparkHome(); - List cp = new ArrayList(); - addToClassPath(cp, getenv("SPARK_CLASSPATH")); + Set cp = new LinkedHashSet<>(); addToClassPath(cp, appClassPath); addToClassPath(cp, getConfDir()); @@ -146,9 +143,28 @@ List buildClassPath(String appClassPath) throws IOException { boolean isTesting = "1".equals(getenv("SPARK_TESTING")); if (prependClasses || isTesting) { String scala = getScalaVersion(); - List projects = Arrays.asList("core", "repl", "mllib", "bagel", "graphx", - "streaming", "tools", "sql/catalyst", "sql/core", "sql/hive", "sql/hive-thriftserver", - "yarn", "launcher"); + List projects = Arrays.asList( + "common/kvstore", + "common/network-common", + "common/network-shuffle", + "common/network-yarn", + "common/sketch", + "common/tags", + "common/unsafe", + "core", + "examples", + "graphx", + "launcher", + "mllib", + "repl", + "resource-managers/mesos", + "resource-managers/yarn", + "sql/catalyst", + "sql/core", + "sql/hive", + "sql/hive-thriftserver", + "streaming" + ); if (prependClasses) { if (!isTesting) { System.err.println( @@ -170,48 +186,22 @@ List buildClassPath(String appClassPath) throws IOException { // Add this path to include jars that are shaded in the final deliverable created during // the maven build. These jars are copied to this directory during the build. addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome)); + addToClassPath(cp, String.format("%s/mllib/target/jars/*", sparkHome)); } - // We can't rely on the ENV_SPARK_ASSEMBLY variable to be set. Certain situations, such as - // when running unit tests, or user code that embeds Spark and creates a SparkContext - // with a local or local-cluster master, will cause this code to be called from an - // environment where that env variable is not guaranteed to exist. - // - // For the testing case, we rely on the test code to set and propagate the test classpath - // appropriately. - // - // For the user code case, we fall back to looking for the Spark assembly under SPARK_HOME. - // That duplicates some of the code in the shell scripts that look for the assembly, though. - String assembly = getenv(ENV_SPARK_ASSEMBLY); - if (assembly == null && !isTesting) { - assembly = findAssembly(); - } - addToClassPath(cp, assembly); - - // Datanucleus jars must be included on the classpath. Datanucleus jars do not work if only - // included in the uber jar as plugin.xml metadata is lost. Both sbt and maven will populate - // "lib_managed/jars/" with the datanucleus jars when Spark is built with Hive - File libdir; - if (new File(sparkHome, "RELEASE").isFile()) { - libdir = new File(sparkHome, "lib"); - } else { - libdir = new File(sparkHome, "lib_managed/jars"); - } - - if (libdir.isDirectory()) { - for (File jar : libdir.listFiles()) { - if (jar.getName().startsWith("datanucleus-")) { - addToClassPath(cp, jar.getAbsolutePath()); - } - } - } else { - checkState(isTesting, "Library directory '%s' does not exist.", libdir.getAbsolutePath()); + // Add Spark jars to the classpath. For the testing case, we rely on the test code to set and + // propagate the test classpath appropriately. For normal invocation, look for the jars + // directory under SPARK_HOME. + boolean isTestingSql = "1".equals(getenv("SPARK_SQL_TESTING")); + String jarsDir = findJarsDir(getSparkHome(), getScalaVersion(), !isTesting && !isTestingSql); + if (jarsDir != null) { + addToClassPath(cp, join(File.separator, jarsDir, "*")); } addToClassPath(cp, getenv("HADOOP_CONF_DIR")); addToClassPath(cp, getenv("YARN_CONF_DIR")); addToClassPath(cp, getenv("SPARK_DIST_CLASSPATH")); - return cp; + return new ArrayList<>(cp); } /** @@ -220,7 +210,7 @@ List buildClassPath(String appClassPath) throws IOException { * @param cp List to which the new entries are appended. * @param entries New classpath entries (separated by File.pathSeparator). */ - private void addToClassPath(List cp, String entries) { + private void addToClassPath(Set cp, String entries) { if (isEmpty(entries)) { return; } @@ -241,13 +231,13 @@ String getScalaVersion() { return scala; } String sparkHome = getSparkHome(); - File scala210 = new File(sparkHome, "launcher/target/scala-2.10"); + File scala212 = new File(sparkHome, "launcher/target/scala-2.12"); File scala211 = new File(sparkHome, "launcher/target/scala-2.11"); - checkState(!scala210.isDirectory() || !scala211.isDirectory(), - "Presence of build for both scala versions (2.10 and 2.11) detected.\n" + + checkState(!scala212.isDirectory() || !scala211.isDirectory(), + "Presence of build for multiple Scala versions detected.\n" + "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); - if (scala210.isDirectory()) { - return "2.10"; + if (scala212.isDirectory()) { + return "2.12"; } else { checkState(scala211.isDirectory(), "Cannot find any build directories."); return "2.11"; @@ -256,6 +246,9 @@ String getScalaVersion() { String getSparkHome() { String path = getenv(ENV_SPARK_HOME); + if (path == null && "1".equals(getenv("SPARK_TESTING"))) { + path = System.getProperty("spark.test.home"); + } checkState(path != null, "Spark home not found; set it explicitly or use the SPARK_HOME environment variable."); return path; @@ -299,51 +292,17 @@ private Properties loadPropertiesFile() throws IOException { } if (propsFile.isFile()) { - FileInputStream fd = null; - try { - fd = new FileInputStream(propsFile); - props.load(new InputStreamReader(fd, "UTF-8")); + try (InputStreamReader isr = new InputStreamReader( + new FileInputStream(propsFile), StandardCharsets.UTF_8)) { + props.load(isr); for (Map.Entry e : props.entrySet()) { e.setValue(e.getValue().toString().trim()); } - } finally { - if (fd != null) { - try { - fd.close(); - } catch (IOException e) { - // Ignore. - } - } } } - return props; } - private String findAssembly() { - String sparkHome = getSparkHome(); - File libdir; - if (new File(sparkHome, "RELEASE").isFile()) { - libdir = new File(sparkHome, "lib"); - checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", - libdir.getAbsolutePath()); - } else { - libdir = new File(sparkHome, String.format("assembly/target/scala-%s", getScalaVersion())); - } - - final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar"); - FileFilter filter = new FileFilter() { - @Override - public boolean accept(File file) { - return file.isFile() && re.matcher(file.getName()).matches(); - } - }; - File[] assemblies = libdir.listFiles(filter); - checkState(assemblies != null && assemblies.length > 0, "No assemblies found in '%s'.", libdir); - checkState(assemblies.length == 1, "Multiple assemblies found in '%s'.", libdir); - return assemblies[0].getAbsolutePath(); - } - private String getConfDir() { String confDir = getenv("SPARK_CONF_DIR"); return confDir != null ? confDir : join(File.separator, getSparkHome(), "conf"); diff --git a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java index de50f14fbdc8..5391d4a50fe4 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/ChildProcAppHandle.java @@ -18,9 +18,9 @@ package org.apache.spark.launcher; import java.io.IOException; +import java.io.InputStream; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.ThreadFactory; import java.util.logging.Level; import java.util.logging.Logger; @@ -30,13 +30,11 @@ class ChildProcAppHandle implements SparkAppHandle { private static final Logger LOG = Logger.getLogger(ChildProcAppHandle.class.getName()); - private static final ThreadFactory REDIRECTOR_FACTORY = - new NamedThreadFactory("launcher-proc-%d"); private final String secret; private final LauncherServer server; - private Process childProc; + private volatile Process childProc; private boolean disposed; private LauncherConnection connection; private List listeners; @@ -98,23 +96,32 @@ public synchronized void disconnect() { @Override public synchronized void kill() { - if (!disposed) { - disconnect(); - } + disconnect(); if (childProc != null) { - childProc.destroy(); + if (childProc.isAlive()) { + childProc.destroyForcibly(); + } childProc = null; } + setState(State.KILLED); } String getSecret() { return secret; } - void setChildProc(Process childProc, String loggerName) { + void setChildProc(Process childProc, String loggerName, InputStream logStream) { this.childProc = childProc; - this.redirector = new OutputRedirector(childProc.getInputStream(), loggerName, - REDIRECTOR_FACTORY); + if (logStream != null) { + this.redirector = new OutputRedirector(logStream, loggerName, + SparkLauncher.REDIRECTOR_FACTORY, this); + } else { + // If there is no log redirection, spawn a thread that will wait for the child process + // to finish. + Thread waiter = SparkLauncher.REDIRECTOR_FACTORY.newThread(this::monitorChild); + waiter.setDaemon(true); + waiter.start(); + } } void setConnection(LauncherConnection connection) { @@ -129,7 +136,7 @@ LauncherConnection getConnection() { return connection; } - void setState(State s) { + synchronized void setState(State s) { if (!state.isFinal()) { state = s; fireEvent(false); @@ -139,12 +146,63 @@ void setState(State s) { } } - void setAppId(String appId) { + synchronized void setAppId(String appId) { this.appId = appId; fireEvent(true); } - private synchronized void fireEvent(boolean isInfoChanged) { + /** + * Wait for the child process to exit and update the handle's state if necessary, accoding to + * the exit code. + */ + void monitorChild() { + Process proc = childProc; + if (proc == null) { + // Process may have already been disposed of, e.g. by calling kill(). + return; + } + + while (proc.isAlive()) { + try { + proc.waitFor(); + } catch (Exception e) { + LOG.log(Level.WARNING, "Exception waiting for child process to exit.", e); + } + } + + synchronized (this) { + if (disposed) { + return; + } + + disconnect(); + + int ec; + try { + ec = proc.exitValue(); + } catch (Exception e) { + LOG.log(Level.WARNING, "Exception getting child process exit code, assuming failure.", e); + ec = 1; + } + + State newState = null; + if (ec != 0) { + // Override state with failure if the current state is not final, or is success. + if (!state.isFinal() || state == State.FINISHED) { + newState = State.FAILED; + } + } else if (!state.isFinal()) { + newState = State.LOST; + } + + if (newState != null) { + state = newState; + fireEvent(false); + } + } + } + + private void fireEvent(boolean isInfoChanged) { if (listeners != null) { for (Listener l : listeners) { if (isInfoChanged) { diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java index d30c2ec5f87b..47d2f8ef4e3d 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java @@ -30,12 +30,11 @@ class CommandBuilderUtils { static final String DEFAULT_MEM = "1g"; static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf"; static final String ENV_SPARK_HOME = "SPARK_HOME"; - static final String ENV_SPARK_ASSEMBLY = "_SPARK_ASSEMBLY"; /** The set of known JVM vendors. */ - static enum JavaVendor { + enum JavaVendor { Oracle, IBM, OpenJDK, Unknown - }; + } /** Returns whether the given string is null or empty. */ static boolean isEmpty(String s) { @@ -147,7 +146,7 @@ static void mergeEnvPathList(Map userEnv, String envKey, String * Output: [ "ab cd", "efgh", "i \" j" ] */ static List parseOptionString(String s) { - List opts = new ArrayList(); + List opts = new ArrayList<>(); StringBuilder opt = new StringBuilder(); boolean inOpt = false; boolean inSingleQuote = false; @@ -314,26 +313,39 @@ static String quoteForCommandString(String s) { } /** - * Adds the default perm gen size option for Spark if the VM requires it and the user hasn't - * set it. + * Get the major version of the java version string supplied. This method + * accepts any JEP-223-compliant strings (9-ea, 9+100), as well as legacy + * version strings such as 1.7.0_79 */ - static void addPermGenSizeOpt(List cmd) { - // Don't set MaxPermSize for IBM Java, or Oracle Java 8 and later. - if (getJavaVendor() == JavaVendor.IBM) { - return; - } - String[] version = System.getProperty("java.version").split("\\."); - if (Integer.parseInt(version[0]) > 1 || Integer.parseInt(version[1]) > 7) { - return; + static int javaMajorVersion(String javaVersion) { + String[] version = javaVersion.split("[+.\\-]+"); + int major = Integer.parseInt(version[0]); + // if major > 1, we're using the JEP-223 version string, e.g., 9-ea, 9+120 + // otherwise the second number is the major version + if (major > 1) { + return major; + } else { + return Integer.parseInt(version[1]); } + } - for (String arg : cmd) { - if (arg.startsWith("-XX:MaxPermSize=")) { - return; + /** + * Find the location of the Spark jars dir, depending on whether we're looking at a build + * or a distribution directory. + */ + static String findJarsDir(String sparkHome, String scalaVersion, boolean failIfNotFound) { + // TODO: change to the correct directory once the assembly build is changed. + File libdir = new File(sparkHome, "jars"); + if (!libdir.isDirectory()) { + libdir = new File(sparkHome, String.format("assembly/target/scala-%s/jars", scalaVersion)); + if (!libdir.isDirectory()) { + checkState(!failIfNotFound, + "Library directory '%s' does not exist; make sure Spark is built.", + libdir.getAbsolutePath()); + return null; } } - - cmd.add("-XX:MaxPermSize=256m"); + return libdir.getAbsolutePath(); } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/FilteredObjectInputStream.java b/launcher/src/main/java/org/apache/spark/launcher/FilteredObjectInputStream.java new file mode 100644 index 000000000000..4d254a0c4c9f --- /dev/null +++ b/launcher/src/main/java/org/apache/spark/launcher/FilteredObjectInputStream.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.launcher; + +import java.io.InputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectStreamClass; +import java.util.Arrays; +import java.util.List; + +/** + * An object input stream that only allows classes used by the launcher protocol to be in the + * serialized stream. See SPARK-20922. + */ +class FilteredObjectInputStream extends ObjectInputStream { + + private static final List ALLOWED_PACKAGES = Arrays.asList( + "org.apache.spark.launcher.", + "java.lang."); + + FilteredObjectInputStream(InputStream is) throws IOException { + super(is); + } + + @Override + protected Class resolveClass(ObjectStreamClass desc) + throws IOException, ClassNotFoundException { + + boolean isValid = ALLOWED_PACKAGES.stream().anyMatch(p -> desc.getName().startsWith(p)); + if (!isValid) { + throw new IllegalArgumentException( + String.format("Unexpected class in stream: %s", desc.getName())); + } + return super.resolveClass(desc); + } + +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherConnection.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherConnection.java index eec264909bbb..b4a8719e2605 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherConnection.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherConnection.java @@ -20,7 +20,6 @@ import java.io.Closeable; import java.io.EOFException; import java.io.IOException; -import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.net.Socket; import java.util.logging.Level; @@ -53,7 +52,7 @@ abstract class LauncherConnection implements Closeable, Runnable { @Override public void run() { try { - ObjectInputStream in = new ObjectInputStream(socket.getInputStream()); + FilteredObjectInputStream in = new FilteredObjectInputStream(socket.getInputStream()); while (!closed) { Message msg = (Message) in.readObject(); handle(msg); diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherProtocol.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherProtocol.java index 50f136497ec1..042f11cd9e43 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherProtocol.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherProtocol.java @@ -17,13 +17,7 @@ package org.apache.spark.launcher; -import java.io.Closeable; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.io.Serializable; -import java.net.Socket; -import java.util.Map; /** * Message definitions for the launcher communication protocol. These messages must remain diff --git a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java index d099ee9aa9da..865d4926da6a 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java +++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java @@ -129,7 +129,7 @@ private LauncherServer() throws IOException { server.setReuseAddress(true); server.bind(new InetSocketAddress(InetAddress.getLoopbackAddress(), 0)); - this.clients = new ArrayList(); + this.clients = new ArrayList<>(); this.threadIds = new AtomicLong(); this.factory = new NamedThreadFactory(THREAD_NAME_FMT); this.pending = new ConcurrentHashMap<>(); @@ -137,12 +137,7 @@ private LauncherServer() throws IOException { this.server = server; this.running = true; - this.serverThread = factory.newThread(new Runnable() { - @Override - public void run() { - acceptConnections(); - } - }); + this.serverThread = factory.newThread(this::acceptConnections); serverThread.start(); } catch (IOException ioe) { close(); @@ -293,15 +288,13 @@ private class ServerConnection extends LauncherConnection { protected void handle(Message msg) throws IOException { try { if (msg instanceof Hello) { - synchronized (timeout) { - timeout.cancel(); - } + timeout.cancel(); timeout = null; Hello hello = (Hello) msg; ChildProcAppHandle handle = pending.remove(hello.secret); if (handle != null) { - handle.setState(SparkAppHandle.State.CONNECTED); handle.setConnection(this); + handle.setState(SparkAppHandle.State.CONNECTED); this.handle = handle; } else { throw new IllegalArgumentException("Received Hello for unknown client."); @@ -339,6 +332,10 @@ public void close() throws IOException { } super.close(); if (handle != null) { + if (!handle.getState().isFinal()) { + LOG.log(Level.WARNING, "Lost connection to spark application."); + handle.setState(SparkAppHandle.State.LOST); + } handle.disconnect(); } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java index a4e3acc674f3..1e34bb8c7327 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/Main.java +++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java @@ -50,7 +50,7 @@ class Main { public static void main(String[] argsArray) throws Exception { checkArgument(argsArray.length > 0, "Not enough arguments: missing class name."); - List args = new ArrayList(Arrays.asList(argsArray)); + List args = new ArrayList<>(Arrays.asList(argsArray)); String className = args.remove(0); boolean printLaunchCommand = !isEmpty(System.getenv("SPARK_PRINT_LAUNCH_COMMAND")); @@ -70,7 +70,7 @@ public static void main(String[] argsArray) throws Exception { // Ignore parsing exceptions. } - List help = new ArrayList(); + List help = new ArrayList<>(); if (parser.className != null) { help.add(parser.CLASS); help.add(parser.className); @@ -82,7 +82,7 @@ public static void main(String[] argsArray) throws Exception { builder = new SparkClassCommandBuilder(className, args); } - Map env = new HashMap(); + Map env = new HashMap<>(); List cmd = builder.buildCommand(env); if (printLaunchCommand) { System.err.println("Spark Command: " + join(" ", cmd)); @@ -130,7 +130,7 @@ private static List prepareBashCommand(List cmd, Map newCmd = new ArrayList(); + List newCmd = new ArrayList<>(); newCmd.add("env"); for (Map.Entry e : childEnv.entrySet()) { @@ -151,7 +151,7 @@ private static class MainClassOptionParser extends SparkSubmitOptionParser { @Override protected boolean handle(String opt, String value) { - if (opt == CLASS) { + if (CLASS.equals(opt)) { className = value; } return false; diff --git a/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java b/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java index 6e7120167d60..6f4b0bb38e03 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java +++ b/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java @@ -21,6 +21,7 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.concurrent.ThreadFactory; import java.util.logging.Level; import java.util.logging.Logger; @@ -33,23 +34,24 @@ class OutputRedirector { private final BufferedReader reader; private final Logger sink; private final Thread thread; + private final ChildProcAppHandle callback; private volatile boolean active; - OutputRedirector(InputStream in, ThreadFactory tf) { - this(in, OutputRedirector.class.getName(), tf); + OutputRedirector(InputStream in, String loggerName, ThreadFactory tf) { + this(in, loggerName, tf, null); } - OutputRedirector(InputStream in, String loggerName, ThreadFactory tf) { + OutputRedirector( + InputStream in, + String loggerName, + ThreadFactory tf, + ChildProcAppHandle callback) { this.active = true; - this.reader = new BufferedReader(new InputStreamReader(in)); - this.thread = tf.newThread(new Runnable() { - @Override - public void run() { - redirect(); - } - }); + this.reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8)); + this.thread = tf.newThread(this::redirect); this.sink = Logger.getLogger(loggerName); + this.callback = callback; thread.start(); } @@ -63,6 +65,10 @@ private void redirect() { } } catch (IOException e) { sink.log(Level.FINE, "Error reading child process output.", e); + } finally { + if (callback != null) { + callback.monitorChild(); + } } } @@ -75,4 +81,8 @@ void stop() { active = false; } + boolean isAlive() { + return thread.isAlive(); + } + } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java index 13dd9f1739fb..cefb4d1a95fb 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java @@ -32,7 +32,7 @@ public interface SparkAppHandle { * * @since 1.6.0 */ - public enum State { + enum State { /** The application has not reported back yet. */ UNKNOWN(false), /** The application has connected to the handle. */ @@ -46,7 +46,9 @@ public enum State { /** The application finished with a failed status. */ FAILED(true), /** The application was killed. */ - KILLED(true); + KILLED(true), + /** The Spark Submit JVM exited with a unknown status. */ + LOST(true); private final boolean isFinal; diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java index 931a24cfd4b1..32724acdc362 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java @@ -17,12 +17,10 @@ package org.apache.spark.launcher; -import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.regex.Pattern; import static org.apache.spark.launcher.CommandBuilderUtils.*; @@ -43,71 +41,73 @@ class SparkClassCommandBuilder extends AbstractCommandBuilder { } @Override - public List buildCommand(Map env) throws IOException { - List javaOptsKeys = new ArrayList(); + public List buildCommand(Map env) + throws IOException, IllegalArgumentException { + List javaOptsKeys = new ArrayList<>(); String memKey = null; String extraClassPath = null; - // Master, Worker, and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + - // SPARK_DAEMON_MEMORY. - if (className.equals("org.apache.spark.deploy.master.Master")) { - javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); - javaOptsKeys.add("SPARK_MASTER_OPTS"); - memKey = "SPARK_DAEMON_MEMORY"; - } else if (className.equals("org.apache.spark.deploy.worker.Worker")) { - javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); - javaOptsKeys.add("SPARK_WORKER_OPTS"); - memKey = "SPARK_DAEMON_MEMORY"; - } else if (className.equals("org.apache.spark.deploy.history.HistoryServer")) { - javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); - javaOptsKeys.add("SPARK_HISTORY_OPTS"); - memKey = "SPARK_DAEMON_MEMORY"; - } else if (className.equals("org.apache.spark.executor.CoarseGrainedExecutorBackend")) { - javaOptsKeys.add("SPARK_JAVA_OPTS"); - javaOptsKeys.add("SPARK_EXECUTOR_OPTS"); - memKey = "SPARK_EXECUTOR_MEMORY"; - } else if (className.equals("org.apache.spark.executor.MesosExecutorBackend")) { - javaOptsKeys.add("SPARK_EXECUTOR_OPTS"); - memKey = "SPARK_EXECUTOR_MEMORY"; - } else if (className.equals("org.apache.spark.deploy.ExternalShuffleService") || - className.equals("org.apache.spark.deploy.mesos.MesosExternalShuffleService")) { - javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); - javaOptsKeys.add("SPARK_SHUFFLE_OPTS"); - memKey = "SPARK_DAEMON_MEMORY"; - } else if (className.startsWith("org.apache.spark.tools.")) { - String sparkHome = getSparkHome(); - File toolsDir = new File(join(File.separator, sparkHome, "tools", "target", - "scala-" + getScalaVersion())); - checkState(toolsDir.isDirectory(), "Cannot find tools build directory."); - - Pattern re = Pattern.compile("spark-tools_.*\\.jar"); - for (File f : toolsDir.listFiles()) { - if (re.matcher(f.getName()).matches()) { - extraClassPath = f.getAbsolutePath(); - break; - } - } - - checkState(extraClassPath != null, - "Failed to find Spark Tools Jar in %s.\n" + - "You need to run \"build/sbt tools/package\" before running %s.", - toolsDir.getAbsolutePath(), className); - - javaOptsKeys.add("SPARK_JAVA_OPTS"); - } else { - javaOptsKeys.add("SPARK_JAVA_OPTS"); - memKey = "SPARK_DRIVER_MEMORY"; + // Master, Worker, HistoryServer, ExternalShuffleService, MesosClusterDispatcher use + // SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. + switch (className) { + case "org.apache.spark.deploy.master.Master": + javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); + javaOptsKeys.add("SPARK_MASTER_OPTS"); + extraClassPath = getenv("SPARK_DAEMON_CLASSPATH"); + memKey = "SPARK_DAEMON_MEMORY"; + break; + case "org.apache.spark.deploy.worker.Worker": + javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); + javaOptsKeys.add("SPARK_WORKER_OPTS"); + extraClassPath = getenv("SPARK_DAEMON_CLASSPATH"); + memKey = "SPARK_DAEMON_MEMORY"; + break; + case "org.apache.spark.deploy.history.HistoryServer": + javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); + javaOptsKeys.add("SPARK_HISTORY_OPTS"); + extraClassPath = getenv("SPARK_DAEMON_CLASSPATH"); + memKey = "SPARK_DAEMON_MEMORY"; + break; + case "org.apache.spark.executor.CoarseGrainedExecutorBackend": + javaOptsKeys.add("SPARK_EXECUTOR_OPTS"); + memKey = "SPARK_EXECUTOR_MEMORY"; + extraClassPath = getenv("SPARK_EXECUTOR_CLASSPATH"); + break; + case "org.apache.spark.executor.MesosExecutorBackend": + javaOptsKeys.add("SPARK_EXECUTOR_OPTS"); + memKey = "SPARK_EXECUTOR_MEMORY"; + extraClassPath = getenv("SPARK_EXECUTOR_CLASSPATH"); + break; + case "org.apache.spark.deploy.mesos.MesosClusterDispatcher": + javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); + extraClassPath = getenv("SPARK_DAEMON_CLASSPATH"); + break; + case "org.apache.spark.deploy.ExternalShuffleService": + case "org.apache.spark.deploy.mesos.MesosExternalShuffleService": + javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS"); + javaOptsKeys.add("SPARK_SHUFFLE_OPTS"); + extraClassPath = getenv("SPARK_DAEMON_CLASSPATH"); + memKey = "SPARK_DAEMON_MEMORY"; + break; + default: + memKey = "SPARK_DRIVER_MEMORY"; + break; } List cmd = buildJavaCommand(extraClassPath); + for (String key : javaOptsKeys) { - addOptionString(cmd, System.getenv(key)); + String envValue = System.getenv(key); + if (!isEmpty(envValue) && envValue.contains("Xmx")) { + String msg = String.format("%s is not allowed to specify max heap(Xmx) memory settings " + + "(was %s). Use the corresponding configuration instead.", key, envValue); + throw new IllegalArgumentException(msg); + } + addOptionString(cmd, envValue); } String mem = firstNonEmpty(memKey != null ? System.getenv(memKey) : null, DEFAULT_MEM); - cmd.add("-Xms" + mem); cmd.add("-Xmx" + mem); - addPermGenSizeOpt(cmd); cmd.add(className); cmd.addAll(classArgs); return cmd; diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index dd1c93af6ca4..718a368a8e73 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -19,11 +19,13 @@ import java.io.File; import java.io.IOException; +import java.io.InputStream; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; import static org.apache.spark.launcher.CommandBuilderUtils.*; @@ -40,6 +42,9 @@ public class SparkLauncher { /** The Spark master. */ public static final String SPARK_MASTER = "spark.master"; + /** The Spark deploy mode. */ + public static final String DEPLOY_MODE = "spark.submit.deployMode"; + /** Configuration key for the driver memory. */ public static final String DRIVER_MEMORY = "spark.driver.memory"; /** Configuration key for the driver class path. */ @@ -60,9 +65,22 @@ public class SparkLauncher { /** Configuration key for the number of executor CPU cores. */ public static final String EXECUTOR_CORES = "spark.executor.cores"; + static final String PYSPARK_DRIVER_PYTHON = "spark.pyspark.driver.python"; + + static final String PYSPARK_PYTHON = "spark.pyspark.python"; + + static final String SPARKR_R_SHELL = "spark.r.shell.command"; + /** Logger name to use when launching a child process. */ public static final String CHILD_PROCESS_LOGGER_NAME = "spark.launcher.childProcLoggerName"; + /** + * A special value for the resource that tells Spark to not try to process the app resource as a + * file. This is useful when the class being executed is added to the application using other + * means - for example, by adding jars using the package download feature. + */ + public static final String NO_RESOURCE = "spark-internal"; + /** * Maximum time (in ms) to wait for a child process to connect back to the launcher server * when using @link{#start()}. @@ -72,7 +90,10 @@ public class SparkLauncher { /** Used internally to create unique logger names. */ private static final AtomicInteger COUNTER = new AtomicInteger(); - static final Map launcherConfig = new HashMap(); + /** Factory for creating OutputRedirector threads. **/ + static final ThreadFactory REDIRECTOR_FACTORY = new NamedThreadFactory("launcher-proc-%d"); + + static final Map launcherConfig = new HashMap<>(); /** * Set a configuration value for the launcher library. These config values do not affect the @@ -89,6 +110,10 @@ public static void setConfig(String name, String value) { // Visible for testing. final SparkSubmitCommandBuilder builder; + File workingDir; + boolean redirectErrorStream; + ProcessBuilder.Redirect errorStream; + ProcessBuilder.Redirect outputStream; public SparkLauncher() { this(null); @@ -348,6 +373,82 @@ public SparkLauncher setVerbose(boolean verbose) { return this; } + /** + * Sets the working directory of spark-submit. + * + * @param dir The directory to set as spark-submit's working directory. + * @return This launcher. + */ + public SparkLauncher directory(File dir) { + workingDir = dir; + return this; + } + + /** + * Specifies that stderr in spark-submit should be redirected to stdout. + * + * @return This launcher. + */ + public SparkLauncher redirectError() { + redirectErrorStream = true; + return this; + } + + /** + * Redirects error output to the specified Redirect. + * + * @param to The method of redirection. + * @return This launcher. + */ + public SparkLauncher redirectError(ProcessBuilder.Redirect to) { + errorStream = to; + return this; + } + + /** + * Redirects standard output to the specified Redirect. + * + * @param to The method of redirection. + * @return This launcher. + */ + public SparkLauncher redirectOutput(ProcessBuilder.Redirect to) { + outputStream = to; + return this; + } + + /** + * Redirects error output to the specified File. + * + * @param errFile The file to which stderr is written. + * @return This launcher. + */ + public SparkLauncher redirectError(File errFile) { + errorStream = ProcessBuilder.Redirect.to(errFile); + return this; + } + + /** + * Redirects error output to the specified File. + * + * @param outFile The file to which stdout is written. + * @return This launcher. + */ + public SparkLauncher redirectOutput(File outFile) { + outputStream = ProcessBuilder.Redirect.to(outFile); + return this; + } + + /** + * Sets all output to be logged and redirected to a logger with the specified name. + * + * @param loggerName The name of the logger to log stdout and stderr. + * @return This launcher. + */ + public SparkLauncher redirectToLog(String loggerName) { + setConf(CHILD_PROCESS_LOGGER_NAME, loggerName); + return this; + } + /** * Launches a sub-process that will start the configured Spark application. *

    @@ -357,7 +458,23 @@ public SparkLauncher setVerbose(boolean verbose) { * @return A process handle for the Spark app. */ public Process launch() throws IOException { - return createBuilder().start(); + ProcessBuilder pb = createBuilder(); + + boolean outputToLog = outputStream == null; + boolean errorToLog = !redirectErrorStream && errorStream == null; + + String loggerName = getLoggerName(); + if (loggerName != null && outputToLog && errorToLog) { + pb.redirectErrorStream(true); + } + + Process childProc = pb.start(); + if (loggerName != null) { + InputStream logStream = outputToLog ? childProc.getInputStream() : childProc.getErrorStream(); + new OutputRedirector(logStream, loggerName, REDIRECTOR_FACTORY); + } + + return childProc; } /** @@ -373,12 +490,13 @@ public Process launch() throws IOException { * a child process, {@link SparkAppHandle#kill()} can still be used to kill the child process. *

    * Currently, all applications are launched as child processes. The child's stdout and stderr - * are merged and written to a logger (see java.util.logging). The logger's name - * can be defined by setting {@link #CHILD_PROCESS_LOGGER_NAME} in the app's configuration. If - * that option is not set, the code will try to derive a name from the application's name or - * main class / script file. If those cannot be determined, an internal, unique name will be - * used. In all cases, the logger name will start with "org.apache.spark.launcher.app", to fit - * more easily into the configuration of commonly-used logging systems. + * are merged and written to a logger (see java.util.logging) only if redirection + * has not otherwise been configured on this SparkLauncher. The logger's name can be + * defined by setting {@link #CHILD_PROCESS_LOGGER_NAME} in the app's configuration. If that + * option is not set, the code will try to derive a name from the application's name or main + * class / script file. If those cannot be determined, an internal, unique name will be used. + * In all cases, the logger name will start with "org.apache.spark.launcher.app", to fit more + * easily into the configuration of commonly-used logging systems. * * @since 1.6.0 * @param listeners Listeners to add to the handle before the app is launched. @@ -390,8 +508,16 @@ public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) thr handle.addListener(l); } - String appName = builder.getEffectiveConfig().get(CHILD_PROCESS_LOGGER_NAME); - if (appName == null) { + String loggerName = getLoggerName(); + ProcessBuilder pb = createBuilder(); + + boolean outputToLog = outputStream == null; + boolean errorToLog = !redirectErrorStream && errorStream == null; + + // Only setup stderr + stdout to logger redirection if user has not otherwise configured output + // redirection. + if (loggerName == null && (outputToLog || errorToLog)) { + String appName; if (builder.appName != null) { appName = builder.appName; } else if (builder.mainClass != null) { @@ -406,16 +532,24 @@ public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) thr } else { appName = String.valueOf(COUNTER.incrementAndGet()); } + String loggerPrefix = getClass().getPackage().getName(); + loggerName = String.format("%s.app.%s", loggerPrefix, appName); + } + + if (outputToLog && errorToLog) { + pb.redirectErrorStream(true); } - String loggerPrefix = getClass().getPackage().getName(); - String loggerName = String.format("%s.app.%s", loggerPrefix, appName); - ProcessBuilder pb = createBuilder().redirectErrorStream(true); pb.environment().put(LauncherProtocol.ENV_LAUNCHER_PORT, String.valueOf(LauncherServer.getServerInstance().getPort())); pb.environment().put(LauncherProtocol.ENV_LAUNCHER_SECRET, handle.getSecret()); try { - handle.setChildProc(pb.start(), loggerName); + Process child = pb.start(); + InputStream logStream = null; + if (loggerName != null) { + logStream = outputToLog ? child.getInputStream() : child.getErrorStream(); + } + handle.setChildProc(child, loggerName, logStream); } catch (IOException ioe) { handle.kill(); throw ioe; @@ -424,17 +558,16 @@ public SparkAppHandle startApplication(SparkAppHandle.Listener... listeners) thr return handle; } - private ProcessBuilder createBuilder() { - List cmd = new ArrayList(); - String script = isWindows() ? "spark-submit.cmd" : "spark-submit"; - cmd.add(join(File.separator, builder.getSparkHome(), "bin", script)); + private ProcessBuilder createBuilder() throws IOException { + List cmd = new ArrayList<>(); + cmd.add(findSparkSubmit()); cmd.addAll(builder.buildSparkSubmitArgs()); // Since the child process is a batch script, let's quote things so that special characters are // preserved, otherwise the batch interpreter will mess up the arguments. Batch scripts are // weird. if (isWindows()) { - List winCmd = new ArrayList(); + List winCmd = new ArrayList<>(); for (String arg : cmd) { winCmd.add(quoteForBatchScript(arg)); } @@ -445,9 +578,42 @@ private ProcessBuilder createBuilder() { for (Map.Entry e : builder.childEnv.entrySet()) { pb.environment().put(e.getKey(), e.getValue()); } + + if (workingDir != null) { + pb.directory(workingDir); + } + + // Only one of redirectError and redirectError(...) can be specified. + // Similarly, if redirectToLog is specified, no other redirections should be specified. + checkState(!redirectErrorStream || errorStream == null, + "Cannot specify both redirectError() and redirectError(...) "); + checkState(getLoggerName() == null || + ((!redirectErrorStream && errorStream == null) || outputStream == null), + "Cannot used redirectToLog() in conjunction with other redirection methods."); + + if (redirectErrorStream) { + pb.redirectErrorStream(true); + } + if (errorStream != null) { + pb.redirectError(errorStream); + } + if (outputStream != null) { + pb.redirectOutput(outputStream); + } + return pb; } + // Visible for testing. + String findSparkSubmit() { + String script = isWindows() ? "spark-submit.cmd" : "spark-submit"; + return join(File.separator, builder.getSparkHome(), "bin", script); + } + + private String getLoggerName() throws IOException { + return builder.getEffectiveConfig().get(CHILD_PROCESS_LOGGER_NAME); + } + private static class ArgumentValidator extends SparkSubmitOptionParser { private final boolean hasValue; @@ -474,6 +640,6 @@ protected void handleExtraArgs(List extra) { // No op. } - }; + } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index 39b46e0db8cc..5f2da036ff9f 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -30,7 +30,8 @@ * driver-side options and special parsing behavior needed for the special-casing certain internal * Spark applications. *

    - * This class has also some special features to aid launching pyspark. + * This class has also some special features to aid launching shells (pyspark and sparkR) and also + * examples. */ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { @@ -62,22 +63,34 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { */ static final String SPARKR_SHELL_RESOURCE = "sparkr-shell"; + /** + * Name of app resource used to identify examples. When running examples, args[0] should be + * this name. The app resource will identify the example class to run. + */ + static final String RUN_EXAMPLE = "run-example"; + + /** + * Prefix for example class names. + */ + static final String EXAMPLE_CLASS_PREFIX = "org.apache.spark.examples."; + /** * This map must match the class names for available special classes, since this modifies the way * command line parsing works. This maps the class name to the resource to use when calling * spark-submit. */ - private static final Map specialClasses = new HashMap(); + private static final Map specialClasses = new HashMap<>(); static { specialClasses.put("org.apache.spark.repl.Main", "spark-shell"); specialClasses.put("org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver", - "spark-internal"); + SparkLauncher.NO_RESOURCE); specialClasses.put("org.apache.spark.sql.hive.thriftserver.HiveThriftServer2", - "spark-internal"); + SparkLauncher.NO_RESOURCE); } final List sparkArgs; - private final boolean printHelp; + private final boolean isAppResourceReq; + private final boolean isExample; /** * Controls whether mixing spark-submit arguments with app arguments is allowed. This is needed @@ -87,35 +100,52 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { private boolean allowsMixedArguments; SparkSubmitCommandBuilder() { - this.sparkArgs = new ArrayList(); - this.printHelp = false; + this.sparkArgs = new ArrayList<>(); + this.isAppResourceReq = true; + this.isExample = false; } SparkSubmitCommandBuilder(List args) { - this.sparkArgs = new ArrayList(); + this.allowsMixedArguments = false; + this.sparkArgs = new ArrayList<>(); + boolean isExample = false; List submitArgs = args; - if (args.size() > 0 && args.get(0).equals(PYSPARK_SHELL)) { - this.allowsMixedArguments = true; - appResource = PYSPARK_SHELL_RESOURCE; - submitArgs = args.subList(1, args.size()); - } else if (args.size() > 0 && args.get(0).equals(SPARKR_SHELL)) { - this.allowsMixedArguments = true; - appResource = SPARKR_SHELL_RESOURCE; - submitArgs = args.subList(1, args.size()); - } else { - this.allowsMixedArguments = false; - } - OptionParser parser = new OptionParser(); - parser.parse(submitArgs); - this.printHelp = parser.helpRequested; + if (args.size() > 0) { + switch (args.get(0)) { + case PYSPARK_SHELL: + this.allowsMixedArguments = true; + appResource = PYSPARK_SHELL; + submitArgs = args.subList(1, args.size()); + break; + + case SPARKR_SHELL: + this.allowsMixedArguments = true; + appResource = SPARKR_SHELL; + submitArgs = args.subList(1, args.size()); + break; + + case RUN_EXAMPLE: + isExample = true; + submitArgs = args.subList(1, args.size()); + } + + this.isExample = isExample; + OptionParser parser = new OptionParser(); + parser.parse(submitArgs); + this.isAppResourceReq = parser.isAppResourceReq; + } else { + this.isExample = isExample; + this.isAppResourceReq = false; + } } @Override - public List buildCommand(Map env) throws IOException { - if (PYSPARK_SHELL_RESOURCE.equals(appResource) && !printHelp) { + public List buildCommand(Map env) + throws IOException, IllegalArgumentException { + if (PYSPARK_SHELL.equals(appResource) && isAppResourceReq) { return buildPySparkShellCommand(env); - } else if (SPARKR_SHELL_RESOURCE.equals(appResource) && !printHelp) { + } else if (SPARKR_SHELL.equals(appResource) && isAppResourceReq) { return buildSparkRCommand(env); } else { return buildSparkSubmitCommand(env); @@ -123,9 +153,13 @@ public List buildCommand(Map env) throws IOException { } List buildSparkSubmitArgs() { - List args = new ArrayList(); + List args = new ArrayList<>(); SparkSubmitOptionParser parser = new SparkSubmitOptionParser(); + if (!allowsMixedArguments && isAppResourceReq) { + checkArgument(appResource != null, "Missing application resource."); + } + if (verbose) { args.add(parser.VERBOSE); } @@ -155,6 +189,10 @@ List buildSparkSubmitArgs() { args.add(propertiesFile); } + if (isExample) { + jars.addAll(findExamplesJars()); + } + if (!jars.isEmpty()) { args.add(parser.JARS); args.add(join(",", jars)); @@ -170,6 +208,9 @@ List buildSparkSubmitArgs() { args.add(join(",", pyFiles)); } + if (isAppResourceReq) { + checkArgument(!isExample || mainClass != null, "Missing example class name."); + } if (mainClass != null) { args.add(parser.CLASS); args.add(mainClass); @@ -184,7 +225,8 @@ List buildSparkSubmitArgs() { return args; } - private List buildSparkSubmitCommand(Map env) throws IOException { + private List buildSparkSubmitCommand(Map env) + throws IOException, IllegalArgumentException { // Load the properties file and check whether spark-submit will be running the app's driver // or just launching a cluster app. When running the driver, the JVM's argument will be // modified to cover the driver's configuration. @@ -198,7 +240,16 @@ private List buildSparkSubmitCommand(Map env) throws IOE addOptionString(cmd, System.getenv("SPARK_DAEMON_JAVA_OPTS")); } addOptionString(cmd, System.getenv("SPARK_SUBMIT_OPTS")); - addOptionString(cmd, System.getenv("SPARK_JAVA_OPTS")); + + // We don't want the client to specify Xmx. These have to be set by their corresponding + // memory flag --driver-memory or configuration entry spark.driver.memory + String driverExtraJavaOptions = config.get(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS); + if (!isEmpty(driverExtraJavaOptions) && driverExtraJavaOptions.contains("Xmx")) { + String msg = String.format("Not allowed to specify max heap(Xmx) memory settings through " + + "java options (was %s). Use the corresponding --driver-memory or " + + "spark.driver.memory configuration instead.", driverExtraJavaOptions); + throw new IllegalArgumentException(msg); + } if (isClientMode) { // Figuring out where the memory value come from is a little tricky due to precedence. @@ -213,14 +264,12 @@ private List buildSparkSubmitCommand(Map env) throws IOE isThriftServer(mainClass) ? System.getenv("SPARK_DAEMON_MEMORY") : null; String memory = firstNonEmpty(tsMemory, config.get(SparkLauncher.DRIVER_MEMORY), System.getenv("SPARK_DRIVER_MEMORY"), System.getenv("SPARK_MEM"), DEFAULT_MEM); - cmd.add("-Xms" + memory); cmd.add("-Xmx" + memory); - addOptionString(cmd, config.get(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS)); + addOptionString(cmd, driverExtraJavaOptions); mergeEnvPathList(env, getLibPathEnvName(), config.get(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH)); } - addPermGenSizeOpt(cmd); cmd.add("org.apache.spark.deploy.SparkSubmit"); cmd.addAll(buildSparkSubmitArgs()); return cmd; @@ -231,24 +280,35 @@ private List buildPySparkShellCommand(Map env) throws IO // the pyspark command line, then run it using spark-submit. if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".py")) { System.err.println( - "WARNING: Running python applications through 'pyspark' is deprecated as of Spark 1.0.\n" + + "Running python applications through 'pyspark' is not supported as of Spark 2.0.\n" + "Use ./bin/spark-submit "); - appResource = appArgs.get(0); - appArgs.remove(0); - return buildCommand(env); + System.exit(-1); } checkArgument(appArgs.isEmpty(), "pyspark does not support any application options."); // When launching the pyspark shell, the spark-submit arguments should be stored in the // PYSPARK_SUBMIT_ARGS env variable. + appResource = PYSPARK_SHELL_RESOURCE; constructEnvVarArgs(env, "PYSPARK_SUBMIT_ARGS"); - // The executable is the PYSPARK_DRIVER_PYTHON env variable set by the pyspark script, - // followed by PYSPARK_DRIVER_PYTHON_OPTS. - List pyargs = new ArrayList(); - pyargs.add(firstNonEmpty(System.getenv("PYSPARK_DRIVER_PYTHON"), "python")); + // Will pick up the binary executable in the following order + // 1. conf spark.pyspark.driver.python + // 2. conf spark.pyspark.python + // 3. environment variable PYSPARK_DRIVER_PYTHON + // 4. environment variable PYSPARK_PYTHON + // 5. python + List pyargs = new ArrayList<>(); + pyargs.add(firstNonEmpty(conf.get(SparkLauncher.PYSPARK_DRIVER_PYTHON), + conf.get(SparkLauncher.PYSPARK_PYTHON), + System.getenv("PYSPARK_DRIVER_PYTHON"), + System.getenv("PYSPARK_PYTHON"), + "python")); String pyOpts = System.getenv("PYSPARK_DRIVER_PYTHON_OPTS"); + if (conf.containsKey(SparkLauncher.PYSPARK_PYTHON)) { + // pass conf spark.pyspark.python to python by environment variable. + env.put("PYSPARK_PYTHON", conf.get(SparkLauncher.PYSPARK_PYTHON)); + } if (!isEmpty(pyOpts)) { pyargs.addAll(parseOptionString(pyOpts)); } @@ -258,12 +318,14 @@ private List buildPySparkShellCommand(Map env) throws IO private List buildSparkRCommand(Map env) throws IOException { if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".R")) { - appResource = appArgs.get(0); - appArgs.remove(0); - return buildCommand(env); + System.err.println( + "Running R applications through 'sparkR' is not supported as of Spark 2.0.\n" + + "Use ./bin/spark-submit "); + System.exit(-1); } // When launching the SparkR shell, store the spark-submit arguments in the SPARKR_SUBMIT_ARGS // env variable. + appResource = SPARKR_SHELL_RESOURCE; constructEnvVarArgs(env, "SPARKR_SUBMIT_ARGS"); // Set shell.R as R_PROFILE_USER to load the SparkR package when the shell comes up. @@ -271,8 +333,9 @@ private List buildSparkRCommand(Map env) throws IOExcept env.put("R_PROFILE_USER", join(File.separator, sparkHome, "R", "lib", "SparkR", "profile", "shell.R")); - List args = new ArrayList(); - args.add(firstNonEmpty(System.getenv("SPARKR_DRIVER_R"), "R")); + List args = new ArrayList<>(); + args.add(firstNonEmpty(conf.get(SparkLauncher.SPARKR_R_SHELL), + System.getenv("SPARKR_DRIVER_R"), "R")); return args; } @@ -294,10 +357,11 @@ private void constructEnvVarArgs( private boolean isClientMode(Map userProps) { String userMaster = firstNonEmpty(master, userProps.get(SparkLauncher.SPARK_MASTER)); - // Default master is "local[*]", so assume client mode in that case. + String userDeployMode = firstNonEmpty(deployMode, userProps.get(SparkLauncher.DEPLOY_MODE)); + // Default master is "local[*]", so assume client mode in that case return userMaster == null || - "client".equals(deployMode) || - (!userMaster.equals("yarn-cluster") && deployMode == null); + "client".equals(userDeployMode) || + (!userMaster.equals("yarn-cluster") && userDeployMode == null); } /** @@ -308,49 +372,96 @@ private boolean isThriftServer(String mainClass) { mainClass.equals("org.apache.spark.sql.hive.thriftserver.HiveThriftServer2")); } + private List findExamplesJars() { + boolean isTesting = "1".equals(getenv("SPARK_TESTING")); + List examplesJars = new ArrayList<>(); + String sparkHome = getSparkHome(); + + File jarsDir; + if (new File(sparkHome, "RELEASE").isFile()) { + jarsDir = new File(sparkHome, "examples/jars"); + } else { + jarsDir = new File(sparkHome, + String.format("examples/target/scala-%s/jars", getScalaVersion())); + } + + boolean foundDir = jarsDir.isDirectory(); + checkState(isTesting || foundDir, "Examples jars directory '%s' does not exist.", + jarsDir.getAbsolutePath()); + + if (foundDir) { + for (File f: jarsDir.listFiles()) { + examplesJars.add(f.getAbsolutePath()); + } + } + return examplesJars; + } private class OptionParser extends SparkSubmitOptionParser { - boolean helpRequested = false; + boolean isAppResourceReq = true; @Override protected boolean handle(String opt, String value) { - if (opt.equals(MASTER)) { - master = value; - } else if (opt.equals(DEPLOY_MODE)) { - deployMode = value; - } else if (opt.equals(PROPERTIES_FILE)) { - propertiesFile = value; - } else if (opt.equals(DRIVER_MEMORY)) { - conf.put(SparkLauncher.DRIVER_MEMORY, value); - } else if (opt.equals(DRIVER_JAVA_OPTIONS)) { - conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, value); - } else if (opt.equals(DRIVER_LIBRARY_PATH)) { - conf.put(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, value); - } else if (opt.equals(DRIVER_CLASS_PATH)) { - conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, value); - } else if (opt.equals(CONF)) { - String[] setConf = value.split("=", 2); - checkArgument(setConf.length == 2, "Invalid argument to %s: %s", CONF, value); - conf.put(setConf[0], setConf[1]); - } else if (opt.equals(CLASS)) { - // The special classes require some special command line handling, since they allow - // mixing spark-submit arguments with arguments that should be propagated to the shell - // itself. Note that for this to work, the "--class" argument must come before any - // non-spark-submit arguments. - mainClass = value; - if (specialClasses.containsKey(value)) { - allowsMixedArguments = true; - appResource = specialClasses.get(value); - } - } else if (opt.equals(HELP) || opt.equals(USAGE_ERROR)) { - helpRequested = true; - sparkArgs.add(opt); - } else { - sparkArgs.add(opt); - if (value != null) { + switch (opt) { + case MASTER: + master = value; + break; + case DEPLOY_MODE: + deployMode = value; + break; + case PROPERTIES_FILE: + propertiesFile = value; + break; + case DRIVER_MEMORY: + conf.put(SparkLauncher.DRIVER_MEMORY, value); + break; + case DRIVER_JAVA_OPTIONS: + conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, value); + break; + case DRIVER_LIBRARY_PATH: + conf.put(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, value); + break; + case DRIVER_CLASS_PATH: + conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, value); + break; + case CONF: + String[] setConf = value.split("=", 2); + checkArgument(setConf.length == 2, "Invalid argument to %s: %s", CONF, value); + conf.put(setConf[0], setConf[1]); + break; + case CLASS: + // The special classes require some special command line handling, since they allow + // mixing spark-submit arguments with arguments that should be propagated to the shell + // itself. Note that for this to work, the "--class" argument must come before any + // non-spark-submit arguments. + mainClass = value; + if (specialClasses.containsKey(value)) { + allowsMixedArguments = true; + appResource = specialClasses.get(value); + } + break; + case KILL_SUBMISSION: + case STATUS: + isAppResourceReq = false; + sparkArgs.add(opt); sparkArgs.add(value); - } + break; + case HELP: + case USAGE_ERROR: + isAppResourceReq = false; + sparkArgs.add(opt); + break; + case VERSION: + isAppResourceReq = false; + sparkArgs.add(opt); + break; + default: + sparkArgs.add(opt); + if (value != null) { + sparkArgs.add(value); + } + break; } return true; } @@ -364,18 +475,25 @@ protected boolean handleUnknown(String opt) { if (allowsMixedArguments) { appArgs.add(opt); return true; + } else if (isExample) { + String className = opt; + if (!className.startsWith(EXAMPLE_CLASS_PREFIX)) { + className = EXAMPLE_CLASS_PREFIX + className; + } + mainClass = className; + appResource = SparkLauncher.NO_RESOURCE; + return false; } else { checkArgument(!opt.startsWith("-"), "Unrecognized option: %s", opt); - sparkArgs.add(opt); + checkState(appResource == null, "Found unrecognized argument but resource is already set."); + appResource = opt; return false; } } @Override protected void handleExtraArgs(List extra) { - for (String arg : extra) { - sparkArgs.add(arg); - } + appArgs.addAll(extra); } } diff --git a/launcher/src/test/java/org/apache/spark/launcher/ChildProcAppHandleSuite.java b/launcher/src/test/java/org/apache/spark/launcher/ChildProcAppHandleSuite.java new file mode 100644 index 000000000000..9f59b41d52d4 --- /dev/null +++ b/launcher/src/test/java/org/apache/spark/launcher/ChildProcAppHandleSuite.java @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.launcher; + +import java.io.File; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import static java.nio.file.attribute.PosixFilePermission.*; + +import org.apache.log4j.AppenderSkeleton; +import org.apache.log4j.spi.LoggingEvent; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import static org.junit.Assert.*; +import static org.junit.Assume.*; + +import static org.apache.spark.launcher.CommandBuilderUtils.*; + +public class ChildProcAppHandleSuite extends BaseSuite { + + private static final List MESSAGES = new ArrayList<>(); + + private static final List TEST_SCRIPT = Arrays.asList( + "#!/bin/sh", + "echo \"output\"", + "echo \"error\" 1>&2", + "while [ -n \"$1\" ]; do EC=$1; shift; done", + "exit $EC"); + + private static File TEST_SCRIPT_PATH; + + @AfterClass + public static void cleanupClass() throws Exception { + if (TEST_SCRIPT_PATH != null) { + TEST_SCRIPT_PATH.delete(); + TEST_SCRIPT_PATH = null; + } + } + + @BeforeClass + public static void setupClass() throws Exception { + TEST_SCRIPT_PATH = File.createTempFile("output-redir-test", ".sh"); + Files.setPosixFilePermissions(TEST_SCRIPT_PATH.toPath(), + EnumSet.of(OWNER_READ, OWNER_EXECUTE, OWNER_WRITE)); + Files.write(TEST_SCRIPT_PATH.toPath(), TEST_SCRIPT); + } + + @Before + public void cleanupLog() { + MESSAGES.clear(); + } + + @Test + public void testRedirectsSimple() throws Exception { + SparkLauncher launcher = new SparkLauncher(); + launcher.redirectError(ProcessBuilder.Redirect.PIPE); + assertNotNull(launcher.errorStream); + assertEquals(launcher.errorStream.type(), ProcessBuilder.Redirect.Type.PIPE); + + launcher.redirectOutput(ProcessBuilder.Redirect.PIPE); + assertNotNull(launcher.outputStream); + assertEquals(launcher.outputStream.type(), ProcessBuilder.Redirect.Type.PIPE); + } + + @Test + public void testRedirectLastWins() throws Exception { + SparkLauncher launcher = new SparkLauncher(); + launcher.redirectError(ProcessBuilder.Redirect.PIPE) + .redirectError(ProcessBuilder.Redirect.INHERIT); + assertEquals(launcher.errorStream.type(), ProcessBuilder.Redirect.Type.INHERIT); + + launcher.redirectOutput(ProcessBuilder.Redirect.PIPE) + .redirectOutput(ProcessBuilder.Redirect.INHERIT); + assertEquals(launcher.outputStream.type(), ProcessBuilder.Redirect.Type.INHERIT); + } + + @Test + public void testRedirectToLog() throws Exception { + assumeFalse(isWindows()); + + SparkAppHandle handle = (ChildProcAppHandle) new TestSparkLauncher() + .startApplication(); + waitFor(handle); + + assertTrue(MESSAGES.contains("output")); + assertTrue(MESSAGES.contains("error")); + } + + @Test + public void testRedirectErrorToLog() throws Exception { + assumeFalse(isWindows()); + + Path err = Files.createTempFile("stderr", "txt"); + err.toFile().deleteOnExit(); + + SparkAppHandle handle = (ChildProcAppHandle) new TestSparkLauncher() + .redirectError(err.toFile()) + .startApplication(); + waitFor(handle); + + assertTrue(MESSAGES.contains("output")); + assertEquals(Arrays.asList("error"), Files.lines(err).collect(Collectors.toList())); + } + + @Test + public void testRedirectOutputToLog() throws Exception { + assumeFalse(isWindows()); + + Path out = Files.createTempFile("stdout", "txt"); + out.toFile().deleteOnExit(); + + SparkAppHandle handle = (ChildProcAppHandle) new TestSparkLauncher() + .redirectOutput(out.toFile()) + .startApplication(); + waitFor(handle); + + assertTrue(MESSAGES.contains("error")); + assertEquals(Arrays.asList("output"), Files.lines(out).collect(Collectors.toList())); + } + + @Test + public void testNoRedirectToLog() throws Exception { + assumeFalse(isWindows()); + + Path out = Files.createTempFile("stdout", "txt"); + Path err = Files.createTempFile("stderr", "txt"); + out.toFile().deleteOnExit(); + err.toFile().deleteOnExit(); + + ChildProcAppHandle handle = (ChildProcAppHandle) new TestSparkLauncher() + .redirectError(err.toFile()) + .redirectOutput(out.toFile()) + .startApplication(); + waitFor(handle); + + assertTrue(MESSAGES.isEmpty()); + assertEquals(Arrays.asList("error"), Files.lines(err).collect(Collectors.toList())); + assertEquals(Arrays.asList("output"), Files.lines(out).collect(Collectors.toList())); + } + + @Test(expected = IllegalArgumentException.class) + public void testBadLogRedirect() throws Exception { + File out = Files.createTempFile("stdout", "txt").toFile(); + out.deleteOnExit(); + new SparkLauncher() + .redirectError() + .redirectOutput(out) + .redirectToLog("foo") + .launch() + .waitFor(); + } + + @Test(expected = IllegalArgumentException.class) + public void testRedirectErrorTwiceFails() throws Exception { + File err = Files.createTempFile("stderr", "txt").toFile(); + err.deleteOnExit(); + new SparkLauncher() + .redirectError() + .redirectError(err) + .launch() + .waitFor(); + } + + @Test + public void testProcMonitorWithOutputRedirection() throws Exception { + assumeFalse(isWindows()); + File err = Files.createTempFile("out", "txt").toFile(); + err.deleteOnExit(); + SparkAppHandle handle = new TestSparkLauncher() + .redirectError() + .redirectOutput(err) + .startApplication(); + waitFor(handle); + assertEquals(SparkAppHandle.State.LOST, handle.getState()); + } + + @Test + public void testProcMonitorWithLogRedirection() throws Exception { + assumeFalse(isWindows()); + SparkAppHandle handle = new TestSparkLauncher() + .redirectToLog(getClass().getName()) + .startApplication(); + waitFor(handle); + assertEquals(SparkAppHandle.State.LOST, handle.getState()); + } + + @Test + public void testFailedChildProc() throws Exception { + assumeFalse(isWindows()); + SparkAppHandle handle = new TestSparkLauncher(1) + .redirectToLog(getClass().getName()) + .startApplication(); + waitFor(handle); + assertEquals(SparkAppHandle.State.FAILED, handle.getState()); + } + + private void waitFor(SparkAppHandle handle) throws Exception { + long deadline = System.nanoTime() + TimeUnit.SECONDS.toNanos(10); + try { + while (!handle.getState().isFinal()) { + assertTrue("Timed out waiting for handle to transition to final state.", + System.nanoTime() < deadline); + TimeUnit.MILLISECONDS.sleep(10); + } + } finally { + if (!handle.getState().isFinal()) { + handle.kill(); + } + } + } + + private static class TestSparkLauncher extends SparkLauncher { + + TestSparkLauncher() { + this(0); + } + + TestSparkLauncher(int ec) { + setAppResource("outputredirtest"); + addAppArgs(String.valueOf(ec)); + } + + @Override + String findSparkSubmit() { + return TEST_SCRIPT_PATH.getAbsolutePath(); + } + + } + + /** + * A log4j appender used by child apps of this test. It records all messages logged through it in + * memory so the test can check them. + */ + public static class LogAppender extends AppenderSkeleton { + + @Override + protected void append(LoggingEvent event) { + MESSAGES.add(event.getMessage().toString()); + } + + @Override + public boolean requiresLayout() { + return false; + } + + @Override + public void close() { + + } + + } +} diff --git a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java index bc513ec9b3d1..9795041233b6 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java @@ -87,12 +87,24 @@ public void testPythonArgQuoting() { assertEquals("\"a \\\"b\\\" c\"", quoteForCommandString("a \"b\" c")); } - private void testOpt(String opts, List expected) { + @Test + public void testJavaMajorVersion() { + assertEquals(6, javaMajorVersion("1.6.0_50")); + assertEquals(7, javaMajorVersion("1.7.0_79")); + assertEquals(8, javaMajorVersion("1.8.0_66")); + assertEquals(9, javaMajorVersion("9-ea")); + assertEquals(9, javaMajorVersion("9+100")); + assertEquals(9, javaMajorVersion("9")); + assertEquals(9, javaMajorVersion("9.1.0")); + assertEquals(10, javaMajorVersion("10")); + } + + private static void testOpt(String opts, List expected) { assertEquals(String.format("test string failed to parse: [[ %s ]]", opts), expected, parseOptionString(opts)); } - private void testInvalidOpt(String opts) { + private static void testInvalidOpt(String opts) { try { parseOptionString(opts); fail("Expected exception for invalid option string."); diff --git a/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java b/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java index dc8fbb58d880..d0f6abe2fe69 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/LauncherServerSuite.java @@ -19,15 +19,19 @@ import java.io.Closeable; import java.io.IOException; +import java.io.ObjectInputStream; import java.net.InetAddress; import java.net.Socket; +import java.net.SocketException; +import java.util.Arrays; +import java.util.List; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import org.junit.Test; import static org.junit.Assert.*; -import static org.mockito.Mockito.*; import static org.apache.spark.launcher.LauncherProtocol.*; @@ -69,48 +73,35 @@ public void testCommunication() throws Exception { Socket s = new Socket(InetAddress.getLoopbackAddress(), LauncherServer.getServerInstance().getPort()); - final Object waitLock = new Object(); + final Semaphore semaphore = new Semaphore(0); handle.addListener(new SparkAppHandle.Listener() { @Override public void stateChanged(SparkAppHandle handle) { - wakeUp(); + semaphore.release(); } - @Override public void infoChanged(SparkAppHandle handle) { - wakeUp(); - } - - private void wakeUp() { - synchronized (waitLock) { - waitLock.notifyAll(); - } + semaphore.release(); } }); client = new TestClient(s); - synchronized (waitLock) { - client.send(new Hello(handle.getSecret(), "1.4.0")); - waitLock.wait(TimeUnit.SECONDS.toMillis(10)); - } + client.send(new Hello(handle.getSecret(), "1.4.0")); + assertTrue(semaphore.tryAcquire(30, TimeUnit.SECONDS)); // Make sure the server matched the client to the handle. assertNotNull(handle.getConnection()); - synchronized (waitLock) { - client.send(new SetAppId("app-id")); - waitLock.wait(TimeUnit.SECONDS.toMillis(10)); - } + client.send(new SetAppId("app-id")); + assertTrue(semaphore.tryAcquire(30, TimeUnit.SECONDS)); assertEquals("app-id", handle.getAppId()); - synchronized (waitLock) { - client.send(new SetState(SparkAppHandle.State.RUNNING)); - waitLock.wait(TimeUnit.SECONDS.toMillis(10)); - } + client.send(new SetState(SparkAppHandle.State.RUNNING)); + assertTrue(semaphore.tryAcquire(1, TimeUnit.SECONDS)); assertEquals(SparkAppHandle.State.RUNNING, handle.getState()); handle.stop(); - Message stopMsg = client.inbound.poll(10, TimeUnit.SECONDS); + Message stopMsg = client.inbound.poll(30, TimeUnit.SECONDS); assertTrue(stopMsg instanceof Stop); } finally { kill(handle); @@ -133,35 +124,69 @@ public void testTimeout() throws Exception { Socket s = new Socket(InetAddress.getLoopbackAddress(), LauncherServer.getServerInstance().getPort()); client = new TestClient(s); + waitForError(client, handle.getSecret()); + } finally { + SparkLauncher.launcherConfig.remove(SparkLauncher.CHILD_CONNECTION_TIMEOUT); + kill(handle); + close(client); + } + } - // Try a few times since the client-side socket may not reflect the server-side close - // immediately. - boolean helloSent = false; - int maxTries = 10; - for (int i = 0; i < maxTries; i++) { - try { - if (!helloSent) { - client.send(new Hello(handle.getSecret(), "1.4.0")); - helloSent = true; - } else { - client.send(new SetAppId("appId")); - } - fail("Expected exception caused by connection timeout."); - } catch (IllegalStateException | IOException e) { - // Expected. - break; - } catch (AssertionError e) { - if (i < maxTries - 1) { - Thread.sleep(100); - } else { - throw new AssertionError("Test failed after " + maxTries + " attempts.", e); - } + @Test + public void testSparkSubmitVmShutsDown() throws Exception { + ChildProcAppHandle handle = LauncherServer.newAppHandle(); + TestClient client = null; + final Semaphore semaphore = new Semaphore(0); + try { + Socket s = new Socket(InetAddress.getLoopbackAddress(), + LauncherServer.getServerInstance().getPort()); + handle.addListener(new SparkAppHandle.Listener() { + public void stateChanged(SparkAppHandle handle) { + semaphore.release(); } + public void infoChanged(SparkAppHandle handle) { + semaphore.release(); + } + }); + client = new TestClient(s); + client.send(new Hello(handle.getSecret(), "1.4.0")); + assertTrue(semaphore.tryAcquire(30, TimeUnit.SECONDS)); + // Make sure the server matched the client to the handle. + assertNotNull(handle.getConnection()); + close(client); + assertTrue(semaphore.tryAcquire(30, TimeUnit.SECONDS)); + assertEquals(SparkAppHandle.State.LOST, handle.getState()); + } finally { + kill(handle); + close(client); + client.clientThread.join(); + } + } + + @Test + public void testStreamFiltering() throws Exception { + ChildProcAppHandle handle = LauncherServer.newAppHandle(); + TestClient client = null; + try { + Socket s = new Socket(InetAddress.getLoopbackAddress(), + LauncherServer.getServerInstance().getPort()); + + client = new TestClient(s); + + try { + client.send(new EvilPayload()); + } catch (SocketException se) { + // SPARK-21522: this can happen if the server closes the socket before the full message has + // been written, so it's expected. It may cause false positives though (socket errors + // happening for other reasons). } + + waitForError(client, handle.getSecret()); + assertEquals(0, EvilPayload.EVIL_BIT); } finally { - SparkLauncher.launcherConfig.remove(SparkLauncher.CHILD_CONNECTION_TIMEOUT); kill(handle); close(client); + client.clientThread.join(); } } @@ -181,6 +206,35 @@ private void close(Closeable c) { } } + /** + * Try a few times to get a client-side error, since the client-side socket may not reflect the + * server-side close immediately. + */ + private void waitForError(TestClient client, String secret) throws Exception { + boolean helloSent = false; + int maxTries = 10; + for (int i = 0; i < maxTries; i++) { + try { + if (!helloSent) { + client.send(new Hello(secret, "1.4.0")); + helloSent = true; + } else { + client.send(new SetAppId("appId")); + } + fail("Expected error but message went through."); + } catch (IllegalStateException | IOException e) { + // Expected. + break; + } catch (AssertionError e) { + if (i < maxTries - 1) { + Thread.sleep(100); + } else { + throw new AssertionError("Test failed after " + maxTries + " attempts.", e); + } + } + } + } + private static class TestClient extends LauncherConnection { final BlockingQueue inbound; @@ -188,7 +242,7 @@ private static class TestClient extends LauncherConnection { TestClient(Socket s) throws IOException { super(s); - this.inbound = new LinkedBlockingQueue(); + this.inbound = new LinkedBlockingQueue<>(); this.clientThread = new Thread(this); clientThread.setName("TestClient"); clientThread.setDaemon(true); @@ -202,4 +256,19 @@ protected void handle(Message msg) throws IOException { } + private static class EvilPayload extends LauncherProtocol.Message { + + static int EVIL_BIT = 0; + + // This field should cause the launcher server to throw an error and not deserialize the + // message. + private List notAllowedField = Arrays.asList("disallowed"); + + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + EVIL_BIT = 1; + } + + } + } diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index 6aad47adbcc8..2e050f841307 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -58,6 +58,26 @@ public void testClusterCmdBuilder() throws Exception { testCmdBuilder(false, false); } + @Test + public void testCliHelpAndNoArg() throws Exception { + List helpArgs = Arrays.asList(parser.HELP); + Map env = new HashMap<>(); + List cmd = buildCommand(helpArgs, env); + assertTrue("--help should be contained in the final cmd.", cmd.contains(parser.HELP)); + + List sparkEmptyArgs = Collections.emptyList(); + cmd = buildCommand(sparkEmptyArgs, env); + assertTrue( + "org.apache.spark.deploy.SparkSubmit should be contained in the final cmd of empty input.", + cmd.contains("org.apache.spark.deploy.SparkSubmit")); + } + + @Test + public void testCliKillAndStatus() throws Exception { + testCLIOpts(parser.STATUS); + testCLIOpts(parser.KILL_SUBMISSION); + } + @Test public void testCliParser() throws Exception { List sparkSubmitArgs = Arrays.asList( @@ -72,14 +92,14 @@ public void testCliParser() throws Exception { parser.CONF, "spark.randomOption=foo", parser.CONF, - SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH + "=/driverLibPath"); - Map env = new HashMap(); + SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH + "=/driverLibPath", + SparkLauncher.NO_RESOURCE); + Map env = new HashMap<>(); List cmd = buildCommand(sparkSubmitArgs, env); assertTrue(findInStringList(env.get(CommandBuilderUtils.getLibPathEnvName()), File.pathSeparator, "/driverLibPath")); assertTrue(findInStringList(findArgValue(cmd, "-cp"), File.pathSeparator, "/driverCp")); - assertTrue("Driver -Xms should be configured.", cmd.contains("-Xms42g")); assertTrue("Driver -Xmx should be configured.", cmd.contains("-Xmx42g")); assertTrue("Command should contain user-defined conf.", Collections.indexOfSubList(cmd, Arrays.asList(parser.CONF, "spark.randomOption=foo")) > 0); @@ -110,7 +130,8 @@ public void testAlternateSyntaxParsing() throws Exception { List sparkSubmitArgs = Arrays.asList( parser.CLASS + "=org.my.Class", parser.MASTER + "=foo", - parser.DEPLOY_MODE + "=bar"); + parser.DEPLOY_MODE + "=bar", + SparkLauncher.NO_RESOURCE); List cmd = newCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); assertEquals("org.my.Class", findArgValue(cmd, parser.CLASS)); @@ -125,7 +146,7 @@ public void testPySparkLauncher() throws Exception { "--master=foo", "--deploy-mode=bar"); - Map env = new HashMap(); + Map env = new HashMap<>(); List cmd = buildCommand(sparkSubmitArgs, env); assertEquals("python", cmd.get(cmd.size() - 1)); assertEquals( @@ -142,7 +163,7 @@ public void testPySparkFallback() throws Exception { "script.py", "arg1"); - Map env = new HashMap(); + Map env = new HashMap<>(); List cmd = buildCommand(sparkSubmitArgs, env); assertEquals("foo", findArgValue(cmd, "--master")); @@ -151,11 +172,52 @@ public void testPySparkFallback() throws Exception { assertEquals("arg1", cmd.get(cmd.size() - 1)); } + @Test + public void testSparkRShell() throws Exception { + List sparkSubmitArgs = Arrays.asList( + SparkSubmitCommandBuilder.SPARKR_SHELL, + "--master=foo", + "--deploy-mode=bar", + "--conf", "spark.r.shell.command=/usr/bin/R"); + + Map env = new HashMap<>(); + List cmd = buildCommand(sparkSubmitArgs, env); + assertEquals("/usr/bin/R", cmd.get(cmd.size() - 1)); + assertEquals( + String.format( + "\"%s\" \"foo\" \"%s\" \"bar\" \"--conf\" \"spark.r.shell.command=/usr/bin/R\" \"%s\"", + parser.MASTER, parser.DEPLOY_MODE, SparkSubmitCommandBuilder.SPARKR_SHELL_RESOURCE), + env.get("SPARKR_SUBMIT_ARGS")); + } + + @Test + public void testExamplesRunner() throws Exception { + List sparkSubmitArgs = Arrays.asList( + SparkSubmitCommandBuilder.RUN_EXAMPLE, + parser.MASTER + "=foo", + parser.DEPLOY_MODE + "=bar", + "SparkPi", + "42"); + + Map env = new HashMap<>(); + List cmd = buildCommand(sparkSubmitArgs, env); + assertEquals("foo", findArgValue(cmd, parser.MASTER)); + assertEquals("bar", findArgValue(cmd, parser.DEPLOY_MODE)); + assertEquals(SparkSubmitCommandBuilder.EXAMPLE_CLASS_PREFIX + "SparkPi", + findArgValue(cmd, parser.CLASS)); + assertEquals("42", cmd.get(cmd.size() - 1)); + } + + @Test(expected = IllegalArgumentException.class) + public void testMissingAppResource() { + new SparkSubmitCommandBuilder().buildSparkSubmitArgs(); + } + private void testCmdBuilder(boolean isDriver, boolean useDefaultPropertyFile) throws Exception { String deployMode = isDriver ? "client" : "cluster"; SparkSubmitCommandBuilder launcher = - newCommandBuilder(Collections.emptyList()); + newCommandBuilder(Collections.emptyList()); launcher.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, System.getProperty("spark.test.home")); launcher.master = "yarn"; @@ -171,25 +233,24 @@ private void testCmdBuilder(boolean isDriver, boolean useDefaultPropertyFile) th launcher.setPropertiesFile(dummyPropsFile.getAbsolutePath()); launcher.conf.put(SparkLauncher.DRIVER_MEMORY, "1g"); launcher.conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, "/driver"); - launcher.conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, "-Ddriver -XX:MaxPermSize=256m"); + launcher.conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, "-Ddriver"); launcher.conf.put(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, "/native"); } else { launcher.childEnv.put("SPARK_CONF_DIR", System.getProperty("spark.test.home") + "/launcher/src/test/resources"); } - Map env = new HashMap(); + Map env = new HashMap<>(); List cmd = launcher.buildCommand(env); // Checks below are different for driver and non-driver mode. if (isDriver) { - assertTrue("Driver -Xms should be configured.", cmd.contains("-Xms1g")); assertTrue("Driver -Xmx should be configured.", cmd.contains("-Xmx1g")); } else { boolean found = false; for (String arg : cmd) { - if (arg.startsWith("-Xms") || arg.startsWith("-Xmx")) { + if (arg.startsWith("-Xmx")) { found = true; break; } @@ -197,16 +258,6 @@ private void testCmdBuilder(boolean isDriver, boolean useDefaultPropertyFile) th assertFalse("Memory arguments should not be set.", found); } - for (String arg : cmd) { - if (arg.startsWith("-XX:MaxPermSize=")) { - if (isDriver) { - assertEquals("-XX:MaxPermSize=256m", arg); - } else { - assertEquals("-XX:MaxPermSize=256m", arg); - } - } - } - String[] cp = findArgValue(cmd, "-cp").split(Pattern.quote(File.pathSeparator)); if (isDriver) { assertTrue("Driver classpath should contain provided entry.", contains("/driver", cp)); @@ -258,7 +309,7 @@ private boolean contains(String needle, String[] haystack) { } private Map parseConf(List cmd, SparkSubmitOptionParser parser) { - Map conf = new HashMap(); + Map conf = new HashMap<>(); for (int i = 0; i < cmd.size(); i++) { if (cmd.get(i).equals(parser.CONF)) { String[] val = cmd.get(i + 1).split("=", 2); @@ -286,7 +337,6 @@ private boolean findInStringList(String list, String sep, String needle) { private SparkSubmitCommandBuilder newCommandBuilder(List args) { SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(args); builder.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, System.getProperty("spark.test.home")); - builder.childEnv.put(CommandBuilderUtils.ENV_SPARK_ASSEMBLY, "dummy"); return builder; } @@ -294,4 +344,12 @@ private List buildCommand(List args, Map env) th return newCommandBuilder(args).buildCommand(env); } + private void testCLIOpts(String opt) throws Exception { + List helpArgs = Arrays.asList(opt, "driver-20160531171222-0000"); + Map env = new HashMap<>(); + List cmd = buildCommand(helpArgs, env); + assertTrue(opt + " should be contained in the final cmd.", + cmd.contains(opt)); + } + } diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java index 3ee5b8cf9689..9ff7aceb581f 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java @@ -23,11 +23,8 @@ import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; import static org.mockito.Mockito.*; -import static org.apache.spark.launcher.SparkSubmitOptionParser.*; - public class SparkSubmitOptionParserSuite extends BaseSuite { private SparkSubmitOptionParser parser; @@ -47,7 +44,7 @@ public void testAllOptions() { count++; verify(parser).handle(eq(optNames[0]), eq(value)); verify(parser, times(count)).handle(anyString(), anyString()); - verify(parser, times(count)).handleExtraArgs(eq(Collections.emptyList())); + verify(parser, times(count)).handleExtraArgs(eq(Collections.emptyList())); } } @@ -57,9 +54,9 @@ public void testAllOptions() { parser.parse(Arrays.asList(name)); count++; switchCount++; - verify(parser, times(switchCount)).handle(eq(switchNames[0]), same((String) null)); + verify(parser, times(switchCount)).handle(eq(switchNames[0]), same(null)); verify(parser, times(count)).handle(anyString(), any(String.class)); - verify(parser, times(count)).handleExtraArgs(eq(Collections.emptyList())); + verify(parser, times(count)).handleExtraArgs(eq(Collections.emptyList())); } } } @@ -83,7 +80,7 @@ public void testEqualSeparatedOption() { List args = Arrays.asList(parser.MASTER + "=" + parser.MASTER); parser.parse(args); verify(parser).handle(eq(parser.MASTER), eq(parser.MASTER)); - verify(parser).handleExtraArgs(eq(Collections.emptyList())); + verify(parser).handleExtraArgs(eq(Collections.emptyList())); } private static class DummyParser extends SparkSubmitOptionParser { diff --git a/launcher/src/test/resources/log4j.properties b/launcher/src/test/resources/log4j.properties index c64b1565e146..366a6b9f817c 100644 --- a/launcher/src/test/resources/log4j.properties +++ b/launcher/src/test/resources/log4j.properties @@ -29,6 +29,9 @@ log4j.appender.childproc.target=System.err log4j.appender.childproc.layout=org.apache.log4j.PatternLayout log4j.appender.childproc.layout.ConversionPattern=%t: %m%n +log4j.appender.outputredirtest=org.apache.spark.launcher.ChildProcAppHandleSuite$LogAppender +log4j.logger.org.apache.spark.launcher.app.outputredirtest=INFO, outputredirtest +log4j.logger.org.apache.spark.launcher.app.outputredirtest.additivity=false + # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN -org.spark-project.jetty.LEVEL=WARN +log4j.logger.org.spark_project.jetty=WARN diff --git a/launcher/src/test/resources/spark-defaults.conf b/launcher/src/test/resources/spark-defaults.conf index 239fc57883e9..3a51208c7c24 100644 --- a/launcher/src/test/resources/spark-defaults.conf +++ b/launcher/src/test/resources/spark-defaults.conf @@ -17,5 +17,5 @@ spark.driver.memory=1g spark.driver.extraClassPath=/driver -spark.driver.extraJavaOptions=-Ddriver -XX:MaxPermSize=256m +spark.driver.extraJavaOptions=-Ddriver spark.driver.extraLibraryPath=/native \ No newline at end of file diff --git a/licenses/LICENSE-jblas.txt b/licenses/LICENSE-jblas.txt deleted file mode 100644 index 5629dafb65b3..000000000000 --- a/licenses/LICENSE-jblas.txt +++ /dev/null @@ -1,31 +0,0 @@ -Copyright (c) 2009, Mikio L. Braun and contributors -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are -met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following - disclaimer in the documentation and/or other materials provided - with the distribution. - - * Neither the name of the Technische Universität Berlin nor the - names of its contributors may be used to endorse or promote - products derived from this software without specific prior - written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/licenses/LICENSE-modernizr.txt b/licenses/LICENSE-modernizr.txt new file mode 100644 index 000000000000..2bf24b9b9f84 --- /dev/null +++ b/licenses/LICENSE-modernizr.txt @@ -0,0 +1,21 @@ +The MIT License (MIT) + +Copyright (c) + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. \ No newline at end of file diff --git a/licenses/LICENSE-postgresql.txt b/licenses/LICENSE-postgresql.txt new file mode 100644 index 000000000000..515bf9af4d43 --- /dev/null +++ b/licenses/LICENSE-postgresql.txt @@ -0,0 +1,24 @@ +PostgreSQL Database Management System +(formerly known as Postgres, then as Postgres95) + +Portions Copyright (c) 1996-2010, PostgreSQL Global Development Group + +Portions Copyright (c) 1994, The Regents of the University of California + +Permission to use, copy, modify, and distribute this software and its +documentation for any purpose, without fee, and without a written agreement +is hereby granted, provided that the above copyright notice and this +paragraph and the following two paragraphs appear in all copies. + +IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR +DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING +LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS +DOCUMENTATION, EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. + +THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY +AND FITNESS FOR A PARTICULAR PURPOSE. THE SOFTWARE PROVIDED HEREUNDER IS +ON AN "AS IS" BASIS, AND THE UNIVERSITY OF CALIFORNIA HAS NO OBLIGATIONS TO +PROVIDE MAINTENANCE, SUPPORT, UPDATES, ENHANCEMENTS, OR MODIFICATIONS. + diff --git a/make-distribution.sh b/make-distribution.sh deleted file mode 100755 index e1c2afdbc6d8..000000000000 --- a/make-distribution.sh +++ /dev/null @@ -1,266 +0,0 @@ -#!/usr/bin/env bash - -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# -# Script to create a binary distribution for easy deploys of Spark. -# The distribution directory defaults to dist/ but can be overridden below. -# The distribution contains fat (assembly) jars that include the Scala library, -# so it is completely self contained. -# It does not contain source or *.class files. - -set -o pipefail -set -e -set -x - -# Figure out where the Spark framework is installed -SPARK_HOME="$(cd "`dirname "$0"`"; pwd)" -DISTDIR="$SPARK_HOME/dist" - -SPARK_TACHYON=false -TACHYON_VERSION="0.8.1" -TACHYON_TGZ="tachyon-${TACHYON_VERSION}-bin.tar.gz" -TACHYON_URL="http://tachyon-project.org/downloads/files/${TACHYON_VERSION}/${TACHYON_TGZ}" - -MAKE_TGZ=false -NAME=none -MVN="$SPARK_HOME/build/mvn" - -function exit_with_usage { - echo "make-distribution.sh - tool for making binary distributions of Spark" - echo "" - echo "usage:" - cl_options="[--name] [--tgz] [--mvn ] [--with-tachyon]" - echo "./make-distribution.sh $cl_options " - echo "See Spark's \"Building Spark\" doc for correct Maven options." - echo "" - exit 1 -} - -# Parse arguments -while (( "$#" )); do - case $1 in - --hadoop) - echo "Error: '--hadoop' is no longer supported:" - echo "Error: use Maven profiles and options -Dhadoop.version and -Dyarn.version instead." - echo "Error: Related profiles include hadoop-1, hadoop-2.2, hadoop-2.3 and hadoop-2.4." - exit_with_usage - ;; - --with-yarn) - echo "Error: '--with-yarn' is no longer supported, use Maven option -Pyarn" - exit_with_usage - ;; - --with-hive) - echo "Error: '--with-hive' is no longer supported, use Maven options -Phive and -Phive-thriftserver" - exit_with_usage - ;; - --skip-java-test) - SKIP_JAVA_TEST=true - ;; - --with-tachyon) - SPARK_TACHYON=true - ;; - --tgz) - MAKE_TGZ=true - ;; - --mvn) - MVN="$2" - shift - ;; - --name) - NAME="$2" - shift - ;; - --help) - exit_with_usage - ;; - *) - break - ;; - esac - shift -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)" - if [ "$RPM_JAVA_HOME" != "%java_home" ]; then - JAVA_HOME="$RPM_JAVA_HOME" - echo "No JAVA_HOME set, proceeding with '$JAVA_HOME' learned from rpm" - fi - fi -fi - -if [ -z "$JAVA_HOME" ]; then - echo "Error: JAVA_HOME is not set, cannot proceed." - exit -1 -fi - -if [ $(command -v git) ]; then - GITREV=$(git rev-parse --short HEAD 2>/dev/null || :) - if [ ! -z "$GITREV" ]; then - GITREVSTRING=" (git revision $GITREV)" - fi - unset GITREV -fi - - -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) -SCALA_VERSION=$("$MVN" help:evaluate -Dexpression=scala.binary.version $@ 2>/dev/null\ - | grep -v "INFO"\ - | tail -n 1) -SPARK_HADOOP_VERSION=$("$MVN" help:evaluate -Dexpression=hadoop.version $@ 2>/dev/null\ - | grep -v "INFO"\ - | tail -n 1) -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\ - # because we use "set -o pipefail" - echo -n) - -if [ "$NAME" == "none" ]; then - NAME=$SPARK_HADOOP_VERSION -fi - -echo "Spark version is $VERSION" - -if [ "$MAKE_TGZ" == "true" ]; then - echo "Making spark-$VERSION-bin-$NAME.tgz" -else - echo "Making distribution for Spark $VERSION in $DISTDIR..." -fi - -if [ "$SPARK_TACHYON" == "true" ]; then - echo "Tachyon Enabled" -else - echo "Tachyon Disabled" -fi - -# Build uber fat JAR -cd "$SPARK_HOME" - -export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" - -# Store the command as an array because $MVN variable might have spaces in it. -# Normal quoting tricks don't work. -# See: http://mywiki.wooledge.org/BashFAQ/050 -BUILD_COMMAND=("$MVN" clean package -DskipTests $@) - -# Actually build the jar -echo -e "\nBuilding with..." -echo -e "\$ ${BUILD_COMMAND[@]}\n" - -"${BUILD_COMMAND[@]}" - -# Make directories -rm -rf "$DISTDIR" -mkdir -p "$DISTDIR/lib" -echo "Spark $VERSION$GITREVSTRING built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE" -echo "Build flags: $@" >> "$DISTDIR/RELEASE" - -# Copy jars -cp "$SPARK_HOME"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" -cp "$SPARK_HOME"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" -# This will fail if the -Pyarn profile is not provided -# In this case, silence the error and ignore the return code of this command -cp "$SPARK_HOME"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/" &> /dev/null || : - -# Copy example sources (needed for python and SQL) -mkdir -p "$DISTDIR/examples/src/main" -cp -r "$SPARK_HOME"/examples/src/main "$DISTDIR/examples/src/" - -if [ "$SPARK_HIVE" == "1" ]; then - cp "$SPARK_HOME"/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/" -fi - -# Copy license and ASF files -cp "$SPARK_HOME/LICENSE" "$DISTDIR" -cp -r "$SPARK_HOME/licenses" "$DISTDIR" -cp "$SPARK_HOME/NOTICE" "$DISTDIR" - -if [ -e "$SPARK_HOME"/CHANGES.txt ]; then - cp "$SPARK_HOME/CHANGES.txt" "$DISTDIR" -fi - -# Copy data files -cp -r "$SPARK_HOME/data" "$DISTDIR" - -# Copy other things -mkdir "$DISTDIR"/conf -cp "$SPARK_HOME"/conf/*.template "$DISTDIR"/conf -cp "$SPARK_HOME/README.md" "$DISTDIR" -cp -r "$SPARK_HOME/bin" "$DISTDIR" -cp -r "$SPARK_HOME/python" "$DISTDIR" -cp -r "$SPARK_HOME/sbin" "$DISTDIR" -cp -r "$SPARK_HOME/ec2" "$DISTDIR" -# Copy SparkR if it exists -if [ -d "$SPARK_HOME"/R/lib/SparkR ]; then - mkdir -p "$DISTDIR"/R/lib - cp -r "$SPARK_HOME/R/lib/SparkR" "$DISTDIR"/R/lib -fi - -# Download and copy in tachyon, if requested -if [ "$SPARK_TACHYON" == "true" ]; then - TMPD=`mktemp -d 2>/dev/null || mktemp -d -t 'disttmp'` - - pushd "$TMPD" > /dev/null - echo "Fetching tachyon tgz" - - TACHYON_DL="${TACHYON_TGZ}.part" - if [ $(command -v curl) ]; then - curl --silent -k -L "${TACHYON_URL}" > "${TACHYON_DL}" && mv "${TACHYON_DL}" "${TACHYON_TGZ}" - elif [ $(command -v wget) ]; then - wget --quiet "${TACHYON_URL}" -O "${TACHYON_DL}" && mv "${TACHYON_DL}" "${TACHYON_TGZ}" - else - printf "You do not have curl or wget installed. please install Tachyon manually.\n" - exit -1 - fi - - tar xzf "${TACHYON_TGZ}" - cp "tachyon-${TACHYON_VERSION}/assembly/target/tachyon-assemblies-${TACHYON_VERSION}-jar-with-dependencies.jar" "$DISTDIR/lib" - mkdir -p "$DISTDIR/tachyon/src/main/java/tachyon/web" - cp -r "tachyon-${TACHYON_VERSION}"/{bin,conf,libexec} "$DISTDIR/tachyon" - cp -r "tachyon-${TACHYON_VERSION}"/servers/src/main/java/tachyon/web "$DISTDIR/tachyon/src/main/java/tachyon/web" - - if [[ `uname -a` == Darwin* ]]; then - # need to run sed differently on osx - nl=$'\n'; sed -i "" -e "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\\$nl export TACHYON_JAR=\$TACHYON_HOME/../lib/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" - else - sed -i "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../lib/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" - fi - - popd > /dev/null - rm -rf "$TMPD" -fi - -if [ "$MAKE_TGZ" == "true" ]; then - TARDIR_NAME=spark-$VERSION-bin-$NAME - TARDIR="$SPARK_HOME/$TARDIR_NAME" - rm -rf "$TARDIR" - cp -r "$DISTDIR" "$TARDIR" - tar czf "spark-$VERSION-bin-$NAME.tgz" -C "$SPARK_HOME" "$TARDIR_NAME" - rm -rf "$TARDIR" -fi diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml new file mode 100644 index 000000000000..043d13609fd2 --- /dev/null +++ b/mllib-local/pom.xml @@ -0,0 +1,89 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.3.0-SNAPSHOT + ../pom.xml + + + spark-mllib-local_2.11 + + mllib-local + + jar + Spark Project ML Local Library + http://spark.apache.org/ + + + + org.scalanlp + breeze_${scala.binary.version} + + + org.apache.commons + commons-math3 + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.mockito + mockito-core + test + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + + + netlib-lgpl + + + com.github.fommil.netlib + all + ${netlib.java.version} + pom + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/impl/Utils.scala b/mllib-local/src/main/scala/org/apache/spark/ml/impl/Utils.scala new file mode 100644 index 000000000000..112de982e463 --- /dev/null +++ b/mllib-local/src/main/scala/org/apache/spark/ml/impl/Utils.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.ml.impl + + +private[ml] object Utils { + + lazy val EPSILON = { + var eps = 1.0 + while ((1.0 + (eps / 2.0)) != 1.0) { + eps /= 2.0 + } + eps + } +} diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala new file mode 100644 index 000000000000..2a0f8c11d0a5 --- /dev/null +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/BLAS.scala @@ -0,0 +1,745 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import com.github.fommil.netlib.{BLAS => NetlibBLAS, F2jBLAS} +import com.github.fommil.netlib.BLAS.{getInstance => NativeBLAS} + +/** + * BLAS routines for MLlib's vectors and matrices. + */ +private[spark] object BLAS extends Serializable { + + @transient private var _f2jBLAS: NetlibBLAS = _ + @transient private var _nativeBLAS: NetlibBLAS = _ + + // For level-1 routines, we use Java implementation. + private[ml] def f2jBLAS: NetlibBLAS = { + if (_f2jBLAS == null) { + _f2jBLAS = new F2jBLAS + } + _f2jBLAS + } + + /** + * y += a * x + */ + def axpy(a: Double, x: Vector, y: Vector): Unit = { + require(x.size == y.size) + y match { + case dy: DenseVector => + x match { + case sx: SparseVector => + axpy(a, sx, dy) + case dx: DenseVector => + axpy(a, dx, dy) + case _ => + throw new UnsupportedOperationException( + s"axpy doesn't support x type ${x.getClass}.") + } + case _ => + throw new IllegalArgumentException( + s"axpy only supports adding to a dense vector but got type ${y.getClass}.") + } + } + + /** + * y += a * x + */ + private def axpy(a: Double, x: DenseVector, y: DenseVector): Unit = { + val n = x.size + f2jBLAS.daxpy(n, a, x.values, 1, y.values, 1) + } + + /** + * y += a * x + */ + private def axpy(a: Double, x: SparseVector, y: DenseVector): Unit = { + val xValues = x.values + val xIndices = x.indices + val yValues = y.values + val nnz = xIndices.length + + if (a == 1.0) { + var k = 0 + while (k < nnz) { + yValues(xIndices(k)) += xValues(k) + k += 1 + } + } else { + var k = 0 + while (k < nnz) { + yValues(xIndices(k)) += a * xValues(k) + k += 1 + } + } + } + + /** Y += a * x */ + private[spark] def axpy(a: Double, X: DenseMatrix, Y: DenseMatrix): Unit = { + require(X.numRows == Y.numRows && X.numCols == Y.numCols, "Dimension mismatch: " + + s"size(X) = ${(X.numRows, X.numCols)} but size(Y) = ${(Y.numRows, Y.numCols)}.") + f2jBLAS.daxpy(X.numRows * X.numCols, a, X.values, 1, Y.values, 1) + } + + /** + * dot(x, y) + */ + def dot(x: Vector, y: Vector): Double = { + require(x.size == y.size, + "BLAS.dot(x: Vector, y:Vector) was given Vectors with non-matching sizes:" + + " x.size = " + x.size + ", y.size = " + y.size) + (x, y) match { + case (dx: DenseVector, dy: DenseVector) => + dot(dx, dy) + case (sx: SparseVector, dy: DenseVector) => + dot(sx, dy) + case (dx: DenseVector, sy: SparseVector) => + dot(sy, dx) + case (sx: SparseVector, sy: SparseVector) => + dot(sx, sy) + case _ => + throw new IllegalArgumentException(s"dot doesn't support (${x.getClass}, ${y.getClass}).") + } + } + + /** + * dot(x, y) + */ + private def dot(x: DenseVector, y: DenseVector): Double = { + val n = x.size + f2jBLAS.ddot(n, x.values, 1, y.values, 1) + } + + /** + * dot(x, y) + */ + private def dot(x: SparseVector, y: DenseVector): Double = { + val xValues = x.values + val xIndices = x.indices + val yValues = y.values + val nnz = xIndices.length + + var sum = 0.0 + var k = 0 + while (k < nnz) { + sum += xValues(k) * yValues(xIndices(k)) + k += 1 + } + sum + } + + /** + * dot(x, y) + */ + private def dot(x: SparseVector, y: SparseVector): Double = { + val xValues = x.values + val xIndices = x.indices + val yValues = y.values + val yIndices = y.indices + val nnzx = xIndices.length + val nnzy = yIndices.length + + var kx = 0 + var ky = 0 + var sum = 0.0 + // y catching x + while (kx < nnzx && ky < nnzy) { + val ix = xIndices(kx) + while (ky < nnzy && yIndices(ky) < ix) { + ky += 1 + } + if (ky < nnzy && yIndices(ky) == ix) { + sum += xValues(kx) * yValues(ky) + ky += 1 + } + kx += 1 + } + sum + } + + /** + * y = x + */ + def copy(x: Vector, y: Vector): Unit = { + val n = y.size + require(x.size == n) + y match { + case dy: DenseVector => + x match { + case sx: SparseVector => + val sxIndices = sx.indices + val sxValues = sx.values + val dyValues = dy.values + val nnz = sxIndices.length + + var i = 0 + var k = 0 + while (k < nnz) { + val j = sxIndices(k) + while (i < j) { + dyValues(i) = 0.0 + i += 1 + } + dyValues(i) = sxValues(k) + i += 1 + k += 1 + } + while (i < n) { + dyValues(i) = 0.0 + i += 1 + } + case dx: DenseVector => + Array.copy(dx.values, 0, dy.values, 0, n) + } + case _ => + throw new IllegalArgumentException(s"y must be dense in copy but got ${y.getClass}") + } + } + + /** + * x = a * x + */ + def scal(a: Double, x: Vector): Unit = { + x match { + case sx: SparseVector => + f2jBLAS.dscal(sx.values.length, a, sx.values, 1) + case dx: DenseVector => + f2jBLAS.dscal(dx.values.length, a, dx.values, 1) + case _ => + throw new IllegalArgumentException(s"scal doesn't support vector type ${x.getClass}.") + } + } + + // For level-3 routines, we use the native BLAS. + private def nativeBLAS: NetlibBLAS = { + if (_nativeBLAS == null) { + _nativeBLAS = NativeBLAS + } + _nativeBLAS + } + + /** + * Adds alpha * x * x.t to a matrix in-place. This is the same as BLAS's ?SPR. + * + * @param U the upper triangular part of the matrix in a [[DenseVector]](column major) + */ + def spr(alpha: Double, v: Vector, U: DenseVector): Unit = { + spr(alpha, v, U.values) + } + + /** + * y := alpha*A*x + beta*y + * + * @param n The order of the n by n matrix A. + * @param A The upper triangular part of A in a [[DenseVector]] (column major). + * @param x The [[DenseVector]] transformed by A. + * @param y The [[DenseVector]] to be modified in place. + */ + def dspmv( + n: Int, + alpha: Double, + A: DenseVector, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + f2jBLAS.dspmv("U", n, alpha, A.values, x.values, 1, beta, y.values, 1) + } + + /** + * Adds alpha * x * x.t to a matrix in-place. This is the same as BLAS's ?SPR. + * + * @param U the upper triangular part of the matrix packed in an array (column major) + */ + def spr(alpha: Double, v: Vector, U: Array[Double]): Unit = { + val n = v.size + v match { + case DenseVector(values) => + NativeBLAS.dspr("U", n, alpha, values, 1, U) + case SparseVector(size, indices, values) => + val nnz = indices.length + var colStartIdx = 0 + var prevCol = 0 + var col = 0 + var j = 0 + var i = 0 + var av = 0.0 + while (j < nnz) { + col = indices(j) + // Skip empty columns. + colStartIdx += (col - prevCol) * (col + prevCol + 1) / 2 + col = indices(j) + av = alpha * values(j) + i = 0 + while (i <= j) { + U(colStartIdx + indices(i)) += av * values(i) + i += 1 + } + j += 1 + prevCol = col + } + } + } + + /** + * A := alpha * x * x^T^ + A + * @param alpha a real scalar that will be multiplied to x * x^T^. + * @param x the vector x that contains the n elements. + * @param A the symmetric matrix A. Size of n x n. + */ + def syr(alpha: Double, x: Vector, A: DenseMatrix) { + val mA = A.numRows + val nA = A.numCols + require(mA == nA, s"A is not a square matrix (and hence is not symmetric). A: $mA x $nA") + require(mA == x.size, s"The size of x doesn't match the rank of A. A: $mA x $nA, x: ${x.size}") + + x match { + case dv: DenseVector => syr(alpha, dv, A) + case sv: SparseVector => syr(alpha, sv, A) + case _ => + throw new IllegalArgumentException(s"syr doesn't support vector type ${x.getClass}.") + } + } + + private def syr(alpha: Double, x: DenseVector, A: DenseMatrix) { + val nA = A.numRows + val mA = A.numCols + + nativeBLAS.dsyr("U", x.size, alpha, x.values, 1, A.values, nA) + + // Fill lower triangular part of A + var i = 0 + while (i < mA) { + var j = i + 1 + while (j < nA) { + A(j, i) = A(i, j) + j += 1 + } + i += 1 + } + } + + private def syr(alpha: Double, x: SparseVector, A: DenseMatrix) { + val mA = A.numCols + val xIndices = x.indices + val xValues = x.values + val nnz = xValues.length + val Avalues = A.values + + var i = 0 + while (i < nnz) { + val multiplier = alpha * xValues(i) + val offset = xIndices(i) * mA + var j = 0 + while (j < nnz) { + Avalues(xIndices(j) + offset) += multiplier * xValues(j) + j += 1 + } + i += 1 + } + } + + /** + * C := alpha * A * B + beta * C + * @param alpha a scalar to scale the multiplication A * B. + * @param A the matrix A that will be left multiplied to B. Size of m x k. + * @param B the matrix B that will be left multiplied by A. Size of k x n. + * @param beta a scalar that can be used to scale matrix C. + * @param C the resulting matrix C. Size of m x n. C.isTransposed must be false. + */ + def gemm( + alpha: Double, + A: Matrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + require(!C.isTransposed, + "The matrix C cannot be the product of a transpose() call. C.isTransposed must be false.") + if (alpha == 0.0 && beta == 1.0) { + // gemm: alpha is equal to 0 and beta is equal to 1. Returning C. + return + } else if (alpha == 0.0) { + f2jBLAS.dscal(C.values.length, beta, C.values, 1) + } else { + A match { + case sparse: SparseMatrix => gemm(alpha, sparse, B, beta, C) + case dense: DenseMatrix => gemm(alpha, dense, B, beta, C) + case _ => + throw new IllegalArgumentException(s"gemm doesn't support matrix type ${A.getClass}.") + } + } + } + + /** + * C := alpha * A * B + beta * C + * For `DenseMatrix` A. + */ + private def gemm( + alpha: Double, + A: DenseMatrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + val tAstr = if (A.isTransposed) "T" else "N" + val tBstr = if (B.isTransposed) "T" else "N" + val lda = if (!A.isTransposed) A.numRows else A.numCols + val ldb = if (!B.isTransposed) B.numRows else B.numCols + + require(A.numCols == B.numRows, + s"The columns of A don't match the rows of B. A: ${A.numCols}, B: ${B.numRows}") + require(A.numRows == C.numRows, + s"The rows of C don't match the rows of A. C: ${C.numRows}, A: ${A.numRows}") + require(B.numCols == C.numCols, + s"The columns of C don't match the columns of B. C: ${C.numCols}, A: ${B.numCols}") + nativeBLAS.dgemm(tAstr, tBstr, A.numRows, B.numCols, A.numCols, alpha, A.values, lda, + B.values, ldb, beta, C.values, C.numRows) + } + + /** + * C := alpha * A * B + beta * C + * For `SparseMatrix` A. + */ + private def gemm( + alpha: Double, + A: SparseMatrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + val mA: Int = A.numRows + val nB: Int = B.numCols + val kA: Int = A.numCols + val kB: Int = B.numRows + + require(kA == kB, s"The columns of A don't match the rows of B. A: $kA, B: $kB") + require(mA == C.numRows, s"The rows of C don't match the rows of A. C: ${C.numRows}, A: $mA") + require(nB == C.numCols, + s"The columns of C don't match the columns of B. C: ${C.numCols}, A: $nB") + + val Avals = A.values + val Bvals = B.values + val Cvals = C.values + val ArowIndices = A.rowIndices + val AcolPtrs = A.colPtrs + + // Slicing is easy in this case. This is the optimal multiplication setting for sparse matrices + if (A.isTransposed) { + var colCounterForB = 0 + if (!B.isTransposed) { // Expensive to put the check inside the loop + while (colCounterForB < nB) { + var rowCounterForA = 0 + val Cstart = colCounterForB * mA + val Bstart = colCounterForB * kA + while (rowCounterForA < mA) { + var i = AcolPtrs(rowCounterForA) + val indEnd = AcolPtrs(rowCounterForA + 1) + var sum = 0.0 + while (i < indEnd) { + sum += Avals(i) * Bvals(Bstart + ArowIndices(i)) + i += 1 + } + val Cindex = Cstart + rowCounterForA + Cvals(Cindex) = beta * Cvals(Cindex) + sum * alpha + rowCounterForA += 1 + } + colCounterForB += 1 + } + } else { + while (colCounterForB < nB) { + var rowCounterForA = 0 + val Cstart = colCounterForB * mA + while (rowCounterForA < mA) { + var i = AcolPtrs(rowCounterForA) + val indEnd = AcolPtrs(rowCounterForA + 1) + var sum = 0.0 + while (i < indEnd) { + sum += Avals(i) * B(ArowIndices(i), colCounterForB) + i += 1 + } + val Cindex = Cstart + rowCounterForA + Cvals(Cindex) = beta * Cvals(Cindex) + sum * alpha + rowCounterForA += 1 + } + colCounterForB += 1 + } + } + } else { + // Scale matrix first if `beta` is not equal to 1.0 + if (beta != 1.0) { + f2jBLAS.dscal(C.values.length, beta, C.values, 1) + } + // Perform matrix multiplication and add to C. The rows of A are multiplied by the columns of + // B, and added to C. + var colCounterForB = 0 // the column to be updated in C + if (!B.isTransposed) { // Expensive to put the check inside the loop + while (colCounterForB < nB) { + var colCounterForA = 0 // The column of A to multiply with the row of B + val Bstart = colCounterForB * kB + val Cstart = colCounterForB * mA + while (colCounterForA < kA) { + var i = AcolPtrs(colCounterForA) + val indEnd = AcolPtrs(colCounterForA + 1) + val Bval = Bvals(Bstart + colCounterForA) * alpha + while (i < indEnd) { + Cvals(Cstart + ArowIndices(i)) += Avals(i) * Bval + i += 1 + } + colCounterForA += 1 + } + colCounterForB += 1 + } + } else { + while (colCounterForB < nB) { + var colCounterForA = 0 // The column of A to multiply with the row of B + val Cstart = colCounterForB * mA + while (colCounterForA < kA) { + var i = AcolPtrs(colCounterForA) + val indEnd = AcolPtrs(colCounterForA + 1) + val Bval = B(colCounterForA, colCounterForB) * alpha + while (i < indEnd) { + Cvals(Cstart + ArowIndices(i)) += Avals(i) * Bval + i += 1 + } + colCounterForA += 1 + } + colCounterForB += 1 + } + } + } + } + + /** + * y := alpha * A * x + beta * y + * @param alpha a scalar to scale the multiplication A * x. + * @param A the matrix A that will be left multiplied to x. Size of m x n. + * @param x the vector x that will be left multiplied by A. Size of n x 1. + * @param beta a scalar that can be used to scale vector y. + * @param y the resulting vector y. Size of m x 1. + */ + def gemv( + alpha: Double, + A: Matrix, + x: Vector, + beta: Double, + y: DenseVector): Unit = { + require(A.numCols == x.size, + s"The columns of A don't match the number of elements of x. A: ${A.numCols}, x: ${x.size}") + require(A.numRows == y.size, + s"The rows of A don't match the number of elements of y. A: ${A.numRows}, y:${y.size}") + if (alpha == 0.0 && beta == 1.0) { + // gemv: alpha is equal to 0 and beta is equal to 1. Returning y. + return + } else if (alpha == 0.0) { + scal(beta, y) + } else { + (A, x) match { + case (smA: SparseMatrix, dvx: DenseVector) => + gemv(alpha, smA, dvx, beta, y) + case (smA: SparseMatrix, svx: SparseVector) => + gemv(alpha, smA, svx, beta, y) + case (dmA: DenseMatrix, dvx: DenseVector) => + gemv(alpha, dmA, dvx, beta, y) + case (dmA: DenseMatrix, svx: SparseVector) => + gemv(alpha, dmA, svx, beta, y) + case _ => + throw new IllegalArgumentException(s"gemv doesn't support running on matrix type " + + s"${A.getClass} and vector type ${x.getClass}.") + } + } + } + + /** + * y := alpha * A * x + beta * y + * For `DenseMatrix` A and `DenseVector` x. + */ + private def gemv( + alpha: Double, + A: DenseMatrix, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + val tStrA = if (A.isTransposed) "T" else "N" + val mA = if (!A.isTransposed) A.numRows else A.numCols + val nA = if (!A.isTransposed) A.numCols else A.numRows + nativeBLAS.dgemv(tStrA, mA, nA, alpha, A.values, mA, x.values, 1, beta, + y.values, 1) + } + + /** + * y := alpha * A * x + beta * y + * For `DenseMatrix` A and `SparseVector` x. + */ + private def gemv( + alpha: Double, + A: DenseMatrix, + x: SparseVector, + beta: Double, + y: DenseVector): Unit = { + val mA: Int = A.numRows + val nA: Int = A.numCols + + val Avals = A.values + + val xIndices = x.indices + val xNnz = xIndices.length + val xValues = x.values + val yValues = y.values + + if (A.isTransposed) { + var rowCounterForA = 0 + while (rowCounterForA < mA) { + var sum = 0.0 + var k = 0 + while (k < xNnz) { + sum += xValues(k) * Avals(xIndices(k) + rowCounterForA * nA) + k += 1 + } + yValues(rowCounterForA) = sum * alpha + beta * yValues(rowCounterForA) + rowCounterForA += 1 + } + } else { + var rowCounterForA = 0 + while (rowCounterForA < mA) { + var sum = 0.0 + var k = 0 + while (k < xNnz) { + sum += xValues(k) * Avals(xIndices(k) * mA + rowCounterForA) + k += 1 + } + yValues(rowCounterForA) = sum * alpha + beta * yValues(rowCounterForA) + rowCounterForA += 1 + } + } + } + + /** + * y := alpha * A * x + beta * y + * For `SparseMatrix` A and `SparseVector` x. + */ + private def gemv( + alpha: Double, + A: SparseMatrix, + x: SparseVector, + beta: Double, + y: DenseVector): Unit = { + val xValues = x.values + val xIndices = x.indices + val xNnz = xIndices.length + + val yValues = y.values + + val mA: Int = A.numRows + val nA: Int = A.numCols + + val Avals = A.values + val Arows = if (!A.isTransposed) A.rowIndices else A.colPtrs + val Acols = if (!A.isTransposed) A.colPtrs else A.rowIndices + + if (A.isTransposed) { + var rowCounter = 0 + while (rowCounter < mA) { + var i = Arows(rowCounter) + val indEnd = Arows(rowCounter + 1) + var sum = 0.0 + var k = 0 + while (i < indEnd && k < xNnz) { + if (xIndices(k) == Acols(i)) { + sum += Avals(i) * xValues(k) + k += 1 + i += 1 + } else if (xIndices(k) < Acols(i)) { + k += 1 + } else { + i += 1 + } + } + yValues(rowCounter) = sum * alpha + beta * yValues(rowCounter) + rowCounter += 1 + } + } else { + if (beta != 1.0) scal(beta, y) + + var colCounterForA = 0 + var k = 0 + while (colCounterForA < nA && k < xNnz) { + if (xIndices(k) == colCounterForA) { + var i = Acols(colCounterForA) + val indEnd = Acols(colCounterForA + 1) + + val xTemp = xValues(k) * alpha + while (i < indEnd) { + val rowIndex = Arows(i) + yValues(Arows(i)) += Avals(i) * xTemp + i += 1 + } + k += 1 + } + colCounterForA += 1 + } + } + } + + /** + * y := alpha * A * x + beta * y + * For `SparseMatrix` A and `DenseVector` x. + */ + private def gemv( + alpha: Double, + A: SparseMatrix, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + val xValues = x.values + val yValues = y.values + val mA: Int = A.numRows + val nA: Int = A.numCols + + val Avals = A.values + val Arows = if (!A.isTransposed) A.rowIndices else A.colPtrs + val Acols = if (!A.isTransposed) A.colPtrs else A.rowIndices + // Slicing is easy in this case. This is the optimal multiplication setting for sparse matrices + if (A.isTransposed) { + var rowCounter = 0 + while (rowCounter < mA) { + var i = Arows(rowCounter) + val indEnd = Arows(rowCounter + 1) + var sum = 0.0 + while (i < indEnd) { + sum += Avals(i) * xValues(Acols(i)) + i += 1 + } + yValues(rowCounter) = beta * yValues(rowCounter) + sum * alpha + rowCounter += 1 + } + } else { + if (beta != 1.0) scal(beta, y) + // Perform matrix-vector multiplication and add to y + var colCounterForA = 0 + while (colCounterForA < nA) { + var i = Acols(colCounterForA) + val indEnd = Acols(colCounterForA + 1) + val xVal = xValues(colCounterForA) * alpha + while (i < indEnd) { + val rowIndex = Arows(i) + yValues(rowIndex) += Avals(i) * xVal + i += 1 + } + colCounterForA += 1 + } + } + } +} diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala new file mode 100644 index 000000000000..66c53624419d --- /dev/null +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala @@ -0,0 +1,1292 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import java.util.{Arrays, Random} + +import scala.collection.mutable.{ArrayBuffer, ArrayBuilder => MArrayBuilder, HashSet => MHashSet} + +import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM} +import com.github.fommil.netlib.BLAS.{getInstance => blas} + +import org.apache.spark.annotation.Since + +/** + * Trait for a local matrix. + */ +@Since("2.0.0") +sealed trait Matrix extends Serializable { + + /** Number of rows. */ + @Since("2.0.0") + def numRows: Int + + /** Number of columns. */ + @Since("2.0.0") + def numCols: Int + + /** Flag that keeps track whether the matrix is transposed or not. False by default. */ + @Since("2.0.0") + val isTransposed: Boolean = false + + /** Indicates whether the values backing this matrix are arranged in column major order. */ + private[ml] def isColMajor: Boolean = !isTransposed + + /** Indicates whether the values backing this matrix are arranged in row major order. */ + private[ml] def isRowMajor: Boolean = isTransposed + + /** Converts to a dense array in column major. */ + @Since("2.0.0") + def toArray: Array[Double] = { + val newArray = new Array[Double](numRows * numCols) + foreachActive { (i, j, v) => + newArray(j * numRows + i) = v + } + newArray + } + + /** + * Returns an iterator of column vectors. + * This operation could be expensive, depending on the underlying storage. + */ + @Since("2.0.0") + def colIter: Iterator[Vector] + + /** + * Returns an iterator of row vectors. + * This operation could be expensive, depending on the underlying storage. + */ + @Since("2.0.0") + def rowIter: Iterator[Vector] = this.transpose.colIter + + /** Converts to a breeze matrix. */ + private[ml] def asBreeze: BM[Double] + + /** Gets the (i, j)-th element. */ + @Since("2.0.0") + def apply(i: Int, j: Int): Double + + /** Return the index for the (i, j)-th element in the backing array. */ + private[ml] def index(i: Int, j: Int): Int + + /** Update element at (i, j) */ + private[ml] def update(i: Int, j: Int, v: Double): Unit + + /** Get a deep copy of the matrix. */ + @Since("2.0.0") + def copy: Matrix + + /** + * Transpose the Matrix. Returns a new `Matrix` instance sharing the same underlying data. + */ + @Since("2.0.0") + def transpose: Matrix + + /** + * Convenience method for `Matrix`-`DenseMatrix` multiplication. + */ + @Since("2.0.0") + def multiply(y: DenseMatrix): DenseMatrix = { + val C: DenseMatrix = DenseMatrix.zeros(numRows, y.numCols) + BLAS.gemm(1.0, this, y, 0.0, C) + C + } + + /** + * Convenience method for `Matrix`-`DenseVector` multiplication. For binary compatibility. + */ + @Since("2.0.0") + def multiply(y: DenseVector): DenseVector = { + multiply(y.asInstanceOf[Vector]) + } + + /** + * Convenience method for `Matrix`-`Vector` multiplication. + */ + @Since("2.0.0") + def multiply(y: Vector): DenseVector = { + val output = new DenseVector(new Array[Double](numRows)) + BLAS.gemv(1.0, this, y, 0.0, output) + output + } + + /** A human readable representation of the matrix */ + override def toString: String = asBreeze.toString() + + /** A human readable representation of the matrix with maximum lines and width */ + @Since("2.0.0") + def toString(maxLines: Int, maxLineWidth: Int): String = asBreeze.toString(maxLines, maxLineWidth) + + /** + * Map the values of this matrix using a function. Generates a new matrix. Performs the + * function on only the backing array. For example, an operation such as addition or + * subtraction will only be performed on the non-zero values in a `SparseMatrix`. + */ + private[spark] def map(f: Double => Double): Matrix + + /** + * Update all the values of this matrix using the function f. Performed in-place on the + * backing array. For example, an operation such as addition or subtraction will only be + * performed on the non-zero values in a `SparseMatrix`. + */ + private[ml] def update(f: Double => Double): Matrix + + /** + * Applies a function `f` to all the active elements of dense and sparse matrix. The ordering + * of the elements are not defined. + * + * @param f the function takes three parameters where the first two parameters are the row + * and column indices respectively with the type `Int`, and the final parameter is the + * corresponding value in the matrix with type `Double`. + */ + @Since("2.2.0") + def foreachActive(f: (Int, Int, Double) => Unit): Unit + + /** + * Find the number of non-zero active values. + */ + @Since("2.0.0") + def numNonzeros: Int + + /** + * Find the number of values stored explicitly. These values can be zero as well. + */ + @Since("2.0.0") + def numActives: Int + + /** + * Converts this matrix to a sparse matrix. + * + * @param colMajor Whether the values of the resulting sparse matrix should be in column major + * or row major order. If `false`, resulting matrix will be row major. + */ + private[ml] def toSparseMatrix(colMajor: Boolean): SparseMatrix + + /** + * Converts this matrix to a sparse matrix in column major order. + */ + @Since("2.2.0") + def toSparseColMajor: SparseMatrix = toSparseMatrix(colMajor = true) + + /** + * Converts this matrix to a sparse matrix in row major order. + */ + @Since("2.2.0") + def toSparseRowMajor: SparseMatrix = toSparseMatrix(colMajor = false) + + /** + * Converts this matrix to a sparse matrix while maintaining the layout of the current matrix. + */ + @Since("2.2.0") + def toSparse: SparseMatrix = toSparseMatrix(colMajor = isColMajor) + + /** + * Converts this matrix to a dense matrix. + * + * @param colMajor Whether the values of the resulting dense matrix should be in column major + * or row major order. If `false`, resulting matrix will be row major. + */ + private[ml] def toDenseMatrix(colMajor: Boolean): DenseMatrix + + /** + * Converts this matrix to a dense matrix while maintaining the layout of the current matrix. + */ + @Since("2.2.0") + def toDense: DenseMatrix = toDenseMatrix(colMajor = isColMajor) + + /** + * Converts this matrix to a dense matrix in row major order. + */ + @Since("2.2.0") + def toDenseRowMajor: DenseMatrix = toDenseMatrix(colMajor = false) + + /** + * Converts this matrix to a dense matrix in column major order. + */ + @Since("2.2.0") + def toDenseColMajor: DenseMatrix = toDenseMatrix(colMajor = true) + + /** + * Returns a matrix in dense or sparse column major format, whichever uses less storage. + */ + @Since("2.2.0") + def compressedColMajor: Matrix = { + if (getDenseSizeInBytes <= getSparseSizeInBytes(colMajor = true)) { + this.toDenseColMajor + } else { + this.toSparseColMajor + } + } + + /** + * Returns a matrix in dense or sparse row major format, whichever uses less storage. + */ + @Since("2.2.0") + def compressedRowMajor: Matrix = { + if (getDenseSizeInBytes <= getSparseSizeInBytes(colMajor = false)) { + this.toDenseRowMajor + } else { + this.toSparseRowMajor + } + } + + /** + * Returns a matrix in dense column major, dense row major, sparse row major, or sparse column + * major format, whichever uses less storage. When dense representation is optimal, it maintains + * the current layout order. + */ + @Since("2.2.0") + def compressed: Matrix = { + val cscSize = getSparseSizeInBytes(colMajor = true) + val csrSize = getSparseSizeInBytes(colMajor = false) + if (getDenseSizeInBytes <= math.min(cscSize, csrSize)) { + // dense matrix size is the same for column major and row major, so maintain current layout + this.toDense + } else if (cscSize <= csrSize) { + this.toSparseColMajor + } else { + this.toSparseRowMajor + } + } + + /** Gets the size of the dense representation of this `Matrix`. */ + private[ml] def getDenseSizeInBytes: Long = { + Matrices.getDenseSize(numCols, numRows) + } + + /** Gets the size of the minimal sparse representation of this `Matrix`. */ + private[ml] def getSparseSizeInBytes(colMajor: Boolean): Long = { + val nnz = numNonzeros + val numPtrs = if (colMajor) numCols + 1L else numRows + 1L + Matrices.getSparseSize(nnz, numPtrs) + } + + /** Gets the current size in bytes of this `Matrix`. Useful for testing */ + private[ml] def getSizeInBytes: Long +} + +/** + * Column-major dense matrix. + * The entry values are stored in a single array of doubles with columns listed in sequence. + * For example, the following matrix + * {{{ + * 1.0 2.0 + * 3.0 4.0 + * 5.0 6.0 + * }}} + * is stored as `[1.0, 3.0, 5.0, 2.0, 4.0, 6.0]`. + * + * @param numRows number of rows + * @param numCols number of columns + * @param values matrix entries in column major if not transposed or in row major otherwise + * @param isTransposed whether the matrix is transposed. If true, `values` stores the matrix in + * row major. + */ +@Since("2.0.0") +class DenseMatrix @Since("2.0.0") ( + @Since("2.0.0") val numRows: Int, + @Since("2.0.0") val numCols: Int, + @Since("2.0.0") val values: Array[Double], + override val isTransposed: Boolean) extends Matrix { + + require(values.length == numRows * numCols, "The number of values supplied doesn't match the " + + s"size of the matrix! values.length: ${values.length}, numRows * numCols: ${numRows * numCols}") + + /** + * Column-major dense matrix. + * The entry values are stored in a single array of doubles with columns listed in sequence. + * For example, the following matrix + * {{{ + * 1.0 2.0 + * 3.0 4.0 + * 5.0 6.0 + * }}} + * is stored as `[1.0, 3.0, 5.0, 2.0, 4.0, 6.0]`. + * + * @param numRows number of rows + * @param numCols number of columns + * @param values matrix entries in column major + */ + @Since("2.0.0") + def this(numRows: Int, numCols: Int, values: Array[Double]) = + this(numRows, numCols, values, false) + + override def equals(o: Any): Boolean = o match { + case m: Matrix => asBreeze == m.asBreeze + case _ => false + } + + override def hashCode: Int = { + Seq(numRows, numCols, toArray).## + } + + private[ml] def asBreeze: BM[Double] = { + if (!isTransposed) { + new BDM[Double](numRows, numCols, values) + } else { + val breezeMatrix = new BDM[Double](numCols, numRows, values) + breezeMatrix.t + } + } + + private[ml] def apply(i: Int): Double = values(i) + + override def apply(i: Int, j: Int): Double = values(index(i, j)) + + private[ml] def index(i: Int, j: Int): Int = { + require(i >= 0 && i < numRows, s"Expected 0 <= i < $numRows, got i = $i.") + require(j >= 0 && j < numCols, s"Expected 0 <= j < $numCols, got j = $j.") + if (!isTransposed) i + numRows * j else j + numCols * i + } + + private[ml] def update(i: Int, j: Int, v: Double): Unit = { + values(index(i, j)) = v + } + + override def copy: DenseMatrix = new DenseMatrix(numRows, numCols, values.clone()) + + private[spark] def map(f: Double => Double) = new DenseMatrix(numRows, numCols, values.map(f), + isTransposed) + + private[ml] def update(f: Double => Double): DenseMatrix = { + val len = values.length + var i = 0 + while (i < len) { + values(i) = f(values(i)) + i += 1 + } + this + } + + override def transpose: DenseMatrix = new DenseMatrix(numCols, numRows, values, !isTransposed) + + override def foreachActive(f: (Int, Int, Double) => Unit): Unit = { + if (!isTransposed) { + // outer loop over columns + var j = 0 + while (j < numCols) { + var i = 0 + val indStart = j * numRows + while (i < numRows) { + f(i, j, values(indStart + i)) + i += 1 + } + j += 1 + } + } else { + // outer loop over rows + var i = 0 + while (i < numRows) { + var j = 0 + val indStart = i * numCols + while (j < numCols) { + f(i, j, values(indStart + j)) + j += 1 + } + i += 1 + } + } + } + + override def numNonzeros: Int = values.count(_ != 0) + + override def numActives: Int = values.length + + /** + * Generate a `SparseMatrix` from the given `DenseMatrix`. + * + * @param colMajor Whether the resulting `SparseMatrix` values will be in column major order. + */ + private[ml] override def toSparseMatrix(colMajor: Boolean): SparseMatrix = { + if (!colMajor) this.transpose.toSparseColMajor.transpose + else { + val spVals: MArrayBuilder[Double] = new MArrayBuilder.ofDouble + val colPtrs: Array[Int] = new Array[Int](numCols + 1) + val rowIndices: MArrayBuilder[Int] = new MArrayBuilder.ofInt + var nnz = 0 + var j = 0 + while (j < numCols) { + var i = 0 + while (i < numRows) { + val v = values(index(i, j)) + if (v != 0.0) { + rowIndices += i + spVals += v + nnz += 1 + } + i += 1 + } + j += 1 + colPtrs(j) = nnz + } + new SparseMatrix(numRows, numCols, colPtrs, rowIndices.result(), spVals.result()) + } + } + + /** + * Generate a `DenseMatrix` from this `DenseMatrix`. + * + * @param colMajor Whether the resulting `DenseMatrix` values will be in column major order. + */ + private[ml] override def toDenseMatrix(colMajor: Boolean): DenseMatrix = { + if (isRowMajor && colMajor) { + new DenseMatrix(numRows, numCols, this.toArray, isTransposed = false) + } else if (isColMajor && !colMajor) { + new DenseMatrix(numRows, numCols, this.transpose.toArray, isTransposed = true) + } else { + this + } + } + + override def colIter: Iterator[Vector] = { + if (isTransposed) { + Iterator.tabulate(numCols) { j => + val col = new Array[Double](numRows) + blas.dcopy(numRows, values, j, numCols, col, 0, 1) + new DenseVector(col) + } + } else { + Iterator.tabulate(numCols) { j => + new DenseVector(values.slice(j * numRows, (j + 1) * numRows)) + } + } + } + + private[ml] def getSizeInBytes: Long = Matrices.getDenseSize(numCols, numRows) +} + +/** + * Factory methods for [[org.apache.spark.ml.linalg.DenseMatrix]]. + */ +@Since("2.0.0") +object DenseMatrix { + + /** + * Generate a `DenseMatrix` consisting of zeros. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values of zeros + */ + @Since("2.0.0") + def zeros(numRows: Int, numCols: Int): DenseMatrix = { + require(numRows.toLong * numCols <= Int.MaxValue, + s"$numRows x $numCols dense matrix is too large to allocate") + new DenseMatrix(numRows, numCols, new Array[Double](numRows * numCols)) + } + + /** + * Generate a `DenseMatrix` consisting of ones. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values of ones + */ + @Since("2.0.0") + def ones(numRows: Int, numCols: Int): DenseMatrix = { + require(numRows.toLong * numCols <= Int.MaxValue, + s"$numRows x $numCols dense matrix is too large to allocate") + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(1.0)) + } + + /** + * Generate an Identity Matrix in `DenseMatrix` format. + * @param n number of rows and columns of the matrix + * @return `DenseMatrix` with size `n` x `n` and values of ones on the diagonal + */ + @Since("2.0.0") + def eye(n: Int): DenseMatrix = { + val identity = DenseMatrix.zeros(n, n) + var i = 0 + while (i < n) { + identity.update(i, i, 1.0) + i += 1 + } + identity + } + + /** + * Generate a `DenseMatrix` consisting of `i.i.d.` uniform random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param rng a random number generator + * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1) + */ + @Since("2.0.0") + def rand(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { + require(numRows.toLong * numCols <= Int.MaxValue, + s"$numRows x $numCols dense matrix is too large to allocate") + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextDouble())) + } + + /** + * Generate a `DenseMatrix` consisting of `i.i.d.` gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param rng a random number generator + * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1) + */ + @Since("2.0.0") + def randn(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { + require(numRows.toLong * numCols <= Int.MaxValue, + s"$numRows x $numCols dense matrix is too large to allocate") + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextGaussian())) + } + + /** + * Generate a diagonal matrix in `DenseMatrix` format from the supplied values. + * @param vector a `Vector` that will form the values on the diagonal of the matrix + * @return Square `DenseMatrix` with size `values.length` x `values.length` and `values` + * on the diagonal + */ + @Since("2.0.0") + def diag(vector: Vector): DenseMatrix = { + val n = vector.size + val matrix = DenseMatrix.zeros(n, n) + val values = vector.toArray + var i = 0 + while (i < n) { + matrix.update(i, i, values(i)) + i += 1 + } + matrix + } +} + +/** + * Column-major sparse matrix. + * The entry values are stored in Compressed Sparse Column (CSC) format. + * For example, the following matrix + * {{{ + * 1.0 0.0 4.0 + * 0.0 3.0 5.0 + * 2.0 0.0 6.0 + * }}} + * is stored as `values: [1.0, 2.0, 3.0, 4.0, 5.0, 6.0]`, + * `rowIndices=[0, 2, 1, 0, 1, 2]`, `colPointers=[0, 2, 3, 6]`. + * + * @param numRows number of rows + * @param numCols number of columns + * @param colPtrs the index corresponding to the start of a new column (if not transposed) + * @param rowIndices the row index of the entry (if not transposed). They must be in strictly + * increasing order for each column + * @param values nonzero matrix entries in column major (if not transposed) + * @param isTransposed whether the matrix is transposed. If true, the matrix can be considered + * Compressed Sparse Row (CSR) format, where `colPtrs` behaves as rowPtrs, + * and `rowIndices` behave as colIndices, and `values` are stored in row major. + */ +@Since("2.0.0") +class SparseMatrix @Since("2.0.0") ( + @Since("2.0.0") val numRows: Int, + @Since("2.0.0") val numCols: Int, + @Since("2.0.0") val colPtrs: Array[Int], + @Since("2.0.0") val rowIndices: Array[Int], + @Since("2.0.0") val values: Array[Double], + override val isTransposed: Boolean) extends Matrix { + + require(values.length == rowIndices.length, "The number of row indices and values don't match! " + + s"values.length: ${values.length}, rowIndices.length: ${rowIndices.length}") + if (isTransposed) { + require(colPtrs.length == numRows + 1, + s"Expecting ${numRows + 1} colPtrs when numRows = $numRows but got ${colPtrs.length}") + } else { + require(colPtrs.length == numCols + 1, + s"Expecting ${numCols + 1} colPtrs when numCols = $numCols but got ${colPtrs.length}") + } + require(values.length == colPtrs.last, "The last value of colPtrs must equal the number of " + + s"elements. values.length: ${values.length}, colPtrs.last: ${colPtrs.last}") + + /** + * Column-major sparse matrix. + * The entry values are stored in Compressed Sparse Column (CSC) format. + * For example, the following matrix + * {{{ + * 1.0 0.0 4.0 + * 0.0 3.0 5.0 + * 2.0 0.0 6.0 + * }}} + * is stored as `values: [1.0, 2.0, 3.0, 4.0, 5.0, 6.0]`, + * `rowIndices=[0, 2, 1, 0, 1, 2]`, `colPointers=[0, 2, 3, 6]`. + * + * @param numRows number of rows + * @param numCols number of columns + * @param colPtrs the index corresponding to the start of a new column + * @param rowIndices the row index of the entry. They must be in strictly increasing + * order for each column + * @param values non-zero matrix entries in column major + */ + @Since("2.0.0") + def this( + numRows: Int, + numCols: Int, + colPtrs: Array[Int], + rowIndices: Array[Int], + values: Array[Double]) = this(numRows, numCols, colPtrs, rowIndices, values, false) + + override def hashCode(): Int = asBreeze.hashCode() + + override def equals(o: Any): Boolean = o match { + case m: Matrix => asBreeze == m.asBreeze + case _ => false + } + + private[ml] def asBreeze: BM[Double] = { + if (!isTransposed) { + new BSM[Double](values, numRows, numCols, colPtrs, rowIndices) + } else { + val breezeMatrix = new BSM[Double](values, numCols, numRows, colPtrs, rowIndices) + breezeMatrix.t + } + } + + override def apply(i: Int, j: Int): Double = { + val ind = index(i, j) + if (ind < 0) 0.0 else values(ind) + } + + private[ml] def index(i: Int, j: Int): Int = { + require(i >= 0 && i < numRows, s"Expected 0 <= i < $numRows, got i = $i.") + require(j >= 0 && j < numCols, s"Expected 0 <= j < $numCols, got j = $j.") + if (!isTransposed) { + Arrays.binarySearch(rowIndices, colPtrs(j), colPtrs(j + 1), i) + } else { + Arrays.binarySearch(rowIndices, colPtrs(i), colPtrs(i + 1), j) + } + } + + private[ml] def update(i: Int, j: Int, v: Double): Unit = { + val ind = index(i, j) + if (ind < 0) { + throw new NoSuchElementException("The given row and column indices correspond to a zero " + + "value. Only non-zero elements in Sparse Matrices can be updated.") + } else { + values(ind) = v + } + } + + override def copy: SparseMatrix = { + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.clone()) + } + + private[spark] def map(f: Double => Double) = + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.map(f), isTransposed) + + private[ml] def update(f: Double => Double): SparseMatrix = { + val len = values.length + var i = 0 + while (i < len) { + values(i) = f(values(i)) + i += 1 + } + this + } + + override def transpose: SparseMatrix = + new SparseMatrix(numCols, numRows, colPtrs, rowIndices, values, !isTransposed) + + override def foreachActive(f: (Int, Int, Double) => Unit): Unit = { + if (!isTransposed) { + var j = 0 + while (j < numCols) { + var idx = colPtrs(j) + val idxEnd = colPtrs(j + 1) + while (idx < idxEnd) { + f(rowIndices(idx), j, values(idx)) + idx += 1 + } + j += 1 + } + } else { + var i = 0 + while (i < numRows) { + var idx = colPtrs(i) + val idxEnd = colPtrs(i + 1) + while (idx < idxEnd) { + val j = rowIndices(idx) + f(i, j, values(idx)) + idx += 1 + } + i += 1 + } + } + } + + override def numNonzeros: Int = values.count(_ != 0) + + override def numActives: Int = values.length + + /** + * Generate a `SparseMatrix` from this `SparseMatrix`, removing explicit zero values if they + * exist. + * + * @param colMajor Whether or not the resulting `SparseMatrix` values are in column major + * order. + */ + private[ml] override def toSparseMatrix(colMajor: Boolean): SparseMatrix = { + if (isColMajor && !colMajor) { + // it is col major and we want row major, use breeze to remove explicit zeros + val breezeTransposed = asBreeze.asInstanceOf[BSM[Double]].t + Matrices.fromBreeze(breezeTransposed).transpose.asInstanceOf[SparseMatrix] + } else if (isRowMajor && colMajor) { + // it is row major and we want col major, use breeze to remove explicit zeros + val breezeTransposed = asBreeze.asInstanceOf[BSM[Double]] + Matrices.fromBreeze(breezeTransposed).asInstanceOf[SparseMatrix] + } else { + val nnz = numNonzeros + if (nnz != numActives) { + // remove explicit zeros + val rr = new Array[Int](nnz) + val vv = new Array[Double](nnz) + val numPtrs = if (isRowMajor) numRows else numCols + val cc = new Array[Int](numPtrs + 1) + var nzIdx = 0 + var j = 0 + while (j < numPtrs) { + var idx = colPtrs(j) + val idxEnd = colPtrs(j + 1) + cc(j) = nzIdx + while (idx < idxEnd) { + if (values(idx) != 0.0) { + vv(nzIdx) = values(idx) + rr(nzIdx) = rowIndices(idx) + nzIdx += 1 + } + idx += 1 + } + j += 1 + } + cc(j) = nnz + new SparseMatrix(numRows, numCols, cc, rr, vv, isTransposed = isTransposed) + } else { + this + } + } + } + + /** + * Generate a `DenseMatrix` from the given `SparseMatrix`. + * + * @param colMajor Whether the resulting `DenseMatrix` values are in column major order. + */ + private[ml] override def toDenseMatrix(colMajor: Boolean): DenseMatrix = { + if (colMajor) new DenseMatrix(numRows, numCols, this.toArray) + else new DenseMatrix(numRows, numCols, this.transpose.toArray, isTransposed = true) + } + + override def colIter: Iterator[Vector] = { + if (isTransposed) { + val indicesArray = Array.fill(numCols)(MArrayBuilder.make[Int]) + val valuesArray = Array.fill(numCols)(MArrayBuilder.make[Double]) + var i = 0 + while (i < numRows) { + var k = colPtrs(i) + val rowEnd = colPtrs(i + 1) + while (k < rowEnd) { + val j = rowIndices(k) + indicesArray(j) += i + valuesArray(j) += values(k) + k += 1 + } + i += 1 + } + Iterator.tabulate(numCols) { j => + val ii = indicesArray(j).result() + val vv = valuesArray(j).result() + new SparseVector(numRows, ii, vv) + } + } else { + Iterator.tabulate(numCols) { j => + val colStart = colPtrs(j) + val colEnd = colPtrs(j + 1) + val ii = rowIndices.slice(colStart, colEnd) + val vv = values.slice(colStart, colEnd) + new SparseVector(numRows, ii, vv) + } + } + } + + private[ml] def getSizeInBytes: Long = Matrices.getSparseSize(numActives, colPtrs.length) +} + +/** + * Factory methods for [[org.apache.spark.ml.linalg.SparseMatrix]]. + */ +@Since("2.0.0") +object SparseMatrix { + + /** + * Generate a `SparseMatrix` from Coordinate List (COO) format. Input must be an array of + * (i, j, value) tuples. Entries that have duplicate values of i and j are + * added together. Tuples where value is equal to zero will be omitted. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param entries Array of (i, j, value) tuples + * @return The corresponding `SparseMatrix` + */ + @Since("2.0.0") + def fromCOO(numRows: Int, numCols: Int, entries: Iterable[(Int, Int, Double)]): SparseMatrix = { + val sortedEntries = entries.toSeq.sortBy(v => (v._2, v._1)) + val numEntries = sortedEntries.size + if (sortedEntries.nonEmpty) { + // Since the entries are sorted by column index, we only need to check the first and the last. + for (col <- Seq(sortedEntries.head._2, sortedEntries.last._2)) { + require(col >= 0 && col < numCols, s"Column index out of range [0, $numCols): $col.") + } + } + val colPtrs = new Array[Int](numCols + 1) + val rowIndices = MArrayBuilder.make[Int] + rowIndices.sizeHint(numEntries) + val values = MArrayBuilder.make[Double] + values.sizeHint(numEntries) + var nnz = 0 + var prevCol = 0 + var prevRow = -1 + var prevVal = 0.0 + // Append a dummy entry to include the last one at the end of the loop. + (sortedEntries.view :+ ((numRows, numCols, 1.0))).foreach { case (i, j, v) => + if (v != 0) { + if (i == prevRow && j == prevCol) { + prevVal += v + } else { + if (prevVal != 0) { + require(prevRow >= 0 && prevRow < numRows, + s"Row index out of range [0, $numRows): $prevRow.") + nnz += 1 + rowIndices += prevRow + values += prevVal + } + prevRow = i + prevVal = v + while (prevCol < j) { + colPtrs(prevCol + 1) = nnz + prevCol += 1 + } + } + } + } + new SparseMatrix(numRows, numCols, colPtrs, rowIndices.result(), values.result()) + } + + /** + * Generate an Identity Matrix in `SparseMatrix` format. + * @param n number of rows and columns of the matrix + * @return `SparseMatrix` with size `n` x `n` and values of ones on the diagonal + */ + @Since("2.0.0") + def speye(n: Int): SparseMatrix = { + new SparseMatrix(n, n, (0 to n).toArray, (0 until n).toArray, Array.fill(n)(1.0)) + } + + /** + * Generates the skeleton of a random `SparseMatrix` with a given random number generator. + * The values of the matrix returned are undefined. + */ + private def genRandMatrix( + numRows: Int, + numCols: Int, + density: Double, + rng: Random): SparseMatrix = { + require(numRows > 0, s"numRows must be greater than 0 but got $numRows") + require(numCols > 0, s"numCols must be greater than 0 but got $numCols") + require(density >= 0.0 && density <= 1.0, + s"density must be a double in the range 0.0 <= d <= 1.0. Currently, density: $density") + val size = numRows.toLong * numCols + val expected = size * density + assert(expected < Int.MaxValue, + "The expected number of nonzeros cannot be greater than Int.MaxValue.") + val nnz = math.ceil(expected).toInt + if (density == 0.0) { + new SparseMatrix(numRows, numCols, new Array[Int](numCols + 1), Array.empty, Array.empty) + } else if (density == 1.0) { + val colPtrs = Array.tabulate(numCols + 1)(j => j * numRows) + val rowIndices = Array.tabulate(size.toInt)(idx => idx % numRows) + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, new Array[Double](numRows * numCols)) + } else if (density < 0.34) { + // draw-by-draw, expected number of iterations is less than 1.5 * nnz + val entries = MHashSet[(Int, Int)]() + while (entries.size < nnz) { + entries += ((rng.nextInt(numRows), rng.nextInt(numCols))) + } + SparseMatrix.fromCOO(numRows, numCols, entries.map(v => (v._1, v._2, 1.0))) + } else { + // selection-rejection method + var idx = 0L + var numSelected = 0 + var j = 0 + val colPtrs = new Array[Int](numCols + 1) + val rowIndices = new Array[Int](nnz) + while (j < numCols && numSelected < nnz) { + var i = 0 + while (i < numRows && numSelected < nnz) { + if (rng.nextDouble() < 1.0 * (nnz - numSelected) / (size - idx)) { + rowIndices(numSelected) = i + numSelected += 1 + } + i += 1 + idx += 1 + } + colPtrs(j + 1) = numSelected + j += 1 + } + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, new Array[Double](nnz)) + } + } + + /** + * Generate a `SparseMatrix` consisting of `i.i.d`. uniform random numbers. The number of non-zero + * elements equal the ceiling of `numRows` x `numCols` x `density` + * + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param density the desired density for the matrix + * @param rng a random number generator + * @return `SparseMatrix` with size `numRows` x `numCols` and values in U(0, 1) + */ + @Since("2.0.0") + def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = { + val mat = genRandMatrix(numRows, numCols, density, rng) + mat.update(i => rng.nextDouble()) + } + + /** + * Generate a `SparseMatrix` consisting of `i.i.d`. gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param density the desired density for the matrix + * @param rng a random number generator + * @return `SparseMatrix` with size `numRows` x `numCols` and values in N(0, 1) + */ + @Since("2.0.0") + def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = { + val mat = genRandMatrix(numRows, numCols, density, rng) + mat.update(i => rng.nextGaussian()) + } + + /** + * Generate a diagonal matrix in `SparseMatrix` format from the supplied values. + * @param vector a `Vector` that will form the values on the diagonal of the matrix + * @return Square `SparseMatrix` with size `values.length` x `values.length` and non-zero + * `values` on the diagonal + */ + @Since("2.0.0") + def spdiag(vector: Vector): SparseMatrix = { + val n = vector.size + vector match { + case sVec: SparseVector => + SparseMatrix.fromCOO(n, n, sVec.indices.zip(sVec.values).map(v => (v._1, v._1, v._2))) + case dVec: DenseVector => + val entries = dVec.values.zipWithIndex + val nnzVals = entries.filter(v => v._1 != 0.0) + SparseMatrix.fromCOO(n, n, nnzVals.map(v => (v._2, v._2, v._1))) + } + } +} + +/** + * Factory methods for [[org.apache.spark.ml.linalg.Matrix]]. + */ +@Since("2.0.0") +object Matrices { + + /** + * Creates a column-major dense matrix. + * + * @param numRows number of rows + * @param numCols number of columns + * @param values matrix entries in column major + */ + @Since("2.0.0") + def dense(numRows: Int, numCols: Int, values: Array[Double]): Matrix = { + new DenseMatrix(numRows, numCols, values) + } + + /** + * Creates a column-major sparse matrix in Compressed Sparse Column (CSC) format. + * + * @param numRows number of rows + * @param numCols number of columns + * @param colPtrs the index corresponding to the start of a new column + * @param rowIndices the row index of the entry + * @param values non-zero matrix entries in column major + */ + @Since("2.0.0") + def sparse( + numRows: Int, + numCols: Int, + colPtrs: Array[Int], + rowIndices: Array[Int], + values: Array[Double]): Matrix = { + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values) + } + + /** + * Creates a Matrix instance from a breeze matrix. + * @param breeze a breeze matrix + * @return a Matrix instance + */ + private[ml] def fromBreeze(breeze: BM[Double]): Matrix = { + breeze match { + case dm: BDM[Double] => + new DenseMatrix(dm.rows, dm.cols, dm.data, dm.isTranspose) + case sm: BSM[Double] => + // There is no isTranspose flag for sparse matrices in Breeze + new SparseMatrix(sm.rows, sm.cols, sm.colPtrs, sm.rowIndices, sm.data) + case _ => + throw new UnsupportedOperationException( + s"Do not support conversion from type ${breeze.getClass.getName}.") + } + } + + /** + * Generate a `Matrix` consisting of zeros. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `Matrix` with size `numRows` x `numCols` and values of zeros + */ + @Since("2.0.0") + def zeros(numRows: Int, numCols: Int): Matrix = DenseMatrix.zeros(numRows, numCols) + + /** + * Generate a `DenseMatrix` consisting of ones. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `Matrix` with size `numRows` x `numCols` and values of ones + */ + @Since("2.0.0") + def ones(numRows: Int, numCols: Int): Matrix = DenseMatrix.ones(numRows, numCols) + + /** + * Generate a dense Identity Matrix in `Matrix` format. + * @param n number of rows and columns of the matrix + * @return `Matrix` with size `n` x `n` and values of ones on the diagonal + */ + @Since("2.0.0") + def eye(n: Int): Matrix = DenseMatrix.eye(n) + + /** + * Generate a sparse Identity Matrix in `Matrix` format. + * @param n number of rows and columns of the matrix + * @return `Matrix` with size `n` x `n` and values of ones on the diagonal + */ + @Since("2.0.0") + def speye(n: Int): Matrix = SparseMatrix.speye(n) + + /** + * Generate a `DenseMatrix` consisting of `i.i.d.` uniform random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param rng a random number generator + * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1) + */ + @Since("2.0.0") + def rand(numRows: Int, numCols: Int, rng: Random): Matrix = + DenseMatrix.rand(numRows, numCols, rng) + + /** + * Generate a `SparseMatrix` consisting of `i.i.d.` gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param density the desired density for the matrix + * @param rng a random number generator + * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1) + */ + @Since("2.0.0") + def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix = + SparseMatrix.sprand(numRows, numCols, density, rng) + + /** + * Generate a `DenseMatrix` consisting of `i.i.d.` gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param rng a random number generator + * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1) + */ + @Since("2.0.0") + def randn(numRows: Int, numCols: Int, rng: Random): Matrix = + DenseMatrix.randn(numRows, numCols, rng) + + /** + * Generate a `SparseMatrix` consisting of `i.i.d.` gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param density the desired density for the matrix + * @param rng a random number generator + * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1) + */ + @Since("2.0.0") + def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix = + SparseMatrix.sprandn(numRows, numCols, density, rng) + + /** + * Generate a diagonal matrix in `Matrix` format from the supplied values. + * @param vector a `Vector` that will form the values on the diagonal of the matrix + * @return Square `Matrix` with size `values.length` x `values.length` and `values` + * on the diagonal + */ + @Since("2.0.0") + def diag(vector: Vector): Matrix = DenseMatrix.diag(vector) + + /** + * Horizontally concatenate a sequence of matrices. The returned matrix will be in the format + * the matrices are supplied in. Supplying a mix of dense and sparse matrices will result in + * a sparse matrix. If the Array is empty, an empty `DenseMatrix` will be returned. + * @param matrices array of matrices + * @return a single `Matrix` composed of the matrices that were horizontally concatenated + */ + @Since("2.0.0") + def horzcat(matrices: Array[Matrix]): Matrix = { + if (matrices.isEmpty) { + return new DenseMatrix(0, 0, Array.empty) + } else if (matrices.length == 1) { + return matrices(0) + } + val numRows = matrices(0).numRows + var hasSparse = false + var numCols = 0 + matrices.foreach { mat => + require(numRows == mat.numRows, "The number of rows of the matrices in this sequence, " + + "don't match!") + mat match { + case sparse: SparseMatrix => hasSparse = true + case dense: DenseMatrix => // empty on purpose + case _ => throw new IllegalArgumentException("Unsupported matrix format. Expected " + + s"SparseMatrix or DenseMatrix. Instead got: ${mat.getClass}") + } + numCols += mat.numCols + } + if (!hasSparse) { + new DenseMatrix(numRows, numCols, matrices.flatMap(_.toArray)) + } else { + var startCol = 0 + val entries: Array[(Int, Int, Double)] = matrices.flatMap { mat => + val nCols = mat.numCols + mat match { + case spMat: SparseMatrix => + val data = new Array[(Int, Int, Double)](spMat.values.length) + var cnt = 0 + spMat.foreachActive { (i, j, v) => + data(cnt) = (i, j + startCol, v) + cnt += 1 + } + startCol += nCols + data + case dnMat: DenseMatrix => + val data = new ArrayBuffer[(Int, Int, Double)]() + dnMat.foreachActive { (i, j, v) => + if (v != 0.0) { + data += Tuple3(i, j + startCol, v) + } + } + startCol += nCols + data + } + } + SparseMatrix.fromCOO(numRows, numCols, entries) + } + } + + /** + * Vertically concatenate a sequence of matrices. The returned matrix will be in the format + * the matrices are supplied in. Supplying a mix of dense and sparse matrices will result in + * a sparse matrix. If the Array is empty, an empty `DenseMatrix` will be returned. + * @param matrices array of matrices + * @return a single `Matrix` composed of the matrices that were vertically concatenated + */ + @Since("2.0.0") + def vertcat(matrices: Array[Matrix]): Matrix = { + if (matrices.isEmpty) { + return new DenseMatrix(0, 0, Array.empty) + } else if (matrices.length == 1) { + return matrices(0) + } + val numCols = matrices(0).numCols + var hasSparse = false + var numRows = 0 + matrices.foreach { mat => + require(numCols == mat.numCols, "The number of rows of the matrices in this sequence, " + + "don't match!") + mat match { + case sparse: SparseMatrix => hasSparse = true + case dense: DenseMatrix => // empty on purpose + case _ => throw new IllegalArgumentException("Unsupported matrix format. Expected " + + s"SparseMatrix or DenseMatrix. Instead got: ${mat.getClass}") + } + numRows += mat.numRows + } + if (!hasSparse) { + val allValues = new Array[Double](numRows * numCols) + var startRow = 0 + matrices.foreach { mat => + var j = 0 + val nRows = mat.numRows + mat.foreachActive { (i, j, v) => + val indStart = j * numRows + startRow + allValues(indStart + i) = v + } + startRow += nRows + } + new DenseMatrix(numRows, numCols, allValues) + } else { + var startRow = 0 + val entries: Array[(Int, Int, Double)] = matrices.flatMap { mat => + val nRows = mat.numRows + mat match { + case spMat: SparseMatrix => + val data = new Array[(Int, Int, Double)](spMat.values.length) + var cnt = 0 + spMat.foreachActive { (i, j, v) => + data(cnt) = (i + startRow, j, v) + cnt += 1 + } + startRow += nRows + data + case dnMat: DenseMatrix => + val data = new ArrayBuffer[(Int, Int, Double)]() + dnMat.foreachActive { (i, j, v) => + if (v != 0.0) { + data += Tuple3(i + startRow, j, v) + } + } + startRow += nRows + data + } + } + SparseMatrix.fromCOO(numRows, numCols, entries) + } + } + + private[ml] def getSparseSize(numActives: Long, numPtrs: Long): Long = { + /* + Sparse matrices store two int arrays, one double array, two ints, and one boolean: + 8 * values.length + 4 * rowIndices.length + 4 * colPtrs.length + arrayHeader * 3 + 2 * 4 + 1 + */ + val doubleBytes = java.lang.Double.BYTES + val intBytes = java.lang.Integer.BYTES + val arrayHeader = 12L + doubleBytes * numActives + intBytes * numActives + intBytes * numPtrs + arrayHeader * 3L + 9L + } + + private[ml] def getDenseSize(numCols: Long, numRows: Long): Long = { + /* + Dense matrices store one double array, two ints, and one boolean: + 8 * values.length + arrayHeader + 2 * 4 + 1 + */ + val doubleBytes = java.lang.Double.BYTES + val arrayHeader = 12L + doubleBytes * numCols * numRows + arrayHeader + 9L + } + +} diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala new file mode 100644 index 000000000000..941b6eca568d --- /dev/null +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala @@ -0,0 +1,746 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import java.lang.{Double => JavaDouble, Integer => JavaInteger, Iterable => JavaIterable} +import java.util + +import scala.annotation.varargs +import scala.collection.JavaConverters._ + +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} + +import org.apache.spark.annotation.Since + +/** + * Represents a numeric vector, whose index type is Int and value type is Double. + * + * @note Users should not implement this interface. + */ +@Since("2.0.0") +sealed trait Vector extends Serializable { + + /** + * Size of the vector. + */ + @Since("2.0.0") + def size: Int + + /** + * Converts the instance to a double array. + */ + @Since("2.0.0") + def toArray: Array[Double] + + override def equals(other: Any): Boolean = { + other match { + case v2: Vector => + if (this.size != v2.size) return false + (this, v2) match { + case (s1: SparseVector, s2: SparseVector) => + Vectors.equals(s1.indices, s1.values, s2.indices, s2.values) + case (s1: SparseVector, d1: DenseVector) => + Vectors.equals(s1.indices, s1.values, 0 until d1.size, d1.values) + case (d1: DenseVector, s1: SparseVector) => + Vectors.equals(0 until d1.size, d1.values, s1.indices, s1.values) + case (_, _) => util.Arrays.equals(this.toArray, v2.toArray) + } + case _ => false + } + } + + /** + * Returns a hash code value for the vector. The hash code is based on its size and its first 128 + * nonzero entries, using a hash algorithm similar to `java.util.Arrays.hashCode`. + */ + override def hashCode(): Int = { + // This is a reference implementation. It calls return in foreachActive, which is slow. + // Subclasses should override it with optimized implementation. + var result: Int = 31 + size + var nnz = 0 + this.foreachActive { (index, value) => + if (nnz < Vectors.MAX_HASH_NNZ) { + // ignore explicit 0 for comparison between sparse and dense + if (value != 0) { + result = 31 * result + index + val bits = java.lang.Double.doubleToLongBits(value) + result = 31 * result + (bits ^ (bits >>> 32)).toInt + nnz += 1 + } + } else { + return result + } + } + result + } + + /** + * Converts the instance to a breeze vector. + */ + private[spark] def asBreeze: BV[Double] + + /** + * Gets the value of the ith element. + * @param i index + */ + @Since("2.0.0") + def apply(i: Int): Double = asBreeze(i) + + /** + * Makes a deep copy of this vector. + */ + @Since("2.0.0") + def copy: Vector = { + throw new NotImplementedError(s"copy is not implemented for ${this.getClass}.") + } + + /** + * Applies a function `f` to all the active elements of dense and sparse vector. + * + * @param f the function takes two parameters where the first parameter is the index of + * the vector with type `Int`, and the second parameter is the corresponding value + * with type `Double`. + */ + @Since("2.0.0") + def foreachActive(f: (Int, Double) => Unit): Unit + + /** + * Number of active entries. An "active entry" is an element which is explicitly stored, + * regardless of its value. Note that inactive entries have value 0. + */ + @Since("2.0.0") + def numActives: Int + + /** + * Number of nonzero elements. This scans all active values and count nonzeros. + */ + @Since("2.0.0") + def numNonzeros: Int + + /** + * Converts this vector to a sparse vector with all explicit zeros removed. + */ + @Since("2.0.0") + def toSparse: SparseVector = toSparseWithSize(numNonzeros) + + /** + * Converts this vector to a sparse vector with all explicit zeros removed when the size is known. + * This method is used to avoid re-computing the number of non-zero elements when it is + * already known. This method should only be called after computing the number of non-zero + * elements via [[numNonzeros]]. e.g. + * {{{ + * val nnz = numNonzeros + * val sv = toSparse(nnz) + * }}} + * + * If `nnz` is under-specified, a [[java.lang.ArrayIndexOutOfBoundsException]] is thrown. + */ + private[linalg] def toSparseWithSize(nnz: Int): SparseVector + + /** + * Converts this vector to a dense vector. + */ + @Since("2.0.0") + def toDense: DenseVector = new DenseVector(this.toArray) + + /** + * Returns a vector in either dense or sparse format, whichever uses less storage. + */ + @Since("2.0.0") + def compressed: Vector = { + val nnz = numNonzeros + // A dense vector needs 8 * size + 8 bytes, while a sparse vector needs 12 * nnz + 20 bytes. + if (1.5 * (nnz + 1.0) < size) { + toSparseWithSize(nnz) + } else { + toDense + } + } + + /** + * Find the index of a maximal element. Returns the first maximal element in case of a tie. + * Returns -1 if vector has length 0. + */ + @Since("2.0.0") + def argmax: Int +} + +/** + * Factory methods for [[org.apache.spark.ml.linalg.Vector]]. + * We don't use the name `Vector` because Scala imports + * `scala.collection.immutable.Vector` by default. + */ +@Since("2.0.0") +object Vectors { + + /** + * Creates a dense vector from its values. + */ + @varargs + @Since("2.0.0") + def dense(firstValue: Double, otherValues: Double*): Vector = + new DenseVector((firstValue +: otherValues).toArray) + + // A dummy implicit is used to avoid signature collision with the one generated by @varargs. + /** + * Creates a dense vector from a double array. + */ + @Since("2.0.0") + def dense(values: Array[Double]): Vector = new DenseVector(values) + + /** + * Creates a sparse vector providing its index array and value array. + * + * @param size vector size. + * @param indices index array, must be strictly increasing. + * @param values value array, must have the same length as indices. + */ + @Since("2.0.0") + def sparse(size: Int, indices: Array[Int], values: Array[Double]): Vector = + new SparseVector(size, indices, values) + + /** + * Creates a sparse vector using unordered (index, value) pairs. + * + * @param size vector size. + * @param elements vector elements in (index, value) pairs. + */ + @Since("2.0.0") + def sparse(size: Int, elements: Seq[(Int, Double)]): Vector = { + val (indices, values) = elements.sortBy(_._1).unzip + new SparseVector(size, indices.toArray, values.toArray) + } + + /** + * Creates a sparse vector using unordered (index, value) pairs in a Java friendly way. + * + * @param size vector size. + * @param elements vector elements in (index, value) pairs. + */ + @Since("2.0.0") + def sparse(size: Int, elements: JavaIterable[(JavaInteger, JavaDouble)]): Vector = { + sparse(size, elements.asScala.map { case (i, x) => + (i.intValue(), x.doubleValue()) + }.toSeq) + } + + /** + * Creates a vector of all zeros. + * + * @param size vector size + * @return a zero vector + */ + @Since("2.0.0") + def zeros(size: Int): Vector = { + new DenseVector(new Array[Double](size)) + } + + /** + * Creates a vector instance from a breeze vector. + */ + private[spark] def fromBreeze(breezeVector: BV[Double]): Vector = { + breezeVector match { + case v: BDV[Double] => + if (v.offset == 0 && v.stride == 1 && v.length == v.data.length) { + new DenseVector(v.data) + } else { + new DenseVector(v.toArray) // Can't use underlying array directly, so make a new one + } + case v: BSV[Double] => + if (v.index.length == v.used) { + new SparseVector(v.length, v.index, v.data) + } else { + new SparseVector(v.length, v.index.slice(0, v.used), v.data.slice(0, v.used)) + } + case v: BV[_] => + sys.error("Unsupported Breeze vector type: " + v.getClass.getName) + } + } + + /** + * Returns the p-norm of this vector. + * @param vector input vector. + * @param p norm. + * @return norm in L^p^ space. + */ + @Since("2.0.0") + def norm(vector: Vector, p: Double): Double = { + require(p >= 1.0, "To compute the p-norm of the vector, we require that you specify a p>=1. " + + s"You specified p=$p.") + val values = vector match { + case DenseVector(vs) => vs + case SparseVector(n, ids, vs) => vs + case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + } + val size = values.length + + if (p == 1) { + var sum = 0.0 + var i = 0 + while (i < size) { + sum += math.abs(values(i)) + i += 1 + } + sum + } else if (p == 2) { + var sum = 0.0 + var i = 0 + while (i < size) { + sum += values(i) * values(i) + i += 1 + } + math.sqrt(sum) + } else if (p == Double.PositiveInfinity) { + var max = 0.0 + var i = 0 + while (i < size) { + val value = math.abs(values(i)) + if (value > max) max = value + i += 1 + } + max + } else { + var sum = 0.0 + var i = 0 + while (i < size) { + sum += math.pow(math.abs(values(i)), p) + i += 1 + } + math.pow(sum, 1.0 / p) + } + } + + /** + * Returns the squared distance between two Vectors. + * @param v1 first Vector. + * @param v2 second Vector. + * @return squared distance between two Vectors. + */ + @Since("2.0.0") + def sqdist(v1: Vector, v2: Vector): Double = { + require(v1.size == v2.size, s"Vector dimensions do not match: Dim(v1)=${v1.size} and Dim(v2)" + + s"=${v2.size}.") + var squaredDistance = 0.0 + (v1, v2) match { + case (v1: SparseVector, v2: SparseVector) => + val v1Values = v1.values + val v1Indices = v1.indices + val v2Values = v2.values + val v2Indices = v2.indices + val nnzv1 = v1Indices.length + val nnzv2 = v2Indices.length + + var kv1 = 0 + var kv2 = 0 + while (kv1 < nnzv1 || kv2 < nnzv2) { + var score = 0.0 + + if (kv2 >= nnzv2 || (kv1 < nnzv1 && v1Indices(kv1) < v2Indices(kv2))) { + score = v1Values(kv1) + kv1 += 1 + } else if (kv1 >= nnzv1 || (kv2 < nnzv2 && v2Indices(kv2) < v1Indices(kv1))) { + score = v2Values(kv2) + kv2 += 1 + } else { + score = v1Values(kv1) - v2Values(kv2) + kv1 += 1 + kv2 += 1 + } + squaredDistance += score * score + } + + case (v1: SparseVector, v2: DenseVector) => + squaredDistance = sqdist(v1, v2) + + case (v1: DenseVector, v2: SparseVector) => + squaredDistance = sqdist(v2, v1) + + case (DenseVector(vv1), DenseVector(vv2)) => + var kv = 0 + val sz = vv1.length + while (kv < sz) { + val score = vv1(kv) - vv2(kv) + squaredDistance += score * score + kv += 1 + } + case _ => + throw new IllegalArgumentException("Do not support vector type " + v1.getClass + + " and " + v2.getClass) + } + squaredDistance + } + + /** + * Returns the squared distance between DenseVector and SparseVector. + */ + private[ml] def sqdist(v1: SparseVector, v2: DenseVector): Double = { + var kv1 = 0 + var kv2 = 0 + val indices = v1.indices + var squaredDistance = 0.0 + val nnzv1 = indices.length + val nnzv2 = v2.size + var iv1 = if (nnzv1 > 0) indices(kv1) else -1 + + while (kv2 < nnzv2) { + var score = 0.0 + if (kv2 != iv1) { + score = v2(kv2) + } else { + score = v1.values(kv1) - v2(kv2) + if (kv1 < nnzv1 - 1) { + kv1 += 1 + iv1 = indices(kv1) + } + } + squaredDistance += score * score + kv2 += 1 + } + squaredDistance + } + + /** + * Check equality between sparse/dense vectors + */ + private[ml] def equals( + v1Indices: IndexedSeq[Int], + v1Values: Array[Double], + v2Indices: IndexedSeq[Int], + v2Values: Array[Double]): Boolean = { + val v1Size = v1Values.length + val v2Size = v2Values.length + var k1 = 0 + var k2 = 0 + var allEqual = true + while (allEqual) { + while (k1 < v1Size && v1Values(k1) == 0) k1 += 1 + while (k2 < v2Size && v2Values(k2) == 0) k2 += 1 + + if (k1 >= v1Size || k2 >= v2Size) { + return k1 >= v1Size && k2 >= v2Size // check end alignment + } + allEqual = v1Indices(k1) == v2Indices(k2) && v1Values(k1) == v2Values(k2) + k1 += 1 + k2 += 1 + } + allEqual + } + + /** Max number of nonzero entries used in computing hash code. */ + private[linalg] val MAX_HASH_NNZ = 128 +} + +/** + * A dense vector represented by a value array. + */ +@Since("2.0.0") +class DenseVector @Since("2.0.0") ( @Since("2.0.0") val values: Array[Double]) extends Vector { + + override def size: Int = values.length + + override def toString: String = values.mkString("[", ",", "]") + + override def toArray: Array[Double] = values + + private[spark] override def asBreeze: BV[Double] = new BDV[Double](values) + + override def apply(i: Int): Double = values(i) + + override def copy: DenseVector = { + new DenseVector(values.clone()) + } + + override def foreachActive(f: (Int, Double) => Unit): Unit = { + var i = 0 + val localValuesSize = values.length + val localValues = values + + while (i < localValuesSize) { + f(i, localValues(i)) + i += 1 + } + } + + override def equals(other: Any): Boolean = super.equals(other) + + override def hashCode(): Int = { + var result: Int = 31 + size + var i = 0 + val end = values.length + var nnz = 0 + while (i < end && nnz < Vectors.MAX_HASH_NNZ) { + val v = values(i) + if (v != 0.0) { + result = 31 * result + i + val bits = java.lang.Double.doubleToLongBits(values(i)) + result = 31 * result + (bits ^ (bits >>> 32)).toInt + nnz += 1 + } + i += 1 + } + result + } + + override def numActives: Int = size + + override def numNonzeros: Int = { + // same as values.count(_ != 0.0) but faster + var nnz = 0 + values.foreach { v => + if (v != 0.0) { + nnz += 1 + } + } + nnz + } + + private[linalg] override def toSparseWithSize(nnz: Int): SparseVector = { + val ii = new Array[Int](nnz) + val vv = new Array[Double](nnz) + var k = 0 + foreachActive { (i, v) => + if (v != 0) { + ii(k) = i + vv(k) = v + k += 1 + } + } + new SparseVector(size, ii, vv) + } + + override def argmax: Int = { + if (size == 0) { + -1 + } else { + var maxIdx = 0 + var maxValue = values(0) + var i = 1 + while (i < size) { + if (values(i) > maxValue) { + maxIdx = i + maxValue = values(i) + } + i += 1 + } + maxIdx + } + } +} + +@Since("2.0.0") +object DenseVector { + + /** Extracts the value array from a dense vector. */ + @Since("2.0.0") + def unapply(dv: DenseVector): Option[Array[Double]] = Some(dv.values) +} + +/** + * A sparse vector represented by an index array and a value array. + * + * @param size size of the vector. + * @param indices index array, assume to be strictly increasing. + * @param values value array, must have the same length as the index array. + */ +@Since("2.0.0") +class SparseVector @Since("2.0.0") ( + override val size: Int, + @Since("2.0.0") val indices: Array[Int], + @Since("2.0.0") val values: Array[Double]) extends Vector { + + // validate the data + { + require(size >= 0, "The size of the requested sparse vector must be greater than 0.") + require(indices.length == values.length, "Sparse vectors require that the dimension of the" + + s" indices match the dimension of the values. You provided ${indices.length} indices and " + + s" ${values.length} values.") + require(indices.length <= size, s"You provided ${indices.length} indices and values, " + + s"which exceeds the specified vector size ${size}.") + + if (indices.nonEmpty) { + require(indices(0) >= 0, s"Found negative index: ${indices(0)}.") + } + var prev = -1 + indices.foreach { i => + require(prev < i, s"Index $i follows $prev and is not strictly increasing") + prev = i + } + require(prev < size, s"Index $prev out of bounds for vector of size $size") + } + + override def toString: String = + s"($size,${indices.mkString("[", ",", "]")},${values.mkString("[", ",", "]")})" + + override def toArray: Array[Double] = { + val data = new Array[Double](size) + var i = 0 + val nnz = indices.length + while (i < nnz) { + data(indices(i)) = values(i) + i += 1 + } + data + } + + override def copy: SparseVector = { + new SparseVector(size, indices.clone(), values.clone()) + } + + private[spark] override def asBreeze: BV[Double] = new BSV[Double](indices, values, size) + + override def foreachActive(f: (Int, Double) => Unit): Unit = { + var i = 0 + val localValuesSize = values.length + val localIndices = indices + val localValues = values + + while (i < localValuesSize) { + f(localIndices(i), localValues(i)) + i += 1 + } + } + + override def equals(other: Any): Boolean = super.equals(other) + + override def hashCode(): Int = { + var result: Int = 31 + size + val end = values.length + var k = 0 + var nnz = 0 + while (k < end && nnz < Vectors.MAX_HASH_NNZ) { + val v = values(k) + if (v != 0.0) { + val i = indices(k) + result = 31 * result + i + val bits = java.lang.Double.doubleToLongBits(v) + result = 31 * result + (bits ^ (bits >>> 32)).toInt + nnz += 1 + } + k += 1 + } + result + } + + override def numActives: Int = values.length + + override def numNonzeros: Int = { + var nnz = 0 + values.foreach { v => + if (v != 0.0) { + nnz += 1 + } + } + nnz + } + + private[linalg] override def toSparseWithSize(nnz: Int): SparseVector = { + if (nnz == numActives) { + this + } else { + val ii = new Array[Int](nnz) + val vv = new Array[Double](nnz) + var k = 0 + foreachActive { (i, v) => + if (v != 0.0) { + ii(k) = i + vv(k) = v + k += 1 + } + } + new SparseVector(size, ii, vv) + } + } + + override def argmax: Int = { + if (size == 0) { + -1 + } else if (numActives == 0) { + 0 + } else { + // Find the max active entry. + var maxIdx = indices(0) + var maxValue = values(0) + var maxJ = 0 + var j = 1 + val na = numActives + while (j < na) { + val v = values(j) + if (v > maxValue) { + maxValue = v + maxIdx = indices(j) + maxJ = j + } + j += 1 + } + + // If the max active entry is nonpositive and there exists inactive ones, find the first zero. + if (maxValue <= 0.0 && na < size) { + if (maxValue == 0.0) { + // If there exists an inactive entry before maxIdx, find it and return its index. + if (maxJ < maxIdx) { + var k = 0 + while (k < maxJ && indices(k) == k) { + k += 1 + } + maxIdx = k + } + } else { + // If the max active value is negative, find and return the first inactive index. + var k = 0 + while (k < na && indices(k) == k) { + k += 1 + } + maxIdx = k + } + } + + maxIdx + } + } + + /** + * Create a slice of this vector based on the given indices. + * @param selectedIndices Unsorted list of indices into the vector. + * This does NOT do bound checking. + * @return New SparseVector with values in the order specified by the given indices. + * + * NOTE: The API needs to be discussed before making this public. + * Also, if we have a version assuming indices are sorted, we should optimize it. + */ + private[spark] def slice(selectedIndices: Array[Int]): SparseVector = { + var currentIdx = 0 + val (sliceInds, sliceVals) = selectedIndices.flatMap { origIdx => + val iIdx = java.util.Arrays.binarySearch(this.indices, origIdx) + val i_v = if (iIdx >= 0) { + Iterator((currentIdx, this.values(iIdx))) + } else { + Iterator() + } + currentIdx += 1 + i_v + }.unzip + new SparseVector(selectedIndices.length, sliceInds.toArray, sliceVals.toArray) + } +} + +@Since("2.0.0") +object SparseVector { + @Since("2.0.0") + def unapply(sv: SparseVector): Option[(Int, Array[Int], Array[Double])] = + Some((sv.size, sv.indices, sv.values)) +} diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala b/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala new file mode 100644 index 000000000000..3167e0c286d4 --- /dev/null +++ b/mllib-local/src/main/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussian.scala @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.stat.distribution + +import breeze.linalg.{diag, eigSym, max, DenseMatrix => BDM, DenseVector => BDV, Vector => BV} + +import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.ml.impl.Utils +import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} + + +/** + * This class provides basic functionality for a Multivariate Gaussian (Normal) Distribution. In + * the event that the covariance matrix is singular, the density will be computed in a + * reduced dimensional subspace under which the distribution is supported. + * (see + * here) + * + * @param mean The mean vector of the distribution + * @param cov The covariance matrix of the distribution + */ +@Since("2.0.0") +@DeveloperApi +class MultivariateGaussian @Since("2.0.0") ( + @Since("2.0.0") val mean: Vector, + @Since("2.0.0") val cov: Matrix) extends Serializable { + + require(cov.numCols == cov.numRows, "Covariance matrix must be square") + require(mean.size == cov.numCols, "Mean vector length must match covariance matrix size") + + /** Private constructor taking Breeze types */ + private[ml] def this(mean: BDV[Double], cov: BDM[Double]) = { + this(Vectors.fromBreeze(mean), Matrices.fromBreeze(cov)) + } + + private val breezeMu = mean.asBreeze.toDenseVector + + /** + * Compute distribution dependent constants: + * rootSigmaInv = D^(-1/2)^ * U.t, where sigma = U * D * U.t + * u = log((2*pi)^(-k/2)^ * det(sigma)^(-1/2)^) + */ + private val (rootSigmaInv: BDM[Double], u: Double) = calculateCovarianceConstants + + /** + * Returns density of this multivariate Gaussian at given point, x + */ + @Since("2.0.0") + def pdf(x: Vector): Double = { + pdf(x.asBreeze) + } + + /** + * Returns the log-density of this multivariate Gaussian at given point, x + */ + @Since("2.0.0") + def logpdf(x: Vector): Double = { + logpdf(x.asBreeze) + } + + /** Returns density of this multivariate Gaussian at given point, x */ + private[ml] def pdf(x: BV[Double]): Double = { + math.exp(logpdf(x)) + } + + /** Returns the log-density of this multivariate Gaussian at given point, x */ + private[ml] def logpdf(x: BV[Double]): Double = { + val delta = x - breezeMu + val v = rootSigmaInv * delta + u + v.t * v * -0.5 + } + + /** + * Calculate distribution dependent components used for the density function: + * pdf(x) = (2*pi)^(-k/2)^ * det(sigma)^(-1/2)^ * exp((-1/2) * (x-mu).t * inv(sigma) * (x-mu)) + * where k is length of the mean vector. + * + * We here compute distribution-fixed parts + * log((2*pi)^(-k/2)^ * det(sigma)^(-1/2)^) + * and + * D^(-1/2)^ * U, where sigma = U * D * U.t + * + * Both the determinant and the inverse can be computed from the singular value decomposition + * of sigma. Noting that covariance matrices are always symmetric and positive semi-definite, + * we can use the eigendecomposition. We also do not compute the inverse directly; noting + * that + * + * sigma = U * D * U.t + * inv(Sigma) = U * inv(D) * U.t + * = (D^{-1/2}^ * U.t).t * (D^{-1/2}^ * U.t) + * + * and thus + * + * -0.5 * (x-mu).t * inv(Sigma) * (x-mu) = -0.5 * norm(D^{-1/2}^ * U.t * (x-mu))^2^ + * + * To guard against singular covariance matrices, this method computes both the + * pseudo-determinant and the pseudo-inverse (Moore-Penrose). Singular values are considered + * to be non-zero only if they exceed a tolerance based on machine precision, matrix size, and + * relation to the maximum singular value (same tolerance used by, e.g., Octave). + */ + private def calculateCovarianceConstants: (BDM[Double], Double) = { + val eigSym.EigSym(d, u) = eigSym(cov.asBreeze.toDenseMatrix) // sigma = u * diag(d) * u.t + + // For numerical stability, values are considered to be non-zero only if they exceed tol. + // This prevents any inverted value from exceeding (eps * n * max(d))^-1 + val tol = Utils.EPSILON * max(d) * d.length + + try { + // log(pseudo-determinant) is sum of the logs of all non-zero singular values + val logPseudoDetSigma = d.activeValuesIterator.filter(_ > tol).map(math.log).sum + + // calculate the root-pseudo-inverse of the diagonal matrix of singular values + // by inverting the square root of all non-zero values + val pinvS = diag(new BDV(d.map(v => if (v > tol) math.sqrt(1.0 / v) else 0.0).toArray)) + + (pinvS * u.t, -0.5 * (mean.size * math.log(2.0 * math.Pi) + logPseudoDetSigma)) + } catch { + case uex: UnsupportedOperationException => + throw new IllegalArgumentException("Covariance matrix has no non-zero singular values") + } + } +} diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/SparkMLFunSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/SparkMLFunSuite.scala new file mode 100644 index 000000000000..cb3b56bba87b --- /dev/null +++ b/mllib-local/src/test/scala/org/apache/spark/ml/SparkMLFunSuite.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.ml + +// scalastyle:off +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +/** + * Base abstract class for all unit tests in Spark for handling common functionality. + */ +private[spark] abstract class SparkMLFunSuite + extends FunSuite + with BeforeAndAfterAll { + // scalastyle:on +} diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/impl/UtilsSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/impl/UtilsSuite.scala new file mode 100644 index 000000000000..20e7c101b565 --- /dev/null +++ b/mllib-local/src/test/scala/org/apache/spark/ml/impl/UtilsSuite.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.ml.impl + +import org.apache.spark.ml.SparkMLFunSuite +import org.apache.spark.ml.impl.Utils.EPSILON + + +class UtilsSuite extends SparkMLFunSuite { + + test("EPSILON") { + assert(1.0 + EPSILON > 1.0, s"EPSILON is too small: $EPSILON.") + assert(1.0 + EPSILON / 2.0 === 1.0, s"EPSILON is too big: $EPSILON.") + } +} diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASSuite.scala new file mode 100644 index 000000000000..877ac6898334 --- /dev/null +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASSuite.scala @@ -0,0 +1,470 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import org.apache.spark.ml.SparkMLFunSuite +import org.apache.spark.ml.linalg.BLAS._ +import org.apache.spark.ml.util.TestingUtils._ + +class BLASSuite extends SparkMLFunSuite { + + test("copy") { + val sx = Vectors.sparse(4, Array(0, 2), Array(1.0, -2.0)) + val dx = Vectors.dense(1.0, 0.0, -2.0, 0.0) + val sy = Vectors.sparse(4, Array(0, 1, 3), Array(2.0, 1.0, 1.0)) + val dy = Array(2.0, 1.0, 0.0, 1.0) + + val dy1 = Vectors.dense(dy.clone()) + copy(sx, dy1) + assert(dy1 ~== dx absTol 1e-15) + + val dy2 = Vectors.dense(dy.clone()) + copy(dx, dy2) + assert(dy2 ~== dx absTol 1e-15) + + intercept[IllegalArgumentException] { + copy(sx, sy) + } + + intercept[IllegalArgumentException] { + copy(dx, sy) + } + + withClue("vector sizes must match") { + intercept[Exception] { + copy(sx, Vectors.dense(0.0, 1.0, 2.0)) + } + } + } + + test("scal") { + val a = 0.1 + val sx = Vectors.sparse(3, Array(0, 2), Array(1.0, -2.0)) + val dx = Vectors.dense(1.0, 0.0, -2.0) + + scal(a, sx) + assert(sx ~== Vectors.sparse(3, Array(0, 2), Array(0.1, -0.2)) absTol 1e-15) + + scal(a, dx) + assert(dx ~== Vectors.dense(0.1, 0.0, -0.2) absTol 1e-15) + } + + test("axpy") { + val alpha = 0.1 + val sx = Vectors.sparse(3, Array(0, 2), Array(1.0, -2.0)) + val dx = Vectors.dense(1.0, 0.0, -2.0) + val dy = Array(2.0, 1.0, 0.0) + val expected = Vectors.dense(2.1, 1.0, -0.2) + + val dy1 = Vectors.dense(dy.clone()) + axpy(alpha, sx, dy1) + assert(dy1 ~== expected absTol 1e-15) + + val dy2 = Vectors.dense(dy.clone()) + axpy(alpha, dx, dy2) + assert(dy2 ~== expected absTol 1e-15) + + val sy = Vectors.sparse(4, Array(0, 1), Array(2.0, 1.0)) + + intercept[IllegalArgumentException] { + axpy(alpha, sx, sy) + } + + intercept[IllegalArgumentException] { + axpy(alpha, dx, sy) + } + + withClue("vector sizes must match") { + intercept[Exception] { + axpy(alpha, sx, Vectors.dense(1.0, 2.0)) + } + } + } + + test("dot") { + val sx = Vectors.sparse(3, Array(0, 2), Array(1.0, -2.0)) + val dx = Vectors.dense(1.0, 0.0, -2.0) + val sy = Vectors.sparse(3, Array(0, 1), Array(2.0, 1.0)) + val dy = Vectors.dense(2.0, 1.0, 0.0) + + assert(dot(sx, sy) ~== 2.0 absTol 1e-15) + assert(dot(sy, sx) ~== 2.0 absTol 1e-15) + assert(dot(sx, dy) ~== 2.0 absTol 1e-15) + assert(dot(dy, sx) ~== 2.0 absTol 1e-15) + assert(dot(dx, dy) ~== 2.0 absTol 1e-15) + assert(dot(dy, dx) ~== 2.0 absTol 1e-15) + + assert(dot(sx, sx) ~== 5.0 absTol 1e-15) + assert(dot(dx, dx) ~== 5.0 absTol 1e-15) + assert(dot(sx, dx) ~== 5.0 absTol 1e-15) + assert(dot(dx, sx) ~== 5.0 absTol 1e-15) + + val sx1 = Vectors.sparse(10, Array(0, 3, 5, 7, 8), Array(1.0, 2.0, 3.0, 4.0, 5.0)) + val sx2 = Vectors.sparse(10, Array(1, 3, 6, 7, 9), Array(1.0, 2.0, 3.0, 4.0, 5.0)) + assert(dot(sx1, sx2) ~== 20.0 absTol 1e-15) + assert(dot(sx2, sx1) ~== 20.0 absTol 1e-15) + + withClue("vector sizes must match") { + intercept[Exception] { + dot(sx, Vectors.dense(2.0, 1.0)) + } + } + } + + test("spr") { + // test dense vector + val alpha = 0.1 + val x = new DenseVector(Array(1.0, 2, 2.1, 4)) + val U = new DenseVector(Array(1.0, 2, 2, 3, 3, 3, 4, 4, 4, 4)) + val expected = new DenseVector(Array(1.1, 2.2, 2.4, 3.21, 3.42, 3.441, 4.4, 4.8, 4.84, 5.6)) + + spr(alpha, x, U) + assert(U ~== expected absTol 1e-9) + + val matrix33 = new DenseVector(Array(1.0, 2, 3, 4, 5)) + withClue("Size of vector must match the rank of matrix") { + intercept[Exception] { + spr(alpha, x, matrix33) + } + } + + // test sparse vector + val sv = new SparseVector(4, Array(0, 3), Array(1.0, 2)) + val U2 = new DenseVector(Array(1.0, 2, 2, 3, 3, 3, 4, 4, 4, 4)) + spr(0.1, sv, U2) + val expectedSparse = new DenseVector(Array(1.1, 2.0, 2.0, 3.0, 3.0, 3.0, 4.2, 4.0, 4.0, 4.4)) + assert(U2 ~== expectedSparse absTol 1e-15) + } + + test("syr") { + val dA = new DenseMatrix(4, 4, + Array(0.0, 1.2, 2.2, 3.1, 1.2, 3.2, 5.3, 4.6, 2.2, 5.3, 1.8, 3.0, 3.1, 4.6, 3.0, 0.8)) + val x = new DenseVector(Array(0.0, 2.7, 3.5, 2.1)) + val alpha = 0.15 + + val expected = new DenseMatrix(4, 4, + Array(0.0, 1.2, 2.2, 3.1, 1.2, 4.2935, 6.7175, 5.4505, 2.2, 6.7175, 3.6375, 4.1025, 3.1, + 5.4505, 4.1025, 1.4615)) + + syr(alpha, x, dA) + + assert(dA ~== expected absTol 1e-15) + + val dB = + new DenseMatrix(3, 4, Array(0.0, 1.2, 2.2, 3.1, 1.2, 3.2, 5.3, 4.6, 2.2, 5.3, 1.8, 3.0)) + + withClue("Matrix A must be a symmetric Matrix") { + intercept[Exception] { + syr(alpha, x, dB) + } + } + + val dC = + new DenseMatrix(3, 3, Array(0.0, 1.2, 2.2, 1.2, 3.2, 5.3, 2.2, 5.3, 1.8)) + + withClue("Size of vector must match the rank of matrix") { + intercept[Exception] { + syr(alpha, x, dC) + } + } + + val y = new DenseVector(Array(0.0, 2.7, 3.5, 2.1, 1.5)) + + withClue("Size of vector must match the rank of matrix") { + intercept[Exception] { + syr(alpha, y, dA) + } + } + + val xSparse = new SparseVector(4, Array(0, 2, 3), Array(1.0, 3.0, 4.0)) + val dD = new DenseMatrix(4, 4, + Array(0.0, 1.2, 2.2, 3.1, 1.2, 3.2, 5.3, 4.6, 2.2, 5.3, 1.8, 3.0, 3.1, 4.6, 3.0, 0.8)) + syr(0.1, xSparse, dD) + val expectedSparse = new DenseMatrix(4, 4, + Array(0.1, 1.2, 2.5, 3.5, 1.2, 3.2, 5.3, 4.6, 2.5, 5.3, 2.7, 4.2, 3.5, 4.6, 4.2, 2.4)) + assert(dD ~== expectedSparse absTol 1e-15) + } + + test("gemm") { + val dA = + new DenseMatrix(4, 3, Array(0.0, 1.0, 0.0, 0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 0.0, 3.0)) + val sA = new SparseMatrix(4, 3, Array(0, 1, 3, 4), Array(1, 0, 2, 3), Array(1.0, 2.0, 1.0, 3.0)) + + val B = new DenseMatrix(3, 2, Array(1.0, 0.0, 0.0, 0.0, 2.0, 1.0)) + val expected = new DenseMatrix(4, 2, Array(0.0, 1.0, 0.0, 0.0, 4.0, 0.0, 2.0, 3.0)) + val BTman = new DenseMatrix(2, 3, Array(1.0, 0.0, 0.0, 2.0, 0.0, 1.0)) + val BT = B.transpose + + assert(dA.multiply(B) ~== expected absTol 1e-15) + assert(sA.multiply(B) ~== expected absTol 1e-15) + + val C1 = new DenseMatrix(4, 2, Array(1.0, 0.0, 2.0, 1.0, 0.0, 0.0, 1.0, 0.0)) + val C2 = C1.copy + val C3 = C1.copy + val C4 = C1.copy + val C5 = C1.copy + val C6 = C1.copy + val C7 = C1.copy + val C8 = C1.copy + val C9 = C1.copy + val C10 = C1.copy + val C11 = C1.copy + val C12 = C1.copy + val C13 = C1.copy + val C14 = C1.copy + val C15 = C1.copy + val C16 = C1.copy + val C17 = C1.copy + val expected2 = new DenseMatrix(4, 2, Array(2.0, 1.0, 4.0, 2.0, 4.0, 0.0, 4.0, 3.0)) + val expected3 = new DenseMatrix(4, 2, Array(2.0, 2.0, 4.0, 2.0, 8.0, 0.0, 6.0, 6.0)) + val expected4 = new DenseMatrix(4, 2, Array(5.0, 0.0, 10.0, 5.0, 0.0, 0.0, 5.0, 0.0)) + val expected5 = C1.copy + + gemm(1.0, dA, B, 2.0, C1) + gemm(1.0, sA, B, 2.0, C2) + gemm(2.0, dA, B, 2.0, C3) + gemm(2.0, sA, B, 2.0, C4) + assert(C1 ~== expected2 absTol 1e-15) + assert(C2 ~== expected2 absTol 1e-15) + assert(C3 ~== expected3 absTol 1e-15) + assert(C4 ~== expected3 absTol 1e-15) + gemm(1.0, dA, B, 0.0, C17) + assert(C17 ~== expected absTol 1e-15) + gemm(1.0, sA, B, 0.0, C17) + assert(C17 ~== expected absTol 1e-15) + + withClue("columns of A don't match the rows of B") { + intercept[Exception] { + gemm(1.0, dA.transpose, B, 2.0, C1) + } + } + + val dATman = + new DenseMatrix(3, 4, Array(0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 1.0, 0.0, 0.0, 0.0, 3.0)) + val sATman = + new SparseMatrix(3, 4, Array(0, 1, 2, 3, 4), Array(1, 0, 1, 2), Array(2.0, 1.0, 1.0, 3.0)) + + val dATT = dATman.transpose + val sATT = sATman.transpose + val BTT = BTman.transpose.asInstanceOf[DenseMatrix] + + assert(dATT.multiply(B) ~== expected absTol 1e-15) + assert(sATT.multiply(B) ~== expected absTol 1e-15) + assert(dATT.multiply(BTT) ~== expected absTol 1e-15) + assert(sATT.multiply(BTT) ~== expected absTol 1e-15) + + gemm(1.0, dATT, BTT, 2.0, C5) + gemm(1.0, sATT, BTT, 2.0, C6) + gemm(2.0, dATT, BTT, 2.0, C7) + gemm(2.0, sATT, BTT, 2.0, C8) + gemm(1.0, dA, BTT, 2.0, C9) + gemm(1.0, sA, BTT, 2.0, C10) + gemm(2.0, dA, BTT, 2.0, C11) + gemm(2.0, sA, BTT, 2.0, C12) + assert(C5 ~== expected2 absTol 1e-15) + assert(C6 ~== expected2 absTol 1e-15) + assert(C7 ~== expected3 absTol 1e-15) + assert(C8 ~== expected3 absTol 1e-15) + assert(C9 ~== expected2 absTol 1e-15) + assert(C10 ~== expected2 absTol 1e-15) + assert(C11 ~== expected3 absTol 1e-15) + assert(C12 ~== expected3 absTol 1e-15) + + gemm(0, dA, B, 5, C13) + gemm(0, sA, B, 5, C14) + gemm(0, dA, B, 1, C15) + gemm(0, sA, B, 1, C16) + assert(C13 ~== expected4 absTol 1e-15) + assert(C14 ~== expected4 absTol 1e-15) + assert(C15 ~== expected5 absTol 1e-15) + assert(C16 ~== expected5 absTol 1e-15) + + } + + test("gemv") { + + val dA = + new DenseMatrix(4, 3, Array(0.0, 1.0, 0.0, 0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 0.0, 3.0)) + val sA = new SparseMatrix(4, 3, Array(0, 1, 3, 4), Array(1, 0, 2, 3), Array(1.0, 2.0, 1.0, 3.0)) + + val dA2 = + new DenseMatrix(4, 3, Array(0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 1.0, 0.0, 0.0, 0.0, 3.0), true) + val sA2 = + new SparseMatrix(4, 3, Array(0, 1, 2, 3, 4), Array(1, 0, 1, 2), Array(2.0, 1.0, 1.0, 3.0), + true) + + val dx = new DenseVector(Array(1.0, 2.0, 3.0)) + val sx = dx.toSparse + val expected = new DenseVector(Array(4.0, 1.0, 2.0, 9.0)) + + assert(dA.multiply(dx) ~== expected absTol 1e-15) + assert(sA.multiply(dx) ~== expected absTol 1e-15) + assert(dA.multiply(sx) ~== expected absTol 1e-15) + assert(sA.multiply(sx) ~== expected absTol 1e-15) + + val y1 = new DenseVector(Array(1.0, 3.0, 1.0, 0.0)) + val y2 = y1.copy + val y3 = y1.copy + val y4 = y1.copy + val y5 = y1.copy + val y6 = y1.copy + val y7 = y1.copy + val y8 = y1.copy + val y9 = y1.copy + val y10 = y1.copy + val y11 = y1.copy + val y12 = y1.copy + val y13 = y1.copy + val y14 = y1.copy + val y15 = y1.copy + val y16 = y1.copy + + val expected2 = new DenseVector(Array(6.0, 7.0, 4.0, 9.0)) + val expected3 = new DenseVector(Array(10.0, 8.0, 6.0, 18.0)) + + gemv(1.0, dA, dx, 2.0, y1) + gemv(1.0, sA, dx, 2.0, y2) + gemv(1.0, dA, sx, 2.0, y3) + gemv(1.0, sA, sx, 2.0, y4) + + gemv(1.0, dA2, dx, 2.0, y5) + gemv(1.0, sA2, dx, 2.0, y6) + gemv(1.0, dA2, sx, 2.0, y7) + gemv(1.0, sA2, sx, 2.0, y8) + + gemv(2.0, dA, dx, 2.0, y9) + gemv(2.0, sA, dx, 2.0, y10) + gemv(2.0, dA, sx, 2.0, y11) + gemv(2.0, sA, sx, 2.0, y12) + + gemv(2.0, dA2, dx, 2.0, y13) + gemv(2.0, sA2, dx, 2.0, y14) + gemv(2.0, dA2, sx, 2.0, y15) + gemv(2.0, sA2, sx, 2.0, y16) + + assert(y1 ~== expected2 absTol 1e-15) + assert(y2 ~== expected2 absTol 1e-15) + assert(y3 ~== expected2 absTol 1e-15) + assert(y4 ~== expected2 absTol 1e-15) + + assert(y5 ~== expected2 absTol 1e-15) + assert(y6 ~== expected2 absTol 1e-15) + assert(y7 ~== expected2 absTol 1e-15) + assert(y8 ~== expected2 absTol 1e-15) + + assert(y9 ~== expected3 absTol 1e-15) + assert(y10 ~== expected3 absTol 1e-15) + assert(y11 ~== expected3 absTol 1e-15) + assert(y12 ~== expected3 absTol 1e-15) + + assert(y13 ~== expected3 absTol 1e-15) + assert(y14 ~== expected3 absTol 1e-15) + assert(y15 ~== expected3 absTol 1e-15) + assert(y16 ~== expected3 absTol 1e-15) + + withClue("columns of A don't match the rows of B") { + intercept[Exception] { + gemv(1.0, dA.transpose, dx, 2.0, y1) + } + intercept[Exception] { + gemv(1.0, sA.transpose, dx, 2.0, y1) + } + intercept[Exception] { + gemv(1.0, dA.transpose, sx, 2.0, y1) + } + intercept[Exception] { + gemv(1.0, sA.transpose, sx, 2.0, y1) + } + } + + val y17 = new DenseVector(Array(0.0, 0.0)) + val y18 = y17.copy + + val sA3 = new SparseMatrix(3, 2, Array(0, 2, 4), Array(1, 2, 0, 1), Array(2.0, 1.0, 1.0, 2.0)) + .transpose + val sA4 = + new SparseMatrix(2, 3, Array(0, 1, 3, 4), Array(1, 0, 1, 0), Array(1.0, 2.0, 2.0, 1.0)) + val sx3 = new SparseVector(3, Array(1, 2), Array(2.0, 1.0)) + + val expected4 = new DenseVector(Array(5.0, 4.0)) + + gemv(1.0, sA3, sx3, 0.0, y17) + gemv(1.0, sA4, sx3, 0.0, y18) + + assert(y17 ~== expected4 absTol 1e-15) + assert(y18 ~== expected4 absTol 1e-15) + + val dAT = + new DenseMatrix(3, 4, Array(0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 1.0, 0.0, 0.0, 0.0, 3.0)) + val sAT = + new SparseMatrix(3, 4, Array(0, 1, 2, 3, 4), Array(1, 0, 1, 2), Array(2.0, 1.0, 1.0, 3.0)) + + val dATT = dAT.transpose + val sATT = sAT.transpose + + assert(dATT.multiply(dx) ~== expected absTol 1e-15) + assert(sATT.multiply(dx) ~== expected absTol 1e-15) + assert(dATT.multiply(sx) ~== expected absTol 1e-15) + assert(sATT.multiply(sx) ~== expected absTol 1e-15) + } + + test("spmv") { + /* + A = [[3.0, -2.0, 2.0, -4.0], + [-2.0, -8.0, 4.0, 7.0], + [2.0, 4.0, -3.0, -3.0], + [-4.0, 7.0, -3.0, 0.0]] + x = [5.0, 2.0, -1.0, -9.0] + Ax = [ 45., -93., 48., -3.] + */ + val A = new DenseVector(Array(3.0, -2.0, -8.0, 2.0, 4.0, -3.0, -4.0, 7.0, -3.0, 0.0)) + val x = new DenseVector(Array(5.0, 2.0, -1.0, -9.0)) + val n = 4 + + val y1 = new DenseVector(Array(-3.0, 6.0, -8.0, -3.0)) + val y2 = y1.copy + val y3 = y1.copy + val y4 = y1.copy + val y5 = y1.copy + val y6 = y1.copy + val y7 = y1.copy + + val expected1 = new DenseVector(Array(42.0, -87.0, 40.0, -6.0)) + val expected2 = new DenseVector(Array(19.5, -40.5, 16.0, -4.5)) + val expected3 = new DenseVector(Array(-25.5, 52.5, -32.0, -1.5)) + val expected4 = new DenseVector(Array(-3.0, 6.0, -8.0, -3.0)) + val expected5 = new DenseVector(Array(43.5, -90.0, 44.0, -4.5)) + val expected6 = new DenseVector(Array(46.5, -96.0, 52.0, -1.5)) + val expected7 = new DenseVector(Array(45.0, -93.0, 48.0, -3.0)) + + dspmv(n, 1.0, A, x, 1.0, y1) + dspmv(n, 0.5, A, x, 1.0, y2) + dspmv(n, -0.5, A, x, 1.0, y3) + dspmv(n, 0.0, A, x, 1.0, y4) + dspmv(n, 1.0, A, x, 0.5, y5) + dspmv(n, 1.0, A, x, -0.5, y6) + dspmv(n, 1.0, A, x, 0.0, y7) + assert(y1 ~== expected1 absTol 1e-8) + assert(y2 ~== expected2 absTol 1e-8) + assert(y3 ~== expected3 absTol 1e-8) + assert(y4 ~== expected4 absTol 1e-8) + assert(y5 ~== expected5 absTol 1e-8) + assert(y6 ~== expected6 absTol 1e-8) + assert(y7 ~== expected7 absTol 1e-8) + } +} diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BreezeMatrixConversionSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BreezeMatrixConversionSuite.scala new file mode 100644 index 000000000000..f07ed20cf0e7 --- /dev/null +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BreezeMatrixConversionSuite.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM} + +import org.apache.spark.ml.SparkMLFunSuite + +class BreezeMatrixConversionSuite extends SparkMLFunSuite { + test("dense matrix to breeze") { + val mat = Matrices.dense(3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val breeze = mat.asBreeze.asInstanceOf[BDM[Double]] + assert(breeze.rows === mat.numRows) + assert(breeze.cols === mat.numCols) + assert(breeze.data.eq(mat.asInstanceOf[DenseMatrix].values), "should not copy data") + } + + test("dense breeze matrix to matrix") { + val breeze = new BDM[Double](3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val mat = Matrices.fromBreeze(breeze).asInstanceOf[DenseMatrix] + assert(mat.numRows === breeze.rows) + assert(mat.numCols === breeze.cols) + assert(mat.values.eq(breeze.data), "should not copy data") + // transposed matrix + val matTransposed = Matrices.fromBreeze(breeze.t).asInstanceOf[DenseMatrix] + assert(matTransposed.numRows === breeze.cols) + assert(matTransposed.numCols === breeze.rows) + assert(matTransposed.values.eq(breeze.data), "should not copy data") + } + + test("sparse matrix to breeze") { + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val mat = Matrices.sparse(3, 2, colPtrs, rowIndices, values) + val breeze = mat.asBreeze.asInstanceOf[BSM[Double]] + assert(breeze.rows === mat.numRows) + assert(breeze.cols === mat.numCols) + assert(breeze.data.eq(mat.asInstanceOf[SparseMatrix].values), "should not copy data") + } + + test("sparse breeze matrix to sparse matrix") { + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val breeze = new BSM[Double](values, 3, 2, colPtrs, rowIndices) + val mat = Matrices.fromBreeze(breeze).asInstanceOf[SparseMatrix] + assert(mat.numRows === breeze.rows) + assert(mat.numCols === breeze.cols) + assert(mat.values.eq(breeze.data), "should not copy data") + val matTransposed = Matrices.fromBreeze(breeze.t).asInstanceOf[SparseMatrix] + assert(matTransposed.numRows === breeze.cols) + assert(matTransposed.numCols === breeze.rows) + assert(!matTransposed.values.eq(breeze.data), "has to copy data") + } +} diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BreezeVectorConversionSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BreezeVectorConversionSuite.scala new file mode 100644 index 000000000000..4c9740b6bca7 --- /dev/null +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BreezeVectorConversionSuite.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} + +import org.apache.spark.ml.SparkMLFunSuite + +/** + * Test Breeze vector conversions. + */ +class BreezeVectorConversionSuite extends SparkMLFunSuite { + + val arr = Array(0.1, 0.2, 0.3, 0.4) + val n = 20 + val indices = Array(0, 3, 5, 10, 13) + val values = Array(0.1, 0.5, 0.3, -0.8, -1.0) + + test("dense to breeze") { + val vec = Vectors.dense(arr) + assert(vec.asBreeze === new BDV[Double](arr)) + } + + test("sparse to breeze") { + val vec = Vectors.sparse(n, indices, values) + assert(vec.asBreeze === new BSV[Double](indices, values, n)) + } + + test("dense breeze to vector") { + val breeze = new BDV[Double](arr) + val vec = Vectors.fromBreeze(breeze).asInstanceOf[DenseVector] + assert(vec.size === arr.length) + assert(vec.values.eq(arr), "should not copy data") + } + + test("sparse breeze to vector") { + val breeze = new BSV[Double](indices, values, n) + val vec = Vectors.fromBreeze(breeze).asInstanceOf[SparseVector] + assert(vec.size === n) + assert(vec.indices.eq(indices), "should not copy data") + assert(vec.values.eq(values), "should not copy data") + } + + test("sparse breeze with partially-used arrays to vector") { + val activeSize = 3 + val breeze = new BSV[Double](indices, values, activeSize, n) + val vec = Vectors.fromBreeze(breeze).asInstanceOf[SparseVector] + assert(vec.size === n) + assert(vec.indices === indices.slice(0, activeSize)) + assert(vec.values === values.slice(0, activeSize)) + } +} diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala new file mode 100644 index 000000000000..ace44165b106 --- /dev/null +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala @@ -0,0 +1,905 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import java.util.Random + +import breeze.linalg.{CSCMatrix, Matrix => BM} +import org.mockito.Mockito.when +import org.scalatest.mockito.MockitoSugar._ +import scala.collection.mutable.{Map => MutableMap} + +import org.apache.spark.ml.SparkMLFunSuite +import org.apache.spark.ml.util.TestingUtils._ + +class MatricesSuite extends SparkMLFunSuite { + test("dense matrix construction") { + val m = 3 + val n = 2 + val values = Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0) + val mat = Matrices.dense(m, n, values).asInstanceOf[DenseMatrix] + assert(mat.numRows === m) + assert(mat.numCols === n) + assert(mat.values.eq(values), "should not copy data") + } + + test("dense matrix construction with wrong dimension") { + intercept[RuntimeException] { + Matrices.dense(3, 2, Array(0.0, 1.0, 2.0)) + } + } + + test("sparse matrix construction") { + val m = 3 + val n = 4 + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 2, 4, 4) + val rowIndices = Array(1, 2, 1, 2) + val mat = Matrices.sparse(m, n, colPtrs, rowIndices, values).asInstanceOf[SparseMatrix] + assert(mat.numRows === m) + assert(mat.numCols === n) + assert(mat.values.eq(values), "should not copy data") + assert(mat.colPtrs.eq(colPtrs), "should not copy data") + assert(mat.rowIndices.eq(rowIndices), "should not copy data") + + val entries: Array[(Int, Int, Double)] = Array((2, 2, 3.0), (1, 0, 1.0), (2, 0, 2.0), + (1, 2, 2.0), (2, 2, 2.0), (1, 2, 2.0), (0, 0, 0.0)) + + val mat2 = SparseMatrix.fromCOO(m, n, entries) + assert(mat.asBreeze === mat2.asBreeze) + assert(mat2.values.length == 4) + } + + test("sparse matrix construction with wrong number of elements") { + intercept[IllegalArgumentException] { + Matrices.sparse(3, 2, Array(0, 1), Array(1, 2, 1), Array(0.0, 1.0, 2.0)) + } + + intercept[IllegalArgumentException] { + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(0.0, 1.0, 2.0)) + } + } + + test("index in matrices incorrect input") { + val sm = Matrices.sparse(3, 2, Array(0, 2, 3), Array(1, 2, 1), Array(0.0, 1.0, 2.0)) + val dm = Matrices.dense(3, 2, Array(0.0, 2.3, 1.4, 3.2, 1.0, 9.1)) + Array(sm, dm).foreach { mat => + intercept[IllegalArgumentException] { mat.index(4, 1) } + intercept[IllegalArgumentException] { mat.index(1, 4) } + intercept[IllegalArgumentException] { mat.index(-1, 2) } + intercept[IllegalArgumentException] { mat.index(1, -2) } + } + } + + test("equals") { + val dm1 = Matrices.dense(2, 2, Array(0.0, 1.0, 2.0, 3.0)) + assert(dm1 === dm1) + assert(dm1 !== dm1.transpose) + + val dm2 = Matrices.dense(2, 2, Array(0.0, 2.0, 1.0, 3.0)) + assert(dm1 === dm2.transpose) + + val sm1 = dm1.asInstanceOf[DenseMatrix].toSparse + assert(sm1 === sm1) + assert(sm1 === dm1) + assert(sm1 !== sm1.transpose) + + val sm2 = dm2.asInstanceOf[DenseMatrix].toSparse + assert(sm1 === sm2.transpose) + assert(sm1 === dm2.transpose) + } + + test("matrix copies are deep copies") { + val m = 3 + val n = 2 + + val denseMat = Matrices.dense(m, n, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val denseCopy = denseMat.copy + + assert(!denseMat.toArray.eq(denseCopy.toArray)) + + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val sparseMat = Matrices.sparse(m, n, colPtrs, rowIndices, values) + val sparseCopy = sparseMat.copy + + assert(!sparseMat.toArray.eq(sparseCopy.toArray)) + } + + test("matrix indexing and updating") { + val m = 3 + val n = 2 + val allValues = Array(0.0, 1.0, 2.0, 3.0, 4.0, 0.0) + + val denseMat = new DenseMatrix(m, n, allValues) + + assert(denseMat(0, 1) === 3.0) + assert(denseMat(0, 1) === denseMat.values(3)) + assert(denseMat(0, 1) === denseMat(3)) + assert(denseMat(0, 0) === 0.0) + + denseMat.update(0, 0, 10.0) + assert(denseMat(0, 0) === 10.0) + assert(denseMat.values(0) === 10.0) + + val sparseValues = Array(1.0, 2.0, 3.0, 4.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 0, 1) + val sparseMat = new SparseMatrix(m, n, colPtrs, rowIndices, sparseValues) + + assert(sparseMat(0, 1) === 3.0) + assert(sparseMat(0, 1) === sparseMat.values(2)) + assert(sparseMat(0, 0) === 0.0) + + intercept[NoSuchElementException] { + sparseMat.update(0, 0, 10.0) + } + + intercept[NoSuchElementException] { + sparseMat.update(2, 1, 10.0) + } + + sparseMat.update(0, 1, 10.0) + assert(sparseMat(0, 1) === 10.0) + assert(sparseMat.values(2) === 10.0) + } + + test("dense to dense") { + /* + dm1 = 4.0 2.0 -8.0 + -1.0 7.0 4.0 + + dm2 = 5.0 -9.0 4.0 + 1.0 -3.0 -8.0 + */ + val dm1 = new DenseMatrix(2, 3, Array(4.0, -1.0, 2.0, 7.0, -8.0, 4.0)) + val dm2 = new DenseMatrix(2, 3, Array(5.0, -9.0, 4.0, 1.0, -3.0, -8.0), isTransposed = true) + + val dm8 = dm1.toDenseColMajor + assert(dm8 === dm1) + assert(dm8.isColMajor) + assert(dm8.values.equals(dm1.values)) + + val dm5 = dm2.toDenseColMajor + assert(dm5 === dm2) + assert(dm5.isColMajor) + assert(dm5.values === Array(5.0, 1.0, -9.0, -3.0, 4.0, -8.0)) + + val dm4 = dm1.toDenseRowMajor + assert(dm4 === dm1) + assert(dm4.isRowMajor) + assert(dm4.values === Array(4.0, 2.0, -8.0, -1.0, 7.0, 4.0)) + + val dm6 = dm2.toDenseRowMajor + assert(dm6 === dm2) + assert(dm6.isRowMajor) + assert(dm6.values.equals(dm2.values)) + + val dm3 = dm1.toDense + assert(dm3 === dm1) + assert(dm3.isColMajor) + assert(dm3.values.equals(dm1.values)) + + val dm9 = dm2.toDense + assert(dm9 === dm2) + assert(dm9.isRowMajor) + assert(dm9.values.equals(dm2.values)) + } + + test("dense to sparse") { + /* + dm1 = 0.0 4.0 5.0 + 0.0 2.0 0.0 + + dm2 = 0.0 4.0 5.0 + 0.0 2.0 0.0 + + dm3 = 0.0 0.0 0.0 + 0.0 0.0 0.0 + */ + val dm1 = new DenseMatrix(2, 3, Array(0.0, 0.0, 4.0, 2.0, 5.0, 0.0)) + val dm2 = new DenseMatrix(2, 3, Array(0.0, 4.0, 5.0, 0.0, 2.0, 0.0), isTransposed = true) + val dm3 = new DenseMatrix(2, 3, Array(0.0, 0.0, 0.0, 0.0, 0.0, 0.0)) + + val sm1 = dm1.toSparseColMajor + assert(sm1 === dm1) + assert(sm1.isColMajor) + assert(sm1.values === Array(4.0, 2.0, 5.0)) + + val sm3 = dm2.toSparseColMajor + assert(sm3 === dm2) + assert(sm3.isColMajor) + assert(sm3.values === Array(4.0, 2.0, 5.0)) + + val sm5 = dm3.toSparseColMajor + assert(sm5 === dm3) + assert(sm5.values === Array.empty[Double]) + assert(sm5.isColMajor) + + val sm2 = dm1.toSparseRowMajor + assert(sm2 === dm1) + assert(sm2.isRowMajor) + assert(sm2.values === Array(4.0, 5.0, 2.0)) + + val sm4 = dm2.toSparseRowMajor + assert(sm4 === dm2) + assert(sm4.isRowMajor) + assert(sm4.values === Array(4.0, 5.0, 2.0)) + + val sm6 = dm3.toSparseRowMajor + assert(sm6 === dm3) + assert(sm6.values === Array.empty[Double]) + assert(sm6.isRowMajor) + + val sm7 = dm1.toSparse + assert(sm7 === dm1) + assert(sm7.values === Array(4.0, 2.0, 5.0)) + assert(sm7.isColMajor) + + val sm10 = dm2.toSparse + assert(sm10 === dm2) + assert(sm10.values === Array(4.0, 5.0, 2.0)) + assert(sm10.isRowMajor) + } + + test("sparse to sparse") { + /* + sm1 = sm2 = sm3 = sm4 = 0.0 4.0 5.0 + 0.0 2.0 0.0 + smZeros = 0.0 0.0 0.0 + 0.0 0.0 0.0 + */ + val sm1 = new SparseMatrix(2, 3, Array(0, 0, 2, 3), Array(0, 1, 0), Array(4.0, 2.0, 5.0)) + val sm2 = new SparseMatrix(2, 3, Array(0, 2, 3), Array(1, 2, 1), Array(4.0, 5.0, 2.0), + isTransposed = true) + val sm3 = new SparseMatrix(2, 3, Array(0, 0, 2, 4), Array(0, 1, 0, 1), + Array(4.0, 2.0, 5.0, 0.0)) + val sm4 = new SparseMatrix(2, 3, Array(0, 2, 4), Array(1, 2, 1, 2), + Array(4.0, 5.0, 2.0, 0.0), isTransposed = true) + val smZeros = new SparseMatrix(2, 3, Array(0, 2, 4, 6), Array(0, 1, 0, 1, 0, 1), + Array(0.0, 0.0, 0.0, 0.0, 0.0, 0.0)) + + val sm6 = sm1.toSparseColMajor + assert(sm6 === sm1) + assert(sm6.isColMajor) + assert(sm6.values.equals(sm1.values)) + + val sm7 = sm2.toSparseColMajor + assert(sm7 === sm2) + assert(sm7.isColMajor) + assert(sm7.values === Array(4.0, 2.0, 5.0)) + + val sm16 = sm3.toSparseColMajor + assert(sm16 === sm3) + assert(sm16.isColMajor) + assert(sm16.values === Array(4.0, 2.0, 5.0)) + + val sm14 = sm4.toSparseColMajor + assert(sm14 === sm4) + assert(sm14.values === Array(4.0, 2.0, 5.0)) + assert(sm14.isColMajor) + + val sm15 = smZeros.toSparseColMajor + assert(sm15 === smZeros) + assert(sm15.values === Array.empty[Double]) + assert(sm15.isColMajor) + + val sm5 = sm1.toSparseRowMajor + assert(sm5 === sm1) + assert(sm5.isRowMajor) + assert(sm5.values === Array(4.0, 5.0, 2.0)) + + val sm8 = sm2.toSparseRowMajor + assert(sm8 === sm2) + assert(sm8.isRowMajor) + assert(sm8.values.equals(sm2.values)) + + val sm10 = sm3.toSparseRowMajor + assert(sm10 === sm3) + assert(sm10.values === Array(4.0, 5.0, 2.0)) + assert(sm10.isRowMajor) + + val sm11 = sm4.toSparseRowMajor + assert(sm11 === sm4) + assert(sm11.values === Array(4.0, 5.0, 2.0)) + assert(sm11.isRowMajor) + + val sm17 = smZeros.toSparseRowMajor + assert(sm17 === smZeros) + assert(sm17.values === Array.empty[Double]) + assert(sm17.isRowMajor) + + val sm9 = sm3.toSparse + assert(sm9 === sm3) + assert(sm9.values === Array(4.0, 2.0, 5.0)) + assert(sm9.isColMajor) + + val sm12 = sm4.toSparse + assert(sm12 === sm4) + assert(sm12.values === Array(4.0, 5.0, 2.0)) + assert(sm12.isRowMajor) + + val sm13 = smZeros.toSparse + assert(sm13 === smZeros) + assert(sm13.values === Array.empty[Double]) + assert(sm13.isColMajor) + } + + test("sparse to dense") { + /* + sm1 = sm2 = 0.0 4.0 5.0 + 0.0 2.0 0.0 + + sm3 = 0.0 0.0 0.0 + 0.0 0.0 0.0 + */ + val sm1 = new SparseMatrix(2, 3, Array(0, 0, 2, 3), Array(0, 1, 0), Array(4.0, 2.0, 5.0)) + val sm2 = new SparseMatrix(2, 3, Array(0, 2, 3), Array(1, 2, 1), Array(4.0, 5.0, 2.0), + isTransposed = true) + val sm3 = new SparseMatrix(2, 3, Array(0, 0, 0, 0), Array.empty[Int], Array.empty[Double]) + + val dm6 = sm1.toDenseColMajor + assert(dm6 === sm1) + assert(dm6.isColMajor) + assert(dm6.values === Array(0.0, 0.0, 4.0, 2.0, 5.0, 0.0)) + + val dm7 = sm2.toDenseColMajor + assert(dm7 === sm2) + assert(dm7.isColMajor) + assert(dm7.values === Array(0.0, 0.0, 4.0, 2.0, 5.0, 0.0)) + + val dm2 = sm1.toDenseRowMajor + assert(dm2 === sm1) + assert(dm2.isRowMajor) + assert(dm2.values === Array(0.0, 4.0, 5.0, 0.0, 2.0, 0.0)) + + val dm4 = sm2.toDenseRowMajor + assert(dm4 === sm2) + assert(dm4.isRowMajor) + assert(dm4.values === Array(0.0, 4.0, 5.0, 0.0, 2.0, 0.0)) + + val dm1 = sm1.toDense + assert(dm1 === sm1) + assert(dm1.isColMajor) + assert(dm1.values === Array(0.0, 0.0, 4.0, 2.0, 5.0, 0.0)) + + val dm3 = sm2.toDense + assert(dm3 === sm2) + assert(dm3.isRowMajor) + assert(dm3.values === Array(0.0, 4.0, 5.0, 0.0, 2.0, 0.0)) + + val dm5 = sm3.toDense + assert(dm5 === sm3) + assert(dm5.isColMajor) + assert(dm5.values === Array.fill(6)(0.0)) + } + + test("compressed dense") { + /* + dm1 = 1.0 0.0 0.0 0.0 + 1.0 0.0 0.0 0.0 + 0.0 0.0 0.0 0.0 + + dm2 = 1.0 1.0 0.0 0.0 + 0.0 0.0 0.0 0.0 + 0.0 0.0 0.0 0.0 + */ + // this should compress to a sparse matrix + val dm1 = new DenseMatrix(3, 4, Array.fill(2)(1.0) ++ Array.fill(10)(0.0)) + + // optimal compression layout is row major since numRows < numCols + val cm1 = dm1.compressed.asInstanceOf[SparseMatrix] + assert(cm1 === dm1) + assert(cm1.isRowMajor) + assert(cm1.getSizeInBytes < dm1.getSizeInBytes) + + // force compressed column major + val cm2 = dm1.compressedColMajor.asInstanceOf[SparseMatrix] + assert(cm2 === dm1) + assert(cm2.isColMajor) + assert(cm2.getSizeInBytes < dm1.getSizeInBytes) + + // optimal compression layout for transpose is column major + val dm2 = dm1.transpose + val cm3 = dm2.compressed.asInstanceOf[SparseMatrix] + assert(cm3 === dm2) + assert(cm3.isColMajor) + assert(cm3.getSizeInBytes < dm2.getSizeInBytes) + + /* + dm3 = 1.0 1.0 1.0 0.0 + 1.0 1.0 0.0 0.0 + 1.0 1.0 0.0 0.0 + + dm4 = 1.0 1.0 1.0 1.0 + 1.0 1.0 1.0 0.0 + 0.0 0.0 0.0 0.0 + */ + // this should compress to a dense matrix + val dm3 = new DenseMatrix(3, 4, Array.fill(7)(1.0) ++ Array.fill(5)(0.0)) + val dm4 = new DenseMatrix(3, 4, Array.fill(7)(1.0) ++ Array.fill(5)(0.0), isTransposed = true) + + val cm4 = dm3.compressed.asInstanceOf[DenseMatrix] + assert(cm4 === dm3) + assert(cm4.isColMajor) + assert(cm4.values.equals(dm3.values)) + assert(cm4.getSizeInBytes === dm3.getSizeInBytes) + + // force compressed row major + val cm5 = dm3.compressedRowMajor.asInstanceOf[DenseMatrix] + assert(cm5 === dm3) + assert(cm5.isRowMajor) + assert(cm5.getSizeInBytes === dm3.getSizeInBytes) + + val cm6 = dm4.compressed.asInstanceOf[DenseMatrix] + assert(cm6 === dm4) + assert(cm6.isRowMajor) + assert(cm6.values.equals(dm4.values)) + assert(cm6.getSizeInBytes === dm4.getSizeInBytes) + + val cm7 = dm4.compressedColMajor.asInstanceOf[DenseMatrix] + assert(cm7 === dm4) + assert(cm7.isColMajor) + assert(cm7.getSizeInBytes === dm4.getSizeInBytes) + + // this has the same size sparse or dense + val dm5 = new DenseMatrix(4, 4, Array.fill(7)(1.0) ++ Array.fill(9)(0.0)) + // should choose dense to break ties + val cm8 = dm5.compressed.asInstanceOf[DenseMatrix] + assert(cm8.getSizeInBytes === dm5.toSparseColMajor.getSizeInBytes) + } + + test("compressed sparse") { + /* + sm1 = 0.0 -1.0 + 0.0 0.0 + 0.0 0.0 + 0.0 0.0 + + sm2 = 0.0 0.0 0.0 0.0 + -1.0 0.0 0.0 0.0 + */ + // these should compress to sparse matrices + val sm1 = new SparseMatrix(4, 2, Array(0, 0, 1), Array(0), Array(-1.0)) + val sm2 = sm1.transpose + + val cm1 = sm1.compressed.asInstanceOf[SparseMatrix] + // optimal is column major + assert(cm1 === sm1) + assert(cm1.isColMajor) + assert(cm1.values.equals(sm1.values)) + assert(cm1.getSizeInBytes === sm1.getSizeInBytes) + + val cm2 = sm1.compressedRowMajor.asInstanceOf[SparseMatrix] + assert(cm2 === sm1) + assert(cm2.isRowMajor) + // forced to be row major, so we have increased the size + assert(cm2.getSizeInBytes > sm1.getSizeInBytes) + assert(cm2.getSizeInBytes < sm1.toDense.getSizeInBytes) + + val cm9 = sm1.compressedColMajor.asInstanceOf[SparseMatrix] + assert(cm9 === sm1) + assert(cm9.values.equals(sm1.values)) + assert(cm9.getSizeInBytes === sm1.getSizeInBytes) + + val cm3 = sm2.compressed.asInstanceOf[SparseMatrix] + assert(cm3 === sm2) + assert(cm3.isRowMajor) + assert(cm3.values.equals(sm2.values)) + assert(cm3.getSizeInBytes === sm2.getSizeInBytes) + + val cm8 = sm2.compressedColMajor.asInstanceOf[SparseMatrix] + assert(cm8 === sm2) + assert(cm8.isColMajor) + // forced to be col major, so we have increased the size + assert(cm8.getSizeInBytes > sm2.getSizeInBytes) + assert(cm8.getSizeInBytes < sm2.toDense.getSizeInBytes) + + val cm10 = sm2.compressedRowMajor.asInstanceOf[SparseMatrix] + assert(cm10 === sm2) + assert(cm10.isRowMajor) + assert(cm10.values.equals(sm2.values)) + assert(cm10.getSizeInBytes === sm2.getSizeInBytes) + + + /* + sm3 = 0.0 -1.0 + 2.0 3.0 + -4.0 9.0 + */ + // this should compress to a dense matrix + val sm3 = new SparseMatrix(3, 2, Array(0, 2, 5), Array(1, 2, 0, 1, 2), + Array(2.0, -4.0, -1.0, 3.0, 9.0)) + + // dense is optimal, and maintains column major + val cm4 = sm3.compressed.asInstanceOf[DenseMatrix] + assert(cm4 === sm3) + assert(cm4.isColMajor) + assert(cm4.getSizeInBytes < sm3.getSizeInBytes) + + val cm5 = sm3.compressedRowMajor.asInstanceOf[DenseMatrix] + assert(cm5 === sm3) + assert(cm5.isRowMajor) + assert(cm5.getSizeInBytes < sm3.getSizeInBytes) + + val cm11 = sm3.compressedColMajor.asInstanceOf[DenseMatrix] + assert(cm11 === sm3) + assert(cm11.isColMajor) + assert(cm11.getSizeInBytes < sm3.getSizeInBytes) + + /* + sm4 = 1.0 0.0 0.0 ... + + sm5 = 1.0 + 0.0 + 0.0 + ... + */ + val sm4 = new SparseMatrix(Int.MaxValue, 1, Array(0, 1), Array(0), Array(1.0)) + val cm6 = sm4.compressed.asInstanceOf[SparseMatrix] + assert(cm6 === sm4) + assert(cm6.isColMajor) + assert(cm6.getSizeInBytes <= sm4.getSizeInBytes) + + val sm5 = new SparseMatrix(1, Int.MaxValue, Array(0, 1), Array(0), Array(1.0), + isTransposed = true) + val cm7 = sm5.compressed.asInstanceOf[SparseMatrix] + assert(cm7 === sm5) + assert(cm7.isRowMajor) + assert(cm7.getSizeInBytes <= sm5.getSizeInBytes) + + // this has the same size sparse or dense + val sm6 = new SparseMatrix(4, 4, Array(0, 4, 7, 7, 7), Array(0, 1, 2, 3, 0, 1, 2), + Array.fill(7)(1.0)) + // should choose dense to break ties + val cm12 = sm6.compressed.asInstanceOf[DenseMatrix] + assert(cm12.getSizeInBytes === sm6.getSizeInBytes) + } + + test("map, update") { + val m = 3 + val n = 2 + val values = Array(1.0, 2.0, 4.0, 5.0) + val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(0, 1, 1, 2) + + val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values) + val deMat1 = new DenseMatrix(m, n, allValues) + val deMat2 = deMat1.map(_ * 2) + val spMat2 = spMat1.map(_ * 2) + deMat1.update(_ * 2) + spMat1.update(_ * 2) + + assert(spMat1.toArray === spMat2.toArray) + assert(deMat1.toArray === deMat2.toArray) + } + + test("transpose") { + val dA = + new DenseMatrix(4, 3, Array(0.0, 1.0, 0.0, 0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 0.0, 3.0)) + val sA = new SparseMatrix(4, 3, Array(0, 1, 3, 4), Array(1, 0, 2, 3), Array(1.0, 2.0, 1.0, 3.0)) + + val dAT = dA.transpose.asInstanceOf[DenseMatrix] + val sAT = sA.transpose.asInstanceOf[SparseMatrix] + val dATexpected = + new DenseMatrix(3, 4, Array(0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 1.0, 0.0, 0.0, 0.0, 3.0)) + val sATexpected = + new SparseMatrix(3, 4, Array(0, 1, 2, 3, 4), Array(1, 0, 1, 2), Array(2.0, 1.0, 1.0, 3.0)) + + assert(dAT.asBreeze === dATexpected.asBreeze) + assert(sAT.asBreeze === sATexpected.asBreeze) + assert(dA(1, 0) === dAT(0, 1)) + assert(dA(2, 1) === dAT(1, 2)) + assert(sA(1, 0) === sAT(0, 1)) + assert(sA(2, 1) === sAT(1, 2)) + + assert(!dA.toArray.eq(dAT.toArray), "has to have a new array") + assert(dA.values.eq(dAT.transpose.asInstanceOf[DenseMatrix].values), "should not copy array") + + assert(dAT.toSparse.asBreeze === sATexpected.asBreeze) + assert(sAT.toDense.asBreeze === dATexpected.asBreeze) + } + + test("foreachActive") { + val m = 3 + val n = 2 + val values = Array(1.0, 2.0, 4.0, 5.0) + val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(0, 1, 1, 2) + + val sp = new SparseMatrix(m, n, colPtrs, rowIndices, values) + val dn = new DenseMatrix(m, n, allValues) + + val dnMap = MutableMap[(Int, Int), Double]() + dn.foreachActive { (i, j, value) => + dnMap.put((i, j), value) + } + assert(dnMap.size === 6) + assert(dnMap((0, 0)) === 1.0) + assert(dnMap((1, 0)) === 2.0) + assert(dnMap((2, 0)) === 0.0) + assert(dnMap((0, 1)) === 0.0) + assert(dnMap((1, 1)) === 4.0) + assert(dnMap((2, 1)) === 5.0) + + val spMap = MutableMap[(Int, Int), Double]() + sp.foreachActive { (i, j, value) => + spMap.put((i, j), value) + } + assert(spMap.size === 4) + assert(spMap((0, 0)) === 1.0) + assert(spMap((1, 0)) === 2.0) + assert(spMap((1, 1)) === 4.0) + assert(spMap((2, 1)) === 5.0) + } + + test("horzcat, vertcat, eye, speye") { + val m = 3 + val n = 2 + val values = Array(1.0, 2.0, 4.0, 5.0) + val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(0, 1, 1, 2) + // transposed versions + val allValuesT = Array(1.0, 0.0, 2.0, 4.0, 0.0, 5.0) + val colPtrsT = Array(0, 1, 3, 4) + val rowIndicesT = Array(0, 0, 1, 1) + + val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values) + val deMat1 = new DenseMatrix(m, n, allValues) + val spMat1T = new SparseMatrix(n, m, colPtrsT, rowIndicesT, values) + val deMat1T = new DenseMatrix(n, m, allValuesT) + + // should equal spMat1 & deMat1 respectively + val spMat1TT = spMat1T.transpose + val deMat1TT = deMat1T.transpose + + val deMat2 = Matrices.eye(3) + val spMat2 = Matrices.speye(3) + val deMat3 = Matrices.eye(2) + val spMat3 = Matrices.speye(2) + + val spHorz = Matrices.horzcat(Array(spMat1, spMat2)) + val spHorz2 = Matrices.horzcat(Array(spMat1, deMat2)) + val spHorz3 = Matrices.horzcat(Array(deMat1, spMat2)) + val deHorz1 = Matrices.horzcat(Array(deMat1, deMat2)) + val deHorz2 = Matrices.horzcat(Array.empty[Matrix]) + + assert(deHorz1.numRows === 3) + assert(spHorz2.numRows === 3) + assert(spHorz3.numRows === 3) + assert(spHorz.numRows === 3) + assert(deHorz1.numCols === 5) + assert(spHorz2.numCols === 5) + assert(spHorz3.numCols === 5) + assert(spHorz.numCols === 5) + assert(deHorz2.numRows === 0) + assert(deHorz2.numCols === 0) + assert(deHorz2.toArray.length === 0) + + assert(deHorz1 ~== spHorz2.asInstanceOf[SparseMatrix].toDense absTol 1e-15) + assert(spHorz2 ~== spHorz3 absTol 1e-15) + assert(spHorz(0, 0) === 1.0) + assert(spHorz(2, 1) === 5.0) + assert(spHorz(0, 2) === 1.0) + assert(spHorz(1, 2) === 0.0) + assert(spHorz(1, 3) === 1.0) + assert(spHorz(2, 4) === 1.0) + assert(spHorz(1, 4) === 0.0) + assert(deHorz1(0, 0) === 1.0) + assert(deHorz1(2, 1) === 5.0) + assert(deHorz1(0, 2) === 1.0) + assert(deHorz1(1, 2) == 0.0) + assert(deHorz1(1, 3) === 1.0) + assert(deHorz1(2, 4) === 1.0) + assert(deHorz1(1, 4) === 0.0) + + // containing transposed matrices + val spHorzT = Matrices.horzcat(Array(spMat1TT, spMat2)) + val spHorz2T = Matrices.horzcat(Array(spMat1TT, deMat2)) + val spHorz3T = Matrices.horzcat(Array(deMat1TT, spMat2)) + val deHorz1T = Matrices.horzcat(Array(deMat1TT, deMat2)) + + assert(deHorz1T ~== deHorz1 absTol 1e-15) + assert(spHorzT ~== spHorz absTol 1e-15) + assert(spHorz2T ~== spHorz2 absTol 1e-15) + assert(spHorz3T ~== spHorz3 absTol 1e-15) + + intercept[IllegalArgumentException] { + Matrices.horzcat(Array(spMat1, spMat3)) + } + + intercept[IllegalArgumentException] { + Matrices.horzcat(Array(deMat1, spMat3)) + } + + val spVert = Matrices.vertcat(Array(spMat1, spMat3)) + val deVert1 = Matrices.vertcat(Array(deMat1, deMat3)) + val spVert2 = Matrices.vertcat(Array(spMat1, deMat3)) + val spVert3 = Matrices.vertcat(Array(deMat1, spMat3)) + val deVert2 = Matrices.vertcat(Array.empty[Matrix]) + + assert(deVert1.numRows === 5) + assert(spVert2.numRows === 5) + assert(spVert3.numRows === 5) + assert(spVert.numRows === 5) + assert(deVert1.numCols === 2) + assert(spVert2.numCols === 2) + assert(spVert3.numCols === 2) + assert(spVert.numCols === 2) + assert(deVert2.numRows === 0) + assert(deVert2.numCols === 0) + assert(deVert2.toArray.length === 0) + + assert(deVert1 ~== spVert2.asInstanceOf[SparseMatrix].toDense absTol 1e-15) + assert(spVert2 ~== spVert3 absTol 1e-15) + assert(spVert(0, 0) === 1.0) + assert(spVert(2, 1) === 5.0) + assert(spVert(3, 0) === 1.0) + assert(spVert(3, 1) === 0.0) + assert(spVert(4, 1) === 1.0) + assert(deVert1(0, 0) === 1.0) + assert(deVert1(2, 1) === 5.0) + assert(deVert1(3, 0) === 1.0) + assert(deVert1(3, 1) === 0.0) + assert(deVert1(4, 1) === 1.0) + + // containing transposed matrices + val spVertT = Matrices.vertcat(Array(spMat1TT, spMat3)) + val deVert1T = Matrices.vertcat(Array(deMat1TT, deMat3)) + val spVert2T = Matrices.vertcat(Array(spMat1TT, deMat3)) + val spVert3T = Matrices.vertcat(Array(deMat1TT, spMat3)) + + assert(deVert1T ~== deVert1 absTol 1e-15) + assert(spVertT ~== spVert absTol 1e-15) + assert(spVert2T ~== spVert2 absTol 1e-15) + assert(spVert3T ~== spVert3 absTol 1e-15) + + intercept[IllegalArgumentException] { + Matrices.vertcat(Array(spMat1, spMat2)) + } + + intercept[IllegalArgumentException] { + Matrices.vertcat(Array(deMat1, spMat2)) + } + } + + test("zeros") { + val mat = Matrices.zeros(2, 3).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 3) + assert(mat.values.forall(_ == 0.0)) + } + + test("ones") { + val mat = Matrices.ones(2, 3).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 3) + assert(mat.values.forall(_ == 1.0)) + } + + test("eye") { + val mat = Matrices.eye(2).asInstanceOf[DenseMatrix] + assert(mat.numCols === 2) + assert(mat.numCols === 2) + assert(mat.values.toSeq === Seq(1.0, 0.0, 0.0, 1.0)) + } + + test("rand") { + val rng = mock[Random] + when(rng.nextDouble()).thenReturn(1.0, 2.0, 3.0, 4.0) + val mat = Matrices.rand(2, 2, rng).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 2) + assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0)) + } + + test("randn") { + val rng = mock[Random] + when(rng.nextGaussian()).thenReturn(1.0, 2.0, 3.0, 4.0) + val mat = Matrices.randn(2, 2, rng).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 2) + assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0)) + } + + test("diag") { + val mat = Matrices.diag(Vectors.dense(1.0, 2.0)).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 2) + assert(mat.values.toSeq === Seq(1.0, 0.0, 0.0, 2.0)) + } + + test("sprand") { + val rng = mock[Random] + when(rng.nextInt(4)).thenReturn(0, 1, 1, 3, 2, 2, 0, 1, 3, 0) + when(rng.nextDouble()).thenReturn(1.0, 2.0, 3.0, 4.0, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0) + val mat = SparseMatrix.sprand(4, 4, 0.25, rng) + assert(mat.numRows === 4) + assert(mat.numCols === 4) + assert(mat.rowIndices.toSeq === Seq(3, 0, 2, 1)) + assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0)) + val mat2 = SparseMatrix.sprand(2, 3, 1.0, rng) + assert(mat2.rowIndices.toSeq === Seq(0, 1, 0, 1, 0, 1)) + assert(mat2.colPtrs.toSeq === Seq(0, 2, 4, 6)) + } + + test("sprandn") { + val rng = mock[Random] + when(rng.nextInt(4)).thenReturn(0, 1, 1, 3, 2, 2, 0, 1, 3, 0) + when(rng.nextGaussian()).thenReturn(1.0, 2.0, 3.0, 4.0) + val mat = SparseMatrix.sprandn(4, 4, 0.25, rng) + assert(mat.numRows === 4) + assert(mat.numCols === 4) + assert(mat.rowIndices.toSeq === Seq(3, 0, 2, 1)) + assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0)) + } + + test("toString") { + val empty = Matrices.ones(0, 0) + empty.toString(0, 0) + + val mat = Matrices.rand(5, 10, new Random()) + mat.toString(-1, -5) + mat.toString(0, 0) + mat.toString(Int.MinValue, Int.MinValue) + mat.toString(Int.MaxValue, Int.MaxValue) + var lines = mat.toString(6, 50).lines.toArray + assert(lines.size == 5 && lines.forall(_.size <= 50)) + + lines = mat.toString(5, 100).lines.toArray + assert(lines.size == 5 && lines.forall(_.size <= 100)) + } + + test("numNonzeros and numActives") { + val dm1 = Matrices.dense(3, 2, Array(0, 0, -1, 1, 0, 1)) + assert(dm1.numNonzeros === 3) + assert(dm1.numActives === 6) + + val sm1 = Matrices.sparse(3, 2, Array(0, 2, 3), Array(0, 2, 1), Array(0.0, -1.2, 0.0)) + assert(sm1.numNonzeros === 1) + assert(sm1.numActives === 3) + } + + test("fromBreeze with sparse matrix") { + // colPtr.last does NOT always equal to values.length in breeze SCSMatrix and + // invocation of compact() may be necessary. Refer to SPARK-11507 + val bm1: BM[Double] = new CSCMatrix[Double]( + Array(1.0, 1, 1), 3, 3, Array(0, 1, 2, 3), Array(0, 1, 2)) + val bm2: BM[Double] = new CSCMatrix[Double]( + Array(1.0, 2, 2, 4), 3, 3, Array(0, 0, 2, 4), Array(1, 2, 1, 2)) + val sum = bm1 + bm2 + Matrices.fromBreeze(sum) + } + + test("row/col iterator") { + val dm = new DenseMatrix(3, 2, Array(0, 1, 2, 3, 4, 0)) + val sm = dm.toSparse + val rows = Seq(Vectors.dense(0, 3), Vectors.dense(1, 4), Vectors.dense(2, 0)) + val cols = Seq(Vectors.dense(0, 1, 2), Vectors.dense(3, 4, 0)) + for (m <- Seq(dm, sm)) { + assert(m.rowIter.toSeq === rows) + assert(m.colIter.toSeq === cols) + assert(m.transpose.rowIter.toSeq === cols) + assert(m.transpose.colIter.toSeq === rows) + } + } +} diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala new file mode 100644 index 000000000000..79acef8214d8 --- /dev/null +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala @@ -0,0 +1,369 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import scala.util.Random + +import breeze.linalg.{squaredDistance => breezeSquaredDistance, DenseMatrix => BDM} + +import org.apache.spark.ml.SparkMLFunSuite +import org.apache.spark.ml.util.TestingUtils._ + +class VectorsSuite extends SparkMLFunSuite { + + val arr = Array(0.1, 0.0, 0.3, 0.4) + val n = 4 + val indices = Array(0, 2, 3) + val values = Array(0.1, 0.3, 0.4) + + test("dense vector construction with varargs") { + val vec = Vectors.dense(arr).asInstanceOf[DenseVector] + assert(vec.size === arr.length) + assert(vec.values.eq(arr)) + } + + test("dense vector construction from a double array") { + val vec = Vectors.dense(arr).asInstanceOf[DenseVector] + assert(vec.size === arr.length) + assert(vec.values.eq(arr)) + } + + test("sparse vector construction") { + val vec = Vectors.sparse(n, indices, values).asInstanceOf[SparseVector] + assert(vec.size === n) + assert(vec.indices.eq(indices)) + assert(vec.values.eq(values)) + } + + test("sparse vector construction with unordered elements") { + val vec = Vectors.sparse(n, indices.zip(values).reverse).asInstanceOf[SparseVector] + assert(vec.size === n) + assert(vec.indices === indices) + assert(vec.values === values) + } + + test("sparse vector construction with mismatched indices/values array") { + intercept[IllegalArgumentException] { + Vectors.sparse(4, Array(1, 2, 3), Array(3.0, 5.0, 7.0, 9.0)) + } + intercept[IllegalArgumentException] { + Vectors.sparse(4, Array(1, 2, 3), Array(3.0, 5.0)) + } + } + + test("sparse vector construction with too many indices vs size") { + intercept[IllegalArgumentException] { + Vectors.sparse(3, Array(1, 2, 3, 4), Array(3.0, 5.0, 7.0, 9.0)) + } + } + + test("sparse vector construction with negative indices") { + intercept[IllegalArgumentException] { + Vectors.sparse(3, Array(-1, 1), Array(3.0, 5.0)) + } + } + + test("dense to array") { + val vec = Vectors.dense(arr).asInstanceOf[DenseVector] + assert(vec.toArray.eq(arr)) + } + + test("dense argmax") { + val vec = Vectors.dense(Array.empty[Double]).asInstanceOf[DenseVector] + assert(vec.argmax === -1) + + val vec2 = Vectors.dense(arr).asInstanceOf[DenseVector] + assert(vec2.argmax === 3) + + val vec3 = Vectors.dense(Array(-1.0, 0.0, -2.0, 1.0)).asInstanceOf[DenseVector] + assert(vec3.argmax === 3) + } + + test("sparse to array") { + val vec = Vectors.sparse(n, indices, values).asInstanceOf[SparseVector] + assert(vec.toArray === arr) + } + + test("sparse argmax") { + val vec = Vectors.sparse(0, Array.empty[Int], Array.empty[Double]).asInstanceOf[SparseVector] + assert(vec.argmax === -1) + + val vec2 = Vectors.sparse(n, indices, values).asInstanceOf[SparseVector] + assert(vec2.argmax === 3) + + val vec3 = Vectors.sparse(5, Array(2, 3, 4), Array(1.0, 0.0, -.7)) + assert(vec3.argmax === 2) + + // check for case that sparse vector is created with + // only negative values {0.0, 0.0,-1.0, -0.7, 0.0} + val vec4 = Vectors.sparse(5, Array(2, 3), Array(-1.0, -.7)) + assert(vec4.argmax === 0) + + val vec5 = Vectors.sparse(11, Array(0, 3, 10), Array(-1.0, -.7, 0.0)) + assert(vec5.argmax === 1) + + val vec6 = Vectors.sparse(11, Array(0, 1, 2), Array(-1.0, -.7, 0.0)) + assert(vec6.argmax === 2) + + val vec7 = Vectors.sparse(5, Array(0, 1, 3), Array(-1.0, 0.0, -.7)) + assert(vec7.argmax === 1) + + val vec8 = Vectors.sparse(5, Array(1, 2), Array(0.0, -1.0)) + assert(vec8.argmax === 0) + + // Check for case when sparse vector is non-empty but the values are empty + val vec9 = Vectors.sparse(100, Array.empty[Int], Array.empty[Double]).asInstanceOf[SparseVector] + assert(vec9.argmax === 0) + + val vec10 = Vectors.sparse(1, Array.empty[Int], Array.empty[Double]).asInstanceOf[SparseVector] + assert(vec10.argmax === 0) + } + + test("vector equals") { + val dv1 = Vectors.dense(arr.clone()) + val dv2 = Vectors.dense(arr.clone()) + val sv1 = Vectors.sparse(n, indices.clone(), values.clone()) + val sv2 = Vectors.sparse(n, indices.clone(), values.clone()) + + val vectors = Seq(dv1, dv2, sv1, sv2) + + for (v <- vectors; u <- vectors) { + assert(v === u) + assert(v.## === u.##) + } + + val another = Vectors.dense(0.1, 0.2, 0.3, 0.4) + + for (v <- vectors) { + assert(v != another) + assert(v.## != another.##) + } + } + + test("vectors equals with explicit 0") { + val dv1 = Vectors.dense(Array(0, 0.9, 0, 0.8, 0)) + val sv1 = Vectors.sparse(5, Array(1, 3), Array(0.9, 0.8)) + val sv2 = Vectors.sparse(5, Array(0, 1, 2, 3, 4), Array(0, 0.9, 0, 0.8, 0)) + + val vectors = Seq(dv1, sv1, sv2) + for (v <- vectors; u <- vectors) { + assert(v === u) + assert(v.## === u.##) + } + + val another = Vectors.sparse(5, Array(0, 1, 3), Array(0, 0.9, 0.2)) + for (v <- vectors) { + assert(v != another) + assert(v.## != another.##) + } + } + + test("indexing dense vectors") { + val vec = Vectors.dense(1.0, 2.0, 3.0, 4.0) + assert(vec(0) === 1.0) + assert(vec(3) === 4.0) + } + + test("indexing sparse vectors") { + val vec = Vectors.sparse(7, Array(0, 2, 4, 6), Array(1.0, 2.0, 3.0, 4.0)) + assert(vec(0) === 1.0) + assert(vec(1) === 0.0) + assert(vec(2) === 2.0) + assert(vec(3) === 0.0) + assert(vec(6) === 4.0) + val vec2 = Vectors.sparse(8, Array(0, 2, 4, 6), Array(1.0, 2.0, 3.0, 4.0)) + assert(vec2(6) === 4.0) + assert(vec2(7) === 0.0) + } + + test("zeros") { + assert(Vectors.zeros(3) === Vectors.dense(0.0, 0.0, 0.0)) + } + + test("Vector.copy") { + val sv = Vectors.sparse(4, Array(0, 2), Array(1.0, 2.0)) + val svCopy = sv.copy + (sv, svCopy) match { + case (sv: SparseVector, svCopy: SparseVector) => + assert(sv.size === svCopy.size) + assert(sv.indices === svCopy.indices) + assert(sv.values === svCopy.values) + assert(!sv.indices.eq(svCopy.indices)) + assert(!sv.values.eq(svCopy.values)) + case _ => + throw new RuntimeException(s"copy returned ${svCopy.getClass} on ${sv.getClass}.") + } + + val dv = Vectors.dense(1.0, 0.0, 2.0) + val dvCopy = dv.copy + (dv, dvCopy) match { + case (dv: DenseVector, dvCopy: DenseVector) => + assert(dv.size === dvCopy.size) + assert(dv.values === dvCopy.values) + assert(!dv.values.eq(dvCopy.values)) + case _ => + throw new RuntimeException(s"copy returned ${dvCopy.getClass} on ${dv.getClass}.") + } + } + + test("fromBreeze") { + val x = BDM.zeros[Double](10, 10) + val v = Vectors.fromBreeze(x(::, 0)) + assert(v.size === x.rows) + } + + test("sqdist") { + val random = new Random() + for (m <- 1 until 1000 by 100) { + val nnz = random.nextInt(m) + + val indices1 = random.shuffle(0 to m - 1).slice(0, nnz).sorted.toArray + val values1 = Array.fill(nnz)(random.nextDouble) + val sparseVector1 = Vectors.sparse(m, indices1, values1) + + val indices2 = random.shuffle(0 to m - 1).slice(0, nnz).sorted.toArray + val values2 = Array.fill(nnz)(random.nextDouble) + val sparseVector2 = Vectors.sparse(m, indices2, values2) + + val denseVector1 = Vectors.dense(sparseVector1.toArray) + val denseVector2 = Vectors.dense(sparseVector2.toArray) + + val squaredDist = breezeSquaredDistance(sparseVector1.asBreeze, sparseVector2.asBreeze) + + // SparseVector vs. SparseVector + assert(Vectors.sqdist(sparseVector1, sparseVector2) ~== squaredDist relTol 1E-8) + // DenseVector vs. SparseVector + assert(Vectors.sqdist(denseVector1, sparseVector2) ~== squaredDist relTol 1E-8) + // DenseVector vs. DenseVector + assert(Vectors.sqdist(denseVector1, denseVector2) ~== squaredDist relTol 1E-8) + } + } + + test("foreachActive") { + val dv = Vectors.dense(0.0, 1.2, 3.1, 0.0) + val sv = Vectors.sparse(4, Seq((1, 1.2), (2, 3.1), (3, 0.0))) + + val dvMap = scala.collection.mutable.Map[Int, Double]() + dv.foreachActive { (index, value) => + dvMap.put(index, value) + } + assert(dvMap.size === 4) + assert(dvMap.get(0) === Some(0.0)) + assert(dvMap.get(1) === Some(1.2)) + assert(dvMap.get(2) === Some(3.1)) + assert(dvMap.get(3) === Some(0.0)) + + val svMap = scala.collection.mutable.Map[Int, Double]() + sv.foreachActive { (index, value) => + svMap.put(index, value) + } + assert(svMap.size === 3) + assert(svMap.get(1) === Some(1.2)) + assert(svMap.get(2) === Some(3.1)) + assert(svMap.get(3) === Some(0.0)) + } + + test("vector p-norm") { + val dv = Vectors.dense(0.0, -1.2, 3.1, 0.0, -4.5, 1.9) + val sv = Vectors.sparse(6, Seq((1, -1.2), (2, 3.1), (3, 0.0), (4, -4.5), (5, 1.9))) + + assert(Vectors.norm(dv, 1.0) ~== dv.toArray.foldLeft(0.0)((a, v) => + a + math.abs(v)) relTol 1E-8) + assert(Vectors.norm(sv, 1.0) ~== sv.toArray.foldLeft(0.0)((a, v) => + a + math.abs(v)) relTol 1E-8) + + assert(Vectors.norm(dv, 2.0) ~== math.sqrt(dv.toArray.foldLeft(0.0)((a, v) => + a + v * v)) relTol 1E-8) + assert(Vectors.norm(sv, 2.0) ~== math.sqrt(sv.toArray.foldLeft(0.0)((a, v) => + a + v * v)) relTol 1E-8) + + assert(Vectors.norm(dv, Double.PositiveInfinity) ~== dv.toArray.map(math.abs).max relTol 1E-8) + assert(Vectors.norm(sv, Double.PositiveInfinity) ~== sv.toArray.map(math.abs).max relTol 1E-8) + + assert(Vectors.norm(dv, 3.7) ~== math.pow(dv.toArray.foldLeft(0.0)((a, v) => + a + math.pow(math.abs(v), 3.7)), 1.0 / 3.7) relTol 1E-8) + assert(Vectors.norm(sv, 3.7) ~== math.pow(sv.toArray.foldLeft(0.0)((a, v) => + a + math.pow(math.abs(v), 3.7)), 1.0 / 3.7) relTol 1E-8) + } + + test("Vector numActive and numNonzeros") { + val dv = Vectors.dense(0.0, 2.0, 3.0, 0.0) + assert(dv.numActives === 4) + assert(dv.numNonzeros === 2) + + val sv = Vectors.sparse(4, Array(0, 1, 2), Array(0.0, 2.0, 3.0)) + assert(sv.numActives === 3) + assert(sv.numNonzeros === 2) + } + + test("Vector toSparse and toDense") { + val dv0 = Vectors.dense(0.0, 2.0, 3.0, 0.0) + assert(dv0.toDense === dv0) + val dv0s = dv0.toSparse + assert(dv0s.numActives === 2) + assert(dv0s === dv0) + + assert(dv0.toSparseWithSize(dv0.numNonzeros) === dv0) + val dv0s2 = dv0.toSparseWithSize(dv0.numNonzeros) + assert(dv0s2.numActives === 2) + assert(dv0s2 === dv0s) + + val sv0 = Vectors.sparse(4, Array(0, 1, 2), Array(0.0, 2.0, 3.0)) + assert(sv0.toDense === sv0) + val sv0s = sv0.toSparse + assert(sv0s.numActives === 2) + assert(sv0s === sv0) + + assert(sv0.toSparseWithSize(sv0.numNonzeros) === sv0) + val sv0s2 = sv0.toSparseWithSize(sv0.numNonzeros) + assert(sv0s2.numActives === 2) + assert(sv0s2 === sv0s) + } + + test("Vector.compressed") { + val dv0 = Vectors.dense(1.0, 2.0, 3.0, 0.0) + val dv0c = dv0.compressed.asInstanceOf[DenseVector] + assert(dv0c === dv0) + + val dv1 = Vectors.dense(0.0, 2.0, 0.0, 0.0) + val dv1c = dv1.compressed.asInstanceOf[SparseVector] + assert(dv1 === dv1c) + assert(dv1c.numActives === 1) + + val sv0 = Vectors.sparse(4, Array(1, 2), Array(2.0, 0.0)) + val sv0c = sv0.compressed.asInstanceOf[SparseVector] + assert(sv0 === sv0c) + assert(sv0c.numActives === 1) + + val sv1 = Vectors.sparse(4, Array(0, 1, 2), Array(1.0, 2.0, 3.0)) + val sv1c = sv1.compressed.asInstanceOf[DenseVector] + assert(sv1 === sv1c) + + val sv2 = Vectors.sparse(Int.MaxValue, Array(0), Array(3.4)) + val sv2c = sv2.compressed.asInstanceOf[SparseVector] + assert(sv2c === sv2) + assert(sv2c.numActives === 1) + } + + test("SparseVector.slice") { + val v = new SparseVector(5, Array(1, 2, 4), Array(1.1, 2.2, 4.4)) + assert(v.slice(Array(0, 2)) === new SparseVector(2, Array(1), Array(2.2))) + assert(v.slice(Array(2, 0)) === new SparseVector(2, Array(0), Array(2.2))) + assert(v.slice(Array(2, 0, 3, 4)) === new SparseVector(4, Array(0, 3), Array(2.2, 4.4))) + } +} diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussianSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussianSuite.scala new file mode 100644 index 000000000000..f9306ed83e87 --- /dev/null +++ b/mllib-local/src/test/scala/org/apache/spark/ml/stat/distribution/MultivariateGaussianSuite.scala @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.stat.distribution + +import org.apache.spark.ml.SparkMLFunSuite +import org.apache.spark.ml.linalg.{Matrices, Vectors} +import org.apache.spark.ml.util.TestingUtils._ + + +class MultivariateGaussianSuite extends SparkMLFunSuite { + + test("univariate") { + val x1 = Vectors.dense(0.0) + val x2 = Vectors.dense(1.5) + + val mu = Vectors.dense(0.0) + val sigma1 = Matrices.dense(1, 1, Array(1.0)) + val dist1 = new MultivariateGaussian(mu, sigma1) + assert(dist1.pdf(x1) ~== 0.39894 absTol 1E-5) + assert(dist1.pdf(x2) ~== 0.12952 absTol 1E-5) + + val sigma2 = Matrices.dense(1, 1, Array(4.0)) + val dist2 = new MultivariateGaussian(mu, sigma2) + assert(dist2.pdf(x1) ~== 0.19947 absTol 1E-5) + assert(dist2.pdf(x2) ~== 0.15057 absTol 1E-5) + } + + test("multivariate") { + val x1 = Vectors.dense(0.0, 0.0) + val x2 = Vectors.dense(1.0, 1.0) + + val mu = Vectors.dense(0.0, 0.0) + val sigma1 = Matrices.dense(2, 2, Array(1.0, 0.0, 0.0, 1.0)) + val dist1 = new MultivariateGaussian(mu, sigma1) + assert(dist1.pdf(x1) ~== 0.15915 absTol 1E-5) + assert(dist1.pdf(x2) ~== 0.05855 absTol 1E-5) + + val sigma2 = Matrices.dense(2, 2, Array(4.0, -1.0, -1.0, 2.0)) + val dist2 = new MultivariateGaussian(mu, sigma2) + assert(dist2.pdf(x1) ~== 0.060155 absTol 1E-5) + assert(dist2.pdf(x2) ~== 0.033971 absTol 1E-5) + } + + test("multivariate degenerate") { + val x1 = Vectors.dense(0.0, 0.0) + val x2 = Vectors.dense(1.0, 1.0) + + val mu = Vectors.dense(0.0, 0.0) + val sigma = Matrices.dense(2, 2, Array(1.0, 1.0, 1.0, 1.0)) + val dist = new MultivariateGaussian(mu, sigma) + assert(dist.pdf(x1) ~== 0.11254 absTol 1E-5) + assert(dist.pdf(x2) ~== 0.068259 absTol 1E-5) + } + + test("SPARK-11302") { + val x = Vectors.dense(629, 640, 1.7188, 618.19) + val mu = Vectors.dense( + 1055.3910505836575, 1070.489299610895, 1.39020554474708, 1040.5907503867697) + val sigma = Matrices.dense(4, 4, Array( + 166769.00466698944, 169336.6705268059, 12.820670788921873, 164243.93314092053, + 169336.6705268059, 172041.5670061245, 21.62590020524533, 166678.01075856484, + 12.820670788921873, 21.62590020524533, 0.872524191943962, 4.283255814732373, + 164243.93314092053, 166678.01075856484, 4.283255814732373, 161848.9196719207)) + val dist = new MultivariateGaussian(mu, sigma) + // Agrees with R's dmvnorm: 7.154782e-05 + assert(dist.pdf(x) ~== 7.154782224045512E-5 absTol 1E-9) + } +} diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtils.scala b/mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtils.scala new file mode 100644 index 000000000000..6c79d77f142e --- /dev/null +++ b/mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtils.scala @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.util + +import org.scalatest.exceptions.TestFailedException + +import org.apache.spark.ml.linalg.{Matrix, Vector} + +object TestingUtils { + + val ABS_TOL_MSG = " using absolute tolerance" + val REL_TOL_MSG = " using relative tolerance" + + /** + * Private helper function for comparing two values using relative tolerance. + * Note that if x or y is extremely close to zero, i.e., smaller than Double.MinPositiveValue, + * the relative tolerance is meaningless, so the exception will be raised to warn users. + */ + private def RelativeErrorComparison(x: Double, y: Double, eps: Double): Boolean = { + // Special case for NaNs + if (x.isNaN && y.isNaN) { + return true + } + val absX = math.abs(x) + val absY = math.abs(y) + val diff = math.abs(x - y) + if (x == y) { + true + } else if (absX < Double.MinPositiveValue || absY < Double.MinPositiveValue) { + throw new TestFailedException( + s"$x or $y is extremely close to zero, so the relative tolerance is meaningless.", 0) + } else { + diff < eps * math.min(absX, absY) + } + } + + /** + * Private helper function for comparing two values using absolute tolerance. + */ + private def AbsoluteErrorComparison(x: Double, y: Double, eps: Double): Boolean = { + // Special case for NaNs + if (x.isNaN && y.isNaN) { + return true + } + math.abs(x - y) < eps + } + + case class CompareDoubleRightSide( + fun: (Double, Double, Double) => Boolean, y: Double, eps: Double, method: String) + + /** + * Implicit class for comparing two double values using relative tolerance or absolute tolerance. + */ + implicit class DoubleWithAlmostEquals(val x: Double) { + + /** + * When the difference of two values are within eps, returns true; otherwise, returns false. + */ + def ~=(r: CompareDoubleRightSide): Boolean = r.fun(x, r.y, r.eps) + + /** + * When the difference of two values are within eps, returns false; otherwise, returns true. + */ + def !~=(r: CompareDoubleRightSide): Boolean = !r.fun(x, r.y, r.eps) + + /** + * Throws exception when the difference of two values are NOT within eps; + * otherwise, returns true. + */ + def ~==(r: CompareDoubleRightSide): Boolean = { + if (!r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Expected $x and ${r.y} to be within ${r.eps}${r.method}.", 0) + } + true + } + + /** + * Throws exception when the difference of two values are within eps; otherwise, returns true. + */ + def !~==(r: CompareDoubleRightSide): Boolean = { + if (r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Did not expect $x and ${r.y} to be within ${r.eps}${r.method}.", 0) + } + true + } + + /** + * Comparison using absolute tolerance. + */ + def absTol(eps: Double): CompareDoubleRightSide = + CompareDoubleRightSide(AbsoluteErrorComparison, x, eps, ABS_TOL_MSG) + + /** + * Comparison using relative tolerance. + */ + def relTol(eps: Double): CompareDoubleRightSide = + CompareDoubleRightSide(RelativeErrorComparison, x, eps, REL_TOL_MSG) + + override def toString: String = x.toString + } + + case class CompareVectorRightSide( + fun: (Vector, Vector, Double) => Boolean, y: Vector, eps: Double, method: String) + + /** + * Implicit class for comparing two vectors using relative tolerance or absolute tolerance. + */ + implicit class VectorWithAlmostEquals(val x: Vector) { + + /** + * When the difference of two vectors are within eps, returns true; otherwise, returns false. + */ + def ~=(r: CompareVectorRightSide): Boolean = r.fun(x, r.y, r.eps) + + /** + * When the difference of two vectors are within eps, returns false; otherwise, returns true. + */ + def !~=(r: CompareVectorRightSide): Boolean = !r.fun(x, r.y, r.eps) + + /** + * Throws exception when the difference of two vectors are NOT within eps; + * otherwise, returns true. + */ + def ~==(r: CompareVectorRightSide): Boolean = { + if (!r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Expected $x and ${r.y} to be within ${r.eps}${r.method} for all elements.", 0) + } + true + } + + /** + * Throws exception when the difference of two vectors are within eps; otherwise, returns true. + */ + def !~==(r: CompareVectorRightSide): Boolean = { + if (r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Did not expect $x and ${r.y} to be within ${r.eps}${r.method} for all elements.", 0) + } + true + } + + /** + * Comparison using absolute tolerance. + */ + def absTol(eps: Double): CompareVectorRightSide = CompareVectorRightSide( + (x: Vector, y: Vector, eps: Double) => { + x.size == y.size && x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 absTol eps) + }, x, eps, ABS_TOL_MSG) + + /** + * Comparison using relative tolerance. Note that comparing against sparse vector + * with elements having value of zero will raise exception because it involves with + * comparing against zero. + */ + def relTol(eps: Double): CompareVectorRightSide = CompareVectorRightSide( + (x: Vector, y: Vector, eps: Double) => { + x.size == y.size && x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) + }, x, eps, REL_TOL_MSG) + + override def toString: String = x.toString + } + + case class CompareMatrixRightSide( + fun: (Matrix, Matrix, Double) => Boolean, y: Matrix, eps: Double, method: String) + + /** + * Implicit class for comparing two matrices using relative tolerance or absolute tolerance. + */ + implicit class MatrixWithAlmostEquals(val x: Matrix) { + + /** + * When the difference of two matrices are within eps, returns true; otherwise, returns false. + */ + def ~=(r: CompareMatrixRightSide): Boolean = r.fun(x, r.y, r.eps) + + /** + * When the difference of two matrices are within eps, returns false; otherwise, returns true. + */ + def !~=(r: CompareMatrixRightSide): Boolean = !r.fun(x, r.y, r.eps) + + /** + * Throws exception when the difference of two matrices are NOT within eps; + * otherwise, returns true. + */ + def ~==(r: CompareMatrixRightSide): Boolean = { + if (!r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Expected \n$x\n and \n${r.y}\n to be within ${r.eps}${r.method} for all elements.", 0) + } + true + } + + /** + * Throws exception when the difference of two matrices are within eps; otherwise, returns true. + */ + def !~==(r: CompareMatrixRightSide): Boolean = { + if (r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Did not expect \n$x\n and \n${r.y}\n to be within " + + s"${r.eps}${r.method} for all elements.", 0) + } + true + } + + /** + * Comparison using absolute tolerance. + */ + def absTol(eps: Double): CompareMatrixRightSide = CompareMatrixRightSide( + (x: Matrix, y: Matrix, eps: Double) => { + x.numRows == y.numRows && x.numCols == y.numCols && + x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 absTol eps) + }, x, eps, ABS_TOL_MSG) + + /** + * Comparison using relative tolerance. Note that comparing against sparse vector + * with elements having value of zero will raise exception because it involves with + * comparing against zero. + */ + def relTol(eps: Double): CompareMatrixRightSide = CompareMatrixRightSide( + (x: Matrix, y: Matrix, eps: Double) => { + x.numRows == y.numRows && x.numCols == y.numCols && + x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) + }, x, eps, REL_TOL_MSG) + + override def toString: String = x.toString + } + +} diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtilsSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtilsSuite.scala new file mode 100644 index 000000000000..2dc0ee32d576 --- /dev/null +++ b/mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtilsSuite.scala @@ -0,0 +1,460 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.util + +import org.scalatest.exceptions.TestFailedException + +import org.apache.spark.ml.SparkMLFunSuite +import org.apache.spark.ml.linalg.{Matrices, Vectors} +import org.apache.spark.ml.util.TestingUtils._ + +class TestingUtilsSuite extends SparkMLFunSuite { + + test("Comparing doubles using relative error.") { + + assert(23.1 ~== 23.52 relTol 0.02) + assert(23.1 ~== 22.74 relTol 0.02) + assert(23.1 ~= 23.52 relTol 0.02) + assert(23.1 ~= 22.74 relTol 0.02) + assert(!(23.1 !~= 23.52 relTol 0.02)) + assert(!(23.1 !~= 22.74 relTol 0.02)) + + // Should throw exception with message when test fails. + intercept[TestFailedException](23.1 !~== 23.52 relTol 0.02) + intercept[TestFailedException](23.1 !~== 22.74 relTol 0.02) + intercept[TestFailedException](23.1 ~== 23.63 relTol 0.02) + intercept[TestFailedException](23.1 ~== 22.34 relTol 0.02) + + assert(23.1 !~== 23.63 relTol 0.02) + assert(23.1 !~== 22.34 relTol 0.02) + assert(23.1 !~= 23.63 relTol 0.02) + assert(23.1 !~= 22.34 relTol 0.02) + assert(!(23.1 ~= 23.63 relTol 0.02)) + assert(!(23.1 ~= 22.34 relTol 0.02)) + + // Comparing against zero should fail the test and throw exception with message + // saying that the relative error is meaningless in this situation. + intercept[TestFailedException](0.1 ~== 0.0 relTol 0.032) + intercept[TestFailedException](0.1 ~= 0.0 relTol 0.032) + intercept[TestFailedException](0.1 !~== 0.0 relTol 0.032) + intercept[TestFailedException](0.1 !~= 0.0 relTol 0.032) + intercept[TestFailedException](0.0 ~== 0.1 relTol 0.032) + intercept[TestFailedException](0.0 ~= 0.1 relTol 0.032) + intercept[TestFailedException](0.0 !~== 0.1 relTol 0.032) + intercept[TestFailedException](0.0 !~= 0.1 relTol 0.032) + + // Comparisons of numbers very close to zero. + assert(10 * Double.MinPositiveValue ~== 9.5 * Double.MinPositiveValue relTol 0.01) + assert(10 * Double.MinPositiveValue !~== 11 * Double.MinPositiveValue relTol 0.01) + + assert(-Double.MinPositiveValue ~== 1.18 * -Double.MinPositiveValue relTol 0.012) + assert(-Double.MinPositiveValue ~== 1.38 * -Double.MinPositiveValue relTol 0.012) + } + + test("Comparing doubles using absolute error.") { + + assert(17.8 ~== 17.99 absTol 0.2) + assert(17.8 ~== 17.61 absTol 0.2) + assert(17.8 ~= 17.99 absTol 0.2) + assert(17.8 ~= 17.61 absTol 0.2) + assert(!(17.8 !~= 17.99 absTol 0.2)) + assert(!(17.8 !~= 17.61 absTol 0.2)) + + // Should throw exception with message when test fails. + intercept[TestFailedException](17.8 !~== 17.99 absTol 0.2) + intercept[TestFailedException](17.8 !~== 17.61 absTol 0.2) + intercept[TestFailedException](17.8 ~== 18.01 absTol 0.2) + intercept[TestFailedException](17.8 ~== 17.59 absTol 0.2) + + assert(17.8 !~== 18.01 absTol 0.2) + assert(17.8 !~== 17.59 absTol 0.2) + assert(17.8 !~= 18.01 absTol 0.2) + assert(17.8 !~= 17.59 absTol 0.2) + assert(!(17.8 ~= 18.01 absTol 0.2)) + assert(!(17.8 ~= 17.59 absTol 0.2)) + + // Comparisons of numbers very close to zero, and both side of zeros + assert( + Double.MinPositiveValue ~== 4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert( + Double.MinPositiveValue !~== 6 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + + assert( + -Double.MinPositiveValue ~== 3 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert( + Double.MinPositiveValue !~== -4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + } + + test("Comparing vectors using relative error.") { + + // Comparisons of two dense vectors + assert(Vectors.dense(Array(3.1, 3.5)) ~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array(3.1, 3.5)) !~== Vectors.dense(Array(3.135, 3.534)) relTol 0.01) + assert(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array(3.1, 3.5)) !~= Vectors.dense(Array(3.135, 3.534)) relTol 0.01) + assert(!(Vectors.dense(Array(3.1, 3.5)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01)) + assert(!(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.135, 3.534)) relTol 0.01)) + assert(Vectors.dense(Array(3.1)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array.empty[Double]) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array(3.1)) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array.empty[Double]) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + + // Should throw exception with message when test fails. + intercept[TestFailedException]( + Vectors.dense(Array(3.1, 3.5)) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + + intercept[TestFailedException]( + Vectors.dense(Array(3.1, 3.5)) ~== Vectors.dense(Array(3.135, 3.534)) relTol 0.01) + + intercept[TestFailedException]( + Vectors.dense(Array(3.1)) ~== Vectors.dense(Array(3.535, 3.534)) relTol 0.01) + + intercept[TestFailedException]( + Vectors.dense(Array.empty[Double]) ~== Vectors.dense(Array(3.135)) relTol 0.01) + + // Comparing against zero should fail the test and throw exception with message + // saying that the relative error is meaningless in this situation. + intercept[TestFailedException]( + Vectors.dense(Array(3.1, 0.01)) ~== Vectors.dense(Array(3.13, 0.0)) relTol 0.01) + + intercept[TestFailedException]( + Vectors.dense(Array(3.1, 0.01)) ~== Vectors.sparse(2, Array(0), Array(3.13)) relTol 0.01) + + // Comparisons of a sparse vector and a dense vector + assert(Vectors.dense(Array(3.1, 3.5)) ~== + Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) + + assert(Vectors.dense(Array(3.1, 3.5)) !~== + Vectors.sparse(2, Array(0, 1), Array(3.135, 3.534)) relTol 0.01) + + assert(Vectors.dense(Array(3.1)) !~== + Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) + + assert(Vectors.dense(Array.empty[Double]) !~== + Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) + } + + test("Comparing vectors using absolute error.") { + + // Comparisons of two dense vectors + assert(Vectors.dense(Array(3.1, 3.5, 0.0)) ~== + Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6) + + assert(Vectors.dense(Array(3.1, 3.5, 0.0)) !~== + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7, 1 + 1E-3)) absTol 1E-6) + + assert(Vectors.dense(Array(3.1, 3.5, 0.0)) ~= + Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6) + + assert(Vectors.dense(Array(3.1, 3.5, 0.0)) !~= + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7, 1 + 1E-3)) absTol 1E-6) + + assert(!(Vectors.dense(Array(3.1, 3.5, 0.0)) !~= + Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6)) + + assert(!(Vectors.dense(Array(3.1, 3.5, 0.0)) ~= + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7, 1 + 1E-3)) absTol 1E-6)) + + assert(Vectors.dense(Array(3.1)) !~= + Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5) + + assert(!(Vectors.dense(Array(3.1)) ~= + Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5)) + + assert(Vectors.dense(Array.empty[Double]) !~= + Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5) + + assert(!(Vectors.dense(Array.empty[Double]) ~= + Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5)) + + assert(Vectors.dense(Array.empty[Double]) ~= + Vectors.dense(Array.empty[Double]) absTol 1E-5) + + // Should throw exception with message when test fails. + intercept[TestFailedException](Vectors.dense(Array(3.1, 3.5, 0.0)) !~== + Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6) + + intercept[TestFailedException](Vectors.dense(Array(3.1, 3.5, 0.0)) ~== + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7, 1 + 1E-3)) absTol 1E-6) + + intercept[TestFailedException](Vectors.dense(Array(3.1)) ~== + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6) + + intercept[TestFailedException](Vectors.dense(Array.empty[Double]) ~== + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6) + + // Comparisons of two sparse vectors + assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) ~== + Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-8, 2.4 + 1E-7)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-8, 2.4 + 1E-7)) ~== + Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~== + Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-3, 2.4)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-3, 2.4)) !~== + Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-6, 2.4)) !~== + Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3) + + assert(Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) !~== + Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3) + + // Comparisons of a dense vector and a sparse vector + assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) ~== + Vectors.dense(Array(3.1 + 1E-8, 0, 2.4 + 1E-7)) absTol 1E-6) + + assert(Vectors.dense(Array(3.1 + 1E-8, 0, 2.4 + 1E-7)) ~== + Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~== + Vectors.dense(Array(3.1, 1E-3, 2.4)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~== + Vectors.dense(Array(3.1)) absTol 1E-6) + + assert(Vectors.dense(Array.empty[Double]) !~== + Vectors.sparse(3, Array(0, 2), Array(0, 2.4)) absTol 1E-6) + + assert(Vectors.sparse(1, Array(0), Array(3.1)) !~== + Vectors.dense(Array(3.1, 3.2)) absTol 1E-6) + + assert(Vectors.dense(Array(3.1)) !~== + Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) absTol 1E-6) + } + + test("Comparing Matrices using absolute error.") { + + // Comparisons of two dense Matrices + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 3.5 + 2E-7, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.1 + 1E-8, 3.5 + 2E-7, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.1 + 1E-5, 3.5 + 2E-6, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.1 + 1E-5, 3.5 + 2E-6, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(!(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.1 + 1E-5, 3.5 + 2E-6, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6)) + + assert(!(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.1 + 1E-7, 3.5 + 2E-8, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6)) + + assert(Matrices.dense(2, 1, Array(3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.1 + 1E-7, 3.5 + 2E-8, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.dense(2, 1, Array(3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.1 + 1E-7, 3.5 + 2E-8, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.dense(0, 0, Array()) !~= + Matrices.dense(2, 2, Array(3.1 + 1E-7, 3.5 + 2E-8, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.dense(0, 0, Array()) !~== + Matrices.dense(2, 2, Array(3.1 + 1E-7, 3.5 + 2E-8, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + // Should throw exception with message when test fails. + intercept[TestFailedException](Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 3.5 + 2E-7, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + intercept[TestFailedException](Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 3.5 + 2E-7, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-9) + + intercept[TestFailedException](Matrices.dense(2, 1, Array(3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 3.5 + 2E-7, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-5) + + intercept[TestFailedException](Matrices.dense(0, 0, Array()) ~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 3.5 + 2E-7, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-5) + + // Comparisons of two sparse Matrices + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) ~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) ~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-9) + + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) !~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-9) + + assert(!(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) ~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5)) absTol 1E-9)) + + assert(!(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5)) absTol 1E-6)) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-9) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.sparse(0, 0, Array(1), Array(0), Array(0)) !~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.sparse(0, 0, Array(1), Array(0), Array(0)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + // Comparisons of a dense Matrix and a sparse Matrix + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.1 + 1E-8, 0, 0, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 0, 0, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 0, 0, 3.5 + 1E-7)) absTol 1E-9) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 0, 0, 3.5 + 1E-7)) absTol 1E-9) + + assert(!(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.1 + 1E-8, 0, 0, 3.5 + 1E-7)) absTol 1E-9)) + + assert(!(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.1 + 1E-8, 0, 0, 3.5 + 1E-7)) absTol 1E-6)) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.dense(2, 1, Array(3.1 + 1E-8, 0)) absTol 1E-6) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.dense(2, 1, Array(3.1 + 1E-8, 0)) absTol 1E-6) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.dense(0, 0, Array()) absTol 1E-6) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.dense(0, 0, Array()) absTol 1E-6) + } + + test("Comparing Matrices using relative error.") { + + // Comparisons of two dense Matrices + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.130, 3.534, 3.130, 3.534)) relTol 0.01) + + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.130, 3.534, 3.130, 3.534)) relTol 0.01) + + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.135, 3.534, 3.135, 3.534)) relTol 0.01) + + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.135, 3.534, 3.135, 3.534)) relTol 0.01) + + assert(!(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.134, 3.535, 3.134, 3.535)) relTol 0.01)) + + assert(!(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.130, 3.534, 3.130, 3.534)) relTol 0.01)) + + assert(Matrices.dense(2, 1, Array(3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) relTol 0.01) + + assert(Matrices.dense(2, 1, Array(3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) relTol 0.01) + + assert(Matrices.dense(0, 0, Array()) !~= + Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) relTol 0.01) + + assert(Matrices.dense(0, 0, Array()) !~== + Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) relTol 0.01) + + // Should throw exception with message when test fails. + intercept[TestFailedException](Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.130, 3.534, 3.130, 3.534)) relTol 0.01) + + intercept[TestFailedException](Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.135, 3.534, 3.135, 3.534)) relTol 0.01) + + intercept[TestFailedException](Matrices.dense(2, 1, Array(3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) relTol 0.01) + + intercept[TestFailedException](Matrices.dense(0, 0, Array()) ~== + Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) relTol 0.01) + + // Comparisons of two sparse Matrices + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) ~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.130, 3.534)) relTol 0.01) + + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) ~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.130, 3.534)) relTol 0.01) + + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.135, 3.534)) relTol 0.01) + + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) !~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.135, 3.534)) relTol 0.01) + + assert(!(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) ~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.135, 3.534)) relTol 0.01)) + + assert(!(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.130, 3.534)) relTol 0.01)) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) relTol 0.01) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) relTol 0.01) + + assert(Matrices.sparse(0, 0, Array(1), Array(0), Array(0)) !~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) relTol 0.01) + + assert(Matrices.sparse(0, 0, Array(1), Array(0), Array(0)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) relTol 0.01) + + // Comparisons of a dense Matrix and a sparse Matrix + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.130, 0, 0, 3.534)) relTol 0.01) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.130, 0, 0, 3.534)) relTol 0.01) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.135, 0, 0, 3.534)) relTol 0.01) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.135, 0, 0, 3.534)) relTol 0.01) + + assert(!(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.135, 0, 0, 3.534)) relTol 0.01)) + + assert(!(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.130, 0, 0, 3.534)) relTol 0.01)) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.dense(2, 1, Array(3.1, 0)) relTol 0.01) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.dense(2, 1, Array(3.1, 0)) relTol 0.01) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.dense(0, 0, Array()) relTol 0.01) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.dense(0, 0, Array()) relTol 0.01) + } +} diff --git a/mllib/pom.xml b/mllib/pom.xml index 70139121d8c7..925b5422a54c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -20,13 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT + spark-parent_2.11 + 2.3.0-SNAPSHOT ../pom.xml - org.apache.spark - spark-mllib_2.10 + spark-mllib_2.11 mllib @@ -35,6 +34,10 @@ http://spark.apache.org/ + + org.scala-lang.modules + scala-parser-combinators_${scala.binary.version} + org.apache.spark spark-core_${scala.binary.version} @@ -63,27 +66,20 @@ ${project.version} - org.jblas - jblas - ${jblas.version} + org.apache.spark + spark-mllib-local_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-mllib-local_${scala.binary.version} + ${project.version} + test-jar test org.scalanlp breeze_${scala.binary.version} - 0.11.2 - - - - junit - junit - - - org.apache.commons - commons-math3 - - org.apache.commons @@ -109,22 +105,24 @@ org.jpmml pmml-model - 1.1.15 - - - com.sun.xml.fastinfoset - FastInfoset - - - com.sun.istack - istack-commons-runtime - - + compile org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + @@ -139,8 +137,38 @@ + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-dependency-plugin + + + + + copy-dependencies + package + + copy-dependencies + + + ${project.build.directory} + false + false + true + true + org.jpmml + true + + + + + + diff --git a/mllib/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/mllib/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index f632dd603c44..a865cbe19b18 100644 --- a/mllib/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/mllib/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -1 +1 @@ -org.apache.spark.ml.source.libsvm.DefaultSource +org.apache.spark.ml.source.libsvm.LibSVMFileFormat diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/README b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/README new file mode 100755 index 000000000000..ec08a5080774 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/README @@ -0,0 +1,12 @@ +Stopwords Corpus + +This corpus contains lists of stop words for several languages. These +are high-frequency grammatical words which are usually ignored in text +retrieval applications. + +They were obtained from: +http://anoncvs.postgresql.org/cvsweb.cgi/pgsql/src/backend/snowball/stopwords/ + +The English list has been augmented +https://github.com/nltk/nltk_data/issues/22 + diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/danish.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/danish.txt new file mode 100644 index 000000000000..ea9e2c4abe5b --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/danish.txt @@ -0,0 +1,94 @@ +og +i +jeg +det +at +en +den +til +er +som +på +de +med +han +af +for +ikke +der +var +mig +sig +men +et +har +om +vi +min +havde +ham +hun +nu +over +da +fra +du +ud +sin +dem +os +op +man +hans +hvor +eller +hvad +skal +selv +her +alle +vil +blev +kunne +ind +når +være +dog +noget +ville +jo +deres +efter +ned +skulle +denne +end +dette +mit +også +under +have +dig +anden +hende +mine +alt +meget +sit +sine +vor +mod +disse +hvis +din +nogle +hos +blive +mange +ad +bliver +hendes +været +thi +jer +sådan \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/dutch.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/dutch.txt new file mode 100644 index 000000000000..023cc2c939b2 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/dutch.txt @@ -0,0 +1,101 @@ +de +en +van +ik +te +dat +die +in +een +hij +het +niet +zijn +is +was +op +aan +met +als +voor +had +er +maar +om +hem +dan +zou +of +wat +mijn +men +dit +zo +door +over +ze +zich +bij +ook +tot +je +mij +uit +der +daar +haar +naar +heb +hoe +heeft +hebben +deze +u +want +nog +zal +me +zij +nu +ge +geen +omdat +iets +worden +toch +al +waren +veel +meer +doen +toen +moet +ben +zonder +kan +hun +dus +alles +onder +ja +eens +hier +wie +werd +altijd +doch +wordt +wezen +kunnen +ons +zelf +tegen +na +reeds +wil +kon +niets +uw +iemand +geweest +andere \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/english.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/english.txt new file mode 100644 index 000000000000..d6094d774a5b --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/english.txt @@ -0,0 +1,181 @@ +i +me +my +myself +we +our +ours +ourselves +you +your +yours +yourself +yourselves +he +him +his +himself +she +her +hers +herself +it +its +itself +they +them +their +theirs +themselves +what +which +who +whom +this +that +these +those +am +is +are +was +were +be +been +being +have +has +had +having +do +does +did +doing +a +an +the +and +but +if +or +because +as +until +while +of +at +by +for +with +about +against +between +into +through +during +before +after +above +below +to +from +up +down +in +out +on +off +over +under +again +further +then +once +here +there +when +where +why +how +all +any +both +each +few +more +most +other +some +such +no +nor +not +only +own +same +so +than +too +very +s +t +can +will +just +don +should +now +i'll +you'll +he'll +she'll +we'll +they'll +i'd +you'd +he'd +she'd +we'd +they'd +i'm +you're +he's +she's +it's +we're +they're +i've +we've +you've +they've +isn't +aren't +wasn't +weren't +haven't +hasn't +hadn't +don't +doesn't +didn't +won't +wouldn't +shan't +shouldn't +mustn't +can't +couldn't +cannot +could +here's +how's +let's +ought +that's +there's +what's +when's +where's +who's +why's +would \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/finnish.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/finnish.txt new file mode 100644 index 000000000000..5b0eb10777d0 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/finnish.txt @@ -0,0 +1,235 @@ +olla +olen +olet +on +olemme +olette +ovat +ole +oli +olisi +olisit +olisin +olisimme +olisitte +olisivat +olit +olin +olimme +olitte +olivat +ollut +olleet +en +et +ei +emme +ette +eivät +minä +minun +minut +minua +minussa +minusta +minuun +minulla +minulta +minulle +sinä +sinun +sinut +sinua +sinussa +sinusta +sinuun +sinulla +sinulta +sinulle +hän +hänen +hänet +häntä +hänessä +hänestä +häneen +hänellä +häneltä +hänelle +me +meidän +meidät +meitä +meissä +meistä +meihin +meillä +meiltä +meille +te +teidän +teidät +teitä +teissä +teistä +teihin +teillä +teiltä +teille +he +heidän +heidät +heitä +heissä +heistä +heihin +heillä +heiltä +heille +tämä +tämän +tätä +tässä +tästä +tähän +tallä +tältä +tälle +tänä +täksi +tuo +tuon +tuotä +tuossa +tuosta +tuohon +tuolla +tuolta +tuolle +tuona +tuoksi +se +sen +sitä +siinä +siitä +siihen +sillä +siltä +sille +sinä +siksi +nämä +näiden +näitä +näissä +näistä +näihin +näillä +näiltä +näille +näinä +näiksi +nuo +noiden +noita +noissa +noista +noihin +noilla +noilta +noille +noina +noiksi +ne +niiden +niitä +niissä +niistä +niihin +niillä +niiltä +niille +niinä +niiksi +kuka +kenen +kenet +ketä +kenessä +kenestä +keneen +kenellä +keneltä +kenelle +kenenä +keneksi +ketkä +keiden +ketkä +keitä +keissä +keistä +keihin +keillä +keiltä +keille +keinä +keiksi +mikä +minkä +minkä +mitä +missä +mistä +mihin +millä +miltä +mille +minä +miksi +mitkä +joka +jonka +jota +jossa +josta +johon +jolla +jolta +jolle +jona +joksi +jotka +joiden +joita +joissa +joista +joihin +joilla +joilta +joille +joina +joiksi +että +ja +jos +koska +kuin +mutta +niin +sekä +sillä +tai +vaan +vai +vaikka +kanssa +mukaan +noin +poikki +yli +kun +niin +nyt +itse \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/french.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/french.txt new file mode 100644 index 000000000000..a59a0424616c --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/french.txt @@ -0,0 +1,156 @@ +au +aux +avec +ce +ces +dans +de +des +du +elle +en +et +eux +il +je +la +le +les +leur +lui +ma +mais +me +même +mes +moi +mon +ne +nos +notre +nous +on +ou +par +pas +pour +qu +que +qui +sa +se +ses +son +sur +ta +te +tes +toi +ton +tu +un +une +vos +votre +vous +c +d +j +l +à +m +n +s +t +y +été +étée +étées +étés +étant +étante +étants +étantes +suis +es +est +sommes +êtes +sont +serai +seras +sera +serons +serez +seront +serais +serait +serions +seriez +seraient +étais +était +étions +étiez +étaient +fus +fut +fûmes +fûtes +furent +sois +soit +soyons +soyez +soient +fusse +fusses +fût +fussions +fussiez +fussent +ayant +ayante +ayantes +ayants +eu +eue +eues +eus +ai +as +avons +avez +ont +aurai +auras +aura +aurons +aurez +auront +aurais +aurait +aurions +auriez +auraient +avais +avait +avions +aviez +avaient +eut +eûmes +eûtes +eurent +aie +aies +ait +ayons +ayez +aient +eusse +eusses +eût +eussions +eussiez +eussent diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/german.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/german.txt new file mode 100644 index 000000000000..7e65190f8ba2 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/german.txt @@ -0,0 +1,231 @@ +aber +alle +allem +allen +aller +alles +als +also +am +an +ander +andere +anderem +anderen +anderer +anderes +anderm +andern +anderr +anders +auch +auf +aus +bei +bin +bis +bist +da +damit +dann +der +den +des +dem +die +das +daß +derselbe +derselben +denselben +desselben +demselben +dieselbe +dieselben +dasselbe +dazu +dein +deine +deinem +deinen +deiner +deines +denn +derer +dessen +dich +dir +du +dies +diese +diesem +diesen +dieser +dieses +doch +dort +durch +ein +eine +einem +einen +einer +eines +einig +einige +einigem +einigen +einiger +einiges +einmal +er +ihn +ihm +es +etwas +euer +eure +eurem +euren +eurer +eures +für +gegen +gewesen +hab +habe +haben +hat +hatte +hatten +hier +hin +hinter +ich +mich +mir +ihr +ihre +ihrem +ihren +ihrer +ihres +euch +im +in +indem +ins +ist +jede +jedem +jeden +jeder +jedes +jene +jenem +jenen +jener +jenes +jetzt +kann +kein +keine +keinem +keinen +keiner +keines +können +könnte +machen +man +manche +manchem +manchen +mancher +manches +mein +meine +meinem +meinen +meiner +meines +mit +muss +musste +nach +nicht +nichts +noch +nun +nur +ob +oder +ohne +sehr +sein +seine +seinem +seinen +seiner +seines +selbst +sich +sie +ihnen +sind +so +solche +solchem +solchen +solcher +solches +soll +sollte +sondern +sonst +über +um +und +uns +unse +unsem +unsen +unser +unses +unter +viel +vom +von +vor +während +war +waren +warst +was +weg +weil +weiter +welche +welchem +welchen +welcher +welches +wenn +werde +werden +wie +wieder +will +wir +wird +wirst +wo +wollen +wollte +würde +würden +zu +zum +zur +zwar +zwischen \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt new file mode 100644 index 000000000000..8d4543a0965d --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt @@ -0,0 +1,199 @@ +a +ahogy +ahol +aki +akik +akkor +alatt +által +általában +amely +amelyek +amelyekben +amelyeket +amelyet +amelynek +ami +amit +amolyan +amíg +amikor +át +abban +ahhoz +annak +arra +arról +az +azok +azon +azt +azzal +azért +aztán +azután +azonban +bár +be +belül +benne +cikk +cikkek +cikkeket +csak +de +e +eddig +egész +egy +egyes +egyetlen +egyéb +egyik +egyre +ekkor +el +elég +ellen +elõ +elõször +elõtt +elsõ +én +éppen +ebben +ehhez +emilyen +ennek +erre +ez +ezt +ezek +ezen +ezzel +ezért +és +fel +felé +hanem +hiszen +hogy +hogyan +igen +így +illetve +ill. +ill +ilyen +ilyenkor +ison +ismét +itt +jó +jól +jobban +kell +kellett +keresztül +keressünk +ki +kívül +között +közül +legalább +lehet +lehetett +legyen +lenne +lenni +lesz +lett +maga +magát +majd +majd +már +más +másik +meg +még +mellett +mert +mely +melyek +mi +mit +míg +miért +milyen +mikor +minden +mindent +mindenki +mindig +mint +mintha +mivel +most +nagy +nagyobb +nagyon +ne +néha +nekem +neki +nem +néhány +nélkül +nincs +olyan +ott +össze +õ +õk +õket +pedig +persze +rá +s +saját +sem +semmi +sok +sokat +sokkal +számára +szemben +szerint +szinte +talán +tehát +teljes +tovább +továbbá +több +úgy +ugyanis +új +újabb +újra +után +utána +utolsó +vagy +vagyis +valaki +valami +valamint +való +vagyok +van +vannak +volt +voltam +voltak +voltunk +vissza +vele +viszont +volna \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/italian.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/italian.txt new file mode 100644 index 000000000000..783b2e0cbfcd --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/italian.txt @@ -0,0 +1,279 @@ +ad +al +allo +ai +agli +all +agl +alla +alle +con +col +coi +da +dal +dallo +dai +dagli +dall +dagl +dalla +dalle +di +del +dello +dei +degli +dell +degl +della +delle +in +nel +nello +nei +negli +nell +negl +nella +nelle +su +sul +sullo +sui +sugli +sull +sugl +sulla +sulle +per +tra +contro +io +tu +lui +lei +noi +voi +loro +mio +mia +miei +mie +tuo +tua +tuoi +tue +suo +sua +suoi +sue +nostro +nostra +nostri +nostre +vostro +vostra +vostri +vostre +mi +ti +ci +vi +lo +la +li +le +gli +ne +il +un +uno +una +ma +ed +se +perché +anche +come +dov +dove +che +chi +cui +non +più +quale +quanto +quanti +quanta +quante +quello +quelli +quella +quelle +questo +questi +questa +queste +si +tutto +tutti +a +c +e +i +l +o +ho +hai +ha +abbiamo +avete +hanno +abbia +abbiate +abbiano +avrò +avrai +avrà +avremo +avrete +avranno +avrei +avresti +avrebbe +avremmo +avreste +avrebbero +avevo +avevi +aveva +avevamo +avevate +avevano +ebbi +avesti +ebbe +avemmo +aveste +ebbero +avessi +avesse +avessimo +avessero +avendo +avuto +avuta +avuti +avute +sono +sei +è +siamo +siete +sia +siate +siano +sarò +sarai +sarà +saremo +sarete +saranno +sarei +saresti +sarebbe +saremmo +sareste +sarebbero +ero +eri +era +eravamo +eravate +erano +fui +fosti +fu +fummo +foste +furono +fossi +fosse +fossimo +fossero +essendo +faccio +fai +facciamo +fanno +faccia +facciate +facciano +farò +farai +farà +faremo +farete +faranno +farei +faresti +farebbe +faremmo +fareste +farebbero +facevo +facevi +faceva +facevamo +facevate +facevano +feci +facesti +fece +facemmo +faceste +fecero +facessi +facesse +facessimo +facessero +facendo +sto +stai +sta +stiamo +stanno +stia +stiate +stiano +starò +starai +starà +staremo +starete +staranno +starei +staresti +starebbe +staremmo +stareste +starebbero +stavo +stavi +stava +stavamo +stavate +stavano +stetti +stesti +stette +stemmo +steste +stettero +stessi +stesse +stessimo +stessero +stando \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/norwegian.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/norwegian.txt new file mode 100644 index 000000000000..cb91702c5e9a --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/norwegian.txt @@ -0,0 +1,176 @@ +og +i +jeg +det +at +en +et +den +til +er +som +på +de +med +han +av +ikke +ikkje +der +så +var +meg +seg +men +ett +har +om +vi +min +mitt +ha +hadde +hun +nå +over +da +ved +fra +du +ut +sin +dem +oss +opp +man +kan +hans +hvor +eller +hva +skal +selv +sjøl +her +alle +vil +bli +ble +blei +blitt +kunne +inn +når +være +kom +noen +noe +ville +dere +som +deres +kun +ja +etter +ned +skulle +denne +for +deg +si +sine +sitt +mot +å +meget +hvorfor +dette +disse +uten +hvordan +ingen +din +ditt +blir +samme +hvilken +hvilke +sånn +inni +mellom +vår +hver +hvem +vors +hvis +både +bare +enn +fordi +før +mange +også +slik +vært +være +båe +begge +siden +dykk +dykkar +dei +deira +deires +deim +di +då +eg +ein +eit +eitt +elles +honom +hjå +ho +hoe +henne +hennar +hennes +hoss +hossen +ikkje +ingi +inkje +korleis +korso +kva +kvar +kvarhelst +kven +kvi +kvifor +me +medan +mi +mine +mykje +no +nokon +noka +nokor +noko +nokre +si +sia +sidan +so +somt +somme +um +upp +vere +vore +verte +vort +varte +vart \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt new file mode 100644 index 000000000000..98b4fdcdf7a2 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt @@ -0,0 +1,203 @@ +de +a +o +que +e +do +da +em +um +para +com +não +uma +os +no +se +na +por +mais +as +dos +como +mas +ao +ele +das +à +seu +sua +ou +quando +muito +nos +já +eu +também +só +pelo +pela +até +isso +ela +entre +depois +sem +mesmo +aos +seus +quem +nas +me +esse +eles +você +essa +num +nem +suas +meu +às +minha +numa +pelos +elas +qual +nós +lhe +deles +essas +esses +pelas +este +dele +tu +te +vocês +vos +lhes +meus +minhas +teu +tua +teus +tuas +nosso +nossa +nossos +nossas +dela +delas +esta +estes +estas +aquele +aquela +aqueles +aquelas +isto +aquilo +estou +está +estamos +estão +estive +esteve +estivemos +estiveram +estava +estávamos +estavam +estivera +estivéramos +esteja +estejamos +estejam +estivesse +estivéssemos +estivessem +estiver +estivermos +estiverem +hei +há +havemos +hão +houve +houvemos +houveram +houvera +houvéramos +haja +hajamos +hajam +houvesse +houvéssemos +houvessem +houver +houvermos +houverem +houverei +houverá +houveremos +houverão +houveria +houveríamos +houveriam +sou +somos +são +era +éramos +eram +fui +foi +fomos +foram +fora +fôramos +seja +sejamos +sejam +fosse +fôssemos +fossem +for +formos +forem +serei +será +seremos +serão +seria +seríamos +seriam +tenho +tem +temos +tém +tinha +tínhamos +tinham +tive +teve +tivemos +tiveram +tivera +tivéramos +tenha +tenhamos +tenham +tivesse +tivéssemos +tivessem +tiver +tivermos +tiverem +terei +terá +teremos +terão +teria +teríamos +teriam \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/russian.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/russian.txt new file mode 100644 index 000000000000..8a800b74497d --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/russian.txt @@ -0,0 +1,151 @@ +и +в +во +не +что +он +на +я +с +со +как +а +то +все +она +так +его +но +да +ты +к +у +же +вы +за +бы +по +только +ее +мне +было +вот +от +меня +еще +нет +о +из +ему +теперь +когда +даже +ну +вдруг +ли +если +уже +или +ни +быть +был +него +до +вас +нибудь +опять +уж +вам +ведь +там +потом +себя +ничего +ей +может +они +тут +где +есть +надо +ней +для +мы +тебя +их +чем +была +сам +чтоб +без +будто +чего +раз +тоже +себе +под +будет +ж +тогда +кто +этот +того +потому +этого +какой +совсем +ним +здесь +этом +один +почти +мой +тем +чтобы +нее +сейчас +были +куда +зачем +всех +никогда +можно +при +наконец +два +об +другой +хоть +после +над +больше +тот +через +эти +нас +про +всего +них +какая +много +разве +три +эту +моя +впрочем +хорошо +свою +этой +перед +иногда +лучше +чуть +том +нельзя +такой +им +более +всегда +конечно +всю +между \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/spanish.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/spanish.txt new file mode 100644 index 000000000000..94f493a8d1e0 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/spanish.txt @@ -0,0 +1,313 @@ +de +la +que +el +en +y +a +los +del +se +las +por +un +para +con +no +una +su +al +lo +como +más +pero +sus +le +ya +o +este +sí +porque +esta +entre +cuando +muy +sin +sobre +también +me +hasta +hay +donde +quien +desde +todo +nos +durante +todos +uno +les +ni +contra +otros +ese +eso +ante +ellos +e +esto +mí +antes +algunos +qué +unos +yo +otro +otras +otra +él +tanto +esa +estos +mucho +quienes +nada +muchos +cual +poco +ella +estar +estas +algunas +algo +nosotros +mi +mis +tú +te +ti +tu +tus +ellas +nosotras +vosostros +vosostras +os +mío +mía +míos +mías +tuyo +tuya +tuyos +tuyas +suyo +suya +suyos +suyas +nuestro +nuestra +nuestros +nuestras +vuestro +vuestra +vuestros +vuestras +esos +esas +estoy +estás +está +estamos +estáis +están +esté +estés +estemos +estéis +estén +estaré +estarás +estará +estaremos +estaréis +estarán +estaría +estarías +estaríamos +estaríais +estarían +estaba +estabas +estábamos +estabais +estaban +estuve +estuviste +estuvo +estuvimos +estuvisteis +estuvieron +estuviera +estuvieras +estuviéramos +estuvierais +estuvieran +estuviese +estuvieses +estuviésemos +estuvieseis +estuviesen +estando +estado +estada +estados +estadas +estad +he +has +ha +hemos +habéis +han +haya +hayas +hayamos +hayáis +hayan +habré +habrás +habrá +habremos +habréis +habrán +habría +habrías +habríamos +habríais +habrían +había +habías +habíamos +habíais +habían +hube +hubiste +hubo +hubimos +hubisteis +hubieron +hubiera +hubieras +hubiéramos +hubierais +hubieran +hubiese +hubieses +hubiésemos +hubieseis +hubiesen +habiendo +habido +habida +habidos +habidas +soy +eres +es +somos +sois +son +sea +seas +seamos +seáis +sean +seré +serás +será +seremos +seréis +serán +sería +serías +seríamos +seríais +serían +era +eras +éramos +erais +eran +fui +fuiste +fue +fuimos +fuisteis +fueron +fuera +fueras +fuéramos +fuerais +fueran +fuese +fueses +fuésemos +fueseis +fuesen +sintiendo +sentido +sentida +sentidos +sentidas +siente +sentid +tengo +tienes +tiene +tenemos +tenéis +tienen +tenga +tengas +tengamos +tengáis +tengan +tendré +tendrás +tendrá +tendremos +tendréis +tendrán +tendría +tendrías +tendríamos +tendríais +tendrían +tenía +tenías +teníamos +teníais +tenían +tuve +tuviste +tuvo +tuvimos +tuvisteis +tuvieron +tuviera +tuvieras +tuviéramos +tuvierais +tuvieran +tuviese +tuvieses +tuviésemos +tuvieseis +tuviesen +teniendo +tenido +tenida +tenidos +tenidas +tened \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/swedish.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/swedish.txt new file mode 100644 index 000000000000..9fae31c1858a --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/swedish.txt @@ -0,0 +1,114 @@ +och +det +att +i +en +jag +hon +som +han +på +den +med +var +sig +för +så +till +är +men +ett +om +hade +de +av +icke +mig +du +henne +då +sin +nu +har +inte +hans +honom +skulle +hennes +där +min +man +ej +vid +kunde +något +från +ut +när +efter +upp +vi +dem +vara +vad +över +än +dig +kan +sina +här +ha +mot +alla +under +någon +eller +allt +mycket +sedan +ju +denna +själv +detta +åt +utan +varit +hur +ingen +mitt +ni +bli +blev +oss +din +dessa +några +deras +blir +mina +samma +vilken +er +sådan +vår +blivit +dess +inom +mellan +sådant +varför +varje +vilka +ditt +vem +vilket +sitta +sådana +vart +dina +vars +vårt +våra +ert +era +vilkas \ No newline at end of file diff --git a/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/turkish.txt b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/turkish.txt new file mode 100644 index 000000000000..4e9708d9d2c5 --- /dev/null +++ b/mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/turkish.txt @@ -0,0 +1,53 @@ +acaba +ama +aslında +az +bazı +belki +biri +birkaç +birşey +biz +bu +çok +çünkü +da +daha +de +defa +diye +eğer +en +gibi +hem +hep +hepsi +her +hiç +için +ile +ise +kez +ki +kim +mı +mu +mü +nasıl +ne +neden +nerde +nerede +nereye +niçin +niye +o +sanki +şey +siz +şu +tüm +ve +veya +ya +yani \ No newline at end of file diff --git a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala index 57e416591de6..1247882d6c1b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala @@ -19,9 +19,9 @@ package org.apache.spark.ml import scala.annotation.varargs -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.ml.param.{ParamMap, ParamPair} -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.Dataset /** * :: DeveloperApi :: @@ -39,8 +39,9 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage { * Estimator's embedded ParamMap. * @return fitted model */ + @Since("2.0.0") @varargs - def fit(dataset: DataFrame, firstParamPair: ParamPair[_], otherParamPairs: ParamPair[_]*): M = { + def fit(dataset: Dataset[_], firstParamPair: ParamPair[_], otherParamPairs: ParamPair[_]*): M = { val map = new ParamMap() .put(firstParamPair) .put(otherParamPairs: _*) @@ -55,14 +56,16 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage { * These values override any specified in this Estimator's embedded ParamMap. * @return fitted model */ - def fit(dataset: DataFrame, paramMap: ParamMap): M = { + @Since("2.0.0") + def fit(dataset: Dataset[_], paramMap: ParamMap): M = { copy(paramMap).fit(dataset) } /** * Fits a model to the input data. */ - def fit(dataset: DataFrame): M + @Since("2.0.0") + def fit(dataset: Dataset[_]): M /** * Fits multiple models to the input data with multiple sets of parameters. @@ -74,7 +77,8 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage { * These values override any specified in this Estimator's embedded ParamMap. * @return fitted models, matching the input parameter maps */ - def fit(dataset: DataFrame, paramMaps: Array[ParamMap]): Seq[M] = { + @Since("2.0.0") + def fit(dataset: Dataset[_], paramMaps: Array[ParamMap]): Seq[M] = { paramMaps.map(fit(dataset, _)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Model.scala b/mllib/src/main/scala/org/apache/spark/ml/Model.scala index 252acc156583..c581fed17727 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Model.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Model.scala @@ -30,7 +30,7 @@ import org.apache.spark.ml.param.ParamMap abstract class Model[M <: Model[M]] extends Transformer { /** * The parent estimator that produced this model. - * Note: For ensembles' component Models, this value can be null. + * @note For ensembles' component Models, this value can be null. */ @transient var parent: Estimator[M] = _ 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 a3e59401c5cf..103082b7b976 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -22,11 +22,16 @@ import java.{util => ju} import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer -import org.apache.spark.Logging -import org.apache.spark.annotation.{DeveloperApi, Experimental} +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, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.param.{Param, ParamMap, Params} -import org.apache.spark.ml.util.Identifiable -import org.apache.spark.sql.DataFrame +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.StructType /** @@ -39,7 +44,14 @@ abstract class PipelineStage extends Params with Logging { /** * :: DeveloperApi :: * - * Derives the output schema from the input schema. + * Check transform validity and derive the output schema from the input schema. + * + * We check validity for interactions between parameters during `transformSchema` and + * raise an exception if any parameter value is invalid. Parameter value checks which + * do not depend on other parameters are handled by `Param.validate()`. + * + * Typical implementation should first conduct verification on schema change and parameter + * validity, including complex parameter interaction checks. */ @DeveloperApi def transformSchema(schema: StructType): StructType @@ -70,46 +82,48 @@ abstract class PipelineStage extends Params with Logging { } /** - * :: Experimental :: * A simple pipeline, which acts as an estimator. A Pipeline consists of a sequence of stages, each - * of which is either an [[Estimator]] or a [[Transformer]]. When [[Pipeline#fit]] is called, the - * stages are executed in order. If a stage is an [[Estimator]], its [[Estimator#fit]] method will + * of which is either an [[Estimator]] or a [[Transformer]]. When `Pipeline.fit` is called, the + * stages are executed in order. If a stage is an [[Estimator]], its `Estimator.fit` method will * be called on the input dataset to fit a model. Then the model, which is a transformer, will be * used to transform the dataset as the input to the next stage. If a stage is a [[Transformer]], - * its [[Transformer#transform]] method will be called to produce the dataset for the next stage. - * The fitted model from a [[Pipeline]] is an [[PipelineModel]], which consists of fitted models and + * its `Transformer.transform` method will be called to produce the dataset for the next stage. + * The fitted model from a [[Pipeline]] is a [[PipelineModel]], which consists of fitted models and * transformers, corresponding to the pipeline stages. If there are no stages, the pipeline acts as * an identity transformer. */ -@Experimental -class Pipeline(override val uid: String) extends Estimator[PipelineModel] { +@Since("1.2.0") +class Pipeline @Since("1.4.0") ( + @Since("1.4.0") override val uid: String) extends Estimator[PipelineModel] with MLWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("pipeline")) /** * param for pipeline stages * @group param */ + @Since("1.2.0") val stages: Param[Array[PipelineStage]] = new Param(this, "stages", "stages of the pipeline") /** @group setParam */ - def setStages(value: Array[PipelineStage]): this.type = { set(stages, value); this } + @Since("1.2.0") + def setStages(value: Array[_ <: PipelineStage]): this.type = { + set(stages, value.asInstanceOf[Array[PipelineStage]]) + this + } // Below, we clone stages so that modifications to the list of stages will not change // the Param value in the Pipeline. /** @group getParam */ + @Since("1.2.0") def getStages: Array[PipelineStage] = $(stages).clone() - override def validateParams(): Unit = { - super.validateParams() - $(stages).foreach(_.validateParams()) - } - /** * Fits the pipeline to the input dataset with additional parameters. If a stage is an - * [[Estimator]], its [[Estimator#fit]] method will be called on the input dataset to fit a model. + * [[Estimator]], its `Estimator.fit` method will be called on the input dataset to fit a model. * Then the model, which is a transformer, will be used to transform the dataset as the input to - * the next stage. If a stage is a [[Transformer]], its [[Transformer#transform]] method will be + * the next stage. If a stage is a [[Transformer]], its `Transformer.transform` method will be * called to produce the dataset for the next stage. The fitted model from a [[Pipeline]] is an * [[PipelineModel]], which consists of fitted models and transformers, corresponding to the * pipeline stages. If there are no stages, the output model acts as an identity transformer. @@ -117,7 +131,8 @@ class Pipeline(override val uid: String) extends Estimator[PipelineModel] { * @param dataset input dataset * @return fitted pipeline */ - override def fit(dataset: DataFrame): PipelineModel = { + @Since("2.0.0") + override def fit(dataset: Dataset[_]): PipelineModel = { transformSchema(dataset.schema, logging = true) val theStages = $(stages) // Search for the last estimator. @@ -140,7 +155,7 @@ class Pipeline(override val uid: String) extends Estimator[PipelineModel] { t case _ => throw new IllegalArgumentException( - s"Do not support stage $stage of type ${stage.getClass}") + s"Does not support stage $stage of type ${stage.getClass}") } if (index < indexOfLastEstimator) { curDataset = transformer.transform(curDataset) @@ -154,50 +169,189 @@ class Pipeline(override val uid: String) extends Estimator[PipelineModel] { new PipelineModel(uid, transformers.toArray).setParent(this) } + @Since("1.4.0") override def copy(extra: ParamMap): Pipeline = { val map = extractParamMap(extra) val newStages = map(stages).map(_.copy(extra)) - new Pipeline().setStages(newStages) + new Pipeline(uid).setStages(newStages) } + @Since("1.2.0") override def transformSchema(schema: StructType): StructType = { val theStages = $(stages) require(theStages.toSet.size == theStages.length, "Cannot have duplicate components in a pipeline.") theStages.foldLeft(schema)((cur, stage) => stage.transformSchema(cur)) } + + @Since("1.6.0") + override def write: MLWriter = new Pipeline.PipelineWriter(this) +} + +@Since("1.6.0") +object Pipeline extends MLReadable[Pipeline] { + + @Since("1.6.0") + override def read: MLReader[Pipeline] = new PipelineReader + + @Since("1.6.0") + override def load(path: String): Pipeline = super.load(path) + + private[Pipeline] class PipelineWriter(instance: Pipeline) extends MLWriter { + + SharedReadWrite.validateStages(instance.getStages) + + override protected def saveImpl(path: String): Unit = + SharedReadWrite.saveImpl(instance, instance.getStages, sc, path) + } + + private class PipelineReader extends MLReader[Pipeline] { + + /** Checked against metadata when loading model */ + private val className = classOf[Pipeline].getName + + override def load(path: String): Pipeline = { + val (uid: String, stages: Array[PipelineStage]) = SharedReadWrite.load(className, sc, path) + new Pipeline(uid).setStages(stages) + } + } + + /** + * Methods for `MLReader` and `MLWriter` shared between [[Pipeline]] and [[PipelineModel]] + */ + private[ml] object SharedReadWrite { + + import org.json4s.JsonDSL._ + + /** Check that all stages are Writable */ + def validateStages(stages: Array[PipelineStage]): Unit = { + stages.foreach { + case stage: MLWritable => // good + case other => + throw new UnsupportedOperationException("Pipeline write will fail on this Pipeline" + + s" because it contains a stage which does not implement Writable. Non-Writable stage:" + + s" ${other.uid} of type ${other.getClass}") + } + } + + /** + * Save metadata and stages for a [[Pipeline]] or [[PipelineModel]] + * - save metadata to path/metadata + * - save stages to stages/IDX_UID + */ + def saveImpl( + instance: Params, + stages: Array[PipelineStage], + sc: SparkContext, + path: String): Unit = { + val stageUids = stages.map(_.uid) + val jsonParams = List("stageUids" -> parse(compact(render(stageUids.toSeq)))) + DefaultParamsWriter.saveMetadata(instance, path, sc, paramMap = Some(jsonParams)) + + // Save stages + val stagesDir = new Path(path, "stages").toString + stages.zipWithIndex.foreach { case (stage, idx) => + stage.asInstanceOf[MLWritable].write.save( + getStagePath(stage.uid, idx, stages.length, stagesDir)) + } + } + + /** + * Load metadata and stages for a [[Pipeline]] or [[PipelineModel]] + * @return (UID, list of stages) + */ + def load( + expectedClassName: String, + sc: SparkContext, + path: String): (String, Array[PipelineStage]) = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, expectedClassName) + + implicit val format = DefaultFormats + val stagesDir = new Path(path, "stages").toString + val stageUids: Array[String] = (metadata.params \ "stageUids").extract[Seq[String]].toArray + val stages: Array[PipelineStage] = stageUids.zipWithIndex.map { case (stageUid, idx) => + val stagePath = SharedReadWrite.getStagePath(stageUid, idx, stageUids.length, stagesDir) + DefaultParamsReader.loadParamsInstance[PipelineStage](stagePath, sc) + } + (metadata.uid, stages) + } + + /** Get path for saving the given stage. */ + def getStagePath(stageUid: String, stageIdx: Int, numStages: Int, stagesDir: String): String = { + val stageIdxDigits = numStages.toString.length + val idxFormat = s"%0${stageIdxDigits}d" + val stageDir = idxFormat.format(stageIdx) + "_" + stageUid + new Path(stagesDir, stageDir).toString + } + } } /** - * :: Experimental :: * Represents a fitted pipeline. */ -@Experimental +@Since("1.2.0") class PipelineModel private[ml] ( - override val uid: String, - val stages: Array[Transformer]) - extends Model[PipelineModel] with Logging { + @Since("1.4.0") override val uid: String, + @Since("1.4.0") val stages: Array[Transformer]) + extends Model[PipelineModel] with MLWritable with Logging { /** A Java/Python-friendly auxiliary constructor. */ private[ml] def this(uid: String, stages: ju.List[Transformer]) = { this(uid, stages.asScala.toArray) } - override def validateParams(): Unit = { - super.validateParams() - stages.foreach(_.validateParams()) - } - - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - stages.foldLeft(dataset)((cur, transformer) => transformer.transform(cur)) + stages.foldLeft(dataset.toDF)((cur, transformer) => transformer.transform(cur)) } + @Since("1.2.0") override def transformSchema(schema: StructType): StructType = { stages.foldLeft(schema)((cur, transformer) => transformer.transformSchema(cur)) } + @Since("1.4.0") override def copy(extra: ParamMap): PipelineModel = { new PipelineModel(uid, stages.map(_.copy(extra))).setParent(parent) } + + @Since("1.6.0") + override def write: MLWriter = new PipelineModel.PipelineModelWriter(this) +} + +@Since("1.6.0") +object PipelineModel extends MLReadable[PipelineModel] { + + import Pipeline.SharedReadWrite + + @Since("1.6.0") + override def read: MLReader[PipelineModel] = new PipelineModelReader + + @Since("1.6.0") + override def load(path: String): PipelineModel = super.load(path) + + private[PipelineModel] class PipelineModelWriter(instance: PipelineModel) extends MLWriter { + + SharedReadWrite.validateStages(instance.stages.asInstanceOf[Array[PipelineStage]]) + + override protected def saveImpl(path: String): Unit = SharedReadWrite.saveImpl(instance, + instance.stages.asInstanceOf[Array[PipelineStage]], sc, path) + } + + private class PipelineModelReader extends MLReader[PipelineModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[PipelineModel].getName + + override def load(path: String): PipelineModel = { + val (uid: String, stages: Array[PipelineStage]) = SharedReadWrite.load(className, sc, path) + val transformers = stages map { + case stage: Transformer => stage + case other => throw new RuntimeException(s"PipelineModel.read loaded a stage but found it" + + s" was not a Transformer. Bad stage ${other.uid} of type ${other.getClass}") + } + new PipelineModel(uid, transformers) + } + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index e0dcd427fae2..08b0cb9b8f6a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -18,15 +18,15 @@ package org.apache.spark.ml import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} -import org.apache.spark.sql.{DataFrame, Row} /** * (private[ml]) Trait for parameters for prediction (regression and classification). @@ -36,10 +36,11 @@ private[ml] trait PredictorParams extends Params /** * Validates and transforms the input schema with the provided param map. + * * @param schema input schema * @param fitting whether this is in fitting * @param featuresDataType SQL DataType for FeaturesType. - * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. + * E.g., `VectorUDT` for vector features. * @return output schema */ protected def validateAndTransformSchema( @@ -49,8 +50,15 @@ private[ml] trait PredictorParams extends Params // TODO: Support casting Array[Double] and Array[Float] to Vector when FeaturesType = Vector SchemaUtils.checkColumnType(schema, $(featuresCol), featuresDataType) if (fitting) { - // TODO: Allow other numeric types - SchemaUtils.checkColumnType(schema, $(labelCol), DoubleType) + SchemaUtils.checkNumericType(schema, $(labelCol)) + + this match { + case p: HasWeightCol => + if (isDefined(p.weightCol) && $(p.weightCol).nonEmpty) { + SchemaUtils.checkNumericType(schema, $(p.weightCol)) + } + case _ => + } } SchemaUtils.appendColumn(schema, $(predictionCol), DoubleType) } @@ -58,10 +66,13 @@ private[ml] trait PredictorParams extends Params /** * :: DeveloperApi :: - * Abstraction for prediction problems (regression and classification). + * Abstraction for prediction problems (regression and classification). It accepts all NumericType + * labels and will automatically cast it to DoubleType in `fit()`. If this predictor supports + * weights, it accepts all NumericType weights, which will be automatically casted to DoubleType + * in `fit()`. * * @tparam FeaturesType Type of features. - * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. + * E.g., `VectorUDT` for vector features. * @tparam Learner Specialization of this class. If you subclass this type, use this type * parameter to specify the concrete type. * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type @@ -83,29 +94,46 @@ abstract class Predictor[ /** @group setParam */ def setPredictionCol(value: String): Learner = set(predictionCol, value).asInstanceOf[Learner] - override def fit(dataset: DataFrame): M = { + override def fit(dataset: Dataset[_]): M = { // This handles a few items such as schema validation. // Developers only need to implement train(). transformSchema(dataset.schema, logging = true) - copyValues(train(dataset).setParent(this)) + + // Cast LabelCol to DoubleType and keep the metadata. + val labelMeta = dataset.schema($(labelCol)).metadata + val labelCasted = dataset.withColumn($(labelCol), col($(labelCol)).cast(DoubleType), labelMeta) + + // Cast WeightCol to DoubleType and keep the metadata. + val casted = this match { + case p: HasWeightCol => + if (isDefined(p.weightCol) && $(p.weightCol).nonEmpty) { + val weightMeta = dataset.schema($(p.weightCol)).metadata + labelCasted.withColumn($(p.weightCol), col($(p.weightCol)).cast(DoubleType), weightMeta) + } else { + labelCasted + } + case _ => labelCasted + } + + copyValues(train(casted).setParent(this)) } override def copy(extra: ParamMap): Learner /** * Train a model using the given dataset and parameters. - * Developers can implement this instead of [[fit()]] to avoid dealing with schema validation + * Developers can implement this instead of `fit()` to avoid dealing with schema validation * and copying parameters into the model. * * @param dataset Training dataset * @return Fitted model */ - protected def train(dataset: DataFrame): M + protected def train(dataset: Dataset[_]): M /** * Returns the SQL DataType corresponding to the FeaturesType type parameter. * - * This is used by [[validateAndTransformSchema()]]. + * This is used by `validateAndTransformSchema()`. * This workaround is needed since SQL has different APIs for Scala and Java. * * The default value is VectorUDT, but it may be overridden if FeaturesType is not Vector. @@ -120,9 +148,10 @@ abstract class Predictor[ * Extract [[labelCol]] and [[featuresCol]] from the given dataset, * and put it in an RDD with strong types. */ - protected def extractLabeledPoints(dataset: DataFrame): RDD[LabeledPoint] = { - dataset.select($(labelCol), $(featuresCol)) - .map { case Row(label: Double, features: Vector) => LabeledPoint(label, features) } + protected def extractLabeledPoints(dataset: Dataset[_]): RDD[LabeledPoint] = { + dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { + case Row(label: Double, features: Vector) => LabeledPoint(label, features) + } } } @@ -131,7 +160,7 @@ abstract class Predictor[ * Abstraction for a model for prediction tasks (regression and classification). * * @tparam FeaturesType Type of features. - * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. + * E.g., `VectorUDT` for vector features. * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type * parameter to specify the concrete type for the corresponding model. */ @@ -152,7 +181,7 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, /** * Returns the SQL DataType corresponding to the FeaturesType type parameter. * - * This is used by [[validateAndTransformSchema()]]. + * This is used by `validateAndTransformSchema()`. * This workaround is needed since SQL has different APIs for Scala and Java. * * The default value is VectorUDT, but it may be overridden if FeaturesType is not Vector. @@ -164,24 +193,24 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, } /** - * Transforms dataset by reading from [[featuresCol]], calling [[predict()]], and storing + * Transforms dataset by reading from [[featuresCol]], calling `predict`, and storing * the predictions as a new column [[predictionCol]]. * * @param dataset input dataset - * @return transformed dataset with [[predictionCol]] of type [[Double]] + * @return transformed dataset with [[predictionCol]] of type `Double` */ - override def transform(dataset: DataFrame): DataFrame = { + override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) if ($(predictionCol).nonEmpty) { transformImpl(dataset) } else { this.logWarning(s"$uid: Predictor.transform() was called as NOOP" + " since no output columns were set.") - dataset + dataset.toDF } } - protected def transformImpl(dataset: DataFrame): DataFrame = { + protected def transformImpl(dataset: Dataset[_]): DataFrame = { val predictUDF = udf { (features: Any) => predict(features.asInstanceOf[FeaturesType]) } @@ -190,7 +219,7 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, /** * Predict label for the given features. - * This internal method is used to implement [[transform()]] and output [[predictionCol]]. + * This internal method is used to implement `transform()` and output [[predictionCol]]. */ protected def predict(features: FeaturesType): Double } 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 3c7bcf7590e6..a3a2b55adc25 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -19,11 +19,11 @@ package org.apache.spark.ml import scala.annotation.varargs -import org.apache.spark.Logging -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -41,9 +41,10 @@ abstract class Transformer extends PipelineStage { * @param otherParamPairs other param pairs, overwrite embedded params * @return transformed dataset */ + @Since("2.0.0") @varargs def transform( - dataset: DataFrame, + dataset: Dataset[_], firstParamPair: ParamPair[_], otherParamPairs: ParamPair[_]*): DataFrame = { val map = new ParamMap() @@ -58,14 +59,16 @@ abstract class Transformer extends PipelineStage { * @param paramMap additional parameters, overwrite embedded params * @return transformed dataset */ - def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + @Since("2.0.0") + def transform(dataset: Dataset[_], paramMap: ParamMap): DataFrame = { this.copy(paramMap).transform(dataset) } /** * Transforms the input dataset. */ - def transform(dataset: DataFrame): DataFrame + @Since("2.0.0") + def transform(dataset: Dataset[_]): DataFrame override def copy(extra: ParamMap): Transformer } @@ -113,10 +116,10 @@ abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, OUT, T]] StructType(outputFields) } - override def transform(dataset: DataFrame): DataFrame = { + override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - dataset.withColumn($(outputCol), - callUDF(this.createTransformFunc, outputDataType, dataset($(inputCol)))) + val transformUDF = udf(this.createTransformFunc, outputDataType) + dataset.withColumn($(outputCol), transformUDF(dataset($(inputCol)))) } override def copy(extra: ParamMap): T = defaultCopy(extra) diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala index 7429f9d652ac..6bbe7e1cb213 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/BreezeUtil.scala @@ -26,38 +26,39 @@ import com.github.fommil.netlib.BLAS.{getInstance => NativeBLAS} private[ann] object BreezeUtil { // TODO: switch to MLlib BLAS interface - private def transposeString(a: BDM[Double]): String = if (a.isTranspose) "T" else "N" + private def transposeString(A: BDM[Double]): String = if (A.isTranspose) "T" else "N" /** * DGEMM: C := alpha * A * B + beta * C * @param alpha alpha - * @param a A - * @param b B + * @param A A + * @param B B * @param beta beta - * @param c C + * @param C C */ - def dgemm(alpha: Double, a: BDM[Double], b: BDM[Double], beta: Double, c: BDM[Double]): Unit = { + def dgemm(alpha: Double, A: BDM[Double], B: BDM[Double], beta: Double, C: BDM[Double]): Unit = { // TODO: add code if matrices isTranspose!!! - require(a.cols == b.rows, "A & B Dimension mismatch!") - require(a.rows == c.rows, "A & C Dimension mismatch!") - require(b.cols == c.cols, "A & C Dimension mismatch!") - NativeBLAS.dgemm(transposeString(a), transposeString(b), c.rows, c.cols, a.cols, - alpha, a.data, a.offset, a.majorStride, b.data, b.offset, b.majorStride, - beta, c.data, c.offset, c.rows) + require(A.cols == B.rows, "A & B Dimension mismatch!") + require(A.rows == C.rows, "A & C Dimension mismatch!") + require(B.cols == C.cols, "A & C Dimension mismatch!") + NativeBLAS.dgemm(transposeString(A), transposeString(B), C.rows, C.cols, A.cols, + alpha, A.data, A.offset, A.majorStride, B.data, B.offset, B.majorStride, + beta, C.data, C.offset, C.rows) } /** * DGEMV: y := alpha * A * x + beta * y * @param alpha alpha - * @param a A + * @param A A * @param x x * @param beta beta * @param y y */ - def dgemv(alpha: Double, a: BDM[Double], x: BDV[Double], beta: Double, y: BDV[Double]): Unit = { - require(a.cols == x.length, "A & b Dimension mismatch!") - NativeBLAS.dgemv(transposeString(a), a.rows, a.cols, - alpha, a.data, a.offset, a.majorStride, x.data, x.offset, x.stride, + def dgemv(alpha: Double, A: BDM[Double], x: BDV[Double], beta: Double, y: BDV[Double]): Unit = { + require(A.cols == x.length, "A & x Dimension mismatch!") + require(A.rows == y.length, "A & y Dimension mismatch!") + NativeBLAS.dgemv(transposeString(A), A.rows, A.cols, + alpha, A.data, A.offset, A.majorStride, x.data, x.offset, x.stride, beta, y.data, y.offset, y.stride) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala index b5258ff34847..014ff07c2115 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala @@ -17,13 +17,16 @@ package org.apache.spark.ml.ann -import breeze.linalg.{*, DenseMatrix => BDM, DenseVector => BDV, Vector => BV, axpy => Baxpy, - sum => Bsum} -import breeze.numerics.{log => Blog, sigmoid => Bsigmoid} +import java.util.Random -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import breeze.linalg.{*, axpy => Baxpy, DenseMatrix => BDM, DenseVector => BDV, Vector => BV} + +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.optimization._ import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.random.XORShiftRandom /** @@ -32,20 +35,47 @@ import org.apache.spark.util.random.XORShiftRandom * */ private[ann] trait Layer extends Serializable { + + /** + * Number of weights that is used to allocate memory for the weights vector + */ + val weightSize: Int + + /** + * Returns the output size given the input size (not counting the stack size). + * Output size is used to allocate memory for the output. + * + * @param inputSize input size + * @return output size + */ + def getOutputSize(inputSize: Int): Int + + /** + * If true, the memory is not allocated for the output of this layer. + * The memory allocated to the previous layer is used to write the output of this layer. + * Developer can set this to true if computing delta of a previous layer + * does not involve its output, so the current layer can write there. + * This also mean that both layers have the same number of outputs. + */ + val inPlace: Boolean + /** - * Returns the instance of the layer based on weights provided - * @param weights vector with layer weights - * @param position position of weights in the vector + * Returns the instance of the layer based on weights provided. + * Size of weights must be equal to weightSize + * + * @param initialWeights vector with layer weights * @return the layer model */ - def getInstance(weights: Vector, position: Int): LayerModel + def createModel(initialWeights: BDV[Double]): LayerModel /** - * Returns the instance of the layer with random generated weights - * @param seed seed + * Returns the instance of the layer with random generated weights. + * + * @param weights vector for weights initialization, must be equal to weightSize + * @param random random number generator * @return the layer model */ - def getInstance(seed: Long): LayerModel + def initModel(weights: BDV[Double], random: Random): LayerModel } /** @@ -54,92 +84,102 @@ private[ann] trait Layer extends Serializable { * Can return weights in Vector format. */ private[ann] trait LayerModel extends Serializable { - /** - * number of weights - */ - val size: Int + val weights: BDV[Double] /** - * Evaluates the data (process the data through the layer) + * Evaluates the data (process the data through the layer). + * Output is allocated based on the size provided by the + * LayerModel implementation and the stack (batch) size. + * Developer is responsible for checking the size of output + * when writing to it. + * * @param data data - * @return processed data + * @param output output (modified in place) */ - def eval(data: BDM[Double]): BDM[Double] + def eval(data: BDM[Double], output: BDM[Double]): Unit /** - * Computes the delta for back propagation - * @param nextDelta delta of the next layer - * @param input input data - * @return delta + * Computes the delta for back propagation. + * Delta is allocated based on the size provided by the + * LayerModel implementation and the stack (batch) size. + * Developer is responsible for checking the size of + * prevDelta when writing to it. + * + * @param delta delta of this layer + * @param output output of this layer + * @param prevDelta the previous delta (modified in place) */ - def prevDelta(nextDelta: BDM[Double], input: BDM[Double]): BDM[Double] + def computePrevDelta(delta: BDM[Double], output: BDM[Double], prevDelta: BDM[Double]): Unit /** - * Computes the gradient + * Computes the gradient. + * cumGrad is a wrapper on the part of the weight vector. + * Size of cumGrad is based on weightSize provided by + * implementation of LayerModel. + * * @param delta delta for this layer * @param input input data - * @return gradient - */ - def grad(delta: BDM[Double], input: BDM[Double]): Array[Double] - - /** - * Returns weights for the layer in a single vector - * @return layer weights + * @param cumGrad cumulative gradient (modified in place) */ - def weights(): Vector + def grad(delta: BDM[Double], input: BDM[Double], cumGrad: BDV[Double]): Unit } /** * Layer properties of affine transformations, that is y=A*x+b + * * @param numIn number of inputs * @param numOut number of outputs */ private[ann] class AffineLayer(val numIn: Int, val numOut: Int) extends Layer { - override def getInstance(weights: Vector, position: Int): LayerModel = { - AffineLayerModel(this, weights, position) - } + override val weightSize = numIn * numOut + numOut - override def getInstance(seed: Long = 11L): LayerModel = { - AffineLayerModel(this, seed) - } + override def getOutputSize(inputSize: Int): Int = numOut + + override val inPlace = false + + override def createModel(weights: BDV[Double]): LayerModel = new AffineLayerModel(weights, this) + + override def initModel(weights: BDV[Double], random: Random): LayerModel = + AffineLayerModel(this, weights, random) } /** - * Model of Affine layer y=A*x+b - * @param w weights (matrix A) - * @param b bias (vector b) + * Model of Affine layer + * + * @param weights weights + * @param layer layer properties */ -private[ann] class AffineLayerModel private(w: BDM[Double], b: BDV[Double]) extends LayerModel { - val size = w.size + b.length - val gwb = new Array[Double](size) - private lazy val gw: BDM[Double] = new BDM[Double](w.rows, w.cols, gwb) - private lazy val gb: BDV[Double] = new BDV[Double](gwb, w.size) - private var z: BDM[Double] = null - private var d: BDM[Double] = null +private[ann] class AffineLayerModel private[ann] ( + val weights: BDV[Double], + val layer: AffineLayer) extends LayerModel { + val w = new BDM[Double](layer.numOut, layer.numIn, weights.data, weights.offset) + val b = + new BDV[Double](weights.data, weights.offset + (layer.numOut * layer.numIn), 1, layer.numOut) + private var ones: BDV[Double] = null - override def eval(data: BDM[Double]): BDM[Double] = { - if (z == null || z.cols != data.cols) z = new BDM[Double](w.rows, data.cols) - z(::, *) := b - BreezeUtil.dgemm(1.0, w, data, 1.0, z) - z + override def eval(data: BDM[Double], output: BDM[Double]): Unit = { + output(::, *) := b + BreezeUtil.dgemm(1.0, w, data, 1.0, output) } - override def prevDelta(nextDelta: BDM[Double], input: BDM[Double]): BDM[Double] = { - if (d == null || d.cols != nextDelta.cols) d = new BDM[Double](w.cols, nextDelta.cols) - BreezeUtil.dgemm(1.0, w.t, nextDelta, 0.0, d) - d + override def computePrevDelta( + delta: BDM[Double], + output: BDM[Double], + prevDelta: BDM[Double]): Unit = { + BreezeUtil.dgemm(1.0, w.t, delta, 0.0, prevDelta) } - override def grad(delta: BDM[Double], input: BDM[Double]): Array[Double] = { - BreezeUtil.dgemm(1.0 / input.cols, delta, input.t, 0.0, gw) + override def grad(delta: BDM[Double], input: BDM[Double], cumGrad: BDV[Double]): Unit = { + // compute gradient of weights + val cumGradientOfWeights = new BDM[Double](w.rows, w.cols, cumGrad.data, cumGrad.offset) + BreezeUtil.dgemm(1.0 / input.cols, delta, input.t, 1.0, cumGradientOfWeights) if (ones == null || ones.length != delta.cols) ones = BDV.ones[Double](delta.cols) - BreezeUtil.dgemv(1.0 / input.cols, delta, ones, 0.0, gb) - gwb + // compute gradient of bias + val cumGradientOfBias = new BDV[Double](cumGrad.data, cumGrad.offset + w.size, 1, b.length) + BreezeUtil.dgemv(1.0 / input.cols, delta, ones, 1.0, cumGradientOfBias) } - - override def weights(): Vector = AffineLayerModel.roll(w, b) } /** @@ -149,73 +189,40 @@ private[ann] object AffineLayerModel { /** * Creates a model of Affine layer + * * @param layer layer properties - * @param weights vector with weights - * @param position position of weights in the vector - * @return model of Affine layer - */ - def apply(layer: AffineLayer, weights: Vector, position: Int): AffineLayerModel = { - val (w, b) = unroll(weights, position, layer.numIn, layer.numOut) - new AffineLayerModel(w, b) - } - - /** - * Creates a model of Affine layer - * @param layer layer properties - * @param seed seed + * @param weights vector for weights initialization + * @param random random number generator * @return model of Affine layer */ - def apply(layer: AffineLayer, seed: Long): AffineLayerModel = { - val (w, b) = randomWeights(layer.numIn, layer.numOut, seed) - new AffineLayerModel(w, b) - } - - /** - * Unrolls the weights from the vector - * @param weights vector with weights - * @param position position of weights for this layer - * @param numIn number of layer inputs - * @param numOut number of layer outputs - * @return matrix A and vector b - */ - def unroll( - weights: Vector, - position: Int, - numIn: Int, - numOut: Int): (BDM[Double], BDV[Double]) = { - val weightsCopy = weights.toArray - // TODO: the array is not copied to BDMs, make sure this is OK! - val a = new BDM[Double](numOut, numIn, weightsCopy, position) - val b = new BDV[Double](weightsCopy, position + (numOut * numIn), 1, numOut) - (a, b) + def apply(layer: AffineLayer, weights: BDV[Double], random: Random): AffineLayerModel = { + randomWeights(layer.numIn, layer.numOut, weights, random) + new AffineLayerModel(weights, layer) } /** - * Roll the layer weights into a vector - * @param a matrix A - * @param b vector b - * @return vector of weights - */ - def roll(a: BDM[Double], b: BDV[Double]): Vector = { - val result = new Array[Double](a.size + b.length) - // TODO: make sure that we need to copy! - System.arraycopy(a.toArray, 0, result, 0, a.size) - System.arraycopy(b.toArray, 0, result, a.size, b.length) - Vectors.dense(result) - } - - /** - * Generate random weights for the layer + * Initialize weights randomly in the interval. + * Uses [Bottou-88] heuristic [-a/sqrt(in); a/sqrt(in)], + * where `a` is chosen in such a way that the weight variance corresponds + * to the points to the maximal curvature of the activation function + * (which is approximately 2.38 for a standard sigmoid). + * * @param numIn number of inputs * @param numOut number of outputs - * @param seed seed - * @return (matrix A, vector b) + * @param weights vector for weights initialization + * @param random random number generator */ - def randomWeights(numIn: Int, numOut: Int, seed: Long = 11L): (BDM[Double], BDV[Double]) = { - val rand: XORShiftRandom = new XORShiftRandom(seed) - val weights = BDM.fill[Double](numOut, numIn){ (rand.nextDouble * 4.8 - 2.4) / numIn } - val bias = BDV.fill[Double](numOut){ (rand.nextDouble * 4.8 - 2.4) / numIn } - (weights, bias) + def randomWeights( + numIn: Int, + numOut: Int, + weights: BDV[Double], + random: Random): Unit = { + var i = 0 + val sqrtIn = math.sqrt(numIn) + while (i < weights.length) { + weights(i) = (random.nextDouble * 4.8 - 2.4) / sqrtIn + i += 1 + } } } @@ -226,44 +233,21 @@ private[ann] trait ActivationFunction extends Serializable { /** * Implements a function - * @param x input data - * @param y output data */ - def eval(x: BDM[Double], y: BDM[Double]): Unit + def eval: Double => Double /** * Implements a derivative of a function (needed for the back propagation) - * @param x input data - * @param y output data - */ - def derivative(x: BDM[Double], y: BDM[Double]): Unit - - /** - * Implements a cross entropy error of a function. - * Needed if the functional layer that contains this function is the output layer - * of the network. - * @param target target output - * @param output computed output - * @param result intermediate result - * @return cross-entropy */ - def crossEntropy(target: BDM[Double], output: BDM[Double], result: BDM[Double]): Double - - /** - * Implements a mean squared error of a function - * @param target target output - * @param output computed output - * @param result intermediate result - * @return mean squared error - */ - def squared(target: BDM[Double], output: BDM[Double], result: BDM[Double]): Double + def derivative: Double => Double } /** - * Implements in-place application of functions + * Implements in-place application of functions in the arrays */ -private[ann] object ActivationFunction { +private[ann] object ApplyInPlace { + // TODO: use Breeze UFunc def apply(x: BDM[Double], y: BDM[Double], func: Double => Double): Unit = { var i = 0 while (i < x.rows) { @@ -276,6 +260,7 @@ private[ann] object ActivationFunction { } } + // TODO: use Breeze UFunc def apply( x1: BDM[Double], x2: BDM[Double], @@ -293,194 +278,129 @@ private[ann] object ActivationFunction { } } -/** - * Implements SoftMax activation function - */ -private[ann] class SoftmaxFunction extends ActivationFunction { - override def eval(x: BDM[Double], y: BDM[Double]): Unit = { - var j = 0 - // find max value to make sure later that exponent is computable - while (j < x.cols) { - var i = 0 - var max = Double.MinValue - while (i < x.rows) { - if (x(i, j) > max) { - max = x(i, j) - } - i += 1 - } - var sum = 0.0 - i = 0 - while (i < x.rows) { - val res = Math.exp(x(i, j) - max) - y(i, j) = res - sum += res - i += 1 - } - i = 0 - while (i < x.rows) { - y(i, j) /= sum - i += 1 - } - j += 1 - } - } - - override def crossEntropy( - output: BDM[Double], - target: BDM[Double], - result: BDM[Double]): Double = { - def m(o: Double, t: Double): Double = o - t - ActivationFunction(output, target, result, m) - -Bsum( target :* Blog(output)) / output.cols - } - - override def derivative(x: BDM[Double], y: BDM[Double]): Unit = { - def sd(z: Double): Double = (1 - z) * z - ActivationFunction(x, y, sd) - } - - override def squared(output: BDM[Double], target: BDM[Double], result: BDM[Double]): Double = { - throw new UnsupportedOperationException("Sorry, squared error is not defined for SoftMax.") - } -} - /** * Implements Sigmoid activation function */ private[ann] class SigmoidFunction extends ActivationFunction { - override def eval(x: BDM[Double], y: BDM[Double]): Unit = { - def s(z: Double): Double = Bsigmoid(z) - ActivationFunction(x, y, s) - } - - override def crossEntropy( - output: BDM[Double], - target: BDM[Double], - result: BDM[Double]): Double = { - def m(o: Double, t: Double): Double = o - t - ActivationFunction(output, target, result, m) - -Bsum(target :* Blog(output)) / output.cols - } - override def derivative(x: BDM[Double], y: BDM[Double]): Unit = { - def sd(z: Double): Double = (1 - z) * z - ActivationFunction(x, y, sd) - } + override def eval: (Double) => Double = x => 1.0 / (1 + math.exp(-x)) - override def squared(output: BDM[Double], target: BDM[Double], result: BDM[Double]): Double = { - // TODO: make it readable - def m(o: Double, t: Double): Double = (o - t) - ActivationFunction(output, target, result, m) - val e = Bsum(result :* result) / 2 / output.cols - def m2(x: Double, o: Double) = x * (o - o * o) - ActivationFunction(result, output, result, m2) - e - } + override def derivative: (Double) => Double = z => (1 - z) * z } /** * Functional layer properties, y = f(x) + * * @param activationFunction activation function */ private[ann] class FunctionalLayer (val activationFunction: ActivationFunction) extends Layer { - override def getInstance(weights: Vector, position: Int): LayerModel = getInstance(0L) - override def getInstance(seed: Long): LayerModel = - FunctionalLayerModel(this) + override val weightSize = 0 + + override def getOutputSize(inputSize: Int): Int = inputSize + + override val inPlace = true + + override def createModel(weights: BDV[Double]): LayerModel = new FunctionalLayerModel(this) + + override def initModel(weights: BDV[Double], random: Random): LayerModel = + createModel(weights) } /** * Functional layer model. Holds no weights. - * @param activationFunction activation function + * + * @param layer functional layer */ -private[ann] class FunctionalLayerModel private (val activationFunction: ActivationFunction) +private[ann] class FunctionalLayerModel private[ann] (val layer: FunctionalLayer) extends LayerModel { - val size = 0 - // matrices for in-place computations - // outputs - private var f: BDM[Double] = null - // delta - private var d: BDM[Double] = null - // matrix for error computation - private var e: BDM[Double] = null - // delta gradient - private lazy val dg = new Array[Double](0) - - override def eval(data: BDM[Double]): BDM[Double] = { - if (f == null || f.cols != data.cols) f = new BDM[Double](data.rows, data.cols) - activationFunction.eval(data, f) - f - } - - override def prevDelta(nextDelta: BDM[Double], input: BDM[Double]): BDM[Double] = { - if (d == null || d.cols != nextDelta.cols) d = new BDM[Double](nextDelta.rows, nextDelta.cols) - activationFunction.derivative(input, d) - d :*= nextDelta - d - } - override def grad(delta: BDM[Double], input: BDM[Double]): Array[Double] = dg + // empty weights + val weights = new BDV[Double](0) - override def weights(): Vector = Vectors.dense(new Array[Double](0)) - - def crossEntropy(output: BDM[Double], target: BDM[Double]): (BDM[Double], Double) = { - if (e == null || e.cols != output.cols) e = new BDM[Double](output.rows, output.cols) - val error = activationFunction.crossEntropy(output, target, e) - (e, error) + override def eval(data: BDM[Double], output: BDM[Double]): Unit = { + ApplyInPlace(data, output, layer.activationFunction.eval) } - def squared(output: BDM[Double], target: BDM[Double]): (BDM[Double], Double) = { - if (e == null || e.cols != output.cols) e = new BDM[Double](output.rows, output.cols) - val error = activationFunction.squared(output, target, e) - (e, error) + override def computePrevDelta( + nextDelta: BDM[Double], + input: BDM[Double], + delta: BDM[Double]): Unit = { + ApplyInPlace(input, delta, layer.activationFunction.derivative) + delta :*= nextDelta } - def error(output: BDM[Double], target: BDM[Double]): (BDM[Double], Double) = { - // TODO: allow user pick error - activationFunction match { - case sigmoid: SigmoidFunction => squared(output, target) - case softmax: SoftmaxFunction => crossEntropy(output, target) - } - } -} - -/** - * Fabric of functional layer models - */ -private[ann] object FunctionalLayerModel { - def apply(layer: FunctionalLayer): FunctionalLayerModel = - new FunctionalLayerModel(layer.activationFunction) + override def grad(delta: BDM[Double], input: BDM[Double], cumGrad: BDV[Double]): Unit = {} } /** * Trait for the artificial neural network (ANN) topology properties */ -private[ann] trait Topology extends Serializable{ - def getInstance(weights: Vector): TopologyModel - def getInstance(seed: Long): TopologyModel +private[ann] trait Topology extends Serializable { + def model(weights: Vector): TopologyModel + def model(seed: Long): TopologyModel } /** * Trait for ANN topology model */ -private[ann] trait TopologyModel extends Serializable{ +private[ann] trait TopologyModel extends Serializable { + + val weights: Vector + /** + * Array of layers + */ + val layers: Array[Layer] + + /** + * Array of layer models + */ + val layerModels: Array[LayerModel] + /** * Forward propagation + * * @param data input data + * @param includeLastLayer Include the last layer in the output. In + * MultilayerPerceptronClassifier, the last layer is always softmax; + * the last layer of outputs is needed for class predictions, but not + * for rawPrediction. + * * @return array of outputs for each of the layers */ - def forward(data: BDM[Double]): Array[BDM[Double]] + def forward(data: BDM[Double], includeLastLayer: Boolean): Array[BDM[Double]] /** - * Prediction of the model - * @param data input data + * Prediction of the model. See {@link ProbabilisticClassificationModel} + * + * @param features input features * @return prediction */ - def predict(data: Vector): Vector + def predict(features: Vector): Vector + + /** + * Raw prediction of the model. See {@link ProbabilisticClassificationModel} + * + * @param features input features + * @return raw prediction + * + * Note: This interface is only used for classification Model. + */ + def predictRaw(features: Vector): Vector + + /** + * Probability of the model. See {@link ProbabilisticClassificationModel} + * + * @param rawPrediction raw prediction vector + * @return probability + * + * Note: This interface is only used for classification Model. + */ + def raw2ProbabilityInPlace(rawPrediction: Vector): Vector /** * Computes gradient for the network + * * @param data input data * @param target target output * @param cumGradient cumulative gradient @@ -489,22 +409,17 @@ private[ann] trait TopologyModel extends Serializable{ */ def computeGradient(data: BDM[Double], target: BDM[Double], cumGradient: Vector, blockSize: Int): Double - - /** - * Returns the weights of the ANN - * @return weights - */ - def weights(): Vector } /** * Feed forward ANN - * @param layers + * + * @param layers Array of layers */ private[ann] class FeedForwardTopology private(val layers: Array[Layer]) extends Topology { - override def getInstance(weights: Vector): TopologyModel = FeedForwardModel(this, weights) + override def model(weights: Vector): TopologyModel = FeedForwardModel(this, weights) - override def getInstance(seed: Long): TopologyModel = FeedForwardModel(this, seed) + override def model(seed: Long): TopologyModel = FeedForwardModel(this, seed) } /** @@ -513,6 +428,7 @@ private[ann] class FeedForwardTopology private(val layers: Array[Layer]) extends private[ml] object FeedForwardTopology { /** * Creates a feed forward topology from the array of layers + * * @param layers array of layers * @return feed forward topology */ @@ -522,18 +438,26 @@ private[ml] object FeedForwardTopology { /** * Creates a multi-layer perceptron + * * @param layerSizes sizes of layers including input and output size - * @param softmax wether to use SoftMax or Sigmoid function for an output layer. + * @param softmaxOnTop whether to use SoftMax or Sigmoid function for an output layer. * Softmax is default * @return multilayer perceptron topology */ - def multiLayerPerceptron(layerSizes: Array[Int], softmax: Boolean = true): FeedForwardTopology = { + def multiLayerPerceptron( + layerSizes: Array[Int], + softmaxOnTop: Boolean = true): FeedForwardTopology = { val layers = new Array[Layer]((layerSizes.length - 1) * 2) - for(i <- 0 until layerSizes.length - 1){ + for (i <- 0 until layerSizes.length - 1) { layers(i * 2) = new AffineLayer(layerSizes(i), layerSizes(i + 1)) layers(i * 2 + 1) = - if (softmax && i == layerSizes.length - 2) { - new FunctionalLayer(new SoftmaxFunction()) + if (i == layerSizes.length - 2) { + if (softmaxOnTop) { + new SoftmaxLayerWithCrossEntropyLoss() + } else { + // TODO: squared error is more natural but converges slower + new SigmoidLayerWithSquaredError() + } } else { new FunctionalLayer(new SigmoidFunction()) } @@ -545,17 +469,46 @@ private[ml] object FeedForwardTopology { /** * Model of Feed Forward Neural Network. * Implements forward, gradient computation and can return weights in vector format. - * @param layerModels models of layers - * @param topology topology of the network + * + * @param weights network weights + * @param topology network topology */ private[ml] class FeedForwardModel private( - val layerModels: Array[LayerModel], + val weights: Vector, val topology: FeedForwardTopology) extends TopologyModel { - override def forward(data: BDM[Double]): Array[BDM[Double]] = { - val outputs = new Array[BDM[Double]](layerModels.length) - outputs(0) = layerModels(0).eval(data) - for (i <- 1 until layerModels.length) { - outputs(i) = layerModels(i).eval(outputs(i-1)) + + val layers = topology.layers + val layerModels = new Array[LayerModel](layers.length) + private var offset = 0 + for (i <- 0 until layers.length) { + layerModels(i) = layers(i).createModel( + new BDV[Double](weights.toArray, offset, 1, layers(i).weightSize)) + offset += layers(i).weightSize + } + private var outputs: Array[BDM[Double]] = null + private var deltas: Array[BDM[Double]] = null + + override def forward(data: BDM[Double], includeLastLayer: Boolean): Array[BDM[Double]] = { + // Initialize output arrays for all layers. Special treatment for InPlace + val currentBatchSize = data.cols + // TODO: allocate outputs as one big array and then create BDMs from it + if (outputs == null || outputs(0).cols != currentBatchSize) { + outputs = new Array[BDM[Double]](layers.length) + var inputSize = data.rows + for (i <- 0 until layers.length) { + if (layers(i).inPlace) { + outputs(i) = outputs(i - 1) + } else { + val outputSize = layers(i).getOutputSize(inputSize) + outputs(i) = new BDM[Double](outputSize, currentBatchSize) + inputSize = outputSize + } + } + } + layerModels(0).eval(data, outputs(0)) + val end = if (includeLastLayer) layerModels.length else layerModels.length - 1 + for (i <- 1 until end) { + layerModels(i).eval(outputs(i - 1), outputs(i)) } outputs } @@ -565,62 +518,55 @@ private[ml] class FeedForwardModel private( target: BDM[Double], cumGradient: Vector, realBatchSize: Int): Double = { - val outputs = forward(data) - val deltas = new Array[BDM[Double]](layerModels.length) + val outputs = forward(data, true) + val currentBatchSize = data.cols + // TODO: allocate deltas as one big array and then create BDMs from it + if (deltas == null || deltas(0).cols != currentBatchSize) { + deltas = new Array[BDM[Double]](layerModels.length) + var inputSize = data.rows + for (i <- 0 until layerModels.length - 1) { + val outputSize = layers(i).getOutputSize(inputSize) + deltas(i) = new BDM[Double](outputSize, currentBatchSize) + inputSize = outputSize + } + } val L = layerModels.length - 1 - val (newE, newError) = layerModels.last match { - case flm: FunctionalLayerModel => flm.error(outputs.last, target) + // TODO: explain why delta of top layer is null (because it might contain loss+layer) + val loss = layerModels.last match { + case levelWithError: LossFunction => levelWithError.loss(outputs.last, target, deltas(L - 1)) case _ => - throw new UnsupportedOperationException("Non-functional layer not supported at the top") + throw new UnsupportedOperationException("Top layer is required to have objective.") } - deltas(L) = new BDM[Double](0, 0) - deltas(L - 1) = newE for (i <- (L - 2) to (0, -1)) { - deltas(i) = layerModels(i + 1).prevDelta(deltas(i + 1), outputs(i + 1)) - } - val grads = new Array[Array[Double]](layerModels.length) - for (i <- 0 until layerModels.length) { - val input = if (i==0) data else outputs(i - 1) - grads(i) = layerModels(i).grad(deltas(i), input) + layerModels(i + 1).computePrevDelta(deltas(i + 1), outputs(i + 1), deltas(i)) } - // update cumGradient val cumGradientArray = cumGradient.toArray var offset = 0 - // TODO: extract roll - for (i <- 0 until grads.length) { - val gradArray = grads(i) - var k = 0 - while (k < gradArray.length) { - cumGradientArray(offset + k) += gradArray(k) - k += 1 - } - offset += gradArray.length - } - newError - } - - // TODO: do we really need to copy the weights? they should be read-only - override def weights(): Vector = { - // TODO: extract roll - var size = 0 - for (i <- 0 until layerModels.length) { - size += layerModels(i).size - } - val array = new Array[Double](size) - var offset = 0 for (i <- 0 until layerModels.length) { - val layerWeights = layerModels(i).weights().toArray - System.arraycopy(layerWeights, 0, array, offset, layerWeights.length) - offset += layerWeights.length + val input = if (i == 0) data else outputs(i - 1) + layerModels(i).grad(deltas(i), input, + new BDV[Double](cumGradientArray, offset, 1, layers(i).weightSize)) + offset += layers(i).weightSize } - Vectors.dense(array) + loss } override def predict(data: Vector): Vector = { val size = data.size - val result = forward(new BDM[Double](size, 1, data.toArray)) + val result = forward(new BDM[Double](size, 1, data.toArray), true) Vectors.dense(result.last.toArray) } + + override def predictRaw(data: Vector): Vector = { + val result = forward(new BDM[Double](data.size, 1, data.toArray), false) + Vectors.dense(result(result.length - 2).toArray) + } + + override def raw2ProbabilityInPlace(data: Vector): Vector = { + val dataMatrix = new BDM[Double](data.size, 1, data.toArray) + layerModels.last.eval(dataMatrix, dataMatrix) + data + } } /** @@ -630,23 +576,21 @@ private[ann] object FeedForwardModel { /** * Creates a model from a topology and weights + * * @param topology topology * @param weights weights * @return model */ def apply(topology: FeedForwardTopology, weights: Vector): FeedForwardModel = { - val layers = topology.layers - val layerModels = new Array[LayerModel](layers.length) - var offset = 0 - for (i <- 0 until layers.length) { - layerModels(i) = layers(i).getInstance(weights, offset) - offset += layerModels(i).size - } - new FeedForwardModel(layerModels, topology) + val expectedWeightSize = topology.layers.map(_.weightSize).sum + require(weights.size == expectedWeightSize, + s"Expected weight vector of size ${expectedWeightSize} but got size ${weights.size}.") + new FeedForwardModel(weights, topology) } /** * Creates a model given a topology and seed + * * @param topology topology * @param seed seed for generating the weights * @return model @@ -654,35 +598,32 @@ private[ann] object FeedForwardModel { def apply(topology: FeedForwardTopology, seed: Long = 11L): FeedForwardModel = { val layers = topology.layers val layerModels = new Array[LayerModel](layers.length) + val weights = BDV.zeros[Double](topology.layers.map(_.weightSize).sum) var offset = 0 - for(i <- 0 until layers.length){ - layerModels(i) = layers(i).getInstance(seed) - offset += layerModels(i).size + val random = new XORShiftRandom(seed) + for (i <- 0 until layers.length) { + layerModels(i) = layers(i). + initModel(new BDV[Double](weights.data, offset, 1, layers(i).weightSize), random) + offset += layers(i).weightSize } - new FeedForwardModel(layerModels, topology) + new FeedForwardModel(Vectors.fromBreeze(weights), topology) } } /** * Neural network gradient. Does nothing but calling Model's gradient + * * @param topology topology * @param dataStacker data stacker */ private[ann] class ANNGradient(topology: Topology, dataStacker: DataStacker) extends Gradient { - - override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { - val gradient = Vectors.zeros(weights.size) - val loss = compute(data, label, weights, gradient) - (gradient, loss) - } - override def compute( - data: Vector, + data: OldVector, label: Double, - weights: Vector, - cumGradient: Vector): Double = { + weights: OldVector, + cumGradient: OldVector): Double = { val (input, target, realBatchSize) = dataStacker.unstack(data) - val model = topology.getInstance(weights) + val model = topology.model(weights) model.computeGradient(input, target, cumGradient, realBatchSize) } } @@ -692,6 +633,7 @@ private[ann] class ANNGradient(topology: Topology, dataStacker: DataStacker) ext * through Optimizer/Gradient interfaces. If stackSize is more than one, makes blocks * or matrices of inputs and outputs and then stack them in one vector. * This can be used for further batch computations after unstacking. + * * @param stackSize stack size * @param inputSize size of the input vectors * @param outputSize size of the output vectors @@ -701,6 +643,7 @@ private[ann] class DataStacker(stackSize: Int, inputSize: Int, outputSize: Int) /** * Stacks the data + * * @param data RDD of vector pairs * @return RDD of double (always zero) and vector that contains the stacked vectors */ @@ -709,8 +652,8 @@ private[ann] class DataStacker(stackSize: Int, inputSize: Int, outputSize: Int) data.map { v => (0.0, Vectors.fromBreeze(BDV.vertcat( - v._1.toBreeze.toDenseVector, - v._2.toBreeze.toDenseVector)) + v._1.asBreeze.toDenseVector, + v._2.asBreeze.toDenseVector)) ) } } else { data.mapPartitions { it => @@ -733,6 +676,7 @@ private[ann] class DataStacker(stackSize: Int, inputSize: Int, outputSize: Int) /** * Unstack the stacked vectors into matrices for batch operations + * * @param data stacked vector * @return pair of matrices holding input and output data and the real stack size */ @@ -751,20 +695,21 @@ private[ann] class DataStacker(stackSize: Int, inputSize: Int, outputSize: Int) private[ann] class ANNUpdater extends Updater { override def compute( - weightsOld: Vector, - gradient: Vector, + weightsOld: OldVector, + gradient: OldVector, stepSize: Double, iter: Int, - regParam: Double): (Vector, Double) = { + regParam: Double): (OldVector, Double) = { val thisIterStepSize = stepSize - val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector - Baxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) - (Vectors.fromBreeze(brzWeights), 0) + val brzWeights: BV[Double] = weightsOld.asBreeze.toDenseVector + Baxpy(-thisIterStepSize, gradient.asBreeze, brzWeights) + (OldVectors.fromBreeze(brzWeights), 0) } } /** * MLlib-style trainer class that trains a network given the data and topology + * * @param topology topology of ANN * @param inputSize input size * @param outputSize output size @@ -774,36 +719,50 @@ private[ml] class FeedForwardTrainer( val inputSize: Int, val outputSize: Int) extends Serializable { - // TODO: what if we need to pass random seed? - private var _weights = topology.getInstance(11L).weights() + private var _seed = this.getClass.getName.hashCode.toLong + private var _weights: Vector = null private var _stackSize = 128 private var dataStacker = new DataStacker(_stackSize, inputSize, outputSize) private var _gradient: Gradient = new ANNGradient(topology, dataStacker) private var _updater: Updater = new ANNUpdater() private var optimizer: Optimizer = LBFGSOptimizer.setConvergenceTol(1e-4).setNumIterations(100) + /** + * Returns seed + */ + def getSeed: Long = _seed + + /** + * Sets seed + */ + def setSeed(value: Long): this.type = { + _seed = value + this + } + /** * Returns weights - * @return weights */ def getWeights: Vector = _weights /** * Sets weights + * * @param value weights * @return trainer */ - def setWeights(value: Vector): FeedForwardTrainer = { + def setWeights(value: Vector): this.type = { _weights = value this } /** * Sets the stack size + * * @param value stack size * @return trainer */ - def setStackSize(value: Int): FeedForwardTrainer = { + def setStackSize(value: Int): this.type = { _stackSize = value dataStacker = new DataStacker(value, inputSize, outputSize) this @@ -811,6 +770,7 @@ private[ml] class FeedForwardTrainer( /** * Sets the SGD optimizer + * * @return SGD optimizer */ def SGDOptimizer: GradientDescent = { @@ -821,6 +781,7 @@ private[ml] class FeedForwardTrainer( /** * Sets the LBFGS optimizer + * * @return LBGS optimizer */ def LBFGSOptimizer: LBFGS = { @@ -831,10 +792,11 @@ private[ml] class FeedForwardTrainer( /** * Sets the updater + * * @param value updater * @return trainer */ - def setUpdater(value: Updater): FeedForwardTrainer = { + def setUpdater(value: Updater): this.type = { _updater = value updateUpdater(value) this @@ -842,10 +804,11 @@ private[ml] class FeedForwardTrainer( /** * Sets the gradient + * * @param value gradient * @return trainer */ - def setGradient(value: Gradient): FeedForwardTrainer = { + def setGradient(value: Gradient): this.type = { _gradient = value updateGradient(value) this @@ -871,12 +834,26 @@ private[ml] class FeedForwardTrainer( /** * Trains the ANN + * * @param data RDD of input and output vector pairs * @return model */ def train(data: RDD[(Vector, Vector)]): TopologyModel = { - val newWeights = optimizer.optimize(dataStacker.stack(data), getWeights) - topology.getInstance(newWeights) + val w = if (getWeights == null) { + // TODO: will make a copy if vector is a subvector of BDV (see Vectors code) + topology.model(_seed).weights + } else { + getWeights + } + // TODO: deprecate standard optimizer because it needs Vector + val trainData = dataStacker.stack(data).map { v => + (v._1, OldVectors.fromML(v._2)) + } + val handlePersistence = trainData.getStorageLevel == StorageLevel.NONE + if (handlePersistence) trainData.persist(StorageLevel.MEMORY_AND_DISK) + val newWeights = optimizer.optimize(trainData, w) + if (handlePersistence) trainData.unpersist() + topology.model(newWeights) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/LossFunction.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/LossFunction.scala new file mode 100644 index 000000000000..3aea568cd652 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/LossFunction.scala @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.ann + +import java.util.Random + +import breeze.linalg.{sum => Bsum, DenseMatrix => BDM, DenseVector => BDV} +import breeze.numerics.{log => brzlog} + +/** + * Trait for loss function + */ +private[ann] trait LossFunction { + /** + * Returns the value of loss function. + * Computes loss based on target and output. + * Writes delta (error) to delta in place. + * Delta is allocated based on the outputSize + * of model implementation. + * + * @param output actual output + * @param target target output + * @param delta delta (updated in place) + * @return loss + */ + def loss(output: BDM[Double], target: BDM[Double], delta: BDM[Double]): Double +} + +private[ann] class SigmoidLayerWithSquaredError extends Layer { + override val weightSize = 0 + override val inPlace = true + + override def getOutputSize(inputSize: Int): Int = inputSize + override def createModel(weights: BDV[Double]): LayerModel = + new SigmoidLayerModelWithSquaredError() + override def initModel(weights: BDV[Double], random: Random): LayerModel = + new SigmoidLayerModelWithSquaredError() +} + +private[ann] class SigmoidLayerModelWithSquaredError + extends FunctionalLayerModel(new FunctionalLayer(new SigmoidFunction)) with LossFunction { + override def loss(output: BDM[Double], target: BDM[Double], delta: BDM[Double]): Double = { + ApplyInPlace(output, target, delta, (o: Double, t: Double) => o - t) + val error = Bsum(delta *:* delta) / 2 / output.cols + ApplyInPlace(delta, output, delta, (x: Double, o: Double) => x * (o - o * o)) + error + } +} + +private[ann] class SoftmaxLayerWithCrossEntropyLoss extends Layer { + override val weightSize = 0 + override val inPlace = true + + override def getOutputSize(inputSize: Int): Int = inputSize + override def createModel(weights: BDV[Double]): LayerModel = + new SoftmaxLayerModelWithCrossEntropyLoss() + override def initModel(weights: BDV[Double], random: Random): LayerModel = + new SoftmaxLayerModelWithCrossEntropyLoss() +} + +private[ann] class SoftmaxLayerModelWithCrossEntropyLoss extends LayerModel with LossFunction { + + // loss layer models do not have weights + val weights = new BDV[Double](0) + + override def eval(data: BDM[Double], output: BDM[Double]): Unit = { + var j = 0 + // find max value to make sure later that exponent is computable + while (j < data.cols) { + var i = 0 + var max = Double.MinValue + while (i < data.rows) { + if (data(i, j) > max) { + max = data(i, j) + } + i += 1 + } + var sum = 0.0 + i = 0 + while (i < data.rows) { + val res = math.exp(data(i, j) - max) + output(i, j) = res + sum += res + i += 1 + } + i = 0 + while (i < data.rows) { + output(i, j) /= sum + i += 1 + } + j += 1 + } + } + override def computePrevDelta( + nextDelta: BDM[Double], + input: BDM[Double], + delta: BDM[Double]): Unit = { + /* loss layer model computes delta in loss function */ + } + + override def grad(delta: BDM[Double], input: BDM[Double], cumGrad: BDV[Double]): Unit = { + /* loss layer model does not have weights */ + } + + override def loss(output: BDM[Double], target: BDM[Double], delta: BDM[Double]): Double = { + ApplyInPlace(output, target, delta, (o: Double, t: Double) => o - t) + -Bsum( target *:* brzlog(output)) / output.cols + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala index 2c29eeb01a92..21a246e454c8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.attribute import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.VectorUDT +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types.{Metadata, MetadataBuilder, StructField} /** @@ -239,7 +239,9 @@ object AttributeGroup { } } - /** Creates an attribute group from a [[StructField]] instance. */ + /** + * Creates an attribute group from a `StructField` instance. + */ def fromStructField(field: StructField): AttributeGroup = { require(field.dataType == new VectorUDT) if (field.metadata.contains(ML_ATTR)) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala index 5c7089b49167..078fecf08828 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala @@ -27,6 +27,9 @@ import org.apache.spark.annotation.DeveloperApi @DeveloperApi sealed abstract class AttributeType(val name: String) +/** + * :: DeveloperApi :: + */ @DeveloperApi object AttributeType { diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala index a7c10333c0d5..1cd2b1ad8409 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.attribute import scala.annotation.varargs import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.types.{DoubleType, NumericType, Metadata, MetadataBuilder, StructField} +import org.apache.spark.sql.types.{DoubleType, Metadata, MetadataBuilder, NumericType, StructField} /** * :: DeveloperApi :: @@ -98,7 +98,7 @@ sealed abstract class Attribute extends Serializable { def toMetadata(): Metadata = toMetadata(Metadata.empty) /** - * Converts to a [[StructField]] with some existing metadata. + * Converts to a `StructField` with some existing metadata. * @param existingMetadata existing metadata to carry over */ def toStructField(existingMetadata: Metadata): StructField = { @@ -109,7 +109,9 @@ sealed abstract class Attribute extends Serializable { StructField(name.get, DoubleType, nullable = false, newMetadata) } - /** Converts to a [[StructField]]. */ + /** + * Converts to a `StructField`. + */ def toStructField(): StructField = toStructField(Metadata.empty) override def toString: String = toMetadataImpl(withType = true).toString @@ -124,7 +126,7 @@ private[attribute] trait AttributeFactory { private[attribute] def fromMetadata(metadata: Metadata): Attribute /** - * Creates an [[Attribute]] from a [[StructField]] instance, optionally preserving name. + * Creates an [[Attribute]] from a `StructField` instance, optionally preserving name. */ private[ml] def decodeStructField(field: StructField, preserveName: Boolean): Attribute = { require(field.dataType.isInstanceOf[NumericType]) @@ -143,7 +145,7 @@ private[attribute] trait AttributeFactory { } /** - * Creates an [[Attribute]] from a [[StructField]] instance. + * Creates an [[Attribute]] from a `StructField` instance. */ def fromStructField(field: StructField): Attribute = decodeStructField(field, false) } @@ -369,12 +371,16 @@ class NominalAttribute private[ml] ( override def withIndex(index: Int): NominalAttribute = copy(index = Some(index)) override def withoutIndex: NominalAttribute = copy(index = None) - /** Copy with new values and empty `numValues`. */ + /** + * Copy with new values and empty `numValues`. + */ def withValues(values: Array[String]): NominalAttribute = { copy(numValues = None, values = Some(values)) } - /** Copy with new values and empty `numValues`. */ + /** + * Copy with new values and empty `numValues`. + */ @varargs def withValues(first: String, others: String*): NominalAttribute = { copy(numValues = None, values = Some((first +: others).toArray)) @@ -385,12 +391,16 @@ class NominalAttribute private[ml] ( copy(values = None) } - /** Copy with a new `numValues` and empty `values`. */ + /** + * Copy with a new `numValues` and empty `values`. + */ def withNumValues(numValues: Int): NominalAttribute = { copy(numValues = Some(numValues), values = None) } - /** Copy without the `numValues`. */ + /** + * Copy without the `numValues`. + */ def withoutNumValues: NominalAttribute = copy(numValues = None) /** @@ -481,7 +491,7 @@ object NominalAttribute extends AttributeFactory { * A binary attribute. * @param name optional name * @param index optional index - * @param values optionla values. If set, its size must be 2. + * @param values optional values. If set, its size must be 2. */ @DeveloperApi class BinaryAttribute private[ml] ( diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java b/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java index e3474f3c1d3f..464ed125695d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java @@ -20,7 +20,7 @@ /** *

    ML attributes

    * - * The ML pipeline API uses {@link org.apache.spark.sql.DataFrame}s as ML datasets. + * The ML pipeline API uses {@link org.apache.spark.sql.Dataset}s as ML datasets. * Each dataset consists of typed columns, e.g., string, double, vector, etc. * However, knowing only the column type may not be sufficient to handle the data properly. * For instance, a double column with values 0.0, 1.0, 2.0, ... may represent some label indices, diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/package.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/package.scala index 7ac21d7d563f..d26acf924c0a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/package.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/package.scala @@ -17,13 +17,12 @@ package org.apache.spark.ml -import org.apache.spark.sql.DataFrame import org.apache.spark.ml.attribute.{Attribute, AttributeGroup} /** * ==ML attributes== * - * The ML pipeline API uses [[DataFrame]]s as ML datasets. + * The ML pipeline API uses `DataFrame`s as ML datasets. * Each dataset consists of typed columns, e.g., string, double, vector, etc. * However, knowing only the column type may not be sufficient to handle the data properly. * For instance, a double column with values 0.0, 1.0, 2.0, ... may represent some label indices, 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 45df557a8990..bc0b49d48d32 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -17,16 +17,18 @@ package org.apache.spark.ml.classification +import org.apache.spark.SparkException import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.ml.{PredictionModel, PredictorParams, Predictor} +import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.shared.HasRawPredictionCol -import org.apache.spark.ml.util.SchemaUtils -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.sql.DataFrame +import org.apache.spark.ml.util.{MetadataUtils, SchemaUtils} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, StructType} - /** * (private[spark]) Params for classification. */ @@ -48,7 +50,7 @@ private[spark] trait ClassifierParams * Single-label binary or multiclass classification. * Classes are indexed {0, 1, ..., numClasses - 1}. * - * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam FeaturesType Type of input features. E.g., `Vector` * @tparam E Concrete Estimator type * @tparam M Concrete Model type */ @@ -63,6 +65,67 @@ abstract class Classifier[ def setRawPredictionCol(value: String): E = set(rawPredictionCol, value).asInstanceOf[E] // TODO: defaultEvaluator (follow-up PR) + + /** + * Extract [[labelCol]] and [[featuresCol]] from the given dataset, + * and put it in an RDD with strong types. + * + * @param dataset DataFrame with columns for labels ([[org.apache.spark.sql.types.NumericType]]) + * and features (`Vector`). + * @param numClasses Number of classes label can take. Labels must be integers in the range + * [0, numClasses). + * @note Throws `SparkException` if any label is a non-integer or is negative + */ + protected def extractLabeledPoints(dataset: Dataset[_], numClasses: Int): RDD[LabeledPoint] = { + require(numClasses > 0, s"Classifier (in extractLabeledPoints) found numClasses =" + + s" $numClasses, but requires numClasses > 0.") + dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { + case Row(label: Double, features: Vector) => + require(label % 1 == 0 && label >= 0 && label < numClasses, s"Classifier was given" + + s" dataset with invalid label $label. Labels must be integers in range" + + s" [0, $numClasses).") + LabeledPoint(label, features) + } + } + + /** + * Get the number of classes. This looks in column metadata first, and if that is missing, + * then this assumes classes are indexed 0,1,...,numClasses-1 and computes numClasses + * by finding the maximum label value. + * + * Label validation (ensuring all labels are integers >= 0) needs to be handled elsewhere, + * such as in `extractLabeledPoints()`. + * + * @param dataset Dataset which contains a column [[labelCol]] + * @param maxNumClasses Maximum number of classes allowed when inferred from data. If numClasses + * is specified in the metadata, then maxNumClasses is ignored. + * @return number of classes + * @throws IllegalArgumentException if metadata does not specify numClasses, and the + * actual numClasses exceeds maxNumClasses + */ + protected def getNumClasses(dataset: Dataset[_], maxNumClasses: Int = 100): Int = { + MetadataUtils.getNumClasses(dataset.schema($(labelCol))) match { + case Some(n: Int) => n + case None => + // Get number of classes from dataset itself. + val maxLabelRow: Array[Row] = dataset.select(max($(labelCol))).take(1) + if (maxLabelRow.isEmpty) { + throw new SparkException("ML algorithm was given empty dataset.") + } + val maxDoubleLabel: Double = maxLabelRow.head.getDouble(0) + require((maxDoubleLabel + 1).isValidInt, s"Classifier found max label value =" + + s" $maxDoubleLabel but requires integers in range [0, ... ${Int.MaxValue})") + val numClasses = maxDoubleLabel.toInt + 1 + require(numClasses <= maxNumClasses, s"Classifier inferred $numClasses from label values" + + s" in column $labelCol, but this exceeded the max numClasses ($maxNumClasses) allowed" + + s" to be inferred from values. To avoid this error for labels with > $maxNumClasses" + + s" classes, specify numClasses explicitly in the metadata; this can be done by applying" + + s" StringIndexer to the label column.") + logInfo(this.getClass.getCanonicalName + s" inferred $numClasses classes for" + + s" labelCol=$labelCol since numClasses was not specified in the column metadata.") + numClasses + } + } } /** @@ -71,7 +134,7 @@ abstract class Classifier[ * Model produced by a [[Classifier]]. * Classes are indexed {0, 1, ..., numClasses - 1}. * - * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam FeaturesType Type of input features. E.g., `Vector` * @tparam M Concrete Model type */ @DeveloperApi @@ -87,13 +150,13 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur /** * Transforms dataset by reading from [[featuresCol]], and appending new columns as specified by * parameters: - * - predicted labels as [[predictionCol]] of type [[Double]] - * - raw predictions (confidences) as [[rawPredictionCol]] of type [[Vector]]. + * - predicted labels as [[predictionCol]] of type `Double` + * - raw predictions (confidences) as [[rawPredictionCol]] of type `Vector`. * * @param dataset input dataset * @return transformed dataset */ - override def transform(dataset: DataFrame): DataFrame = { + override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) // Output selected columns only. @@ -124,15 +187,15 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur logWarning(s"$uid: ClassificationModel.transform() was called as NOOP" + " since no output columns were set.") } - outputData + outputData.toDF } /** * Predict label for the given features. - * This internal method is used to implement [[transform()]] and output [[predictionCol]]. + * This internal method is used to implement `transform()` and output [[predictionCol]]. * * This default implementation for classification predicts the index of the maximum value - * from [[predictRaw()]]. + * from `predictRaw()`. */ override protected def predict(features: FeaturesType): Double = { raw2prediction(predictRaw(features)) @@ -142,7 +205,7 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur * Raw prediction for each possible label. * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives * a measure of confidence in each possible label (where larger = more confident). - * This internal method is used to implement [[transform()]] and output [[rawPredictionCol]]. + * This internal method is used to implement `transform()` and output [[rawPredictionCol]]. * * @return vector where element i is the raw prediction for label i. * This raw prediction may be any real number, where a larger value indicates greater diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index b0157f7ce24e..9f60f0896ec5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -17,66 +17,122 @@ package org.apache.spark.ml.classification -import org.apache.spark.annotation.Experimental +import org.apache.hadoop.fs.Path +import org.json4s.{DefaultFormats, JObject} +import org.json4s.JsonDSL._ + +import org.apache.spark.annotation.Since +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, DecisionTreeParams, Node, TreeClassifierParams} +import org.apache.spark.ml.tree._ +import org.apache.spark.ml.tree.DecisionTreeModelReadWrite._ import org.apache.spark.ml.tree.impl.RandomForest -import org.apache.spark.ml.util.{Identifiable, MetadataUtils} -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.util._ import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.Dataset + /** - * :: Experimental :: - * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] learning algorithm + * Decision tree learning algorithm (http://en.wikipedia.org/wiki/Decision_tree_learning) * for classification. * It supports both binary and multiclass labels, as well as both continuous and categorical * features. */ -@Experimental -final class DecisionTreeClassifier(override val uid: String) +@Since("1.4.0") +class DecisionTreeClassifier @Since("1.4.0") ( + @Since("1.4.0") override val uid: String) extends ProbabilisticClassifier[Vector, DecisionTreeClassifier, DecisionTreeClassificationModel] - with DecisionTreeParams with TreeClassifierParams { + with DecisionTreeClassifierParams with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("dtc")) // Override parameter setters from parent trait for Java API compatibility. - override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + /** @group setParam */ + @Since("1.4.0") + override def setMaxDepth(value: Int): this.type = set(maxDepth, value) - override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + /** @group setParam */ + @Since("1.4.0") + override def setMaxBins(value: Int): this.type = set(maxBins, value) - override def setMinInstancesPerNode(value: Int): this.type = - super.setMinInstancesPerNode(value) + /** @group setParam */ + @Since("1.4.0") + override def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) - override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + /** @group setParam */ + @Since("1.4.0") + override def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) - override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + /** @group expertSetParam */ + @Since("1.4.0") + override def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) - override def setCacheNodeIds(value: Boolean): this.type = super.setCacheNodeIds(value) + /** @group expertSetParam */ + @Since("1.4.0") + override def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) + + /** + * Specifies how often to checkpoint the cached node IDs. + * E.g. 10 means that the cache will get checkpointed every 10 iterations. + * This is only used if cacheNodeIds is true and if the checkpoint directory is set in + * [[org.apache.spark.SparkContext]]. + * Must be at least 1. + * (default = 10) + * @group setParam + */ + @Since("1.4.0") + override def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) - override def setCheckpointInterval(value: Int): this.type = super.setCheckpointInterval(value) + /** @group setParam */ + @Since("1.4.0") + override def setImpurity(value: String): this.type = set(impurity, value) - override def setImpurity(value: String): this.type = super.setImpurity(value) + /** @group setParam */ + @Since("1.6.0") + override def setSeed(value: Long): this.type = set(seed, value) - override protected def train(dataset: DataFrame): DecisionTreeClassificationModel = { + override protected def train(dataset: Dataset[_]): DecisionTreeClassificationModel = { val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) - val numClasses: Int = MetadataUtils.getNumClasses(dataset.schema($(labelCol))) match { - case Some(n: Int) => n - case None => throw new IllegalArgumentException("DecisionTreeClassifier was given input" + - s" with invalid label column ${$(labelCol)}, without the number of classes" + - " specified. See StringIndexer.") - // TODO: Automatically index labels: SPARK-7126 + val numClasses: Int = getNumClasses(dataset) + + if (isDefined(thresholds)) { + require($(thresholds).length == numClasses, this.getClass.getSimpleName + + ".train() called with non-matching numClasses and thresholds.length." + + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") } - val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) + + val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset, numClasses) val strategy = getOldStrategy(categoricalFeatures, numClasses) + + val instr = Instrumentation.create(this, oldDataset) + instr.logParams(params: _*) + val trees = RandomForest.run(oldDataset, strategy, numTrees = 1, featureSubsetStrategy = "all", - seed = 0L, parentUID = Some(uid)) - trees.head.asInstanceOf[DecisionTreeClassificationModel] + seed = $(seed), instr = Some(instr), parentUID = Some(uid)) + + val m = trees.head.asInstanceOf[DecisionTreeClassificationModel] + instr.logSuccess(m) + m + } + + /** (private[ml]) Train a decision tree on an RDD */ + private[ml] def train(data: RDD[LabeledPoint], + oldStrategy: OldStrategy): DecisionTreeClassificationModel = { + val instr = Instrumentation.create(this, data) + instr.logParams(params: _*) + + val trees = RandomForest.run(data, oldStrategy, numTrees = 1, featureSubsetStrategy = "all", + seed = 0L, instr = Some(instr), parentUID = Some(uid)) + + val m = trees.head.asInstanceOf[DecisionTreeClassificationModel] + instr.logSuccess(m) + m } /** (private[ml]) Create a Strategy instance to use with the old API. */ @@ -87,29 +143,33 @@ final class DecisionTreeClassifier(override val uid: String) subsamplingRate = 1.0) } + @Since("1.4.1") override def copy(extra: ParamMap): DecisionTreeClassifier = defaultCopy(extra) } -@Experimental -object DecisionTreeClassifier { +@Since("1.4.0") +object DecisionTreeClassifier extends DefaultParamsReadable[DecisionTreeClassifier] { /** Accessor for supported impurities: entropy, gini */ + @Since("1.4.0") final val supportedImpurities: Array[String] = TreeClassifierParams.supportedImpurities + + @Since("2.0.0") + override def load(path: String): DecisionTreeClassifier = super.load(path) } /** - * :: Experimental :: - * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for classification. + * Decision tree model (http://en.wikipedia.org/wiki/Decision_tree_learning) for classification. * It supports both binary and multiclass labels, as well as both continuous and categorical * features. */ -@Experimental -final class DecisionTreeClassificationModel private[ml] ( - override val uid: String, - override val rootNode: Node, - override val numFeatures: Int, - override val numClasses: Int) +@Since("1.4.0") +class DecisionTreeClassificationModel private[ml] ( + @Since("1.4.0")override val uid: String, + @Since("1.4.0")override val rootNode: Node, + @Since("1.6.0")override val numFeatures: Int, + @Since("1.5.0")override val numClasses: Int) extends ProbabilisticClassificationModel[Vector, DecisionTreeClassificationModel] - with DecisionTreeModel with Serializable { + with DecisionTreeModel with DecisionTreeClassifierParams with MLWritable with Serializable { require(rootNode != null, "DecisionTreeClassificationModel given null rootNode, but it requires a non-null rootNode.") @@ -140,25 +200,91 @@ final class DecisionTreeClassificationModel private[ml] ( } } + @Since("1.4.0") override def copy(extra: ParamMap): DecisionTreeClassificationModel = { copyValues(new DecisionTreeClassificationModel(uid, rootNode, numFeatures, numClasses), extra) .setParent(parent) } + @Since("1.4.0") override def toString: String = { s"DecisionTreeClassificationModel (uid=$uid) of depth $depth with $numNodes nodes" } - /** (private[ml]) Convert to a model in the old API */ - private[ml] def toOld: OldDecisionTreeModel = { + /** + * Estimate of the importance of each feature. + * + * This generalizes the idea of "Gini" importance to other losses, + * following the explanation of Gini importance from "Random Forests" documentation + * by Leo Breiman and Adele Cutler, and following the implementation from scikit-learn. + * + * This feature importance is calculated as follows: + * - importance(feature j) = sum (over nodes which split on feature j) of the gain, + * where gain is scaled by the number of instances passing through node + * - Normalize importances for tree to sum to 1. + * + * @note Feature importance for single decision trees can have high variance due to + * correlated predictor variables. Consider using a [[RandomForestClassifier]] + * to determine feature importance instead. + */ + @Since("2.0.0") + lazy val featureImportances: Vector = TreeEnsembleModel.featureImportances(this, numFeatures) + + /** Convert to spark.mllib DecisionTreeModel (losing some information) */ + override private[spark] def toOld: OldDecisionTreeModel = { new OldDecisionTreeModel(rootNode.toOld(1), OldAlgo.Classification) } + + @Since("2.0.0") + override def write: MLWriter = + new DecisionTreeClassificationModel.DecisionTreeClassificationModelWriter(this) } -private[ml] object DecisionTreeClassificationModel { +@Since("2.0.0") +object DecisionTreeClassificationModel extends MLReadable[DecisionTreeClassificationModel] { + + @Since("2.0.0") + override def read: MLReader[DecisionTreeClassificationModel] = + new DecisionTreeClassificationModelReader + + @Since("2.0.0") + override def load(path: String): DecisionTreeClassificationModel = super.load(path) + + private[DecisionTreeClassificationModel] + class DecisionTreeClassificationModelWriter(instance: DecisionTreeClassificationModel) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val extraMetadata: JObject = Map( + "numFeatures" -> instance.numFeatures, + "numClasses" -> instance.numClasses) + DefaultParamsWriter.saveMetadata(instance, path, sc, Some(extraMetadata)) + val (nodeData, _) = NodeData.build(instance.rootNode, 0) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(nodeData).write.parquet(dataPath) + } + } + + private class DecisionTreeClassificationModelReader + extends MLReader[DecisionTreeClassificationModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[DecisionTreeClassificationModel].getName + + override def load(path: String): DecisionTreeClassificationModel = { + implicit val format = DefaultFormats + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] + val numClasses = (metadata.metadata \ "numClasses").extract[Int] + val root = loadTreeNodes(path, metadata, sparkSession) + val model = new DecisionTreeClassificationModel(metadata.uid, root, numFeatures, numClasses) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } - /** (private[ml]) Convert a model from the old API */ - def fromOld( + /** Convert a model from the old API */ + private[ml] def fromOld( oldModel: OldDecisionTreeModel, parent: DecisionTreeClassifier, categoricalFeatures: Map[Int, Int], diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 74aef94bf767..3da809ce5f77 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -18,62 +18,98 @@ package org.apache.spark.ml.classification import com.github.fommil.netlib.BLAS.{getInstance => blas} - -import org.apache.spark.Logging -import org.apache.spark.annotation.Experimental -import org.apache.spark.ml.{PredictionModel, Predictor} -import org.apache.spark.ml.param.{Param, ParamMap} +import org.json4s.{DefaultFormats, JObject} +import org.json4s.JsonDSL._ + +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.regression.DecisionTreeRegressionModel -import org.apache.spark.ml.tree.{DecisionTreeModel, GBTParams, TreeClassifierParams, TreeEnsembleModel} -import org.apache.spark.ml.util.{Identifiable, MetadataUtils} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.{GradientBoostedTrees => OldGBT} +import org.apache.spark.ml.tree._ +import org.apache.spark.ml.tree.impl.GradientBoostedTrees +import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} -import org.apache.spark.mllib.tree.loss.{LogLoss => OldLogLoss, Loss => OldLoss} import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, DataFrame} +import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.DoubleType /** - * :: Experimental :: - * [[http://en.wikipedia.org/wiki/Gradient_boosting Gradient-Boosted Trees (GBTs)]] + * Gradient-Boosted Trees (GBTs) (http://en.wikipedia.org/wiki/Gradient_boosting) * learning algorithm for classification. * It supports binary labels, as well as both continuous and categorical features. - * Note: Multiclass labels are not currently supported. + * + * The implementation is based upon: J.H. Friedman. "Stochastic Gradient Boosting." 1999. + * + * Notes on Gradient Boosting vs. TreeBoost: + * - This implementation is for Stochastic Gradient Boosting, not for TreeBoost. + * - Both algorithms learn tree ensembles by minimizing loss functions. + * - TreeBoost (Friedman, 1999) additionally modifies the outputs at tree leaf nodes + * based on the loss function, whereas the original gradient boosting method does not. + * - We expect to implement TreeBoost in the future: + * [https://issues.apache.org/jira/browse/SPARK-4240] + * + * @note Multiclass labels are not currently supported. */ -@Experimental -final class GBTClassifier(override val uid: String) - extends Predictor[Vector, GBTClassifier, GBTClassificationModel] - with GBTParams with TreeClassifierParams with Logging { +@Since("1.4.0") +class GBTClassifier @Since("1.4.0") ( + @Since("1.4.0") override val uid: String) + extends ProbabilisticClassifier[Vector, GBTClassifier, GBTClassificationModel] + with GBTClassifierParams with DefaultParamsWritable with Logging { + @Since("1.4.0") def this() = this(Identifiable.randomUID("gbtc")) // Override parameter setters from parent trait for Java API compatibility. // Parameters from TreeClassifierParams: - override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + /** @group setParam */ + @Since("1.4.0") + override def setMaxDepth(value: Int): this.type = set(maxDepth, value) - override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + /** @group setParam */ + @Since("1.4.0") + override def setMaxBins(value: Int): this.type = set(maxBins, value) - override def setMinInstancesPerNode(value: Int): this.type = - super.setMinInstancesPerNode(value) + /** @group setParam */ + @Since("1.4.0") + override def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) - override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + /** @group setParam */ + @Since("1.4.0") + override def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) - override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + /** @group expertSetParam */ + @Since("1.4.0") + override def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) - override def setCacheNodeIds(value: Boolean): this.type = super.setCacheNodeIds(value) + /** @group expertSetParam */ + @Since("1.4.0") + override def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) - override def setCheckpointInterval(value: Int): this.type = super.setCheckpointInterval(value) + /** + * Specifies how often to checkpoint the cached node IDs. + * E.g. 10 means that the cache will get checkpointed every 10 iterations. + * This is only used if cacheNodeIds is true and if the checkpoint directory is set in + * [[org.apache.spark.SparkContext]]. + * Must be at least 1. + * (default = 10) + * @group setParam + */ + @Since("1.4.0") + override def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) /** * The impurity setting is ignored for GBT models. * Individual trees are built using impurity "Variance." + * + * @group setParam */ + @Since("1.4.0") override def setImpurity(value: String): this.type = { logWarning("GBTClassifier.setImpurity should NOT be used") this @@ -81,116 +117,145 @@ final class GBTClassifier(override val uid: String) // Parameters from TreeEnsembleParams: - override def setSubsamplingRate(value: Double): this.type = super.setSubsamplingRate(value) + /** @group setParam */ + @Since("1.4.0") + override def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) - override def setSeed(value: Long): this.type = { - logWarning("The 'seed' parameter is currently ignored by Gradient Boosting.") - super.setSeed(value) - } + /** @group setParam */ + @Since("1.4.0") + override def setSeed(value: Long): this.type = set(seed, value) // Parameters from GBTParams: - override def setMaxIter(value: Int): this.type = super.setMaxIter(value) - - override def setStepSize(value: Double): this.type = super.setStepSize(value) - - // Parameters for GBTClassifier: + /** @group setParam */ + @Since("1.4.0") + override def setMaxIter(value: Int): this.type = set(maxIter, value) - /** - * Loss function which GBT tries to minimize. (case-insensitive) - * Supported: "logistic" - * (default = logistic) - * @group param - */ - val lossType: Param[String] = new Param[String](this, "lossType", "Loss function which GBT" + - " tries to minimize (case-insensitive). Supported options:" + - s" ${GBTClassifier.supportedLossTypes.mkString(", ")}", - (value: String) => GBTClassifier.supportedLossTypes.contains(value.toLowerCase)) + /** @group setParam */ + @Since("1.4.0") + override def setStepSize(value: Double): this.type = set(stepSize, value) - setDefault(lossType -> "logistic") + // Parameters from GBTClassifierParams: /** @group setParam */ + @Since("1.4.0") def setLossType(value: String): this.type = set(lossType, value) - /** @group getParam */ - def getLossType: String = $(lossType).toLowerCase - - /** (private[ml]) Convert new loss to old loss. */ - override private[ml] def getOldLossType: OldLoss = { - getLossType match { - case "logistic" => OldLogLoss - case _ => - // Should never happen because of check in setter method. - throw new RuntimeException(s"GBTClassifier was given bad loss type: $getLossType") - } - } - - override protected def train(dataset: DataFrame): GBTClassificationModel = { + override protected def train(dataset: Dataset[_]): GBTClassificationModel = { val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) - val numClasses: Int = MetadataUtils.getNumClasses(dataset.schema($(labelCol))) match { - case Some(n: Int) => n - case None => throw new IllegalArgumentException("GBTClassifier was given input" + - s" with invalid label column ${$(labelCol)}, without the number of classes" + - " specified. See StringIndexer.") - // TODO: Automatically index labels: SPARK-7126 - } - require(numClasses == 2, - s"GBTClassifier only supports binary classification but was given numClasses = $numClasses") - val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) + // We copy and modify this from Classifier.extractLabeledPoints since GBT only supports + // 2 classes now. This lets us provide a more precise error message. + val oldDataset: RDD[LabeledPoint] = + dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { + case Row(label: Double, features: Vector) => + require(label == 0 || label == 1, s"GBTClassifier was given" + + s" dataset with invalid label $label. Labels must be in {0,1}; note that" + + s" GBTClassifier currently only supports binary classification.") + LabeledPoint(label, features) + } val numFeatures = oldDataset.first().features.size val boostingStrategy = super.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Classification) - val oldGBT = new OldGBT(boostingStrategy) - val oldModel = oldGBT.run(oldDataset) - GBTClassificationModel.fromOld(oldModel, this, categoricalFeatures, numFeatures) + + val numClasses = 2 + if (isDefined(thresholds)) { + require($(thresholds).length == numClasses, this.getClass.getSimpleName + + ".train() called with non-matching numClasses and thresholds.length." + + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") + } + + val instr = Instrumentation.create(this, oldDataset) + instr.logParams(labelCol, featuresCol, predictionCol, impurity, lossType, + maxDepth, maxBins, maxIter, maxMemoryInMB, minInfoGain, minInstancesPerNode, + seed, stepSize, subsamplingRate, cacheNodeIds, checkpointInterval) + instr.logNumFeatures(numFeatures) + instr.logNumClasses(numClasses) + + val (baseLearners, learnerWeights) = GradientBoostedTrees.run(oldDataset, boostingStrategy, + $(seed)) + val m = new GBTClassificationModel(uid, baseLearners, learnerWeights, numFeatures) + instr.logSuccess(m) + m } + @Since("1.4.1") override def copy(extra: ParamMap): GBTClassifier = defaultCopy(extra) } -@Experimental -object GBTClassifier { - // The losses below should be lowercase. +@Since("1.4.0") +object GBTClassifier extends DefaultParamsReadable[GBTClassifier] { + /** Accessor for supported loss settings: logistic */ - final val supportedLossTypes: Array[String] = Array("logistic").map(_.toLowerCase) + @Since("1.4.0") + final val supportedLossTypes: Array[String] = GBTClassifierParams.supportedLossTypes + + @Since("2.0.0") + override def load(path: String): GBTClassifier = super.load(path) } /** - * :: Experimental :: - * [[http://en.wikipedia.org/wiki/Gradient_boosting Gradient-Boosted Trees (GBTs)]] + * Gradient-Boosted Trees (GBTs) (http://en.wikipedia.org/wiki/Gradient_boosting) * model for classification. * It supports binary labels, as well as both continuous and categorical features. - * Note: Multiclass labels are not currently supported. + * * @param _trees Decision trees in the ensemble. * @param _treeWeights Weights for the decision trees in the ensemble. + * + * @note Multiclass labels are not currently supported. */ -@Experimental -final class GBTClassificationModel private[ml]( - override val uid: String, +@Since("1.6.0") +class GBTClassificationModel private[ml]( + @Since("1.6.0") override val uid: String, private val _trees: Array[DecisionTreeRegressionModel], private val _treeWeights: Array[Double], - override val numFeatures: Int) - extends PredictionModel[Vector, GBTClassificationModel] - with TreeEnsembleModel with Serializable { + @Since("1.6.0") override val numFeatures: Int, + @Since("2.2.0") override val numClasses: Int) + extends ProbabilisticClassificationModel[Vector, GBTClassificationModel] + with GBTClassifierParams with TreeEnsembleModel[DecisionTreeRegressionModel] + with MLWritable with Serializable { - require(numTrees > 0, "GBTClassificationModel requires at least 1 tree.") + require(_trees.nonEmpty, "GBTClassificationModel requires at least 1 tree.") require(_trees.length == _treeWeights.length, "GBTClassificationModel given trees, treeWeights" + s" of non-matching lengths (${_trees.length}, ${_treeWeights.length}, respectively).") /** * Construct a GBTClassificationModel + * * @param _trees Decision trees in the ensemble. * @param _treeWeights Weights for the decision trees in the ensemble. + * @param numFeatures The number of features. */ + private[ml] def this( + uid: String, + _trees: Array[DecisionTreeRegressionModel], + _treeWeights: Array[Double], + numFeatures: Int) = + this(uid, _trees, _treeWeights, numFeatures, 2) + + /** + * Construct a GBTClassificationModel + * + * @param _trees Decision trees in the ensemble. + * @param _treeWeights Weights for the decision trees in the ensemble. + */ + @Since("1.6.0") def this(uid: String, _trees: Array[DecisionTreeRegressionModel], _treeWeights: Array[Double]) = - this(uid, _trees, _treeWeights, -1) + this(uid, _trees, _treeWeights, -1, 2) - override def trees: Array[DecisionTreeModel] = _trees.asInstanceOf[Array[DecisionTreeModel]] + @Since("1.4.0") + override def trees: Array[DecisionTreeRegressionModel] = _trees + /** + * Number of trees in ensemble + */ + @Since("2.0.0") + val getNumTrees: Int = trees.length + + @Since("1.4.0") override def treeWeights: Array[Double] = _treeWeights - override protected def transformImpl(dataset: DataFrame): DataFrame = { - val bcastModel = dataset.sqlContext.sparkContext.broadcast(this) + override protected def transformImpl(dataset: Dataset[_]): DataFrame = { + val bcastModel = dataset.sparkSession.sparkContext.broadcast(this) val predictUDF = udf { (features: Any) => bcastModel.value.predict(features.asInstanceOf[Vector]) } @@ -198,36 +263,136 @@ final class GBTClassificationModel private[ml]( } override protected def predict(features: Vector): Double = { - // TODO: When we add a generic Boosting class, handle transform there? SPARK-7129 - // Classifies by thresholding sum of weighted tree predictions - val treePredictions = _trees.map(_.rootNode.predictImpl(features).prediction) - val prediction = blas.ddot(numTrees, treePredictions, 1, _treeWeights, 1) - if (prediction > 0.0) 1.0 else 0.0 + // If thresholds defined, use predictRaw to get probabilities, otherwise use optimization + if (isDefined(thresholds)) { + super.predict(features) + } else { + if (margin(features) > 0.0) 1.0 else 0.0 + } } + override protected def predictRaw(features: Vector): Vector = { + val prediction: Double = margin(features) + Vectors.dense(Array(-prediction, prediction)) + } + + override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { + rawPrediction match { + case dv: DenseVector => + dv.values(0) = loss.computeProbability(dv.values(0)) + dv.values(1) = 1.0 - dv.values(0) + dv + case sv: SparseVector => + throw new RuntimeException("Unexpected error in GBTClassificationModel:" + + " raw2probabilityInPlace encountered SparseVector") + } + } + + /** Number of trees in ensemble */ + val numTrees: Int = trees.length + + @Since("1.4.0") override def copy(extra: ParamMap): GBTClassificationModel = { - copyValues(new GBTClassificationModel(uid, _trees, _treeWeights, numFeatures), + copyValues(new GBTClassificationModel(uid, _trees, _treeWeights, numFeatures, numClasses), extra).setParent(parent) } + @Since("1.4.0") override def toString: String = { s"GBTClassificationModel (uid=$uid) with $numTrees trees" } + /** + * Estimate of the importance of each feature. + * + * Each feature's importance is the average of its importance across all trees in the ensemble + * The importance vector is normalized to sum to 1. This method is suggested by Hastie et al. + * (Hastie, Tibshirani, Friedman. "The Elements of Statistical Learning, 2nd Edition." 2001.) + * and follows the implementation from scikit-learn. + + * See `DecisionTreeClassificationModel.featureImportances` + */ + @Since("2.0.0") + lazy val featureImportances: Vector = TreeEnsembleModel.featureImportances(trees, numFeatures) + + /** Raw prediction for the positive class. */ + private def margin(features: Vector): Double = { + val treePredictions = _trees.map(_.rootNode.predictImpl(features).prediction) + blas.ddot(numTrees, treePredictions, 1, _treeWeights, 1) + } + /** (private[ml]) Convert to a model in the old API */ private[ml] def toOld: OldGBTModel = { new OldGBTModel(OldAlgo.Classification, _trees.map(_.toOld), _treeWeights) } + + // hard coded loss, which is not meant to be changed in the model + private val loss = getOldLossType + + @Since("2.0.0") + override def write: MLWriter = new GBTClassificationModel.GBTClassificationModelWriter(this) } -private[ml] object GBTClassificationModel { +@Since("2.0.0") +object GBTClassificationModel extends MLReadable[GBTClassificationModel] { + + private val numFeaturesKey: String = "numFeatures" + private val numTreesKey: String = "numTrees" + + @Since("2.0.0") + override def read: MLReader[GBTClassificationModel] = new GBTClassificationModelReader + + @Since("2.0.0") + override def load(path: String): GBTClassificationModel = super.load(path) + + private[GBTClassificationModel] + class GBTClassificationModelWriter(instance: GBTClassificationModel) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + + val extraMetadata: JObject = Map( + numFeaturesKey -> instance.numFeatures, + numTreesKey -> instance.getNumTrees) + EnsembleModelReadWrite.saveImpl(instance, path, sparkSession, extraMetadata) + } + } + + private class GBTClassificationModelReader extends MLReader[GBTClassificationModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[GBTClassificationModel].getName + private val treeClassName = classOf[DecisionTreeRegressionModel].getName + + override def load(path: String): GBTClassificationModel = { + implicit val format = DefaultFormats + val (metadata: Metadata, treesData: Array[(Metadata, Node)], treeWeights: Array[Double]) = + EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName) + val numFeatures = (metadata.metadata \ numFeaturesKey).extract[Int] + val numTrees = (metadata.metadata \ numTreesKey).extract[Int] + + val trees: Array[DecisionTreeRegressionModel] = treesData.map { + case (treeMetadata, root) => + val tree = + new DecisionTreeRegressionModel(treeMetadata.uid, root, numFeatures) + DefaultParamsReader.getAndSetParams(tree, treeMetadata) + tree + } + require(numTrees == trees.length, s"GBTClassificationModel.load expected $numTrees" + + s" trees based on metadata but found ${trees.length} trees.") + val model = new GBTClassificationModel(metadata.uid, + trees, treeWeights, numFeatures) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } - /** (private[ml]) Convert a model from the old API */ - def fromOld( + /** Convert a model from the old API */ + private[ml] def fromOld( oldModel: OldGBTModel, parent: GBTClassifier, categoricalFeatures: Map[Int, Int], - numFeatures: Int = -1): GBTClassificationModel = { + numFeatures: Int = -1, + numClasses: Int = 2): GBTClassificationModel = { require(oldModel.algo == OldAlgo.Classification, "Cannot convert GradientBoostedTreesModel" + s" with algo=${oldModel.algo} (old API) to GBTClassificationModel (new API).") val newTrees = oldModel.trees.map { tree => @@ -235,6 +400,6 @@ private[ml] object GBTClassificationModel { DecisionTreeRegressionModel.fromOld(tree, null, categoricalFeatures) } val uid = if (parent != null) parent.uid else Identifiable.randomUID("gbtc") - new GBTClassificationModel(parent.uid, newTrees, oldModel.treeWeights, numFeatures) + new GBTClassificationModel(uid, newTrees, oldModel.treeWeights, numFeatures, numClasses) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala new file mode 100644 index 000000000000..ce400f4f1faf --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -0,0 +1,384 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.classification + +import scala.collection.mutable + +import breeze.linalg.{DenseVector => BDV} +import breeze.optimize.{CachedDiffFunction, OWLQN => BreezeOWLQN} +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.optim.aggregator.HingeAggregator +import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction} +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.functions.{col, lit} + +/** Params for linear SVM Classifier. */ +private[classification] trait LinearSVCParams extends ClassifierParams with HasRegParam + with HasMaxIter with HasFitIntercept with HasTol with HasStandardization with HasWeightCol + with HasAggregationDepth with HasThreshold { + + /** + * Param for threshold in binary classification prediction. + * For LinearSVC, this threshold is applied to the rawPrediction, rather than a probability. + * This threshold can be any real number, where Inf will make all predictions 0.0 + * and -Inf will make all predictions 1.0. + * Default: 0.0 + * + * @group param + */ + final override val threshold: DoubleParam = new DoubleParam(this, "threshold", + "threshold in binary classification prediction applied to rawPrediction") +} + +/** + * :: Experimental :: + * + * + * Linear SVM Classifier + * + * This binary classifier optimizes the Hinge Loss using the OWLQN optimizer. + * Only supports L2 regularization currently. + * + */ +@Since("2.2.0") +@Experimental +class LinearSVC @Since("2.2.0") ( + @Since("2.2.0") override val uid: String) + extends Classifier[Vector, LinearSVC, LinearSVCModel] + with LinearSVCParams with DefaultParamsWritable { + + @Since("2.2.0") + def this() = this(Identifiable.randomUID("linearsvc")) + + /** + * Set the regularization parameter. + * Default is 0.0. + * + * @group setParam + */ + @Since("2.2.0") + def setRegParam(value: Double): this.type = set(regParam, value) + setDefault(regParam -> 0.0) + + /** + * Set the maximum number of iterations. + * Default is 100. + * + * @group setParam + */ + @Since("2.2.0") + def setMaxIter(value: Int): this.type = set(maxIter, value) + setDefault(maxIter -> 100) + + /** + * Whether to fit an intercept term. + * Default is true. + * + * @group setParam + */ + @Since("2.2.0") + def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) + setDefault(fitIntercept -> true) + + /** + * Set the convergence tolerance of iterations. + * Smaller values will lead to higher accuracy at the cost of more iterations. + * Default is 1E-6. + * + * @group setParam + */ + @Since("2.2.0") + def setTol(value: Double): this.type = set(tol, value) + setDefault(tol -> 1E-6) + + /** + * Whether to standardize the training features before fitting the model. + * Default is true. + * + * @group setParam + */ + @Since("2.2.0") + def setStandardization(value: Boolean): this.type = set(standardization, value) + setDefault(standardization -> true) + + /** + * Set the value of param [[weightCol]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * Default is not set, so all instances have weight one. + * + * @group setParam + */ + @Since("2.2.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + + /** + * Set threshold in binary classification. + * + * @group setParam + */ + @Since("2.2.0") + def setThreshold(value: Double): this.type = set(threshold, value) + setDefault(threshold -> 0.0) + + /** + * Suggested depth for treeAggregate (greater than or equal to 2). + * If the dimensions of features or the number of partitions are large, + * this param could be adjusted to a larger size. + * Default is 2. + * + * @group expertSetParam + */ + @Since("2.2.0") + def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) + setDefault(aggregationDepth -> 2) + + @Since("2.2.0") + override def copy(extra: ParamMap): LinearSVC = defaultCopy(extra) + + override protected def train(dataset: Dataset[_]): LinearSVCModel = { + val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) + val instances: RDD[Instance] = + dataset.select(col($(labelCol)), w, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, features: Vector) => + Instance(label, weight, features) + } + + val instr = Instrumentation.create(this, instances) + instr.logParams(regParam, maxIter, fitIntercept, tol, standardization, threshold, + aggregationDepth) + + val (summarizer, labelSummarizer) = { + val seqOp = (c: (MultivariateOnlineSummarizer, MultiClassSummarizer), + instance: Instance) => + (c._1.add(instance.features, instance.weight), c._2.add(instance.label, instance.weight)) + + val combOp = (c1: (MultivariateOnlineSummarizer, MultiClassSummarizer), + c2: (MultivariateOnlineSummarizer, MultiClassSummarizer)) => + (c1._1.merge(c2._1), c1._2.merge(c2._2)) + + instances.treeAggregate( + (new MultivariateOnlineSummarizer, new MultiClassSummarizer) + )(seqOp, combOp, $(aggregationDepth)) + } + + val histogram = labelSummarizer.histogram + val numInvalid = labelSummarizer.countInvalid + val numFeatures = summarizer.mean.size + val numFeaturesPlusIntercept = if (getFitIntercept) numFeatures + 1 else numFeatures + + val numClasses = MetadataUtils.getNumClasses(dataset.schema($(labelCol))) match { + case Some(n: Int) => + require(n >= histogram.length, s"Specified number of classes $n was " + + s"less than the number of unique labels ${histogram.length}.") + n + case None => histogram.length + } + require(numClasses == 2, s"LinearSVC only supports binary classification." + + s" $numClasses classes detected in $labelCol") + instr.logNumClasses(numClasses) + instr.logNumFeatures(numFeatures) + + val (coefficientVector, interceptVector, objectiveHistory) = { + if (numInvalid != 0) { + val msg = s"Classification labels should be in [0 to ${numClasses - 1}]. " + + s"Found $numInvalid invalid labels." + logError(msg) + throw new SparkException(msg) + } + + val featuresStd = summarizer.variance.toArray.map(math.sqrt) + val getFeaturesStd = (j: Int) => featuresStd(j) + val regParamL2 = $(regParam) + val bcFeaturesStd = instances.context.broadcast(featuresStd) + val regularization = if (regParamL2 != 0.0) { + val shouldApply = (idx: Int) => idx >= 0 && idx < numFeatures + Some(new L2Regularization(regParamL2, shouldApply, + if ($(standardization)) None else Some(getFeaturesStd))) + } else { + None + } + + val getAggregatorFunc = new HingeAggregator(bcFeaturesStd, $(fitIntercept))(_) + val costFun = new RDDLossFunction(instances, getAggregatorFunc, regularization, + $(aggregationDepth)) + + def regParamL1Fun = (index: Int) => 0D + val optimizer = new BreezeOWLQN[Int, BDV[Double]]($(maxIter), 10, regParamL1Fun, $(tol)) + val initialCoefWithIntercept = Vectors.zeros(numFeaturesPlusIntercept) + + val states = optimizer.iterations(new CachedDiffFunction(costFun), + initialCoefWithIntercept.asBreeze.toDenseVector) + + val scaledObjectiveHistory = mutable.ArrayBuilder.make[Double] + var state: optimizer.State = null + while (states.hasNext) { + state = states.next() + scaledObjectiveHistory += state.adjustedValue + } + + bcFeaturesStd.destroy(blocking = false) + if (state == null) { + val msg = s"${optimizer.getClass.getName} failed." + logError(msg) + throw new SparkException(msg) + } + + /* + The coefficients are trained in the scaled space; we're converting them back to + the original space. + Note that the intercept in scaled space and original space is the same; + as a result, no scaling is needed. + */ + val rawCoefficients = state.x.toArray + val coefficientArray = Array.tabulate(numFeatures) { i => + if (featuresStd(i) != 0.0) { + rawCoefficients(i) / featuresStd(i) + } else { + 0.0 + } + } + + val intercept = if ($(fitIntercept)) { + rawCoefficients(numFeaturesPlusIntercept - 1) + } else { + 0.0 + } + (Vectors.dense(coefficientArray), intercept, scaledObjectiveHistory.result()) + } + + val model = copyValues(new LinearSVCModel(uid, coefficientVector, interceptVector)) + instr.logSuccess(model) + model + } +} + +@Since("2.2.0") +object LinearSVC extends DefaultParamsReadable[LinearSVC] { + + @Since("2.2.0") + override def load(path: String): LinearSVC = super.load(path) +} + +/** + * :: Experimental :: + * Linear SVM Model trained by [[LinearSVC]] + */ +@Since("2.2.0") +@Experimental +class LinearSVCModel private[classification] ( + @Since("2.2.0") override val uid: String, + @Since("2.2.0") val coefficients: Vector, + @Since("2.2.0") val intercept: Double) + extends ClassificationModel[Vector, LinearSVCModel] + with LinearSVCParams with MLWritable { + + @Since("2.2.0") + override val numClasses: Int = 2 + + @Since("2.2.0") + override val numFeatures: Int = coefficients.size + + @Since("2.2.0") + def setThreshold(value: Double): this.type = set(threshold, value) + setDefault(threshold, 0.0) + + @Since("2.2.0") + def setWeightCol(value: Double): this.type = set(threshold, value) + + private val margin: Vector => Double = (features) => { + BLAS.dot(features, coefficients) + intercept + } + + override protected def predict(features: Vector): Double = { + if (margin(features) > $(threshold)) 1.0 else 0.0 + } + + override protected def predictRaw(features: Vector): Vector = { + val m = margin(features) + Vectors.dense(-m, m) + } + + override protected def raw2prediction(rawPrediction: Vector): Double = { + if (rawPrediction(1) > $(threshold)) 1.0 else 0.0 + } + + @Since("2.2.0") + override def copy(extra: ParamMap): LinearSVCModel = { + copyValues(new LinearSVCModel(uid, coefficients, intercept), extra).setParent(parent) + } + + @Since("2.2.0") + override def write: MLWriter = new LinearSVCModel.LinearSVCWriter(this) + +} + + +@Since("2.2.0") +object LinearSVCModel extends MLReadable[LinearSVCModel] { + + @Since("2.2.0") + override def read: MLReader[LinearSVCModel] = new LinearSVCReader + + @Since("2.2.0") + override def load(path: String): LinearSVCModel = super.load(path) + + /** [[MLWriter]] instance for [[LinearSVCModel]] */ + private[LinearSVCModel] + class LinearSVCWriter(instance: LinearSVCModel) + extends MLWriter with Logging { + + private case class Data(coefficients: Vector, intercept: Double) + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.coefficients, instance.intercept) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class LinearSVCReader extends MLReader[LinearSVCModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[LinearSVCModel].getName + + override def load(path: String): LinearSVCModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.format("parquet").load(dataPath) + val Row(coefficients: Vector, intercept: Double) = + data.select("coefficients", "intercept").head() + val model = new LinearSVCModel(metadata.uid, coefficients, intercept) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } +} 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 f5fca686df14..fa191604218d 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 @@ -17,64 +17,98 @@ package org.apache.spark.ml.classification +import java.util.Locale + import scala.collection.mutable import breeze.linalg.{DenseVector => BDV} -import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS, OWLQN => BreezeOWLQN} +import breeze.optimize.{CachedDiffFunction, LBFGS => BreezeLBFGS, LBFGSB => BreezeLBFGSB, OWLQN => BreezeOWLQN} +import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkException} -import org.apache.spark.annotation.Experimental +import org.apache.spark.SparkException +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.optim.aggregator.LogisticAggregator +import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg._ -import org.apache.spark.mllib.linalg.BLAS._ -import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.evaluation.{BinaryClassificationMetrics, MulticlassMetrics} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.{col, lit} +import org.apache.spark.sql.types.{DataType, DoubleType, StructType} import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.VersionUtils /** * Params for logistic regression. */ private[classification] trait LogisticRegressionParams extends ProbabilisticClassifierParams with HasRegParam with HasElasticNetParam with HasMaxIter with HasFitIntercept with HasTol - with HasStandardization with HasWeightCol with HasThreshold { + with HasStandardization with HasWeightCol with HasThreshold with HasAggregationDepth { + + import org.apache.spark.ml.classification.LogisticRegression.supportedFamilyNames /** * Set threshold in binary classification, in range [0, 1]. * - * If the estimated probability of class label 1 is > threshold, then predict 1, else 0. - * A high threshold encourages the model to predict 0 more often; + * If the estimated probability of class label 1 is greater than threshold, then predict 1, + * else 0. A high threshold encourages the model to predict 0 more often; * a low threshold encourages the model to predict 1 more often. * * Note: Calling this with threshold p is equivalent to calling `setThresholds(Array(1-p, p))`. - * When [[setThreshold()]] is called, any user-set value for [[thresholds]] will be cleared. - * If both [[threshold]] and [[thresholds]] are set in a ParamMap, then they must be + * When `setThreshold()` is called, any user-set value for `thresholds` will be cleared. + * If both `threshold` and `thresholds` are set in a ParamMap, then they must be * equivalent. * * Default is 0.5. + * * @group setParam */ + // TODO: Implement SPARK-11543? def setThreshold(value: Double): this.type = { if (isSet(thresholds)) clear(thresholds) set(threshold, value) } + /** + * Param for the name of family which is a description of the label distribution + * to be used in the model. + * Supported options: + * - "auto": Automatically select the family based on the number of classes: + * If numClasses == 1 || numClasses == 2, set to "binomial". + * Else, set to "multinomial" + * - "binomial": Binary logistic regression with pivoting. + * - "multinomial": Multinomial logistic (softmax) regression without pivoting. + * Default is "auto". + * + * @group param + */ + @Since("2.1.0") + final val family: Param[String] = new Param(this, "family", + "The name of family which is a description of the label distribution to be used in the " + + s"model. Supported options: ${supportedFamilyNames.mkString(", ")}.", + (value: String) => supportedFamilyNames.contains(value.toLowerCase(Locale.ROOT))) + + /** @group getParam */ + @Since("2.1.0") + def getFamily: String = $(family) + /** * Get threshold for binary classification. * - * If [[threshold]] is set, returns that value. - * Otherwise, if [[thresholds]] is set with length 2 (i.e., binary classification), + * If `thresholds` is set with length 2 (i.e., binary classification), * this returns the equivalent threshold: {{{1 / (1 + thresholds(0) / thresholds(1))}}}. - * Otherwise, returns [[threshold]] default value. + * Otherwise, returns `threshold` if set, or its default value if unset. * * @group getParam - * @throws IllegalArgumentException if [[thresholds]] is set to an array of length other than 2. + * @throws IllegalArgumentException if `thresholds` is set to an array of length other than 2. */ override def getThreshold: Double = { checkThresholdConsistency() @@ -90,12 +124,13 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas /** * Set thresholds in multiclass (or binary) classification to adjust the probability of - * predicting each class. Array must have length equal to the number of classes, with values >= 0. + * predicting each class. Array must have length equal to the number of classes, + * with values greater than 0, excepting that at most one value may be 0. * The class with largest value p/t is predicted, where p is the original probability of that - * class and t is the class' threshold. + * class and t is the class's threshold. * - * Note: When [[setThresholds()]] is called, any user-set value for [[threshold]] will be cleared. - * If both [[threshold]] and [[thresholds]] are set in a ParamMap, then they must be + * Note: When `setThresholds()` is called, any user-set value for `threshold` will be cleared. + * If both `threshold` and `thresholds` are set in a ParamMap, then they must be * equivalent. * * @group setParam @@ -108,8 +143,8 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas /** * Get thresholds for binary or multiclass classification. * - * If [[thresholds]] is set, return its value. - * Otherwise, if [[threshold]] is set, return the equivalent thresholds for binary + * If `thresholds` is set, return its value. + * Otherwise, if `threshold` is set, return the equivalent thresholds for binary * classification: (1-threshold, threshold). * If neither are set, throw an exception. * @@ -126,8 +161,9 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas } /** - * If [[threshold]] and [[thresholds]] are both set, ensures they are consistent. - * @throws IllegalArgumentException if [[threshold]] and [[thresholds]] are not equivalent + * If `threshold` and `thresholds` are both set, ensures they are consistent. + * + * @throws IllegalArgumentException if `threshold` and `thresholds` are not equivalent */ protected def checkThresholdConsistency(): Unit = { if (isSet(threshold) && isSet(thresholds)) { @@ -142,67 +178,178 @@ private[classification] trait LogisticRegressionParams extends ProbabilisticClas } } - override def validateParams(): Unit = { + /** + * The lower bounds on coefficients if fitting under bound constrained optimization. + * The bound matrix must be compatible with the shape (1, number of features) for binomial + * regression, or (number of classes, number of features) for multinomial regression. + * Otherwise, it throws exception. + * Default is none. + * + * @group expertParam + */ + @Since("2.2.0") + val lowerBoundsOnCoefficients: Param[Matrix] = new Param(this, "lowerBoundsOnCoefficients", + "The lower bounds on coefficients if fitting under bound constrained optimization.") + + /** @group expertGetParam */ + @Since("2.2.0") + def getLowerBoundsOnCoefficients: Matrix = $(lowerBoundsOnCoefficients) + + /** + * The upper bounds on coefficients if fitting under bound constrained optimization. + * The bound matrix must be compatible with the shape (1, number of features) for binomial + * regression, or (number of classes, number of features) for multinomial regression. + * Otherwise, it throws exception. + * Default is none. + * + * @group expertParam + */ + @Since("2.2.0") + val upperBoundsOnCoefficients: Param[Matrix] = new Param(this, "upperBoundsOnCoefficients", + "The upper bounds on coefficients if fitting under bound constrained optimization.") + + /** @group expertGetParam */ + @Since("2.2.0") + def getUpperBoundsOnCoefficients: Matrix = $(upperBoundsOnCoefficients) + + /** + * The lower bounds on intercepts if fitting under bound constrained optimization. + * The bounds vector size must be equal to 1 for binomial regression, or the number + * of classes for multinomial regression. Otherwise, it throws exception. + * Default is none. + * + * @group expertParam + */ + @Since("2.2.0") + val lowerBoundsOnIntercepts: Param[Vector] = new Param(this, "lowerBoundsOnIntercepts", + "The lower bounds on intercepts if fitting under bound constrained optimization.") + + /** @group expertGetParam */ + @Since("2.2.0") + def getLowerBoundsOnIntercepts: Vector = $(lowerBoundsOnIntercepts) + + /** + * The upper bounds on intercepts if fitting under bound constrained optimization. + * The bound vector size must be equal to 1 for binomial regression, or the number + * of classes for multinomial regression. Otherwise, it throws exception. + * Default is none. + * + * @group expertParam + */ + @Since("2.2.0") + val upperBoundsOnIntercepts: Param[Vector] = new Param(this, "upperBoundsOnIntercepts", + "The upper bounds on intercepts if fitting under bound constrained optimization.") + + /** @group expertGetParam */ + @Since("2.2.0") + def getUpperBoundsOnIntercepts: Vector = $(upperBoundsOnIntercepts) + + protected def usingBoundConstrainedOptimization: Boolean = { + isSet(lowerBoundsOnCoefficients) || isSet(upperBoundsOnCoefficients) || + isSet(lowerBoundsOnIntercepts) || isSet(upperBoundsOnIntercepts) + } + + override protected def validateAndTransformSchema( + schema: StructType, + fitting: Boolean, + featuresDataType: DataType): StructType = { checkThresholdConsistency() + if (usingBoundConstrainedOptimization) { + require($(elasticNetParam) == 0.0, "Fitting under bound constrained optimization only " + + s"supports L2 regularization, but got elasticNetParam = $getElasticNetParam.") + } + if (!$(fitIntercept)) { + require(!isSet(lowerBoundsOnIntercepts) && !isSet(upperBoundsOnIntercepts), + "Please don't set bounds on intercepts if fitting without intercept.") + } + super.validateAndTransformSchema(schema, fitting, featuresDataType) } } /** - * :: Experimental :: - * Logistic regression. - * Currently, this class only supports binary classification. It will support multiclass - * in the future. + * Logistic regression. Supports: + * - Multinomial logistic (softmax) regression. + * - Binomial logistic regression. + * + * This class supports fitting traditional logistic regression model by LBFGS/OWLQN and + * bound (box) constrained logistic regression model by LBFGSB. */ -@Experimental -class LogisticRegression(override val uid: String) +@Since("1.2.0") +class LogisticRegression @Since("1.2.0") ( + @Since("1.4.0") override val uid: String) extends ProbabilisticClassifier[Vector, LogisticRegression, LogisticRegressionModel] - with LogisticRegressionParams with Logging { + with LogisticRegressionParams with DefaultParamsWritable with Logging { + @Since("1.4.0") def this() = this(Identifiable.randomUID("logreg")) /** * Set the regularization parameter. * Default is 0.0. + * * @group setParam */ + @Since("1.2.0") def setRegParam(value: Double): this.type = set(regParam, value) setDefault(regParam -> 0.0) /** * Set the ElasticNet mixing parameter. - * For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty. - * For 0 < alpha < 1, the penalty is a combination of L1 and L2. + * For alpha = 0, the penalty is an L2 penalty. + * For alpha = 1, it is an L1 penalty. + * For alpha in (0,1), the penalty is a combination of L1 and L2. * Default is 0.0 which is an L2 penalty. + * + * Note: Fitting under bound constrained optimization only supports L2 regularization, + * so throws exception if this param is non-zero value. + * * @group setParam */ + @Since("1.4.0") def setElasticNetParam(value: Double): this.type = set(elasticNetParam, value) setDefault(elasticNetParam -> 0.0) /** * Set the maximum number of iterations. * Default is 100. + * * @group setParam */ + @Since("1.2.0") def setMaxIter(value: Int): this.type = set(maxIter, value) setDefault(maxIter -> 100) /** * Set the convergence tolerance of iterations. - * Smaller value will lead to higher accuracy with the cost of more iterations. + * Smaller value will lead to higher accuracy at the cost of more iterations. * Default is 1E-6. + * * @group setParam */ + @Since("1.4.0") def setTol(value: Double): this.type = set(tol, value) setDefault(tol -> 1E-6) /** * Whether to fit an intercept term. * Default is true. + * * @group setParam */ + @Since("1.4.0") def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) setDefault(fitIntercept -> true) + /** + * Sets the value of param [[family]]. + * Default is "auto". + * + * @group setParam + */ + @Since("2.1.0") + def setFamily(value: String): this.type = set(family, value) + setDefault(family -> "auto") + /** * Whether to standardize the training features before fitting the model. * The coefficients of models will be always returned on the original scale, @@ -210,39 +357,153 @@ class LogisticRegression(override val uid: String) * the models should be always converged to the same solution when no regularization * is applied. In R's GLMNET package, the default behavior is true as well. * Default is true. + * * @group setParam */ + @Since("1.5.0") def setStandardization(value: Boolean): this.type = set(standardization, value) setDefault(standardization -> true) + @Since("1.5.0") override def setThreshold(value: Double): this.type = super.setThreshold(value) + setDefault(threshold -> 0.5) + @Since("1.5.0") override def getThreshold: Double = super.getThreshold /** - * Whether to over-/under-sample training instances according to the given weights in weightCol. - * If empty, all instances are treated equally (weight 1.0). - * Default is empty, so all instances have weight one. + * Sets the value of param [[weightCol]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * Default is not set, so all instances have weight one. + * * @group setParam */ + @Since("1.6.0") def setWeightCol(value: String): this.type = set(weightCol, value) - setDefault(weightCol -> "") + @Since("1.5.0") override def setThresholds(value: Array[Double]): this.type = super.setThresholds(value) + @Since("1.5.0") override def getThresholds: Array[Double] = super.getThresholds - override protected def train(dataset: DataFrame): LogisticRegressionModel = { - // Extract columns from data. If dataset is persisted, do not persist oldDataset. - val w = if ($(weightCol).isEmpty) lit(1.0) else col($(weightCol)) - val instances: RDD[Instance] = dataset.select(col($(labelCol)), w, col($(featuresCol))).map { - case Row(label: Double, weight: Double, features: Vector) => - Instance(label, weight, features) + /** + * Suggested depth for treeAggregate (greater than or equal to 2). + * If the dimensions of features or the number of partitions are large, + * this param could be adjusted to a larger size. + * Default is 2. + * + * @group expertSetParam + */ + @Since("2.1.0") + def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) + setDefault(aggregationDepth -> 2) + + /** + * Set the lower bounds on coefficients if fitting under bound constrained optimization. + * + * @group expertSetParam + */ + @Since("2.2.0") + def setLowerBoundsOnCoefficients(value: Matrix): this.type = set(lowerBoundsOnCoefficients, value) + + /** + * Set the upper bounds on coefficients if fitting under bound constrained optimization. + * + * @group expertSetParam + */ + @Since("2.2.0") + def setUpperBoundsOnCoefficients(value: Matrix): this.type = set(upperBoundsOnCoefficients, value) + + /** + * Set the lower bounds on intercepts if fitting under bound constrained optimization. + * + * @group expertSetParam + */ + @Since("2.2.0") + def setLowerBoundsOnIntercepts(value: Vector): this.type = set(lowerBoundsOnIntercepts, value) + + /** + * Set the upper bounds on intercepts if fitting under bound constrained optimization. + * + * @group expertSetParam + */ + @Since("2.2.0") + def setUpperBoundsOnIntercepts(value: Vector): this.type = set(upperBoundsOnIntercepts, value) + + private def assertBoundConstrainedOptimizationParamsValid( + numCoefficientSets: Int, + numFeatures: Int): Unit = { + if (isSet(lowerBoundsOnCoefficients)) { + require($(lowerBoundsOnCoefficients).numRows == numCoefficientSets && + $(lowerBoundsOnCoefficients).numCols == numFeatures, + "The shape of LowerBoundsOnCoefficients must be compatible with (1, number of features) " + + "for binomial regression, or (number of classes, number of features) for multinomial " + + "regression, but found: " + + s"(${getLowerBoundsOnCoefficients.numRows}, ${getLowerBoundsOnCoefficients.numCols}).") + } + if (isSet(upperBoundsOnCoefficients)) { + require($(upperBoundsOnCoefficients).numRows == numCoefficientSets && + $(upperBoundsOnCoefficients).numCols == numFeatures, + "The shape of upperBoundsOnCoefficients must be compatible with (1, number of features) " + + "for binomial regression, or (number of classes, number of features) for multinomial " + + "regression, but found: " + + s"(${getUpperBoundsOnCoefficients.numRows}, ${getUpperBoundsOnCoefficients.numCols}).") + } + if (isSet(lowerBoundsOnIntercepts)) { + require($(lowerBoundsOnIntercepts).size == numCoefficientSets, "The size of " + + "lowerBoundsOnIntercepts must be equal to 1 for binomial regression, or the number of " + + s"classes for multinomial regression, but found: ${getLowerBoundsOnIntercepts.size}.") + } + if (isSet(upperBoundsOnIntercepts)) { + require($(upperBoundsOnIntercepts).size == numCoefficientSets, "The size of " + + "upperBoundsOnIntercepts must be equal to 1 for binomial regression, or the number of " + + s"classes for multinomial regression, but found: ${getUpperBoundsOnIntercepts.size}.") + } + if (isSet(lowerBoundsOnCoefficients) && isSet(upperBoundsOnCoefficients)) { + require($(lowerBoundsOnCoefficients).toArray.zip($(upperBoundsOnCoefficients).toArray) + .forall(x => x._1 <= x._2), "LowerBoundsOnCoefficients should always be " + + "less than or equal to upperBoundsOnCoefficients, but found: " + + s"lowerBoundsOnCoefficients = $getLowerBoundsOnCoefficients, " + + s"upperBoundsOnCoefficients = $getUpperBoundsOnCoefficients.") + } + if (isSet(lowerBoundsOnIntercepts) && isSet(upperBoundsOnIntercepts)) { + require($(lowerBoundsOnIntercepts).toArray.zip($(upperBoundsOnIntercepts).toArray) + .forall(x => x._1 <= x._2), "LowerBoundsOnIntercepts should always be " + + "less than or equal to upperBoundsOnIntercepts, but found: " + + s"lowerBoundsOnIntercepts = $getLowerBoundsOnIntercepts, " + + s"upperBoundsOnIntercepts = $getUpperBoundsOnIntercepts.") } + } + + private var optInitialModel: Option[LogisticRegressionModel] = None + + private[spark] def setInitialModel(model: LogisticRegressionModel): this.type = { + this.optInitialModel = Some(model) + this + } + + override protected[spark] def train(dataset: Dataset[_]): LogisticRegressionModel = { + val handlePersistence = dataset.storageLevel == StorageLevel.NONE + train(dataset, handlePersistence) + } + + protected[spark] def train( + dataset: Dataset[_], + handlePersistence: Boolean): LogisticRegressionModel = { + val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) + val instances: RDD[Instance] = + dataset.select(col($(labelCol)), w, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, features: Vector) => + Instance(label, weight, features) + } - val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) + val instr = Instrumentation.create(this, instances) + instr.logParams(regParam, elasticNetParam, standardization, threshold, + maxIter, tol, fitIntercept) + val (summarizer, labelSummarizer) = { val seqOp = (c: (MultivariateOnlineSummarizer, MultiClassSummarizer), instance: Instance) => @@ -253,165 +514,495 @@ class LogisticRegression(override val uid: String) (c1._1.merge(c2._1), c1._2.merge(c2._2)) instances.treeAggregate( - new MultivariateOnlineSummarizer, new MultiClassSummarizer)(seqOp, combOp) + (new MultivariateOnlineSummarizer, new MultiClassSummarizer) + )(seqOp, combOp, $(aggregationDepth)) } val histogram = labelSummarizer.histogram val numInvalid = labelSummarizer.countInvalid - val numClasses = histogram.length val numFeatures = summarizer.mean.size + val numFeaturesPlusIntercept = if (getFitIntercept) numFeatures + 1 else numFeatures + + val numClasses = MetadataUtils.getNumClasses(dataset.schema($(labelCol))) match { + case Some(n: Int) => + require(n >= histogram.length, s"Specified number of classes $n was " + + s"less than the number of unique labels ${histogram.length}.") + n + case None => histogram.length + } - if (numInvalid != 0) { - val msg = s"Classification labels should be in {0 to ${numClasses - 1} " + - s"Found $numInvalid invalid labels." - logError(msg) - throw new SparkException(msg) + val isMultinomial = getFamily.toLowerCase(Locale.ROOT) match { + case "binomial" => + require(numClasses == 1 || numClasses == 2, s"Binomial family only supports 1 or 2 " + + s"outcome classes but found $numClasses.") + false + case "multinomial" => true + case "auto" => numClasses > 2 + case other => throw new IllegalArgumentException(s"Unsupported family: $other") } + val numCoefficientSets = if (isMultinomial) numClasses else 1 - if (numClasses > 2) { - val msg = s"Currently, LogisticRegression with ElasticNet in ML package only supports " + - s"binary classification. Found $numClasses in the input dataset." - logError(msg) - throw new SparkException(msg) + // Check params interaction is valid if fitting under bound constrained optimization. + if (usingBoundConstrainedOptimization) { + assertBoundConstrainedOptimizationParamsValid(numCoefficientSets, numFeatures) } - val featuresMean = summarizer.mean.toArray - val featuresStd = summarizer.variance.toArray.map(math.sqrt) + if (isDefined(thresholds)) { + require($(thresholds).length == numClasses, this.getClass.getSimpleName + + ".train() called with non-matching numClasses and thresholds.length." + + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") + } - val regParamL1 = $(elasticNetParam) * $(regParam) - val regParamL2 = (1.0 - $(elasticNetParam)) * $(regParam) + instr.logNumClasses(numClasses) + instr.logNumFeatures(numFeatures) - val costFun = new LogisticCostFun(instances, numClasses, $(fitIntercept), $(standardization), - featuresStd, featuresMean, regParamL2) + val (coefficientMatrix, interceptVector, objectiveHistory) = { + if (numInvalid != 0) { + val msg = s"Classification labels should be in [0 to ${numClasses - 1}]. " + + s"Found $numInvalid invalid labels." + logError(msg) + throw new SparkException(msg) + } - val optimizer = if ($(elasticNetParam) == 0.0 || $(regParam) == 0.0) { - new BreezeLBFGS[BDV[Double]]($(maxIter), 10, $(tol)) - } else { - def regParamL1Fun = (index: Int) => { - // Remove the L1 penalization on the intercept - if (index == numFeatures) { + val isConstantLabel = histogram.count(_ != 0.0) == 1 + + if ($(fitIntercept) && isConstantLabel && !usingBoundConstrainedOptimization) { + logWarning(s"All labels are the same value and fitIntercept=true, so the coefficients " + + s"will be zeros. Training is not needed.") + val constantLabelIndex = Vectors.dense(histogram).argmax + val coefMatrix = new SparseMatrix(numCoefficientSets, numFeatures, + new Array[Int](numCoefficientSets + 1), Array.empty[Int], Array.empty[Double], + isTransposed = true).compressed + val interceptVec = if (isMultinomial) { + Vectors.sparse(numClasses, Seq((constantLabelIndex, Double.PositiveInfinity))) + } else { + Vectors.dense(if (numClasses == 2) Double.PositiveInfinity else Double.NegativeInfinity) + } + (coefMatrix, interceptVec, Array.empty[Double]) + } else { + if (!$(fitIntercept) && isConstantLabel) { + logWarning(s"All labels belong to a single class and fitIntercept=false. It's a " + + s"dangerous ground, so the algorithm may not converge.") + } + + val featuresMean = summarizer.mean.toArray + val featuresStd = summarizer.variance.toArray.map(math.sqrt) + + if (!$(fitIntercept) && (0 until numFeatures).exists { i => + featuresStd(i) == 0.0 && featuresMean(i) != 0.0 }) { + logWarning("Fitting LogisticRegressionModel without intercept on dataset with " + + "constant nonzero column, Spark MLlib outputs zero coefficients for constant " + + "nonzero columns. This behavior is the same as R glmnet but different from LIBSVM.") + } + + val regParamL1 = $(elasticNetParam) * $(regParam) + val regParamL2 = (1.0 - $(elasticNetParam)) * $(regParam) + + val bcFeaturesStd = instances.context.broadcast(featuresStd) + val getAggregatorFunc = new LogisticAggregator(bcFeaturesStd, numClasses, $(fitIntercept), + multinomial = isMultinomial)(_) + val getFeaturesStd = (j: Int) => if (j >= 0 && j < numCoefficientSets * numFeatures) { + featuresStd(j / numCoefficientSets) + } else { 0.0 + } + + val regularization = if (regParamL2 != 0.0) { + val shouldApply = (idx: Int) => idx >= 0 && idx < numFeatures * numCoefficientSets + Some(new L2Regularization(regParamL2, shouldApply, + if ($(standardization)) None else Some(getFeaturesStd))) } else { - if ($(standardization)) { - regParamL1 + None + } + + val costFun = new RDDLossFunction(instances, getAggregatorFunc, regularization, + $(aggregationDepth)) + + val numCoeffsPlusIntercepts = numFeaturesPlusIntercept * numCoefficientSets + + val (lowerBounds, upperBounds): (Array[Double], Array[Double]) = { + if (usingBoundConstrainedOptimization) { + val lowerBounds = Array.fill[Double](numCoeffsPlusIntercepts)(Double.NegativeInfinity) + val upperBounds = Array.fill[Double](numCoeffsPlusIntercepts)(Double.PositiveInfinity) + val isSetLowerBoundsOnCoefficients = isSet(lowerBoundsOnCoefficients) + val isSetUpperBoundsOnCoefficients = isSet(upperBoundsOnCoefficients) + val isSetLowerBoundsOnIntercepts = isSet(lowerBoundsOnIntercepts) + val isSetUpperBoundsOnIntercepts = isSet(upperBoundsOnIntercepts) + + var i = 0 + while (i < numCoeffsPlusIntercepts) { + val coefficientSetIndex = i % numCoefficientSets + val featureIndex = i / numCoefficientSets + if (featureIndex < numFeatures) { + if (isSetLowerBoundsOnCoefficients) { + lowerBounds(i) = $(lowerBoundsOnCoefficients)( + coefficientSetIndex, featureIndex) * featuresStd(featureIndex) + } + if (isSetUpperBoundsOnCoefficients) { + upperBounds(i) = $(upperBoundsOnCoefficients)( + coefficientSetIndex, featureIndex) * featuresStd(featureIndex) + } + } else { + if (isSetLowerBoundsOnIntercepts) { + lowerBounds(i) = $(lowerBoundsOnIntercepts)(coefficientSetIndex) + } + if (isSetUpperBoundsOnIntercepts) { + upperBounds(i) = $(upperBoundsOnIntercepts)(coefficientSetIndex) + } + } + i += 1 + } + (lowerBounds, upperBounds) } else { - // If `standardization` is false, we still standardize the data - // to improve the rate of convergence; as a result, we have to - // perform this reverse standardization by penalizing each component - // differently to get effectively the same objective function when - // the training dataset is not standardized. - if (featuresStd(index) != 0.0) regParamL1 / featuresStd(index) else 0.0 + (null, null) } } - } - new BreezeOWLQN[Int, BDV[Double]]($(maxIter), 10, regParamL1Fun, $(tol)) - } - - val initialCoefficientsWithIntercept = - Vectors.zeros(if ($(fitIntercept)) numFeatures + 1 else numFeatures) - - if ($(fitIntercept)) { - /* - For binary logistic regression, when we initialize the coefficients as zeros, - it will converge faster if we initialize the intercept such that - it follows the distribution of the labels. - - {{{ - P(0) = 1 / (1 + \exp(b)), and - P(1) = \exp(b) / (1 + \exp(b)) - }}}, hence - {{{ - b = \log{P(1) / P(0)} = \log{count_1 / count_0} - }}} - */ - initialCoefficientsWithIntercept.toArray(numFeatures) - = math.log(histogram(1) / histogram(0)) - } - - val states = optimizer.iterations(new CachedDiffFunction(costFun), - initialCoefficientsWithIntercept.toBreeze.toDenseVector) - - val (coefficients, intercept, objectiveHistory) = { - /* - Note that in Logistic Regression, the objective history (loss + regularization) - is log-likelihood which is invariance under feature standardization. As a result, - the objective history from optimizer is the same as the one in the original space. - */ - val arrayBuilder = mutable.ArrayBuilder.make[Double] - var state: optimizer.State = null - while (states.hasNext) { - state = states.next() - arrayBuilder += state.adjustedValue - } - if (state == null) { - val msg = s"${optimizer.getClass.getName} failed." - logError(msg) - throw new SparkException(msg) - } + val optimizer = if ($(elasticNetParam) == 0.0 || $(regParam) == 0.0) { + if (lowerBounds != null && upperBounds != null) { + new BreezeLBFGSB( + BDV[Double](lowerBounds), BDV[Double](upperBounds), $(maxIter), 10, $(tol)) + } else { + new BreezeLBFGS[BDV[Double]]($(maxIter), 10, $(tol)) + } + } else { + val standardizationParam = $(standardization) + def regParamL1Fun = (index: Int) => { + // Remove the L1 penalization on the intercept + val isIntercept = $(fitIntercept) && index >= numFeatures * numCoefficientSets + if (isIntercept) { + 0.0 + } else { + if (standardizationParam) { + regParamL1 + } else { + val featureIndex = index / numCoefficientSets + // If `standardization` is false, we still standardize the data + // to improve the rate of convergence; as a result, we have to + // perform this reverse standardization by penalizing each component + // differently to get effectively the same objective function when + // the training dataset is not standardized. + if (featuresStd(featureIndex) != 0.0) { + regParamL1 / featuresStd(featureIndex) + } else { + 0.0 + } + } + } + } + new BreezeOWLQN[Int, BDV[Double]]($(maxIter), 10, regParamL1Fun, $(tol)) + } - /* - The coefficients are trained in the scaled space; we're converting them back to - the original space. - Note that the intercept in scaled space and original space is the same; - as a result, no scaling is needed. - */ - val rawCoefficients = state.x.toArray.clone() - var i = 0 - while (i < numFeatures) { - rawCoefficients(i) *= { if (featuresStd(i) != 0.0) 1.0 / featuresStd(i) else 0.0 } - i += 1 - } + /* + The coefficients are laid out in column major order during training. Here we initialize + a column major matrix of initial coefficients. + */ + val initialCoefWithInterceptMatrix = + Matrices.zeros(numCoefficientSets, numFeaturesPlusIntercept) + + val initialModelIsValid = optInitialModel match { + case Some(_initialModel) => + val providedCoefs = _initialModel.coefficientMatrix + val modelIsValid = (providedCoefs.numRows == numCoefficientSets) && + (providedCoefs.numCols == numFeatures) && + (_initialModel.interceptVector.size == numCoefficientSets) && + (_initialModel.getFitIntercept == $(fitIntercept)) + if (!modelIsValid) { + logWarning(s"Initial coefficients will be ignored! Its dimensions " + + s"(${providedCoefs.numRows}, ${providedCoefs.numCols}) did not match the " + + s"expected size ($numCoefficientSets, $numFeatures)") + } + modelIsValid + case None => false + } - if ($(fitIntercept)) { - (Vectors.dense(rawCoefficients.dropRight(1)).compressed, rawCoefficients.last, - arrayBuilder.result()) - } else { - (Vectors.dense(rawCoefficients).compressed, 0.0, arrayBuilder.result()) + if (initialModelIsValid) { + val providedCoef = optInitialModel.get.coefficientMatrix + providedCoef.foreachActive { (classIndex, featureIndex, value) => + // We need to scale the coefficients since they will be trained in the scaled space + initialCoefWithInterceptMatrix.update(classIndex, featureIndex, + value * featuresStd(featureIndex)) + } + if ($(fitIntercept)) { + optInitialModel.get.interceptVector.foreachActive { (classIndex, value) => + initialCoefWithInterceptMatrix.update(classIndex, numFeatures, value) + } + } + } else if ($(fitIntercept) && isMultinomial) { + /* + For multinomial logistic regression, when we initialize the coefficients as zeros, + it will converge faster if we initialize the intercepts such that + it follows the distribution of the labels. + {{{ + P(1) = \exp(b_1) / Z + ... + P(K) = \exp(b_K) / Z + where Z = \sum_{k=1}^{K} \exp(b_k) + }}} + Since this doesn't have a unique solution, one of the solutions that satisfies the + above equations is + {{{ + \exp(b_k) = count_k * \exp(\lambda) + b_k = \log(count_k) * \lambda + }}} + \lambda is a free parameter, so choose the phase \lambda such that the + mean is centered. This yields + {{{ + b_k = \log(count_k) + b_k' = b_k - \mean(b_k) + }}} + */ + val rawIntercepts = histogram.map(math.log1p) // add 1 for smoothing (log1p(x) = log(1+x)) + val rawMean = rawIntercepts.sum / rawIntercepts.length + rawIntercepts.indices.foreach { i => + initialCoefWithInterceptMatrix.update(i, numFeatures, rawIntercepts(i) - rawMean) + } + } else if ($(fitIntercept)) { + /* + For binary logistic regression, when we initialize the coefficients as zeros, + it will converge faster if we initialize the intercept such that + it follows the distribution of the labels. + + {{{ + P(0) = 1 / (1 + \exp(b)), and + P(1) = \exp(b) / (1 + \exp(b)) + }}}, hence + {{{ + b = \log{P(1) / P(0)} = \log{count_1 / count_0} + }}} + */ + initialCoefWithInterceptMatrix.update(0, numFeatures, + math.log(histogram(1) / histogram(0))) + } + + if (usingBoundConstrainedOptimization) { + // Make sure all initial values locate in the corresponding bound. + var i = 0 + while (i < numCoeffsPlusIntercepts) { + val coefficientSetIndex = i % numCoefficientSets + val featureIndex = i / numCoefficientSets + if (initialCoefWithInterceptMatrix(coefficientSetIndex, featureIndex) < lowerBounds(i)) + { + initialCoefWithInterceptMatrix.update( + coefficientSetIndex, featureIndex, lowerBounds(i)) + } else if ( + initialCoefWithInterceptMatrix(coefficientSetIndex, featureIndex) > upperBounds(i)) + { + initialCoefWithInterceptMatrix.update( + coefficientSetIndex, featureIndex, upperBounds(i)) + } + i += 1 + } + } + + val states = optimizer.iterations(new CachedDiffFunction(costFun), + new BDV[Double](initialCoefWithInterceptMatrix.toArray)) + + /* + Note that in Logistic Regression, the objective history (loss + regularization) + is log-likelihood which is invariant under feature standardization. As a result, + the objective history from optimizer is the same as the one in the original space. + */ + val arrayBuilder = mutable.ArrayBuilder.make[Double] + var state: optimizer.State = null + while (states.hasNext) { + state = states.next() + arrayBuilder += state.adjustedValue + } + bcFeaturesStd.destroy(blocking = false) + + if (state == null) { + val msg = s"${optimizer.getClass.getName} failed." + logError(msg) + throw new SparkException(msg) + } + + /* + The coefficients are trained in the scaled space; we're converting them back to + the original space. + + Additionally, since the coefficients were laid out in column major order during training + to avoid extra computation, we convert them back to row major before passing them to the + model. + + Note that the intercept in scaled space and original space is the same; + as a result, no scaling is needed. + */ + val allCoefficients = state.x.toArray.clone() + val allCoefMatrix = new DenseMatrix(numCoefficientSets, numFeaturesPlusIntercept, + allCoefficients) + val denseCoefficientMatrix = new DenseMatrix(numCoefficientSets, numFeatures, + new Array[Double](numCoefficientSets * numFeatures), isTransposed = true) + val interceptVec = if ($(fitIntercept) || !isMultinomial) { + Vectors.zeros(numCoefficientSets) + } else { + Vectors.sparse(numCoefficientSets, Seq.empty) + } + // separate intercepts and coefficients from the combined matrix + allCoefMatrix.foreachActive { (classIndex, featureIndex, value) => + val isIntercept = $(fitIntercept) && (featureIndex == numFeatures) + if (!isIntercept && featuresStd(featureIndex) != 0.0) { + denseCoefficientMatrix.update(classIndex, featureIndex, + value / featuresStd(featureIndex)) + } + if (isIntercept) interceptVec.toArray(classIndex) = value + } + + if ($(regParam) == 0.0 && isMultinomial && !usingBoundConstrainedOptimization) { + /* + When no regularization is applied, the multinomial coefficients lack identifiability + because we do not use a pivot class. We can add any constant value to the coefficients + and get the same likelihood. So here, we choose the mean centered coefficients for + reproducibility. This method follows the approach in glmnet, described here: + + Friedman, et al. "Regularization Paths for Generalized Linear Models via + Coordinate Descent," https://core.ac.uk/download/files/153/6287975.pdf + */ + val centers = Array.fill(numFeatures)(0.0) + denseCoefficientMatrix.foreachActive { case (i, j, v) => + centers(j) += v + } + centers.transform(_ / numCoefficientSets) + denseCoefficientMatrix.foreachActive { case (i, j, v) => + denseCoefficientMatrix.update(i, j, v - centers(j)) + } + } + + // center the intercepts when using multinomial algorithm + if ($(fitIntercept) && isMultinomial && !usingBoundConstrainedOptimization) { + val interceptArray = interceptVec.toArray + val interceptMean = interceptArray.sum / interceptArray.length + (0 until interceptVec.size).foreach { i => interceptArray(i) -= interceptMean } + } + (denseCoefficientMatrix.compressed, interceptVec.compressed, arrayBuilder.result()) } } if (handlePersistence) instances.unpersist() - val model = copyValues(new LogisticRegressionModel(uid, coefficients, intercept)) - val logRegSummary = new BinaryLogisticRegressionTrainingSummary( - model.transform(dataset), - $(probabilityCol), - $(labelCol), - $(featuresCol), - objectiveHistory) - model.setSummary(logRegSummary) + val model = copyValues(new LogisticRegressionModel(uid, coefficientMatrix, interceptVector, + numClasses, isMultinomial)) + + val (summaryModel, probabilityColName, predictionColName) = model.findSummaryModel() + val logRegSummary = if (numClasses <= 2) { + new BinaryLogisticRegressionTrainingSummaryImpl( + summaryModel.transform(dataset), + probabilityColName, + predictionColName, + $(labelCol), + $(featuresCol), + objectiveHistory) + } else { + new LogisticRegressionTrainingSummaryImpl( + summaryModel.transform(dataset), + probabilityColName, + predictionColName, + $(labelCol), + $(featuresCol), + objectiveHistory) + } + model.setSummary(Some(logRegSummary)) + instr.logSuccess(model) + model } + @Since("1.4.0") override def copy(extra: ParamMap): LogisticRegression = defaultCopy(extra) } +@Since("1.6.0") +object LogisticRegression extends DefaultParamsReadable[LogisticRegression] { + + @Since("1.6.0") + override def load(path: String): LogisticRegression = super.load(path) + + private[classification] val supportedFamilyNames = + Array("auto", "binomial", "multinomial").map(_.toLowerCase(Locale.ROOT)) +} + /** - * :: Experimental :: * Model produced by [[LogisticRegression]]. */ -@Experimental -class LogisticRegressionModel private[ml] ( - override val uid: String, - val coefficients: Vector, - val intercept: Double) +@Since("1.4.0") +class LogisticRegressionModel private[spark] ( + @Since("1.4.0") override val uid: String, + @Since("2.1.0") val coefficientMatrix: Matrix, + @Since("2.1.0") val interceptVector: Vector, + @Since("1.3.0") override val numClasses: Int, + private val isMultinomial: Boolean) extends ProbabilisticClassificationModel[Vector, LogisticRegressionModel] - with LogisticRegressionParams { + with LogisticRegressionParams with MLWritable { + + require(coefficientMatrix.numRows == interceptVector.size, s"Dimension mismatch! Expected " + + s"coefficientMatrix.numRows == interceptVector.size, but ${coefficientMatrix.numRows} != " + + s"${interceptVector.size}") + + private[spark] def this(uid: String, coefficients: Vector, intercept: Double) = + this(uid, new DenseMatrix(1, coefficients.size, coefficients.toArray, isTransposed = true), + Vectors.dense(intercept), 2, isMultinomial = false) + + /** + * A vector of model coefficients for "binomial" logistic regression. If this model was trained + * using the "multinomial" family then an exception is thrown. + * + * @return Vector + */ + @Since("2.0.0") + def coefficients: Vector = if (isMultinomial) { + throw new SparkException("Multinomial models contain a matrix of coefficients, use " + + "coefficientMatrix instead.") + } else { + _coefficients + } + + // convert to appropriate vector representation without replicating data + private lazy val _coefficients: Vector = { + require(coefficientMatrix.isTransposed, + "LogisticRegressionModel coefficients should be row major for binomial model.") + coefficientMatrix match { + case dm: DenseMatrix => Vectors.dense(dm.values) + case sm: SparseMatrix => Vectors.sparse(coefficientMatrix.numCols, sm.rowIndices, sm.values) + } + } - @deprecated("Use coefficients instead.", "1.6.0") - def weights: Vector = coefficients + /** + * The model intercept for "binomial" logistic regression. If this model was fit with the + * "multinomial" family then an exception is thrown. + * + * @return Double + */ + @Since("1.3.0") + def intercept: Double = if (isMultinomial) { + throw new SparkException("Multinomial models contain a vector of intercepts, use " + + "interceptVector instead.") + } else { + _intercept + } + + private lazy val _intercept = interceptVector.toArray.head + @Since("1.5.0") override def setThreshold(value: Double): this.type = super.setThreshold(value) + @Since("1.5.0") override def getThreshold: Double = super.getThreshold + @Since("1.5.0") override def setThresholds(value: Array[Double]): this.type = super.setThresholds(value) + @Since("1.5.0") override def getThresholds: Array[Double] = super.getThresholds /** Margin (rawPrediction) for class label 1. For binary classification only. */ private val margin: Vector => Double = (features) => { - BLAS.dot(features, coefficients) + intercept + BLAS.dot(features, _coefficients) + _intercept + } + + /** Margin (rawPrediction) for each class label. */ + private val margins: Vector => Vector = (features) => { + val m = interceptVector.toDense.copy + BLAS.gemv(1.0, coefficientMatrix, features, 1.0, m) + m } /** Score (probability) for class label 1. For binary classification only. */ @@ -420,48 +1011,88 @@ class LogisticRegressionModel private[ml] ( 1.0 / (1.0 + math.exp(-m)) } - override val numFeatures: Int = coefficients.size - - override val numClasses: Int = 2 + @Since("1.6.0") + override val numFeatures: Int = coefficientMatrix.numCols private var trainingSummary: Option[LogisticRegressionTrainingSummary] = None /** - * Gets summary of model on training set. An exception is - * thrown if `trainingSummary == None`. + * Gets summary of model on training set. An exception is thrown + * if `trainingSummary == None`. + */ + @Since("1.5.0") + def summary: LogisticRegressionTrainingSummary = trainingSummary.getOrElse { + throw new SparkException("No training summary available for this LogisticRegressionModel") + } + + /** + * Gets summary of model on training set. An exception is thrown + * if `trainingSummary == None` or it is a multiclass model. */ - def summary: LogisticRegressionTrainingSummary = trainingSummary match { - case Some(summ) => summ - case None => - throw new SparkException( - "No training summary available for this LogisticRegressionModel", - new NullPointerException()) + @Since("2.3.0") + def binarySummary: BinaryLogisticRegressionTrainingSummary = summary match { + case b: BinaryLogisticRegressionTrainingSummary => b + case _ => + throw new RuntimeException("Cannot create a binary summary for a non-binary model" + + s"(numClasses=${numClasses}), use summary instead.") } - private[classification] def setSummary( - summary: LogisticRegressionTrainingSummary): this.type = { - this.trainingSummary = Some(summary) + /** + * If the probability and prediction columns are set, this method returns the current model, + * otherwise it generates new columns for them and sets them as columns on a new copy of + * the current model + */ + private[classification] def findSummaryModel(): + (LogisticRegressionModel, String, String) = { + val model = if ($(probabilityCol).isEmpty && $(predictionCol).isEmpty) { + copy(ParamMap.empty) + .setProbabilityCol("probability_" + java.util.UUID.randomUUID.toString) + .setPredictionCol("prediction_" + java.util.UUID.randomUUID.toString) + } else if ($(probabilityCol).isEmpty) { + copy(ParamMap.empty).setProbabilityCol("probability_" + java.util.UUID.randomUUID.toString) + } else if ($(predictionCol).isEmpty) { + copy(ParamMap.empty).setPredictionCol("prediction_" + java.util.UUID.randomUUID.toString) + } else { + this + } + (model, model.getProbabilityCol, model.getPredictionCol) + } + + private[classification] + def setSummary(summary: Option[LogisticRegressionTrainingSummary]): this.type = { + this.trainingSummary = summary this } /** Indicates whether a training summary exists for this model instance. */ + @Since("1.5.0") def hasSummary: Boolean = trainingSummary.isDefined /** - * Evaluates the model on a testset. + * Evaluates the model on a test dataset. + * * @param dataset Test dataset to evaluate model on. */ - // TODO: decide on a good name before exposing to public API - private[classification] def evaluate(dataset: DataFrame): LogisticRegressionSummary = { - new BinaryLogisticRegressionSummary( - this.transform(dataset), $(probabilityCol), $(labelCol), $(featuresCol)) + @Since("2.0.0") + def evaluate(dataset: Dataset[_]): LogisticRegressionSummary = { + // Handle possible missing or invalid prediction columns + val (summaryModel, probabilityColName, predictionColName) = findSummaryModel() + if (numClasses > 2) { + new LogisticRegressionSummaryImpl(summaryModel.transform(dataset), + probabilityColName, predictionColName, $(labelCol), $(featuresCol)) + } else { + new BinaryLogisticRegressionSummaryImpl(summaryModel.transform(dataset), + probabilityColName, predictionColName, $(labelCol), $(featuresCol)) + } } /** * Predict label for the given feature vector. - * The behavior of this can be adjusted using [[thresholds]]. + * The behavior of this can be adjusted using `thresholds`. */ - override protected def predict(features: Vector): Double = { + override protected def predict(features: Vector): Double = if (isMultinomial) { + super.predict(features) + } else { // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden. if (score(features) > getThreshold) 1 else 0 } @@ -469,13 +1100,47 @@ class LogisticRegressionModel private[ml] ( override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { rawPrediction match { case dv: DenseVector => - var i = 0 - val size = dv.size - while (i < size) { - dv.values(i) = 1.0 / (1.0 + math.exp(-dv.values(i))) - i += 1 + if (isMultinomial) { + val size = dv.size + val values = dv.values + + // get the maximum margin + val maxMarginIndex = rawPrediction.argmax + val maxMargin = rawPrediction(maxMarginIndex) + + if (maxMargin == Double.PositiveInfinity) { + var k = 0 + while (k < size) { + values(k) = if (k == maxMarginIndex) 1.0 else 0.0 + k += 1 + } + } else { + val sum = { + var temp = 0.0 + var k = 0 + while (k < numClasses) { + values(k) = if (maxMargin > 0) { + math.exp(values(k) - maxMargin) + } else { + math.exp(values(k)) + } + temp += values(k) + k += 1 + } + temp + } + BLAS.scal(1 / sum, dv) + } + dv + } else { + var i = 0 + val size = dv.size + while (i < size) { + dv.values(i) = 1.0 / (1.0 + math.exp(-dv.values(i))) + i += 1 + } + dv } - dv case sv: SparseVector => throw new RuntimeException("Unexpected error in LogisticRegressionModel:" + " raw2probabilitiesInPlace encountered SparseVector") @@ -483,44 +1148,141 @@ class LogisticRegressionModel private[ml] ( } override protected def predictRaw(features: Vector): Vector = { - val m = margin(features) - Vectors.dense(-m, m) + if (isMultinomial) { + margins(features) + } else { + val m = margin(features) + Vectors.dense(-m, m) + } } + @Since("1.4.0") override def copy(extra: ParamMap): LogisticRegressionModel = { - val newModel = copyValues(new LogisticRegressionModel(uid, coefficients, intercept), extra) - if (trainingSummary.isDefined) newModel.setSummary(trainingSummary.get) - newModel.setParent(parent) + val newModel = copyValues(new LogisticRegressionModel(uid, coefficientMatrix, interceptVector, + numClasses, isMultinomial), extra) + newModel.setSummary(trainingSummary).setParent(parent) } override protected def raw2prediction(rawPrediction: Vector): Double = { - // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden. - val t = getThreshold - val rawThreshold = if (t == 0.0) { - Double.NegativeInfinity - } else if (t == 1.0) { - Double.PositiveInfinity + if (isMultinomial) { + super.raw2prediction(rawPrediction) } else { - math.log(t / (1.0 - t)) + // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden. + val t = getThreshold + val rawThreshold = if (t == 0.0) { + Double.NegativeInfinity + } else if (t == 1.0) { + Double.PositiveInfinity + } else { + math.log(t / (1.0 - t)) + } + if (rawPrediction(1) > rawThreshold) 1 else 0 } - if (rawPrediction(1) > rawThreshold) 1 else 0 } override protected def probability2prediction(probability: Vector): Double = { - // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden. - if (probability(1) > getThreshold) 1 else 0 + if (isMultinomial) { + super.probability2prediction(probability) + } else { + // Note: We should use getThreshold instead of $(threshold) since getThreshold is overridden. + if (probability(1) > getThreshold) 1 else 0 + } } + + /** + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. + * + * For [[LogisticRegressionModel]], this does NOT currently save the training [[summary]]. + * An option to save [[summary]] may be added in the future. + * + * This also does not save the [[parent]] currently. + */ + @Since("1.6.0") + override def write: MLWriter = new LogisticRegressionModel.LogisticRegressionModelWriter(this) } + +@Since("1.6.0") +object LogisticRegressionModel extends MLReadable[LogisticRegressionModel] { + + @Since("1.6.0") + override def read: MLReader[LogisticRegressionModel] = new LogisticRegressionModelReader + + @Since("1.6.0") + override def load(path: String): LogisticRegressionModel = super.load(path) + + /** [[MLWriter]] instance for [[LogisticRegressionModel]] */ + private[LogisticRegressionModel] + class LogisticRegressionModelWriter(instance: LogisticRegressionModel) + extends MLWriter with Logging { + + private case class Data( + numClasses: Int, + numFeatures: Int, + interceptVector: Vector, + coefficientMatrix: Matrix, + isMultinomial: Boolean) + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + // Save model data: numClasses, numFeatures, intercept, coefficients + val data = Data(instance.numClasses, instance.numFeatures, instance.interceptVector, + instance.coefficientMatrix, instance.isMultinomial) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class LogisticRegressionModelReader extends MLReader[LogisticRegressionModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[LogisticRegressionModel].getName + + override def load(path: String): LogisticRegressionModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val (major, minor) = VersionUtils.majorMinorVersion(metadata.sparkVersion) + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.format("parquet").load(dataPath) + + val model = if (major.toInt < 2 || (major.toInt == 2 && minor.toInt == 0)) { + // 2.0 and before + val Row(numClasses: Int, numFeatures: Int, intercept: Double, coefficients: Vector) = + MLUtils.convertVectorColumnsToML(data, "coefficients") + .select("numClasses", "numFeatures", "intercept", "coefficients") + .head() + val coefficientMatrix = + new DenseMatrix(1, coefficients.size, coefficients.toArray, isTransposed = true) + val interceptVector = Vectors.dense(intercept) + new LogisticRegressionModel(metadata.uid, coefficientMatrix, + interceptVector, numClasses, isMultinomial = false) + } else { + // 2.1+ + val Row(numClasses: Int, numFeatures: Int, interceptVector: Vector, + coefficientMatrix: Matrix, isMultinomial: Boolean) = data + .select("numClasses", "numFeatures", "interceptVector", "coefficientMatrix", + "isMultinomial").head() + new LogisticRegressionModel(metadata.uid, coefficientMatrix, interceptVector, + numClasses, isMultinomial) + } + + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } +} + + /** * MultiClassSummarizer computes the number of distinct labels and corresponding counts, * and validates the data to see if the labels used for k class multi-label classification - * are in the range of {0, 1, ..., k - 1} in a online fashion. + * are in the range of {0, 1, ..., k - 1} in an online fashion. * * Two MultilabelSummarizer can be merged together to have a statistical summary of the * corresponding joint dataset. */ -private[classification] class MultiClassSummarizer extends Serializable { +private[ml] class MultiClassSummarizer extends Serializable { // The first element of value in distinctMap is the actually number of instances, // and the second element of value is sum of the weights. private val distinctMap = new mutable.HashMap[Int, (Long, Double)] @@ -528,12 +1290,13 @@ private[classification] class MultiClassSummarizer extends Serializable { /** * Add a new label into this MultilabelSummarizer, and update the distinct map. + * * @param label The label for this data point. * @param weight The weight of this instances. * @return This MultilabelSummarizer */ def add(label: Double, weight: Double = 1.0): this.type = { - require(weight >= 0.0, s"instance weight, ${weight} has to be >= 0.0") + require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0") if (weight == 0.0) return this @@ -575,7 +1338,7 @@ private[classification] class MultiClassSummarizer extends Serializable { def countInvalid: Long = totalInvalidCnt /** @return The number of distinct labels in the input dataset. */ - def numClasses: Int = distinctMap.keySet.max + 1 + def numClasses: Int = if (distinctMap.isEmpty) 0 else distinctMap.keySet.max + 1 /** @return The weightSum of each label in the input dataset. */ def histogram: Array[Double] = { @@ -591,113 +1354,215 @@ private[classification] class MultiClassSummarizer extends Serializable { } /** - * Abstraction for multinomial Logistic Regression Training results. - * Currently, the training summary ignores the training weights except - * for the objective trace. - */ -sealed trait LogisticRegressionTrainingSummary extends LogisticRegressionSummary { - - /** objective function (scaled loss + regularization) at each iteration. */ - def objectiveHistory: Array[Double] - - /** Number of training iterations until termination */ - def totalIterations: Int = objectiveHistory.length - -} - -/** - * Abstraction for Logistic Regression Results for a given model. + * :: Experimental :: + * Abstraction for logistic regression results for a given model. + * + * Currently, the summary ignores the instance weights. */ +@Experimental sealed trait LogisticRegressionSummary extends Serializable { - /** Dataframe outputted by the model's `transform` method. */ + /** + * Dataframe output by the model's `transform` method. + */ + @Since("1.5.0") def predictions: DataFrame - /** Field in "predictions" which gives the calibrated probability of each instance as a vector. */ + /** Field in "predictions" which gives the probability of each class as a vector. */ + @Since("1.5.0") def probabilityCol: String - /** Field in "predictions" which gives the true label of each instance. */ + /** Field in "predictions" which gives the prediction of each class. */ + @Since("2.3.0") + def predictionCol: String + + /** Field in "predictions" which gives the true label of each instance (if available). */ + @Since("1.5.0") def labelCol: String /** Field in "predictions" which gives the features of each instance as a vector. */ + @Since("1.6.0") def featuresCol: String + @transient private val multiclassMetrics = { + new MulticlassMetrics( + predictions.select( + col(predictionCol), + col(labelCol).cast(DoubleType)) + .rdd.map { case Row(prediction: Double, label: Double) => (prediction, label) }) + } + + /** + * Returns the sequence of labels in ascending order. This order matches the order used + * in metrics which are specified as arrays over labels, e.g., truePositiveRateByLabel. + * + * Note: In most cases, it will be values {0.0, 1.0, ..., numClasses-1}, However, if the + * training set is missing a label, then all of the arrays over labels + * (e.g., from truePositiveRateByLabel) will be of length numClasses-1 instead of the + * expected numClasses. + */ + @Since("2.3.0") + def labels: Array[Double] = multiclassMetrics.labels + + /** Returns true positive rate for each label (category). */ + @Since("2.3.0") + def truePositiveRateByLabel: Array[Double] = recallByLabel + + /** Returns false positive rate for each label (category). */ + @Since("2.3.0") + def falsePositiveRateByLabel: Array[Double] = { + multiclassMetrics.labels.map(label => multiclassMetrics.falsePositiveRate(label)) + } + + /** Returns precision for each label (category). */ + @Since("2.3.0") + def precisionByLabel: Array[Double] = { + multiclassMetrics.labels.map(label => multiclassMetrics.precision(label)) + } + + /** Returns recall for each label (category). */ + @Since("2.3.0") + def recallByLabel: Array[Double] = { + multiclassMetrics.labels.map(label => multiclassMetrics.recall(label)) + } + + /** Returns f-measure for each label (category). */ + @Since("2.3.0") + def fMeasureByLabel(beta: Double): Array[Double] = { + multiclassMetrics.labels.map(label => multiclassMetrics.fMeasure(label, beta)) + } + + /** Returns f1-measure for each label (category). */ + @Since("2.3.0") + def fMeasureByLabel: Array[Double] = fMeasureByLabel(1.0) + + /** + * Returns accuracy. + * (equals to the total number of correctly classified instances + * out of the total number of instances.) + */ + @Since("2.3.0") + def accuracy: Double = multiclassMetrics.accuracy + + /** + * Returns weighted true positive rate. + * (equals to precision, recall and f-measure) + */ + @Since("2.3.0") + def weightedTruePositiveRate: Double = weightedRecall + + /** Returns weighted false positive rate. */ + @Since("2.3.0") + def weightedFalsePositiveRate: Double = multiclassMetrics.weightedFalsePositiveRate + + /** + * Returns weighted averaged recall. + * (equals to precision, recall and f-measure) + */ + @Since("2.3.0") + def weightedRecall: Double = multiclassMetrics.weightedRecall + + /** Returns weighted averaged precision. */ + @Since("2.3.0") + def weightedPrecision: Double = multiclassMetrics.weightedPrecision + + /** Returns weighted averaged f-measure. */ + @Since("2.3.0") + def weightedFMeasure(beta: Double): Double = multiclassMetrics.weightedFMeasure(beta) + + /** Returns weighted averaged f1-measure. */ + @Since("2.3.0") + def weightedFMeasure: Double = multiclassMetrics.weightedFMeasure(1.0) + + /** + * Convenient method for casting to binary logistic regression summary. + * This method will throws an Exception if the summary is not a binary summary. + */ + @Since("2.3.0") + def asBinary: BinaryLogisticRegressionSummary = this match { + case b: BinaryLogisticRegressionSummary => b + case _ => + throw new RuntimeException("Cannot cast to a binary summary.") + } } /** * :: Experimental :: - * Logistic regression training results. - * @param predictions dataframe outputted by the model's `transform` method. - * @param probabilityCol field in "predictions" which gives the calibrated probability of - * each instance as a vector. - * @param labelCol field in "predictions" which gives the true label of each instance. - * @param featuresCol field in "predictions" which gives the features of each instance as a vector. - * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. + * Abstraction for multiclass logistic regression training results. + * Currently, the training summary ignores the training weights except + * for the objective trace. */ @Experimental -class BinaryLogisticRegressionTrainingSummary private[classification] ( - predictions: DataFrame, - probabilityCol: String, - labelCol: String, - featuresCol: String, - val objectiveHistory: Array[Double]) - extends BinaryLogisticRegressionSummary(predictions, probabilityCol, labelCol, featuresCol) - with LogisticRegressionTrainingSummary { +sealed trait LogisticRegressionTrainingSummary extends LogisticRegressionSummary { + + /** objective function (scaled loss + regularization) at each iteration. */ + @Since("1.5.0") + def objectiveHistory: Array[Double] + + /** Number of training iterations. */ + @Since("1.5.0") + def totalIterations: Int = objectiveHistory.length } /** * :: Experimental :: - * Binary Logistic regression results for a given model. - * @param predictions dataframe outputted by the model's `transform` method. - * @param probabilityCol field in "predictions" which gives the calibrated probability of - * each instance. - * @param labelCol field in "predictions" which gives the true label of each instance. - * @param featuresCol field in "predictions" which gives the features of each instance as a vector. + * Abstraction for binary logistic regression results for a given model. + * + * Currently, the summary ignores the instance weights. */ @Experimental -class BinaryLogisticRegressionSummary private[classification] ( - @transient override val predictions: DataFrame, - override val probabilityCol: String, - override val labelCol: String, - override val featuresCol: String) extends LogisticRegressionSummary { +sealed trait BinaryLogisticRegressionSummary extends LogisticRegressionSummary { - private val sqlContext = predictions.sqlContext - import sqlContext.implicits._ + private val sparkSession = predictions.sparkSession + import sparkSession.implicits._ - /** - * Returns a BinaryClassificationMetrics object. - */ // TODO: Allow the user to vary the number of bins using a setBins method in // BinaryClassificationMetrics. For now the default is set to 100. @transient private val binaryMetrics = new BinaryClassificationMetrics( - predictions.select(probabilityCol, labelCol).map { + predictions.select(col(probabilityCol), col(labelCol).cast(DoubleType)).rdd.map { case Row(score: Vector, label: Double) => (score(1), label) }, 100 ) /** * Returns the receiver operating characteristic (ROC) curve, - * which is an Dataframe having two fields (FPR, TPR) + * which is a Dataframe having two fields (FPR, TPR) * with (0.0, 0.0) prepended and (1.0, 1.0) appended to it. - * @see http://en.wikipedia.org/wiki/Receiver_operating_characteristic + * See http://en.wikipedia.org/wiki/Receiver_operating_characteristic + * + * @note This ignores instance weights (setting all to 1.0) from `LogisticRegression.weightCol`. + * This will change in later Spark versions. */ + @Since("1.5.0") @transient lazy val roc: DataFrame = binaryMetrics.roc().toDF("FPR", "TPR") /** * Computes the area under the receiver operating characteristic (ROC) curve. + * + * @note This ignores instance weights (setting all to 1.0) from `LogisticRegression.weightCol`. + * This will change in later Spark versions. */ + @Since("1.5.0") lazy val areaUnderROC: Double = binaryMetrics.areaUnderROC() /** - * Returns the precision-recall curve, which is an Dataframe containing + * Returns the precision-recall curve, which is a Dataframe containing * two fields recall, precision with (0.0, 1.0) prepended to it. + * + * @note This ignores instance weights (setting all to 1.0) from `LogisticRegression.weightCol`. + * This will change in later Spark versions. */ + @Since("1.5.0") @transient lazy val pr: DataFrame = binaryMetrics.pr().toDF("recall", "precision") /** * Returns a dataframe with two fields (threshold, F-Measure) curve with beta = 1.0. + * + * @note This ignores instance weights (setting all to 1.0) from `LogisticRegression.weightCol`. + * This will change in later Spark versions. */ + @Since("1.5.0") @transient lazy val fMeasureByThreshold: DataFrame = { binaryMetrics.fMeasureByThreshold().toDF("threshold", "F-Measure") } @@ -706,7 +1571,11 @@ class BinaryLogisticRegressionSummary private[classification] ( * Returns a dataframe with two fields (threshold, precision) curve. * Every possible probability obtained in transforming the dataset are used * as thresholds used in calculating the precision. + * + * @note This ignores instance weights (setting all to 1.0) from `LogisticRegression.weightCol`. + * This will change in later Spark versions. */ + @Since("1.5.0") @transient lazy val precisionByThreshold: DataFrame = { binaryMetrics.precisionByThreshold().toDF("threshold", "precision") } @@ -715,217 +1584,108 @@ class BinaryLogisticRegressionSummary private[classification] ( * Returns a dataframe with two fields (threshold, recall) curve. * Every possible probability obtained in transforming the dataset are used * as thresholds used in calculating the recall. + * + * @note This ignores instance weights (setting all to 1.0) from `LogisticRegression.weightCol`. + * This will change in later Spark versions. */ + @Since("1.5.0") @transient lazy val recallByThreshold: DataFrame = { binaryMetrics.recallByThreshold().toDF("threshold", "recall") } } /** - * LogisticAggregator computes the gradient and loss for binary logistic loss function, as used - * in binary classification for instances in sparse or dense vector in a online fashion. - * - * Note that multinomial logistic loss is not supported yet! - * - * Two LogisticAggregator can be merged together to have a summary of loss and gradient of - * the corresponding joint dataset. - * - * @param coefficients The coefficients corresponding to the features. - * @param numClasses the number of possible outcomes for k classes classification problem in - * Multinomial Logistic Regression. - * @param fitIntercept Whether to fit an intercept term. - * @param featuresStd The standard deviation values of the features. - * @param featuresMean The mean values of the features. + * :: Experimental :: + * Abstraction for binary logistic regression training results. + * Currently, the training summary ignores the training weights except + * for the objective trace. */ -private class LogisticAggregator( - coefficients: Vector, - numClasses: Int, - fitIntercept: Boolean, - featuresStd: Array[Double], - featuresMean: Array[Double]) extends Serializable { - - private var weightSum = 0.0 - private var lossSum = 0.0 - - private val coefficientsArray = coefficients match { - case dv: DenseVector => dv.values - case _ => - throw new IllegalArgumentException( - s"coefficients only supports dense vector but got type ${coefficients.getClass}.") - } - - private val dim = if (fitIntercept) coefficientsArray.length - 1 else coefficientsArray.length - - private val gradientSumArray = Array.ofDim[Double](coefficientsArray.length) - - /** - * Add a new training instance to this LogisticAggregator, and update the loss and gradient - * of the objective function. - * - * @param instance The instance of data point to be added. - * @return This LogisticAggregator object. - */ - def add(instance: Instance): this.type = { - instance match { case Instance(label, weight, features) => - require(dim == features.size, s"Dimensions mismatch when adding new instance." + - s" Expecting $dim but got ${features.size}.") - require(weight >= 0.0, s"instance weight, ${weight} has to be >= 0.0") - - if (weight == 0.0) return this - - val localCoefficientsArray = coefficientsArray - val localGradientSumArray = gradientSumArray - - numClasses match { - case 2 => - // For Binary Logistic Regression. - val margin = - { - var sum = 0.0 - features.foreachActive { (index, value) => - if (featuresStd(index) != 0.0 && value != 0.0) { - sum += localCoefficientsArray(index) * (value / featuresStd(index)) - } - } - sum + { - if (fitIntercept) localCoefficientsArray(dim) else 0.0 - } - } - - val multiplier = weight * (1.0 / (1.0 + math.exp(margin)) - label) - - features.foreachActive { (index, value) => - if (featuresStd(index) != 0.0 && value != 0.0) { - localGradientSumArray(index) += multiplier * (value / featuresStd(index)) - } - } - - if (fitIntercept) { - localGradientSumArray(dim) += multiplier - } - - if (label > 0) { - // The following is equivalent to log(1 + exp(margin)) but more numerically stable. - lossSum += weight * MLUtils.log1pExp(margin) - } else { - lossSum += weight * (MLUtils.log1pExp(margin) - margin) - } - case _ => - new NotImplementedError("LogisticRegression with ElasticNet in ML package " + - "only supports binary classification for now.") - } - weightSum += weight - this - } - } - - /** - * Merge another LogisticAggregator, and update the loss and gradient - * of the objective function. - * (Note that it's in place merging; as a result, `this` object will be modified.) - * - * @param other The other LogisticAggregator to be merged. - * @return This LogisticAggregator object. - */ - def merge(other: LogisticAggregator): this.type = { - require(dim == other.dim, s"Dimensions mismatch when merging with another " + - s"LeastSquaresAggregator. Expecting $dim but got ${other.dim}.") - - if (other.weightSum != 0.0) { - weightSum += other.weightSum - lossSum += other.lossSum - - var i = 0 - val localThisGradientSumArray = this.gradientSumArray - val localOtherGradientSumArray = other.gradientSumArray - val len = localThisGradientSumArray.length - while (i < len) { - localThisGradientSumArray(i) += localOtherGradientSumArray(i) - i += 1 - } - } - this - } - - def loss: Double = { - require(weightSum > 0.0, s"The effective number of instances should be " + - s"greater than 0.0, but $weightSum.") - lossSum / weightSum - } - - def gradient: Vector = { - require(weightSum > 0.0, s"The effective number of instances should be " + - s"greater than 0.0, but $weightSum.") - val result = Vectors.dense(gradientSumArray.clone()) - scal(1.0 / weightSum, result) - result - } -} +@Experimental +sealed trait BinaryLogisticRegressionTrainingSummary extends BinaryLogisticRegressionSummary + with LogisticRegressionTrainingSummary /** - * LogisticCostFun implements Breeze's DiffFunction[T] for a multinomial logistic loss function, - * as used in multi-class classification (it is also used in binary logistic regression). - * It returns the loss and gradient with L2 regularization at a particular point (coefficients). - * It's used in Breeze's convex optimization routines. + * Multiclass logistic regression training results. + * + * @param predictions dataframe output by the model's `transform` method. + * @param probabilityCol field in "predictions" which gives the probability of + * each class as a vector. + * @param predictionCol field in "predictions" which gives the prediction for a data instance as a + * double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param featuresCol field in "predictions" which gives the features of each instance as a vector. + * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. */ -private class LogisticCostFun( - instances: RDD[Instance], - numClasses: Int, - fitIntercept: Boolean, - standardization: Boolean, - featuresStd: Array[Double], - featuresMean: Array[Double], - regParamL2: Double) extends DiffFunction[BDV[Double]] { - - override def calculate(coefficients: BDV[Double]): (Double, BDV[Double]) = { - val numFeatures = featuresStd.length - val coeffs = Vectors.fromBreeze(coefficients) - - val logisticAggregator = { - val seqOp = (c: LogisticAggregator, instance: Instance) => c.add(instance) - val combOp = (c1: LogisticAggregator, c2: LogisticAggregator) => c1.merge(c2) - - instances.treeAggregate( - new LogisticAggregator(coeffs, numClasses, fitIntercept, featuresStd, featuresMean) - )(seqOp, combOp) - } +private class LogisticRegressionTrainingSummaryImpl( + predictions: DataFrame, + probabilityCol: String, + predictionCol: String, + labelCol: String, + featuresCol: String, + override val objectiveHistory: Array[Double]) + extends LogisticRegressionSummaryImpl( + predictions, probabilityCol, predictionCol, labelCol, featuresCol) + with LogisticRegressionTrainingSummary - val totalGradientArray = logisticAggregator.gradient.toArray +/** + * Multiclass logistic regression results for a given model. + * + * @param predictions dataframe output by the model's `transform` method. + * @param probabilityCol field in "predictions" which gives the probability of + * each class as a vector. + * @param predictionCol field in "predictions" which gives the prediction for a data instance as a + * double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param featuresCol field in "predictions" which gives the features of each instance as a vector. + */ +private class LogisticRegressionSummaryImpl( + @transient override val predictions: DataFrame, + override val probabilityCol: String, + override val predictionCol: String, + override val labelCol: String, + override val featuresCol: String) + extends LogisticRegressionSummary - // regVal is the sum of coefficients squares excluding intercept for L2 regularization. - val regVal = if (regParamL2 == 0.0) { - 0.0 - } else { - var sum = 0.0 - coeffs.foreachActive { (index, value) => - // If `fitIntercept` is true, the last term which is intercept doesn't - // contribute to the regularization. - if (index != numFeatures) { - // The following code will compute the loss of the regularization; also - // the gradient of the regularization, and add back to totalGradientArray. - sum += { - if (standardization) { - totalGradientArray(index) += regParamL2 * value - value * value - } else { - if (featuresStd(index) != 0.0) { - // If `standardization` is false, we still standardize the data - // to improve the rate of convergence; as a result, we have to - // perform this reverse standardization by penalizing each component - // differently to get effectively the same objective function when - // the training dataset is not standardized. - val temp = value / (featuresStd(index) * featuresStd(index)) - totalGradientArray(index) += regParamL2 * temp - value * temp - } else { - 0.0 - } - } - } - } - } - 0.5 * regParamL2 * sum - } +/** + * Binary logistic regression training results. + * + * @param predictions dataframe output by the model's `transform` method. + * @param probabilityCol field in "predictions" which gives the probability of + * each class as a vector. + * @param predictionCol field in "predictions" which gives the prediction for a data instance as a + * double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param featuresCol field in "predictions" which gives the features of each instance as a vector. + * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. + */ +private class BinaryLogisticRegressionTrainingSummaryImpl( + predictions: DataFrame, + probabilityCol: String, + predictionCol: String, + labelCol: String, + featuresCol: String, + override val objectiveHistory: Array[Double]) + extends BinaryLogisticRegressionSummaryImpl( + predictions, probabilityCol, predictionCol, labelCol, featuresCol) + with BinaryLogisticRegressionTrainingSummary - (logisticAggregator.loss + regVal, new BDV(totalGradientArray)) - } -} +/** + * Binary logistic regression results for a given model. + * + * @param predictions dataframe output by the model's `transform` method. + * @param probabilityCol field in "predictions" which gives the probability of + * each class as a vector. + * @param predictionCol field in "predictions" which gives the prediction of + * each class as a double. + * @param labelCol field in "predictions" which gives the true label of each instance. + * @param featuresCol field in "predictions" which gives the features of each instance as a vector. + */ +private class BinaryLogisticRegressionSummaryImpl( + predictions: DataFrame, + probabilityCol: String, + predictionCol: String, + labelCol: String, + featuresCol: String) + extends LogisticRegressionSummaryImpl( + predictions, probabilityCol, predictionCol, labelCol, featuresCol) + with BinaryLogisticRegressionSummary diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index cd7462596dd9..fd4c98f22132 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -19,33 +19,37 @@ package org.apache.spark.ml.classification import scala.collection.JavaConverters._ -import org.apache.spark.annotation.Experimental -import org.apache.spark.ml.param.shared.{HasTol, HasMaxIter, HasSeed} -import org.apache.spark.ml.{PredictorParams, PredictionModel, Predictor} -import org.apache.spark.ml.param.{IntParam, ParamValidators, IntArrayParam, ParamMap} -import org.apache.spark.ml.util.Identifiable -import org.apache.spark.ml.ann.{FeedForwardTrainer, FeedForwardTopology} -import org.apache.spark.mllib.linalg.{Vectors, Vector} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.sql.DataFrame +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since +import org.apache.spark.ml.ann.{FeedForwardTopology, FeedForwardTrainer} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util._ +import org.apache.spark.sql.Dataset /** Params for Multilayer Perceptron. */ -private[ml] trait MultilayerPerceptronParams extends PredictorParams - with HasSeed with HasMaxIter with HasTol { +private[classification] trait MultilayerPerceptronParams extends ProbabilisticClassifierParams + with HasSeed with HasMaxIter with HasTol with HasStepSize with HasSolver { + + import MultilayerPerceptronClassifier._ + /** * Layer sizes including input size and output size. - * Default: Array(1, 1) + * * @group param */ + @Since("1.5.0") final val layers: IntArrayParam = new IntArrayParam(this, "layers", - "Sizes of layers from input layer to output layer" + - " E.g., Array(780, 100, 10) means 780 inputs, " + + "Sizes of layers from input layer to output layer. " + + "E.g., Array(780, 100, 10) means 780 inputs, " + "one hidden layer with 100 neurons and output layer of 10 neurons.", - // TODO: how to check ALSO that all elements are greater than 0? - ParamValidators.arrayLengthGt(1) - ) + (t: Array[Int]) => t.forall(ParamValidators.gt(0)) && t.length > 1) /** @group getParam */ + @Since("1.5.0") final def getLayers: Array[Int] = $(layers) /** @@ -54,18 +58,48 @@ private[ml] trait MultilayerPerceptronParams extends PredictorParams * a partition then it is adjusted to the size of this data. * Recommended size is between 10 and 1000. * Default: 128 + * * @group expertParam */ + @Since("1.5.0") final val blockSize: IntParam = new IntParam(this, "blockSize", "Block size for stacking input data in matrices. Data is stacked within partitions." + " If block size is more than remaining data in a partition then " + "it is adjusted to the size of this data. Recommended size is between 10 and 1000", ParamValidators.gt(0)) - /** @group getParam */ + /** @group expertGetParam */ + @Since("1.5.0") final def getBlockSize: Int = $(blockSize) - setDefault(maxIter -> 100, tol -> 1e-4, layers -> Array(1, 1), blockSize -> 128) + /** + * The solver algorithm for optimization. + * Supported options: "gd" (minibatch gradient descent) or "l-bfgs". + * Default: "l-bfgs" + * + * @group expertParam + */ + @Since("2.0.0") + final override val solver: Param[String] = new Param[String](this, "solver", + "The solver algorithm for optimization. Supported options: " + + s"${supportedSolvers.mkString(", ")}. (Default l-bfgs)", + ParamValidators.inArray[String](supportedSolvers)) + + /** + * The initial weights of the model. + * + * @group expertParam + */ + @Since("2.0.0") + final val initialWeights: Param[Vector] = new Param[Vector](this, "initialWeights", + "The initial weights of the model") + + /** @group expertGetParam */ + @Since("2.0.0") + final def getInitialWeights: Vector = $(initialWeights) + + setDefault(maxIter -> 100, tol -> 1e-6, blockSize -> 128, + solver -> LBFGS, stepSize -> 0.03) } /** Label to vector converter. */ @@ -99,91 +133,183 @@ private object LabelConverter { } /** - * :: Experimental :: * Classifier trainer based on the Multilayer Perceptron. * Each layer has sigmoid activation function, output layer has softmax. * Number of inputs has to be equal to the size of feature vectors. * Number of outputs has to be equal to the total number of labels. * */ -@Experimental -class MultilayerPerceptronClassifier(override val uid: String) - extends Predictor[Vector, MultilayerPerceptronClassifier, MultilayerPerceptronClassificationModel] - with MultilayerPerceptronParams { +@Since("1.5.0") +class MultilayerPerceptronClassifier @Since("1.5.0") ( + @Since("1.5.0") override val uid: String) + extends ProbabilisticClassifier[Vector, MultilayerPerceptronClassifier, + MultilayerPerceptronClassificationModel] + with MultilayerPerceptronParams with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("mlpc")) - /** @group setParam */ + /** + * Sets the value of param [[layers]]. + * + * @group setParam + */ + @Since("1.5.0") def setLayers(value: Array[Int]): this.type = set(layers, value) - /** @group setParam */ + /** + * Sets the value of param [[blockSize]]. + * Default is 128. + * + * @group expertSetParam + */ + @Since("1.5.0") def setBlockSize(value: Int): this.type = set(blockSize, value) + /** + * Sets the value of param [[solver]]. + * Default is "l-bfgs". + * + * @group expertSetParam + */ + @Since("2.0.0") + def setSolver(value: String): this.type = set(solver, value) + /** * Set the maximum number of iterations. * Default is 100. + * * @group setParam */ + @Since("1.5.0") def setMaxIter(value: Int): this.type = set(maxIter, value) /** * Set the convergence tolerance of iterations. * Smaller value will lead to higher accuracy with the cost of more iterations. - * Default is 1E-4. + * Default is 1E-6. + * * @group setParam */ + @Since("1.5.0") def setTol(value: Double): this.type = set(tol, value) /** - * Set the seed for weights initialization. + * Set the seed for weights initialization if weights are not set + * * @group setParam */ + @Since("1.5.0") def setSeed(value: Long): this.type = set(seed, value) + /** + * Sets the value of param [[initialWeights]]. + * + * @group expertSetParam + */ + @Since("2.0.0") + def setInitialWeights(value: Vector): this.type = set(initialWeights, value) + + /** + * Sets the value of param [[stepSize]] (applicable only for solver "gd"). + * Default is 0.03. + * + * @group setParam + */ + @Since("2.0.0") + def setStepSize(value: Double): this.type = set(stepSize, value) + + @Since("1.5.0") override def copy(extra: ParamMap): MultilayerPerceptronClassifier = defaultCopy(extra) /** * Train a model using the given dataset and parameters. - * Developers can implement this instead of [[fit()]] to avoid dealing with schema validation + * Developers can implement this instead of `fit()` to avoid dealing with schema validation * and copying parameters into the model. * * @param dataset Training dataset * @return Fitted model */ - override protected def train(dataset: DataFrame): MultilayerPerceptronClassificationModel = { + override protected def train(dataset: Dataset[_]): MultilayerPerceptronClassificationModel = { + val instr = Instrumentation.create(this, dataset) + instr.logParams(labelCol, featuresCol, predictionCol, layers, maxIter, tol, + blockSize, solver, stepSize, seed) + val myLayers = $(layers) val labels = myLayers.last + instr.logNumClasses(labels) + instr.logNumFeatures(myLayers.head) + val lpData = extractLabeledPoints(dataset) val data = lpData.map(lp => LabelConverter.encodeLabeledPoint(lp, labels)) - val topology = FeedForwardTopology.multiLayerPerceptron(myLayers, true) - val FeedForwardTrainer = new FeedForwardTrainer(topology, myLayers(0), myLayers.last) - FeedForwardTrainer.LBFGSOptimizer.setConvergenceTol($(tol)).setNumIterations($(maxIter)) - FeedForwardTrainer.setStackSize($(blockSize)) - val mlpModel = FeedForwardTrainer.train(data) - new MultilayerPerceptronClassificationModel(uid, myLayers, mlpModel.weights()) + val topology = FeedForwardTopology.multiLayerPerceptron(myLayers, softmaxOnTop = true) + val trainer = new FeedForwardTrainer(topology, myLayers(0), myLayers.last) + if (isDefined(initialWeights)) { + trainer.setWeights($(initialWeights)) + } else { + trainer.setSeed($(seed)) + } + if ($(solver) == MultilayerPerceptronClassifier.LBFGS) { + trainer.LBFGSOptimizer + .setConvergenceTol($(tol)) + .setNumIterations($(maxIter)) + } else if ($(solver) == MultilayerPerceptronClassifier.GD) { + trainer.SGDOptimizer + .setNumIterations($(maxIter)) + .setConvergenceTol($(tol)) + .setStepSize($(stepSize)) + } else { + throw new IllegalArgumentException( + s"The solver $solver is not supported by MultilayerPerceptronClassifier.") + } + trainer.setStackSize($(blockSize)) + val mlpModel = trainer.train(data) + val model = new MultilayerPerceptronClassificationModel(uid, myLayers, mlpModel.weights) + + instr.logSuccess(model) + model } } +@Since("2.0.0") +object MultilayerPerceptronClassifier + extends DefaultParamsReadable[MultilayerPerceptronClassifier] { + + /** String name for "l-bfgs" solver. */ + private[classification] val LBFGS = "l-bfgs" + + /** String name for "gd" (minibatch gradient descent) solver. */ + private[classification] val GD = "gd" + + /** Set of solvers that MultilayerPerceptronClassifier supports. */ + private[classification] val supportedSolvers = Array(LBFGS, GD) + + @Since("2.0.0") + override def load(path: String): MultilayerPerceptronClassifier = super.load(path) +} + /** - * :: Experimental :: * Classification model based on the Multilayer Perceptron. * Each layer has sigmoid activation function, output layer has softmax. + * * @param uid uid * @param layers array of layer sizes including input and output layers - * @param weights vector of initial weights for the model that consists of the weights of layers - * @return prediction model + * @param weights the weights of layers */ -@Experimental +@Since("1.5.0") class MultilayerPerceptronClassificationModel private[ml] ( - override val uid: String, - val layers: Array[Int], - val weights: Vector) - extends PredictionModel[Vector, MultilayerPerceptronClassificationModel] - with Serializable { + @Since("1.5.0") override val uid: String, + @Since("1.5.0") val layers: Array[Int], + @Since("2.0.0") val weights: Vector) + extends ProbabilisticClassificationModel[Vector, MultilayerPerceptronClassificationModel] + with Serializable with MLWritable { + @Since("1.6.0") override val numFeatures: Int = layers.head - private val mlpModel = FeedForwardTopology.multiLayerPerceptron(layers, true).getInstance(weights) + private[ml] val mlpModel = FeedForwardTopology + .multiLayerPerceptron(layers, softmaxOnTop = true) + .model(weights) /** * Returns layers in a Java List. @@ -194,13 +320,76 @@ class MultilayerPerceptronClassificationModel private[ml] ( /** * Predict label for the given features. - * This internal method is used to implement [[transform()]] and output [[predictionCol]]. + * This internal method is used to implement `transform()` and output [[predictionCol]]. */ override protected def predict(features: Vector): Double = { LabelConverter.decodeLabel(mlpModel.predict(features)) } + @Since("1.5.0") override def copy(extra: ParamMap): MultilayerPerceptronClassificationModel = { - copyValues(new MultilayerPerceptronClassificationModel(uid, layers, weights), extra) + val copied = new MultilayerPerceptronClassificationModel(uid, layers, weights).setParent(parent) + copyValues(copied, extra) + } + + @Since("2.0.0") + override def write: MLWriter = + new MultilayerPerceptronClassificationModel.MultilayerPerceptronClassificationModelWriter(this) + + override protected def raw2probabilityInPlace(rawPrediction: Vector): Vector = { + mlpModel.raw2ProbabilityInPlace(rawPrediction) + } + + override protected def predictRaw(features: Vector): Vector = mlpModel.predictRaw(features) + + override def numClasses: Int = layers.last +} + +@Since("2.0.0") +object MultilayerPerceptronClassificationModel + extends MLReadable[MultilayerPerceptronClassificationModel] { + + @Since("2.0.0") + override def read: MLReader[MultilayerPerceptronClassificationModel] = + new MultilayerPerceptronClassificationModelReader + + @Since("2.0.0") + override def load(path: String): MultilayerPerceptronClassificationModel = super.load(path) + + /** [[MLWriter]] instance for [[MultilayerPerceptronClassificationModel]] */ + private[MultilayerPerceptronClassificationModel] + class MultilayerPerceptronClassificationModelWriter( + instance: MultilayerPerceptronClassificationModel) extends MLWriter { + + private case class Data(layers: Array[Int], weights: Vector) + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + // Save model data: layers, weights + val data = Data(instance.layers, instance.weights) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class MultilayerPerceptronClassificationModelReader + extends MLReader[MultilayerPerceptronClassificationModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[MultilayerPerceptronClassificationModel].getName + + override def load(path: String): MultilayerPerceptronClassificationModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath).select("layers", "weights").head() + val layers = data.getAs[Seq[Int]](0).toArray + val weights = data.getAs[Vector](1) + val model = new MultilayerPerceptronClassificationModel(metadata.uid, layers, weights) + + DefaultParamsReader.getAndSetParams(model, metadata) + model + } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index a14dcecbaf5b..0293e03d4743 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -17,21 +17,22 @@ package org.apache.spark.ml.classification -import org.apache.spark.SparkException -import org.apache.spark.annotation.Experimental +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since import org.apache.spark.ml.PredictorParams +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} -import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.classification.{NaiveBayes => OldNaiveBayes, NaiveBayesModel => OldNaiveBayesModel} -import org.apache.spark.mllib.linalg._ -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.spark.ml.param.shared.HasWeightCol +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.functions.{col, lit} /** * Params for Naive Bayes Classifiers. */ -private[ml] trait NaiveBayesParams extends PredictorParams { +private[classification] trait NaiveBayesParams extends PredictorParams with HasWeightCol { /** * The smoothing parameter. @@ -52,28 +53,35 @@ private[ml] trait NaiveBayesParams extends PredictorParams { */ final val modelType: Param[String] = new Param[String](this, "modelType", "The model type " + "which is a string (case-sensitive). Supported options: multinomial (default) and bernoulli.", - ParamValidators.inArray[String](OldNaiveBayes.supportedModelTypes.toArray)) + ParamValidators.inArray[String](NaiveBayes.supportedModelTypes.toArray)) /** @group getParam */ final def getModelType: String = $(modelType) } +// scalastyle:off line.size.limit /** - * :: Experimental :: * Naive Bayes Classifiers. - * It supports both Multinomial NB - * ([[http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html]]) + * It supports Multinomial NB + * (see + * here) * which can handle finitely supported discrete data. For example, by converting documents into * TF-IDF vectors, it can be used for document classification. By making every vector a * binary (0/1) data, it can also be used as Bernoulli NB - * ([[http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html]]). + * (see + * here). * The input feature values must be nonnegative. */ -@Experimental -class NaiveBayes(override val uid: String) +// scalastyle:on line.size.limit +@Since("1.5.0") +class NaiveBayes @Since("1.5.0") ( + @Since("1.5.0") override val uid: String) extends ProbabilisticClassifier[Vector, NaiveBayes, NaiveBayesModel] - with NaiveBayesParams { + with NaiveBayesParams with DefaultParamsWritable { + + import NaiveBayes._ + @Since("1.5.0") def this() = this(Identifiable.randomUID("nb")) /** @@ -81,6 +89,7 @@ class NaiveBayes(override val uid: String) * Default is 1.0. * @group setParam */ + @Since("1.5.0") def setSmoothing(value: Double): this.type = set(smoothing, value) setDefault(smoothing -> 1.0) @@ -90,33 +99,182 @@ class NaiveBayes(override val uid: String) * Default is "multinomial" * @group setParam */ + @Since("1.5.0") def setModelType(value: String): this.type = set(modelType, value) - setDefault(modelType -> OldNaiveBayes.Multinomial) + setDefault(modelType -> NaiveBayes.Multinomial) + + /** + * Sets the value of param [[weightCol]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * Default is not set, so all instances have weight one. + * + * @group setParam + */ + @Since("2.1.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + + override protected def train(dataset: Dataset[_]): NaiveBayesModel = { + trainWithLabelCheck(dataset, positiveLabel = true) + } + + /** + * ml assumes input labels in range [0, numClasses). But this implementation + * is also called by mllib NaiveBayes which allows other kinds of input labels + * such as {-1, +1}. `positiveLabel` is used to determine whether the label + * should be checked and it should be removed when we remove mllib NaiveBayes. + */ + private[spark] def trainWithLabelCheck( + dataset: Dataset[_], + positiveLabel: Boolean): NaiveBayesModel = { + if (positiveLabel && isDefined(thresholds)) { + val numClasses = getNumClasses(dataset) + require($(thresholds).length == numClasses, this.getClass.getSimpleName + + ".train() called with non-matching numClasses and thresholds.length." + + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") + } + + val modelTypeValue = $(modelType) + val requireValues: Vector => Unit = { + modelTypeValue match { + case Multinomial => + requireNonnegativeValues + case Bernoulli => + requireZeroOneBernoulliValues + case _ => + // This should never happen. + throw new UnknownError(s"Invalid modelType: ${$(modelType)}.") + } + } + + val instr = Instrumentation.create(this, dataset) + instr.logParams(labelCol, featuresCol, weightCol, predictionCol, rawPredictionCol, + probabilityCol, modelType, smoothing, thresholds) + + val numFeatures = dataset.select(col($(featuresCol))).head().getAs[Vector](0).size + instr.logNumFeatures(numFeatures) + val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) + + // Aggregates term frequencies per label. + // TODO: Calling aggregateByKey and collect creates two stages, we can implement something + // TODO: similar to reduceByKeyLocally to save one stage. + val aggregated = dataset.select(col($(labelCol)), w, col($(featuresCol))).rdd + .map { row => (row.getDouble(0), (row.getDouble(1), row.getAs[Vector](2))) + }.aggregateByKey[(Double, DenseVector)]((0.0, Vectors.zeros(numFeatures).toDense))( + seqOp = { + case ((weightSum: Double, featureSum: DenseVector), (weight, features)) => + requireValues(features) + BLAS.axpy(weight, features, featureSum) + (weightSum + weight, featureSum) + }, + combOp = { + case ((weightSum1, featureSum1), (weightSum2, featureSum2)) => + BLAS.axpy(1.0, featureSum2, featureSum1) + (weightSum1 + weightSum2, featureSum1) + }).collect().sortBy(_._1) + + val numLabels = aggregated.length + instr.logNumClasses(numLabels) + val numDocuments = aggregated.map(_._2._1).sum + + val labelArray = new Array[Double](numLabels) + val piArray = new Array[Double](numLabels) + val thetaArray = new Array[Double](numLabels * numFeatures) + + val lambda = $(smoothing) + val piLogDenom = math.log(numDocuments + numLabels * lambda) + var i = 0 + aggregated.foreach { case (label, (n, sumTermFreqs)) => + labelArray(i) = label + piArray(i) = math.log(n + lambda) - piLogDenom + val thetaLogDenom = $(modelType) match { + case Multinomial => math.log(sumTermFreqs.values.sum + numFeatures * lambda) + case Bernoulli => math.log(n + 2.0 * lambda) + case _ => + // This should never happen. + throw new UnknownError(s"Invalid modelType: ${$(modelType)}.") + } + var j = 0 + while (j < numFeatures) { + thetaArray(i * numFeatures + j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom + j += 1 + } + i += 1 + } - override protected def train(dataset: DataFrame): NaiveBayesModel = { - val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) - val oldModel = OldNaiveBayes.train(oldDataset, $(smoothing), $(modelType)) - NaiveBayesModel.fromOld(oldModel, this) + val pi = Vectors.dense(piArray) + val theta = new DenseMatrix(numLabels, numFeatures, thetaArray, true) + val model = new NaiveBayesModel(uid, pi, theta).setOldLabels(labelArray) + instr.logSuccess(model) + model } + @Since("1.5.0") override def copy(extra: ParamMap): NaiveBayes = defaultCopy(extra) } +@Since("1.6.0") +object NaiveBayes extends DefaultParamsReadable[NaiveBayes] { + /** String name for multinomial model type. */ + private[classification] val Multinomial: String = "multinomial" + + /** String name for Bernoulli model type. */ + private[classification] val Bernoulli: String = "bernoulli" + + /* Set of modelTypes that NaiveBayes supports */ + private[classification] val supportedModelTypes = Set(Multinomial, Bernoulli) + + private[NaiveBayes] def requireNonnegativeValues(v: Vector): Unit = { + val values = v match { + case sv: SparseVector => sv.values + case dv: DenseVector => dv.values + } + + require(values.forall(_ >= 0.0), + s"Naive Bayes requires nonnegative feature values but found $v.") + } + + private[NaiveBayes] def requireZeroOneBernoulliValues(v: Vector): Unit = { + val values = v match { + case sv: SparseVector => sv.values + case dv: DenseVector => dv.values + } + + require(values.forall(v => v == 0.0 || v == 1.0), + s"Bernoulli naive Bayes requires 0 or 1 feature values but found $v.") + } + + @Since("1.6.0") + override def load(path: String): NaiveBayes = super.load(path) +} + /** - * :: Experimental :: * Model produced by [[NaiveBayes]] * @param pi log of class priors, whose dimension is C (number of classes) * @param theta log of class conditional probabilities, whose dimension is C (number of classes) * by D (number of features) */ -@Experimental +@Since("1.5.0") class NaiveBayesModel private[ml] ( - override val uid: String, - val pi: Vector, - val theta: Matrix) - extends ProbabilisticClassificationModel[Vector, NaiveBayesModel] with NaiveBayesParams { + @Since("1.5.0") override val uid: String, + @Since("2.0.0") val pi: Vector, + @Since("2.0.0") val theta: Matrix) + extends ProbabilisticClassificationModel[Vector, NaiveBayesModel] + with NaiveBayesParams with MLWritable { - import OldNaiveBayes.{Bernoulli, Multinomial} + import NaiveBayes.{Bernoulli, Multinomial} + + /** + * mllib NaiveBayes is a wrapper of ml implementation currently. + * Input labels of mllib could be {-1, +1} and mllib NaiveBayesModel exposes labels, + * both of which are different from ml, so we should store the labels sequentially + * to be called by mllib. This should be removed when we remove mllib NaiveBayes. + */ + private[spark] var oldLabels: Array[Double] = null + + private[spark] def setOldLabels(labels: Array[Double]): this.type = { + this.oldLabels = labels + this + } /** * Bernoulli scoring requires log(condprob) if 1, log(1-condprob) if 0. @@ -127,7 +285,7 @@ class NaiveBayesModel private[ml] ( case Multinomial => (None, None) case Bernoulli => val negTheta = theta.map(value => math.log(1.0 - math.exp(value))) - val ones = new DenseVector(Array.fill(theta.numCols){1.0}) + val ones = new DenseVector(Array.fill(theta.numCols) {1.0}) val thetaMinusNegTheta = theta.map { value => value - math.log(1.0 - math.exp(value)) } @@ -137,8 +295,10 @@ class NaiveBayesModel private[ml] ( throw new UnknownError(s"Invalid modelType: ${$(modelType)}.") } + @Since("1.6.0") override val numFeatures: Int = theta.numCols + @Since("1.5.0") override val numClasses: Int = pi.size private def multinomialCalculation(features: Vector) = { @@ -149,10 +309,8 @@ class NaiveBayesModel private[ml] ( private def bernoulliCalculation(features: Vector) = { features.foreachActive((_, value) => - if (value != 0.0 && value != 1.0) { - throw new SparkException( - s"Bernoulli naive Bayes requires 0 or 1 feature values but found $features.") - } + require(value == 0.0 || value == 1.0, + s"Bernoulli naive Bayes requires 0 or 1 feature values but found $features.") ) val prob = thetaMinusNegTheta.get.multiply(features) BLAS.axpy(1.0, pi, prob) @@ -195,27 +353,62 @@ class NaiveBayesModel private[ml] ( } } + @Since("1.5.0") override def copy(extra: ParamMap): NaiveBayesModel = { copyValues(new NaiveBayesModel(uid, pi, theta).setParent(this.parent), extra) } + @Since("1.5.0") override def toString: String = { s"NaiveBayesModel (uid=$uid) with ${pi.size} classes" } + @Since("1.6.0") + override def write: MLWriter = new NaiveBayesModel.NaiveBayesModelWriter(this) } -private[ml] object NaiveBayesModel { - - /** Convert a model from the old API */ - def fromOld( - oldModel: OldNaiveBayesModel, - parent: NaiveBayes): NaiveBayesModel = { - val uid = if (parent != null) parent.uid else Identifiable.randomUID("nb") - val labels = Vectors.dense(oldModel.labels) - val pi = Vectors.dense(oldModel.pi) - val theta = new DenseMatrix(oldModel.labels.length, oldModel.theta(0).length, - oldModel.theta.flatten, true) - new NaiveBayesModel(uid, pi, theta) +@Since("1.6.0") +object NaiveBayesModel extends MLReadable[NaiveBayesModel] { + + @Since("1.6.0") + override def read: MLReader[NaiveBayesModel] = new NaiveBayesModelReader + + @Since("1.6.0") + override def load(path: String): NaiveBayesModel = super.load(path) + + /** [[MLWriter]] instance for [[NaiveBayesModel]] */ + private[NaiveBayesModel] class NaiveBayesModelWriter(instance: NaiveBayesModel) extends MLWriter { + + private case class Data(pi: Vector, theta: Matrix) + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + // Save model data: pi, theta + val data = Data(instance.pi, instance.theta) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class NaiveBayesModelReader extends MLReader[NaiveBayesModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[NaiveBayesModel].getName + + override def load(path: String): NaiveBayesModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + val vecConverted = MLUtils.convertVectorColumnsToML(data, "pi") + val Row(pi: Vector, theta: Matrix) = MLUtils.convertMatrixColumnsToML(vecConverted, "theta") + .select("pi", "theta") + .head() + val model = new NaiveBayesModel(metadata.uid, pi, theta) + + DefaultParamsReader.getAndSetParams(model, metadata) + model + } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index debc164bf243..3ab99b35ece2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -19,24 +19,30 @@ package org.apache.spark.ml.classification import java.util.UUID +import scala.concurrent.Future +import scala.concurrent.duration.Duration import scala.language.existentials -import org.apache.spark.annotation.Experimental +import org.apache.hadoop.fs.Path +import org.json4s.{DefaultFormats, JObject, _} +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Since import org.apache.spark.ml._ import org.apache.spark.ml.attribute._ -import org.apache.spark.ml.param.{Param, ParamMap} -import org.apache.spark.ml.util.{Identifiable, MetadataUtils} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.param.{Param, ParamMap, ParamPair, Params} +import org.apache.spark.ml.param.shared.{HasParallelism, HasWeightCol} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.ThreadUtils -/** - * Params for [[OneVsRest]]. - */ -private[ml] trait OneVsRestParams extends PredictorParams { - +private[ml] trait ClassifierTypeTrait { // scalastyle:off structural.type type ClassifierType = Classifier[F, E, M] forSome { type F @@ -44,6 +50,13 @@ private[ml] trait OneVsRestParams extends PredictorParams { type E <: Classifier[F, E, M] } // scalastyle:on structural.type +} + +/** + * Params for [[OneVsRest]]. + */ +private[ml] trait OneVsRestParams extends PredictorParams + with ClassifierTypeTrait with HasWeightCol { /** * param for the base binary classifier that we reduce multiclass classification into. @@ -57,8 +70,56 @@ private[ml] trait OneVsRestParams extends PredictorParams { def getClassifier: ClassifierType = $(classifier) } +private[ml] object OneVsRestParams extends ClassifierTypeTrait { + + def validateParams(instance: OneVsRestParams): Unit = { + def checkElement(elem: Params, name: String): Unit = elem match { + case stage: MLWritable => // good + case other => + throw new UnsupportedOperationException("OneVsRest write will fail " + + s" because it contains $name which does not implement MLWritable." + + s" Non-Writable $name: ${other.uid} of type ${other.getClass}") + } + + instance match { + case ovrModel: OneVsRestModel => ovrModel.models.foreach(checkElement(_, "model")) + case _ => // no need to check OneVsRest here + } + + checkElement(instance.getClassifier, "classifier") + } + + def saveImpl( + path: String, + instance: OneVsRestParams, + sc: SparkContext, + extraMetadata: Option[JObject] = None): Unit = { + + val params = instance.extractParamMap().toSeq + val jsonParams = render(params + .filter { case ParamPair(p, v) => p.name != "classifier" } + .map { case ParamPair(p, v) => p.name -> parse(p.jsonEncode(v)) } + .toList) + + DefaultParamsWriter.saveMetadata(instance, path, sc, extraMetadata, Some(jsonParams)) + + val classifierPath = new Path(path, "classifier").toString + instance.getClassifier.asInstanceOf[MLWritable].save(classifierPath) + } + + def loadImpl( + path: String, + sc: SparkContext, + expectedClassName: String): (DefaultParamsReader.Metadata, ClassifierType) = { + + val metadata = DefaultParamsReader.loadMetadata(path, sc, expectedClassName) + val classifierPath = new Path(path, "classifier").toString + val estimator = DefaultParamsReader.loadParamsInstance[ClassifierType](classifierPath, sc) + (metadata, estimator) + } +} + /** - * :: Experimental :: * Model produced by [[OneVsRest]]. * This stores the models resulting from training k binary classifiers: one for each class. * Each example is scored against all k models, and the model with the highest score @@ -70,18 +131,28 @@ private[ml] trait OneVsRestParams extends PredictorParams { * The i-th model is produced by testing the i-th class (taking label 1) vs the rest * (taking label 0). */ -@Experimental +@Since("1.4.0") final class OneVsRestModel private[ml] ( - override val uid: String, - labelMetadata: Metadata, - val models: Array[_ <: ClassificationModel[_, _]]) - extends Model[OneVsRestModel] with OneVsRestParams { + @Since("1.4.0") override val uid: String, + private[ml] val labelMetadata: Metadata, + @Since("1.4.0") val models: Array[_ <: ClassificationModel[_, _]]) + extends Model[OneVsRestModel] with OneVsRestParams with MLWritable { + + /** @group setParam */ + @Since("2.1.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + /** @group setParam */ + @Since("2.1.0") + def setPredictionCol(value: String): this.type = set(predictionCol, value) + + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema, fitting = false, getClassifier.featuresDataType) } - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { // Check schema transformSchema(dataset.schema, logging = true) @@ -94,7 +165,7 @@ final class OneVsRestModel private[ml] ( val newDataset = dataset.withColumn(accColName, initUDF()) // persist if underlying dataset is not persistent. - val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE + val handlePersistence = dataset.storageLevel == StorageLevel.NONE if (handlePersistence) { newDataset.persist(StorageLevel.MEMORY_AND_DISK) } @@ -110,13 +181,14 @@ final class OneVsRestModel private[ml] ( val updateUDF = udf { (predictions: Map[Int, Double], prediction: Vector) => predictions + ((index, prediction(1))) } - val transformedDataset = model.transform(df).select(columns : _*) + model.setFeaturesCol($(featuresCol)) + val transformedDataset = model.transform(df).select(columns: _*) val updatedDataset = transformedDataset .withColumn(tmpColName, updateUDF(col(accColName), col(rawPredictionCol))) val newColumns = origCols ++ List(col(tmpColName)) // switch out the intermediate column with the accumulator column - updatedDataset.select(newColumns : _*).withColumnRenamed(tmpColName, accColName) + updatedDataset.select(newColumns: _*).withColumnRenamed(tmpColName, accColName) } if (handlePersistence) { @@ -134,66 +206,168 @@ final class OneVsRestModel private[ml] ( .drop(accColName) } + @Since("1.4.1") override def copy(extra: ParamMap): OneVsRestModel = { val copied = new OneVsRestModel( uid, labelMetadata, models.map(_.copy(extra).asInstanceOf[ClassificationModel[_, _]])) copyValues(copied, extra).setParent(parent) } + + @Since("2.0.0") + override def write: MLWriter = new OneVsRestModel.OneVsRestModelWriter(this) +} + +@Since("2.0.0") +object OneVsRestModel extends MLReadable[OneVsRestModel] { + + @Since("2.0.0") + override def read: MLReader[OneVsRestModel] = new OneVsRestModelReader + + @Since("2.0.0") + override def load(path: String): OneVsRestModel = super.load(path) + + /** [[MLWriter]] instance for [[OneVsRestModel]] */ + private[OneVsRestModel] class OneVsRestModelWriter(instance: OneVsRestModel) extends MLWriter { + + OneVsRestParams.validateParams(instance) + + override protected def saveImpl(path: String): Unit = { + val extraJson = ("labelMetadata" -> instance.labelMetadata.json) ~ + ("numClasses" -> instance.models.length) + OneVsRestParams.saveImpl(path, instance, sc, Some(extraJson)) + instance.models.map(_.asInstanceOf[MLWritable]).zipWithIndex.foreach { case (model, idx) => + val modelPath = new Path(path, s"model_$idx").toString + model.save(modelPath) + } + } + } + + private class OneVsRestModelReader extends MLReader[OneVsRestModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[OneVsRestModel].getName + + override def load(path: String): OneVsRestModel = { + implicit val format = DefaultFormats + val (metadata, classifier) = OneVsRestParams.loadImpl(path, sc, className) + val labelMetadata = Metadata.fromJson((metadata.metadata \ "labelMetadata").extract[String]) + val numClasses = (metadata.metadata \ "numClasses").extract[Int] + val models = Range(0, numClasses).toArray.map { idx => + val modelPath = new Path(path, s"model_$idx").toString + DefaultParamsReader.loadParamsInstance[ClassificationModel[_, _]](modelPath, sc) + } + val ovrModel = new OneVsRestModel(metadata.uid, labelMetadata, models) + DefaultParamsReader.getAndSetParams(ovrModel, metadata) + ovrModel.set("classifier", classifier) + ovrModel + } + } } /** - * :: Experimental :: - * * Reduction of Multiclass Classification to Binary Classification. * Performs reduction using one against all strategy. * For a multiclass classification with k classes, train k models (one per class). * Each example is scored against all k models and the model with highest score * is picked to label the example. */ -@Experimental -final class OneVsRest(override val uid: String) - extends Estimator[OneVsRestModel] with OneVsRestParams { +@Since("1.4.0") +final class OneVsRest @Since("1.4.0") ( + @Since("1.4.0") override val uid: String) + extends Estimator[OneVsRestModel] with OneVsRestParams with HasParallelism with MLWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("oneVsRest")) /** @group setParam */ + @Since("1.4.0") def setClassifier(value: Classifier[_, _, _]): this.type = { set(classifier, value.asInstanceOf[ClassifierType]) } /** @group setParam */ + @Since("1.5.0") def setLabelCol(value: String): this.type = set(labelCol, value) /** @group setParam */ + @Since("1.5.0") def setFeaturesCol(value: String): this.type = set(featuresCol, value) /** @group setParam */ + @Since("1.5.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) + /** + * The implementation of parallel one vs. rest runs the classification for + * each class in a separate threads. + * + * @group expertSetParam + */ + @Since("2.3.0") + def setParallelism(value: Int): this.type = { + set(parallelism, value) + } + + /** + * Sets the value of param [[weightCol]]. + * + * This is ignored if weight is not supported by [[classifier]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * Default is not set, so all instances have weight one. + * + * @group setParam + */ + @Since("2.3.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema, fitting = true, getClassifier.featuresDataType) } - override def fit(dataset: DataFrame): OneVsRestModel = { + @Since("2.0.0") + override def fit(dataset: Dataset[_]): OneVsRestModel = { + transformSchema(dataset.schema) + + val instr = Instrumentation.create(this, dataset) + instr.logParams(labelCol, featuresCol, predictionCol, parallelism) + instr.logNamedValue("classifier", $(classifier).getClass.getCanonicalName) + // determine number of classes either from metadata if provided, or via computation. val labelSchema = dataset.schema($(labelCol)) val computeNumClasses: () => Int = () => { - val Row(maxLabelIndex: Double) = dataset.agg(max($(labelCol))).head() + val Row(maxLabelIndex: Double) = dataset.agg(max(col($(labelCol)).cast(DoubleType))).head() // classes are assumed to be numbered from 0,...,maxLabelIndex maxLabelIndex.toInt + 1 } val numClasses = MetadataUtils.getNumClasses(labelSchema).fold(computeNumClasses())(identity) + instr.logNumClasses(numClasses) + + val weightColIsUsed = isDefined(weightCol) && $(weightCol).nonEmpty && { + getClassifier match { + case _: HasWeightCol => true + case c => + logWarning(s"weightCol is ignored, as it is not supported by $c now.") + false + } + } - val multiclassLabeled = dataset.select($(labelCol), $(featuresCol)) + val multiclassLabeled = if (weightColIsUsed) { + dataset.select($(labelCol), $(featuresCol), $(weightCol)) + } else { + dataset.select($(labelCol), $(featuresCol)) + } // persist if underlying dataset is not persistent. - val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE + val handlePersistence = dataset.storageLevel == StorageLevel.NONE if (handlePersistence) { multiclassLabeled.persist(StorageLevel.MEMORY_AND_DISK) } + val executionContext = getExecutionContext + // create k columns, one for each binary classifier. - val models = Range(0, numClasses).par.map { index => + val modelFutures = Range(0, numClasses).map { index => // generate new label metadata for the binary problem. val newLabelMeta = BinaryAttribute.defaultAttr.withName("label").toMetadata() val labelColName = "mc2b$" + index @@ -204,8 +378,19 @@ final class OneVsRest(override val uid: String) paramMap.put(classifier.labelCol -> labelColName) paramMap.put(classifier.featuresCol -> getFeaturesCol) paramMap.put(classifier.predictionCol -> getPredictionCol) - classifier.fit(trainingDataset, paramMap) - }.toArray[ClassificationModel[_, _]] + Future { + if (weightColIsUsed) { + val classifier_ = classifier.asInstanceOf[ClassifierType with HasWeightCol] + paramMap.put(classifier_.weightCol -> getWeightCol) + classifier_.fit(trainingDataset, paramMap) + } else { + classifier.fit(trainingDataset, paramMap) + } + }(executionContext) + } + val models = modelFutures + .map(ThreadUtils.awaitResult(_, Duration.Inf)).toArray[ClassificationModel[_, _]] + instr.logNumFeatures(models.head.numFeatures) if (handlePersistence) { multiclassLabeled.unpersist() @@ -219,9 +404,11 @@ final class OneVsRest(override val uid: String) case attr: Attribute => attr } val model = new OneVsRestModel(uid, labelAttribute.toMetadata(), models).setParent(this) + instr.logSuccess(model) copyValues(model) } + @Since("1.4.1") override def copy(extra: ParamMap): OneVsRest = { val copied = defaultCopy(extra).asInstanceOf[OneVsRest] if (isDefined(classifier)) { @@ -229,4 +416,40 @@ final class OneVsRest(override val uid: String) } copied } + + @Since("2.0.0") + override def write: MLWriter = new OneVsRest.OneVsRestWriter(this) +} + +@Since("2.0.0") +object OneVsRest extends MLReadable[OneVsRest] { + + @Since("2.0.0") + override def read: MLReader[OneVsRest] = new OneVsRestReader + + @Since("2.0.0") + override def load(path: String): OneVsRest = super.load(path) + + /** [[MLWriter]] instance for [[OneVsRest]] */ + private[OneVsRest] class OneVsRestWriter(instance: OneVsRest) extends MLWriter { + + OneVsRestParams.validateParams(instance) + + override protected def saveImpl(path: String): Unit = { + OneVsRestParams.saveImpl(path, instance, sc) + } + } + + private class OneVsRestReader extends MLReader[OneVsRest] { + + /** Checked against metadata when loading model */ + private val className = classOf[OneVsRest].getName + + override def load(path: String): OneVsRest = { + val (metadata, classifier) = OneVsRestParams.loadImpl(path, sc, className) + val ovr = new OneVsRest(metadata.uid) + DefaultParamsReader.getAndSetParams(ovr, metadata) + ovr.setClassifier(classifier) + } + } } 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 fdd1851ae550..ef0813480991 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -18,10 +18,10 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.linalg.{DenseVector, Vector, VectorUDT} import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.SchemaUtils -import org.apache.spark.mllib.linalg.{DenseVector, Vector, VectorUDT, Vectors} -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, StructType} @@ -45,7 +45,7 @@ private[classification] trait ProbabilisticClassifierParams * * Single-label binary or multiclass classifier which can output class conditional probabilities. * - * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam FeaturesType Type of input features. E.g., `Vector` * @tparam E Concrete Estimator type * @tparam M Concrete Model type */ @@ -70,7 +70,7 @@ abstract class ProbabilisticClassifier[ * Model produced by a [[ProbabilisticClassifier]]. * Classes are indexed {0, 1, ..., numClasses - 1}. * - * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam FeaturesType Type of input features. E.g., `Vector` * @tparam M Concrete Model type */ @DeveloperApi @@ -83,19 +83,24 @@ abstract class ProbabilisticClassificationModel[ def setProbabilityCol(value: String): M = set(probabilityCol, value).asInstanceOf[M] /** @group setParam */ - def setThresholds(value: Array[Double]): M = set(thresholds, value).asInstanceOf[M] + def setThresholds(value: Array[Double]): M = { + require(value.length == numClasses, this.getClass.getSimpleName + + ".setThresholds() called with non-matching numClasses and thresholds.length." + + s" numClasses=$numClasses, but thresholds has length ${value.length}") + set(thresholds, value).asInstanceOf[M] + } /** * Transforms dataset by reading from [[featuresCol]], and appending new columns as specified by * parameters: - * - predicted labels as [[predictionCol]] of type [[Double]] - * - raw predictions (confidences) as [[rawPredictionCol]] of type [[Vector]] - * - probability of each class as [[probabilityCol]] of type [[Vector]]. + * - predicted labels as [[predictionCol]] of type `Double` + * - raw predictions (confidences) as [[rawPredictionCol]] of type `Vector` + * - probability of each class as [[probabilityCol]] of type `Vector`. * * @param dataset input dataset * @return transformed dataset */ - override def transform(dataset: DataFrame): DataFrame = { + override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) if (isDefined(thresholds)) { require($(thresholds).length == numClasses, this.getClass.getSimpleName + @@ -145,7 +150,7 @@ abstract class ProbabilisticClassificationModel[ this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + " since no output columns were set.") } - outputData + outputData.toDF } /** @@ -153,13 +158,15 @@ abstract class ProbabilisticClassificationModel[ * doing the computation in-place. * These predictions are also called class conditional probabilities. * - * This internal method is used to implement [[transform()]] and output [[probabilityCol]]. + * This internal method is used to implement `transform()` and output [[probabilityCol]]. * * @return Estimated class conditional probabilities (modified input vector) */ protected def raw2probabilityInPlace(rawPrediction: Vector): Vector - /** Non-in-place version of [[raw2probabilityInPlace()]] */ + /** + * Non-in-place version of `raw2probabilityInPlace()` + */ protected def raw2probability(rawPrediction: Vector): Vector = { val probs = rawPrediction.copy raw2probabilityInPlace(probs) @@ -177,7 +184,7 @@ abstract class ProbabilisticClassificationModel[ * Predict the probability of each class given the features. * These predictions are also called class conditional probabilities. * - * This internal method is used to implement [[transform()]] and output [[probabilityCol]]. + * This internal method is used to implement `transform()` and output [[probabilityCol]]. * * @return Estimated class conditional probabilities */ @@ -195,12 +202,24 @@ abstract class ProbabilisticClassificationModel[ if (!isDefined(thresholds)) { probability.argmax } else { - val thresholds: Array[Double] = getThresholds - val scaledProbability: Array[Double] = - probability.toArray.zip(thresholds).map { case (p, t) => - if (t == 0.0) Double.PositiveInfinity else p / t + val thresholds = getThresholds + var argMax = 0 + var max = Double.NegativeInfinity + var i = 0 + val probabilitySize = probability.size + while (i < probabilitySize) { + // Thresholds are all > 0, excepting that at most one may be 0. + // The single class whose threshold is 0, if any, will always be predicted + // ('scaled' = +Infinity). However in the case that this class also has + // 0 probability, the class will not be selected ('scaled' is NaN). + val scaled = probability(i) / thresholds(i) + if (scaled > max) { + max = scaled + argMax = i } - Vectors.dense(scaledProbability).argmax + i += 1 + } + argMax } } } @@ -210,7 +229,7 @@ private[ml] object ProbabilisticClassificationModel { /** * Normalize a vector of raw predictions to be a multinomial probability vector, in place. * - * The input raw predictions should be >= 0. + * The input raw predictions should be nonnegative. * The output vector sums to 1, unless the input vector is all-0 (in which case the output is * all-0 too). * diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index bae329692a68..ab4c23520928 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -17,122 +17,176 @@ package org.apache.spark.ml.classification -import org.apache.spark.annotation.Experimental -import org.apache.spark.ml.tree.impl.RandomForest +import org.json4s.{DefaultFormats, JObject} +import org.json4s.JsonDSL._ + +import org.apache.spark.annotation.Since +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, RandomForestParams, TreeClassifierParams, TreeEnsembleModel} -import org.apache.spark.ml.util.{Identifiable, MetadataUtils} -import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.tree._ +import org.apache.spark.ml.tree.impl.RandomForest +import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ /** - * :: Experimental :: - * [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] learning algorithm for + * Random Forest learning algorithm for * classification. * It supports both binary and multiclass labels, as well as both continuous and categorical * features. */ -@Experimental -final class RandomForestClassifier(override val uid: String) +@Since("1.4.0") +class RandomForestClassifier @Since("1.4.0") ( + @Since("1.4.0") override val uid: String) extends ProbabilisticClassifier[Vector, RandomForestClassifier, RandomForestClassificationModel] - with RandomForestParams with TreeClassifierParams { + with RandomForestClassifierParams with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("rfc")) // Override parameter setters from parent trait for Java API compatibility. // Parameters from TreeClassifierParams: - override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + /** @group setParam */ + @Since("1.4.0") + override def setMaxDepth(value: Int): this.type = set(maxDepth, value) - override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + /** @group setParam */ + @Since("1.4.0") + override def setMaxBins(value: Int): this.type = set(maxBins, value) - override def setMinInstancesPerNode(value: Int): this.type = - super.setMinInstancesPerNode(value) + /** @group setParam */ + @Since("1.4.0") + override def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) - override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + /** @group setParam */ + @Since("1.4.0") + override def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) - override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + /** @group expertSetParam */ + @Since("1.4.0") + override def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) - override def setCacheNodeIds(value: Boolean): this.type = super.setCacheNodeIds(value) + /** @group expertSetParam */ + @Since("1.4.0") + override def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) - override def setCheckpointInterval(value: Int): this.type = super.setCheckpointInterval(value) + /** + * Specifies how often to checkpoint the cached node IDs. + * E.g. 10 means that the cache will get checkpointed every 10 iterations. + * This is only used if cacheNodeIds is true and if the checkpoint directory is set in + * [[org.apache.spark.SparkContext]]. + * Must be at least 1. + * (default = 10) + * @group setParam + */ + @Since("1.4.0") + override def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) - override def setImpurity(value: String): this.type = super.setImpurity(value) + /** @group setParam */ + @Since("1.4.0") + override def setImpurity(value: String): this.type = set(impurity, value) // Parameters from TreeEnsembleParams: - override def setSubsamplingRate(value: Double): this.type = super.setSubsamplingRate(value) + /** @group setParam */ + @Since("1.4.0") + override def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) - override def setSeed(value: Long): this.type = super.setSeed(value) + /** @group setParam */ + @Since("1.4.0") + override def setSeed(value: Long): this.type = set(seed, value) // Parameters from RandomForestParams: - override def setNumTrees(value: Int): this.type = super.setNumTrees(value) + /** @group setParam */ + @Since("1.4.0") + override def setNumTrees(value: Int): this.type = set(numTrees, value) + /** @group setParam */ + @Since("1.4.0") override def setFeatureSubsetStrategy(value: String): this.type = - super.setFeatureSubsetStrategy(value) + set(featureSubsetStrategy, value) - override protected def train(dataset: DataFrame): RandomForestClassificationModel = { + override protected def train(dataset: Dataset[_]): RandomForestClassificationModel = { val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) - val numClasses: Int = MetadataUtils.getNumClasses(dataset.schema($(labelCol))) match { - case Some(n: Int) => n - case None => throw new IllegalArgumentException("RandomForestClassifier was given input" + - s" with invalid label column ${$(labelCol)}, without the number of classes" + - " specified. See StringIndexer.") - // TODO: Automatically index labels: SPARK-7126 + val numClasses: Int = getNumClasses(dataset) + + if (isDefined(thresholds)) { + require($(thresholds).length == numClasses, this.getClass.getSimpleName + + ".train() called with non-matching numClasses and thresholds.length." + + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") } - val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) + + val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset, numClasses) val strategy = super.getOldStrategy(categoricalFeatures, numClasses, OldAlgo.Classification, getOldImpurity) - val trees = - RandomForest.run(oldDataset, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed) - .map(_.asInstanceOf[DecisionTreeClassificationModel]) + + val instr = Instrumentation.create(this, oldDataset) + instr.logParams(labelCol, featuresCol, predictionCol, probabilityCol, rawPredictionCol, + impurity, numTrees, featureSubsetStrategy, maxDepth, maxBins, maxMemoryInMB, minInfoGain, + minInstancesPerNode, seed, subsamplingRate, thresholds, cacheNodeIds, checkpointInterval) + + val trees = RandomForest + .run(oldDataset, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) + .map(_.asInstanceOf[DecisionTreeClassificationModel]) + val numFeatures = oldDataset.first().features.size - new RandomForestClassificationModel(trees, numFeatures, numClasses) + val m = new RandomForestClassificationModel(uid, trees, numFeatures, numClasses) + instr.logSuccess(m) + m } + @Since("1.4.1") override def copy(extra: ParamMap): RandomForestClassifier = defaultCopy(extra) } -@Experimental -object RandomForestClassifier { +@Since("1.4.0") +object RandomForestClassifier extends DefaultParamsReadable[RandomForestClassifier] { /** Accessor for supported impurity settings: entropy, gini */ + @Since("1.4.0") final val supportedImpurities: Array[String] = TreeClassifierParams.supportedImpurities /** Accessor for supported featureSubsetStrategy settings: auto, all, onethird, sqrt, log2 */ + @Since("1.4.0") final val supportedFeatureSubsetStrategies: Array[String] = RandomForestParams.supportedFeatureSubsetStrategies + + @Since("2.0.0") + override def load(path: String): RandomForestClassifier = super.load(path) } /** - * :: Experimental :: - * [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] model for classification. + * Random Forest model for classification. * It supports both binary and multiclass labels, as well as both continuous and categorical * features. + * * @param _trees Decision trees in the ensemble. - * Warning: These have null parents. + * Warning: These have null parents. */ -@Experimental -final class RandomForestClassificationModel private[ml] ( - override val uid: String, +@Since("1.4.0") +class RandomForestClassificationModel private[ml] ( + @Since("1.5.0") override val uid: String, private val _trees: Array[DecisionTreeClassificationModel], - override val numFeatures: Int, - override val numClasses: Int) + @Since("1.6.0") override val numFeatures: Int, + @Since("1.5.0") override val numClasses: Int) extends ProbabilisticClassificationModel[Vector, RandomForestClassificationModel] - with TreeEnsembleModel with Serializable { + with RandomForestClassifierParams with TreeEnsembleModel[DecisionTreeClassificationModel] + with MLWritable with Serializable { - require(numTrees > 0, "RandomForestClassificationModel requires at least 1 tree.") + require(_trees.nonEmpty, "RandomForestClassificationModel requires at least 1 tree.") /** * Construct a random forest classification model, with all trees weighted equally. + * * @param trees Component trees */ private[ml] def this( @@ -141,15 +195,17 @@ final class RandomForestClassificationModel private[ml] ( numClasses: Int) = this(Identifiable.randomUID("rfc"), trees, numFeatures, numClasses) - override def trees: Array[DecisionTreeModel] = _trees.asInstanceOf[Array[DecisionTreeModel]] + @Since("1.4.0") + override def trees: Array[DecisionTreeClassificationModel] = _trees // Note: We may add support for weights (based on tree performance) later on. - private lazy val _treeWeights: Array[Double] = Array.fill[Double](numTrees)(1.0) + private lazy val _treeWeights: Array[Double] = Array.fill[Double](_trees.length)(1.0) + @Since("1.4.0") override def treeWeights: Array[Double] = _treeWeights - override protected def transformImpl(dataset: DataFrame): DataFrame = { - val bcastModel = dataset.sqlContext.sparkContext.broadcast(this) + override protected def transformImpl(dataset: Dataset[_]): DataFrame = { + val bcastModel = dataset.sparkSession.sparkContext.broadcast(this) val predictUDF = udf { (features: Any) => bcastModel.value.predict(features.asInstanceOf[Vector]) } @@ -186,42 +242,97 @@ final class RandomForestClassificationModel private[ml] ( } } + @Since("1.4.0") override def copy(extra: ParamMap): RandomForestClassificationModel = { copyValues(new RandomForestClassificationModel(uid, _trees, numFeatures, numClasses), extra) .setParent(parent) } + @Since("1.4.0") override def toString: String = { - s"RandomForestClassificationModel (uid=$uid) with $numTrees trees" + s"RandomForestClassificationModel (uid=$uid) with $getNumTrees trees" } /** * Estimate of the importance of each feature. * - * This generalizes the idea of "Gini" importance to other losses, - * following the explanation of Gini importance from "Random Forests" documentation - * by Leo Breiman and Adele Cutler, and following the implementation from scikit-learn. + * Each feature's importance is the average of its importance across all trees in the ensemble + * The importance vector is normalized to sum to 1. This method is suggested by Hastie et al. + * (Hastie, Tibshirani, Friedman. "The Elements of Statistical Learning, 2nd Edition." 2001.) + * and follows the implementation from scikit-learn. * - * This feature importance is calculated as follows: - * - Average over trees: - * - importance(feature j) = sum (over nodes which split on feature j) of the gain, - * where gain is scaled by the number of instances passing through node - * - Normalize importances for tree based on total number of training instances used - * to build tree. - * - Normalize feature importance vector to sum to 1. + * @see `DecisionTreeClassificationModel.featureImportances` */ - lazy val featureImportances: Vector = RandomForest.featureImportances(trees, numFeatures) + @Since("1.5.0") + lazy val featureImportances: Vector = TreeEnsembleModel.featureImportances(trees, numFeatures) /** (private[ml]) Convert to a model in the old API */ private[ml] def toOld: OldRandomForestModel = { new OldRandomForestModel(OldAlgo.Classification, _trees.map(_.toOld)) } + + @Since("2.0.0") + override def write: MLWriter = + new RandomForestClassificationModel.RandomForestClassificationModelWriter(this) } -private[ml] object RandomForestClassificationModel { +@Since("2.0.0") +object RandomForestClassificationModel extends MLReadable[RandomForestClassificationModel] { + + @Since("2.0.0") + override def read: MLReader[RandomForestClassificationModel] = + new RandomForestClassificationModelReader + + @Since("2.0.0") + override def load(path: String): RandomForestClassificationModel = super.load(path) + + private[RandomForestClassificationModel] + class RandomForestClassificationModelWriter(instance: RandomForestClassificationModel) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + // Note: numTrees is not currently used, but could be nice to store for fast querying. + val extraMetadata: JObject = Map( + "numFeatures" -> instance.numFeatures, + "numClasses" -> instance.numClasses, + "numTrees" -> instance.getNumTrees) + EnsembleModelReadWrite.saveImpl(instance, path, sparkSession, extraMetadata) + } + } + + private class RandomForestClassificationModelReader + extends MLReader[RandomForestClassificationModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[RandomForestClassificationModel].getName + private val treeClassName = classOf[DecisionTreeClassificationModel].getName + + override def load(path: String): RandomForestClassificationModel = { + implicit val format = DefaultFormats + val (metadata: Metadata, treesData: Array[(Metadata, Node)], _) = + EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName) + val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] + val numClasses = (metadata.metadata \ "numClasses").extract[Int] + val numTrees = (metadata.metadata \ "numTrees").extract[Int] + + val trees: Array[DecisionTreeClassificationModel] = treesData.map { + case (treeMetadata, root) => + val tree = + new DecisionTreeClassificationModel(treeMetadata.uid, root, numFeatures, numClasses) + DefaultParamsReader.getAndSetParams(tree, treeMetadata) + tree + } + require(numTrees == trees.length, s"RandomForestClassificationModel.load expected $numTrees" + + s" trees based on metadata but found ${trees.length} trees.") + + val model = new RandomForestClassificationModel(metadata.uid, trees, numFeatures, numClasses) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } - /** (private[ml]) Convert a model from the old API */ - def fromOld( + /** Convert a model from the old API */ + private[ml] def fromOld( oldModel: OldRandomForestModel, parent: RandomForestClassifier, categoricalFeatures: Map[Int, Int], diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala new file mode 100644 index 000000000000..4c20e6563bad --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -0,0 +1,305 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.clustering + +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.clustering.{BisectingKMeans => MLlibBisectingKMeans, BisectingKMeansModel => MLlibBisectingKMeansModel} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.functions.{col, udf} +import org.apache.spark.sql.types.{IntegerType, StructType} + + +/** + * Common params for BisectingKMeans and BisectingKMeansModel + */ +private[clustering] trait BisectingKMeansParams extends Params + with HasMaxIter with HasFeaturesCol with HasSeed with HasPredictionCol { + + /** + * The desired number of leaf clusters. Must be > 1. Default: 4. + * The actual number could be smaller if there are no divisible leaf clusters. + * @group param + */ + @Since("2.0.0") + final val k = new IntParam(this, "k", "The desired number of leaf clusters. " + + "Must be > 1.", ParamValidators.gt(1)) + + /** @group getParam */ + @Since("2.0.0") + def getK: Int = $(k) + + /** + * The minimum number of points (if greater than or equal to 1.0) or the minimum proportion + * of points (if less than 1.0) of a divisible cluster (default: 1.0). + * @group expertParam + */ + @Since("2.0.0") + final val minDivisibleClusterSize = new DoubleParam(this, "minDivisibleClusterSize", + "The minimum number of points (if >= 1.0) or the minimum proportion " + + "of points (if < 1.0) of a divisible cluster.", ParamValidators.gt(0.0)) + + /** @group expertGetParam */ + @Since("2.0.0") + def getMinDivisibleClusterSize: Double = $(minDivisibleClusterSize) + + /** + * Validates and transforms the input schema. + * @param schema input schema + * @return output schema + */ + protected def validateAndTransformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) + SchemaUtils.appendColumn(schema, $(predictionCol), IntegerType) + } +} + +/** + * Model fitted by BisectingKMeans. + * + * @param parentModel a model trained by [[org.apache.spark.mllib.clustering.BisectingKMeans]]. + */ +@Since("2.0.0") +class BisectingKMeansModel private[ml] ( + @Since("2.0.0") override val uid: String, + private val parentModel: MLlibBisectingKMeansModel + ) extends Model[BisectingKMeansModel] with BisectingKMeansParams with MLWritable { + + @Since("2.0.0") + override def copy(extra: ParamMap): BisectingKMeansModel = { + val copied = copyValues(new BisectingKMeansModel(uid, parentModel), extra) + copied.setSummary(trainingSummary).setParent(this.parent) + } + + /** @group setParam */ + @Since("2.1.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** @group setParam */ + @Since("2.1.0") + def setPredictionCol(value: String): this.type = set(predictionCol, value) + + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) + val predictUDF = udf((vector: Vector) => predict(vector)) + dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) + } + + @Since("2.0.0") + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } + + private[clustering] def predict(features: Vector): Int = parentModel.predict(features) + + @Since("2.0.0") + def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) + + /** + * Computes the sum of squared distances between the input points and their corresponding cluster + * centers. + */ + @Since("2.0.0") + def computeCost(dataset: Dataset[_]): Double = { + SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new VectorUDT) + val data = dataset.select(col($(featuresCol))).rdd.map { case Row(point: Vector) => point } + parentModel.computeCost(data.map(OldVectors.fromML)) + } + + @Since("2.0.0") + override def write: MLWriter = new BisectingKMeansModel.BisectingKMeansModelWriter(this) + + private var trainingSummary: Option[BisectingKMeansSummary] = None + + private[clustering] def setSummary(summary: Option[BisectingKMeansSummary]): this.type = { + this.trainingSummary = summary + this + } + + /** + * Return true if there exists summary of model. + */ + @Since("2.1.0") + def hasSummary: Boolean = trainingSummary.nonEmpty + + /** + * Gets summary of model on training set. An exception is + * thrown if `trainingSummary == None`. + */ + @Since("2.1.0") + def summary: BisectingKMeansSummary = trainingSummary.getOrElse { + throw new SparkException( + s"No training summary available for the ${this.getClass.getSimpleName}") + } +} + +object BisectingKMeansModel extends MLReadable[BisectingKMeansModel] { + @Since("2.0.0") + override def read: MLReader[BisectingKMeansModel] = new BisectingKMeansModelReader + + @Since("2.0.0") + override def load(path: String): BisectingKMeansModel = super.load(path) + + /** [[MLWriter]] instance for [[BisectingKMeansModel]] */ + private[BisectingKMeansModel] + class BisectingKMeansModelWriter(instance: BisectingKMeansModel) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + val dataPath = new Path(path, "data").toString + instance.parentModel.save(sc, dataPath) + } + } + + private class BisectingKMeansModelReader extends MLReader[BisectingKMeansModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[BisectingKMeansModel].getName + + override def load(path: String): BisectingKMeansModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val mllibModel = MLlibBisectingKMeansModel.load(sc, dataPath) + val model = new BisectingKMeansModel(metadata.uid, mllibModel) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } +} + +/** + * A bisecting k-means algorithm based on the paper "A comparison of document clustering techniques" + * by Steinbach, Karypis, and Kumar, with modification to fit Spark. + * The algorithm starts from a single cluster that contains all points. + * Iteratively it finds divisible clusters on the bottom level and bisects each of them using + * k-means, until there are `k` leaf clusters in total or no leaf clusters are divisible. + * The bisecting steps of clusters on the same level are grouped together to increase parallelism. + * If bisecting all divisible clusters on the bottom level would result more than `k` leaf clusters, + * larger clusters get higher priority. + * + * @see + * Steinbach, Karypis, and Kumar, A comparison of document clustering techniques, + * KDD Workshop on Text Mining, 2000. + */ +@Since("2.0.0") +class BisectingKMeans @Since("2.0.0") ( + @Since("2.0.0") override val uid: String) + extends Estimator[BisectingKMeansModel] with BisectingKMeansParams with DefaultParamsWritable { + + setDefault( + k -> 4, + maxIter -> 20, + minDivisibleClusterSize -> 1.0) + + @Since("2.0.0") + override def copy(extra: ParamMap): BisectingKMeans = defaultCopy(extra) + + @Since("2.0.0") + def this() = this(Identifiable.randomUID("bisecting-kmeans")) + + /** @group setParam */ + @Since("2.0.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** @group setParam */ + @Since("2.0.0") + def setPredictionCol(value: String): this.type = set(predictionCol, value) + + /** @group setParam */ + @Since("2.0.0") + def setK(value: Int): this.type = set(k, value) + + /** @group setParam */ + @Since("2.0.0") + def setMaxIter(value: Int): this.type = set(maxIter, value) + + /** @group setParam */ + @Since("2.0.0") + def setSeed(value: Long): this.type = set(seed, value) + + /** @group expertSetParam */ + @Since("2.0.0") + def setMinDivisibleClusterSize(value: Double): this.type = set(minDivisibleClusterSize, value) + + @Since("2.0.0") + override def fit(dataset: Dataset[_]): BisectingKMeansModel = { + transformSchema(dataset.schema, logging = true) + val rdd: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } + + val instr = Instrumentation.create(this, rdd) + instr.logParams(featuresCol, predictionCol, k, maxIter, seed, minDivisibleClusterSize) + + val bkm = new MLlibBisectingKMeans() + .setK($(k)) + .setMaxIterations($(maxIter)) + .setMinDivisibleClusterSize($(minDivisibleClusterSize)) + .setSeed($(seed)) + val parentModel = bkm.run(rdd) + val model = copyValues(new BisectingKMeansModel(uid, parentModel).setParent(this)) + val summary = new BisectingKMeansSummary( + model.transform(dataset), $(predictionCol), $(featuresCol), $(k)) + model.setSummary(Some(summary)) + instr.logSuccess(model) + model + } + + @Since("2.0.0") + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } +} + + +@Since("2.0.0") +object BisectingKMeans extends DefaultParamsReadable[BisectingKMeans] { + + @Since("2.0.0") + override def load(path: String): BisectingKMeans = super.load(path) +} + + +/** + * :: Experimental :: + * Summary of BisectingKMeans. + * + * @param predictions `DataFrame` produced by `BisectingKMeansModel.transform()`. + * @param predictionCol Name for column of predicted clusters in `predictions`. + * @param featuresCol Name for column of features in `predictions`. + * @param k Number of clusters. + */ +@Since("2.1.0") +@Experimental +class BisectingKMeansSummary private[clustering] ( + predictions: DataFrame, + predictionCol: String, + featuresCol: String, + k: Int) extends ClusteringSummary(predictions, predictionCol, featuresCol, k) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/ClusteringSummary.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/ClusteringSummary.scala new file mode 100644 index 000000000000..44e832b058b6 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/ClusteringSummary.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.clustering + +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.{DataFrame, Row} + +/** + * :: Experimental :: + * Summary of clustering algorithms. + * + * @param predictions `DataFrame` produced by model.transform(). + * @param predictionCol Name for column of predicted clusters in `predictions`. + * @param featuresCol Name for column of features in `predictions`. + * @param k Number of clusters. + */ +@Experimental +class ClusteringSummary private[clustering] ( + @transient val predictions: DataFrame, + val predictionCol: String, + val featuresCol: String, + val k: Int) extends Serializable { + + /** + * Cluster centers of the transformed data. + */ + @transient lazy val cluster: DataFrame = predictions.select(predictionCol) + + /** + * Size of (number of data points in) each cluster. + */ + lazy val clusterSizes: Array[Long] = { + val sizes = Array.fill[Long](k)(0) + cluster.groupBy(predictionCol).count().select(predictionCol, "count").collect().foreach { + case Row(cluster: Int, count: Long) => sizes(cluster) = count + } + sizes + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala new file mode 100644 index 000000000000..f19ad7a5a693 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -0,0 +1,703 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.clustering + +import breeze.linalg.{DenseVector => BDV} +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.impl.Utils.EPSILON +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.stat.distribution.MultivariateGaussian +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Matrix => OldMatrix, + Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.functions.{col, udf} +import org.apache.spark.sql.types.{IntegerType, StructType} + + +/** + * Common params for GaussianMixture and GaussianMixtureModel + */ +private[clustering] trait GaussianMixtureParams extends Params with HasMaxIter with HasFeaturesCol + with HasSeed with HasPredictionCol with HasProbabilityCol with HasTol { + + /** + * Number of independent Gaussians in the mixture model. Must be greater than 1. Default: 2. + * + * @group param + */ + @Since("2.0.0") + final val k = new IntParam(this, "k", "Number of independent Gaussians in the mixture model. " + + "Must be > 1.", ParamValidators.gt(1)) + + /** @group getParam */ + @Since("2.0.0") + def getK: Int = $(k) + + /** + * Validates and transforms the input schema. + * + * @param schema input schema + * @return output schema + */ + protected def validateAndTransformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) + val schemaWithPredictionCol = SchemaUtils.appendColumn(schema, $(predictionCol), IntegerType) + SchemaUtils.appendColumn(schemaWithPredictionCol, $(probabilityCol), new VectorUDT) + } +} + +/** + * Multivariate Gaussian Mixture Model (GMM) consisting of k Gaussians, where points + * are drawn from each Gaussian i with probability weights(i). + * + * @param weights Weight for each Gaussian distribution in the mixture. + * This is a multinomial probability distribution over the k Gaussians, + * where weights(i) is the weight for Gaussian i, and weights sum to 1. + * @param gaussians Array of `MultivariateGaussian` where gaussians(i) represents + * the Multivariate Gaussian (Normal) Distribution for Gaussian i + */ +@Since("2.0.0") +class GaussianMixtureModel private[ml] ( + @Since("2.0.0") override val uid: String, + @Since("2.0.0") val weights: Array[Double], + @Since("2.0.0") val gaussians: Array[MultivariateGaussian]) + extends Model[GaussianMixtureModel] with GaussianMixtureParams with MLWritable { + + /** @group setParam */ + @Since("2.1.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** @group setParam */ + @Since("2.1.0") + def setPredictionCol(value: String): this.type = set(predictionCol, value) + + /** @group setParam */ + @Since("2.1.0") + def setProbabilityCol(value: String): this.type = set(probabilityCol, value) + + @Since("2.0.0") + override def copy(extra: ParamMap): GaussianMixtureModel = { + val copied = copyValues(new GaussianMixtureModel(uid, weights, gaussians), extra) + copied.setSummary(trainingSummary).setParent(this.parent) + } + + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) + val predUDF = udf((vector: Vector) => predict(vector)) + val probUDF = udf((vector: Vector) => predictProbability(vector)) + dataset.withColumn($(predictionCol), predUDF(col($(featuresCol)))) + .withColumn($(probabilityCol), probUDF(col($(featuresCol)))) + } + + @Since("2.0.0") + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } + + private[clustering] def predict(features: Vector): Int = { + val r = predictProbability(features) + r.argmax + } + + private[clustering] def predictProbability(features: Vector): Vector = { + val probs: Array[Double] = + GaussianMixtureModel.computeProbabilities(features.asBreeze.toDenseVector, gaussians, weights) + Vectors.dense(probs) + } + + /** + * Retrieve Gaussian distributions as a DataFrame. + * Each row represents a Gaussian Distribution. + * Two columns are defined: mean and cov. + * Schema: + * {{{ + * root + * |-- mean: vector (nullable = true) + * |-- cov: matrix (nullable = true) + * }}} + */ + @Since("2.0.0") + def gaussiansDF: DataFrame = { + val modelGaussians = gaussians.map { gaussian => + (OldVectors.fromML(gaussian.mean), OldMatrices.fromML(gaussian.cov)) + } + SparkSession.builder().getOrCreate().createDataFrame(modelGaussians).toDF("mean", "cov") + } + + /** + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. + * + * For [[GaussianMixtureModel]], this does NOT currently save the training [[summary]]. + * An option to save [[summary]] may be added in the future. + * + */ + @Since("2.0.0") + override def write: MLWriter = new GaussianMixtureModel.GaussianMixtureModelWriter(this) + + private var trainingSummary: Option[GaussianMixtureSummary] = None + + private[clustering] def setSummary(summary: Option[GaussianMixtureSummary]): this.type = { + this.trainingSummary = summary + this + } + + /** + * Return true if there exists summary of model. + */ + @Since("2.0.0") + def hasSummary: Boolean = trainingSummary.nonEmpty + + /** + * Gets summary of model on training set. An exception is + * thrown if `trainingSummary == None`. + */ + @Since("2.0.0") + def summary: GaussianMixtureSummary = trainingSummary.getOrElse { + throw new RuntimeException( + s"No training summary available for the ${this.getClass.getSimpleName}") + } +} + +@Since("2.0.0") +object GaussianMixtureModel extends MLReadable[GaussianMixtureModel] { + + @Since("2.0.0") + override def read: MLReader[GaussianMixtureModel] = new GaussianMixtureModelReader + + @Since("2.0.0") + override def load(path: String): GaussianMixtureModel = super.load(path) + + /** [[MLWriter]] instance for [[GaussianMixtureModel]] */ + private[GaussianMixtureModel] class GaussianMixtureModelWriter( + instance: GaussianMixtureModel) extends MLWriter { + + private case class Data(weights: Array[Double], mus: Array[OldVector], sigmas: Array[OldMatrix]) + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + // Save model data: weights and gaussians + val weights = instance.weights + val gaussians = instance.gaussians + val mus = gaussians.map(g => OldVectors.fromML(g.mean)) + val sigmas = gaussians.map(c => OldMatrices.fromML(c.cov)) + val data = Data(weights, mus, sigmas) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class GaussianMixtureModelReader extends MLReader[GaussianMixtureModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[GaussianMixtureModel].getName + + override def load(path: String): GaussianMixtureModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + + val dataPath = new Path(path, "data").toString + val row = sparkSession.read.parquet(dataPath).select("weights", "mus", "sigmas").head() + val weights = row.getSeq[Double](0).toArray + val mus = row.getSeq[OldVector](1).toArray + val sigmas = row.getSeq[OldMatrix](2).toArray + require(mus.length == sigmas.length, "Length of Mu and Sigma array must match") + require(mus.length == weights.length, "Length of weight and Gaussian array must match") + + val gaussians = mus.zip(sigmas).map { + case (mu, sigma) => + new MultivariateGaussian(mu.asML, sigma.asML) + } + val model = new GaussianMixtureModel(metadata.uid, weights, gaussians) + + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } + + /** + * Compute the probability (partial assignment) for each cluster for the given data point. + * + * @param features Data point + * @param dists Gaussians for model + * @param weights Weights for each Gaussian + * @return Probability (partial assignment) for each of the k clusters + */ + private[clustering] + def computeProbabilities( + features: BDV[Double], + dists: Array[MultivariateGaussian], + weights: Array[Double]): Array[Double] = { + val p = weights.zip(dists).map { + case (weight, dist) => EPSILON + weight * dist.pdf(features) + } + val pSum = p.sum + var i = 0 + while (i < weights.length) { + p(i) /= pSum + i += 1 + } + p + } +} + +/** + * Gaussian Mixture clustering. + * + * This class performs expectation maximization for multivariate Gaussian + * Mixture Models (GMMs). A GMM represents a composite distribution of + * independent Gaussian distributions with associated "mixing" weights + * specifying each's contribution to the composite. + * + * Given a set of sample points, this class will maximize the log-likelihood + * for a mixture of k Gaussians, iterating until the log-likelihood changes by + * less than convergenceTol, or until it has reached the max number of iterations. + * While this process is generally guaranteed to converge, it is not guaranteed + * to find a global optimum. + * + * @note This algorithm is limited in its number of features since it requires storing a covariance + * matrix which has size quadratic in the number of features. Even when the number of features does + * not exceed this limit, this algorithm may perform poorly on high-dimensional data. + * This is due to high-dimensional data (a) making it difficult to cluster at all (based + * on statistical/theoretical arguments) and (b) numerical issues with Gaussian distributions. + */ +@Since("2.0.0") +class GaussianMixture @Since("2.0.0") ( + @Since("2.0.0") override val uid: String) + extends Estimator[GaussianMixtureModel] with GaussianMixtureParams with DefaultParamsWritable { + + setDefault( + k -> 2, + maxIter -> 100, + tol -> 0.01) + + @Since("2.0.0") + override def copy(extra: ParamMap): GaussianMixture = defaultCopy(extra) + + @Since("2.0.0") + def this() = this(Identifiable.randomUID("GaussianMixture")) + + /** @group setParam */ + @Since("2.0.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** @group setParam */ + @Since("2.0.0") + def setPredictionCol(value: String): this.type = set(predictionCol, value) + + /** @group setParam */ + @Since("2.0.0") + def setProbabilityCol(value: String): this.type = set(probabilityCol, value) + + /** @group setParam */ + @Since("2.0.0") + def setK(value: Int): this.type = set(k, value) + + /** @group setParam */ + @Since("2.0.0") + def setMaxIter(value: Int): this.type = set(maxIter, value) + + /** @group setParam */ + @Since("2.0.0") + def setTol(value: Double): this.type = set(tol, value) + + /** @group setParam */ + @Since("2.0.0") + def setSeed(value: Long): this.type = set(seed, value) + + /** + * Number of samples per cluster to use when initializing Gaussians. + */ + private val numSamples = 5 + + @Since("2.0.0") + override def fit(dataset: Dataset[_]): GaussianMixtureModel = { + transformSchema(dataset.schema, logging = true) + + val sc = dataset.sparkSession.sparkContext + val numClusters = $(k) + + val instances: RDD[Vector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(features: Vector) => features + }.cache() + + // Extract the number of features. + val numFeatures = instances.first().size + require(numFeatures < GaussianMixture.MAX_NUM_FEATURES, s"GaussianMixture cannot handle more " + + s"than ${GaussianMixture.MAX_NUM_FEATURES} features because the size of the covariance" + + s" matrix is quadratic in the number of features.") + + val instr = Instrumentation.create(this, instances) + instr.logParams(featuresCol, predictionCol, probabilityCol, k, maxIter, seed, tol) + instr.logNumFeatures(numFeatures) + + val shouldDistributeGaussians = GaussianMixture.shouldDistributeGaussians( + numClusters, numFeatures) + + // TODO: SPARK-15785 Support users supplied initial GMM. + val (weights, gaussians) = initRandom(instances, numClusters, numFeatures) + + var logLikelihood = Double.MinValue + var logLikelihoodPrev = 0.0 + + var iter = 0 + while (iter < $(maxIter) && math.abs(logLikelihood - logLikelihoodPrev) > $(tol)) { + + val bcWeights = instances.sparkContext.broadcast(weights) + val bcGaussians = instances.sparkContext.broadcast(gaussians) + + // aggregate the cluster contribution for all sample points + val sums = instances.treeAggregate( + new ExpectationAggregator(numFeatures, bcWeights, bcGaussians))( + seqOp = (c, v) => (c, v) match { + case (aggregator, instance) => aggregator.add(instance) + }, + combOp = (c1, c2) => (c1, c2) match { + case (aggregator1, aggregator2) => aggregator1.merge(aggregator2) + }) + + bcWeights.destroy(blocking = false) + bcGaussians.destroy(blocking = false) + + /* + Create new distributions based on the partial assignments + (often referred to as the "M" step in literature) + */ + val sumWeights = sums.weights.sum + + if (shouldDistributeGaussians) { + val numPartitions = math.min(numClusters, 1024) + val tuples = Seq.tabulate(numClusters) { i => + (sums.means(i), sums.covs(i), sums.weights(i)) + } + val (ws, gs) = sc.parallelize(tuples, numPartitions).map { case (mean, cov, weight) => + GaussianMixture.updateWeightsAndGaussians(mean, cov, weight, sumWeights) + }.collect().unzip + Array.copy(ws, 0, weights, 0, ws.length) + Array.copy(gs, 0, gaussians, 0, gs.length) + } else { + var i = 0 + while (i < numClusters) { + val (weight, gaussian) = GaussianMixture.updateWeightsAndGaussians( + sums.means(i), sums.covs(i), sums.weights(i), sumWeights) + weights(i) = weight + gaussians(i) = gaussian + i += 1 + } + } + + logLikelihoodPrev = logLikelihood // current becomes previous + logLikelihood = sums.logLikelihood // this is the freshly computed log-likelihood + iter += 1 + } + + val gaussianDists = gaussians.map { case (mean, covVec) => + val cov = GaussianMixture.unpackUpperTriangularMatrix(numFeatures, covVec.values) + new MultivariateGaussian(mean, cov) + } + + val model = copyValues(new GaussianMixtureModel(uid, weights, gaussianDists)).setParent(this) + val summary = new GaussianMixtureSummary(model.transform(dataset), + $(predictionCol), $(probabilityCol), $(featuresCol), $(k), logLikelihood) + model.setSummary(Some(summary)) + instr.logSuccess(model) + model + } + + @Since("2.0.0") + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } + + /** + * Initialize weights and corresponding gaussian distributions at random. + * + * We start with uniform weights, a random mean from the data, and diagonal covariance matrices + * using component variances derived from the samples. + * + * @param instances The training instances. + * @param numClusters The number of clusters. + * @param numFeatures The number of features of training instance. + * @return The initialized weights and corresponding gaussian distributions. Note the + * covariance matrix of multivariate gaussian distribution is symmetric and + * we only save the upper triangular part as a dense vector (column major). + */ + private def initRandom( + instances: RDD[Vector], + numClusters: Int, + numFeatures: Int): (Array[Double], Array[(DenseVector, DenseVector)]) = { + val samples = instances.takeSample(withReplacement = true, numClusters * numSamples, $(seed)) + val weights: Array[Double] = Array.fill(numClusters)(1.0 / numClusters) + val gaussians: Array[(DenseVector, DenseVector)] = Array.tabulate(numClusters) { i => + val slice = samples.view(i * numSamples, (i + 1) * numSamples) + val mean = { + val v = new DenseVector(new Array[Double](numFeatures)) + var i = 0 + while (i < numSamples) { + BLAS.axpy(1.0, slice(i), v) + i += 1 + } + BLAS.scal(1.0 / numSamples, v) + v + } + /* + Construct matrix where diagonal entries are element-wise + variance of input vectors (computes biased variance). + Since the covariance matrix of multivariate gaussian distribution is symmetric, + only the upper triangular part of the matrix (column major) will be saved as + a dense vector in order to reduce the shuffled data size. + */ + val cov = { + val ss = new DenseVector(new Array[Double](numFeatures)).asBreeze + slice.foreach(xi => ss += (xi.asBreeze - mean.asBreeze) ^:^ 2.0) + val diagVec = Vectors.fromBreeze(ss) + BLAS.scal(1.0 / numSamples, diagVec) + val covVec = new DenseVector(Array.fill[Double]( + numFeatures * (numFeatures + 1) / 2)(0.0)) + diagVec.toArray.zipWithIndex.foreach { case (v: Double, i: Int) => + covVec.values(i + i * (i + 1) / 2) = v + } + covVec + } + (mean, cov) + } + (weights, gaussians) + } +} + +@Since("2.0.0") +object GaussianMixture extends DefaultParamsReadable[GaussianMixture] { + + /** Limit number of features such that numFeatures^2^ < Int.MaxValue */ + private[clustering] val MAX_NUM_FEATURES = math.sqrt(Int.MaxValue).toInt + + @Since("2.0.0") + override def load(path: String): GaussianMixture = super.load(path) + + /** + * Heuristic to distribute the computation of the [[MultivariateGaussian]]s, approximately when + * numFeatures > 25 except for when numClusters is very small. + * + * @param numClusters Number of clusters + * @param numFeatures Number of features + */ + private[clustering] def shouldDistributeGaussians( + numClusters: Int, + numFeatures: Int): Boolean = { + ((numClusters - 1.0) / numClusters) * numFeatures > 25.0 + } + + /** + * Convert an n * (n + 1) / 2 dimension array representing the upper triangular part of a matrix + * into an n * n array representing the full symmetric matrix (column major). + * + * @param n The order of the n by n matrix. + * @param triangularValues The upper triangular part of the matrix packed in an array + * (column major). + * @return A dense matrix which represents the symmetric matrix in column major. + */ + private[clustering] def unpackUpperTriangularMatrix( + n: Int, + triangularValues: Array[Double]): DenseMatrix = { + val symmetricValues = new Array[Double](n * n) + var r = 0 + var i = 0 + while (i < n) { + var j = 0 + while (j <= i) { + symmetricValues(i * n + j) = triangularValues(r) + symmetricValues(j * n + i) = triangularValues(r) + r += 1 + j += 1 + } + i += 1 + } + new DenseMatrix(n, n, symmetricValues) + } + + /** + * Update the weight, mean and covariance of gaussian distribution. + * + * @param mean The mean of the gaussian distribution. + * @param cov The covariance matrix of the gaussian distribution. Note we only + * save the upper triangular part as a dense vector (column major). + * @param weight The weight of the gaussian distribution. + * @param sumWeights The sum of weights of all clusters. + * @return The updated weight, mean and covariance. + */ + private[clustering] def updateWeightsAndGaussians( + mean: DenseVector, + cov: DenseVector, + weight: Double, + sumWeights: Double): (Double, (DenseVector, DenseVector)) = { + BLAS.scal(1.0 / weight, mean) + BLAS.spr(-weight, mean, cov) + BLAS.scal(1.0 / weight, cov) + val newWeight = weight / sumWeights + val newGaussian = (mean, cov) + (newWeight, newGaussian) + } +} + +/** + * ExpectationAggregator computes the partial expectation results. + * + * @param numFeatures The number of features. + * @param bcWeights The broadcast weights for each Gaussian distribution in the mixture. + * @param bcGaussians The broadcast array of Multivariate Gaussian (Normal) Distribution + * in the mixture. Note only upper triangular part of the covariance + * matrix of each distribution is stored as dense vector (column major) + * in order to reduce shuffled data size. + */ +private class ExpectationAggregator( + numFeatures: Int, + bcWeights: Broadcast[Array[Double]], + bcGaussians: Broadcast[Array[(DenseVector, DenseVector)]]) extends Serializable { + + private val k: Int = bcWeights.value.length + private var totalCnt: Long = 0L + private var newLogLikelihood: Double = 0.0 + private lazy val newWeights: Array[Double] = new Array[Double](k) + private lazy val newMeans: Array[DenseVector] = Array.fill(k)( + new DenseVector(Array.fill[Double](numFeatures)(0.0))) + private lazy val newCovs: Array[DenseVector] = Array.fill(k)( + new DenseVector(Array.fill[Double](numFeatures * (numFeatures + 1) / 2)(0.0))) + + @transient private lazy val oldGaussians = { + bcGaussians.value.map { case (mean, covVec) => + val cov = GaussianMixture.unpackUpperTriangularMatrix(numFeatures, covVec.values) + new MultivariateGaussian(mean, cov) + } + } + + def count: Long = totalCnt + + def logLikelihood: Double = newLogLikelihood + + def weights: Array[Double] = newWeights + + def means: Array[DenseVector] = newMeans + + def covs: Array[DenseVector] = newCovs + + /** + * Add a new training instance to this ExpectationAggregator, update the weights, + * means and covariances for each distributions, and update the log likelihood. + * + * @param instance The instance of data point to be added. + * @return This ExpectationAggregator object. + */ + def add(instance: Vector): this.type = { + val localWeights = bcWeights.value + val localOldGaussians = oldGaussians + + val prob = new Array[Double](k) + var probSum = 0.0 + var i = 0 + while (i < k) { + val p = EPSILON + localWeights(i) * localOldGaussians(i).pdf(instance) + prob(i) = p + probSum += p + i += 1 + } + + newLogLikelihood += math.log(probSum) + val localNewWeights = newWeights + val localNewMeans = newMeans + val localNewCovs = newCovs + i = 0 + while (i < k) { + prob(i) /= probSum + localNewWeights(i) += prob(i) + BLAS.axpy(prob(i), instance, localNewMeans(i)) + BLAS.spr(prob(i), instance, localNewCovs(i)) + i += 1 + } + + totalCnt += 1 + this + } + + /** + * Merge another ExpectationAggregator, update the weights, means and covariances + * for each distributions, and update the log likelihood. + * (Note that it's in place merging; as a result, `this` object will be modified.) + * + * @param other The other ExpectationAggregator to be merged. + * @return This ExpectationAggregator object. + */ + def merge(other: ExpectationAggregator): this.type = { + if (other.count != 0) { + totalCnt += other.totalCnt + + val localThisNewWeights = this.newWeights + val localOtherNewWeights = other.newWeights + val localThisNewMeans = this.newMeans + val localOtherNewMeans = other.newMeans + val localThisNewCovs = this.newCovs + val localOtherNewCovs = other.newCovs + var i = 0 + while (i < k) { + localThisNewWeights(i) += localOtherNewWeights(i) + BLAS.axpy(1.0, localOtherNewMeans(i), localThisNewMeans(i)) + BLAS.axpy(1.0, localOtherNewCovs(i), localThisNewCovs(i)) + i += 1 + } + newLogLikelihood += other.newLogLikelihood + } + this + } +} + +/** + * :: Experimental :: + * Summary of GaussianMixture. + * + * @param predictions `DataFrame` produced by `GaussianMixtureModel.transform()`. + * @param predictionCol Name for column of predicted clusters in `predictions`. + * @param probabilityCol Name for column of predicted probability of each cluster + * in `predictions`. + * @param featuresCol Name for column of features in `predictions`. + * @param k Number of clusters. + * @param logLikelihood Total log-likelihood for this model on the given data. + */ +@Since("2.0.0") +@Experimental +class GaussianMixtureSummary private[clustering] ( + predictions: DataFrame, + predictionCol: String, + @Since("2.0.0") val probabilityCol: String, + featuresCol: String, + k: Int, + @Since("2.2.0") val logLikelihood: Double) + extends ClusteringSummary(predictions, predictionCol, featuresCol, k) { + + /** + * Probability of each cluster. + */ + @Since("2.0.0") + @transient lazy val probability: DataFrame = predictions.select(probabilityCol) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 509be6300239..f2af7fe082b4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -17,17 +17,24 @@ package org.apache.spark.ml.clustering -import org.apache.spark.annotation.{Since, Experimental} -import org.apache.spark.ml.param.{Param, Params, IntParam, ParamMap} -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.{Identifiable, SchemaUtils} +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util._ import org.apache.spark.mllib.clustering.{KMeans => MLlibKMeans, KMeansModel => MLlibKMeansModel} -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{IntegerType, StructType} -import org.apache.spark.sql.{DataFrame, Row} - +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.VersionUtils.majorVersion /** * Common params for KMeans and KMeansModel @@ -36,11 +43,14 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe with HasSeed with HasPredictionCol with HasTol { /** - * Set the number of clusters to create (k). Must be > 1. Default: 2. + * The number of clusters to create (k). Must be > 1. Note that it is possible for fewer than + * k clusters to be returned, for example, if there are fewer than k distinct points to cluster. + * Default: 2. * @group param */ @Since("1.5.0") - final val k = new IntParam(this, "k", "number of clusters to create", (x: Int) => x > 1) + final val k = new IntParam(this, "k", "The number of clusters to create. " + + "Must be > 1.", ParamValidators.gt(1)) /** @group getParam */ @Since("1.5.0") @@ -53,7 +63,8 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe * @group expertParam */ @Since("1.5.0") - final val initMode = new Param[String](this, "initMode", "initialization algorithm", + final val initMode = new Param[String](this, "initMode", "The initialization algorithm. " + + "Supported options: 'random' and 'k-means||'.", (value: String) => MLlibKMeans.validateInitMode(value)) /** @group expertGetParam */ @@ -62,12 +73,12 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe /** * Param for the number of steps for the k-means|| initialization mode. This is an advanced - * setting -- the default of 5 is almost always enough. Must be > 0. Default: 5. + * setting -- the default of 2 is almost always enough. Must be > 0. Default: 2. * @group expertParam */ @Since("1.5.0") - final val initSteps = new IntParam(this, "initSteps", "number of steps for k-means||", - (value: Int) => value > 0) + final val initSteps = new IntParam(this, "initSteps", "The number of steps for k-means|| " + + "initialization mode. Must be > 0.", ParamValidators.gt(0)) /** @group expertGetParam */ @Since("1.5.0") @@ -85,25 +96,33 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe } /** - * :: Experimental :: * Model fitted by KMeans. * * @param parentModel a model trained by spark.mllib.clustering.KMeans. */ @Since("1.5.0") -@Experimental class KMeansModel private[ml] ( @Since("1.5.0") override val uid: String, - private val parentModel: MLlibKMeansModel) extends Model[KMeansModel] with KMeansParams { + private val parentModel: MLlibKMeansModel) + extends Model[KMeansModel] with KMeansParams with MLWritable { @Since("1.5.0") override def copy(extra: ParamMap): KMeansModel = { - val copied = new KMeansModel(uid, parentModel) - copyValues(copied, extra) + val copied = copyValues(new KMeansModel(uid, parentModel), extra) + copied.setSummary(trainingSummary).setParent(this.parent) } - @Since("1.5.0") - override def transform(dataset: DataFrame): DataFrame = { + /** @group setParam */ + @Since("2.0.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** @group setParam */ + @Since("2.0.0") + def setPredictionCol(value: String): this.type = set(predictionCol, value) + + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) val predictUDF = udf((vector: Vector) => predict(vector)) dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) } @@ -115,39 +134,132 @@ class KMeansModel private[ml] ( private[clustering] def predict(features: Vector): Int = parentModel.predict(features) - @Since("1.5.0") - def clusterCenters: Array[Vector] = parentModel.clusterCenters + @Since("2.0.0") + def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) /** * Return the K-means cost (sum of squared distances of points to their nearest center) for this * model on the given data. */ // TODO: Replace the temp fix when we have proper evaluators defined for clustering. - @Since("1.6.0") - def computeCost(dataset: DataFrame): Double = { + @Since("2.0.0") + def computeCost(dataset: Dataset[_]): Double = { SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new VectorUDT) - val data = dataset.select(col($(featuresCol))).map { case Row(point: Vector) => point } + val data: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } parentModel.computeCost(data) } + + /** + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. + * + * For [[KMeansModel]], this does NOT currently save the training [[summary]]. + * An option to save [[summary]] may be added in the future. + * + */ + @Since("1.6.0") + override def write: MLWriter = new KMeansModel.KMeansModelWriter(this) + + private var trainingSummary: Option[KMeansSummary] = None + + private[clustering] def setSummary(summary: Option[KMeansSummary]): this.type = { + this.trainingSummary = summary + this + } + + /** + * Return true if there exists summary of model. + */ + @Since("2.0.0") + def hasSummary: Boolean = trainingSummary.nonEmpty + + /** + * Gets summary of model on training set. An exception is + * thrown if `trainingSummary == None`. + */ + @Since("2.0.0") + def summary: KMeansSummary = trainingSummary.getOrElse { + throw new SparkException( + s"No training summary available for the ${this.getClass.getSimpleName}") + } +} + +@Since("1.6.0") +object KMeansModel extends MLReadable[KMeansModel] { + + @Since("1.6.0") + override def read: MLReader[KMeansModel] = new KMeansModelReader + + @Since("1.6.0") + override def load(path: String): KMeansModel = super.load(path) + + /** Helper class for storing model data */ + private case class Data(clusterIdx: Int, clusterCenter: Vector) + + /** + * We store all cluster centers in a single row and use this class to store model data by + * Spark 1.6 and earlier. A model can be loaded from such older data for backward compatibility. + */ + private case class OldData(clusterCenters: Array[OldVector]) + + /** [[MLWriter]] instance for [[KMeansModel]] */ + private[KMeansModel] class KMeansModelWriter(instance: KMeansModel) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + // Save model data: cluster centers + val data: Array[Data] = instance.clusterCenters.zipWithIndex.map { case (center, idx) => + Data(idx, center) + } + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(data).repartition(1).write.parquet(dataPath) + } + } + + private class KMeansModelReader extends MLReader[KMeansModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[KMeansModel].getName + + override def load(path: String): KMeansModel = { + // Import implicits for Dataset Encoder + val sparkSession = super.sparkSession + import sparkSession.implicits._ + + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + + val clusterCenters = if (majorVersion(metadata.sparkVersion) >= 2) { + val data: Dataset[Data] = sparkSession.read.parquet(dataPath).as[Data] + data.collect().sortBy(_.clusterIdx).map(_.clusterCenter).map(OldVectors.fromML) + } else { + // Loads KMeansModel stored with the old format used by Spark 1.6 and earlier. + sparkSession.read.parquet(dataPath).as[OldData].head().clusterCenters + } + val model = new KMeansModel(metadata.uid, new MLlibKMeansModel(clusterCenters)) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } } /** - * :: Experimental :: * K-means clustering with support for k-means|| initialization proposed by Bahmani et al. * - * @see [[http://dx.doi.org/10.14778/2180912.2180915 Bahmani et al., Scalable k-means++.]] + * @see Bahmani et al., Scalable k-means++. */ @Since("1.5.0") -@Experimental class KMeans @Since("1.5.0") ( @Since("1.5.0") override val uid: String) - extends Estimator[KMeansModel] with KMeansParams { + extends Estimator[KMeansModel] with KMeansParams with DefaultParamsWritable { setDefault( k -> 2, maxIter -> 20, initMode -> MLlibKMeans.K_MEANS_PARALLEL, - initSteps -> 5, + initSteps -> 2, tol -> 1e-4) @Since("1.5.0") @@ -188,10 +300,21 @@ class KMeans @Since("1.5.0") ( @Since("1.5.0") def setSeed(value: Long): this.type = set(seed, value) - @Since("1.5.0") - override def fit(dataset: DataFrame): KMeansModel = { - val rdd = dataset.select(col($(featuresCol))).map { case Row(point: Vector) => point } + @Since("2.0.0") + override def fit(dataset: Dataset[_]): KMeansModel = { + transformSchema(dataset.schema, logging = true) + + val handlePersistence = dataset.storageLevel == StorageLevel.NONE + val instances: RDD[OldVector] = dataset.select(col($(featuresCol))).rdd.map { + case Row(point: Vector) => OldVectors.fromML(point) + } + if (handlePersistence) { + instances.persist(StorageLevel.MEMORY_AND_DISK) + } + + val instr = Instrumentation.create(this, instances) + instr.logParams(featuresCol, predictionCol, k, initMode, initSteps, maxIter, seed, tol) val algo = new MLlibKMeans() .setK($(k)) .setInitializationMode($(initMode)) @@ -199,9 +322,17 @@ class KMeans @Since("1.5.0") ( .setMaxIterations($(maxIter)) .setSeed($(seed)) .setEpsilon($(tol)) - val parentModel = algo.run(rdd) - val model = new KMeansModel(uid, parentModel) - copyValues(model) + val parentModel = algo.run(instances, Option(instr)) + val model = copyValues(new KMeansModel(uid, parentModel).setParent(this)) + val summary = new KMeansSummary( + model.transform(dataset), $(predictionCol), $(featuresCol), $(k)) + + model.setSummary(Some(summary)) + instr.logSuccess(model) + if (handlePersistence) { + instances.unpersist() + } + model } @Since("1.5.0") @@ -210,3 +341,26 @@ class KMeans @Since("1.5.0") ( } } +@Since("1.6.0") +object KMeans extends DefaultParamsReadable[KMeans] { + + @Since("1.6.0") + override def load(path: String): KMeans = super.load(path) +} + +/** + * :: Experimental :: + * Summary of KMeans. + * + * @param predictions `DataFrame` produced by `KMeansModel.transform()`. + * @param predictionCol Name for column of predicted clusters in `predictions`. + * @param featuresCol Name for column of features in `predictions`. + * @param k Number of clusters. + */ +@Since("2.0.0") +@Experimental +class KMeansSummary private[clustering] ( + predictions: DataFrame, + predictionCol: String, + featuresCol: String, + k: Int) extends ClusteringSummary(predictions, predictionCol, featuresCol, k) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala new file mode 100644 index 000000000000..3da29b1c816b --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -0,0 +1,964 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.clustering + +import java.util.Locale + +import org.apache.hadoop.fs.Path +import org.json4s.DefaultFormats +import org.json4s.JsonAST.JObject +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Matrix, Vector, Vectors, VectorUDT} +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared.{HasCheckpointInterval, HasFeaturesCol, HasMaxIter, HasSeed} +import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.DefaultParamsReader.Metadata +import org.apache.spark.mllib.clustering.{DistributedLDAModel => OldDistributedLDAModel, + EMLDAOptimizer => OldEMLDAOptimizer, LDA => OldLDA, LDAModel => OldLDAModel, + LDAOptimizer => OldLDAOptimizer, LocalLDAModel => OldLocalLDAModel, + OnlineLDAOptimizer => OldOnlineLDAOptimizer} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.MatrixImplicits._ +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.functions.{col, monotonically_increasing_id, udf} +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.PeriodicCheckpointer +import org.apache.spark.util.VersionUtils + +private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasMaxIter + with HasSeed with HasCheckpointInterval { + + /** + * Param for the number of topics (clusters) to infer. Must be > 1. Default: 10. + * + * @group param + */ + @Since("1.6.0") + final val k = new IntParam(this, "k", "The number of topics (clusters) to infer. " + + "Must be > 1.", ParamValidators.gt(1)) + + /** @group getParam */ + @Since("1.6.0") + def getK: Int = $(k) + + /** + * Concentration parameter (commonly named "alpha") for the prior placed on documents' + * distributions over topics ("theta"). + * + * This is the parameter to a Dirichlet distribution, where larger values mean more smoothing + * (more regularization). + * + * If not set by the user, then docConcentration is set automatically. If set to + * singleton vector [alpha], then alpha is replicated to a vector of length k in fitting. + * Otherwise, the [[docConcentration]] vector must be length k. + * (default = automatic) + * + * Optimizer-specific parameter settings: + * - EM + * - Currently only supports symmetric distributions, so all values in the vector should be + * the same. + * - Values should be greater than 1.0 + * - default = uniformly (50 / k) + 1, where 50/k is common in LDA libraries and +1 follows + * from Asuncion et al. (2009), who recommend a +1 adjustment for EM. + * - Online + * - Values should be greater than or equal to 0 + * - default = uniformly (1.0 / k), following the implementation from + * here. + * + * @group param + */ + @Since("1.6.0") + final val docConcentration = new DoubleArrayParam(this, "docConcentration", + "Concentration parameter (commonly named \"alpha\") for the prior placed on documents'" + + " distributions over topics (\"theta\").", (alpha: Array[Double]) => alpha.forall(_ >= 0.0)) + + /** @group getParam */ + @Since("1.6.0") + def getDocConcentration: Array[Double] = $(docConcentration) + + /** Get docConcentration used by spark.mllib LDA */ + protected def getOldDocConcentration: Vector = { + if (isSet(docConcentration)) { + Vectors.dense(getDocConcentration) + } else { + Vectors.dense(-1.0) + } + } + + /** + * Concentration parameter (commonly named "beta" or "eta") for the prior placed on topics' + * distributions over terms. + * + * This is the parameter to a symmetric Dirichlet distribution. + * + * Note: The topics' distributions over terms are called "beta" in the original LDA paper + * by Blei et al., but are called "phi" in many later papers such as Asuncion et al., 2009. + * + * If not set by the user, then topicConcentration is set automatically. + * (default = automatic) + * + * Optimizer-specific parameter settings: + * - EM + * - Value should be greater than 1.0 + * - default = 0.1 + 1, where 0.1 gives a small amount of smoothing and +1 follows + * Asuncion et al. (2009), who recommend a +1 adjustment for EM. + * - Online + * - Value should be greater than or equal to 0 + * - default = (1.0 / k), following the implementation from + * here. + * + * @group param + */ + @Since("1.6.0") + final val topicConcentration = new DoubleParam(this, "topicConcentration", + "Concentration parameter (commonly named \"beta\" or \"eta\") for the prior placed on topic'" + + " distributions over terms.", ParamValidators.gtEq(0)) + + /** @group getParam */ + @Since("1.6.0") + def getTopicConcentration: Double = $(topicConcentration) + + /** Get topicConcentration used by spark.mllib LDA */ + protected def getOldTopicConcentration: Double = { + if (isSet(topicConcentration)) { + getTopicConcentration + } else { + -1.0 + } + } + + /** Supported values for Param [[optimizer]]. */ + @Since("1.6.0") + final val supportedOptimizers: Array[String] = Array("online", "em") + + /** + * Optimizer or inference algorithm used to estimate the LDA model. + * Currently supported (case-insensitive): + * - "online": Online Variational Bayes (default) + * - "em": Expectation-Maximization + * + * For details, see the following papers: + * - Online LDA: + * Hoffman, Blei and Bach. "Online Learning for Latent Dirichlet Allocation." + * Neural Information Processing Systems, 2010. + * See here + * - EM: + * Asuncion et al. "On Smoothing and Inference for Topic Models." + * Uncertainty in Artificial Intelligence, 2009. + * See here + * + * @group param + */ + @Since("1.6.0") + final val optimizer = new Param[String](this, "optimizer", "Optimizer or inference" + + " algorithm used to estimate the LDA model. Supported: " + supportedOptimizers.mkString(", "), + (value: String) => supportedOptimizers.contains(value.toLowerCase(Locale.ROOT))) + + /** @group getParam */ + @Since("1.6.0") + def getOptimizer: String = $(optimizer) + + /** + * Output column with estimates of the topic mixture distribution for each document (often called + * "theta" in the literature). Returns a vector of zeros for an empty document. + * + * This uses a variational approximation following Hoffman et al. (2010), where the approximate + * distribution is called "gamma." Technically, this method returns this approximation "gamma" + * for each document. + * + * @group param + */ + @Since("1.6.0") + final val topicDistributionCol = new Param[String](this, "topicDistributionCol", "Output column" + + " with estimates of the topic mixture distribution for each document (often called \"theta\"" + + " in the literature). Returns a vector of zeros for an empty document.") + + setDefault(topicDistributionCol -> "topicDistribution") + + /** @group getParam */ + @Since("1.6.0") + def getTopicDistributionCol: String = $(topicDistributionCol) + + /** + * For Online optimizer only: [[optimizer]] = "online". + * + * A (positive) learning parameter that downweights early iterations. Larger values make early + * iterations count less. + * This is called "tau0" in the Online LDA paper (Hoffman et al., 2010) + * Default: 1024, following Hoffman et al. + * + * @group expertParam + */ + @Since("1.6.0") + final val learningOffset = new DoubleParam(this, "learningOffset", "(For online optimizer)" + + " A (positive) learning parameter that downweights early iterations. Larger values make early" + + " iterations count less.", + ParamValidators.gt(0)) + + /** @group expertGetParam */ + @Since("1.6.0") + def getLearningOffset: Double = $(learningOffset) + + /** + * For Online optimizer only: [[optimizer]] = "online". + * + * Learning rate, set as an exponential decay rate. + * This should be between (0.5, 1.0] to guarantee asymptotic convergence. + * This is called "kappa" in the Online LDA paper (Hoffman et al., 2010). + * Default: 0.51, based on Hoffman et al. + * + * @group expertParam + */ + @Since("1.6.0") + final val learningDecay = new DoubleParam(this, "learningDecay", "(For online optimizer)" + + " Learning rate, set as an exponential decay rate. This should be between (0.5, 1.0] to" + + " guarantee asymptotic convergence.", ParamValidators.gt(0)) + + /** @group expertGetParam */ + @Since("1.6.0") + def getLearningDecay: Double = $(learningDecay) + + /** + * For Online optimizer only: [[optimizer]] = "online". + * + * Fraction of the corpus to be sampled and used in each iteration of mini-batch gradient descent, + * in range (0, 1]. + * + * Note that this should be adjusted in synch with `LDA.maxIter` + * so the entire corpus is used. Specifically, set both so that + * maxIterations * miniBatchFraction greater than or equal to 1. + * + * Note: This is the same as the `miniBatchFraction` parameter in + * [[org.apache.spark.mllib.clustering.OnlineLDAOptimizer]]. + * + * Default: 0.05, i.e., 5% of total documents. + * + * @group param + */ + @Since("1.6.0") + final val subsamplingRate = new DoubleParam(this, "subsamplingRate", "(For online optimizer)" + + " Fraction of the corpus to be sampled and used in each iteration of mini-batch" + + " gradient descent, in range (0, 1].", + ParamValidators.inRange(0.0, 1.0, lowerInclusive = false, upperInclusive = true)) + + /** @group getParam */ + @Since("1.6.0") + def getSubsamplingRate: Double = $(subsamplingRate) + + /** + * For Online optimizer only (currently): [[optimizer]] = "online". + * + * Indicates whether the docConcentration (Dirichlet parameter for + * document-topic distribution) will be optimized during training. + * Setting this to true will make the model more expressive and fit the training data better. + * Default: false + * + * @group expertParam + */ + @Since("1.6.0") + final val optimizeDocConcentration = new BooleanParam(this, "optimizeDocConcentration", + "(For online optimizer only, currently) Indicates whether the docConcentration" + + " (Dirichlet parameter for document-topic distribution) will be optimized during training.") + + /** @group expertGetParam */ + @Since("1.6.0") + def getOptimizeDocConcentration: Boolean = $(optimizeDocConcentration) + + /** + * For EM optimizer only: [[optimizer]] = "em". + * + * If using checkpointing, this indicates whether to keep the last + * checkpoint. If false, then the checkpoint will be deleted. Deleting the checkpoint can + * cause failures if a data partition is lost, so set this bit with care. + * Note that checkpoints will be cleaned up via reference counting, regardless. + * + * See `DistributedLDAModel.getCheckpointFiles` for getting remaining checkpoints and + * `DistributedLDAModel.deleteCheckpointFiles` for removing remaining checkpoints. + * + * Default: true + * + * @group expertParam + */ + @Since("2.0.0") + final val keepLastCheckpoint = new BooleanParam(this, "keepLastCheckpoint", + "(For EM optimizer) If using checkpointing, this indicates whether to keep the last" + + " checkpoint. If false, then the checkpoint will be deleted. Deleting the checkpoint can" + + " cause failures if a data partition is lost, so set this bit with care.") + + /** @group expertGetParam */ + @Since("2.0.0") + def getKeepLastCheckpoint: Boolean = $(keepLastCheckpoint) + + /** + * Validates and transforms the input schema. + * + * @param schema input schema + * @return output schema + */ + protected def validateAndTransformSchema(schema: StructType): StructType = { + if (isSet(docConcentration)) { + if (getDocConcentration.length != 1) { + require(getDocConcentration.length == getK, s"LDA docConcentration was of length" + + s" ${getDocConcentration.length}, but k = $getK. docConcentration must be an array of" + + s" length either 1 (scalar) or k (num topics).") + } + getOptimizer.toLowerCase(Locale.ROOT) match { + case "online" => + require(getDocConcentration.forall(_ >= 0), + "For Online LDA optimizer, docConcentration values must be >= 0. Found values: " + + getDocConcentration.mkString(",")) + case "em" => + require(getDocConcentration.forall(_ >= 0), + "For EM optimizer, docConcentration values must be >= 1. Found values: " + + getDocConcentration.mkString(",")) + } + } + if (isSet(topicConcentration)) { + getOptimizer.toLowerCase(Locale.ROOT) match { + case "online" => + require(getTopicConcentration >= 0, s"For Online LDA optimizer, topicConcentration" + + s" must be >= 0. Found value: $getTopicConcentration") + case "em" => + require(getTopicConcentration >= 0, s"For EM optimizer, topicConcentration" + + s" must be >= 1. Found value: $getTopicConcentration") + } + } + SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) + SchemaUtils.appendColumn(schema, $(topicDistributionCol), new VectorUDT) + } + + private[clustering] def getOldOptimizer: OldLDAOptimizer = + getOptimizer.toLowerCase(Locale.ROOT) match { + case "online" => + new OldOnlineLDAOptimizer() + .setTau0($(learningOffset)) + .setKappa($(learningDecay)) + .setMiniBatchFraction($(subsamplingRate)) + .setOptimizeDocConcentration($(optimizeDocConcentration)) + case "em" => + new OldEMLDAOptimizer() + .setKeepLastCheckpoint($(keepLastCheckpoint)) + } +} + +private object LDAParams { + + /** + * Equivalent to [[DefaultParamsReader.getAndSetParams()]], but handles [[LDA]] and [[LDAModel]] + * formats saved with Spark 1.6, which differ from the formats in Spark 2.0+. + * + * @param model [[LDA]] or [[LDAModel]] instance. This instance will be modified with + * [[Param]] values extracted from metadata. + * @param metadata Loaded model metadata + */ + def getAndSetParams(model: LDAParams, metadata: Metadata): Unit = { + VersionUtils.majorMinorVersion(metadata.sparkVersion) match { + case (1, 6) => + implicit val format = DefaultFormats + metadata.params match { + case JObject(pairs) => + pairs.foreach { case (paramName, jsonValue) => + val origParam = + if (paramName == "topicDistribution") "topicDistributionCol" else paramName + val param = model.getParam(origParam) + val value = param.jsonDecode(compact(render(jsonValue))) + model.set(param, value) + } + case _ => + throw new IllegalArgumentException( + s"Cannot recognize JSON metadata: ${metadata.metadataJson}.") + } + case _ => // 2.0+ + DefaultParamsReader.getAndSetParams(model, metadata) + } + } +} + + +/** + * Model fitted by [[LDA]]. + * + * @param vocabSize Vocabulary size (number of terms or words in the vocabulary) + * @param sparkSession Used to construct local DataFrames for returning query results + */ +@Since("1.6.0") +abstract class LDAModel private[ml] ( + @Since("1.6.0") override val uid: String, + @Since("1.6.0") val vocabSize: Int, + @Since("1.6.0") @transient private[ml] val sparkSession: SparkSession) + extends Model[LDAModel] with LDAParams with Logging with MLWritable { + + // NOTE to developers: + // This abstraction should contain all important functionality for basic LDA usage. + // Specializations of this class can contain expert-only functionality. + + /** + * Underlying spark.mllib model. + * If this model was produced by Online LDA, then this is the only model representation. + * If this model was produced by EM, then this local representation may be built lazily. + */ + @Since("1.6.0") + private[clustering] def oldLocalModel: OldLocalLDAModel + + /** Returns underlying spark.mllib model, which may be local or distributed */ + @Since("1.6.0") + private[clustering] def getModel: OldLDAModel + + private[ml] def getEffectiveDocConcentration: Array[Double] = getModel.docConcentration.toArray + + private[ml] def getEffectiveTopicConcentration: Double = getModel.topicConcentration + + /** + * The features for LDA should be a `Vector` representing the word counts in a document. + * The vector should be of length vocabSize, with counts for each term (word). + * + * @group setParam + */ + @Since("1.6.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + @Since("2.2.0") + def setTopicDistributionCol(value: String): this.type = set(topicDistributionCol, value) + + /** @group setParam */ + @Since("1.6.0") + def setSeed(value: Long): this.type = set(seed, value) + + /** + * Transforms the input dataset. + * + * WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when [[optimizer]] + * is set to "em"), this involves collecting a large [[topicsMatrix]] to the driver. + * This implementation may be changed in the future. + */ + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + if ($(topicDistributionCol).nonEmpty) { + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer = oldLocalModel.getTopicDistributionMethod(sparkSession.sparkContext) + + val t = udf { (v: Vector) => transformer(OldVectors.fromML(v)).asML } + dataset.withColumn($(topicDistributionCol), t(col($(featuresCol)))).toDF() + } else { + logWarning("LDAModel.transform was called without any output columns. Set an output column" + + " such as topicDistributionCol to produce results.") + dataset.toDF() + } + } + + @Since("1.6.0") + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } + + /** + * Value for [[docConcentration]] estimated from data. + * If Online LDA was used and [[optimizeDocConcentration]] was set to false, + * then this returns the fixed (given) value for the [[docConcentration]] parameter. + */ + @Since("2.0.0") + def estimatedDocConcentration: Vector = getModel.docConcentration + + /** + * Inferred topics, where each topic is represented by a distribution over terms. + * This is a matrix of size vocabSize x k, where each column is a topic. + * No guarantees are given about the ordering of the topics. + * + * WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by + * the Expectation-Maximization ("em") [[optimizer]], then this method could involve + * collecting a large amount of data to the driver (on the order of vocabSize x k). + */ + @Since("2.0.0") + def topicsMatrix: Matrix = oldLocalModel.topicsMatrix.asML + + /** Indicates whether this instance is of type [[DistributedLDAModel]] */ + @Since("1.6.0") + def isDistributed: Boolean + + /** + * Calculates a lower bound on the log likelihood of the entire corpus. + * + * See Equation (16) in the Online LDA paper (Hoffman et al., 2010). + * + * WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when [[optimizer]] + * is set to "em"), this involves collecting a large [[topicsMatrix]] to the driver. + * This implementation may be changed in the future. + * + * @param dataset test corpus to use for calculating log likelihood + * @return variational lower bound on the log likelihood of the entire corpus + */ + @Since("2.0.0") + def logLikelihood(dataset: Dataset[_]): Double = { + val oldDataset = LDA.getOldDataset(dataset, $(featuresCol)) + oldLocalModel.logLikelihood(oldDataset) + } + + /** + * Calculate an upper bound on perplexity. (Lower is better.) + * See Equation (16) in the Online LDA paper (Hoffman et al., 2010). + * + * WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when [[optimizer]] + * is set to "em"), this involves collecting a large [[topicsMatrix]] to the driver. + * This implementation may be changed in the future. + * + * @param dataset test corpus to use for calculating perplexity + * @return Variational upper bound on log perplexity per token. + */ + @Since("2.0.0") + def logPerplexity(dataset: Dataset[_]): Double = { + val oldDataset = LDA.getOldDataset(dataset, $(featuresCol)) + oldLocalModel.logPerplexity(oldDataset) + } + + /** + * Return the topics described by their top-weighted terms. + * + * @param maxTermsPerTopic Maximum number of terms to collect for each topic. + * Default value of 10. + * @return Local DataFrame with one topic per Row, with columns: + * - "topic": IntegerType: topic index + * - "termIndices": ArrayType(IntegerType): term indices, sorted in order of decreasing + * term importance + * - "termWeights": ArrayType(DoubleType): corresponding sorted term weights + */ + @Since("1.6.0") + def describeTopics(maxTermsPerTopic: Int): DataFrame = { + val topics = getModel.describeTopics(maxTermsPerTopic).zipWithIndex.map { + case ((termIndices, termWeights), topic) => + (topic, termIndices.toSeq, termWeights.toSeq) + } + sparkSession.createDataFrame(topics).toDF("topic", "termIndices", "termWeights") + } + + @Since("1.6.0") + def describeTopics(): DataFrame = describeTopics(10) +} + + +/** + * + * Local (non-distributed) model fitted by [[LDA]]. + * + * This model stores the inferred topics only; it does not store info about the training dataset. + */ +@Since("1.6.0") +class LocalLDAModel private[ml] ( + uid: String, + vocabSize: Int, + @Since("1.6.0") override private[clustering] val oldLocalModel: OldLocalLDAModel, + sparkSession: SparkSession) + extends LDAModel(uid, vocabSize, sparkSession) { + + @Since("1.6.0") + override def copy(extra: ParamMap): LocalLDAModel = { + val copied = new LocalLDAModel(uid, vocabSize, oldLocalModel, sparkSession) + copyValues(copied, extra).setParent(parent).asInstanceOf[LocalLDAModel] + } + + override private[clustering] def getModel: OldLDAModel = oldLocalModel + + @Since("1.6.0") + override def isDistributed: Boolean = false + + @Since("1.6.0") + override def write: MLWriter = new LocalLDAModel.LocalLDAModelWriter(this) +} + + +@Since("1.6.0") +object LocalLDAModel extends MLReadable[LocalLDAModel] { + + private[LocalLDAModel] + class LocalLDAModelWriter(instance: LocalLDAModel) extends MLWriter { + + private case class Data( + vocabSize: Int, + topicsMatrix: Matrix, + docConcentration: Vector, + topicConcentration: Double, + gammaShape: Double) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val oldModel = instance.oldLocalModel + val data = Data(instance.vocabSize, oldModel.topicsMatrix, oldModel.docConcentration, + oldModel.topicConcentration, oldModel.gammaShape) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class LocalLDAModelReader extends MLReader[LocalLDAModel] { + + private val className = classOf[LocalLDAModel].getName + + override def load(path: String): LocalLDAModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + val vectorConverted = MLUtils.convertVectorColumnsToML(data, "docConcentration") + val matrixConverted = MLUtils.convertMatrixColumnsToML(vectorConverted, "topicsMatrix") + val Row(vocabSize: Int, topicsMatrix: Matrix, docConcentration: Vector, + topicConcentration: Double, gammaShape: Double) = + matrixConverted.select("vocabSize", "topicsMatrix", "docConcentration", + "topicConcentration", "gammaShape").head() + val oldModel = new OldLocalLDAModel(topicsMatrix, docConcentration, topicConcentration, + gammaShape) + val model = new LocalLDAModel(metadata.uid, vocabSize, oldModel, sparkSession) + LDAParams.getAndSetParams(model, metadata) + model + } + } + + @Since("1.6.0") + override def read: MLReader[LocalLDAModel] = new LocalLDAModelReader + + @Since("1.6.0") + override def load(path: String): LocalLDAModel = super.load(path) +} + + +/** + * + * Distributed model fitted by [[LDA]]. + * This type of model is currently only produced by Expectation-Maximization (EM). + * + * This model stores the inferred topics, the full training dataset, and the topic distribution + * for each training document. + * + * @param oldLocalModelOption Used to implement [[oldLocalModel]] as a lazy val, but keeping + * `copy()` cheap. + */ +@Since("1.6.0") +class DistributedLDAModel private[ml] ( + uid: String, + vocabSize: Int, + private val oldDistributedModel: OldDistributedLDAModel, + sparkSession: SparkSession, + private var oldLocalModelOption: Option[OldLocalLDAModel]) + extends LDAModel(uid, vocabSize, sparkSession) { + + override private[clustering] def oldLocalModel: OldLocalLDAModel = { + if (oldLocalModelOption.isEmpty) { + oldLocalModelOption = Some(oldDistributedModel.toLocal) + } + oldLocalModelOption.get + } + + override private[clustering] def getModel: OldLDAModel = oldDistributedModel + + /** + * Convert this distributed model to a local representation. This discards info about the + * training dataset. + * + * WARNING: This involves collecting a large [[topicsMatrix]] to the driver. + */ + @Since("1.6.0") + def toLocal: LocalLDAModel = new LocalLDAModel(uid, vocabSize, oldLocalModel, sparkSession) + + @Since("1.6.0") + override def copy(extra: ParamMap): DistributedLDAModel = { + val copied = new DistributedLDAModel( + uid, vocabSize, oldDistributedModel, sparkSession, oldLocalModelOption) + copyValues(copied, extra).setParent(parent) + copied + } + + @Since("1.6.0") + override def isDistributed: Boolean = true + + /** + * Log likelihood of the observed tokens in the training set, + * given the current parameter estimates: + * log P(docs | topics, topic distributions for docs, Dirichlet hyperparameters) + * + * Notes: + * - This excludes the prior; for that, use [[logPrior]]. + * - Even with [[logPrior]], this is NOT the same as the data log likelihood given the + * hyperparameters. + * - This is computed from the topic distributions computed during training. If you call + * `logLikelihood()` on the same training dataset, the topic distributions will be computed + * again, possibly giving different results. + */ + @Since("1.6.0") + lazy val trainingLogLikelihood: Double = oldDistributedModel.logLikelihood + + /** + * Log probability of the current parameter estimate: + * log P(topics, topic distributions for docs | Dirichlet hyperparameters) + */ + @Since("1.6.0") + lazy val logPrior: Double = oldDistributedModel.logPrior + + private var _checkpointFiles: Array[String] = oldDistributedModel.checkpointFiles + + /** + * :: DeveloperApi :: + * + * If using checkpointing and `LDA.keepLastCheckpoint` is set to true, then there may be + * saved checkpoint files. This method is provided so that users can manage those files. + * + * Note that removing the checkpoints can cause failures if a partition is lost and is needed + * by certain [[DistributedLDAModel]] methods. Reference counting will clean up the checkpoints + * when this model and derivative data go out of scope. + * + * @return Checkpoint files from training + */ + @DeveloperApi + @Since("2.0.0") + def getCheckpointFiles: Array[String] = _checkpointFiles + + /** + * :: DeveloperApi :: + * + * Remove any remaining checkpoint files from training. + * + * @see [[getCheckpointFiles]] + */ + @DeveloperApi + @Since("2.0.0") + def deleteCheckpointFiles(): Unit = { + val hadoopConf = sparkSession.sparkContext.hadoopConfiguration + _checkpointFiles.foreach(PeriodicCheckpointer.removeCheckpointFile(_, hadoopConf)) + _checkpointFiles = Array.empty[String] + } + + @Since("1.6.0") + override def write: MLWriter = new DistributedLDAModel.DistributedWriter(this) +} + + +@Since("1.6.0") +object DistributedLDAModel extends MLReadable[DistributedLDAModel] { + + private[DistributedLDAModel] + class DistributedWriter(instance: DistributedLDAModel) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val modelPath = new Path(path, "oldModel").toString + instance.oldDistributedModel.save(sc, modelPath) + } + } + + private class DistributedLDAModelReader extends MLReader[DistributedLDAModel] { + + private val className = classOf[DistributedLDAModel].getName + + override def load(path: String): DistributedLDAModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val modelPath = new Path(path, "oldModel").toString + val oldModel = OldDistributedLDAModel.load(sc, modelPath) + val model = new DistributedLDAModel(metadata.uid, oldModel.vocabSize, + oldModel, sparkSession, None) + LDAParams.getAndSetParams(model, metadata) + model + } + } + + @Since("1.6.0") + override def read: MLReader[DistributedLDAModel] = new DistributedLDAModelReader + + @Since("1.6.0") + override def load(path: String): DistributedLDAModel = super.load(path) +} + + +/** + * + * Latent Dirichlet Allocation (LDA), a topic model designed for text documents. + * + * Terminology: + * - "term" = "word": an element of the vocabulary + * - "token": instance of a term appearing in a document + * - "topic": multinomial distribution over terms representing some concept + * - "document": one piece of text, corresponding to one row in the input data + * + * Original LDA paper (journal version): + * Blei, Ng, and Jordan. "Latent Dirichlet Allocation." JMLR, 2003. + * + * Input data (featuresCol): + * LDA is given a collection of documents as input data, via the featuresCol parameter. + * Each document is specified as a `Vector` of length vocabSize, where each entry is the + * count for the corresponding term (word) in the document. Feature transformers such as + * [[org.apache.spark.ml.feature.Tokenizer]] and [[org.apache.spark.ml.feature.CountVectorizer]] + * can be useful for converting text to word count vectors. + * + * @see + * Latent Dirichlet allocation (Wikipedia) + */ +@Since("1.6.0") +class LDA @Since("1.6.0") ( + @Since("1.6.0") override val uid: String) + extends Estimator[LDAModel] with LDAParams with DefaultParamsWritable { + + @Since("1.6.0") + def this() = this(Identifiable.randomUID("lda")) + + setDefault(maxIter -> 20, k -> 10, optimizer -> "online", checkpointInterval -> 10, + learningOffset -> 1024, learningDecay -> 0.51, subsamplingRate -> 0.05, + optimizeDocConcentration -> true, keepLastCheckpoint -> true) + + /** + * The features for LDA should be a `Vector` representing the word counts in a document. + * The vector should be of length vocabSize, with counts for each term (word). + * + * @group setParam + */ + @Since("1.6.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** @group setParam */ + @Since("1.6.0") + def setMaxIter(value: Int): this.type = set(maxIter, value) + + /** @group setParam */ + @Since("1.6.0") + def setSeed(value: Long): this.type = set(seed, value) + + /** @group setParam */ + @Since("1.6.0") + def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) + + /** @group setParam */ + @Since("1.6.0") + def setK(value: Int): this.type = set(k, value) + + /** @group setParam */ + @Since("1.6.0") + def setDocConcentration(value: Array[Double]): this.type = set(docConcentration, value) + + /** @group setParam */ + @Since("1.6.0") + def setDocConcentration(value: Double): this.type = set(docConcentration, Array(value)) + + /** @group setParam */ + @Since("1.6.0") + def setTopicConcentration(value: Double): this.type = set(topicConcentration, value) + + /** @group setParam */ + @Since("1.6.0") + def setOptimizer(value: String): this.type = set(optimizer, value) + + /** @group setParam */ + @Since("1.6.0") + def setTopicDistributionCol(value: String): this.type = set(topicDistributionCol, value) + + /** @group expertSetParam */ + @Since("1.6.0") + def setLearningOffset(value: Double): this.type = set(learningOffset, value) + + /** @group expertSetParam */ + @Since("1.6.0") + def setLearningDecay(value: Double): this.type = set(learningDecay, value) + + /** @group setParam */ + @Since("1.6.0") + def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) + + /** @group expertSetParam */ + @Since("1.6.0") + def setOptimizeDocConcentration(value: Boolean): this.type = set(optimizeDocConcentration, value) + + /** @group expertSetParam */ + @Since("2.0.0") + def setKeepLastCheckpoint(value: Boolean): this.type = set(keepLastCheckpoint, value) + + @Since("1.6.0") + override def copy(extra: ParamMap): LDA = defaultCopy(extra) + + @Since("2.0.0") + override def fit(dataset: Dataset[_]): LDAModel = { + transformSchema(dataset.schema, logging = true) + + val instr = Instrumentation.create(this, dataset) + instr.logParams(featuresCol, topicDistributionCol, k, maxIter, subsamplingRate, + checkpointInterval, keepLastCheckpoint, optimizeDocConcentration, topicConcentration, + learningDecay, optimizer, learningOffset, seed) + + val oldLDA = new OldLDA() + .setK($(k)) + .setDocConcentration(getOldDocConcentration) + .setTopicConcentration(getOldTopicConcentration) + .setMaxIterations($(maxIter)) + .setSeed($(seed)) + .setCheckpointInterval($(checkpointInterval)) + .setOptimizer(getOldOptimizer) + // TODO: persist here, or in old LDA? + val oldData = LDA.getOldDataset(dataset, $(featuresCol)) + val oldModel = oldLDA.run(oldData) + val newModel = oldModel match { + case m: OldLocalLDAModel => + new LocalLDAModel(uid, m.vocabSize, m, dataset.sparkSession) + case m: OldDistributedLDAModel => + new DistributedLDAModel(uid, m.vocabSize, m, dataset.sparkSession, None) + } + + instr.logNumFeatures(newModel.vocabSize) + val model = copyValues(newModel).setParent(this) + instr.logSuccess(model) + model + } + + @Since("1.6.0") + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } +} + +@Since("2.0.0") +object LDA extends MLReadable[LDA] { + + /** Get dataset for spark.mllib LDA */ + private[clustering] def getOldDataset( + dataset: Dataset[_], + featuresCol: String): RDD[(Long, OldVector)] = { + dataset + .withColumn("docId", monotonically_increasing_id()) + .select("docId", featuresCol) + .rdd + .map { case Row(docId: Long, features: Vector) => + (docId, OldVectors.fromML(features)) + } + } + + private class LDAReader extends MLReader[LDA] { + + private val className = classOf[LDA].getName + + override def load(path: String): LDA = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val model = new LDA(metadata.uid) + LDAParams.getAndSetParams(model, metadata) + model + } + } + + override def read: MLReader[LDA] = new LDAReader + + @Since("2.0.0") + override def load(path: String): LDA = super.load(path) +} 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 1fe3abaca81c..bff72b20e1c3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -18,29 +18,31 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.{Identifiable, SchemaUtils} +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType /** * :: Experimental :: * Evaluator for binary classification, which expects two input columns: rawPrediction and label. + * The rawPrediction column can be of type double (binary 0/1 prediction, or probability of label 1) + * or of type vector (length-2 vector of raw predictions, scores, or label probabilities). */ @Since("1.2.0") @Experimental class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override val uid: String) - extends Evaluator with HasRawPredictionCol with HasLabelCol { + extends Evaluator with HasRawPredictionCol with HasLabelCol with DefaultParamsWritable { @Since("1.2.0") def this() = this(Identifiable.randomUID("binEval")) /** - * param for metric name in evaluation - * Default: areaUnderROC + * param for metric name in evaluation (supports `"areaUnderROC"` (default), `"areaUnderPR"`) * @group param */ @Since("1.2.0") @@ -62,30 +64,23 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va @Since("1.5.0") def setRawPredictionCol(value: String): this.type = set(rawPredictionCol, value) - /** - * @group setParam - * @deprecated use [[setRawPredictionCol()]] instead - */ - @deprecated("use setRawPredictionCol instead", "1.5.0") - @Since("1.2.0") - def setScoreCol(value: String): this.type = set(rawPredictionCol, value) - /** @group setParam */ @Since("1.2.0") def setLabelCol(value: String): this.type = set(labelCol, value) setDefault(metricName -> "areaUnderROC") - @Since("1.2.0") - override def evaluate(dataset: DataFrame): Double = { + @Since("2.0.0") + override def evaluate(dataset: Dataset[_]): Double = { val schema = dataset.schema - SchemaUtils.checkColumnType(schema, $(rawPredictionCol), new VectorUDT) - SchemaUtils.checkColumnType(schema, $(labelCol), DoubleType) + SchemaUtils.checkColumnTypes(schema, $(rawPredictionCol), Seq(DoubleType, new VectorUDT)) + SchemaUtils.checkNumericType(schema, $(labelCol)) // TODO: When dataset metadata has been implemented, check rawPredictionCol vector length = 2. - val scoreAndLabels = dataset.select($(rawPredictionCol), $(labelCol)) - .map { case Row(rawPrediction: Vector, label: Double) => - (rawPrediction(1), label) + val scoreAndLabels = + dataset.select(col($(rawPredictionCol)), col($(labelCol)).cast(DoubleType)).rdd.map { + case Row(rawPrediction: Vector, label: Double) => (rawPrediction(1), label) + case Row(rawPrediction: Double, label: Double) => (rawPrediction, label) } val metrics = new BinaryClassificationMetrics(scoreAndLabels) val metric = $(metricName) match { @@ -105,3 +100,10 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va @Since("1.4.1") override def copy(extra: ParamMap): BinaryClassificationEvaluator = defaultCopy(extra) } + +@Since("1.6.0") +object BinaryClassificationEvaluator extends DefaultParamsReadable[BinaryClassificationEvaluator] { + + @Since("1.6.0") + override def load(path: String): BinaryClassificationEvaluator = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala new file mode 100644 index 000000000000..d6ec5223237b --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala @@ -0,0 +1,436 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.evaluation + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors, VectorUDT} +import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators} +import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasPredictionCol} +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.functions.{avg, col, udf} +import org.apache.spark.sql.types.DoubleType + +/** + * :: Experimental :: + * + * Evaluator for clustering results. + * The metric computes the Silhouette measure + * using the squared Euclidean distance. + * + * The Silhouette is a measure for the validation + * of the consistency within clusters. It ranges + * between 1 and -1, where a value close to 1 + * means that the points in a cluster are close + * to the other points in the same cluster and + * far from the points of the other clusters. + */ +@Experimental +@Since("2.3.0") +class ClusteringEvaluator @Since("2.3.0") (@Since("2.3.0") override val uid: String) + extends Evaluator with HasPredictionCol with HasFeaturesCol with DefaultParamsWritable { + + @Since("2.3.0") + def this() = this(Identifiable.randomUID("cluEval")) + + @Since("2.3.0") + override def copy(pMap: ParamMap): ClusteringEvaluator = this.defaultCopy(pMap) + + @Since("2.3.0") + override def isLargerBetter: Boolean = true + + /** @group setParam */ + @Since("2.3.0") + def setPredictionCol(value: String): this.type = set(predictionCol, value) + + /** @group setParam */ + @Since("2.3.0") + def setFeaturesCol(value: String): this.type = set(featuresCol, value) + + /** + * param for metric name in evaluation + * (supports `"silhouette"` (default)) + * @group param + */ + @Since("2.3.0") + val metricName: Param[String] = { + val allowedParams = ParamValidators.inArray(Array("silhouette")) + new Param( + this, "metricName", "metric name in evaluation (silhouette)", allowedParams) + } + + /** @group getParam */ + @Since("2.3.0") + def getMetricName: String = $(metricName) + + /** @group setParam */ + @Since("2.3.0") + def setMetricName(value: String): this.type = set(metricName, value) + + setDefault(metricName -> "silhouette") + + @Since("2.3.0") + override def evaluate(dataset: Dataset[_]): Double = { + SchemaUtils.checkColumnType(dataset.schema, $(featuresCol), new VectorUDT) + SchemaUtils.checkNumericType(dataset.schema, $(predictionCol)) + + $(metricName) match { + case "silhouette" => + SquaredEuclideanSilhouette.computeSilhouetteScore( + dataset, $(predictionCol), $(featuresCol) + ) + } + } +} + + +@Since("2.3.0") +object ClusteringEvaluator + extends DefaultParamsReadable[ClusteringEvaluator] { + + @Since("2.3.0") + override def load(path: String): ClusteringEvaluator = super.load(path) + +} + + +/** + * SquaredEuclideanSilhouette computes the average of the + * Silhouette over all the data of the dataset, which is + * a measure of how appropriately the data have been clustered. + * + * The Silhouette for each point `i` is defined as: + * + *
    + * $$ + * s_{i} = \frac{b_{i}-a_{i}}{max\{a_{i},b_{i}\}} + * $$ + *
    + * + * which can be rewritten as + * + *
    + * $$ + * s_{i}= \begin{cases} + * 1-\frac{a_{i}}{b_{i}} & \text{if } a_{i} \leq b_{i} \\ + * \frac{b_{i}}{a_{i}}-1 & \text{if } a_{i} \gt b_{i} \end{cases} + * $$ + *
    + * + * where `$a_{i}$` is the average dissimilarity of `i` with all other data + * within the same cluster, `$b_{i}$` is the lowest average dissimilarity + * of `i` to any other cluster, of which `i` is not a member. + * `$a_{i}$` can be interpreted as how well `i` is assigned to its cluster + * (the smaller the value, the better the assignment), while `$b_{i}$` is + * a measure of how well `i` has not been assigned to its "neighboring cluster", + * ie. the nearest cluster to `i`. + * + * Unfortunately, the naive implementation of the algorithm requires to compute + * the distance of each couple of points in the dataset. Since the computation of + * the distance measure takes `D` operations - if `D` is the number of dimensions + * of each point, the computational complexity of the algorithm is `O(N^2^*D)`, where + * `N` is the cardinality of the dataset. Of course this is not scalable in `N`, + * which is the critical number in a Big Data context. + * + * The algorithm which is implemented in this object, instead, is an efficient + * and parallel implementation of the Silhouette using the squared Euclidean + * distance measure. + * + * With this assumption, the total distance of the point `X` + * to the points `$C_{i}$` belonging to the cluster `$\Gamma$` is: + * + *
    + * $$ + * \sum\limits_{i=1}^N d(X, C_{i} ) = + * \sum\limits_{i=1}^N \Big( \sum\limits_{j=1}^D (x_{j}-c_{ij})^2 \Big) + * = \sum\limits_{i=1}^N \Big( \sum\limits_{j=1}^D x_{j}^2 + + * \sum\limits_{j=1}^D c_{ij}^2 -2\sum\limits_{j=1}^D x_{j}c_{ij} \Big) + * = \sum\limits_{i=1}^N \sum\limits_{j=1}^D x_{j}^2 + + * \sum\limits_{i=1}^N \sum\limits_{j=1}^D c_{ij}^2 + * -2 \sum\limits_{i=1}^N \sum\limits_{j=1}^D x_{j}c_{ij} + * $$ + *
    + * + * where `$x_{j}$` is the `j`-th dimension of the point `X` and + * `$c_{ij}$` is the `j`-th dimension of the `i`-th point in cluster `$\Gamma$`. + * + * Then, the first term of the equation can be rewritten as: + * + *
    + * $$ + * \sum\limits_{i=1}^N \sum\limits_{j=1}^D x_{j}^2 = N \xi_{X} \text{ , + * with } \xi_{X} = \sum\limits_{j=1}^D x_{j}^2 + * $$ + *
    + * + * where `$\xi_{X}$` is fixed for each point and it can be precomputed. + * + * Moreover, the second term is fixed for each cluster too, + * thus we can name it `$\Psi_{\Gamma}$` + * + *
    + * $$ + * \sum\limits_{i=1}^N \sum\limits_{j=1}^D c_{ij}^2 = + * \sum\limits_{i=1}^N \xi_{C_{i}} = \Psi_{\Gamma} + * $$ + *
    + * + * Last, the third element becomes + * + *
    + * $$ + * \sum\limits_{i=1}^N \sum\limits_{j=1}^D x_{j}c_{ij} = + * \sum\limits_{j=1}^D \Big(\sum\limits_{i=1}^N c_{ij} \Big) x_{j} + * $$ + *
    + * + * thus defining the vector + * + *
    + * $$ + * Y_{\Gamma}:Y_{\Gamma j} = \sum\limits_{i=1}^N c_{ij} , j=0, ..., D + * $$ + *
    + * + * which is fixed for each cluster `$\Gamma$`, we have + * + *
    + * $$ + * \sum\limits_{j=1}^D \Big(\sum\limits_{i=1}^N c_{ij} \Big) x_{j} = + * \sum\limits_{j=1}^D Y_{\Gamma j} x_{j} + * $$ + *
    + * + * In this way, the previous equation becomes + * + *
    + * $$ + * N\xi_{X} + \Psi_{\Gamma} - 2 \sum\limits_{j=1}^D Y_{\Gamma j} x_{j} + * $$ + *
    + * + * and the average distance of a point to a cluster can be computed as + * + *
    + * $$ + * \frac{\sum\limits_{i=1}^N d(X, C_{i} )}{N} = + * \frac{N\xi_{X} + \Psi_{\Gamma} - 2 \sum\limits_{j=1}^D Y_{\Gamma j} x_{j}}{N} = + * \xi_{X} + \frac{\Psi_{\Gamma} }{N} - 2 \frac{\sum\limits_{j=1}^D Y_{\Gamma j} x_{j}}{N} + * $$ + *
    + * + * Thus, it is enough to precompute: the constant `$\xi_{X}$` for each point `X`; the + * constants `$\Psi_{\Gamma}$`, `N` and the vector `$Y_{\Gamma}$` for + * each cluster `$\Gamma$`. + * + * In the implementation, the precomputed values for the clusters + * are distributed among the worker nodes via broadcasted variables, + * because we can assume that the clusters are limited in number and + * anyway they are much fewer than the points. + * + * The main strengths of this algorithm are the low computational complexity + * and the intrinsic parallelism. The precomputed information for each point + * and for each cluster can be computed with a computational complexity + * which is `O(N/W)`, where `N` is the number of points in the dataset and + * `W` is the number of worker nodes. After that, every point can be + * analyzed independently of the others. + * + * For every point we need to compute the average distance to all the clusters. + * Since the formula above requires `O(D)` operations, this phase has a + * computational complexity which is `O(C*D*N/W)` where `C` is the number of + * clusters (which we assume quite low), `D` is the number of dimensions, + * `N` is the number of points in the dataset and `W` is the number + * of worker nodes. + */ +private[evaluation] object SquaredEuclideanSilhouette { + + private[this] var kryoRegistrationPerformed: Boolean = false + + /** + * This method registers the class + * [[org.apache.spark.ml.evaluation.SquaredEuclideanSilhouette.ClusterStats]] + * for kryo serialization. + * + * @param sc `SparkContext` to be used + */ + def registerKryoClasses(sc: SparkContext): Unit = { + if (!kryoRegistrationPerformed) { + sc.getConf.registerKryoClasses( + Array( + classOf[SquaredEuclideanSilhouette.ClusterStats] + ) + ) + kryoRegistrationPerformed = true + } + } + + case class ClusterStats(featureSum: Vector, squaredNormSum: Double, numOfPoints: Long) + + /** + * The method takes the input dataset and computes the aggregated values + * about a cluster which are needed by the algorithm. + * + * @param df The DataFrame which contains the input data + * @param predictionCol The name of the column which contains the predicted cluster id + * for the point. + * @param featuresCol The name of the column which contains the feature vector of the point. + * @return A [[scala.collection.immutable.Map]] which associates each cluster id + * to a [[ClusterStats]] object (which contains the precomputed values `N`, + * `$\Psi_{\Gamma}$` and `$Y_{\Gamma}$` for a cluster). + */ + def computeClusterStats( + df: DataFrame, + predictionCol: String, + featuresCol: String): Map[Double, ClusterStats] = { + val numFeatures = df.select(col(featuresCol)).first().getAs[Vector](0).size + val clustersStatsRDD = df.select( + col(predictionCol).cast(DoubleType), col(featuresCol), col("squaredNorm")) + .rdd + .map { row => (row.getDouble(0), (row.getAs[Vector](1), row.getDouble(2))) } + .aggregateByKey[(DenseVector, Double, Long)]((Vectors.zeros(numFeatures).toDense, 0.0, 0L))( + seqOp = { + case ( + (featureSum: DenseVector, squaredNormSum: Double, numOfPoints: Long), + (features, squaredNorm) + ) => + BLAS.axpy(1.0, features, featureSum) + (featureSum, squaredNormSum + squaredNorm, numOfPoints + 1) + }, + combOp = { + case ( + (featureSum1, squaredNormSum1, numOfPoints1), + (featureSum2, squaredNormSum2, numOfPoints2) + ) => + BLAS.axpy(1.0, featureSum2, featureSum1) + (featureSum1, squaredNormSum1 + squaredNormSum2, numOfPoints1 + numOfPoints2) + } + ) + + clustersStatsRDD + .collectAsMap() + .mapValues { + case (featureSum: DenseVector, squaredNormSum: Double, numOfPoints: Long) => + SquaredEuclideanSilhouette.ClusterStats(featureSum, squaredNormSum, numOfPoints) + } + .toMap + } + + /** + * It computes the Silhouette coefficient for a point. + * + * @param broadcastedClustersMap A map of the precomputed values for each cluster. + * @param features The [[org.apache.spark.ml.linalg.Vector]] representing the current point. + * @param clusterId The id of the cluster the current point belongs to. + * @param squaredNorm The `$\Xi_{X}$` (which is the squared norm) precomputed for the point. + * @return The Silhouette for the point. + */ + def computeSilhouetteCoefficient( + broadcastedClustersMap: Broadcast[Map[Double, ClusterStats]], + features: Vector, + clusterId: Double, + squaredNorm: Double): Double = { + + def compute(squaredNorm: Double, point: Vector, clusterStats: ClusterStats): Double = { + val pointDotClusterFeaturesSum = BLAS.dot(point, clusterStats.featureSum) + + squaredNorm + + clusterStats.squaredNormSum / clusterStats.numOfPoints - + 2 * pointDotClusterFeaturesSum / clusterStats.numOfPoints + } + + // Here we compute the average dissimilarity of the + // current point to any cluster of which the point + // is not a member. + // The cluster with the lowest average dissimilarity + // - i.e. the nearest cluster to the current point - + // is said to be the "neighboring cluster". + var neighboringClusterDissimilarity = Double.MaxValue + broadcastedClustersMap.value.keySet.foreach { + c => + if (c != clusterId) { + val dissimilarity = compute(squaredNorm, features, broadcastedClustersMap.value(c)) + if(dissimilarity < neighboringClusterDissimilarity) { + neighboringClusterDissimilarity = dissimilarity + } + } + } + val currentCluster = broadcastedClustersMap.value(clusterId) + // adjustment for excluding the node itself from + // the computation of the average dissimilarity + val currentClusterDissimilarity = if (currentCluster.numOfPoints == 1) { + 0 + } else { + compute(squaredNorm, features, currentCluster) * currentCluster.numOfPoints / + (currentCluster.numOfPoints - 1) + } + + (currentClusterDissimilarity compare neighboringClusterDissimilarity).signum match { + case -1 => 1 - (currentClusterDissimilarity / neighboringClusterDissimilarity) + case 1 => (neighboringClusterDissimilarity / currentClusterDissimilarity) - 1 + case 0 => 0.0 + } + } + + /** + * Compute the mean Silhouette values of all samples. + * + * @param dataset The input dataset (previously clustered) on which compute the Silhouette. + * @param predictionCol The name of the column which contains the predicted cluster id + * for the point. + * @param featuresCol The name of the column which contains the feature vector of the point. + * @return The average of the Silhouette values of the clustered data. + */ + def computeSilhouetteScore( + dataset: Dataset[_], + predictionCol: String, + featuresCol: String): Double = { + SquaredEuclideanSilhouette.registerKryoClasses(dataset.sparkSession.sparkContext) + + val squaredNormUDF = udf { + features: Vector => math.pow(Vectors.norm(features, 2.0), 2.0) + } + val dfWithSquaredNorm = dataset.withColumn("squaredNorm", squaredNormUDF(col(featuresCol))) + + // compute aggregate values for clusters needed by the algorithm + val clustersStatsMap = SquaredEuclideanSilhouette + .computeClusterStats(dfWithSquaredNorm, predictionCol, featuresCol) + + // Silhouette is reasonable only when the number of clusters is grater then 1 + assert(clustersStatsMap.size > 1, "Number of clusters must be greater than one.") + + val bClustersStatsMap = dataset.sparkSession.sparkContext.broadcast(clustersStatsMap) + + val computeSilhouetteCoefficientUDF = udf { + computeSilhouetteCoefficient(bClustersStatsMap, _: Vector, _: Double, _: Double) + } + + val silhouetteScore = dfWithSquaredNorm + .select(avg( + computeSilhouetteCoefficientUDF( + col(featuresCol), col(predictionCol).cast(DoubleType), col("squaredNorm")) + )) + .collect()(0) + .getDouble(0) + + bClustersStatsMap.destroy() + + silhouetteScore + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala index 0f22cca3a78d..e7b949ddce34 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.ml.param.{ParamMap, Params} -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.Dataset /** * :: DeveloperApi :: @@ -30,27 +30,30 @@ import org.apache.spark.sql.DataFrame abstract class Evaluator extends Params { /** - * Evaluates model output and returns a scalar metric (larger is better). + * Evaluates model output and returns a scalar metric. + * The value of [[isLargerBetter]] specifies whether larger values are better. * * @param dataset a dataset that contains labels/observations and predictions. * @param paramMap parameter map that specifies the input columns and output metrics * @return metric */ - @Since("1.5.0") - def evaluate(dataset: DataFrame, paramMap: ParamMap): Double = { + @Since("2.0.0") + def evaluate(dataset: Dataset[_], paramMap: ParamMap): Double = { this.copy(paramMap).evaluate(dataset) } /** - * Evaluates the output. + * Evaluates model output and returns a scalar metric. + * The value of [[isLargerBetter]] specifies whether larger values are better. + * * @param dataset a dataset that contains labels/observations and predictions. * @return metric */ - @Since("1.5.0") - def evaluate(dataset: DataFrame): Double + @Since("2.0.0") + def evaluate(dataset: Dataset[_]): Double /** - * Indicates whether the metric returned by [[evaluate()]] should be maximized (true, default) + * Indicates whether the metric returned by `evaluate` should be maximized (true, default) * or minimized (false). * A given evaluator may support multiple metrics which may be maximized or minimized. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index df5f04ca5a8d..794b1e7d9d88 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -18,36 +18,37 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.{Experimental, Since} -import org.apache.spark.ml.param.{ParamMap, ParamValidators, Param} +import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators} import org.apache.spark.ml.param.shared.{HasLabelCol, HasPredictionCol} -import org.apache.spark.ml.util.{SchemaUtils, Identifiable} +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.sql.{Row, DataFrame} +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType /** * :: Experimental :: - * Evaluator for multiclass classification, which expects two input columns: score and label. + * Evaluator for multiclass classification, which expects two input columns: prediction and label. */ @Since("1.5.0") @Experimental class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") override val uid: String) - extends Evaluator with HasPredictionCol with HasLabelCol { + extends Evaluator with HasPredictionCol with HasLabelCol with DefaultParamsWritable { @Since("1.5.0") def this() = this(Identifiable.randomUID("mcEval")) /** - * param for metric name in evaluation (supports `"f1"` (default), `"precision"`, `"recall"`, - * `"weightedPrecision"`, `"weightedRecall"`) + * param for metric name in evaluation (supports `"f1"` (default), `"weightedPrecision"`, + * `"weightedRecall"`, `"accuracy"`) * @group param */ @Since("1.5.0") val metricName: Param[String] = { - val allowedParams = ParamValidators.inArray(Array("f1", "precision", - "recall", "weightedPrecision", "weightedRecall")) + val allowedParams = ParamValidators.inArray(Array("f1", "weightedPrecision", + "weightedRecall", "accuracy")) new Param(this, "metricName", "metric name in evaluation " + - "(f1|precision|recall|weightedPrecision|weightedRecall)", allowedParams) + "(f1|weightedPrecision|weightedRecall|accuracy)", allowedParams) } /** @group getParam */ @@ -68,36 +69,37 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid setDefault(metricName -> "f1") - @Since("1.5.0") - override def evaluate(dataset: DataFrame): Double = { + @Since("2.0.0") + override def evaluate(dataset: Dataset[_]): Double = { val schema = dataset.schema SchemaUtils.checkColumnType(schema, $(predictionCol), DoubleType) - SchemaUtils.checkColumnType(schema, $(labelCol), DoubleType) + SchemaUtils.checkNumericType(schema, $(labelCol)) - val predictionAndLabels = dataset.select($(predictionCol), $(labelCol)) - .map { case Row(prediction: Double, label: Double) => - (prediction, label) - } + val predictionAndLabels = + dataset.select(col($(predictionCol)), col($(labelCol)).cast(DoubleType)).rdd.map { + case Row(prediction: Double, label: Double) => (prediction, label) + } val metrics = new MulticlassMetrics(predictionAndLabels) val metric = $(metricName) match { case "f1" => metrics.weightedFMeasure - case "precision" => metrics.precision - case "recall" => metrics.recall case "weightedPrecision" => metrics.weightedPrecision case "weightedRecall" => metrics.weightedRecall + case "accuracy" => metrics.accuracy } metric } @Since("1.5.0") - override def isLargerBetter: Boolean = $(metricName) match { - case "f1" => true - case "precision" => true - case "recall" => true - case "weightedPrecision" => true - case "weightedRecall" => true - } + override def isLargerBetter: Boolean = true @Since("1.5.0") override def copy(extra: ParamMap): MulticlassClassificationEvaluator = defaultCopy(extra) } + +@Since("1.6.0") +object MulticlassClassificationEvaluator + extends DefaultParamsReadable[MulticlassClassificationEvaluator] { + + @Since("1.6.0") + override def load(path: String): MulticlassClassificationEvaluator = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala index ba012f444d3e..031cd0d635bf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala @@ -20,9 +20,9 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators} import org.apache.spark.ml.param.shared.{HasLabelCol, HasPredictionCol} -import org.apache.spark.ml.util.{Identifiable, SchemaUtils} +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} import org.apache.spark.mllib.evaluation.RegressionMetrics -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DoubleType, FloatType} @@ -33,17 +33,18 @@ import org.apache.spark.sql.types.{DoubleType, FloatType} @Since("1.4.0") @Experimental final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val uid: String) - extends Evaluator with HasPredictionCol with HasLabelCol { + extends Evaluator with HasPredictionCol with HasLabelCol with DefaultParamsWritable { @Since("1.4.0") def this() = this(Identifiable.randomUID("regEval")) /** - * param for metric name in evaluation (supports `"rmse"` (default), `"mse"`, `"r2"`, and `"mae"`) + * Param for metric name in evaluation. Supports: + * - `"rmse"` (default): root mean squared error + * - `"mse"`: mean squared error + * - `"r2"`: R^2^ metric + * - `"mae"`: mean absolute error * - * Because we will maximize evaluation value (ref: `CrossValidator`), - * when we evaluate a metric that is needed to minimize (e.g., `"rmse"`, `"mse"`, `"mae"`), - * we take and output the negative of this metric. * @group param */ @Since("1.4.0") @@ -70,19 +71,16 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui setDefault(metricName -> "rmse") - @Since("1.4.0") - override def evaluate(dataset: DataFrame): Double = { + @Since("2.0.0") + override def evaluate(dataset: Dataset[_]): Double = { val schema = dataset.schema - val predictionType = schema($(predictionCol)).dataType - require(predictionType == FloatType || predictionType == DoubleType) - val labelType = schema($(labelCol)).dataType - require(labelType == FloatType || labelType == DoubleType) + SchemaUtils.checkColumnTypes(schema, $(predictionCol), Seq(DoubleType, FloatType)) + SchemaUtils.checkNumericType(schema, $(labelCol)) val predictionAndLabels = dataset .select(col($(predictionCol)).cast(DoubleType), col($(labelCol)).cast(DoubleType)) - .map { case Row(prediction: Double, label: Double) => - (prediction, label) - } + .rdd + .map { case Row(prediction: Double, label: Double) => (prediction, label) } val metrics = new RegressionMetrics(predictionAndLabels) val metric = $(metricName) match { case "rmse" => metrics.rootMeanSquaredError @@ -104,3 +102,10 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui @Since("1.5.0") override def copy(extra: ParamMap): RegressionEvaluator = defaultCopy(extra) } + +@Since("1.6.0") +object RegressionEvaluator extends DefaultParamsReadable[RegressionEvaluator] { + + @Since("1.6.0") + override def load(path: String): RegressionEvaluator = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index edad75443645..2b0862c60fdf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -17,24 +17,27 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental +import scala.collection.mutable.ArrayBuilder + +import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.BinaryAttribute +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} -import org.apache.spark.ml.util.{Identifiable, SchemaUtils} +import org.apache.spark.ml.util._ import org.apache.spark.sql._ import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DoubleType, StructType} +import org.apache.spark.sql.types._ /** - * :: Experimental :: * Binarize a column of continuous features given a threshold. */ -@Experimental -final class Binarizer(override val uid: String) - extends Transformer with HasInputCol with HasOutputCol { +@Since("1.4.0") +final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) + extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("binarizer")) /** @@ -44,46 +47,87 @@ final class Binarizer(override val uid: String) * Default: 0.0 * @group param */ + @Since("1.4.0") val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold used to binarize continuous features") /** @group getParam */ + @Since("1.4.0") def getThreshold: Double = $(threshold) /** @group setParam */ + @Since("1.4.0") def setThreshold(value: Double): this.type = set(threshold, value) setDefault(threshold -> 0.0) /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) - override def transform(dataset: DataFrame): DataFrame = { - transformSchema(dataset.schema, logging = true) + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + val outputSchema = transformSchema(dataset.schema, logging = true) + val schema = dataset.schema + val inputType = schema($(inputCol)).dataType val td = $(threshold) - val binarizer = udf { in: Double => if (in > td) 1.0 else 0.0 } - val outputColName = $(outputCol) - val metadata = BinaryAttribute.defaultAttr.withName(outputColName).toMetadata() - dataset.select(col("*"), - binarizer(col($(inputCol))).as(outputColName, metadata)) + + val binarizerDouble = udf { in: Double => if (in > td) 1.0 else 0.0 } + val binarizerVector = udf { (data: Vector) => + val indices = ArrayBuilder.make[Int] + val values = ArrayBuilder.make[Double] + + data.foreachActive { (index, value) => + if (value > td) { + indices += index + values += 1.0 + } + } + + Vectors.sparse(data.size, indices.result(), values.result()).compressed + } + + val metadata = outputSchema($(outputCol)).metadata + + inputType match { + case DoubleType => + dataset.select(col("*"), binarizerDouble(col($(inputCol))).as($(outputCol), metadata)) + case _: VectorUDT => + dataset.select(col("*"), binarizerVector(col($(inputCol))).as($(outputCol), metadata)) + } } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(inputCol), DoubleType) - - val inputFields = schema.fields + val inputType = schema($(inputCol)).dataType val outputColName = $(outputCol) - require(inputFields.forall(_.name != outputColName), - s"Output column $outputColName already exists.") - - val attr = BinaryAttribute.defaultAttr.withName(outputColName) - val outputFields = inputFields :+ attr.toStructField() - StructType(outputFields) + val outCol: StructField = inputType match { + case DoubleType => + BinaryAttribute.defaultAttr.withName(outputColName).toStructField() + case _: VectorUDT => + StructField(outputColName, new VectorUDT) + case _ => + throw new IllegalArgumentException(s"Data type $inputType is not supported.") + } + + if (schema.fieldNames.contains(outputColName)) { + throw new IllegalArgumentException(s"Output column $outputColName already exists.") + } + StructType(schema.fields :+ outCol) } + @Since("1.4.1") override def copy(extra: ParamMap): Binarizer = defaultCopy(extra) } + +@Since("1.6.0") +object Binarizer extends DefaultParamsReadable[Binarizer] { + + @Since("1.6.0") + override def load(path: String): Binarizer = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala new file mode 100644 index 000000000000..36a46ca6ff4b --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSH.scala @@ -0,0 +1,237 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import scala.util.Random + +import breeze.linalg.normalize +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared.HasSeed +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.StructType + +/** + * :: Experimental :: + * + * Params for [[BucketedRandomProjectionLSH]]. + */ +private[ml] trait BucketedRandomProjectionLSHParams extends Params { + + /** + * The length of each hash bucket, a larger bucket lowers the false negative rate. The number of + * buckets will be `(max L2 norm of input vectors) / bucketLength`. + * + * + * If input vectors are normalized, 1-10 times of pow(numRecords, -1/inputDim) would be a + * reasonable value + * @group param + */ + val bucketLength: DoubleParam = new DoubleParam(this, "bucketLength", + "the length of each hash bucket, a larger bucket lowers the false negative rate.", + ParamValidators.gt(0)) + + /** @group getParam */ + final def getBucketLength: Double = $(bucketLength) +} + +/** + * :: Experimental :: + * + * Model produced by [[BucketedRandomProjectionLSH]], where multiple random vectors are stored. The + * vectors are normalized to be unit vectors and each vector is used in a hash function: + * `h_i(x) = floor(r_i.dot(x) / bucketLength)` + * where `r_i` is the i-th random unit vector. The number of buckets will be `(max L2 norm of input + * vectors) / bucketLength`. + * + * @param randUnitVectors An array of random unit vectors. Each vector represents a hash function. + */ +@Experimental +@Since("2.1.0") +class BucketedRandomProjectionLSHModel private[ml]( + override val uid: String, + private[ml] val randUnitVectors: Array[Vector]) + extends LSHModel[BucketedRandomProjectionLSHModel] with BucketedRandomProjectionLSHParams { + + @Since("2.1.0") + override protected[ml] val hashFunction: Vector => Array[Vector] = { + key: Vector => { + val hashValues: Array[Double] = randUnitVectors.map({ + randUnitVector => Math.floor(BLAS.dot(key, randUnitVector) / $(bucketLength)) + }) + // TODO: Output vectors of dimension numHashFunctions in SPARK-18450 + hashValues.map(Vectors.dense(_)) + } + } + + @Since("2.1.0") + override protected[ml] def keyDistance(x: Vector, y: Vector): Double = { + Math.sqrt(Vectors.sqdist(x, y)) + } + + @Since("2.1.0") + override protected[ml] def hashDistance(x: Seq[Vector], y: Seq[Vector]): Double = { + // Since it's generated by hashing, it will be a pair of dense vectors. + x.zip(y).map(vectorPair => Vectors.sqdist(vectorPair._1, vectorPair._2)).min + } + + @Since("2.1.0") + override def copy(extra: ParamMap): BucketedRandomProjectionLSHModel = { + val copied = new BucketedRandomProjectionLSHModel(uid, randUnitVectors).setParent(parent) + copyValues(copied, extra) + } + + @Since("2.1.0") + override def write: MLWriter = { + new BucketedRandomProjectionLSHModel.BucketedRandomProjectionLSHModelWriter(this) + } +} + +/** + * :: Experimental :: + * + * This [[BucketedRandomProjectionLSH]] implements Locality Sensitive Hashing functions for + * Euclidean distance metrics. + * + * The input is dense or sparse vectors, each of which represents a point in the Euclidean + * distance space. The output will be vectors of configurable dimension. Hash values in the + * same dimension are calculated by the same hash function. + * + * References: + * + * 1. + * Wikipedia on Stable Distributions + * + * 2. Wang, Jingdong et al. "Hashing for similarity search: A survey." arXiv preprint + * arXiv:1408.2927 (2014). + */ +@Experimental +@Since("2.1.0") +class BucketedRandomProjectionLSH(override val uid: String) + extends LSH[BucketedRandomProjectionLSHModel] + with BucketedRandomProjectionLSHParams with HasSeed { + + @Since("2.1.0") + override def setInputCol(value: String): this.type = super.setInputCol(value) + + @Since("2.1.0") + override def setOutputCol(value: String): this.type = super.setOutputCol(value) + + @Since("2.1.0") + override def setNumHashTables(value: Int): this.type = super.setNumHashTables(value) + + @Since("2.1.0") + def this() = { + this(Identifiable.randomUID("brp-lsh")) + } + + /** @group setParam */ + @Since("2.1.0") + def setBucketLength(value: Double): this.type = set(bucketLength, value) + + /** @group setParam */ + @Since("2.1.0") + def setSeed(value: Long): this.type = set(seed, value) + + @Since("2.1.0") + override protected[this] def createRawLSHModel( + inputDim: Int): BucketedRandomProjectionLSHModel = { + val rand = new Random($(seed)) + val randUnitVectors: Array[Vector] = { + Array.fill($(numHashTables)) { + val randArray = Array.fill(inputDim)(rand.nextGaussian()) + Vectors.fromBreeze(normalize(breeze.linalg.Vector(randArray))) + } + } + new BucketedRandomProjectionLSHModel(uid, randUnitVectors) + } + + @Since("2.1.0") + override def transformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) + validateAndTransformSchema(schema) + } + + @Since("2.1.0") + override def copy(extra: ParamMap): this.type = defaultCopy(extra) +} + +@Since("2.1.0") +object BucketedRandomProjectionLSH extends DefaultParamsReadable[BucketedRandomProjectionLSH] { + + @Since("2.1.0") + override def load(path: String): BucketedRandomProjectionLSH = super.load(path) +} + +@Since("2.1.0") +object BucketedRandomProjectionLSHModel extends MLReadable[BucketedRandomProjectionLSHModel] { + + @Since("2.1.0") + override def read: MLReader[BucketedRandomProjectionLSHModel] = { + new BucketedRandomProjectionLSHModelReader + } + + @Since("2.1.0") + override def load(path: String): BucketedRandomProjectionLSHModel = super.load(path) + + private[BucketedRandomProjectionLSHModel] class BucketedRandomProjectionLSHModelWriter( + instance: BucketedRandomProjectionLSHModel) extends MLWriter { + + // TODO: Save using the existing format of Array[Vector] once SPARK-12878 is resolved. + private case class Data(randUnitVectors: Matrix) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val numRows = instance.randUnitVectors.length + require(numRows > 0) + val numCols = instance.randUnitVectors.head.size + val values = instance.randUnitVectors.map(_.toArray).reduce(Array.concat(_, _)) + val randMatrix = Matrices.dense(numRows, numCols, values) + val data = Data(randMatrix) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class BucketedRandomProjectionLSHModelReader + extends MLReader[BucketedRandomProjectionLSHModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[BucketedRandomProjectionLSHModel].getName + + override def load(path: String): BucketedRandomProjectionLSHModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + val Row(randUnitVectors: Matrix) = MLUtils.convertMatrixColumnsToML(data, "randUnitVectors") + .select("randUnitVectors") + .head() + val model = new BucketedRandomProjectionLSHModel(metadata.uid, + randUnitVectors.rowIter.toArray) + + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala index 6fdf25b015b0..6a11a75d1d56 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala @@ -20,62 +20,101 @@ package org.apache.spark.ml.feature import java.{util => ju} import org.apache.spark.SparkException -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Since import org.apache.spark.ml.Model import org.apache.spark.ml.attribute.NominalAttribute import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} -import org.apache.spark.ml.util.{Identifiable, SchemaUtils} +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol, HasOutputCol} +import org.apache.spark.ml.util._ import org.apache.spark.sql._ +import org.apache.spark.sql.expressions.UserDefinedFunction import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} /** - * :: Experimental :: * `Bucketizer` maps a column of continuous features to a column of feature buckets. */ -@Experimental -final class Bucketizer(override val uid: String) - extends Model[Bucketizer] with HasInputCol with HasOutputCol { +@Since("1.4.0") +final class Bucketizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) + extends Model[Bucketizer] with HasHandleInvalid with HasInputCol with HasOutputCol + with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("bucketizer")) /** * Parameter for mapping continuous features into buckets. With n+1 splits, there are n buckets. * A bucket defined by splits x,y holds values in the range [x,y) except the last bucket, which - * also includes y. Splits should be strictly increasing. + * also includes y. Splits should be of length greater than or equal to 3 and strictly increasing. * Values at -inf, inf must be explicitly provided to cover all Double values; * otherwise, values outside the splits specified will be treated as errors. + * + * See also [[handleInvalid]], which can optionally create an additional bucket for NaN values. + * * @group param */ + @Since("1.4.0") val splits: DoubleArrayParam = new DoubleArrayParam(this, "splits", "Split points for mapping continuous features into buckets. With n+1 splits, there are n " + "buckets. A bucket defined by splits x,y holds values in the range [x,y) except the last " + - "bucket, which also includes y. The splits should be strictly increasing. " + - "Values at -inf, inf must be explicitly provided to cover all Double values; " + + "bucket, which also includes y. The splits should be of length >= 3 and strictly " + + "increasing. Values at -inf, inf must be explicitly provided to cover all Double values; " + "otherwise, values outside the splits specified will be treated as errors.", Bucketizer.checkSplits) /** @group getParam */ + @Since("1.4.0") def getSplits: Array[Double] = $(splits) /** @group setParam */ + @Since("1.4.0") def setSplits(value: Array[Double]): this.type = set(splits, value) /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) - override def transform(dataset: DataFrame): DataFrame = { + /** + * Param for how to handle invalid entries. Options are 'skip' (filter out rows with + * invalid values), 'error' (throw an error), or 'keep' (keep invalid values in a special + * additional bucket). + * Default: "error" + * @group param + */ + @Since("2.1.0") + override val handleInvalid: Param[String] = new Param[String](this, "handleInvalid", + "how to handle invalid entries. Options are skip (filter out rows with invalid values), " + + "error (throw an error), or keep (keep invalid values in a special additional bucket).", + ParamValidators.inArray(Bucketizer.supportedHandleInvalids)) + + /** @group setParam */ + @Since("2.1.0") + def setHandleInvalid(value: String): this.type = set(handleInvalid, value) + setDefault(handleInvalid, Bucketizer.ERROR_INVALID) + + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema) - val bucketizer = udf { feature: Double => - Bucketizer.binarySearchForBuckets($(splits), feature) + val (filteredDataset, keepInvalid) = { + if (getHandleInvalid == Bucketizer.SKIP_INVALID) { + // "skip" NaN option is set, will filter out NaN values in the dataset + (dataset.na.drop().toDF(), false) + } else { + (dataset.toDF(), getHandleInvalid == Bucketizer.KEEP_INVALID) + } } - val newCol = bucketizer(dataset($(inputCol))) - val newField = prepOutputField(dataset.schema) - dataset.withColumn($(outputCol), newCol, newField.metadata) + + val bucketizer: UserDefinedFunction = udf { (feature: Double) => + Bucketizer.binarySearchForBuckets($(splits), feature, keepInvalid) + }.withName("bucketizer") + + val newCol = bucketizer(filteredDataset($(inputCol)).cast(DoubleType)) + val newField = prepOutputField(filteredDataset.schema) + filteredDataset.withColumn($(outputCol), newCol, newField.metadata) } private def prepOutputField(schema: StructType): StructField = { @@ -85,38 +124,68 @@ final class Bucketizer(override val uid: String) attr.toStructField() } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(inputCol), DoubleType) + SchemaUtils.checkNumericType(schema, $(inputCol)) SchemaUtils.appendColumn(schema, prepOutputField(schema)) } + @Since("1.4.1") override def copy(extra: ParamMap): Bucketizer = { defaultCopy[Bucketizer](extra).setParent(parent) } } -private[feature] object Bucketizer { - /** We require splits to be of length >= 3 and to be in strictly increasing order. */ - def checkSplits(splits: Array[Double]): Boolean = { +@Since("1.6.0") +object Bucketizer extends DefaultParamsReadable[Bucketizer] { + + private[feature] val SKIP_INVALID: String = "skip" + private[feature] val ERROR_INVALID: String = "error" + private[feature] val KEEP_INVALID: String = "keep" + private[feature] val supportedHandleInvalids: Array[String] = + Array(SKIP_INVALID, ERROR_INVALID, KEEP_INVALID) + + /** + * We require splits to be of length >= 3 and to be in strictly increasing order. + * No NaN split should be accepted. + */ + private[feature] def checkSplits(splits: Array[Double]): Boolean = { if (splits.length < 3) { false } else { var i = 0 val n = splits.length - 1 while (i < n) { - if (splits(i) >= splits(i + 1)) return false + if (splits(i) >= splits(i + 1) || splits(i).isNaN) return false i += 1 } - true + !splits(n).isNaN } } /** * Binary searching in several buckets to place each data point. + * @param splits array of split points + * @param feature data point + * @param keepInvalid NaN flag. + * Set "true" to make an extra bucket for NaN values; + * Set "false" to report an error for NaN values + * @return bucket for each data point * @throws SparkException if a feature is < splits.head or > splits.last */ - def binarySearchForBuckets(splits: Array[Double], feature: Double): Double = { - if (feature == splits.last) { + + private[feature] def binarySearchForBuckets( + splits: Array[Double], + feature: Double, + keepInvalid: Boolean): Double = { + if (feature.isNaN) { + if (keepInvalid) { + splits.length - 1 + } else { + throw new SparkException("Bucketizer encountered NaN value. To handle or skip NaNs," + + " try setting Bucketizer.handleInvalid.") + } + } else if (feature == splits.last) { splits.length - 2 } else { val idx = ju.Arrays.binarySearch(splits, feature) @@ -134,4 +203,7 @@ private[feature] object Bucketizer { } } } + + @Since("1.6.0") + override def load(path: String): Bucketizer = super.load(path) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index 5e4061fba549..16abc4949dea 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -17,16 +17,20 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since import org.apache.spark.ml._ import org.apache.spark.ml.attribute.{AttributeGroup, _} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.Identifiable -import org.apache.spark.ml.util.SchemaUtils +import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.feature.{ChiSqSelector => OldChiSqSelector} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} @@ -38,89 +42,239 @@ private[feature] trait ChiSqSelectorParams extends Params with HasFeaturesCol with HasOutputCol with HasLabelCol { /** - * Number of features that selector will select (ordered by statistic value descending). If the - * number of features is < numTopFeatures, then this will select all features. The default value - * of numTopFeatures is 50. + * Number of features that selector will select, ordered by ascending p-value. If the + * number of features is less than numTopFeatures, then this will select all features. + * Only applicable when selectorType = "numTopFeatures". + * The default value of numTopFeatures is 50. + * * @group param */ + @Since("1.6.0") final val numTopFeatures = new IntParam(this, "numTopFeatures", - "Number of features that selector will select, ordered by statistics value descending. If the" + + "Number of features that selector will select, ordered by ascending p-value. If the" + " number of features is < numTopFeatures, then this will select all features.", ParamValidators.gtEq(1)) setDefault(numTopFeatures -> 50) /** @group getParam */ + @Since("1.6.0") def getNumTopFeatures: Int = $(numTopFeatures) + + /** + * Percentile of features that selector will select, ordered by statistics value descending. + * Only applicable when selectorType = "percentile". + * Default value is 0.1. + * @group param + */ + @Since("2.1.0") + final val percentile = new DoubleParam(this, "percentile", + "Percentile of features that selector will select, ordered by ascending p-value.", + ParamValidators.inRange(0, 1)) + setDefault(percentile -> 0.1) + + /** @group getParam */ + @Since("2.1.0") + def getPercentile: Double = $(percentile) + + /** + * The highest p-value for features to be kept. + * Only applicable when selectorType = "fpr". + * Default value is 0.05. + * @group param + */ + @Since("2.1.0") + final val fpr = new DoubleParam(this, "fpr", "The highest p-value for features to be kept.", + ParamValidators.inRange(0, 1)) + setDefault(fpr -> 0.05) + + /** @group getParam */ + @Since("2.1.0") + def getFpr: Double = $(fpr) + + /** + * The upper bound of the expected false discovery rate. + * Only applicable when selectorType = "fdr". + * Default value is 0.05. + * @group param + */ + @Since("2.2.0") + final val fdr = new DoubleParam(this, "fdr", + "The upper bound of the expected false discovery rate.", ParamValidators.inRange(0, 1)) + setDefault(fdr -> 0.05) + + /** @group getParam */ + def getFdr: Double = $(fdr) + + /** + * The upper bound of the expected family-wise error rate. + * Only applicable when selectorType = "fwe". + * Default value is 0.05. + * @group param + */ + @Since("2.2.0") + final val fwe = new DoubleParam(this, "fwe", + "The upper bound of the expected family-wise error rate.", ParamValidators.inRange(0, 1)) + setDefault(fwe -> 0.05) + + /** @group getParam */ + def getFwe: Double = $(fwe) + + /** + * The selector type of the ChisqSelector. + * Supported options: "numTopFeatures" (default), "percentile", "fpr", "fdr", "fwe". + * @group param + */ + @Since("2.1.0") + final val selectorType = new Param[String](this, "selectorType", + "The selector type of the ChisqSelector. " + + "Supported options: " + OldChiSqSelector.supportedSelectorTypes.mkString(", "), + ParamValidators.inArray[String](OldChiSqSelector.supportedSelectorTypes)) + setDefault(selectorType -> OldChiSqSelector.NumTopFeatures) + + /** @group getParam */ + @Since("2.1.0") + def getSelectorType: String = $(selectorType) } /** - * :: Experimental :: * Chi-Squared feature selection, which selects categorical features to use for predicting a * categorical label. + * The selector supports different selection methods: `numTopFeatures`, `percentile`, `fpr`, + * `fdr`, `fwe`. + * - `numTopFeatures` chooses a fixed number of top features according to a chi-squared test. + * - `percentile` is similar but chooses a fraction of all features instead of a fixed number. + * - `fpr` chooses all features whose p-value are below a threshold, thus controlling the false + * positive rate of selection. + * - `fdr` uses the [Benjamini-Hochberg procedure] + * (https://en.wikipedia.org/wiki/False_discovery_rate#Benjamini.E2.80.93Hochberg_procedure) + * to choose all features whose false discovery rate is below a threshold. + * - `fwe` chooses all features whose p-values are below a threshold. The threshold is scaled by + * 1/numFeatures, thus controlling the family-wise error rate of selection. + * By default, the selection method is `numTopFeatures`, with the default number of top features + * set to 50. */ -@Experimental -final class ChiSqSelector(override val uid: String) - extends Estimator[ChiSqSelectorModel] with ChiSqSelectorParams { +@Since("1.6.0") +final class ChiSqSelector @Since("1.6.0") (@Since("1.6.0") override val uid: String) + extends Estimator[ChiSqSelectorModel] with ChiSqSelectorParams with DefaultParamsWritable { + @Since("1.6.0") def this() = this(Identifiable.randomUID("chiSqSelector")) /** @group setParam */ + @Since("1.6.0") def setNumTopFeatures(value: Int): this.type = set(numTopFeatures, value) /** @group setParam */ + @Since("2.1.0") + def setPercentile(value: Double): this.type = set(percentile, value) + + /** @group setParam */ + @Since("2.1.0") + def setFpr(value: Double): this.type = set(fpr, value) + + /** @group setParam */ + @Since("2.2.0") + def setFdr(value: Double): this.type = set(fdr, value) + + /** @group setParam */ + @Since("2.2.0") + def setFwe(value: Double): this.type = set(fwe, value) + + /** @group setParam */ + @Since("2.1.0") + def setSelectorType(value: String): this.type = set(selectorType, value) + + /** @group setParam */ + @Since("1.6.0") def setFeaturesCol(value: String): this.type = set(featuresCol, value) /** @group setParam */ + @Since("1.6.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.6.0") def setLabelCol(value: String): this.type = set(labelCol, value) - override def fit(dataset: DataFrame): ChiSqSelectorModel = { + @Since("2.0.0") + override def fit(dataset: Dataset[_]): ChiSqSelectorModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(labelCol), $(featuresCol)).map { - case Row(label: Double, features: Vector) => - LabeledPoint(label, features) - } - val chiSqSelector = new feature.ChiSqSelector($(numTopFeatures)).fit(input) - copyValues(new ChiSqSelectorModel(uid, chiSqSelector).setParent(this)) + val input: RDD[OldLabeledPoint] = + dataset.select(col($(labelCol)).cast(DoubleType), col($(featuresCol))).rdd.map { + case Row(label: Double, features: Vector) => + OldLabeledPoint(label, OldVectors.fromML(features)) + } + val selector = new feature.ChiSqSelector() + .setSelectorType($(selectorType)) + .setNumTopFeatures($(numTopFeatures)) + .setPercentile($(percentile)) + .setFpr($(fpr)) + .setFdr($(fdr)) + .setFwe($(fwe)) + val model = selector.fit(input) + copyValues(new ChiSqSelectorModel(uid, model).setParent(this)) } + @Since("1.6.0") override def transformSchema(schema: StructType): StructType = { + val otherPairs = OldChiSqSelector.supportedSelectorTypes.filter(_ != $(selectorType)) + otherPairs.foreach { paramName: String => + if (isSet(getParam(paramName))) { + logWarning(s"Param $paramName will take no effect when selector type = ${$(selectorType)}.") + } + } SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) - SchemaUtils.checkColumnType(schema, $(labelCol), DoubleType) + SchemaUtils.checkNumericType(schema, $(labelCol)) SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT) } + @Since("1.6.0") override def copy(extra: ParamMap): ChiSqSelector = defaultCopy(extra) } +@Since("1.6.0") +object ChiSqSelector extends DefaultParamsReadable[ChiSqSelector] { + + @Since("1.6.0") + override def load(path: String): ChiSqSelector = super.load(path) +} + /** - * :: Experimental :: * Model fitted by [[ChiSqSelector]]. */ -@Experimental +@Since("1.6.0") final class ChiSqSelectorModel private[ml] ( - override val uid: String, + @Since("1.6.0") override val uid: String, private val chiSqSelector: feature.ChiSqSelectorModel) - extends Model[ChiSqSelectorModel] with ChiSqSelectorParams { + extends Model[ChiSqSelectorModel] with ChiSqSelectorParams with MLWritable { + + import ChiSqSelectorModel._ + + /** list of indices to select (filter). */ + @Since("1.6.0") + val selectedFeatures: Array[Int] = chiSqSelector.selectedFeatures /** @group setParam */ + @Since("1.6.0") def setFeaturesCol(value: String): this.type = set(featuresCol, value) /** @group setParam */ + @Since("1.6.0") def setOutputCol(value: String): this.type = set(outputCol, value) - /** @group setParam */ - def setLabelCol(value: String): this.type = set(labelCol, value) - - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { val transformedSchema = transformSchema(dataset.schema, logging = true) val newField = transformedSchema.last - val selector = udf { chiSqSelector.transform _ } + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer: Vector => Vector = v => chiSqSelector.transform(OldVectors.fromML(v)).asML + + val selector = udf(transformer) dataset.withColumn($(outputCol), selector(col($(featuresCol))), newField.metadata) } + @Since("1.6.0") override def transformSchema(schema: StructType): StructType = { SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) val newField = prepOutputField(schema) @@ -143,8 +297,51 @@ final class ChiSqSelectorModel private[ml] ( newAttributeGroup.toStructField() } + @Since("1.6.0") override def copy(extra: ParamMap): ChiSqSelectorModel = { val copied = new ChiSqSelectorModel(uid, chiSqSelector) copyValues(copied, extra).setParent(parent) } + + @Since("1.6.0") + override def write: MLWriter = new ChiSqSelectorModelWriter(this) +} + +@Since("1.6.0") +object ChiSqSelectorModel extends MLReadable[ChiSqSelectorModel] { + + private[ChiSqSelectorModel] + class ChiSqSelectorModelWriter(instance: ChiSqSelectorModel) extends MLWriter { + + private case class Data(selectedFeatures: Seq[Int]) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.selectedFeatures.toSeq) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class ChiSqSelectorModelReader extends MLReader[ChiSqSelectorModel] { + + private val className = classOf[ChiSqSelectorModel].getName + + override def load(path: String): ChiSqSelectorModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath).select("selectedFeatures").head() + val selectedFeatures = data.getAs[Seq[Int]](0).toArray + val oldModel = new feature.ChiSqSelectorModel(selectedFeatures) + val model = new ChiSqSelectorModel(metadata.uid, oldModel) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } + + @Since("1.6.0") + override def read: MLReader[ChiSqSelectorModel] = new ChiSqSelectorModelReader + + @Since("1.6.0") + override def load(path: String): ChiSqSelectorModel = super.load(path) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala index 49028e4b8506..1ebe29703bc4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizer.scala @@ -16,17 +16,19 @@ */ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since import org.apache.spark.broadcast.Broadcast +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} -import org.apache.spark.ml.util.{Identifiable, SchemaUtils} -import org.apache.spark.ml.{Estimator, Model} -import org.apache.spark.mllib.linalg.{VectorUDT, Vectors} +import org.apache.spark.ml.util._ import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.DataFrame import org.apache.spark.util.collection.OpenHashMap /** @@ -51,10 +53,11 @@ private[feature] trait CountVectorizerParams extends Params with HasInputCol wit /** * Specifies the minimum number of different documents a term must appear in to be included * in the vocabulary. - * If this is an integer >= 1, this specifies the number of documents the term must appear in; - * if this is a double in [0,1), then this specifies the fraction of documents. + * If this is an integer greater than or equal to 1, this specifies the number of documents + * the term must appear in; if this is a double in [0,1), then this specifies the fraction of + * documents. * - * Default: 1 + * Default: 1.0 * @group param */ val minDF: DoubleParam = new DoubleParam(this, "minDF", "Specifies the minimum number of" + @@ -68,22 +71,23 @@ private[feature] trait CountVectorizerParams extends Params with HasInputCol wit /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(inputCol), new ArrayType(StringType, true)) + val typeCandidates = List(new ArrayType(StringType, true), new ArrayType(StringType, false)) + SchemaUtils.checkColumnTypes(schema, $(inputCol), typeCandidates) SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT) } /** * Filter to ignore rare words in a document. For each document, terms with * frequency/count less than the given threshold are ignored. - * If this is an integer >= 1, then this specifies a count (of times the term must appear - * in the document); + * If this is an integer greater than or equal to 1, then this specifies a count (of times the + * term must appear in the document); * if this is a double in [0,1), then this specifies a fraction (out of the document's token * count). * * Note that the parameter is only used in transform of [[CountVectorizerModel]] and does not * affect fitting. * - * Default: 1 + * Default: 1.0 * @group param */ val minTF: DoubleParam = new DoubleParam(this, "minTF", "Filter to ignore rare words in" + @@ -93,43 +97,64 @@ private[feature] trait CountVectorizerParams extends Params with HasInputCol wit " of the document's token count). Note that the parameter is only used in transform of" + " CountVectorizerModel and does not affect fitting.", ParamValidators.gtEq(0.0)) - setDefault(minTF -> 1) - /** @group getParam */ def getMinTF: Double = $(minTF) + + /** + * Binary toggle to control the output vector values. + * If True, all nonzero counts (after minTF filter applied) are set to 1. This is useful for + * discrete probabilistic models that model binary events rather than integer counts. + * Default: false + * @group param + */ + val binary: BooleanParam = + new BooleanParam(this, "binary", "If True, all non zero counts are set to 1.") + + /** @group getParam */ + def getBinary: Boolean = $(binary) + + setDefault(vocabSize -> (1 << 18), minDF -> 1.0, minTF -> 1.0, binary -> false) } /** - * :: Experimental :: * Extracts a vocabulary from document collections and generates a [[CountVectorizerModel]]. */ -@Experimental -class CountVectorizer(override val uid: String) - extends Estimator[CountVectorizerModel] with CountVectorizerParams { +@Since("1.5.0") +class CountVectorizer @Since("1.5.0") (@Since("1.5.0") override val uid: String) + extends Estimator[CountVectorizerModel] with CountVectorizerParams with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("cntVec")) /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.5.0") def setVocabSize(value: Int): this.type = set(vocabSize, value) /** @group setParam */ + @Since("1.5.0") def setMinDF(value: Double): this.type = set(minDF, value) /** @group setParam */ + @Since("1.5.0") def setMinTF(value: Double): this.type = set(minTF, value) - setDefault(vocabSize -> (1 << 18), minDF -> 1) + /** @group setParam */ + @Since("2.0.0") + def setBinary(value: Boolean): this.type = set(binary, value) - override def fit(dataset: DataFrame): CountVectorizerModel = { + @Since("2.0.0") + override def fit(dataset: Dataset[_]): CountVectorizerModel = { transformSchema(dataset.schema, logging = true) val vocSize = $(vocabSize) - val input = dataset.select($(inputCol)).map(_.getAs[Seq[String]](0)) + val input = dataset.select($(inputCol)).rdd.map(_.getAs[Seq[String]](0)) val minDf = if ($(minDF) >= 1.0) { $(minDF) } else { @@ -149,58 +174,74 @@ class CountVectorizer(override val uid: String) (word, count) }.cache() val fullVocabSize = wordCounts.count() - val vocab: Array[String] = { - val tmpSortedWC: Array[(String, Long)] = if (fullVocabSize <= vocSize) { - // Use all terms - wordCounts.collect().sortBy(-_._2) - } else { - // Sort terms to select vocab - wordCounts.sortBy(_._2, ascending = false).take(vocSize) - } - tmpSortedWC.map(_._1) - } + + val vocab = wordCounts + .top(math.min(fullVocabSize, vocSize).toInt)(Ordering.by(_._2)) + .map(_._1) require(vocab.length > 0, "The vocabulary size should be > 0. Lower minDF as necessary.") copyValues(new CountVectorizerModel(uid, vocab).setParent(this)) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.5.0") override def copy(extra: ParamMap): CountVectorizer = defaultCopy(extra) } +@Since("1.6.0") +object CountVectorizer extends DefaultParamsReadable[CountVectorizer] { + + @Since("1.6.0") + override def load(path: String): CountVectorizer = super.load(path) +} + /** - * :: Experimental :: * Converts a text document to a sparse vector of token counts. * @param vocabulary An Array over terms. Only the terms in the vocabulary will be counted. */ -@Experimental -class CountVectorizerModel(override val uid: String, val vocabulary: Array[String]) - extends Model[CountVectorizerModel] with CountVectorizerParams { +@Since("1.5.0") +class CountVectorizerModel( + @Since("1.5.0") override val uid: String, + @Since("1.5.0") val vocabulary: Array[String]) + extends Model[CountVectorizerModel] with CountVectorizerParams with MLWritable { + + import CountVectorizerModel._ + @Since("1.5.0") def this(vocabulary: Array[String]) = { this(Identifiable.randomUID("cntVecModel"), vocabulary) set(vocabSize, vocabulary.length) } /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.5.0") def setMinTF(value: Double): this.type = set(minTF, value) + /** @group setParam */ + @Since("2.0.0") + def setBinary(value: Boolean): this.type = set(binary, value) + /** Dictionary created from [[vocabulary]] and its indices, broadcast once for [[transform()]] */ private var broadcastDict: Option[Broadcast[Map[String, Int]]] = None - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) if (broadcastDict.isEmpty) { val dict = vocabulary.zipWithIndex.toMap - broadcastDict = Some(dataset.sqlContext.sparkContext.broadcast(dict)) + broadcastDict = Some(dataset.sparkSession.sparkContext.broadcast(dict)) } val dictBr = broadcastDict.get val minTf = $(minTF) @@ -214,22 +255,69 @@ class CountVectorizerModel(override val uid: String, val vocabulary: Array[Strin } tokenCount += 1 } - val effectiveMinTF = if (minTf >= 1.0) { - minTf + val effectiveMinTF = if (minTf >= 1.0) minTf else tokenCount * minTf + val effectiveCounts = if ($(binary)) { + termCounts.filter(_._2 >= effectiveMinTF).map(p => (p._1, 1.0)).toSeq } else { - tokenCount * minTf + termCounts.filter(_._2 >= effectiveMinTF).toSeq } - Vectors.sparse(dictBr.value.size, termCounts.filter(_._2 >= effectiveMinTF).toSeq) + + Vectors.sparse(dictBr.value.size, effectiveCounts) } dataset.withColumn($(outputCol), vectorizer(col($(inputCol)))) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.5.0") override def copy(extra: ParamMap): CountVectorizerModel = { val copied = new CountVectorizerModel(uid, vocabulary).setParent(parent) copyValues(copied, extra) } + + @Since("1.6.0") + override def write: MLWriter = new CountVectorizerModelWriter(this) +} + +@Since("1.6.0") +object CountVectorizerModel extends MLReadable[CountVectorizerModel] { + + private[CountVectorizerModel] + class CountVectorizerModelWriter(instance: CountVectorizerModel) extends MLWriter { + + private case class Data(vocabulary: Seq[String]) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.vocabulary) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class CountVectorizerModelReader extends MLReader[CountVectorizerModel] { + + private val className = classOf[CountVectorizerModel].getName + + override def load(path: String): CountVectorizerModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + .select("vocabulary") + .head() + val vocabulary = data.getAs[Seq[String]](0).toArray + val model = new CountVectorizerModel(metadata.uid, vocabulary) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } + + @Since("1.6.0") + override def read: MLReader[CountVectorizerModel] = new CountVectorizerModelReader + + @Since("1.6.0") + override def load(path: String): CountVectorizerModel = super.load(path) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index 228347635c92..682787a83011 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -19,26 +19,27 @@ package org.apache.spark.ml.feature import edu.emory.mathcs.jtransforms.dct._ -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.BooleanParam -import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg.{Vector, VectorUDT, Vectors} +import org.apache.spark.ml.util._ import org.apache.spark.sql.types.DataType /** - * :: Experimental :: * A feature transformer that takes the 1D discrete cosine transform of a real vector. No zero * padding is performed on the input vector. * It returns a real vector of the same length representing the DCT. The return vector is scaled * such that the transform matrix is unitary (aka scaled DCT-II). * - * More information on [[https://en.wikipedia.org/wiki/Discrete_cosine_transform#DCT-II Wikipedia]]. + * More information on + * DCT-II in Discrete cosine transform (Wikipedia). */ -@Experimental -class DCT(override val uid: String) - extends UnaryTransformer[Vector, Vector, DCT] { +@Since("1.5.0") +class DCT @Since("1.5.0") (@Since("1.5.0") override val uid: String) + extends UnaryTransformer[Vector, Vector, DCT] with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("dct")) /** @@ -46,13 +47,16 @@ class DCT(override val uid: String) * Default: false * @group param */ + @Since("1.5.0") def inverse: BooleanParam = new BooleanParam( this, "inverse", "Set transformer to perform inverse DCT") /** @group setParam */ + @Since("1.5.0") def setInverse(value: Boolean): this.type = set(inverse, value) /** @group getParam */ + @Since("1.5.0") def getInverse: Boolean = $(inverse) setDefault(inverse -> false) @@ -70,3 +74,10 @@ class DCT(override val uid: String) override protected def outputDataType: DataType = new VectorUDT } + +@Since("1.6.0") +object DCT extends DefaultParamsReadable[DCT] { + + @Since("1.6.0") + override def load(path: String): DCT = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index a359cb8f37ec..f860b3a787b4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -17,43 +17,54 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer -import org.apache.spark.ml.param.{ParamMap, Param} -import org.apache.spark.ml.util.Identifiable +import org.apache.spark.ml.linalg.{Vector, VectorUDT} +import org.apache.spark.ml.param.Param +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable} import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.sql.types.DataType /** - * :: Experimental :: * Outputs the Hadamard product (i.e., the element-wise product) of each input vector with a * provided "weight" vector. In other words, it scales each column of the dataset by a scalar * multiplier. */ -@Experimental -class ElementwiseProduct(override val uid: String) - extends UnaryTransformer[Vector, Vector, ElementwiseProduct] { +@Since("1.4.0") +class ElementwiseProduct @Since("1.4.0") (@Since("1.4.0") override val uid: String) + extends UnaryTransformer[Vector, Vector, ElementwiseProduct] with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("elemProd")) /** - * the vector to multiply with input vectors - * @group param - */ + * the vector to multiply with input vectors + * @group param + */ + @Since("2.0.0") val scalingVec: Param[Vector] = new Param(this, "scalingVec", "vector for hadamard product") /** @group setParam */ + @Since("2.0.0") def setScalingVec(value: Vector): this.type = set(scalingVec, value) /** @group getParam */ + @Since("2.0.0") def getScalingVec: Vector = getOrDefault(scalingVec) override protected def createTransformFunc: Vector => Vector = { require(params.contains(scalingVec), s"transformation requires a weight vector") val elemScaler = new feature.ElementwiseProduct($(scalingVec)) - elemScaler.transform + v => elemScaler.transform(v) } override protected def outputDataType: DataType = new VectorUDT() } + +@Since("2.0.0") +object ElementwiseProduct extends DefaultParamsReadable[ElementwiseProduct] { + + @Since("2.0.0") + override def load(path: String): ElementwiseProduct = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala new file mode 100644 index 000000000000..4615daed20fb --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.Transformer +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators} +import org.apache.spark.ml.param.shared.{HasInputCols, HasOutputCol} +import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} +import org.apache.spark.mllib.feature.{HashingTF => OldHashingTF} +import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils +import org.apache.spark.util.collection.OpenHashMap + +/** + * Feature hashing projects a set of categorical or numerical features into a feature vector of + * specified dimension (typically substantially smaller than that of the original feature + * space). This is done using the hashing trick (https://en.wikipedia.org/wiki/Feature_hashing) + * to map features to indices in the feature vector. + * + * The [[FeatureHasher]] transformer operates on multiple columns. Each column may contain either + * numeric or categorical features. Behavior and handling of column data types is as follows: + * -Numeric columns: For numeric features, the hash value of the column name is used to map the + * feature value to its index in the feature vector. Numeric features are never + * treated as categorical, even when they are integers. You must explicitly + * convert numeric columns containing categorical features to strings first. + * -String columns: For categorical features, the hash value of the string "column_name=value" + * is used to map to the vector index, with an indicator value of `1.0`. + * Thus, categorical features are "one-hot" encoded + * (similarly to using [[OneHotEncoder]] with `dropLast=false`). + * -Boolean columns: Boolean values are treated in the same way as string columns. That is, + * boolean features are represented as "column_name=true" or "column_name=false", + * with an indicator value of `1.0`. + * + * Null (missing) values are ignored (implicitly zero in the resulting feature vector). + * + * The hash function used here is also the MurmurHash 3 used in [[HashingTF]]. Since a simple modulo + * on the hashed value is used to determine the vector index, it is advisable to use a power of two + * as the numFeatures parameter; otherwise the features will not be mapped evenly to the vector + * indices. + * + * {{{ + * val df = Seq( + * (2.0, true, "1", "foo"), + * (3.0, false, "2", "bar") + * ).toDF("real", "bool", "stringNum", "string") + * + * val hasher = new FeatureHasher() + * .setInputCols("real", "bool", "stringNum", "string") + * .setOutputCol("features") + * + * hasher.transform(df).show(false) + * + * +----+-----+---------+------+------------------------------------------------------+ + * |real|bool |stringNum|string|features | + * +----+-----+---------+------+------------------------------------------------------+ + * |2.0 |true |1 |foo |(262144,[51871,63643,174475,253195],[1.0,1.0,2.0,1.0])| + * |3.0 |false|2 |bar |(262144,[6031,80619,140467,174475],[1.0,1.0,1.0,3.0]) | + * +----+-----+---------+------+------------------------------------------------------+ + * }}} + */ +@Experimental +@Since("2.3.0") +class FeatureHasher(@Since("2.3.0") override val uid: String) extends Transformer + with HasInputCols with HasOutputCol with DefaultParamsWritable { + + @Since("2.3.0") + def this() = this(Identifiable.randomUID("featureHasher")) + + /** + * Number of features. Should be greater than 0. + * (default = 2^18^) + * @group param + */ + @Since("2.3.0") + val numFeatures = new IntParam(this, "numFeatures", "number of features (> 0)", + ParamValidators.gt(0)) + + setDefault(numFeatures -> (1 << 18)) + + /** @group getParam */ + @Since("2.3.0") + def getNumFeatures: Int = $(numFeatures) + + /** @group setParam */ + @Since("2.3.0") + def setNumFeatures(value: Int): this.type = set(numFeatures, value) + + /** @group setParam */ + @Since("2.3.0") + def setInputCols(values: String*): this.type = setInputCols(values.toArray) + + /** @group setParam */ + @Since("2.3.0") + def setInputCols(value: Array[String]): this.type = set(inputCols, value) + + /** @group setParam */ + @Since("2.3.0") + def setOutputCol(value: String): this.type = set(outputCol, value) + + @Since("2.3.0") + override def transform(dataset: Dataset[_]): DataFrame = { + val hashFunc: Any => Int = OldHashingTF.murmur3Hash + val n = $(numFeatures) + val localInputCols = $(inputCols) + + val outputSchema = transformSchema(dataset.schema) + val realFields = outputSchema.fields.filter { f => + f.dataType.isInstanceOf[NumericType] + }.map(_.name).toSet + + def getDouble(x: Any): Double = { + x match { + case n: java.lang.Number => + n.doubleValue() + case other => + // will throw ClassCastException if it cannot be cast, as would row.getDouble + other.asInstanceOf[Double] + } + } + + val hashFeatures = udf { row: Row => + val map = new OpenHashMap[Int, Double]() + localInputCols.foreach { colName => + val fieldIndex = row.fieldIndex(colName) + if (!row.isNullAt(fieldIndex)) { + val (rawIdx, value) = if (realFields(colName)) { + // numeric values are kept as is, with vector index based on hash of "column_name" + val value = getDouble(row.get(fieldIndex)) + val hash = hashFunc(colName) + (hash, value) + } else { + // string and boolean values are treated as categorical, with an indicator value of 1.0 + // and vector index based on hash of "column_name=value" + val value = row.get(fieldIndex).toString + val fieldName = s"$colName=$value" + val hash = hashFunc(fieldName) + (hash, 1.0) + } + val idx = Utils.nonNegativeMod(rawIdx, n) + map.changeValue(idx, value, v => v + value) + } + } + Vectors.sparse(n, map.toSeq) + } + + val metadata = outputSchema($(outputCol)).metadata + dataset.select( + col("*"), + hashFeatures(struct($(inputCols).map(col): _*)).as($(outputCol), metadata)) + } + + @Since("2.3.0") + override def copy(extra: ParamMap): FeatureHasher = defaultCopy(extra) + + @Since("2.3.0") + override def transformSchema(schema: StructType): StructType = { + val fields = schema($(inputCols).toSet) + fields.foreach { fieldSchema => + val dataType = fieldSchema.dataType + val fieldName = fieldSchema.name + require(dataType.isInstanceOf[NumericType] || + dataType.isInstanceOf[StringType] || + dataType.isInstanceOf[BooleanType], + s"FeatureHasher requires columns to be of NumericType, BooleanType or StringType. " + + s"Column $fieldName was $dataType") + } + val attrGroup = new AttributeGroup($(outputCol), $(numFeatures)) + SchemaUtils.appendColumn(schema, attrGroup.toStructField()) + } +} + +@Since("2.3.0") +object FeatureHasher extends DefaultParamsReadable[FeatureHasher] { + + @Since("2.3.0") + override def load(path: String): FeatureHasher = super.load(path) +} 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 319d23e46cef..db432b6fefaf 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 @@ -17,56 +17,90 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.AttributeGroup -import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators} +import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} -import org.apache.spark.ml.util.{Identifiable, SchemaUtils} +import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{ArrayType, StructType} /** - * :: Experimental :: * Maps a sequence of terms to their term frequencies using the hashing trick. + * Currently we use Austin Appleby's MurmurHash 3 algorithm (MurmurHash3_x86_32) + * to calculate the hash code value for the term object. + * Since a simple modulo is used to transform the hash function to a column index, + * it is advisable to use a power of two as the numFeatures parameter; + * otherwise the features will not be mapped evenly to the columns. */ -@Experimental -class HashingTF(override val uid: String) extends Transformer with HasInputCol with HasOutputCol { +@Since("1.2.0") +class HashingTF @Since("1.4.0") (@Since("1.4.0") override val uid: String) + extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + @Since("1.2.0") def this() = this(Identifiable.randomUID("hashingTF")) /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** - * Number of features. Should be > 0. + * Number of features. Should be greater than 0. * (default = 2^18^) * @group param */ + @Since("1.2.0") val numFeatures = new IntParam(this, "numFeatures", "number of features (> 0)", ParamValidators.gt(0)) - setDefault(numFeatures -> (1 << 18)) + /** + * Binary toggle to control term frequency counts. + * If true, all non-zero counts are set to 1. This is useful for discrete probabilistic + * models that model binary events rather than integer counts. + * (default = false) + * @group param + */ + @Since("2.0.0") + val binary = new BooleanParam(this, "binary", "If true, all non zero counts are set to 1. " + + "This is useful for discrete probabilistic models that model binary events rather " + + "than integer counts") + + setDefault(numFeatures -> (1 << 18), binary -> false) /** @group getParam */ + @Since("1.2.0") def getNumFeatures: Int = $(numFeatures) /** @group setParam */ + @Since("1.2.0") def setNumFeatures(value: Int): this.type = set(numFeatures, value) - override def transform(dataset: DataFrame): DataFrame = { + /** @group getParam */ + @Since("2.0.0") + def getBinary: Boolean = $(binary) + + /** @group setParam */ + @Since("2.0.0") + def setBinary(value: Boolean): this.type = set(binary, value) + + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { val outputSchema = transformSchema(dataset.schema) - val hashingTF = new feature.HashingTF($(numFeatures)) - val t = udf { terms: Seq[_] => hashingTF.transform(terms) } + val hashingTF = new feature.HashingTF($(numFeatures)).setBinary($(binary)) + // TODO: Make the hashingTF.transform natively in ml framework to avoid extra conversion. + val t = udf { terms: Seq[_] => hashingTF.transform(terms).asML } val metadata = outputSchema($(outputCol)).metadata dataset.select(col("*"), t(col($(inputCol))).as($(outputCol), metadata)) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { val inputType = schema($(inputCol)).dataType require(inputType.isInstanceOf[ArrayType], @@ -75,5 +109,13 @@ class HashingTF(override val uid: String) extends Transformer with HasInputCol w SchemaUtils.appendColumn(schema, attrGroup.toStructField()) } + @Since("1.4.1") override def copy(extra: ParamMap): HashingTF = defaultCopy(extra) } + +@Since("1.6.0") +object HashingTF extends DefaultParamsReadable[HashingTF] { + + @Since("1.6.0") + override def load(path: String): HashingTF = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index 4c36df75d8aa..46a0730f5ddb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -17,13 +17,18 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since import org.apache.spark.ml._ +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.{Identifiable, SchemaUtils} +import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType @@ -34,12 +39,13 @@ import org.apache.spark.sql.types.StructType private[feature] trait IDFBase extends Params with HasInputCol with HasOutputCol { /** - * The minimum of documents in which a term should appear. + * The minimum number of documents in which a term should appear. * Default: 0 * @group param */ final val minDocFreq = new IntParam( - this, "minDocFreq", "minimum of documents in which a term should appear for filtering") + this, "minDocFreq", "minimum number of documents in which a term should appear for filtering" + + " (>= 0)", ParamValidators.gtEq(0)) setDefault(minDocFreq -> 0) @@ -56,65 +62,134 @@ private[feature] trait IDFBase extends Params with HasInputCol with HasOutputCol } /** - * :: Experimental :: * Compute the Inverse Document Frequency (IDF) given a collection of documents. */ -@Experimental -final class IDF(override val uid: String) extends Estimator[IDFModel] with IDFBase { +@Since("1.4.0") +final class IDF @Since("1.4.0") (@Since("1.4.0") override val uid: String) + extends Estimator[IDFModel] with IDFBase with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("idf")) /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.4.0") def setMinDocFreq(value: Int): this.type = set(minDocFreq, value) - override def fit(dataset: DataFrame): IDFModel = { + @Since("2.0.0") + override def fit(dataset: Dataset[_]): IDFModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val idf = new feature.IDF($(minDocFreq)).fit(input) copyValues(new IDFModel(uid, idf).setParent(this)) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.4.1") override def copy(extra: ParamMap): IDF = defaultCopy(extra) } +@Since("1.6.0") +object IDF extends DefaultParamsReadable[IDF] { + + @Since("1.6.0") + override def load(path: String): IDF = super.load(path) +} + /** - * :: Experimental :: * Model fitted by [[IDF]]. */ -@Experimental +@Since("1.4.0") class IDFModel private[ml] ( - override val uid: String, + @Since("1.4.0") override val uid: String, idfModel: feature.IDFModel) - extends Model[IDFModel] with IDFBase { + extends Model[IDFModel] with IDFBase with MLWritable { + + import IDFModel._ /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val idf = udf { vec: Vector => idfModel.transform(vec) } + // TODO: Make the idfModel.transform natively in ml framework to avoid extra conversion. + val idf = udf { vec: Vector => idfModel.transform(OldVectors.fromML(vec)).asML } dataset.withColumn($(outputCol), idf(col($(inputCol)))) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.4.1") override def copy(extra: ParamMap): IDFModel = { val copied = new IDFModel(uid, idfModel) copyValues(copied, extra).setParent(parent) } + + /** Returns the IDF vector. */ + @Since("2.0.0") + def idf: Vector = idfModel.idf.asML + + @Since("1.6.0") + override def write: MLWriter = new IDFModelWriter(this) +} + +@Since("1.6.0") +object IDFModel extends MLReadable[IDFModel] { + + private[IDFModel] class IDFModelWriter(instance: IDFModel) extends MLWriter { + + private case class Data(idf: Vector) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.idf) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class IDFModelReader extends MLReader[IDFModel] { + + private val className = classOf[IDFModel].getName + + override def load(path: String): IDFModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + val Row(idf: Vector) = MLUtils.convertVectorColumnsToML(data, "idf") + .select("idf") + .head() + val model = new IDFModel(metadata.uid, new feature.IDFModel(OldVectors.fromML(idf))) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } + + @Since("1.6.0") + override def read: MLReader[IDFModel] = new IDFModelReader + + @Since("1.6.0") + override def load(path: String): IDFModel = super.load(path) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala new file mode 100644 index 000000000000..1f36eced3d08 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala @@ -0,0 +1,287 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared.HasInputCols +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +/** + * Params for [[Imputer]] and [[ImputerModel]]. + */ +private[feature] trait ImputerParams extends Params with HasInputCols { + + /** + * The imputation strategy. Currently only "mean" and "median" are supported. + * If "mean", then replace missing values using the mean value of the feature. + * If "median", then replace missing values using the approximate median value of the feature. + * Default: mean + * + * @group param + */ + final val strategy: Param[String] = new Param(this, "strategy", s"strategy for imputation. " + + s"If ${Imputer.mean}, then replace missing values using the mean value of the feature. " + + s"If ${Imputer.median}, then replace missing values using the median value of the feature.", + ParamValidators.inArray[String](Array(Imputer.mean, Imputer.median))) + + /** @group getParam */ + def getStrategy: String = $(strategy) + + /** + * The placeholder for the missing values. All occurrences of missingValue will be imputed. + * Note that null values are always treated as missing. + * Default: Double.NaN + * + * @group param + */ + final val missingValue: DoubleParam = new DoubleParam(this, "missingValue", + "The placeholder for the missing values. All occurrences of missingValue will be imputed") + + /** @group getParam */ + def getMissingValue: Double = $(missingValue) + + /** + * Param for output column names. + * @group param + */ + final val outputCols: StringArrayParam = new StringArrayParam(this, "outputCols", + "output column names") + + /** @group getParam */ + final def getOutputCols: Array[String] = $(outputCols) + + /** Validates and transforms the input schema. */ + protected def validateAndTransformSchema(schema: StructType): StructType = { + require($(inputCols).length == $(inputCols).distinct.length, s"inputCols contains" + + s" duplicates: (${$(inputCols).mkString(", ")})") + require($(outputCols).length == $(outputCols).distinct.length, s"outputCols contains" + + s" duplicates: (${$(outputCols).mkString(", ")})") + require($(inputCols).length == $(outputCols).length, s"inputCols(${$(inputCols).length})" + + s" and outputCols(${$(outputCols).length}) should have the same length") + val outputFields = $(inputCols).zip($(outputCols)).map { case (inputCol, outputCol) => + val inputField = schema(inputCol) + SchemaUtils.checkColumnTypes(schema, inputCol, Seq(DoubleType, FloatType)) + StructField(outputCol, inputField.dataType, inputField.nullable) + } + StructType(schema ++ outputFields) + } +} + +/** + * :: Experimental :: + * Imputation estimator for completing missing values, either using the mean or the median + * of the columns in which the missing values are located. The input columns should be of + * DoubleType or FloatType. Currently Imputer does not support categorical features + * (SPARK-15041) and possibly creates incorrect values for a categorical feature. + * + * Note that the mean/median value is computed after filtering out missing values. + * All Null values in the input columns are treated as missing, and so are also imputed. For + * computing median, DataFrameStatFunctions.approxQuantile is used with a relative error of 0.001. + */ +@Experimental +@Since("2.2.0") +class Imputer @Since("2.2.0") (@Since("2.2.0") override val uid: String) + extends Estimator[ImputerModel] with ImputerParams with DefaultParamsWritable { + + @Since("2.2.0") + def this() = this(Identifiable.randomUID("imputer")) + + /** @group setParam */ + @Since("2.2.0") + def setInputCols(value: Array[String]): this.type = set(inputCols, value) + + /** @group setParam */ + @Since("2.2.0") + def setOutputCols(value: Array[String]): this.type = set(outputCols, value) + + /** + * Imputation strategy. Available options are ["mean", "median"]. + * @group setParam + */ + @Since("2.2.0") + def setStrategy(value: String): this.type = set(strategy, value) + + /** @group setParam */ + @Since("2.2.0") + def setMissingValue(value: Double): this.type = set(missingValue, value) + + setDefault(strategy -> Imputer.mean, missingValue -> Double.NaN) + + override def fit(dataset: Dataset[_]): ImputerModel = { + transformSchema(dataset.schema, logging = true) + val spark = dataset.sparkSession + + val cols = $(inputCols).map { inputCol => + when(col(inputCol).equalTo($(missingValue)), null) + .when(col(inputCol).isNaN, null) + .otherwise(col(inputCol)) + .cast("double") + .as(inputCol) + } + + val results = $(strategy) match { + case Imputer.mean => + // Function avg will ignore null automatically. + // For a column only containing null, avg will return null. + val row = dataset.select(cols.map(avg): _*).head() + Array.range(0, $(inputCols).length).map { i => + if (row.isNullAt(i)) { + Double.NaN + } else { + row.getDouble(i) + } + } + + case Imputer.median => + // Function approxQuantile will ignore null automatically. + // For a column only containing null, approxQuantile will return an empty array. + dataset.select(cols: _*).stat.approxQuantile($(inputCols), Array(0.5), 0.001) + .map { array => + if (array.isEmpty) { + Double.NaN + } else { + array.head + } + } + } + + val emptyCols = $(inputCols).zip(results).filter(_._2.isNaN).map(_._1) + if (emptyCols.nonEmpty) { + throw new SparkException(s"surrogate cannot be computed. " + + s"All the values in ${emptyCols.mkString(",")} are Null, Nan or " + + s"missingValue(${$(missingValue)})") + } + + val rows = spark.sparkContext.parallelize(Seq(Row.fromSeq(results))) + val schema = StructType($(inputCols).map(col => StructField(col, DoubleType, nullable = false))) + val surrogateDF = spark.createDataFrame(rows, schema) + copyValues(new ImputerModel(uid, surrogateDF).setParent(this)) + } + + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } + + override def copy(extra: ParamMap): Imputer = defaultCopy(extra) +} + +@Since("2.2.0") +object Imputer extends DefaultParamsReadable[Imputer] { + + /** strategy names that Imputer currently supports. */ + private[feature] val mean = "mean" + private[feature] val median = "median" + + @Since("2.2.0") + override def load(path: String): Imputer = super.load(path) +} + +/** + * :: Experimental :: + * Model fitted by [[Imputer]]. + * + * @param surrogateDF a DataFrame containing inputCols and their corresponding surrogates, + * which are used to replace the missing values in the input DataFrame. + */ +@Experimental +@Since("2.2.0") +class ImputerModel private[ml] ( + @Since("2.2.0") override val uid: String, + @Since("2.2.0") val surrogateDF: DataFrame) + extends Model[ImputerModel] with ImputerParams with MLWritable { + + import ImputerModel._ + + /** @group setParam */ + def setInputCols(value: Array[String]): this.type = set(inputCols, value) + + /** @group setParam */ + def setOutputCols(value: Array[String]): this.type = set(outputCols, value) + + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) + var outputDF = dataset + val surrogates = surrogateDF.select($(inputCols).map(col): _*).head().toSeq + + $(inputCols).zip($(outputCols)).zip(surrogates).foreach { + case ((inputCol, outputCol), surrogate) => + val inputType = dataset.schema(inputCol).dataType + val ic = col(inputCol) + outputDF = outputDF.withColumn(outputCol, + when(ic.isNull, surrogate) + .when(ic === $(missingValue), surrogate) + .otherwise(ic) + .cast(inputType)) + } + outputDF.toDF() + } + + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } + + override def copy(extra: ParamMap): ImputerModel = { + val copied = new ImputerModel(uid, surrogateDF) + copyValues(copied, extra).setParent(parent) + } + + @Since("2.2.0") + override def write: MLWriter = new ImputerModelWriter(this) +} + + +@Since("2.2.0") +object ImputerModel extends MLReadable[ImputerModel] { + + private[ImputerModel] class ImputerModelWriter(instance: ImputerModel) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val dataPath = new Path(path, "data").toString + instance.surrogateDF.repartition(1).write.parquet(dataPath) + } + } + + private class ImputerReader extends MLReader[ImputerModel] { + + private val className = classOf[ImputerModel].getName + + override def load(path: String): ImputerModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val surrogateDF = sqlContext.read.parquet(dataPath) + val model = new ImputerModel(metadata.uid, surrogateDF) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } + + @Since("2.2.0") + override def read: MLReader[ImputerModel] = new ImputerReader + + @Since("2.2.0") + override def load(path: String): ImputerModel = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala index 12176757aee3..dd56fbbfa2b6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Instance.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector /** * Class that represents an instance of weighted data point with label and features. @@ -27,3 +27,24 @@ import org.apache.spark.mllib.linalg.Vector * @param features The vector of features for this data point. */ private[ml] case class Instance(label: Double, weight: Double, features: Vector) + +/** + * Case class that represents an instance of data point with + * label, weight, offset and features. + * This is mainly used in GeneralizedLinearRegression currently. + * + * @param label Label for this data point. + * @param weight The weight of this instance. + * @param offset The offset used for this data point. + * @param features The vector of features for this data point. + */ +private[ml] case class OffsetInstance( + label: Double, + weight: Double, + offset: Double, + features: Vector) { + + /** Converts to an [[Instance]] object by leaving out the offset. */ + def toInstance: Instance = Instance(label, weight, features) + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala index 37f7862476cf..4ff1d0ef356f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Interaction.scala @@ -20,19 +20,18 @@ package org.apache.spark.ml.feature import scala.collection.mutable.ArrayBuilder import org.apache.spark.SparkException -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Since +import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.Identifiable -import org.apache.spark.ml.Transformer -import org.apache.spark.mllib.linalg.{Vector, VectorUDT, Vectors} -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ /** - * :: Experimental :: * Implements the feature interaction transform. This transformer takes in Double and Vector type * columns and outputs a flattened vector of their feature interactions. To handle interaction, * we first one-hot encode any nominal features. Then, a vector of the feature cross-products is @@ -42,26 +41,34 @@ import org.apache.spark.sql.types._ * `Vector(6, 8)` if all input features were numeric. If the first feature was instead nominal * with four categories, the output would then be `Vector(0, 0, 0, 0, 3, 4, 0, 0)`. */ -@Experimental -class Interaction(override val uid: String) extends Transformer - with HasInputCols with HasOutputCol { +@Since("1.6.0") +class Interaction @Since("1.6.0") (@Since("1.6.0") override val uid: String) extends Transformer + with HasInputCols with HasOutputCol with DefaultParamsWritable { + @Since("1.6.0") def this() = this(Identifiable.randomUID("interaction")) /** @group setParam */ + @Since("1.6.0") def setInputCols(values: Array[String]): this.type = set(inputCols, values) /** @group setParam */ + @Since("1.6.0") def setOutputCol(value: String): this.type = set(outputCol, value) // optimistic schema; does not contain any ML attributes + @Since("1.6.0") override def transformSchema(schema: StructType): StructType = { - validateParams() + require(get(inputCols).isDefined, "Input cols must be defined first.") + require(get(outputCol).isDefined, "Output col must be defined first.") + require($(inputCols).length > 0, "Input cols must have non-zero length.") + require($(inputCols).distinct.length == $(inputCols).length, "Input cols must be distinct.") StructType(schema.fields :+ StructField($(outputCol), new VectorUDT, false)) } - override def transform(dataset: DataFrame): DataFrame = { - validateParams() + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) val inputFeatures = $(inputCols).map(c => dataset.schema(c)) val featureEncoders = getFeatureEncoders(inputFeatures) val featureAttrs = getFeatureAttrs(inputFeatures) @@ -129,7 +136,7 @@ class Interaction(override val uid: String) extends Transformer case _: VectorUDT => val attrs = AttributeGroup.fromStructField(f).attributes.getOrElse( throw new SparkException("Vector attributes must be defined for interaction.")) - attrs.map(getNumFeatures).toArray + attrs.map(getNumFeatures) } new FeatureEncoder(numFeatures) }.toArray @@ -208,14 +215,16 @@ class Interaction(override val uid: String) extends Transformer } } + @Since("1.6.0") override def copy(extra: ParamMap): Interaction = defaultCopy(extra) - override def validateParams(): Unit = { - require(get(inputCols).isDefined, "Input cols must be defined first.") - require(get(outputCol).isDefined, "Output col must be defined first.") - require($(inputCols).length > 0, "Input cols must have non-zero length.") - require($(inputCols).distinct.length == $(inputCols).length, "Input cols must be distinct.") - } +} + +@Since("1.6.0") +object Interaction extends DefaultParamsReadable[Interaction] { + + @Since("1.6.0") + override def load(path: String): Interaction = super.load(path) } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala new file mode 100644 index 000000000000..1c9f47a0b201 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LSH.scala @@ -0,0 +1,326 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import scala.util.Random + +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} +import org.apache.spark.ml.param.{IntParam, ParamValidators} +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.util._ +import org.apache.spark.sql._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +/** + * Params for [[LSH]]. + */ +private[ml] trait LSHParams extends HasInputCol with HasOutputCol { + /** + * Param for the number of hash tables used in LSH OR-amplification. + * + * LSH OR-amplification can be used to reduce the false negative rate. Higher values for this + * param lead to a reduced false negative rate, at the expense of added computational complexity. + * @group param + */ + final val numHashTables: IntParam = new IntParam(this, "numHashTables", "number of hash " + + "tables, where increasing number of hash tables lowers the false negative rate, and " + + "decreasing it improves the running performance", ParamValidators.gt(0)) + + /** @group getParam */ + final def getNumHashTables: Int = $(numHashTables) + + setDefault(numHashTables -> 1) + + /** + * Transform the Schema for LSH + * @param schema The schema of the input dataset without [[outputCol]]. + * @return A derived schema with [[outputCol]] added. + */ + protected[this] final def validateAndTransformSchema(schema: StructType): StructType = { + SchemaUtils.appendColumn(schema, $(outputCol), DataTypes.createArrayType(new VectorUDT)) + } +} + +/** + * Model produced by [[LSH]]. + */ +private[ml] abstract class LSHModel[T <: LSHModel[T]] + extends Model[T] with LSHParams with MLWritable { + self: T => + + /** + * The hash function of LSH, mapping an input feature vector to multiple hash vectors. + * @return The mapping of LSH function. + */ + protected[ml] val hashFunction: Vector => Array[Vector] + + /** + * Calculate the distance between two different keys using the distance metric corresponding + * to the hashFunction. + * @param x One input vector in the metric space. + * @param y One input vector in the metric space. + * @return The distance between x and y. + */ + protected[ml] def keyDistance(x: Vector, y: Vector): Double + + /** + * Calculate the distance between two different hash Vectors. + * + * @param x One of the hash vector. + * @param y Another hash vector. + * @return The distance between hash vectors x and y. + */ + protected[ml] def hashDistance(x: Seq[Vector], y: Seq[Vector]): Double + + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) + val transformUDF = udf(hashFunction, DataTypes.createArrayType(new VectorUDT)) + dataset.withColumn($(outputCol), transformUDF(dataset($(inputCol)))) + } + + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } + + // TODO: Fix the MultiProbe NN Search in SPARK-18454 + private[feature] def approxNearestNeighbors( + dataset: Dataset[_], + key: Vector, + numNearestNeighbors: Int, + singleProbe: Boolean, + distCol: String): Dataset[_] = { + require(numNearestNeighbors > 0, "The number of nearest neighbors cannot be less than 1") + // Get Hash Value of the key + val keyHash = hashFunction(key) + val modelDataset: DataFrame = if (!dataset.columns.contains($(outputCol))) { + transform(dataset) + } else { + dataset.toDF() + } + + val modelSubset = if (singleProbe) { + def sameBucket(x: Seq[Vector], y: Seq[Vector]): Boolean = { + x.zip(y).exists(tuple => tuple._1 == tuple._2) + } + + // In the origin dataset, find the hash value that hash the same bucket with the key + val sameBucketWithKeyUDF = udf((x: Seq[Vector]) => + sameBucket(x, keyHash), DataTypes.BooleanType) + + modelDataset.filter(sameBucketWithKeyUDF(col($(outputCol)))) + } else { + // In the origin dataset, find the hash value that is closest to the key + // Limit the use of hashDist since it's controversial + val hashDistUDF = udf((x: Seq[Vector]) => hashDistance(x, keyHash), DataTypes.DoubleType) + val hashDistCol = hashDistUDF(col($(outputCol))) + + // Compute threshold to get exact k elements. + // TODO: SPARK-18409: Use approxQuantile to get the threshold + val modelDatasetSortedByHash = modelDataset.sort(hashDistCol).limit(numNearestNeighbors) + val thresholdDataset = modelDatasetSortedByHash.select(max(hashDistCol)) + val hashThreshold = thresholdDataset.take(1).head.getDouble(0) + + // Filter the dataset where the hash value is less than the threshold. + modelDataset.filter(hashDistCol <= hashThreshold) + } + + // Get the top k nearest neighbor by their distance to the key + val keyDistUDF = udf((x: Vector) => keyDistance(x, key), DataTypes.DoubleType) + val modelSubsetWithDistCol = modelSubset.withColumn(distCol, keyDistUDF(col($(inputCol)))) + modelSubsetWithDistCol.sort(distCol).limit(numNearestNeighbors) + } + + /** + * Given a large dataset and an item, approximately find at most k items which have the closest + * distance to the item. If the [[outputCol]] is missing, the method will transform the data; if + * the [[outputCol]] exists, it will use the [[outputCol]]. This allows caching of the + * transformed data when necessary. + * + * @note This method is experimental and will likely change behavior in the next release. + * + * @param dataset The dataset to search for nearest neighbors of the key. + * @param key Feature vector representing the item to search for. + * @param numNearestNeighbors The maximum number of nearest neighbors. + * @param distCol Output column for storing the distance between each result row and the key. + * @return A dataset containing at most k items closest to the key. A column "distCol" is added + * to show the distance between each row and the key. + */ + def approxNearestNeighbors( + dataset: Dataset[_], + key: Vector, + numNearestNeighbors: Int, + distCol: String): Dataset[_] = { + approxNearestNeighbors(dataset, key, numNearestNeighbors, true, distCol) + } + + /** + * Overloaded method for approxNearestNeighbors. Use "distCol" as default distCol. + */ + def approxNearestNeighbors( + dataset: Dataset[_], + key: Vector, + numNearestNeighbors: Int): Dataset[_] = { + approxNearestNeighbors(dataset, key, numNearestNeighbors, true, "distCol") + } + + /** + * Preprocess step for approximate similarity join. Transform and explode the [[outputCol]] to + * two explodeCols: entry and value. "entry" is the index in hash vector, and "value" is the + * value of corresponding value of the index in the vector. + * + * @param dataset The dataset to transform and explode. + * @param explodeCols The alias for the exploded columns, must be a seq of two strings. + * @return A dataset containing idCol, inputCol and explodeCols. + */ + private[this] def processDataset( + dataset: Dataset[_], + inputName: String, + explodeCols: Seq[String]): Dataset[_] = { + require(explodeCols.size == 2, "explodeCols must be two strings.") + val modelDataset: DataFrame = if (!dataset.columns.contains($(outputCol))) { + transform(dataset) + } else { + dataset.toDF() + } + modelDataset.select( + struct(col("*")).as(inputName), posexplode(col($(outputCol))).as(explodeCols)) + } + + /** + * Recreate a column using the same column name but different attribute id. Used in approximate + * similarity join. + * @param dataset The dataset where a column need to recreate. + * @param colName The name of the column to recreate. + * @param tmpColName A temporary column name which does not conflict with existing columns. + * @return + */ + private[this] def recreateCol( + dataset: Dataset[_], + colName: String, + tmpColName: String): Dataset[_] = { + dataset + .withColumnRenamed(colName, tmpColName) + .withColumn(colName, col(tmpColName)) + .drop(tmpColName) + } + + /** + * Join two datasets to approximately find all pairs of rows whose distance are smaller than + * the threshold. If the [[outputCol]] is missing, the method will transform the data; if the + * [[outputCol]] exists, it will use the [[outputCol]]. This allows caching of the transformed + * data when necessary. + * + * @param datasetA One of the datasets to join. + * @param datasetB Another dataset to join. + * @param threshold The threshold for the distance of row pairs. + * @param distCol Output column for storing the distance between each pair of rows. + * @return A joined dataset containing pairs of rows. The original rows are in columns + * "datasetA" and "datasetB", and a column "distCol" is added to show the distance + * between each pair. + */ + def approxSimilarityJoin( + datasetA: Dataset[_], + datasetB: Dataset[_], + threshold: Double, + distCol: String): Dataset[_] = { + + val leftColName = "datasetA" + val rightColName = "datasetB" + val explodeCols = Seq("entry", "hashValue") + val explodedA = processDataset(datasetA, leftColName, explodeCols) + + // If this is a self join, we need to recreate the inputCol of datasetB to avoid ambiguity. + // TODO: Remove recreateCol logic once SPARK-17154 is resolved. + val explodedB = if (datasetA != datasetB) { + processDataset(datasetB, rightColName, explodeCols) + } else { + val recreatedB = recreateCol(datasetB, $(inputCol), s"${$(inputCol)}#${Random.nextString(5)}") + processDataset(recreatedB, rightColName, explodeCols) + } + + // Do a hash join on where the exploded hash values are equal. + val joinedDataset = explodedA.join(explodedB, explodeCols) + .drop(explodeCols: _*).distinct() + + // Add a new column to store the distance of the two rows. + val distUDF = udf((x: Vector, y: Vector) => keyDistance(x, y), DataTypes.DoubleType) + val joinedDatasetWithDist = joinedDataset.select(col("*"), + distUDF(col(s"$leftColName.${$(inputCol)}"), col(s"$rightColName.${$(inputCol)}")).as(distCol) + ) + + // Filter the joined datasets where the distance are smaller than the threshold. + joinedDatasetWithDist.filter(col(distCol) < threshold) + } + + /** + * Overloaded method for approxSimilarityJoin. Use "distCol" as default distCol. + */ + def approxSimilarityJoin( + datasetA: Dataset[_], + datasetB: Dataset[_], + threshold: Double): Dataset[_] = { + approxSimilarityJoin(datasetA, datasetB, threshold, "distCol") + } +} + +/** + * Locality Sensitive Hashing for different metrics space. Support basic transformation with a new + * hash column, approximate nearest neighbor search with a dataset and a key, and approximate + * similarity join of two datasets. + * + * This LSH class implements OR-amplification: more than 1 hash functions can be chosen, and each + * input vector are hashed by all hash functions. Two input vectors are defined to be in the same + * bucket as long as ANY one of the hash value matches. + * + * References: + * (1) Gionis, Aristides, Piotr Indyk, and Rajeev Motwani. "Similarity search in high dimensions + * via hashing." VLDB 7 Sep. 1999: 518-529. + * (2) Wang, Jingdong et al. "Hashing for similarity search: A survey." arXiv preprint + * arXiv:1408.2927 (2014). + */ +private[ml] abstract class LSH[T <: LSHModel[T]] + extends Estimator[T] with LSHParams with DefaultParamsWritable { + self: Estimator[T] => + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + /** @group setParam */ + def setNumHashTables(value: Int): this.type = set(numHashTables, value) + + /** + * Validate and create a new instance of concrete LSHModel. Because different LSHModel may have + * different initial setting, developer needs to define how their LSHModel is created instead of + * using reflection in this abstract class. + * @param inputDim The dimension of the input dataset + * @return A new LSHModel instance without any params + */ + protected[this] def createRawLSHModel(inputDim: Int): T + + override def fit(dataset: Dataset[_]): T = { + transformSchema(dataset.schema, logging = true) + val inputDim = dataset.select(col($(inputCol))).head().get(0).asInstanceOf[Vector].size + val model = createRawLSHModel(inputDim).setParent(this) + copyValues(model) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala new file mode 100644 index 000000000000..c5d0ec1a8d35 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import scala.beans.BeanInfo + +import org.apache.spark.annotation.Since +import org.apache.spark.ml.linalg.Vector + +/** + * + * Class that represents the features and label of a data point. + * + * @param label Label for this data point. + * @param features List of features for this data point. + */ +@Since("2.0.0") +@BeanInfo +case class LabeledPoint(@Since("2.0.0") label: Double, @Since("2.0.0") features: Vector) { + override def toString: String = { + s"($label,$features)" + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala new file mode 100644 index 000000000000..85f9732f79f6 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MaxAbsScaler.scala @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.ml.param.{ParamMap, Params} +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.rdd.RDD +import org.apache.spark.sql._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{StructField, StructType} + +/** + * Params for [[MaxAbsScaler]] and [[MaxAbsScalerModel]]. + */ +private[feature] trait MaxAbsScalerParams extends Params with HasInputCol with HasOutputCol { + + /** Validates and transforms the input schema. */ + protected def validateAndTransformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) + require(!schema.fieldNames.contains($(outputCol)), + s"Output column ${$(outputCol)} already exists.") + val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) + StructType(outputFields) + } +} + +/** + * Rescale each feature individually to range [-1, 1] by dividing through the largest maximum + * absolute value in each feature. It does not shift/center the data, and thus does not destroy + * any sparsity. + */ +@Since("2.0.0") +class MaxAbsScaler @Since("2.0.0") (@Since("2.0.0") override val uid: String) + extends Estimator[MaxAbsScalerModel] with MaxAbsScalerParams with DefaultParamsWritable { + + @Since("2.0.0") + def this() = this(Identifiable.randomUID("maxAbsScal")) + + /** @group setParam */ + @Since("2.0.0") + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + @Since("2.0.0") + def setOutputCol(value: String): this.type = set(outputCol, value) + + @Since("2.0.0") + override def fit(dataset: Dataset[_]): MaxAbsScalerModel = { + transformSchema(dataset.schema, logging = true) + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } + val summary = Statistics.colStats(input) + val minVals = summary.min.toArray + val maxVals = summary.max.toArray + val n = minVals.length + val maxAbs = Array.tabulate(n) { i => math.max(math.abs(minVals(i)), math.abs(maxVals(i))) } + + copyValues(new MaxAbsScalerModel(uid, Vectors.dense(maxAbs)).setParent(this)) + } + + @Since("2.0.0") + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } + + @Since("2.0.0") + override def copy(extra: ParamMap): MaxAbsScaler = defaultCopy(extra) +} + +@Since("2.0.0") +object MaxAbsScaler extends DefaultParamsReadable[MaxAbsScaler] { + + @Since("2.0.0") + override def load(path: String): MaxAbsScaler = super.load(path) +} + +/** + * Model fitted by [[MaxAbsScaler]]. + * + */ +@Since("2.0.0") +class MaxAbsScalerModel private[ml] ( + @Since("2.0.0") override val uid: String, + @Since("2.0.0") val maxAbs: Vector) + extends Model[MaxAbsScalerModel] with MaxAbsScalerParams with MLWritable { + + import MaxAbsScalerModel._ + + /** @group setParam */ + @Since("2.0.0") + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + @Since("2.0.0") + def setOutputCol(value: String): this.type = set(outputCol, value) + + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) + // TODO: this looks hack, we may have to handle sparse and dense vectors separately. + val maxAbsUnzero = Vectors.dense(maxAbs.toArray.map(x => if (x == 0) 1 else x)) + val reScale = udf { (vector: Vector) => + val brz = vector.asBreeze / maxAbsUnzero.asBreeze + Vectors.fromBreeze(brz) + } + dataset.withColumn($(outputCol), reScale(col($(inputCol)))) + } + + @Since("2.0.0") + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } + + @Since("2.0.0") + override def copy(extra: ParamMap): MaxAbsScalerModel = { + val copied = new MaxAbsScalerModel(uid, maxAbs) + copyValues(copied, extra).setParent(parent) + } + + @Since("1.6.0") + override def write: MLWriter = new MaxAbsScalerModelWriter(this) +} + +@Since("2.0.0") +object MaxAbsScalerModel extends MLReadable[MaxAbsScalerModel] { + + private[MaxAbsScalerModel] + class MaxAbsScalerModelWriter(instance: MaxAbsScalerModel) extends MLWriter { + + private case class Data(maxAbs: Vector) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = new Data(instance.maxAbs) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class MaxAbsScalerModelReader extends MLReader[MaxAbsScalerModel] { + + private val className = classOf[MaxAbsScalerModel].getName + + override def load(path: String): MaxAbsScalerModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val Row(maxAbs: Vector) = sparkSession.read.parquet(dataPath) + .select("maxAbs") + .head() + val model = new MaxAbsScalerModel(metadata.uid, maxAbs) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } + + @Since("2.0.0") + override def read: MLReader[MaxAbsScalerModel] = new MaxAbsScalerModelReader + + @Since("2.0.0") + override def load(path: String): MaxAbsScalerModel = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala new file mode 100644 index 000000000000..145422a05919 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinHashLSH.scala @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import scala.util.Random + +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.param.shared.HasSeed +import org.apache.spark.ml.util._ +import org.apache.spark.sql.types.StructType + +/** + * :: Experimental :: + * + * Model produced by [[MinHashLSH]], where multiple hash functions are stored. Each hash function + * is picked from the following family of hash functions, where a_i and b_i are randomly chosen + * integers less than prime: + * `h_i(x) = ((x \cdot a_i + b_i) \mod prime)` + * + * This hash family is approximately min-wise independent according to the reference. + * + * Reference: + * Tom Bohman, Colin Cooper, and Alan Frieze. "Min-wise independent linear permutations." + * Electronic Journal of Combinatorics 7 (2000): R26. + * + * @param randCoefficients Pairs of random coefficients. Each pair is used by one hash function. + */ +@Experimental +@Since("2.1.0") +class MinHashLSHModel private[ml]( + override val uid: String, + private[ml] val randCoefficients: Array[(Int, Int)]) + extends LSHModel[MinHashLSHModel] { + + @Since("2.1.0") + override protected[ml] val hashFunction: Vector => Array[Vector] = { + elems: Vector => { + require(elems.numNonzeros > 0, "Must have at least 1 non zero entry.") + val elemsList = elems.toSparse.indices.toList + val hashValues = randCoefficients.map { case (a, b) => + elemsList.map { elem: Int => + ((1 + elem) * a + b) % MinHashLSH.HASH_PRIME + }.min.toDouble + } + // TODO: Output vectors of dimension numHashFunctions in SPARK-18450 + hashValues.map(Vectors.dense(_)) + } + } + + @Since("2.1.0") + override protected[ml] def keyDistance(x: Vector, y: Vector): Double = { + val xSet = x.toSparse.indices.toSet + val ySet = y.toSparse.indices.toSet + val intersectionSize = xSet.intersect(ySet).size.toDouble + val unionSize = xSet.size + ySet.size - intersectionSize + assert(unionSize > 0, "The union of two input sets must have at least 1 elements") + 1 - intersectionSize / unionSize + } + + @Since("2.1.0") + override protected[ml] def hashDistance(x: Seq[Vector], y: Seq[Vector]): Double = { + // Since it's generated by hashing, it will be a pair of dense vectors. + // TODO: This hashDistance function requires more discussion in SPARK-18454 + x.zip(y).map(vectorPair => + vectorPair._1.toArray.zip(vectorPair._2.toArray).count(pair => pair._1 != pair._2) + ).min + } + + @Since("2.1.0") + override def copy(extra: ParamMap): MinHashLSHModel = { + val copied = new MinHashLSHModel(uid, randCoefficients).setParent(parent) + copyValues(copied, extra) + } + + @Since("2.1.0") + override def write: MLWriter = new MinHashLSHModel.MinHashLSHModelWriter(this) +} + +/** + * :: Experimental :: + * + * LSH class for Jaccard distance. + * + * The input can be dense or sparse vectors, but it is more efficient if it is sparse. For example, + * `Vectors.sparse(10, Array((2, 1.0), (3, 1.0), (5, 1.0)))` + * means there are 10 elements in the space. This set contains elements 2, 3, and 5. Also, any + * input vector must have at least 1 non-zero index, and all non-zero values are + * treated as binary "1" values. + * + * References: + * Wikipedia on MinHash + */ +@Experimental +@Since("2.1.0") +class MinHashLSH(override val uid: String) extends LSH[MinHashLSHModel] with HasSeed { + + @Since("2.1.0") + override def setInputCol(value: String): this.type = super.setInputCol(value) + + @Since("2.1.0") + override def setOutputCol(value: String): this.type = super.setOutputCol(value) + + @Since("2.1.0") + override def setNumHashTables(value: Int): this.type = super.setNumHashTables(value) + + @Since("2.1.0") + def this() = { + this(Identifiable.randomUID("mh-lsh")) + } + + /** @group setParam */ + @Since("2.1.0") + def setSeed(value: Long): this.type = set(seed, value) + + @Since("2.1.0") + override protected[ml] def createRawLSHModel(inputDim: Int): MinHashLSHModel = { + require(inputDim <= MinHashLSH.HASH_PRIME, + s"The input vector dimension $inputDim exceeds the threshold ${MinHashLSH.HASH_PRIME}.") + val rand = new Random($(seed)) + val randCoefs: Array[(Int, Int)] = Array.fill($(numHashTables)) { + (1 + rand.nextInt(MinHashLSH.HASH_PRIME - 1), rand.nextInt(MinHashLSH.HASH_PRIME - 1)) + } + new MinHashLSHModel(uid, randCoefs) + } + + @Since("2.1.0") + override def transformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) + validateAndTransformSchema(schema) + } + + @Since("2.1.0") + override def copy(extra: ParamMap): this.type = defaultCopy(extra) +} + +@Since("2.1.0") +object MinHashLSH extends DefaultParamsReadable[MinHashLSH] { + // A large prime smaller than sqrt(2^63 − 1) + private[ml] val HASH_PRIME = 2038074743 + + @Since("2.1.0") + override def load(path: String): MinHashLSH = super.load(path) +} + +@Since("2.1.0") +object MinHashLSHModel extends MLReadable[MinHashLSHModel] { + + @Since("2.1.0") + override def read: MLReader[MinHashLSHModel] = new MinHashLSHModelReader + + @Since("2.1.0") + override def load(path: String): MinHashLSHModel = super.load(path) + + private[MinHashLSHModel] class MinHashLSHModelWriter(instance: MinHashLSHModel) + extends MLWriter { + + private case class Data(randCoefficients: Array[Int]) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.randCoefficients.flatMap(tuple => Array(tuple._1, tuple._2))) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class MinHashLSHModelReader extends MLReader[MinHashLSHModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[MinHashLSHModel].getName + + override def load(path: String): MinHashLSHModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath).select("randCoefficients").head() + val randCoefficients = data.getAs[Seq[Int]](0).grouped(2) + .map(tuple => (tuple(0), tuple(1))).toArray + val model = new MinHashLSHModel(metadata.uid, randCoefficients) + + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala index 1b494ec8b172..f648deced54c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala @@ -17,13 +17,19 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental -import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} -import org.apache.spark.ml.param.{ParamMap, DoubleParam, Params} -import org.apache.spark.ml.util.Identifiable +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since import org.apache.spark.ml.{Estimator, Model} -import org.apache.spark.mllib.linalg.{Vector, VectorUDT, Vectors} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.ml.param.{DoubleParam, ParamMap, Params} +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -57,68 +63,84 @@ private[feature] trait MinMaxScalerParams extends Params with HasInputCol with H /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType): StructType = { - val inputType = schema($(inputCol)).dataType - require(inputType.isInstanceOf[VectorUDT], - s"Input column ${$(inputCol)} must be a vector column") + require($(min) < $(max), s"The specified min(${$(min)}) is larger or equal to max(${$(max)})") + SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) require(!schema.fieldNames.contains($(outputCol)), s"Output column ${$(outputCol)} already exists.") val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) StructType(outputFields) } - override def validateParams(): Unit = { - require($(min) < $(max), s"The specified min(${$(min)}) is larger or equal to max(${$(max)})") - } } /** - * :: Experimental :: * Rescale each feature individually to a common range [min, max] linearly using column summary * statistics, which is also known as min-max normalization or Rescaling. The rescaled value for - * feature E is calculated as, + * feature E is calculated as: + * + *
    + * $$ + * Rescaled(e_i) = \frac{e_i - E_{min}}{E_{max} - E_{min}} * (max - min) + min + * $$ + *
    * - * Rescaled(e_i) = \frac{e_i - E_{min}}{E_{max} - E_{min}} * (max - min) + min + * For the case \(E_{max} == E_{min}\), \(Rescaled(e_i) = 0.5 * (max + min)\). * - * For the case E_{max} == E_{min}, Rescaled(e_i) = 0.5 * (max + min) - * Note that since zero values will probably be transformed to non-zero values, output of the + * @note Since zero values will probably be transformed to non-zero values, output of the * transformer will be DenseVector even for sparse input. */ -@Experimental -class MinMaxScaler(override val uid: String) - extends Estimator[MinMaxScalerModel] with MinMaxScalerParams { +@Since("1.5.0") +class MinMaxScaler @Since("1.5.0") (@Since("1.5.0") override val uid: String) + extends Estimator[MinMaxScalerModel] with MinMaxScalerParams with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("minMaxScal")) setDefault(min -> 0.0, max -> 1.0) /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.5.0") def setMin(value: Double): this.type = set(min, value) /** @group setParam */ + @Since("1.5.0") def setMax(value: Double): this.type = set(max, value) - override def fit(dataset: DataFrame): MinMaxScalerModel = { + @Since("2.0.0") + override def fit(dataset: Dataset[_]): MinMaxScalerModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val summary = Statistics.colStats(input) copyValues(new MinMaxScalerModel(uid, summary.min, summary.max).setParent(this)) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.5.0") override def copy(extra: ParamMap): MinMaxScaler = defaultCopy(extra) } +@Since("1.6.0") +object MinMaxScaler extends DefaultParamsReadable[MinMaxScaler] { + + @Since("1.6.0") + override def load(path: String): MinMaxScaler = super.load(path) +} + /** - * :: Experimental :: * Model fitted by [[MinMaxScaler]]. * * @param originalMin min value for each original column during fitting @@ -126,27 +148,35 @@ class MinMaxScaler(override val uid: String) * * TODO: The transformer does not yet set the metadata in the output column (SPARK-8529). */ -@Experimental +@Since("1.5.0") class MinMaxScalerModel private[ml] ( - override val uid: String, - val originalMin: Vector, - val originalMax: Vector) - extends Model[MinMaxScalerModel] with MinMaxScalerParams { + @Since("1.5.0") override val uid: String, + @Since("2.0.0") val originalMin: Vector, + @Since("2.0.0") val originalMax: Vector) + extends Model[MinMaxScalerModel] with MinMaxScalerParams with MLWritable { + + import MinMaxScalerModel._ /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.5.0") def setMin(value: Double): this.type = set(min, value) /** @group setParam */ + @Since("1.5.0") def setMax(value: Double): this.type = set(max, value) - override def transform(dataset: DataFrame): DataFrame = { - val originalRange = (originalMax.toBreeze - originalMin.toBreeze).toArray + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) + val originalRange = (originalMax.asBreeze - originalMin.asBreeze).toArray val minArray = originalMin.toArray val reScale = udf { (vector: Vector) => @@ -154,11 +184,13 @@ class MinMaxScalerModel private[ml] ( // 0 in sparse vector will probably be rescaled to non-zero val values = vector.toArray - val size = values.size + val size = values.length var i = 0 while (i < size) { - val raw = if (originalRange(i) != 0) (values(i) - minArray(i)) / originalRange(i) else 0.5 - values(i) = raw * scale + $(min) + if (!values(i).isNaN) { + val raw = if (originalRange(i) != 0) (values(i) - minArray(i)) / originalRange(i) else 0.5 + values(i) = raw * scale + $(min) + } i += 1 } Vectors.dense(values) @@ -167,12 +199,58 @@ class MinMaxScalerModel private[ml] ( dataset.withColumn($(outputCol), reScale(col($(inputCol)))) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.5.0") override def copy(extra: ParamMap): MinMaxScalerModel = { val copied = new MinMaxScalerModel(uid, originalMin, originalMax) copyValues(copied, extra).setParent(parent) } + + @Since("1.6.0") + override def write: MLWriter = new MinMaxScalerModelWriter(this) +} + +@Since("1.6.0") +object MinMaxScalerModel extends MLReadable[MinMaxScalerModel] { + + private[MinMaxScalerModel] + class MinMaxScalerModelWriter(instance: MinMaxScalerModel) extends MLWriter { + + private case class Data(originalMin: Vector, originalMax: Vector) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = new Data(instance.originalMin, instance.originalMax) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class MinMaxScalerModelReader extends MLReader[MinMaxScalerModel] { + + private val className = classOf[MinMaxScalerModel].getName + + override def load(path: String): MinMaxScalerModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + val Row(originalMin: Vector, originalMax: Vector) = + MLUtils.convertVectorColumnsToML(data, "originalMin", "originalMax") + .select("originalMin", "originalMax") + .head() + val model = new MinMaxScalerModel(metadata.uid, originalMin, originalMax) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } + + @Since("1.6.0") + override def read: MLReader[MinMaxScalerModel] = new MinMaxScalerModelReader + + @Since("1.6.0") + override def load(path: String): MinMaxScalerModel = super.load(path) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala index 8de10eb51f92..c8760f9dc178 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala @@ -17,14 +17,13 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.param._ -import org.apache.spark.ml.util.Identifiable +import org.apache.spark.ml.util._ import org.apache.spark.sql.types.{ArrayType, DataType, StringType} /** - * :: Experimental :: * A feature transformer that converts the input array of strings into an array of n-grams. Null * values in the input array are ignored. * It returns an array of n-grams where each n-gram is represented by a space-separated string of @@ -34,24 +33,28 @@ import org.apache.spark.sql.types.{ArrayType, DataType, StringType} * When the input array length is less than n (number of elements per n-gram), no n-grams are * returned. */ -@Experimental -class NGram(override val uid: String) - extends UnaryTransformer[Seq[String], Seq[String], NGram] { +@Since("1.5.0") +class NGram @Since("1.5.0") (@Since("1.5.0") override val uid: String) + extends UnaryTransformer[Seq[String], Seq[String], NGram] with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("ngram")) /** - * Minimum n-gram length, >= 1. + * Minimum n-gram length, greater than or equal to 1. * Default: 2, bigram features * @group param */ + @Since("1.5.0") val n: IntParam = new IntParam(this, "n", "number elements per n-gram (>=1)", ParamValidators.gtEq(1)) /** @group setParam */ + @Since("1.5.0") def setN(value: Int): this.type = set(n, value) /** @group getParam */ + @Since("1.5.0") def getN: Int = $(n) setDefault(n -> 2) @@ -67,3 +70,10 @@ class NGram(override val uid: String) override protected def outputDataType: DataType = new ArrayType(StringType, false) } + +@Since("1.6.0") +object NGram extends DefaultParamsReadable[NGram] { + + @Since("1.6.0") + override def load(path: String): NGram = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala index 8282e5ffa17f..6e96545c8cb7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala @@ -17,42 +17,54 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param.{DoubleParam, ParamValidators} -import org.apache.spark.ml.util.Identifiable +import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} import org.apache.spark.sql.types.DataType /** - * :: Experimental :: * Normalize a vector to have unit norm using the given p-norm. */ -@Experimental -class Normalizer(override val uid: String) extends UnaryTransformer[Vector, Vector, Normalizer] { +@Since("1.4.0") +class Normalizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) + extends UnaryTransformer[Vector, Vector, Normalizer] with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("normalizer")) /** - * Normalization in L^p^ space. Must be >= 1. + * Normalization in L^p^ space. Must be greater than equal to 1. * (default: p = 2) * @group param */ + @Since("1.4.0") val p = new DoubleParam(this, "p", "the p norm value", ParamValidators.gtEq(1)) setDefault(p -> 2.0) /** @group getParam */ + @Since("1.4.0") def getP: Double = $(p) /** @group setParam */ + @Since("1.4.0") def setP(value: Double): this.type = set(p, value) override protected def createTransformFunc: Vector => Vector = { val normalizer = new feature.Normalizer($(p)) - normalizer.transform + vector => normalizer.transform(OldVectors.fromML(vector)).asML } override protected def outputDataType: DataType = new VectorUDT() } + +@Since("1.6.0") +object Normalizer extends DefaultParamsReadable[Normalizer] { + + @Since("1.6.0") + override def load(path: String): Normalizer = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index 9c60d4084ec4..a669da183e2c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -17,59 +17,70 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} -import org.apache.spark.ml.util.{Identifiable, SchemaUtils} -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.sql.DataFrame +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} -import org.apache.spark.sql.types.{DoubleType, StructType} +import org.apache.spark.sql.types.{DoubleType, NumericType, StructType} /** - * :: Experimental :: * A one-hot encoder that maps a column of category indices to a column of binary vectors, with * at most a single one-value per row that indicates the input category index. * For example with 5 categories, an input value of 2.0 would map to an output vector of * `[0.0, 0.0, 1.0, 0.0]`. - * The last category is not included by default (configurable via [[OneHotEncoder!.dropLast]] + * The last category is not included by default (configurable via `OneHotEncoder!.dropLast` * because it makes the vector entries sum up to one, and hence linearly dependent. * So an input value of 4.0 maps to `[0.0, 0.0, 0.0, 0.0]`. - * Note that this is different from scikit-learn's OneHotEncoder, which keeps all categories. + * + * @note This is different from scikit-learn's OneHotEncoder, which keeps all categories. * The output vectors are sparse. * - * @see [[StringIndexer]] for converting categorical values into category indices + * @see `StringIndexer` for converting categorical values into category indices */ -@Experimental -class OneHotEncoder(override val uid: String) extends Transformer - with HasInputCol with HasOutputCol { +@Since("1.4.0") +class OneHotEncoder @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Transformer + with HasInputCol with HasOutputCol with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("oneHot")) /** * Whether to drop the last category in the encoded vector (default: true) * @group param */ + @Since("1.4.0") final val dropLast: BooleanParam = new BooleanParam(this, "dropLast", "whether to drop the last category") setDefault(dropLast -> true) + /** @group getParam */ + @Since("2.0.0") + def getDropLast: Boolean = $(dropLast) + /** @group setParam */ + @Since("1.4.0") def setDropLast(value: Boolean): this.type = set(dropLast, value) /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { val inputColName = $(inputCol) val outputColName = $(outputCol) - SchemaUtils.checkColumnType(schema, inputColName, DoubleType) + require(schema(inputColName).dataType.isInstanceOf[NumericType], + s"Input column must be of type NumericType but got ${schema(inputColName).dataType}") val inputFields = schema.fields require(!inputFields.exists(_.name == outputColName), s"Output column $outputColName already exists.") @@ -120,7 +131,8 @@ class OneHotEncoder(override val uid: String) extends Transformer StructType(outputFields) } - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { // schema transformation val inputColName = $(inputCol) val outputColName = $(outputCol) @@ -129,10 +141,12 @@ class OneHotEncoder(override val uid: String) extends Transformer transformSchema(dataset.schema)(outputColName)) if (outputAttrGroup.size < 0) { // If the number of attributes is unknown, we check the values from the input column. - val numAttrs = dataset.select(col(inputColName).cast(DoubleType)).map(_.getDouble(0)) - .aggregate(0.0)( + val numAttrs = dataset.select(col(inputColName).cast(DoubleType)).rdd.map(_.getDouble(0)) + .treeAggregate(0.0)( (m, x) => { - assert(x >=0.0 && x == x.toInt, + assert(x <= Int.MaxValue, + s"OneHotEncoder only supports up to ${Int.MaxValue} indices, but got $x") + assert(x >= 0.0 && x == x.toInt, s"Values from column $inputColName must be indices, but got $x.") math.max(m, x) }, @@ -151,8 +165,8 @@ class OneHotEncoder(override val uid: String) extends Transformer // data transformation val size = outputAttrGroup.size val oneValue = Array(1.0) - val emptyValues = Array[Double]() - val emptyIndices = Array[Int]() + val emptyValues = Array.empty[Double] + val emptyIndices = Array.empty[Int] val encode = udf { label: Double => if (label < size) { Vectors.sparse(size, Array(label.toInt), oneValue) @@ -164,5 +178,13 @@ class OneHotEncoder(override val uid: String) extends Transformer dataset.select(col("*"), encode(col(inputColName).cast(DoubleType)).as(outputColName, metadata)) } + @Since("1.4.1") override def copy(extra: ParamMap): OneHotEncoder = defaultCopy(extra) } + +@Since("1.6.0") +object OneHotEncoder extends DefaultParamsReadable[OneHotEncoder] { + + @Since("1.6.0") + override def load(path: String): OneHotEncoder = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index 539084704b65..4143d864d793 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -17,16 +17,24 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since import org.apache.spark.ml._ +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.Identifiable +import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{DenseMatrix => OldDenseMatrix, DenseVector => OldDenseVector, + Matrices => OldMatrices, Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.MatrixImplicits._ +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.util.VersionUtils.majorVersion /** * Params for [[PCA]] and [[PCAModel]]. @@ -37,94 +45,189 @@ private[feature] trait PCAParams extends Params with HasInputCol with HasOutputC * The number of principal components. * @group param */ - final val k: IntParam = new IntParam(this, "k", "the number of principal components") + final val k: IntParam = new IntParam(this, "k", "the number of principal components (> 0)", + ParamValidators.gt(0)) /** @group getParam */ def getK: Int = $(k) + /** Validates and transforms the input schema. */ + protected def validateAndTransformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) + require(!schema.fieldNames.contains($(outputCol)), + s"Output column ${$(outputCol)} already exists.") + val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) + StructType(outputFields) + } + } /** - * :: Experimental :: - * PCA trains a model to project vectors to a low-dimensional space using PCA. + * PCA trains a model to project vectors to a lower dimensional space of the top `PCA!.k` + * principal components. */ -@Experimental -class PCA (override val uid: String) extends Estimator[PCAModel] with PCAParams { +@Since("1.5.0") +class PCA @Since("1.5.0") ( + @Since("1.5.0") override val uid: String) + extends Estimator[PCAModel] with PCAParams with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("pca")) /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.5.0") def setK(value: Int): this.type = set(k, value) /** * Computes a [[PCAModel]] that contains the principal components of the input vectors. */ - override def fit(dataset: DataFrame): PCAModel = { + @Since("2.0.0") + override def fit(dataset: Dataset[_]): PCAModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).map { case Row(v: Vector) => v} + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val pca = new feature.PCA(k = $(k)) val pcaModel = pca.fit(input) - copyValues(new PCAModel(uid, pcaModel).setParent(this)) + copyValues(new PCAModel(uid, pcaModel.pc, pcaModel.explainedVariance).setParent(this)) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { - val inputType = schema($(inputCol)).dataType - require(inputType.isInstanceOf[VectorUDT], - s"Input column ${$(inputCol)} must be a vector column") - require(!schema.fieldNames.contains($(outputCol)), - s"Output column ${$(outputCol)} already exists.") - val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) - StructType(outputFields) + validateAndTransformSchema(schema) } + @Since("1.5.0") override def copy(extra: ParamMap): PCA = defaultCopy(extra) } +@Since("1.6.0") +object PCA extends DefaultParamsReadable[PCA] { + + @Since("1.6.0") + override def load(path: String): PCA = super.load(path) +} + /** - * :: Experimental :: - * Model fitted by [[PCA]]. + * Model fitted by [[PCA]]. Transforms vectors to a lower dimensional space. + * + * @param pc A principal components Matrix. Each column is one principal component. + * @param explainedVariance A vector of proportions of variance explained by + * each principal component. */ -@Experimental +@Since("1.5.0") class PCAModel private[ml] ( - override val uid: String, - pcaModel: feature.PCAModel) - extends Model[PCAModel] with PCAParams { + @Since("1.5.0") override val uid: String, + @Since("2.0.0") val pc: DenseMatrix, + @Since("2.0.0") val explainedVariance: DenseVector) + extends Model[PCAModel] with PCAParams with MLWritable { + + import PCAModel._ /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** * Transform a vector by computed Principal Components. - * NOTE: Vectors to be transformed must be the same length - * as the source vectors given to [[PCA.fit()]]. + * + * @note Vectors to be transformed must be the same length as the source vectors given + * to `PCA.fit()`. */ - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val pcaOp = udf { pcaModel.transform _ } + val pcaModel = new feature.PCAModel($(k), + OldMatrices.fromML(pc).asInstanceOf[OldDenseMatrix], + OldVectors.fromML(explainedVariance).asInstanceOf[OldDenseVector]) + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer: Vector => Vector = v => pcaModel.transform(OldVectors.fromML(v)).asML + + val pcaOp = udf(transformer) dataset.withColumn($(outputCol), pcaOp(col($(inputCol)))) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { - val inputType = schema($(inputCol)).dataType - require(inputType.isInstanceOf[VectorUDT], - s"Input column ${$(inputCol)} must be a vector column") - require(!schema.fieldNames.contains($(outputCol)), - s"Output column ${$(outputCol)} already exists.") - val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) - StructType(outputFields) + validateAndTransformSchema(schema) } + @Since("1.5.0") override def copy(extra: ParamMap): PCAModel = { - val copied = new PCAModel(uid, pcaModel) + val copied = new PCAModel(uid, pc, explainedVariance) copyValues(copied, extra).setParent(parent) } + + @Since("1.6.0") + override def write: MLWriter = new PCAModelWriter(this) +} + +@Since("1.6.0") +object PCAModel extends MLReadable[PCAModel] { + + private[PCAModel] class PCAModelWriter(instance: PCAModel) extends MLWriter { + + private case class Data(pc: DenseMatrix, explainedVariance: DenseVector) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.pc, instance.explainedVariance) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class PCAModelReader extends MLReader[PCAModel] { + + private val className = classOf[PCAModel].getName + + /** + * Loads a [[PCAModel]] from data located at the input path. Note that the model includes an + * `explainedVariance` member that is not recorded by Spark 1.6 and earlier. A model + * can be loaded from such older data but will have an empty vector for + * `explainedVariance`. + * + * @param path path to serialized model data + * @return a [[PCAModel]] + */ + override def load(path: String): PCAModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + + val dataPath = new Path(path, "data").toString + val model = if (majorVersion(metadata.sparkVersion) >= 2) { + val Row(pc: DenseMatrix, explainedVariance: DenseVector) = + sparkSession.read.parquet(dataPath) + .select("pc", "explainedVariance") + .head() + new PCAModel(metadata.uid, pc, explainedVariance) + } else { + // pc field is the old matrix format in Spark <= 1.6 + // explainedVariance field is not present in Spark <= 1.6 + val Row(pc: OldDenseMatrix) = sparkSession.read.parquet(dataPath).select("pc").head() + new PCAModel(metadata.uid, pc.asML, + Vectors.dense(Array.empty[Double]).asInstanceOf[DenseVector]) + } + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } + + @Since("1.6.0") + override def read: MLReader[PCAModel] = new PCAModelReader + + @Since("1.6.0") + override def load(path: String): PCAModel = super.load(path) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index d85e468562d4..292f9496a456 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -19,41 +19,49 @@ package org.apache.spark.ml.feature import scala.collection.mutable -import org.apache.spark.annotation.Experimental +import org.apache.commons.math3.util.CombinatoricsUtils + +import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer -import org.apache.spark.ml.param.{ParamMap, IntParam, ParamValidators} -import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg._ +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators} +import org.apache.spark.ml.util._ import org.apache.spark.sql.types.DataType /** - * :: Experimental :: * Perform feature expansion in a polynomial space. As said in wikipedia of Polynomial Expansion, - * which is available at [[http://en.wikipedia.org/wiki/Polynomial_expansion]], "In mathematics, an - * expansion of a product of sums expresses it as a sum of products by using the fact that - * multiplication distributes over addition". Take a 2-variable feature vector as an example: - * `(x, y)`, if we want to expand it with degree 2, then we get `(x, x * x, y, x * y, y * y)`. + * which is available at + * Polynomial expansion (Wikipedia) + * , "In mathematics, an expansion of a product of sums expresses it as a sum of products by using + * the fact that multiplication distributes over addition". Take a 2-variable feature vector + * as an example: `(x, y)`, if we want to expand it with degree 2, then we get + * `(x, x * x, y, x * y, y * y)`. */ -@Experimental -class PolynomialExpansion(override val uid: String) - extends UnaryTransformer[Vector, Vector, PolynomialExpansion] { +@Since("1.4.0") +class PolynomialExpansion @Since("1.4.0") (@Since("1.4.0") override val uid: String) + extends UnaryTransformer[Vector, Vector, PolynomialExpansion] with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("poly")) /** - * The polynomial degree to expand, which should be >= 1. A value of 1 means no expansion. + * The polynomial degree to expand, which should be greater than equal to 1. A value of 1 means + * no expansion. * Default: 2 * @group param */ + @Since("1.4.0") val degree = new IntParam(this, "degree", "the polynomial degree to expand (>= 1)", - ParamValidators.gt(1)) + ParamValidators.gtEq(1)) setDefault(degree -> 2) /** @group getParam */ + @Since("1.4.0") def getDegree: Int = $(degree) /** @group setParam */ + @Since("1.4.0") def setDegree(value: Int): this.type = set(degree, value) override protected def createTransformFunc: Vector => Vector = { v => @@ -62,6 +70,7 @@ class PolynomialExpansion(override val uid: String) override protected def outputDataType: DataType = new VectorUDT() + @Since("1.4.1") override def copy(extra: ParamMap): PolynomialExpansion = defaultCopy(extra) } @@ -70,21 +79,24 @@ class PolynomialExpansion(override val uid: String) * (n + d choose d) (including 1 and first-order values). For example, let f([a, b, c], 3) be the * function that expands [a, b, c] to their monomials of degree 3. We have the following recursion: * - * {{{ - * f([a, b, c], 3) = f([a, b], 3) ++ f([a, b], 2) * c ++ f([a, b], 1) * c^2 ++ [c^3] - * }}} + *
    + * $$ + * f([a, b, c], 3) &= f([a, b], 3) ++ f([a, b], 2) * c ++ f([a, b], 1) * c^2 ++ [c^3] + * $$ + *
    * * To handle sparsity, if c is zero, we can skip all monomials that contain it. We remember the * current index and increment it properly for sparse input. */ -private[feature] object PolynomialExpansion { +@Since("1.6.0") +object PolynomialExpansion extends DefaultParamsReadable[PolynomialExpansion] { - private def choose(n: Int, k: Int): Int = { - Range(n, n - k, -1).product / Range(k, 1, -1).product + private def getPolySize(numFeatures: Int, degree: Int): Int = { + val n = CombinatoricsUtils.binomialCoefficient(numFeatures + degree, degree) + require(n <= Integer.MAX_VALUE) + n.toInt } - private def getPolySize(numFeatures: Int, degree: Int): Int = choose(numFeatures + degree, degree) - private def expandDense( values: Array[Double], lastIdx: Int, @@ -169,11 +181,14 @@ private[feature] object PolynomialExpansion { new SparseVector(polySize - 1, polyIndices.result(), polyValues.result()) } - def expand(v: Vector, degree: Int): Vector = { + private[feature] def expand(v: Vector, degree: Int): Vector = { v match { case dv: DenseVector => expand(dv, degree) case sv: SparseVector => expand(sv, degree) case _ => throw new IllegalArgumentException } } + + @Since("1.6.0") + override def load(path: String): PolynomialExpansion = super.load(path) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index 46b836da9cfd..95e8830283de 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -17,64 +17,121 @@ package org.apache.spark.ml.feature -import scala.collection.mutable - -import org.apache.spark.Logging -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging import org.apache.spark.ml._ import org.apache.spark.ml.attribute.NominalAttribute -import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} -import org.apache.spark.ml.param.{IntParam, _} +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.sql.types.{DoubleType, StructType} -import org.apache.spark.sql.{DataFrame, Row} -import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.types.StructType /** * Params for [[QuantileDiscretizer]]. */ -private[feature] trait QuantileDiscretizerBase extends Params with HasInputCol with HasOutputCol { +private[feature] trait QuantileDiscretizerBase extends Params + with HasHandleInvalid with HasInputCol with HasOutputCol { /** - * Maximum number of buckets (quantiles, or categories) into which data points are grouped. Must - * be >= 2. + * Number of buckets (quantiles, or categories) into which data points are grouped. Must + * be greater than or equal to 2. + * + * See also [[handleInvalid]], which can optionally create an additional bucket for NaN values. + * * default: 2 * @group param */ - val numBuckets = new IntParam(this, "numBuckets", "Maximum number of buckets (quantiles, or " + + val numBuckets = new IntParam(this, "numBuckets", "Number of buckets (quantiles, or " + "categories) into which data points are grouped. Must be >= 2.", ParamValidators.gtEq(2)) setDefault(numBuckets -> 2) /** @group getParam */ def getNumBuckets: Int = getOrDefault(numBuckets) + + /** + * Relative error (see documentation for + * `org.apache.spark.sql.DataFrameStatFunctions.approxQuantile` for description) + * Must be in the range [0, 1]. + * default: 0.001 + * @group param + */ + val relativeError = new DoubleParam(this, "relativeError", "The relative target precision " + + "for the approximate quantile algorithm used to generate buckets. " + + "Must be in the range [0, 1].", ParamValidators.inRange(0.0, 1.0)) + setDefault(relativeError -> 0.001) + + /** @group getParam */ + def getRelativeError: Double = getOrDefault(relativeError) + + /** + * Param for how to handle invalid entries. Options are 'skip' (filter out rows with + * invalid values), 'error' (throw an error), or 'keep' (keep invalid values in a special + * additional bucket). + * Default: "error" + * @group param + */ + @Since("2.1.0") + override val handleInvalid: Param[String] = new Param[String](this, "handleInvalid", + "how to handle invalid entries. Options are skip (filter out rows with invalid values), " + + "error (throw an error), or keep (keep invalid values in a special additional bucket).", + ParamValidators.inArray(Bucketizer.supportedHandleInvalids)) + setDefault(handleInvalid, Bucketizer.ERROR_INVALID) + } /** - * :: Experimental :: * `QuantileDiscretizer` takes a column with continuous features and outputs a column with binned - * categorical features. The bin ranges are chosen by taking a sample of the data and dividing it - * into roughly equal parts. The lower and upper bin bounds will be -Infinity and +Infinity, - * covering all real values. This attempts to find numBuckets partitions based on a sample of data, - * but it may find fewer depending on the data sample values. + * categorical features. The number of bins can be set using the `numBuckets` parameter. It is + * possible that the number of buckets used will be smaller than this value, for example, if there + * are too few distinct values of the input to create enough distinct quantiles. + * + * NaN handling: + * null and NaN values will be ignored from the column during `QuantileDiscretizer` fitting. This + * will produce a `Bucketizer` model for making predictions. During the transformation, + * `Bucketizer` will raise an error when it finds NaN values in the dataset, but the user can + * also choose to either keep or remove NaN values within the dataset by setting `handleInvalid`. + * If the user chooses to keep NaN values, they will be handled specially and placed into their own + * bucket, for example, if 4 buckets are used, then non-NaN data will be put into buckets[0-3], + * but NaNs will be counted in a special bucket[4]. + * + * Algorithm: The bin ranges are chosen using an approximate algorithm (see the documentation for + * `org.apache.spark.sql.DataFrameStatFunctions.approxQuantile` + * for a detailed description). The precision of the approximation can be controlled with the + * `relativeError` parameter. The lower and upper bin bounds will be `-Infinity` and `+Infinity`, + * covering all real values. */ -@Experimental -final class QuantileDiscretizer(override val uid: String) - extends Estimator[Bucketizer] with QuantileDiscretizerBase { +@Since("1.6.0") +final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val uid: String) + extends Estimator[Bucketizer] with QuantileDiscretizerBase with DefaultParamsWritable { + @Since("1.6.0") def this() = this(Identifiable.randomUID("quantileDiscretizer")) /** @group setParam */ + @Since("2.0.0") + def setRelativeError(value: Double): this.type = set(relativeError, value) + + /** @group setParam */ + @Since("1.6.0") def setNumBuckets(value: Int): this.type = set(numBuckets, value) /** @group setParam */ + @Since("1.6.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.6.0") def setOutputCol(value: String): this.type = set(outputCol, value) + /** @group setParam */ + @Since("2.1.0") + def setHandleInvalid(value: String): this.type = set(handleInvalid, value) + + @Since("1.6.0") override def transformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(inputCol), DoubleType) + SchemaUtils.checkNumericType(schema, $(inputCol)) val inputFields = schema.fields require(inputFields.forall(_.name != $(outputCol)), s"Output column ${$(outputCol)} already exists.") @@ -83,94 +140,32 @@ final class QuantileDiscretizer(override val uid: String) StructType(outputFields) } - override def fit(dataset: DataFrame): Bucketizer = { - val samples = QuantileDiscretizer.getSampledInput(dataset.select($(inputCol)), $(numBuckets)) - .map { case Row(feature: Double) => feature } - val candidates = QuantileDiscretizer.findSplitCandidates(samples, $(numBuckets) - 1) - val splits = QuantileDiscretizer.getSplits(candidates) - val bucketizer = new Bucketizer(uid).setSplits(splits) - copyValues(bucketizer) + @Since("2.0.0") + override def fit(dataset: Dataset[_]): Bucketizer = { + transformSchema(dataset.schema, logging = true) + val splits = dataset.stat.approxQuantile($(inputCol), + (0.0 to 1.0 by 1.0/$(numBuckets)).toArray, $(relativeError)) + splits(0) = Double.NegativeInfinity + splits(splits.length - 1) = Double.PositiveInfinity + + val distinctSplits = splits.distinct + if (splits.length != distinctSplits.length) { + log.warn(s"Some quantiles were identical. Bucketing to ${distinctSplits.length - 1}" + + s" buckets as a result.") + } + val bucketizer = new Bucketizer(uid) + .setSplits(distinctSplits.sorted) + .setHandleInvalid($(handleInvalid)) + copyValues(bucketizer.setParent(this)) } + @Since("1.6.0") override def copy(extra: ParamMap): QuantileDiscretizer = defaultCopy(extra) } -private[feature] object QuantileDiscretizer extends Logging { - /** - * Sampling from the given dataset to collect quantile statistics. - */ - def getSampledInput(dataset: DataFrame, numBins: Int): Array[Row] = { - val totalSamples = dataset.count() - require(totalSamples > 0, - "QuantileDiscretizer requires non-empty input dataset but was given an empty input.") - val requiredSamples = math.max(numBins * numBins, 10000) - val fraction = math.min(requiredSamples / dataset.count(), 1.0) - dataset.sample(withReplacement = false, fraction, new XORShiftRandom().nextInt()).collect() - } +@Since("1.6.0") +object QuantileDiscretizer extends DefaultParamsReadable[QuantileDiscretizer] with Logging { - /** - * Compute split points with respect to the sample distribution. - */ - def findSplitCandidates(samples: Array[Double], numSplits: Int): Array[Double] = { - val valueCountMap = samples.foldLeft(Map.empty[Double, Int]) { (m, x) => - m + ((x, m.getOrElse(x, 0) + 1)) - } - val valueCounts = valueCountMap.toSeq.sortBy(_._1).toArray ++ Array((Double.MaxValue, 1)) - val possibleSplits = valueCounts.length - 1 - if (possibleSplits <= numSplits) { - valueCounts.dropRight(1).map(_._1) - } else { - val stride: Double = math.ceil(samples.length.toDouble / (numSplits + 1)) - val splitsBuilder = mutable.ArrayBuilder.make[Double] - var index = 1 - // currentCount: sum of counts of values that have been visited - var currentCount = valueCounts(0)._2 - // targetCount: target value for `currentCount`. If `currentCount` is closest value to - // `targetCount`, then current value is a split threshold. After finding a split threshold, - // `targetCount` is added by stride. - var targetCount = stride - while (index < valueCounts.length) { - val previousCount = currentCount - currentCount += valueCounts(index)._2 - val previousGap = math.abs(previousCount - targetCount) - val currentGap = math.abs(currentCount - targetCount) - // If adding count of current value to currentCount makes the gap between currentCount and - // targetCount smaller, previous value is a split threshold. - if (previousGap < currentGap) { - splitsBuilder += valueCounts(index - 1)._1 - targetCount += stride - } - index += 1 - } - splitsBuilder.result() - } - } - - /** - * Adjust split candidates to proper splits by: adding positive/negative infinity to both sides as - * needed, and adding a default split value of 0 if no good candidates are found. - */ - def getSplits(candidates: Array[Double]): Array[Double] = { - val effectiveValues = if (candidates.size != 0) { - if (candidates.head == Double.NegativeInfinity - && candidates.last == Double.PositiveInfinity) { - candidates.drop(1).dropRight(1) - } else if (candidates.head == Double.NegativeInfinity) { - candidates.drop(1) - } else if (candidates.last == Double.PositiveInfinity) { - candidates.dropRight(1) - } else { - candidates - } - } else { - candidates - } - - if (effectiveValues.size == 0) { - Array(Double.NegativeInfinity, 0, Double.PositiveInfinity) - } else { - Array(Double.NegativeInfinity) ++ effectiveValues ++ Array(Double.PositiveInfinity) - } - } + @Since("1.6.0") + override def load(path: String): QuantileDiscretizer = super.load(path) } - diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index 5c43a41bee3b..7da3339f8b48 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -20,20 +20,103 @@ package org.apache.spark.ml.feature import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.spark.annotation.Experimental -import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model, Pipeline, PipelineModel, PipelineStage, Transformer} -import org.apache.spark.ml.param.{Param, ParamMap} -import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasLabelCol} -import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg.VectorUDT -import org.apache.spark.sql.DataFrame +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.linalg.VectorUDT +import org.apache.spark.ml.param.{BooleanParam, Param, ParamMap, ParamValidators} +import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasHandleInvalid, HasLabelCol} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types._ /** * Base trait for [[RFormula]] and [[RFormulaModel]]. */ -private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol { +private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol with HasHandleInvalid { + + /** + * R formula parameter. The formula is provided in string form. + * @group param + */ + @Since("1.5.0") + val formula: Param[String] = new Param(this, "formula", "R model formula") + + /** @group getParam */ + @Since("1.5.0") + def getFormula: String = $(formula) + + /** + * Force to index label whether it is numeric or string type. + * Usually we index label only when it is string type. + * If the formula was used by classification algorithms, + * we can force to index label even it is numeric type by setting this param with true. + * Default: false. + * @group param + */ + @Since("2.1.0") + val forceIndexLabel: BooleanParam = new BooleanParam(this, "forceIndexLabel", + "Force to index label whether it is numeric or string") + setDefault(forceIndexLabel -> false) + + /** @group getParam */ + @Since("2.1.0") + def getForceIndexLabel: Boolean = $(forceIndexLabel) + + /** + * Param for how to handle invalid data (unseen or NULL values) in features and label column + * of string type. Options are 'skip' (filter out rows with invalid data), + * 'error' (throw an error), or 'keep' (put invalid data in a special additional + * bucket, at index numLabels). + * Default: "error" + * @group param + */ + @Since("2.3.0") + final override val handleInvalid: Param[String] = new Param[String](this, "handleInvalid", + "How to handle invalid data (unseen or NULL values) in features and label column of string " + + "type. Options are 'skip' (filter out rows with invalid data), error (throw an error), " + + "or 'keep' (put invalid data in a special additional bucket, at index numLabels).", + ParamValidators.inArray(StringIndexer.supportedHandleInvalids)) + setDefault(handleInvalid, StringIndexer.ERROR_INVALID) + + /** + * Param for how to order categories of a string FEATURE column used by `StringIndexer`. + * The last category after ordering is dropped when encoding strings. + * Supported options: 'frequencyDesc', 'frequencyAsc', 'alphabetDesc', 'alphabetAsc'. + * The default value is 'frequencyDesc'. When the ordering is set to 'alphabetDesc', `RFormula` + * drops the same category as R when encoding strings. + * + * The options are explained using an example `'b', 'a', 'b', 'a', 'c', 'b'`: + * {{{ + * +-----------------+---------------------------------------+----------------------------------+ + * | Option | Category mapped to 0 by StringIndexer | Category dropped by RFormula | + * +-----------------+---------------------------------------+----------------------------------+ + * | 'frequencyDesc' | most frequent category ('b') | least frequent category ('c') | + * | 'frequencyAsc' | least frequent category ('c') | most frequent category ('b') | + * | 'alphabetDesc' | last alphabetical category ('c') | first alphabetical category ('a')| + * | 'alphabetAsc' | first alphabetical category ('a') | last alphabetical category ('c') | + * +-----------------+---------------------------------------+----------------------------------+ + * }}} + * Note that this ordering option is NOT used for the label column. When the label column is + * indexed, it uses the default descending frequency ordering in `StringIndexer`. + * + * @group param + */ + @Since("2.3.0") + final val stringIndexerOrderType: Param[String] = new Param(this, "stringIndexerOrderType", + "How to order categories of a string FEATURE column used by StringIndexer. " + + "The last category after ordering is dropped when encoding strings. " + + s"Supported options: ${StringIndexer.supportedStringOrderType.mkString(", ")}. " + + "The default value is 'frequencyDesc'. When the ordering is set to 'alphabetDesc', " + + "RFormula drops the same category as R when encoding strings.", + ParamValidators.inArray(StringIndexer.supportedStringOrderType)) + setDefault(stringIndexerOrderType, StringIndexer.frequencyDesc) + + /** @group getParam */ + @Since("2.3.0") + def getStringIndexerOrderType: String = $(stringIndexerOrderType) protected def hasLabelCol(schema: StructType): Boolean = { schema.map(_.name).contains($(labelCol)) @@ -45,41 +128,73 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol { * Implements the transforms required for fitting a dataset against an R model formula. Currently * we support a limited subset of the R operators, including '~', '.', ':', '+', and '-'. Also see * the R formula docs here: http://stat.ethz.ch/R-manual/R-patched/library/stats/html/formula.html + * + * The basic operators are: + * - `~` separate target and terms + * - `+` concat terms, "+ 0" means removing intercept + * - `-` remove a term, "- 1" means removing intercept + * - `:` interaction (multiplication for numeric values, or binarized categorical values) + * - `.` all columns except target + * + * Suppose `a` and `b` are double columns, we use the following simple examples + * to illustrate the effect of `RFormula`: + * - `y ~ a + b` means model `y ~ w0 + w1 * a + w2 * b` where `w0` is the intercept and `w1, w2` + * are coefficients. + * - `y ~ a + b + a:b - 1` means model `y ~ w1 * a + w2 * b + w3 * a * b` where `w1, w2, w3` + * are coefficients. + * + * RFormula produces a vector column of features and a double or string column of label. + * Like when formulas are used in R for linear regression, string input columns will be one-hot + * encoded, and numeric columns will be cast to doubles. + * If the label column is of type string, it will be first transformed to double with + * `StringIndexer`. If the label column does not exist in the DataFrame, the output label column + * will be created from the specified response variable in the formula. */ @Experimental -class RFormula(override val uid: String) extends Estimator[RFormulaModel] with RFormulaBase { +@Since("1.5.0") +class RFormula @Since("1.5.0") (@Since("1.5.0") override val uid: String) + extends Estimator[RFormulaModel] with RFormulaBase with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("rFormula")) - /** - * R formula parameter. The formula is provided in string form. - * @group param - */ - val formula: Param[String] = new Param(this, "formula", "R model formula") - /** * Sets the formula to use for this transformer. Must be called before use. * @group setParam * @param value an R formula in string form (e.g. "y ~ x + z") */ + @Since("1.5.0") def setFormula(value: String): this.type = set(formula, value) - /** @group getParam */ - def getFormula: String = $(formula) + /** @group setParam */ + @Since("2.3.0") + def setHandleInvalid(value: String): this.type = set(handleInvalid, value) /** @group setParam */ + @Since("1.5.0") def setFeaturesCol(value: String): this.type = set(featuresCol, value) /** @group setParam */ + @Since("1.5.0") def setLabelCol(value: String): this.type = set(labelCol, value) + /** @group setParam */ + @Since("2.1.0") + def setForceIndexLabel(value: Boolean): this.type = set(forceIndexLabel, value) + + /** @group setParam */ + @Since("2.3.0") + def setStringIndexerOrderType(value: String): this.type = set(stringIndexerOrderType, value) + /** Whether the formula specifies fitting an intercept. */ private[ml] def hasIntercept: Boolean = { require(isDefined(formula), "Formula must be defined first.") RFormulaParser.parse($(formula)).hasIntercept } - override def fit(dataset: DataFrame): RFormulaModel = { + @Since("2.0.0") + override def fit(dataset: Dataset[_]): RFormulaModel = { + transformSchema(dataset.schema, logging = true) require(isDefined(formula), "Formula must be defined first.") val parsedFormula = RFormulaParser.parse($(formula)) val resolvedFormula = parsedFormula.resolve(dataset.schema) @@ -101,6 +216,9 @@ class RFormula(override val uid: String) extends Estimator[RFormulaModel] with R encoderStages += new StringIndexer() .setInputCol(term) .setOutputCol(indexCol) + .setStringOrderType($(stringIndexerOrderType)) + .setHandleInvalid($(handleInvalid)) + prefixesToRewrite(indexCol + "_") = term + "_" (term, indexCol) case _ => (term, term) @@ -108,12 +226,20 @@ class RFormula(override val uid: String) extends Estimator[RFormulaModel] with R }.toMap // Then we handle one-hot encoding and interactions between terms. + var keepReferenceCategory = false val encodedTerms = resolvedFormula.terms.map { case Seq(term) if dataset.schema(term).dataType == StringType => val encodedCol = tmpColumn("onehot") - encoderStages += new OneHotEncoder() + var encoder = new OneHotEncoder() .setInputCol(indexed(term)) .setOutputCol(encodedCol) + // Formula w/o intercept, one of the categories in the first category feature is + // being used as reference category, we will not drop any category for that feature. + if (!hasIntercept && !keepReferenceCategory) { + encoder = encoder.setDropLast(false) + keepReferenceCategory = true + } + encoderStages += encoder prefixesToRewrite(encodedCol + "_") = term + "_" encodedCol case Seq(term) => @@ -133,19 +259,23 @@ class RFormula(override val uid: String) extends Estimator[RFormulaModel] with R encoderStages += new VectorAttributeRewriter($(featuresCol), prefixesToRewrite.toMap) encoderStages += new ColumnPruner(tempColumns.toSet) - if (dataset.schema.fieldNames.contains(resolvedFormula.label) && - dataset.schema(resolvedFormula.label).dataType == StringType) { + if ((dataset.schema.fieldNames.contains(resolvedFormula.label) && + dataset.schema(resolvedFormula.label).dataType == StringType) || $(forceIndexLabel)) { encoderStages += new StringIndexer() .setInputCol(resolvedFormula.label) .setOutputCol($(labelCol)) + .setHandleInvalid($(handleInvalid)) } val pipelineModel = new Pipeline(uid).setStages(encoderStages.toArray).fit(dataset) copyValues(new RFormulaModel(uid, resolvedFormula, pipelineModel).setParent(this)) } + @Since("1.5.0") // optimistic schema; does not contain any ML attributes override def transformSchema(schema: StructType): StructType = { + require(!hasLabelCol(schema) || !$(forceIndexLabel), + "If label column already exists, forceIndexLabel can not be set with true.") if (hasLabelCol(schema)) { StructType(schema.fields :+ StructField($(featuresCol), new VectorUDT, true)) } else { @@ -154,33 +284,47 @@ class RFormula(override val uid: String) extends Estimator[RFormulaModel] with R } } + @Since("1.5.0") override def copy(extra: ParamMap): RFormula = defaultCopy(extra) - override def toString: String = s"RFormula(${get(formula)}) (uid=$uid)" + @Since("2.0.0") + override def toString: String = s"RFormula(${get(formula).getOrElse("")}) (uid=$uid)" +} + +@Since("2.0.0") +object RFormula extends DefaultParamsReadable[RFormula] { + + @Since("2.0.0") + override def load(path: String): RFormula = super.load(path) } /** * :: Experimental :: - * A fitted RFormula. Fitting is required to determine the factor levels of formula terms. + * Model fitted by [[RFormula]]. Fitting is required to determine the factor levels of + * formula terms. + * * @param resolvedFormula the fitted R formula. * @param pipelineModel the fitted feature model, including factor to index mappings. */ @Experimental +@Since("1.5.0") class RFormulaModel private[feature]( - override val uid: String, - resolvedFormula: ResolvedRFormula, - pipelineModel: PipelineModel) - extends Model[RFormulaModel] with RFormulaBase { + @Since("1.5.0") override val uid: String, + private[ml] val resolvedFormula: ResolvedRFormula, + private[ml] val pipelineModel: PipelineModel) + extends Model[RFormulaModel] with RFormulaBase with MLWritable { - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { checkCanTransform(dataset.schema) transformLabel(pipelineModel.transform(dataset)) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { checkCanTransform(schema) val withFeatures = pipelineModel.transformSchema(schema) - if (hasLabelCol(withFeatures)) { + if (resolvedFormula.label.isEmpty || hasLabelCol(withFeatures)) { withFeatures } else if (schema.exists(_.name == resolvedFormula.label)) { val nullable = schema(resolvedFormula.label).dataType match { @@ -195,15 +339,19 @@ class RFormulaModel private[feature]( } } - override def copy(extra: ParamMap): RFormulaModel = copyValues( - new RFormulaModel(uid, resolvedFormula, pipelineModel)) + @Since("1.5.0") + override def copy(extra: ParamMap): RFormulaModel = { + val copied = new RFormulaModel(uid, resolvedFormula, pipelineModel).setParent(parent) + copyValues(copied, extra) + } - override def toString: String = s"RFormulaModel(${resolvedFormula}) (uid=$uid)" + @Since("2.0.0") + override def toString: String = s"RFormulaModel($resolvedFormula) (uid=$uid)" - private def transformLabel(dataset: DataFrame): DataFrame = { + private def transformLabel(dataset: Dataset[_]): DataFrame = { val labelName = resolvedFormula.label - if (hasLabelCol(dataset.schema)) { - dataset + if (labelName.isEmpty || hasLabelCol(dataset.schema)) { + dataset.toDF } else if (dataset.schema.exists(_.name == labelName)) { dataset.schema(labelName).dataType match { case _: NumericType | BooleanType => @@ -214,7 +362,7 @@ class RFormulaModel private[feature]( } else { // Ignore the label field. This is a hack so that this transformer can also work on test // datasets in a Pipeline. - dataset + dataset.toDF } } @@ -222,8 +370,62 @@ class RFormulaModel private[feature]( val columnNames = schema.map(_.name) require(!columnNames.contains($(featuresCol)), "Features column already exists.") require( - !columnNames.contains($(labelCol)) || schema($(labelCol)).dataType == DoubleType, - "Label column already exists and is not of type DoubleType.") + !columnNames.contains($(labelCol)) || schema($(labelCol)).dataType.isInstanceOf[NumericType], + "Label column already exists and is not of type NumericType.") + } + + @Since("2.0.0") + override def write: MLWriter = new RFormulaModel.RFormulaModelWriter(this) +} + +@Since("2.0.0") +object RFormulaModel extends MLReadable[RFormulaModel] { + + @Since("2.0.0") + override def read: MLReader[RFormulaModel] = new RFormulaModelReader + + @Since("2.0.0") + override def load(path: String): RFormulaModel = super.load(path) + + /** [[MLWriter]] instance for [[RFormulaModel]] */ + private[RFormulaModel] class RFormulaModelWriter(instance: RFormulaModel) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + // Save model data: resolvedFormula + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(instance.resolvedFormula)) + .repartition(1).write.parquet(dataPath) + // Save pipeline model + val pmPath = new Path(path, "pipelineModel").toString + instance.pipelineModel.save(pmPath) + } + } + + private class RFormulaModelReader extends MLReader[RFormulaModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[RFormulaModel].getName + + override def load(path: String): RFormulaModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath).select("label", "terms", "hasIntercept").head() + val label = data.getString(0) + val terms = data.getAs[Seq[Seq[String]]](1) + val hasIntercept = data.getBoolean(2) + val resolvedRFormula = ResolvedRFormula(label, terms, hasIntercept) + + val pmPath = new Path(path, "pipelineModel").toString + val pipelineModel = PipelineModel.load(pmPath) + + val model = new RFormulaModel(metadata.uid, resolvedRFormula, pipelineModel) + + DefaultParamsReader.getAndSetParams(model, metadata) + model + } } } @@ -231,12 +433,15 @@ class RFormulaModel private[feature]( * Utility transformer for removing temporary columns from a DataFrame. * TODO(ekl) make this a public transformer */ -private class ColumnPruner(columnsToPrune: Set[String]) extends Transformer { - override val uid = Identifiable.randomUID("columnPruner") +private class ColumnPruner(override val uid: String, val columnsToPrune: Set[String]) + extends Transformer with MLWritable { + + def this(columnsToPrune: Set[String]) = + this(Identifiable.randomUID("columnPruner"), columnsToPrune) - override def transform(dataset: DataFrame): DataFrame = { + override def transform(dataset: Dataset[_]): DataFrame = { val columnsToKeep = dataset.columns.filter(!columnsToPrune.contains(_)) - dataset.select(columnsToKeep.map(dataset.col) : _*) + dataset.select(columnsToKeep.map(dataset.col): _*) } override def transformSchema(schema: StructType): StructType = { @@ -244,6 +449,48 @@ private class ColumnPruner(columnsToPrune: Set[String]) extends Transformer { } override def copy(extra: ParamMap): ColumnPruner = defaultCopy(extra) + + override def write: MLWriter = new ColumnPruner.ColumnPrunerWriter(this) +} + +private object ColumnPruner extends MLReadable[ColumnPruner] { + + override def read: MLReader[ColumnPruner] = new ColumnPrunerReader + + override def load(path: String): ColumnPruner = super.load(path) + + /** [[MLWriter]] instance for [[ColumnPruner]] */ + private[ColumnPruner] class ColumnPrunerWriter(instance: ColumnPruner) extends MLWriter { + + private case class Data(columnsToPrune: Seq[String]) + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + // Save model data: columnsToPrune + val data = Data(instance.columnsToPrune.toSeq) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class ColumnPrunerReader extends MLReader[ColumnPruner] { + + /** Checked against metadata when loading model */ + private val className = classOf[ColumnPruner].getName + + override def load(path: String): ColumnPruner = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath).select("columnsToPrune").head() + val columnsToPrune = data.getAs[Seq[String]](0).toSet + val pruner = new ColumnPruner(metadata.uid, columnsToPrune) + + DefaultParamsReader.getAndSetParams(pruner, metadata) + pruner + } + } } /** @@ -257,25 +504,23 @@ private class ColumnPruner(columnsToPrune: Set[String]) extends Transformer { * by the value in the map. */ private class VectorAttributeRewriter( - vectorCol: String, - prefixesToRewrite: Map[String, String]) - extends Transformer { + override val uid: String, + val vectorCol: String, + val prefixesToRewrite: Map[String, String]) + extends Transformer with MLWritable { - override val uid = Identifiable.randomUID("vectorAttrRewriter") + def this(vectorCol: String, prefixesToRewrite: Map[String, String]) = + this(Identifiable.randomUID("vectorAttrRewriter"), vectorCol, prefixesToRewrite) - override def transform(dataset: DataFrame): DataFrame = { + override def transform(dataset: Dataset[_]): DataFrame = { val metadata = { val group = AttributeGroup.fromStructField(dataset.schema(vectorCol)) val attrs = group.attributes.get.map { attr => if (attr.name.isDefined) { - val name = attr.name.get - val replacement = prefixesToRewrite.filter { case (k, _) => name.startsWith(k) } - if (replacement.nonEmpty) { - val (k, v) = replacement.headOption.get - attr.withName(v + name.stripPrefix(k)) - } else { - attr + val name = prefixesToRewrite.foldLeft(attr.name.get) { case (curName, (from, to)) => + curName.replace(from, to) } + attr.withName(name) } else { attr } @@ -284,7 +529,7 @@ private class VectorAttributeRewriter( } val otherCols = dataset.columns.filter(_ != vectorCol).map(dataset.col) val rewrittenCol = dataset.col(vectorCol).as(vectorCol, metadata) - dataset.select((otherCols :+ rewrittenCol): _*) + dataset.select(otherCols :+ rewrittenCol : _*) } override def transformSchema(schema: StructType): StructType = { @@ -294,4 +539,48 @@ private class VectorAttributeRewriter( } override def copy(extra: ParamMap): VectorAttributeRewriter = defaultCopy(extra) + + override def write: MLWriter = new VectorAttributeRewriter.VectorAttributeRewriterWriter(this) +} + +private object VectorAttributeRewriter extends MLReadable[VectorAttributeRewriter] { + + override def read: MLReader[VectorAttributeRewriter] = new VectorAttributeRewriterReader + + override def load(path: String): VectorAttributeRewriter = super.load(path) + + /** [[MLWriter]] instance for [[VectorAttributeRewriter]] */ + private[VectorAttributeRewriter] + class VectorAttributeRewriterWriter(instance: VectorAttributeRewriter) extends MLWriter { + + private case class Data(vectorCol: String, prefixesToRewrite: Map[String, String]) + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + // Save model data: vectorCol, prefixesToRewrite + val data = Data(instance.vectorCol, instance.prefixesToRewrite) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class VectorAttributeRewriterReader extends MLReader[VectorAttributeRewriter] { + + /** Checked against metadata when loading model */ + private val className = classOf[VectorAttributeRewriter].getName + + override def load(path: String): VectorAttributeRewriter = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath).select("vectorCol", "prefixesToRewrite").head() + val vectorCol = data.getString(0) + val prefixesToRewrite = data.getAs[Map[String, String]](1) + val rewriter = new VectorAttributeRewriter(metadata.uid, vectorCol, prefixesToRewrite) + + DefaultParamsReader.getAndSetParams(rewriter, metadata) + rewriter + } + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala index 4079b387e183..32835fb3aa6d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import scala.collection.mutable import scala.util.parsing.combinator.RegexParsers -import org.apache.spark.mllib.linalg.VectorUDT +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types._ /** @@ -63,6 +63,9 @@ private[ml] case class ParsedRFormula(label: ColumnRef, terms: Seq[Term]) { ResolvedRFormula(label.value, includedTerms.distinct, hasIntercept) } + /** Whether this formula specifies fitting with response variable. */ + def hasLabel: Boolean = label.value.nonEmpty + /** Whether this formula specifies fitting with an intercept term. */ def hasIntercept: Boolean = { var intercept = true @@ -96,7 +99,7 @@ private[ml] case class ParsedRFormula(label: ColumnRef, terms: Seq[Term]) { }).map(_.distinct) // Deduplicates feature interactions, for example, a:b is the same as b:a. - var seen = mutable.Set[Set[String]]() + val seen = mutable.Set[Set[String]]() validInteractions.flatMap { case t if seen.contains(t.toSet) => None @@ -123,7 +126,19 @@ private[ml] case class ParsedRFormula(label: ColumnRef, terms: Seq[Term]) { * @param hasIntercept whether the formula specifies fitting with an intercept. */ private[ml] case class ResolvedRFormula( - label: String, terms: Seq[Seq[String]], hasIntercept: Boolean) + label: String, terms: Seq[Seq[String]], hasIntercept: Boolean) { + + override def toString: String = { + val ts = terms.map { + case t if t.length > 1 => + s"${t.mkString("{", ",", "}")}" + case t => + t.mkString + } + val termStr = ts.mkString("[", ",", "]") + s"ResolvedRFormula(label=$label, terms=$termStr, hasIntercept=$hasIntercept)" + } +} /** * R formula terms. See the R formula docs here for more information: @@ -159,6 +174,10 @@ private[ml] object RFormulaParser extends RegexParsers { private val columnRef: Parser[ColumnRef] = "([a-zA-Z]|\\.[a-zA-Z_])[a-zA-Z0-9._]*".r ^^ { case a => ColumnRef(a) } + private val empty: Parser[ColumnRef] = "" ^^ { case a => ColumnRef("") } + + private val label: Parser[ColumnRef] = columnRef | empty + private val dot: Parser[InteractableTerm] = "\\.".r ^^ { case _ => Dot } private val interaction: Parser[List[InteractableTerm]] = rep1sep(columnRef | dot, ":") @@ -174,7 +193,7 @@ private[ml] object RFormulaParser extends RegexParsers { } private val formula: Parser[ParsedRFormula] = - (columnRef ~ "~" ~ terms) ^^ { case r ~ "~" ~ t => ParsedRFormula(r, t) } + (label ~ "~" ~ terms) ^^ { case r ~ "~" ~ t => ParsedRFormula(r, t) } def parse(value: String): ParsedRFormula = parseAll(formula, value) match { case Success(result, _) => result diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala index 95e430563873..62c1972aab12 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala @@ -17,56 +17,83 @@ package org.apache.spark.ml.feature -import org.apache.spark.SparkContext -import org.apache.spark.annotation.Experimental -import org.apache.spark.ml.param.{ParamMap, Param} +import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer -import org.apache.spark.ml.util.Identifiable -import org.apache.spark.sql.{SQLContext, DataFrame, Row} +import org.apache.spark.ml.param.{Param, ParamMap} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.types.StructType /** - * :: Experimental :: - * Implements the transforms which are defined by SQL statement. - * Currently we only support SQL syntax like 'SELECT ... FROM __THIS__' + * Implements the transformations which are defined by SQL statement. + * Currently we only support SQL syntax like 'SELECT ... FROM __THIS__ ...' * where '__THIS__' represents the underlying table of the input dataset. + * The select clause specifies the fields, constants, and expressions to display in + * the output, it can be any select clause that Spark SQL supports. Users can also + * use Spark SQL built-in function and UDFs to operate on these selected columns. + * For example, [[SQLTransformer]] supports statements like: + * {{{ + * SELECT a, a + b AS a_b FROM __THIS__ + * SELECT a, SQRT(b) AS b_sqrt FROM __THIS__ where a > 5 + * SELECT a, b, SUM(c) AS c_sum FROM __THIS__ GROUP BY a, b + * }}} */ -@Experimental -class SQLTransformer (override val uid: String) extends Transformer { +@Since("1.6.0") +class SQLTransformer @Since("1.6.0") (@Since("1.6.0") override val uid: String) extends Transformer + with DefaultParamsWritable { + @Since("1.6.0") def this() = this(Identifiable.randomUID("sql")) /** * SQL statement parameter. The statement is provided in string form. + * * @group param */ + @Since("1.6.0") final val statement: Param[String] = new Param[String](this, "statement", "SQL statement") /** @group setParam */ + @Since("1.6.0") def setStatement(value: String): this.type = set(statement, value) /** @group getParam */ + @Since("1.6.0") def getStatement: String = $(statement) private val tableIdentifier: String = "__THIS__" - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) val tableName = Identifiable.randomUID(uid) - dataset.registerTempTable(tableName) + dataset.createOrReplaceTempView(tableName) val realStatement = $(statement).replace(tableIdentifier, tableName) - val outputDF = dataset.sqlContext.sql(realStatement) - outputDF + val result = dataset.sparkSession.sql(realStatement) + dataset.sparkSession.catalog.dropTempView(tableName) + result } + @Since("1.6.0") override def transformSchema(schema: StructType): StructType = { - val sc = SparkContext.getOrCreate() - val sqlContext = SQLContext.getOrCreate(sc) - val dummyRDD = sc.parallelize(Seq(Row.empty)) - val dummyDF = sqlContext.createDataFrame(dummyRDD, schema) - dummyDF.registerTempTable(tableIdentifier) - val outputSchema = sqlContext.sql($(statement)).schema + val spark = SparkSession.builder().getOrCreate() + val dummyRDD = spark.sparkContext.parallelize(Seq(Row.empty)) + val dummyDF = spark.createDataFrame(dummyRDD, schema) + val tableName = Identifiable.randomUID(uid) + val realStatement = $(statement).replace(tableIdentifier, tableName) + dummyDF.createOrReplaceTempView(tableName) + val outputSchema = spark.sql(realStatement).schema + spark.catalog.dropTempView(tableName) outputSchema } + @Since("1.6.0") override def copy(extra: ParamMap): SQLTransformer = defaultCopy(extra) } + +@Since("1.6.0") +object SQLTransformer extends DefaultParamsReadable[SQLTransformer] { + + @Since("1.6.0") + override def load(path: String): SQLTransformer = super.load(path) +} 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 f6d0b0c0e9e7..8f125d8fd51d 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 @@ -17,13 +17,19 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since import org.apache.spark.ml._ +import org.apache.spark.ml.linalg.{Vector, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.Identifiable +import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -34,108 +40,186 @@ import org.apache.spark.sql.types.{StructField, StructType} private[feature] trait StandardScalerParams extends Params with HasInputCol with HasOutputCol { /** - * Centers the data with mean before scaling. - * It will build a dense output, so this does not work on sparse input - * and will raise an exception. + * Whether to center the data with mean before scaling. + * It will build a dense output, so take care when applying to sparse input. * Default: false * @group param */ - val withMean: BooleanParam = new BooleanParam(this, "withMean", "Center data with mean") + val withMean: BooleanParam = new BooleanParam(this, "withMean", + "Whether to center data with mean") + + /** @group getParam */ + def getWithMean: Boolean = $(withMean) /** - * Scales the data to unit standard deviation. + * Whether to scale the data to unit standard deviation. * Default: true * @group param */ - val withStd: BooleanParam = new BooleanParam(this, "withStd", "Scale to unit standard deviation") + val withStd: BooleanParam = new BooleanParam(this, "withStd", + "Whether to scale the data to unit standard deviation") + + /** @group getParam */ + def getWithStd: Boolean = $(withStd) + + /** Validates and transforms the input schema. */ + protected def validateAndTransformSchema(schema: StructType): StructType = { + SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) + require(!schema.fieldNames.contains($(outputCol)), + s"Output column ${$(outputCol)} already exists.") + val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) + StructType(outputFields) + } + + setDefault(withMean -> false, withStd -> true) } /** - * :: Experimental :: * Standardizes features by removing the mean and scaling to unit variance using column summary * statistics on the samples in the training set. + * + * The "unit std" is computed using the + * + * corrected sample standard deviation, + * which is computed as the square root of the unbiased sample variance. */ -@Experimental -class StandardScaler(override val uid: String) extends Estimator[StandardScalerModel] - with StandardScalerParams { +@Since("1.2.0") +class StandardScaler @Since("1.4.0") ( + @Since("1.4.0") override val uid: String) + extends Estimator[StandardScalerModel] with StandardScalerParams with DefaultParamsWritable { + @Since("1.2.0") def this() = this(Identifiable.randomUID("stdScal")) - setDefault(withMean -> false, withStd -> true) - /** @group setParam */ + @Since("1.2.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.2.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.4.0") def setWithMean(value: Boolean): this.type = set(withMean, value) /** @group setParam */ + @Since("1.4.0") def setWithStd(value: Boolean): this.type = set(withStd, value) - override def fit(dataset: DataFrame): StandardScalerModel = { + @Since("2.0.0") + override def fit(dataset: Dataset[_]): StandardScalerModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).map { case Row(v: Vector) => v } + val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } val scaler = new feature.StandardScaler(withMean = $(withMean), withStd = $(withStd)) val scalerModel = scaler.fit(input) - copyValues(new StandardScalerModel(uid, scalerModel).setParent(this)) + copyValues(new StandardScalerModel(uid, scalerModel.std, scalerModel.mean).setParent(this)) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { - val inputType = schema($(inputCol)).dataType - require(inputType.isInstanceOf[VectorUDT], - s"Input column ${$(inputCol)} must be a vector column") - require(!schema.fieldNames.contains($(outputCol)), - s"Output column ${$(outputCol)} already exists.") - val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) - StructType(outputFields) + validateAndTransformSchema(schema) } + @Since("1.4.1") override def copy(extra: ParamMap): StandardScaler = defaultCopy(extra) } +@Since("1.6.0") +object StandardScaler extends DefaultParamsReadable[StandardScaler] { + + @Since("1.6.0") + override def load(path: String): StandardScaler = super.load(path) +} + /** - * :: Experimental :: * Model fitted by [[StandardScaler]]. + * + * @param std Standard deviation of the StandardScalerModel + * @param mean Mean of the StandardScalerModel */ -@Experimental +@Since("1.2.0") class StandardScalerModel private[ml] ( - override val uid: String, - scaler: feature.StandardScalerModel) - extends Model[StandardScalerModel] with StandardScalerParams { - - /** Standard deviation of the StandardScalerModel */ - val std: Vector = scaler.std + @Since("1.4.0") override val uid: String, + @Since("2.0.0") val std: Vector, + @Since("2.0.0") val mean: Vector) + extends Model[StandardScalerModel] with StandardScalerParams with MLWritable { - /** Mean of the StandardScalerModel */ - val mean: Vector = scaler.mean + import StandardScalerModel._ /** @group setParam */ + @Since("1.2.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.2.0") def setOutputCol(value: String): this.type = set(outputCol, value) - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val scale = udf { scaler.transform _ } + val scaler = new feature.StandardScalerModel(std, mean, $(withStd), $(withMean)) + + // TODO: Make the transformer natively in ml framework to avoid extra conversion. + val transformer: Vector => Vector = v => scaler.transform(OldVectors.fromML(v)).asML + + val scale = udf(transformer) dataset.withColumn($(outputCol), scale(col($(inputCol)))) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { - val inputType = schema($(inputCol)).dataType - require(inputType.isInstanceOf[VectorUDT], - s"Input column ${$(inputCol)} must be a vector column") - require(!schema.fieldNames.contains($(outputCol)), - s"Output column ${$(outputCol)} already exists.") - val outputFields = schema.fields :+ StructField($(outputCol), new VectorUDT, false) - StructType(outputFields) + validateAndTransformSchema(schema) } + @Since("1.4.1") override def copy(extra: ParamMap): StandardScalerModel = { - val copied = new StandardScalerModel(uid, scaler) + val copied = new StandardScalerModel(uid, std, mean) copyValues(copied, extra).setParent(parent) } + + @Since("1.6.0") + override def write: MLWriter = new StandardScalerModelWriter(this) +} + +@Since("1.6.0") +object StandardScalerModel extends MLReadable[StandardScalerModel] { + + private[StandardScalerModel] + class StandardScalerModelWriter(instance: StandardScalerModel) extends MLWriter { + + private case class Data(std: Vector, mean: Vector) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.std, instance.mean) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class StandardScalerModelReader extends MLReader[StandardScalerModel] { + + private val className = classOf[StandardScalerModel].getName + + override def load(path: String): StandardScalerModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + val Row(std: Vector, mean: Vector) = MLUtils.convertVectorColumnsToML(data, "std", "mean") + .select("std", "mean") + .head() + val model = new StandardScalerModel(metadata.uid, std, mean) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } + + @Since("1.6.0") + override def read: MLReader[StandardScalerModel] = new StandardScalerModelReader + + @Since("1.6.0") + override def load(path: String): StandardScalerModel = super.load(path) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala old mode 100644 new mode 100755 index 2a79582625e9..3fcd84c029e6 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala @@ -17,141 +17,129 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.param.{BooleanParam, ParamMap, StringArrayParam} import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} -import org.apache.spark.ml.util.Identifiable -import org.apache.spark.sql.DataFrame +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} -import org.apache.spark.sql.types.{ArrayType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, StringType, StructType} /** - * stop words list - */ -private[spark] object StopWords { - - /** - * Use the same default stopwords list as scikit-learn. - * The original list can be found from "Glasgow Information Retrieval Group" - * [[http://ir.dcs.gla.ac.uk/resources/linguistic_utils/stop_words]] - */ - val English = Array( "a", "about", "above", "across", "after", "afterwards", "again", - "against", "all", "almost", "alone", "along", "already", "also", "although", "always", - "am", "among", "amongst", "amoungst", "amount", "an", "and", "another", - "any", "anyhow", "anyone", "anything", "anyway", "anywhere", "are", - "around", "as", "at", "back", "be", "became", "because", "become", - "becomes", "becoming", "been", "before", "beforehand", "behind", "being", - "below", "beside", "besides", "between", "beyond", "bill", "both", - "bottom", "but", "by", "call", "can", "cannot", "cant", "co", "con", - "could", "couldnt", "cry", "de", "describe", "detail", "do", "done", - "down", "due", "during", "each", "eg", "eight", "either", "eleven", "else", - "elsewhere", "empty", "enough", "etc", "even", "ever", "every", "everyone", - "everything", "everywhere", "except", "few", "fifteen", "fify", "fill", - "find", "fire", "first", "five", "for", "former", "formerly", "forty", - "found", "four", "from", "front", "full", "further", "get", "give", "go", - "had", "has", "hasnt", "have", "he", "hence", "her", "here", "hereafter", - "hereby", "herein", "hereupon", "hers", "herself", "him", "himself", "his", - "how", "however", "hundred", "i", "ie", "if", "in", "inc", "indeed", - "interest", "into", "is", "it", "its", "itself", "keep", "last", "latter", - "latterly", "least", "less", "ltd", "made", "many", "may", "me", - "meanwhile", "might", "mill", "mine", "more", "moreover", "most", "mostly", - "move", "much", "must", "my", "myself", "name", "namely", "neither", - "never", "nevertheless", "next", "nine", "no", "nobody", "none", "noone", - "nor", "not", "nothing", "now", "nowhere", "of", "off", "often", "on", - "once", "one", "only", "onto", "or", "other", "others", "otherwise", "our", - "ours", "ourselves", "out", "over", "own", "part", "per", "perhaps", - "please", "put", "rather", "re", "same", "see", "seem", "seemed", - "seeming", "seems", "serious", "several", "she", "should", "show", "side", - "since", "sincere", "six", "sixty", "so", "some", "somehow", "someone", - "something", "sometime", "sometimes", "somewhere", "still", "such", - "system", "take", "ten", "than", "that", "the", "their", "them", - "themselves", "then", "thence", "there", "thereafter", "thereby", - "therefore", "therein", "thereupon", "these", "they", "thick", "thin", - "third", "this", "those", "though", "three", "through", "throughout", - "thru", "thus", "to", "together", "too", "top", "toward", "towards", - "twelve", "twenty", "two", "un", "under", "until", "up", "upon", "us", - "very", "via", "was", "we", "well", "were", "what", "whatever", "when", - "whence", "whenever", "where", "whereafter", "whereas", "whereby", - "wherein", "whereupon", "wherever", "whether", "which", "while", "whither", - "who", "whoever", "whole", "whom", "whose", "why", "will", "with", - "within", "without", "would", "yet", "you", "your", "yours", "yourself", "yourselves") -} - -/** - * :: Experimental :: * A feature transformer that filters out stop words from input. - * Note: null values from input array are preserved unless adding null to stopWords explicitly. - * @see [[http://en.wikipedia.org/wiki/Stop_words]] + * + * @note null values from input array are preserved unless adding null to stopWords + * explicitly. + * + * @see Stop words (Wikipedia) */ -@Experimental -class StopWordsRemover(override val uid: String) - extends Transformer with HasInputCol with HasOutputCol { +@Since("1.5.0") +class StopWordsRemover @Since("1.5.0") (@Since("1.5.0") override val uid: String) + extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("stopWords")) /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** - * the stop words set to be filtered out - * Default: [[StopWords.English]] + * The words to be filtered out. + * Default: English stop words + * @see `StopWordsRemover.loadDefaultStopWords()` * @group param */ - val stopWords: StringArrayParam = new StringArrayParam(this, "stopWords", "stop words") + @Since("1.5.0") + val stopWords: StringArrayParam = + new StringArrayParam(this, "stopWords", "the words to be filtered out") /** @group setParam */ + @Since("1.5.0") def setStopWords(value: Array[String]): this.type = set(stopWords, value) /** @group getParam */ + @Since("1.5.0") def getStopWords: Array[String] = $(stopWords) /** - * whether to do a case sensitive comparison over the stop words + * Whether to do a case sensitive comparison over the stop words. * Default: false * @group param */ + @Since("1.5.0") val caseSensitive: BooleanParam = new BooleanParam(this, "caseSensitive", - "whether to do case-sensitive comparison during filtering") + "whether to do a case-sensitive comparison over the stop words") /** @group setParam */ + @Since("1.5.0") def setCaseSensitive(value: Boolean): this.type = set(caseSensitive, value) /** @group getParam */ + @Since("1.5.0") def getCaseSensitive: Boolean = $(caseSensitive) - setDefault(stopWords -> StopWords.English, caseSensitive -> false) + setDefault(stopWords -> StopWordsRemover.loadDefaultStopWords("english"), caseSensitive -> false) - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { val outputSchema = transformSchema(dataset.schema) val t = if ($(caseSensitive)) { - val stopWordsSet = $(stopWords).toSet - udf { terms: Seq[String] => - terms.filter(s => !stopWordsSet.contains(s)) - } - } else { - val toLower = (s: String) => if (s != null) s.toLowerCase else s - val lowerStopWords = $(stopWords).map(toLower(_)).toSet - udf { terms: Seq[String] => - terms.filter(s => !lowerStopWords.contains(toLower(s))) - } + val stopWordsSet = $(stopWords).toSet + udf { terms: Seq[String] => + terms.filter(s => !stopWordsSet.contains(s)) + } + } else { + // TODO: support user locale (SPARK-15064) + val toLower = (s: String) => if (s != null) s.toLowerCase else s + val lowerStopWords = $(stopWords).map(toLower(_)).toSet + udf { terms: Seq[String] => + terms.filter(s => !lowerStopWords.contains(toLower(s))) + } } - val metadata = outputSchema($(outputCol)).metadata dataset.select(col("*"), t(col($(inputCol))).as($(outputCol), metadata)) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { val inputType = schema($(inputCol)).dataType require(inputType.sameType(ArrayType(StringType)), s"Input type must be ArrayType(StringType) but got $inputType.") - val outputFields = schema.fields :+ - StructField($(outputCol), inputType, schema($(inputCol)).nullable) - StructType(outputFields) + SchemaUtils.appendColumn(schema, $(outputCol), inputType, schema($(inputCol)).nullable) } + @Since("1.5.0") override def copy(extra: ParamMap): StopWordsRemover = defaultCopy(extra) } + +@Since("1.6.0") +object StopWordsRemover extends DefaultParamsReadable[StopWordsRemover] { + + private[feature] + val supportedLanguages = Set("danish", "dutch", "english", "finnish", "french", "german", + "hungarian", "italian", "norwegian", "portuguese", "russian", "spanish", "swedish", "turkish") + + @Since("1.6.0") + override def load(path: String): StopWordsRemover = super.load(path) + + /** + * Loads the default stop words for the given language. + * Supported languages: danish, dutch, english, finnish, french, german, hungarian, + * italian, norwegian, portuguese, russian, spanish, swedish, turkish + * @see + * here + */ + @Since("2.0.0") + def loadDefaultStopWords(language: String): Array[String] = { + require(supportedLanguages.contains(language), + s"$language is not in the supported language list: ${supportedLanguages.mkString(", ")}.") + val is = getClass.getResourceAsStream(s"/org/apache/spark/ml/feature/stopwords/$language.txt") + scala.io.Source.fromInputStream(is)(scala.io.Codec.UTF8).getLines().toArray + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index 486274cd75a1..2679ec310c47 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -17,15 +17,18 @@ package org.apache.spark.ml.feature +import scala.language.existentials + +import org.apache.hadoop.fs.Path + import org.apache.spark.SparkException -import org.apache.spark.annotation.Experimental -import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.annotation.Since +import org.apache.spark.ml.{Estimator, Model, Transformer} import org.apache.spark.ml.attribute.{Attribute, NominalAttribute} import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.Transformer -import org.apache.spark.ml.util.Identifiable -import org.apache.spark.sql.DataFrame +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol, HasOutputCol} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.apache.spark.util.collection.OpenHashMap @@ -33,8 +36,48 @@ import org.apache.spark.util.collection.OpenHashMap /** * Base trait for [[StringIndexer]] and [[StringIndexerModel]]. */ -private[feature] trait StringIndexerBase extends Params with HasInputCol with HasOutputCol - with HasHandleInvalid { +private[feature] trait StringIndexerBase extends Params with HasHandleInvalid with HasInputCol + with HasOutputCol { + + /** + * Param for how to handle invalid data (unseen labels or NULL values). + * Options are 'skip' (filter out rows with invalid data), + * 'error' (throw an error), or 'keep' (put invalid data in a special additional + * bucket, at index numLabels). + * Default: "error" + * @group param + */ + @Since("1.6.0") + override val handleInvalid: Param[String] = new Param[String](this, "handleInvalid", + "How to handle invalid data (unseen labels or NULL values). " + + "Options are 'skip' (filter out rows with invalid data), error (throw an error), " + + "or 'keep' (put invalid data in a special additional bucket, at index numLabels).", + ParamValidators.inArray(StringIndexer.supportedHandleInvalids)) + + setDefault(handleInvalid, StringIndexer.ERROR_INVALID) + + /** + * Param for how to order labels of string column. The first label after ordering is assigned + * an index of 0. + * Options are: + * - 'frequencyDesc': descending order by label frequency (most frequent label assigned 0) + * - 'frequencyAsc': ascending order by label frequency (least frequent label assigned 0) + * - 'alphabetDesc': descending alphabetical order + * - 'alphabetAsc': ascending alphabetical order + * Default is 'frequencyDesc'. + * + * @group param + */ + @Since("2.3.0") + final val stringOrderType: Param[String] = new Param(this, "stringOrderType", + "How to order labels of string column. " + + "The first label after ordering is assigned an index of 0. " + + s"Supported options: ${StringIndexer.supportedStringOrderType.mkString(", ")}.", + ParamValidators.inArray(StringIndexer.supportedStringOrderType)) + + /** @group getParam */ + @Since("2.3.0") + def getStringOrderType: String = $(stringOrderType) /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType): StructType = { @@ -54,61 +97,101 @@ private[feature] trait StringIndexerBase extends Params with HasInputCol with Ha } /** - * :: Experimental :: * A label indexer that maps a string column of labels to an ML column of label indices. * If the input column is numeric, we cast it to string and index the string values. - * The indices are in [0, numLabels), ordered by label frequencies. - * So the most frequent label gets index 0. + * The indices are in [0, numLabels). By default, this is ordered by label frequencies + * so the most frequent label gets index 0. The ordering behavior is controlled by + * setting `stringOrderType`. * - * @see [[IndexToString]] for the inverse transformation + * @see `IndexToString` for the inverse transformation */ -@Experimental -class StringIndexer(override val uid: String) extends Estimator[StringIndexerModel] - with StringIndexerBase { +@Since("1.4.0") +class StringIndexer @Since("1.4.0") ( + @Since("1.4.0") override val uid: String) extends Estimator[StringIndexerModel] + with StringIndexerBase with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("strIdx")) /** @group setParam */ + @Since("1.6.0") def setHandleInvalid(value: String): this.type = set(handleInvalid, value) - setDefault(handleInvalid, "error") /** @group setParam */ + @Since("2.3.0") + def setStringOrderType(value: String): this.type = set(stringOrderType, value) + setDefault(stringOrderType, StringIndexer.frequencyDesc) + + /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) - - override def fit(dataset: DataFrame): StringIndexerModel = { - val counts = dataset.select(col($(inputCol)).cast(StringType)) - .map(_.getString(0)) - .countByValue() - val labels = counts.toSeq.sortBy(-_._2).map(_._1).toArray + @Since("2.0.0") + override def fit(dataset: Dataset[_]): StringIndexerModel = { + transformSchema(dataset.schema, logging = true) + val values = dataset.na.drop(Array($(inputCol))) + .select(col($(inputCol)).cast(StringType)) + .rdd.map(_.getString(0)) + val labels = $(stringOrderType) match { + case StringIndexer.frequencyDesc => values.countByValue().toSeq.sortBy(-_._2) + .map(_._1).toArray + case StringIndexer.frequencyAsc => values.countByValue().toSeq.sortBy(_._2) + .map(_._1).toArray + case StringIndexer.alphabetDesc => values.distinct.collect.sortWith(_ > _) + case StringIndexer.alphabetAsc => values.distinct.collect.sortWith(_ < _) + } copyValues(new StringIndexerModel(uid, labels).setParent(this)) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.4.1") override def copy(extra: ParamMap): StringIndexer = defaultCopy(extra) } +@Since("1.6.0") +object StringIndexer extends DefaultParamsReadable[StringIndexer] { + private[feature] val SKIP_INVALID: String = "skip" + private[feature] val ERROR_INVALID: String = "error" + private[feature] val KEEP_INVALID: String = "keep" + private[feature] val supportedHandleInvalids: Array[String] = + Array(SKIP_INVALID, ERROR_INVALID, KEEP_INVALID) + private[feature] val frequencyDesc: String = "frequencyDesc" + private[feature] val frequencyAsc: String = "frequencyAsc" + private[feature] val alphabetDesc: String = "alphabetDesc" + private[feature] val alphabetAsc: String = "alphabetAsc" + private[feature] val supportedStringOrderType: Array[String] = + Array(frequencyDesc, frequencyAsc, alphabetDesc, alphabetAsc) + + @Since("1.6.0") + override def load(path: String): StringIndexer = super.load(path) +} + /** - * :: Experimental :: * Model fitted by [[StringIndexer]]. * - * NOTE: During transformation, if the input column does not exist, - * [[StringIndexerModel.transform]] would return the input dataset unmodified. - * This is a temporary fix for the case when target labels do not exist during prediction. - * * @param labels Ordered list of labels, corresponding to indices to be assigned. + * + * @note During transformation, if the input column does not exist, + * `StringIndexerModel.transform` would return the input dataset unmodified. + * This is a temporary fix for the case when target labels do not exist during prediction. */ -@Experimental +@Since("1.4.0") class StringIndexerModel ( - override val uid: String, - val labels: Array[String]) extends Model[StringIndexerModel] with StringIndexerBase { + @Since("1.4.0") override val uid: String, + @Since("1.5.0") val labels: Array[String]) + extends Model[StringIndexerModel] with StringIndexerBase with MLWritable { + import StringIndexerModel._ + + @Since("1.5.0") def this(labels: Array[String]) = this(Identifiable.randomUID("strIdx"), labels) private val labelToIndex: OpenHashMap[String, Double] = { @@ -123,46 +206,68 @@ class StringIndexerModel ( } /** @group setParam */ + @Since("1.6.0") def setHandleInvalid(value: String): this.type = set(handleInvalid, value) - setDefault(handleInvalid, "error") /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { if (!dataset.schema.fieldNames.contains($(inputCol))) { logInfo(s"Input column ${$(inputCol)} does not exist during transformation. " + "Skip StringIndexerModel.") - return dataset + return dataset.toDF } + transformSchema(dataset.schema, logging = true) - val indexer = udf { label: String => - if (labelToIndex.contains(label)) { - labelToIndex(label) - } else { - throw new SparkException(s"Unseen label: $label.") - } + val filteredLabels = getHandleInvalid match { + case StringIndexer.KEEP_INVALID => labels :+ "__unknown" + case _ => labels } val metadata = NominalAttribute.defaultAttr - .withName($(inputCol)).withValues(labels).toMetadata() + .withName($(outputCol)).withValues(filteredLabels).toMetadata() // If we are skipping invalid records, filter them out. - val filteredDataset = (getHandleInvalid) match { - case "skip" => { + val (filteredDataset, keepInvalid) = getHandleInvalid match { + case StringIndexer.SKIP_INVALID => val filterer = udf { label: String => labelToIndex.contains(label) } - dataset.where(filterer(dataset($(inputCol)))) + (dataset.na.drop(Array($(inputCol))).where(filterer(dataset($(inputCol)))), false) + case _ => (dataset, getHandleInvalid == StringIndexer.KEEP_INVALID) + } + + val indexer = udf { label: String => + if (label == null) { + if (keepInvalid) { + labels.length + } else { + throw new SparkException("StringIndexer encountered NULL value. To handle or skip " + + "NULLS, try setting StringIndexer.handleInvalid.") + } + } else { + if (labelToIndex.contains(label)) { + labelToIndex(label) + } else if (keepInvalid) { + labels.length + } else { + throw new SparkException(s"Unseen label: $label. To handle unseen labels, " + + s"set Param handleInvalid to ${StringIndexer.KEEP_INVALID}.") + } } - case _ => dataset } + filteredDataset.select(col("*"), indexer(dataset($(inputCol)).cast(StringType)).as($(outputCol), metadata)) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { if (schema.fieldNames.contains($(inputCol))) { validateAndTransformSchema(schema) @@ -172,52 +277,100 @@ class StringIndexerModel ( } } + @Since("1.4.1") override def copy(extra: ParamMap): StringIndexerModel = { val copied = new StringIndexerModel(uid, labels) copyValues(copied, extra).setParent(parent) } + + @Since("1.6.0") + override def write: StringIndexModelWriter = new StringIndexModelWriter(this) +} + +@Since("1.6.0") +object StringIndexerModel extends MLReadable[StringIndexerModel] { + + private[StringIndexerModel] + class StringIndexModelWriter(instance: StringIndexerModel) extends MLWriter { + + private case class Data(labels: Array[String]) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.labels) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class StringIndexerModelReader extends MLReader[StringIndexerModel] { + + private val className = classOf[StringIndexerModel].getName + + override def load(path: String): StringIndexerModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + .select("labels") + .head() + val labels = data.getAs[Seq[String]](0).toArray + val model = new StringIndexerModel(metadata.uid, labels) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } + + @Since("1.6.0") + override def read: MLReader[StringIndexerModel] = new StringIndexerModelReader + + @Since("1.6.0") + override def load(path: String): StringIndexerModel = super.load(path) } /** - * :: Experimental :: - * A [[Transformer]] that maps a column of indices back to a new column of corresponding + * A `Transformer` that maps a column of indices back to a new column of corresponding * string values. * The index-string mapping is either from the ML attributes of the input column, * or from user-supplied labels (which take precedence over ML attributes). * - * @see [[StringIndexer]] for converting strings into indices + * @see `StringIndexer` for converting strings into indices */ -@Experimental -class IndexToString private[ml] ( - override val uid: String) extends Transformer - with HasInputCol with HasOutputCol { +@Since("1.5.0") +class IndexToString @Since("2.2.0") (@Since("1.5.0") override val uid: String) + extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("idxToStr")) /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.5.0") def setLabels(value: Array[String]): this.type = set(labels, value) /** * Optional param for array of labels specifying index-string mapping. * - * Default: Empty array, in which case [[inputCol]] metadata is used for labels. + * Default: Not specified, in which case [[inputCol]] metadata is used for labels. * @group param */ + @Since("1.5.0") final val labels: StringArrayParam = new StringArrayParam(this, "labels", "Optional array of labels specifying index-string mapping." + " If not provided or if empty, then metadata from inputCol is used instead.") - setDefault(labels, Array.empty[String]) /** @group getParam */ + @Since("1.5.0") final def getLabels: Array[String] = $(labels) + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { val inputColName = $(inputCol) val inputDataType = schema(inputColName).dataType @@ -232,10 +385,12 @@ class IndexToString private[ml] ( StructType(outputFields) } - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) val inputColSchema = dataset.schema($(inputCol)) // If the labels array is empty use column metadata - val values = if ($(labels).isEmpty) { + val values = if (!isDefined(labels) || $(labels).isEmpty) { Attribute.fromStructField(inputColSchema) .asInstanceOf[NominalAttribute].values.get } else { @@ -254,7 +409,15 @@ class IndexToString private[ml] ( indexer(dataset($(inputCol)).cast(DoubleType)).as(outputColName)) } + @Since("1.5.0") override def copy(extra: ParamMap): IndexToString = { defaultCopy(extra) } } + +@Since("1.6.0") +object IndexToString extends DefaultParamsReadable[IndexToString] { + + @Since("1.6.0") + override def load(path: String): IndexToString = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 248288ca73e9..cfaf6c0e610b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -17,21 +17,22 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.param._ -import org.apache.spark.ml.util.Identifiable +import org.apache.spark.ml.util._ import org.apache.spark.sql.types.{ArrayType, DataType, StringType} /** - * :: Experimental :: * A tokenizer that converts the input string to lowercase and then splits it by white spaces. * * @see [[RegexTokenizer]] */ -@Experimental -class Tokenizer(override val uid: String) extends UnaryTransformer[String, Seq[String], Tokenizer] { +@Since("1.2.0") +class Tokenizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) + extends UnaryTransformer[String, Seq[String], Tokenizer] with DefaultParamsWritable { + @Since("1.2.0") def this() = this(Identifiable.randomUID("tok")) override protected def createTransformFunc: String => Seq[String] = { @@ -44,34 +45,45 @@ class Tokenizer(override val uid: String) extends UnaryTransformer[String, Seq[S override protected def outputDataType: DataType = new ArrayType(StringType, true) + @Since("1.4.1") override def copy(extra: ParamMap): Tokenizer = defaultCopy(extra) } +@Since("1.6.0") +object Tokenizer extends DefaultParamsReadable[Tokenizer] { + + @Since("1.6.0") + override def load(path: String): Tokenizer = super.load(path) +} + /** - * :: Experimental :: * A regex based tokenizer that extracts tokens either by using the provided regex pattern to split * the text (default) or repeatedly matching the regex (if `gaps` is false). * Optional parameters also allow filtering tokens using a minimal length. * It returns an array of strings that can be empty. */ -@Experimental -class RegexTokenizer(override val uid: String) - extends UnaryTransformer[String, Seq[String], RegexTokenizer] { +@Since("1.4.0") +class RegexTokenizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) + extends UnaryTransformer[String, Seq[String], RegexTokenizer] with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("regexTok")) /** - * Minimum token length, >= 0. + * Minimum token length, greater than or equal to 0. * Default: 1, to avoid returning empty strings * @group param */ + @Since("1.4.0") val minTokenLength: IntParam = new IntParam(this, "minTokenLength", "minimum token length (>= 0)", ParamValidators.gtEq(0)) /** @group setParam */ + @Since("1.4.0") def setMinTokenLength(value: Int): this.type = set(minTokenLength, value) /** @group getParam */ + @Since("1.4.0") def getMinTokenLength: Int = $(minTokenLength) /** @@ -79,12 +91,15 @@ class RegexTokenizer(override val uid: String) * Default: true * @group param */ + @Since("1.4.0") val gaps: BooleanParam = new BooleanParam(this, "gaps", "Set regex to match gaps or tokens") /** @group setParam */ + @Since("1.4.0") def setGaps(value: Boolean): this.type = set(gaps, value) /** @group getParam */ + @Since("1.4.0") def getGaps: Boolean = $(gaps) /** @@ -92,18 +107,39 @@ class RegexTokenizer(override val uid: String) * Default: `"\\s+"` * @group param */ + @Since("1.4.0") val pattern: Param[String] = new Param(this, "pattern", "regex pattern used for tokenizing") /** @group setParam */ + @Since("1.4.0") def setPattern(value: String): this.type = set(pattern, value) /** @group getParam */ + @Since("1.4.0") def getPattern: String = $(pattern) - setDefault(minTokenLength -> 1, gaps -> true, pattern -> "\\s+") + /** + * Indicates whether to convert all characters to lowercase before tokenizing. + * Default: true + * @group param + */ + @Since("1.6.0") + final val toLowercase: BooleanParam = new BooleanParam(this, "toLowercase", + "whether to convert all characters to lowercase before tokenizing.") - override protected def createTransformFunc: String => Seq[String] = { str => + /** @group setParam */ + @Since("1.6.0") + def setToLowercase(value: Boolean): this.type = set(toLowercase, value) + + /** @group getParam */ + @Since("1.6.0") + def getToLowercase: Boolean = $(toLowercase) + + setDefault(minTokenLength -> 1, gaps -> true, pattern -> "\\s+", toLowercase -> true) + + override protected def createTransformFunc: String => Seq[String] = { originStr => val re = $(pattern).r + val str = if ($(toLowercase)) originStr.toLowerCase() else originStr val tokens = if ($(gaps)) re.split(str).toSeq else re.findAllIn(str).toSeq val minLength = $(minTokenLength) tokens.filter(_.length >= minLength) @@ -115,5 +151,13 @@ class RegexTokenizer(override val uid: String) override protected def outputDataType: DataType = new ArrayType(StringType, true) + @Since("1.4.1") override def copy(extra: ParamMap): RegexTokenizer = defaultCopy(extra) } + +@Since("1.6.0") +object RegexTokenizer extends DefaultParamsReadable[RegexTokenizer] { + + @Since("1.6.0") + override def load(path: String): RegexTokenizer = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala index 086917fa680f..73f27d1a423d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -20,37 +20,41 @@ package org.apache.spark.ml.feature import scala.collection.mutable.ArrayBuilder import org.apache.spark.SparkException -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute, UnresolvedAttribute} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.linalg.{Vector, VectorUDT, Vectors} -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ /** - * :: Experimental :: * A feature transformer that merges multiple columns into a vector column. */ -@Experimental -class VectorAssembler(override val uid: String) - extends Transformer with HasInputCols with HasOutputCol { +@Since("1.4.0") +class VectorAssembler @Since("1.4.0") (@Since("1.4.0") override val uid: String) + extends Transformer with HasInputCols with HasOutputCol with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("vecAssembler")) /** @group setParam */ + @Since("1.4.0") def setInputCols(value: Array[String]): this.type = set(inputCols, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) // Schema transformation. val schema = dataset.schema - lazy val first = dataset.first() + lazy val first = dataset.toDF.first() val attrs = $(inputCols).flatMap { c => val field = schema(c) val index = schema.fieldIndex(c) @@ -70,20 +74,22 @@ class VectorAssembler(override val uid: String) val group = AttributeGroup.fromStructField(field) if (group.attributes.isDefined) { // If attributes are defined, copy them with updated names. - group.attributes.get.map { attr => + group.attributes.get.zipWithIndex.map { case (attr, i) => if (attr.name.isDefined) { // TODO: Define a rigorous naming scheme. attr.withName(c + "_" + attr.name.get) } else { - attr + attr.withName(c + "_" + i) } } } else { // Otherwise, treat all attributes as numeric. If we cannot get the number of attributes // from metadata, check the first row. val numAttrs = group.numAttributes.getOrElse(first.getAs[Vector](index).size) - Array.fill(numAttrs)(NumericAttribute.defaultAttr) + Array.tabulate(numAttrs)(i => NumericAttribute.defaultAttr.withName(c + "_" + i)) } + case otherType => + throw new SparkException(s"VectorAssembler does not support the $otherType type") } } val metadata = new AttributeGroup($(outputCol), attrs).toMetadata() @@ -100,18 +106,22 @@ class VectorAssembler(override val uid: String) } } - dataset.select(col("*"), assembleFunc(struct(args : _*)).as($(outputCol), metadata)) + dataset.select(col("*"), assembleFunc(struct(args: _*)).as($(outputCol), metadata)) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { val inputColNames = $(inputCols) val outputColName = $(outputCol) - val inputDataTypes = inputColNames.map(name => schema(name).dataType) - inputDataTypes.foreach { - case _: NumericType | BooleanType => - case t if t.isInstanceOf[VectorUDT] => - case other => - throw new IllegalArgumentException(s"Data type $other is not supported.") + val incorrectColumns = inputColNames.flatMap { name => + schema(name).dataType match { + case _: NumericType | BooleanType => None + case t if t.isInstanceOf[VectorUDT] => None + case other => Some(s"Data type $other of column $name is not supported.") + } + } + if (incorrectColumns.nonEmpty) { + throw new IllegalArgumentException(incorrectColumns.mkString("\n")) } if (schema.fieldNames.contains(outputColName)) { throw new IllegalArgumentException(s"Output column $outputColName already exists.") @@ -119,10 +129,15 @@ class VectorAssembler(override val uid: String) StructType(schema.fields :+ new StructField(outputColName, new VectorUDT, true)) } + @Since("1.4.1") override def copy(extra: ParamMap): VectorAssembler = defaultCopy(extra) } -private object VectorAssembler { +@Since("1.6.0") +object VectorAssembler extends DefaultParamsReadable[VectorAssembler] { + + @Since("1.6.0") + override def load(path: String): VectorAssembler = super.load(path) private[feature] def assemble(vv: Any*): Vector = { val indices = ArrayBuilder.make[Int] diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala index 52e0599e38d8..d371da762c55 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala @@ -22,14 +22,16 @@ import java.util.{Map => JMap} import scala.collection.JavaConverters._ -import org.apache.spark.annotation.Experimental +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.attribute._ -import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators, Params} +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, VectorUDT} +import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.{Identifiable, SchemaUtils} -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, VectorUDT} -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.udf import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.util.collection.OpenHashSet @@ -39,8 +41,8 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu /** * Threshold for the number of values a categorical feature can take. - * If a feature is found to have > maxCategories values, then it is declared continuous. - * Must be >= 2. + * If a feature is found to have {@literal >} maxCategories values, then it is declared + * continuous. Must be greater than or equal to 2. * * (default = 20) * @group param @@ -57,8 +59,7 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu } /** - * :: Experimental :: - * Class for indexing categorical feature columns in a dataset of [[Vector]]. + * Class for indexing categorical feature columns in a dataset of `Vector`. * * This has 2 usage modes: * - Automatically identify categorical features (default behavior) @@ -75,7 +76,8 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu * - Warning: This can cause problems if features are continuous since this will collect ALL * unique values to the driver. * - E.g.: Feature 0 has unique values {-1.0, 0.0}, and feature 1 values {1.0, 3.0, 5.0}. - * If maxCategories >= 3, then both features will be declared categorical. + * If maxCategories is greater than or equal to 3, then both features will be declared + * categorical. * * This returns a model which can transform categorical features to use 0-based indices. * @@ -91,27 +93,33 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu * - Add warning if a categorical feature has only 1 category. * - Add option for allowing unknown categories. */ -@Experimental -class VectorIndexer(override val uid: String) extends Estimator[VectorIndexerModel] - with VectorIndexerParams { +@Since("1.4.0") +class VectorIndexer @Since("1.4.0") ( + @Since("1.4.0") override val uid: String) + extends Estimator[VectorIndexerModel] with VectorIndexerParams with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("vecIdx")) /** @group setParam */ + @Since("1.4.0") def setMaxCategories(value: Int): this.type = set(maxCategories, value) /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) - override def fit(dataset: DataFrame): VectorIndexerModel = { + @Since("2.0.0") + override def fit(dataset: Dataset[_]): VectorIndexerModel = { transformSchema(dataset.schema, logging = true) val firstRow = dataset.select($(inputCol)).take(1) require(firstRow.length == 1, s"VectorIndexer cannot be fit on an empty dataset.") val numFeatures = firstRow(0).getAs[Vector](0).size - val vectorDataset = dataset.select($(inputCol)).map { case Row(v: Vector) => v } + val vectorDataset = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => v } val maxCats = $(maxCategories) val categoryStats: VectorIndexer.CategoryStats = vectorDataset.mapPartitions { iter => val localCatStats = new VectorIndexer.CategoryStats(numFeatures, maxCats) @@ -123,6 +131,7 @@ class VectorIndexer(override val uid: String) extends Estimator[VectorIndexerMod copyValues(model) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { // We do not transfer feature metadata since we do not know what types of features we will // produce in transform(). @@ -133,10 +142,15 @@ class VectorIndexer(override val uid: String) extends Estimator[VectorIndexerMod SchemaUtils.appendColumn(schema, $(outputCol), dataType) } + @Since("1.4.1") override def copy(extra: ParamMap): VectorIndexer = defaultCopy(extra) } -private object VectorIndexer { +@Since("1.6.0") +object VectorIndexer extends DefaultParamsReadable[VectorIndexer] { + + @Since("1.6.0") + override def load(path: String): VectorIndexer = super.load(path) /** * Helper class for tracking unique values for each feature. @@ -146,7 +160,7 @@ private object VectorIndexer { * @param numFeatures This class fails if it encounters a Vector whose length is not numFeatures. * @param maxCategories This class caps the number of unique values collected at maxCategories. */ - class CategoryStats(private val numFeatures: Int, private val maxCategories: Int) + private class CategoryStats(private val numFeatures: Int, private val maxCategories: Int) extends Serializable { /** featureValueSets[feature index] = set of unique values */ @@ -232,8 +246,8 @@ private object VectorIndexer { } /** - * :: Experimental :: - * Transform categorical features to use 0-based indices instead of their original values. + * Model fitted by [[VectorIndexer]]. Transform categorical features to use 0-based indices + * instead of their original values. * - Categorical features are mapped to indices. * - Continuous features (columns) are left unchanged. * This also appends metadata to the output column, marking features as Numeric (continuous), @@ -247,14 +261,17 @@ private object VectorIndexer { * Values are maps from original features values to 0-based category indices. * If a feature is not in this map, it is treated as continuous. */ -@Experimental +@Since("1.4.0") class VectorIndexerModel private[ml] ( - override val uid: String, - val numFeatures: Int, - val categoryMaps: Map[Int, Map[Double, Int]]) - extends Model[VectorIndexerModel] with VectorIndexerParams { + @Since("1.4.0") override val uid: String, + @Since("1.4.0") val numFeatures: Int, + @Since("1.4.0") val categoryMaps: Map[Int, Map[Double, Int]]) + extends Model[VectorIndexerModel] with VectorIndexerParams with MLWritable { + + import VectorIndexerModel._ /** Java-friendly version of [[categoryMaps]] */ + @Since("1.4.0") def javaCategoryMaps: JMap[JInt, JMap[JDouble, JInt]] = { categoryMaps.mapValues(_.asJava).asJava.asInstanceOf[JMap[JInt, JMap[JDouble, JInt]]] } @@ -332,12 +349,15 @@ class VectorIndexerModel private[ml] ( } /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) val newField = prepOutputField(dataset.schema) val transformUDF = udf { (vector: Vector) => transformFunc(vector) } @@ -345,6 +365,7 @@ class VectorIndexerModel private[ml] ( dataset.withColumn($(outputCol), newCol, newField.metadata) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { val dataType = new VectorUDT require(isDefined(inputCol), @@ -404,8 +425,53 @@ class VectorIndexerModel private[ml] ( newAttributeGroup.toStructField() } + @Since("1.4.1") override def copy(extra: ParamMap): VectorIndexerModel = { val copied = new VectorIndexerModel(uid, numFeatures, categoryMaps) copyValues(copied, extra).setParent(parent) } + + @Since("1.6.0") + override def write: MLWriter = new VectorIndexerModelWriter(this) +} + +@Since("1.6.0") +object VectorIndexerModel extends MLReadable[VectorIndexerModel] { + + private[VectorIndexerModel] + class VectorIndexerModelWriter(instance: VectorIndexerModel) extends MLWriter { + + private case class Data(numFeatures: Int, categoryMaps: Map[Int, Map[Double, Int]]) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.numFeatures, instance.categoryMaps) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class VectorIndexerModelReader extends MLReader[VectorIndexerModel] { + + private val className = classOf[VectorIndexerModel].getName + + override def load(path: String): VectorIndexerModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + .select("numFeatures", "categoryMaps") + .head() + val numFeatures = data.getAs[Int](0) + val categoryMaps = data.getAs[Map[Int, Map[Double, Int]]](1) + val model = new VectorIndexerModel(metadata.uid, numFeatures, categoryMaps) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } + + @Since("1.6.0") + override def read: MLReader[VectorIndexerModel] = new VectorIndexerModelReader + + @Since("1.6.0") + override def load(path: String): VectorIndexerModel = super.load(path) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala index fb3387d4aa9b..e3e462d07e10 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorSlicer.scala @@ -17,33 +17,33 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.{Attribute, AttributeGroup} -import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.{IntArrayParam, ParamMap, StringArrayParam} -import org.apache.spark.ml.util.{Identifiable, MetadataUtils, SchemaUtils} -import org.apache.spark.mllib.linalg._ -import org.apache.spark.sql.DataFrame +import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType /** - * :: Experimental :: * This class takes a feature vector and outputs a new feature vector with a subarray of the * original features. * - * The subset of features can be specified with either indices ([[setIndices()]]) - * or names ([[setNames()]]). At least one feature must be selected. Duplicate features + * The subset of features can be specified with either indices (`setIndices()`) + * or names (`setNames()`). At least one feature must be selected. Duplicate features * are not allowed, so there can be no overlap between selected indices and names. * * The output vector will order features with the selected indices first (in the order given), * followed by the selected names (in the order given). */ -@Experimental -final class VectorSlicer(override val uid: String) - extends Transformer with HasInputCol with HasOutputCol { +@Since("1.5.0") +final class VectorSlicer @Since("1.5.0") (@Since("1.5.0") override val uid: String) + extends Transformer with HasInputCol with HasOutputCol with DefaultParamsWritable { + @Since("1.5.0") def this() = this(Identifiable.randomUID("vectorSlicer")) /** @@ -52,6 +52,7 @@ final class VectorSlicer(override val uid: String) * Default: Empty array * @group param */ + @Since("1.5.0") val indices = new IntArrayParam(this, "indices", "An array of indices to select features from a vector column." + " There can be no overlap with names.", VectorSlicer.validIndices) @@ -59,9 +60,11 @@ final class VectorSlicer(override val uid: String) setDefault(indices -> Array.empty[Int]) /** @group getParam */ + @Since("1.5.0") def getIndices: Array[Int] = $(indices) /** @group setParam */ + @Since("1.5.0") def setIndices(value: Array[Int]): this.type = set(indices, value) /** @@ -71,6 +74,7 @@ final class VectorSlicer(override val uid: String) * Default: Empty Array * @group param */ + @Since("1.5.0") val names = new StringArrayParam(this, "names", "An array of feature names to select features from a vector column." + " There can be no overlap with indices.", VectorSlicer.validNames) @@ -78,23 +82,23 @@ final class VectorSlicer(override val uid: String) setDefault(names -> Array.empty[String]) /** @group getParam */ + @Since("1.5.0") def getNames: Array[String] = $(names) /** @group setParam */ + @Since("1.5.0") def setNames(value: Array[String]): this.type = set(names, value) /** @group setParam */ + @Since("1.5.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.5.0") def setOutputCol(value: String): this.type = set(outputCol, value) - override def validateParams(): Unit = { - require($(indices).length > 0 || $(names).length > 0, - s"VectorSlicer requires that at least one feature be selected.") - } - - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { // Validity checks transformSchema(dataset.schema) val inputAttr = AttributeGroup.fromStructField(dataset.schema($(inputCol))) @@ -138,7 +142,10 @@ final class VectorSlicer(override val uid: String) indFeatures ++ nameFeatures } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { + require($(indices).length > 0 || $(names).length > 0, + s"VectorSlicer requires that at least one feature be selected.") SchemaUtils.checkColumnType(schema, $(inputCol), new VectorUDT) if (schema.fieldNames.contains($(outputCol))) { @@ -150,13 +157,15 @@ final class VectorSlicer(override val uid: String) StructType(outputFields) } + @Since("1.5.0") override def copy(extra: ParamMap): VectorSlicer = defaultCopy(extra) } -private[feature] object VectorSlicer { +@Since("1.6.0") +object VectorSlicer extends DefaultParamsReadable[VectorSlicer] { /** Return true if given feature indices are valid */ - def validIndices(indices: Array[Int]): Boolean = { + private[feature] def validIndices(indices: Array[Int]): Boolean = { if (indices.isEmpty) { true } else { @@ -165,7 +174,10 @@ private[feature] object VectorSlicer { } /** Return true if given feature names are valid */ - def validNames(names: Array[String]): Boolean = { + private[feature] def validNames(names: Array[String]): Boolean = { names.forall(_.nonEmpty) && names.length == names.distinct.length } + + @Since("1.6.0") + override def load(path: String): VectorSlicer = super.load(path) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index 9edab3af913c..fe3306e1e50d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -17,19 +17,20 @@ package org.apache.spark.ml.feature -import org.apache.spark.annotation.Experimental -import org.apache.spark.SparkContext +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.{Identifiable, SchemaUtils} +import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{VectorUDT, Vector, Vectors} -import org.apache.spark.mllib.linalg.BLAS._ -import org.apache.spark.sql.DataFrame +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.SQLContext import org.apache.spark.sql.types._ +import org.apache.spark.util.{Utils, VersionUtils} /** * Params for [[Word2Vec]] and [[Word2VecModel]]. @@ -43,19 +44,34 @@ private[feature] trait Word2VecBase extends Params * @group param */ final val vectorSize = new IntParam( - this, "vectorSize", "the dimension of codes after transforming from words") + this, "vectorSize", "the dimension of codes after transforming from words (> 0)", + ParamValidators.gt(0)) setDefault(vectorSize -> 100) /** @group getParam */ def getVectorSize: Int = $(vectorSize) + /** + * The window size (context words from [-window, window]). + * Default: 5 + * @group expertParam + */ + final val windowSize = new IntParam( + this, "windowSize", "the window size (context words from [-window, window]) (> 0)", + ParamValidators.gt(0)) + setDefault(windowSize -> 5) + + /** @group expertGetParam */ + def getWindowSize: Int = $(windowSize) + /** * Number of partitions for sentences of words. * Default: 1 * @group param */ final val numPartitions = new IntParam( - this, "numPartitions", "number of partitions for sentences of words") + this, "numPartitions", "number of partitions for sentences of words (> 0)", + ParamValidators.gt(0)) setDefault(numPartitions -> 1) /** @group getParam */ @@ -68,12 +84,27 @@ private[feature] trait Word2VecBase extends Params * @group param */ final val minCount = new IntParam(this, "minCount", "the minimum number of times a token must " + - "appear to be included in the word2vec model's vocabulary") + "appear to be included in the word2vec model's vocabulary (>= 0)", ParamValidators.gtEq(0)) setDefault(minCount -> 5) /** @group getParam */ def getMinCount: Int = $(minCount) + /** + * Sets the maximum length (in words) of each sentence in the input data. + * Any sentence longer than this threshold will be divided into chunks of + * up to `maxSentenceLength` size. + * Default: 1000 + * @group param + */ + final val maxSentenceLength = new IntParam(this, "maxSentenceLength", "Maximum length " + + "(in words) of each sentence in the input data. Any sentence longer than this threshold will " + + "be divided into chunks up to the size (> 0)", ParamValidators.gt(0)) + setDefault(maxSentenceLength -> 1000) + + /** @group getParam */ + def getMaxSentenceLength: Int = $(maxSentenceLength) + setDefault(stepSize -> 0.025) setDefault(maxIter -> 1) @@ -81,48 +112,68 @@ private[feature] trait Word2VecBase extends Params * Validate and transform the input schema. */ protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(inputCol), new ArrayType(StringType, true)) + val typeCandidates = List(new ArrayType(StringType, true), new ArrayType(StringType, false)) + SchemaUtils.checkColumnTypes(schema, $(inputCol), typeCandidates) SchemaUtils.appendColumn(schema, $(outputCol), new VectorUDT) } } /** - * :: Experimental :: * Word2Vec trains a model of `Map(String, Vector)`, i.e. transforms a word into a code for further * natural language processing or machine learning process. */ -@Experimental -final class Word2Vec(override val uid: String) extends Estimator[Word2VecModel] with Word2VecBase { +@Since("1.4.0") +final class Word2Vec @Since("1.4.0") ( + @Since("1.4.0") override val uid: String) + extends Estimator[Word2VecModel] with Word2VecBase with DefaultParamsWritable { + @Since("1.4.0") def this() = this(Identifiable.randomUID("w2v")) /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** @group setParam */ + @Since("1.4.0") def setVectorSize(value: Int): this.type = set(vectorSize, value) + /** @group expertSetParam */ + @Since("1.6.0") + def setWindowSize(value: Int): this.type = set(windowSize, value) + /** @group setParam */ + @Since("1.4.0") def setStepSize(value: Double): this.type = set(stepSize, value) /** @group setParam */ + @Since("1.4.0") def setNumPartitions(value: Int): this.type = set(numPartitions, value) /** @group setParam */ + @Since("1.4.0") def setMaxIter(value: Int): this.type = set(maxIter, value) /** @group setParam */ + @Since("1.4.0") def setSeed(value: Long): this.type = set(seed, value) /** @group setParam */ + @Since("1.4.0") def setMinCount(value: Int): this.type = set(minCount, value) - override def fit(dataset: DataFrame): Word2VecModel = { + /** @group setParam */ + @Since("2.0.0") + def setMaxSentenceLength(value: Int): this.type = set(maxSentenceLength, value) + + @Since("2.0.0") + override def fit(dataset: Dataset[_]): Word2VecModel = { transformSchema(dataset.schema, logging = true) - val input = dataset.select($(inputCol)).map(_.getAs[Seq[String]](0)) + val input = dataset.select($(inputCol)).rdd.map(_.getAs[Seq[String]](0)) val wordVectors = new feature.Word2Vec() .setLearningRate($(stepSize)) .setMinCount($(minCount)) @@ -130,100 +181,243 @@ final class Word2Vec(override val uid: String) extends Estimator[Word2VecModel] .setNumPartitions($(numPartitions)) .setSeed($(seed)) .setVectorSize($(vectorSize)) + .setWindowSize($(windowSize)) + .setMaxSentenceLength($(maxSentenceLength)) .fit(input) copyValues(new Word2VecModel(uid, wordVectors).setParent(this)) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.4.1") override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra) } +@Since("1.6.0") +object Word2Vec extends DefaultParamsReadable[Word2Vec] { + + @Since("1.6.0") + override def load(path: String): Word2Vec = super.load(path) +} + /** - * :: Experimental :: * Model fitted by [[Word2Vec]]. */ -@Experimental +@Since("1.4.0") class Word2VecModel private[ml] ( - override val uid: String, - wordVectors: feature.Word2VecModel) - extends Model[Word2VecModel] with Word2VecBase { + @Since("1.4.0") override val uid: String, + @transient private val wordVectors: feature.Word2VecModel) + extends Model[Word2VecModel] with Word2VecBase with MLWritable { + import Word2VecModel._ /** * Returns a dataframe with two fields, "word" and "vector", with "word" being a String and * and the vector the DenseVector that it is mapped to. */ + @Since("1.5.0") @transient lazy val getVectors: DataFrame = { - val sc = SparkContext.getOrCreate() - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().getOrCreate() val wordVec = wordVectors.getVectors.mapValues(vec => Vectors.dense(vec.map(_.toDouble))) - sc.parallelize(wordVec.toSeq).toDF("word", "vector") + spark.createDataFrame(wordVec.toSeq).toDF("word", "vector") } /** - * Find "num" number of words closest in similarity to the given word. - * Returns a dataframe with the words and the cosine similarities between the - * synonyms and the given word. + * Find "num" number of words closest in similarity to the given word, not + * including the word itself. + * @return a dataframe with columns "word" and "similarity" of the word and the cosine + * similarities between the synonyms and the given word. */ + @Since("1.5.0") def findSynonyms(word: String, num: Int): DataFrame = { - findSynonyms(wordVectors.transform(word), num) + val spark = SparkSession.builder().getOrCreate() + spark.createDataFrame(findSynonymsArray(word, num)).toDF("word", "similarity") } /** - * Find "num" number of words closest to similarity to the given vector representation - * of the word. Returns a dataframe with the words and the cosine similarities between the - * synonyms and the given word vector. + * Find "num" number of words whose vector representation is most similar to the supplied vector. + * If the supplied vector is the vector representation of a word in the model's vocabulary, + * that word will be in the results. + * @return a dataframe with columns "word" and "similarity" of the word and the cosine + * similarities between the synonyms and the given word vector. */ - def findSynonyms(word: Vector, num: Int): DataFrame = { - val sc = SparkContext.getOrCreate() - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ - sc.parallelize(wordVectors.findSynonyms(word, num)).toDF("word", "similarity") + @Since("2.0.0") + def findSynonyms(vec: Vector, num: Int): DataFrame = { + val spark = SparkSession.builder().getOrCreate() + spark.createDataFrame(findSynonymsArray(vec, num)).toDF("word", "similarity") + } + + /** + * Find "num" number of words whose vector representation is most similar to the supplied vector. + * If the supplied vector is the vector representation of a word in the model's vocabulary, + * that word will be in the results. + * @return an array of the words and the cosine similarities between the synonyms given + * word vector. + */ + @Since("2.2.0") + def findSynonymsArray(vec: Vector, num: Int): Array[(String, Double)] = { + wordVectors.findSynonyms(vec, num) + } + + /** + * Find "num" number of words closest in similarity to the given word, not + * including the word itself. + * @return an array of the words and the cosine similarities between the synonyms given + * word vector. + */ + @Since("2.2.0") + def findSynonymsArray(word: String, num: Int): Array[(String, Double)] = { + wordVectors.findSynonyms(word, num) } /** @group setParam */ + @Since("1.4.0") def setInputCol(value: String): this.type = set(inputCol, value) /** @group setParam */ + @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) /** * Transform a sentence column to a vector column to represent the whole sentence. The transform * is performed by averaging all word vectors it contains. */ - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val bWordVectors = dataset.sqlContext.sparkContext.broadcast(wordVectors) + val vectors = wordVectors.getVectors + .mapValues(vv => Vectors.dense(vv.map(_.toDouble))) + .map(identity) // mapValues doesn't return a serializable map (SI-7005) + val bVectors = dataset.sparkSession.sparkContext.broadcast(vectors) + val d = $(vectorSize) val word2Vec = udf { sentence: Seq[String] => - if (sentence.size == 0) { - Vectors.sparse($(vectorSize), Array.empty[Int], Array.empty[Double]) + if (sentence.isEmpty) { + Vectors.sparse(d, Array.empty[Int], Array.empty[Double]) } else { - val cum = Vectors.zeros($(vectorSize)) - val model = bWordVectors.value.getVectors - for (word <- sentence) { - if (model.contains(word)) { - axpy(1.0, bWordVectors.value.transform(word), cum) - } else { - // pass words which not belong to model + val sum = Vectors.zeros(d) + sentence.foreach { word => + bVectors.value.get(word).foreach { v => + BLAS.axpy(1.0, v, sum) } } - scal(1.0 / sentence.size, cum) - cum + BLAS.scal(1.0 / sentence.size, sum) + sum } } dataset.withColumn($(outputCol), word2Vec(col($(inputCol)))) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.4.1") override def copy(extra: ParamMap): Word2VecModel = { val copied = new Word2VecModel(uid, wordVectors) copyValues(copied, extra).setParent(parent) } + + @Since("1.6.0") + override def write: MLWriter = new Word2VecModelWriter(this) +} + +@Since("1.6.0") +object Word2VecModel extends MLReadable[Word2VecModel] { + + private case class Data(word: String, vector: Array[Float]) + + private[Word2VecModel] + class Word2VecModelWriter(instance: Word2VecModel) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + + val wordVectors = instance.wordVectors.getVectors + val dataPath = new Path(path, "data").toString + val bufferSizeInBytes = Utils.byteStringAsBytes( + sc.conf.get("spark.kryoserializer.buffer.max", "64m")) + val numPartitions = Word2VecModelWriter.calculateNumberOfPartitions( + bufferSizeInBytes, instance.wordVectors.wordIndex.size, instance.getVectorSize) + val spark = sparkSession + import spark.implicits._ + spark.createDataset[(String, Array[Float])](wordVectors.toSeq) + .repartition(numPartitions) + .map { case (word, vector) => Data(word, vector) } + .toDF() + .write + .parquet(dataPath) + } + } + + private[feature] + object Word2VecModelWriter { + /** + * Calculate the number of partitions to use in saving the model. + * [SPARK-11994] - We want to partition the model in partitions smaller than + * spark.kryoserializer.buffer.max + * @param bufferSizeInBytes Set to spark.kryoserializer.buffer.max + * @param numWords Vocab size + * @param vectorSize Vector length for each word + */ + def calculateNumberOfPartitions( + bufferSizeInBytes: Long, + numWords: Int, + vectorSize: Int): Int = { + val floatSize = 4L // Use Long to help avoid overflow + val averageWordSize = 15 + // Calculate the approximate size of the model. + // Assuming an average word size of 15 bytes, the formula is: + // (floatSize * vectorSize + 15) * numWords + val approximateSizeInBytes = (floatSize * vectorSize + averageWordSize) * numWords + val numPartitions = (approximateSizeInBytes / bufferSizeInBytes) + 1 + require(numPartitions < 10e8, s"Word2VecModel calculated that it needs $numPartitions " + + s"partitions to save this model, which is too large. Try increasing " + + s"spark.kryoserializer.buffer.max so that Word2VecModel can use fewer partitions.") + numPartitions.toInt + } + } + + private class Word2VecModelReader extends MLReader[Word2VecModel] { + + private val className = classOf[Word2VecModel].getName + + override def load(path: String): Word2VecModel = { + val spark = sparkSession + import spark.implicits._ + + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val (major, minor) = VersionUtils.majorMinorVersion(metadata.sparkVersion) + + val dataPath = new Path(path, "data").toString + + val oldModel = if (major < 2 || (major == 2 && minor < 2)) { + val data = spark.read.parquet(dataPath) + .select("wordIndex", "wordVectors") + .head() + val wordIndex = data.getAs[Map[String, Int]](0) + val wordVectors = data.getAs[Seq[Float]](1).toArray + new feature.Word2VecModel(wordIndex, wordVectors) + } else { + val wordVectorsMap = spark.read.parquet(dataPath).as[Data] + .collect() + .map(wordVector => (wordVector.word, wordVector.vector)) + .toMap + new feature.Word2VecModel(wordVectorsMap) + } + + val model = new Word2VecModel(metadata.uid, oldModel) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } + + @Since("1.6.0") + override def read: MLReader[Word2VecModel] = new Word2VecModelReader + + @Since("1.6.0") + override def load(path: String): Word2VecModel = super.load(path) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/package-info.java b/mllib/src/main/scala/org/apache/spark/ml/feature/package-info.java index c22d2e0cd2d9..ce7f33505687 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/package-info.java +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/package-info.java @@ -22,8 +22,8 @@ * The `ml.feature` package provides common feature transformers that help convert raw data or * features into more suitable forms for model fitting. * Most feature transformers are implemented as {@link org.apache.spark.ml.Transformer}s, which - * transforms one {@link org.apache.spark.sql.DataFrame} into another, e.g., - * {@link org.apache.spark.feature.HashingTF}. + * transforms one {@link org.apache.spark.sql.Dataset} into another, e.g., + * {@link org.apache.spark.ml.feature.HashingTF}. * Some feature transformers are implemented as {@link org.apache.spark.ml.Estimator}}s, because the * transformation requires some aggregated information of the dataset, e.g., document * frequencies in {@link org.apache.spark.ml.feature.IDF}. @@ -31,7 +31,7 @@ * obtain the model first, e.g., {@link org.apache.spark.ml.feature.IDFModel}, in order to apply * transformation. * The transformation is usually done by appending new columns to the input - * {@link org.apache.spark.sql.DataFrame}, so all input columns are carried over. + * {@link org.apache.spark.sql.Dataset}, so all input columns are carried over. * * We try to make each transformer minimal, so it becomes flexible to assemble feature * transformation pipelines. @@ -46,7 +46,7 @@ * import org.apache.spark.api.java.JavaRDD; * import static org.apache.spark.sql.types.DataTypes.*; * import org.apache.spark.sql.types.StructType; - * import org.apache.spark.sql.DataFrame; + * import org.apache.spark.sql.Dataset; * import org.apache.spark.sql.RowFactory; * import org.apache.spark.sql.Row; * @@ -61,12 +61,12 @@ * createStructField("id", IntegerType, false), * createStructField("text", StringType, false), * createStructField("rating", DoubleType, false))); - * JavaRDD rowRDD = jsc.parallelize( + * JavaRDD<Row> rowRDD = jsc.parallelize( * Arrays.asList( * RowFactory.create(0, "Hi I heard about Spark", 3.0), * RowFactory.create(1, "I wish Java could use case classes", 4.0), * RowFactory.create(2, "Logistic regression models are neat", 4.0))); - * DataFrame df = jsql.createDataFrame(rowRDD, schema); + * Dataset<Row> dataset = jsql.createDataFrame(rowRDD, schema); * // define feature transformers * RegexTokenizer tok = new RegexTokenizer() * .setInputCol("text") @@ -88,10 +88,10 @@ * // assemble and fit the feature transformation pipeline * Pipeline pipeline = new Pipeline() * .setStages(new PipelineStage[] {tok, sw, tf, idf, assembler}); - * PipelineModel model = pipeline.fit(df); + * PipelineModel model = pipeline.fit(dataset); * * // save transformed features with raw data - * model.transform(df) + * model.transform(dataset) * .select("id", "text", "rating", "features") * .write().format("parquet").save("/output/path"); * diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala index 4571ab26800c..6ff970cc72df 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/package.scala @@ -18,20 +18,19 @@ package org.apache.spark.ml import org.apache.spark.ml.feature.{HashingTF, IDF, IDFModel, VectorAssembler} -import org.apache.spark.sql.DataFrame /** * == Feature transformers == * * The `ml.feature` package provides common feature transformers that help convert raw data or * features into more suitable forms for model fitting. - * Most feature transformers are implemented as [[Transformer]]s, which transform one [[DataFrame]] + * Most feature transformers are implemented as [[Transformer]]s, which transform one `DataFrame` * into another, e.g., [[HashingTF]]. * Some feature transformers are implemented as [[Estimator]]s, because the transformation requires * some aggregated information of the dataset, e.g., document frequencies in [[IDF]]. - * For those feature transformers, calling [[Estimator!.fit]] is required to obtain the model first, + * For those feature transformers, calling `Estimator.fit` is required to obtain the model first, * e.g., [[IDFModel]], in order to apply transformation. - * The transformation is usually done by appending new columns to the input [[DataFrame]], so all + * The transformation is usually done by appending new columns to the input `DataFrame`, so all * input columns are carried over. * * We try to make each transformer minimal, so it becomes flexible to assemble feature @@ -44,7 +43,7 @@ import org.apache.spark.sql.DataFrame * import org.apache.spark.ml.Pipeline * * // a DataFrame with three columns: id (integer), text (string), and rating (double). - * val df = sqlContext.createDataFrame(Seq( + * val df = spark.createDataFrame(Seq( * (0, "Hi I heard about Spark", 3.0), * (1, "I wish Java could use case classes", 4.0), * (2, "Logistic regression models are neat", 4.0) @@ -84,6 +83,7 @@ import org.apache.spark.sql.DataFrame * input dataset, while MLlib's feature transformers operate lazily on individual columns, * which is more efficient and flexible to handle large and complex datasets. * - * @see [[http://scikit-learn.org/stable/modules/preprocessing.html scikit-learn.preprocessing]] + * @see + * scikit-learn.preprocessing */ package object feature diff --git a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala new file mode 100644 index 000000000000..aa7871d6ff29 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala @@ -0,0 +1,364 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.fpm + +import scala.reflect.ClassTag + +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared.HasPredictionCol +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.fpm.{AssociationRules => MLlibAssociationRules, + FPGrowth => MLlibFPGrowth} +import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset +import org.apache.spark.sql._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +/** + * Common params for FPGrowth and FPGrowthModel + */ +private[fpm] trait FPGrowthParams extends Params with HasPredictionCol { + + /** + * Items column name. + * Default: "items" + * @group param + */ + @Since("2.2.0") + val itemsCol: Param[String] = new Param[String](this, "itemsCol", "items column name") + setDefault(itemsCol -> "items") + + /** @group getParam */ + @Since("2.2.0") + def getItemsCol: String = $(itemsCol) + + /** + * Minimal support level of the frequent pattern. [0.0, 1.0]. Any pattern that appears + * more than (minSupport * size-of-the-dataset) times will be output in the frequent itemsets. + * Default: 0.3 + * @group param + */ + @Since("2.2.0") + val minSupport: DoubleParam = new DoubleParam(this, "minSupport", + "the minimal support level of a frequent pattern", + ParamValidators.inRange(0.0, 1.0)) + setDefault(minSupport -> 0.3) + + /** @group getParam */ + @Since("2.2.0") + def getMinSupport: Double = $(minSupport) + + /** + * Number of partitions (at least 1) used by parallel FP-growth. By default the param is not + * set, and partition number of the input dataset is used. + * @group expertParam + */ + @Since("2.2.0") + val numPartitions: IntParam = new IntParam(this, "numPartitions", + "Number of partitions used by parallel FP-growth", ParamValidators.gtEq[Int](1)) + + /** @group expertGetParam */ + @Since("2.2.0") + def getNumPartitions: Int = $(numPartitions) + + /** + * Minimal confidence for generating Association Rule. minConfidence will not affect the mining + * for frequent itemsets, but will affect the association rules generation. + * Default: 0.8 + * @group param + */ + @Since("2.2.0") + val minConfidence: DoubleParam = new DoubleParam(this, "minConfidence", + "minimal confidence for generating Association Rule", + ParamValidators.inRange(0.0, 1.0)) + setDefault(minConfidence -> 0.8) + + /** @group getParam */ + @Since("2.2.0") + def getMinConfidence: Double = $(minConfidence) + + /** + * Validates and transforms the input schema. + * @param schema input schema + * @return output schema + */ + @Since("2.2.0") + protected def validateAndTransformSchema(schema: StructType): StructType = { + val inputType = schema($(itemsCol)).dataType + require(inputType.isInstanceOf[ArrayType], + s"The input column must be ArrayType, but got $inputType.") + SchemaUtils.appendColumn(schema, $(predictionCol), schema($(itemsCol)).dataType) + } +} + +/** + * :: Experimental :: + * A parallel FP-growth algorithm to mine frequent itemsets. The algorithm is described in + * Li et al., PFP: Parallel FP-Growth for Query + * Recommendation. PFP distributes computation in such a way that each worker executes an + * independent group of mining tasks. The FP-Growth algorithm is described in + * Han et al., Mining frequent patterns without + * candidate generation. Note null values in the itemsCol column are ignored during fit(). + * + * @see + * Association rule learning (Wikipedia) + */ +@Since("2.2.0") +@Experimental +class FPGrowth @Since("2.2.0") ( + @Since("2.2.0") override val uid: String) + extends Estimator[FPGrowthModel] with FPGrowthParams with DefaultParamsWritable { + + @Since("2.2.0") + def this() = this(Identifiable.randomUID("fpgrowth")) + + /** @group setParam */ + @Since("2.2.0") + def setMinSupport(value: Double): this.type = set(minSupport, value) + + /** @group expertSetParam */ + @Since("2.2.0") + def setNumPartitions(value: Int): this.type = set(numPartitions, value) + + /** @group setParam */ + @Since("2.2.0") + def setMinConfidence(value: Double): this.type = set(minConfidence, value) + + /** @group setParam */ + @Since("2.2.0") + def setItemsCol(value: String): this.type = set(itemsCol, value) + + /** @group setParam */ + @Since("2.2.0") + def setPredictionCol(value: String): this.type = set(predictionCol, value) + + @Since("2.2.0") + override def fit(dataset: Dataset[_]): FPGrowthModel = { + transformSchema(dataset.schema, logging = true) + genericFit(dataset) + } + + private def genericFit[T: ClassTag](dataset: Dataset[_]): FPGrowthModel = { + val data = dataset.select($(itemsCol)) + val items = data.where(col($(itemsCol)).isNotNull).rdd.map(r => r.getSeq[T](0).toArray) + val mllibFP = new MLlibFPGrowth().setMinSupport($(minSupport)) + if (isSet(numPartitions)) { + mllibFP.setNumPartitions($(numPartitions)) + } + val parentModel = mllibFP.run(items) + val rows = parentModel.freqItemsets.map(f => Row(f.items, f.freq)) + val schema = StructType(Seq( + StructField("items", dataset.schema($(itemsCol)).dataType, nullable = false), + StructField("freq", LongType, nullable = false))) + val frequentItems = dataset.sparkSession.createDataFrame(rows, schema) + copyValues(new FPGrowthModel(uid, frequentItems)).setParent(this) + } + + @Since("2.2.0") + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } + + @Since("2.2.0") + override def copy(extra: ParamMap): FPGrowth = defaultCopy(extra) +} + + +@Since("2.2.0") +object FPGrowth extends DefaultParamsReadable[FPGrowth] { + + @Since("2.2.0") + override def load(path: String): FPGrowth = super.load(path) +} + +/** + * :: Experimental :: + * Model fitted by FPGrowth. + * + * @param freqItemsets frequent itemsets in the format of DataFrame("items"[Array], "freq"[Long]) + */ +@Since("2.2.0") +@Experimental +class FPGrowthModel private[ml] ( + @Since("2.2.0") override val uid: String, + @Since("2.2.0") @transient val freqItemsets: DataFrame) + extends Model[FPGrowthModel] with FPGrowthParams with MLWritable { + + /** @group setParam */ + @Since("2.2.0") + def setMinConfidence(value: Double): this.type = set(minConfidence, value) + + /** @group setParam */ + @Since("2.2.0") + def setItemsCol(value: String): this.type = set(itemsCol, value) + + /** @group setParam */ + @Since("2.2.0") + def setPredictionCol(value: String): this.type = set(predictionCol, value) + + /** + * Cache minConfidence and associationRules to avoid redundant computation for association rules + * during transform. The associationRules will only be re-computed when minConfidence changed. + */ + @transient private var _cachedMinConf: Double = Double.NaN + + @transient private var _cachedRules: DataFrame = _ + + /** + * Get association rules fitted using the minConfidence. Returns a dataframe + * with three fields, "antecedent", "consequent" and "confidence", where "antecedent" and + * "consequent" are Array[T] and "confidence" is Double. + */ + @Since("2.2.0") + @transient def associationRules: DataFrame = { + if ($(minConfidence) == _cachedMinConf) { + _cachedRules + } else { + _cachedRules = AssociationRules + .getAssociationRulesFromFP(freqItemsets, "items", "freq", $(minConfidence)) + _cachedMinConf = $(minConfidence) + _cachedRules + } + } + + /** + * The transform method first generates the association rules according to the frequent itemsets. + * Then for each transaction in itemsCol, the transform method will compare its items against the + * antecedents of each association rule. If the record contains all the antecedents of a + * specific association rule, the rule will be considered as applicable and its consequents + * will be added to the prediction result. The transform method will summarize the consequents + * from all the applicable rules as prediction. The prediction column has the same data type as + * the input column(Array[T]) and will not contain existing items in the input column. The null + * values in the itemsCol columns are treated as empty sets. + * WARNING: internally it collects association rules to the driver and uses broadcast for + * efficiency. This may bring pressure to driver memory for large set of association rules. + */ + @Since("2.2.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) + genericTransform(dataset) + } + + private def genericTransform(dataset: Dataset[_]): DataFrame = { + val rules: Array[(Seq[Any], Seq[Any])] = associationRules.select("antecedent", "consequent") + .rdd.map(r => (r.getSeq(0), r.getSeq(1))) + .collect().asInstanceOf[Array[(Seq[Any], Seq[Any])]] + val brRules = dataset.sparkSession.sparkContext.broadcast(rules) + + val dt = dataset.schema($(itemsCol)).dataType + // For each rule, examine the input items and summarize the consequents + val predictUDF = udf((items: Seq[_]) => { + if (items != null) { + val itemset = items.toSet + brRules.value.filter(_._1.forall(itemset.contains)) + .flatMap(_._2.filter(!itemset.contains(_))).distinct + } else { + Seq.empty + }}, dt) + dataset.withColumn($(predictionCol), predictUDF(col($(itemsCol)))) + } + + @Since("2.2.0") + override def transformSchema(schema: StructType): StructType = { + validateAndTransformSchema(schema) + } + + @Since("2.2.0") + override def copy(extra: ParamMap): FPGrowthModel = { + val copied = new FPGrowthModel(uid, freqItemsets) + copyValues(copied, extra).setParent(this.parent) + } + + @Since("2.2.0") + override def write: MLWriter = new FPGrowthModel.FPGrowthModelWriter(this) +} + +@Since("2.2.0") +object FPGrowthModel extends MLReadable[FPGrowthModel] { + + @Since("2.2.0") + override def read: MLReader[FPGrowthModel] = new FPGrowthModelReader + + @Since("2.2.0") + override def load(path: String): FPGrowthModel = super.load(path) + + /** [[MLWriter]] instance for [[FPGrowthModel]] */ + private[FPGrowthModel] + class FPGrowthModelWriter(instance: FPGrowthModel) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val dataPath = new Path(path, "data").toString + instance.freqItemsets.write.parquet(dataPath) + } + } + + private class FPGrowthModelReader extends MLReader[FPGrowthModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[FPGrowthModel].getName + + override def load(path: String): FPGrowthModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val frequentItems = sparkSession.read.parquet(dataPath) + val model = new FPGrowthModel(metadata.uid, frequentItems) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } +} + +private[fpm] object AssociationRules { + + /** + * Computes the association rules with confidence above minConfidence. + * @param dataset DataFrame("items"[Array], "freq"[Long]) containing frequent itemsets obtained + * from algorithms like [[FPGrowth]]. + * @param itemsCol column name for frequent itemsets + * @param freqCol column name for appearance count of the frequent itemsets + * @param minConfidence minimum confidence for generating the association rules + * @return a DataFrame("antecedent"[Array], "consequent"[Array], "confidence"[Double]) + * containing the association rules. + */ + def getAssociationRulesFromFP[T: ClassTag]( + dataset: Dataset[_], + itemsCol: String, + freqCol: String, + minConfidence: Double): DataFrame = { + + val freqItemSetRdd = dataset.select(itemsCol, freqCol).rdd + .map(row => new FreqItemset(row.getSeq[T](0).toArray, row.getLong(1))) + val rows = new MLlibAssociationRules() + .setMinConfidence(minConfidence) + .run(freqItemSetRdd) + .map(r => Row(r.antecedent, r.consequent, r.confidence)) + + val dt = dataset.schema(itemsCol).dataType + val schema = StructType(Seq( + StructField("antecedent", dt, nullable = false), + StructField("consequent", dt, nullable = false), + StructField("confidence", DoubleType, nullable = false))) + val rules = dataset.sparkSession.createDataFrame(rows, schema) + rules + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/JsonVectorConverter.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/JsonVectorConverter.scala new file mode 100644 index 000000000000..781e69f8d63d --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/JsonVectorConverter.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods.{compact, parse => parseJson, render} + +private[ml] object JsonVectorConverter { + + /** + * Parses the JSON representation of a vector into a [[Vector]]. + */ + def fromJson(json: String): Vector = { + implicit val formats = DefaultFormats + val jValue = parseJson(json) + (jValue \ "type").extract[Int] match { + case 0 => // sparse + val size = (jValue \ "size").extract[Int] + val indices = (jValue \ "indices").extract[Seq[Int]].toArray + val values = (jValue \ "values").extract[Seq[Double]].toArray + Vectors.sparse(size, indices, values) + case 1 => // dense + val values = (jValue \ "values").extract[Seq[Double]].toArray + Vectors.dense(values) + case _ => + throw new IllegalArgumentException(s"Cannot parse $json into a vector.") + } + } + + /** + * Coverts the vector to a JSON string. + */ + def toJson(v: Vector): String = { + v match { + case SparseVector(size, indices, values) => + val jValue = ("type" -> 0) ~ + ("size" -> size) ~ + ("indices" -> indices.toSeq) ~ + ("values" -> values.toSeq) + compact(render(jValue)) + case DenseVector(values) => + val jValue = ("type" -> 1) ~ ("values" -> values.toSeq) + compact(render(jValue)) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala new file mode 100644 index 000000000000..f4a8556c71f6 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeArrayData} +import org.apache.spark.sql.types._ + +/** + * User-defined type for [[Matrix]] in [[mllib-local]] which allows easy interaction with SQL + * via [[org.apache.spark.sql.Dataset]]. + */ +private[spark] class MatrixUDT extends UserDefinedType[Matrix] { + + override def sqlType: StructType = { + // type: 0 = sparse, 1 = dense + // the dense matrix is built by numRows, numCols, values and isTransposed, all of which are + // set as not nullable, except values since in the future, support for binary matrices might + // be added for which values are not needed. + // the sparse matrix needs colPtrs and rowIndices, which are set as + // null, while building the dense matrix. + StructType(Seq( + StructField("type", ByteType, nullable = false), + StructField("numRows", IntegerType, nullable = false), + StructField("numCols", IntegerType, nullable = false), + StructField("colPtrs", ArrayType(IntegerType, containsNull = false), nullable = true), + StructField("rowIndices", ArrayType(IntegerType, containsNull = false), nullable = true), + StructField("values", ArrayType(DoubleType, containsNull = false), nullable = true), + StructField("isTransposed", BooleanType, nullable = false) + )) + } + + override def serialize(obj: Matrix): InternalRow = { + val row = new GenericInternalRow(7) + obj match { + case sm: SparseMatrix => + row.setByte(0, 0) + row.setInt(1, sm.numRows) + row.setInt(2, sm.numCols) + row.update(3, UnsafeArrayData.fromPrimitiveArray(sm.colPtrs)) + row.update(4, UnsafeArrayData.fromPrimitiveArray(sm.rowIndices)) + row.update(5, UnsafeArrayData.fromPrimitiveArray(sm.values)) + row.setBoolean(6, sm.isTransposed) + + case dm: DenseMatrix => + row.setByte(0, 1) + row.setInt(1, dm.numRows) + row.setInt(2, dm.numCols) + row.setNullAt(3) + row.setNullAt(4) + row.update(5, UnsafeArrayData.fromPrimitiveArray(dm.values)) + row.setBoolean(6, dm.isTransposed) + } + row + } + + override def deserialize(datum: Any): Matrix = { + datum match { + case row: InternalRow => + require(row.numFields == 7, + s"MatrixUDT.deserialize given row with length ${row.numFields} but requires length == 7") + val tpe = row.getByte(0) + val numRows = row.getInt(1) + val numCols = row.getInt(2) + val values = row.getArray(5).toDoubleArray() + val isTransposed = row.getBoolean(6) + tpe match { + case 0 => + val colPtrs = row.getArray(3).toIntArray() + val rowIndices = row.getArray(4).toIntArray() + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values, isTransposed) + case 1 => + new DenseMatrix(numRows, numCols, values, isTransposed) + } + } + } + + override def userClass: Class[Matrix] = classOf[Matrix] + + override def equals(o: Any): Boolean = { + o match { + case v: MatrixUDT => true + case _ => false + } + } + + // see [SPARK-8647], this achieves the needed constant hash code without constant no. + override def hashCode(): Int = classOf[MatrixUDT].getName.hashCode() + + override def typeName: String = "matrix" + + override def pyUDT: String = "pyspark.ml.linalg.MatrixUDT" + + private[spark] override def asNullable: MatrixUDT = this +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/SQLDataTypes.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/SQLDataTypes.scala new file mode 100644 index 000000000000..a66ba27a7b9c --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/SQLDataTypes.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.sql.types.DataType + +/** + * :: DeveloperApi :: + * SQL data types for vectors and matrices. + */ +@Since("2.0.0") +@DeveloperApi +object SQLDataTypes { + + /** Data type for [[Vector]]. */ + val VectorType: DataType = new VectorUDT + + /** Data type for [[Matrix]]. */ + val MatrixType: DataType = new MatrixUDT +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala new file mode 100644 index 000000000000..37f173bc2046 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeArrayData} +import org.apache.spark.sql.types._ + +/** + * User-defined type for [[Vector]] in [[mllib-local]] which allows easy interaction with SQL + * via [[org.apache.spark.sql.Dataset]]. + */ +private[spark] class VectorUDT extends UserDefinedType[Vector] { + + override final def sqlType: StructType = _sqlType + + override def serialize(obj: Vector): InternalRow = { + obj match { + case SparseVector(size, indices, values) => + val row = new GenericInternalRow(4) + row.setByte(0, 0) + row.setInt(1, size) + row.update(2, UnsafeArrayData.fromPrimitiveArray(indices)) + row.update(3, UnsafeArrayData.fromPrimitiveArray(values)) + row + case DenseVector(values) => + val row = new GenericInternalRow(4) + row.setByte(0, 1) + row.setNullAt(1) + row.setNullAt(2) + row.update(3, UnsafeArrayData.fromPrimitiveArray(values)) + row + } + } + + override def deserialize(datum: Any): Vector = { + datum match { + case row: InternalRow => + require(row.numFields == 4, + s"VectorUDT.deserialize given row with length ${row.numFields} but requires length == 4") + val tpe = row.getByte(0) + tpe match { + case 0 => + val size = row.getInt(1) + val indices = row.getArray(2).toIntArray() + val values = row.getArray(3).toDoubleArray() + new SparseVector(size, indices, values) + case 1 => + val values = row.getArray(3).toDoubleArray() + new DenseVector(values) + } + } + } + + override def pyUDT: String = "pyspark.ml.linalg.VectorUDT" + + override def userClass: Class[Vector] = classOf[Vector] + + override def equals(o: Any): Boolean = { + o match { + case v: VectorUDT => true + case _ => false + } + } + + // see [SPARK-8647], this achieves the needed constant hash code without constant no. + override def hashCode(): Int = classOf[VectorUDT].getName.hashCode() + + override def typeName: String = "vector" + + private[spark] override def asNullable: VectorUDT = this + + private[this] val _sqlType = { + // type: 0 = sparse, 1 = dense + // We only use "values" for dense vectors, and "size", "indices", and "values" for sparse + // vectors. The "values" field is nullable because we might want to add binary vectors later, + // which uses "size" and "indices", but not "values". + StructType(Seq( + StructField("type", ByteType, nullable = false), + StructField("size", IntegerType, nullable = true), + StructField("indices", ArrayType(IntegerType, containsNull = false), nullable = true), + StructField("values", ArrayType(DoubleType, containsNull = false), nullable = true))) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala new file mode 100644 index 000000000000..6961b45f55e4 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquares.scala @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.optim + +import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.{Instance, OffsetInstance} +import org.apache.spark.ml.linalg._ +import org.apache.spark.rdd.RDD + +/** + * Model fitted by [[IterativelyReweightedLeastSquares]]. + * @param coefficients model coefficients + * @param intercept model intercept + * @param diagInvAtWA diagonal of matrix (A^T * W * A)^-1 in the last iteration + * @param numIterations number of iterations + */ +private[ml] class IterativelyReweightedLeastSquaresModel( + val coefficients: DenseVector, + val intercept: Double, + val diagInvAtWA: DenseVector, + val numIterations: Int) extends Serializable + +/** + * Implements the method of iteratively reweighted least squares (IRLS) which is used to solve + * certain optimization problems by an iterative method. In each step of the iterations, it + * involves solving a weighted least squares (WLS) problem by [[WeightedLeastSquares]]. + * It can be used to find maximum likelihood estimates of a generalized linear model (GLM), + * find M-estimator in robust regression and other optimization problems. + * + * @param initialModel the initial guess model. + * @param reweightFunc the reweight function which is used to update working labels and weights + * at each iteration. + * @param fitIntercept whether to fit intercept. + * @param regParam L2 regularization parameter used by WLS. + * @param maxIter maximum number of iterations. + * @param tol the convergence tolerance. + * + * @see P. J. Green, Iteratively + * Reweighted Least Squares for Maximum Likelihood Estimation, and some Robust + * and Resistant Alternatives, Journal of the Royal Statistical Society. + * Series B, 1984. + */ +private[ml] class IterativelyReweightedLeastSquares( + val initialModel: WeightedLeastSquaresModel, + val reweightFunc: (OffsetInstance, WeightedLeastSquaresModel) => (Double, Double), + val fitIntercept: Boolean, + val regParam: Double, + val maxIter: Int, + val tol: Double) extends Logging with Serializable { + + def fit(instances: RDD[OffsetInstance]): IterativelyReweightedLeastSquaresModel = { + + var converged = false + var iter = 0 + + var model: WeightedLeastSquaresModel = initialModel + var oldModel: WeightedLeastSquaresModel = null + + while (iter < maxIter && !converged) { + + oldModel = model + + // Update working labels and weights using reweightFunc + val newInstances = instances.map { instance => + val (newLabel, newWeight) = reweightFunc(instance, oldModel) + Instance(newLabel, newWeight, instance.features) + } + + // Estimate new model + model = new WeightedLeastSquares(fitIntercept, regParam, elasticNetParam = 0.0, + standardizeFeatures = false, standardizeLabel = false).fit(newInstances) + + // Check convergence + val oldCoefficients = oldModel.coefficients + val coefficients = model.coefficients + BLAS.axpy(-1.0, coefficients, oldCoefficients) + val maxTolOfCoefficients = oldCoefficients.toArray.foldLeft(0.0) { (x, y) => + math.max(math.abs(x), math.abs(y)) + } + val maxTol = math.max(maxTolOfCoefficients, math.abs(oldModel.intercept - model.intercept)) + + if (maxTol < tol) { + converged = true + logInfo(s"IRLS converged in $iter iterations.") + } + + logInfo(s"Iteration $iter : relative tolerance = $maxTol") + iter = iter + 1 + + if (iter == maxIter) { + logInfo(s"IRLS reached the max number of iterations: $maxIter.") + } + + } + + new IterativelyReweightedLeastSquaresModel( + model.coefficients, model.intercept, model.diagInvAtWA, iter) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/NormalEquationSolver.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/NormalEquationSolver.scala new file mode 100644 index 000000000000..dc3bcc662733 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/NormalEquationSolver.scala @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.ml.optim + +import scala.collection.mutable + +import breeze.linalg.{DenseVector => BDV} +import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS, OWLQN => BreezeOWLQN} + +import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vectors} +import org.apache.spark.mllib.linalg.CholeskyDecomposition + +/** + * A class to hold the solution to the normal equations A^T^ W A x = A^T^ W b. + * + * @param coefficients The least squares coefficients. The last element in the coefficients + * is the intercept when bias is added to A. + * @param aaInv An option containing the upper triangular part of (A^T^ W A)^-1^, in column major + * format. None when an optimization program is used to solve the normal equations. + * @param objectiveHistory Option containing the objective history when an optimization program is + * used to solve the normal equations. None when an analytic solver is used. + */ +private[optim] class NormalEquationSolution( + val coefficients: Array[Double], + val aaInv: Option[Array[Double]], + val objectiveHistory: Option[Array[Double]]) + +/** + * Interface for classes that solve the normal equations locally. + */ +private[optim] sealed trait NormalEquationSolver { + + /** Solve the normal equations from summary statistics. */ + def solve( + bBar: Double, + bbBar: Double, + abBar: DenseVector, + aaBar: DenseVector, + aBar: DenseVector): NormalEquationSolution +} + +/** + * A class that solves the normal equations directly, using Cholesky decomposition. + */ +private[optim] class CholeskySolver extends NormalEquationSolver { + + override def solve( + bBar: Double, + bbBar: Double, + abBar: DenseVector, + aaBar: DenseVector, + aBar: DenseVector): NormalEquationSolution = { + val k = abBar.size + val x = CholeskyDecomposition.solve(aaBar.values, abBar.values) + val aaInv = CholeskyDecomposition.inverse(aaBar.values, k) + + new NormalEquationSolution(x, Some(aaInv), None) + } +} + +/** + * A class for solving the normal equations using Quasi-Newton optimization methods. + */ +private[optim] class QuasiNewtonSolver( + fitIntercept: Boolean, + maxIter: Int, + tol: Double, + l1RegFunc: Option[(Int) => Double]) extends NormalEquationSolver { + + override def solve( + bBar: Double, + bbBar: Double, + abBar: DenseVector, + aaBar: DenseVector, + aBar: DenseVector): NormalEquationSolution = { + val numFeatures = aBar.size + val numFeaturesPlusIntercept = if (fitIntercept) numFeatures + 1 else numFeatures + val initialCoefficientsWithIntercept = new Array[Double](numFeaturesPlusIntercept) + if (fitIntercept) { + initialCoefficientsWithIntercept(numFeaturesPlusIntercept - 1) = bBar + } + + val costFun = + new NormalEquationCostFun(bBar, bbBar, abBar, aaBar, aBar, fitIntercept, numFeatures) + val optimizer = l1RegFunc.map { func => + new BreezeOWLQN[Int, BDV[Double]](maxIter, 10, func, tol) + }.getOrElse(new BreezeLBFGS[BDV[Double]](maxIter, 10, tol)) + + val states = optimizer.iterations(new CachedDiffFunction(costFun), + new BDV[Double](initialCoefficientsWithIntercept)) + + val arrayBuilder = mutable.ArrayBuilder.make[Double] + var state: optimizer.State = null + while (states.hasNext) { + state = states.next() + arrayBuilder += state.adjustedValue + } + val x = state.x.toArray.clone() + new NormalEquationSolution(x, None, Some(arrayBuilder.result())) + } + + /** + * NormalEquationCostFun implements Breeze's DiffFunction[T] for the normal equation. + * It returns the loss and gradient with L2 regularization at a particular point (coefficients). + * It's used in Breeze's convex optimization routines. + */ + private class NormalEquationCostFun( + bBar: Double, + bbBar: Double, + ab: DenseVector, + aa: DenseVector, + aBar: DenseVector, + fitIntercept: Boolean, + numFeatures: Int) extends DiffFunction[BDV[Double]] { + + private val numFeaturesPlusIntercept = if (fitIntercept) numFeatures + 1 else numFeatures + + override def calculate(coefficients: BDV[Double]): (Double, BDV[Double]) = { + val coef = Vectors.fromBreeze(coefficients).toDense + if (fitIntercept) { + var j = 0 + var dotProd = 0.0 + val coefValues = coef.values + val aBarValues = aBar.values + while (j < numFeatures) { + dotProd += coefValues(j) * aBarValues(j) + j += 1 + } + coefValues(numFeatures) = bBar - dotProd + } + val aax = new DenseVector(new Array[Double](numFeaturesPlusIntercept)) + BLAS.dspmv(numFeaturesPlusIntercept, 1.0, aa, coef, 1.0, aax) + // loss = 1/2 (b^T W b - 2 x^T A^T W b + x^T A^T W A x) + val loss = 0.5 * bbBar - BLAS.dot(ab, coef) + 0.5 * BLAS.dot(coef, aax) + // gradient = A^T W A x - A^T W b + BLAS.axpy(-1.0, ab, aax) + (loss, aax.asBreeze.toDenseVector) + } + } +} + +/** + * Exception thrown when solving a linear system Ax = b for which the matrix A is non-invertible + * (singular). + */ +private[spark] class SingularMatrixException(message: String, cause: Throwable) + extends IllegalArgumentException(message, cause) { + + def this(message: String) = this(message, null) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala index 8617722ae542..c5c9c8eb2bd2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala @@ -17,56 +17,81 @@ package org.apache.spark.ml.optim -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.Instance -import org.apache.spark.mllib.linalg._ +import org.apache.spark.ml.linalg._ import org.apache.spark.rdd.RDD /** * Model fitted by [[WeightedLeastSquares]]. + * * @param coefficients model coefficients * @param intercept model intercept * @param diagInvAtWA diagonal of matrix (A^T * W * A)^-1 + * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. */ private[ml] class WeightedLeastSquaresModel( val coefficients: DenseVector, val intercept: Double, - val diagInvAtWA: DenseVector) extends Serializable + val diagInvAtWA: DenseVector, + val objectiveHistory: Array[Double]) extends Serializable { + + def predict(features: Vector): Double = { + BLAS.dot(coefficients, features) + intercept + } +} /** * Weighted least squares solver via normal equation. * Given weighted observations (w,,i,,, a,,i,,, b,,i,,), we use the following weighted least squares * formulation: * - * min,,x,z,, 1/2 sum,,i,, w,,i,, (a,,i,,^T^ x + z - b,,i,,)^2^ / sum,,i,, w_i - * + 1/2 lambda / delta sum,,j,, (sigma,,j,, x,,j,,)^2^, + * min,,x,z,, 1/2 sum,,i,, w,,i,, (a,,i,,^T^ x + z - b,,i,,)^2^ / sum,,i,, w,,i,, + * + lambda / delta (1/2 (1 - alpha) sum,,j,, (sigma,,j,, x,,j,,)^2^ + * + alpha sum,,j,, abs(sigma,,j,, x,,j,,)), * - * where lambda is the regularization parameter, and delta and sigma,,j,, are controlled by - * [[standardizeLabel]] and [[standardizeFeatures]], respectively. + * where lambda is the regularization parameter, alpha is the ElasticNet mixing parameter, + * and delta and sigma,,j,, are controlled by [[standardizeLabel]] and [[standardizeFeatures]], + * respectively. * * Set [[regParam]] to 0.0 and turn off both [[standardizeFeatures]] and [[standardizeLabel]] to * match R's `lm`. * Turn on [[standardizeLabel]] to match R's `glmnet`. * + * @note The coefficients and intercept are always trained in the scaled space, but are returned + * on the original scale. [[standardizeFeatures]] and [[standardizeLabel]] can be used to + * control whether regularization is applied in the original space or the scaled space. * @param fitIntercept whether to fit intercept. If false, z is 0.0. - * @param regParam L2 regularization parameter (lambda) - * @param standardizeFeatures whether to standardize features. If true, sigma_,,j,, is the + * @param regParam Regularization parameter (lambda). + * @param elasticNetParam the ElasticNet mixing parameter (alpha). + * @param standardizeFeatures whether to standardize features. If true, sigma,,j,, is the * population standard deviation of the j-th column of A. Otherwise, * sigma,,j,, is 1.0. * @param standardizeLabel whether to standardize label. If true, delta is the population standard * deviation of the label column b. Otherwise, delta is 1.0. + * @param solverType the type of solver to use for optimization. + * @param maxIter maximum number of iterations. Only for QuasiNewton solverType. + * @param tol the convergence tolerance of the iterations. Only for QuasiNewton solverType. */ private[ml] class WeightedLeastSquares( val fitIntercept: Boolean, val regParam: Double, + val elasticNetParam: Double, val standardizeFeatures: Boolean, - val standardizeLabel: Boolean) extends Logging with Serializable { + val standardizeLabel: Boolean, + val solverType: WeightedLeastSquares.Solver = WeightedLeastSquares.Auto, + val maxIter: Int = 100, + val tol: Double = 1e-6) extends Logging with Serializable { import WeightedLeastSquares._ require(regParam >= 0.0, s"regParam cannot be negative: $regParam") if (regParam == 0.0) { logWarning("regParam is zero, which might cause numerical instability and overfitting.") } + require(elasticNetParam >= 0.0 && elasticNetParam <= 1.0, + s"elasticNetParam must be in [0, 1]: $elasticNetParam") + require(maxIter >= 0, s"maxIter must be a positive integer: $maxIter") + require(tol >= 0.0, s"tol must be >= 0, but was set to $tol") /** * Creates a [[WeightedLeastSquaresModel]] from an RDD of [[Instance]]s. @@ -76,64 +101,238 @@ private[ml] class WeightedLeastSquares( summary.validate() logInfo(s"Number of instances: ${summary.count}.") val k = if (fitIntercept) summary.k + 1 else summary.k + val numFeatures = summary.k val triK = summary.triK val wSum = summary.wSum - val bBar = summary.bBar - val bStd = summary.bStd - val aBar = summary.aBar - val aVar = summary.aVar - val abBar = summary.abBar - val aaBar = summary.aaBar - val aaValues = aaBar.values - - // add regularization to diagonals + + val rawBStd = summary.bStd + val rawBBar = summary.bBar + // if b is constant (rawBStd is zero), then b cannot be scaled. In this case + // setting bStd=abs(rawBBar) ensures that b is not scaled anymore in l-bfgs algorithm. + val bStd = if (rawBStd == 0.0) math.abs(rawBBar) else rawBStd + + if (rawBStd == 0) { + if (fitIntercept || rawBBar == 0.0) { + if (rawBBar == 0.0) { + logWarning(s"Mean and standard deviation of the label are zero, so the coefficients " + + s"and the intercept will all be zero; as a result, training is not needed.") + } else { + logWarning(s"The standard deviation of the label is zero, so the coefficients will be " + + s"zeros and the intercept will be the mean of the label; as a result, " + + s"training is not needed.") + } + val coefficients = new DenseVector(Array.ofDim(numFeatures)) + val intercept = rawBBar + val diagInvAtWA = new DenseVector(Array(0D)) + return new WeightedLeastSquaresModel(coefficients, intercept, diagInvAtWA, Array(0D)) + } else { + require(!(regParam > 0.0 && standardizeLabel), "The standard deviation of the label is " + + "zero. Model cannot be regularized with standardization=true") + logWarning(s"The standard deviation of the label is zero. Consider setting " + + s"fitIntercept=true.") + } + } + + val bBar = summary.bBar / bStd + val bbBar = summary.bbBar / (bStd * bStd) + + val aStd = summary.aStd + val aStdValues = aStd.values + + val aBar = { + val _aBar = summary.aBar + val _aBarValues = _aBar.values + var i = 0 + // scale aBar to standardized space in-place + while (i < numFeatures) { + if (aStdValues(i) == 0.0) { + _aBarValues(i) = 0.0 + } else { + _aBarValues(i) /= aStdValues(i) + } + i += 1 + } + _aBar + } + val aBarValues = aBar.values + + val abBar = { + val _abBar = summary.abBar + val _abBarValues = _abBar.values + var i = 0 + // scale abBar to standardized space in-place + while (i < numFeatures) { + if (aStdValues(i) == 0.0) { + _abBarValues(i) = 0.0 + } else { + _abBarValues(i) /= (aStdValues(i) * bStd) + } + i += 1 + } + _abBar + } + val abBarValues = abBar.values + + val aaBar = { + val _aaBar = summary.aaBar + val _aaBarValues = _aaBar.values + var j = 0 + var p = 0 + // scale aaBar to standardized space in-place + while (j < numFeatures) { + val aStdJ = aStdValues(j) + var i = 0 + while (i <= j) { + val aStdI = aStdValues(i) + if (aStdJ == 0.0 || aStdI == 0.0) { + _aaBarValues(p) = 0.0 + } else { + _aaBarValues(p) /= (aStdI * aStdJ) + } + p += 1 + i += 1 + } + j += 1 + } + _aaBar + } + val aaBarValues = aaBar.values + + val effectiveRegParam = regParam / bStd + val effectiveL1RegParam = elasticNetParam * effectiveRegParam + val effectiveL2RegParam = (1.0 - elasticNetParam) * effectiveRegParam + + // add L2 regularization to diagonals var i = 0 var j = 2 while (i < triK) { - var lambda = regParam - if (standardizeFeatures) { - lambda *= aVar(j - 2) + var lambda = effectiveL2RegParam + if (!standardizeFeatures) { + val std = aStdValues(j - 2) + if (std != 0.0) { + lambda /= (std * std) + } else { + lambda = 0.0 + } } - if (standardizeLabel) { - // TODO: handle the case when bStd = 0 - lambda /= bStd + if (!standardizeLabel) { + lambda *= bStd } - aaValues(i) += lambda + aaBarValues(i) += lambda i += j j += 1 } - val aa = if (fitIntercept) { - Array.concat(aaBar.values, aBar.values, Array(1.0)) + val aa = getAtA(aaBarValues, aBarValues) + val ab = getAtB(abBarValues, bBar) + + val solver = if ((solverType == WeightedLeastSquares.Auto && elasticNetParam != 0.0 && + regParam != 0.0) || (solverType == WeightedLeastSquares.QuasiNewton)) { + val effectiveL1RegFun: Option[(Int) => Double] = if (effectiveL1RegParam != 0.0) { + Some((index: Int) => { + if (fitIntercept && index == numFeatures) { + 0.0 + } else { + if (standardizeFeatures) { + effectiveL1RegParam + } else { + if (aStdValues(index) != 0.0) effectiveL1RegParam / aStdValues(index) else 0.0 + } + } + }) + } else { + None + } + new QuasiNewtonSolver(fitIntercept, maxIter, tol, effectiveL1RegFun) } else { - aaBar.values + new CholeskySolver } - val ab = if (fitIntercept) { - Array.concat(abBar.values, Array(bBar)) - } else { - abBar.values + + val solution = solver match { + case cholesky: CholeskySolver => + try { + cholesky.solve(bBar, bbBar, ab, aa, aBar) + } catch { + // if Auto solver is used and Cholesky fails due to singular AtA, then fall back to + // Quasi-Newton solver. + case _: SingularMatrixException if solverType == WeightedLeastSquares.Auto => + logWarning("Cholesky solver failed due to singular covariance matrix. " + + "Retrying with Quasi-Newton solver.") + // ab and aa were modified in place, so reconstruct them + val _aa = getAtA(aaBarValues, aBarValues) + val _ab = getAtB(abBarValues, bBar) + val newSolver = new QuasiNewtonSolver(fitIntercept, maxIter, tol, None) + newSolver.solve(bBar, bbBar, _ab, _aa, aBar) + } + case qn: QuasiNewtonSolver => + qn.solve(bBar, bbBar, ab, aa, aBar) } - val x = CholeskyDecomposition.solve(aa, ab) + val (coefficientArray, intercept) = if (fitIntercept) { + (solution.coefficients.slice(0, solution.coefficients.length - 1), + solution.coefficients.last * bStd) + } else { + (solution.coefficients, 0.0) + } - val aaInv = CholeskyDecomposition.inverse(aa, k) + // convert the coefficients from the scaled space to the original space + var q = 0 + val len = coefficientArray.length + while (q < len) { + coefficientArray(q) *= { if (aStdValues(q) != 0.0) bStd / aStdValues(q) else 0.0 } + q += 1 + } // aaInv is a packed upper triangular matrix, here we get all elements on diagonal - val diagInvAtWA = new DenseVector((1 to k).map { i => - aaInv(i + (i - 1) * i / 2 - 1) / wSum }.toArray) + val diagInvAtWA = solution.aaInv.map { inv => + new DenseVector((1 to k).map { i => + val multiplier = if (i == k && fitIntercept) { + 1.0 + } else { + aStdValues(i - 1) * aStdValues(i - 1) + } + inv(i + (i - 1) * i / 2 - 1) / (wSum * multiplier) + }.toArray) + }.getOrElse(new DenseVector(Array(0D))) + + new WeightedLeastSquaresModel(new DenseVector(coefficientArray), intercept, diagInvAtWA, + solution.objectiveHistory.getOrElse(Array(0D))) + } - val (coefficients, intercept) = if (fitIntercept) { - (new DenseVector(x.slice(0, x.length - 1)), x.last) + /** Construct A^T^ A (append bias if necessary). */ + private def getAtA(aaBar: Array[Double], aBar: Array[Double]): DenseVector = { + if (fitIntercept) { + new DenseVector(Array.concat(aaBar, aBar, Array(1.0))) } else { - (new DenseVector(x), 0.0) + new DenseVector(aaBar.clone()) } + } - new WeightedLeastSquaresModel(coefficients, intercept, diagInvAtWA) + /** Construct A^T^ b (append bias if necessary). */ + private def getAtB(abBar: Array[Double], bBar: Double): DenseVector = { + if (fitIntercept) { + new DenseVector(Array.concat(abBar, Array(bBar))) + } else { + new DenseVector(abBar.clone()) + } } } private[ml] object WeightedLeastSquares { + /** + * In order to take the normal equation approach efficiently, [[WeightedLeastSquares]] + * only supports the number of features is no more than 4096. + */ + val MAX_NUM_FEATURES: Int = 4096 + + sealed trait Solver + case object Auto extends Solver + case object Cholesky extends Solver + case object QuasiNewton extends Solver + + val supportedSolvers = Array(Auto, Cholesky, QuasiNewton) + /** * Aggregator to provide necessary summary statistics for solving [[WeightedLeastSquares]]. */ @@ -152,8 +351,8 @@ private[ml] object WeightedLeastSquares { private var aaSum: DenseVector = _ private def init(k: Int): Unit = { - require(k <= 4096, "In order to take the normal equation approach efficiently, " + - s"we set the max number of features to 4096 but got $k.") + require(k <= MAX_NUM_FEATURES, "In order to take the normal equation approach efficiently, " + + s"we set the max number of features to $MAX_NUM_FEATURES but got $k.") this.k = k triK = k * (k + 1) / 2 count = 0L @@ -233,10 +432,19 @@ private[ml] object WeightedLeastSquares { */ def bBar: Double = bSum / wSum + /** + * Weighted mean of squared labels. + */ + def bbBar: Double = bbSum / wSum + /** * Weighted population standard deviation of labels. */ - def bStd: Double = math.sqrt(bbSum / wSum - bBar * bBar) + def bStd: Double = { + // We prevent variance from negative value caused by numerical error. + val variance = math.max(bbSum / wSum - bBar * bBar, 0.0) + math.sqrt(variance) + } /** * Weighted mean of (label * features). @@ -256,6 +464,25 @@ private[ml] object WeightedLeastSquares { output } + /** + * Weighted population standard deviation of features. + */ + def aStd: DenseVector = { + val std = Array.ofDim[Double](k) + var i = 0 + var j = 2 + val aaValues = aaSum.values + while (i < triK) { + val l = j - 2 + val aw = aSum(l) / wSum + // We prevent variance from negative value caused by numerical error. + std(l) = math.sqrt(math.max(aaValues(i) / wSum - aw * aw, 0.0)) + i += j + j += 1 + } + new DenseVector(std) + } + /** * Weighted population variance of features. */ @@ -267,7 +494,8 @@ private[ml] object WeightedLeastSquares { while (i < triK) { val l = j - 2 val aw = aSum(l) / wSum - variance(l) = aaValues(i) / wSum - aw * aw + // We prevent variance from negative value caused by numerical error. + variance(l) = math.max(aaValues(i) / wSum - aw * aw, 0.0) i += j j += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/DifferentiableLossAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/DifferentiableLossAggregator.scala new file mode 100644 index 000000000000..403c28ff732f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/DifferentiableLossAggregator.scala @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.ml.optim.aggregator + +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} + +/** + * A parent trait for aggregators used in fitting MLlib models. This parent trait implements + * some of the common code shared between concrete instances of aggregators. Subclasses of this + * aggregator need only implement the `add` method. + * + * @tparam Datum The type of the instances added to the aggregator to update the loss and gradient. + * @tparam Agg Specialization of [[DifferentiableLossAggregator]]. Classes that subclass this + * type need to use this parameter to specify the concrete type of the aggregator. + */ +private[ml] trait DifferentiableLossAggregator[ + Datum, + Agg <: DifferentiableLossAggregator[Datum, Agg]] extends Serializable { + + self: Agg => // enforce classes that extend this to be the same type as `Agg` + + protected var weightSum: Double = 0.0 + protected var lossSum: Double = 0.0 + + /** The dimension of the gradient array. */ + protected val dim: Int + + /** Array of gradient values that are mutated when new instances are added to the aggregator. */ + protected lazy val gradientSumArray: Array[Double] = Array.ofDim[Double](dim) + + /** Add a single data point to this aggregator. */ + def add(instance: Datum): Agg + + /** Merge two aggregators. The `this` object will be modified in place and returned. */ + def merge(other: Agg): Agg = { + require(dim == other.dim, s"Dimensions mismatch when merging with another " + + s"${getClass.getSimpleName}. Expecting $dim but got ${other.dim}.") + + if (other.weightSum != 0) { + weightSum += other.weightSum + lossSum += other.lossSum + + var i = 0 + val localThisGradientSumArray = this.gradientSumArray + val localOtherGradientSumArray = other.gradientSumArray + while (i < dim) { + localThisGradientSumArray(i) += localOtherGradientSumArray(i) + i += 1 + } + } + this + } + + /** The current weighted averaged gradient. */ + def gradient: Vector = { + require(weightSum > 0.0, s"The effective number of instances should be " + + s"greater than 0.0, but was $weightSum.") + val result = Vectors.dense(gradientSumArray.clone()) + BLAS.scal(1.0 / weightSum, result) + result + } + + /** Weighted count of instances in this aggregator. */ + def weight: Double = weightSum + + /** The current loss value of this aggregator. */ + def loss: Double = { + require(weightSum > 0.0, s"The effective number of instances should be " + + s"greater than 0.0, but was $weightSum.") + lossSum / weightSum + } + +} + diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala new file mode 100644 index 000000000000..0300500a34ec --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HingeAggregator.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.optim.aggregator + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg._ + +/** + * HingeAggregator computes the gradient and loss for Hinge loss function as used in + * binary classification for instances in sparse or dense vector in an online fashion. + * + * Two HingeAggregators can be merged together to have a summary of loss and gradient of + * the corresponding joint dataset. + * + * This class standardizes feature values during computation using bcFeaturesStd. + * + * @param bcCoefficients The coefficients corresponding to the features. + * @param fitIntercept Whether to fit an intercept term. + * @param bcFeaturesStd The standard deviation values of the features. + */ +private[ml] class HingeAggregator( + bcFeaturesStd: Broadcast[Array[Double]], + fitIntercept: Boolean)(bcCoefficients: Broadcast[Vector]) + extends DifferentiableLossAggregator[Instance, HingeAggregator] { + + private val numFeatures: Int = bcFeaturesStd.value.length + private val numFeaturesPlusIntercept: Int = if (fitIntercept) numFeatures + 1 else numFeatures + @transient private lazy val coefficientsArray = bcCoefficients.value match { + case DenseVector(values) => values + case _ => throw new IllegalArgumentException(s"coefficients only supports dense vector" + + s" but got type ${bcCoefficients.value.getClass}.") + } + protected override val dim: Int = numFeaturesPlusIntercept + + /** + * Add a new training instance to this HingeAggregator, and update the loss and gradient + * of the objective function. + * + * @param instance The instance of data point to be added. + * @return This HingeAggregator object. + */ + def add(instance: Instance): this.type = { + instance match { case Instance(label, weight, features) => + require(numFeatures == features.size, s"Dimensions mismatch when adding new instance." + + s" Expecting $numFeatures but got ${features.size}.") + require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0") + + if (weight == 0.0) return this + val localFeaturesStd = bcFeaturesStd.value + val localCoefficients = coefficientsArray + val localGradientSumArray = gradientSumArray + + val dotProduct = { + var sum = 0.0 + features.foreachActive { (index, value) => + if (localFeaturesStd(index) != 0.0 && value != 0.0) { + sum += localCoefficients(index) * value / localFeaturesStd(index) + } + } + if (fitIntercept) sum += localCoefficients(numFeaturesPlusIntercept - 1) + sum + } + // Our loss function with {0, 1} labels is max(0, 1 - (2y - 1) (f_w(x))) + // Therefore the gradient is -(2y - 1)*x + val labelScaled = 2 * label - 1.0 + val loss = if (1.0 > labelScaled * dotProduct) { + (1.0 - labelScaled * dotProduct) * weight + } else { + 0.0 + } + + if (1.0 > labelScaled * dotProduct) { + val gradientScale = -labelScaled * weight + features.foreachActive { (index, value) => + if (localFeaturesStd(index) != 0.0 && value != 0.0) { + localGradientSumArray(index) += value * gradientScale / localFeaturesStd(index) + } + } + if (fitIntercept) { + localGradientSumArray(localGradientSumArray.length - 1) += gradientScale + } + } + + lossSum += loss + weightSum += weight + this + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregator.scala new file mode 100644 index 000000000000..1994b0e40e52 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregator.scala @@ -0,0 +1,224 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.ml.optim.aggregator + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} + +/** + * LeastSquaresAggregator computes the gradient and loss for a Least-squared loss function, + * as used in linear regression for samples in sparse or dense vector in an online fashion. + * + * Two LeastSquaresAggregator can be merged together to have a summary of loss and gradient of + * the corresponding joint dataset. + * + * For improving the convergence rate during the optimization process, and also preventing against + * features with very large variances exerting an overly large influence during model training, + * package like R's GLMNET performs the scaling to unit variance and removing the mean to reduce + * the condition number, and then trains the model in scaled space but returns the coefficients in + * the original scale. See page 9 in http://cran.r-project.org/web/packages/glmnet/glmnet.pdf + * + * However, we don't want to apply the `StandardScaler` on the training dataset, and then cache + * the standardized dataset since it will create a lot of overhead. As a result, we perform the + * scaling implicitly when we compute the objective function. The following is the mathematical + * derivation. + * + * Note that we don't deal with intercept by adding bias here, because the intercept + * can be computed using closed form after the coefficients are converged. + * See this discussion for detail. + * http://stats.stackexchange.com/questions/13617/how-is-the-intercept-computed-in-glmnet + * + * When training with intercept enabled, + * The objective function in the scaled space is given by + * + *
    + * $$ + * L = 1/2n ||\sum_i w_i(x_i - \bar{x_i}) / \hat{x_i} - (y - \bar{y}) / \hat{y}||^2, + * $$ + *
    + * + * where $\bar{x_i}$ is the mean of $x_i$, $\hat{x_i}$ is the standard deviation of $x_i$, + * $\bar{y}$ is the mean of label, and $\hat{y}$ is the standard deviation of label. + * + * If we fitting the intercept disabled (that is forced through 0.0), + * we can use the same equation except we set $\bar{y}$ and $\bar{x_i}$ to 0 instead + * of the respective means. + * + * This can be rewritten as + * + *
    + * $$ + * \begin{align} + * L &= 1/2n ||\sum_i (w_i/\hat{x_i})x_i - \sum_i (w_i/\hat{x_i})\bar{x_i} - y / \hat{y} + * + \bar{y} / \hat{y}||^2 \\ + * &= 1/2n ||\sum_i w_i^\prime x_i - y / \hat{y} + offset||^2 = 1/2n diff^2 + * \end{align} + * $$ + *
    + * + * where $w_i^\prime$ is the effective coefficients defined by $w_i/\hat{x_i}$, offset is + * + *
    + * $$ + * - \sum_i (w_i/\hat{x_i})\bar{x_i} + \bar{y} / \hat{y}. + * $$ + *
    + * + * and diff is + * + *
    + * $$ + * \sum_i w_i^\prime x_i - y / \hat{y} + offset + * $$ + *
    + * + * Note that the effective coefficients and offset don't depend on training dataset, + * so they can be precomputed. + * + * Now, the first derivative of the objective function in scaled space is + * + *
    + * $$ + * \frac{\partial L}{\partial w_i} = diff/N (x_i - \bar{x_i}) / \hat{x_i} + * $$ + *
    + * + * However, $(x_i - \bar{x_i})$ will densify the computation, so it's not + * an ideal formula when the training dataset is sparse format. + * + * This can be addressed by adding the dense $\bar{x_i} / \hat{x_i}$ terms + * in the end by keeping the sum of diff. The first derivative of total + * objective function from all the samples is + * + * + *
    + * $$ + * \begin{align} + * \frac{\partial L}{\partial w_i} &= + * 1/N \sum_j diff_j (x_{ij} - \bar{x_i}) / \hat{x_i} \\ + * &= 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) - diffSum \bar{x_i} / \hat{x_i}) \\ + * &= 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) + correction_i) + * \end{align} + * $$ + *
    + * + * where $correction_i = - diffSum \bar{x_i} / \hat{x_i}$ + * + * A simple math can show that diffSum is actually zero, so we don't even + * need to add the correction terms in the end. From the definition of diff, + * + *
    + * $$ + * \begin{align} + * diffSum &= \sum_j (\sum_i w_i(x_{ij} - \bar{x_i}) + * / \hat{x_i} - (y_j - \bar{y}) / \hat{y}) \\ + * &= N * (\sum_i w_i(\bar{x_i} - \bar{x_i}) / \hat{x_i} - (\bar{y} - \bar{y}) / \hat{y}) \\ + * &= 0 + * \end{align} + * $$ + *
    + * + * As a result, the first derivative of the total objective function only depends on + * the training dataset, which can be easily computed in distributed fashion, and is + * sparse format friendly. + * + *
    + * $$ + * \frac{\partial L}{\partial w_i} = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) + * $$ + *
    + * + * @note The constructor is curried, since the cost function will repeatedly create new versions + * of this class for different coefficient vectors. + * + * @param labelStd The standard deviation value of the label. + * @param labelMean The mean value of the label. + * @param fitIntercept Whether to fit an intercept term. + * @param bcFeaturesStd The broadcast standard deviation values of the features. + * @param bcFeaturesMean The broadcast mean values of the features. + * @param bcCoefficients The broadcast coefficients corresponding to the features. + */ +private[ml] class LeastSquaresAggregator( + labelStd: Double, + labelMean: Double, + fitIntercept: Boolean, + bcFeaturesStd: Broadcast[Array[Double]], + bcFeaturesMean: Broadcast[Array[Double]])(bcCoefficients: Broadcast[Vector]) + extends DifferentiableLossAggregator[Instance, LeastSquaresAggregator] { + require(labelStd > 0.0, s"${this.getClass.getName} requires the label standard " + + s"deviation to be positive.") + + private val numFeatures = bcFeaturesStd.value.length + protected override val dim: Int = numFeatures + // make transient so we do not serialize between aggregation stages + @transient private lazy val featuresStd = bcFeaturesStd.value + @transient private lazy val effectiveCoefAndOffset = { + val coefficientsArray = bcCoefficients.value.toArray.clone() + val featuresMean = bcFeaturesMean.value + var sum = 0.0 + var i = 0 + val len = coefficientsArray.length + while (i < len) { + if (featuresStd(i) != 0.0) { + coefficientsArray(i) /= featuresStd(i) + sum += coefficientsArray(i) * featuresMean(i) + } else { + coefficientsArray(i) = 0.0 + } + i += 1 + } + val offset = if (fitIntercept) labelMean / labelStd - sum else 0.0 + (Vectors.dense(coefficientsArray), offset) + } + // do not use tuple assignment above because it will circumvent the @transient tag + @transient private lazy val effectiveCoefficientsVector = effectiveCoefAndOffset._1 + @transient private lazy val offset = effectiveCoefAndOffset._2 + + /** + * Add a new training instance to this LeastSquaresAggregator, and update the loss and gradient + * of the objective function. + * + * @param instance The instance of data point to be added. + * @return This LeastSquaresAggregator object. + */ + def add(instance: Instance): LeastSquaresAggregator = { + instance match { case Instance(label, weight, features) => + require(numFeatures == features.size, s"Dimensions mismatch when adding new sample." + + s" Expecting $numFeatures but got ${features.size}.") + require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0") + + if (weight == 0.0) return this + + val diff = BLAS.dot(features, effectiveCoefficientsVector) - label / labelStd + offset + + if (diff != 0) { + val localGradientSumArray = gradientSumArray + val localFeaturesStd = featuresStd + features.foreachActive { (index, value) => + val fStd = localFeaturesStd(index) + if (fStd != 0.0 && value != 0.0) { + localGradientSumArray(index) += weight * diff * value / fStd + } + } + lossSum += weight * diff * diff / 2.0 + } + weightSum += weight + this + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala new file mode 100644 index 000000000000..272d36dd94ae --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregator.scala @@ -0,0 +1,366 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.ml.optim.aggregator + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{DenseVector, Vector} +import org.apache.spark.mllib.util.MLUtils + +/** + * LogisticAggregator computes the gradient and loss for binary or multinomial logistic (softmax) + * loss function, as used in classification for instances in sparse or dense vector in an online + * fashion. + * + * Two LogisticAggregators can be merged together to have a summary of loss and gradient of + * the corresponding joint dataset. + * + * For improving the convergence rate during the optimization process and also to prevent against + * features with very large variances exerting an overly large influence during model training, + * packages like R's GLMNET perform the scaling to unit variance and remove the mean in order to + * reduce the condition number. The model is then trained in this scaled space, but returns the + * coefficients in the original scale. See page 9 in + * http://cran.r-project.org/web/packages/glmnet/glmnet.pdf + * + * However, we don't want to apply the [[org.apache.spark.ml.feature.StandardScaler]] on the + * training dataset, and then cache the standardized dataset since it will create a lot of overhead. + * As a result, we perform the scaling implicitly when we compute the objective function (though + * we do not subtract the mean). + * + * Note that there is a difference between multinomial (softmax) and binary loss. The binary case + * uses one outcome class as a "pivot" and regresses the other class against the pivot. In the + * multinomial case, the softmax loss function is used to model each class probability + * independently. Using softmax loss produces `K` sets of coefficients, while using a pivot class + * produces `K - 1` sets of coefficients (a single coefficient vector in the binary case). In the + * binary case, we can say that the coefficients are shared between the positive and negative + * classes. When regularization is applied, multinomial (softmax) loss will produce a result + * different from binary loss since the positive and negative don't share the coefficients while the + * binary regression shares the coefficients between positive and negative. + * + * The following is a mathematical derivation for the multinomial (softmax) loss. + * + * The probability of the multinomial outcome $y$ taking on any of the K possible outcomes is: + * + *
    + * $$ + * P(y_i=0|\vec{x}_i, \beta) = \frac{e^{\vec{x}_i^T \vec{\beta}_0}}{\sum_{k=0}^{K-1} + * e^{\vec{x}_i^T \vec{\beta}_k}} \\ + * P(y_i=1|\vec{x}_i, \beta) = \frac{e^{\vec{x}_i^T \vec{\beta}_1}}{\sum_{k=0}^{K-1} + * e^{\vec{x}_i^T \vec{\beta}_k}}\\ + * P(y_i=K-1|\vec{x}_i, \beta) = \frac{e^{\vec{x}_i^T \vec{\beta}_{K-1}}\,}{\sum_{k=0}^{K-1} + * e^{\vec{x}_i^T \vec{\beta}_k}} + * $$ + *
    + * + * The model coefficients $\beta = (\beta_0, \beta_1, \beta_2, ..., \beta_{K-1})$ become a matrix + * which has dimension of $K \times (N+1)$ if the intercepts are added. If the intercepts are not + * added, the dimension will be $K \times N$. + * + * Note that the coefficients in the model above lack identifiability. That is, any constant scalar + * can be added to all of the coefficients and the probabilities remain the same. + * + *
    + * $$ + * \begin{align} + * \frac{e^{\vec{x}_i^T \left(\vec{\beta}_0 + \vec{c}\right)}}{\sum_{k=0}^{K-1} + * e^{\vec{x}_i^T \left(\vec{\beta}_k + \vec{c}\right)}} + * = \frac{e^{\vec{x}_i^T \vec{\beta}_0}e^{\vec{x}_i^T \vec{c}}\,}{e^{\vec{x}_i^T \vec{c}} + * \sum_{k=0}^{K-1} e^{\vec{x}_i^T \vec{\beta}_k}} + * = \frac{e^{\vec{x}_i^T \vec{\beta}_0}}{\sum_{k=0}^{K-1} e^{\vec{x}_i^T \vec{\beta}_k}} + * \end{align} + * $$ + *
    + * + * However, when regularization is added to the loss function, the coefficients are indeed + * identifiable because there is only one set of coefficients which minimizes the regularization + * term. When no regularization is applied, we choose the coefficients with the minimum L2 + * penalty for consistency and reproducibility. For further discussion see: + * + * Friedman, et al. "Regularization Paths for Generalized Linear Models via Coordinate Descent" + * + * The loss of objective function for a single instance of data (we do not include the + * regularization term here for simplicity) can be written as + * + *
    + * $$ + * \begin{align} + * \ell\left(\beta, x_i\right) &= -log{P\left(y_i \middle| \vec{x}_i, \beta\right)} \\ + * &= log\left(\sum_{k=0}^{K-1}e^{\vec{x}_i^T \vec{\beta}_k}\right) - \vec{x}_i^T \vec{\beta}_y\\ + * &= log\left(\sum_{k=0}^{K-1} e^{margins_k}\right) - margins_y + * \end{align} + * $$ + *
    + * + * where ${margins}_k = \vec{x}_i^T \vec{\beta}_k$. + * + * For optimization, we have to calculate the first derivative of the loss function, and a simple + * calculation shows that + * + *
    + * $$ + * \begin{align} + * \frac{\partial \ell(\beta, \vec{x}_i, w_i)}{\partial \beta_{j, k}} + * &= x_{i,j} \cdot w_i \cdot \left(\frac{e^{\vec{x}_i \cdot \vec{\beta}_k}}{\sum_{k'=0}^{K-1} + * e^{\vec{x}_i \cdot \vec{\beta}_{k'}}\,} - I_{y=k}\right) \\ + * &= x_{i, j} \cdot w_i \cdot multiplier_k + * \end{align} + * $$ + *
    + * + * where $w_i$ is the sample weight, $I_{y=k}$ is an indicator function + * + *
    + * $$ + * I_{y=k} = \begin{cases} + * 1 & y = k \\ + * 0 & else + * \end{cases} + * $$ + *
    + * + * and + * + *
    + * $$ + * multiplier_k = \left(\frac{e^{\vec{x}_i \cdot \vec{\beta}_k}}{\sum_{k=0}^{K-1} + * e^{\vec{x}_i \cdot \vec{\beta}_k}} - I_{y=k}\right) + * $$ + *
    + * + * If any of margins is larger than 709.78, the numerical computation of multiplier and loss + * function will suffer from arithmetic overflow. This issue occurs when there are outliers in + * data which are far away from the hyperplane, and this will cause the failing of training once + * infinity is introduced. Note that this is only a concern when max(margins) > 0. + * + * Fortunately, when max(margins) = maxMargin > 0, the loss function and the multiplier can + * easily be rewritten into the following equivalent numerically stable formula. + * + *
    + * $$ + * \ell\left(\beta, x\right) = log\left(\sum_{k=0}^{K-1} e^{margins_k - maxMargin}\right) - + * margins_{y} + maxMargin + * $$ + *
    + * + * Note that each term, $(margins_k - maxMargin)$ in the exponential is no greater than zero; as a + * result, overflow will not happen with this formula. + * + * For $multiplier$, a similar trick can be applied as the following, + * + *
    + * $$ + * multiplier_k = \left(\frac{e^{\vec{x}_i \cdot \vec{\beta}_k - maxMargin}}{\sum_{k'=0}^{K-1} + * e^{\vec{x}_i \cdot \vec{\beta}_{k'} - maxMargin}} - I_{y=k}\right) + * $$ + *
    + * + * + * @param bcCoefficients The broadcast coefficients corresponding to the features. + * @param bcFeaturesStd The broadcast standard deviation values of the features. + * @param numClasses the number of possible outcomes for k classes classification problem in + * Multinomial Logistic Regression. + * @param fitIntercept Whether to fit an intercept term. + * @param multinomial Whether to use multinomial (softmax) or binary loss + * @note In order to avoid unnecessary computation during calculation of the gradient updates + * we lay out the coefficients in column major order during training. This allows us to + * perform feature standardization once, while still retaining sequential memory access + * for speed. We convert back to row major order when we create the model, + * since this form is optimal for the matrix operations used for prediction. + */ +private[ml] class LogisticAggregator( + bcFeaturesStd: Broadcast[Array[Double]], + numClasses: Int, + fitIntercept: Boolean, + multinomial: Boolean)(bcCoefficients: Broadcast[Vector]) + extends DifferentiableLossAggregator[Instance, LogisticAggregator] with Logging { + + private val numFeatures = bcFeaturesStd.value.length + private val numFeaturesPlusIntercept = if (fitIntercept) numFeatures + 1 else numFeatures + private val coefficientSize = bcCoefficients.value.size + protected override val dim: Int = coefficientSize + if (multinomial) { + require(numClasses == coefficientSize / numFeaturesPlusIntercept, s"The number of " + + s"coefficients should be ${numClasses * numFeaturesPlusIntercept} but was $coefficientSize") + } else { + require(coefficientSize == numFeaturesPlusIntercept, s"Expected $numFeaturesPlusIntercept " + + s"coefficients but got $coefficientSize") + require(numClasses == 1 || numClasses == 2, s"Binary logistic aggregator requires numClasses " + + s"in {1, 2} but found $numClasses.") + } + + @transient private lazy val coefficientsArray: Array[Double] = bcCoefficients.value match { + case DenseVector(values) => values + case _ => throw new IllegalArgumentException(s"coefficients only supports dense vector but " + + s"got type ${bcCoefficients.value.getClass}.)") + } + + if (multinomial && numClasses <= 2) { + logInfo(s"Multinomial logistic regression for binary classification yields separate " + + s"coefficients for positive and negative classes. When no regularization is applied, the" + + s"result will be effectively the same as binary logistic regression. When regularization" + + s"is applied, multinomial loss will produce a result different from binary loss.") + } + + /** Update gradient and loss using binary loss function. */ + private def binaryUpdateInPlace(features: Vector, weight: Double, label: Double): Unit = { + + val localFeaturesStd = bcFeaturesStd.value + val localCoefficients = coefficientsArray + val localGradientArray = gradientSumArray + val margin = - { + var sum = 0.0 + features.foreachActive { (index, value) => + if (localFeaturesStd(index) != 0.0 && value != 0.0) { + sum += localCoefficients(index) * value / localFeaturesStd(index) + } + } + if (fitIntercept) sum += localCoefficients(numFeaturesPlusIntercept - 1) + sum + } + + val multiplier = weight * (1.0 / (1.0 + math.exp(margin)) - label) + + features.foreachActive { (index, value) => + if (localFeaturesStd(index) != 0.0 && value != 0.0) { + localGradientArray(index) += multiplier * value / localFeaturesStd(index) + } + } + + if (fitIntercept) { + localGradientArray(numFeaturesPlusIntercept - 1) += multiplier + } + + if (label > 0) { + // The following is equivalent to log(1 + exp(margin)) but more numerically stable. + lossSum += weight * MLUtils.log1pExp(margin) + } else { + lossSum += weight * (MLUtils.log1pExp(margin) - margin) + } + } + + /** Update gradient and loss using multinomial (softmax) loss function. */ + private def multinomialUpdateInPlace(features: Vector, weight: Double, label: Double): Unit = { + // TODO: use level 2 BLAS operations + /* + Note: this can still be used when numClasses = 2 for binary + logistic regression without pivoting. + */ + val localFeaturesStd = bcFeaturesStd.value + val localCoefficients = coefficientsArray + val localGradientArray = gradientSumArray + + // marginOfLabel is margins(label) in the formula + var marginOfLabel = 0.0 + var maxMargin = Double.NegativeInfinity + + val margins = new Array[Double](numClasses) + features.foreachActive { (index, value) => + if (localFeaturesStd(index) != 0.0 && value != 0.0) { + val stdValue = value / localFeaturesStd(index) + var j = 0 + while (j < numClasses) { + margins(j) += localCoefficients(index * numClasses + j) * stdValue + j += 1 + } + } + } + var i = 0 + while (i < numClasses) { + if (fitIntercept) { + margins(i) += localCoefficients(numClasses * numFeatures + i) + } + if (i == label.toInt) marginOfLabel = margins(i) + if (margins(i) > maxMargin) { + maxMargin = margins(i) + } + i += 1 + } + + /** + * When maxMargin is greater than 0, the original formula could cause overflow. + * We address this by subtracting maxMargin from all the margins, so it's guaranteed + * that all of the new margins will be smaller than zero to prevent arithmetic overflow. + */ + val multipliers = new Array[Double](numClasses) + val sum = { + var temp = 0.0 + var i = 0 + while (i < numClasses) { + if (maxMargin > 0) margins(i) -= maxMargin + val exp = math.exp(margins(i)) + temp += exp + multipliers(i) = exp + i += 1 + } + temp + } + + margins.indices.foreach { i => + multipliers(i) = multipliers(i) / sum - (if (label == i) 1.0 else 0.0) + } + features.foreachActive { (index, value) => + if (localFeaturesStd(index) != 0.0 && value != 0.0) { + val stdValue = value / localFeaturesStd(index) + var j = 0 + while (j < numClasses) { + localGradientArray(index * numClasses + j) += weight * multipliers(j) * stdValue + j += 1 + } + } + } + if (fitIntercept) { + var i = 0 + while (i < numClasses) { + localGradientArray(numFeatures * numClasses + i) += weight * multipliers(i) + i += 1 + } + } + + val loss = if (maxMargin > 0) { + math.log(sum) - marginOfLabel + maxMargin + } else { + math.log(sum) - marginOfLabel + } + lossSum += weight * loss + } + + /** + * Add a new training instance to this LogisticAggregator, and update the loss and gradient + * of the objective function. + * + * @param instance The instance of data point to be added. + * @return This LogisticAggregator object. + */ + def add(instance: Instance): this.type = { + instance match { case Instance(label, weight, features) => + require(numFeatures == features.size, s"Dimensions mismatch when adding new instance." + + s" Expecting $numFeatures but got ${features.size}.") + require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0") + + if (weight == 0.0) return this + + if (multinomial) { + multinomialUpdateInPlace(features, weight, label) + } else { + binaryUpdateInPlace(features, weight, label) + } + weightSum += weight + this + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/loss/DifferentiableRegularization.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/loss/DifferentiableRegularization.scala new file mode 100644 index 000000000000..929374eda13a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/loss/DifferentiableRegularization.scala @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.ml.optim.loss + +import breeze.optimize.DiffFunction + +import org.apache.spark.ml.linalg._ + +/** + * A Breeze diff function which represents a cost function for differentiable regularization + * of parameters. e.g. L2 regularization: 1 / 2 regParam * beta dot beta + * + * @tparam T The type of the coefficients being regularized. + */ +private[ml] trait DifferentiableRegularization[T] extends DiffFunction[T] { + + /** Magnitude of the regularization penalty. */ + def regParam: Double + +} + +/** + * A Breeze diff function for computing the L2 regularized loss and gradient of a vector of + * coefficients. + * + * @param regParam The magnitude of the regularization. + * @param shouldApply A function (Int => Boolean) indicating whether a given index should have + * regularization applied to it. Usually we don't apply regularization to + * the intercept. + * @param applyFeaturesStd Option for a function which maps coefficient index (column major) to the + * feature standard deviation. Since we always standardize the data during + * training, if `standardization` is false, we have to reverse + * standardization by penalizing each component differently by this param. + * If `standardization` is true, this should be `None`. + */ +private[ml] class L2Regularization( + override val regParam: Double, + shouldApply: Int => Boolean, + applyFeaturesStd: Option[Int => Double]) extends DifferentiableRegularization[Vector] { + + override def calculate(coefficients: Vector): (Double, Vector) = { + coefficients match { + case dv: DenseVector => + var sum = 0.0 + val gradient = new Array[Double](dv.size) + dv.values.indices.filter(shouldApply).foreach { j => + val coef = coefficients(j) + applyFeaturesStd match { + case Some(getStd) => + // If `standardization` is false, we still standardize the data + // to improve the rate of convergence; as a result, we have to + // perform this reverse standardization by penalizing each component + // differently to get effectively the same objective function when + // the training dataset is not standardized. + val std = getStd(j) + if (std != 0.0) { + val temp = coef / (std * std) + sum += coef * temp + gradient(j) = regParam * temp + } else { + 0.0 + } + case None => + // If `standardization` is true, compute L2 regularization normally. + sum += coef * coef + gradient(j) = coef * regParam + } + } + (0.5 * sum * regParam, Vectors.dense(gradient)) + case _: SparseVector => + throw new IllegalArgumentException("Sparse coefficients are not currently supported.") + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/loss/RDDLossFunction.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/loss/RDDLossFunction.scala new file mode 100644 index 000000000000..387f7c5b1ff3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/loss/RDDLossFunction.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.ml.optim.loss + +import scala.reflect.ClassTag + +import breeze.linalg.{DenseVector => BDV} +import breeze.optimize.DiffFunction + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} +import org.apache.spark.ml.optim.aggregator.DifferentiableLossAggregator +import org.apache.spark.rdd.RDD + +/** + * This class computes the gradient and loss of a differentiable loss function by mapping a + * [[DifferentiableLossAggregator]] over an [[RDD]]. The loss function is the + * sum of the loss computed on a single instance across all points in the RDD. Therefore, the actual + * analytical form of the loss function is specified by the aggregator, which computes each points + * contribution to the overall loss. + * + * A differentiable regularization component can also be added by providing a + * [[DifferentiableRegularization]] loss function. + * + * @param instances RDD containing the data to compute the loss function over. + * @param getAggregator A function which gets a new loss aggregator in every tree aggregate step. + * @param regularization An option representing the regularization loss function to apply to the + * coefficients. + * @param aggregationDepth The aggregation depth of the tree aggregation step. + * @tparam Agg Specialization of [[DifferentiableLossAggregator]], representing the concrete type + * of the aggregator. + */ +private[ml] class RDDLossFunction[ + T: ClassTag, + Agg <: DifferentiableLossAggregator[T, Agg]: ClassTag]( + instances: RDD[T], + getAggregator: (Broadcast[Vector] => Agg), + regularization: Option[DifferentiableRegularization[Vector]], + aggregationDepth: Int = 2) + extends DiffFunction[BDV[Double]] { + + override def calculate(coefficients: BDV[Double]): (Double, BDV[Double]) = { + val bcCoefficients = instances.context.broadcast(Vectors.fromBreeze(coefficients)) + val thisAgg = getAggregator(bcCoefficients) + val seqOp = (agg: Agg, x: T) => agg.add(x) + val combOp = (agg1: Agg, agg2: Agg) => agg1.merge(agg2) + val newAgg = instances.treeAggregate(thisAgg)(seqOp, combOp, aggregationDepth) + val gradient = newAgg.gradient + val regLoss = regularization.map { regFun => + val (regLoss, regGradient) = regFun.calculate(Vectors.fromBreeze(coefficients)) + BLAS.axpy(1.0, regGradient, gradient) + regLoss + }.getOrElse(0.0) + bcCoefficients.destroy(blocking = false) + (newAgg.loss + regLoss, gradient.asBreeze.toDenseVector) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/package-info.java b/mllib/src/main/scala/org/apache/spark/ml/package-info.java index 87f4223964ad..cb97382207b0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/package-info.java +++ b/mllib/src/main/scala/org/apache/spark/ml/package-info.java @@ -16,10 +16,7 @@ */ /** - * Spark ML is a BETA component that adds a new set of machine learning APIs to let users quickly - * assemble and configure practical machine learning pipelines. + * DataFrame-based machine learning APIs to let users quickly assemble and configure practical + * machine learning pipelines. */ -@Experimental package org.apache.spark.ml; - -import org.apache.spark.annotation.Experimental; 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 c589d06d9f7e..a445c675e41e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/package.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/package.scala @@ -18,8 +18,8 @@ package org.apache.spark /** - * Spark ML is a BETA component that adds a new set of machine learning APIs to let users quickly - * assemble and configure practical machine learning pipelines. + * DataFrame-based 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 diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 8361406f8729..ac68b825af53 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -18,16 +18,19 @@ package org.apache.spark.ml.param import java.lang.reflect.Modifier +import java.util.{List => JList} import java.util.NoSuchElementException import scala.annotation.varargs -import scala.collection.mutable import scala.collection.JavaConverters._ +import scala.collection.mutable import org.json4s._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.ml.linalg.JsonVectorConverter +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.util.Identifiable /** @@ -57,9 +60,8 @@ class Param[T](val parent: String, val name: String, val doc: String, val isVali /** * Assert that the given value is valid for this parameter. * - * Note: Parameter checks involving interactions between multiple parameters should be - * implemented in [[Params.validateParams()]]. Checks for input/output columns should be - * implemented in [[org.apache.spark.ml.PipelineStage.transformSchema()]]. + * Note: Parameter checks involving interactions between multiple parameters and input/output + * columns should be implemented in [[org.apache.spark.ml.PipelineStage.transformSchema()]]. * * DEVELOPERS: This method is only called by [[ParamPair]], which means that all parameters * should be specified via [[ParamPair]]. @@ -81,33 +83,30 @@ class Param[T](val parent: String, val name: String, val doc: String, val isVali def w(value: T): ParamPair[T] = this -> value /** Creates a param pair with the given value (for Scala). */ + // scalastyle:off def ->(value: T): ParamPair[T] = ParamPair(this, value) + // scalastyle:on - /** Encodes a param value into JSON, which can be decoded by [[jsonDecode()]]. */ + /** Encodes a param value into JSON, which can be decoded by `jsonDecode()`. */ def jsonEncode(value: T): String = { value match { case x: String => compact(render(JString(x))) + case v: Vector => + JsonVectorConverter.toJson(v) case _ => throw new NotImplementedError( - "The default jsonEncode only supports string. " + + "The default jsonEncode only supports string and vector. " + s"${this.getClass.getName} must override jsonEncode for ${value.getClass.getName}.") } } /** Decodes a param value from JSON. */ - def jsonDecode(json: String): T = { - parse(json) match { - case JString(x) => - x.asInstanceOf[T] - case _ => - throw new NotImplementedError( - "The default jsonDecode only supports string. " + - s"${this.getClass.getName} must override jsonDecode to support its value type.") - } - } + def jsonDecode(json: String): T = Param.jsonDecode[T](json) + + private[this] val stringRepresentation = s"${parent}__$name" - override final def toString: String = s"${parent}__$name" + override final def toString: String = stringRepresentation override final def hashCode: Int = toString.## @@ -119,9 +118,29 @@ class Param[T](val parent: String, val name: String, val doc: String, val isVali } } +private[ml] object Param { + + /** Decodes a param value from JSON. */ + def jsonDecode[T](json: String): T = { + parse(json) match { + case JString(x) => + x.asInstanceOf[T] + case JObject(v) => + val keys = v.map(_._1) + assert(keys.contains("type") && keys.contains("values"), + s"Expect a JSON serialized vector but cannot find fields 'type' and 'values' in $json.") + JsonVectorConverter.fromJson(json).asInstanceOf[T] + case _ => + throw new NotImplementedError( + "The default jsonDecode only supports string and vector. " + + s"${this.getClass.getName} must override jsonDecode to support its value type.") + } + } +} + /** * :: DeveloperApi :: - * Factory methods for common validation functions for [[Param.isValid]]. + * Factory methods for common validation functions for `Param.isValid`. * The numerical methods only support Int, Long, Float, and Double. */ @DeveloperApi @@ -146,32 +165,39 @@ object ParamValidators { s" of unexpected input type: ${value.getClass}") } - /** Check if value > lowerBound */ + /** + * Check if value is greater than lowerBound + */ def gt[T](lowerBound: Double): T => Boolean = { (value: T) => getDouble(value) > lowerBound } - /** Check if value >= lowerBound */ + /** + * Check if value is greater than or equal to lowerBound + */ def gtEq[T](lowerBound: Double): T => Boolean = { (value: T) => getDouble(value) >= lowerBound } - /** Check if value < upperBound */ + /** + * Check if value is less than upperBound + */ def lt[T](upperBound: Double): T => Boolean = { (value: T) => getDouble(value) < upperBound } - /** Check if value <= upperBound */ + /** + * Check if value is less than or equal to upperBound + */ def ltEq[T](upperBound: Double): T => Boolean = { (value: T) => getDouble(value) <= upperBound } /** * Check for value in range lowerBound to upperBound. - * @param lowerInclusive If true, check for value >= lowerBound. - * If false, check for value > lowerBound. - * @param upperInclusive If true, check for value <= upperBound. - * If false, check for value < upperBound. + * + * @param lowerInclusive if true, range includes value = lowerBound + * @param upperInclusive if true, range includes value = upperBound */ def inRange[T]( lowerBound: Double, @@ -184,7 +210,7 @@ object ParamValidators { lowerValid && upperValid } - /** Version of [[inRange()]] which uses inclusive be default: [lowerBound, upperBound] */ + /** Version of `inRange()` which uses inclusive be default: [lowerBound, upperBound] */ def inRange[T](lowerBound: Double, upperBound: Double): T => Boolean = { inRange[T](lowerBound, upperBound, lowerInclusive = true, upperInclusive = true) } @@ -209,7 +235,7 @@ object ParamValidators { /** * :: DeveloperApi :: - * Specialized version of [[Param[Double]]] for Java. + * Specialized version of `Param[Double]` for Java. */ @DeveloperApi class DoubleParam(parent: String, name: String, doc: String, isValid: Double => Boolean) @@ -269,7 +295,7 @@ private[param] object DoubleParam { /** * :: DeveloperApi :: - * Specialized version of [[Param[Int]]] for Java. + * Specialized version of `Param[Int]` for Java. */ @DeveloperApi class IntParam(parent: String, name: String, doc: String, isValid: Int => Boolean) @@ -298,7 +324,7 @@ class IntParam(parent: String, name: String, doc: String, isValid: Int => Boolea /** * :: DeveloperApi :: - * Specialized version of [[Param[Float]]] for Java. + * Specialized version of `Param[Float]` for Java. */ @DeveloperApi class FloatParam(parent: String, name: String, doc: String, isValid: Float => Boolean) @@ -359,7 +385,7 @@ private object FloatParam { /** * :: DeveloperApi :: - * Specialized version of [[Param[Long]]] for Java. + * Specialized version of `Param[Long]` for Java. */ @DeveloperApi class LongParam(parent: String, name: String, doc: String, isValid: Long => Boolean) @@ -388,7 +414,7 @@ class LongParam(parent: String, name: String, doc: String, isValid: Long => Bool /** * :: DeveloperApi :: - * Specialized version of [[Param[Boolean]]] for Java. + * Specialized version of `Param[Boolean]` for Java. */ @DeveloperApi class BooleanParam(parent: String, name: String, doc: String) // No need for isValid @@ -411,7 +437,7 @@ class BooleanParam(parent: String, name: String, doc: String) // No need for isV /** * :: DeveloperApi :: - * Specialized version of [[Param[Array[String]]]] for Java. + * Specialized version of `Param[Array[String]]` for Java. */ @DeveloperApi class StringArrayParam(parent: Params, name: String, doc: String, isValid: Array[String] => Boolean) @@ -420,7 +446,7 @@ class StringArrayParam(parent: Params, name: String, doc: String, isValid: Array def this(parent: Params, name: String, doc: String) = this(parent, name, doc, ParamValidators.alwaysTrue) - /** Creates a param pair with a [[java.util.List]] of values (for Java and Python). */ + /** Creates a param pair with a `java.util.List` of values (for Java and Python). */ def w(value: java.util.List[String]): ParamPair[Array[String]] = w(value.asScala.toArray) override def jsonEncode(value: Array[String]): String = { @@ -436,7 +462,7 @@ class StringArrayParam(parent: Params, name: String, doc: String, isValid: Array /** * :: DeveloperApi :: - * Specialized version of [[Param[Array[Double]]]] for Java. + * Specialized version of `Param[Array[Double]]` for Java. */ @DeveloperApi class DoubleArrayParam(parent: Params, name: String, doc: String, isValid: Array[Double] => Boolean) @@ -445,7 +471,7 @@ class DoubleArrayParam(parent: Params, name: String, doc: String, isValid: Array def this(parent: Params, name: String, doc: String) = this(parent, name, doc, ParamValidators.alwaysTrue) - /** Creates a param pair with a [[java.util.List]] of values (for Java and Python). */ + /** Creates a param pair with a `java.util.List` of values (for Java and Python). */ def w(value: java.util.List[java.lang.Double]): ParamPair[Array[Double]] = w(value.asScala.map(_.asInstanceOf[Double]).toArray) @@ -466,7 +492,7 @@ class DoubleArrayParam(parent: Params, name: String, doc: String, isValid: Array /** * :: DeveloperApi :: - * Specialized version of [[Param[Array[Int]]]] for Java. + * Specialized version of `Param[Array[Int]]` for Java. */ @DeveloperApi class IntArrayParam(parent: Params, name: String, doc: String, isValid: Array[Int] => Boolean) @@ -475,7 +501,7 @@ class IntArrayParam(parent: Params, name: String, doc: String, isValid: Array[In def this(parent: Params, name: String, doc: String) = this(parent, name, doc, ParamValidators.alwaysTrue) - /** Creates a param pair with a [[java.util.List]] of values (for Java and Python). */ + /** Creates a param pair with a `java.util.List` of values (for Java and Python). */ def w(value: java.util.List[java.lang.Integer]): ParamPair[Array[Int]] = w(value.asScala.map(_.asInstanceOf[Int]).toArray) @@ -491,11 +517,12 @@ class IntArrayParam(parent: Params, name: String, doc: String, isValid: Array[In } /** - * :: Experimental :: * A param and its value. */ -@Experimental -case class ParamPair[T](param: Param[T], value: T) { +@Since("1.2.0") +case class ParamPair[T] @Since("1.2.0") ( + @Since("1.2.0") param: Param[T], + @Since("1.2.0") value: T) { // This is *the* place Param.validate is called. Whenever a parameter is specified, we should // always construct a ParamPair so that validate is called. param.validate(value) @@ -513,7 +540,7 @@ trait Params extends Identifiable with Serializable { * Returns all params sorted by their names. The default implementation uses Java reflection to * list all public methods that have no arguments and return [[Param]]. * - * Note: Developer should not use this method in constructor because we cannot guarantee that + * @note Developer should not use this method in constructor because we cannot guarantee that * this variable gets initialized before other params. */ lazy val params: Array[Param[_]] = { @@ -526,19 +553,6 @@ trait Params extends Identifiable with Serializable { .map(m => m.invoke(this).asInstanceOf[Param[_]]) } - /** - * Validates parameter values stored internally. - * Raise an exception if any parameter value is invalid. - * - * This only needs to check for interactions between parameters. - * Parameter value checks which do not depend on other parameters are handled by - * [[Param.validate()]]. This method does not handle input/output column parameters; - * those are checked during schema validation. - */ - def validateParams(): Unit = { - // Do nothing by default. Override to handle Param interactions. - } - /** * Explains a param. * @param param input param, must belong to this instance. @@ -558,8 +572,7 @@ trait Params extends Identifiable with Serializable { } /** - * Explains all params of this instance. - * @see [[explainParam()]] + * Explains all params of this instance. See `explainParam()`. */ def explainParams(): String = { params.map(explainParam).mkString("\n") @@ -592,7 +605,7 @@ trait Params extends Identifiable with Serializable { /** * Sets a parameter in the embedded param map. */ - protected final def set[T](param: Param[T], value: T): this.type = { + final def set[T](param: Param[T], value: T): this.type = { set(param -> value) } @@ -639,7 +652,9 @@ trait Params extends Identifiable with Serializable { throw new NoSuchElementException(s"Failed to find a default value for ${param.name}")) } - /** An alias for [[getOrDefault()]]. */ + /** + * An alias for `getOrDefault()`. + */ protected final def $[T](param: Param[T]): T = getOrDefault(param) /** @@ -656,7 +671,7 @@ trait Params extends Identifiable with Serializable { /** * Sets default values for a list of params. * - * Note: Java developers should use the single-parameter [[setDefault()]]. + * Note: Java developers should use the single-parameter `setDefault`. * Annotating this with varargs can cause compilation failures due to a Scala compiler bug. * See SPARK-9268. * @@ -690,8 +705,7 @@ trait Params extends Identifiable with Serializable { /** * Creates a copy of this instance with the same UID and some extra params. * Subclasses should implement this method and set the return type properly. - * - * @see [[defaultCopy()]] + * See `defaultCopy()`. */ def copy(extra: ParamMap): Params @@ -708,14 +722,15 @@ trait Params extends Identifiable with Serializable { /** * Extracts the embedded default param values and user-supplied values, and then merges them with * extra values from input into a flat param map, where the latter value is used if there exist - * conflicts, i.e., with ordering: default param values < user-supplied values < extra. + * conflicts, i.e., with ordering: + * default param values less than user-supplied values less than extra. */ final def extractParamMap(extra: ParamMap): ParamMap = { defaultParamMap ++ paramMap ++ extra } /** - * [[extractParamMap]] with no extra values. + * `extractParamMap` with no extra values. */ final def extractParamMap(): ParamMap = { extractParamMap(ParamMap.empty) @@ -736,14 +751,14 @@ trait Params extends Identifiable with Serializable { * Copies param values from this instance to another instance for params shared by them. * * This handles default Params and explicitly set Params separately. - * Default Params are copied from and to [[defaultParamMap]], and explicitly set Params are - * copied from and to [[paramMap]]. + * Default Params are copied from and to `defaultParamMap`, and explicitly set Params are + * copied from and to `paramMap`. * Warning: This implicitly assumes that this [[Params]] instance and the target instance * share the same set of default Params. * * @param to the target instance, which should work with the same set of default Params as this * source instance - * @param extra extra params to be copied to the target's [[paramMap]] + * @param extra extra params to be copied to the target's `paramMap` * @return the target instance with param values copied */ protected def copyValues[T <: Params](to: T, extra: ParamMap = ParamMap.empty): T = { @@ -766,17 +781,16 @@ trait Params extends Identifiable with Serializable { * :: DeveloperApi :: * Java-friendly wrapper for [[Params]]. * Java developers who need to extend [[Params]] should use this class instead. - * If you need to extend a abstract class which already extends [[Params]], then that abstract + * If you need to extend an abstract class which already extends [[Params]], then that abstract * class should be Java-friendly as well. */ @DeveloperApi abstract class JavaParams extends Params /** - * :: Experimental :: * A param to value map. */ -@Experimental +@Since("1.2.0") final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) extends Serializable { @@ -789,17 +803,20 @@ final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) /** * Creates an empty param map. */ + @Since("1.2.0") def this() = this(mutable.Map.empty) /** * Puts a (param, value) pair (overwrites if the input param exists). */ + @Since("1.2.0") def put[T](param: Param[T], value: T): this.type = put(param -> value) /** * Puts a list of param pairs (overwrites if the input params exists). */ @varargs + @Since("1.2.0") def put(paramPairs: ParamPair[_]*): this.type = { paramPairs.foreach { p => map(p.param.asInstanceOf[Param[Any]]) = p.value @@ -807,9 +824,15 @@ final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) this } + /** Put param pairs with a `java.util.List` of values for Python. */ + private[ml] def put(paramPairs: JList[ParamPair[_]]): this.type = { + put(paramPairs.asScala: _*) + } + /** * Optionally returns the value associated with a param. */ + @Since("1.2.0") def get[T](param: Param[T]): Option[T] = { map.get(param.asInstanceOf[Param[Any]]).asInstanceOf[Option[T]] } @@ -817,6 +840,7 @@ final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) /** * Returns the value associated with a param or a default value. */ + @Since("1.4.0") def getOrElse[T](param: Param[T], default: T): T = { get(param).getOrElse(default) } @@ -825,6 +849,7 @@ final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) * Gets the value of the input param or its default value if it does not exist. * Raises a NoSuchElementException if there is no value associated with the input param. */ + @Since("1.2.0") def apply[T](param: Param[T]): T = { get(param).getOrElse { throw new NoSuchElementException(s"Cannot find param ${param.name}.") @@ -834,6 +859,7 @@ final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) /** * Checks whether a parameter is explicitly specified. */ + @Since("1.2.0") def contains(param: Param[_]): Boolean = { map.contains(param.asInstanceOf[Param[Any]]) } @@ -841,6 +867,7 @@ final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) /** * Removes a key from this map and returns its value associated previously as an option. */ + @Since("1.4.0") def remove[T](param: Param[T]): Option[T] = { map.remove(param.asInstanceOf[Param[Any]]).asInstanceOf[Option[T]] } @@ -848,16 +875,23 @@ final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) /** * Filters this param map for the given parent. */ + @Since("1.2.0") def filter(parent: Params): ParamMap = { - val filtered = map.filterKeys(_.parent == parent) - new ParamMap(filtered.asInstanceOf[mutable.Map[Param[Any], Any]]) + // Don't use filterKeys because mutable.Map#filterKeys + // returns the instance of collections.Map, not mutable.Map. + // Otherwise, we get ClassCastException. + // Not using filterKeys also avoid SI-6654 + val filtered = map.filter { case (k, _) => k.parent == parent.uid } + new ParamMap(filtered) } /** * Creates a copy of this param map. */ + @Since("1.2.0") def copy: ParamMap = new ParamMap(map.clone()) + @Since("1.2.0") override def toString: String = { map.toSeq.sortBy(_._1.name).map { case (param, value) => s"\t${param.parent}-${param.name}: $value" @@ -868,6 +902,7 @@ final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) * Returns a new param map that contains parameters in this map and the given map, * where the latter overwrites this if there exist conflicts. */ + @Since("1.2.0") def ++(other: ParamMap): ParamMap = { // TODO: Provide a better method name for Java users. new ParamMap(this.map ++ other.map) @@ -876,6 +911,7 @@ final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) /** * Adds all parameters from the input param map into this param map. */ + @Since("1.2.0") def ++=(other: ParamMap): this.type = { // TODO: Provide a better method name for Java users. this.map ++= other.map @@ -885,30 +921,39 @@ final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) /** * Converts this param map to a sequence of param pairs. */ + @Since("1.2.0") def toSeq: Seq[ParamPair[_]] = { map.toSeq.map { case (param, value) => ParamPair(param, value) } } + /** Java-friendly method for Python API */ + private[ml] def toList: java.util.List[ParamPair[_]] = { + this.toSeq.asJava + } + /** * Number of param pairs in this map. */ + @Since("1.3.0") def size: Int = map.size } -@Experimental +@Since("1.2.0") object ParamMap { /** * Returns an empty param map. */ + @Since("1.2.0") def empty: ParamMap = new ParamMap() /** * Constructs a param map by specifying its entries. */ @varargs + @Since("1.2.0") def apply(paramPairs: ParamPair[_]*): ParamMap = { new ParamMap().put(paramPairs: _*) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/HasParallelism.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/HasParallelism.scala new file mode 100644 index 000000000000..021d0b3e3416 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/HasParallelism.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.param.shared + +import scala.concurrent.ExecutionContext + +import org.apache.spark.ml.param.{IntParam, Params, ParamValidators} +import org.apache.spark.util.ThreadUtils + +/** + * Trait to define a level of parallelism for algorithms that are able to use + * multithreaded execution, and provide a thread-pool based execution context. + */ +private[ml] trait HasParallelism extends Params { + + /** + * The number of threads to use when running parallel algorithms. + * Default is 1 for serial execution + * + * @group expertParam + */ + val parallelism = new IntParam(this, "parallelism", + "the number of threads to use when running parallel algorithms", ParamValidators.gtEq(1)) + + setDefault(parallelism -> 1) + + /** @group expertGetParam */ + def getParallelism: Int = $(parallelism) + + /** + * Create a new execution context with a thread-pool that has a maximum number of threads + * set to the value of [[parallelism]]. If this param is set to 1, a same-thread executor + * will be used to run in serial. + */ + private[ml] def getExecutionContext: ExecutionContext = { + getParallelism match { + case 1 => + ThreadUtils.sameThread + case n => + ExecutionContext.fromExecutorService(ThreadUtils + .newDaemonCachedThreadPool(s"${this.getClass.getSimpleName}-thread-pool", n)) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala index c7bca1243092..1860fe836174 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -20,6 +20,7 @@ package org.apache.spark.ml.param.shared import java.io.PrintWriter import scala.reflect.ClassTag +import scala.xml.Utility /** * Code generator for shared params (sharedParams.scala). Run under the Spark folder with @@ -44,15 +45,18 @@ private[shared] object SharedParamsCodeGen { " probabilities. Note: Not all models output well-calibrated probability estimates!" + " These probabilities should be treated as confidences, not precise probabilities", Some("\"probability\"")), + ParamDesc[String]("varianceCol", "Column name for the biased sample variance of prediction"), ParamDesc[Double]("threshold", - "threshold in binary classification prediction, in range [0, 1]", Some("0.5"), - isValid = "ParamValidators.inRange(0, 1)", finalMethods = false), + "threshold in binary classification prediction, in range [0, 1]", + isValid = "ParamValidators.inRange(0, 1)", finalMethods = false, finalFields = false), ParamDesc[Array[Double]]("thresholds", "Thresholds in multi-class classification" + " to adjust the probability of predicting each class." + - " Array must have length equal to the number of classes, with values >= 0." + + " Array must have length equal to the number of classes, with values > 0" + + " excepting that at most one value may be 0." + " The class with largest value p/t is predicted, where p is the original probability" + - " of that class and t is the class' threshold.", - isValid = "(t: Array[Double]) => t.forall(_ >= 0)", finalMethods = false), + " of that class and t is the class's threshold", + isValid = "(t: Array[Double]) => t.forall(_ >= 0) && t.count(_ == 0) <= 1", + finalMethods = false), ParamDesc[String]("inputCol", "input column name"), ParamDesc[Array[String]]("inputCols", "input column names"), ParamDesc[String]("outputCol", "output column name", Some("uid + \"__output\"")), @@ -61,21 +65,24 @@ private[shared] object SharedParamsCodeGen { "every 10 iterations", isValid = "(interval: Int) => interval == -1 || interval >= 1"), ParamDesc[Boolean]("fitIntercept", "whether to fit an intercept term", Some("true")), ParamDesc[String]("handleInvalid", "how to handle invalid entries. Options are skip (which " + - "will filter out rows with bad values), or error (which will throw an errror). More " + - "options may be added later.", - isValid = "ParamValidators.inArray(Array(\"skip\", \"error\"))"), + "will filter out rows with bad values), or error (which will throw an error). More " + + "options may be added later", + isValid = "ParamValidators.inArray(Array(\"skip\", \"error\"))", finalFields = false), ParamDesc[Boolean]("standardization", "whether to standardize the training features" + " before fitting the model", Some("true")), ParamDesc[Long]("seed", "random seed", Some("this.getClass.getName.hashCode.toLong")), ParamDesc[Double]("elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]." + " For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty", isValid = "ParamValidators.inRange(0, 1)"), - ParamDesc[Double]("tol", "the convergence tolerance for iterative algorithms"), - ParamDesc[Double]("stepSize", "Step size to be used for each iteration of optimization."), + ParamDesc[Double]("tol", "the convergence tolerance for iterative algorithms (>= 0)", + isValid = "ParamValidators.gtEq(0)"), + ParamDesc[Double]("stepSize", "Step size to be used for each iteration of optimization (>" + + " 0)", isValid = "ParamValidators.gt(0)", finalFields = false), ParamDesc[String]("weightCol", "weight column name. If this is not set or empty, we treat " + - "all instance weights as 1.0."), - ParamDesc[String]("solver", "the solver algorithm for optimization. If this is not set or " + - "empty, default value is 'auto'.", Some("\"auto\""))) + "all instance weights as 1.0"), + ParamDesc[String]("solver", "the solver algorithm for optimization", finalFields = false), + ParamDesc[Int]("aggregationDepth", "suggested depth for treeAggregate (>= 2)", Some("2"), + isValid = "ParamValidators.gtEq(2)", isExpertParam = true)) val code = genSharedParams(params) val file = "src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala" @@ -90,7 +97,9 @@ private[shared] object SharedParamsCodeGen { doc: String, defaultValueStr: Option[String] = None, isValid: String = "", - finalMethods: Boolean = true) { + finalMethods: Boolean = true, + finalFields: Boolean = true, + isExpertParam: Boolean = false) { require(name.matches("[a-z][a-zA-Z0-9]*"), s"Param name $name is invalid.") require(doc.nonEmpty) // TODO: more rigorous on doc @@ -148,11 +157,23 @@ private[shared] object SharedParamsCodeGen { } else { "" } + val groupStr = if (param.isExpertParam) { + Array("expertParam", "expertGetParam") + } else { + Array("param", "getParam") + } val methodStr = if (param.finalMethods) { "final def" } else { "def" } + val fieldStr = if (param.finalFields) { + "final val" + } else { + "val" + } + + val htmlCompliantDoc = Utility.escape(doc) s""" |/** @@ -161,12 +182,12 @@ private[shared] object SharedParamsCodeGen { |private[ml] trait Has$Name extends Params { | | /** - | * Param for $doc. - | * @group param + | * Param for $htmlCompliantDoc. + | * @group ${groupStr(0)} | */ - | final val $name: $Param = new $Param(this, "$name", "$doc"$isValid) + | $fieldStr $name: $Param = new $Param(this, "$name", "$doc"$isValid) |$setDefault - | /** @group getParam */ + | /** @group ${groupStr(1)} */ | $methodStr get$Name: $T = $$($name) |} |""".stripMargin diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala index cb2a060a34dd..6061d9ca0a08 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -29,7 +29,7 @@ import org.apache.spark.ml.param._ private[ml] trait HasRegParam extends Params { /** - * Param for regularization parameter (>= 0). + * Param for regularization parameter (>= 0). * @group param */ final val regParam: DoubleParam = new DoubleParam(this, "regParam", "regularization parameter (>= 0)", ParamValidators.gtEq(0)) @@ -44,7 +44,7 @@ private[ml] trait HasRegParam extends Params { private[ml] trait HasMaxIter extends Params { /** - * Param for maximum number of iterations (>= 0). + * Param for maximum number of iterations (>= 0). * @group param */ final val maxIter: IntParam = new IntParam(this, "maxIter", "maximum number of iterations (>= 0)", ParamValidators.gtEq(0)) @@ -139,7 +139,22 @@ private[ml] trait HasProbabilityCol extends Params { } /** - * Trait for shared param threshold (default: 0.5). + * Trait for shared param varianceCol. + */ +private[ml] trait HasVarianceCol extends Params { + + /** + * Param for Column name for the biased sample variance of prediction. + * @group param + */ + final val varianceCol: Param[String] = new Param[String](this, "varianceCol", "Column name for the biased sample variance of prediction") + + /** @group getParam */ + final def getVarianceCol: String = $(varianceCol) +} + +/** + * Trait for shared param threshold. */ private[ml] trait HasThreshold extends Params { @@ -147,9 +162,7 @@ private[ml] trait HasThreshold extends Params { * Param for threshold in binary classification prediction, in range [0, 1]. * @group param */ - final val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in binary classification prediction, in range [0, 1]", ParamValidators.inRange(0, 1)) - - setDefault(threshold, 0.5) + val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in binary classification prediction, in range [0, 1]", ParamValidators.inRange(0, 1)) /** @group getParam */ def getThreshold: Double = $(threshold) @@ -161,10 +174,10 @@ private[ml] trait HasThreshold extends Params { private[ml] trait HasThresholds extends Params { /** - * Param for Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values >= 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class' threshold.. + * Param for Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values > 0 excepting that at most one value may be 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class's threshold. * @group param */ - final val thresholds: DoubleArrayParam = new DoubleArrayParam(this, "thresholds", "Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values >= 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class' threshold.", (t: Array[Double]) => t.forall(_ >= 0)) + final val thresholds: DoubleArrayParam = new DoubleArrayParam(this, "thresholds", "Thresholds in multi-class classification to adjust the probability of predicting each class. Array must have length equal to the number of classes, with values > 0 excepting that at most one value may be 0. The class with largest value p/t is predicted, where p is the original probability of that class and t is the class's threshold", (t: Array[Double]) => t.forall(_ >= 0) && t.count(_ == 0) <= 1) /** @group getParam */ def getThresholds: Array[Double] = $(thresholds) @@ -223,7 +236,7 @@ private[ml] trait HasOutputCol extends Params { private[ml] trait HasCheckpointInterval extends Params { /** - * Param for set checkpoint interval (>= 1) or disable checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations. + * Param for set checkpoint interval (>= 1) or disable checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations. * @group param */ final val checkpointInterval: IntParam = new IntParam(this, "checkpointInterval", "set checkpoint interval (>= 1) or disable checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations", (interval: Int) => interval == -1 || interval >= 1) @@ -255,10 +268,10 @@ private[ml] trait HasFitIntercept extends Params { private[ml] trait HasHandleInvalid extends Params { /** - * Param for how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later.. + * Param for how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an error). More options may be added later. * @group param */ - final val handleInvalid: Param[String] = new Param[String](this, "handleInvalid", "how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an errror). More options may be added later.", ParamValidators.inArray(Array("skip", "error"))) + val handleInvalid: Param[String] = new Param[String](this, "handleInvalid", "how to handle invalid entries. Options are skip (which will filter out rows with bad values), or error (which will throw an error). More options may be added later", ParamValidators.inArray(Array("skip", "error"))) /** @group getParam */ final def getHandleInvalid: String = $(handleInvalid) @@ -319,10 +332,10 @@ private[ml] trait HasElasticNetParam extends Params { private[ml] trait HasTol extends Params { /** - * Param for the convergence tolerance for iterative algorithms. + * Param for the convergence tolerance for iterative algorithms (>= 0). * @group param */ - final val tol: DoubleParam = new DoubleParam(this, "tol", "the convergence tolerance for iterative algorithms") + final val tol: DoubleParam = new DoubleParam(this, "tol", "the convergence tolerance for iterative algorithms (>= 0)", ParamValidators.gtEq(0)) /** @group getParam */ final def getTol: Double = $(tol) @@ -334,10 +347,10 @@ private[ml] trait HasTol extends Params { private[ml] trait HasStepSize extends Params { /** - * Param for Step size to be used for each iteration of optimization.. + * Param for Step size to be used for each iteration of optimization (> 0). * @group param */ - final val stepSize: DoubleParam = new DoubleParam(this, "stepSize", "Step size to be used for each iteration of optimization.") + val stepSize: DoubleParam = new DoubleParam(this, "stepSize", "Step size to be used for each iteration of optimization (> 0)", ParamValidators.gt(0)) /** @group getParam */ final def getStepSize: Double = $(stepSize) @@ -349,29 +362,44 @@ private[ml] trait HasStepSize extends Params { private[ml] trait HasWeightCol extends Params { /** - * Param for weight column name. If this is not set or empty, we treat all instance weights as 1.0.. + * Param for weight column name. If this is not set or empty, we treat all instance weights as 1.0. * @group param */ - final val weightCol: Param[String] = new Param[String](this, "weightCol", "weight column name. If this is not set or empty, we treat all instance weights as 1.0.") + final val weightCol: Param[String] = new Param[String](this, "weightCol", "weight column name. If this is not set or empty, we treat all instance weights as 1.0") /** @group getParam */ final def getWeightCol: String = $(weightCol) } /** - * Trait for shared param solver (default: "auto"). + * Trait for shared param solver. */ private[ml] trait HasSolver extends Params { /** - * Param for the solver algorithm for optimization. If this is not set or empty, default value is 'auto'.. + * Param for the solver algorithm for optimization. * @group param */ - final val solver: Param[String] = new Param[String](this, "solver", "the solver algorithm for optimization. If this is not set or empty, default value is 'auto'.") - - setDefault(solver, "auto") + val solver: Param[String] = new Param[String](this, "solver", "the solver algorithm for optimization") /** @group getParam */ final def getSolver: String = $(solver) } + +/** + * Trait for shared param aggregationDepth (default: 2). + */ +private[ml] trait HasAggregationDepth extends Params { + + /** + * Param for suggested depth for treeAggregate (>= 2). + * @group expertParam + */ + final val aggregationDepth: IntParam = new IntParam(this, "aggregationDepth", "suggested depth for treeAggregate (>= 2)", ParamValidators.gtEq(2)) + + setDefault(aggregationDepth, 2) + + /** @group expertGetParam */ + final def getAggregationDepth: Int = $(aggregationDepth) +} // scalastyle:on diff --git a/mllib/src/main/scala/org/apache/spark/ml/python/MLSerDe.scala b/mllib/src/main/scala/org/apache/spark/ml/python/MLSerDe.scala new file mode 100644 index 000000000000..da62f8518e36 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/python/MLSerDe.scala @@ -0,0 +1,218 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.python + +import java.io.OutputStream +import java.nio.{ByteBuffer, ByteOrder} + +import net.razorvine.pickle._ + +import org.apache.spark.api.python.SerDeUtil +import org.apache.spark.ml.linalg._ +import org.apache.spark.mllib.api.python.SerDeBase + +/** + * SerDe utility functions for pyspark.ml. + */ +private[spark] object MLSerDe extends SerDeBase with Serializable { + + override val PYSPARK_PACKAGE = "pyspark.ml" + + // Pickler for DenseVector + private[python] class DenseVectorPickler extends BasePickler[DenseVector] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val vector: DenseVector = obj.asInstanceOf[DenseVector] + val bytes = new Array[Byte](8 * vector.size) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + val db = bb.asDoubleBuffer() + db.put(vector.values) + + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(bytes.length)) + out.write(bytes) + out.write(Opcodes.TUPLE1) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 1) { + throw new PickleException("length of args should be 1") + } + val bytes = getBytes(args(0)) + val bb = ByteBuffer.wrap(bytes, 0, bytes.length) + bb.order(ByteOrder.nativeOrder()) + val db = bb.asDoubleBuffer() + val ans = new Array[Double](bytes.length / 8) + db.get(ans) + Vectors.dense(ans) + } + } + + // Pickler for DenseMatrix + private[python] class DenseMatrixPickler extends BasePickler[DenseMatrix] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val m: DenseMatrix = obj.asInstanceOf[DenseMatrix] + val bytes = new Array[Byte](8 * m.values.length) + val order = ByteOrder.nativeOrder() + val isTransposed = if (m.isTransposed) 1 else 0 + ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().put(m.values) + + out.write(Opcodes.MARK) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(m.numRows)) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(m.numCols)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(bytes.length)) + out.write(bytes) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(isTransposed)) + out.write(Opcodes.TUPLE) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 4) { + throw new PickleException("length of args should be 4") + } + val bytes = getBytes(args(2)) + val n = bytes.length / 8 + val values = new Array[Double](n) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().get(values) + val isTransposed = args(3).asInstanceOf[Int] == 1 + new DenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], values, isTransposed) + } + } + + // Pickler for SparseMatrix + private[python] class SparseMatrixPickler extends BasePickler[SparseMatrix] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val s = obj.asInstanceOf[SparseMatrix] + val order = ByteOrder.nativeOrder() + + val colPtrsBytes = new Array[Byte](4 * s.colPtrs.length) + val indicesBytes = new Array[Byte](4 * s.rowIndices.length) + val valuesBytes = new Array[Byte](8 * s.values.length) + val isTransposed = if (s.isTransposed) 1 else 0 + ByteBuffer.wrap(colPtrsBytes).order(order).asIntBuffer().put(s.colPtrs) + ByteBuffer.wrap(indicesBytes).order(order).asIntBuffer().put(s.rowIndices) + ByteBuffer.wrap(valuesBytes).order(order).asDoubleBuffer().put(s.values) + + out.write(Opcodes.MARK) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(s.numRows)) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(s.numCols)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(colPtrsBytes.length)) + out.write(colPtrsBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(indicesBytes.length)) + out.write(indicesBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(valuesBytes.length)) + out.write(valuesBytes) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(isTransposed)) + out.write(Opcodes.TUPLE) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 6) { + throw new PickleException("length of args should be 6") + } + val order = ByteOrder.nativeOrder() + val colPtrsBytes = getBytes(args(2)) + val indicesBytes = getBytes(args(3)) + val valuesBytes = getBytes(args(4)) + val colPtrs = new Array[Int](colPtrsBytes.length / 4) + val rowIndices = new Array[Int](indicesBytes.length / 4) + val values = new Array[Double](valuesBytes.length / 8) + ByteBuffer.wrap(colPtrsBytes).order(order).asIntBuffer().get(colPtrs) + ByteBuffer.wrap(indicesBytes).order(order).asIntBuffer().get(rowIndices) + ByteBuffer.wrap(valuesBytes).order(order).asDoubleBuffer().get(values) + val isTransposed = args(5).asInstanceOf[Int] == 1 + new SparseMatrix( + args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], colPtrs, rowIndices, values, + isTransposed) + } + } + + // Pickler for SparseVector + private[python] class SparseVectorPickler extends BasePickler[SparseVector] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + val v: SparseVector = obj.asInstanceOf[SparseVector] + val n = v.indices.length + val indiceBytes = new Array[Byte](4 * n) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().put(v.indices) + val valueBytes = new Array[Byte](8 * n) + ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().put(v.values) + + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(v.size)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(indiceBytes.length)) + out.write(indiceBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(valueBytes.length)) + out.write(valueBytes) + out.write(Opcodes.TUPLE3) + } + + def construct(args: Array[Object]): Object = { + if (args.length != 3) { + throw new PickleException("length of args should be 3") + } + val size = args(0).asInstanceOf[Int] + val indiceBytes = getBytes(args(1)) + val valueBytes = getBytes(args(2)) + val n = indiceBytes.length / 4 + val indices = new Array[Int](n) + val values = new Array[Double](n) + if (n > 0) { + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().get(indices) + ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().get(values) + } + new SparseVector(size, indices, values) + } + } + + var initialized = false + // This should be called before trying to serialize any above classes + // In cluster mode, this should be put in the closure + override def initialize(): Unit = { + SerDeUtil.initialize() + synchronized { + if (!initialized) { + new DenseVectorPickler().register() + new DenseMatrixPickler().register() + new SparseMatrixPickler().register() + new SparseVectorPickler().register() + initialized = true + } + } + } + // will not called in Executor automatically + initialize() +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala new file mode 100644 index 000000000000..80d03ab03c87 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.SparkException +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.feature.RFormula +import org.apache.spark.ml.regression.{AFTSurvivalRegression, AFTSurvivalRegressionModel} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class AFTSurvivalRegressionWrapper private ( + val pipeline: PipelineModel, + val features: Array[String]) extends MLWritable { + + private val aftModel: AFTSurvivalRegressionModel = + pipeline.stages(1).asInstanceOf[AFTSurvivalRegressionModel] + + lazy val rCoefficients: Array[Double] = if (aftModel.getFitIntercept) { + Array(aftModel.intercept) ++ aftModel.coefficients.toArray ++ Array(math.log(aftModel.scale)) + } else { + aftModel.coefficients.toArray ++ Array(math.log(aftModel.scale)) + } + + lazy val rFeatures: Array[String] = if (aftModel.getFitIntercept) { + Array("(Intercept)") ++ features ++ Array("Log(scale)") + } else { + features ++ Array("Log(scale)") + } + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset).drop(aftModel.getFeaturesCol) + } + + override def write: MLWriter = + new AFTSurvivalRegressionWrapper.AFTSurvivalRegressionWrapperWriter(this) +} + +private[r] object AFTSurvivalRegressionWrapper extends MLReadable[AFTSurvivalRegressionWrapper] { + + private def formulaRewrite(formula: String): (String, String) = { + var rewritedFormula: String = null + var censorCol: String = null + + val regex = """Surv\(([^,]+), ([^,]+)\) ~ (.+)""".r + try { + val regex(label, censor, features) = formula + // TODO: Support dot operator. + if (features.contains(".")) { + throw new UnsupportedOperationException( + "Terms of survreg formula can not support dot operator.") + } + rewritedFormula = label.trim + "~" + features.trim + censorCol = censor.trim + } catch { + case e: MatchError => + throw new SparkException(s"Could not parse formula: $formula") + } + + (rewritedFormula, censorCol) + } + + + def fit( + formula: String, + data: DataFrame, + aggregationDepth: Int, + stringIndexerOrderType: String): AFTSurvivalRegressionWrapper = { + + val (rewritedFormula, censorCol) = formulaRewrite(formula) + + val rFormula = new RFormula().setFormula(rewritedFormula) + .setStringIndexerOrderType(stringIndexerOrderType) + RWrapperUtils.checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // get feature names from output schema + val schema = rFormulaModel.transform(data).schema + val featureAttrs = AttributeGroup.fromStructField(schema(rFormula.getFeaturesCol)) + .attributes.get + val features = featureAttrs.map(_.name.get) + + val aft = new AFTSurvivalRegression() + .setCensorCol(censorCol) + .setFitIntercept(rFormula.hasIntercept) + .setFeaturesCol(rFormula.getFeaturesCol) + .setAggregationDepth(aggregationDepth) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, aft)) + .fit(data) + + new AFTSurvivalRegressionWrapper(pipeline, features) + } + + override def read: MLReader[AFTSurvivalRegressionWrapper] = new AFTSurvivalRegressionWrapperReader + + override def load(path: String): AFTSurvivalRegressionWrapper = super.load(path) + + class AFTSurvivalRegressionWrapperWriter(instance: AFTSurvivalRegressionWrapper) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("features" -> instance.features.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + + instance.pipeline.save(pipelinePath) + } + } + + class AFTSurvivalRegressionWrapperReader extends MLReader[AFTSurvivalRegressionWrapper] { + + override def load(path: String): AFTSurvivalRegressionWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val features = (rMetadata \ "features").extract[Array[String]] + + val pipeline = PipelineModel.load(pipelinePath) + new AFTSurvivalRegressionWrapper(pipeline, features) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/ALSWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/ALSWrapper.scala new file mode 100644 index 000000000000..ad13cced4667 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/ALSWrapper.scala @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.recommendation.{ALS, ALSModel} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class ALSWrapper private ( + val alsModel: ALSModel, + val ratingCol: String) extends MLWritable { + + lazy val userCol: String = alsModel.getUserCol + lazy val itemCol: String = alsModel.getItemCol + lazy val userFactors: DataFrame = alsModel.userFactors + lazy val itemFactors: DataFrame = alsModel.itemFactors + lazy val rank: Int = alsModel.rank + + def transform(dataset: Dataset[_]): DataFrame = { + alsModel.transform(dataset) + } + + override def write: MLWriter = new ALSWrapper.ALSWrapperWriter(this) +} + +private[r] object ALSWrapper extends MLReadable[ALSWrapper] { + + def fit( // scalastyle:ignore + data: DataFrame, + ratingCol: String, + userCol: String, + itemCol: String, + rank: Int, + regParam: Double, + maxIter: Int, + implicitPrefs: Boolean, + alpha: Double, + nonnegative: Boolean, + numUserBlocks: Int, + numItemBlocks: Int, + checkpointInterval: Int, + seed: Int): ALSWrapper = { + + val als = new ALS() + .setRatingCol(ratingCol) + .setUserCol(userCol) + .setItemCol(itemCol) + .setRank(rank) + .setRegParam(regParam) + .setMaxIter(maxIter) + .setImplicitPrefs(implicitPrefs) + .setAlpha(alpha) + .setNonnegative(nonnegative) + .setNumBlocks(numUserBlocks) + .setNumItemBlocks(numItemBlocks) + .setCheckpointInterval(checkpointInterval) + .setSeed(seed.toLong) + + val alsModel: ALSModel = als.fit(data) + + new ALSWrapper(alsModel, ratingCol) + } + + override def read: MLReader[ALSWrapper] = new ALSWrapperReader + + override def load(path: String): ALSWrapper = super.load(path) + + class ALSWrapperWriter(instance: ALSWrapper) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val modelPath = new Path(path, "model").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("ratingCol" -> instance.ratingCol) + val rMetadataJson: String = compact(render(rMetadata)) + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + + instance.alsModel.save(modelPath) + } + } + + class ALSWrapperReader extends MLReader[ALSWrapper] { + + override def load(path: String): ALSWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val modelPath = new Path(path, "model").toString + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val ratingCol = (rMetadata \ "ratingCol").extract[String] + val alsModel = ALSModel.load(modelPath) + + new ALSWrapper(alsModel, ratingCol) + } + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/BisectingKMeansWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/BisectingKMeansWrapper.scala new file mode 100644 index 000000000000..71712c1c5eec --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/BisectingKMeansWrapper.scala @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.clustering.{BisectingKMeans, BisectingKMeansModel} +import org.apache.spark.ml.feature.RFormula +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class BisectingKMeansWrapper private ( + val pipeline: PipelineModel, + val features: Array[String], + val size: Array[Long], + val isLoaded: Boolean = false) extends MLWritable { + private val bisectingKmeansModel: BisectingKMeansModel = + pipeline.stages.last.asInstanceOf[BisectingKMeansModel] + + lazy val coefficients: Array[Double] = bisectingKmeansModel.clusterCenters.flatMap(_.toArray) + + lazy val k: Int = bisectingKmeansModel.getK + + // If the model is loaded from a saved model, cluster is NULL. It is checked on R side + lazy val cluster: DataFrame = bisectingKmeansModel.summary.cluster + + def fitted(method: String): DataFrame = { + if (method == "centers") { + bisectingKmeansModel.summary.predictions.drop(bisectingKmeansModel.getFeaturesCol) + } else if (method == "classes") { + bisectingKmeansModel.summary.cluster + } else { + throw new UnsupportedOperationException( + s"Method (centers or classes) required but $method found.") + } + } + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset).drop(bisectingKmeansModel.getFeaturesCol) + } + + override def write: MLWriter = new BisectingKMeansWrapper.BisectingKMeansWrapperWriter(this) +} + +private[r] object BisectingKMeansWrapper extends MLReadable[BisectingKMeansWrapper] { + + def fit( + data: DataFrame, + formula: String, + k: Int, + maxIter: Int, + seed: String, + minDivisibleClusterSize: Double + ): BisectingKMeansWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + .setFeaturesCol("features") + RWrapperUtils.checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // get feature names from output schema + val schema = rFormulaModel.transform(data).schema + val featureAttrs = AttributeGroup.fromStructField(schema(rFormulaModel.getFeaturesCol)) + .attributes.get + val features = featureAttrs.map(_.name.get) + + val bisectingKmeans = new BisectingKMeans() + .setK(k) + .setMaxIter(maxIter) + .setMinDivisibleClusterSize(minDivisibleClusterSize) + .setFeaturesCol(rFormula.getFeaturesCol) + + if (seed != null && seed.length > 0) bisectingKmeans.setSeed(seed.toInt) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, bisectingKmeans)) + .fit(data) + + val bisectingKmeansModel: BisectingKMeansModel = + pipeline.stages.last.asInstanceOf[BisectingKMeansModel] + val size: Array[Long] = bisectingKmeansModel.summary.clusterSizes + + new BisectingKMeansWrapper(pipeline, features, size) + } + + override def read: MLReader[BisectingKMeansWrapper] = new BisectingKMeansWrapperReader + + override def load(path: String): BisectingKMeansWrapper = super.load(path) + + class BisectingKMeansWrapperWriter(instance: BisectingKMeansWrapper) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("features" -> instance.features.toSeq) ~ + ("size" -> instance.size.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + instance.pipeline.save(pipelinePath) + } + } + + class BisectingKMeansWrapperReader extends MLReader[BisectingKMeansWrapper] { + + override def load(path: String): BisectingKMeansWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + val pipeline = PipelineModel.load(pipelinePath) + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val features = (rMetadata \ "features").extract[Array[String]] + val size = (rMetadata \ "size").extract[Array[Long]] + new BisectingKMeansWrapper(pipeline, features, size, isLoaded = true) + } + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassificationWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassificationWrapper.scala new file mode 100644 index 000000000000..a90cae5869b2 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassificationWrapper.scala @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.classification.{DecisionTreeClassificationModel, DecisionTreeClassifier} +import org.apache.spark.ml.feature.{IndexToString, RFormula} +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.r.RWrapperUtils._ +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class DecisionTreeClassifierWrapper private ( + val pipeline: PipelineModel, + val formula: String, + val features: Array[String]) extends MLWritable { + + import DecisionTreeClassifierWrapper._ + + private val dtcModel: DecisionTreeClassificationModel = + pipeline.stages(1).asInstanceOf[DecisionTreeClassificationModel] + + lazy val numFeatures: Int = dtcModel.numFeatures + lazy val featureImportances: Vector = dtcModel.featureImportances + lazy val maxDepth: Int = dtcModel.getMaxDepth + + def summary: String = dtcModel.toDebugString + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset) + .drop(PREDICTED_LABEL_INDEX_COL) + .drop(dtcModel.getFeaturesCol) + .drop(dtcModel.getLabelCol) + } + + override def write: MLWriter = new + DecisionTreeClassifierWrapper.DecisionTreeClassifierWrapperWriter(this) +} + +private[r] object DecisionTreeClassifierWrapper extends MLReadable[DecisionTreeClassifierWrapper] { + + val PREDICTED_LABEL_INDEX_COL = "pred_label_idx" + val PREDICTED_LABEL_COL = "prediction" + + def fit( // scalastyle:ignore + data: DataFrame, + formula: String, + maxDepth: Int, + maxBins: Int, + impurity: String, + minInstancesPerNode: Int, + minInfoGain: Double, + checkpointInterval: Int, + seed: String, + maxMemoryInMB: Int, + cacheNodeIds: Boolean, + handleInvalid: String): DecisionTreeClassifierWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + .setForceIndexLabel(true) + .setHandleInvalid(handleInvalid) + checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // get labels and feature names from output schema + val (features, labels) = getFeaturesAndLabels(rFormulaModel, data) + + // assemble and fit the pipeline + val dtc = new DecisionTreeClassifier() + .setMaxDepth(maxDepth) + .setMaxBins(maxBins) + .setImpurity(impurity) + .setMinInstancesPerNode(minInstancesPerNode) + .setMinInfoGain(minInfoGain) + .setCheckpointInterval(checkpointInterval) + .setMaxMemoryInMB(maxMemoryInMB) + .setCacheNodeIds(cacheNodeIds) + .setFeaturesCol(rFormula.getFeaturesCol) + .setLabelCol(rFormula.getLabelCol) + .setPredictionCol(PREDICTED_LABEL_INDEX_COL) + if (seed != null && seed.length > 0) dtc.setSeed(seed.toLong) + + val idxToStr = new IndexToString() + .setInputCol(PREDICTED_LABEL_INDEX_COL) + .setOutputCol(PREDICTED_LABEL_COL) + .setLabels(labels) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, dtc, idxToStr)) + .fit(data) + + new DecisionTreeClassifierWrapper(pipeline, formula, features) + } + + override def read: MLReader[DecisionTreeClassifierWrapper] = + new DecisionTreeClassifierWrapperReader + + override def load(path: String): DecisionTreeClassifierWrapper = super.load(path) + + class DecisionTreeClassifierWrapperWriter(instance: DecisionTreeClassifierWrapper) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("formula" -> instance.formula) ~ + ("features" -> instance.features.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + instance.pipeline.save(pipelinePath) + } + } + + class DecisionTreeClassifierWrapperReader extends MLReader[DecisionTreeClassifierWrapper] { + + override def load(path: String): DecisionTreeClassifierWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + val pipeline = PipelineModel.load(pipelinePath) + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val formula = (rMetadata \ "formula").extract[String] + val features = (rMetadata \ "features").extract[Array[String]] + + new DecisionTreeClassifierWrapper(pipeline, formula, features) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressionWrapper.scala new file mode 100644 index 000000000000..de712d67e6df --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressionWrapper.scala @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.feature.RFormula +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class DecisionTreeRegressorWrapper private ( + val pipeline: PipelineModel, + val formula: String, + val features: Array[String]) extends MLWritable { + + private val dtrModel: DecisionTreeRegressionModel = + pipeline.stages(1).asInstanceOf[DecisionTreeRegressionModel] + + lazy val numFeatures: Int = dtrModel.numFeatures + lazy val featureImportances: Vector = dtrModel.featureImportances + lazy val maxDepth: Int = dtrModel.getMaxDepth + + def summary: String = dtrModel.toDebugString + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset).drop(dtrModel.getFeaturesCol) + } + + override def write: MLWriter = new + DecisionTreeRegressorWrapper.DecisionTreeRegressorWrapperWriter(this) +} + +private[r] object DecisionTreeRegressorWrapper extends MLReadable[DecisionTreeRegressorWrapper] { + def fit( // scalastyle:ignore + data: DataFrame, + formula: String, + maxDepth: Int, + maxBins: Int, + impurity: String, + minInstancesPerNode: Int, + minInfoGain: Double, + checkpointInterval: Int, + seed: String, + maxMemoryInMB: Int, + cacheNodeIds: Boolean): DecisionTreeRegressorWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + RWrapperUtils.checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // get feature names from output schema + val schema = rFormulaModel.transform(data).schema + val featureAttrs = AttributeGroup.fromStructField(schema(rFormulaModel.getFeaturesCol)) + .attributes.get + val features = featureAttrs.map(_.name.get) + + // assemble and fit the pipeline + val dtr = new DecisionTreeRegressor() + .setMaxDepth(maxDepth) + .setMaxBins(maxBins) + .setImpurity(impurity) + .setMinInstancesPerNode(minInstancesPerNode) + .setMinInfoGain(minInfoGain) + .setCheckpointInterval(checkpointInterval) + .setMaxMemoryInMB(maxMemoryInMB) + .setCacheNodeIds(cacheNodeIds) + .setFeaturesCol(rFormula.getFeaturesCol) + if (seed != null && seed.length > 0) dtr.setSeed(seed.toLong) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, dtr)) + .fit(data) + + new DecisionTreeRegressorWrapper(pipeline, formula, features) + } + + override def read: MLReader[DecisionTreeRegressorWrapper] = new DecisionTreeRegressorWrapperReader + + override def load(path: String): DecisionTreeRegressorWrapper = super.load(path) + + class DecisionTreeRegressorWrapperWriter(instance: DecisionTreeRegressorWrapper) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("formula" -> instance.formula) ~ + ("features" -> instance.features.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + instance.pipeline.save(pipelinePath) + } + } + + class DecisionTreeRegressorWrapperReader extends MLReader[DecisionTreeRegressorWrapper] { + + override def load(path: String): DecisionTreeRegressorWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + val pipeline = PipelineModel.load(pipelinePath) + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val formula = (rMetadata \ "formula").extract[String] + val features = (rMetadata \ "features").extract[Array[String]] + + new DecisionTreeRegressorWrapper(pipeline, formula, features) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/FPGrowthWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/FPGrowthWrapper.scala new file mode 100644 index 000000000000..b8151d8d9070 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/FPGrowthWrapper.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.fpm.{FPGrowth, FPGrowthModel} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class FPGrowthWrapper private (val fpGrowthModel: FPGrowthModel) extends MLWritable { + def freqItemsets: DataFrame = fpGrowthModel.freqItemsets + def associationRules: DataFrame = fpGrowthModel.associationRules + + def transform(dataset: Dataset[_]): DataFrame = { + fpGrowthModel.transform(dataset) + } + + override def write: MLWriter = new FPGrowthWrapper.FPGrowthWrapperWriter(this) +} + +private[r] object FPGrowthWrapper extends MLReadable[FPGrowthWrapper] { + + def fit( + data: DataFrame, + minSupport: Double, + minConfidence: Double, + itemsCol: String, + numPartitions: Integer): FPGrowthWrapper = { + val fpGrowth = new FPGrowth() + .setMinSupport(minSupport) + .setMinConfidence(minConfidence) + .setItemsCol(itemsCol) + + if (numPartitions != null && numPartitions > 0) { + fpGrowth.setNumPartitions(numPartitions) + } + + val fpGrowthModel = fpGrowth.fit(data) + + new FPGrowthWrapper(fpGrowthModel) + } + + override def read: MLReader[FPGrowthWrapper] = new FPGrowthWrapperReader + + class FPGrowthWrapperReader extends MLReader[FPGrowthWrapper] { + override def load(path: String): FPGrowthWrapper = { + val modelPath = new Path(path, "model").toString + val fPGrowthModel = FPGrowthModel.load(modelPath) + + new FPGrowthWrapper(fPGrowthModel) + } + } + + class FPGrowthWrapperWriter(instance: FPGrowthWrapper) extends MLWriter { + override protected def saveImpl(path: String): Unit = { + val modelPath = new Path(path, "model").toString + val rMetadataPath = new Path(path, "rMetadata").toString + + val rMetadataJson: String = compact(render( + "class" -> instance.getClass.getName + )) + + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + + instance.fpGrowthModel.save(modelPath) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GBTClassificationWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GBTClassificationWrapper.scala new file mode 100644 index 000000000000..ecaeec5a7791 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GBTClassificationWrapper.scala @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.classification.{GBTClassificationModel, GBTClassifier} +import org.apache.spark.ml.feature.{IndexToString, RFormula} +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.r.RWrapperUtils._ +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class GBTClassifierWrapper private ( + val pipeline: PipelineModel, + val formula: String, + val features: Array[String]) extends MLWritable { + + import GBTClassifierWrapper._ + + private val gbtcModel: GBTClassificationModel = + pipeline.stages(1).asInstanceOf[GBTClassificationModel] + + lazy val numFeatures: Int = gbtcModel.numFeatures + lazy val featureImportances: Vector = gbtcModel.featureImportances + lazy val numTrees: Int = gbtcModel.getNumTrees + lazy val treeWeights: Array[Double] = gbtcModel.treeWeights + lazy val maxDepth: Int = gbtcModel.getMaxDepth + + def summary: String = gbtcModel.toDebugString + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset) + .drop(PREDICTED_LABEL_INDEX_COL) + .drop(gbtcModel.getFeaturesCol) + .drop(gbtcModel.getLabelCol) + } + + override def write: MLWriter = new + GBTClassifierWrapper.GBTClassifierWrapperWriter(this) +} + +private[r] object GBTClassifierWrapper extends MLReadable[GBTClassifierWrapper] { + + val PREDICTED_LABEL_INDEX_COL = "pred_label_idx" + val PREDICTED_LABEL_COL = "prediction" + + def fit( // scalastyle:ignore + data: DataFrame, + formula: String, + maxDepth: Int, + maxBins: Int, + maxIter: Int, + stepSize: Double, + minInstancesPerNode: Int, + minInfoGain: Double, + checkpointInterval: Int, + lossType: String, + seed: String, + subsamplingRate: Double, + maxMemoryInMB: Int, + cacheNodeIds: Boolean, + handleInvalid: String): GBTClassifierWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + .setForceIndexLabel(true) + .setHandleInvalid(handleInvalid) + checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // get labels and feature names from output schema + val (features, labels) = getFeaturesAndLabels(rFormulaModel, data) + + // assemble and fit the pipeline + val rfc = new GBTClassifier() + .setMaxDepth(maxDepth) + .setMaxBins(maxBins) + .setMaxIter(maxIter) + .setStepSize(stepSize) + .setMinInstancesPerNode(minInstancesPerNode) + .setMinInfoGain(minInfoGain) + .setCheckpointInterval(checkpointInterval) + .setLossType(lossType) + .setSubsamplingRate(subsamplingRate) + .setMaxMemoryInMB(maxMemoryInMB) + .setCacheNodeIds(cacheNodeIds) + .setFeaturesCol(rFormula.getFeaturesCol) + .setLabelCol(rFormula.getLabelCol) + .setPredictionCol(PREDICTED_LABEL_INDEX_COL) + if (seed != null && seed.length > 0) rfc.setSeed(seed.toLong) + + val idxToStr = new IndexToString() + .setInputCol(PREDICTED_LABEL_INDEX_COL) + .setOutputCol(PREDICTED_LABEL_COL) + .setLabels(labels) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, rfc, idxToStr)) + .fit(data) + + new GBTClassifierWrapper(pipeline, formula, features) + } + + override def read: MLReader[GBTClassifierWrapper] = new GBTClassifierWrapperReader + + override def load(path: String): GBTClassifierWrapper = super.load(path) + + class GBTClassifierWrapperWriter(instance: GBTClassifierWrapper) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("formula" -> instance.formula) ~ + ("features" -> instance.features.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + instance.pipeline.save(pipelinePath) + } + } + + class GBTClassifierWrapperReader extends MLReader[GBTClassifierWrapper] { + + override def load(path: String): GBTClassifierWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + val pipeline = PipelineModel.load(pipelinePath) + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val formula = (rMetadata \ "formula").extract[String] + val features = (rMetadata \ "features").extract[Array[String]] + + new GBTClassifierWrapper(pipeline, formula, features) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressionWrapper.scala new file mode 100644 index 000000000000..b568d7859221 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressionWrapper.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.feature.RFormula +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.regression.{GBTRegressionModel, GBTRegressor} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class GBTRegressorWrapper private ( + val pipeline: PipelineModel, + val formula: String, + val features: Array[String]) extends MLWritable { + + private val gbtrModel: GBTRegressionModel = + pipeline.stages(1).asInstanceOf[GBTRegressionModel] + + lazy val numFeatures: Int = gbtrModel.numFeatures + lazy val featureImportances: Vector = gbtrModel.featureImportances + lazy val numTrees: Int = gbtrModel.getNumTrees + lazy val treeWeights: Array[Double] = gbtrModel.treeWeights + lazy val maxDepth: Int = gbtrModel.getMaxDepth + + def summary: String = gbtrModel.toDebugString + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset).drop(gbtrModel.getFeaturesCol) + } + + override def write: MLWriter = new + GBTRegressorWrapper.GBTRegressorWrapperWriter(this) +} + +private[r] object GBTRegressorWrapper extends MLReadable[GBTRegressorWrapper] { + def fit( // scalastyle:ignore + data: DataFrame, + formula: String, + maxDepth: Int, + maxBins: Int, + maxIter: Int, + stepSize: Double, + minInstancesPerNode: Int, + minInfoGain: Double, + checkpointInterval: Int, + lossType: String, + seed: String, + subsamplingRate: Double, + maxMemoryInMB: Int, + cacheNodeIds: Boolean): GBTRegressorWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + RWrapperUtils.checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // get feature names from output schema + val schema = rFormulaModel.transform(data).schema + val featureAttrs = AttributeGroup.fromStructField(schema(rFormulaModel.getFeaturesCol)) + .attributes.get + val features = featureAttrs.map(_.name.get) + + // assemble and fit the pipeline + val rfr = new GBTRegressor() + .setMaxDepth(maxDepth) + .setMaxBins(maxBins) + .setMaxIter(maxIter) + .setStepSize(stepSize) + .setMinInstancesPerNode(minInstancesPerNode) + .setMinInfoGain(minInfoGain) + .setCheckpointInterval(checkpointInterval) + .setLossType(lossType) + .setSubsamplingRate(subsamplingRate) + .setMaxMemoryInMB(maxMemoryInMB) + .setCacheNodeIds(cacheNodeIds) + .setFeaturesCol(rFormula.getFeaturesCol) + if (seed != null && seed.length > 0) rfr.setSeed(seed.toLong) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, rfr)) + .fit(data) + + new GBTRegressorWrapper(pipeline, formula, features) + } + + override def read: MLReader[GBTRegressorWrapper] = new GBTRegressorWrapperReader + + override def load(path: String): GBTRegressorWrapper = super.load(path) + + class GBTRegressorWrapperWriter(instance: GBTRegressorWrapper) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("formula" -> instance.formula) ~ + ("features" -> instance.features.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + instance.pipeline.save(pipelinePath) + } + } + + class GBTRegressorWrapperReader extends MLReader[GBTRegressorWrapper] { + + override def load(path: String): GBTRegressorWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + val pipeline = PipelineModel.load(pipelinePath) + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val formula = (rMetadata \ "formula").extract[String] + val features = (rMetadata \ "features").extract[Array[String]] + + new GBTRegressorWrapper(pipeline, formula, features) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala new file mode 100644 index 000000000000..9a98a8b18b14 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.clustering.{GaussianMixture, GaussianMixtureModel} +import org.apache.spark.ml.feature.RFormula +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.util.{MLReadable, MLReader, MLWritable, MLWriter} +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.functions._ + +private[r] class GaussianMixtureWrapper private ( + val pipeline: PipelineModel, + val dim: Int, + val logLikelihood: Double, + val isLoaded: Boolean = false) extends MLWritable { + + private val gmm: GaussianMixtureModel = pipeline.stages(1).asInstanceOf[GaussianMixtureModel] + + lazy val k: Int = gmm.getK + + lazy val lambda: Array[Double] = gmm.weights + + lazy val mu: Array[Double] = gmm.gaussians.flatMap(_.mean.toArray) + + lazy val sigma: Array[Double] = gmm.gaussians.flatMap(_.cov.toArray) + + lazy val vectorToArray = udf { probability: Vector => probability.toArray } + lazy val posterior: DataFrame = gmm.summary.probability + .withColumn("posterior", vectorToArray(col(gmm.summary.probabilityCol))) + .drop(gmm.summary.probabilityCol) + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset).drop(gmm.getFeaturesCol) + } + + override def write: MLWriter = new GaussianMixtureWrapper.GaussianMixtureWrapperWriter(this) + +} + +private[r] object GaussianMixtureWrapper extends MLReadable[GaussianMixtureWrapper] { + + def fit( + data: DataFrame, + formula: String, + k: Int, + maxIter: Int, + tol: Double): GaussianMixtureWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + .setFeaturesCol("features") + RWrapperUtils.checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // get feature names from output schema + val schema = rFormulaModel.transform(data).schema + val featureAttrs = AttributeGroup.fromStructField(schema(rFormulaModel.getFeaturesCol)) + .attributes.get + val features = featureAttrs.map(_.name.get) + val dim = features.length + + val gm = new GaussianMixture() + .setK(k) + .setMaxIter(maxIter) + .setTol(tol) + .setFeaturesCol(rFormula.getFeaturesCol) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, gm)) + .fit(data) + + val gmm: GaussianMixtureModel = pipeline.stages(1).asInstanceOf[GaussianMixtureModel] + val logLikelihood: Double = gmm.summary.logLikelihood + + new GaussianMixtureWrapper(pipeline, dim, logLikelihood) + } + + override def read: MLReader[GaussianMixtureWrapper] = new GaussianMixtureWrapperReader + + override def load(path: String): GaussianMixtureWrapper = super.load(path) + + class GaussianMixtureWrapperWriter(instance: GaussianMixtureWrapper) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("dim" -> instance.dim) ~ + ("logLikelihood" -> instance.logLikelihood) + val rMetadataJson: String = compact(render(rMetadata)) + + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + instance.pipeline.save(pipelinePath) + } + } + + class GaussianMixtureWrapperReader extends MLReader[GaussianMixtureWrapper] { + + override def load(path: String): GaussianMixtureWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + val pipeline = PipelineModel.load(pipelinePath) + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val dim = (rMetadata \ "dim").extract[Int] + val logLikelihood = (rMetadata \ "logLikelihood").extract[Double] + new GaussianMixtureWrapper(pipeline, dim, logLikelihood, isLoaded = true) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala new file mode 100644 index 000000000000..64575b0cb0cb --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import java.util.Locale + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.feature.RFormula +import org.apache.spark.ml.r.RWrapperUtils._ +import org.apache.spark.ml.regression._ +import org.apache.spark.ml.util._ +import org.apache.spark.sql._ + +private[r] class GeneralizedLinearRegressionWrapper private ( + val pipeline: PipelineModel, + val rFeatures: Array[String], + val rCoefficients: Array[Double], + val rDispersion: Double, + val rNullDeviance: Double, + val rDeviance: Double, + val rResidualDegreeOfFreedomNull: Long, + val rResidualDegreeOfFreedom: Long, + val rAic: Double, + val rNumIterations: Int, + val isLoaded: Boolean = false) extends MLWritable { + + private val glm: GeneralizedLinearRegressionModel = + pipeline.stages(1).asInstanceOf[GeneralizedLinearRegressionModel] + + lazy val rDevianceResiduals: DataFrame = glm.summary.residuals() + + lazy val rFamily: String = glm.getFamily + + def residuals(residualsType: String): DataFrame = glm.summary.residuals(residualsType) + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset).drop(glm.getFeaturesCol) + } + + override def write: MLWriter = + new GeneralizedLinearRegressionWrapper.GeneralizedLinearRegressionWrapperWriter(this) +} + +private[r] object GeneralizedLinearRegressionWrapper + extends MLReadable[GeneralizedLinearRegressionWrapper] { + + // scalastyle:off + def fit( + formula: String, + data: DataFrame, + family: String, + link: String, + tol: Double, + maxIter: Int, + weightCol: String, + regParam: Double, + variancePower: Double, + linkPower: Double, + stringIndexerOrderType: String, + offsetCol: String): GeneralizedLinearRegressionWrapper = { + // scalastyle:on + val rFormula = new RFormula().setFormula(formula) + .setStringIndexerOrderType(stringIndexerOrderType) + checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // assemble and fit the pipeline + val glr = new GeneralizedLinearRegression() + .setFamily(family) + .setFitIntercept(rFormula.hasIntercept) + .setTol(tol) + .setMaxIter(maxIter) + .setRegParam(regParam) + .setFeaturesCol(rFormula.getFeaturesCol) + // set variancePower and linkPower if family is tweedie; otherwise, set link function + if (family.toLowerCase(Locale.ROOT) == "tweedie") { + glr.setVariancePower(variancePower).setLinkPower(linkPower) + } else { + glr.setLink(link) + } + if (weightCol != null) glr.setWeightCol(weightCol) + if (offsetCol != null) glr.setOffsetCol(offsetCol) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, glr)) + .fit(data) + + val glm: GeneralizedLinearRegressionModel = + pipeline.stages(1).asInstanceOf[GeneralizedLinearRegressionModel] + val summary = glm.summary + + val rFeatures: Array[String] = if (glm.getFitIntercept) { + Array("(Intercept)") ++ summary.featureNames + } else { + summary.featureNames + } + + val rCoefficients: Array[Double] = if (summary.isNormalSolver) { + summary.coefficientsWithStatistics.map(_._2) ++ + summary.coefficientsWithStatistics.map(_._3) ++ + summary.coefficientsWithStatistics.map(_._4) ++ + summary.coefficientsWithStatistics.map(_._5) + } else { + if (glm.getFitIntercept) { + Array(glm.intercept) ++ glm.coefficients.toArray + } else { + glm.coefficients.toArray + } + } + + val rDispersion: Double = summary.dispersion + val rNullDeviance: Double = summary.nullDeviance + val rDeviance: Double = summary.deviance + val rResidualDegreeOfFreedomNull: Long = summary.residualDegreeOfFreedomNull + val rResidualDegreeOfFreedom: Long = summary.residualDegreeOfFreedom + val rAic: Double = if (family.toLowerCase(Locale.ROOT) == "tweedie" && + !Array(0.0, 1.0, 2.0).exists(x => math.abs(x - variancePower) < 1e-8)) { + 0.0 + } else { + summary.aic + } + val rNumIterations: Int = summary.numIterations + + new GeneralizedLinearRegressionWrapper(pipeline, rFeatures, rCoefficients, rDispersion, + rNullDeviance, rDeviance, rResidualDegreeOfFreedomNull, rResidualDegreeOfFreedom, + rAic, rNumIterations) + } + + override def read: MLReader[GeneralizedLinearRegressionWrapper] = + new GeneralizedLinearRegressionWrapperReader + + override def load(path: String): GeneralizedLinearRegressionWrapper = super.load(path) + + class GeneralizedLinearRegressionWrapperWriter(instance: GeneralizedLinearRegressionWrapper) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("rFeatures" -> instance.rFeatures.toSeq) ~ + ("rCoefficients" -> instance.rCoefficients.toSeq) ~ + ("rDispersion" -> instance.rDispersion) ~ + ("rNullDeviance" -> instance.rNullDeviance) ~ + ("rDeviance" -> instance.rDeviance) ~ + ("rResidualDegreeOfFreedomNull" -> instance.rResidualDegreeOfFreedomNull) ~ + ("rResidualDegreeOfFreedom" -> instance.rResidualDegreeOfFreedom) ~ + ("rAic" -> instance.rAic) ~ + ("rNumIterations" -> instance.rNumIterations) + val rMetadataJson: String = compact(render(rMetadata)) + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + + instance.pipeline.save(pipelinePath) + } + } + + class GeneralizedLinearRegressionWrapperReader + extends MLReader[GeneralizedLinearRegressionWrapper] { + + override def load(path: String): GeneralizedLinearRegressionWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val rFeatures = (rMetadata \ "rFeatures").extract[Array[String]] + val rCoefficients = (rMetadata \ "rCoefficients").extract[Array[Double]] + val rDispersion = (rMetadata \ "rDispersion").extract[Double] + val rNullDeviance = (rMetadata \ "rNullDeviance").extract[Double] + val rDeviance = (rMetadata \ "rDeviance").extract[Double] + val rResidualDegreeOfFreedomNull = (rMetadata \ "rResidualDegreeOfFreedomNull").extract[Long] + val rResidualDegreeOfFreedom = (rMetadata \ "rResidualDegreeOfFreedom").extract[Long] + val rAic = (rMetadata \ "rAic").extract[Double] + val rNumIterations = (rMetadata \ "rNumIterations").extract[Int] + + val pipeline = PipelineModel.load(pipelinePath) + + new GeneralizedLinearRegressionWrapper(pipeline, rFeatures, rCoefficients, rDispersion, + rNullDeviance, rDeviance, rResidualDegreeOfFreedomNull, rResidualDegreeOfFreedom, + rAic, rNumIterations, isLoaded = true) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala new file mode 100644 index 000000000000..d31ebb46afb9 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.feature.RFormula +import org.apache.spark.ml.regression.{IsotonicRegression, IsotonicRegressionModel} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class IsotonicRegressionWrapper private ( + val pipeline: PipelineModel, + val features: Array[String]) extends MLWritable { + + private val isotonicRegressionModel: IsotonicRegressionModel = + pipeline.stages(1).asInstanceOf[IsotonicRegressionModel] + + lazy val boundaries: Array[Double] = isotonicRegressionModel.boundaries.toArray + + lazy val predictions: Array[Double] = isotonicRegressionModel.predictions.toArray + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset).drop(isotonicRegressionModel.getFeaturesCol) + } + + override def write: MLWriter = new IsotonicRegressionWrapper.IsotonicRegressionWrapperWriter(this) +} + +private[r] object IsotonicRegressionWrapper + extends MLReadable[IsotonicRegressionWrapper] { + + def fit( + data: DataFrame, + formula: String, + isotonic: Boolean, + featureIndex: Int, + weightCol: String): IsotonicRegressionWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + .setFeaturesCol("features") + RWrapperUtils.checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // get feature names from output schema + val schema = rFormulaModel.transform(data).schema + val featureAttrs = AttributeGroup.fromStructField(schema(rFormulaModel.getFeaturesCol)) + .attributes.get + val features = featureAttrs.map(_.name.get) + require(features.size == 1) + + // assemble and fit the pipeline + val isotonicRegression = new IsotonicRegression() + .setIsotonic(isotonic) + .setFeatureIndex(featureIndex) + .setFeaturesCol(rFormula.getFeaturesCol) + + if (weightCol != null) isotonicRegression.setWeightCol(weightCol) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, isotonicRegression)) + .fit(data) + + new IsotonicRegressionWrapper(pipeline, features) + } + + override def read: MLReader[IsotonicRegressionWrapper] = new IsotonicRegressionWrapperReader + + override def load(path: String): IsotonicRegressionWrapper = super.load(path) + + class IsotonicRegressionWrapperWriter(instance: IsotonicRegressionWrapper) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("features" -> instance.features.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + + instance.pipeline.save(pipelinePath) + } + } + + class IsotonicRegressionWrapperReader extends MLReader[IsotonicRegressionWrapper] { + + override def load(path: String): IsotonicRegressionWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val features = (rMetadata \ "features").extract[Array[String]] + + val pipeline = PipelineModel.load(pipelinePath) + new IsotonicRegressionWrapper(pipeline, features) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala new file mode 100644 index 000000000000..8d596863b459 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.clustering.{KMeans, KMeansModel} +import org.apache.spark.ml.feature.RFormula +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class KMeansWrapper private ( + val pipeline: PipelineModel, + val features: Array[String], + val size: Array[Long], + val isLoaded: Boolean = false) extends MLWritable { + + private val kMeansModel: KMeansModel = pipeline.stages(1).asInstanceOf[KMeansModel] + + lazy val coefficients: Array[Double] = kMeansModel.clusterCenters.flatMap(_.toArray) + + lazy val k: Int = kMeansModel.getK + + lazy val cluster: DataFrame = kMeansModel.summary.cluster + + lazy val clusterSize: Int = kMeansModel.clusterCenters.size + + def fitted(method: String): DataFrame = { + if (method == "centers") { + kMeansModel.summary.predictions.drop(kMeansModel.getFeaturesCol) + } else if (method == "classes") { + kMeansModel.summary.cluster + } else { + throw new UnsupportedOperationException( + s"Method (centers or classes) required but $method found.") + } + } + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset).drop(kMeansModel.getFeaturesCol) + } + + override def write: MLWriter = new KMeansWrapper.KMeansWrapperWriter(this) +} + +private[r] object KMeansWrapper extends MLReadable[KMeansWrapper] { + + def fit( + data: DataFrame, + formula: String, + k: Int, + maxIter: Int, + initMode: String, + seed: String, + initSteps: Int, + tol: Double): KMeansWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + .setFeaturesCol("features") + RWrapperUtils.checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // get feature names from output schema + val schema = rFormulaModel.transform(data).schema + val featureAttrs = AttributeGroup.fromStructField(schema(rFormulaModel.getFeaturesCol)) + .attributes.get + val features = featureAttrs.map(_.name.get) + + val kMeans = new KMeans() + .setK(k) + .setMaxIter(maxIter) + .setInitMode(initMode) + .setFeaturesCol(rFormula.getFeaturesCol) + .setInitSteps(initSteps) + .setTol(tol) + + if (seed != null && seed.length > 0) kMeans.setSeed(seed.toInt) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, kMeans)) + .fit(data) + + val kMeansModel: KMeansModel = pipeline.stages(1).asInstanceOf[KMeansModel] + val size: Array[Long] = kMeansModel.summary.clusterSizes + + new KMeansWrapper(pipeline, features, size) + } + + override def read: MLReader[KMeansWrapper] = new KMeansWrapperReader + + override def load(path: String): KMeansWrapper = super.load(path) + + class KMeansWrapperWriter(instance: KMeansWrapper) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("features" -> instance.features.toSeq) ~ + ("size" -> instance.size.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + instance.pipeline.save(pipelinePath) + } + } + + class KMeansWrapperReader extends MLReader[KMeansWrapper] { + + override def load(path: String): KMeansWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + val pipeline = PipelineModel.load(pipelinePath) + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val features = (rMetadata \ "features").extract[Array[String]] + val size = (rMetadata \ "size").extract[Array[Long]] + new KMeansWrapper(pipeline, features, size, isLoaded = true) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/KSTestWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/KSTestWrapper.scala new file mode 100644 index 000000000000..21531eb057ad --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/KSTestWrapper.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.spark.mllib.stat.Statistics.kolmogorovSmirnovTest +import org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult +import org.apache.spark.sql.{DataFrame, Row} + +private[r] class KSTestWrapper private ( + val testResult: KolmogorovSmirnovTestResult, + val distName: String, + val distParams: Array[Double]) { + + lazy val pValue = testResult.pValue + + lazy val statistic = testResult.statistic + + lazy val nullHypothesis = testResult.nullHypothesis + + lazy val degreesOfFreedom = testResult.degreesOfFreedom + + def summary: String = testResult.toString +} + +private[r] object KSTestWrapper { + + def test( + data: DataFrame, + featureName: String, + distName: String, + distParams: Array[Double]): KSTestWrapper = { + + val rddData = data.select(featureName).rdd.map { + case Row(feature: Double) => feature + } + + val ksTestResult = kolmogorovSmirnovTest(rddData, distName, distParams : _*) + + new KSTestWrapper(ksTestResult, distName, distParams) + } +} + diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/LDAWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/LDAWrapper.scala new file mode 100644 index 000000000000..e096bf1f29f3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/LDAWrapper.scala @@ -0,0 +1,224 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import scala.collection.mutable + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.SparkException +import org.apache.spark.ml.{Pipeline, PipelineModel, PipelineStage} +import org.apache.spark.ml.clustering.{DistributedLDAModel, LDA, LDAModel} +import org.apache.spark.ml.feature.{CountVectorizer, CountVectorizerModel, RegexTokenizer, StopWordsRemover} +import org.apache.spark.ml.linalg.{Vector, VectorUDT} +import org.apache.spark.ml.param.ParamPair +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.StringType + + +private[r] class LDAWrapper private ( + val pipeline: PipelineModel, + val logLikelihood: Double, + val logPerplexity: Double, + val vocabulary: Array[String]) extends MLWritable { + + import LDAWrapper._ + + private val lda: LDAModel = pipeline.stages.last.asInstanceOf[LDAModel] + + // The following variables were called by R side code only when the LDA model is distributed + lazy private val distributedModel = + pipeline.stages.last.asInstanceOf[DistributedLDAModel] + lazy val trainingLogLikelihood: Double = distributedModel.trainingLogLikelihood + lazy val logPrior: Double = distributedModel.logPrior + + private val preprocessor: PipelineModel = + new PipelineModel(s"${Identifiable.randomUID(pipeline.uid)}", pipeline.stages.dropRight(1)) + + def transform(data: Dataset[_]): DataFrame = { + val vec2ary = udf { vec: Vector => vec.toArray } + val outputCol = lda.getTopicDistributionCol + val tempCol = s"${Identifiable.randomUID(outputCol)}" + val preprocessed = preprocessor.transform(data) + lda.transform(preprocessed, ParamPair(lda.topicDistributionCol, tempCol)) + .withColumn(outputCol, vec2ary(col(tempCol))) + .drop(TOKENIZER_COL, STOPWORDS_REMOVER_COL, COUNT_VECTOR_COL, tempCol) + } + + def computeLogPerplexity(data: Dataset[_]): Double = { + lda.logPerplexity(preprocessor.transform(data)) + } + + def topics(maxTermsPerTopic: Int): DataFrame = { + val topicIndices: DataFrame = lda.describeTopics(maxTermsPerTopic) + if (vocabulary.isEmpty || vocabulary.length < vocabSize) { + topicIndices + } else { + val index2term = udf { indices: mutable.WrappedArray[Int] => indices.map(i => vocabulary(i)) } + topicIndices + .select(col("topic"), index2term(col("termIndices")).as("term"), col("termWeights")) + } + } + + lazy val isDistributed: Boolean = lda.isDistributed + lazy val vocabSize: Int = lda.vocabSize + lazy val docConcentration: Array[Double] = lda.getEffectiveDocConcentration + lazy val topicConcentration: Double = lda.getEffectiveTopicConcentration + + override def write: MLWriter = new LDAWrapper.LDAWrapperWriter(this) +} + +private[r] object LDAWrapper extends MLReadable[LDAWrapper] { + + val TOKENIZER_COL = s"${Identifiable.randomUID("rawTokens")}" + val STOPWORDS_REMOVER_COL = s"${Identifiable.randomUID("tokens")}" + val COUNT_VECTOR_COL = s"${Identifiable.randomUID("features")}" + + private def getPreStages( + features: String, + customizedStopWords: Array[String], + maxVocabSize: Int): Array[PipelineStage] = { + val tokenizer = new RegexTokenizer() + .setInputCol(features) + .setOutputCol(TOKENIZER_COL) + val stopWordsRemover = new StopWordsRemover() + .setInputCol(TOKENIZER_COL) + .setOutputCol(STOPWORDS_REMOVER_COL) + stopWordsRemover.setStopWords(stopWordsRemover.getStopWords ++ customizedStopWords) + val countVectorizer = new CountVectorizer() + .setVocabSize(maxVocabSize) + .setInputCol(STOPWORDS_REMOVER_COL) + .setOutputCol(COUNT_VECTOR_COL) + + Array(tokenizer, stopWordsRemover, countVectorizer) + } + + def fit( + data: DataFrame, + features: String, + k: Int, + maxIter: Int, + optimizer: String, + subsamplingRate: Double, + topicConcentration: Double, + docConcentration: Array[Double], + customizedStopWords: Array[String], + maxVocabSize: Int): LDAWrapper = { + + val lda = new LDA() + .setK(k) + .setMaxIter(maxIter) + .setSubsamplingRate(subsamplingRate) + .setOptimizer(optimizer) + + val featureSchema = data.schema(features) + val stages = featureSchema.dataType match { + case d: StringType => + getPreStages(features, customizedStopWords, maxVocabSize) ++ + Array(lda.setFeaturesCol(COUNT_VECTOR_COL)) + case d: VectorUDT => + Array(lda.setFeaturesCol(features)) + case _ => + throw new SparkException( + s"Unsupported input features type of ${featureSchema.dataType.typeName}," + + s" only String type and Vector type are supported now.") + } + + if (topicConcentration != -1) { + lda.setTopicConcentration(topicConcentration) + } else { + // Auto-set topicConcentration + } + + if (docConcentration.length == 1) { + if (docConcentration.head != -1) { + lda.setDocConcentration(docConcentration.head) + } else { + // Auto-set docConcentration + } + } else { + lda.setDocConcentration(docConcentration) + } + + val pipeline = new Pipeline().setStages(stages) + val model = pipeline.fit(data) + + val vocabulary: Array[String] = featureSchema.dataType match { + case d: StringType => + val countVectorModel = model.stages(2).asInstanceOf[CountVectorizerModel] + countVectorModel.vocabulary + case _ => Array.empty[String] + } + + val ldaModel: LDAModel = model.stages.last.asInstanceOf[LDAModel] + val preprocessor: PipelineModel = + new PipelineModel(s"${Identifiable.randomUID(pipeline.uid)}", model.stages.dropRight(1)) + + val preprocessedData = preprocessor.transform(data) + + new LDAWrapper( + model, + ldaModel.logLikelihood(preprocessedData), + ldaModel.logPerplexity(preprocessedData), + vocabulary) + } + + override def read: MLReader[LDAWrapper] = new LDAWrapperReader + + override def load(path: String): LDAWrapper = super.load(path) + + class LDAWrapperWriter(instance: LDAWrapper) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("logLikelihood" -> instance.logLikelihood) ~ + ("logPerplexity" -> instance.logPerplexity) ~ + ("vocabulary" -> instance.vocabulary.toList) + val rMetadataJson: String = compact(render(rMetadata)) + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + + instance.pipeline.save(pipelinePath) + } + } + + class LDAWrapperReader extends MLReader[LDAWrapper] { + + override def load(path: String): LDAWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val logLikelihood = (rMetadata \ "logLikelihood").extract[Double] + val logPerplexity = (rMetadata \ "logPerplexity").extract[Double] + val vocabulary = (rMetadata \ "vocabulary").extract[List[String]].toArray + + val pipeline = PipelineModel.load(pipelinePath) + new LDAWrapper(pipeline, logLikelihood, logPerplexity, vocabulary) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/LinearSVCWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/LinearSVCWrapper.scala new file mode 100644 index 000000000000..7a22a71c3a81 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/LinearSVCWrapper.scala @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.classification.{LinearSVC, LinearSVCModel} +import org.apache.spark.ml.feature.{IndexToString, RFormula} +import org.apache.spark.ml.r.RWrapperUtils._ +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class LinearSVCWrapper private ( + val pipeline: PipelineModel, + val features: Array[String], + val labels: Array[String]) extends MLWritable { + import LinearSVCWrapper._ + + private val svcModel: LinearSVCModel = + pipeline.stages(1).asInstanceOf[LinearSVCModel] + + lazy val rFeatures: Array[String] = if (svcModel.getFitIntercept) { + Array("(Intercept)") ++ features + } else { + features + } + + lazy val rCoefficients: Array[Double] = if (svcModel.getFitIntercept) { + Array(svcModel.intercept) ++ svcModel.coefficients.toArray + } else { + svcModel.coefficients.toArray + } + + lazy val numClasses: Int = svcModel.numClasses + + lazy val numFeatures: Int = svcModel.numFeatures + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset) + .drop(PREDICTED_LABEL_INDEX_COL) + .drop(svcModel.getFeaturesCol) + .drop(svcModel.getLabelCol) + } + + override def write: MLWriter = new LinearSVCWrapper.LinearSVCWrapperWriter(this) +} + +private[r] object LinearSVCWrapper + extends MLReadable[LinearSVCWrapper] { + + val PREDICTED_LABEL_INDEX_COL = "pred_label_idx" + val PREDICTED_LABEL_COL = "prediction" + + def fit( + data: DataFrame, + formula: String, + regParam: Double, + maxIter: Int, + tol: Double, + standardization: Boolean, + threshold: Double, + weightCol: String, + aggregationDepth: Int, + handleInvalid: String + ): LinearSVCWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + .setForceIndexLabel(true) + .setHandleInvalid(handleInvalid) + checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + val fitIntercept = rFormula.hasIntercept + + // get labels and feature names from output schema + val (features, labels) = getFeaturesAndLabels(rFormulaModel, data) + + // assemble and fit the pipeline + val svc = new LinearSVC() + .setRegParam(regParam) + .setMaxIter(maxIter) + .setTol(tol) + .setFitIntercept(fitIntercept) + .setStandardization(standardization) + .setFeaturesCol(rFormula.getFeaturesCol) + .setLabelCol(rFormula.getLabelCol) + .setPredictionCol(PREDICTED_LABEL_INDEX_COL) + .setThreshold(threshold) + .setAggregationDepth(aggregationDepth) + + if (weightCol != null) svc.setWeightCol(weightCol) + + val idxToStr = new IndexToString() + .setInputCol(PREDICTED_LABEL_INDEX_COL) + .setOutputCol(PREDICTED_LABEL_COL) + .setLabels(labels) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, svc, idxToStr)) + .fit(data) + + new LinearSVCWrapper(pipeline, features, labels) + } + + override def read: MLReader[LinearSVCWrapper] = new LinearSVCWrapperReader + + override def load(path: String): LinearSVCWrapper = super.load(path) + + class LinearSVCWrapperWriter(instance: LinearSVCWrapper) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("features" -> instance.features.toSeq) ~ + ("labels" -> instance.labels.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + + instance.pipeline.save(pipelinePath) + } + } + + class LinearSVCWrapperReader extends MLReader[LinearSVCWrapper] { + + override def load(path: String): LinearSVCWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val features = (rMetadata \ "features").extract[Array[String]] + val labels = (rMetadata \ "labels").extract[Array[String]] + + val pipeline = PipelineModel.load(pipelinePath) + new LinearSVCWrapper(pipeline, features, labels) + } + } +} + diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/LogisticRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/LogisticRegressionWrapper.scala new file mode 100644 index 000000000000..18acf7d21656 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/LogisticRegressionWrapper.scala @@ -0,0 +1,216 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.classification.{LogisticRegression, LogisticRegressionModel} +import org.apache.spark.ml.feature.{IndexToString, RFormula} +import org.apache.spark.ml.linalg.{Matrices, Vector, Vectors} +import org.apache.spark.ml.r.RWrapperUtils._ +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class LogisticRegressionWrapper private ( + val pipeline: PipelineModel, + val features: Array[String], + val labels: Array[String]) extends MLWritable { + + import LogisticRegressionWrapper._ + + private val lrModel: LogisticRegressionModel = + pipeline.stages(1).asInstanceOf[LogisticRegressionModel] + + lazy val rFeatures: Array[String] = if (lrModel.getFitIntercept) { + Array("(Intercept)") ++ features + } else { + features + } + + lazy val rCoefficients: Array[Double] = { + val numRows = lrModel.coefficientMatrix.numRows + val numCols = lrModel.coefficientMatrix.numCols + val numColsWithIntercept = if (lrModel.getFitIntercept) numCols + 1 else numCols + val coefficients: Array[Double] = new Array[Double](numRows * numColsWithIntercept) + val coefficientVectors: Seq[Vector] = lrModel.coefficientMatrix.rowIter.toSeq + var i = 0 + if (lrModel.getFitIntercept) { + while (i < numRows) { + coefficients(i * numColsWithIntercept) = lrModel.interceptVector(i) + System.arraycopy(coefficientVectors(i).toArray, 0, + coefficients, i * numColsWithIntercept + 1, numCols) + i += 1 + } + } else { + while (i < numRows) { + System.arraycopy(coefficientVectors(i).toArray, 0, + coefficients, i * numColsWithIntercept, numCols) + i += 1 + } + } + coefficients + } + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset) + .drop(PREDICTED_LABEL_INDEX_COL) + .drop(lrModel.getFeaturesCol) + .drop(lrModel.getLabelCol) + } + + override def write: MLWriter = new LogisticRegressionWrapper.LogisticRegressionWrapperWriter(this) +} + +private[r] object LogisticRegressionWrapper + extends MLReadable[LogisticRegressionWrapper] { + + val PREDICTED_LABEL_INDEX_COL = "pred_label_idx" + val PREDICTED_LABEL_COL = "prediction" + + def fit( // scalastyle:ignore + data: DataFrame, + formula: String, + regParam: Double, + elasticNetParam: Double, + maxIter: Int, + tol: Double, + family: String, + standardization: Boolean, + thresholds: Array[Double], + weightCol: String, + aggregationDepth: Int, + numRowsOfBoundsOnCoefficients: Int, + numColsOfBoundsOnCoefficients: Int, + lowerBoundsOnCoefficients: Array[Double], + upperBoundsOnCoefficients: Array[Double], + lowerBoundsOnIntercepts: Array[Double], + upperBoundsOnIntercepts: Array[Double], + handleInvalid: String + ): LogisticRegressionWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + .setForceIndexLabel(true) + .setHandleInvalid(handleInvalid) + checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + val fitIntercept = rFormula.hasIntercept + + // get labels and feature names from output schema + val (features, labels) = getFeaturesAndLabels(rFormulaModel, data) + + // assemble and fit the pipeline + val lr = new LogisticRegression() + .setRegParam(regParam) + .setElasticNetParam(elasticNetParam) + .setMaxIter(maxIter) + .setTol(tol) + .setFitIntercept(fitIntercept) + .setFamily(family) + .setStandardization(standardization) + .setFeaturesCol(rFormula.getFeaturesCol) + .setLabelCol(rFormula.getLabelCol) + .setPredictionCol(PREDICTED_LABEL_INDEX_COL) + .setAggregationDepth(aggregationDepth) + + if (thresholds.length > 1) { + lr.setThresholds(thresholds) + } else { + lr.setThreshold(thresholds(0)) + } + + if (weightCol != null) lr.setWeightCol(weightCol) + + if (numRowsOfBoundsOnCoefficients != 0 && + numColsOfBoundsOnCoefficients != 0 && lowerBoundsOnCoefficients != null) { + val coef = Matrices.dense(numRowsOfBoundsOnCoefficients, + numColsOfBoundsOnCoefficients, lowerBoundsOnCoefficients) + lr.setLowerBoundsOnCoefficients(coef) + } + + if (numRowsOfBoundsOnCoefficients != 0 && + numColsOfBoundsOnCoefficients != 0 && upperBoundsOnCoefficients != null) { + val coef = Matrices.dense(numRowsOfBoundsOnCoefficients, + numColsOfBoundsOnCoefficients, upperBoundsOnCoefficients) + lr.setUpperBoundsOnCoefficients(coef) + } + + if (lowerBoundsOnIntercepts != null) { + val intercept = Vectors.dense(lowerBoundsOnIntercepts) + lr.setLowerBoundsOnIntercepts(intercept) + } + + if (upperBoundsOnIntercepts != null) { + val intercept = Vectors.dense(upperBoundsOnIntercepts) + lr.setUpperBoundsOnIntercepts(intercept) + } + + val idxToStr = new IndexToString() + .setInputCol(PREDICTED_LABEL_INDEX_COL) + .setOutputCol(PREDICTED_LABEL_COL) + .setLabels(labels) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, lr, idxToStr)) + .fit(data) + + new LogisticRegressionWrapper(pipeline, features, labels) + } + + override def read: MLReader[LogisticRegressionWrapper] = new LogisticRegressionWrapperReader + + override def load(path: String): LogisticRegressionWrapper = super.load(path) + + class LogisticRegressionWrapperWriter(instance: LogisticRegressionWrapper) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("features" -> instance.features.toSeq) ~ + ("labels" -> instance.labels.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + + instance.pipeline.save(pipelinePath) + } + } + + class LogisticRegressionWrapperReader extends MLReader[LogisticRegressionWrapper] { + + override def load(path: String): LogisticRegressionWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val features = (rMetadata \ "features").extract[Array[String]] + val labels = (rMetadata \ "labels").extract[Array[String]] + + val pipeline = PipelineModel.load(pipelinePath) + new LogisticRegressionWrapper(pipeline, features, labels) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/MultilayerPerceptronClassifierWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/MultilayerPerceptronClassifierWrapper.scala new file mode 100644 index 000000000000..62f642142701 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/MultilayerPerceptronClassifierWrapper.scala @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.classification.{MultilayerPerceptronClassificationModel, MultilayerPerceptronClassifier} +import org.apache.spark.ml.feature.{IndexToString, RFormula} +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.r.RWrapperUtils._ +import org.apache.spark.ml.util.{MLReadable, MLReader, MLWritable, MLWriter} +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class MultilayerPerceptronClassifierWrapper private ( + val pipeline: PipelineModel + ) extends MLWritable { + + import MultilayerPerceptronClassifierWrapper._ + + private val mlpModel: MultilayerPerceptronClassificationModel = + pipeline.stages(1).asInstanceOf[MultilayerPerceptronClassificationModel] + + lazy val weights: Array[Double] = mlpModel.weights.toArray + lazy val layers: Array[Int] = mlpModel.layers + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset) + .drop(mlpModel.getFeaturesCol) + .drop(mlpModel.getLabelCol) + .drop(PREDICTED_LABEL_INDEX_COL) + } + + /** + * Returns an [[MLWriter]] instance for this ML instance. + */ + override def write: MLWriter = + new MultilayerPerceptronClassifierWrapper.MultilayerPerceptronClassifierWrapperWriter(this) +} + +private[r] object MultilayerPerceptronClassifierWrapper + extends MLReadable[MultilayerPerceptronClassifierWrapper] { + + val PREDICTED_LABEL_INDEX_COL = "pred_label_idx" + val PREDICTED_LABEL_COL = "prediction" + + def fit( // scalastyle:ignore + data: DataFrame, + formula: String, + blockSize: Int, + layers: Array[Int], + solver: String, + maxIter: Int, + tol: Double, + stepSize: Double, + seed: String, + initialWeights: Array[Double], + handleInvalid: String + ): MultilayerPerceptronClassifierWrapper = { + val rFormula = new RFormula() + .setFormula(formula) + .setForceIndexLabel(true) + .setHandleInvalid(handleInvalid) + checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + // get labels and feature names from output schema + val (_, labels) = getFeaturesAndLabels(rFormulaModel, data) + + // assemble and fit the pipeline + val mlp = new MultilayerPerceptronClassifier() + .setLayers(layers) + .setBlockSize(blockSize) + .setSolver(solver) + .setMaxIter(maxIter) + .setTol(tol) + .setStepSize(stepSize) + .setFeaturesCol(rFormula.getFeaturesCol) + .setLabelCol(rFormula.getLabelCol) + .setPredictionCol(PREDICTED_LABEL_INDEX_COL) + if (seed != null && seed.length > 0) mlp.setSeed(seed.toInt) + if (initialWeights != null) { + require(initialWeights.length > 0) + mlp.setInitialWeights(Vectors.dense(initialWeights)) + } + + val idxToStr = new IndexToString() + .setInputCol(PREDICTED_LABEL_INDEX_COL) + .setOutputCol(PREDICTED_LABEL_COL) + .setLabels(labels) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, mlp, idxToStr)) + .fit(data) + + new MultilayerPerceptronClassifierWrapper(pipeline) + } + + /** + * Returns an [[MLReader]] instance for this class. + */ + override def read: MLReader[MultilayerPerceptronClassifierWrapper] = + new MultilayerPerceptronClassifierWrapperReader + + override def load(path: String): MultilayerPerceptronClassifierWrapper = super.load(path) + + class MultilayerPerceptronClassifierWrapperReader + extends MLReader[MultilayerPerceptronClassifierWrapper]{ + + override def load(path: String): MultilayerPerceptronClassifierWrapper = { + implicit val format = DefaultFormats + val pipelinePath = new Path(path, "pipeline").toString + + val pipeline = PipelineModel.load(pipelinePath) + new MultilayerPerceptronClassifierWrapper(pipeline) + } + } + + class MultilayerPerceptronClassifierWrapperWriter(instance: MultilayerPerceptronClassifierWrapper) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = "class" -> instance.getClass.getName + val rMetadataJson: String = compact(render(rMetadata)) + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + + instance.pipeline.save(pipelinePath) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala new file mode 100644 index 000000000000..fbf9f462ff5f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.classification.{NaiveBayes, NaiveBayesModel} +import org.apache.spark.ml.feature.{IndexToString, RFormula} +import org.apache.spark.ml.r.RWrapperUtils._ +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class NaiveBayesWrapper private ( + val pipeline: PipelineModel, + val labels: Array[String], + val features: Array[String]) extends MLWritable { + + import NaiveBayesWrapper._ + + private val naiveBayesModel: NaiveBayesModel = pipeline.stages(1).asInstanceOf[NaiveBayesModel] + + lazy val apriori: Array[Double] = naiveBayesModel.pi.toArray.map(math.exp) + + lazy val tables: Array[Double] = naiveBayesModel.theta.toArray.map(math.exp) + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset) + .drop(PREDICTED_LABEL_INDEX_COL) + .drop(naiveBayesModel.getFeaturesCol) + .drop(naiveBayesModel.getLabelCol) + } + + override def write: MLWriter = new NaiveBayesWrapper.NaiveBayesWrapperWriter(this) +} + +private[r] object NaiveBayesWrapper extends MLReadable[NaiveBayesWrapper] { + + val PREDICTED_LABEL_INDEX_COL = "pred_label_idx" + val PREDICTED_LABEL_COL = "prediction" + + def fit( + formula: String, + data: DataFrame, + smoothing: Double, + handleInvalid: String): NaiveBayesWrapper = { + val rFormula = new RFormula() + .setFormula(formula) + .setForceIndexLabel(true) + .setHandleInvalid(handleInvalid) + checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + // get labels and feature names from output schema + val (features, labels) = getFeaturesAndLabels(rFormulaModel, data) + // assemble and fit the pipeline + val naiveBayes = new NaiveBayes() + .setSmoothing(smoothing) + .setModelType("bernoulli") + .setFeaturesCol(rFormula.getFeaturesCol) + .setLabelCol(rFormula.getLabelCol) + .setPredictionCol(PREDICTED_LABEL_INDEX_COL) + val idxToStr = new IndexToString() + .setInputCol(PREDICTED_LABEL_INDEX_COL) + .setOutputCol(PREDICTED_LABEL_COL) + .setLabels(labels) + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, naiveBayes, idxToStr)) + .fit(data) + new NaiveBayesWrapper(pipeline, labels, features) + } + + override def read: MLReader[NaiveBayesWrapper] = new NaiveBayesWrapperReader + + override def load(path: String): NaiveBayesWrapper = super.load(path) + + class NaiveBayesWrapperWriter(instance: NaiveBayesWrapper) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("labels" -> instance.labels.toSeq) ~ + ("features" -> instance.features.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + + instance.pipeline.save(pipelinePath) + } + } + + class NaiveBayesWrapperReader extends MLReader[NaiveBayesWrapper] { + + override def load(path: String): NaiveBayesWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val labels = (rMetadata \ "labels").extract[Array[String]] + val features = (rMetadata \ "features").extract[Array[String]] + + val pipeline = PipelineModel.load(pipelinePath) + new NaiveBayesWrapper(pipeline, labels, features) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RWrapperUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RWrapperUtils.scala new file mode 100644 index 000000000000..665e50af67d4 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RWrapperUtils.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.spark.internal.Logging +import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NominalAttribute} +import org.apache.spark.ml.feature.{RFormula, RFormulaModel} +import org.apache.spark.ml.util.Identifiable +import org.apache.spark.sql.Dataset + +private[r] object RWrapperUtils extends Logging { + + /** + * DataFrame column check. + * When loading libsvm data, default columns "features" and "label" will be added. + * And "features" would conflict with RFormula default feature column names. + * Here is to change the column name to avoid "column already exists" error. + * + * @param rFormula RFormula instance + * @param data Input dataset + */ + def checkDataColumns(rFormula: RFormula, data: Dataset[_]): Unit = { + if (data.schema.fieldNames.contains(rFormula.getFeaturesCol)) { + val newFeaturesName = s"${Identifiable.randomUID(rFormula.getFeaturesCol)}" + logInfo(s"data containing ${rFormula.getFeaturesCol} column, " + + s"using new name $newFeaturesName instead") + rFormula.setFeaturesCol(newFeaturesName) + } + + if (rFormula.getForceIndexLabel && data.schema.fieldNames.contains(rFormula.getLabelCol)) { + val newLabelName = s"${Identifiable.randomUID(rFormula.getLabelCol)}" + logInfo(s"data containing ${rFormula.getLabelCol} column and we force to index label, " + + s"using new name $newLabelName instead") + rFormula.setLabelCol(newLabelName) + } + } + + /** + * Get the feature names and original labels from the schema + * of DataFrame transformed by RFormulaModel. + * + * @param rFormulaModel The RFormulaModel instance. + * @param data Input dataset. + * @return The feature names and original labels. + */ + def getFeaturesAndLabels( + rFormulaModel: RFormulaModel, + data: Dataset[_]): (Array[String], Array[String]) = { + val schema = rFormulaModel.transform(data).schema + val featureAttrs = AttributeGroup.fromStructField(schema(rFormulaModel.getFeaturesCol)) + .attributes.get + val features = featureAttrs.map(_.name.get) + val labelAttr = Attribute.fromStructField(schema(rFormulaModel.getLabelCol)) + .asInstanceOf[NominalAttribute] + val labels = labelAttr.values.get + (features, labels) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala new file mode 100644 index 000000000000..ba6445a73030 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s.DefaultFormats +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.SparkException +import org.apache.spark.ml.util.MLReader + +/** + * This is the Scala stub of SparkR read.ml. It will dispatch the call to corresponding + * model wrapper loading function according the class name extracted from rMetadata of the path. + */ +private[r] object RWrappers extends MLReader[Object] { + + override def load(path: String): Object = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val className = (rMetadata \ "class").extract[String] + className match { + case "org.apache.spark.ml.r.NaiveBayesWrapper" => NaiveBayesWrapper.load(path) + case "org.apache.spark.ml.r.AFTSurvivalRegressionWrapper" => + AFTSurvivalRegressionWrapper.load(path) + case "org.apache.spark.ml.r.GeneralizedLinearRegressionWrapper" => + GeneralizedLinearRegressionWrapper.load(path) + case "org.apache.spark.ml.r.KMeansWrapper" => + KMeansWrapper.load(path) + case "org.apache.spark.ml.r.MultilayerPerceptronClassifierWrapper" => + MultilayerPerceptronClassifierWrapper.load(path) + case "org.apache.spark.ml.r.LDAWrapper" => + LDAWrapper.load(path) + case "org.apache.spark.ml.r.IsotonicRegressionWrapper" => + IsotonicRegressionWrapper.load(path) + case "org.apache.spark.ml.r.GaussianMixtureWrapper" => + GaussianMixtureWrapper.load(path) + case "org.apache.spark.ml.r.ALSWrapper" => + ALSWrapper.load(path) + case "org.apache.spark.ml.r.LogisticRegressionWrapper" => + LogisticRegressionWrapper.load(path) + case "org.apache.spark.ml.r.RandomForestRegressorWrapper" => + RandomForestRegressorWrapper.load(path) + case "org.apache.spark.ml.r.RandomForestClassifierWrapper" => + RandomForestClassifierWrapper.load(path) + case "org.apache.spark.ml.r.DecisionTreeRegressorWrapper" => + DecisionTreeRegressorWrapper.load(path) + case "org.apache.spark.ml.r.DecisionTreeClassifierWrapper" => + DecisionTreeClassifierWrapper.load(path) + case "org.apache.spark.ml.r.GBTRegressorWrapper" => + GBTRegressorWrapper.load(path) + case "org.apache.spark.ml.r.GBTClassifierWrapper" => + GBTClassifierWrapper.load(path) + case "org.apache.spark.ml.r.BisectingKMeansWrapper" => + BisectingKMeansWrapper.load(path) + case "org.apache.spark.ml.r.LinearSVCWrapper" => + LinearSVCWrapper.load(path) + case "org.apache.spark.ml.r.FPGrowthWrapper" => + FPGrowthWrapper.load(path) + case _ => + throw new SparkException(s"SparkR read.ml does not support load $className") + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassificationWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassificationWrapper.scala new file mode 100644 index 000000000000..132345fb9a6d --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassificationWrapper.scala @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.classification.{RandomForestClassificationModel, RandomForestClassifier} +import org.apache.spark.ml.feature.{IndexToString, RFormula} +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.r.RWrapperUtils._ +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class RandomForestClassifierWrapper private ( + val pipeline: PipelineModel, + val formula: String, + val features: Array[String]) extends MLWritable { + + import RandomForestClassifierWrapper._ + + private val rfcModel: RandomForestClassificationModel = + pipeline.stages(1).asInstanceOf[RandomForestClassificationModel] + + lazy val numFeatures: Int = rfcModel.numFeatures + lazy val featureImportances: Vector = rfcModel.featureImportances + lazy val numTrees: Int = rfcModel.getNumTrees + lazy val treeWeights: Array[Double] = rfcModel.treeWeights + lazy val maxDepth: Int = rfcModel.getMaxDepth + + def summary: String = rfcModel.toDebugString + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset) + .drop(PREDICTED_LABEL_INDEX_COL) + .drop(rfcModel.getFeaturesCol) + .drop(rfcModel.getLabelCol) + } + + override def write: MLWriter = new + RandomForestClassifierWrapper.RandomForestClassifierWrapperWriter(this) +} + +private[r] object RandomForestClassifierWrapper extends MLReadable[RandomForestClassifierWrapper] { + + val PREDICTED_LABEL_INDEX_COL = "pred_label_idx" + val PREDICTED_LABEL_COL = "prediction" + + def fit( // scalastyle:ignore + data: DataFrame, + formula: String, + maxDepth: Int, + maxBins: Int, + numTrees: Int, + impurity: String, + minInstancesPerNode: Int, + minInfoGain: Double, + checkpointInterval: Int, + featureSubsetStrategy: String, + seed: String, + subsamplingRate: Double, + maxMemoryInMB: Int, + cacheNodeIds: Boolean, + handleInvalid: String): RandomForestClassifierWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + .setForceIndexLabel(true) + .setHandleInvalid(handleInvalid) + checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // get labels and feature names from output schema + val (features, labels) = getFeaturesAndLabels(rFormulaModel, data) + + // assemble and fit the pipeline + val rfc = new RandomForestClassifier() + .setMaxDepth(maxDepth) + .setMaxBins(maxBins) + .setNumTrees(numTrees) + .setImpurity(impurity) + .setMinInstancesPerNode(minInstancesPerNode) + .setMinInfoGain(minInfoGain) + .setCheckpointInterval(checkpointInterval) + .setFeatureSubsetStrategy(featureSubsetStrategy) + .setSubsamplingRate(subsamplingRate) + .setMaxMemoryInMB(maxMemoryInMB) + .setCacheNodeIds(cacheNodeIds) + .setFeaturesCol(rFormula.getFeaturesCol) + .setLabelCol(rFormula.getLabelCol) + .setPredictionCol(PREDICTED_LABEL_INDEX_COL) + if (seed != null && seed.length > 0) rfc.setSeed(seed.toLong) + + val idxToStr = new IndexToString() + .setInputCol(PREDICTED_LABEL_INDEX_COL) + .setOutputCol(PREDICTED_LABEL_COL) + .setLabels(labels) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, rfc, idxToStr)) + .fit(data) + + new RandomForestClassifierWrapper(pipeline, formula, features) + } + + override def read: MLReader[RandomForestClassifierWrapper] = + new RandomForestClassifierWrapperReader + + override def load(path: String): RandomForestClassifierWrapper = super.load(path) + + class RandomForestClassifierWrapperWriter(instance: RandomForestClassifierWrapper) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("formula" -> instance.formula) ~ + ("features" -> instance.features.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + instance.pipeline.save(pipelinePath) + } + } + + class RandomForestClassifierWrapperReader extends MLReader[RandomForestClassifierWrapper] { + + override def load(path: String): RandomForestClassifierWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + val pipeline = PipelineModel.load(pipelinePath) + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val formula = (rMetadata \ "formula").extract[String] + val features = (rMetadata \ "features").extract[Array[String]] + + new RandomForestClassifierWrapper(pipeline, formula, features) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressionWrapper.scala new file mode 100644 index 000000000000..038bd79c7022 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressionWrapper.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.feature.RFormula +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.regression.{RandomForestRegressionModel, RandomForestRegressor} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class RandomForestRegressorWrapper private ( + val pipeline: PipelineModel, + val formula: String, + val features: Array[String]) extends MLWritable { + + private val rfrModel: RandomForestRegressionModel = + pipeline.stages(1).asInstanceOf[RandomForestRegressionModel] + + lazy val numFeatures: Int = rfrModel.numFeatures + lazy val featureImportances: Vector = rfrModel.featureImportances + lazy val numTrees: Int = rfrModel.getNumTrees + lazy val treeWeights: Array[Double] = rfrModel.treeWeights + lazy val maxDepth: Int = rfrModel.getMaxDepth + + def summary: String = rfrModel.toDebugString + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset).drop(rfrModel.getFeaturesCol) + } + + override def write: MLWriter = new + RandomForestRegressorWrapper.RandomForestRegressorWrapperWriter(this) +} + +private[r] object RandomForestRegressorWrapper extends MLReadable[RandomForestRegressorWrapper] { + def fit( // scalastyle:ignore + data: DataFrame, + formula: String, + maxDepth: Int, + maxBins: Int, + numTrees: Int, + impurity: String, + minInstancesPerNode: Int, + minInfoGain: Double, + checkpointInterval: Int, + featureSubsetStrategy: String, + seed: String, + subsamplingRate: Double, + maxMemoryInMB: Int, + cacheNodeIds: Boolean): RandomForestRegressorWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + RWrapperUtils.checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // get feature names from output schema + val schema = rFormulaModel.transform(data).schema + val featureAttrs = AttributeGroup.fromStructField(schema(rFormulaModel.getFeaturesCol)) + .attributes.get + val features = featureAttrs.map(_.name.get) + + // assemble and fit the pipeline + val rfr = new RandomForestRegressor() + .setMaxDepth(maxDepth) + .setMaxBins(maxBins) + .setNumTrees(numTrees) + .setImpurity(impurity) + .setMinInstancesPerNode(minInstancesPerNode) + .setMinInfoGain(minInfoGain) + .setCheckpointInterval(checkpointInterval) + .setFeatureSubsetStrategy(featureSubsetStrategy) + .setSubsamplingRate(subsamplingRate) + .setMaxMemoryInMB(maxMemoryInMB) + .setCacheNodeIds(cacheNodeIds) + .setFeaturesCol(rFormula.getFeaturesCol) + if (seed != null && seed.length > 0) rfr.setSeed(seed.toLong) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, rfr)) + .fit(data) + + new RandomForestRegressorWrapper(pipeline, formula, features) + } + + override def read: MLReader[RandomForestRegressorWrapper] = new RandomForestRegressorWrapperReader + + override def load(path: String): RandomForestRegressorWrapper = super.load(path) + + class RandomForestRegressorWrapperWriter(instance: RandomForestRegressorWrapper) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("formula" -> instance.formula) ~ + ("features" -> instance.features.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + instance.pipeline.save(pipelinePath) + } + } + + class RandomForestRegressorWrapperReader extends MLReader[RandomForestRegressorWrapper] { + + override def load(path: String): RandomForestRegressorWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + val pipeline = PipelineModel.load(pipelinePath) + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val formula = (rMetadata \ "formula").extract[String] + val features = (rMetadata \ "features").extract[Array[String]] + + new RandomForestRegressorWrapper(pipeline, formula, features) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/SparkRWrappers.scala b/mllib/src/main/scala/org/apache/spark/ml/r/SparkRWrappers.scala deleted file mode 100644 index 5be2f8693621..000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/r/SparkRWrappers.scala +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.api.r - -import org.apache.spark.ml.attribute._ -import org.apache.spark.ml.feature.RFormula -import org.apache.spark.ml.classification.{LogisticRegression, LogisticRegressionModel} -import org.apache.spark.ml.regression.{LinearRegression, LinearRegressionModel} -import org.apache.spark.ml.{Pipeline, PipelineModel} -import org.apache.spark.sql.DataFrame - -private[r] object SparkRWrappers { - def fitRModelFormula( - value: String, - df: DataFrame, - family: String, - lambda: Double, - alpha: Double, - standardize: Boolean, - solver: String): PipelineModel = { - val formula = new RFormula().setFormula(value) - val estimator = family match { - case "gaussian" => new LinearRegression() - .setRegParam(lambda) - .setElasticNetParam(alpha) - .setFitIntercept(formula.hasIntercept) - .setStandardization(standardize) - .setSolver(solver) - case "binomial" => new LogisticRegression() - .setRegParam(lambda) - .setElasticNetParam(alpha) - .setFitIntercept(formula.hasIntercept) - .setStandardization(standardize) - } - val pipeline = new Pipeline().setStages(Array(formula, estimator)) - pipeline.fit(df) - } - - def getModelCoefficients(model: PipelineModel): Array[Double] = { - model.stages.last match { - case m: LinearRegressionModel => - Array(m.intercept) ++ m.coefficients.toArray - case m: LogisticRegressionModel => - Array(m.intercept) ++ m.coefficients.toArray - } - } - - def getModelFeatures(model: PipelineModel): Array[String] = { - model.stages.last match { - case m: LinearRegressionModel => - val attrs = AttributeGroup.fromStructField( - m.summary.predictions.schema(m.summary.featuresCol)) - Array("(Intercept)") ++ attrs.attributes.get.map(_.name.get) - case m: LogisticRegressionModel => - val attrs = AttributeGroup.fromStructField( - m.summary.predictions.schema(m.summary.featuresCol)) - Array("(Intercept)") ++ attrs.attributes.get.map(_.name.get) - } - } -} 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 535f266b9a94..3d5fd1794de2 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 @@ -19,29 +19,34 @@ package org.apache.spark.ml.recommendation import java.{util => ju} import java.io.IOException +import java.util.Locale import scala.collection.mutable import scala.reflect.ClassTag -import scala.util.Sorting +import scala.util.{Sorting, Try} import scala.util.hashing.byteswap64 import com.github.fommil.netlib.BLAS.{getInstance => blas} -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ -import org.apache.spark.{Logging, Partitioner} -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.{Dependency, Partitioner, ShuffleDependency, SparkContext} +import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.BLAS import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.{Identifiable, SchemaUtils} +import org.apache.spark.ml.util._ import org.apache.spark.mllib.linalg.CholeskyDecomposition import org.apache.spark.mllib.optimization.NNLS import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.{DoubleType, FloatType, IntegerType, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.Utils +import org.apache.spark.util.{BoundedPriorityQueue, Utils} import org.apache.spark.util.collection.{OpenHashMap, OpenHashSet, SortDataFormat, Sorter} import org.apache.spark.util.random.XORShiftRandom @@ -50,24 +55,75 @@ import org.apache.spark.util.random.XORShiftRandom */ private[recommendation] trait ALSModelParams extends Params with HasPredictionCol { /** - * Param for the column name for user ids. + * Param for the column name for user ids. Ids must be integers. Other + * numeric types are supported for this column, but will be cast to integers as long as they + * fall within the integer value range. * Default: "user" * @group param */ - val userCol = new Param[String](this, "userCol", "column name for user ids") + val userCol = new Param[String](this, "userCol", "column name for user ids. Ids must be within " + + "the integer value range.") /** @group getParam */ def getUserCol: String = $(userCol) /** - * Param for the column name for item ids. + * Param for the column name for item ids. Ids must be integers. Other + * numeric types are supported for this column, but will be cast to integers as long as they + * fall within the integer value range. * Default: "item" * @group param */ - val itemCol = new Param[String](this, "itemCol", "column name for item ids") + val itemCol = new Param[String](this, "itemCol", "column name for item ids. Ids must be within " + + "the integer value range.") /** @group getParam */ def getItemCol: String = $(itemCol) + + /** + * Attempts to safely cast a user/item id to an Int. Throws an exception if the value is + * out of integer range or contains a fractional part. + */ + protected[recommendation] val checkedCast = udf { (n: Any) => + n match { + case v: Int => v // Avoid unnecessary casting + case v: Number => + val intV = v.intValue + // Checks if number within Int range and has no fractional part. + if (v.doubleValue == intV) { + intV + } else { + throw new IllegalArgumentException(s"ALS only supports values in Integer range " + + s"and without fractional part for columns ${$(userCol)} and ${$(itemCol)}. " + + s"Value $n was either out of Integer range or contained a fractional part that " + + s"could not be converted.") + } + case _ => throw new IllegalArgumentException(s"ALS only supports values in Integer range " + + s"for columns ${$(userCol)} and ${$(itemCol)}. Value $n was not numeric.") + } + } + + /** + * Param for strategy for dealing with unknown or new users/items at prediction time. + * This may be useful in cross-validation or production scenarios, for handling user/item ids + * the model has not seen in the training data. + * Supported values: + * - "nan": predicted value for unknown ids will be NaN. + * - "drop": rows in the input DataFrame containing unknown ids will be dropped from + * the output DataFrame containing predictions. + * Default: "nan". + * @group expertParam + */ + val coldStartStrategy = new Param[String](this, "coldStartStrategy", + "strategy for dealing with unknown or new users/items at prediction time. This may be " + + "useful in cross-validation or production scenarios, for handling user/item ids the model " + + "has not seen in the training data. Supported values: " + + s"${ALSModel.supportedColdStartStrategies.mkString(",")}.", + (s: String) => + ALSModel.supportedColdStartStrategies.contains(s.toLowerCase(Locale.ROOT))) + + /** @group expertGetParam */ + def getColdStartStrategy: String = $(coldStartStrategy).toLowerCase(Locale.ROOT) } /** @@ -77,7 +133,7 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w with HasPredictionCol with HasCheckpointInterval with HasSeed { /** - * Param for rank of the matrix factorization (>= 1). + * Param for rank of the matrix factorization (positive). * Default: 10 * @group param */ @@ -87,7 +143,7 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w def getRank: Int = $(rank) /** - * Param for number of user blocks (>= 1). + * Param for number of user blocks (positive). * Default: 10 * @group param */ @@ -98,7 +154,7 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w def getNumUserBlocks: Int = $(numUserBlocks) /** - * Param for number of item blocks (>= 1). + * Param for number of item blocks (positive). * Default: 10 * @group param */ @@ -119,7 +175,7 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w def getImplicitPrefs: Boolean = $(implicitPrefs) /** - * Param for the alpha parameter in the implicit preference formulation (>= 0). + * Param for the alpha parameter in the implicit preference formulation (nonnegative). * Default: 1.0 * @group param */ @@ -150,81 +206,284 @@ private[recommendation] trait ALSParams extends ALSModelParams with HasMaxIter w /** @group getParam */ def getNonnegative: Boolean = $(nonnegative) + /** + * Param for StorageLevel for intermediate datasets. Pass in a string representation of + * `StorageLevel`. Cannot be "NONE". + * Default: "MEMORY_AND_DISK". + * + * @group expertParam + */ + val intermediateStorageLevel = new Param[String](this, "intermediateStorageLevel", + "StorageLevel for intermediate datasets. Cannot be 'NONE'.", + (s: String) => Try(StorageLevel.fromString(s)).isSuccess && s != "NONE") + + /** @group expertGetParam */ + def getIntermediateStorageLevel: String = $(intermediateStorageLevel) + + /** + * Param for StorageLevel for ALS model factors. Pass in a string representation of + * `StorageLevel`. + * Default: "MEMORY_AND_DISK". + * + * @group expertParam + */ + val finalStorageLevel = new Param[String](this, "finalStorageLevel", + "StorageLevel for ALS model factors.", + (s: String) => Try(StorageLevel.fromString(s)).isSuccess) + + /** @group expertGetParam */ + def getFinalStorageLevel: String = $(finalStorageLevel) + setDefault(rank -> 10, maxIter -> 10, regParam -> 0.1, numUserBlocks -> 10, numItemBlocks -> 10, implicitPrefs -> false, alpha -> 1.0, userCol -> "user", itemCol -> "item", - ratingCol -> "rating", nonnegative -> false, checkpointInterval -> 10) + ratingCol -> "rating", nonnegative -> false, checkpointInterval -> 10, + intermediateStorageLevel -> "MEMORY_AND_DISK", finalStorageLevel -> "MEMORY_AND_DISK", + coldStartStrategy -> "nan") /** * Validates and transforms the input schema. + * * @param schema input schema * @return output schema */ protected def validateAndTransformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(userCol), IntegerType) - SchemaUtils.checkColumnType(schema, $(itemCol), IntegerType) - val ratingType = schema($(ratingCol)).dataType - require(ratingType == FloatType || ratingType == DoubleType) + // user and item will be cast to Int + SchemaUtils.checkNumericType(schema, $(userCol)) + SchemaUtils.checkNumericType(schema, $(itemCol)) + // rating will be cast to Float + SchemaUtils.checkNumericType(schema, $(ratingCol)) SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) } } /** - * :: Experimental :: * Model fitted by ALS. * * @param rank rank of the matrix factorization model * @param userFactors a DataFrame that stores user factors in two columns: `id` and `features` * @param itemFactors a DataFrame that stores item factors in two columns: `id` and `features` */ -@Experimental +@Since("1.3.0") class ALSModel private[ml] ( - override val uid: String, - val rank: Int, + @Since("1.4.0") override val uid: String, + @Since("1.4.0") val rank: Int, @transient val userFactors: DataFrame, @transient val itemFactors: DataFrame) - extends Model[ALSModel] with ALSModelParams { + extends Model[ALSModel] with ALSModelParams with MLWritable { /** @group setParam */ + @Since("1.4.0") def setUserCol(value: String): this.type = set(userCol, value) /** @group setParam */ + @Since("1.4.0") def setItemCol(value: String): this.type = set(itemCol, value) /** @group setParam */ + @Since("1.3.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) - override def transform(dataset: DataFrame): DataFrame = { - // Register a UDF for DataFrame, and then - // create a new column named map(predictionCol) by running the predict UDF. - val predict = udf { (userFeatures: Seq[Float], itemFeatures: Seq[Float]) => - if (userFeatures != null && itemFeatures != null) { - blas.sdot(rank, userFeatures.toArray, 1, itemFeatures.toArray, 1) - } else { - Float.NaN - } + /** @group expertSetParam */ + @Since("2.2.0") + def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) + + private val predict = udf { (featuresA: Seq[Float], featuresB: Seq[Float]) => + if (featuresA != null && featuresB != null) { + // TODO(SPARK-19759): try dot-producting on Seqs or another non-converted type for + // potential optimization. + blas.sdot(rank, featuresA.toArray, 1, featuresB.toArray, 1) + } else { + Float.NaN } - dataset - .join(userFactors, dataset($(userCol)) === userFactors("id"), "left") - .join(itemFactors, dataset($(itemCol)) === itemFactors("id"), "left") + } + + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema) + // create a new column named map(predictionCol) by running the predict UDF. + val predictions = dataset + .join(userFactors, + checkedCast(dataset($(userCol))) === userFactors("id"), "left") + .join(itemFactors, + checkedCast(dataset($(itemCol))) === itemFactors("id"), "left") .select(dataset("*"), predict(userFactors("features"), itemFactors("features")).as($(predictionCol))) + getColdStartStrategy match { + case ALSModel.Drop => + predictions.na.drop("all", Seq($(predictionCol))) + case ALSModel.NaN => + predictions + } } + @Since("1.3.0") override def transformSchema(schema: StructType): StructType = { - SchemaUtils.checkColumnType(schema, $(userCol), IntegerType) - SchemaUtils.checkColumnType(schema, $(itemCol), IntegerType) + // user and item will be cast to Int + SchemaUtils.checkNumericType(schema, $(userCol)) + SchemaUtils.checkNumericType(schema, $(itemCol)) SchemaUtils.appendColumn(schema, $(predictionCol), FloatType) } + @Since("1.5.0") override def copy(extra: ParamMap): ALSModel = { val copied = new ALSModel(uid, rank, userFactors, itemFactors) copyValues(copied, extra).setParent(parent) } + + @Since("1.6.0") + override def write: MLWriter = new ALSModel.ALSModelWriter(this) + + /** + * Returns top `numItems` items recommended for each user, for all users. + * @param numItems max number of recommendations for each user + * @return a DataFrame of (userCol: Int, recommendations), where recommendations are + * stored as an array of (itemCol: Int, rating: Float) Rows. + */ + @Since("2.2.0") + def recommendForAllUsers(numItems: Int): DataFrame = { + recommendForAll(userFactors, itemFactors, $(userCol), $(itemCol), numItems) + } + + /** + * Returns top `numUsers` users recommended for each item, for all items. + * @param numUsers max number of recommendations for each item + * @return a DataFrame of (itemCol: Int, recommendations), where recommendations are + * stored as an array of (userCol: Int, rating: Float) Rows. + */ + @Since("2.2.0") + def recommendForAllItems(numUsers: Int): DataFrame = { + recommendForAll(itemFactors, userFactors, $(itemCol), $(userCol), numUsers) + } + + /** + * Makes recommendations for all users (or items). + * + * Note: the previous approach used for computing top-k recommendations + * used a cross-join followed by predicting a score for each row of the joined dataset. + * However, this results in exploding the size of intermediate data. While Spark SQL makes it + * relatively efficient, the approach implemented here is significantly more efficient. + * + * This approach groups factors into blocks and computes the top-k elements per block, + * using dot product and an efficient [[BoundedPriorityQueue]] (instead of gemm). + * It then computes the global top-k by aggregating the per block top-k elements with + * a [[TopByKeyAggregator]]. This significantly reduces the size of intermediate and shuffle data. + * This is the DataFrame equivalent to the approach used in + * [[org.apache.spark.mllib.recommendation.MatrixFactorizationModel]]. + * + * @param srcFactors src factors for which to generate recommendations + * @param dstFactors dst factors used to make recommendations + * @param srcOutputColumn name of the column for the source ID in the output DataFrame + * @param dstOutputColumn name of the column for the destination ID in the output DataFrame + * @param num max number of recommendations for each record + * @return a DataFrame of (srcOutputColumn: Int, recommendations), where recommendations are + * stored as an array of (dstOutputColumn: Int, rating: Float) Rows. + */ + private def recommendForAll( + srcFactors: DataFrame, + dstFactors: DataFrame, + srcOutputColumn: String, + dstOutputColumn: String, + num: Int): DataFrame = { + import srcFactors.sparkSession.implicits._ + + val srcFactorsBlocked = blockify(srcFactors.as[(Int, Array[Float])]) + val dstFactorsBlocked = blockify(dstFactors.as[(Int, Array[Float])]) + val ratings = srcFactorsBlocked.crossJoin(dstFactorsBlocked) + .as[(Seq[(Int, Array[Float])], Seq[(Int, Array[Float])])] + .flatMap { case (srcIter, dstIter) => + val m = srcIter.size + val n = math.min(dstIter.size, num) + val output = new Array[(Int, Int, Float)](m * n) + var i = 0 + val pq = new BoundedPriorityQueue[(Int, Float)](num)(Ordering.by(_._2)) + srcIter.foreach { case (srcId, srcFactor) => + dstIter.foreach { case (dstId, dstFactor) => + // We use F2jBLAS which is faster than a call to native BLAS for vector dot product + val score = BLAS.f2jBLAS.sdot(rank, srcFactor, 1, dstFactor, 1) + pq += dstId -> score + } + pq.foreach { case (dstId, score) => + output(i) = (srcId, dstId, score) + i += 1 + } + pq.clear() + } + output.toSeq + } + // We'll force the IDs to be Int. Unfortunately this converts IDs to Int in the output. + val topKAggregator = new TopByKeyAggregator[Int, Int, Float](num, Ordering.by(_._2)) + val recs = ratings.as[(Int, Int, Float)].groupByKey(_._1).agg(topKAggregator.toColumn) + .toDF("id", "recommendations") + + val arrayType = ArrayType( + new StructType() + .add(dstOutputColumn, IntegerType) + .add("rating", FloatType) + ) + recs.select($"id".as(srcOutputColumn), $"recommendations".cast(arrayType)) + } + + /** + * Blockifies factors to improve the efficiency of cross join + * TODO: SPARK-20443 - expose blockSize as a param? + */ + private def blockify( + factors: Dataset[(Int, Array[Float])], + blockSize: Int = 4096): Dataset[Seq[(Int, Array[Float])]] = { + import factors.sparkSession.implicits._ + factors.mapPartitions(_.grouped(blockSize)) + } + } +@Since("1.6.0") +object ALSModel extends MLReadable[ALSModel] { + + private val NaN = "nan" + private val Drop = "drop" + private[recommendation] final val supportedColdStartStrategies = Array(NaN, Drop) + + @Since("1.6.0") + override def read: MLReader[ALSModel] = new ALSModelReader + + @Since("1.6.0") + override def load(path: String): ALSModel = super.load(path) + + private[ALSModel] class ALSModelWriter(instance: ALSModel) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val extraMetadata = "rank" -> instance.rank + DefaultParamsWriter.saveMetadata(instance, path, sc, Some(extraMetadata)) + val userPath = new Path(path, "userFactors").toString + instance.userFactors.write.format("parquet").save(userPath) + val itemPath = new Path(path, "itemFactors").toString + instance.itemFactors.write.format("parquet").save(itemPath) + } + } + + private class ALSModelReader extends MLReader[ALSModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[ALSModel].getName + + override def load(path: String): ALSModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + implicit val format = DefaultFormats + val rank = (metadata.metadata \ "rank").extract[Int] + val userPath = new Path(path, "userFactors").toString + val userFactors = sparkSession.read.format("parquet").load(userPath) + val itemPath = new Path(path, "itemFactors").toString + val itemFactors = sparkSession.read.format("parquet").load(itemPath) + + val model = new ALSModel(metadata.uid, rank, userFactors, itemFactors) + + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } +} /** - * :: Experimental :: * Alternating Least Squares (ALS) matrix factorization. * * ALS attempts to estimate the ratings matrix `R` as the product of two lower-rank matrices, @@ -245,99 +504,145 @@ class ALSModel private[ml] ( * * For implicit preference data, the algorithm used is based on * "Collaborative Filtering for Implicit Feedback Datasets", available at - * [[http://dx.doi.org/10.1109/ICDM.2008.22]], adapted for the blocked approach used here. + * http://dx.doi.org/10.1109/ICDM.2008.22, adapted for the blocked approach used here. * * Essentially instead of finding the low-rank approximations to the rating matrix `R`, * this finds the approximations for a preference matrix `P` where the elements of `P` are 1 if - * r > 0 and 0 if r <= 0. The ratings then act as 'confidence' values related to strength of - * indicated user + * r is greater than 0 and 0 if r is less than or equal to 0. The ratings then act as 'confidence' + * values related to strength of indicated user * preferences rather than explicit ratings given to items. */ -@Experimental -class ALS(override val uid: String) extends Estimator[ALSModel] with ALSParams { +@Since("1.3.0") +class ALS(@Since("1.4.0") override val uid: String) extends Estimator[ALSModel] with ALSParams + with DefaultParamsWritable { import org.apache.spark.ml.recommendation.ALS.Rating + @Since("1.4.0") def this() = this(Identifiable.randomUID("als")) /** @group setParam */ + @Since("1.3.0") def setRank(value: Int): this.type = set(rank, value) /** @group setParam */ + @Since("1.3.0") def setNumUserBlocks(value: Int): this.type = set(numUserBlocks, value) /** @group setParam */ + @Since("1.3.0") def setNumItemBlocks(value: Int): this.type = set(numItemBlocks, value) /** @group setParam */ + @Since("1.3.0") def setImplicitPrefs(value: Boolean): this.type = set(implicitPrefs, value) /** @group setParam */ + @Since("1.3.0") def setAlpha(value: Double): this.type = set(alpha, value) /** @group setParam */ + @Since("1.3.0") def setUserCol(value: String): this.type = set(userCol, value) /** @group setParam */ + @Since("1.3.0") def setItemCol(value: String): this.type = set(itemCol, value) /** @group setParam */ + @Since("1.3.0") def setRatingCol(value: String): this.type = set(ratingCol, value) /** @group setParam */ + @Since("1.3.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) /** @group setParam */ + @Since("1.3.0") def setMaxIter(value: Int): this.type = set(maxIter, value) /** @group setParam */ + @Since("1.3.0") def setRegParam(value: Double): this.type = set(regParam, value) /** @group setParam */ + @Since("1.3.0") def setNonnegative(value: Boolean): this.type = set(nonnegative, value) /** @group setParam */ + @Since("1.4.0") def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) /** @group setParam */ + @Since("1.3.0") def setSeed(value: Long): this.type = set(seed, value) + /** @group expertSetParam */ + @Since("2.0.0") + def setIntermediateStorageLevel(value: String): this.type = set(intermediateStorageLevel, value) + + /** @group expertSetParam */ + @Since("2.0.0") + def setFinalStorageLevel(value: String): this.type = set(finalStorageLevel, value) + + /** @group expertSetParam */ + @Since("2.2.0") + def setColdStartStrategy(value: String): this.type = set(coldStartStrategy, value) + /** * Sets both numUserBlocks and numItemBlocks to the specific value. + * * @group setParam */ + @Since("1.3.0") def setNumBlocks(value: Int): this.type = { setNumUserBlocks(value) setNumItemBlocks(value) this } - override def fit(dataset: DataFrame): ALSModel = { - import dataset.sqlContext.implicits._ + @Since("2.0.0") + override def fit(dataset: Dataset[_]): ALSModel = { + transformSchema(dataset.schema) + import dataset.sparkSession.implicits._ + val r = if ($(ratingCol) != "") col($(ratingCol)).cast(FloatType) else lit(1.0f) val ratings = dataset - .select(col($(userCol)).cast(IntegerType), col($(itemCol)).cast(IntegerType), r) + .select(checkedCast(col($(userCol))), checkedCast(col($(itemCol))), r) + .rdd .map { row => Rating(row.getInt(0), row.getInt(1), row.getFloat(2)) } + + val instr = Instrumentation.create(this, ratings) + instr.logParams(rank, numUserBlocks, numItemBlocks, implicitPrefs, alpha, userCol, + itemCol, ratingCol, predictionCol, maxIter, regParam, nonnegative, checkpointInterval, + seed, intermediateStorageLevel, finalStorageLevel) + val (userFactors, itemFactors) = ALS.train(ratings, rank = $(rank), numUserBlocks = $(numUserBlocks), numItemBlocks = $(numItemBlocks), maxIter = $(maxIter), regParam = $(regParam), implicitPrefs = $(implicitPrefs), alpha = $(alpha), nonnegative = $(nonnegative), + intermediateRDDStorageLevel = StorageLevel.fromString($(intermediateStorageLevel)), + finalRDDStorageLevel = StorageLevel.fromString($(finalStorageLevel)), checkpointInterval = $(checkpointInterval), seed = $(seed)) val userDF = userFactors.toDF("id", "features") val itemDF = itemFactors.toDF("id", "features") val model = new ALSModel(uid, $(rank), userDF, itemDF).setParent(this) + instr.logSuccess(model) copyValues(model) } + @Since("1.3.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } + @Since("1.5.0") override def copy(extra: ParamMap): ALS = defaultCopy(extra) } + /** * :: DeveloperApi :: * An implementation of ALS that supports generic ID types, specialized for Int and Long. This is @@ -347,7 +652,7 @@ class ALS(override val uid: String) extends Estimator[ALSModel] with ALSParams { * than 2 billion. */ @DeveloperApi -object ALS extends Logging { +object ALS extends DefaultParamsReadable[ALS] with Logging { /** * :: DeveloperApi :: @@ -356,6 +661,9 @@ object ALS extends Logging { @DeveloperApi case class Rating[@specialized(Int, Long) ID](user: ID, item: ID, rating: Float) + @Since("1.6.0") + override def load(path: String): ALS = super.load(path) + /** Trait for least squares solvers applied to the normal equation. */ private[recommendation] trait LeastSquaresNESolver extends Serializable { /** Solves a least squares problem with regularization (possibly with other constraints). */ @@ -415,7 +723,7 @@ object ALS extends Logging { } /** - * Solves a nonnegative least squares problem with L2 regularizatin: + * Solves a nonnegative least squares problem with L2 regularization: * * min_x_ norm(A x - b)^2^ + lambda * n * norm(x)^2^ * subject to x >= 0 @@ -455,11 +763,15 @@ object ALS extends Logging { /** * Representing a normal equation to solve the following weighted least squares problem: * - * minimize \sum,,i,, c,,i,, (a,,i,,^T^ x - b,,i,,)^2^ + lambda * x^T^ x. + * minimize \sum,,i,, c,,i,, (a,,i,,^T^ x - d,,i,,)^2^ + lambda * x^T^ x. * * Its normal equation is given by * - * \sum,,i,, c,,i,, (a,,i,, a,,i,,^T^ x - b,,i,, a,,i,,) + lambda * x = 0. + * \sum,,i,, c,,i,, (a,,i,, a,,i,,^T^ x - d,,i,, a,,i,,) + lambda * x = 0. + * + * Distributing and letting b,,i,, = c,,i,, * d,,i,, + * + * \sum,,i,, c,,i,, a,,i,, a,,i,,^T^ x - b,,i,, a,,i,, + lambda * x = 0. */ private[recommendation] class NormalEquation(val k: Int) extends Serializable { @@ -488,7 +800,7 @@ object ALS extends Logging { copyToDouble(a) blas.dspr(upper, k, c, da, 1, ata) if (b != 0.0) { - blas.daxpy(k, c * b, da, 1, atb, 1) + blas.daxpy(k, b, da, 1, atb, 1) } this } @@ -511,6 +823,28 @@ object ALS extends Logging { /** * :: DeveloperApi :: * Implementation of the ALS algorithm. + * + * This implementation of the ALS factorization algorithm partitions the two sets of factors among + * Spark workers so as to reduce network communication by only sending one copy of each factor + * vector to each Spark worker on each iteration, and only if needed. This is achieved by + * precomputing some information about the ratings matrix to determine which users require which + * item factors and vice versa. See the Scaladoc for `InBlock` for a detailed explanation of how + * the precomputation is done. + * + * In addition, since each iteration of calculating the factor matrices depends on the known + * ratings, which are spread across Spark partitions, a naive implementation would incur + * significant network communication overhead between Spark workers, as the ratings RDD would be + * repeatedly shuffled during each iteration. This implementation reduces that overhead by + * performing the shuffling operation up front, precomputing each partition's ratings dependencies + * and duplicating those values to the appropriate workers before starting iterations to solve for + * the factor matrices. See the Scaladoc for `OutBlock` for a detailed explanation of how the + * precomputation is done. + * + * Note that the term "rating block" is a bit of a misnomer, as the ratings are not partitioned by + * contiguous blocks from the ratings matrix but by a hash function on the rating's location in + * the matrix. If it helps you to visualize the partitions, it is easier to think of the term + * "block" as referring to a subset of an RDD containing the ratings rather than a contiguous + * submatrix of the ratings matrix. */ @DeveloperApi def train[ID: ClassTag]( // scalastyle:ignore @@ -519,7 +853,7 @@ object ALS extends Logging { numUserBlocks: Int = 10, numItemBlocks: Int = 10, maxIter: Int = 10, - regParam: Double = 1.0, + regParam: Double = 0.1, implicitPrefs: Boolean = false, alpha: Double = 1.0, nonnegative: Boolean = false, @@ -528,43 +862,57 @@ object ALS extends Logging { checkpointInterval: Int = 10, seed: Long = 0L)( implicit ord: Ordering[ID]): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { + + require(!ratings.isEmpty(), s"No ratings available from $ratings") require(intermediateRDDStorageLevel != StorageLevel.NONE, "ALS is not designed to run without persisting intermediate RDDs.") + val sc = ratings.sparkContext + + // Precompute the rating dependencies of each partition val userPart = new ALSPartitioner(numUserBlocks) val itemPart = new ALSPartitioner(numItemBlocks) - val userLocalIndexEncoder = new LocalIndexEncoder(userPart.numPartitions) - val itemLocalIndexEncoder = new LocalIndexEncoder(itemPart.numPartitions) - val solver = if (nonnegative) new NNLSSolver else new CholeskySolver val blockRatings = partitionRatings(ratings, userPart, itemPart) .persist(intermediateRDDStorageLevel) val (userInBlocks, userOutBlocks) = makeBlocks("user", blockRatings, userPart, itemPart, intermediateRDDStorageLevel) - // materialize blockRatings and user blocks - userOutBlocks.count() + userOutBlocks.count() // materialize blockRatings and user blocks val swappedBlockRatings = blockRatings.map { case ((userBlockId, itemBlockId), RatingBlock(userIds, itemIds, localRatings)) => ((itemBlockId, userBlockId), RatingBlock(itemIds, userIds, localRatings)) } val (itemInBlocks, itemOutBlocks) = makeBlocks("item", swappedBlockRatings, itemPart, userPart, intermediateRDDStorageLevel) - // materialize item blocks - itemOutBlocks.count() + itemOutBlocks.count() // materialize item blocks + + // Encoders for storing each user/item's partition ID and index within its partition using a + // single integer; used as an optimization + val userLocalIndexEncoder = new LocalIndexEncoder(userPart.numPartitions) + val itemLocalIndexEncoder = new LocalIndexEncoder(itemPart.numPartitions) + + // These are the user and item factor matrices that, once trained, are multiplied together to + // estimate the rating matrix. The two matrices are stored in RDDs, partitioned by column such + // that each factor column resides on the same Spark worker as its corresponding user or item. val seedGen = new XORShiftRandom(seed) var userFactors = initialize(userInBlocks, rank, seedGen.nextLong()) var itemFactors = initialize(itemInBlocks, rank, seedGen.nextLong()) + + val solver = if (nonnegative) new NNLSSolver else new CholeskySolver + var previousCheckpointFile: Option[String] = None val shouldCheckpoint: Int => Boolean = (iter) => sc.checkpointDir.isDefined && checkpointInterval != -1 && (iter % checkpointInterval == 0) val deletePreviousCheckpointFile: () => Unit = () => previousCheckpointFile.foreach { file => try { - FileSystem.get(sc.hadoopConfiguration).delete(new Path(file), true) + val checkpointFile = new Path(file) + checkpointFile.getFileSystem(sc.hadoopConfiguration).delete(checkpointFile, true) } catch { case e: IOException => logWarning(s"Cannot delete checkpoint file $file:", e) } } + if (implicitPrefs) { for (iter <- 1 to maxIter) { userFactors.setName(s"userFactors-$iter").persist(intermediateRDDStorageLevel) @@ -574,13 +922,15 @@ object ALS extends Logging { previousItemFactors.unpersist() itemFactors.setName(s"itemFactors-$iter").persist(intermediateRDDStorageLevel) // TODO: Generalize PeriodicGraphCheckpointer and use it here. + val deps = itemFactors.dependencies if (shouldCheckpoint(iter)) { - itemFactors.checkpoint() // itemFactors gets materialized in computeFactors. + itemFactors.checkpoint() // itemFactors gets materialized in computeFactors } val previousUserFactors = userFactors userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, itemLocalIndexEncoder, implicitPrefs, alpha, solver) if (shouldCheckpoint(iter)) { + ALS.cleanShuffleDependencies(sc, deps) deletePreviousCheckpointFile() previousCheckpointFile = itemFactors.getCheckpointFile } @@ -591,8 +941,10 @@ object ALS extends Logging { itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, userLocalIndexEncoder, solver = solver) if (shouldCheckpoint(iter)) { + val deps = itemFactors.dependencies itemFactors.checkpoint() itemFactors.count() // checkpoint item factors and cut lineage + ALS.cleanShuffleDependencies(sc, deps) deletePreviousCheckpointFile() previousCheckpointFile = itemFactors.getCheckpointFile } @@ -641,33 +993,160 @@ object ALS extends Logging { private type FactorBlock = Array[Array[Float]] /** - * Out-link block that stores, for each dst (item/user) block, which src (user/item) factors to - * send. For example, outLinkBlock(0) contains the local indices (not the original src IDs) of the - * src factors in this block to send to dst block 0. + * A mapping of the columns of the items factor matrix that are needed when calculating each row + * of the users factor matrix, and vice versa. + * + * Specifically, when calculating a user factor vector, since only those columns of the items + * factor matrix that correspond to the items that that user has rated are needed, we can avoid + * having to repeatedly copy the entire items factor matrix to each worker later in the algorithm + * by precomputing these dependencies for all users, storing them in an RDD of `OutBlock`s. The + * items' dependencies on the columns of the users factor matrix is computed similarly. + * + * =Example= + * + * Using the example provided in the `InBlock` Scaladoc, `userOutBlocks` would look like the + * following: + * + * {{{ + * userOutBlocks.collect() == Seq( + * 0 -> Array(Array(0, 1), Array(0, 1)), + * 1 -> Array(Array(0), Array(0)) + * ) + * }}} + * + * Each value in this map-like sequence is of type `Array[Array[Int]]`. The values in the + * inner array are the ranks of the sorted user IDs in that partition; so in the example above, + * `Array(0, 1)` in partition 0 refers to user IDs 0 and 6, since when all unique user IDs in + * partition 0 are sorted, 0 is the first ID and 6 is the second. The position of each inner + * array in its enclosing outer array denotes the partition number to which item IDs map; in the + * example, the first `Array(0, 1)` is in position 0 of its outer array, denoting item IDs that + * map to partition 0. + * + * In summary, the data structure encodes the following information: + * + * * There are ratings with user IDs 0 and 6 (encoded in `Array(0, 1)`, where 0 and 1 are the + * indices of the user IDs 0 and 6 on partition 0) whose item IDs map to partitions 0 and 1 + * (represented by the fact that `Array(0, 1)` appears in both the 0th and 1st positions). + * + * * There are ratings with user ID 3 (encoded in `Array(0)`, where 0 is the index of the user + * ID 3 on partition 1) whose item IDs map to partitions 0 and 1 (represented by the fact that + * `Array(0)` appears in both the 0th and 1st positions). */ private type OutBlock = Array[Array[Int]] /** - * In-link block for computing src (user/item) factors. This includes the original src IDs - * of the elements within this block as well as encoded dst (item/user) indices and corresponding - * ratings. The dst indices are in the form of (blockId, localIndex), which are not the original - * dst IDs. To compute src factors, we expect receiving dst factors that match the dst indices. - * For example, if we have an in-link record + * In-link block for computing user and item factor matrices. + * + * The ALS algorithm partitions the columns of the users factor matrix evenly among Spark workers. + * Since each column of the factor matrix is calculated using the known ratings of the correspond- + * ing user, and since the ratings don't change across iterations, the ALS algorithm preshuffles + * the ratings to the appropriate partitions, storing them in `InBlock` objects. + * + * The ratings shuffled by item ID are computed similarly and also stored in `InBlock` objects. + * Note that this means every rating is stored twice, once as shuffled by user ID and once by item + * ID. This is a necessary tradeoff, since in general a rating will not be on the same worker + * when partitioned by user as by item. + * + * =Example= + * + * Say we have a small collection of eight items to offer the seven users in our application. We + * have some known ratings given by the users, as seen in the matrix below: + * + * {{{ + * Items + * 0 1 2 3 4 5 6 7 + * +---+---+---+---+---+---+---+---+ + * 0 | |0.1| | |0.4| | |0.7| + * +---+---+---+---+---+---+---+---+ + * 1 | | | | | | | | | + * +---+---+---+---+---+---+---+---+ + * U 2 | | | | | | | | | + * s +---+---+---+---+---+---+---+---+ + * e 3 | |3.1| | |3.4| | |3.7| + * r +---+---+---+---+---+---+---+---+ + * s 4 | | | | | | | | | + * +---+---+---+---+---+---+---+---+ + * 5 | | | | | | | | | + * +---+---+---+---+---+---+---+---+ + * 6 | |6.1| | |6.4| | |6.7| + * +---+---+---+---+---+---+---+---+ + * }}} + * + * The ratings are represented as an RDD, passed to the `partitionRatings` method as the `ratings` + * parameter: * - * {srcId: 0, dstBlockId: 2, dstLocalIndex: 3, rating: 5.0}, + * {{{ + * ratings.collect() == Seq( + * Rating(0, 1, 0.1f), + * Rating(0, 4, 0.4f), + * Rating(0, 7, 0.7f), + * Rating(3, 1, 3.1f), + * Rating(3, 4, 3.4f), + * Rating(3, 7, 3.7f), + * Rating(6, 1, 6.1f), + * Rating(6, 4, 6.4f), + * Rating(6, 7, 6.7f) + * ) + * }}} * - * and assume that the dst factors are stored as dstFactors: Map[Int, Array[Array[Float]]], which - * is a blockId to dst factors map, the corresponding dst factor of the record is dstFactor(2)(3). + * Say that we are using two partitions to calculate each factor matrix: * - * We use a CSC-like (compressed sparse column) format to store the in-link information. So we can - * compute src factors one after another using only one normal equation instance. + * {{{ + * val userPart = new ALSPartitioner(2) + * val itemPart = new ALSPartitioner(2) + * val blockRatings = partitionRatings(ratings, userPart, itemPart) + * }}} + * + * Ratings are mapped to partitions using the user/item IDs modulo the number of partitions. With + * two partitions, ratings with even-valued user IDs are shuffled to partition 0 while those with + * odd-valued user IDs are shuffled to partition 1: + * + * {{{ + * userInBlocks.collect() == Seq( + * 0 -> Seq( + * // Internally, the class stores the ratings in a more optimized format than + * // a sequence of `Rating`s, but for clarity we show it as such here. + * Rating(0, 1, 0.1f), + * Rating(0, 4, 0.4f), + * Rating(0, 7, 0.7f), + * Rating(6, 1, 6.1f), + * Rating(6, 4, 6.4f), + * Rating(6, 7, 6.7f) + * ), + * 1 -> Seq( + * Rating(3, 1, 3.1f), + * Rating(3, 4, 3.4f), + * Rating(3, 7, 3.7f) + * ) + * ) + * }}} + * + * Similarly, ratings with even-valued item IDs are shuffled to partition 0 while those with + * odd-valued item IDs are shuffled to partition 1: + * + * {{{ + * itemInBlocks.collect() == Seq( + * 0 -> Seq( + * Rating(0, 4, 0.4f), + * Rating(3, 4, 3.4f), + * Rating(6, 4, 6.4f) + * ), + * 1 -> Seq( + * Rating(0, 1, 0.1f), + * Rating(0, 7, 0.7f), + * Rating(3, 1, 3.1f), + * Rating(3, 7, 3.7f), + * Rating(6, 1, 6.1f), + * Rating(6, 7, 6.7f) + * ) + * ) + * }}} * * @param srcIds src ids (ordered) * @param dstPtrs dst pointers. Elements in range [dstPtrs(i), dstPtrs(i+1)) of dst indices and * ratings are associated with srcIds(i). * @param dstEncodedIndices encoded dst indices * @param ratings ratings - * * @see [[LocalIndexEncoder]] */ private[recommendation] case class InBlock[@specialized(Int, Long) ID: ClassTag]( @@ -723,7 +1202,7 @@ object ALS extends Logging { } /** - * Builder for [[RatingBlock]]. [[mutable.ArrayBuilder]] is used to avoid boxing/unboxing. + * Builder for [[RatingBlock]]. `mutable.ArrayBuilder` is used to avoid boxing/unboxing. */ private[recommendation] class RatingBlockBuilder[@specialized(Int, Long) ID: ClassTag] extends Serializable { @@ -758,29 +1237,34 @@ object ALS extends Logging { } /** - * Partitions raw ratings into blocks. + * Groups an RDD of [[Rating]]s by the user partition and item partition to which each `Rating` + * maps according to the given partitioners. The returned pair RDD holds the ratings, encoded in + * a memory-efficient format but otherwise unchanged, keyed by the (user partition ID, item + * partition ID) pair. + * + * Performance note: This is an expensive operation that performs an RDD shuffle. + * + * Implementation note: This implementation produces the same result as the following but + * generates fewer intermediate objects: + * + * {{{ + * ratings.map { r => + * ((srcPart.getPartition(r.user), dstPart.getPartition(r.item)), r) + * }.aggregateByKey(new RatingBlockBuilder)( + * seqOp = (b, r) => b.add(r), + * combOp = (b0, b1) => b0.merge(b1.build())) + * .mapValues(_.build()) + * }}} * * @param ratings raw ratings * @param srcPart partitioner for src IDs * @param dstPart partitioner for dst IDs - * * @return an RDD of rating blocks in the form of ((srcBlockId, dstBlockId), ratingBlock) */ private def partitionRatings[ID: ClassTag]( ratings: RDD[Rating[ID]], srcPart: Partitioner, dstPart: Partitioner): RDD[((Int, Int), RatingBlock[ID])] = { - - /* The implementation produces the same result as the following but generates less objects. - - ratings.map { r => - ((srcPart.getPartition(r.user), dstPart.getPartition(r.item)), r) - }.aggregateByKey(new RatingBlockBuilder)( - seqOp = (b, r) => b.add(r), - combOp = (b0, b1) => b0.merge(b1.build())) - .mapValues(_.build()) - */ - val numPartitions = srcPart.numPartitions * dstPart.numPartitions ratings.mapPartitions { iter => val builders = Array.fill(numPartitions)(new RatingBlockBuilder[ID]) @@ -812,6 +1296,7 @@ object ALS extends Logging { /** * Builder for uncompressed in-blocks of (srcId, dstEncodedIndex, rating) tuples. + * * @param encoder encoder for dst indices */ private[recommendation] class UncompressedInBlockBuilder[@specialized(Int, Long) ID: ClassTag]( @@ -867,8 +1352,8 @@ object ALS extends Logging { def length: Int = srcIds.length /** - * Compresses the block into an [[InBlock]]. The algorithm is the same as converting a - * sparse matrix from coordinate list (COO) format into compressed sparse column (CSC) format. + * Compresses the block into an `InBlock`. The algorithm is the same as converting a sparse + * matrix from coordinate list (COO) format into compressed sparse column (CSC) format. * Sorting is done using Spark's built-in Timsort to avoid generating too many objects. */ def compress(): InBlock[ID] = { @@ -881,14 +1366,12 @@ object ALS extends Logging { uniqueSrcIdsBuilder += preSrcId var curCount = 1 var i = 1 - var j = 0 while (i < sz) { val srcId = srcIds(i) if (srcId != preSrcId) { uniqueSrcIdsBuilder += srcId dstCountsBuilder += curCount preSrcId = srcId - j += 1 curCount = 0 } curCount += 1 @@ -1012,6 +1495,7 @@ object ALS extends Logging { /** * Creates in-blocks and out-blocks from rating blocks. + * * @param prefix prefix for in/out-block names * @param ratingBlocks rating blocks * @param srcPart partitioner for src IDs @@ -1100,7 +1584,6 @@ object ALS extends Logging { * @param implicitPrefs whether to use implicit preference * @param alpha the alpha constant in the implicit preference formulation * @param solver solver for least squares problems - * * @return dst factors */ private def computeFactors[ID]( @@ -1145,15 +1628,15 @@ object ALS extends Logging { val srcFactor = sortedSrcFactors(blockId)(localIndex) val rating = ratings(i) if (implicitPrefs) { - // Extension to the original paper to handle b < 0. confidence is a function of |b| - // instead so that it is never negative. c1 is confidence - 1.0. + // Extension to the original paper to handle rating < 0. confidence is a function + // of |rating| instead so that it is never negative. c1 is confidence - 1. val c1 = alpha * math.abs(rating) - // For rating <= 0, the corresponding preference is 0. So the term below is only added - // for rating > 0. Because YtY is already added, we need to adjust the scaling here. - if (rating > 0) { + // For rating <= 0, the corresponding preference is 0. So the second argument of add + // is only there for rating > 0. + if (rating > 0.0) { numExplicits += 1 - ls.add(srcFactor, (c1 + 1.0) / c1, c1) } + ls.add(srcFactor, if (rating > 0.0) 1.0 + c1 else 0.0, c1) } else { ls.add(srcFactor, rating) numExplicits += 1 @@ -1219,9 +1702,36 @@ object ALS extends Logging { } /** - * Partitioner used by ALS. We requires that getPartition is a projection. That is, for any key k, - * we have getPartition(getPartition(k)) = getPartition(k). Since the the default HashPartitioner + * Partitioner used by ALS. We require that getPartition is a projection. That is, for any key k, + * we have getPartition(getPartition(k)) = getPartition(k). Since the default HashPartitioner * satisfies this requirement, we simply use a type alias here. */ private[recommendation] type ALSPartitioner = org.apache.spark.HashPartitioner + + /** + * Private function to clean up all of the shuffles files from the dependencies and their parents. + */ + private[spark] def cleanShuffleDependencies[T]( + sc: SparkContext, + deps: Seq[Dependency[_]], + blocking: Boolean = false): Unit = { + // If there is no reference tracking we skip clean up. + sc.cleaner.foreach { cleaner => + /** + * Clean the shuffles & all of its parents. + */ + def cleanEagerly(dep: Dependency[_]): Unit = { + if (dep.isInstanceOf[ShuffleDependency[_, _, _]]) { + val shuffleId = dep.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId + cleaner.doCleanupShuffle(shuffleId, blocking) + } + val rdd = dep.rdd + val rddDeps = rdd.dependencies + if (rdd.getStorageLevel == StorageLevel.NONE && rddDeps != null) { + rddDeps.foreach(cleanEagerly) + } + } + deps.foreach(cleanEagerly) + } + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/TopByKeyAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/TopByKeyAggregator.scala new file mode 100644 index 000000000000..517179c0eb9a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/TopByKeyAggregator.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.recommendation + +import scala.language.implicitConversions +import scala.reflect.runtime.universe.TypeTag + +import org.apache.spark.sql.{Encoder, Encoders} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.expressions.Aggregator +import org.apache.spark.util.BoundedPriorityQueue + + +/** + * Works on rows of the form (K1, K2, V) where K1 & K2 are IDs and V is the score value. Finds + * the top `num` K2 items based on the given Ordering. + */ +private[recommendation] class TopByKeyAggregator[K1: TypeTag, K2: TypeTag, V: TypeTag] + (num: Int, ord: Ordering[(K2, V)]) + extends Aggregator[(K1, K2, V), BoundedPriorityQueue[(K2, V)], Array[(K2, V)]] { + + override def zero: BoundedPriorityQueue[(K2, V)] = new BoundedPriorityQueue[(K2, V)](num)(ord) + + override def reduce( + q: BoundedPriorityQueue[(K2, V)], + a: (K1, K2, V)): BoundedPriorityQueue[(K2, V)] = { + q += {(a._2, a._3)} + } + + override def merge( + q1: BoundedPriorityQueue[(K2, V)], + q2: BoundedPriorityQueue[(K2, V)]): BoundedPriorityQueue[(K2, V)] = { + q1 ++= q2 + } + + override def finish(r: BoundedPriorityQueue[(K2, V)]): Array[(K2, V)] = { + r.toArray.sorted(ord.reverse) + } + + override def bufferEncoder: Encoder[BoundedPriorityQueue[(K2, V)]] = { + Encoders.kryo[BoundedPriorityQueue[(K2, V)]] + } + + override def outputEncoder: Encoder[Array[(K2, V)]] = ExpressionEncoder[Array[(K2, V)]]() +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index b7d095872ffa..4b46c3831d75 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -21,17 +21,22 @@ import scala.collection.mutable import breeze.linalg.{DenseVector => BDV} import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS} - -import org.apache.spark.{SparkException, Logging} -import org.apache.spark.annotation.{Since, Experimental} -import org.apache.spark.ml.{Model, Estimator} +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.{SchemaUtils, Identifiable} -import org.apache.spark.mllib.linalg.{Vector, Vectors, VectorUDT} -import org.apache.spark.mllib.linalg.BLAS +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer +import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, DataFrame} +import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DoubleType, StructType} import org.apache.spark.storage.StorageLevel @@ -41,7 +46,7 @@ import org.apache.spark.storage.StorageLevel */ private[regression] trait AFTSurvivalRegressionParams extends Params with HasFeaturesCol with HasLabelCol with HasPredictionCol with HasMaxIter - with HasTol with HasFitIntercept with Logging { + with HasTol with HasFitIntercept with HasAggregationDepth with Logging { /** * Param for censor column name. @@ -86,8 +91,8 @@ private[regression] trait AFTSurvivalRegressionParams extends Params def getQuantilesCol: String = $(quantilesCol) /** Checks whether the input has quantiles column name. */ - protected[regression] def hasQuantilesCol: Boolean = { - isDefined(quantilesCol) && $(quantilesCol) != "" + private[regression] def hasQuantilesCol: Boolean = { + isDefined(quantilesCol) && $(quantilesCol).nonEmpty } /** @@ -101,26 +106,30 @@ private[regression] trait AFTSurvivalRegressionParams extends Params fitting: Boolean): StructType = { SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT) if (fitting) { - SchemaUtils.checkColumnType(schema, $(censorCol), DoubleType) - SchemaUtils.checkColumnType(schema, $(labelCol), DoubleType) + SchemaUtils.checkNumericType(schema, $(censorCol)) + SchemaUtils.checkNumericType(schema, $(labelCol)) } - if (hasQuantilesCol) { + + val schemaWithQuantilesCol = if (hasQuantilesCol) { SchemaUtils.appendColumn(schema, $(quantilesCol), new VectorUDT) - } - SchemaUtils.appendColumn(schema, $(predictionCol), DoubleType) + } else schema + + SchemaUtils.appendColumn(schemaWithQuantilesCol, $(predictionCol), DoubleType) } } /** * :: Experimental :: * Fit a parametric survival regression model named accelerated failure time (AFT) model - * ([[https://en.wikipedia.org/wiki/Accelerated_failure_time_model]]) + * (see + * Accelerated failure time model (Wikipedia)) * based on the Weibull distribution of the survival time. */ @Experimental @Since("1.6.0") class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: String) - extends Estimator[AFTSurvivalRegressionModel] with AFTSurvivalRegressionParams with Logging { + extends Estimator[AFTSurvivalRegressionModel] with AFTSurvivalRegressionParams + with DefaultParamsWritable with Logging { @Since("1.6.0") def this() = this(Identifiable.randomUID("aftSurvReg")) @@ -177,28 +186,67 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S def setTol(value: Double): this.type = set(tol, value) setDefault(tol -> 1E-6) + /** + * Suggested depth for treeAggregate (greater than or equal to 2). + * If the dimensions of features or the number of partitions are large, + * this param could be adjusted to a larger size. + * Default is 2. + * @group expertSetParam + */ + @Since("2.1.0") + def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) + setDefault(aggregationDepth -> 2) + /** * Extract [[featuresCol]], [[labelCol]] and [[censorCol]] from input dataset, * and put it in an RDD with strong types. */ - protected[ml] def extractAFTPoints(dataset: DataFrame): RDD[AFTPoint] = { - dataset.select($(featuresCol), $(labelCol), $(censorCol)).map { - case Row(features: Vector, label: Double, censor: Double) => - AFTPoint(features, label, censor) - } + protected[ml] def extractAFTPoints(dataset: Dataset[_]): RDD[AFTPoint] = { + dataset.select(col($(featuresCol)), col($(labelCol)).cast(DoubleType), + col($(censorCol)).cast(DoubleType)).rdd.map { + case Row(features: Vector, label: Double, censor: Double) => + AFTPoint(features, label, censor) + } } - @Since("1.6.0") - override def fit(dataset: DataFrame): AFTSurvivalRegressionModel = { - validateAndTransformSchema(dataset.schema, fitting = true) + @Since("2.0.0") + override def fit(dataset: Dataset[_]): AFTSurvivalRegressionModel = { + transformSchema(dataset.schema, logging = true) val instances = extractAFTPoints(dataset) - val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE + val handlePersistence = dataset.storageLevel == StorageLevel.NONE if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) - val costFun = new AFTCostFun(instances, $(fitIntercept)) + val featuresSummarizer = { + val seqOp = (c: MultivariateOnlineSummarizer, v: AFTPoint) => c.add(v.features) + val combOp = (c1: MultivariateOnlineSummarizer, c2: MultivariateOnlineSummarizer) => { + c1.merge(c2) + } + instances.treeAggregate( + new MultivariateOnlineSummarizer + )(seqOp, combOp, $(aggregationDepth)) + } + + val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val numFeatures = featuresStd.size + + val instr = Instrumentation.create(this, dataset) + instr.logParams(labelCol, featuresCol, censorCol, predictionCol, quantilesCol, + fitIntercept, maxIter, tol, aggregationDepth) + instr.logNamedValue("quantileProbabilities.size", $(quantileProbabilities).length) + instr.logNumFeatures(numFeatures) + + if (!$(fitIntercept) && (0 until numFeatures).exists { i => + featuresStd(i) == 0.0 && featuresSummarizer.mean(i) != 0.0 }) { + logWarning("Fitting AFTSurvivalRegressionModel without intercept on dataset with " + + "constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero " + + "columns. This behavior is different from R survival::survreg.") + } + + val bcFeaturesStd = instances.context.broadcast(featuresStd) + + val costFun = new AFTCostFun(instances, $(fitIntercept), bcFeaturesStd, $(aggregationDepth)) val optimizer = new BreezeLBFGS[BDV[Double]]($(maxIter), 10, $(tol)) - val numFeatures = dataset.select($(featuresCol)).take(1)(0).getAs[Vector](0).size /* The parameters vector has three parts: the first element: Double, log(sigma), the log of scale parameter @@ -208,7 +256,7 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S val initialParameters = Vectors.zeros(numFeatures + 2) val states = optimizer.iterations(new CachedDiffFunction(costFun), - initialParameters.toBreeze.toDenseVector) + initialParameters.asBreeze.toDenseVector) val parameters = { val arrayBuilder = mutable.ArrayBuilder.make[Double] @@ -221,17 +269,25 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S val msg = s"${optimizer.getClass.getName} failed." throw new SparkException(msg) } - state.x.toArray.clone() } + bcFeaturesStd.destroy(blocking = false) if (handlePersistence) instances.unpersist() - val coefficients = Vectors.dense(parameters.slice(2, parameters.length)) + val rawCoefficients = parameters.slice(2, parameters.length) + var i = 0 + while (i < numFeatures) { + rawCoefficients(i) *= { if (featuresStd(i) != 0.0) 1.0 / featuresStd(i) else 0.0 } + i += 1 + } + val coefficients = Vectors.dense(rawCoefficients) val intercept = parameters(1) val scale = math.exp(parameters(0)) - val model = new AFTSurvivalRegressionModel(uid, coefficients, intercept, scale) - copyValues(model.setParent(this)) + val model = copyValues(new AFTSurvivalRegressionModel(uid, coefficients, + intercept, scale).setParent(this)) + instr.logSuccess(model) + model } @Since("1.6.0") @@ -243,6 +299,13 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S override def copy(extra: ParamMap): AFTSurvivalRegression = defaultCopy(extra) } +@Since("1.6.0") +object AFTSurvivalRegression extends DefaultParamsReadable[AFTSurvivalRegression] { + + @Since("1.6.0") + override def load(path: String): AFTSurvivalRegression = super.load(path) +} + /** * :: Experimental :: * Model produced by [[AFTSurvivalRegression]]. @@ -251,10 +314,10 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S @Since("1.6.0") class AFTSurvivalRegressionModel private[ml] ( @Since("1.6.0") override val uid: String, - @Since("1.6.0") val coefficients: Vector, + @Since("2.0.0") val coefficients: Vector, @Since("1.6.0") val intercept: Double, @Since("1.6.0") val scale: Double) - extends Model[AFTSurvivalRegressionModel] with AFTSurvivalRegressionParams { + extends Model[AFTSurvivalRegressionModel] with AFTSurvivalRegressionParams with MLWritable { /** @group setParam */ @Since("1.6.0") @@ -272,7 +335,7 @@ class AFTSurvivalRegressionModel private[ml] ( @Since("1.6.0") def setQuantilesCol(value: String): this.type = set(quantilesCol, value) - @Since("1.6.0") + @Since("2.0.0") def predictQuantiles(features: Vector): Vector = { // scale parameter for the Weibull distribution of lifetime val lambda = math.exp(BLAS.dot(coefficients, features) + intercept) @@ -284,14 +347,14 @@ class AFTSurvivalRegressionModel private[ml] ( Vectors.dense(quantiles) } - @Since("1.6.0") + @Since("2.0.0") def predict(features: Vector): Double = { math.exp(BLAS.dot(coefficients, features) + intercept) } - @Since("1.6.0") - override def transform(dataset: DataFrame): DataFrame = { - transformSchema(dataset.schema) + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) val predictUDF = udf { features: Vector => predict(features) } val predictQuantilesUDF = udf { features: Vector => predictQuantiles(features)} if (hasQuantilesCol) { @@ -312,11 +375,63 @@ class AFTSurvivalRegressionModel private[ml] ( copyValues(new AFTSurvivalRegressionModel(uid, coefficients, intercept, scale), extra) .setParent(parent) } + + @Since("1.6.0") + override def write: MLWriter = + new AFTSurvivalRegressionModel.AFTSurvivalRegressionModelWriter(this) +} + +@Since("1.6.0") +object AFTSurvivalRegressionModel extends MLReadable[AFTSurvivalRegressionModel] { + + @Since("1.6.0") + override def read: MLReader[AFTSurvivalRegressionModel] = new AFTSurvivalRegressionModelReader + + @Since("1.6.0") + override def load(path: String): AFTSurvivalRegressionModel = super.load(path) + + /** [[MLWriter]] instance for [[AFTSurvivalRegressionModel]] */ + private[AFTSurvivalRegressionModel] class AFTSurvivalRegressionModelWriter ( + instance: AFTSurvivalRegressionModel + ) extends MLWriter with Logging { + + private case class Data(coefficients: Vector, intercept: Double, scale: Double) + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + // Save model data: coefficients, intercept, scale + val data = Data(instance.coefficients, instance.intercept, instance.scale) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class AFTSurvivalRegressionModelReader extends MLReader[AFTSurvivalRegressionModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[AFTSurvivalRegressionModel].getName + + override def load(path: String): AFTSurvivalRegressionModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + val Row(coefficients: Vector, intercept: Double, scale: Double) = + MLUtils.convertVectorColumnsToML(data, "coefficients") + .select("coefficients", "intercept", "scale") + .head() + val model = new AFTSurvivalRegressionModel(metadata.uid, coefficients, intercept, scale) + + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } } /** * AFTAggregator computes the gradient and loss for a AFT loss function, - * as used in AFT survival regression for samples in sparse or dense vector in a online fashion. + * as used in AFT survival regression for samples in sparse or dense vector in an online fashion. * * The loss function and likelihood function under the AFT model based on: * Lawless, J. F., Statistical Models and Methods for Lifetime Data, @@ -325,74 +440,108 @@ class AFTSurvivalRegressionModel private[ml] ( * Two AFTAggregator can be merged together to have a summary of loss and gradient of * the corresponding joint dataset. * - * Given the values of the covariates x^{'}, for random lifetime t_{i} of subjects i = 1, ..., n, + * Given the values of the covariates $x^{'}$, for random lifetime $t_{i}$ of subjects i = 1,..,n, * with possible right-censoring, the likelihood function under the AFT model is given as - * {{{ - * L(\beta,\sigma)=\prod_{i=1}^n[\frac{1}{\sigma}f_{0} - * (\frac{\log{t_{i}}-x^{'}\beta}{\sigma})]^{\delta_{i}}S_{0} - * (\frac{\log{t_{i}}-x^{'}\beta}{\sigma})^{1-\delta_{i}} - * }}} - * Where \delta_{i} is the indicator of the event has occurred i.e. uncensored or not. - * Using \epsilon_{i}=\frac{\log{t_{i}}-x^{'}\beta}{\sigma}, the log-likelihood function + * + *
    + * $$ + * L(\beta,\sigma)=\prod_{i=1}^n[\frac{1}{\sigma}f_{0} + * (\frac{\log{t_{i}}-x^{'}\beta}{\sigma})]^{\delta_{i}}S_{0} + * (\frac{\log{t_{i}}-x^{'}\beta}{\sigma})^{1-\delta_{i}} + * $$ + *
    + * + * Where $\delta_{i}$ is the indicator of the event has occurred i.e. uncensored or not. + * Using $\epsilon_{i}=\frac{\log{t_{i}}-x^{'}\beta}{\sigma}$, the log-likelihood function * assumes the form - * {{{ - * \iota(\beta,\sigma)=\sum_{i=1}^{n}[-\delta_{i}\log\sigma+ - * \delta_{i}\log{f_{0}}(\epsilon_{i})+(1-\delta_{i})\log{S_{0}(\epsilon_{i})}] - * }}} - * Where S_{0}(\epsilon_{i}) is the baseline survivor function, - * and f_{0}(\epsilon_{i}) is corresponding density function. + * + *
    + * $$ + * \iota(\beta,\sigma)=\sum_{i=1}^{n}[-\delta_{i}\log\sigma+ + * \delta_{i}\log{f_{0}}(\epsilon_{i})+(1-\delta_{i})\log{S_{0}(\epsilon_{i})}] + * $$ + *
    + * Where $S_{0}(\epsilon_{i})$ is the baseline survivor function, + * and $f_{0}(\epsilon_{i})$ is corresponding density function. * * The most commonly used log-linear survival regression method is based on the Weibull * distribution of the survival time. The Weibull distribution for lifetime corresponding * to extreme value distribution for log of the lifetime, - * and the S_{0}(\epsilon) function is - * {{{ - * S_{0}(\epsilon_{i})=\exp(-e^{\epsilon_{i}}) - * }}} - * the f_{0}(\epsilon_{i}) function is - * {{{ - * f_{0}(\epsilon_{i})=e^{\epsilon_{i}}\exp(-e^{\epsilon_{i}}) - * }}} + * and the $S_{0}(\epsilon)$ function is + * + *
    + * $$ + * S_{0}(\epsilon_{i})=\exp(-e^{\epsilon_{i}}) + * $$ + *
    + * + * and the $f_{0}(\epsilon_{i})$ function is + * + *
    + * $$ + * f_{0}(\epsilon_{i})=e^{\epsilon_{i}}\exp(-e^{\epsilon_{i}}) + * $$ + *
    + * * The log-likelihood function for Weibull distribution of lifetime is - * {{{ - * \iota(\beta,\sigma)= - * -\sum_{i=1}^n[\delta_{i}\log\sigma-\delta_{i}\epsilon_{i}+e^{\epsilon_{i}}] - * }}} + * + *
    + * $$ + * \iota(\beta,\sigma)= + * -\sum_{i=1}^n[\delta_{i}\log\sigma-\delta_{i}\epsilon_{i}+e^{\epsilon_{i}}] + * $$ + *
    + * * Due to minimizing the negative log-likelihood equivalent to maximum a posteriori probability, - * the loss function we use to optimize is -\iota(\beta,\sigma). - * The gradient functions for \beta and \log\sigma respectively are - * {{{ - * \frac{\partial (-\iota)}{\partial \beta}= - * \sum_{1=1}^{n}[\delta_{i}-e^{\epsilon_{i}}]\frac{x_{i}}{\sigma} - * }}} - * {{{ - * \frac{\partial (-\iota)}{\partial (\log\sigma)}= - * \sum_{i=1}^{n}[\delta_{i}+(\delta_{i}-e^{\epsilon_{i}})\epsilon_{i}] - * }}} - * @param parameters including three part: The log of scale parameter, the intercept and - * regression coefficients corresponding to the features. + * the loss function we use to optimize is $-\iota(\beta,\sigma)$. + * The gradient functions for $\beta$ and $\log\sigma$ respectively are + * + *
    + * $$ + * \frac{\partial (-\iota)}{\partial \beta}= + * \sum_{1=1}^{n}[\delta_{i}-e^{\epsilon_{i}}]\frac{x_{i}}{\sigma} \\ + * + * \frac{\partial (-\iota)}{\partial (\log\sigma)}= + * \sum_{i=1}^{n}[\delta_{i}+(\delta_{i}-e^{\epsilon_{i}})\epsilon_{i}] + * $$ + *
    + * + * @param bcParameters The broadcasted value includes three part: The log of scale parameter, + * the intercept and regression coefficients corresponding to the features. * @param fitIntercept Whether to fit an intercept term. + * @param bcFeaturesStd The broadcast standard deviation values of the features. */ -private class AFTAggregator(parameters: BDV[Double], fitIntercept: Boolean) - extends Serializable { - - // beta is the intercept and regression coefficients to the covariates - private val beta = parameters.slice(1, parameters.length) +private class AFTAggregator( + bcParameters: Broadcast[BDV[Double]], + fitIntercept: Boolean, + bcFeaturesStd: Broadcast[Array[Double]]) extends Serializable { + + private val length = bcParameters.value.length + // make transient so we do not serialize between aggregation stages + @transient private lazy val parameters = bcParameters.value + // the regression coefficients to the covariates + @transient private lazy val coefficients = parameters.slice(2, length) + @transient private lazy val intercept = parameters(1) // sigma is the scale parameter of the AFT model - private val sigma = math.exp(parameters(0)) + @transient private lazy val sigma = math.exp(parameters(0)) private var totalCnt: Long = 0L private var lossSum = 0.0 - private var gradientBetaSum = BDV.zeros[Double](beta.length) - private var gradientLogSigmaSum = 0.0 + // Here we optimize loss function over log(sigma), intercept and coefficients + private lazy val gradientSumArray = Array.ofDim[Double](length) def count: Long = totalCnt + def loss: Double = { + require(totalCnt > 0.0, s"The number of instances should be " + + s"greater than 0.0, but got $totalCnt.") + lossSum / totalCnt + } + def gradient: BDV[Double] = { + require(totalCnt > 0.0, s"The number of instances should be " + + s"greater than 0.0, but got $totalCnt.") + new BDV(gradientSumArray.map(_ / totalCnt.toDouble)) + } - def loss: Double = if (totalCnt == 0) 1.0 else lossSum / totalCnt - - // Here we optimize loss function over beta and log(sigma) - def gradient: BDV[Double] = BDV.vertcat(BDV(Array(gradientLogSigmaSum / totalCnt.toDouble)), - gradientBetaSum/totalCnt.toDouble) /** * Add a new training data to this AFTAggregator, and update the loss and gradient @@ -402,26 +551,36 @@ private class AFTAggregator(parameters: BDV[Double], fitIntercept: Boolean) * @return This AFTAggregator object. */ def add(data: AFTPoint): this.type = { - - // TODO: Don't create a new xi vector each time. - val xi = if (fitIntercept) { - Vectors.dense(Array(1.0) ++ data.features.toArray).toBreeze - } else { - Vectors.dense(Array(0.0) ++ data.features.toArray).toBreeze - } + val xi = data.features val ti = data.label val delta = data.censor - val epsilon = (math.log(ti) - beta.dot(xi)) / sigma - lossSum += math.log(sigma) * delta - lossSum += (math.exp(epsilon) - delta * epsilon) + require(ti > 0.0, "The lifetime or label should be greater than 0.") - // Sanity check (should never occur): - assert(!lossSum.isInfinity, - s"AFTAggregator loss sum is infinity. Error for unknown reason.") + val localFeaturesStd = bcFeaturesStd.value - gradientBetaSum += xi * (delta - math.exp(epsilon)) / sigma - gradientLogSigmaSum += delta + (delta - math.exp(epsilon)) * epsilon + val margin = { + var sum = 0.0 + xi.foreachActive { (index, value) => + if (localFeaturesStd(index) != 0.0 && value != 0.0) { + sum += coefficients(index) * (value / localFeaturesStd(index)) + } + } + sum + intercept + } + val epsilon = (math.log(ti) - margin) / sigma + + lossSum += delta * math.log(sigma) - delta * epsilon + math.exp(epsilon) + + val multiplier = (delta - math.exp(epsilon)) / sigma + + gradientSumArray(0) += delta + multiplier * sigma * epsilon + gradientSumArray(1) += { if (fitIntercept) multiplier else 0.0 } + xi.foreachActive { (index, value) => + if (localFeaturesStd(index) != 0.0 && value != 0.0) { + gradientSumArray(index + 2) += multiplier * (value / localFeaturesStd(index)) + } + } totalCnt += 1 this @@ -436,12 +595,15 @@ private class AFTAggregator(parameters: BDV[Double], fitIntercept: Boolean) * @return This AFTAggregator object. */ def merge(other: AFTAggregator): this.type = { - if (totalCnt != 0) { + if (other.count != 0) { totalCnt += other.totalCnt lossSum += other.lossSum - gradientBetaSum += other.gradientBetaSum - gradientLogSigmaSum += other.gradientLogSigmaSum + var i = 0 + while (i < length) { + this.gradientSumArray(i) += other.gradientSumArray(i) + i += 1 + } } this } @@ -452,19 +614,26 @@ private class AFTAggregator(parameters: BDV[Double], fitIntercept: Boolean) * It returns the loss and gradient at a particular point (parameters). * It's used in Breeze's convex optimization routines. */ -private class AFTCostFun(data: RDD[AFTPoint], fitIntercept: Boolean) - extends DiffFunction[BDV[Double]] { +private class AFTCostFun( + data: RDD[AFTPoint], + fitIntercept: Boolean, + bcFeaturesStd: Broadcast[Array[Double]], + aggregationDepth: Int) extends DiffFunction[BDV[Double]] { override def calculate(parameters: BDV[Double]): (Double, BDV[Double]) = { - val aftAggregator = data.treeAggregate(new AFTAggregator(parameters, fitIntercept))( + val bcParameters = data.context.broadcast(parameters) + + val aftAggregator = data.treeAggregate( + new AFTAggregator(bcParameters, fitIntercept, bcFeaturesStd))( seqOp = (c, v) => (c, v) match { case (aggregator, instance) => aggregator.add(instance) }, combOp = (c1, c2) => (c1, c2) match { case (aggregator1, aggregator2) => aggregator1.merge(aggregator2) - }) + }, depth = aggregationDepth) + bcParameters.destroy(blocking = false) (aftAggregator.loss, aftAggregator.gradient) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index 04420fc6e825..01c5cc1c7efa 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -17,68 +17,117 @@ package org.apache.spark.ml.regression -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.hadoop.fs.Path +import org.json4s.{DefaultFormats, JObject} +import org.json4s.JsonDSL._ + +import org.apache.spark.annotation.Since import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, DecisionTreeParams, Node, TreeRegressorParams} +import org.apache.spark.ml.tree._ +import org.apache.spark.ml.tree.DecisionTreeModelReadWrite._ import org.apache.spark.ml.tree.impl.RandomForest -import org.apache.spark.ml.util.{Identifiable, MetadataUtils} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.util._ import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.functions._ + /** - * :: Experimental :: - * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] learning algorithm - * for regression. + * Decision tree + * learning algorithm for regression. * It supports both continuous and categorical features. */ @Since("1.4.0") -@Experimental -final class DecisionTreeRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) +class DecisionTreeRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Predictor[Vector, DecisionTreeRegressor, DecisionTreeRegressionModel] - with DecisionTreeParams with TreeRegressorParams { + with DecisionTreeRegressorParams with DefaultParamsWritable { @Since("1.4.0") def this() = this(Identifiable.randomUID("dtr")) // Override parameter setters from parent trait for Java API compatibility. + /** @group setParam */ @Since("1.4.0") - override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + override def setMaxDepth(value: Int): this.type = set(maxDepth, value) + /** @group setParam */ @Since("1.4.0") - override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + override def setMaxBins(value: Int): this.type = set(maxBins, value) + /** @group setParam */ @Since("1.4.0") - override def setMinInstancesPerNode(value: Int): this.type = - super.setMinInstancesPerNode(value) + override def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) + /** @group setParam */ @Since("1.4.0") - override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + override def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) + /** @group expertSetParam */ @Since("1.4.0") - override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + override def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) + /** @group expertSetParam */ @Since("1.4.0") - override def setCacheNodeIds(value: Boolean): this.type = super.setCacheNodeIds(value) + override def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) + /** + * Specifies how often to checkpoint the cached node IDs. + * E.g. 10 means that the cache will get checkpointed every 10 iterations. + * This is only used if cacheNodeIds is true and if the checkpoint directory is set in + * [[org.apache.spark.SparkContext]]. + * Must be at least 1. + * (default = 10) + * @group setParam + */ @Since("1.4.0") - override def setCheckpointInterval(value: Int): this.type = super.setCheckpointInterval(value) + override def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) + /** @group setParam */ @Since("1.4.0") - override def setImpurity(value: String): this.type = super.setImpurity(value) + override def setImpurity(value: String): this.type = set(impurity, value) + + /** @group setParam */ + @Since("1.6.0") + override def setSeed(value: Long): this.type = set(seed, value) - override protected def train(dataset: DataFrame): DecisionTreeRegressionModel = { + /** @group setParam */ + @Since("2.0.0") + def setVarianceCol(value: String): this.type = set(varianceCol, value) + + override protected def train(dataset: Dataset[_]): DecisionTreeRegressionModel = { val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) val strategy = getOldStrategy(categoricalFeatures) + + val instr = Instrumentation.create(this, oldDataset) + instr.logParams(params: _*) + val trees = RandomForest.run(oldDataset, strategy, numTrees = 1, featureSubsetStrategy = "all", - seed = 0L, parentUID = Some(uid)) - trees.head.asInstanceOf[DecisionTreeRegressionModel] + seed = $(seed), instr = Some(instr), parentUID = Some(uid)) + + val m = trees.head.asInstanceOf[DecisionTreeRegressionModel] + instr.logSuccess(m) + m + } + + /** (private[ml]) Train a decision tree on an RDD */ + private[ml] def train(data: RDD[LabeledPoint], + oldStrategy: OldStrategy): DecisionTreeRegressionModel = { + val instr = Instrumentation.create(this, data) + instr.logParams(params: _*) + + val trees = RandomForest.run(data, oldStrategy, numTrees = 1, featureSubsetStrategy = "all", + seed = $(seed), instr = Some(instr), parentUID = Some(uid)) + + val m = trees.head.asInstanceOf[DecisionTreeRegressionModel] + instr.logSuccess(m) + m } /** (private[ml]) Create a Strategy instance to use with the old API. */ @@ -92,29 +141,33 @@ final class DecisionTreeRegressor @Since("1.4.0") (@Since("1.4.0") override val } @Since("1.4.0") -@Experimental -object DecisionTreeRegressor { +object DecisionTreeRegressor extends DefaultParamsReadable[DecisionTreeRegressor] { /** Accessor for supported impurities: variance */ final val supportedImpurities: Array[String] = TreeRegressorParams.supportedImpurities + + @Since("2.0.0") + override def load(path: String): DecisionTreeRegressor = super.load(path) } /** - * :: Experimental :: - * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for regression. + * + * Decision tree (Wikipedia) model for regression. * It supports both continuous and categorical features. * @param rootNode Root of the decision tree */ @Since("1.4.0") -@Experimental -final class DecisionTreeRegressionModel private[ml] ( +class DecisionTreeRegressionModel private[ml] ( override val uid: String, override val rootNode: Node, override val numFeatures: Int) extends PredictionModel[Vector, DecisionTreeRegressionModel] - with DecisionTreeModel with Serializable { + with DecisionTreeModel with DecisionTreeRegressorParams with MLWritable with Serializable { + + /** @group setParam */ + def setVarianceCol(value: String): this.type = set(varianceCol, value) require(rootNode != null, - "DecisionTreeClassificationModel given null rootNode, but it requires a non-null rootNode.") + "DecisionTreeRegressionModel given null rootNode, but it requires a non-null rootNode.") /** * Construct a decision tree regression model. @@ -127,6 +180,30 @@ final class DecisionTreeRegressionModel private[ml] ( rootNode.predictImpl(features).prediction } + /** We need to update this function if we ever add other impurity measures. */ + protected def predictVariance(features: Vector): Double = { + rootNode.predictImpl(features).impurityStats.calculate() + } + + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) + transformImpl(dataset) + } + + override protected def transformImpl(dataset: Dataset[_]): DataFrame = { + val predictUDF = udf { (features: Vector) => predict(features) } + val predictVarianceUDF = udf { (features: Vector) => predictVariance(features) } + var output = dataset.toDF() + if ($(predictionCol).nonEmpty) { + output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) + } + if (isDefined(varianceCol) && $(varianceCol).nonEmpty) { + output = output.withColumn($(varianceCol), predictVarianceUDF(col($(featuresCol)))) + } + output + } + @Since("1.4.0") override def copy(extra: ParamMap): DecisionTreeRegressionModel = { copyValues(new DecisionTreeRegressionModel(uid, rootNode, numFeatures), extra).setParent(parent) @@ -137,16 +214,78 @@ final class DecisionTreeRegressionModel private[ml] ( s"DecisionTreeRegressionModel (uid=$uid) of depth $depth with $numNodes nodes" } - /** Convert to a model in the old API */ - private[ml] def toOld: OldDecisionTreeModel = { + /** + * Estimate of the importance of each feature. + * + * This generalizes the idea of "Gini" importance to other losses, + * following the explanation of Gini importance from "Random Forests" documentation + * by Leo Breiman and Adele Cutler, and following the implementation from scikit-learn. + * + * This feature importance is calculated as follows: + * - importance(feature j) = sum (over nodes which split on feature j) of the gain, + * where gain is scaled by the number of instances passing through node + * - Normalize importances for tree to sum to 1. + * + * @note Feature importance for single decision trees can have high variance due to + * correlated predictor variables. Consider using a [[RandomForestRegressor]] + * to determine feature importance instead. + */ + @Since("2.0.0") + lazy val featureImportances: Vector = TreeEnsembleModel.featureImportances(this, numFeatures) + + /** Convert to spark.mllib DecisionTreeModel (losing some information) */ + override private[spark] def toOld: OldDecisionTreeModel = { new OldDecisionTreeModel(rootNode.toOld(1), OldAlgo.Regression) } + + @Since("2.0.0") + override def write: MLWriter = + new DecisionTreeRegressionModel.DecisionTreeRegressionModelWriter(this) } -private[ml] object DecisionTreeRegressionModel { +@Since("2.0.0") +object DecisionTreeRegressionModel extends MLReadable[DecisionTreeRegressionModel] { + + @Since("2.0.0") + override def read: MLReader[DecisionTreeRegressionModel] = + new DecisionTreeRegressionModelReader + + @Since("2.0.0") + override def load(path: String): DecisionTreeRegressionModel = super.load(path) + + private[DecisionTreeRegressionModel] + class DecisionTreeRegressionModelWriter(instance: DecisionTreeRegressionModel) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val extraMetadata: JObject = Map( + "numFeatures" -> instance.numFeatures) + DefaultParamsWriter.saveMetadata(instance, path, sc, Some(extraMetadata)) + val (nodeData, _) = NodeData.build(instance.rootNode, 0) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(nodeData).write.parquet(dataPath) + } + } + + private class DecisionTreeRegressionModelReader + extends MLReader[DecisionTreeRegressionModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[DecisionTreeRegressionModel].getName + + override def load(path: String): DecisionTreeRegressionModel = { + implicit val format = DefaultFormats + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] + val root = loadTreeNodes(path, metadata, sparkSession) + val model = new DecisionTreeRegressionModel(metadata.uid, root, numFeatures) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } - /** (private[ml]) Convert a model from the old API */ - def fromOld( + /** Convert a model from the old API */ + private[ml] def fromOld( oldModel: OldDecisionTreeModel, parent: DecisionTreeRegressor, categoricalFeatures: Map[Int, Int], diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 07144cc7cfbd..08d175cb9444 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -18,35 +18,46 @@ package org.apache.spark.ml.regression import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.json4s.{DefaultFormats, JObject} +import org.json4s.JsonDSL._ -import org.apache.spark.Logging -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging import org.apache.spark.ml.{PredictionModel, Predictor} -import org.apache.spark.ml.param.{Param, ParamMap} -import org.apache.spark.ml.tree.{DecisionTreeModel, GBTParams, TreeEnsembleModel, TreeRegressorParams} -import org.apache.spark.ml.util.{Identifiable, MetadataUtils} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.{GradientBoostedTrees => OldGBT} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.tree._ +import org.apache.spark.ml.tree.impl.GradientBoostedTrees +import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} -import org.apache.spark.mllib.tree.loss.{AbsoluteError => OldAbsoluteError, Loss => OldLoss, SquaredError => OldSquaredError} import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.DoubleType /** - * :: Experimental :: - * [[http://en.wikipedia.org/wiki/Gradient_boosting Gradient-Boosted Trees (GBTs)]] + * Gradient-Boosted Trees (GBTs) * learning algorithm for regression. * It supports both continuous and categorical features. + * + * The implementation is based upon: J.H. Friedman. "Stochastic Gradient Boosting." 1999. + * + * Notes on Gradient Boosting vs. TreeBoost: + * - This implementation is for Stochastic Gradient Boosting, not for TreeBoost. + * - Both algorithms learn tree ensembles by minimizing loss functions. + * - TreeBoost (Friedman, 1999) additionally modifies the outputs at tree leaf nodes + * based on the loss function, whereas the original gradient boosting method does not. + * - When the loss is SquaredError, these methods give the same result, but they could differ + * for other loss functions. + * - We expect to implement TreeBoost in the future: + * [https://issues.apache.org/jira/browse/SPARK-4240] */ @Since("1.4.0") -@Experimental -final class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) +class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Predictor[Vector, GBTRegressor, GBTRegressionModel] - with GBTParams with TreeRegressorParams with Logging { + with GBTRegressorParams with DefaultParamsWritable with Logging { @Since("1.4.0") def this() = this(Identifiable.randomUID("gbtr")) @@ -54,31 +65,48 @@ final class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: Stri // Override parameter setters from parent trait for Java API compatibility. // Parameters from TreeRegressorParams: + + /** @group setParam */ @Since("1.4.0") - override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + override def setMaxDepth(value: Int): this.type = set(maxDepth, value) + /** @group setParam */ @Since("1.4.0") - override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + override def setMaxBins(value: Int): this.type = set(maxBins, value) + /** @group setParam */ @Since("1.4.0") - override def setMinInstancesPerNode(value: Int): this.type = - super.setMinInstancesPerNode(value) + override def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) + /** @group setParam */ @Since("1.4.0") - override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + override def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) + /** @group expertSetParam */ @Since("1.4.0") - override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + override def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) + /** @group expertSetParam */ @Since("1.4.0") - override def setCacheNodeIds(value: Boolean): this.type = super.setCacheNodeIds(value) + override def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) + /** + * Specifies how often to checkpoint the cached node IDs. + * E.g. 10 means that the cache will get checkpointed every 10 iterations. + * This is only used if cacheNodeIds is true and if the checkpoint directory is set in + * [[org.apache.spark.SparkContext]]. + * Must be at least 1. + * (default = 10) + * @group setParam + */ @Since("1.4.0") - override def setCheckpointInterval(value: Int): this.type = super.setCheckpointInterval(value) + override def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) /** * The impurity setting is ignored for GBT models. * Individual trees are built using impurity "Variance." + * + * @group setParam */ @Since("1.4.0") override def setImpurity(value: String): this.type = { @@ -87,66 +115,49 @@ final class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: Stri } // Parameters from TreeEnsembleParams: + + /** @group setParam */ @Since("1.4.0") - override def setSubsamplingRate(value: Double): this.type = super.setSubsamplingRate(value) + override def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) + /** @group setParam */ @Since("1.4.0") - override def setSeed(value: Long): this.type = { - logWarning("The 'seed' parameter is currently ignored by Gradient Boosting.") - super.setSeed(value) - } + override def setSeed(value: Long): this.type = set(seed, value) // Parameters from GBTParams: - @Since("1.4.0") - override def setMaxIter(value: Int): this.type = super.setMaxIter(value) + /** @group setParam */ @Since("1.4.0") - override def setStepSize(value: Double): this.type = super.setStepSize(value) + override def setMaxIter(value: Int): this.type = set(maxIter, value) - // Parameters for GBTRegressor: - - /** - * Loss function which GBT tries to minimize. (case-insensitive) - * Supported: "squared" (L2) and "absolute" (L1) - * (default = squared) - * @group param - */ + /** @group setParam */ @Since("1.4.0") - val lossType: Param[String] = new Param[String](this, "lossType", "Loss function which GBT" + - " tries to minimize (case-insensitive). Supported options:" + - s" ${GBTRegressor.supportedLossTypes.mkString(", ")}", - (value: String) => GBTRegressor.supportedLossTypes.contains(value.toLowerCase)) + override def setStepSize(value: Double): this.type = set(stepSize, value) - setDefault(lossType -> "squared") + // Parameters from GBTRegressorParams: /** @group setParam */ @Since("1.4.0") def setLossType(value: String): this.type = set(lossType, value) - /** @group getParam */ - @Since("1.4.0") - def getLossType: String = $(lossType).toLowerCase - - /** (private[ml]) Convert new loss to old loss. */ - override private[ml] def getOldLossType: OldLoss = { - getLossType match { - case "squared" => OldSquaredError - case "absolute" => OldAbsoluteError - case _ => - // Should never happen because of check in setter method. - throw new RuntimeException(s"GBTRegressorParams was given bad loss type: $getLossType") - } - } - - override protected def train(dataset: DataFrame): GBTRegressionModel = { + override protected def train(dataset: Dataset[_]): GBTRegressionModel = { val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) val numFeatures = oldDataset.first().features.size val boostingStrategy = super.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Regression) - val oldGBT = new OldGBT(boostingStrategy) - val oldModel = oldGBT.run(oldDataset) - GBTRegressionModel.fromOld(oldModel, this, categoricalFeatures, numFeatures) + + val instr = Instrumentation.create(this, oldDataset) + instr.logParams(labelCol, featuresCol, predictionCol, impurity, lossType, + maxDepth, maxBins, maxIter, maxMemoryInMB, minInfoGain, minInstancesPerNode, + seed, stepSize, subsamplingRate, cacheNodeIds, checkpointInterval) + instr.logNumFeatures(numFeatures) + + val (baseLearners, learnerWeights) = GradientBoostedTrees.run(oldDataset, boostingStrategy, + $(seed)) + val m = new GBTRegressionModel(uid, baseLearners, learnerWeights, numFeatures) + instr.logSuccess(m) + m } @Since("1.4.0") @@ -154,34 +165,34 @@ final class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: Stri } @Since("1.4.0") -@Experimental -object GBTRegressor { - // The losses below should be lowercase. +object GBTRegressor extends DefaultParamsReadable[GBTRegressor] { + /** Accessor for supported loss settings: squared (L2), absolute (L1) */ @Since("1.4.0") - final val supportedLossTypes: Array[String] = Array("squared", "absolute").map(_.toLowerCase) + final val supportedLossTypes: Array[String] = GBTRegressorParams.supportedLossTypes + + @Since("2.0.0") + override def load(path: String): GBTRegressor = super.load(path) } /** - * :: Experimental :: - * - * [[http://en.wikipedia.org/wiki/Gradient_boosting Gradient-Boosted Trees (GBTs)]] + * Gradient-Boosted Trees (GBTs) * model for regression. * It supports both continuous and categorical features. * @param _trees Decision trees in the ensemble. * @param _treeWeights Weights for the decision trees in the ensemble. */ @Since("1.4.0") -@Experimental -final class GBTRegressionModel private[ml]( +class GBTRegressionModel private[ml]( override val uid: String, private val _trees: Array[DecisionTreeRegressionModel], private val _treeWeights: Array[Double], override val numFeatures: Int) extends PredictionModel[Vector, GBTRegressionModel] - with TreeEnsembleModel with Serializable { + with GBTRegressorParams with TreeEnsembleModel[DecisionTreeRegressionModel] + with MLWritable with Serializable { - require(numTrees > 0, "GBTRegressionModel requires at least 1 tree.") + require(_trees.nonEmpty, "GBTRegressionModel requires at least 1 tree.") require(_trees.length == _treeWeights.length, "GBTRegressionModel given trees, treeWeights of" + s" non-matching lengths (${_trees.length}, ${_treeWeights.length}, respectively).") @@ -195,13 +206,19 @@ final class GBTRegressionModel private[ml]( this(uid, _trees, _treeWeights, -1) @Since("1.4.0") - override def trees: Array[DecisionTreeModel] = _trees.asInstanceOf[Array[DecisionTreeModel]] + override def trees: Array[DecisionTreeRegressionModel] = _trees + + /** + * Number of trees in ensemble + */ + @Since("2.0.0") + val getNumTrees: Int = trees.length @Since("1.4.0") override def treeWeights: Array[Double] = _treeWeights - override protected def transformImpl(dataset: DataFrame): DataFrame = { - val bcastModel = dataset.sqlContext.sparkContext.broadcast(this) + override protected def transformImpl(dataset: Dataset[_]): DataFrame = { + val bcastModel = dataset.sparkSession.sparkContext.broadcast(this) val predictUDF = udf { (features: Any) => bcastModel.value.predict(features.asInstanceOf[Vector]) } @@ -215,6 +232,9 @@ final class GBTRegressionModel private[ml]( blas.ddot(numTrees, treePredictions, 1, _treeWeights, 1) } + /** Number of trees in ensemble */ + val numTrees: Int = trees.length + @Since("1.4.0") override def copy(extra: ParamMap): GBTRegressionModel = { copyValues(new GBTRegressionModel(uid, _trees, _treeWeights, numFeatures), @@ -226,16 +246,81 @@ final class GBTRegressionModel private[ml]( s"GBTRegressionModel (uid=$uid) with $numTrees trees" } + /** + * Estimate of the importance of each feature. + * + * Each feature's importance is the average of its importance across all trees in the ensemble + * The importance vector is normalized to sum to 1. This method is suggested by Hastie et al. + * (Hastie, Tibshirani, Friedman. "The Elements of Statistical Learning, 2nd Edition." 2001.) + * and follows the implementation from scikit-learn. + * + * @see `DecisionTreeRegressionModel.featureImportances` + */ + @Since("2.0.0") + lazy val featureImportances: Vector = TreeEnsembleModel.featureImportances(trees, numFeatures) + /** (private[ml]) Convert to a model in the old API */ private[ml] def toOld: OldGBTModel = { new OldGBTModel(OldAlgo.Regression, _trees.map(_.toOld), _treeWeights) } + + @Since("2.0.0") + override def write: MLWriter = new GBTRegressionModel.GBTRegressionModelWriter(this) } -private[ml] object GBTRegressionModel { +@Since("2.0.0") +object GBTRegressionModel extends MLReadable[GBTRegressionModel] { + + @Since("2.0.0") + override def read: MLReader[GBTRegressionModel] = new GBTRegressionModelReader + + @Since("2.0.0") + override def load(path: String): GBTRegressionModel = super.load(path) + + private[GBTRegressionModel] + class GBTRegressionModelWriter(instance: GBTRegressionModel) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val extraMetadata: JObject = Map( + "numFeatures" -> instance.numFeatures, + "numTrees" -> instance.getNumTrees) + EnsembleModelReadWrite.saveImpl(instance, path, sparkSession, extraMetadata) + } + } + + private class GBTRegressionModelReader extends MLReader[GBTRegressionModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[GBTRegressionModel].getName + private val treeClassName = classOf[DecisionTreeRegressionModel].getName + + override def load(path: String): GBTRegressionModel = { + implicit val format = DefaultFormats + val (metadata: Metadata, treesData: Array[(Metadata, Node)], treeWeights: Array[Double]) = + EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName) + + val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] + val numTrees = (metadata.metadata \ "numTrees").extract[Int] + + val trees: Array[DecisionTreeRegressionModel] = treesData.map { + case (treeMetadata, root) => + val tree = + new DecisionTreeRegressionModel(treeMetadata.uid, root, numFeatures) + DefaultParamsReader.getAndSetParams(tree, treeMetadata) + tree + } + + require(numTrees == trees.length, s"GBTRegressionModel.load expected $numTrees" + + s" trees based on metadata but found ${trees.length} trees.") + + val model = new GBTRegressionModel(metadata.uid, trees, treeWeights, numFeatures) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } - /** (private[ml]) Convert a model from the old API */ - def fromOld( + /** Convert a model from the old API */ + private[ml] def fromOld( oldModel: OldGBTModel, parent: GBTRegressor, categoricalFeatures: Map[Int, Int], @@ -247,6 +332,6 @@ private[ml] object GBTRegressionModel { DecisionTreeRegressionModel.fromOld(tree, null, categoricalFeatures) } val uid = if (parent != null) parent.uid else Identifiable.randomUID("gbtr") - new GBTRegressionModel(parent.uid, newTrees, oldModel.treeWeights, numFeatures) + new GBTRegressionModel(uid, newTrees, oldModel.treeWeights, numFeatures) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala new file mode 100644 index 000000000000..917a4d238d46 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -0,0 +1,1570 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.regression + +import java.util.Locale + +import breeze.stats.{distributions => dist} +import org.apache.commons.lang3.StringUtils +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging +import org.apache.spark.ml.PredictorParams +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.feature.{Instance, OffsetInstance} +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} +import org.apache.spark.ml.optim._ +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util._ +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{DataType, DoubleType, StructType} + +/** + * Params for Generalized Linear Regression. + */ +private[regression] trait GeneralizedLinearRegressionBase extends PredictorParams + with HasFitIntercept with HasMaxIter with HasTol with HasRegParam with HasWeightCol + with HasSolver with Logging { + + import GeneralizedLinearRegression._ + + /** + * Param for the name of family which is a description of the error distribution + * to be used in the model. + * Supported options: "gaussian", "binomial", "poisson", "gamma" and "tweedie". + * Default is "gaussian". + * + * @group param + */ + @Since("2.0.0") + final val family: Param[String] = new Param(this, "family", + "The name of family which is a description of the error distribution to be used in the " + + s"model. Supported options: ${supportedFamilyNames.mkString(", ")}.", + (value: String) => supportedFamilyNames.contains(value.toLowerCase(Locale.ROOT))) + + /** @group getParam */ + @Since("2.0.0") + def getFamily: String = $(family) + + /** + * Param for the power in the variance function of the Tweedie distribution which provides + * the relationship between the variance and mean of the distribution. + * Only applicable to the Tweedie family. + * (see + * Tweedie Distribution (Wikipedia)) + * Supported values: 0 and [1, Inf). + * Note that variance power 0, 1, or 2 corresponds to the Gaussian, Poisson or Gamma + * family, respectively. + * + * @group param + */ + @Since("2.2.0") + final val variancePower: DoubleParam = new DoubleParam(this, "variancePower", + "The power in the variance function of the Tweedie distribution which characterizes " + + "the relationship between the variance and mean of the distribution. " + + "Only applicable to the Tweedie family. Supported values: 0 and [1, Inf).", + (x: Double) => x >= 1.0 || x == 0.0) + + /** @group getParam */ + @Since("2.2.0") + def getVariancePower: Double = $(variancePower) + + /** + * Param for the name of link function which provides the relationship + * between the linear predictor and the mean of the distribution function. + * Supported options: "identity", "log", "inverse", "logit", "probit", "cloglog" and "sqrt". + * This is used only when family is not "tweedie". The link function for the "tweedie" family + * must be specified through [[linkPower]]. + * + * @group param + */ + @Since("2.0.0") + final val link: Param[String] = new Param(this, "link", "The name of link function " + + "which provides the relationship between the linear predictor and the mean of the " + + s"distribution function. Supported options: ${supportedLinkNames.mkString(", ")}", + (value: String) => supportedLinkNames.contains(value.toLowerCase(Locale.ROOT))) + + /** @group getParam */ + @Since("2.0.0") + def getLink: String = $(link) + + /** + * Param for the index in the power link function. Only applicable to the Tweedie family. + * Note that link power 0, 1, -1 or 0.5 corresponds to the Log, Identity, Inverse or Sqrt + * link, respectively. + * When not set, this value defaults to 1 - [[variancePower]], which matches the R "statmod" + * package. + * + * @group param + */ + @Since("2.2.0") + final val linkPower: DoubleParam = new DoubleParam(this, "linkPower", + "The index in the power link function. Only applicable to the Tweedie family.") + + /** @group getParam */ + @Since("2.2.0") + def getLinkPower: Double = $(linkPower) + + /** + * Param for link prediction (linear predictor) column name. + * Default is not set, which means we do not output link prediction. + * + * @group param + */ + @Since("2.0.0") + final val linkPredictionCol: Param[String] = new Param[String](this, "linkPredictionCol", + "link prediction (linear predictor) column name") + + /** @group getParam */ + @Since("2.0.0") + def getLinkPredictionCol: String = $(linkPredictionCol) + + /** + * Param for offset column name. If this is not set or empty, we treat all instance offsets + * as 0.0. The feature specified as offset has a constant coefficient of 1.0. + * + * @group param + */ + @Since("2.3.0") + final val offsetCol: Param[String] = new Param[String](this, "offsetCol", "The offset " + + "column name. If this is not set or empty, we treat all instance offsets as 0.0") + + /** @group getParam */ + @Since("2.3.0") + def getOffsetCol: String = $(offsetCol) + + /** Checks whether weight column is set and nonempty. */ + private[regression] def hasWeightCol: Boolean = + isSet(weightCol) && $(weightCol).nonEmpty + + /** Checks whether offset column is set and nonempty. */ + private[regression] def hasOffsetCol: Boolean = + isSet(offsetCol) && $(offsetCol).nonEmpty + + /** Checks whether we should output link prediction. */ + private[regression] def hasLinkPredictionCol: Boolean = { + isDefined(linkPredictionCol) && $(linkPredictionCol).nonEmpty + } + + /** + * The solver algorithm for optimization. + * Supported options: "irls" (iteratively reweighted least squares). + * Default: "irls" + * + * @group param + */ + @Since("2.0.0") + final override val solver: Param[String] = new Param[String](this, "solver", + "The solver algorithm for optimization. Supported options: " + + s"${supportedSolvers.mkString(", ")}. (Default irls)", + ParamValidators.inArray[String](supportedSolvers)) + + @Since("2.0.0") + override def validateAndTransformSchema( + schema: StructType, + fitting: Boolean, + featuresDataType: DataType): StructType = { + if ($(family).toLowerCase(Locale.ROOT) == "tweedie") { + if (isSet(link)) { + logWarning("When family is tweedie, use param linkPower to specify link function. " + + "Setting param link will take no effect.") + } + } else { + if (isSet(variancePower)) { + logWarning("When family is not tweedie, setting param variancePower will take no effect.") + } + if (isSet(linkPower)) { + logWarning("When family is not tweedie, use param link to specify link function. " + + "Setting param linkPower will take no effect.") + } + if (isSet(link)) { + require(supportedFamilyAndLinkPairs.contains( + Family.fromParams(this) -> Link.fromParams(this)), + s"Generalized Linear Regression with ${$(family)} family " + + s"does not support ${$(link)} link function.") + } + } + + val newSchema = super.validateAndTransformSchema(schema, fitting, featuresDataType) + + if (hasOffsetCol) { + SchemaUtils.checkNumericType(schema, $(offsetCol)) + } + + if (hasLinkPredictionCol) { + SchemaUtils.appendColumn(newSchema, $(linkPredictionCol), DoubleType) + } else { + newSchema + } + } +} + +/** + * :: Experimental :: + * + * Fit a Generalized Linear Model + * (see + * Generalized linear model (Wikipedia)) + * specified by giving a symbolic description of the linear + * predictor (link function) and a description of the error distribution (family). + * It supports "gaussian", "binomial", "poisson", "gamma" and "tweedie" as family. + * Valid link functions for each family is listed below. The first link function of each family + * is the default one. + * - "gaussian" : "identity", "log", "inverse" + * - "binomial" : "logit", "probit", "cloglog" + * - "poisson" : "log", "identity", "sqrt" + * - "gamma" : "inverse", "identity", "log" + * - "tweedie" : power link function specified through "linkPower". The default link power in + * the tweedie family is 1 - variancePower. + */ +@Experimental +@Since("2.0.0") +class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val uid: String) + extends Regressor[Vector, GeneralizedLinearRegression, GeneralizedLinearRegressionModel] + with GeneralizedLinearRegressionBase with DefaultParamsWritable with Logging { + + import GeneralizedLinearRegression._ + + @Since("2.0.0") + def this() = this(Identifiable.randomUID("glm")) + + /** + * Sets the value of param [[family]]. + * Default is "gaussian". + * + * @group setParam + */ + @Since("2.0.0") + def setFamily(value: String): this.type = set(family, value) + setDefault(family -> Gaussian.name) + + /** + * Sets the value of param [[variancePower]]. + * Used only when family is "tweedie". + * Default is 0.0, which corresponds to the "gaussian" family. + * + * @group setParam + */ + @Since("2.2.0") + def setVariancePower(value: Double): this.type = set(variancePower, value) + setDefault(variancePower -> 0.0) + + /** + * Sets the value of param [[linkPower]]. + * Used only when family is "tweedie". + * + * @group setParam + */ + @Since("2.2.0") + def setLinkPower(value: Double): this.type = set(linkPower, value) + + /** + * Sets the value of param [[link]]. + * Used only when family is not "tweedie". + * + * @group setParam + */ + @Since("2.0.0") + def setLink(value: String): this.type = set(link, value) + + /** + * Sets if we should fit the intercept. + * Default is true. + * + * @group setParam + */ + @Since("2.0.0") + def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value) + + /** + * Sets the maximum number of iterations (applicable for solver "irls"). + * Default is 25. + * + * @group setParam + */ + @Since("2.0.0") + def setMaxIter(value: Int): this.type = set(maxIter, value) + setDefault(maxIter -> 25) + + /** + * Sets the convergence tolerance of iterations. + * Smaller value will lead to higher accuracy with the cost of more iterations. + * Default is 1E-6. + * + * @group setParam + */ + @Since("2.0.0") + def setTol(value: Double): this.type = set(tol, value) + setDefault(tol -> 1E-6) + + /** + * Sets the regularization parameter for L2 regularization. + * The regularization term is + *
    + * $$ + * 0.5 * regParam * L2norm(coefficients)^2 + * $$ + *
    + * Default is 0.0. + * + * @group setParam + */ + @Since("2.0.0") + def setRegParam(value: Double): this.type = set(regParam, value) + setDefault(regParam -> 0.0) + + /** + * Sets the value of param [[weightCol]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * Default is not set, so all instances have weight one. + * In the Binomial family, weights correspond to number of trials and should be integer. + * Non-integer weights are rounded to integer in AIC calculation. + * + * @group setParam + */ + @Since("2.0.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + + /** + * Sets the value of param [[offsetCol]]. + * If this is not set or empty, we treat all instance offsets as 0.0. + * Default is not set, so all instances have offset 0.0. + * + * @group setParam + */ + @Since("2.3.0") + def setOffsetCol(value: String): this.type = set(offsetCol, value) + + /** + * Sets the solver algorithm used for optimization. + * Currently only supports "irls" which is also the default solver. + * + * @group setParam + */ + @Since("2.0.0") + def setSolver(value: String): this.type = set(solver, value) + setDefault(solver -> IRLS) + + /** + * Sets the link prediction (linear predictor) column name. + * + * @group setParam + */ + @Since("2.0.0") + def setLinkPredictionCol(value: String): this.type = set(linkPredictionCol, value) + + override protected def train(dataset: Dataset[_]): GeneralizedLinearRegressionModel = { + val familyAndLink = FamilyAndLink(this) + + val numFeatures = dataset.select(col($(featuresCol))).first().getAs[Vector](0).size + val instr = Instrumentation.create(this, dataset) + instr.logParams(labelCol, featuresCol, weightCol, offsetCol, predictionCol, linkPredictionCol, + family, solver, fitIntercept, link, maxIter, regParam, tol) + instr.logNumFeatures(numFeatures) + + if (numFeatures > WeightedLeastSquares.MAX_NUM_FEATURES) { + val msg = "Currently, GeneralizedLinearRegression only supports number of features" + + s" <= ${WeightedLeastSquares.MAX_NUM_FEATURES}. Found $numFeatures in the input dataset." + throw new SparkException(msg) + } + + require(numFeatures > 0 || $(fitIntercept), + "GeneralizedLinearRegression was given data with 0 features, and with Param fitIntercept " + + "set to false. To fit a model with 0 features, fitIntercept must be set to true." ) + + val w = if (!hasWeightCol) lit(1.0) else col($(weightCol)) + val offset = if (!hasOffsetCol) lit(0.0) else col($(offsetCol)).cast(DoubleType) + + val model = if (familyAndLink.family == Gaussian && familyAndLink.link == Identity) { + // TODO: Make standardizeFeatures and standardizeLabel configurable. + val instances: RDD[Instance] = + dataset.select(col($(labelCol)), w, offset, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, offset: Double, features: Vector) => + Instance(label - offset, weight, features) + } + val optimizer = new WeightedLeastSquares($(fitIntercept), $(regParam), elasticNetParam = 0.0, + standardizeFeatures = true, standardizeLabel = true) + val wlsModel = optimizer.fit(instances) + val model = copyValues( + new GeneralizedLinearRegressionModel(uid, wlsModel.coefficients, wlsModel.intercept) + .setParent(this)) + val trainingSummary = new GeneralizedLinearRegressionTrainingSummary(dataset, model, + wlsModel.diagInvAtWA.toArray, 1, getSolver) + model.setSummary(Some(trainingSummary)) + } else { + val instances: RDD[OffsetInstance] = + dataset.select(col($(labelCol)), w, offset, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, offset: Double, features: Vector) => + OffsetInstance(label, weight, offset, features) + } + // Fit Generalized Linear Model by iteratively reweighted least squares (IRLS). + val initialModel = familyAndLink.initialize(instances, $(fitIntercept), $(regParam)) + val optimizer = new IterativelyReweightedLeastSquares(initialModel, + familyAndLink.reweightFunc, $(fitIntercept), $(regParam), $(maxIter), $(tol)) + val irlsModel = optimizer.fit(instances) + val model = copyValues( + new GeneralizedLinearRegressionModel(uid, irlsModel.coefficients, irlsModel.intercept) + .setParent(this)) + val trainingSummary = new GeneralizedLinearRegressionTrainingSummary(dataset, model, + irlsModel.diagInvAtWA.toArray, irlsModel.numIterations, getSolver) + model.setSummary(Some(trainingSummary)) + } + + instr.logSuccess(model) + model + } + + @Since("2.0.0") + override def copy(extra: ParamMap): GeneralizedLinearRegression = defaultCopy(extra) +} + +@Since("2.0.0") +object GeneralizedLinearRegression extends DefaultParamsReadable[GeneralizedLinearRegression] { + + @Since("2.0.0") + override def load(path: String): GeneralizedLinearRegression = super.load(path) + + /** + * Set of family (except for tweedie) and link pairs that GeneralizedLinearRegression supports. + * The link function of the Tweedie family is specified through param linkPower. + */ + private[regression] lazy val supportedFamilyAndLinkPairs = Set( + Gaussian -> Identity, Gaussian -> Log, Gaussian -> Inverse, + Binomial -> Logit, Binomial -> Probit, Binomial -> CLogLog, + Poisson -> Log, Poisson -> Identity, Poisson -> Sqrt, + Gamma -> Inverse, Gamma -> Identity, Gamma -> Log + ) + + /** String name for "irls" (iteratively reweighted least squares) solver. */ + private[regression] val IRLS = "irls" + + /** Set of solvers that GeneralizedLinearRegression supports. */ + private[regression] val supportedSolvers = Array(IRLS) + + /** Set of family names that GeneralizedLinearRegression supports. */ + private[regression] lazy val supportedFamilyNames = + supportedFamilyAndLinkPairs.map(_._1.name).toArray :+ "tweedie" + + /** Set of link names that GeneralizedLinearRegression supports. */ + private[regression] lazy val supportedLinkNames = + supportedFamilyAndLinkPairs.map(_._2.name).toArray + + private[regression] val epsilon: Double = 1E-16 + + /** + * Wrapper of family and link combination used in the model. + */ + private[regression] class FamilyAndLink(val family: Family, val link: Link) extends Serializable { + + /** Linear predictor based on given mu. */ + def predict(mu: Double): Double = link.link(family.project(mu)) + + /** Fitted value based on linear predictor eta. */ + def fitted(eta: Double): Double = family.project(link.unlink(eta)) + + /** + * Get the initial guess model for [[IterativelyReweightedLeastSquares]]. + */ + def initialize( + instances: RDD[OffsetInstance], + fitIntercept: Boolean, + regParam: Double): WeightedLeastSquaresModel = { + val newInstances = instances.map { instance => + val mu = family.initialize(instance.label, instance.weight) + val eta = predict(mu) - instance.offset + Instance(eta, instance.weight, instance.features) + } + // TODO: Make standardizeFeatures and standardizeLabel configurable. + val initialModel = new WeightedLeastSquares(fitIntercept, regParam, elasticNetParam = 0.0, + standardizeFeatures = true, standardizeLabel = true) + .fit(newInstances) + initialModel + } + + /** + * The reweight function used to update working labels and weights + * at each iteration of [[IterativelyReweightedLeastSquares]]. + */ + val reweightFunc: (OffsetInstance, WeightedLeastSquaresModel) => (Double, Double) = { + (instance: OffsetInstance, model: WeightedLeastSquaresModel) => { + val eta = model.predict(instance.features) + instance.offset + val mu = fitted(eta) + val newLabel = eta - instance.offset + (instance.label - mu) * link.deriv(mu) + val newWeight = instance.weight / (math.pow(this.link.deriv(mu), 2.0) * family.variance(mu)) + (newLabel, newWeight) + } + } + } + + private[regression] object FamilyAndLink { + + /** + * Constructs the FamilyAndLink object from a parameter map + */ + def apply(params: GeneralizedLinearRegressionBase): FamilyAndLink = { + val familyObj = Family.fromParams(params) + val linkObj = + if ((params.getFamily.toLowerCase(Locale.ROOT) != "tweedie" && + params.isSet(params.link)) || + (params.getFamily.toLowerCase(Locale.ROOT) == "tweedie" && + params.isSet(params.linkPower))) { + Link.fromParams(params) + } else { + familyObj.defaultLink + } + new FamilyAndLink(familyObj, linkObj) + } + } + + /** + * A description of the error distribution to be used in the model. + * + * @param name the name of the family. + */ + private[regression] abstract class Family(val name: String) extends Serializable { + + /** The default link instance of this family. */ + val defaultLink: Link + + /** Initialize the starting value for mu. */ + def initialize(y: Double, weight: Double): Double + + /** The variance of the endogenous variable's mean, given the value mu. */ + def variance(mu: Double): Double + + /** Deviance of (y, mu) pair. */ + def deviance(y: Double, mu: Double, weight: Double): Double + + /** + * Akaike Information Criterion (AIC) value of the family for a given dataset. + * + * @param predictions an RDD of (y, mu, weight) of instances in evaluation dataset + * @param deviance the deviance for the fitted model in evaluation dataset + * @param numInstances number of instances in evaluation dataset + * @param weightSum weights sum of instances in evaluation dataset + */ + def aic( + predictions: RDD[(Double, Double, Double)], + deviance: Double, + numInstances: Double, + weightSum: Double): Double + + /** Trim the fitted value so that it will be in valid range. */ + def project(mu: Double): Double = mu + } + + private[regression] object Family { + + /** + * Gets the [[Family]] object based on param family and variancePower. + * If param family is set with "gaussian", "binomial", "poisson" or "gamma", + * return the corresponding object directly; otherwise, construct a Tweedie object + * according to variancePower. + * + * @param params the parameter map containing family name and variance power + */ + def fromParams(params: GeneralizedLinearRegressionBase): Family = { + params.getFamily.toLowerCase(Locale.ROOT) match { + case Gaussian.name => Gaussian + case Binomial.name => Binomial + case Poisson.name => Poisson + case Gamma.name => Gamma + case "tweedie" => + params.getVariancePower match { + case 0.0 => Gaussian + case 1.0 => Poisson + case 2.0 => Gamma + case others => new Tweedie(others) + } + } + } + } + + /** + * Tweedie exponential family distribution. + * This includes the special cases of Gaussian, Poisson and Gamma. + */ + private[regression] class Tweedie(val variancePower: Double) + extends Family("tweedie") { + + override val defaultLink: Link = new Power(1.0 - variancePower) + + override def initialize(y: Double, weight: Double): Double = { + if (variancePower >= 1.0 && variancePower < 2.0) { + require(y >= 0.0, s"The response variable of $name($variancePower) family " + + s"should be non-negative, but got $y") + } else if (variancePower >= 2.0) { + require(y > 0.0, s"The response variable of $name($variancePower) family " + + s"should be positive, but got $y") + } + if (y == 0) Tweedie.delta else y + } + + override def variance(mu: Double): Double = math.pow(mu, variancePower) + + private def yp(y: Double, mu: Double, p: Double): Double = { + if (p == 0) { + math.log(y / mu) + } else { + (math.pow(y, p) - math.pow(mu, p)) / p + } + } + + override def deviance(y: Double, mu: Double, weight: Double): Double = { + // Force y >= delta for Poisson or compound Poisson + val y1 = if (variancePower >= 1.0 && variancePower < 2.0) { + math.max(y, Tweedie.delta) + } else { + y + } + 2.0 * weight * + (y * yp(y1, mu, 1.0 - variancePower) - yp(y, mu, 2.0 - variancePower)) + } + + override def aic( + predictions: RDD[(Double, Double, Double)], + deviance: Double, + numInstances: Double, + weightSum: Double): Double = { + /* + This depends on the density of the Tweedie distribution. + Only implemented for Gaussian, Poisson and Gamma at this point. + */ + throw new UnsupportedOperationException("No AIC available for the tweedie family") + } + + override def project(mu: Double): Double = { + if (mu < epsilon) { + epsilon + } else if (mu.isInfinity) { + Double.MaxValue + } else { + mu + } + } + } + + private[regression] object Tweedie{ + + /** Constant used in initialization and deviance to avoid numerical issues. */ + val delta: Double = 0.1 + } + + /** + * Gaussian exponential family distribution. + * The default link for the Gaussian family is the identity link. + */ + private[regression] object Gaussian extends Tweedie(0.0) { + + override val name: String = "gaussian" + + override val defaultLink: Link = Identity + + override def initialize(y: Double, weight: Double): Double = y + + override def variance(mu: Double): Double = 1.0 + + override def deviance(y: Double, mu: Double, weight: Double): Double = { + weight * (y - mu) * (y - mu) + } + + override def aic( + predictions: RDD[(Double, Double, Double)], + deviance: Double, + numInstances: Double, + weightSum: Double): Double = { + val wt = predictions.map(x => math.log(x._3)).sum() + numInstances * (math.log(deviance / numInstances * 2.0 * math.Pi) + 1.0) + 2.0 - wt + } + + override def project(mu: Double): Double = { + if (mu.isNegInfinity) { + Double.MinValue + } else if (mu.isPosInfinity) { + Double.MaxValue + } else { + mu + } + } + } + + /** + * Binomial exponential family distribution. + * The default link for the Binomial family is the logit link. + */ + private[regression] object Binomial extends Family("binomial") { + + val defaultLink: Link = Logit + + override def initialize(y: Double, weight: Double): Double = { + val mu = (weight * y + 0.5) / (weight + 1.0) + require(mu > 0.0 && mu < 1.0, "The response variable of Binomial family" + + s"should be in range (0, 1), but got $mu") + mu + } + + override def variance(mu: Double): Double = mu * (1.0 - mu) + + private def ylogy(y: Double, mu: Double): Double = { + if (y == 0) 0.0 else y * math.log(y / mu) + } + + override def deviance(y: Double, mu: Double, weight: Double): Double = { + 2.0 * weight * (ylogy(y, mu) + ylogy(1.0 - y, 1.0 - mu)) + } + + override def aic( + predictions: RDD[(Double, Double, Double)], + deviance: Double, + numInstances: Double, + weightSum: Double): Double = { + -2.0 * predictions.map { case (y: Double, mu: Double, weight: Double) => + // weights for Binomial distribution correspond to number of trials + val wt = math.round(weight).toInt + if (wt == 0) { + 0.0 + } else { + dist.Binomial(wt, mu).logProbabilityOf(math.round(y * weight).toInt) + } + }.sum() + } + + override def project(mu: Double): Double = { + if (mu < epsilon) { + epsilon + } else if (mu > 1.0 - epsilon) { + 1.0 - epsilon + } else { + mu + } + } + } + + /** + * Poisson exponential family distribution. + * The default link for the Poisson family is the log link. + */ + private[regression] object Poisson extends Tweedie(1.0) { + + override val name: String = "poisson" + + override val defaultLink: Link = Log + + override def initialize(y: Double, weight: Double): Double = { + require(y >= 0.0, "The response variable of Poisson family " + + s"should be non-negative, but got $y") + /* + Force Poisson mean > 0 to avoid numerical instability in IRLS. + R uses y + delta for initialization. See poisson()$initialize. + */ + math.max(y, Tweedie.delta) + } + + override def variance(mu: Double): Double = mu + + override def deviance(y: Double, mu: Double, weight: Double): Double = { + 2.0 * weight * (y * math.log(y / mu) - (y - mu)) + } + + override def aic( + predictions: RDD[(Double, Double, Double)], + deviance: Double, + numInstances: Double, + weightSum: Double): Double = { + -2.0 * predictions.map { case (y: Double, mu: Double, weight: Double) => + weight * dist.Poisson(mu).logProbabilityOf(y.toInt) + }.sum() + } + } + + /** + * Gamma exponential family distribution. + * The default link for the Gamma family is the inverse link. + */ + private[regression] object Gamma extends Tweedie(2.0) { + + override val name: String = "gamma" + + override val defaultLink: Link = Inverse + + override def initialize(y: Double, weight: Double): Double = { + require(y > 0.0, "The response variable of Gamma family " + + s"should be positive, but got $y") + y + } + + override def variance(mu: Double): Double = mu * mu + + override def deviance(y: Double, mu: Double, weight: Double): Double = { + -2.0 * weight * (math.log(y / mu) - (y - mu)/mu) + } + + override def aic( + predictions: RDD[(Double, Double, Double)], + deviance: Double, + numInstances: Double, + weightSum: Double): Double = { + val disp = deviance / weightSum + -2.0 * predictions.map { case (y: Double, mu: Double, weight: Double) => + weight * dist.Gamma(1.0 / disp, mu * disp).logPdf(y) + }.sum() + 2.0 + } + } + + /** + * A description of the link function to be used in the model. + * The link function provides the relationship between the linear predictor + * and the mean of the distribution function. + * + * @param name the name of link function. + */ + private[regression] abstract class Link(val name: String) extends Serializable { + + /** The link function. */ + def link(mu: Double): Double + + /** Derivative of the link function. */ + def deriv(mu: Double): Double + + /** The inverse link function. */ + def unlink(eta: Double): Double + } + + private[regression] object Link { + + /** + * Gets the [[Link]] object based on param family, link and linkPower. + * If param family is set with "tweedie", return or construct link function object + * according to linkPower; otherwise, return link function object according to link. + * + * @param params the parameter map containing family, link and linkPower + */ + def fromParams(params: GeneralizedLinearRegressionBase): Link = { + if (params.getFamily.toLowerCase(Locale.ROOT) == "tweedie") { + params.getLinkPower match { + case 0.0 => Log + case 1.0 => Identity + case -1.0 => Inverse + case 0.5 => Sqrt + case others => new Power(others) + } + } else { + params.getLink.toLowerCase(Locale.ROOT) match { + case Identity.name => Identity + case Logit.name => Logit + case Log.name => Log + case Inverse.name => Inverse + case Probit.name => Probit + case CLogLog.name => CLogLog + case Sqrt.name => Sqrt + } + } + } + } + + /** Power link function class */ + private[regression] class Power(val linkPower: Double) + extends Link("power") { + + override def link(mu: Double): Double = { + if (linkPower == 0.0) { + math.log(mu) + } else { + math.pow(mu, linkPower) + } + } + + override def deriv(mu: Double): Double = { + if (linkPower == 0.0) { + 1.0 / mu + } else { + linkPower * math.pow(mu, linkPower - 1.0) + } + } + + override def unlink(eta: Double): Double = { + if (linkPower == 0.0) { + math.exp(eta) + } else { + math.pow(eta, 1.0 / linkPower) + } + } + } + + private[regression] object Identity extends Power(1.0) { + + override val name: String = "identity" + + override def link(mu: Double): Double = mu + + override def deriv(mu: Double): Double = 1.0 + + override def unlink(eta: Double): Double = eta + } + + private[regression] object Logit extends Link("logit") { + + override def link(mu: Double): Double = math.log(mu / (1.0 - mu)) + + override def deriv(mu: Double): Double = 1.0 / (mu * (1.0 - mu)) + + override def unlink(eta: Double): Double = 1.0 / (1.0 + math.exp(-1.0 * eta)) + } + + private[regression] object Log extends Power(0.0) { + + override val name: String = "log" + + override def link(mu: Double): Double = math.log(mu) + + override def deriv(mu: Double): Double = 1.0 / mu + + override def unlink(eta: Double): Double = math.exp(eta) + } + + private[regression] object Inverse extends Power(-1.0) { + + override val name: String = "inverse" + + override def link(mu: Double): Double = 1.0 / mu + + override def deriv(mu: Double): Double = -1.0 * math.pow(mu, -2.0) + + override def unlink(eta: Double): Double = 1.0 / eta + } + + private[regression] object Probit extends Link("probit") { + + override def link(mu: Double): Double = dist.Gaussian(0.0, 1.0).inverseCdf(mu) + + override def deriv(mu: Double): Double = { + 1.0 / dist.Gaussian(0.0, 1.0).pdf(dist.Gaussian(0.0, 1.0).inverseCdf(mu)) + } + + override def unlink(eta: Double): Double = dist.Gaussian(0.0, 1.0).cdf(eta) + } + + private[regression] object CLogLog extends Link("cloglog") { + + override def link(mu: Double): Double = math.log(-1.0 * math.log(1 - mu)) + + override def deriv(mu: Double): Double = 1.0 / ((mu - 1.0) * math.log(1.0 - mu)) + + override def unlink(eta: Double): Double = 1.0 - math.exp(-1.0 * math.exp(eta)) + } + + private[regression] object Sqrt extends Power(0.5) { + + override val name: String = "sqrt" + + override def link(mu: Double): Double = math.sqrt(mu) + + override def deriv(mu: Double): Double = 1.0 / (2.0 * math.sqrt(mu)) + + override def unlink(eta: Double): Double = eta * eta + } +} + +/** + * :: Experimental :: + * Model produced by [[GeneralizedLinearRegression]]. + */ +@Experimental +@Since("2.0.0") +class GeneralizedLinearRegressionModel private[ml] ( + @Since("2.0.0") override val uid: String, + @Since("2.0.0") val coefficients: Vector, + @Since("2.0.0") val intercept: Double) + extends RegressionModel[Vector, GeneralizedLinearRegressionModel] + with GeneralizedLinearRegressionBase with MLWritable { + + /** + * Sets the link prediction (linear predictor) column name. + * + * @group setParam + */ + @Since("2.0.0") + def setLinkPredictionCol(value: String): this.type = set(linkPredictionCol, value) + + import GeneralizedLinearRegression._ + + private lazy val familyAndLink = FamilyAndLink(this) + + override protected def predict(features: Vector): Double = { + predict(features, 0.0) + } + + /** + * Calculates the predicted value when offset is set. + */ + private def predict(features: Vector, offset: Double): Double = { + val eta = predictLink(features, offset) + familyAndLink.fitted(eta) + } + + /** + * Calculates the link prediction (linear predictor) of the given instance. + */ + private def predictLink(features: Vector, offset: Double): Double = { + BLAS.dot(features, coefficients) + intercept + offset + } + + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema) + transformImpl(dataset) + } + + override protected def transformImpl(dataset: Dataset[_]): DataFrame = { + val predictUDF = udf { (features: Vector, offset: Double) => predict(features, offset) } + val predictLinkUDF = udf { (features: Vector, offset: Double) => predictLink(features, offset) } + + val offset = if (!hasOffsetCol) lit(0.0) else col($(offsetCol)).cast(DoubleType) + var output = dataset + if ($(predictionCol).nonEmpty) { + output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)), offset)) + } + if (hasLinkPredictionCol) { + output = output.withColumn($(linkPredictionCol), predictLinkUDF(col($(featuresCol)), offset)) + } + output.toDF() + } + + private var trainingSummary: Option[GeneralizedLinearRegressionTrainingSummary] = None + + /** + * Gets R-like summary of model on training set. An exception is + * thrown if there is no summary available. + */ + @Since("2.0.0") + def summary: GeneralizedLinearRegressionTrainingSummary = trainingSummary.getOrElse { + throw new SparkException( + "No training summary available for this GeneralizedLinearRegressionModel") + } + + /** + * Indicates if [[summary]] is available. + */ + @Since("2.0.0") + def hasSummary: Boolean = trainingSummary.nonEmpty + + private[regression] + def setSummary(summary: Option[GeneralizedLinearRegressionTrainingSummary]): this.type = { + this.trainingSummary = summary + this + } + + /** + * Evaluate the model on the given dataset, returning a summary of the results. + */ + @Since("2.0.0") + def evaluate(dataset: Dataset[_]): GeneralizedLinearRegressionSummary = { + new GeneralizedLinearRegressionSummary(dataset, this) + } + + @Since("2.0.0") + override def copy(extra: ParamMap): GeneralizedLinearRegressionModel = { + val copied = copyValues(new GeneralizedLinearRegressionModel(uid, coefficients, intercept), + extra) + copied.setSummary(trainingSummary).setParent(parent) + } + + /** + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. + * + * For [[GeneralizedLinearRegressionModel]], this does NOT currently save the + * training [[summary]]. An option to save [[summary]] may be added in the future. + * + */ + @Since("2.0.0") + override def write: MLWriter = + new GeneralizedLinearRegressionModel.GeneralizedLinearRegressionModelWriter(this) + + override val numFeatures: Int = coefficients.size +} + +@Since("2.0.0") +object GeneralizedLinearRegressionModel extends MLReadable[GeneralizedLinearRegressionModel] { + + @Since("2.0.0") + override def read: MLReader[GeneralizedLinearRegressionModel] = + new GeneralizedLinearRegressionModelReader + + @Since("2.0.0") + override def load(path: String): GeneralizedLinearRegressionModel = super.load(path) + + /** [[MLWriter]] instance for [[GeneralizedLinearRegressionModel]] */ + private[GeneralizedLinearRegressionModel] + class GeneralizedLinearRegressionModelWriter(instance: GeneralizedLinearRegressionModel) + extends MLWriter with Logging { + + private case class Data(intercept: Double, coefficients: Vector) + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + // Save model data: intercept, coefficients + val data = Data(instance.intercept, instance.coefficients) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class GeneralizedLinearRegressionModelReader + extends MLReader[GeneralizedLinearRegressionModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[GeneralizedLinearRegressionModel].getName + + override def load(path: String): GeneralizedLinearRegressionModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + .select("intercept", "coefficients").head() + val intercept = data.getDouble(0) + val coefficients = data.getAs[Vector](1) + + val model = new GeneralizedLinearRegressionModel(metadata.uid, coefficients, intercept) + + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } +} + +/** + * :: Experimental :: + * Summary of [[GeneralizedLinearRegression]] model and predictions. + * + * @param dataset Dataset to be summarized. + * @param origModel Model to be summarized. This is copied to create an internal + * model which cannot be modified from outside. + */ +@Since("2.0.0") +@Experimental +class GeneralizedLinearRegressionSummary private[regression] ( + dataset: Dataset[_], + origModel: GeneralizedLinearRegressionModel) extends Serializable { + + import GeneralizedLinearRegression._ + + /** + * Field in "predictions" which gives the predicted value of each instance. + * This is set to a new column name if the original model's `predictionCol` is not set. + */ + @Since("2.0.0") + val predictionCol: String = { + if (origModel.isDefined(origModel.predictionCol) && origModel.getPredictionCol.nonEmpty) { + origModel.getPredictionCol + } else { + "prediction_" + java.util.UUID.randomUUID.toString + } + } + + /** + * Private copy of model to ensure Params are not modified outside this class. + * Coefficients is not a deep copy, but that is acceptable. + * + * @note [[predictionCol]] must be set correctly before the value of [[model]] is set, + * and [[model]] must be set before [[predictions]] is set! + */ + protected val model: GeneralizedLinearRegressionModel = + origModel.copy(ParamMap.empty).setPredictionCol(predictionCol) + + /** + * Predictions output by the model's `transform` method. + */ + @Since("2.0.0") @transient val predictions: DataFrame = model.transform(dataset) + + private[regression] lazy val familyLink: FamilyAndLink = FamilyAndLink(model) + + private[regression] lazy val family: Family = familyLink.family + + private[regression] lazy val link: Link = familyLink.link + + /** Number of instances in DataFrame predictions. */ + @Since("2.2.0") + lazy val numInstances: Long = predictions.count() + + + /** + * Name of features. If the name cannot be retrieved from attributes, + * set default names to feature column name with numbered suffix "_0", "_1", and so on. + */ + private[ml] lazy val featureNames: Array[String] = { + val featureAttrs = AttributeGroup.fromStructField( + dataset.schema(model.getFeaturesCol)).attributes + if (featureAttrs.isDefined) { + featureAttrs.get.map(_.name.get) + } else { + Array.tabulate[String](origModel.numFeatures)((x: Int) => model.getFeaturesCol + "_" + x) + } + } + + /** The numeric rank of the fitted linear model. */ + @Since("2.0.0") + lazy val rank: Long = if (model.getFitIntercept) { + model.coefficients.size + 1 + } else { + model.coefficients.size + } + + /** Degrees of freedom. */ + @Since("2.0.0") + lazy val degreesOfFreedom: Long = numInstances - rank + + /** The residual degrees of freedom. */ + @Since("2.0.0") + lazy val residualDegreeOfFreedom: Long = degreesOfFreedom + + /** The residual degrees of freedom for the null model. */ + @Since("2.0.0") + lazy val residualDegreeOfFreedomNull: Long = { + if (model.getFitIntercept) numInstances - 1 else numInstances + } + + private def label: Column = col(model.getLabelCol).cast(DoubleType) + + private def prediction: Column = col(predictionCol) + + private def weight: Column = { + if (!model.hasWeightCol) lit(1.0) else col(model.getWeightCol) + } + + private def offset: Column = { + if (!model.hasOffsetCol) lit(0.0) else col(model.getOffsetCol).cast(DoubleType) + } + + private[regression] lazy val devianceResiduals: DataFrame = { + val drUDF = udf { (y: Double, mu: Double, weight: Double) => + val r = math.sqrt(math.max(family.deviance(y, mu, weight), 0.0)) + if (y > mu) r else -1.0 * r + } + predictions.select( + drUDF(label, prediction, weight).as("devianceResiduals")) + } + + private[regression] lazy val pearsonResiduals: DataFrame = { + val prUDF = udf { mu: Double => family.variance(mu) } + predictions.select(label.minus(prediction) + .multiply(sqrt(weight)).divide(sqrt(prUDF(prediction))).as("pearsonResiduals")) + } + + private[regression] lazy val workingResiduals: DataFrame = { + val wrUDF = udf { (y: Double, mu: Double) => (y - mu) * link.deriv(mu) } + predictions.select(wrUDF(label, prediction).as("workingResiduals")) + } + + private[regression] lazy val responseResiduals: DataFrame = { + predictions.select(label.minus(prediction).as("responseResiduals")) + } + + /** + * Get the default residuals (deviance residuals) of the fitted model. + */ + @Since("2.0.0") + def residuals(): DataFrame = devianceResiduals + + /** + * Get the residuals of the fitted model by type. + * + * @param residualsType The type of residuals which should be returned. + * Supported options: deviance, pearson, working and response. + */ + @Since("2.0.0") + def residuals(residualsType: String): DataFrame = { + residualsType match { + case "deviance" => devianceResiduals + case "pearson" => pearsonResiduals + case "working" => workingResiduals + case "response" => responseResiduals + case other => throw new UnsupportedOperationException( + s"The residuals type $other is not supported by Generalized Linear Regression.") + } + } + + /** + * The deviance for the null model. + */ + @Since("2.0.0") + lazy val nullDeviance: Double = { + val intercept: Double = if (!model.getFitIntercept) { + 0.0 + } else { + /* + Estimate intercept analytically when there is no offset, or when there is offset but + the model is Gaussian family with identity link. Otherwise, fit an intercept only model. + */ + if (!model.hasOffsetCol || + (model.hasOffsetCol && family == Gaussian && link == Identity)) { + val agg = predictions.agg(sum(weight.multiply( + label.minus(offset))), sum(weight)).first() + link.link(agg.getDouble(0) / agg.getDouble(1)) + } else { + // Create empty feature column and fit intercept only model using param setting from model + val featureNull = "feature_" + java.util.UUID.randomUUID.toString + val paramMap = model.extractParamMap() + paramMap.put(model.featuresCol, featureNull) + if (family.name != "tweedie") { + paramMap.remove(model.variancePower) + } + val emptyVectorUDF = udf{ () => Vectors.zeros(0) } + model.parent.fit( + dataset.withColumn(featureNull, emptyVectorUDF()), paramMap + ).intercept + } + } + predictions.select(label, offset, weight).rdd.map { + case Row(y: Double, offset: Double, weight: Double) => + family.deviance(y, link.unlink(intercept + offset), weight) + }.sum() + } + + /** + * The deviance for the fitted model. + */ + @Since("2.0.0") + lazy val deviance: Double = { + predictions.select(label, prediction, weight).rdd.map { + case Row(label: Double, pred: Double, weight: Double) => + family.deviance(label, pred, weight) + }.sum() + } + + /** + * The dispersion of the fitted model. + * It is taken as 1.0 for the "binomial" and "poisson" families, and otherwise + * estimated by the residual Pearson's Chi-Squared statistic (which is defined as + * sum of the squares of the Pearson residuals) divided by the residual degrees of freedom. + */ + @Since("2.0.0") + lazy val dispersion: Double = if ( + model.getFamily.toLowerCase(Locale.ROOT) == Binomial.name || + model.getFamily.toLowerCase(Locale.ROOT) == Poisson.name) { + 1.0 + } else { + val rss = pearsonResiduals.agg(sum(pow(col("pearsonResiduals"), 2.0))).first().getDouble(0) + rss / degreesOfFreedom + } + + /** Akaike Information Criterion (AIC) for the fitted model. */ + @Since("2.0.0") + lazy val aic: Double = { + val weightSum = predictions.select(weight).agg(sum(weight)).first().getDouble(0) + val t = predictions.select( + label, prediction, weight).rdd.map { + case Row(label: Double, pred: Double, weight: Double) => + (label, pred, weight) + } + family.aic(t, deviance, numInstances, weightSum) + 2 * rank + } +} + +/** + * :: Experimental :: + * Summary of [[GeneralizedLinearRegression]] fitting and model. + * + * @param dataset Dataset to be summarized. + * @param origModel Model to be summarized. This is copied to create an internal + * model which cannot be modified from outside. + * @param diagInvAtWA diagonal of matrix (A^T * W * A)^-1 in the last iteration + * @param numIterations number of iterations + * @param solver the solver algorithm used for model training + */ +@Since("2.0.0") +@Experimental +class GeneralizedLinearRegressionTrainingSummary private[regression] ( + dataset: Dataset[_], + origModel: GeneralizedLinearRegressionModel, + private val diagInvAtWA: Array[Double], + @Since("2.0.0") val numIterations: Int, + @Since("2.0.0") val solver: String) + extends GeneralizedLinearRegressionSummary(dataset, origModel) with Serializable { + + import GeneralizedLinearRegression._ + + /** + * Whether the underlying `WeightedLeastSquares` using the "normal" solver. + */ + private[ml] val isNormalSolver: Boolean = { + diagInvAtWA.length != 1 || diagInvAtWA(0) != 0 + } + + /** + * Standard error of estimated coefficients and intercept. + * This value is only available when the underlying `WeightedLeastSquares` + * using the "normal" solver. + * + * If `GeneralizedLinearRegression.fitIntercept` is set to true, + * then the last element returned corresponds to the intercept. + */ + @Since("2.0.0") + lazy val coefficientStandardErrors: Array[Double] = { + if (isNormalSolver) { + diagInvAtWA.map(_ * dispersion).map(math.sqrt) + } else { + throw new UnsupportedOperationException( + "No Std. Error of coefficients available for this GeneralizedLinearRegressionModel") + } + } + + /** + * T-statistic of estimated coefficients and intercept. + * This value is only available when the underlying `WeightedLeastSquares` + * using the "normal" solver. + * + * If `GeneralizedLinearRegression.fitIntercept` is set to true, + * then the last element returned corresponds to the intercept. + */ + @Since("2.0.0") + lazy val tValues: Array[Double] = { + if (isNormalSolver) { + val estimate = if (model.getFitIntercept) { + Array.concat(model.coefficients.toArray, Array(model.intercept)) + } else { + model.coefficients.toArray + } + estimate.zip(coefficientStandardErrors).map { x => x._1 / x._2 } + } else { + throw new UnsupportedOperationException( + "No t-statistic available for this GeneralizedLinearRegressionModel") + } + } + + /** + * Two-sided p-value of estimated coefficients and intercept. + * This value is only available when the underlying `WeightedLeastSquares` + * using the "normal" solver. + * + * If `GeneralizedLinearRegression.fitIntercept` is set to true, + * then the last element returned corresponds to the intercept. + */ + @Since("2.0.0") + lazy val pValues: Array[Double] = { + if (isNormalSolver) { + if (model.getFamily.toLowerCase(Locale.ROOT) == Binomial.name || + model.getFamily.toLowerCase(Locale.ROOT) == Poisson.name) { + tValues.map { x => 2.0 * (1.0 - dist.Gaussian(0.0, 1.0).cdf(math.abs(x))) } + } else { + tValues.map { x => + 2.0 * (1.0 - dist.StudentsT(degreesOfFreedom.toDouble).cdf(math.abs(x))) + } + } + } else { + throw new UnsupportedOperationException( + "No p-value available for this GeneralizedLinearRegressionModel") + } + } + + /** + * Coefficients with statistics: feature name, coefficients, standard error, tValue and pValue. + */ + private[ml] lazy val coefficientsWithStatistics: Array[ + (String, Double, Double, Double, Double)] = { + var featureNamesLocal = featureNames + var coefficientsArray = model.coefficients.toArray + var index = Array.range(0, coefficientsArray.length) + if (model.getFitIntercept) { + featureNamesLocal = featureNamesLocal :+ "(Intercept)" + coefficientsArray = coefficientsArray :+ model.intercept + // Reorder so that intercept comes first + index = (coefficientsArray.length - 1) +: index + } + index.map { i => + (featureNamesLocal(i), coefficientsArray(i), coefficientStandardErrors(i), + tValues(i), pValues(i)) + } + } + + override def toString: String = { + if (isNormalSolver) { + + def round(x: Double): String = { + BigDecimal(x).setScale(4, BigDecimal.RoundingMode.HALF_UP).toString + } + + val colNames = Array("Feature", "Estimate", "Std Error", "T Value", "P Value") + + val data = coefficientsWithStatistics.map { row => + val strRow = row.productIterator.map { cell => + val str = cell match { + case s: String => s + case n: Double => round(n) + } + // Truncate if length > 20 + if (str.length > 20) { + str.substring(0, 17) + "..." + } else { + str + } + } + strRow.toArray + } + + // Compute the width of each column + val colWidths = colNames.map(_.length) + data.foreach { strRow => + strRow.zipWithIndex.foreach { case (cell: String, i: Int) => + colWidths(i) = math.max(colWidths(i), cell.length) + } + } + + val sb = new StringBuilder + + // Output coefficients with statistics + sb.append("Coefficients:\n") + colNames.zipWithIndex.map { case (colName: String, i: Int) => + StringUtils.leftPad(colName, colWidths(i)) + }.addString(sb, "", " ", "\n") + + data.foreach { case strRow: Array[String] => + strRow.zipWithIndex.map { case (cell: String, i: Int) => + StringUtils.leftPad(cell.toString, colWidths(i)) + }.addString(sb, "", " ", "\n") + } + + sb.append("\n") + sb.append(s"(Dispersion parameter for ${family.name} family taken to be " + + s"${round(dispersion)})") + + sb.append("\n") + val nd = s"Null deviance: ${round(nullDeviance)} on $degreesOfFreedom degrees of freedom" + val rd = s"Residual deviance: ${round(deviance)} on $residualDegreeOfFreedom degrees of " + + "freedom" + val l = math.max(nd.length, rd.length) + sb.append(StringUtils.leftPad(nd, l)) + sb.append("\n") + sb.append(StringUtils.leftPad(rd, l)) + + if (family.name != "tweedie") { + sb.append("\n") + sb.append(s"AIC: " + round(aic)) + } + + sb.toString() + } else { + throw new UnsupportedOperationException( + "No summary available for this GeneralizedLinearRegressionModel") + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala index a1fe01b04710..8faab52ea474 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/IsotonicRegression.scala @@ -17,16 +17,20 @@ package org.apache.spark.ml.regression -import org.apache.spark.Logging -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasLabelCol, HasPredictionCol, HasWeightCol} -import org.apache.spark.ml.util.{Identifiable, SchemaUtils} -import org.apache.spark.mllib.linalg.{Vector, VectorUDT, Vectors} -import org.apache.spark.mllib.regression.{IsotonicRegression => MLlibIsotonicRegression, IsotonicRegressionModel => MLlibIsotonicRegressionModel} +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.regression.IsotonicRegressionModel.IsotonicRegressionModelWriter +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.regression.{IsotonicRegression => MLlibIsotonicRegression} +import org.apache.spark.mllib.regression.{IsotonicRegressionModel => MLlibIsotonicRegressionModel} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.{col, lit, udf} import org.apache.spark.sql.types.{DoubleType, StructType} import org.apache.spark.storage.StorageLevel @@ -45,19 +49,20 @@ private[regression] trait IsotonicRegressionBase extends Params with HasFeatures */ final val isotonic: BooleanParam = new BooleanParam(this, "isotonic", - "whether the output sequence should be isotonic/increasing (true) or" + + "whether the output sequence should be isotonic/increasing (true) or " + "antitonic/decreasing (false)") /** @group getParam */ final def getIsotonic: Boolean = $(isotonic) /** - * Param for the index of the feature if [[featuresCol]] is a vector column (default: `0`), no + * Param for the index of the feature if `featuresCol` is a vector column (default: `0`), no * effect otherwise. * @group param */ final val featureIndex: IntParam = new IntParam(this, "featureIndex", - "The index of the feature if featuresCol is a vector column, no effect otherwise.") + "The index of the feature if featuresCol is a vector column, no effect otherwise (>= 0)", + ParamValidators.gtEq(0)) /** @group getParam */ final def getFeatureIndex: Int = $(featureIndex) @@ -65,15 +70,15 @@ private[regression] trait IsotonicRegressionBase extends Params with HasFeatures setDefault(isotonic -> true, featureIndex -> 0) /** Checks whether the input has weight column. */ - protected[ml] def hasWeightCol: Boolean = { - isDefined(weightCol) && $(weightCol) != "" + private[regression] def hasWeightCol: Boolean = { + isDefined(weightCol) && $(weightCol).nonEmpty } /** * Extracts (label, feature, weight) from input dataset. */ protected[ml] def extractWeightedLabeledPoints( - dataset: DataFrame): RDD[(Double, Double, Double)] = { + dataset: Dataset[_]): RDD[(Double, Double, Double)] = { val f = if (dataset.schema($(featuresCol)).dataType.isInstanceOf[VectorUDT]) { val idx = $(featureIndex) val extract = udf { v: Vector => v(idx) } @@ -81,14 +86,11 @@ private[regression] trait IsotonicRegressionBase extends Params with HasFeatures } else { col($(featuresCol)) } - val w = if (hasWeightCol) { - col($(weightCol)) - } else { - lit(1.0) - } - dataset.select(col($(labelCol)), f, w) - .map { case Row(label: Double, feature: Double, weight: Double) => - (label, feature, weight) + val w = if (hasWeightCol) col($(weightCol)).cast(DoubleType) else lit(1.0) + + dataset.select(col($(labelCol)).cast(DoubleType), f, w).rdd.map { + case Row(label: Double, feature: Double, weight: Double) => + (label, feature, weight) } } @@ -102,9 +104,9 @@ private[regression] trait IsotonicRegressionBase extends Params with HasFeatures schema: StructType, fitting: Boolean): StructType = { if (fitting) { - SchemaUtils.checkColumnType(schema, $(labelCol), DoubleType) + SchemaUtils.checkNumericType(schema, $(labelCol)) if (hasWeightCol) { - SchemaUtils.checkColumnType(schema, $(weightCol), DoubleType) + SchemaUtils.checkNumericType(schema, $(weightCol)) } else { logInfo("The weight column is not defined. Treat all instance weights as 1.0.") } @@ -116,7 +118,6 @@ private[regression] trait IsotonicRegressionBase extends Params with HasFeatures } /** - * :: Experimental :: * Isotonic regression. * * Currently implemented using parallelized pool adjacent violators algorithm. @@ -125,9 +126,9 @@ private[regression] trait IsotonicRegressionBase extends Params with HasFeatures * Uses [[org.apache.spark.mllib.regression.IsotonicRegression]]. */ @Since("1.5.0") -@Experimental class IsotonicRegression @Since("1.5.0") (@Since("1.5.0") override val uid: String) - extends Estimator[IsotonicRegressionModel] with IsotonicRegressionBase { + extends Estimator[IsotonicRegressionModel] + with IsotonicRegressionBase with DefaultParamsWritable { @Since("1.5.0") def this() = this(Identifiable.randomUID("isoReg")) @@ -159,18 +160,26 @@ class IsotonicRegression @Since("1.5.0") (@Since("1.5.0") override val uid: Stri @Since("1.5.0") override def copy(extra: ParamMap): IsotonicRegression = defaultCopy(extra) - @Since("1.5.0") - override def fit(dataset: DataFrame): IsotonicRegressionModel = { - validateAndTransformSchema(dataset.schema, fitting = true) + @Since("2.0.0") + override def fit(dataset: Dataset[_]): IsotonicRegressionModel = { + transformSchema(dataset.schema, logging = true) // Extract columns from data. If dataset is persisted, do not persist oldDataset. val instances = extractWeightedLabeledPoints(dataset) - val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE + val handlePersistence = dataset.storageLevel == StorageLevel.NONE if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) + val instr = Instrumentation.create(this, dataset) + instr.logParams(labelCol, featuresCol, weightCol, predictionCol, featureIndex, isotonic) + instr.logNumFeatures(1) + val isotonicRegression = new MLlibIsotonicRegression().setIsotonic($(isotonic)) val oldModel = isotonicRegression.run(instances) - copyValues(new IsotonicRegressionModel(uid, oldModel).setParent(this)) + if (handlePersistence) instances.unpersist() + + val model = copyValues(new IsotonicRegressionModel(uid, oldModel).setParent(this)) + instr.logSuccess(model) + model } @Since("1.5.0") @@ -179,22 +188,27 @@ class IsotonicRegression @Since("1.5.0") (@Since("1.5.0") override val uid: Stri } } +@Since("1.6.0") +object IsotonicRegression extends DefaultParamsReadable[IsotonicRegression] { + + @Since("1.6.0") + override def load(path: String): IsotonicRegression = super.load(path) +} + /** - * :: Experimental :: * Model fitted by IsotonicRegression. * Predicts using a piecewise linear function. * - * For detailed rules see [[org.apache.spark.mllib.regression.IsotonicRegressionModel.predict()]]. + * For detailed rules see `org.apache.spark.mllib.regression.IsotonicRegressionModel.predict()`. * * @param oldModel A [[org.apache.spark.mllib.regression.IsotonicRegressionModel]] * model trained by [[org.apache.spark.mllib.regression.IsotonicRegression]]. */ @Since("1.5.0") -@Experimental class IsotonicRegressionModel private[ml] ( override val uid: String, private val oldModel: MLlibIsotonicRegressionModel) - extends Model[IsotonicRegressionModel] with IsotonicRegressionBase { + extends Model[IsotonicRegressionModel] with IsotonicRegressionBase with MLWritable { /** @group setParam */ @Since("1.5.0") @@ -209,14 +223,14 @@ class IsotonicRegressionModel private[ml] ( def setFeatureIndex(value: Int): this.type = set(featureIndex, value) /** Boundaries in increasing order for which predictions are known. */ - @Since("1.5.0") + @Since("2.0.0") def boundaries: Vector = Vectors.dense(oldModel.boundaries) /** * Predictions associated with the boundaries at the same index, monotone because of isotonic * regression. */ - @Since("1.5.0") + @Since("2.0.0") def predictions: Vector = Vectors.dense(oldModel.predictions) @Since("1.5.0") @@ -224,8 +238,9 @@ class IsotonicRegressionModel private[ml] ( copyValues(new IsotonicRegressionModel(uid, oldModel), extra).setParent(parent) } - @Since("1.5.0") - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) val predict = dataset.schema($(featuresCol)).dataType match { case DoubleType => udf { feature: Double => oldModel.predict(feature) } @@ -240,4 +255,61 @@ class IsotonicRegressionModel private[ml] ( override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema, fitting = false) } + + @Since("1.6.0") + override def write: MLWriter = + new IsotonicRegressionModelWriter(this) +} + +@Since("1.6.0") +object IsotonicRegressionModel extends MLReadable[IsotonicRegressionModel] { + + @Since("1.6.0") + override def read: MLReader[IsotonicRegressionModel] = new IsotonicRegressionModelReader + + @Since("1.6.0") + override def load(path: String): IsotonicRegressionModel = super.load(path) + + /** [[MLWriter]] instance for [[IsotonicRegressionModel]] */ + private[IsotonicRegressionModel] class IsotonicRegressionModelWriter ( + instance: IsotonicRegressionModel + ) extends MLWriter with Logging { + + private case class Data( + boundaries: Array[Double], + predictions: Array[Double], + isotonic: Boolean) + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + // Save model data: boundaries, predictions, isotonic + val data = Data( + instance.oldModel.boundaries, instance.oldModel.predictions, instance.oldModel.isotonic) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class IsotonicRegressionModelReader extends MLReader[IsotonicRegressionModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[IsotonicRegressionModel].getName + + override def load(path: String): IsotonicRegressionModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + .select("boundaries", "predictions", "isotonic").head() + val boundaries = data.getAs[Seq[Double]](0).toArray + val predictions = data.getAs[Seq[Double]](1).toArray + val isotonic = data.getBoolean(2) + val model = new IsotonicRegressionModel( + metadata.uid, new MLlibIsotonicRegressionModel(boundaries, predictions, isotonic)) + + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } } 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 913140e58198..df1aa609c1b7 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 @@ -20,24 +20,31 @@ package org.apache.spark.ml.regression import scala.collection.mutable import breeze.linalg.{DenseVector => BDV} -import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS, OWLQN => BreezeOWLQN} +import breeze.optimize.{CachedDiffFunction, LBFGS => BreezeLBFGS, OWLQN => BreezeOWLQN} import breeze.stats.distributions.StudentsT +import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkException} -import org.apache.spark.ml.feature.Instance -import org.apache.spark.ml.optim.WeightedLeastSquares +import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging import org.apache.spark.ml.PredictorParams -import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.BLAS._ +import org.apache.spark.ml.optim.WeightedLeastSquares +import org.apache.spark.ml.optim.aggregator.LeastSquaresAggregator +import org.apache.spark.ml.optim.loss.{L2Regularization, RDDLossFunction} +import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators} import org.apache.spark.ml.param.shared._ -import org.apache.spark.ml.util.Identifiable +import org.apache.spark.ml.util._ import org.apache.spark.mllib.evaluation.RegressionMetrics -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.linalg.BLAS._ +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer +import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.DoubleType import org.apache.spark.storage.StorageLevel /** @@ -46,26 +53,48 @@ import org.apache.spark.storage.StorageLevel private[regression] trait LinearRegressionParams extends PredictorParams with HasRegParam with HasElasticNetParam with HasMaxIter with HasTol with HasFitIntercept with HasStandardization with HasWeightCol with HasSolver + with HasAggregationDepth { + + import LinearRegression._ + + /** + * The solver algorithm for optimization. + * Supported options: "l-bfgs", "normal" and "auto". + * Default: "auto" + * + * @group param + */ + @Since("1.6.0") + final override val solver: Param[String] = new Param[String](this, "solver", + "The solver algorithm for optimization. Supported options: " + + s"${supportedSolvers.mkString(", ")}. (Default auto)", + ParamValidators.inArray[String](supportedSolvers)) +} /** - * :: Experimental :: * Linear regression. * * The learning objective is to minimize the squared error, with regularization. * The specific squared error loss function used is: - * L = 1/2n ||A coefficients - y||^2^ * - * This support multiple types of regularization: + *
    + * $$ + * L = 1/2n ||A coefficients - y||^2^ + * $$ + *
    + * + * This supports multiple types of regularization: * - none (a.k.a. ordinary least squares) * - L2 (ridge regression) * - L1 (Lasso) * - L2 + L1 (elastic net) */ @Since("1.3.0") -@Experimental class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String) extends Regressor[Vector, LinearRegression, LinearRegressionModel] - with LinearRegressionParams with Logging { + with LinearRegressionParams with DefaultParamsWritable with Logging { + + import LinearRegression._ @Since("1.4.0") def this() = this(Identifiable.randomUID("linReg")) @@ -73,6 +102,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String /** * Set the regularization parameter. * Default is 0.0. + * * @group setParam */ @Since("1.3.0") @@ -80,8 +110,9 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String setDefault(regParam -> 0.0) /** - * Set if we should fit the intercept + * Set if we should fit the intercept. * Default is true. + * * @group setParam */ @Since("1.5.0") @@ -91,10 +122,13 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String /** * Whether to standardize the training features before fitting the model. * The coefficients of models will be always returned on the original scale, - * so it will be transparent for users. Note that with/without standardization, - * the models should be always converged to the same solution when no regularization - * is applied. In R's GLMNET package, the default behavior is true as well. + * so it will be transparent for users. * Default is true. + * + * @note With/without standardization, the models should be always converged + * to the same solution when no regularization is applied. In R's GLMNET package, + * the default behavior is true as well. + * * @group setParam */ @Since("1.5.0") @@ -103,9 +137,11 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String /** * Set the ElasticNet mixing parameter. - * For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty. - * For 0 < alpha < 1, the penalty is a combination of L1 and L2. + * For alpha = 0, the penalty is an L2 penalty. + * For alpha = 1, it is an L1 penalty. + * For alpha in (0,1), the penalty is a combination of L1 and L2. * Default is 0.0 which is an L2 penalty. + * * @group setParam */ @Since("1.4.0") @@ -115,6 +151,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String /** * Set the maximum number of iterations. * Default is 100. + * * @group setParam */ @Since("1.3.0") @@ -125,6 +162,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String * Set the convergence tolerance of iterations. * Smaller value will lead to higher accuracy with the cost of more iterations. * Default is 1E-6. + * * @group setParam */ @Since("1.4.0") @@ -133,71 +171,87 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String /** * Whether to over-/under-sample training instances according to the given weights in weightCol. - * If empty, all instances are treated equally (weight 1.0). - * Default is empty, so all instances have weight one. + * If not set or empty, all instances are treated equally (weight 1.0). + * Default is not set, so all instances have weight one. + * * @group setParam */ @Since("1.6.0") def setWeightCol(value: String): this.type = set(weightCol, value) - setDefault(weightCol -> "") /** * Set the solver algorithm used for optimization. * In case of linear regression, this can be "l-bfgs", "normal" and "auto". - * The default value is "auto" which means that the solver algorithm is - * selected automatically. + * - "l-bfgs" denotes Limited-memory BFGS which is a limited-memory quasi-Newton + * optimization method. + * - "normal" denotes using Normal Equation as an analytical solution to the linear regression + * problem. This solver is limited to `LinearRegression.MAX_FEATURES_FOR_NORMAL_SOLVER`. + * - "auto" (default) means that the solver algorithm is selected automatically. + * The Normal Equations solver will be used when possible, but this will automatically fall + * back to iterative optimization methods when needed. + * * @group setParam */ @Since("1.6.0") def setSolver(value: String): this.type = set(solver, value) - setDefault(solver -> "auto") + setDefault(solver -> Auto) - override protected def train(dataset: DataFrame): LinearRegressionModel = { + /** + * Suggested depth for treeAggregate (greater than or equal to 2). + * If the dimensions of features or the number of partitions are large, + * this param could be adjusted to a larger size. + * Default is 2. + * + * @group expertSetParam + */ + @Since("2.1.0") + def setAggregationDepth(value: Int): this.type = set(aggregationDepth, value) + setDefault(aggregationDepth -> 2) + + override protected def train(dataset: Dataset[_]): LinearRegressionModel = { // Extract the number of features before deciding optimization solver. - val numFeatures = dataset.select(col($(featuresCol))).limit(1).map { - case Row(features: Vector) => features.size - }.first() - val w = if ($(weightCol).isEmpty) lit(1.0) else col($(weightCol)) - - if (($(solver) == "auto" && $(elasticNetParam) == 0.0 && numFeatures <= 4096) || - $(solver) == "normal") { - require($(elasticNetParam) == 0.0, "Only L2 regularization can be used when normal " + - "solver is used.'") - // For low dimensional data, WeightedLeastSquares is more efficiently since the + val numFeatures = dataset.select(col($(featuresCol))).first().getAs[Vector](0).size + val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) + + val instances: RDD[Instance] = dataset.select( + col($(labelCol)), w, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, features: Vector) => + Instance(label, weight, features) + } + + val instr = Instrumentation.create(this, dataset) + instr.logParams(labelCol, featuresCol, weightCol, predictionCol, solver, tol, + elasticNetParam, fitIntercept, maxIter, regParam, standardization, aggregationDepth) + instr.logNumFeatures(numFeatures) + + if (($(solver) == Auto && + numFeatures <= WeightedLeastSquares.MAX_NUM_FEATURES) || $(solver) == Normal) { + // For low dimensional data, WeightedLeastSquares is more efficient since the // training algorithm only requires one pass through the data. (SPARK-10668) - val instances: RDD[Instance] = dataset.select( - col($(labelCol)), w, col($(featuresCol))).map { - case Row(label: Double, weight: Double, features: Vector) => - Instance(label, weight, features) - } val optimizer = new WeightedLeastSquares($(fitIntercept), $(regParam), - $(standardization), true) + elasticNetParam = $(elasticNetParam), $(standardization), true, + solverType = WeightedLeastSquares.Auto, maxIter = $(maxIter), tol = $(tol)) val model = optimizer.fit(instances) // When it is trained by WeightedLeastSquares, training summary does not - // attached returned model. + // attach returned model. val lrModel = copyValues(new LinearRegressionModel(uid, model.coefficients, model.intercept)) - // WeightedLeastSquares does not run through iterations. So it does not generate - // an objective history. val (summaryModel, predictionColName) = lrModel.findSummaryModelAndPredictionCol() val trainingSummary = new LinearRegressionTrainingSummary( summaryModel.transform(dataset), predictionColName, $(labelCol), + $(featuresCol), summaryModel, model.diagInvAtWA.toArray, - $(featuresCol), - Array(0D)) - - return lrModel.setSummary(trainingSummary) - } + model.objectiveHistory) - val instances: RDD[Instance] = dataset.select(col($(labelCol)), w, col($(featuresCol))).map { - case Row(label: Double, weight: Double, features: Vector) => - Instance(label, weight, features) + lrModel.setSummary(Some(trainingSummary)) + instr.logSuccess(lrModel) + return lrModel } - val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE + val handlePersistence = dataset.storageLevel == StorageLevel.NONE if (handlePersistence) instances.persist(StorageLevel.MEMORY_AND_DISK) val (featuresSummarizer, ySummarizer) = { @@ -211,39 +265,68 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String (c1._1.merge(c2._1), c1._2.merge(c2._2)) instances.treeAggregate( - new MultivariateOnlineSummarizer, new MultivariateOnlineSummarizer)(seqOp, combOp) + (new MultivariateOnlineSummarizer, new MultivariateOnlineSummarizer) + )(seqOp, combOp, $(aggregationDepth)) } val yMean = ySummarizer.mean(0) - val yStd = math.sqrt(ySummarizer.variance(0)) - - // If the yStd is zero, then the intercept is yMean with zero coefficient; - // as a result, training is not needed. - if (yStd == 0.0) { - logWarning(s"The standard deviation of the label is zero, so the coefficients will be " + - s"zeros and the intercept will be the mean of the label; as a result, " + - s"training is not needed.") - if (handlePersistence) instances.unpersist() - val coefficients = Vectors.sparse(numFeatures, Seq()) - val intercept = yMean - - val model = new LinearRegressionModel(uid, coefficients, intercept) - // Handle possible missing or invalid prediction columns - val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() - - val trainingSummary = new LinearRegressionTrainingSummary( - summaryModel.transform(dataset), - predictionColName, - $(labelCol), - model, - Array(0D), - $(featuresCol), - Array(0D)) - return copyValues(model.setSummary(trainingSummary)) + val rawYStd = math.sqrt(ySummarizer.variance(0)) + if (rawYStd == 0.0) { + if ($(fitIntercept) || yMean == 0.0) { + // If the rawYStd==0 and fitIntercept==true, then the intercept is yMean with + // zero coefficient; as a result, training is not needed. + // Also, if yMean==0 and rawYStd==0, all the coefficients are zero regardless of + // the fitIntercept. + if (yMean == 0.0) { + logWarning(s"Mean and standard deviation of the label are zero, so the coefficients " + + s"and the intercept will all be zero; as a result, training is not needed.") + } else { + logWarning(s"The standard deviation of the label is zero, so the coefficients will be " + + s"zeros and the intercept will be the mean of the label; as a result, " + + s"training is not needed.") + } + if (handlePersistence) instances.unpersist() + val coefficients = Vectors.sparse(numFeatures, Seq.empty) + val intercept = yMean + + val model = copyValues(new LinearRegressionModel(uid, coefficients, intercept)) + // Handle possible missing or invalid prediction columns + val (summaryModel, predictionColName) = model.findSummaryModelAndPredictionCol() + + val trainingSummary = new LinearRegressionTrainingSummary( + summaryModel.transform(dataset), + predictionColName, + $(labelCol), + $(featuresCol), + model, + Array(0D), + Array(0D)) + + model.setSummary(Some(trainingSummary)) + instr.logSuccess(model) + return model + } else { + require($(regParam) == 0.0, "The standard deviation of the label is zero. " + + "Model cannot be regularized.") + logWarning(s"The standard deviation of the label is zero. " + + "Consider setting fitIntercept=true.") + } } + // if y is constant (rawYStd is zero), then y cannot be scaled. In this case + // setting yStd=abs(yMean) ensures that y is not scaled anymore in l-bfgs algorithm. + val yStd = if (rawYStd > 0) rawYStd else math.abs(yMean) val featuresMean = featuresSummarizer.mean.toArray val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val bcFeaturesMean = instances.context.broadcast(featuresMean) + val bcFeaturesStd = instances.context.broadcast(featuresStd) + + if (!$(fitIntercept) && (0 until numFeatures).exists { i => + featuresStd(i) == 0.0 && featuresMean(i) != 0.0 }) { + logWarning("Fitting LinearRegressionModel without intercept on dataset with " + + "constant nonzero column, Spark MLlib outputs zero coefficients for constant nonzero " + + "columns. This behavior is the same as R glmnet but different from LIBSVM.") + } // Since we implicitly do the feature scaling when we compute the cost function // to improve the convergence, the effective regParam will be changed. @@ -251,14 +334,25 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String val effectiveL1RegParam = $(elasticNetParam) * effectiveRegParam val effectiveL2RegParam = (1.0 - $(elasticNetParam)) * effectiveRegParam - val costFun = new LeastSquaresCostFun(instances, yStd, yMean, $(fitIntercept), - $(standardization), featuresStd, featuresMean, effectiveL2RegParam) + val getAggregatorFunc = new LeastSquaresAggregator(yStd, yMean, $(fitIntercept), + bcFeaturesStd, bcFeaturesMean)(_) + val getFeaturesStd = (j: Int) => if (j >= 0 && j < numFeatures) featuresStd(j) else 0.0 + val regularization = if (effectiveL2RegParam != 0.0) { + val shouldApply = (idx: Int) => idx >= 0 && idx < numFeatures + Some(new L2Regularization(effectiveL2RegParam, shouldApply, + if ($(standardization)) None else Some(getFeaturesStd))) + } else { + None + } + val costFun = new RDDLossFunction(instances, getAggregatorFunc, regularization, + $(aggregationDepth)) val optimizer = if ($(elasticNetParam) == 0.0 || effectiveRegParam == 0.0) { new BreezeLBFGS[BDV[Double]]($(maxIter), 10, $(tol)) } else { + val standardizationParam = $(standardization) def effectiveL1RegFun = (index: Int) => { - if ($(standardization)) { + if (standardizationParam) { effectiveL1RegParam } else { // If `standardization` is false, we still standardize the data @@ -274,15 +368,18 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String val initialCoefficients = Vectors.zeros(numFeatures) val states = optimizer.iterations(new CachedDiffFunction(costFun), - initialCoefficients.toBreeze.toDenseVector) + initialCoefficients.asBreeze.toDenseVector) val (coefficients, objectiveHistory) = { /* Note that in Linear Regression, the objective history (loss + regularization) returned from optimizer is computed in the scaled space given by the following formula. - {{{ - L = 1/2n||\sum_i w_i(x_i - \bar{x_i}) / \hat{x_i} - (y - \bar{y}) / \hat{y}||^2 + regTerms - }}} +
    + $$ + L &= 1/2n||\sum_i w_i(x_i - \bar{x_i}) / \hat{x_i} - (y - \bar{y}) / \hat{y}||^2 + + regTerms \\ + $$ +
    */ val arrayBuilder = mutable.ArrayBuilder.make[Double] var state: optimizer.State = null @@ -296,6 +393,9 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String throw new SparkException(msg) } + bcFeaturesMean.destroy(blocking = false) + bcFeaturesStd.destroy(blocking = false) + /* The coefficients are trained in the scaled space; we're converting them back to the original space. @@ -332,35 +432,60 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String summaryModel.transform(dataset), predictionColName, $(labelCol), + $(featuresCol), model, Array(0D), - $(featuresCol), objectiveHistory) - model.setSummary(trainingSummary) + + model.setSummary(Some(trainingSummary)) + instr.logSuccess(model) + model } @Since("1.4.0") override def copy(extra: ParamMap): LinearRegression = defaultCopy(extra) } +@Since("1.6.0") +object LinearRegression extends DefaultParamsReadable[LinearRegression] { + + @Since("1.6.0") + override def load(path: String): LinearRegression = super.load(path) + + /** + * When using `LinearRegression.solver` == "normal", the solver must limit the number of + * features to at most this number. The entire covariance matrix X^T^X will be collected + * to the driver. This limit helps prevent memory overflow errors. + */ + @Since("2.1.0") + val MAX_FEATURES_FOR_NORMAL_SOLVER: Int = WeightedLeastSquares.MAX_NUM_FEATURES + + /** String name for "auto". */ + private[regression] val Auto = "auto" + + /** String name for "normal". */ + private[regression] val Normal = "normal" + + /** String name for "l-bfgs". */ + private[regression] val LBFGS = "l-bfgs" + + /** Set of solvers that LinearRegression supports. */ + private[regression] val supportedSolvers = Array(Auto, Normal, LBFGS) +} + /** - * :: Experimental :: * Model produced by [[LinearRegression]]. */ @Since("1.3.0") -@Experimental class LinearRegressionModel private[ml] ( - override val uid: String, - val coefficients: Vector, - val intercept: Double) + @Since("1.4.0") override val uid: String, + @Since("2.0.0") val coefficients: Vector, + @Since("1.3.0") val intercept: Double) extends RegressionModel[Vector, LinearRegressionModel] - with LinearRegressionParams { + with LinearRegressionParams with MLWritable { private var trainingSummary: Option[LinearRegressionTrainingSummary] = None - @deprecated("Use coefficients instead.", "1.6.0") - def weights: Vector = coefficients - override val numFeatures: Int = coefficients.size /** @@ -368,16 +493,13 @@ class LinearRegressionModel private[ml] ( * thrown if `trainingSummary == None`. */ @Since("1.5.0") - def summary: LinearRegressionTrainingSummary = trainingSummary match { - case Some(summ) => summ - case None => - throw new SparkException( - "No training summary available for this LinearRegressionModel", - new NullPointerException()) + def summary: LinearRegressionTrainingSummary = trainingSummary.getOrElse { + throw new SparkException("No training summary available for this LinearRegressionModel") } - private[regression] def setSummary(summary: LinearRegressionTrainingSummary): this.type = { - this.trainingSummary = Some(summary) + private[regression] + def setSummary(summary: Option[LinearRegressionTrainingSummary]): this.type = { + this.trainingSummary = summary this } @@ -386,15 +508,16 @@ class LinearRegressionModel private[ml] ( def hasSummary: Boolean = trainingSummary.isDefined /** - * Evaluates the model on a testset. + * Evaluates the model on a test dataset. + * * @param dataset Test dataset to evaluate model on. */ - // TODO: decide on a good name before exposing to public API - private[regression] def evaluate(dataset: DataFrame): LinearRegressionSummary = { + @Since("2.0.0") + def evaluate(dataset: Dataset[_]): LinearRegressionSummary = { // Handle possible missing or invalid prediction columns val (summaryModel, predictionColName) = findSummaryModelAndPredictionCol() new LinearRegressionSummary(summaryModel.transform(dataset), predictionColName, - $(labelCol), this, Array(0D)) + $(labelCol), $(featuresCol), summaryModel, Array(0D)) } /** @@ -405,7 +528,7 @@ class LinearRegressionModel private[ml] ( private[regression] def findSummaryModelAndPredictionCol(): (LinearRegressionModel, String) = { $(predictionCol) match { case "" => - val predictionColName = "prediction_" + java.util.UUID.randomUUID.toString() + val predictionColName = "prediction_" + java.util.UUID.randomUUID.toString (copy(ParamMap.empty).setPredictionCol(predictionColName), predictionColName) case p => (this, p) } @@ -419,16 +542,74 @@ class LinearRegressionModel private[ml] ( @Since("1.4.0") override def copy(extra: ParamMap): LinearRegressionModel = { val newModel = copyValues(new LinearRegressionModel(uid, coefficients, intercept), extra) - if (trainingSummary.isDefined) newModel.setSummary(trainingSummary.get) - newModel.setParent(parent) + newModel.setSummary(trainingSummary).setParent(parent) + } + + /** + * Returns a [[org.apache.spark.ml.util.MLWriter]] instance for this ML instance. + * + * For [[LinearRegressionModel]], this does NOT currently save the training [[summary]]. + * An option to save [[summary]] may be added in the future. + * + * This also does not save the [[parent]] currently. + */ + @Since("1.6.0") + override def write: MLWriter = new LinearRegressionModel.LinearRegressionModelWriter(this) +} + +@Since("1.6.0") +object LinearRegressionModel extends MLReadable[LinearRegressionModel] { + + @Since("1.6.0") + override def read: MLReader[LinearRegressionModel] = new LinearRegressionModelReader + + @Since("1.6.0") + override def load(path: String): LinearRegressionModel = super.load(path) + + /** [[MLWriter]] instance for [[LinearRegressionModel]] */ + private[LinearRegressionModel] class LinearRegressionModelWriter(instance: LinearRegressionModel) + extends MLWriter with Logging { + + private case class Data(intercept: Double, coefficients: Vector) + + override protected def saveImpl(path: String): Unit = { + // Save metadata and Params + DefaultParamsWriter.saveMetadata(instance, path, sc) + // Save model data: intercept, coefficients + val data = Data(instance.intercept, instance.coefficients) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class LinearRegressionModelReader extends MLReader[LinearRegressionModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[LinearRegressionModel].getName + + override def load(path: String): LinearRegressionModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.format("parquet").load(dataPath) + val Row(intercept: Double, coefficients: Vector) = + MLUtils.convertVectorColumnsToML(data, "coefficients") + .select("intercept", "coefficients") + .head() + val model = new LinearRegressionModel(metadata.uid, coefficients, intercept) + + DefaultParamsReader.getAndSetParams(model, metadata) + model + } } } /** * :: Experimental :: * Linear regression training results. Currently, the training summary ignores the - * training coefficients except for the objective trace. - * @param predictions predictions outputted by the model's `transform` method. + * training weights except for the objective trace. + * + * @param predictions predictions output by the model's `transform` method. * @param objectiveHistory objective function (scaled loss + regularization) at each iteration. */ @Since("1.5.0") @@ -437,13 +618,25 @@ class LinearRegressionTrainingSummary private[regression] ( predictions: DataFrame, predictionCol: String, labelCol: String, + featuresCol: String, model: LinearRegressionModel, diagInvAtWA: Array[Double], - val featuresCol: String, val objectiveHistory: Array[Double]) - extends LinearRegressionSummary(predictions, predictionCol, labelCol, model, diagInvAtWA) { + extends LinearRegressionSummary( + predictions, + predictionCol, + labelCol, + featuresCol, + model, + diagInvAtWA) { - /** Number of training iterations until termination */ + /** + * Number of training iterations until termination + * + * This value is only available when using the "l-bfgs" solver. + * + * @see `LinearRegression.solver` + */ @Since("1.5.0") val totalIterations = objectiveHistory.length @@ -452,7 +645,12 @@ class LinearRegressionTrainingSummary private[regression] ( /** * :: Experimental :: * Linear regression results evaluated on a dataset. - * @param predictions predictions outputted by the model's `transform` method. + * + * @param predictions predictions output by the model's `transform` method. + * @param predictionCol Field in "predictions" which gives the predicted value of the label at + * each instance. + * @param labelCol Field in "predictions" which gives the true label of each instance. + * @param featuresCol Field in "predictions" which gives the features of each instance as a vector. */ @Since("1.5.0") @Experimental @@ -460,18 +658,25 @@ class LinearRegressionSummary private[regression] ( @transient val predictions: DataFrame, val predictionCol: String, val labelCol: String, - val model: LinearRegressionModel, - val diagInvAtWA: Array[Double]) extends Serializable { + val featuresCol: String, + private val privateModel: LinearRegressionModel, + private val diagInvAtWA: Array[Double]) extends Serializable { @transient private val metrics = new RegressionMetrics( predictions - .select(predictionCol, labelCol) - .map { case Row(pred: Double, label: Double) => (pred, label) } ) + .select(col(predictionCol), col(labelCol).cast(DoubleType)) + .rdd + .map { case Row(pred: Double, label: Double) => (pred, label) }, + !privateModel.getFitIntercept) /** * Returns the explained variance regression score. * explainedVariance = 1 - variance(y - \hat{y}) / variance(y) - * Reference: [[http://en.wikipedia.org/wiki/Explained_variation]] + * Reference: + * Wikipedia explain variation + * + * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. + * This will change in later Spark versions. */ @Since("1.5.0") val explainedVariance: Double = metrics.explainedVariance @@ -479,6 +684,9 @@ class LinearRegressionSummary private[regression] ( /** * Returns the mean absolute error, which is a risk function corresponding to the * expected value of the absolute error loss or l1-norm loss. + * + * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. + * This will change in later Spark versions. */ @Since("1.5.0") val meanAbsoluteError: Double = metrics.meanAbsoluteError @@ -486,6 +694,9 @@ class LinearRegressionSummary private[regression] ( /** * Returns the mean squared error, which is a risk function corresponding to the * expected value of the squared error loss or quadratic loss. + * + * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. + * This will change in later Spark versions. */ @Since("1.5.0") val meanSquaredError: Double = metrics.meanSquaredError @@ -493,13 +704,20 @@ class LinearRegressionSummary private[regression] ( /** * Returns the root mean squared error, which is defined as the square root of * the mean squared error. + * + * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. + * This will change in later Spark versions. */ @Since("1.5.0") val rootMeanSquaredError: Double = metrics.rootMeanSquaredError /** * Returns R^2^, the coefficient of determination. - * Reference: [[http://en.wikipedia.org/wiki/Coefficient_of_determination]] + * Reference: + * Wikipedia coefficient of determination + * + * @note This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. + * This will change in later Spark versions. */ @Since("1.5.0") val r2: Double = metrics.r2 @@ -515,10 +733,11 @@ class LinearRegressionSummary private[regression] ( lazy val numInstances: Long = predictions.count() /** Degrees of freedom */ - private val degreesOfFreedom: Long = if (model.getFitIntercept) { - numInstances - model.coefficients.size - 1 + @Since("2.2.0") + val degreesOfFreedom: Long = if (privateModel.getFitIntercept) { + numInstances - privateModel.coefficients.size - 1 } else { - numInstances - model.coefficients.size + numInstances - privateModel.coefficients.size } /** @@ -526,9 +745,15 @@ class LinearRegressionSummary private[regression] ( * the square root of the instance weights. */ lazy val devianceResiduals: Array[Double] = { - val weighted = if (model.getWeightCol.isEmpty) lit(1.0) else sqrt(col(model.getWeightCol)) - val dr = predictions.select(col(model.getLabelCol).minus(col(model.getPredictionCol)) - .multiply(weighted).as("weightedResiduals")) + val weighted = + if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { + lit(1.0) + } else { + sqrt(col(privateModel.getWeightCol)) + } + val dr = predictions + .select(col(privateModel.getLabelCol).minus(col(privateModel.getPredictionCol)) + .multiply(weighted).as("weightedResiduals")) .select(min(col("weightedResiduals")).as("min"), max(col("weightedResiduals")).as("max")) .first() Array(dr.getDouble(0), dr.getDouble(1)) @@ -536,37 +761,50 @@ class LinearRegressionSummary private[regression] ( /** * Standard error of estimated coefficients and intercept. + * This value is only available when using the "normal" solver. + * + * If `LinearRegression.fitIntercept` is set to true, + * then the last element returned corresponds to the intercept. + * + * @see `LinearRegression.solver` */ lazy val coefficientStandardErrors: Array[Double] = { if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { throw new UnsupportedOperationException( "No Std. Error of coefficients available for this LinearRegressionModel") } else { - val rss = if (model.getWeightCol.isEmpty) { - meanSquaredError * numInstances - } else { - val t = udf { (pred: Double, label: Double, weight: Double) => - math.pow(label - pred, 2.0) * weight } - predictions.select(t(col(model.getPredictionCol), col(model.getLabelCol), - col(model.getWeightCol)).as("wse")).agg(sum(col("wse"))).first().getDouble(0) - } + val rss = + if (!privateModel.isDefined(privateModel.weightCol) || privateModel.getWeightCol.isEmpty) { + meanSquaredError * numInstances + } else { + val t = udf { (pred: Double, label: Double, weight: Double) => + math.pow(label - pred, 2.0) * weight } + predictions.select(t(col(privateModel.getPredictionCol), col(privateModel.getLabelCol), + col(privateModel.getWeightCol)).as("wse")).agg(sum(col("wse"))).first().getDouble(0) + } val sigma2 = rss / degreesOfFreedom - diagInvAtWA.map(_ * sigma2).map(math.sqrt(_)) + diagInvAtWA.map(_ * sigma2).map(math.sqrt) } } /** * T-statistic of estimated coefficients and intercept. + * This value is only available when using the "normal" solver. + * + * If `LinearRegression.fitIntercept` is set to true, + * then the last element returned corresponds to the intercept. + * + * @see `LinearRegression.solver` */ lazy val tValues: Array[Double] = { if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { throw new UnsupportedOperationException( "No t-statistic available for this LinearRegressionModel") } else { - val estimate = if (model.getFitIntercept) { - Array.concat(model.coefficients.toArray, Array(model.intercept)) + val estimate = if (privateModel.getFitIntercept) { + Array.concat(privateModel.coefficients.toArray, Array(privateModel.intercept)) } else { - model.coefficients.toArray + privateModel.coefficients.toArray } estimate.zip(coefficientStandardErrors).map { x => x._1 / x._2 } } @@ -574,6 +812,12 @@ class LinearRegressionSummary private[regression] ( /** * Two-sided p-value of estimated coefficients and intercept. + * This value is only available when using the "normal" solver. + * + * If `LinearRegression.fitIntercept` is set to true, + * then the last element returned corresponds to the intercept. + * + * @see `LinearRegression.solver` */ lazy val pValues: Array[Double] = { if (diagInvAtWA.length == 1 && diagInvAtWA(0) == 0) { @@ -586,270 +830,3 @@ class LinearRegressionSummary private[regression] ( } -/** - * LeastSquaresAggregator computes the gradient and loss for a Least-squared loss function, - * as used in linear regression for samples in sparse or dense vector in a online fashion. - * - * Two LeastSquaresAggregator can be merged together to have a summary of loss and gradient of - * the corresponding joint dataset. - * - * For improving the convergence rate during the optimization process, and also preventing against - * features with very large variances exerting an overly large influence during model training, - * package like R's GLMNET performs the scaling to unit variance and removing the mean to reduce - * the condition number, and then trains the model in scaled space but returns the coefficients in - * the original scale. See page 9 in http://cran.r-project.org/web/packages/glmnet/glmnet.pdf - * - * However, we don't want to apply the `StandardScaler` on the training dataset, and then cache - * the standardized dataset since it will create a lot of overhead. As a result, we perform the - * scaling implicitly when we compute the objective function. The following is the mathematical - * derivation. - * - * Note that we don't deal with intercept by adding bias here, because the intercept - * can be computed using closed form after the coefficients are converged. - * See this discussion for detail. - * http://stats.stackexchange.com/questions/13617/how-is-the-intercept-computed-in-glmnet - * - * When training with intercept enabled, - * The objective function in the scaled space is given by - * {{{ - * L = 1/2n ||\sum_i w_i(x_i - \bar{x_i}) / \hat{x_i} - (y - \bar{y}) / \hat{y}||^2, - * }}} - * where \bar{x_i} is the mean of x_i, \hat{x_i} is the standard deviation of x_i, - * \bar{y} is the mean of label, and \hat{y} is the standard deviation of label. - * - * If we fitting the intercept disabled (that is forced through 0.0), - * we can use the same equation except we set \bar{y} and \bar{x_i} to 0 instead - * of the respective means. - * - * This can be rewritten as - * {{{ - * L = 1/2n ||\sum_i (w_i/\hat{x_i})x_i - \sum_i (w_i/\hat{x_i})\bar{x_i} - y / \hat{y} - * + \bar{y} / \hat{y}||^2 - * = 1/2n ||\sum_i w_i^\prime x_i - y / \hat{y} + offset||^2 = 1/2n diff^2 - * }}} - * where w_i^\prime^ is the effective coefficients defined by w_i/\hat{x_i}, offset is - * {{{ - * - \sum_i (w_i/\hat{x_i})\bar{x_i} + \bar{y} / \hat{y}. - * }}}, and diff is - * {{{ - * \sum_i w_i^\prime x_i - y / \hat{y} + offset - * }}} - * - * - * Note that the effective coefficients and offset don't depend on training dataset, - * so they can be precomputed. - * - * Now, the first derivative of the objective function in scaled space is - * {{{ - * \frac{\partial L}{\partial\w_i} = diff/N (x_i - \bar{x_i}) / \hat{x_i} - * }}} - * However, ($x_i - \bar{x_i}$) will densify the computation, so it's not - * an ideal formula when the training dataset is sparse format. - * - * This can be addressed by adding the dense \bar{x_i} / \har{x_i} terms - * in the end by keeping the sum of diff. The first derivative of total - * objective function from all the samples is - * {{{ - * \frac{\partial L}{\partial\w_i} = - * 1/N \sum_j diff_j (x_{ij} - \bar{x_i}) / \hat{x_i} - * = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) - diffSum \bar{x_i}) / \hat{x_i}) - * = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) + correction_i) - * }}}, - * where correction_i = - diffSum \bar{x_i}) / \hat{x_i} - * - * A simple math can show that diffSum is actually zero, so we don't even - * need to add the correction terms in the end. From the definition of diff, - * {{{ - * diffSum = \sum_j (\sum_i w_i(x_{ij} - \bar{x_i}) / \hat{x_i} - (y_j - \bar{y}) / \hat{y}) - * = N * (\sum_i w_i(\bar{x_i} - \bar{x_i}) / \hat{x_i} - (\bar{y_j} - \bar{y}) / \hat{y}) - * = 0 - * }}} - * - * As a result, the first derivative of the total objective function only depends on - * the training dataset, which can be easily computed in distributed fashion, and is - * sparse format friendly. - * {{{ - * \frac{\partial L}{\partial\w_i} = 1/N ((\sum_j diff_j x_{ij} / \hat{x_i}) - * }}}, - * - * @param coefficients The coefficients corresponding to the features. - * @param labelStd The standard deviation value of the label. - * @param labelMean The mean value of the label. - * @param fitIntercept Whether to fit an intercept term. - * @param featuresStd The standard deviation values of the features. - * @param featuresMean The mean values of the features. - */ -private class LeastSquaresAggregator( - coefficients: Vector, - labelStd: Double, - labelMean: Double, - fitIntercept: Boolean, - featuresStd: Array[Double], - featuresMean: Array[Double]) extends Serializable { - - private var totalCnt: Long = 0L - private var weightSum: Double = 0.0 - private var lossSum = 0.0 - - private val (effectiveCoefficientsArray: Array[Double], offset: Double, dim: Int) = { - val coefficientsArray = coefficients.toArray.clone() - var sum = 0.0 - var i = 0 - val len = coefficientsArray.length - while (i < len) { - if (featuresStd(i) != 0.0) { - coefficientsArray(i) /= featuresStd(i) - sum += coefficientsArray(i) * featuresMean(i) - } else { - coefficientsArray(i) = 0.0 - } - i += 1 - } - val offset = if (fitIntercept) labelMean / labelStd - sum else 0.0 - (coefficientsArray, offset, coefficientsArray.length) - } - - private val effectiveCoefficientsVector = Vectors.dense(effectiveCoefficientsArray) - - private val gradientSumArray = Array.ofDim[Double](dim) - - /** - * Add a new training instance to this LeastSquaresAggregator, and update the loss and gradient - * of the objective function. - * - * @param instance The instance of data point to be added. - * @return This LeastSquaresAggregator object. - */ - def add(instance: Instance): this.type = { - instance match { case Instance(label, weight, features) => - require(dim == features.size, s"Dimensions mismatch when adding new sample." + - s" Expecting $dim but got ${features.size}.") - require(weight >= 0.0, s"instance weight, ${weight} has to be >= 0.0") - - if (weight == 0.0) return this - - val diff = dot(features, effectiveCoefficientsVector) - label / labelStd + offset - - if (diff != 0) { - val localGradientSumArray = gradientSumArray - features.foreachActive { (index, value) => - if (featuresStd(index) != 0.0 && value != 0.0) { - localGradientSumArray(index) += weight * diff * value / featuresStd(index) - } - } - lossSum += weight * diff * diff / 2.0 - } - - totalCnt += 1 - weightSum += weight - this - } - } - - /** - * Merge another LeastSquaresAggregator, and update the loss and gradient - * of the objective function. - * (Note that it's in place merging; as a result, `this` object will be modified.) - * - * @param other The other LeastSquaresAggregator to be merged. - * @return This LeastSquaresAggregator object. - */ - def merge(other: LeastSquaresAggregator): this.type = { - require(dim == other.dim, s"Dimensions mismatch when merging with another " + - s"LeastSquaresAggregator. Expecting $dim but got ${other.dim}.") - - if (other.weightSum != 0) { - totalCnt += other.totalCnt - weightSum += other.weightSum - lossSum += other.lossSum - - var i = 0 - val localThisGradientSumArray = this.gradientSumArray - val localOtherGradientSumArray = other.gradientSumArray - while (i < dim) { - localThisGradientSumArray(i) += localOtherGradientSumArray(i) - i += 1 - } - } - this - } - - def count: Long = totalCnt - - def loss: Double = { - require(weightSum > 0.0, s"The effective number of instances should be " + - s"greater than 0.0, but $weightSum.") - lossSum / weightSum - } - - def gradient: Vector = { - require(weightSum > 0.0, s"The effective number of instances should be " + - s"greater than 0.0, but $weightSum.") - val result = Vectors.dense(gradientSumArray.clone()) - scal(1.0 / weightSum, result) - result - } -} - -/** - * LeastSquaresCostFun implements Breeze's DiffFunction[T] for Least Squares cost. - * It returns the loss and gradient with L2 regularization at a particular point (coefficients). - * It's used in Breeze's convex optimization routines. - */ -private class LeastSquaresCostFun( - instances: RDD[Instance], - labelStd: Double, - labelMean: Double, - fitIntercept: Boolean, - standardization: Boolean, - featuresStd: Array[Double], - featuresMean: Array[Double], - effectiveL2regParam: Double) extends DiffFunction[BDV[Double]] { - - override def calculate(coefficients: BDV[Double]): (Double, BDV[Double]) = { - val coeffs = Vectors.fromBreeze(coefficients) - - val leastSquaresAggregator = { - val seqOp = (c: LeastSquaresAggregator, instance: Instance) => c.add(instance) - val combOp = (c1: LeastSquaresAggregator, c2: LeastSquaresAggregator) => c1.merge(c2) - - instances.treeAggregate( - new LeastSquaresAggregator(coeffs, labelStd, labelMean, fitIntercept, featuresStd, - featuresMean))(seqOp, combOp) - } - - val totalGradientArray = leastSquaresAggregator.gradient.toArray - - val regVal = if (effectiveL2regParam == 0.0) { - 0.0 - } else { - var sum = 0.0 - coeffs.foreachActive { (index, value) => - // The following code will compute the loss of the regularization; also - // the gradient of the regularization, and add back to totalGradientArray. - sum += { - if (standardization) { - totalGradientArray(index) += effectiveL2regParam * value - value * value - } else { - if (featuresStd(index) != 0.0) { - // If `standardization` is false, we still standardize the data - // to improve the rate of convergence; as a result, we have to - // perform this reverse standardization by penalizing each component - // differently to get effectively the same objective function when - // the training dataset is not standardized. - val temp = value / (featuresStd(index) * featuresStd(index)) - totalGradientArray(index) += effectiveL2regParam * temp - value * temp - } else { - 0.0 - } - } - } - } - 0.5 * effectiveL2regParam * sum - } - - (leastSquaresAggregator.loss + regVal, new BDV(totalGradientArray)) - } -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index 71e40b513ee0..a58da50fad97 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -17,31 +17,34 @@ package org.apache.spark.ml.regression -import org.apache.spark.annotation.{Experimental, Since} +import org.json4s.{DefaultFormats, JObject} +import org.json4s.JsonDSL._ + +import org.apache.spark.annotation.Since import org.apache.spark.ml.{PredictionModel, Predictor} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.tree.{DecisionTreeModel, RandomForestParams, TreeEnsembleModel, TreeRegressorParams} +import org.apache.spark.ml.tree._ import org.apache.spark.ml.tree.impl.RandomForest -import org.apache.spark.ml.util.{Identifiable, MetadataUtils} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions._ /** - * :: Experimental :: - * [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] learning algorithm for regression. + * Random Forest + * learning algorithm for regression. * It supports both continuous and categorical features. */ @Since("1.4.0") -@Experimental -final class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) +class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Predictor[Vector, RandomForestRegressor, RandomForestRegressionModel] - with RandomForestParams with TreeRegressorParams { + with RandomForestRegressorParams with DefaultParamsWritable { @Since("1.4.0") def this() = this(Identifiable.randomUID("rfr")) @@ -49,57 +52,88 @@ final class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val // Override parameter setters from parent trait for Java API compatibility. // Parameters from TreeRegressorParams: + + /** @group setParam */ @Since("1.4.0") - override def setMaxDepth(value: Int): this.type = super.setMaxDepth(value) + override def setMaxDepth(value: Int): this.type = set(maxDepth, value) + /** @group setParam */ @Since("1.4.0") - override def setMaxBins(value: Int): this.type = super.setMaxBins(value) + override def setMaxBins(value: Int): this.type = set(maxBins, value) + /** @group setParam */ @Since("1.4.0") - override def setMinInstancesPerNode(value: Int): this.type = - super.setMinInstancesPerNode(value) + override def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) + /** @group setParam */ @Since("1.4.0") - override def setMinInfoGain(value: Double): this.type = super.setMinInfoGain(value) + override def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) + /** @group expertSetParam */ @Since("1.4.0") - override def setMaxMemoryInMB(value: Int): this.type = super.setMaxMemoryInMB(value) + override def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) + /** @group expertSetParam */ @Since("1.4.0") - override def setCacheNodeIds(value: Boolean): this.type = super.setCacheNodeIds(value) + override def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) + /** + * Specifies how often to checkpoint the cached node IDs. + * E.g. 10 means that the cache will get checkpointed every 10 iterations. + * This is only used if cacheNodeIds is true and if the checkpoint directory is set in + * [[org.apache.spark.SparkContext]]. + * Must be at least 1. + * (default = 10) + * @group setParam + */ @Since("1.4.0") - override def setCheckpointInterval(value: Int): this.type = super.setCheckpointInterval(value) + override def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) + /** @group setParam */ @Since("1.4.0") - override def setImpurity(value: String): this.type = super.setImpurity(value) + override def setImpurity(value: String): this.type = set(impurity, value) // Parameters from TreeEnsembleParams: + + /** @group setParam */ @Since("1.4.0") - override def setSubsamplingRate(value: Double): this.type = super.setSubsamplingRate(value) + override def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) + /** @group setParam */ @Since("1.4.0") - override def setSeed(value: Long): this.type = super.setSeed(value) + override def setSeed(value: Long): this.type = set(seed, value) // Parameters from RandomForestParams: + + /** @group setParam */ @Since("1.4.0") - override def setNumTrees(value: Int): this.type = super.setNumTrees(value) + override def setNumTrees(value: Int): this.type = set(numTrees, value) + /** @group setParam */ @Since("1.4.0") override def setFeatureSubsetStrategy(value: String): this.type = - super.setFeatureSubsetStrategy(value) + set(featureSubsetStrategy, value) - override protected def train(dataset: DataFrame): RandomForestRegressionModel = { + override protected def train(dataset: Dataset[_]): RandomForestRegressionModel = { val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) val strategy = super.getOldStrategy(categoricalFeatures, numClasses = 0, OldAlgo.Regression, getOldImpurity) - val trees = - RandomForest.run(oldDataset, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed) - .map(_.asInstanceOf[DecisionTreeRegressionModel]) + + val instr = Instrumentation.create(this, oldDataset) + instr.logParams(labelCol, featuresCol, predictionCol, impurity, numTrees, + featureSubsetStrategy, maxDepth, maxBins, maxMemoryInMB, minInfoGain, + minInstancesPerNode, seed, subsamplingRate, cacheNodeIds, checkpointInterval) + + val trees = RandomForest + .run(oldDataset, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) + .map(_.asInstanceOf[DecisionTreeRegressionModel]) + val numFeatures = oldDataset.first().features.size - new RandomForestRegressionModel(trees, numFeatures) + val m = new RandomForestRegressionModel(uid, trees, numFeatures) + instr.logSuccess(m) + m } @Since("1.4.0") @@ -107,8 +141,7 @@ final class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val } @Since("1.4.0") -@Experimental -object RandomForestRegressor { +object RandomForestRegressor extends DefaultParamsReadable[RandomForestRegressor]{ /** Accessor for supported impurity settings: variance */ @Since("1.4.0") final val supportedImpurities: Array[String] = TreeRegressorParams.supportedImpurities @@ -117,44 +150,49 @@ object RandomForestRegressor { @Since("1.4.0") final val supportedFeatureSubsetStrategies: Array[String] = RandomForestParams.supportedFeatureSubsetStrategies + + @Since("2.0.0") + override def load(path: String): RandomForestRegressor = super.load(path) + } /** - * :: Experimental :: - * [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] model for regression. + * Random Forest model for regression. * It supports both continuous and categorical features. + * * @param _trees Decision trees in the ensemble. * @param numFeatures Number of features used by this model */ @Since("1.4.0") -@Experimental -final class RandomForestRegressionModel private[ml] ( +class RandomForestRegressionModel private[ml] ( override val uid: String, private val _trees: Array[DecisionTreeRegressionModel], override val numFeatures: Int) extends PredictionModel[Vector, RandomForestRegressionModel] - with TreeEnsembleModel with Serializable { + with RandomForestRegressorParams with TreeEnsembleModel[DecisionTreeRegressionModel] + with MLWritable with Serializable { - require(numTrees > 0, "RandomForestRegressionModel requires at least 1 tree.") + require(_trees.nonEmpty, "RandomForestRegressionModel requires at least 1 tree.") /** * Construct a random forest regression model, with all trees weighted equally. + * * @param trees Component trees */ private[ml] def this(trees: Array[DecisionTreeRegressionModel], numFeatures: Int) = this(Identifiable.randomUID("rfr"), trees, numFeatures) @Since("1.4.0") - override def trees: Array[DecisionTreeModel] = _trees.asInstanceOf[Array[DecisionTreeModel]] + override def trees: Array[DecisionTreeRegressionModel] = _trees // Note: We may add support for weights (based on tree performance) later on. - private lazy val _treeWeights: Array[Double] = Array.fill[Double](numTrees)(1.0) + private lazy val _treeWeights: Array[Double] = Array.fill[Double](_trees.length)(1.0) @Since("1.4.0") override def treeWeights: Array[Double] = _treeWeights - override protected def transformImpl(dataset: DataFrame): DataFrame = { - val bcastModel = dataset.sqlContext.sparkContext.broadcast(this) + override protected def transformImpl(dataset: Dataset[_]): DataFrame = { + val bcastModel = dataset.sparkSession.sparkContext.broadcast(this) val predictUDF = udf { (features: Any) => bcastModel.value.predict(features.asInstanceOf[Vector]) } @@ -165,7 +203,7 @@ final class RandomForestRegressionModel private[ml] ( // TODO: When we add a generic Bagging class, handle transform there. SPARK-7128 // Predict average of tree predictions. // Ignore the weights since all are 1.0 for now. - _trees.map(_.rootNode.predictImpl(features).prediction).sum / numTrees + _trees.map(_.rootNode.predictImpl(features).prediction).sum / getNumTrees } @Since("1.4.0") @@ -175,36 +213,83 @@ final class RandomForestRegressionModel private[ml] ( @Since("1.4.0") override def toString: String = { - s"RandomForestRegressionModel (uid=$uid) with $numTrees trees" + s"RandomForestRegressionModel (uid=$uid) with $getNumTrees trees" } /** * Estimate of the importance of each feature. * - * This generalizes the idea of "Gini" importance to other losses, - * following the explanation of Gini importance from "Random Forests" documentation - * by Leo Breiman and Adele Cutler, and following the implementation from scikit-learn. + * Each feature's importance is the average of its importance across all trees in the ensemble + * The importance vector is normalized to sum to 1. This method is suggested by Hastie et al. + * (Hastie, Tibshirani, Friedman. "The Elements of Statistical Learning, 2nd Edition." 2001.) + * and follows the implementation from scikit-learn. * - * This feature importance is calculated as follows: - * - Average over trees: - * - importance(feature j) = sum (over nodes which split on feature j) of the gain, - * where gain is scaled by the number of instances passing through node - * - Normalize importances for tree based on total number of training instances used - * to build tree. - * - Normalize feature importance vector to sum to 1. + * @see `DecisionTreeRegressionModel.featureImportances` */ - lazy val featureImportances: Vector = RandomForest.featureImportances(trees, numFeatures) + @Since("1.5.0") + lazy val featureImportances: Vector = TreeEnsembleModel.featureImportances(trees, numFeatures) /** (private[ml]) Convert to a model in the old API */ private[ml] def toOld: OldRandomForestModel = { new OldRandomForestModel(OldAlgo.Regression, _trees.map(_.toOld)) } + + @Since("2.0.0") + override def write: MLWriter = + new RandomForestRegressionModel.RandomForestRegressionModelWriter(this) } -private[ml] object RandomForestRegressionModel { +@Since("2.0.0") +object RandomForestRegressionModel extends MLReadable[RandomForestRegressionModel] { + + @Since("2.0.0") + override def read: MLReader[RandomForestRegressionModel] = new RandomForestRegressionModelReader + + @Since("2.0.0") + override def load(path: String): RandomForestRegressionModel = super.load(path) + + private[RandomForestRegressionModel] + class RandomForestRegressionModelWriter(instance: RandomForestRegressionModel) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val extraMetadata: JObject = Map( + "numFeatures" -> instance.numFeatures, + "numTrees" -> instance.getNumTrees) + EnsembleModelReadWrite.saveImpl(instance, path, sparkSession, extraMetadata) + } + } + + private class RandomForestRegressionModelReader extends MLReader[RandomForestRegressionModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[RandomForestRegressionModel].getName + private val treeClassName = classOf[DecisionTreeRegressionModel].getName + + override def load(path: String): RandomForestRegressionModel = { + implicit val format = DefaultFormats + val (metadata: Metadata, treesData: Array[(Metadata, Node)], treeWeights: Array[Double]) = + EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName) + val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] + val numTrees = (metadata.metadata \ "numTrees").extract[Int] + + val trees: Array[DecisionTreeRegressionModel] = treesData.map { case (treeMetadata, root) => + val tree = + new DecisionTreeRegressionModel(treeMetadata.uid, root, numFeatures) + DefaultParamsReader.getAndSetParams(tree, treeMetadata) + tree + } + require(numTrees == trees.length, s"RandomForestRegressionModel.load expected $numTrees" + + s" trees based on metadata but found ${trees.length} trees.") + + val model = new RandomForestRegressionModel(metadata.uid, trees, numFeatures) + DefaultParamsReader.getAndSetParams(model, metadata) + model + } + } - /** (private[ml]) Convert a model from the old API */ - def fromOld( + /** Convert a model from the old API */ + private[ml] def fromOld( oldModel: OldRandomForestModel, parent: RandomForestRegressor, categoricalFeatures: Map[Int, Int], @@ -215,6 +300,7 @@ private[ml] object RandomForestRegressionModel { // parent for each tree is null since there is no good way to set this. DecisionTreeRegressionModel.fromOld(tree, null, categoricalFeatures) } - new RandomForestRegressionModel(parent.uid, newTrees, numFeatures) + val uid = if (parent != null) parent.uid else Identifiable.randomUID("rfr") + new RandomForestRegressionModel(uid, newTrees, numFeatures) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala index c72ef2968032..c0a1683d3cb6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala @@ -18,19 +18,16 @@ package org.apache.spark.ml.regression import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.ml.{PredictionModel, PredictorParams, Predictor} +import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} /** - * :: DeveloperApi :: - * * Single-label regression * * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] * @tparam Learner Concrete Estimator type * @tparam M Concrete Model type */ -@DeveloperApi private[spark] abstract class Regressor[ FeaturesType, Learner <: Regressor[FeaturesType, Learner, M], @@ -43,7 +40,7 @@ private[spark] abstract class Regressor[ /** * :: DeveloperApi :: * - * Model produced by a [[Regressor]]. + * Model produced by a `Regressor`. * * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] * @tparam M Concrete Model type. diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMDataSource.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMDataSource.scala new file mode 100644 index 000000000000..e4de8483cfa3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMDataSource.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.source.libsvm + +/** + * `libsvm` package implements Spark SQL data source API for loading LIBSVM data as `DataFrame`. + * The loaded `DataFrame` has two columns: `label` containing labels stored as doubles and + * `features` containing feature vectors stored as `Vector`s. + * + * To use LIBSVM data source, you need to set "libsvm" as the format in `DataFrameReader` and + * optionally specify options, for example: + * {{{ + * // Scala + * val df = spark.read.format("libsvm") + * .option("numFeatures", "780") + * .load("data/mllib/sample_libsvm_data.txt") + * + * // Java + * Dataset df = spark.read().format("libsvm") + * .option("numFeatures, "780") + * .load("data/mllib/sample_libsvm_data.txt"); + * }}} + * + * LIBSVM data source supports the following options: + * - "numFeatures": number of features. + * If unspecified or nonpositive, the number of features will be determined automatically at the + * cost of one additional pass. + * This is also useful when the dataset is already split into multiple files and you want to load + * them separately, because some features may not present in certain files, which leads to + * inconsistent feature dimensions. + * - "vectorType": feature vector type, "sparse" (default) or "dense". + * + * @note This class is public for documentation purpose. Please don't use this class directly. + * Rather, use the data source API as illustrated above. + * + * @see LIBSVM datasets + */ +class LibSVMDataSource private() {} diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMOptions.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMOptions.scala new file mode 100644 index 000000000000..6900b4153a7e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMOptions.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.ml.source.libsvm + +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap + +/** + * Options for the LibSVM data source. + */ +private[libsvm] class LibSVMOptions(@transient private val parameters: CaseInsensitiveMap[String]) + extends Serializable { + + import LibSVMOptions._ + + def this(parameters: Map[String, String]) = this(CaseInsensitiveMap(parameters)) + + /** + * Number of features. If unspecified or nonpositive, the number of features will be determined + * automatically at the cost of one additional pass. + */ + val numFeatures = parameters.get(NUM_FEATURES).map(_.toInt).filter(_ > 0) + + val isSparse = parameters.getOrElse(VECTOR_TYPE, SPARSE_VECTOR_TYPE) match { + case SPARSE_VECTOR_TYPE => true + case DENSE_VECTOR_TYPE => false + case o => throw new IllegalArgumentException(s"Invalid value `$o` for parameter " + + s"`$VECTOR_TYPE`. Expected types are `sparse` and `dense`.") + } +} + +private[libsvm] object LibSVMOptions { + val NUM_FEATURES = "numFeatures" + val VECTOR_TYPE = "vectorType" + val DENSE_VECTOR_TYPE = "dense" + val SPARSE_VECTOR_TYPE = "sparse" +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 1f627777fc68..4e84ff044f55 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -17,100 +17,167 @@ package org.apache.spark.ml.source.libsvm -import com.google.common.base.Objects +import java.io.IOException -import org.apache.spark.Logging -import org.apache.spark.annotation.Since -import org.apache.spark.mllib.linalg.{Vector, VectorUDT} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} + +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vectors, VectorUDT} import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrameReader, DataFrame, Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.{DoubleType, StructField, StructType} - -/** - * LibSVMRelation provides the DataFrame constructed from LibSVM format data. - * @param path File path of LibSVM format - * @param numFeatures The number of features - * @param vectorType The type of vector. It can be 'sparse' or 'dense' - * @param sqlContext The Spark SQLContext - */ -private[libsvm] class LibSVMRelation(val path: String, val numFeatures: Int, val vectorType: String) - (@transient val sqlContext: SQLContext) - extends BaseRelation with TableScan with Logging with Serializable { - - override def schema: StructType = StructType( - StructField("label", DoubleType, nullable = false) :: - StructField("features", new VectorUDT(), nullable = false) :: Nil - ) - - override def buildScan(): RDD[Row] = { - val sc = sqlContext.sparkContext - val baseRdd = MLUtils.loadLibSVMFile(sc, path, numFeatures) - val sparse = vectorType == "sparse" - baseRdd.map { pt => - val features = if (sparse) pt.features.toSparse else pt.features.toDense - Row(pt.label, features) +import org.apache.spark.sql.types._ +import org.apache.spark.util.SerializableConfiguration + +private[libsvm] class LibSVMOutputWriter( + path: String, + dataSchema: StructType, + context: TaskAttemptContext) + extends OutputWriter { + + private val writer = CodecStreams.createOutputStreamWriter(context, new Path(path)) + + // This `asInstanceOf` is safe because it's guaranteed by `LibSVMFileFormat.verifySchema` + private val udt = dataSchema(1).dataType.asInstanceOf[VectorUDT] + + override def write(row: InternalRow): Unit = { + val label = row.getDouble(0) + val vector = udt.deserialize(row.getStruct(1, udt.sqlType.length)) + writer.write(label.toString) + vector.foreachActive { case (i, v) => + writer.write(s" ${i + 1}:$v") } - } - override def hashCode(): Int = { - Objects.hashCode(path, Double.box(numFeatures), vectorType) + writer.write('\n') } - override def equals(other: Any): Boolean = other match { - case that: LibSVMRelation => - path == that.path && - numFeatures == that.numFeatures && - vectorType == that.vectorType - case _ => - false + override def close(): Unit = { + writer.close() } } -/** - * `libsvm` package implements Spark SQL data source API for loading LIBSVM data as [[DataFrame]]. - * The loaded [[DataFrame]] has two columns: `label` containing labels stored as doubles and - * `features` containing feature vectors stored as [[Vector]]s. - * - * To use LIBSVM data source, you need to set "libsvm" as the format in [[DataFrameReader]] and - * optionally specify options, for example: - * {{{ - * // Scala - * val df = sqlContext.read.format("libsvm") - * .option("numFeatures", "780") - * .load("data/mllib/sample_libsvm_data.txt") - * - * // Java - * DataFrame df = sqlContext.read.format("libsvm") - * .option("numFeatures, "780") - * .load("data/mllib/sample_libsvm_data.txt"); - * }}} - * - * LIBSVM data source supports the following options: - * - "numFeatures": number of features. - * If unspecified or nonpositive, the number of features will be determined automatically at the - * cost of one additional pass. - * This is also useful when the dataset is already split into multiple files and you want to load - * them separately, because some features may not present in certain files, which leads to - * inconsistent feature dimensions. - * - "vectorType": feature vector type, "sparse" (default) or "dense". - * - * @see [[https://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/ LIBSVM datasets]] - */ -@Since("1.6.0") -class DefaultSource extends RelationProvider with DataSourceRegister { +/** @see [[LibSVMDataSource]] for public documentation. */ +// If this is moved or renamed, please update DataSource's backwardCompatibilityMap. +private[libsvm] class LibSVMFileFormat + extends TextBasedFileFormat + with DataSourceRegister + with Logging { - @Since("1.6.0") override def shortName(): String = "libsvm" - @Since("1.6.0") - override def createRelation(sqlContext: SQLContext, parameters: Map[String, String]) - : BaseRelation = { - val path = parameters.getOrElse("path", - throw new IllegalArgumentException("'path' must be specified")) - val numFeatures = parameters.getOrElse("numFeatures", "-1").toInt - val vectorType = parameters.getOrElse("vectorType", "sparse") - new LibSVMRelation(path, numFeatures, vectorType)(sqlContext) + override def toString: String = "LibSVM" + + private def verifySchema(dataSchema: StructType, forWriting: Boolean): Unit = { + if ( + dataSchema.size != 2 || + !dataSchema(0).dataType.sameType(DataTypes.DoubleType) || + !dataSchema(1).dataType.sameType(new VectorUDT()) || + !(forWriting || dataSchema(1).metadata.getLong(LibSVMOptions.NUM_FEATURES).toInt > 0) + ) { + throw new IOException(s"Illegal schema for libsvm data, schema=$dataSchema") + } + } + + override def inferSchema( + sparkSession: SparkSession, + options: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = { + val libSVMOptions = new LibSVMOptions(options) + val numFeatures: Int = libSVMOptions.numFeatures.getOrElse { + require(files.nonEmpty, "No input path specified for libsvm data") + logWarning( + "'numFeatures' option not specified, determining the number of features by going " + + "though the input. If you know the number in advance, please specify it via " + + "'numFeatures' option to avoid the extra scan.") + + val paths = files.map(_.getPath.toUri.toString) + val parsed = MLUtils.parseLibSVMFile(sparkSession, paths) + MLUtils.computeNumFeatures(parsed) + } + + val featuresMetadata = new MetadataBuilder() + .putLong(LibSVMOptions.NUM_FEATURES, numFeatures) + .build() + + Some( + StructType( + StructField("label", DoubleType, nullable = false) :: + StructField("features", new VectorUDT(), nullable = false, featuresMetadata) :: Nil)) + } + + override def prepareWrite( + sparkSession: SparkSession, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + verifySchema(dataSchema, true) + new OutputWriterFactory { + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new LibSVMOutputWriter(path, dataSchema, context) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + ".libsvm" + CodecStreams.getCompressionExtension(context) + } + } + } + + override def buildReader( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { + verifySchema(dataSchema, false) + val numFeatures = dataSchema("features").metadata.getLong(LibSVMOptions.NUM_FEATURES).toInt + assert(numFeatures > 0) + + val libSVMOptions = new LibSVMOptions(options) + val isSparse = libSVMOptions.isSparse + + val broadcastedHadoopConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + + (file: PartitionedFile) => { + val linesReader = new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => linesReader.close())) + + val points = linesReader + .map(_.toString.trim) + .filterNot(line => line.isEmpty || line.startsWith("#")) + .map { line => + val (label, indices, values) = MLUtils.parseLibSVMRecord(line) + LabeledPoint(label, Vectors.sparse(numFeatures, indices, values)) + } + + val converter = RowEncoder(dataSchema) + val fullOutput = dataSchema.map { f => + AttributeReference(f.name, f.dataType, f.nullable, f.metadata)() + } + val requiredOutput = fullOutput.filter { a => + requiredSchema.fieldNames.contains(a.name) + } + + val requiredColumns = GenerateUnsafeProjection.generate(requiredOutput, fullOutput) + + points.map { pt => + val features = if (isSparse) pt.features.toSparse else pt.features.toDense + requiredColumns(converter.toRow(Row(pt.label, features))) + } + } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/stat/ChiSquareTest.scala b/mllib/src/main/scala/org/apache/spark/ml/stat/ChiSquareTest.scala new file mode 100644 index 000000000000..5b38ca73e801 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/stat/ChiSquareTest.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.stat + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.ml.util.SchemaUtils +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} +import org.apache.spark.mllib.stat.{Statistics => OldStatistics} +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions.col + + +/** + * :: Experimental :: + * + * Chi-square hypothesis testing for categorical data. + * + * See Wikipedia for more information + * on the Chi-squared test. + */ +@Experimental +@Since("2.2.0") +object ChiSquareTest { + + /** Used to construct output schema of tests */ + private case class ChiSquareResult( + pValues: Vector, + degreesOfFreedom: Array[Int], + statistics: Vector) + + /** + * Conduct Pearson's independence test for every feature against the label. For each feature, the + * (feature, label) pairs are converted into a contingency matrix for which the Chi-squared + * statistic is computed. All label and feature values must be categorical. + * + * The null hypothesis is that the occurrence of the outcomes is statistically independent. + * + * @param dataset DataFrame of categorical labels and categorical features. + * Real-valued features will be treated as categorical for each distinct value. + * @param featuresCol Name of features column in dataset, of type `Vector` (`VectorUDT`) + * @param labelCol Name of label column in dataset, of any numerical type + * @return DataFrame containing the test result for every feature against the label. + * This DataFrame will contain a single Row with the following fields: + * - `pValues: Vector` + * - `degreesOfFreedom: Array[Int]` + * - `statistics: Vector` + * Each of these fields has one value per feature. + */ + @Since("2.2.0") + def test(dataset: DataFrame, featuresCol: String, labelCol: String): DataFrame = { + val spark = dataset.sparkSession + import spark.implicits._ + + SchemaUtils.checkColumnType(dataset.schema, featuresCol, new VectorUDT) + SchemaUtils.checkNumericType(dataset.schema, labelCol) + val rdd = dataset.select(col(labelCol).cast("double"), col(featuresCol)).as[(Double, Vector)] + .rdd.map { case (label, features) => OldLabeledPoint(label, OldVectors.fromML(features)) } + val testResults = OldStatistics.chiSqTest(rdd) + val pValues: Vector = Vectors.dense(testResults.map(_.pValue)) + val degreesOfFreedom: Array[Int] = testResults.map(_.degreesOfFreedom) + val statistics: Vector = Vectors.dense(testResults.map(_.statistic)) + spark.createDataFrame(Seq(ChiSquareResult(pValues, degreesOfFreedom, statistics))) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/stat/Correlation.scala b/mllib/src/main/scala/org/apache/spark/ml/stat/Correlation.scala new file mode 100644 index 000000000000..6e885d7c8aec --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/stat/Correlation.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.stat + +import scala.collection.JavaConverters._ + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.linalg.{SQLDataTypes, Vector} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.stat.{Statistics => OldStatistics} +import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.types.{StructField, StructType} + +/** + * API for correlation functions in MLlib, compatible with DataFrames and Datasets. + * + * The functions in this package generalize the functions in [[org.apache.spark.sql.Dataset#stat]] + * to spark.ml's Vector types. + */ +@Since("2.2.0") +@Experimental +object Correlation { + + /** + * :: Experimental :: + * Compute the correlation matrix for the input Dataset of Vectors using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * @param dataset A dataset or a dataframe + * @param column The name of the column of vectors for which the correlation coefficient needs + * to be computed. This must be a column of the dataset, and it must contain + * Vector objects. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return A dataframe that contains the correlation matrix of the column of vectors. This + * dataframe contains a single row and a single column of name + * '$METHODNAME($COLUMN)'. + * @throws IllegalArgumentException if the column is not a valid column in the dataset, or if + * the content of this column is not of type Vector. + * + * Here is how to access the correlation coefficient: + * {{{ + * val data: Dataset[Vector] = ... + * val Row(coeff: Matrix) = Correlation.corr(data, "value").head + * // coeff now contains the Pearson correlation matrix. + * }}} + * + * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column + * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], + * which is fairly costly. Cache the input Dataset before calling corr with `method = "spearman"` + * to avoid recomputing the common lineage. + */ + @Since("2.2.0") + def corr(dataset: Dataset[_], column: String, method: String): DataFrame = { + val rdd = dataset.select(column).rdd.map { + case Row(v: Vector) => OldVectors.fromML(v) + } + val oldM = OldStatistics.corr(rdd, method) + val name = s"$method($column)" + val schema = StructType(Array(StructField(name, SQLDataTypes.MatrixType, nullable = false))) + dataset.sparkSession.createDataFrame(Seq(Row(oldM.asML)).asJava, schema) + } + + /** + * Compute the Pearson correlation matrix for the input Dataset of Vectors. + */ + @Since("2.2.0") + def corr(dataset: Dataset[_], column: String): DataFrame = { + corr(dataset, column, "pearson") + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala new file mode 100644 index 000000000000..cae41edb7aca --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala @@ -0,0 +1,597 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.stat + +import java.io._ + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Expression, UnsafeArrayData} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete, TypedImperativeAggregate} +import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.types._ + +/** + * A builder object that provides summary statistics about a given column. + * + * Users should not directly create such builders, but instead use one of the methods in + * [[Summarizer]]. + */ +@Experimental +@Since("2.3.0") +sealed abstract class SummaryBuilder { + /** + * Returns an aggregate object that contains the summary of the column with the requested metrics. + * @param featuresCol a column that contains features Vector object. + * @param weightCol a column that contains weight value. + * @return an aggregate column that contains the statistics. The exact content of this + * structure is determined during the creation of the builder. + */ + @Since("2.3.0") + def summary(featuresCol: Column, weightCol: Column): Column + + @Since("2.3.0") + def summary(featuresCol: Column): Column = summary(featuresCol, lit(1.0)) +} + +/** + * Tools for vectorized statistics on MLlib Vectors. + * + * The methods in this package provide various statistics for Vectors contained inside DataFrames. + * + * This class lets users pick the statistics they would like to extract for a given column. Here is + * an example in Scala: + * {{{ + * val dataframe = ... // Some dataframe containing a feature column + * val allStats = dataframe.select(Summarizer.metrics("min", "max").summary($"features")) + * val Row(Row(min_, max_)) = allStats.first() + * }}} + * + * If one wants to get a single metric, shortcuts are also available: + * {{{ + * val meanDF = dataframe.select(Summarizer.mean($"features")) + * val Row(mean_) = meanDF.first() + * }}} + * + * Note: Currently, the performance of this interface is about 2x~3x slower then using the RDD + * interface. + */ +@Experimental +@Since("2.3.0") +object Summarizer extends Logging { + + import SummaryBuilderImpl._ + + /** + * Given a list of metrics, provides a builder that it turns computes metrics from a column. + * + * See the documentation of [[Summarizer]] for an example. + * + * The following metrics are accepted (case sensitive): + * - mean: a vector that contains the coefficient-wise mean. + * - variance: a vector tha contains the coefficient-wise variance. + * - count: the count of all vectors seen. + * - numNonzeros: a vector with the number of non-zeros for each coefficients + * - max: the maximum for each coefficient. + * - min: the minimum for each coefficient. + * - normL2: the Euclidian norm for each coefficient. + * - normL1: the L1 norm of each coefficient (sum of the absolute values). + * @param firstMetric the metric being provided + * @param metrics additional metrics that can be provided. + * @return a builder. + * @throws IllegalArgumentException if one of the metric names is not understood. + * + * Note: Currently, the performance of this interface is about 2x~3x slower then using the RDD + * interface. + */ + @Since("2.3.0") + def metrics(firstMetric: String, metrics: String*): SummaryBuilder = { + val (typedMetrics, computeMetrics) = getRelevantMetrics(Seq(firstMetric) ++ metrics) + new SummaryBuilderImpl(typedMetrics, computeMetrics) + } + + @Since("2.3.0") + def mean(col: Column): Column = getSingleMetric(col, "mean") + + @Since("2.3.0") + def variance(col: Column): Column = getSingleMetric(col, "variance") + + @Since("2.3.0") + def count(col: Column): Column = getSingleMetric(col, "count") + + @Since("2.3.0") + def numNonZeros(col: Column): Column = getSingleMetric(col, "numNonZeros") + + @Since("2.3.0") + def max(col: Column): Column = getSingleMetric(col, "max") + + @Since("2.3.0") + def min(col: Column): Column = getSingleMetric(col, "min") + + @Since("2.3.0") + def normL1(col: Column): Column = getSingleMetric(col, "normL1") + + @Since("2.3.0") + def normL2(col: Column): Column = getSingleMetric(col, "normL2") + + private def getSingleMetric(col: Column, metric: String): Column = { + val c1 = metrics(metric).summary(col) + c1.getField(metric).as(s"$metric($col)") + } +} + +private[ml] class SummaryBuilderImpl( + requestedMetrics: Seq[SummaryBuilderImpl.Metric], + requestedCompMetrics: Seq[SummaryBuilderImpl.ComputeMetric] + ) extends SummaryBuilder { + + override def summary(featuresCol: Column, weightCol: Column): Column = { + + val agg = SummaryBuilderImpl.MetricsAggregate( + requestedMetrics, + requestedCompMetrics, + featuresCol.expr, + weightCol.expr, + mutableAggBufferOffset = 0, + inputAggBufferOffset = 0) + + new Column(AggregateExpression(agg, mode = Complete, isDistinct = false)) + } +} + +private[ml] object SummaryBuilderImpl extends Logging { + + def implementedMetrics: Seq[String] = allMetrics.map(_._1).sorted + + @throws[IllegalArgumentException]("When the list is empty or not a subset of known metrics") + def getRelevantMetrics(requested: Seq[String]): (Seq[Metric], Seq[ComputeMetric]) = { + val all = requested.map { req => + val (_, metric, _, deps) = allMetrics.find(_._1 == req).getOrElse { + throw new IllegalArgumentException(s"Metric $req cannot be found." + + s" Valid metrics are $implementedMetrics") + } + metric -> deps + } + // Do not sort, otherwise the user has to look the schema to see the order that it + // is going to be given in. + val metrics = all.map(_._1) + val computeMetrics = all.flatMap(_._2).distinct.sortBy(_.toString) + metrics -> computeMetrics + } + + def structureForMetrics(metrics: Seq[Metric]): StructType = { + val dict = allMetrics.map { case (name, metric, dataType, _) => + (metric, (name, dataType)) + }.toMap + val fields = metrics.map(dict.apply).map { case (name, dataType) => + StructField(name, dataType, nullable = false) + } + StructType(fields) + } + + private val arrayDType = ArrayType(DoubleType, containsNull = false) + private val arrayLType = ArrayType(LongType, containsNull = false) + + /** + * All the metrics that can be currently computed by Spark for vectors. + * + * This list associates the user name, the internal (typed) name, and the list of computation + * metrics that need to de computed internally to get the final result. + */ + private val allMetrics: Seq[(String, Metric, DataType, Seq[ComputeMetric])] = Seq( + ("mean", Mean, arrayDType, Seq(ComputeMean, ComputeWeightSum)), + ("variance", Variance, arrayDType, Seq(ComputeWeightSum, ComputeMean, ComputeM2n)), + ("count", Count, LongType, Seq()), + ("numNonZeros", NumNonZeros, arrayLType, Seq(ComputeNNZ)), + ("max", Max, arrayDType, Seq(ComputeMax, ComputeNNZ)), + ("min", Min, arrayDType, Seq(ComputeMin, ComputeNNZ)), + ("normL2", NormL2, arrayDType, Seq(ComputeM2)), + ("normL1", NormL1, arrayDType, Seq(ComputeL1)) + ) + + /** + * The metrics that are currently implemented. + */ + sealed trait Metric extends Serializable + private[stat] case object Mean extends Metric + private[stat] case object Variance extends Metric + private[stat] case object Count extends Metric + private[stat] case object NumNonZeros extends Metric + private[stat] case object Max extends Metric + private[stat] case object Min extends Metric + private[stat] case object NormL2 extends Metric + private[stat] case object NormL1 extends Metric + + /** + * The running metrics that are going to be computed. + * + * There is a bipartite graph between the metrics and the computed metrics. + */ + sealed trait ComputeMetric extends Serializable + private[stat] case object ComputeMean extends ComputeMetric + private[stat] case object ComputeM2n extends ComputeMetric + private[stat] case object ComputeM2 extends ComputeMetric + private[stat] case object ComputeL1 extends ComputeMetric + private[stat] case object ComputeWeightSum extends ComputeMetric + private[stat] case object ComputeNNZ extends ComputeMetric + private[stat] case object ComputeMax extends ComputeMetric + private[stat] case object ComputeMin extends ComputeMetric + + private[stat] class SummarizerBuffer( + requestedMetrics: Seq[Metric], + requestedCompMetrics: Seq[ComputeMetric] + ) extends Serializable { + + private var n = 0 + private var currMean: Array[Double] = null + private var currM2n: Array[Double] = null + private var currM2: Array[Double] = null + private var currL1: Array[Double] = null + private var totalCnt: Long = 0 + private var totalWeightSum: Double = 0.0 + private var weightSquareSum: Double = 0.0 + private var weightSum: Array[Double] = null + private var nnz: Array[Long] = null + private var currMax: Array[Double] = null + private var currMin: Array[Double] = null + + def this() { + this( + Seq(Mean, Variance, Count, NumNonZeros, Max, Min, NormL2, NormL1), + Seq(ComputeMean, ComputeM2n, ComputeM2, ComputeL1, + ComputeWeightSum, ComputeNNZ, ComputeMax, ComputeMin) + ) + } + + /** + * Add a new sample to this summarizer, and update the statistical summary. + */ + def add(instance: Vector, weight: Double): this.type = { + require(weight >= 0.0, s"sample weight, $weight has to be >= 0.0") + if (weight == 0.0) return this + + if (n == 0) { + require(instance.size > 0, s"Vector should have dimension larger than zero.") + n = instance.size + + if (requestedCompMetrics.contains(ComputeMean)) { currMean = Array.ofDim[Double](n) } + if (requestedCompMetrics.contains(ComputeM2n)) { currM2n = Array.ofDim[Double](n) } + if (requestedCompMetrics.contains(ComputeM2)) { currM2 = Array.ofDim[Double](n) } + if (requestedCompMetrics.contains(ComputeL1)) { currL1 = Array.ofDim[Double](n) } + if (requestedCompMetrics.contains(ComputeWeightSum)) { weightSum = Array.ofDim[Double](n) } + if (requestedCompMetrics.contains(ComputeNNZ)) { nnz = Array.ofDim[Long](n) } + if (requestedCompMetrics.contains(ComputeMax)) { + currMax = Array.fill[Double](n)(Double.MinValue) + } + if (requestedCompMetrics.contains(ComputeMin)) { + currMin = Array.fill[Double](n)(Double.MaxValue) + } + } + + require(n == instance.size, s"Dimensions mismatch when adding new sample." + + s" Expecting $n but got ${instance.size}.") + + val localCurrMean = currMean + val localCurrM2n = currM2n + val localCurrM2 = currM2 + val localCurrL1 = currL1 + val localWeightSum = weightSum + val localNumNonzeros = nnz + val localCurrMax = currMax + val localCurrMin = currMin + instance.foreachActive { (index, value) => + if (value != 0.0) { + if (localCurrMax != null && localCurrMax(index) < value) { + localCurrMax(index) = value + } + if (localCurrMin != null && localCurrMin(index) > value) { + localCurrMin(index) = value + } + + if (localWeightSum != null) { + if (localCurrMean != null) { + val prevMean = localCurrMean(index) + val diff = value - prevMean + localCurrMean(index) = prevMean + weight * diff / (localWeightSum(index) + weight) + + if (localCurrM2n != null) { + localCurrM2n(index) += weight * (value - localCurrMean(index)) * diff + } + } + localWeightSum(index) += weight + } + + if (localCurrM2 != null) { + localCurrM2(index) += weight * value * value + } + if (localCurrL1 != null) { + localCurrL1(index) += weight * math.abs(value) + } + + if (localNumNonzeros != null) { + localNumNonzeros(index) += 1 + } + } + } + + totalWeightSum += weight + weightSquareSum += weight * weight + totalCnt += 1 + this + } + + def add(instance: Vector): this.type = add(instance, 1.0) + + /** + * Merge another SummarizerBuffer, and update the statistical summary. + * (Note that it's in place merging; as a result, `this` object will be modified.) + * + * @param other The other MultivariateOnlineSummarizer to be merged. + */ + def merge(other: SummarizerBuffer): this.type = { + if (this.totalWeightSum != 0.0 && other.totalWeightSum != 0.0) { + require(n == other.n, s"Dimensions mismatch when merging with another summarizer. " + + s"Expecting $n but got ${other.n}.") + totalCnt += other.totalCnt + totalWeightSum += other.totalWeightSum + weightSquareSum += other.weightSquareSum + var i = 0 + while (i < n) { + if (weightSum != null) { + val thisWeightSum = weightSum(i) + val otherWeightSum = other.weightSum(i) + val totalWeightSum = thisWeightSum + otherWeightSum + + if (totalWeightSum != 0.0) { + if (currMean != null) { + val deltaMean = other.currMean(i) - currMean(i) + // merge mean together + currMean(i) += deltaMean * otherWeightSum / totalWeightSum + + if (currM2n != null) { + // merge m2n together + currM2n(i) += other.currM2n(i) + + deltaMean * deltaMean * thisWeightSum * otherWeightSum / totalWeightSum + } + } + } + weightSum(i) = totalWeightSum + } + + // merge m2 together + if (currM2 != null) { currM2(i) += other.currM2(i) } + // merge l1 together + if (currL1 != null) { currL1(i) += other.currL1(i) } + // merge max and min + if (currMax != null) { currMax(i) = math.max(currMax(i), other.currMax(i)) } + if (currMin != null) { currMin(i) = math.min(currMin(i), other.currMin(i)) } + if (nnz != null) { nnz(i) = nnz(i) + other.nnz(i) } + i += 1 + } + } else if (totalWeightSum == 0.0 && other.totalWeightSum != 0.0) { + this.n = other.n + if (other.currMean != null) { this.currMean = other.currMean.clone() } + if (other.currM2n != null) { this.currM2n = other.currM2n.clone() } + if (other.currM2 != null) { this.currM2 = other.currM2.clone() } + if (other.currL1 != null) { this.currL1 = other.currL1.clone() } + this.totalCnt = other.totalCnt + this.totalWeightSum = other.totalWeightSum + this.weightSquareSum = other.weightSquareSum + if (other.weightSum != null) { this.weightSum = other.weightSum.clone() } + if (other.nnz != null) { this.nnz = other.nnz.clone() } + if (other.currMax != null) { this.currMax = other.currMax.clone() } + if (other.currMin != null) { this.currMin = other.currMin.clone() } + } + this + } + + /** + * Sample mean of each dimension. + */ + def mean: Vector = { + require(requestedMetrics.contains(Mean)) + require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") + + val realMean = Array.ofDim[Double](n) + var i = 0 + while (i < n) { + realMean(i) = currMean(i) * (weightSum(i) / totalWeightSum) + i += 1 + } + Vectors.dense(realMean) + } + + /** + * Unbiased estimate of sample variance of each dimension. + */ + def variance: Vector = { + require(requestedMetrics.contains(Variance)) + require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") + + val realVariance = Array.ofDim[Double](n) + + val denominator = totalWeightSum - (weightSquareSum / totalWeightSum) + + // Sample variance is computed, if the denominator is less than 0, the variance is just 0. + if (denominator > 0.0) { + val deltaMean = currMean + var i = 0 + val len = currM2n.length + while (i < len) { + // We prevent variance from negative value caused by numerical error. + realVariance(i) = math.max((currM2n(i) + deltaMean(i) * deltaMean(i) * weightSum(i) * + (totalWeightSum - weightSum(i)) / totalWeightSum) / denominator, 0.0) + i += 1 + } + } + Vectors.dense(realVariance) + } + + /** + * Sample size. + */ + def count: Long = totalCnt + + /** + * Number of nonzero elements in each dimension. + * + */ + def numNonzeros: Vector = { + require(requestedMetrics.contains(NumNonZeros)) + require(totalCnt > 0, s"Nothing has been added to this summarizer.") + + Vectors.dense(nnz.map(_.toDouble)) + } + + /** + * Maximum value of each dimension. + */ + def max: Vector = { + require(requestedMetrics.contains(Max)) + require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") + + var i = 0 + while (i < n) { + if ((nnz(i) < totalCnt) && (currMax(i) < 0.0)) currMax(i) = 0.0 + i += 1 + } + Vectors.dense(currMax) + } + + /** + * Minimum value of each dimension. + */ + def min: Vector = { + require(requestedMetrics.contains(Min)) + require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") + + var i = 0 + while (i < n) { + if ((nnz(i) < totalCnt) && (currMin(i) > 0.0)) currMin(i) = 0.0 + i += 1 + } + Vectors.dense(currMin) + } + + /** + * L2 (Euclidian) norm of each dimension. + */ + def normL2: Vector = { + require(requestedMetrics.contains(NormL2)) + require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") + + val realMagnitude = Array.ofDim[Double](n) + + var i = 0 + val len = currM2.length + while (i < len) { + realMagnitude(i) = math.sqrt(currM2(i)) + i += 1 + } + Vectors.dense(realMagnitude) + } + + /** + * L1 norm of each dimension. + */ + def normL1: Vector = { + require(requestedMetrics.contains(NormL1)) + require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") + + Vectors.dense(currL1) + } + } + + private case class MetricsAggregate( + requestedMetrics: Seq[Metric], + requestedComputeMetrics: Seq[ComputeMetric], + featuresExpr: Expression, + weightExpr: Expression, + mutableAggBufferOffset: Int, + inputAggBufferOffset: Int) + extends TypedImperativeAggregate[SummarizerBuffer] { + + override def eval(state: SummarizerBuffer): InternalRow = { + val metrics = requestedMetrics.map { + case Mean => UnsafeArrayData.fromPrimitiveArray(state.mean.toArray) + case Variance => UnsafeArrayData.fromPrimitiveArray(state.variance.toArray) + case Count => state.count + case NumNonZeros => UnsafeArrayData.fromPrimitiveArray( + state.numNonzeros.toArray.map(_.toLong)) + case Max => UnsafeArrayData.fromPrimitiveArray(state.max.toArray) + case Min => UnsafeArrayData.fromPrimitiveArray(state.min.toArray) + case NormL2 => UnsafeArrayData.fromPrimitiveArray(state.normL2.toArray) + case NormL1 => UnsafeArrayData.fromPrimitiveArray(state.normL1.toArray) + } + InternalRow.apply(metrics: _*) + } + + override def children: Seq[Expression] = featuresExpr :: weightExpr :: Nil + + override def update(state: SummarizerBuffer, row: InternalRow): SummarizerBuffer = { + val features = udt.deserialize(featuresExpr.eval(row)) + val weight = weightExpr.eval(row).asInstanceOf[Double] + state.add(features, weight) + state + } + + override def merge(state: SummarizerBuffer, + other: SummarizerBuffer): SummarizerBuffer = { + state.merge(other) + } + + override def nullable: Boolean = false + + override def createAggregationBuffer(): SummarizerBuffer + = new SummarizerBuffer(requestedMetrics, requestedComputeMetrics) + + override def serialize(state: SummarizerBuffer): Array[Byte] = { + // TODO: Use ByteBuffer to optimize + val bos = new ByteArrayOutputStream() + val oos = new ObjectOutputStream(bos) + oos.writeObject(state) + bos.toByteArray + } + + override def deserialize(bytes: Array[Byte]): SummarizerBuffer = { + // TODO: Use ByteBuffer to optimize + val bis = new ByteArrayInputStream(bytes) + val ois = new ObjectInputStream(bis) + ois.readObject().asInstanceOf[SummarizerBuffer] + } + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): MetricsAggregate = { + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + } + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): MetricsAggregate = { + copy(inputAggBufferOffset = newInputAggBufferOffset) + } + + override lazy val dataType: DataType = structureForMetrics(requestedMetrics) + + override def prettyName: String = "aggregate_metrics" + + } + + private[this] val udt = new VectorUDT + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala index d89682611e3f..07e98a142b10 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala @@ -17,17 +17,14 @@ package org.apache.spark.ml.tree -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.linalg.Vector import org.apache.spark.mllib.tree.impurity.ImpurityCalculator -import org.apache.spark.mllib.tree.model.{InformationGainStats => OldInformationGainStats, - Node => OldNode, Predict => OldPredict, ImpurityStats} +import org.apache.spark.mllib.tree.model.{ImpurityStats, + InformationGainStats => OldInformationGainStats, Node => OldNode, Predict => OldPredict} /** - * :: DeveloperApi :: * Decision tree node interface. */ -@DeveloperApi sealed abstract class Node extends Serializable { // TODO: Add aggregate stats (once available). This will happen after we move the DecisionTree @@ -78,6 +75,9 @@ sealed abstract class Node extends Serializable { * @return Max feature index used in a split, or -1 if there are no splits (single leaf node). */ private[ml] def maxSplitFeatureIndex(): Int + + /** Returns a deep copy of the subtree rooted at this node. */ + private[tree] def deepCopy(): Node } private[ml] object Node { @@ -106,13 +106,11 @@ private[ml] object Node { } /** - * :: DeveloperApi :: * Decision tree leaf node. * @param prediction Prediction this node makes * @param impurity Impurity measure at this node (for training data) */ -@DeveloperApi -final class LeafNode private[ml] ( +class LeafNode private[ml] ( override val prediction: Double, override val impurity: Double, override private[ml] val impurityStats: ImpurityCalculator) extends Node { @@ -137,21 +135,23 @@ final class LeafNode private[ml] ( } override private[ml] def maxSplitFeatureIndex(): Int = -1 + + override private[tree] def deepCopy(): Node = { + new LeafNode(prediction, impurity, impurityStats) + } } /** - * :: DeveloperApi :: * Internal Decision Tree node. * @param prediction Prediction this node would make if it were a leaf node * @param impurity Impurity measure at this node (for training data) - * @param gain Information gain value. - * Values < 0 indicate missing values; this quirk will be removed with future updates. + * @param gain Information gain value. Values less than 0 indicate missing values; + * this quirk will be removed with future updates. * @param leftChild Left-hand child node * @param rightChild Right-hand child node * @param split Information about the test used to split to the left or right child. */ -@DeveloperApi -final class InternalNode private[ml] ( +class InternalNode private[ml] ( override val prediction: Double, override val impurity: Double, val gain: Double, @@ -160,6 +160,9 @@ final class InternalNode private[ml] ( val split: Split, override private[ml] val impurityStats: ImpurityCalculator) extends Node { + // Note to developers: The constructor argument impurityStats should be reconsidered before we + // make the constructor public. We may be able to improve the representation. + override def toString: String = { s"InternalNode(prediction = $prediction, impurity = $impurity, split = $split)" } @@ -203,6 +206,11 @@ final class InternalNode private[ml] ( math.max(split.featureIndex, math.max(leftChild.maxSplitFeatureIndex(), rightChild.maxSplitFeatureIndex())) } + + override private[tree] def deepCopy(): Node = { + new InternalNode(prediction, impurity, gain, leftChild.deepCopy(), rightChild.deepCopy(), + split, impurityStats) + } } private object InternalNode { @@ -286,11 +294,12 @@ private[tree] class LearningNode( * * @param binnedFeatures Binned feature vector for data point. * @param splits possible splits for all features, indexed (numFeatures)(numSplits) - * @return Leaf index if the data point reaches a leaf. - * Otherwise, last node reachable in tree matching this example. - * Note: This is the global node index, i.e., the index used in the tree. - * This index is different from the index used during training a particular - * group of nodes on one call to [[findBestSplits()]]. + * @return Leaf index if the data point reaches a leaf. + * Otherwise, last node reachable in tree matching this example. + * Note: This is the global node index, i.e., the index used in the tree. + * This index is different from the index used during training a particular + * group of nodes on one call to + * [[org.apache.spark.ml.tree.impl.RandomForest.findBestSplits()]]. */ def predictImpl(binnedFeatures: Array[Int], splits: Array[Array[Split]]): Int = { if (this.isLeaf || this.split.isEmpty) { @@ -386,9 +395,9 @@ private[tree] object LearningNode { var levelsToGo = indexToLevel(nodeIndex) while (levelsToGo > 0) { if ((nodeIndex & (1 << levelsToGo - 1)) == 0) { - tmpNode = tmpNode.leftChild.asInstanceOf[LearningNode] + tmpNode = tmpNode.leftChild.get } else { - tmpNode = tmpNode.rightChild.asInstanceOf[LearningNode] + tmpNode = tmpNode.rightChild.get } levelsToGo -= 1 } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala index 78199cc2df58..dff44e2d49ec 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Split.scala @@ -17,18 +17,18 @@ package org.apache.spark.ml.tree -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.Vector +import java.util.Objects + +import org.apache.spark.annotation.Since +import org.apache.spark.ml.linalg.Vector import org.apache.spark.mllib.tree.configuration.{FeatureType => OldFeatureType} import org.apache.spark.mllib.tree.model.{Split => OldSplit} /** - * :: DeveloperApi :: * Interface for a "Split," which specifies a test made at a decision tree node * to choose the left or right path. */ -@DeveloperApi sealed trait Split extends Serializable { /** Index of feature which this split tests */ @@ -65,18 +65,16 @@ private[tree] object Split { } /** - * :: DeveloperApi :: * Split which tests a categorical feature. * @param featureIndex Index of the feature to test * @param _leftCategories If the feature value is in this set of categories, then the split goes * left. Otherwise, it goes right. * @param numCategories Number of categories for this feature. */ -@DeveloperApi -final class CategoricalSplit private[ml] ( +class CategoricalSplit private[ml] ( override val featureIndex: Int, _leftCategories: Array[Double], - private val numCategories: Int) + @Since("2.0.0") val numCategories: Int) extends Split { require(_leftCategories.forall(cat => 0 <= cat && cat < numCategories), "Invalid leftCategories" + @@ -112,12 +110,15 @@ final class CategoricalSplit private[ml] ( } } - override def equals(o: Any): Boolean = { - o match { - case other: CategoricalSplit => featureIndex == other.featureIndex && - isLeft == other.isLeft && categories == other.categories - case _ => false - } + override def hashCode(): Int = { + val state = Seq(featureIndex, isLeft, categories) + state.map(Objects.hashCode).foldLeft(0)((a, b) => 31 * a + b) + } + + override def equals(o: Any): Boolean = o match { + case other: CategoricalSplit => featureIndex == other.featureIndex && + isLeft == other.isLeft && categories == other.categories + case _ => false } override private[tree] def toOld: OldSplit = { @@ -148,14 +149,12 @@ final class CategoricalSplit private[ml] ( } /** - * :: DeveloperApi :: * Split which tests a continuous feature. * @param featureIndex Index of the feature to test - * @param threshold If the feature value is <= this threshold, then the split goes left. - * Otherwise, it goes right. + * @param threshold If the feature value is less than or equal to this threshold, then the + * split goes left. Otherwise, it goes right. */ -@DeveloperApi -final class ContinuousSplit private[ml] (override val featureIndex: Int, val threshold: Double) +class ContinuousSplit private[ml] (override val featureIndex: Int, val threshold: Double) extends Split { override private[ml] def shouldGoLeft(features: Vector): Boolean = { @@ -181,6 +180,11 @@ final class ContinuousSplit private[ml] (override val featureIndex: Int, val thr } } + override def hashCode(): Int = { + val state = Seq(featureIndex, threshold) + state.map(Objects.hashCode).foldLeft(0)((a, b) => 31 * a + b) + } + override private[tree] def toOld: OldSplit = { OldSplit(featureIndex, threshold, OldFeatureType.Continuous, List.empty[Double]) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala similarity index 99% rename from mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala rename to mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala index 572815df0bc4..4e372702f0c6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.tree.impl +package org.apache.spark.ml.tree.impl import org.apache.commons.math3.distribution.PoissonDistribution diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DTStatsAggregator.scala similarity index 77% rename from mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala rename to mllib/src/main/scala/org/apache/spark/ml/tree/impl/DTStatsAggregator.scala index 7985ed4b4c0f..5aeea1443d49 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DTStatsAggregator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.tree.impl +package org.apache.spark.ml.tree.impl import org.apache.spark.mllib.tree.impurity._ @@ -73,25 +73,34 @@ private[spark] class DTStatsAggregator( * Flat array of elements. * Index for start of stats for a (feature, bin) is: * index = featureOffsets(featureIndex) + binIndex * statsSize - * Note: For unordered features, - * the left child stats have binIndex in [0, numBins(featureIndex) / 2)) - * and the right child stats in [numBins(featureIndex) / 2), numBins(featureIndex)) */ private val allStats: Array[Double] = new Array[Double](allStatsSize) + /** + * Array of parent node sufficient stats. + * Note: parent stats need to be explicitly tracked in the [[DTStatsAggregator]] for unordered + * categorical features, because the parent [[Node]] object does not have [[ImpurityStats]] + * on the first iteration. + */ + private val parentStats: Array[Double] = new Array[Double](statsSize) /** * Get an [[ImpurityCalculator]] for a given (node, feature, bin). - * @param featureOffset For ordered features, this is a pre-computed (node, feature) offset + * + * @param featureOffset This is a pre-computed (node, feature) offset * from [[getFeatureOffset]]. - * For unordered features, this is a pre-computed - * (node, feature, left/right child) offset from - * [[getLeftRightFeatureOffsets]]. */ def getImpurityCalculator(featureOffset: Int, binIndex: Int): ImpurityCalculator = { impurityAggregator.getCalculator(allStats, featureOffset + binIndex * statsSize) } + /** + * Get an [[ImpurityCalculator]] for the parent node. + */ + def getParentImpurityCalculator(): ImpurityCalculator = { + impurityAggregator.getCalculator(parentStats, 0) + } + /** * Update the stats for a given (feature, bin) for ordered features, using the given label. */ @@ -100,14 +109,19 @@ private[spark] class DTStatsAggregator( impurityAggregator.update(allStats, i, label, instanceWeight) } + /** + * Update the parent node stats using the given label. + */ + def updateParent(label: Double, instanceWeight: Double): Unit = { + impurityAggregator.update(parentStats, 0, label, instanceWeight) + } + /** * Faster version of [[update]]. * Update the stats for a given (feature, bin), using the given label. - * @param featureOffset For ordered features, this is a pre-computed feature offset + * + * @param featureOffset This is a pre-computed feature offset * from [[getFeatureOffset]]. - * For unordered features, this is a pre-computed - * (feature, left/right child) offset from - * [[getLeftRightFeatureOffsets]]. */ def featureUpdate( featureOffset: Int, @@ -124,22 +138,11 @@ private[spark] class DTStatsAggregator( */ def getFeatureOffset(featureIndex: Int): Int = featureOffsets(featureIndex) - /** - * Pre-compute feature offset for use with [[featureUpdate]]. - * For unordered features only. - */ - def getLeftRightFeatureOffsets(featureIndex: Int): (Int, Int) = { - val baseOffset = featureOffsets(featureIndex) - (baseOffset, baseOffset + (numBins(featureIndex) >> 1) * statsSize) - } - /** * For a given feature, merge the stats for two bins. - * @param featureOffset For ordered features, this is a pre-computed feature offset + * + * @param featureOffset This is a pre-computed feature offset * from [[getFeatureOffset]]. - * For unordered features, this is a pre-computed - * (feature, left/right child) offset from - * [[getLeftRightFeatureOffsets]]. * @param binIndex The other bin is merged into this bin. * @param otherBinIndex This bin is not modified. */ @@ -162,6 +165,17 @@ private[spark] class DTStatsAggregator( allStats(i) += other.allStats(i) i += 1 } + + require(statsSize == other.statsSize, + s"DTStatsAggregator.merge requires that both aggregators have the same length parent " + + s"stats vectors. This aggregator's parent stats are length $statsSize, " + + s"but the other is ${other.statsSize}.") + var j = 0 + while (j < statsSize) { + parentStats(j) += other.parentStats(j) + j += 1 + } + this } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala similarity index 88% rename from mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala rename to mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala index 21ee49c45788..8a9dcb486b7b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala @@ -15,12 +15,14 @@ * limitations under the License. */ -package org.apache.spark.mllib.tree.impl +package org.apache.spark.ml.tree.impl import scala.collection.mutable +import scala.util.Try -import org.apache.spark.Logging -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.tree.RandomForestParams import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ import org.apache.spark.mllib.tree.configuration.Strategy @@ -33,7 +35,7 @@ import org.apache.spark.rdd.RDD * @param numClasses For classification: labels can take values {0, ..., numClasses - 1}. * For regression: fixed at 0 (no meaning). * @param maxBins Maximum number of bins, for all features. - * @param featureArity Map: categorical feature index --> arity. + * @param featureArity Map: categorical feature index to arity. * I.e., the feature takes values in {0, ..., arity - 1}. * @param numBins Number of bins for each feature. */ @@ -67,11 +69,11 @@ private[spark] class DecisionTreeMetadata( /** * Number of splits for the given feature. - * For unordered features, there are 2 bins per split. + * For unordered features, there is 1 bin per split. * For ordered features, there is 1 more bin than split. */ def numSplits(featureIndex: Int): Int = if (isUnordered(featureIndex)) { - numBins(featureIndex) >> 1 + numBins(featureIndex) } else { numBins(featureIndex) - 1 } @@ -111,6 +113,8 @@ private[spark] object DecisionTreeMetadata extends Logging { throw new IllegalArgumentException(s"DecisionTree requires size of input RDD > 0, " + s"but was given by empty one.") } + require(numFeatures > 0, s"DecisionTree requires number of features > 0, " + + s"but was given an empty features vector") val numExamples = input.count() val numClasses = strategy.algo match { case Classification => strategy.numClasses @@ -183,11 +187,23 @@ private[spark] object DecisionTreeMetadata extends Logging { } case _ => featureSubsetStrategy } + val numFeaturesPerNode: Int = _featureSubsetStrategy match { case "all" => numFeatures case "sqrt" => math.sqrt(numFeatures).ceil.toInt case "log2" => math.max(1, (math.log(numFeatures) / math.log(2)).ceil.toInt) case "onethird" => (numFeatures / 3.0).ceil.toInt + case _ => + Try(_featureSubsetStrategy.toInt).filter(_ > 0).toOption match { + case Some(value) => math.min(value, numFeatures) + case None => + Try(_featureSubsetStrategy.toDouble).filter(_ > 0).filter(_ <= 1.0).toOption match { + case Some(value) => math.ceil(value * numFeatures).toInt + case _ => throw new IllegalArgumentException(s"Supported values:" + + s" ${RandomForestParams.supportedFeatureSubsetStrategies.mkString(", ")}," + + s" (0.0-1.0], [1-n].") + } + } } new DecisionTreeMetadata(numFeatures, numExamples, numClasses, numBins.max, @@ -212,6 +228,6 @@ private[spark] object DecisionTreeMetadata extends Logging { * there are math.pow(2, arity - 1) - 1 such splits. * Each split has 2 corresponding bins. */ - def numUnorderedBins(arity: Int): Int = 2 * ((1 << arity - 1) - 1) + def numUnorderedBins(arity: Int): Int = (1 << arity - 1) - 1 } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala new file mode 100644 index 000000000000..e32447a79abb --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala @@ -0,0 +1,375 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.tree.impl + +import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} +import org.apache.spark.mllib.tree.configuration.{BoostingStrategy => OldBoostingStrategy} +import org.apache.spark.mllib.tree.impurity.{Variance => OldVariance} +import org.apache.spark.mllib.tree.loss.{Loss => OldLoss} +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.util.PeriodicRDDCheckpointer +import org.apache.spark.storage.StorageLevel + + +private[spark] object GradientBoostedTrees extends Logging { + + /** + * Method to train a gradient boosting model + * @param input Training dataset: RDD of `LabeledPoint`. + * @param seed Random seed. + * @return tuple of ensemble models and weights: + * (array of decision tree models, array of model weights) + */ + def run( + input: RDD[LabeledPoint], + boostingStrategy: OldBoostingStrategy, + seed: Long): (Array[DecisionTreeRegressionModel], Array[Double]) = { + val algo = boostingStrategy.treeStrategy.algo + algo match { + case OldAlgo.Regression => + GradientBoostedTrees.boost(input, input, boostingStrategy, validate = false, seed) + case OldAlgo.Classification => + // Map labels to -1, +1 so binary classification can be treated as regression. + val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) + GradientBoostedTrees.boost(remappedInput, remappedInput, boostingStrategy, validate = false, + seed) + case _ => + throw new IllegalArgumentException(s"$algo is not supported by gradient boosting.") + } + } + + /** + * Method to validate a gradient boosting model + * @param input Training dataset: RDD of `LabeledPoint`. + * @param validationInput Validation dataset. + * This dataset should be different from the training dataset, + * but it should follow the same distribution. + * E.g., these two datasets could be created from an original dataset + * by using `org.apache.spark.rdd.RDD.randomSplit()` + * @param seed Random seed. + * @return tuple of ensemble models and weights: + * (array of decision tree models, array of model weights) + */ + def runWithValidation( + input: RDD[LabeledPoint], + validationInput: RDD[LabeledPoint], + boostingStrategy: OldBoostingStrategy, + seed: Long): (Array[DecisionTreeRegressionModel], Array[Double]) = { + val algo = boostingStrategy.treeStrategy.algo + algo match { + case OldAlgo.Regression => + GradientBoostedTrees.boost(input, validationInput, boostingStrategy, validate = true, seed) + case OldAlgo.Classification => + // Map labels to -1, +1 so binary classification can be treated as regression. + val remappedInput = input.map( + x => new LabeledPoint((x.label * 2) - 1, x.features)) + val remappedValidationInput = validationInput.map( + x => new LabeledPoint((x.label * 2) - 1, x.features)) + GradientBoostedTrees.boost(remappedInput, remappedValidationInput, boostingStrategy, + validate = true, seed) + case _ => + throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.") + } + } + + /** + * Compute the initial predictions and errors for a dataset for the first + * iteration of gradient boosting. + * @param data: training data. + * @param initTreeWeight: learning rate assigned to the first tree. + * @param initTree: first DecisionTreeModel. + * @param loss: evaluation metric. + * @return an RDD with each element being a zip of the prediction and error + * corresponding to every sample. + */ + def computeInitialPredictionAndError( + data: RDD[LabeledPoint], + initTreeWeight: Double, + initTree: DecisionTreeRegressionModel, + loss: OldLoss): RDD[(Double, Double)] = { + data.map { lp => + val pred = updatePrediction(lp.features, 0.0, initTree, initTreeWeight) + val error = loss.computeError(pred, lp.label) + (pred, error) + } + } + + /** + * Update a zipped predictionError RDD + * (as obtained with computeInitialPredictionAndError) + * @param data: training data. + * @param predictionAndError: predictionError RDD + * @param treeWeight: Learning rate. + * @param tree: Tree using which the prediction and error should be updated. + * @param loss: evaluation metric. + * @return an RDD with each element being a zip of the prediction and error + * corresponding to each sample. + */ + def updatePredictionError( + data: RDD[LabeledPoint], + predictionAndError: RDD[(Double, Double)], + treeWeight: Double, + tree: DecisionTreeRegressionModel, + loss: OldLoss): RDD[(Double, Double)] = { + + val newPredError = data.zip(predictionAndError).mapPartitions { iter => + iter.map { case (lp, (pred, error)) => + val newPred = updatePrediction(lp.features, pred, tree, treeWeight) + val newError = loss.computeError(newPred, lp.label) + (newPred, newError) + } + } + newPredError + } + + /** + * Add prediction from a new boosting iteration to an existing prediction. + * + * @param features Vector of features representing a single data point. + * @param prediction The existing prediction. + * @param tree New Decision Tree model. + * @param weight Tree weight. + * @return Updated prediction. + */ + def updatePrediction( + features: Vector, + prediction: Double, + tree: DecisionTreeRegressionModel, + weight: Double): Double = { + prediction + tree.rootNode.predictImpl(features).prediction * weight + } + + /** + * Method to calculate error of the base learner for the gradient boosting calculation. + * Note: This method is not used by the gradient boosting algorithm but is useful for debugging + * purposes. + * @param data Training dataset: RDD of `LabeledPoint`. + * @param trees Boosted Decision Tree models + * @param treeWeights Learning rates at each boosting iteration. + * @param loss evaluation metric. + * @return Measure of model error on data + */ + def computeError( + data: RDD[LabeledPoint], + trees: Array[DecisionTreeRegressionModel], + treeWeights: Array[Double], + loss: OldLoss): Double = { + data.map { lp => + val predicted = trees.zip(treeWeights).foldLeft(0.0) { case (acc, (model, weight)) => + updatePrediction(lp.features, acc, model, weight) + } + loss.computeError(predicted, lp.label) + }.mean() + } + + /** + * Method to compute error or loss for every iteration of gradient boosting. + * + * @param data RDD of `LabeledPoint` + * @param trees Boosted Decision Tree models + * @param treeWeights Learning rates at each boosting iteration. + * @param loss evaluation metric. + * @param algo algorithm for the ensemble, either Classification or Regression + * @return an array with index i having the losses or errors for the ensemble + * containing the first i+1 trees + */ + def evaluateEachIteration( + data: RDD[LabeledPoint], + trees: Array[DecisionTreeRegressionModel], + treeWeights: Array[Double], + loss: OldLoss, + algo: OldAlgo.Value): Array[Double] = { + + val sc = data.sparkContext + val remappedData = algo match { + case OldAlgo.Classification => data.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) + case _ => data + } + + val broadcastTrees = sc.broadcast(trees) + val localTreeWeights = treeWeights + val treesIndices = trees.indices + + val dataCount = remappedData.count() + val evaluation = remappedData.map { point => + treesIndices.map { idx => + val prediction = broadcastTrees.value(idx) + .rootNode + .predictImpl(point.features) + .prediction + prediction * localTreeWeights(idx) + } + .scanLeft(0.0)(_ + _).drop(1) + .map(prediction => loss.computeError(prediction, point.label)) + } + .aggregate(treesIndices.map(_ => 0.0))( + (aggregated, row) => treesIndices.map(idx => aggregated(idx) + row(idx)), + (a, b) => treesIndices.map(idx => a(idx) + b(idx))) + .map(_ / dataCount) + + broadcastTrees.destroy(blocking = false) + evaluation.toArray + } + + /** + * Internal method for performing regression using trees as base learners. + * @param input training dataset + * @param validationInput validation dataset, ignored if validate is set to false. + * @param boostingStrategy boosting parameters + * @param validate whether or not to use the validation dataset. + * @param seed Random seed. + * @return tuple of ensemble models and weights: + * (array of decision tree models, array of model weights) + */ + def boost( + input: RDD[LabeledPoint], + validationInput: RDD[LabeledPoint], + boostingStrategy: OldBoostingStrategy, + validate: Boolean, + seed: Long): (Array[DecisionTreeRegressionModel], Array[Double]) = { + val timer = new TimeTracker() + timer.start("total") + timer.start("init") + + boostingStrategy.assertValid() + + // Initialize gradient boosting parameters + val numIterations = boostingStrategy.numIterations + val baseLearners = new Array[DecisionTreeRegressionModel](numIterations) + val baseLearnerWeights = new Array[Double](numIterations) + val loss = boostingStrategy.loss + val learningRate = boostingStrategy.learningRate + // Prepare strategy for individual trees, which use regression with variance impurity. + val treeStrategy = boostingStrategy.treeStrategy.copy + val validationTol = boostingStrategy.validationTol + treeStrategy.algo = OldAlgo.Regression + treeStrategy.impurity = OldVariance + treeStrategy.assertValid() + + // Cache input + val persistedInput = if (input.getStorageLevel == StorageLevel.NONE) { + input.persist(StorageLevel.MEMORY_AND_DISK) + true + } else { + false + } + + // Prepare periodic checkpointers + val predErrorCheckpointer = new PeriodicRDDCheckpointer[(Double, Double)]( + treeStrategy.getCheckpointInterval, input.sparkContext) + val validatePredErrorCheckpointer = new PeriodicRDDCheckpointer[(Double, Double)]( + treeStrategy.getCheckpointInterval, input.sparkContext) + + timer.stop("init") + + logDebug("##########") + logDebug("Building tree 0") + logDebug("##########") + + // Initialize tree + timer.start("building tree 0") + val firstTree = new DecisionTreeRegressor().setSeed(seed) + val firstTreeModel = firstTree.train(input, treeStrategy) + val firstTreeWeight = 1.0 + baseLearners(0) = firstTreeModel + baseLearnerWeights(0) = firstTreeWeight + + var predError: RDD[(Double, Double)] = + computeInitialPredictionAndError(input, firstTreeWeight, firstTreeModel, loss) + predErrorCheckpointer.update(predError) + logDebug("error of gbt = " + predError.values.mean()) + + // Note: A model of type regression is used since we require raw prediction + timer.stop("building tree 0") + + var validatePredError: RDD[(Double, Double)] = + computeInitialPredictionAndError(validationInput, firstTreeWeight, firstTreeModel, loss) + if (validate) validatePredErrorCheckpointer.update(validatePredError) + var bestValidateError = if (validate) validatePredError.values.mean() else 0.0 + var bestM = 1 + + var m = 1 + var doneLearning = false + while (m < numIterations && !doneLearning) { + // Update data with pseudo-residuals + val data = predError.zip(input).map { case ((pred, _), point) => + LabeledPoint(-loss.gradient(pred, point.label), point.features) + } + + timer.start(s"building tree $m") + logDebug("###################################################") + logDebug("Gradient boosting tree iteration " + m) + logDebug("###################################################") + val dt = new DecisionTreeRegressor().setSeed(seed + m) + val model = dt.train(data, treeStrategy) + timer.stop(s"building tree $m") + // Update partial model + baseLearners(m) = model + // Note: The setting of baseLearnerWeights is incorrect for losses other than SquaredError. + // Technically, the weight should be optimized for the particular loss. + // However, the behavior should be reasonable, though not optimal. + baseLearnerWeights(m) = learningRate + + predError = updatePredictionError( + input, predError, baseLearnerWeights(m), baseLearners(m), loss) + predErrorCheckpointer.update(predError) + logDebug("error of gbt = " + predError.values.mean()) + + if (validate) { + // Stop training early if + // 1. Reduction in error is less than the validationTol or + // 2. If the error increases, that is if the model is overfit. + // We want the model returned corresponding to the best validation error. + + validatePredError = updatePredictionError( + validationInput, validatePredError, baseLearnerWeights(m), baseLearners(m), loss) + validatePredErrorCheckpointer.update(validatePredError) + val currentValidateError = validatePredError.values.mean() + if (bestValidateError - currentValidateError < validationTol * Math.max( + currentValidateError, 0.01)) { + doneLearning = true + } else if (currentValidateError < bestValidateError) { + bestValidateError = currentValidateError + bestM = m + 1 + } + } + m += 1 + } + + timer.stop("total") + + logInfo("Internal timing for DecisionTree:") + logInfo(s"$timer") + + predErrorCheckpointer.unpersistDataSet() + predErrorCheckpointer.deleteAllCheckpoints() + validatePredErrorCheckpointer.unpersistDataSet() + validatePredErrorCheckpointer.deleteAllCheckpoints() + if (persistedInput) input.unpersist() + + if (validate) { + (baseLearners.slice(0, bestM), baseLearnerWeights.slice(0, bestM)) + } else { + (baseLearners, baseLearnerWeights) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala index 1ee01131d633..a7c5f489dea8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala @@ -21,12 +21,10 @@ import java.io.IOException import scala.collection.mutable -import org.apache.hadoop.fs.{Path, FileSystem} +import org.apache.hadoop.fs.Path -import org.apache.spark.Logging -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.ml.tree.{LearningNode, Split} -import org.apache.spark.mllib.tree.impl.BaggedPoint import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -79,8 +77,8 @@ private[spark] class NodeIdCache( // Indicates whether we can checkpoint private val canCheckpoint = nodeIdsForInstances.sparkContext.getCheckpointDir.nonEmpty - // FileSystem instance for deleting checkpoints as needed - private val fs = FileSystem.get(nodeIdsForInstances.sparkContext.hadoopConfiguration) + // Hadoop Configuration for deleting checkpoints as needed + private val hadoopConf = nodeIdsForInstances.sparkContext.hadoopConfiguration /** * Update the node index values in the cache. @@ -132,7 +130,9 @@ private[spark] class NodeIdCache( val old = checkpointQueue.dequeue() // Since the old checkpoint is not deleted by Spark, we'll manually delete it here. try { - fs.delete(new Path(old.getCheckpointFile.get), true) + val path = new Path(old.getCheckpointFile.get) + val fs = path.getFileSystem(hadoopConf) + fs.delete(path, true) } catch { case e: IOException => logError("Decision Tree learning using cacheNodeIds failed to remove checkpoint" + @@ -156,7 +156,9 @@ private[spark] class NodeIdCache( val old = checkpointQueue.dequeue() if (old.getCheckpointFile.isDefined) { try { - fs.delete(new Path(old.getCheckpointFile.get), true) + val path = new Path(old.getCheckpointFile.get) + val fs = path.getFileSystem(hadoopConf) + fs.delete(path, true) } catch { case e: IOException => logError("Decision Tree learning using cacheNodeIds failed to remove checkpoint" + @@ -171,7 +173,6 @@ private[spark] class NodeIdCache( } } -@DeveloperApi private[spark] object NodeIdCache { /** * Initialize the node Id cache with initial node Id values. diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index 4a3b12d1440b..acfc6399c553 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -22,28 +22,67 @@ import java.io.IOException import scala.collection.mutable import scala.util.Random -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ml.classification.DecisionTreeClassificationModel +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree._ -import org.apache.spark.mllib.linalg.{Vectors, Vector} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.util.Instrumentation import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} -import org.apache.spark.mllib.tree.impl.{BaggedPoint, DTStatsAggregator, DecisionTreeMetadata, - TimeTracker} import org.apache.spark.mllib.tree.impurity.ImpurityCalculator import org.apache.spark.mllib.tree.model.ImpurityStats import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.random.{SamplingUtils, XORShiftRandom} -private[ml] object RandomForest extends Logging { +/** + * ALGORITHM + * + * This is a sketch of the algorithm to help new developers. + * + * The algorithm partitions data by instances (rows). + * On each iteration, the algorithm splits a set of nodes. In order to choose the best split + * for a given node, sufficient statistics are collected from the distributed data. + * For each node, the statistics are collected to some worker node, and that worker selects + * the best split. + * + * This setup requires discretization of continuous features. This binning is done in the + * findSplits() method during initialization, after which each continuous feature becomes + * an ordered discretized feature with at most maxBins possible values. + * + * The main loop in the algorithm operates on a queue of nodes (nodeStack). These nodes + * lie at the periphery of the tree being trained. If multiple trees are being trained at once, + * then this queue contains nodes from all of them. Each iteration works roughly as follows: + * On the master node: + * - Some number of nodes are pulled off of the queue (based on the amount of memory + * required for their sufficient statistics). + * - For random forests, if featureSubsetStrategy is not "all," then a subset of candidate + * features are chosen for each node. See method selectNodesToSplit(). + * On worker nodes, via method findBestSplits(): + * - The worker makes one pass over its subset of instances. + * - For each (tree, node, feature, split) tuple, the worker collects statistics about + * splitting. Note that the set of (tree, node) pairs is limited to the nodes selected + * from the queue for this iteration. The set of features considered can also be limited + * based on featureSubsetStrategy. + * - For each node, the statistics for that node are aggregated to a particular worker + * via reduceByKey(). The designated worker chooses the best (feature, split) pair, + * or chooses to stop splitting if the stopping criteria are met. + * On the master node: + * - The master collects all decisions about splitting nodes and updates the model. + * - The updated model is passed to the workers on the next iteration. + * This process continues until the node queue is empty. + * + * Most of the methods in this implementation support the statistics aggregation, which is + * the heaviest part of the computation. In general, this implementation is bound by either + * the cost of statistics computation on workers or by communicating the sufficient statistics. + */ +private[spark] object RandomForest extends Logging { /** * Train a random forest. - * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] + * + * @param input Training data: RDD of `LabeledPoint` * @return an unweighted set of trees */ def run( @@ -52,6 +91,7 @@ private[ml] object RandomForest extends Logging { numTrees: Int, featureSubsetStrategy: String, seed: Long, + instr: Option[Instrumentation[_]], parentUID: Option[String] = None): Array[DecisionTreeModel] = { val timer = new TimeTracker() @@ -63,19 +103,20 @@ private[ml] object RandomForest extends Logging { val retaggedInput = input.retag(classOf[LabeledPoint]) val metadata = DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy) - logDebug("algo = " + strategy.algo) - logDebug("numTrees = " + numTrees) - logDebug("seed = " + seed) - logDebug("maxBins = " + metadata.maxBins) - logDebug("featureSubsetStrategy = " + featureSubsetStrategy) - logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode) - logDebug("subsamplingRate = " + strategy.subsamplingRate) + instr match { + case Some(instrumentation) => + instrumentation.logNumFeatures(metadata.numFeatures) + instrumentation.logNumClasses(metadata.numClasses) + case None => + logInfo("numFeatures: " + metadata.numFeatures) + logInfo("numClasses: " + metadata.numClasses) + } // Find the splits and the corresponding bins (interval between the splits) using a sample // of the input data. - timer.start("findSplitsBins") + timer.start("findSplits") val splits = findSplits(retaggedInput, metadata, seed) - timer.stop("findSplitsBins") + timer.stop("findSplits") logDebug("numBins: feature: number of bins") logDebug(Range(0, metadata.numFeatures).map { featureIndex => s"\t$featureIndex\t${metadata.numBins(featureIndex)}" @@ -100,22 +141,6 @@ private[ml] object RandomForest extends Logging { // TODO: Calculate memory usage more precisely. val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") - val maxMemoryPerNode = { - val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) { - // Find numFeaturesPerNode largest bins to get an upper bound on memory usage. - Some(metadata.numBins.zipWithIndex.sortBy(- _._1) - .take(metadata.numFeaturesPerNode).map(_._2)) - } else { - None - } - RandomForest.aggregateSizeForNode(metadata, featureSubset) * 8L - } - require(maxMemoryPerNode <= maxMemoryUsage, - s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," + - " which is too small for the given features." + - s" Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}") - - timer.stop("init") /* * The main idea here is to perform group-wise training of the decision tree nodes thus @@ -136,29 +161,42 @@ private[ml] object RandomForest extends Logging { None } - // FIFO queue of nodes to train: (treeIndex, node) - val nodeQueue = new mutable.Queue[(Int, LearningNode)]() + /* + Stack of nodes to train: (treeIndex, node) + The reason this is a stack is that we train many trees at once, but we want to focus on + completing trees, rather than training all simultaneously. If we are splitting nodes from + 1 tree, then the new nodes to split will be put at the top of this stack, so we will continue + training the same tree in the next iteration. This focus allows us to send fewer trees to + workers on each iteration; see topNodesForGroup below. + */ + val nodeStack = new mutable.ArrayStack[(Int, LearningNode)] val rng = new Random() rng.setSeed(seed) // Allocate and queue root nodes. val topNodes = Array.fill[LearningNode](numTrees)(LearningNode.emptyNode(nodeIndex = 1)) - Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex)))) + Range(0, numTrees).foreach(treeIndex => nodeStack.push((treeIndex, topNodes(treeIndex)))) - while (nodeQueue.nonEmpty) { + timer.stop("init") + + while (nodeStack.nonEmpty) { // Collect some nodes to split, and choose features for each node (if subsampling). // Each group of nodes may come from one or multiple trees, and at multiple levels. val (nodesForGroup, treeToNodeToIndexInfo) = - RandomForest.selectNodesToSplit(nodeQueue, maxMemoryUsage, metadata, rng) + RandomForest.selectNodesToSplit(nodeStack, maxMemoryUsage, metadata, rng) // Sanity check (should never occur): assert(nodesForGroup.nonEmpty, s"RandomForest selected empty nodesForGroup. Error for unknown reason.") + // Only send trees to worker if they contain nodes being split this iteration. + val topNodesForGroup: Map[Int, LearningNode] = + nodesForGroup.keys.map(treeIdx => treeIdx -> topNodes(treeIdx)).toMap + // Choose node splits, and enqueue new nodes as needed. timer.start("findBestSplits") - RandomForest.findBestSplits(baggedInput, metadata, topNodes, nodesForGroup, - treeToNodeToIndexInfo, splits, nodeQueue, timer, nodeIdCache) + RandomForest.findBestSplits(baggedInput, metadata, topNodesForGroup, nodesForGroup, + treeToNodeToIndexInfo, splits, nodeStack, timer, nodeIdCache) timer.stop("findBestSplits") } @@ -244,8 +282,7 @@ private[ml] object RandomForest extends Logging { if (unorderedFeatures.contains(featureIndex)) { // Unordered feature val featureValue = treePoint.binnedFeatures(featureIndex) - val (leftNodeFeatureOffset, rightNodeFeatureOffset) = - agg.getLeftRightFeatureOffsets(featureIndexIdx) + val leftNodeFeatureOffset = agg.getFeatureOffset(featureIndexIdx) // Update the left or right bin for each split. val numSplits = agg.metadata.numSplits(featureIndex) val featureSplits = splits(featureIndex) @@ -253,8 +290,6 @@ private[ml] object RandomForest extends Logging { while (splitIndex < numSplits) { if (featureSplits(splitIndex).shouldGoLeft(featureValue, featureSplits)) { agg.featureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label, instanceWeight) - } else { - agg.featureUpdate(rightNodeFeatureOffset, splitIndex, treePoint.label, instanceWeight) } splitIndex += 1 } @@ -308,15 +343,16 @@ private[ml] object RandomForest extends Logging { /** * Given a group of nodes, this finds the best split for each node. * - * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] + * @param input Training data: RDD of [[TreePoint]] * @param metadata Learning and dataset metadata - * @param topNodes Root node for each tree. Used for matching instances with nodes. + * @param topNodesForGroup For each tree in group, tree index -> root node. + * Used for matching instances with nodes. * @param nodesForGroup Mapping: treeIndex --> nodes to be split in tree * @param treeToNodeToIndexInfo Mapping: treeIndex --> nodeIndex --> nodeIndexInfo, * where nodeIndexInfo stores the index in the group and the * feature subsets (if using feature subsets). * @param splits possible splits for all features, indexed (numFeatures)(numSplits) - * @param nodeQueue Queue of nodes to split, with values (treeIndex, node). + * @param nodeStack Queue of nodes to split, with values (treeIndex, node). * Updated with new non-leaf nodes which are created. * @param nodeIdCache Node Id cache containing an RDD of Array[Int] where * each value in the array is the data point's node Id @@ -327,11 +363,11 @@ private[ml] object RandomForest extends Logging { private[tree] def findBestSplits( input: RDD[BaggedPoint[TreePoint]], metadata: DecisionTreeMetadata, - topNodes: Array[LearningNode], + topNodesForGroup: Map[Int, LearningNode], nodesForGroup: Map[Int, Array[LearningNode]], treeToNodeToIndexInfo: Map[Int, Map[Int, NodeIndexInfo]], splits: Array[Array[Split]], - nodeQueue: mutable.Queue[(Int, LearningNode)], + nodeStack: mutable.ArrayStack[(Int, LearningNode)], timer: TimeTracker = new TimeTracker, nodeIdCache: Option[NodeIdCache] = None): Unit = { @@ -394,6 +430,7 @@ private[ml] object RandomForest extends Logging { mixedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, splits, metadata.unorderedFeatures, instanceWeight, featuresForNode) } + agg(aggNodeIndex).updateParent(baggedPoint.datum.label, instanceWeight) } } @@ -412,7 +449,8 @@ private[ml] object RandomForest extends Logging { agg: Array[DTStatsAggregator], baggedPoint: BaggedPoint[TreePoint]): Array[DTStatsAggregator] = { treeToNodeToIndexInfo.foreach { case (treeIndex, nodeIndexToInfo) => - val nodeIndex = topNodes(treeIndex).predictImpl(baggedPoint.datum.binnedFeatures, splits) + val nodeIndex = + topNodesForGroup(treeIndex).predictImpl(baggedPoint.datum.binnedFeatures, splits) nodeBinSeqOp(treeIndex, nodeIndexToInfo.getOrElse(nodeIndex, null), agg, baggedPoint) } agg @@ -466,7 +504,7 @@ private[ml] object RandomForest extends Logging { timer.start("chooseSplits") // In each partition, iterate all instances and compute aggregate stats for each node, - // yield an (nodeIndex, nodeAggregateStats) pair for each node. + // yield a (nodeIndex, nodeAggregateStats) pair for each node. // After a `reduceByKey` operation, // stats of a node will be shuffled to a particular partition and be combined together, // then best splits for nodes are found there. @@ -474,13 +512,13 @@ private[ml] object RandomForest extends Logging { val nodeToFeatures = getNodeToFeatures(treeToNodeToIndexInfo) val nodeToFeaturesBc = input.sparkContext.broadcast(nodeToFeatures) - val partitionAggregates : RDD[(Int, DTStatsAggregator)] = if (nodeIdCache.nonEmpty) { + val partitionAggregates: RDD[(Int, DTStatsAggregator)] = if (nodeIdCache.nonEmpty) { input.zip(nodeIdCache.get.nodeIdsForInstances).mapPartitions { points => // Construct a nodeStatsAggregators array to hold node aggregate stats, // each node will have a nodeStatsAggregator val nodeStatsAggregators = Array.tabulate(numNodes) { nodeIndex => - val featuresForNode = nodeToFeaturesBc.value.flatMap { nodeToFeatures => - Some(nodeToFeatures(nodeIndex)) + val featuresForNode = nodeToFeaturesBc.value.map { nodeToFeatures => + nodeToFeatures(nodeIndex) } new DTStatsAggregator(metadata, featuresForNode) } @@ -568,10 +606,10 @@ private[ml] object RandomForest extends Logging { // enqueue left child and right child if they are not leaves if (!leftChildIsLeaf) { - nodeQueue.enqueue((treeIndex, node.leftChild.get)) + nodeStack.push((treeIndex, node.leftChild.get)) } if (!rightChildIsLeaf) { - nodeQueue.enqueue((treeIndex, node.rightChild.get)) + nodeStack.push((treeIndex, node.rightChild.get)) } logDebug("leftChildIndex = " + node.leftChild.get.id + @@ -589,7 +627,9 @@ private[ml] object RandomForest extends Logging { } /** - * Calculate the impurity statistics for a give (feature, split) based upon left/right aggregates. + * Calculate the impurity statistics for a given (feature, split) based upon left/right + * aggregates. + * * @param stats the recycle impurity statistics for this feature's all splits, * only 'impurity' and 'impurityCalculator' are valid between each iteration * @param leftImpurityCalculator left node aggregates for this (feature, split) @@ -647,10 +687,11 @@ private[ml] object RandomForest extends Logging { /** * Find the best split for a node. + * * @param binAggregates Bin statistics. * @return tuple for best split: (Split, information gain, prediction at node) */ - private def binsToBestSplit( + private[tree] def binsToBestSplit( binAggregates: DTStatsAggregator, splits: Array[Array[Split]], featuresForNode: Option[Array[Int]], @@ -658,20 +699,23 @@ private[ml] object RandomForest extends Logging { // Calculate InformationGain and ImpurityStats if current node is top node val level = LearningNode.indexToLevel(node.id) - var gainAndImpurityStats: ImpurityStats = if (level ==0) { + var gainAndImpurityStats: ImpurityStats = if (level == 0) { null } else { node.stats } + val validFeatureSplits = + Range(0, binAggregates.metadata.numFeaturesPerNode).view.map { featureIndexIdx => + featuresForNode.map(features => (featureIndexIdx, features(featureIndexIdx))) + .getOrElse((featureIndexIdx, featureIndexIdx)) + }.withFilter { case (_, featureIndex) => + binAggregates.metadata.numSplits(featureIndex) != 0 + } + // For each (feature, split), calculate the gain, and select the best (feature, split). - val (bestSplit, bestSplitStats) = - Range(0, binAggregates.metadata.numFeaturesPerNode).map { featureIndexIdx => - val featureIndex = if (featuresForNode.nonEmpty) { - featuresForNode.get.apply(featureIndexIdx) - } else { - featureIndexIdx - } + val splitsAndImpurityInfo = + validFeatureSplits.map { case (featureIndexIdx, featureIndex) => val numSplits = binAggregates.metadata.numSplits(featureIndex) if (binAggregates.metadata.isContinuous(featureIndex)) { // Cumulative sum (scanLeft) of bin statistics. @@ -697,13 +741,12 @@ private[ml] object RandomForest extends Logging { (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) } else if (binAggregates.metadata.isUnordered(featureIndex)) { // Unordered categorical feature - val (leftChildOffset, rightChildOffset) = - binAggregates.getLeftRightFeatureOffsets(featureIndexIdx) + val leftChildOffset = binAggregates.getFeatureOffset(featureIndexIdx) val (bestFeatureSplitIndex, bestFeatureGainStats) = Range(0, numSplits).map { splitIndex => val leftChildStats = binAggregates.getImpurityCalculator(leftChildOffset, splitIndex) - val rightChildStats = - binAggregates.getImpurityCalculator(rightChildOffset, splitIndex) + val rightChildStats = binAggregates.getParentImpurityCalculator() + .subtract(leftChildStats) gainAndImpurityStats = calculateImpurityStats(gainAndImpurityStats, leftChildStats, rightChildStats, binAggregates.metadata) (splitIndex, gainAndImpurityStats) @@ -720,32 +763,30 @@ private[ml] object RandomForest extends Logging { * * centroidForCategories is a list: (category, centroid) */ - val centroidForCategories = if (binAggregates.metadata.isMulticlass) { - // For categorical variables in multiclass classification, - // the bins are ordered by the impurity of their corresponding labels. - Range(0, numCategories).map { case featureValue => - val categoryStats = - binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) - val centroid = if (categoryStats.count != 0) { + val centroidForCategories = Range(0, numCategories).map { case featureValue => + val categoryStats = + binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) + val centroid = if (categoryStats.count != 0) { + if (binAggregates.metadata.isMulticlass) { + // multiclass classification + // For categorical variables in multiclass classification, + // the bins are ordered by the impurity of their corresponding labels. categoryStats.calculate() + } else if (binAggregates.metadata.isClassification) { + // binary classification + // For categorical variables in binary classification, + // the bins are ordered by the count of class 1. + categoryStats.stats(1) } else { - Double.MaxValue - } - (featureValue, centroid) - } - } else { // regression or binary classification - // For categorical variables in regression and binary classification, - // the bins are ordered by the centroid of their corresponding labels. - Range(0, numCategories).map { case featureValue => - val categoryStats = - binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) - val centroid = if (categoryStats.count != 0) { + // regression + // For categorical variables in regression and binary classification, + // the bins are ordered by the prediction. categoryStats.predict - } else { - Double.MaxValue } - (featureValue, centroid) + } else { + Double.MaxValue } + (featureValue, centroid) } logDebug("Centroids for categorical variable: " + centroidForCategories.mkString(",")) @@ -787,13 +828,31 @@ private[ml] object RandomForest extends Logging { new CategoricalSplit(featureIndex, categoriesForSplit.toArray, numCategories) (bestFeatureSplit, bestFeatureGainStats) } - }.maxBy(_._2.gain) + } + val (bestSplit, bestSplitStats) = + if (splitsAndImpurityInfo.isEmpty) { + // If no valid splits for features, then this split is invalid, + // return invalid information gain stats. Take any split and continue. + // Splits is empty, so arbitrarily choose to split on any threshold + val dummyFeatureIndex = featuresForNode.map(_.head).getOrElse(0) + val parentImpurityCalculator = binAggregates.getParentImpurityCalculator() + if (binAggregates.metadata.isContinuous(dummyFeatureIndex)) { + (new ContinuousSplit(dummyFeatureIndex, 0), + ImpurityStats.getInvalidImpurityStats(parentImpurityCalculator)) + } else { + val numCategories = binAggregates.metadata.featureArity(dummyFeatureIndex) + (new CategoricalSplit(dummyFeatureIndex, Array(), numCategories), + ImpurityStats.getInvalidImpurityStats(parentImpurityCalculator)) + } + } else { + splitsAndImpurityInfo.maxBy(_._2.gain) + } (bestSplit, bestSplitStats) } /** - * Returns splits and bins for decision tree calculation. + * Returns splits for decision tree calculation. * Continuous and categorical features are handled differently. * * Continuous features: @@ -813,27 +872,24 @@ private[ml] object RandomForest extends Logging { * and for multiclass classification with a high-arity feature, * there is one bin per category. * - * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] + * @param input Training data: RDD of [[LabeledPoint]] * @param metadata Learning and dataset metadata * @param seed random seed - * @return A tuple of (splits, bins). - * Splits is an Array of [[org.apache.spark.mllib.tree.model.Split]] - * of size (numFeatures, numSplits). - * Bins is an Array of [[org.apache.spark.mllib.tree.model.Bin]] - * of size (numFeatures, numBins). + * @return Splits, an Array of [[Split]] + * of size (numFeatures, numSplits) */ protected[tree] def findSplits( input: RDD[LabeledPoint], metadata: DecisionTreeMetadata, - seed : Long): Array[Array[Split]] = { + seed: Long): Array[Array[Split]] = { logDebug("isMulticlass = " + metadata.isMulticlass) val numFeatures = metadata.numFeatures // Sample the input only if there are continuous features. - val hasContinuousFeatures = Range(0, numFeatures).exists(metadata.isContinuous) - val sampledInput = if (hasContinuousFeatures) { + val continuousFeatures = Range(0, numFeatures).filter(metadata.isContinuous) + val sampledInput = if (continuousFeatures.nonEmpty) { // Calculate the number of samples for approximate quantile calculation. val requiredSamples = math.max(metadata.maxBins * metadata.maxBins, 10000) val fraction = if (requiredSamples < metadata.numExamples) { @@ -842,58 +898,56 @@ private[ml] object RandomForest extends Logging { 1.0 } logDebug("fraction of data used for calculating quantiles = " + fraction) - input.sample(withReplacement = false, fraction, new XORShiftRandom(seed).nextInt()).collect() + input.sample(withReplacement = false, fraction, new XORShiftRandom(seed).nextInt()) } else { - new Array[LabeledPoint](0) + input.sparkContext.emptyRDD[LabeledPoint] } - val splits = new Array[Array[Split]](numFeatures) - - // Find all splits. - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - if (metadata.isContinuous(featureIndex)) { - val featureSamples = sampledInput.map(_.features(featureIndex)) - val featureSplits = findSplitsForContinuousFeature(featureSamples, metadata, featureIndex) + findSplitsBySorting(sampledInput, metadata, continuousFeatures) + } - val numSplits = featureSplits.length - logDebug(s"featureIndex = $featureIndex, numSplits = $numSplits") - splits(featureIndex) = new Array[Split](numSplits) + private def findSplitsBySorting( + input: RDD[LabeledPoint], + metadata: DecisionTreeMetadata, + continuousFeatures: IndexedSeq[Int]): Array[Array[Split]] = { + + val continuousSplits: scala.collection.Map[Int, Array[Split]] = { + // reduce the parallelism for split computations when there are less + // continuous features than input partitions. this prevents tasks from + // being spun up that will definitely do no work. + val numPartitions = math.min(continuousFeatures.length, input.partitions.length) + + input + .flatMap(point => continuousFeatures.map(idx => (idx, point.features(idx)))) + .groupByKey(numPartitions) + .map { case (idx, samples) => + val thresholds = findSplitsForContinuousFeature(samples, metadata, idx) + val splits: Array[Split] = thresholds.map(thresh => new ContinuousSplit(idx, thresh)) + logDebug(s"featureIndex = $idx, numSplits = ${splits.length}") + (idx, splits) + }.collectAsMap() + } - var splitIndex = 0 - while (splitIndex < numSplits) { - val threshold = featureSplits(splitIndex) - splits(featureIndex)(splitIndex) = new ContinuousSplit(featureIndex, threshold) - splitIndex += 1 - } - } else { - // Categorical feature - if (metadata.isUnordered(featureIndex)) { - val numSplits = metadata.numSplits(featureIndex) - val featureArity = metadata.featureArity(featureIndex) - // TODO: Use an implicit representation mapping each category to a subset of indices. - // I.e., track indices such that we can calculate the set of bins for which - // feature value x splits to the left. - // Unordered features - // 2^(maxFeatureValue - 1) - 1 combinations - splits(featureIndex) = new Array[Split](numSplits) - var splitIndex = 0 - while (splitIndex < numSplits) { - val categories: List[Double] = - extractMultiClassCategories(splitIndex + 1, featureArity) - splits(featureIndex)(splitIndex) = - new CategoricalSplit(featureIndex, categories.toArray, featureArity) - splitIndex += 1 - } - } else { - // Ordered features - // 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) + val numFeatures = metadata.numFeatures + val splits: Array[Array[Split]] = Array.tabulate(numFeatures) { + case i if metadata.isContinuous(i) => + val split = continuousSplits(i) + metadata.setNumSplits(i, split.length) + split + + case i if metadata.isCategorical(i) && metadata.isUnordered(i) => + // Unordered features + // 2^(maxFeatureValue - 1) - 1 combinations + val featureArity = metadata.featureArity(i) + Array.tabulate[Split](metadata.numSplits(i)) { splitIndex => + val categories = extractMultiClassCategories(splitIndex + 1, featureArity) + new CategoricalSplit(i, categories.toArray, featureArity) } - } - featureIndex += 1 + + case i if metadata.isCategorical(i) => + // Ordered features + // Splits are constructed as needed during training. + Array.empty[Split] } splits } @@ -927,37 +981,46 @@ private[ml] object RandomForest extends Logging { * NOTE: Returned number of splits is set based on `featureSamples` and * could be different from the specified `numSplits`. * The `numSplits` attribute in the `DecisionTreeMetadata` class will be set accordingly. + * * @param featureSamples feature values of each sample * @param metadata decision tree metadata * NOTE: `metadata.numbins` will be changed accordingly * if there are not enough splits to be found * @param featureIndex feature index to find splits - * @return array of splits + * @return array of split thresholds */ private[tree] def findSplitsForContinuousFeature( - featureSamples: Array[Double], + featureSamples: Iterable[Double], metadata: DecisionTreeMetadata, featureIndex: Int): Array[Double] = { require(metadata.isContinuous(featureIndex), "findSplitsForContinuousFeature can only be used to find splits for a continuous feature.") - val splits = { + val splits: Array[Double] = if (featureSamples.isEmpty) { + Array.empty[Double] + } else { val numSplits = metadata.numSplits(featureIndex) // get count for each distinct value - val valueCountMap = featureSamples.foldLeft(Map.empty[Double, Int]) { (m, x) => - m + ((x, m.getOrElse(x, 0) + 1)) + val (valueCountMap, numSamples) = featureSamples.foldLeft((Map.empty[Double, Int], 0)) { + case ((m, cnt), x) => + (m + ((x, m.getOrElse(x, 0) + 1)), cnt + 1) } // sort distinct values val valueCounts = valueCountMap.toSeq.sortBy(_._1).toArray - // if possible splits is not enough or just enough, just return all possible splits - val possibleSplits = valueCounts.length - if (possibleSplits <= numSplits) { - valueCounts.map(_._1) + val possibleSplits = valueCounts.length - 1 + if (possibleSplits == 0) { + // constant feature + Array.empty[Double] + } else if (possibleSplits <= numSplits) { + // if possible splits is not enough or just enough, just return all possible splits + (1 to possibleSplits) + .map(index => (valueCounts(index - 1)._1 + valueCounts(index)._1) / 2.0) + .toArray } else { // stride between splits - val stride: Double = featureSamples.length.toDouble / (numSplits + 1) + val stride: Double = numSamples.toDouble / (numSplits + 1) logDebug("stride = " + stride) // iterate `valueCount` to find splits @@ -979,7 +1042,7 @@ private[ml] object RandomForest extends Logging { // makes the gap between currentCount and targetCount smaller, // previous value is a split threshold. if (previousGap < currentGap) { - splitsBuilder += valueCounts(index - 1)._1 + splitsBuilder += (valueCounts(index - 1)._1 + valueCounts(index)._1) / 2.0 targetCount += stride } index += 1 @@ -988,14 +1051,6 @@ private[ml] object RandomForest extends Logging { splitsBuilder.result() } } - - // TODO: Do not fail; just ignore the useless feature. - assert(splits.length > 0, - s"DecisionTree could not handle feature $featureIndex since it had only 1 unique value." + - " Please remove this feature and then try again.") - // set number of splits accordingly - metadata.setNumSplits(featureIndex, splits.length) - splits } @@ -1009,7 +1064,7 @@ private[ml] object RandomForest extends Logging { * will be needed; this allows an adaptive number of nodes since different nodes may require * different amounts of memory (if featureSubsetStrategy is not "all"). * - * @param nodeQueue Queue of nodes to split. + * @param nodeStack Queue of nodes to split. * @param maxMemoryUsage Bound on size of aggregate statistics. * @return (nodesForGroup, treeToNodeToIndexInfo). * nodesForGroup holds the nodes to split: treeIndex --> nodes in tree. @@ -1021,7 +1076,7 @@ private[ml] object RandomForest extends Logging { * The feature indices are None if not subsampling features. */ private[tree] def selectNodesToSplit( - nodeQueue: mutable.Queue[(Int, LearningNode)], + nodeStack: mutable.ArrayStack[(Int, LearningNode)], maxMemoryUsage: Long, metadata: DecisionTreeMetadata, rng: Random): (Map[Int, Array[LearningNode]], Map[Int, Map[Int, NodeIndexInfo]]) = { @@ -1032,8 +1087,11 @@ private[ml] object RandomForest extends Logging { new mutable.HashMap[Int, mutable.HashMap[Int, NodeIndexInfo]]() var memUsage: Long = 0L var numNodesInGroup = 0 - while (nodeQueue.nonEmpty && memUsage < maxMemoryUsage) { - val (treeIndex, node) = nodeQueue.head + // If maxMemoryInMB is set very small, we want to still try to split 1 node, + // so we allow one iteration if memUsage == 0. + var groupDone = false + while (nodeStack.nonEmpty && !groupDone) { + val (treeIndex, node) = nodeStack.top // Choose subset of features for node (if subsampling). val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) { Some(SamplingUtils.reservoirSampleAndCount(Range(0, @@ -1043,16 +1101,24 @@ private[ml] object RandomForest extends Logging { } // Check if enough memory remains to add this node to the group. val nodeMemUsage = RandomForest.aggregateSizeForNode(metadata, featureSubset) * 8L - if (memUsage + nodeMemUsage <= maxMemoryUsage) { - nodeQueue.dequeue() + if (memUsage + nodeMemUsage <= maxMemoryUsage || memUsage == 0) { + nodeStack.pop() mutableNodesForGroup.getOrElseUpdate(treeIndex, new mutable.ArrayBuffer[LearningNode]()) += node mutableTreeToNodeToIndexInfo .getOrElseUpdate(treeIndex, new mutable.HashMap[Int, NodeIndexInfo]())(node.id) = new NodeIndexInfo(numNodesInGroup, featureSubset) + numNodesInGroup += 1 + memUsage += nodeMemUsage + } else { + groupDone = true } - numNodesInGroup += 1 - memUsage += nodeMemUsage + } + if (memUsage > maxMemoryUsage) { + // If maxMemoryUsage is 0, we should still allow splitting 1 node. + logWarning(s"Tree learning is using approximately $memUsage bytes per iteration, which" + + s" exceeds requested limit maxMemoryUsage=$maxMemoryUsage. This allows splitting" + + s" $numNodesInGroup nodes in this iteration.") } // Convert mutable maps to immutable ones. val nodesForGroup: Map[Int, Array[LearningNode]] = @@ -1063,6 +1129,7 @@ private[ml] object RandomForest extends Logging { /** * Get the number of values to be stored for this node in the bin aggregates. + * * @param featureSubset Indices of features which may be split at this node. * If None, then use all features. */ @@ -1080,95 +1147,4 @@ private[ml] object RandomForest extends Logging { 3 * totalBins } } - - /** - * Given a Random Forest model, compute the importance of each feature. - * This generalizes the idea of "Gini" importance to other losses, - * following the explanation of Gini importance from "Random Forests" documentation - * by Leo Breiman and Adele Cutler, and following the implementation from scikit-learn. - * - * This feature importance is calculated as follows: - * - Average over trees: - * - importance(feature j) = sum (over nodes which split on feature j) of the gain, - * where gain is scaled by the number of instances passing through node - * - Normalize importances for tree based on total number of training instances used - * to build tree. - * - Normalize feature importance vector to sum to 1. - * - * Note: This should not be used with Gradient-Boosted Trees. It only makes sense for - * independently trained trees. - * @param trees Unweighted forest of trees - * @param numFeatures Number of features in model (even if not all are explicitly used by - * the model). - * If -1, then numFeatures is set based on the max feature index in all trees. - * @return Feature importance values, of length numFeatures. - */ - private[ml] def featureImportances(trees: Array[DecisionTreeModel], numFeatures: Int): Vector = { - val totalImportances = new OpenHashMap[Int, Double]() - trees.foreach { tree => - // Aggregate feature importance vector for this tree - val importances = new OpenHashMap[Int, Double]() - computeFeatureImportance(tree.rootNode, importances) - // Normalize importance vector for this tree, and add it to total. - // TODO: In the future, also support normalizing by tree.rootNode.impurityStats.count? - val treeNorm = importances.map(_._2).sum - if (treeNorm != 0) { - importances.foreach { case (idx, impt) => - val normImpt = impt / treeNorm - totalImportances.changeValue(idx, normImpt, _ + normImpt) - } - } - } - // Normalize importances - normalizeMapValues(totalImportances) - // Construct vector - val d = if (numFeatures != -1) { - numFeatures - } else { - // Find max feature index used in trees - val maxFeatureIndex = trees.map(_.maxSplitFeatureIndex()).max - maxFeatureIndex + 1 - } - if (d == 0) { - assert(totalImportances.size == 0, s"Unknown error in computing RandomForest feature" + - s" importance: No splits in forest, but some non-zero importances.") - } - val (indices, values) = totalImportances.iterator.toSeq.sortBy(_._1).unzip - Vectors.sparse(d, indices.toArray, values.toArray) - } - - /** - * Recursive method for computing feature importances for one tree. - * This walks down the tree, adding to the importance of 1 feature at each node. - * @param node Current node in recursion - * @param importances Aggregate feature importances, modified by this method - */ - private[impl] def computeFeatureImportance( - node: Node, - importances: OpenHashMap[Int, Double]): Unit = { - node match { - case n: InternalNode => - val feature = n.split.featureIndex - val scaledGain = n.gain * n.impurityStats.count - importances.changeValue(feature, scaledGain, _ + scaledGain) - computeFeatureImportance(n.leftChild, importances) - computeFeatureImportance(n.rightChild, importances) - case n: LeafNode => - // do nothing - } - } - - /** - * Normalize the values of this map to sum to 1, in place. - * If all values are 0, this method does nothing. - * @param map Map with non-negative values. - */ - private[impl] def normalizeMapValues(map: OpenHashMap[Int, Double]): Unit = { - val total = map.map(_._2).sum - if (total != 0) { - val keys = map.iterator.map(_._1).toArray - keys.foreach { key => map.changeValue(key, 0.0, _ / total) } - } - } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TimeTracker.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TimeTracker.scala similarity index 98% rename from mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TimeTracker.scala rename to mllib/src/main/scala/org/apache/spark/ml/tree/impl/TimeTracker.scala index 70afaa162b2e..4cc250aa462e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TimeTracker.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TimeTracker.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.tree.impl +package org.apache.spark.ml.tree.impl import scala.collection.mutable.{HashMap => MutableHashMap} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala index 9fa27e5e1f72..a6ac64a0463c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala @@ -17,9 +17,8 @@ package org.apache.spark.ml.tree.impl +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.tree.{ContinuousSplit, Split} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.impl.DecisionTreeMetadata import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala index b77191156f68..4aa4c3617e7f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala @@ -17,14 +17,29 @@ package org.apache.spark.ml.tree -import org.apache.spark.mllib.linalg.{Vectors, Vector} +import scala.reflect.ClassTag + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param.{Param, Params} +import org.apache.spark.ml.tree.DecisionTreeModelReadWrite.NodeData +import org.apache.spark.ml.util.{DefaultParamsReader, DefaultParamsWriter} +import org.apache.spark.ml.util.DefaultParamsReader.Metadata +import org.apache.spark.mllib.tree.impurity.ImpurityCalculator +import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Dataset, SparkSession} +import org.apache.spark.util.collection.OpenHashMap /** * Abstraction for Decision Tree models. * * TODO: Add support for predicting probabilities and raw predictions SPARK-3727 */ -private[ml] trait DecisionTreeModel { +private[spark] trait DecisionTreeModel { /** Root of the decision tree */ def rootNode: Node @@ -56,23 +71,29 @@ private[ml] trait DecisionTreeModel { /** * Trace down the tree, and return the largest feature index used in any split. + * * @return Max feature index used in a split, or -1 if there are no splits (single leaf node). */ private[ml] def maxSplitFeatureIndex(): Int = rootNode.maxSplitFeatureIndex() + + /** Convert to spark.mllib DecisionTreeModel (losing some information) */ + private[spark] def toOld: OldDecisionTreeModel } /** * Abstraction for models which are ensembles of decision trees * * TODO: Add support for predicting probabilities and raw predictions SPARK-3727 + * + * @tparam M Type of tree model in this ensemble */ -private[ml] trait TreeEnsembleModel { +private[ml] trait TreeEnsembleModel[M <: DecisionTreeModel] { // Note: We use getTrees since subclasses of TreeEnsembleModel will store subclasses of // DecisionTreeModel. /** Trees in this ensemble. Warning: These have null parent Estimators. */ - def trees: Array[DecisionTreeModel] + def trees: Array[M] /** Weights for each tree, zippable with [[trees]] */ def treeWeights: Array[Double] @@ -84,7 +105,7 @@ private[ml] trait TreeEnsembleModel { /** Summary of the model */ override def toString: String = { // Implementing classes should generally override this method to be more descriptive. - s"TreeEnsembleModel with $numTrees trees" + s"TreeEnsembleModel with ${trees.length} trees" } /** Full description of model */ @@ -95,9 +116,364 @@ private[ml] trait TreeEnsembleModel { }.fold("")(_ + _) } - /** Number of trees in ensemble */ - val numTrees: Int = trees.length - /** Total number of nodes, summed over all trees in the ensemble. */ lazy val totalNumNodes: Int = trees.map(_.numNodes).sum } + +private[ml] object TreeEnsembleModel { + + /** + * Given a tree ensemble model, compute the importance of each feature. + * This generalizes the idea of "Gini" importance to other losses, + * following the explanation of Gini importance from "Random Forests" documentation + * by Leo Breiman and Adele Cutler, and following the implementation from scikit-learn. + * + * For collections of trees, including boosting and bagging, Hastie et al. + * propose to use the average of single tree importances across all trees in the ensemble. + * + * This feature importance is calculated as follows: + * - Average over trees: + * - importance(feature j) = sum (over nodes which split on feature j) of the gain, + * where gain is scaled by the number of instances passing through node + * - Normalize importances for tree to sum to 1. + * - Normalize feature importance vector to sum to 1. + * + * References: + * - Hastie, Tibshirani, Friedman. "The Elements of Statistical Learning, 2nd Edition." 2001. + * + * @param trees Unweighted collection of trees + * @param numFeatures Number of features in model (even if not all are explicitly used by + * the model). + * If -1, then numFeatures is set based on the max feature index in all trees. + * @return Feature importance values, of length numFeatures. + */ + def featureImportances[M <: DecisionTreeModel](trees: Array[M], numFeatures: Int): Vector = { + val totalImportances = new OpenHashMap[Int, Double]() + trees.foreach { tree => + // Aggregate feature importance vector for this tree + val importances = new OpenHashMap[Int, Double]() + computeFeatureImportance(tree.rootNode, importances) + // Normalize importance vector for this tree, and add it to total. + // TODO: In the future, also support normalizing by tree.rootNode.impurityStats.count? + val treeNorm = importances.map(_._2).sum + if (treeNorm != 0) { + importances.foreach { case (idx, impt) => + val normImpt = impt / treeNorm + totalImportances.changeValue(idx, normImpt, _ + normImpt) + } + } + } + // Normalize importances + normalizeMapValues(totalImportances) + // Construct vector + val d = if (numFeatures != -1) { + numFeatures + } else { + // Find max feature index used in trees + val maxFeatureIndex = trees.map(_.maxSplitFeatureIndex()).max + maxFeatureIndex + 1 + } + if (d == 0) { + assert(totalImportances.size == 0, s"Unknown error in computing feature" + + s" importance: No splits found, but some non-zero importances.") + } + val (indices, values) = totalImportances.iterator.toSeq.sortBy(_._1).unzip + Vectors.sparse(d, indices.toArray, values.toArray) + } + + /** + * Given a Decision Tree model, compute the importance of each feature. + * This generalizes the idea of "Gini" importance to other losses, + * following the explanation of Gini importance from "Random Forests" documentation + * by Leo Breiman and Adele Cutler, and following the implementation from scikit-learn. + * + * This feature importance is calculated as follows: + * - importance(feature j) = sum (over nodes which split on feature j) of the gain, + * where gain is scaled by the number of instances passing through node + * - Normalize importances for tree to sum to 1. + * + * @param tree Decision tree to compute importances for. + * @param numFeatures Number of features in model (even if not all are explicitly used by + * the model). + * If -1, then numFeatures is set based on the max feature index in all trees. + * @return Feature importance values, of length numFeatures. + */ + def featureImportances[M <: DecisionTreeModel : ClassTag](tree: M, numFeatures: Int): Vector = { + featureImportances(Array(tree), numFeatures) + } + + /** + * Recursive method for computing feature importances for one tree. + * This walks down the tree, adding to the importance of 1 feature at each node. + * + * @param node Current node in recursion + * @param importances Aggregate feature importances, modified by this method + */ + def computeFeatureImportance( + node: Node, + importances: OpenHashMap[Int, Double]): Unit = { + node match { + case n: InternalNode => + val feature = n.split.featureIndex + val scaledGain = n.gain * n.impurityStats.count + importances.changeValue(feature, scaledGain, _ + scaledGain) + computeFeatureImportance(n.leftChild, importances) + computeFeatureImportance(n.rightChild, importances) + case n: LeafNode => + // do nothing + } + } + + /** + * Normalize the values of this map to sum to 1, in place. + * If all values are 0, this method does nothing. + * + * @param map Map with non-negative values. + */ + def normalizeMapValues(map: OpenHashMap[Int, Double]): Unit = { + val total = map.map(_._2).sum + if (total != 0) { + val keys = map.iterator.map(_._1).toArray + keys.foreach { key => map.changeValue(key, 0.0, _ / total) } + } + } +} + +/** Helper classes for tree model persistence */ +private[ml] object DecisionTreeModelReadWrite { + + /** + * Info for a [[org.apache.spark.ml.tree.Split]] + * + * @param featureIndex Index of feature split on + * @param leftCategoriesOrThreshold For categorical feature, set of leftCategories. + * For continuous feature, threshold. + * @param numCategories For categorical feature, number of categories. + * For continuous feature, -1. + */ + case class SplitData( + featureIndex: Int, + leftCategoriesOrThreshold: Array[Double], + numCategories: Int) { + + def getSplit: Split = { + if (numCategories != -1) { + new CategoricalSplit(featureIndex, leftCategoriesOrThreshold, numCategories) + } else { + assert(leftCategoriesOrThreshold.length == 1, s"DecisionTree split data expected" + + s" 1 threshold for ContinuousSplit, but found thresholds: " + + leftCategoriesOrThreshold.mkString(", ")) + new ContinuousSplit(featureIndex, leftCategoriesOrThreshold(0)) + } + } + } + + object SplitData { + def apply(split: Split): SplitData = split match { + case s: CategoricalSplit => + SplitData(s.featureIndex, s.leftCategories, s.numCategories) + case s: ContinuousSplit => + SplitData(s.featureIndex, Array(s.threshold), -1) + } + } + + /** + * Info for a [[Node]] + * + * @param id Index used for tree reconstruction. Indices follow a pre-order traversal. + * @param impurityStats Stats array. Impurity type is stored in metadata. + * @param gain Gain, or arbitrary value if leaf node. + * @param leftChild Left child index, or arbitrary value if leaf node. + * @param rightChild Right child index, or arbitrary value if leaf node. + * @param split Split info, or arbitrary value if leaf node. + */ + case class NodeData( + id: Int, + prediction: Double, + impurity: Double, + impurityStats: Array[Double], + gain: Double, + leftChild: Int, + rightChild: Int, + split: SplitData) + + object NodeData { + /** + * Create [[NodeData]] instances for this node and all children. + * + * @param id Current ID. IDs are assigned via a pre-order traversal. + * @return (sequence of nodes in pre-order traversal order, largest ID in subtree) + * The nodes are returned in pre-order traversal (root first) so that it is easy to + * get the ID of the subtree's root node. + */ + def build(node: Node, id: Int): (Seq[NodeData], Int) = node match { + case n: InternalNode => + val (leftNodeData, leftIdx) = build(n.leftChild, id + 1) + val (rightNodeData, rightIdx) = build(n.rightChild, leftIdx + 1) + val thisNodeData = NodeData(id, n.prediction, n.impurity, n.impurityStats.stats, + n.gain, leftNodeData.head.id, rightNodeData.head.id, SplitData(n.split)) + (thisNodeData +: (leftNodeData ++ rightNodeData), rightIdx) + case _: LeafNode => + (Seq(NodeData(id, node.prediction, node.impurity, node.impurityStats.stats, + -1.0, -1, -1, SplitData(-1, Array.empty[Double], -1))), + id) + } + } + + /** + * Load a decision tree from a file. + * @return Root node of reconstructed tree + */ + def loadTreeNodes( + path: String, + metadata: DefaultParamsReader.Metadata, + sparkSession: SparkSession): Node = { + import sparkSession.implicits._ + implicit val format = DefaultFormats + + // Get impurity to construct ImpurityCalculator for each node + val impurityType: String = { + val impurityJson: JValue = metadata.getParamValue("impurity") + Param.jsonDecode[String](compact(render(impurityJson))) + } + + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath).as[NodeData] + buildTreeFromNodes(data.collect(), impurityType) + } + + /** + * Given all data for all nodes in a tree, rebuild the tree. + * @param data Unsorted node data + * @param impurityType Impurity type for this tree + * @return Root node of reconstructed tree + */ + def buildTreeFromNodes(data: Array[NodeData], impurityType: String): Node = { + // Load all nodes, sorted by ID. + val nodes = data.sortBy(_.id) + // Sanity checks; could remove + assert(nodes.head.id == 0, s"Decision Tree load failed. Expected smallest node ID to be 0," + + s" but found ${nodes.head.id}") + assert(nodes.last.id == nodes.length - 1, s"Decision Tree load failed. Expected largest" + + s" node ID to be ${nodes.length - 1}, but found ${nodes.last.id}") + // We fill `finalNodes` in reverse order. Since node IDs are assigned via a pre-order + // traversal, this guarantees that child nodes will be built before parent nodes. + val finalNodes = new Array[Node](nodes.length) + nodes.reverseIterator.foreach { case n: NodeData => + val impurityStats = ImpurityCalculator.getCalculator(impurityType, n.impurityStats) + val node = if (n.leftChild != -1) { + val leftChild = finalNodes(n.leftChild) + val rightChild = finalNodes(n.rightChild) + new InternalNode(n.prediction, n.impurity, n.gain, leftChild, rightChild, + n.split.getSplit, impurityStats) + } else { + new LeafNode(n.prediction, n.impurity, impurityStats) + } + finalNodes(n.id) = node + } + // Return the root node + finalNodes.head + } +} + +private[ml] object EnsembleModelReadWrite { + + /** + * Helper method for saving a tree ensemble to disk. + * + * @param instance Tree ensemble model + * @param path Path to which to save the ensemble model. + * @param extraMetadata Metadata such as numFeatures, numClasses, numTrees. + */ + def saveImpl[M <: Params with TreeEnsembleModel[_ <: DecisionTreeModel]]( + instance: M, + path: String, + sql: SparkSession, + extraMetadata: JObject): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sql.sparkContext, Some(extraMetadata)) + val treesMetadataWeights: Array[(Int, String, Double)] = instance.trees.zipWithIndex.map { + case (tree, treeID) => + (treeID, + DefaultParamsWriter.getMetadataToSave(tree.asInstanceOf[Params], sql.sparkContext), + instance.treeWeights(treeID)) + } + val treesMetadataPath = new Path(path, "treesMetadata").toString + sql.createDataFrame(treesMetadataWeights).toDF("treeID", "metadata", "weights") + .write.parquet(treesMetadataPath) + val dataPath = new Path(path, "data").toString + val nodeDataRDD = sql.sparkContext.parallelize(instance.trees.zipWithIndex).flatMap { + case (tree, treeID) => EnsembleNodeData.build(tree, treeID) + } + sql.createDataFrame(nodeDataRDD).write.parquet(dataPath) + } + + /** + * Helper method for loading a tree ensemble from disk. + * This reconstructs all trees, returning the root nodes. + * @param path Path given to `saveImpl` + * @param className Class name for ensemble model type + * @param treeClassName Class name for tree model type in the ensemble + * @return (ensemble metadata, array over trees of (tree metadata, root node)), + * where the root node is linked with all descendents + * @see `saveImpl` for how the model was saved + */ + def loadImpl( + path: String, + sql: SparkSession, + className: String, + treeClassName: String): (Metadata, Array[(Metadata, Node)], Array[Double]) = { + import sql.implicits._ + implicit val format = DefaultFormats + val metadata = DefaultParamsReader.loadMetadata(path, sql.sparkContext, className) + + // Get impurity to construct ImpurityCalculator for each node + val impurityType: String = { + val impurityJson: JValue = metadata.getParamValue("impurity") + Param.jsonDecode[String](compact(render(impurityJson))) + } + + val treesMetadataPath = new Path(path, "treesMetadata").toString + val treesMetadataRDD: RDD[(Int, (Metadata, Double))] = sql.read.parquet(treesMetadataPath) + .select("treeID", "metadata", "weights").as[(Int, String, Double)].rdd.map { + case (treeID: Int, json: String, weights: Double) => + treeID -> ((DefaultParamsReader.parseMetadata(json, treeClassName), weights)) + } + + val treesMetadataWeights = treesMetadataRDD.sortByKey().values.collect() + val treesMetadata = treesMetadataWeights.map(_._1) + val treesWeights = treesMetadataWeights.map(_._2) + + val dataPath = new Path(path, "data").toString + val nodeData: Dataset[EnsembleNodeData] = + sql.read.parquet(dataPath).as[EnsembleNodeData] + val rootNodesRDD: RDD[(Int, Node)] = + nodeData.rdd.map(d => (d.treeID, d.nodeData)).groupByKey().map { + case (treeID: Int, nodeData: Iterable[NodeData]) => + treeID -> DecisionTreeModelReadWrite.buildTreeFromNodes(nodeData.toArray, impurityType) + } + val rootNodes: Array[Node] = rootNodesRDD.sortByKey().values.collect() + (metadata, treesMetadata.zip(rootNodes), treesWeights) + } + + /** + * Info for one [[Node]] in a tree ensemble + * + * @param treeID Tree index + * @param nodeData Data for this node + */ + case class EnsembleNodeData( + treeID: Int, + nodeData: NodeData) + + object EnsembleNodeData { + /** + * Create [[EnsembleNodeData]] instances for the given tree. + * + * @return Sequence of nodes for this tree + */ + def build(tree: DecisionTreeModel, treeID: Int): Seq[EnsembleNodeData] = { + val (nodeData: Seq[NodeData], _) = NodeData.build(tree.rootNode, 0) + nodeData.map(nd => EnsembleNodeData(treeID, nd)) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala index 281ba6eeffa9..47079d9c6bb1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala @@ -17,19 +17,26 @@ package org.apache.spark.ml.tree +import java.util.Locale + +import scala.util.Try + import org.apache.spark.ml.PredictorParams import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, BoostingStrategy => OldBoostingStrategy, Strategy => OldStrategy} import org.apache.spark.mllib.tree.impurity.{Entropy => OldEntropy, Gini => OldGini, Impurity => OldImpurity, Variance => OldVariance} -import org.apache.spark.mllib.tree.loss.{Loss => OldLoss} +import org.apache.spark.mllib.tree.loss.{AbsoluteError => OldAbsoluteError, ClassificationLoss => OldClassificationLoss, LogLoss => OldLogLoss, Loss => OldLoss, SquaredError => OldSquaredError} +import org.apache.spark.sql.types.{DataType, DoubleType, StructType} /** * Parameters for Decision Tree-based algorithms. * * Note: Marked as private and DeveloperApi since this may be made public in the future. */ -private[ml] trait DecisionTreeParams extends PredictorParams with HasCheckpointInterval { +private[ml] trait DecisionTreeParams extends PredictorParams + with HasCheckpointInterval with HasSeed { /** * Maximum depth of the tree (>= 0). @@ -68,14 +75,17 @@ private[ml] trait DecisionTreeParams extends PredictorParams with HasCheckpointI /** * Minimum information gain for a split to be considered at a tree node. + * Should be >= 0.0. * (default = 0.0) * @group param */ final val minInfoGain: DoubleParam = new DoubleParam(this, "minInfoGain", - "Minimum information gain for a split to be considered at a tree node.") + "Minimum information gain for a split to be considered at a tree node.", + ParamValidators.gtEq(0.0)) /** - * Maximum memory in MB allocated to histogram aggregation. + * Maximum memory in MB allocated to histogram aggregation. If too small, then 1 node will be + * split per iteration, and its aggregates may exceed this size. * (default = 256 MB) * @group expertParam */ @@ -99,51 +109,78 @@ private[ml] trait DecisionTreeParams extends PredictorParams with HasCheckpointI setDefault(maxDepth -> 5, maxBins -> 32, minInstancesPerNode -> 1, minInfoGain -> 0.0, maxMemoryInMB -> 256, cacheNodeIds -> false, checkpointInterval -> 10) - /** @group setParam */ + /** + * @deprecated This method is deprecated and will be removed in 3.0.0. + * @group setParam + */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") def setMaxDepth(value: Int): this.type = set(maxDepth, value) /** @group getParam */ final def getMaxDepth: Int = $(maxDepth) - /** @group setParam */ + /** + * @deprecated This method is deprecated and will be removed in 3.0.0. + * @group setParam + */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") def setMaxBins(value: Int): this.type = set(maxBins, value) /** @group getParam */ final def getMaxBins: Int = $(maxBins) - /** @group setParam */ + /** + * @deprecated This method is deprecated and will be removed in 3.0.0. + * @group setParam + */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) /** @group getParam */ final def getMinInstancesPerNode: Int = $(minInstancesPerNode) - /** @group setParam */ + /** + * @deprecated This method is deprecated and will be removed in 3.0.0. + * @group setParam + */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) /** @group getParam */ final def getMinInfoGain: Double = $(minInfoGain) - /** @group expertSetParam */ + /** + * @deprecated This method is deprecated and will be removed in 3.0.0. + * @group setParam + */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") + def setSeed(value: Long): this.type = set(seed, value) + + /** + * @deprecated This method is deprecated and will be removed in 3.0.0. + * @group expertSetParam + */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) /** @group expertGetParam */ final def getMaxMemoryInMB: Int = $(maxMemoryInMB) - /** @group expertSetParam */ + /** + * @deprecated This method is deprecated and will be removed in 3.0.0. + * @group expertSetParam + */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) /** @group expertGetParam */ final def getCacheNodeIds: Boolean = $(cacheNodeIds) /** - * Specifies how often to checkpoint the cached node IDs. - * E.g. 10 means that the cache will get checkpointed every 10 iterations. - * This is only used if cacheNodeIds is true and if the checkpoint directory is set in - * [[org.apache.spark.SparkContext]]. - * Must be >= 1. - * (default = 10) - * @group expertSetParam + * @deprecated This method is deprecated and will be removed in 3.0.0. + * @group setParam */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) /** (private[ml]) Create a Strategy instance to use with the old API. */ @@ -183,15 +220,20 @@ private[ml] trait TreeClassifierParams extends Params { final val impurity: Param[String] = new Param[String](this, "impurity", "Criterion used for" + " information gain calculation (case-insensitive). Supported options:" + s" ${TreeClassifierParams.supportedImpurities.mkString(", ")}", - (value: String) => TreeClassifierParams.supportedImpurities.contains(value.toLowerCase)) + (value: String) => + TreeClassifierParams.supportedImpurities.contains(value.toLowerCase(Locale.ROOT))) setDefault(impurity -> "gini") - /** @group setParam */ + /** + * @deprecated This method is deprecated and will be removed in 3.0.0. + * @group setParam + */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") def setImpurity(value: String): this.type = set(impurity, value) /** @group getParam */ - final def getImpurity: String = $(impurity).toLowerCase + final def getImpurity: String = $(impurity).toLowerCase(Locale.ROOT) /** Convert new impurity to old impurity. */ private[ml] def getOldImpurity: OldImpurity = { @@ -208,9 +250,13 @@ private[ml] trait TreeClassifierParams extends Params { private[ml] object TreeClassifierParams { // These options should be lowercase. - final val supportedImpurities: Array[String] = Array("entropy", "gini").map(_.toLowerCase) + final val supportedImpurities: Array[String] = + Array("entropy", "gini").map(_.toLowerCase(Locale.ROOT)) } +private[ml] trait DecisionTreeClassifierParams + extends DecisionTreeParams with TreeClassifierParams + /** * Parameters for Decision Tree-based regression algorithms. */ @@ -225,15 +271,20 @@ private[ml] trait TreeRegressorParams extends Params { final val impurity: Param[String] = new Param[String](this, "impurity", "Criterion used for" + " information gain calculation (case-insensitive). Supported options:" + s" ${TreeRegressorParams.supportedImpurities.mkString(", ")}", - (value: String) => TreeRegressorParams.supportedImpurities.contains(value.toLowerCase)) + (value: String) => + TreeRegressorParams.supportedImpurities.contains(value.toLowerCase(Locale.ROOT))) setDefault(impurity -> "variance") - /** @group setParam */ + /** + * @deprecated This method is deprecated and will be removed in 3.0.0. + * @group setParam + */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") def setImpurity(value: String): this.type = set(impurity, value) /** @group getParam */ - final def getImpurity: String = $(impurity).toLowerCase + final def getImpurity: String = $(impurity).toLowerCase(Locale.ROOT) /** Convert new impurity to old impurity. */ private[ml] def getOldImpurity: OldImpurity = { @@ -249,7 +300,24 @@ private[ml] trait TreeRegressorParams extends Params { private[ml] object TreeRegressorParams { // These options should be lowercase. - final val supportedImpurities: Array[String] = Array("variance").map(_.toLowerCase) + final val supportedImpurities: Array[String] = + Array("variance").map(_.toLowerCase(Locale.ROOT)) +} + +private[ml] trait DecisionTreeRegressorParams extends DecisionTreeParams + with TreeRegressorParams with HasVarianceCol { + + override protected def validateAndTransformSchema( + schema: StructType, + fitting: Boolean, + featuresDataType: DataType): StructType = { + val newSchema = super.validateAndTransformSchema(schema, fitting, featuresDataType) + if (isDefined(varianceCol) && $(varianceCol).nonEmpty) { + SchemaUtils.appendColumn(newSchema, $(varianceCol), DoubleType) + } else { + newSchema + } + } } /** @@ -257,7 +325,7 @@ private[ml] object TreeRegressorParams { * * Note: Marked as private and DeveloperApi since this may be made public in the future. */ -private[ml] trait TreeEnsembleParams extends DecisionTreeParams with HasSeed { +private[ml] trait TreeEnsembleParams extends DecisionTreeParams { /** * Fraction of the training data used for learning each decision tree, in range (0, 1]. @@ -270,15 +338,16 @@ private[ml] trait TreeEnsembleParams extends DecisionTreeParams with HasSeed { setDefault(subsamplingRate -> 1.0) - /** @group setParam */ + /** + * @deprecated This method is deprecated and will be removed in 3.0.0. + * @group setParam + */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) /** @group getParam */ final def getSubsamplingRate: Double = $(subsamplingRate) - /** @group setParam */ - def setSeed(value: Long): this.type = set(seed, value) - /** * Create a Strategy instance to use with the old API. * NOTE: The caller should set impurity and seed. @@ -294,8 +363,6 @@ private[ml] trait TreeEnsembleParams extends DecisionTreeParams with HasSeed { /** * Parameters for Random Forest algorithms. - * - * Note: Marked as private and DeveloperApi since this may be made public in the future. */ private[ml] trait RandomForestParams extends TreeEnsembleParams { @@ -304,11 +371,27 @@ private[ml] trait RandomForestParams extends TreeEnsembleParams { * If 1, then no bootstrapping is used. If > 1, then bootstrapping is done. * TODO: Change to always do bootstrapping (simpler). SPARK-7130 * (default = 20) + * + * Note: The reason that we cannot add this to both GBT and RF (i.e. in TreeEnsembleParams) + * is the param `maxIter` controls how many trees a GBT has. The semantics in the algorithms + * are a bit different. * @group param */ final val numTrees: IntParam = new IntParam(this, "numTrees", "Number of trees to train (>= 1)", ParamValidators.gtEq(1)) + setDefault(numTrees -> 20) + + /** + * @deprecated This method is deprecated and will be removed in 3.0.0. + * @group setParam + */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") + def setNumTrees(value: Int): this.type = set(numTrees, value) + + /** @group getParam */ + final def getNumTrees: Int = $(numTrees) + /** * The number of features to consider for splits at each tree node. * Supported options: @@ -320,6 +403,8 @@ private[ml] trait RandomForestParams extends TreeEnsembleParams { * - "onethird": use 1/3 of the features * - "sqrt": use sqrt(number of features) * - "log2": use log2(number of features) + * - "n": when n is in the range (0, 1.0], use n * number of features. When n + * is in the range (1, number of features), use n features. * (default = "auto") * * These various settings are based on the following references: @@ -327,39 +412,47 @@ private[ml] trait RandomForestParams extends TreeEnsembleParams { * - sqrt: recommended by Breiman manual for random forests * - The defaults of sqrt (classification) and onethird (regression) match the R randomForest * package. - * @see [[http://www.stat.berkeley.edu/~breiman/randomforest2001.pdf Breiman (2001)]] - * @see [[http://www.stat.berkeley.edu/~breiman/Using_random_forests_V3.1.pdf Breiman manual for - * random forests]] + * @see Breiman (2001) + * @see + * Breiman manual for random forests * * @group param */ final val featureSubsetStrategy: Param[String] = new Param[String](this, "featureSubsetStrategy", "The number of features to consider for splits at each tree node." + - s" Supported options: ${RandomForestParams.supportedFeatureSubsetStrategies.mkString(", ")}", + s" Supported options: ${RandomForestParams.supportedFeatureSubsetStrategies.mkString(", ")}" + + s", (0.0-1.0], [1-n].", (value: String) => - RandomForestParams.supportedFeatureSubsetStrategies.contains(value.toLowerCase)) + RandomForestParams.supportedFeatureSubsetStrategies.contains( + value.toLowerCase(Locale.ROOT)) + || Try(value.toInt).filter(_ > 0).isSuccess + || Try(value.toDouble).filter(_ > 0).filter(_ <= 1.0).isSuccess) - setDefault(numTrees -> 20, featureSubsetStrategy -> "auto") + setDefault(featureSubsetStrategy -> "auto") - /** @group setParam */ - def setNumTrees(value: Int): this.type = set(numTrees, value) - - /** @group getParam */ - final def getNumTrees: Int = $(numTrees) - - /** @group setParam */ + /** + * @deprecated This method is deprecated and will be removed in 3.0.0. + * @group setParam + */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") def setFeatureSubsetStrategy(value: String): this.type = set(featureSubsetStrategy, value) /** @group getParam */ - final def getFeatureSubsetStrategy: String = $(featureSubsetStrategy).toLowerCase + final def getFeatureSubsetStrategy: String = $(featureSubsetStrategy).toLowerCase(Locale.ROOT) } -private[ml] object RandomForestParams { +private[spark] object RandomForestParams { // These options should be lowercase. final val supportedFeatureSubsetStrategies: Array[String] = - Array("auto", "all", "onethird", "sqrt", "log2").map(_.toLowerCase) + Array("auto", "all", "onethird", "sqrt", "log2").map(_.toLowerCase(Locale.ROOT)) } +private[ml] trait RandomForestClassifierParams + extends RandomForestParams with TreeClassifierParams + +private[ml] trait RandomForestRegressorParams + extends RandomForestParams with TreeRegressorParams + /** * Parameters for Gradient-Boosted Tree algorithms. * @@ -378,24 +471,31 @@ private[ml] trait GBTParams extends TreeEnsembleParams with HasMaxIter with HasS // final val validationTol: DoubleParam = new DoubleParam(this, "validationTol", "") // validationTol -> 1e-5 - setDefault(maxIter -> 20, stepSize -> 0.1) - - /** @group setParam */ + /** + * @deprecated This method is deprecated and will be removed in 3.0.0. + * @group setParam + */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") def setMaxIter(value: Int): this.type = set(maxIter, value) /** - * Step size (a.k.a. learning rate) in interval (0, 1] for shrinking the contribution of each - * estimator. + * Param for Step size (a.k.a. learning rate) in interval (0, 1] for shrinking + * the contribution of each estimator. * (default = 0.1) + * @group param + */ + final override val stepSize: DoubleParam = new DoubleParam(this, "stepSize", "Step size " + + "(a.k.a. learning rate) in interval (0, 1] for shrinking the contribution of each estimator.", + ParamValidators.inRange(0, 1, lowerInclusive = false, upperInclusive = true)) + + /** + * @deprecated This method is deprecated and will be removed in 3.0.0. * @group setParam */ + @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") def setStepSize(value: Double): this.type = set(stepSize, value) - override def validateParams(): Unit = { - require(ParamValidators.inRange(0, 1, lowerInclusive = false, upperInclusive = true)( - getStepSize), "GBT parameter stepSize should be in interval (0, 1], " + - s"but it given invalid value $getStepSize.") - } + setDefault(maxIter -> 20, stepSize -> 0.1) /** (private[ml]) Create a BoostingStrategy instance to use with the old API. */ private[ml] def getOldBoostingStrategy( @@ -409,3 +509,78 @@ private[ml] trait GBTParams extends TreeEnsembleParams with HasMaxIter with HasS /** Get old Gradient Boosting Loss type */ private[ml] def getOldLossType: OldLoss } + +private[ml] object GBTClassifierParams { + // The losses below should be lowercase. + /** Accessor for supported loss settings: logistic */ + final val supportedLossTypes: Array[String] = + Array("logistic").map(_.toLowerCase(Locale.ROOT)) +} + +private[ml] trait GBTClassifierParams extends GBTParams with TreeClassifierParams { + + /** + * Loss function which GBT tries to minimize. (case-insensitive) + * Supported: "logistic" + * (default = logistic) + * @group param + */ + val lossType: Param[String] = new Param[String](this, "lossType", "Loss function which GBT" + + " tries to minimize (case-insensitive). Supported options:" + + s" ${GBTClassifierParams.supportedLossTypes.mkString(", ")}", + (value: String) => + GBTClassifierParams.supportedLossTypes.contains(value.toLowerCase(Locale.ROOT))) + + setDefault(lossType -> "logistic") + + /** @group getParam */ + def getLossType: String = $(lossType).toLowerCase(Locale.ROOT) + + /** (private[ml]) Convert new loss to old loss. */ + override private[ml] def getOldLossType: OldClassificationLoss = { + getLossType match { + case "logistic" => OldLogLoss + case _ => + // Should never happen because of check in setter method. + throw new RuntimeException(s"GBTClassifier was given bad loss type: $getLossType") + } + } +} + +private[ml] object GBTRegressorParams { + // The losses below should be lowercase. + /** Accessor for supported loss settings: squared (L2), absolute (L1) */ + final val supportedLossTypes: Array[String] = + Array("squared", "absolute").map(_.toLowerCase(Locale.ROOT)) +} + +private[ml] trait GBTRegressorParams extends GBTParams with TreeRegressorParams { + + /** + * Loss function which GBT tries to minimize. (case-insensitive) + * Supported: "squared" (L2) and "absolute" (L1) + * (default = squared) + * @group param + */ + val lossType: Param[String] = new Param[String](this, "lossType", "Loss function which GBT" + + " tries to minimize (case-insensitive). Supported options:" + + s" ${GBTRegressorParams.supportedLossTypes.mkString(", ")}", + (value: String) => + GBTRegressorParams.supportedLossTypes.contains(value.toLowerCase(Locale.ROOT))) + + setDefault(lossType -> "squared") + + /** @group getParam */ + def getLossType: String = $(lossType).toLowerCase(Locale.ROOT) + + /** (private[ml]) Convert new loss to old loss. */ + override private[ml] def getOldLossType: OldLoss = { + getLossType match { + case "squared" => OldSquaredError + case "absolute" => OldAbsoluteError + case _ => + // Should never happen because of check in setter method. + throw new RuntimeException(s"GBTRegressorParams was given bad loss type: $getLossType") + } + } +} 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 77d9948ed86b..7c81cb96e07f 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 @@ -17,25 +17,35 @@ package org.apache.spark.ml.tuning -import com.github.fommil.netlib.F2jBLAS +import java.util.{List => JList} -import org.apache.spark.Logging -import org.apache.spark.annotation.Experimental -import org.apache.spark.ml._ +import scala.collection.JavaConverters._ +import scala.concurrent.Future +import scala.concurrent.duration.Duration + +import org.apache.hadoop.fs.Path +import org.json4s.DefaultFormats + +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging +import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator -import org.apache.spark.ml.param._ -import org.apache.spark.ml.util.Identifiable +import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators} +import org.apache.spark.ml.param.shared.HasParallelism +import org.apache.spark.ml.util._ import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.StructType +import org.apache.spark.util.ThreadUtils /** * Params for [[CrossValidator]] and [[CrossValidatorModel]]. */ private[ml] trait CrossValidatorParams extends ValidatorParams { /** - * Param for number of folds for cross validation. Must be >= 2. + * Param for number of folds for cross validation. Must be >= 2. * Default: 3 + * * @group param */ val numFolds: IntParam = new IntParam(this, "numFolds", @@ -48,57 +58,100 @@ private[ml] trait CrossValidatorParams extends ValidatorParams { } /** - * :: Experimental :: - * K-fold cross validation. + * K-fold cross validation performs model selection by splitting the dataset into a set of + * non-overlapping randomly partitioned folds which are used as separate training and test datasets + * e.g., with k=3 folds, K-fold cross validation will generate 3 (training, test) dataset pairs, + * each of which uses 2/3 of the data for training and 1/3 for testing. Each fold is used as the + * test set exactly once. */ -@Experimental -class CrossValidator(override val uid: String) extends Estimator[CrossValidatorModel] - with CrossValidatorParams with Logging { +@Since("1.2.0") +class CrossValidator @Since("1.2.0") (@Since("1.4.0") override val uid: String) + extends Estimator[CrossValidatorModel] + with CrossValidatorParams with HasParallelism with MLWritable with Logging { + @Since("1.2.0") def this() = this(Identifiable.randomUID("cv")) - private val f2jBLAS = new F2jBLAS - /** @group setParam */ + @Since("1.2.0") def setEstimator(value: Estimator[_]): this.type = set(estimator, value) /** @group setParam */ + @Since("1.2.0") def setEstimatorParamMaps(value: Array[ParamMap]): this.type = set(estimatorParamMaps, value) /** @group setParam */ + @Since("1.2.0") def setEvaluator(value: Evaluator): this.type = set(evaluator, value) /** @group setParam */ + @Since("1.2.0") def setNumFolds(value: Int): this.type = set(numFolds, value) - override def fit(dataset: DataFrame): CrossValidatorModel = { + /** @group setParam */ + @Since("2.0.0") + def setSeed(value: Long): this.type = set(seed, value) + + /** + * Set the mamixum level of parallelism to evaluate models in parallel. + * Default is 1 for serial evaluation + * + * @group expertSetParam + */ + @Since("2.3.0") + def setParallelism(value: Int): this.type = set(parallelism, value) + + @Since("2.0.0") + override def fit(dataset: Dataset[_]): CrossValidatorModel = { val schema = dataset.schema transformSchema(schema, logging = true) - val sqlCtx = dataset.sqlContext + val sparkSession = dataset.sparkSession val est = $(estimator) val eval = $(evaluator) val epm = $(estimatorParamMaps) - val numModels = epm.length - val metrics = new Array[Double](epm.length) - val splits = MLUtils.kFold(dataset.rdd, $(numFolds), 0) - splits.zipWithIndex.foreach { case ((training, validation), splitIndex) => - val trainingDataset = sqlCtx.createDataFrame(training, schema).cache() - val validationDataset = sqlCtx.createDataFrame(validation, schema).cache() - // multi-model training + + // Create execution context based on $(parallelism) + val executionContext = getExecutionContext + + val instr = Instrumentation.create(this, dataset) + instr.logParams(numFolds, seed, parallelism) + logTuningParams(instr) + + // Compute metrics for each model over each split + val splits = MLUtils.kFold(dataset.toDF.rdd, $(numFolds), $(seed)) + val metrics = splits.zipWithIndex.map { case ((training, validation), splitIndex) => + val trainingDataset = sparkSession.createDataFrame(training, schema).cache() + val validationDataset = sparkSession.createDataFrame(validation, schema).cache() 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) { - // TODO: duplicate evaluator to take extra params from input - val metric = eval.evaluate(models(i).transform(validationDataset, epm(i))) - logDebug(s"Got metric $metric for model trained with ${epm(i)}.") - metrics(i) += metric - i += 1 + + // Fit models in a Future for training in parallel + val modelFutures = epm.map { paramMap => + Future[Model[_]] { + val model = est.fit(trainingDataset, paramMap) + model.asInstanceOf[Model[_]] + } (executionContext) + } + + // Unpersist training data only when all models have trained + Future.sequence[Model[_], Iterable](modelFutures)(implicitly, executionContext) + .onComplete { _ => trainingDataset.unpersist() } (executionContext) + + // Evaluate models in a Future that will calulate a metric and allow model to be cleaned up + val foldMetricFutures = modelFutures.zip(epm).map { case (modelFuture, paramMap) => + modelFuture.map { model => + // TODO: duplicate evaluator to take extra params from input + val metric = eval.evaluate(model.transform(validationDataset, paramMap)) + logDebug(s"Got metric $metric for model trained with $paramMap.") + metric + } (executionContext) } + + // Wait for metrics to be calculated before unpersisting validation dataset + val foldMetrics = foldMetricFutures.map(ThreadUtils.awaitResult(_, Duration.Inf)) validationDataset.unpersist() - } - f2jBLAS.dscal(numModels, 1.0 / $(numFolds), metrics, 1) + foldMetrics + }.transpose.map(_.sum / $(numFolds)) // Calculate average metric over all splits + logInfo(s"Average cross-validation metrics: ${metrics.toSeq}") val (bestMetric, bestIndex) = if (eval.isLargerBetter) metrics.zipWithIndex.maxBy(_._1) @@ -106,21 +159,14 @@ class CrossValidator(override val uid: String) extends Estimator[CrossValidatorM logInfo(s"Best set of parameters:\n${epm(bestIndex)}") logInfo(s"Best cross-validation metric: $bestMetric.") val bestModel = est.fit(dataset, epm(bestIndex)).asInstanceOf[Model[_]] + instr.logSuccess(bestModel) copyValues(new CrossValidatorModel(uid, bestModel, metrics).setParent(this)) } - override def transformSchema(schema: StructType): StructType = { - $(estimator).transformSchema(schema) - } - - override def validateParams(): Unit = { - super.validateParams() - val est = $(estimator) - for (paramMap <- $(estimatorParamMaps)) { - est.copy(paramMap).validateParams() - } - } + @Since("1.4.0") + override def transformSchema(schema: StructType): StructType = transformSchemaImpl(schema) + @Since("1.4.0") override def copy(extra: ParamMap): CrossValidator = { val copied = defaultCopy(extra).asInstanceOf[CrossValidator] if (copied.isDefined(estimator)) { @@ -131,36 +177,85 @@ class CrossValidator(override val uid: String) extends Estimator[CrossValidatorM } copied } + + // Currently, this only works if all [[Param]]s in [[estimatorParamMaps]] are simple types. + // E.g., this may fail if a [[Param]] is an instance of an [[Estimator]]. + // However, this case should be unusual. + @Since("1.6.0") + override def write: MLWriter = new CrossValidator.CrossValidatorWriter(this) +} + +@Since("1.6.0") +object CrossValidator extends MLReadable[CrossValidator] { + + @Since("1.6.0") + override def read: MLReader[CrossValidator] = new CrossValidatorReader + + @Since("1.6.0") + override def load(path: String): CrossValidator = super.load(path) + + private[CrossValidator] class CrossValidatorWriter(instance: CrossValidator) extends MLWriter { + + ValidatorParams.validateParams(instance) + + override protected def saveImpl(path: String): Unit = + ValidatorParams.saveImpl(path, instance, sc) + } + + private class CrossValidatorReader extends MLReader[CrossValidator] { + + /** Checked against metadata when loading model */ + private val className = classOf[CrossValidator].getName + + override def load(path: String): CrossValidator = { + implicit val format = DefaultFormats + + val (metadata, estimator, evaluator, estimatorParamMaps) = + ValidatorParams.loadImpl(path, sc, className) + val cv = new CrossValidator(metadata.uid) + .setEstimator(estimator) + .setEvaluator(evaluator) + .setEstimatorParamMaps(estimatorParamMaps) + DefaultParamsReader.getAndSetParams(cv, metadata, + skipParams = Option(List("estimatorParamMaps"))) + cv + } + } } /** - * :: Experimental :: - * Model from k-fold cross validation. + * CrossValidatorModel contains the model with the highest average cross-validation + * metric across folds and uses this model to transform input data. CrossValidatorModel + * also tracks the metrics for each param map evaluated. * * @param bestModel The best model selected from k-fold cross validation. * @param avgMetrics Average cross-validation metrics for each paramMap in - * [[estimatorParamMaps]], in the corresponding order. + * `CrossValidator.estimatorParamMaps`, in the corresponding order. */ -@Experimental +@Since("1.2.0") class CrossValidatorModel private[ml] ( - override val uid: String, - val bestModel: Model[_], - val avgMetrics: Array[Double]) - extends Model[CrossValidatorModel] with CrossValidatorParams { + @Since("1.4.0") override val uid: String, + @Since("1.2.0") val bestModel: Model[_], + @Since("1.5.0") val avgMetrics: Array[Double]) + extends Model[CrossValidatorModel] with CrossValidatorParams with MLWritable { - override def validateParams(): Unit = { - bestModel.validateParams() + /** A Python-friendly auxiliary constructor. */ + private[ml] def this(uid: String, bestModel: Model[_], avgMetrics: JList[Double]) = { + this(uid, bestModel, avgMetrics.asScala.toArray) } - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) bestModel.transform(dataset) } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { bestModel.transformSchema(schema) } + @Since("1.4.0") override def copy(extra: ParamMap): CrossValidatorModel = { val copied = new CrossValidatorModel( uid, @@ -168,4 +263,55 @@ class CrossValidatorModel private[ml] ( avgMetrics.clone()) copyValues(copied, extra).setParent(parent) } + + @Since("1.6.0") + override def write: MLWriter = new CrossValidatorModel.CrossValidatorModelWriter(this) +} + +@Since("1.6.0") +object CrossValidatorModel extends MLReadable[CrossValidatorModel] { + + @Since("1.6.0") + override def read: MLReader[CrossValidatorModel] = new CrossValidatorModelReader + + @Since("1.6.0") + override def load(path: String): CrossValidatorModel = super.load(path) + + private[CrossValidatorModel] + class CrossValidatorModelWriter(instance: CrossValidatorModel) extends MLWriter { + + ValidatorParams.validateParams(instance) + + override protected def saveImpl(path: String): Unit = { + import org.json4s.JsonDSL._ + val extraMetadata = "avgMetrics" -> instance.avgMetrics.toSeq + ValidatorParams.saveImpl(path, instance, sc, Some(extraMetadata)) + val bestModelPath = new Path(path, "bestModel").toString + instance.bestModel.asInstanceOf[MLWritable].save(bestModelPath) + } + } + + private class CrossValidatorModelReader extends MLReader[CrossValidatorModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[CrossValidatorModel].getName + + override def load(path: String): CrossValidatorModel = { + implicit val format = DefaultFormats + + val (metadata, estimator, evaluator, estimatorParamMaps) = + ValidatorParams.loadImpl(path, sc, className) + val bestModelPath = new Path(path, "bestModel").toString + val bestModel = DefaultParamsReader.loadParamsInstance[Model[_]](bestModelPath, sc) + val avgMetrics = (metadata.metadata \ "avgMetrics").extract[Seq[Double]].toArray + + val model = new CrossValidatorModel(metadata.uid, bestModel, avgMetrics) + model.set(model.estimator, estimator) + .set(model.evaluator, evaluator) + .set(model.estimatorParamMaps, estimatorParamMaps) + DefaultParamsReader.getAndSetParams(model, metadata, + skipParams = Option(List("estimatorParamMaps"))) + model + } + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/ParamGridBuilder.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/ParamGridBuilder.scala index 98a8f0330ca4..d369e7a61cdc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/ParamGridBuilder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/ParamGridBuilder.scala @@ -20,21 +20,21 @@ package org.apache.spark.ml.tuning import scala.annotation.varargs import scala.collection.mutable -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.Since import org.apache.spark.ml.param._ /** - * :: Experimental :: * Builder for a param grid used in grid search-based model selection. */ -@Experimental -class ParamGridBuilder { +@Since("1.2.0") +class ParamGridBuilder @Since("1.2.0") { private val paramGrid = mutable.Map.empty[Param[_], Iterable[_]] /** * Sets the given parameters in this grid to fixed values. */ + @Since("1.2.0") def baseOn(paramMap: ParamMap): this.type = { baseOn(paramMap.toSeq: _*) this @@ -43,6 +43,7 @@ class ParamGridBuilder { /** * Sets the given parameters in this grid to fixed values. */ + @Since("1.2.0") @varargs def baseOn(paramPairs: ParamPair[_]*): this.type = { paramPairs.foreach { p => @@ -54,6 +55,7 @@ class ParamGridBuilder { /** * Adds a param with multiple values (overwrites if the input param exists). */ + @Since("1.2.0") def addGrid[T](param: Param[T], values: Iterable[T]): this.type = { paramGrid.put(param, values) this @@ -64,13 +66,15 @@ class ParamGridBuilder { /** * Adds a double param with multiple values. */ + @Since("1.2.0") def addGrid(param: DoubleParam, values: Array[Double]): this.type = { addGrid[Double](param, values) } /** - * Adds a int param with multiple values. + * Adds an int param with multiple values. */ + @Since("1.2.0") def addGrid(param: IntParam, values: Array[Int]): this.type = { addGrid[Int](param, values) } @@ -78,6 +82,7 @@ class ParamGridBuilder { /** * Adds a float param with multiple values. */ + @Since("1.2.0") def addGrid(param: FloatParam, values: Array[Float]): this.type = { addGrid[Float](param, values) } @@ -85,6 +90,7 @@ class ParamGridBuilder { /** * Adds a long param with multiple values. */ + @Since("1.2.0") def addGrid(param: LongParam, values: Array[Long]): this.type = { addGrid[Long](param, values) } @@ -92,6 +98,7 @@ class ParamGridBuilder { /** * Adds a boolean param with true and false. */ + @Since("1.2.0") def addGrid(param: BooleanParam): this.type = { addGrid[Boolean](param, Array(true, false)) } @@ -99,6 +106,7 @@ class ParamGridBuilder { /** * Builds and returns all combinations of parameters specified by the param grid. */ + @Since("1.2.0") def build(): Array[ParamMap] = { var paramMaps = Array(new ParamMap) paramGrid.foreach { case (param, values) => diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala index 73a14b831015..6e3ad4070680 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala @@ -17,14 +17,27 @@ package org.apache.spark.ml.tuning -import org.apache.spark.Logging -import org.apache.spark.annotation.Experimental -import org.apache.spark.ml.evaluation.Evaluator +import java.io.IOException +import java.util.{List => JList} + +import scala.collection.JavaConverters._ +import scala.concurrent.Future +import scala.concurrent.duration.Duration +import scala.language.existentials + +import org.apache.hadoop.fs.Path +import org.json4s.DefaultFormats + +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.evaluation.Evaluator import org.apache.spark.ml.param.{DoubleParam, ParamMap, ParamValidators} -import org.apache.spark.ml.util.Identifiable -import org.apache.spark.sql.DataFrame +import org.apache.spark.ml.param.shared.HasParallelism +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.StructType +import org.apache.spark.util.ThreadUtils /** * Params for [[TrainValidationSplit]] and [[TrainValidationSplitModel]]. @@ -33,6 +46,7 @@ private[ml] trait TrainValidationSplitParams extends ValidatorParams { /** * Param for ratio between train and validation data. Must be between 0 and 1. * Default: 0.75 + * * @group param */ val trainRatio: DoubleParam = new DoubleParam(this, "trainRatio", @@ -45,57 +59,95 @@ private[ml] trait TrainValidationSplitParams extends ValidatorParams { } /** - * :: Experimental :: * Validation for hyper-parameter tuning. * Randomly splits the input dataset into train and validation sets, * and uses evaluation metric on the validation set to select the best model. * Similar to [[CrossValidator]], but only splits the set once. */ -@Experimental -class TrainValidationSplit(override val uid: String) extends Estimator[TrainValidationSplitModel] - with TrainValidationSplitParams with Logging { +@Since("1.5.0") +class TrainValidationSplit @Since("1.5.0") (@Since("1.5.0") override val uid: String) + extends Estimator[TrainValidationSplitModel] + with TrainValidationSplitParams with HasParallelism with MLWritable with Logging { + @Since("1.5.0") def this() = this(Identifiable.randomUID("tvs")) /** @group setParam */ + @Since("1.5.0") def setEstimator(value: Estimator[_]): this.type = set(estimator, value) /** @group setParam */ + @Since("1.5.0") def setEstimatorParamMaps(value: Array[ParamMap]): this.type = set(estimatorParamMaps, value) /** @group setParam */ + @Since("1.5.0") def setEvaluator(value: Evaluator): this.type = set(evaluator, value) /** @group setParam */ + @Since("1.5.0") def setTrainRatio(value: Double): this.type = set(trainRatio, value) - override def fit(dataset: DataFrame): TrainValidationSplitModel = { + /** @group setParam */ + @Since("2.0.0") + def setSeed(value: Long): this.type = set(seed, value) + + /** + * Set the mamixum level of parallelism to evaluate models in parallel. + * Default is 1 for serial evaluation + * + * @group expertSetParam + */ + @Since("2.3.0") + def setParallelism(value: Int): this.type = set(parallelism, value) + + @Since("2.0.0") + override def fit(dataset: Dataset[_]): TrainValidationSplitModel = { val schema = dataset.schema transformSchema(schema, logging = true) - val sqlCtx = dataset.sqlContext val est = $(estimator) val eval = $(evaluator) val epm = $(estimatorParamMaps) - val numModels = epm.length - val metrics = new Array[Double](epm.length) - val Array(training, validation) = - dataset.rdd.randomSplit(Array($(trainRatio), 1 - $(trainRatio))) - val trainingDataset = sqlCtx.createDataFrame(training, schema).cache() - val validationDataset = sqlCtx.createDataFrame(validation, schema).cache() + // Create execution context based on $(parallelism) + val executionContext = getExecutionContext + + val instr = Instrumentation.create(this, dataset) + instr.logParams(trainRatio, seed, parallelism) + logTuningParams(instr) - // multi-model training + val Array(trainingDataset, validationDataset) = + dataset.randomSplit(Array($(trainRatio), 1 - $(trainRatio)), $(seed)) + trainingDataset.cache() + validationDataset.cache() + + // Fit models in a Future for training in parallel logDebug(s"Train split with multiple sets of parameters.") - val models = est.fit(trainingDataset, epm).asInstanceOf[Seq[Model[_]]] - trainingDataset.unpersist() - var i = 0 - while (i < numModels) { - // TODO: duplicate evaluator to take extra params from input - val metric = eval.evaluate(models(i).transform(validationDataset, epm(i))) - logDebug(s"Got metric $metric for model trained with ${epm(i)}.") - metrics(i) += metric - i += 1 + val modelFutures = epm.map { paramMap => + Future[Model[_]] { + val model = est.fit(trainingDataset, paramMap) + model.asInstanceOf[Model[_]] + } (executionContext) + } + + // Unpersist training data only when all models have trained + Future.sequence[Model[_], Iterable](modelFutures)(implicitly, executionContext) + .onComplete { _ => trainingDataset.unpersist() } (executionContext) + + // Evaluate models in a Future that will calulate a metric and allow model to be cleaned up + val metricFutures = modelFutures.zip(epm).map { case (modelFuture, paramMap) => + modelFuture.map { model => + // TODO: duplicate evaluator to take extra params from input + val metric = eval.evaluate(model.transform(validationDataset, paramMap)) + logDebug(s"Got metric $metric for model trained with $paramMap.") + metric + } (executionContext) } + + // Wait for all metrics to be calculated + val metrics = metricFutures.map(ThreadUtils.awaitResult(_, Duration.Inf)) + + // Unpersist validation set once all metrics have been produced validationDataset.unpersist() logInfo(s"Train validation split metrics: ${metrics.toSeq}") @@ -105,21 +157,14 @@ class TrainValidationSplit(override val uid: String) extends Estimator[TrainVali logInfo(s"Best set of parameters:\n${epm(bestIndex)}") logInfo(s"Best train validation split metric: $bestMetric.") val bestModel = est.fit(dataset, epm(bestIndex)).asInstanceOf[Model[_]] + instr.logSuccess(bestModel) copyValues(new TrainValidationSplitModel(uid, bestModel, metrics).setParent(this)) } - override def transformSchema(schema: StructType): StructType = { - $(estimator).transformSchema(schema) - } - - override def validateParams(): Unit = { - super.validateParams() - val est = $(estimator) - for (paramMap <- $(estimatorParamMaps)) { - est.copy(paramMap).validateParams() - } - } + @Since("1.5.0") + override def transformSchema(schema: StructType): StructType = transformSchemaImpl(schema) + @Since("1.5.0") override def copy(extra: ParamMap): TrainValidationSplit = { val copied = defaultCopy(extra).asInstanceOf[TrainValidationSplit] if (copied.isDefined(estimator)) { @@ -130,41 +175,137 @@ class TrainValidationSplit(override val uid: String) extends Estimator[TrainVali } copied } + + @Since("2.0.0") + override def write: MLWriter = new TrainValidationSplit.TrainValidationSplitWriter(this) +} + +@Since("2.0.0") +object TrainValidationSplit extends MLReadable[TrainValidationSplit] { + + @Since("2.0.0") + override def read: MLReader[TrainValidationSplit] = new TrainValidationSplitReader + + @Since("2.0.0") + override def load(path: String): TrainValidationSplit = super.load(path) + + private[TrainValidationSplit] class TrainValidationSplitWriter(instance: TrainValidationSplit) + extends MLWriter { + + ValidatorParams.validateParams(instance) + + override protected def saveImpl(path: String): Unit = + ValidatorParams.saveImpl(path, instance, sc) + } + + private class TrainValidationSplitReader extends MLReader[TrainValidationSplit] { + + /** Checked against metadata when loading model */ + private val className = classOf[TrainValidationSplit].getName + + override def load(path: String): TrainValidationSplit = { + implicit val format = DefaultFormats + + val (metadata, estimator, evaluator, estimatorParamMaps) = + ValidatorParams.loadImpl(path, sc, className) + val tvs = new TrainValidationSplit(metadata.uid) + .setEstimator(estimator) + .setEvaluator(evaluator) + .setEstimatorParamMaps(estimatorParamMaps) + DefaultParamsReader.getAndSetParams(tvs, metadata, + skipParams = Option(List("estimatorParamMaps"))) + tvs + } + } } /** - * :: Experimental :: * Model from train validation split. * * @param uid Id. * @param bestModel Estimator determined best model. * @param validationMetrics Evaluated validation metrics. */ -@Experimental +@Since("1.5.0") class TrainValidationSplitModel private[ml] ( - override val uid: String, - val bestModel: Model[_], - val validationMetrics: Array[Double]) - extends Model[TrainValidationSplitModel] with TrainValidationSplitParams { + @Since("1.5.0") override val uid: String, + @Since("1.5.0") val bestModel: Model[_], + @Since("1.5.0") val validationMetrics: Array[Double]) + extends Model[TrainValidationSplitModel] with TrainValidationSplitParams with MLWritable { - override def validateParams(): Unit = { - bestModel.validateParams() + /** A Python-friendly auxiliary constructor. */ + private[ml] def this(uid: String, bestModel: Model[_], validationMetrics: JList[Double]) = { + this(uid, bestModel, validationMetrics.asScala.toArray) } - override def transform(dataset: DataFrame): DataFrame = { + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) bestModel.transform(dataset) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { bestModel.transformSchema(schema) } + @Since("1.5.0") override def copy(extra: ParamMap): TrainValidationSplitModel = { val copied = new TrainValidationSplitModel ( uid, bestModel.copy(extra).asInstanceOf[Model[_]], validationMetrics.clone()) - copyValues(copied, extra) + copyValues(copied, extra).setParent(parent) + } + + @Since("2.0.0") + override def write: MLWriter = new TrainValidationSplitModel.TrainValidationSplitModelWriter(this) +} + +@Since("2.0.0") +object TrainValidationSplitModel extends MLReadable[TrainValidationSplitModel] { + + @Since("2.0.0") + override def read: MLReader[TrainValidationSplitModel] = new TrainValidationSplitModelReader + + @Since("2.0.0") + override def load(path: String): TrainValidationSplitModel = super.load(path) + + private[TrainValidationSplitModel] + class TrainValidationSplitModelWriter(instance: TrainValidationSplitModel) extends MLWriter { + + ValidatorParams.validateParams(instance) + + override protected def saveImpl(path: String): Unit = { + import org.json4s.JsonDSL._ + val extraMetadata = "validationMetrics" -> instance.validationMetrics.toSeq + ValidatorParams.saveImpl(path, instance, sc, Some(extraMetadata)) + val bestModelPath = new Path(path, "bestModel").toString + instance.bestModel.asInstanceOf[MLWritable].save(bestModelPath) + } + } + + private class TrainValidationSplitModelReader extends MLReader[TrainValidationSplitModel] { + + /** Checked against metadata when loading model */ + private val className = classOf[TrainValidationSplitModel].getName + + override def load(path: String): TrainValidationSplitModel = { + implicit val format = DefaultFormats + + val (metadata, estimator, evaluator, estimatorParamMaps) = + ValidatorParams.loadImpl(path, sc, className) + val bestModelPath = new Path(path, "bestModel").toString + val bestModel = DefaultParamsReader.loadParamsInstance[Model[_]](bestModelPath, sc) + val validationMetrics = (metadata.metadata \ "validationMetrics").extract[Seq[Double]].toArray + + val model = new TrainValidationSplitModel(metadata.uid, bestModel, validationMetrics) + model.set(model.estimator, estimator) + .set(model.evaluator, evaluator) + .set(model.estimatorParamMaps, estimatorParamMaps) + DefaultParamsReader.getAndSetParams(model, metadata, + skipParams = Option(List("estimatorParamMaps"))) + model + } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala index 8897ab0825ac..363304ef1014 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala @@ -17,20 +17,27 @@ package org.apache.spark.ml.tuning -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.ml.Estimator +import org.apache.hadoop.fs.Path +import org.json4s.{DefaultFormats, _} +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.SparkContext +import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator -import org.apache.spark.ml.param.{ParamMap, Param, Params} +import org.apache.spark.ml.param.{Param, ParamMap, ParamPair, Params} +import org.apache.spark.ml.param.shared.HasSeed +import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.DefaultParamsReader.Metadata +import org.apache.spark.sql.types.StructType /** - * :: DeveloperApi :: * Common params for [[TrainValidationSplitParams]] and [[CrossValidatorParams]]. */ -@DeveloperApi -private[ml] trait ValidatorParams extends Params { +private[ml] trait ValidatorParams extends HasSeed with Params { /** * param for the estimator to be validated + * * @group param */ val estimator: Param[Estimator[_]] = new Param(this, "estimator", "estimator for selection") @@ -40,6 +47,7 @@ private[ml] trait ValidatorParams extends Params { /** * param for estimator param maps + * * @group param */ val estimatorParamMaps: Param[Array[ParamMap]] = @@ -50,6 +58,7 @@ private[ml] trait ValidatorParams extends Params { /** * param for the evaluator used to select hyper-parameters that maximize the validated metric + * * @group param */ val evaluator: Param[Evaluator] = new Param(this, "evaluator", @@ -57,4 +66,148 @@ private[ml] trait ValidatorParams extends Params { /** @group getParam */ def getEvaluator: Evaluator = $(evaluator) + + protected def transformSchemaImpl(schema: StructType): StructType = { + require($(estimatorParamMaps).nonEmpty, s"Validator requires non-empty estimatorParamMaps") + val firstEstimatorParamMap = $(estimatorParamMaps).head + val est = $(estimator) + for (paramMap <- $(estimatorParamMaps).tail) { + est.copy(paramMap).transformSchema(schema) + } + est.copy(firstEstimatorParamMap).transformSchema(schema) + } + + /** + * Instrumentation logging for tuning params including the inner estimator and evaluator info. + */ + protected def logTuningParams(instrumentation: Instrumentation[_]): Unit = { + instrumentation.logNamedValue("estimator", $(estimator).getClass.getCanonicalName) + instrumentation.logNamedValue("evaluator", $(evaluator).getClass.getCanonicalName) + instrumentation.logNamedValue("estimatorParamMapsLength", $(estimatorParamMaps).length) + } +} + +private[ml] object ValidatorParams { + /** + * Check that [[ValidatorParams.evaluator]] and [[ValidatorParams.estimator]] are Writable. + * This does not check [[ValidatorParams.estimatorParamMaps]]. + */ + def validateParams(instance: ValidatorParams): Unit = { + def checkElement(elem: Params, name: String): Unit = elem match { + case stage: MLWritable => // good + case other => + throw new UnsupportedOperationException(instance.getClass.getName + " write will fail " + + s" because it contains $name which does not implement Writable." + + s" Non-Writable $name: ${other.uid} of type ${other.getClass}") + } + checkElement(instance.getEvaluator, "evaluator") + checkElement(instance.getEstimator, "estimator") + // Check to make sure all Params apply to this estimator. Throw an error if any do not. + // Extraneous Params would cause problems when loading the estimatorParamMaps. + val uidToInstance: Map[String, Params] = MetaAlgorithmReadWrite.getUidMap(instance) + instance.getEstimatorParamMaps.foreach { case pMap: ParamMap => + pMap.toSeq.foreach { case ParamPair(p, v) => + require(uidToInstance.contains(p.parent), s"ValidatorParams save requires all Params in" + + s" estimatorParamMaps to apply to this ValidatorParams, its Estimator, or its" + + s" Evaluator. An extraneous Param was found: $p") + } + } + } + + /** + * Generic implementation of save for [[ValidatorParams]] types. + * This handles all [[ValidatorParams]] fields and saves [[Param]] values, but the implementing + * class needs to handle model data. + */ + def saveImpl( + path: String, + instance: ValidatorParams, + sc: SparkContext, + extraMetadata: Option[JObject] = None): Unit = { + import org.json4s.JsonDSL._ + + var numParamsNotJson = 0 + val estimatorParamMapsJson = compact(render( + instance.getEstimatorParamMaps.map { case paramMap => + paramMap.toSeq.map { case ParamPair(p, v) => + v match { + case writeableObj: DefaultParamsWritable => + val relativePath = "epm_" + p.name + numParamsNotJson + val paramPath = new Path(path, relativePath).toString + numParamsNotJson += 1 + writeableObj.save(paramPath) + Map("parent" -> p.parent, "name" -> p.name, + "value" -> compact(render(JString(relativePath))), + "isJson" -> compact(render(JBool(false)))) + case _: MLWritable => + throw new NotImplementedError("ValidatorParams.saveImpl does not handle parameters " + + "of type: MLWritable that are not DefaultParamsWritable") + case _ => + Map("parent" -> p.parent, "name" -> p.name, "value" -> p.jsonEncode(v), + "isJson" -> compact(render(JBool(true)))) + } + } + }.toSeq + )) + + val params = instance.extractParamMap().toSeq + val skipParams = List("estimator", "evaluator", "estimatorParamMaps") + val jsonParams = render(params + .filter { case ParamPair(p, v) => !skipParams.contains(p.name)} + .map { case ParamPair(p, v) => + p.name -> parse(p.jsonEncode(v)) + }.toList ++ List("estimatorParamMaps" -> parse(estimatorParamMapsJson)) + ) + + DefaultParamsWriter.saveMetadata(instance, path, sc, extraMetadata, Some(jsonParams)) + + val evaluatorPath = new Path(path, "evaluator").toString + instance.getEvaluator.asInstanceOf[MLWritable].save(evaluatorPath) + val estimatorPath = new Path(path, "estimator").toString + instance.getEstimator.asInstanceOf[MLWritable].save(estimatorPath) + } + + /** + * Generic implementation of load for [[ValidatorParams]] types. + * This handles all [[ValidatorParams]] fields, but the implementing + * class needs to handle model data and special [[Param]] values. + */ + def loadImpl[M <: Model[M]]( + path: String, + sc: SparkContext, + expectedClassName: String): (Metadata, Estimator[M], Evaluator, Array[ParamMap]) = { + + val metadata = DefaultParamsReader.loadMetadata(path, sc, expectedClassName) + + implicit val format = DefaultFormats + val evaluatorPath = new Path(path, "evaluator").toString + val evaluator = DefaultParamsReader.loadParamsInstance[Evaluator](evaluatorPath, sc) + val estimatorPath = new Path(path, "estimator").toString + val estimator = DefaultParamsReader.loadParamsInstance[Estimator[M]](estimatorPath, sc) + + val uidToParams = Map(evaluator.uid -> evaluator) ++ MetaAlgorithmReadWrite.getUidMap(estimator) + + val estimatorParamMaps: Array[ParamMap] = + (metadata.params \ "estimatorParamMaps").extract[Seq[Seq[Map[String, String]]]].map { + pMap => + val paramPairs = pMap.map { case pInfo: Map[String, String] => + val est = uidToParams(pInfo("parent")) + val param = est.getParam(pInfo("name")) + // [Spark-21221] introduced the isJson field + if (!pInfo.contains("isJson") || + (pInfo.contains("isJson") && pInfo("isJson").toBoolean.booleanValue())) { + val value = param.jsonDecode(pInfo("value")) + param -> value + } else { + val relativePath = param.jsonDecode(pInfo("value")).toString + val value = DefaultParamsReader + .loadParamsInstance[MLWritable](new Path(path, relativePath).toString, sc) + param -> value + } + } + ParamMap(paramPairs: _*) + }.toArray + + (metadata, estimator, evaluator, estimatorParamMaps) + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala b/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala new file mode 100644 index 000000000000..7c46f45c5971 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.util + +import java.util.concurrent.atomic.AtomicLong + +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.internal.Logging +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.param.Param +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Dataset + +/** + * A small wrapper that defines a training session for an estimator, and some methods to log + * useful information during this session. + * + * A new instance is expected to be created within fit(). + * + * @param estimator the estimator that is being fit + * @param dataset the training dataset + * @tparam E the type of the estimator + */ +private[spark] class Instrumentation[E <: Estimator[_]] private ( + estimator: E, dataset: RDD[_]) extends Logging { + + private val id = Instrumentation.counter.incrementAndGet() + private val prefix = { + val className = estimator.getClass.getSimpleName + s"$className-${estimator.uid}-${dataset.hashCode()}-$id: " + } + + init() + + private def init(): Unit = { + log(s"training: numPartitions=${dataset.partitions.length}" + + s" storageLevel=${dataset.getStorageLevel}") + } + + /** + * Logs a message with a prefix that uniquely identifies the training session. + */ + def log(msg: String): Unit = { + logInfo(prefix + msg) + } + + /** + * Logs the value of the given parameters for the estimator being used in this session. + */ + def logParams(params: Param[_]*): Unit = { + val pairs: Seq[(String, JValue)] = for { + p <- params + value <- estimator.get(p) + } yield { + val cast = p.asInstanceOf[Param[Any]] + p.name -> parse(cast.jsonEncode(value)) + } + log(compact(render(map2jvalue(pairs.toMap)))) + } + + def logNumFeatures(num: Long): Unit = { + log(compact(render("numFeatures" -> num))) + } + + def logNumClasses(num: Long): Unit = { + log(compact(render("numClasses" -> num))) + } + + /** + * Logs the value with customized name field. + */ + def logNamedValue(name: String, value: String): Unit = { + log(compact(render(name -> value))) + } + + def logNamedValue(name: String, value: Long): Unit = { + log(compact(render(name -> value))) + } + + /** + * Logs the successful completion of the training session. + */ + def logSuccess(model: Model[_]): Unit = { + log(s"training finished") + } +} + +/** + * Some common methods for logging information about a training session. + */ +private[spark] object Instrumentation { + private val counter = new AtomicLong(0) + + /** + * Creates an instrumentation object for a training session. + */ + def create[E <: Estimator[_]]( + estimator: E, dataset: Dataset[_]): Instrumentation[E] = { + create[E](estimator, dataset.rdd) + } + + /** + * Creates an instrumentation object for a training session. + */ + def create[E <: Estimator[_]]( + estimator: E, dataset: RDD[_]): Instrumentation[E] = { + new Instrumentation[E](estimator, dataset) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala index 96a38a3bde96..3e19f2718394 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/MetadataUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.util import scala.collection.immutable.HashMap import org.apache.spark.ml.attribute._ -import org.apache.spark.mllib.linalg.VectorUDT +import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types.StructField @@ -48,7 +48,7 @@ private[spark] object MetadataUtils { * If a feature does not have metadata, it is assumed to be continuous. * If a feature is Nominal, then it must have the number of values * specified. - * @return Map: feature index --> number of categories. + * @return Map: feature index to number of categories. * The map's set of keys will be the set of categorical feature indices. */ def getCategoricalFeatures(featuresSchema: StructField): Map[Int, Int] = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala new file mode 100644 index 000000000000..7188da353126 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala @@ -0,0 +1,490 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.util + +import java.io.IOException + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.{DefaultFormats, JObject} +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging +import org.apache.spark.ml._ +import org.apache.spark.ml.classification.{OneVsRest, OneVsRestModel} +import org.apache.spark.ml.feature.RFormulaModel +import org.apache.spark.ml.param.{ParamPair, Params} +import org.apache.spark.ml.tuning.ValidatorParams +import org.apache.spark.sql.{SparkSession, SQLContext} +import org.apache.spark.util.Utils + +/** + * Trait for `MLWriter` and `MLReader`. + */ +private[util] sealed trait BaseReadWrite { + private var optionSparkSession: Option[SparkSession] = None + + /** + * Sets the Spark SQLContext to use for saving/loading. + * + * @deprecated Use session instead. This method will be removed in 3.0.0. + */ + @Since("1.6.0") + @deprecated("Use session instead. This method will be removed in 3.0.0.", "2.0.0") + def context(sqlContext: SQLContext): this.type = { + optionSparkSession = Option(sqlContext.sparkSession) + this + } + + /** + * Sets the Spark Session to use for saving/loading. + */ + @Since("2.0.0") + def session(sparkSession: SparkSession): this.type = { + optionSparkSession = Option(sparkSession) + this + } + + /** + * Returns the user-specified Spark Session or the default. + */ + protected final def sparkSession: SparkSession = { + if (optionSparkSession.isEmpty) { + optionSparkSession = Some(SparkSession.builder().getOrCreate()) + } + optionSparkSession.get + } + + /** + * Returns the user-specified SQL context or the default. + */ + protected final def sqlContext: SQLContext = sparkSession.sqlContext + + /** Returns the underlying `SparkContext`. */ + protected final def sc: SparkContext = sparkSession.sparkContext +} + +/** + * Abstract class for utility classes that can save ML instances. + */ +@Since("1.6.0") +abstract class MLWriter extends BaseReadWrite with Logging { + + protected var shouldOverwrite: Boolean = false + + /** + * Saves the ML instances to the input path. + */ + @Since("1.6.0") + @throws[IOException]("If the input path already exists but overwrite is not enabled.") + def save(path: String): Unit = { + new FileSystemOverwrite().handleOverwrite(path, shouldOverwrite, sc) + saveImpl(path) + } + + /** + * `save()` handles overwriting and then calls this method. Subclasses should override this + * method to implement the actual saving of the instance. + */ + @Since("1.6.0") + protected def saveImpl(path: String): Unit + + /** + * Overwrites if the output path already exists. + */ + @Since("1.6.0") + def overwrite(): this.type = { + shouldOverwrite = true + this + } + + // override for Java compatibility + override def session(sparkSession: SparkSession): this.type = super.session(sparkSession) + + // override for Java compatibility + override def context(sqlContext: SQLContext): this.type = super.session(sqlContext.sparkSession) +} + +/** + * Trait for classes that provide `MLWriter`. + */ +@Since("1.6.0") +trait MLWritable { + + /** + * Returns an `MLWriter` instance for this ML instance. + */ + @Since("1.6.0") + def write: MLWriter + + /** + * Saves this ML instance to the input path, a shortcut of `write.save(path)`. + */ + @Since("1.6.0") + @throws[IOException]("If the input path already exists but overwrite is not enabled.") + def save(path: String): Unit = write.save(path) +} + +/** + * :: DeveloperApi :: + * + * Helper trait for making simple `Params` types writable. If a `Params` class stores + * all data as [[org.apache.spark.ml.param.Param]] values, then extending this trait will provide + * a default implementation of writing saved instances of the class. + * This only handles simple [[org.apache.spark.ml.param.Param]] types; e.g., it will not handle + * [[org.apache.spark.sql.Dataset]]. + * + * @see `DefaultParamsReadable`, the counterpart to this trait + */ +@DeveloperApi +trait DefaultParamsWritable extends MLWritable { self: Params => + + override def write: MLWriter = new DefaultParamsWriter(this) +} + +/** + * Abstract class for utility classes that can load ML instances. + * + * @tparam T ML instance type + */ +@Since("1.6.0") +abstract class MLReader[T] extends BaseReadWrite { + + /** + * Loads the ML component from the input path. + */ + @Since("1.6.0") + def load(path: String): T + + // override for Java compatibility + override def session(sparkSession: SparkSession): this.type = super.session(sparkSession) + + // override for Java compatibility + override def context(sqlContext: SQLContext): this.type = super.session(sqlContext.sparkSession) +} + +/** + * Trait for objects that provide `MLReader`. + * + * @tparam T ML instance type + */ +@Since("1.6.0") +trait MLReadable[T] { + + /** + * Returns an `MLReader` instance for this class. + */ + @Since("1.6.0") + def read: MLReader[T] + + /** + * Reads an ML instance from the input path, a shortcut of `read.load(path)`. + * + * @note Implementing classes should override this to be Java-friendly. + */ + @Since("1.6.0") + def load(path: String): T = read.load(path) +} + + +/** + * :: DeveloperApi :: + * + * Helper trait for making simple `Params` types readable. If a `Params` class stores + * all data as [[org.apache.spark.ml.param.Param]] values, then extending this trait will provide + * a default implementation of reading saved instances of the class. + * This only handles simple [[org.apache.spark.ml.param.Param]] types; e.g., it will not handle + * [[org.apache.spark.sql.Dataset]]. + * + * @tparam T ML instance type + * @see `DefaultParamsWritable`, the counterpart to this trait + */ +@DeveloperApi +trait DefaultParamsReadable[T] extends MLReadable[T] { + + override def read: MLReader[T] = new DefaultParamsReader[T] +} + +/** + * Default `MLWriter` implementation for transformers and estimators that contain basic + * (json4s-serializable) params and no data. This will not handle more complex params or types with + * data (e.g., models with coefficients). + * + * @param instance object to save + */ +private[ml] class DefaultParamsWriter(instance: Params) extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + } +} + +private[ml] object DefaultParamsWriter { + + /** + * Saves metadata + Params to: path + "/metadata" + * - class + * - timestamp + * - sparkVersion + * - uid + * - paramMap + * - (optionally, extra metadata) + * + * @param extraMetadata Extra metadata to be saved at same level as uid, paramMap, etc. + * @param paramMap If given, this is saved in the "paramMap" field. + * Otherwise, all [[org.apache.spark.ml.param.Param]]s are encoded using + * [[org.apache.spark.ml.param.Param.jsonEncode()]]. + */ + def saveMetadata( + instance: Params, + path: String, + sc: SparkContext, + extraMetadata: Option[JObject] = None, + paramMap: Option[JValue] = None): Unit = { + val metadataPath = new Path(path, "metadata").toString + val metadataJson = getMetadataToSave(instance, sc, extraMetadata, paramMap) + sc.parallelize(Seq(metadataJson), 1).saveAsTextFile(metadataPath) + } + + /** + * Helper for [[saveMetadata()]] which extracts the JSON to save. + * This is useful for ensemble models which need to save metadata for many sub-models. + * + * @see [[saveMetadata()]] for details on what this includes. + */ + def getMetadataToSave( + instance: Params, + sc: SparkContext, + extraMetadata: Option[JObject] = None, + paramMap: Option[JValue] = None): String = { + val uid = instance.uid + val cls = instance.getClass.getName + val params = instance.extractParamMap().toSeq.asInstanceOf[Seq[ParamPair[Any]]] + val jsonParams = paramMap.getOrElse(render(params.map { case ParamPair(p, v) => + p.name -> parse(p.jsonEncode(v)) + }.toList)) + val basicMetadata = ("class" -> cls) ~ + ("timestamp" -> System.currentTimeMillis()) ~ + ("sparkVersion" -> sc.version) ~ + ("uid" -> uid) ~ + ("paramMap" -> jsonParams) + val metadata = extraMetadata match { + case Some(jObject) => + basicMetadata ~ jObject + case None => + basicMetadata + } + val metadataJson: String = compact(render(metadata)) + metadataJson + } +} + +/** + * Default `MLReader` implementation for transformers and estimators that contain basic + * (json4s-serializable) params and no data. This will not handle more complex params or types with + * data (e.g., models with coefficients). + * + * @tparam T ML instance type + * TODO: Consider adding check for correct class name. + */ +private[ml] class DefaultParamsReader[T] extends MLReader[T] { + + override def load(path: String): T = { + val metadata = DefaultParamsReader.loadMetadata(path, sc) + val cls = Utils.classForName(metadata.className) + val instance = + cls.getConstructor(classOf[String]).newInstance(metadata.uid).asInstanceOf[Params] + DefaultParamsReader.getAndSetParams(instance, metadata) + instance.asInstanceOf[T] + } +} + +private[ml] object DefaultParamsReader { + + /** + * All info from metadata file. + * + * @param params paramMap, as a `JValue` + * @param metadata All metadata, including the other fields + * @param metadataJson Full metadata file String (for debugging) + */ + case class Metadata( + className: String, + uid: String, + timestamp: Long, + sparkVersion: String, + params: JValue, + metadata: JValue, + metadataJson: String) { + + /** + * Get the JSON value of the [[org.apache.spark.ml.param.Param]] of the given name. + * This can be useful for getting a Param value before an instance of `Params` + * is available. + */ + def getParamValue(paramName: String): JValue = { + implicit val format = DefaultFormats + params match { + case JObject(pairs) => + val values = pairs.filter { case (pName, jsonValue) => + pName == paramName + }.map(_._2) + assert(values.length == 1, s"Expected one instance of Param '$paramName' but found" + + s" ${values.length} in JSON Params: " + pairs.map(_.toString).mkString(", ")) + values.head + case _ => + throw new IllegalArgumentException( + s"Cannot recognize JSON metadata: $metadataJson.") + } + } + } + + /** + * Load metadata saved using [[DefaultParamsWriter.saveMetadata()]] + * + * @param expectedClassName If non empty, this is checked against the loaded metadata. + * @throws IllegalArgumentException if expectedClassName is specified and does not match metadata + */ + def loadMetadata(path: String, sc: SparkContext, expectedClassName: String = ""): Metadata = { + val metadataPath = new Path(path, "metadata").toString + val metadataStr = sc.textFile(metadataPath, 1).first() + parseMetadata(metadataStr, expectedClassName) + } + + /** + * Parse metadata JSON string produced by [[DefaultParamsWriter.getMetadataToSave()]]. + * This is a helper function for [[loadMetadata()]]. + * + * @param metadataStr JSON string of metadata + * @param expectedClassName If non empty, this is checked against the loaded metadata. + * @throws IllegalArgumentException if expectedClassName is specified and does not match metadata + */ + def parseMetadata(metadataStr: String, expectedClassName: String = ""): Metadata = { + val metadata = parse(metadataStr) + + implicit val format = DefaultFormats + val className = (metadata \ "class").extract[String] + val uid = (metadata \ "uid").extract[String] + val timestamp = (metadata \ "timestamp").extract[Long] + val sparkVersion = (metadata \ "sparkVersion").extract[String] + val params = metadata \ "paramMap" + if (expectedClassName.nonEmpty) { + require(className == expectedClassName, s"Error loading metadata: Expected class name" + + s" $expectedClassName but found class name $className") + } + + Metadata(className, uid, timestamp, sparkVersion, params, metadata, metadataStr) + } + + /** + * Extract Params from metadata, and set them in the instance. + * This works if all Params (except params included by `skipParams` list) implement + * [[org.apache.spark.ml.param.Param.jsonDecode()]]. + * + * @param skipParams The params included in `skipParams` won't be set. This is useful if some + * params don't implement [[org.apache.spark.ml.param.Param.jsonDecode()]] + * and need special handling. + * TODO: Move to [[Metadata]] method + */ + def getAndSetParams( + instance: Params, + metadata: Metadata, + skipParams: Option[List[String]] = None): Unit = { + implicit val format = DefaultFormats + metadata.params match { + case JObject(pairs) => + pairs.foreach { case (paramName, jsonValue) => + if (skipParams == None || !skipParams.get.contains(paramName)) { + val param = instance.getParam(paramName) + val value = param.jsonDecode(compact(render(jsonValue))) + instance.set(param, value) + } + } + case _ => + throw new IllegalArgumentException( + s"Cannot recognize JSON metadata: ${metadata.metadataJson}.") + } + } + + /** + * Load a `Params` instance from the given path, and return it. + * This assumes the instance implements [[MLReadable]]. + */ + def loadParamsInstance[T](path: String, sc: SparkContext): T = { + val metadata = DefaultParamsReader.loadMetadata(path, sc) + val cls = Utils.classForName(metadata.className) + cls.getMethod("read").invoke(null).asInstanceOf[MLReader[T]].load(path) + } +} + +/** + * Default Meta-Algorithm read and write implementation. + */ +private[ml] object MetaAlgorithmReadWrite { + /** + * Examine the given estimator (which may be a compound estimator) and extract a mapping + * from UIDs to corresponding `Params` instances. + */ + def getUidMap(instance: Params): Map[String, Params] = { + val uidList = getUidMapImpl(instance) + val uidMap = uidList.toMap + if (uidList.size != uidMap.size) { + throw new RuntimeException(s"${instance.getClass.getName}.load found a compound estimator" + + s" with stages with duplicate UIDs. List of UIDs: ${uidList.map(_._1).mkString(", ")}.") + } + uidMap + } + + private def getUidMapImpl(instance: Params): List[(String, Params)] = { + val subStages: Array[Params] = instance match { + case p: Pipeline => p.getStages.asInstanceOf[Array[Params]] + case pm: PipelineModel => pm.stages.asInstanceOf[Array[Params]] + case v: ValidatorParams => Array(v.getEstimator, v.getEvaluator) + case ovr: OneVsRest => Array(ovr.getClassifier) + case ovrModel: OneVsRestModel => Array(ovrModel.getClassifier) ++ ovrModel.models + case rformModel: RFormulaModel => Array(rformModel.pipelineModel) + case _: Params => Array.empty[Params] + } + val subStageMaps = subStages.flatMap(getUidMapImpl) + List((instance.uid, instance)) ++ subStageMaps + } +} + +private[ml] class FileSystemOverwrite extends Logging { + + def handleOverwrite(path: String, shouldOverwrite: Boolean, sc: SparkContext): Unit = { + val hadoopConf = sc.hadoopConfiguration + val outputPath = new Path(path) + val fs = outputPath.getFileSystem(hadoopConf) + val qualifiedOutputPath = outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + if (fs.exists(qualifiedOutputPath)) { + if (shouldOverwrite) { + logInfo(s"Path $path already exists. It will be overwritten.") + // TODO: Revert back to the original content if save is not successful. + fs.delete(qualifiedOutputPath, true) + } else { + throw new IOException(s"Path $path already exists. To overwrite it, " + + s"please use write.overwrite().save(path) for Scala and use " + + s"write().overwrite().save(path) for Java and Python.") + } + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala index 76f651488aef..334410c9620d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.util -import org.apache.spark.sql.types.{DataType, StructField, StructType} +import org.apache.spark.sql.types.{DataType, NumericType, StructField, StructType} /** @@ -43,6 +43,37 @@ private[spark] object SchemaUtils { s"Column $colName must be of type $dataType but was actually $actualDataType.$message") } + /** + * Check whether the given schema contains a column of one of the require data types. + * @param colName column name + * @param dataTypes required column data types + */ + def checkColumnTypes( + schema: StructType, + colName: String, + dataTypes: Seq[DataType], + msg: String = ""): Unit = { + val actualDataType = schema(colName).dataType + val message = if (msg != null && msg.trim.length > 0) " " + msg else "" + require(dataTypes.exists(actualDataType.equals), + s"Column $colName must be of type equal to one of the following types: " + + s"${dataTypes.mkString("[", ", ", "]")} but was actually of type $actualDataType.$message") + } + + /** + * Check whether the given schema contains a column of the numeric data type. + * @param colName column name + */ + def checkNumericType( + schema: StructType, + colName: String, + msg: String = ""): Unit = { + val actualDataType = schema(colName).dataType + val message = if (msg != null && msg.trim.length > 0) " " + msg else "" + require(actualDataType.isInstanceOf[NumericType], s"Column $colName must be of type " + + s"NumericType but was actually of type $actualDataType.$message") + } + /** * Appends a new column to the input schema. This fails if the given output column already exists. * @param schema input schema @@ -54,12 +85,10 @@ private[spark] object SchemaUtils { def appendColumn( schema: StructType, colName: String, - dataType: DataType): StructType = { + dataType: DataType, + nullable: Boolean = false): StructType = { if (colName.isEmpty) return schema - val fieldNames = schema.fieldNames - require(!fieldNames.contains(colName), s"Column $colName already exists.") - val outputFields = schema.fields :+ StructField(colName, dataType, nullable = false) - StructType(outputFields) + appendColumn(schema, StructField(colName, dataType, nullable)) } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala b/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala index 8d4174124b5c..e539deca4b03 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala @@ -19,7 +19,8 @@ package org.apache.spark.ml.util import scala.collection.mutable -import org.apache.spark.{Accumulator, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.util.LongAccumulator /** * Abstract class for stopwatches. @@ -102,12 +103,12 @@ private[spark] class DistributedStopwatch( sc: SparkContext, override val name: String) extends Stopwatch { - private val elapsedTime: Accumulator[Long] = sc.accumulator(0L, s"DistributedStopwatch($name)") + private val elapsedTime: LongAccumulator = sc.longAccumulator(s"DistributedStopwatch($name)") override def elapsed(): Long = elapsedTime.value override protected def add(duration: Long): Unit = { - elapsedTime += duration + elapsedTime.add(duration) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/JavaPackage.java b/mllib/src/main/scala/org/apache/spark/mllib/JavaPackage.java new file mode 100644 index 000000000000..22e34524aa59 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/JavaPackage.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib; + +import org.apache.spark.annotation.AlphaComponent; + +/** + * A dummy class as a workaround to show the package doc of spark.mllib in generated + * Java API docs. + * @see + * JDK-4492654 + */ +@AlphaComponent +public class JavaPackage { + private JavaPackage() {} +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala index ee933f4cfcaf..e6d1dceebed4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala @@ -17,8 +17,7 @@ package org.apache.spark.mllib.api.python -import org.apache.spark.api.java.JavaRDD -import org.apache.spark.mllib.fpm.{FPGrowth, FPGrowthModel} +import org.apache.spark.mllib.fpm.FPGrowthModel import org.apache.spark.rdd.RDD /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/GaussianMixtureModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/GaussianMixtureModelWrapper.scala index 0ec88ef77d69..364d5eea08ce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/GaussianMixtureModelWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/GaussianMixtureModelWrapper.scala @@ -17,36 +17,31 @@ package org.apache.spark.mllib.api.python -import java.util.{List => JList} - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConverters import org.apache.spark.SparkContext -import org.apache.spark.mllib.linalg.{Vector, Vectors, Matrix} import org.apache.spark.mllib.clustering.GaussianMixtureModel +import org.apache.spark.mllib.linalg.{Vector, Vectors} /** - * Wrapper around GaussianMixtureModel to provide helper methods in Python - */ + * Wrapper around GaussianMixtureModel to provide helper methods in Python + */ private[python] class GaussianMixtureModelWrapper(model: GaussianMixtureModel) { val weights: Vector = Vectors.dense(model.weights) val k: Int = weights.size /** - * Returns gaussians as a List of Vectors and Matrices corresponding each MultivariateGaussian - */ - val gaussians: JList[Object] = { - val modelGaussians = model.gaussians - var i = 0 - var mu = ArrayBuffer.empty[Vector] - var sigma = ArrayBuffer.empty[Matrix] - while (i < k) { - mu += modelGaussians(i).mu - sigma += modelGaussians(i).sigma - i += 1 + * Returns gaussians as a List of Vectors and Matrices corresponding each MultivariateGaussian + */ + val gaussians: Array[Byte] = { + val modelGaussians = model.gaussians.map { gaussian => + Array[Any](gaussian.mu, gaussian.sigma) } - List(mu.toArray, sigma.toArray).map(_.asInstanceOf[Object]).asJava + SerDe.dumps(JavaConverters.seqAsJavaListConverter(modelGaussians).asJava) + } + + def predictSoft(point: Vector): Vector = { + Vectors.dense(model.predictSoft(point)) } def save(sc: SparkContext, path: String): Unit = model.save(sc, path) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/LDAModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/LDAModelWrapper.scala new file mode 100644 index 000000000000..63282eee6e65 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/LDAModelWrapper.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.mllib.api.python + +import scala.collection.JavaConverters + +import org.apache.spark.SparkContext +import org.apache.spark.mllib.clustering.LDAModel +import org.apache.spark.mllib.linalg.Matrix + +/** + * Wrapper around LDAModel to provide helper methods in Python + */ +private[python] class LDAModelWrapper(model: LDAModel) { + + def topicsMatrix(): Matrix = model.topicsMatrix + + def vocabSize(): Int = model.vocabSize + + def describeTopics(): Array[Byte] = describeTopics(this.model.vocabSize) + + def describeTopics(maxTermsPerTopic: Int): Array[Byte] = { + val topics = model.describeTopics(maxTermsPerTopic).map { case (terms, termWeights) => + val jTerms = JavaConverters.seqAsJavaListConverter(terms).asJava + val jTermWeights = JavaConverters.seqAsJavaListConverter(termWeights).asJava + Array[Any](jTerms, jTermWeights) + } + SerDe.dumps(JavaConverters.seqAsJavaListConverter(topics).asJava) + } + + def save(sc: SparkContext, path: String): Unit = model.save(sc, path) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PowerIterationClusteringModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PowerIterationClusteringModelWrapper.scala index bc6041b22173..6530870b83a1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PowerIterationClusteringModelWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PowerIterationClusteringModelWrapper.scala @@ -17,8 +17,8 @@ package org.apache.spark.mllib.api.python -import org.apache.spark.rdd.RDD import org.apache.spark.mllib.clustering.PowerIterationClusteringModel +import org.apache.spark.rdd.RDD /** * A Wrapper of PowerIterationClusteringModel to provide helper method for Python diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 40c41806cdfe..b32d3f252ae5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -19,16 +19,15 @@ package org.apache.spark.mllib.api.python import java.io.OutputStream import java.nio.{ByteBuffer, ByteOrder} +import java.nio.charset.StandardCharsets import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer import scala.language.existentials import scala.reflect.ClassTag import net.razorvine.pickle._ -import org.apache.spark.SparkContext import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.python.SerDeUtil import org.apache.spark.mllib.classification._ @@ -42,20 +41,19 @@ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.random.{RandomRDDs => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.stat.{KernelDensity, MultivariateStatisticalSummary, Statistics} import org.apache.spark.mllib.stat.correlation.CorrelationNames import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.stat.test.{ChiSqTestResult, KolmogorovSmirnovTestResult} -import org.apache.spark.mllib.stat.{ - KernelDensity, MultivariateStatisticalSummary, Statistics} +import org.apache.spark.mllib.tree.{DecisionTree, GradientBoostedTrees, RandomForest} import org.apache.spark.mllib.tree.configuration.{Algo, BoostingStrategy, Strategy} import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.loss.Losses -import org.apache.spark.mllib.tree.model.{DecisionTreeModel, GradientBoostedTreesModel, RandomForestModel} -import org.apache.spark.mllib.tree.{DecisionTree, GradientBoostedTrees, RandomForest} -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.util.LinearDataGenerator +import org.apache.spark.mllib.tree.model.{DecisionTreeModel, GradientBoostedTreesModel, + RandomForestModel} +import org.apache.spark.mllib.util.{LinearDataGenerator, MLUtils} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -120,6 +118,23 @@ private[python] class PythonMLLibAPI extends Serializable { } } + /** + * Java stub for Python mllib BisectingKMeans.run() + */ + def trainBisectingKMeans( + data: JavaRDD[Vector], + k: Int, + maxIterations: Int, + minDivisibleClusterSize: Double, + seed: java.lang.Long): BisectingKMeansModel = { + val kmeans = new BisectingKMeans() + .setK(k) + .setMaxIterations(maxIterations) + .setMinDivisibleClusterSize(minDivisibleClusterSize) + if (seed != null) kmeans.setSeed(seed) + kmeans.run(data) + } + /** * Java stub for Python mllib LinearRegressionWithSGD.train() */ @@ -134,7 +149,7 @@ private[python] class PythonMLLibAPI extends Serializable { intercept: Boolean, validateData: Boolean, convergenceTol: Double): JList[Object] = { - val lrAlg = new LinearRegressionWithSGD() + val lrAlg = new LinearRegressionWithSGD(1.0, 100, 0.0, 1.0) lrAlg.setIntercept(intercept) .setValidateData(validateData) lrAlg.optimizer @@ -163,7 +178,7 @@ private[python] class PythonMLLibAPI extends Serializable { intercept: Boolean, validateData: Boolean, convergenceTol: Double): JList[Object] = { - val lassoAlg = new LassoWithSGD() + val lassoAlg = new LassoWithSGD(1.0, 100, 0.01, 1.0) lassoAlg.setIntercept(intercept) .setValidateData(validateData) lassoAlg.optimizer @@ -191,7 +206,7 @@ private[python] class PythonMLLibAPI extends Serializable { intercept: Boolean, validateData: Boolean, convergenceTol: Double): JList[Object] = { - val ridgeAlg = new RidgeRegressionWithSGD() + val ridgeAlg = new RidgeRegressionWithSGD(1.0, 100, 0.01, 1.0) ridgeAlg.setIntercept(intercept) .setValidateData(validateData) ridgeAlg.optimizer @@ -250,7 +265,7 @@ private[python] class PythonMLLibAPI extends Serializable { intercept: Boolean, validateData: Boolean, convergenceTol: Double): JList[Object] = { - val LogRegAlg = new LogisticRegressionWithSGD() + val LogRegAlg = new LogisticRegressionWithSGD(1.0, 100, 0.01, 1.0) LogRegAlg.setIntercept(intercept) .setValidateData(validateData) LogRegAlg.optimizer @@ -341,7 +356,6 @@ private[python] class PythonMLLibAPI extends Serializable { val kMeansAlg = new KMeans() .setK(k) .setMaxIterations(maxIterations) - .setRuns(runs) .setInitializationMode(initializationMode) .setInitializationSteps(initializationSteps) .setEpsilon(epsilon) @@ -413,7 +427,7 @@ private[python] class PythonMLLibAPI extends Serializable { val weight = wt.toArray val mean = mu.map(_.asInstanceOf[DenseVector]) val sigma = si.map(_.asInstanceOf[DenseMatrix]) - val gaussians = Array.tabulate(weight.length){ + val gaussians = Array.tabulate(weight.length) { i => new MultivariateGaussian(mean(i), sigma(i)) } val model = new GaussianMixtureModel(weight, gaussians) @@ -517,7 +531,7 @@ private[python] class PythonMLLibAPI extends Serializable { topicConcentration: Double, seed: java.lang.Long, checkpointInterval: Int, - optimizer: String): LDAModel = { + optimizer: String): LDAModelWrapper = { val algo = new LDA() .setK(k) .setMaxIterations(maxIterations) @@ -535,7 +549,16 @@ private[python] class PythonMLLibAPI extends Serializable { case _ => throw new IllegalArgumentException("input values contains invalid type value.") } } - algo.run(documents) + val model = algo.run(documents) + new LDAModelWrapper(model) + } + + /** + * Load a LDA model + */ + def loadLDAModel(jsc: JavaSparkContext, path: String): LDAModelWrapper = { + val model = DistributedLDAModel.load(jsc.sc, path) + new LDAModelWrapper(model) } @@ -611,8 +634,22 @@ private[python] class PythonMLLibAPI extends Serializable { * Extra care needs to be taken in the Python code to ensure it gets freed on * exit; see the Py4J documentation. */ - def fitChiSqSelector(numTopFeatures: Int, data: JavaRDD[LabeledPoint]): ChiSqSelectorModel = { - new ChiSqSelector(numTopFeatures).fit(data.rdd) + def fitChiSqSelector( + selectorType: String, + numTopFeatures: Int, + percentile: Double, + fpr: Double, + fdr: Double, + fwe: Double, + data: JavaRDD[LabeledPoint]): ChiSqSelectorModel = { + new ChiSqSelector() + .setSelectorType(selectorType) + .setNumTopFeatures(numTopFeatures) + .setPercentile(percentile) + .setFpr(fpr) + .setFdr(fdr) + .setFwe(fwe) + .fit(data.rdd) } /** @@ -646,6 +683,7 @@ private[python] class PythonMLLibAPI extends Serializable { * @param numPartitions number of partitions * @param numIterations number of iterations * @param seed initial seed for random generator + * @param windowSize size of window * @return A handle to java Word2VecModelWrapper instance at python side */ def trainWord2VecModel( @@ -654,15 +692,17 @@ private[python] class PythonMLLibAPI extends Serializable { learningRate: Double, numPartitions: Int, numIterations: Int, - seed: Long, - minCount: Int): Word2VecModelWrapper = { + seed: java.lang.Long, + minCount: Int, + windowSize: Int): Word2VecModelWrapper = { val word2vec = new Word2Vec() .setVectorSize(vectorSize) .setLearningRate(learningRate) .setNumPartitions(numPartitions) .setNumIterations(numIterations) - .setSeed(seed) .setMinCount(minCount) + .setWindowSize(windowSize) + if (seed != null) word2vec.setSeed(seed) try { val model = word2vec.fit(dataJRDD.rdd.persist(StorageLevel.MEMORY_AND_DISK_SER)) new Word2VecModelWrapper(model) @@ -671,39 +711,6 @@ private[python] class PythonMLLibAPI extends Serializable { } } - private[python] class Word2VecModelWrapper(model: Word2VecModel) { - def transform(word: String): Vector = { - model.transform(word) - } - - /** - * Transforms an RDD of words to its vector representation - * @param rdd an RDD of words - * @return an RDD of vector representations of words - */ - def transform(rdd: JavaRDD[String]): JavaRDD[Vector] = { - rdd.rdd.map(model.transform) - } - - def findSynonyms(word: String, num: Int): JList[Object] = { - val vec = transform(word) - findSynonyms(vec, num) - } - - def findSynonyms(vector: Vector, num: Int): JList[Object] = { - val result = model.findSynonyms(vector, num) - val similarity = Vectors.dense(result.map(_._2)) - val words = result.map(_._1) - List(words, similarity).map(_.asInstanceOf[Object]).asJava - } - - def getVectors: JMap[String, JList[Float]] = { - model.getVectors.map({case (k, v) => (k, v.toList.asJava)}).asJava - } - - def save(sc: SparkContext, path: String): Unit = model.save(sc, path) - } - /** * Java stub for Python mllib DecisionTree.train(). * This stub returns a handle to the Java object instead of the content of the Java object. @@ -758,7 +765,7 @@ private[python] class PythonMLLibAPI extends Serializable { impurityStr: String, maxDepth: Int, maxBins: Int, - seed: Int): RandomForestModel = { + seed: java.lang.Long): RandomForestModel = { val algo = Algo.fromString(algoStr) val impurity = Impurities.fromString(impurityStr) @@ -770,11 +777,13 @@ private[python] class PythonMLLibAPI extends Serializable { maxBins = maxBins, categoricalFeaturesInfo = categoricalFeaturesInfo.asScala.toMap) val cached = data.rdd.persist(StorageLevel.MEMORY_AND_DISK) + // Only done because methods below want an int, not an optional Long + val intSeed = getSeedOrDefault(seed).toInt try { if (algo == Algo.Classification) { - RandomForest.trainClassifier(cached, strategy, numTrees, featureSubsetStrategy, seed) + RandomForest.trainClassifier(cached, strategy, numTrees, featureSubsetStrategy, intSeed) } else { - RandomForest.trainRegressor(cached, strategy, numTrees, featureSubsetStrategy, seed) + RandomForest.trainRegressor(cached, strategy, numTrees, featureSubsetStrategy, intSeed) } } finally { cached.unpersist(blocking = false) @@ -1060,7 +1069,7 @@ private[python] class PythonMLLibAPI extends Serializable { * Java stub for the constructor of Python mllib RankingMetrics */ def newRankingMetrics(predictionAndLabels: DataFrame): RankingMetrics[Any] = { - new RankingMetrics(predictionAndLabels.map( + new RankingMetrics(predictionAndLabels.rdd.map( r => (r.getSeq(0).toArray[Any], r.getSeq(1).toArray[Any]))) } @@ -1143,7 +1152,7 @@ private[python] class PythonMLLibAPI extends Serializable { def createIndexedRowMatrix(rows: DataFrame, numRows: Long, numCols: Int): IndexedRowMatrix = { // We use DataFrames for serialization of IndexedRows from Python, // so map each Row in the DataFrame back to an IndexedRow. - val indexedRows = rows.map { + val indexedRows = rows.rdd.map { case Row(index: Long, vector: Vector) => IndexedRow(index, vector) } new IndexedRowMatrix(indexedRows, numRows, numCols) @@ -1155,7 +1164,7 @@ private[python] class PythonMLLibAPI extends Serializable { def createCoordinateMatrix(rows: DataFrame, numRows: Long, numCols: Long): CoordinateMatrix = { // We use DataFrames for serialization of MatrixEntry entries from // Python, so map each Row in the DataFrame back to a MatrixEntry. - val entries = rows.map { + val entries = rows.rdd.map { case Row(i: Long, j: Long, value: Double) => MatrixEntry(i, j, value) } new CoordinateMatrix(entries, numRows, numCols) @@ -1169,7 +1178,7 @@ private[python] class PythonMLLibAPI extends Serializable { // We use DataFrames for serialization of sub-matrix blocks from // Python, so map each Row in the DataFrame back to a // ((blockRowIndex, blockColIndex), sub-matrix) tuple. - val blockTuples = blocks.map { + val blockTuples = blocks.rdd.map { case Row(Row(blockRowIndex: Long, blockColIndex: Long), subMatrix: Matrix) => ((blockRowIndex.toInt, blockColIndex.toInt), subMatrix) } @@ -1182,8 +1191,9 @@ private[python] class PythonMLLibAPI extends Serializable { def getIndexedRows(indexedRowMatrix: IndexedRowMatrix): DataFrame = { // We use DataFrames for serialization of IndexedRows to Python, // so return a DataFrame. - val sqlContext = new SQLContext(indexedRowMatrix.rows.sparkContext) - sqlContext.createDataFrame(indexedRowMatrix.rows) + val sc = indexedRowMatrix.rows.sparkContext + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + spark.createDataFrame(indexedRowMatrix.rows) } /** @@ -1192,8 +1202,9 @@ private[python] class PythonMLLibAPI extends Serializable { def getMatrixEntries(coordinateMatrix: CoordinateMatrix): DataFrame = { // We use DataFrames for serialization of MatrixEntry entries to // Python, so return a DataFrame. - val sqlContext = new SQLContext(coordinateMatrix.entries.sparkContext) - sqlContext.createDataFrame(coordinateMatrix.entries) + val sc = coordinateMatrix.entries.sparkContext + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + spark.createDataFrame(coordinateMatrix.entries) } /** @@ -1202,23 +1213,52 @@ private[python] class PythonMLLibAPI extends Serializable { def getMatrixBlocks(blockMatrix: BlockMatrix): DataFrame = { // We use DataFrames for serialization of sub-matrix blocks to // Python, so return a DataFrame. - val sqlContext = new SQLContext(blockMatrix.blocks.sparkContext) - sqlContext.createDataFrame(blockMatrix.blocks) + val sc = blockMatrix.blocks.sparkContext + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + spark.createDataFrame(blockMatrix.blocks) + } + + /** + * Python-friendly version of [[MLUtils.convertVectorColumnsToML()]]. + */ + def convertVectorColumnsToML(dataset: DataFrame, cols: JArrayList[String]): DataFrame = { + MLUtils.convertVectorColumnsToML(dataset, cols.asScala: _*) + } + + /** + * Python-friendly version of [[MLUtils.convertVectorColumnsFromML()]] + */ + def convertVectorColumnsFromML(dataset: DataFrame, cols: JArrayList[String]): DataFrame = { + MLUtils.convertVectorColumnsFromML(dataset, cols.asScala: _*) + } + + /** + * Python-friendly version of [[MLUtils.convertMatrixColumnsToML()]]. + */ + def convertMatrixColumnsToML(dataset: DataFrame, cols: JArrayList[String]): DataFrame = { + MLUtils.convertMatrixColumnsToML(dataset, cols.asScala: _*) + } + + /** + * Python-friendly version of [[MLUtils.convertMatrixColumnsFromML()]] + */ + def convertMatrixColumnsFromML(dataset: DataFrame, cols: JArrayList[String]): DataFrame = { + MLUtils.convertMatrixColumnsFromML(dataset, cols.asScala: _*) } } /** - * SerDe utility functions for PythonMLLibAPI. + * Basic SerDe utility class. */ -private[spark] object SerDe extends Serializable { +private[spark] abstract class SerDeBase { - val PYSPARK_PACKAGE = "pyspark.mllib" - val LATIN1 = "ISO-8859-1" + val PYSPARK_PACKAGE: String + def initialize(): Unit /** * Base class used for pickle */ - private[python] abstract class BasePickler[T: ClassTag] + private[spark] abstract class BasePickler[T: ClassTag] extends IObjectPickler with IObjectConstructor { private val cls = implicitly[ClassTag[T]].runtimeClass @@ -1235,7 +1275,7 @@ private[spark] object SerDe extends Serializable { def pickle(obj: Object, out: OutputStream, pickler: Pickler): Unit = { if (obj == this) { out.write(Opcodes.GLOBAL) - out.write((module + "\n" + name + "\n").getBytes) + out.write((module + "\n" + name + "\n").getBytes(StandardCharsets.UTF_8)) } else { pickler.save(this) // it will be memorized by Pickler saveState(obj, out, pickler) @@ -1261,13 +1301,76 @@ private[spark] object SerDe extends Serializable { if (obj.getClass.isArray) { obj.asInstanceOf[Array[Byte]] } else { - obj.asInstanceOf[String].getBytes(LATIN1) + // This must be ISO 8859-1 / Latin 1, not UTF-8, to interoperate correctly + obj.asInstanceOf[String].getBytes(StandardCharsets.ISO_8859_1) } } private[python] def saveState(obj: Object, out: OutputStream, pickler: Pickler) } + def dumps(obj: AnyRef): Array[Byte] = { + obj match { + // Pickler in Python side cannot deserialize Scala Array normally. See SPARK-12834. + case array: Array[_] => new Pickler().dumps(array.toSeq.asJava) + case _ => new Pickler().dumps(obj) + } + } + + def loads(bytes: Array[Byte]): AnyRef = { + new Unpickler().loads(bytes) + } + + /* convert object into Tuple */ + def asTupleRDD(rdd: RDD[Array[Any]]): RDD[(Int, Int)] = { + rdd.map(x => (x(0).asInstanceOf[Int], x(1).asInstanceOf[Int])) + } + + /* convert RDD[Tuple2[,]] to RDD[Array[Any]] */ + def fromTuple2RDD(rdd: RDD[(Any, Any)]): RDD[Array[Any]] = { + rdd.map(x => Array(x._1, x._2)) + } + + /** + * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by + * PySpark. + */ + def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { + jRDD.rdd.mapPartitions { iter => + initialize() // let it called in executor + new SerDeUtil.AutoBatchedPickler(iter) + } + } + + /** + * Convert an RDD of serialized Python objects to RDD of objects, that is usable by PySpark. + */ + def pythonToJava(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Any] = { + pyRDD.rdd.mapPartitions { iter => + initialize() // let it called in executor + val unpickle = new Unpickler + iter.flatMap { row => + val obj = unpickle.loads(row) + if (batched) { + obj match { + case list: JArrayList[_] => list.asScala + case arr: Array[_] => arr + } + } else { + Seq(obj) + } + } + }.toJavaRDD() + } +} + +/** + * SerDe utility functions for PythonMLLibAPI. + */ +private[spark] object SerDe extends SerDeBase with Serializable { + + override val PYSPARK_PACKAGE = "pyspark.mllib" + // Pickler for DenseVector private[python] class DenseVectorPickler extends BasePickler[DenseVector] { @@ -1305,7 +1408,7 @@ private[spark] object SerDe extends Serializable { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { val m: DenseMatrix = obj.asInstanceOf[DenseMatrix] - val bytes = new Array[Byte](8 * m.values.size) + val bytes = new Array[Byte](8 * m.values.length) val order = ByteOrder.nativeOrder() val isTransposed = if (m.isTransposed) 1 else 0 ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().put(m.values) @@ -1397,7 +1500,7 @@ private[spark] object SerDe extends Serializable { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { val v: SparseVector = obj.asInstanceOf[SparseVector] - val n = v.indices.size + val n = v.indices.length val indiceBytes = new Array[Byte](4 * n) val order = ByteOrder.nativeOrder() ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().put(v.indices) @@ -1434,7 +1537,7 @@ private[spark] object SerDe extends Serializable { } } - // Pickler for LabeledPoint + // Pickler for MLlib LabeledPoint private[python] class LabeledPointPickler extends BasePickler[LabeledPoint] { def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = { @@ -1462,15 +1565,25 @@ private[spark] object SerDe extends Serializable { if (args.length != 3) { throw new PickleException("should be 3") } - new Rating(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], + new Rating(ratingsIdCheckLong(args(0)), ratingsIdCheckLong(args(1)), args(2).asInstanceOf[Double]) } + + private def ratingsIdCheckLong(obj: Object): Int = { + try { + obj.asInstanceOf[Int] + } catch { + case ex: ClassCastException => + throw new PickleException(s"Ratings id ${obj.toString} exceeds " + + s"max integer value of ${Int.MaxValue}", ex) + } + } } var initialized = false // This should be called before trying to serialize any above classes // In cluster mode, this should be put in the closure - def initialize(): Unit = { + override def initialize(): Unit = { SerDeUtil.initialize() synchronized { if (!initialized) { @@ -1486,54 +1599,4 @@ private[spark] object SerDe extends Serializable { } // will not called in Executor automatically initialize() - - def dumps(obj: AnyRef): Array[Byte] = { - new Pickler().dumps(obj) - } - - def loads(bytes: Array[Byte]): AnyRef = { - new Unpickler().loads(bytes) - } - - /* convert object into Tuple */ - def asTupleRDD(rdd: RDD[Array[Any]]): RDD[(Int, Int)] = { - rdd.map(x => (x(0).asInstanceOf[Int], x(1).asInstanceOf[Int])) - } - - /* convert RDD[Tuple2[,]] to RDD[Array[Any]] */ - def fromTuple2RDD(rdd: RDD[(Any, Any)]): RDD[Array[Any]] = { - rdd.map(x => Array(x._1, x._2)) - } - - /** - * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by - * PySpark. - */ - def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { - jRDD.rdd.mapPartitions { iter => - initialize() // let it called in executor - new SerDeUtil.AutoBatchedPickler(iter) - } - } - - /** - * Convert an RDD of serialized Python objects to RDD of objects, that is usable by PySpark. - */ - def pythonToJava(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Any] = { - pyRDD.rdd.mapPartitions { iter => - initialize() // let it called in executor - val unpickle = new Unpickler - iter.flatMap { row => - val obj = unpickle.loads(row) - if (batched) { - obj match { - case list: JArrayList[_] => list.asScala - case arr: Array[_] => arr - } - } else { - Seq(obj) - } - } - }.toJavaRDD() - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala new file mode 100644 index 000000000000..4d6520d0b2ee --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.api.python + +import java.util.{List => JList, Map => JMap} + +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkContext +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.feature.Word2VecModel +import org.apache.spark.mllib.linalg.{Vector, Vectors} + +/** + * Wrapper around Word2VecModel to provide helper methods in Python + */ +private[python] class Word2VecModelWrapper(model: Word2VecModel) { + def transform(word: String): Vector = { + model.transform(word) + } + + /** + * Transforms an RDD of words to its vector representation + * @param rdd an RDD of words + * @return an RDD of vector representations of words + */ + def transform(rdd: JavaRDD[String]): JavaRDD[Vector] = { + rdd.rdd.map(model.transform) + } + + /** + * Finds synonyms of a word; do not include the word itself in results. + * @param word a word + * @param num number of synonyms to find + * @return a list consisting of a list of words and a vector of cosine similarities + */ + def findSynonyms(word: String, num: Int): JList[Object] = { + prepareResult(model.findSynonyms(word, num)) + } + + /** + * Finds words similar to the vector representation of a word without + * filtering results. + * @param vector a vector + * @param num number of synonyms to find + * @return a list consisting of a list of words and a vector of cosine similarities + */ + def findSynonyms(vector: Vector, num: Int): JList[Object] = { + prepareResult(model.findSynonyms(vector, num)) + } + + private def prepareResult(result: Array[(String, Double)]) = { + val similarity = Vectors.dense(result.map(_._2)) + val words = result.map(_._1) + List(words, similarity).map(_.asInstanceOf[Object]).asJava + } + + + def getVectors: JMap[String, JList[Float]] = { + model.getVectors.map { case (k, v) => + (k, v.toList.asJava) + }.asJava + } + + def save(sc: SparkContext, path: String): Unit = model.save(sc, path) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 2d52abc122bf..4b650000736e 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 @@ -19,15 +19,18 @@ package org.apache.spark.mllib.classification import org.apache.spark.SparkContext import org.apache.spark.annotation.Since +import org.apache.spark.ml.linalg.DenseMatrix +import org.apache.spark.ml.util.Identifiable import org.apache.spark.mllib.classification.impl.GLMClassificationModel +import org.apache.spark.mllib.linalg.{DenseVector, Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.dot -import org.apache.spark.mllib.linalg.{DenseVector, Vector} import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.pmml.PMMLExportable 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 - +import org.apache.spark.sql.SparkSession +import org.apache.spark.storage.StorageLevel /** * Classification model trained using Multinomial/Binary Logistic Regression. @@ -84,7 +87,7 @@ class LogisticRegressionModel @Since("1.3.0") ( /** * Sets the threshold that separates positive predictions from negative predictions * in Binary Logistic Regression. An example with prediction score greater than or equal to - * this threshold is identified as an positive, and negative otherwise. The default value is 0.5. + * this threshold is identified as a positive, and negative otherwise. The default value is 0.5. * It is only used for binary classification. */ @Since("1.0.0") @@ -198,10 +201,12 @@ object LogisticRegressionModel extends Loader[LogisticRegressionModel] { /** * Train a classification model for Binary Logistic Regression * using Stochastic Gradient Descent. By default L2 regularization is used, - * which can be changed via [[LogisticRegressionWithSGD.optimizer]]. - * NOTE: Labels used in Logistic Regression should be {0, 1, ..., k - 1} - * for k classes multi-label classification problem. + * which can be changed via `LogisticRegressionWithSGD.optimizer`. + * * Using [[LogisticRegressionWithLBFGS]] is recommended over this. + * + * @note Labels used in Logistic Regression should be {0, 1, ..., k - 1} + * for k classes multi-label classification problem. */ @Since("0.8.0") class LogisticRegressionWithSGD private[mllib] ( @@ -226,6 +231,7 @@ class LogisticRegressionWithSGD private[mllib] ( * numIterations: 100, regParm: 0.01, miniBatchFraction: 1.0}. */ @Since("0.8.0") + @deprecated("Use ml.classification.LogisticRegression or LogisticRegressionWithLBFGS", "2.0.0") def this() = this(1.0, 100, 0.01, 1.0) override protected[mllib] def createModel(weights: Vector, intercept: Double) = { @@ -235,9 +241,11 @@ class LogisticRegressionWithSGD private[mllib] ( /** * Top-level methods for calling Logistic Regression using Stochastic Gradient Descent. - * NOTE: Labels used in Logistic Regression should be {0, 1} + * + * @note Labels used in Logistic Regression should be {0, 1} */ @Since("0.8.0") +@deprecated("Use ml.classification.LogisticRegression or LogisticRegressionWithLBFGS", "2.0.0") object LogisticRegressionWithSGD { // NOTE(shivaram): We use multiple train methods instead of default arguments to support // Java programs. @@ -247,7 +255,6 @@ object LogisticRegressionWithSGD { * number of iterations of gradient descent using the specified step size. Each iteration uses * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in * gradient descent are initialized using the initial weights provided. - * NOTE: Labels used in Logistic Regression should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. @@ -255,6 +262,8 @@ object LogisticRegressionWithSGD { * @param miniBatchFraction Fraction of data to be used per iteration. * @param initialWeights Initial set of weights to be used. Array should be equal in size to * the number of features in the data. + * + * @note Labels used in Logistic Regression should be {0, 1} */ @Since("1.0.0") def train( @@ -271,13 +280,13 @@ object LogisticRegressionWithSGD { * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed * number of iterations of gradient descent using the specified step size. Each iteration uses * `miniBatchFraction` fraction of the data to calculate the gradient. - * NOTE: Labels used in Logistic Regression should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. * @param stepSize Step size to be used for each iteration of gradient descent. - * @param miniBatchFraction Fraction of data to be used per iteration. + * + * @note Labels used in Logistic Regression should be {0, 1} */ @Since("1.0.0") def train( @@ -293,13 +302,13 @@ object LogisticRegressionWithSGD { * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed * number of iterations of gradient descent using the specified step size. We use the entire data * set to update the gradient in each iteration. - * NOTE: Labels used in Logistic Regression should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. - * @param numIterations Number of iterations of gradient descent to run. * @return a LogisticRegressionModel which has the weights and offset from training. + * + * @note Labels used in Logistic Regression should be {0, 1} */ @Since("1.0.0") def train( @@ -313,11 +322,12 @@ object LogisticRegressionWithSGD { * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed * number of iterations of gradient descent using a step size of 1.0. We use the entire data set * to update the gradient in each iteration. - * NOTE: Labels used in Logistic Regression should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. * @return a LogisticRegressionModel which has the weights and offset from training. + * + * @note Labels used in Logistic Regression should be {0, 1} */ @Since("1.0.0") def train( @@ -330,7 +340,15 @@ object LogisticRegressionWithSGD { /** * Train a classification model for Multinomial/Binary Logistic Regression using * Limited-memory BFGS. Standard feature scaling and L2 regularization are used by default. - * NOTE: Labels used in Logistic Regression should be {0, 1, ..., k - 1} + * + * Earlier implementations of LogisticRegressionWithLBFGS applies a regularization + * penalty to all elements including the intercept. If this is called with one of + * standard updaters (L1Updater, or SquaredL2Updater) this is translated + * into a call to ml.LogisticRegression, otherwise this will use the existing mllib + * GeneralizedLinearAlgorithm trainer, resulting in a regularization penalty to the + * intercept. + * + * @note Labels used in Logistic Regression should be {0, 1, ..., k - 1} * for k classes multi-label classification problem. */ @Since("1.1.0") @@ -374,4 +392,76 @@ class LogisticRegressionWithLBFGS new LogisticRegressionModel(weights, intercept, numFeatures, numOfLinearPredictor + 1) } } + + /** + * Run Logistic Regression with the configured parameters on an input RDD + * of LabeledPoint entries. + * + * If a known updater is used calls the ml implementation, to avoid + * applying a regularization penalty to the intercept, otherwise + * defaults to the mllib implementation. If more than two classes + * or feature scaling is disabled, always uses mllib implementation. + * If using ml implementation, uses ml code to generate initial weights. + */ + override def run(input: RDD[LabeledPoint]): LogisticRegressionModel = { + run(input, generateInitialWeights(input), userSuppliedWeights = false) + } + + /** + * Run Logistic Regression with the configured parameters on an input RDD + * of LabeledPoint entries starting from the initial weights provided. + * + * If a known updater is used calls the ml implementation, to avoid + * applying a regularization penalty to the intercept, otherwise + * defaults to the mllib implementation. If more than two classes + * or feature scaling is disabled, always uses mllib implementation. + * Uses user provided weights. + * + * In the ml LogisticRegression implementation, the number of corrections + * used in the LBFGS update can not be configured. So `optimizer.setNumCorrections()` + * will have no effect if we fall into that route. + */ + override def run(input: RDD[LabeledPoint], initialWeights: Vector): LogisticRegressionModel = { + run(input, initialWeights, userSuppliedWeights = true) + } + + private def run(input: RDD[LabeledPoint], initialWeights: Vector, userSuppliedWeights: Boolean): + LogisticRegressionModel = { + // ml's Logistic regression only supports binary classification currently. + if (numOfLinearPredictor == 1) { + def runWithMlLogisticRegression(elasticNetParam: Double) = { + // Prepare the ml LogisticRegression based on our settings + val lr = new org.apache.spark.ml.classification.LogisticRegression() + lr.setRegParam(optimizer.getRegParam()) + lr.setElasticNetParam(elasticNetParam) + lr.setStandardization(useFeatureScaling) + if (userSuppliedWeights) { + val uid = Identifiable.randomUID("logreg-static") + lr.setInitialModel(new org.apache.spark.ml.classification.LogisticRegressionModel(uid, + new DenseMatrix(1, initialWeights.size, initialWeights.toArray), + Vectors.dense(1.0).asML, 2, false)) + } + lr.setFitIntercept(addIntercept) + lr.setMaxIter(optimizer.getNumIterations()) + lr.setTol(optimizer.getConvergenceTol()) + // Convert our input into a DataFrame + val spark = SparkSession.builder().sparkContext(input.context).getOrCreate() + val df = spark.createDataFrame(input.map(_.asML)) + // Determine if we should cache the DF + val handlePersistence = input.getStorageLevel == StorageLevel.NONE + // Train our model + val mlLogisticRegressionModel = lr.train(df, handlePersistence) + // convert the model + val weights = Vectors.dense(mlLogisticRegressionModel.coefficients.toArray) + createModel(weights, mlLogisticRegressionModel.intercept) + } + optimizer.getUpdater() match { + case x: SquaredL2Updater => runWithMlLogisticRegression(0.0) + case x: L1Updater => runWithMlLogisticRegression(1.0) + case _ => super.run(input, initialWeights) + } + } else { + super.run(input, initialWeights) + } + } } 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 a956084ae06e..9e8774732efe 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 @@ -24,13 +24,15 @@ import scala.collection.JavaConverters._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkContext, SparkException} +import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.Since -import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix, DenseVector, SparseVector, Vector} +import org.apache.spark.internal.Logging +import org.apache.spark.ml.classification.{NaiveBayes => NewNaiveBayes} +import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix, DenseVector, 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} +import org.apache.spark.sql.SparkSession /** * Model for Naive Bayes Classifiers. @@ -74,7 +76,7 @@ class NaiveBayesModel private[spark] ( case Multinomial => (None, None) case Bernoulli => val negTheta = thetaMatrix.map(value => math.log(1.0 - math.exp(value))) - val ones = new DenseVector(Array.fill(thetaMatrix.numCols){1.0}) + val ones = new DenseVector(Array.fill(thetaMatrix.numCols) {1.0}) val thetaMinusNegTheta = thetaMatrix.map { value => value - math.log(1.0 - math.exp(value)) } @@ -192,8 +194,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { modelType: String) def save(sc: SparkContext, path: String, data: Data): Unit = { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // Create JSON metadata. val metadata = compact(render( @@ -202,15 +203,14 @@ 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() - dataRDD.write.parquet(dataPath(path)) + spark.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath(path)) } @Since("1.3.0") def load(sc: SparkContext, path: String): NaiveBayesModel = { - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // Load Parquet data. - val dataRDD = sqlContext.read.parquet(dataPath(path)) + val dataRDD = spark.read.parquet(dataPath(path)) // Check schema explicitly since erasure makes it hard to use match-case for checking. checkSchema[Data](dataRDD.schema) val dataArray = dataRDD.select("labels", "pi", "theta", "modelType").take(1) @@ -239,8 +239,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { theta: Array[Array[Double]]) def save(sc: SparkContext, path: String, data: Data): Unit = { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // Create JSON metadata. val metadata = compact(render( @@ -249,14 +248,13 @@ 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() - dataRDD.write.parquet(dataPath(path)) + spark.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath(path)) } def load(sc: SparkContext, path: String): NaiveBayesModel = { - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // Load Parquet data. - val dataRDD = sqlContext.read.parquet(dataPath(path)) + val dataRDD = spark.read.parquet(dataPath(path)) // Check schema explicitly since erasure makes it hard to use match-case for checking. checkSchema[Data](dataRDD.schema) val dataArray = dataRDD.select("labels", "pi", "theta").take(1) @@ -304,18 +302,17 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { /** * Trains a Naive Bayes model given an RDD of `(label, features)` pairs. * - * This is the Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all kinds of - * discrete data. For example, by converting documents into TF-IDF vectors, it can be used for - * document classification. By making every vector a 0-1 vector, it can also be used as - * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). The input feature values must be nonnegative. + * This is the Multinomial NB (see here) which can + * handle all kinds of discrete data. For example, by converting documents into TF-IDF + * vectors, it can be used for document classification. By making every vector a 0-1 vector, + * it can also be used as Bernoulli NB (see here). + * The input feature values must be nonnegative. */ @Since("0.9.0") class NaiveBayes private ( private var lambda: Double, private var modelType: String) extends Serializable with Logging { - import NaiveBayes.{Bernoulli, Multinomial} - @Since("1.4.0") def this(lambda: Double) = this(lambda, NaiveBayes.Multinomial) @@ -325,6 +322,8 @@ class NaiveBayes private ( /** Set the smoothing parameter. Default: 1.0. */ @Since("0.9.0") def setLambda(lambda: Double): NaiveBayes = { + require(lambda >= 0, + s"Smoothing parameter must be nonnegative but got $lambda") this.lambda = lambda this } @@ -356,82 +355,33 @@ class NaiveBayes private ( */ @Since("0.9.0") def run(data: RDD[LabeledPoint]): NaiveBayesModel = { - val requireNonnegativeValues: Vector => Unit = (v: Vector) => { - val values = v match { - case sv: SparseVector => sv.values - case dv: DenseVector => dv.values - } - if (!values.forall(_ >= 0.0)) { - throw new SparkException(s"Naive Bayes requires nonnegative feature values but found $v.") - } - } + val spark = SparkSession + .builder() + .sparkContext(data.context) + .getOrCreate() - val requireZeroOneBernoulliValues: Vector => Unit = (v: Vector) => { - val values = v match { - case sv: SparseVector => sv.values - case dv: DenseVector => dv.values - } - if (!values.forall(v => v == 0.0 || v == 1.0)) { - throw new SparkException( - s"Bernoulli naive Bayes requires 0 or 1 feature values but found $v.") - } - } + import spark.implicits._ - // Aggregates term frequencies per label. - // TODO: Calling combineByKey and collect creates two stages, we can implement something - // TODO: similar to reduceByKeyLocally to save one stage. - val aggregated = data.map(p => (p.label, p.features)).combineByKey[(Long, DenseVector)]( - createCombiner = (v: Vector) => { - if (modelType == Bernoulli) { - requireZeroOneBernoulliValues(v) - } else { - requireNonnegativeValues(v) - } - (1L, v.copy.toDense) - }, - mergeValue = (c: (Long, DenseVector), v: Vector) => { - requireNonnegativeValues(v) - BLAS.axpy(1.0, v, c._2) - (c._1 + 1L, c._2) - }, - mergeCombiners = (c1: (Long, DenseVector), c2: (Long, DenseVector)) => { - BLAS.axpy(1.0, c2._2, c1._2) - (c1._1 + c2._1, c1._2) - } - ).collect().sortBy(_._1) + val nb = new NewNaiveBayes() + .setModelType(modelType) + .setSmoothing(lambda) - val numLabels = aggregated.length - var numDocuments = 0L - aggregated.foreach { case (_, (n, _)) => - numDocuments += n - } - val numFeatures = aggregated.head match { case (_, (_, v)) => v.size } - - val labels = new Array[Double](numLabels) - val pi = new Array[Double](numLabels) - val theta = Array.fill(numLabels)(new Array[Double](numFeatures)) - - val piLogDenom = math.log(numDocuments + numLabels * lambda) - var i = 0 - aggregated.foreach { case (label, (n, sumTermFreqs)) => - labels(i) = label - pi(i) = math.log(n + lambda) - piLogDenom - val thetaLogDenom = modelType match { - case Multinomial => math.log(sumTermFreqs.values.sum + numFeatures * lambda) - case Bernoulli => math.log(n + 2.0 * lambda) - case _ => - // This should never happen. - throw new UnknownError(s"Invalid modelType: $modelType.") - } - var j = 0 - while (j < numFeatures) { - theta(i)(j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom - j += 1 - } - i += 1 + val dataset = data.map { case LabeledPoint(label, features) => (label, features.asML) } + .toDF("label", "features") + + // mllib NaiveBayes allows input labels like {-1, +1}, so set `positiveLabel` as false. + val newModel = nb.trainWithLabelCheck(dataset, positiveLabel = false) + + val pi = newModel.pi.toArray + val theta = Array.fill[Double](newModel.numClasses, newModel.numFeatures)(0.0) + newModel.theta.foreachActive { + case (i, j, v) => + theta(i)(j) = v } - new NaiveBayesModel(labels, pi, theta, modelType) + assert(newModel.oldLabels != null, + "The underlying ML NaiveBayes training does not produce labels.") + new NaiveBayesModel(newModel.oldLabels, pi, theta, modelType) } } @@ -442,20 +392,20 @@ class NaiveBayes private ( object NaiveBayes { /** String name for multinomial model type. */ - private[spark] val Multinomial: String = "multinomial" + private[classification] val Multinomial: String = "multinomial" /** String name for Bernoulli model type. */ - private[spark] val Bernoulli: String = "bernoulli" + private[classification] val Bernoulli: String = "bernoulli" /* Set of modelTypes that NaiveBayes supports */ - private[spark] val supportedModelTypes = Set(Multinomial, Bernoulli) + private[classification] val supportedModelTypes = Set(Multinomial, Bernoulli) /** * Trains a Naive Bayes model given an RDD of `(label, features)` pairs. * - * This is the default Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all - * kinds of discrete data. For example, by converting documents into TF-IDF vectors, it - * can be used for document classification. + * This is the default Multinomial NB (see here) + * which can handle all kinds of discrete data. For example, by converting documents into + * TF-IDF vectors, it can be used for document classification. * * This version of the method uses a default smoothing parameter of 1.0. * @@ -470,9 +420,9 @@ object NaiveBayes { /** * Trains a Naive Bayes model given an RDD of `(label, features)` pairs. * - * This is the default Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all - * kinds of discrete data. For example, by converting documents into TF-IDF vectors, it - * can be used for document classification. + * This is the default Multinomial NB (see here) + * which can handle all kinds of discrete data. For example, by converting documents + * into TF-IDF vectors, it can be used for document classification. * * @param input RDD of `(label, array of features)` pairs. Every vector should be a frequency * vector or a count vector. @@ -486,9 +436,10 @@ object NaiveBayes { /** * Trains a Naive Bayes model given an RDD of `(label, features)` pairs. * - * The model type can be set to either Multinomial NB ([[http://tinyurl.com/lsdw6p]]) - * or Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). The Multinomial NB can handle - * discrete count data and can be called by setting the model type to "multinomial". + * The model type can be set to either Multinomial NB (see + * here) or Bernoulli NB (see here). + * The Multinomial NB can handle discrete count data and can be called by setting the model + * type to "multinomial". * For example, it can be used with word counts or TF_IDF vectors of documents. * The Bernoulli model fits presence or absence (0-1) counts. By making every vector a * 0-1 vector and setting the model type to "bernoulli", the fits and predicts as 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 a8d3fd4177a2..5fb04ed0ee9a 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 @@ -44,7 +44,7 @@ class SVMModel @Since("1.1.0") ( /** * Sets the threshold that separates positive predictions from negative predictions. An example - * with prediction score greater than or equal to this threshold is identified as an positive, + * with prediction score greater than or equal to this threshold is identified as a positive, * and negative otherwise. The default value is 0.0. */ @Since("1.0.0") @@ -72,7 +72,7 @@ class SVMModel @Since("1.1.0") ( dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { - val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept + val margin = weightMatrix.asBreeze.dot(dataMatrix.asBreeze) + intercept threshold match { case Some(t) => if (margin > t) 1.0 else 0.0 case None => margin @@ -124,8 +124,9 @@ object SVMModel extends Loader[SVMModel] { /** * Train a Support Vector Machine (SVM) using Stochastic Gradient Descent. By default L2 - * regularization is used, which can be changed via [[SVMWithSGD.optimizer]]. - * NOTE: Labels used in SVM should be {0, 1}. + * regularization is used, which can be changed via `SVMWithSGD.optimizer`. + * + * @note Labels used in SVM should be {0, 1}. */ @Since("0.8.0") class SVMWithSGD private ( @@ -158,7 +159,9 @@ class SVMWithSGD private ( } /** - * Top-level methods for calling SVM. NOTE: Labels used in SVM should be {0, 1}. + * Top-level methods for calling SVM. + * + * @note Labels used in SVM should be {0, 1}. */ @Since("0.8.0") object SVMWithSGD { @@ -169,8 +172,6 @@ object SVMWithSGD { * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in * gradient descent are initialized using the initial weights provided. * - * NOTE: Labels used in SVM should be {0, 1}. - * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. * @param stepSize Step size to be used for each iteration of gradient descent. @@ -178,6 +179,8 @@ object SVMWithSGD { * @param miniBatchFraction Fraction of data to be used per iteration. * @param initialWeights Initial set of weights to be used. Array should be equal in size to * the number of features in the data. + * + * @note Labels used in SVM should be {0, 1}. */ @Since("0.8.0") def train( @@ -195,7 +198,8 @@ object SVMWithSGD { * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using the specified step size. Each iteration uses * `miniBatchFraction` fraction of the data to calculate the gradient. - * NOTE: Labels used in SVM should be {0, 1} + * + * @note Labels used in SVM should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. @@ -217,13 +221,14 @@ object SVMWithSGD { * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using the specified step size. We use the entire data set to * update the gradient in each iteration. - * NOTE: Labels used in SVM should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. * @param regParam Regularization parameter. * @param numIterations Number of iterations of gradient descent to run. * @return a SVMModel which has the weights and offset from training. + * + * @note Labels used in SVM should be {0, 1} */ @Since("0.8.0") def train( @@ -238,11 +243,12 @@ object SVMWithSGD { * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using a step size of 1.0. We use the entire data set to * update the gradient in each iteration. - * NOTE: Labels used in SVM should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. * @return a SVMModel which has the weights and offset from training. + * + * @note Labels used in SVM should be {0, 1} */ @Since("0.8.0") def train(input: RDD[LabeledPoint], numIterations: Int): SVMModel = { 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 fe09f6b75d28..84491181d077 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 @@ -23,7 +23,7 @@ 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.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} /** * Helper class for import/export of GLM classification models. @@ -51,8 +51,7 @@ private[classification] object GLMClassificationModel { weights: Vector, intercept: Double, threshold: Option[Double]): Unit = { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // Create JSON metadata. val metadata = compact(render( @@ -62,7 +61,7 @@ private[classification] object GLMClassificationModel { // Create Parquet data. val data = Data(weights, intercept, threshold) - sc.parallelize(Seq(data), 1).toDF().write.parquet(Loader.dataPath(path)) + spark.createDataFrame(Seq(data)).repartition(1).write.parquet(Loader.dataPath(path)) } /** @@ -73,13 +72,13 @@ private[classification] object GLMClassificationModel { * @param modelClass String name for model class (used for error messages) */ def loadData(sc: SparkContext, path: String, modelClass: String): Data = { - val datapath = Loader.dataPath(path) - val sqlContext = new SQLContext(sc) - val dataRDD = sqlContext.read.parquet(datapath) + val dataPath = Loader.dataPath(path) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val dataRDD = spark.read.parquet(dataPath) val dataArray = dataRDD.select("weights", "intercept", "threshold").take(1) - assert(dataArray.size == 1, s"Unable to load $modelClass data from: $datapath") + assert(dataArray.length == 1, s"Unable to load $modelClass data from: $dataPath") val data = dataArray(0) - assert(data.size == 3, s"Unable to load $modelClass data from: $datapath") + assert(data.size == 3, s"Unable to load $modelClass data from: $dataPath") val (weights, intercept) = data match { case Row(weights: Vector, intercept: Double, _) => (weights, intercept) @@ -92,5 +91,4 @@ private[classification] object GLMClassificationModel { Data(weights, intercept, threshold) } } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala new file mode 100644 index 000000000000..ae98e24a7568 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala @@ -0,0 +1,501 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.clustering + +import java.util.Random + +import scala.annotation.tailrec +import scala.collection.mutable + +import org.apache.spark.annotation.Since +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging +import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + +/** + * A bisecting k-means algorithm based on the paper "A comparison of document clustering techniques" + * by Steinbach, Karypis, and Kumar, with modification to fit Spark. + * The algorithm starts from a single cluster that contains all points. + * Iteratively it finds divisible clusters on the bottom level and bisects each of them using + * k-means, until there are `k` leaf clusters in total or no leaf clusters are divisible. + * The bisecting steps of clusters on the same level are grouped together to increase parallelism. + * If bisecting all divisible clusters on the bottom level would result more than `k` leaf clusters, + * larger clusters get higher priority. + * + * @param k the desired number of leaf clusters (default: 4). The actual number could be smaller if + * there are no divisible leaf clusters. + * @param maxIterations the max number of k-means iterations to split clusters (default: 20) + * @param minDivisibleClusterSize the minimum number of points (if greater than or equal 1.0) or + * the minimum proportion of points (if less than 1.0) of a divisible + * cluster (default: 1) + * @param seed a random seed (default: hash value of the class name) + * + * @see + * Steinbach, Karypis, and Kumar, A comparison of document clustering techniques, + * KDD Workshop on Text Mining, 2000. + */ +@Since("1.6.0") +class BisectingKMeans private ( + private var k: Int, + private var maxIterations: Int, + private var minDivisibleClusterSize: Double, + private var seed: Long) extends Logging { + + import BisectingKMeans._ + + /** + * Constructs with the default configuration + */ + @Since("1.6.0") + def this() = this(4, 20, 1.0, classOf[BisectingKMeans].getName.##) + + /** + * Sets the desired number of leaf clusters (default: 4). + * The actual number could be smaller if there are no divisible leaf clusters. + */ + @Since("1.6.0") + def setK(k: Int): this.type = { + require(k > 0, s"k must be positive but got $k.") + this.k = k + this + } + + /** + * Gets the desired number of leaf clusters. + */ + @Since("1.6.0") + def getK: Int = this.k + + /** + * Sets the max number of k-means iterations to split clusters (default: 20). + */ + @Since("1.6.0") + def setMaxIterations(maxIterations: Int): this.type = { + require(maxIterations > 0, s"maxIterations must be positive but got $maxIterations.") + this.maxIterations = maxIterations + this + } + + /** + * Gets the max number of k-means iterations to split clusters. + */ + @Since("1.6.0") + def getMaxIterations: Int = this.maxIterations + + /** + * Sets the minimum number of points (if greater than or equal to `1.0`) or the minimum proportion + * of points (if less than `1.0`) of a divisible cluster (default: 1). + */ + @Since("1.6.0") + def setMinDivisibleClusterSize(minDivisibleClusterSize: Double): this.type = { + require(minDivisibleClusterSize > 0.0, + s"minDivisibleClusterSize must be positive but got $minDivisibleClusterSize.") + this.minDivisibleClusterSize = minDivisibleClusterSize + this + } + + /** + * Gets the minimum number of points (if greater than or equal to `1.0`) or the minimum proportion + * of points (if less than `1.0`) of a divisible cluster. + */ + @Since("1.6.0") + def getMinDivisibleClusterSize: Double = minDivisibleClusterSize + + /** + * Sets the random seed (default: hash value of the class name). + */ + @Since("1.6.0") + def setSeed(seed: Long): this.type = { + this.seed = seed + this + } + + /** + * Gets the random seed. + */ + @Since("1.6.0") + def getSeed: Long = this.seed + + /** + * Runs the bisecting k-means algorithm. + * @param input RDD of vectors + * @return model for the bisecting kmeans + */ + @Since("1.6.0") + def run(input: RDD[Vector]): BisectingKMeansModel = { + if (input.getStorageLevel == StorageLevel.NONE) { + logWarning(s"The input RDD ${input.id} is not directly cached, which may hurt performance if" + + " its parent RDDs are also not cached.") + } + val d = input.map(_.size).first() + logInfo(s"Feature dimension: $d.") + // Compute and cache vector norms for fast distance computation. + val norms = input.map(v => Vectors.norm(v, 2.0)).persist(StorageLevel.MEMORY_AND_DISK) + val vectors = input.zip(norms).map { case (x, norm) => new VectorWithNorm(x, norm) } + var assignments = vectors.map(v => (ROOT_INDEX, v)) + var activeClusters = summarize(d, assignments) + val rootSummary = activeClusters(ROOT_INDEX) + val n = rootSummary.size + logInfo(s"Number of points: $n.") + logInfo(s"Initial cost: ${rootSummary.cost}.") + val minSize = if (minDivisibleClusterSize >= 1.0) { + math.ceil(minDivisibleClusterSize).toLong + } else { + math.ceil(minDivisibleClusterSize * n).toLong + } + logInfo(s"The minimum number of points of a divisible cluster is $minSize.") + var inactiveClusters = mutable.Seq.empty[(Long, ClusterSummary)] + val random = new Random(seed) + var numLeafClustersNeeded = k - 1 + var level = 1 + var preIndices: RDD[Long] = null + var indices: RDD[Long] = null + while (activeClusters.nonEmpty && numLeafClustersNeeded > 0 && level < LEVEL_LIMIT) { + // Divisible clusters are sufficiently large and have non-trivial cost. + var divisibleClusters = activeClusters.filter { case (_, summary) => + (summary.size >= minSize) && (summary.cost > MLUtils.EPSILON * summary.size) + } + // If we don't need all divisible clusters, take the larger ones. + if (divisibleClusters.size > numLeafClustersNeeded) { + divisibleClusters = divisibleClusters.toSeq.sortBy { case (_, summary) => + -summary.size + }.take(numLeafClustersNeeded) + .toMap + } + if (divisibleClusters.nonEmpty) { + val divisibleIndices = divisibleClusters.keys.toSet + logInfo(s"Dividing ${divisibleIndices.size} clusters on level $level.") + var newClusterCenters = divisibleClusters.flatMap { case (index, summary) => + val (left, right) = splitCenter(summary.center, random) + Iterator((leftChildIndex(index), left), (rightChildIndex(index), right)) + }.map(identity) // workaround for a Scala bug (SI-7005) that produces a not serializable map + var newClusters: Map[Long, ClusterSummary] = null + var newAssignments: RDD[(Long, VectorWithNorm)] = null + for (iter <- 0 until maxIterations) { + newAssignments = updateAssignments(assignments, divisibleIndices, newClusterCenters) + .filter { case (index, _) => + divisibleIndices.contains(parentIndex(index)) + } + newClusters = summarize(d, newAssignments) + newClusterCenters = newClusters.mapValues(_.center).map(identity) + } + if (preIndices != null) preIndices.unpersist() + preIndices = indices + indices = updateAssignments(assignments, divisibleIndices, newClusterCenters).keys + .persist(StorageLevel.MEMORY_AND_DISK) + assignments = indices.zip(vectors) + inactiveClusters ++= activeClusters + activeClusters = newClusters + numLeafClustersNeeded -= divisibleClusters.size + } else { + logInfo(s"None active and divisible clusters left on level $level. Stop iterations.") + inactiveClusters ++= activeClusters + activeClusters = Map.empty + } + level += 1 + } + if(indices != null) indices.unpersist() + val clusters = activeClusters ++ inactiveClusters + val root = buildTree(clusters) + new BisectingKMeansModel(root) + } + + /** + * Java-friendly version of `run()`. + */ + def run(data: JavaRDD[Vector]): BisectingKMeansModel = run(data.rdd) +} + +private object BisectingKMeans extends Serializable { + + /** The index of the root node of a tree. */ + private val ROOT_INDEX: Long = 1 + + private val MAX_DIVISIBLE_CLUSTER_INDEX: Long = Long.MaxValue / 2 + + private val LEVEL_LIMIT = math.log10(Long.MaxValue) / math.log10(2) + + /** Returns the left child index of the given node index. */ + private def leftChildIndex(index: Long): Long = { + require(index <= MAX_DIVISIBLE_CLUSTER_INDEX, s"Child index out of bound: 2 * $index.") + 2 * index + } + + /** Returns the right child index of the given node index. */ + private def rightChildIndex(index: Long): Long = { + require(index <= MAX_DIVISIBLE_CLUSTER_INDEX, s"Child index out of bound: 2 * $index + 1.") + 2 * index + 1 + } + + /** Returns the parent index of the given node index, or 0 if the input is 1 (root). */ + private def parentIndex(index: Long): Long = { + index / 2 + } + + /** + * Summarizes data by each cluster as Map. + * @param d feature dimension + * @param assignments pairs of point and its cluster index + * @return a map from cluster indices to corresponding cluster summaries + */ + private def summarize( + d: Int, + assignments: RDD[(Long, VectorWithNorm)]): Map[Long, ClusterSummary] = { + assignments.aggregateByKey(new ClusterSummaryAggregator(d))( + seqOp = (agg, v) => agg.add(v), + combOp = (agg1, agg2) => agg1.merge(agg2) + ).mapValues(_.summary) + .collect().toMap + } + + /** + * Cluster summary aggregator. + * @param d feature dimension + */ + private class ClusterSummaryAggregator(val d: Int) extends Serializable { + private var n: Long = 0L + private val sum: Vector = Vectors.zeros(d) + private var sumSq: Double = 0.0 + + /** Adds a point. */ + def add(v: VectorWithNorm): this.type = { + n += 1L + // TODO: use a numerically stable approach to estimate cost + sumSq += v.norm * v.norm + BLAS.axpy(1.0, v.vector, sum) + this + } + + /** Merges another aggregator. */ + def merge(other: ClusterSummaryAggregator): this.type = { + n += other.n + sumSq += other.sumSq + BLAS.axpy(1.0, other.sum, sum) + this + } + + /** Returns the summary. */ + def summary: ClusterSummary = { + val mean = sum.copy + if (n > 0L) { + BLAS.scal(1.0 / n, mean) + } + val center = new VectorWithNorm(mean) + val cost = math.max(sumSq - n * center.norm * center.norm, 0.0) + new ClusterSummary(n, center, cost) + } + } + + /** + * Bisects a cluster center. + * + * @param center current cluster center + * @param random a random number generator + * @return initial centers + */ + private def splitCenter( + center: VectorWithNorm, + random: Random): (VectorWithNorm, VectorWithNorm) = { + val d = center.vector.size + val norm = center.norm + val level = 1e-4 * norm + val noise = Vectors.dense(Array.fill(d)(random.nextDouble())) + val left = center.vector.copy + BLAS.axpy(-level, noise, left) + val right = center.vector.copy + BLAS.axpy(level, noise, right) + (new VectorWithNorm(left), new VectorWithNorm(right)) + } + + /** + * Updates assignments. + * @param assignments current assignments + * @param divisibleIndices divisible cluster indices + * @param newClusterCenters new cluster centers + * @return new assignments + */ + private def updateAssignments( + assignments: RDD[(Long, VectorWithNorm)], + divisibleIndices: Set[Long], + newClusterCenters: Map[Long, VectorWithNorm]): RDD[(Long, VectorWithNorm)] = { + assignments.map { case (index, v) => + if (divisibleIndices.contains(index)) { + val children = Seq(leftChildIndex(index), rightChildIndex(index)) + val newClusterChildren = children.filter(newClusterCenters.contains(_)) + if (newClusterChildren.nonEmpty) { + val selected = newClusterChildren.minBy { child => + KMeans.fastSquaredDistance(newClusterCenters(child), v) + } + (selected, v) + } else { + (index, v) + } + } else { + (index, v) + } + } + } + + /** + * Builds a clustering tree by re-indexing internal and leaf clusters. + * @param clusters a map from cluster indices to corresponding cluster summaries + * @return the root node of the clustering tree + */ + private def buildTree(clusters: Map[Long, ClusterSummary]): ClusteringTreeNode = { + var leafIndex = 0 + var internalIndex = -1 + + /** + * Builds a subtree from this given node index. + */ + def buildSubTree(rawIndex: Long): ClusteringTreeNode = { + val cluster = clusters(rawIndex) + val size = cluster.size + val center = cluster.center + val cost = cluster.cost + val isInternal = clusters.contains(leftChildIndex(rawIndex)) + if (isInternal) { + val index = internalIndex + internalIndex -= 1 + val leftIndex = leftChildIndex(rawIndex) + val rightIndex = rightChildIndex(rawIndex) + val indexes = Seq(leftIndex, rightIndex).filter(clusters.contains(_)) + val height = math.sqrt(indexes.map { childIndex => + KMeans.fastSquaredDistance(center, clusters(childIndex).center) + }.max) + val children = indexes.map(buildSubTree(_)).toArray + new ClusteringTreeNode(index, size, center, cost, height, children) + } else { + val index = leafIndex + leafIndex += 1 + val height = 0.0 + new ClusteringTreeNode(index, size, center, cost, height, Array.empty) + } + } + + buildSubTree(ROOT_INDEX) + } + + /** + * Summary of a cluster. + * + * @param size the number of points within this cluster + * @param center the center of the points within this cluster + * @param cost the sum of squared distances to the center + */ + private case class ClusterSummary(size: Long, center: VectorWithNorm, cost: Double) +} + +/** + * Represents a node in a clustering tree. + * + * @param index node index, negative for internal nodes and non-negative for leaf nodes + * @param size size of the cluster + * @param centerWithNorm cluster center with norm + * @param cost cost of the cluster, i.e., the sum of squared distances to the center + * @param height height of the node in the dendrogram. Currently this is defined as the max distance + * from the center to the centers of the children's, but subject to change. + * @param children children nodes + */ +@Since("1.6.0") +private[clustering] class ClusteringTreeNode private[clustering] ( + val index: Int, + val size: Long, + private[clustering] val centerWithNorm: VectorWithNorm, + val cost: Double, + val height: Double, + val children: Array[ClusteringTreeNode]) extends Serializable { + + /** Whether this is a leaf node. */ + val isLeaf: Boolean = children.isEmpty + + require((isLeaf && index >= 0) || (!isLeaf && index < 0)) + + /** Cluster center. */ + def center: Vector = centerWithNorm.vector + + /** Predicts the leaf cluster node index that the input point belongs to. */ + def predict(point: Vector): Int = { + val (index, _) = predict(new VectorWithNorm(point)) + index + } + + /** Returns the full prediction path from root to leaf. */ + def predictPath(point: Vector): Array[ClusteringTreeNode] = { + predictPath(new VectorWithNorm(point)).toArray + } + + /** Returns the full prediction path from root to leaf. */ + private def predictPath(pointWithNorm: VectorWithNorm): List[ClusteringTreeNode] = { + if (isLeaf) { + this :: Nil + } else { + val selected = children.minBy { child => + KMeans.fastSquaredDistance(child.centerWithNorm, pointWithNorm) + } + selected :: selected.predictPath(pointWithNorm) + } + } + + /** + * Computes the cost (squared distance to the predicted leaf cluster center) of the input point. + */ + def computeCost(point: Vector): Double = { + val (_, cost) = predict(new VectorWithNorm(point)) + cost + } + + /** + * Predicts the cluster index and the cost of the input point. + */ + private def predict(pointWithNorm: VectorWithNorm): (Int, Double) = { + predict(pointWithNorm, KMeans.fastSquaredDistance(centerWithNorm, pointWithNorm)) + } + + /** + * Predicts the cluster index and the cost of the input point. + * @param pointWithNorm input point + * @param cost the cost to the current center + * @return (predicted leaf cluster index, cost) + */ + @tailrec + private def predict(pointWithNorm: VectorWithNorm, cost: Double): (Int, Double) = { + if (isLeaf) { + (index, cost) + } else { + val (selectedChild, minCost) = children.map { child => + (child, KMeans.fastSquaredDistance(child.centerWithNorm, pointWithNorm)) + }.minBy(_._2) + selectedChild.predict(pointWithNorm, minCost) + } + } + + /** + * Returns all leaf nodes from this node. + */ + def leafNodes: Array[ClusteringTreeNode] = { + if (isLeaf) { + Array(this) + } else { + children.flatMap(_.leafNodes) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala new file mode 100644 index 000000000000..633bda6aac80 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.clustering + +import org.json4s._ +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Since +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.util.{Loader, Saveable} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Row, SparkSession} + +/** + * Clustering model produced by [[BisectingKMeans]]. + * The prediction is done level-by-level from the root node to a leaf node, and at each node among + * its children the closest to the input point is selected. + * + * @param root the root node of the clustering tree + */ +@Since("1.6.0") +class BisectingKMeansModel private[clustering] ( + private[clustering] val root: ClusteringTreeNode + ) extends Serializable with Saveable with Logging { + + /** + * Leaf cluster centers. + */ + @Since("1.6.0") + def clusterCenters: Array[Vector] = root.leafNodes.map(_.center) + + /** + * Number of leaf clusters. + */ + lazy val k: Int = clusterCenters.length + + /** + * Predicts the index of the cluster that the input point belongs to. + */ + @Since("1.6.0") + def predict(point: Vector): Int = { + root.predict(point) + } + + /** + * Predicts the indices of the clusters that the input points belong to. + */ + @Since("1.6.0") + def predict(points: RDD[Vector]): RDD[Int] = { + points.map { p => root.predict(p) } + } + + /** + * Java-friendly version of `predict()`. + */ + @Since("1.6.0") + def predict(points: JavaRDD[Vector]): JavaRDD[java.lang.Integer] = + predict(points.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Integer]] + + /** + * Computes the squared distance between the input point and the cluster center it belongs to. + */ + @Since("1.6.0") + def computeCost(point: Vector): Double = { + root.computeCost(point) + } + + /** + * Computes the sum of squared distances between the input points and their corresponding cluster + * centers. + */ + @Since("1.6.0") + def computeCost(data: RDD[Vector]): Double = { + data.map(root.computeCost).sum() + } + + /** + * Java-friendly version of `computeCost()`. + */ + @Since("1.6.0") + def computeCost(data: JavaRDD[Vector]): Double = this.computeCost(data.rdd) + + @Since("2.0.0") + override def save(sc: SparkContext, path: String): Unit = { + BisectingKMeansModel.SaveLoadV1_0.save(sc, this, path) + } + + override protected def formatVersion: String = "1.0" +} + +@Since("2.0.0") +object BisectingKMeansModel extends Loader[BisectingKMeansModel] { + + @Since("2.0.0") + override def load(sc: SparkContext, path: String): BisectingKMeansModel = { + val (loadedClassName, formatVersion, metadata) = Loader.loadMetadata(sc, path) + implicit val formats = DefaultFormats + val rootId = (metadata \ "rootId").extract[Int] + val classNameV1_0 = SaveLoadV1_0.thisClassName + (loadedClassName, formatVersion) match { + case (classNameV1_0, "1.0") => + val model = SaveLoadV1_0.load(sc, path, rootId) + model + case _ => throw new Exception( + s"BisectingKMeansModel.load did not recognize model with (className, format version):" + + s"($loadedClassName, $formatVersion). Supported:\n" + + s" ($classNameV1_0, 1.0)") + } + } + + private case class Data(index: Int, size: Long, center: Vector, norm: Double, cost: Double, + height: Double, children: Seq[Int]) + + private object Data { + def apply(r: Row): Data = Data(r.getInt(0), r.getLong(1), r.getAs[Vector](2), r.getDouble(3), + r.getDouble(4), r.getDouble(5), r.getSeq[Int](6)) + } + + private[clustering] object SaveLoadV1_0 { + private val thisFormatVersion = "1.0" + + private[clustering] + val thisClassName = "org.apache.spark.mllib.clustering.BisectingKMeansModel" + + def save(sc: SparkContext, model: BisectingKMeansModel, path: String): Unit = { + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) + ~ ("rootId" -> model.root.index))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) + + val data = getNodes(model.root).map(node => Data(node.index, node.size, + node.centerWithNorm.vector, node.centerWithNorm.norm, node.cost, node.height, + node.children.map(_.index))) + spark.createDataFrame(data).write.parquet(Loader.dataPath(path)) + } + + private def getNodes(node: ClusteringTreeNode): Array[ClusteringTreeNode] = { + if (node.children.isEmpty) { + Array(node) + } else { + node.children.flatMap(getNodes(_)) ++ Array(node) + } + } + + def load(sc: SparkContext, path: String, rootId: Int): BisectingKMeansModel = { + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val rows = spark.read.parquet(Loader.dataPath(path)) + Loader.checkSchema[Data](rows.schema) + val data = rows.select("index", "size", "center", "norm", "cost", "height", "children") + val nodes = data.rdd.map(Data.apply).collect().map(d => (d.index, d)).toMap + val rootNode = buildTree(rootId, nodes) + new BisectingKMeansModel(rootNode) + } + + private def buildTree(rootId: Int, nodes: Map[Int, Data]): ClusteringTreeNode = { + val root = nodes.get(rootId).get + if (root.children.isEmpty) { + new ClusteringTreeNode(root.index, root.size, new VectorWithNorm(root.center, root.norm), + root.cost, root.height, new Array[ClusteringTreeNode](0)) + } else { + val children = root.children.map(c => buildTree(c, nodes)) + new ClusteringTreeNode(root.index, root.size, new VectorWithNorm(root.center, root.norm), + root.cost, root.height, children.toArray) + } + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala index 7b203e2f4081..4d952ac88c9b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala @@ -41,14 +41,16 @@ import org.apache.spark.util.Utils * While this process is generally guaranteed to converge, it is not guaranteed * to find a global optimum. * - * Note: For high-dimensional data (with many features), this algorithm may perform poorly. - * This is due to high-dimensional data (a) making it difficult to cluster at all (based - * on statistical/theoretical arguments) and (b) numerical issues with Gaussian distributions. + * @param k Number of independent Gaussians in the mixture model. + * @param convergenceTol Maximum change in log-likelihood at which convergence + * is considered to have occurred. + * @param maxIterations Maximum number of iterations allowed. * - * @param k The number of independent Gaussians in the mixture model - * @param convergenceTol The maximum change in log-likelihood at which convergence - * is considered to have occurred. - * @param maxIterations The maximum number of iterations to perform + * @note This algorithm is limited in its number of features since it requires storing a covariance + * matrix which has size quadratic in the number of features. Even when the number of features does + * not exceed this limit, this algorithm may perform poorly on high-dimensional data. + * This is due to high-dimensional data (a) making it difficult to cluster at all (based + * on statistical/theoretical arguments) and (b) numerical issues with Gaussian distributions. */ @Since("1.3.0") class GaussianMixture private ( @@ -78,11 +80,9 @@ class GaussianMixture private ( */ @Since("1.3.0") def setInitialModel(model: GaussianMixtureModel): this.type = { - if (model.k == k) { - initialModel = Some(model) - } else { - throw new IllegalArgumentException("mismatched cluster count (model.k != k)") - } + require(model.k == k, + s"Mismatched cluster count (model.k ${model.k} != k ${k})") + initialModel = Some(model) this } @@ -97,6 +97,8 @@ class GaussianMixture private ( */ @Since("1.3.0") def setK(k: Int): this.type = { + require(k > 0, + s"Number of Gaussians must be positive but got ${k}") this.k = k this } @@ -108,16 +110,18 @@ class GaussianMixture private ( def getK: Int = k /** - * Set the maximum number of iterations to run. Default: 100 + * Set the maximum number of iterations allowed. Default: 100 */ @Since("1.3.0") def setMaxIterations(maxIterations: Int): this.type = { + require(maxIterations >= 0, + s"Maximum of iterations must be nonnegative but got ${maxIterations}") this.maxIterations = maxIterations this } /** - * Return the maximum number of iterations to run + * Return the maximum number of iterations allowed */ @Since("1.3.0") def getMaxIterations: Int = maxIterations @@ -128,6 +132,8 @@ class GaussianMixture private ( */ @Since("1.3.0") def setConvergenceTol(convergenceTol: Double): this.type = { + require(convergenceTol >= 0.0, + s"Convergence tolerance must be nonnegative but got ${convergenceTol}") this.convergenceTol = convergenceTol this } @@ -162,10 +168,13 @@ class GaussianMixture private ( val sc = data.sparkContext // we will operate on the data as breeze data - val breezeData = data.map(_.toBreeze).cache() + val breezeData = data.map(_.asBreeze).cache() // Get length of the input vectors val d = breezeData.first().length + require(d < GaussianMixture.MAX_NUM_FEATURES, s"GaussianMixture cannot handle more " + + s"than ${GaussianMixture.MAX_NUM_FEATURES} features because the size of the covariance" + + s" matrix is quadratic in the number of features.") val shouldDistributeGaussians = GaussianMixture.shouldDistributeGaussians(k, d) @@ -177,13 +186,12 @@ class GaussianMixture private ( val (weights, gaussians) = initialModel match { case Some(gmm) => (gmm.weights, gmm.gaussians) - case None => { + case None => val samples = breezeData.takeSample(withReplacement = true, k * nSamples, seed) (Array.fill(k)(1.0 / k), Array.tabulate(k) { i => val slice = samples.view(i * nSamples, (i + 1) * nSamples) new MultivariateGaussian(vectorMean(slice), initCovariance(slice)) }) - } } var llh = Double.MinValue // current log-likelihood @@ -195,7 +203,7 @@ class GaussianMixture private ( val compute = sc.broadcast(ExpectationSum.add(weights, gaussians)_) // aggregate the cluster contribution for all sample points - val sums = breezeData.aggregate(ExpectationSum.zero(k, d))(compute.value, _ += _) + val sums = breezeData.treeAggregate(ExpectationSum.zero(k, d))(compute.value, _ += _) // Create new distributions based on the partial assignments // (often referred to as the "M" step in literature) @@ -208,8 +216,8 @@ class GaussianMixture private ( val (ws, gs) = sc.parallelize(tuples, numPartitions).map { case (mean, sigma, weight) => updateWeightsAndGaussians(mean, sigma, weight, sumWeights) }.collect().unzip - Array.copy(ws.toArray, 0, weights, 0, ws.length) - Array.copy(gs.toArray, 0, gaussians, 0, gs.length) + Array.copy(ws, 0, weights, 0, ws.length) + Array.copy(gs, 0, gaussians, 0, gs.length) } else { var i = 0 while (i < k) { @@ -224,13 +232,14 @@ class GaussianMixture private ( llhp = llh // current becomes previous llh = sums.logLikelihood // this is the freshly computed log-likelihood iter += 1 + compute.destroy(blocking = false) } new GaussianMixtureModel(weights, gaussians) } /** - * Java-friendly version of [[run()]] + * Java-friendly version of `run()` */ @Since("1.3.0") def run(data: JavaRDD[Vector]): GaussianMixtureModel = run(data.rdd) @@ -262,15 +271,19 @@ class GaussianMixture private ( private def initCovariance(x: IndexedSeq[BV[Double]]): BreezeMatrix[Double] = { val mu = vectorMean(x) val ss = BDV.zeros[Double](x(0).length) - x.foreach(xi => ss += (xi - mu) :^ 2.0) + x.foreach(xi => ss += (xi - mu) ^:^ 2.0) diag(ss / x.length.toDouble) } } private[clustering] object GaussianMixture { + + /** Limit number of features such that numFeatures^2^ < Int.MaxValue */ + private[clustering] val MAX_NUM_FEATURES = math.sqrt(Int.MaxValue).toInt + /** - * Heuristic to distribute the computation of the [[MultivariateGaussian]]s, approximately when - * d > 25 except for when k is very small. + * Heuristic to distribute the computation of the `MultivariateGaussian`s, approximately when + * d is greater than 25 except for when k is very small. * @param k Number of topics * @param d Number of features */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 2115f7d99c18..afbe4f978b28 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -18,7 +18,6 @@ package org.apache.spark.mllib.clustering import breeze.linalg.{DenseVector => BreezeVector} - import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ @@ -26,11 +25,11 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD -import org.apache.spark.mllib.linalg.{Vector, Matrices, Matrix} +import org.apache.spark.mllib.linalg.{Matrix, Vector} import org.apache.spark.mllib.stat.distribution.MultivariateGaussian -import org.apache.spark.mllib.util.{MLUtils, Loader, Saveable} +import org.apache.spark.mllib.util.{Loader, MLUtils, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{SQLContext, Row} +import org.apache.spark.sql.{Row, SparkSession} /** * Multivariate Gaussian Mixture Model (GMM) consisting of k Gaussians, where points @@ -76,12 +75,12 @@ class GaussianMixtureModel @Since("1.3.0") ( */ @Since("1.5.0") def predict(point: Vector): Int = { - val r = computeSoftAssignments(point.toBreeze.toDenseVector, gaussians, weights, k) + val r = predictSoft(point) r.indexOf(r.max) } /** - * Java-friendly version of [[predict()]] + * Java-friendly version of `predict()` */ @Since("1.4.0") def predict(points: JavaRDD[Vector]): JavaRDD[java.lang.Integer] = @@ -97,7 +96,7 @@ class GaussianMixtureModel @Since("1.3.0") ( val bcDists = sc.broadcast(gaussians) val bcWeights = sc.broadcast(weights) points.map { x => - computeSoftAssignments(x.toBreeze.toDenseVector, bcDists.value, bcWeights.value, k) + computeSoftAssignments(x.asBreeze.toDenseVector, bcDists.value, bcWeights.value, k) } } @@ -106,7 +105,7 @@ class GaussianMixtureModel @Since("1.3.0") ( */ @Since("1.4.0") def predictSoft(point: Vector): Array[Double] = { - computeSoftAssignments(point.toBreeze.toDenseVector, gaussians, weights, k) + computeSoftAssignments(point.asBreeze.toDenseVector, gaussians, weights, k) } /** @@ -144,9 +143,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { path: String, weights: Array[Double], gaussians: Array[MultivariateGaussian]): Unit = { - - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // Create JSON metadata. val metadata = compact(render @@ -157,13 +154,13 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { val dataArray = Array.tabulate(weights.length) { i => Data(weights(i), gaussians(i).mu, gaussians(i).sigma) } - sc.parallelize(dataArray, 1).toDF().write.parquet(Loader.dataPath(path)) + spark.createDataFrame(dataArray).repartition(1).write.parquet(Loader.dataPath(path)) } def load(sc: SparkContext, path: String): GaussianMixtureModel = { val dataPath = Loader.dataPath(path) - val sqlContext = new SQLContext(sc) - val dataFrame = sqlContext.read.parquet(dataPath) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val dataFrame = spark.read.parquet(dataPath) // Check schema explicitly since erasure makes it hard to use match-case for checking. Loader.checkSchema[Data](dataFrame.schema) val dataArray = dataFrame.select("weight", "mu", "sigma").collect() @@ -178,22 +175,21 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { } @Since("1.4.0") - override def load(sc: SparkContext, path: String) : GaussianMixtureModel = { + override def load(sc: SparkContext, path: String): GaussianMixtureModel = { val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) implicit val formats = DefaultFormats val k = (metadata \ "k").extract[Int] val classNameV1_0 = SaveLoadV1_0.classNameV1_0 (loadedClassName, version) match { - case (classNameV1_0, "1.0") => { + case (classNameV1_0, "1.0") => val model = SaveLoadV1_0.load(sc, path) require(model.weights.length == k, s"GaussianMixtureModel requires weights of length $k " + s"got weights of length ${model.weights.length}") require(model.gaussians.length == k, - s"GaussianMixtureModel requires gaussians of length $k" + + s"GaussianMixtureModel requires gaussians of length $k " + s"got gaussians of length ${model.gaussians.length}") model - } case _ => throw new Exception( s"GaussianMixtureModel.load did not recognize model with (className, format version):" + s"($loadedClassName, $version). Supported:\n" + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 2895db7c9061..49043b5acb80 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -19,8 +19,11 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.ArrayBuffer -import org.apache.spark.Logging -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging +import org.apache.spark.ml.clustering.{KMeans => NewKMeans} +import org.apache.spark.ml.util.Instrumentation import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.{axpy, scal} import org.apache.spark.mllib.util.MLUtils @@ -30,9 +33,8 @@ import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom /** - * K-means clustering with support for multiple parallel runs and a k-means++ like initialization - * mode (the k-means|| algorithm by Bahmani et al). When multiple concurrent runs are requested, - * they are executed together with joint passes over the data for efficiency. + * K-means clustering with a k-means++ like initialization mode + * (the k-means|| algorithm by Bahmani et al). * * This is an iterative algorithm that will make multiple passes over the data, so any RDDs given * to it should be cached by the user. @@ -41,45 +43,54 @@ import org.apache.spark.util.random.XORShiftRandom class KMeans private ( private var k: Int, private var maxIterations: Int, - private var runs: Int, private var initializationMode: String, private var initializationSteps: Int, private var epsilon: Double, private var seed: Long) extends Serializable with Logging { /** - * Constructs a KMeans instance with default parameters: {k: 2, maxIterations: 20, runs: 1, - * initializationMode: "k-means||", initializationSteps: 5, epsilon: 1e-4, seed: random}. + * Constructs a KMeans instance with default parameters: {k: 2, maxIterations: 20, + * initializationMode: "k-means||", initializationSteps: 2, epsilon: 1e-4, seed: random}. */ @Since("0.8.0") - def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4, Utils.random.nextLong()) + def this() = this(2, 20, KMeans.K_MEANS_PARALLEL, 2, 1e-4, Utils.random.nextLong()) /** * Number of clusters to create (k). + * + * @note It is possible for fewer than k clusters to + * be returned, for example, if there are fewer than k distinct points to cluster. */ @Since("1.4.0") def getK: Int = k /** - * Set the number of clusters to create (k). Default: 2. + * Set the number of clusters to create (k). + * + * @note It is possible for fewer than k clusters to + * be returned, for example, if there are fewer than k distinct points to cluster. Default: 2. */ @Since("0.8.0") def setK(k: Int): this.type = { + require(k > 0, + s"Number of clusters must be positive but got ${k}") this.k = k this } /** - * Maximum number of iterations to run. + * Maximum number of iterations allowed. */ @Since("1.4.0") def getMaxIterations: Int = maxIterations /** - * Set maximum number of iterations to run. Default: 20. + * Set maximum number of iterations allowed. Default: 20. */ @Since("0.8.0") def setMaxIterations(maxIterations: Int): this.type = { + require(maxIterations >= 0, + s"Maximum of iterations must be nonnegative but got ${maxIterations}") this.maxIterations = maxIterations this } @@ -103,26 +114,22 @@ class KMeans private ( } /** - * :: Experimental :: - * Number of runs of the algorithm to execute in parallel. + * This function has no effect since Spark 2.0.0. */ @Since("1.4.0") - @deprecated("Support for runs is deprecated. This param will have no effect in 1.7.0.", "1.6.0") - def getRuns: Int = runs + @deprecated("This has no effect and always returns 1", "2.1.0") + def getRuns: Int = { + logWarning("Getting number of runs has no effect since Spark 2.0.0.") + 1 + } /** - * :: Experimental :: - * Set the number of runs of the algorithm to execute in parallel. We initialize the algorithm - * this many times with random starting conditions (configured by the initialization mode), then - * return the best clustering found over any run. Default: 1. + * This function has no effect since Spark 2.0.0. */ @Since("0.8.0") - @deprecated("Support for runs is deprecated. This param will have no effect in 1.7.0.", "1.6.0") + @deprecated("This has no effect", "2.1.0") def setRuns(runs: Int): this.type = { - if (runs <= 0) { - throw new IllegalArgumentException("Number of runs must be positive") - } - this.runs = runs + logWarning("Setting number of runs has no effect since Spark 2.0.0.") this } @@ -134,13 +141,12 @@ class KMeans private ( /** * Set the number of steps for the k-means|| initialization mode. This is an advanced - * setting -- the default of 5 is almost always enough. Default: 5. + * setting -- the default of 2 is almost always enough. Default: 2. */ @Since("0.8.0") def setInitializationSteps(initializationSteps: Int): this.type = { - if (initializationSteps <= 0) { - throw new IllegalArgumentException("Number of initialization steps must be positive") - } + require(initializationSteps > 0, + s"Number of initialization steps must be positive but got ${initializationSteps}") this.initializationSteps = initializationSteps this } @@ -157,6 +163,8 @@ class KMeans private ( */ @Since("0.8.0") def setEpsilon(epsilon: Double): this.type = { + require(epsilon >= 0, + s"Distance threshold must be nonnegative but got ${epsilon}") this.epsilon = epsilon this } @@ -198,6 +206,12 @@ class KMeans private ( */ @Since("0.8.0") def run(data: RDD[Vector]): KMeansModel = { + run(data, None) + } + + private[spark] def run( + data: RDD[Vector], + instr: Option[Instrumentation[NewKMeans]]): KMeansModel = { if (data.getStorageLevel == StorageLevel.NONE) { logWarning("The input data is not directly cached, which may hurt performance if its" @@ -210,7 +224,7 @@ class KMeans private ( val zippedData = data.zip(norms).map { case (v, norm) => new VectorWithNorm(v, norm) } - val model = runAlgorithm(zippedData) + val model = runAlgorithm(zippedData, instr) norms.unpersist() // Warn at the end of the run as well, for increased visibility. @@ -224,112 +238,82 @@ class KMeans private ( /** * Implementation of K-Means algorithm. */ - private def runAlgorithm(data: RDD[VectorWithNorm]): KMeansModel = { + private def runAlgorithm( + data: RDD[VectorWithNorm], + instr: Option[Instrumentation[NewKMeans]]): KMeansModel = { val sc = data.sparkContext val initStartTime = System.nanoTime() - // Only one run is allowed when initialModel is given - val numRuns = if (initialModel.nonEmpty) { - if (runs > 1) logWarning("Ignoring runs; one run is allowed when initialModel is given.") - 1 - } else { - runs - } - val centers = initialModel match { - case Some(kMeansCenters) => { - Array(kMeansCenters.clusterCenters.map(s => new VectorWithNorm(s))) - } - case None => { + case Some(kMeansCenters) => + kMeansCenters.clusterCenters.map(new VectorWithNorm(_)) + case None => if (initializationMode == KMeans.RANDOM) { initRandom(data) } else { initKMeansParallel(data) } - } } val initTimeInSeconds = (System.nanoTime() - initStartTime) / 1e9 - logInfo(s"Initialization with $initializationMode took " + "%.3f".format(initTimeInSeconds) + - " seconds.") - - val active = Array.fill(numRuns)(true) - val costs = Array.fill(numRuns)(0.0) + logInfo(f"Initialization with $initializationMode took $initTimeInSeconds%.3f seconds.") - var activeRuns = new ArrayBuffer[Int] ++ (0 until numRuns) + var converged = false + var cost = 0.0 var iteration = 0 val iterationStartTime = System.nanoTime() - // Execute iterations of Lloyd's algorithm until all runs have converged - while (iteration < maxIterations && !activeRuns.isEmpty) { - type WeightedPoint = (Vector, Long) - def mergeContribs(x: WeightedPoint, y: WeightedPoint): WeightedPoint = { - axpy(1.0, x._1, y._1) - (y._1, x._2 + y._2) - } - - val activeCenters = activeRuns.map(r => centers(r)).toArray - val costAccums = activeRuns.map(_ => sc.accumulator(0.0)) + instr.foreach(_.logNumFeatures(centers.head.vector.size)) - val bcActiveCenters = sc.broadcast(activeCenters) + // Execute iterations of Lloyd's algorithm until converged + while (iteration < maxIterations && !converged) { + val costAccum = sc.doubleAccumulator + val bcCenters = sc.broadcast(centers) - // Find the sum and count of points mapping to each center - val totalContribs = data.mapPartitions { points => - val thisActiveCenters = bcActiveCenters.value - val runs = thisActiveCenters.length - val k = thisActiveCenters(0).length - val dims = thisActiveCenters(0)(0).vector.size + // Find the new centers + val newCenters = data.mapPartitions { points => + val thisCenters = bcCenters.value + val dims = thisCenters.head.vector.size - val sums = Array.fill(runs, k)(Vectors.zeros(dims)) - val counts = Array.fill(runs, k)(0L) + val sums = Array.fill(thisCenters.length)(Vectors.zeros(dims)) + val counts = Array.fill(thisCenters.length)(0L) points.foreach { point => - (0 until runs).foreach { i => - val (bestCenter, cost) = KMeans.findClosest(thisActiveCenters(i), point) - costAccums(i) += cost - val sum = sums(i)(bestCenter) - axpy(1.0, point.vector, sum) - counts(i)(bestCenter) += 1 - } + val (bestCenter, cost) = KMeans.findClosest(thisCenters, point) + costAccum.add(cost) + val sum = sums(bestCenter) + axpy(1.0, point.vector, sum) + counts(bestCenter) += 1 } - val contribs = for (i <- 0 until runs; j <- 0 until k) yield { - ((i, j), (sums(i)(j), counts(i)(j))) - } - contribs.iterator - }.reduceByKey(mergeContribs).collectAsMap() - - // Update the cluster centers and costs for each active run - for ((run, i) <- activeRuns.zipWithIndex) { - var changed = false - var j = 0 - while (j < k) { - val (sum, count) = totalContribs((i, j)) - if (count != 0) { - scal(1.0 / count, sum) - val newCenter = new VectorWithNorm(sum) - if (KMeans.fastSquaredDistance(newCenter, centers(run)(j)) > epsilon * epsilon) { - changed = true - } - centers(run)(j) = newCenter - } - j += 1 - } - if (!changed) { - active(run) = false - logInfo("Run " + run + " finished in " + (iteration + 1) + " iterations") + counts.indices.filter(counts(_) > 0).map(j => (j, (sums(j), counts(j)))).iterator + }.reduceByKey { case ((sum1, count1), (sum2, count2)) => + axpy(1.0, sum2, sum1) + (sum1, count1 + count2) + }.mapValues { case (sum, count) => + scal(1.0 / count, sum) + new VectorWithNorm(sum) + }.collectAsMap() + + bcCenters.destroy(blocking = false) + + // Update the cluster centers and costs + converged = true + newCenters.foreach { case (j, newCenter) => + if (converged && KMeans.fastSquaredDistance(newCenter, centers(j)) > epsilon * epsilon) { + converged = false } - costs(run) = costAccums(i).value + centers(j) = newCenter } - activeRuns = activeRuns.filter(active(_)) + cost = costAccum.value iteration += 1 } val iterationTimeInSeconds = (System.nanoTime() - iterationStartTime) / 1e9 - logInfo(s"Iterations took " + "%.3f".format(iterationTimeInSeconds) + " seconds.") + logInfo(f"Iterations took $iterationTimeInSeconds%.3f seconds.") if (iteration == maxIterations) { logInfo(s"KMeans reached the max number of iterations: $maxIterations.") @@ -337,124 +321,89 @@ class KMeans private ( logInfo(s"KMeans converged in $iteration iterations.") } - val (minCost, bestRun) = costs.zipWithIndex.min + logInfo(s"The cost is $cost.") - logInfo(s"The cost for the best run is $minCost.") - - new KMeansModel(centers(bestRun).map(_.vector)) + new KMeansModel(centers.map(_.vector)) } /** - * Initialize `runs` sets of cluster centers at random. + * Initialize a set of cluster centers at random. */ - private def initRandom(data: RDD[VectorWithNorm]) - : Array[Array[VectorWithNorm]] = { - // Sample all the cluster centers in one pass to avoid repeated scans - val sample = data.takeSample(true, runs * k, new XORShiftRandom(this.seed).nextInt()).toSeq - Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k).map { v => - new VectorWithNorm(Vectors.dense(v.vector.toArray), v.norm) - }.toArray) + private def initRandom(data: RDD[VectorWithNorm]): Array[VectorWithNorm] = { + // Select without replacement; may still produce duplicates if the data has < k distinct + // points, so deduplicate the centroids to match the behavior of k-means|| in the same situation + data.takeSample(false, k, new XORShiftRandom(this.seed).nextInt()) + .map(_.vector).distinct.map(new VectorWithNorm(_)) } /** - * Initialize `runs` sets of cluster centers using the k-means|| algorithm by Bahmani et al. + * Initialize a set of cluster centers using the k-means|| algorithm by Bahmani et al. * (Bahmani et al., Scalable K-Means++, VLDB 2012). This is a variant of k-means++ that tries - * to find with dissimilar cluster centers by starting with a random center and then doing + * to find dissimilar cluster centers by starting with a random center and then doing * passes where more centers are chosen with probability proportional to their squared distance * to the current cluster set. It results in a provable approximation to an optimal clustering. * * The original paper can be found at http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf. */ - private def initKMeansParallel(data: RDD[VectorWithNorm]) - : Array[Array[VectorWithNorm]] = { + private[clustering] def initKMeansParallel(data: RDD[VectorWithNorm]): Array[VectorWithNorm] = { // Initialize empty centers and point costs. - val centers = Array.tabulate(runs)(r => ArrayBuffer.empty[VectorWithNorm]) - var costs = data.map(_ => Array.fill(runs)(Double.PositiveInfinity)) + var costs = data.map(_ => Double.PositiveInfinity) - // Initialize each run's first center to a random point. + // Initialize the first center to a random point. val seed = new XORShiftRandom(this.seed).nextInt() - val sample = data.takeSample(true, runs, seed).toSeq - val newCenters = Array.tabulate(runs)(r => ArrayBuffer(sample(r).toDense)) - - /** Merges new centers to centers. */ - def mergeNewCenters(): Unit = { - var r = 0 - while (r < runs) { - centers(r) ++= newCenters(r) - newCenters(r).clear() - r += 1 - } - } + val sample = data.takeSample(false, 1, seed) + // Could be empty if data is empty; fail with a better message early: + require(sample.nonEmpty, s"No samples available from $data") - // On each step, sample 2 * k points on average for each run with probability proportional - // to their squared distance from that run's centers. Note that only distances between points + val centers = ArrayBuffer[VectorWithNorm]() + var newCenters = Seq(sample.head.toDense) + centers ++= newCenters + + // On each step, sample 2 * k points on average with probability proportional + // to their squared distance from the centers. Note that only distances between points // and new centers are computed in each iteration. var step = 0 + val bcNewCentersList = ArrayBuffer[Broadcast[_]]() while (step < initializationSteps) { val bcNewCenters = data.context.broadcast(newCenters) + bcNewCentersList += bcNewCenters val preCosts = costs costs = data.zip(preCosts).map { case (point, cost) => - Array.tabulate(runs) { r => - math.min(KMeans.pointCost(bcNewCenters.value(r), point), cost(r)) - } - }.persist(StorageLevel.MEMORY_AND_DISK) - val sumCosts = costs - .aggregate(new Array[Double](runs))( - seqOp = (s, v) => { - // s += v - var r = 0 - while (r < runs) { - s(r) += v(r) - r += 1 - } - s - }, - combOp = (s0, s1) => { - // s0 += s1 - var r = 0 - while (r < runs) { - s0(r) += s1(r) - r += 1 - } - s0 - } - ) + math.min(KMeans.pointCost(bcNewCenters.value, point), cost) + }.persist(StorageLevel.MEMORY_AND_DISK) + val sumCosts = costs.sum() + + bcNewCenters.unpersist(blocking = false) preCosts.unpersist(blocking = false) - val chosen = data.zip(costs).mapPartitionsWithIndex { (index, pointsWithCosts) => + + val chosen = data.zip(costs).mapPartitionsWithIndex { (index, pointCosts) => val rand = new XORShiftRandom(seed ^ (step << 16) ^ index) - pointsWithCosts.flatMap { case (p, c) => - val rs = (0 until runs).filter { r => - rand.nextDouble() < 2.0 * c(r) * k / sumCosts(r) - } - if (rs.length > 0) Some(p, rs) else None - } + pointCosts.filter { case (_, c) => rand.nextDouble() < 2.0 * c * k / sumCosts }.map(_._1) }.collect() - mergeNewCenters() - chosen.foreach { case (p, rs) => - rs.foreach(newCenters(_) += p.toDense) - } + newCenters = chosen.map(_.toDense) + centers ++= newCenters step += 1 } - mergeNewCenters() costs.unpersist(blocking = false) + bcNewCentersList.foreach(_.destroy(false)) - // Finally, we might have a set of more than k candidate centers for each run; weigh each - // candidate by the number of points in the dataset mapping to it and run a local k-means++ - // on the weighted centers to pick just k of them - val bcCenters = data.context.broadcast(centers) - val weightMap = data.flatMap { p => - Iterator.tabulate(runs) { r => - ((r, KMeans.findClosest(bcCenters.value(r), p)._1), 1.0) - } - }.reduceByKey(_ + _).collectAsMap() - val finalCenters = (0 until runs).par.map { r => - val myCenters = centers(r).toArray - val myWeights = (0 until myCenters.length).map(i => weightMap.getOrElse((r, i), 0.0)).toArray - LocalKMeans.kMeansPlusPlus(r, myCenters, myWeights, k, 30) - } + val distinctCenters = centers.map(_.vector).distinct.map(new VectorWithNorm(_)) + + if (distinctCenters.size <= k) { + distinctCenters.toArray + } else { + // Finally, we might have a set of more than k distinct candidate centers; weight each + // candidate by the number of points in the dataset mapping to it and run a local k-means++ + // on the weighted centers to pick k of them + val bcCenters = data.context.broadcast(distinctCenters) + val countMap = data.map(KMeans.findClosest(bcCenters.value, _)._1).countByValue() + + bcCenters.destroy(blocking = false) - finalCenters.toArray + val myWeights = distinctCenters.indices.map(countMap.getOrElse(_, 0L).toDouble).toArray + LocalKMeans.kMeansPlusPlus(0, distinctCenters.toArray, myWeights, k, 30) + } } } @@ -474,14 +423,63 @@ object KMeans { /** * Trains a k-means model using the given set of parameters. * - * @param data training points stored as `RDD[Vector]` - * @param k number of clusters - * @param maxIterations max number of iterations - * @param runs number of parallel runs, defaults to 1. The best model is returned. - * @param initializationMode initialization model, either "random" or "k-means||" (default). - * @param seed random seed value for cluster initialization + * @param data Training points as an `RDD` of `Vector` types. + * @param k Number of clusters to create. + * @param maxIterations Maximum number of iterations allowed. + * @param initializationMode The initialization algorithm. This can either be "random" or + * "k-means||". (default: "k-means||") + * @param seed Random seed for cluster initialization. Default is to generate seed based + * on system time. + */ + @Since("2.1.0") + def train( + data: RDD[Vector], + k: Int, + maxIterations: Int, + initializationMode: String, + seed: Long): KMeansModel = { + new KMeans().setK(k) + .setMaxIterations(maxIterations) + .setInitializationMode(initializationMode) + .setSeed(seed) + .run(data) + } + + /** + * Trains a k-means model using the given set of parameters. + * + * @param data Training points as an `RDD` of `Vector` types. + * @param k Number of clusters to create. + * @param maxIterations Maximum number of iterations allowed. + * @param initializationMode The initialization algorithm. This can either be "random" or + * "k-means||". (default: "k-means||") + */ + @Since("2.1.0") + def train( + data: RDD[Vector], + k: Int, + maxIterations: Int, + initializationMode: String): KMeansModel = { + new KMeans().setK(k) + .setMaxIterations(maxIterations) + .setInitializationMode(initializationMode) + .run(data) + } + + /** + * Trains a k-means model using the given set of parameters. + * + * @param data Training points as an `RDD` of `Vector` types. + * @param k Number of clusters to create. + * @param maxIterations Maximum number of iterations allowed. + * @param runs This param has no effect since Spark 2.0.0. + * @param initializationMode The initialization algorithm. This can either be "random" or + * "k-means||". (default: "k-means||") + * @param seed Random seed for cluster initialization. Default is to generate seed based + * on system time. */ @Since("1.3.0") + @deprecated("Use train method without 'runs'", "2.1.0") def train( data: RDD[Vector], k: Int, @@ -491,7 +489,6 @@ object KMeans { seed: Long): KMeansModel = { new KMeans().setK(k) .setMaxIterations(maxIterations) - .setRuns(runs) .setInitializationMode(initializationMode) .setSeed(seed) .run(data) @@ -500,13 +497,15 @@ object KMeans { /** * Trains a k-means model using the given set of parameters. * - * @param data training points stored as `RDD[Vector]` - * @param k number of clusters - * @param maxIterations max number of iterations - * @param runs number of parallel runs, defaults to 1. The best model is returned. - * @param initializationMode initialization model, either "random" or "k-means||" (default). + * @param data Training points as an `RDD` of `Vector` types. + * @param k Number of clusters to create. + * @param maxIterations Maximum number of iterations allowed. + * @param runs This param has no effect since Spark 2.0.0. + * @param initializationMode The initialization algorithm. This can either be "random" or + * "k-means||". (default: "k-means||") */ @Since("0.8.0") + @deprecated("Use train method without 'runs'", "2.1.0") def train( data: RDD[Vector], k: Int, @@ -515,7 +514,6 @@ object KMeans { initializationMode: String): KMeansModel = { new KMeans().setK(k) .setMaxIterations(maxIterations) - .setRuns(runs) .setInitializationMode(initializationMode) .run(data) } @@ -528,19 +526,24 @@ object KMeans { data: RDD[Vector], k: Int, maxIterations: Int): KMeansModel = { - train(data, k, maxIterations, 1, K_MEANS_PARALLEL) + new KMeans().setK(k) + .setMaxIterations(maxIterations) + .run(data) } /** * Trains a k-means model using specified parameters and the default values for unspecified. */ @Since("0.8.0") + @deprecated("Use train method without 'runs'", "2.1.0") def train( data: RDD[Vector], k: Int, maxIterations: Int, runs: Int): KMeansModel = { - train(data, k, maxIterations, runs, K_MEANS_PARALLEL) + new KMeans().setK(k) + .setMaxIterations(maxIterations) + .run(data) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index a74158498272..3ad08c46d204 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -23,15 +23,14 @@ import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ +import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.Row +import org.apache.spark.sql.{Row, SparkSession} /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. @@ -40,6 +39,9 @@ import org.apache.spark.sql.Row class KMeansModel @Since("1.1.0") (@Since("1.0.0") val clusterCenters: Array[Vector]) extends Saveable with Serializable with PMMLExportable { + private val clusterCentersWithNorm = + if (clusterCenters == null) null else clusterCenters.map(new VectorWithNorm(_)) + /** * A Java-friendly constructor that takes an Iterable of Vectors. */ @@ -50,7 +52,7 @@ class KMeansModel @Since("1.1.0") (@Since("1.0.0") val clusterCenters: Array[Vec * Total number of clusters. */ @Since("0.8.0") - def k: Int = clusterCenters.length + def k: Int = clusterCentersWithNorm.length /** * Returns the cluster index that a given point belongs to. @@ -65,8 +67,7 @@ class KMeansModel @Since("1.1.0") (@Since("1.0.0") val clusterCenters: Array[Vec */ @Since("1.0.0") def predict(points: RDD[Vector]): RDD[Int] = { - val centersWithNorm = clusterCentersWithNorm - val bcCentersWithNorm = points.context.broadcast(centersWithNorm) + val bcCentersWithNorm = points.context.broadcast(clusterCentersWithNorm) points.map(p => KMeans.findClosest(bcCentersWithNorm.value, new VectorWithNorm(p))._1) } @@ -83,13 +84,13 @@ class KMeansModel @Since("1.1.0") (@Since("1.0.0") val clusterCenters: Array[Vec */ @Since("0.8.0") def computeCost(data: RDD[Vector]): Double = { - val centersWithNorm = clusterCentersWithNorm - val bcCentersWithNorm = data.context.broadcast(centersWithNorm) - data.map(p => KMeans.pointCost(bcCentersWithNorm.value, new VectorWithNorm(p))).sum() + val bcCentersWithNorm = data.context.broadcast(clusterCentersWithNorm) + val cost = data + .map(p => KMeans.pointCost(bcCentersWithNorm.value, new VectorWithNorm(p))).sum() + bcCentersWithNorm.destroy(blocking = false) + cost } - private def clusterCentersWithNorm: Iterable[VectorWithNorm] = - clusterCenters.map(new VectorWithNorm(_)) @Since("1.4.0") override def save(sc: SparkContext, path: String): Unit = { @@ -124,28 +125,27 @@ object KMeansModel extends Loader[KMeansModel] { val thisClassName = "org.apache.spark.mllib.clustering.KMeansModel" def save(sc: SparkContext, model: KMeansModel, path: String): Unit = { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("k" -> model.k))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) - val dataRDD = sc.parallelize(model.clusterCenters.zipWithIndex).map { case (point, id) => - Cluster(id, point) - }.toDF() - dataRDD.write.parquet(Loader.dataPath(path)) + val dataRDD = sc.parallelize(model.clusterCentersWithNorm.zipWithIndex).map { case (p, id) => + Cluster(id, p.vector) + } + spark.createDataFrame(dataRDD).write.parquet(Loader.dataPath(path)) } def load(sc: SparkContext, path: String): KMeansModel = { implicit val formats = DefaultFormats - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) assert(className == thisClassName) assert(formatVersion == thisFormatVersion) val k = (metadata \ "k").extract[Int] - val centroids = sqlContext.read.parquet(Loader.dataPath(path)) + val centroids = spark.read.parquet(Loader.dataPath(path)) Loader.checkSchema[Cluster](centroids.schema) - val localCentroids = centroids.map(Cluster.apply).collect() - assert(k == localCentroids.size) + val localCentroids = centroids.rdd.map(Cluster.apply).collect() + assert(k == localCentroids.length) new KMeansModel(localCentroids.sortBy(_.id).map(_.point)) } } 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 eb802a365ed6..4aa647236b31 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 @@ -17,12 +17,14 @@ package org.apache.spark.mllib.clustering +import java.util.Locale + import breeze.linalg.{DenseVector => BDV} -import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.java.JavaPairRDD import org.apache.spark.graphx._ +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils @@ -39,8 +41,8 @@ import org.apache.spark.util.Utils * - Original LDA paper (journal version): * Blei, Ng, and Jordan. "Latent Dirichlet Allocation." JMLR, 2003. * - * @see [[http://en.wikipedia.org/wiki/Latent_Dirichlet_allocation Latent Dirichlet allocation - * (Wikipedia)]] + * @see + * Latent Dirichlet allocation (Wikipedia) */ @Since("1.3.0") class LDA private ( @@ -61,14 +63,13 @@ class LDA private ( ldaOptimizer = new EMLDAOptimizer) /** - * Number of topics to infer. I.e., the number of soft cluster centers. - * + * Number of topics to infer, i.e., the number of soft cluster centers. */ @Since("1.3.0") def getK: Int = k /** - * Number of topics to infer. I.e., the number of soft cluster centers. + * Set the number of topics to infer, i.e., the number of soft cluster centers. * (default = 10) */ @Since("1.3.0") @@ -92,7 +93,7 @@ class LDA private ( * distributions over topics ("theta"). * * This method assumes the Dirichlet distribution is symmetric and can be described by a single - * [[Double]] parameter. It should fail if docConcentration is asymmetric. + * `Double` parameter. It should fail if docConcentration is asymmetric. */ @Since("1.3.0") def getDocConcentration: Double = { @@ -114,30 +115,31 @@ class LDA private ( * * If set to a singleton vector Vector(-1), then docConcentration is set automatically. If set to * singleton vector Vector(t) where t != -1, then t is replicated to a vector of length k during - * [[LDAOptimizer.initialize()]]. Otherwise, the [[docConcentration]] vector must be length k. + * `LDAOptimizer.initialize()`. Otherwise, the `docConcentration` vector must be length k. * (default = Vector(-1) = automatic) * * Optimizer-specific parameter settings: * - EM * - Currently only supports symmetric distributions, so all values in the vector should be * the same. - * - Values should be > 1.0 + * - Values should be greater than 1.0 * - default = uniformly (50 / k) + 1, where 50/k is common in LDA libraries and +1 follows * from Asuncion et al. (2009), who recommend a +1 adjustment for EM. * - Online - * - Values should be >= 0 + * - Values should be greater than or equal to 0 * - default = uniformly (1.0 / k), following the implementation from - * [[https://github.com/Blei-Lab/onlineldavb]]. + * here. */ @Since("1.5.0") def setDocConcentration(docConcentration: Vector): this.type = { - require(docConcentration.size > 0, "docConcentration must have > 0 elements") + require(docConcentration.size == 1 || docConcentration.size == k, + s"Size of docConcentration must be 1 or ${k} but got ${docConcentration.size}") this.docConcentration = docConcentration this } /** - * Replicates a [[Double]] docConcentration to create a symmetric prior. + * Replicates a `Double` docConcentration to create a symmetric prior. */ @Since("1.3.0") def setDocConcentration(docConcentration: Double): this.type = { @@ -158,13 +160,13 @@ class LDA private ( def getAlpha: Double = getDocConcentration /** - * Alias for [[setDocConcentration()]] + * Alias for `setDocConcentration()` */ @Since("1.5.0") def setAlpha(alpha: Vector): this.type = setDocConcentration(alpha) /** - * Alias for [[setDocConcentration()]] + * Alias for `setDocConcentration()` */ @Since("1.3.0") def setAlpha(alpha: Double): this.type = setDocConcentration(alpha) @@ -175,7 +177,7 @@ class LDA private ( * * This is the parameter to a symmetric Dirichlet distribution. * - * Note: The topics' distributions over terms are called "beta" in the original LDA paper + * @note The topics' distributions over terms are called "beta" in the original LDA paper * by Blei et al., but are called "phi" in many later papers such as Asuncion et al., 2009. */ @Since("1.3.0") @@ -187,7 +189,7 @@ class LDA private ( * * This is the parameter to a symmetric Dirichlet distribution. * - * Note: The topics' distributions over terms are called "beta" in the original LDA paper + * @note The topics' distributions over terms are called "beta" in the original LDA paper * by Blei et al., but are called "phi" in many later papers such as Asuncion et al., 2009. * * If set to -1, then topicConcentration is set automatically. @@ -195,13 +197,13 @@ class LDA private ( * * Optimizer-specific parameter settings: * - EM - * - Value should be > 1.0 + * - Value should be greater than 1.0 * - default = 0.1 + 1, where 0.1 gives a small amount of smoothing and +1 follows * Asuncion et al. (2009), who recommend a +1 adjustment for EM. * - Online - * - Value should be >= 0 + * - Value should be greater than or equal to 0 * - default = (1.0 / k), following the implementation from - * [[https://github.com/Blei-Lab/onlineldavb]]. + * here. */ @Since("1.3.0") def setTopicConcentration(topicConcentration: Double): this.type = { @@ -216,35 +218,37 @@ class LDA private ( def getBeta: Double = getTopicConcentration /** - * Alias for [[setTopicConcentration()]] + * Alias for `setTopicConcentration()` */ @Since("1.3.0") def setBeta(beta: Double): this.type = setTopicConcentration(beta) /** - * Maximum number of iterations for learning. + * Maximum number of iterations allowed. */ @Since("1.3.0") def getMaxIterations: Int = maxIterations /** - * Maximum number of iterations for learning. + * Set the maximum number of iterations allowed. * (default = 20) */ @Since("1.3.0") def setMaxIterations(maxIterations: Int): this.type = { + require(maxIterations >= 0, + s"Maximum of iterations must be nonnegative but got ${maxIterations}") this.maxIterations = maxIterations this } /** - * Random seed + * Random seed for cluster initialization. */ @Since("1.3.0") def getSeed: Long = seed /** - * Random seed + * Set the random seed for cluster initialization. */ @Since("1.3.0") def setSeed(seed: Long): this.type = { @@ -259,15 +263,18 @@ class LDA private ( def getCheckpointInterval: Int = checkpointInterval /** - * Period (in iterations) between checkpoints (default = 10). Checkpointing helps with recovery - * (when nodes fail). It also helps with eliminating temporary shuffle files on disk, which can be - * important when LDA is run for many iterations. If the checkpoint directory is not set in - * [[org.apache.spark.SparkContext]], this setting is ignored. + * Parameter for set checkpoint interval (greater than or equal to 1) or disable checkpoint (-1). + * E.g. 10 means that the cache will get checkpointed every 10 iterations. Checkpointing helps + * with recovery (when nodes fail). It also helps with eliminating temporary shuffle files on + * disk, which can be important when LDA is run for many iterations. If the checkpoint directory + * is not set in [[org.apache.spark.SparkContext]], this setting is ignored. (default = 10) * * @see [[org.apache.spark.SparkContext#setCheckpointDir]] */ @Since("1.3.0") def setCheckpointInterval(checkpointInterval: Int): this.type = { + require(checkpointInterval == -1 || checkpointInterval > 0, + s"Period between checkpoints must be -1 or positive but got ${checkpointInterval}") this.checkpointInterval = checkpointInterval this } @@ -301,7 +308,7 @@ class LDA private ( @Since("1.4.0") def setOptimizer(optimizerName: String): this.type = { this.ldaOptimizer = - optimizerName.toLowerCase match { + optimizerName.toLowerCase(Locale.ROOT) match { case "em" => new EMLDAOptimizer case "online" => new OnlineLDAOptimizer case other => @@ -316,7 +323,7 @@ class LDA private ( * @param documents RDD of documents, which are term (word) count vectors paired with IDs. * The term count vectors are "bags of words" with a fixed-size vocabulary * (where the vocabulary size is the length of the vector). - * Document IDs must be unique and >= 0. + * Document IDs must be unique and greater than or equal to 0. * @return Inferred LDA model */ @Since("1.3.0") @@ -335,7 +342,7 @@ class LDA private ( } /** - * Java-friendly version of [[run()]] + * Java-friendly version of `run()` */ @Since("1.3.0") def run(documents: JavaPairRDD[java.lang.Long, Vector]): LDAModel = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 31d8a9fdea1c..4ab420058f33 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.clustering -import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax, argtopk, normalize, sum} +import breeze.linalg.{argmax, argtopk, normalize, sum, DenseMatrix => BDM, DenseVector => BDV} import breeze.numerics.{exp, lgamma} import org.apache.hadoop.fs.Path import org.json4s.DefaultFormats @@ -25,13 +25,13 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} import org.apache.spark.graphx.{Edge, EdgeContext, Graph, VertexId} import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector, Vectors} import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.util.BoundedPriorityQueue /** @@ -66,7 +66,7 @@ abstract class LDAModel private[clustering] extends Saveable { * * This is the parameter to a symmetric Dirichlet distribution. * - * Note: The topics' distributions over terms are called "beta" in the original LDA paper + * @note The topics' distributions over terms are called "beta" in the original LDA paper * by Blei et al., but are called "phi" in many later papers such as Asuncion et al., 2009. */ @Since("1.5.0") @@ -171,7 +171,7 @@ abstract class LDAModel private[clustering] extends Saveable { * The term count vectors are "bags of words" with a fixed-size vocabulary * (where the vocabulary size is the length of the vector). * This must use the same vocabulary (ordering of term counts) as in training. - * Document IDs must be unique and >= 0. + * Document IDs must be unique and greater than or equal to 0. * @return Estimated topic distribution for each document. * The returned RDD may be zipped with the given RDD, where each returned vector * is a multinomial distribution over topics. @@ -183,16 +183,15 @@ abstract class LDAModel private[clustering] extends Saveable { /** * Local LDA model. * This model stores only the inferred topics. - * It may be used for computing topics for new documents, but it may give less accurate answers - * than the [[DistributedLDAModel]]. + * * @param topics Inferred topics (vocabSize x k matrix). */ @Since("1.3.0") -class LocalLDAModel private[clustering] ( +class LocalLDAModel private[spark] ( @Since("1.3.0") val topics: Matrix, @Since("1.5.0") override val docConcentration: Vector, @Since("1.5.0") override val topicConcentration: Double, - override protected[clustering] val gammaShape: Double = 100) + override protected[spark] val gammaShape: Double = 100) extends LDAModel with Serializable { @Since("1.3.0") @@ -206,7 +205,7 @@ class LocalLDAModel private[clustering] ( @Since("1.3.0") override def describeTopics(maxTermsPerTopic: Int): Array[(Array[Int], Array[Double])] = { - val brzTopics = topics.toBreeze.toDenseMatrix + val brzTopics = topics.asBreeze.toDenseMatrix Range(0, k).map { topicIndex => val topic = normalize(brzTopics(::, topicIndex), 1.0) val (termWeights, terms) = @@ -234,11 +233,11 @@ class LocalLDAModel private[clustering] ( */ @Since("1.5.0") def logLikelihood(documents: RDD[(Long, Vector)]): Double = logLikelihoodBound(documents, - docConcentration, topicConcentration, topicsMatrix.toBreeze.toDenseMatrix, gammaShape, k, + docConcentration, topicConcentration, topicsMatrix.asBreeze.toDenseMatrix, gammaShape, k, vocabSize) /** - * Java-friendly version of [[logLikelihood]] + * Java-friendly version of `logLikelihood` */ @Since("1.5.0") def logLikelihood(documents: JavaPairRDD[java.lang.Long, Vector]): Double = { @@ -246,7 +245,7 @@ class LocalLDAModel private[clustering] ( } /** - * Calculate an upper bound bound on perplexity. (Lower is better.) + * Calculate an upper bound on perplexity. (Lower is better.) * See Equation (16) in original Online LDA paper. * * @param documents test corpus to use for calculating perplexity @@ -260,7 +259,9 @@ class LocalLDAModel private[clustering] ( -logLikelihood(documents) / corpusTokenCount } - /** Java-friendly version of [[logPerplexity]] */ + /** + * Java-friendly version of `logPerplexity` + */ @Since("1.5.0") def logPerplexity(documents: JavaPairRDD[java.lang.Long, Vector]): Double = { logPerplexity(documents.rdd.asInstanceOf[RDD[(Long, Vector)]]) @@ -292,7 +293,7 @@ class LocalLDAModel private[clustering] ( gammaShape: Double, k: Int, vocabSize: Long): Double = { - val brzAlpha = alpha.toBreeze.toDenseVector + val brzAlpha = alpha.asBreeze.toDenseVector // transpose because dirichletExpectation normalizes by row and we need to normalize // by topic (columns of lambda) val Elogbeta = LDAUtils.dirichletExpectation(lambda.t).t @@ -304,7 +305,7 @@ class LocalLDAModel private[clustering] ( documents.filter(_._2.numNonzeros > 0).map { case (id: Long, termCounts: Vector) => val localElogbeta = ElogbetaBc.value var docBound = 0.0D - val (gammad: BDV[Double], _) = OnlineLDAOptimizer.variationalTopicInference( + val (gammad: BDV[Double], _, _) = OnlineLDAOptimizer.variationalTopicInference( termCounts, exp(localElogbeta), brzAlpha, gammaShape, k) val Elogthetad: BDV[Double] = LDAUtils.dirichletExpectation(gammad) @@ -313,17 +314,18 @@ class LocalLDAModel private[clustering] ( docBound += count * LDAUtils.logSumExp(Elogthetad + localElogbeta(idx, ::).t) } // E[log p(theta | alpha) - log q(theta | gamma)] - docBound += sum((brzAlpha - gammad) :* Elogthetad) + docBound += sum((brzAlpha - gammad) *:* Elogthetad) docBound += sum(lgamma(gammad) - lgamma(brzAlpha)) docBound += lgamma(sum(brzAlpha)) - lgamma(sum(gammad)) docBound }.sum() + ElogbetaBc.destroy(blocking = false) // Bound component for prob(topic-term distributions): // E[log p(beta | eta) - log q(beta | lambda)] val sumEta = eta * vocabSize - val topicsPart = sum((eta - lambda) :* Elogbeta) + + val topicsPart = sum((eta - lambda) *:* Elogbeta) + sum(lgamma(lambda) - lgamma(eta)) + sum(lgamma(sumEta) - lgamma(sum(lambda(::, breeze.linalg.*)))) @@ -345,17 +347,17 @@ class LocalLDAModel private[clustering] ( def topicDistributions(documents: RDD[(Long, Vector)]): RDD[(Long, Vector)] = { // Double transpose because dirichletExpectation normalizes by row and we need to normalize // by topic (columns of lambda) - val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.toBreeze.toDenseMatrix.t).t) + val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.asBreeze.toDenseMatrix.t).t) val expElogbetaBc = documents.sparkContext.broadcast(expElogbeta) - val docConcentrationBrz = this.docConcentration.toBreeze + val docConcentrationBrz = this.docConcentration.asBreeze val gammaShape = this.gammaShape val k = this.k documents.map { case (id: Long, termCounts: Vector) => if (termCounts.numNonzeros == 0) { - (id, Vectors.zeros(k)) + (id, Vectors.zeros(k)) } else { - val (gamma, _) = OnlineLDAOptimizer.variationalTopicInference( + val (gamma, _, _) = OnlineLDAOptimizer.variationalTopicInference( termCounts, expElogbetaBc.value, docConcentrationBrz, @@ -367,7 +369,56 @@ class LocalLDAModel private[clustering] ( } /** - * Java-friendly version of [[topicDistributions]] + * Get a method usable as a UDF for `topicDistributions()` + */ + private[spark] def getTopicDistributionMethod(sc: SparkContext): Vector => Vector = { + val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.asBreeze.toDenseMatrix.t).t) + val docConcentrationBrz = this.docConcentration.asBreeze + val gammaShape = this.gammaShape + val k = this.k + + (termCounts: Vector) => + if (termCounts.numNonzeros == 0) { + Vectors.zeros(k) + } else { + val (gamma, _, _) = OnlineLDAOptimizer.variationalTopicInference( + termCounts, + expElogbeta, + docConcentrationBrz, + gammaShape, + k) + Vectors.dense(normalize(gamma, 1.0).toArray) + } + } + + /** + * Predicts the topic mixture distribution for a document (often called "theta" in the + * literature). Returns a vector of zeros for an empty document. + * + * Note this means to allow quick query for single document. For batch documents, please refer + * to `topicDistributions()` to avoid overhead. + * + * @param document document to predict topic mixture distributions for + * @return topic mixture distribution for the document + */ + @Since("2.0.0") + def topicDistribution(document: Vector): Vector = { + val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.asBreeze.toDenseMatrix.t).t) + if (document.numNonzeros == 0) { + Vectors.zeros(this.k) + } else { + val (gamma, _, _) = OnlineLDAOptimizer.variationalTopicInference( + document, + expElogbeta, + this.docConcentration.asBreeze, + gammaShape, + this.k) + Vectors.dense(normalize(gamma, 1.0).toArray) + } + } + + /** + * Java-friendly version of `topicDistributions` */ @Since("1.4.1") def topicDistributions( @@ -378,7 +429,11 @@ class LocalLDAModel private[clustering] ( } -@Experimental +/** + * Local (non-distributed) model fitted by [[LDA]]. + * + * This model stores the inferred topics only; it does not store info about the training dataset. + */ @Since("1.5.0") object LocalLDAModel extends Loader[LocalLDAModel] { @@ -399,9 +454,7 @@ object LocalLDAModel extends Loader[LocalLDAModel] { docConcentration: Vector, topicConcentration: Double, gammaShape: Double): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ - + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val k = topicsMatrix.numCols val metadata = compact(render (("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ @@ -411,11 +464,11 @@ object LocalLDAModel extends Loader[LocalLDAModel] { ("gammaShape" -> gammaShape))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) - val topicsDenseMatrix = topicsMatrix.toBreeze.toDenseMatrix + val topicsDenseMatrix = topicsMatrix.asBreeze.toDenseMatrix val topics = Range(0, k).map { topicInd => Data(Vectors.dense((topicsDenseMatrix(::, topicInd).toArray)), topicInd) - }.toSeq - sc.parallelize(topics, 1).toDF().write.parquet(Loader.dataPath(path)) + } + spark.createDataFrame(topics).repartition(1).write.parquet(Loader.dataPath(path)) } def load( @@ -425,8 +478,8 @@ object LocalLDAModel extends Loader[LocalLDAModel] { topicConcentration: Double, gammaShape: Double): LocalLDAModel = { val dataPath = Loader.dataPath(path) - val sqlContext = SQLContext.getOrCreate(sc) - val dataFrame = sqlContext.read.parquet(dataPath) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val dataFrame = spark.read.parquet(dataPath) Loader.checkSchema[Data](dataFrame.schema) val topics = dataFrame.collect() @@ -435,7 +488,7 @@ object LocalLDAModel extends Loader[LocalLDAModel] { val brzTopics = BDM.zeros[Double](vocabSize, k) topics.foreach { case Row(vec: Vector, ind: Int) => - brzTopics(::, ind) := vec.toBreeze + brzTopics(::, ind) := vec.asBreeze } val topicsMat = Matrices.fromBreeze(brzTopics) @@ -477,8 +530,6 @@ object LocalLDAModel extends Loader[LocalLDAModel] { /** * Distributed LDA model. * This model stores the inferred topics, the full training dataset, and the topic distributions. - * When computing topics for new documents, it may give more accurate answers - * than the [[LocalLDAModel]]. */ @Since("1.3.0") class DistributedLDAModel private[clustering] ( @@ -489,7 +540,8 @@ class DistributedLDAModel private[clustering] ( @Since("1.5.0") override val docConcentration: Vector, @Since("1.5.0") override val topicConcentration: Double, private[spark] val iterationTimes: Array[Double], - override protected[clustering] val gammaShape: Double = 100) + override protected[clustering] val gammaShape: Double = DistributedLDAModel.defaultGammaShape, + private[spark] val checkpointFiles: Array[String] = Array.empty[String]) extends LDAModel { import LDA._ @@ -669,7 +721,7 @@ class DistributedLDAModel private[clustering] ( val N_wj = edgeContext.attr val smoothed_N_wk: TopicCounts = edgeContext.dstAttr + (eta - 1.0) val smoothed_N_kj: TopicCounts = edgeContext.srcAttr + (alpha - 1.0) - val phi_wk: TopicCounts = smoothed_N_wk :/ smoothed_N_k + val phi_wk: TopicCounts = smoothed_N_wk /:/ smoothed_N_k val theta_kj: TopicCounts = normalize(smoothed_N_kj, 1.0) val tokenLogLikelihood = N_wj * math.log(phi_wk.dot(theta_kj)) edgeContext.sendToDst(tokenLogLikelihood) @@ -696,13 +748,13 @@ class DistributedLDAModel private[clustering] ( if (isTermVertex(vertex)) { val N_wk = vertex._2 val smoothed_N_wk: TopicCounts = N_wk + (eta - 1.0) - val phi_wk: TopicCounts = smoothed_N_wk :/ smoothed_N_k - (eta - 1.0) * sum(phi_wk.map(math.log)) + val phi_wk: TopicCounts = smoothed_N_wk /:/ smoothed_N_k + sumPrior + (eta - 1.0) * sum(phi_wk.map(math.log)) } else { val N_kj = vertex._2 val smoothed_N_kj: TopicCounts = N_kj + (alpha - 1.0) val theta_kj: TopicCounts = normalize(smoothed_N_kj, 1.0) - (alpha - 1.0) * sum(theta_kj.map(math.log)) + sumPrior + (alpha - 1.0) * sum(theta_kj.map(math.log)) } } graph.vertices.aggregate(0.0)(seqOp, _ + _) @@ -739,11 +791,11 @@ class DistributedLDAModel private[clustering] ( val topIndices = argtopk(topicCounts, k) val sumCounts = sum(topicCounts) val weights = if (sumCounts != 0) { - topicCounts(topIndices) / sumCounts + topicCounts(topIndices).toArray.map(_ / sumCounts) } else { - topicCounts(topIndices) + topicCounts(topIndices).toArray } - (docID.toLong, topIndices.toArray, weights.toArray) + (docID.toLong, topIndices.toArray, weights) } } @@ -761,22 +813,31 @@ class DistributedLDAModel private[clustering] ( override protected def formatVersion = "1.0" - /** - * Java-friendly version of [[topicDistributions]] - */ @Since("1.5.0") override def save(sc: SparkContext, path: String): Unit = { + // Note: This intentionally does not save checkpointFiles. DistributedLDAModel.SaveLoadV1_0.save( sc, path, graph, globalTopicTotals, k, vocabSize, docConcentration, topicConcentration, iterationTimes, gammaShape) } } - -@Experimental +/** + * Distributed model fitted by [[LDA]]. + * This type of model is currently only produced by Expectation-Maximization (EM). + * + * This model stores the inferred topics, the full training dataset, and the topic distribution + * for each training document. + */ @Since("1.5.0") object DistributedLDAModel extends Loader[DistributedLDAModel] { + /** + * The [[DistributedLDAModel]] constructor's default arguments assume gammaShape = 100 + * to ensure equivalence in LDAModel.toLocal conversion. + */ + private[clustering] val defaultGammaShape: Double = 100 + private object SaveLoadV1_0 { val thisFormatVersion = "1.0" @@ -803,8 +864,7 @@ object DistributedLDAModel extends Loader[DistributedLDAModel] { topicConcentration: Double, iterationTimes: Array[Double], gammaShape: Double): Unit = { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val metadata = compact(render (("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ @@ -816,18 +876,17 @@ object DistributedLDAModel extends Loader[DistributedLDAModel] { sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) val newPath = new Path(Loader.dataPath(path), "globalTopicTotals").toUri.toString - sc.parallelize(Seq(Data(Vectors.fromBreeze(globalTopicTotals)))).toDF() - .write.parquet(newPath) + spark.createDataFrame(Seq(Data(Vectors.fromBreeze(globalTopicTotals)))).write.parquet(newPath) val verticesPath = new Path(Loader.dataPath(path), "topicCounts").toUri.toString - graph.vertices.map { case (ind, vertex) => + spark.createDataFrame(graph.vertices.map { case (ind, vertex) => VertexData(ind, Vectors.fromBreeze(vertex)) - }.toDF().write.parquet(verticesPath) + }).write.parquet(verticesPath) val edgesPath = new Path(Loader.dataPath(path), "tokenCounts").toUri.toString - graph.edges.map { case Edge(srcId, dstId, prop) => + spark.createDataFrame(graph.edges.map { case Edge(srcId, dstId, prop) => EdgeData(srcId, dstId, prop) - }.toDF().write.parquet(edgesPath) + }).write.parquet(edgesPath) } def load( @@ -841,21 +900,21 @@ object DistributedLDAModel extends Loader[DistributedLDAModel] { val dataPath = new Path(Loader.dataPath(path), "globalTopicTotals").toUri.toString val vertexDataPath = new Path(Loader.dataPath(path), "topicCounts").toUri.toString val edgeDataPath = new Path(Loader.dataPath(path), "tokenCounts").toUri.toString - val sqlContext = SQLContext.getOrCreate(sc) - val dataFrame = sqlContext.read.parquet(dataPath) - val vertexDataFrame = sqlContext.read.parquet(vertexDataPath) - val edgeDataFrame = sqlContext.read.parquet(edgeDataPath) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val dataFrame = spark.read.parquet(dataPath) + val vertexDataFrame = spark.read.parquet(vertexDataPath) + val edgeDataFrame = spark.read.parquet(edgeDataPath) Loader.checkSchema[Data](dataFrame.schema) Loader.checkSchema[VertexData](vertexDataFrame.schema) Loader.checkSchema[EdgeData](edgeDataFrame.schema) val globalTopicTotals: LDA.TopicCounts = - dataFrame.first().getAs[Vector](0).toBreeze.toDenseVector - val vertices: RDD[(VertexId, LDA.TopicCounts)] = vertexDataFrame.map { - case Row(ind: Long, vec: Vector) => (ind, vec.toBreeze.toDenseVector) + dataFrame.first().getAs[Vector](0).asBreeze.toDenseVector + val vertices: RDD[(VertexId, LDA.TopicCounts)] = vertexDataFrame.rdd.map { + case Row(ind: Long, vec: Vector) => (ind, vec.asBreeze.toDenseVector) } - val edges: RDD[Edge[LDA.TokenCount]] = edgeDataFrame.map { + val edges: RDD[Edge[LDA.TokenCount]] = edgeDataFrame.rdd.map { case Row(srcId: Long, dstId: Long, prop: Double) => Edge(srcId, dstId, prop) } val graph: Graph[LDA.TopicCounts, LDA.TokenCount] = Graph(vertices, edges) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala index 17c0609800e9..d633893e55f5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala @@ -19,16 +19,16 @@ package org.apache.spark.mllib.clustering import java.util.Random -import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, all, normalize, sum} -import breeze.numerics.{trigamma, abs, exp} +import breeze.linalg.{all, normalize, sum, DenseMatrix => BDM, DenseVector => BDV} +import breeze.numerics.{abs, exp, trigamma} import breeze.stats.distributions.{Gamma, RandBasis} import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.graphx._ -import org.apache.spark.graphx.impl.GraphImpl -import org.apache.spark.mllib.impl.PeriodicGraphCheckpointer +import org.apache.spark.graphx.util.PeriodicGraphCheckpointer import org.apache.spark.mllib.linalg.{DenseVector, Matrices, SparseVector, Vector, Vectors} import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel /** * :: DeveloperApi :: @@ -38,7 +38,7 @@ import org.apache.spark.rdd.RDD */ @Since("1.4.0") @DeveloperApi -sealed trait LDAOptimizer { +trait LDAOptimizer { /* DEVELOPERS NOTE: @@ -81,9 +81,31 @@ final class EMLDAOptimizer extends LDAOptimizer { import LDA._ + // Adjustable parameters + private var keepLastCheckpoint: Boolean = true + + /** + * If using checkpointing, this indicates whether to keep the last checkpoint (vs clean up). + */ + @Since("2.0.0") + def getKeepLastCheckpoint: Boolean = this.keepLastCheckpoint + /** - * The following fields will only be initialized through the initialize() method + * If using checkpointing, this indicates whether to keep the last checkpoint (vs clean up). + * Deleting the checkpoint can cause failures if a data partition is lost, so set this bit with + * care. + * + * Default: true + * + * @note Checkpoints will be cleaned up via reference counting, regardless. */ + @Since("2.0.0") + def setKeepLastCheckpoint(keepLastCheckpoint: Boolean): this.type = { + this.keepLastCheckpoint = keepLastCheckpoint + this + } + + // The following fields will only be initialized through the initialize() method private[clustering] var graph: Graph[TopicCounts, TokenCount] = null private[clustering] var k: Int = 0 private[clustering] var vocabSize: Int = 0 @@ -95,7 +117,9 @@ final class EMLDAOptimizer extends LDAOptimizer { /** * Compute bipartite term/doc graph. */ - override private[clustering] def initialize(docs: RDD[(Long, Vector)], lda: LDA): LDAOptimizer = { + override private[clustering] def initialize( + docs: RDD[(Long, Vector)], + lda: LDA): EMLDAOptimizer = { // EMLDAOptimizer currently only supports symmetric document-topic priors val docConcentration = lda.getDocConcentration @@ -116,7 +140,7 @@ final class EMLDAOptimizer extends LDAOptimizer { // For each document, create an edge (Document -> Term) for each unique term in the document. val edges: RDD[Edge[TokenCount]] = docs.flatMap { case (docID: Long, termCounts: Vector) => // Add edges for terms with non-zero counts. - termCounts.toBreeze.activeIterator.filter(_._2 != 0.0).map { case (term, cnt) => + termCounts.asBreeze.activeIterator.filter(_._2 != 0.0).map { case (term, cnt) => Edge(docID, term2index(term), cnt) } } @@ -186,7 +210,7 @@ final class EMLDAOptimizer extends LDAOptimizer { graph.aggregateMessages[(Boolean, TopicCounts)](sendMsg, mergeMsg) .mapValues(_._2) // Update the vertex descriptors with the new counts. - val newGraph = GraphImpl.fromExistingRDDs(docTopicDistributions, graph.edges) + val newGraph = Graph(docTopicDistributions, graph.edges) graph = newGraph graphCheckpointer.update(newGraph) globalTopicTotals = computeGlobalTopicTotals() @@ -207,12 +231,18 @@ final class EMLDAOptimizer extends LDAOptimizer { override private[clustering] def getLDAModel(iterationTimes: Array[Double]): LDAModel = { require(graph != null, "graph is null, EMLDAOptimizer not initialized.") - this.graphCheckpointer.deleteAllCheckpoints() + val checkpointFiles: Array[String] = if (keepLastCheckpoint) { + this.graphCheckpointer.deleteAllCheckpointsButLast() + this.graphCheckpointer.getAllCheckpointFiles + } else { + this.graphCheckpointer.deleteAllCheckpoints() + Array.empty[String] + } // The constructor's default arguments assume gammaShape = 100 to ensure equivalence in - // LDAModel.toLocal conversion + // LDAModel.toLocal conversion. new DistributedLDAModel(this.graph, this.globalTopicTotals, this.k, this.vocabSize, Vectors.dense(Array.fill(this.k)(this.docConcentration)), this.topicConcentration, - iterationTimes) + iterationTimes, DistributedLDAModel.defaultGammaShape, checkpointFiles) } } @@ -320,9 +350,9 @@ final class OnlineLDAOptimizer extends LDAOptimizer { * Mini-batch fraction in (0, 1], which sets the fraction of document sampled and used in * each iteration. * - * Note that this should be adjusted in synch with [[LDA.setMaxIterations()]] + * @note This should be adjusted in synch with `LDA.setMaxIterations()` * so the entire corpus is used. Specifically, set both so that - * maxIterations * miniBatchFraction >= 1. + * maxIterations * miniBatchFraction is at least 1. * * Default: 0.05, i.e., 5% of total documents. */ @@ -430,7 +460,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer { val vocabSize = this.vocabSize val expElogbeta = exp(LDAUtils.dirichletExpectation(lambda)).t val expElogbetaBc = batch.sparkContext.broadcast(expElogbeta) - val alpha = this.alpha.toBreeze + val alpha = this.alpha.asBreeze val gammaShape = this.gammaShape val stats: RDD[(BDM[Double], List[BDV[Double]])] = batch.mapPartitions { docs => @@ -439,22 +469,20 @@ final class OnlineLDAOptimizer extends LDAOptimizer { val stat = BDM.zeros[Double](k, vocabSize) var gammaPart = List[BDV[Double]]() nonEmptyDocs.foreach { case (_, termCounts: Vector) => - val ids: List[Int] = termCounts match { - case v: DenseVector => (0 until v.size).toList - case v: SparseVector => v.indices.toList - } - val (gammad, sstats) = OnlineLDAOptimizer.variationalTopicInference( + val (gammad, sstats, ids) = OnlineLDAOptimizer.variationalTopicInference( termCounts, expElogbetaBc.value, alpha, gammaShape, k) stat(::, ids) := stat(::, ids).toDenseMatrix + sstats gammaPart = gammad :: gammaPart } Iterator((stat, gammaPart)) - } - val statsSum: BDM[Double] = stats.map(_._1).reduce(_ += _) - expElogbetaBc.unpersist() + }.persist(StorageLevel.MEMORY_AND_DISK) + val statsSum: BDM[Double] = stats.map(_._1).treeAggregate(BDM.zeros[Double](k, vocabSize))( + _ += _, _ += _) val gammat: BDM[Double] = breeze.linalg.DenseMatrix.vertcat( - stats.map(_._2).reduce(_ ++ _).map(_.toDenseMatrix): _*) - val batchResult = statsSum :* expElogbeta.t + stats.map(_._2).flatMap(list => list).collect().map(_.toDenseMatrix): _*) + stats.unpersist() + expElogbetaBc.destroy(false) + val batchResult = statsSum *:* expElogbeta.t // Note that this is an optimization to avoid batch.count updateLambda(batchResult, (miniBatchFraction * corpusSize).ceil.toInt) @@ -483,9 +511,10 @@ final class OnlineLDAOptimizer extends LDAOptimizer { private def updateAlpha(gammat: BDM[Double]): Unit = { val weight = rho() val N = gammat.rows.toDouble - val alpha = this.alpha.toBreeze.toDenseVector - val logphat: BDM[Double] = sum(LDAUtils.dirichletExpectation(gammat)(::, breeze.linalg.*)) / N - val gradf = N * (-LDAUtils.dirichletExpectation(alpha) + logphat.toDenseVector) + val alpha = this.alpha.asBreeze.toDenseVector + val logphat: BDV[Double] = + sum(LDAUtils.dirichletExpectation(gammat)(::, breeze.linalg.*)).t / N + val gradf = N * (-LDAUtils.dirichletExpectation(alpha) + logphat) val c = N * trigamma(sum(alpha)) val q = -N * trigamma(alpha) @@ -493,7 +522,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer { val dalpha = -(gradf - b) / q - if (all((weight * dalpha + alpha) :> 0D)) { + if (all((weight * dalpha + alpha) >:> 0D)) { alpha :+= weight * dalpha this.alpha = Vectors.dense(alpha.toArray) } @@ -534,14 +563,17 @@ private[clustering] object OnlineLDAOptimizer { * * An optimization (Lee, Seung: Algorithms for non-negative matrix factorization, NIPS 2001) * avoids explicit computation of variational parameter `phi`. - * @see [[http://citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.31.7566]] + * @see here + * + * @return Returns a tuple of `gammad` - estimate of gamma, the topic distribution, `sstatsd` - + * statistics for updating lambda and `ids` - list of termCounts vector indices. */ private[clustering] def variationalTopicInference( termCounts: Vector, expElogbeta: BDM[Double], alpha: breeze.linalg.Vector[Double], gammaShape: Double, - k: Int): (BDV[Double], BDM[Double]) = { + k: Int): (BDV[Double], BDM[Double], List[Int]) = { val (ids: List[Int], cts: Array[Double]) = termCounts match { case v: DenseVector => ((0 until v.size).toList, v.values) case v: SparseVector => (v.indices.toList, v.values) @@ -552,7 +584,7 @@ private[clustering] object OnlineLDAOptimizer { val expElogthetad: BDV[Double] = exp(LDAUtils.dirichletExpectation(gammad)) // K val expElogbetad = expElogbeta(ids, ::).toDenseMatrix // ids * K - val phiNorm: BDV[Double] = expElogbetad * expElogthetad :+ 1e-100 // ids + val phiNorm: BDV[Double] = expElogbetad * expElogthetad +:+ 1e-100 // ids var meanGammaChange = 1D val ctsVector = new BDV[Double](cts) // ids @@ -560,14 +592,14 @@ private[clustering] object OnlineLDAOptimizer { while (meanGammaChange > 1e-3) { val lastgamma = gammad.copy // K K * ids ids - gammad := (expElogthetad :* (expElogbetad.t * (ctsVector :/ phiNorm))) :+ alpha + gammad := (expElogthetad *:* (expElogbetad.t * (ctsVector /:/ phiNorm))) +:+ alpha expElogthetad := exp(LDAUtils.dirichletExpectation(gammad)) // TODO: Keep more values in log space, and only exponentiate when needed. - phiNorm := expElogbetad * expElogthetad :+ 1e-100 + phiNorm := expElogbetad * expElogthetad +:+ 1e-100 meanGammaChange = sum(abs(gammad - lastgamma)) / k } - val sstatsd = expElogthetad.asDenseMatrix.t * (ctsVector :/ phiNorm).asDenseMatrix - (gammad, sstatsd) + val sstatsd = expElogthetad.asDenseMatrix.t * (ctsVector /:/ phiNorm).asDenseMatrix + (gammad, sstatsd, ids) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAUtils.scala index a9ba7b60bad0..c4bbe51a46c3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAUtils.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.mllib.clustering -import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, max, sum} +import breeze.linalg.{max, sum, DenseMatrix => BDM, DenseVector => BDV} import breeze.numerics._ /** @@ -25,11 +25,11 @@ import breeze.numerics._ private[clustering] object LDAUtils { /** * Log Sum Exp with overflow protection using the identity: - * For any a: \log \sum_{n=1}^N \exp\{x_n\} = a + \log \sum_{n=1}^N \exp\{x_n - a\} + * For any a: $\log \sum_{n=1}^N \exp\{x_n\} = a + \log \sum_{n=1}^N \exp\{x_n - a\}$ */ private[clustering] def logSumExp(x: BDV[Double]): Double = { val a = max(x) - a + log(sum(exp(x :- a))) + a + log(sum(exp(x -:- a))) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala index b2f140e1b135..53587670a5db 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -19,9 +19,9 @@ package org.apache.spark.mllib.clustering import scala.util.Random -import org.apache.spark.Logging -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.BLAS.{axpy, scal} +import org.apache.spark.mllib.linalg.Vectors /** * An utility object to run K-means locally. This is private to the ML package because it's used @@ -46,17 +46,15 @@ private[mllib] object LocalKMeans extends Logging { // Initialize centers by sampling using the k-means++ procedure. centers(0) = pickWeighted(rand, points, weights).toDense + val costArray = points.map(KMeans.fastSquaredDistance(_, centers(0))) + for (i <- 1 until k) { - // Pick the next center with a probability proportional to cost under current centers - val curCenters = centers.view.take(i) - val sum = points.view.zip(weights).map { case (p, w) => - w * KMeans.pointCost(curCenters, p) - }.sum + val sum = costArray.zip(weights).map(p => p._1 * p._2).sum val r = rand.nextDouble() * sum var cumulativeScore = 0.0 var j = 0 while (j < points.length && cumulativeScore < r) { - cumulativeScore += weights(j) * KMeans.pointCost(curCenters, points(j)) + cumulativeScore += weights(j) * costArray(j) j += 1 } if (j == 0) { @@ -66,6 +64,12 @@ private[mllib] object LocalKMeans extends Logging { } else { centers(i) = points(j - 1).toDense } + + // update costArray + for (p <- points.indices) { + costArray(p) = math.min(KMeans.fastSquaredDistance(points(p), centers(i)), costArray(p)) + } + } // Run up to maxIterations iterations of Lloyd's algorithm 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 7cd9b08fa8e0..b2437b845f82 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,26 +17,26 @@ package org.apache.spark.mllib.clustering -import org.json4s.JsonDSL._ import org.json4s._ +import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ +import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.graphx._ -import org.apache.spark.graphx.impl.GraphImpl +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{Loader, MLUtils, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.util.random.XORShiftRandom -import org.apache.spark.{Logging, SparkContext, SparkException} /** * Model produced by [[PowerIterationClustering]]. * * @param k number of clusters - * @param assignments an RDD of clustering [[PowerIterationClustering#Assignment]]s + * @param assignments an RDD of clustering `PowerIterationClustering#Assignment`s */ @Since("1.3.0") class PowerIterationClusteringModel @Since("1.3.0") ( @@ -70,31 +70,29 @@ object PowerIterationClusteringModel extends Loader[PowerIterationClusteringMode @Since("1.4.0") def save(sc: SparkContext, model: PowerIterationClusteringModel, path: String): Unit = { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("k" -> model.k))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) - val dataRDD = model.assignments.toDF() - dataRDD.write.parquet(Loader.dataPath(path)) + spark.createDataFrame(model.assignments).write.parquet(Loader.dataPath(path)) } @Since("1.4.0") def load(sc: SparkContext, path: String): PowerIterationClusteringModel = { implicit val formats = DefaultFormats - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) assert(className == thisClassName) assert(formatVersion == thisFormatVersion) val k = (metadata \ "k").extract[Int] - val assignments = sqlContext.read.parquet(Loader.dataPath(path)) + val assignments = spark.read.parquet(Loader.dataPath(path)) Loader.checkSchema[PowerIterationClustering.Assignment](assignments.schema) - val assignmentsRDD = assignments.map { + val assignmentsRDD = assignments.rdd.map { case Row(id: Long, cluster: Int) => PowerIterationClustering.Assignment(id, cluster) } @@ -105,15 +103,18 @@ object PowerIterationClusteringModel extends Loader[PowerIterationClusteringMode /** * Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by - * [[http://www.icml2010.org/papers/387.pdf Lin and Cohen]]. From the abstract: PIC finds a very - * low-dimensional embedding of a dataset using truncated power iteration on a normalized pair-wise - * similarity matrix of the data. + * Lin and Cohen. From the abstract: PIC finds + * a very low-dimensional embedding of a dataset using truncated power iteration on a normalized + * pair-wise similarity matrix of the data. * * @param k Number of clusters. * @param maxIterations Maximum number of iterations of the PIC algorithm. - * @param initMode Initialization mode. + * @param initMode Set the initialization mode. This can be either "random" to use a random vector + * as vertex properties, or "degree" to use normalized sum similarities. + * Default: random. * - * @see [[http://en.wikipedia.org/wiki/Spectral_clustering Spectral clustering (Wikipedia)]] + * @see + * Spectral clustering (Wikipedia) */ @Since("1.3.0") class PowerIterationClustering private[clustering] ( @@ -135,6 +136,8 @@ class PowerIterationClustering private[clustering] ( */ @Since("1.3.0") def setK(k: Int): this.type = { + require(k > 0, + s"Number of clusters must be positive but got ${k}") this.k = k this } @@ -144,6 +147,8 @@ class PowerIterationClustering private[clustering] ( */ @Since("1.3.0") def setMaxIterations(maxIterations: Int): this.type = { + require(maxIterations >= 0, + s"Maximum of iterations must be nonnegative but got ${maxIterations}") this.maxIterations = maxIterations this } @@ -206,7 +211,7 @@ class PowerIterationClustering private[clustering] ( } /** - * A Java-friendly version of [[PowerIterationClustering.run]]. + * A Java-friendly version of `PowerIterationClustering.run`. */ @Since("1.3.0") def run(similarities: JavaRDD[(java.lang.Long, java.lang.Long, java.lang.Double)]) @@ -254,7 +259,7 @@ object PowerIterationClustering extends Logging { val j = ctx.dstId val s = ctx.attr if (s < 0.0) { - throw new SparkException("Similarity must be nonnegative but found s($i, $j) = $s.") + throw new SparkException(s"Similarity must be nonnegative but found s($i, $j) = $s.") } if (s > 0.0) { ctx.sendToSrc(s) @@ -262,10 +267,12 @@ object PowerIterationClustering extends Logging { }, mergeMsg = _ + _, TripletFields.EdgeOnly) - GraphImpl.fromExistingRDDs(vD, graph.edges) + Graph(vD, graph.edges) .mapTriplets( e => e.attr / math.max(e.srcAttr, MLUtils.EPSILON), - TripletFields.Src) + new TripletFields(/* useSrc */ true, + /* useDst */ false, + /* useEdge */ true)) } /** @@ -276,7 +283,7 @@ object PowerIterationClustering extends Logging { : Graph[Double, Double] = { val edges = similarities.flatMap { case (i, j, s) => if (s < 0.0) { - throw new SparkException("Similarity must be nonnegative but found s($i, $j) = $s.") + throw new SparkException(s"Similarity must be nonnegative but found s($i, $j) = $s.") } if (i != j) { Seq(Edge(i, j, s), Edge(j, i, s)) @@ -291,10 +298,12 @@ object PowerIterationClustering extends Logging { }, mergeMsg = _ + _, TripletFields.EdgeOnly) - GraphImpl.fromExistingRDDs(vD, gA.edges) + Graph(vD, gA.edges) .mapTriplets( e => e.attr / math.max(e.srcAttr, MLUtils.EPSILON), - TripletFields.Src) + new TripletFields(/* useSrc */ true, + /* useDst */ false, + /* useEdge */ true)) } /** @@ -315,7 +324,7 @@ object PowerIterationClustering extends Logging { }, preservesPartitioning = true).cache() val sum = r.values.map(math.abs).sum() val v0 = r.mapValues(x => x / sum) - GraphImpl.fromExistingRDDs(VertexRDD(v0), g.edges) + Graph(VertexRDD(v0), g.edges) } /** @@ -330,7 +339,7 @@ object PowerIterationClustering extends Logging { def initDegreeVector(g: Graph[Double, Double]): Graph[Double, Double] = { val sum = g.vertices.values.sum() val v0 = g.vertices.mapValues(_ / sum) - GraphImpl.fromExistingRDDs(VertexRDD(v0), g.edges) + Graph(VertexRDD(v0), g.edges) } /** @@ -355,7 +364,9 @@ object PowerIterationClustering extends Logging { val v = curG.aggregateMessages[Double]( sendMsg = ctx => ctx.sendToSrc(ctx.attr * ctx.dstAttr), mergeMsg = _ + _, - TripletFields.Dst).cache() + new TripletFields(/* useSrc */ false, + /* useDst */ true, + /* useEdge */ true)).cache() // normalize v val norm = v.values.map(math.abs).sum() logInfo(s"$msgPrefix: norm(v) = $norm.") @@ -368,7 +379,7 @@ object PowerIterationClustering extends Logging { diffDelta = math.abs(delta - prevDelta) logInfo(s"$msgPrefix: diff(delta) = $diffDelta.") // update v - curG = GraphImpl.fromExistingRDDs(VertexRDD(v1), g.edges) + curG = Graph(VertexRDD(v1), g.edges) prevDelta = delta } curG.vertices @@ -385,7 +396,6 @@ object PowerIterationClustering extends Logging { val points = v.mapValues(x => Vectors.dense(x)).cache() val model = new KMeans() .setK(k) - .setRuns(5) .setSeed(0L) .run(points.values) points.mapValues(p => model.predict(p)).cache() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala index 80843719f50b..3ca75e8cdb97 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala @@ -19,12 +19,12 @@ package org.apache.spark.mllib.clustering import scala.reflect.ClassTag -import org.apache.spark.Logging import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaSparkContext._ +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaDStream} +import org.apache.spark.streaming.api.java.{JavaDStream, JavaPairDStream} import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom @@ -39,10 +39,14 @@ import org.apache.spark.util.random.XORShiftRandom * generalized to incorporate forgetfullness (i.e. decay). * The update rule (for each cluster) is: * - * {{{ - * c_t+1 = [(c_t * n_t * a) + (x_t * m_t)] / [n_t + m_t] - * n_t+t = n_t * a + m_t - * }}} + *
    + * $$ + * \begin{align} + * c_t+1 &= [(c_t * n_t * a) + (x_t * m_t)] / [n_t + m_t] \\ + * n_t+t &= n_t * a + m_t + * \end{align} + * $$ + *
    * * Where c_t is the previously estimated centroid for that cluster, * n_t is the number of points assigned to it thus far, x_t is the centroid @@ -135,13 +139,13 @@ class StreamingKMeansModel @Since("1.2.0") ( while (j < dim) { val x = largestClusterCenter(j) val p = 1e-14 * math.max(math.abs(x), 1.0) - largestClusterCenter.toBreeze(j) = x + p - smallestClusterCenter.toBreeze(j) = x - p + largestClusterCenter.asBreeze(j) = x + p + smallestClusterCenter.asBreeze(j) = x - p j += 1 } } - this + new StreamingKMeansModel(clusterCenters, clusterWeights) } } @@ -178,24 +182,32 @@ class StreamingKMeans @Since("1.2.0") ( */ @Since("1.2.0") def setK(k: Int): this.type = { + require(k > 0, + s"Number of clusters must be positive but got ${k}") this.k = k this } /** - * Set the decay factor directly (for forgetful algorithms). + * Set the forgetfulness of the previous centroids. */ @Since("1.2.0") def setDecayFactor(a: Double): this.type = { + require(a >= 0, + s"Decay factor must be nonnegative but got ${a}") this.decayFactor = a this } /** - * Set the half life and time unit ("batches" or "points") for forgetful algorithms. + * Set the half life and time unit ("batches" or "points"). If points, then the decay factor + * is raised to the power of number of new points and if batches, then decay factor will be + * used as is. */ @Since("1.2.0") def setHalfLife(halfLife: Double, timeUnit: String): this.type = { + require(halfLife > 0, + s"Half life must be positive but got ${halfLife}") if (timeUnit != StreamingKMeans.BATCHES && timeUnit != StreamingKMeans.POINTS) { throw new IllegalArgumentException("Invalid time unit for decay: " + timeUnit) } @@ -210,6 +222,12 @@ class StreamingKMeans @Since("1.2.0") ( */ @Since("1.2.0") def setInitialCenters(centers: Array[Vector], weights: Array[Double]): this.type = { + require(centers.size == weights.size, + "Number of initial centers must be equal to number of weights") + require(centers.size == k, + s"Number of initial centers must be ${k} but got ${centers.size}") + require(weights.forall(_ >= 0), + s"Weight for each inital center must be nonnegative but got [${weights.mkString(" ")}]") model = new StreamingKMeansModel(centers, weights) this } @@ -223,6 +241,10 @@ class StreamingKMeans @Since("1.2.0") ( */ @Since("1.2.0") def setRandomCenters(dim: Int, weight: Double, seed: Long = Utils.random.nextLong): this.type = { + require(dim > 0, + s"Number of dimensions must be positive but got ${dim}") + require(weight >= 0, + s"Weight for each center must be nonnegative but got ${weight}") val random = new XORShiftRandom(seed) val centers = Array.fill(k)(Vectors.dense(Array.fill(dim)(random.nextGaussian()))) val weights = Array.fill(k)(weight) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala index 078fbfbe4f0e..003d1411a9cf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala @@ -17,8 +17,8 @@ package org.apache.spark.mllib.evaluation -import org.apache.spark.rdd.RDD import org.apache.spark.mllib.rdd.RDDFunctions._ +import org.apache.spark.rdd.RDD /** * Computes the area under the curve (AUC) using the trapezoidal rule. @@ -39,7 +39,7 @@ private[evaluation] object AreaUnderCurve { /** * Returns the area under the given curve. * - * @param curve a RDD of ordered 2D points stored in pairs representing a curve + * @param curve an RDD of ordered 2D points stored in pairs representing a curve */ def of(curve: RDD[(Double, Double)]): Double = { curve.sliding(2).aggregate(0.0)( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala index 12cf22095720..2cfcf38eb4ca 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.evaluation import org.apache.spark.annotation.Since -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.mllib.evaluation.binary._ import org.apache.spark.rdd.{RDD, UnionRDD} import org.apache.spark.sql.DataFrame @@ -58,7 +58,7 @@ class BinaryClassificationMetrics @Since("1.3.0") ( * @param scoreAndLabels a DataFrame with two double columns: score and label */ private[mllib] def this(scoreAndLabels: DataFrame) = - this(scoreAndLabels.map(r => (r.getDouble(0), r.getDouble(1)))) + this(scoreAndLabels.rdd.map(r => (r.getDouble(0), r.getDouble(1)))) /** * Unpersist intermediate RDDs used in the computation. @@ -78,7 +78,8 @@ class BinaryClassificationMetrics @Since("1.3.0") ( * Returns the receiver operating characteristic (ROC) curve, * which is an RDD of (false positive rate, true positive rate) * with (0.0, 0.0) prepended and (1.0, 1.0) appended to it. - * @see http://en.wikipedia.org/wiki/Receiver_operating_characteristic + * @see + * Receiver operating characteristic (Wikipedia) */ @Since("1.0.0") def roc(): RDD[(Double, Double)] = { @@ -97,15 +98,16 @@ class BinaryClassificationMetrics @Since("1.3.0") ( /** * Returns the precision-recall curve, which is an RDD of (recall, precision), - * NOT (precision, recall), with (0.0, 1.0) prepended to it. - * @see http://en.wikipedia.org/wiki/Precision_and_recall + * NOT (precision, recall), with (0.0, p) prepended to it, where p is the precision + * associated with the lowest recall on the curve. + * @see + * Precision and recall (Wikipedia) */ @Since("1.0.0") def pr(): RDD[(Double, Double)] = { val prCurve = createCurve(Recall, Precision) - val sc = confusions.context - val first = sc.makeRDD(Seq((0.0, 1.0)), 1) - first.union(prCurve) + val (_, firstPrecision) = prCurve.first() + confusions.context.parallelize(Seq((0.0, firstPrecision)), 1).union(prCurve) } /** @@ -118,7 +120,7 @@ class BinaryClassificationMetrics @Since("1.3.0") ( * Returns the (threshold, F-Measure) curve. * @param beta the beta factor in F-Measure computation. * @return an RDD of (threshold, F-Measure) pairs. - * @see http://en.wikipedia.org/wiki/F1_score + * @see F1 score (Wikipedia) */ @Since("1.0.0") def fMeasureByThreshold(beta: Double): RDD[(Double, Double)] = createCurve(FMeasure(beta)) @@ -189,8 +191,7 @@ class BinaryClassificationMetrics @Since("1.3.0") ( Iterator(agg) }.collect() val partitionwiseCumulativeCounts = - agg.scanLeft(new BinaryLabelCounter())( - (agg: BinaryLabelCounter, c: BinaryLabelCounter) => agg.clone() += c) + agg.scanLeft(new BinaryLabelCounter())((agg, c) => agg.clone() += c) val totalCount = partitionwiseCumulativeCounts.last logInfo(s"Total counts: $totalCount") val cumulativeCounts = binnedCounts.mapPartitionsWithIndex( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala index c5104960cfcb..9a6a8dbdccbf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala @@ -25,7 +25,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame /** - * ::Experimental:: * Evaluator for multiclass classification. * * @param predictionAndLabels an RDD of (prediction, label) pairs. @@ -38,7 +37,7 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl * @param predictionAndLabels a DataFrame with two double columns: prediction and label */ private[mllib] def this(predictionAndLabels: DataFrame) = - this(predictionAndLabels.map(r => (r.getDouble(0), r.getDouble(1)))) + this(predictionAndLabels.rdd.map(r => (r.getDouble(0), r.getDouble(1)))) private lazy val labelCountByClass: Map[Double, Long] = predictionAndLabels.values.countByValue() private lazy val labelCount: Long = labelCountByClass.values.sum @@ -66,7 +65,7 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl */ @Since("1.1.0") def confusionMatrix: Matrix = { - val n = labels.size + val n = labels.length val values = Array.ofDim[Double](n * n) var i = 0 while (i < n) { @@ -139,7 +138,8 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl * Returns precision */ @Since("1.1.0") - lazy val precision: Double = tpByClass.values.sum.toDouble / labelCount + @deprecated("Use accuracy.", "2.0.0") + lazy val precision: Double = accuracy /** * Returns recall @@ -148,14 +148,24 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl * of all false negatives) */ @Since("1.1.0") - lazy val recall: Double = precision + @deprecated("Use accuracy.", "2.0.0") + lazy val recall: Double = accuracy /** * Returns f-measure * (equals to precision and recall because precision equals recall) */ @Since("1.1.0") - lazy val fMeasure: Double = precision + @deprecated("Use accuracy.", "2.0.0") + lazy val fMeasure: Double = accuracy + + /** + * Returns accuracy + * (equals to the total number of correctly classified instances + * out of the total number of instances.) + */ + @Since("2.0.0") + lazy val accuracy: Double = tpByClass.values.sum.toDouble / labelCount /** * Returns weighted true positive rate diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala index c100b3c9ec14..77bd0aa30dda 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala @@ -19,7 +19,6 @@ package org.apache.spark.mllib.evaluation import org.apache.spark.annotation.Since import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext._ import org.apache.spark.sql.DataFrame /** @@ -35,7 +34,9 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double] * @param predictionAndLabels a DataFrame with two double array columns: prediction and label */ private[mllib] def this(predictionAndLabels: DataFrame) = - this(predictionAndLabels.map(r => (r.getSeq[Double](0).toArray, r.getSeq[Double](1).toArray))) + this(predictionAndLabels.rdd.map { r => + (r.getSeq[Double](0).toArray, r.getSeq[Double](1).toArray) + }) private lazy val numDocs: Long = predictionAndLabels.count() @@ -56,8 +57,8 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double] */ @Since("1.2.0") lazy val accuracy: Double = predictionAndLabels.map { case (predictions, labels) => - labels.intersect(predictions).size.toDouble / - (labels.size + predictions.size - labels.intersect(predictions).size)}.sum / numDocs + labels.intersect(predictions).length.toDouble / + (labels.length + predictions.length - labels.intersect(predictions).length)}.sum / numDocs /** @@ -65,7 +66,7 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double] */ @Since("1.2.0") lazy val hammingLoss: Double = predictionAndLabels.map { case (predictions, labels) => - labels.size + predictions.size - 2 * labels.intersect(predictions).size + labels.length + predictions.length - 2 * labels.intersect(predictions).length }.sum / (numDocs * numLabels) /** @@ -73,8 +74,8 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double] */ @Since("1.2.0") lazy val precision: Double = predictionAndLabels.map { case (predictions, labels) => - if (predictions.size > 0) { - predictions.intersect(labels).size.toDouble / predictions.size + if (predictions.length > 0) { + predictions.intersect(labels).length.toDouble / predictions.length } else { 0 } @@ -85,7 +86,7 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double] */ @Since("1.2.0") lazy val recall: Double = predictionAndLabels.map { case (predictions, labels) => - labels.intersect(predictions).size.toDouble / labels.size + labels.intersect(predictions).length.toDouble / labels.length }.sum / numDocs /** @@ -93,7 +94,7 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double] */ @Since("1.2.0") lazy val f1Measure: Double = predictionAndLabels.map { case (predictions, labels) => - 2.0 * predictions.intersect(labels).size / (predictions.size + labels.size) + 2.0 * predictions.intersect(labels).length / (predictions.length + labels.length) }.sum / numDocs private lazy val tpPerClass = predictionAndLabels.flatMap { case (predictions, labels) => @@ -151,7 +152,7 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double] */ @Since("1.2.0") lazy val microPrecision: Double = { - val sumFp = fpPerClass.foldLeft(0L){ case(cum, (_, fp)) => cum + fp} + val sumFp = fpPerClass.foldLeft(0L) { case(cum, (_, fp)) => cum + fp} sumTp.toDouble / (sumTp + sumFp) } @@ -161,7 +162,7 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double] */ @Since("1.2.0") lazy val microRecall: Double = { - val sumFn = fnPerClass.foldLeft(0.0){ case(cum, (_, fn)) => cum + fn} + val sumFn = fnPerClass.foldLeft(0.0) { case(cum, (_, fn)) => cum + fn} sumTp.toDouble / (sumTp + sumFn) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala index cc01936dd34b..b98aa0534152 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala @@ -22,16 +22,15 @@ import java.{lang => jl} import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import org.apache.spark.Logging import org.apache.spark.annotation.Since -import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} +import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD /** - * ::Experimental:: * Evaluator for ranking algorithms. * - * Java users should use [[RankingMetrics$.of]] to create a [[RankingMetrics]] instance. + * Java users should use `RankingMetrics$.of` to create a [[RankingMetrics]] instance. * * @param predictionAndLabels an RDD of (predicted ranking, ground truth set) pairs. */ @@ -42,9 +41,9 @@ class RankingMetrics[T: ClassTag](predictionAndLabels: RDD[(Array[T], Array[T])] /** * Compute the average precision of all the queries, truncated at ranking position k. * - * If for a query, the ranking algorithm returns n (n < k) results, the precision value will be - * computed as #(relevant items retrieved) / k. This formula also applies when the size of the - * ground truth set is less than k. + * If for a query, the ranking algorithm returns n (n is less than k) results, the precision + * value will be computed as #(relevant items retrieved) / k. This formula also applies when + * the size of the ground truth set is less than k. * * If a query has an empty ground truth set, zero will be used as precision together with * a log warning. @@ -83,7 +82,7 @@ class RankingMetrics[T: ClassTag](predictionAndLabels: RDD[(Array[T], Array[T])] /** * Returns the mean average precision (MAP) of all the queries. * If a query has an empty ground truth set, the average precision will be zero and a log - * warining is generated. + * warning is generated. */ lazy val meanAveragePrecision: Double = { predictionAndLabels.map { case (pred, lab) => @@ -140,7 +139,7 @@ class RankingMetrics[T: ClassTag](predictionAndLabels: RDD[(Array[T], Array[T])] var i = 0 while (i < n) { val gain = 1.0 / math.log(i + 2) - if (labSet.contains(pred(i))) { + if (i < pred.length && labSet.contains(pred(i))) { dcg += gain } if (i < labSetSize) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala index 1d8f4fe340fb..020676cac5a6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala @@ -18,20 +18,27 @@ package org.apache.spark.mllib.evaluation import org.apache.spark.annotation.Since -import org.apache.spark.rdd.RDD -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, MultivariateOnlineSummarizer} +import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, MultivariateStatisticalSummary} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame /** * Evaluator for regression. * - * @param predictionAndObservations an RDD of (prediction, observation) pairs. + * @param predictionAndObservations an RDD of (prediction, observation) pairs + * @param throughOrigin True if the regression is through the origin. For example, in linear + * regression, it will be true without fitting intercept. */ @Since("1.2.0") -class RegressionMetrics @Since("1.2.0") ( - predictionAndObservations: RDD[(Double, Double)]) extends Logging { +class RegressionMetrics @Since("2.0.0") ( + predictionAndObservations: RDD[(Double, Double)], throughOrigin: Boolean) + extends Logging { + + @Since("1.2.0") + def this(predictionAndObservations: RDD[(Double, Double)]) = + this(predictionAndObservations, false) /** * An auxiliary constructor taking a DataFrame. @@ -39,7 +46,7 @@ class RegressionMetrics @Since("1.2.0") ( * prediction and observation */ private[mllib] def this(predictionAndObservations: DataFrame) = - this(predictionAndObservations.map(r => (r.getDouble(0), r.getDouble(1)))) + this(predictionAndObservations.rdd.map(r => (r.getDouble(0), r.getDouble(1)))) /** * Use MultivariateOnlineSummarizer to calculate summary statistics of observations and errors. @@ -47,12 +54,14 @@ class RegressionMetrics @Since("1.2.0") ( private lazy val summary: MultivariateStatisticalSummary = { val summary: MultivariateStatisticalSummary = predictionAndObservations.map { case (prediction, observation) => Vectors.dense(observation, observation - prediction) - }.aggregate(new MultivariateOnlineSummarizer())( + }.treeAggregate(new MultivariateOnlineSummarizer())( (summary, v) => summary.add(v), (sum1, sum2) => sum1.merge(sum2) ) summary } + + private lazy val SSy = math.pow(summary.normL2(0), 2) private lazy val SSerr = math.pow(summary.normL2(1), 2) private lazy val SStot = summary.variance(0) * (summary.count - 1) private lazy val SSreg = { @@ -64,8 +73,9 @@ class RegressionMetrics @Since("1.2.0") ( /** * Returns the variance explained by regression. - * explainedVariance = \sum_i (\hat{y_i} - \bar{y})^2 / n - * @see [[https://en.wikipedia.org/wiki/Fraction_of_variance_unexplained]] + * explainedVariance = $\sum_i (\hat{y_i} - \bar{y})^2^ / n$ + * @see + * Fraction of variance unexplained (Wikipedia) */ @Since("1.2.0") def explainedVariance: Double = { @@ -101,10 +111,18 @@ class RegressionMetrics @Since("1.2.0") ( /** * Returns R^2^, the unadjusted coefficient of determination. - * @see [[http://en.wikipedia.org/wiki/Coefficient_of_determination]] + * @see + * Coefficient of determination (Wikipedia) + * In case of regression through the origin, the definition of R^2^ is to be modified. + * @see + * J. G. Eisenhauer, Regression through the Origin. Teaching Statistics 25, 76-80 (2003) */ @Since("1.2.0") def r2: Double = { - 1 - SSerr / SStot + if (throughOrigin) { + 1 - SSerr / SSy + } else { + 1 - SSerr / SStot + } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala index be3319d60ce2..5a4c6aef50b7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala @@ -62,7 +62,7 @@ private[evaluation] object Recall extends BinaryClassificationMetricComputer { * F-Measure. Defined as 0 if both precision and recall are 0. EG in the case that all examples * are false positives. * @param beta the beta constant in F-Measure - * @see http://en.wikipedia.org/wiki/F1_score + * @see F1 score (Wikipedia) */ private[evaluation] case class FMeasure(beta: Double) extends BinaryClassificationMetricComputer { private val beta2 = beta * beta diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala index d4d022afde05..32f15552094e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala @@ -23,26 +23,27 @@ import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ +import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.stat.Statistics import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext -import org.apache.spark.sql.{SQLContext, Row} +import org.apache.spark.sql.{Row, SparkSession} /** * Chi Squared selector model. * - * @param selectedFeatures list of indices to select (filter). Must be ordered asc + * @param selectedFeatures list of indices to select (filter). */ @Since("1.3.0") class ChiSqSelectorModel @Since("1.3.0") ( @Since("1.3.0") val selectedFeatures: Array[Int]) extends VectorTransformer with Saveable { - require(isSorted(selectedFeatures), "Array has to be sorted asc") + private val filterIndices = selectedFeatures.sorted + @deprecated("not intended for subclasses to use", "2.1.0") protected def isSorted(array: Array[Int]): Boolean = { var i = 1 val len = array.length @@ -61,7 +62,7 @@ class ChiSqSelectorModel @Since("1.3.0") ( */ @Since("1.3.0") override def transform(vector: Vector): Vector = { - compress(vector, selectedFeatures) + compress(vector) } /** @@ -69,9 +70,8 @@ class ChiSqSelectorModel @Since("1.3.0") ( * Preserves the order of filtered features the same as their indices are stored. * Might be moved to Vector as .slice * @param features vector - * @param filterIndices indices of features to filter, must be ordered asc */ - private def compress(features: Vector, filterIndices: Array[Int]): Vector = { + private def compress(features: Vector): Vector = { features match { case SparseVector(size, indices, values) => val newSize = filterIndices.length @@ -134,8 +134,8 @@ object ChiSqSelectorModel extends Loader[ChiSqSelectorModel] { val thisClassName = "org.apache.spark.mllib.feature.ChiSqSelectorModel" def save(sc: SparkContext, model: ChiSqSelectorModel, path: String): Unit = { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) @@ -144,42 +144,106 @@ object ChiSqSelectorModel extends Loader[ChiSqSelectorModel] { val dataArray = Array.tabulate(model.selectedFeatures.length) { i => Data(model.selectedFeatures(i)) } - sc.parallelize(dataArray, 1).toDF().write.parquet(Loader.dataPath(path)) - + spark.createDataFrame(dataArray).repartition(1).write.parquet(Loader.dataPath(path)) } def load(sc: SparkContext, path: String): ChiSqSelectorModel = { implicit val formats = DefaultFormats - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) assert(className == thisClassName) assert(formatVersion == thisFormatVersion) - val dataFrame = sqlContext.read.parquet(Loader.dataPath(path)) + val dataFrame = spark.read.parquet(Loader.dataPath(path)) val dataArray = dataFrame.select("feature") // Check schema explicitly since erasure makes it hard to use match-case for checking. Loader.checkSchema[Data](dataFrame.schema) - val features = dataArray.map { - case Row(feature: Int) => (feature) + val features = dataArray.rdd.map { + case Row(feature: Int) => feature }.collect() - return new ChiSqSelectorModel(features) + new ChiSqSelectorModel(features) } } } /** * Creates a ChiSquared feature selector. - * @param numTopFeatures number of features that selector will select - * (ordered by statistic value descending) - * Note that if the number of features is < numTopFeatures, then this will - * select all features. + * The selector supports different selection methods: `numTopFeatures`, `percentile`, `fpr`, + * `fdr`, `fwe`. + * - `numTopFeatures` chooses a fixed number of top features according to a chi-squared test. + * - `percentile` is similar but chooses a fraction of all features instead of a fixed number. + * - `fpr` chooses all features whose p-values are below a threshold, thus controlling the false + * positive rate of selection. + * - `fdr` uses the [Benjamini-Hochberg procedure] + * (https://en.wikipedia.org/wiki/False_discovery_rate#Benjamini.E2.80.93Hochberg_procedure) + * to choose all features whose false discovery rate is below a threshold. + * - `fwe` chooses all features whose p-values are below a threshold. The threshold is scaled by + * 1/numFeatures, thus controlling the family-wise error rate of selection. + * By default, the selection method is `numTopFeatures`, with the default number of top features + * set to 50. */ @Since("1.3.0") -class ChiSqSelector @Since("1.3.0") ( - @Since("1.3.0") val numTopFeatures: Int) extends Serializable { +class ChiSqSelector @Since("2.1.0") () extends Serializable { + var numTopFeatures: Int = 50 + var percentile: Double = 0.1 + var fpr: Double = 0.05 + var fdr: Double = 0.05 + var fwe: Double = 0.05 + var selectorType = ChiSqSelector.NumTopFeatures + + /** + * The is the same to call this() and setNumTopFeatures(numTopFeatures) + */ + @Since("1.3.0") + def this(numTopFeatures: Int) { + this() + this.numTopFeatures = numTopFeatures + } + + @Since("1.6.0") + def setNumTopFeatures(value: Int): this.type = { + numTopFeatures = value + this + } + + @Since("2.1.0") + def setPercentile(value: Double): this.type = { + require(0.0 <= value && value <= 1.0, "Percentile must be in [0,1]") + percentile = value + this + } + + @Since("2.1.0") + def setFpr(value: Double): this.type = { + require(0.0 <= value && value <= 1.0, "FPR must be in [0,1]") + fpr = value + this + } + + @Since("2.2.0") + def setFdr(value: Double): this.type = { + require(0.0 <= value && value <= 1.0, "FDR must be in [0,1]") + fdr = value + this + } + + @Since("2.2.0") + def setFwe(value: Double): this.type = { + require(0.0 <= value && value <= 1.0, "FWE must be in [0,1]") + fwe = value + this + } + + @Since("2.1.0") + def setSelectorType(value: String): this.type = { + require(ChiSqSelector.supportedSelectorTypes.contains(value), + s"ChiSqSelector Type: $value was not supported.") + selectorType = value + this + } /** * Returns a ChiSquared feature selector. @@ -190,11 +254,60 @@ class ChiSqSelector @Since("1.3.0") ( */ @Since("1.3.0") def fit(data: RDD[LabeledPoint]): ChiSqSelectorModel = { - val indices = Statistics.chiSqTest(data) - .zipWithIndex.sortBy { case (res, _) => -res.statistic } - .take(numTopFeatures) - .map { case (_, indices) => indices } - .sorted + val chiSqTestResult = Statistics.chiSqTest(data).zipWithIndex + val features = selectorType match { + case ChiSqSelector.NumTopFeatures => + chiSqTestResult + .sortBy { case (res, _) => res.pValue } + .take(numTopFeatures) + case ChiSqSelector.Percentile => + chiSqTestResult + .sortBy { case (res, _) => res.pValue } + .take((chiSqTestResult.length * percentile).toInt) + case ChiSqSelector.FPR => + chiSqTestResult + .filter { case (res, _) => res.pValue < fpr } + case ChiSqSelector.FDR => + // This uses the Benjamini-Hochberg procedure. + // https://en.wikipedia.org/wiki/False_discovery_rate#Benjamini.E2.80.93Hochberg_procedure + val tempRes = chiSqTestResult + .sortBy { case (res, _) => res.pValue } + val maxIndex = tempRes + .zipWithIndex + .filter { case ((res, _), index) => + res.pValue <= fdr * (index + 1) / chiSqTestResult.length } + .map { case (_, index) => index } + .max + tempRes.take(maxIndex + 1) + case ChiSqSelector.FWE => + chiSqTestResult + .filter { case (res, _) => res.pValue < fwe / chiSqTestResult.length } + case errorType => + throw new IllegalStateException(s"Unknown ChiSqSelector Type: $errorType") + } + val indices = features.map { case (_, index) => index } new ChiSqSelectorModel(indices) } } + +private[spark] object ChiSqSelector { + + /** String name for `numTopFeatures` selector type. */ + private[spark] val NumTopFeatures: String = "numTopFeatures" + + /** String name for `percentile` selector type. */ + private[spark] val Percentile: String = "percentile" + + /** String name for `fpr` selector type. */ + private[spark] val FPR: String = "fpr" + + /** String name for `fdr` selector type. */ + private[spark] val FDR: String = "fdr" + + /** String name for `fwe` selector type. */ + private[spark] val FWE: String = "fwe" + + + /** Set of selector types that ChiSqSelector supports. */ + val supportedSelectorTypes: Array[String] = Array(NumTopFeatures, Percentile, FPR, FDR, FWE) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala index c93ed64183ad..9abdd44a635d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala @@ -22,10 +22,13 @@ import java.lang.{Iterable => JavaIterable} import scala.collection.JavaConverters._ import scala.collection.mutable +import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.rdd.RDD +import org.apache.spark.unsafe.hash.Murmur3_x86_32._ +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils /** @@ -36,16 +39,55 @@ import org.apache.spark.util.Utils @Since("1.1.0") class HashingTF(val numFeatures: Int) extends Serializable { + import HashingTF._ + + private var binary = false + private var hashAlgorithm = HashingTF.Murmur3 + /** */ @Since("1.1.0") def this() = this(1 << 20) + /** + * If true, term frequency vector will be binary such that non-zero term counts will be set to 1 + * (default: false) + */ + @Since("2.0.0") + def setBinary(value: Boolean): this.type = { + binary = value + this + } + + /** + * Set the hash algorithm used when mapping term to integer. + * (default: murmur3) + */ + @Since("2.0.0") + def setHashAlgorithm(value: String): this.type = { + hashAlgorithm = value + this + } + /** * Returns the index of the input term. */ @Since("1.1.0") - def indexOf(term: Any): Int = Utils.nonNegativeMod(term.##, numFeatures) + def indexOf(term: Any): Int = { + Utils.nonNegativeMod(getHashFunction(term), numFeatures) + } + + /** + * Get the hash function corresponding to the current [[hashAlgorithm]] setting. + */ + private def getHashFunction: Any => Int = hashAlgorithm match { + case Murmur3 => murmur3Hash + case Native => nativeHash + case _ => + // This should never happen. + throw new IllegalArgumentException( + s"HashingTF does not recognize hash algorithm $hashAlgorithm") + } /** * Transforms the input document into a sparse term frequency vector. @@ -53,9 +95,11 @@ class HashingTF(val numFeatures: Int) extends Serializable { @Since("1.1.0") def transform(document: Iterable[_]): Vector = { val termFrequencies = mutable.HashMap.empty[Int, Double] + val setTF = if (binary) (i: Int) => 1.0 else (i: Int) => termFrequencies.getOrElse(i, 0.0) + 1.0 + val hashFunc: Any => Int = getHashFunction document.foreach { term => - val i = indexOf(term) - termFrequencies.put(i, termFrequencies.getOrElse(i, 0.0) + 1.0) + val i = Utils.nonNegativeMod(hashFunc(term), numFeatures) + termFrequencies.put(i, setTF(i)) } Vectors.sparse(numFeatures, termFrequencies.toSeq) } @@ -84,3 +128,41 @@ class HashingTF(val numFeatures: Int) extends Serializable { dataset.rdd.map(this.transform).toJavaRDD() } } + +object HashingTF { + + private[HashingTF] val Native: String = "native" + + private[HashingTF] val Murmur3: String = "murmur3" + + private val seed = 42 + + /** + * Calculate a hash code value for the term object using the native Scala implementation. + * This is the default hash algorithm used in Spark 1.6 and earlier. + */ + private[HashingTF] def nativeHash(term: Any): Int = term.## + + /** + * Calculate a hash code value for the term object using + * Austin Appleby's MurmurHash 3 algorithm (MurmurHash3_x86_32). + * This is the default hash algorithm used from Spark 2.0 onwards. + */ + private[spark] def murmur3Hash(term: Any): Int = { + term match { + case null => seed + case b: Boolean => hashInt(if (b) 1 else 0, seed) + case b: Byte => hashInt(b, seed) + case s: Short => hashInt(s, seed) + case i: Int => hashInt(i, seed) + case l: Long => hashLong(l, seed) + case f: Float => hashInt(java.lang.Float.floatToIntBits(f), seed) + case d: Double => hashLong(java.lang.Double.doubleToLongBits(d), seed) + case s: String => + val utf8 = UTF8String.fromString(s) + hashUnsafeBytes(utf8.getBaseObject, utf8.getBaseOffset, utf8.numBytes(), seed) + case _ => throw new SparkException("HashingTF with murmur3 algorithm does not " + + s"support type ${term.getClass.getCanonicalName} of input data.") + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala index cffa9fba05c8..bb4b37ef21a8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala @@ -88,7 +88,7 @@ private object IDF { } doc match { case SparseVector(size, indices, values) => - val nnz = indices.size + val nnz = indices.length var k = 0 while (k < nnz) { if (values(k) > 0) { @@ -97,7 +97,7 @@ private object IDF { k += 1 } case DenseVector(values) => - val n = values.size + val n = values.length var j = 0 while (j < n) { if (values(j) > 0.0) { @@ -204,14 +204,14 @@ private object IDFModel { * Transforms a term frequency (TF) vector to a TF-IDF vector with a IDF vector * * @param idf an IDF vector - * @param v a term frequence vector + * @param v a term frequency vector * @return a TF-IDF vector */ def transform(idf: Vector, v: Vector): Vector = { val n = v.size v match { case SparseVector(size, indices, values) => - val nnz = indices.size + val nnz = indices.length val newValues = new Array[Double](nnz) var k = 0 while (k < nnz) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala index af0c8e1d8a9d..99fcb36f27e3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala @@ -55,7 +55,7 @@ class Normalizer @Since("1.1.0") (p: Double) extends VectorTransformer { vector match { case DenseVector(vs) => val values = vs.clone() - val size = values.size + val size = values.length var i = 0 while (i < size) { values(i) /= norm @@ -64,7 +64,7 @@ class Normalizer @Since("1.1.0") (p: Double) extends VectorTransformer { Vectors.dense(values) case SparseVector(size, ids, vs) => val values = vs.clone() - val nnz = values.size + val nnz = values.length var i = 0 while (i < nnz) { values(i) /= norm diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala index ecb3c1e6c1c8..a01503f4b80a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.feature -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.linalg.distributed.RowMatrix @@ -30,7 +30,8 @@ import org.apache.spark.rdd.RDD */ @Since("1.4.0") class PCA @Since("1.4.0") (@Since("1.4.0") val k: Int) { - require(k >= 1, s"PCA requires a number of principal components k >= 1 but was given $k") + require(k > 0, + s"Number of principal components must be positive but got ${k}") /** * Computes a [[PCAModel]] that contains the principal components of the input vectors. @@ -39,11 +40,18 @@ class PCA @Since("1.4.0") (@Since("1.4.0") val k: Int) { */ @Since("1.4.0") def fit(sources: RDD[Vector]): PCAModel = { - require(k <= sources.first().size, - s"source vector size is ${sources.first().size} must be greater than k=$k") + val numFeatures = sources.first().size + require(k <= numFeatures, + s"source vector size $numFeatures must be no less than k=$k") + + require(PCAUtil.memoryCost(k, numFeatures) < Int.MaxValue, + "The param k and numFeatures is too large for SVD computation. " + + "Try reducing the parameter k for PCA, or reduce the input feature " + + "vector dimension to make this tractable.") val mat = new RowMatrix(sources) - val pc = mat.computePrincipalComponents(k) match { + val (pc, explainedVariance) = mat.computePrincipalComponentsAndExplainedVariance(k) + val densePC = pc match { case dm: DenseMatrix => dm case sm: SparseMatrix => @@ -56,13 +64,18 @@ class PCA @Since("1.4.0") (@Since("1.4.0") val k: Int) { case m => throw new IllegalArgumentException("Unsupported matrix format. Expected " + s"SparseMatrix or DenseMatrix. Instead got: ${m.getClass}") - } - new PCAModel(k, pc) + val denseExplainedVariance = explainedVariance match { + case dv: DenseVector => + dv + case sv: SparseVector => + sv.toDense + } + new PCAModel(k, densePC, denseExplainedVariance) } /** - * Java-friendly version of [[fit()]] + * Java-friendly version of `fit()`. */ @Since("1.4.0") def fit(sources: JavaRDD[Vector]): PCAModel = fit(sources.rdd) @@ -77,12 +90,13 @@ class PCA @Since("1.4.0") (@Since("1.4.0") val k: Int) { @Since("1.4.0") class PCAModel private[spark] ( @Since("1.4.0") val k: Int, - @Since("1.4.0") val pc: DenseMatrix) extends VectorTransformer { + @Since("1.4.0") val pc: DenseMatrix, + @Since("1.6.0") val explainedVariance: DenseVector) extends VectorTransformer { /** * Transform a vector by computed Principal Components. * * @param vector vector to be transformed. - * Vector must be the same length as the source vectors given to [[PCA.fit()]]. + * Vector must be the same length as the source vectors given to `PCA.fit()`. * @return transformed vector. Vector will be of length k. */ @Since("1.4.0") @@ -101,3 +115,17 @@ class PCAModel private[spark] ( } } } + +private[feature] object PCAUtil { + + // This memory cost formula is from breeze code: + // https://github.com/scalanlp/breeze/blob/ + // 6e541be066d547a097f5089165cd7c38c3ca276d/math/src/main/scala/breeze/linalg/ + // functions/svd.scala#L87 + def memoryCost(k: Int, numFeatures: Int): Long = { + 3L * math.min(k, numFeatures) * math.min(k, numFeatures) + + math.max(math.max(k, numFeatures), 4L * math.min(k, numFeatures) + * math.min(k, numFeatures) + 4L * math.min(k, numFeatures)) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala index 6fe573c52894..7667936a3f85 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -17,8 +17,8 @@ package org.apache.spark.mllib.feature -import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.rdd.RDD @@ -27,8 +27,12 @@ import org.apache.spark.rdd.RDD * Standardizes features by removing the mean and scaling to unit std using column summary * statistics on the samples in the training set. * + * The "unit std" is computed using the corrected sample standard deviation + * (https://en.wikipedia.org/wiki/Standard_deviation#Corrected_sample_standard_deviation), + * which is computed as the square root of the unbiased sample variance. + * * @param withMean False by default. Centers the data with mean before scaling. It will build a - * dense output, so this does not work on sparse input and will raise an exception. + * dense output, so take care when applying to sparse input. * @param withStd True by default. Scales the data to unit standard deviation. */ @Since("1.1.0") @@ -92,6 +96,9 @@ class StandardScalerModel @Since("1.3.0") ( @Since("1.3.0") def this(std: Vector) = this(std, null) + /** + * :: DeveloperApi :: + */ @Since("1.3.0") @DeveloperApi def setWithMean(withMean: Boolean): this.type = { @@ -100,6 +107,9 @@ class StandardScalerModel @Since("1.3.0") ( this } + /** + * :: DeveloperApi :: + */ @Since("1.3.0") @DeveloperApi def setWithStd(withStd: Boolean): this.type = { @@ -129,31 +139,32 @@ class StandardScalerModel @Since("1.3.0") ( // the member variables are accessed, `invokespecial` will be called which is expensive. // This can be avoid by having a local reference of `shift`. val localShift = shift - vector match { - case DenseVector(vs) => - val values = vs.clone() - val size = values.size - if (withStd) { - var i = 0 - while (i < size) { - values(i) = if (std(i) != 0.0) (values(i) - localShift(i)) * (1.0 / std(i)) else 0.0 - i += 1 - } - } else { - var i = 0 - while (i < size) { - values(i) -= localShift(i) - i += 1 - } - } - Vectors.dense(values) - case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + // Must have a copy of the values since it will be modified in place + val values = vector match { + // specially handle DenseVector because its toArray does not clone already + case d: DenseVector => d.values.clone() + case v: Vector => v.toArray + } + val size = values.length + if (withStd) { + var i = 0 + while (i < size) { + values(i) = if (std(i) != 0.0) (values(i) - localShift(i)) * (1.0 / std(i)) else 0.0 + i += 1 + } + } else { + var i = 0 + while (i < size) { + values(i) -= localShift(i) + i += 1 + } } + Vectors.dense(values) } else if (withStd) { vector match { case DenseVector(vs) => val values = vs.clone() - val size = values.size + val size = values.length var i = 0 while(i < size) { values(i) *= (if (std(i) != 0.0) 1.0 / std(i) else 0.0) @@ -164,7 +175,7 @@ class StandardScalerModel @Since("1.3.0") ( // For sparse vector, the `index` array inside sparse vector object will not be changed, // so we can re-use it to save memory. val values = vs.clone() - val nnz = values.size + val nnz = values.length var i = 0 while (i < nnz) { values(i) *= (if (std(indices(i)) != 0.0) 1.0 / std(indices(i)) else 0.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala index 5778fd1d0925..9db725097ae9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala @@ -47,13 +47,13 @@ trait VectorTransformer extends Serializable { */ @Since("1.1.0") def transform(data: RDD[Vector]): RDD[Vector] = { - // Later in #1498 , all RDD objects are sent via broadcasting instead of akka. + // Later in #1498 , all RDD objects are sent via broadcasting instead of RPC. // So it should be no longer necessary to explicitly broadcast `this` object. data.map(x => this.transform(x)) } /** - * Applies transformation on an JavaRDD[Vector]. + * Applies transformation on a JavaRDD[Vector]. * * @param data JavaRDD[Vector] to be transformed. * @return transformed JavaRDD[Vector]. 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 f3e4d346e358..6f96813497b6 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,24 +21,24 @@ import java.lang.{Iterable => JavaIterable} import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.collection.mutable.ArrayBuilder import com.github.fommil.netlib.BLAS.{getInstance => blas} - import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.Logging import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd._ +import org.apache.spark.sql.SparkSession +import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom -import org.apache.spark.sql.SQLContext /** * Entry in vocabulary @@ -77,12 +77,28 @@ class Word2Vec extends Serializable with Logging { private var numIterations = 1 private var seed = Utils.random.nextLong() private var minCount = 5 + private var maxSentenceLength = 1000 + + /** + * Sets the maximum length (in words) of each sentence in the input data. + * Any sentence longer than this threshold will be divided into chunks of + * up to `maxSentenceLength` size (default: 1000) + */ + @Since("2.0.0") + def setMaxSentenceLength(maxSentenceLength: Int): this.type = { + require(maxSentenceLength > 0, + s"Maximum length of sentences must be positive but got ${maxSentenceLength}") + this.maxSentenceLength = maxSentenceLength + this + } /** * Sets vector size (default: 100). */ @Since("1.1.0") def setVectorSize(vectorSize: Int): this.type = { + require(vectorSize > 0, + s"vector size must be positive but got ${vectorSize}") this.vectorSize = vectorSize this } @@ -92,6 +108,8 @@ class Word2Vec extends Serializable with Logging { */ @Since("1.1.0") def setLearningRate(learningRate: Double): this.type = { + require(learningRate > 0, + s"Initial learning rate must be positive but got ${learningRate}") this.learningRate = learningRate this } @@ -101,7 +119,8 @@ class Word2Vec extends Serializable with Logging { */ @Since("1.1.0") def setNumPartitions(numPartitions: Int): this.type = { - require(numPartitions > 0, s"numPartitions must be greater than 0 but got $numPartitions") + require(numPartitions > 0, + s"Number of partitions must be positive but got ${numPartitions}") this.numPartitions = numPartitions this } @@ -112,6 +131,8 @@ class Word2Vec extends Serializable with Logging { */ @Since("1.1.0") def setNumIterations(numIterations: Int): this.type = { + require(numIterations >= 0, + s"Number of iterations must be nonnegative but got ${numIterations}") this.numIterations = numIterations this } @@ -125,12 +146,25 @@ class Word2Vec extends Serializable with Logging { this } + /** + * Sets the window of words (default: 5) + */ + @Since("1.6.0") + def setWindowSize(window: Int): this.type = { + require(window > 0, + s"Window of words must be positive but got ${window}") + this.window = window + this + } + /** * Sets minCount, the minimum number of times a token must appear to be included in the word2vec * model's vocabulary (default: 5). */ @Since("1.3.0") def setMinCount(minCount: Int): this.type = { + require(minCount >= 0, + s"Minimum number of times must be nonnegative but got ${minCount}") this.minCount = minCount this } @@ -138,26 +172,27 @@ class Word2Vec extends Serializable with Logging { private val EXP_TABLE_SIZE = 1000 private val MAX_EXP = 6 private val MAX_CODE_LENGTH = 40 - private val MAX_SENTENCE_LENGTH = 1000 /** context words from [-window, window] */ - private val window = 5 + private var window = 5 - private var trainWordsCount = 0 + private var trainWordsCount = 0L private var vocabSize = 0 - private var vocab: Array[VocabWord] = null - private var vocabHash = mutable.HashMap.empty[String, Int] + @transient private var vocab: Array[VocabWord] = null + @transient private var vocabHash = mutable.HashMap.empty[String, Int] + + private def learnVocab[S <: Iterable[String]](dataset: RDD[S]): Unit = { + val words = dataset.flatMap(x => x) - private def learnVocab(words: RDD[String]): Unit = { vocab = words.map(w => (w, 1)) .reduceByKey(_ + _) + .filter(_._2 >= minCount) .map(x => VocabWord( x._1, x._2, new Array[Int](MAX_CODE_LENGTH), new Array[Int](MAX_CODE_LENGTH), 0)) - .filter(_.cn >= minCount) .collect() .sortWith((a, b) => a.cn > b.cn) @@ -171,7 +206,7 @@ class Word2Vec extends Serializable with Logging { trainWordsCount += vocab(a).cn a += 1 } - logInfo("trainWordsCount = " + trainWordsCount) + logInfo(s"vocabSize = $vocabSize, trainWordsCount = $trainWordsCount") } private def createExpTable(): Array[Float] = { @@ -264,15 +299,14 @@ class Word2Vec extends Serializable with Logging { /** * Computes the vector representation of each word in vocabulary. - * @param dataset an RDD of words + * @param dataset an RDD of sentences, + * each sentence is expressed as an iterable collection of words * @return a Word2VecModel */ @Since("1.1.0") def fit[S <: Iterable[String]](dataset: RDD[S]): Word2VecModel = { - val words = dataset.flatMap(x => x) - - learnVocab(words) + learnVocab(dataset) createBinaryTree() @@ -281,47 +315,54 @@ class Word2Vec extends Serializable with Logging { val expTable = sc.broadcast(createExpTable()) val bcVocab = sc.broadcast(vocab) val bcVocabHash = sc.broadcast(vocabHash) + try { + doFit(dataset, sc, expTable, bcVocab, bcVocabHash) + } finally { + expTable.destroy(blocking = false) + bcVocab.destroy(blocking = false) + bcVocabHash.destroy(blocking = false) + } + } - val sentences: RDD[Array[Int]] = words.mapPartitions { iter => - new Iterator[Array[Int]] { - def hasNext: Boolean = iter.hasNext - - def next(): Array[Int] = { - val sentence = ArrayBuilder.make[Int] - var sentenceLength = 0 - while (iter.hasNext && sentenceLength < MAX_SENTENCE_LENGTH) { - val word = bcVocabHash.value.get(iter.next()) - word match { - case Some(w) => - sentence += w - sentenceLength += 1 - case None => - } - } - sentence.result() - } + private def doFit[S <: Iterable[String]]( + dataset: RDD[S], sc: SparkContext, + expTable: Broadcast[Array[Float]], + bcVocab: Broadcast[Array[VocabWord]], + bcVocabHash: Broadcast[mutable.HashMap[String, Int]]) = { + // each partition is a collection of sentences, + // will be translated into arrays of Index integer + val sentences: RDD[Array[Int]] = dataset.mapPartitions { sentenceIter => + // Each sentence will map to 0 or more Array[Int] + sentenceIter.flatMap { sentence => + // Sentence of words, some of which map to a word index + val wordIndexes = sentence.flatMap(bcVocabHash.value.get) + // break wordIndexes into trunks of maxSentenceLength when has more + wordIndexes.grouped(maxSentenceLength).map(_.toArray) } } val newSentences = sentences.repartition(numPartitions).cache() val initRandom = new XORShiftRandom(seed) - if (vocabSize.toLong * vectorSize * 8 >= Int.MaxValue) { + if (vocabSize.toLong * vectorSize >= Int.MaxValue) { throw new RuntimeException("Please increase minCount or decrease vectorSize in Word2Vec" + " to avoid an OOM. You are highly recommended to make your vocabSize*vectorSize, " + - "which is " + vocabSize + "*" + vectorSize + " for now, less than `Int.MaxValue/8`.") + "which is " + vocabSize + "*" + vectorSize + " for now, less than `Int.MaxValue`.") } val syn0Global = Array.fill[Float](vocabSize * vectorSize)((initRandom.nextFloat() - 0.5f) / vectorSize) val syn1Global = new Array[Float](vocabSize * vectorSize) var alpha = learningRate + for (k <- 1 to numIterations) { + val bcSyn0Global = sc.broadcast(syn0Global) + val bcSyn1Global = sc.broadcast(syn1Global) val partial = newSentences.mapPartitionsWithIndex { case (idx, iter) => val random = new XORShiftRandom(seed ^ ((idx + 1) << 16) ^ ((-k - 1) << 8)) val syn0Modify = new Array[Int](vocabSize) val syn1Modify = new Array[Int](vocabSize) - val model = iter.foldLeft((syn0Global, syn1Global, 0, 0)) { + val model = iter.foldLeft((bcSyn0Global.value, bcSyn1Global.value, 0L, 0L)) { case ((syn0, syn1, lastWordCount, wordCount), sentence) => var lwc = lastWordCount var wc = wordCount @@ -333,9 +374,9 @@ class Word2Vec extends Serializable with Logging { if (alpha < learningRate * 0.0001) alpha = learningRate * 0.0001 logInfo("wordCount = " + wordCount + ", alpha = " + alpha) } - wc += sentence.size + wc += sentence.length var pos = 0 - while (pos < sentence.size) { + while (pos < sentence.length) { val word = sentence(pos) val b = random.nextInt(window) // Train Skip-gram @@ -343,7 +384,7 @@ class Word2Vec extends Serializable with Logging { while (a < window * 2 + 1 - b) { if (a != window) { val c = pos - window + a - if (c >= 0 && c < sentence.size) { + if (c >= 0 && c < sentence.length) { val lastWord = sentence(c) val l1 = lastWord * vectorSize val neu1e = new Array[Float](vectorSize) @@ -405,6 +446,8 @@ class Word2Vec extends Serializable with Logging { } i += 1 } + bcSyn0Global.destroy(false) + bcSyn1Global.destroy(false) } newSentences.unpersist() @@ -432,9 +475,9 @@ class Word2Vec extends Serializable with Logging { * (i * vectorSize, i * vectorSize + vectorSize) */ @Since("1.1.0") -class Word2VecModel private[mllib] ( - private val wordIndex: Map[String, Int], - private val wordVectors: Array[Float]) extends Serializable with Saveable { +class Word2VecModel private[spark] ( + private[spark] val wordIndex: Map[String, Int], + private[spark] val wordVectors: Array[Float]) extends Serializable with Saveable { private val numWords = wordIndex.size // vectorSize: Dimension of each word's vector. @@ -448,8 +491,8 @@ class Word2VecModel private[mllib] ( // wordVecNorms: Array of length numWords, each value being the Euclidean norm // of the wordVector. - private val wordVecNorms: Array[Double] = { - val wordVecNorms = new Array[Double](numWords) + private val wordVecNorms: Array[Float] = { + val wordVecNorms = new Array[Float](numWords) var i = 0 while (i < numWords) { val vec = wordVectors.slice(i * vectorSize, i * vectorSize + vectorSize) @@ -464,15 +507,6 @@ class Word2VecModel private[mllib] ( this(Word2VecModel.buildWordIndex(model), Word2VecModel.buildWordVectors(model)) } - private def cosineSimilarity(v1: Array[Float], v2: Array[Float]): Double = { - require(v1.length == v2.length, "Vectors should have the same length") - val n = v1.length - val norm1 = blas.snrm2(n, v1, 1) - val norm2 = blas.snrm2(n, v2, 1) - if (norm1 == 0 || norm2 == 0) return 0.0 - blas.sdot(n, v1, 1, v2, 1) / norm1 / norm2 - } - override protected def formatVersion = "1.0" @Since("1.4.0") @@ -497,7 +531,7 @@ class Word2VecModel private[mllib] ( } /** - * Find synonyms of a word + * Find synonyms of a word; do not include the word itself in results. * @param word a word * @param num number of synonyms to find * @return array of (word, cosineSimilarity) @@ -505,39 +539,77 @@ class Word2VecModel private[mllib] ( @Since("1.1.0") def findSynonyms(word: String, num: Int): Array[(String, Double)] = { val vector = transform(word) - findSynonyms(vector, num) + findSynonyms(vector, num, Some(word)) } /** - * Find synonyms of the vector representation of a word + * Find synonyms of the vector representation of a word, possibly + * including any words in the model vocabulary whose vector respresentation + * is the supplied vector. * @param vector vector representation of a word * @param num number of synonyms to find * @return array of (word, cosineSimilarity) */ @Since("1.1.0") def findSynonyms(vector: Vector, num: Int): Array[(String, Double)] = { + findSynonyms(vector, num, None) + } + + /** + * Find synonyms of the vector representation of a word, rejecting + * words identical to the value of wordOpt, if one is supplied. + * @param vector vector representation of a word + * @param num number of synonyms to find + * @param wordOpt optionally, a word to reject from the results list + * @return array of (word, cosineSimilarity) + */ + private def findSynonyms( + vector: Vector, + num: Int, + wordOpt: Option[String]): Array[(String, Double)] = { require(num > 0, "Number of similar words should > 0") - // TODO: optimize top-k + val fVector = vector.toArray.map(_.toFloat) - val cosineVec = Array.fill[Float](numWords)(0) + val cosineVec = new Array[Float](numWords) val alpha: Float = 1 val beta: Float = 0 - + // Normalize input vector before blas.sgemv to avoid Inf value + val vecNorm = blas.snrm2(vectorSize, fVector, 1) + if (vecNorm != 0.0f) { + blas.sscal(vectorSize, 1 / vecNorm, fVector, 0, 1) + } blas.sgemv( "T", vectorSize, numWords, alpha, wordVectors, vectorSize, fVector, 1, beta, cosineVec, 1) - // Need not divide with the norm of the given vector since it is constant. - val cosVec = cosineVec.map(_.toDouble) - var ind = 0 - while (ind < numWords) { - cosVec(ind) /= wordVecNorms(ind) - ind += 1 + var i = 0 + while (i < numWords) { + val norm = wordVecNorms(i) + if (norm == 0.0f) { + cosineVec(i) = 0.0f + } else { + cosineVec(i) /= norm + } + i += 1 + } + + val pq = new BoundedPriorityQueue[(String, Float)](num + 1)(Ordering.by(_._2)) + + var j = 0 + while (j < numWords) { + pq += Tuple2(wordList(j), cosineVec(j)) + j += 1 } - wordList.zip(cosVec) - .toSeq - .sortBy(- _._2) - .take(num + 1) - .tail + + val scored = pq.toSeq.sortBy(-_._2) + + val filtered = wordOpt match { + case Some(w) => scored.filter(tup => w != tup._1) + case None => scored + } + + filtered + .take(num) + .map { case (word, score) => (word, score.toDouble) } .toArray } @@ -550,6 +622,7 @@ class Word2VecModel private[mllib] ( (word, wordVectors.slice(vectorSize * ind, vectorSize * ind + vectorSize)) } } + } @Since("1.4.0") @@ -561,7 +634,7 @@ object Word2VecModel extends Loader[Word2VecModel] { private def buildWordVectors(model: Map[String, Array[Float]]): Array[Float] = { require(model.nonEmpty, "Word2VecMap should be non-empty") - val (vectorSize, numWords) = (model.head._2.size, model.size) + val (vectorSize, numWords) = (model.head._2.length, model.size) val wordList = model.keys.toArray val wordVectors = new Array[Float](vectorSize * numWords) var i = 0 @@ -581,9 +654,8 @@ object Word2VecModel extends Loader[Word2VecModel] { case class Data(word: String, vector: Array[Float]) def load(sc: SparkContext, path: String): Word2VecModel = { - val dataPath = Loader.dataPath(path) - val sqlContext = new SQLContext(sc) - val dataFrame = sqlContext.read.parquet(dataPath) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val dataFrame = spark.read.parquet(Loader.dataPath(path)) // Check schema explicitly since erasure makes it hard to use match-case for checking. Loader.checkSchema[Data](dataFrame.schema) @@ -593,19 +665,27 @@ object Word2VecModel extends Loader[Word2VecModel] { } def save(sc: SparkContext, path: String, model: Map[String, Array[Float]]): Unit = { + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ - - val vectorSize = model.values.head.size + val vectorSize = model.values.head.length val numWords = model.size - val metadata = compact(render - (("class" -> classNameV1_0) ~ ("version" -> formatVersionV1_0) ~ - ("vectorSize" -> vectorSize) ~ ("numWords" -> numWords))) + val metadata = compact(render( + ("class" -> classNameV1_0) ~ ("version" -> formatVersionV1_0) ~ + ("vectorSize" -> vectorSize) ~ ("numWords" -> numWords))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) + // We want to partition the model in partitions smaller than + // spark.kryoserializer.buffer.max + val bufferSize = Utils.byteStringAsBytes( + spark.conf.get("spark.kryoserializer.buffer.max", "64m")) + // We calculate the approximate size of the model + // We only calculate the array size, considering an + // average string size of 15 bytes, the formula is: + // (floatSize * vectorSize + 15) * numWords + val approxSize = (4L * vectorSize + 15) * numWords + val nPartitions = ((approxSize / bufferSize) + 1).toInt val dataArray = model.toSeq.map { case (w, v) => Data(w, v) } - sc.parallelize(dataArray.toSeq, 1).toDF().write.parquet(Loader.dataPath(path)) + spark.createDataFrame(dataArray).repartition(nPartitions).write.parquet(Loader.dataPath(path)) } } @@ -620,7 +700,7 @@ object Word2VecModel extends Loader[Word2VecModel] { (loadedClassName, loadedVersion) match { case (classNameV1_0, "1.0") => val model = SaveLoadV1_0.load(sc, path) - val vectorSize = model.getVectors.values.head.size + val vectorSize = model.getVectors.values.head.length val numWords = model.getVectors.size require(expectedVectorSize == vectorSize, s"Word2VecModel requires each word to be mapped to a vector of size " + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index 07eb750b06a3..acb83ac31aff 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -19,23 +19,20 @@ package org.apache.spark.mllib.fpm import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import org.apache.spark.Logging -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.internal.Logging import org.apache.spark.mllib.fpm.AssociationRules.Rule import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset import org.apache.spark.rdd.RDD /** - * :: Experimental :: - * - * Generates association rules from a [[RDD[FreqItemset[Item]]]. This method only generates + * Generates association rules from a `RDD[FreqItemset[Item]]`. This method only generates * association rules which have a single item as the consequent. * */ @Since("1.5.0") -@Experimental class AssociationRules private[fpm] ( private var minConfidence: Double) extends Logging with Serializable { @@ -50,15 +47,16 @@ class AssociationRules private[fpm] ( */ @Since("1.5.0") def setMinConfidence(minConfidence: Double): this.type = { - require(minConfidence >= 0.0 && minConfidence <= 1.0) + require(minConfidence >= 0.0 && minConfidence <= 1.0, + s"Minimal confidence must be in range [0, 1] but got ${minConfidence}") this.minConfidence = minConfidence this } /** - * Computes the association rules with confidence above [[minConfidence]]. + * Computes the association rules with confidence above `minConfidence`. * @param freqItemsets frequent itemset model obtained from [[FPGrowth]] - * @return a [[Set[Rule[Item]]] containing the assocation rules. + * @return a `Set[Rule[Item]]` containing the association rules. * */ @Since("1.5.0") @@ -82,7 +80,9 @@ class AssociationRules private[fpm] ( }.filter(_.confidence >= minConfidence) } - /** Java-friendly version of [[run]]. */ + /** + * Java-friendly version of `run`. + */ @Since("1.5.0") def run[Item](freqItemsets: JavaRDD[FreqItemset[Item]]): JavaRDD[Rule[Item]] = { val tag = fakeClassTag[Item] @@ -94,8 +94,6 @@ class AssociationRules private[fpm] ( object AssociationRules { /** - * :: Experimental :: - * * An association rule between sets of items. * @param antecedent hypotheses of the rule. Java users should call [[Rule#javaAntecedent]] * instead. @@ -105,7 +103,6 @@ object AssociationRules { * */ @Since("1.5.0") - @Experimental class Rule[Item] private[fpm] ( @Since("1.5.0") val antecedent: Array[Item], @Since("1.5.0") val consequent: Array[Item], diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index 70ef1ed30c71..f6b1143272d1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -20,28 +20,39 @@ package org.apache.spark.mllib.fpm import java.{util => ju} import java.lang.{Iterable => JavaIterable} -import scala.collection.mutable import scala.collection.JavaConverters._ +import scala.collection.mutable import scala.reflect.ClassTag +import scala.reflect.runtime.universe._ + +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods.{compact, render} -import org.apache.spark.{HashPartitioner, Logging, Partitioner, SparkException} +import org.apache.spark.{HashPartitioner, Partitioner, SparkContext, SparkException} import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.internal.Logging import org.apache.spark.mllib.fpm.FPGrowth._ +import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel /** * Model trained by [[FPGrowth]], which holds frequent itemsets. - * @param freqItemsets frequent itemset, which is an RDD of [[FreqItemset]] + * @param freqItemsets frequent itemset, which is an RDD of `FreqItemset` * @tparam Item item type */ @Since("1.3.0") class FPGrowthModel[Item: ClassTag] @Since("1.3.0") ( - @Since("1.3.0") val freqItemsets: RDD[FreqItemset[Item]]) extends Serializable { + @Since("1.3.0") val freqItemsets: RDD[FreqItemset[Item]]) + extends Saveable with Serializable { /** - * Generates association rules for the [[Item]]s in [[freqItemsets]]. + * Generates association rules for the `Item`s in [[freqItemsets]]. * @param confidence minimal confidence of the rules produced */ @Since("1.5.0") @@ -49,22 +60,105 @@ class FPGrowthModel[Item: ClassTag] @Since("1.3.0") ( val associationRules = new AssociationRules(confidence) associationRules.run(freqItemsets) } + + /** + * Save this model to the given path. + * It only works for Item datatypes supported by DataFrames. + * + * This saves: + * - human-readable (JSON) model metadata to path/metadata/ + * - Parquet formatted data to path/data/ + * + * The model may be loaded using `FPGrowthModel.load`. + * + * @param sc Spark context used to save model data. + * @param path Path specifying the directory in which to save this model. + * If the directory already exists, this method throws an exception. + */ + @Since("2.0.0") + override def save(sc: SparkContext, path: String): Unit = { + FPGrowthModel.SaveLoadV1_0.save(this, path) + } + + override protected val formatVersion: String = "1.0" +} + +@Since("2.0.0") +object FPGrowthModel extends Loader[FPGrowthModel[_]] { + + @Since("2.0.0") + override def load(sc: SparkContext, path: String): FPGrowthModel[_] = { + FPGrowthModel.SaveLoadV1_0.load(sc, path) + } + + private[fpm] object SaveLoadV1_0 { + + private val thisFormatVersion = "1.0" + + private val thisClassName = "org.apache.spark.mllib.fpm.FPGrowthModel" + + def save(model: FPGrowthModel[_], path: String): Unit = { + val sc = model.freqItemsets.sparkContext + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) + + // Get the type of item class + val sample = model.freqItemsets.first().items(0) + val className = sample.getClass.getCanonicalName + val classSymbol = runtimeMirror(getClass.getClassLoader).staticClass(className) + val tpe = classSymbol.selfType + + val itemType = ScalaReflection.schemaFor(tpe).dataType + val fields = Array(StructField("items", ArrayType(itemType)), + StructField("freq", LongType)) + val schema = StructType(fields) + val rowDataRDD = model.freqItemsets.map { x => + Row(x.items.toSeq, x.freq) + } + spark.createDataFrame(rowDataRDD, schema).write.parquet(Loader.dataPath(path)) + } + + def load(sc: SparkContext, path: String): FPGrowthModel[_] = { + implicit val formats = DefaultFormats + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + + val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) + assert(className == thisClassName) + assert(formatVersion == thisFormatVersion) + + val freqItemsets = spark.read.parquet(Loader.dataPath(path)) + val sample = freqItemsets.select("items").head().get(0) + loadImpl(freqItemsets, sample) + } + + def loadImpl[Item: ClassTag](freqItemsets: DataFrame, sample: Item): FPGrowthModel[Item] = { + val freqItemsetsRDD = freqItemsets.select("items", "freq").rdd.map { x => + val items = x.getAs[Seq[Item]](0).toArray + val freq = x.getLong(1) + new FreqItemset(items, freq) + } + new FPGrowthModel(freqItemsetsRDD) + } + } } /** * A parallel FP-growth algorithm to mine frequent itemsets. The algorithm is described in - * [[http://dx.doi.org/10.1145/1454008.1454027 Li et al., PFP: Parallel FP-Growth for Query - * Recommendation]]. PFP distributes computation in such a way that each worker executes an + * Li et al., PFP: Parallel FP-Growth for Query + * Recommendation. PFP distributes computation in such a way that each worker executes an * independent group of mining tasks. The FP-Growth algorithm is described in - * [[http://dx.doi.org/10.1145/335191.335372 Han et al., Mining frequent patterns without candidate - * generation]]. + * Han et al., Mining frequent patterns without + * candidate generation. * - * @param minSupport the minimal support level of the frequent pattern, any pattern appears + * @param minSupport the minimal support level of the frequent pattern, any pattern that appears * more than (minSupport * size-of-the-dataset) times will be output * @param numPartitions number of partitions used by parallel FP-growth * - * @see [[http://en.wikipedia.org/wiki/Association_rule_learning Association rule learning - * (Wikipedia)]] + * @see + * Association rule learning (Wikipedia) * */ @Since("1.3.0") @@ -86,6 +180,8 @@ class FPGrowth private ( */ @Since("1.3.0") def setMinSupport(minSupport: Double): this.type = { + require(minSupport >= 0.0 && minSupport <= 1.0, + s"Minimal support level must be in range [0, 1] but got ${minSupport}") this.minSupport = minSupport this } @@ -96,6 +192,8 @@ class FPGrowth private ( */ @Since("1.3.0") def setNumPartitions(numPartitions: Int): this.type = { + require(numPartitions > 0, + s"Number of partitions must be positive but got ${numPartitions}") this.numPartitions = numPartitions this } @@ -120,7 +218,9 @@ class FPGrowth private ( new FPGrowthModel(freqItemsets) } - /** Java-friendly version of [[run]]. */ + /** + * Java-friendly version of `run`. + */ @Since("1.3.0") def run[Item, Basket <: JavaIterable[Item]](data: JavaRDD[Basket]): FPGrowthModel[Item] = { implicit val tag = fakeClassTag[Item] @@ -139,7 +239,7 @@ class FPGrowth private ( partitioner: Partitioner): Array[Item] = { data.flatMap { t => val uniq = t.toSet - if (t.size != uniq.size) { + if (t.length != uniq.size) { throw new SparkException(s"Items in a transaction must be unique but got ${t.toSeq}.") } t @@ -211,7 +311,7 @@ object FPGrowth { /** * Frequent itemset. - * @param items items in this itemset. Java users should call [[FreqItemset#javaItems]] instead. + * @param items items in this itemset. Java users should call `FreqItemset.javaItems` instead. * @param freq frequency * @tparam Item item type * @@ -229,5 +329,9 @@ object FPGrowth { def javaItems: java.util.List[Item] = { items.toList.asJava } + + override def toString: String = { + s"${items.mkString("{", ",", "}")}: $freq" + } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala index 1d2d777c0079..b0fa287473c3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala @@ -126,7 +126,7 @@ private[fpm] object FPTree { def isRoot: Boolean = parent == null } - /** Summary of a item in an FP-Tree. */ + /** Summary of an item in an FP-Tree. */ private class Summary[T] extends Serializable { var count: Long = 0L val nodes: ListBuffer[Node[T]] = ListBuffer.empty diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/LocalPrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/LocalPrefixSpan.scala index 3ea10779a183..659f875a6dc9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/LocalPrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/LocalPrefixSpan.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.fpm import scala.collection.mutable -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Calculate all patterns of a projected database in local mode. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala index 97916daa2e9a..3f8d65a378e2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala @@ -20,37 +20,45 @@ package org.apache.spark.mllib.fpm import java.{lang => jl, util => ju} import java.util.concurrent.atomic.AtomicInteger -import scala.collection.mutable import scala.collection.JavaConverters._ +import scala.collection.mutable import scala.reflect.ClassTag +import scala.reflect.runtime.universe._ + +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods.{compact, render} -import org.apache.spark.Logging -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.internal.Logging +import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel /** - * :: Experimental :: - * * A parallel PrefixSpan algorithm to mine frequent sequential patterns. * The PrefixSpan algorithm is described in J. Pei, et al., PrefixSpan: Mining Sequential Patterns - * Efficiently by Prefix-Projected Pattern Growth ([[http://doi.org/10.1109/ICDE.2001.914830]]). + * Efficiently by Prefix-Projected Pattern Growth + * (see here). * - * @param minSupport the minimal support level of the sequential pattern, any pattern appears - * more than (minSupport * size-of-the-dataset) times will be output - * @param maxPatternLength the maximal length of the sequential pattern, any pattern appears + * @param minSupport the minimal support level of the sequential pattern, any pattern that appears + * more than (minSupport * size-of-the-dataset) times will be output + * @param maxPatternLength the maximal length of the sequential pattern, any pattern that appears * less than maxPatternLength will be output * @param maxLocalProjDBSize The maximum number of items (including delimiters used in the internal * storage format) allowed in a projected database before local * processing. If a projected database exceeds this size, another * iteration of distributed prefix growth is run. * - * @see [[https://en.wikipedia.org/wiki/Sequential_Pattern_Mining Sequential Pattern Mining - * (Wikipedia)]] + * @see Sequential Pattern Mining + * (Wikipedia) */ -@Experimental @Since("1.5.0") class PrefixSpan private ( private var minSupport: Double, @@ -136,45 +144,13 @@ class PrefixSpan private ( logInfo(s"minimum count for a frequent pattern: $minCount") // Find frequent items. - val freqItemAndCounts = data.flatMap { itemsets => - val uniqItems = mutable.Set.empty[Item] - itemsets.foreach { _.foreach { item => - uniqItems += item - }} - uniqItems.toIterator.map((_, 1L)) - }.reduceByKey(_ + _) - .filter { case (_, count) => - count >= minCount - }.collect() - val freqItems = freqItemAndCounts.sortBy(-_._2).map(_._1) + val freqItems = findFrequentItems(data, minCount) logInfo(s"number of frequent items: ${freqItems.length}") // Keep only frequent items from input sequences and convert them to internal storage. val itemToInt = freqItems.zipWithIndex.toMap - val dataInternalRepr = data.flatMap { itemsets => - val allItems = mutable.ArrayBuilder.make[Int] - var containsFreqItems = false - allItems += 0 - itemsets.foreach { itemsets => - val items = mutable.ArrayBuilder.make[Int] - itemsets.foreach { item => - if (itemToInt.contains(item)) { - items += itemToInt(item) + 1 // using 1-indexing in internal format - } - } - val result = items.result() - if (result.nonEmpty) { - containsFreqItems = true - allItems ++= result.sorted - } - allItems += 0 - } - if (containsFreqItems) { - Iterator.single(allItems.result()) - } else { - Iterator.empty - } - }.persist(StorageLevel.MEMORY_AND_DISK) + val dataInternalRepr = toDatabaseInternalRepr(data, itemToInt) + .persist(StorageLevel.MEMORY_AND_DISK) val results = genFreqPatterns(dataInternalRepr, minCount, maxPatternLength, maxLocalProjDBSize) @@ -203,7 +179,7 @@ class PrefixSpan private ( } /** - * A Java-friendly version of [[run()]] that reads sequences from a [[JavaRDD]] and returns + * A Java-friendly version of `run()` that reads sequences from a `JavaRDD` and returns * frequent sequences in a [[PrefixSpanModel]]. * @param data ordered sequences of itemsets stored as Java Iterable of Iterables * @tparam Item item type @@ -220,10 +196,70 @@ class PrefixSpan private ( } -@Experimental @Since("1.5.0") object PrefixSpan extends Logging { + /** + * This methods finds all frequent items in a input dataset. + * + * @param data Sequences of itemsets. + * @param minCount The minimal number of sequence an item should be present in to be frequent + * + * @return An array of Item containing only frequent items. + */ + private[fpm] def findFrequentItems[Item: ClassTag]( + data: RDD[Array[Array[Item]]], + minCount: Long): Array[Item] = { + + data.flatMap { itemsets => + val uniqItems = mutable.Set.empty[Item] + itemsets.foreach(set => uniqItems ++= set) + uniqItems.toIterator.map((_, 1L)) + }.reduceByKey(_ + _).filter { case (_, count) => + count >= minCount + }.sortBy(-_._2).map(_._1).collect() + } + + /** + * This methods cleans the input dataset from un-frequent items, and translate it's item + * to their corresponding Int identifier. + * + * @param data Sequences of itemsets. + * @param itemToInt A map allowing translation of frequent Items to their Int Identifier. + * The map should only contain frequent item. + * + * @return The internal repr of the inputted dataset. With properly placed zero delimiter. + */ + private[fpm] def toDatabaseInternalRepr[Item: ClassTag]( + data: RDD[Array[Array[Item]]], + itemToInt: Map[Item, Int]): RDD[Array[Int]] = { + + data.flatMap { itemsets => + val allItems = mutable.ArrayBuilder.make[Int] + var containsFreqItems = false + allItems += 0 + itemsets.foreach { itemsets => + val items = mutable.ArrayBuilder.make[Int] + itemsets.foreach { item => + if (itemToInt.contains(item)) { + items += itemToInt(item) + 1 // using 1-indexing in internal format + } + } + val result = items.result() + if (result.nonEmpty) { + containsFreqItems = true + allItems ++= result.sorted + allItems += 0 + } + } + if (containsFreqItems) { + Iterator.single(allItems.result()) + } else { + Iterator.empty + } + } + } + /** * Find the complete set of frequent sequential patterns in the input sequences. * @param data ordered sequences of itemsets. We represent a sequence internally as Array[Int], @@ -359,13 +395,13 @@ object PrefixSpan extends Logging { * Items are represented by positive integers, and items in each itemset must be distinct and * ordered. * we use 0 as the delimiter between itemsets. - * For example, a sequence `<(12)(31)1>` is represented by `[0, 1, 2, 0, 1, 3, 0, 1, 0]`. - * The postfix of this sequence w.r.t. to prefix `<1>` is `<(_2)(13)1>`. + * For example, a sequence `(12)(31)1` is represented by `[0, 1, 2, 0, 1, 3, 0, 1, 0]`. + * The postfix of this sequence w.r.t. to prefix `1` is `(_2)(13)1`. * We may reuse the original items array `[0, 1, 2, 0, 1, 3, 0, 1, 0]` to represent the postfix, * and mark the start index of the postfix, which is `2` in this example. * So the active items in this postfix are `[2, 0, 1, 3, 0, 1, 0]`. * We also remember the start indices of partial projections, the ones that split an itemset. - * For example, another possible partial projection w.r.t. `<1>` is `<(_3)1>`. + * For example, another possible partial projection w.r.t. `1` is `(_3)1`. * We remember the start indices of partial projections, which is `[2, 5]` in this example. * This data structure makes it easier to do projections. * @@ -541,7 +577,7 @@ object PrefixSpan extends Logging { } /** - * Represents a frequence sequence. + * Represents a frequent sequence. * @param sequence a sequence of itemsets stored as an Array of Arrays * @param freq frequency * @tparam Item item type @@ -566,4 +602,88 @@ object PrefixSpan extends Logging { @Since("1.5.0") class PrefixSpanModel[Item] @Since("1.5.0") ( @Since("1.5.0") val freqSequences: RDD[PrefixSpan.FreqSequence[Item]]) - extends Serializable + extends Saveable with Serializable { + + /** + * Save this model to the given path. + * It only works for Item datatypes supported by DataFrames. + * + * This saves: + * - human-readable (JSON) model metadata to path/metadata/ + * - Parquet formatted data to path/data/ + * + * The model may be loaded using `PrefixSpanModel.load`. + * + * @param sc Spark context used to save model data. + * @param path Path specifying the directory in which to save this model. + * If the directory already exists, this method throws an exception. + */ + @Since("2.0.0") + override def save(sc: SparkContext, path: String): Unit = { + PrefixSpanModel.SaveLoadV1_0.save(this, path) + } + + override protected val formatVersion: String = "1.0" +} + +@Since("2.0.0") +object PrefixSpanModel extends Loader[PrefixSpanModel[_]] { + + @Since("2.0.0") + override def load(sc: SparkContext, path: String): PrefixSpanModel[_] = { + PrefixSpanModel.SaveLoadV1_0.load(sc, path) + } + + private[fpm] object SaveLoadV1_0 { + + private val thisFormatVersion = "1.0" + + private val thisClassName = "org.apache.spark.mllib.fpm.PrefixSpanModel" + + def save(model: PrefixSpanModel[_], path: String): Unit = { + val sc = model.freqSequences.sparkContext + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) + + // Get the type of item class + val sample = model.freqSequences.first().sequence(0)(0) + val className = sample.getClass.getCanonicalName + val classSymbol = runtimeMirror(getClass.getClassLoader).staticClass(className) + val tpe = classSymbol.selfType + + val itemType = ScalaReflection.schemaFor(tpe).dataType + val fields = Array(StructField("sequence", ArrayType(ArrayType(itemType))), + StructField("freq", LongType)) + val schema = StructType(fields) + val rowDataRDD = model.freqSequences.map { x => + Row(x.sequence, x.freq) + } + spark.createDataFrame(rowDataRDD, schema).write.parquet(Loader.dataPath(path)) + } + + def load(sc: SparkContext, path: String): PrefixSpanModel[_] = { + implicit val formats = DefaultFormats + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + + val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) + assert(className == thisClassName) + assert(formatVersion == thisFormatVersion) + + val freqSequences = spark.read.parquet(Loader.dataPath(path)) + val sample = freqSequences.select("sequence").head().get(0) + loadImpl(freqSequences, sample) + } + + def loadImpl[Item: ClassTag](freqSequences: DataFrame, sample: Item): PrefixSpanModel[Item] = { + val freqSequencesRDD = freqSequences.select("sequence", "freq").rdd.map { x => + val sequence = x.getAs[Seq[Seq[Item]]](0).map(_.toArray).toArray + val freq = x.getLong(1) + new PrefixSpan.FreqSequence(sequence, freq) + } + new PrefixSpanModel(freqSequencesRDD) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicCheckpointer.scala b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicCheckpointer.scala deleted file mode 100644 index 72d3aabc9b1f..000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicCheckpointer.scala +++ /dev/null @@ -1,154 +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.mllib.impl - -import scala.collection.mutable - -import org.apache.hadoop.fs.{Path, FileSystem} - -import org.apache.spark.{SparkContext, Logging} -import org.apache.spark.storage.StorageLevel - - -/** - * This abstraction helps with persisting and checkpointing RDDs and types derived from RDDs - * (such as Graphs and DataFrames). In documentation, we use the phrase "Dataset" to refer to - * the distributed data type (RDD, Graph, etc.). - * - * Specifically, this abstraction automatically handles persisting and (optionally) checkpointing, - * as well as unpersisting and removing checkpoint files. - * - * Users should call update() when a new Dataset has been created, - * before the Dataset has been materialized. After updating [[PeriodicCheckpointer]], users are - * responsible for materializing the Dataset to ensure that persisting and checkpointing actually - * occur. - * - * When update() is called, this does the following: - * - Persist new Dataset (if not yet persisted), and put in queue of persisted Datasets. - * - Unpersist Datasets from queue until there are at most 3 persisted Datasets. - * - If using checkpointing and the checkpoint interval has been reached, - * - Checkpoint the new Dataset, and put in a queue of checkpointed Datasets. - * - Remove older checkpoints. - * - * WARNINGS: - * - This class should NOT be copied (since copies may conflict on which Datasets should be - * checkpointed). - * - This class removes checkpoint files once later Datasets have been checkpointed. - * However, references to the older Datasets will still return isCheckpointed = true. - * - * @param checkpointInterval Datasets will be checkpointed at this interval - * @param sc SparkContext for the Datasets given to this checkpointer - * @tparam T Dataset type, such as RDD[Double] - */ -private[mllib] abstract class PeriodicCheckpointer[T]( - val checkpointInterval: Int, - val sc: SparkContext) extends Logging { - - /** FIFO queue of past checkpointed Datasets */ - private val checkpointQueue = mutable.Queue[T]() - - /** FIFO queue of past persisted Datasets */ - private val persistedQueue = mutable.Queue[T]() - - /** Number of times [[update()]] has been called */ - private var updateCount = 0 - - /** - * Update with a new Dataset. Handle persistence and checkpointing as needed. - * Since this handles persistence and checkpointing, this should be called before the Dataset - * has been materialized. - * - * @param newData New Dataset created from previous Datasets in the lineage. - */ - def update(newData: T): Unit = { - persist(newData) - persistedQueue.enqueue(newData) - // We try to maintain 2 Datasets in persistedQueue to support the semantics of this class: - // Users should call [[update()]] when a new Dataset has been created, - // before the Dataset has been materialized. - while (persistedQueue.size > 3) { - val dataToUnpersist = persistedQueue.dequeue() - unpersist(dataToUnpersist) - } - updateCount += 1 - - // Handle checkpointing (after persisting) - if ((updateCount % checkpointInterval) == 0 && sc.getCheckpointDir.nonEmpty) { - // Add new checkpoint before removing old checkpoints. - checkpoint(newData) - checkpointQueue.enqueue(newData) - // Remove checkpoints before the latest one. - var canDelete = true - while (checkpointQueue.size > 1 && canDelete) { - // Delete the oldest checkpoint only if the next checkpoint exists. - if (isCheckpointed(checkpointQueue.head)) { - removeCheckpointFile() - } else { - canDelete = false - } - } - } - } - - /** Checkpoint the Dataset */ - protected def checkpoint(data: T): Unit - - /** Return true iff the Dataset is checkpointed */ - protected def isCheckpointed(data: T): Boolean - - /** - * Persist the Dataset. - * Note: This should handle checking the current [[StorageLevel]] of the Dataset. - */ - protected def persist(data: T): Unit - - /** Unpersist the Dataset */ - protected def unpersist(data: T): Unit - - /** Get list of checkpoint files for this given Dataset */ - protected def getCheckpointFiles(data: T): Iterable[String] - - /** - * Call this at the end to delete any remaining checkpoint files. - */ - def deleteAllCheckpoints(): Unit = { - while (checkpointQueue.nonEmpty) { - removeCheckpointFile() - } - } - - /** - * Dequeue the oldest checkpointed Dataset, and remove its checkpoint files. - * This prints a warning but does not fail if the files cannot be removed. - */ - private def removeCheckpointFile(): Unit = { - val old = checkpointQueue.dequeue() - // Since the old checkpoint is not deleted by Spark, we manually delete it. - val fs = FileSystem.get(sc.hadoopConfiguration) - getCheckpointFiles(old).foreach { checkpointFile => - try { - fs.delete(new Path(checkpointFile), true) - } catch { - case e: Exception => - logWarning("PeriodicCheckpointer could not remove old checkpoint file: " + - checkpointFile) - } - } - } - -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala index df9f4ae145b8..cb9774224568 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.linalg import com.github.fommil.netlib.{BLAS => NetlibBLAS, F2jBLAS} import com.github.fommil.netlib.BLAS.{getInstance => NativeBLAS} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * BLAS routines for MLlib's vectors and matrices. @@ -31,7 +31,7 @@ private[spark] object BLAS extends Serializable with Logging { @transient private var _nativeBLAS: NetlibBLAS = _ // For level-1 routines, we use Java implementation. - private def f2jBLAS: NetlibBLAS = { + private[mllib] def f2jBLAS: NetlibBLAS = { if (_f2jBLAS == null) { _f2jBLAS = new F2jBLAS } @@ -75,7 +75,7 @@ private[spark] object BLAS extends Serializable with Logging { val xValues = x.values val xIndices = x.indices val yValues = y.values - val nnz = xIndices.size + val nnz = xIndices.length if (a == 1.0) { var k = 0 @@ -135,7 +135,7 @@ private[spark] object BLAS extends Serializable with Logging { val xValues = x.values val xIndices = x.indices val yValues = y.values - val nnz = xIndices.size + val nnz = xIndices.length var sum = 0.0 var k = 0 @@ -154,8 +154,8 @@ private[spark] object BLAS extends Serializable with Logging { val xIndices = x.indices val yValues = y.values val yIndices = y.indices - val nnzx = xIndices.size - val nnzy = yIndices.size + val nnzx = xIndices.length + val nnzy = yIndices.length var kx = 0 var ky = 0 @@ -188,7 +188,7 @@ private[spark] object BLAS extends Serializable with Logging { val sxIndices = sx.indices val sxValues = sx.values val dyValues = dy.values - val nnz = sxIndices.size + val nnz = sxIndices.length var i = 0 var k = 0 @@ -237,7 +237,7 @@ private[spark] object BLAS extends Serializable with Logging { } /** - * Adds alpha * x * x.t to a matrix in-place. This is the same as BLAS's ?SPR. + * Adds alpha * v * v.t to a matrix in-place. This is the same as BLAS's ?SPR. * * @param U the upper triangular part of the matrix in a [[DenseVector]](column major) */ @@ -246,7 +246,7 @@ private[spark] object BLAS extends Serializable with Logging { } /** - * Adds alpha * x * x.t to a matrix in-place. This is the same as BLAS's ?SPR. + * Adds alpha * v * v.t to a matrix in-place. This is the same as BLAS's ?SPR. * * @param U the upper triangular part of the matrix packed in an array (column major) */ @@ -267,7 +267,6 @@ private[spark] object BLAS extends Serializable with Logging { col = indices(j) // Skip empty columns. colStartIdx += (col - prevCol) * (col + prevCol + 1) / 2 - col = indices(j) av = alpha * values(j) i = 0 while (i <= j) { @@ -420,7 +419,7 @@ private[spark] object BLAS extends Serializable with Logging { val AcolPtrs = A.colPtrs // Slicing is easy in this case. This is the optimal multiplication setting for sparse matrices - if (A.isTransposed){ + if (A.isTransposed) { var colCounterForB = 0 if (!B.isTransposed) { // Expensive to put the check inside the loop while (colCounterForB < nB) { @@ -638,12 +637,16 @@ private[spark] object BLAS extends Serializable with Logging { val indEnd = Arows(rowCounter + 1) var sum = 0.0 var k = 0 - while (k < xNnz && i < indEnd) { + while (i < indEnd && k < xNnz) { if (xIndices(k) == Acols(i)) { sum += Avals(i) * xValues(k) + k += 1 + i += 1 + } else if (xIndices(k) < Acols(i)) { + k += 1 + } else { i += 1 } - k += 1 } yValues(rowCounter) = sum * alpha + beta * yValues(rowCounter) rowCounter += 1 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/CholeskyDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/CholeskyDecomposition.scala index 0cd371e9cce3..68771f1afbe8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/CholeskyDecomposition.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/CholeskyDecomposition.scala @@ -20,6 +20,8 @@ package org.apache.spark.mllib.linalg import com.github.fommil.netlib.LAPACK.{getInstance => lapack} import org.netlib.util.intW +import org.apache.spark.ml.optim.SingularMatrixException + /** * Compute Cholesky decomposition. */ @@ -33,11 +35,10 @@ private[spark] object CholeskyDecomposition { * @return the solution array */ def solve(A: Array[Double], bx: Array[Double]): Array[Double] = { - val k = bx.size + val k = bx.length val info = new intW(0) lapack.dppsv("U", k, 1, A, bx, k, info) - val code = info.`val` - assert(code == 0, s"lapack.dpotrs returned $code.") + checkReturnValue(info, "dppsv") bx } @@ -52,8 +53,20 @@ private[spark] object CholeskyDecomposition { def inverse(UAi: Array[Double], k: Int): Array[Double] = { val info = new intW(0) lapack.dpptri("U", k, UAi, info) - val code = info.`val` - assert(code == 0, s"lapack.dpptri returned $code.") + checkReturnValue(info, "dpptri") UAi } + + private def checkReturnValue(info: intW, method: String): Unit = { + info.`val` match { + case code if code < 0 => + throw new IllegalStateException(s"LAPACK.$method returned $code; arg ${-code} is illegal") + case code if code > 0 => + throw new SingularMatrixException ( + s"LAPACK.$method returned $code because A is not positive definite. Is A derived from " + + "a singular matrix (e.g. collinear column values)?") + case _ => // do nothing + } + } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/EigenValueDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/EigenValueDecomposition.scala index 863abe86d38d..c7c1a5404e5e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/EigenValueDecomposition.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/EigenValueDecomposition.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.linalg import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV} import com.github.fommil.netlib.ARPACK -import org.netlib.util.{intW, doubleW} +import org.netlib.util.{doubleW, intW} /** * Compute eigen-decomposition. @@ -32,7 +32,7 @@ private[mllib] object EigenValueDecomposition { * * @param mul a function that multiplies the symmetric matrix with a DenseVector. * @param n dimension of the square matrix (maximum Int.MaxValue). - * @param k number of leading eigenvalues required, 0 < k < n. + * @param k number of leading eigenvalues required, where k must be positive and less than n. * @param tol tolerance of the eigs computation. * @param maxIterations the maximum number of Arnoldi update iterations. * @return a dense vector of eigenvalues in descending order and a dense matrix of eigenvectors @@ -78,13 +78,13 @@ private[mllib] object EigenValueDecomposition { require(n * ncv.toLong <= Integer.MAX_VALUE && ncv * (ncv.toLong + 8) <= Integer.MAX_VALUE, s"k = $k and/or n = $n are too large to compute an eigendecomposition") - var ido = new intW(0) - var info = new intW(0) - var resid = new Array[Double](n) - var v = new Array[Double](n * ncv) - var workd = new Array[Double](n * 3) - var workl = new Array[Double](ncv * (ncv + 8)) - var ipntr = new Array[Int](11) + val ido = new intW(0) + val info = new intW(0) + val resid = new Array[Double](n) + val v = new Array[Double](n * ncv) + val workd = new Array[Double](n * 3) + val workl = new Array[Double](ncv * (ncv + 8)) + val ipntr = new Array[Int](11) // call ARPACK's reverse communication, first iteration with ido = 0 arpack.dsaupd(ido, bmat, n, which, nev.`val`, tolW, resid, ncv, v, n, iparam, ipntr, workd, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 8879dcf75c9b..bf9b4cfe15b2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -19,14 +19,16 @@ package org.apache.spark.mllib.linalg import java.util.{Arrays, Random} -import scala.collection.mutable.{ArrayBuilder => MArrayBuilder, HashSet => MHashSet, ArrayBuffer} +import scala.collection.mutable.{ArrayBuffer, ArrayBuilder => MArrayBuilder, HashSet => MHashSet} +import scala.language.implicitConversions import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM} +import com.github.fommil.netlib.BLAS.{getInstance => blas} -import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.annotation.Since +import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeArrayData} import org.apache.spark.sql.types._ /** @@ -58,8 +60,22 @@ sealed trait Matrix extends Serializable { newArray } + /** + * Returns an iterator of column vectors. + * This operation could be expensive, depending on the underlying storage. + */ + @Since("2.0.0") + def colIter: Iterator[Vector] + + /** + * Returns an iterator of row vectors. + * This operation could be expensive, depending on the underlying storage. + */ + @Since("2.0.0") + def rowIter: Iterator[Vector] = this.transpose.colIter + /** Converts to a breeze matrix. */ - private[mllib] def toBreeze: BM[Double] + private[mllib] def asBreeze: BM[Double] /** Gets the (i, j)-th element. */ @Since("1.3.0") @@ -75,11 +91,15 @@ sealed trait Matrix extends Serializable { @Since("1.2.0") def copy: Matrix - /** Transpose the Matrix. Returns a new `Matrix` instance sharing the same underlying data. */ + /** + * Transpose the Matrix. Returns a new `Matrix` instance sharing the same underlying data. + */ @Since("1.3.0") def transpose: Matrix - /** Convenience method for `Matrix`-`DenseMatrix` multiplication. */ + /** + * Convenience method for `Matrix`-`DenseMatrix` multiplication. + */ @Since("1.2.0") def multiply(y: DenseMatrix): DenseMatrix = { val C: DenseMatrix = DenseMatrix.zeros(numRows, y.numCols) @@ -87,13 +107,17 @@ sealed trait Matrix extends Serializable { C } - /** Convenience method for `Matrix`-`DenseVector` multiplication. For binary compatibility. */ + /** + * Convenience method for `Matrix`-`DenseVector` multiplication. For binary compatibility. + */ @Since("1.2.0") def multiply(y: DenseVector): DenseVector = { multiply(y.asInstanceOf[Vector]) } - /** Convenience method for `Matrix`-`Vector` multiplication. */ + /** + * Convenience method for `Matrix`-`Vector` multiplication. + */ @Since("1.4.0") def multiply(y: Vector): DenseVector = { val output = new DenseVector(new Array[Double](numRows)) @@ -102,20 +126,24 @@ sealed trait Matrix extends Serializable { } /** A human readable representation of the matrix */ - override def toString: String = toBreeze.toString() + override def toString: String = asBreeze.toString() /** A human readable representation of the matrix with maximum lines and width */ @Since("1.4.0") - def toString(maxLines: Int, maxLineWidth: Int): String = toBreeze.toString(maxLines, maxLineWidth) + def toString(maxLines: Int, maxLineWidth: Int): String = asBreeze.toString(maxLines, maxLineWidth) - /** Map the values of this matrix using a function. Generates a new matrix. Performs the - * function on only the backing array. For example, an operation such as addition or - * subtraction will only be performed on the non-zero values in a `SparseMatrix`. */ + /** + * Map the values of this matrix using a function. Generates a new matrix. Performs the + * function on only the backing array. For example, an operation such as addition or + * subtraction will only be performed on the non-zero values in a `SparseMatrix`. + */ private[spark] def map(f: Double => Double): Matrix - /** Update all the values of this matrix using the function f. Performed in-place on the - * backing array. For example, an operation such as addition or subtraction will only be - * performed on the non-zero values in a `SparseMatrix`. */ + /** + * Update all the values of this matrix using the function f. Performed in-place on the + * backing array. For example, an operation such as addition or subtraction will only be + * performed on the non-zero values in a `SparseMatrix`. + */ private[mllib] def update(f: Double => Double): Matrix /** @@ -139,9 +167,15 @@ sealed trait Matrix extends Serializable { */ @Since("1.5.0") def numActives: Int + + /** + * Convert this matrix to the new mllib-local representation. + * This does NOT copy the data; it copies references. + */ + @Since("2.0.0") + def asML: newlinalg.Matrix } -@DeveloperApi private[spark] class MatrixUDT extends UserDefinedType[Matrix] { override def sqlType: StructType = { @@ -162,16 +196,16 @@ private[spark] class MatrixUDT extends UserDefinedType[Matrix] { )) } - override def serialize(obj: Any): InternalRow = { - val row = new GenericMutableRow(7) + override def serialize(obj: Matrix): InternalRow = { + val row = new GenericInternalRow(7) obj match { case sm: SparseMatrix => row.setByte(0, 0) row.setInt(1, sm.numRows) row.setInt(2, sm.numCols) - row.update(3, new GenericArrayData(sm.colPtrs.map(_.asInstanceOf[Any]))) - row.update(4, new GenericArrayData(sm.rowIndices.map(_.asInstanceOf[Any]))) - row.update(5, new GenericArrayData(sm.values.map(_.asInstanceOf[Any]))) + row.update(3, UnsafeArrayData.fromPrimitiveArray(sm.colPtrs)) + row.update(4, UnsafeArrayData.fromPrimitiveArray(sm.rowIndices)) + row.update(5, UnsafeArrayData.fromPrimitiveArray(sm.values)) row.setBoolean(6, sm.isTransposed) case dm: DenseMatrix => @@ -180,7 +214,7 @@ private[spark] class MatrixUDT extends UserDefinedType[Matrix] { row.setInt(2, dm.numCols) row.setNullAt(3) row.setNullAt(4) - row.update(5, new GenericArrayData(dm.values.map(_.asInstanceOf[Any]))) + row.update(5, UnsafeArrayData.fromPrimitiveArray(dm.values)) row.setBoolean(6, dm.isTransposed) } row @@ -274,15 +308,15 @@ class DenseMatrix @Since("1.3.0") ( this(numRows, numCols, values, false) override def equals(o: Any): Boolean = o match { - case m: Matrix => toBreeze == m.toBreeze + case m: Matrix => asBreeze == m.asBreeze case _ => false } override def hashCode: Int = { - com.google.common.base.Objects.hashCode(numRows : Integer, numCols: Integer, toArray) + com.google.common.base.Objects.hashCode(numRows: Integer, numCols: Integer, toArray) } - private[mllib] def toBreeze: BM[Double] = { + private[mllib] def asBreeze: BM[Double] = { if (!isTransposed) { new BDM[Double](numRows, numCols, values) } else { @@ -386,6 +420,26 @@ class DenseMatrix @Since("1.3.0") ( } new SparseMatrix(numRows, numCols, colPtrs, rowIndices.result(), spVals.result()) } + + @Since("2.0.0") + override def colIter: Iterator[Vector] = { + if (isTransposed) { + Iterator.tabulate(numCols) { j => + val col = new Array[Double](numRows) + blas.dcopy(numRows, values, j, numCols, col, 0, 1) + new DenseVector(col) + } + } else { + Iterator.tabulate(numCols) { j => + new DenseVector(values.slice(j * numRows, (j + 1) * numRows)) + } + } + } + + @Since("2.0.0") + override def asML: newlinalg.DenseMatrix = { + new newlinalg.DenseMatrix(numRows, numCols, values, isTransposed) + } } /** @@ -482,6 +536,14 @@ object DenseMatrix { } matrix } + + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(m: newlinalg.DenseMatrix): DenseMatrix = { + new DenseMatrix(m.numRows, m.numCols, m.values, m.isTransposed) + } } /** @@ -518,10 +580,13 @@ class SparseMatrix @Since("1.3.0") ( require(values.length == rowIndices.length, "The number of row indices and values don't match! " + s"values.length: ${values.length}, rowIndices.length: ${rowIndices.length}") - // The Or statement is for the case when the matrix is transposed - require(colPtrs.length == numCols + 1 || colPtrs.length == numRows + 1, "The length of the " + - "column indices should be the number of columns + 1. Currently, colPointers.length: " + - s"${colPtrs.length}, numCols: $numCols") + if (isTransposed) { + require(colPtrs.length == numRows + 1, + s"Expecting ${numRows + 1} colPtrs when numRows = $numRows but got ${colPtrs.length}") + } else { + require(colPtrs.length == numCols + 1, + s"Expecting ${numCols + 1} colPtrs when numCols = $numCols but got ${colPtrs.length}") + } require(values.length == colPtrs.last, "The last value of colPtrs must equal the number of " + s"elements. values.length: ${values.length}, colPtrs.last: ${colPtrs.last}") @@ -553,11 +618,13 @@ class SparseMatrix @Since("1.3.0") ( values: Array[Double]) = this(numRows, numCols, colPtrs, rowIndices, values, false) override def equals(o: Any): Boolean = o match { - case m: Matrix => toBreeze == m.toBreeze + case m: Matrix => asBreeze == m.asBreeze case _ => false } - private[mllib] def toBreeze: BM[Double] = { + override def hashCode(): Int = asBreeze.hashCode + + private[mllib] def asBreeze: BM[Double] = { if (!isTransposed) { new BSM[Double](values, numRows, numCols, colPtrs, rowIndices) } else { @@ -584,7 +651,7 @@ class SparseMatrix @Since("1.3.0") ( private[mllib] def update(i: Int, j: Int, v: Double): Unit = { val ind = index(i, j) - if (ind == -1) { + if (ind < 0) { throw new NoSuchElementException("The given row and column indices correspond to a zero " + "value. Only non-zero elements in Sparse Matrices can be updated.") } else { @@ -656,6 +723,43 @@ class SparseMatrix @Since("1.3.0") ( @Since("1.5.0") override def numActives: Int = values.length + @Since("2.0.0") + override def colIter: Iterator[Vector] = { + if (isTransposed) { + val indicesArray = Array.fill(numCols)(MArrayBuilder.make[Int]) + val valuesArray = Array.fill(numCols)(MArrayBuilder.make[Double]) + var i = 0 + while (i < numRows) { + var k = colPtrs(i) + val rowEnd = colPtrs(i + 1) + while (k < rowEnd) { + val j = rowIndices(k) + indicesArray(j) += i + valuesArray(j) += values(k) + k += 1 + } + i += 1 + } + Iterator.tabulate(numCols) { j => + val ii = indicesArray(j).result() + val vv = valuesArray(j).result() + new SparseVector(numRows, ii, vv) + } + } else { + Iterator.tabulate(numCols) { j => + val colStart = colPtrs(j) + val colEnd = colPtrs(j + 1) + val ii = rowIndices.slice(colStart, colEnd) + val vv = values.slice(colStart, colEnd) + new SparseVector(numRows, ii, vv) + } + } + } + + @Since("2.0.0") + override def asML: newlinalg.SparseMatrix = { + new newlinalg.SparseMatrix(numRows, numCols, colPtrs, rowIndices, values, isTransposed) + } } /** @@ -693,7 +797,7 @@ object SparseMatrix { var prevRow = -1 var prevVal = 0.0 // Append a dummy entry to include the last one at the end of the loop. - (sortedEntries.view :+ (numRows, numCols, 1.0)).foreach { case (i, j, v) => + (sortedEntries.view :+ ((numRows, numCols, 1.0))).foreach { case (i, j, v) => if (v != 0) { if (i == prevRow && j == prevCol) { prevVal += v @@ -746,7 +850,7 @@ object SparseMatrix { "The expected number of nonzeros cannot be greater than Int.MaxValue.") val nnz = math.ceil(expected).toInt if (density == 0.0) { - new SparseMatrix(numRows, numCols, new Array[Int](numCols + 1), Array[Int](), Array[Double]()) + new SparseMatrix(numRows, numCols, new Array[Int](numCols + 1), Array.empty, Array.empty) } else if (density == 1.0) { val colPtrs = Array.tabulate(numCols + 1)(j => j * numRows) val rowIndices = Array.tabulate(size.toInt)(idx => idx % numRows) @@ -830,6 +934,14 @@ object SparseMatrix { SparseMatrix.fromCOO(n, n, nnzVals.map(v => (v._2, v._2, v._1))) } } + + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(m: newlinalg.SparseMatrix): SparseMatrix = { + new SparseMatrix(m.numRows, m.numCols, m.colPtrs, m.rowIndices, m.values, m.isTransposed) + } } /** @@ -880,7 +992,16 @@ object Matrices { new DenseMatrix(dm.rows, dm.cols, dm.data, dm.isTranspose) case sm: BSM[Double] => // There is no isTranspose flag for sparse matrices in Breeze - new SparseMatrix(sm.rows, sm.cols, sm.colPtrs, sm.rowIndices, sm.data) + val nsm = if (sm.rowIndices.length > sm.activeSize) { + // This sparse matrix has trailing zeros. + // Remove them by compacting the matrix. + val csm = sm.copy + csm.compact() + csm + } else { + sm + } + new SparseMatrix(nsm.rows, nsm.cols, nsm.colPtrs, nsm.rowIndices, nsm.data) case _ => throw new UnsupportedOperationException( s"Do not support conversion from type ${breeze.getClass.getName}.") @@ -986,8 +1107,8 @@ object Matrices { @Since("1.3.0") def horzcat(matrices: Array[Matrix]): Matrix = { if (matrices.isEmpty) { - return new DenseMatrix(0, 0, Array[Double]()) - } else if (matrices.size == 1) { + return new DenseMatrix(0, 0, Array.empty) + } else if (matrices.length == 1) { return matrices(0) } val numRows = matrices(0).numRows @@ -1024,7 +1145,7 @@ object Matrices { val data = new ArrayBuffer[(Int, Int, Double)]() dnMat.foreachActive { (i, j, v) => if (v != 0.0) { - data.append((i, j + startCol, v)) + data += Tuple3(i, j + startCol, v) } } startCol += nCols @@ -1045,8 +1166,8 @@ object Matrices { @Since("1.3.0") def vertcat(matrices: Array[Matrix]): Matrix = { if (matrices.isEmpty) { - return new DenseMatrix(0, 0, Array[Double]()) - } else if (matrices.size == 1) { + return new DenseMatrix(0, 0, Array.empty[Double]) + } else if (matrices.length == 1) { return matrices(0) } val numCols = matrices(0).numCols @@ -1094,7 +1215,7 @@ object Matrices { val data = new ArrayBuffer[(Int, Int, Double)]() dnMat.foreachActive { (i, j, v) => if (v != 0.0) { - data.append((i + startRow, j, v)) + data += Tuple3(i + startRow, j, v) } } startRow += nRows @@ -1104,4 +1225,36 @@ object Matrices { SparseMatrix.fromCOO(numRows, numCols, entries) } } + + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(m: newlinalg.Matrix): Matrix = m match { + case dm: newlinalg.DenseMatrix => + DenseMatrix.fromML(dm) + case sm: newlinalg.SparseMatrix => + SparseMatrix.fromML(sm) + } +} + +/** + * Implicit methods available in Scala for converting [[org.apache.spark.mllib.linalg.Matrix]] to + * [[org.apache.spark.ml.linalg.Matrix]] and vice versa. + */ +private[spark] object MatrixImplicits { + + implicit def mllibMatrixToMLMatrix(m: Matrix): newlinalg.Matrix = m.asML + + implicit def mllibDenseMatrixToMLDenseMatrix(m: DenseMatrix): newlinalg.DenseMatrix = m.asML + + implicit def mllibSparseMatrixToMLSparseMatrix(m: SparseMatrix): newlinalg.SparseMatrix = m.asML + + implicit def mlMatrixToMLlibMatrix(m: newlinalg.Matrix): Matrix = Matrices.fromML(m) + + implicit def mlDenseMatrixToMLlibDenseMatrix(m: newlinalg.DenseMatrix): DenseMatrix = + Matrices.fromML(m).asInstanceOf[DenseMatrix] + + implicit def mlSparseMatrixToMLlibSparseMatrix(m: newlinalg.SparseMatrix): SparseMatrix = + Matrices.fromML(m).asInstanceOf[SparseMatrix] } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala index 4591cb88ef15..8024b1c0031f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.linalg -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since /** * Represents singular value decomposition (SVD) factors. @@ -26,10 +26,8 @@ import org.apache.spark.annotation.{Experimental, Since} case class SingularValueDecomposition[UType, VType](U: UType, s: Vector, V: VType) /** - * :: Experimental :: * Represents QR factors. */ @Since("1.5.0") -@Experimental case class QRDecomposition[QType, RType](Q: QType, R: RType) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index bd9badc03c34..fd9605c01362 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -17,26 +17,30 @@ package org.apache.spark.mllib.linalg -import java.util import java.lang.{Double => JavaDouble, Integer => JavaInteger, Iterable => JavaIterable} +import java.util import scala.annotation.varargs import scala.collection.JavaConverters._ +import scala.language.implicitConversions import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods.{compact, parse => parseJson, render} import org.apache.spark.SparkException import org.apache.spark.annotation.{AlphaComponent, Since} +import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.mllib.util.NumericParser import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeArrayData} import org.apache.spark.sql.types._ /** * Represents a numeric vector, whose index type is Int and value type is Double. * - * Note: Users should not implement this interface. + * @note Users should not implement this interface. */ @SQLUserDefinedType(udt = classOf[VectorUDT]) @Since("1.0.0") @@ -73,7 +77,7 @@ sealed trait Vector extends Serializable { /** * Returns a hash code value for the vector. The hash code is based on its size and its first 128 - * nonzero entries, using a hash algorithm similar to [[java.util.Arrays.hashCode]]. + * nonzero entries, using a hash algorithm similar to `java.util.Arrays.hashCode`. */ override def hashCode(): Int = { // This is a reference implementation. It calls return in foreachActive, which is slow. @@ -99,14 +103,14 @@ sealed trait Vector extends Serializable { /** * Converts the instance to a breeze vector. */ - private[spark] def toBreeze: BV[Double] + private[spark] def asBreeze: BV[Double] /** * Gets the value of the ith element. * @param i index */ @Since("1.1.0") - def apply(i: Int): Double = toBreeze(i) + def apply(i: Int): Double = asBreeze(i) /** * Makes a deep copy of this vector. @@ -128,7 +132,9 @@ sealed trait Vector extends Serializable { /** * Number of active entries. An "active entry" is an element which is explicitly stored, - * regardless of its value. Note that inactive entries have value 0. + * regardless of its value. + * + * @note Inactive entries have value 0. */ @Since("1.4.0") def numActives: Int @@ -143,7 +149,21 @@ sealed trait Vector extends Serializable { * Converts this vector to a sparse vector with all explicit zeros removed. */ @Since("1.4.0") - def toSparse: SparseVector + def toSparse: SparseVector = toSparseWithSize(numNonzeros) + + /** + * Converts this vector to a sparse vector with all explicit zeros removed when the size is known. + * This method is used to avoid re-computing the number of non-zero elements when it is + * already known. This method should only be called after computing the number of non-zero + * elements via [[numNonzeros]]. e.g. + * {{{ + * val nnz = numNonzeros + * val sv = toSparse(nnz) + * }}} + * + * If `nnz` is under-specified, a [[java.lang.ArrayIndexOutOfBoundsException]] is thrown. + */ + private[linalg] def toSparseWithSize(nnz: Int): SparseVector /** * Converts this vector to a dense vector. @@ -159,7 +179,7 @@ sealed trait Vector extends Serializable { val nnz = numNonzeros // A dense vector needs 8 * size + 8 bytes, while a sparse vector needs 12 * nnz + 20 bytes. if (1.5 * (nnz + 1.0) < size) { - toSparse + toSparseWithSize(nnz) } else { toDense } @@ -171,13 +191,26 @@ sealed trait Vector extends Serializable { */ @Since("1.5.0") def argmax: Int + + /** + * Converts the vector to a JSON string. + */ + @Since("1.6.0") + def toJson: String + + /** + * Convert this vector to the new mllib-local representation. + * This does NOT copy the data; it copies references. + */ + @Since("2.0.0") + def asML: newlinalg.Vector } /** * :: AlphaComponent :: * * User-defined type for [[Vector]] which allows easy interaction with SQL - * via [[org.apache.spark.sql.DataFrame]]. + * via [[org.apache.spark.sql.Dataset]]. */ @AlphaComponent class VectorUDT extends UserDefinedType[Vector] { @@ -194,21 +227,21 @@ class VectorUDT extends UserDefinedType[Vector] { StructField("values", ArrayType(DoubleType, containsNull = false), nullable = true))) } - override def serialize(obj: Any): InternalRow = { + override def serialize(obj: Vector): InternalRow = { obj match { case SparseVector(size, indices, values) => - val row = new GenericMutableRow(4) + val row = new GenericInternalRow(4) row.setByte(0, 0) row.setInt(1, size) - row.update(2, new GenericArrayData(indices.map(_.asInstanceOf[Any]))) - row.update(3, new GenericArrayData(values.map(_.asInstanceOf[Any]))) + row.update(2, UnsafeArrayData.fromPrimitiveArray(indices)) + row.update(3, UnsafeArrayData.fromPrimitiveArray(values)) row case DenseVector(values) => - val row = new GenericMutableRow(4) + val row = new GenericInternalRow(4) row.setByte(0, 1) row.setNullAt(1) row.setNullAt(2) - row.update(3, new GenericArrayData(values.map(_.asInstanceOf[Any]))) + row.update(3, UnsafeArrayData.fromPrimitiveArray(values)) row } } @@ -254,7 +287,7 @@ class VectorUDT extends UserDefinedType[Vector] { /** * Factory methods for [[org.apache.spark.mllib.linalg.Vector]]. * We don't use the name `Vector` because Scala imports - * [[scala.collection.immutable.Vector]] by default. + * `scala.collection.immutable.Vector` by default. */ @Since("1.0.0") object Vectors { @@ -332,13 +365,34 @@ object Vectors { } /** - * Parses a string resulted from [[Vector.toString]] into a [[Vector]]. + * Parses a string resulted from `Vector.toString` into a [[Vector]]. */ @Since("1.1.0") def parse(s: String): Vector = { parseNumeric(NumericParser.parse(s)) } + /** + * Parses the JSON representation of a vector into a [[Vector]]. + */ + @Since("1.6.0") + def fromJson(json: String): Vector = { + implicit val formats = DefaultFormats + val jValue = parseJson(json) + (jValue \ "type").extract[Int] match { + case 0 => // sparse + val size = (jValue \ "size").extract[Int] + val indices = (jValue \ "indices").extract[Seq[Int]].toArray + val values = (jValue \ "values").extract[Seq[Double]].toArray + sparse(size, indices, values) + case 1 => // dense + val values = (jValue \ "values").extract[Seq[Double]].toArray + dense(values) + case _ => + throw new IllegalArgumentException(s"Cannot parse $json into a vector.") + } + } + private[mllib] def parseNumeric(any: Any): Vector = { any match { case values: Array[Double] => @@ -543,6 +597,17 @@ object Vectors { /** Max number of nonzero entries used in computing hash code. */ private[linalg] val MAX_HASH_NNZ = 128 + + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(v: newlinalg.Vector): Vector = v match { + case dv: newlinalg.DenseVector => + DenseVector.fromML(dv) + case sv: newlinalg.SparseVector => + SparseVector.fromML(sv) + } } /** @@ -561,7 +626,7 @@ class DenseVector @Since("1.0.0") ( @Since("1.0.0") override def toArray: Array[Double] = values - private[spark] override def toBreeze: BV[Double] = new BDV[Double](values) + private[spark] override def asBreeze: BV[Double] = new BDV[Double](values) @Since("1.0.0") override def apply(i: Int): Double = values(i) @@ -583,6 +648,8 @@ class DenseVector @Since("1.0.0") ( } } + override def equals(other: Any): Boolean = super.equals(other) + override def hashCode(): Int = { var result: Int = 31 + size var i = 0 @@ -616,9 +683,7 @@ class DenseVector @Since("1.0.0") ( nnz } - @Since("1.4.0") - override def toSparse: SparseVector = { - val nnz = numNonzeros + private[linalg] override def toSparseWithSize(nnz: Int): SparseVector = { val ii = new Array[Int](nnz) val vv = new Array[Double](nnz) var k = 0 @@ -650,6 +715,17 @@ class DenseVector @Since("1.0.0") ( maxIdx } } + + @Since("1.6.0") + override def toJson: String = { + val jValue = ("type" -> 1) ~ ("values" -> values.toSeq) + compact(render(jValue)) + } + + @Since("2.0.0") + override def asML: newlinalg.DenseVector = { + new newlinalg.DenseVector(values) + } } @Since("1.3.0") @@ -658,10 +734,18 @@ object DenseVector { /** Extracts the value array from a dense vector. */ @Since("1.3.0") def unapply(dv: DenseVector): Option[Array[Double]] = Some(dv.values) + + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(v: newlinalg.DenseVector): DenseVector = { + new DenseVector(v.values) + } } /** - * A sparse vector represented by an index array and an value array. + * A sparse vector represented by an index array and a value array. * * @param size size of the vector. * @param indices index array, assume to be strictly increasing. @@ -700,7 +784,7 @@ class SparseVector @Since("1.0.0") ( new SparseVector(size, indices.clone(), values.clone()) } - private[spark] override def toBreeze: BV[Double] = new BSV[Double](indices, values, size) + private[spark] override def asBreeze: BV[Double] = new BSV[Double](indices, values, size) @Since("1.6.0") override def foreachActive(f: (Int, Double) => Unit): Unit = { @@ -715,6 +799,8 @@ class SparseVector @Since("1.0.0") ( } } + override def equals(other: Any): Boolean = super.equals(other) + override def hashCode(): Int = { var result: Int = 31 + size val end = values.length @@ -748,9 +834,7 @@ class SparseVector @Since("1.0.0") ( nnz } - @Since("1.4.0") - override def toSparse: SparseVector = { - val nnz = numNonzeros + private[linalg] override def toSparseWithSize(nnz: Int): SparseVector = { if (nnz == numActives) { this } else { @@ -772,6 +856,8 @@ class SparseVector @Since("1.0.0") ( override def argmax: Int = { if (size == 0) { -1 + } else if (numActives == 0) { + 0 } else { // Find the max active entry. var maxIdx = indices(0) @@ -837,6 +923,20 @@ class SparseVector @Since("1.0.0") ( }.unzip new SparseVector(selectedIndices.length, sliceInds.toArray, sliceVals.toArray) } + + @Since("1.6.0") + override def toJson: String = { + val jValue = ("type" -> 0) ~ + ("size" -> size) ~ + ("indices" -> indices.toSeq) ~ + ("values" -> values.toSeq) + compact(render(jValue)) + } + + @Since("2.0.0") + override def asML: newlinalg.SparseVector = { + new newlinalg.SparseVector(size, indices, values) + } } @Since("1.3.0") @@ -844,4 +944,33 @@ object SparseVector { @Since("1.3.0") def unapply(sv: SparseVector): Option[(Int, Array[Int], Array[Double])] = Some((sv.size, sv.indices, sv.values)) + + /** + * Convert new linalg type to spark.mllib type. Light copy; only copies references + */ + @Since("2.0.0") + def fromML(v: newlinalg.SparseVector): SparseVector = { + new SparseVector(v.size, v.indices, v.values) + } +} + +/** + * Implicit methods available in Scala for converting [[org.apache.spark.mllib.linalg.Vector]] to + * [[org.apache.spark.ml.linalg.Vector]] and vice versa. + */ +private[spark] object VectorImplicits { + + implicit def mllibVectorToMLVector(v: Vector): newlinalg.Vector = v.asML + + implicit def mllibDenseVectorToMLDenseVector(v: DenseVector): newlinalg.DenseVector = v.asML + + implicit def mllibSparseVectorToMLSparseVector(v: SparseVector): newlinalg.SparseVector = v.asML + + implicit def mlVectorToMLlibVector(v: newlinalg.Vector): Vector = Vectors.fromML(v) + + implicit def mlDenseVectorToMLlibDenseVector(v: newlinalg.DenseVector): DenseVector = + Vectors.fromML(v).asInstanceOf[DenseVector] + + implicit def mlSparseVectorToMLlibSparseVector(v: newlinalg.SparseVector): SparseVector = + Vectors.fromML(v).asInstanceOf[SparseVector] } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 09527dcf5d9e..7caacd13b345 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -19,11 +19,12 @@ package org.apache.spark.mllib.linalg.distributed import scala.collection.mutable.ArrayBuffer -import breeze.linalg.{DenseMatrix => BDM} +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, Matrix => BM, SparseVector => BSV, Vector => BV} -import org.apache.spark.{Logging, Partitioner, SparkException} +import org.apache.spark.{Partitioner, SparkException} import org.apache.spark.annotation.Since -import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Matrix, SparseMatrix} +import org.apache.spark.internal.Logging +import org.apache.spark.mllib.linalg._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -176,7 +177,7 @@ class BlockMatrix @Since("1.3.0") ( val numColBlocks = math.ceil(numCols() * 1.0 / colsPerBlock).toInt private[mllib] def createPartitioner(): GridPartitioner = - GridPartitioner(numRowBlocks, numColBlocks, suggestedNumPartitions = blocks.partitions.size) + GridPartitioner(numRowBlocks, numColBlocks, suggestedNumPartitions = blocks.partitions.length) private lazy val blockInfo = blocks.mapValues(block => (block.numRows, block.numCols)).cache() @@ -256,23 +257,47 @@ class BlockMatrix @Since("1.3.0") ( val colStart = blockColIndex.toLong * colsPerBlock val entryValues = new ArrayBuffer[MatrixEntry]() mat.foreachActive { (i, j, v) => - if (v != 0.0) entryValues.append(new MatrixEntry(rowStart + i, colStart + j, v)) + if (v != 0.0) entryValues += new MatrixEntry(rowStart + i, colStart + j, v) } entryValues } new CoordinateMatrix(entryRDD, numRows(), numCols()) } + /** Converts to IndexedRowMatrix. The number of columns must be within the integer range. */ @Since("1.3.0") def toIndexedRowMatrix(): IndexedRowMatrix = { - require(numCols() < Int.MaxValue, "The number of columns must be within the integer range. " + - s"numCols: ${numCols()}") - // TODO: This implementation may be optimized - toCoordinateMatrix().toIndexedRowMatrix() + val cols = numCols().toInt + + require(cols < Int.MaxValue, s"The number of columns should be less than Int.MaxValue ($cols).") + + val rows = blocks.flatMap { case ((blockRowIdx, blockColIdx), mat) => + mat.rowIter.zipWithIndex.map { + case (vector, rowIdx) => + blockRowIdx * rowsPerBlock + rowIdx -> ((blockColIdx, vector.asBreeze)) + } + }.groupByKey().map { case (rowIdx, vectors) => + val numberNonZeroPerRow = vectors.map(_._2.activeSize).sum.toDouble / cols.toDouble + + val wholeVector = if (numberNonZeroPerRow <= 0.1) { // Sparse at 1/10th nnz + BSV.zeros[Double](cols) + } else { + BDV.zeros[Double](cols) + } + + vectors.foreach { case (blockColIdx: Int, vec: BV[_]) => + val offset = colsPerBlock * blockColIdx + wholeVector(offset until Math.min(cols, offset + colsPerBlock)) := vec + } + new IndexedRow(rowIdx, Vectors.fromBreeze(wholeVector)) + } + new IndexedRowMatrix(rows) } - /** Collect the distributed matrix on the driver as a `DenseMatrix`. */ + /** + * Collect the distributed matrix on the driver as a `DenseMatrix`. + */ @Since("1.3.0") def toLocalMatrix(): Matrix = { require(numRows() < Int.MaxValue, "The number of rows of this matrix should be less than " + @@ -317,40 +342,72 @@ class BlockMatrix @Since("1.3.0") ( } /** - * Adds two block matrices together. The matrices must have the same size and matching - * `rowsPerBlock` and `colsPerBlock` values. If one of the blocks that are being added are - * instances of [[SparseMatrix]], the resulting sub matrix will also be a [[SparseMatrix]], even - * if it is being added to a [[DenseMatrix]]. If two dense matrices are added, the output will - * also be a [[DenseMatrix]]. + * For given matrices `this` and `other` of compatible dimensions and compatible block dimensions, + * it applies a binary function on their corresponding blocks. + * + * @param other The second BlockMatrix argument for the operator specified by `binMap` + * @param binMap A function taking two breeze matrices and returning a breeze matrix + * @return A [[BlockMatrix]] whose blocks are the results of a specified binary map on blocks + * of `this` and `other`. + * Note: `blockMap` ONLY works for `add` and `subtract` methods and it does not support + * operators such as (a, b) => -a + b + * TODO: Make the use of zero matrices more storage efficient. */ - @Since("1.3.0") - def add(other: BlockMatrix): BlockMatrix = { + private[mllib] def blockMap( + other: BlockMatrix, + binMap: (BM[Double], BM[Double]) => BM[Double]): BlockMatrix = { require(numRows() == other.numRows(), "Both matrices must have the same number of rows. " + s"A.numRows: ${numRows()}, B.numRows: ${other.numRows()}") require(numCols() == other.numCols(), "Both matrices must have the same number of columns. " + s"A.numCols: ${numCols()}, B.numCols: ${other.numCols()}") if (rowsPerBlock == other.rowsPerBlock && colsPerBlock == other.colsPerBlock) { - val addedBlocks = blocks.cogroup(other.blocks, createPartitioner()) + val newBlocks = blocks.cogroup(other.blocks, createPartitioner()) .map { case ((blockRowIndex, blockColIndex), (a, b)) => if (a.size > 1 || b.size > 1) { throw new SparkException("There are multiple MatrixBlocks with indices: " + s"($blockRowIndex, $blockColIndex). Please remove them.") } if (a.isEmpty) { - new MatrixBlock((blockRowIndex, blockColIndex), b.head) + val zeroBlock = BM.zeros[Double](b.head.numRows, b.head.numCols) + val result = binMap(zeroBlock, b.head.asBreeze) + new MatrixBlock((blockRowIndex, blockColIndex), Matrices.fromBreeze(result)) } else if (b.isEmpty) { new MatrixBlock((blockRowIndex, blockColIndex), a.head) } else { - val result = a.head.toBreeze + b.head.toBreeze + val result = binMap(a.head.asBreeze, b.head.asBreeze) new MatrixBlock((blockRowIndex, blockColIndex), Matrices.fromBreeze(result)) } } - new BlockMatrix(addedBlocks, rowsPerBlock, colsPerBlock, numRows(), numCols()) + new BlockMatrix(newBlocks, rowsPerBlock, colsPerBlock, numRows(), numCols()) } else { - throw new SparkException("Cannot add matrices with different block dimensions") + throw new SparkException("Cannot perform on matrices with different block dimensions") } } + /** + * Adds the given block matrix `other` to `this` block matrix: `this + other`. + * The matrices must have the same size and matching `rowsPerBlock` and `colsPerBlock` + * values. If one of the blocks that are being added are instances of `SparseMatrix`, + * the resulting sub matrix will also be a `SparseMatrix`, even if it is being added + * to a `DenseMatrix`. If two dense matrices are added, the output will also be a + * `DenseMatrix`. + */ + @Since("1.3.0") + def add(other: BlockMatrix): BlockMatrix = + blockMap(other, (x: BM[Double], y: BM[Double]) => x + y) + + /** + * Subtracts the given block matrix `other` from `this` block matrix: `this - other`. + * The matrices must have the same size and matching `rowsPerBlock` and `colsPerBlock` + * values. If one of the blocks that are being subtracted are instances of `SparseMatrix`, + * the resulting sub matrix will also be a `SparseMatrix`, even if it is being subtracted + * from a `DenseMatrix`. If two dense matrices are subtracted, the output will also be a + * `DenseMatrix`. + */ + @Since("2.0.0") + def subtract(other: BlockMatrix): BlockMatrix = + blockMap(other, (x: BM[Double], y: BM[Double]) => x - y) + /** Block (i,j) --> Set of destination partitions */ private type BlockDestinations = Map[(Int, Int), Set[Int]] @@ -368,43 +425,78 @@ class BlockMatrix @Since("1.3.0") ( */ private[distributed] def simulateMultiply( other: BlockMatrix, - partitioner: GridPartitioner): (BlockDestinations, BlockDestinations) = { - val leftMatrix = blockInfo.keys.collect() // blockInfo should already be cached - val rightMatrix = other.blocks.keys.collect() + partitioner: GridPartitioner, + midDimSplitNum: Int): (BlockDestinations, BlockDestinations) = { + val leftMatrix = blockInfo.keys.collect() + val rightMatrix = other.blockInfo.keys.collect() + + val rightCounterpartsHelper = rightMatrix.groupBy(_._1).mapValues(_.map(_._2)) val leftDestinations = leftMatrix.map { case (rowIndex, colIndex) => - val rightCounterparts = rightMatrix.filter(_._1 == colIndex) - val partitions = rightCounterparts.map(b => partitioner.getPartition((rowIndex, b._2))) - ((rowIndex, colIndex), partitions.toSet) + val rightCounterparts = rightCounterpartsHelper.getOrElse(colIndex, Array.empty[Int]) + val partitions = rightCounterparts.map(b => partitioner.getPartition((rowIndex, b))) + val midDimSplitIndex = colIndex % midDimSplitNum + ((rowIndex, colIndex), + partitions.toSet.map((pid: Int) => pid * midDimSplitNum + midDimSplitIndex)) }.toMap + + val leftCounterpartsHelper = leftMatrix.groupBy(_._2).mapValues(_.map(_._1)) val rightDestinations = rightMatrix.map { case (rowIndex, colIndex) => - val leftCounterparts = leftMatrix.filter(_._2 == rowIndex) - val partitions = leftCounterparts.map(b => partitioner.getPartition((b._1, colIndex))) - ((rowIndex, colIndex), partitions.toSet) + val leftCounterparts = leftCounterpartsHelper.getOrElse(rowIndex, Array.empty[Int]) + val partitions = leftCounterparts.map(b => partitioner.getPartition((b, colIndex))) + val midDimSplitIndex = rowIndex % midDimSplitNum + ((rowIndex, colIndex), + partitions.toSet.map((pid: Int) => pid * midDimSplitNum + midDimSplitIndex)) }.toMap + (leftDestinations, rightDestinations) } /** * Left multiplies this [[BlockMatrix]] to `other`, another [[BlockMatrix]]. The `colsPerBlock` * of this matrix must equal the `rowsPerBlock` of `other`. If `other` contains - * [[SparseMatrix]], they will have to be converted to a [[DenseMatrix]]. The output - * [[BlockMatrix]] will only consist of blocks of [[DenseMatrix]]. This may cause + * `SparseMatrix`, they will have to be converted to a `DenseMatrix`. The output + * [[BlockMatrix]] will only consist of blocks of `DenseMatrix`. This may cause * some performance issues until support for multiplying two sparse matrices is added. * - * Note: The behavior of multiply has changed in 1.6.0. `multiply` used to throw an error when + * @note The behavior of multiply has changed in 1.6.0. `multiply` used to throw an error when * there were blocks with duplicate indices. Now, the blocks with duplicate indices will be added * with each other. */ @Since("1.3.0") def multiply(other: BlockMatrix): BlockMatrix = { + multiply(other, 1) + } + + /** + * Left multiplies this [[BlockMatrix]] to `other`, another [[BlockMatrix]]. The `colsPerBlock` + * of this matrix must equal the `rowsPerBlock` of `other`. If `other` contains + * `SparseMatrix`, they will have to be converted to a `DenseMatrix`. The output + * [[BlockMatrix]] will only consist of blocks of `DenseMatrix`. This may cause + * some performance issues until support for multiplying two sparse matrices is added. + * Blocks with duplicate indices will be added with each other. + * + * @param other Matrix `B` in `A * B = C` + * @param numMidDimSplits Number of splits to cut on the middle dimension when doing + * multiplication. For example, when multiplying a Matrix `A` of + * size `m x n` with Matrix `B` of size `n x k`, this parameter + * configures the parallelism to use when grouping the matrices. The + * parallelism will increase from `m x k` to `m x k x numMidDimSplits`, + * which in some cases also reduces total shuffled data. + */ + @Since("2.2.0") + def multiply( + other: BlockMatrix, + numMidDimSplits: Int): BlockMatrix = { require(numCols() == other.numRows(), "The number of columns of A and the number of rows " + s"of B must be equal. A.numCols: ${numCols()}, B.numRows: ${other.numRows()}. If you " + "think they should be equal, try setting the dimensions of A and B explicitly while " + "initializing them.") + require(numMidDimSplits > 0, "numMidDimSplits should be a positive integer.") if (colsPerBlock == other.rowsPerBlock) { val resultPartitioner = GridPartitioner(numRowBlocks, other.numColBlocks, math.max(blocks.partitions.length, other.blocks.partitions.length)) - val (leftDestinations, rightDestinations) = simulateMultiply(other, resultPartitioner) + val (leftDestinations, rightDestinations) + = simulateMultiply(other, resultPartitioner, numMidDimSplits) // Each block of A must be multiplied with the corresponding blocks in the columns of B. val flatA = blocks.flatMap { case ((blockRowIndex, blockColIndex), block) => val destinations = leftDestinations.getOrElse((blockRowIndex, blockColIndex), Set.empty) @@ -415,7 +507,11 @@ class BlockMatrix @Since("1.3.0") ( val destinations = rightDestinations.getOrElse((blockRowIndex, blockColIndex), Set.empty) destinations.map(j => (j, (blockRowIndex, blockColIndex, block))) } - val newBlocks = flatA.cogroup(flatB, resultPartitioner).flatMap { case (pId, (a, b)) => + val intermediatePartitioner = new Partitioner { + override def numPartitions: Int = resultPartitioner.numPartitions * numMidDimSplits + override def getPartition(key: Any): Int = key.asInstanceOf[Int] + } + val newBlocks = flatA.cogroup(flatB, intermediatePartitioner).flatMap { case (pId, (a, b)) => a.flatMap { case (leftRowIndex, leftColIndex, leftBlock) => b.filter(_._1 == leftColIndex).map { case (rightRowIndex, rightColIndex, rightBlock) => val C = rightBlock match { @@ -424,7 +520,7 @@ class BlockMatrix @Since("1.3.0") ( case _ => throw new SparkException(s"Unrecognized matrix type ${rightBlock.getClass}.") } - ((leftRowIndex, rightColIndex), C.toBreeze) + ((leftRowIndex, rightColIndex), C.asBreeze) } } }.reduceByKey(resultPartitioner, (a, b) => a + b).mapValues(Matrices.fromBreeze) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 8a70f34e70f6..0d223de9b6f7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -20,11 +20,11 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark.annotation.Since -import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.{Matrix, SparseMatrix, Vectors} +import org.apache.spark.rdd.RDD /** - * Represents an entry in an distributed matrix. + * Represents an entry in a distributed matrix. * @param i row index * @param j column index * @param value value of the entry @@ -101,14 +101,16 @@ class CoordinateMatrix @Since("1.0.0") ( toIndexedRowMatrix().toRowMatrix() } - /** Converts to BlockMatrix. Creates blocks of [[SparseMatrix]] with size 1024 x 1024. */ + /** + * Converts to BlockMatrix. Creates blocks of `SparseMatrix` with size 1024 x 1024. + */ @Since("1.3.0") def toBlockMatrix(): BlockMatrix = { toBlockMatrix(1024, 1024) } /** - * Converts to BlockMatrix. Creates blocks of [[SparseMatrix]]. + * Converts to BlockMatrix. Creates blocks of `SparseMatrix`. * @param rowsPerBlock The number of rows of each block. The blocks at the bottom edge may have * a smaller value. Must be an integer value greater than 0. * @param colsPerBlock The number of columns of each block. The blocks at the right edge may have @@ -123,6 +125,13 @@ class CoordinateMatrix @Since("1.0.0") ( s"colsPerBlock needs to be greater than 0. colsPerBlock: $colsPerBlock") val m = numRows() val n = numCols() + + // Since block matrices require an integer row and col index + require(math.ceil(m.toDouble / rowsPerBlock) <= Int.MaxValue, + "Number of rows divided by rowsPerBlock cannot exceed maximum integer.") + require(math.ceil(n.toDouble / colsPerBlock) <= Int.MaxValue, + "Number of cols divided by colsPerBlock cannot exceed maximum integer.") + val numRowBlocks = math.ceil(m.toDouble / rowsPerBlock).toInt val numColBlocks = math.ceil(n.toDouble / colsPerBlock).toInt val partitioner = GridPartitioner(numRowBlocks, numColBlocks, entries.partitions.length) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala index 976299124ced..8890662d99b5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -20,9 +20,9 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark.annotation.Since -import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.linalg.SingularValueDecomposition +import org.apache.spark.rdd.RDD /** * Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. @@ -90,14 +90,16 @@ class IndexedRowMatrix @Since("1.0.0") ( new RowMatrix(rows.map(_.vector), 0L, nCols) } - /** Converts to BlockMatrix. Creates blocks of [[SparseMatrix]] with size 1024 x 1024. */ + /** + * Converts to BlockMatrix. Creates blocks with size 1024 x 1024. + */ @Since("1.3.0") def toBlockMatrix(): BlockMatrix = { toBlockMatrix(1024, 1024) } /** - * Converts to BlockMatrix. Creates blocks of [[SparseMatrix]]. + * Converts to BlockMatrix. Blocks may be sparse or dense depending on the sparsity of the rows. * @param rowsPerBlock The number of rows of each block. The blocks at the bottom edge may have * a smaller value. Must be an integer value greater than 0. * @param colsPerBlock The number of columns of each block. The blocks at the right edge may have @@ -106,8 +108,70 @@ class IndexedRowMatrix @Since("1.0.0") ( */ @Since("1.3.0") def toBlockMatrix(rowsPerBlock: Int, colsPerBlock: Int): BlockMatrix = { - // TODO: This implementation may be optimized - toCoordinateMatrix().toBlockMatrix(rowsPerBlock, colsPerBlock) + require(rowsPerBlock > 0, + s"rowsPerBlock needs to be greater than 0. rowsPerBlock: $rowsPerBlock") + require(colsPerBlock > 0, + s"colsPerBlock needs to be greater than 0. colsPerBlock: $colsPerBlock") + + val m = numRows() + val n = numCols() + + // Since block matrices require an integer row index + require(math.ceil(m.toDouble / rowsPerBlock) <= Int.MaxValue, + "Number of rows divided by rowsPerBlock cannot exceed maximum integer.") + + // The remainder calculations only matter when m % rowsPerBlock != 0 or n % colsPerBlock != 0 + val remainderRowBlockIndex = m / rowsPerBlock + val remainderColBlockIndex = n / colsPerBlock + val remainderRowBlockSize = (m % rowsPerBlock).toInt + val remainderColBlockSize = (n % colsPerBlock).toInt + val numRowBlocks = math.ceil(m.toDouble / rowsPerBlock).toInt + val numColBlocks = math.ceil(n.toDouble / colsPerBlock).toInt + + val blocks = rows.flatMap { ir: IndexedRow => + val blockRow = ir.index / rowsPerBlock + val rowInBlock = ir.index % rowsPerBlock + + ir.vector match { + case SparseVector(size, indices, values) => + indices.zip(values).map { case (index, value) => + val blockColumn = index / colsPerBlock + val columnInBlock = index % colsPerBlock + ((blockRow.toInt, blockColumn.toInt), (rowInBlock.toInt, Array((value, columnInBlock)))) + } + case DenseVector(values) => + values.grouped(colsPerBlock) + .zipWithIndex + .map { case (values, blockColumn) => + ((blockRow.toInt, blockColumn), (rowInBlock.toInt, values.zipWithIndex)) + } + } + }.groupByKey(GridPartitioner(numRowBlocks, numColBlocks, rows.getNumPartitions)).map { + case ((blockRow, blockColumn), itr) => + val actualNumRows = + if (blockRow == remainderRowBlockIndex) remainderRowBlockSize else rowsPerBlock + val actualNumColumns = + if (blockColumn == remainderColBlockIndex) remainderColBlockSize else colsPerBlock + + val arraySize = actualNumRows * actualNumColumns + val matrixAsArray = new Array[Double](arraySize) + var countForValues = 0 + itr.foreach { case (rowWithinBlock, valuesWithColumns) => + valuesWithColumns.foreach { case (value, columnWithinBlock) => + matrixAsArray.update(columnWithinBlock * actualNumRows + rowWithinBlock, value) + countForValues += 1 + } + } + val denseMatrix = new DenseMatrix(actualNumRows, actualNumColumns, matrixAsArray) + val finalMatrix = if (countForValues / arraySize.toDouble >= 0.1) { + denseMatrix + } else { + denseMatrix.toSparse + } + + ((blockRow, blockColumn), finalMatrix) + } + new BlockMatrix(blocks, rowsPerBlock, colsPerBlock, m, n) } /** @@ -120,9 +184,9 @@ class IndexedRowMatrix @Since("1.0.0") ( val rowIndex = row.index row.vector match { case SparseVector(size, indices, values) => - Iterator.tabulate(indices.size)(i => MatrixEntry(rowIndex, indices(i), values(i))) + Iterator.tabulate(indices.length)(i => MatrixEntry(rowIndex, indices(i), values(i))) case DenseVector(values) => - Iterator.tabulate(values.size)(i => MatrixEntry(rowIndex, i, values(i))) + Iterator.tabulate(values.length)(i => MatrixEntry(rowIndex, i, values(i))) } } new CoordinateMatrix(entries, numRows(), numCols()) @@ -189,6 +253,8 @@ class IndexedRowMatrix @Since("1.0.0") ( /** * Computes the Gramian matrix `A^T A`. + * + * @note This cannot be computed on matrices with more than 65535 columns. */ @Since("1.0.0") def computeGramianMatrix(): Matrix = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 52c0f19c645d..78a8810052ae 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -21,17 +21,17 @@ import java.util.Arrays import scala.collection.mutable.ListBuffer -import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, SparseVector => BSV, axpy => brzAxpy, - svd => brzSvd, MatrixSingularException, inv} +import breeze.linalg.{axpy => brzAxpy, inv, svd => brzSvd, DenseMatrix => BDM, DenseVector => BDV, + MatrixSingularException, SparseVector => BSV} import breeze.numerics.{sqrt => brzSqrt} -import org.apache.spark.Logging import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, MultivariateStatisticalSummary} import org.apache.spark.rdd.RDD -import org.apache.spark.util.random.XORShiftRandom import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.random.XORShiftRandom /** * Represents a row-oriented distributed Matrix with no meaningful row indices. @@ -92,7 +92,7 @@ class RowMatrix @Since("1.0.0") ( val vbr = rows.context.broadcast(v) rows.treeAggregate(BDV.zeros[Double](n))( seqOp = (U, r) => { - val rBrz = r.toBreeze + val rBrz = r.asBreeze val a = rBrz.dot(vbr.value) rBrz match { // use specialized axpy for better performance @@ -106,8 +106,9 @@ class RowMatrix @Since("1.0.0") ( } /** - * Computes the Gramian matrix `A^T A`. Note that this cannot be computed on matrices with - * more than 65535 columns. + * Computes the Gramian matrix `A^T A`. + * + * @note This cannot be computed on matrices with more than 65535 columns. */ @Since("1.0.0") def computeGramianMatrix(): Matrix = { @@ -115,10 +116,10 @@ class RowMatrix @Since("1.0.0") ( checkNumColumns(n) // Computes n*(n+1)/2, avoiding overflow in the multiplication. // This succeeds when n <= 65535, which is checked above - val nt: Int = if (n % 2 == 0) ((n / 2) * (n + 1)) else (n * ((n + 1) / 2)) + val nt = if (n % 2 == 0) ((n / 2) * (n + 1)) else (n * ((n + 1) / 2)) // Compute the upper triangular part of the gram matrix. - val GU = rows.treeAggregate(new BDV[Double](new Array[Double](nt)))( + val GU = rows.treeAggregate(new BDV[Double](nt))( seqOp = (U, v) => { BLAS.spr(1.0, v, U.data) U @@ -168,9 +169,6 @@ class RowMatrix @Since("1.0.0") ( * ARPACK is set to 300 or k * 3, whichever is larger. The numerical tolerance for ARPACK's * eigen-decomposition is set to 1e-10. * - * @note The conditions that decide which method to use internally and the default parameters are - * subject to change. - * * @param k number of leading singular values to keep (0 < k <= n). * It might return less than k if * there are numerically zero singular values or there are not enough Ritz values @@ -180,6 +178,9 @@ class RowMatrix @Since("1.0.0") ( * @param rCond the reciprocal condition number. All singular values smaller than rCond * sigma(0) * are treated as zero, where sigma(0) is the largest singular value. * @return SingularValueDecomposition(U, s, V). U = null if computeU = false. + * + * @note The conditions that decide which method to use internally and the default parameters are + * subject to change. */ @Since("1.0.0") def computeSVD( @@ -250,12 +251,12 @@ class RowMatrix @Since("1.0.0") ( val (sigmaSquares: BDV[Double], u: BDM[Double]) = computeMode match { case SVDMode.LocalARPACK => require(k < n, s"k must be smaller than n in local-eigs mode but got k=$k and n=$n.") - val G = computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] + val G = computeGramianMatrix().asBreeze.asInstanceOf[BDM[Double]] EigenValueDecomposition.symmetricEigs(v => G * v, n, k, tol, maxIter) case SVDMode.LocalLAPACK => // breeze (v0.10) svd latent constraint, 7 * n * n + 4 * n < Int.MaxValue require(n < 17515, s"$n exceeds the breeze svd capability") - val G = computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] + val G = computeGramianMatrix().asBreeze.asInstanceOf[BDM[Double]] val brzSvd.SVD(uFull: BDM[Double], sigmaSquaresFull: BDV[Double], _) = brzSvd(G) (sigmaSquaresFull, uFull) case SVDMode.DistARPACK => @@ -319,34 +320,28 @@ class RowMatrix @Since("1.0.0") ( } /** - * Computes the covariance matrix, treating each row as an observation. Note that this cannot - * be computed on matrices with more than 65535 columns. + * Computes the covariance matrix, treating each row as an observation. + * * @return a local dense matrix of size n x n + * + * @note This cannot be computed on matrices with more than 65535 columns. */ @Since("1.0.0") def computeCovariance(): Matrix = { val n = numCols().toInt checkNumColumns(n) - val (m, mean) = rows.treeAggregate[(Long, BDV[Double])]((0L, BDV.zeros[Double](n)))( - seqOp = (s: (Long, BDV[Double]), v: Vector) => (s._1 + 1L, s._2 += v.toBreeze), - combOp = (s1: (Long, BDV[Double]), s2: (Long, BDV[Double])) => - (s1._1 + s2._1, s1._2 += s2._2) - ) - - if (m <= 1) { - sys.error(s"RowMatrix.computeCovariance called on matrix with only $m rows." + - " Cannot compute the covariance of a RowMatrix with <= 1 row.") - } - updateNumRows(m) - - mean :/= m.toDouble + val summary = computeColumnSummaryStatistics() + val m = summary.count + require(m > 1, s"RowMatrix.computeCovariance called on matrix with only $m rows." + + " Cannot compute the covariance of a RowMatrix with <= 1 row.") + val mean = summary.mean // We use the formula Cov(X, Y) = E[X * Y] - E[X] E[Y], which is not accurate if E[X * Y] is // large but Cov(X, Y) is small, but it is good for sparse computation. // TODO: find a fast and stable way for sparse data. - val G = computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] + val G = computeGramianMatrix().asBreeze var i = 0 var j = 0 @@ -368,7 +363,8 @@ class RowMatrix @Since("1.0.0") ( } /** - * Computes the top k principal components. + * Computes the top k principal components and a vector of proportions of + * variance explained by each principal component. * Rows correspond to observations and columns correspond to variables. * The principal components are stored a local matrix of size n-by-k. * Each column corresponds for one principal component, @@ -376,27 +372,45 @@ class RowMatrix @Since("1.0.0") ( * The row data do not need to be "centered" first; it is not necessary for * the mean of each column to be 0. * - * Note that this cannot be computed on matrices with more than 65535 columns. - * * @param k number of top principal components. - * @return a matrix of size n-by-k, whose columns are principal components + * @return a matrix of size n-by-k, whose columns are principal components, and + * a vector of values which indicate how much variance each principal component + * explains + * + * @note This cannot be computed on matrices with more than 65535 columns. */ - @Since("1.0.0") - def computePrincipalComponents(k: Int): Matrix = { + @Since("1.6.0") + def computePrincipalComponentsAndExplainedVariance(k: Int): (Matrix, Vector) = { val n = numCols().toInt require(k > 0 && k <= n, s"k = $k out of range (0, n = $n]") - val Cov = computeCovariance().toBreeze.asInstanceOf[BDM[Double]] + val Cov = computeCovariance().asBreeze.asInstanceOf[BDM[Double]] + + val brzSvd.SVD(u: BDM[Double], s: BDV[Double], _) = brzSvd(Cov) - val brzSvd.SVD(u: BDM[Double], _, _) = brzSvd(Cov) + val eigenSum = s.data.sum + val explainedVariance = s.data.map(_ / eigenSum) if (k == n) { - Matrices.dense(n, k, u.data) + (Matrices.dense(n, k, u.data), Vectors.dense(explainedVariance)) } else { - Matrices.dense(n, k, Arrays.copyOfRange(u.data, 0, n * k)) + (Matrices.dense(n, k, Arrays.copyOfRange(u.data, 0, n * k)), + Vectors.dense(Arrays.copyOfRange(explainedVariance, 0, k))) } } + /** + * Computes the top k principal components only. + * + * @param k number of top principal components. + * @return a matrix of size n-by-k, whose columns are principal components + * @see computePrincipalComponentsAndExplainedVariance + */ + @Since("1.0.0") + def computePrincipalComponents(k: Int): Matrix = { + computePrincipalComponentsAndExplainedVariance(k)._1 + } + /** * Computes column-wise summary statistics. */ @@ -425,14 +439,14 @@ class RowMatrix @Since("1.0.0") ( require(B.isInstanceOf[DenseMatrix], s"Only support dense matrix at this time but found ${B.getClass.getName}.") - val Bb = rows.context.broadcast(B.toBreeze.asInstanceOf[BDM[Double]].toDenseVector.toArray) + val Bb = rows.context.broadcast(B.asBreeze.asInstanceOf[BDM[Double]].toDenseVector.toArray) val AB = rows.mapPartitions { iter => val Bi = Bb.value iter.map { row => val v = BDV.zeros[Double](k) var i = 0 while (i < k) { - v(i) = row.toBreeze.dot(new BDV(Bi, i * n, 1, n)) + v(i) = row.asBreeze.dot(new BDV(Bi, i * n, 1, n)) i += 1 } Vectors.fromBreeze(v) @@ -517,7 +531,7 @@ class RowMatrix @Since("1.0.0") ( * decomposition (factorization) for the [[RowMatrix]] of a tall and skinny shape. * Reference: * Paul G. Constantine, David F. Gleich. "Tall and skinny QR factorizations in MapReduce - * architectures" ([[http://dx.doi.org/10.1145/1996092.1996103]]) + * architectures" (see here) * * @param computeQ whether to computeQ * @return QRDecomposition(Q, R), Q = null if computeQ = false. @@ -526,21 +540,22 @@ class RowMatrix @Since("1.0.0") ( def tallSkinnyQR(computeQ: Boolean = false): QRDecomposition[RowMatrix, Matrix] = { val col = numCols().toInt // split rows horizontally into smaller matrices, and compute QR for each of them - val blockQRs = rows.glom().map { partRows => + val blockQRs = rows.retag(classOf[Vector]).glom().filter(_.length != 0).map { partRows => val bdm = BDM.zeros[Double](partRows.length, col) var i = 0 partRows.foreach { row => - bdm(i, ::) := row.toBreeze.t + bdm(i, ::) := row.asBreeze.t i += 1 } breeze.linalg.qr.reduced(bdm).r } // combine the R part from previous results vertically into a tall matrix - val combinedR = blockQRs.treeReduce{ (r1, r2) => + val combinedR = blockQRs.treeReduce { (r1, r2) => val stackedR = BDM.vertcat(r1, r2) breeze.linalg.qr.reduced(stackedR).r } + val finalR = Matrices.fromBreeze(combinedR.toDenseMatrix) val finalQ = if (computeQ) { try { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 240baeb5a158..88c73241fb55 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -67,72 +67,93 @@ abstract class Gradient extends Serializable { * http://statweb.stanford.edu/~tibs/ElemStatLearn/ , Eq. (4.17) on page 119 gives the formula of * multinomial logistic regression model. A simple calculation shows that * - * {{{ - * P(y=0|x, w) = 1 / (1 + \sum_i^{K-1} \exp(x w_i)) - * P(y=1|x, w) = exp(x w_1) / (1 + \sum_i^{K-1} \exp(x w_i)) - * ... - * P(y=K-1|x, w) = exp(x w_{K-1}) / (1 + \sum_i^{K-1} \exp(x w_i)) - * }}} + *
    + * $$ + * P(y=0|x, w) = 1 / (1 + \sum_i^{K-1} \exp(x w_i))\\ + * P(y=1|x, w) = exp(x w_1) / (1 + \sum_i^{K-1} \exp(x w_i))\\ + * ...\\ + * P(y=K-1|x, w) = exp(x w_{K-1}) / (1 + \sum_i^{K-1} \exp(x w_i))\\ + * $$ + *
    * * for K classes multiclass classification problem. * - * The model weights w = (w_1, w_2, ..., w_{K-1})^T becomes a matrix which has dimension of + * The model weights \(w = (w_1, w_2, ..., w_{K-1})^T\) becomes a matrix which has dimension of * (K-1) * (N+1) if the intercepts are added. If the intercepts are not added, the dimension * will be (K-1) * N. * * As a result, the loss of objective function for a single instance of data can be written as - * {{{ - * l(w, x) = -log P(y|x, w) = -\alpha(y) log P(y=0|x, w) - (1-\alpha(y)) log P(y|x, w) - * = log(1 + \sum_i^{K-1}\exp(x w_i)) - (1-\alpha(y)) x w_{y-1} - * = log(1 + \sum_i^{K-1}\exp(margins_i)) - (1-\alpha(y)) margins_{y-1} - * }}} + *
    + * $$ + * \begin{align} + * l(w, x) &= -log P(y|x, w) = -\alpha(y) log P(y=0|x, w) - (1-\alpha(y)) log P(y|x, w) \\ + * &= log(1 + \sum_i^{K-1}\exp(x w_i)) - (1-\alpha(y)) x w_{y-1} \\ + * &= log(1 + \sum_i^{K-1}\exp(margins_i)) - (1-\alpha(y)) margins_{y-1} + * \end{align} + * $$ + *
    * - * where \alpha(i) = 1 if i != 0, and - * \alpha(i) = 0 if i == 0, - * margins_i = x w_i. + * where $\alpha(i) = 1$ if \(i \ne 0\), and + * $\alpha(i) = 0$ if \(i == 0\), + * \(margins_i = x w_i\). * * For optimization, we have to calculate the first derivative of the loss function, and * a simple calculation shows that * - * {{{ - * \frac{\partial l(w, x)}{\partial w_{ij}} - * = (\exp(x w_i) / (1 + \sum_k^{K-1} \exp(x w_k)) - (1-\alpha(y)\delta_{y, i+1})) * x_j - * = multiplier_i * x_j - * }}} + *
    + * $$ + * \begin{align} + * \frac{\partial l(w, x)}{\partial w_{ij}} &= + * (\exp(x w_i) / (1 + \sum_k^{K-1} \exp(x w_k)) - (1-\alpha(y)\delta_{y, i+1})) * x_j \\ + * &= multiplier_i * x_j + * \end{align} + * $$ + *
    * - * where \delta_{i, j} = 1 if i == j, - * \delta_{i, j} = 0 if i != j, and + * where $\delta_{i, j} = 1$ if \(i == j\), + * $\delta_{i, j} = 0$ if \(i != j\), and * multiplier = - * \exp(margins_i) / (1 + \sum_k^{K-1} \exp(margins_i)) - (1-\alpha(y)\delta_{y, i+1}) + * $\exp(margins_i) / (1 + \sum_k^{K-1} \exp(margins_i)) - (1-\alpha(y)\delta_{y, i+1})$ * * If any of margins is larger than 709.78, the numerical computation of multiplier and loss * function will be suffered from arithmetic overflow. This issue occurs when there are outliers * in data which are far away from hyperplane, and this will cause the failing of training once - * infinity / infinity is introduced. Note that this is only a concern when max(margins) > 0. + * infinity / infinity is introduced. Note that this is only a concern when max(margins) + * {@literal >} 0. * - * Fortunately, when max(margins) = maxMargin > 0, the loss function and the multiplier can be - * easily rewritten into the following equivalent numerically stable formula. + * Fortunately, when max(margins) = maxMargin {@literal >} 0, the loss function and the multiplier + * can be easily rewritten into the following equivalent numerically stable formula. * - * {{{ - * l(w, x) = log(1 + \sum_i^{K-1}\exp(margins_i)) - (1-\alpha(y)) margins_{y-1} - * = log(\exp(-maxMargin) + \sum_i^{K-1}\exp(margins_i - maxMargin)) + maxMargin - * - (1-\alpha(y)) margins_{y-1} - * = log(1 + sum) + maxMargin - (1-\alpha(y)) margins_{y-1} - * }}} + *
    + * $$ + * \begin{align} + * l(w, x) &= log(1 + \sum_i^{K-1}\exp(margins_i)) - (1-\alpha(y)) margins_{y-1} \\ + * &= log(\exp(-maxMargin) + \sum_i^{K-1}\exp(margins_i - maxMargin)) + maxMargin + * - (1-\alpha(y)) margins_{y-1} \\ + * &= log(1 + sum) + maxMargin - (1-\alpha(y)) margins_{y-1} + * \end{align} + * $$ + *
    * - * where sum = \exp(-maxMargin) + \sum_i^{K-1}\exp(margins_i - maxMargin) - 1. + * where sum = $\exp(-maxMargin) + \sum_i^{K-1}\exp(margins_i - maxMargin) - 1$. * - * Note that each term, (margins_i - maxMargin) in \exp is smaller than zero; as a result, + * Note that each term, $(margins_i - maxMargin)$ in $\exp$ is smaller than zero; as a result, * overflow will not happen with this formula. * * For multiplier, similar trick can be applied as the following, * - * {{{ - * multiplier = \exp(margins_i) / (1 + \sum_k^{K-1} \exp(margins_i)) - (1-\alpha(y)\delta_{y, i+1}) - * = \exp(margins_i - maxMargin) / (1 + sum) - (1-\alpha(y)\delta_{y, i+1}) - * }}} + *
    + * $$ + * \begin{align} + * multiplier + * &= \exp(margins_i) / + * (1 + \sum_k^{K-1} \exp(margins_i)) - (1-\alpha(y)\delta_{y, i+1}) \\ + * &= \exp(margins_i - maxMargin) / (1 + sum) - (1-\alpha(y)\delta_{y, i+1}) + * \end{align} + * $$ + *
    * - * where each term in \exp is also smaller than zero, so overflow is not a concern. + * where each term in $\exp$ is also smaller than zero, so overflow is not a concern. * * For the detailed mathematical derivation, see the reference at * http://www.slideshare.net/dbtsai/2014-0620-mlor-36132297 @@ -146,12 +167,6 @@ class LogisticGradient(numClasses: Int) extends Gradient { def this() = this(2) - override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { - val gradient = Vectors.zeros(weights.size) - val loss = compute(data, label, weights, gradient) - (gradient, loss) - } - override def compute( data: Vector, label: Double, @@ -291,7 +306,8 @@ class LeastSquaresGradient extends Gradient { * :: DeveloperApi :: * Compute gradient and loss for a Hinge loss function, as used in SVM binary classification. * See also the documentation for the precise formulation. - * NOTE: This assumes that the labels are {0,1} + * + * @note This assumes that the labels are {0,1} */ @DeveloperApi class HingeGradient extends Gradient { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 3b663b5defb0..593cdd602faf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -19,12 +19,12 @@ package org.apache.spark.mllib.optimization import scala.collection.mutable.ArrayBuffer -import breeze.linalg.{DenseVector => BDV, norm} +import breeze.linalg.{norm, DenseVector => BDV} -import org.apache.spark.annotation.{Experimental, DeveloperApi} -import org.apache.spark.Logging +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.rdd.RDD -import org.apache.spark.mllib.linalg.{Vectors, Vector} /** @@ -46,17 +46,19 @@ class GradientDescent private[spark] (private var gradient: Gradient, private va * In subsequent steps, the step size will decrease with stepSize/sqrt(t) */ def setStepSize(step: Double): this.type = { + require(step > 0, + s"Initial step size must be positive but got ${step}") this.stepSize = step this } /** - * :: Experimental :: * Set fraction of data to be used for each SGD iteration. * Default 1.0 (corresponding to deterministic/classical gradient descent) */ - @Experimental def setMiniBatchFraction(fraction: Double): this.type = { + require(fraction > 0 && fraction <= 1.0, + s"Fraction for mini-batch SGD must be in range (0, 1] but got ${fraction}") this.miniBatchFraction = fraction this } @@ -65,6 +67,8 @@ class GradientDescent private[spark] (private var gradient: Gradient, private va * Set the number of iterations for SGD. Default 100. */ def setNumIterations(iters: Int): this.type = { + require(iters >= 0, + s"Number of iterations must be nonnegative but got ${iters}") this.numIterations = iters this } @@ -73,6 +77,8 @@ class GradientDescent private[spark] (private var gradient: Gradient, private va * Set the regularization parameter. Default 0.0. */ def setRegParam(regParam: Double): this.type = { + require(regParam >= 0, + s"Regularization parameter must be nonnegative but got ${regParam}") this.regParam = regParam this } @@ -81,15 +87,18 @@ class GradientDescent private[spark] (private var gradient: Gradient, private va * Set the convergence tolerance. Default 0.001 * convergenceTol is a condition which decides iteration termination. * The end of iteration is decided based on below logic. - * - If the norm of the new solution vector is >1, the diff of solution vectors - * is compared to relative tolerance which means normalizing by the norm of - * the new solution vector. - * - If the norm of the new solution vector is <=1, the diff of solution vectors - * is compared to absolute tolerance which is not normalizing. + * + * - If the norm of the new solution vector is greater than 1, the diff of solution vectors + * is compared to relative tolerance which means normalizing by the norm of + * the new solution vector. + * - If the norm of the new solution vector is less than or equal to 1, the diff of solution + * vectors is compared to absolute tolerance which is not normalizing. + * * Must be between 0.0 and 1.0 inclusively. */ def setConvergenceTol(tolerance: Double): this.type = { - require(0.0 <= tolerance && tolerance <= 1.0) + require(tolerance >= 0.0 && tolerance <= 1.0, + s"Convergence tolerance must be in range [0, 1] but got ${tolerance}") this.convergenceTol = tolerance this } @@ -186,6 +195,11 @@ object GradientDescent extends Logging { "< 1.0 can be unstable because of the stochasticity in sampling.") } + if (numIterations * miniBatchFraction < 1.0) { + logWarning("Not all examples will be used if numIterations * miniBatchFraction < 1.0: " + + s"numIterations=$numIterations and miniBatchFraction=$miniBatchFraction") + } + val stochasticLossHistory = new ArrayBuffer[Double](numIterations) // Record previous weight and current one to calculate solution vector difference @@ -232,13 +246,14 @@ object GradientDescent extends Logging { // c: (grad, loss, count) (c1._1 += c2._1, c1._2 + c2._2, c1._3 + c2._3) }) + bcWeights.destroy(blocking = false) if (miniBatchSize > 0) { /** * lossSum is computed using the weights from the previous iteration * and regVal is the regularization value computed in the previous iteration as well. */ - stochasticLossHistory.append(lossSum / miniBatchSize + regVal) + stochasticLossHistory += lossSum / miniBatchSize + regVal val update = updater.compute( weights, Vectors.fromBreeze(gradientSum / miniBatchSize.toDouble), stepSize, i, regParam) @@ -265,7 +280,7 @@ object GradientDescent extends Logging { } /** - * Alias of [[runMiniBatchSGD]] with convergenceTol set to default value of 0.001. + * Alias of `runMiniBatchSGD` with convergenceTol set to default value of 0.001. */ def runMiniBatchSGD( data: RDD[(Double, Vector)], @@ -285,8 +300,8 @@ object GradientDescent extends Logging { currentWeights: Vector, convergenceTol: Double): Boolean = { // To compare with convergence tolerance. - val previousBDV = previousWeights.toBreeze.toDenseVector - val currentBDV = currentWeights.toBreeze.toDenseVector + val previousBDV = previousWeights.asBreeze.toDenseVector + val currentBDV = currentWeights.asBreeze.toDenseVector // This represents the difference of updated weights in the iteration. val solutionVecDiff: Double = norm(previousBDV - currentBDV) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index efedc112d380..21ec287e497d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -18,13 +18,12 @@ package org.apache.spark.mllib.optimization import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseVector => BDV} import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS} -import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.axpy import org.apache.spark.rdd.RDD @@ -32,7 +31,8 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: * Class used to solve an optimization problem using Limited-memory BFGS. - * Reference: [[http://en.wikipedia.org/wiki/Limited-memory_BFGS]] + * Reference: + * Wikipedia on Limited-memory BFGS * @param gradient Gradient function to be used. * @param updater Updater to be used to update weights after every iteration. */ @@ -41,7 +41,7 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) extends Optimizer with Logging { private var numCorrections = 10 - private var convergenceTol = 1E-4 + private var convergenceTol = 1E-6 private var maxNumIterations = 100 private var regParam = 0.0 @@ -49,51 +49,69 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) * Set the number of corrections used in the LBFGS update. Default 10. * Values of numCorrections less than 3 are not recommended; large values * of numCorrections will result in excessive computing time. - * 3 < numCorrections < 10 is recommended. - * Restriction: numCorrections > 0 + * numCorrections must be positive, and values from 4 to 9 are generally recommended. */ def setNumCorrections(corrections: Int): this.type = { - assert(corrections > 0) + require(corrections > 0, + s"Number of corrections must be positive but got ${corrections}") this.numCorrections = corrections this } /** - * Set the convergence tolerance of iterations for L-BFGS. Default 1E-4. + * Set the convergence tolerance of iterations for L-BFGS. Default 1E-6. * Smaller value will lead to higher accuracy with the cost of more iterations. * This value must be nonnegative. Lower convergence values are less tolerant * and therefore generally cause more iterations to be run. */ def setConvergenceTol(tolerance: Double): this.type = { + require(tolerance >= 0, + s"Convergence tolerance must be nonnegative but got ${tolerance}") this.convergenceTol = tolerance this } - /** - * Set the maximal number of iterations for L-BFGS. Default 100. - * @deprecated use [[LBFGS#setNumIterations]] instead + /* + * Get the convergence tolerance of iterations. */ - @deprecated("use setNumIterations instead", "1.1.0") - def setMaxNumIterations(iters: Int): this.type = { - this.setNumIterations(iters) + private[mllib] def getConvergenceTol(): Double = { + this.convergenceTol } /** * Set the maximal number of iterations for L-BFGS. Default 100. */ def setNumIterations(iters: Int): this.type = { + require(iters >= 0, + s"Maximum of iterations must be nonnegative but got ${iters}") this.maxNumIterations = iters this } + /** + * Get the maximum number of iterations for L-BFGS. Defaults to 100. + */ + private[mllib] def getNumIterations(): Int = { + this.maxNumIterations + } + /** * Set the regularization parameter. Default 0.0. */ def setRegParam(regParam: Double): this.type = { + require(regParam >= 0, + s"Regularization parameter must be nonnegative but got ${regParam}") this.regParam = regParam this } + /** + * Get the regularization parameter. + */ + private[mllib] def getRegParam(): Double = { + this.regParam + } + /** * Set the gradient function (of the loss function of one single data example) * to be used for L-BFGS. @@ -113,6 +131,13 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) this } + /** + * Returns the updater, limited to internal use. + */ + private[mllib] def getUpdater(): Updater = { + updater + } + override def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector = { val (weights, _) = LBFGS.runLBFGS( data, @@ -175,7 +200,7 @@ object LBFGS extends Logging { val lbfgs = new BreezeLBFGS[BDV[Double]](maxNumIterations, numCorrections, convergenceTol) val states = - lbfgs.iterations(new CachedDiffFunction(costFun), initialWeights.toBreeze.toDenseVector) + lbfgs.iterations(new CachedDiffFunction(costFun), initialWeights.asBreeze.toDenseVector) /** * NOTE: lossSum and loss is computed using the weights from the previous iteration @@ -187,6 +212,7 @@ object LBFGS extends Logging { state = states.next() } lossHistory += state.value + val weights = Vectors.fromBreeze(state.x) val lossHistoryArray = lossHistory.result() @@ -215,16 +241,27 @@ object LBFGS extends Logging { val bcW = data.context.broadcast(w) val localGradient = gradient - val (gradientSum, lossSum) = data.treeAggregate((Vectors.zeros(n), 0.0))( - seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => - val l = localGradient.compute( - features, label, bcW.value, grad) - (grad, loss + l) - }, - combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => - axpy(1.0, grad2, grad1) - (grad1, loss1 + loss2) - }) + val seqOp = (c: (Vector, Double), v: (Double, Vector)) => + (c, v) match { + case ((grad, loss), (label, features)) => + val denseGrad = grad.toDense + val l = localGradient.compute(features, label, bcW.value, denseGrad) + (denseGrad, loss + l) + } + + val combOp = (c1: (Vector, Double), c2: (Vector, Double)) => + (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => + val denseGrad1 = grad1.toDense + val denseGrad2 = grad2.toDense + axpy(1.0, denseGrad2, denseGrad1) + (denseGrad1, loss1 + loss2) + } + + val zeroSparseVector = Vectors.sparse(n, Seq.empty) + val (gradientSum, lossSum) = data.treeAggregate((zeroSparseVector, 0.0))(seqOp, combOp) + + // broadcasted model is not needed anymore + bcW.destroy(blocking = false) /** * regVal is sum of weight squares if it's L2 updater; @@ -256,7 +293,7 @@ object LBFGS extends Logging { // gradientTotal = gradientSum / numExamples + gradientTotal axpy(1.0 / numExamples, gradientSum, gradientTotal) - (loss, gradientTotal.toBreeze.asInstanceOf[BDV[Double]]) + (loss, gradientTotal.asBreeze.asInstanceOf[BDV[Double]]) } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala index 64d52bae0090..86632ae33595 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala @@ -53,8 +53,13 @@ private[spark] object NNLS { * projected gradient method. That is, find x minimising ||Ax - b||_2 given A^T A and A^T b. * * We solve the problem - * min_x 1/2 x^T ata x^T - x^T atb - * subject to x >= 0 + * + *
    + * $$ + * min_x 1/2 x^T ata x^T - x^T atb + * $$ + *
    + * where x is nonnegative. * * The method used is similar to one described by Polyak (B. T. Polyak, The conjugate gradient * method in extremal problems, Zh. Vychisl. Mat. Mat. Fiz. 9(4)(1969), pp. 94-112) for bound- diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index 7f6d94571b5e..d8e56720967d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -17,10 +17,9 @@ package org.apache.spark.mllib.optimization -import org.apache.spark.rdd.RDD - import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index 9f463e0cafb6..142f0ec6b902 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -19,10 +19,10 @@ package org.apache.spark.mllib.optimization import scala.math._ -import breeze.linalg.{norm => brzNorm, axpy => brzAxpy, Vector => BV} +import breeze.linalg.{axpy => brzAxpy, norm => brzNorm, Vector => BV} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * :: DeveloperApi :: @@ -75,8 +75,8 @@ class SimpleUpdater extends Updater { iter: Int, regParam: Double): (Vector, Double) = { val thisIterStepSize = stepSize / math.sqrt(iter) - val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector - brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) + val brzWeights: BV[Double] = weightsOld.asBreeze.toDenseVector + brzAxpy(-thisIterStepSize, gradient.asBreeze, brzWeights) (Vectors.fromBreeze(brzWeights), 0) } @@ -87,7 +87,7 @@ class SimpleUpdater extends Updater { * Updater for L1 regularized problems. * R(w) = ||w||_1 * Uses a step-size decreasing with the square root of the number of iterations. - + * * Instead of subgradient of the regularizer, the proximal operator for the * L1 regularization is applied after the gradient step. This is known to * result in better sparsity of the intermediate solution. @@ -95,9 +95,9 @@ class SimpleUpdater extends Updater { * The corresponding proximal operator for the L1 norm is the soft-thresholding * function. That is, each weight component is shrunk towards 0 by shrinkageVal. * - * If w > shrinkageVal, set weight component to w-shrinkageVal. - * If w < -shrinkageVal, set weight component to w+shrinkageVal. - * If -shrinkageVal < w < shrinkageVal, set weight component to 0. + * If w is greater than shrinkageVal, set weight component to w-shrinkageVal. + * If w is less than -shrinkageVal, set weight component to w+shrinkageVal. + * If w is (-shrinkageVal, shrinkageVal), set weight component to 0. * * Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) */ @@ -111,8 +111,8 @@ class L1Updater extends Updater { regParam: Double): (Vector, Double) = { val thisIterStepSize = stepSize / math.sqrt(iter) // Take gradient step - val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector - brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) + val brzWeights: BV[Double] = weightsOld.asBreeze.toDenseVector + brzAxpy(-thisIterStepSize, gradient.asBreeze, brzWeights) // Apply proximal operator (soft thresholding) val shrinkageVal = regParam * thisIterStepSize var i = 0 @@ -146,9 +146,9 @@ class SquaredL2Updater extends Updater { // w' = w - thisIterStepSize * (gradient + regParam * w) // w' = (1 - thisIterStepSize * regParam) * w - thisIterStepSize * gradient val thisIterStepSize = stepSize / math.sqrt(iter) - val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector + val brzWeights: BV[Double] = weightsOld.asBreeze.toDenseVector brzWeights :*= (1.0 - thisIterStepSize * regParam) - brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) + brzAxpy(-thisIterStepSize, gradient.asBreeze, brzWeights) val norm = brzNorm(brzWeights, 2.0) (Vectors.fromBreeze(brzWeights), 0.5 * regParam * norm * norm) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/package-info.java b/mllib/src/main/scala/org/apache/spark/mllib/package-info.java index 4991bc9e972c..72b71b7cd9b1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/package-info.java +++ b/mllib/src/main/scala/org/apache/spark/mllib/package-info.java @@ -16,6 +16,26 @@ */ /** - * Spark's machine learning library. + * RDD-based machine learning APIs (in maintenance mode). + * + * The spark.mllib package is in maintenance mode as of the Spark 2.0.0 release to + * encourage migration to the DataFrame-based APIs under the spark.ml package. + * While in maintenance mode, + *
      + *
    • + * no new features in the RDD-based spark.mllib package will be accepted, unless + * they block implementing new features in the DataFrame-based spark.ml package; + *
    • + *
    • + * bug fixes in the RDD-based APIs will still be accepted. + *
    • + *
    + * + * The developers will continue adding more features to the DataFrame-based APIs in the 2.x series + * to reach feature parity with the RDD-based APIs. + * And once we reach feature parity, this package will be deprecated. + * + * @see SPARK-4591 to + * track the progress of feature parity */ -package org.apache.spark.mllib; \ No newline at end of file +package org.apache.spark.mllib; diff --git a/mllib/src/main/scala/org/apache/spark/mllib/package.scala b/mllib/src/main/scala/org/apache/spark/mllib/package.scala index 5c2b2160c030..8323afcb6a83 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/package.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/package.scala @@ -18,6 +18,21 @@ package org.apache.spark /** - * Spark's machine learning library. + * RDD-based machine learning APIs (in maintenance mode). + * + * The `spark.mllib` package is in maintenance mode as of the Spark 2.0.0 release to encourage + * migration to the DataFrame-based APIs under the [[org.apache.spark.ml]] package. + * While in maintenance mode, + * + * - no new features in the RDD-based `spark.mllib` package will be accepted, unless they block + * implementing new features in the DataFrame-based `spark.ml` package; + * - bug fixes in the RDD-based APIs will still be accepted. + * + * The developers will continue adding more features to the DataFrame-based APIs in the 2.x series + * to reach feature parity with the RDD-based APIs. + * And once we reach feature parity, this package will be deprecated. + * + * @see SPARK-4591 to track + * the progress of feature parity */ package object mllib diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala index 274ac7c99553..5d61796f1de6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala @@ -23,7 +23,7 @@ import javax.xml.transform.stream.StreamResult import org.jpmml.model.JAXBUtil import org.apache.spark.SparkContext -import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.pmml.export.PMMLModelExportFactory /** @@ -45,20 +45,16 @@ trait PMMLExportable { } /** - * :: Experimental :: * Export the model to a local file in PMML format */ - @Experimental @Since("1.4.0") def toPMML(localPath: String): Unit = { toPMML(new StreamResult(new File(localPath))) } /** - * :: Experimental :: * Export the model to a directory on a distributed file system in PMML format */ - @Experimental @Since("1.4.0") def toPMML(sc: SparkContext, path: String): Unit = { val pmml = toPMML() @@ -66,20 +62,16 @@ trait PMMLExportable { } /** - * :: Experimental :: * Export the model to the OutputStream in PMML format */ - @Experimental @Since("1.4.0") def toPMML(outputStream: OutputStream): Unit = { toPMML(new StreamResult(outputStream)) } /** - * :: Experimental :: * Export the model to a String in PMML format */ - @Experimental @Since("1.4.0") def toPMML(): String = { val writer = new StringWriter diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala index 622b53a252ac..a8c32f72bfde 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala @@ -27,9 +27,9 @@ import org.apache.spark.mllib.regression.GeneralizedLinearModel * PMML Model Export for GeneralizedLinearModel class with binary ClassificationModel */ private[mllib] class BinaryClassificationPMMLModelExport( - model : GeneralizedLinearModel, - description : String, - normalizationMethod : RegressionNormalizationMethodType, + model: GeneralizedLinearModel, + description: String, + normalizationMethod: RegressionNormalizationMethodType, threshold: Double) extends PMMLModelExport { @@ -45,7 +45,7 @@ private[mllib] class BinaryClassificationPMMLModelExport( val fields = new SArray[FieldName](model.weights.size) val dataDictionary = new DataDictionary val miningSchema = new MiningSchema - val regressionTableYES = new RegressionTable(model.intercept).withTargetCategory("1") + val regressionTableYES = new RegressionTable(model.intercept).setTargetCategory("1") var interceptNO = threshold if (RegressionNormalizationMethodType.LOGIT == normalizationMethod) { if (threshold <= 0) { @@ -56,35 +56,35 @@ private[mllib] class BinaryClassificationPMMLModelExport( interceptNO = -math.log(1 / threshold - 1) } } - val regressionTableNO = new RegressionTable(interceptNO).withTargetCategory("0") + val regressionTableNO = new RegressionTable(interceptNO).setTargetCategory("0") val regressionModel = new RegressionModel() - .withFunctionName(MiningFunctionType.CLASSIFICATION) - .withMiningSchema(miningSchema) - .withModelName(description) - .withNormalizationMethod(normalizationMethod) - .withRegressionTables(regressionTableYES, regressionTableNO) + .setFunctionName(MiningFunctionType.CLASSIFICATION) + .setMiningSchema(miningSchema) + .setModelName(description) + .setNormalizationMethod(normalizationMethod) + .addRegressionTables(regressionTableYES, regressionTableNO) for (i <- 0 until model.weights.size) { fields(i) = FieldName.create("field_" + i) - dataDictionary.withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + dataDictionary.addDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) miningSchema - .withMiningFields(new MiningField(fields(i)) - .withUsageType(FieldUsageType.ACTIVE)) - regressionTableYES.withNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) + .addMiningFields(new MiningField(fields(i)) + .setUsageType(FieldUsageType.ACTIVE)) + regressionTableYES.addNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) } // add target field val targetField = FieldName.create("target") dataDictionary - .withDataFields(new DataField(targetField, OpType.CATEGORICAL, DataType.STRING)) + .addDataFields(new DataField(targetField, OpType.CATEGORICAL, DataType.STRING)) miningSchema - .withMiningFields(new MiningField(targetField) - .withUsageType(FieldUsageType.TARGET)) + .addMiningFields(new MiningField(targetField) + .setUsageType(FieldUsageType.TARGET)) - dataDictionary.withNumberOfFields(dataDictionary.getDataFields.size) + dataDictionary.setNumberOfFields(dataDictionary.getDataFields.size) pmml.setDataDictionary(dataDictionary) - pmml.withModels(regressionModel) + pmml.addModels(regressionModel) } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala index 1874786af000..4d951d2973a6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala @@ -45,31 +45,31 @@ private[mllib] class GeneralizedLinearPMMLModelExport( val miningSchema = new MiningSchema val regressionTable = new RegressionTable(model.intercept) val regressionModel = new RegressionModel() - .withFunctionName(MiningFunctionType.REGRESSION) - .withMiningSchema(miningSchema) - .withModelName(description) - .withRegressionTables(regressionTable) + .setFunctionName(MiningFunctionType.REGRESSION) + .setMiningSchema(miningSchema) + .setModelName(description) + .addRegressionTables(regressionTable) for (i <- 0 until model.weights.size) { fields(i) = FieldName.create("field_" + i) - dataDictionary.withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + dataDictionary.addDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) miningSchema - .withMiningFields(new MiningField(fields(i)) - .withUsageType(FieldUsageType.ACTIVE)) - regressionTable.withNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) + .addMiningFields(new MiningField(fields(i)) + .setUsageType(FieldUsageType.ACTIVE)) + regressionTable.addNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) } // for completeness add target field val targetField = FieldName.create("target") - dataDictionary.withDataFields(new DataField(targetField, OpType.CONTINUOUS, DataType.DOUBLE)) + dataDictionary.addDataFields(new DataField(targetField, OpType.CONTINUOUS, DataType.DOUBLE)) miningSchema - .withMiningFields(new MiningField(targetField) - .withUsageType(FieldUsageType.TARGET)) + .addMiningFields(new MiningField(targetField) + .setUsageType(FieldUsageType.TARGET)) - dataDictionary.withNumberOfFields(dataDictionary.getDataFields.size) + dataDictionary.setNumberOfFields(dataDictionary.getDataFields.size) pmml.setDataDictionary(dataDictionary) - pmml.withModels(regressionModel) + pmml.addModels(regressionModel) } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala index 069e7afc9fca..255c6140e541 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala @@ -26,14 +26,14 @@ import org.apache.spark.mllib.clustering.KMeansModel /** * PMML Model Export for KMeansModel class */ -private[mllib] class KMeansPMMLModelExport(model : KMeansModel) extends PMMLModelExport{ +private[mllib] class KMeansPMMLModelExport(model: KMeansModel) extends PMMLModelExport{ populateKMeansPMML(model) /** * Export the input KMeansModel model to PMML format. */ - private def populateKMeansPMML(model : KMeansModel): Unit = { + private def populateKMeansPMML(model: KMeansModel): Unit = { pmml.getHeader.setDescription("k-means clustering") if (model.clusterCenters.length > 0) { @@ -42,42 +42,42 @@ private[mllib] class KMeansPMMLModelExport(model : KMeansModel) extends PMMLMode val dataDictionary = new DataDictionary val miningSchema = new MiningSchema val comparisonMeasure = new ComparisonMeasure() - .withKind(ComparisonMeasure.Kind.DISTANCE) - .withMeasure(new SquaredEuclidean()) + .setKind(ComparisonMeasure.Kind.DISTANCE) + .setMeasure(new SquaredEuclidean()) val clusteringModel = new ClusteringModel() - .withModelName("k-means") - .withMiningSchema(miningSchema) - .withComparisonMeasure(comparisonMeasure) - .withFunctionName(MiningFunctionType.CLUSTERING) - .withModelClass(ClusteringModel.ModelClass.CENTER_BASED) - .withNumberOfClusters(model.clusterCenters.length) + .setModelName("k-means") + .setMiningSchema(miningSchema) + .setComparisonMeasure(comparisonMeasure) + .setFunctionName(MiningFunctionType.CLUSTERING) + .setModelClass(ClusteringModel.ModelClass.CENTER_BASED) + .setNumberOfClusters(model.clusterCenters.length) for (i <- 0 until clusterCenter.size) { fields(i) = FieldName.create("field_" + i) - dataDictionary.withDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) + dataDictionary.addDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) miningSchema - .withMiningFields(new MiningField(fields(i)) - .withUsageType(FieldUsageType.ACTIVE)) - clusteringModel.withClusteringFields( - new ClusteringField(fields(i)).withCompareFunction(CompareFunctionType.ABS_DIFF)) + .addMiningFields(new MiningField(fields(i)) + .setUsageType(FieldUsageType.ACTIVE)) + clusteringModel.addClusteringFields( + new ClusteringField(fields(i)).setCompareFunction(CompareFunctionType.ABS_DIFF)) } - dataDictionary.withNumberOfFields(dataDictionary.getDataFields.size) + dataDictionary.setNumberOfFields(dataDictionary.getDataFields.size) - for (i <- 0 until model.clusterCenters.length) { + for (i <- model.clusterCenters.indices) { val cluster = new Cluster() - .withName("cluster_" + i) - .withArray(new org.dmg.pmml.Array() - .withType(Array.Type.REAL) - .withN(clusterCenter.size) - .withValue(model.clusterCenters(i).toArray.mkString(" "))) + .setName("cluster_" + i) + .setArray(new org.dmg.pmml.Array() + .setType(Array.Type.REAL) + .setN(clusterCenter.size) + .setValue(model.clusterCenters(i).toArray.mkString(" "))) // we don't have the size of the single cluster but only the centroids (withValue) // .withSize(value) - clusteringModel.withClusters(cluster) + clusteringModel.addClusters(cluster) } pmml.setDataDictionary(dataDictionary) - pmml.withModels(clusteringModel) + pmml.addModels(clusteringModel) } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala index c5fdecd3ca17..f5ca1c221d66 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.pmml.export import java.text.SimpleDateFormat -import java.util.Date +import java.util.{Date, Locale} import scala.beans.BeanProperty @@ -30,18 +30,14 @@ private[mllib] trait PMMLModelExport { * Holder of the exported model in PMML format */ @BeanProperty - val pmml: PMML = new PMML - - setHeader(pmml) - - private def setHeader(pmml: PMML): Unit = { + val pmml: PMML = { val version = getClass.getPackage.getImplementationVersion - val app = new Application().withName("Apache Spark MLlib").withVersion(version) + val app = new Application("Apache Spark MLlib").setVersion(version) val timestamp = new Timestamp() - .withContent(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss").format(new Date())) + .addContent(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss", Locale.US).format(new Date())) val header = new Header() - .withApplication(app) - .withTimestamp(timestamp) - pmml.setHeader(header) + .setApplication(app) + .setTimestamp(timestamp) + new PMML("4.2", header, null) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala index 9eab7efc160d..fa04f8eb5e79 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala @@ -19,8 +19,8 @@ package org.apache.spark.mllib.random import org.apache.commons.math3.distribution._ -import org.apache.spark.annotation.{Since, DeveloperApi} -import org.apache.spark.util.random.{XORShiftRandom, Pseudorandom} +import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.util.random.{Pseudorandom, XORShiftRandom} /** * :: DeveloperApi :: diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala index b0a716936ae6..258b1763bba8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala @@ -57,7 +57,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#uniformRDD]]. + * Java-friendly version of `RandomRDDs.uniformRDD`. */ @Since("1.1.0") def uniformJavaRDD( @@ -69,7 +69,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#uniformJavaRDD]] with the default seed. + * `RandomRDDs.uniformJavaRDD` with the default seed. */ @Since("1.1.0") def uniformJavaRDD(jsc: JavaSparkContext, size: Long, numPartitions: Int): JavaDoubleRDD = { @@ -77,7 +77,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#uniformJavaRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.uniformJavaRDD` with the default number of partitions and the default seed. */ @Since("1.1.0") def uniformJavaRDD(jsc: JavaSparkContext, size: Long): JavaDoubleRDD = { @@ -107,7 +107,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#normalRDD]]. + * Java-friendly version of `RandomRDDs.normalRDD`. */ @Since("1.1.0") def normalJavaRDD( @@ -119,7 +119,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#normalJavaRDD]] with the default seed. + * `RandomRDDs.normalJavaRDD` with the default seed. */ @Since("1.1.0") def normalJavaRDD(jsc: JavaSparkContext, size: Long, numPartitions: Int): JavaDoubleRDD = { @@ -127,7 +127,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#normalJavaRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.normalJavaRDD` with the default number of partitions and the default seed. */ @Since("1.1.0") def normalJavaRDD(jsc: JavaSparkContext, size: Long): JavaDoubleRDD = { @@ -157,7 +157,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#poissonRDD]]. + * Java-friendly version of `RandomRDDs.poissonRDD`. */ @Since("1.1.0") def poissonJavaRDD( @@ -170,7 +170,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#poissonJavaRDD]] with the default seed. + * `RandomRDDs.poissonJavaRDD` with the default seed. */ @Since("1.1.0") def poissonJavaRDD( @@ -182,7 +182,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#poissonJavaRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.poissonJavaRDD` with the default number of partitions and the default seed. */ @Since("1.1.0") def poissonJavaRDD(jsc: JavaSparkContext, mean: Double, size: Long): JavaDoubleRDD = { @@ -212,7 +212,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#exponentialRDD]]. + * Java-friendly version of `RandomRDDs.exponentialRDD`. */ @Since("1.3.0") def exponentialJavaRDD( @@ -225,7 +225,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#exponentialJavaRDD]] with the default seed. + * `RandomRDDs.exponentialJavaRDD` with the default seed. */ @Since("1.3.0") def exponentialJavaRDD( @@ -237,7 +237,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#exponentialJavaRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.exponentialJavaRDD` with the default number of partitions and the default seed. */ @Since("1.3.0") def exponentialJavaRDD(jsc: JavaSparkContext, mean: Double, size: Long): JavaDoubleRDD = { @@ -249,8 +249,8 @@ object RandomRDDs { * shape and scale. * * @param sc SparkContext used to create the RDD. - * @param shape shape parameter (> 0) for the gamma distribution - * @param scale scale parameter (> 0) for the gamma distribution + * @param shape shape parameter (greater than 0) for the gamma distribution + * @param scale scale parameter (greater than 0) for the gamma distribution * @param size Size of the RDD. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). * @param seed Random seed (default: a random long integer). @@ -269,7 +269,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#gammaRDD]]. + * Java-friendly version of `RandomRDDs.gammaRDD`. */ @Since("1.3.0") def gammaJavaRDD( @@ -283,7 +283,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#gammaJavaRDD]] with the default seed. + * `RandomRDDs.gammaJavaRDD` with the default seed. */ @Since("1.3.0") def gammaJavaRDD( @@ -296,7 +296,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#gammaJavaRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.gammaJavaRDD` with the default number of partitions and the default seed. */ @Since("1.3.0") def gammaJavaRDD( @@ -332,7 +332,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#logNormalRDD]]. + * Java-friendly version of `RandomRDDs.logNormalRDD`. */ @Since("1.3.0") def logNormalJavaRDD( @@ -346,7 +346,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#logNormalJavaRDD]] with the default seed. + * `RandomRDDs.logNormalJavaRDD` with the default seed. */ @Since("1.3.0") def logNormalJavaRDD( @@ -359,7 +359,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#logNormalJavaRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.logNormalJavaRDD` with the default number of partitions and the default seed. */ @Since("1.3.0") def logNormalJavaRDD( @@ -418,7 +418,8 @@ object RandomRDDs { } /** - * [[RandomRDDs#randomJavaRDD]] with the default seed. + * :: DeveloperApi :: + * `RandomRDDs.randomJavaRDD` with the default seed. */ @DeveloperApi @Since("1.6.0") @@ -431,15 +432,16 @@ object RandomRDDs { } /** - * [[RandomRDDs#randomJavaRDD]] with the default seed & numPartitions + * :: DeveloperApi :: + * `RandomRDDs.randomJavaRDD` with the default seed & numPartitions */ @DeveloperApi @Since("1.6.0") def randomJavaRDD[T]( - jsc: JavaSparkContext, - generator: RandomDataGenerator[T], - size: Long): JavaRDD[T] = { - randomJavaRDD(jsc, generator, size, 0); + jsc: JavaSparkContext, + generator: RandomDataGenerator[T], + size: Long): JavaRDD[T] = { + randomJavaRDD(jsc, generator, size, 0) } // TODO Generate RDD[Vector] from multivariate distributions. @@ -467,7 +469,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#uniformVectorRDD]]. + * Java-friendly version of `RandomRDDs.uniformVectorRDD`. */ @Since("1.1.0") def uniformJavaVectorRDD( @@ -480,7 +482,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#uniformJavaVectorRDD]] with the default seed. + * `RandomRDDs.uniformJavaVectorRDD` with the default seed. */ @Since("1.1.0") def uniformJavaVectorRDD( @@ -492,7 +494,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#uniformJavaVectorRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.uniformJavaVectorRDD` with the default number of partitions and the default seed. */ @Since("1.1.0") def uniformJavaVectorRDD( @@ -525,7 +527,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#normalVectorRDD]]. + * Java-friendly version of `RandomRDDs.normalVectorRDD`. */ @Since("1.1.0") def normalJavaVectorRDD( @@ -538,7 +540,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#normalJavaVectorRDD]] with the default seed. + * `RandomRDDs.normalJavaVectorRDD` with the default seed. */ @Since("1.1.0") def normalJavaVectorRDD( @@ -550,7 +552,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#normalJavaVectorRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.normalJavaVectorRDD` with the default number of partitions and the default seed. */ @Since("1.1.0") def normalJavaVectorRDD( @@ -588,7 +590,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#logNormalVectorRDD]]. + * Java-friendly version of `RandomRDDs.logNormalVectorRDD`. */ @Since("1.3.0") def logNormalJavaVectorRDD( @@ -603,7 +605,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#logNormalJavaVectorRDD]] with the default seed. + * `RandomRDDs.logNormalJavaVectorRDD` with the default seed. */ @Since("1.3.0") def logNormalJavaVectorRDD( @@ -617,7 +619,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#logNormalJavaVectorRDD]] with the default number of partitions and + * `RandomRDDs.logNormalJavaVectorRDD` with the default number of partitions and * the default seed. */ @Since("1.3.0") @@ -655,7 +657,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#poissonVectorRDD]]. + * Java-friendly version of `RandomRDDs.poissonVectorRDD`. */ @Since("1.1.0") def poissonJavaVectorRDD( @@ -669,7 +671,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#poissonJavaVectorRDD]] with the default seed. + * `RandomRDDs.poissonJavaVectorRDD` with the default seed. */ @Since("1.1.0") def poissonJavaVectorRDD( @@ -682,7 +684,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#poissonJavaVectorRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.poissonJavaVectorRDD` with the default number of partitions and the default seed. */ @Since("1.1.0") def poissonJavaVectorRDD( @@ -719,7 +721,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#exponentialVectorRDD]]. + * Java-friendly version of `RandomRDDs.exponentialVectorRDD`. */ @Since("1.3.0") def exponentialJavaVectorRDD( @@ -733,7 +735,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#exponentialJavaVectorRDD]] with the default seed. + * `RandomRDDs.exponentialJavaVectorRDD` with the default seed. */ @Since("1.3.0") def exponentialJavaVectorRDD( @@ -746,7 +748,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#exponentialJavaVectorRDD]] with the default number of partitions + * `RandomRDDs.exponentialJavaVectorRDD` with the default number of partitions * and the default seed. */ @Since("1.3.0") @@ -764,8 +766,8 @@ object RandomRDDs { * gamma distribution with the input shape and scale. * * @param sc SparkContext used to create the RDD. - * @param shape shape parameter (> 0) for the gamma distribution. - * @param scale scale parameter (> 0) for the gamma distribution. + * @param shape shape parameter (greater than 0) for the gamma distribution. + * @param scale scale parameter (greater than 0) for the gamma distribution. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`) @@ -786,7 +788,7 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#gammaVectorRDD]]. + * Java-friendly version of `RandomRDDs.gammaVectorRDD`. */ @Since("1.3.0") def gammaJavaVectorRDD( @@ -801,7 +803,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#gammaJavaVectorRDD]] with the default seed. + * `RandomRDDs.gammaJavaVectorRDD` with the default seed. */ @Since("1.3.0") def gammaJavaVectorRDD( @@ -815,7 +817,7 @@ object RandomRDDs { } /** - * [[RandomRDDs#gammaJavaVectorRDD]] with the default number of partitions and the default seed. + * `RandomRDDs.gammaJavaVectorRDD` with the default number of partitions and the default seed. */ @Since("1.3.0") def gammaJavaVectorRDD( @@ -854,7 +856,8 @@ object RandomRDDs { } /** - * Java-friendly version of [[RandomRDDs#randomVectorRDD]]. + * :: DeveloperApi :: + * Java-friendly version of `RandomRDDs.randomVectorRDD`. */ @DeveloperApi @Since("1.6.0") @@ -869,7 +872,8 @@ object RandomRDDs { } /** - * [[RandomRDDs#randomJavaVectorRDD]] with the default seed. + * :: DeveloperApi :: + * `RandomRDDs.randomJavaVectorRDD` with the default seed. */ @DeveloperApi @Since("1.6.0") @@ -883,7 +887,8 @@ object RandomRDDs { } /** - * [[RandomRDDs#randomJavaVectorRDD]] with the default number of partitions and the default seed. + * :: DeveloperApi :: + * `RandomRDDs.randomJavaVectorRDD` with the default number of partitions and the default seed. */ @DeveloperApi @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala index 1b93e2d764c6..e28e1af5b0a2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctions.scala @@ -25,6 +25,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.util.BoundedPriorityQueue /** + * :: DeveloperApi :: * Machine learning specific Pair RDD functions. */ @DeveloperApi @@ -46,10 +47,13 @@ class MLPairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) extends Se combOp = (queue1, queue2) => { queue1 ++= queue2 } - ).mapValues(_.toArray.sorted(ord.reverse)) // This is an min-heap, so we reverse the order. + ).mapValues(_.toArray.sorted(ord.reverse)) // This is a min-heap, so we reverse the order. } } +/** + * :: DeveloperApi :: + */ @DeveloperApi object MLPairRDDFunctions { /** Implicit conversion from a pair RDD to MLPairRDDFunctions. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala index 78172843be56..32e6ecf6308e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala @@ -24,54 +24,39 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD /** + * :: DeveloperApi :: * Machine learning specific RDD functions. */ @DeveloperApi class RDDFunctions[T: ClassTag](self: RDD[T]) extends Serializable { /** - * Returns a RDD from grouping items of its parent RDD in fixed size blocks by passing a sliding + * Returns an RDD from grouping items of its parent RDD in fixed size blocks by passing a sliding * window over them. The ordering is first based on the partition index and then the ordering of * items within each partition. This is similar to sliding in Scala collections, except that it * becomes an empty RDD if the window size is greater than the total number of items. It needs to * trigger a Spark job if the parent RDD has more than one partitions and the window size is * greater than 1. */ - def sliding(windowSize: Int): RDD[Array[T]] = { + def sliding(windowSize: Int, step: Int): RDD[Array[T]] = { require(windowSize > 0, s"Sliding window size must be positive, but got $windowSize.") - if (windowSize == 1) { + if (windowSize == 1 && step == 1) { self.map(Array(_)) } else { - new SlidingRDD[T](self, windowSize) + new SlidingRDD[T](self, windowSize, step) } } /** - * Reduces the elements of this RDD in a multi-level tree pattern. - * - * @param depth suggested depth of the tree (default: 2) - * @see [[org.apache.spark.rdd.RDD#treeReduce]] - * @deprecated Use [[org.apache.spark.rdd.RDD#treeReduce]] instead. + * `sliding(Int, Int)*` with step = 1. */ - @deprecated("Use RDD.treeReduce instead.", "1.3.0") - def treeReduce(f: (T, T) => T, depth: Int = 2): T = self.treeReduce(f, depth) + def sliding(windowSize: Int): RDD[Array[T]] = sliding(windowSize, 1) - /** - * Aggregates the elements of this RDD in a multi-level tree pattern. - * - * @param depth suggested depth of the tree (default: 2) - * @see [[org.apache.spark.rdd.RDD#treeAggregate]] - * @deprecated Use [[org.apache.spark.rdd.RDD#treeAggregate]] instead. - */ - @deprecated("Use RDD.treeAggregate instead.", "1.3.0") - def treeAggregate[U: ClassTag](zeroValue: U)( - seqOp: (U, T) => U, - combOp: (U, U) => U, - depth: Int = 2): U = { - self.treeAggregate(zeroValue)(seqOp, combOp, depth) - } } +/** + * :: DeveloperApi :: + */ @DeveloperApi object RDDFunctions { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala index f8cea7ecea6b..92bc66949ae8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala @@ -17,15 +17,15 @@ package org.apache.spark.mllib.rdd +import scala.reflect.ClassTag +import scala.util.Random + import org.apache.spark.{Partition, SparkContext, TaskContext} import org.apache.spark.mllib.linalg.{DenseVector, Vector} import org.apache.spark.mllib.random.RandomDataGenerator import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils -import scala.reflect.ClassTag -import scala.util.Random - private[mllib] class RandomRDDPartition[T](override val index: Int, val size: Int, val generator: RandomDataGenerator[T], diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala index 1facf83d806d..365b2a06110f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala @@ -20,17 +20,17 @@ package org.apache.spark.mllib.rdd import scala.collection.mutable import scala.reflect.ClassTag -import org.apache.spark.{TaskContext, Partition} +import org.apache.spark.{Partition, TaskContext} import org.apache.spark.rdd.RDD private[mllib] -class SlidingRDDPartition[T](val idx: Int, val prev: Partition, val tail: Seq[T]) +class SlidingRDDPartition[T](val idx: Int, val prev: Partition, val tail: Seq[T], val offset: Int) extends Partition with Serializable { override val index: Int = idx } /** - * Represents a RDD from grouping items of its parent RDD in fixed size blocks by passing a sliding + * Represents an RDD from grouping items of its parent RDD in fixed size blocks by passing a sliding * window over them. The ordering is first based on the partition index and then the ordering of * items within each partition. This is similar to sliding in Scala collections, except that it * becomes an empty RDD if the window size is greater than the total number of items. It needs to @@ -40,19 +40,24 @@ class SlidingRDDPartition[T](val idx: Int, val prev: Partition, val tail: Seq[T] * * @param parent the parent RDD * @param windowSize the window size, must be greater than 1 + * @param step step size for windows * - * @see [[org.apache.spark.mllib.rdd.RDDFunctions#sliding]] + * @see `org.apache.spark.mllib.rdd.RDDFunctions.sliding(Int, Int)*` + * @see `scala.collection.IterableLike.sliding(Int, Int)*` */ private[mllib] -class SlidingRDD[T: ClassTag](@transient val parent: RDD[T], val windowSize: Int) +class SlidingRDD[T: ClassTag](@transient val parent: RDD[T], val windowSize: Int, val step: Int) extends RDD[Array[T]](parent) { - require(windowSize > 1, s"Window size must be greater than 1, but got $windowSize.") + require(windowSize > 0 && step > 0 && !(windowSize == 1 && step == 1), + "Window size and step must be greater than 0, " + + s"and they cannot be both 1, but got windowSize = $windowSize and step = $step.") override def compute(split: Partition, context: TaskContext): Iterator[Array[T]] = { val part = split.asInstanceOf[SlidingRDDPartition[T]] (firstParent[T].iterator(part.prev, context) ++ part.tail) - .sliding(windowSize) + .drop(part.offset) + .sliding(windowSize, step) .withPartial(false) .map(_.toArray) } @@ -62,40 +67,42 @@ class SlidingRDD[T: ClassTag](@transient val parent: RDD[T], val windowSize: Int override def getPartitions: Array[Partition] = { val parentPartitions = parent.partitions - val n = parentPartitions.size + val n = parentPartitions.length if (n == 0) { Array.empty } else if (n == 1) { - Array(new SlidingRDDPartition[T](0, parentPartitions(0), Seq.empty)) + Array(new SlidingRDDPartition[T](0, parentPartitions(0), Seq.empty, 0)) } else { - val n1 = n - 1 val w1 = windowSize - 1 - // Get the first w1 items of each partition, starting from the second partition. - val nextHeads = - parent.context.runJob(parent, (iter: Iterator[T]) => iter.take(w1).toArray, 1 until n) - val partitions = mutable.ArrayBuffer[SlidingRDDPartition[T]]() + // Get partition sizes and first w1 elements. + val (sizes, heads) = parent.mapPartitions { iter => + val w1Array = iter.take(w1).toArray + Iterator.single((w1Array.length + iter.length, w1Array)) + }.collect().unzip + val partitions = mutable.ArrayBuffer.empty[SlidingRDDPartition[T]] var i = 0 + var cumSize = 0 var partitionIndex = 0 - while (i < n1) { - var j = i - val tail = mutable.ListBuffer[T]() - // Keep appending to the current tail until appended a head of size w1. - while (j < n1 && nextHeads(j).size < w1) { - tail ++= nextHeads(j) - j += 1 + while (i < n) { + val mod = cumSize % step + val offset = if (mod == 0) 0 else step - mod + val size = sizes(i) + if (offset < size) { + val tail = mutable.ListBuffer.empty[T] + // Keep appending to the current tail until it has w1 elements. + var j = i + 1 + while (j < n && tail.length < w1) { + tail ++= heads(j).take(w1 - tail.length) + j += 1 + } + if (sizes(i) + tail.length >= offset + windowSize) { + partitions += + new SlidingRDDPartition[T](partitionIndex, parentPartitions(i), tail, offset) + partitionIndex += 1 + } } - if (j < n1) { - tail ++= nextHeads(j) - j += 1 - } - partitions += new SlidingRDDPartition[T](partitionIndex, parentPartitions(i), tail) - partitionIndex += 1 - // Skip appended heads. - i = j - } - // If the head of last partition has size w1, we also need to add this partition. - if (nextHeads.last.size == w1) { - partitions += new SlidingRDDPartition[T](partitionIndex, parentPartitions(n1), Seq.empty) + cumSize += size + i += 1 } partitions.toArray } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 33aaf853e599..14288221b694 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -17,9 +17,9 @@ package org.apache.spark.mllib.recommendation -import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.ml.recommendation.{ALS => NewALS} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -54,11 +54,12 @@ case class Rating @Since("0.8.0") ( * * For implicit preference data, the algorithm used is based on * "Collaborative Filtering for Implicit Feedback Datasets", available at - * [[http://dx.doi.org/10.1109/ICDM.2008.22]], adapted for the blocked approach used here. + * here, adapted for the blocked approach + * used here. * * Essentially instead of finding the low-rank approximations to the rating matrix `R`, * this finds the approximations for a preference matrix `P` where the elements of `P` are 1 if - * r > 0 and 0 if r <= 0. The ratings then act as 'confidence' values related to strength of + * r > 0 and 0 if r <= 0. The ratings then act as 'confidence' values related to strength of * indicated user * preferences rather than explicit ratings given to items. */ @@ -97,6 +98,8 @@ class ALS private ( */ @Since("0.8.0") def setBlocks(numBlocks: Int): this.type = { + require(numBlocks == -1 || numBlocks > 0, + s"Number of blocks must be -1 or positive but got ${numBlocks}") this.numUserBlocks = numBlocks this.numProductBlocks = numBlocks this @@ -107,6 +110,8 @@ class ALS private ( */ @Since("1.1.0") def setUserBlocks(numUserBlocks: Int): this.type = { + require(numUserBlocks == -1 || numUserBlocks > 0, + s"Number of blocks must be -1 or positive but got ${numUserBlocks}") this.numUserBlocks = numUserBlocks this } @@ -116,6 +121,8 @@ class ALS private ( */ @Since("1.1.0") def setProductBlocks(numProductBlocks: Int): this.type = { + require(numProductBlocks == -1 || numProductBlocks > 0, + s"Number of product blocks must be -1 or positive but got ${numProductBlocks}") this.numProductBlocks = numProductBlocks this } @@ -123,6 +130,8 @@ class ALS private ( /** Set the rank of the feature matrices computed (number of features). Default: 10. */ @Since("0.8.0") def setRank(rank: Int): this.type = { + require(rank > 0, + s"Rank of the feature matrices must be positive but got ${rank}") this.rank = rank this } @@ -130,6 +139,8 @@ class ALS private ( /** Set the number of iterations to run. Default: 10. */ @Since("0.8.0") def setIterations(iterations: Int): this.type = { + require(iterations >= 0, + s"Number of iterations must be nonnegative but got ${iterations}") this.iterations = iterations this } @@ -137,6 +148,8 @@ class ALS private ( /** Set the regularization parameter, lambda. Default: 0.01. */ @Since("0.8.0") def setLambda(lambda: Double): this.type = { + require(lambda >= 0.0, + s"Regularization parameter must be nonnegative but got ${lambda}") this.lambda = lambda this } @@ -204,6 +217,7 @@ class ALS private ( } /** + * :: DeveloperApi :: * Set period (in iterations) between checkpoints (default = 10). Checkpointing helps with * recovery (when nodes fail) and StackOverflow exceptions caused by long lineage. It also helps * with eliminating temporary shuffle files on disk, which can be important when there are many @@ -218,20 +232,22 @@ class ALS private ( } /** - * Run ALS with the configured parameters on an input RDD of (user, product, rating) triples. + * Run ALS with the configured parameters on an input RDD of [[Rating]] objects. * Returns a MatrixFactorizationModel with feature vectors for each user and product. */ @Since("0.8.0") def run(ratings: RDD[Rating]): MatrixFactorizationModel = { + require(!ratings.isEmpty(), s"No ratings available from $ratings") + val sc = ratings.context val numUserBlocks = if (this.numUserBlocks == -1) { - math.max(sc.defaultParallelism, ratings.partitions.size / 2) + math.max(sc.defaultParallelism, ratings.partitions.length / 2) } else { this.numUserBlocks } val numProductBlocks = if (this.numProductBlocks == -1) { - math.max(sc.defaultParallelism, ratings.partitions.size / 2) + math.max(sc.defaultParallelism, ratings.partitions.length / 2) } else { this.numProductBlocks } @@ -267,7 +283,7 @@ class ALS private ( } /** - * Java-friendly version of [[ALS.run]]. + * Java-friendly version of `ALS.run`. */ @Since("1.3.0") def run(ratings: JavaRDD[Rating]): MatrixFactorizationModel = run(ratings.rdd) @@ -279,18 +295,17 @@ class ALS private ( @Since("0.8.0") object ALS { /** - * Train a matrix factorization model given an RDD of ratings given by users to some products, - * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the - * product of two lower-rank matrices of a given rank (number of features). To solve for these - * features, we run a given number of iterations of ALS. This is done using a level of - * parallelism given by `blocks`. + * Train a matrix factorization model given an RDD of ratings by users for a subset of products. + * The ratings matrix is approximated as the product of two lower-rank matrices of a given rank + * (number of features). To solve for these features, ALS is run iteratively with a configurable + * level of parallelism. * - * @param ratings RDD of (userID, productID, rating) pairs - * @param rank number of features to use - * @param iterations number of iterations of ALS (recommended: 10-20) - * @param lambda regularization factor (recommended: 0.01) + * @param ratings RDD of [[Rating]] objects with userID, productID, and rating + * @param rank number of features to use (also referred to as the number of latent factors) + * @param iterations number of iterations of ALS + * @param lambda regularization parameter * @param blocks level of parallelism to split computation into - * @param seed random seed + * @param seed random seed for initial matrix factorization model */ @Since("0.9.1") def train( @@ -305,16 +320,15 @@ object ALS { } /** - * Train a matrix factorization model given an RDD of ratings given by users to some products, - * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the - * product of two lower-rank matrices of a given rank (number of features). To solve for these - * features, we run a given number of iterations of ALS. This is done using a level of - * parallelism given by `blocks`. + * Train a matrix factorization model given an RDD of ratings by users for a subset of products. + * The ratings matrix is approximated as the product of two lower-rank matrices of a given rank + * (number of features). To solve for these features, ALS is run iteratively with a configurable + * level of parallelism. * - * @param ratings RDD of (userID, productID, rating) pairs - * @param rank number of features to use - * @param iterations number of iterations of ALS (recommended: 10-20) - * @param lambda regularization factor (recommended: 0.01) + * @param ratings RDD of [[Rating]] objects with userID, productID, and rating + * @param rank number of features to use (also referred to as the number of latent factors) + * @param iterations number of iterations of ALS + * @param lambda regularization parameter * @param blocks level of parallelism to split computation into */ @Since("0.8.0") @@ -329,16 +343,15 @@ object ALS { } /** - * Train a matrix factorization model given an RDD of ratings given by users to some products, - * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the - * product of two lower-rank matrices of a given rank (number of features). To solve for these - * features, we run a given number of iterations of ALS. The level of parallelism is determined - * automatically based on the number of partitions in `ratings`. + * Train a matrix factorization model given an RDD of ratings by users for a subset of products. + * The ratings matrix is approximated as the product of two lower-rank matrices of a given rank + * (number of features). To solve for these features, ALS is run iteratively with a level of + * parallelism automatically based on the number of partitions in `ratings`. * - * @param ratings RDD of (userID, productID, rating) pairs - * @param rank number of features to use - * @param iterations number of iterations of ALS (recommended: 10-20) - * @param lambda regularization factor (recommended: 0.01) + * @param ratings RDD of [[Rating]] objects with userID, productID, and rating + * @param rank number of features to use (also referred to as the number of latent factors) + * @param iterations number of iterations of ALS + * @param lambda regularization parameter */ @Since("0.8.0") def train(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double) @@ -347,15 +360,14 @@ object ALS { } /** - * Train a matrix factorization model given an RDD of ratings given by users to some products, - * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the - * product of two lower-rank matrices of a given rank (number of features). To solve for these - * features, we run a given number of iterations of ALS. The level of parallelism is determined - * automatically based on the number of partitions in `ratings`. + * Train a matrix factorization model given an RDD of ratings by users for a subset of products. + * The ratings matrix is approximated as the product of two lower-rank matrices of a given rank + * (number of features). To solve for these features, ALS is run iteratively with a level of + * parallelism automatically based on the number of partitions in `ratings`. * - * @param ratings RDD of (userID, productID, rating) pairs - * @param rank number of features to use - * @param iterations number of iterations of ALS (recommended: 10-20) + * @param ratings RDD of [[Rating]] objects with userID, productID, and rating + * @param rank number of features to use (also referred to as the number of latent factors) + * @param iterations number of iterations of ALS */ @Since("0.8.0") def train(ratings: RDD[Rating], rank: Int, iterations: Int) @@ -371,12 +383,12 @@ object ALS { * a level of parallelism given by `blocks`. * * @param ratings RDD of (userID, productID, rating) pairs - * @param rank number of features to use - * @param iterations number of iterations of ALS (recommended: 10-20) - * @param lambda regularization factor (recommended: 0.01) + * @param rank number of features to use (also referred to as the number of latent factors) + * @param iterations number of iterations of ALS + * @param lambda regularization parameter * @param blocks level of parallelism to split computation into * @param alpha confidence parameter - * @param seed random seed + * @param seed random seed for initial matrix factorization model */ @Since("0.8.1") def trainImplicit( @@ -392,16 +404,15 @@ object ALS { } /** - * Train a matrix factorization model given an RDD of 'implicit preferences' given by users - * to some products, in the form of (userID, productID, preference) pairs. We approximate the - * ratings matrix as the product of two lower-rank matrices of a given rank (number of features). - * To solve for these features, we run a given number of iterations of ALS. This is done using - * a level of parallelism given by `blocks`. + * Train a matrix factorization model given an RDD of 'implicit preferences' of users for a + * subset of products. The ratings matrix is approximated as the product of two lower-rank + * matrices of a given rank (number of features). To solve for these features, ALS is run + * iteratively with a configurable level of parallelism. * - * @param ratings RDD of (userID, productID, rating) pairs - * @param rank number of features to use - * @param iterations number of iterations of ALS (recommended: 10-20) - * @param lambda regularization factor (recommended: 0.01) + * @param ratings RDD of [[Rating]] objects with userID, productID, and rating + * @param rank number of features to use (also referred to as the number of latent factors) + * @param iterations number of iterations of ALS + * @param lambda regularization parameter * @param blocks level of parallelism to split computation into * @param alpha confidence parameter */ @@ -418,16 +429,16 @@ object ALS { } /** - * Train a matrix factorization model given an RDD of 'implicit preferences' given by users to - * some products, in the form of (userID, productID, preference) pairs. We approximate the - * ratings matrix as the product of two lower-rank matrices of a given rank (number of features). - * To solve for these features, we run a given number of iterations of ALS. The level of - * parallelism is determined automatically based on the number of partitions in `ratings`. + * Train a matrix factorization model given an RDD of 'implicit preferences' of users for a + * subset of products. The ratings matrix is approximated as the product of two lower-rank + * matrices of a given rank (number of features). To solve for these features, ALS is run + * iteratively with a level of parallelism determined automatically based on the number of + * partitions in `ratings`. * - * @param ratings RDD of (userID, productID, rating) pairs - * @param rank number of features to use - * @param iterations number of iterations of ALS (recommended: 10-20) - * @param lambda regularization factor (recommended: 0.01) + * @param ratings RDD of [[Rating]] objects with userID, productID, and rating + * @param rank number of features to use (also referred to as the number of latent factors) + * @param iterations number of iterations of ALS + * @param lambda regularization parameter * @param alpha confidence parameter */ @Since("0.8.1") @@ -437,16 +448,15 @@ object ALS { } /** - * Train a matrix factorization model given an RDD of 'implicit preferences' ratings given by - * users to some products, in the form of (userID, productID, rating) pairs. We approximate the - * ratings matrix as the product of two lower-rank matrices of a given rank (number of features). - * To solve for these features, we run a given number of iterations of ALS. The level of - * parallelism is determined automatically based on the number of partitions in `ratings`. - * Model parameters `alpha` and `lambda` are set to reasonable default values + * Train a matrix factorization model given an RDD of 'implicit preferences' of users for a + * subset of products. The ratings matrix is approximated as the product of two lower-rank + * matrices of a given rank (number of features). To solve for these features, ALS is run + * iteratively with a level of parallelism determined automatically based on the number of + * partitions in `ratings`. * - * @param ratings RDD of (userID, productID, rating) pairs - * @param rank number of features to use - * @param iterations number of iterations of ALS (recommended: 10-20) + * @param ratings RDD of [[Rating]] objects with userID, productID, and rating + * @param rank number of features to use (also referred to as the number of latent factors) + * @param iterations number of iterations of ALS */ @Since("0.8.1") def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int) 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 46562eb2ad0f..ac709ad72f0c 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 @@ -20,8 +20,6 @@ package org.apache.spark.mllib.recommendation import java.io.IOException import java.lang.{Integer => JavaInteger} -import scala.collection.mutable - import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.hadoop.fs.Path @@ -29,27 +27,29 @@ import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} -import org.apache.spark.mllib.linalg._ +import org.apache.spark.internal.Logging +import org.apache.spark.mllib.linalg.BLAS import org.apache.spark.mllib.rdd.MLPairRDDFunctions._ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.BoundedPriorityQueue /** * Model representing the result of matrix factorization. * - * Note: If you create the model directly using constructor, please be aware that fast prediction - * requires cached user/product features and their associated partitioners. - * * @param rank Rank for the features in this model. * @param userFeatures RDD of tuples where each tuple represents the userId and * the features computed for this user. * @param productFeatures RDD of tuples where each tuple represents the productId * and the features computed for this product. + * + * @note If you create the model directly using constructor, please be aware that fast prediction + * requires cached user/product features and their associated partitioners. */ @Since("0.8.0") class MatrixFactorizationModel @Since("0.8.0") ( @@ -145,7 +145,7 @@ class MatrixFactorizationModel @Since("0.8.0") ( } /** - * Java-friendly version of [[MatrixFactorizationModel.predict]]. + * Java-friendly version of `MatrixFactorizationModel.predict`. */ @Since("1.2.0") def predict(usersProducts: JavaPairRDD[JavaInteger, JavaInteger]): JavaRDD[Rating] = { @@ -194,7 +194,7 @@ class MatrixFactorizationModel @Since("0.8.0") ( * - human-readable (JSON) model metadata to path/metadata/ * - Parquet formatted data to path/data/ * - * The model may be loaded using [[Loader.load]]. + * The model may be loaded using `Loader.load`. * * @param sc Spark context used to save model data. * @param path Path specifying the directory in which to save this model. @@ -206,7 +206,7 @@ class MatrixFactorizationModel @Since("0.8.0") ( } /** - * Recommends topK products for all users. + * Recommends top products for all users. * * @param num how many products to return for every user. * @return [(Int, Array[Rating])] objects, where every tuple contains a userID and an array of @@ -224,7 +224,7 @@ class MatrixFactorizationModel @Since("0.8.0") ( /** - * Recommends topK users for all products. + * Recommends top users for all products. * * @param num how many users to return for every product. * @return [(Int, Array[Rating])] objects, where every tuple contains a productID and an array @@ -261,6 +261,19 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { /** * Makes recommendations for all users (or products). + * + * Note: the previous approach used for computing top-k recommendations aimed to group + * individual factor vectors into blocks, so that Level 3 BLAS operations (gemm) could + * be used for efficiency. However, this causes excessive GC pressure due to the large + * arrays required for intermediate result storage, as well as a high sensitivity to the + * block size used. + * + * The following approach still groups factors into blocks, but instead computes the + * top-k elements per block, using dot product and an efficient [[BoundedPriorityQueue]] + * (instead of gemm). This avoids any large intermediate data structures and results + * in significantly reduced GC pressure as well as shuffle data, which far outweighs + * any cost incurred from not using Level 3 BLAS operations. + * * @param rank rank * @param srcFeatures src features to receive recommendations * @param dstFeatures dst features used to make recommendations @@ -273,53 +286,47 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { srcFeatures: RDD[(Int, Array[Double])], dstFeatures: RDD[(Int, Array[Double])], num: Int): RDD[(Int, Array[(Int, Double)])] = { - val srcBlocks = blockify(rank, srcFeatures) - val dstBlocks = blockify(rank, dstFeatures) - val ratings = srcBlocks.cartesian(dstBlocks).flatMap { - case ((srcIds, srcFactors), (dstIds, dstFactors)) => - val m = srcIds.length - val n = dstIds.length - val ratings = srcFactors.transpose.multiply(dstFactors) - val output = new Array[(Int, (Int, Double))](m * n) - var k = 0 - ratings.foreachActive { (i, j, r) => - output(k) = (srcIds(i), (dstIds(j), r)) - k += 1 + val srcBlocks = blockify(srcFeatures) + val dstBlocks = blockify(dstFeatures) + val ratings = srcBlocks.cartesian(dstBlocks).flatMap { case (srcIter, dstIter) => + val m = srcIter.size + val n = math.min(dstIter.size, num) + val output = new Array[(Int, (Int, Double))](m * n) + var i = 0 + val pq = new BoundedPriorityQueue[(Int, Double)](n)(Ordering.by(_._2)) + srcIter.foreach { case (srcId, srcFactor) => + dstIter.foreach { case (dstId, dstFactor) => + // We use F2jBLAS which is faster than a call to native BLAS for vector dot product + val score = BLAS.f2jBLAS.ddot(rank, srcFactor, 1, dstFactor, 1) + pq += dstId -> score + } + pq.foreach { case (dstId, score) => + output(i) = (srcId, (dstId, score)) + i += 1 } - output.toSeq + pq.clear() + } + output.toSeq } ratings.topByKey(num)(Ordering.by(_._2)) } /** - * Blockifies features to use Level-3 BLAS. + * Blockifies features to improve the efficiency of cartesian product + * TODO: SPARK-20443 - expose blockSize as a param? */ private def blockify( - rank: Int, - features: RDD[(Int, Array[Double])]): RDD[(Array[Int], DenseMatrix)] = { - val blockSize = 4096 // TODO: tune the block size - val blockStorage = rank * blockSize + features: RDD[(Int, Array[Double])], + blockSize: Int = 4096): RDD[Seq[(Int, Array[Double])]] = { features.mapPartitions { iter => - iter.grouped(blockSize).map { grouped => - val ids = mutable.ArrayBuilder.make[Int] - ids.sizeHint(blockSize) - val factors = mutable.ArrayBuilder.make[Double] - factors.sizeHint(blockStorage) - var i = 0 - grouped.foreach { case (id, factor) => - ids += id - factors ++= factor - i += 1 - } - (ids.result(), new DenseMatrix(rank, i, factors.result())) - } + iter.grouped(blockSize) } } /** * Load a model from the given path. * - * The model should have been saved by [[Saveable.save]]. + * The model should have been saved by `Saveable.save`. * * @param sc Spark context used for loading model files. * @param path Path specifying the directory to which the model was saved. @@ -353,8 +360,8 @@ 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._ + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + import spark.implicits._ val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("rank" -> model.rank))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) @@ -364,18 +371,18 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { def load(sc: SparkContext, path: String): MatrixFactorizationModel = { implicit val formats = DefaultFormats - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val (className, formatVersion, metadata) = loadMetadata(sc, path) assert(className == thisClassName) assert(formatVersion == thisFormatVersion) val rank = (metadata \ "rank").extract[Int] - val userFeatures = sqlContext.read.parquet(userPath(path)) - .map { case Row(id: Int, features: Seq[_]) => + val userFeatures = spark.read.parquet(userPath(path)).rdd.map { + case Row(id: Int, features: Seq[_]) => + (id, features.asInstanceOf[Seq[Double]].toArray) + } + val productFeatures = spark.read.parquet(productPath(path)).rdd.map { + case Row(id: Int, features: Seq[_]) => (id, features.asInstanceOf[Seq[Double]].toArray) - } - val productFeatures = sqlContext.read.parquet(productPath(path)) - .map { case Row(id: Int, features: Seq[_]) => - (id, features.asInstanceOf[Seq[Double]].toArray) } new MatrixFactorizationModel(rank, userFeatures, productFeatures) } 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 8f657bfb9c73..4d5aaf14111c 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 @@ -17,13 +17,14 @@ package org.apache.spark.mllib.regression +import org.apache.spark.SparkException import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging import org.apache.spark.mllib.feature.StandardScaler -import org.apache.spark.{Logging, SparkException} -import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.optimization._ -import org.apache.spark.mllib.linalg.{Vectors, Vector} import org.apache.spark.mllib.util.MLUtils._ +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel /** @@ -140,7 +141,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * translated back to resulting model weights, so it's transparent to users. * Note: This technique is used in both libsvm and glmnet packages. Default false. */ - private var useFeatureScaling = false + private[mllib] var useFeatureScaling = false /** * The dimension of training features. @@ -196,12 +197,9 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] } /** - * Run the algorithm with the configured parameters on an input - * RDD of LabeledPoint entries. - * + * Generate the initial weights when the user does not supply them */ - @Since("0.8.0") - def run(input: RDD[LabeledPoint]): M = { + protected def generateInitialWeights(input: RDD[LabeledPoint]): Vector = { if (numFeatures < 0) { numFeatures = input.map(_.features.size).first() } @@ -217,16 +215,23 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * TODO: See if we can deprecate `intercept` in `GeneralizedLinearModel`, and always * have the intercept as part of weights to have consistent design. */ - val initialWeights = { - if (numOfLinearPredictor == 1) { - Vectors.zeros(numFeatures) - } else if (addIntercept) { - Vectors.zeros((numFeatures + 1) * numOfLinearPredictor) - } else { - Vectors.zeros(numFeatures * numOfLinearPredictor) - } + if (numOfLinearPredictor == 1) { + Vectors.zeros(numFeatures) + } else if (addIntercept) { + Vectors.zeros((numFeatures + 1) * numOfLinearPredictor) + } else { + Vectors.zeros(numFeatures * numOfLinearPredictor) } - run(input, initialWeights) + } + + /** + * Run the algorithm with the configured parameters on an input + * RDD of LabeledPoint entries. + * + */ + @Since("0.8.0") + def run(input: RDD[LabeledPoint]): M = { + run(input, generateInitialWeights(input)) } /** @@ -346,7 +351,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] val partialWeightsArray = scaler.transform( Vectors.dense(weightsArray.slice(start, end))).toArray - System.arraycopy(partialWeightsArray, 0, weightsArray, start, partialWeightsArray.size) + System.arraycopy(partialWeightsArray, 0, weightsArray, start, partialWeightsArray.length) i += 1 } weights = Vectors.dense(weightsArray) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala index ec78ea24539b..8347ccad6b71 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala @@ -28,13 +28,13 @@ import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.SparkContext +import org.apache.spark.{RangePartitioner, SparkContext} import org.apache.spark.annotation.Since import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * Regression model for isotonic regression. @@ -136,7 +136,7 @@ class IsotonicRegressionModel @Since("1.3.0") ( // higher than all values, in between two values or exact match. if (insertIndex == 0) { predictions.head - } else if (insertIndex == boundaries.length){ + } else if (insertIndex == boundaries.length) { predictions.last } else if (foundIndex < 0) { linearInterpolation( @@ -185,21 +185,21 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { boundaries: Array[Double], predictions: Array[Double], isotonic: Boolean): Unit = { - val sqlContext = new SQLContext(sc) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() val metadata = compact(render( ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("isotonic" -> isotonic))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) - sqlContext.createDataFrame( + spark.createDataFrame( boundaries.toSeq.zip(predictions).map { case (b, p) => Data(b, p) } ).write.parquet(dataPath(path)) } def load(sc: SparkContext, path: String): (Array[Double], Array[Double]) = { - val sqlContext = new SQLContext(sc) - val dataRDD = sqlContext.read.parquet(dataPath(path)) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val dataRDD = spark.read.parquet(dataPath(path)) checkSchema[Data](dataRDD.schema) val dataArray = dataRDD.select("boundary", "prediction").collect() @@ -221,7 +221,7 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { val (boundaries, predictions) = SaveLoadV1_0.load(sc, path) new IsotonicRegressionModel(boundaries, predictions, isotonic) case _ => throw new Exception( - s"IsotonicRegressionModel.load did not recognize model with (className, format version):" + + s"IsotonicRegressionModel.load did not recognize model with (className, format version): " + s"($loadedClassName, $version). Supported:\n" + s" ($classNameV1_0, 1.0)" ) @@ -235,25 +235,23 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { * Only univariate (single feature) algorithm supported. * * Sequential PAV implementation based on: - * Tibshirani, Ryan J., Holger Hoefling, and Robert Tibshirani. - * "Nearly-isotonic regression." Technometrics 53.1 (2011): 54-61. - * Available from [[http://www.stat.cmu.edu/~ryantibs/papers/neariso.pdf]] + * Grotzinger, S. J., and C. Witzgall. + * "Projections onto order simplexes." Applied mathematics and Optimization 12.1 (1984): 247-270. * * Sequential PAV parallelization based on: * Kearsley, Anthony J., Richard A. Tapia, and Michael W. Trosset. * "An approach to parallelizing isotonic regression." * Applied Mathematics and Parallel Computing. Physica-Verlag HD, 1996. 141-147. - * Available from [[http://softlib.rice.edu/pub/CRPC-TRs/reports/CRPC-TR96640.pdf]] + * Available from here * - * @see [[http://en.wikipedia.org/wiki/Isotonic_regression Isotonic regression (Wikipedia)]] + * @see Isotonic regression + * (Wikipedia) */ @Since("1.3.0") class IsotonicRegression private (private var isotonic: Boolean) extends Serializable { /** * Constructs IsotonicRegression instance with default parameter isotonic = true. - * - * @return New instance of IsotonicRegression. */ @Since("1.3.0") def this() = this(true) @@ -312,90 +310,118 @@ class IsotonicRegression private (private var isotonic: Boolean) extends Seriali } /** - * Performs a pool adjacent violators algorithm (PAV). - * Uses approach with single processing of data where violators - * in previously processed data created by pooling are fixed immediately. - * Uses optimization of discovering monotonicity violating sequences (blocks). + * Performs a pool adjacent violators algorithm (PAV). Implements the algorithm originally + * described in [1], using the formulation from [2, 3]. Uses an array to keep track of start + * and end indices of blocks. * - * @param input Input data of tuples (label, feature, weight). + * [1] Grotzinger, S. J., and C. Witzgall. "Projections onto order simplexes." Applied + * mathematics and Optimization 12.1 (1984): 247-270. + * + * [2] Best, Michael J., and Nilotpal Chakravarti. "Active set algorithms for isotonic + * regression; a unifying framework." Mathematical Programming 47.1-3 (1990): 425-439. + * + * [3] Best, Michael J., Nilotpal Chakravarti, and Vasant A. Ubhaya. "Minimizing separable convex + * functions subject to simple chain constraints." SIAM Journal on Optimization 10.3 (2000): + * 658-672. + * + * @param input Input data of tuples (label, feature, weight). Weights must + be non-negative. * @return Result tuples (label, feature, weight) where labels were updated * to form a monotone sequence as per isotonic regression definition. */ private def poolAdjacentViolators( input: Array[(Double, Double, Double)]): Array[(Double, Double, Double)] = { - if (input.isEmpty) { - return Array.empty + val cleanInput = input.filter{ case (y, x, weight) => + require( + weight >= 0.0, + s"Negative weight at point ($y, $x, $weight). Weights must be non-negative" + ) + weight > 0 } - // Pools sub array within given bounds assigning weighted average value to all elements. - def pool(input: Array[(Double, Double, Double)], start: Int, end: Int): Unit = { - val poolSubArray = input.slice(start, end + 1) + if (cleanInput.isEmpty) { + return Array.empty + } - val weightedSum = poolSubArray.map(lp => lp._1 * lp._3).sum - val weight = poolSubArray.map(_._3).sum + // Keeps track of the start and end indices of the blocks. if [i, j] is a valid block from + // cleanInput(i) to cleanInput(j) (inclusive), then blockBounds(i) = j and blockBounds(j) = i + // Initially, each data point is its own block. + val blockBounds = Array.range(0, cleanInput.length) - var i = start - while (i <= end) { - input(i) = (weightedSum / weight, input(i)._2, input(i)._3) - i = i + 1 - } + // Keep track of the sum of weights and sum of weight * y for each block. weights(start) + // gives the values for the block. Entries that are not at the start of a block + // are meaningless. + val weights: Array[(Double, Double)] = cleanInput.map { case (y, _, weight) => + (weight, weight * y) } - var i = 0 - val len = input.length - while (i < len) { - var j = i + // a few convenience functions to make the code more readable - // Find monotonicity violating sequence, if any. - while (j < len - 1 && input(j)._1 > input(j + 1)._1) { - j = j + 1 - } + // blockStart and blockEnd have identical implementations. We create two different + // functions to make the code more expressive + def blockEnd(start: Int): Int = blockBounds(start) + def blockStart(end: Int): Int = blockBounds(end) - // If monotonicity was not violated, move to next data point. - if (i == j) { - i = i + 1 - } else { - // Otherwise pool the violating sequence - // and check if pooling caused monotonicity violation in previously processed points. - while (i >= 0 && input(i)._1 > input(i + 1)._1) { - pool(input, i, j) - i = i - 1 - } + // the next block starts at the index after the end of this block + def nextBlock(start: Int): Int = blockEnd(start) + 1 - i = j - } + // the previous block ends at the index before the start of this block + // we then use blockStart to find the start + def prevBlock(start: Int): Int = blockStart(start - 1) + + // Merge two adjacent blocks, updating blockBounds and weights to reflect the merge + // Return the start index of the merged block + def merge(block1: Int, block2: Int): Int = { + assert( + blockEnd(block1) + 1 == block2, + s"Attempting to merge non-consecutive blocks [${block1}, ${blockEnd(block1)}]" + + s" and [${block2}, ${blockEnd(block2)}]. This is likely a bug in the isotonic regression" + + " implementation. Please file a bug report." + ) + blockBounds(block1) = blockEnd(block2) + blockBounds(blockEnd(block2)) = block1 + val w1 = weights(block1) + val w2 = weights(block2) + weights(block1) = (w1._1 + w2._1, w1._2 + w2._2) + block1 } - // For points having the same prediction, we only keep two boundary points. - val compressed = ArrayBuffer.empty[(Double, Double, Double)] + // average value of a block + def average(start: Int): Double = weights(start)._2 / weights(start)._1 - var (curLabel, curFeature, curWeight) = input.head - var rightBound = curFeature - def merge(): Unit = { - compressed += ((curLabel, curFeature, curWeight)) - if (rightBound > curFeature) { - compressed += ((curLabel, rightBound, 0.0)) + // Implement Algorithm PAV from [3]. + // Merge on >= instead of > because it eliminates adjacent blocks with the same average, and we + // want to compress our output as much as possible. Both give correct results. + var i = 0 + while (nextBlock(i) < cleanInput.length) { + if (average(i) >= average(nextBlock(i))) { + merge(i, nextBlock(i)) + while((i > 0) && (average(prevBlock(i)) >= average(i))) { + i = merge(prevBlock(i), i) + } + } else { + i = nextBlock(i) } } - i = 1 - while (i < input.length) { - val (label, feature, weight) = input(i) - if (label == curLabel) { - curWeight += weight - rightBound = feature + + // construct the output by walking through the blocks in order + val output = ArrayBuffer.empty[(Double, Double, Double)] + i = 0 + while (i < cleanInput.length) { + // If block size is > 1, a point at the start and end of the block, + // each receiving half the weight. Otherwise, a single point with + // all the weight. + if (cleanInput(blockEnd(i))._2 > cleanInput(i)._2) { + output += ((average(i), cleanInput(i)._2, weights(i)._1 / 2)) + output += ((average(i), cleanInput(blockEnd(i))._2, weights(i)._1 / 2)) } else { - merge() - curLabel = label - curFeature = feature - curWeight = weight - rightBound = curFeature + output += ((average(i), cleanInput(i)._2, weights(i)._1)) } - i += 1 + i = nextBlock(i) } - merge() - compressed.toArray + output.toArray } /** @@ -408,9 +434,11 @@ class IsotonicRegression private (private var isotonic: Boolean) extends Seriali */ private def parallelPoolAdjacentViolators( input: RDD[(Double, Double, Double)]): Array[(Double, Double, Double)] = { - val parallelStepResult = input - .sortBy(x => (x._2, x._1)) - .glom() + val keyedInput = input.keyBy(_._2) + val parallelStepResult = keyedInput + .partitionBy(new RangePartitioner(keyedInput.getNumPartitions, keyedInput)) + .values + .mapPartitions(p => Iterator(p.toArray.sortBy(x => (x._2, x._1)))) .flatMap(poolAdjacentViolators) .collect() .sortBy(x => (x._2, x._1)) // Sort again because collect() doesn't promise ordering. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala index c284ad232537..4381d6ab20cc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -19,10 +19,11 @@ package org.apache.spark.mllib.regression import scala.beans.BeanInfo +import org.apache.spark.SparkException import org.apache.spark.annotation.Since -import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.NumericParser -import org.apache.spark.SparkException /** * Class that represents the features and labels of a data point. @@ -38,6 +39,10 @@ case class LabeledPoint @Since("1.0.0") ( override def toString: String = { s"($label,$features)" } + + private[spark] def asML: NewLabeledPoint = { + NewLabeledPoint(label, features.asML) + } } /** @@ -67,4 +72,8 @@ object LabeledPoint { LabeledPoint(label, features) } } + + private[spark] def fromML(point: NewLabeledPoint): LabeledPoint = { + LabeledPoint(point.label, Vectors.fromML(point.features)) + } } 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 a9aba173fa0e..cef1b4f51b84 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 @@ -23,7 +23,7 @@ import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression.impl.GLMRegressionModel -import org.apache.spark.mllib.util.{Saveable, Loader} +import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD /** @@ -44,7 +44,7 @@ class LassoModel @Since("1.1.0") ( dataMatrix: Vector, weightMatrix: Vector, intercept: Double): Double = { - weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept + weightMatrix.asBreeze.dot(dataMatrix.asBreeze) + intercept } @Since("1.3.0") @@ -85,7 +85,7 @@ object LassoModel extends Loader[LassoModel] { * See also the documentation for the precise formulation. */ @Since("0.8.0") -class LassoWithSGD private ( +class LassoWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, private var regParam: Double, @@ -106,6 +106,8 @@ class LassoWithSGD private ( * regParam: 0.01, miniBatchFraction: 1.0}. */ @Since("0.8.0") + @deprecated("Use ml.regression.LinearRegression with elasticNetParam = 1.0. Note the default " + + "regParam is 0.01 for LassoWithSGD, but is 0.0 for LinearRegression.", "2.0.0") def this() = this(1.0, 100, 0.01, 1.0) override protected def createModel(weights: Vector, intercept: Double) = { @@ -118,6 +120,8 @@ class LassoWithSGD private ( * */ @Since("0.8.0") +@deprecated("Use ml.regression.LinearRegression with elasticNetParam = 1.0. Note the default " + + "regParam is 0.01 for LassoWithSGD, but is 0.0 for LinearRegression.", "2.0.0") object LassoWithSGD { /** 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 4996ace5df85..60262fdc497a 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 @@ -23,7 +23,7 @@ import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.pmml.PMMLExportable import org.apache.spark.mllib.regression.impl.GLMRegressionModel -import org.apache.spark.mllib.util.{Saveable, Loader} +import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD /** @@ -44,7 +44,7 @@ class LinearRegressionModel @Since("1.1.0") ( dataMatrix: Vector, weightMatrix: Vector, intercept: Double): Double = { - weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept + weightMatrix.asBreeze.dot(dataMatrix.asBreeze) + intercept } @Since("1.3.0") @@ -89,6 +89,7 @@ object LinearRegressionModel extends Loader[LinearRegressionModel] { class LinearRegressionWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, + private var regParam: Double, private var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[LinearRegressionModel] with Serializable { @@ -98,6 +99,7 @@ class LinearRegressionWithSGD private[mllib] ( override val optimizer = new GradientDescent(gradient, updater) .setStepSize(stepSize) .setNumIterations(numIterations) + .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) /** @@ -105,7 +107,8 @@ class LinearRegressionWithSGD private[mllib] ( * numIterations: 100, miniBatchFraction: 1.0}. */ @Since("0.8.0") - def this() = this(1.0, 100, 1.0) + @deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0") + def this() = this(1.0, 100, 0.0, 1.0) override protected[mllib] def createModel(weights: Vector, intercept: Double) = { new LinearRegressionModel(weights, intercept) @@ -117,6 +120,7 @@ class LinearRegressionWithSGD private[mllib] ( * */ @Since("0.8.0") +@deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0") object LinearRegressionWithSGD { /** @@ -141,7 +145,7 @@ object LinearRegressionWithSGD { stepSize: Double, miniBatchFraction: Double, initialWeights: Vector): LinearRegressionModel = { - new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction) + new LinearRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction) .run(input, initialWeights) } @@ -163,7 +167,7 @@ object LinearRegressionWithSGD { numIterations: Int, stepSize: Double, miniBatchFraction: Double): LinearRegressionModel = { - new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction).run(input) + new LinearRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run(input) } /** 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 0a44ff559d55..52977ac4f062 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 @@ -45,7 +45,7 @@ class RidgeRegressionModel @Since("1.1.0") ( dataMatrix: Vector, weightMatrix: Vector, intercept: Double): Double = { - weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept + weightMatrix.asBreeze.dot(dataMatrix.asBreeze) + intercept } @Since("1.3.0") @@ -86,7 +86,7 @@ object RidgeRegressionModel extends Loader[RidgeRegressionModel] { * See also the documentation for the precise formulation. */ @Since("0.8.0") -class RidgeRegressionWithSGD private ( +class RidgeRegressionWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, private var regParam: Double, @@ -107,6 +107,8 @@ class RidgeRegressionWithSGD private ( * regParam: 0.01, miniBatchFraction: 1.0}. */ @Since("0.8.0") + @deprecated("Use ml.regression.LinearRegression with elasticNetParam = 0.0. Note the default " + + "regParam is 0.01 for RidgeRegressionWithSGD, but is 0.0 for LinearRegression.", "2.0.0") def this() = this(1.0, 100, 0.01, 1.0) override protected def createModel(weights: Vector, intercept: Double) = { @@ -119,6 +121,8 @@ class RidgeRegressionWithSGD private ( * */ @Since("0.8.0") +@deprecated("Use ml.regression.LinearRegression with elasticNetParam = 0.0. Note the default " + + "regParam is 0.01 for RidgeRegressionWithSGD, but is 0.0 for LinearRegression.", "2.0.0") object RidgeRegressionWithSGD { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala index 73948b2d9851..f44c8fe35145 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala @@ -19,9 +19,9 @@ package org.apache.spark.mllib.regression import scala.reflect.ClassTag -import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vector import org.apache.spark.streaming.api.java.{JavaDStream, JavaPairDStream} import org.apache.spark.streaming.dstream.DStream @@ -29,7 +29,7 @@ import org.apache.spark.streaming.dstream.DStream /** * :: DeveloperApi :: * StreamingLinearAlgorithm implements methods for continuously - * training a generalized linear model model on streaming data, + * training a generalized linear model on streaming data, * and using it for prediction on (possibly different) streaming data. * * This class takes as type parameters a GeneralizedLinearModel, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala index fe2a46b9eecc..84764963b5f3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala @@ -43,6 +43,7 @@ import org.apache.spark.mllib.linalg.Vector class StreamingLinearRegressionWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, + private var regParam: Double, private var miniBatchFraction: Double) extends StreamingLinearAlgorithm[LinearRegressionModel, LinearRegressionWithSGD] with Serializable { @@ -54,10 +55,10 @@ class StreamingLinearRegressionWithSGD private[mllib] ( * (see `StreamingLinearAlgorithm`) */ @Since("1.1.0") - def this() = this(0.1, 50, 1.0) + def this() = this(0.1, 50, 0.0, 1.0) @Since("1.1.0") - val algorithm = new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction) + val algorithm = new LinearRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction) protected var model: Option[LinearRegressionModel] = None @@ -70,6 +71,15 @@ class StreamingLinearRegressionWithSGD private[mllib] ( this } + /** + * Set the regularization parameter. Default: 0.0. + */ + @Since("2.0.0") + def setRegParam(regParam: Double): this.type = { + this.algorithm.optimizer.setRegParam(regParam) + this + } + /** * Set the number of iterations of gradient descent to run per update. Default: 50. */ 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 317d3a570263..cd90e97cc538 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 @@ -23,7 +23,7 @@ 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, SparkSession} /** * Helper methods for import/export of GLM regression models. @@ -47,8 +47,7 @@ private[regression] object GLMRegressionModel { modelClass: String, weights: Vector, intercept: Double): Unit = { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // Create JSON metadata. val metadata = compact(render( @@ -58,9 +57,7 @@ private[regression] object GLMRegressionModel { // Create Parquet data. val data = Data(weights, intercept) - val dataRDD: DataFrame = sc.parallelize(Seq(data), 1).toDF() - // TODO: repartition with 1 partition after SPARK-5532 gets fixed - dataRDD.write.parquet(Loader.dataPath(path)) + spark.createDataFrame(Seq(data)).repartition(1).write.parquet(Loader.dataPath(path)) } /** @@ -70,17 +67,17 @@ private[regression] object GLMRegressionModel { * The length of the weights vector should equal numFeatures. */ def loadData(sc: SparkContext, path: String, modelClass: String, numFeatures: Int): Data = { - val datapath = Loader.dataPath(path) - val sqlContext = new SQLContext(sc) - val dataRDD = sqlContext.read.parquet(datapath) + val dataPath = Loader.dataPath(path) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val dataRDD = spark.read.parquet(dataPath) val dataArray = dataRDD.select("weights", "intercept").take(1) - assert(dataArray.size == 1, s"Unable to load $modelClass data from: $datapath") + assert(dataArray.length == 1, s"Unable to load $modelClass data from: $dataPath") val data = dataArray(0) - assert(data.size == 2, s"Unable to load $modelClass data from: $datapath") + assert(data.size == 2, s"Unable to load $modelClass data from: $dataPath") data match { case Row(weights: Vector, intercept: Double) => assert(weights.size == numFeatures, s"Expected $numFeatures features, but" + - s" found ${weights.size} features when loading $modelClass weights from $datapath") + s" found ${weights.size} features when loading $modelClass weights from $dataPath") Data(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 201333c3690d..8121880cfb23 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -18,24 +18,27 @@ package org.apache.spark.mllib.stat import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * :: DeveloperApi :: * MultivariateOnlineSummarizer implements [[MultivariateStatisticalSummary]] to compute the mean, * variance, minimum, maximum, counts, and nonzero counts for instances in sparse or dense vector - * format in a online fashion. + * format in an online fashion. * * Two MultivariateOnlineSummarizer can be merged together to have a statistical summary of * the corresponding joint dataset. * * A numerically stable algorithm is implemented to compute the mean and variance of instances: - * Reference: [[http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance variance-wiki]] + * Reference: + * variance-wiki * Zero elements (including explicit zero values) are skipped when calling add(), * to have time complexity O(nnz) instead of O(n) for each column. * * For weighted instances, the unbiased estimation of variance is defined by the reliability - * weights: [[https://en.wikipedia.org/wiki/Weighted_arithmetic_mean#Reliability_weights]]. + * weights: + * see + * Reliability weights (Wikipedia). */ @Since("1.1.0") @DeveloperApi @@ -47,9 +50,10 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S private var currM2: Array[Double] = _ private var currL1: Array[Double] = _ private var totalCnt: Long = 0 - private var weightSum: Double = 0.0 + private var totalWeightSum: Double = 0.0 private var weightSquareSum: Double = 0.0 - private var nnz: Array[Double] = _ + private var weightSum: Array[Double] = _ + private var nnz: Array[Long] = _ private var currMax: Array[Double] = _ private var currMin: Array[Double] = _ @@ -74,7 +78,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S currM2n = Array.ofDim[Double](n) currM2 = Array.ofDim[Double](n) currL1 = Array.ofDim[Double](n) - nnz = Array.ofDim[Double](n) + weightSum = Array.ofDim[Double](n) + nnz = Array.ofDim[Long](n) currMax = Array.fill[Double](n)(Double.MinValue) currMin = Array.fill[Double](n)(Double.MaxValue) } @@ -86,7 +91,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S val localCurrM2n = currM2n val localCurrM2 = currM2 val localCurrL1 = currL1 - val localNnz = nnz + val localWeightSum = weightSum + val localNumNonzeros = nnz val localCurrMax = currMax val localCurrMin = currMin instance.foreachActive { (index, value) => @@ -100,16 +106,17 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S val prevMean = localCurrMean(index) val diff = value - prevMean - localCurrMean(index) = prevMean + weight * diff / (localNnz(index) + weight) + localCurrMean(index) = prevMean + weight * diff / (localWeightSum(index) + weight) localCurrM2n(index) += weight * (value - localCurrMean(index)) * diff localCurrM2(index) += weight * value * value localCurrL1(index) += weight * math.abs(value) - localNnz(index) += weight + localWeightSum(index) += weight + localNumNonzeros(index) += 1 } } - weightSum += weight + totalWeightSum += weight weightSquareSum += weight * weight totalCnt += 1 this @@ -124,17 +131,18 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S */ @Since("1.1.0") def merge(other: MultivariateOnlineSummarizer): this.type = { - if (this.weightSum != 0.0 && other.weightSum != 0.0) { + if (this.totalWeightSum != 0.0 && other.totalWeightSum != 0.0) { require(n == other.n, s"Dimensions mismatch when merging with another summarizer. " + s"Expecting $n but got ${other.n}.") totalCnt += other.totalCnt - weightSum += other.weightSum + totalWeightSum += other.totalWeightSum weightSquareSum += other.weightSquareSum var i = 0 while (i < n) { - val thisNnz = nnz(i) - val otherNnz = other.nnz(i) + val thisNnz = weightSum(i) + val otherNnz = other.weightSum(i) val totalNnz = thisNnz + otherNnz + val totalCnnz = nnz(i) + other.nnz(i) if (totalNnz != 0.0) { val deltaMean = other.currMean(i) - currMean(i) // merge mean together @@ -149,18 +157,20 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S currMax(i) = math.max(currMax(i), other.currMax(i)) currMin(i) = math.min(currMin(i), other.currMin(i)) } - nnz(i) = totalNnz + weightSum(i) = totalNnz + nnz(i) = totalCnnz i += 1 } - } else if (weightSum == 0.0 && other.weightSum != 0.0) { + } else if (totalWeightSum == 0.0 && other.totalWeightSum != 0.0) { this.n = other.n this.currMean = other.currMean.clone() this.currM2n = other.currM2n.clone() this.currM2 = other.currM2.clone() this.currL1 = other.currL1.clone() this.totalCnt = other.totalCnt - this.weightSum = other.weightSum + this.totalWeightSum = other.totalWeightSum this.weightSquareSum = other.weightSquareSum + this.weightSum = other.weightSum.clone() this.nnz = other.nnz.clone() this.currMax = other.currMax.clone() this.currMin = other.currMin.clone() @@ -174,12 +184,12 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S */ @Since("1.1.0") override def mean: Vector = { - require(weightSum > 0, s"Nothing has been added to this summarizer.") + require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") val realMean = Array.ofDim[Double](n) var i = 0 while (i < n) { - realMean(i) = currMean(i) * (nnz(i) / weightSum) + realMean(i) = currMean(i) * (weightSum(i) / totalWeightSum) i += 1 } Vectors.dense(realMean) @@ -191,11 +201,11 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S */ @Since("1.1.0") override def variance: Vector = { - require(weightSum > 0, s"Nothing has been added to this summarizer.") + require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") val realVariance = Array.ofDim[Double](n) - val denominator = weightSum - (weightSquareSum / weightSum) + val denominator = totalWeightSum - (weightSquareSum / totalWeightSum) // Sample variance is computed, if the denominator is less than 0, the variance is just 0. if (denominator > 0.0) { @@ -203,8 +213,9 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S var i = 0 val len = currM2n.length while (i < len) { - realVariance(i) = (currM2n(i) + deltaMean(i) * deltaMean(i) * nnz(i) * - (weightSum - nnz(i)) / weightSum) / denominator + // We prevent variance from negative value caused by numerical error. + realVariance(i) = math.max((currM2n(i) + deltaMean(i) * deltaMean(i) * weightSum(i) * + (totalWeightSum - weightSum(i)) / totalWeightSum) / denominator, 0.0) i += 1 } } @@ -224,9 +235,9 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S */ @Since("1.1.0") override def numNonzeros: Vector = { - require(weightSum > 0, s"Nothing has been added to this summarizer.") + require(totalCnt > 0, s"Nothing has been added to this summarizer.") - Vectors.dense(nnz) + Vectors.dense(nnz.map(_.toDouble)) } /** @@ -235,11 +246,11 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S */ @Since("1.1.0") override def max: Vector = { - require(weightSum > 0, s"Nothing has been added to this summarizer.") + require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") var i = 0 while (i < n) { - if ((nnz(i) < weightSum) && (currMax(i) < 0.0)) currMax(i) = 0.0 + if ((nnz(i) < totalCnt) && (currMax(i) < 0.0)) currMax(i) = 0.0 i += 1 } Vectors.dense(currMax) @@ -251,11 +262,11 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S */ @Since("1.1.0") override def min: Vector = { - require(weightSum > 0, s"Nothing has been added to this summarizer.") + require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") var i = 0 while (i < n) { - if ((nnz(i) < weightSum) && (currMin(i) > 0.0)) currMin(i) = 0.0 + if ((nnz(i) < totalCnt) && (currMin(i) > 0.0)) currMin(i) = 0.0 i += 1 } Vectors.dense(currMin) @@ -267,7 +278,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S */ @Since("1.2.0") override def normL2: Vector = { - require(weightSum > 0, s"Nothing has been added to this summarizer.") + require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") val realMagnitude = Array.ofDim[Double](n) @@ -286,7 +297,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S */ @Since("1.2.0") override def normL1: Vector = { - require(weightSum > 0, s"Nothing has been added to this summarizer.") + require(totalWeightSum > 0, s"Nothing has been added to this summarizer.") Vectors.dense(currL1) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala index bcb33a7a0467..5ebbfb2b6298 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -20,9 +20,9 @@ package org.apache.spark.mllib.stat import scala.annotation.varargs import org.apache.spark.annotation.Since -import org.apache.spark.api.java.{JavaRDD, JavaDoubleRDD} -import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.stat.correlation.Correlations import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest, @@ -60,15 +60,15 @@ object Statistics { * Compute the correlation matrix for the input RDD of Vectors using the specified method. * Methods currently supported: `pearson` (default), `spearman`. * - * Note that for Spearman, a rank correlation, we need to create an RDD[Double] for each column - * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], - * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to - * avoid recomputing the common lineage. - * * @param X an RDD[Vector] for which the correlation matrix is to be computed. * @param method String specifying the method to use for computing correlation. * Supported: `pearson` (default), `spearman` * @return Correlation matrix comparing columns in X. + * + * @note For Spearman, a rank correlation, we need to create an RDD[Double] for each column + * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], + * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to + * avoid recomputing the common lineage. */ @Since("1.1.0") def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) @@ -77,18 +77,18 @@ object Statistics { * Compute the Pearson correlation for the input RDDs. * Returns NaN if either vector has 0 variance. * - * Note: the two input RDDs need to have the same number of partitions and the same number of - * elements in each partition. - * * @param x RDD[Double] of the same cardinality as y. * @param y RDD[Double] of the same cardinality as x. * @return A Double containing the Pearson correlation between the two input RDD[Double]s + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. */ @Since("1.1.0") def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) /** - * Java-friendly version of [[corr()]] + * Java-friendly version of `corr()` */ @Since("1.4.1") def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = @@ -98,21 +98,21 @@ object Statistics { * Compute the correlation for the input RDDs using the specified method. * Methods currently supported: `pearson` (default), `spearman`. * - * Note: the two input RDDs need to have the same number of partitions and the same number of - * elements in each partition. - * * @param x RDD[Double] of the same cardinality as y. * @param y RDD[Double] of the same cardinality as x. * @param method String specifying the method to use for computing correlation. * Supported: `pearson` (default), `spearman` * @return A Double containing the correlation between the two input RDD[Double]s using the * specified method. + * + * @note The two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. */ @Since("1.1.0") def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) /** - * Java-friendly version of [[corr()]] + * Java-friendly version of `corr()` */ @Since("1.4.1") def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = @@ -122,15 +122,15 @@ object Statistics { * Conduct Pearson's chi-squared goodness of fit test of the observed data against the * expected distribution. * - * Note: the two input Vectors need to have the same size. - * `observed` cannot contain negative values. - * `expected` cannot contain nonpositive values. - * * @param observed Vector containing the observed categorical counts/relative frequencies. * @param expected Vector containing the expected categorical counts/relative frequencies. * `expected` is rescaled if the `expected` sum differs from the `observed` sum. * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, * the method used, and the null hypothesis. + * + * @note The two input Vectors need to have the same size. + * `observed` cannot contain negative values. + * `expected` cannot contain nonpositive values. */ @Since("1.1.0") def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { @@ -141,11 +141,11 @@ object Statistics { * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform * distribution, with each category having an expected frequency of `1 / observed.size`. * - * Note: `observed` cannot contain negative values. - * * @param observed Vector containing the observed categorical counts/relative frequencies. * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, * the method used, and the null hypothesis. + * + * @note `observed` cannot contain negative values. */ @Since("1.1.0") def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) @@ -176,7 +176,9 @@ object Statistics { ChiSqTest.chiSquaredFeatures(data) } - /** Java-friendly version of [[chiSqTest()]] */ + /** + * Java-friendly version of `chiSqTest()` + */ @Since("1.5.0") def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) @@ -186,7 +188,8 @@ object Statistics { * distribution of the sample data and the theoretical distribution we can provide a test for the * the null hypothesis that the sample data comes from that theoretical distribution. * For more information on KS Test: - * @see [[https://en.wikipedia.org/wiki/Kolmogorov%E2%80%93Smirnov_test]] + * @see + * Kolmogorov-Smirnov test (Wikipedia) * * @param data an `RDD[Double]` containing the sample of data to test * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value @@ -217,7 +220,9 @@ object Statistics { KolmogorovSmirnovTest.testOneSample(data, distName, params: _*) } - /** Java-friendly version of [[kolmogorovSmirnovTest()]] */ + /** + * Java-friendly version of `kolmogorovSmirnovTest()` + */ @Since("1.5.0") @varargs def kolmogorovSmirnovTest( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala index 8a821d1b23ba..e478c31bc9a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.stat.correlation import breeze.linalg.{DenseMatrix => BDM} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector} import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.rdd.RDD @@ -52,10 +52,10 @@ private[stat] object PearsonCorrelation extends Correlation with Logging { /** * Compute the Pearson correlation matrix from the covariance matrix. - * 0 covariance results in a correlation value of Double.NaN. + * 0 variance results in a correlation value of Double.NaN. */ def computeCorrelationMatrixFromCovariance(covarianceMatrix: Matrix): Matrix = { - val cov = covarianceMatrix.toBreeze.asInstanceOf[BDM[Double]] + val cov = covarianceMatrix.asBreeze.asInstanceOf[BDM[Double]] val n = cov.cols // Compute the standard deviation on the diagonals first diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala index 4a6c677f06d2..ee51d332399e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala @@ -19,8 +19,7 @@ package org.apache.spark.mllib.stat.correlation import scala.collection.mutable.ArrayBuffer -import org.apache.spark.Logging -import org.apache.spark.SparkContext._ +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Matrix, Vector, Vectors} import org.apache.spark.rdd.RDD @@ -58,7 +57,7 @@ private[stat] object SpearmanCorrelation extends Correlation with Logging { var preCol = -1 var preVal = Double.NaN var startRank = -1.0 - var cachedUids = ArrayBuffer.empty[Long] + val cachedUids = ArrayBuffer.empty[Long] val flush: () => Iterable[(Long, (Int, Double))] = () => { val averageRank = startRank + (cachedUids.size - 1) / 2.0 val output = cachedUids.map { uid => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala index 0724af93088c..4cf662e03634 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala @@ -17,10 +17,10 @@ package org.apache.spark.mllib.stat.distribution -import breeze.linalg.{DenseVector => DBV, DenseMatrix => DBM, diag, max, eigSym, Vector => BV} +import breeze.linalg.{diag, eigSym, max, DenseMatrix => DBM, DenseVector => DBV, Vector => BV} import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.mllib.linalg.{Vectors, Vector, Matrices, Matrix} +import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector, Vectors} import org.apache.spark.mllib.util.MLUtils /** @@ -28,7 +28,8 @@ import org.apache.spark.mllib.util.MLUtils * This class provides basic functionality for a Multivariate Gaussian (Normal) Distribution. In * the event that the covariance matrix is singular, the density will be computed in a * reduced dimensional subspace under which the distribution is supported. - * (see [[http://en.wikipedia.org/wiki/Multivariate_normal_distribution#Degenerate_case]]) + * (see + * Degenerate case in Multivariate normal distribution (Wikipedia)) * * @param mu The mean vector of the distribution * @param sigma The covariance matrix of the distribution @@ -42,7 +43,7 @@ class MultivariateGaussian @Since("1.3.0") ( require(sigma.numCols == sigma.numRows, "Covariance matrix must be square") require(mu.size == sigma.numCols, "Mean vector length must match covariance matrix size") - private val breezeMu = mu.toBreeze.toDenseVector + private val breezeMu = mu.asBreeze.toDenseVector /** * private[mllib] constructor @@ -61,18 +62,20 @@ class MultivariateGaussian @Since("1.3.0") ( */ private val (rootSigmaInv: DBM[Double], u: Double) = calculateCovarianceConstants - /** Returns density of this multivariate Gaussian at given point, x - */ - @Since("1.3.0") + /** + * Returns density of this multivariate Gaussian at given point, x + */ + @Since("1.3.0") def pdf(x: Vector): Double = { - pdf(x.toBreeze) + pdf(x.asBreeze) } - /** Returns the log-density of this multivariate Gaussian at given point, x - */ - @Since("1.3.0") + /** + * Returns the log-density of this multivariate Gaussian at given point, x + */ + @Since("1.3.0") def logpdf(x: Vector): Double = { - logpdf(x.toBreeze) + logpdf(x.asBreeze) } /** Returns density of this multivariate Gaussian at given point, x */ @@ -116,7 +119,7 @@ class MultivariateGaussian @Since("1.3.0") ( * relation to the maximum singular value (same tolerance used by, e.g., Octave). */ private def calculateCovarianceConstants: (DBM[Double], Double) = { - val eigSym.EigSym(d, u) = eigSym(sigma.toBreeze.toDenseMatrix) // sigma = u * diag(d) * u.t + val eigSym.EigSym(d, u) = eigSym(sigma.asBreeze.toDenseMatrix) // sigma = u * diag(d) * u.t // For numerical stability, values are considered to be non-zero only if they exceed tol. // This prevents any inverted value from exceeding (eps * n * max(d))^-1 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala index 23c8d7c7c807..ee51248e5355 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala @@ -17,16 +17,17 @@ package org.apache.spark.mllib.stat.test +import scala.collection.mutable + import breeze.linalg.{DenseMatrix => BDM} import org.apache.commons.math3.distribution.ChiSquaredDistribution -import org.apache.spark.{SparkException, Logging} +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD -import scala.collection.mutable - /** * Conduct the chi-squared test for the input RDDs using the specified method. * Goodness-of-fit test is conducted on two `Vectors`, whereas test of independence is conducted @@ -40,7 +41,7 @@ import scala.collection.mutable * * More information on Chi-squared test: http://en.wikipedia.org/wiki/Chi-squared_test */ -private[stat] object ChiSqTest extends Logging { +private[spark] object ChiSqTest extends Logging { /** * @param name String name for the method. @@ -69,6 +70,11 @@ private[stat] object ChiSqTest extends Logging { } } + /** + * Max number of categories when indexing labels and features + */ + private[spark] val maxCategories: Int = 10000 + /** * Conduct Pearson's independence test for each feature against the label across the input RDD. * The contingency table is constructed from the raw (feature, label) pairs and used to conduct @@ -77,7 +83,6 @@ private[stat] object ChiSqTest extends Logging { */ def chiSquaredFeatures(data: RDD[LabeledPoint], methodName: String = PEARSON.name): Array[ChiSqTestResult] = { - val maxCategories = 10000 val numCols = data.first().features.size val results = new Array[ChiSqTestResult](numCols) var labels: Map[Double, Int] = null @@ -109,7 +114,9 @@ private[stat] object ChiSqTest extends Logging { } i += 1 distinctLabels += label - features.toArray.view.zipWithIndex.slice(startCol, endCol).map { case (feature, col) => + val brzFeatures = features.asBreeze + (startCol until endCol).map { col => + val feature = brzFeatures(col) allDistinctFeatures(col) += feature (col, feature, label) } @@ -122,7 +129,7 @@ private[stat] object ChiSqTest extends Logging { pairCounts.keys.filter(_._1 == startCol).map(_._3).toArray.distinct.zipWithIndex.toMap } val numLabels = labels.size - pairCounts.keys.groupBy(_._1).map { case (col, keys) => + pairCounts.keys.groupBy(_._1).foreach { case (col, keys) => val features = keys.map(_._2).toArray.distinct.zipWithIndex.toMap val numRows = features.size val contingency = new BDM(numRows, numLabels, new Array[Double](numRows * numLabels)) @@ -143,7 +150,7 @@ private[stat] object ChiSqTest extends Logging { * Uniform distribution is assumed when `expected` is not passed in. */ def chiSquared(observed: Vector, - expected: Vector = Vectors.dense(Array[Double]()), + expected: Vector = Vectors.dense(Array.empty[Double]), methodName: String = PEARSON.name): ChiSqTestResult = { // Validate input arguments diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala index 2b3ed6df486c..d17f7047c5b2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala @@ -22,7 +22,7 @@ import scala.annotation.varargs import org.apache.commons.math3.distribution.{NormalDistribution, RealDistribution} import org.apache.commons.math3.stat.inference.{KolmogorovSmirnovTest => CommonMathKolmogorovSmirnovTest} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD /** @@ -31,7 +31,8 @@ import org.apache.spark.rdd.RDD * distribution of the sample data and the theoretical distribution we can provide a test for the * the null hypothesis that the sample data comes from that theoretical distribution. * For more information on KS Test: - * @see [[https://en.wikipedia.org/wiki/Kolmogorov%E2%80%93Smirnov_test]] + * @see + * Kolmogorov-Smirnov test (Wikipedia) * * Implementation note: We seek to implement the KS test with a minimal number of distributed * passes. We sort the RDD, and then perform the following operations on a per-partition basis: @@ -45,7 +46,7 @@ import org.apache.spark.rdd.RDD * many elements are in each partition. Once these three values have been returned for every * partition, we can collect and operate locally. Locally, we can now adjust each distance by the * appropriate constant (the cumulative sum of number of elements in the prior partitions divided by - * thedata set size). Finally, we take the maximum absolute value, and this is the statistic. + * the data set size). Finally, we take the maximum absolute value, and this is the statistic. */ private[stat] object KolmogorovSmirnovTest extends Logging { @@ -124,7 +125,8 @@ private[stat] object KolmogorovSmirnovTest extends Logging { val pResults = partDiffs.foldLeft(initAcc) { case ((pMin, pMax, pCt), (dl, dp)) => (math.min(pMin, dl), math.max(pMax, dp), pCt + 1) } - val results = if (pResults == initAcc) Array[(Double, Double, Double)]() else Array(pResults) + val results = + if (pResults == initAcc) Array.empty[(Double, Double, Double)] else Array(pResults) results.iterator } @@ -166,7 +168,7 @@ private[stat] object KolmogorovSmirnovTest extends Logging { : KolmogorovSmirnovTestResult = { val distObj = distName match { - case "norm" => { + case "norm" => if (params.nonEmpty) { // parameters are passed, then can only be 2 require(params.length == 2, "Normal distribution requires mean and standard " + @@ -178,7 +180,6 @@ private[stat] object KolmogorovSmirnovTest extends Logging { "initialized to standard normal (i.e. N(0, 1))") new NormalDistribution(0, 1) } - } case _ => throw new UnsupportedOperationException(s"$distName not yet supported through" + s" convenience method. Current options are:['norm'].") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala index 75c6a51d0957..80c6ef0ea1aa 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala @@ -17,27 +17,44 @@ package org.apache.spark.mllib.stat.test -import org.apache.spark.Logging -import org.apache.spark.annotation.{Experimental, Since} -import org.apache.spark.rdd.RDD +import scala.beans.BeanInfo + +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging +import org.apache.spark.streaming.api.java.JavaDStream import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.StatCounter /** - * :: Experimental :: + * Class that represents the group and value of a sample. + * + * @param isExperiment if the sample is of the experiment group. + * @param value numeric value of the observation. + */ +@Since("1.6.0") +@BeanInfo +case class BinarySample @Since("1.6.0") ( + @Since("1.6.0") isExperiment: Boolean, + @Since("1.6.0") value: Double) { + override def toString: String = { + s"($isExperiment, $value)" + } +} + +/** * Performs online 2-sample significance testing for a stream of (Boolean, Double) pairs. The * Boolean identifies which sample each observation comes from, and the Double is the numeric value * of the observation. * * To address novelty affects, the `peacePeriod` specifies a set number of initial - * [[org.apache.spark.rdd.RDD]] batches of the [[DStream]] to be dropped from significance testing. + * [[org.apache.spark.rdd.RDD]] batches of the `DStream` to be dropped from significance testing. * * The `windowSize` sets the number of batches each significance test is to be performed over. The * window is sliding with a stride length of 1 batch. Setting windowSize to 0 will perform * cumulative processing, using all batches seen so far. * * Different tests may be used for assessing statistical significance depending on assumptions - * satisfied by data. For more details, see [[StreamingTestMethod]]. The `testMethod` specifies + * satisfied by data. For more details, see `StreamingTestMethod`. The `testMethod` specifies * which test will be used. * * Use a builder pattern to construct a streaming test in an application, for example: @@ -49,7 +66,6 @@ import org.apache.spark.util.StatCounter * .registerStream(DStream) * }}} */ -@Experimental @Since("1.6.0") class StreamingTest @Since("1.6.0") () extends Logging with Serializable { private var peacePeriod: Int = 0 @@ -81,15 +97,15 @@ class StreamingTest @Since("1.6.0") () extends Logging with Serializable { } /** - * Register a [[DStream]] of values for significance testing. + * Register a `DStream` of values for significance testing. * - * @param data stream of (key,value) pairs where the key denotes group membership (true = - * experiment, false = control) and the value is the numerical metric to test for - * significance + * @param data stream of BinarySample(key,value) pairs where the key denotes group membership + * (true = experiment, false = control) and the value is the numerical metric to + * test for significance * @return stream of significance testing results */ @Since("1.6.0") - def registerStream(data: DStream[(Boolean, Double)]): DStream[StreamingTestResult] = { + def registerStream(data: DStream[BinarySample]): DStream[StreamingTestResult] = { val dataAfterPeacePeriod = dropPeacePeriod(data) val summarizedData = summarizeByKeyAndWindow(dataAfterPeacePeriod) val pairedSummaries = pairSummaries(summarizedData) @@ -97,23 +113,37 @@ class StreamingTest @Since("1.6.0") () extends Logging with Serializable { testMethod.doTest(pairedSummaries) } + /** + * Register a `JavaDStream` of values for significance testing. + * + * @param data stream of BinarySample(isExperiment,value) pairs where the isExperiment denotes + * group (true = experiment, false = control) and the value is the numerical metric + * to test for significance + * @return stream of significance testing results + */ + @Since("1.6.0") + def registerStream(data: JavaDStream[BinarySample]): JavaDStream[StreamingTestResult] = { + JavaDStream.fromDStream(registerStream(data.dstream)) + } + /** Drop all batches inside the peace period. */ private[stat] def dropPeacePeriod( - data: DStream[(Boolean, Double)]): DStream[(Boolean, Double)] = { + data: DStream[BinarySample]): DStream[BinarySample] = { data.transform { (rdd, time) => if (time.milliseconds > data.slideDuration.milliseconds * peacePeriod) { rdd } else { - data.context.sparkContext.parallelize(Seq()) + data.context.sparkContext.parallelize(Seq.empty) } } } /** Compute summary statistics over each key and the specified test window size. */ private[stat] def summarizeByKeyAndWindow( - data: DStream[(Boolean, Double)]): DStream[(Boolean, StatCounter)] = { + data: DStream[BinarySample]): DStream[(Boolean, StatCounter)] = { + val categoryValuePair = data.map(sample => (sample.isExperiment, sample.value)) if (this.windowSize == 0) { - data.updateStateByKey[StatCounter]( + categoryValuePair.updateStateByKey[StatCounter]( (newValues: Seq[Double], oldSummary: Option[StatCounter]) => { val newSummary = oldSummary.getOrElse(new StatCounter()) newSummary.merge(newValues) @@ -121,7 +151,7 @@ class StreamingTest @Since("1.6.0") () extends Logging with Serializable { }) } else { val windowDuration = data.slideDuration * this.windowSize - data + categoryValuePair .groupByKeyAndWindow(windowDuration) .mapValues { values => val summary = new StatCounter() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTestMethod.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTestMethod.scala index a7eaed51b4d5..14ac14d6d61f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTestMethod.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTestMethod.scala @@ -26,7 +26,7 @@ import com.twitter.chill.MeatLocker import org.apache.commons.math3.stat.descriptive.StatisticalSummaryValues import org.apache.commons.math3.stat.inference.TTest -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.StatCounter @@ -73,7 +73,7 @@ private[stat] sealed trait StreamingTestMethod extends Serializable { * This test does not assume equal variance between the two samples and does not assume equal * sample size. * - * @see http://en.wikipedia.org/wiki/Welch%27s_t_test + * @see Welch's t-test (Wikipedia) */ private[stat] object WelchTTest extends StreamingTestMethod with Logging { @@ -115,7 +115,7 @@ private[stat] object WelchTTest extends StreamingTestMethod with Logging { * mean. This test assumes equal variance between the two samples and does not assume equal sample * size. For unequal variances, Welch's t-test should be used instead. * - * @see http://en.wikipedia.org/wiki/Student%27s_t-test + * @see Student's t-test (Wikipedia) */ private[stat] object StudentTTest extends StreamingTestMethod with Logging { @@ -152,8 +152,8 @@ private[stat] object StudentTTest extends StreamingTestMethod with Logging { private[stat] object StreamingTestMethod { // Note: after new `StreamingTestMethod`s are implemented, please update this map. private final val TEST_NAME_TO_OBJECT: Map[String, StreamingTestMethod] = Map( - "welch"->WelchTTest, - "student"->StudentTTest) + "welch" -> WelchTTest, + "student" -> StudentTTest) def getTestMethodFromName(method: String): StreamingTestMethod = TEST_NAME_TO_OBJECT.get(method) match { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala index 8a29fd39a910..5cfc05a3dd2d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.stat.test -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since /** * Trait for hypothesis test results. @@ -94,10 +94,8 @@ class ChiSqTestResult private[stat] (override val pValue: Double, } /** - * :: Experimental :: * Object containing the test results for the Kolmogorov-Smirnov test. */ -@Experimental @Since("1.5.0") class KolmogorovSmirnovTestResult private[stat] ( @Since("1.5.0") override val pValue: Double, @@ -113,10 +111,8 @@ class KolmogorovSmirnovTestResult private[stat] ( } /** - * :: Experimental :: * Object containing the test results for streaming testing. */ -@Experimental @Since("1.6.0") private[stat] class StreamingTestResult @Since("1.6.0") ( @Since("1.6.0") override val pValue: Double, 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 af1f7e74c004..e5aece779826 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 @@ -18,45 +18,51 @@ package org.apache.spark.mllib.tree import scala.collection.JavaConverters._ -import scala.collection.mutable -import org.apache.spark.Logging import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.RandomForest.NodeIndexInfo -import org.apache.spark.mllib.tree.configuration.Strategy 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.configuration.Strategy 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 + /** * A class which implements a decision tree learning algorithm for classification and regression. * It supports both continuous and categorical features. + * * @param strategy The configuration parameters for the tree algorithm which specify the type - * of algorithm (classification, regression, etc.), feature type (continuous, + * of decision tree (classification or regression), feature type (continuous, * categorical), depth of the tree, quantile calculation strategy, etc. + * @param seed Random seed. */ @Since("1.0.0") -class DecisionTree @Since("1.0.0") (private val strategy: Strategy) +class DecisionTree private[spark] (private val strategy: Strategy, private val seed: Int) extends Serializable with Logging { + /** + * @param strategy The configuration parameters for the tree algorithm which specify the type + * of decision tree (classification or regression), feature type (continuous, + * categorical), depth of the tree, quantile calculation strategy, etc. + */ + @Since("1.0.0") + def this(strategy: Strategy) = this(strategy, seed = 0) + strategy.assertValid() /** * Method to train a decision tree model over an RDD - * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] - * @return DecisionTreeModel that can be used for prediction + * + * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * @return DecisionTreeModel that can be used for prediction. */ @Since("1.2.0") def run(input: RDD[LabeledPoint]): DecisionTreeModel = { - // Note: random seed will not be used since numTrees = 1. - val rf = new RandomForest(strategy, numTrees = 1, featureSubsetStrategy = "all", seed = 0) + val rf = new RandomForest(strategy, numTrees = 1, featureSubsetStrategy = "all", seed = seed) val rfModel = rf.run(input) rfModel.trees(0) } @@ -69,19 +75,19 @@ object DecisionTree extends Serializable with Logging { * Method to train a decision tree model. * The method supports binary and multiclass classification and regression. * - * Note: Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] - * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] - * is recommended to clearly separate classification and regression. - * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * For classification, labels should take values {0, 1, ..., numClasses-1}. * For regression, labels are real numbers. * @param strategy The configuration parameters for the tree algorithm which specify the type - * of algorithm (classification, regression, etc.), feature type (continuous, + * of decision tree (classification or regression), feature type (continuous, * categorical), depth of the tree, quantile calculation strategy, etc. - * @return DecisionTreeModel that can be used for prediction + * @return DecisionTreeModel that can be used for prediction. + * + * @note Using `org.apache.spark.mllib.tree.DecisionTree.trainClassifier` + * and `org.apache.spark.mllib.tree.DecisionTree.trainRegressor` + * is recommended to clearly separate classification and regression. */ - @Since("1.0.0") + @Since("1.0.0") def train(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeModel = { new DecisionTree(strategy).run(input) } @@ -90,18 +96,18 @@ object DecisionTree extends Serializable with Logging { * Method to train a decision tree model. * The method supports binary and multiclass classification and regression. * - * Note: Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] - * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] - * is recommended to clearly separate classification and regression. - * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * For classification, labels should take values {0, 1, ..., numClasses-1}. * For regression, labels are real numbers. - * @param algo algorithm, classification or regression - * @param impurity impurity criterion used for information gain calculation - * @param maxDepth Maximum depth of the tree. - * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * @return DecisionTreeModel that can be used for prediction + * @param algo Type of decision tree, either classification or regression. + * @param impurity Criterion used for information gain calculation. + * @param maxDepth Maximum depth of the tree (e.g. depth 0 means 1 leaf node, depth 1 means + * 1 internal node + 2 leaf nodes). + * @return DecisionTreeModel that can be used for prediction. + * + * @note Using `org.apache.spark.mllib.tree.DecisionTree.trainClassifier` + * and `org.apache.spark.mllib.tree.DecisionTree.trainRegressor` + * is recommended to clearly separate classification and regression. */ @Since("1.0.0") def train( @@ -117,19 +123,19 @@ object DecisionTree extends Serializable with Logging { * Method to train a decision tree model. * The method supports binary and multiclass classification and regression. * - * Note: Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] - * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] - * is recommended to clearly separate classification and regression. - * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * For classification, labels should take values {0, 1, ..., numClasses-1}. * For regression, labels are real numbers. - * @param algo algorithm, classification or regression - * @param impurity impurity criterion used for information gain calculation - * @param maxDepth Maximum depth of the tree. - * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * @param numClasses number of classes for classification. Default value of 2. - * @return DecisionTreeModel that can be used for prediction + * @param algo Type of decision tree, either classification or regression. + * @param impurity Criterion used for information gain calculation. + * @param maxDepth Maximum depth of the tree (e.g. depth 0 means 1 leaf node, depth 1 means + * 1 internal node + 2 leaf nodes). + * @param numClasses Number of classes for classification. Default value of 2. + * @return DecisionTreeModel that can be used for prediction. + * + * @note Using `org.apache.spark.mllib.tree.DecisionTree.trainClassifier` + * and `org.apache.spark.mllib.tree.DecisionTree.trainRegressor` + * is recommended to clearly separate classification and regression. */ @Since("1.2.0") def train( @@ -146,24 +152,24 @@ object DecisionTree extends Serializable with Logging { * Method to train a decision tree model. * The method supports binary and multiclass classification and regression. * - * Note: Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] - * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] - * is recommended to clearly separate classification and regression. - * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * For classification, labels should take values {0, 1, ..., numClasses-1}. * For regression, labels are real numbers. - * @param algo classification or regression - * @param impurity criterion used for information gain calculation - * @param maxDepth Maximum depth of the tree. - * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * @param numClasses number of classes for classification. Default value of 2. - * @param maxBins maximum number of bins used for splitting features - * @param quantileCalculationStrategy algorithm for calculating quantiles - * @param categoricalFeaturesInfo Map storing arity of categorical features. - * E.g., an entry (n -> k) indicates that feature n is categorical - * with k categories indexed from 0: {0, 1, ..., k-1}. - * @return DecisionTreeModel that can be used for prediction + * @param algo Type of decision tree, either classification or regression. + * @param impurity Criterion used for information gain calculation. + * @param maxDepth Maximum depth of the tree (e.g. depth 0 means 1 leaf node, depth 1 means + * 1 internal node + 2 leaf nodes). + * @param numClasses Number of classes for classification. Default value of 2. + * @param maxBins Maximum number of bins used for splitting features. + * @param quantileCalculationStrategy Algorithm for calculating quantiles. + * @param categoricalFeaturesInfo Map storing arity of categorical features. An entry (n to k) + * indicates that feature n is categorical with k categories + * indexed from 0: {0, 1, ..., k-1}. + * @return DecisionTreeModel that can be used for prediction. + * + * @note Using `org.apache.spark.mllib.tree.DecisionTree.trainClassifier` + * and `org.apache.spark.mllib.tree.DecisionTree.trainRegressor` + * is recommended to clearly separate classification and regression. */ @Since("1.0.0") def train( @@ -185,18 +191,18 @@ object DecisionTree extends Serializable with Logging { * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * Labels should take values {0, 1, ..., numClasses-1}. - * @param numClasses number of classes for classification. - * @param categoricalFeaturesInfo Map storing arity of categorical features. - * E.g., an entry (n -> k) indicates that feature n is categorical - * with k categories indexed from 0: {0, 1, ..., k-1}. + * @param numClasses Number of classes for classification. + * @param categoricalFeaturesInfo Map storing arity of categorical features. An entry (n to k) + * indicates that feature n is categorical with k categories + * indexed from 0: {0, 1, ..., k-1}. * @param impurity Criterion used for information gain calculation. * Supported values: "gini" (recommended) or "entropy". - * @param maxDepth Maximum depth of the tree. - * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * (suggested value: 5) - * @param maxBins maximum number of bins used for splitting features - * (suggested value: 32) - * @return DecisionTreeModel that can be used for prediction + * @param maxDepth Maximum depth of the tree (e.g. depth 0 means 1 leaf node, depth 1 means + * 1 internal node + 2 leaf nodes). + * (suggested value: 5) + * @param maxBins Maximum number of bins used for splitting features. + * (suggested value: 32) + * @return DecisionTreeModel that can be used for prediction. */ @Since("1.1.0") def trainClassifier( @@ -212,7 +218,7 @@ object DecisionTree extends Serializable with Logging { } /** - * Java-friendly API for [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] + * Java-friendly API for `org.apache.spark.mllib.tree.DecisionTree.trainClassifier` */ @Since("1.1.0") def trainClassifier( @@ -232,17 +238,17 @@ object DecisionTree extends Serializable with Logging { * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * Labels are real numbers. - * @param categoricalFeaturesInfo Map storing arity of categorical features. - * E.g., an entry (n -> k) indicates that feature n is categorical - * with k categories indexed from 0: {0, 1, ..., k-1}. + * @param categoricalFeaturesInfo Map storing arity of categorical features. An entry (n to k) + * indicates that feature n is categorical with k categories + * indexed from 0: {0, 1, ..., k-1}. * @param impurity Criterion used for information gain calculation. - * Supported values: "variance". - * @param maxDepth Maximum depth of the tree. - * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * (suggested value: 5) - * @param maxBins maximum number of bins used for splitting features - * (suggested value: 32) - * @return DecisionTreeModel that can be used for prediction + * The only supported value for regression is "variance". + * @param maxDepth Maximum depth of the tree (e.g. depth 0 means 1 leaf node, depth 1 means + * 1 internal node + 2 leaf nodes). + * (suggested value: 5) + * @param maxBins Maximum number of bins used for splitting features. + * (suggested value: 32) + * @return DecisionTreeModel that can be used for prediction. */ @Since("1.1.0") def trainRegressor( @@ -256,7 +262,7 @@ object DecisionTree extends Serializable with Logging { } /** - * Java-friendly API for [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + * Java-friendly API for `org.apache.spark.mllib.tree.DecisionTree.trainRegressor` */ @Since("1.1.0") def trainRegressor( @@ -269,909 +275,4 @@ object DecisionTree extends Serializable with Logging { categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, impurity, maxDepth, maxBins) } - - /** - * Get the node index corresponding to this data point. - * This function mimics prediction, passing an example from the root node down to a leaf - * or unsplit node; that node's index is returned. - * - * @param node Node in tree from which to classify the given data point. - * @param binnedFeatures Binned feature vector for data point. - * @param bins possible bins for all features, indexed (numFeatures)(numBins) - * @param unorderedFeatures Set of indices of unordered features. - * @return Leaf index if the data point reaches a leaf. - * Otherwise, last node reachable in tree matching this example. - * Note: This is the global node index, i.e., the index used in the tree. - * This index is different from the index used during training a particular - * group of nodes on one call to [[findBestSplits()]]. - */ - private def predictNodeIndex( - node: Node, - binnedFeatures: Array[Int], - bins: Array[Array[Bin]], - unorderedFeatures: Set[Int]): Int = { - if (node.isLeaf || node.split.isEmpty) { - // Node is either leaf, or has not yet been split. - node.id - } else { - val featureIndex = node.split.get.feature - val splitLeft = node.split.get.featureType match { - case Continuous => { - val binIndex = binnedFeatures(featureIndex) - val featureValueUpperBound = bins(featureIndex)(binIndex).highSplit.threshold - // bin binIndex has range (bin.lowSplit.threshold, bin.highSplit.threshold] - // We do not need to check lowSplit since bins are separated by splits. - featureValueUpperBound <= node.split.get.threshold - } - case Categorical => { - val featureValue = binnedFeatures(featureIndex) - node.split.get.categories.contains(featureValue) - } - case _ => throw new RuntimeException(s"predictNodeIndex failed for unknown reason.") - } - if (node.leftNode.isEmpty || node.rightNode.isEmpty) { - // Return index from next layer of nodes to train - if (splitLeft) { - Node.leftChildIndex(node.id) - } else { - Node.rightChildIndex(node.id) - } - } else { - if (splitLeft) { - predictNodeIndex(node.leftNode.get, binnedFeatures, bins, unorderedFeatures) - } else { - predictNodeIndex(node.rightNode.get, binnedFeatures, bins, unorderedFeatures) - } - } - } - } - - /** - * Helper for binSeqOp, for data which can contain a mix of ordered and unordered features. - * - * For ordered features, a single bin is updated. - * For unordered features, bins correspond to subsets of categories; either the left or right bin - * for each subset is updated. - * - * @param agg Array storing aggregate calculation, with a set of sufficient statistics for - * each (feature, bin). - * @param treePoint Data point being aggregated. - * @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, - splits: Array[Array[Split]], - unorderedFeatures: Set[Int], - instanceWeight: Double, - featuresForNode: Option[Array[Int]]): Unit = { - val numFeaturesPerNode = if (featuresForNode.nonEmpty) { - // Use subsampled features - featuresForNode.get.size - } else { - // Use all features - agg.metadata.numFeatures - } - // Iterate over features. - var featureIndexIdx = 0 - while (featureIndexIdx < numFeaturesPerNode) { - val featureIndex = if (featuresForNode.nonEmpty) { - featuresForNode.get.apply(featureIndexIdx) - } else { - featureIndexIdx - } - if (unorderedFeatures.contains(featureIndex)) { - // Unordered feature - val featureValue = treePoint.binnedFeatures(featureIndex) - val (leftNodeFeatureOffset, rightNodeFeatureOffset) = - agg.getLeftRightFeatureOffsets(featureIndexIdx) - // Update the left or right bin for each split. - val numSplits = agg.metadata.numSplits(featureIndex) - var splitIndex = 0 - while (splitIndex < numSplits) { - if (splits(featureIndex)(splitIndex).categories.contains(featureValue)) { - agg.featureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label, - instanceWeight) - } else { - agg.featureUpdate(rightNodeFeatureOffset, splitIndex, treePoint.label, - instanceWeight) - } - splitIndex += 1 - } - } else { - // Ordered feature - val binIndex = treePoint.binnedFeatures(featureIndex) - agg.update(featureIndexIdx, binIndex, treePoint.label, instanceWeight) - } - featureIndexIdx += 1 - } - } - - /** - * Helper for binSeqOp, for regression and for classification with only ordered features. - * - * For each feature, the sufficient statistics of one bin are updated. - * - * @param agg Array storing aggregate calculation, with a set of sufficient statistics for - * each (feature, bin). - * @param treePoint Data point being aggregated. - * @param instanceWeight Weight (importance) of instance in dataset. - */ - private def orderedBinSeqOp( - agg: DTStatsAggregator, - treePoint: TreePoint, - instanceWeight: Double, - featuresForNode: Option[Array[Int]]): Unit = { - val label = treePoint.label - - // Iterate over features. - if (featuresForNode.nonEmpty) { - // Use subsampled features - var featureIndexIdx = 0 - while (featureIndexIdx < featuresForNode.get.size) { - val binIndex = treePoint.binnedFeatures(featuresForNode.get.apply(featureIndexIdx)) - agg.update(featureIndexIdx, binIndex, label, instanceWeight) - featureIndexIdx += 1 - } - } else { - // Use all features - val numFeatures = agg.metadata.numFeatures - var featureIndex = 0 - while (featureIndex < numFeatures) { - val binIndex = treePoint.binnedFeatures(featureIndex) - agg.update(featureIndex, binIndex, label, instanceWeight) - featureIndex += 1 - } - } - } - - /** - * Given a group of nodes, this finds the best split for each node. - * - * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] - * @param metadata Learning and dataset metadata - * @param topNodes Root node for each tree. Used for matching instances with nodes. - * @param nodesForGroup Mapping: treeIndex --> nodes to be split in tree - * @param treeToNodeToIndexInfo Mapping: treeIndex --> nodeIndex --> nodeIndexInfo, - * where nodeIndexInfo stores the index in the group and the - * feature subsets (if using feature subsets). - * @param splits possible splits for all features, indexed (numFeatures)(numSplits) - * @param bins possible bins for all features, indexed (numFeatures)(numBins) - * @param nodeQueue Queue of nodes to split, with values (treeIndex, node). - * Updated with new non-leaf nodes which are created. - * @param nodeIdCache Node Id cache containing an RDD of Array[Int] where - * each value in the array is the data point's node Id - * for a corresponding tree. This is used to prevent the need - * to pass the entire tree to the executors during - * the node stat aggregation phase. - */ - private[tree] def findBestSplits( - input: RDD[BaggedPoint[TreePoint]], - metadata: DecisionTreeMetadata, - topNodes: Array[Node], - nodesForGroup: Map[Int, Array[Node]], - treeToNodeToIndexInfo: Map[Int, Map[Int, NodeIndexInfo]], - splits: Array[Array[Split]], - bins: Array[Array[Bin]], - nodeQueue: mutable.Queue[(Int, Node)], - timer: TimeTracker = new TimeTracker, - nodeIdCache: Option[NodeIdCache] = None): Unit = { - - /* - * The high-level descriptions of the best split optimizations are noted here. - * - * *Group-wise training* - * We perform bin calculations for groups of nodes to reduce the number of - * passes over the data. Each iteration requires more computation and storage, - * but saves several iterations over the data. - * - * *Bin-wise computation* - * We use a bin-wise best split computation strategy instead of a straightforward best split - * computation strategy. Instead of analyzing each sample for contribution to the left/right - * child node impurity of every split, we first categorize each feature of a sample into a - * bin. We exploit this structure to calculate aggregates for bins and then use these aggregates - * to calculate information gain for each split. - * - * *Aggregation over partitions* - * Instead of performing a flatMap/reduceByKey operation, we exploit the fact that we know - * the number of splits in advance. Thus, we store the aggregates (at the appropriate - * indices) in a single array for all bins and rely upon the RDD aggregate method to - * drastically reduce the communication overhead. - */ - - // numNodes: Number of nodes in this group - val numNodes = nodesForGroup.values.map(_.size).sum - logDebug("numNodes = " + numNodes) - logDebug("numFeatures = " + metadata.numFeatures) - logDebug("numClasses = " + metadata.numClasses) - logDebug("isMulticlass = " + metadata.isMulticlass) - logDebug("isMulticlassWithCategoricalFeatures = " + - metadata.isMulticlassWithCategoricalFeatures) - logDebug("using nodeIdCache = " + nodeIdCache.nonEmpty.toString) - - /** - * Performs a sequential aggregation over a partition for a particular tree and node. - * - * For each feature, the aggregate sufficient statistics are updated for the relevant - * bins. - * - * @param treeIndex Index of the tree that we want to perform aggregation for. - * @param nodeInfo The node info for the tree node. - * @param agg Array storing aggregate calculation, with a set of sufficient statistics - * for each (node, feature, bin). - * @param baggedPoint Data point being aggregated. - */ - def nodeBinSeqOp( - treeIndex: Int, - nodeInfo: RandomForest.NodeIndexInfo, - agg: Array[DTStatsAggregator], - baggedPoint: BaggedPoint[TreePoint]): Unit = { - if (nodeInfo != null) { - val aggNodeIndex = nodeInfo.nodeIndexInGroup - val featuresForNode = nodeInfo.featureSubset - val instanceWeight = baggedPoint.subsampleWeights(treeIndex) - if (metadata.unorderedFeatures.isEmpty) { - orderedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, instanceWeight, featuresForNode) - } else { - mixedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, splits, - metadata.unorderedFeatures, instanceWeight, featuresForNode) - } - } - } - - /** - * Performs a sequential aggregation over a partition. - * - * Each data point contributes to one node. For each feature, - * the aggregate sufficient statistics are updated for the relevant bins. - * - * @param agg Array storing aggregate calculation, with a set of sufficient statistics for - * each (node, feature, bin). - * @param baggedPoint Data point being aggregated. - * @return agg - */ - def binSeqOp( - agg: Array[DTStatsAggregator], - baggedPoint: BaggedPoint[TreePoint]): Array[DTStatsAggregator] = { - treeToNodeToIndexInfo.foreach { case (treeIndex, nodeIndexToInfo) => - val nodeIndex = predictNodeIndex(topNodes(treeIndex), baggedPoint.datum.binnedFeatures, - bins, metadata.unorderedFeatures) - nodeBinSeqOp(treeIndex, nodeIndexToInfo.getOrElse(nodeIndex, null), agg, baggedPoint) - } - - agg - } - - /** - * Do the same thing as binSeqOp, but with nodeIdCache. - */ - def binSeqOpWithNodeIdCache( - agg: Array[DTStatsAggregator], - dataPoint: (BaggedPoint[TreePoint], Array[Int])): Array[DTStatsAggregator] = { - treeToNodeToIndexInfo.foreach { case (treeIndex, nodeIndexToInfo) => - val baggedPoint = dataPoint._1 - val nodeIdCache = dataPoint._2 - val nodeIndex = nodeIdCache(treeIndex) - nodeBinSeqOp(treeIndex, nodeIndexToInfo.getOrElse(nodeIndex, null), agg, baggedPoint) - } - - agg - } - - /** - * Get node index in group --> features indices map, - * which is a short cut to find feature indices for a node given node index in group - * @param treeToNodeToIndexInfo - * @return - */ - def getNodeToFeatures(treeToNodeToIndexInfo: Map[Int, Map[Int, NodeIndexInfo]]) - : Option[Map[Int, Array[Int]]] = if (!metadata.subsamplingFeatures) { - None - } else { - val mutableNodeToFeatures = new mutable.HashMap[Int, Array[Int]]() - treeToNodeToIndexInfo.values.foreach { nodeIdToNodeInfo => - nodeIdToNodeInfo.values.foreach { nodeIndexInfo => - assert(nodeIndexInfo.featureSubset.isDefined) - mutableNodeToFeatures(nodeIndexInfo.nodeIndexInGroup) = nodeIndexInfo.featureSubset.get - } - } - Some(mutableNodeToFeatures.toMap) - } - - // array of nodes to train indexed by node index in group - val nodes = new Array[Node](numNodes) - nodesForGroup.foreach { case (treeIndex, nodesForTree) => - nodesForTree.foreach { node => - nodes(treeToNodeToIndexInfo(treeIndex)(node.id).nodeIndexInGroup) = node - } - } - - // Calculate best splits for all nodes in the group - timer.start("chooseSplits") - - // In each partition, iterate all instances and compute aggregate stats for each node, - // yield an (nodeIndex, nodeAggregateStats) pair for each node. - // After a `reduceByKey` operation, - // stats of a node will be shuffled to a particular partition and be combined together, - // then best splits for nodes are found there. - // Finally, only best Splits for nodes are collected to driver to construct decision tree. - val nodeToFeatures = getNodeToFeatures(treeToNodeToIndexInfo) - val nodeToFeaturesBc = input.sparkContext.broadcast(nodeToFeatures) - - val partitionAggregates : RDD[(Int, DTStatsAggregator)] = if (nodeIdCache.nonEmpty) { - input.zip(nodeIdCache.get.nodeIdsForInstances).mapPartitions { points => - // Construct a nodeStatsAggregators array to hold node aggregate stats, - // each node will have a nodeStatsAggregator - val nodeStatsAggregators = Array.tabulate(numNodes) { nodeIndex => - val featuresForNode = nodeToFeaturesBc.value.flatMap { nodeToFeatures => - Some(nodeToFeatures(nodeIndex)) - } - new DTStatsAggregator(metadata, featuresForNode) - } - - // iterator all instances in current partition and update aggregate stats - points.foreach(binSeqOpWithNodeIdCache(nodeStatsAggregators, _)) - - // transform nodeStatsAggregators array to (nodeIndex, nodeAggregateStats) pairs, - // which can be combined with other partition using `reduceByKey` - nodeStatsAggregators.view.zipWithIndex.map(_.swap).iterator - } - } else { - input.mapPartitions { points => - // Construct a nodeStatsAggregators array to hold node aggregate stats, - // each node will have a nodeStatsAggregator - val nodeStatsAggregators = Array.tabulate(numNodes) { nodeIndex => - val featuresForNode = nodeToFeaturesBc.value.flatMap { nodeToFeatures => - Some(nodeToFeatures(nodeIndex)) - } - new DTStatsAggregator(metadata, featuresForNode) - } - - // iterator all instances in current partition and update aggregate stats - points.foreach(binSeqOp(nodeStatsAggregators, _)) - - // transform nodeStatsAggregators array to (nodeIndex, nodeAggregateStats) pairs, - // which can be combined with other partition using `reduceByKey` - nodeStatsAggregators.view.zipWithIndex.map(_.swap).iterator - } - } - - val nodeToBestSplits = partitionAggregates.reduceByKey((a, b) => a.merge(b)) - .map { case (nodeIndex, aggStats) => - val featuresForNode = nodeToFeaturesBc.value.map { nodeToFeatures => - nodeToFeatures(nodeIndex) - } - - // find best split for each node - val (split: Split, stats: InformationGainStats, predict: Predict) = - binsToBestSplit(aggStats, splits, featuresForNode, nodes(nodeIndex)) - (nodeIndex, (split, stats, predict)) - }.collectAsMap() - - timer.stop("chooseSplits") - - val nodeIdUpdaters = if (nodeIdCache.nonEmpty) { - Array.fill[mutable.Map[Int, NodeIndexUpdater]]( - metadata.numTrees)(mutable.Map[Int, NodeIndexUpdater]()) - } else { - null - } - - // Iterate over all nodes in this group. - nodesForGroup.foreach { case (treeIndex, nodesForTree) => - nodesForTree.foreach { node => - val nodeIndex = node.id - val nodeInfo = treeToNodeToIndexInfo(treeIndex)(nodeIndex) - val aggNodeIndex = nodeInfo.nodeIndexInGroup - val (split: Split, stats: InformationGainStats, predict: Predict) = - nodeToBestSplits(aggNodeIndex) - logDebug("best split = " + split) - - // Extract info for this node. Create children if not leaf. - val isLeaf = (stats.gain <= 0) || (Node.indexToLevel(nodeIndex) == metadata.maxDepth) - assert(node.id == nodeIndex) - node.predict = predict - node.isLeaf = isLeaf - node.stats = Some(stats) - node.impurity = stats.impurity - logDebug("Node = " + node) - - if (!isLeaf) { - node.split = Some(split) - val childIsLeaf = (Node.indexToLevel(nodeIndex) + 1) == metadata.maxDepth - val leftChildIsLeaf = childIsLeaf || (stats.leftImpurity == 0.0) - val rightChildIsLeaf = childIsLeaf || (stats.rightImpurity == 0.0) - node.leftNode = Some(Node(Node.leftChildIndex(nodeIndex), - stats.leftPredict, stats.leftImpurity, leftChildIsLeaf)) - node.rightNode = Some(Node(Node.rightChildIndex(nodeIndex), - stats.rightPredict, stats.rightImpurity, rightChildIsLeaf)) - - if (nodeIdCache.nonEmpty) { - val nodeIndexUpdater = NodeIndexUpdater( - split = split, - nodeIndex = nodeIndex) - nodeIdUpdaters(treeIndex).put(nodeIndex, nodeIndexUpdater) - } - - // enqueue left child and right child if they are not leaves - if (!leftChildIsLeaf) { - nodeQueue.enqueue((treeIndex, node.leftNode.get)) - } - if (!rightChildIsLeaf) { - nodeQueue.enqueue((treeIndex, node.rightNode.get)) - } - - logDebug("leftChildIndex = " + node.leftNode.get.id + - ", impurity = " + stats.leftImpurity) - logDebug("rightChildIndex = " + node.rightNode.get.id + - ", impurity = " + stats.rightImpurity) - } - } - } - - if (nodeIdCache.nonEmpty) { - // Update the cache if needed. - nodeIdCache.get.updateNodeIndices(input, nodeIdUpdaters, bins) - } - } - - /** - * Calculate the information gain for a given (feature, split) based upon left/right aggregates. - * @param leftImpurityCalculator left node aggregates for this (feature, split) - * @param rightImpurityCalculator right node aggregate for this (feature, split) - * @return information gain and statistics for split - */ - private def calculateGainForSplit( - leftImpurityCalculator: ImpurityCalculator, - rightImpurityCalculator: ImpurityCalculator, - metadata: DecisionTreeMetadata, - impurity: Double): InformationGainStats = { - val leftCount = leftImpurityCalculator.count - val rightCount = rightImpurityCalculator.count - - // If left child or right child doesn't satisfy minimum instances per node, - // then this split is invalid, return invalid information gain stats. - if ((leftCount < metadata.minInstancesPerNode) || - (rightCount < metadata.minInstancesPerNode)) { - return InformationGainStats.invalidInformationGainStats - } - - val totalCount = leftCount + rightCount - - val leftImpurity = leftImpurityCalculator.calculate() // Note: This equals 0 if count = 0 - val rightImpurity = rightImpurityCalculator.calculate() - - val leftWeight = leftCount / totalCount.toDouble - val rightWeight = rightCount / totalCount.toDouble - - val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - - // if information gain doesn't satisfy minimum information gain, - // then this split is invalid, return invalid information gain stats. - if (gain < metadata.minInfoGain) { - return InformationGainStats.invalidInformationGainStats - } - - // calculate left and right predict - val leftPredict = calculatePredict(leftImpurityCalculator) - val rightPredict = calculatePredict(rightImpurityCalculator) - - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, - leftPredict, rightPredict) - } - - private def calculatePredict(impurityCalculator: ImpurityCalculator): Predict = { - val predict = impurityCalculator.predict - val prob = impurityCalculator.prob(predict) - new Predict(predict, prob) - } - - /** - * Calculate predict value for current node, given stats of any split. - * Note that this function is called only once for each node. - * @param leftImpurityCalculator left node aggregates for a split - * @param rightImpurityCalculator right node aggregates for a split - * @return predict value and impurity for current node - */ - private def calculatePredictImpurity( - leftImpurityCalculator: ImpurityCalculator, - rightImpurityCalculator: ImpurityCalculator): (Predict, Double) = { - val parentNodeAgg = leftImpurityCalculator.copy - parentNodeAgg.add(rightImpurityCalculator) - val predict = calculatePredict(parentNodeAgg) - val impurity = parentNodeAgg.calculate() - - (predict, impurity) - } - - /** - * Find the best split for a node. - * @param binAggregates Bin statistics. - * @return tuple for best split: (Split, information gain, prediction at node) - */ - private def binsToBestSplit( - binAggregates: DTStatsAggregator, - splits: Array[Array[Split]], - featuresForNode: Option[Array[Int]], - node: Node): (Split, InformationGainStats, Predict) = { - - // calculate predict and impurity if current node is top node - val level = Node.indexToLevel(node.id) - var predictWithImpurity: Option[(Predict, Double)] = if (level == 0) { - None - } else { - Some((node.predict, node.impurity)) - } - - // For each (feature, split), calculate the gain, and select the best (feature, split). - val (bestSplit, bestSplitStats) = - Range(0, binAggregates.metadata.numFeaturesPerNode).map { featureIndexIdx => - val featureIndex = if (featuresForNode.nonEmpty) { - featuresForNode.get.apply(featureIndexIdx) - } else { - featureIndexIdx - } - val numSplits = binAggregates.metadata.numSplits(featureIndex) - if (binAggregates.metadata.isContinuous(featureIndex)) { - // Cumulative sum (scanLeft) of bin statistics. - // Afterwards, binAggregates for a bin is the sum of aggregates for - // that bin + all preceding bins. - val nodeFeatureOffset = binAggregates.getFeatureOffset(featureIndexIdx) - var splitIndex = 0 - while (splitIndex < numSplits) { - binAggregates.mergeForFeature(nodeFeatureOffset, splitIndex + 1, splitIndex) - splitIndex += 1 - } - // Find best split. - val (bestFeatureSplitIndex, bestFeatureGainStats) = - Range(0, numSplits).map { case splitIdx => - val leftChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, splitIdx) - val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, numSplits) - rightChildStats.subtract(leftChildStats) - predictWithImpurity = Some(predictWithImpurity.getOrElse( - calculatePredictImpurity(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, - rightChildStats, binAggregates.metadata, predictWithImpurity.get._2) - (splitIdx, gainStats) - }.maxBy(_._2.gain) - (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) - } else if (binAggregates.metadata.isUnordered(featureIndex)) { - // Unordered categorical feature - val (leftChildOffset, rightChildOffset) = - binAggregates.getLeftRightFeatureOffsets(featureIndexIdx) - val (bestFeatureSplitIndex, bestFeatureGainStats) = - Range(0, numSplits).map { splitIndex => - val leftChildStats = binAggregates.getImpurityCalculator(leftChildOffset, splitIndex) - val rightChildStats = binAggregates.getImpurityCalculator(rightChildOffset, splitIndex) - predictWithImpurity = Some(predictWithImpurity.getOrElse( - calculatePredictImpurity(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, - rightChildStats, binAggregates.metadata, predictWithImpurity.get._2) - (splitIndex, gainStats) - }.maxBy(_._2.gain) - (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) - } else { - // Ordered categorical feature - val nodeFeatureOffset = binAggregates.getFeatureOffset(featureIndexIdx) - val numBins = binAggregates.metadata.numBins(featureIndex) - - /* Each bin is one category (feature value). - * The bins are ordered based on centroidForCategories, and this ordering determines which - * splits are considered. (With K categories, we consider K - 1 possible splits.) - * - * centroidForCategories is a list: (category, centroid) - */ - val centroidForCategories = if (binAggregates.metadata.isMulticlass) { - // For categorical variables in multiclass classification, - // the bins are ordered by the impurity of their corresponding labels. - Range(0, numBins).map { case featureValue => - val categoryStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) - val centroid = if (categoryStats.count != 0) { - categoryStats.calculate() - } else { - Double.MaxValue - } - (featureValue, centroid) - } - } else { // regression or binary classification - // For categorical variables in regression and binary classification, - // the bins are ordered by the centroid of their corresponding labels. - Range(0, numBins).map { case featureValue => - val categoryStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) - val centroid = if (categoryStats.count != 0) { - categoryStats.predict - } else { - Double.MaxValue - } - (featureValue, centroid) - } - } - - logDebug("Centroids for categorical variable: " + centroidForCategories.mkString(",")) - - // bins sorted by centroids - val categoriesSortedByCentroid = centroidForCategories.toList.sortBy(_._2) - - logDebug("Sorted centroids for categorical variable = " + - categoriesSortedByCentroid.mkString(",")) - - // Cumulative sum (scanLeft) of bin statistics. - // Afterwards, binAggregates for a bin is the sum of aggregates for - // that bin + all preceding bins. - var splitIndex = 0 - while (splitIndex < numSplits) { - val currentCategory = categoriesSortedByCentroid(splitIndex)._1 - val nextCategory = categoriesSortedByCentroid(splitIndex + 1)._1 - binAggregates.mergeForFeature(nodeFeatureOffset, nextCategory, currentCategory) - splitIndex += 1 - } - // lastCategory = index of bin with total aggregates for this (node, feature) - val lastCategory = categoriesSortedByCentroid.last._1 - // Find best split. - val (bestFeatureSplitIndex, bestFeatureGainStats) = - Range(0, numSplits).map { splitIndex => - val featureValue = categoriesSortedByCentroid(splitIndex)._1 - val leftChildStats = - binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) - val rightChildStats = - binAggregates.getImpurityCalculator(nodeFeatureOffset, lastCategory) - rightChildStats.subtract(leftChildStats) - predictWithImpurity = Some(predictWithImpurity.getOrElse( - calculatePredictImpurity(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, - rightChildStats, binAggregates.metadata, predictWithImpurity.get._2) - (splitIndex, gainStats) - }.maxBy(_._2.gain) - val categoriesForSplit = - categoriesSortedByCentroid.map(_._1.toDouble).slice(0, bestFeatureSplitIndex + 1) - val bestFeatureSplit = - new Split(featureIndex, Double.MinValue, Categorical, categoriesForSplit) - (bestFeatureSplit, bestFeatureGainStats) - } - }.maxBy(_._2.gain) - - (bestSplit, bestSplitStats, predictWithImpurity.get._1) - } - - /** - * Returns splits and bins for decision tree calculation. - * Continuous and categorical features are handled differently. - * - * Continuous features: - * For each feature, there are numBins - 1 possible splits representing the possible binary - * decisions at each node in the tree. - * This finds locations (feature values) for splits using a subsample of the data. - * - * Categorical features: - * For each feature, there is 1 bin per split. - * Splits and bins are handled in 2 ways: - * (a) "unordered features" - * For multiclass classification with a low-arity feature - * (i.e., if isMulticlass && isSpaceSufficientForAllCategoricalSplits), - * the feature is split based on subsets of categories. - * (b) "ordered features" - * For regression and binary classification, - * and for multiclass classification with a high-arity feature, - * there is one bin per category. - * - * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] - * @param metadata Learning and dataset metadata - * @return A tuple of (splits, bins). - * Splits is an Array of [[org.apache.spark.mllib.tree.model.Split]] - * of size (numFeatures, numSplits). - * Bins is an Array of [[org.apache.spark.mllib.tree.model.Bin]] - * of size (numFeatures, numBins). - */ - protected[tree] def findSplitsBins( - input: RDD[LabeledPoint], - metadata: DecisionTreeMetadata): (Array[Array[Split]], Array[Array[Bin]]) = { - - logDebug("isMulticlass = " + metadata.isMulticlass) - - val numFeatures = metadata.numFeatures - - // Sample the input only if there are continuous features. - val continuousFeatures = Range(0, numFeatures).filter(metadata.isContinuous) - val sampledInput = if (continuousFeatures.nonEmpty) { - // Calculate the number of samples for approximate quantile calculation. - val requiredSamples = math.max(metadata.maxBins * metadata.maxBins, 10000) - val fraction = if (requiredSamples < metadata.numExamples) { - requiredSamples.toDouble / metadata.numExamples - } else { - 1.0 - } - logDebug("fraction of data used for calculating quantiles = " + fraction) - input.sample(withReplacement = false, fraction, new XORShiftRandom().nextInt()) - } else { - input.sparkContext.emptyRDD[LabeledPoint] - } - - metadata.quantileStrategy match { - case Sort => - findSplitsBinsBySorting(sampledInput, metadata, continuousFeatures) - case MinMax => - throw new UnsupportedOperationException("minmax not supported yet.") - case ApproxHist => - throw new UnsupportedOperationException("approximate histogram not supported yet.") - } - } - - private def findSplitsBinsBySorting( - input: RDD[LabeledPoint], - metadata: DecisionTreeMetadata, - continuousFeatures: IndexedSeq[Int]): (Array[Array[Split]], Array[Array[Bin]]) = { - def findSplits( - featureIndex: Int, - featureSamples: Iterable[Double]): (Int, (Array[Split], Array[Bin])) = { - val splits = { - val featureSplits = findSplitsForContinuousFeature( - featureSamples.toArray, - metadata, - featureIndex) - logDebug(s"featureIndex = $featureIndex, numSplits = ${featureSplits.length}") - - featureSplits.map(threshold => new Split(featureIndex, threshold, Continuous, Nil)) - } - - val bins = { - val lowSplit = new DummyLowSplit(featureIndex, Continuous) - val highSplit = new DummyHighSplit(featureIndex, Continuous) - - // tack the dummy splits on either side of the computed splits - val allSplits = lowSplit +: splits.toSeq :+ highSplit - - // slide across the split points pairwise to allocate the bins - allSplits.sliding(2).map { - case Seq(left, right) => new Bin(left, right, Continuous, Double.MinValue) - }.toArray - } - - (featureIndex, (splits, bins)) - } - - val continuousSplits = { - // reduce the parallelism for split computations when there are less - // continuous features than input partitions. this prevents tasks from - // being spun up that will definitely do no work. - val numPartitions = math.min(continuousFeatures.length, input.partitions.length) - - input - .flatMap(point => continuousFeatures.map(idx => (idx, point.features(idx)))) - .groupByKey(numPartitions) - .map { case (k, v) => findSplits(k, v) } - .collectAsMap() - } - - val numFeatures = metadata.numFeatures - val (splits, bins) = Range(0, numFeatures).unzip { - case i if metadata.isContinuous(i) => - val (split, bin) = continuousSplits(i) - metadata.setNumSplits(i, split.length) - (split, bin) - - case i if metadata.isCategorical(i) && metadata.isUnordered(i) => - // Unordered features - // 2^(maxFeatureValue - 1) - 1 combinations - val featureArity = metadata.featureArity(i) - val split = Range(0, metadata.numSplits(i)).map { splitIndex => - val categories = extractMultiClassCategories(splitIndex + 1, featureArity) - new Split(i, Double.MinValue, Categorical, categories) - } - - // 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. - (split.toArray, Array.empty[Bin]) - - case i if metadata.isCategorical(i) => - // Ordered features - // Bins correspond to feature values, so we do not need to compute splits or bins - // beforehand. Splits are constructed as needed during training. - (Array.empty[Split], Array.empty[Bin]) - } - - (splits.toArray, bins.toArray) - } - - /** - * Nested method to extract list of eligible categories given an index. It extracts the - * position of ones in a binary representation of the input. If binary - * representation of an number is 01101 (13), the output list should (3.0, 2.0, - * 0.0). The maxFeatureValue depict the number of rightmost digits that will be tested for ones. - */ - private[tree] def extractMultiClassCategories( - input: Int, - maxFeatureValue: Int): List[Double] = { - var categories = List[Double]() - var j = 0 - var bitShiftedInput = input - while (j < maxFeatureValue) { - if (bitShiftedInput % 2 != 0) { - // updating the list of categories. - categories = j.toDouble :: categories - } - // Right shift by one - bitShiftedInput = bitShiftedInput >> 1 - j += 1 - } - categories - } - - /** - * Find splits for a continuous feature - * NOTE: Returned number of splits is set based on `featureSamples` and - * could be different from the specified `numSplits`. - * The `numSplits` attribute in the `DecisionTreeMetadata` class will be set accordingly. - * @param featureSamples feature values of each sample - * @param metadata decision tree metadata - * NOTE: `metadata.numbins` will be changed accordingly - * if there are not enough splits to be found - * @param featureIndex feature index to find splits - * @return array of splits - */ - private[tree] def findSplitsForContinuousFeature( - featureSamples: Array[Double], - metadata: DecisionTreeMetadata, - featureIndex: Int): Array[Double] = { - require(metadata.isContinuous(featureIndex), - "findSplitsForContinuousFeature can only be used to find splits for a continuous feature.") - - val splits = { - val numSplits = metadata.numSplits(featureIndex) - - // get count for each distinct value - val valueCountMap = featureSamples.foldLeft(Map.empty[Double, Int]) { (m, x) => - m + ((x, m.getOrElse(x, 0) + 1)) - } - // sort distinct values - val valueCounts = valueCountMap.toSeq.sortBy(_._1).toArray - - // if possible splits is not enough or just enough, just return all possible splits - val possibleSplits = valueCounts.length - if (possibleSplits <= numSplits) { - valueCounts.map(_._1) - } else { - // stride between splits - val stride: Double = featureSamples.length.toDouble / (numSplits + 1) - logDebug("stride = " + stride) - - // iterate `valueCount` to find splits - val splitsBuilder = Array.newBuilder[Double] - var index = 1 - // currentCount: sum of counts of values that have been visited - var currentCount = valueCounts(0)._2 - // targetCount: target value for `currentCount`. - // If `currentCount` is closest value to `targetCount`, - // then current value is a split threshold. - // After finding a split threshold, `targetCount` is added by stride. - var targetCount = stride - while (index < valueCounts.length) { - val previousCount = currentCount - currentCount += valueCounts(index)._2 - val previousGap = math.abs(previousCount - targetCount) - val currentGap = math.abs(currentCount - targetCount) - // If adding count of current value to currentCount - // makes the gap between currentCount and targetCount smaller, - // previous value is a split threshold. - if (previousGap < currentGap) { - splitsBuilder += valueCounts(index - 1)._1 - targetCount += stride - } - index += 1 - } - - splitsBuilder.result() - } - } - - // TODO: Do not fail; just ignore the useless feature. - assert(splits.length > 0, - s"DecisionTree could not handle feature $featureIndex since it had only 1 unique value." + - " Please remove this feature and then try again.") - - // the split metadata must be updated on the driver - - splits - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index 729a21157482..df2c1b02f4f4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -17,22 +17,19 @@ package org.apache.spark.mllib.tree -import org.apache.spark.Logging import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD -import org.apache.spark.mllib.impl.PeriodicRDDCheckpointer +import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} +import org.apache.spark.ml.tree.impl.{GradientBoostedTrees => NewGBT} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.BoostingStrategy -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.impl.TimeTracker -import org.apache.spark.mllib.tree.impurity.Variance -import org.apache.spark.mllib.tree.model.{DecisionTreeModel, GradientBoostedTreesModel} +import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel /** * A class that implements - * [[http://en.wikipedia.org/wiki/Gradient_boosting Stochastic Gradient Boosting]] + * Stochastic Gradient Boosting * for regression and binary classification. * * The implementation is based upon: @@ -47,33 +44,37 @@ import org.apache.spark.storage.StorageLevel * for other loss functions. * * @param boostingStrategy Parameters for the gradient boosting algorithm. + * @param seed Random seed. */ @Since("1.2.0") -class GradientBoostedTrees @Since("1.2.0") (private val boostingStrategy: BoostingStrategy) +class GradientBoostedTrees private[spark] ( + private val boostingStrategy: BoostingStrategy, + private val seed: Int) extends Serializable with Logging { + /** + * @param boostingStrategy Parameters for the gradient boosting algorithm. + */ + @Since("1.2.0") + def this(boostingStrategy: BoostingStrategy) = this(boostingStrategy, seed = 0) + /** * Method to train a gradient boosting model + * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * @return a gradient boosted trees model that can be used for prediction + * @return GradientBoostedTreesModel that can be used for prediction. */ @Since("1.2.0") def run(input: RDD[LabeledPoint]): GradientBoostedTreesModel = { val algo = boostingStrategy.treeStrategy.algo - algo match { - case Regression => - GradientBoostedTrees.boost(input, input, boostingStrategy, validate = false) - case Classification => - // Map labels to -1, +1 so binary classification can be treated as regression. - val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) - GradientBoostedTrees.boost(remappedInput, remappedInput, boostingStrategy, validate = false) - case _ => - throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.") - } + val (trees, treeWeights) = NewGBT.run(input.map { point => + NewLabeledPoint(point.label, point.features.asML) + }, boostingStrategy, seed.toLong) + new GradientBoostedTreesModel(algo, trees.map(_.toOld), treeWeights) } /** - * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoostedTrees!#run]]. + * Java-friendly API for `org.apache.spark.mllib.tree.GradientBoostedTrees.run`. */ @Since("1.2.0") def run(input: JavaRDD[LabeledPoint]): GradientBoostedTreesModel = { @@ -82,37 +83,30 @@ class GradientBoostedTrees @Since("1.2.0") (private val boostingStrategy: Boosti /** * Method to validate a gradient boosting model + * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * @param validationInput Validation dataset. * This dataset should be different from the training dataset, * but it should follow the same distribution. * E.g., these two datasets could be created from an original dataset - * by using [[org.apache.spark.rdd.RDD.randomSplit()]] - * @return a gradient boosted trees model that can be used for prediction + * by using `org.apache.spark.rdd.RDD.randomSplit()` + * @return GradientBoostedTreesModel that can be used for prediction. */ @Since("1.4.0") def runWithValidation( input: RDD[LabeledPoint], validationInput: RDD[LabeledPoint]): GradientBoostedTreesModel = { val algo = boostingStrategy.treeStrategy.algo - algo match { - case Regression => - GradientBoostedTrees.boost(input, validationInput, boostingStrategy, validate = true) - case Classification => - // Map labels to -1, +1 so binary classification can be treated as regression. - val remappedInput = input.map( - x => new LabeledPoint((x.label * 2) - 1, x.features)) - val remappedValidationInput = validationInput.map( - x => new LabeledPoint((x.label * 2) - 1, x.features)) - GradientBoostedTrees.boost(remappedInput, remappedValidationInput, boostingStrategy, - validate = true) - case _ => - throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.") - } + val (trees, treeWeights) = NewGBT.runWithValidation(input.map { point => + NewLabeledPoint(point.label, point.features.asML) + }, validationInput.map { point => + NewLabeledPoint(point.label, point.features.asML) + }, boostingStrategy, seed.toLong) + new GradientBoostedTreesModel(algo, trees.map(_.toOld), treeWeights) } /** - * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoostedTrees!#runWithValidation]]. + * Java-friendly API for `org.apache.spark.mllib.tree.GradientBoostedTrees.runWithValidation`. */ @Since("1.4.0") def runWithValidation( @@ -132,17 +126,17 @@ object GradientBoostedTrees extends Logging { * For classification, labels should take values {0, 1, ..., numClasses-1}. * For regression, labels are real numbers. * @param boostingStrategy Configuration options for the boosting algorithm. - * @return a gradient boosted trees model that can be used for prediction + * @return GradientBoostedTreesModel that can be used for prediction. */ @Since("1.2.0") def train( input: RDD[LabeledPoint], boostingStrategy: BoostingStrategy): GradientBoostedTreesModel = { - new GradientBoostedTrees(boostingStrategy).run(input) + new GradientBoostedTrees(boostingStrategy, seed = 0).run(input) } /** - * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoostedTrees$#train]] + * Java-friendly API for `org.apache.spark.mllib.tree.GradientBoostedTrees.train` */ @Since("1.2.0") def train( @@ -150,145 +144,4 @@ object GradientBoostedTrees extends Logging { boostingStrategy: BoostingStrategy): GradientBoostedTreesModel = { train(input.rdd, boostingStrategy) } - - /** - * Internal method for performing regression using trees as base learners. - * @param input training dataset - * @param validationInput validation dataset, ignored if validate is set to false. - * @param boostingStrategy boosting parameters - * @param validate whether or not to use the validation dataset. - * @return a gradient boosted trees model that can be used for prediction - */ - private def boost( - input: RDD[LabeledPoint], - validationInput: RDD[LabeledPoint], - boostingStrategy: BoostingStrategy, - validate: Boolean): GradientBoostedTreesModel = { - val timer = new TimeTracker() - timer.start("total") - timer.start("init") - - boostingStrategy.assertValid() - - // Initialize gradient boosting parameters - val numIterations = boostingStrategy.numIterations - val baseLearners = new Array[DecisionTreeModel](numIterations) - val baseLearnerWeights = new Array[Double](numIterations) - val loss = boostingStrategy.loss - val learningRate = boostingStrategy.learningRate - // Prepare strategy for individual trees, which use regression with variance impurity. - val treeStrategy = boostingStrategy.treeStrategy.copy - val validationTol = boostingStrategy.validationTol - treeStrategy.algo = Regression - treeStrategy.impurity = Variance - treeStrategy.assertValid() - - // Cache input - val persistedInput = if (input.getStorageLevel == StorageLevel.NONE) { - input.persist(StorageLevel.MEMORY_AND_DISK) - true - } else { - false - } - - // Prepare periodic checkpointers - val predErrorCheckpointer = new PeriodicRDDCheckpointer[(Double, Double)]( - treeStrategy.getCheckpointInterval, input.sparkContext) - val validatePredErrorCheckpointer = new PeriodicRDDCheckpointer[(Double, Double)]( - treeStrategy.getCheckpointInterval, input.sparkContext) - - timer.stop("init") - - logDebug("##########") - logDebug("Building tree 0") - logDebug("##########") - - // Initialize tree - timer.start("building tree 0") - val firstTreeModel = new DecisionTree(treeStrategy).run(input) - val firstTreeWeight = 1.0 - baseLearners(0) = firstTreeModel - baseLearnerWeights(0) = firstTreeWeight - - var predError: RDD[(Double, Double)] = GradientBoostedTreesModel. - computeInitialPredictionAndError(input, firstTreeWeight, firstTreeModel, loss) - predErrorCheckpointer.update(predError) - logDebug("error of gbt = " + predError.values.mean()) - - // Note: A model of type regression is used since we require raw prediction - timer.stop("building tree 0") - - var validatePredError: RDD[(Double, Double)] = GradientBoostedTreesModel. - computeInitialPredictionAndError(validationInput, firstTreeWeight, firstTreeModel, loss) - if (validate) validatePredErrorCheckpointer.update(validatePredError) - var bestValidateError = if (validate) validatePredError.values.mean() else 0.0 - var bestM = 1 - - var m = 1 - var doneLearning = false - while (m < numIterations && !doneLearning) { - // Update data with pseudo-residuals - val data = predError.zip(input).map { case ((pred, _), point) => - LabeledPoint(-loss.gradient(pred, point.label), point.features) - } - - timer.start(s"building tree $m") - logDebug("###################################################") - logDebug("Gradient boosting tree iteration " + m) - logDebug("###################################################") - val model = new DecisionTree(treeStrategy).run(data) - timer.stop(s"building tree $m") - // Update partial model - baseLearners(m) = model - // Note: The setting of baseLearnerWeights is incorrect for losses other than SquaredError. - // Technically, the weight should be optimized for the particular loss. - // However, the behavior should be reasonable, though not optimal. - baseLearnerWeights(m) = learningRate - - predError = GradientBoostedTreesModel.updatePredictionError( - input, predError, baseLearnerWeights(m), baseLearners(m), loss) - predErrorCheckpointer.update(predError) - logDebug("error of gbt = " + predError.values.mean()) - - if (validate) { - // Stop training early if - // 1. Reduction in error is less than the validationTol or - // 2. If the error increases, that is if the model is overfit. - // We want the model returned corresponding to the best validation error. - - validatePredError = GradientBoostedTreesModel.updatePredictionError( - validationInput, validatePredError, baseLearnerWeights(m), baseLearners(m), loss) - validatePredErrorCheckpointer.update(validatePredError) - val currentValidateError = validatePredError.values.mean() - if (bestValidateError - currentValidateError < validationTol * Math.max( - currentValidateError, 0.01)) { - doneLearning = true - } else if (currentValidateError < bestValidateError) { - bestValidateError = currentValidateError - bestM = m + 1 - } - } - m += 1 - } - - timer.stop("total") - - logInfo("Internal timing for DecisionTree:") - logInfo(s"$timer") - - predErrorCheckpointer.deleteAllCheckpoints() - validatePredErrorCheckpointer.deleteAllCheckpoints() - if (persistedInput) input.unpersist() - - if (validate) { - new GradientBoostedTreesModel( - boostingStrategy.treeStrategy.algo, - baseLearners.slice(0, bestM), - baseLearnerWeights.slice(0, bestM)) - } else { - new GradientBoostedTreesModel( - boostingStrategy.treeStrategy.algo, baseLearners, baseLearnerWeights) - } - } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index a684cdd18c2f..d1331a57de27 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -17,29 +17,26 @@ package org.apache.spark.mllib.tree -import java.io.IOException - -import scala.collection.mutable import scala.collection.JavaConverters._ +import scala.util.Try -import org.apache.spark.Logging import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging +import org.apache.spark.ml.tree.{DecisionTreeModel => NewDTModel, RandomForestParams => NewRFParams} +import org.apache.spark.ml.tree.impl.{RandomForest => NewRandomForest} import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ -import org.apache.spark.mllib.tree.impl.{BaggedPoint, DecisionTreeMetadata, NodeIdCache, - TimeTracker, TreePoint} +import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impurity.Impurities import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils -import org.apache.spark.util.random.SamplingUtils + /** - * A class that implements a [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] + * A class that implements a Random Forest * learning algorithm for classification and regression. * It supports both continuous and categorical features. * @@ -48,21 +45,27 @@ import org.apache.spark.util.random.SamplingUtils * - sqrt: recommended by Breiman manual for random forests * - The defaults of sqrt (classification) and onethird (regression) match the R randomForest * package. - * @see [[http://www.stat.berkeley.edu/~breiman/randomforest2001.pdf Breiman (2001)]] - * @see [[http://www.stat.berkeley.edu/~breiman/Using_random_forests_V3.1.pdf Breiman manual for - * random forests]] * + * @see Breiman (2001) + * @see + * Breiman manual for random forests * @param strategy The configuration parameters for the random forest algorithm which specify - * the type of algorithm (classification, regression, etc.), feature type + * the type of random forest (classification or regression), feature type * (continuous, categorical), depth of the tree, quantile calculation strategy, * etc. - * @param numTrees If 1, then no bootstrapping is used. If > 1, then bootstrapping is done. + * @param numTrees If 1, then no bootstrapping is used. If greater than 1, then bootstrapping is + * done. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto", "all", "sqrt", "log2", "onethird". + * Supported values: "auto", "all", "sqrt", "log2", "onethird". + * Supported numerical values: "(0.0-1.0]", "[1-n]". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; - * if numTrees > 1 (forest) set to "sqrt" for classification and - * to "onethird" for regression. + * if numTrees is greater than 1 (forest) set to "sqrt" for + * classification and to "onethird" for regression. + * If a real value "n" in the range (0, 1.0] is set, + * use n * number of features. + * If an integer value "n" in the range (1, num features) is set, + * use n features. * @param seed Random seed for bootstrapping and choosing feature subsets. */ private class RandomForest ( @@ -72,188 +75,25 @@ private class RandomForest ( private val seed: Int) extends Serializable with Logging { - /* - ALGORITHM - This is a sketch of the algorithm to help new developers. - - The algorithm partitions data by instances (rows). - On each iteration, the algorithm splits a set of nodes. In order to choose the best split - for a given node, sufficient statistics are collected from the distributed data. - For each node, the statistics are collected to some worker node, and that worker selects - the best split. - - This setup requires discretization of continuous features. This binning is done in the - findSplitsBins() method during initialization, after which each continuous feature becomes - an ordered discretized feature with at most maxBins possible values. - - The main loop in the algorithm operates on a queue of nodes (nodeQueue). These nodes - lie at the periphery of the tree being trained. If multiple trees are being trained at once, - then this queue contains nodes from all of them. Each iteration works roughly as follows: - On the master node: - - Some number of nodes are pulled off of the queue (based on the amount of memory - required for their sufficient statistics). - - For random forests, if featureSubsetStrategy is not "all," then a subset of candidate - features are chosen for each node. See method selectNodesToSplit(). - On worker nodes, via method findBestSplits(): - - The worker makes one pass over its subset of instances. - - For each (tree, node, feature, split) tuple, the worker collects statistics about - splitting. Note that the set of (tree, node) pairs is limited to the nodes selected - from the queue for this iteration. The set of features considered can also be limited - based on featureSubsetStrategy. - - For each node, the statistics for that node are aggregated to a particular worker - via reduceByKey(). The designated worker chooses the best (feature, split) pair, - or chooses to stop splitting if the stopping criteria are met. - On the master node: - - The master collects all decisions about splitting nodes and updates the model. - - The updated model is passed to the workers on the next iteration. - This process continues until the node queue is empty. - - Most of the methods in this implementation support the statistics aggregation, which is - the heaviest part of the computation. In general, this implementation is bound by either - the cost of statistics computation on workers or by communicating the sufficient statistics. - */ - strategy.assertValid() require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.") - require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy), + require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy) + || Try(featureSubsetStrategy.toInt).filter(_ > 0).isSuccess + || Try(featureSubsetStrategy.toDouble).filter(_ > 0).filter(_ <= 1.0).isSuccess, s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." + - s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.") + s" Supported values: ${NewRFParams.supportedFeatureSubsetStrategies.mkString(", ")}," + + s" (0.0-1.0], [1-n].") /** * Method to train a decision tree model over an RDD - * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] - * @return a random forest model that can be used for prediction + * + * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * @return RandomForestModel that can be used for prediction. */ def run(input: RDD[LabeledPoint]): RandomForestModel = { - - val timer = new TimeTracker() - - timer.start("total") - - timer.start("init") - - val retaggedInput = input.retag(classOf[LabeledPoint]) - val metadata = - DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy) - logDebug("algo = " + strategy.algo) - logDebug("numTrees = " + numTrees) - logDebug("seed = " + seed) - logDebug("maxBins = " + metadata.maxBins) - logDebug("featureSubsetStrategy = " + featureSubsetStrategy) - logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode) - logDebug("subsamplingRate = " + strategy.subsamplingRate) - - // Find the splits and the corresponding bins (interval between the splits) using a sample - // of the input data. - timer.start("findSplitsBins") - val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata) - timer.stop("findSplitsBins") - logDebug("numBins: feature: number of bins") - logDebug(Range(0, metadata.numFeatures).map { featureIndex => - s"\t$featureIndex\t${metadata.numBins(featureIndex)}" - }.mkString("\n")) - - // Bin feature values (TreePoint representation). - // Cache input RDD for speedup during multiple passes. - val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata) - - val withReplacement = if (numTrees > 1) true else false - - val baggedInput - = BaggedPoint.convertToBaggedRDD(treeInput, - strategy.subsamplingRate, numTrees, - withReplacement, seed).persist(StorageLevel.MEMORY_AND_DISK) - - // depth of the decision tree - val maxDepth = strategy.maxDepth - require(maxDepth <= 30, - s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.") - - // Max memory usage for aggregates - // TODO: Calculate memory usage more precisely. - val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L - logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") - val maxMemoryPerNode = { - val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) { - // Find numFeaturesPerNode largest bins to get an upper bound on memory usage. - Some(metadata.numBins.zipWithIndex.sortBy(- _._1) - .take(metadata.numFeaturesPerNode).map(_._2)) - } else { - None - } - RandomForest.aggregateSizeForNode(metadata, featureSubset) * 8L - } - require(maxMemoryPerNode <= maxMemoryUsage, - s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," + - " which is too small for the given features." + - s" Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}") - - timer.stop("init") - - /* - * The main idea here is to perform group-wise training of the decision tree nodes thus - * reducing the passes over the data from (# nodes) to (# nodes / maxNumberOfNodesPerGroup). - * Each data sample is handled by a particular node (or it reaches a leaf and is not used - * in lower levels). - */ - - // Create an RDD of node Id cache. - // At first, all the rows belong to the root nodes (node Id == 1). - val nodeIdCache = if (strategy.useNodeIdCache) { - Some(NodeIdCache.init( - data = baggedInput, - numTrees = numTrees, - checkpointInterval = strategy.checkpointInterval, - initVal = 1)) - } else { - None - } - - // FIFO queue of nodes to train: (treeIndex, node) - val nodeQueue = new mutable.Queue[(Int, Node)]() - - val rng = new scala.util.Random() - rng.setSeed(seed) - - // Allocate and queue root nodes. - val topNodes: Array[Node] = Array.fill[Node](numTrees)(Node.emptyNode(nodeIndex = 1)) - Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex)))) - - while (nodeQueue.nonEmpty) { - // Collect some nodes to split, and choose features for each node (if subsampling). - // Each group of nodes may come from one or multiple trees, and at multiple levels. - val (nodesForGroup, treeToNodeToIndexInfo) = - RandomForest.selectNodesToSplit(nodeQueue, maxMemoryUsage, metadata, rng) - // Sanity check (should never occur): - assert(nodesForGroup.size > 0, - s"RandomForest selected empty nodesForGroup. Error for unknown reason.") - - // Choose node splits, and enqueue new nodes as needed. - timer.start("findBestSplits") - DecisionTree.findBestSplits(baggedInput, metadata, topNodes, nodesForGroup, - treeToNodeToIndexInfo, splits, bins, nodeQueue, timer, nodeIdCache = nodeIdCache) - timer.stop("findBestSplits") - } - - baggedInput.unpersist() - - timer.stop("total") - - logInfo("Internal timing for DecisionTree:") - logInfo(s"$timer") - - // Delete any remaining checkpoints used for node Id cache. - if (nodeIdCache.nonEmpty) { - try { - nodeIdCache.get.deleteAllCheckpoints() - } catch { - case e: IOException => - logWarning(s"delete all checkpoints failed. Error reason: ${e.getMessage}") - } - } - - val trees = topNodes.map(topNode => new DecisionTreeModel(topNode, strategy.algo)) - new RandomForestModel(strategy.algo, trees) + val trees: Array[NewDTModel] = NewRandomForest.run(input.map(_.asML), strategy, numTrees, + featureSubsetStrategy, seed.toLong, None) + new RandomForestModel(strategy.algo, trees.map(_.toOld)) } } @@ -269,12 +109,12 @@ object RandomForest extends Serializable with Logging { * @param strategy Parameters for training each tree in the forest. * @param numTrees Number of trees in the random forest. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto", "all", "sqrt", "log2", "onethird". + * Supported values: "auto", "all", "sqrt", "log2", "onethird". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; - * if numTrees > 1 (forest) set to "sqrt". - * @param seed Random seed for bootstrapping and choosing feature subsets. - * @return a random forest model that can be used for prediction + * if numTrees is greater than 1 (forest) set to "sqrt". + * @param seed Random seed for bootstrapping and choosing feature subsets. + * @return RandomForestModel that can be used for prediction. */ @Since("1.2.0") def trainClassifier( @@ -294,25 +134,25 @@ object RandomForest extends Serializable with Logging { * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * Labels should take values {0, 1, ..., numClasses-1}. - * @param numClasses number of classes for classification. - * @param categoricalFeaturesInfo Map storing arity of categorical features. - * E.g., an entry (n -> k) indicates that feature n is categorical - * with k categories indexed from 0: {0, 1, ..., k-1}. + * @param numClasses Number of classes for classification. + * @param categoricalFeaturesInfo Map storing arity of categorical features. An entry (n to k) + * indicates that feature n is categorical with k categories + * indexed from 0: {0, 1, ..., k-1}. * @param numTrees Number of trees in the random forest. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto", "all", "sqrt", "log2", "onethird". + * Supported values: "auto", "all", "sqrt", "log2", "onethird". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; - * if numTrees > 1 (forest) set to "sqrt". + * if numTrees is greater than 1 (forest) set to "sqrt". * @param impurity Criterion used for information gain calculation. * Supported values: "gini" (recommended) or "entropy". - * @param maxDepth Maximum depth of the tree. - * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * (suggested value: 4) - * @param maxBins maximum number of bins used for splitting features - * (suggested value: 100) - * @param seed Random seed for bootstrapping and choosing feature subsets. - * @return a random forest model that can be used for prediction + * @param maxDepth Maximum depth of the tree (e.g. depth 0 means 1 leaf node, depth 1 means + * 1 internal node + 2 leaf nodes). + * (suggested value: 4) + * @param maxBins Maximum number of bins used for splitting features + * (suggested value: 100) + * @param seed Random seed for bootstrapping and choosing feature subsets. + * @return RandomForestModel that can be used for prediction. */ @Since("1.2.0") def trainClassifier( @@ -332,7 +172,7 @@ object RandomForest extends Serializable with Logging { } /** - * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainClassifier]] + * Java-friendly API for `org.apache.spark.mllib.tree.RandomForest.trainClassifier` */ @Since("1.2.0") def trainClassifier( @@ -358,12 +198,12 @@ object RandomForest extends Serializable with Logging { * @param strategy Parameters for training each tree in the forest. * @param numTrees Number of trees in the random forest. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto", "all", "sqrt", "log2", "onethird". + * Supported values: "auto", "all", "sqrt", "log2", "onethird". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; - * if numTrees > 1 (forest) set to "onethird". - * @param seed Random seed for bootstrapping and choosing feature subsets. - * @return a random forest model that can be used for prediction + * if numTrees is greater than 1 (forest) set to "onethird". + * @param seed Random seed for bootstrapping and choosing feature subsets. + * @return RandomForestModel that can be used for prediction. */ @Since("1.2.0") def trainRegressor( @@ -383,24 +223,24 @@ object RandomForest extends Serializable with Logging { * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * Labels are real numbers. - * @param categoricalFeaturesInfo Map storing arity of categorical features. - * E.g., an entry (n -> k) indicates that feature n is categorical - * with k categories indexed from 0: {0, 1, ..., k-1}. + * @param categoricalFeaturesInfo Map storing arity of categorical features. An entry (n to k) + * indicates that feature n is categorical with k categories + * indexed from 0: {0, 1, ..., k-1}. * @param numTrees Number of trees in the random forest. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto", "all", "sqrt", "log2", "onethird". + * Supported values: "auto", "all", "sqrt", "log2", "onethird". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; - * if numTrees > 1 (forest) set to "onethird". + * if numTrees is greater than 1 (forest) set to "onethird". * @param impurity Criterion used for information gain calculation. - * Supported values: "variance". - * @param maxDepth Maximum depth of the tree. - * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * (suggested value: 4) - * @param maxBins maximum number of bins used for splitting features - * (suggested value: 100) - * @param seed Random seed for bootstrapping and choosing feature subsets. - * @return a random forest model that can be used for prediction + * The only supported value for regression is "variance". + * @param maxDepth Maximum depth of the tree. (e.g., depth 0 means 1 leaf node, depth 1 means + * 1 internal node + 2 leaf nodes). + * (suggested value: 4) + * @param maxBins Maximum number of bins used for splitting features. + * (suggested value: 100) + * @param seed Random seed for bootstrapping and choosing feature subsets. + * @return RandomForestModel that can be used for prediction. */ @Since("1.2.0") def trainRegressor( @@ -419,7 +259,7 @@ object RandomForest extends Serializable with Logging { } /** - * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainRegressor]] + * Java-friendly API for `org.apache.spark.mllib.tree.RandomForest.trainRegressor` */ @Since("1.2.0") def trainRegressor( @@ -440,86 +280,5 @@ object RandomForest extends Serializable with Logging { * List of supported feature subset sampling strategies. */ @Since("1.2.0") - val supportedFeatureSubsetStrategies: Array[String] = - Array("auto", "all", "sqrt", "log2", "onethird") - - private[tree] class NodeIndexInfo( - val nodeIndexInGroup: Int, - val featureSubset: Option[Array[Int]]) extends Serializable - - /** - * Pull nodes off of the queue, and collect a group of nodes to be split on this iteration. - * This tracks the memory usage for aggregates and stops adding nodes when too much memory - * will be needed; this allows an adaptive number of nodes since different nodes may require - * different amounts of memory (if featureSubsetStrategy is not "all"). - * - * @param nodeQueue Queue of nodes to split. - * @param maxMemoryUsage Bound on size of aggregate statistics. - * @return (nodesForGroup, treeToNodeToIndexInfo). - * nodesForGroup holds the nodes to split: treeIndex --> nodes in tree. - * - * treeToNodeToIndexInfo holds indices selected features for each node: - * treeIndex --> (global) node index --> (node index in group, feature indices). - * The (global) node index is the index in the tree; the node index in group is the - * index in [0, numNodesInGroup) of the node in this group. - * The feature indices are None if not subsampling features. - */ - private[tree] def selectNodesToSplit( - nodeQueue: mutable.Queue[(Int, Node)], - maxMemoryUsage: Long, - metadata: DecisionTreeMetadata, - rng: scala.util.Random): (Map[Int, Array[Node]], Map[Int, Map[Int, NodeIndexInfo]]) = { - // Collect some nodes to split: - // nodesForGroup(treeIndex) = nodes to split - val mutableNodesForGroup = new mutable.HashMap[Int, mutable.ArrayBuffer[Node]]() - val mutableTreeToNodeToIndexInfo = - new mutable.HashMap[Int, mutable.HashMap[Int, NodeIndexInfo]]() - var memUsage: Long = 0L - var numNodesInGroup = 0 - while (nodeQueue.nonEmpty && memUsage < maxMemoryUsage) { - val (treeIndex, node) = nodeQueue.head - // Choose subset of features for node (if subsampling). - val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) { - Some(SamplingUtils.reservoirSampleAndCount(Range(0, - metadata.numFeatures).iterator, metadata.numFeaturesPerNode, rng.nextLong)._1) - } else { - None - } - // Check if enough memory remains to add this node to the group. - val nodeMemUsage = RandomForest.aggregateSizeForNode(metadata, featureSubset) * 8L - if (memUsage + nodeMemUsage <= maxMemoryUsage) { - nodeQueue.dequeue() - mutableNodesForGroup.getOrElseUpdate(treeIndex, new mutable.ArrayBuffer[Node]()) += node - mutableTreeToNodeToIndexInfo - .getOrElseUpdate(treeIndex, new mutable.HashMap[Int, NodeIndexInfo]())(node.id) - = new NodeIndexInfo(numNodesInGroup, featureSubset) - } - numNodesInGroup += 1 - memUsage += nodeMemUsage - } - // Convert mutable maps to immutable ones. - val nodesForGroup: Map[Int, Array[Node]] = mutableNodesForGroup.mapValues(_.toArray).toMap - val treeToNodeToIndexInfo = mutableTreeToNodeToIndexInfo.mapValues(_.toMap).toMap - (nodesForGroup, treeToNodeToIndexInfo) - } - - /** - * Get the number of values to be stored for this node in the bin aggregates. - * @param featureSubset Indices of features which may be split at this node. - * If None, then use all features. - */ - private[tree] def aggregateSizeForNode( - metadata: DecisionTreeMetadata, - featureSubset: Option[Array[Int]]): Long = { - val totalBins = if (featureSubset.nonEmpty) { - featureSubset.get.map(featureIndex => metadata.numBins(featureIndex).toLong).sum - } else { - metadata.numBins.map(_.toLong).sum - } - if (metadata.isClassification) { - metadata.numClasses * totalBins - } else { - 3 * totalBins - } - } + val supportedFeatureSubsetStrategies: Array[String] = NewRFParams.supportedFeatureSubsetStrategies } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala index 853c7319ec44..2436ce40866e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala @@ -17,14 +17,12 @@ package org.apache.spark.mllib.tree.configuration -import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.annotation.Since /** - * :: Experimental :: * Enum to select the algorithm for the decision tree */ @Since("1.0.0") -@Experimental object Algo extends Enumeration { @Since("1.0.0") type Algo = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala index d2513a9d5c5b..4334b316cc83 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala @@ -21,7 +21,7 @@ import scala.beans.BeanProperty import org.apache.spark.annotation.Since import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.loss.{LogLoss, SquaredError, Loss} +import org.apache.spark.mllib.tree.loss.{LogLoss, Loss, SquaredError} /** * Configuration options for [[org.apache.spark.mllib.tree.GradientBoostedTrees]]. @@ -36,14 +36,14 @@ import org.apache.spark.mllib.tree.loss.{LogLoss, SquaredError, Loss} * @param validationTol validationTol is a condition which decides iteration termination when * runWithValidation is used. * The end of iteration is decided based on below logic: - * If the current loss on the validation set is > 0.01, the diff + * If the current loss on the validation set is greater than 0.01, the diff * of validation error is compared to relative tolerance which is * validationTol * (current loss on the validation set). - * If the current loss on the validation set is <= 0.01, the diff - * of validation error is compared to absolute tolerance which is + * If the current loss on the validation set is less than or equal to 0.01, + * the diff of validation error is compared to absolute tolerance which is * validationTol * 0.01. * Ignored when - * [[org.apache.spark.mllib.tree.GradientBoostedTrees.run()]] is used. + * `org.apache.spark.mllib.tree.GradientBoostedTrees.run()` is used. */ @Since("1.2.0") case class BoostingStrategy @Since("1.4.0") ( @@ -59,7 +59,7 @@ case class BoostingStrategy @Since("1.4.0") ( * Check validity of parameters. * Throws exception if invalid. */ - private[tree] def assertValid(): Unit = { + private[spark] def assertValid(): Unit = { treeStrategy.algo match { case Classification => require(treeStrategy.numClasses == 2, @@ -92,8 +92,8 @@ object BoostingStrategy { /** * Returns default configuration for the boosting algorithm * @param algo Learning goal. Supported: - * [[org.apache.spark.mllib.tree.configuration.Algo.Classification]], - * [[org.apache.spark.mllib.tree.configuration.Algo.Regression]] + * `org.apache.spark.mllib.tree.configuration.Algo.Classification`, + * `org.apache.spark.mllib.tree.configuration.Algo.Regression` * @return Configuration for boosting algorithm */ @Since("1.3.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 372d6617a401..58e8f5be7b9f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -21,21 +21,21 @@ import scala.beans.BeanProperty import scala.collection.JavaConverters._ import org.apache.spark.annotation.Since -import org.apache.spark.mllib.tree.impurity.{Variance, Entropy, Gini, Impurity} import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ +import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Impurity, Variance} /** * Stores all the configuration options for tree construction * @param algo Learning goal. Supported: - * [[org.apache.spark.mllib.tree.configuration.Algo.Classification]], - * [[org.apache.spark.mllib.tree.configuration.Algo.Regression]] + * `org.apache.spark.mllib.tree.configuration.Algo.Classification`, + * `org.apache.spark.mllib.tree.configuration.Algo.Regression` * @param impurity Criterion used for information gain calculation. * Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]], * [[org.apache.spark.mllib.tree.impurity.Entropy]]. * Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]]. - * @param maxDepth Maximum depth of the tree. - * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * @param maxDepth Maximum depth of the tree (e.g. depth 0 means 1 leaf node, depth 1 means + * 1 internal node + 2 leaf nodes). * @param numClasses Number of classes for classification. * (Ignored for regression.) * Default value is 2 (binary classification). @@ -43,12 +43,11 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * for choosing how to split on features at each node. * More bins give higher granularity. * @param quantileCalculationStrategy Algorithm for calculating quantiles. Supported: - * [[org.apache.spark.mllib.tree.configuration.QuantileStrategy.Sort]] + * `org.apache.spark.mllib.tree.configuration.QuantileStrategy.Sort` * @param categoricalFeaturesInfo A map storing information about the categorical variables and the - * number of discrete values they take. For example, an entry (n -> - * k) implies the feature n is categorical with k categories 0, - * 1, 2, ... , k-1. It's important to note that features are - * zero-indexed. + * number of discrete values they take. An entry (n to k) + * indicates that feature n is categorical with k categories + * indexed from 0: {0, 1, ..., k-1}. * @param minInstancesPerNode Minimum number of instances each child must have after split. * Default value is 1. If a split cause left or right child * to have less than minInstancesPerNode, @@ -57,10 +56,11 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * If a split has less information gain than minInfoGain, * this split will not be considered as a valid split. * @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. Default value is - * 256 MB. + * 256 MB. If too small, then 1 node will be split per iteration, and + * its aggregates may exceed this size. * @param subsamplingRate Fraction of the training data used for learning decision tree. * @param useNodeIdCache If this is true, instead of passing trees to executors, the algorithm will - * maintain a separate RDD of node Id cache for each row. + * maintain a separate RDD of node Id cache for each row. * @param checkpointInterval How often to checkpoint when the node Id cache gets updated. * E.g. 10 means that the cache will get checkpointed every 10 updates. If * the checkpoint directory is not set in @@ -134,7 +134,7 @@ class Strategy @Since("1.3.0") ( * Check validity of parameters. * Throws exception if invalid. */ - private[tree] def assertValid(): Unit = { + private[spark] def assertValid(): Unit = { algo match { case Classification => require(numClasses >= 2, @@ -202,8 +202,4 @@ object Strategy { numClasses = 0) } - @deprecated("Use Strategy.defaultStrategy instead.", "1.5.0") - @Since("1.2.0") - def defaultStategy(algo: Algo): Strategy = defaultStrategy(algo) - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/NodeIdCache.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/NodeIdCache.scala deleted file mode 100644 index 1c611976a930..000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/NodeIdCache.scala +++ /dev/null @@ -1,196 +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.mllib.tree.impl - -import scala.collection.mutable - -import org.apache.hadoop.fs.{Path, FileSystem} - -import org.apache.spark.rdd.RDD -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.tree.configuration.FeatureType._ -import org.apache.spark.storage.StorageLevel -import org.apache.spark.mllib.tree.model.{Bin, Node, Split} - -/** - * :: DeveloperApi :: - * This is used by the node id cache to find the child id that a data point would belong to. - * @param split Split information. - * @param nodeIndex The current node index of a data point that this will update. - */ -@DeveloperApi -private[tree] case class NodeIndexUpdater( - split: Split, - nodeIndex: Int) { - /** - * Determine a child node index based on the feature value and the split. - * @param binnedFeatures Binned feature values. - * @param bins Bin information to convert the bin indices to approximate feature values. - * @return Child node index to update to. - */ - def updateNodeIndex(binnedFeatures: Array[Int], bins: Array[Array[Bin]]): Int = { - if (split.featureType == Continuous) { - val featureIndex = split.feature - val binIndex = binnedFeatures(featureIndex) - val featureValueUpperBound = bins(featureIndex)(binIndex).highSplit.threshold - if (featureValueUpperBound <= split.threshold) { - Node.leftChildIndex(nodeIndex) - } else { - Node.rightChildIndex(nodeIndex) - } - } else { - if (split.categories.contains(binnedFeatures(split.feature).toDouble)) { - Node.leftChildIndex(nodeIndex) - } else { - Node.rightChildIndex(nodeIndex) - } - } - } -} - -/** - * :: DeveloperApi :: - * A given TreePoint would belong to a particular node per tree. - * Each row in the nodeIdsForInstances RDD is an array over trees of the node index - * in each tree. Initially, values should all be 1 for root node. - * The nodeIdsForInstances RDD needs to be updated at each iteration. - * @param nodeIdsForInstances The initial values in the cache - * (should be an Array of all 1's (meaning the root nodes)). - * @param checkpointInterval The checkpointing interval - * (how often should the cache be checkpointed.). - */ -@DeveloperApi -private[spark] class NodeIdCache( - var nodeIdsForInstances: RDD[Array[Int]], - val checkpointInterval: Int) { - - // Keep a reference to a previous node Ids for instances. - // Because we will keep on re-persisting updated node Ids, - // we want to unpersist the previous RDD. - private var prevNodeIdsForInstances: RDD[Array[Int]] = null - - // To keep track of the past checkpointed RDDs. - private val checkpointQueue = mutable.Queue[RDD[Array[Int]]]() - private var rddUpdateCount = 0 - - /** - * Update the node index values in the cache. - * This updates the RDD and its lineage. - * TODO: Passing bin information to executors seems unnecessary and costly. - * @param data The RDD of training rows. - * @param nodeIdUpdaters A map of node index updaters. - * The key is the indices of nodes that we want to update. - * @param bins Bin information needed to find child node indices. - */ - def updateNodeIndices( - data: RDD[BaggedPoint[TreePoint]], - nodeIdUpdaters: Array[mutable.Map[Int, NodeIndexUpdater]], - bins: Array[Array[Bin]]): Unit = { - if (prevNodeIdsForInstances != null) { - // Unpersist the previous one if one exists. - prevNodeIdsForInstances.unpersist() - } - - prevNodeIdsForInstances = nodeIdsForInstances - nodeIdsForInstances = data.zip(nodeIdsForInstances).map { - case (point, node) => { - var treeId = 0 - while (treeId < nodeIdUpdaters.length) { - val nodeIdUpdater = nodeIdUpdaters(treeId).getOrElse(node(treeId), null) - if (nodeIdUpdater != null) { - val newNodeIndex = nodeIdUpdater.updateNodeIndex( - binnedFeatures = point.datum.binnedFeatures, - bins = bins) - node(treeId) = newNodeIndex - } - - treeId += 1 - } - - node - } - } - - // Keep on persisting new ones. - nodeIdsForInstances.persist(StorageLevel.MEMORY_AND_DISK) - rddUpdateCount += 1 - - // Handle checkpointing if the directory is not None. - if (nodeIdsForInstances.sparkContext.getCheckpointDir.nonEmpty && - (rddUpdateCount % checkpointInterval) == 0) { - // Let's see if we can delete previous checkpoints. - var canDelete = true - while (checkpointQueue.size > 1 && canDelete) { - // We can delete the oldest checkpoint iff - // the next checkpoint actually exists in the file system. - if (checkpointQueue.get(1).get.getCheckpointFile.isDefined) { - val old = checkpointQueue.dequeue() - - // Since the old checkpoint is not deleted by Spark, - // we'll manually delete it here. - val fs = FileSystem.get(old.sparkContext.hadoopConfiguration) - fs.delete(new Path(old.getCheckpointFile.get), true) - } else { - canDelete = false - } - } - - nodeIdsForInstances.checkpoint() - checkpointQueue.enqueue(nodeIdsForInstances) - } - } - - /** - * Call this after training is finished to delete any remaining checkpoints. - */ - def deleteAllCheckpoints(): Unit = { - while (checkpointQueue.nonEmpty) { - val old = checkpointQueue.dequeue() - for (checkpointFile <- old.getCheckpointFile) { - val fs = FileSystem.get(old.sparkContext.hadoopConfiguration) - fs.delete(new Path(checkpointFile), true) - } - } - if (prevNodeIdsForInstances != null) { - // Unpersist the previous one if one exists. - prevNodeIdsForInstances.unpersist() - } - } -} - -@DeveloperApi -private[spark] object NodeIdCache { - /** - * Initialize the node Id cache with initial node Id values. - * @param data The RDD of training rows. - * @param numTrees The number of trees that we want to create cache for. - * @param checkpointInterval The checkpointing interval - * (how often should the cache be checkpointed.). - * @param initVal The initial values in the cache. - * @return A node Id cache containing an RDD of initial root node Indices. - */ - def init( - data: RDD[BaggedPoint[TreePoint]], - numTrees: Int, - checkpointInterval: Int, - initVal: Int = 1): NodeIdCache = { - new NodeIdCache( - data.map(_ => Array.fill[Int](numTrees)(initVal)), - checkpointInterval) - } -} 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 deleted file mode 100644 index 21919d69a38a..000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala +++ /dev/null @@ -1,150 +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.mllib.tree.impl - -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.model.Bin -import org.apache.spark.rdd.RDD - - -/** - * Internal representation of LabeledPoint for DecisionTree. - * This bins feature values based on a subsampled of data as follows: - * (a) Continuous features are binned into ranges. - * (b) Unordered categorical features are binned based on subsets of feature values. - * "Unordered categorical features" are categorical features with low arity used in - * multiclass classification. - * (c) Ordered categorical features are binned based on feature values. - * "Ordered categorical features" are categorical features with high arity, - * or any categorical feature used in regression or binary classification. - * - * @param label Label from LabeledPoint - * @param binnedFeatures Binned feature values. - * Same length as LabeledPoint.features, but values are bin indices. - */ -private[spark] class TreePoint(val label: Double, val binnedFeatures: Array[Int]) - extends Serializable { -} - -private[spark] object TreePoint { - - /** - * Convert an input dataset into its TreePoint representation, - * binning feature values in preparation for DecisionTree training. - * @param input Input dataset. - * @param bins Bins for features, of size (numFeatures, numBins). - * @param metadata Learning and dataset metadata - * @return TreePoint dataset representation - */ - def convertToTreeRDD( - input: RDD[LabeledPoint], - bins: Array[Array[Bin]], - metadata: DecisionTreeMetadata): RDD[TreePoint] = { - // Construct arrays for featureArity for efficiency in the inner loop. - val featureArity: Array[Int] = new Array[Int](metadata.numFeatures) - var featureIndex = 0 - while (featureIndex < metadata.numFeatures) { - featureArity(featureIndex) = metadata.featureArity.getOrElse(featureIndex, 0) - featureIndex += 1 - } - input.map { x => - TreePoint.labeledPointToTreePoint(x, bins, featureArity) - } - } - - /** - * Convert one LabeledPoint into its TreePoint representation. - * @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. - */ - private def labeledPointToTreePoint( - labeledPoint: LabeledPoint, - bins: Array[Array[Bin]], - 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), - bins) - featureIndex += 1 - } - new TreePoint(labeledPoint.label, arr) - } - - /** - * Find bin for one (labeledPoint, feature). - * - * @param featureArity 0 for continuous features; number of categories for categorical features. - * @param bins Bins for features, of size (numFeatures, numBins). - */ - private def findBin( - featureIndex: Int, - labeledPoint: LabeledPoint, - featureArity: Int, - bins: Array[Array[Bin]]): Int = { - - /** - * Binary search helper method for continuous feature. - */ - def binarySearchForBins(): Int = { - val binForFeatures = bins(featureIndex) - val feature = labeledPoint.features(featureIndex) - var left = 0 - var right = binForFeatures.length - 1 - while (left <= right) { - val mid = left + (right - left) / 2 - val bin = binForFeatures(mid) - val lowThreshold = bin.lowSplit.threshold - val highThreshold = bin.highSplit.threshold - if ((lowThreshold < feature) && (highThreshold >= feature)) { - return mid - } else if (lowThreshold >= feature) { - right = mid - 1 - } else { - left = mid + 1 - } - } - -1 - } - - if (featureArity == 0) { - // Perform binary search for finding bin for continuous features. - val binIndex = binarySearchForBins() - if (binIndex == -1) { - throw new RuntimeException("No bin was found for continuous feature." + - " This error can occur when given invalid data values (such as NaN)." + - s" Feature index: $featureIndex. Feature value: ${labeledPoint.features(featureIndex)}") - } - binIndex - } else { - // Categorical feature bins are indexed by feature values. - val featureValue = labeledPoint.features(featureIndex) - if (featureValue < 0 || featureValue >= featureArity) { - throw new IllegalArgumentException( - s"DecisionTree given invalid data:" + - s" Feature $featureIndex is categorical with values in" + - s" {0,...,${featureArity - 1}," + - s" but a data point gives it value $featureValue.\n" + - " Bad data point: " + labeledPoint.toString) - } - featureValue.toInt - } - } -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 73df6b054a8c..d4448da9eef5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -17,15 +17,12 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.annotation.{DeveloperApi, Since} /** - * :: Experimental :: - * Class for calculating [[http://en.wikipedia.org/wiki/Binary_entropy_function entropy]] during - * binary classification. + * Class for calculating entropy during multiclass classification. */ @Since("1.0.0") -@Experimental object Entropy extends Impurity { private[tree] def log2(x: Double) = scala.math.log(x) / scala.math.log(2) @@ -85,7 +82,7 @@ object Entropy extends Impurity { * Note: Instances of this class do not hold the data; they operate on views of the data. * @param numClasses Number of classes for label. */ -private[tree] class EntropyAggregator(numClasses: Int) +private[spark] class EntropyAggregator(numClasses: Int) extends ImpurityAggregator(numClasses) with Serializable { /** @@ -113,7 +110,6 @@ private[tree] class EntropyAggregator(numClasses: Int) def getCalculator(allStats: Array[Double], offset: Int): EntropyCalculator = { new EntropyCalculator(allStats.view(offset, offset + statsSize).toArray) } - } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index f21845b21a80..c5e34ffa4f2e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -17,16 +17,14 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.annotation.{DeveloperApi, Since} /** - * :: Experimental :: - * Class for calculating the - * [[http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity Gini impurity]] - * during binary classification. + * Class for calculating the Gini impurity + * (http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity) + * during multiclass classification. */ @Since("1.0.0") -@Experimental object Gini extends Impurity { /** @@ -81,7 +79,7 @@ object Gini extends Impurity { * Note: Instances of this class do not hold the data; they operate on views of the data. * @param numClasses Number of classes for label. */ -private[tree] class GiniAggregator(numClasses: Int) +private[spark] class GiniAggregator(numClasses: Int) extends ImpurityAggregator(numClasses) with Serializable { /** @@ -109,7 +107,6 @@ private[tree] class GiniAggregator(numClasses: Int) def getCalculator(allStats: Array[Double], offset: Int): GiniCalculator = { new GiniCalculator(allStats.view(offset, offset + statsSize).toArray) } - } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index 4637dcceea7f..f151a6a01b65 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -17,17 +17,17 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import java.util.Locale + +import org.apache.spark.annotation.{DeveloperApi, Since} /** - * :: Experimental :: * Trait for calculating information gain. * This trait is used for * (a) setting the impurity parameter in [[org.apache.spark.mllib.tree.configuration.Strategy]] * (b) calculating impurity values from sufficient statistics. */ @Since("1.0.0") -@Experimental trait Impurity extends Serializable { /** @@ -89,7 +89,6 @@ private[spark] abstract class ImpurityAggregator(val statsSize: Int) extends Ser * @param offset Start index of stats for this (node, feature, bin). */ def getCalculator(allStats: Array[Double], offset: Int): ImpurityCalculator - } /** @@ -163,7 +162,7 @@ private[spark] abstract class ImpurityCalculator(val stats: Array[Double]) exten * Fails if the array is empty. */ protected def indexOfLargestArrayElement(array: Array[Double]): Int = { - val result = array.foldLeft(-1, Double.MinValue, 0) { + val result = array.foldLeft((-1, Double.MinValue, 0)) { case ((maxIndex, maxValue, currentIndex), currentValue) => if (currentValue > maxValue) { (currentIndex, currentValue, currentIndex + 1) @@ -179,3 +178,21 @@ private[spark] abstract class ImpurityCalculator(val stats: Array[Double]) exten } } + +private[spark] object ImpurityCalculator { + + /** + * Create an [[ImpurityCalculator]] instance of the given impurity type and with + * the given stats. + */ + def getCalculator(impurity: String, stats: Array[Double]): ImpurityCalculator = { + impurity.toLowerCase(Locale.ROOT) match { + case "gini" => new GiniCalculator(stats) + case "entropy" => new EntropyCalculator(stats) + case "variance" => new VarianceCalculator(stats) + case _ => + throw new IllegalArgumentException( + s"ImpurityCalculator builder did not recognize impurity type: $impurity") + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index a74197278d6f..c9bf0db4de3c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -17,14 +17,12 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} +import org.apache.spark.annotation.{DeveloperApi, Since} /** - * :: Experimental :: * Class for calculating variance during regression */ @Since("1.0.0") -@Experimental object Variance extends Impurity { /** @@ -71,7 +69,7 @@ object Variance extends Impurity { * in order to compute impurity from a sample. * Note: Instances of this class do not hold the data; they operate on views of the data. */ -private[tree] class VarianceAggregator() +private[spark] class VarianceAggregator() extends ImpurityAggregator(statsSize = 3) with Serializable { /** @@ -93,7 +91,6 @@ private[tree] class VarianceAggregator() def getCalculator(allStats: Array[Double], offset: Int): VarianceCalculator = { new VarianceCalculator(allStats.view(offset, offset + statsSize).toArray) } - } /** @@ -104,9 +101,9 @@ private[tree] class VarianceAggregator() */ private[spark] class VarianceCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) { - require(stats.size == 3, + require(stats.length == 3, s"VarianceCalculator requires sufficient statistics array stats to be of length 3," + - s" but was given array of length ${stats.size}.") + s" but was given array of length ${stats.length}.") /** * Make a deep copy of this [[ImpurityCalculator]]. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala index bab7b8c6cadf..9b60d018d0ed 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala @@ -18,8 +18,6 @@ package org.apache.spark.mllib.tree.loss import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.model.TreeEnsembleModel /** @@ -47,7 +45,7 @@ object AbsoluteError extends Loss { if (label - prediction < 0) 1.0 else -1.0 } - override private[mllib] def computeError(prediction: Double, label: Double): Double = { + override private[spark] def computeError(prediction: Double, label: Double): Double = { val err = label - prediction math.abs(err) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala index b2b4594712f0..9339f0a23c1b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala @@ -18,11 +18,8 @@ package org.apache.spark.mllib.tree.loss import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.model.TreeEnsembleModel import org.apache.spark.mllib.util.MLUtils - /** * :: DeveloperApi :: * Class for log loss calculation (for classification). @@ -34,7 +31,7 @@ import org.apache.spark.mllib.util.MLUtils */ @Since("1.2.0") @DeveloperApi -object LogLoss extends Loss { +object LogLoss extends ClassificationLoss { /** * Method to calculate the loss gradients for the gradient boosting calculation for binary @@ -49,9 +46,16 @@ object LogLoss extends Loss { - 4.0 * label / (1.0 + math.exp(2.0 * label * prediction)) } - override private[mllib] def computeError(prediction: Double, label: Double): Double = { + override private[spark] def computeError(prediction: Double, label: Double): Double = { val margin = 2.0 * label * prediction // The following is equivalent to 2.0 * log(1 + exp(-margin)) but more numerically stable. 2.0 * MLUtils.log1pExp(-margin) } + + /** + * Returns the estimated probability of a label of 1.0. + */ + override private[spark] def computeProbability(margin: Double): Double = { + 1.0 / (1.0 + math.exp(-2.0 * margin)) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala index 687cde325ffe..e7ffb3f8f53c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala @@ -22,7 +22,6 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel import org.apache.spark.rdd.RDD - /** * :: DeveloperApi :: * Trait for adding "pluggable" loss functions for the gradient boosting algorithm. @@ -42,11 +41,13 @@ trait Loss extends Serializable { /** * Method to calculate error of the base learner for the gradient boosting calculation. - * Note: This method is not used by the gradient boosting algorithm but is useful for debugging - * purposes. + * * @param model Model of the weak learner. * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * @return Measure of model error on data + * + * @note This method is not used by the gradient boosting algorithm but is useful for debugging + * purposes. */ @Since("1.2.0") def computeError(model: TreeEnsembleModel, data: RDD[LabeledPoint]): Double = { @@ -55,11 +56,20 @@ trait Loss extends Serializable { /** * Method to calculate loss when the predictions are already known. - * Note: This method is used in the method evaluateEachIteration to avoid recomputing the - * predicted values from previously fit trees. + * * @param prediction Predicted label. * @param label True label. * @return Measure of model error on datapoint. + * + * @note This method is used in the method evaluateEachIteration to avoid recomputing the + * predicted values from previously fit trees. + */ + private[spark] def computeError(prediction: Double, label: Double): Double +} + +private[spark] trait ClassificationLoss extends Loss { + /** + * Computes the class probability given the margin. */ - private[mllib] def computeError(prediction: Double, label: Double): Double + private[spark] def computeProbability(margin: Double): Double } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala index 3f7d3d38be16..4eb6810c46b2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala @@ -18,8 +18,6 @@ package org.apache.spark.mllib.tree.loss import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.model.TreeEnsembleModel /** @@ -47,7 +45,7 @@ object SquaredError extends Loss { - 2.0 * (label - prediction) } - override private[mllib] def computeError(prediction: Double, label: Double): Double = { + override private[spark] def computeError(prediction: Double, label: Double): Double = { val err = label - prediction err * err } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala deleted file mode 100644 index 0cad473782af..000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala +++ /dev/null @@ -1,47 +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.mllib.tree.model - -import org.apache.spark.mllib.tree.configuration.FeatureType._ - -/** - * Used for "binning" the feature values for faster best split calculation. - * - * For a continuous feature, the bin is determined by a low and a high split, - * where an example with featureValue falls into the bin s.t. - * lowSplit.threshold < featureValue <= highSplit.threshold. - * - * For ordered categorical features, there is a 1-1-1 correspondence between - * bins, splits, and feature values. The bin is determined by category/feature value. - * However, the bins are not necessarily ordered by feature value; - * they are ordered using impurity. - * - * For unordered categorical features, there is a 1-1 correspondence between bins, splits, - * where bins and splits correspond to subsets of feature values (in highSplit.categories). - * An unordered feature with k categories uses (1 << k - 1) - 1 bins, corresponding to all - * partitionings of categories into 2 disjoint, non-empty sets. - * - * @param lowSplit signifying the lower threshold for the continuous feature to be - * accepted in the bin - * @param highSplit signifying the upper threshold for the continuous feature to be - * accepted in the bin - * @param featureType type of feature -- categorical or continuous - * @param category categorical label value accepted in the bin for ordered features - */ -private[tree] -case class Bin(lowSplit: Split, highSplit: Split, featureType: FeatureType, category: Double) 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 54c136aecf66..27618e122aef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -23,15 +23,16 @@ import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.tree.configuration.{Algo, FeatureType} import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.util.Utils /** @@ -74,8 +75,8 @@ class DecisionTreeModel @Since("1.0.0") ( * @return JavaRDD of predictions for each of the given data points */ @Since("1.2.0") - def predict(features: JavaRDD[Vector]): JavaRDD[Double] = { - predict(features.rdd) + def predict(features: JavaRDD[Vector]): JavaRDD[java.lang.Double] = { + predict(features.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Double]] } /** @@ -155,7 +156,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { feature: Int, threshold: Double, featureType: Int, - categories: Seq[Double]) { // TODO: Change to List once SPARK-3365 is fixed + categories: Seq[Double]) { def toSplit: Split = { new Split(feature, threshold, FeatureType(featureType), categories.toList) } @@ -201,9 +202,6 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { } def save(sc: SparkContext, path: String, model: DecisionTreeModel): Unit = { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ - // SPARK-6120: We do a hacky check here so users understand why save() is failing // when they run the ML guide example. // TODO: Fix this issue for real. @@ -234,26 +232,25 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { // Create Parquet data. val nodes = model.topNode.subtreeIterator.toSeq - val dataRDD: DataFrame = sc.parallelize(nodes) - .map(NodeData.apply(0, _)) - .toDF() - dataRDD.write.parquet(Loader.dataPath(path)) + val dataRDD = sc.parallelize(nodes).map(NodeData.apply(0, _)) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + spark.createDataFrame(dataRDD).write.parquet(Loader.dataPath(path)) } def load(sc: SparkContext, path: String, algo: String, numNodes: Int): DecisionTreeModel = { - val datapath = Loader.dataPath(path) - val sqlContext = new SQLContext(sc) // Load Parquet data. - val dataRDD = sqlContext.read.parquet(datapath) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val dataPath = Loader.dataPath(path) + val dataRDD = spark.read.parquet(dataPath) // Check schema explicitly since erasure makes it hard to use match-case for checking. Loader.checkSchema[NodeData](dataRDD.schema) - val nodes = dataRDD.map(NodeData.apply) + val nodes = dataRDD.rdd.map(NodeData.apply) // Build node data into a tree. val trees = constructTrees(nodes) - assert(trees.size == 1, - "Decision tree should contain exactly one tree but got ${trees.size} trees.") + assert(trees.length == 1, + s"Decision tree should contain exactly one tree but got ${trees.size} trees.") val model = new DecisionTreeModel(trees(0), Algo.fromString(algo)) - assert(model.numNodes == numNodes, s"Unable to load DecisionTreeModel data from: $datapath." + + assert(model.numNodes == numNodes, s"Unable to load DecisionTreeModel data from: $dataPath." + s" Expected $numNodes nodes but found ${model.numNodes}") model } @@ -266,7 +263,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { .map { case (treeId, data) => (treeId, constructTree(data)) }.sortBy(_._1) - val numTrees = trees.size + val numTrees = trees.length val treeIndices = trees.map(_._1).toSeq assert(treeIndices == (0 until numTrees), s"Tree indices must start from 0 and increment by 1, but we found $treeIndices.") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index 091a0462c204..f3dbfd96e181 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -68,18 +68,7 @@ class InformationGainStats( } } -private[spark] object InformationGainStats { - /** - * An [[org.apache.spark.mllib.tree.model.InformationGainStats]] object to - * denote that current split doesn't satisfies minimum info gain or - * minimum number of instances per node. - */ - val invalidInformationGainStats = new InformationGainStats(Double.MinValue, -1.0, -1.0, -1.0, - new Predict(0.0, 0.0), new Predict(0.0, 0.0)) -} - /** - * :: DeveloperApi :: * Impurity statistics for each split * @param gain information gain value * @param impurity current node impurity @@ -89,7 +78,6 @@ private[spark] object InformationGainStats { * @param valid whether the current split satisfies minimum info gain or * minimum number of instances per node */ -@DeveloperApi private[spark] class ImpurityStats( val gain: Double, val impurity: Double, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index ea6e5aa5d94e..5fd053647aa4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -18,9 +18,9 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.Logging -import org.apache.spark.mllib.tree.configuration.FeatureType._ +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.tree.configuration.FeatureType._ /** * :: DeveloperApi :: @@ -56,34 +56,13 @@ class Node @Since("1.2.0") ( s"split = $split, stats = $stats" } - /** - * build the left node and right nodes if not leaf - * @param nodes array of nodes - */ - @Since("1.0.0") - @deprecated("build should no longer be used since trees are constructed on-the-fly in training", - "1.2.0") - def build(nodes: Array[Node]): Unit = { - logDebug("building node " + id + " at level " + Node.indexToLevel(id)) - logDebug("id = " + id + ", split = " + split) - logDebug("stats = " + stats) - logDebug("predict = " + predict) - logDebug("impurity = " + impurity) - if (!isLeaf) { - leftNode = Some(nodes(Node.leftChildIndex(id))) - rightNode = Some(nodes(Node.rightChildIndex(id))) - leftNode.get.build(nodes) - rightNode.get.build(nodes) - } - } - /** * predict value if node is not leaf * @param features feature value * @return predicted value */ @Since("1.1.0") - def predict(features: Vector) : Double = { + def predict(features: Vector): Double = { if (isLeaf) { predict.predict } else { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala index 06ceff19d863..1dbdd2d860ef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.annotation.{DeveloperApi, Since} /** + * :: DeveloperApi :: * Predicted value for a node * @param predict predicted value * @param prob probability of the label (classification only) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index b85a66c05a81..bda5e662779c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -19,15 +19,13 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType -import org.apache.spark.mllib.tree.configuration.FeatureType -import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType /** * :: DeveloperApi :: * Split applied to a feature * @param feature feature index * @param threshold Threshold for continuous feature. - * Split left if feature <= threshold, else right. + * Split left if feature is less than or equal to threshold, else right. * @param featureType type of feature -- categorical or continuous * @param categories Split left if categorical feature value is in this set, else right. */ 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 90e032e3d984..b1e82656a240 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala @@ -24,9 +24,10 @@ import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.annotation.Since +import org.apache.spark.SparkContext +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.java.JavaRDD +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo @@ -35,7 +36,7 @@ import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._ import org.apache.spark.mllib.tree.loss.Loss import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.util.Utils /** @@ -150,31 +151,24 @@ class GradientBoostedTreesModel @Since("1.2.0") ( case _ => data } - val numIterations = trees.length - val evaluationArray = Array.fill(numIterations)(0.0) - val localTreeWeights = treeWeights - - var predictionAndError = GradientBoostedTreesModel.computeInitialPredictionAndError( - remappedData, localTreeWeights(0), trees(0), loss) - - evaluationArray(0) = predictionAndError.values.mean() - val broadcastTrees = sc.broadcast(trees) - (1 until numIterations).foreach { nTree => - predictionAndError = remappedData.zip(predictionAndError).mapPartitions { iter => - val currentTree = broadcastTrees.value(nTree) - val currentTreeWeight = localTreeWeights(nTree) - iter.map { case (point, (pred, error)) => - val newPred = pred + currentTree.predict(point.features) * currentTreeWeight - val newError = loss.computeError(newPred, point.label) - (newPred, newError) - } - } - evaluationArray(nTree) = predictionAndError.values.mean() + val localTreeWeights = treeWeights + val treesIndices = trees.indices + + val dataCount = remappedData.count() + val evaluation = remappedData.map { point => + treesIndices + .map(idx => broadcastTrees.value(idx).predict(point.features) * localTreeWeights(idx)) + .scanLeft(0.0)(_ + _).drop(1) + .map(prediction => loss.computeError(prediction, point.label)) } + .aggregate(treesIndices.map(_ => 0.0))( + (aggregated, row) => treesIndices.map(idx => aggregated(idx) + row(idx)), + (a, b) => treesIndices.map(idx => a(idx) + b(idx))) + .map(_ / dataCount) - broadcastTrees.unpersist() - evaluationArray + broadcastTrees.destroy(blocking = false) + evaluation.toArray } override protected def formatVersion: String = GradientBoostedTreesModel.formatVersion @@ -186,16 +180,18 @@ class GradientBoostedTreesModel @Since("1.2.0") ( object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { /** + * :: DeveloperApi :: * Compute the initial predictions and errors for a dataset for the first * iteration of gradient boosting. * @param data: training data. * @param initTreeWeight: learning rate assigned to the first tree. * @param initTree: first DecisionTreeModel. * @param loss: evaluation metric. - * @return a RDD with each element being a zip of the prediction and error + * @return an RDD with each element being a zip of the prediction and error * corresponding to every sample. */ @Since("1.4.0") + @DeveloperApi def computeInitialPredictionAndError( data: RDD[LabeledPoint], initTreeWeight: Double, @@ -209,6 +205,7 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { } /** + * :: DeveloperApi :: * Update a zipped predictionError RDD * (as obtained with computeInitialPredictionAndError) * @param data: training data. @@ -216,10 +213,11 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { * @param treeWeight: Learning rate. * @param tree: Tree using which the prediction and error should be updated. * @param loss: evaluation metric. - * @return a RDD with each element being a zip of the prediction and error + * @return an RDD with each element being a zip of the prediction and error * corresponding to each sample. */ @Since("1.4.0") + @DeveloperApi def updatePredictionError( data: RDD[LabeledPoint], predictionAndError: RDD[(Double, Double)], @@ -343,7 +341,7 @@ private[tree] sealed class TreeEnsembleModel( def predict(features: RDD[Vector]): RDD[Double] = features.map(x => predict(x)) /** - * Java-friendly version of [[org.apache.spark.mllib.tree.model.TreeEnsembleModel#predict]]. + * Java-friendly version of `org.apache.spark.mllib.tree.model.TreeEnsembleModel.predict`. */ def predict(features: JavaRDD[Vector]): JavaRDD[java.lang.Double] = { predict(features.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Double]] @@ -408,8 +406,7 @@ private[tree] object TreeEnsembleModel extends Logging { case class EnsembleNodeData(treeId: Int, node: NodeData) def save(sc: SparkContext, path: String, model: TreeEnsembleModel, className: String): Unit = { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() // SPARK-6120: We do a hacky check here so users understand why save() is failing // when they run the ML guide example. @@ -445,8 +442,8 @@ private[tree] object TreeEnsembleModel extends Logging { // Create Parquet data. val dataRDD = sc.parallelize(model.trees.zipWithIndex).flatMap { case (tree, treeId) => tree.topNode.subtreeIterator.toSeq.map(node => NodeData(treeId, node)) - }.toDF() - dataRDD.write.parquet(Loader.dataPath(path)) + } + spark.createDataFrame(dataRDD).write.parquet(Loader.dataPath(path)) } /** @@ -467,10 +464,10 @@ private[tree] object TreeEnsembleModel extends Logging { sc: SparkContext, path: String, treeAlgo: String): Array[DecisionTreeModel] = { - val datapath = Loader.dataPath(path) - val sqlContext = new SQLContext(sc) - val nodes = sqlContext.read.parquet(datapath).map(NodeData.apply) - val trees = constructTrees(nodes) + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + import spark.implicits._ + val nodes = spark.read.parquet(Loader.dataPath(path)).map(NodeData.apply) + val trees = constructTrees(nodes.rdd) trees.map(new DecisionTreeModel(_, Algo.fromString(treeAlgo))) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index dffe6e78939e..2c712d8f821a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -17,8 +17,8 @@ package org.apache.spark.mllib.util -import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala index 00fd1606a369..7f84be9f3782 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala @@ -86,6 +86,7 @@ object KMeansDataGenerator { val data = generateKMeansRDD(sc, numPoints, k, d, r, parts) data.map(_.mkString(" ")).saveAsTextFile(outputPath) + sc.stop() System.exit(0) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 6ff07eed6cfd..58fd010e4905 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -24,7 +24,7 @@ import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.SparkContext import org.apache.spark.annotation.{DeveloperApi, Since} -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.{BLAS, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD @@ -131,42 +131,34 @@ object LinearDataGenerator { eps: Double, sparsity: Double): Seq[LabeledPoint] = { require(0.0 <= sparsity && sparsity <= 1.0) - val rnd = new Random(seed) - val x = Array.fill[Array[Double]](nPoints)( - Array.fill[Double](weights.length)(rnd.nextDouble())) - - val sparseRnd = new Random(seed) - x.foreach { v => - var i = 0 - val len = v.length - while (i < len) { - if (sparseRnd.nextDouble() < sparsity) { - v(i) = 0.0 - } else { - v(i) = (v(i) - 0.5) * math.sqrt(12.0 * xVariance(i)) + xMean(i) - } - i += 1 - } - } - val y = x.map { xi => - blas.ddot(weights.length, xi, 1, weights, 1) + intercept + eps * rnd.nextGaussian() - } + val rnd = new Random(seed) + def rndElement(i: Int) = {(rnd.nextDouble() - 0.5) * math.sqrt(12.0 * xVariance(i)) + xMean(i)} - y.zip(x).map { p => - if (sparsity == 0.0) { + if (sparsity == 0.0) { + (0 until nPoints).map { _ => + val features = Vectors.dense(weights.indices.map { rndElement(_) }.toArray) + val label = BLAS.dot(Vectors.dense(weights), features) + + intercept + eps * rnd.nextGaussian() // Return LabeledPoints with DenseVector - LabeledPoint(p._1, Vectors.dense(p._2)) - } else { + LabeledPoint(label, features) + } + } else { + (0 until nPoints).map { _ => + val indices = weights.indices.filter { _ => rnd.nextDouble() <= sparsity} + val values = indices.map { rndElement(_) } + val features = Vectors.sparse(weights.length, indices.toArray, values.toArray) + val label = BLAS.dot(Vectors.dense(weights), features) + + intercept + eps * rnd.nextGaussian() // Return LabeledPoints with SparseVector - LabeledPoint(p._1, Vectors.dense(p._2).toSparse) + LabeledPoint(label, features) } } } /** * Generate an RDD containing sample data for Linear Regression models - including Ridge, Lasso, - * and uregularized variants. + * and unregularized variants. * * @param sc SparkContext to be used for generating the RDD. * @param nexamples Number of examples that will be contained in the RDD. @@ -183,7 +175,7 @@ object LinearDataGenerator { nfeatures: Int, eps: Double, nparts: Int = 2, - intercept: Double = 0.0) : RDD[LabeledPoint] = { + intercept: Double = 0.0): RDD[LabeledPoint] = { val random = new Random(42) // Random values distributed uniformly in [-0.5, 0.5] val w = Array.fill(nfeatures)(random.nextDouble() - 0.5) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index 33477ee20ebb..68835bc79677 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -19,11 +19,11 @@ package org.apache.spark.mllib.util import scala.util.Random -import org.apache.spark.annotation.{Since, DeveloperApi} import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 906bd30563bd..42c5bcdd39f7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -19,11 +19,10 @@ package org.apache.spark.mllib.util import java.{util => ju} -import scala.language.postfixOps import scala.util.Random import org.apache.spark.SparkContext -import org.apache.spark.annotation.{Since, DeveloperApi} +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix} import org.apache.spark.rdd.RDD @@ -105,8 +104,7 @@ object MFDataGenerator { // optionally generate testing data if (test) { - val testSampSize = math.min( - math.round(sampSize * testSampFact), math.round(mn - sampSize)).toInt + val testSampSize = math.min(math.round(sampSize * testSampFact).toInt, mn - sampSize) val testOmega = shuffled.slice(sampSize, sampSize + testSampSize) val testOrdered = testOmega.sortWith(_ < _).toArray val testData: RDD[(Int, Int, Double)] = sc.parallelize(testOrdered) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 414ea99cfd8c..14af8b5c7387 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -17,23 +17,29 @@ package org.apache.spark.mllib.util +import scala.annotation.varargs import scala.reflect.ClassTag -import org.apache.spark.annotation.Since import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.PartitionwiseSampledRDD -import org.apache.spark.util.random.BernoulliCellSampler -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vector, Vectors} +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.{MatrixUDT => MLMatrixUDT, VectorUDT => MLVectorUDT} +import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.linalg.BLAS.dot +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.rdd.{PartitionwiseSampledRDD, RDD} +import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.datasources.text.TextFileFormat +import org.apache.spark.sql.functions._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.random.BernoulliCellSampler /** - * Helper methods to load, save and pre-process data used in ML Lib. + * Helper methods to load, save and pre-process data used in MLLib. */ @Since("0.8.0") -object MLUtils { +object MLUtils extends Logging { private[mllib] lazy val EPSILON = { var eps = 1.0 @@ -51,7 +57,6 @@ object MLUtils { * where the indices are one-based and in ascending order. * This method parses each line into a [[org.apache.spark.mllib.regression.LabeledPoint]], * where the feature indices are converted to zero-based. - * * @param sc Spark context * @param path file or directory path in any Hadoop-supported file system URI * @param numFeatures number of features, which will be determined from the input data if a @@ -68,41 +73,14 @@ object MLUtils { path: String, numFeatures: Int, minPartitions: Int): RDD[LabeledPoint] = { - val parsed = sc.textFile(path, minPartitions) - .map(_.trim) - .filter(line => !(line.isEmpty || line.startsWith("#"))) - .map { line => - val items = line.split(' ') - val label = items.head.toDouble - val (indices, values) = items.tail.filter(_.nonEmpty).map { item => - val indexAndValue = item.split(':') - val index = indexAndValue(0).toInt - 1 // Convert 1-based indices to 0-based. - val value = indexAndValue(1).toDouble - (index, value) - }.unzip - - // check if indices are one-based and in ascending order - var previous = -1 - var i = 0 - val indicesLength = indices.length - while (i < indicesLength) { - val current = indices(i) - require(current > previous, "indices should be one-based and in ascending order" ) - previous = current - i += 1 - } - - (label, indices.toArray, values.toArray) - } + val parsed = parseLibSVMFile(sc, path, minPartitions) // Determine number of features. val d = if (numFeatures > 0) { numFeatures } else { parsed.persist(StorageLevel.MEMORY_ONLY) - parsed.map { case (label, indices, values) => - indices.lastOption.getOrElse(0) - }.reduce(math.max) + 1 + computeNumFeatures(parsed) } parsed.map { case (label, indices, values) => @@ -110,17 +88,64 @@ object MLUtils { } } - // Convenient methods for `loadLibSVMFile`. + private[spark] def computeNumFeatures(rdd: RDD[(Double, Array[Int], Array[Double])]): Int = { + rdd.map { case (label, indices, values) => + indices.lastOption.getOrElse(0) + }.reduce(math.max) + 1 + } - @Since("1.0.0") - @deprecated("use method without multiclass argument, which no longer has effect", "1.1.0") - def loadLibSVMFile( + private[spark] def parseLibSVMFile( sc: SparkContext, path: String, - multiclass: Boolean, - numFeatures: Int, - minPartitions: Int): RDD[LabeledPoint] = - loadLibSVMFile(sc, path, numFeatures, minPartitions) + minPartitions: Int): RDD[(Double, Array[Int], Array[Double])] = { + sc.textFile(path, minPartitions) + .map(_.trim) + .filter(line => !(line.isEmpty || line.startsWith("#"))) + .map(parseLibSVMRecord) + } + + private[spark] def parseLibSVMFile( + sparkSession: SparkSession, paths: Seq[String]): RDD[(Double, Array[Int], Array[Double])] = { + val lines = sparkSession.baseRelationToDataFrame( + DataSource.apply( + sparkSession, + paths = paths, + className = classOf[TextFileFormat].getName + ).resolveRelation(checkFilesExist = false)) + .select("value") + + import lines.sqlContext.implicits._ + + lines.select(trim($"value").as("line")) + .filter(not((length($"line") === 0).or($"line".startsWith("#")))) + .as[String] + .rdd + .map(MLUtils.parseLibSVMRecord) + } + + private[spark] def parseLibSVMRecord(line: String): (Double, Array[Int], Array[Double]) = { + val items = line.split(' ') + val label = items.head.toDouble + val (indices, values) = items.tail.filter(_.nonEmpty).map { item => + val indexAndValue = item.split(':') + val index = indexAndValue(0).toInt - 1 // Convert 1-based indices to 0-based. + val value = indexAndValue(1).toDouble + (index, value) + }.unzip + + // check if indices are one-based and in ascending order + var previous = -1 + var i = 0 + val indicesLength = indices.length + while (i < indicesLength) { + val current = indices(i) + require(current > previous, s"indices should be one-based and in ascending order;" + + s""" found current=$current, previous=$previous; line="$line"""") + previous = current + i += 1 + } + (label, indices.toArray, values.toArray) + } /** * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], with the default number of @@ -133,23 +158,6 @@ object MLUtils { numFeatures: Int): RDD[LabeledPoint] = loadLibSVMFile(sc, path, numFeatures, sc.defaultMinPartitions) - @Since("1.0.0") - @deprecated("use method without multiclass argument, which no longer has effect", "1.1.0") - def loadLibSVMFile( - sc: SparkContext, - path: String, - multiclass: Boolean, - numFeatures: Int): RDD[LabeledPoint] = - loadLibSVMFile(sc, path, numFeatures) - - @Since("1.0.0") - @deprecated("use method without multiclass argument, which no longer has effect", "1.1.0") - def loadLibSVMFile( - sc: SparkContext, - path: String, - multiclass: Boolean): RDD[LabeledPoint] = - loadLibSVMFile(sc, path) - /** * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], with number of * features determined automatically and the default number of partitions. @@ -162,8 +170,7 @@ object MLUtils { * Save labeled data in LIBSVM format. * @param data an RDD of LabeledPoint to be saved * @param dir directory to save the data - * - * @see [[org.apache.spark.mllib.util.MLUtils#loadLibSVMFile]] + * @see `org.apache.spark.mllib.util.MLUtils.loadLibSVMFile` */ @Since("1.0.0") def saveAsLibSVMFile(data: RDD[LabeledPoint], dir: String) { @@ -216,48 +223,6 @@ object MLUtils { def loadLabeledPoints(sc: SparkContext, dir: String): RDD[LabeledPoint] = loadLabeledPoints(sc, dir, sc.defaultMinPartitions) - /** - * Load labeled data from a file. The data format used here is - * L, f1 f2 ... - * where f1, f2 are feature values in Double and L is the corresponding label as Double. - * - * @param sc SparkContext - * @param dir Directory to the input data files. - * @return An RDD of LabeledPoint. Each labeled point has two elements: the first element is - * the label, and the second element represents the feature values (an array of Double). - * - * @deprecated Should use [[org.apache.spark.rdd.RDD#saveAsTextFile]] for saving and - * [[org.apache.spark.mllib.util.MLUtils#loadLabeledPoints]] for loading. - */ - @Since("1.0.0") - @deprecated("Should use MLUtils.loadLabeledPoints instead.", "1.0.1") - def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { - sc.textFile(dir).map { line => - val parts = line.split(',') - val label = parts(0).toDouble - val features = Vectors.dense(parts(1).trim().split(' ').map(_.toDouble)) - LabeledPoint(label, features) - } - } - - /** - * Save labeled data to a file. The data format used here is - * L, f1 f2 ... - * where f1, f2 are feature values in Double and L is the corresponding label as Double. - * - * @param data An RDD of LabeledPoints containing data to be saved. - * @param dir Directory to save the data. - * - * @deprecated Should use [[org.apache.spark.rdd.RDD#saveAsTextFile]] for saving and - * [[org.apache.spark.mllib.util.MLUtils#loadLabeledPoints]] for loading. - */ - @Since("1.0.0") - @deprecated("Should use RDD[LabeledPoint].saveAsTextFile instead.", "1.0.1") - def saveLabeledData(data: RDD[LabeledPoint], dir: String) { - val dataStr = data.map(x => x.label + "," + x.features.toArray.mkString(" ")) - dataStr.saveAsTextFile(dir) - } - /** * Return a k element array of pairs of RDDs with the first element of each pair * containing the training data, a complement of the validation data and the second @@ -265,6 +230,14 @@ object MLUtils { */ @Since("1.0.0") def kFold[T: ClassTag](rdd: RDD[T], numFolds: Int, seed: Int): Array[(RDD[T], RDD[T])] = { + kFold(rdd, numFolds, seed.toLong) + } + + /** + * Version of `kFold()` taking a Long seed. + */ + @Since("2.0.0") + def kFold[T: ClassTag](rdd: RDD[T], numFolds: Int, seed: Long): Array[(RDD[T], RDD[T])] = { val numFoldsF = numFolds.toFloat (1 to numFolds).map { fold => val sampler = new BernoulliCellSampler[T]((fold - 1) / numFoldsF, fold / numFoldsF, @@ -304,6 +277,211 @@ object MLUtils { } } + /** + * Converts vector columns in an input Dataset from the [[org.apache.spark.mllib.linalg.Vector]] + * type to the new [[org.apache.spark.ml.linalg.Vector]] type under the `spark.ml` package. + * @param dataset input dataset + * @param cols a list of vector columns to be converted. New vector columns will be ignored. If + * unspecified, all old vector columns will be converted except nested ones. + * @return the input `DataFrame` with old vector columns converted to the new vector type + */ + @Since("2.0.0") + @varargs + def convertVectorColumnsToML(dataset: Dataset[_], cols: String*): DataFrame = { + val schema = dataset.schema + val colSet = if (cols.nonEmpty) { + cols.flatMap { c => + val dataType = schema(c).dataType + if (dataType.getClass == classOf[VectorUDT]) { + Some(c) + } else { + // ignore new vector columns and raise an exception on other column types + require(dataType.getClass == classOf[MLVectorUDT], + s"Column $c must be old Vector type to be converted to new type but got $dataType.") + None + } + }.toSet + } else { + schema.fields + .filter(_.dataType.getClass == classOf[VectorUDT]) + .map(_.name) + .toSet + } + + if (colSet.isEmpty) { + return dataset.toDF() + } + + logWarning("Vector column conversion has serialization overhead. " + + "Please migrate your datasets and workflows to use the spark.ml package.") + + // TODO: This implementation has performance issues due to unnecessary serialization. + // TODO: It is better (but trickier) if we can cast the old vector type to new type directly. + val convertToML = udf { v: Vector => v.asML } + val exprs = schema.fields.map { field => + val c = field.name + if (colSet.contains(c)) { + convertToML(col(c)).as(c, field.metadata) + } else { + col(c) + } + } + dataset.select(exprs: _*) + } + + /** + * Converts vector columns in an input Dataset to the [[org.apache.spark.mllib.linalg.Vector]] + * type from the new [[org.apache.spark.ml.linalg.Vector]] type under the `spark.ml` package. + * @param dataset input dataset + * @param cols a list of vector columns to be converted. Old vector columns will be ignored. If + * unspecified, all new vector columns will be converted except nested ones. + * @return the input `DataFrame` with new vector columns converted to the old vector type + */ + @Since("2.0.0") + @varargs + def convertVectorColumnsFromML(dataset: Dataset[_], cols: String*): DataFrame = { + val schema = dataset.schema + val colSet = if (cols.nonEmpty) { + cols.flatMap { c => + val dataType = schema(c).dataType + if (dataType.getClass == classOf[MLVectorUDT]) { + Some(c) + } else { + // ignore old vector columns and raise an exception on other column types + require(dataType.getClass == classOf[VectorUDT], + s"Column $c must be new Vector type to be converted to old type but got $dataType.") + None + } + }.toSet + } else { + schema.fields + .filter(_.dataType.getClass == classOf[MLVectorUDT]) + .map(_.name) + .toSet + } + + if (colSet.isEmpty) { + return dataset.toDF() + } + + logWarning("Vector column conversion has serialization overhead. " + + "Please migrate your datasets and workflows to use the spark.ml package.") + + // TODO: This implementation has performance issues due to unnecessary serialization. + // TODO: It is better (but trickier) if we can cast the new vector type to old type directly. + val convertFromML = udf { Vectors.fromML _ } + val exprs = schema.fields.map { field => + val c = field.name + if (colSet.contains(c)) { + convertFromML(col(c)).as(c, field.metadata) + } else { + col(c) + } + } + dataset.select(exprs: _*) + } + + /** + * Converts Matrix columns in an input Dataset from the [[org.apache.spark.mllib.linalg.Matrix]] + * type to the new [[org.apache.spark.ml.linalg.Matrix]] type under the `spark.ml` package. + * @param dataset input dataset + * @param cols a list of matrix columns to be converted. New matrix columns will be ignored. If + * unspecified, all old matrix columns will be converted except nested ones. + * @return the input `DataFrame` with old matrix columns converted to the new matrix type + */ + @Since("2.0.0") + @varargs + def convertMatrixColumnsToML(dataset: Dataset[_], cols: String*): DataFrame = { + val schema = dataset.schema + val colSet = if (cols.nonEmpty) { + cols.flatMap { c => + val dataType = schema(c).dataType + if (dataType.getClass == classOf[MatrixUDT]) { + Some(c) + } else { + // ignore new matrix columns and raise an exception on other column types + require(dataType.getClass == classOf[MLMatrixUDT], + s"Column $c must be old Matrix type to be converted to new type but got $dataType.") + None + } + }.toSet + } else { + schema.fields + .filter(_.dataType.getClass == classOf[MatrixUDT]) + .map(_.name) + .toSet + } + + if (colSet.isEmpty) { + return dataset.toDF() + } + + logWarning("Matrix column conversion has serialization overhead. " + + "Please migrate your datasets and workflows to use the spark.ml package.") + + val convertToML = udf { v: Matrix => v.asML } + val exprs = schema.fields.map { field => + val c = field.name + if (colSet.contains(c)) { + convertToML(col(c)).as(c, field.metadata) + } else { + col(c) + } + } + dataset.select(exprs: _*) + } + + /** + * Converts matrix columns in an input Dataset to the [[org.apache.spark.mllib.linalg.Matrix]] + * type from the new [[org.apache.spark.ml.linalg.Matrix]] type under the `spark.ml` package. + * @param dataset input dataset + * @param cols a list of matrix columns to be converted. Old matrix columns will be ignored. If + * unspecified, all new matrix columns will be converted except nested ones. + * @return the input `DataFrame` with new matrix columns converted to the old matrix type + */ + @Since("2.0.0") + @varargs + def convertMatrixColumnsFromML(dataset: Dataset[_], cols: String*): DataFrame = { + val schema = dataset.schema + val colSet = if (cols.nonEmpty) { + cols.flatMap { c => + val dataType = schema(c).dataType + if (dataType.getClass == classOf[MLMatrixUDT]) { + Some(c) + } else { + // ignore old matrix columns and raise an exception on other column types + require(dataType.getClass == classOf[MatrixUDT], + s"Column $c must be new Matrix type to be converted to old type but got $dataType.") + None + } + }.toSet + } else { + schema.fields + .filter(_.dataType.getClass == classOf[MLMatrixUDT]) + .map(_.name) + .toSet + } + + if (colSet.isEmpty) { + return dataset.toDF() + } + + logWarning("Matrix column conversion has serialization overhead. " + + "Please migrate your datasets and workflows to use the spark.ml package.") + + val convertFromML = udf { Matrices.fromML _ } + val exprs = schema.fields.map { field => + val c = field.name + if (colSet.contains(c)) { + convertFromML(col(c)).as(c, field.metadata) + } else { + col(c) + } + } + dataset.select(exprs: _*) + } + + /** * Returns the squared Euclidean distance between two vectors. The following formula will be used * if it does not introduce too much numerical error: @@ -312,7 +490,6 @@ object MLUtils { * * When both vector norms are given, this is faster than computing the squared distance directly, * especially when one of the vectors is a sparse vector. - * * @param v1 the first vector * @param norm1 the norm of the first vector, non-negative * @param v2 the second vector @@ -365,7 +542,6 @@ object MLUtils { * When `x` is positive and large, computing `math.log(1 + math.exp(x))` will lead to arithmetic * overflow. This will happen when `x > 709.78` which is not a very large number. * It can be addressed by rewriting the formula into `x + math.log1p(math.exp(-x))` when `x > 0`. - * * @param x a floating-point value as input. * @return the result of `math.log(1 + math.exp(x))`. */ 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 a841c5caf014..2c613348c2d9 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 @@ -98,7 +98,7 @@ private[mllib] object NumericParser { } } else if (token == ")") { parsing = false - } else if (token.trim.isEmpty){ + } else if (token.trim.isEmpty) { // ignore whitespaces between delim chars, e.g. ", [" } else { // expecting a number diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index cde597939617..c9468606544d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -55,7 +55,7 @@ object SVMDataGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) - val trueWeights = Array.fill[Double](nfeatures + 1)(globalRnd.nextGaussian()) + val trueWeights = Array.fill[Double](nfeatures)(globalRnd.nextGaussian()) val data: RDD[LabeledPoint] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) 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 4d71d534a077..da0eb04764c5 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 @@ -45,7 +45,7 @@ trait Saveable { * - human-readable (JSON) model metadata to path/metadata/ * - Parquet formatted data to path/data/ * - * The model may be loaded using [[Loader.load]]. + * The model may be loaded using `Loader.load`. * * @param sc Spark context used to save model data. * @param path Path specifying the directory in which to save this model. @@ -72,7 +72,7 @@ trait Loader[M <: Saveable] { /** * Load a model from the given path. * - * The model should have been saved by [[Saveable.save]]. + * The model should have been saved by `Saveable.save`. * * @param sc Spark context used for loading model files. * @param path Path specifying the directory to which the model was saved. diff --git a/mllib/src/test/java/org/apache/spark/SharedSparkSession.java b/mllib/src/test/java/org/apache/spark/SharedSparkSession.java new file mode 100644 index 000000000000..43779878890d --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/SharedSparkSession.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark; + +import java.io.IOException; +import java.io.Serializable; + +import org.junit.After; +import org.junit.Before; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; + +public abstract class SharedSparkSession implements Serializable { + + protected transient SparkSession spark; + protected transient JavaSparkContext jsc; + + @Before + public void setUp() throws IOException { + spark = SparkSession.builder() + .master("local[2]") + .appName(getClass().getSimpleName()) + .getOrCreate(); + jsc = new JavaSparkContext(spark.sparkContext()); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } +} 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 0a8c9e595467..9b209006bc36 100644 --- a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java @@ -17,41 +17,32 @@ package org.apache.spark.ml; -import org.junit.After; -import org.junit.Before; +import java.io.IOException; + import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.ml.classification.LogisticRegression; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.feature.StandardScaler; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; /** * Test Pipeline construction and fitting in Java. */ -public class JavaPipelineSuite { +public class JavaPipelineSuite extends SharedSparkSession { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - private transient DataFrame dataset; + private transient Dataset dataset; - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaPipelineSuite"); - jsql = new SQLContext(jsc); + @Override + public void setUp() throws IOException { + super.setUp(); JavaRDD points = jsc.parallelize(generateLogisticInputAsList(1.0, 1.0, 100, 42), 2); - dataset = jsql.createDataFrame(points, LabeledPoint.class); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; + dataset = spark.createDataFrame(points, LabeledPoint.class); } @Test @@ -62,10 +53,10 @@ public void pipeline() { LogisticRegression lr = new LogisticRegression() .setFeaturesCol("scaledFeatures"); Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[] {scaler, lr}); + .setStages(new PipelineStage[]{scaler, lr}); PipelineModel model = pipeline.fit(dataset); - model.transform(dataset).registerTempTable("prediction"); - DataFrame predictions = jsql.sql("SELECT label, probability, prediction FROM prediction"); + model.transform(dataset).createOrReplaceTempView("prediction"); + Dataset predictions = spark.sql("SELECT label, probability, prediction FROM prediction"); predictions.collectAsList(); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/attribute/JavaAttributeSuite.java b/mllib/src/test/java/org/apache/spark/ml/attribute/JavaAttributeSuite.java index b74bbed23143..15cde0d3c045 100644 --- a/mllib/src/test/java/org/apache/spark/ml/attribute/JavaAttributeSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/attribute/JavaAttributeSuite.java @@ -17,8 +17,8 @@ package org.apache.spark.ml.attribute; -import org.junit.Test; import org.junit.Assert; +import org.junit.Test; public class JavaAttributeSuite { diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java index 60f25e5cce43..5aba4e8f7de0 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaDecisionTreeClassifierSuite.java @@ -17,36 +17,19 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.tree.impl.TreeTests; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; - -public class JavaDecisionTreeClassifierSuite implements Serializable { - - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaDecisionTreeClassifierSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaDecisionTreeClassifierSuite extends SharedSparkSession { @Test public void runDT() { @@ -54,10 +37,10 @@ public void runDT() { double A = 2.0; double B = -1.5; - JavaRDD data = sc.parallelize( + JavaRDD data = jsc.parallelize( LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); - Map categoricalFeatures = new HashMap(); - DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); + Map categoricalFeatures = new HashMap<>(); + Dataset dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); // This tests setters. Training with various options is tested in Scala. DecisionTreeClassifier dt = new DecisionTreeClassifier() @@ -69,7 +52,7 @@ public void runDT() { .setCacheNodeIds(false) .setCheckpointInterval(10) .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern - for (String impurity: DecisionTreeClassifier.supportedImpurities()) { + for (String impurity : DecisionTreeClassifier.supportedImpurities()) { dt.setImpurity(impurity); } DecisionTreeClassificationModel model = dt.fit(dataFrame); diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java index 3c69467fa119..74bb46bd217a 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaGBTClassifierSuite.java @@ -17,36 +17,19 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.tree.impl.TreeTests; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; - -public class JavaGBTClassifierSuite implements Serializable { - - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaGBTClassifierSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaGBTClassifierSuite extends SharedSparkSession { @Test public void runDT() { @@ -54,10 +37,10 @@ public void runDT() { double A = 2.0; double B = -1.5; - JavaRDD data = sc.parallelize( + JavaRDD data = jsc.parallelize( LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); - Map categoricalFeatures = new HashMap(); - DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); + Map categoricalFeatures = new HashMap<>(); + Dataset dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); // This tests setters. Training with various options is tested in Scala. GBTClassifier rf = new GBTClassifier() @@ -73,7 +56,7 @@ public void runDT() { .setMaxIter(3) .setStepSize(0.1) .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern - for (String lossType: GBTClassifier.supportedLossTypes()) { + for (String lossType : GBTClassifier.supportedLossTypes()) { rf.setLossType(lossType); } GBTClassificationModel model = rf.fit(dataFrame); 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 fd22eb6dca01..004102103d52 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 @@ -17,48 +17,34 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; -import java.lang.Math; +import java.io.IOException; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +public class JavaLogisticRegressionSuite extends SharedSparkSession { -public class JavaLogisticRegressionSuite implements Serializable { - - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - private transient DataFrame dataset; + private transient Dataset dataset; private transient JavaRDD datasetRDD; private double eps = 1e-5; - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaLogisticRegressionSuite"); - jsql = new SQLContext(jsc); + @Override + public void setUp() throws IOException { + super.setUp(); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); - dataset = jsql.createDataFrame(datasetRDD, LabeledPoint.class); - dataset.registerTempTable("dataset"); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; + dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); + dataset.createOrReplaceTempView("dataset"); } @Test @@ -66,8 +52,8 @@ public void logisticRegressionDefaultParams() { LogisticRegression lr = new LogisticRegression(); Assert.assertEquals(lr.getLabelCol(), "label"); LogisticRegressionModel model = lr.fit(dataset); - model.transform(dataset).registerTempTable("prediction"); - DataFrame predictions = jsql.sql("SELECT label, probability, prediction FROM prediction"); + model.transform(dataset).createOrReplaceTempView("prediction"); + Dataset predictions = spark.sql("SELECT label, probability, prediction FROM prediction"); predictions.collectAsList(); // Check defaults Assert.assertEquals(0.5, model.getThreshold(), eps); @@ -95,24 +81,24 @@ public void logisticRegressionWithSetters() { // Modify model params, and check that the params worked. model.setThreshold(1.0); - model.transform(dataset).registerTempTable("predAllZero"); - DataFrame predAllZero = jsql.sql("SELECT prediction, myProbability FROM predAllZero"); - for (Row r: predAllZero.collectAsList()) { + model.transform(dataset).createOrReplaceTempView("predAllZero"); + Dataset predAllZero = spark.sql("SELECT prediction, myProbability FROM predAllZero"); + for (Row r : predAllZero.collectAsList()) { Assert.assertEquals(0.0, r.getDouble(0), eps); } // Call transform with params, and check that the params worked. model.transform(dataset, model.threshold().w(0.0), model.probabilityCol().w("myProb")) - .registerTempTable("predNotAllZero"); - DataFrame predNotAllZero = jsql.sql("SELECT prediction, myProb FROM predNotAllZero"); + .createOrReplaceTempView("predNotAllZero"); + Dataset predNotAllZero = spark.sql("SELECT prediction, myProb FROM predNotAllZero"); boolean foundNonZero = false; - for (Row r: predNotAllZero.collectAsList()) { + for (Row r : predNotAllZero.collectAsList()) { if (r.getDouble(0) != 0.0) foundNonZero = true; } Assert.assertTrue(foundNonZero); // Call fit() with new params, and check as many params as we can. LogisticRegressionModel model2 = lr.fit(dataset, lr.maxIter().w(5), lr.regParam().w(0.1), - lr.threshold().w(0.4), lr.probabilityCol().w("theProb")); + lr.threshold().w(0.4), lr.probabilityCol().w("theProb")); LogisticRegression parent2 = (LogisticRegression) model2.parent(); Assert.assertEquals(5, parent2.getMaxIter()); Assert.assertEquals(0.1, parent2.getRegParam(), eps); @@ -128,11 +114,11 @@ public void logisticRegressionPredictorClassifierMethods() { LogisticRegressionModel model = lr.fit(dataset); Assert.assertEquals(2, model.numClasses()); - model.transform(dataset).registerTempTable("transformed"); - DataFrame trans1 = jsql.sql("SELECT rawPrediction, probability FROM transformed"); - for (Row row: trans1.collect()) { - Vector raw = (Vector)row.get(0); - Vector prob = (Vector)row.get(1); + model.transform(dataset).createOrReplaceTempView("transformed"); + Dataset trans1 = spark.sql("SELECT rawPrediction, probability FROM transformed"); + for (Row row : trans1.collectAsList()) { + Vector raw = (Vector) row.get(0); + Vector prob = (Vector) row.get(1); Assert.assertEquals(raw.size(), 2); Assert.assertEquals(prob.size(), 2); double probFromRaw1 = 1.0 / (1.0 + Math.exp(-raw.apply(1))); @@ -140,11 +126,11 @@ public void logisticRegressionPredictorClassifierMethods() { Assert.assertEquals(0, Math.abs(prob.apply(0) - (1.0 - probFromRaw1)), eps); } - DataFrame trans2 = jsql.sql("SELECT prediction, probability FROM transformed"); - for (Row row: trans2.collect()) { + Dataset trans2 = spark.sql("SELECT prediction, probability FROM transformed"); + for (Row row : trans2.collectAsList()) { double pred = row.getDouble(0); - Vector prob = (Vector)row.get(1); - double probOfPred = prob.apply((int)pred); + Vector prob = (Vector) row.get(1); + double probOfPred = prob.apply((int) pred); for (int i = 0; i < prob.size(); ++i) { Assert.assertTrue(probOfPred >= prob.apply(i)); } diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java index ec6b4bf3c0f8..6d0604d8f9a5 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java @@ -17,57 +17,39 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.Arrays; +import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; -public class JavaMultilayerPerceptronClassifierSuite implements Serializable { - - private transient JavaSparkContext jsc; - private transient SQLContext sqlContext; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaLogisticRegressionSuite"); - sqlContext = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - sqlContext = null; - } +public class JavaMultilayerPerceptronClassifierSuite extends SharedSparkSession { @Test public void testMLPC() { - DataFrame dataFrame = sqlContext.createDataFrame( - jsc.parallelize(Arrays.asList( - new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), - new LabeledPoint(1.0, Vectors.dense(0.0, 1.0)), - new LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), - new LabeledPoint(0.0, Vectors.dense(1.0, 1.0)))), - LabeledPoint.class); + List data = Arrays.asList( + new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), + new LabeledPoint(1.0, Vectors.dense(0.0, 1.0)), + new LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), + new LabeledPoint(0.0, Vectors.dense(1.0, 1.0)) + ); + Dataset dataFrame = spark.createDataFrame(data, LabeledPoint.class); + MultilayerPerceptronClassifier mlpc = new MultilayerPerceptronClassifier() - .setLayers(new int[] {2, 5, 2}) + .setLayers(new int[]{2, 5, 2}) .setBlockSize(1) - .setSeed(11L) + .setSeed(123L) .setMaxIter(100); MultilayerPerceptronClassificationModel model = mlpc.fit(dataFrame); - DataFrame result = model.transform(dataFrame); - Row[] predictionAndLabels = result.select("prediction", "label").collect(); - for (Row r: predictionAndLabels) { + Dataset result = model.transform(dataFrame); + List predictionAndLabels = result.select("prediction", "label").collectAsList(); + for (Row r : predictionAndLabels) { Assert.assertEquals((int) r.getDouble(0), (int) r.getDouble(1)); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java index f5f690eabd12..c2a9e7b58b47 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java @@ -17,47 +17,27 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaNaiveBayesSuite implements Serializable { +public class JavaNaiveBayesSuite extends SharedSparkSession { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaLogisticRegressionSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } - - public void validatePrediction(DataFrame predictionAndLabels) { - for (Row r : predictionAndLabels.collect()) { + public void validatePrediction(Dataset predictionAndLabels) { + for (Row r : predictionAndLabels.collectAsList()) { double prediction = r.getAs(0); double label = r.getAs(1); assertEquals(label, prediction, 1E-5); @@ -89,11 +69,11 @@ public void testNaiveBayes() { new StructField("features", new VectorUDT(), false, Metadata.empty()) }); - DataFrame dataset = jsql.createDataFrame(data, schema); + Dataset dataset = spark.createDataFrame(data, schema); NaiveBayes nb = new NaiveBayes().setSmoothing(0.5).setModelType("multinomial"); NaiveBayesModel model = nb.fit(dataset); - DataFrame predictionAndLabels = model.transform(dataset).select("prediction", "label"); + Dataset predictionAndLabels = model.transform(dataset).select("prediction", "label"); validatePrediction(predictionAndLabels); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java index cbabafe1b541..6194167bda35 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java @@ -17,67 +17,57 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; +import java.io.IOException; import java.util.List; import scala.collection.JavaConverters; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateMultinomialLogisticInput; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateMultinomialLogisticInput; -public class JavaOneVsRestSuite implements Serializable { +public class JavaOneVsRestSuite extends SharedSparkSession { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - private transient DataFrame dataset; - private transient JavaRDD datasetRDD; + private transient Dataset dataset; + private transient JavaRDD datasetRDD; - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaLOneVsRestSuite"); - jsql = new SQLContext(jsc); - int nPoints = 3; + @Override + public void setUp() throws IOException { + super.setUp(); + int nPoints = 3; - // The following coefficients and xMean/xVariance are computed from iris dataset with lambda=0.2. - // As a result, we are drawing samples from probability distribution of an actual model. - double[] coefficients = { - -0.57997, 0.912083, -0.371077, -0.819866, 2.688191, - -0.16624, -0.84355, -0.048509, -0.301789, 4.170682 }; + // The following coefficients and xMean/xVariance are computed from iris dataset with + // lambda=0.2. + // As a result, we are drawing samples from probability distribution of an actual model. + double[] coefficients = { + -0.57997, 0.912083, -0.371077, -0.819866, 2.688191, + -0.16624, -0.84355, -0.048509, -0.301789, 4.170682}; - double[] xMean = {5.843, 3.057, 3.758, 1.199}; - double[] xVariance = {0.6856, 0.1899, 3.116, 0.581}; - List points = JavaConverters.seqAsJavaListConverter( - generateMultinomialLogisticInput(coefficients, xMean, xVariance, true, nPoints, 42) - ).asJava(); - datasetRDD = jsc.parallelize(points, 2); - dataset = jsql.createDataFrame(datasetRDD, LabeledPoint.class); - } + double[] xMean = {5.843, 3.057, 3.758, 1.199}; + double[] xVariance = {0.6856, 0.1899, 3.116, 0.581}; + List points = JavaConverters.seqAsJavaListConverter( + generateMultinomialLogisticInput(coefficients, xMean, xVariance, true, nPoints, 42) + ).asJava(); + datasetRDD = jsc.parallelize(points, 2); + dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); + } - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } - - @Test - public void oneVsRestDefaultParams() { - OneVsRest ova = new OneVsRest(); - ova.setClassifier(new LogisticRegression()); - Assert.assertEquals(ova.getLabelCol() , "label"); - Assert.assertEquals(ova.getPredictionCol() , "prediction"); - OneVsRestModel ovaModel = ova.fit(dataset); - DataFrame predictions = ovaModel.transform(dataset).select("label", "prediction"); - predictions.collectAsList(); - Assert.assertEquals(ovaModel.getLabelCol(), "label"); - Assert.assertEquals(ovaModel.getPredictionCol() , "prediction"); - } + @Test + public void oneVsRestDefaultParams() { + OneVsRest ova = new OneVsRest(); + ova.setClassifier(new LogisticRegression()); + Assert.assertEquals(ova.getLabelCol(), "label"); + Assert.assertEquals(ova.getPredictionCol(), "prediction"); + OneVsRestModel ovaModel = ova.fit(dataset); + Dataset predictions = ovaModel.transform(dataset).select("label", "prediction"); + predictions.collectAsList(); + Assert.assertEquals(ovaModel.getLabelCol(), "label"); + Assert.assertEquals(ovaModel.getPredictionCol(), "prediction"); + } } diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java index a66a1e12927b..dd98513f37ec 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaRandomForestClassifierSuite.java @@ -17,37 +17,21 @@ package org.apache.spark.ml.classification; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; +import org.junit.Assert; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.tree.impl.TreeTests; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; - -public class JavaRandomForestClassifierSuite implements Serializable { - - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaRandomForestClassifierSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaRandomForestClassifierSuite extends SharedSparkSession { @Test public void runDT() { @@ -55,10 +39,10 @@ public void runDT() { double A = 2.0; double B = -1.5; - JavaRDD data = sc.parallelize( + JavaRDD data = jsc.parallelize( LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); - Map categoricalFeatures = new HashMap(); - DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); + Map categoricalFeatures = new HashMap<>(); + Dataset dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 2); // This tests setters. Training with various options is tested in Scala. RandomForestClassifier rf = new RandomForestClassifier() @@ -73,12 +57,30 @@ public void runDT() { .setSeed(1234) .setNumTrees(3) .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern - for (String impurity: RandomForestClassifier.supportedImpurities()) { + for (String impurity : RandomForestClassifier.supportedImpurities()) { rf.setImpurity(impurity); } - for (String featureSubsetStrategy: RandomForestClassifier.supportedFeatureSubsetStrategies()) { + for (String featureSubsetStrategy : RandomForestClassifier.supportedFeatureSubsetStrategies()) { rf.setFeatureSubsetStrategy(featureSubsetStrategy); } + String[] realStrategies = {".1", ".10", "0.10", "0.1", "0.9", "1.0"}; + for (String strategy : realStrategies) { + rf.setFeatureSubsetStrategy(strategy); + } + String[] integerStrategies = {"1", "10", "100", "1000", "10000"}; + for (String strategy : integerStrategies) { + rf.setFeatureSubsetStrategy(strategy); + } + String[] invalidStrategies = {"-.1", "-.10", "-0.10", ".0", "0.0", "1.1", "0"}; + for (String strategy : invalidStrategies) { + try { + rf.setFeatureSubsetStrategy(strategy); + Assert.fail("Expected exception to be thrown for invalid strategies"); + } catch (Exception e) { + Assert.assertTrue(e instanceof IllegalArgumentException); + } + } + RandomForestClassificationModel model = rf.fit(dataFrame); model.transform(dataFrame); diff --git a/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java index d09fa7fd5637..1be6f96f4c94 100644 --- a/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/clustering/JavaKMeansSuite.java @@ -17,41 +17,28 @@ package org.apache.spark.ml.clustering; -import java.io.Serializable; +import java.io.IOException; import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; -public class JavaKMeansSuite implements Serializable { +public class JavaKMeansSuite extends SharedSparkSession { private transient int k = 5; - private transient JavaSparkContext sc; - private transient DataFrame dataset; - private transient SQLContext sql; + private transient Dataset dataset; - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaKMeansSuite"); - sql = new SQLContext(sc); - - dataset = KMeansSuite.generateKMeansData(sql, 50, 3, k); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; + @Override + public void setUp() throws IOException { + super.setUp(); + dataset = KMeansSuite.generateKMeansData(spark, 50, 3, k); } @Test @@ -62,10 +49,10 @@ public void fitAndTransform() { Vector[] centers = model.clusterCenters(); assertEquals(k, centers.length); - DataFrame transformed = model.transform(dataset); + Dataset transformed = model.transform(dataset); List columns = Arrays.asList(transformed.columns()); List expectedColumns = Arrays.asList("features", "prediction"); - for (String column: expectedColumns) { + for (String column : expectedColumns) { assertTrue(columns.contains(column)); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java index 8a1e5ef01565..87639380bdcf 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java @@ -18,47 +18,30 @@ package org.apache.spark.ml.feature; import java.util.Arrays; +import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaBucketizerSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaBucketizerSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaBucketizerSuite extends SharedSparkSession { @Test public void bucketizerTest() { double[] splits = {-0.5, 0.0, 0.5}; - StructType schema = new StructType(new StructField[] { + StructType schema = new StructType(new StructField[]{ new StructField("feature", DataTypes.DoubleType, false, Metadata.empty()) }); - DataFrame dataset = jsql.createDataFrame( + Dataset dataset = spark.createDataFrame( Arrays.asList( RowFactory.create(-0.5), RowFactory.create(-0.3), @@ -71,7 +54,7 @@ public void bucketizerTest() { .setOutputCol("result") .setSplits(splits); - Row[] result = bucketizer.transform(dataset).select("result").collect(); + List result = bucketizer.transform(dataset).select("result").collectAsList(); for (Row r : result) { double index = r.getDouble(0); diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java index 39da47381b12..b7956b6fd3e9 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java @@ -18,46 +18,30 @@ package org.apache.spark.ml.feature; import java.util.Arrays; +import java.util.List; import edu.emory.mathcs.jtransforms.dct.DoubleDCT_1D; -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.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaDCTSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaDCTSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaDCTSuite extends SharedSparkSession { @Test public void javaCompatibilityTest() { - double[] input = new double[] {1D, 2D, 3D, 4D}; - DataFrame dataset = jsql.createDataFrame( + double[] input = new double[]{1D, 2D, 3D, 4D}; + Dataset dataset = spark.createDataFrame( Arrays.asList(RowFactory.create(Vectors.dense(input))), new StructType(new StructField[]{ new StructField("vec", (new VectorUDT()), false, Metadata.empty()) @@ -70,8 +54,8 @@ public void javaCompatibilityTest() { .setInputCol("vec") .setOutputCol("resultVec"); - Row[] result = dct.transform(dataset).select("resultVec").collect(); - Vector resultVec = result[0].getAs("resultVec"); + List result = dct.transform(dataset).select("resultVec").collectAsList(); + Vector resultVec = result.get(0).getAs("resultVec"); Assert.assertArrayEquals(expectedResult, resultVec.toArray(), 1e-6); } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java index d12332c2a02a..57696d0150a8 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java @@ -20,39 +20,21 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaHashingTFSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaHashingTFSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaHashingTFSuite extends SharedSparkSession { @Test public void hashingTF() { @@ -66,21 +48,21 @@ public void hashingTF() { new StructField("sentence", DataTypes.StringType, false, Metadata.empty()) }); - DataFrame sentenceData = jsql.createDataFrame(data, schema); + Dataset sentenceData = spark.createDataFrame(data, schema); Tokenizer tokenizer = new Tokenizer() .setInputCol("sentence") .setOutputCol("words"); - DataFrame wordsData = tokenizer.transform(sentenceData); + Dataset wordsData = tokenizer.transform(sentenceData); int numFeatures = 20; HashingTF hashingTF = new HashingTF() .setInputCol("words") .setOutputCol("rawFeatures") .setNumFeatures(numFeatures); - DataFrame featurizedData = hashingTF.transform(wordsData); + Dataset featurizedData = hashingTF.transform(wordsData); IDF idf = new IDF().setInputCol("rawFeatures").setOutputCol("features"); IDFModel idfModel = idf.fit(featurizedData); - DataFrame rescaledData = idfModel.transform(featurizedData); - for (Row r : rescaledData.select("features", "label").take(3)) { + Dataset rescaledData = idfModel.transform(featurizedData); + for (Row r : rescaledData.select("features", "label").takeAsList(3)) { Vector features = r.getAs(0); Assert.assertEquals(features.size(), numFeatures); } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java index e17d549c5059..6f877b566875 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java @@ -19,31 +19,15 @@ import java.util.Arrays; -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; -public class JavaNormalizerSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaNormalizerSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaNormalizerSuite extends SharedSparkSession { @Test public void normalizer() { @@ -53,17 +37,17 @@ public void normalizer() { new VectorIndexerSuite.FeatureData(Vectors.dense(1.0, 3.0)), new VectorIndexerSuite.FeatureData(Vectors.dense(1.0, 4.0)) )); - DataFrame dataFrame = jsql.createDataFrame(points, VectorIndexerSuite.FeatureData.class); + Dataset dataFrame = spark.createDataFrame(points, VectorIndexerSuite.FeatureData.class); Normalizer normalizer = new Normalizer() .setInputCol("features") .setOutputCol("normFeatures"); // Normalize each Vector using $L^2$ norm. - DataFrame l2NormData = normalizer.transform(dataFrame, normalizer.p().w(2)); + Dataset l2NormData = normalizer.transform(dataFrame, normalizer.p().w(2)); l2NormData.count(); // Normalize each Vector using $L^\infty$ norm. - DataFrame lInfNormData = + Dataset lInfNormData = normalizer.transform(dataFrame, normalizer.p().w(Double.POSITIVE_INFINITY)); lInfNormData.count(); } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java index e8f329f9cf29..683ceffeaed0 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java @@ -21,39 +21,20 @@ import java.util.Arrays; import java.util.List; -import scala.Tuple2; - -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.function.Function; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.distributed.RowMatrix; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.mllib.linalg.DenseVector; import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; - -public class JavaPCASuite implements Serializable { - private transient JavaSparkContext jsc; - private transient SQLContext sqlContext; - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaPCASuite"); - sqlContext = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaPCASuite extends SharedSparkSession { public static class VectorPair implements Serializable { private Vector features = Vectors.dense(0.0); @@ -85,22 +66,25 @@ public void testPCA() { ); JavaRDD dataRDD = jsc.parallelize(points, 2); - RowMatrix mat = new RowMatrix(dataRDD.rdd()); + RowMatrix mat = new RowMatrix(dataRDD.map( + (Vector vector) -> (org.apache.spark.mllib.linalg.Vector) new DenseVector(vector.toArray()) + ).rdd()); + Matrix pc = mat.computePrincipalComponents(3); - JavaRDD expected = mat.multiply(pc).rows().toJavaRDD(); - - JavaRDD featuresExpected = dataRDD.zip(expected).map( - new Function, VectorPair>() { - public VectorPair call(Tuple2 pair) { - VectorPair featuresExpected = new VectorPair(); - featuresExpected.setFeatures(pair._1()); - featuresExpected.setExpected(pair._2()); - return featuresExpected; - } - } - ); - DataFrame df = sqlContext.createDataFrame(featuresExpected, VectorPair.class); + mat.multiply(pc).rows().toJavaRDD(); + + JavaRDD expected = mat.multiply(pc).rows().toJavaRDD() + .map(org.apache.spark.mllib.linalg.Vector::asML); + + JavaRDD featuresExpected = dataRDD.zip(expected).map(pair -> { + VectorPair featuresExpected1 = new VectorPair(); + featuresExpected1.setFeatures(pair._1()); + featuresExpected1.setExpected(pair._2()); + return featuresExpected1; + }); + + Dataset df = spark.createDataFrame(featuresExpected, VectorPair.class); PCAModel pca = new PCA() .setInputCol("features") .setOutputCol("pca_features") @@ -108,7 +92,11 @@ public VectorPair call(Tuple2 pair) { .fit(df); List result = pca.transform(df).select("pca_features", "expected").toJavaRDD().collect(); for (Row r : result) { - Assert.assertEquals(r.get(1), r.get(0)); + Vector calculatedVector = (Vector) r.get(0); + Vector expectedVector = (Vector) r.get(1); + for (int i = 0; i < calculatedVector.size(); i++) { + Assert.assertEquals(calculatedVector.apply(i), expectedVector.apply(i), 1.0e-8); + } } } } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java index bf8eefd71905..df5d34fbe94e 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java @@ -20,39 +20,21 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaPolynomialExpansionSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaPolynomialExpansionSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaPolynomialExpansionSuite extends SharedSparkSession { @Test public void polynomialExpansionTest() { @@ -73,20 +55,20 @@ public void polynomialExpansionTest() { ) ); - StructType schema = new StructType(new StructField[] { + StructType schema = new StructType(new StructField[]{ new StructField("features", new VectorUDT(), false, Metadata.empty()), new StructField("expected", new VectorUDT(), false, Metadata.empty()) }); - DataFrame dataset = jsql.createDataFrame(data, schema); + Dataset dataset = spark.createDataFrame(data, schema); - Row[] pairs = polyExpansion.transform(dataset) + List pairs = polyExpansion.transform(dataset) .select("polyFeatures", "expected") - .collect(); + .collectAsList(); for (Row r : pairs) { - double[] polyFeatures = ((Vector)r.get(0)).toArray(); - double[] expected = ((Vector)r.get(1)).toArray(); + double[] polyFeatures = ((Vector) r.get(0)).toArray(); + double[] expected = ((Vector) r.get(1)).toArray(); Assert.assertArrayEquals(polyFeatures, expected, 1e-1); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java index ed74363f59e3..dbc0b1db5c00 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java @@ -20,30 +20,14 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; -public class JavaStandardScalerSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaStandardScalerSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaStandardScalerSuite extends SharedSparkSession { @Test public void standardScaler() { @@ -53,7 +37,7 @@ public void standardScaler() { new VectorIndexerSuite.FeatureData(Vectors.dense(1.0, 3.0)), new VectorIndexerSuite.FeatureData(Vectors.dense(1.0, 4.0)) ); - DataFrame dataFrame = jsql.createDataFrame(jsc.parallelize(points, 2), + Dataset dataFrame = spark.createDataFrame(jsc.parallelize(points, 2), VectorIndexerSuite.FeatureData.class); StandardScaler scaler = new StandardScaler() .setInputCol("features") @@ -65,7 +49,7 @@ public void standardScaler() { StandardScalerModel scalerModel = scaler.fit(dataFrame); // Normalize each feature to have unit standard deviation. - DataFrame scaledData = scalerModel.transform(dataFrame); + Dataset scaledData = scalerModel.transform(dataFrame); scaledData.count(); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java index 848d9f8aa928..6480b57e1f79 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java @@ -20,38 +20,19 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -public class JavaStopWordsRemoverSuite { - - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaStopWordsRemoverSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaStopWordsRemoverSuite extends SharedSparkSession { @Test public void javaCompatibilityTest() { @@ -63,10 +44,11 @@ public void javaCompatibilityTest() { RowFactory.create(Arrays.asList("I", "saw", "the", "red", "baloon")), RowFactory.create(Arrays.asList("Mary", "had", "a", "little", "lamb")) ); - StructType schema = new StructType(new StructField[] { - new StructField("raw", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty()) + StructType schema = new StructType(new StructField[]{ + new StructField("raw", DataTypes.createArrayType(DataTypes.StringType), false, + Metadata.empty()) }); - DataFrame dataset = jsql.createDataFrame(data, schema); + Dataset dataset = spark.createDataFrame(data, schema); remover.transform(dataset).collect(); } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java index 6b2c48ef1c34..c1928a26b609 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStringIndexerSuite.java @@ -20,59 +20,44 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; +import static org.apache.spark.sql.types.DataTypes.*; + 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.SharedSparkSession; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import static org.apache.spark.sql.types.DataTypes.*; - -public class JavaStringIndexerSuite { - private transient JavaSparkContext jsc; - private transient SQLContext sqlContext; - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaStringIndexerSuite"); - sqlContext = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - sqlContext = null; - } +public class JavaStringIndexerSuite extends SharedSparkSession { @Test public void testStringIndexer() { - StructType schema = createStructType(new StructField[] { + StructType schema = createStructType(new StructField[]{ createStructField("id", IntegerType, false), createStructField("label", StringType, false) }); List data = Arrays.asList( - c(0, "a"), c(1, "b"), c(2, "c"), c(3, "a"), c(4, "a"), c(5, "c")); - DataFrame dataset = sqlContext.createDataFrame(data, schema); + cr(0, "a"), cr(1, "b"), cr(2, "c"), cr(3, "a"), cr(4, "a"), cr(5, "c")); + Dataset dataset = spark.createDataFrame(data, schema); StringIndexer indexer = new StringIndexer() .setInputCol("label") .setOutputCol("labelIndex"); - DataFrame output = indexer.fit(dataset).transform(dataset); + Dataset output = indexer.fit(dataset).transform(dataset); - Assert.assertArrayEquals( - new Row[] { c(0, 0.0), c(1, 2.0), c(2, 1.0), c(3, 0.0), c(4, 0.0), c(5, 1.0) }, - output.orderBy("id").select("id", "labelIndex").collect()); + Assert.assertEquals( + Arrays.asList(cr(0, 0.0), cr(1, 2.0), cr(2, 1.0), cr(3, 0.0), cr(4, 0.0), cr(5, 1.0)), + output.orderBy("id").select("id", "labelIndex").collectAsList()); } - /** An alias for RowFactory.create. */ - private Row c(Object... values) { + /** + * An alias for RowFactory.create. + */ + private Row cr(Object... values) { return RowFactory.create(values); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java index 02309ce63219..27550a3d5c37 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java @@ -18,33 +18,17 @@ package org.apache.spark.ml.feature; import java.util.Arrays; +import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; 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.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; -public class JavaTokenizerSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaTokenizerSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaTokenizerSuite extends SharedSparkSession { @Test public void regexTokenizer() { @@ -53,18 +37,19 @@ public void regexTokenizer() { .setOutputCol("tokens") .setPattern("\\s") .setGaps(true) + .setToLowercase(false) .setMinTokenLength(3); JavaRDD rdd = jsc.parallelize(Arrays.asList( - new TokenizerTestData("Test of tok.", new String[] {"Test", "tok."}), - new TokenizerTestData("Te,st. punct", new String[] {"Te,st.", "punct"}) + new TokenizerTestData("Test of tok.", new String[]{"Test", "tok."}), + new TokenizerTestData("Te,st. punct", new String[]{"Te,st.", "punct"}) )); - DataFrame dataset = jsql.createDataFrame(rdd, TokenizerTestData.class); + Dataset dataset = spark.createDataFrame(rdd, TokenizerTestData.class); - Row[] pairs = myRegExTokenizer.transform(dataset) + List pairs = myRegExTokenizer.transform(dataset) .select("tokens", "wantedTokens") - .collect(); + .collectAsList(); for (Row r : pairs) { Assert.assertEquals(r.get(0), r.get(1)); diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java index e28377757093..583652badb8f 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorAssemblerSuite.java @@ -19,42 +19,26 @@ import java.util.Arrays; -import org.junit.After; +import static org.apache.spark.sql.types.DataTypes.*; + 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.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.VectorUDT; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.VectorUDT; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.types.*; -import static org.apache.spark.sql.types.DataTypes.*; - -public class JavaVectorAssemblerSuite { - private transient JavaSparkContext jsc; - private transient SQLContext sqlContext; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaVectorAssemblerSuite"); - sqlContext = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaVectorAssemblerSuite extends SharedSparkSession { @Test public void testVectorAssembler() { - StructType schema = createStructType(new StructField[] { + StructType schema = createStructType(new StructField[]{ createStructField("id", IntegerType, false), createStructField("x", DoubleType, false), createStructField("y", new VectorUDT(), false), @@ -64,14 +48,14 @@ public void testVectorAssembler() { }); Row row = RowFactory.create( 0, 0.0, Vectors.dense(1.0, 2.0), "a", - Vectors.sparse(2, new int[] {1}, new double[] {3.0}), 10L); - DataFrame dataset = sqlContext.createDataFrame(Arrays.asList(row), schema); + Vectors.sparse(2, new int[]{1}, new double[]{3.0}), 10L); + Dataset dataset = spark.createDataFrame(Arrays.asList(row), schema); VectorAssembler assembler = new VectorAssembler() - .setInputCols(new String[] {"x", "y", "z", "n"}) + .setInputCols(new String[]{"x", "y", "z", "n"}) .setOutputCol("features"); - DataFrame output = assembler.transform(dataset); + Dataset output = assembler.transform(dataset); Assert.assertEquals( - Vectors.sparse(6, new int[] {1, 2, 4, 5}, new double[] {1.0, 2.0, 3.0, 10.0}), + Vectors.sparse(6, new int[]{1, 2, 4, 5}, new double[]{1.0, 2.0, 3.0, 10.0}), output.select("features").first().getAs(0)); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java index bfcca62fa1c9..ca8fae3a48b9 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java @@ -17,36 +17,21 @@ package org.apache.spark.ml.feature; -import java.io.Serializable; import java.util.Arrays; import java.util.List; import java.util.Map; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.feature.VectorIndexerSuite.FeatureData; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; -public class JavaVectorIndexerSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaVectorIndexerSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaVectorIndexerSuite extends SharedSparkSession { @Test public void vectorIndexerAPI() { @@ -56,8 +41,7 @@ public void vectorIndexerAPI() { new FeatureData(Vectors.dense(1.0, 3.0)), new FeatureData(Vectors.dense(1.0, 4.0)) ); - SQLContext sqlContext = new SQLContext(sc); - DataFrame data = sqlContext.createDataFrame(sc.parallelize(points, 2), FeatureData.class); + Dataset data = spark.createDataFrame(jsc.parallelize(points, 2), FeatureData.class); VectorIndexer indexer = new VectorIndexer() .setInputCol("features") .setOutputCol("indexed") @@ -66,6 +50,6 @@ public void vectorIndexerAPI() { Assert.assertEquals(model.numFeatures(), 2); Map> categoryMaps = model.javaCategoryMaps(); Assert.assertEquals(categoryMaps.size(), 1); - DataFrame indexedData = model.transform(data); + Dataset indexedData = model.transform(data); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java index 00174e6a683d..3dc2e1f89614 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java @@ -20,40 +20,22 @@ import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.attribute.Attribute; import org.apache.spark.ml.attribute.AttributeGroup; import org.apache.spark.ml.attribute.NumericAttribute; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.StructType; -public class JavaVectorSlicerSuite { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaVectorSlicerSuite"); - jsql = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaVectorSlicerSuite extends SharedSparkSession { @Test public void vectorSlice() { @@ -69,16 +51,17 @@ public void vectorSlice() { RowFactory.create(Vectors.dense(-2.0, 2.3, 0.0)) ); - DataFrame dataset = jsql.createDataFrame(data, (new StructType()).add(group.toStructField())); + Dataset dataset = + spark.createDataFrame(data, (new StructType()).add(group.toStructField())); VectorSlicer vectorSlicer = new VectorSlicer() .setInputCol("userFeatures").setOutputCol("features"); vectorSlicer.setIndices(new int[]{1}).setNames(new String[]{"f3"}); - DataFrame output = vectorSlicer.transform(dataset); + Dataset output = vectorSlicer.transform(dataset); - for (Row r : output.select("userFeatures", "features").take(2)) { + for (Row r : output.select("userFeatures", "features").takeAsList(2)) { Vector features = r.getAs(1); Assert.assertEquals(features.size(), 2); } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java index 0c0c1c4d12d0..d0a849fd11c7 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java @@ -19,42 +19,24 @@ import java.util.Arrays; -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.mllib.linalg.Vector; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.types.*; -public class JavaWord2VecSuite { - private transient JavaSparkContext jsc; - private transient SQLContext sqlContext; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaWord2VecSuite"); - sqlContext = new SQLContext(jsc); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } +public class JavaWord2VecSuite extends SharedSparkSession { @Test public void testJavaWord2Vec() { StructType schema = new StructType(new StructField[]{ new StructField("text", new ArrayType(DataTypes.StringType, true), false, Metadata.empty()) }); - DataFrame documentDF = sqlContext.createDataFrame( + Dataset documentDF = spark.createDataFrame( Arrays.asList( RowFactory.create(Arrays.asList("Hi I heard about Spark".split(" "))), RowFactory.create(Arrays.asList("I wish Java could use case classes".split(" "))), @@ -67,10 +49,10 @@ public void testJavaWord2Vec() { .setVectorSize(3) .setMinCount(0); Word2VecModel model = word2Vec.fit(documentDF); - DataFrame result = model.transform(documentDF); + Dataset result = model.transform(documentDF); - for (Row r: result.select("result").collect()) { - double[] polyFeatures = ((Vector)r.get(0)).toArray(); + for (Row r : result.select("result").collectAsList()) { + double[] polyFeatures = ((Vector) r.get(0)).toArray(); Assert.assertEquals(polyFeatures.length, 3); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/linalg/JavaSQLDataTypesSuite.java b/mllib/src/test/java/org/apache/spark/ml/linalg/JavaSQLDataTypesSuite.java new file mode 100644 index 000000000000..bd64a7186eac --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/linalg/JavaSQLDataTypesSuite.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg; + +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.spark.ml.linalg.SQLDataTypes.*; + +public class JavaSQLDataTypesSuite { + @Test + public void testSQLDataTypes() { + Assert.assertEquals(new VectorUDT(), VectorType()); + Assert.assertEquals(new MatrixUDT(), MatrixType()); + } +} diff --git a/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java b/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java index fa777f3d42a9..1077e103a3b8 100644 --- a/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java @@ -19,31 +19,14 @@ import java.util.Arrays; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; - /** * Test Param and related classes in Java */ public class JavaParamsSuite { - private transient JavaSparkContext jsc; - - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaParamsSuite"); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; - } - @Test public void testParams() { JavaTestParams testParams = new JavaTestParams(); @@ -51,7 +34,7 @@ public void testParams() { testParams.setMyIntParam(2).setMyDoubleParam(0.4).setMyStringParam("a"); Assert.assertEquals(testParams.getMyDoubleParam(), 0.4, 0.0); Assert.assertEquals(testParams.getMyStringParam(), "a"); - Assert.assertArrayEquals(testParams.getMyDoubleArrayParam(), new double[] {1.0, 2.0}, 0.0); + Assert.assertArrayEquals(testParams.getMyDoubleArrayParam(), new double[]{1.0, 2.0}, 0.0); } @Test diff --git a/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java b/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java index 65841182df9b..1ad5f7a442da 100644 --- a/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java +++ b/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java @@ -45,9 +45,14 @@ public String uid() { } private IntParam myIntParam_; - public IntParam myIntParam() { return myIntParam_; } - public int getMyIntParam() { return (Integer)getOrDefault(myIntParam_); } + public IntParam myIntParam() { + return myIntParam_; + } + + public int getMyIntParam() { + return (Integer) getOrDefault(myIntParam_); + } public JavaTestParams setMyIntParam(int value) { set(myIntParam_, value); @@ -55,9 +60,14 @@ public JavaTestParams setMyIntParam(int value) { } private DoubleParam myDoubleParam_; - public DoubleParam myDoubleParam() { return myDoubleParam_; } - public double getMyDoubleParam() { return (Double)getOrDefault(myDoubleParam_); } + public DoubleParam myDoubleParam() { + return myDoubleParam_; + } + + public double getMyDoubleParam() { + return (Double) getOrDefault(myDoubleParam_); + } public JavaTestParams setMyDoubleParam(double value) { set(myDoubleParam_, value); @@ -65,9 +75,14 @@ public JavaTestParams setMyDoubleParam(double value) { } private Param myStringParam_; - public Param myStringParam() { return myStringParam_; } - public String getMyStringParam() { return getOrDefault(myStringParam_); } + public Param myStringParam() { + return myStringParam_; + } + + public String getMyStringParam() { + return getOrDefault(myStringParam_); + } public JavaTestParams setMyStringParam(String value) { set(myStringParam_, value); @@ -75,9 +90,14 @@ public JavaTestParams setMyStringParam(String value) { } private DoubleArrayParam myDoubleArrayParam_; - public DoubleArrayParam myDoubleArrayParam() { return myDoubleArrayParam_; } - public double[] getMyDoubleArrayParam() { return getOrDefault(myDoubleArrayParam_); } + public DoubleArrayParam myDoubleArrayParam() { + return myDoubleArrayParam_; + } + + public double[] getMyDoubleArrayParam() { + return getOrDefault(myDoubleArrayParam_); + } public JavaTestParams setMyDoubleArrayParam(double[] value) { set(myDoubleArrayParam_, value); @@ -89,14 +109,14 @@ private void init() { myDoubleParam_ = new DoubleParam(this, "myDoubleParam", "this is a double param", ParamValidators.inRange(0.0, 1.0)); List validStrings = Arrays.asList("a", "b"); - myStringParam_ = new Param(this, "myStringParam", "this is a string param", + myStringParam_ = new Param<>(this, "myStringParam", "this is a string param", ParamValidators.inArray(validStrings)); myDoubleArrayParam_ = new DoubleArrayParam(this, "myDoubleArrayParam", "this is a double param"); setDefault(myIntParam(), 1); setDefault(myDoubleParam(), 0.5); - setDefault(myDoubleArrayParam(), new double[] {1.0, 2.0}); + setDefault(myDoubleArrayParam(), new double[]{1.0, 2.0}); } @Override diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java index ebe800e749e0..1da85ed9dab4 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaDecisionTreeRegressorSuite.java @@ -17,36 +17,21 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.tree.impl.TreeTests; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; -public class JavaDecisionTreeRegressorSuite implements Serializable { - - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaDecisionTreeRegressorSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaDecisionTreeRegressorSuite extends SharedSparkSession { @Test public void runDT() { @@ -54,10 +39,10 @@ public void runDT() { double A = 2.0; double B = -1.5; - JavaRDD data = sc.parallelize( + JavaRDD data = jsc.parallelize( LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); - Map categoricalFeatures = new HashMap(); - DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 0); + Map categoricalFeatures = new HashMap<>(); + Dataset dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 0); // This tests setters. Training with various options is tested in Scala. DecisionTreeRegressor dt = new DecisionTreeRegressor() @@ -69,7 +54,7 @@ public void runDT() { .setCacheNodeIds(false) .setCheckpointInterval(10) .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern - for (String impurity: DecisionTreeRegressor.supportedImpurities()) { + for (String impurity : DecisionTreeRegressor.supportedImpurities()) { dt.setImpurity(impurity); } DecisionTreeRegressionModel model = dt.fit(dataFrame); diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java index fc8c13db07e6..7fd9b1feb7f8 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaGBTRegressorSuite.java @@ -17,36 +17,21 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.impl.TreeTests; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.tree.impl.TreeTests; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; -public class JavaGBTRegressorSuite implements Serializable { - - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaGBTRegressorSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaGBTRegressorSuite extends SharedSparkSession { @Test public void runDT() { @@ -54,10 +39,10 @@ public void runDT() { double A = 2.0; double B = -1.5; - JavaRDD data = sc.parallelize( + JavaRDD data = jsc.parallelize( LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); - Map categoricalFeatures = new HashMap(); - DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 0); + Map categoricalFeatures = new HashMap<>(); + Dataset dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 0); GBTRegressor rf = new GBTRegressor() .setMaxDepth(2) @@ -72,7 +57,7 @@ public void runDT() { .setMaxIter(3) .setStepSize(0.1) .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern - for (String lossType: GBTRegressor.supportedLossTypes()) { + for (String lossType : GBTRegressor.supportedLossTypes()) { rf.setLossType(lossType); } GBTRegressionModel model = rf.fit(dataFrame); 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 4fb0b0d1092b..6cdcdda1a648 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 @@ -17,44 +17,30 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; +import java.io.IOException; import java.util.List; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite - .generateLogisticInputAsList; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; - -public class JavaLinearRegressionSuite implements Serializable { - - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - private transient DataFrame dataset; +public class JavaLinearRegressionSuite extends SharedSparkSession { + private transient Dataset dataset; private transient JavaRDD datasetRDD; - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); - jsql = new SQLContext(jsc); + @Override + public void setUp() throws IOException { + super.setUp(); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); datasetRDD = jsc.parallelize(points, 2); - dataset = jsql.createDataFrame(datasetRDD, LabeledPoint.class); - dataset.registerTempTable("dataset"); - } - - @After - public void tearDown() { - jsc.stop(); - jsc = null; + dataset = spark.createDataFrame(datasetRDD, LabeledPoint.class); + dataset.createOrReplaceTempView("dataset"); } @Test @@ -63,8 +49,8 @@ public void linearRegressionDefaultParams() { assertEquals("label", lr.getLabelCol()); assertEquals("auto", lr.getSolver()); LinearRegressionModel model = lr.fit(dataset); - model.transform(dataset).registerTempTable("prediction"); - DataFrame predictions = jsql.sql("SELECT label, prediction FROM prediction"); + model.transform(dataset).createOrReplaceTempView("prediction"); + Dataset predictions = spark.sql("SELECT label, prediction FROM prediction"); predictions.collect(); // Check defaults assertEquals("features", model.getFeaturesCol()); @@ -75,8 +61,8 @@ public void linearRegressionDefaultParams() { public void linearRegressionWithSetters() { // Set params, train, and check as many params as we can. LinearRegression lr = new LinearRegression() - .setMaxIter(10) - .setRegParam(1.0).setSolver("l-bfgs"); + .setMaxIter(10) + .setRegParam(1.0).setSolver("l-bfgs"); LinearRegressionModel model = lr.fit(dataset); LinearRegression parent = (LinearRegression) model.parent(); assertEquals(10, parent.getMaxIter()); @@ -84,7 +70,7 @@ public void linearRegressionWithSetters() { // Call fit() with new params, and check as many params as we can. LinearRegressionModel model2 = - lr.fit(dataset, lr.maxIter().w(5), lr.regParam().w(0.1), lr.predictionCol().w("thePred")); + lr.fit(dataset, lr.maxIter().w(5), lr.regParam().w(0.1), lr.predictionCol().w("thePred")); LinearRegression parent2 = (LinearRegression) model2.parent(); assertEquals(5, parent2.getMaxIter()); assertEquals(0.1, parent2.getRegParam(), 0.0); diff --git a/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java b/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java index a00ce5e249c3..4ba13e2e06c8 100644 --- a/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaRandomForestRegressorSuite.java @@ -17,37 +17,23 @@ package org.apache.spark.ml.regression; -import java.io.Serializable; import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Before; +import org.junit.Assert; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.classification.LogisticRegressionSuite; -import org.apache.spark.ml.impl.TreeTests; -import org.apache.spark.mllib.linalg.Vector; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.ml.classification.LogisticRegressionSuite; +import org.apache.spark.ml.feature.LabeledPoint; +import org.apache.spark.ml.linalg.Vector; +import org.apache.spark.ml.tree.impl.TreeTests; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; -public class JavaRandomForestRegressorSuite implements Serializable { - - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaRandomForestRegressorSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaRandomForestRegressorSuite extends SharedSparkSession { @Test public void runDT() { @@ -55,10 +41,10 @@ public void runDT() { double A = 2.0; double B = -1.5; - JavaRDD data = sc.parallelize( + JavaRDD data = jsc.parallelize( LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); - Map categoricalFeatures = new HashMap(); - DataFrame dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 0); + Map categoricalFeatures = new HashMap<>(); + Dataset dataFrame = TreeTests.setMetadata(data, categoricalFeatures, 0); // This tests setters. Training with various options is tested in Scala. RandomForestRegressor rf = new RandomForestRegressor() @@ -73,12 +59,30 @@ public void runDT() { .setSeed(1234) .setNumTrees(3) .setMaxDepth(2); // duplicate setMaxDepth to check builder pattern - for (String impurity: RandomForestRegressor.supportedImpurities()) { + for (String impurity : RandomForestRegressor.supportedImpurities()) { rf.setImpurity(impurity); } - for (String featureSubsetStrategy: RandomForestRegressor.supportedFeatureSubsetStrategies()) { + for (String featureSubsetStrategy : RandomForestRegressor.supportedFeatureSubsetStrategies()) { rf.setFeatureSubsetStrategy(featureSubsetStrategy); } + String[] realStrategies = {".1", ".10", "0.10", "0.1", "0.9", "1.0"}; + for (String strategy : realStrategies) { + rf.setFeatureSubsetStrategy(strategy); + } + String[] integerStrategies = {"1", "10", "100", "1000", "10000"}; + for (String strategy : integerStrategies) { + rf.setFeatureSubsetStrategy(strategy); + } + String[] invalidStrategies = {"-.1", "-.10", "-0.10", ".0", "0.0", "1.1", "0"}; + for (String strategy : invalidStrategies) { + try { + rf.setFeatureSubsetStrategy(strategy); + Assert.fail("Expected exception to be thrown for invalid strategies"); + } catch (Exception e) { + Assert.assertTrue(e instanceof IllegalArgumentException); + } + } + RandomForestRegressionModel model = rf.fit(dataFrame); model.transform(dataFrame); diff --git a/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java b/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java index 2976b38e4503..fa39f4560c8a 100644 --- a/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java @@ -19,56 +19,48 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; -import com.google.common.base.Charsets; import com.google.common.io.Files; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.mllib.linalg.DenseVector; -import org.apache.spark.mllib.linalg.Vectors; -import org.apache.spark.sql.DataFrame; +import org.apache.spark.SharedSparkSession; +import org.apache.spark.ml.linalg.DenseVector; +import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; import org.apache.spark.util.Utils; /** * Test LibSVMRelation in Java. */ -public class JavaLibSVMRelationSuite { - private transient JavaSparkContext jsc; - private transient SQLContext sqlContext; +public class JavaLibSVMRelationSuite extends SharedSparkSession { private File tempDir; private String path; - @Before + @Override public void setUp() throws IOException { - jsc = new JavaSparkContext("local", "JavaLibSVMRelationSuite"); - sqlContext = new SQLContext(jsc); - + super.setUp(); tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "datasource"); File file = new File(tempDir, "part-00000"); String s = "1 1:1.0 3:2.0 5:3.0\n0\n0 2:4.0 4:5.0 6:6.0"; - Files.write(s, file, Charsets.US_ASCII); + Files.write(s, file, StandardCharsets.UTF_8); path = tempDir.toURI().toString(); } - @After + @Override public void tearDown() { - jsc.stop(); - jsc = null; + super.tearDown(); Utils.deleteRecursively(tempDir); } @Test public void verifyLibSVMDF() { - DataFrame dataset = sqlContext.read().format("libsvm").option("vectorType", "dense") + Dataset dataset = spark.read().format("libsvm").option("vectorType", "dense") .load(path); Assert.assertEquals("label", dataset.columns()[0]); Assert.assertEquals("features", dataset.columns()[1]); 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 08eeca53f072..692d5ad591e8 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 @@ -17,49 +17,39 @@ package org.apache.spark.ml.tuning; -import java.io.Serializable; +import java.io.IOException; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.SharedSparkSession; import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator; +import org.apache.spark.ml.feature.LabeledPoint; import org.apache.spark.ml.param.ParamMap; -import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import static org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInputAsList; -public class JavaCrossValidatorSuite implements Serializable { - private transient JavaSparkContext jsc; - private transient SQLContext jsql; - private transient DataFrame dataset; +public class JavaCrossValidatorSuite extends SharedSparkSession { - @Before - public void setUp() { - jsc = new JavaSparkContext("local", "JavaCrossValidatorSuite"); - jsql = new SQLContext(jsc); - List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); - dataset = jsql.createDataFrame(jsc.parallelize(points, 2), LabeledPoint.class); - } + private transient Dataset dataset; - @After - public void tearDown() { - jsc.stop(); - jsc = null; + @Override + public void setUp() throws IOException { + super.setUp(); + List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); + dataset = spark.createDataFrame(jsc.parallelize(points, 2), LabeledPoint.class); } @Test public void crossValidationWithLogisticRegression() { LogisticRegression lr = new LogisticRegression(); ParamMap[] lrParamMaps = new ParamGridBuilder() - .addGrid(lr.regParam(), new double[] {0.001, 1000.0}) - .addGrid(lr.maxIter(), new int[] {0, 10}) + .addGrid(lr.regParam(), new double[]{0.001, 1000.0}) + .addGrid(lr.maxIter(), new int[]{0, 10}) .build(); BinaryClassificationEvaluator eval = new BinaryClassificationEvaluator(); CrossValidator cv = new CrossValidator() diff --git a/mllib/src/test/java/org/apache/spark/ml/util/IdentifiableSuite.scala b/mllib/src/test/java/org/apache/spark/ml/util/IdentifiableSuite.scala index 928301523fba..878bc66ee37c 100644 --- a/mllib/src/test/java/org/apache/spark/ml/util/IdentifiableSuite.scala +++ b/mllib/src/test/java/org/apache/spark/ml/util/IdentifiableSuite.scala @@ -37,4 +37,5 @@ object IdentifiableSuite { class Test(override val uid: String) extends Identifiable { def this() = this(Identifiable.randomUID("test")) } + } diff --git a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java new file mode 100644 index 000000000000..e4f678fef1d1 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.util; + +import java.io.File; +import java.io.IOException; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.spark.SharedSparkSession; +import org.apache.spark.util.Utils; + +public class JavaDefaultReadWriteSuite extends SharedSparkSession { + File tempDir = null; + + @Override + public void setUp() throws IOException { + super.setUp(); + tempDir = Utils.createTempDir( + System.getProperty("java.io.tmpdir"), "JavaDefaultReadWriteSuite"); + } + + @Override + public void tearDown() { + super.tearDown(); + Utils.deleteRecursively(tempDir); + } + + @Test + public void testDefaultReadWrite() throws IOException { + String uid = "my_params"; + MyParams instance = new MyParams(uid); + instance.set(instance.intParam(), 2); + String outputPath = new File(tempDir, uid).getPath(); + instance.save(outputPath); + try { + instance.save(outputPath); + Assert.fail( + "Write without overwrite enabled should fail if the output directory already exists."); + } catch (IOException e) { + // expected + } + instance.write().session(spark).overwrite().save(outputPath); + MyParams newInstance = MyParams.load(outputPath); + Assert.assertEquals("UID should match.", instance.uid(), newInstance.uid()); + Assert.assertEquals("Params should be preserved.", + 2, newInstance.getOrDefault(newInstance.intParam())); + } +} diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java index 862221d48798..c04e2e69541b 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java @@ -17,36 +17,20 @@ package org.apache.spark.mllib.classification; -import java.io.Serializable; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; - import org.apache.spark.mllib.regression.LabeledPoint; -public class JavaLogisticRegressionSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaLogisticRegressionSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaLogisticRegressionSuite extends SharedSparkSession { int validatePrediction(List validationData, LogisticRegressionModel model) { int numAccurate = 0; - for (LabeledPoint point: validationData) { + for (LabeledPoint point : validationData) { Double prediction = model.predict(point.features()); if (prediction == point.label()) { numAccurate++; @@ -61,16 +45,16 @@ public void runLRUsingConstructor() { double A = 2.0; double B = -1.5; - JavaRDD testRDD = sc.parallelize( - LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + JavaRDD testRDD = jsc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); List validationData = - LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); LogisticRegressionWithSGD lrImpl = new LogisticRegressionWithSGD(); lrImpl.setIntercept(true); lrImpl.optimizer().setStepSize(1.0) - .setRegParam(1.0) - .setNumIterations(100); + .setRegParam(1.0) + .setNumIterations(100); LogisticRegressionModel model = lrImpl.run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); @@ -83,13 +67,13 @@ public void runLRUsingStaticMethods() { double A = 0.0; double B = -2.5; - JavaRDD testRDD = sc.parallelize( - LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + JavaRDD testRDD = jsc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); List validationData = - LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); LogisticRegressionModel model = LogisticRegressionWithSGD.train( - testRDD.rdd(), 100, 1.0, 1.0); + testRDD.rdd(), 100, 1.0, 1.0); int numAccurate = validatePrediction(validationData, model); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java index 3771c0ea7ad8..65db3d014fdc 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java @@ -17,36 +17,20 @@ package org.apache.spark.mllib.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; -public class JavaNaiveBayesSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaNaiveBayesSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaNaiveBayesSuite extends SharedSparkSession { private static final List POINTS = Arrays.asList( new LabeledPoint(0, Vectors.dense(1.0, 0.0, 0.0)), @@ -57,9 +41,9 @@ public void tearDown() { new LabeledPoint(2, Vectors.dense(0.0, 0.0, 2.0)) ); - private int validatePrediction(List points, NaiveBayesModel model) { + private static int validatePrediction(List points, NaiveBayesModel model) { int correct = 0; - for (LabeledPoint p: points) { + for (LabeledPoint p : points) { if (model.predict(p.features()) == p.label()) { correct += 1; } @@ -69,7 +53,7 @@ private int validatePrediction(List points, NaiveBayesModel model) @Test public void runUsingConstructor() { - JavaRDD testRDD = sc.parallelize(POINTS, 2).cache(); + JavaRDD testRDD = jsc.parallelize(POINTS, 2).cache(); NaiveBayes nb = new NaiveBayes().setLambda(1.0); NaiveBayesModel model = nb.run(testRDD.rdd()); @@ -80,7 +64,7 @@ public void runUsingConstructor() { @Test public void runUsingStaticMethods() { - JavaRDD testRDD = sc.parallelize(POINTS, 2).cache(); + JavaRDD testRDD = jsc.parallelize(POINTS, 2).cache(); NaiveBayesModel model1 = NaiveBayes.train(testRDD.rdd()); int numAccurate1 = validatePrediction(POINTS, model1); @@ -93,13 +77,9 @@ public void runUsingStaticMethods() { @Test public void testPredictJavaRDD() { - JavaRDD examples = sc.parallelize(POINTS, 2).cache(); + JavaRDD examples = jsc.parallelize(POINTS, 2).cache(); NaiveBayesModel model = NaiveBayes.train(examples.rdd()); - JavaRDD vectors = examples.map(new Function() { - @Override - public Vector call(LabeledPoint v) throws Exception { - return v.features(); - }}); + JavaRDD vectors = examples.map(LabeledPoint::features); JavaRDD predictions = model.predict(vectors); // Should be able to get the first prediction. predictions.first(); diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java index 31b9f3e8d438..0f54e684e447 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java @@ -17,35 +17,20 @@ package org.apache.spark.mllib.classification; -import java.io.Serializable; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.regression.LabeledPoint; -public class JavaSVMSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaSVMSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaSVMSuite extends SharedSparkSession { int validatePrediction(List validationData, SVMModel model) { int numAccurate = 0; - for (LabeledPoint point: validationData) { + for (LabeledPoint point : validationData) { Double prediction = model.predict(point.features()); if (prediction == point.label()) { numAccurate++; @@ -60,16 +45,16 @@ public void runSVMUsingConstructor() { double A = 2.0; double[] weights = {-1.5, 1.0}; - JavaRDD testRDD = sc.parallelize(SVMSuite.generateSVMInputAsList(A, - weights, nPoints, 42), 2).cache(); + JavaRDD testRDD = jsc.parallelize(SVMSuite.generateSVMInputAsList(A, + weights, nPoints, 42), 2).cache(); List validationData = - SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17); + SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17); SVMWithSGD svmSGDImpl = new SVMWithSGD(); svmSGDImpl.setIntercept(true); svmSGDImpl.optimizer().setStepSize(1.0) - .setRegParam(1.0) - .setNumIterations(100); + .setRegParam(1.0) + .setNumIterations(100); SVMModel model = svmSGDImpl.run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); @@ -82,10 +67,10 @@ public void runSVMUsingStaticMethods() { double A = 0.0; double[] weights = {-1.5, 1.0}; - JavaRDD testRDD = sc.parallelize(SVMSuite.generateSVMInputAsList(A, - weights, nPoints, 42), 2).cache(); + JavaRDD testRDD = jsc.parallelize(SVMSuite.generateSVMInputAsList(A, + weights, nPoints, 42), 2).cache(); List validationData = - SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17); + SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17); SVMModel model = SVMWithSGD.train(testRDD.rdd(), 100, 1.0, 1.0, 1.0); diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java index c9e5ee22f327..8c6bced52dd7 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.classification; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -37,7 +36,7 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStreamingLogisticRegressionSuite implements Serializable { +public class JavaStreamingLogisticRegressionSuite { protected transient JavaStreamingContext ssc; @@ -66,8 +65,8 @@ public void javaAPI() { JavaDStream training = attachTestInputStream(ssc, Arrays.asList(trainingBatch, trainingBatch), 2); List> testBatch = Arrays.asList( - new Tuple2(10, Vectors.dense(1.0)), - new Tuple2(11, Vectors.dense(0.0))); + new Tuple2<>(10, Vectors.dense(1.0)), + new Tuple2<>(11, Vectors.dense(0.0))); JavaPairDStream test = JavaPairDStream.fromJavaDStream( attachTestInputStream(ssc, Arrays.asList(testBatch, testBatch), 2)); StreamingLogisticRegressionWithSGD slr = new StreamingLogisticRegressionWithSGD() diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java new file mode 100644 index 000000000000..b4196c6ecdf7 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.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.mllib.clustering; + +import java.util.Arrays; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.spark.SharedSparkSession; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; + +public class JavaBisectingKMeansSuite extends SharedSparkSession { + + @Test + public void twoDimensionalData() { + JavaRDD points = jsc.parallelize(Arrays.asList( + Vectors.dense(4, -1), + Vectors.dense(4, 1), + Vectors.sparse(2, new int[]{0}, new double[]{1.0}) + ), 2); + + BisectingKMeans bkm = new BisectingKMeans() + .setK(4) + .setMaxIterations(2) + .setSeed(1L); + BisectingKMeansModel model = bkm.run(points); + Assert.assertEquals(3, model.k()); + Assert.assertArrayEquals(new double[]{3.0, 0.0}, model.root().center().toArray(), 1e-12); + for (ClusteringTreeNode child : model.root().children()) { + double[] center = child.center().toArray(); + if (center[0] > 2) { + Assert.assertEquals(2, child.size()); + Assert.assertArrayEquals(new double[]{4.0, 0.0}, center, 1e-12); + } else { + Assert.assertEquals(1, child.size()); + Assert.assertArrayEquals(new double[]{1.0, 0.0}, center, 1e-12); + } + } + } +} diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java index 123f78da54e3..bf7671993777 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java @@ -17,34 +17,19 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - import static org.junit.Assert.assertEquals; +import org.junit.Test; + +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -public class JavaGaussianMixtureSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaGaussianMixture"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaGaussianMixtureSuite extends SharedSparkSession { @Test public void runGaussianMixture() { @@ -54,7 +39,7 @@ public void runGaussianMixture() { Vectors.dense(1.0, 4.0, 6.0) ); - JavaRDD data = sc.parallelize(points, 2); + JavaRDD data = jsc.parallelize(points, 2); GaussianMixtureModel model = new GaussianMixture().setK(2).setMaxIterations(1).setSeed(1234) .run(data); assertEquals(model.gaussians().length, 2); diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java index ad06676c72ac..270e636f8211 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java @@ -17,33 +17,19 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; +import static org.junit.Assert.assertEquals; + import org.junit.Test; -import static org.junit.Assert.*; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -public class JavaKMeansSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaKMeans"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaKMeansSuite extends SharedSparkSession { @Test public void runKMeansUsingStaticMethods() { @@ -55,7 +41,7 @@ public void runKMeansUsingStaticMethods() { Vector expectedCenter = Vectors.dense(1.0, 3.0, 4.0); - JavaRDD data = sc.parallelize(points, 2); + JavaRDD data = jsc.parallelize(points, 2); KMeansModel model = KMeans.train(data.rdd(), 1, 1, 1, KMeans.K_MEANS_PARALLEL()); assertEquals(1, model.clusterCenters().length); assertEquals(expectedCenter, model.clusterCenters()[0]); @@ -74,7 +60,7 @@ public void runKMeansUsingConstructor() { Vector expectedCenter = Vectors.dense(1.0, 3.0, 4.0); - JavaRDD data = sc.parallelize(points, 2); + JavaRDD data = jsc.parallelize(points, 2); KMeansModel model = new KMeans().setK(1).setMaxIterations(5).run(data.rdd()); assertEquals(1, model.clusterCenters().length); assertEquals(expectedCenter, model.clusterCenters()[0]); @@ -94,7 +80,7 @@ public void testPredictJavaRDD() { Vectors.dense(1.0, 3.0, 0.0), Vectors.dense(1.0, 4.0, 6.0) ); - JavaRDD data = sc.parallelize(points, 2); + JavaRDD data = jsc.parallelize(points, 2); KMeansModel model = new KMeans().setK(1).setMaxIterations(5).run(data.rdd()); JavaRDD predictions = model.predict(data); // Should be able to get the first prediction. diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java index 3fea359a3b46..38ee2507f2e1 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java @@ -17,55 +17,43 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import scala.Tuple2; import scala.Tuple3; -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; -import org.apache.spark.api.java.function.Function; +import org.apache.spark.SharedSparkSession; 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.mllib.linalg.Matrix; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; -public class JavaLDASuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaLDA"); - ArrayList> tinyCorpus = new ArrayList>(); +public class JavaLDASuite extends SharedSparkSession { + @Override + public void setUp() throws IOException { + super.setUp(); + List> tinyCorpus = new ArrayList<>(); for (int i = 0; i < LDASuite.tinyCorpus().length; i++) { - tinyCorpus.add(new Tuple2((Long)LDASuite.tinyCorpus()[i]._1(), - LDASuite.tinyCorpus()[i]._2())); + tinyCorpus.add(new Tuple2<>((Long) LDASuite.tinyCorpus()[i]._1(), + LDASuite.tinyCorpus()[i]._2())); } - JavaRDD> tmpCorpus = sc.parallelize(tinyCorpus, 2); + JavaRDD> tmpCorpus = jsc.parallelize(tinyCorpus, 2); corpus = JavaPairRDD.fromJavaRDD(tmpCorpus); } - @After - public void tearDown() { - sc.stop(); - sc = null; - } - @Test public void localLDAModel() { Matrix topics = LDASuite.tinyTopics(); double[] topicConcentration = new double[topics.numRows()]; Arrays.fill(topicConcentration, 1.0D / topics.numRows()); - LocalLDAModel model = new LocalLDAModel(topics, Vectors.dense(topicConcentration), 1D, 100D); + LocalLDAModel model = new LocalLDAModel(topics, Vectors.dense(topicConcentration), 1.0, 100.0); // Check: basic parameters assertEquals(model.k(), tinyK); @@ -95,21 +83,21 @@ public void distributedLDAModel() { .setMaxIterations(5) .setSeed(12345); - DistributedLDAModel model = (DistributedLDAModel)lda.run(corpus); + DistributedLDAModel model = (DistributedLDAModel) lda.run(corpus); // Check: basic parameters LocalLDAModel localModel = model.toLocal(); - assertEquals(model.k(), k); - assertEquals(localModel.k(), k); - assertEquals(model.vocabSize(), tinyVocabSize); - assertEquals(localModel.vocabSize(), tinyVocabSize); - assertEquals(model.topicsMatrix(), localModel.topicsMatrix()); + assertEquals(k, model.k()); + assertEquals(k, localModel.k()); + assertEquals(tinyVocabSize, model.vocabSize()); + assertEquals(tinyVocabSize, localModel.vocabSize()); + assertEquals(localModel.topicsMatrix(), model.topicsMatrix()); // Check: topic summaries Tuple2[] roundedTopicSummary = model.describeTopics(); - assertEquals(roundedTopicSummary.length, k); + assertEquals(k, roundedTopicSummary.length); Tuple2[] roundedLocalTopicSummary = localModel.describeTopics(); - assertEquals(roundedLocalTopicSummary.length, k); + assertEquals(k, roundedLocalTopicSummary.length); // Check: log probabilities assertTrue(model.logLikelihood() < 0.0); @@ -119,12 +107,8 @@ public void distributedLDAModel() { JavaPairRDD topicDistributions = model.javaTopicDistributions(); // SPARK-5562. since the topicDistribution returns the distribution of the non empty docs // over topics. Compare it against nonEmptyCorpus instead of corpus - JavaPairRDD nonEmptyCorpus = corpus.filter( - new Function, Boolean>() { - public Boolean call(Tuple2 tuple2) { - return Vectors.norm(tuple2._2(), 1.0) != 0.0; - } - }); + JavaPairRDD nonEmptyCorpus = + corpus.filter(tuple2 -> Vectors.norm(tuple2._2(), 1.0) != 0.0); assertEquals(topicDistributions.count(), nonEmptyCorpus.count()); // Check: javaTopTopicsPerDocuments @@ -144,7 +128,7 @@ public Boolean call(Tuple2 tuple2) { } @Test - public void OnlineOptimizerCompatibility() { + public void onlineOptimizerCompatibility() { int k = 3; double topicSmoothing = 1.2; double termSmoothing = 1.2; @@ -167,19 +151,19 @@ public void OnlineOptimizerCompatibility() { LDAModel model = lda.run(corpus); // Check: basic parameters - assertEquals(model.k(), k); - assertEquals(model.vocabSize(), tinyVocabSize); + assertEquals(k, model.k()); + assertEquals(tinyVocabSize, model.vocabSize()); // Check: topic summaries Tuple2[] roundedTopicSummary = model.describeTopics(); - assertEquals(roundedTopicSummary.length, k); + assertEquals(k, roundedTopicSummary.length); Tuple2[] roundedLocalTopicSummary = model.describeTopics(); - assertEquals(roundedLocalTopicSummary.length, k); + assertEquals(k, roundedLocalTopicSummary.length); } @Test public void localLdaMethods() { - JavaRDD> docs = sc.parallelize(toyData, 2); + JavaRDD> docs = jsc.parallelize(toyData, 2); JavaPairRDD pairedDocs = JavaPairRDD.fromJavaRDD(docs); // check: topicDistributions @@ -189,9 +173,9 @@ public void localLdaMethods() { double logPerplexity = toyModel.logPerplexity(pairedDocs); // check: logLikelihood. - ArrayList> docsSingleWord = new ArrayList>(); - docsSingleWord.add(new Tuple2(0L, Vectors.dense(1.0, 0.0, 0.0))); - JavaPairRDD single = JavaPairRDD.fromJavaRDD(sc.parallelize(docsSingleWord)); + List> docsSingleWord = new ArrayList<>(); + docsSingleWord.add(new Tuple2<>(0L, Vectors.dense(1.0, 0.0, 0.0))); + JavaPairRDD single = JavaPairRDD.fromJavaRDD(jsc.parallelize(docsSingleWord)); double logLikelihood = toyModel.logLikelihood(single); } @@ -199,9 +183,9 @@ public void localLdaMethods() { private static int tinyVocabSize = LDASuite.tinyVocabSize(); private static Matrix tinyTopics = LDASuite.tinyTopics(); private static Tuple2[] tinyTopicDescription = - LDASuite.tinyTopicDescription(); + LDASuite.tinyTopicDescription(); private JavaPairRDD corpus; private LocalLDAModel toyModel = LDASuite.toyModel(); - private ArrayList> toyData = LDASuite.javaToyData(); + private List> toyData = LDASuite.javaToyData(); } diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java index d644766d1e54..d41fc0e4dca9 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.clustering; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -27,8 +26,6 @@ import org.junit.Before; import org.junit.Test; -import static org.apache.spark.streaming.JavaTestUtils.*; - import org.apache.spark.SparkConf; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; @@ -36,8 +33,9 @@ import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; +import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStreamingKMeansSuite implements Serializable { +public class JavaStreamingKMeansSuite { protected transient JavaStreamingContext ssc; @@ -66,8 +64,8 @@ public void javaAPI() { JavaDStream training = attachTestInputStream(ssc, Arrays.asList(trainingBatch, trainingBatch), 2); List> testBatch = Arrays.asList( - new Tuple2(10, Vectors.dense(1.0)), - new Tuple2(11, Vectors.dense(0.0))); + new Tuple2<>(10, Vectors.dense(1.0)), + new Tuple2<>(11, Vectors.dense(0.0))); JavaPairDStream test = JavaPairDStream.fromJavaDStream( attachTestInputStream(ssc, Arrays.asList(testBatch, testBatch), 2)); StreamingKMeans skmeans = new StreamingKMeans() diff --git a/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java index fa4d334801ce..e9d7e4fdbe8c 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/evaluation/JavaRankingMetricsSuite.java @@ -17,41 +17,32 @@ package org.apache.spark.mllib.evaluation; -import java.io.Serializable; +import java.io.IOException; import java.util.Arrays; import java.util.List; import scala.Tuple2; import scala.Tuple2$; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class JavaRankingMetricsSuite implements Serializable { - private transient JavaSparkContext sc; +public class JavaRankingMetricsSuite extends SharedSparkSession { private transient JavaRDD, List>> predictionAndLabels; - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaRankingMetricsSuite"); - predictionAndLabels = sc.parallelize(Arrays.asList( + @Override + public void setUp() throws IOException { + super.setUp(); + predictionAndLabels = jsc.parallelize(Arrays.asList( Tuple2$.MODULE$.apply( Arrays.asList(1, 6, 2, 7, 8, 3, 9, 10, 4, 5), Arrays.asList(1, 2, 3, 4, 5)), Tuple2$.MODULE$.apply( - Arrays.asList(4, 1, 5, 6, 2, 7, 3, 8, 9, 10), Arrays.asList(1, 2, 3)), + Arrays.asList(4, 1, 5, 6, 2, 7, 3, 8, 9, 10), Arrays.asList(1, 2, 3)), Tuple2$.MODULE$.apply( - Arrays.asList(1, 2, 3, 4, 5), Arrays.asList())), 2); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; + Arrays.asList(1, 2, 3, 4, 5), Arrays.asList())), 2); } @Test diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java index 8a320afa4b13..05128ea34342 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java @@ -17,39 +17,24 @@ package org.apache.spark.mllib.feature; -import java.io.Serializable; import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; -public class JavaTfIdfSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaTfIdfSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaTfIdfSuite extends SharedSparkSession { @Test public void tfIdf() { // The tests are to check Java compatibility. HashingTF tf = new HashingTF(); @SuppressWarnings("unchecked") - JavaRDD> documents = sc.parallelize(Arrays.asList( + JavaRDD> documents = jsc.parallelize(Arrays.asList( Arrays.asList("this is a sentence".split(" ")), Arrays.asList("this is another sentence".split(" ")), Arrays.asList("this is still a sentence".split(" "))), 2); @@ -59,7 +44,7 @@ public void tfIdf() { JavaRDD tfIdfs = idf.fit(termFreqs).transform(termFreqs); List localTfIdfs = tfIdfs.collect(); int indexOfThis = tf.indexOf("this"); - for (Vector v: localTfIdfs) { + for (Vector v : localTfIdfs) { Assert.assertEquals(0.0, v.apply(indexOfThis), 1e-15); } } @@ -69,7 +54,7 @@ public void tfIdfMinimumDocumentFrequency() { // The tests are to check Java compatibility. HashingTF tf = new HashingTF(); @SuppressWarnings("unchecked") - JavaRDD> documents = sc.parallelize(Arrays.asList( + JavaRDD> documents = jsc.parallelize(Arrays.asList( Arrays.asList("this is a sentence".split(" ")), Arrays.asList("this is another sentence".split(" ")), Arrays.asList("this is still a sentence".split(" "))), 2); @@ -79,7 +64,7 @@ public void tfIdfMinimumDocumentFrequency() { JavaRDD tfIdfs = idf.fit(termFreqs).transform(termFreqs); List localTfIdfs = tfIdfs.collect(); int indexOfThis = tf.indexOf("this"); - for (Vector v: localTfIdfs) { + for (Vector v : localTfIdfs) { Assert.assertEquals(0.0, v.apply(indexOfThis), 1e-15); } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java index e13ed07e283d..3e3abddbee63 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java @@ -17,34 +17,20 @@ package org.apache.spark.mllib.feature; -import java.io.Serializable; import java.util.Arrays; import java.util.List; +import com.google.common.base.Strings; + import scala.Tuple2; -import com.google.common.base.Strings; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; - -public class JavaWord2VecSuite implements Serializable { - private transient JavaSparkContext sc; - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaWord2VecSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaWord2VecSuite extends SharedSparkSession { @Test @SuppressWarnings("unchecked") @@ -53,7 +39,7 @@ public void word2Vec() { String sentence = Strings.repeat("a b ", 100) + Strings.repeat("a c ", 10); List words = Arrays.asList(sentence.split(" ")); List> localDoc = Arrays.asList(words, words); - JavaRDD> doc = sc.parallelize(localDoc); + JavaRDD> doc = jsc.parallelize(localDoc); Word2Vec word2vec = new Word2Vec() .setVectorSize(10) .setSeed(42L); diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java index 2bef7a860975..15de566c886d 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java @@ -16,42 +16,26 @@ */ package org.apache.spark.mllib.fpm; -import java.io.Serializable; import java.util.Arrays; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset; -public class JavaAssociationRulesSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaFPGrowth"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaAssociationRulesSuite extends SharedSparkSession { @Test public void runAssociationRules() { @SuppressWarnings("unchecked") - JavaRDD> freqItemsets = sc.parallelize(Arrays.asList( - new FreqItemset(new String[] {"a"}, 15L), - new FreqItemset(new String[] {"b"}, 35L), - new FreqItemset(new String[] {"a", "b"}, 12L) + JavaRDD> freqItemsets = jsc.parallelize(Arrays.asList( + new FreqItemset<>(new String[]{"a"}, 15L), + new FreqItemset<>(new String[]{"b"}, 35L), + new FreqItemset<>(new String[]{"a", "b"}, 12L) )); JavaRDD> results = (new AssociationRules()).run(freqItemsets); } } - diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java index 154f75d75e4a..46e9dd8b5982 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java @@ -17,37 +17,25 @@ package org.apache.spark.mllib.fpm; -import java.io.Serializable; +import java.io.File; import java.util.Arrays; import java.util.List; -import org.junit.After; -import org.junit.Before; +import static org.junit.Assert.assertEquals; + import org.junit.Test; -import static org.junit.Assert.*; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; - -public class JavaFPGrowthSuite implements Serializable { - private transient JavaSparkContext sc; +import org.apache.spark.util.Utils; - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaFPGrowth"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaFPGrowthSuite extends SharedSparkSession { @Test public void runFPGrowth() { @SuppressWarnings("unchecked") - JavaRDD> rdd = sc.parallelize(Arrays.asList( + JavaRDD> rdd = jsc.parallelize(Arrays.asList( Arrays.asList("r z h k p".split(" ")), Arrays.asList("z y x w v u t s".split(" ")), Arrays.asList("s x o n r".split(" ")), @@ -63,10 +51,50 @@ public void runFPGrowth() { List> freqItemsets = model.freqItemsets().toJavaRDD().collect(); assertEquals(18, freqItemsets.size()); - for (FPGrowth.FreqItemset itemset: freqItemsets) { + for (FPGrowth.FreqItemset itemset : freqItemsets) { // Test return types. List items = itemset.javaItems(); long freq = itemset.freq(); } } + + @Test + public void runFPGrowthSaveLoad() { + + @SuppressWarnings("unchecked") + JavaRDD> rdd = jsc.parallelize(Arrays.asList( + Arrays.asList("r z h k p".split(" ")), + Arrays.asList("z y x w v u t s".split(" ")), + Arrays.asList("s x o n r".split(" ")), + Arrays.asList("x z y m t s q e".split(" ")), + Arrays.asList("z".split(" ")), + Arrays.asList("x z y r q t p".split(" "))), 2); + + FPGrowthModel model = new FPGrowth() + .setMinSupport(0.5) + .setNumPartitions(2) + .run(rdd); + + File tempDir = Utils.createTempDir( + System.getProperty("java.io.tmpdir"), "JavaFPGrowthSuite"); + String outputPath = tempDir.getPath(); + + try { + model.save(spark.sparkContext(), outputPath); + @SuppressWarnings("unchecked") + FPGrowthModel newModel = + (FPGrowthModel) FPGrowthModel.load(spark.sparkContext(), outputPath); + List> freqItemsets = newModel.freqItemsets().toJavaRDD() + .collect(); + assertEquals(18, freqItemsets.size()); + + for (FPGrowth.FreqItemset itemset : freqItemsets) { + // Test return types. + List items = itemset.javaItems(); + long freq = itemset.freq(); + } + } finally { + Utils.deleteRecursively(tempDir); + } + } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java index 34daf5fbde80..32d3141149a7 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaPrefixSpanSuite.java @@ -17,35 +17,23 @@ package org.apache.spark.mllib.fpm; +import java.io.File; import java.util.Arrays; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.fpm.PrefixSpan.FreqSequence; +import org.apache.spark.util.Utils; -public class JavaPrefixSpanSuite { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaPrefixSpan"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaPrefixSpanSuite extends SharedSparkSession { @Test public void runPrefixSpan() { - JavaRDD>> sequences = sc.parallelize(Arrays.asList( + JavaRDD>> sequences = jsc.parallelize(Arrays.asList( Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3)), Arrays.asList(Arrays.asList(1), Arrays.asList(3, 2), Arrays.asList(1, 2)), Arrays.asList(Arrays.asList(1, 2), Arrays.asList(5)), @@ -59,9 +47,46 @@ public void runPrefixSpan() { List> localFreqSeqs = freqSeqs.collect(); Assert.assertEquals(5, localFreqSeqs.size()); // Check that each frequent sequence could be materialized. - for (PrefixSpan.FreqSequence freqSeq: localFreqSeqs) { + for (PrefixSpan.FreqSequence freqSeq : localFreqSeqs) { List> seq = freqSeq.javaSequence(); long freq = freqSeq.freq(); } } + + @Test + public void runPrefixSpanSaveLoad() { + JavaRDD>> sequences = jsc.parallelize(Arrays.asList( + Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3)), + Arrays.asList(Arrays.asList(1), Arrays.asList(3, 2), Arrays.asList(1, 2)), + Arrays.asList(Arrays.asList(1, 2), Arrays.asList(5)), + Arrays.asList(Arrays.asList(6)) + ), 2); + PrefixSpan prefixSpan = new PrefixSpan() + .setMinSupport(0.5) + .setMaxPatternLength(5); + PrefixSpanModel model = prefixSpan.run(sequences); + + File tempDir = Utils.createTempDir( + System.getProperty("java.io.tmpdir"), "JavaPrefixSpanSuite"); + String outputPath = tempDir.getPath(); + + try { + model.save(spark.sparkContext(), outputPath); + @SuppressWarnings("unchecked") + PrefixSpanModel newModel = + (PrefixSpanModel) PrefixSpanModel.load(spark.sparkContext(), outputPath); + JavaRDD> freqSeqs = newModel.freqSequences().toJavaRDD(); + List> localFreqSeqs = freqSeqs.collect(); + Assert.assertEquals(5, localFreqSeqs.size()); + // Check that each frequent sequence could be materialized. + for (PrefixSpan.FreqSequence freqSeq : localFreqSeqs) { + List> seq = freqSeq.javaSequence(); + long freq = freqSeq.freq(); + } + } finally { + Utils.deleteRecursively(tempDir); + } + + + } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java index 8beea102efd0..f427846b9ad1 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java @@ -17,147 +17,148 @@ package org.apache.spark.mllib.linalg; -import static org.junit.Assert.*; -import org.junit.Test; - -import java.io.Serializable; import java.util.Random; -public class JavaMatricesSuite implements Serializable { - - @Test - public void randMatrixConstruction() { - Random rng = new Random(24); - Matrix r = Matrices.rand(3, 4, rng); - rng.setSeed(24); - DenseMatrix dr = DenseMatrix.rand(3, 4, rng); - assertArrayEquals(r.toArray(), dr.toArray(), 0.0); - - rng.setSeed(24); - Matrix rn = Matrices.randn(3, 4, rng); - rng.setSeed(24); - DenseMatrix drn = DenseMatrix.randn(3, 4, rng); - assertArrayEquals(rn.toArray(), drn.toArray(), 0.0); - - rng.setSeed(24); - Matrix s = Matrices.sprand(3, 4, 0.5, rng); - rng.setSeed(24); - SparseMatrix sr = SparseMatrix.sprand(3, 4, 0.5, rng); - assertArrayEquals(s.toArray(), sr.toArray(), 0.0); - - rng.setSeed(24); - Matrix sn = Matrices.sprandn(3, 4, 0.5, rng); - rng.setSeed(24); - SparseMatrix srn = SparseMatrix.sprandn(3, 4, 0.5, rng); - assertArrayEquals(sn.toArray(), srn.toArray(), 0.0); - } - - @Test - public void identityMatrixConstruction() { - Matrix r = Matrices.eye(2); - DenseMatrix dr = DenseMatrix.eye(2); - SparseMatrix sr = SparseMatrix.speye(2); - assertArrayEquals(r.toArray(), dr.toArray(), 0.0); - assertArrayEquals(sr.toArray(), dr.toArray(), 0.0); - assertArrayEquals(r.toArray(), new double[]{1.0, 0.0, 0.0, 1.0}, 0.0); - } - - @Test - public void diagonalMatrixConstruction() { - Vector v = Vectors.dense(1.0, 0.0, 2.0); - Vector sv = Vectors.sparse(3, new int[]{0, 2}, new double[]{1.0, 2.0}); - - Matrix m = Matrices.diag(v); - Matrix sm = Matrices.diag(sv); - DenseMatrix d = DenseMatrix.diag(v); - DenseMatrix sd = DenseMatrix.diag(sv); - SparseMatrix s = SparseMatrix.spdiag(v); - SparseMatrix ss = SparseMatrix.spdiag(sv); - - assertArrayEquals(m.toArray(), sm.toArray(), 0.0); - assertArrayEquals(d.toArray(), sm.toArray(), 0.0); - assertArrayEquals(d.toArray(), sd.toArray(), 0.0); - assertArrayEquals(sd.toArray(), s.toArray(), 0.0); - assertArrayEquals(s.toArray(), ss.toArray(), 0.0); - assertArrayEquals(s.values(), ss.values(), 0.0); - assertEquals(2, s.values().length); - assertEquals(2, ss.values().length); - assertEquals(4, s.colPtrs().length); - assertEquals(4, ss.colPtrs().length); - } - - @Test - public void zerosMatrixConstruction() { - Matrix z = Matrices.zeros(2, 2); - Matrix one = Matrices.ones(2, 2); - DenseMatrix dz = DenseMatrix.zeros(2, 2); - DenseMatrix done = DenseMatrix.ones(2, 2); - - assertArrayEquals(z.toArray(), new double[]{0.0, 0.0, 0.0, 0.0}, 0.0); - assertArrayEquals(dz.toArray(), new double[]{0.0, 0.0, 0.0, 0.0}, 0.0); - assertArrayEquals(one.toArray(), new double[]{1.0, 1.0, 1.0, 1.0}, 0.0); - assertArrayEquals(done.toArray(), new double[]{1.0, 1.0, 1.0, 1.0}, 0.0); - } - - @Test - public void sparseDenseConversion() { - int m = 3; - int n = 2; - double[] values = new double[]{1.0, 2.0, 4.0, 5.0}; - double[] allValues = new double[]{1.0, 2.0, 0.0, 0.0, 4.0, 5.0}; - int[] colPtrs = new int[]{0, 2, 4}; - int[] rowIndices = new int[]{0, 1, 1, 2}; - - SparseMatrix spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values); - DenseMatrix deMat1 = new DenseMatrix(m, n, allValues); - - SparseMatrix spMat2 = deMat1.toSparse(); - DenseMatrix deMat2 = spMat1.toDense(); - - assertArrayEquals(spMat1.toArray(), spMat2.toArray(), 0.0); - assertArrayEquals(deMat1.toArray(), deMat2.toArray(), 0.0); - } - - @Test - public void concatenateMatrices() { - int m = 3; - int n = 2; - - Random rng = new Random(42); - SparseMatrix spMat1 = SparseMatrix.sprand(m, n, 0.5, rng); - rng.setSeed(42); - DenseMatrix deMat1 = DenseMatrix.rand(m, n, rng); - Matrix deMat2 = Matrices.eye(3); - Matrix spMat2 = Matrices.speye(3); - Matrix deMat3 = Matrices.eye(2); - Matrix spMat3 = Matrices.speye(2); - - Matrix spHorz = Matrices.horzcat(new Matrix[]{spMat1, spMat2}); - Matrix deHorz1 = Matrices.horzcat(new Matrix[]{deMat1, deMat2}); - Matrix deHorz2 = Matrices.horzcat(new Matrix[]{spMat1, deMat2}); - Matrix deHorz3 = Matrices.horzcat(new Matrix[]{deMat1, spMat2}); - - assertEquals(3, deHorz1.numRows()); - assertEquals(3, deHorz2.numRows()); - assertEquals(3, deHorz3.numRows()); - assertEquals(3, spHorz.numRows()); - assertEquals(5, deHorz1.numCols()); - assertEquals(5, deHorz2.numCols()); - assertEquals(5, deHorz3.numCols()); - assertEquals(5, spHorz.numCols()); - - Matrix spVert = Matrices.vertcat(new Matrix[]{spMat1, spMat3}); - Matrix deVert1 = Matrices.vertcat(new Matrix[]{deMat1, deMat3}); - Matrix deVert2 = Matrices.vertcat(new Matrix[]{spMat1, deMat3}); - Matrix deVert3 = Matrices.vertcat(new Matrix[]{deMat1, spMat3}); - - assertEquals(5, deVert1.numRows()); - assertEquals(5, deVert2.numRows()); - assertEquals(5, deVert3.numRows()); - assertEquals(5, spVert.numRows()); - assertEquals(2, deVert1.numCols()); - assertEquals(2, deVert2.numCols()); - assertEquals(2, deVert3.numCols()); - assertEquals(2, spVert.numCols()); - } +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +public class JavaMatricesSuite { + + @Test + public void randMatrixConstruction() { + Random rng = new Random(24); + Matrix r = Matrices.rand(3, 4, rng); + rng.setSeed(24); + DenseMatrix dr = DenseMatrix.rand(3, 4, rng); + assertArrayEquals(r.toArray(), dr.toArray(), 0.0); + + rng.setSeed(24); + Matrix rn = Matrices.randn(3, 4, rng); + rng.setSeed(24); + DenseMatrix drn = DenseMatrix.randn(3, 4, rng); + assertArrayEquals(rn.toArray(), drn.toArray(), 0.0); + + rng.setSeed(24); + Matrix s = Matrices.sprand(3, 4, 0.5, rng); + rng.setSeed(24); + SparseMatrix sr = SparseMatrix.sprand(3, 4, 0.5, rng); + assertArrayEquals(s.toArray(), sr.toArray(), 0.0); + + rng.setSeed(24); + Matrix sn = Matrices.sprandn(3, 4, 0.5, rng); + rng.setSeed(24); + SparseMatrix srn = SparseMatrix.sprandn(3, 4, 0.5, rng); + assertArrayEquals(sn.toArray(), srn.toArray(), 0.0); + } + + @Test + public void identityMatrixConstruction() { + Matrix r = Matrices.eye(2); + DenseMatrix dr = DenseMatrix.eye(2); + SparseMatrix sr = SparseMatrix.speye(2); + assertArrayEquals(r.toArray(), dr.toArray(), 0.0); + assertArrayEquals(sr.toArray(), dr.toArray(), 0.0); + assertArrayEquals(r.toArray(), new double[]{1.0, 0.0, 0.0, 1.0}, 0.0); + } + + @Test + public void diagonalMatrixConstruction() { + Vector v = Vectors.dense(1.0, 0.0, 2.0); + Vector sv = Vectors.sparse(3, new int[]{0, 2}, new double[]{1.0, 2.0}); + + Matrix m = Matrices.diag(v); + Matrix sm = Matrices.diag(sv); + DenseMatrix d = DenseMatrix.diag(v); + DenseMatrix sd = DenseMatrix.diag(sv); + SparseMatrix s = SparseMatrix.spdiag(v); + SparseMatrix ss = SparseMatrix.spdiag(sv); + + assertArrayEquals(m.toArray(), sm.toArray(), 0.0); + assertArrayEquals(d.toArray(), sm.toArray(), 0.0); + assertArrayEquals(d.toArray(), sd.toArray(), 0.0); + assertArrayEquals(sd.toArray(), s.toArray(), 0.0); + assertArrayEquals(s.toArray(), ss.toArray(), 0.0); + assertArrayEquals(s.values(), ss.values(), 0.0); + assertEquals(2, s.values().length); + assertEquals(2, ss.values().length); + assertEquals(4, s.colPtrs().length); + assertEquals(4, ss.colPtrs().length); + } + + @Test + public void zerosMatrixConstruction() { + Matrix z = Matrices.zeros(2, 2); + Matrix one = Matrices.ones(2, 2); + DenseMatrix dz = DenseMatrix.zeros(2, 2); + DenseMatrix done = DenseMatrix.ones(2, 2); + + assertArrayEquals(z.toArray(), new double[]{0.0, 0.0, 0.0, 0.0}, 0.0); + assertArrayEquals(dz.toArray(), new double[]{0.0, 0.0, 0.0, 0.0}, 0.0); + assertArrayEquals(one.toArray(), new double[]{1.0, 1.0, 1.0, 1.0}, 0.0); + assertArrayEquals(done.toArray(), new double[]{1.0, 1.0, 1.0, 1.0}, 0.0); + } + + @Test + public void sparseDenseConversion() { + int m = 3; + int n = 2; + double[] values = new double[]{1.0, 2.0, 4.0, 5.0}; + double[] allValues = new double[]{1.0, 2.0, 0.0, 0.0, 4.0, 5.0}; + int[] colPtrs = new int[]{0, 2, 4}; + int[] rowIndices = new int[]{0, 1, 1, 2}; + + SparseMatrix spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values); + DenseMatrix deMat1 = new DenseMatrix(m, n, allValues); + + SparseMatrix spMat2 = deMat1.toSparse(); + DenseMatrix deMat2 = spMat1.toDense(); + + assertArrayEquals(spMat1.toArray(), spMat2.toArray(), 0.0); + assertArrayEquals(deMat1.toArray(), deMat2.toArray(), 0.0); + } + + @Test + public void concatenateMatrices() { + int m = 3; + int n = 2; + + Random rng = new Random(42); + SparseMatrix spMat1 = SparseMatrix.sprand(m, n, 0.5, rng); + rng.setSeed(42); + DenseMatrix deMat1 = DenseMatrix.rand(m, n, rng); + Matrix deMat2 = Matrices.eye(3); + Matrix spMat2 = Matrices.speye(3); + Matrix deMat3 = Matrices.eye(2); + Matrix spMat3 = Matrices.speye(2); + + Matrix spHorz = Matrices.horzcat(new Matrix[]{spMat1, spMat2}); + Matrix deHorz1 = Matrices.horzcat(new Matrix[]{deMat1, deMat2}); + Matrix deHorz2 = Matrices.horzcat(new Matrix[]{spMat1, deMat2}); + Matrix deHorz3 = Matrices.horzcat(new Matrix[]{deMat1, spMat2}); + + assertEquals(3, deHorz1.numRows()); + assertEquals(3, deHorz2.numRows()); + assertEquals(3, deHorz3.numRows()); + assertEquals(3, spHorz.numRows()); + assertEquals(5, deHorz1.numCols()); + assertEquals(5, deHorz2.numCols()); + assertEquals(5, deHorz3.numCols()); + assertEquals(5, spHorz.numCols()); + + Matrix spVert = Matrices.vertcat(new Matrix[]{spMat1, spMat3}); + Matrix deVert1 = Matrices.vertcat(new Matrix[]{deMat1, deMat3}); + Matrix deVert2 = Matrices.vertcat(new Matrix[]{spMat1, deMat3}); + Matrix deVert3 = Matrices.vertcat(new Matrix[]{deMat1, spMat3}); + + assertEquals(5, deVert1.numRows()); + assertEquals(5, deVert2.numRows()); + assertEquals(5, deVert3.numRows()); + assertEquals(5, spVert.numRows()); + assertEquals(2, deVert1.numCols()); + assertEquals(2, deVert2.numCols()); + assertEquals(2, deVert3.numCols()); + assertEquals(2, spVert.numCols()); + } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java index 77c8c6274f37..f67f555e418a 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java @@ -17,15 +17,15 @@ package org.apache.spark.mllib.linalg; -import java.io.Serializable; import java.util.Arrays; +import static org.junit.Assert.assertArrayEquals; + import scala.Tuple2; import org.junit.Test; -import static org.junit.Assert.*; -public class JavaVectorsSuite implements Serializable { +public class JavaVectorsSuite { @Test public void denseArrayConstruction() { @@ -37,8 +37,8 @@ public void denseArrayConstruction() { public void sparseArrayConstruction() { @SuppressWarnings("unchecked") Vector v = Vectors.sparse(3, Arrays.asList( - new Tuple2(0, 2.0), - new Tuple2(2, 3.0))); + new Tuple2<>(0, 2.0), + new Tuple2<>(2, 3.0))); assertArrayEquals(new double[]{2.0, 0.0, 3.0}, v.toArray(), 0.0); } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/distributed/JavaRowMatrixSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/distributed/JavaRowMatrixSuite.java new file mode 100644 index 000000000000..c01af405491b --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/distributed/JavaRowMatrixSuite.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg.distributed; + +import java.util.Arrays; + +import org.junit.Test; + +import org.apache.spark.SharedSparkSession; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.QRDecomposition; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; + +public class JavaRowMatrixSuite extends SharedSparkSession { + + @Test + public void rowMatrixQRDecomposition() { + Vector v1 = Vectors.dense(1.0, 10.0, 100.0); + Vector v2 = Vectors.dense(2.0, 20.0, 200.0); + Vector v3 = Vectors.dense(3.0, 30.0, 300.0); + + JavaRDD rows = jsc.parallelize(Arrays.asList(v1, v2, v3), 1); + RowMatrix mat = new RowMatrix(rows.rdd()); + + QRDecomposition result = mat.tallSkinnyQR(true); + } +} diff --git a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java index 5728df5aeebd..6d114024c31b 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java @@ -20,40 +20,26 @@ import java.io.Serializable; import java.util.Arrays; -import org.apache.spark.api.java.JavaRDD; import org.junit.Assert; -import org.junit.After; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaDoubleRDD; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.mllib.linalg.Vector; import static org.apache.spark.mllib.random.RandomRDDs.*; -public class JavaRandomRDDsSuite { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaRandomRDDsSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaRandomRDDsSuite extends SharedSparkSession { @Test public void testUniformRDD() { long m = 1000L; int p = 2; long seed = 1L; - JavaDoubleRDD rdd1 = uniformJavaRDD(sc, m); - JavaDoubleRDD rdd2 = uniformJavaRDD(sc, m, p); - JavaDoubleRDD rdd3 = uniformJavaRDD(sc, m, p, seed); - for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaDoubleRDD rdd1 = uniformJavaRDD(jsc, m); + JavaDoubleRDD rdd2 = uniformJavaRDD(jsc, m, p); + JavaDoubleRDD rdd3 = uniformJavaRDD(jsc, m, p, seed); + for (JavaDoubleRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -63,10 +49,10 @@ public void testNormalRDD() { long m = 1000L; int p = 2; long seed = 1L; - JavaDoubleRDD rdd1 = normalJavaRDD(sc, m); - JavaDoubleRDD rdd2 = normalJavaRDD(sc, m, p); - JavaDoubleRDD rdd3 = normalJavaRDD(sc, m, p, seed); - for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaDoubleRDD rdd1 = normalJavaRDD(jsc, m); + JavaDoubleRDD rdd2 = normalJavaRDD(jsc, m, p); + JavaDoubleRDD rdd3 = normalJavaRDD(jsc, m, p, seed); + for (JavaDoubleRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -78,10 +64,10 @@ public void testLNormalRDD() { long m = 1000L; int p = 2; long seed = 1L; - JavaDoubleRDD rdd1 = logNormalJavaRDD(sc, mean, std, m); - JavaDoubleRDD rdd2 = logNormalJavaRDD(sc, mean, std, m, p); - JavaDoubleRDD rdd3 = logNormalJavaRDD(sc, mean, std, m, p, seed); - for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaDoubleRDD rdd1 = logNormalJavaRDD(jsc, mean, std, m); + JavaDoubleRDD rdd2 = logNormalJavaRDD(jsc, mean, std, m, p); + JavaDoubleRDD rdd3 = logNormalJavaRDD(jsc, mean, std, m, p, seed); + for (JavaDoubleRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -92,10 +78,10 @@ public void testPoissonRDD() { long m = 1000L; int p = 2; long seed = 1L; - JavaDoubleRDD rdd1 = poissonJavaRDD(sc, mean, m); - JavaDoubleRDD rdd2 = poissonJavaRDD(sc, mean, m, p); - JavaDoubleRDD rdd3 = poissonJavaRDD(sc, mean, m, p, seed); - for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaDoubleRDD rdd1 = poissonJavaRDD(jsc, mean, m); + JavaDoubleRDD rdd2 = poissonJavaRDD(jsc, mean, m, p); + JavaDoubleRDD rdd3 = poissonJavaRDD(jsc, mean, m, p, seed); + for (JavaDoubleRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -106,10 +92,10 @@ public void testExponentialRDD() { long m = 1000L; int p = 2; long seed = 1L; - JavaDoubleRDD rdd1 = exponentialJavaRDD(sc, mean, m); - JavaDoubleRDD rdd2 = exponentialJavaRDD(sc, mean, m, p); - JavaDoubleRDD rdd3 = exponentialJavaRDD(sc, mean, m, p, seed); - for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaDoubleRDD rdd1 = exponentialJavaRDD(jsc, mean, m); + JavaDoubleRDD rdd2 = exponentialJavaRDD(jsc, mean, m, p); + JavaDoubleRDD rdd3 = exponentialJavaRDD(jsc, mean, m, p, seed); + for (JavaDoubleRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -117,14 +103,14 @@ public void testExponentialRDD() { @Test public void testGammaRDD() { double shape = 1.0; - double scale = 2.0; + double jscale = 2.0; long m = 1000L; int p = 2; long seed = 1L; - JavaDoubleRDD rdd1 = gammaJavaRDD(sc, shape, scale, m); - JavaDoubleRDD rdd2 = gammaJavaRDD(sc, shape, scale, m, p); - JavaDoubleRDD rdd3 = gammaJavaRDD(sc, shape, scale, m, p, seed); - for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaDoubleRDD rdd1 = gammaJavaRDD(jsc, shape, jscale, m); + JavaDoubleRDD rdd2 = gammaJavaRDD(jsc, shape, jscale, m, p); + JavaDoubleRDD rdd3 = gammaJavaRDD(jsc, shape, jscale, m, p, seed); + for (JavaDoubleRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -137,10 +123,10 @@ public void testUniformVectorRDD() { int n = 10; int p = 2; long seed = 1L; - JavaRDD rdd1 = uniformJavaVectorRDD(sc, m, n); - JavaRDD rdd2 = uniformJavaVectorRDD(sc, m, n, p); - JavaRDD rdd3 = uniformJavaVectorRDD(sc, m, n, p, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = uniformJavaVectorRDD(jsc, m, n); + JavaRDD rdd2 = uniformJavaVectorRDD(jsc, m, n, p); + JavaRDD rdd3 = uniformJavaVectorRDD(jsc, m, n, p, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -153,10 +139,10 @@ public void testNormalVectorRDD() { int n = 10; int p = 2; long seed = 1L; - JavaRDD rdd1 = normalJavaVectorRDD(sc, m, n); - JavaRDD rdd2 = normalJavaVectorRDD(sc, m, n, p); - JavaRDD rdd3 = normalJavaVectorRDD(sc, m, n, p, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = normalJavaVectorRDD(jsc, m, n); + JavaRDD rdd2 = normalJavaVectorRDD(jsc, m, n, p); + JavaRDD rdd3 = normalJavaVectorRDD(jsc, m, n, p, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -166,15 +152,15 @@ public void testNormalVectorRDD() { @SuppressWarnings("unchecked") public void testLogNormalVectorRDD() { double mean = 4.0; - double std = 2.0; + double std = 2.0; long m = 100L; int n = 10; int p = 2; long seed = 1L; - JavaRDD rdd1 = logNormalJavaVectorRDD(sc, mean, std, m, n); - JavaRDD rdd2 = logNormalJavaVectorRDD(sc, mean, std, m, n, p); - JavaRDD rdd3 = logNormalJavaVectorRDD(sc, mean, std, m, n, p, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = logNormalJavaVectorRDD(jsc, mean, std, m, n); + JavaRDD rdd2 = logNormalJavaVectorRDD(jsc, mean, std, m, n, p); + JavaRDD rdd3 = logNormalJavaVectorRDD(jsc, mean, std, m, n, p, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -188,10 +174,10 @@ public void testPoissonVectorRDD() { int n = 10; int p = 2; long seed = 1L; - JavaRDD rdd1 = poissonJavaVectorRDD(sc, mean, m, n); - JavaRDD rdd2 = poissonJavaVectorRDD(sc, mean, m, n, p); - JavaRDD rdd3 = poissonJavaVectorRDD(sc, mean, m, n, p, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = poissonJavaVectorRDD(jsc, mean, m, n); + JavaRDD rdd2 = poissonJavaVectorRDD(jsc, mean, m, n, p); + JavaRDD rdd3 = poissonJavaVectorRDD(jsc, mean, m, n, p, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -205,10 +191,10 @@ public void testExponentialVectorRDD() { int n = 10; int p = 2; long seed = 1L; - JavaRDD rdd1 = exponentialJavaVectorRDD(sc, mean, m, n); - JavaRDD rdd2 = exponentialJavaVectorRDD(sc, mean, m, n, p); - JavaRDD rdd3 = exponentialJavaVectorRDD(sc, mean, m, n, p, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = exponentialJavaVectorRDD(jsc, mean, m, n); + JavaRDD rdd2 = exponentialJavaVectorRDD(jsc, mean, m, n, p); + JavaRDD rdd3 = exponentialJavaVectorRDD(jsc, mean, m, n, p, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -218,15 +204,15 @@ public void testExponentialVectorRDD() { @SuppressWarnings("unchecked") public void testGammaVectorRDD() { double shape = 1.0; - double scale = 2.0; + double jscale = 2.0; long m = 100L; int n = 10; int p = 2; long seed = 1L; - JavaRDD rdd1 = gammaJavaVectorRDD(sc, shape, scale, m, n); - JavaRDD rdd2 = gammaJavaVectorRDD(sc, shape, scale, m, n, p); - JavaRDD rdd3 = gammaJavaVectorRDD(sc, shape, scale, m, n, p, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = gammaJavaVectorRDD(jsc, shape, jscale, m, n); + JavaRDD rdd2 = gammaJavaVectorRDD(jsc, shape, jscale, m, n, p); + JavaRDD rdd3 = gammaJavaVectorRDD(jsc, shape, jscale, m, n, p, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -238,10 +224,10 @@ public void testArbitrary() { long seed = 1L; int numPartitions = 0; StringGenerator gen = new StringGenerator(); - JavaRDD rdd1 = randomJavaRDD(sc, gen, size); - JavaRDD rdd2 = randomJavaRDD(sc, gen, size, numPartitions); - JavaRDD rdd3 = randomJavaRDD(sc, gen, size, numPartitions, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = randomJavaRDD(jsc, gen, size); + JavaRDD rdd2 = randomJavaRDD(jsc, gen, size, numPartitions); + JavaRDD rdd3 = randomJavaRDD(jsc, gen, size, numPartitions, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(size, rdd.count()); Assert.assertEquals(2, rdd.first().length()); } @@ -255,10 +241,10 @@ public void testRandomVectorRDD() { int n = 10; int p = 2; long seed = 1L; - JavaRDD rdd1 = randomJavaVectorRDD(sc, generator, m, n); - JavaRDD rdd2 = randomJavaVectorRDD(sc, generator, m, n, p); - JavaRDD rdd3 = randomJavaVectorRDD(sc, generator, m, n, p, seed); - for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { + JavaRDD rdd1 = randomJavaVectorRDD(jsc, generator, m, n); + JavaRDD rdd2 = randomJavaVectorRDD(jsc, generator, m, n, p); + JavaRDD rdd3 = randomJavaVectorRDD(jsc, generator, m, n, p, seed); + for (JavaRDD rdd : Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -271,10 +257,12 @@ class StringGenerator implements RandomDataGenerator, Serializable { public String nextValue() { return "42"; } + @Override public StringGenerator copy() { return new StringGenerator(); } + @Override public void setSeed(long seed) { } diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java index 271dda4662e0..363ab42546d1 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java @@ -17,58 +17,42 @@ package org.apache.spark.mllib.recommendation; -import java.io.Serializable; import java.util.ArrayList; import java.util.List; import scala.Tuple2; import scala.Tuple3; -import org.jblas.DoubleMatrix; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class JavaALSSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaALS"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } - - void validatePrediction( - MatrixFactorizationModel model, - int users, - int products, - DoubleMatrix trueRatings, - double matchThreshold, - boolean implicitPrefs, - DoubleMatrix truePrefs) { - List> localUsersProducts = new ArrayList(users * products); - for (int u=0; u < users; ++u) { - for (int p=0; p < products; ++p) { - localUsersProducts.add(new Tuple2(u, p)); +public class JavaALSSuite extends SharedSparkSession { + + private void validatePrediction( + MatrixFactorizationModel model, + int users, + int products, + double[] trueRatings, + double matchThreshold, + boolean implicitPrefs, + double[] truePrefs) { + List> localUsersProducts = new ArrayList<>(users * products); + for (int u = 0; u < users; ++u) { + for (int p = 0; p < products; ++p) { + localUsersProducts.add(new Tuple2<>(u, p)); } } - JavaPairRDD usersProducts = sc.parallelizePairs(localUsersProducts); + JavaPairRDD usersProducts = jsc.parallelizePairs(localUsersProducts); List predictedRatings = model.predict(usersProducts).collect(); Assert.assertEquals(users * products, predictedRatings.size()); if (!implicitPrefs) { - for (Rating r: predictedRatings) { + for (Rating r : predictedRatings) { double prediction = r.rating(); - double correct = trueRatings.get(r.user(), r.product()); + double correct = trueRatings[r.product() * users + r.user()]; Assert.assertTrue(String.format("Prediction=%2.4f not below match threshold of %2.2f", prediction, matchThreshold), Math.abs(prediction - correct) < matchThreshold); } @@ -77,11 +61,11 @@ void validatePrediction( // (ref Mahout's implicit ALS tests) double sqErr = 0.0; double denom = 0.0; - for (Rating r: predictedRatings) { + for (Rating r : predictedRatings) { double prediction = r.rating(); - double truePref = truePrefs.get(r.user(), r.product()); + double truePref = truePrefs[r.product() * users + r.user()]; double confidence = 1.0 + - /* alpha = */ 1.0 * Math.abs(trueRatings.get(r.user(), r.product())); + /* alpha = 1.0 * ... */ Math.abs(trueRatings[r.product() * users + r.user()]); double err = confidence * (truePref - prediction) * (truePref - prediction); sqErr += err; denom += confidence; @@ -98,10 +82,10 @@ public void runALSUsingStaticMethods() { int iterations = 15; int users = 50; int products = 100; - Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( - users, products, features, 0.7, false, false); + Tuple3, double[], double[]> testData = + ALSSuite.generateRatingsAsJava(users, products, features, 0.7, false, false); - JavaRDD data = sc.parallelize(testData._1()); + JavaRDD data = jsc.parallelize(testData._1()); MatrixFactorizationModel model = ALS.train(data.rdd(), features, iterations); validatePrediction(model, users, products, testData._2(), 0.3, false, testData._3()); } @@ -112,10 +96,10 @@ public void runALSUsingConstructor() { int iterations = 15; int users = 100; int products = 200; - Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( - users, products, features, 0.7, false, false); + Tuple3, double[], double[]> testData = + ALSSuite.generateRatingsAsJava(users, products, features, 0.7, false, false); - JavaRDD data = sc.parallelize(testData._1()); + JavaRDD data = jsc.parallelize(testData._1()); MatrixFactorizationModel model = new ALS().setRank(features) .setIterations(iterations) @@ -129,10 +113,10 @@ public void runImplicitALSUsingStaticMethods() { int iterations = 15; int users = 80; int products = 160; - Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( - users, products, features, 0.7, true, false); + Tuple3, double[], double[]> testData = + ALSSuite.generateRatingsAsJava(users, products, features, 0.7, true, false); - JavaRDD data = sc.parallelize(testData._1()); + JavaRDD data = jsc.parallelize(testData._1()); MatrixFactorizationModel model = ALS.trainImplicit(data.rdd(), features, iterations); validatePrediction(model, users, products, testData._2(), 0.4, true, testData._3()); } @@ -143,10 +127,10 @@ public void runImplicitALSUsingConstructor() { int iterations = 15; int users = 100; int products = 200; - Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( - users, products, features, 0.7, true, false); + Tuple3, double[], double[]> testData = + ALSSuite.generateRatingsAsJava(users, products, features, 0.7, true, false); - JavaRDD data = sc.parallelize(testData._1()); + JavaRDD data = jsc.parallelize(testData._1()); MatrixFactorizationModel model = new ALS().setRank(features) .setIterations(iterations) @@ -161,10 +145,10 @@ public void runImplicitALSWithNegativeWeight() { int iterations = 15; int users = 80; int products = 160; - Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( - users, products, features, 0.7, true, true); + Tuple3, double[], double[]> testData = + ALSSuite.generateRatingsAsJava(users, products, features, 0.7, true, true); - JavaRDD data = sc.parallelize(testData._1()); + JavaRDD data = jsc.parallelize(testData._1()); MatrixFactorizationModel model = new ALS().setRank(features) .setIterations(iterations) .setImplicitPrefs(true) @@ -179,9 +163,9 @@ public void runRecommend() { int iterations = 10; int users = 200; int products = 50; - Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( - users, products, features, 0.7, true, false); - JavaRDD data = sc.parallelize(testData._1()); + List testData = ALSSuite.generateRatingsAsJava( + users, products, features, 0.7, true, false)._1(); + JavaRDD data = jsc.parallelize(testData); MatrixFactorizationModel model = new ALS().setRank(features) .setIterations(iterations) .setImplicitPrefs(true) @@ -194,7 +178,7 @@ public void runRecommend() { private static void validateRecommendations(Rating[] recommendations, int howMany) { Assert.assertEquals(howMany, recommendations.length); for (int i = 1; i < recommendations.length; i++) { - Assert.assertTrue(recommendations[i-1].rating() >= recommendations[i].rating()); + Assert.assertTrue(recommendations[i - 1].rating() >= recommendations[i].rating()); } Assert.assertTrue(recommendations[0].rating() > 0.7); } diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java index 32c2f4f3395b..dbd4cbfd2b74 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java @@ -17,30 +17,26 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import scala.Tuple3; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaDoubleRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -public class JavaIsotonicRegressionSuite implements Serializable { - private transient JavaSparkContext sc; +public class JavaIsotonicRegressionSuite extends SharedSparkSession { - private List> generateIsotonicInput(double[] labels) { - ArrayList> input = new ArrayList(labels.length); + private static List> generateIsotonicInput(double[] labels) { + List> input = new ArrayList<>(labels.length); for (int i = 1; i <= labels.length; i++) { - input.add(new Tuple3(labels[i-1], (double) i, 1d)); + input.add(new Tuple3<>(labels[i - 1], (double) i, 1.0)); } return input; @@ -48,29 +44,18 @@ private List> generateIsotonicInput(double[] labe private IsotonicRegressionModel runIsotonicRegression(double[] labels) { JavaRDD> trainRDD = - sc.parallelize(generateIsotonicInput(labels), 2).cache(); + jsc.parallelize(generateIsotonicInput(labels), 2).cache(); return new IsotonicRegression().run(trainRDD); } - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } - @Test public void testIsotonicRegressionJavaRDD() { IsotonicRegressionModel model = runIsotonicRegression(new double[]{1, 2, 3, 3, 1, 6, 7, 8, 11, 9, 10, 12}); Assert.assertArrayEquals( - new double[] {1, 2, 7d/3, 7d/3, 6, 7, 8, 10, 10, 12}, model.predictions(), 1e-14); + new double[]{1, 2, 7.0 / 3, 7.0 / 3, 6, 7, 8, 10, 10, 12}, model.predictions(), 1.0e-14); } @Test @@ -78,13 +63,13 @@ public void testIsotonicRegressionPredictionsJavaRDD() { IsotonicRegressionModel model = runIsotonicRegression(new double[]{1, 2, 3, 3, 1, 6, 7, 8, 11, 9, 10, 12}); - JavaDoubleRDD testRDD = sc.parallelizeDoubles(Arrays.asList(0.0, 1.0, 9.5, 12.0, 13.0)); + JavaDoubleRDD testRDD = jsc.parallelizeDoubles(Arrays.asList(0.0, 1.0, 9.5, 12.0, 13.0)); List predictions = model.predict(testRDD).collect(); - Assert.assertTrue(predictions.get(0) == 1d); - Assert.assertTrue(predictions.get(1) == 1d); - Assert.assertTrue(predictions.get(2) == 10d); - Assert.assertTrue(predictions.get(3) == 12d); - Assert.assertTrue(predictions.get(4) == 12d); + Assert.assertEquals(1.0, predictions.get(0).doubleValue(), 1.0e-14); + Assert.assertEquals(1.0, predictions.get(1).doubleValue(), 1.0e-14); + Assert.assertEquals(10.0, predictions.get(2).doubleValue(), 1.0e-14); + Assert.assertEquals(12.0, predictions.get(3).doubleValue(), 1.0e-14); + Assert.assertEquals(12.0, predictions.get(4).doubleValue(), 1.0e-14); } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java index 8950b48888b7..1458cc72bc17 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java @@ -17,35 +17,20 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.util.LinearDataGenerator; -public class JavaLassoSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaLassoSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } +public class JavaLassoSuite extends SharedSparkSession { int validatePrediction(List validationData, LassoModel model) { int numAccurate = 0; - for (LabeledPoint point: validationData) { + for (LabeledPoint point : validationData) { Double prediction = model.predict(point.features()); // A prediction is off if the prediction is more than 0.5 away from expected value. if (Math.abs(prediction - point.label()) <= 0.5) { @@ -61,15 +46,15 @@ public void runLassoUsingConstructor() { double A = 0.0; double[] weights = {-1.5, 1.0e-2}; - JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, - weights, nPoints, 42, 0.1), 2).cache(); + JavaRDD testRDD = jsc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, + weights, nPoints, 42, 0.1), 2).cache(); List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); LassoWithSGD lassoSGDImpl = new LassoWithSGD(); lassoSGDImpl.optimizer().setStepSize(1.0) - .setRegParam(0.01) - .setNumIterations(20); + .setRegParam(0.01) + .setNumIterations(20); LassoModel model = lassoSGDImpl.run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); @@ -82,10 +67,10 @@ public void runLassoUsingStaticMethods() { double A = 0.0; double[] weights = {-1.5, 1.0e-2}; - JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, - weights, nPoints, 42, 0.1), 2).cache(); + JavaRDD testRDD = jsc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, + weights, nPoints, 42, 0.1), 2).cache(); List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); LassoModel model = LassoWithSGD.train(testRDD.rdd(), 100, 1.0, 0.01, 1.0); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java index 24c4c20d9af1..86c723aa0074 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -17,42 +17,27 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.function.Function; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.util.LinearDataGenerator; -public class JavaLinearRegressionSuite implements Serializable { - private transient JavaSparkContext sc; +public class JavaLinearRegressionSuite extends SharedSparkSession { - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } - - int validatePrediction(List validationData, LinearRegressionModel model) { + private static int validatePrediction( + List validationData, LinearRegressionModel model) { int numAccurate = 0; - for (LabeledPoint point: validationData) { - Double prediction = model.predict(point.features()); - // A prediction is off if the prediction is more than 0.5 away from expected value. - if (Math.abs(prediction - point.label()) <= 0.5) { - numAccurate++; - } + for (LabeledPoint point : validationData) { + Double prediction = model.predict(point.features()); + // A prediction is off if the prediction is more than 0.5 away from expected value. + if (Math.abs(prediction - point.label()) <= 0.5) { + numAccurate++; + } } return numAccurate; } @@ -63,10 +48,10 @@ public void runLinearRegressionUsingConstructor() { double A = 3.0; double[] weights = {10, 10}; - JavaRDD testRDD = sc.parallelize( - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); + JavaRDD testRDD = jsc.parallelize( + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); linSGDImpl.setIntercept(true); @@ -82,10 +67,10 @@ public void runLinearRegressionUsingStaticMethods() { double A = 0.0; double[] weights = {10, 10}; - JavaRDD testRDD = sc.parallelize( - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); + JavaRDD testRDD = jsc.parallelize( + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); LinearRegressionModel model = LinearRegressionWithSGD.train(testRDD.rdd(), 100); @@ -98,16 +83,11 @@ public void testPredictJavaRDD() { int nPoints = 100; double A = 0.0; double[] weights = {10, 10}; - JavaRDD testRDD = sc.parallelize( + JavaRDD testRDD = jsc.parallelize( LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); LinearRegressionModel model = linSGDImpl.run(testRDD.rdd()); - JavaRDD vectors = testRDD.map(new Function() { - @Override - public Vector call(LabeledPoint v) throws Exception { - return v.features(); - } - }); + JavaRDD vectors = testRDD.map(LabeledPoint::features); JavaRDD predictions = model.predict(vectors); // Should be able to get the first prediction. predictions.first(); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java index 7266eec23580..cb0097741234 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -17,57 +17,45 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.List; +import java.util.Random; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.jblas.DoubleMatrix; - +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.util.LinearDataGenerator; -public class JavaRidgeRegressionSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaRidgeRegressionSuite"); - } +public class JavaRidgeRegressionSuite extends SharedSparkSession { - @After - public void tearDown() { - sc.stop(); - sc = null; - } - - double predictionError(List validationData, RidgeRegressionModel model) { + private static double predictionError(List validationData, + RidgeRegressionModel model) { double errorSum = 0; - for (LabeledPoint point: validationData) { + for (LabeledPoint point : validationData) { Double prediction = model.predict(point.features()); errorSum += (prediction - point.label()) * (prediction - point.label()); } return errorSum / validationData.size(); } - List generateRidgeData(int numPoints, int numFeatures, double std) { - org.jblas.util.Random.seed(42); + private static List generateRidgeData(int numPoints, int numFeatures, double std) { // Pick weights as random values distributed uniformly in [-0.5, 0.5] - DoubleMatrix w = DoubleMatrix.rand(numFeatures, 1).subi(0.5); - return LinearDataGenerator.generateLinearInputAsList(0.0, w.data, numPoints, 42, std); + Random random = new Random(42); + double[] w = new double[numFeatures]; + for (int i = 0; i < w.length; i++) { + w[i] = random.nextDouble() - 0.5; + } + return LinearDataGenerator.generateLinearInputAsList(0.0, w, numPoints, 42, std); } @Test public void runRidgeRegressionUsingConstructor() { int numExamples = 50; int numFeatures = 20; - List data = generateRidgeData(2*numExamples, numFeatures, 10.0); + List data = generateRidgeData(2 * numExamples, numFeatures, 10.0); - JavaRDD testRDD = sc.parallelize(data.subList(0, numExamples)); + JavaRDD testRDD = jsc.parallelize(data.subList(0, numExamples)); List validationData = data.subList(numExamples, 2 * numExamples); RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD(); @@ -91,7 +79,7 @@ public void runRidgeRegressionUsingStaticMethods() { int numFeatures = 20; List data = generateRidgeData(2 * numExamples, numFeatures, 10.0); - JavaRDD testRDD = sc.parallelize(data.subList(0, numExamples)); + JavaRDD testRDD = jsc.parallelize(data.subList(0, numExamples)); List validationData = data.subList(numExamples, 2 * numExamples); RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.0); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java index dbf6488d4108..ab554475d59a 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.regression; -import java.io.Serializable; import java.util.Arrays; import java.util.List; @@ -36,7 +35,7 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext; import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStreamingLinearRegressionSuite implements Serializable { +public class JavaStreamingLinearRegressionSuite { protected transient JavaStreamingContext ssc; @@ -65,8 +64,8 @@ public void javaAPI() { JavaDStream training = attachTestInputStream(ssc, Arrays.asList(trainingBatch, trainingBatch), 2); List> testBatch = Arrays.asList( - new Tuple2(10, Vectors.dense(1.0)), - new Tuple2(11, Vectors.dense(0.0))); + new Tuple2<>(10, Vectors.dense(1.0)), + new Tuple2<>(11, Vectors.dense(0.0))); JavaPairDStream test = JavaPairDStream.fromJavaDStream( attachTestInputStream(ssc, Arrays.asList(testBatch, testBatch), 2)); StreamingLinearRegressionWithSGD slr = new StreamingLinearRegressionWithSGD() diff --git a/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java index 4795809e47a4..1abaa39eadc2 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java @@ -17,52 +17,70 @@ package org.apache.spark.mllib.stat; -import java.io.Serializable; - import java.util.Arrays; +import java.util.List; import org.junit.After; import org.junit.Before; import org.junit.Test; - import static org.junit.Assert.assertEquals; -import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.stat.test.BinarySample; import org.apache.spark.mllib.stat.test.ChiSqTestResult; import org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult; +import org.apache.spark.mllib.stat.test.StreamingTest; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import static org.apache.spark.streaming.JavaTestUtils.*; -public class JavaStatisticsSuite implements Serializable { - private transient JavaSparkContext sc; +public class JavaStatisticsSuite { + private transient SparkSession spark; + private transient JavaSparkContext jsc; + private transient JavaStreamingContext ssc; @Before public void setUp() { - sc = new JavaSparkContext("local", "JavaStatistics"); + SparkConf conf = new SparkConf() + .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); + spark = SparkSession.builder() + .master("local[2]") + .appName("JavaStatistics") + .config(conf) + .getOrCreate(); + jsc = new JavaSparkContext(spark.sparkContext()); + ssc = new JavaStreamingContext(jsc, new Duration(1000)); + ssc.checkpoint("checkpoint"); } @After public void tearDown() { - sc.stop(); - sc = null; + spark.stop(); + ssc.stop(); + spark = null; } @Test public void testCorr() { - JavaRDD x = sc.parallelize(Arrays.asList(1.0, 2.0, 3.0, 4.0)); - JavaRDD y = sc.parallelize(Arrays.asList(1.1, 2.2, 3.1, 4.3)); + JavaRDD x = jsc.parallelize(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + JavaRDD y = jsc.parallelize(Arrays.asList(1.1, 2.2, 3.1, 4.3)); Double corr1 = Statistics.corr(x, y); Double corr2 = Statistics.corr(x, y, "pearson"); // Check default method - assertEquals(corr1, corr2); + assertEquals(corr1, corr2, 1e-5); } @Test public void kolmogorovSmirnovTest() { - JavaDoubleRDD data = sc.parallelizeDoubles(Arrays.asList(0.2, 1.0, -1.0, 2.0)); + JavaDoubleRDD data = jsc.parallelizeDoubles(Arrays.asList(0.2, 1.0, -1.0, 2.0)); KolmogorovSmirnovTestResult testResult1 = Statistics.kolmogorovSmirnovTest(data, "norm"); KolmogorovSmirnovTestResult testResult2 = Statistics.kolmogorovSmirnovTest( data, "norm", 0.0, 1.0); @@ -70,10 +88,27 @@ public void kolmogorovSmirnovTest() { @Test public void chiSqTest() { - JavaRDD data = sc.parallelize(Arrays.asList( + JavaRDD data = jsc.parallelize(Arrays.asList( new LabeledPoint(0.0, Vectors.dense(0.1, 2.3)), new LabeledPoint(1.0, Vectors.dense(1.5, 5.1)), new LabeledPoint(0.0, Vectors.dense(2.4, 8.1)))); ChiSqTestResult[] testResults = Statistics.chiSqTest(data); } + + @Test + public void streamingTest() { + List trainingBatch = Arrays.asList( + new BinarySample(true, 1.0), + new BinarySample(false, 2.0)); + JavaDStream training = + attachTestInputStream(ssc, Arrays.asList(trainingBatch, trainingBatch), 2); + int numBatches = 2; + StreamingTest model = new StreamingTest() + .setWindowSize(0) + .setPeacePeriod(0) + .setTestMethod("welch"); + model.registerStream(training); + attachTestOutputStream(training); + runStreams(ssc, numBatches, numBatches); + } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java b/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java index 9925aae441af..d2fe6bb2ca71 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java @@ -17,41 +17,26 @@ package org.apache.spark.mllib.tree; -import java.io.Serializable; import java.util.HashMap; import java.util.List; -import org.junit.After; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; +import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.mllib.tree.configuration.Algo; import org.apache.spark.mllib.tree.configuration.Strategy; import org.apache.spark.mllib.tree.impurity.Gini; import org.apache.spark.mllib.tree.model.DecisionTreeModel; +public class JavaDecisionTreeSuite extends SharedSparkSession { -public class JavaDecisionTreeSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaDecisionTreeSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - } - - int validatePrediction(List validationData, DecisionTreeModel model) { + private static int validatePrediction( + List validationData, DecisionTreeModel model) { int numCorrect = 0; - for (LabeledPoint point: validationData) { + for (LabeledPoint point : validationData) { Double prediction = model.predict(point.features()); if (prediction == point.label()) { numCorrect++; @@ -63,40 +48,43 @@ int validatePrediction(List validationData, DecisionTreeModel mode @Test public void runDTUsingConstructor() { List arr = DecisionTreeSuite.generateCategoricalDataPointsAsJavaList(); - JavaRDD rdd = sc.parallelize(arr); - HashMap categoricalFeaturesInfo = new HashMap(); + JavaRDD rdd = jsc.parallelize(arr); + HashMap categoricalFeaturesInfo = new HashMap<>(); categoricalFeaturesInfo.put(1, 2); // feature 1 has 2 categories int maxDepth = 4; int numClasses = 2; int maxBins = 100; Strategy strategy = new Strategy(Algo.Classification(), Gini.instance(), maxDepth, numClasses, - maxBins, categoricalFeaturesInfo); + maxBins, categoricalFeaturesInfo); DecisionTree learner = new DecisionTree(strategy); DecisionTreeModel model = learner.run(rdd.rdd()); int numCorrect = validatePrediction(arr, model); - Assert.assertTrue(numCorrect == rdd.count()); + Assert.assertEquals(numCorrect, rdd.count()); } @Test public void runDTUsingStaticMethods() { List arr = DecisionTreeSuite.generateCategoricalDataPointsAsJavaList(); - JavaRDD rdd = sc.parallelize(arr); - HashMap categoricalFeaturesInfo = new HashMap(); + JavaRDD rdd = jsc.parallelize(arr); + HashMap categoricalFeaturesInfo = new HashMap<>(); categoricalFeaturesInfo.put(1, 2); // feature 1 has 2 categories int maxDepth = 4; int numClasses = 2; int maxBins = 100; Strategy strategy = new Strategy(Algo.Classification(), Gini.instance(), maxDepth, numClasses, - maxBins, categoricalFeaturesInfo); + maxBins, categoricalFeaturesInfo); DecisionTreeModel model = DecisionTree$.MODULE$.train(rdd.rdd(), strategy); + // java compatibility test + JavaRDD predictions = model.predict(rdd.map(LabeledPoint::features)); + int numCorrect = validatePrediction(arr, model); - Assert.assertTrue(numCorrect == rdd.count()); + Assert.assertEquals(numCorrect, rdd.count()); } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/util/JavaMLUtilsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/util/JavaMLUtilsSuite.java new file mode 100644 index 000000000000..e271a0a77c78 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/util/JavaMLUtilsSuite.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.util; + +import java.util.Arrays; +import java.util.Collections; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.spark.SharedSparkSession; +import org.apache.spark.mllib.linalg.*; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +public class JavaMLUtilsSuite extends SharedSparkSession { + + @Test + public void testConvertVectorColumnsToAndFromML() { + Vector x = Vectors.dense(2.0); + Dataset dataset = spark.createDataFrame( + Collections.singletonList(new LabeledPoint(1.0, x)), LabeledPoint.class + ).select("label", "features"); + Dataset newDataset1 = MLUtils.convertVectorColumnsToML(dataset); + Row new1 = newDataset1.first(); + Assert.assertEquals(RowFactory.create(1.0, x.asML()), new1); + Row new2 = MLUtils.convertVectorColumnsToML(dataset, "features").first(); + Assert.assertEquals(new1, new2); + Row old1 = MLUtils.convertVectorColumnsFromML(newDataset1).first(); + Assert.assertEquals(RowFactory.create(1.0, x), old1); + } + + @Test + public void testConvertMatrixColumnsToAndFromML() { + Matrix x = Matrices.dense(2, 1, new double[]{1.0, 2.0}); + StructType schema = new StructType(new StructField[]{ + new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("features", new MatrixUDT(), false, Metadata.empty()) + }); + Dataset dataset = spark.createDataFrame( + Arrays.asList( + RowFactory.create(1.0, x)), + schema); + + Dataset newDataset1 = MLUtils.convertMatrixColumnsToML(dataset); + Row new1 = newDataset1.first(); + Assert.assertEquals(RowFactory.create(1.0, x.asML()), new1); + Row new2 = MLUtils.convertMatrixColumnsToML(dataset, "features").first(); + Assert.assertEquals(new1, new2); + Row old1 = MLUtils.convertMatrixColumnsFromML(newDataset1).first(); + Assert.assertEquals(RowFactory.create(1.0, x), old1); + } +} diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index 75e3b53a093f..fd51f8faf56b 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN +log4j.logger.org.spark_project.jetty=WARN diff --git a/mllib/src/test/resources/test-data/iris.libsvm b/mllib/src/test/resources/test-data/iris.libsvm new file mode 100644 index 000000000000..db959010255d --- /dev/null +++ b/mllib/src/test/resources/test-data/iris.libsvm @@ -0,0 +1,150 @@ +0.0 1:5.1 2:3.5 3:1.4 4:0.2 +0.0 1:4.9 2:3.0 3:1.4 4:0.2 +0.0 1:4.7 2:3.2 3:1.3 4:0.2 +0.0 1:4.6 2:3.1 3:1.5 4:0.2 +0.0 1:5.0 2:3.6 3:1.4 4:0.2 +0.0 1:5.4 2:3.9 3:1.7 4:0.4 +0.0 1:4.6 2:3.4 3:1.4 4:0.3 +0.0 1:5.0 2:3.4 3:1.5 4:0.2 +0.0 1:4.4 2:2.9 3:1.4 4:0.2 +0.0 1:4.9 2:3.1 3:1.5 4:0.1 +0.0 1:5.4 2:3.7 3:1.5 4:0.2 +0.0 1:4.8 2:3.4 3:1.6 4:0.2 +0.0 1:4.8 2:3.0 3:1.4 4:0.1 +0.0 1:4.3 2:3.0 3:1.1 4:0.1 +0.0 1:5.8 2:4.0 3:1.2 4:0.2 +0.0 1:5.7 2:4.4 3:1.5 4:0.4 +0.0 1:5.4 2:3.9 3:1.3 4:0.4 +0.0 1:5.1 2:3.5 3:1.4 4:0.3 +0.0 1:5.7 2:3.8 3:1.7 4:0.3 +0.0 1:5.1 2:3.8 3:1.5 4:0.3 +0.0 1:5.4 2:3.4 3:1.7 4:0.2 +0.0 1:5.1 2:3.7 3:1.5 4:0.4 +0.0 1:4.6 2:3.6 3:1.0 4:0.2 +0.0 1:5.1 2:3.3 3:1.7 4:0.5 +0.0 1:4.8 2:3.4 3:1.9 4:0.2 +0.0 1:5.0 2:3.0 3:1.6 4:0.2 +0.0 1:5.0 2:3.4 3:1.6 4:0.4 +0.0 1:5.2 2:3.5 3:1.5 4:0.2 +0.0 1:5.2 2:3.4 3:1.4 4:0.2 +0.0 1:4.7 2:3.2 3:1.6 4:0.2 +0.0 1:4.8 2:3.1 3:1.6 4:0.2 +0.0 1:5.4 2:3.4 3:1.5 4:0.4 +0.0 1:5.2 2:4.1 3:1.5 4:0.1 +0.0 1:5.5 2:4.2 3:1.4 4:0.2 +0.0 1:4.9 2:3.1 3:1.5 4:0.1 +0.0 1:5.0 2:3.2 3:1.2 4:0.2 +0.0 1:5.5 2:3.5 3:1.3 4:0.2 +0.0 1:4.9 2:3.1 3:1.5 4:0.1 +0.0 1:4.4 2:3.0 3:1.3 4:0.2 +0.0 1:5.1 2:3.4 3:1.5 4:0.2 +0.0 1:5.0 2:3.5 3:1.3 4:0.3 +0.0 1:4.5 2:2.3 3:1.3 4:0.3 +0.0 1:4.4 2:3.2 3:1.3 4:0.2 +0.0 1:5.0 2:3.5 3:1.6 4:0.6 +0.0 1:5.1 2:3.8 3:1.9 4:0.4 +0.0 1:4.8 2:3.0 3:1.4 4:0.3 +0.0 1:5.1 2:3.8 3:1.6 4:0.2 +0.0 1:4.6 2:3.2 3:1.4 4:0.2 +0.0 1:5.3 2:3.7 3:1.5 4:0.2 +0.0 1:5.0 2:3.3 3:1.4 4:0.2 +1.0 1:7.0 2:3.2 3:4.7 4:1.4 +1.0 1:6.4 2:3.2 3:4.5 4:1.5 +1.0 1:6.9 2:3.1 3:4.9 4:1.5 +1.0 1:5.5 2:2.3 3:4.0 4:1.3 +1.0 1:6.5 2:2.8 3:4.6 4:1.5 +1.0 1:5.7 2:2.8 3:4.5 4:1.3 +1.0 1:6.3 2:3.3 3:4.7 4:1.6 +1.0 1:4.9 2:2.4 3:3.3 4:1.0 +1.0 1:6.6 2:2.9 3:4.6 4:1.3 +1.0 1:5.2 2:2.7 3:3.9 4:1.4 +1.0 1:5.0 2:2.0 3:3.5 4:1.0 +1.0 1:5.9 2:3.0 3:4.2 4:1.5 +1.0 1:6.0 2:2.2 3:4.0 4:1.0 +1.0 1:6.1 2:2.9 3:4.7 4:1.4 +1.0 1:5.6 2:2.9 3:3.6 4:1.3 +1.0 1:6.7 2:3.1 3:4.4 4:1.4 +1.0 1:5.6 2:3.0 3:4.5 4:1.5 +1.0 1:5.8 2:2.7 3:4.1 4:1.0 +1.0 1:6.2 2:2.2 3:4.5 4:1.5 +1.0 1:5.6 2:2.5 3:3.9 4:1.1 +1.0 1:5.9 2:3.2 3:4.8 4:1.8 +1.0 1:6.1 2:2.8 3:4.0 4:1.3 +1.0 1:6.3 2:2.5 3:4.9 4:1.5 +1.0 1:6.1 2:2.8 3:4.7 4:1.2 +1.0 1:6.4 2:2.9 3:4.3 4:1.3 +1.0 1:6.6 2:3.0 3:4.4 4:1.4 +1.0 1:6.8 2:2.8 3:4.8 4:1.4 +1.0 1:6.7 2:3.0 3:5.0 4:1.7 +1.0 1:6.0 2:2.9 3:4.5 4:1.5 +1.0 1:5.7 2:2.6 3:3.5 4:1.0 +1.0 1:5.5 2:2.4 3:3.8 4:1.1 +1.0 1:5.5 2:2.4 3:3.7 4:1.0 +1.0 1:5.8 2:2.7 3:3.9 4:1.2 +1.0 1:6.0 2:2.7 3:5.1 4:1.6 +1.0 1:5.4 2:3.0 3:4.5 4:1.5 +1.0 1:6.0 2:3.4 3:4.5 4:1.6 +1.0 1:6.7 2:3.1 3:4.7 4:1.5 +1.0 1:6.3 2:2.3 3:4.4 4:1.3 +1.0 1:5.6 2:3.0 3:4.1 4:1.3 +1.0 1:5.5 2:2.5 3:4.0 4:1.3 +1.0 1:5.5 2:2.6 3:4.4 4:1.2 +1.0 1:6.1 2:3.0 3:4.6 4:1.4 +1.0 1:5.8 2:2.6 3:4.0 4:1.2 +1.0 1:5.0 2:2.3 3:3.3 4:1.0 +1.0 1:5.6 2:2.7 3:4.2 4:1.3 +1.0 1:5.7 2:3.0 3:4.2 4:1.2 +1.0 1:5.7 2:2.9 3:4.2 4:1.3 +1.0 1:6.2 2:2.9 3:4.3 4:1.3 +1.0 1:5.1 2:2.5 3:3.0 4:1.1 +1.0 1:5.7 2:2.8 3:4.1 4:1.3 +2.0 1:6.3 2:3.3 3:6.0 4:2.5 +2.0 1:5.8 2:2.7 3:5.1 4:1.9 +2.0 1:7.1 2:3.0 3:5.9 4:2.1 +2.0 1:6.3 2:2.9 3:5.6 4:1.8 +2.0 1:6.5 2:3.0 3:5.8 4:2.2 +2.0 1:7.6 2:3.0 3:6.6 4:2.1 +2.0 1:4.9 2:2.5 3:4.5 4:1.7 +2.0 1:7.3 2:2.9 3:6.3 4:1.8 +2.0 1:6.7 2:2.5 3:5.8 4:1.8 +2.0 1:7.2 2:3.6 3:6.1 4:2.5 +2.0 1:6.5 2:3.2 3:5.1 4:2.0 +2.0 1:6.4 2:2.7 3:5.3 4:1.9 +2.0 1:6.8 2:3.0 3:5.5 4:2.1 +2.0 1:5.7 2:2.5 3:5.0 4:2.0 +2.0 1:5.8 2:2.8 3:5.1 4:2.4 +2.0 1:6.4 2:3.2 3:5.3 4:2.3 +2.0 1:6.5 2:3.0 3:5.5 4:1.8 +2.0 1:7.7 2:3.8 3:6.7 4:2.2 +2.0 1:7.7 2:2.6 3:6.9 4:2.3 +2.0 1:6.0 2:2.2 3:5.0 4:1.5 +2.0 1:6.9 2:3.2 3:5.7 4:2.3 +2.0 1:5.6 2:2.8 3:4.9 4:2.0 +2.0 1:7.7 2:2.8 3:6.7 4:2.0 +2.0 1:6.3 2:2.7 3:4.9 4:1.8 +2.0 1:6.7 2:3.3 3:5.7 4:2.1 +2.0 1:7.2 2:3.2 3:6.0 4:1.8 +2.0 1:6.2 2:2.8 3:4.8 4:1.8 +2.0 1:6.1 2:3.0 3:4.9 4:1.8 +2.0 1:6.4 2:2.8 3:5.6 4:2.1 +2.0 1:7.2 2:3.0 3:5.8 4:1.6 +2.0 1:7.4 2:2.8 3:6.1 4:1.9 +2.0 1:7.9 2:3.8 3:6.4 4:2.0 +2.0 1:6.4 2:2.8 3:5.6 4:2.2 +2.0 1:6.3 2:2.8 3:5.1 4:1.5 +2.0 1:6.1 2:2.6 3:5.6 4:1.4 +2.0 1:7.7 2:3.0 3:6.1 4:2.3 +2.0 1:6.3 2:3.4 3:5.6 4:2.4 +2.0 1:6.4 2:3.1 3:5.5 4:1.8 +2.0 1:6.0 2:3.0 3:4.8 4:1.8 +2.0 1:6.9 2:3.1 3:5.4 4:2.1 +2.0 1:6.7 2:3.1 3:5.6 4:2.4 +2.0 1:6.9 2:3.1 3:5.1 4:2.3 +2.0 1:5.8 2:2.7 3:5.1 4:1.9 +2.0 1:6.8 2:3.2 3:5.9 4:2.3 +2.0 1:6.7 2:3.3 3:5.7 4:2.5 +2.0 1:6.7 2:3.0 3:5.2 4:2.3 +2.0 1:6.3 2:2.5 3:5.0 4:1.9 +2.0 1:6.5 2:3.0 3:5.2 4:2.0 +2.0 1:6.2 2:3.4 3:5.4 4:2.3 +2.0 1:5.9 2:3.0 3:5.1 4:1.8 diff --git a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala index 1f2c9b75b617..7848eae931a0 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala @@ -19,17 +19,24 @@ package org.apache.spark.ml import scala.collection.JavaConverters._ +import org.apache.hadoop.fs.Path import org.mockito.Matchers.{any, eq => meq} import org.mockito.Mockito.when -import org.scalatest.mock.MockitoSugar.mock +import org.scalatest.mockito.MockitoSugar.mock import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.HashingTF -import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.sql.DataFrame +import org.apache.spark.ml.Pipeline.SharedReadWrite +import org.apache.spark.ml.feature.{HashingTF, MinMaxScaler} +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.param.{IntParam, ParamMap} +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.types.StructType -class PipelineSuite extends SparkFunSuite { +class PipelineSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ abstract class MyModel extends Model[MyModel] @@ -46,6 +53,12 @@ class PipelineSuite extends SparkFunSuite { val dataset3 = mock[DataFrame] val dataset4 = mock[DataFrame] + when(dataset0.toDF).thenReturn(dataset0) + when(dataset1.toDF).thenReturn(dataset1) + when(dataset2.toDF).thenReturn(dataset2) + when(dataset3.toDF).thenReturn(dataset3) + when(dataset4.toDF).thenReturn(dataset4) + when(estimator0.copy(any[ParamMap])).thenReturn(estimator0) when(model0.copy(any[ParamMap])).thenReturn(model0) when(transformer1.copy(any[ParamMap])).thenReturn(transformer1) @@ -66,7 +79,7 @@ class PipelineSuite extends SparkFunSuite { .setStages(Array(estimator0, transformer1, estimator2, transformer3)) val pipelineModel = pipeline.fit(dataset0) - MLTestingUtils.checkCopy(pipelineModel) + MLTestingUtils.checkCopyAndUids(pipeline, pipelineModel) assert(pipelineModel.stages.length === 4) assert(pipelineModel.stages(0).eq(model0)) @@ -88,13 +101,31 @@ class PipelineSuite extends SparkFunSuite { } } + test("Pipeline.copy") { + val hashingTF = new HashingTF() + .setNumFeatures(100) + val pipeline = new Pipeline("pipeline").setStages(Array[Transformer](hashingTF)) + val copied = pipeline.copy(ParamMap(hashingTF.numFeatures -> 10)) + + assert(copied.uid === pipeline.uid, + "copy should create an instance with the same UID") + assert(copied.getStages(0).asInstanceOf[HashingTF].getNumFeatures === 10, + "copy should handle extra stage params") + } + test("PipelineModel.copy") { val hashingTF = new HashingTF() .setNumFeatures(100) - val model = new PipelineModel("pipeline", Array[Transformer](hashingTF)) + val model = new PipelineModel("pipelineModel", Array[Transformer](hashingTF)) + .setParent(new Pipeline()) val copied = model.copy(ParamMap(hashingTF.numFeatures -> 10)) - require(copied.stages(0).asInstanceOf[HashingTF].getNumFeatures === 10, + + assert(copied.uid === model.uid, + "copy should create an instance with the same UID") + assert(copied.stages(0).asInstanceOf[HashingTF].getNumFeatures === 10, "copy should handle extra stage params") + assert(copied.parent === model.parent, + "copy should create an instance with the same parent") } test("pipeline model constructors") { @@ -111,4 +142,135 @@ class PipelineSuite extends SparkFunSuite { assert(pipelineModel1.uid === "pipeline1") assert(pipelineModel1.stages === stages) } + + test("Pipeline read/write") { + val writableStage = new WritableStage("writableStage").setIntParam(56) + val pipeline = new Pipeline().setStages(Array(writableStage)) + + val pipeline2 = testDefaultReadWrite(pipeline, testParams = false) + assert(pipeline2.getStages.length === 1) + assert(pipeline2.getStages(0).isInstanceOf[WritableStage]) + val writableStage2 = pipeline2.getStages(0).asInstanceOf[WritableStage] + assert(writableStage.getIntParam === writableStage2.getIntParam) + } + + test("Pipeline read/write with non-Writable stage") { + val unWritableStage = new UnWritableStage("unwritableStage") + val unWritablePipeline = new Pipeline().setStages(Array(unWritableStage)) + withClue("Pipeline.write should fail when Pipeline contains non-Writable stage") { + intercept[UnsupportedOperationException] { + unWritablePipeline.write + } + } + } + + test("PipelineModel read/write") { + val writableStage = new WritableStage("writableStage").setIntParam(56) + val pipeline = + new PipelineModel("pipeline_89329327", Array(writableStage.asInstanceOf[Transformer])) + + val pipeline2 = testDefaultReadWrite(pipeline, testParams = false) + assert(pipeline2.stages.length === 1) + assert(pipeline2.stages(0).isInstanceOf[WritableStage]) + val writableStage2 = pipeline2.stages(0).asInstanceOf[WritableStage] + assert(writableStage.getIntParam === writableStage2.getIntParam) + } + + test("PipelineModel read/write: getStagePath") { + val stageUid = "myStage" + val stagesDir = new Path("pipeline", "stages").toString + def testStage(stageIdx: Int, numStages: Int, expectedPrefix: String): Unit = { + val path = SharedReadWrite.getStagePath(stageUid, stageIdx, numStages, stagesDir) + val expected = new Path(stagesDir, expectedPrefix + "_" + stageUid).toString + assert(path === expected) + } + testStage(0, 1, "0") + testStage(0, 9, "0") + testStage(0, 10, "00") + testStage(1, 10, "01") + testStage(12, 999, "012") + } + + test("PipelineModel read/write with non-Writable stage") { + val unWritableStage = new UnWritableStage("unwritableStage") + val unWritablePipeline = + new PipelineModel("pipeline_328957", Array(unWritableStage.asInstanceOf[Transformer])) + withClue("PipelineModel.write should fail when PipelineModel contains non-Writable stage") { + intercept[UnsupportedOperationException] { + unWritablePipeline.write + } + } + } + + test("pipeline validateParams") { + val df = Seq( + (1, Vectors.dense(0.0, 1.0, 4.0), 1.0), + (2, Vectors.dense(1.0, 0.0, 4.0), 2.0), + (3, Vectors.dense(1.0, 0.0, 5.0), 3.0), + (4, Vectors.dense(0.0, 0.0, 5.0), 4.0) + ).toDF("id", "features", "label") + + intercept[IllegalArgumentException] { + val scaler = new MinMaxScaler() + .setInputCol("features") + .setOutputCol("features_scaled") + .setMin(10) + .setMax(0) + val pipeline = new Pipeline().setStages(Array(scaler)) + pipeline.fit(df) + } + } + + test("Pipeline.setStages should handle Java Arrays being non-covariant") { + val stages0 = Array(new UnWritableStage("b")) + val stages1 = Array(new WritableStage("a")) + val steps = stages0 ++ stages1 + val p = new Pipeline().setStages(steps) + } +} + + +/** + * Used to test [[Pipeline]] with `MLWritable` stages + */ +class WritableStage(override val uid: String) extends Transformer with MLWritable { + + final val intParam: IntParam = new IntParam(this, "intParam", "doc") + + def getIntParam: Int = $(intParam) + + def setIntParam(value: Int): this.type = set(intParam, value) + + setDefault(intParam -> 0) + + override def copy(extra: ParamMap): WritableStage = defaultCopy(extra) + + override def write: MLWriter = new DefaultParamsWriter(this) + + override def transform(dataset: Dataset[_]): DataFrame = dataset.toDF + + override def transformSchema(schema: StructType): StructType = schema +} + +object WritableStage extends MLReadable[WritableStage] { + + override def read: MLReader[WritableStage] = new DefaultParamsReader[WritableStage] + + override def load(path: String): WritableStage = super.load(path) +} + +/** + * Used to test [[Pipeline]] with non-`MLWritable` stages + */ +class UnWritableStage(override val uid: String) extends Transformer { + + final val intParam: IntParam = new IntParam(this, "intParam", "doc") + + setDefault(intParam -> 0) + + override def copy(extra: ParamMap): UnWritableStage = defaultCopy(extra) + + override def transform(dataset: Dataset[_]): DataFrame = dataset.toDF + + override def transformSchema(schema: StructType): StructType = schema } diff --git a/mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala new file mode 100644 index 000000000000..ec45e32d412a --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.param.shared.HasWeightCol +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +class PredictorSuite extends SparkFunSuite with MLlibTestSparkContext { + + import PredictorSuite._ + + test("should support all NumericType labels and weights, and not support other types") { + val df = spark.createDataFrame(Seq( + (0, 1, Vectors.dense(0, 2, 3)), + (1, 2, Vectors.dense(0, 3, 9)), + (0, 3, Vectors.dense(0, 2, 6)) + )).toDF("label", "weight", "features") + + val types = + Seq(ShortType, LongType, IntegerType, FloatType, ByteType, DoubleType, DecimalType(10, 0)) + + val predictor = new MockPredictor().setWeightCol("weight") + + types.foreach { t => + predictor.fit(df.select(col("label").cast(t), col("weight").cast(t), col("features"))) + } + + intercept[IllegalArgumentException] { + predictor.fit(df.select(col("label").cast(StringType), col("weight"), col("features"))) + } + + intercept[IllegalArgumentException] { + predictor.fit(df.select(col("label"), col("weight").cast(StringType), col("features"))) + } + } +} + +object PredictorSuite { + + class MockPredictor(override val uid: String) + extends Predictor[Vector, MockPredictor, MockPredictionModel] with HasWeightCol { + + def this() = this(Identifiable.randomUID("mockpredictor")) + + def setWeightCol(value: String): this.type = set(weightCol, value) + + override def train(dataset: Dataset[_]): MockPredictionModel = { + require(dataset.schema("label").dataType == DoubleType) + require(dataset.schema("weight").dataType == DoubleType) + new MockPredictionModel(uid) + } + + override def copy(extra: ParamMap): MockPredictor = + throw new NotImplementedError() + } + + class MockPredictionModel(override val uid: String) + extends PredictionModel[Vector, MockPredictionModel] { + + def this() = this(Identifiable.randomUID("mockpredictormodel")) + + override def predict(features: Vector): Double = + throw new NotImplementedError() + + override def copy(extra: ParamMap): MockPredictionModel = + throw new NotImplementedError() + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala index 1292e57d7c01..35586320cb82 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/ann/ANNSuite.scala @@ -18,10 +18,9 @@ package org.apache.spark.ml.ann import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ - class ANNSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -42,7 +41,7 @@ class ANNSuite extends SparkFunSuite with MLlibTestSparkContext { val dataSample = rddData.first() val layerSizes = dataSample._1.size +: hiddenLayersTopology :+ dataSample._2.size val topology = FeedForwardTopology.multiLayerPerceptron(layerSizes, false) - val initialWeights = FeedForwardModel(topology, 23124).weights() + val initialWeights = FeedForwardModel(topology, 23124).weights val trainer = new FeedForwardTrainer(topology, 2, 1) trainer.setWeights(initialWeights) trainer.LBFGSOptimizer.setNumIterations(20) @@ -76,10 +75,11 @@ class ANNSuite extends SparkFunSuite with MLlibTestSparkContext { val dataSample = rddData.first() val layerSizes = dataSample._1.size +: hiddenLayersTopology :+ dataSample._2.size val topology = FeedForwardTopology.multiLayerPerceptron(layerSizes, false) - val initialWeights = FeedForwardModel(topology, 23124).weights() + val initialWeights = FeedForwardModel(topology, 23124).weights val trainer = new FeedForwardTrainer(topology, 2, 2) - trainer.SGDOptimizer.setNumIterations(2000) - trainer.setWeights(initialWeights) + // TODO: add a test for SGD + trainer.LBFGSOptimizer.setConvergenceTol(1e-4).setNumIterations(20) + trainer.setWeights(initialWeights).setStackSize(1) val model = trainer.train(rddData) val predictionAndLabels = rddData.map { case (input, label) => (model.predict(input), label) diff --git a/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.scala new file mode 100644 index 000000000000..2f225645bdfc --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/ann/GradientSuite.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.ml.ann + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.mllib.util.MLlibTestSparkContext + +class GradientSuite extends SparkFunSuite with MLlibTestSparkContext { + + test("Gradient computation against numerical differentiation") { + val input = new BDM[Double](3, 1, Array(1.0, 1.0, 1.0)) + // output must contain zeros and one 1 for SoftMax + val target = new BDM[Double](2, 1, Array(0.0, 1.0)) + val topology = FeedForwardTopology.multiLayerPerceptron(Array(3, 4, 2), softmaxOnTop = false) + val layersWithErrors = Seq( + new SigmoidLayerWithSquaredError(), + new SoftmaxLayerWithCrossEntropyLoss() + ) + // check all layers that provide loss computation + // 1) compute loss and gradient given the model and initial weights + // 2) modify weights with small number epsilon (per dimension i) + // 3) compute new loss + // 4) ((newLoss - loss) / epsilon) should be close to the i-th component of the gradient + for (layerWithError <- layersWithErrors) { + topology.layers(topology.layers.length - 1) = layerWithError + val model = topology.model(seed = 12L) + val weights = model.weights.toArray + val numWeights = weights.size + val gradient = Vectors.dense(Array.fill[Double](numWeights)(0.0)) + val loss = model.computeGradient(input, target, gradient, 1) + val eps = 1e-4 + var i = 0 + val tol = 1e-4 + while (i < numWeights) { + val originalValue = weights(i) + weights(i) += eps + val newModel = topology.model(Vectors.dense(weights)) + val newLoss = computeLoss(input, target, newModel) + val derivativeEstimate = (newLoss - loss) / eps + assert(math.abs(gradient(i) - derivativeEstimate) < tol, "Layer failed gradient check: " + + layerWithError.getClass) + weights(i) = originalValue + i += 1 + } + } + } + + private def computeLoss(input: BDM[Double], target: BDM[Double], model: TopologyModel): Double = { + val outputs = model.forward(input, true) + model.layerModels.last match { + case layerWithLoss: LossFunction => + layerWithLoss.loss(outputs.last, target, new BDM[Double](target.rows, target.cols)) + case _ => + throw new UnsupportedOperationException("Top layer is required to have loss." + + " Failed layer:" + model.layerModels.last.getClass) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala new file mode 100644 index 000000000000..de712079329d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.classification + +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.classification.ClassifierSuite.MockClassifier +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.util.Identifiable +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Dataset} + +class ClassifierSuite extends SparkFunSuite with MLlibTestSparkContext { + + import testImplicits._ + + private def getTestData(labels: Seq[Double]): DataFrame = { + labels.map { label: Double => LabeledPoint(label, Vectors.dense(0.0)) }.toDF() + } + + test("extractLabeledPoints") { + val c = new MockClassifier + // Valid dataset + val df0 = getTestData(Seq(0.0, 2.0, 1.0, 5.0)) + c.extractLabeledPoints(df0, 6).count() + // Invalid datasets + val df1 = getTestData(Seq(0.0, -2.0, 1.0, 5.0)) + withClue("Classifier should fail if label is negative") { + val e: SparkException = intercept[SparkException] { + c.extractLabeledPoints(df1, 6).count() + } + assert(e.getMessage.contains("given dataset with invalid label")) + } + val df2 = getTestData(Seq(0.0, 2.1, 1.0, 5.0)) + withClue("Classifier should fail if label is not an integer") { + val e: SparkException = intercept[SparkException] { + c.extractLabeledPoints(df2, 6).count() + } + assert(e.getMessage.contains("given dataset with invalid label")) + } + // extractLabeledPoints with numClasses specified + withClue("Classifier should fail if label is >= numClasses") { + val e: SparkException = intercept[SparkException] { + c.extractLabeledPoints(df0, numClasses = 5).count() + } + assert(e.getMessage.contains("given dataset with invalid label")) + } + withClue("Classifier.extractLabeledPoints should fail if numClasses <= 0") { + val e: IllegalArgumentException = intercept[IllegalArgumentException] { + c.extractLabeledPoints(df0, numClasses = 0).count() + } + assert(e.getMessage.contains("but requires numClasses > 0")) + } + } + + test("getNumClasses") { + val c = new MockClassifier + // Valid dataset + val df0 = getTestData(Seq(0.0, 2.0, 1.0, 5.0)) + assert(c.getNumClasses(df0) === 6) + // Invalid datasets + val df1 = getTestData(Seq(0.0, 2.0, 1.0, 5.1)) + withClue("getNumClasses should fail if label is max label not an integer") { + val e: IllegalArgumentException = intercept[IllegalArgumentException] { + c.getNumClasses(df1) + } + assert(e.getMessage.contains("requires integers in range")) + } + val df2 = getTestData(Seq(0.0, 2.0, 1.0, Int.MaxValue.toDouble)) + withClue("getNumClasses should fail if label is max label is >= Int.MaxValue") { + val e: IllegalArgumentException = intercept[IllegalArgumentException] { + c.getNumClasses(df2) + } + assert(e.getMessage.contains("requires integers in range")) + } + } +} + +object ClassifierSuite { + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "predictionCol" -> "myPrediction", + "rawPredictionCol" -> "myRawPrediction" + ) + + class MockClassifier(override val uid: String) + extends Classifier[Vector, MockClassifier, MockClassificationModel] { + + def this() = this(Identifiable.randomUID("mockclassifier")) + + override def copy(extra: ParamMap): MockClassifier = throw new NotImplementedError() + + override def train(dataset: Dataset[_]): MockClassificationModel = + throw new NotImplementedError() + + // Make methods public + override def extractLabeledPoints(dataset: Dataset[_], numClasses: Int): RDD[LabeledPoint] = + super.extractLabeledPoints(dataset, numClasses) + def getNumClasses(dataset: Dataset[_]): Int = super.getNumClasses(dataset) + } + + class MockClassificationModel(override val uid: String) + extends ClassificationModel[Vector, MockClassificationModel] { + + def this() = this(Identifiable.randomUID("mockclassificationmodel")) + + protected def predictRaw(features: Vector): Vector = throw new NotImplementedError() + + override def copy(extra: ParamMap): MockClassificationModel = throw new NotImplementedError() + + override def numClasses: Int = throw new NotImplementedError() + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala index 815f6fd99758..98c879ece62d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala @@ -18,21 +18,23 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.impl.TreeTests +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.tree.LeafNode -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.tree.{CategoricalSplit, InternalNode, LeafNode} +import org.apache.spark.ml.tree.impl.TreeTests +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, DecisionTreeSuite => OldDecisionTreeSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.Row +import org.apache.spark.sql.{DataFrame, Row} -class DecisionTreeClassifierSuite extends SparkFunSuite with MLlibTestSparkContext { +class DecisionTreeClassifierSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { import DecisionTreeClassifierSuite.compareAPIs + import testImplicits._ private var categoricalDataPointsRDD: RDD[LabeledPoint] = _ private var orderedLabeledPointsWithLabel0RDD: RDD[LabeledPoint] = _ @@ -44,17 +46,18 @@ class DecisionTreeClassifierSuite extends SparkFunSuite with MLlibTestSparkConte override def beforeAll() { super.beforeAll() categoricalDataPointsRDD = - sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()) + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()).map(_.asML) orderedLabeledPointsWithLabel0RDD = - sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel0()) + sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel0()).map(_.asML) orderedLabeledPointsWithLabel1RDD = - sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()) + sc.parallelize(OldDecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()).map(_.asML) categoricalDataPointsForMulticlassRDD = - sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlass()) + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlass()).map(_.asML) continuousDataPointsForMulticlassRDD = - sc.parallelize(OldDecisionTreeSuite.generateContinuousDataPointsForMulticlass()) + sc.parallelize(OldDecisionTreeSuite.generateContinuousDataPointsForMulticlass()).map(_.asML) categoricalDataPointsForMulticlassForOrderedFeaturesRDD = sc.parallelize( OldDecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures()) + .map(_.asML) } test("params") { @@ -72,7 +75,8 @@ class DecisionTreeClassifierSuite extends SparkFunSuite with MLlibTestSparkConte .setImpurity("gini") .setMaxDepth(2) .setMaxBins(100) - val categoricalFeatures = Map(0 -> 3, 1-> 3) + .setSeed(1) + val categoricalFeatures = Map(0 -> 3, 1 -> 3) val numClasses = 2 compareAPIs(categoricalDataPointsRDD, dt, categoricalFeatures, numClasses) } @@ -174,7 +178,7 @@ class DecisionTreeClassifierSuite extends SparkFunSuite with MLlibTestSparkConte } test("Multiclass classification tree with 10-ary (ordered) categorical features," + - " with just enough bins") { + " with just enough bins") { val rdd = categoricalDataPointsForMulticlassForOrderedFeaturesRDD val dt = new DecisionTreeClassifier() .setImpurity("Gini") @@ -213,7 +217,7 @@ class DecisionTreeClassifierSuite extends SparkFunSuite with MLlibTestSparkConte .setMaxBins(2) .setMaxDepth(2) .setMinInstancesPerNode(2) - val categoricalFeatures = Map(0 -> 2, 1-> 2) + val categoricalFeatures = Map(0 -> 2, 1 -> 2) val numClasses = 2 compareAPIs(rdd, dt, categoricalFeatures, numClasses) } @@ -245,8 +249,7 @@ class DecisionTreeClassifierSuite extends SparkFunSuite with MLlibTestSparkConte val newData: DataFrame = TreeTests.setMetadata(rdd, categoricalFeatures, numClasses) val newTree = dt.fit(newData) - // copied model must have the same parent. - MLTestingUtils.checkCopy(newTree) + MLTestingUtils.checkCopyAndUids(dt, newTree) val predictions = newTree.transform(newData) .select(newTree.getPredictionCol, newTree.getRawPredictionCol, newTree.getProbabilityCol) @@ -259,6 +262,9 @@ class DecisionTreeClassifierSuite extends SparkFunSuite with MLlibTestSparkConte assert(Vectors.dense(rawPred.toArray.map(_ / sum)) === probPred, "probability prediction mismatch") } + + ProbabilisticClassifierSuite.testPredictMethods[ + Vector, DecisionTreeClassificationModel](newTree, newData) } test("training with 1-category categorical feature") { @@ -271,32 +277,130 @@ class DecisionTreeClassifierSuite extends SparkFunSuite with MLlibTestSparkConte )) val df = TreeTests.setMetadata(data, Map(0 -> 1), 2) val dt = new DecisionTreeClassifier().setMaxDepth(3) + dt.fit(df) + } + + test("Use soft prediction for binary classification with ordered categorical features") { + // The following dataset is set up such that the best split is {1} vs. {0, 2}. + // If the hard prediction is used to order the categories, then {0} vs. {1, 2} is chosen. + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(1.0, Vectors.dense(0.0)), + LabeledPoint(0.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(2.0)), + LabeledPoint(0.0, Vectors.dense(2.0)), + LabeledPoint(0.0, Vectors.dense(2.0)), + LabeledPoint(1.0, Vectors.dense(2.0))) + val data = sc.parallelize(arr) + val df = TreeTests.setMetadata(data, Map(0 -> 3), 2) + + // Must set maxBins s.t. the feature will be treated as an ordered categorical feature. + val dt = new DecisionTreeClassifier() + .setImpurity("gini") + .setMaxDepth(1) + .setMaxBins(3) + val model = dt.fit(df) + model.rootNode match { + case n: InternalNode => + n.split match { + case s: CategoricalSplit => + assert(s.leftCategories === Array(1.0)) + case other => + fail(s"All splits should be categorical, but got ${other.getClass.getName}: $other.") + } + case other => + fail(s"Root node should be an internal node, but got ${other.getClass.getName}: $other.") + } + } + + test("Feature importance with toy data") { + val dt = new DecisionTreeClassifier() + .setImpurity("gini") + .setMaxDepth(3) + .setSeed(123) + + // In this data, feature 1 is very important. + val data: RDD[LabeledPoint] = TreeTests.featureImportanceData(sc) + val numFeatures = data.first().features.size + val categoricalFeatures = (0 to numFeatures).map(i => (i, 2)).toMap + val df = TreeTests.setMetadata(data, categoricalFeatures, 2) + val model = dt.fit(df) + + val importances = model.featureImportances + val mostImportantFeature = importances.argmax + assert(mostImportantFeature === 1) + assert(importances.toArray.sum === 1.0) + assert(importances.toArray.forall(_ >= 0.0)) + } + + test("should support all NumericType labels and not support other types") { + val dt = new DecisionTreeClassifier().setMaxDepth(1) + MLTestingUtils.checkNumericTypes[DecisionTreeClassificationModel, DecisionTreeClassifier]( + dt, spark) { (expected, actual) => + TreeTests.checkEqual(expected, actual) + } + } + + test("Fitting without numClasses in metadata") { + val df: DataFrame = TreeTests.featureImportanceData(sc).toDF() + val dt = new DecisionTreeClassifier().setMaxDepth(1) + dt.fit(df) } ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// - // TODO: Reinstate test once save/load are implemented SPARK-6725 - /* - test("model save/load") { - val tempDir = Utils.createTempDir() - val path = tempDir.toURI.toString - - val oldModel = OldDecisionTreeSuite.createModel(OldAlgo.Classification) - val newModel = DecisionTreeClassificationModel.fromOld(oldModel) - - // Save model, load it back, and compare. - try { - newModel.save(sc, path) - val sameNewModel = DecisionTreeClassificationModel.load(sc, path) - TreeTests.checkEqual(newModel, sameNewModel) - } finally { - Utils.deleteRecursively(tempDir) + test("read/write") { + def checkModelData( + model: DecisionTreeClassificationModel, + model2: DecisionTreeClassificationModel): Unit = { + TreeTests.checkEqual(model, model2) + assert(model.numFeatures === model2.numFeatures) + assert(model.numClasses === model2.numClasses) } + + val dt = new DecisionTreeClassifier() + val rdd = TreeTests.getTreeReadWriteData(sc) + + val allParamSettings = TreeTests.allParamSettings ++ Map("impurity" -> "entropy") + + // Categorical splits with tree depth 2 + val categoricalData: DataFrame = + TreeTests.setMetadata(rdd, Map(0 -> 2, 1 -> 3), numClasses = 2) + testEstimatorAndModelReadWrite(dt, categoricalData, allParamSettings, + allParamSettings, checkModelData) + + // Continuous splits with tree depth 2 + val continuousData: DataFrame = + TreeTests.setMetadata(rdd, Map.empty[Int, Int], numClasses = 2) + testEstimatorAndModelReadWrite(dt, continuousData, allParamSettings, + allParamSettings, checkModelData) + + // Continuous splits with tree depth 0 + testEstimatorAndModelReadWrite(dt, continuousData, allParamSettings ++ Map("maxDepth" -> 0), + allParamSettings ++ Map("maxDepth" -> 0), checkModelData) + } + + test("SPARK-20043: " + + "ImpurityCalculator builder fails for uppercase impurity type Gini in model read/write") { + val rdd = TreeTests.getTreeReadWriteData(sc) + val data: DataFrame = + TreeTests.setMetadata(rdd, Map.empty[Int, Int], numClasses = 2) + + val dt = new DecisionTreeClassifier() + .setImpurity("Gini") + .setMaxDepth(2) + val model = dt.fit(data) + + testDefaultReadWrite(model) } - */ } private[ml] object DecisionTreeClassifierSuite extends SparkFunSuite { @@ -312,7 +416,7 @@ private[ml] object DecisionTreeClassifierSuite extends SparkFunSuite { numClasses: Int): Unit = { val numFeatures = data.first().features.size val oldStrategy = dt.getOldStrategy(categoricalFeatures, numClasses) - val oldTree = OldDecisionTree.train(data, oldStrategy) + val oldTree = OldDecisionTree.train(data.map(OldLabeledPoint.fromML), oldStrategy) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses) val newTree = dt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala index 039141aeb6f6..8000143d4d14 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala @@ -17,26 +17,33 @@ package org.apache.spark.ml.classification -import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.impl.TreeTests +import com.github.fommil.netlib.BLAS + +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree.LeafNode -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.tree.impl.TreeTests +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, GradientBoostedTrees => OldGBT} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} +import org.apache.spark.mllib.tree.loss.LogLoss import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.util.Utils - /** * Test suite for [[GBTClassifier]]. */ -class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext { +class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext + with DefaultReadWriteTest { + import testImplicits._ import GBTClassifierSuite.compareAPIs // Combinations for estimators, learning rates and subsamplingRate @@ -46,24 +53,185 @@ class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext { private var data: RDD[LabeledPoint] = _ private var trainData: RDD[LabeledPoint] = _ private var validationData: RDD[LabeledPoint] = _ + private val eps: Double = 1e-5 + private val absEps: Double = 1e-8 override def beforeAll() { super.beforeAll() data = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100), 2) + .map(_.asML) trainData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 120), 2) + .map(_.asML) validationData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 80), 2) + .map(_.asML) } test("params") { ParamsSuite.checkParams(new GBTClassifier) val model = new GBTClassificationModel("gbtc", Array(new DecisionTreeRegressionModel("dtr", new LeafNode(0.0, 0.0, null), 1)), - Array(1.0), 1) + Array(1.0), 1, 2) ParamsSuite.checkParams(model) } + test("GBTClassifier: default params") { + val gbt = new GBTClassifier + assert(gbt.getLabelCol === "label") + assert(gbt.getFeaturesCol === "features") + assert(gbt.getPredictionCol === "prediction") + assert(gbt.getRawPredictionCol === "rawPrediction") + assert(gbt.getProbabilityCol === "probability") + val df = trainData.toDF() + val model = gbt.fit(df) + model.transform(df) + .select("label", "probability", "prediction", "rawPrediction") + .collect() + intercept[NoSuchElementException] { + model.getThresholds + } + assert(model.getFeaturesCol === "features") + assert(model.getPredictionCol === "prediction") + assert(model.getRawPredictionCol === "rawPrediction") + assert(model.getProbabilityCol === "probability") + assert(model.hasParent) + + MLTestingUtils.checkCopyAndUids(gbt, model) + } + + test("setThreshold, getThreshold") { + val gbt = new GBTClassifier + + // default + withClue("GBTClassifier should not have thresholds set by default.") { + intercept[NoSuchElementException] { + gbt.getThresholds + } + } + + // Set via thresholds + val gbt2 = new GBTClassifier + val threshold = Array(0.3, 0.7) + gbt2.setThresholds(threshold) + assert(gbt2.getThresholds === threshold) + } + + test("thresholds prediction") { + val gbt = new GBTClassifier + val df = trainData.toDF() + val binaryModel = gbt.fit(df) + + // should predict all zeros + binaryModel.setThresholds(Array(0.0, 1.0)) + val binaryZeroPredictions = binaryModel.transform(df).select("prediction").collect() + assert(binaryZeroPredictions.forall(_.getDouble(0) === 0.0)) + + // should predict all ones + binaryModel.setThresholds(Array(1.0, 0.0)) + val binaryOnePredictions = binaryModel.transform(df).select("prediction").collect() + assert(binaryOnePredictions.forall(_.getDouble(0) === 1.0)) + + + val gbtBase = new GBTClassifier + val model = gbtBase.fit(df) + val basePredictions = model.transform(df).select("prediction").collect() + + // constant threshold scaling is the same as no thresholds + binaryModel.setThresholds(Array(1.0, 1.0)) + val scaledPredictions = binaryModel.transform(df).select("prediction").collect() + assert(scaledPredictions.zip(basePredictions).forall { case (scaled, base) => + scaled.getDouble(0) === base.getDouble(0) + }) + + // force it to use the predict method + model.setRawPredictionCol("").setProbabilityCol("").setThresholds(Array(0, 1)) + val predictionsWithPredict = model.transform(df).select("prediction").collect() + assert(predictionsWithPredict.forall(_.getDouble(0) === 0.0)) + } + + test("GBTClassifier: Predictor, Classifier methods") { + val rawPredictionCol = "rawPrediction" + val predictionCol = "prediction" + val labelCol = "label" + val featuresCol = "features" + val probabilityCol = "probability" + + val gbt = new GBTClassifier().setSeed(123) + val trainingDataset = trainData.toDF(labelCol, featuresCol) + val gbtModel = gbt.fit(trainingDataset) + assert(gbtModel.numClasses === 2) + val numFeatures = trainingDataset.select(featuresCol).first().getAs[Vector](0).size + assert(gbtModel.numFeatures === numFeatures) + + val blas = BLAS.getInstance() + + val validationDataset = validationData.toDF(labelCol, featuresCol) + val results = gbtModel.transform(validationDataset) + // check that raw prediction is tree predictions dot tree weights + results.select(rawPredictionCol, featuresCol).collect().foreach { + case Row(raw: Vector, features: Vector) => + assert(raw.size === 2) + val treePredictions = gbtModel.trees.map(_.rootNode.predictImpl(features).prediction) + val prediction = blas.ddot(gbtModel.numTrees, treePredictions, 1, gbtModel.treeWeights, 1) + assert(raw ~== Vectors.dense(-prediction, prediction) relTol eps) + } + + // Compare rawPrediction with probability + results.select(rawPredictionCol, probabilityCol).collect().foreach { + case Row(raw: Vector, prob: Vector) => + assert(raw.size === 2) + assert(prob.size === 2) + // Note: we should check other loss types for classification if they are added + val predFromRaw = raw.toDense.values.map(value => LogLoss.computeProbability(value)) + assert(prob(0) ~== predFromRaw(0) relTol eps) + assert(prob(1) ~== predFromRaw(1) relTol eps) + assert(prob(0) + prob(1) ~== 1.0 absTol absEps) + } + + // Compare prediction with probability + results.select(predictionCol, probabilityCol).collect().foreach { + case Row(pred: Double, prob: Vector) => + val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2 + assert(pred == predFromProb) + } + + // force it to use raw2prediction + gbtModel.setRawPredictionCol(rawPredictionCol).setProbabilityCol("") + val resultsUsingRaw2Predict = + gbtModel.transform(validationDataset).select(predictionCol).as[Double].collect() + resultsUsingRaw2Predict.zip(results.select(predictionCol).as[Double].collect()).foreach { + case (pred1, pred2) => assert(pred1 === pred2) + } + + // force it to use probability2prediction + gbtModel.setRawPredictionCol("").setProbabilityCol(probabilityCol) + val resultsUsingProb2Predict = + gbtModel.transform(validationDataset).select(predictionCol).as[Double].collect() + resultsUsingProb2Predict.zip(results.select(predictionCol).as[Double].collect()).foreach { + case (pred1, pred2) => assert(pred1 === pred2) + } + + // force it to use predict + gbtModel.setRawPredictionCol("").setProbabilityCol("") + val resultsUsingPredict = + gbtModel.transform(validationDataset).select(predictionCol).as[Double].collect() + resultsUsingPredict.zip(results.select(predictionCol).as[Double].collect()).foreach { + case (pred1, pred2) => assert(pred1 === pred2) + } + + ProbabilisticClassifierSuite.testPredictMethods[ + Vector, GBTClassificationModel](gbtModel, validationDataset) + } + + test("GBT parameter stepSize should be in interval (0, 1]") { + withClue("GBT parameter stepSize should be in interval (0, 1]") { + intercept[IllegalArgumentException] { + new GBTClassifier().setStepSize(10) + } + } + } + test("Binary classification with continuous features: Log Loss") { val categoricalFeatures = Map.empty[Int, Int] testCombinations.foreach { @@ -74,6 +242,7 @@ class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext { .setLossType("logistic") .setMaxIter(maxIter) .setStepSize(learningRate) + .setSeed(123) compareAPIs(data, None, gbt, categoricalFeatures) } } @@ -91,15 +260,23 @@ class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext { .setMaxIter(5) .setStepSize(0.1) .setCheckpointInterval(2) + .setSeed(123) val model = gbt.fit(df) - // copied model must have the same parent. - MLTestingUtils.checkCopy(model) + MLTestingUtils.checkCopyAndUids(gbt, model) sc.checkpointDir = None Utils.deleteRecursively(tempDir) } + test("should support all NumericType labels and not support other types") { + val gbt = new GBTClassifier().setMaxDepth(1) + MLTestingUtils.checkNumericTypes[GBTClassificationModel, GBTClassifier]( + gbt, spark) { (expected, actual) => + TreeTests.checkEqual(expected, actual) + } + } + // TODO: Reinstate test once runWithValidation is implemented SPARK-7132 /* test("runWithValidation stops early and performs better on a validation dataset") { @@ -118,31 +295,89 @@ class GBTClassifierSuite extends SparkFunSuite with MLlibTestSparkContext { } */ + test("Fitting without numClasses in metadata") { + val df: DataFrame = TreeTests.featureImportanceData(sc).toDF() + val gbt = new GBTClassifier().setMaxDepth(1).setMaxIter(1) + gbt.fit(df) + } + + test("extractLabeledPoints with bad data") { + def getTestData(labels: Seq[Double]): DataFrame = { + labels.map { label: Double => LabeledPoint(label, Vectors.dense(0.0)) }.toDF() + } + + val gbt = new GBTClassifier().setMaxDepth(1).setMaxIter(1) + // Invalid datasets + val df1 = getTestData(Seq(0.0, -1.0, 1.0, 0.0)) + withClue("Classifier should fail if label is negative") { + val e: SparkException = intercept[SparkException] { + gbt.fit(df1) + } + assert(e.getMessage.contains("currently only supports binary classification")) + } + val df2 = getTestData(Seq(0.0, 0.1, 1.0, 0.0)) + withClue("Classifier should fail if label is not an integer") { + val e: SparkException = intercept[SparkException] { + gbt.fit(df2) + } + assert(e.getMessage.contains("currently only supports binary classification")) + } + val df3 = getTestData(Seq(0.0, 2.0, 1.0, 0.0)) + withClue("Classifier should fail if label is >= 2") { + val e: SparkException = intercept[SparkException] { + gbt.fit(df3) + } + assert(e.getMessage.contains("currently only supports binary classification")) + } + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests of feature importance + ///////////////////////////////////////////////////////////////////////////// + test("Feature importance with toy data") { + val numClasses = 2 + val gbt = new GBTClassifier() + .setImpurity("Gini") + .setMaxDepth(3) + .setMaxIter(5) + .setSubsamplingRate(1.0) + .setStepSize(0.5) + .setSeed(123) + + // In this data, feature 1 is very important. + val data: RDD[LabeledPoint] = TreeTests.featureImportanceData(sc) + val categoricalFeatures = Map.empty[Int, Int] + val df: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses) + + val importances = gbt.fit(df).featureImportances + val mostImportantFeature = importances.argmax + assert(mostImportantFeature === 1) + assert(importances.toArray.sum === 1.0) + assert(importances.toArray.forall(_ >= 0.0)) + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// - // TODO: Reinstate test once save/load are implemented SPARK-6725 - /* test("model save/load") { - val tempDir = Utils.createTempDir() - val path = tempDir.toURI.toString + def checkModelData( + model: GBTClassificationModel, + model2: GBTClassificationModel): Unit = { + TreeTests.checkEqual(model, model2) + assert(model.numFeatures === model2.numFeatures) + } - val trees = Range(0, 3).map(_ => OldDecisionTreeSuite.createModel(OldAlgo.Regression)).toArray - val treeWeights = Array(0.1, 0.3, 1.1) - val oldModel = new OldGBTModel(OldAlgo.Classification, trees, treeWeights) - val newModel = GBTClassificationModel.fromOld(oldModel) + val gbt = new GBTClassifier() + val rdd = TreeTests.getTreeReadWriteData(sc) - // Save model, load it back, and compare. - try { - newModel.save(sc, path) - val sameNewModel = GBTClassificationModel.load(sc, path) - TreeTests.checkEqual(newModel, sameNewModel) - } finally { - Utils.deleteRecursively(tempDir) - } + val allParamSettings = TreeTests.allParamSettings ++ Map("lossType" -> "logistic") + + val continuousData: DataFrame = + TreeTests.setMetadata(rdd, Map.empty[Int, Int], numClasses = 2) + testEstimatorAndModelReadWrite(gbt, continuousData, allParamSettings, + allParamSettings, checkModelData) } - */ } private object GBTClassifierSuite extends SparkFunSuite { @@ -159,13 +394,14 @@ private object GBTClassifierSuite extends SparkFunSuite { val numFeatures = data.first().features.size val oldBoostingStrategy = gbt.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Classification) - val oldGBT = new OldGBT(oldBoostingStrategy) - val oldModel = oldGBT.run(data) + val oldGBT = new OldGBT(oldBoostingStrategy, gbt.getSeed.toInt) + val oldModel = oldGBT.run(data.map(OldLabeledPoint.fromML)) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 2) val newModel = gbt.fit(newData) // Use parent from newTree since this is not checked anyways. val oldModelAsNew = GBTClassificationModel.fromOld( - oldModel, newModel.parent.asInstanceOf[GBTClassifier], categoricalFeatures, numFeatures) + oldModel, newModel.parent.asInstanceOf[GBTClassifier], categoricalFeatures, + numFeatures, numClasses = 2) TreeTests.checkEqual(oldModelAsNew, newModel) assert(newModel.numFeatures === numFeatures) assert(oldModelAsNew.numFeatures === numFeatures) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala new file mode 100644 index 000000000000..41a5d22dd628 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala @@ -0,0 +1,309 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.classification + +import scala.util.Random + +import breeze.linalg.{DenseVector => BDV} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.classification.LinearSVCSuite._ +import org.apache.spark.ml.feature.{Instance, LabeledPoint} +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.optim.aggregator.HingeAggregator +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.functions.udf + + +class LinearSVCSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + + private val nPoints = 50 + @transient var smallBinaryDataset: Dataset[_] = _ + @transient var smallValidationDataset: Dataset[_] = _ + @transient var binaryDataset: Dataset[_] = _ + + @transient var smallSparseBinaryDataset: Dataset[_] = _ + @transient var smallSparseValidationDataset: Dataset[_] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + // NOTE: Intercept should be small for generating equal 0s and 1s + val A = 0.01 + val B = -1.5 + val C = 1.0 + smallBinaryDataset = generateSVMInput(A, Array[Double](B, C), nPoints, 42).toDF() + smallValidationDataset = generateSVMInput(A, Array[Double](B, C), nPoints, 17).toDF() + binaryDataset = generateSVMInput(1.0, Array[Double](1.0, 2.0, 3.0, 4.0), 10000, 42).toDF() + + // Dataset for testing SparseVector + val toSparse: Vector => SparseVector = _.asInstanceOf[DenseVector].toSparse + val sparse = udf(toSparse) + smallSparseBinaryDataset = smallBinaryDataset.withColumn("features", sparse('features)) + smallSparseValidationDataset = smallValidationDataset.withColumn("features", sparse('features)) + + } + + /** + * Enable the ignored test to export the dataset into CSV format, + * so we can validate the training accuracy compared with R's e1071 package. + */ + ignore("export test data into CSV format") { + binaryDataset.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile("target/tmp/LinearSVC/binaryDataset") + } + + test("Linear SVC binary classification") { + val svm = new LinearSVC() + val model = svm.fit(smallBinaryDataset) + assert(model.transform(smallValidationDataset) + .where("prediction=label").count() > nPoints * 0.8) + val sparseModel = svm.fit(smallSparseBinaryDataset) + checkModels(model, sparseModel) + } + + test("Linear SVC binary classification with regularization") { + val svm = new LinearSVC() + val model = svm.setRegParam(0.1).fit(smallBinaryDataset) + assert(model.transform(smallValidationDataset) + .where("prediction=label").count() > nPoints * 0.8) + val sparseModel = svm.fit(smallSparseBinaryDataset) + checkModels(model, sparseModel) + } + + test("params") { + ParamsSuite.checkParams(new LinearSVC) + val model = new LinearSVCModel("linearSVC", Vectors.dense(0.0), 0.0) + ParamsSuite.checkParams(model) + } + + test("linear svc: default params") { + val lsvc = new LinearSVC() + assert(lsvc.getRegParam === 0.0) + assert(lsvc.getMaxIter === 100) + assert(lsvc.getFitIntercept) + assert(lsvc.getTol === 1E-6) + assert(lsvc.getStandardization) + assert(!lsvc.isDefined(lsvc.weightCol)) + assert(lsvc.getThreshold === 0.0) + assert(lsvc.getAggregationDepth === 2) + assert(lsvc.getLabelCol === "label") + assert(lsvc.getFeaturesCol === "features") + assert(lsvc.getPredictionCol === "prediction") + assert(lsvc.getRawPredictionCol === "rawPrediction") + val model = lsvc.setMaxIter(5).fit(smallBinaryDataset) + model.transform(smallBinaryDataset) + .select("label", "prediction", "rawPrediction") + .collect() + assert(model.getThreshold === 0.0) + assert(model.getFeaturesCol === "features") + assert(model.getPredictionCol === "prediction") + assert(model.getRawPredictionCol === "rawPrediction") + assert(model.intercept !== 0.0) + assert(model.hasParent) + assert(model.numFeatures === 2) + + MLTestingUtils.checkCopyAndUids(lsvc, model) + } + + test("LinearSVC threshold acts on rawPrediction") { + val lsvc = + new LinearSVCModel(uid = "myLSVCM", coefficients = Vectors.dense(1.0), intercept = 0.0) + val df = spark.createDataFrame(Seq( + (1, Vectors.dense(1e-7)), + (0, Vectors.dense(0.0)), + (-1, Vectors.dense(-1e-7)))).toDF("id", "features") + + def checkOneResult( + model: LinearSVCModel, + threshold: Double, + expected: Set[(Int, Double)]): Unit = { + model.setThreshold(threshold) + val results = model.transform(df).select("id", "prediction").collect() + .map(r => (r.getInt(0), r.getDouble(1))) + .toSet + assert(results === expected, s"Failed for threshold = $threshold") + } + + def checkResults(threshold: Double, expected: Set[(Int, Double)]): Unit = { + // Check via code path using Classifier.raw2prediction + lsvc.setRawPredictionCol("rawPrediction") + checkOneResult(lsvc, threshold, expected) + // Check via code path using Classifier.predict + lsvc.setRawPredictionCol("") + checkOneResult(lsvc, threshold, expected) + } + + checkResults(0.0, Set((1, 1.0), (0, 0.0), (-1, 0.0))) + checkResults(Double.PositiveInfinity, Set((1, 0.0), (0, 0.0), (-1, 0.0))) + checkResults(Double.NegativeInfinity, Set((1, 1.0), (0, 1.0), (-1, 1.0))) + } + + test("linear svc doesn't fit intercept when fitIntercept is off") { + val lsvc = new LinearSVC().setFitIntercept(false).setMaxIter(5) + val model = lsvc.fit(smallBinaryDataset) + assert(model.intercept === 0.0) + + val lsvc2 = new LinearSVC().setFitIntercept(true).setMaxIter(5) + val model2 = lsvc2.fit(smallBinaryDataset) + assert(model2.intercept !== 0.0) + } + + test("sparse coefficients in HingeAggregator") { + val bcCoefficients = spark.sparkContext.broadcast(Vectors.sparse(2, Array(0), Array(1.0))) + val bcFeaturesStd = spark.sparkContext.broadcast(Array(1.0)) + val agg = new HingeAggregator(bcFeaturesStd, true)(bcCoefficients) + val thrown = withClue("LinearSVCAggregator cannot handle sparse coefficients") { + intercept[IllegalArgumentException] { + agg.add(Instance(1.0, 1.0, Vectors.dense(1.0))) + } + } + assert(thrown.getMessage.contains("coefficients only supports dense")) + + bcCoefficients.destroy(blocking = false) + bcFeaturesStd.destroy(blocking = false) + } + + test("linearSVC with sample weights") { + def modelEquals(m1: LinearSVCModel, m2: LinearSVCModel): Unit = { + assert(m1.coefficients ~== m2.coefficients absTol 0.05) + assert(m1.intercept ~== m2.intercept absTol 0.05) + } + + val estimator = new LinearSVC().setRegParam(0.01).setTol(0.01) + val dataset = smallBinaryDataset + MLTestingUtils.testArbitrarilyScaledWeights[LinearSVCModel, LinearSVC]( + dataset.as[LabeledPoint], estimator, modelEquals) + MLTestingUtils.testOutliersWithSmallWeights[LinearSVCModel, LinearSVC]( + dataset.as[LabeledPoint], estimator, 2, modelEquals, outlierRatio = 3) + MLTestingUtils.testOversamplingVsWeighting[LinearSVCModel, LinearSVC]( + dataset.as[LabeledPoint], estimator, modelEquals, 42L) + } + + test("linearSVC comparison with R e1071 and scikit-learn") { + val trainer1 = new LinearSVC() + .setRegParam(0.00002) // set regParam = 2.0 / datasize / c + .setMaxIter(200) + .setTol(1e-4) + val model1 = trainer1.fit(binaryDataset) + + /* + Use the following R code to load the data and train the model using glmnet package. + + library(e1071) + data <- read.csv("path/target/tmp/LinearSVC/binaryDataset/part-00000", header=FALSE) + label <- factor(data$V1) + features <- as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) + svm_model <- svm(features, label, type='C', kernel='linear', cost=10, scale=F, tolerance=1e-4) + w <- -t(svm_model$coefs) %*% svm_model$SV + w + svm_model$rho + + > w + data.V2 data.V3 data.V4 data.V5 + [1,] 7.310338 14.89741 22.21005 29.83508 + > svm_model$rho + [1] 7.440177 + + */ + val coefficientsR = Vectors.dense(7.310338, 14.89741, 22.21005, 29.83508) + val interceptR = 7.440177 + assert(model1.intercept ~== interceptR relTol 1E-2) + assert(model1.coefficients ~== coefficientsR relTol 1E-2) + + /* + Use the following python code to load the data and train the model using scikit-learn package. + + import numpy as np + from sklearn import svm + f = open("path/target/tmp/LinearSVC/binaryDataset/part-00000") + data = np.loadtxt(f, delimiter=",") + X = data[:, 1:] # select columns 1 through end + y = data[:, 0] # select column 0 as label + clf = svm.LinearSVC(fit_intercept=True, C=10, loss='hinge', tol=1e-4, random_state=42) + m = clf.fit(X, y) + print m.coef_ + print m.intercept_ + + [[ 7.24690165 14.77029087 21.99924004 29.5575729 ]] + [ 7.36947518] + */ + + val coefficientsSK = Vectors.dense(7.24690165, 14.77029087, 21.99924004, 29.5575729) + val interceptSK = 7.36947518 + assert(model1.intercept ~== interceptSK relTol 1E-3) + assert(model1.coefficients ~== coefficientsSK relTol 4E-3) + } + + test("read/write: SVM") { + def checkModelData(model: LinearSVCModel, model2: LinearSVCModel): Unit = { + assert(model.intercept === model2.intercept) + assert(model.coefficients === model2.coefficients) + assert(model.numFeatures === model2.numFeatures) + } + val svm = new LinearSVC() + testEstimatorAndModelReadWrite(svm, smallBinaryDataset, LinearSVCSuite.allParamSettings, + LinearSVCSuite.allParamSettings, checkModelData) + } +} + +object LinearSVCSuite { + + val allParamSettings: Map[String, Any] = Map( + "regParam" -> 0.01, + "maxIter" -> 2, // intentionally small + "fitIntercept" -> true, + "tol" -> 0.8, + "standardization" -> false, + "threshold" -> 0.6, + "predictionCol" -> "myPredict", + "rawPredictionCol" -> "myRawPredict", + "aggregationDepth" -> 3 + ) + + // Generate noisy input of the form Y = signum(x.dot(weights) + intercept + noise) + def generateSVMInput( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + val weightsMat = new BDV(weights) + val x = Array.fill[Array[Double]](nPoints)( + Array.fill[Double](weights.length)(rnd.nextDouble() * 2.0 - 1.0)) + val y = x.map { xi => + val yD = new BDV(xi).dot(weightsMat) + intercept + 0.01 * rnd.nextGaussian() + if (yD > 0) 1.0 else 0.0 + } + y.zip(x).map(p => LabeledPoint(p._1, Vectors.dense(p._2))) + } + + def checkModels(model1: LinearSVCModel, model2: LinearSVCModel): Unit = { + assert(model1.intercept == model2.intercept) + assert(model1.coefficients.equals(model2.coefficients)) + } + +} + diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 325faf37e8ee..14f550890d23 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -17,45 +17,60 @@ package org.apache.spark.ml.classification +import scala.collection.JavaConverters._ import scala.language.existentials import scala.util.Random - -import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.Instance -import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import scala.util.control.Breaks._ + +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.attribute.NominalAttribute +import org.apache.spark.ml.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.feature.{Instance, LabeledPoint} +import org.apache.spark.ml.linalg.{DenseMatrix, Matrices, Matrix, SparseMatrix, Vector, Vectors} +import org.apache.spark.ml.optim.aggregator.LogisticAggregator +import org.apache.spark.ml.param.{ParamMap, ParamsSuite} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.functions.{col, lit, rand} +import org.apache.spark.sql.types.LongType + +class LogisticRegressionSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { -class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { + import testImplicits._ - @transient var dataset: DataFrame = _ - @transient var binaryDataset: DataFrame = _ + private val seed = 42 + @transient var smallBinaryDataset: Dataset[_] = _ + @transient var smallMultinomialDataset: Dataset[_] = _ + @transient var binaryDataset: Dataset[_] = _ + @transient var multinomialDataset: Dataset[_] = _ + @transient var multinomialDatasetWithZeroVar: Dataset[_] = _ private val eps: Double = 1e-5 override def beforeAll(): Unit = { super.beforeAll() - dataset = sqlContext.createDataFrame(generateLogisticInput(1.0, 1.0, nPoints = 100, seed = 42)) + smallBinaryDataset = generateLogisticInput(1.0, 1.0, nPoints = 100, seed = seed).toDF() + + smallMultinomialDataset = { + val nPoints = 100 + val coefficients = Array( + -0.57997, 0.912083, -0.371077, + -0.16624, -0.84355, -0.048509) + + val xMean = Array(5.843, 3.057) + val xVariance = Array(0.6856, 0.1899) + + val testData = generateMultinomialLogisticInput( + coefficients, xMean, xVariance, addIntercept = true, nPoints, seed) + + val df = sc.parallelize(testData, 4).toDF() + df.cache() + df + } - /* - Here is the instruction describing how to export the test data into CSV format - so we can validate the training accuracy compared with R's glmnet package. - - import org.apache.spark.mllib.classification.LogisticRegressionSuite - val nPoints = 10000 - val coefficients = Array(-0.57997, 0.912083, -0.371077, -0.819866, 2.688191) - val xMean = Array(5.843, 3.057, 3.758, 1.199) - val xVariance = Array(0.6856, 0.1899, 3.116, 0.581) - val data = sc.parallelize(LogisticRegressionSuite.generateMultinomialLogisticInput( - coefficients, xMean, xVariance, true, nPoints, 42), 1) - data.map(x=> x.label + ", " + x.features(0) + ", " + x.features(1) + ", " - + x.features(2) + ", " + x.features(3)).saveAsTextFile("path") - */ binaryDataset = { val nPoints = 10000 val coefficients = Array(-0.57997, 0.912083, -0.371077, -0.819866, 2.688191) @@ -63,10 +78,63 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { val xVariance = Array(0.6856, 0.1899, 3.116, 0.581) val testData = - generateMultinomialLogisticInput(coefficients, xMean, xVariance, true, nPoints, 42) + generateMultinomialLogisticInput(coefficients, xMean, xVariance, + addIntercept = true, nPoints, seed) - sqlContext.createDataFrame(sc.parallelize(testData, 4)) + sc.parallelize(testData, 4).toDF().withColumn("weight", rand(seed)) } + + multinomialDataset = { + val nPoints = 10000 + val coefficients = Array( + -0.57997, 0.912083, -0.371077, -0.819866, 2.688191, + -0.16624, -0.84355, -0.048509, -0.301789, 4.170682) + + val xMean = Array(5.843, 3.057, 3.758, 1.199) + val xVariance = Array(0.6856, 0.1899, 3.116, 0.581) + + val testData = generateMultinomialLogisticInput( + coefficients, xMean, xVariance, addIntercept = true, nPoints, seed) + + val df = sc.parallelize(testData, 4).toDF().withColumn("weight", rand(seed)) + df.cache() + df + } + + multinomialDatasetWithZeroVar = { + val nPoints = 100 + val coefficients = Array( + -0.57997, 0.912083, -0.371077, + -0.16624, -0.84355, -0.048509) + + val xMean = Array(5.843, 3.0) + val xVariance = Array(0.6856, 0.0) + + val testData = generateMultinomialLogisticInput( + coefficients, xMean, xVariance, addIntercept = true, nPoints, seed) + + val df = sc.parallelize(testData, 4).toDF().withColumn("weight", lit(1.0)) + df.cache() + df + } + } + + /** + * Enable the ignored test to export the dataset into CSV format, + * so we can validate the training accuracy compared with R's glmnet package. + */ + ignore("export test data into CSV format") { + binaryDataset.rdd.map { case Row(label: Double, features: Vector, weight: Double) => + label + "," + weight + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile("target/tmp/LogisticRegressionSuite/binaryDataset") + multinomialDataset.rdd.map { case Row(label: Double, features: Vector, weight: Double) => + label + "," + weight + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile("target/tmp/LogisticRegressionSuite/multinomialDataset") + multinomialDatasetWithZeroVar.rdd.map { + case Row(label: Double, features: Vector, weight: Double) => + label + "," + weight + "," + features.toArray.mkString(",") + }.repartition(1) + .saveAsTextFile("target/tmp/LogisticRegressionSuite/multinomialDatasetWithZeroVar") } test("params") { @@ -82,11 +150,12 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { assert(lr.getPredictionCol === "prediction") assert(lr.getRawPredictionCol === "rawPrediction") assert(lr.getProbabilityCol === "probability") - assert(lr.getWeightCol === "") + assert(lr.getFamily === "auto") + assert(!lr.isDefined(lr.weightCol)) assert(lr.getFitIntercept) assert(lr.getStandardization) - val model = lr.fit(dataset) - model.transform(dataset) + val model = lr.fit(smallBinaryDataset) + model.transform(smallBinaryDataset) .select("label", "probability", "prediction", "rawPrediction") .collect() assert(model.getThreshold === 0.5) @@ -96,10 +165,125 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { assert(model.getProbabilityCol === "probability") assert(model.intercept !== 0.0) assert(model.hasParent) + + MLTestingUtils.checkCopyAndUids(lr, model) + assert(model.hasSummary) + val copiedModel = model.copy(ParamMap.empty) + assert(copiedModel.hasSummary) + model.setSummary(None) + assert(!model.hasSummary) + } + + test("logistic regression: illegal params") { + val lowerBoundsOnCoefficients = Matrices.dense(1, 4, Array(1.0, 0.0, 1.0, 0.0)) + val upperBoundsOnCoefficients1 = Matrices.dense(1, 4, Array(0.0, 1.0, 1.0, 0.0)) + val upperBoundsOnCoefficients2 = Matrices.dense(1, 3, Array(1.0, 0.0, 1.0)) + val lowerBoundsOnIntercepts = Vectors.dense(1.0) + + // Work well when only set bound in one side. + new LogisticRegression() + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .fit(binaryDataset) + + withClue("bound constrained optimization only supports L2 regularization") { + intercept[IllegalArgumentException] { + new LogisticRegression() + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setElasticNetParam(1.0) + .fit(binaryDataset) + } + } + + withClue("lowerBoundsOnCoefficients should less than or equal to upperBoundsOnCoefficients") { + intercept[IllegalArgumentException] { + new LogisticRegression() + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setUpperBoundsOnCoefficients(upperBoundsOnCoefficients1) + .fit(binaryDataset) + } + } + + withClue("the coefficients bound matrix mismatched with shape (1, number of features)") { + intercept[IllegalArgumentException] { + new LogisticRegression() + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setUpperBoundsOnCoefficients(upperBoundsOnCoefficients2) + .fit(binaryDataset) + } + } + + withClue("bounds on intercepts should not be set if fitting without intercept") { + intercept[IllegalArgumentException] { + new LogisticRegression() + .setLowerBoundsOnIntercepts(lowerBoundsOnIntercepts) + .setFitIntercept(false) + .fit(binaryDataset) + } + } + } + + test("empty probabilityCol or predictionCol") { + val lr = new LogisticRegression().setMaxIter(1) + val datasetFieldNames = smallBinaryDataset.schema.fieldNames.toSet + def checkSummarySchema(model: LogisticRegressionModel, columns: Seq[String]): Unit = { + val fieldNames = model.summary.predictions.schema.fieldNames + assert(model.hasSummary) + assert(datasetFieldNames.subsetOf(fieldNames.toSet)) + columns.foreach { c => assert(fieldNames.exists(_.startsWith(c))) } + } + // check that the summary model adds the appropriate columns + Seq(("binomial", smallBinaryDataset), ("multinomial", smallMultinomialDataset)).foreach { + case (family, dataset) => + lr.setFamily(family) + lr.setProbabilityCol("").setPredictionCol("prediction") + val modelNoProb = lr.fit(dataset) + checkSummarySchema(modelNoProb, Seq("probability_")) + + lr.setProbabilityCol("probability").setPredictionCol("") + val modelNoPred = lr.fit(dataset) + checkSummarySchema(modelNoPred, Seq("prediction_")) + + lr.setProbabilityCol("").setPredictionCol("") + val modelNoPredNoProb = lr.fit(dataset) + checkSummarySchema(modelNoPredNoProb, Seq("prediction_", "probability_")) + } + } + + test("check summary types for binary and multiclass") { + val lr = new LogisticRegression() + .setFamily("binomial") + .setMaxIter(1) + + val blorModel = lr.fit(smallBinaryDataset) + assert(blorModel.summary.isInstanceOf[BinaryLogisticRegressionTrainingSummary]) + assert(blorModel.summary.asBinary.isInstanceOf[BinaryLogisticRegressionSummary]) + assert(blorModel.binarySummary.isInstanceOf[BinaryLogisticRegressionTrainingSummary]) + + val mlorModel = lr.setFamily("multinomial").fit(smallMultinomialDataset) + assert(mlorModel.summary.isInstanceOf[LogisticRegressionTrainingSummary]) + withClue("cannot get binary summary for multiclass model") { + intercept[RuntimeException] { + mlorModel.binarySummary + } + } + withClue("cannot cast summary to binary summary multiclass model") { + intercept[RuntimeException] { + mlorModel.summary.asBinary + } + } + + val mlorBinaryModel = lr.setFamily("multinomial").fit(smallBinaryDataset) + assert(mlorBinaryModel.summary.isInstanceOf[BinaryLogisticRegressionTrainingSummary]) + assert(mlorBinaryModel.binarySummary.isInstanceOf[BinaryLogisticRegressionTrainingSummary]) + + val blorSummary = blorModel.evaluate(smallBinaryDataset) + val mlorSummary = mlorModel.evaluate(smallMultinomialDataset) + assert(blorSummary.isInstanceOf[BinaryLogisticRegressionSummary]) + assert(mlorSummary.isInstanceOf[LogisticRegressionSummary]) } test("setThreshold, getThreshold") { - val lr = new LogisticRegression + val lr = new LogisticRegression().setFamily("binomial") // default assert(lr.getThreshold === 0.5, "LogisticRegression.threshold should default to 0.5") withClue("LogisticRegression should not have thresholds set by default.") { @@ -116,7 +300,7 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { lr.setThreshold(0.5) assert(lr.getThresholds === Array(0.5, 0.5)) // Set via thresholds - val lr2 = new LogisticRegression + val lr2 = new LogisticRegression().setFamily("binomial") lr2.setThresholds(Array(0.3, 0.7)) val expectedThreshold = 1.0 / (1.0 + 0.3 / 0.7) assert(lr2.getThreshold ~== expectedThreshold relTol 1E-7) @@ -130,21 +314,77 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { // thresholds and threshold must be consistent: values withClue("fit with ParamMap should throw error if threshold, thresholds do not match.") { intercept[IllegalArgumentException] { - val lr2model = lr2.fit(dataset, + lr2.fit(smallBinaryDataset, + lr2.thresholds -> Array(0.3, 0.7), lr2.threshold -> (expectedThreshold / 2.0)) + } + } + withClue("fit with ParamMap should throw error if threshold, thresholds do not match.") { + intercept[IllegalArgumentException] { + val lr2model = lr2.fit(smallBinaryDataset, lr2.thresholds -> Array(0.3, 0.7), lr2.threshold -> (expectedThreshold / 2.0)) lr2model.getThreshold } } } + test("thresholds prediction") { + val blr = new LogisticRegression().setFamily("binomial") + val binaryModel = blr.fit(smallBinaryDataset) + + binaryModel.setThreshold(1.0) + val binaryZeroPredictions = + binaryModel.transform(smallBinaryDataset).select("prediction").collect() + assert(binaryZeroPredictions.forall(_.getDouble(0) === 0.0)) + + binaryModel.setThreshold(0.0) + val binaryOnePredictions = + binaryModel.transform(smallBinaryDataset).select("prediction").collect() + assert(binaryOnePredictions.forall(_.getDouble(0) === 1.0)) + + + val mlr = new LogisticRegression().setFamily("multinomial") + val model = mlr.fit(smallMultinomialDataset) + val basePredictions = model.transform(smallMultinomialDataset).select("prediction").collect() + + // should predict all zeros + model.setThresholds(Array(1, 1000, 1000)) + val zeroPredictions = model.transform(smallMultinomialDataset).select("prediction").collect() + assert(zeroPredictions.forall(_.getDouble(0) === 0.0)) + + // should predict all ones + model.setThresholds(Array(1000, 1, 1000)) + val onePredictions = model.transform(smallMultinomialDataset).select("prediction").collect() + assert(onePredictions.forall(_.getDouble(0) === 1.0)) + + // should predict all twos + model.setThresholds(Array(1000, 1000, 1)) + val twoPredictions = model.transform(smallMultinomialDataset).select("prediction").collect() + assert(twoPredictions.forall(_.getDouble(0) === 2.0)) + + // constant threshold scaling is the same as no thresholds + model.setThresholds(Array(1000, 1000, 1000)) + val scaledPredictions = model.transform(smallMultinomialDataset).select("prediction").collect() + assert(scaledPredictions.zip(basePredictions).forall { case (scaled, base) => + scaled.getDouble(0) === base.getDouble(0) + }) + + // force it to use the predict method + model.setRawPredictionCol("").setProbabilityCol("").setThresholds(Array(0, 1, 1)) + val predictionsWithPredict = + model.transform(smallMultinomialDataset).select("prediction").collect() + assert(predictionsWithPredict.forall(_.getDouble(0) === 0.0)) + } + test("logistic regression doesn't fit intercept when fitIntercept is off") { - val lr = new LogisticRegression + val lr = new LogisticRegression().setFamily("binomial") lr.setFitIntercept(false) - val model = lr.fit(dataset) + val model = lr.fit(smallBinaryDataset) assert(model.intercept === 0.0) - // copied model must have the same parent. - MLTestingUtils.checkCopy(model) + val mlr = new LogisticRegression().setFamily("multinomial") + mlr.setFitIntercept(false) + val mlrModel = mlr.fit(smallMultinomialDataset) + assert(mlrModel.interceptVector === Vectors.sparse(3, Seq())) } test("logistic regression with setters") { @@ -154,7 +394,7 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { .setRegParam(1.0) .setThreshold(0.6) .setProbabilityCol("myProbability") - val model = lr.fit(dataset) + val model = lr.fit(smallBinaryDataset) val parent = model.parent.asInstanceOf[LogisticRegression] assert(parent.getMaxIter === 10) assert(parent.getRegParam === 1.0) @@ -163,16 +403,16 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { // Modify model params, and check that the params worked. model.setThreshold(1.0) - val predAllZero = model.transform(dataset) + val predAllZero = model.transform(smallBinaryDataset) .select("prediction", "myProbability") .collect() .map { case Row(pred: Double, prob: Vector) => pred } assert(predAllZero.forall(_ === 0), s"With threshold=1.0, expected predictions to be all 0, but only" + - s" ${predAllZero.count(_ === 0)} of ${dataset.count()} were 0.") + s" ${predAllZero.count(_ === 0)} of ${smallBinaryDataset.count()} were 0.") // Call transform with params, and check that the params worked. val predNotAllZero = - model.transform(dataset, model.threshold -> 0.0, + model.transform(smallBinaryDataset, model.threshold -> 0.0, model.probabilityCol -> "myProb") .select("prediction", "myProb") .collect() @@ -181,7 +421,7 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { // Call fit() with new params, and check as many params as we can. lr.setThresholds(Array(0.6, 0.4)) - val model2 = lr.fit(dataset, lr.maxIter -> 5, lr.regParam -> 0.1, + val model2 = lr.fit(smallBinaryDataset, lr.maxIter -> 5, lr.regParam -> 0.1, lr.probabilityCol -> "theProb") val parent2 = model2.parent.asInstanceOf[LogisticRegression] assert(parent2.getMaxIter === 5) @@ -191,17 +431,93 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { assert(model2.getProbabilityCol === "theProb") } - test("logistic regression: Predictor, Classifier methods") { - val sqlContext = this.sqlContext - val lr = new LogisticRegression + test("multinomial logistic regression: Predictor, Classifier methods") { + val sqlContext = smallMultinomialDataset.sqlContext + import sqlContext.implicits._ + val mlr = new LogisticRegression().setFamily("multinomial") + + val model = mlr.fit(smallMultinomialDataset) + assert(model.numClasses === 3) + val numFeatures = smallMultinomialDataset.select("features").first().getAs[Vector](0).size + assert(model.numFeatures === numFeatures) + + val results = model.transform(smallMultinomialDataset) + // check that raw prediction is coefficients dot features + intercept + results.select("rawPrediction", "features").collect().foreach { + case Row(raw: Vector, features: Vector) => + assert(raw.size === 3) + val margins = Array.tabulate(3) { k => + var margin = 0.0 + features.foreachActive { (index, value) => + margin += value * model.coefficientMatrix(k, index) + } + margin += model.interceptVector(k) + margin + } + assert(raw ~== Vectors.dense(margins) relTol eps) + } + + // Compare rawPrediction with probability + results.select("rawPrediction", "probability").collect().foreach { + case Row(raw: Vector, prob: Vector) => + assert(raw.size === 3) + assert(prob.size === 3) + val max = raw.toArray.max + val subtract = if (max > 0) max else 0.0 + val sum = raw.toArray.map(x => math.exp(x - subtract)).sum + val probFromRaw0 = math.exp(raw(0) - subtract) / sum + val probFromRaw1 = math.exp(raw(1) - subtract) / sum + assert(prob(0) ~== probFromRaw0 relTol eps) + assert(prob(1) ~== probFromRaw1 relTol eps) + assert(prob(2) ~== 1.0 - probFromRaw1 - probFromRaw0 relTol eps) + } + + // Compare prediction with probability + results.select("prediction", "probability").collect().foreach { + case Row(pred: Double, prob: Vector) => + val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2 + assert(pred == predFromProb) + } + + // force it to use raw2prediction + model.setRawPredictionCol("rawPrediction").setProbabilityCol("") + val resultsUsingRaw2Predict = + model.transform(smallMultinomialDataset).select("prediction").as[Double].collect() + resultsUsingRaw2Predict.zip(results.select("prediction").as[Double].collect()).foreach { + case (pred1, pred2) => assert(pred1 === pred2) + } + + // force it to use probability2prediction + model.setRawPredictionCol("").setProbabilityCol("probability") + val resultsUsingProb2Predict = + model.transform(smallMultinomialDataset).select("prediction").as[Double].collect() + resultsUsingProb2Predict.zip(results.select("prediction").as[Double].collect()).foreach { + case (pred1, pred2) => assert(pred1 === pred2) + } + + // force it to use predict + model.setRawPredictionCol("").setProbabilityCol("") + val resultsUsingPredict = + model.transform(smallMultinomialDataset).select("prediction").as[Double].collect() + resultsUsingPredict.zip(results.select("prediction").as[Double].collect()).foreach { + case (pred1, pred2) => assert(pred1 === pred2) + } - val model = lr.fit(dataset) + ProbabilisticClassifierSuite.testPredictMethods[ + Vector, LogisticRegressionModel](model, smallMultinomialDataset) + } + + test("binary logistic regression: Predictor, Classifier methods") { + val sqlContext = smallBinaryDataset.sqlContext + import sqlContext.implicits._ + val lr = new LogisticRegression().setFamily("binomial") + + val model = lr.fit(smallBinaryDataset) assert(model.numClasses === 2) - val numFeatures = dataset.select("features").first().getAs[Vector](0).size + val numFeatures = smallBinaryDataset.select("features").first().getAs[Vector](0).size assert(model.numFeatures === numFeatures) - val threshold = model.getThreshold - val results = model.transform(dataset) + val results = model.transform(smallBinaryDataset) // Compare rawPrediction with probability results.select("rawPrediction", "probability").collect().foreach { @@ -219,6 +535,100 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2 assert(pred == predFromProb) } + + // force it to use raw2prediction + model.setRawPredictionCol("rawPrediction").setProbabilityCol("") + val resultsUsingRaw2Predict = + model.transform(smallBinaryDataset).select("prediction").as[Double].collect() + resultsUsingRaw2Predict.zip(results.select("prediction").as[Double].collect()).foreach { + case (pred1, pred2) => assert(pred1 === pred2) + } + + // force it to use probability2prediction + model.setRawPredictionCol("").setProbabilityCol("probability") + val resultsUsingProb2Predict = + model.transform(smallBinaryDataset).select("prediction").as[Double].collect() + resultsUsingProb2Predict.zip(results.select("prediction").as[Double].collect()).foreach { + case (pred1, pred2) => assert(pred1 === pred2) + } + + // force it to use predict + model.setRawPredictionCol("").setProbabilityCol("") + val resultsUsingPredict = + model.transform(smallBinaryDataset).select("prediction").as[Double].collect() + resultsUsingPredict.zip(results.select("prediction").as[Double].collect()).foreach { + case (pred1, pred2) => assert(pred1 === pred2) + } + + ProbabilisticClassifierSuite.testPredictMethods[ + Vector, LogisticRegressionModel](model, smallBinaryDataset) + } + + test("coefficients and intercept methods") { + val mlr = new LogisticRegression().setMaxIter(1).setFamily("multinomial") + val mlrModel = mlr.fit(smallMultinomialDataset) + val thrownCoef = intercept[SparkException] { + mlrModel.coefficients + } + val thrownIntercept = intercept[SparkException] { + mlrModel.intercept + } + assert(thrownCoef.getMessage().contains("use coefficientMatrix instead")) + assert(thrownIntercept.getMessage().contains("use interceptVector instead")) + + val blr = new LogisticRegression().setMaxIter(1).setFamily("binomial") + val blrModel = blr.fit(smallBinaryDataset) + assert(blrModel.coefficients.size === 1) + assert(blrModel.intercept !== 0.0) + } + + test("sparse coefficients in LogisticAggregator") { + val bcCoefficientsBinary = spark.sparkContext.broadcast(Vectors.sparse(2, Array(0), Array(1.0))) + val bcFeaturesStd = spark.sparkContext.broadcast(Array(1.0)) + val binaryAgg = new LogisticAggregator(bcFeaturesStd, 2, + fitIntercept = true, multinomial = false)(bcCoefficientsBinary) + val thrownBinary = withClue("binary logistic aggregator cannot handle sparse coefficients") { + intercept[IllegalArgumentException] { + binaryAgg.add(Instance(1.0, 1.0, Vectors.dense(1.0))) + } + } + assert(thrownBinary.getMessage.contains("coefficients only supports dense")) + + val bcCoefficientsMulti = spark.sparkContext.broadcast(Vectors.sparse(6, Array(0), Array(1.0))) + val multinomialAgg = new LogisticAggregator(bcFeaturesStd, 3, + fitIntercept = true, multinomial = true)(bcCoefficientsMulti) + val thrown = withClue("multinomial logistic aggregator cannot handle sparse coefficients") { + intercept[IllegalArgumentException] { + multinomialAgg.add(Instance(1.0, 1.0, Vectors.dense(1.0))) + } + } + assert(thrown.getMessage.contains("coefficients only supports dense")) + bcCoefficientsBinary.destroy(blocking = false) + bcFeaturesStd.destroy(blocking = false) + bcCoefficientsMulti.destroy(blocking = false) + } + + test("overflow prediction for multiclass") { + val model = new LogisticRegressionModel("mLogReg", + Matrices.dense(3, 2, Array(0.0, 0.0, 0.0, 1.0, 2.0, 3.0)), + Vectors.dense(0.0, 0.0, 0.0), 3, true) + val overFlowData = Seq( + LabeledPoint(1.0, Vectors.dense(0.0, 1000.0)), + LabeledPoint(1.0, Vectors.dense(0.0, -1.0)) + ).toDF() + val results = model.transform(overFlowData).select("rawPrediction", "probability").collect() + + // probabilities are correct when margins have to be adjusted + val raw1 = results(0).getAs[Vector](0) + val prob1 = results(0).getAs[Vector](1) + assert(raw1 === Vectors.dense(1000.0, 2000.0, 3000.0)) + assert(prob1 ~== Vectors.dense(0.0, 0.0, 1.0) absTol eps) + + // probabilities are correct when margins don't have to be adjusted + val raw2 = results(1).getAs[Vector](0) + val prob2 = results(1).getAs[Vector](1) + assert(raw2 === Vectors.dense(-1.0, -2.0, -3.0)) + assert(prob2 ~== Vectors.dense(0.66524096, 0.24472847, 0.09003057) relTol eps) } test("MultiClassSummarizer") { @@ -246,6 +656,10 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { assert(summarizer4.countInvalid === 2) assert(summarizer4.numClasses === 4) + val summarizer5 = new MultiClassSummarizer + assert(summarizer5.histogram.isEmpty) + assert(summarizer5.numClasses === 0) + // small map merges large one val summarizerA = summarizer1.merge(summarizer2) assert(summarizerA.hashCode() === summarizer2.hashCode()) @@ -285,31 +699,35 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { test("binary logistic regression with intercept without regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(true).setStandardization(true) + .setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(true).setStandardization(false) + .setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. + Use the following R code to load the data and train the model using glmnet package. + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0, + lambda = 0)) + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 2.7355261 + data.V3 -0.5734389 + data.V4 0.8911736 + data.V5 -0.3878645 + data.V6 -0.8060570 - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 0)) - coefficients - - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 2.8366423 - data.V2 -0.5895848 - data.V3 0.8931147 - data.V4 -0.3925051 - data.V5 -0.7996864 */ - val interceptR = 2.8366423 - val coefficientsR = Vectors.dense(-0.5895848, 0.8931147, -0.3925051, -0.7996864) + val coefficientsR = Vectors.dense(-0.5734389, 0.8911736, -0.3878645, -0.8060570) + val interceptR = 2.7355261 assert(model1.intercept ~== interceptR relTol 1E-3) assert(model1.coefficients ~= coefficientsR relTol 1E-3) @@ -319,413 +737,566 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { assert(model2.coefficients ~= coefficientsR relTol 1E-3) } + test("binary logistic regression with intercept without regularization with bound") { + // Bound constrained optimization with bound on one side. + val upperBoundsOnCoefficients = Matrices.dense(1, 4, Array(1.0, 0.0, 1.0, 0.0)) + val upperBoundsOnIntercepts = Vectors.dense(1.0) + + val trainer1 = new LogisticRegression() + .setUpperBoundsOnCoefficients(upperBoundsOnCoefficients) + .setUpperBoundsOnIntercepts(upperBoundsOnIntercepts) + .setFitIntercept(true) + .setStandardization(true) + .setWeightCol("weight") + val trainer2 = new LogisticRegression() + .setUpperBoundsOnCoefficients(upperBoundsOnCoefficients) + .setUpperBoundsOnIntercepts(upperBoundsOnIntercepts) + .setFitIntercept(true) + .setStandardization(false) + .setWeightCol("weight") + + val model1 = trainer1.fit(binaryDataset) + val model2 = trainer2.fit(binaryDataset) + + // The solution is generated by https://github.com/yanboliang/bound-optimization. + val coefficientsExpected1 = Vectors.dense(0.06079437, 0.0, -0.26351059, -0.59102199) + val interceptExpected1 = 1.0 + + assert(model1.intercept ~== interceptExpected1 relTol 1E-3) + assert(model1.coefficients ~= coefficientsExpected1 relTol 1E-3) + + // Without regularization, with or without standardization will converge to the same solution. + assert(model2.intercept ~== interceptExpected1 relTol 1E-3) + assert(model2.coefficients ~= coefficientsExpected1 relTol 1E-3) + + // Bound constrained optimization with bound on both side. + val lowerBoundsOnCoefficients = Matrices.dense(1, 4, Array(0.0, -1.0, 0.0, -1.0)) + val lowerBoundsOnIntercepts = Vectors.dense(0.0) + + val trainer3 = new LogisticRegression() + .setUpperBoundsOnCoefficients(upperBoundsOnCoefficients) + .setUpperBoundsOnIntercepts(upperBoundsOnIntercepts) + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setLowerBoundsOnIntercepts(lowerBoundsOnIntercepts) + .setFitIntercept(true) + .setStandardization(true) + .setWeightCol("weight") + val trainer4 = new LogisticRegression() + .setUpperBoundsOnCoefficients(upperBoundsOnCoefficients) + .setUpperBoundsOnIntercepts(upperBoundsOnIntercepts) + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setLowerBoundsOnIntercepts(lowerBoundsOnIntercepts) + .setFitIntercept(true) + .setStandardization(false) + .setWeightCol("weight") + + val model3 = trainer3.fit(binaryDataset) + val model4 = trainer4.fit(binaryDataset) + + // The solution is generated by https://github.com/yanboliang/bound-optimization. + val coefficientsExpected3 = Vectors.dense(0.0, 0.0, 0.0, -0.71708632) + val interceptExpected3 = 0.58776113 + + assert(model3.intercept ~== interceptExpected3 relTol 1E-3) + assert(model3.coefficients ~= coefficientsExpected3 relTol 1E-3) + + // Without regularization, with or without standardization will converge to the same solution. + assert(model4.intercept ~== interceptExpected3 relTol 1E-3) + assert(model4.coefficients ~= coefficientsExpected3 relTol 1E-3) + + // Bound constrained optimization with infinite bound on both side. + val trainer5 = new LogisticRegression() + .setUpperBoundsOnCoefficients(Matrices.dense(1, 4, Array.fill(4)(Double.PositiveInfinity))) + .setUpperBoundsOnIntercepts(Vectors.dense(Double.PositiveInfinity)) + .setLowerBoundsOnCoefficients(Matrices.dense(1, 4, Array.fill(4)(Double.NegativeInfinity))) + .setLowerBoundsOnIntercepts(Vectors.dense(Double.NegativeInfinity)) + .setFitIntercept(true) + .setStandardization(true) + .setWeightCol("weight") + val trainer6 = new LogisticRegression() + .setUpperBoundsOnCoefficients(Matrices.dense(1, 4, Array.fill(4)(Double.PositiveInfinity))) + .setUpperBoundsOnIntercepts(Vectors.dense(Double.PositiveInfinity)) + .setLowerBoundsOnCoefficients(Matrices.dense(1, 4, Array.fill(4)(Double.NegativeInfinity))) + .setLowerBoundsOnIntercepts(Vectors.dense(Double.NegativeInfinity)) + .setFitIntercept(true) + .setStandardization(false) + .setWeightCol("weight") + + val model5 = trainer5.fit(binaryDataset) + val model6 = trainer6.fit(binaryDataset) + + // The solution is generated by https://github.com/yanboliang/bound-optimization. + // It should be same as unbound constrained optimization with LBFGS. + val coefficientsExpected5 = Vectors.dense(-0.5734389, 0.8911736, -0.3878645, -0.8060570) + val interceptExpected5 = 2.7355261 + + assert(model5.intercept ~== interceptExpected5 relTol 1E-3) + assert(model5.coefficients ~= coefficientsExpected5 relTol 1E-3) + + // Without regularization, with or without standardization will converge to the same solution. + assert(model6.intercept ~== interceptExpected5 relTol 1E-3) + assert(model6.coefficients ~= coefficientsExpected5 relTol 1E-3) + } + test("binary logistic regression without intercept without regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(false).setStandardization(true) + .setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(false).setStandardization(false) + .setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = - coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 0, intercept=FALSE)) - coefficients + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0, + lambda = 0, intercept=FALSE)) + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V3 -0.3448461 + data.V4 1.2776453 + data.V5 -0.3539178 + data.V6 -0.7469384 - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - data.V2 -0.3534996 - data.V3 1.2964482 - data.V4 -0.3571741 - data.V5 -0.7407946 */ - val interceptR = 0.0 - val coefficientsR = Vectors.dense(-0.3534996, 1.2964482, -0.3571741, -0.7407946) + val coefficientsR = Vectors.dense(-0.3448461, 1.2776453, -0.3539178, -0.7469384) - assert(model1.intercept ~== interceptR relTol 1E-3) + assert(model1.intercept ~== 0.0 relTol 1E-3) assert(model1.coefficients ~= coefficientsR relTol 1E-2) // Without regularization, with or without standardization should converge to the same solution. - assert(model2.intercept ~== interceptR relTol 1E-3) + assert(model2.intercept ~== 0.0 relTol 1E-3) assert(model2.coefficients ~= coefficientsR relTol 1E-2) } + test("binary logistic regression without intercept without regularization with bound") { + val upperBoundsOnCoefficients = Matrices.dense(1, 4, Array(1.0, 0.0, 1.0, 0.0)).toSparse + + val trainer1 = new LogisticRegression() + .setUpperBoundsOnCoefficients(upperBoundsOnCoefficients) + .setFitIntercept(false) + .setStandardization(true) + .setWeightCol("weight") + val trainer2 = new LogisticRegression() + .setUpperBoundsOnCoefficients(upperBoundsOnCoefficients) + .setFitIntercept(false) + .setStandardization(false) + .setWeightCol("weight") + + val model1 = trainer1.fit(binaryDataset) + val model2 = trainer2.fit(binaryDataset) + + // The solution is generated by https://github.com/yanboliang/bound-optimization. + val coefficientsExpected = Vectors.dense(0.20847553, 0.0, -0.24240289, -0.55568071) + + assert(model1.intercept ~== 0.0 relTol 1E-3) + assert(model1.coefficients ~= coefficientsExpected relTol 1E-3) + + // Without regularization, with or without standardization will converge to the same solution. + assert(model2.intercept ~== 0.0 relTol 1E-3) + assert(model2.coefficients ~= coefficientsExpected relTol 1E-3) + } + test("binary logistic regression with intercept with L1 regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(true) + .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(false) + .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 1, lambda = 0.12)) - coefficients + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 1, + lambda = 0.12, standardize=T)) + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) -0.06775980 + data.V3 . + data.V4 . + data.V5 -0.03933146 + data.V6 -0.03047580 - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) -0.05627428 - data.V2 . - data.V3 . - data.V4 -0.04325749 - data.V5 -0.02481551 */ - val interceptR1 = -0.05627428 - val coefficientsR1 = Vectors.dense(0.0, 0.0, -0.04325749, -0.02481551) + val coefficientsRStd = Vectors.dense(0.0, 0.0, -0.03933146, -0.03047580) + val interceptRStd = -0.06775980 - assert(model1.intercept ~== interceptR1 relTol 1E-2) - assert(model1.coefficients ~= coefficientsR1 absTol 2E-2) + assert(model1.intercept ~== interceptRStd relTol 1E-2) + assert(model1.coefficients ~= coefficientsRStd absTol 2E-2) /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 1, lambda = 0.12, - standardize=FALSE)) - coefficients + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 1, + lambda = 0.12, standardize=F)) + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 0.3544768 + data.V3 . + data.V4 . + data.V5 -0.1626191 + data.V6 . - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 0.3722152 - data.V2 . - data.V3 . - data.V4 -0.1665453 - data.V5 . */ - val interceptR2 = 0.3722152 - val coefficientsR2 = Vectors.dense(0.0, 0.0, -0.1665453, 0.0) + val coefficientsR = Vectors.dense(0.0, 0.0, -0.1626191, 0.0) + val interceptR = 0.3544768 - assert(model2.intercept ~== interceptR2 relTol 1E-2) - assert(model2.coefficients ~= coefficientsR2 absTol 1E-3) + assert(model2.intercept ~== interceptR relTol 1E-2) + assert(model2.coefficients ~== coefficientsR absTol 1E-3) } test("binary logistic regression without intercept with L1 regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(true) + .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(false) + .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 1, lambda = 0.12, - intercept=FALSE)) - coefficients + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="binomial", alpha = 1, + lambda = 0.12, intercept=F, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 1, + lambda = 0.12, intercept=F, standardize=F)) + coefficientsStd + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V3 . + data.V4 . + data.V5 -0.04967635 + data.V6 -0.04757757 + + coefficients + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V3 . + data.V4 . + data.V5 -0.08433195 + data.V6 . - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - data.V2 . - data.V3 . - data.V4 -0.05189203 - data.V5 -0.03891782 */ - val interceptR1 = 0.0 - val coefficientsR1 = Vectors.dense(0.0, 0.0, -0.05189203, -0.03891782) - - assert(model1.intercept ~== interceptR1 relTol 1E-3) - assert(model1.coefficients ~= coefficientsR1 absTol 1E-3) - - /* - Using the following R code to load the data and train the model using glmnet package. + val coefficientsRStd = Vectors.dense(0.0, 0.0, -0.04967635, -0.04757757) - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 1, lambda = 0.12, - intercept=FALSE, standardize=FALSE)) - coefficients - - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - data.V2 . - data.V3 . - data.V4 -0.08420782 - data.V5 . - */ - val interceptR2 = 0.0 - val coefficientsR2 = Vectors.dense(0.0, 0.0, -0.08420782, 0.0) + val coefficientsR = Vectors.dense(0.0, 0.0, -0.08433195, 0.0) - assert(model2.intercept ~== interceptR2 absTol 1E-3) - assert(model2.coefficients ~= coefficientsR2 absTol 1E-3) + assert(model1.intercept ~== 0.0 absTol 1E-3) + assert(model1.coefficients ~= coefficientsRStd absTol 1E-3) + assert(model2.intercept ~== 0.0 absTol 1E-3) + assert(model2.coefficients ~= coefficientsR absTol 1E-3) } test("binary logistic regression with intercept with L2 regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(true) + .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(false) + .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 1.37)) - coefficients + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0, + lambda = 1.37, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0, + lambda = 1.37, standardize=F)) + coefficientsStd + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 0.12707703 + data.V3 -0.06980967 + data.V4 0.10803933 + data.V5 -0.04800404 + data.V6 -0.10165096 + + coefficients + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 0.46613016 + data.V3 -0.04944529 + data.V4 0.02326772 + data.V5 -0.11362772 + data.V6 -0.06312848 - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 0.15021751 - data.V2 -0.07251837 - data.V3 0.10724191 - data.V4 -0.04865309 - data.V5 -0.10062872 */ - val interceptR1 = 0.15021751 - val coefficientsR1 = Vectors.dense(-0.07251837, 0.10724191, -0.04865309, -0.10062872) + val coefficientsRStd = Vectors.dense(-0.06980967, 0.10803933, -0.04800404, -0.10165096) + val interceptRStd = 0.12707703 + val coefficientsR = Vectors.dense(-0.04944529, 0.02326772, -0.11362772, -0.06312848) + val interceptR = 0.46613016 - assert(model1.intercept ~== interceptR1 relTol 1E-3) - assert(model1.coefficients ~= coefficientsR1 relTol 1E-3) + assert(model1.intercept ~== interceptRStd relTol 1E-3) + assert(model1.coefficients ~= coefficientsRStd relTol 1E-3) + assert(model2.intercept ~== interceptR relTol 1E-3) + assert(model2.coefficients ~= coefficientsR relTol 1E-3) + } - /* - Using the following R code to load the data and train the model using glmnet package. + test("binary logistic regression with intercept with L2 regularization with bound") { + val upperBoundsOnCoefficients = Matrices.dense(1, 4, Array(1.0, 0.0, 1.0, 0.0)) + val upperBoundsOnIntercepts = Vectors.dense(1.0) + + val trainer1 = new LogisticRegression() + .setUpperBoundsOnCoefficients(upperBoundsOnCoefficients) + .setUpperBoundsOnIntercepts(upperBoundsOnIntercepts) + .setRegParam(1.37) + .setFitIntercept(true) + .setStandardization(true) + .setWeightCol("weight") + val trainer2 = new LogisticRegression() + .setUpperBoundsOnCoefficients(upperBoundsOnCoefficients) + .setUpperBoundsOnIntercepts(upperBoundsOnIntercepts) + .setRegParam(1.37) + .setFitIntercept(true) + .setStandardization(false) + .setWeightCol("weight") - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 1.37, - standardize=FALSE)) - coefficients + val model1 = trainer1.fit(binaryDataset) + val model2 = trainer2.fit(binaryDataset) - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 0.48657516 - data.V2 -0.05155371 - data.V3 0.02301057 - data.V4 -0.11482896 - data.V5 -0.06266838 - */ - val interceptR2 = 0.48657516 - val coefficientsR2 = Vectors.dense(-0.05155371, 0.02301057, -0.11482896, -0.06266838) + // The solution is generated by https://github.com/yanboliang/bound-optimization. + val coefficientsExpectedWithStd = Vectors.dense(-0.06985003, 0.0, -0.04794278, -0.10168595) + val interceptExpectedWithStd = 0.45750141 + val coefficientsExpected = Vectors.dense(-0.0494524, 0.0, -0.11360797, -0.06313577) + val interceptExpected = 0.53722967 - assert(model2.intercept ~== interceptR2 relTol 1E-3) - assert(model2.coefficients ~= coefficientsR2 relTol 1E-3) + assert(model1.intercept ~== interceptExpectedWithStd relTol 1E-3) + assert(model1.coefficients ~= coefficientsExpectedWithStd relTol 1E-3) + assert(model2.intercept ~== interceptExpected relTol 1E-3) + assert(model2.coefficients ~= coefficientsExpected relTol 1E-3) } test("binary logistic regression without intercept with L2 regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(true) + .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(false) + .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 1.37, - intercept=FALSE)) - coefficients - - 5 x 1 sparse Matrix of class "dgCMatrix" + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0, + lambda = 1.37, intercept=F, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0, + lambda = 1.37, intercept=F, standardize=F)) + coefficientsStd + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V3 -0.06000152 + data.V4 0.12598737 + data.V5 -0.04669009 + data.V6 -0.09941025 + + coefficients + 5 x 1 sparse Matrix of class "dgCMatrix" s0 - (Intercept) . - data.V2 -0.06099165 - data.V3 0.12857058 - data.V4 -0.04708770 - data.V5 -0.09799775 + (Intercept) . + data.V3 -0.005482255 + data.V4 0.048106338 + data.V5 -0.093411640 + data.V6 -0.054149798 + */ - val interceptR1 = 0.0 - val coefficientsR1 = Vectors.dense(-0.06099165, 0.12857058, -0.04708770, -0.09799775) + val coefficientsRStd = Vectors.dense(-0.06000152, 0.12598737, -0.04669009, -0.09941025) + val coefficientsR = Vectors.dense(-0.005482255, 0.048106338, -0.093411640, -0.054149798) - assert(model1.intercept ~== interceptR1 absTol 1E-3) - assert(model1.coefficients ~= coefficientsR1 relTol 1E-2) + assert(model1.intercept ~== 0.0 absTol 1E-3) + assert(model1.coefficients ~= coefficientsRStd relTol 1E-2) + assert(model2.intercept ~== 0.0 absTol 1E-3) + assert(model2.coefficients ~= coefficientsR relTol 1E-2) + } - /* - Using the following R code to load the data and train the model using glmnet package. + test("binary logistic regression without intercept with L2 regularization with bound") { + val upperBoundsOnCoefficients = Matrices.dense(1, 4, Array(1.0, 0.0, 1.0, 0.0)) + + val trainer1 = new LogisticRegression() + .setUpperBoundsOnCoefficients(upperBoundsOnCoefficients) + .setRegParam(1.37) + .setFitIntercept(false) + .setStandardization(true) + .setWeightCol("weight") + val trainer2 = new LogisticRegression() + .setUpperBoundsOnCoefficients(upperBoundsOnCoefficients) + .setRegParam(1.37) + .setFitIntercept(false) + .setStandardization(false) + .setWeightCol("weight") - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 1.37, - intercept=FALSE, standardize=FALSE)) - coefficients + val model1 = trainer1.fit(binaryDataset) + val model2 = trainer2.fit(binaryDataset) - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - data.V2 -0.005679651 - data.V3 0.048967094 - data.V4 -0.093714016 - data.V5 -0.053314311 - */ - val interceptR2 = 0.0 - val coefficientsR2 = Vectors.dense(-0.005679651, 0.048967094, -0.093714016, -0.053314311) + // The solution is generated by https://github.com/yanboliang/bound-optimization. + val coefficientsExpectedWithStd = Vectors.dense(-0.00796538, 0.0, -0.0394228, -0.0873314) + val coefficientsExpected = Vectors.dense(0.01105972, 0.0, -0.08574949, -0.05079558) - assert(model2.intercept ~== interceptR2 absTol 1E-3) - assert(model2.coefficients ~= coefficientsR2 relTol 1E-2) + assert(model1.intercept ~== 0.0 relTol 1E-3) + assert(model1.coefficients ~= coefficientsExpectedWithStd relTol 1E-3) + assert(model2.intercept ~== 0.0 relTol 1E-3) + assert(model2.coefficients ~= coefficientsExpected relTol 1E-3) } test("binary logistic regression with intercept with ElasticNet regularization") { - val trainer1 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(true) + val trainer1 = (new LogisticRegression).setFitIntercept(true).setMaxIter(200) + .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(true) - .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(false) + .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0.38, lambda = 0.21)) - coefficients - - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 0.57734851 - data.V2 -0.05310287 - data.V3 . - data.V4 -0.08849250 - data.V5 -0.15458796 - */ - val interceptR1 = 0.57734851 - val coefficientsR1 = Vectors.dense(-0.05310287, 0.0, -0.08849250, -0.15458796) - - assert(model1.intercept ~== interceptR1 relTol 6E-3) - assert(model1.coefficients ~== coefficientsR1 absTol 5E-3) - - /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0.38, lambda = 0.21, - standardize=FALSE)) - coefficients + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0.38, + lambda = 0.21, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0.38, + lambda = 0.21, standardize=F)) + coefficientsStd + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 0.49991996 + data.V3 -0.04131110 + data.V4 . + data.V5 -0.08585233 + data.V6 -0.15875400 + + coefficients + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 0.5024256 + data.V3 . + data.V4 . + data.V5 -0.1846038 + data.V6 -0.0559614 - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 0.51555993 - data.V2 . - data.V3 . - data.V4 -0.18807395 - data.V5 -0.05350074 */ - val interceptR2 = 0.51555993 - val coefficientsR2 = Vectors.dense(0.0, 0.0, -0.18807395, -0.05350074) - - assert(model2.intercept ~== interceptR2 relTol 6E-3) - assert(model2.coefficients ~= coefficientsR2 absTol 1E-3) + val coefficientsRStd = Vectors.dense(-0.04131110, 0.0, -0.08585233, -0.15875400) + val interceptRStd = 0.49991996 + val coefficientsR = Vectors.dense(0.0, 0.0, -0.1846038, -0.0559614) + val interceptR = 0.5024256 + + assert(model1.intercept ~== interceptRStd relTol 6E-3) + assert(model1.coefficients ~== coefficientsRStd absTol 5E-3) + assert(model2.intercept ~== interceptR relTol 6E-3) + assert(model2.coefficients ~= coefficientsR absTol 1E-3) } test("binary logistic regression without intercept with ElasticNet regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(true) + .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(true).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(false) - .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(false) + .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(false).setWeightCol("weight") val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0.38, lambda = 0.21, - intercept=FALSE)) - coefficients - - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - data.V2 -0.001005743 - data.V3 0.072577857 - data.V4 -0.081203769 - data.V5 -0.142534158 - */ - val interceptR1 = 0.0 - val coefficientsR1 = Vectors.dense(-0.001005743, 0.072577857, -0.081203769, -0.142534158) - - assert(model1.intercept ~== interceptR1 relTol 1E-3) - assert(model1.coefficients ~= coefficientsR1 absTol 1E-2) - - /* - Using the following R code to load the data and train the model using glmnet package. - - library("glmnet") - data <- read.csv("path", header=FALSE) - label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 0.38, lambda = 0.21, - intercept=FALSE, standardize=FALSE)) - coefficients + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0.38, + lambda = 0.21, intercept=FALSE, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0.38, + lambda = 0.21, intercept=FALSE, standardize=F)) + coefficientsStd + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V3 . + data.V4 0.06859390 + data.V5 -0.07900058 + data.V6 -0.14684320 + + coefficients + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V3 . + data.V4 0.03060637 + data.V5 -0.11126742 + data.V6 . - 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - data.V2 . - data.V3 0.03345223 - data.V4 -0.11304532 - data.V5 . */ - val interceptR2 = 0.0 - val coefficientsR2 = Vectors.dense(0.0, 0.03345223, -0.11304532, 0.0) + val coefficientsRStd = Vectors.dense(0.0, 0.06859390, -0.07900058, -0.14684320) + val coefficientsR = Vectors.dense(0.0, 0.03060637, -0.11126742, 0.0) - assert(model2.intercept ~== interceptR2 absTol 1E-3) - assert(model2.coefficients ~= coefficientsR2 absTol 1E-3) + assert(model1.intercept ~== 0.0 relTol 1E-3) + assert(model1.coefficients ~= coefficientsRStd absTol 1E-2) + assert(model2.intercept ~== 0.0 absTol 1E-3) + assert(model2.coefficients ~= coefficientsR absTol 1E-3) } test("binary logistic regression with intercept with strong L1 regularization") { - val trainer1 = (new LogisticRegression).setFitIntercept(true) + val trainer1 = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") .setElasticNetParam(1.0).setRegParam(6.0).setStandardization(true) - val trainer2 = (new LogisticRegression).setFitIntercept(true) + val trainer2 = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") .setElasticNetParam(1.0).setRegParam(6.0).setStandardization(false) val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) - val histogram = binaryDataset.map { case Row(label: Double, features: Vector) => label } + val histogram = binaryDataset.as[Instance].rdd.map { i => (i.label, i.weight)} .treeAggregate(new MultiClassSummarizer)( seqOp = (c, v) => (c, v) match { - case (classSummarizer: MultiClassSummarizer, label: Double) => classSummarizer.add(label) + case (classSummarizer: MultiClassSummarizer, (label: Double, weight: Double)) => + classSummarizer.add(label, weight) }, combOp = (c1, c2) => (c1, c2) match { case (classSummarizer1: MultiClassSummarizer, classSummarizer2: MultiClassSummarizer) => @@ -758,117 +1329,1616 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { library("glmnet") data <- read.csv("path", header=FALSE) label = factor(data$V1) - features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) - coefficients = coef(glmnet(features,label, family="binomial", alpha = 1.0, lambda = 6.0)) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 1.0, + lambda = 6.0)) coefficients 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) -0.2480643 - data.V2 0.0000000 - data.V3 . - data.V4 . - data.V5 . + s0 + (Intercept) -0.2516986 + data.V3 0.0000000 + data.V4 . + data.V5 . + data.V6 . */ - val interceptR = -0.248065 + val interceptR = -0.2516986 val coefficientsR = Vectors.dense(0.0, 0.0, 0.0, 0.0) assert(model1.intercept ~== interceptR relTol 1E-5) assert(model1.coefficients ~== coefficientsR absTol 1E-6) } + test("multinomial logistic regression with intercept with strong L1 regularization") { + val trainer1 = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") + .setElasticNetParam(1.0).setRegParam(6.0).setStandardization(true) + val trainer2 = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") + .setElasticNetParam(1.0).setRegParam(6.0).setStandardization(false) + + val sqlContext = multinomialDataset.sqlContext + import sqlContext.implicits._ + val model1 = trainer1.fit(multinomialDataset) + val model2 = trainer2.fit(multinomialDataset) + + val histogram = multinomialDataset.as[Instance].rdd.map(i => (i.label, i.weight)) + .treeAggregate(new MultiClassSummarizer)( + seqOp = (c, v) => (c, v) match { + case (classSummarizer: MultiClassSummarizer, (label: Double, weight: Double)) => + classSummarizer.add(label, weight) + }, + combOp = (c1, c2) => (c1, c2) match { + case (classSummarizer1: MultiClassSummarizer, classSummarizer2: MultiClassSummarizer) => + classSummarizer1.merge(classSummarizer2) + }).histogram + val numFeatures = multinomialDataset.as[Instance].first().features.size + val numClasses = histogram.length + + /* + For multinomial logistic regression with strong L1 regularization, all the coefficients + will be zeros. As a result, the intercepts will be proportional to the log counts in the + histogram. + {{{ + \exp(b_k) = count_k * \exp(\lambda) + b_k = \log(count_k) * \lambda + }}} + \lambda is a free parameter, so choose the phase \lambda such that the + mean is centered. This yields + {{{ + b_k = \log(count_k) + b_k' = b_k - \mean(b_k) + }}} + */ + val rawInterceptsTheory = histogram.map(c => math.log(c + 1)) // add 1 for smoothing + val rawMean = rawInterceptsTheory.sum / rawInterceptsTheory.length + val interceptsTheory = Vectors.dense(rawInterceptsTheory.map(_ - rawMean)) + val coefficientsTheory = new DenseMatrix(numClasses, numFeatures, + Array.fill[Double](numClasses * numFeatures)(0.0), isTransposed = true) + + assert(model1.interceptVector ~== interceptsTheory relTol 1E-3) + assert(model1.coefficientMatrix ~= coefficientsTheory absTol 1E-6) + + assert(model2.interceptVector ~== interceptsTheory relTol 1E-3) + assert(model2.coefficientMatrix ~= coefficientsTheory absTol 1E-6) + } + + test("multinomial logistic regression with intercept without regularization") { + val trainer1 = (new LogisticRegression).setFitIntercept(true) + .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(true).setWeightCol("weight") + val trainer2 = (new LogisticRegression).setFitIntercept(true) + .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(false).setWeightCol("weight") + + val model1 = trainer1.fit(multinomialDataset) + val model2 = trainer2.fit(multinomialDataset) + + /* + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = as.factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", + alpha = 0, lambda = 0)) + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -2.10320093 + data.V3 0.24337896 + data.V4 -0.05916156 + data.V5 0.14446790 + data.V6 0.35976165 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + 0.3394473 + data.V3 -0.3443375 + data.V4 0.9181331 + data.V5 -0.2283959 + data.V6 -0.4388066 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + 1.76375361 + data.V3 0.10095851 + data.V4 -0.85897154 + data.V5 0.08392798 + data.V6 0.07904499 + + + */ + val coefficientsR = new DenseMatrix(3, 4, Array( + 0.24337896, -0.05916156, 0.14446790, 0.35976165, + -0.3443375, 0.9181331, -0.2283959, -0.4388066, + 0.10095851, -0.85897154, 0.08392798, 0.07904499), isTransposed = true) + val interceptsR = Vectors.dense(-2.10320093, 0.3394473, 1.76375361) + + model1.coefficientMatrix.colIter.foreach(v => assert(v.toArray.sum ~== 0.0 absTol eps)) + model2.coefficientMatrix.colIter.foreach(v => assert(v.toArray.sum ~== 0.0 absTol eps)) + + assert(model1.coefficientMatrix ~== coefficientsR relTol 0.05) + assert(model1.coefficientMatrix.toArray.sum ~== 0.0 absTol eps) + assert(model1.interceptVector ~== interceptsR relTol 0.05) + assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps) + assert(model2.coefficientMatrix ~== coefficientsR relTol 0.05) + assert(model2.coefficientMatrix.toArray.sum ~== 0.0 absTol eps) + assert(model2.interceptVector ~== interceptsR relTol 0.05) + assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps) + } + + test("multinomial logistic regression with zero variance (SPARK-21681)") { + val sqlContext = multinomialDatasetWithZeroVar.sqlContext + import sqlContext.implicits._ + val mlr = new LogisticRegression().setFamily("multinomial").setFitIntercept(true) + .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(true).setWeightCol("weight") + + val model = mlr.fit(multinomialDatasetWithZeroVar) + + /* + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = as.factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", + alpha = 0, lambda = 0)) + coefficients + $`0` + 3 x 1 sparse Matrix of class "dgCMatrix" + s0 + 0.2658824 + data.V3 0.1881871 + data.V4 . + + $`1` + 3 x 1 sparse Matrix of class "dgCMatrix" + s0 + 0.53604701 + data.V3 -0.02412645 + data.V4 . + + $`2` + 3 x 1 sparse Matrix of class "dgCMatrix" + s0 + -0.8019294 + data.V3 -0.1640607 + data.V4 . + */ + + val coefficientsR = new DenseMatrix(3, 2, Array( + 0.1881871, 0.0, + -0.02412645, 0.0, + -0.1640607, 0.0), isTransposed = true) + val interceptsR = Vectors.dense(0.2658824, 0.53604701, -0.8019294) + + model.coefficientMatrix.colIter.foreach(v => assert(v.toArray.sum ~== 0.0 absTol eps)) + + assert(model.coefficientMatrix ~== coefficientsR relTol 0.05) + assert(model.coefficientMatrix.toArray.sum ~== 0.0 absTol eps) + assert(model.interceptVector ~== interceptsR relTol 0.05) + assert(model.interceptVector.toArray.sum ~== 0.0 absTol eps) + } + + test("multinomial logistic regression with intercept without regularization with bound") { + // Bound constrained optimization with bound on one side. + val lowerBoundsOnCoefficients = Matrices.dense(3, 4, Array.fill(12)(1.0)) + val lowerBoundsOnIntercepts = Vectors.dense(Array.fill(3)(1.0)) + + val trainer1 = new LogisticRegression() + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setLowerBoundsOnIntercepts(lowerBoundsOnIntercepts) + .setFitIntercept(true) + .setStandardization(true) + .setWeightCol("weight") + val trainer2 = new LogisticRegression() + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setLowerBoundsOnIntercepts(lowerBoundsOnIntercepts) + .setFitIntercept(true) + .setStandardization(false) + .setWeightCol("weight") + + val model1 = trainer1.fit(multinomialDataset) + val model2 = trainer2.fit(multinomialDataset) + + // The solution is generated by https://github.com/yanboliang/bound-optimization. + val coefficientsExpected1 = new DenseMatrix(3, 4, Array( + 2.52076464, 2.73596057, 1.87984904, 2.73264492, + 1.93302281, 3.71363303, 1.50681746, 1.93398782, + 2.37839917, 1.93601818, 1.81924758, 2.45191255), isTransposed = true) + val interceptsExpected1 = Vectors.dense(1.00010477, 3.44237083, 4.86740286) + + checkCoefficientsEquivalent(model1.coefficientMatrix, coefficientsExpected1) + assert(model1.interceptVector ~== interceptsExpected1 relTol 0.01) + checkCoefficientsEquivalent(model2.coefficientMatrix, coefficientsExpected1) + assert(model2.interceptVector ~== interceptsExpected1 relTol 0.01) + + // Bound constrained optimization with bound on both side. + val upperBoundsOnCoefficients = Matrices.dense(3, 4, Array.fill(12)(2.0)) + val upperBoundsOnIntercepts = Vectors.dense(Array.fill(3)(2.0)) + + val trainer3 = new LogisticRegression() + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setLowerBoundsOnIntercepts(lowerBoundsOnIntercepts) + .setUpperBoundsOnCoefficients(upperBoundsOnCoefficients) + .setUpperBoundsOnIntercepts(upperBoundsOnIntercepts) + .setFitIntercept(true) + .setStandardization(true) + .setWeightCol("weight") + val trainer4 = new LogisticRegression() + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setLowerBoundsOnIntercepts(lowerBoundsOnIntercepts) + .setUpperBoundsOnCoefficients(upperBoundsOnCoefficients) + .setUpperBoundsOnIntercepts(upperBoundsOnIntercepts) + .setFitIntercept(true) + .setStandardization(false) + .setWeightCol("weight") + + val model3 = trainer3.fit(multinomialDataset) + val model4 = trainer4.fit(multinomialDataset) + + // The solution is generated by https://github.com/yanboliang/bound-optimization. + val coefficientsExpected3 = new DenseMatrix(3, 4, Array( + 1.61967097, 1.16027835, 1.45131448, 1.97390431, + 1.30529317, 2.0, 1.12985473, 1.26652854, + 1.61647195, 1.0, 1.40642959, 1.72985589), isTransposed = true) + val interceptsExpected3 = Vectors.dense(1.0, 2.0, 2.0) + + checkCoefficientsEquivalent(model3.coefficientMatrix, coefficientsExpected3) + assert(model3.interceptVector ~== interceptsExpected3 relTol 0.01) + checkCoefficientsEquivalent(model4.coefficientMatrix, coefficientsExpected3) + assert(model4.interceptVector ~== interceptsExpected3 relTol 0.01) + + // Bound constrained optimization with infinite bound on both side. + val trainer5 = new LogisticRegression() + .setLowerBoundsOnCoefficients(Matrices.dense(3, 4, Array.fill(12)(Double.NegativeInfinity))) + .setLowerBoundsOnIntercepts(Vectors.dense(Array.fill(3)(Double.NegativeInfinity))) + .setUpperBoundsOnCoefficients(Matrices.dense(3, 4, Array.fill(12)(Double.PositiveInfinity))) + .setUpperBoundsOnIntercepts(Vectors.dense(Array.fill(3)(Double.PositiveInfinity))) + .setFitIntercept(true) + .setStandardization(true) + .setWeightCol("weight") + val trainer6 = new LogisticRegression() + .setLowerBoundsOnCoefficients(Matrices.dense(3, 4, Array.fill(12)(Double.NegativeInfinity))) + .setLowerBoundsOnIntercepts(Vectors.dense(Array.fill(3)(Double.NegativeInfinity))) + .setUpperBoundsOnCoefficients(Matrices.dense(3, 4, Array.fill(12)(Double.PositiveInfinity))) + .setUpperBoundsOnIntercepts(Vectors.dense(Array.fill(3)(Double.PositiveInfinity))) + .setFitIntercept(true) + .setStandardization(false) + .setWeightCol("weight") + + val model5 = trainer5.fit(multinomialDataset) + val model6 = trainer6.fit(multinomialDataset) + + // The solution is generated by https://github.com/yanboliang/bound-optimization. + // It should be same as unbound constrained optimization with LBFGS. + val coefficientsExpected5 = new DenseMatrix(3, 4, Array( + 0.24337896, -0.05916156, 0.14446790, 0.35976165, + -0.3443375, 0.9181331, -0.2283959, -0.4388066, + 0.10095851, -0.85897154, 0.08392798, 0.07904499), isTransposed = true) + val interceptsExpected5 = Vectors.dense(-2.10320093, 0.3394473, 1.76375361) + + checkCoefficientsEquivalent(model5.coefficientMatrix, coefficientsExpected5) + assert(model5.interceptVector ~== interceptsExpected5 relTol 0.01) + checkCoefficientsEquivalent(model6.coefficientMatrix, coefficientsExpected5) + assert(model6.interceptVector ~== interceptsExpected5 relTol 0.01) + } + + test("multinomial logistic regression without intercept without regularization") { + + val trainer1 = (new LogisticRegression).setFitIntercept(false) + .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(true).setWeightCol("weight") + val trainer2 = (new LogisticRegression).setFitIntercept(false) + .setElasticNetParam(0.0).setRegParam(0.0).setStandardization(false).setWeightCol("weight") + + val model1 = trainer1.fit(multinomialDataset) + val model2 = trainer2.fit(multinomialDataset) + + /* + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = as.factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0, + lambda = 0, intercept=F)) + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 0.07276291 + data.V4 -0.36325496 + data.V5 0.12015088 + data.V6 0.31397340 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 -0.3180040 + data.V4 0.9679074 + data.V5 -0.2252219 + data.V6 -0.4319914 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 0.2452411 + data.V4 -0.6046524 + data.V5 0.1050710 + data.V6 0.1180180 + + + */ + val coefficientsR = new DenseMatrix(3, 4, Array( + 0.07276291, -0.36325496, 0.12015088, 0.31397340, + -0.3180040, 0.9679074, -0.2252219, -0.4319914, + 0.2452411, -0.6046524, 0.1050710, 0.1180180), isTransposed = true) + + model1.coefficientMatrix.colIter.foreach(v => assert(v.toArray.sum ~== 0.0 absTol eps)) + model2.coefficientMatrix.colIter.foreach(v => assert(v.toArray.sum ~== 0.0 absTol eps)) + + assert(model1.coefficientMatrix ~== coefficientsR relTol 0.05) + assert(model1.coefficientMatrix.toArray.sum ~== 0.0 absTol eps) + assert(model1.interceptVector.toArray === Array.fill(3)(0.0)) + assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps) + assert(model2.coefficientMatrix ~== coefficientsR relTol 0.05) + assert(model2.coefficientMatrix.toArray.sum ~== 0.0 absTol eps) + assert(model2.interceptVector.toArray === Array.fill(3)(0.0)) + assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps) + } + + test("multinomial logistic regression without intercept without regularization with bound") { + val lowerBoundsOnCoefficients = Matrices.dense(3, 4, Array.fill(12)(1.0)) + + val trainer1 = new LogisticRegression() + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setFitIntercept(false) + .setStandardization(true) + .setWeightCol("weight") + val trainer2 = new LogisticRegression() + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setFitIntercept(false) + .setStandardization(false) + .setWeightCol("weight") + + val model1 = trainer1.fit(multinomialDataset) + val model2 = trainer2.fit(multinomialDataset) + + // The solution is generated by https://github.com/yanboliang/bound-optimization. + val coefficientsExpected = new DenseMatrix(3, 4, Array( + 1.62410051, 1.38219391, 1.34486618, 1.74641729, + 1.23058989, 2.71787825, 1.0, 1.00007073, + 1.79478632, 1.14360459, 1.33011603, 1.55093897), isTransposed = true) + + checkCoefficientsEquivalent(model1.coefficientMatrix, coefficientsExpected) + assert(model1.interceptVector.toArray === Array.fill(3)(0.0)) + checkCoefficientsEquivalent(model2.coefficientMatrix, coefficientsExpected) + assert(model2.interceptVector.toArray === Array.fill(3)(0.0)) + } + + test("multinomial logistic regression with intercept with L1 regularization") { + + // use tighter constraints because OWL-QN solver takes longer to converge + val trainer1 = (new LogisticRegression).setFitIntercept(true) + .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(true) + .setMaxIter(300).setTol(1e-10).setWeightCol("weight") + val trainer2 = (new LogisticRegression).setFitIntercept(true) + .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(false) + .setMaxIter(300).setTol(1e-10).setWeightCol("weight") + + val model1 = trainer1.fit(multinomialDataset) + val model2 = trainer2.fit(multinomialDataset) + + /* + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = as.factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="multinomial", + alpha = 1, lambda = 0.05, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 1, + lambda = 0.05, standardize=F)) + coefficientsStd + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -0.62244703 + data.V3 . + data.V4 . + data.V5 . + data.V6 0.08419825 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -0.2804845 + data.V3 -0.1336960 + data.V4 0.3717091 + data.V5 -0.1530363 + data.V6 -0.2035286 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + 0.9029315 + data.V3 . + data.V4 -0.4629737 + data.V5 . + data.V6 . + + + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -0.44215290 + data.V3 . + data.V4 . + data.V5 0.01767089 + data.V6 0.02542866 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + 0.76308326 + data.V3 -0.06818576 + data.V4 . + data.V5 -0.20446351 + data.V6 -0.13017924 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -0.3209304 + data.V3 . + data.V4 . + data.V5 . + data.V6 . + + + */ + val coefficientsRStd = new DenseMatrix(3, 4, Array( + 0.0, 0.0, 0.0, 0.08419825, + -0.1336960, 0.3717091, -0.1530363, -0.2035286, + 0.0, -0.4629737, 0.0, 0.0), isTransposed = true) + val interceptsRStd = Vectors.dense(-0.62244703, -0.2804845, 0.9029315) + val coefficientsR = new DenseMatrix(3, 4, Array( + 0.0, 0.0, 0.01767089, 0.02542866, + -0.06818576, 0.0, -0.20446351, -0.13017924, + 0.0, 0.0, 0.0, 0.0), isTransposed = true) + val interceptsR = Vectors.dense(-0.44215290, 0.76308326, -0.3209304) + + assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.02) + assert(model1.interceptVector ~== interceptsRStd relTol 0.1) + assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps) + assert(model2.coefficientMatrix ~== coefficientsR absTol 0.02) + assert(model2.interceptVector ~== interceptsR relTol 0.1) + assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps) + } + + test("multinomial logistic regression without intercept with L1 regularization") { + val trainer1 = (new LogisticRegression).setFitIntercept(false) + .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(true).setWeightCol("weight") + val trainer2 = (new LogisticRegression).setFitIntercept(false) + .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(false).setWeightCol("weight") + + val model1 = trainer1.fit(multinomialDataset) + val model2 = trainer2.fit(multinomialDataset) + /* + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = as.factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 1, + lambda = 0.05, intercept=F, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 1, + lambda = 0.05, intercept=F, standardize=F)) + coefficientsStd + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 . + data.V4 . + data.V5 . + data.V6 0.01144225 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 -0.1678787 + data.V4 0.5385351 + data.V5 -0.1573039 + data.V6 -0.2471624 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 . + data.V4 . + data.V5 . + data.V6 . + + + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 . + data.V4 . + data.V5 . + data.V6 . + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 . + data.V4 0.1929409 + data.V5 -0.1889121 + data.V6 -0.1010413 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 . + data.V4 . + data.V5 . + data.V6 . + + + */ + val coefficientsRStd = new DenseMatrix(3, 4, Array( + 0.0, 0.0, 0.0, 0.01144225, + -0.1678787, 0.5385351, -0.1573039, -0.2471624, + 0.0, 0.0, 0.0, 0.0), isTransposed = true) + + val coefficientsR = new DenseMatrix(3, 4, Array( + 0.0, 0.0, 0.0, 0.0, + 0.0, 0.1929409, -0.1889121, -0.1010413, + 0.0, 0.0, 0.0, 0.0), isTransposed = true) + + assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01) + assert(model1.interceptVector.toArray === Array.fill(3)(0.0)) + assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps) + assert(model2.coefficientMatrix ~== coefficientsR absTol 0.01) + assert(model2.interceptVector.toArray === Array.fill(3)(0.0)) + assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps) + } + + test("multinomial logistic regression with intercept with L2 regularization") { + val trainer1 = (new LogisticRegression).setFitIntercept(true) + .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(true).setWeightCol("weight") + val trainer2 = (new LogisticRegression).setFitIntercept(true) + .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(false).setWeightCol("weight") + + val model1 = trainer1.fit(multinomialDataset) + val model2 = trainer2.fit(multinomialDataset) + /* + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = as.factor(data$V1) + w = data$V2 + features = as.matrix(data.frame( data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="multinomial", + alpha = 0, lambda = 0.1, intercept=T, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0, + lambda = 0.1, intercept=T, standardize=F)) + coefficientsStd + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -1.5898288335 + data.V3 0.1691226336 + data.V4 0.0002983651 + data.V5 0.1001732896 + data.V6 0.2554575585 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + 0.2125746 + data.V3 -0.2304586 + data.V4 0.6153492 + data.V5 -0.1537017 + data.V6 -0.2975443 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + 1.37725427 + data.V3 0.06133600 + data.V4 -0.61564761 + data.V5 0.05352840 + data.V6 0.04208671 + + + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -1.5681088 + data.V3 0.1508182 + data.V4 0.0121955 + data.V5 0.1217930 + data.V6 0.2162850 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + 1.1217130 + data.V3 -0.2028984 + data.V4 0.2862431 + data.V5 -0.1843559 + data.V6 -0.2481218 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + 0.44639579 + data.V3 0.05208012 + data.V4 -0.29843864 + data.V5 0.06256289 + data.V6 0.03183676 + + + */ + val coefficientsRStd = new DenseMatrix(3, 4, Array( + 0.1691226336, 0.0002983651, 0.1001732896, 0.2554575585, + -0.2304586, 0.6153492, -0.1537017, -0.2975443, + 0.06133600, -0.61564761, 0.05352840, 0.04208671), isTransposed = true) + val interceptsRStd = Vectors.dense(-1.5898288335, 0.2125746, 1.37725427) + val coefficientsR = new DenseMatrix(3, 4, Array( + 0.1508182, 0.0121955, 0.1217930, 0.2162850, + -0.2028984, 0.2862431, -0.1843559, -0.2481218, + 0.05208012, -0.29843864, 0.06256289, 0.03183676), isTransposed = true) + val interceptsR = Vectors.dense(-1.5681088, 1.1217130, 0.44639579) + + assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.001) + assert(model1.interceptVector ~== interceptsRStd relTol 0.05) + assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps) + assert(model2.coefficientMatrix ~== coefficientsR relTol 0.05) + assert(model2.interceptVector ~== interceptsR relTol 0.05) + assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps) + } + + test("multinomial logistic regression with intercept with L2 regularization with bound") { + val lowerBoundsOnCoefficients = Matrices.dense(3, 4, Array.fill(12)(1.0)) + val lowerBoundsOnIntercepts = Vectors.dense(Array.fill(3)(1.0)) + + val trainer1 = new LogisticRegression() + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setLowerBoundsOnIntercepts(lowerBoundsOnIntercepts) + .setRegParam(0.1) + .setFitIntercept(true) + .setStandardization(true) + .setWeightCol("weight") + val trainer2 = new LogisticRegression() + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setLowerBoundsOnIntercepts(lowerBoundsOnIntercepts) + .setRegParam(0.1) + .setFitIntercept(true) + .setStandardization(false) + .setWeightCol("weight") + + val model1 = trainer1.fit(multinomialDataset) + val model2 = trainer2.fit(multinomialDataset) + + // The solution is generated by https://github.com/yanboliang/bound-optimization. + val coefficientsExpectedWithStd = new DenseMatrix(3, 4, Array( + 1.0, 1.0, 1.0, 1.01647497, + 1.0, 1.44105616, 1.0, 1.0, + 1.0, 1.0, 1.0, 1.0), isTransposed = true) + val interceptsExpectedWithStd = Vectors.dense(2.52055893, 1.0, 2.560682) + val coefficientsExpected = new DenseMatrix(3, 4, Array( + 1.0, 1.0, 1.03189386, 1.0, + 1.0, 1.0, 1.0, 1.0, + 1.0, 1.0, 1.0, 1.0), isTransposed = true) + val interceptsExpected = Vectors.dense(1.06418835, 1.0, 1.20494701) + + assert(model1.coefficientMatrix ~== coefficientsExpectedWithStd relTol 0.01) + assert(model1.interceptVector ~== interceptsExpectedWithStd relTol 0.01) + assert(model2.coefficientMatrix ~== coefficientsExpected relTol 0.01) + assert(model2.interceptVector ~== interceptsExpected relTol 0.01) + } + + test("multinomial logistic regression without intercept with L2 regularization") { + val trainer1 = (new LogisticRegression).setFitIntercept(false) + .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(true).setWeightCol("weight") + val trainer2 = (new LogisticRegression).setFitIntercept(false) + .setElasticNetParam(0.0).setRegParam(0.1).setStandardization(false).setWeightCol("weight") + + val model1 = trainer1.fit(multinomialDataset) + val model2 = trainer2.fit(multinomialDataset) + /* + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = as.factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0, + lambda = 0.1, intercept=F, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0, + lambda = 0.1, intercept=F, standardize=F)) + coefficientsStd + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 0.04048126 + data.V4 -0.23075758 + data.V5 0.08228864 + data.V6 0.22277648 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 -0.2149745 + data.V4 0.6478666 + data.V5 -0.1515158 + data.V6 -0.2930498 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 0.17449321 + data.V4 -0.41710901 + data.V5 0.06922716 + data.V6 0.07027332 + + + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 -0.003949652 + data.V4 -0.142982415 + data.V5 0.091439598 + data.V6 0.179286241 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 -0.09071124 + data.V4 0.39752531 + data.V5 -0.16233832 + data.V6 -0.22206059 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 0.09466090 + data.V4 -0.25454290 + data.V5 0.07089872 + data.V6 0.04277435 + + + */ + val coefficientsRStd = new DenseMatrix(3, 4, Array( + 0.04048126, -0.23075758, 0.08228864, 0.22277648, + -0.2149745, 0.6478666, -0.1515158, -0.2930498, + 0.17449321, -0.41710901, 0.06922716, 0.07027332), isTransposed = true) + + val coefficientsR = new DenseMatrix(3, 4, Array( + -0.003949652, -0.142982415, 0.091439598, 0.179286241, + -0.09071124, 0.39752531, -0.16233832, -0.22206059, + 0.09466090, -0.25454290, 0.07089872, 0.04277435), isTransposed = true) + + assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01) + assert(model1.interceptVector.toArray === Array.fill(3)(0.0)) + assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps) + assert(model2.coefficientMatrix ~== coefficientsR absTol 0.01) + assert(model2.interceptVector.toArray === Array.fill(3)(0.0)) + assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps) + } + + test("multinomial logistic regression without intercept with L2 regularization with bound") { + val lowerBoundsOnCoefficients = Matrices.dense(3, 4, Array.fill(12)(1.0)) + + val trainer1 = new LogisticRegression() + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setRegParam(0.1) + .setFitIntercept(false) + .setStandardization(true) + .setWeightCol("weight") + val trainer2 = new LogisticRegression() + .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) + .setRegParam(0.1) + .setFitIntercept(false) + .setStandardization(false) + .setWeightCol("weight") + + val model1 = trainer1.fit(multinomialDataset) + val model2 = trainer2.fit(multinomialDataset) + + // The solution is generated by https://github.com/yanboliang/bound-optimization. + val coefficientsExpectedWithStd = new DenseMatrix(3, 4, Array( + 1.01324653, 1.0, 1.0, 1.0415767, + 1.0, 1.0, 1.0, 1.0, + 1.02244888, 1.0, 1.0, 1.0), isTransposed = true) + val coefficientsExpected = new DenseMatrix(3, 4, Array( + 1.0, 1.0, 1.03932259, 1.0, + 1.0, 1.0, 1.0, 1.0, + 1.0, 1.0, 1.03274649, 1.0), isTransposed = true) + + assert(model1.coefficientMatrix ~== coefficientsExpectedWithStd absTol 0.01) + assert(model1.interceptVector.toArray === Array.fill(3)(0.0)) + assert(model2.coefficientMatrix ~== coefficientsExpected absTol 0.01) + assert(model2.interceptVector.toArray === Array.fill(3)(0.0)) + } + + test("multinomial logistic regression with intercept with elasticnet regularization") { + val trainer1 = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") + .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(true) + .setMaxIter(300).setTol(1e-10) + val trainer2 = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") + .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(false) + .setMaxIter(300).setTol(1e-10) + + val model1 = trainer1.fit(multinomialDataset) + val model2 = trainer2.fit(multinomialDataset) + /* + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = as.factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0.5, + lambda = 0.1, intercept=T, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0.5, + lambda = 0.1, intercept=T, standardize=F)) + coefficientsStd + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -0.50133383 + data.V3 . + data.V4 . + data.V5 . + data.V6 0.08351653 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -0.3151913 + data.V3 -0.1058702 + data.V4 0.3183251 + data.V5 -0.1212969 + data.V6 -0.1629778 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + 0.8165252 + data.V3 . + data.V4 -0.3943069 + data.V5 . + data.V6 . + + + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -0.38857157 + data.V3 . + data.V4 . + data.V5 0.02384198 + data.V6 0.03127749 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + 0.62492165 + data.V3 -0.04949061 + data.V4 . + data.V5 -0.18584462 + data.V6 -0.08952455 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + -0.2363501 + data.V3 . + data.V4 . + data.V5 . + data.V6 . + + + */ + val coefficientsRStd = new DenseMatrix(3, 4, Array( + 0.0, 0.0, 0.0, 0.08351653, + -0.1058702, 0.3183251, -0.1212969, -0.1629778, + 0.0, -0.3943069, 0.0, 0.0), isTransposed = true) + val interceptsRStd = Vectors.dense(-0.50133383, -0.3151913, 0.8165252) + val coefficientsR = new DenseMatrix(3, 4, Array( + 0.0, 0.0, 0.02384198, 0.03127749, + -0.04949061, 0.0, -0.18584462, -0.08952455, + 0.0, 0.0, 0.0, 0.0), isTransposed = true) + val interceptsR = Vectors.dense(-0.38857157, 0.62492165, -0.2363501) + + assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01) + assert(model1.interceptVector ~== interceptsRStd absTol 0.01) + assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps) + assert(model2.coefficientMatrix ~== coefficientsR absTol 0.01) + assert(model2.interceptVector ~== interceptsR absTol 0.01) + assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps) + } + + test("multinomial logistic regression without intercept with elasticnet regularization") { + val trainer1 = (new LogisticRegression).setFitIntercept(false).setWeightCol("weight") + .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(true) + .setMaxIter(300).setTol(1e-10) + val trainer2 = (new LogisticRegression).setFitIntercept(false).setWeightCol("weight") + .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(false) + .setMaxIter(300).setTol(1e-10) + + val model1 = trainer1.fit(multinomialDataset) + val model2 = trainer2.fit(multinomialDataset) + /* + Use the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = as.factor(data$V1) + w = data$V2 + features = as.matrix(data.frame(data$V3, data$V4, data$V5, data$V6)) + coefficientsStd = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0.5, + lambda = 0.1, intercept=F, standardize=T)) + coefficients = coef(glmnet(features, label, weights=w, family="multinomial", alpha = 0.5, + lambda = 0.1, intercept=F, standardize=F)) + coefficientsStd + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 . + data.V4 . + data.V5 . + data.V6 0.03238285 + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 -0.1328284 + data.V4 0.4219321 + data.V5 -0.1247544 + data.V6 -0.1893318 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 0.004572312 + data.V4 . + data.V5 . + data.V6 . + + + coefficients + $`0` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 . + data.V4 . + data.V5 . + data.V6 . + + $`1` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 . + data.V4 0.14571623 + data.V5 -0.16456351 + data.V6 -0.05866264 + + $`2` + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + . + data.V3 . + data.V4 . + data.V5 . + data.V6 . + + + */ + val coefficientsRStd = new DenseMatrix(3, 4, Array( + 0.0, 0.0, 0.0, 0.03238285, + -0.1328284, 0.4219321, -0.1247544, -0.1893318, + 0.004572312, 0.0, 0.0, 0.0), isTransposed = true) + + val coefficientsR = new DenseMatrix(3, 4, Array( + 0.0, 0.0, 0.0, 0.0, + 0.0, 0.14571623, -0.16456351, -0.05866264, + 0.0, 0.0, 0.0, 0.0), isTransposed = true) + + assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01) + assert(model1.interceptVector.toArray === Array.fill(3)(0.0)) + assert(model1.interceptVector.toArray.sum ~== 0.0 absTol eps) + assert(model2.coefficientMatrix ~== coefficientsR absTol 0.01) + assert(model2.interceptVector.toArray === Array.fill(3)(0.0)) + assert(model2.interceptVector.toArray.sum ~== 0.0 absTol eps) + } + test("evaluate on test set") { // Evaluate on test set should be same as that of the transformed training data. val lr = new LogisticRegression() .setMaxIter(10) .setRegParam(1.0) .setThreshold(0.6) - val model = lr.fit(dataset) - val summary = model.summary.asInstanceOf[BinaryLogisticRegressionSummary] - - val sameSummary = model.evaluate(dataset).asInstanceOf[BinaryLogisticRegressionSummary] - assert(summary.areaUnderROC === sameSummary.areaUnderROC) - assert(summary.roc.collect() === sameSummary.roc.collect()) - assert(summary.pr.collect === sameSummary.pr.collect()) + .setFamily("binomial") + val blorModel = lr.fit(smallBinaryDataset) + val blorSummary = blorModel.binarySummary + + val sameBlorSummary = + blorModel.evaluate(smallBinaryDataset).asInstanceOf[BinaryLogisticRegressionSummary] + assert(blorSummary.areaUnderROC === sameBlorSummary.areaUnderROC) + assert(blorSummary.roc.collect() === sameBlorSummary.roc.collect()) + assert(blorSummary.pr.collect === sameBlorSummary.pr.collect()) assert( - summary.fMeasureByThreshold.collect() === sameSummary.fMeasureByThreshold.collect()) - assert(summary.recallByThreshold.collect() === sameSummary.recallByThreshold.collect()) + blorSummary.fMeasureByThreshold.collect() === sameBlorSummary.fMeasureByThreshold.collect()) assert( - summary.precisionByThreshold.collect() === sameSummary.precisionByThreshold.collect()) + blorSummary.recallByThreshold.collect() === sameBlorSummary.recallByThreshold.collect()) + assert( + blorSummary.precisionByThreshold.collect() === sameBlorSummary.precisionByThreshold.collect()) + assert(blorSummary.labels === sameBlorSummary.labels) + assert(blorSummary.truePositiveRateByLabel === sameBlorSummary.truePositiveRateByLabel) + assert(blorSummary.falsePositiveRateByLabel === sameBlorSummary.falsePositiveRateByLabel) + assert(blorSummary.precisionByLabel === sameBlorSummary.precisionByLabel) + assert(blorSummary.recallByLabel === sameBlorSummary.recallByLabel) + assert(blorSummary.fMeasureByLabel === sameBlorSummary.fMeasureByLabel) + assert(blorSummary.accuracy === sameBlorSummary.accuracy) + assert(blorSummary.weightedTruePositiveRate === sameBlorSummary.weightedTruePositiveRate) + assert(blorSummary.weightedFalsePositiveRate === sameBlorSummary.weightedFalsePositiveRate) + assert(blorSummary.weightedRecall === sameBlorSummary.weightedRecall) + assert(blorSummary.weightedPrecision === sameBlorSummary.weightedPrecision) + assert(blorSummary.weightedFMeasure === sameBlorSummary.weightedFMeasure) + + lr.setFamily("multinomial") + val mlorModel = lr.fit(smallMultinomialDataset) + val mlorSummary = mlorModel.summary + + val mlorSameSummary = mlorModel.evaluate(smallMultinomialDataset) + + assert(mlorSummary.truePositiveRateByLabel === mlorSameSummary.truePositiveRateByLabel) + assert(mlorSummary.falsePositiveRateByLabel === mlorSameSummary.falsePositiveRateByLabel) + assert(mlorSummary.precisionByLabel === mlorSameSummary.precisionByLabel) + assert(mlorSummary.recallByLabel === mlorSameSummary.recallByLabel) + assert(mlorSummary.fMeasureByLabel === mlorSameSummary.fMeasureByLabel) + assert(mlorSummary.accuracy === mlorSameSummary.accuracy) + assert(mlorSummary.weightedTruePositiveRate === mlorSameSummary.weightedTruePositiveRate) + assert(mlorSummary.weightedFalsePositiveRate === mlorSameSummary.weightedFalsePositiveRate) + assert(mlorSummary.weightedPrecision === mlorSameSummary.weightedPrecision) + assert(mlorSummary.weightedRecall === mlorSameSummary.weightedRecall) + assert(mlorSummary.weightedFMeasure === mlorSameSummary.weightedFMeasure) + } + + test("evaluate with labels that are not doubles") { + // Evaluate a test set with Label that is a numeric type other than Double + val blor = new LogisticRegression() + .setMaxIter(1) + .setRegParam(1.0) + .setFamily("binomial") + val blorModel = blor.fit(smallBinaryDataset) + val blorSummary = blorModel.evaluate(smallBinaryDataset) + .asInstanceOf[BinaryLogisticRegressionSummary] + + val blorLongLabelData = smallBinaryDataset.select(col(blorModel.getLabelCol).cast(LongType), + col(blorModel.getFeaturesCol)) + val blorLongSummary = blorModel.evaluate(blorLongLabelData) + .asInstanceOf[BinaryLogisticRegressionSummary] + + assert(blorSummary.areaUnderROC === blorLongSummary.areaUnderROC) + + val mlor = new LogisticRegression() + .setMaxIter(1) + .setRegParam(1.0) + .setFamily("multinomial") + val mlorModel = mlor.fit(smallMultinomialDataset) + val mlorSummary = mlorModel.evaluate(smallMultinomialDataset) + + val mlorLongLabelData = smallMultinomialDataset.select( + col(mlorModel.getLabelCol).cast(LongType), + col(mlorModel.getFeaturesCol)) + val mlorLongSummary = mlorModel.evaluate(mlorLongLabelData) + + assert(mlorSummary.accuracy === mlorLongSummary.accuracy) } test("statistics on training data") { // Test that loss is monotonically decreasing. - val lr = new LogisticRegression() + val blor = new LogisticRegression() .setMaxIter(10) .setRegParam(1.0) - .setThreshold(0.6) - val model = lr.fit(dataset) + .setFamily("binomial") + val blorModel = blor.fit(smallBinaryDataset) assert( - model.summary + blorModel.summary .objectiveHistory .sliding(2) .forall(x => x(0) >= x(1))) + val mlor = new LogisticRegression() + .setMaxIter(10) + .setRegParam(1.0) + .setFamily("multinomial") + val mlorModel = mlor.fit(smallMultinomialDataset) + assert( + mlorModel.summary + .objectiveHistory + .sliding(2) + .forall(x => x(0) >= x(1))) } - test("binary logistic regression with weighted samples") { - val (dataset, weightedDataset) = { - val nPoints = 1000 - val coefficients = Array(-0.57997, 0.912083, -0.371077, -0.819866, 2.688191) - val xMean = Array(5.843, 3.057, 3.758, 1.199) - val xVariance = Array(0.6856, 0.1899, 3.116, 0.581) - val testData = - generateMultinomialLogisticInput(coefficients, xMean, xVariance, true, nPoints, 42) - - // Let's over-sample the positive samples twice. - val data1 = testData.flatMap { case labeledPoint: LabeledPoint => - if (labeledPoint.label == 1.0) { - Iterator(labeledPoint, labeledPoint) - } else { - Iterator(labeledPoint) - } + test("logistic regression with sample weights") { + def modelEquals(m1: LogisticRegressionModel, m2: LogisticRegressionModel): Unit = { + assert(m1.coefficientMatrix ~== m2.coefficientMatrix absTol 0.05) + assert(m1.interceptVector ~== m2.interceptVector absTol 0.05) + } + val testParams = Seq( + ("binomial", smallBinaryDataset, 2), + ("multinomial", smallMultinomialDataset, 3) + ) + testParams.foreach { case (family, dataset, numClasses) => + val estimator = new LogisticRegression().setFamily(family) + MLTestingUtils.testArbitrarilyScaledWeights[LogisticRegressionModel, LogisticRegression]( + dataset.as[LabeledPoint], estimator, modelEquals) + MLTestingUtils.testOutliersWithSmallWeights[LogisticRegressionModel, LogisticRegression]( + dataset.as[LabeledPoint], estimator, numClasses, modelEquals, outlierRatio = 3) + MLTestingUtils.testOversamplingVsWeighting[LogisticRegressionModel, LogisticRegression]( + dataset.as[LabeledPoint], estimator, modelEquals, seed) + } + } + + test("set family") { + val lr = new LogisticRegression().setMaxIter(1) + // don't set anything for binary classification + val model1 = lr.fit(binaryDataset) + assert(model1.coefficientMatrix.numRows === 1 && model1.coefficientMatrix.numCols === 4) + assert(model1.interceptVector.size === 1) + + // set to multinomial for binary classification + val model2 = lr.setFamily("multinomial").fit(binaryDataset) + assert(model2.coefficientMatrix.numRows === 2 && model2.coefficientMatrix.numCols === 4) + assert(model2.interceptVector.size === 2) + + // set to binary for binary classification + val model3 = lr.setFamily("binomial").fit(binaryDataset) + assert(model3.coefficientMatrix.numRows === 1 && model3.coefficientMatrix.numCols === 4) + assert(model3.interceptVector.size === 1) + + // don't set anything for multiclass classification + val mlr = new LogisticRegression().setMaxIter(1) + val model4 = mlr.fit(multinomialDataset) + assert(model4.coefficientMatrix.numRows === 3 && model4.coefficientMatrix.numCols === 4) + assert(model4.interceptVector.size === 3) + + // set to binary for multiclass classification + mlr.setFamily("binomial") + val thrown = intercept[IllegalArgumentException] { + mlr.fit(multinomialDataset) + } + assert(thrown.getMessage.contains("Binomial family only supports 1 or 2 outcome classes")) + + // set to multinomial for multiclass + mlr.setFamily("multinomial") + val model5 = mlr.fit(multinomialDataset) + assert(model5.coefficientMatrix.numRows === 3 && model5.coefficientMatrix.numCols === 4) + assert(model5.interceptVector.size === 3) + } + + test("set initial model") { + val lr = new LogisticRegression().setFamily("binomial") + val model1 = lr.fit(smallBinaryDataset) + val lr2 = new LogisticRegression().setInitialModel(model1).setMaxIter(5).setFamily("binomial") + val model2 = lr2.fit(smallBinaryDataset) + val predictions1 = model1.transform(smallBinaryDataset).select("prediction").collect() + val predictions2 = model2.transform(smallBinaryDataset).select("prediction").collect() + predictions1.zip(predictions2).foreach { case (Row(p1: Double), Row(p2: Double)) => + assert(p1 === p2) + } + assert(model2.summary.totalIterations === 1) + + val lr3 = new LogisticRegression().setFamily("multinomial") + val model3 = lr3.fit(smallMultinomialDataset) + val lr4 = new LogisticRegression() + .setInitialModel(model3).setMaxIter(5).setFamily("multinomial") + val model4 = lr4.fit(smallMultinomialDataset) + val predictions3 = model3.transform(smallMultinomialDataset).select("prediction").collect() + val predictions4 = model4.transform(smallMultinomialDataset).select("prediction").collect() + predictions3.zip(predictions4).foreach { case (Row(p1: Double), Row(p2: Double)) => + assert(p1 === p2) + } + assert(model4.summary.totalIterations === 1) + } + + test("binary logistic regression with all labels the same") { + val sameLabels = smallBinaryDataset + .withColumn("zeroLabel", lit(0.0)) + .withColumn("oneLabel", lit(1.0)) + + // fitIntercept=true + val lrIntercept = new LogisticRegression() + .setFitIntercept(true) + .setMaxIter(3) + .setFamily("binomial") + + val allZeroInterceptModel = lrIntercept + .setLabelCol("zeroLabel") + .fit(sameLabels) + assert(allZeroInterceptModel.coefficients ~== Vectors.dense(0.0) absTol 1E-3) + assert(allZeroInterceptModel.intercept === Double.NegativeInfinity) + assert(allZeroInterceptModel.summary.totalIterations === 0) + + val allOneInterceptModel = lrIntercept + .setLabelCol("oneLabel") + .fit(sameLabels) + assert(allOneInterceptModel.coefficients ~== Vectors.dense(0.0) absTol 1E-3) + assert(allOneInterceptModel.intercept === Double.PositiveInfinity) + assert(allOneInterceptModel.summary.totalIterations === 0) + + // fitIntercept=false + val lrNoIntercept = new LogisticRegression() + .setFitIntercept(false) + .setMaxIter(3) + .setFamily("binomial") + + val allZeroNoInterceptModel = lrNoIntercept + .setLabelCol("zeroLabel") + .fit(sameLabels) + assert(allZeroNoInterceptModel.intercept === 0.0) + assert(allZeroNoInterceptModel.summary.totalIterations > 0) + + val allOneNoInterceptModel = lrNoIntercept + .setLabelCol("oneLabel") + .fit(sameLabels) + assert(allOneNoInterceptModel.intercept === 0.0) + assert(allOneNoInterceptModel.summary.totalIterations > 0) + } + + test("multiclass logistic regression with all labels the same") { + val constantData = Seq( + LabeledPoint(4.0, Vectors.dense(0.0)), + LabeledPoint(4.0, Vectors.dense(1.0)), + LabeledPoint(4.0, Vectors.dense(2.0))).toDF() + val mlr = new LogisticRegression().setFamily("multinomial") + val model = mlr.fit(constantData) + val results = model.transform(constantData) + results.select("rawPrediction", "probability", "prediction").collect().foreach { + case Row(raw: Vector, prob: Vector, pred: Double) => + assert(raw === Vectors.dense(Array(0.0, 0.0, 0.0, 0.0, Double.PositiveInfinity))) + assert(prob === Vectors.dense(Array(0.0, 0.0, 0.0, 0.0, 1.0))) + assert(pred === 4.0) + } + assert(model.summary.totalIterations === 0) + + // force the model to be trained with only one class + val constantZeroData = Seq( + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(0.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(2.0))).toDF() + val modelZeroLabel = mlr.setFitIntercept(false).fit(constantZeroData) + val resultsZero = modelZeroLabel.transform(constantZeroData) + resultsZero.select("rawPrediction", "probability", "prediction").collect().foreach { + case Row(raw: Vector, prob: Vector, pred: Double) => + assert(prob === Vectors.dense(Array(1.0))) + assert(pred === 0.0) + } + assert(modelZeroLabel.summary.totalIterations > 0) + + // ensure that the correct value is predicted when numClasses passed through metadata + val labelMeta = NominalAttribute.defaultAttr.withName("label").withNumValues(6).toMetadata() + val constantDataWithMetadata = constantData + .select(constantData("label").as("label", labelMeta), constantData("features")) + val modelWithMetadata = mlr.setFitIntercept(true).fit(constantDataWithMetadata) + val resultsWithMetadata = modelWithMetadata.transform(constantDataWithMetadata) + resultsWithMetadata.select("rawPrediction", "probability", "prediction").collect().foreach { + case Row(raw: Vector, prob: Vector, pred: Double) => + assert(raw === Vectors.dense(Array(0.0, 0.0, 0.0, 0.0, Double.PositiveInfinity, 0.0))) + assert(prob === Vectors.dense(Array(0.0, 0.0, 0.0, 0.0, 1.0, 0.0))) + assert(pred === 4.0) + } + require(modelWithMetadata.summary.totalIterations === 0) + } + + test("compressed storage for constant label") { + /* + When the label is constant and fit intercept is true, all the coefficients will be + zeros, and so the model coefficients should be stored as sparse data structures, except + when the matrix dimensions are very small. + */ + val moreClassesThanFeatures = Seq( + LabeledPoint(4.0, Vectors.dense(Array.fill(5)(0.0))), + LabeledPoint(4.0, Vectors.dense(Array.fill(5)(1.0))), + LabeledPoint(4.0, Vectors.dense(Array.fill(5)(2.0)))).toDF() + val mlr = new LogisticRegression().setFamily("multinomial").setFitIntercept(true) + val model = mlr.fit(moreClassesThanFeatures) + assert(model.coefficientMatrix.isInstanceOf[SparseMatrix]) + assert(model.coefficientMatrix.isColMajor) + + // in this case, it should be stored as row major + val moreFeaturesThanClasses = Seq( + LabeledPoint(1.0, Vectors.dense(Array.fill(5)(0.0))), + LabeledPoint(1.0, Vectors.dense(Array.fill(5)(1.0))), + LabeledPoint(1.0, Vectors.dense(Array.fill(5)(2.0)))).toDF() + val model2 = mlr.fit(moreFeaturesThanClasses) + assert(model2.coefficientMatrix.isInstanceOf[SparseMatrix]) + assert(model2.coefficientMatrix.isRowMajor) + + val blr = new LogisticRegression().setFamily("binomial").setFitIntercept(true) + val blrModel = blr.fit(moreFeaturesThanClasses) + assert(blrModel.coefficientMatrix.isInstanceOf[SparseMatrix]) + assert(blrModel.coefficientMatrix.asInstanceOf[SparseMatrix].colPtrs.length === 2) + } + + test("compressed coefficients") { + + val trainer1 = new LogisticRegression() + .setRegParam(0.1) + .setElasticNetParam(1.0) + + // compressed row major is optimal + val model1 = trainer1.fit(multinomialDataset.limit(100)) + assert(model1.coefficientMatrix.isInstanceOf[SparseMatrix]) + assert(model1.coefficientMatrix.isRowMajor) + + // compressed column major is optimal since there are more classes than features + val labelMeta = NominalAttribute.defaultAttr.withName("label").withNumValues(6).toMetadata() + val model2 = trainer1.fit(multinomialDataset + .withColumn("label", col("label").as("label", labelMeta)).limit(100)) + assert(model2.coefficientMatrix.isInstanceOf[SparseMatrix]) + assert(model2.coefficientMatrix.isColMajor) + + // coefficients are dense without L1 regularization + val trainer2 = new LogisticRegression() + .setElasticNetParam(0.0) + val model3 = trainer2.fit(multinomialDataset.limit(100)) + assert(model3.coefficientMatrix.isInstanceOf[DenseMatrix]) + } + + test("numClasses specified in metadata/inferred") { + val lr = new LogisticRegression().setMaxIter(1).setFamily("multinomial") + + // specify more classes than unique label values + val labelMeta = NominalAttribute.defaultAttr.withName("label").withNumValues(4).toMetadata() + val df = smallMultinomialDataset.select(smallMultinomialDataset("label").as("label", labelMeta), + smallMultinomialDataset("features")) + val model1 = lr.fit(df) + assert(model1.numClasses === 4) + assert(model1.interceptVector.size === 4) + + // specify two classes when there are really three + val labelMeta1 = NominalAttribute.defaultAttr.withName("label").withNumValues(2).toMetadata() + val df1 = smallMultinomialDataset + .select(smallMultinomialDataset("label").as("label", labelMeta1), + smallMultinomialDataset("features")) + val thrown = intercept[IllegalArgumentException] { + lr.fit(df1) + } + assert(thrown.getMessage.contains("less than the number of unique labels")) + + // lr should infer the number of classes if not specified + val model3 = lr.fit(smallMultinomialDataset) + assert(model3.numClasses === 3) + } + + test("read/write") { + def checkModelData(model: LogisticRegressionModel, model2: LogisticRegressionModel): Unit = { + assert(model.intercept === model2.intercept) + assert(model.coefficients.toArray === model2.coefficients.toArray) + assert(model.numClasses === model2.numClasses) + assert(model.numFeatures === model2.numFeatures) + } + val lr = new LogisticRegression() + testEstimatorAndModelReadWrite(lr, smallBinaryDataset, LogisticRegressionSuite.allParamSettings, + LogisticRegressionSuite.allParamSettings, checkModelData) + } + + test("should support all NumericType labels and weights, and not support other types") { + val lr = new LogisticRegression().setMaxIter(1) + MLTestingUtils.checkNumericTypes[LogisticRegressionModel, LogisticRegression]( + lr, spark) { (expected, actual) => + assert(expected.intercept === actual.intercept) + assert(expected.coefficients.toArray === actual.coefficients.toArray) } + } - val rnd = new Random(8392) - val data2 = testData.flatMap { case LabeledPoint(label: Double, features: Vector) => - if (rnd.nextGaussian() > 0.0) { - if (label == 1.0) { - Iterator( - Instance(label, 1.2, features), - Instance(label, 0.8, features), - Instance(0.0, 0.0, features)) - } else { - Iterator( - Instance(label, 0.3, features), - Instance(1.0, 0.0, features), - Instance(label, 0.1, features), - Instance(label, 0.6, features)) - } - } else { - if (label == 1.0) { - Iterator(Instance(label, 2.0, features)) - } else { - Iterator(Instance(label, 1.0, features)) + test("string params should be case-insensitive") { + val lr = new LogisticRegression() + Seq(("AuTo", smallBinaryDataset), ("biNoMial", smallBinaryDataset), + ("mulTinomIAl", smallMultinomialDataset)).foreach { case (family, data) => + lr.setFamily(family) + assert(lr.getFamily === family) + val model = lr.fit(data) + assert(model.getFamily === family) + } + } +} + +object LogisticRegressionSuite { + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = ProbabilisticClassifierSuite.allParamSettings ++ Map( + "probabilityCol" -> "myProbability", + "thresholds" -> Array(0.4, 0.6), + "regParam" -> 0.01, + "elasticNetParam" -> 0.1, + "maxIter" -> 2, // intentionally small + "fitIntercept" -> true, + "tol" -> 0.8, + "standardization" -> false, + "threshold" -> 0.6 + ) + + def generateLogisticInputAsList( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): java.util.List[LabeledPoint] = { + generateLogisticInput(offset, scale, nPoints, seed).asJava + } + + // Generate input of the form Y = logistic(offset + scale*X) + def generateLogisticInput( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + + val y = (0 until nPoints).map { i => + val p = 1.0 / (1.0 + math.exp(-(offset + scale * x1(i)))) + if (rnd.nextDouble() < p) 1.0 else 0.0 + } + + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(Array(x1(i))))) + testData + } + + /** + * Generates `k` classes multinomial synthetic logistic input in `n` dimensional space given the + * model weights and mean/variance of the features. The synthetic data will be drawn from + * the probability distribution constructed by weights using the following formula. + * + * P(y = 0 | x) = 1 / norm + * P(y = 1 | x) = exp(x * w_1) / norm + * P(y = 2 | x) = exp(x * w_2) / norm + * ... + * P(y = k-1 | x) = exp(x * w_{k-1}) / norm + * where norm = 1 + exp(x * w_1) + exp(x * w_2) + ... + exp(x * w_{k-1}) + * + * @param weights matrix is flatten into a vector; as a result, the dimension of weights vector + * will be (k - 1) * (n + 1) if `addIntercept == true`, and + * if `addIntercept != true`, the dimension will be (k - 1) * n. + * @param xMean the mean of the generated features. Lots of time, if the features are not properly + * standardized, the algorithm with poor implementation will have difficulty + * to converge. + * @param xVariance the variance of the generated features. + * @param addIntercept whether to add intercept. + * @param nPoints the number of instance of generated data. + * @param seed the seed for random generator. For consistent testing result, it will be fixed. + */ + def generateMultinomialLogisticInput( + weights: Array[Double], + xMean: Array[Double], + xVariance: Array[Double], + addIntercept: Boolean, + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + + val xDim = xMean.length + val xWithInterceptsDim = if (addIntercept) xDim + 1 else xDim + val nClasses = weights.length / xWithInterceptsDim + 1 + + val x = Array.fill[Vector](nPoints)(Vectors.dense(Array.fill[Double](xDim)(rnd.nextGaussian()))) + + x.foreach { vector => + // This doesn't work if `vector` is a sparse vector. + val vectorArray = vector.toArray + var i = 0 + val len = vectorArray.length + while (i < len) { + vectorArray(i) = vectorArray(i) * math.sqrt(xVariance(i)) + xMean(i) + i += 1 + } + } + + val y = (0 until nPoints).map { idx => + val xArray = x(idx).toArray + val margins = Array.ofDim[Double](nClasses) + val probs = Array.ofDim[Double](nClasses) + + for (i <- 0 until nClasses - 1) { + for (j <- 0 until xDim) margins(i + 1) += weights(i * xWithInterceptsDim + j) * xArray(j) + if (addIntercept) margins(i + 1) += weights((i + 1) * xWithInterceptsDim - 1) + } + // Preventing the overflow when we compute the probability + val maxMargin = margins.max + if (maxMargin > 0) for (i <- 0 until nClasses) margins(i) -= maxMargin + + // Computing the probabilities for each class from the margins. + val norm = { + var temp = 0.0 + for (i <- 0 until nClasses) { + probs(i) = math.exp(margins(i)) + temp += probs(i) + } + temp + } + for (i <- 0 until nClasses) probs(i) /= norm + + // Compute the cumulative probability so we can generate a random number and assign a label. + for (i <- 1 until nClasses) probs(i) += probs(i - 1) + val p = rnd.nextDouble() + var y = 0 + breakable { + for (i <- 0 until nClasses) { + if (p < probs(i)) { + y = i + break } } } - - (sqlContext.createDataFrame(sc.parallelize(data1, 4)), - sqlContext.createDataFrame(sc.parallelize(data2, 4))) + y } - val trainer1a = (new LogisticRegression).setFitIntercept(true) - .setRegParam(0.0).setStandardization(true) - val trainer1b = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") - .setRegParam(0.0).setStandardization(true) - val model1a0 = trainer1a.fit(dataset) - val model1a1 = trainer1a.fit(weightedDataset) - val model1b = trainer1b.fit(weightedDataset) - assert(model1a0.coefficients !~= model1a1.coefficients absTol 1E-3) - assert(model1a0.intercept !~= model1a1.intercept absTol 1E-3) - assert(model1a0.coefficients ~== model1b.coefficients absTol 1E-3) - assert(model1a0.intercept ~== model1b.intercept absTol 1E-3) + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), x(i))) + testData + } + /** + * When no regularization is applied, the multinomial coefficients lack identifiability + * because we do not use a pivot class. We can add any constant value to the coefficients + * and get the same likelihood. If fitting under bound constrained optimization, we don't + * choose the mean centered coefficients like what we do for unbound problems, since they + * may out of the bounds. We use this function to check whether two coefficients are equivalent. + */ + def checkCoefficientsEquivalent(coefficients1: Matrix, coefficients2: Matrix): Unit = { + coefficients1.colIter.zip(coefficients2.colIter).foreach { case (col1: Vector, col2: Vector) => + (col1.asBreeze - col2.asBreeze).toArray.toSeq.sliding(2).foreach { + case Seq(v1, v2) => assert(v1 ~= v2 absTol 1E-3) + } + } } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala index 17db8c44777d..d3141ec70856 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifierSuite.scala @@ -18,38 +18,137 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.MLTestingUtils +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ -import org.apache.spark.sql.Row +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.functions._ -class MultilayerPerceptronClassifierSuite extends SparkFunSuite with MLlibTestSparkContext { +class MultilayerPerceptronClassifierSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { - test("XOR function learning as binary classification problem with two outputs.") { - val dataFrame = sqlContext.createDataFrame(Seq( - (Vectors.dense(0.0, 0.0), 0.0), - (Vectors.dense(0.0, 1.0), 1.0), - (Vectors.dense(1.0, 0.0), 1.0), - (Vectors.dense(1.0, 1.0), 0.0)) + import testImplicits._ + + @transient var dataset: Dataset[_] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + dataset = Seq( + (Vectors.dense(0.0, 0.0), 0.0), + (Vectors.dense(0.0, 1.0), 1.0), + (Vectors.dense(1.0, 0.0), 1.0), + (Vectors.dense(1.0, 1.0), 0.0) ).toDF("features", "label") + } + + test("Input Validation") { + val mlpc = new MultilayerPerceptronClassifier() + intercept[IllegalArgumentException] { + mlpc.setLayers(Array.empty[Int]) + } + intercept[IllegalArgumentException] { + mlpc.setLayers(Array[Int](1)) + } + intercept[IllegalArgumentException] { + mlpc.setLayers(Array[Int](0, 1)) + } + intercept[IllegalArgumentException] { + mlpc.setLayers(Array[Int](1, 0)) + } + mlpc.setLayers(Array[Int](1, 1)) + } + + test("XOR function learning as binary classification problem with two outputs.") { val layers = Array[Int](2, 5, 2) val trainer = new MultilayerPerceptronClassifier() .setLayers(layers) .setBlockSize(1) - .setSeed(11L) + .setSeed(123L) .setMaxIter(100) - val model = trainer.fit(dataFrame) - val result = model.transform(dataFrame) + .setSolver("l-bfgs") + val model = trainer.fit(dataset) + val result = model.transform(dataset) + MLTestingUtils.checkCopyAndUids(trainer, model) val predictionAndLabels = result.select("prediction", "label").collect() predictionAndLabels.foreach { case Row(p: Double, l: Double) => assert(p == l) } } - // TODO: implement a more rigorous test + test("Predicted class probabilities: calibration on toy dataset") { + val layers = Array[Int](4, 5, 2) + + val strongDataset = Seq( + (Vectors.dense(1, 2, 3, 4), 0d, Vectors.dense(1d, 0d)), + (Vectors.dense(4, 3, 2, 1), 1d, Vectors.dense(0d, 1d)), + (Vectors.dense(1, 1, 1, 1), 0d, Vectors.dense(.5, .5)), + (Vectors.dense(1, 1, 1, 1), 1d, Vectors.dense(.5, .5)) + ).toDF("features", "label", "expectedProbability") + val trainer = new MultilayerPerceptronClassifier() + .setLayers(layers) + .setBlockSize(1) + .setSeed(123L) + .setMaxIter(100) + .setSolver("l-bfgs") + val model = trainer.fit(strongDataset) + val result = model.transform(strongDataset) + result.select("probability", "expectedProbability").collect().foreach { + case Row(p: Vector, e: Vector) => + assert(p ~== e absTol 1e-3) + } + ProbabilisticClassifierSuite.testPredictMethods[ + Vector, MultilayerPerceptronClassificationModel](model, strongDataset) + } + + test("test model probability") { + val layers = Array[Int](2, 5, 2) + val trainer = new MultilayerPerceptronClassifier() + .setLayers(layers) + .setBlockSize(1) + .setSeed(123L) + .setMaxIter(100) + .setSolver("l-bfgs") + val model = trainer.fit(dataset) + model.setProbabilityCol("probability") + val result = model.transform(dataset) + val features2prob = udf { features: Vector => model.mlpModel.predict(features) } + result.select(features2prob(col("features")), col("probability")).collect().foreach { + case Row(p1: Vector, p2: Vector) => + assert(p1 ~== p2 absTol 1e-3) + } + } + + test("Test setWeights by training restart") { + val dataFrame = Seq( + (Vectors.dense(0.0, 0.0), 0.0), + (Vectors.dense(0.0, 1.0), 1.0), + (Vectors.dense(1.0, 0.0), 1.0), + (Vectors.dense(1.0, 1.0), 0.0) + ).toDF("features", "label") + val layers = Array[Int](2, 5, 2) + val trainer = new MultilayerPerceptronClassifier() + .setLayers(layers) + .setBlockSize(1) + .setSeed(12L) + .setMaxIter(1) + .setTol(1e-6) + val initialWeights = trainer.fit(dataFrame).weights + trainer.setInitialWeights(initialWeights.copy) + val weights1 = trainer.fit(dataFrame).weights + trainer.setInitialWeights(initialWeights.copy) + val weights2 = trainer.fit(dataFrame).weights + assert(weights1 ~== weights2 absTol 10e-5, + "Training should produce the same weights given equal initial weights and number of steps") + } + test("3 class classification with 2 hidden layers") { val nPoints = 1000 @@ -61,33 +160,69 @@ class MultilayerPerceptronClassifierSuite extends SparkFunSuite with MLlibTestSp val xMean = Array(5.843, 3.057, 3.758, 1.199) val xVariance = Array(0.6856, 0.1899, 3.116, 0.581) - val rdd = sc.parallelize(generateMultinomialLogisticInput( - coefficients, xMean, xVariance, true, nPoints, 42), 2) - val dataFrame = sqlContext.createDataFrame(rdd).toDF("label", "features") + // the input seed is somewhat magic, to make this test pass + val data = generateMultinomialLogisticInput( + coefficients, xMean, xVariance, true, nPoints, 1).toDS() + val dataFrame = data.toDF("label", "features") val numClasses = 3 val numIterations = 100 val layers = Array[Int](4, 5, 4, numClasses) val trainer = new MultilayerPerceptronClassifier() .setLayers(layers) .setBlockSize(1) - .setSeed(11L) + .setSeed(11L) // currently this seed is ignored .setMaxIter(numIterations) val model = trainer.fit(dataFrame) val numFeatures = dataFrame.select("features").first().getAs[Vector](0).size assert(model.numFeatures === numFeatures) - val mlpPredictionAndLabels = model.transform(dataFrame).select("prediction", "label") - .map { case Row(p: Double, l: Double) => (p, l) } + val mlpPredictionAndLabels = model.transform(dataFrame).select("prediction", "label").rdd.map { + case Row(p: Double, l: Double) => (p, l) + } // train multinomial logistic regression val lr = new LogisticRegressionWithLBFGS() .setIntercept(true) .setNumClasses(numClasses) lr.optimizer.setRegParam(0.0) .setNumIterations(numIterations) - val lrModel = lr.run(rdd) - val lrPredictionAndLabels = lrModel.predict(rdd.map(_.features)).zip(rdd.map(_.label)) + val lrModel = lr.run(data.rdd.map(OldLabeledPoint.fromML)) + val lrPredictionAndLabels = + lrModel.predict(data.rdd.map(p => OldVectors.fromML(p.features))).zip(data.rdd.map(_.label)) // MLP's predictions should not differ a lot from LR's. val lrMetrics = new MulticlassMetrics(lrPredictionAndLabels) val mlpMetrics = new MulticlassMetrics(mlpPredictionAndLabels) - assert(mlpMetrics.confusionMatrix ~== lrMetrics.confusionMatrix absTol 100) + assert(mlpMetrics.confusionMatrix.asML ~== lrMetrics.confusionMatrix.asML absTol 100) + } + + test("read/write: MultilayerPerceptronClassifier") { + val mlp = new MultilayerPerceptronClassifier() + .setLayers(Array(2, 3, 2)) + .setMaxIter(5) + .setBlockSize(2) + .setSeed(42) + .setTol(0.1) + .setFeaturesCol("myFeatures") + .setLabelCol("myLabel") + .setPredictionCol("myPrediction") + + testDefaultReadWrite(mlp, testParams = true) + } + + test("read/write: MultilayerPerceptronClassificationModel") { + val mlp = new MultilayerPerceptronClassifier().setLayers(Array(2, 3, 2)).setMaxIter(5) + val mlpModel = mlp.fit(dataset) + val newMlpModel = testDefaultReadWrite(mlpModel, testParams = true) + assert(newMlpModel.layers === mlpModel.layers) + assert(newMlpModel.weights === mlpModel.weights) + } + + test("should support all NumericType labels and not support other types") { + val layers = Array(3, 2) + val mpc = new MultilayerPerceptronClassifier().setLayers(layers).setMaxIter(1) + MLTestingUtils.checkNumericTypes[ + MultilayerPerceptronClassificationModel, MultilayerPerceptronClassifier]( + mpc, spark) { (expected, actual) => + assert(expected.layers === actual.layers) + assert(expected.weights === actual.weights) + } } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 98bc9511163e..9730dd68a3b2 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -17,19 +17,44 @@ package org.apache.spark.ml.classification -import breeze.linalg.{Vector => BV} +import scala.util.Random -import org.apache.spark.SparkFunSuite +import breeze.linalg.{DenseVector => BDV, Vector => BV} +import breeze.stats.distributions.{Multinomial => BrzMultinomial} + +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.classification.NaiveBayes.{Bernoulli, Multinomial} +import org.apache.spark.ml.classification.NaiveBayesSuite._ +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.mllib.classification.NaiveBayes.{Multinomial, Bernoulli} -import org.apache.spark.mllib.linalg._ +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ -import org.apache.spark.mllib.classification.NaiveBayesSuite._ -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.Row +import org.apache.spark.sql.{DataFrame, Dataset, Row} + +class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + + @transient var dataset: Dataset[_] = _ + @transient var bernoulliDataset: Dataset[_] = _ -class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { + private val seed = 42 + + override def beforeAll(): Unit = { + super.beforeAll() + + val pi = Array(0.3, 0.3, 0.4).map(math.log) + val theta = Array( + Array(0.30, 0.30, 0.30, 0.30), // label 0 + Array(0.30, 0.30, 0.30, 0.30), // label 1 + Array(0.40, 0.40, 0.40, 0.40) // label 2 + ).map(_.map(math.log)) + + dataset = generateNaiveBayesInput(pi, theta, 100, seed).toDF() + bernoulliDataset = generateNaiveBayesInput(pi, theta, 100, seed, "bernoulli").toDF() + } def validatePrediction(predictionAndLabels: DataFrame): Unit = { val numOfErrorPredictions = predictionAndLabels.collect().count { @@ -50,7 +75,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { } def expectedMultinomialProbabilities(model: NaiveBayesModel, feature: Vector): Vector = { - val logClassProbs: BV[Double] = model.pi.toBreeze + model.theta.multiply(feature).toBreeze + val logClassProbs: BV[Double] = model.pi.asBreeze + model.theta.multiply(feature).asBreeze val classProbs = logClassProbs.toArray.map(math.exp) val classProbsSum = classProbs.sum Vectors.dense(classProbs.map(_ / classProbsSum)) @@ -59,8 +84,8 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { def expectedBernoulliProbabilities(model: NaiveBayesModel, feature: Vector): Vector = { val negThetaMatrix = model.theta.map(v => math.log(1.0 - math.exp(v))) val negFeature = Vectors.dense(feature.toArray.map(v => 1.0 - v)) - val piTheta: BV[Double] = model.pi.toBreeze + model.theta.multiply(feature).toBreeze - val logClassProbs: BV[Double] = piTheta + negThetaMatrix.multiply(negFeature).toBreeze + val piTheta: BV[Double] = model.pi.asBreeze + model.theta.multiply(feature).asBreeze + val logClassProbs: BV[Double] = piTheta + negThetaMatrix.multiply(negFeature).asBreeze val classProbs = logClassProbs.toArray.map(math.exp) val classProbsSum = classProbs.sum Vectors.dense(classProbs.map(_ / classProbsSum)) @@ -71,7 +96,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { model: NaiveBayesModel, modelType: String): Unit = { featureAndProbabilities.collect().foreach { - case Row(features: Vector, probability: Vector) => { + case Row(features: Vector, probability: Vector) => assert(probability.toArray.sum ~== 1.0 relTol 1.0e-10) val expected = modelType match { case Multinomial => @@ -82,10 +107,14 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { throw new UnknownError(s"Invalid modelType: $modelType.") } assert(probability ~== expected relTol 1.0e-10) - } } } + test("model types") { + assert(Multinomial === "multinomial") + assert(Bernoulli === "bernoulli") + } + test("params") { ParamsSuite.checkParams(new NaiveBayes) val model = new NaiveBayesModel("nb", pi = Vectors.dense(Array(0.2, 0.8)), @@ -113,16 +142,17 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { val pi = Vectors.dense(piArray) val theta = new DenseMatrix(3, 4, thetaArray.flatten, true) - val testDataset = sqlContext.createDataFrame(generateNaiveBayesInput( - piArray, thetaArray, nPoints, 42, "multinomial")) + val testDataset = + generateNaiveBayesInput(piArray, thetaArray, nPoints, seed, "multinomial").toDF() val nb = new NaiveBayes().setSmoothing(1.0).setModelType("multinomial") val model = nb.fit(testDataset) validateModelFit(pi, theta, model) assert(model.hasParent) + MLTestingUtils.checkCopyAndUids(nb, model) - val validationDataset = sqlContext.createDataFrame(generateNaiveBayesInput( - piArray, thetaArray, nPoints, 17, "multinomial")) + val validationDataset = + generateNaiveBayesInput(piArray, thetaArray, nPoints, 17, "multinomial").toDF() val predictionAndLabels = model.transform(validationDataset).select("prediction", "label") validatePrediction(predictionAndLabels) @@ -130,6 +160,32 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { val featureAndProbabilities = model.transform(validationDataset) .select("features", "probability") validateProbabilities(featureAndProbabilities, model, "multinomial") + + ProbabilisticClassifierSuite.testPredictMethods[ + Vector, NaiveBayesModel](model, testDataset) + } + + test("Naive Bayes with weighted samples") { + val numClasses = 3 + def modelEquals(m1: NaiveBayesModel, m2: NaiveBayesModel): Unit = { + assert(m1.pi ~== m2.pi relTol 0.01) + assert(m1.theta ~== m2.theta relTol 0.01) + } + val testParams = Seq[(String, Dataset[_])]( + ("bernoulli", bernoulliDataset), + ("multinomial", dataset) + ) + testParams.foreach { case (family, dataset) => + // NaiveBayes is sensitive to constant scaling of the weights unless smoothing is set to 0 + val estimatorNoSmoothing = new NaiveBayes().setSmoothing(0.0).setModelType(family) + val estimatorWithSmoothing = new NaiveBayes().setModelType(family) + MLTestingUtils.testArbitrarilyScaledWeights[NaiveBayesModel, NaiveBayes]( + dataset.as[LabeledPoint], estimatorNoSmoothing, modelEquals) + MLTestingUtils.testOutliersWithSmallWeights[NaiveBayesModel, NaiveBayes]( + dataset.as[LabeledPoint], estimatorWithSmoothing, numClasses, modelEquals, outlierRatio = 3) + MLTestingUtils.testOversamplingVsWeighting[NaiveBayesModel, NaiveBayes]( + dataset.as[LabeledPoint], estimatorWithSmoothing, modelEquals, seed) + } } test("Naive Bayes Bernoulli") { @@ -143,16 +199,16 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { val pi = Vectors.dense(piArray) val theta = new DenseMatrix(3, 12, thetaArray.flatten, true) - val testDataset = sqlContext.createDataFrame(generateNaiveBayesInput( - piArray, thetaArray, nPoints, 45, "bernoulli")) + val testDataset = + generateNaiveBayesInput(piArray, thetaArray, nPoints, 45, "bernoulli").toDF() val nb = new NaiveBayes().setSmoothing(1.0).setModelType("bernoulli") val model = nb.fit(testDataset) validateModelFit(pi, theta, model) assert(model.hasParent) - val validationDataset = sqlContext.createDataFrame(generateNaiveBayesInput( - piArray, thetaArray, nPoints, 20, "bernoulli")) + val validationDataset = + generateNaiveBayesInput(piArray, thetaArray, nPoints, 20, "bernoulli").toDF() val predictionAndLabels = model.transform(validationDataset).select("prediction", "label") validatePrediction(predictionAndLabels) @@ -160,5 +216,144 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { val featureAndProbabilities = model.transform(validationDataset) .select("features", "probability") validateProbabilities(featureAndProbabilities, model, "bernoulli") + + ProbabilisticClassifierSuite.testPredictMethods[ + Vector, NaiveBayesModel](model, testDataset) + } + + test("detect negative values") { + val dense = spark.createDataFrame(Seq( + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(-1.0)), + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(1.0, Vectors.dense(0.0)))) + intercept[SparkException] { + new NaiveBayes().fit(dense) + } + val sparse = spark.createDataFrame(Seq( + LabeledPoint(1.0, Vectors.sparse(1, Array(0), Array(1.0))), + LabeledPoint(0.0, Vectors.sparse(1, Array(0), Array(-1.0))), + LabeledPoint(1.0, Vectors.sparse(1, Array(0), Array(1.0))), + LabeledPoint(1.0, Vectors.sparse(1, Array.empty, Array.empty)))) + intercept[SparkException] { + new NaiveBayes().fit(sparse) + } + val nan = spark.createDataFrame(Seq( + LabeledPoint(1.0, Vectors.sparse(1, Array(0), Array(1.0))), + LabeledPoint(0.0, Vectors.sparse(1, Array(0), Array(Double.NaN))), + LabeledPoint(1.0, Vectors.sparse(1, Array(0), Array(1.0))), + LabeledPoint(1.0, Vectors.sparse(1, Array.empty, Array.empty)))) + intercept[SparkException] { + new NaiveBayes().fit(nan) + } + } + + test("detect non zero or one values in Bernoulli") { + val badTrain = spark.createDataFrame(Seq( + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(2.0)), + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(1.0, Vectors.dense(0.0)))) + + intercept[SparkException] { + new NaiveBayes().setModelType(Bernoulli).setSmoothing(1.0).fit(badTrain) + } + + val okTrain = spark.createDataFrame(Seq( + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(1.0, Vectors.dense(1.0)))) + + val model = new NaiveBayes().setModelType(Bernoulli).setSmoothing(1.0).fit(okTrain) + + val badPredict = spark.createDataFrame(Seq( + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(1.0, Vectors.dense(2.0)), + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(1.0, Vectors.dense(0.0)))) + + intercept[SparkException] { + model.transform(badPredict).collect() + } + } + + test("read/write") { + def checkModelData(model: NaiveBayesModel, model2: NaiveBayesModel): Unit = { + assert(model.pi === model2.pi) + assert(model.theta === model2.theta) + } + val nb = new NaiveBayes() + testEstimatorAndModelReadWrite(nb, dataset, NaiveBayesSuite.allParamSettings, + NaiveBayesSuite.allParamSettings, checkModelData) + } + + test("should support all NumericType labels and weights, and not support other types") { + val nb = new NaiveBayes() + MLTestingUtils.checkNumericTypes[NaiveBayesModel, NaiveBayes]( + nb, spark) { (expected, actual) => + assert(expected.pi === actual.pi) + assert(expected.theta === actual.theta) + } + } +} + +object NaiveBayesSuite { + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "predictionCol" -> "myPrediction", + "smoothing" -> 0.1 + ) + + private def calcLabel(p: Double, pi: Array[Double]): Int = { + var sum = 0.0 + for (j <- 0 until pi.length) { + sum += pi(j) + if (p < sum) return j + } + -1 + } + + // Generate input of the form Y = (theta * x).argmax() + def generateNaiveBayesInput( + pi: Array[Double], // 1XC + theta: Array[Array[Double]], // CXD + nPoints: Int, + seed: Int, + modelType: String = Multinomial, + sample: Int = 10): Seq[LabeledPoint] = { + val D = theta(0).length + val rnd = new Random(seed) + val _pi = pi.map(math.exp) + val _theta = theta.map(row => row.map(math.exp)) + + for (i <- 0 until nPoints) yield { + val y = calcLabel(rnd.nextDouble(), _pi) + val xi = modelType match { + case Bernoulli => Array.tabulate[Double] (D) { j => + if (rnd.nextDouble () < _theta(y)(j) ) 1 else 0 + } + case Multinomial => + val mult = BrzMultinomial(BDV(_theta(y))) + val emptyMap = (0 until D).map(x => (x, 0.0)).toMap + val counts = emptyMap ++ mult.sample(sample).groupBy(x => x).map { + case (index, reps) => (index, reps.size.toDouble) + } + counts.toArray.sortBy(_._1).map(_._2) + case _ => + // This should never happen. + throw new UnknownError(s"Invalid modelType: $modelType.") + } + + LabeledPoint(y, Vectors.dense(xi)) + } } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala index 5ea71c5317b7..25bad59b9c9c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala @@ -19,23 +19,28 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.NominalAttribute +import org.apache.spark.ml.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.feature.StringIndexer +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.{ParamMap, ParamsSuite} -import org.apache.spark.ml.util.{MLTestingUtils, MetadataUtils} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MetadataUtils, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.classification.LogisticRegressionSuite._ import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.Metadata -class OneVsRestSuite extends SparkFunSuite with MLlibTestSparkContext { +class OneVsRestSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { - @transient var dataset: DataFrame = _ + import testImplicits._ + + @transient var dataset: Dataset[_] = _ @transient var rdd: RDD[LabeledPoint] = _ override def beforeAll(): Unit = { @@ -53,7 +58,7 @@ class OneVsRestSuite extends SparkFunSuite with MLlibTestSparkContext { val xVariance = Array(0.6856, 0.1899, 3.116, 0.581) rdd = sc.parallelize(generateMultinomialLogisticInput( coefficients, xMean, xVariance, true, nPoints, 42), 2) - dataset = sqlContext.createDataFrame(rdd) + dataset = rdd.toDF() } test("params") { @@ -71,30 +76,67 @@ class OneVsRestSuite extends SparkFunSuite with MLlibTestSparkContext { assert(ova.getPredictionCol === "prediction") val ovaModel = ova.fit(dataset) - // copied model must have the same parent. - MLTestingUtils.checkCopy(ovaModel) + MLTestingUtils.checkCopyAndUids(ova, ovaModel) - assert(ovaModel.models.size === numClasses) + assert(ovaModel.models.length === numClasses) val transformedDataset = ovaModel.transform(dataset) // check for label metadata in prediction col val predictionColSchema = transformedDataset.schema(ovaModel.getPredictionCol) assert(MetadataUtils.getNumClasses(predictionColSchema) === Some(3)) - val ovaResults = transformedDataset - .select("prediction", "label") - .map(row => (row.getDouble(0), row.getDouble(1))) + val ovaResults = transformedDataset.select("prediction", "label").rdd.map { + row => (row.getDouble(0), row.getDouble(1)) + } val lr = new LogisticRegressionWithLBFGS().setIntercept(true).setNumClasses(numClasses) lr.optimizer.setRegParam(0.1).setNumIterations(100) - val model = lr.run(rdd) - val results = model.predict(rdd.map(_.features)).zip(rdd.map(_.label)) + val model = lr.run(rdd.map(OldLabeledPoint.fromML)) + val results = model.predict(rdd.map(p => OldVectors.fromML(p.features))).zip(rdd.map(_.label)) // determine the #confusion matrix in each class. // bound how much error we allow compared to multinomial logistic regression. val expectedMetrics = new MulticlassMetrics(results) val ovaMetrics = new MulticlassMetrics(ovaResults) - assert(expectedMetrics.confusionMatrix ~== ovaMetrics.confusionMatrix absTol 400) + assert(expectedMetrics.confusionMatrix.asML ~== ovaMetrics.confusionMatrix.asML absTol 400) + } + + test("one-vs-rest: tuning parallelism does not change output") { + val ovaPar1 = new OneVsRest() + .setClassifier(new LogisticRegression) + + val ovaModelPar1 = ovaPar1.fit(dataset) + + val transformedDatasetPar1 = ovaModelPar1.transform(dataset) + + val ovaResultsPar1 = transformedDatasetPar1.select("prediction", "label").rdd.map { + row => (row.getDouble(0), row.getDouble(1)) + } + + val ovaPar2 = new OneVsRest() + .setClassifier(new LogisticRegression) + .setParallelism(2) + + val ovaModelPar2 = ovaPar2.fit(dataset) + + val transformedDatasetPar2 = ovaModelPar2.transform(dataset) + + val ovaResultsPar2 = transformedDatasetPar2.select("prediction", "label").rdd.map { + row => (row.getDouble(0), row.getDouble(1)) + } + + val metricsPar1 = new MulticlassMetrics(ovaResultsPar1) + val metricsPar2 = new MulticlassMetrics(ovaResultsPar2) + assert(metricsPar1.confusionMatrix == metricsPar2.confusionMatrix) + + ovaModelPar1.models.zip(ovaModelPar2.models).foreach { + case (lrModel1: LogisticRegressionModel, lrModel2: LogisticRegressionModel) => + assert(lrModel1.coefficients ~== lrModel2.coefficients relTol 1E-3) + assert(lrModel1.intercept ~== lrModel2.intercept relTol 1E-3) + case other => + throw new AssertionError(s"Loaded OneVsRestModel expected model of type" + + s" LogisticRegressionModel but found ${other.getClass.getName}") + } } test("one-vs-rest: pass label metadata correctly during train") { @@ -132,6 +174,17 @@ class OneVsRestSuite extends SparkFunSuite with MLlibTestSparkContext { assert(outputFields.contains("p")) } + test("SPARK-18625 : OneVsRestModel should support setFeaturesCol and setPredictionCol") { + val ova = new OneVsRest().setClassifier(new LogisticRegression) + val ovaModel = ova.fit(dataset) + val dataset2 = dataset.select(col("label").as("y"), col("features").as("fea")) + ovaModel.setFeaturesCol("fea") + ovaModel.setPredictionCol("pred") + val transformedDataset = ovaModel.transform(dataset2) + val outputFields = transformedDataset.schema.fieldNames.toSet + assert(outputFields === Set("y", "fea", "pred")) + } + test("SPARK-8049: OneVsRest shouldn't output temp columns") { val logReg = new LogisticRegression() .setMaxIter(1) @@ -141,6 +194,16 @@ class OneVsRestSuite extends SparkFunSuite with MLlibTestSparkContext { assert(output.schema.fieldNames.toSet === Set("label", "features", "prediction")) } + test("SPARK-21306: OneVsRest should support setWeightCol") { + val dataset2 = dataset.withColumn("weight", lit(1)) + // classifier inherits hasWeightCol + val ova = new OneVsRest().setWeightCol("weight").setClassifier(new LogisticRegression()) + assert(ova.fit(dataset2) !== null) + // classifier doesn't inherit hasWeightCol + val ova2 = new OneVsRest().setWeightCol("weight").setClassifier(new DecisionTreeClassifier()) + assert(ova2.fit(dataset2) !== null) + } + test("OneVsRest.copy and OneVsRestModel.copy") { val lr = new LogisticRegression() .setMaxIter(1) @@ -160,6 +223,84 @@ class OneVsRestSuite extends SparkFunSuite with MLlibTestSparkContext { require(m.getThreshold === 0.1, "copy should handle extra model params") } } + + test("read/write: OneVsRest") { + val lr = new LogisticRegression().setMaxIter(10).setRegParam(0.01) + + val ova = new OneVsRest() + .setClassifier(lr) + .setLabelCol("myLabel") + .setFeaturesCol("myFeature") + .setPredictionCol("myPrediction") + + val ova2 = testDefaultReadWrite(ova, testParams = false) + assert(ova.uid === ova2.uid) + assert(ova.getFeaturesCol === ova2.getFeaturesCol) + assert(ova.getLabelCol === ova2.getLabelCol) + assert(ova.getPredictionCol === ova2.getPredictionCol) + + ova2.getClassifier match { + case lr2: LogisticRegression => + assert(lr.uid === lr2.uid) + assert(lr.getMaxIter === lr2.getMaxIter) + assert(lr.getRegParam === lr2.getRegParam) + case other => + throw new AssertionError(s"Loaded OneVsRest expected classifier of type" + + s" LogisticRegression but found ${other.getClass.getName}") + } + } + + test("read/write: OneVsRestModel") { + def checkModelData(model: OneVsRestModel, model2: OneVsRestModel): Unit = { + assert(model.uid === model2.uid) + assert(model.getFeaturesCol === model2.getFeaturesCol) + assert(model.getLabelCol === model2.getLabelCol) + assert(model.getPredictionCol === model2.getPredictionCol) + + val classifier = model.getClassifier.asInstanceOf[LogisticRegression] + + model2.getClassifier match { + case lr2: LogisticRegression => + assert(classifier.uid === lr2.uid) + assert(classifier.getMaxIter === lr2.getMaxIter) + assert(classifier.getRegParam === lr2.getRegParam) + case other => + throw new AssertionError(s"Loaded OneVsRestModel expected classifier of type" + + s" LogisticRegression but found ${other.getClass.getName}") + } + + assert(model.labelMetadata === model2.labelMetadata) + model.models.zip(model2.models).foreach { + case (lrModel1: LogisticRegressionModel, lrModel2: LogisticRegressionModel) => + assert(lrModel1.uid === lrModel2.uid) + assert(lrModel1.coefficients === lrModel2.coefficients) + assert(lrModel1.intercept === lrModel2.intercept) + case other => + throw new AssertionError(s"Loaded OneVsRestModel expected model of type" + + s" LogisticRegressionModel but found ${other.getClass.getName}") + } + } + + val lr = new LogisticRegression().setMaxIter(10).setRegParam(0.01) + val ova = new OneVsRest().setClassifier(lr) + val ovaModel = ova.fit(dataset) + val newOvaModel = testDefaultReadWrite(ovaModel, testParams = false) + checkModelData(ovaModel, newOvaModel) + } + + test("should support all NumericType labels and not support other types") { + val ovr = new OneVsRest().setClassifier(new LogisticRegression().setMaxIter(1)) + MLTestingUtils.checkNumericTypes[OneVsRestModel, OneVsRest]( + ovr, spark) { (expected, actual) => + val expectedModels = expected.models.map(m => m.asInstanceOf[LogisticRegressionModel]) + val actualModels = actual.models.map(m => m.asInstanceOf[LogisticRegressionModel]) + assert(expectedModels.length === actualModels.length) + expectedModels.zip(actualModels).foreach { case (e, a) => + assert(e.intercept === a.intercept) + assert(e.coefficients.toArray === a.coefficients.toArray) + } + } + } } private class MockLogisticRegression(uid: String) extends LogisticRegression(uid) { @@ -168,7 +309,7 @@ private class MockLogisticRegression(uid: String) extends LogisticRegression(uid setMaxIter(1) - override protected def train(dataset: DataFrame): LogisticRegressionModel = { + override protected[spark] def train(dataset: Dataset[_]): LogisticRegressionModel = { val labelSchema = dataset.schema($(labelCol)) // check for label attribute propagation. assert(MetadataUtils.getNumClasses(labelSchema).forall(_ == 2)) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala index fb5f00e0646c..4ecd5a05365e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/ProbabilisticClassifierSuite.scala @@ -18,7 +18,10 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.sql.{Dataset, Row} final class TestProbabilisticClassificationModel( override val uid: String, @@ -36,8 +39,8 @@ final class TestProbabilisticClassificationModel( rawPrediction } - def friendlyPredict(input: Vector): Double = { - predict(input) + def friendlyPredict(values: Double*): Double = { + predict(Vectors.dense(values.toArray)) } } @@ -45,15 +48,107 @@ final class TestProbabilisticClassificationModel( class ProbabilisticClassifierSuite extends SparkFunSuite { test("test thresholding") { - val thresholds = Array(0.5, 0.2) val testModel = new TestProbabilisticClassificationModel("myuid", 2, 2) - .setThresholds(thresholds) - assert(testModel.friendlyPredict(Vectors.dense(Array(1.0, 1.0))) === 1.0) - assert(testModel.friendlyPredict(Vectors.dense(Array(1.0, 0.2))) === 0.0) + .setThresholds(Array(0.5, 0.2)) + assert(testModel.friendlyPredict(1.0, 1.0) === 1.0) + assert(testModel.friendlyPredict(1.0, 0.2) === 0.0) } test("test thresholding not required") { val testModel = new TestProbabilisticClassificationModel("myuid", 2, 2) - assert(testModel.friendlyPredict(Vectors.dense(Array(1.0, 2.0))) === 1.0) + assert(testModel.friendlyPredict(1.0, 2.0) === 1.0) } + + test("test tiebreak") { + val testModel = new TestProbabilisticClassificationModel("myuid", 2, 2) + .setThresholds(Array(0.4, 0.4)) + assert(testModel.friendlyPredict(0.6, 0.6) === 0.0) + } + + test("test one zero threshold") { + val testModel = new TestProbabilisticClassificationModel("myuid", 2, 2) + .setThresholds(Array(0.0, 0.1)) + assert(testModel.friendlyPredict(1.0, 10.0) === 0.0) + assert(testModel.friendlyPredict(0.0, 10.0) === 1.0) + } + + test("bad thresholds") { + intercept[IllegalArgumentException] { + new TestProbabilisticClassificationModel("myuid", 2, 2).setThresholds(Array(0.0, 0.0)) + } + intercept[IllegalArgumentException] { + new TestProbabilisticClassificationModel("myuid", 2, 2).setThresholds(Array(-0.1, 0.1)) + } + } +} + +object ProbabilisticClassifierSuite { + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = ClassifierSuite.allParamSettings ++ Map( + "probabilityCol" -> "myProbability", + "thresholds" -> Array(0.4, 0.6) + ) + + /** + * Helper for testing that a ProbabilisticClassificationModel computes + * the same predictions across all combinations of output columns + * (rawPrediction/probability/prediction) turned on/off. Makes sure the + * output column values match by comparing vs. the case with all 3 output + * columns turned on. + */ + def testPredictMethods[ + FeaturesType, + M <: ProbabilisticClassificationModel[FeaturesType, M]]( + model: M, testData: Dataset[_]): Unit = { + + val allColModel = model.copy(ParamMap.empty) + .setRawPredictionCol("rawPredictionAll") + .setProbabilityCol("probabilityAll") + .setPredictionCol("predictionAll") + val allColResult = allColModel.transform(testData) + + for (rawPredictionCol <- Seq("", "rawPredictionSingle")) { + for (probabilityCol <- Seq("", "probabilitySingle")) { + for (predictionCol <- Seq("", "predictionSingle")) { + val newModel = model.copy(ParamMap.empty) + .setRawPredictionCol(rawPredictionCol) + .setProbabilityCol(probabilityCol) + .setPredictionCol(predictionCol) + + val result = newModel.transform(allColResult) + + import org.apache.spark.sql.functions._ + + val resultRawPredictionCol = + if (rawPredictionCol.isEmpty) col("rawPredictionAll") else col(rawPredictionCol) + val resultProbabilityCol = + if (probabilityCol.isEmpty) col("probabilityAll") else col(probabilityCol) + val resultPredictionCol = + if (predictionCol.isEmpty) col("predictionAll") else col(predictionCol) + + result.select( + resultRawPredictionCol, col("rawPredictionAll"), + resultProbabilityCol, col("probabilityAll"), + resultPredictionCol, col("predictionAll") + ).collect().foreach { + case Row( + rawPredictionSingle: Vector, rawPredictionAll: Vector, + probabilitySingle: Vector, probabilityAll: Vector, + predictionSingle: Double, predictionAll: Double + ) => { + assert(rawPredictionSingle ~== rawPredictionAll relTol 1E-3) + assert(probabilitySingle ~== probabilityAll relTol 1E-3) + assert(predictionSingle ~== predictionAll relTol 1E-3) + } + } + } + } + } + } + } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala index deb8ec771cb2..2cca2e6c0469 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.impl.TreeTests +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.tree.LeafNode -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.tree.impl.TreeTests +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, RandomForest => OldRandomForest} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -34,9 +35,11 @@ import org.apache.spark.sql.{DataFrame, Row} /** * Test suite for [[RandomForestClassifier]]. */ -class RandomForestClassifierSuite extends SparkFunSuite with MLlibTestSparkContext { +class RandomForestClassifierSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { import RandomForestClassifierSuite.compareAPIs + import testImplicits._ private var orderedLabeledPoints50_1000: RDD[LabeledPoint] = _ private var orderedLabeledPoints5_20: RDD[LabeledPoint] = _ @@ -45,8 +48,10 @@ class RandomForestClassifierSuite extends SparkFunSuite with MLlibTestSparkConte super.beforeAll() orderedLabeledPoints50_1000 = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)) + .map(_.asML) orderedLabeledPoints5_20 = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 5, 20)) + .map(_.asML) } ///////////////////////////////////////////////////////////////////////////// @@ -105,7 +110,7 @@ class RandomForestClassifierSuite extends SparkFunSuite with MLlibTestSparkConte compareAPIs(rdd, rf, categoricalFeatures, numClasses) } - test("subsampling rate in RandomForest"){ + test("subsampling rate in RandomForest") { val rdd = orderedLabeledPoints5_20 val categoricalFeatures = Map.empty[Int, Int] val numClasses = 2 @@ -136,8 +141,7 @@ class RandomForestClassifierSuite extends SparkFunSuite with MLlibTestSparkConte val df: DataFrame = TreeTests.setMetadata(rdd, categoricalFeatures, numClasses) val model = rf.fit(df) - // copied model must have the same parent. - MLTestingUtils.checkCopy(model) + MLTestingUtils.checkCopyAndUids(rf, model) val predictions = model.transform(df) .select(rf.getPredictionCol, rf.getRawPredictionCol, rf.getProbabilityCol) @@ -151,11 +155,20 @@ class RandomForestClassifierSuite extends SparkFunSuite with MLlibTestSparkConte "probability prediction mismatch") assert(probPred.toArray.sum ~== 1.0 relTol 1E-5) } + ProbabilisticClassifierSuite.testPredictMethods[ + Vector, RandomForestClassificationModel](model, df) + } + + test("Fitting without numClasses in metadata") { + val df: DataFrame = TreeTests.featureImportanceData(sc).toDF() + val rf = new RandomForestClassifier().setMaxDepth(1).setNumTrees(1) + rf.fit(df) } ///////////////////////////////////////////////////////////////////////////// // Tests of feature importance ///////////////////////////////////////////////////////////////////////////// + test("Feature importance with toy data") { val numClasses = 2 val rf = new RandomForestClassifier() @@ -167,46 +180,48 @@ class RandomForestClassifierSuite extends SparkFunSuite with MLlibTestSparkConte .setSeed(123) // In this data, feature 1 is very important. - val data: RDD[LabeledPoint] = sc.parallelize(Seq( - new LabeledPoint(0, Vectors.dense(1, 0, 0, 0, 1)), - new LabeledPoint(1, Vectors.dense(1, 1, 0, 1, 0)), - new LabeledPoint(1, Vectors.dense(1, 1, 0, 0, 0)), - new LabeledPoint(0, Vectors.dense(1, 0, 0, 0, 0)), - new LabeledPoint(1, Vectors.dense(1, 1, 0, 0, 0)) - )) + val data: RDD[LabeledPoint] = TreeTests.featureImportanceData(sc) val categoricalFeatures = Map.empty[Int, Int] val df: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses) val importances = rf.fit(df).featureImportances val mostImportantFeature = importances.argmax assert(mostImportantFeature === 1) + assert(importances.toArray.sum === 1.0) + assert(importances.toArray.forall(_ >= 0.0)) + } + + test("should support all NumericType labels and not support other types") { + val rf = new RandomForestClassifier().setMaxDepth(1) + MLTestingUtils.checkNumericTypes[RandomForestClassificationModel, RandomForestClassifier]( + rf, spark) { (expected, actual) => + TreeTests.checkEqual(expected, actual) + } } ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// - // TODO: Reinstate test once save/load are implemented SPARK-6725 - /* - test("model save/load") { - val tempDir = Utils.createTempDir() - val path = tempDir.toURI.toString - - val trees = - Range(0, 3).map(_ => OldDecisionTreeSuite.createModel(OldAlgo.Classification)).toArray - val oldModel = new OldRandomForestModel(OldAlgo.Classification, trees) - val newModel = RandomForestClassificationModel.fromOld(oldModel) - - // Save model, load it back, and compare. - try { - newModel.save(sc, path) - val sameNewModel = RandomForestClassificationModel.load(sc, path) - TreeTests.checkEqual(newModel, sameNewModel) - } finally { - Utils.deleteRecursively(tempDir) + test("read/write") { + def checkModelData( + model: RandomForestClassificationModel, + model2: RandomForestClassificationModel): Unit = { + TreeTests.checkEqual(model, model2) + assert(model.numFeatures === model2.numFeatures) + assert(model.numClasses === model2.numClasses) } + + val rf = new RandomForestClassifier().setNumTrees(2) + val rdd = TreeTests.getTreeReadWriteData(sc) + + val allParamSettings = TreeTests.allParamSettings ++ Map("impurity" -> "entropy") + + val continuousData: DataFrame = + TreeTests.setMetadata(rdd, Map.empty[Int, Int], numClasses = 2) + testEstimatorAndModelReadWrite(rf, continuousData, allParamSettings, + allParamSettings, checkModelData) } - */ } private object RandomForestClassifierSuite extends SparkFunSuite { @@ -224,7 +239,8 @@ private object RandomForestClassifierSuite extends SparkFunSuite { val oldStrategy = rf.getOldStrategy(categoricalFeatures, numClasses, OldAlgo.Classification, rf.getOldImpurity) val oldModel = OldRandomForest.trainClassifier( - data, oldStrategy, rf.getNumTrees, rf.getFeatureSubsetStrategy, rf.getSeed.toInt) + data.map(OldLabeledPoint.fromML), oldStrategy, rf.getNumTrees, rf.getFeatureSubsetStrategy, + rf.getSeed.toInt) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses) val newModel = rf.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala new file mode 100644 index 000000000000..fa7471fa2d65 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.clustering + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.Dataset + +class BisectingKMeansSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + final val k = 5 + @transient var dataset: Dataset[_] = _ + + @transient var sparseDataset: Dataset[_] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + dataset = KMeansSuite.generateKMeansData(spark, 50, 3, k) + sparseDataset = KMeansSuite.generateSparseData(spark, 10, 1000, 42) + } + + test("default parameters") { + val bkm = new BisectingKMeans() + + assert(bkm.getK === 4) + assert(bkm.getFeaturesCol === "features") + assert(bkm.getPredictionCol === "prediction") + assert(bkm.getMaxIter === 20) + assert(bkm.getMinDivisibleClusterSize === 1.0) + val model = bkm.setMaxIter(1).fit(dataset) + + MLTestingUtils.checkCopyAndUids(bkm, model) + assert(model.hasSummary) + val copiedModel = model.copy(ParamMap.empty) + assert(copiedModel.hasSummary) + } + + test("SPARK-16473: Verify Bisecting K-Means does not fail in edge case where" + + "one cluster is empty after split") { + val bkm = new BisectingKMeans() + .setK(k) + .setMinDivisibleClusterSize(4) + .setMaxIter(4) + .setSeed(123) + + // Verify fit does not fail on very sparse data + val model = bkm.fit(sparseDataset) + val result = model.transform(sparseDataset) + val numClusters = result.select("prediction").distinct().collect().length + // Verify we hit the edge case + assert(numClusters < k && numClusters > 1) + } + + test("setter/getter") { + val bkm = new BisectingKMeans() + .setK(9) + .setMinDivisibleClusterSize(2.0) + .setFeaturesCol("test_feature") + .setPredictionCol("test_prediction") + .setMaxIter(33) + .setSeed(123) + + assert(bkm.getK === 9) + assert(bkm.getFeaturesCol === "test_feature") + assert(bkm.getPredictionCol === "test_prediction") + assert(bkm.getMaxIter === 33) + assert(bkm.getMinDivisibleClusterSize === 2.0) + assert(bkm.getSeed === 123) + + intercept[IllegalArgumentException] { + new BisectingKMeans().setK(1) + } + + intercept[IllegalArgumentException] { + new BisectingKMeans().setMinDivisibleClusterSize(0) + } + } + + test("fit, transform and summary") { + val predictionColName = "bisecting_kmeans_prediction" + val bkm = new BisectingKMeans().setK(k).setPredictionCol(predictionColName).setSeed(1) + val model = bkm.fit(dataset) + assert(model.clusterCenters.length === k) + + val transformed = model.transform(dataset) + val expectedColumns = Array("features", predictionColName) + expectedColumns.foreach { column => + assert(transformed.columns.contains(column)) + } + val clusters = + transformed.select(predictionColName).rdd.map(_.getInt(0)).distinct().collect().toSet + assert(clusters.size === k) + assert(clusters === Set(0, 1, 2, 3, 4)) + assert(model.computeCost(dataset) < 0.1) + assert(model.hasParent) + + // Check validity of model summary + val numRows = dataset.count() + assert(model.hasSummary) + val summary: BisectingKMeansSummary = model.summary + assert(summary.predictionCol === predictionColName) + assert(summary.featuresCol === "features") + assert(summary.predictions.count() === numRows) + for (c <- Array(predictionColName, "features")) { + assert(summary.predictions.columns.contains(c)) + } + assert(summary.cluster.columns === Array(predictionColName)) + val clusterSizes = summary.clusterSizes + assert(clusterSizes.length === k) + assert(clusterSizes.sum === numRows) + assert(clusterSizes.forall(_ >= 0)) + + model.setSummary(None) + assert(!model.hasSummary) + } + + test("read/write") { + def checkModelData(model: BisectingKMeansModel, model2: BisectingKMeansModel): Unit = { + assert(model.clusterCenters === model2.clusterCenters) + } + val bisectingKMeans = new BisectingKMeans() + testEstimatorAndModelReadWrite(bisectingKMeans, dataset, BisectingKMeansSuite.allParamSettings, + BisectingKMeansSuite.allParamSettings, checkModelData) + } +} + +object BisectingKMeansSuite { + val allParamSettings: Map[String, Any] = Map( + "k" -> 3, + "maxIter" -> 2, + "seed" -> -1L, + "minDivisibleClusterSize" -> 2.0 + ) +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala new file mode 100644 index 000000000000..08b800b7e418 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala @@ -0,0 +1,308 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.clustering + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{DenseMatrix, Matrices, Vector, Vectors} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.stat.distribution.MultivariateGaussian +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{Dataset, Row} + + +class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext + with DefaultReadWriteTest { + + import testImplicits._ + import GaussianMixtureSuite._ + + final val k = 5 + private val seed = 538009335 + @transient var dataset: Dataset[_] = _ + @transient var denseDataset: Dataset[_] = _ + @transient var sparseDataset: Dataset[_] = _ + @transient var decompositionDataset: Dataset[_] = _ + @transient var rDataset: Dataset[_] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + dataset = KMeansSuite.generateKMeansData(spark, 50, 3, k) + denseDataset = denseData.map(FeatureData).toDF() + sparseDataset = denseData.map { point => + FeatureData(point.toSparse) + }.toDF() + decompositionDataset = decompositionData.map(FeatureData).toDF() + rDataset = rData.map(FeatureData).toDF() + } + + test("gmm fails on high dimensional data") { + val df = Seq( + Vectors.sparse(GaussianMixture.MAX_NUM_FEATURES + 1, Array(0, 4), Array(3.0, 8.0)), + Vectors.sparse(GaussianMixture.MAX_NUM_FEATURES + 1, Array(1, 5), Array(4.0, 9.0))) + .map(Tuple1.apply).toDF("features") + val gm = new GaussianMixture() + withClue(s"GMM should restrict the maximum number of features to be < " + + s"${GaussianMixture.MAX_NUM_FEATURES}") { + intercept[IllegalArgumentException] { + gm.fit(df) + } + } + } + + test("default parameters") { + val gm = new GaussianMixture() + + assert(gm.getK === 2) + assert(gm.getFeaturesCol === "features") + assert(gm.getPredictionCol === "prediction") + assert(gm.getMaxIter === 100) + assert(gm.getTol === 0.01) + val model = gm.setMaxIter(1).fit(dataset) + + MLTestingUtils.checkCopyAndUids(gm, model) + assert(model.hasSummary) + val copiedModel = model.copy(ParamMap.empty) + assert(copiedModel.hasSummary) + } + + test("set parameters") { + val gm = new GaussianMixture() + .setK(9) + .setFeaturesCol("test_feature") + .setPredictionCol("test_prediction") + .setProbabilityCol("test_probability") + .setMaxIter(33) + .setSeed(123) + .setTol(1e-3) + + assert(gm.getK === 9) + assert(gm.getFeaturesCol === "test_feature") + assert(gm.getPredictionCol === "test_prediction") + assert(gm.getProbabilityCol === "test_probability") + assert(gm.getMaxIter === 33) + assert(gm.getSeed === 123) + assert(gm.getTol === 1e-3) + } + + test("parameters validation") { + intercept[IllegalArgumentException] { + new GaussianMixture().setK(1) + } + } + + test("fit, transform and summary") { + val predictionColName = "gm_prediction" + val probabilityColName = "gm_probability" + val gm = new GaussianMixture().setK(k).setMaxIter(2).setPredictionCol(predictionColName) + .setProbabilityCol(probabilityColName).setSeed(1) + val model = gm.fit(dataset) + assert(model.hasParent) + assert(model.weights.length === k) + assert(model.gaussians.length === k) + + val transformed = model.transform(dataset) + val expectedColumns = Array("features", predictionColName, probabilityColName) + expectedColumns.foreach { column => + assert(transformed.columns.contains(column)) + } + + // Check prediction matches the highest probability, and probabilities sum to one. + transformed.select(predictionColName, probabilityColName).collect().foreach { + case Row(pred: Int, prob: Vector) => + val probArray = prob.toArray + val predFromProb = probArray.zipWithIndex.maxBy(_._1)._2 + assert(pred === predFromProb) + assert(probArray.sum ~== 1.0 absTol 1E-5) + } + + // Check validity of model summary + val numRows = dataset.count() + assert(model.hasSummary) + val summary: GaussianMixtureSummary = model.summary + assert(summary.predictionCol === predictionColName) + assert(summary.probabilityCol === probabilityColName) + assert(summary.featuresCol === "features") + assert(summary.predictions.count() === numRows) + for (c <- Array(predictionColName, probabilityColName, "features")) { + assert(summary.predictions.columns.contains(c)) + } + assert(summary.cluster.columns === Array(predictionColName)) + assert(summary.probability.columns === Array(probabilityColName)) + val clusterSizes = summary.clusterSizes + assert(clusterSizes.length === k) + assert(clusterSizes.sum === numRows) + assert(clusterSizes.forall(_ >= 0)) + + model.setSummary(None) + assert(!model.hasSummary) + } + + test("read/write") { + def checkModelData(model: GaussianMixtureModel, model2: GaussianMixtureModel): Unit = { + assert(model.weights === model2.weights) + assert(model.gaussians.map(_.mean) === model2.gaussians.map(_.mean)) + assert(model.gaussians.map(_.cov) === model2.gaussians.map(_.cov)) + } + val gm = new GaussianMixture() + testEstimatorAndModelReadWrite(gm, dataset, GaussianMixtureSuite.allParamSettings, + GaussianMixtureSuite.allParamSettings, checkModelData) + } + + test("univariate dense/sparse data with two clusters") { + val weights = Array(2.0 / 3.0, 1.0 / 3.0) + val means = Array(Vectors.dense(5.1604), Vectors.dense(-4.3673)) + val covs = Array(Matrices.dense(1, 1, Array(0.86644)), Matrices.dense(1, 1, Array(1.1098))) + val gaussians = means.zip(covs).map { case (mean, cov) => + new MultivariateGaussian(mean, cov) + } + val expected = new GaussianMixtureModel("dummy", weights, gaussians) + + Seq(denseDataset, sparseDataset).foreach { dataset => + val actual = new GaussianMixture().setK(2).setSeed(seed).fit(dataset) + modelEquals(expected, actual) + } + } + + test("check distributed decomposition") { + val k = 5 + val d = decompositionData.head.size + assert(GaussianMixture.shouldDistributeGaussians(k, d)) + + val gmm = new GaussianMixture().setK(k).setSeed(seed).fit(decompositionDataset) + assert(gmm.getK === k) + } + + test("multivariate data and check againt R mvnormalmixEM") { + /* + Using the following R code to generate data and train the model using mixtools package. + library(mvtnorm) + library(mixtools) + set.seed(1) + a <- rmvnorm(7, c(0, 0)) + b <- rmvnorm(8, c(10, 10)) + data <- rbind(a, b) + model <- mvnormalmixEM(data, k = 2) + model$lambda + + [1] 0.4666667 0.5333333 + + model$mu + + [1] 0.11731091 -0.06192351 + [1] 10.363673 9.897081 + + model$sigma + + [[1]] + [,1] [,2] + [1,] 0.62049934 0.06880802 + [2,] 0.06880802 1.27431874 + + [[2]] + [,1] [,2] + [1,] 0.2961543 0.160783 + [2,] 0.1607830 1.008878 + + model$loglik + + [1] -46.89499 + */ + val weights = Array(0.5333333, 0.4666667) + val means = Array(Vectors.dense(10.363673, 9.897081), Vectors.dense(0.11731091, -0.06192351)) + val covs = Array(Matrices.dense(2, 2, Array(0.2961543, 0.1607830, 0.160783, 1.008878)), + Matrices.dense(2, 2, Array(0.62049934, 0.06880802, 0.06880802, 1.27431874))) + val gaussians = means.zip(covs).map { case (mean, cov) => + new MultivariateGaussian(mean, cov) + } + + val expected = new GaussianMixtureModel("dummy", weights, gaussians) + val actual = new GaussianMixture().setK(2).setSeed(seed).fit(rDataset) + modelEquals(expected, actual) + + val llk = actual.summary.logLikelihood + assert(llk ~== -46.89499 absTol 1E-5) + } + + test("upper triangular matrix unpacking") { + /* + The full symmetric matrix is as follows: + 1.0 2.5 3.8 0.9 + 2.5 2.0 7.2 3.8 + 3.8 7.2 3.0 1.0 + 0.9 3.8 1.0 4.0 + */ + val triangularValues = Array(1.0, 2.5, 2.0, 3.8, 7.2, 3.0, 0.9, 3.8, 1.0, 4.0) + val symmetricValues = Array(1.0, 2.5, 3.8, 0.9, 2.5, 2.0, 7.2, 3.8, + 3.8, 7.2, 3.0, 1.0, 0.9, 3.8, 1.0, 4.0) + val symmetricMatrix = new DenseMatrix(4, 4, symmetricValues) + val expectedMatrix = GaussianMixture.unpackUpperTriangularMatrix(4, triangularValues) + assert(symmetricMatrix === expectedMatrix) + } +} + +object GaussianMixtureSuite extends SparkFunSuite { + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "predictionCol" -> "myPrediction", + "probabilityCol" -> "myProbability", + "k" -> 3, + "maxIter" -> 2, + "tol" -> 0.01 + ) + + val denseData = Seq( + Vectors.dense(-5.1971), Vectors.dense(-2.5359), Vectors.dense(-3.8220), + Vectors.dense(-5.2211), Vectors.dense(-5.0602), Vectors.dense( 4.7118), + Vectors.dense( 6.8989), Vectors.dense( 3.4592), Vectors.dense( 4.6322), + Vectors.dense( 5.7048), Vectors.dense( 4.6567), Vectors.dense( 5.5026), + Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734) + ) + + val decompositionData: Seq[Vector] = Seq.tabulate(25) { i: Int => + Vectors.dense(Array.tabulate(50)(i + _.toDouble)) + } + + val rData = Seq( + Vectors.dense(-0.6264538, 0.1836433), Vectors.dense(-0.8356286, 1.5952808), + Vectors.dense(0.3295078, -0.8204684), Vectors.dense(0.4874291, 0.7383247), + Vectors.dense(0.5757814, -0.3053884), Vectors.dense(1.5117812, 0.3898432), + Vectors.dense(-0.6212406, -2.2146999), Vectors.dense(11.1249309, 9.9550664), + Vectors.dense(9.9838097, 10.9438362), Vectors.dense(10.8212212, 10.5939013), + Vectors.dense(10.9189774, 10.7821363), Vectors.dense(10.0745650, 8.0106483), + Vectors.dense(10.6198257, 9.9438713), Vectors.dense(9.8442045, 8.5292476), + Vectors.dense(9.5218499, 10.4179416) + ) + + case class FeatureData(features: Vector) + + def modelEquals(m1: GaussianMixtureModel, m2: GaussianMixtureModel): Unit = { + assert(m1.weights.length === m2.weights.length) + for (i <- m1.weights.indices) { + assert(m1.weights(i) ~== m2.weights(i) absTol 1E-3) + assert(m1.gaussians(i).mean ~== m2.gaussians(i).mean absTol 1E-3) + assert(m1.gaussians(i).cov ~== m2.gaussians(i).cov absTol 1E-3) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index c05f90550d16..119fe1dead9a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -17,32 +17,27 @@ package org.apache.spark.ml.clustering +import scala.util.Random + import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.clustering.{KMeans => MLlibKMeans} -import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} private[clustering] case class TestRow(features: Vector) -object KMeansSuite { - def generateKMeansData(sql: SQLContext, rows: Int, dim: Int, k: Int): DataFrame = { - val sc = sql.sparkContext - val rdd = sc.parallelize(1 to rows).map(i => Vectors.dense(Array.fill(dim)((i % k).toDouble))) - .map(v => new TestRow(v)) - sql.createDataFrame(rdd) - } -} - -class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { +class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { final val k = 5 - @transient var dataset: DataFrame = _ + @transient var dataset: Dataset[_] = _ override def beforeAll(): Unit = { super.beforeAll() - dataset = KMeansSuite.generateKMeansData(sqlContext, 50, 3, k) + dataset = KMeansSuite.generateKMeansData(spark, 50, 3, k) } test("default parameters") { @@ -53,8 +48,14 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { assert(kmeans.getPredictionCol === "prediction") assert(kmeans.getMaxIter === 20) assert(kmeans.getInitMode === MLlibKMeans.K_MEANS_PARALLEL) - assert(kmeans.getInitSteps === 5) + assert(kmeans.getInitSteps === 2) assert(kmeans.getTol === 1e-4) + val model = kmeans.setMaxIter(1).fit(dataset) + + MLTestingUtils.checkCopyAndUids(kmeans, model) + assert(model.hasSummary) + val copiedModel = model.copy(ParamMap.empty) + assert(copiedModel.hasSummary) } test("set parameters") { @@ -90,7 +91,7 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { } } - test("fit & transform") { + test("fit, transform and summary") { val predictionColName = "kmeans_prediction" val kmeans = new KMeans().setK(k).setPredictionCol(predictionColName).setSeed(1) val model = kmeans.fit(dataset) @@ -101,9 +102,86 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { expectedColumns.foreach { column => assert(transformed.columns.contains(column)) } - val clusters = transformed.select(predictionColName).map(_.getInt(0)).distinct().collect().toSet + val clusters = + transformed.select(predictionColName).rdd.map(_.getInt(0)).distinct().collect().toSet assert(clusters.size === k) assert(clusters === Set(0, 1, 2, 3, 4)) assert(model.computeCost(dataset) < 0.1) + assert(model.hasParent) + + // Check validity of model summary + val numRows = dataset.count() + assert(model.hasSummary) + val summary: KMeansSummary = model.summary + assert(summary.predictionCol === predictionColName) + assert(summary.featuresCol === "features") + assert(summary.predictions.count() === numRows) + for (c <- Array(predictionColName, "features")) { + assert(summary.predictions.columns.contains(c)) + } + assert(summary.cluster.columns === Array(predictionColName)) + val clusterSizes = summary.clusterSizes + assert(clusterSizes.length === k) + assert(clusterSizes.sum === numRows) + assert(clusterSizes.forall(_ >= 0)) + + model.setSummary(None) + assert(!model.hasSummary) } + + test("KMeansModel transform with non-default feature and prediction cols") { + val featuresColName = "kmeans_model_features" + val predictionColName = "kmeans_model_prediction" + + val model = new KMeans().setK(k).setSeed(1).fit(dataset) + model.setFeaturesCol(featuresColName).setPredictionCol(predictionColName) + + val transformed = model.transform(dataset.withColumnRenamed("features", featuresColName)) + Seq(featuresColName, predictionColName).foreach { column => + assert(transformed.columns.contains(column)) + } + assert(model.getFeaturesCol == featuresColName) + assert(model.getPredictionCol == predictionColName) + } + + test("read/write") { + def checkModelData(model: KMeansModel, model2: KMeansModel): Unit = { + assert(model.clusterCenters === model2.clusterCenters) + } + val kmeans = new KMeans() + testEstimatorAndModelReadWrite(kmeans, dataset, KMeansSuite.allParamSettings, + KMeansSuite.allParamSettings, checkModelData) + } +} + +object KMeansSuite { + def generateKMeansData(spark: SparkSession, rows: Int, dim: Int, k: Int): DataFrame = { + val sc = spark.sparkContext + val rdd = sc.parallelize(1 to rows).map(i => Vectors.dense(Array.fill(dim)((i % k).toDouble))) + .map(v => new TestRow(v)) + spark.createDataFrame(rdd) + } + + def generateSparseData(spark: SparkSession, rows: Int, dim: Int, seed: Int): DataFrame = { + val sc = spark.sparkContext + val random = new Random(seed) + val nnz = random.nextInt(dim) + val rdd = sc.parallelize(1 to rows) + .map(i => Vectors.sparse(dim, random.shuffle(0 to dim - 1).slice(0, nnz).sorted.toArray, + Array.fill(nnz)(random.nextDouble()))) + .map(v => new TestRow(v)) + spark.createDataFrame(rdd) + } + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "predictionCol" -> "myPrediction", + "k" -> 3, + "maxIter" -> 2, + "tol" -> 0.01 + ) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala new file mode 100644 index 000000000000..e73bbc18d76b --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala @@ -0,0 +1,326 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.clustering + +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql._ + + +object LDASuite { + def generateLDAData( + spark: SparkSession, + rows: Int, + k: Int, + vocabSize: Int): DataFrame = { + val avgWC = 1 // average instances of each word in a doc + val sc = spark.sparkContext + val rng = new java.util.Random() + rng.setSeed(1) + val rdd = sc.parallelize(1 to rows).map { i => + Vectors.dense(Array.fill(vocabSize)(rng.nextInt(2 * avgWC).toDouble)) + }.map(v => new TestRow(v)) + spark.createDataFrame(rdd) + } + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "k" -> 3, + "maxIter" -> 2, + "checkpointInterval" -> 30, + "learningOffset" -> 1023.0, + "learningDecay" -> 0.52, + "subsamplingRate" -> 0.051, + "docConcentration" -> Array(2.0) + ) +} + + +class LDASuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + + val k: Int = 5 + val vocabSize: Int = 30 + @transient var dataset: Dataset[_] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + dataset = LDASuite.generateLDAData(spark, 50, k, vocabSize) + } + + test("default parameters") { + val lda = new LDA() + + assert(lda.getFeaturesCol === "features") + assert(lda.getMaxIter === 20) + assert(lda.isDefined(lda.seed)) + assert(lda.getCheckpointInterval === 10) + assert(lda.getK === 10) + assert(!lda.isSet(lda.docConcentration)) + assert(!lda.isSet(lda.topicConcentration)) + assert(lda.getOptimizer === "online") + assert(lda.getLearningDecay === 0.51) + assert(lda.getLearningOffset === 1024) + assert(lda.getSubsamplingRate === 0.05) + assert(lda.getOptimizeDocConcentration) + assert(lda.getTopicDistributionCol === "topicDistribution") + } + + test("set parameters") { + val lda = new LDA() + .setFeaturesCol("test_feature") + .setMaxIter(33) + .setSeed(123) + .setCheckpointInterval(7) + .setK(9) + .setTopicConcentration(0.56) + .setTopicDistributionCol("myOutput") + + assert(lda.getFeaturesCol === "test_feature") + assert(lda.getMaxIter === 33) + assert(lda.getSeed === 123) + assert(lda.getCheckpointInterval === 7) + assert(lda.getK === 9) + assert(lda.getTopicConcentration === 0.56) + assert(lda.getTopicDistributionCol === "myOutput") + + + // setOptimizer + lda.setOptimizer("em") + assert(lda.getOptimizer === "em") + lda.setOptimizer("online") + assert(lda.getOptimizer === "online") + lda.setLearningDecay(0.53) + assert(lda.getLearningDecay === 0.53) + lda.setLearningOffset(1027) + assert(lda.getLearningOffset === 1027) + lda.setSubsamplingRate(0.06) + assert(lda.getSubsamplingRate === 0.06) + lda.setOptimizeDocConcentration(false) + assert(!lda.getOptimizeDocConcentration) + } + + test("parameters validation") { + val lda = new LDA() + + // misc Params + intercept[IllegalArgumentException] { + new LDA().setK(1) + } + intercept[IllegalArgumentException] { + new LDA().setOptimizer("no_such_optimizer") + } + intercept[IllegalArgumentException] { + new LDA().setDocConcentration(-1.1) + } + intercept[IllegalArgumentException] { + new LDA().setTopicConcentration(-1.1) + } + + val dummyDF = Seq((1, Vectors.dense(1.0, 2.0))).toDF("id", "features") + + // validate parameters + lda.transformSchema(dummyDF.schema) + lda.setDocConcentration(1.1) + lda.transformSchema(dummyDF.schema) + lda.setDocConcentration(Range(0, lda.getK).map(_ + 2.0).toArray) + lda.transformSchema(dummyDF.schema) + lda.setDocConcentration(Range(0, lda.getK - 1).map(_ + 2.0).toArray) + withClue("LDA docConcentration validity check failed for bad array length") { + intercept[IllegalArgumentException] { + lda.transformSchema(dummyDF.schema) + } + } + + // Online LDA + intercept[IllegalArgumentException] { + new LDA().setLearningOffset(0) + } + intercept[IllegalArgumentException] { + new LDA().setLearningDecay(0) + } + intercept[IllegalArgumentException] { + new LDA().setSubsamplingRate(0) + } + intercept[IllegalArgumentException] { + new LDA().setSubsamplingRate(1.1) + } + } + + test("fit & transform with Online LDA") { + val lda = new LDA().setK(k).setSeed(1).setOptimizer("online").setMaxIter(2) + val model = lda.fit(dataset) + + MLTestingUtils.checkCopyAndUids(lda, model) + + assert(model.isInstanceOf[LocalLDAModel]) + assert(model.vocabSize === vocabSize) + assert(model.estimatedDocConcentration.size === k) + assert(model.topicsMatrix.numRows === vocabSize) + assert(model.topicsMatrix.numCols === k) + assert(!model.isDistributed) + + // transform() + val transformed = model.transform(dataset) + val expectedColumns = Array("features", lda.getTopicDistributionCol) + expectedColumns.foreach { column => + assert(transformed.columns.contains(column)) + } + transformed.select(lda.getTopicDistributionCol).collect().foreach { r => + val topicDistribution = r.getAs[Vector](0) + assert(topicDistribution.size === k) + assert(topicDistribution.toArray.forall(w => w >= 0.0 && w <= 1.0)) + } + + // logLikelihood, logPerplexity + val ll = model.logLikelihood(dataset) + assert(ll <= 0.0 && ll != Double.NegativeInfinity) + val lp = model.logPerplexity(dataset) + assert(lp >= 0.0 && lp != Double.PositiveInfinity) + + // describeTopics + val topics = model.describeTopics(3) + assert(topics.count() === k) + assert(topics.select("topic").rdd.map(_.getInt(0)).collect().toSet === Range(0, k).toSet) + topics.select("termIndices").collect().foreach { case r: Row => + val termIndices = r.getAs[Seq[Int]](0) + assert(termIndices.length === 3 && termIndices.toSet.size === 3) + } + topics.select("termWeights").collect().foreach { case r: Row => + val termWeights = r.getAs[Seq[Double]](0) + assert(termWeights.length === 3 && termWeights.forall(w => w >= 0.0 && w <= 1.0)) + } + } + + test("fit & transform with EM LDA") { + val lda = new LDA().setK(k).setSeed(1).setOptimizer("em").setMaxIter(2) + val model_ = lda.fit(dataset) + + MLTestingUtils.checkCopyAndUids(lda, model_) + + assert(model_.isInstanceOf[DistributedLDAModel]) + val model = model_.asInstanceOf[DistributedLDAModel] + assert(model.vocabSize === vocabSize) + assert(model.estimatedDocConcentration.size === k) + assert(model.topicsMatrix.numRows === vocabSize) + assert(model.topicsMatrix.numCols === k) + assert(model.isDistributed) + + val localModel = model.toLocal + assert(localModel.isInstanceOf[LocalLDAModel]) + + // training logLikelihood, logPrior + val ll = model.trainingLogLikelihood + assert(ll <= 0.0 && ll != Double.NegativeInfinity) + val lp = model.logPrior + assert(lp <= 0.0 && lp != Double.NegativeInfinity) + } + + test("read/write LocalLDAModel") { + def checkModelData(model: LDAModel, model2: LDAModel): Unit = { + assert(model.vocabSize === model2.vocabSize) + assert(Vectors.dense(model.topicsMatrix.toArray) ~== + Vectors.dense(model2.topicsMatrix.toArray) absTol 1e-6) + assert(Vectors.dense(model.getDocConcentration) ~== + Vectors.dense(model2.getDocConcentration) absTol 1e-6) + } + val lda = new LDA() + testEstimatorAndModelReadWrite(lda, dataset, LDASuite.allParamSettings, + LDASuite.allParamSettings, checkModelData) + } + + test("read/write DistributedLDAModel") { + def checkModelData(model: LDAModel, model2: LDAModel): Unit = { + assert(model.vocabSize === model2.vocabSize) + assert(Vectors.dense(model.topicsMatrix.toArray) ~== + Vectors.dense(model2.topicsMatrix.toArray) absTol 1e-6) + assert(Vectors.dense(model.getDocConcentration) ~== + Vectors.dense(model2.getDocConcentration) absTol 1e-6) + val logPrior = model.asInstanceOf[DistributedLDAModel].logPrior + val logPrior2 = model2.asInstanceOf[DistributedLDAModel].logPrior + val trainingLogLikelihood = + model.asInstanceOf[DistributedLDAModel].trainingLogLikelihood + val trainingLogLikelihood2 = + model2.asInstanceOf[DistributedLDAModel].trainingLogLikelihood + assert(logPrior ~== logPrior2 absTol 1e-6) + assert(trainingLogLikelihood ~== trainingLogLikelihood2 absTol 1e-6) + } + val lda = new LDA() + testEstimatorAndModelReadWrite(lda, dataset, + LDASuite.allParamSettings ++ Map("optimizer" -> "em"), + LDASuite.allParamSettings ++ Map("optimizer" -> "em"), checkModelData) + } + + test("EM LDA checkpointing: save last checkpoint") { + // Checkpoint dir is set by MLlibTestSparkContext + val lda = new LDA().setK(2).setSeed(1).setOptimizer("em").setMaxIter(3).setCheckpointInterval(1) + val model_ = lda.fit(dataset) + assert(model_.isInstanceOf[DistributedLDAModel]) + val model = model_.asInstanceOf[DistributedLDAModel] + + // There should be 1 checkpoint remaining. + assert(model.getCheckpointFiles.length === 1) + val checkpointFile = new Path(model.getCheckpointFiles.head) + val fs = checkpointFile.getFileSystem(spark.sparkContext.hadoopConfiguration) + assert(fs.exists(checkpointFile)) + model.deleteCheckpointFiles() + assert(model.getCheckpointFiles.isEmpty) + } + + test("EM LDA checkpointing: remove last checkpoint") { + // Checkpoint dir is set by MLlibTestSparkContext + val lda = new LDA().setK(2).setSeed(1).setOptimizer("em").setMaxIter(3).setCheckpointInterval(1) + .setKeepLastCheckpoint(false) + val model_ = lda.fit(dataset) + assert(model_.isInstanceOf[DistributedLDAModel]) + val model = model_.asInstanceOf[DistributedLDAModel] + + assert(model.getCheckpointFiles.isEmpty) + } + + test("EM LDA disable checkpointing") { + // Checkpoint dir is set by MLlibTestSparkContext + val lda = new LDA().setK(2).setSeed(1).setOptimizer("em").setMaxIter(3) + .setCheckpointInterval(-1) + val model_ = lda.fit(dataset) + assert(model_.isInstanceOf[DistributedLDAModel]) + val model = model_.asInstanceOf[DistributedLDAModel] + + assert(model.getCheckpointFiles.isEmpty) + } + + test("string params should be case-insensitive") { + val lda = new LDA() + Seq("eM", "oNLinE").foreach { optimizer => + lda.setOptimizer(optimizer) + assert(lda.getOptimizer === optimizer) + val model = lda.fit(dataset) + assert(model.getOptimizer === optimizer) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala index def869fe6677..ede284712b1c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala @@ -18,11 +18,61 @@ package org.apache.spark.ml.evaluation import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.mllib.util.MLlibTestSparkContext -class BinaryClassificationEvaluatorSuite extends SparkFunSuite { +class BinaryClassificationEvaluatorSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ test("params") { ParamsSuite.checkParams(new BinaryClassificationEvaluator) } + + test("read/write") { + val evaluator = new BinaryClassificationEvaluator() + .setRawPredictionCol("myRawPrediction") + .setLabelCol("myLabel") + .setMetricName("areaUnderPR") + testDefaultReadWrite(evaluator) + } + + test("should accept both vector and double raw prediction col") { + val evaluator = new BinaryClassificationEvaluator() + .setMetricName("areaUnderPR") + + val vectorDF = Seq( + (0d, Vectors.dense(12, 2.5)), + (1d, Vectors.dense(1, 3)), + (0d, Vectors.dense(10, 2)) + ).toDF("label", "rawPrediction") + assert(evaluator.evaluate(vectorDF) === 1.0) + + val doubleDF = Seq( + (0d, 0d), + (1d, 1d), + (0d, 0d) + ).toDF("label", "rawPrediction") + assert(evaluator.evaluate(doubleDF) === 1.0) + + val stringDF = Seq( + (0d, "0d"), + (1d, "1d"), + (0d, "0d") + ).toDF("label", "rawPrediction") + val thrown = intercept[IllegalArgumentException] { + evaluator.evaluate(stringDF) + } + assert(thrown.getMessage.replace("\n", "") contains "Column rawPrediction must be of type " + + "equal to one of the following types: [DoubleType, ") + assert(thrown.getMessage.replace("\n", "") contains "but was actually of type StringType.") + } + + test("should support all NumericType labels and not support other types") { + val evaluator = new BinaryClassificationEvaluator().setRawPredictionCol("prediction") + MLTestingUtils.checkNumericTypes(evaluator, spark) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala new file mode 100644 index 000000000000..e60ebbd7c852 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.evaluation + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.types.IntegerType + + +class ClusteringEvaluatorSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + + test("params") { + ParamsSuite.checkParams(new ClusteringEvaluator) + } + + test("read/write") { + val evaluator = new ClusteringEvaluator() + .setPredictionCol("myPrediction") + .setFeaturesCol("myLabel") + testDefaultReadWrite(evaluator) + } + + /* + Use the following python code to load the data and evaluate it using scikit-learn package. + + from sklearn import datasets + from sklearn.metrics import silhouette_score + iris = datasets.load_iris() + round(silhouette_score(iris.data, iris.target, metric='sqeuclidean'), 10) + + 0.6564679231 + */ + test("squared euclidean Silhouette") { + val iris = ClusteringEvaluatorSuite.irisDataset(spark) + val evaluator = new ClusteringEvaluator() + .setFeaturesCol("features") + .setPredictionCol("label") + + assert(evaluator.evaluate(iris) ~== 0.6564679231 relTol 1e-5) + } + + test("number of clusters must be greater than one") { + val iris = ClusteringEvaluatorSuite.irisDataset(spark) + .where($"label" === 0.0) + val evaluator = new ClusteringEvaluator() + .setFeaturesCol("features") + .setPredictionCol("label") + + val e = intercept[AssertionError]{ + evaluator.evaluate(iris) + } + assert(e.getMessage.contains("Number of clusters must be greater than one")) + } + +} + +object ClusteringEvaluatorSuite { + def irisDataset(spark: SparkSession): DataFrame = { + + val irisPath = Thread.currentThread() + .getContextClassLoader + .getResource("test-data/iris.libsvm") + .toString + + spark.read.format("libsvm").load(irisPath) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala index 6d8412b0b370..1a3a8a13a2d0 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala @@ -19,10 +19,25 @@ package org.apache.spark.ml.evaluation import org.apache.spark.SparkFunSuite import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.mllib.util.MLlibTestSparkContext -class MulticlassClassificationEvaluatorSuite extends SparkFunSuite { +class MulticlassClassificationEvaluatorSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { test("params") { ParamsSuite.checkParams(new MulticlassClassificationEvaluator) } + + test("read/write") { + val evaluator = new MulticlassClassificationEvaluator() + .setPredictionCol("myPrediction") + .setLabelCol("myLabel") + .setMetricName("accuracy") + testDefaultReadWrite(evaluator) + } + + test("should support all NumericType labels and not support other types") { + MLTestingUtils.checkNumericTypes(new MulticlassClassificationEvaluator, spark) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala index aa722da32393..c1a156959618 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala @@ -20,10 +20,14 @@ package org.apache.spark.ml.evaluation import org.apache.spark.SparkFunSuite import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.regression.LinearRegression +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext} import org.apache.spark.mllib.util.TestingUtils._ -class RegressionEvaluatorSuite extends SparkFunSuite with MLlibTestSparkContext { +class RegressionEvaluatorSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ test("params") { ParamsSuite.checkParams(new RegressionEvaluator) @@ -40,9 +44,9 @@ class RegressionEvaluatorSuite extends SparkFunSuite with MLlibTestSparkContext * data.map(x=> x.label + ", " + x.features(0) + ", " + x.features(1)) * .saveAsTextFile("path") */ - val dataset = sqlContext.createDataFrame( - sc.parallelize(LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2)) + val dataset = LinearDataGenerator.generateLinearInput( + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1) + .map(_.asML).toDF() /** * Using the following R code to load the data, train the model and evaluate metrics. @@ -63,14 +67,26 @@ class RegressionEvaluatorSuite extends SparkFunSuite with MLlibTestSparkContext // default = rmse val evaluator = new RegressionEvaluator() - assert(evaluator.evaluate(predictions) ~== 0.1019382 absTol 0.001) + assert(evaluator.evaluate(predictions) ~== 0.1013829 absTol 0.01) // r2 score evaluator.setMetricName("r2") - assert(evaluator.evaluate(predictions) ~== 0.9998196 absTol 0.001) + assert(evaluator.evaluate(predictions) ~== 0.9998387 absTol 0.01) // mae evaluator.setMetricName("mae") - assert(evaluator.evaluate(predictions) ~== 0.08036075 absTol 0.001) + assert(evaluator.evaluate(predictions) ~== 0.08399089 absTol 0.01) + } + + test("read/write") { + val evaluator = new RegressionEvaluator() + .setPredictionCol("myPrediction") + .setLabelCol("myLabel") + .setMetricName("r2") + testDefaultReadWrite(evaluator) + } + + test("should support all NumericType labels and not support other types") { + MLTestingUtils.checkNumericTypes(new RegressionEvaluator, spark) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala index 208604398366..4455d3521087 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BinarizerSuite.scala @@ -18,11 +18,15 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Row} -class BinarizerSuite extends SparkFunSuite with MLlibTestSparkContext { +class BinarizerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ @transient var data: Array[Double] = _ @@ -37,8 +41,7 @@ class BinarizerSuite extends SparkFunSuite with MLlibTestSparkContext { test("Binarize continuous features with default parameter") { val defaultBinarized: Array[Double] = data.map(x => if (x > 0.0) 1.0 else 0.0) - val dataFrame: DataFrame = sqlContext.createDataFrame( - data.zip(defaultBinarized)).toDF("feature", "expected") + val dataFrame: DataFrame = data.zip(defaultBinarized).toSeq.toDF("feature", "expected") val binarizer: Binarizer = new Binarizer() .setInputCol("feature") @@ -53,8 +56,7 @@ class BinarizerSuite extends SparkFunSuite with MLlibTestSparkContext { test("Binarize continuous features with setter") { val threshold: Double = 0.2 val thresholdBinarized: Array[Double] = data.map(x => if (x > threshold) 1.0 else 0.0) - val dataFrame: DataFrame = sqlContext.createDataFrame( - data.zip(thresholdBinarized)).toDF("feature", "expected") + val dataFrame: DataFrame = data.zip(thresholdBinarized).toSeq.toDF("feature", "expected") val binarizer: Binarizer = new Binarizer() .setInputCol("feature") @@ -66,4 +68,47 @@ class BinarizerSuite extends SparkFunSuite with MLlibTestSparkContext { assert(x === y, "The feature value is not correct after binarization.") } } + + test("Binarize vector of continuous features with default parameter") { + val defaultBinarized: Array[Double] = data.map(x => if (x > 0.0) 1.0 else 0.0) + val dataFrame: DataFrame = Seq( + (Vectors.dense(data), Vectors.dense(defaultBinarized)) + ).toDF("feature", "expected") + + val binarizer: Binarizer = new Binarizer() + .setInputCol("feature") + .setOutputCol("binarized_feature") + + binarizer.transform(dataFrame).select("binarized_feature", "expected").collect().foreach { + case Row(x: Vector, y: Vector) => + assert(x == y, "The feature value is not correct after binarization.") + } + } + + test("Binarize vector of continuous features with setter") { + val threshold: Double = 0.2 + val defaultBinarized: Array[Double] = data.map(x => if (x > threshold) 1.0 else 0.0) + val dataFrame: DataFrame = Seq( + (Vectors.dense(data), Vectors.dense(defaultBinarized)) + ).toDF("feature", "expected") + + val binarizer: Binarizer = new Binarizer() + .setInputCol("feature") + .setOutputCol("binarized_feature") + .setThreshold(threshold) + + binarizer.transform(dataFrame).select("binarized_feature", "expected").collect().foreach { + case Row(x: Vector, y: Vector) => + assert(x == y, "The feature value is not correct after binarization.") + } + } + + + test("read/write") { + val t = new Binarizer() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setThreshold(0.1) + testDefaultReadWrite(t) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSHSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSHSuite.scala new file mode 100644 index 000000000000..7175c721bff3 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BucketedRandomProjectionLSHSuite.scala @@ -0,0 +1,216 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import breeze.numerics.{cos, sin} +import breeze.numerics.constants.Pi + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.Dataset + +class BucketedRandomProjectionLSHSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + @transient var dataset: Dataset[_] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + val data = { + for (i <- -10 until 10; j <- -10 until 10) yield Vectors.dense(i.toDouble, j.toDouble) + } + dataset = spark.createDataFrame(data.map(Tuple1.apply)).toDF("keys") + } + + test("params") { + ParamsSuite.checkParams(new BucketedRandomProjectionLSH) + val model = new BucketedRandomProjectionLSHModel( + "brp", randUnitVectors = Array(Vectors.dense(1.0, 0.0))) + ParamsSuite.checkParams(model) + } + + test("BucketedRandomProjectionLSH: default params") { + val brp = new BucketedRandomProjectionLSH + assert(brp.getNumHashTables === 1.0) + } + + test("read/write") { + def checkModelData( + model: BucketedRandomProjectionLSHModel, + model2: BucketedRandomProjectionLSHModel): Unit = { + model.randUnitVectors.zip(model2.randUnitVectors) + .foreach(pair => assert(pair._1 === pair._2)) + } + val mh = new BucketedRandomProjectionLSH() + val settings = Map("inputCol" -> "keys", "outputCol" -> "values", "bucketLength" -> 1.0) + testEstimatorAndModelReadWrite(mh, dataset, settings, settings, checkModelData) + } + + test("hashFunction") { + val randUnitVectors = Array(Vectors.dense(0.0, 1.0), Vectors.dense(1.0, 0.0)) + val model = new BucketedRandomProjectionLSHModel("brp", randUnitVectors) + model.set(model.bucketLength, 0.5) + val res = model.hashFunction(Vectors.dense(1.23, 4.56)) + assert(res.length == 2) + assert(res(0).equals(Vectors.dense(9.0))) + assert(res(1).equals(Vectors.dense(2.0))) + } + + test("keyDistance") { + val model = new BucketedRandomProjectionLSHModel("brp", Array(Vectors.dense(0.0, 1.0))) + val keyDist = model.keyDistance(Vectors.dense(1, 2), Vectors.dense(-2, -2)) + assert(keyDist === 5) + } + + test("BucketedRandomProjectionLSH: randUnitVectors") { + val brp = new BucketedRandomProjectionLSH() + .setNumHashTables(20) + .setInputCol("keys") + .setOutputCol("values") + .setBucketLength(1.0) + .setSeed(12345) + val brpModel = brp.fit(dataset) + val unitVectors = brpModel.randUnitVectors + unitVectors.foreach { v: Vector => + assert(Vectors.norm(v, 2.0) ~== 1.0 absTol 1e-14) + } + + MLTestingUtils.checkCopyAndUids(brp, brpModel) + } + + test("BucketedRandomProjectionLSH: test of LSH property") { + // Project from 2 dimensional Euclidean Space to 1 dimensions + val brp = new BucketedRandomProjectionLSH() + .setInputCol("keys") + .setOutputCol("values") + .setBucketLength(1.0) + .setSeed(12345) + + val (falsePositive, falseNegative) = LSHTest.calculateLSHProperty(dataset, brp, 8.0, 2.0) + assert(falsePositive < 0.4) + assert(falseNegative < 0.4) + } + + test("BucketedRandomProjectionLSH with high dimension data: test of LSH property") { + val numDim = 100 + val data = { + for (i <- 0 until numDim; j <- Seq(-2, -1, 1, 2)) + yield Vectors.sparse(numDim, Seq((i, j.toDouble))) + } + val df = spark.createDataFrame(data.map(Tuple1.apply)).toDF("keys") + + // Project from 100 dimensional Euclidean Space to 10 dimensions + val brp = new BucketedRandomProjectionLSH() + .setNumHashTables(10) + .setInputCol("keys") + .setOutputCol("values") + .setBucketLength(2.5) + .setSeed(12345) + + val (falsePositive, falseNegative) = LSHTest.calculateLSHProperty(df, brp, 3.0, 2.0) + assert(falsePositive < 0.3) + assert(falseNegative < 0.3) + } + + test("approxNearestNeighbors for bucketed random projection") { + val key = Vectors.dense(1.2, 3.4) + + val brp = new BucketedRandomProjectionLSH() + .setNumHashTables(2) + .setInputCol("keys") + .setOutputCol("values") + .setBucketLength(4.0) + .setSeed(12345) + + val (precision, recall) = LSHTest.calculateApproxNearestNeighbors(brp, dataset, key, 100, + singleProbe = true) + assert(precision >= 0.6) + assert(recall >= 0.6) + } + + test("approxNearestNeighbors with multiple probing") { + val key = Vectors.dense(1.2, 3.4) + + val brp = new BucketedRandomProjectionLSH() + .setNumHashTables(20) + .setInputCol("keys") + .setOutputCol("values") + .setBucketLength(1.0) + .setSeed(12345) + + val (precision, recall) = LSHTest.calculateApproxNearestNeighbors(brp, dataset, key, 100, + singleProbe = false) + assert(precision >= 0.7) + assert(recall >= 0.7) + } + + test("approxNearestNeighbors for numNeighbors <= 0") { + val key = Vectors.dense(1.2, 3.4) + + val model = new BucketedRandomProjectionLSHModel( + "brp", randUnitVectors = Array(Vectors.dense(1.0, 0.0))) + + intercept[IllegalArgumentException] { + model.approxNearestNeighbors(dataset, key, 0) + } + intercept[IllegalArgumentException] { + model.approxNearestNeighbors(dataset, key, -1) + } + } + + test("approxSimilarityJoin for bucketed random projection on different dataset") { + val data2 = { + for (i <- 0 until 24) yield Vectors.dense(10 * sin(Pi / 12 * i), 10 * cos(Pi / 12 * i)) + } + val dataset2 = spark.createDataFrame(data2.map(Tuple1.apply)).toDF("keys") + + val brp = new BucketedRandomProjectionLSH() + .setNumHashTables(2) + .setInputCol("keys") + .setOutputCol("values") + .setBucketLength(4.0) + .setSeed(12345) + + val (precision, recall) = LSHTest.calculateApproxSimilarityJoin(brp, dataset, dataset2, 1.0) + assert(precision == 1.0) + assert(recall >= 0.7) + } + + test("approxSimilarityJoin for self join") { + val data = { + for (i <- 0 until 24) yield Vectors.dense(10 * sin(Pi / 12 * i), 10 * cos(Pi / 12 * i)) + } + val df = spark.createDataFrame(data.map(Tuple1.apply)).toDF("keys") + + val brp = new BucketedRandomProjectionLSH() + .setNumHashTables(2) + .setInputCol("keys") + .setOutputCol("values") + .setBucketLength(4.0) + .setSeed(12345) + + val (precision, recall) = LSHTest.calculateApproxSimilarityJoin(brp, df, df, 3.0) + assert(precision == 1.0) + assert(recall >= 0.7) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala index 0eba34fda622..420fb17ddce8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/BucketizerSuite.scala @@ -20,14 +20,18 @@ package org.apache.spark.ml.feature import scala.util.Random import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ -class BucketizerSuite extends SparkFunSuite with MLlibTestSparkContext { +class BucketizerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ test("params") { ParamsSuite.checkParams(new Bucketizer) @@ -38,8 +42,7 @@ class BucketizerSuite extends SparkFunSuite with MLlibTestSparkContext { val splits = Array(-0.5, 0.0, 0.5) val validData = Array(-0.5, -0.3, 0.0, 0.2) val expectedBuckets = Array(0.0, 0.0, 1.0, 1.0) - val dataFrame: DataFrame = - sqlContext.createDataFrame(validData.zip(expectedBuckets)).toDF("feature", "expected") + val dataFrame: DataFrame = validData.zip(expectedBuckets).toSeq.toDF("feature", "expected") val bucketizer: Bucketizer = new Bucketizer() .setInputCol("feature") @@ -55,13 +58,13 @@ class BucketizerSuite extends SparkFunSuite with MLlibTestSparkContext { // Check for exceptions when using a set of invalid feature values. val invalidData1: Array[Double] = Array(-0.9) ++ validData val invalidData2 = Array(0.51) ++ validData - val badDF1 = sqlContext.createDataFrame(invalidData1.zipWithIndex).toDF("feature", "idx") + val badDF1 = invalidData1.zipWithIndex.toSeq.toDF("feature", "idx") withClue("Invalid feature value -0.9 was not caught as an invalid feature!") { intercept[SparkException] { bucketizer.transform(badDF1).collect() } } - val badDF2 = sqlContext.createDataFrame(invalidData2.zipWithIndex).toDF("feature", "idx") + val badDF2 = invalidData2.zipWithIndex.toSeq.toDF("feature", "idx") withClue("Invalid feature value 0.51 was not caught as an invalid feature!") { intercept[SparkException] { bucketizer.transform(badDF2).collect() @@ -73,19 +76,59 @@ class BucketizerSuite extends SparkFunSuite with MLlibTestSparkContext { val splits = Array(Double.NegativeInfinity, -0.5, 0.0, 0.5, Double.PositiveInfinity) val validData = Array(-0.9, -0.5, -0.3, 0.0, 0.2, 0.5, 0.9) val expectedBuckets = Array(0.0, 1.0, 1.0, 2.0, 2.0, 3.0, 3.0) - val dataFrame: DataFrame = - sqlContext.createDataFrame(validData.zip(expectedBuckets)).toDF("feature", "expected") + val dataFrame: DataFrame = validData.zip(expectedBuckets).toSeq.toDF("feature", "expected") + + val bucketizer: Bucketizer = new Bucketizer() + .setInputCol("feature") + .setOutputCol("result") + .setSplits(splits) + + bucketizer.transform(dataFrame).select("result", "expected").collect().foreach { + case Row(x: Double, y: Double) => + assert(x === y, + s"The feature value is not correct after bucketing. Expected $y but found $x") + } + } + + test("Bucket continuous features, with NaN data but non-NaN splits") { + val splits = Array(Double.NegativeInfinity, -0.5, 0.0, 0.5, Double.PositiveInfinity) + val validData = Array(-0.9, -0.5, -0.3, 0.0, 0.2, 0.5, 0.9, Double.NaN, Double.NaN, Double.NaN) + val expectedBuckets = Array(0.0, 1.0, 1.0, 2.0, 2.0, 3.0, 3.0, 4.0, 4.0, 4.0) + val dataFrame: DataFrame = validData.zip(expectedBuckets).toSeq.toDF("feature", "expected") val bucketizer: Bucketizer = new Bucketizer() .setInputCol("feature") .setOutputCol("result") .setSplits(splits) + bucketizer.setHandleInvalid("keep") bucketizer.transform(dataFrame).select("result", "expected").collect().foreach { case Row(x: Double, y: Double) => assert(x === y, s"The feature value is not correct after bucketing. Expected $y but found $x") } + + bucketizer.setHandleInvalid("skip") + val skipResults: Array[Double] = bucketizer.transform(dataFrame) + .select("result").as[Double].collect() + assert(skipResults.length === 7) + assert(skipResults.forall(_ !== 4.0)) + + bucketizer.setHandleInvalid("error") + withClue("Bucketizer should throw error when setHandleInvalid=error and given NaN values") { + intercept[SparkException] { + bucketizer.transform(dataFrame).collect() + } + } + } + + test("Bucket continuous features, with NaN splits") { + val splits = Array(Double.NegativeInfinity, -0.5, 0.0, 0.5, Double.PositiveInfinity, Double.NaN) + withClue("Invalid NaN split was not caught during Bucketizer initialization") { + intercept[IllegalArgumentException] { + new Bucketizer().setSplits(splits) + } + } } test("Binary search correctness on hand-picked examples") { @@ -108,10 +151,42 @@ class BucketizerSuite extends SparkFunSuite with MLlibTestSparkContext { val data = Array.fill(100)(Random.nextDouble()) val splits: Array[Double] = Double.NegativeInfinity +: Array.fill(10)(Random.nextDouble()).sorted :+ Double.PositiveInfinity - val bsResult = Vectors.dense(data.map(x => Bucketizer.binarySearchForBuckets(splits, x))) + val bsResult = Vectors.dense(data.map(x => + Bucketizer.binarySearchForBuckets(splits, x, false))) val lsResult = Vectors.dense(data.map(x => BucketizerSuite.linearSearchForBuckets(splits, x))) assert(bsResult ~== lsResult absTol 1e-5) } + + test("read/write") { + val t = new Bucketizer() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setSplits(Array(0.1, 0.8, 0.9)) + testDefaultReadWrite(t) + } + + test("Bucket numeric features") { + val splits = Array(-3.0, 0.0, 3.0) + val data = Array(-2.0, -1.0, 0.0, 1.0, 2.0) + val expectedBuckets = Array(0.0, 0.0, 1.0, 1.0, 1.0) + val dataFrame: DataFrame = data.zip(expectedBuckets).toSeq.toDF("feature", "expected") + + val bucketizer: Bucketizer = new Bucketizer() + .setInputCol("feature") + .setOutputCol("result") + .setSplits(splits) + + val types = Seq(ShortType, IntegerType, LongType, FloatType, DoubleType, + ByteType, DecimalType(10, 0)) + for (mType <- types) { + val df = dataFrame.withColumn("feature", col("feature").cast(mType)) + bucketizer.transform(df).select("result", "expected").collect().foreach { + case Row(x: Double, y: Double) => + assert(x === y, "The result is not correct after bucketing in type " + + mType.toString + ". " + s"Expected $y but found $x.") + } + } + } } private object BucketizerSuite extends SparkFunSuite { @@ -131,7 +206,7 @@ private object BucketizerSuite extends SparkFunSuite { /** Check all values in splits, plus values between all splits. */ def checkBinarySearch(splits: Array[Double]): Unit = { def testFeature(feature: Double, expectedBucket: Double): Unit = { - assert(Bucketizer.binarySearchForBuckets(splits, feature) === expectedBucket, + assert(Bucketizer.binarySearchForBuckets(splits, feature, false) === expectedBucket, s"Expected feature value $feature to be in bucket $expectedBucket with splits:" + s" ${splits.mkString(", ")}") } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala index e5a42967bd2c..c83909c4498f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala @@ -18,44 +18,173 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ -import org.apache.spark.sql.{Row, SQLContext} - -class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext { - test("Test Chi-Square selector") { - val sqlContext = SQLContext.getOrCreate(sc) - import sqlContext.implicits._ - - val data = Seq( - LabeledPoint(0.0, Vectors.sparse(3, Array((0, 8.0), (1, 7.0)))), - LabeledPoint(1.0, Vectors.sparse(3, Array((1, 9.0), (2, 6.0)))), - LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 8.0))), - LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 5.0))) - ) - - val preFilteredData = Seq( - Vectors.dense(0.0), - Vectors.dense(6.0), - Vectors.dense(8.0), - Vectors.dense(5.0) - ) - - val df = sc.parallelize(data.zip(preFilteredData)) - .map(x => (x._1.label, x._1.features, x._2)) - .toDF("label", "data", "preFilteredData") - - val model = new ChiSqSelector() - .setNumTopFeatures(1) - .setFeaturesCol("data") - .setLabelCol("label") - .setOutputCol("filtered") - - model.fit(df).transform(df).select("filtered", "preFilteredData").collect().foreach { - case Row(vec1: Vector, vec2: Vector) => - assert(vec1 ~== vec2 absTol 1e-1) +import org.apache.spark.sql.{Dataset, Row} + +class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext + with DefaultReadWriteTest { + + @transient var dataset: Dataset[_] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + // Toy dataset, including the top feature for a chi-squared test. + // These data are chosen such that each feature's test has a distinct p-value. + /* + * Contingency tables + * feature1 = {6.0, 0.0, 8.0} + * class 0 1 2 + * 6.0||1|0|0| + * 0.0||0|3|0| + * 8.0||0|0|2| + * degree of freedom = 4, statistic = 12, pValue = 0.017 + * + * feature2 = {7.0, 9.0} + * class 0 1 2 + * 7.0||1|0|0| + * 9.0||0|3|2| + * degree of freedom = 2, statistic = 6, pValue = 0.049 + * + * feature3 = {0.0, 6.0, 3.0, 8.0} + * class 0 1 2 + * 0.0||1|0|0| + * 6.0||0|1|2| + * 3.0||0|1|0| + * 8.0||0|1|0| + * degree of freedom = 6, statistic = 8.66, pValue = 0.193 + * + * feature4 = {7.0, 0.0, 5.0, 4.0} + * class 0 1 2 + * 7.0||1|0|0| + * 0.0||0|2|0| + * 5.0||0|1|1| + * 4.0||0|0|1| + * degree of freedom = 6, statistic = 9.5, pValue = 0.147 + * + * feature5 = {6.0, 5.0, 4.0, 0.0} + * class 0 1 2 + * 6.0||1|1|0| + * 5.0||0|2|0| + * 4.0||0|0|1| + * 0.0||0|0|1| + * degree of freedom = 6, statistic = 8.0, pValue = 0.238 + * + * feature6 = {0.0, 9.0, 5.0, 4.0} + * class 0 1 2 + * 0.0||1|0|1| + * 9.0||0|1|0| + * 5.0||0|1|0| + * 4.0||0|1|1| + * degree of freedom = 6, statistic = 5, pValue = 0.54 + * + * To verify the results with R, run: + * library(stats) + * x1 <- c(6.0, 0.0, 0.0, 0.0, 8.0, 8.0) + * x2 <- c(7.0, 9.0, 9.0, 9.0, 9.0, 9.0) + * x3 <- c(0.0, 6.0, 3.0, 8.0, 6.0, 6.0) + * x4 <- c(7.0, 0.0, 0.0, 5.0, 5.0, 4.0) + * x5 <- c(6.0, 5.0, 5.0, 6.0, 4.0, 0.0) + * x6 <- c(0.0, 9.0, 5.0, 4.0, 4.0, 0.0) + * y <- c(0.0, 1.0, 1.0, 1.0, 2.0, 2.0) + * chisq.test(x1,y) + * chisq.test(x2,y) + * chisq.test(x3,y) + * chisq.test(x4,y) + * chisq.test(x5,y) + * chisq.test(x6,y) + */ + + dataset = spark.createDataFrame(Seq( + (0.0, Vectors.sparse(6, Array((0, 6.0), (1, 7.0), (3, 7.0), (4, 6.0))), Vectors.dense(6.0)), + (1.0, Vectors.sparse(6, Array((1, 9.0), (2, 6.0), (4, 5.0), (5, 9.0))), Vectors.dense(0.0)), + (1.0, Vectors.sparse(6, Array((1, 9.0), (2, 3.0), (4, 5.0), (5, 5.0))), Vectors.dense(0.0)), + (1.0, Vectors.dense(Array(0.0, 9.0, 8.0, 5.0, 6.0, 4.0)), Vectors.dense(0.0)), + (2.0, Vectors.dense(Array(8.0, 9.0, 6.0, 5.0, 4.0, 4.0)), Vectors.dense(8.0)), + (2.0, Vectors.dense(Array(8.0, 9.0, 6.0, 4.0, 0.0, 0.0)), Vectors.dense(8.0)) + )).toDF("label", "features", "topFeature") + } + + test("params") { + ParamsSuite.checkParams(new ChiSqSelector) + val model = new ChiSqSelectorModel("myModel", + new org.apache.spark.mllib.feature.ChiSqSelectorModel(Array(1, 3, 4))) + ParamsSuite.checkParams(model) + } + + test("Test Chi-Square selector: numTopFeatures") { + val selector = new ChiSqSelector() + .setOutputCol("filtered").setSelectorType("numTopFeatures").setNumTopFeatures(1) + val model = ChiSqSelectorSuite.testSelector(selector, dataset) + MLTestingUtils.checkCopyAndUids(selector, model) + } + + test("Test Chi-Square selector: percentile") { + val selector = new ChiSqSelector() + .setOutputCol("filtered").setSelectorType("percentile").setPercentile(0.17) + ChiSqSelectorSuite.testSelector(selector, dataset) + } + + test("Test Chi-Square selector: fpr") { + val selector = new ChiSqSelector() + .setOutputCol("filtered").setSelectorType("fpr").setFpr(0.02) + ChiSqSelectorSuite.testSelector(selector, dataset) + } + + test("Test Chi-Square selector: fdr") { + val selector = new ChiSqSelector() + .setOutputCol("filtered").setSelectorType("fdr").setFdr(0.12) + ChiSqSelectorSuite.testSelector(selector, dataset) + } + + test("Test Chi-Square selector: fwe") { + val selector = new ChiSqSelector() + .setOutputCol("filtered").setSelectorType("fwe").setFwe(0.12) + ChiSqSelectorSuite.testSelector(selector, dataset) + } + + test("read/write") { + def checkModelData(model: ChiSqSelectorModel, model2: ChiSqSelectorModel): Unit = { + assert(model.selectedFeatures === model2.selectedFeatures) } + val nb = new ChiSqSelector + testEstimatorAndModelReadWrite(nb, dataset, ChiSqSelectorSuite.allParamSettings, + ChiSqSelectorSuite.allParamSettings, checkModelData) + } + + test("should support all NumericType labels and not support other types") { + val css = new ChiSqSelector() + MLTestingUtils.checkNumericTypes[ChiSqSelectorModel, ChiSqSelector]( + css, spark) { (expected, actual) => + assert(expected.selectedFeatures === actual.selectedFeatures) + } } } + +object ChiSqSelectorSuite { + + private def testSelector(selector: ChiSqSelector, dataset: Dataset[_]): ChiSqSelectorModel = { + val selectorModel = selector.fit(dataset) + selectorModel.transform(dataset).select("filtered", "topFeature").collect() + .foreach { case Row(vec1: Vector, vec2: Vector) => + assert(vec1 ~== vec2 absTol 1e-1) + } + selectorModel + } + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "selectorType" -> "percentile", + "numTopFeatures" -> 1, + "percentile" -> 0.12, + "outputCol" -> "myOutput" + ) +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala index e192fa4850af..f213145f1ba0 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizerSuite.scala @@ -17,22 +17,27 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row -class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext { +class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext + with DefaultReadWriteTest { + + import testImplicits._ test("params") { + ParamsSuite.checkParams(new CountVectorizer) ParamsSuite.checkParams(new CountVectorizerModel(Array("empty"))) } private def split(s: String): Seq[String] = s.split("\\s+") test("CountVectorizerModel common cases") { - val df = sqlContext.createDataFrame(Seq( + val df = Seq( (0, split("a b c d"), Vectors.sparse(4, Seq((0, 1.0), (1, 1.0), (2, 1.0), (3, 1.0)))), (1, split("a b b c d a"), @@ -41,7 +46,7 @@ class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext { (3, split(""), Vectors.sparse(4, Seq())), // empty string (4, split("a notInDict d"), Vectors.sparse(4, Seq((0, 1.0), (3, 1.0)))) // with words not in vocabulary - )).toDF("id", "words", "expected") + ).toDF("id", "words", "expected") val cv = new CountVectorizerModel(Array("a", "b", "c", "d")) .setInputCol("words") .setOutputCol("features") @@ -52,31 +57,33 @@ class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext { } test("CountVectorizer common cases") { - val df = sqlContext.createDataFrame(Seq( + val df = Seq( (0, split("a b c d e"), Vectors.sparse(5, Seq((0, 1.0), (1, 1.0), (2, 1.0), (3, 1.0), (4, 1.0)))), (1, split("a a a a a a"), Vectors.sparse(5, Seq((0, 6.0)))), - (2, split("c"), Vectors.sparse(5, Seq((2, 1.0)))), - (3, split("b b b b b"), Vectors.sparse(5, Seq((1, 5.0))))) + (2, split("c c"), Vectors.sparse(5, Seq((2, 2.0)))), + (3, split("d"), Vectors.sparse(5, Seq((3, 1.0)))), + (4, split("b b b b b"), Vectors.sparse(5, Seq((1, 5.0)))) ).toDF("id", "words", "expected") val cv = new CountVectorizer() .setInputCol("words") .setOutputCol("features") - .fit(df) - assert(cv.vocabulary === Array("a", "b", "c", "d", "e")) + val cvm = cv.fit(df) + MLTestingUtils.checkCopyAndUids(cv, cvm) + assert(cvm.vocabulary.toSet === Set("a", "b", "c", "d", "e")) - cv.transform(df).select("features", "expected").collect().foreach { + cvm.transform(df).select("features", "expected").collect().foreach { case Row(features: Vector, expected: Vector) => assert(features ~== expected absTol 1e-14) } } test("CountVectorizer vocabSize and minDF") { - val df = sqlContext.createDataFrame(Seq( - (0, split("a b c d"), Vectors.sparse(3, Seq((0, 1.0), (1, 1.0)))), - (1, split("a b c"), Vectors.sparse(3, Seq((0, 1.0), (1, 1.0)))), - (2, split("a b"), Vectors.sparse(3, Seq((0, 1.0), (1, 1.0)))), - (3, split("a"), Vectors.sparse(3, Seq((0, 1.0))))) + val df = Seq( + (0, split("a b c d"), Vectors.sparse(2, Seq((0, 1.0), (1, 1.0)))), + (1, split("a b c"), Vectors.sparse(2, Seq((0, 1.0), (1, 1.0)))), + (2, split("a b"), Vectors.sparse(2, Seq((0, 1.0), (1, 1.0)))), + (3, split("a"), Vectors.sparse(2, Seq((0, 1.0)))) ).toDF("id", "words", "expected") val cvModel = new CountVectorizer() .setInputCol("words") @@ -114,9 +121,9 @@ class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext { test("CountVectorizer throws exception when vocab is empty") { intercept[IllegalArgumentException] { - val df = sqlContext.createDataFrame(Seq( + val df = Seq( (0, split("a a b b c c")), - (1, split("aa bb cc"))) + (1, split("aa bb cc")) ).toDF("id", "words") val cvModel = new CountVectorizer() .setInputCol("words") @@ -128,11 +135,11 @@ class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext { } test("CountVectorizerModel with minTF count") { - val df = sqlContext.createDataFrame(Seq( + val df = Seq( (0, split("a a a b b c c c d "), Vectors.sparse(4, Seq((0, 3.0), (2, 3.0)))), (1, split("c c c c c c"), Vectors.sparse(4, Seq((2, 6.0)))), (2, split("a"), Vectors.sparse(4, Seq())), - (3, split("e e e e e"), Vectors.sparse(4, Seq()))) + (3, split("e e e e e"), Vectors.sparse(4, Seq())) ).toDF("id", "words", "expected") // minTF: count @@ -147,14 +154,14 @@ class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext { } test("CountVectorizerModel with minTF freq") { - val df = sqlContext.createDataFrame(Seq( + val df = Seq( (0, split("a a a b b c c c d "), Vectors.sparse(4, Seq((0, 3.0), (2, 3.0)))), (1, split("c c c c c c"), Vectors.sparse(4, Seq((2, 6.0)))), (2, split("a"), Vectors.sparse(4, Seq((0, 1.0)))), - (3, split("e e e e e"), Vectors.sparse(4, Seq()))) + (3, split("e e e e e"), Vectors.sparse(4, Seq())) ).toDF("id", "words", "expected") - // minTF: count + // minTF: set frequency val cv = new CountVectorizerModel(Array("a", "b", "c", "d")) .setInputCol("words") .setOutputCol("features") @@ -164,4 +171,53 @@ class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext { assert(features ~== expected absTol 1e-14) } } + + test("CountVectorizerModel and CountVectorizer with binary") { + val df = Seq( + (0, split("a a a a b b b b c d"), + Vectors.sparse(4, Seq((0, 1.0), (1, 1.0), (2, 1.0), (3, 1.0)))), + (1, split("c c c"), Vectors.sparse(4, Seq((2, 1.0)))), + (2, split("a"), Vectors.sparse(4, Seq((0, 1.0)))) + ).toDF("id", "words", "expected") + + // CountVectorizer test + val cv = new CountVectorizer() + .setInputCol("words") + .setOutputCol("features") + .setBinary(true) + .fit(df) + cv.transform(df).select("features", "expected").collect().foreach { + case Row(features: Vector, expected: Vector) => + assert(features ~== expected absTol 1e-14) + } + + // CountVectorizerModel test + val cv2 = new CountVectorizerModel(cv.vocabulary) + .setInputCol("words") + .setOutputCol("features") + .setBinary(true) + cv2.transform(df).select("features", "expected").collect().foreach { + case Row(features: Vector, expected: Vector) => + assert(features ~== expected absTol 1e-14) + } + } + + test("CountVectorizer read/write") { + val t = new CountVectorizer() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setMinDF(0.5) + .setMinTF(3.0) + .setVocabSize(10) + testDefaultReadWrite(t) + } + + test("CountVectorizerModel read/write") { + val instance = new CountVectorizerModel("myCountVectorizerModel", Array("a", "b", "c")) + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setMinTF(3.0) + val newInstance = testDefaultReadWrite(instance) + assert(newInstance.vocabulary === instance.vocabulary) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala index 37ed2367c33f..8dd3dd75e1be 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala @@ -22,14 +22,17 @@ import scala.beans.BeanInfo import edu.emory.mathcs.jtransforms.dct.DoubleDCT_1D import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.Row @BeanInfo case class DCTTestData(vec: Vector, wantedVec: Vector) -class DCTSuite extends SparkFunSuite with MLlibTestSparkContext { +class DCTSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ test("forward transform of discrete cosine matches jTransforms result") { val data = Vectors.dense((0 until 128).map(_ => 2D * math.random - 1D).toArray) @@ -45,18 +48,24 @@ class DCTSuite extends SparkFunSuite with MLlibTestSparkContext { testDCT(data, inverse) } + test("read/write") { + val t = new DCT() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setInverse(true) + testDefaultReadWrite(t) + } + private def testDCT(data: Vector, inverse: Boolean): Unit = { val expectedResultBuffer = data.toArray.clone() if (inverse) { - (new DoubleDCT_1D(data.size)).inverse(expectedResultBuffer, true) + new DoubleDCT_1D(data.size).inverse(expectedResultBuffer, true) } else { - (new DoubleDCT_1D(data.size)).forward(expectedResultBuffer, true) + new DoubleDCT_1D(data.size).forward(expectedResultBuffer, true) } val expectedResult = Vectors.dense(expectedResultBuffer) - val dataset = sqlContext.createDataFrame(Seq( - DCTTestData(data, expectedResult) - )) + val dataset = Seq(DCTTestData(data, expectedResult)).toDF() val transformer = new DCT() .setInputCol("vec") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala new file mode 100644 index 000000000000..a4cca27be781 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ElementwiseProductSuite.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.mllib.util.MLlibTestSparkContext + +class ElementwiseProductSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + test("read/write") { + val ep = new ElementwiseProduct() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setScalingVec(Vectors.dense(0.1, 0.2)) + testDefaultReadWrite(ep) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/FeatureHasherSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/FeatureHasherSuite.scala new file mode 100644 index 000000000000..407371a82666 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/FeatureHasherSuite.scala @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types._ + +class FeatureHasherSuite extends SparkFunSuite + with MLlibTestSparkContext + with DefaultReadWriteTest { + + import testImplicits._ + + import HashingTFSuite.murmur3FeatureIdx + + implicit private val vectorEncoder = ExpressionEncoder[Vector]() + + test("params") { + ParamsSuite.checkParams(new FeatureHasher) + } + + test("specify input cols using varargs or array") { + val featureHasher1 = new FeatureHasher() + .setInputCols("int", "double", "float", "stringNum", "string") + val featureHasher2 = new FeatureHasher() + .setInputCols(Array("int", "double", "float", "stringNum", "string")) + assert(featureHasher1.getInputCols === featureHasher2.getInputCols) + } + + test("feature hashing") { + val df = Seq( + (2.0, true, "1", "foo"), + (3.0, false, "2", "bar") + ).toDF("real", "bool", "stringNum", "string") + + val n = 100 + val hasher = new FeatureHasher() + .setInputCols("real", "bool", "stringNum", "string") + .setOutputCol("features") + .setNumFeatures(n) + val output = hasher.transform(df) + val attrGroup = AttributeGroup.fromStructField(output.schema("features")) + assert(attrGroup.numAttributes === Some(n)) + + val features = output.select("features").as[Vector].collect() + // Assume perfect hash on field names + def idx: Any => Int = murmur3FeatureIdx(n) + // check expected indices + val expected = Seq( + Vectors.sparse(n, Seq((idx("real"), 2.0), (idx("bool=true"), 1.0), + (idx("stringNum=1"), 1.0), (idx("string=foo"), 1.0))), + Vectors.sparse(n, Seq((idx("real"), 3.0), (idx("bool=false"), 1.0), + (idx("stringNum=2"), 1.0), (idx("string=bar"), 1.0))) + ) + assert(features.zip(expected).forall { case (e, a) => e ~== a absTol 1e-14 }) + } + + test("hashing works for all numeric types") { + val df = Seq(5.0, 10.0, 15.0).toDF("real") + + val hasher = new FeatureHasher() + .setInputCols("real") + .setOutputCol("features") + + val expectedResult = hasher.transform(df).select("features").as[Vector].collect() + // check all numeric types work as expected. String & boolean types are tested in default case + val types = + Seq(ShortType, LongType, IntegerType, FloatType, ByteType, DoubleType, DecimalType(10, 0)) + types.foreach { t => + val castDF = df.select(col("real").cast(t)) + val castResult = hasher.transform(castDF).select("features").as[Vector].collect() + withClue(s"FeatureHasher works for all numeric types (testing $t): ") { + assert(castResult.zip(expectedResult).forall { case (actual, expected) => + actual ~== expected absTol 1e-14 + }) + } + } + } + + test("invalid input type should fail") { + val df = Seq( + Vectors.dense(1), + Vectors.dense(2) + ).toDF("vec") + + intercept[IllegalArgumentException] { + new FeatureHasher().setInputCols("vec").transform(df) + } + } + + test("hash collisions sum feature values") { + val df = Seq( + (1.0, "foo", "foo"), + (2.0, "bar", "baz") + ).toDF("real", "string1", "string2") + + val n = 1 + val hasher = new FeatureHasher() + .setInputCols("real", "string1", "string2") + .setOutputCol("features") + .setNumFeatures(n) + + val features = hasher.transform(df).select("features").as[Vector].collect() + def idx: Any => Int = murmur3FeatureIdx(n) + // everything should hash into one field + assert(idx("real") === idx("string1=foo")) + assert(idx("string1=foo") === idx("string2=foo")) + assert(idx("string2=foo") === idx("string1=bar")) + assert(idx("string1=bar") === idx("string2=baz")) + val expected = Seq( + Vectors.sparse(n, Seq((idx("string1=foo"), 3.0))), + Vectors.sparse(n, Seq((idx("string2=bar"), 4.0))) + ) + assert(features.zip(expected).forall { case (e, a) => e ~== a absTol 1e-14 }) + } + + test("ignores null values in feature hashing") { + import org.apache.spark.sql.functions._ + + val df = Seq( + (2.0, "foo", null), + (3.0, "bar", "baz") + ).toDF("real", "string1", "string2").select( + when(col("real") === 3.0, null).otherwise(col("real")).alias("real"), + col("string1"), + col("string2") + ) + + val n = 100 + val hasher = new FeatureHasher() + .setInputCols("real", "string1", "string2") + .setOutputCol("features") + .setNumFeatures(n) + + val features = hasher.transform(df).select("features").as[Vector].collect() + def idx: Any => Int = murmur3FeatureIdx(n) + val expected = Seq( + Vectors.sparse(n, Seq((idx("real"), 2.0), (idx("string1=foo"), 1.0))), + Vectors.sparse(n, Seq((idx("string1=bar"), 1.0), (idx("string2=baz"), 1.0))) + ) + assert(features.zip(expected).forall { case (e, a) => e ~== a absTol 1e-14 }) + } + + test("unicode column names and values") { + // scalastyle:off nonascii + val df = Seq((2.0, "中文")).toDF("中文", "unicode") + + val n = 100 + val hasher = new FeatureHasher() + .setInputCols("中文", "unicode") + .setOutputCol("features") + .setNumFeatures(n) + + val features = hasher.transform(df).select("features").as[Vector].collect() + def idx: Any => Int = murmur3FeatureIdx(n) + val expected = Seq( + Vectors.sparse(n, Seq((idx("中文"), 2.0), (idx("unicode=中文"), 1.0))) + ) + assert(features.zip(expected).forall { case (e, a) => e ~== a absTol 1e-14 }) + // scalastyle:on nonascii + } + + test("read/write") { + val t = new FeatureHasher() + .setInputCols(Array("myCol1", "myCol2", "myCol3")) + .setOutputCol("myOutputCol") + .setNumFeatures(10) + testDefaultReadWrite(t) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala index 4157b84b29d0..a46272fdce1f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/HashingTFSuite.scala @@ -19,22 +19,25 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.feature.{HashingTF => MLlibHashingTF} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.Utils -class HashingTFSuite extends SparkFunSuite with MLlibTestSparkContext { +class HashingTFSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + import HashingTFSuite.murmur3FeatureIdx test("params") { ParamsSuite.checkParams(new HashingTF) } test("hashingTF") { - val df = sqlContext.createDataFrame(Seq( - (0, "a a b b c d".split(" ").toSeq) - )).toDF("id", "words") + val df = Seq((0, "a a b b c d".split(" ").toSeq)).toDF("id", "words") val n = 100 val hashingTF = new HashingTF() .setInputCol("words") @@ -45,9 +48,42 @@ class HashingTFSuite extends SparkFunSuite with MLlibTestSparkContext { require(attrGroup.numAttributes === Some(n)) val features = output.select("features").first().getAs[Vector](0) // Assume perfect hash on "a", "b", "c", and "d". - def idx(any: Any): Int = Utils.nonNegativeMod(any.##, n) + def idx: Any => Int = murmur3FeatureIdx(n) val expected = Vectors.sparse(n, Seq((idx("a"), 2.0), (idx("b"), 2.0), (idx("c"), 1.0), (idx("d"), 1.0))) assert(features ~== expected absTol 1e-14) } + + test("applying binary term freqs") { + val df = Seq((0, "a a b c c c".split(" ").toSeq)).toDF("id", "words") + val n = 100 + val hashingTF = new HashingTF() + .setInputCol("words") + .setOutputCol("features") + .setNumFeatures(n) + .setBinary(true) + val output = hashingTF.transform(df) + val features = output.select("features").first().getAs[Vector](0) + def idx: Any => Int = murmur3FeatureIdx(n) // Assume perfect hash on input features + val expected = Vectors.sparse(n, + Seq((idx("a"), 1.0), (idx("b"), 1.0), (idx("c"), 1.0))) + assert(features ~== expected absTol 1e-14) + } + + test("read/write") { + val t = new HashingTF() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setNumFeatures(10) + testDefaultReadWrite(t) + } + +} + +object HashingTFSuite { + + private[feature] def murmur3FeatureIdx(numFeatures: Int)(term: Any): Int = { + Utils.nonNegativeMod(MLlibHashingTF.murmur3Hash(term), numFeatures) + } + } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala index 08f80af03429..005edf73d29b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala @@ -18,14 +18,18 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature.{IDFModel => OldIDFModel} -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row -class IDFSuite extends SparkFunSuite with MLlibTestSparkContext { +class IDFSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ def scaleDataWithIDF(dataSet: Array[Vector], model: Vector): Array[Vector] = { dataSet.map { @@ -59,12 +63,14 @@ class IDFSuite extends SparkFunSuite with MLlibTestSparkContext { }) val expected = scaleDataWithIDF(data, idf) - val df = sqlContext.createDataFrame(data.zip(expected)).toDF("features", "expected") + val df = data.zip(expected).toSeq.toDF("features", "expected") - val idfModel = new IDF() + val idfEst = new IDF() .setInputCol("features") .setOutputCol("idfValue") - .fit(df) + val idfModel = idfEst.fit(df) + + MLTestingUtils.checkCopyAndUids(idfEst, idfModel) idfModel.transform(df).select("idfValue", "expected").collect().foreach { case Row(x: Vector, y: Vector) => @@ -85,7 +91,7 @@ class IDFSuite extends SparkFunSuite with MLlibTestSparkContext { }) val expected = scaleDataWithIDF(data, idf) - val df = sqlContext.createDataFrame(data.zip(expected)).toDF("features", "expected") + val df = data.zip(expected).toSeq.toDF("features", "expected") val idfModel = new IDF() .setInputCol("features") @@ -98,4 +104,20 @@ class IDFSuite extends SparkFunSuite with MLlibTestSparkContext { assert(x ~== y absTol 1e-5, "Transformed vector is different with expected vector.") } } + + test("IDF read/write") { + val t = new IDF() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setMinDocFreq(5) + testDefaultReadWrite(t) + } + + test("IDFModel read/write") { + val instance = new IDFModel("myIDFModel", new OldIDFModel(Vectors.dense(1.0, 2.0))) + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + val newInstance = testDefaultReadWrite(instance) + assert(newInstance.idf === instance.idf) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala new file mode 100644 index 000000000000..ee2ba73fa96d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ImputerSuite.scala @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.ml.feature + +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.sql.{DataFrame, Row} + +class ImputerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + test("Imputer for Double with default missing Value NaN") { + val df = spark.createDataFrame( Seq( + (0, 1.0, 4.0, 1.0, 1.0, 4.0, 4.0), + (1, 11.0, 12.0, 11.0, 11.0, 12.0, 12.0), + (2, 3.0, Double.NaN, 3.0, 3.0, 10.0, 12.0), + (3, Double.NaN, 14.0, 5.0, 3.0, 14.0, 14.0) + )).toDF("id", "value1", "value2", "expected_mean_value1", "expected_median_value1", + "expected_mean_value2", "expected_median_value2") + val imputer = new Imputer() + .setInputCols(Array("value1", "value2")) + .setOutputCols(Array("out1", "out2")) + ImputerSuite.iterateStrategyTest(imputer, df) + } + + test("Imputer should handle NaNs when computing surrogate value, if missingValue is not NaN") { + val df = spark.createDataFrame( Seq( + (0, 1.0, 1.0, 1.0), + (1, 3.0, 3.0, 3.0), + (2, Double.NaN, Double.NaN, Double.NaN), + (3, -1.0, 2.0, 3.0) + )).toDF("id", "value", "expected_mean_value", "expected_median_value") + val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out")) + .setMissingValue(-1.0) + ImputerSuite.iterateStrategyTest(imputer, df) + } + + test("Imputer for Float with missing Value -1.0") { + val df = spark.createDataFrame( Seq( + (0, 1.0F, 1.0F, 1.0F), + (1, 3.0F, 3.0F, 3.0F), + (2, 10.0F, 10.0F, 10.0F), + (3, 10.0F, 10.0F, 10.0F), + (4, -1.0F, 6.0F, 3.0F) + )).toDF("id", "value", "expected_mean_value", "expected_median_value") + val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out")) + .setMissingValue(-1) + ImputerSuite.iterateStrategyTest(imputer, df) + } + + test("Imputer should impute null as well as 'missingValue'") { + val rawDf = spark.createDataFrame( Seq( + (0, 4.0, 4.0, 4.0), + (1, 10.0, 10.0, 10.0), + (2, 10.0, 10.0, 10.0), + (3, Double.NaN, 8.0, 10.0), + (4, -1.0, 8.0, 10.0) + )).toDF("id", "rawValue", "expected_mean_value", "expected_median_value") + val df = rawDf.selectExpr("*", "IF(rawValue=-1.0, null, rawValue) as value") + val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out")) + ImputerSuite.iterateStrategyTest(imputer, df) + } + + test("Imputer throws exception when surrogate cannot be computed") { + val df = spark.createDataFrame( Seq( + (0, Double.NaN, 1.0, 1.0), + (1, Double.NaN, 3.0, 3.0), + (2, Double.NaN, Double.NaN, Double.NaN) + )).toDF("id", "value", "expected_mean_value", "expected_median_value") + Seq("mean", "median").foreach { strategy => + val imputer = new Imputer().setInputCols(Array("value")).setOutputCols(Array("out")) + .setStrategy(strategy) + withClue("Imputer should fail all the values are invalid") { + val e: SparkException = intercept[SparkException] { + val model = imputer.fit(df) + } + assert(e.getMessage.contains("surrogate cannot be computed")) + } + } + } + + test("Imputer input & output column validation") { + val df = spark.createDataFrame( Seq( + (0, 1.0, 1.0, 1.0), + (1, Double.NaN, 3.0, 3.0), + (2, Double.NaN, Double.NaN, Double.NaN) + )).toDF("id", "value1", "value2", "value3") + Seq("mean", "median").foreach { strategy => + withClue("Imputer should fail if inputCols and outputCols are different length") { + val e: IllegalArgumentException = intercept[IllegalArgumentException] { + val imputer = new Imputer().setStrategy(strategy) + .setInputCols(Array("value1", "value2")) + .setOutputCols(Array("out1")) + val model = imputer.fit(df) + } + assert(e.getMessage.contains("should have the same length")) + } + + withClue("Imputer should fail if inputCols contains duplicates") { + val e: IllegalArgumentException = intercept[IllegalArgumentException] { + val imputer = new Imputer().setStrategy(strategy) + .setInputCols(Array("value1", "value1")) + .setOutputCols(Array("out1", "out2")) + val model = imputer.fit(df) + } + assert(e.getMessage.contains("inputCols contains duplicates")) + } + + withClue("Imputer should fail if outputCols contains duplicates") { + val e: IllegalArgumentException = intercept[IllegalArgumentException] { + val imputer = new Imputer().setStrategy(strategy) + .setInputCols(Array("value1", "value2")) + .setOutputCols(Array("out1", "out1")) + val model = imputer.fit(df) + } + assert(e.getMessage.contains("outputCols contains duplicates")) + } + } + } + + test("Imputer read/write") { + val t = new Imputer() + .setInputCols(Array("myInputCol")) + .setOutputCols(Array("myOutputCol")) + .setMissingValue(-1.0) + testDefaultReadWrite(t) + } + + test("ImputerModel read/write") { + val spark = this.spark + import spark.implicits._ + val surrogateDF = Seq(1.234).toDF("myInputCol") + + val instance = new ImputerModel( + "myImputer", surrogateDF) + .setInputCols(Array("myInputCol")) + .setOutputCols(Array("myOutputCol")) + val newInstance = testDefaultReadWrite(instance) + assert(newInstance.surrogateDF.columns === instance.surrogateDF.columns) + assert(newInstance.surrogateDF.collect() === instance.surrogateDF.collect()) + } + +} + +object ImputerSuite { + + /** + * Imputation strategy. Available options are ["mean", "median"]. + * @param df DataFrame with columns "id", "value", "expected_mean", "expected_median" + */ + def iterateStrategyTest(imputer: Imputer, df: DataFrame): Unit = { + val inputCols = imputer.getInputCols + + Seq("mean", "median").foreach { strategy => + imputer.setStrategy(strategy) + val model = imputer.fit(df) + val resultDF = model.transform(df) + imputer.getInputCols.zip(imputer.getOutputCols).foreach { case (inputCol, outputCol) => + resultDF.select(s"expected_${strategy}_$inputCol", outputCol).collect().foreach { + case Row(exp: Float, out: Float) => + assert((exp.isNaN && out.isNaN) || (exp == out), + s"Imputed values differ. Expected: $exp, actual: $out") + case Row(exp: Double, out: Double) => + assert((exp.isNaN && out.isNaN) || (exp ~== out absTol 1e-5), + s"Imputed values differ. Expected: $exp, actual: $out") + } + } + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala index 2beb62ca0823..54f059e5f143 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/InteractionSuite.scala @@ -21,12 +21,16 @@ import scala.collection.mutable.ArrayBuilder import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.functions.col -class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext { +class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + test("params") { ParamsSuite.checkParams(new Interaction()) } @@ -58,11 +62,10 @@ class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext { } test("numeric interaction") { - val data = sqlContext.createDataFrame( - Seq( - (2, Vectors.dense(3.0, 4.0)), - (1, Vectors.dense(1.0, 5.0))) - ).toDF("a", "b") + val data = Seq( + (2, Vectors.dense(3.0, 4.0)), + (1, Vectors.dense(1.0, 5.0)) + ).toDF("a", "b") val groupAttr = new AttributeGroup( "b", Array[Attribute]( @@ -73,11 +76,10 @@ class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext { col("b").as("b", groupAttr.toMetadata())) val trans = new Interaction().setInputCols(Array("a", "b")).setOutputCol("features") val res = trans.transform(df) - val expected = sqlContext.createDataFrame( - Seq( - (2, Vectors.dense(3.0, 4.0), Vectors.dense(6.0, 8.0)), - (1, Vectors.dense(1.0, 5.0), Vectors.dense(1.0, 5.0))) - ).toDF("a", "b", "features") + val expected = Seq( + (2, Vectors.dense(3.0, 4.0), Vectors.dense(6.0, 8.0)), + (1, Vectors.dense(1.0, 5.0), Vectors.dense(1.0, 5.0)) + ).toDF("a", "b", "features") assert(res.collect() === expected.collect()) val attrs = AttributeGroup.fromStructField(res.schema("features")) val expectedAttrs = new AttributeGroup( @@ -89,11 +91,10 @@ class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext { } test("nominal interaction") { - val data = sqlContext.createDataFrame( - Seq( - (2, Vectors.dense(3.0, 4.0)), - (1, Vectors.dense(1.0, 5.0))) - ).toDF("a", "b") + val data = Seq( + (2, Vectors.dense(3.0, 4.0)), + (1, Vectors.dense(1.0, 5.0)) + ).toDF("a", "b") val groupAttr = new AttributeGroup( "b", Array[Attribute]( @@ -105,11 +106,10 @@ class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext { col("b").as("b", groupAttr.toMetadata())) val trans = new Interaction().setInputCols(Array("a", "b")).setOutputCol("features") val res = trans.transform(df) - val expected = sqlContext.createDataFrame( - Seq( - (2, Vectors.dense(3.0, 4.0), Vectors.dense(0, 0, 0, 0, 3, 4)), - (1, Vectors.dense(1.0, 5.0), Vectors.dense(0, 0, 1, 5, 0, 0))) - ).toDF("a", "b", "features") + val expected = Seq( + (2, Vectors.dense(3.0, 4.0), Vectors.dense(0, 0, 0, 0, 3, 4)), + (1, Vectors.dense(1.0, 5.0), Vectors.dense(0, 0, 1, 5, 0, 0)) + ).toDF("a", "b", "features") assert(res.collect() === expected.collect()) val attrs = AttributeGroup.fromStructField(res.schema("features")) val expectedAttrs = new AttributeGroup( @@ -125,10 +125,9 @@ class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext { } test("default attr names") { - val data = sqlContext.createDataFrame( - Seq( + val data = Seq( (2, Vectors.dense(0.0, 4.0), 1.0), - (1, Vectors.dense(1.0, 5.0), 10.0)) + (1, Vectors.dense(1.0, 5.0), 10.0) ).toDF("a", "b", "c") val groupAttr = new AttributeGroup( "b", @@ -141,11 +140,10 @@ class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext { col("c").as("c", NumericAttribute.defaultAttr.toMetadata())) val trans = new Interaction().setInputCols(Array("a", "b", "c")).setOutputCol("features") val res = trans.transform(df) - val expected = sqlContext.createDataFrame( - Seq( - (2, Vectors.dense(0.0, 4.0), 1.0, Vectors.dense(0, 0, 0, 0, 0, 0, 1, 0, 4)), - (1, Vectors.dense(1.0, 5.0), 10.0, Vectors.dense(0, 0, 0, 0, 10, 50, 0, 0, 0))) - ).toDF("a", "b", "c", "features") + val expected = Seq( + (2, Vectors.dense(0.0, 4.0), 1.0, Vectors.dense(0, 0, 0, 0, 0, 0, 1, 0, 4)), + (1, Vectors.dense(1.0, 5.0), 10.0, Vectors.dense(0, 0, 0, 0, 10, 50, 0, 0, 0)) + ).toDF("a", "b", "c", "features") assert(res.collect() === expected.collect()) val attrs = AttributeGroup.fromStructField(res.schema("features")) val expectedAttrs = new AttributeGroup( @@ -162,4 +160,11 @@ class InteractionSuite extends SparkFunSuite with MLlibTestSparkContext { new NumericAttribute(Some("a_2:b_1:c"), Some(9)))) assert(attrs === expectedAttrs) } + + test("read/write") { + val t = new Interaction() + .setInputCols(Array("myInputCol", "myInputCol2")) + .setOutputCol("myOutputCol") + testDefaultReadWrite(t) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala new file mode 100644 index 000000000000..db4f56ed60d3 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/LSHTest.scala @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.apache.spark.ml.linalg.{Vector, VectorUDT} +import org.apache.spark.ml.util.{MLTestingUtils, SchemaUtils} +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.DataTypes + +private[ml] object LSHTest { + /** + * For any locality sensitive function h in a metric space, we meed to verify whether + * the following property is satisfied. + * + * There exist dist1, dist2, p1, p2, so that for any two elements e1 and e2, + * If dist(e1, e2) is less than or equal to dist1, then Pr{h(x) == h(y)} is greater than + * or equal to p1 + * If dist(e1, e2) is greater than or equal to dist2, then Pr{h(x) == h(y)} is less than + * or equal to p2 + * + * This is called locality sensitive property. This method checks the property on an + * existing dataset and calculate the probabilities. + * (https://en.wikipedia.org/wiki/Locality-sensitive_hashing#Definition) + * + * This method hashes each elements to hash buckets using LSH, and calculate the false positive + * and false negative: + * False positive: Of all (e1, e2) sharing any bucket, the probability of dist(e1, e2) is greater + * than distFP + * False negative: Of all (e1, e2) not sharing buckets, the probability of dist(e1, e2) is less + * than distFN + * + * @param dataset The dataset to verify the locality sensitive hashing property. + * @param lsh The lsh instance to perform the hashing + * @param distFP Distance threshold for false positive + * @param distFN Distance threshold for false negative + * @tparam T The class type of lsh + * @return A tuple of two doubles, representing the false positive and false negative rate + */ + def calculateLSHProperty[T <: LSHModel[T]]( + dataset: Dataset[_], + lsh: LSH[T], + distFP: Double, + distFN: Double): (Double, Double) = { + val model = lsh.fit(dataset) + val inputCol = model.getInputCol + val outputCol = model.getOutputCol + val transformedData = model.transform(dataset) + + MLTestingUtils.checkCopyAndUids(lsh, model) + + // Check output column type + SchemaUtils.checkColumnType( + transformedData.schema, model.getOutputCol, DataTypes.createArrayType(new VectorUDT)) + + // Check output column dimensions + val headHashValue = transformedData.select(outputCol).head().get(0).asInstanceOf[Seq[Vector]] + assert(headHashValue.length == model.getNumHashTables) + + // Perform a cross join and label each pair of same_bucket and distance + val pairs = transformedData.as("a").crossJoin(transformedData.as("b")) + val distUDF = udf((x: Vector, y: Vector) => model.keyDistance(x, y), DataTypes.DoubleType) + val sameBucket = udf((x: Seq[Vector], y: Seq[Vector]) => model.hashDistance(x, y) == 0.0, + DataTypes.BooleanType) + val result = pairs + .withColumn("same_bucket", sameBucket(col(s"a.$outputCol"), col(s"b.$outputCol"))) + .withColumn("distance", distUDF(col(s"a.$inputCol"), col(s"b.$inputCol"))) + + // Compute the probabilities based on the join result + val positive = result.filter(col("same_bucket")) + val negative = result.filter(!col("same_bucket")) + val falsePositiveCount = positive.filter(col("distance") > distFP).count().toDouble + val falseNegativeCount = negative.filter(col("distance") < distFN).count().toDouble + (falsePositiveCount / positive.count(), falseNegativeCount / negative.count()) + } + + /** + * Compute the precision and recall of approximate nearest neighbors + * @param lsh The lsh instance + * @param dataset the dataset to look for the key + * @param key The key to hash for the item + * @param k The maximum number of items closest to the key + * @param singleProbe True for using single-probe; false for multi-probe + * @tparam T The class type of lsh + * @return A tuple of two doubles, representing precision and recall rate + */ + def calculateApproxNearestNeighbors[T <: LSHModel[T]]( + lsh: LSH[T], + dataset: Dataset[_], + key: Vector, + k: Int, + singleProbe: Boolean): (Double, Double) = { + val model = lsh.fit(dataset) + + // Compute expected + val distUDF = udf((x: Vector) => model.keyDistance(x, key), DataTypes.DoubleType) + val expected = dataset.sort(distUDF(col(model.getInputCol))).limit(k) + + // Compute actual + val actual = model.approxNearestNeighbors(dataset, key, k, singleProbe, "distCol") + + assert(actual.schema.sameType(model + .transformSchema(dataset.schema) + .add("distCol", DataTypes.DoubleType)) + ) + + if (!singleProbe) { + assert(actual.count() == k) + } + + // Compute precision and recall + val correctCount = expected.join(actual, model.getInputCol).count().toDouble + (correctCount / actual.count(), correctCount / expected.count()) + } + + /** + * Compute the precision and recall of approximate similarity join + * @param lsh The lsh instance + * @param datasetA One of the datasets to join + * @param datasetB Another dataset to join + * @param threshold The threshold for the distance of record pairs + * @tparam T The class type of lsh + * @return A tuple of two doubles, representing precision and recall rate + */ + def calculateApproxSimilarityJoin[T <: LSHModel[T]]( + lsh: LSH[T], + datasetA: Dataset[_], + datasetB: Dataset[_], + threshold: Double): (Double, Double) = { + val model = lsh.fit(datasetA) + val inputCol = model.getInputCol + + // Compute expected + val distUDF = udf((x: Vector, y: Vector) => model.keyDistance(x, y), DataTypes.DoubleType) + val expected = datasetA.as("a").crossJoin(datasetB.as("b")) + .filter(distUDF(col(s"a.$inputCol"), col(s"b.$inputCol")) < threshold) + + // Compute actual + val actual = model.approxSimilarityJoin(datasetA, datasetB, threshold) + + SchemaUtils.checkColumnType(actual.schema, "distCol", DataTypes.DoubleType) + assert(actual.schema.apply("datasetA").dataType + .sameType(model.transformSchema(datasetA.schema))) + assert(actual.schema.apply("datasetB").dataType + .sameType(model.transformSchema(datasetB.schema))) + + // Compute precision and recall + val correctCount = actual.filter(col("distCol") < threshold).count().toDouble + (correctCount / actual.count(), correctCount / expected.count()) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala new file mode 100644 index 000000000000..918da4f9388d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/MaxAbsScalerSuite.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.ml.feature + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.Row + +class MaxAbsScalerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + + test("MaxAbsScaler fit basic case") { + val data = Array( + Vectors.dense(1, 0, 100), + Vectors.dense(2, 0, 0), + Vectors.sparse(3, Array(0, 2), Array(-2, -100)), + Vectors.sparse(3, Array(0), Array(-1.5))) + + val expected: Array[Vector] = Array( + Vectors.dense(0.5, 0, 1), + Vectors.dense(1, 0, 0), + Vectors.sparse(3, Array(0, 2), Array(-1, -1)), + Vectors.sparse(3, Array(0), Array(-0.75))) + + val df = data.zip(expected).toSeq.toDF("features", "expected") + val scaler = new MaxAbsScaler() + .setInputCol("features") + .setOutputCol("scaled") + + val model = scaler.fit(df) + model.transform(df).select("expected", "scaled").collect() + .foreach { case Row(vector1: Vector, vector2: Vector) => + assert(vector1.equals(vector2), s"MaxAbsScaler ut error: $vector2 should be $vector1") + } + + MLTestingUtils.checkCopyAndUids(scaler, model) + } + + test("MaxAbsScaler read/write") { + val t = new MaxAbsScaler() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + testDefaultReadWrite(t) + } + + test("MaxAbsScalerModel read/write") { + val instance = new MaxAbsScalerModel( + "myMaxAbsScalerModel", Vectors.dense(1.0, 10.0)) + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + val newInstance = testDefaultReadWrite(instance) + assert(newInstance.maxAbs === instance.maxAbs) + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/MinHashLSHSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/MinHashLSHSuite.scala new file mode 100644 index 000000000000..96df68dbdf05 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/MinHashLSHSuite.scala @@ -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. + */ + +package org.apache.spark.ml.feature + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.Dataset + +class MinHashLSHSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + @transient var dataset: Dataset[_] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + val data = { + for (i <- 0 to 95) yield Vectors.sparse(100, (i until i + 5).map((_, 1.0))) + } + dataset = spark.createDataFrame(data.map(Tuple1.apply)).toDF("keys") + } + + test("params") { + ParamsSuite.checkParams(new MinHashLSH) + val model = new MinHashLSHModel("mh", randCoefficients = Array((1, 0))) + ParamsSuite.checkParams(model) + } + + test("MinHashLSH: default params") { + val rp = new MinHashLSH + assert(rp.getNumHashTables === 1.0) + } + + test("read/write") { + def checkModelData(model: MinHashLSHModel, model2: MinHashLSHModel): Unit = { + assertResult(model.randCoefficients)(model2.randCoefficients) + } + val mh = new MinHashLSH() + val settings = Map("inputCol" -> "keys", "outputCol" -> "values") + testEstimatorAndModelReadWrite(mh, dataset, settings, settings, checkModelData) + } + + test("Model copy and uid checks") { + val mh = new MinHashLSH() + .setInputCol("keys") + .setOutputCol("values") + val model = mh.fit(dataset) + assert(mh.uid === model.uid) + MLTestingUtils.checkCopyAndUids(mh, model) + } + + test("hashFunction") { + val model = new MinHashLSHModel("mh", randCoefficients = Array((0, 1), (1, 2), (3, 0))) + val res = model.hashFunction(Vectors.sparse(10, Seq((2, 1.0), (3, 1.0), (5, 1.0), (7, 1.0)))) + assert(res.length == 3) + assert(res(0).equals(Vectors.dense(1.0))) + assert(res(1).equals(Vectors.dense(5.0))) + assert(res(2).equals(Vectors.dense(9.0))) + } + + test("hashFunction: empty vector") { + val model = new MinHashLSHModel("mh", randCoefficients = Array((0, 1), (1, 2), (3, 0))) + intercept[IllegalArgumentException] { + model.hashFunction(Vectors.sparse(10, Seq())) + } + } + + test("keyDistance") { + val model = new MinHashLSHModel("mh", randCoefficients = Array((1, 0))) + val v1 = Vectors.sparse(10, Seq((2, 1.0), (3, 1.0), (5, 1.0), (7, 1.0))) + val v2 = Vectors.sparse(10, Seq((1, 1.0), (3, 1.0), (5, 1.0), (7, 1.0), (9, 1.0))) + val keyDist = model.keyDistance(v1, v2) + assert(keyDist === 0.5) + } + + test("MinHashLSH: test of LSH property") { + val mh = new MinHashLSH() + .setInputCol("keys") + .setOutputCol("values") + .setSeed(12344) + + val (falsePositive, falseNegative) = LSHTest.calculateLSHProperty(dataset, mh, 0.75, 0.5) + assert(falsePositive < 0.3) + assert(falseNegative < 0.3) + } + + test("MinHashLSH: test of inputDim > prime") { + val mh = new MinHashLSH() + .setInputCol("keys") + .setOutputCol("values") + .setSeed(12344) + + val data = { + for (i <- 0 to 2) yield Vectors.sparse(Int.MaxValue, (i until i + 5).map((_, 1.0))) + } + val badDataset = spark.createDataFrame(data.map(Tuple1.apply)).toDF("keys") + intercept[IllegalArgumentException] { + mh.fit(badDataset) + } + } + + test("approxNearestNeighbors for min hash") { + val mh = new MinHashLSH() + .setNumHashTables(20) + .setInputCol("keys") + .setOutputCol("values") + .setSeed(12345) + + val key: Vector = Vectors.sparse(100, + (0 until 100).filter(_.toString.contains("1")).map((_, 1.0))) + + val (precision, recall) = LSHTest.calculateApproxNearestNeighbors(mh, dataset, key, 20, + singleProbe = true) + assert(precision >= 0.7) + assert(recall >= 0.7) + } + + test("approxNearestNeighbors for numNeighbors <= 0") { + val model = new MinHashLSHModel("mh", randCoefficients = Array((1, 0))) + + val key: Vector = Vectors.sparse(100, + (0 until 100).filter(_.toString.contains("1")).map((_, 1.0))) + + intercept[IllegalArgumentException] { + model.approxNearestNeighbors(dataset, key, 0) + } + intercept[IllegalArgumentException] { + model.approxNearestNeighbors(dataset, key, -1) + } + } + + test("approxSimilarityJoin for min hash on different dataset") { + val data1 = { + for (i <- 0 until 20) yield Vectors.sparse(100, (5 * i until 5 * i + 5).map((_, 1.0))) + } + val df1 = spark.createDataFrame(data1.map(Tuple1.apply)).toDF("keys") + + val data2 = { + for (i <- 0 until 30) yield Vectors.sparse(100, (3 * i until 3 * i + 3).map((_, 1.0))) + } + val df2 = spark.createDataFrame(data2.map(Tuple1.apply)).toDF("keys") + + val mh = new MinHashLSH() + .setNumHashTables(20) + .setInputCol("keys") + .setOutputCol("values") + .setSeed(12345) + + val (precision, recall) = LSHTest.calculateApproxSimilarityJoin(mh, df1, df2, 0.5) + assert(precision == 1.0) + assert(recall >= 0.7) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala index c04dda41eea3..51db74eb739c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/MinMaxScalerSuite.scala @@ -18,16 +18,16 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.Row -class MinMaxScalerSuite extends SparkFunSuite with MLlibTestSparkContext { +class MinMaxScalerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { - test("MinMaxScaler fit basic case") { - val sqlContext = new SQLContext(sc) + import testImplicits._ + test("MinMaxScaler fit basic case") { val data = Array( Vectors.dense(1, 0, Long.MinValue), Vectors.dense(2, 0, 0), @@ -40,7 +40,7 @@ class MinMaxScalerSuite extends SparkFunSuite with MLlibTestSparkContext { Vectors.sparse(3, Array(0, 2), Array(5, 5)), Vectors.sparse(3, Array(0), Array(-2.5))) - val df = sqlContext.createDataFrame(data.zip(expected)).toDF("features", "expected") + val df = data.zip(expected).toSeq.toDF("features", "expected") val scaler = new MinMaxScaler() .setInputCol("features") .setOutputCol("scaled") @@ -53,20 +53,68 @@ class MinMaxScalerSuite extends SparkFunSuite with MLlibTestSparkContext { assert(vector1.equals(vector2), "Transformed vector is different with expected.") } - // copied model must have the same parent. - MLTestingUtils.checkCopy(model) + MLTestingUtils.checkCopyAndUids(scaler, model) } test("MinMaxScaler arguments max must be larger than min") { withClue("arguments max must be larger than min") { + val dummyDF = Seq((1, Vectors.dense(1.0, 2.0))).toDF("id", "features") intercept[IllegalArgumentException] { - val scaler = new MinMaxScaler().setMin(10).setMax(0) - scaler.validateParams() + val scaler = new MinMaxScaler().setMin(10).setMax(0).setInputCol("features") + scaler.transformSchema(dummyDF.schema) } intercept[IllegalArgumentException] { - val scaler = new MinMaxScaler().setMin(0).setMax(0) - scaler.validateParams() + val scaler = new MinMaxScaler().setMin(0).setMax(0).setInputCol("features") + scaler.transformSchema(dummyDF.schema) } } } + + test("MinMaxScaler read/write") { + val t = new MinMaxScaler() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setMax(1.0) + .setMin(-1.0) + testDefaultReadWrite(t) + } + + test("MinMaxScalerModel read/write") { + val instance = new MinMaxScalerModel( + "myMinMaxScalerModel", Vectors.dense(-1.0, 0.0), Vectors.dense(1.0, 10.0)) + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setMin(-1.0) + .setMax(1.0) + val newInstance = testDefaultReadWrite(instance) + assert(newInstance.originalMin === instance.originalMin) + assert(newInstance.originalMax === instance.originalMax) + } + + test("MinMaxScaler should remain NaN value") { + val data = Array( + Vectors.dense(1, Double.NaN, 2.0, 2.0), + Vectors.dense(2, 2.0, 0.0, 3.0), + Vectors.dense(3, Double.NaN, 0.0, 1.0), + Vectors.dense(6, 2.0, 2.0, Double.NaN)) + + val expected: Array[Vector] = Array( + Vectors.dense(-5.0, Double.NaN, 5.0, 0.0), + Vectors.dense(-3.0, 0.0, -5.0, 5.0), + Vectors.dense(-1.0, Double.NaN, -5.0, -5.0), + Vectors.dense(5.0, 0.0, 5.0, Double.NaN)) + + val df = data.zip(expected).toSeq.toDF("features", "expected") + val scaler = new MinMaxScaler() + .setInputCol("features") + .setOutputCol("scaled") + .setMin(-5) + .setMax(5) + + val model = scaler.fit(df) + model.transform(df).select("expected", "scaled").collect() + .foreach { case Row(vector1: Vector, vector2: Vector) => + assert(vector1.equals(vector2), "Transformed vector is different with expected.") + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala index ab97e3dbc6ee..d4975c0b4e20 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala @@ -20,24 +20,26 @@ package org.apache.spark.ml.feature import scala.beans.BeanInfo import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{Dataset, Row} @BeanInfo case class NGramTestData(inputTokens: Array[String], wantedNGrams: Array[String]) -class NGramSuite extends SparkFunSuite with MLlibTestSparkContext { +class NGramSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + import org.apache.spark.ml.feature.NGramSuite._ + import testImplicits._ test("default behavior yields bigram features") { val nGram = new NGram() .setInputCol("inputTokens") .setOutputCol("nGrams") - val dataset = sqlContext.createDataFrame(Seq( - NGramTestData( - Array("Test", "for", "ngram", "."), - Array("Test for", "for ngram", "ngram .") - ))) + val dataset = Seq(NGramTestData( + Array("Test", "for", "ngram", "."), + Array("Test for", "for ngram", "ngram .") + )).toDF() testNGram(nGram, dataset) } @@ -46,11 +48,10 @@ class NGramSuite extends SparkFunSuite with MLlibTestSparkContext { .setInputCol("inputTokens") .setOutputCol("nGrams") .setN(4) - val dataset = sqlContext.createDataFrame(Seq( - NGramTestData( - Array("a", "b", "c", "d", "e"), - Array("a b c d", "b c d e") - ))) + val dataset = Seq(NGramTestData( + Array("a", "b", "c", "d", "e"), + Array("a b c d", "b c d e") + )).toDF() testNGram(nGram, dataset) } @@ -59,11 +60,7 @@ class NGramSuite extends SparkFunSuite with MLlibTestSparkContext { .setInputCol("inputTokens") .setOutputCol("nGrams") .setN(4) - val dataset = sqlContext.createDataFrame(Seq( - NGramTestData( - Array(), - Array() - ))) + val dataset = Seq(NGramTestData(Array(), Array())).toDF() testNGram(nGram, dataset) } @@ -72,18 +69,25 @@ class NGramSuite extends SparkFunSuite with MLlibTestSparkContext { .setInputCol("inputTokens") .setOutputCol("nGrams") .setN(6) - val dataset = sqlContext.createDataFrame(Seq( - NGramTestData( - Array("a", "b", "c", "d", "e"), - Array() - ))) + val dataset = Seq(NGramTestData( + Array("a", "b", "c", "d", "e"), + Array() + )).toDF() testNGram(nGram, dataset) } + + test("read/write") { + val t = new NGram() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setN(3) + testDefaultReadWrite(t) + } } object NGramSuite extends SparkFunSuite { - def testNGram(t: NGram, dataset: DataFrame): Unit = { + def testNGram(t: NGram, dataset: Dataset[_]): Unit = { t.transform(dataset) .select("nGrams", "wantedNGrams") .collect() diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala index 9f03470b7f32..c75027fb4553 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala @@ -18,13 +18,16 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row} -class NormalizerSuite extends SparkFunSuite with MLlibTestSparkContext { +class NormalizerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ @transient var data: Array[Vector] = _ @transient var dataFrame: DataFrame = _ @@ -60,8 +63,7 @@ class NormalizerSuite extends SparkFunSuite with MLlibTestSparkContext { Vectors.sparse(3, Seq()) ) - val sqlContext = new SQLContext(sc) - dataFrame = sqlContext.createDataFrame(sc.parallelize(data, 2).map(NormalizerSuite.FeatureData)) + dataFrame = data.map(NormalizerSuite.FeatureData).toSeq.toDF() normalizer = new Normalizer() .setInputCol("features") .setOutputCol("normalized_features") @@ -104,6 +106,14 @@ class NormalizerSuite extends SparkFunSuite with MLlibTestSparkContext { assertValues(result, l1Normalized) } + + test("read/write") { + val t = new Normalizer() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setP(3.0) + testDefaultReadWrite(t) + } } private object NormalizerSuite { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala index 321eeb843941..c44c6813a94b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala @@ -19,17 +19,22 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.{AttributeGroup, BinaryAttribute, NominalAttribute} +import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types._ -class OneHotEncoderSuite extends SparkFunSuite with MLlibTestSparkContext { +class OneHotEncoderSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ def stringIndexed(): DataFrame = { - val data = sc.parallelize(Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")), 2) - val df = sqlContext.createDataFrame(data).toDF("id", "label") + val data = Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")) + val df = data.toDF("id", "label") val indexer = new StringIndexer() .setInputCol("label") .setOutputCol("labelIndex") @@ -46,10 +51,12 @@ class OneHotEncoderSuite extends SparkFunSuite with MLlibTestSparkContext { val encoder = new OneHotEncoder() .setInputCol("labelIndex") .setOutputCol("labelVec") - .setDropLast(false) + assert(encoder.getDropLast === true) + encoder.setDropLast(false) + assert(encoder.getDropLast === false) val encoded = encoder.transform(transformed) - val output = encoded.select("id", "labelVec").map { r => + val output = encoded.select("id", "labelVec").rdd.map { r => val vec = r.getAs[Vector](1) (r.getInt(0), vec(0), vec(1), vec(2)) }.collect().toSet @@ -66,7 +73,7 @@ class OneHotEncoderSuite extends SparkFunSuite with MLlibTestSparkContext { .setOutputCol("labelVec") val encoded = encoder.transform(transformed) - val output = encoded.select("id", "labelVec").map { r => + val output = encoded.select("id", "labelVec").rdd.map { r => val vec = r.getAs[Vector](1) (r.getInt(0), vec(0), vec(1)) }.collect().toSet @@ -78,7 +85,7 @@ class OneHotEncoderSuite extends SparkFunSuite with MLlibTestSparkContext { test("input column with ML attribute") { val attr = NominalAttribute.defaultAttr.withValues("small", "medium", "large") - val df = sqlContext.createDataFrame(Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply)).toDF("size") + val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("size") .select(col("size").as("size", attr.toMetadata())) val encoder = new OneHotEncoder() .setInputCol("size") @@ -91,7 +98,7 @@ class OneHotEncoderSuite extends SparkFunSuite with MLlibTestSparkContext { } test("input column without ML attribute") { - val df = sqlContext.createDataFrame(Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply)).toDF("index") + val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("index") val encoder = new OneHotEncoder() .setInputCol("index") .setOutputCol("encoded") @@ -101,4 +108,40 @@ class OneHotEncoderSuite extends SparkFunSuite with MLlibTestSparkContext { assert(group.getAttr(0) === BinaryAttribute.defaultAttr.withName("0").withIndex(0)) assert(group.getAttr(1) === BinaryAttribute.defaultAttr.withName("1").withIndex(1)) } + + test("read/write") { + val t = new OneHotEncoder() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setDropLast(false) + testDefaultReadWrite(t) + } + + test("OneHotEncoder with varying types") { + val df = stringIndexed() + val dfWithTypes = df + .withColumn("shortLabel", df("labelIndex").cast(ShortType)) + .withColumn("longLabel", df("labelIndex").cast(LongType)) + .withColumn("intLabel", df("labelIndex").cast(IntegerType)) + .withColumn("floatLabel", df("labelIndex").cast(FloatType)) + .withColumn("decimalLabel", df("labelIndex").cast(DecimalType(10, 0))) + val cols = Array("labelIndex", "shortLabel", "longLabel", "intLabel", + "floatLabel", "decimalLabel") + for (col <- cols) { + val encoder = new OneHotEncoder() + .setInputCol(col) + .setOutputCol("labelVec") + .setDropLast(false) + val encoded = encoder.transform(dfWithTypes) + + val output = encoded.select("id", "labelVec").rdd.map { r => + val vec = r.getAs[Vector](1) + (r.getInt(0), vec(0), vec(1), vec(2)) + }.collect().toSet + // a -> 0, b -> 2, c -> 1 + val expected = Set((0, 1.0, 0.0, 0.0), (1, 0.0, 0.0, 1.0), (2, 0.0, 1.0, 0.0), + (3, 1.0, 0.0, 0.0), (4, 1.0, 0.0, 0.0), (5, 0.0, 1.0, 0.0)) + assert(output === expected) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala index 30c500f87a76..3067a52a4df7 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/PCASuite.scala @@ -18,21 +18,24 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.util.MLTestingUtils +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} import org.apache.spark.mllib.linalg.distributed.RowMatrix -import org.apache.spark.mllib.linalg.{Vector, Vectors, DenseMatrix, Matrices} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ -import org.apache.spark.mllib.feature.{PCAModel => OldPCAModel} import org.apache.spark.sql.Row -class PCASuite extends SparkFunSuite with MLlibTestSparkContext { +class PCASuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ test("params") { ParamsSuite.checkParams(new PCA) val mat = Matrices.dense(2, 2, Array(0.0, 1.0, 2.0, 3.0)).asInstanceOf[DenseMatrix] - val model = new PCAModel("pca", new OldPCAModel(2, mat)) + val explainedVariance = Vectors.dense(0.5, 0.5).asInstanceOf[DenseVector] + val model = new PCAModel("pca", mat, explainedVariance) ParamsSuite.checkParams(model) } @@ -45,24 +48,40 @@ class PCASuite extends SparkFunSuite with MLlibTestSparkContext { val dataRDD = sc.parallelize(data, 2) - val mat = new RowMatrix(dataRDD) + val mat = new RowMatrix(dataRDD.map(OldVectors.fromML)) val pc = mat.computePrincipalComponents(3) - val expected = mat.multiply(pc).rows + val expected = mat.multiply(pc).rows.map(_.asML) - val df = sqlContext.createDataFrame(dataRDD.zip(expected)).toDF("features", "expected") + val df = dataRDD.zip(expected).toDF("features", "expected") val pca = new PCA() .setInputCol("features") .setOutputCol("pca_features") .setK(3) - .fit(df) - // copied model must have the same parent. - MLTestingUtils.checkCopy(pca) + val pcaModel = pca.fit(df) + + MLTestingUtils.checkCopyAndUids(pca, pcaModel) - pca.transform(df).select("pca_features", "expected").collect().foreach { + pcaModel.transform(df).select("pca_features", "expected").collect().foreach { case Row(x: Vector, y: Vector) => assert(x ~== y absTol 1e-5, "Transformed vector is different with expected vector.") } } + + test("PCA read/write") { + val t = new PCA() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setK(3) + testDefaultReadWrite(t) + } + + test("PCAModel read/write") { + val instance = new PCAModel("myPCAModel", + Matrices.dense(2, 2, Array(0.0, 1.0, 2.0, 3.0)).asInstanceOf[DenseMatrix], + Vectors.dense(0.5, 0.5).asInstanceOf[DenseVector]) + val newInstance = testDefaultReadWrite(instance) + assert(newInstance.pc === instance.pc) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala index 29eebd8960eb..e4b0ddf98bfa 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/PolynomialExpansionSuite.scala @@ -17,38 +17,51 @@ package org.apache.spark.ml.feature -import org.apache.spark.ml.param.ParamsSuite import org.scalatest.exceptions.TestFailedException import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.Row -class PolynomialExpansionSuite extends SparkFunSuite with MLlibTestSparkContext { +class PolynomialExpansionSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ test("params") { ParamsSuite.checkParams(new PolynomialExpansion) } - test("Polynomial expansion with default parameter") { - val data = Array( - Vectors.sparse(3, Seq((0, -2.0), (1, 2.3))), - Vectors.dense(-2.0, 2.3), - Vectors.dense(0.0, 0.0, 0.0), - Vectors.dense(0.6, -1.1, -3.0), - Vectors.sparse(3, Seq()) - ) + private val data = Array( + Vectors.sparse(3, Seq((0, -2.0), (1, 2.3))), + Vectors.dense(-2.0, 2.3), + Vectors.dense(0.0, 0.0, 0.0), + Vectors.dense(0.6, -1.1, -3.0), + Vectors.sparse(3, Seq()) + ) + + private val twoDegreeExpansion: Array[Vector] = Array( + Vectors.sparse(9, Array(0, 1, 2, 3, 4), Array(-2.0, 4.0, 2.3, -4.6, 5.29)), + Vectors.dense(-2.0, 4.0, 2.3, -4.6, 5.29), + Vectors.dense(new Array[Double](9)), + Vectors.dense(0.6, 0.36, -1.1, -0.66, 1.21, -3.0, -1.8, 3.3, 9.0), + Vectors.sparse(9, Array.empty, Array.empty)) + + private val threeDegreeExpansion: Array[Vector] = Array( + Vectors.sparse(19, Array(0, 1, 2, 3, 4, 5, 6, 7, 8), + Array(-2.0, 4.0, -8.0, 2.3, -4.6, 9.2, 5.29, -10.58, 12.17)), + Vectors.dense(-2.0, 4.0, -8.0, 2.3, -4.6, 9.2, 5.29, -10.58, 12.17), + Vectors.dense(new Array[Double](19)), + Vectors.dense(0.6, 0.36, 0.216, -1.1, -0.66, -0.396, 1.21, 0.726, -1.331, -3.0, -1.8, + -1.08, 3.3, 1.98, -3.63, 9.0, 5.4, -9.9, -27.0), + Vectors.sparse(19, Array.empty, Array.empty)) - val twoDegreeExpansion: Array[Vector] = Array( - Vectors.sparse(9, Array(0, 1, 2, 3, 4), Array(-2.0, 4.0, 2.3, -4.6, 5.29)), - Vectors.dense(-2.0, 4.0, 2.3, -4.6, 5.29), - Vectors.dense(new Array[Double](9)), - Vectors.dense(0.6, 0.36, -1.1, -0.66, 1.21, -3.0, -1.8, 3.3, 9.0), - Vectors.sparse(9, Array.empty, Array.empty)) - - val df = sqlContext.createDataFrame(data.zip(twoDegreeExpansion)).toDF("features", "expected") + test("Polynomial expansion with default parameter") { + val df = data.zip(twoDegreeExpansion).toSeq.toDF("features", "expected") val polynomialExpansion = new PolynomialExpansion() .setInputCol("features") @@ -65,24 +78,7 @@ class PolynomialExpansionSuite extends SparkFunSuite with MLlibTestSparkContext } test("Polynomial expansion with setter") { - val data = Array( - Vectors.sparse(3, Seq((0, -2.0), (1, 2.3))), - Vectors.dense(-2.0, 2.3), - Vectors.dense(0.0, 0.0, 0.0), - Vectors.dense(0.6, -1.1, -3.0), - Vectors.sparse(3, Seq()) - ) - - val threeDegreeExpansion: Array[Vector] = Array( - Vectors.sparse(19, Array(0, 1, 2, 3, 4, 5, 6, 7, 8), - Array(-2.0, 4.0, -8.0, 2.3, -4.6, 9.2, 5.29, -10.58, 12.17)), - Vectors.dense(-2.0, 4.0, -8.0, 2.3, -4.6, 9.2, 5.29, -10.58, 12.17), - Vectors.dense(new Array[Double](19)), - Vectors.dense(0.6, 0.36, 0.216, -1.1, -0.66, -0.396, 1.21, 0.726, -1.331, -3.0, -1.8, - -1.08, 3.3, 1.98, -3.63, 9.0, 5.4, -9.9, -27.0), - Vectors.sparse(19, Array.empty, Array.empty)) - - val df = sqlContext.createDataFrame(data.zip(threeDegreeExpansion)).toDF("features", "expected") + val df = data.zip(threeDegreeExpansion).toSeq.toDF("features", "expected") val polynomialExpansion = new PolynomialExpansion() .setInputCol("features") @@ -98,5 +94,52 @@ class PolynomialExpansionSuite extends SparkFunSuite with MLlibTestSparkContext throw new TestFailedException("Unmatched data types after polynomial expansion", 0) } } + + test("Polynomial expansion with degree 1 is identity on vectors") { + val df = data.zip(data).toSeq.toDF("features", "expected") + + val polynomialExpansion = new PolynomialExpansion() + .setInputCol("features") + .setOutputCol("polyFeatures") + .setDegree(1) + + polynomialExpansion.transform(df).select("polyFeatures", "expected").collect().foreach { + case Row(expanded: Vector, expected: Vector) => + assert(expanded ~== expected absTol 1e-1) + case _ => + throw new TestFailedException("Unmatched data types after polynomial expansion", 0) + } + } + + test("read/write") { + val t = new PolynomialExpansion() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setDegree(3) + testDefaultReadWrite(t) + } + + test("SPARK-17027. Integer overflow in PolynomialExpansion.getPolySize") { + val data: Array[(Vector, Int, Int)] = Array( + (Vectors.dense(1.0, 2.0, 3.0, 4.0, 5.0), 3002, 4367), + (Vectors.sparse(5, Seq((0, 1.0), (4, 5.0))), 3002, 4367), + (Vectors.dense(1.0, 2.0, 3.0, 4.0, 5.0, 6.0), 8007, 12375) + ) + + val df = data.toSeq.toDF("features", "expectedPoly10size", "expectedPoly11size") + + val t = new PolynomialExpansion() + .setInputCol("features") + .setOutputCol("polyFeatures") + + for (i <- Seq(10, 11)) { + val transformed = t.setDegree(i) + .transform(df) + .select(s"expectedPoly${i}size", "polyFeatures") + .rdd.map { case Row(expected: Int, v: Vector) => expected == v.size } + + assert(transformed.collect.forall(identity)) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala index b2bdd8935f90..f219f775b218 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala @@ -17,82 +17,133 @@ package org.apache.spark.ml.feature -import org.apache.spark.ml.attribute.{Attribute, NominalAttribute} +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{Row, SQLContext} -import org.apache.spark.{SparkContext, SparkFunSuite} - -class QuantileDiscretizerSuite extends SparkFunSuite with MLlibTestSparkContext { - import org.apache.spark.ml.feature.QuantileDiscretizerSuite._ - - test("Test quantile discretizer") { - checkDiscretizedData(sc, - Array[Double](1, 2, 3, 3, 3, 3, 3, 3, 3), - 10, - Array[Double](1, 2, 3, 3, 3, 3, 3, 3, 3), - Array("-Infinity, 1.0", "1.0, 2.0", "2.0, 3.0", "3.0, Infinity")) - - checkDiscretizedData(sc, - Array[Double](1, 2, 3, 3, 3, 3, 3, 3, 3), - 4, - Array[Double](1, 2, 3, 3, 3, 3, 3, 3, 3), - Array("-Infinity, 1.0", "1.0, 2.0", "2.0, 3.0", "3.0, Infinity")) - - checkDiscretizedData(sc, - Array[Double](1, 2, 3, 3, 3, 3, 3, 3, 3), - 3, - Array[Double](0, 1, 2, 2, 2, 2, 2, 2, 2), - Array("-Infinity, 2.0", "2.0, 3.0", "3.0, Infinity")) - - checkDiscretizedData(sc, - Array[Double](1, 2, 3, 3, 3, 3, 3, 3, 3), - 2, - Array[Double](0, 1, 1, 1, 1, 1, 1, 1, 1), - Array("-Infinity, 2.0", "2.0, Infinity")) +import org.apache.spark.sql._ +import org.apache.spark.sql.functions.udf +class QuantileDiscretizerSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + test("Test observed number of buckets and their sizes match expected values") { + val spark = this.spark + import spark.implicits._ + + val datasetSize = 100000 + val numBuckets = 5 + val df = sc.parallelize(1.0 to datasetSize by 1.0).map(Tuple1.apply).toDF("input") + val discretizer = new QuantileDiscretizer() + .setInputCol("input") + .setOutputCol("result") + .setNumBuckets(numBuckets) + val result = discretizer.fit(df).transform(df) + + val observedNumBuckets = result.select("result").distinct.count + assert(observedNumBuckets === numBuckets, + "Observed number of buckets does not equal expected number of buckets.") + + val relativeError = discretizer.getRelativeError + val isGoodBucket = udf { + (size: Int) => math.abs( size - (datasetSize / numBuckets)) <= (relativeError * datasetSize) + } + val numGoodBuckets = result.groupBy("result").count.filter(isGoodBucket($"count")).count + assert(numGoodBuckets === numBuckets, + "Bucket sizes are not within expected relative error tolerance.") } - test("Test getting splits") { - val splitTestPoints = Array( - Array[Double]() -> Array(Double.NegativeInfinity, 0, Double.PositiveInfinity), - Array(Double.NegativeInfinity) -> Array(Double.NegativeInfinity, 0, Double.PositiveInfinity), - Array(Double.PositiveInfinity) -> Array(Double.NegativeInfinity, 0, Double.PositiveInfinity), - Array(Double.NegativeInfinity, Double.PositiveInfinity) - -> Array(Double.NegativeInfinity, 0, Double.PositiveInfinity), - Array(0.0) -> Array(Double.NegativeInfinity, 0, Double.PositiveInfinity), - Array(1.0) -> Array(Double.NegativeInfinity, 1, Double.PositiveInfinity), - Array(0.0, 1.0) -> Array(Double.NegativeInfinity, 0, 1, Double.PositiveInfinity) - ) - for ((ori, res) <- splitTestPoints) { - assert(QuantileDiscretizer.getSplits(ori) === res, "Returned splits are invalid.") + test("Test on data with high proportion of duplicated values") { + val spark = this.spark + import spark.implicits._ + + val numBuckets = 5 + val expectedNumBuckets = 3 + val df = sc.parallelize(Array(1.0, 3.0, 2.0, 1.0, 1.0, 2.0, 3.0, 2.0, 2.0, 2.0, 1.0, 3.0)) + .map(Tuple1.apply).toDF("input") + val discretizer = new QuantileDiscretizer() + .setInputCol("input") + .setOutputCol("result") + .setNumBuckets(numBuckets) + val result = discretizer.fit(df).transform(df) + val observedNumBuckets = result.select("result").distinct.count + assert(observedNumBuckets == expectedNumBuckets, + s"Observed number of buckets are not correct." + + s" Expected $expectedNumBuckets but found $observedNumBuckets") + } + + test("Test transform on data with NaN value") { + val spark = this.spark + import spark.implicits._ + + val numBuckets = 3 + val validData = Array(-0.9, -0.5, -0.3, 0.0, 0.2, 0.5, 0.9, Double.NaN, Double.NaN, Double.NaN) + val expectedKeep = Array(0.0, 0.0, 1.0, 1.0, 2.0, 2.0, 2.0, 3.0, 3.0, 3.0) + val expectedSkip = Array(0.0, 0.0, 1.0, 1.0, 2.0, 2.0, 2.0) + + val discretizer = new QuantileDiscretizer() + .setInputCol("input") + .setOutputCol("result") + .setNumBuckets(numBuckets) + + withClue("QuantileDiscretizer with handleInvalid=error should throw exception for NaN values") { + val dataFrame: DataFrame = validData.toSeq.toDF("input") + intercept[SparkException] { + discretizer.fit(dataFrame).transform(dataFrame).collect() + } + } + + List(("keep", expectedKeep), ("skip", expectedSkip)).foreach{ + case(u, v) => + discretizer.setHandleInvalid(u) + val dataFrame: DataFrame = validData.zip(v).toSeq.toDF("input", "expected") + val result = discretizer.fit(dataFrame).transform(dataFrame) + result.select("result", "expected").collect().foreach { + case Row(x: Double, y: Double) => + assert(x === y, + s"The feature value is not correct after bucketing. Expected $y but found $x") + } } } -} -private object QuantileDiscretizerSuite extends SparkFunSuite { + test("Test transform method on unseen data") { + val spark = this.spark + import spark.implicits._ - def checkDiscretizedData( - sc: SparkContext, - data: Array[Double], - numBucket: Int, - expectedResult: Array[Double], - expectedAttrs: Array[String]): Unit = { - val sqlCtx = SQLContext.getOrCreate(sc) - import sqlCtx.implicits._ + val trainDF = sc.parallelize(1.0 to 100.0 by 1.0).map(Tuple1.apply).toDF("input") + val testDF = sc.parallelize(-10.0 to 110.0 by 1.0).map(Tuple1.apply).toDF("input") + val discretizer = new QuantileDiscretizer() + .setInputCol("input") + .setOutputCol("result") + .setNumBuckets(5) - val df = sc.parallelize(data.map(Tuple1.apply)).toDF("input") - val discretizer = new QuantileDiscretizer().setInputCol("input").setOutputCol("result") - .setNumBuckets(numBucket) - val result = discretizer.fit(df).transform(df) + val result = discretizer.fit(trainDF).transform(testDF) + val firstBucketSize = result.filter(result("result") === 0.0).count + val lastBucketSize = result.filter(result("result") === 4.0).count + + assert(firstBucketSize === 30L, + s"Size of first bucket ${firstBucketSize} did not equal expected value of 30.") + assert(lastBucketSize === 31L, + s"Size of last bucket ${lastBucketSize} did not equal expected value of 31.") + } + + test("read/write") { + val t = new QuantileDiscretizer() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setNumBuckets(6) + testDefaultReadWrite(t) + } - val transformedFeatures = result.select("result").collect() - .map { case Row(transformedFeature: Double) => transformedFeature } - val transformedAttrs = Attribute.fromStructField(result.schema("result")) - .asInstanceOf[NominalAttribute].values.get + test("Verify resulting model has parent") { + val spark = this.spark + import spark.implicits._ - assert(transformedFeatures === expectedResult, - "Transformed features do not equal expected features.") - assert(transformedAttrs === expectedAttrs, - "Transformed attributes do not equal expected attributes.") + val df = sc.parallelize(1 to 100).map(Tuple1.apply).toDF("input") + val discretizer = new QuantileDiscretizer() + .setInputCol("input") + .setOutputCol("result") + .setNumBuckets(5) + val model = discretizer.fit(df) + assert(model.hasParent) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala index dc20a5ec2152..5d09c90ec6df 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala @@ -17,29 +17,33 @@ package org.apache.spark.ml.feature -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.types.DoubleType + +class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ -class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext { test("params") { ParamsSuite.checkParams(new RFormula()) } test("transform numeric data") { val formula = new RFormula().setFormula("id ~ v1 + v2") - val original = sqlContext.createDataFrame( - Seq((0, 1.0, 3.0), (2, 2.0, 5.0))).toDF("id", "v1", "v2") + val original = Seq((0, 1.0, 3.0), (2, 2.0, 5.0)).toDF("id", "v1", "v2") val model = formula.fit(original) + MLTestingUtils.checkCopyAndUids(formula, model) val result = model.transform(original) val resultSchema = model.transformSchema(original.schema) - val expected = sqlContext.createDataFrame( - Seq( - (0, 1.0, 3.0, Vectors.dense(1.0, 3.0), 0.0), - (2, 2.0, 5.0, Vectors.dense(2.0, 5.0), 2.0)) - ).toDF("id", "v1", "v2", "features", "label") + val expected = Seq( + (0, 1.0, 3.0, Vectors.dense(1.0, 3.0), 0.0), + (2, 2.0, 5.0, Vectors.dense(2.0, 5.0), 2.0) + ).toDF("id", "v1", "v2", "features", "label") // TODO(ekl) make schema comparisons ignore metadata, to avoid .toString assert(result.schema.toString == resultSchema.toString) assert(resultSchema == expected.schema) @@ -48,27 +52,32 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext { test("features column already exists") { val formula = new RFormula().setFormula("y ~ x").setFeaturesCol("x") - val original = sqlContext.createDataFrame(Seq((0, 1.0), (2, 2.0))).toDF("x", "y") - intercept[IllegalArgumentException] { - formula.fit(original) - } + val original = Seq((0, 1.0), (2, 2.0)).toDF("x", "y") intercept[IllegalArgumentException] { formula.fit(original) } } - test("label column already exists") { + test("label column already exists and forceIndexLabel was set with false") { val formula = new RFormula().setFormula("y ~ x").setLabelCol("y") - val original = sqlContext.createDataFrame(Seq((0, 1.0), (2, 2.0))).toDF("x", "y") + val original = Seq((0, 1.0), (2, 2.0)).toDF("x", "y") val model = formula.fit(original) val resultSchema = model.transformSchema(original.schema) assert(resultSchema.length == 3) assert(resultSchema.toString == model.transform(original).schema.toString) } - test("label column already exists but is not double type") { + test("label column already exists but forceIndexLabel was set with true") { + val formula = new RFormula().setFormula("y ~ x").setLabelCol("y").setForceIndexLabel(true) + val original = spark.createDataFrame(Seq((0, 1.0), (2, 2.0))).toDF("x", "y") + intercept[IllegalArgumentException] { + formula.fit(original) + } + } + + test("label column already exists but is not numeric type") { val formula = new RFormula().setFormula("y ~ x").setLabelCol("y") - val original = sqlContext.createDataFrame(Seq((0, 1), (2, 2))).toDF("x", "y") + val original = Seq((0, true), (2, false)).toDF("x", "y") val model = formula.fit(original) intercept[IllegalArgumentException] { model.transformSchema(original.schema) @@ -80,7 +89,7 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext { test("allow missing label column for test datasets") { val formula = new RFormula().setFormula("y ~ x").setLabelCol("label") - val original = sqlContext.createDataFrame(Seq((0, 1.0), (2, 2.0))).toDF("x", "_not_y") + val original = Seq((0, 1.0), (2, 2.0)).toDF("x", "_not_y") val model = formula.fit(original) val resultSchema = model.transformSchema(original.schema) assert(resultSchema.length == 3) @@ -88,49 +97,243 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext { assert(resultSchema.toString == model.transform(original).schema.toString) } + test("allow empty label") { + val original = Seq((1, 2.0, 3.0), (4, 5.0, 6.0), (7, 8.0, 9.0)).toDF("id", "a", "b") + val formula = new RFormula().setFormula("~ a + b") + val model = formula.fit(original) + val result = model.transform(original) + val resultSchema = model.transformSchema(original.schema) + val expected = Seq( + (1, 2.0, 3.0, Vectors.dense(2.0, 3.0)), + (4, 5.0, 6.0, Vectors.dense(5.0, 6.0)), + (7, 8.0, 9.0, Vectors.dense(8.0, 9.0)) + ).toDF("id", "a", "b", "features") + assert(result.schema.toString == resultSchema.toString) + assert(result.collect() === expected.collect()) + } + test("encodes string terms") { val formula = new RFormula().setFormula("id ~ a + b") - val original = sqlContext.createDataFrame( - Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "baz", 5)) - ).toDF("id", "a", "b") + val original = Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "baz", 5)) + .toDF("id", "a", "b") val model = formula.fit(original) val result = model.transform(original) val resultSchema = model.transformSchema(original.schema) - val expected = sqlContext.createDataFrame( - Seq( + val expected = Seq( (1, "foo", 4, Vectors.dense(0.0, 1.0, 4.0), 1.0), (2, "bar", 4, Vectors.dense(1.0, 0.0, 4.0), 2.0), (3, "bar", 5, Vectors.dense(1.0, 0.0, 5.0), 3.0), - (4, "baz", 5, Vectors.dense(0.0, 0.0, 5.0), 4.0)) + (4, "baz", 5, Vectors.dense(0.0, 0.0, 5.0), 4.0) ).toDF("id", "a", "b", "features", "label") assert(result.schema.toString == resultSchema.toString) assert(result.collect() === expected.collect()) } + test("encodes string terms with string indexer order type") { + val formula = new RFormula().setFormula("id ~ a + b") + val original = Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "aaz", 5)) + .toDF("id", "a", "b") + + val expected = Seq( + Seq( + (1, "foo", 4, Vectors.dense(0.0, 0.0, 4.0), 1.0), + (2, "bar", 4, Vectors.dense(1.0, 0.0, 4.0), 2.0), + (3, "bar", 5, Vectors.dense(1.0, 0.0, 5.0), 3.0), + (4, "aaz", 5, Vectors.dense(0.0, 1.0, 5.0), 4.0) + ).toDF("id", "a", "b", "features", "label"), + Seq( + (1, "foo", 4, Vectors.dense(0.0, 1.0, 4.0), 1.0), + (2, "bar", 4, Vectors.dense(0.0, 0.0, 4.0), 2.0), + (3, "bar", 5, Vectors.dense(0.0, 0.0, 5.0), 3.0), + (4, "aaz", 5, Vectors.dense(1.0, 0.0, 5.0), 4.0) + ).toDF("id", "a", "b", "features", "label"), + Seq( + (1, "foo", 4, Vectors.dense(1.0, 0.0, 4.0), 1.0), + (2, "bar", 4, Vectors.dense(0.0, 1.0, 4.0), 2.0), + (3, "bar", 5, Vectors.dense(0.0, 1.0, 5.0), 3.0), + (4, "aaz", 5, Vectors.dense(0.0, 0.0, 5.0), 4.0) + ).toDF("id", "a", "b", "features", "label"), + Seq( + (1, "foo", 4, Vectors.dense(0.0, 0.0, 4.0), 1.0), + (2, "bar", 4, Vectors.dense(0.0, 1.0, 4.0), 2.0), + (3, "bar", 5, Vectors.dense(0.0, 1.0, 5.0), 3.0), + (4, "aaz", 5, Vectors.dense(1.0, 0.0, 5.0), 4.0) + ).toDF("id", "a", "b", "features", "label") + ) + + var idx = 0 + for (orderType <- StringIndexer.supportedStringOrderType) { + val model = formula.setStringIndexerOrderType(orderType).fit(original) + val result = model.transform(original) + val resultSchema = model.transformSchema(original.schema) + assert(result.schema.toString == resultSchema.toString) + assert(result.collect() === expected(idx).collect()) + idx += 1 + } + } + + test("test consistency with R when encoding string terms") { + /* + R code: + + df <- data.frame(id = c(1, 2, 3, 4), + a = c("foo", "bar", "bar", "aaz"), + b = c(4, 4, 5, 5)) + model.matrix(id ~ a + b, df)[, -1] + + abar afoo b + 0 1 4 + 1 0 4 + 1 0 5 + 0 0 5 + */ + val original = Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "aaz", 5)) + .toDF("id", "a", "b") + val formula = new RFormula().setFormula("id ~ a + b") + .setStringIndexerOrderType(StringIndexer.alphabetDesc) + + /* + Note that the category dropped after encoding is the same between R and Spark + (i.e., "aaz" is treated as the reference level). + However, the column order is still different: + R renders the columns in ascending alphabetical order ("bar", "foo"), while + RFormula renders the columns in descending alphabetical order ("foo", "bar"). + */ + val expected = Seq( + (1, "foo", 4, Vectors.dense(1.0, 0.0, 4.0), 1.0), + (2, "bar", 4, Vectors.dense(0.0, 1.0, 4.0), 2.0), + (3, "bar", 5, Vectors.dense(0.0, 1.0, 5.0), 3.0), + (4, "aaz", 5, Vectors.dense(0.0, 0.0, 5.0), 4.0) + ).toDF("id", "a", "b", "features", "label") + + val model = formula.fit(original) + val result = model.transform(original) + val resultSchema = model.transformSchema(original.schema) + assert(result.schema.toString == resultSchema.toString) + assert(result.collect() === expected.collect()) + } + + test("formula w/o intercept, we should output reference category when encoding string terms") { + /* + R code: + + df <- data.frame(id = c(1, 2, 3, 4), + a = c("foo", "bar", "bar", "baz"), + b = c("zq", "zz", "zz", "zz"), + c = c(4, 4, 5, 5)) + model.matrix(id ~ a + b + c - 1, df) + + abar abaz afoo bzz c + 1 0 0 1 0 4 + 2 1 0 0 1 4 + 3 1 0 0 1 5 + 4 0 1 0 1 5 + + model.matrix(id ~ a:b + c - 1, df) + + c abar:bzq abaz:bzq afoo:bzq abar:bzz abaz:bzz afoo:bzz + 1 4 0 0 1 0 0 0 + 2 4 0 0 0 1 0 0 + 3 5 0 0 0 1 0 0 + 4 5 0 0 0 0 1 0 + */ + val original = Seq((1, "foo", "zq", 4), (2, "bar", "zz", 4), (3, "bar", "zz", 5), + (4, "baz", "zz", 5)).toDF("id", "a", "b", "c") + + val formula1 = new RFormula().setFormula("id ~ a + b + c - 1") + .setStringIndexerOrderType(StringIndexer.alphabetDesc) + val model1 = formula1.fit(original) + val result1 = model1.transform(original) + val resultSchema1 = model1.transformSchema(original.schema) + // Note the column order is different between R and Spark. + val expected1 = Seq( + (1, "foo", "zq", 4, Vectors.sparse(5, Array(0, 4), Array(1.0, 4.0)), 1.0), + (2, "bar", "zz", 4, Vectors.dense(0.0, 0.0, 1.0, 1.0, 4.0), 2.0), + (3, "bar", "zz", 5, Vectors.dense(0.0, 0.0, 1.0, 1.0, 5.0), 3.0), + (4, "baz", "zz", 5, Vectors.dense(0.0, 1.0, 0.0, 1.0, 5.0), 4.0) + ).toDF("id", "a", "b", "c", "features", "label") + assert(result1.schema.toString == resultSchema1.toString) + assert(result1.collect() === expected1.collect()) + + val attrs1 = AttributeGroup.fromStructField(result1.schema("features")) + val expectedAttrs1 = new AttributeGroup( + "features", + Array[Attribute]( + new BinaryAttribute(Some("a_foo"), Some(1)), + new BinaryAttribute(Some("a_baz"), Some(2)), + new BinaryAttribute(Some("a_bar"), Some(3)), + new BinaryAttribute(Some("b_zz"), Some(4)), + new NumericAttribute(Some("c"), Some(5)))) + assert(attrs1 === expectedAttrs1) + + // There is no impact for string terms interaction. + val formula2 = new RFormula().setFormula("id ~ a:b + c - 1") + .setStringIndexerOrderType(StringIndexer.alphabetDesc) + val model2 = formula2.fit(original) + val result2 = model2.transform(original) + val resultSchema2 = model2.transformSchema(original.schema) + // Note the column order is different between R and Spark. + val expected2 = Seq( + (1, "foo", "zq", 4, Vectors.sparse(7, Array(1, 6), Array(1.0, 4.0)), 1.0), + (2, "bar", "zz", 4, Vectors.sparse(7, Array(4, 6), Array(1.0, 4.0)), 2.0), + (3, "bar", "zz", 5, Vectors.sparse(7, Array(4, 6), Array(1.0, 5.0)), 3.0), + (4, "baz", "zz", 5, Vectors.sparse(7, Array(2, 6), Array(1.0, 5.0)), 4.0) + ).toDF("id", "a", "b", "c", "features", "label") + assert(result2.schema.toString == resultSchema2.toString) + assert(result2.collect() === expected2.collect()) + + val attrs2 = AttributeGroup.fromStructField(result2.schema("features")) + val expectedAttrs2 = new AttributeGroup( + "features", + Array[Attribute]( + new NumericAttribute(Some("a_foo:b_zz"), Some(1)), + new NumericAttribute(Some("a_foo:b_zq"), Some(2)), + new NumericAttribute(Some("a_baz:b_zz"), Some(3)), + new NumericAttribute(Some("a_baz:b_zq"), Some(4)), + new NumericAttribute(Some("a_bar:b_zz"), Some(5)), + new NumericAttribute(Some("a_bar:b_zq"), Some(6)), + new NumericAttribute(Some("c"), Some(7)))) + assert(attrs2 === expectedAttrs2) + } + test("index string label") { val formula = new RFormula().setFormula("id ~ a + b") - val original = sqlContext.createDataFrame( + val original = Seq(("male", "foo", 4), ("female", "bar", 4), ("female", "bar", 5), ("male", "baz", 5)) - ).toDF("id", "a", "b") + .toDF("id", "a", "b") val model = formula.fit(original) val result = model.transform(original) - val resultSchema = model.transformSchema(original.schema) - val expected = sqlContext.createDataFrame( - Seq( + val expected = Seq( ("male", "foo", 4, Vectors.dense(0.0, 1.0, 4.0), 1.0), ("female", "bar", 4, Vectors.dense(1.0, 0.0, 4.0), 0.0), ("female", "bar", 5, Vectors.dense(1.0, 0.0, 5.0), 0.0), - ("male", "baz", 5, Vectors.dense(0.0, 0.0, 5.0), 1.0)) + ("male", "baz", 5, Vectors.dense(0.0, 0.0, 5.0), 1.0) ).toDF("id", "a", "b", "features", "label") // assert(result.schema.toString == resultSchema.toString) assert(result.collect() === expected.collect()) } + test("force to index label even it is numeric type") { + val formula = new RFormula().setFormula("id ~ a + b").setForceIndexLabel(true) + val original = spark.createDataFrame( + Seq((1.0, "foo", 4), (1.0, "bar", 4), (0.0, "bar", 5), (1.0, "baz", 5)) + ).toDF("id", "a", "b") + val model = formula.fit(original) + val result = model.transform(original) + val expected = spark.createDataFrame( + Seq( + (1.0, "foo", 4, Vectors.dense(0.0, 1.0, 4.0), 0.0), + (1.0, "bar", 4, Vectors.dense(1.0, 0.0, 4.0), 0.0), + (0.0, "bar", 5, Vectors.dense(1.0, 0.0, 5.0), 1.0), + (1.0, "baz", 5, Vectors.dense(0.0, 0.0, 5.0), 0.0)) + ).toDF("id", "a", "b", "features", "label") + assert(result.collect() === expected.collect()) + } + test("attribute generation") { val formula = new RFormula().setFormula("id ~ a + b") - val original = sqlContext.createDataFrame( - Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "baz", 5)) - ).toDF("id", "a", "b") + val original = Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "baz", 5)) + .toDF("id", "a", "b") val model = formula.fit(original) val result = model.transform(original) val attrs = AttributeGroup.fromStructField(result.schema("features")) @@ -143,18 +346,51 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext { assert(attrs === expectedAttrs) } + test("vector attribute generation") { + val formula = new RFormula().setFormula("id ~ vec") + val original = Seq((1, Vectors.dense(0.0, 1.0)), (2, Vectors.dense(1.0, 2.0))) + .toDF("id", "vec") + val model = formula.fit(original) + val result = model.transform(original) + val attrs = AttributeGroup.fromStructField(result.schema("features")) + val expectedAttrs = new AttributeGroup( + "features", + Array[Attribute]( + new NumericAttribute(Some("vec_0"), Some(1)), + new NumericAttribute(Some("vec_1"), Some(2)))) + assert(attrs === expectedAttrs) + } + + test("vector attribute generation with unnamed input attrs") { + val formula = new RFormula().setFormula("id ~ vec2") + val base = Seq((1, Vectors.dense(0.0, 1.0)), (2, Vectors.dense(1.0, 2.0))) + .toDF("id", "vec") + val metadata = new AttributeGroup( + "vec2", + Array[Attribute]( + NumericAttribute.defaultAttr, + NumericAttribute.defaultAttr)).toMetadata() + val original = base.select(base.col("id"), base.col("vec").as("vec2", metadata)) + val model = formula.fit(original) + val result = model.transform(original) + val attrs = AttributeGroup.fromStructField(result.schema("features")) + val expectedAttrs = new AttributeGroup( + "features", + Array[Attribute]( + new NumericAttribute(Some("vec2_0"), Some(1)), + new NumericAttribute(Some("vec2_1"), Some(2)))) + assert(attrs === expectedAttrs) + } + test("numeric interaction") { val formula = new RFormula().setFormula("a ~ b:c:d") - val original = sqlContext.createDataFrame( - Seq((1, 2, 4, 2), (2, 3, 4, 1)) - ).toDF("a", "b", "c", "d") + val original = Seq((1, 2, 4, 2), (2, 3, 4, 1)).toDF("a", "b", "c", "d") val model = formula.fit(original) val result = model.transform(original) - val expected = sqlContext.createDataFrame( - Seq( - (1, 2, 4, 2, Vectors.dense(16.0), 1.0), - (2, 3, 4, 1, Vectors.dense(12.0), 2.0)) - ).toDF("a", "b", "c", "d", "features", "label") + val expected = Seq( + (1, 2, 4, 2, Vectors.dense(16.0), 1.0), + (2, 3, 4, 1, Vectors.dense(12.0), 2.0) + ).toDF("a", "b", "c", "d", "features", "label") assert(result.collect() === expected.collect()) val attrs = AttributeGroup.fromStructField(result.schema("features")) val expectedAttrs = new AttributeGroup( @@ -165,20 +401,19 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext { test("factor numeric interaction") { val formula = new RFormula().setFormula("id ~ a:b") - val original = sqlContext.createDataFrame( + val original = Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "baz", 5), (4, "baz", 5), (4, "baz", 5)) - ).toDF("id", "a", "b") + .toDF("id", "a", "b") val model = formula.fit(original) val result = model.transform(original) - val expected = sqlContext.createDataFrame( - Seq( - (1, "foo", 4, Vectors.dense(0.0, 0.0, 4.0), 1.0), - (2, "bar", 4, Vectors.dense(0.0, 4.0, 0.0), 2.0), - (3, "bar", 5, Vectors.dense(0.0, 5.0, 0.0), 3.0), - (4, "baz", 5, Vectors.dense(5.0, 0.0, 0.0), 4.0), - (4, "baz", 5, Vectors.dense(5.0, 0.0, 0.0), 4.0), - (4, "baz", 5, Vectors.dense(5.0, 0.0, 0.0), 4.0)) - ).toDF("id", "a", "b", "features", "label") + val expected = Seq( + (1, "foo", 4, Vectors.dense(0.0, 0.0, 4.0), 1.0), + (2, "bar", 4, Vectors.dense(0.0, 4.0, 0.0), 2.0), + (3, "bar", 5, Vectors.dense(0.0, 5.0, 0.0), 3.0), + (4, "baz", 5, Vectors.dense(5.0, 0.0, 0.0), 4.0), + (4, "baz", 5, Vectors.dense(5.0, 0.0, 0.0), 4.0), + (4, "baz", 5, Vectors.dense(5.0, 0.0, 0.0), 4.0) + ).toDF("id", "a", "b", "features", "label") assert(result.collect() === expected.collect()) val attrs = AttributeGroup.fromStructField(result.schema("features")) val expectedAttrs = new AttributeGroup( @@ -192,17 +427,15 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext { test("factor factor interaction") { val formula = new RFormula().setFormula("id ~ a:b") - val original = sqlContext.createDataFrame( - Seq((1, "foo", "zq"), (2, "bar", "zq"), (3, "bar", "zz")) - ).toDF("id", "a", "b") + val original = + Seq((1, "foo", "zq"), (2, "bar", "zq"), (3, "bar", "zz")).toDF("id", "a", "b") val model = formula.fit(original) val result = model.transform(original) - val expected = sqlContext.createDataFrame( - Seq( - (1, "foo", "zq", Vectors.dense(0.0, 0.0, 1.0, 0.0), 1.0), - (2, "bar", "zq", Vectors.dense(1.0, 0.0, 0.0, 0.0), 2.0), - (3, "bar", "zz", Vectors.dense(0.0, 1.0, 0.0, 0.0), 3.0)) - ).toDF("id", "a", "b", "features", "label") + val expected = Seq( + (1, "foo", "zq", Vectors.dense(0.0, 0.0, 1.0, 0.0), 1.0), + (2, "bar", "zq", Vectors.dense(1.0, 0.0, 0.0, 0.0), 2.0), + (3, "bar", "zz", Vectors.dense(0.0, 1.0, 0.0, 0.0), 3.0) + ).toDF("id", "a", "b", "features", "label") assert(result.collect() === expected.collect()) val attrs = AttributeGroup.fromStructField(result.schema("features")) val expectedAttrs = new AttributeGroup( @@ -214,4 +447,105 @@ class RFormulaSuite extends SparkFunSuite with MLlibTestSparkContext { new NumericAttribute(Some("a_foo:b_zz"), Some(4)))) assert(attrs === expectedAttrs) } + + test("read/write: RFormula") { + val rFormula = new RFormula() + .setFormula("id ~ a:b") + .setFeaturesCol("myFeatures") + .setLabelCol("myLabels") + + testDefaultReadWrite(rFormula) + } + + test("read/write: RFormulaModel") { + def checkModelData(model: RFormulaModel, model2: RFormulaModel): Unit = { + assert(model.uid === model2.uid) + + assert(model.resolvedFormula.label === model2.resolvedFormula.label) + assert(model.resolvedFormula.terms === model2.resolvedFormula.terms) + assert(model.resolvedFormula.hasIntercept === model2.resolvedFormula.hasIntercept) + + assert(model.pipelineModel.uid === model2.pipelineModel.uid) + + model.pipelineModel.stages.zip(model2.pipelineModel.stages).foreach { + case (transformer1, transformer2) => + assert(transformer1.uid === transformer2.uid) + assert(transformer1.params === transformer2.params) + } + } + + val dataset = Seq((1, "foo", "zq"), (2, "bar", "zq"), (3, "bar", "zz")).toDF("id", "a", "b") + + val rFormula = new RFormula().setFormula("id ~ a:b") + + val model = rFormula.fit(dataset) + val newModel = testDefaultReadWrite(model) + checkModelData(model, newModel) + } + + test("should support all NumericType labels") { + val formula = new RFormula().setFormula("label ~ features") + .setLabelCol("x") + .setFeaturesCol("y") + val dfs = MLTestingUtils.genRegressionDFWithNumericLabelCol(spark) + val expected = formula.fit(dfs(DoubleType)) + val actuals = dfs.keys.filter(_ != DoubleType).map(t => formula.fit(dfs(t))) + actuals.foreach { actual => + assert(expected.pipelineModel.stages.length === actual.pipelineModel.stages.length) + expected.pipelineModel.stages.zip(actual.pipelineModel.stages).foreach { + case (exTransformer, acTransformer) => + assert(exTransformer.params === acTransformer.params) + } + assert(expected.resolvedFormula.label === actual.resolvedFormula.label) + assert(expected.resolvedFormula.terms === actual.resolvedFormula.terms) + assert(expected.resolvedFormula.hasIntercept === actual.resolvedFormula.hasIntercept) + } + } + + test("handle unseen features or labels") { + val df1 = Seq((1, "foo", "zq"), (2, "bar", "zq"), (3, "bar", "zz")).toDF("id", "a", "b") + val df2 = Seq((1, "foo", "zq"), (2, "bar", "zq"), (3, "bar", "zy")).toDF("id", "a", "b") + + // Handle unseen features. + val formula1 = new RFormula().setFormula("id ~ a + b") + intercept[SparkException] { + formula1.fit(df1).transform(df2).collect() + } + val result1 = formula1.setHandleInvalid("skip").fit(df1).transform(df2) + val result2 = formula1.setHandleInvalid("keep").fit(df1).transform(df2) + + val expected1 = Seq( + (1, "foo", "zq", Vectors.dense(0.0, 1.0), 1.0), + (2, "bar", "zq", Vectors.dense(1.0, 1.0), 2.0) + ).toDF("id", "a", "b", "features", "label") + val expected2 = Seq( + (1, "foo", "zq", Vectors.dense(0.0, 1.0, 1.0, 0.0), 1.0), + (2, "bar", "zq", Vectors.dense(1.0, 0.0, 1.0, 0.0), 2.0), + (3, "bar", "zy", Vectors.dense(1.0, 0.0, 0.0, 0.0), 3.0) + ).toDF("id", "a", "b", "features", "label") + + assert(result1.collect() === expected1.collect()) + assert(result2.collect() === expected2.collect()) + + // Handle unseen labels. + val formula2 = new RFormula().setFormula("b ~ a + id") + intercept[SparkException] { + formula2.fit(df1).transform(df2).collect() + } + val result3 = formula2.setHandleInvalid("skip").fit(df1).transform(df2) + val result4 = formula2.setHandleInvalid("keep").fit(df1).transform(df2) + + val expected3 = Seq( + (1, "foo", "zq", Vectors.dense(0.0, 1.0), 0.0), + (2, "bar", "zq", Vectors.dense(1.0, 2.0), 0.0) + ).toDF("id", "a", "b", "features", "label") + val expected4 = Seq( + (1, "foo", "zq", Vectors.dense(0.0, 1.0, 1.0), 0.0), + (2, "bar", "zq", Vectors.dense(1.0, 0.0, 2.0), 0.0), + (3, "bar", "zy", Vectors.dense(1.0, 0.0, 3.0), 2.0) + ).toDF("id", "a", "b", "features", "label") + + assert(result3.collect() === expected3.collect()) + assert(result4.collect() === expected4.collect()) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala index d19052881ae4..753f890c4830 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/SQLTransformerSuite.scala @@ -19,26 +19,45 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.types.{LongType, StructField, StructType} -class SQLTransformerSuite extends SparkFunSuite with MLlibTestSparkContext { +class SQLTransformerSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ test("params") { ParamsSuite.checkParams(new SQLTransformer()) } test("transform numeric data") { - val original = sqlContext.createDataFrame( - Seq((0, 1.0, 3.0), (2, 2.0, 5.0))).toDF("id", "v1", "v2") + val original = Seq((0, 1.0, 3.0), (2, 2.0, 5.0)).toDF("id", "v1", "v2") val sqlTrans = new SQLTransformer().setStatement( "SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__") val result = sqlTrans.transform(original) val resultSchema = sqlTrans.transformSchema(original.schema) - val expected = sqlContext.createDataFrame( - Seq((0, 1.0, 3.0, 4.0, 3.0), (2, 2.0, 5.0, 7.0, 10.0))) + val expected = Seq((0, 1.0, 3.0, 4.0, 3.0), (2, 2.0, 5.0, 7.0, 10.0)) .toDF("id", "v1", "v2", "v3", "v4") assert(result.schema.toString == resultSchema.toString) assert(resultSchema == expected.schema) assert(result.collect().toSeq == expected.collect().toSeq) + assert(original.sparkSession.catalog.listTables().count() == 0) + } + + test("read/write") { + val t = new SQLTransformer() + .setStatement("select * from __THIS__") + testDefaultReadWrite(t) + } + + test("transformSchema") { + val df = spark.range(10) + val outputSchema = new SQLTransformer() + .setStatement("SELECT id + 1 AS id1 FROM __THIS__") + .transformSchema(df.schema) + val expected = StructType(Seq(StructField("id1", LongType, nullable = false))) + assert(outputSchema === expected) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala new file mode 100644 index 000000000000..350ba44baa1e --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StandardScalerSuite.scala @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{DataFrame, Row} + +class StandardScalerSuite extends SparkFunSuite with MLlibTestSparkContext + with DefaultReadWriteTest { + + import testImplicits._ + + @transient var data: Array[Vector] = _ + @transient var resWithStd: Array[Vector] = _ + @transient var resWithMean: Array[Vector] = _ + @transient var resWithBoth: Array[Vector] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + data = Array( + Vectors.dense(-2.0, 2.3, 0.0), + Vectors.dense(0.0, -5.1, 1.0), + Vectors.dense(1.7, -0.6, 3.3) + ) + resWithMean = Array( + Vectors.dense(-1.9, 3.433333333333, -1.433333333333), + Vectors.dense(0.1, -3.966666666667, -0.433333333333), + Vectors.dense(1.8, 0.533333333333, 1.866666666667) + ) + resWithStd = Array( + Vectors.dense(-1.079898494312, 0.616834091415, 0.0), + Vectors.dense(0.0, -1.367762550529, 0.590968109266), + Vectors.dense(0.917913720165, -0.160913241239, 1.950194760579) + ) + resWithBoth = Array( + Vectors.dense(-1.0259035695965, 0.920781324866, -0.8470542899497), + Vectors.dense(0.0539949247156, -1.063815317078, -0.256086180682), + Vectors.dense(0.9719086448809, 0.143033992212, 1.103140470631) + ) + } + + def assertResult(df: DataFrame): Unit = { + df.select("standardized_features", "expected").collect().foreach { + case Row(vector1: Vector, vector2: Vector) => + assert(vector1 ~== vector2 absTol 1E-5, + "The vector value is not correct after standardization.") + } + } + + test("params") { + ParamsSuite.checkParams(new StandardScaler) + ParamsSuite.checkParams(new StandardScalerModel("empty", + Vectors.dense(1.0), Vectors.dense(2.0))) + } + + test("Standardization with default parameter") { + val df0 = data.zip(resWithStd).toSeq.toDF("features", "expected") + + val standardScalerEst0 = new StandardScaler() + .setInputCol("features") + .setOutputCol("standardized_features") + val standardScaler0 = standardScalerEst0.fit(df0) + MLTestingUtils.checkCopyAndUids(standardScalerEst0, standardScaler0) + + assertResult(standardScaler0.transform(df0)) + } + + test("Standardization with setter") { + val df1 = data.zip(resWithBoth).toSeq.toDF("features", "expected") + val df2 = data.zip(resWithMean).toSeq.toDF("features", "expected") + val df3 = data.zip(data).toSeq.toDF("features", "expected") + + val standardScaler1 = new StandardScaler() + .setInputCol("features") + .setOutputCol("standardized_features") + .setWithMean(true) + .setWithStd(true) + .fit(df1) + + val standardScaler2 = new StandardScaler() + .setInputCol("features") + .setOutputCol("standardized_features") + .setWithMean(true) + .setWithStd(false) + .fit(df2) + + val standardScaler3 = new StandardScaler() + .setInputCol("features") + .setOutputCol("standardized_features") + .setWithMean(false) + .setWithStd(false) + .fit(df3) + + assertResult(standardScaler1.transform(df1)) + assertResult(standardScaler2.transform(df2)) + assertResult(standardScaler3.transform(df3)) + } + + test("sparse data and withMean") { + val someSparseData = Array( + Vectors.sparse(3, Array(0, 1), Array(-2.0, 2.3)), + Vectors.sparse(3, Array(1, 2), Array(-5.1, 1.0)), + Vectors.dense(1.7, -0.6, 3.3) + ) + val df = someSparseData.zip(resWithMean).toSeq.toDF("features", "expected") + val standardScaler = new StandardScaler() + .setInputCol("features") + .setOutputCol("standardized_features") + .setWithMean(true) + .setWithStd(false) + .fit(df) + assertResult(standardScaler.transform(df)) + } + + test("StandardScaler read/write") { + val t = new StandardScaler() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setWithStd(false) + .setWithMean(true) + testDefaultReadWrite(t) + } + + test("StandardScalerModel read/write") { + val instance = new StandardScalerModel("myStandardScalerModel", + Vectors.dense(1.0, 2.0), Vectors.dense(3.0, 4.0)) + val newInstance = testDefaultReadWrite(instance) + assert(newInstance.std === instance.std) + assert(newInstance.mean === instance.mean) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala old mode 100644 new mode 100755 index e0d433f566c2..5262b146b184 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala @@ -18,11 +18,12 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{Dataset, Row} object StopWordsRemoverSuite extends SparkFunSuite { - def testStopWordsRemover(t: StopWordsRemover, dataset: DataFrame): Unit = { + def testStopWordsRemover(t: StopWordsRemover, dataset: Dataset[_]): Unit = { t.transform(dataset) .select("filtered", "expected") .collect() @@ -32,21 +33,42 @@ object StopWordsRemoverSuite extends SparkFunSuite { } } -class StopWordsRemoverSuite extends SparkFunSuite with MLlibTestSparkContext { +class StopWordsRemoverSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + import StopWordsRemoverSuite._ + import testImplicits._ test("StopWordsRemover default") { val remover = new StopWordsRemover() .setInputCol("raw") .setOutputCol("filtered") - val dataSet = sqlContext.createDataFrame(Seq( + val dataSet = Seq( (Seq("test", "test"), Seq("test", "test")), (Seq("a", "b", "c", "d"), Seq("b", "c", "d")), (Seq("a", "the", "an"), Seq()), (Seq("A", "The", "AN"), Seq()), (Seq(null), Seq(null)), (Seq(), Seq()) - )).toDF("raw", "expected") + ).toDF("raw", "expected") + + testStopWordsRemover(remover, dataSet) + } + + test("StopWordsRemover with particular stop words list") { + val stopWords = Array("test", "a", "an", "the") + val remover = new StopWordsRemover() + .setInputCol("raw") + .setOutputCol("filtered") + .setStopWords(stopWords) + val dataSet = Seq( + (Seq("test", "test"), Seq()), + (Seq("a", "b", "c", "d"), Seq("b", "c", "d")), + (Seq("a", "the", "an"), Seq()), + (Seq("A", "The", "AN"), Seq()), + (Seq(null), Seq(null)), + (Seq(), Seq()) + ).toDF("raw", "expected") testStopWordsRemover(remover, dataSet) } @@ -56,25 +78,82 @@ class StopWordsRemoverSuite extends SparkFunSuite with MLlibTestSparkContext { .setInputCol("raw") .setOutputCol("filtered") .setCaseSensitive(true) - val dataSet = sqlContext.createDataFrame(Seq( + val dataSet = Seq( (Seq("A"), Seq("A")), (Seq("The", "the"), Seq("The")) - )).toDF("raw", "expected") + ).toDF("raw", "expected") testStopWordsRemover(remover, dataSet) } - test("StopWordsRemover with additional words") { - val stopWords = StopWords.English ++ Array("python", "scala") + test("default stop words of supported languages are not empty") { + StopWordsRemover.supportedLanguages.foreach { lang => + assert(StopWordsRemover.loadDefaultStopWords(lang).nonEmpty, + s"The default stop words of $lang cannot be empty.") + } + } + + test("StopWordsRemover with language selection") { + val stopWords = StopWordsRemover.loadDefaultStopWords("turkish") val remover = new StopWordsRemover() .setInputCol("raw") .setOutputCol("filtered") .setStopWords(stopWords) - val dataSet = sqlContext.createDataFrame(Seq( + val dataSet = Seq( + (Seq("acaba", "ama", "biri"), Seq()), + (Seq("hep", "her", "scala"), Seq("scala")) + ).toDF("raw", "expected") + + testStopWordsRemover(remover, dataSet) + } + + test("StopWordsRemover with ignored words") { + val stopWords = StopWordsRemover.loadDefaultStopWords("english").toSet -- Set("a") + val remover = new StopWordsRemover() + .setInputCol("raw") + .setOutputCol("filtered") + .setStopWords(stopWords.toArray) + val dataSet = Seq( + (Seq("python", "scala", "a"), Seq("python", "scala", "a")), + (Seq("Python", "Scala", "swift"), Seq("Python", "Scala", "swift")) + ).toDF("raw", "expected") + + testStopWordsRemover(remover, dataSet) + } + + test("StopWordsRemover with additional words") { + val stopWords = StopWordsRemover.loadDefaultStopWords("english").toSet ++ Set("python", "scala") + val remover = new StopWordsRemover() + .setInputCol("raw") + .setOutputCol("filtered") + .setStopWords(stopWords.toArray) + val dataSet = Seq( (Seq("python", "scala", "a"), Seq()), (Seq("Python", "Scala", "swift"), Seq("swift")) - )).toDF("raw", "expected") + ).toDF("raw", "expected") testStopWordsRemover(remover, dataSet) } + + test("read/write") { + val t = new StopWordsRemover() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setStopWords(Array("the", "a")) + .setCaseSensitive(true) + testDefaultReadWrite(t) + } + + test("StopWordsRemover output column already exists") { + val outputCol = "expected" + val remover = new StopWordsRemover() + .setInputCol("raw") + .setOutputCol(outputCol) + val dataSet = Seq((Seq("The", "the", "swift"), Seq("swift"))).toDF("raw", outputCol) + + val thrown = intercept[IllegalArgumentException] { + testStopWordsRemover(remover, dataSet) + } + assert(thrown.getMessage == s"requirement failed: Column $outputCol already exists.") + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala index ddcdb5f4212b..027b1fbc6657 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala @@ -17,16 +17,19 @@ package org.apache.spark.ml.feature -import org.apache.spark.sql.types.{StringType, StructType, StructField, DoubleType} import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.attribute.{Attribute, NominalAttribute} import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.util.MLTestingUtils +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Row import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types.{DoubleType, StringType, StructField, StructType} -class StringIndexerSuite extends SparkFunSuite with MLlibTestSparkContext { +class StringIndexerSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ test("params") { ParamsSuite.checkParams(new StringIndexer) @@ -37,21 +40,20 @@ class StringIndexerSuite extends SparkFunSuite with MLlibTestSparkContext { } test("StringIndexer") { - val data = sc.parallelize(Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")), 2) - val df = sqlContext.createDataFrame(data).toDF("id", "label") + val data = Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")) + val df = data.toDF("id", "label") val indexer = new StringIndexer() .setInputCol("label") .setOutputCol("labelIndex") - .fit(df) + val indexerModel = indexer.fit(df) - // copied model must have the same parent. - MLTestingUtils.checkCopy(indexer) + MLTestingUtils.checkCopyAndUids(indexer, indexerModel) - val transformed = indexer.transform(df) + val transformed = indexerModel.transform(df) val attr = Attribute.fromStructField(transformed.schema("labelIndex")) .asInstanceOf[NominalAttribute] assert(attr.values.get === Array("a", "c", "b")) - val output = transformed.select("id", "labelIndex").map { r => + val output = transformed.select("id", "labelIndex").rdd.map { r => (r.getInt(0), r.getDouble(1)) }.collect().toSet // a -> 0, b -> 2, c -> 1 @@ -60,10 +62,10 @@ class StringIndexerSuite extends SparkFunSuite with MLlibTestSparkContext { } test("StringIndexerUnseen") { - val data = sc.parallelize(Seq((0, "a"), (1, "b"), (4, "b")), 2) - val data2 = sc.parallelize(Seq((0, "a"), (1, "b"), (2, "c")), 2) - val df = sqlContext.createDataFrame(data).toDF("id", "label") - val df2 = sqlContext.createDataFrame(data2).toDF("id", "label") + val data = Seq((0, "a"), (1, "b"), (4, "b")) + val data2 = Seq((0, "a"), (1, "b"), (2, "c"), (3, "d")) + val df = data.toDF("id", "label") + val df2 = data2.toDF("id", "label") val indexer = new StringIndexer() .setInputCol("label") .setOutputCol("labelIndex") @@ -72,27 +74,37 @@ class StringIndexerSuite extends SparkFunSuite with MLlibTestSparkContext { intercept[SparkException] { indexer.transform(df2).collect() } - val indexerSkipInvalid = new StringIndexer() - .setInputCol("label") - .setOutputCol("labelIndex") - .setHandleInvalid("skip") - .fit(df) + + indexer.setHandleInvalid("skip") // Verify that we skip the c record - val transformed = indexerSkipInvalid.transform(df2) - val attr = Attribute.fromStructField(transformed.schema("labelIndex")) + val transformedSkip = indexer.transform(df2) + val attrSkip = Attribute.fromStructField(transformedSkip.schema("labelIndex")) .asInstanceOf[NominalAttribute] - assert(attr.values.get === Array("b", "a")) - val output = transformed.select("id", "labelIndex").map { r => + assert(attrSkip.values.get === Array("b", "a")) + val outputSkip = transformedSkip.select("id", "labelIndex").rdd.map { r => (r.getInt(0), r.getDouble(1)) }.collect().toSet // a -> 1, b -> 0 - val expected = Set((0, 1.0), (1, 0.0)) - assert(output === expected) + val expectedSkip = Set((0, 1.0), (1, 0.0)) + assert(outputSkip === expectedSkip) + + indexer.setHandleInvalid("keep") + // Verify that we keep the unseen records + val transformedKeep = indexer.transform(df2) + val attrKeep = Attribute.fromStructField(transformedKeep.schema("labelIndex")) + .asInstanceOf[NominalAttribute] + assert(attrKeep.values.get === Array("b", "a", "__unknown")) + val outputKeep = transformedKeep.select("id", "labelIndex").rdd.map { r => + (r.getInt(0), r.getDouble(1)) + }.collect().toSet + // a -> 1, b -> 0, c -> 2, d -> 3 + val expectedKeep = Set((0, 1.0), (1, 0.0), (2, 2.0), (3, 2.0)) + assert(outputKeep === expectedKeep) } test("StringIndexer with a numeric input column") { - val data = sc.parallelize(Seq((0, 100), (1, 200), (2, 300), (3, 100), (4, 100), (5, 300)), 2) - val df = sqlContext.createDataFrame(data).toDF("id", "label") + val data = Seq((0, 100), (1, 200), (2, 300), (3, 100), (4, 100), (5, 300)) + val df = data.toDF("id", "label") val indexer = new StringIndexer() .setInputCol("label") .setOutputCol("labelIndex") @@ -101,7 +113,7 @@ class StringIndexerSuite extends SparkFunSuite with MLlibTestSparkContext { val attr = Attribute.fromStructField(transformed.schema("labelIndex")) .asInstanceOf[NominalAttribute] assert(attr.values.get === Array("100", "300", "200")) - val output = transformed.select("id", "labelIndex").map { r => + val output = transformed.select("id", "labelIndex").rdd.map { r => (r.getInt(0), r.getDouble(1)) }.collect().toSet // 100 -> 0, 200 -> 2, 300 -> 1 @@ -109,12 +121,93 @@ class StringIndexerSuite extends SparkFunSuite with MLlibTestSparkContext { assert(output === expected) } + test("StringIndexer with NULLs") { + val data: Seq[(Int, String)] = Seq((0, "a"), (1, "b"), (2, "b"), (3, null)) + val data2: Seq[(Int, String)] = Seq((0, "a"), (1, "b"), (3, null)) + val df = data.toDF("id", "label") + val df2 = data2.toDF("id", "label") + + val indexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("labelIndex") + + withClue("StringIndexer should throw error when setHandleInvalid=error " + + "when given NULL values") { + intercept[SparkException] { + indexer.setHandleInvalid("error") + indexer.fit(df).transform(df2).collect() + } + } + + indexer.setHandleInvalid("skip") + val transformedSkip = indexer.fit(df).transform(df2) + val attrSkip = Attribute + .fromStructField(transformedSkip.schema("labelIndex")) + .asInstanceOf[NominalAttribute] + assert(attrSkip.values.get === Array("b", "a")) + val outputSkip = transformedSkip.select("id", "labelIndex").rdd.map { r => + (r.getInt(0), r.getDouble(1)) + }.collect().toSet + // a -> 1, b -> 0 + val expectedSkip = Set((0, 1.0), (1, 0.0)) + assert(outputSkip === expectedSkip) + + indexer.setHandleInvalid("keep") + val transformedKeep = indexer.fit(df).transform(df2) + val attrKeep = Attribute + .fromStructField(transformedKeep.schema("labelIndex")) + .asInstanceOf[NominalAttribute] + assert(attrKeep.values.get === Array("b", "a", "__unknown")) + val outputKeep = transformedKeep.select("id", "labelIndex").rdd.map { r => + (r.getInt(0), r.getDouble(1)) + }.collect().toSet + // a -> 1, b -> 0, null -> 2 + val expectedKeep = Set((0, 1.0), (1, 0.0), (3, 2.0)) + assert(outputKeep === expectedKeep) + } + test("StringIndexerModel should keep silent if the input column does not exist.") { val indexerModel = new StringIndexerModel("indexer", Array("a", "b", "c")) .setInputCol("label") .setOutputCol("labelIndex") - val df = sqlContext.range(0L, 10L) - assert(indexerModel.transform(df).eq(df)) + val df = spark.range(0L, 10L).toDF() + assert(indexerModel.transform(df).collect().toSet === df.collect().toSet) + } + + test("StringIndexerModel can't overwrite output column") { + val df = Seq((1, 2), (3, 4)).toDF("input", "output") + intercept[IllegalArgumentException] { + new StringIndexer() + .setInputCol("input") + .setOutputCol("output") + .fit(df) + } + + val indexer = new StringIndexer() + .setInputCol("input") + .setOutputCol("indexedInput") + .fit(df) + + intercept[IllegalArgumentException] { + indexer.setOutputCol("output").transform(df) + } + } + + test("StringIndexer read/write") { + val t = new StringIndexer() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setHandleInvalid("skip") + testDefaultReadWrite(t) + } + + test("StringIndexerModel read/write") { + val instance = new StringIndexerModel("myStringIndexerModel", Array("a", "b", "c")) + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setHandleInvalid("skip") + val newInstance = testDefaultReadWrite(instance) + assert(newInstance.labels === instance.labels) } test("IndexToString params") { @@ -124,9 +217,7 @@ class StringIndexerSuite extends SparkFunSuite with MLlibTestSparkContext { test("IndexToString.transform") { val labels = Array("a", "b", "c") - val df0 = sqlContext.createDataFrame(Seq( - (0, "a"), (1, "b"), (2, "c"), (0, "a") - )).toDF("index", "expected") + val df0 = Seq((0, "a"), (1, "b"), (2, "c"), (0, "a")).toDF("index", "expected") val idxToStr0 = new IndexToString() .setInputCol("index") @@ -150,8 +241,8 @@ class StringIndexerSuite extends SparkFunSuite with MLlibTestSparkContext { } test("StringIndexer, IndexToString are inverses") { - val data = sc.parallelize(Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")), 2) - val df = sqlContext.createDataFrame(data).toDF("id", "label") + val data = Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")) + val df = data.toDF("id", "label") val indexer = new StringIndexer() .setInputCol("label") .setOutputCol("labelIndex") @@ -173,4 +264,54 @@ class StringIndexerSuite extends SparkFunSuite with MLlibTestSparkContext { val outSchema = idxToStr.transformSchema(inSchema) assert(outSchema("output").dataType === StringType) } + + test("IndexToString read/write") { + val t = new IndexToString() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setLabels(Array("a", "b", "c")) + testDefaultReadWrite(t) + } + + test("SPARK 18698: construct IndexToString with custom uid") { + val uid = "customUID" + val t = new IndexToString(uid) + assert(t.uid == uid) + } + + test("StringIndexer metadata") { + val data = Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")) + val df = data.toDF("id", "label") + val indexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("labelIndex") + .fit(df) + val transformed = indexer.transform(df) + val attrs = + NominalAttribute.decodeStructField(transformed.schema("labelIndex"), preserveName = true) + assert(attrs.name.nonEmpty && attrs.name.get === "labelIndex") + } + + test("StringIndexer order types") { + val data = Seq((0, "b"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "b")) + val df = data.toDF("id", "label") + val indexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("labelIndex") + + val expected = Seq(Set((0, 0.0), (1, 0.0), (2, 2.0), (3, 1.0), (4, 1.0), (5, 0.0)), + Set((0, 2.0), (1, 2.0), (2, 0.0), (3, 1.0), (4, 1.0), (5, 2.0)), + Set((0, 1.0), (1, 1.0), (2, 0.0), (3, 2.0), (4, 2.0), (5, 1.0)), + Set((0, 1.0), (1, 1.0), (2, 2.0), (3, 0.0), (4, 0.0), (5, 1.0))) + + var idx = 0 + for (orderType <- StringIndexer.supportedStringOrderType) { + val transformed = indexer.setStringOrderType(orderType).fit(df).transform(df) + val output = transformed.select("id", "labelIndex").rdd.map { r => + (r.getInt(0), r.getDouble(1)) + }.collect().toSet + assert(output === expected(idx)) + idx += 1 + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala index e5fd21c3f6fc..c895659a2d8b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala @@ -21,21 +21,32 @@ import scala.beans.BeanInfo import org.apache.spark.SparkFunSuite import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{Dataset, Row} @BeanInfo case class TokenizerTestData(rawText: String, wantedTokens: Array[String]) -class TokenizerSuite extends SparkFunSuite { +class TokenizerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { test("params") { ParamsSuite.checkParams(new Tokenizer) } + + test("read/write") { + val t = new Tokenizer() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + testDefaultReadWrite(t) + } } -class RegexTokenizerSuite extends SparkFunSuite with MLlibTestSparkContext { +class RegexTokenizerSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + import org.apache.spark.ml.feature.RegexTokenizerSuite._ + import testImplicits._ test("params") { ParamsSuite.checkParams(new RegexTokenizer) @@ -47,33 +58,56 @@ class RegexTokenizerSuite extends SparkFunSuite with MLlibTestSparkContext { .setPattern("\\w+|\\p{Punct}") .setInputCol("rawText") .setOutputCol("tokens") - val dataset0 = sqlContext.createDataFrame(Seq( - TokenizerTestData("Test for tokenization.", Array("Test", "for", "tokenization", ".")), - TokenizerTestData("Te,st. punct", Array("Te", ",", "st", ".", "punct")) - )) + val dataset0 = Seq( + TokenizerTestData("Test for tokenization.", Array("test", "for", "tokenization", ".")), + TokenizerTestData("Te,st. punct", Array("te", ",", "st", ".", "punct")) + ).toDF() testRegexTokenizer(tokenizer0, dataset0) - val dataset1 = sqlContext.createDataFrame(Seq( - TokenizerTestData("Test for tokenization.", Array("Test", "for", "tokenization")), + val dataset1 = Seq( + TokenizerTestData("Test for tokenization.", Array("test", "for", "tokenization")), TokenizerTestData("Te,st. punct", Array("punct")) - )) + ).toDF() tokenizer0.setMinTokenLength(3) testRegexTokenizer(tokenizer0, dataset1) val tokenizer2 = new RegexTokenizer() .setInputCol("rawText") .setOutputCol("tokens") - val dataset2 = sqlContext.createDataFrame(Seq( - TokenizerTestData("Test for tokenization.", Array("Test", "for", "tokenization.")), - TokenizerTestData("Te,st. punct", Array("Te,st.", "punct")) - )) + val dataset2 = Seq( + TokenizerTestData("Test for tokenization.", Array("test", "for", "tokenization.")), + TokenizerTestData("Te,st. punct", Array("te,st.", "punct")) + ).toDF() testRegexTokenizer(tokenizer2, dataset2) } + + test("RegexTokenizer with toLowercase false") { + val tokenizer = new RegexTokenizer() + .setInputCol("rawText") + .setOutputCol("tokens") + .setToLowercase(false) + val dataset = Seq( + TokenizerTestData("JAVA SCALA", Array("JAVA", "SCALA")), + TokenizerTestData("java scala", Array("java", "scala")) + ).toDF() + testRegexTokenizer(tokenizer, dataset) + } + + test("read/write") { + val t = new RegexTokenizer() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setMinTokenLength(2) + .setGaps(false) + .setPattern("hi") + .setToLowercase(false) + testDefaultReadWrite(t) + } } object RegexTokenizerSuite extends SparkFunSuite { - def testRegexTokenizer(t: RegexTokenizer, dataset: DataFrame): Unit = { + def testRegexTokenizer(t: RegexTokenizer, dataset: Dataset[_]): Unit = { t.transform(dataset) .select("tokens", "wantedTokens") .collect() diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala index bb4d5b983e0d..6aef1c683702 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala @@ -19,13 +19,17 @@ package org.apache.spark.ml.feature import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.ml.attribute.{AttributeGroup, NominalAttribute, NumericAttribute} +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.Row import org.apache.spark.sql.functions.col -class VectorAssemblerSuite extends SparkFunSuite with MLlibTestSparkContext { +class VectorAssemblerSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ test("params") { ParamsSuite.checkParams(new VectorAssembler) @@ -55,9 +59,9 @@ class VectorAssemblerSuite extends SparkFunSuite with MLlibTestSparkContext { } test("VectorAssembler") { - val df = sqlContext.createDataFrame(Seq( + val df = Seq( (0, 0.0, Vectors.dense(1.0, 2.0), "a", Vectors.sparse(2, Array(1), Array(3.0)), 10L) - )).toDF("id", "x", "y", "name", "z", "n") + ).toDF("id", "x", "y", "name", "z", "n") val assembler = new VectorAssembler() .setInputCols(Array("x", "y", "z", "n")) .setOutputCol("features") @@ -67,6 +71,20 @@ class VectorAssemblerSuite extends SparkFunSuite with MLlibTestSparkContext { } } + test("transform should throw an exception in case of unsupported type") { + val df = Seq(("a", "b", "c")).toDF("a", "b", "c") + val assembler = new VectorAssembler() + .setInputCols(Array("a", "b", "c")) + .setOutputCol("features") + val thrown = intercept[IllegalArgumentException] { + assembler.transform(df) + } + assert(thrown.getMessage contains + "Data type StringType of column a is not supported.\n" + + "Data type StringType of column b is not supported.\n" + + "Data type StringType of column c is not supported.") + } + test("ML attributes") { val browser = NominalAttribute.defaultAttr.withValues("chrome", "firefox", "safari") val hour = NumericAttribute.defaultAttr.withMin(0.0).withMax(24.0) @@ -74,7 +92,7 @@ class VectorAssemblerSuite extends SparkFunSuite with MLlibTestSparkContext { NominalAttribute.defaultAttr.withName("gender").withValues("male", "female"), NumericAttribute.defaultAttr.withName("salary"))) val row = (1.0, 0.5, 1, Vectors.dense(1.0, 1000.0), Vectors.sparse(2, Array(1), Array(2.0))) - val df = sqlContext.createDataFrame(Seq(row)).toDF("browser", "hour", "count", "user", "ad") + val df = Seq(row).toDF("browser", "hour", "count", "user", "ad") .select( col("browser").as("browser", browser.toMetadata()), col("hour").as("hour", hour.toMetadata()), @@ -98,7 +116,14 @@ class VectorAssemblerSuite extends SparkFunSuite with MLlibTestSparkContext { assert(userGenderOut === user.getAttr("gender").withName("user_gender").withIndex(3)) val userSalaryOut = features.getAttr(4) assert(userSalaryOut === user.getAttr("salary").withName("user_salary").withIndex(4)) - assert(features.getAttr(5) === NumericAttribute.defaultAttr.withIndex(5)) - assert(features.getAttr(6) === NumericAttribute.defaultAttr.withIndex(6)) + assert(features.getAttr(5) === NumericAttribute.defaultAttr.withIndex(5).withName("ad_0")) + assert(features.getAttr(6) === NumericAttribute.defaultAttr.withIndex(6).withName("ad_1")) + } + + test("read/write") { + val t = new VectorAssembler() + .setInputCols(Array("myInputCol", "myInputCol2")) + .setOutputCol("myOutputCol") + testDefaultReadWrite(t) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala index 8cb0a2cf14d3..f2cca8aa82e8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala @@ -19,17 +19,20 @@ package org.apache.spark.ml.feature import scala.beans.{BeanInfo, BeanProperty} -import org.apache.spark.{Logging, SparkException, SparkFunSuite} +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.linalg.{SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame -class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { +class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext + with DefaultReadWriteTest with Logging { + import testImplicits._ import VectorIndexerSuite.FeatureData // identical, of length 3 @@ -83,11 +86,11 @@ class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext with L checkPair(densePoints1Seq, sparsePoints1Seq) checkPair(densePoints2Seq, sparsePoints2Seq) - densePoints1 = sqlContext.createDataFrame(sc.parallelize(densePoints1Seq, 2).map(FeatureData)) - sparsePoints1 = sqlContext.createDataFrame(sc.parallelize(sparsePoints1Seq, 2).map(FeatureData)) - densePoints2 = sqlContext.createDataFrame(sc.parallelize(densePoints2Seq, 2).map(FeatureData)) - sparsePoints2 = sqlContext.createDataFrame(sc.parallelize(sparsePoints2Seq, 2).map(FeatureData)) - badPoints = sqlContext.createDataFrame(sc.parallelize(badPointsSeq, 2).map(FeatureData)) + densePoints1 = densePoints1Seq.map(FeatureData).toDF() + sparsePoints1 = sparsePoints1Seq.map(FeatureData).toDF() + densePoints2 = densePoints2Seq.map(FeatureData).toDF() + sparsePoints2 = sparsePoints2Seq.map(FeatureData).toDF() + badPoints = badPointsSeq.map(FeatureData).toDF() } private def getIndexer: VectorIndexer = @@ -100,7 +103,7 @@ class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext with L } test("Cannot fit an empty DataFrame") { - val rdd = sqlContext.createDataFrame(sc.parallelize(Array.empty[Vector], 2).map(FeatureData)) + val rdd = Array.empty[Vector].map(FeatureData).toSeq.toDF() val vectorIndexer = getIndexer intercept[IllegalArgumentException] { vectorIndexer.fit(rdd) @@ -111,15 +114,21 @@ class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext with L val vectorIndexer = getIndexer val model = vectorIndexer.fit(densePoints1) // vectors of length 3 - // copied model must have the same parent. - MLTestingUtils.checkCopy(model) + MLTestingUtils.checkCopyAndUids(vectorIndexer, model) model.transform(densePoints1) // should work model.transform(sparsePoints1) // should work - intercept[SparkException] { + // If the data is local Dataset, it throws AssertionError directly. + intercept[AssertionError] { model.transform(densePoints2).collect() logInfo("Did not throw error when fit, transform were called on vectors of different lengths") } + // If the data is distributed Dataset, it throws SparkException + // which is the wrapper of AssertionError. + intercept[SparkException] { + model.transform(densePoints2.repartition(2)).collect() + logInfo("Did not throw error when fit, transform were called on vectors of different lengths") + } intercept[SparkException] { vectorIndexer.fit(badPoints) logInfo("Did not throw error when fitting vectors of different lengths in same RDD.") @@ -158,7 +167,7 @@ class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext with L // Chose correct categorical features assert(categoryMaps.keys.toSet === categoricalFeatures) val transformed = model.transform(data).select("indexed") - val indexedRDD: RDD[Vector] = transformed.map(_.getAs[Vector](0)) + val indexedRDD: RDD[Vector] = transformed.rdd.map(_.getAs[Vector](0)) val featureAttrs = AttributeGroup.fromStructField(transformed.schema("indexed")) assert(featureAttrs.name === "indexed") assert(featureAttrs.attributes.get.length === model.numFeatures) @@ -215,7 +224,8 @@ class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext with L val points = data.collect().map(_.getAs[Vector](0)) val vectorIndexer = getIndexer.setMaxCategories(maxCategories) val model = vectorIndexer.fit(data) - val indexedPoints = model.transform(data).select("indexed").map(_.getAs[Vector](0)).collect() + val indexedPoints = + model.transform(data).select("indexed").rdd.map(_.getAs[Vector](0)).collect() points.zip(indexedPoints).foreach { case (orig: SparseVector, indexed: SparseVector) => assert(orig.indices.length == indexed.indices.length) @@ -251,6 +261,23 @@ class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext with L } } } + + test("VectorIndexer read/write") { + val t = new VectorIndexer() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setMaxCategories(30) + testDefaultReadWrite(t) + } + + test("VectorIndexerModel read/write") { + val categoryMaps = Map(0 -> Map(0.0 -> 0, 1.0 -> 1), 1 -> Map(0.0 -> 0, 1.0 -> 1, + 2.0 -> 2, 3.0 -> 3), 2 -> Map(0.0 -> 0, -1.0 -> 1, 2.0 -> 2)) + val instance = new VectorIndexerModel("myVectorIndexerModel", 3, categoryMaps) + val newInstance = testDefaultReadWrite(instance) + assert(newInstance.numFeatures === instance.numFeatures) + assert(newInstance.categoryMaps === instance.categoryMaps) + } } private[feature] object VectorIndexerSuite { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala index a6c2fba8360d..1746ce53107c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorSlicerSuite.scala @@ -19,22 +19,23 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute} +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.DefaultReadWriteTest import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.types.{StructField, StructType} -class VectorSlicerSuite extends SparkFunSuite with MLlibTestSparkContext { +class VectorSlicerSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { test("params") { - val slicer = new VectorSlicer + val slicer = new VectorSlicer().setInputCol("feature") ParamsSuite.checkParams(slicer) assert(slicer.getIndices.length === 0) assert(slicer.getNames.length === 0) withClue("VectorSlicer should not have any features selected by default") { intercept[IllegalArgumentException] { - slicer.validateParams() + slicer.transformSchema(StructType(Seq(StructField("feature", new VectorUDT, true)))) } } } @@ -53,8 +54,6 @@ class VectorSlicerSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Test vector slicer") { - val sqlContext = new SQLContext(sc) - val data = Array( Vectors.sparse(5, Seq((0, -2.0), (1, 2.3))), Vectors.dense(-2.0, 2.3, 0.0, 0.0, 1.0), @@ -80,7 +79,7 @@ class VectorSlicerSuite extends SparkFunSuite with MLlibTestSparkContext { val resultAttrGroup = new AttributeGroup("expected", resultAttrs.asInstanceOf[Array[Attribute]]) val rdd = sc.parallelize(data.zip(expected)).map { case (a, b) => Row(a, b) } - val df = sqlContext.createDataFrame(rdd, + val df = spark.createDataFrame(rdd, StructType(Array(attrGroup.toStructField(), resultAttrGroup.toStructField()))) val vectorSlicer = new VectorSlicer().setInputCol("features").setOutputCol("result") @@ -106,4 +105,13 @@ class VectorSlicerSuite extends SparkFunSuite with MLlibTestSparkContext { vectorSlicer.setIndices(Array.empty).setNames(Array("f1", "f4")) validateResults(vectorSlicer.transform(df)) } + + test("read/write") { + val t = new VectorSlicer() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setIndices(Array(1, 3)) + .setNames(Array("a", "d")) + testDefaultReadWrite(t) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala index a2e46f202995..6183606a7b2a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala @@ -18,15 +18,16 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.feature.{Word2VecModel => OldWord2VecModel} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.Row +import org.apache.spark.util.Utils -class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { +class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { test("params") { ParamsSuite.checkParams(new Word2Vec) @@ -35,8 +36,9 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Word2Vec") { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + + val spark = this.spark + import spark.implicits._ val sentence = "a b " * 100 + "a c " * 10 val numOfWords = sentence.split(" ").size @@ -56,26 +58,28 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { val docDF = doc.zip(expected).toDF("text", "expected") - val model = new Word2Vec() + val w2v = new Word2Vec() .setVectorSize(3) .setInputCol("text") .setOutputCol("result") .setSeed(42L) - .fit(docDF) + val model = w2v.fit(docDF) - // copied model must have the same parent. - MLTestingUtils.checkCopy(model) + MLTestingUtils.checkCopyAndUids(w2v, model) + // These expectations are just magic values, characterizing the current + // behavior. The test needs to be updated to be more general, see SPARK-11502 + val magicExp = Vectors.dense(0.30153007534417237, -0.6833061711354689, 0.5116530778733167) model.transform(docDF).select("result", "expected").collect().foreach { case Row(vector1: Vector, vector2: Vector) => - assert(vector1 ~== vector2 absTol 1E-5, "Transformed vector is different with expected.") + assert(vector1 ~== magicExp absTol 1E-5, "Transformed vector is different with expected.") } } test("getVectors") { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = this.spark + import spark.implicits._ val sentence = "a b " * 100 + "a c " * 10 val doc = sc.parallelize(Seq(sentence, sentence)).map(line => line.split(" ")) @@ -96,11 +100,18 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { .setSeed(42L) .fit(docDF) - val realVectors = model.getVectors.sort("word").select("vector").map { + val realVectors = model.getVectors.sort("word").select("vector").rdd.map { case Row(v: Vector) => v }.collect() + // These expectations are just magic values, characterizing the current + // behavior. The test needs to be updated to be more general, see SPARK-11502 + val magicExpected = Seq( + Vectors.dense(0.3326166272163391, -0.5603077411651611, -0.2309209555387497), + Vectors.dense(0.32463887333869934, -0.9306551218032837, 1.393115520477295), + Vectors.dense(-0.27150997519493103, 0.4372006058692932, -0.13465698063373566) + ) - realVectors.zip(expectedVectors).foreach { + realVectors.zip(magicExpected).foreach { case (real, expected) => assert(real ~== expected absTol 1E-5, "Actual vector is different from expected.") } @@ -108,8 +119,8 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { test("findSynonyms") { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = this.spark + import spark.implicits._ val sentence = "a b " * 100 + "a c " * 10 val doc = sc.parallelize(Seq(sentence, sentence)).map(line => line.split(" ")) @@ -122,16 +133,117 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { .setSeed(42L) .fit(docDF) - val expectedSimilarity = Array(0.2789285076917586, -0.6336972059851644) - val (synonyms, similarity) = model.findSynonyms("a", 2).map { + val expected = Map(("b", 0.2608488929093532), ("c", -0.8271274846926078)) + val findSynonymsResult = model.findSynonyms("a", 2).rdd.map { case Row(w: String, sim: Double) => (w, sim) - }.collect().unzip + }.collectAsMap() - assert(synonyms.toArray === Array("b", "c")) - expectedSimilarity.zip(similarity).map { - case (expected, actual) => assert(math.abs((expected - actual) / expected) < 1E-5) + expected.foreach { + case (expectedSynonym, expectedSimilarity) => + assert(findSynonymsResult.contains(expectedSynonym)) + assert(expectedSimilarity ~== findSynonymsResult.get(expectedSynonym).get absTol 1E-5) } + val findSynonymsArrayResult = model.findSynonymsArray("a", 2).toMap + findSynonymsResult.foreach { + case (expectedSynonym, expectedSimilarity) => + assert(findSynonymsArrayResult.contains(expectedSynonym)) + assert(expectedSimilarity ~== findSynonymsArrayResult.get(expectedSynonym).get absTol 1E-5) + } + } + + test("window size") { + + val spark = this.spark + import spark.implicits._ + + val sentence = "a q s t q s t b b b s t m s t m q " * 100 + "a c " * 10 + val doc = sc.parallelize(Seq(sentence, sentence)).map(line => line.split(" ")) + val docDF = doc.zip(doc).toDF("text", "alsotext") + + val model = new Word2Vec() + .setVectorSize(3) + .setWindowSize(2) + .setInputCol("text") + .setOutputCol("result") + .setSeed(42L) + .fit(docDF) + + val (synonyms, similarity) = model.findSynonyms("a", 6).rdd.map { + case Row(w: String, sim: Double) => (w, sim) + }.collect().unzip + + // Increase the window size + val biggerModel = new Word2Vec() + .setVectorSize(3) + .setInputCol("text") + .setOutputCol("result") + .setSeed(42L) + .setWindowSize(10) + .fit(docDF) + + val (synonymsLarger, similarityLarger) = model.findSynonyms("a", 6).rdd.map { + case Row(w: String, sim: Double) => (w, sim) + }.collect().unzip + // The similarity score should be very different with the larger window + assert(math.abs(similarity(5) - similarityLarger(5) / similarity(5)) > 1E-5) + } + + test("Word2Vec read/write numPartitions calculation") { + val smallModelNumPartitions = Word2VecModel.Word2VecModelWriter.calculateNumberOfPartitions( + Utils.byteStringAsBytes("64m"), numWords = 10, vectorSize = 5) + assert(smallModelNumPartitions === 1) + val largeModelNumPartitions = Word2VecModel.Word2VecModelWriter.calculateNumberOfPartitions( + Utils.byteStringAsBytes("64m"), numWords = 1000000, vectorSize = 5000) + assert(largeModelNumPartitions > 1) + } + + test("Word2Vec read/write") { + val t = new Word2Vec() + .setInputCol("myInputCol") + .setOutputCol("myOutputCol") + .setMaxIter(2) + .setMinCount(8) + .setNumPartitions(1) + .setSeed(42L) + .setStepSize(0.01) + .setVectorSize(100) + .setMaxSentenceLength(500) + testDefaultReadWrite(t) } + + test("Word2VecModel read/write") { + val word2VecMap = Map( + ("china", Array(0.50f, 0.50f, 0.50f, 0.50f)), + ("japan", Array(0.40f, 0.50f, 0.50f, 0.50f)), + ("taiwan", Array(0.60f, 0.50f, 0.50f, 0.50f)), + ("korea", Array(0.45f, 0.60f, 0.60f, 0.60f)) + ) + val oldModel = new OldWord2VecModel(word2VecMap) + val instance = new Word2VecModel("myWord2VecModel", oldModel) + val newInstance = testDefaultReadWrite(instance) + assert(newInstance.getVectors.collect() === instance.getVectors.collect()) + } + + test("Word2Vec works with input that is non-nullable (NGram)") { + val spark = this.spark + import spark.implicits._ + + val sentence = "a q s t q s t b b b s t m s t m q " + val docDF = sc.parallelize(Seq(sentence, sentence)).map(_.split(" ")).toDF("text") + + val ngram = new NGram().setN(2).setInputCol("text").setOutputCol("ngrams") + val ngramDF = ngram.transform(docDF) + + val model = new Word2Vec() + .setVectorSize(2) + .setInputCol("ngrams") + .setOutputCol("result") + .fit(ngramDF) + + // Just test that this transformation succeeds + model.transform(ngramDF).collect() + } + } diff --git a/mllib/src/test/scala/org/apache/spark/ml/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/fpm/FPGrowthSuite.scala new file mode 100644 index 000000000000..87f8b9034dde --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/fpm/FPGrowthSuite.scala @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.ml.fpm + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + @transient var dataset: Dataset[_] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + dataset = FPGrowthSuite.getFPGrowthData(spark) + } + + test("FPGrowth fit and transform with different data types") { + Array(IntegerType, StringType, ShortType, LongType, ByteType).foreach { dt => + val data = dataset.withColumn("items", col("items").cast(ArrayType(dt))) + val model = new FPGrowth().setMinSupport(0.5).fit(data) + val generatedRules = model.setMinConfidence(0.5).associationRules + val expectedRules = spark.createDataFrame(Seq( + (Array("2"), Array("1"), 1.0), + (Array("1"), Array("2"), 0.75) + )).toDF("antecedent", "consequent", "confidence") + .withColumn("antecedent", col("antecedent").cast(ArrayType(dt))) + .withColumn("consequent", col("consequent").cast(ArrayType(dt))) + assert(expectedRules.sort("antecedent").rdd.collect().sameElements( + generatedRules.sort("antecedent").rdd.collect())) + + val transformed = model.transform(data) + val expectedTransformed = spark.createDataFrame(Seq( + (0, Array("1", "2"), Array.emptyIntArray), + (0, Array("1", "2"), Array.emptyIntArray), + (0, Array("1", "2"), Array.emptyIntArray), + (0, Array("1", "3"), Array(2)) + )).toDF("id", "items", "prediction") + .withColumn("items", col("items").cast(ArrayType(dt))) + .withColumn("prediction", col("prediction").cast(ArrayType(dt))) + assert(expectedTransformed.collect().toSet.equals( + transformed.collect().toSet)) + } + } + + test("FPGrowth getFreqItems") { + val model = new FPGrowth().setMinSupport(0.7).fit(dataset) + val expectedFreq = spark.createDataFrame(Seq( + (Array("1"), 4L), + (Array("2"), 3L), + (Array("1", "2"), 3L), + (Array("2", "1"), 3L) // duplicate as the items sequence is not guaranteed + )).toDF("items", "expectedFreq") + val freqItems = model.freqItemsets + + val checkDF = freqItems.join(expectedFreq, "items") + assert(checkDF.count() == 3 && checkDF.filter(col("freq") === col("expectedFreq")).count() == 3) + } + + test("FPGrowth getFreqItems with Null") { + val df = spark.createDataFrame(Seq( + (1, Array("1", "2", "3", "5")), + (2, Array("1", "2", "3", "4")), + (3, null.asInstanceOf[Array[String]]) + )).toDF("id", "items") + val model = new FPGrowth().setMinSupport(0.7).fit(dataset) + val prediction = model.transform(df) + assert(prediction.select("prediction").where("id=3").first().getSeq[String](0).isEmpty) + } + + test("FPGrowth prediction should not contain duplicates") { + // This should generate rule 1 -> 3, 2 -> 3 + val dataset = spark.createDataFrame(Seq( + Array("1", "3"), + Array("2", "3") + ).map(Tuple1(_))).toDF("items") + val model = new FPGrowth().fit(dataset) + + val prediction = model.transform( + spark.createDataFrame(Seq(Tuple1(Array("1", "2")))).toDF("items") + ).first().getAs[Seq[String]]("prediction") + + assert(prediction === Seq("3")) + } + + test("FPGrowthModel setMinConfidence should affect rules generation and transform") { + val model = new FPGrowth().setMinSupport(0.1).setMinConfidence(0.1).fit(dataset) + val oldRulesNum = model.associationRules.count() + val oldPredict = model.transform(dataset) + + model.setMinConfidence(0.8765) + assert(oldRulesNum > model.associationRules.count()) + assert(!model.transform(dataset).collect().toSet.equals(oldPredict.collect().toSet)) + + // association rules should stay the same for same minConfidence + model.setMinConfidence(0.1) + assert(oldRulesNum === model.associationRules.count()) + assert(model.transform(dataset).collect().toSet.equals(oldPredict.collect().toSet)) + } + + test("FPGrowth parameter check") { + val fpGrowth = new FPGrowth().setMinSupport(0.4567) + val model = fpGrowth.fit(dataset) + .setMinConfidence(0.5678) + assert(fpGrowth.getMinSupport === 0.4567) + assert(model.getMinConfidence === 0.5678) + // numPartitions should not have default value. + assert(fpGrowth.isDefined(fpGrowth.numPartitions) === false) + MLTestingUtils.checkCopyAndUids(fpGrowth, model) + ParamsSuite.checkParams(fpGrowth) + ParamsSuite.checkParams(model) + } + + test("read/write") { + def checkModelData(model: FPGrowthModel, model2: FPGrowthModel): Unit = { + assert(model.freqItemsets.collect().toSet.equals( + model2.freqItemsets.collect().toSet)) + assert(model.associationRules.collect().toSet.equals( + model2.associationRules.collect().toSet)) + assert(model.setMinConfidence(0.9).associationRules.collect().toSet.equals( + model2.setMinConfidence(0.9).associationRules.collect().toSet)) + } + val fPGrowth = new FPGrowth() + testEstimatorAndModelReadWrite(fPGrowth, dataset, FPGrowthSuite.allParamSettings, + FPGrowthSuite.allParamSettings, checkModelData) + } +} + +object FPGrowthSuite { + + def getFPGrowthData(spark: SparkSession): DataFrame = { + spark.createDataFrame(Seq( + (0, Array("1", "2")), + (0, Array("1", "2")), + (0, Array("1", "2")), + (0, Array("1", "3")) + )).toDF("id", "items") + } + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "minSupport" -> 0.321, + "minConfidence" -> 0.456, + "numPartitions" -> 5, + "predictionCol" -> "myPrediction" + ) +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/impl/TreeTests.scala b/mllib/src/test/scala/org/apache/spark/ml/impl/TreeTests.scala deleted file mode 100644 index 460849c79f04..000000000000 --- a/mllib/src/test/scala/org/apache/spark/ml/impl/TreeTests.scala +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.impl - -import scala.collection.JavaConverters._ - -import org.apache.spark.SparkFunSuite -import org.apache.spark.api.java.JavaRDD -import org.apache.spark.ml.attribute.{AttributeGroup, NominalAttribute, NumericAttribute} -import org.apache.spark.ml.tree._ -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{SQLContext, DataFrame} - - -private[ml] object TreeTests extends SparkFunSuite { - - /** - * Convert the given data to a DataFrame, and set the features and label metadata. - * @param data Dataset. Categorical features and labels must already have 0-based indices. - * This must be non-empty. - * @param categoricalFeatures Map: categorical feature index -> number of distinct values - * @param numClasses Number of classes label can take. If 0, mark as continuous. - * @return DataFrame with metadata - */ - def setMetadata( - data: RDD[LabeledPoint], - categoricalFeatures: Map[Int, Int], - numClasses: Int): DataFrame = { - val sqlContext = new SQLContext(data.sparkContext) - import sqlContext.implicits._ - val df = data.toDF() - val numFeatures = data.first().features.size - val featuresAttributes = Range(0, numFeatures).map { feature => - if (categoricalFeatures.contains(feature)) { - NominalAttribute.defaultAttr.withIndex(feature).withNumValues(categoricalFeatures(feature)) - } else { - NumericAttribute.defaultAttr.withIndex(feature) - } - }.toArray - val featuresMetadata = new AttributeGroup("features", featuresAttributes).toMetadata() - val labelAttribute = if (numClasses == 0) { - NumericAttribute.defaultAttr.withName("label") - } else { - NominalAttribute.defaultAttr.withName("label").withNumValues(numClasses) - } - val labelMetadata = labelAttribute.toMetadata() - df.select(df("features").as("features", featuresMetadata), - df("label").as("label", labelMetadata)) - } - - /** Java-friendly version of [[setMetadata()]] */ - def setMetadata( - data: JavaRDD[LabeledPoint], - categoricalFeatures: java.util.Map[java.lang.Integer, java.lang.Integer], - numClasses: Int): DataFrame = { - setMetadata(data.rdd, categoricalFeatures.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, - numClasses) - } - - /** - * Check if the two trees are exactly the same. - * Note: I hesitate to override Node.equals since it could cause problems if users - * make mistakes such as creating loops of Nodes. - * If the trees are not equal, this prints the two trees and throws an exception. - */ - def checkEqual(a: DecisionTreeModel, b: DecisionTreeModel): Unit = { - try { - checkEqual(a.rootNode, b.rootNode) - } catch { - case ex: Exception => - throw new AssertionError("checkEqual failed since the two trees were not identical.\n" + - "TREE A:\n" + a.toDebugString + "\n" + - "TREE B:\n" + b.toDebugString + "\n", ex) - } - } - - /** - * Return true iff the two nodes and their descendants are exactly the same. - * Note: I hesitate to override Node.equals since it could cause problems if users - * make mistakes such as creating loops of Nodes. - */ - private def checkEqual(a: Node, b: Node): Unit = { - assert(a.prediction === b.prediction) - assert(a.impurity === b.impurity) - (a, b) match { - case (aye: InternalNode, bee: InternalNode) => - assert(aye.split === bee.split) - checkEqual(aye.leftChild, bee.leftChild) - checkEqual(aye.rightChild, bee.rightChild) - case (aye: LeafNode, bee: LeafNode) => // do nothing - case _ => - throw new AssertionError("Found mismatched nodes") - } - } - - /** - * Check if the two models are exactly the same. - * If the models are not equal, this throws an exception. - */ - def checkEqual(a: TreeEnsembleModel, b: TreeEnsembleModel): Unit = { - try { - a.trees.zip(b.trees).foreach { case (treeA, treeB) => - TreeTests.checkEqual(treeA, treeB) - } - assert(a.treeWeights === b.treeWeights) - } catch { - case ex: Exception => throw new AssertionError( - "checkEqual failed since the two tree ensembles were not identical") - } - } - - /** - * Helper method for constructing a tree for testing. - * Given left, right children, construct a parent node. - * @param split Split for parent node - * @return Parent node with children attached - */ - def buildParentNode(left: Node, right: Node, split: Split): Node = { - val leftImp = left.impurityStats - val rightImp = right.impurityStats - val parentImp = leftImp.copy.add(rightImp) - val leftWeight = leftImp.count / parentImp.count.toDouble - val rightWeight = rightImp.count / parentImp.count.toDouble - val gain = parentImp.calculate() - - (leftWeight * leftImp.calculate() + rightWeight * rightImp.calculate()) - val pred = parentImp.predict - new InternalNode(pred, parentImp.calculate(), gain, left, right, split, parentImp) - } -} diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/JsonVectorConverterSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/JsonVectorConverterSuite.scala new file mode 100644 index 000000000000..53d57f0f6e28 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/JsonVectorConverterSuite.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import org.json4s.jackson.JsonMethods.parse + +import org.apache.spark.SparkFunSuite + +class JsonVectorConverterSuite extends SparkFunSuite { + + test("toJson/fromJson") { + val sv0 = Vectors.sparse(0, Array.empty, Array.empty) + val sv1 = Vectors.sparse(1, Array.empty, Array.empty) + val sv2 = Vectors.sparse(2, Array(1), Array(2.0)) + val dv0 = Vectors.dense(Array.empty[Double]) + val dv1 = Vectors.dense(1.0) + val dv2 = Vectors.dense(0.0, 2.0) + for (v <- Seq(sv0, sv1, sv2, dv0, dv1, dv2)) { + val json = JsonVectorConverter.toJson(v) + parse(json) // `json` should be a valid JSON string + val u = JsonVectorConverter.fromJson(json) + assert(u.getClass === v.getClass, "toJson/fromJson should preserve vector types.") + assert(u === v, "toJson/fromJson should preserve vector values.") + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/MatrixUDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/MatrixUDTSuite.scala new file mode 100644 index 000000000000..bdceba7887ca --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/MatrixUDTSuite.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types._ + +class MatrixUDTSuite extends SparkFunSuite { + + test("preloaded MatrixUDT") { + val dm1 = new DenseMatrix(2, 2, Array(0.9, 1.2, 2.3, 9.8)) + val dm2 = new DenseMatrix(3, 2, Array(0.0, 1.21, 2.3, 9.8, 9.0, 0.0)) + val dm3 = new DenseMatrix(0, 0, Array()) + val sm1 = dm1.toSparse + val sm2 = dm2.toSparse + val sm3 = dm3.toSparse + + for (m <- Seq(dm1, dm2, dm3, sm1, sm2, sm3)) { + val udt = UDTRegistration.getUDTFor(m.getClass.getName).get.newInstance() + .asInstanceOf[MatrixUDT] + assert(m === udt.deserialize(udt.serialize(m))) + assert(udt.typeName == "matrix") + assert(udt.simpleString == "matrix") + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/SQLDataTypesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/SQLDataTypesSuite.scala new file mode 100644 index 000000000000..0bd0c32f19d0 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/SQLDataTypesSuite.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import org.apache.spark.SparkFunSuite + +class SQLDataTypesSuite extends SparkFunSuite { + test("sqlDataTypes") { + assert(SQLDataTypes.VectorType === new VectorUDT) + assert(SQLDataTypes.MatrixType === new MatrixUDT) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala new file mode 100644 index 000000000000..6ddb12cb76aa --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.linalg + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.sql.catalyst.JavaTypeInference +import org.apache.spark.sql.types._ + +class VectorUDTSuite extends SparkFunSuite { + + test("preloaded VectorUDT") { + val dv1 = Vectors.dense(Array.empty[Double]) + val dv2 = Vectors.dense(1.0, 2.0) + val sv1 = Vectors.sparse(2, Array.empty, Array.empty) + val sv2 = Vectors.sparse(2, Array(1), Array(2.0)) + + for (v <- Seq(dv1, dv2, sv1, sv2)) { + val udt = UDTRegistration.getUDTFor(v.getClass.getName).get.newInstance() + .asInstanceOf[VectorUDT] + assert(v === udt.deserialize(udt.serialize(v))) + assert(udt.typeName == "vector") + assert(udt.simpleString == "vector") + } + } + + test("JavaTypeInference with VectorUDT") { + val (dataType, _) = JavaTypeInference.inferDataType(classOf[LabeledPoint]) + assert(dataType.asInstanceOf[StructType].fields.map(_.dataType) + === Seq(new VectorUDT, DoubleType)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala new file mode 100644 index 000000000000..6d143504fcf5 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/IterativelyReweightedLeastSquaresSuite.scala @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.optim + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.{Instance, OffsetInstance} +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.rdd.RDD + +class IterativelyReweightedLeastSquaresSuite extends SparkFunSuite with MLlibTestSparkContext { + + private var instances1: RDD[OffsetInstance] = _ + private var instances2: RDD[OffsetInstance] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + /* + R code: + + A <- matrix(c(0, 1, 2, 3, 5, 2, 1, 3), 4, 2) + b <- c(1, 0, 1, 0) + w <- c(1, 2, 3, 4) + */ + instances1 = sc.parallelize(Seq( + OffsetInstance(1.0, 1.0, 0.0, Vectors.dense(0.0, 5.0).toSparse), + OffsetInstance(0.0, 2.0, 0.0, Vectors.dense(1.0, 2.0)), + OffsetInstance(1.0, 3.0, 0.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(0.0, 4.0, 0.0, Vectors.dense(3.0, 3.0)) + ), 2) + /* + R code: + + A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) + b <- c(2, 8, 3, 9) + w <- c(1, 2, 3, 4) + */ + instances2 = sc.parallelize(Seq( + OffsetInstance(2.0, 1.0, 0.0, Vectors.dense(0.0, 5.0).toSparse), + OffsetInstance(8.0, 2.0, 0.0, Vectors.dense(1.0, 7.0)), + OffsetInstance(3.0, 3.0, 0.0, Vectors.dense(2.0, 11.0)), + OffsetInstance(9.0, 4.0, 0.0, Vectors.dense(3.0, 13.0)) + ), 2) + } + + test("IRLS against GLM with Binomial errors") { + /* + R code: + + df <- as.data.frame(cbind(A, b)) + for (formula in c(b ~ . -1, b ~ .)) { + model <- glm(formula, family="binomial", data=df, weights=w) + print(as.vector(coef(model))) + } + + [1] -0.30216651 -0.04452045 + [1] 3.5651651 -1.2334085 -0.7348971 + */ + val expected = Seq( + Vectors.dense(0.0, -0.30216651, -0.04452045), + Vectors.dense(3.5651651, -1.2334085, -0.7348971)) + + import IterativelyReweightedLeastSquaresSuite._ + + var idx = 0 + for (fitIntercept <- Seq(false, true)) { + val newInstances = instances1.map { instance => + val mu = (instance.label + 0.5) / 2.0 + val eta = math.log(mu / (1.0 - mu)) + Instance(eta, instance.weight, instance.features) + } + val initial = new WeightedLeastSquares(fitIntercept, regParam = 0.0, elasticNetParam = 0.0, + standardizeFeatures = false, standardizeLabel = false).fit(newInstances) + val irls = new IterativelyReweightedLeastSquares(initial, BinomialReweightFunc, + fitIntercept, regParam = 0.0, maxIter = 25, tol = 1e-8).fit(instances1) + val actual = Vectors.dense(irls.intercept, irls.coefficients(0), irls.coefficients(1)) + assert(actual ~== expected(idx) absTol 1e-4) + idx += 1 + } + } + + test("IRLS against GLM with Poisson errors") { + /* + R code: + + df <- as.data.frame(cbind(A, b)) + for (formula in c(b ~ . -1, b ~ .)) { + model <- glm(formula, family="poisson", data=df, weights=w) + print(as.vector(coef(model))) + } + + [1] -0.09607792 0.18375613 + [1] 6.299947 3.324107 -1.081766 + */ + val expected = Seq( + Vectors.dense(0.0, -0.09607792, 0.18375613), + Vectors.dense(6.299947, 3.324107, -1.081766)) + + import IterativelyReweightedLeastSquaresSuite._ + + var idx = 0 + for (fitIntercept <- Seq(false, true)) { + val yMean = instances2.map(_.label).mean + val newInstances = instances2.map { instance => + val mu = (instance.label + yMean) / 2.0 + val eta = math.log(mu) + Instance(eta, instance.weight, instance.features) + } + val initial = new WeightedLeastSquares(fitIntercept, regParam = 0.0, elasticNetParam = 0.0, + standardizeFeatures = false, standardizeLabel = false).fit(newInstances) + val irls = new IterativelyReweightedLeastSquares(initial, PoissonReweightFunc, + fitIntercept, regParam = 0.0, maxIter = 25, tol = 1e-8).fit(instances2) + val actual = Vectors.dense(irls.intercept, irls.coefficients(0), irls.coefficients(1)) + assert(actual ~== expected(idx) absTol 1e-4) + idx += 1 + } + } + + test("IRLS against L1Regression") { + /* + R code: + + library(quantreg) + + df <- as.data.frame(cbind(A, b)) + for (formula in c(b ~ . -1, b ~ .)) { + model <- rq(formula, data=df, weights=w) + print(as.vector(coef(model))) + } + + [1] 1.266667 0.400000 + [1] 29.5 17.0 -5.5 + */ + val expected = Seq( + Vectors.dense(0.0, 1.266667, 0.400000), + Vectors.dense(29.5, 17.0, -5.5)) + + import IterativelyReweightedLeastSquaresSuite._ + + var idx = 0 + for (fitIntercept <- Seq(false, true)) { + val initial = new WeightedLeastSquares(fitIntercept, regParam = 0.0, elasticNetParam = 0.0, + standardizeFeatures = false, standardizeLabel = false).fit(instances2.map(_.toInstance)) + val irls = new IterativelyReweightedLeastSquares(initial, L1RegressionReweightFunc, + fitIntercept, regParam = 0.0, maxIter = 200, tol = 1e-7).fit(instances2) + val actual = Vectors.dense(irls.intercept, irls.coefficients(0), irls.coefficients(1)) + assert(actual ~== expected(idx) absTol 1e-4) + idx += 1 + } + } +} + +object IterativelyReweightedLeastSquaresSuite { + + def BinomialReweightFunc( + instance: OffsetInstance, + model: WeightedLeastSquaresModel): (Double, Double) = { + val eta = model.predict(instance.features) + instance.offset + val mu = 1.0 / (1.0 + math.exp(-1.0 * eta)) + val z = eta - instance.offset + (instance.label - mu) / (mu * (1.0 - mu)) + val w = mu * (1 - mu) * instance.weight + (z, w) + } + + def PoissonReweightFunc( + instance: OffsetInstance, + model: WeightedLeastSquaresModel): (Double, Double) = { + val eta = model.predict(instance.features) + instance.offset + val mu = math.exp(eta) + val z = eta - instance.offset + (instance.label - mu) / mu + val w = mu * instance.weight + (z, w) + } + + def L1RegressionReweightFunc( + instance: OffsetInstance, + model: WeightedLeastSquaresModel): (Double, Double) = { + val eta = model.predict(instance.features) + instance.offset + val e = math.max(math.abs(eta - instance.label), 1e-7) + val w = 1 / e + val y = instance.label + (y, w) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala index b542ba3dc54d..093d02ea7a14 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/WeightedLeastSquaresSuite.scala @@ -19,14 +19,18 @@ package org.apache.spark.ml.optim import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.Instance -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.linalg.{BLAS, Vectors} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD class WeightedLeastSquaresSuite extends SparkFunSuite with MLlibTestSparkContext { private var instances: RDD[Instance] = _ + private var instancesConstLabel: RDD[Instance] = _ + private var instancesConstZeroLabel: RDD[Instance] = _ + private var collinearInstances: RDD[Instance] = _ + private var constantFeaturesInstances: RDD[Instance] = _ override def beforeAll(): Unit = { super.beforeAll() @@ -43,6 +47,135 @@ class WeightedLeastSquaresSuite extends SparkFunSuite with MLlibTestSparkContext Instance(23.0, 3.0, Vectors.dense(2.0, 11.0)), Instance(29.0, 4.0, Vectors.dense(3.0, 13.0)) ), 2) + + /* + R code: + + A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) + b.const <- c(17, 17, 17, 17) + w <- c(1, 2, 3, 4) + */ + instancesConstLabel = sc.parallelize(Seq( + Instance(17.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), + Instance(17.0, 2.0, Vectors.dense(1.0, 7.0)), + Instance(17.0, 3.0, Vectors.dense(2.0, 11.0)), + Instance(17.0, 4.0, Vectors.dense(3.0, 13.0)) + ), 2) + + /* + A <- matrix(c(1, 2, 3, 4, 2, 4, 6, 8), 4, 2) + b <- c(1, 2, 3, 4) + w <- c(1, 1, 1, 1) + */ + collinearInstances = sc.parallelize(Seq( + Instance(1.0, 1.0, Vectors.dense(1.0, 2.0)), + Instance(2.0, 1.0, Vectors.dense(2.0, 4.0)), + Instance(3.0, 1.0, Vectors.dense(3.0, 6.0)), + Instance(4.0, 1.0, Vectors.dense(4.0, 8.0)) + ), 2) + + /* + R code: + + A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) + b.const <- c(0, 0, 0, 0) + w <- c(1, 2, 3, 4) + */ + instancesConstZeroLabel = sc.parallelize(Seq( + Instance(0.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), + Instance(0.0, 2.0, Vectors.dense(1.0, 7.0)), + Instance(0.0, 3.0, Vectors.dense(2.0, 11.0)), + Instance(0.0, 4.0, Vectors.dense(3.0, 13.0)) + ), 2) + + /* + R code: + + A <- matrix(c(1, 1, 1, 1, 5, 7, 11, 13), 4, 2) + b <- c(17, 19, 23, 29) + w <- c(1, 2, 3, 4) + */ + constantFeaturesInstances = sc.parallelize(Seq( + Instance(17.0, 1.0, Vectors.dense(1.0, 5.0)), + Instance(19.0, 2.0, Vectors.dense(1.0, 7.0)), + Instance(23.0, 3.0, Vectors.dense(1.0, 11.0)), + Instance(29.0, 4.0, Vectors.dense(1.0, 13.0)) + ), 2) + } + + test("WLS with strong L1 regularization") { + /* + We initialize the coefficients for WLS QN solver to be weighted average of the label. Check + here that with only an intercept the model converges to bBar. + */ + val bAgg = instances.collect().foldLeft((0.0, 0.0)) { + case ((sum, weightSum), Instance(l, w, f)) => (sum + w * l, weightSum + w) + } + val bBar = bAgg._1 / bAgg._2 + val wls = new WeightedLeastSquares(true, 10, 1.0, true, true) + val model = wls.fit(instances) + assert(model.intercept ~== bBar relTol 1e-6) + } + + test("diagonal inverse of AtWA") { + /* + library(Matrix) + A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) + w <- c(1, 2, 3, 4) + W <- Diagonal(length(w), w) + A.intercept <- cbind(A, rep.int(1, length(w))) + AtA.intercept <- t(A.intercept) %*% W %*% A.intercept + inv.intercept <- solve(AtA.intercept) + print(diag(inv.intercept)) + [1] 4.02 0.50 12.02 + + AtA <- t(A) %*% W %*% A + inv <- solve(AtA) + print(diag(inv)) + [1] 0.48336106 0.02079867 + + */ + val expectedWithIntercept = Vectors.dense(4.02, 0.50, 12.02) + val expected = Vectors.dense(0.48336106, 0.02079867) + val wlsWithIntercept = new WeightedLeastSquares(fitIntercept = true, regParam = 0.0, + elasticNetParam = 0.0, standardizeFeatures = true, standardizeLabel = true, + solverType = WeightedLeastSquares.Cholesky) + val wlsModelWithIntercept = wlsWithIntercept.fit(instances) + val wls = new WeightedLeastSquares(false, 0.0, 0.0, true, true, + solverType = WeightedLeastSquares.Cholesky) + val wlsModel = wls.fit(instances) + + assert(expectedWithIntercept ~== wlsModelWithIntercept.diagInvAtWA relTol 1e-4) + assert(expected ~== wlsModel.diagInvAtWA relTol 1e-4) + } + + test("two collinear features") { + // Cholesky solver does not handle singular input + intercept[SingularMatrixException] { + new WeightedLeastSquares(fitIntercept = false, regParam = 0.0, elasticNetParam = 0.0, + standardizeFeatures = false, standardizeLabel = false, + solverType = WeightedLeastSquares.Cholesky).fit(collinearInstances) + } + + // Cholesky should not throw an exception since regularization is applied + new WeightedLeastSquares(fitIntercept = false, regParam = 1.0, elasticNetParam = 0.0, + standardizeFeatures = false, standardizeLabel = false, + solverType = WeightedLeastSquares.Cholesky).fit(collinearInstances) + + // quasi-newton solvers should handle singular input and make correct predictions + // auto solver should try Cholesky first, then fall back to QN + for (fitIntercept <- Seq(false, true); + standardization <- Seq(false, true); + solver <- Seq(WeightedLeastSquares.Auto, WeightedLeastSquares.QuasiNewton)) { + val singularModel = new WeightedLeastSquares(fitIntercept, regParam = 0.0, + elasticNetParam = 0.0, standardizeFeatures = standardization, + standardizeLabel = standardization, solverType = solver).fit(collinearInstances) + + collinearInstances.collect().foreach { case Instance(l, w, f) => + val pred = BLAS.dot(singularModel.coefficients, f) + singularModel.intercept + assert(pred ~== l absTol 1e-6) + } + } } test("WLS against lm") { @@ -65,8 +198,282 @@ class WeightedLeastSquaresSuite extends SparkFunSuite with MLlibTestSparkContext var idx = 0 for (fitIntercept <- Seq(false, true)) { - val wls = new WeightedLeastSquares( - fitIntercept, regParam = 0.0, standardizeFeatures = false, standardizeLabel = false) + for (standardization <- Seq(false, true)) { + for (solver <- WeightedLeastSquares.supportedSolvers) { + val wls = new WeightedLeastSquares(fitIntercept, regParam = 0.0, elasticNetParam = 0.0, + standardizeFeatures = standardization, standardizeLabel = standardization, + solverType = solver).fit(instances) + val actual = Vectors.dense(wls.intercept, wls.coefficients(0), wls.coefficients(1)) + assert(actual ~== expected(idx) absTol 1e-4) + } + } + idx += 1 + } + } + + test("WLS against lm when label is constant and no regularization") { + /* + R code: + + df.const.label <- as.data.frame(cbind(A, b.const)) + for (formula in c(b.const ~ . -1, b.const ~ .)) { + model <- lm(formula, data=df.const.label, weights=w) + print(as.vector(coef(model))) + } + + [1] -9.221298 3.394343 + [1] 17 0 0 + */ + + val expected = Seq( + Vectors.dense(0.0, -9.221298, 3.394343), + Vectors.dense(17.0, 0.0, 0.0)) + + var idx = 0 + for (fitIntercept <- Seq(false, true)) { + for (standardization <- Seq(false, true)) { + for (solver <- WeightedLeastSquares.supportedSolvers) { + val wls = new WeightedLeastSquares(fitIntercept, regParam = 0.0, elasticNetParam = 0.0, + standardizeFeatures = standardization, standardizeLabel = standardization, + solverType = solver).fit(instancesConstLabel) + val actual = Vectors.dense(wls.intercept, wls.coefficients(0), wls.coefficients(1)) + assert(actual ~== expected(idx) absTol 1e-4) + } + } + idx += 1 + } + + // when label is constant zero, and fitIntercept is false, we should not train and get all zeros + for (solver <- WeightedLeastSquares.supportedSolvers) { + val wls = new WeightedLeastSquares(fitIntercept = false, regParam = 0.0, + elasticNetParam = 0.0, standardizeFeatures = true, standardizeLabel = true, + solverType = solver).fit(instancesConstZeroLabel) + val actual = Vectors.dense(wls.intercept, wls.coefficients(0), wls.coefficients(1)) + assert(actual === Vectors.dense(0.0, 0.0, 0.0)) + assert(wls.objectiveHistory === Array(0.0)) + } + } + + test("WLS with regularization when label is constant") { + // if regParam is non-zero and standardization is true, the problem is ill-defined and + // an exception is thrown. + for (solver <- WeightedLeastSquares.supportedSolvers) { + val wls = new WeightedLeastSquares(fitIntercept = false, regParam = 0.1, + elasticNetParam = 0.0, standardizeFeatures = true, standardizeLabel = true, + solverType = solver) + intercept[IllegalArgumentException]{ + wls.fit(instancesConstLabel) + } + } + } + + test("WLS against glmnet with constant features") { + // Cholesky solver does not handle singular input with no regularization + for (fitIntercept <- Seq(false, true); + standardization <- Seq(false, true)) { + val wls = new WeightedLeastSquares(fitIntercept, regParam = 0.0, elasticNetParam = 0.0, + standardizeFeatures = standardization, standardizeLabel = standardization, + solverType = WeightedLeastSquares.Cholesky) + intercept[SingularMatrixException] { + wls.fit(constantFeaturesInstances) + } + } + + // Cholesky also fails when regularization is added but we don't wish to standardize + val wls = new WeightedLeastSquares(fitIntercept = true, regParam = 0.5, elasticNetParam = 0.0, + standardizeFeatures = false, standardizeLabel = false, + solverType = WeightedLeastSquares.Cholesky) + intercept[SingularMatrixException] { + wls.fit(constantFeaturesInstances) + } + + /* + for (intercept in c(FALSE, TRUE)) { + model <- glmnet(A, b, weights=w, intercept=intercept, lambda=0.5, + standardize=T, alpha=0.0, thresh=1E-14) + print(as.vector(coef(model))) + } + [1] 0.000000 0.000000 2.235802 + [1] 9.798771 0.000000 1.365503 + */ + // should not fail when regularization and standardization are added + val expectedCholesky = Seq( + Vectors.dense(0.0, 0.0, 2.235802), + Vectors.dense(9.798771, 0.0, 1.365503) + ) + var idx = 0 + for (fitIntercept <- Seq(false, true)) { + val wls = new WeightedLeastSquares(fitIntercept = fitIntercept, regParam = 0.5, + elasticNetParam = 0.0, standardizeFeatures = true, + standardizeLabel = true, solverType = WeightedLeastSquares.Cholesky) + .fit(constantFeaturesInstances) + val actual = Vectors.dense(wls.intercept, wls.coefficients(0), wls.coefficients(1)) + assert(actual ~== expectedCholesky(idx) absTol 1e-6) + idx += 1 + } + + /* + for (intercept in c(FALSE, TRUE)) { + for (standardize in c(FALSE, TRUE)) { + for (regParams in list(c(0.0, 0.0), c(0.5, 0.0), c(0.5, 0.5), c(0.5, 1.0))) { + model <- glmnet(A, b, weights=w, intercept=intercept, lambda=regParams[1], + standardize=standardize, alpha=regParams[2], thresh=1E-14) + print(as.vector(coef(model))) + } + } + } + [1] 0.000000 0.000000 2.253012 + [1] 0.000000 0.000000 2.250857 + [1] 0.000000 0.000000 2.249784 + [1] 0.000000 0.000000 2.248709 + [1] 0.000000 0.000000 2.253012 + [1] 0.000000 0.000000 2.235802 + [1] 0.000000 0.000000 2.238297 + [1] 0.000000 0.000000 2.240811 + [1] 8.218905 0.000000 1.517413 + [1] 8.434286 0.000000 1.496703 + [1] 8.648497 0.000000 1.476106 + [1] 8.865672 0.000000 1.455224 + [1] 8.218905 0.000000 1.517413 + [1] 9.798771 0.000000 1.365503 + [1] 9.919095 0.000000 1.353933 + [1] 10.052804 0.000000 1.341077 + */ + val expectedQuasiNewton = Seq( + Vectors.dense(0.000000, 0.000000, 2.253012), + Vectors.dense(0.000000, 0.000000, 2.250857), + Vectors.dense(0.000000, 0.000000, 2.249784), + Vectors.dense(0.000000, 0.000000, 2.248709), + Vectors.dense(0.000000, 0.000000, 2.253012), + Vectors.dense(0.000000, 0.000000, 2.235802), + Vectors.dense(0.000000, 0.000000, 2.238297), + Vectors.dense(0.000000, 0.000000, 2.240811), + Vectors.dense(8.218905, 0.000000, 1.517413), + Vectors.dense(8.434286, 0.000000, 1.496703), + Vectors.dense(8.648497, 0.000000, 1.476106), + Vectors.dense(8.865672, 0.000000, 1.455224), + Vectors.dense(8.218905, 0.000000, 1.517413), + Vectors.dense(9.798771, 0.000000, 1.365503), + Vectors.dense(9.919095, 0.000000, 1.353933), + Vectors.dense(10.052804, 0.000000, 1.341077)) + + idx = 0 + for (fitIntercept <- Seq(false, true); + standardization <- Seq(false, true); + (lambda, alpha) <- Seq((0.0, 0.0), (0.5, 0.0), (0.5, 0.5), (0.5, 1.0))) { + val wls = new WeightedLeastSquares(fitIntercept, regParam = lambda, elasticNetParam = alpha, + standardizeFeatures = standardization, standardizeLabel = true, + solverType = WeightedLeastSquares.QuasiNewton) + val model = wls.fit(constantFeaturesInstances) + val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) + assert(actual ~== expectedQuasiNewton(idx) absTol 1e-6) + + idx += 1 + } + } + + test("WLS against glmnet with L1/ElasticNet regularization") { + /* + R code: + + library(glmnet) + + for (intercept in c(FALSE, TRUE)) { + for (lambda in c(0.1, 0.5, 1.0)) { + for (standardize in c(FALSE, TRUE)) { + for (alpha in c(0.1, 0.5, 1.0)) { + model <- glmnet(A, b, weights=w, intercept=intercept, lambda=lambda, + standardize=standardize, alpha=alpha, thresh=1E-14) + print(as.vector(coef(model))) + } + } + } + } + [1] 0.000000 -3.292821 2.921188 + [1] 0.000000 -3.230854 2.908484 + [1] 0.000000 -3.145586 2.891014 + [1] 0.000000 -2.919246 2.841724 + [1] 0.000000 -2.938323 2.846369 + [1] 0.000000 -2.965397 2.852838 + [1] 0.000000 -2.137858 2.684464 + [1] 0.000000 -1.680094 2.590844 + [1] 0.0000000 -0.8194631 2.4151405 + [1] 0.0000000 -0.9608375 2.4301013 + [1] 0.0000000 -0.6187922 2.3634907 + [1] 0.000000 0.000000 2.240811 + [1] 0.000000 -1.346573 2.521293 + [1] 0.0000000 -0.3680456 2.3212362 + [1] 0.000000 0.000000 2.244406 + [1] 0.000000 0.000000 2.219816 + [1] 0.000000 0.000000 2.223694 + [1] 0.00000 0.00000 2.22861 + [1] 13.5631592 3.2811513 0.3725517 + [1] 13.6953934 3.3336271 0.3497454 + [1] 13.9600276 3.4600170 0.2999941 + [1] 14.2389889 3.6589920 0.2349065 + [1] 15.2374080 4.2119643 0.0325638 + [1] 15.4 4.3 0.0 + [1] 10.442365 1.246065 1.063991 + [1] 8.9580718 0.1938471 1.4090610 + [1] 8.865672 0.000000 1.455224 + [1] 13.0430927 2.4927151 0.5741805 + [1] 13.814429 2.722027 0.455915 + [1] 16.2 3.9 0.0 + [1] 9.8904768 0.7574694 1.2110177 + [1] 9.072226 0.000000 1.435363 + [1] 9.512438 0.000000 1.393035 + [1] 13.3677796 2.1721216 0.6046132 + [1] 14.2554457 2.2285185 0.5084151 + [1] 17.2 3.4 0.0 + */ + + val expected = Seq( + Vectors.dense(0, -3.2928206726474, 2.92118822588649), + Vectors.dense(0, -3.23085414359003, 2.90848366035008), + Vectors.dense(0, -3.14558628299477, 2.89101408157209), + Vectors.dense(0, -2.91924558816421, 2.84172398097327), + Vectors.dense(0, -2.93832343383477, 2.84636891947663), + Vectors.dense(0, -2.96539689593024, 2.85283836322185), + Vectors.dense(0, -2.13785756976542, 2.68446351346705), + Vectors.dense(0, -1.68009377560774, 2.59084422793154), + Vectors.dense(0, -0.819463123385533, 2.41514053108346), + Vectors.dense(0, -0.960837488151064, 2.43010130999756), + Vectors.dense(0, -0.618792151647599, 2.36349074148962), + Vectors.dense(0, 0, 2.24081114726441), + Vectors.dense(0, -1.34657309253953, 2.52129296638512), + Vectors.dense(0, -0.368045602821844, 2.32123616258871), + Vectors.dense(0, 0, 2.24440619621343), + Vectors.dense(0, 0, 2.21981559944924), + Vectors.dense(0, 0, 2.22369447413621), + Vectors.dense(0, 0, 2.22861024633605), + Vectors.dense(13.5631591827557, 3.28115132060568, 0.372551747695477), + Vectors.dense(13.6953934007661, 3.3336271417751, 0.349745414969587), + Vectors.dense(13.960027608754, 3.46001702257532, 0.29999407173994), + Vectors.dense(14.2389889013085, 3.65899196445023, 0.234906458633754), + Vectors.dense(15.2374079667397, 4.21196428071551, 0.0325637953681963), + Vectors.dense(15.4, 4.3, 0), + Vectors.dense(10.4423647474653, 1.24606545153166, 1.06399080283378), + Vectors.dense(8.95807177856822, 0.193847088148233, 1.4090609658784), + Vectors.dense(8.86567164179104, 0, 1.45522388059702), + Vectors.dense(13.0430927453034, 2.49271514356687, 0.574180477650271), + Vectors.dense(13.8144287399675, 2.72202744354555, 0.455915035859752), + Vectors.dense(16.2, 3.9, 0), + Vectors.dense(9.89047681835741, 0.757469417613661, 1.21101772561685), + Vectors.dense(9.07222551185964, 0, 1.43536293155196), + Vectors.dense(9.51243781094527, 0, 1.39303482587065), + Vectors.dense(13.3677796362763, 2.17212164262107, 0.604613180623227), + Vectors.dense(14.2554457236073, 2.22851848830683, 0.508415124978748), + Vectors.dense(17.2, 3.4, 0) + ) + + var idx = 0 + for (fitIntercept <- Seq(false, true); + regParam <- Seq(0.1, 0.5, 1.0); + standardization <- Seq(false, true); + elasticNetParam <- Seq(0.1, 0.5, 1.0)) { + val wls = new WeightedLeastSquares(fitIntercept, regParam, elasticNetParam, + standardizeFeatures = standardization, standardizeLabel = true, + solverType = WeightedLeastSquares.Auto) .fit(instances) val actual = Vectors.dense(wls.intercept, wls.coefficients(0), wls.coefficients(1)) assert(actual ~== expected(idx) absTol 1e-4) @@ -74,7 +481,7 @@ class WeightedLeastSquaresSuite extends SparkFunSuite with MLlibTestSparkContext } } - test("WLS against glmnet") { + test("WLS against glmnet with L2 regularization") { /* R code: @@ -121,12 +528,14 @@ class WeightedLeastSquaresSuite extends SparkFunSuite with MLlibTestSparkContext var idx = 0 for (fitIntercept <- Seq(false, true); regParam <- Seq(0.0, 0.1, 1.0); - standardizeFeatures <- Seq(false, true)) { - val wls = new WeightedLeastSquares( - fitIntercept, regParam, standardizeFeatures, standardizeLabel = true) - .fit(instances) - val actual = Vectors.dense(wls.intercept, wls.coefficients(0), wls.coefficients(1)) - assert(actual ~== expected(idx) absTol 1e-4) + standardization <- Seq(false, true)) { + for (solver <- WeightedLeastSquares.supportedSolvers) { + val wls = new WeightedLeastSquares(fitIntercept, regParam, elasticNetParam = 0.0, + standardizeFeatures = standardization, standardizeLabel = true, solverType = solver) + .fit(instances) + val actual = Vectors.dense(wls.intercept, wls.coefficients(0), wls.coefficients(1)) + assert(actual ~== expected(idx) absTol 1e-4) + } idx += 1 } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/DifferentiableLossAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/DifferentiableLossAggregatorSuite.scala new file mode 100644 index 000000000000..9fddf09babb0 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/DifferentiableLossAggregatorSuite.scala @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.ml.optim.aggregator + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.classification.MultiClassSummarizer +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer + +class DifferentiableLossAggregatorSuite extends SparkFunSuite { + + import DifferentiableLossAggregatorSuite.TestAggregator + + private val instances1 = Array( + Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), + Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), + Instance(2.0, 0.3, Vectors.dense(4.0, 0.5)) + ) + private val instances2 = Seq( + Instance(0.2, 0.4, Vectors.dense(0.8, 2.5)), + Instance(0.8, 0.9, Vectors.dense(2.0, 1.3)), + Instance(1.5, 0.2, Vectors.dense(3.0, 0.2)) + ) + + private def assertEqual[T, Agg <: DifferentiableLossAggregator[T, Agg]]( + agg1: DifferentiableLossAggregator[T, Agg], + agg2: DifferentiableLossAggregator[T, Agg]): Unit = { + assert(agg1.weight === agg2.weight) + assert(agg1.loss === agg2.loss) + assert(agg1.gradient === agg2.gradient) + } + + test("empty aggregator") { + val numFeatures = 5 + val coef = Vectors.dense(Array.fill(numFeatures)(1.0)) + val agg = new TestAggregator(numFeatures)(coef) + withClue("cannot get loss for empty aggregator") { + intercept[IllegalArgumentException] { + agg.loss + } + } + withClue("cannot get gradient for empty aggregator") { + intercept[IllegalArgumentException] { + agg.gradient + } + } + } + + test("aggregator initialization") { + val numFeatures = 3 + val coef = Vectors.dense(Array.fill(numFeatures)(1.0)) + val agg = new TestAggregator(numFeatures)(coef) + agg.add(Instance(1.0, 0.3, Vectors.dense(Array.fill(numFeatures)(1.0)))) + assert(agg.gradient.size === 3) + assert(agg.weight === 0.3) + } + + test("merge aggregators") { + val coefficients = Vectors.dense(0.5, -0.1) + val agg1 = new TestAggregator(2)(coefficients) + val agg2 = new TestAggregator(2)(coefficients) + val aggBadDim = new TestAggregator(1)(Vectors.dense(0.5)) + aggBadDim.add(Instance(1.0, 1.0, Vectors.dense(1.0))) + instances1.foreach(agg1.add) + + // merge incompatible aggregators + withClue("cannot merge aggregators with different dimensions") { + intercept[IllegalArgumentException] { + agg1.merge(aggBadDim) + } + } + + // merge empty other + val mergedEmptyOther = agg1.merge(agg2) + assertEqual(mergedEmptyOther, agg1) + assert(mergedEmptyOther === agg1) + + // merge empty this + val agg3 = new TestAggregator(2)(coefficients) + val mergedEmptyThis = agg3.merge(agg1) + assertEqual(mergedEmptyThis, agg1) + assert(mergedEmptyThis !== agg1) + + instances2.foreach(agg2.add) + val (loss1, weight1, grad1) = (agg1.loss, agg1.weight, agg1.gradient) + val (loss2, weight2, grad2) = (agg2.loss, agg2.weight, agg2.gradient) + val merged = agg1.merge(agg2) + + // check pointers are equal + assert(merged === agg1) + + // loss should be weighted average of the two individual losses + assert(merged.loss === (loss1 * weight1 + loss2 * weight2) / (weight1 + weight2)) + assert(merged.weight === weight1 + weight2) + + // gradient should be weighted average of individual gradients + val addedGradients = Vectors.dense(grad1.toArray.clone()) + BLAS.scal(weight1, addedGradients) + BLAS.axpy(weight2, grad2, addedGradients) + BLAS.scal(1 / (weight1 + weight2), addedGradients) + assert(merged.gradient === addedGradients) + } + + test("loss, gradient, weight") { + val coefficients = Vectors.dense(0.5, -0.1) + val agg = new TestAggregator(2)(coefficients) + instances1.foreach(agg.add) + val errors = instances1.map { case Instance(label, _, features) => + label - BLAS.dot(features, coefficients) + } + val expectedLoss = errors.zip(instances1).map { case (error: Double, instance: Instance) => + instance.weight * error * error / 2.0 + } + val expectedGradient = Vectors.dense(0.0, 0.0) + errors.zip(instances1).foreach { case (error, instance) => + BLAS.axpy(instance.weight * error, instance.features, expectedGradient) + } + BLAS.scal(1.0 / agg.weight, expectedGradient) + val weightSum = instances1.map(_.weight).sum + + assert(agg.weight ~== weightSum relTol 1e-5) + assert(agg.loss ~== expectedLoss.sum / weightSum relTol 1e-5) + assert(agg.gradient ~== expectedGradient relTol 1e-5) + } +} + +object DifferentiableLossAggregatorSuite { + /** + * Dummy aggregator that represents least squares cost with no intercept. + */ + class TestAggregator(numFeatures: Int)(coefficients: Vector) + extends DifferentiableLossAggregator[Instance, TestAggregator] { + + protected override val dim: Int = numFeatures + + override def add(instance: Instance): TestAggregator = { + val error = instance.label - BLAS.dot(coefficients, instance.features) + weightSum += instance.weight + lossSum += instance.weight * error * error / 2.0 + (0 until dim).foreach { j => + gradientSumArray(j) += instance.weight * error * instance.features(j) + } + this + } + } + + /** Get feature and label summarizers for provided data. */ + private[ml] def getRegressionSummarizers( + instances: Array[Instance]): (MultivariateOnlineSummarizer, MultivariateOnlineSummarizer) = { + val seqOp = (c: (MultivariateOnlineSummarizer, MultivariateOnlineSummarizer), + instance: Instance) => + (c._1.add(instance.features, instance.weight), + c._2.add(Vectors.dense(instance.label), instance.weight)) + + val combOp = (c1: (MultivariateOnlineSummarizer, MultivariateOnlineSummarizer), + c2: (MultivariateOnlineSummarizer, MultivariateOnlineSummarizer)) => + (c1._1.merge(c2._1), c1._2.merge(c2._2)) + + instances.aggregate( + (new MultivariateOnlineSummarizer, new MultivariateOnlineSummarizer) + )(seqOp, combOp) + } + + /** Get feature and label summarizers for provided data. */ + private[ml] def getClassificationSummarizers( + instances: Array[Instance]): (MultivariateOnlineSummarizer, MultiClassSummarizer) = { + val seqOp = (c: (MultivariateOnlineSummarizer, MultiClassSummarizer), + instance: Instance) => + (c._1.add(instance.features, instance.weight), + c._2.add(instance.label, instance.weight)) + + val combOp = (c1: (MultivariateOnlineSummarizer, MultiClassSummarizer), + c2: (MultivariateOnlineSummarizer, MultiClassSummarizer)) => + (c1._1.merge(c2._1), c1._2.merge(c2._2)) + + instances.aggregate( + (new MultivariateOnlineSummarizer, new MultiClassSummarizer) + )(seqOp, combOp) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeAggregatorSuite.scala new file mode 100644 index 000000000000..61b48ffa1094 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/HingeAggregatorSuite.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.ml.optim.aggregator + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext + +class HingeAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { + + import DifferentiableLossAggregatorSuite.getClassificationSummarizers + + @transient var instances: Array[Instance] = _ + @transient var instancesConstantFeature: Array[Instance] = _ + @transient var instancesConstantFeatureFiltered: Array[Instance] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + instances = Array( + Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), + Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), + Instance(0.0, 0.3, Vectors.dense(4.0, 0.5)) + ) + instancesConstantFeature = Array( + Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), + Instance(1.0, 0.5, Vectors.dense(1.0, 1.0)), + Instance(1.0, 0.3, Vectors.dense(1.0, 0.5)) + ) + instancesConstantFeatureFiltered = Array( + Instance(0.0, 0.1, Vectors.dense(2.0)), + Instance(1.0, 0.5, Vectors.dense(1.0)), + Instance(2.0, 0.3, Vectors.dense(0.5)) + ) + } + + /** Get summary statistics for some data and create a new HingeAggregator. */ + private def getNewAggregator( + instances: Array[Instance], + coefficients: Vector, + fitIntercept: Boolean): HingeAggregator = { + val (featuresSummarizer, ySummarizer) = + DifferentiableLossAggregatorSuite.getClassificationSummarizers(instances) + val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val bcFeaturesStd = spark.sparkContext.broadcast(featuresStd) + val bcCoefficients = spark.sparkContext.broadcast(coefficients) + new HingeAggregator(bcFeaturesStd, fitIntercept)(bcCoefficients) + } + + test("aggregator add method input size") { + val coefArray = Array(1.0, 2.0) + val interceptArray = Array(2.0) + val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ interceptArray), + fitIntercept = true) + withClue("HingeAggregator features dimension must match coefficients dimension") { + intercept[IllegalArgumentException] { + agg.add(Instance(1.0, 1.0, Vectors.dense(2.0))) + } + } + } + + test("negative weight") { + val coefArray = Array(1.0, 2.0) + val interceptArray = Array(2.0) + val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ interceptArray), + fitIntercept = true) + withClue("HingeAggregator does not support negative instance weights") { + intercept[IllegalArgumentException] { + agg.add(Instance(1.0, -1.0, Vectors.dense(2.0, 1.0))) + } + } + } + + test("check sizes") { + val rng = new scala.util.Random + val numFeatures = instances.head.features.size + val coefWithIntercept = Vectors.dense(Array.fill(numFeatures + 1)(rng.nextDouble)) + val coefWithoutIntercept = Vectors.dense(Array.fill(numFeatures)(rng.nextDouble)) + val aggIntercept = getNewAggregator(instances, coefWithIntercept, fitIntercept = true) + val aggNoIntercept = getNewAggregator(instances, coefWithoutIntercept, + fitIntercept = false) + instances.foreach(aggIntercept.add) + instances.foreach(aggNoIntercept.add) + + assert(aggIntercept.gradient.size === numFeatures + 1) + assert(aggNoIntercept.gradient.size === numFeatures) + } + + test("check correctness") { + val coefArray = Array(1.0, 2.0) + val intercept = 1.0 + val numFeatures = instances.head.features.size + val (featuresSummarizer, _) = getClassificationSummarizers(instances) + val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val weightSum = instances.map(_.weight).sum + + val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ Array(intercept)), + fitIntercept = true) + instances.foreach(agg.add) + + // compute the loss + val stdCoef = coefArray.indices.map(i => coefArray(i) / featuresStd(i)).toArray + val lossSum = instances.map { case Instance(l, w, f) => + val margin = BLAS.dot(Vectors.dense(stdCoef), f) + intercept + val labelScaled = 2 * l - 1.0 + if (1.0 > labelScaled * margin) { + (1.0 - labelScaled * margin) * w + } else { + 0.0 + } + }.sum + val loss = lossSum / weightSum + + // compute the gradients + val gradientCoef = new Array[Double](numFeatures) + var gradientIntercept = 0.0 + instances.foreach { case Instance(l, w, f) => + val margin = BLAS.dot(f, Vectors.dense(coefArray)) + intercept + if (1.0 > (2 * l - 1.0) * margin) { + gradientCoef.indices.foreach { i => + gradientCoef(i) += f(i) * -(2 * l - 1.0) * w / featuresStd(i) + } + gradientIntercept += -(2 * l - 1.0) * w + } + } + val gradient = Vectors.dense((gradientCoef ++ Array(gradientIntercept)).map(_ / weightSum)) + + assert(loss ~== agg.loss relTol 0.01) + assert(gradient ~== agg.gradient relTol 0.01) + } + + test("check with zero standard deviation") { + val binaryCoefArray = Array(1.0, 2.0) + val intercept = 1.0 + val aggConstantFeatureBinary = getNewAggregator(instancesConstantFeature, + Vectors.dense(binaryCoefArray ++ Array(intercept)), fitIntercept = true) + instancesConstantFeature.foreach(aggConstantFeatureBinary.add) + + val aggConstantFeatureBinaryFiltered = getNewAggregator(instancesConstantFeatureFiltered, + Vectors.dense(binaryCoefArray ++ Array(intercept)), fitIntercept = true) + instancesConstantFeatureFiltered.foreach(aggConstantFeatureBinaryFiltered.add) + + // constant features should not affect gradient + assert(aggConstantFeatureBinary.gradient(0) === 0.0) + assert(aggConstantFeatureBinary.gradient(1) == aggConstantFeatureBinaryFiltered.gradient(0)) + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregatorSuite.scala new file mode 100644 index 000000000000..35b694462470 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LeastSquaresAggregatorSuite.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.ml.optim.aggregator + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext + +class LeastSquaresAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { + + import DifferentiableLossAggregatorSuite.getRegressionSummarizers + + @transient var instances: Array[Instance] = _ + @transient var instancesConstantFeature: Array[Instance] = _ + @transient var instancesConstantLabel: Array[Instance] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + instances = Array( + Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), + Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), + Instance(2.0, 0.3, Vectors.dense(4.0, 0.5)) + ) + instancesConstantFeature = Array( + Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), + Instance(1.0, 0.5, Vectors.dense(1.0, 1.0)), + Instance(2.0, 0.3, Vectors.dense(1.0, 0.5)) + ) + instancesConstantLabel = Array( + Instance(1.0, 0.1, Vectors.dense(1.0, 2.0)), + Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), + Instance(1.0, 0.3, Vectors.dense(4.0, 0.5)) + ) + } + + /** Get summary statistics for some data and create a new LeastSquaresAggregator. */ + private def getNewAggregator( + instances: Array[Instance], + coefficients: Vector, + fitIntercept: Boolean): LeastSquaresAggregator = { + val (featuresSummarizer, ySummarizer) = getRegressionSummarizers(instances) + val yStd = math.sqrt(ySummarizer.variance(0)) + val yMean = ySummarizer.mean(0) + val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val bcFeaturesStd = spark.sparkContext.broadcast(featuresStd) + val featuresMean = featuresSummarizer.mean + val bcFeaturesMean = spark.sparkContext.broadcast(featuresMean.toArray) + val bcCoefficients = spark.sparkContext.broadcast(coefficients) + new LeastSquaresAggregator(yStd, yMean, fitIntercept, bcFeaturesStd, + bcFeaturesMean)(bcCoefficients) + } + + test("aggregator add method input size") { + val coefficients = Vectors.dense(1.0, 2.0) + val agg = getNewAggregator(instances, coefficients, fitIntercept = true) + withClue("LeastSquaresAggregator features dimension must match coefficients dimension") { + intercept[IllegalArgumentException] { + agg.add(Instance(1.0, 1.0, Vectors.dense(2.0))) + } + } + } + + test("negative weight") { + val coefficients = Vectors.dense(1.0, 2.0) + val agg = getNewAggregator(instances, coefficients, fitIntercept = true) + withClue("LeastSquaresAggregator does not support negative instance weights.") { + intercept[IllegalArgumentException] { + agg.add(Instance(1.0, -1.0, Vectors.dense(2.0, 1.0))) + } + } + } + + test("check sizes") { + val coefficients = Vectors.dense(1.0, 2.0) + val aggIntercept = getNewAggregator(instances, coefficients, fitIntercept = true) + val aggNoIntercept = getNewAggregator(instances, coefficients, fitIntercept = false) + instances.foreach(aggIntercept.add) + instances.foreach(aggNoIntercept.add) + + // least squares agg does not include intercept in its gradient array + assert(aggIntercept.gradient.size === 2) + assert(aggNoIntercept.gradient.size === 2) + } + + test("check correctness") { + /* + Check that the aggregator computes loss/gradient for: + 0.5 * sum_i=1^N ([sum_j=1^D beta_j * ((x_j - x_j,bar) / sigma_j)] - ((y - ybar) / sigma_y))^2 + */ + val coefficients = Vectors.dense(1.0, 2.0) + val numFeatures = coefficients.size + val (featuresSummarizer, ySummarizer) = getRegressionSummarizers(instances) + val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val featuresMean = featuresSummarizer.mean.toArray + val yStd = math.sqrt(ySummarizer.variance(0)) + val yMean = ySummarizer.mean(0) + + val agg = getNewAggregator(instances, coefficients, fitIntercept = true) + instances.foreach(agg.add) + + // compute (y - pred) analytically + val errors = instances.map { case Instance(l, w, f) => + val scaledFeatures = (0 until numFeatures).map { j => + (f.toArray(j) - featuresMean(j)) / featuresStd(j) + }.toArray + val scaledLabel = (l - yMean) / yStd + BLAS.dot(coefficients, Vectors.dense(scaledFeatures)) - scaledLabel + } + + // compute expected loss sum analytically + val expectedLoss = errors.zip(instances).map { case (error, instance) => + instance.weight * error * error / 2.0 + } + + // compute gradient analytically from instances + val expectedGradient = Vectors.dense(0.0, 0.0) + errors.zip(instances).foreach { case (error, instance) => + val scaledFeatures = (0 until numFeatures).map { j => + instance.weight * instance.features.toArray(j) / featuresStd(j) + }.toArray + BLAS.axpy(error, Vectors.dense(scaledFeatures), expectedGradient) + } + + val weightSum = instances.map(_.weight).sum + BLAS.scal(1.0 / weightSum, expectedGradient) + assert(agg.loss ~== (expectedLoss.sum / weightSum) relTol 1e-5) + assert(agg.gradient ~== expectedGradient relTol 1e-5) + } + + test("check with zero standard deviation") { + val coefficients = Vectors.dense(1.0, 2.0) + val aggConstantFeature = getNewAggregator(instancesConstantFeature, coefficients, + fitIntercept = true) + instances.foreach(aggConstantFeature.add) + // constant features should not affect gradient + assert(aggConstantFeature.gradient(0) === 0.0) + + withClue("LeastSquaresAggregator does not support zero standard deviation of the label") { + intercept[IllegalArgumentException] { + getNewAggregator(instancesConstantLabel, coefficients, fitIntercept = true) + } + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala new file mode 100644 index 000000000000..4c7913d5d257 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/aggregator/LogisticAggregatorSuite.scala @@ -0,0 +1,280 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.ml.optim.aggregator + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{BLAS, Matrices, Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext + +class LogisticAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { + + import DifferentiableLossAggregatorSuite.getClassificationSummarizers + + @transient var instances: Array[Instance] = _ + @transient var instancesConstantFeature: Array[Instance] = _ + @transient var instancesConstantFeatureFiltered: Array[Instance] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + instances = Array( + Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), + Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), + Instance(2.0, 0.3, Vectors.dense(4.0, 0.5)) + ) + instancesConstantFeature = Array( + Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), + Instance(1.0, 0.5, Vectors.dense(1.0, 1.0)), + Instance(2.0, 0.3, Vectors.dense(1.0, 0.5)) + ) + instancesConstantFeatureFiltered = Array( + Instance(0.0, 0.1, Vectors.dense(2.0)), + Instance(1.0, 0.5, Vectors.dense(1.0)), + Instance(2.0, 0.3, Vectors.dense(0.5)) + ) + } + + /** Get summary statistics for some data and create a new LogisticAggregator. */ + private def getNewAggregator( + instances: Array[Instance], + coefficients: Vector, + fitIntercept: Boolean, + isMultinomial: Boolean): LogisticAggregator = { + val (featuresSummarizer, ySummarizer) = + DifferentiableLossAggregatorSuite.getClassificationSummarizers(instances) + val numClasses = ySummarizer.histogram.length + val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val bcFeaturesStd = spark.sparkContext.broadcast(featuresStd) + val bcCoefficients = spark.sparkContext.broadcast(coefficients) + new LogisticAggregator(bcFeaturesStd, numClasses, fitIntercept, isMultinomial)(bcCoefficients) + } + + test("aggregator add method input size") { + val coefArray = Array(1.0, 2.0, -2.0, 3.0, 0.0, -1.0) + val interceptArray = Array(4.0, 2.0, -3.0) + val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ interceptArray), + fitIntercept = true, isMultinomial = true) + withClue("LogisticAggregator features dimension must match coefficients dimension") { + intercept[IllegalArgumentException] { + agg.add(Instance(1.0, 1.0, Vectors.dense(2.0))) + } + } + } + + test("negative weight") { + val coefArray = Array(1.0, 2.0, -2.0, 3.0, 0.0, -1.0) + val interceptArray = Array(4.0, 2.0, -3.0) + val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ interceptArray), + fitIntercept = true, isMultinomial = true) + withClue("LogisticAggregator does not support negative instance weights") { + intercept[IllegalArgumentException] { + agg.add(Instance(1.0, -1.0, Vectors.dense(2.0, 1.0))) + } + } + } + + test("check sizes multinomial") { + val rng = new scala.util.Random + val numFeatures = instances.head.features.size + val numClasses = instances.map(_.label).toSet.size + val coefWithIntercept = Vectors.dense( + Array.fill(numClasses * (numFeatures + 1))(rng.nextDouble)) + val coefWithoutIntercept = Vectors.dense( + Array.fill(numClasses * numFeatures)(rng.nextDouble)) + val aggIntercept = getNewAggregator(instances, coefWithIntercept, fitIntercept = true, + isMultinomial = true) + val aggNoIntercept = getNewAggregator(instances, coefWithoutIntercept, fitIntercept = false, + isMultinomial = true) + instances.foreach(aggIntercept.add) + instances.foreach(aggNoIntercept.add) + + assert(aggIntercept.gradient.size === (numFeatures + 1) * numClasses) + assert(aggNoIntercept.gradient.size === numFeatures * numClasses) + } + + test("check sizes binomial") { + val rng = new scala.util.Random + val binaryInstances = instances.filter(_.label < 2.0) + val numFeatures = binaryInstances.head.features.size + val coefWithIntercept = Vectors.dense(Array.fill(numFeatures + 1)(rng.nextDouble)) + val coefWithoutIntercept = Vectors.dense(Array.fill(numFeatures)(rng.nextDouble)) + val aggIntercept = getNewAggregator(binaryInstances, coefWithIntercept, fitIntercept = true, + isMultinomial = false) + val aggNoIntercept = getNewAggregator(binaryInstances, coefWithoutIntercept, + fitIntercept = false, isMultinomial = false) + binaryInstances.foreach(aggIntercept.add) + binaryInstances.foreach(aggNoIntercept.add) + + assert(aggIntercept.gradient.size === numFeatures + 1) + assert(aggNoIntercept.gradient.size === numFeatures) + } + + test("check correctness multinomial") { + /* + Check that the aggregator computes loss/gradient for: + -sum_i w_i * (beta_y dot x_i - log(sum_k e^(beta_k dot x_i))) + */ + val coefArray = Array(1.0, 2.0, -2.0, 3.0, 0.0, -1.0) + val interceptArray = Array(4.0, 2.0, -3.0) + val numFeatures = instances.head.features.size + val numClasses = instances.map(_.label).toSet.size + val intercepts = Vectors.dense(interceptArray) + val (featuresSummarizer, ySummarizer) = getClassificationSummarizers(instances) + val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val weightSum = instances.map(_.weight).sum + + val agg = getNewAggregator(instances, Vectors.dense(coefArray ++ interceptArray), + fitIntercept = true, isMultinomial = true) + instances.foreach(agg.add) + + // compute the loss + val stdCoef = coefArray.indices.map(i => coefArray(i) / featuresStd(i / numClasses)).toArray + val linearPredictors = instances.map { case Instance(l, w, f) => + val result = intercepts.copy.toDense + BLAS.gemv(1.0, Matrices.dense(numClasses, numFeatures, stdCoef), f, 1.0, result) + (l, w, result) + } + + // sum_i w * beta_k dot x_i + val sumLinear = linearPredictors.map { case (l, w, p) => + w * p(l.toInt) + }.sum + + // sum_i w * log(sum_k e^(beta_K dot x_i)) + val sumLogs = linearPredictors.map { case (l, w, p) => + w * math.log(p.values.map(math.exp).sum) + }.sum + val loss = (sumLogs - sumLinear) / weightSum + + + // compute the gradients + val gradientCoef = new Array[Double](numFeatures * numClasses) + val gradientIntercept = new Array[Double](numClasses) + instances.foreach { case Instance(l, w, f) => + val margin = intercepts.copy.toDense + BLAS.gemv(1.0, Matrices.dense(numClasses, numFeatures, stdCoef), f, 1.0, margin) + val sum = margin.values.map(math.exp).sum + + gradientCoef.indices.foreach { i => + val fStd = f(i / numClasses) / featuresStd(i / numClasses) + val cidx = i % numClasses + if (cidx == l.toInt) gradientCoef(i) -= w * fStd + gradientCoef(i) += w * math.exp(margin(cidx)) / sum * fStd + } + + gradientIntercept.indices.foreach { i => + val cidx = i % numClasses + if (cidx == l.toInt) gradientIntercept(i) -= w + gradientIntercept(i) += w * math.exp(margin(cidx)) / sum + } + } + val gradient = Vectors.dense((gradientCoef ++ gradientIntercept).map(_ / weightSum)) + + assert(loss ~== agg.loss relTol 0.01) + assert(gradient ~== agg.gradient relTol 0.01) + } + + test("check correctness binomial") { + /* + Check that the aggregator computes loss/gradient for: + -sum_i y_i * log(1 / (1 + e^(-beta dot x_i)) + (1 - y_i) * log(1 - 1 / (1 + e^(-beta dot x_i)) + */ + val binaryInstances = instances.map { instance => + if (instance.label <= 1.0) instance else Instance(0.0, instance.weight, instance.features) + } + val coefArray = Array(1.0, 2.0) + val intercept = 1.0 + val numFeatures = binaryInstances.head.features.size + val (featuresSummarizer, _) = getClassificationSummarizers(binaryInstances) + val featuresStd = featuresSummarizer.variance.toArray.map(math.sqrt) + val weightSum = binaryInstances.map(_.weight).sum + + val agg = getNewAggregator(binaryInstances, Vectors.dense(coefArray ++ Array(intercept)), + fitIntercept = true, isMultinomial = false) + binaryInstances.foreach(agg.add) + + // compute the loss + val stdCoef = coefArray.indices.map(i => coefArray(i) / featuresStd(i)).toArray + val lossSum = binaryInstances.map { case Instance(l, w, f) => + val margin = BLAS.dot(Vectors.dense(stdCoef), f) + intercept + val prob = 1.0 / (1.0 + math.exp(-margin)) + -w * l * math.log(prob) - w * (1.0 - l) * math.log(1.0 - prob) + }.sum + val loss = lossSum / weightSum + + // compute the gradients + val gradientCoef = new Array[Double](numFeatures) + var gradientIntercept = 0.0 + binaryInstances.foreach { case Instance(l, w, f) => + val margin = BLAS.dot(f, Vectors.dense(coefArray)) + intercept + gradientCoef.indices.foreach { i => + gradientCoef(i) += w * (1.0 / (1.0 + math.exp(-margin)) - l) * f(i) / featuresStd(i) + } + gradientIntercept += w * (1.0 / (1.0 + math.exp(-margin)) - l) + } + val gradient = Vectors.dense((gradientCoef ++ Array(gradientIntercept)).map(_ / weightSum)) + + assert(loss ~== agg.loss relTol 0.01) + assert(gradient ~== agg.gradient relTol 0.01) + } + + test("check with zero standard deviation") { + val binaryInstances = instancesConstantFeature.map { instance => + if (instance.label <= 1.0) instance else Instance(0.0, instance.weight, instance.features) + } + val binaryInstancesFiltered = instancesConstantFeatureFiltered.map { instance => + if (instance.label <= 1.0) instance else Instance(0.0, instance.weight, instance.features) + } + val coefArray = Array(1.0, 2.0, -2.0, 3.0, 0.0, -1.0) + val coefArrayFiltered = Array(3.0, 0.0, -1.0) + val interceptArray = Array(4.0, 2.0, -3.0) + val aggConstantFeature = getNewAggregator(instancesConstantFeature, + Vectors.dense(coefArray ++ interceptArray), fitIntercept = true, isMultinomial = true) + val aggConstantFeatureFiltered = getNewAggregator(instancesConstantFeatureFiltered, + Vectors.dense(coefArrayFiltered ++ interceptArray), fitIntercept = true, isMultinomial = true) + + instancesConstantFeature.foreach(aggConstantFeature.add) + instancesConstantFeatureFiltered.foreach(aggConstantFeatureFiltered.add) + + // constant features should not affect gradient + def validateGradient(grad: Vector, gradFiltered: Vector, numCoefficientSets: Int): Unit = { + for (i <- 0 until numCoefficientSets) { + assert(grad(i) === 0.0) + assert(grad(numCoefficientSets + i) == gradFiltered(i)) + } + } + + validateGradient(aggConstantFeature.gradient, aggConstantFeatureFiltered.gradient, 3) + + val binaryCoefArray = Array(1.0, 2.0) + val binaryCoefArrayFiltered = Array(2.0) + val intercept = 1.0 + val aggConstantFeatureBinary = getNewAggregator(binaryInstances, + Vectors.dense(binaryCoefArray ++ Array(intercept)), fitIntercept = true, + isMultinomial = false) + val aggConstantFeatureBinaryFiltered = getNewAggregator(binaryInstancesFiltered, + Vectors.dense(binaryCoefArrayFiltered ++ Array(intercept)), fitIntercept = true, + isMultinomial = false) + binaryInstances.foreach(aggConstantFeatureBinary.add) + binaryInstancesFiltered.foreach(aggConstantFeatureBinaryFiltered.add) + + // constant features should not affect gradient + validateGradient(aggConstantFeatureBinary.gradient, + aggConstantFeatureBinaryFiltered.gradient, 1) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/loss/DifferentiableRegularizationSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/loss/DifferentiableRegularizationSuite.scala new file mode 100644 index 000000000000..4377a6bd75db --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/loss/DifferentiableRegularizationSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.ml.optim.loss + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{BLAS, Vectors} + +class DifferentiableRegularizationSuite extends SparkFunSuite { + + test("L2 regularization") { + val shouldApply = (_: Int) => true + val regParam = 0.3 + val coefficients = Vectors.dense(Array(1.0, 3.0, -2.0)) + val numFeatures = coefficients.size + + // check without features standard + val regFun = new L2Regularization(regParam, shouldApply, None) + val (loss, grad) = regFun.calculate(coefficients) + assert(loss === 0.5 * regParam * BLAS.dot(coefficients, coefficients)) + assert(grad === Vectors.dense(coefficients.toArray.map(_ * regParam))) + + // check with features standard + val featuresStd = Array(0.1, 1.1, 0.5) + val regFunStd = new L2Regularization(regParam, shouldApply, Some(featuresStd)) + val (lossStd, gradStd) = regFunStd.calculate(coefficients) + val expectedLossStd = 0.5 * regParam * (0 until numFeatures).map { j => + coefficients(j) * coefficients(j) / (featuresStd(j) * featuresStd(j)) + }.sum + val expectedGradientStd = Vectors.dense((0 until numFeatures).map { j => + regParam * coefficients(j) / (featuresStd(j) * featuresStd(j)) + }.toArray) + assert(lossStd === expectedLossStd) + assert(gradStd === expectedGradientStd) + + // check should apply + val shouldApply2 = (i: Int) => i == 1 + val regFunApply = new L2Regularization(regParam, shouldApply2, None) + val (lossApply, gradApply) = regFunApply.calculate(coefficients) + assert(lossApply === 0.5 * regParam * coefficients(1) * coefficients(1)) + assert(gradApply === Vectors.dense(0.0, coefficients(1) * regParam, 0.0)) + + // check with zero features standard + val featuresStdZero = Array(0.1, 0.0, 0.5) + val regFunStdZero = new L2Regularization(regParam, shouldApply, Some(featuresStdZero)) + val (_, gradStdZero) = regFunStdZero.calculate(coefficients) + assert(gradStdZero(1) == 0.0) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/optim/loss/RDDLossFunctionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/optim/loss/RDDLossFunctionSuite.scala new file mode 100644 index 000000000000..f70da5750f2d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/optim/loss/RDDLossFunctionSuite.scala @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.ml.optim.loss + +import org.apache.spark.SparkFunSuite +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors} +import org.apache.spark.ml.optim.aggregator.DifferentiableLossAggregatorSuite.TestAggregator +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.rdd.RDD + +class RDDLossFunctionSuite extends SparkFunSuite with MLlibTestSparkContext { + + @transient var instances: RDD[Instance] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + instances = sc.parallelize(Seq( + Instance(0.0, 0.1, Vectors.dense(1.0, 2.0)), + Instance(1.0, 0.5, Vectors.dense(1.5, 1.0)), + Instance(2.0, 0.3, Vectors.dense(4.0, 0.5)) + )) + } + + test("regularization") { + val coefficients = Vectors.dense(0.5, -0.1) + val regLossFun = new L2Regularization(0.1, (_: Int) => true, None) + val getAgg = (bvec: Broadcast[Vector]) => new TestAggregator(2)(bvec.value) + val lossNoReg = new RDDLossFunction(instances, getAgg, None) + val lossWithReg = new RDDLossFunction(instances, getAgg, Some(regLossFun)) + + val (loss1, grad1) = lossNoReg.calculate(coefficients.asBreeze.toDenseVector) + val (regLoss, regGrad) = regLossFun.calculate(coefficients) + val (loss2, grad2) = lossWithReg.calculate(coefficients.asBreeze.toDenseVector) + + BLAS.axpy(1.0, Vectors.fromBreeze(grad1), regGrad) + assert(regGrad ~== Vectors.fromBreeze(grad2) relTol 1e-5) + assert(loss1 + regLoss === loss2) + } + + test("empty RDD") { + val rdd = sc.parallelize(Seq.empty[Instance]) + val coefficients = Vectors.dense(0.5, -0.1) + val getAgg = (bv: Broadcast[Vector]) => new TestAggregator(2)(bv.value) + val lossFun = new RDDLossFunction(rdd, getAgg, None) + withClue("cannot calculate cost for empty dataset") { + intercept[IllegalArgumentException]{ + lossFun.calculate(coefficients.asBreeze.toDenseVector) + } + } + } + + test("versus aggregating on an iterable") { + val coefficients = Vectors.dense(0.5, -0.1) + val getAgg = (bv: Broadcast[Vector]) => new TestAggregator(2)(bv.value) + val lossFun = new RDDLossFunction(instances, getAgg, None) + val (loss, grad) = lossFun.calculate(coefficients.asBreeze.toDenseVector) + + // just map the aggregator over the instances array + val agg = new TestAggregator(2)(coefficients) + instances.collect().foreach(agg.add) + + assert(loss === agg.loss) + assert(Vectors.fromBreeze(grad) === agg.gradient) + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala index eeb03dba2f82..78a33e05e0e4 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala @@ -17,7 +17,11 @@ package org.apache.spark.ml.param +import java.io.{ByteArrayOutputStream, ObjectOutputStream} + import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.MyParams class ParamsSuite extends SparkFunSuite { @@ -80,7 +84,7 @@ class ParamsSuite extends SparkFunSuite { } } - { // StringParam + { // Param[String] val param = new Param[String](dummy, "name", "doc") // Currently we do not support null. for (value <- Seq("", "1", "abc", "quote\"", "newline\n")) { @@ -89,6 +93,19 @@ class ParamsSuite extends SparkFunSuite { } } + { // Param[Vector] + val param = new Param[Vector](dummy, "name", "doc") + val values = Seq( + Vectors.dense(Array.empty[Double]), + Vectors.dense(0.0, 2.0), + Vectors.sparse(0, Array.empty, Array.empty), + Vectors.sparse(2, Array(1), Array(2.0))) + for (value <- values) { + val json = param.jsonEncode(value) + assert(param.jsonDecode(json) === value) + } + } + { // IntArrayParam val param = new IntArrayParam(dummy, "name", "doc") val values: Seq[Array[Int]] = Seq( @@ -138,7 +155,7 @@ class ParamsSuite extends SparkFunSuite { test("param") { val solver = new TestParams() val uid = solver.uid - import solver.{maxIter, inputCol} + import solver.{inputCol, maxIter} assert(maxIter.name === "maxIter") assert(maxIter.doc === "maximum number of iterations (>= 0)") @@ -181,7 +198,7 @@ class ParamsSuite extends SparkFunSuite { test("param map") { val solver = new TestParams() - import solver.{maxIter, inputCol} + import solver.{inputCol, maxIter} val map0 = ParamMap.empty @@ -220,7 +237,7 @@ class ParamsSuite extends SparkFunSuite { test("params") { val solver = new TestParams() - import solver.{handleInvalid, maxIter, inputCol} + import solver.{handleInvalid, inputCol, maxIter} val params = solver.params assert(params.length === 3) @@ -251,15 +268,10 @@ class ParamsSuite extends SparkFunSuite { solver.getParam("abc") } - intercept[IllegalArgumentException] { - solver.validateParams() - } - solver.copy(ParamMap(inputCol -> "input")).validateParams() solver.setInputCol("input") assert(solver.isSet(inputCol)) assert(solver.isDefined(inputCol)) assert(solver.getInputCol === "input") - solver.validateParams() intercept[IllegalArgumentException] { ParamMap(maxIter -> -10) } @@ -335,12 +347,37 @@ class ParamsSuite extends SparkFunSuite { val t3 = t.copy(ParamMap(t.maxIter -> 20)) assert(t3.isSet(t3.maxIter)) } + + test("Filtering ParamMap") { + val params1 = new MyParams("my_params1") + val params2 = new MyParams("my_params2") + val paramMap = ParamMap( + params1.intParam -> 1, + params2.intParam -> 1, + params1.doubleParam -> 0.2, + params2.doubleParam -> 0.2) + val filteredParamMap = paramMap.filter(params1) + + assert(filteredParamMap.size === 2) + filteredParamMap.toSeq.foreach { + case ParamPair(p, _) => + assert(p.parent === params1.uid) + } + + // At the previous implementation of ParamMap#filter, + // mutable.Map#filterKeys was used internally but + // the return type of the method is not serializable (see SI-6654). + // Now mutable.Map#filter is used instead of filterKeys and the return type is serializable. + // So let's ensure serializability. + val objOut = new ObjectOutputStream(new ByteArrayOutputStream()) + objOut.writeObject(filteredParamMap) + } } object ParamsSuite extends SparkFunSuite { /** - * Checks common requirements for [[Params.params]]: + * Checks common requirements for `Params.params`: * - params are ordered by names * - param parent has the same UID as the object's UID * - param name is the same as the param method name diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala index 9d23547f2844..7d990ce0bcfd 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala @@ -34,10 +34,5 @@ class TestParams(override val uid: String) extends Params with HasHandleInvalid def clearMaxIter(): this.type = clear(maxIter) - override def validateParams(): Unit = { - super.validateParams() - require(isDefined(inputCol)) - } - override def copy(extra: ParamMap): TestParams = defaultCopy(extra) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/python/MLSerDeSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/python/MLSerDeSuite.scala new file mode 100644 index 000000000000..3bb760f2ecc1 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/python/MLSerDeSuite.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.python + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{DenseMatrix, Matrices, SparseMatrix, Vectors} + +class MLSerDeSuite extends SparkFunSuite { + + MLSerDe.initialize() + + test("pickle vector") { + val vectors = Seq( + Vectors.dense(Array.empty[Double]), + Vectors.dense(0.0), + Vectors.dense(0.0, -2.0), + Vectors.sparse(0, Array.empty[Int], Array.empty[Double]), + Vectors.sparse(1, Array.empty[Int], Array.empty[Double]), + Vectors.sparse(2, Array(1), Array(-2.0))) + vectors.foreach { v => + val u = MLSerDe.loads(MLSerDe.dumps(v)) + assert(u.getClass === v.getClass) + assert(u === v) + } + } + + test("pickle double") { + for (x <- List(123.0, -10.0, 0.0, Double.MaxValue, Double.MinValue, Double.NaN)) { + val deser = MLSerDe.loads(MLSerDe.dumps(x.asInstanceOf[AnyRef])).asInstanceOf[Double] + // We use `equals` here for comparison because we cannot use `==` for NaN + assert(x.equals(deser)) + } + } + + test("pickle matrix") { + val values = Array[Double](0, 1.2, 3, 4.56, 7, 8) + val matrix = Matrices.dense(2, 3, values) + val nm = MLSerDe.loads(MLSerDe.dumps(matrix)).asInstanceOf[DenseMatrix] + assert(matrix === nm) + + // Test conversion for empty matrix + val empty = Array.empty[Double] + val emptyMatrix = Matrices.dense(0, 0, empty) + val ne = MLSerDe.loads(MLSerDe.dumps(emptyMatrix)).asInstanceOf[DenseMatrix] + assert(emptyMatrix == ne) + + val sm = new SparseMatrix(3, 2, Array(0, 1, 3), Array(1, 0, 2), Array(0.9, 1.2, 3.4)) + val nsm = MLSerDe.loads(MLSerDe.dumps(sm)).asInstanceOf[SparseMatrix] + assert(sm.toArray === nsm.toArray) + + val smt = new SparseMatrix( + 3, 3, Array(0, 2, 3, 5), Array(0, 2, 1, 0, 2), Array(0.9, 1.2, 3.4, 5.7, 8.9), + isTransposed = true) + val nsmt = MLSerDe.loads(MLSerDe.dumps(smt)).asInstanceOf[SparseMatrix] + assert(smt.toArray === nsmt.toArray) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/r/RWrapperUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/r/RWrapperUtilsSuite.scala new file mode 100644 index 000000000000..27b03918d951 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/r/RWrapperUtilsSuite.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.r + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.{RFormula, RFormulaModel} +import org.apache.spark.mllib.util.MLlibTestSparkContext + +class RWrapperUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { + + test("avoid libsvm data column name conflicting") { + val rFormula = new RFormula().setFormula("label ~ features") + val data = spark.read.format("libsvm").load("../data/mllib/sample_libsvm_data.txt") + + // if not checking column name, then IllegalArgumentException + intercept[IllegalArgumentException] { + rFormula.fit(data) + } + + // after checking, model build is ok + RWrapperUtils.checkDataColumns(rFormula, data) + + assert(rFormula.getLabelCol == "label") + assert(rFormula.getFeaturesCol.startsWith("features_")) + + val model = rFormula.fit(data) + assert(model.isInstanceOf[RFormulaModel]) + + assert(model.getLabelCol == "label") + assert(model.getFeaturesCol.startsWith("features_")) + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index eadc80e0e62b..ac7319110159 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 @@ -20,34 +20,43 @@ package org.apache.spark.ml.recommendation import java.io.File import java.util.Random +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.WrappedArray import scala.language.existentials import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.commons.io.FileUtils +import org.apache.commons.io.filefilter.TrueFileFilter +import org.scalatest.BeforeAndAfterEach -import org.apache.spark.{Logging, SparkException, SparkFunSuite} +import org.apache.spark._ +import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.recommendation.ALS._ -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.recommendation.ALS.Rating +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.recommendation.MatrixFactorizationModelSuite import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted} +import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.types._ +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils -class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { - - private var tempDir: File = _ +class ALSSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest with Logging { override def beforeAll(): Unit = { super.beforeAll() - tempDir = Utils.createTempDir() sc.setCheckpointDir(tempDir.getAbsolutePath) } override def afterAll(): Unit = { - Utils.deleteRecursively(tempDir) super.afterAll() } @@ -71,7 +80,7 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { val k = 2 val ne0 = new NormalEquation(k) .add(Array(1.0f, 2.0f), 3.0) - .add(Array(4.0f, 5.0f), 6.0, 2.0) // weighted + .add(Array(4.0f, 5.0f), 12.0, 2.0) // weighted assert(ne0.k === k) assert(ne0.triK === k * (k + 1) / 2) // NumPy code that computes the expected values: @@ -186,7 +195,7 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { assert(compressed.dstPtrs.toSeq === Seq(0, 2, 3, 4, 5)) var decompressed = ArrayBuffer.empty[(Int, Int, Int, Float)] var i = 0 - while (i < compressed.srcIds.size) { + while (i < compressed.srcIds.length) { var j = compressed.dstPtrs(i) while (j < compressed.dstPtrs(i + 1)) { val dstEncodedIndex = compressed.dstEncodedIndices(j) @@ -200,6 +209,70 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { assert(decompressed.toSet === expected) } + test("CheckedCast") { + val checkedCast = new ALS().checkedCast + val df = spark.range(1) + + withClue("Valid Integer Ids") { + df.select(checkedCast(lit(123))).collect() + } + + withClue("Valid Long Ids") { + df.select(checkedCast(lit(1231L))).collect() + } + + withClue("Valid Decimal Ids") { + df.select(checkedCast(lit(123).cast(DecimalType(15, 2)))).collect() + } + + withClue("Valid Double Ids") { + df.select(checkedCast(lit(123.0))).collect() + } + + val msg = "either out of Integer range or contained a fractional part" + withClue("Invalid Long: out of range") { + val e: SparkException = intercept[SparkException] { + df.select(checkedCast(lit(1231000000000L))).collect() + } + assert(e.getMessage.contains(msg)) + } + + withClue("Invalid Decimal: out of range") { + val e: SparkException = intercept[SparkException] { + df.select(checkedCast(lit(1231000000000.0).cast(DecimalType(15, 2)))).collect() + } + assert(e.getMessage.contains(msg)) + } + + withClue("Invalid Decimal: fractional part") { + val e: SparkException = intercept[SparkException] { + df.select(checkedCast(lit(123.1).cast(DecimalType(15, 2)))).collect() + } + assert(e.getMessage.contains(msg)) + } + + withClue("Invalid Double: out of range") { + val e: SparkException = intercept[SparkException] { + df.select(checkedCast(lit(1231000000000.0))).collect() + } + assert(e.getMessage.contains(msg)) + } + + withClue("Invalid Double: fractional part") { + val e: SparkException = intercept[SparkException] { + df.select(checkedCast(lit(123.1))).collect() + } + assert(e.getMessage.contains(msg)) + } + + withClue("Invalid Type") { + val e: SparkException = intercept[SparkException] { + df.select(checkedCast(lit("123.1"))).collect() + } + assert(e.getMessage.contains("was not numeric")) + } + } + /** * Generates an explicit feedback dataset for testing ALS. * @param numUsers number of users @@ -255,37 +328,7 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { rank: Int, noiseStd: Double = 0.0, seed: Long = 11L): (RDD[Rating[Int]], RDD[Rating[Int]]) = { - // The assumption of the implicit feedback model is that unobserved ratings are more likely to - // be negatives. - val positiveFraction = 0.8 - val negativeFraction = 1.0 - positiveFraction - val trainingFraction = 0.6 - val testFraction = 0.3 - val totalFraction = trainingFraction + testFraction - val random = new Random(seed) - val userFactors = genFactors(numUsers, rank, random) - val itemFactors = genFactors(numItems, rank, random) - val training = ArrayBuffer.empty[Rating[Int]] - val test = ArrayBuffer.empty[Rating[Int]] - for ((userId, userFactor) <- userFactors; (itemId, itemFactor) <- itemFactors) { - val rating = blas.sdot(rank, userFactor, 1, itemFactor, 1) - val threshold = if (rating > 0) positiveFraction else negativeFraction - val observed = random.nextDouble() < threshold - if (observed) { - val x = random.nextDouble() - if (x < totalFraction) { - if (x < trainingFraction) { - val noise = noiseStd * random.nextGaussian() - training += Rating(userId, itemId, rating + noise.toFloat) - } else { - test += Rating(userId, itemId, rating) - } - } - } - } - logInfo(s"Generated an implicit feedback dataset with ${training.size} ratings for training " + - s"and ${test.size} for test.") - (sc.parallelize(training, 2), sc.parallelize(test, 2)) + ALSSuite.genImplicitTestData(sc, numUsers, numItems, rank, noiseStd, seed) } /** @@ -303,14 +346,38 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { random: Random, a: Float = -1.0f, b: Float = 1.0f): Seq[(Int, Array[Float])] = { - require(size > 0 && size < Int.MaxValue / 3) - require(b > a) - val ids = mutable.Set.empty[Int] - while (ids.size < size) { - ids += random.nextInt() - } - val width = b - a - ids.toSeq.sorted.map(id => (id, Array.fill(rank)(a + random.nextFloat() * width))) + ALSSuite.genFactors(size, rank, random, a, b) + } + + /** + * Train ALS using the given training set and parameters + * @param training training dataset + * @param rank rank of the matrix factorization + * @param maxIter max number of iterations + * @param regParam regularization constant + * @param implicitPrefs whether to use implicit preference + * @param numUserBlocks number of user blocks + * @param numItemBlocks number of item blocks + * @return a trained ALSModel + */ + def trainALS( + training: RDD[Rating[Int]], + rank: Int, + maxIter: Int, + regParam: Double, + implicitPrefs: Boolean = false, + numUserBlocks: Int = 2, + numItemBlocks: Int = 3): ALSModel = { + val spark = this.spark + import spark.implicits._ + val als = new ALS() + .setRank(rank) + .setRegParam(regParam) + .setImplicitPrefs(implicitPrefs) + .setNumUserBlocks(numUserBlocks) + .setNumItemBlocks(numItemBlocks) + .setSeed(0) + als.fit(training.toDF()) } /** @@ -335,8 +402,8 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { numUserBlocks: Int = 2, numItemBlocks: Int = 3, targetRMSE: Double = 0.05): Unit = { - val sqlContext = this.sqlContext - import sqlContext.implicits._ + val spark = this.spark + import spark.implicits._ val als = new ALS() .setRank(rank) .setRegParam(regParam) @@ -346,11 +413,10 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { .setSeed(0) val alpha = als.getAlpha val model = als.fit(training.toDF()) - val predictions = model.transform(test.toDF()) - .select("rating", "prediction") - .map { case Row(rating: Float, prediction: Float) => + val predictions = model.transform(test.toDF()).select("rating", "prediction").rdd.map { + case Row(rating: Float, prediction: Float) => (rating.toDouble, prediction.toDouble) - } + } val rmse = if (implicitPrefs) { // TODO: Use a better (rank-based?) evaluation metric for implicit feedback. @@ -376,8 +442,7 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { logInfo(s"Test RMSE is $rmse.") assert(rmse < targetRMSE) - // copied model must have the same parent. - MLTestingUtils.checkCopy(model) + MLTestingUtils.checkCopyAndUids(als, model) } test("exact rank-1 matrix") { @@ -423,6 +488,20 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { targetRMSE = 0.3) } + test("implicit feedback regression") { + val trainingWithNeg = sc.parallelize(Array(Rating(0, 0, 1), Rating(1, 1, 1), Rating(0, 1, -3))) + val trainingWithZero = sc.parallelize(Array(Rating(0, 0, 1), Rating(1, 1, 1), Rating(0, 1, 0))) + val modelWithNeg = + trainALS(trainingWithNeg, rank = 1, maxIter = 5, regParam = 0.01, implicitPrefs = true) + val modelWithZero = + trainALS(trainingWithZero, rank = 1, maxIter = 5, regParam = 0.01, implicitPrefs = true) + val userFactorsNeg = modelWithNeg.userFactors + val itemFactorsNeg = modelWithNeg.itemFactors + val userFactorsZero = modelWithZero.userFactors + val itemFactorsZero = modelWithZero.itemFactors + assert(userFactorsNeg.intersect(userFactorsZero).count() == 0) + assert(itemFactorsNeg.intersect(itemFactorsZero).count() == 0) + } test("using generic ID types") { val (ratings, _) = genImplicitTestData(numUsers = 20, numItems = 40, rank = 2, noiseStd = 0.01) @@ -483,4 +562,486 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { ALS.train(ratings, rank = 1, maxIter = 50, numUserBlocks = 2, numItemBlocks = 2, implicitPrefs = true, seed = 0) } + + test("read/write") { + val spark = this.spark + import spark.implicits._ + import ALSSuite._ + val (ratings, _) = genExplicitTestData(numUsers = 4, numItems = 4, rank = 1) + + def getFactors(df: DataFrame): Set[(Int, Array[Float])] = { + df.select("id", "features").collect().map { case r => + (r.getInt(0), r.getAs[Array[Float]](1)) + }.toSet + } + + def checkModelData(model: ALSModel, model2: ALSModel): Unit = { + assert(model.rank === model2.rank) + assert(getFactors(model.userFactors) === getFactors(model2.userFactors)) + assert(getFactors(model.itemFactors) === getFactors(model2.itemFactors)) + } + + val als = new ALS() + testEstimatorAndModelReadWrite(als, ratings.toDF(), allEstimatorParamSettings, + allModelParamSettings, checkModelData) + } + + test("input type validation") { + val spark = this.spark + import spark.implicits._ + + // check that ALS can handle all numeric types for rating column + // and user/item columns (when the user/item ids are within Int range) + val als = new ALS().setMaxIter(1).setRank(1) + Seq(("user", IntegerType), ("item", IntegerType), ("rating", FloatType)).foreach { + case (colName, sqlType) => + MLTestingUtils.checkNumericTypesALS(als, spark, colName, sqlType) { + (ex, act) => + ex.userFactors.first().getSeq[Float](1) === act.userFactors.first.getSeq[Float](1) + } { (ex, act, _) => + ex.transform(_: DataFrame).select("prediction").first.getDouble(0) ~== + act.transform(_: DataFrame).select("prediction").first.getDouble(0) absTol 1e-6 + } + } + // check user/item ids falling outside of Int range + val big = Int.MaxValue.toLong + 1 + val small = Int.MinValue.toDouble - 1 + val df = Seq( + (0, 0L, 0d, 1, 1L, 1d, 3.0), + (0, big, small, 0, big, small, 2.0), + (1, 1L, 1d, 0, 0L, 0d, 5.0) + ).toDF("user", "user_big", "user_small", "item", "item_big", "item_small", "rating") + val msg = "either out of Integer range or contained a fractional part" + withClue("fit should fail when ids exceed integer range. ") { + assert(intercept[SparkException] { + als.fit(df.select(df("user_big").as("user"), df("item"), df("rating"))) + }.getCause.getMessage.contains(msg)) + assert(intercept[SparkException] { + als.fit(df.select(df("user_small").as("user"), df("item"), df("rating"))) + }.getCause.getMessage.contains(msg)) + assert(intercept[SparkException] { + als.fit(df.select(df("item_big").as("item"), df("user"), df("rating"))) + }.getCause.getMessage.contains(msg)) + assert(intercept[SparkException] { + als.fit(df.select(df("item_small").as("item"), df("user"), df("rating"))) + }.getCause.getMessage.contains(msg)) + } + withClue("transform should fail when ids exceed integer range. ") { + val model = als.fit(df) + assert(intercept[SparkException] { + model.transform(df.select(df("user_big").as("user"), df("item"))).first + }.getMessage.contains(msg)) + assert(intercept[SparkException] { + model.transform(df.select(df("user_small").as("user"), df("item"))).first + }.getMessage.contains(msg)) + assert(intercept[SparkException] { + model.transform(df.select(df("item_big").as("item"), df("user"))).first + }.getMessage.contains(msg)) + assert(intercept[SparkException] { + model.transform(df.select(df("item_small").as("item"), df("user"))).first + }.getMessage.contains(msg)) + } + } + + test("SPARK-18268: ALS with empty RDD should fail with better message") { + val ratings = sc.parallelize(Array.empty[Rating[Int]]) + intercept[IllegalArgumentException] { + ALS.train(ratings) + } + } + + test("ALS cold start user/item prediction strategy") { + val spark = this.spark + import spark.implicits._ + import org.apache.spark.sql.functions._ + + val (ratings, _) = genExplicitTestData(numUsers = 4, numItems = 4, rank = 1) + val data = ratings.toDF + val knownUser = data.select(max("user")).as[Int].first() + val unknownUser = knownUser + 10 + val knownItem = data.select(max("item")).as[Int].first() + val unknownItem = knownItem + 20 + val test = Seq( + (unknownUser, unknownItem), + (knownUser, unknownItem), + (unknownUser, knownItem), + (knownUser, knownItem) + ).toDF("user", "item") + + val als = new ALS().setMaxIter(1).setRank(1) + // default is 'nan' + val defaultModel = als.fit(data) + val defaultPredictions = defaultModel.transform(test).select("prediction").as[Float].collect() + assert(defaultPredictions.length == 4) + assert(defaultPredictions.slice(0, 3).forall(_.isNaN)) + assert(!defaultPredictions.last.isNaN) + + // check 'drop' strategy should filter out rows with unknown users/items + val dropPredictions = defaultModel + .setColdStartStrategy("drop") + .transform(test) + .select("prediction").as[Float].collect() + assert(dropPredictions.length == 1) + assert(!dropPredictions.head.isNaN) + assert(dropPredictions.head ~== defaultPredictions.last relTol 1e-14) + } + + test("case insensitive cold start param value") { + val spark = this.spark + import spark.implicits._ + val (ratings, _) = genExplicitTestData(numUsers = 2, numItems = 2, rank = 1) + val data = ratings.toDF + val model = new ALS().fit(data) + Seq("nan", "NaN", "Nan", "drop", "DROP", "Drop").foreach { s => + model.setColdStartStrategy(s).transform(data) + } + } + + private def getALSModel = { + val spark = this.spark + import spark.implicits._ + + val userFactors = Seq( + (0, Array(6.0f, 4.0f)), + (1, Array(3.0f, 4.0f)), + (2, Array(3.0f, 6.0f)) + ).toDF("id", "features") + val itemFactors = Seq( + (3, Array(5.0f, 6.0f)), + (4, Array(6.0f, 2.0f)), + (5, Array(3.0f, 6.0f)), + (6, Array(4.0f, 1.0f)) + ).toDF("id", "features") + val als = new ALS().setRank(2) + new ALSModel(als.uid, als.getRank, userFactors, itemFactors) + .setUserCol("user") + .setItemCol("item") + } + + test("recommendForAllUsers with k <, = and > num_items") { + val model = getALSModel + val numUsers = model.userFactors.count + val numItems = model.itemFactors.count + val expected = Map( + 0 -> Array((3, 54f), (4, 44f), (5, 42f), (6, 28f)), + 1 -> Array((3, 39f), (5, 33f), (4, 26f), (6, 16f)), + 2 -> Array((3, 51f), (5, 45f), (4, 30f), (6, 18f)) + ) + + Seq(2, 4, 6).foreach { k => + val n = math.min(k, numItems).toInt + val expectedUpToN = expected.mapValues(_.slice(0, n)) + val topItems = model.recommendForAllUsers(k) + assert(topItems.count() == numUsers) + assert(topItems.columns.contains("user")) + checkRecommendations(topItems, expectedUpToN, "item") + } + } + + test("recommendForAllItems with k <, = and > num_users") { + val model = getALSModel + val numUsers = model.userFactors.count + val numItems = model.itemFactors.count + val expected = Map( + 3 -> Array((0, 54f), (2, 51f), (1, 39f)), + 4 -> Array((0, 44f), (2, 30f), (1, 26f)), + 5 -> Array((2, 45f), (0, 42f), (1, 33f)), + 6 -> Array((0, 28f), (2, 18f), (1, 16f)) + ) + + Seq(2, 3, 4).foreach { k => + val n = math.min(k, numUsers).toInt + val expectedUpToN = expected.mapValues(_.slice(0, n)) + val topUsers = getALSModel.recommendForAllItems(k) + assert(topUsers.count() == numItems) + assert(topUsers.columns.contains("item")) + checkRecommendations(topUsers, expectedUpToN, "user") + } + } + + private def checkRecommendations( + topK: DataFrame, + expected: Map[Int, Array[(Int, Float)]], + dstColName: String): Unit = { + val spark = this.spark + import spark.implicits._ + + assert(topK.columns.contains("recommendations")) + topK.as[(Int, Seq[(Int, Float)])].collect().foreach { case (id: Int, recs: Seq[(Int, Float)]) => + assert(recs === expected(id)) + } + topK.collect().foreach { row => + val recs = row.getAs[WrappedArray[Row]]("recommendations") + assert(recs(0).fieldIndex(dstColName) == 0) + assert(recs(0).fieldIndex("rating") == 1) + } + } +} + +class ALSCleanerSuite extends SparkFunSuite with BeforeAndAfterEach { + override def beforeEach(): Unit = { + super.beforeEach() + // Once `Utils.getOrCreateLocalRootDirs` is called, it is cached in `Utils.localRootDirs`. + // Unless this is manually cleared before and after a test, it returns the same directory + // set before even if 'spark.local.dir' is configured afterwards. + Utils.clearLocalRootDirs() + } + + override def afterEach(): Unit = { + Utils.clearLocalRootDirs() + super.afterEach() + } + + test("ALS shuffle cleanup standalone") { + val conf = new SparkConf() + val localDir = Utils.createTempDir() + val checkpointDir = Utils.createTempDir() + def getAllFiles: Set[File] = + FileUtils.listFiles(localDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet + try { + conf.set("spark.local.dir", localDir.getAbsolutePath) + val sc = new SparkContext("local[2]", "test", conf) + try { + sc.setCheckpointDir(checkpointDir.getAbsolutePath) + // Test checkpoint and clean parents + val input = sc.parallelize(1 to 1000) + val keyed = input.map(x => (x % 20, 1)) + val shuffled = keyed.reduceByKey(_ + _) + val keysOnly = shuffled.keys + val deps = keysOnly.dependencies + keysOnly.count() + ALS.cleanShuffleDependencies(sc, deps, true) + val resultingFiles = getAllFiles + assert(resultingFiles === Set()) + // Ensure running count again works fine even if we kill the shuffle files. + keysOnly.count() + } finally { + sc.stop() + } + } finally { + Utils.deleteRecursively(localDir) + Utils.deleteRecursively(checkpointDir) + } + } + + test("ALS shuffle cleanup in algorithm") { + val conf = new SparkConf() + val localDir = Utils.createTempDir() + val checkpointDir = Utils.createTempDir() + def getAllFiles: Set[File] = + FileUtils.listFiles(localDir, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE).asScala.toSet + try { + conf.set("spark.local.dir", localDir.getAbsolutePath) + val sc = new SparkContext("local[2]", "ALSCleanerSuite", conf) + try { + sc.setCheckpointDir(checkpointDir.getAbsolutePath) + // Generate test data + val (training, _) = ALSSuite.genImplicitTestData(sc, 20, 5, 1, 0.2, 0) + // Implicitly test the cleaning of parents during ALS training + val spark = SparkSession.builder + .sparkContext(sc) + .getOrCreate() + import spark.implicits._ + val als = new ALS() + .setRank(1) + .setRegParam(1e-5) + .setSeed(0) + .setCheckpointInterval(1) + .setMaxIter(7) + val model = als.fit(training.toDF()) + val resultingFiles = getAllFiles + // We expect the last shuffles files, block ratings, user factors, and item factors to be + // around but no more. + val pattern = "shuffle_(\\d+)_.+\\.data".r + val rddIds = resultingFiles.flatMap { f => + pattern.findAllIn(f.getName()).matchData.map { _.group(1) } } + assert(rddIds.size === 4) + } finally { + sc.stop() + } + } finally { + Utils.deleteRecursively(localDir) + Utils.deleteRecursively(checkpointDir) + } + } +} + +class ALSStorageSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest with Logging { + + test("invalid storage params") { + intercept[IllegalArgumentException] { + new ALS().setIntermediateStorageLevel("foo") + } + intercept[IllegalArgumentException] { + new ALS().setIntermediateStorageLevel("NONE") + } + intercept[IllegalArgumentException] { + new ALS().setFinalStorageLevel("foo") + } + } + + test("default and non-default storage params set correct RDD StorageLevels") { + val spark = this.spark + import spark.implicits._ + val data = Seq( + (0, 0, 1.0), + (0, 1, 2.0), + (1, 2, 3.0), + (1, 0, 2.0) + ).toDF("user", "item", "rating") + val als = new ALS().setMaxIter(1).setRank(1) + // add listener to check intermediate RDD default storage levels + val defaultListener = new IntermediateRDDStorageListener + sc.addSparkListener(defaultListener) + val model = als.fit(data) + // check final factor RDD default storage levels + val defaultFactorRDDs = sc.getPersistentRDDs.collect { + case (id, rdd) if rdd.name == "userFactors" || rdd.name == "itemFactors" => + rdd.name -> ((id, rdd.getStorageLevel)) + }.toMap + defaultFactorRDDs.foreach { case (_, (id, level)) => + assert(level == StorageLevel.MEMORY_AND_DISK) + } + defaultListener.storageLevels.foreach(level => assert(level == StorageLevel.MEMORY_AND_DISK)) + + // add listener to check intermediate RDD non-default storage levels + val nonDefaultListener = new IntermediateRDDStorageListener + sc.addSparkListener(nonDefaultListener) + val nonDefaultModel = als + .setFinalStorageLevel("MEMORY_ONLY") + .setIntermediateStorageLevel("DISK_ONLY") + .fit(data) + // check final factor RDD non-default storage levels + val levels = sc.getPersistentRDDs.collect { + case (id, rdd) if rdd.name == "userFactors" && rdd.id != defaultFactorRDDs("userFactors")._1 + || rdd.name == "itemFactors" && rdd.id != defaultFactorRDDs("itemFactors")._1 => + rdd.getStorageLevel + } + levels.foreach(level => assert(level == StorageLevel.MEMORY_ONLY)) + nonDefaultListener.storageLevels.foreach(level => assert(level == StorageLevel.DISK_ONLY)) + } +} + +private class IntermediateRDDStorageListener extends SparkListener { + + val storageLevels: mutable.ArrayBuffer[StorageLevel] = mutable.ArrayBuffer() + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { + val stageLevels = stageCompleted.stageInfo.rddInfos.collect { + case info if info.name.contains("Blocks") || info.name.contains("Factors-") => + info.storageLevel + } + storageLevels ++= stageLevels + } + +} + +object ALSSuite extends Logging { + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allModelParamSettings: Map[String, Any] = Map( + "predictionCol" -> "myPredictionCol" + ) + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allEstimatorParamSettings: Map[String, Any] = allModelParamSettings ++ Map( + "maxIter" -> 1, + "rank" -> 1, + "regParam" -> 0.01, + "numUserBlocks" -> 2, + "numItemBlocks" -> 2, + "implicitPrefs" -> true, + "alpha" -> 0.9, + "nonnegative" -> true, + "checkpointInterval" -> 20, + "intermediateStorageLevel" -> "MEMORY_ONLY", + "finalStorageLevel" -> "MEMORY_AND_DISK_SER" + ) + + // Helper functions to generate test data we share between ALS test suites + + /** + * Generates random user/item factors, with i.i.d. values drawn from U(a, b). + * @param size number of users/items + * @param rank number of features + * @param random random number generator + * @param a min value of the support (default: -1) + * @param b max value of the support (default: 1) + * @return a sequence of (ID, factors) pairs + */ + private def genFactors( + size: Int, + rank: Int, + random: Random, + a: Float = -1.0f, + b: Float = 1.0f): Seq[(Int, Array[Float])] = { + require(size > 0 && size < Int.MaxValue / 3) + require(b > a) + val ids = mutable.Set.empty[Int] + while (ids.size < size) { + ids += random.nextInt() + } + val width = b - a + ids.toSeq.sorted.map(id => (id, Array.fill(rank)(a + random.nextFloat() * width))) + } + + /** + * Generates an implicit feedback dataset for testing ALS. + * + * @param sc SparkContext + * @param numUsers number of users + * @param numItems number of items + * @param rank rank + * @param noiseStd the standard deviation of additive Gaussian noise on training data + * @param seed random seed + * @return (training, test) + */ + def genImplicitTestData( + sc: SparkContext, + numUsers: Int, + numItems: Int, + rank: Int, + noiseStd: Double = 0.0, + seed: Long = 11L): (RDD[Rating[Int]], RDD[Rating[Int]]) = { + // The assumption of the implicit feedback model is that unobserved ratings are more likely to + // be negatives. + val positiveFraction = 0.8 + val negativeFraction = 1.0 - positiveFraction + val trainingFraction = 0.6 + val testFraction = 0.3 + val totalFraction = trainingFraction + testFraction + val random = new Random(seed) + val userFactors = genFactors(numUsers, rank, random) + val itemFactors = genFactors(numItems, rank, random) + val training = ArrayBuffer.empty[Rating[Int]] + val test = ArrayBuffer.empty[Rating[Int]] + for ((userId, userFactor) <- userFactors; (itemId, itemFactor) <- itemFactors) { + val rating = blas.sdot(rank, userFactor, 1, itemFactor, 1) + val threshold = if (rating > 0) positiveFraction else negativeFraction + val observed = random.nextDouble() < threshold + if (observed) { + val x = random.nextDouble() + if (x < totalFraction) { + if (x < trainingFraction) { + val noise = noiseStd * random.nextGaussian() + training += Rating(userId, itemId, rating + noise.toFloat) + } else { + test += Rating(userId, itemId, rating) + } + } + } + } + logInfo(s"Generated an implicit feedback dataset with ${training.size} ratings for training " + + s"and ${test.size} for test.") + (sc.parallelize(training, 2), sc.parallelize(test, 2)) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/TopByKeyAggregatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/TopByKeyAggregatorSuite.scala new file mode 100644 index 000000000000..5e763a8e908b --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/TopByKeyAggregatorSuite.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.recommendation + +import org.apache.spark.SparkFunSuite +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.Dataset + + +class TopByKeyAggregatorSuite extends SparkFunSuite with MLlibTestSparkContext { + + private def getTopK(k: Int): Dataset[(Int, Array[(Int, Float)])] = { + val sqlContext = spark.sqlContext + import sqlContext.implicits._ + + val topKAggregator = new TopByKeyAggregator[Int, Int, Float](k, Ordering.by(_._2)) + Seq( + (0, 3, 54f), + (0, 4, 44f), + (0, 5, 42f), + (0, 6, 28f), + (1, 3, 39f), + (2, 3, 51f), + (2, 5, 45f), + (2, 6, 18f) + ).toDS().groupByKey(_._1).agg(topKAggregator.toColumn) + } + + test("topByKey with k < #items") { + val topK = getTopK(2) + assert(topK.count() === 3) + + val expected = Map( + 0 -> Array((3, 54f), (4, 44f)), + 1 -> Array((3, 39f)), + 2 -> Array((3, 51f), (5, 45f)) + ) + checkTopK(topK, expected) + } + + test("topByKey with k > #items") { + val topK = getTopK(5) + assert(topK.count() === 3) + + val expected = Map( + 0 -> Array((3, 54f), (4, 44f), (5, 42f), (6, 28f)), + 1 -> Array((3, 39f)), + 2 -> Array((3, 51f), (5, 45f), (6, 18f)) + ) + checkTopK(topK, expected) + } + + private def checkTopK( + topK: Dataset[(Int, Array[(Int, Float)])], + expected: Map[Int, Array[(Int, Float)]]): Unit = { + topK.collect().foreach { case (id, recs) => assert(recs === expected(id)) } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala index 359f31027172..02e5c6d294f4 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala @@ -20,27 +20,48 @@ package org.apache.spark.ml.regression import scala.util.Random import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.random.{ExponentialGenerator, WeibullGenerator} -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{Row, DataFrame} +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.functions.{col, lit} +import org.apache.spark.sql.types._ -class AFTSurvivalRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { +class AFTSurvivalRegressionSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ @transient var datasetUnivariate: DataFrame = _ @transient var datasetMultivariate: DataFrame = _ + @transient var datasetUnivariateScaled: DataFrame = _ override def beforeAll(): Unit = { super.beforeAll() - datasetUnivariate = sqlContext.createDataFrame( - sc.parallelize(generateAFTInput( - 1, Array(5.5), Array(0.8), 1000, 42, 1.0, 2.0, 2.0))) - datasetMultivariate = sqlContext.createDataFrame( - sc.parallelize(generateAFTInput( - 2, Array(0.9, -1.3), Array(0.7, 1.2), 1000, 42, 1.5, 2.5, 2.0))) + datasetUnivariate = generateAFTInput( + 1, Array(5.5), Array(0.8), 1000, 42, 1.0, 2.0, 2.0).toDF() + datasetMultivariate = generateAFTInput( + 2, Array(0.9, -1.3), Array(0.7, 1.2), 1000, 42, 1.5, 2.5, 2.0).toDF() + datasetUnivariateScaled = sc.parallelize( + generateAFTInput(1, Array(5.5), Array(0.8), 1000, 42, 1.0, 2.0, 2.0)).map { x => + AFTPoint(Vectors.dense(x.features(0) * 1.0E3), x.label, x.censor) + }.toDF() + } + + /** + * Enable the ignored test to export the dataset into CSV format, + * so we can validate the training accuracy compared with R's survival package. + */ + ignore("export test data into CSV format") { + datasetUnivariate.rdd.map { case Row(features: Vector, label: Double, censor: Double) => + features.toArray.mkString(",") + "," + censor + "," + label + }.repartition(1).saveAsTextFile("target/tmp/AFTSurvivalRegressionSuite/datasetUnivariate") + datasetMultivariate.rdd.map { case Row(features: Vector, label: Double, censor: Double) => + features.toArray.mkString(",") + "," + censor + "," + label + }.repartition(1).saveAsTextFile("target/tmp/AFTSurvivalRegressionSuite/datasetMultivariate") } test("params") { @@ -62,8 +83,7 @@ class AFTSurvivalRegressionSuite extends SparkFunSuite with MLlibTestSparkContex .setQuantilesCol("quantiles") .fit(datasetUnivariate) - // copied model must have the same parent. - MLTestingUtils.checkCopy(model) + MLTestingUtils.checkCopyAndUids(aftr, model) model.transform(datasetUnivariate) .select("label", "prediction", "quantiles") @@ -332,4 +352,98 @@ class AFTSurvivalRegressionSuite extends SparkFunSuite with MLlibTestSparkContex assert(prediction ~== model.predict(features) relTol 1E-5) } } + + test("should support all NumericType labels, and not support other types") { + val aft = new AFTSurvivalRegression().setMaxIter(1) + MLTestingUtils.checkNumericTypes[AFTSurvivalRegressionModel, AFTSurvivalRegression]( + aft, spark, isClassification = false) { (expected, actual) => + assert(expected.intercept === actual.intercept) + assert(expected.coefficients === actual.coefficients) + } + } + + test("should support all NumericType censors, and not support other types") { + val df = spark.createDataFrame(Seq( + (1, Vectors.dense(1)), + (2, Vectors.dense(2)), + (3, Vectors.dense(3)), + (4, Vectors.dense(4)) + )).toDF("label", "features") + .withColumn("censor", lit(0.0)) + val aft = new AFTSurvivalRegression().setMaxIter(1) + val expected = aft.fit(df) + + val types = Seq(ShortType, LongType, IntegerType, FloatType, ByteType, DecimalType(10, 0)) + types.foreach { t => + val actual = aft.fit(df.select(col("label"), col("features"), + col("censor").cast(t))) + assert(expected.intercept === actual.intercept) + assert(expected.coefficients === actual.coefficients) + } + + val dfWithStringCensors = spark.createDataFrame(Seq( + (0, Vectors.dense(0, 2, 3), "0") + )).toDF("label", "features", "censor") + val thrown = intercept[IllegalArgumentException] { + aft.fit(dfWithStringCensors) + } + assert(thrown.getMessage.contains( + "Column censor must be of type NumericType but was actually of type StringType")) + } + + test("numerical stability of standardization") { + val trainer = new AFTSurvivalRegression() + val model1 = trainer.fit(datasetUnivariate) + val model2 = trainer.fit(datasetUnivariateScaled) + + /** + * During training we standardize the dataset first, so no matter how we multiple + * a scaling factor into the dataset, the convergence rate should be the same, + * and the coefficients should equal to the original coefficients multiple by + * the scaling factor. It will have no effect on the intercept and scale. + */ + assert(model1.coefficients(0) ~== model2.coefficients(0) * 1.0E3 absTol 0.01) + assert(model1.intercept ~== model2.intercept absTol 0.01) + assert(model1.scale ~== model2.scale absTol 0.01) + } + + test("read/write") { + def checkModelData( + model: AFTSurvivalRegressionModel, + model2: AFTSurvivalRegressionModel): Unit = { + assert(model.intercept === model2.intercept) + assert(model.coefficients === model2.coefficients) + assert(model.scale === model2.scale) + } + val aft = new AFTSurvivalRegression() + testEstimatorAndModelReadWrite(aft, datasetMultivariate, + AFTSurvivalRegressionSuite.allParamSettings, AFTSurvivalRegressionSuite.allParamSettings, + checkModelData) + } + + test("SPARK-15892: Incorrectly merged AFTAggregator with zero total count") { + // This `dataset` will contain an empty partition because it has two rows but + // the parallelism is bigger than that. Because the issue was about `AFTAggregator`s + // being merged incorrectly when it has an empty partition, running the codes below + // should not throw an exception. + val dataset = sc.parallelize(generateAFTInput( + 1, Array(5.5), Array(0.8), 2, 42, 1.0, 2.0, 2.0), numSlices = 3).toDF() + val trainer = new AFTSurvivalRegression() + trainer.fit(dataset) + } +} + +object AFTSurvivalRegressionSuite { + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "predictionCol" -> "myPrediction", + "fitIntercept" -> true, + "maxIter" -> 2, + "tol" -> 0.01 + ) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala index 868fb8eecb8b..642f266891b5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala @@ -18,17 +18,20 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.impl.TreeTests -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.tree.impl.TreeTests +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, DecisionTreeSuite => OldDecisionTreeSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Row} - -class DecisionTreeRegressorSuite extends SparkFunSuite with MLlibTestSparkContext { +class DecisionTreeRegressorSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { import DecisionTreeRegressorSuite.compareAPIs @@ -37,7 +40,7 @@ class DecisionTreeRegressorSuite extends SparkFunSuite with MLlibTestSparkContex override def beforeAll() { super.beforeAll() categoricalDataPointsRDD = - sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints()) + sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints().map(_.asML)) } ///////////////////////////////////////////////////////////////////////////// @@ -49,7 +52,8 @@ class DecisionTreeRegressorSuite extends SparkFunSuite with MLlibTestSparkContex .setImpurity("variance") .setMaxDepth(2) .setMaxBins(100) - val categoricalFeatures = Map(0 -> 3, 1-> 3) + .setSeed(1) + val categoricalFeatures = Map(0 -> 3, 1 -> 3) compareAPIs(categoricalDataPointsRDD, dt, categoricalFeatures) } @@ -58,25 +62,123 @@ class DecisionTreeRegressorSuite extends SparkFunSuite with MLlibTestSparkContex .setImpurity("variance") .setMaxDepth(2) .setMaxBins(100) - val categoricalFeatures = Map(0 -> 2, 1-> 2) + val categoricalFeatures = Map(0 -> 2, 1 -> 2) compareAPIs(categoricalDataPointsRDD, dt, categoricalFeatures) } test("copied model must have the same parent") { - val categoricalFeatures = Map(0 -> 2, 1-> 2) + val categoricalFeatures = Map(0 -> 2, 1 -> 2) val df = TreeTests.setMetadata(categoricalDataPointsRDD, categoricalFeatures, numClasses = 0) - val model = new DecisionTreeRegressor() + val dtr = new DecisionTreeRegressor() .setImpurity("variance") .setMaxDepth(2) - .setMaxBins(8).fit(df) - MLTestingUtils.checkCopy(model) + .setMaxBins(8) + val model = dtr.fit(df) + MLTestingUtils.checkCopyAndUids(dtr, model) + } + + test("predictVariance") { + val dt = new DecisionTreeRegressor() + .setImpurity("variance") + .setMaxDepth(2) + .setMaxBins(100) + .setPredictionCol("") + .setVarianceCol("variance") + val categoricalFeatures = Map(0 -> 2, 1 -> 2) + + val df = TreeTests.setMetadata(categoricalDataPointsRDD, categoricalFeatures, numClasses = 0) + val model = dt.fit(df) + + val predictions = model.transform(df) + .select(model.getFeaturesCol, model.getVarianceCol) + .collect() + + predictions.foreach { case Row(features: Vector, variance: Double) => + val expectedVariance = model.rootNode.predictImpl(features).impurityStats.calculate() + assert(variance === expectedVariance, + s"Expected variance $expectedVariance but got $variance.") + } + + val varianceData: RDD[LabeledPoint] = TreeTests.varianceData(sc) + val varianceDF = TreeTests.setMetadata(varianceData, Map.empty[Int, Int], 0) + dt.setMaxDepth(1) + .setMaxBins(6) + .setSeed(0) + val transformVarDF = dt.fit(varianceDF).transform(varianceDF) + val calculatedVariances = transformVarDF.select(dt.getVarianceCol).collect().map { + case Row(variance: Double) => variance + } + + // Since max depth is set to 1, the best split point is that which splits the data + // into (0.0, 1.0, 2.0) and (10.0, 12.0, 14.0). The predicted variance for each + // data point in the left node is 0.667 and for each data point in the right node + // is 2.667 + val expectedVariances = Array(0.667, 0.667, 0.667, 2.667, 2.667, 2.667) + calculatedVariances.zip(expectedVariances).foreach { case (actual, expected) => + assert(actual ~== expected absTol 1e-3) + } + } + + test("Feature importance with toy data") { + val dt = new DecisionTreeRegressor() + .setImpurity("variance") + .setMaxDepth(3) + .setSeed(123) + + // In this data, feature 1 is very important. + val data: RDD[LabeledPoint] = TreeTests.featureImportanceData(sc) + val categoricalFeatures = Map.empty[Int, Int] + val df: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, 0) + + val model = dt.fit(df) + + val importances = model.featureImportances + val mostImportantFeature = importances.argmax + assert(mostImportantFeature === 1) + assert(importances.toArray.sum === 1.0) + assert(importances.toArray.forall(_ >= 0.0)) + } + + test("should support all NumericType labels and not support other types") { + val dt = new DecisionTreeRegressor().setMaxDepth(1) + MLTestingUtils.checkNumericTypes[DecisionTreeRegressionModel, DecisionTreeRegressor]( + dt, spark, isClassification = false) { (expected, actual) => + TreeTests.checkEqual(expected, actual) + } } ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// - // TODO: test("model save/load") SPARK-6725 + test("read/write") { + def checkModelData( + model: DecisionTreeRegressionModel, + model2: DecisionTreeRegressionModel): Unit = { + TreeTests.checkEqual(model, model2) + assert(model.numFeatures === model2.numFeatures) + } + + val dt = new DecisionTreeRegressor() + val rdd = TreeTests.getTreeReadWriteData(sc) + + // Categorical splits with tree depth 2 + val categoricalData: DataFrame = + TreeTests.setMetadata(rdd, Map(0 -> 2, 1 -> 3), numClasses = 0) + testEstimatorAndModelReadWrite(dt, categoricalData, + TreeTests.allParamSettings, TreeTests.allParamSettings, checkModelData) + + // Continuous splits with tree depth 2 + val continuousData: DataFrame = + TreeTests.setMetadata(rdd, Map.empty[Int, Int], numClasses = 0) + testEstimatorAndModelReadWrite(dt, continuousData, + TreeTests.allParamSettings, TreeTests.allParamSettings, checkModelData) + + // Continuous splits with tree depth 0 + testEstimatorAndModelReadWrite(dt, continuousData, + TreeTests.allParamSettings ++ Map("maxDepth" -> 0), + TreeTests.allParamSettings ++ Map("maxDepth" -> 0), checkModelData) + } } private[ml] object DecisionTreeRegressorSuite extends SparkFunSuite { @@ -91,7 +193,7 @@ private[ml] object DecisionTreeRegressorSuite extends SparkFunSuite { categoricalFeatures: Map[Int, Int]): Unit = { val numFeatures = data.first().features.size val oldStrategy = dt.getOldStrategy(categoricalFeatures) - val oldTree = OldDecisionTree.train(data, oldStrategy) + val oldTree = OldDecisionTree.train(data.map(OldLabeledPoint.fromML), oldStrategy) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) val newTree = dt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala index 09326600e620..2da25f7e0100 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala @@ -18,10 +18,11 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.impl.TreeTests -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.tree.impl.TreeTests +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, GradientBoostedTrees => OldGBT} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -29,13 +30,14 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame import org.apache.spark.util.Utils - /** * Test suite for [[GBTRegressor]]. */ -class GBTRegressorSuite extends SparkFunSuite with MLlibTestSparkContext { +class GBTRegressorSuite extends SparkFunSuite with MLlibTestSparkContext + with DefaultReadWriteTest { import GBTRegressorSuite.compareAPIs + import testImplicits._ // Combinations for estimators, learning rates and subsamplingRate private val testCombinations = @@ -48,13 +50,16 @@ class GBTRegressorSuite extends SparkFunSuite with MLlibTestSparkContext { override def beforeAll() { super.beforeAll() data = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100), 2) + .map(_.asML) trainData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 120), 2) + .map(_.asML) validationData = sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 20, 80), 2) + .map(_.asML) } - test("Regression with continuous features: SquaredError") { + test("Regression with continuous features") { val categoricalFeatures = Map.empty[Int, Int] GBTRegressor.supportedLossTypes.foreach { loss => testCombinations.foreach { @@ -65,29 +70,29 @@ class GBTRegressorSuite extends SparkFunSuite with MLlibTestSparkContext { .setLossType(loss) .setMaxIter(maxIter) .setStepSize(learningRate) + .setSeed(123) compareAPIs(data, None, gbt, categoricalFeatures) } } } test("GBTRegressor behaves reasonably on toy data") { - val df = sqlContext.createDataFrame(Seq( + val df = Seq( LabeledPoint(10, Vectors.dense(1, 2, 3, 4)), LabeledPoint(-5, Vectors.dense(6, 3, 2, 1)), LabeledPoint(11, Vectors.dense(2, 2, 3, 4)), LabeledPoint(-6, Vectors.dense(6, 4, 2, 1)), LabeledPoint(9, Vectors.dense(1, 2, 6, 4)), LabeledPoint(-4, Vectors.dense(6, 3, 2, 2)) - )) + ).toDF() val gbt = new GBTRegressor() .setMaxDepth(2) .setMaxIter(2) val model = gbt.fit(df) - // copied model must have the same parent. - MLTestingUtils.checkCopy(model) + MLTestingUtils.checkCopyAndUids(gbt, model) val preds = model.transform(df) - val predictions = preds.select("prediction").map(_.getDouble(0)) + val predictions = preds.select("prediction").rdd.map(_.getDouble(0)) // Checks based on SPARK-8736 (to ensure it is not doing classification) assert(predictions.max() > 2) assert(predictions.min() < -1) @@ -98,17 +103,25 @@ class GBTRegressorSuite extends SparkFunSuite with MLlibTestSparkContext { val path = tempDir.toURI.toString sc.setCheckpointDir(path) - val df = sqlContext.createDataFrame(data) + val df = data.toDF() val gbt = new GBTRegressor() .setMaxDepth(2) .setMaxIter(5) .setStepSize(0.1) .setCheckpointInterval(2) + .setSeed(123) val model = gbt.fit(df) sc.checkpointDir = None Utils.deleteRecursively(tempDir) + } + test("should support all NumericType labels and not support other types") { + val gbt = new GBTRegressor().setMaxDepth(1) + MLTestingUtils.checkNumericTypes[GBTRegressionModel, GBTRegressor]( + gbt, spark, isClassification = false) { (expected, actual) => + TreeTests.checkEqual(expected, actual) + } } // TODO: Reinstate test once runWithValidation is implemented SPARK-7132 @@ -129,31 +142,50 @@ class GBTRegressorSuite extends SparkFunSuite with MLlibTestSparkContext { } */ + ///////////////////////////////////////////////////////////////////////////// + // Tests of feature importance + ///////////////////////////////////////////////////////////////////////////// + test("Feature importance with toy data") { + val gbt = new GBTRegressor() + .setMaxDepth(3) + .setMaxIter(5) + .setSubsamplingRate(1.0) + .setStepSize(0.5) + .setSeed(123) + + // In this data, feature 1 is very important. + val data: RDD[LabeledPoint] = TreeTests.featureImportanceData(sc) + val categoricalFeatures = Map.empty[Int, Int] + val df: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, 0) + + val importances = gbt.fit(df).featureImportances + val mostImportantFeature = importances.argmax + assert(mostImportantFeature === 1) + assert(importances.toArray.sum === 1.0) + assert(importances.toArray.forall(_ >= 0.0)) + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// - // TODO: Reinstate test once save/load are implemented SPARK-6725 - /* test("model save/load") { - val tempDir = Utils.createTempDir() - val path = tempDir.toURI.toString - - val trees = Range(0, 3).map(_ => OldDecisionTreeSuite.createModel(OldAlgo.Regression)).toArray - val treeWeights = Array(0.1, 0.3, 1.1) - val oldModel = new OldGBTModel(OldAlgo.Regression, trees, treeWeights) - val newModel = GBTRegressionModel.fromOld(oldModel) - - // Save model, load it back, and compare. - try { - newModel.save(sc, path) - val sameNewModel = GBTRegressionModel.load(sc, path) - TreeTests.checkEqual(newModel, sameNewModel) - } finally { - Utils.deleteRecursively(tempDir) + def checkModelData( + model: GBTRegressionModel, + model2: GBTRegressionModel): Unit = { + TreeTests.checkEqual(model, model2) + assert(model.numFeatures === model2.numFeatures) } + + val gbt = new GBTRegressor() + val rdd = TreeTests.getTreeReadWriteData(sc) + + val allParamSettings = TreeTests.allParamSettings ++ Map("lossType" -> "squared") + val continuousData: DataFrame = + TreeTests.setMetadata(rdd, Map.empty[Int, Int], numClasses = 0) + testEstimatorAndModelReadWrite(gbt, continuousData, allParamSettings, + allParamSettings, checkModelData) } - */ } private object GBTRegressorSuite extends SparkFunSuite { @@ -169,8 +201,8 @@ private object GBTRegressorSuite extends SparkFunSuite { categoricalFeatures: Map[Int, Int]): Unit = { val numFeatures = data.first().features.size val oldBoostingStrategy = gbt.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Regression) - val oldGBT = new OldGBT(oldBoostingStrategy) - val oldModel = oldGBT.run(data) + val oldGBT = new OldGBT(oldBoostingStrategy, gbt.getSeed.toInt) + val oldModel = oldGBT.run(data.map(OldLabeledPoint.fromML)) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) val newModel = gbt.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala new file mode 100644 index 000000000000..df7dee869d05 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala @@ -0,0 +1,1726 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.regression + +import scala.util.Random + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.classification.LogisticRegressionSuite._ +import org.apache.spark.ml.feature.{Instance, OffsetInstance} +import org.apache.spark.ml.feature.{LabeledPoint, RFormula} +import org.apache.spark.ml.linalg.{BLAS, DenseVector, Vector, Vectors} +import org.apache.spark.ml.param.{ParamMap, ParamsSuite} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.random._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.FloatType + +class GeneralizedLinearRegressionSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + + private val seed: Int = 42 + @transient var datasetGaussianIdentity: DataFrame = _ + @transient var datasetGaussianLog: DataFrame = _ + @transient var datasetGaussianInverse: DataFrame = _ + @transient var datasetBinomial: DataFrame = _ + @transient var datasetPoissonLog: DataFrame = _ + @transient var datasetPoissonLogWithZero: DataFrame = _ + @transient var datasetPoissonIdentity: DataFrame = _ + @transient var datasetPoissonSqrt: DataFrame = _ + @transient var datasetGammaInverse: DataFrame = _ + @transient var datasetGammaIdentity: DataFrame = _ + @transient var datasetGammaLog: DataFrame = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + import GeneralizedLinearRegressionSuite._ + + datasetGaussianIdentity = generateGeneralizedLinearRegressionInput( + intercept = 2.5, coefficients = Array(2.2, 0.6), xMean = Array(2.9, 10.5), + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, + family = "gaussian", link = "identity").toDF() + + datasetGaussianLog = generateGeneralizedLinearRegressionInput( + intercept = 0.25, coefficients = Array(0.22, 0.06), xMean = Array(2.9, 10.5), + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, + family = "gaussian", link = "log").toDF() + + datasetGaussianInverse = generateGeneralizedLinearRegressionInput( + intercept = 2.5, coefficients = Array(2.2, 0.6), xMean = Array(2.9, 10.5), + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, + family = "gaussian", link = "inverse").toDF() + + datasetBinomial = { + val nPoints = 10000 + val coefficients = Array(-0.57997, 0.912083, -0.371077, -0.819866, 2.688191) + val xMean = Array(5.843, 3.057, 3.758, 1.199) + val xVariance = Array(0.6856, 0.1899, 3.116, 0.581) + + val testData = + generateMultinomialLogisticInput(coefficients, xMean, xVariance, + addIntercept = true, nPoints, seed) + + testData.toDF() + } + + datasetPoissonLog = generateGeneralizedLinearRegressionInput( + intercept = 0.25, coefficients = Array(0.22, 0.06), xMean = Array(2.9, 10.5), + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, + family = "poisson", link = "log").toDF() + + datasetPoissonLogWithZero = Seq( + LabeledPoint(0.0, Vectors.dense(18, 1.0)), + LabeledPoint(1.0, Vectors.dense(12, 0.0)), + LabeledPoint(0.0, Vectors.dense(15, 0.0)), + LabeledPoint(0.0, Vectors.dense(13, 2.0)), + LabeledPoint(0.0, Vectors.dense(15, 1.0)), + LabeledPoint(1.0, Vectors.dense(16, 1.0)) + ).toDF() + + datasetPoissonIdentity = generateGeneralizedLinearRegressionInput( + intercept = 2.5, coefficients = Array(2.2, 0.6), xMean = Array(2.9, 10.5), + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, + family = "poisson", link = "identity").toDF() + + datasetPoissonSqrt = generateGeneralizedLinearRegressionInput( + intercept = 2.5, coefficients = Array(2.2, 0.6), xMean = Array(2.9, 10.5), + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, + family = "poisson", link = "sqrt").toDF() + + datasetGammaInverse = generateGeneralizedLinearRegressionInput( + intercept = 2.5, coefficients = Array(2.2, 0.6), xMean = Array(2.9, 10.5), + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, + family = "gamma", link = "inverse").toDF() + + datasetGammaIdentity = generateGeneralizedLinearRegressionInput( + intercept = 2.5, coefficients = Array(2.2, 0.6), xMean = Array(2.9, 10.5), + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, + family = "gamma", link = "identity").toDF() + + datasetGammaLog = generateGeneralizedLinearRegressionInput( + intercept = 0.25, coefficients = Array(0.22, 0.06), xMean = Array(2.9, 10.5), + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, noiseLevel = 0.01, + family = "gamma", link = "log").toDF() + } + + /** + * Enable the ignored test to export the dataset into CSV format, + * so we can validate the training accuracy compared with R's glm and glmnet package. + */ + ignore("export test data into CSV format") { + datasetGaussianIdentity.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile( + "target/tmp/GeneralizedLinearRegressionSuite/datasetGaussianIdentity") + datasetGaussianLog.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile( + "target/tmp/GeneralizedLinearRegressionSuite/datasetGaussianLog") + datasetGaussianInverse.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile( + "target/tmp/GeneralizedLinearRegressionSuite/datasetGaussianInverse") + datasetBinomial.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile( + "target/tmp/GeneralizedLinearRegressionSuite/datasetBinomial") + datasetPoissonLog.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile( + "target/tmp/GeneralizedLinearRegressionSuite/datasetPoissonLog") + datasetPoissonLogWithZero.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile( + "target/tmp/GeneralizedLinearRegressionSuite/datasetPoissonLogWithZero") + datasetPoissonIdentity.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile( + "target/tmp/GeneralizedLinearRegressionSuite/datasetPoissonIdentity") + datasetPoissonSqrt.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile( + "target/tmp/GeneralizedLinearRegressionSuite/datasetPoissonSqrt") + datasetGammaInverse.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile( + "target/tmp/GeneralizedLinearRegressionSuite/datasetGammaInverse") + datasetGammaIdentity.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile( + "target/tmp/GeneralizedLinearRegressionSuite/datasetGammaIdentity") + datasetGammaLog.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile( + "target/tmp/GeneralizedLinearRegressionSuite/datasetGammaLog") + } + + test("params") { + ParamsSuite.checkParams(new GeneralizedLinearRegression) + val model = new GeneralizedLinearRegressionModel("genLinReg", Vectors.dense(0.0), 0.0) + ParamsSuite.checkParams(model) + } + + test("generalized linear regression: default params") { + val glr = new GeneralizedLinearRegression + assert(glr.getLabelCol === "label") + assert(glr.getFeaturesCol === "features") + assert(glr.getPredictionCol === "prediction") + assert(glr.getFitIntercept) + assert(glr.getTol === 1E-6) + assert(!glr.isDefined(glr.weightCol)) + assert(glr.getRegParam === 0.0) + assert(glr.getSolver == "irls") + assert(glr.getVariancePower === 0.0) + + // TODO: Construct model directly instead of via fitting. + val model = glr.setFamily("gaussian").setLink("identity") + .fit(datasetGaussianIdentity) + + MLTestingUtils.checkCopyAndUids(glr, model) + assert(model.hasSummary) + val copiedModel = model.copy(ParamMap.empty) + assert(copiedModel.hasSummary) + model.setSummary(None) + assert(!model.hasSummary) + + assert(model.getFeaturesCol === "features") + assert(model.getPredictionCol === "prediction") + assert(model.intercept !== 0.0) + assert(model.hasParent) + assert(model.getFamily === "gaussian") + assert(model.getLink === "identity") + } + + test("generalized linear regression: gaussian family against glm") { + /* + R code: + f1 <- data$V1 ~ data$V2 + data$V3 - 1 + f2 <- data$V1 ~ data$V2 + data$V3 + + data <- read.csv("path", header=FALSE) + for (formula in c(f1, f2)) { + model <- glm(formula, family="gaussian", data=data) + print(as.vector(coef(model))) + } + + [1] 2.2960999 0.8087933 + [1] 2.5002642 2.2000403 0.5999485 + + data <- read.csv("path", header=FALSE) + model1 <- glm(f1, family=gaussian(link=log), data=data, start=c(0,0)) + model2 <- glm(f2, family=gaussian(link=log), data=data, start=c(0,0,0)) + print(as.vector(coef(model1))) + print(as.vector(coef(model2))) + + [1] 0.23069326 0.07993778 + [1] 0.25001858 0.22002452 0.05998789 + + data <- read.csv("path", header=FALSE) + for (formula in c(f1, f2)) { + model <- glm(formula, family=gaussian(link=inverse), data=data) + print(as.vector(coef(model))) + } + + [1] 2.3010179 0.8198976 + [1] 2.4108902 2.2130248 0.6086152 + */ + + val expected = Seq( + Vectors.dense(0.0, 2.2960999, 0.8087933), + Vectors.dense(2.5002642, 2.2000403, 0.5999485), + Vectors.dense(0.0, 0.23069326, 0.07993778), + Vectors.dense(0.25001858, 0.22002452, 0.05998789), + Vectors.dense(0.0, 2.3010179, 0.8198976), + Vectors.dense(2.4108902, 2.2130248, 0.6086152)) + + import GeneralizedLinearRegression._ + + var idx = 0 + for ((link, dataset) <- Seq(("identity", datasetGaussianIdentity), ("log", datasetGaussianLog), + ("inverse", datasetGaussianInverse))) { + for (fitIntercept <- Seq(false, true)) { + val trainer = new GeneralizedLinearRegression().setFamily("gaussian").setLink(link) + .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") + val model = trainer.fit(dataset) + val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) + assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with gaussian family, " + + s"$link link and fitIntercept = $fitIntercept.") + + val familyLink = FamilyAndLink(trainer) + model.transform(dataset).select("features", "prediction", "linkPrediction").collect() + .foreach { + case Row(features: DenseVector, prediction1: Double, linkPrediction1: Double) => + val eta = BLAS.dot(features, model.coefficients) + model.intercept + val prediction2 = familyLink.fitted(eta) + val linkPrediction2 = eta + assert(prediction1 ~= prediction2 relTol 1E-5, "Prediction mismatch: GLM with " + + s"gaussian family, $link link and fitIntercept = $fitIntercept.") + assert(linkPrediction1 ~= linkPrediction2 relTol 1E-5, "Link Prediction mismatch: " + + s"GLM with gaussian family, $link link and fitIntercept = $fitIntercept.") + } + + idx += 1 + } + } + } + + test("generalized linear regression: gaussian family against glmnet") { + /* + R code: + library(glmnet) + data <- read.csv("path", header=FALSE) + label = data$V1 + features = as.matrix(data.frame(data$V2, data$V3)) + for (intercept in c(FALSE, TRUE)) { + for (lambda in c(0.0, 0.1, 1.0)) { + model <- glmnet(features, label, family="gaussian", intercept=intercept, + lambda=lambda, alpha=0, thresh=1E-14) + print(as.vector(coef(model))) + } + } + + [1] 0.0000000 2.2961005 0.8087932 + [1] 0.0000000 2.2130368 0.8309556 + [1] 0.0000000 1.7176137 0.9610657 + [1] 2.5002642 2.2000403 0.5999485 + [1] 3.1106389 2.0935142 0.5712711 + [1] 6.7597127 1.4581054 0.3994266 + */ + + val expected = Seq( + Vectors.dense(0.0, 2.2961005, 0.8087932), + Vectors.dense(0.0, 2.2130368, 0.8309556), + Vectors.dense(0.0, 1.7176137, 0.9610657), + Vectors.dense(2.5002642, 2.2000403, 0.5999485), + Vectors.dense(3.1106389, 2.0935142, 0.5712711), + Vectors.dense(6.7597127, 1.4581054, 0.3994266)) + + var idx = 0 + for (fitIntercept <- Seq(false, true); + regParam <- Seq(0.0, 0.1, 1.0)) { + val trainer = new GeneralizedLinearRegression().setFamily("gaussian") + .setFitIntercept(fitIntercept).setRegParam(regParam) + val model = trainer.fit(datasetGaussianIdentity) + val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) + assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with gaussian family, " + + s"fitIntercept = $fitIntercept and regParam = $regParam.") + + idx += 1 + } + } + + test("generalized linear regression: binomial family against glm") { + /* + R code: + f1 <- data$V1 ~ data$V2 + data$V3 + data$V4 + data$V5 - 1 + f2 <- data$V1 ~ data$V2 + data$V3 + data$V4 + data$V5 + data <- read.csv("path", header=FALSE) + + for (formula in c(f1, f2)) { + model <- glm(formula, family="binomial", data=data) + print(as.vector(coef(model))) + } + + [1] -0.3560284 1.3010002 -0.3570805 -0.7406762 + [1] 2.8367406 -0.5896187 0.8931655 -0.3925169 -0.7996989 + + for (formula in c(f1, f2)) { + model <- glm(formula, family=binomial(link=probit), data=data) + print(as.vector(coef(model))) + } + + [1] -0.2134390 0.7800646 -0.2144267 -0.4438358 + [1] 1.6995366 -0.3524694 0.5332651 -0.2352985 -0.4780850 + + for (formula in c(f1, f2)) { + model <- glm(formula, family=binomial(link=cloglog), data=data) + print(as.vector(coef(model))) + } + + [1] -0.2832198 0.8434144 -0.2524727 -0.5293452 + [1] 1.5063590 -0.4038015 0.6133664 -0.2687882 -0.5541758 + */ + val expected = Seq( + Vectors.dense(0.0, -0.3560284, 1.3010002, -0.3570805, -0.7406762), + Vectors.dense(2.8367406, -0.5896187, 0.8931655, -0.3925169, -0.7996989), + Vectors.dense(0.0, -0.2134390, 0.7800646, -0.2144267, -0.4438358), + Vectors.dense(1.6995366, -0.3524694, 0.5332651, -0.2352985, -0.4780850), + Vectors.dense(0.0, -0.2832198, 0.8434144, -0.2524727, -0.5293452), + Vectors.dense(1.5063590, -0.4038015, 0.6133664, -0.2687882, -0.5541758)) + + import GeneralizedLinearRegression._ + + var idx = 0 + for ((link, dataset) <- Seq(("logit", datasetBinomial), ("probit", datasetBinomial), + ("cloglog", datasetBinomial))) { + for (fitIntercept <- Seq(false, true)) { + val trainer = new GeneralizedLinearRegression().setFamily("binomial").setLink(link) + .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") + val model = trainer.fit(dataset) + val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1), + model.coefficients(2), model.coefficients(3)) + assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with binomial family, " + + s"$link link and fitIntercept = $fitIntercept.") + + val familyLink = FamilyAndLink(trainer) + model.transform(dataset).select("features", "prediction", "linkPrediction").collect() + .foreach { + case Row(features: DenseVector, prediction1: Double, linkPrediction1: Double) => + val eta = BLAS.dot(features, model.coefficients) + model.intercept + val prediction2 = familyLink.fitted(eta) + val linkPrediction2 = eta + assert(prediction1 ~= prediction2 relTol 1E-5, "Prediction mismatch: GLM with " + + s"binomial family, $link link and fitIntercept = $fitIntercept.") + assert(linkPrediction1 ~= linkPrediction2 relTol 1E-5, "Link Prediction mismatch: " + + s"GLM with binomial family, $link link and fitIntercept = $fitIntercept.") + } + + idx += 1 + } + } + } + + test("generalized linear regression: poisson family against glm") { + /* + R code: + f1 <- data$V1 ~ data$V2 + data$V3 - 1 + f2 <- data$V1 ~ data$V2 + data$V3 + + data <- read.csv("path", header=FALSE) + for (formula in c(f1, f2)) { + model <- glm(formula, family="poisson", data=data) + print(as.vector(coef(model))) + } + + [1] 0.22999393 0.08047088 + [1] 0.25022353 0.21998599 0.05998621 + + data <- read.csv("path", header=FALSE) + for (formula in c(f1, f2)) { + model <- glm(formula, family=poisson(link=identity), data=data) + print(as.vector(coef(model))) + } + + [1] 2.2929501 0.8119415 + [1] 2.5012730 2.1999407 0.5999107 + + data <- read.csv("path", header=FALSE) + for (formula in c(f1, f2)) { + model <- glm(formula, family=poisson(link=sqrt), data=data) + print(as.vector(coef(model))) + } + + [1] 2.2958947 0.8090515 + [1] 2.5000480 2.1999972 0.5999968 + */ + val expected = Seq( + Vectors.dense(0.0, 0.22999393, 0.08047088), + Vectors.dense(0.25022353, 0.21998599, 0.05998621), + Vectors.dense(0.0, 2.2929501, 0.8119415), + Vectors.dense(2.5012730, 2.1999407, 0.5999107), + Vectors.dense(0.0, 2.2958947, 0.8090515), + Vectors.dense(2.5000480, 2.1999972, 0.5999968)) + + import GeneralizedLinearRegression._ + + var idx = 0 + for ((link, dataset) <- Seq(("log", datasetPoissonLog), ("identity", datasetPoissonIdentity), + ("sqrt", datasetPoissonSqrt))) { + for (fitIntercept <- Seq(false, true)) { + val trainer = new GeneralizedLinearRegression().setFamily("poisson").setLink(link) + .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") + val model = trainer.fit(dataset) + val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) + assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with poisson family, " + + s"$link link and fitIntercept = $fitIntercept.") + + val familyLink = FamilyAndLink(trainer) + model.transform(dataset).select("features", "prediction", "linkPrediction").collect() + .foreach { + case Row(features: DenseVector, prediction1: Double, linkPrediction1: Double) => + val eta = BLAS.dot(features, model.coefficients) + model.intercept + val prediction2 = familyLink.fitted(eta) + val linkPrediction2 = eta + assert(prediction1 ~= prediction2 relTol 1E-5, "Prediction mismatch: GLM with " + + s"poisson family, $link link and fitIntercept = $fitIntercept.") + assert(linkPrediction1 ~= linkPrediction2 relTol 1E-5, "Link Prediction mismatch: " + + s"GLM with poisson family, $link link and fitIntercept = $fitIntercept.") + } + + idx += 1 + } + } + } + + test("generalized linear regression: poisson family against glm (with zero values)") { + /* + R code: + f1 <- data$V1 ~ data$V2 + data$V3 - 1 + f2 <- data$V1 ~ data$V2 + data$V3 + + data <- read.csv("path", header=FALSE) + for (formula in c(f1, f2)) { + model <- glm(formula, family="poisson", data=data) + print(as.vector(coef(model))) + } + [1] -0.0457441 -0.6833928 + [1] 1.8121235 -0.1747493 -0.5815417 + */ + val expected = Seq( + Vectors.dense(0.0, -0.0457441, -0.6833928), + Vectors.dense(1.8121235, -0.1747493, -0.5815417)) + + import GeneralizedLinearRegression._ + + var idx = 0 + val link = "log" + val dataset = datasetPoissonLogWithZero + for (fitIntercept <- Seq(false, true)) { + val trainer = new GeneralizedLinearRegression().setFamily("poisson").setLink(link) + .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") + val model = trainer.fit(dataset) + val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) + assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with poisson family, " + + s"$link link and fitIntercept = $fitIntercept (with zero values).") + idx += 1 + } + } + + test("generalized linear regression: gamma family against glm") { + /* + R code: + f1 <- data$V1 ~ data$V2 + data$V3 - 1 + f2 <- data$V1 ~ data$V2 + data$V3 + + data <- read.csv("path", header=FALSE) + for (formula in c(f1, f2)) { + model <- glm(formula, family="Gamma", data=data) + print(as.vector(coef(model))) + } + + [1] 2.3392419 0.8058058 + [1] 2.3507700 2.2533574 0.6042991 + + data <- read.csv("path", header=FALSE) + for (formula in c(f1, f2)) { + model <- glm(formula, family=Gamma(link=identity), data=data) + print(as.vector(coef(model))) + } + + [1] 2.2908883 0.8147796 + [1] 2.5002406 2.1998346 0.6000059 + + data <- read.csv("path", header=FALSE) + for (formula in c(f1, f2)) { + model <- glm(formula, family=Gamma(link=log), data=data) + print(as.vector(coef(model))) + } + + [1] 0.22958970 0.08091066 + [1] 0.25003210 0.21996957 0.06000215 + */ + val expected = Seq( + Vectors.dense(0.0, 2.3392419, 0.8058058), + Vectors.dense(2.3507700, 2.2533574, 0.6042991), + Vectors.dense(0.0, 2.2908883, 0.8147796), + Vectors.dense(2.5002406, 2.1998346, 0.6000059), + Vectors.dense(0.0, 0.22958970, 0.08091066), + Vectors.dense(0.25003210, 0.21996957, 0.06000215)) + + import GeneralizedLinearRegression._ + + var idx = 0 + for ((link, dataset) <- Seq(("inverse", datasetGammaInverse), + ("identity", datasetGammaIdentity), ("log", datasetGammaLog))) { + for (fitIntercept <- Seq(false, true)) { + val trainer = new GeneralizedLinearRegression().setFamily("Gamma").setLink(link) + .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") + val model = trainer.fit(dataset) + val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) + assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with gamma family, " + + s"$link link and fitIntercept = $fitIntercept.") + + val familyLink = FamilyAndLink(trainer) + model.transform(dataset).select("features", "prediction", "linkPrediction").collect() + .foreach { + case Row(features: DenseVector, prediction1: Double, linkPrediction1: Double) => + val eta = BLAS.dot(features, model.coefficients) + model.intercept + val prediction2 = familyLink.fitted(eta) + val linkPrediction2 = eta + assert(prediction1 ~= prediction2 relTol 1E-5, "Prediction mismatch: GLM with " + + s"gamma family, $link link and fitIntercept = $fitIntercept.") + assert(linkPrediction1 ~= linkPrediction2 relTol 1E-5, "Link Prediction mismatch: " + + s"GLM with gamma family, $link link and fitIntercept = $fitIntercept.") + } + + idx += 1 + } + } + } + + test("generalized linear regression: tweedie family against glm") { + /* + R code: + library(statmod) + df <- as.data.frame(matrix(c( + 1.0, 1.0, 0.0, 5.0, + 0.5, 1.0, 1.0, 2.0, + 1.0, 1.0, 2.0, 1.0, + 2.0, 1.0, 3.0, 3.0), 4, 4, byrow = TRUE)) + + f1 <- V1 ~ -1 + V3 + V4 + f2 <- V1 ~ V3 + V4 + + for (f in c(f1, f2)) { + for (lp in c(0, 1, -1)) + for (vp in c(1.6, 2.5)) { + model <- glm(f, df, family = tweedie(var.power = vp, link.power = lp)) + print(as.vector(coef(model))) + } + } + [1] 0.1496480 -0.0122283 + [1] 0.1373567 -0.0120673 + [1] 0.3919109 0.1846094 + [1] 0.3684426 0.1810662 + [1] 0.1759887 0.2195818 + [1] 0.1108561 0.2059430 + [1] -1.3163732 0.4378139 0.2464114 + [1] -1.4396020 0.4817364 0.2680088 + [1] -0.7090230 0.6256309 0.3294324 + [1] -0.9524928 0.7304267 0.3792687 + [1] 2.1188978 -0.3360519 -0.2067023 + [1] 2.1659028 -0.3499170 -0.2128286 + */ + val datasetTweedie = Seq( + Instance(1.0, 1.0, Vectors.dense(0.0, 5.0)), + Instance(0.5, 1.0, Vectors.dense(1.0, 2.0)), + Instance(1.0, 1.0, Vectors.dense(2.0, 1.0)), + Instance(2.0, 1.0, Vectors.dense(3.0, 3.0)) + ).toDF() + + val expected = Seq( + Vectors.dense(0, 0.149648, -0.0122283), + Vectors.dense(0, 0.1373567, -0.0120673), + Vectors.dense(0, 0.3919109, 0.1846094), + Vectors.dense(0, 0.3684426, 0.1810662), + Vectors.dense(0, 0.1759887, 0.2195818), + Vectors.dense(0, 0.1108561, 0.205943), + Vectors.dense(-1.3163732, 0.4378139, 0.2464114), + Vectors.dense(-1.439602, 0.4817364, 0.2680088), + Vectors.dense(-0.709023, 0.6256309, 0.3294324), + Vectors.dense(-0.9524928, 0.7304267, 0.3792687), + Vectors.dense(2.1188978, -0.3360519, -0.2067023), + Vectors.dense(2.1659028, -0.349917, -0.2128286)) + + import GeneralizedLinearRegression._ + + var idx = 0 + for (fitIntercept <- Seq(false, true); + linkPower <- Seq(0.0, 1.0, -1.0); + variancePower <- Seq(1.6, 2.5)) { + val trainer = new GeneralizedLinearRegression().setFamily("tweedie") + .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") + .setVariancePower(variancePower).setLinkPower(linkPower) + val model = trainer.fit(datasetTweedie) + val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) + assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with tweedie family, " + + s"linkPower = $linkPower, fitIntercept = $fitIntercept " + + s"and variancePower = $variancePower.") + + val familyLink = FamilyAndLink(trainer) + model.transform(datasetTweedie).select("features", "prediction", "linkPrediction").collect() + .foreach { + case Row(features: DenseVector, prediction1: Double, linkPrediction1: Double) => + val eta = BLAS.dot(features, model.coefficients) + model.intercept + val prediction2 = familyLink.fitted(eta) + val linkPrediction2 = eta + assert(prediction1 ~= prediction2 relTol 1E-5, "Prediction mismatch: GLM with " + + s"tweedie family, linkPower = $linkPower, fitIntercept = $fitIntercept " + + s"and variancePower = $variancePower.") + assert(linkPrediction1 ~= linkPrediction2 relTol 1E-5, "Link Prediction mismatch: " + + s"GLM with tweedie family, linkPower = $linkPower, fitIntercept = $fitIntercept " + + s"and variancePower = $variancePower.") + } + idx += 1 + } + } + + test("generalized linear regression: tweedie family against glm (default power link)") { + /* + R code: + library(statmod) + df <- as.data.frame(matrix(c( + 1.0, 1.0, 0.0, 5.0, + 0.5, 1.0, 1.0, 2.0, + 1.0, 1.0, 2.0, 1.0, + 2.0, 1.0, 3.0, 3.0), 4, 4, byrow = TRUE)) + var.power <- c(0, 1, 2, 1.5) + f1 <- V1 ~ -1 + V3 + V4 + f2 <- V1 ~ V3 + V4 + for (f in c(f1, f2)) { + for (vp in var.power) { + model <- glm(f, df, family = tweedie(var.power = vp)) + print(as.vector(coef(model))) + } + } + [1] 0.4310345 0.1896552 + [1] 0.15776482 -0.01189032 + [1] 0.1468853 0.2116519 + [1] 0.2282601 0.2132775 + [1] -0.5158730 0.5555556 0.2936508 + [1] -1.2689559 0.4230934 0.2388465 + [1] 2.137852 -0.341431 -0.209090 + [1] 1.5953393 -0.1884985 -0.1106335 + */ + val datasetTweedie = Seq( + Instance(1.0, 1.0, Vectors.dense(0.0, 5.0)), + Instance(0.5, 1.0, Vectors.dense(1.0, 2.0)), + Instance(1.0, 1.0, Vectors.dense(2.0, 1.0)), + Instance(2.0, 1.0, Vectors.dense(3.0, 3.0)) + ).toDF() + + val expected = Seq( + Vectors.dense(0, 0.4310345, 0.1896552), + Vectors.dense(0, 0.15776482, -0.01189032), + Vectors.dense(0, 0.1468853, 0.2116519), + Vectors.dense(0, 0.2282601, 0.2132775), + Vectors.dense(-0.515873, 0.5555556, 0.2936508), + Vectors.dense(-1.2689559, 0.4230934, 0.2388465), + Vectors.dense(2.137852, -0.341431, -0.20909), + Vectors.dense(1.5953393, -0.1884985, -0.1106335)) + + import GeneralizedLinearRegression._ + + var idx = 0 + for (fitIntercept <- Seq(false, true)) { + for (variancePower <- Seq(0.0, 1.0, 2.0, 1.5)) { + val trainer = new GeneralizedLinearRegression().setFamily("tweedie") + .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") + .setVariancePower(variancePower) + val model = trainer.fit(datasetTweedie) + val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) + assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with tweedie family, " + + s"fitIntercept = $fitIntercept and variancePower = $variancePower.") + + val familyLink = FamilyAndLink(trainer) + model.transform(datasetTweedie).select("features", "prediction", "linkPrediction").collect() + .foreach { + case Row(features: DenseVector, prediction1: Double, linkPrediction1: Double) => + val eta = BLAS.dot(features, model.coefficients) + model.intercept + val prediction2 = familyLink.fitted(eta) + val linkPrediction2 = eta + assert(prediction1 ~= prediction2 relTol 1E-5, "Prediction mismatch: GLM with " + + s"tweedie family, fitIntercept = $fitIntercept " + + s"and variancePower = $variancePower.") + assert(linkPrediction1 ~= linkPrediction2 relTol 1E-5, "Link Prediction mismatch: " + + s"GLM with tweedie family, fitIntercept = $fitIntercept " + + s"and variancePower = $variancePower.") + } + idx += 1 + } + } + } + + test("generalized linear regression: intercept only") { + /* + R code: + + library(statmod) + y <- c(1.0, 0.5, 0.7, 0.3) + w <- c(1, 2, 3, 4) + for (fam in list(binomial(), Gamma(), gaussian(), poisson(), tweedie(1.6))) { + model1 <- glm(y ~ 1, family = fam) + model2 <- glm(y ~ 1, family = fam, weights = w) + print(as.vector(c(coef(model1), coef(model2)))) + } + [1] 0.5108256 0.1201443 + [1] 1.600000 1.886792 + [1] 0.625 0.530 + [1] -0.4700036 -0.6348783 + [1] 1.325782 1.463641 + */ + + val dataset = Seq( + Instance(1.0, 1.0, Vectors.zeros(0)), + Instance(0.5, 2.0, Vectors.zeros(0)), + Instance(0.7, 3.0, Vectors.zeros(0)), + Instance(0.3, 4.0, Vectors.zeros(0)) + ).toDF() + + val expected = Seq(0.5108256, 0.1201443, 1.600000, 1.886792, 0.625, 0.530, + -0.4700036, -0.6348783, 1.325782, 1.463641) + + import GeneralizedLinearRegression._ + + var idx = 0 + for (family <- GeneralizedLinearRegression.supportedFamilyNames.sortWith(_ < _)) { + for (useWeight <- Seq(false, true)) { + val trainer = new GeneralizedLinearRegression().setFamily(family) + if (useWeight) trainer.setWeightCol("weight") + if (family == "tweedie") trainer.setVariancePower(1.6) + val model = trainer.fit(dataset) + val actual = model.intercept + assert(actual ~== expected(idx) absTol 1E-3, "Model mismatch: intercept only GLM with " + + s"useWeight = $useWeight and family = $family.") + assert(model.coefficients === new DenseVector(Array.empty[Double])) + idx += 1 + } + } + + // throw exception for empty model + val trainer = new GeneralizedLinearRegression().setFitIntercept(false) + withClue("Specified model is empty with neither intercept nor feature") { + intercept[IllegalArgumentException] { + trainer.fit(dataset) + } + } + } + + test("generalized linear regression with weight and offset") { + /* + R code: + library(statmod) + + df <- as.data.frame(matrix(c( + 0.2, 1.0, 2.0, 0.0, 5.0, + 0.5, 2.1, 0.5, 1.0, 2.0, + 0.9, 0.4, 1.0, 2.0, 1.0, + 0.7, 0.7, 0.0, 3.0, 3.0), 4, 5, byrow = TRUE)) + families <- list(binomial, Gamma, gaussian, poisson, tweedie(1.5)) + f1 <- V1 ~ -1 + V4 + V5 + f2 <- V1 ~ V4 + V5 + for (f in c(f1, f2)) { + for (fam in families) { + model <- glm(f, df, family = fam, weights = V2, offset = V3) + print(as.vector(coef(model))) + } + } + [1] 0.9419107 -0.6864404 + [1] -0.2869094 0.7857710 + [1] 0.5169222 -0.3344444 + [1] 0.1812436 -0.6568422 + [1] 0.1055254 0.2979113 + [1] -0.2147117 0.9911750 -0.6356096 + [1] 0.3390397 -0.3406099 0.6870259 + [1] -0.05990345 0.53188982 -0.32118415 + [1] -1.5616130 0.6646470 -0.3192581 + [1] 0.3665034 0.1039416 0.1484616 + */ + val dataset = Seq( + OffsetInstance(0.2, 1.0, 2.0, Vectors.dense(0.0, 5.0)), + OffsetInstance(0.5, 2.1, 0.5, Vectors.dense(1.0, 2.0)), + OffsetInstance(0.9, 0.4, 1.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(0.7, 0.7, 0.0, Vectors.dense(3.0, 3.0)) + ).toDF() + + val expected = Seq( + Vectors.dense(0, 0.9419107, -0.6864404), + Vectors.dense(0, -0.2869094, 0.785771), + Vectors.dense(0, 0.5169222, -0.3344444), + Vectors.dense(0, 0.1812436, -0.6568422), + Vectors.dense(0, 0.1055254, 0.2979113), + Vectors.dense(-0.2147117, 0.991175, -0.6356096), + Vectors.dense(0.3390397, -0.3406099, 0.6870259), + Vectors.dense(-0.05990345, 0.53188982, -0.32118415), + Vectors.dense(-1.561613, 0.664647, -0.3192581), + Vectors.dense(0.3665034, 0.1039416, 0.1484616)) + + import GeneralizedLinearRegression._ + + var idx = 0 + for (fitIntercept <- Seq(false, true)) { + for (family <- GeneralizedLinearRegression.supportedFamilyNames.sortWith(_ < _)) { + val trainer = new GeneralizedLinearRegression().setFamily(family) + .setFitIntercept(fitIntercept).setOffsetCol("offset") + .setWeightCol("weight").setLinkPredictionCol("linkPrediction") + if (family == "tweedie") trainer.setVariancePower(1.5) + val model = trainer.fit(dataset) + val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) + assert(actual ~= expected(idx) absTol 1e-4, s"Model mismatch: GLM with family = $family," + + s" and fitIntercept = $fitIntercept.") + + val familyLink = FamilyAndLink(trainer) + model.transform(dataset).select("features", "offset", "prediction", "linkPrediction") + .collect().foreach { + case Row(features: DenseVector, offset: Double, prediction1: Double, + linkPrediction1: Double) => + val eta = BLAS.dot(features, model.coefficients) + model.intercept + offset + val prediction2 = familyLink.fitted(eta) + val linkPrediction2 = eta + assert(prediction1 ~= prediction2 relTol 1E-5, "Prediction mismatch: GLM with " + + s"family = $family, and fitIntercept = $fitIntercept.") + assert(linkPrediction1 ~= linkPrediction2 relTol 1E-5, "Link Prediction mismatch: " + + s"GLM with family = $family, and fitIntercept = $fitIntercept.") + } + + idx += 1 + } + } + } + + test("glm summary: gaussian family with weight and offset") { + /* + R code: + + A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) + b <- c(17, 19, 23, 29) + w <- c(1, 2, 3, 4) + off <- c(2, 3, 1, 4) + df <- as.data.frame(cbind(A, b)) + */ + val dataset = Seq( + OffsetInstance(17.0, 1.0, 2.0, Vectors.dense(0.0, 5.0).toSparse), + OffsetInstance(19.0, 2.0, 3.0, Vectors.dense(1.0, 7.0)), + OffsetInstance(23.0, 3.0, 1.0, Vectors.dense(2.0, 11.0)), + OffsetInstance(29.0, 4.0, 4.0, Vectors.dense(3.0, 13.0)) + ).toDF() + /* + R code: + + model <- glm(formula = "b ~ .", family = "gaussian", data = df, + weights = w, offset = off) + summary(model) + + Deviance Residuals: + 1 2 3 4 + 0.9600 -0.6788 -0.5543 0.4800 + + Coefficients: + Estimate Std. Error t value Pr(>|t|) + (Intercept) 5.5400 4.8040 1.153 0.455 + V1 -0.9600 2.7782 -0.346 0.788 + V2 1.7000 0.9798 1.735 0.333 + + (Dispersion parameter for gaussian family taken to be 1.92) + + Null deviance: 152.10 on 3 degrees of freedom + Residual deviance: 1.92 on 1 degrees of freedom + AIC: 13.238 + + Number of Fisher Scoring iterations: 2 + + residuals(model, type = "pearson") + 1 2 3 4 + 0.9600000 -0.6788225 -0.5542563 0.4800000 + residuals(model, type = "working") + 1 2 3 4 + 0.96 -0.48 -0.32 0.24 + residuals(model, type = "response") + 1 2 3 4 + 0.96 -0.48 -0.32 0.24 + */ + val trainer = new GeneralizedLinearRegression() + .setWeightCol("weight").setOffsetCol("offset") + + val model = trainer.fit(dataset) + + val coefficientsR = Vectors.dense(Array(-0.96, 1.7)) + val interceptR = 5.54 + val devianceResidualsR = Array(0.96, -0.67882, -0.55426, 0.48) + val pearsonResidualsR = Array(0.96, -0.67882, -0.55426, 0.48) + val workingResidualsR = Array(0.96, -0.48, -0.32, 0.24) + val responseResidualsR = Array(0.96, -0.48, -0.32, 0.24) + val seCoefR = Array(2.7782, 0.9798, 4.804) + val tValsR = Array(-0.34555, 1.73506, 1.15321) + val pValsR = Array(0.78819, 0.33286, 0.45478) + val dispersionR = 1.92 + val nullDevianceR = 152.1 + val residualDevianceR = 1.92 + val residualDegreeOfFreedomNullR = 3 + val residualDegreeOfFreedomR = 1 + val aicR = 13.23758 + + assert(model.hasSummary) + val summary = model.summary + assert(summary.isInstanceOf[GeneralizedLinearRegressionTrainingSummary]) + + val devianceResiduals = summary.residuals() + .select(col("devianceResiduals")) + .collect() + .map(_.getDouble(0)) + val pearsonResiduals = summary.residuals("pearson") + .select(col("pearsonResiduals")) + .collect() + .map(_.getDouble(0)) + val workingResiduals = summary.residuals("working") + .select(col("workingResiduals")) + .collect() + .map(_.getDouble(0)) + val responseResiduals = summary.residuals("response") + .select(col("responseResiduals")) + .collect() + .map(_.getDouble(0)) + + assert(model.coefficients ~== coefficientsR absTol 1E-3) + assert(model.intercept ~== interceptR absTol 1E-3) + devianceResiduals.zip(devianceResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + pearsonResiduals.zip(pearsonResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + workingResiduals.zip(workingResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + responseResiduals.zip(responseResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + summary.coefficientStandardErrors.zip(seCoefR).foreach{ x => + assert(x._1 ~== x._2 absTol 1E-3) } + summary.tValues.zip(tValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } + summary.pValues.zip(pValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } + assert(summary.dispersion ~== dispersionR absTol 1E-3) + assert(summary.nullDeviance ~== nullDevianceR absTol 1E-3) + assert(summary.deviance ~== residualDevianceR absTol 1E-3) + assert(summary.residualDegreeOfFreedom === residualDegreeOfFreedomR) + assert(summary.residualDegreeOfFreedomNull === residualDegreeOfFreedomNullR) + assert(summary.aic ~== aicR absTol 1E-3) + assert(summary.solver === "irls") + + val summary2: GeneralizedLinearRegressionSummary = model.evaluate(dataset) + assert(summary.predictions.columns.toSet === summary2.predictions.columns.toSet) + assert(summary.predictionCol === summary2.predictionCol) + assert(summary.rank === summary2.rank) + assert(summary.degreesOfFreedom === summary2.degreesOfFreedom) + assert(summary.residualDegreeOfFreedom === summary2.residualDegreeOfFreedom) + assert(summary.residualDegreeOfFreedomNull === summary2.residualDegreeOfFreedomNull) + assert(summary.nullDeviance === summary2.nullDeviance) + assert(summary.deviance === summary2.deviance) + assert(summary.dispersion === summary2.dispersion) + assert(summary.aic === summary2.aic) + } + + test("glm summary: binomial family with weight and offset") { + /* + R code: + + df <- as.data.frame(matrix(c( + 0.2, 1.0, 2.0, 0.0, 5.0, + 0.5, 2.1, 0.5, 1.0, 2.0, + 0.9, 0.4, 1.0, 2.0, 1.0, + 0.7, 0.7, 0.0, 3.0, 3.0), 4, 5, byrow = TRUE)) + */ + val dataset = Seq( + OffsetInstance(0.2, 1.0, 2.0, Vectors.dense(0.0, 5.0)), + OffsetInstance(0.5, 2.1, 0.5, Vectors.dense(1.0, 2.0)), + OffsetInstance(0.9, 0.4, 1.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(0.7, 0.7, 0.0, Vectors.dense(3.0, 3.0)) + ).toDF() + /* + R code: + + model <- glm(formula = "V1 ~ V4 + V5", family = "binomial", data = df, + weights = V2, offset = V3) + summary(model) + + Deviance Residuals: + 1 2 3 4 + 0.002584 -0.003800 0.012478 -0.001796 + + Coefficients: + Estimate Std. Error z value Pr(>|z|) + (Intercept) -0.2147 3.5687 -0.060 0.952 + V4 0.9912 1.2344 0.803 0.422 + V5 -0.6356 0.9669 -0.657 0.511 + + (Dispersion parameter for binomial family taken to be 1) + + Null deviance: 2.17560881 on 3 degrees of freedom + Residual deviance: 0.00018005 on 1 degrees of freedom + AIC: 10.245 + + Number of Fisher Scoring iterations: 4 + + residuals(model, type = "pearson") + 1 2 3 4 + 0.002586113 -0.003799744 0.012372235 -0.001796892 + residuals(model, type = "working") + 1 2 3 4 + 0.006477857 -0.005244163 0.063541250 -0.004691064 + residuals(model, type = "response") + 1 2 3 4 + 0.0010324375 -0.0013110318 0.0060225522 -0.0009832738 + */ + val trainer = new GeneralizedLinearRegression() + .setFamily("Binomial") + .setWeightCol("weight") + .setOffsetCol("offset") + + val model = trainer.fit(dataset) + + val coefficientsR = Vectors.dense(Array(0.99117, -0.63561)) + val interceptR = -0.21471 + val devianceResidualsR = Array(0.00258, -0.0038, 0.01248, -0.0018) + val pearsonResidualsR = Array(0.00259, -0.0038, 0.01237, -0.0018) + val workingResidualsR = Array(0.00648, -0.00524, 0.06354, -0.00469) + val responseResidualsR = Array(0.00103, -0.00131, 0.00602, -0.00098) + val seCoefR = Array(1.23439, 0.9669, 3.56866) + val tValsR = Array(0.80297, -0.65737, -0.06017) + val pValsR = Array(0.42199, 0.51094, 0.95202) + val dispersionR = 1.0 + val nullDevianceR = 2.17561 + val residualDevianceR = 0.00018 + val residualDegreeOfFreedomNullR = 3 + val residualDegreeOfFreedomR = 1 + val aicR = 10.24453 + + val summary = model.summary + val devianceResiduals = summary.residuals() + .select(col("devianceResiduals")) + .collect() + .map(_.getDouble(0)) + val pearsonResiduals = summary.residuals("pearson") + .select(col("pearsonResiduals")) + .collect() + .map(_.getDouble(0)) + val workingResiduals = summary.residuals("working") + .select(col("workingResiduals")) + .collect() + .map(_.getDouble(0)) + val responseResiduals = summary.residuals("response") + .select(col("responseResiduals")) + .collect() + .map(_.getDouble(0)) + + assert(model.coefficients ~== coefficientsR absTol 1E-3) + assert(model.intercept ~== interceptR absTol 1E-3) + devianceResiduals.zip(devianceResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + pearsonResiduals.zip(pearsonResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + workingResiduals.zip(workingResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + responseResiduals.zip(responseResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + summary.coefficientStandardErrors.zip(seCoefR).foreach{ x => + assert(x._1 ~== x._2 absTol 1E-3) } + summary.tValues.zip(tValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } + summary.pValues.zip(pValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } + assert(summary.dispersion === dispersionR) + assert(summary.nullDeviance ~== nullDevianceR absTol 1E-3) + assert(summary.deviance ~== residualDevianceR absTol 1E-3) + assert(summary.residualDegreeOfFreedom === residualDegreeOfFreedomR) + assert(summary.residualDegreeOfFreedomNull === residualDegreeOfFreedomNullR) + assert(summary.aic ~== aicR absTol 1E-3) + assert(summary.solver === "irls") + } + + test("glm summary: poisson family with weight and offset") { + /* + R code: + + A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) + b <- c(2, 8, 3, 9) + w <- c(1, 2, 3, 4) + off <- c(2, 3, 1, 4) + df <- as.data.frame(cbind(A, b)) + */ + val dataset = Seq( + OffsetInstance(2.0, 1.0, 2.0, Vectors.dense(0.0, 5.0).toSparse), + OffsetInstance(8.0, 2.0, 3.0, Vectors.dense(1.0, 7.0)), + OffsetInstance(3.0, 3.0, 1.0, Vectors.dense(2.0, 11.0)), + OffsetInstance(9.0, 4.0, 4.0, Vectors.dense(3.0, 13.0)) + ).toDF() + /* + R code: + + model <- glm(formula = "b ~ .", family = "poisson", data = df, + weights = w, offset = off) + summary(model) + + Deviance Residuals: + 1 2 3 4 + -2.0480 1.2315 1.8293 -0.7107 + + Coefficients: + Estimate Std. Error z value Pr(>|z|) + (Intercept) -4.5678 1.9625 -2.328 0.0199 + V1 -2.8784 1.1683 -2.464 0.0137 + V2 0.8859 0.4170 2.124 0.0336 + + (Dispersion parameter for poisson family taken to be 1) + + Null deviance: 22.5585 on 3 degrees of freedom + Residual deviance: 9.5622 on 1 degrees of freedom + AIC: 51.242 + + Number of Fisher Scoring iterations: 5 + + residuals(model, type = "pearson") + 1 2 3 4 + -1.7480418 1.3037611 2.0750099 -0.6972966 + residuals(model, type = "working") + 1 2 3 4 + -0.6891489 0.3833588 0.9710682 -0.1096590 + residuals(model, type = "response") + 1 2 3 4 + -4.433948 2.216974 1.477983 -1.108487 + */ + val trainer = new GeneralizedLinearRegression() + .setFamily("Poisson") + .setWeightCol("weight") + .setOffsetCol("offset") + + val model = trainer.fit(dataset) + + val coefficientsR = Vectors.dense(Array(-2.87843, 0.88589)) + val interceptR = -4.56784 + val devianceResidualsR = Array(-2.04796, 1.23149, 1.82933, -0.71066) + val pearsonResidualsR = Array(-1.74804, 1.30376, 2.07501, -0.6973) + val workingResidualsR = Array(-0.68915, 0.38336, 0.97107, -0.10966) + val responseResidualsR = Array(-4.43395, 2.21697, 1.47798, -1.10849) + val seCoefR = Array(1.16826, 0.41703, 1.96249) + val tValsR = Array(-2.46387, 2.12428, -2.32757) + val pValsR = Array(0.01374, 0.03365, 0.01993) + val dispersionR = 1.0 + val nullDevianceR = 22.55853 + val residualDevianceR = 9.5622 + val residualDegreeOfFreedomNullR = 3 + val residualDegreeOfFreedomR = 1 + val aicR = 51.24218 + + val summary = model.summary + val devianceResiduals = summary.residuals() + .select(col("devianceResiduals")) + .collect() + .map(_.getDouble(0)) + val pearsonResiduals = summary.residuals("pearson") + .select(col("pearsonResiduals")) + .collect() + .map(_.getDouble(0)) + val workingResiduals = summary.residuals("working") + .select(col("workingResiduals")) + .collect() + .map(_.getDouble(0)) + val responseResiduals = summary.residuals("response") + .select(col("responseResiduals")) + .collect() + .map(_.getDouble(0)) + + assert(model.coefficients ~== coefficientsR absTol 1E-3) + assert(model.intercept ~== interceptR absTol 1E-3) + devianceResiduals.zip(devianceResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + pearsonResiduals.zip(pearsonResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + workingResiduals.zip(workingResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + responseResiduals.zip(responseResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + summary.coefficientStandardErrors.zip(seCoefR).foreach{ x => + assert(x._1 ~== x._2 absTol 1E-3) } + summary.tValues.zip(tValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } + summary.pValues.zip(pValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } + assert(summary.dispersion === dispersionR) + assert(summary.nullDeviance ~== nullDevianceR absTol 1E-3) + assert(summary.deviance ~== residualDevianceR absTol 1E-3) + assert(summary.residualDegreeOfFreedom === residualDegreeOfFreedomR) + assert(summary.residualDegreeOfFreedomNull === residualDegreeOfFreedomNullR) + assert(summary.aic ~== aicR absTol 1E-3) + assert(summary.solver === "irls") + } + + test("glm summary: gamma family with weight and offset") { + /* + R code: + + A <- matrix(c(0, 5, 1, 2, 2, 1, 3, 3), 4, 2, byrow = TRUE) + b <- c(1, 2, 1, 2) + w <- c(1, 2, 3, 4) + off <- c(0, 0.5, 1, 0) + df <- as.data.frame(cbind(A, b)) + */ + val dataset = Seq( + OffsetInstance(1.0, 1.0, 0.0, Vectors.dense(0.0, 5.0)), + OffsetInstance(2.0, 2.0, 0.5, Vectors.dense(1.0, 2.0)), + OffsetInstance(1.0, 3.0, 1.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(2.0, 4.0, 0.0, Vectors.dense(3.0, 3.0)) + ).toDF() + /* + R code: + + model <- glm(formula = "b ~ .", family = "Gamma", data = df, + weights = w, offset = off) + summary(model) + + Deviance Residuals: + 1 2 3 4 + -0.17095 0.19867 -0.23604 0.03241 + + Coefficients: + Estimate Std. Error t value Pr(>|t|) + (Intercept) -0.56474 0.23866 -2.366 0.255 + V1 0.07695 0.06931 1.110 0.467 + V2 0.28068 0.07320 3.835 0.162 + + (Dispersion parameter for Gamma family taken to be 0.1212174) + + Null deviance: 2.02568 on 3 degrees of freedom + Residual deviance: 0.12546 on 1 degrees of freedom + AIC: 0.93388 + + Number of Fisher Scoring iterations: 4 + + residuals(model, type = "pearson") + 1 2 3 4 + -0.16134949 0.20807694 -0.22544551 0.03258777 + residuals(model, type = "working") + 1 2 3 4 + 0.135315831 -0.084390309 0.113219135 -0.008279688 + residuals(model, type = "response") + 1 2 3 4 + -0.1923918 0.2565224 -0.1496381 0.0320653 + */ + val trainer = new GeneralizedLinearRegression() + .setFamily("Gamma") + .setWeightCol("weight") + .setOffsetCol("offset") + + val model = trainer.fit(dataset) + + val coefficientsR = Vectors.dense(Array(0.07695, 0.28068)) + val interceptR = -0.56474 + val devianceResidualsR = Array(-0.17095, 0.19867, -0.23604, 0.03241) + val pearsonResidualsR = Array(-0.16135, 0.20808, -0.22545, 0.03259) + val workingResidualsR = Array(0.13532, -0.08439, 0.11322, -0.00828) + val responseResidualsR = Array(-0.19239, 0.25652, -0.14964, 0.03207) + val seCoefR = Array(0.06931, 0.0732, 0.23866) + val tValsR = Array(1.11031, 3.83453, -2.3663) + val pValsR = Array(0.46675, 0.16241, 0.25454) + val dispersionR = 0.12122 + val nullDevianceR = 2.02568 + val residualDevianceR = 0.12546 + val residualDegreeOfFreedomNullR = 3 + val residualDegreeOfFreedomR = 1 + val aicR = 0.93388 + + val summary = model.summary + val devianceResiduals = summary.residuals() + .select(col("devianceResiduals")) + .collect() + .map(_.getDouble(0)) + val pearsonResiduals = summary.residuals("pearson") + .select(col("pearsonResiduals")) + .collect() + .map(_.getDouble(0)) + val workingResiduals = summary.residuals("working") + .select(col("workingResiduals")) + .collect() + .map(_.getDouble(0)) + val responseResiduals = summary.residuals("response") + .select(col("responseResiduals")) + .collect() + .map(_.getDouble(0)) + + assert(model.coefficients ~== coefficientsR absTol 1E-3) + assert(model.intercept ~== interceptR absTol 1E-3) + devianceResiduals.zip(devianceResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + pearsonResiduals.zip(pearsonResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + workingResiduals.zip(workingResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + responseResiduals.zip(responseResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + summary.coefficientStandardErrors.zip(seCoefR).foreach{ x => + assert(x._1 ~== x._2 absTol 1E-3) } + summary.tValues.zip(tValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } + summary.pValues.zip(pValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } + assert(summary.dispersion ~== dispersionR absTol 1E-3) + assert(summary.nullDeviance ~== nullDevianceR absTol 1E-3) + assert(summary.deviance ~== residualDevianceR absTol 1E-3) + assert(summary.residualDegreeOfFreedom === residualDegreeOfFreedomR) + assert(summary.residualDegreeOfFreedomNull === residualDegreeOfFreedomNullR) + assert(summary.aic ~== aicR absTol 1E-3) + assert(summary.solver === "irls") + } + + test("glm summary: tweedie family with weight and offset") { + /* + R code: + + df <- as.data.frame(matrix(c( + 1.0, 1.0, 1.0, 0.0, 5.0, + 0.5, 2.0, 3.0, 1.0, 2.0, + 1.0, 3.0, 2.0, 2.0, 1.0, + 0.0, 4.0, 0.0, 3.0, 3.0), 4, 5, byrow = TRUE)) + */ + val dataset = Seq( + OffsetInstance(1.0, 1.0, 1.0, Vectors.dense(0.0, 5.0)), + OffsetInstance(0.5, 2.0, 3.0, Vectors.dense(1.0, 2.0)), + OffsetInstance(1.0, 3.0, 2.0, Vectors.dense(2.0, 1.0)), + OffsetInstance(0.0, 4.0, 0.0, Vectors.dense(3.0, 3.0)) + ).toDF() + /* + R code: + + library(statmod) + model <- glm(V1 ~ V4 + V5, data = df, weights = V2, offset = V3, + family = tweedie(var.power = 1.6, link.power = 0.0)) + summary(model) + + Deviance Residuals: + 1 2 3 4 + 0.8917 -2.1396 1.2252 -1.7946 + + Coefficients: + Estimate Std. Error t value Pr(>|t|) + (Intercept) -0.03047 3.65000 -0.008 0.995 + V4 -1.14577 1.41674 -0.809 0.567 + V5 -0.36585 0.97065 -0.377 0.771 + + (Dispersion parameter for Tweedie family taken to be 6.334961) + + Null deviance: 12.784 on 3 degrees of freedom + Residual deviance: 10.095 on 1 degrees of freedom + AIC: NA + + Number of Fisher Scoring iterations: 18 + + residuals(model, type = "pearson") + 1 2 3 4 + 1.1472554 -1.4642569 1.4935199 -0.8025842 + residuals(model, type = "working") + 1 2 3 4 + 1.3624928 -0.8322375 0.9894580 -1.0000000 + residuals(model, type = "response") + 1 2 3 4 + 0.57671828 -2.48040354 0.49735052 -0.01040646 + */ + val trainer = new GeneralizedLinearRegression() + .setFamily("tweedie") + .setVariancePower(1.6) + .setLinkPower(0.0) + .setWeightCol("weight") + .setOffsetCol("offset") + + val model = trainer.fit(dataset) + + val coefficientsR = Vectors.dense(Array(-1.14577, -0.36585)) + val interceptR = -0.03047 + val devianceResidualsR = Array(0.89171, -2.13961, 1.2252, -1.79463) + val pearsonResidualsR = Array(1.14726, -1.46426, 1.49352, -0.80258) + val workingResidualsR = Array(1.36249, -0.83224, 0.98946, -1) + val responseResidualsR = Array(0.57672, -2.4804, 0.49735, -0.01041) + val seCoefR = Array(1.41674, 0.97065, 3.65) + val tValsR = Array(-0.80873, -0.37691, -0.00835) + val pValsR = Array(0.56707, 0.77053, 0.99468) + val dispersionR = 6.33496 + val nullDevianceR = 12.78358 + val residualDevianceR = 10.09488 + val residualDegreeOfFreedomNullR = 3 + val residualDegreeOfFreedomR = 1 + + val summary = model.summary + + val devianceResiduals = summary.residuals() + .select(col("devianceResiduals")) + .collect() + .map(_.getDouble(0)) + val pearsonResiduals = summary.residuals("pearson") + .select(col("pearsonResiduals")) + .collect() + .map(_.getDouble(0)) + val workingResiduals = summary.residuals("working") + .select(col("workingResiduals")) + .collect() + .map(_.getDouble(0)) + val responseResiduals = summary.residuals("response") + .select(col("responseResiduals")) + .collect() + .map(_.getDouble(0)) + + assert(model.coefficients ~== coefficientsR absTol 1E-3) + assert(model.intercept ~== interceptR absTol 1E-3) + devianceResiduals.zip(devianceResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + pearsonResiduals.zip(pearsonResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + workingResiduals.zip(workingResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + responseResiduals.zip(responseResidualsR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-3) } + + summary.coefficientStandardErrors.zip(seCoefR).foreach{ x => + assert(x._1 ~== x._2 absTol 1E-3) } + summary.tValues.zip(tValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } + summary.pValues.zip(pValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } + + assert(summary.dispersion ~== dispersionR absTol 1E-3) + assert(summary.nullDeviance ~== nullDevianceR absTol 1E-3) + assert(summary.deviance ~== residualDevianceR absTol 1E-3) + assert(summary.residualDegreeOfFreedom === residualDegreeOfFreedomR) + assert(summary.residualDegreeOfFreedomNull === residualDegreeOfFreedomNullR) + assert(summary.solver === "irls") + } + + test("glm handle collinear features") { + val collinearInstances = Seq( + Instance(1.0, 1.0, Vectors.dense(1.0, 2.0)), + Instance(2.0, 1.0, Vectors.dense(2.0, 4.0)), + Instance(3.0, 1.0, Vectors.dense(3.0, 6.0)), + Instance(4.0, 1.0, Vectors.dense(4.0, 8.0)) + ).toDF() + val trainer = new GeneralizedLinearRegression() + val model = trainer.fit(collinearInstances) + // to make it clear that underlying WLS did not solve analytically + intercept[UnsupportedOperationException] { + model.summary.coefficientStandardErrors + } + intercept[UnsupportedOperationException] { + model.summary.pValues + } + intercept[UnsupportedOperationException] { + model.summary.tValues + } + } + + test("read/write") { + def checkModelData( + model: GeneralizedLinearRegressionModel, + model2: GeneralizedLinearRegressionModel): Unit = { + assert(model.intercept === model2.intercept) + assert(model.coefficients.toArray === model2.coefficients.toArray) + } + + val glr = new GeneralizedLinearRegression() + testEstimatorAndModelReadWrite(glr, datasetPoissonLog, + GeneralizedLinearRegressionSuite.allParamSettings, + GeneralizedLinearRegressionSuite.allParamSettings, checkModelData) + } + + test("should support all NumericType labels and weights, and not support other types") { + val glr = new GeneralizedLinearRegression().setMaxIter(1) + MLTestingUtils.checkNumericTypes[ + GeneralizedLinearRegressionModel, GeneralizedLinearRegression]( + glr, spark, isClassification = false) { (expected, actual) => + assert(expected.intercept === actual.intercept) + assert(expected.coefficients === actual.coefficients) + } + } + + test("glm accepts Dataset[LabeledPoint]") { + val context = spark + import context.implicits._ + new GeneralizedLinearRegression() + .setFamily("gaussian") + .fit(datasetGaussianIdentity.as[LabeledPoint]) + } + + test("glm summary: feature name") { + // dataset1 with no attribute + val dataset1 = Seq( + Instance(2.0, 1.0, Vectors.dense(0.0, 5.0)), + Instance(8.0, 2.0, Vectors.dense(1.0, 7.0)), + Instance(3.0, 3.0, Vectors.dense(2.0, 11.0)), + Instance(9.0, 4.0, Vectors.dense(3.0, 13.0)), + Instance(2.0, 5.0, Vectors.dense(2.0, 3.0)) + ).toDF() + + // dataset2 with attribute + val datasetTmp = Seq( + (2.0, 1.0, 0.0, 5.0), + (8.0, 2.0, 1.0, 7.0), + (3.0, 3.0, 2.0, 11.0), + (9.0, 4.0, 3.0, 13.0), + (2.0, 5.0, 2.0, 3.0) + ).toDF("y", "w", "x1", "x2") + val formula = new RFormula().setFormula("y ~ x1 + x2") + val dataset2 = formula.fit(datasetTmp).transform(datasetTmp) + + val expectedFeature = Seq(Array("features_0", "features_1"), Array("x1", "x2")) + + var idx = 0 + for (dataset <- Seq(dataset1, dataset2)) { + val model = new GeneralizedLinearRegression().fit(dataset) + model.summary.featureNames.zip(expectedFeature(idx)) + .foreach{ x => assert(x._1 === x._2) } + idx += 1 + } + } + + test("glm summary: coefficient with statistics") { + /* + R code: + + A <- matrix(c(0, 1, 2, 3, 2, 5, 7, 11, 13, 3), 5, 2) + b <- c(2, 8, 3, 9, 2) + df <- as.data.frame(cbind(A, b)) + model <- glm(formula = "b ~ .", data = df) + summary(model) + + Coefficients: + Estimate Std. Error t value Pr(>|t|) + (Intercept) 0.7903 4.0129 0.197 0.862 + V1 0.2258 2.1153 0.107 0.925 + V2 0.4677 0.5815 0.804 0.506 + */ + val dataset = Seq( + Instance(2.0, 1.0, Vectors.dense(0.0, 5.0)), + Instance(8.0, 2.0, Vectors.dense(1.0, 7.0)), + Instance(3.0, 3.0, Vectors.dense(2.0, 11.0)), + Instance(9.0, 4.0, Vectors.dense(3.0, 13.0)), + Instance(2.0, 5.0, Vectors.dense(2.0, 3.0)) + ).toDF() + + val expectedFeature = Seq(Array("features_0", "features_1"), + Array("(Intercept)", "features_0", "features_1")) + val expectedEstimate = Seq(Vectors.dense(0.2884, 0.538), + Vectors.dense(0.7903, 0.2258, 0.4677)) + val expectedStdError = Seq(Vectors.dense(1.724, 0.3787), + Vectors.dense(4.0129, 2.1153, 0.5815)) + + var idx = 0 + for (fitIntercept <- Seq(false, true)) { + val trainer = new GeneralizedLinearRegression() + .setFamily("gaussian") + .setFitIntercept(fitIntercept) + val model = trainer.fit(dataset) + val coefficientsWithStatistics = model.summary.coefficientsWithStatistics + + coefficientsWithStatistics.map(_._1).zip(expectedFeature(idx)).foreach { x => + assert(x._1 === x._2, "Feature name mismatch in coefficientsWithStatistics") } + assert(Vectors.dense(coefficientsWithStatistics.map(_._2)) ~= expectedEstimate(idx) + absTol 1E-3, "Coefficients mismatch in coefficientsWithStatistics") + assert(Vectors.dense(coefficientsWithStatistics.map(_._3)) ~= expectedStdError(idx) + absTol 1E-3, "Standard error mismatch in coefficientsWithStatistics") + idx += 1 + } + } + + test("generalized linear regression: regularization parameter") { + /* + R code: + + a1 <- c(0, 1, 2, 3) + a2 <- c(5, 2, 1, 3) + b <- c(1, 0, 1, 0) + data <- as.data.frame(cbind(a1, a2, b)) + df <- suppressWarnings(createDataFrame(data)) + + for (regParam in c(0.0, 0.1, 1.0)) { + model <- spark.glm(df, b ~ a1 + a2, regParam = regParam) + print(as.vector(summary(model)$aic)) + } + + [1] 12.88188 + [1] 12.92681 + [1] 13.32836 + */ + val dataset = Seq( + LabeledPoint(1, Vectors.dense(5, 0)), + LabeledPoint(0, Vectors.dense(2, 1)), + LabeledPoint(1, Vectors.dense(1, 2)), + LabeledPoint(0, Vectors.dense(3, 3)) + ).toDF() + val expected = Seq(12.88188, 12.92681, 13.32836) + + var idx = 0 + for (regParam <- Seq(0.0, 0.1, 1.0)) { + val trainer = new GeneralizedLinearRegression() + .setRegParam(regParam) + .setLabelCol("label") + .setFeaturesCol("features") + val model = trainer.fit(dataset) + val actual = model.summary.aic + assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with regParam = $regParam.") + idx += 1 + } + } + + test("evaluate with labels that are not doubles") { + // Evaulate with a dataset that contains Labels not as doubles to verify correct casting + val dataset = Seq( + Instance(17.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), + Instance(19.0, 1.0, Vectors.dense(1.0, 7.0)), + Instance(23.0, 1.0, Vectors.dense(2.0, 11.0)), + Instance(29.0, 1.0, Vectors.dense(3.0, 13.0)) + ).toDF() + + val trainer = new GeneralizedLinearRegression() + .setMaxIter(1) + val model = trainer.fit(dataset) + assert(model.hasSummary) + val summary = model.summary + + val longLabelDataset = dataset.select(col(model.getLabelCol).cast(FloatType), + col(model.getFeaturesCol)) + val evalSummary = model.evaluate(longLabelDataset) + // The calculations below involve pattern matching with Label as a double + assert(evalSummary.nullDeviance === summary.nullDeviance) + assert(evalSummary.deviance === summary.deviance) + assert(evalSummary.aic === summary.aic) + } +} + +object GeneralizedLinearRegressionSuite { + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "family" -> "poisson", + "link" -> "log", + "fitIntercept" -> true, + "maxIter" -> 2, // intentionally small + "tol" -> 0.8, + "regParam" -> 0.01, + "predictionCol" -> "myPrediction", + "variancePower" -> 1.0) + + def generateGeneralizedLinearRegressionInput( + intercept: Double, + coefficients: Array[Double], + xMean: Array[Double], + xVariance: Array[Double], + nPoints: Int, + seed: Int, + noiseLevel: Double, + family: String, + link: String): Seq[LabeledPoint] = { + + val rnd = new Random(seed) + def rndElement(i: Int) = { + (rnd.nextDouble() - 0.5) * math.sqrt(12.0 * xVariance(i)) + xMean(i) + } + val (generator, mean) = family match { + case "gaussian" => (new StandardNormalGenerator, 0.0) + case "poisson" => (new PoissonGenerator(1.0), 1.0) + case "gamma" => (new GammaGenerator(1.0, 1.0), 1.0) + } + generator.setSeed(seed) + + (0 until nPoints).map { _ => + val features = Vectors.dense(coefficients.indices.map(rndElement).toArray) + val eta = BLAS.dot(Vectors.dense(coefficients), features) + intercept + val mu = link match { + case "identity" => eta + case "log" => math.exp(eta) + case "sqrt" => math.pow(eta, 2.0) + case "inverse" => 1.0 / eta + } + val label = mu + noiseLevel * (generator.nextValue() - mean) + // Return LabeledPoints with DenseVector + LabeledPoint(label, features) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala index 59f4193abc8f..180f5f7ce5ab 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/IsotonicRegressionSuite.scala @@ -18,22 +18,24 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Row} -class IsotonicRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { +class IsotonicRegressionSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + private def generateIsotonicInput(labels: Seq[Double]): DataFrame = { - sqlContext.createDataFrame( - labels.zipWithIndex.map { case (label, i) => (label, i.toDouble, 1.0) } - ).toDF("label", "features", "weight") + labels.zipWithIndex.map { case (label, i) => (label, i.toDouble, 1.0) } + .toDF("label", "features", "weight") } private def generatePredictionInput(features: Seq[Double]): DataFrame = { - sqlContext.createDataFrame(features.map(Tuple1.apply)) - .toDF("features") + features.map(Tuple1.apply).toDF("features") } test("isotonic regression predictions") { @@ -44,7 +46,7 @@ class IsotonicRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { val predictions = model .transform(dataset) - .select("prediction").map { case Row(pred) => + .select("prediction").rdd.map { case Row(pred) => pred }.collect() @@ -64,7 +66,7 @@ class IsotonicRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { val predictions = model .transform(features) - .select("prediction").map { + .select("prediction").rdd.map { case Row(pred) => pred }.collect() @@ -91,8 +93,7 @@ class IsotonicRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { val model = ir.fit(dataset) - // copied model must have the same parent. - MLTestingUtils.checkCopy(model) + MLTestingUtils.checkCopyAndUids(ir, model) model.transform(dataset) .select("label", "features", "prediction", "weight") @@ -143,10 +144,10 @@ class IsotonicRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { } test("vector features column with feature index") { - val dataset = sqlContext.createDataFrame(Seq( + val dataset = Seq( (4.0, Vectors.dense(0.0, 1.0)), (3.0, Vectors.dense(0.0, 2.0)), - (5.0, Vectors.sparse(2, Array(1), Array(3.0)))) + (5.0, Vectors.sparse(2, Array(1), Array(3.0))) ).toDF("label", "features") val ir = new IsotonicRegression() @@ -158,10 +159,47 @@ class IsotonicRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { val predictions = model .transform(features) - .select("prediction").map { + .select("prediction").rdd.map { case Row(pred) => pred }.collect() assert(predictions === Array(3.5, 5.0, 5.0, 5.0)) } + + test("read/write") { + val dataset = generateIsotonicInput(Seq(1, 2, 3, 1, 6, 17, 16, 17, 18)) + + def checkModelData(model: IsotonicRegressionModel, model2: IsotonicRegressionModel): Unit = { + assert(model.boundaries === model2.boundaries) + assert(model.predictions === model2.predictions) + assert(model.isotonic === model2.isotonic) + } + + val ir = new IsotonicRegression() + testEstimatorAndModelReadWrite(ir, dataset, IsotonicRegressionSuite.allParamSettings, + IsotonicRegressionSuite.allParamSettings, checkModelData) + } + + test("should support all NumericType labels and weights, and not support other types") { + val ir = new IsotonicRegression() + MLTestingUtils.checkNumericTypes[IsotonicRegressionModel, IsotonicRegression]( + ir, spark, isClassification = false) { (expected, actual) => + assert(expected.boundaries === actual.boundaries) + assert(expected.predictions === actual.predictions) + } + } +} + +object IsotonicRegressionSuite { + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "predictionCol" -> "myPrediction", + "isotonic" -> true, + "featureIndex" -> 0 + ) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index a1d86fe8feda..f470dca7dbd0 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -21,59 +21,56 @@ import scala.util.Random import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.Instance -import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.{Vector, DenseVector, Vectors} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{DenseVector, Vector, Vectors} +import org.apache.spark.ml.param.{ParamMap, ParamsSuite} +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext} -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row} -class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { +class LinearRegressionSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ private val seed: Int = 42 @transient var datasetWithDenseFeature: DataFrame = _ + @transient var datasetWithStrongNoise: DataFrame = _ @transient var datasetWithDenseFeatureWithoutIntercept: DataFrame = _ @transient var datasetWithSparseFeature: DataFrame = _ @transient var datasetWithWeight: DataFrame = _ + @transient var datasetWithWeightConstantLabel: DataFrame = _ + @transient var datasetWithWeightZeroLabel: DataFrame = _ - /* - In `LinearRegressionSuite`, we will make sure that the model trained by SparkML - is the same as the one trained by R's glmnet package. The following instruction - describes how to reproduce the data in R. - - import org.apache.spark.mllib.util.LinearDataGenerator - val data = - sc.parallelize(LinearDataGenerator.generateLinearInput(6.3, Array(4.7, 7.2), - Array(0.9, -1.3), Array(0.7, 1.2), 10000, 42, 0.1), 2) - data.map(x=> x.label + ", " + x.features(0) + ", " + x.features(1)).coalesce(1) - .saveAsTextFile("path") - */ override def beforeAll(): Unit = { super.beforeAll() - datasetWithDenseFeature = sqlContext.createDataFrame( - sc.parallelize(LinearDataGenerator.generateLinearInput( - intercept = 6.3, weights = Array(4.7, 7.2), xMean = Array(0.9, -1.3), - xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2)) + datasetWithDenseFeature = sc.parallelize(LinearDataGenerator.generateLinearInput( + intercept = 6.3, weights = Array(4.7, 7.2), xMean = Array(0.9, -1.3), + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2).map(_.asML).toDF() + + datasetWithStrongNoise = sc.parallelize(LinearDataGenerator.generateLinearInput( + intercept = 6.3, weights = Array(4.7, 7.2), xMean = Array(0.9, -1.3), + xVariance = Array(0.7, 1.2), nPoints = 100, seed, eps = 5.0), 2).map(_.asML).toDF() + /* - datasetWithoutIntercept is not needed for correctness testing but is useful for illustrating - training model without intercept + datasetWithDenseFeatureWithoutIntercept is not needed for correctness testing + but is useful for illustrating training model without intercept */ - datasetWithDenseFeatureWithoutIntercept = sqlContext.createDataFrame( - sc.parallelize(LinearDataGenerator.generateLinearInput( + datasetWithDenseFeatureWithoutIntercept = sc.parallelize( + LinearDataGenerator.generateLinearInput( intercept = 0.0, weights = Array(4.7, 7.2), xMean = Array(0.9, -1.3), - xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2)) + xVariance = Array(0.7, 1.2), nPoints = 10000, seed, eps = 0.1), 2).map(_.asML).toDF() val r = new Random(seed) - // When feature size is larger than 4096, normal optimizer is choosed + // When feature size is larger than 4096, normal optimizer is chosen // as the solver of linear regression in the case of "auto" mode. val featureSize = 4100 - datasetWithSparseFeature = sqlContext.createDataFrame( - sc.parallelize(LinearDataGenerator.generateLinearInput( - intercept = 0.0, weights = Seq.fill(featureSize)(r.nextDouble).toArray, - xMean = Seq.fill(featureSize)(r.nextDouble).toArray, - xVariance = Seq.fill(featureSize)(r.nextDouble).toArray, nPoints = 200, - seed, eps = 0.1, sparsity = 0.7), 2)) + datasetWithSparseFeature = sc.parallelize(LinearDataGenerator.generateLinearInput( + intercept = 0.0, weights = Seq.fill(featureSize)(r.nextDouble()).toArray, + xMean = Seq.fill(featureSize)(r.nextDouble()).toArray, + xVariance = Seq.fill(featureSize)(r.nextDouble()).toArray, nPoints = 200, + seed, eps = 0.1, sparsity = 0.7), 2).map(_.asML).toDF() /* R code: @@ -83,13 +80,54 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { w <- c(1, 2, 3, 4) df <- as.data.frame(cbind(A, b)) */ - datasetWithWeight = sqlContext.createDataFrame( - sc.parallelize(Seq( - Instance(17.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), - Instance(19.0, 2.0, Vectors.dense(1.0, 7.0)), - Instance(23.0, 3.0, Vectors.dense(2.0, 11.0)), - Instance(29.0, 4.0, Vectors.dense(3.0, 13.0)) - ), 2)) + datasetWithWeight = sc.parallelize(Seq( + Instance(17.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), + Instance(19.0, 2.0, Vectors.dense(1.0, 7.0)), + Instance(23.0, 3.0, Vectors.dense(2.0, 11.0)), + Instance(29.0, 4.0, Vectors.dense(3.0, 13.0)) + ), 2).toDF() + + /* + R code: + + A <- matrix(c(0, 1, 2, 3, 5, 7, 11, 13), 4, 2) + b.const <- c(17, 17, 17, 17) + w <- c(1, 2, 3, 4) + df.const.label <- as.data.frame(cbind(A, b.const)) + */ + datasetWithWeightConstantLabel = sc.parallelize(Seq( + Instance(17.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), + Instance(17.0, 2.0, Vectors.dense(1.0, 7.0)), + Instance(17.0, 3.0, Vectors.dense(2.0, 11.0)), + Instance(17.0, 4.0, Vectors.dense(3.0, 13.0)) + ), 2).toDF() + + datasetWithWeightZeroLabel = sc.parallelize(Seq( + Instance(0.0, 1.0, Vectors.dense(0.0, 5.0).toSparse), + Instance(0.0, 2.0, Vectors.dense(1.0, 7.0)), + Instance(0.0, 3.0, Vectors.dense(2.0, 11.0)), + Instance(0.0, 4.0, Vectors.dense(3.0, 13.0)) + ), 2).toDF() + } + + /** + * Enable the ignored test to export the dataset into CSV format, + * so we can validate the training accuracy compared with R's glmnet package. + */ + ignore("export test data into CSV format") { + datasetWithDenseFeature.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile("target/tmp/LinearRegressionSuite/datasetWithDenseFeature") + + datasetWithDenseFeatureWithoutIntercept.rdd.map { + case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile( + "target/tmp/LinearRegressionSuite/datasetWithDenseFeatureWithoutIntercept") + + datasetWithSparseFeature.rdd.map { case Row(label: Double, features: Vector) => + label + "," + features.toArray.mkString(",") + }.repartition(1).saveAsTextFile("target/tmp/LinearRegressionSuite/datasetWithSparseFeature") } test("params") { @@ -110,8 +148,12 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { assert(lir.getSolver == "auto") val model = lir.fit(datasetWithDenseFeature) - // copied model must have the same parent. - MLTestingUtils.checkCopy(model) + MLTestingUtils.checkCopyAndUids(lir, model) + assert(model.hasSummary) + val copiedModel = model.copy(ParamMap.empty) + assert(copiedModel.hasSummary) + model.setSummary(None) + assert(!model.hasSummary) model.transform(datasetWithDenseFeature) .select("label", "prediction") @@ -124,6 +166,42 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { assert(model.numFeatures === numFeatures) } + test("linear regression handles singular matrices") { + // check for both constant columns with intercept (zero std) and collinear + val singularDataConstantColumn = sc.parallelize(Seq( + Instance(17.0, 1.0, Vectors.dense(1.0, 5.0).toSparse), + Instance(19.0, 2.0, Vectors.dense(1.0, 7.0)), + Instance(23.0, 3.0, Vectors.dense(1.0, 11.0)), + Instance(29.0, 4.0, Vectors.dense(1.0, 13.0)) + ), 2).toDF() + + Seq("auto", "l-bfgs", "normal").foreach { solver => + val trainer = new LinearRegression().setSolver(solver).setFitIntercept(true) + val model = trainer.fit(singularDataConstantColumn) + // to make it clear that WLS did not solve analytically + intercept[UnsupportedOperationException] { + model.summary.coefficientStandardErrors + } + assert(model.summary.objectiveHistory !== Array(0.0)) + } + + val singularDataCollinearFeatures = sc.parallelize(Seq( + Instance(17.0, 1.0, Vectors.dense(10.0, 5.0).toSparse), + Instance(19.0, 2.0, Vectors.dense(14.0, 7.0)), + Instance(23.0, 3.0, Vectors.dense(22.0, 11.0)), + Instance(29.0, 4.0, Vectors.dense(26.0, 13.0)) + ), 2).toDF() + + Seq("auto", "l-bfgs", "normal").foreach { solver => + val trainer = new LinearRegression().setSolver(solver).setFitIntercept(true) + val model = trainer.fit(singularDataCollinearFeatures) + intercept[UnsupportedOperationException] { + model.summary.coefficientStandardErrors + } + assert(model.summary.objectiveHistory !== Array(0.0)) + } + } + test("linear regression with intercept without regularization") { Seq("auto", "l-bfgs", "normal").foreach { solver => val trainer1 = new LinearRegression().setSolver(solver) @@ -183,31 +261,31 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { 3 x 1 sparse Matrix of class "dgCMatrix" s0 (Intercept) . - as.numeric.data.V2. 6.995908 - as.numeric.data.V3. 5.275131 + as.numeric.data.V2. 6.973403 + as.numeric.data.V3. 5.284370 */ - val coefficientsR = Vectors.dense(6.995908, 5.275131) + val coefficientsR = Vectors.dense(6.973403, 5.284370) - assert(model1.intercept ~== 0 absTol 1E-3) - assert(model1.coefficients ~= coefficientsR relTol 1E-3) - assert(model2.intercept ~== 0 absTol 1E-3) - assert(model2.coefficients ~= coefficientsR relTol 1E-3) + assert(model1.intercept ~== 0 absTol 1E-2) + assert(model1.coefficients ~= coefficientsR relTol 1E-2) + assert(model2.intercept ~== 0 absTol 1E-2) + assert(model2.coefficients ~= coefficientsR relTol 1E-2) /* Then again with the data with no intercept: - > coefficientsWithourIntercept + > coefficientsWithoutIntercept 3 x 1 sparse Matrix of class "dgCMatrix" s0 (Intercept) . as.numeric.data3.V2. 4.70011 as.numeric.data3.V3. 7.19943 */ - val coefficientsWithourInterceptR = Vectors.dense(4.70011, 7.19943) + val coefficientsWithoutInterceptR = Vectors.dense(4.70011, 7.19943) assert(modelWithoutIntercept1.intercept ~== 0 absTol 1E-3) - assert(modelWithoutIntercept1.coefficients ~= coefficientsWithourInterceptR relTol 1E-3) + assert(modelWithoutIntercept1.coefficients ~= coefficientsWithoutInterceptR relTol 1E-3) assert(modelWithoutIntercept2.intercept ~== 0 absTol 1E-3) - assert(modelWithoutIntercept2.coefficients ~= coefficientsWithourInterceptR relTol 1E-3) + assert(modelWithoutIntercept2.coefficients ~= coefficientsWithoutInterceptR relTol 1E-3) } } @@ -218,55 +296,47 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { val trainer2 = (new LinearRegression).setElasticNetParam(1.0).setRegParam(0.57) .setSolver(solver).setStandardization(false) - // Normal optimizer is not supported with only L1 regularization case. - if (solver == "normal") { - intercept[IllegalArgumentException] { - trainer1.fit(datasetWithDenseFeature) - trainer2.fit(datasetWithDenseFeature) - } - } else { - val model1 = trainer1.fit(datasetWithDenseFeature) - val model2 = trainer2.fit(datasetWithDenseFeature) - - /* - coefficients <- coef(glmnet(features, label, family="gaussian", - alpha = 1.0, lambda = 0.57 )) - > coefficients - 3 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 6.24300 - as.numeric.data.V2. 4.024821 - as.numeric.data.V3. 6.679841 - */ - val interceptR1 = 6.24300 - val coefficientsR1 = Vectors.dense(4.024821, 6.679841) - assert(model1.intercept ~== interceptR1 relTol 1E-3) - assert(model1.coefficients ~= coefficientsR1 relTol 1E-3) - - /* - coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 1.0, - lambda = 0.57, standardize=FALSE )) - > coefficients - 3 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 6.416948 - as.numeric.data.V2. 3.893869 - as.numeric.data.V3. 6.724286 - */ - val interceptR2 = 6.416948 - val coefficientsR2 = Vectors.dense(3.893869, 6.724286) - - assert(model2.intercept ~== interceptR2 relTol 1E-3) - assert(model2.coefficients ~= coefficientsR2 relTol 1E-3) - - model1.transform(datasetWithDenseFeature).select("features", "prediction") - .collect().foreach { - case Row(features: DenseVector, prediction1: Double) => - val prediction2 = - features(0) * model1.coefficients(0) + features(1) * model1.coefficients(1) + - model1.intercept - assert(prediction1 ~== prediction2 relTol 1E-5) - } + val model1 = trainer1.fit(datasetWithDenseFeature) + val model2 = trainer2.fit(datasetWithDenseFeature) + + /* + coefficients <- coef(glmnet(features, label, family="gaussian", + alpha = 1.0, lambda = 0.57 )) + > coefficients + 3 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 6.242284 + as.numeric.d1.V2. 4.019605 + as.numeric.d1.V3. 6.679538 + */ + val interceptR1 = 6.242284 + val coefficientsR1 = Vectors.dense(4.019605, 6.679538) + assert(model1.intercept ~== interceptR1 relTol 1E-2) + assert(model1.coefficients ~= coefficientsR1 relTol 1E-2) + + /* + coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 1.0, + lambda = 0.57, standardize=FALSE )) + > coefficients + 3 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 6.416948 + as.numeric.data.V2. 3.893869 + as.numeric.data.V3. 6.724286 + */ + val interceptR2 = 6.416948 + val coefficientsR2 = Vectors.dense(3.893869, 6.724286) + + assert(model2.intercept ~== interceptR2 relTol 1E-3) + assert(model2.coefficients ~= coefficientsR2 relTol 1E-3) + + model1.transform(datasetWithDenseFeature).select("features", "prediction") + .collect().foreach { + case Row(features: DenseVector, prediction1: Double) => + val prediction2 = + features(0) * model1.coefficients(0) + features(1) * model1.coefficients(1) + + model1.intercept + assert(prediction1 ~== prediction2 relTol 1E-5) } } } @@ -278,56 +348,48 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { val trainer2 = (new LinearRegression).setElasticNetParam(1.0).setRegParam(0.57) .setFitIntercept(false).setStandardization(false).setSolver(solver) - // Normal optimizer is not supported with only L1 regularization case. - if (solver == "normal") { - intercept[IllegalArgumentException] { - trainer1.fit(datasetWithDenseFeature) - trainer2.fit(datasetWithDenseFeature) - } - } else { - val model1 = trainer1.fit(datasetWithDenseFeature) - val model2 = trainer2.fit(datasetWithDenseFeature) - - /* - coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 1.0, - lambda = 0.57, intercept=FALSE )) - > coefficients - 3 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - as.numeric.data.V2. 6.299752 - as.numeric.data.V3. 4.772913 - */ - val interceptR1 = 0.0 - val coefficientsR1 = Vectors.dense(6.299752, 4.772913) - - assert(model1.intercept ~== interceptR1 absTol 1E-3) - assert(model1.coefficients ~= coefficientsR1 relTol 1E-3) - - /* - coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 1.0, - lambda = 0.57, intercept=FALSE, standardize=FALSE )) - > coefficients - 3 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - as.numeric.data.V2. 6.232193 - as.numeric.data.V3. 4.764229 - */ - val interceptR2 = 0.0 - val coefficientsR2 = Vectors.dense(6.232193, 4.764229) - - assert(model2.intercept ~== interceptR2 absTol 1E-3) - assert(model2.coefficients ~= coefficientsR2 relTol 1E-3) - - model1.transform(datasetWithDenseFeature).select("features", "prediction") - .collect().foreach { - case Row(features: DenseVector, prediction1: Double) => - val prediction2 = - features(0) * model1.coefficients(0) + features(1) * model1.coefficients(1) + - model1.intercept - assert(prediction1 ~== prediction2 relTol 1E-5) - } + val model1 = trainer1.fit(datasetWithDenseFeature) + val model2 = trainer2.fit(datasetWithDenseFeature) + + /* + coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 1.0, + lambda = 0.57, intercept=FALSE )) + > coefficients + 3 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + as.numeric.data.V2. 6.272927 + as.numeric.data.V3. 4.782604 + */ + val interceptR1 = 0.0 + val coefficientsR1 = Vectors.dense(6.272927, 4.782604) + + assert(model1.intercept ~== interceptR1 absTol 1E-2) + assert(model1.coefficients ~= coefficientsR1 relTol 1E-2) + + /* + coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 1.0, + lambda = 0.57, intercept=FALSE, standardize=FALSE )) + > coefficients + 3 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + as.numeric.data.V2. 6.207817 + as.numeric.data.V3. 4.775780 + */ + val interceptR2 = 0.0 + val coefficientsR2 = Vectors.dense(6.207817, 4.775780) + + assert(model2.intercept ~== interceptR2 absTol 1E-2) + assert(model2.coefficients ~= coefficientsR2 relTol 1E-2) + + model1.transform(datasetWithDenseFeature).select("features", "prediction") + .collect().foreach { + case Row(features: DenseVector, prediction1: Double) => + val prediction2 = + features(0) * model1.coefficients(0) + features(1) * model1.coefficients(1) + + model1.intercept + assert(prediction1 ~== prediction2 relTol 1E-5) } } } @@ -346,15 +408,15 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { > coefficients 3 x 1 sparse Matrix of class "dgCMatrix" s0 - (Intercept) 5.269376 - as.numeric.data.V2. 3.736216 - as.numeric.data.V3. 5.712356) + (Intercept) 5.260103 + as.numeric.d1.V2. 3.725522 + as.numeric.d1.V3. 5.711203 */ - val interceptR1 = 5.269376 - val coefficientsR1 = Vectors.dense(3.736216, 5.712356) + val interceptR1 = 5.260103 + val coefficientsR1 = Vectors.dense(3.725522, 5.711203) - assert(model1.intercept ~== interceptR1 relTol 1E-3) - assert(model1.coefficients ~= coefficientsR1 relTol 1E-3) + assert(model1.intercept ~== interceptR1 relTol 1E-2) + assert(model1.coefficients ~= coefficientsR1 relTol 1E-2) /* coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 0.0, lambda = 2.3, @@ -362,15 +424,15 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { > coefficients 3 x 1 sparse Matrix of class "dgCMatrix" s0 - (Intercept) 5.791109 - as.numeric.data.V2. 3.435466 - as.numeric.data.V3. 5.910406 + (Intercept) 5.790885 + as.numeric.d1.V2. 3.432373 + as.numeric.d1.V3. 5.919196 */ - val interceptR2 = 5.791109 - val coefficientsR2 = Vectors.dense(3.435466, 5.910406) + val interceptR2 = 5.790885 + val coefficientsR2 = Vectors.dense(3.432373, 5.919196) - assert(model2.intercept ~== interceptR2 relTol 1E-3) - assert(model2.coefficients ~= coefficientsR2 relTol 1E-3) + assert(model2.intercept ~== interceptR2 relTol 1E-2) + assert(model2.coefficients ~= coefficientsR2 relTol 1E-2) model1.transform(datasetWithDenseFeature).select("features", "prediction").collect().foreach { case Row(features: DenseVector, prediction1: Double) => @@ -397,15 +459,15 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { > coefficients 3 x 1 sparse Matrix of class "dgCMatrix" s0 - (Intercept) . - as.numeric.data.V2. 5.522875 - as.numeric.data.V3. 4.214502 + (Intercept) . + as.numeric.d1.V2. 5.493430 + as.numeric.d1.V3. 4.223082 */ val interceptR1 = 0.0 - val coefficientsR1 = Vectors.dense(5.522875, 4.214502) + val coefficientsR1 = Vectors.dense(5.493430, 4.223082) - assert(model1.intercept ~== interceptR1 absTol 1E-3) - assert(model1.coefficients ~= coefficientsR1 relTol 1E-3) + assert(model1.intercept ~== interceptR1 absTol 1E-2) + assert(model1.coefficients ~= coefficientsR1 relTol 1E-2) /* coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 0.0, lambda = 2.3, @@ -414,14 +476,14 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { 3 x 1 sparse Matrix of class "dgCMatrix" s0 (Intercept) . - as.numeric.data.V2. 5.263704 - as.numeric.data.V3. 4.187419 + as.numeric.d1.V2. 5.244324 + as.numeric.d1.V3. 4.203106 */ val interceptR2 = 0.0 - val coefficientsR2 = Vectors.dense(5.263704, 4.187419) + val coefficientsR2 = Vectors.dense(5.244324, 4.203106) - assert(model2.intercept ~== interceptR2 absTol 1E-3) - assert(model2.coefficients ~= coefficientsR2 relTol 1E-3) + assert(model2.intercept ~== interceptR2 absTol 1E-2) + assert(model2.coefficients ~= coefficientsR2 relTol 1E-2) model1.transform(datasetWithDenseFeature).select("features", "prediction").collect().foreach { case Row(features: DenseVector, prediction1: Double) => @@ -440,56 +502,48 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { val trainer2 = (new LinearRegression).setElasticNetParam(0.3).setRegParam(1.6) .setStandardization(false).setSolver(solver) - // Normal optimizer is not supported with non-zero elasticnet parameter. - if (solver == "normal") { - intercept[IllegalArgumentException] { - trainer1.fit(datasetWithDenseFeature) - trainer2.fit(datasetWithDenseFeature) - } - } else { - val model1 = trainer1.fit(datasetWithDenseFeature) - val model2 = trainer2.fit(datasetWithDenseFeature) - - /* - coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 0.3, - lambda = 1.6 )) - > coefficients - 3 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 6.324108 - as.numeric.data.V2. 3.168435 - as.numeric.data.V3. 5.200403 - */ - val interceptR1 = 5.696056 - val coefficientsR1 = Vectors.dense(3.670489, 6.001122) - - assert(model1.intercept ~== interceptR1 relTol 1E-3) - assert(model1.coefficients ~= coefficientsR1 relTol 1E-3) - - /* - coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 0.3, lambda = 1.6 - standardize=FALSE)) - > coefficients - 3 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) 6.114723 - as.numeric.data.V2. 3.409937 - as.numeric.data.V3. 6.146531 - */ - val interceptR2 = 6.114723 - val coefficientsR2 = Vectors.dense(3.409937, 6.146531) - - assert(model2.intercept ~== interceptR2 relTol 1E-3) - assert(model2.coefficients ~= coefficientsR2 relTol 1E-3) - - model1.transform(datasetWithDenseFeature).select("features", "prediction") - .collect().foreach { - case Row(features: DenseVector, prediction1: Double) => - val prediction2 = - features(0) * model1.coefficients(0) + features(1) * model1.coefficients(1) + - model1.intercept - assert(prediction1 ~== prediction2 relTol 1E-5) - } + val model1 = trainer1.fit(datasetWithDenseFeature) + val model2 = trainer2.fit(datasetWithDenseFeature) + + /* + coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 0.3, + lambda = 1.6 )) + > coefficients + 3 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 5.689855 + as.numeric.d1.V2. 3.661181 + as.numeric.d1.V3. 6.000274 + */ + val interceptR1 = 5.689855 + val coefficientsR1 = Vectors.dense(3.661181, 6.000274) + + assert(model1.intercept ~== interceptR1 relTol 1E-2) + assert(model1.coefficients ~= coefficientsR1 relTol 1E-2) + + /* + coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 0.3, lambda = 1.6 + standardize=FALSE)) + > coefficients + 3 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 6.113890 + as.numeric.d1.V2. 3.407021 + as.numeric.d1.V3. 6.152512 + */ + val interceptR2 = 6.113890 + val coefficientsR2 = Vectors.dense(3.407021, 6.152512) + + assert(model2.intercept ~== interceptR2 relTol 1E-2) + assert(model2.coefficients ~= coefficientsR2 relTol 1E-2) + + model1.transform(datasetWithDenseFeature).select("features", "prediction") + .collect().foreach { + case Row(features: DenseVector, prediction1: Double) => + val prediction2 = + features(0) * model1.coefficients(0) + features(1) * model1.coefficients(1) + + model1.intercept + assert(prediction1 ~== prediction2 relTol 1E-5) } } } @@ -501,63 +555,147 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { val trainer2 = (new LinearRegression).setElasticNetParam(0.3).setRegParam(1.6) .setFitIntercept(false).setStandardization(false).setSolver(solver) - // Normal optimizer is not supported with non-zero elasticnet parameter. - if (solver == "normal") { + val model1 = trainer1.fit(datasetWithDenseFeature) + val model2 = trainer2.fit(datasetWithDenseFeature) + + /* + coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 0.3, + lambda = 1.6, intercept=FALSE )) + > coefficients + 3 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + as.numeric.d1.V2. 5.643748 + as.numeric.d1.V3. 4.331519 + */ + val interceptR1 = 0.0 + val coefficientsR1 = Vectors.dense(5.643748, 4.331519) + + assert(model1.intercept ~== interceptR1 absTol 1E-2) + assert(model1.coefficients ~= coefficientsR1 relTol 1E-2) + + /* + coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 0.3, + lambda = 1.6, intercept=FALSE, standardize=FALSE )) + > coefficients + 3 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + as.numeric.d1.V2. 5.455902 + as.numeric.d1.V3. 4.312266 + + */ + val interceptR2 = 0.0 + val coefficientsR2 = Vectors.dense(5.455902, 4.312266) + + assert(model2.intercept ~== interceptR2 absTol 1E-2) + assert(model2.coefficients ~= coefficientsR2 relTol 1E-2) + + model1.transform(datasetWithDenseFeature).select("features", "prediction") + .collect().foreach { + case Row(features: DenseVector, prediction1: Double) => + val prediction2 = + features(0) * model1.coefficients(0) + features(1) * model1.coefficients(1) + + model1.intercept + assert(prediction1 ~== prediction2 relTol 1E-5) + } + } + } + + test("linear regression model with constant label") { + /* + R code: + for (formula in c(b.const ~ . -1, b.const ~ .)) { + model <- lm(formula, data=df.const.label, weights=w) + print(as.vector(coef(model))) + } + [1] -9.221298 3.394343 + [1] 17 0 0 + */ + val expected = Seq( + Vectors.dense(0.0, -9.221298, 3.394343), + Vectors.dense(17.0, 0.0, 0.0)) + + Seq("auto", "l-bfgs", "normal").foreach { solver => + var idx = 0 + for (fitIntercept <- Seq(false, true)) { + val model1 = new LinearRegression() + .setFitIntercept(fitIntercept) + .setWeightCol("weight") + .setPredictionCol("myPrediction") + .setSolver(solver) + .fit(datasetWithWeightConstantLabel) + val actual1 = Vectors.dense(model1.intercept, model1.coefficients(0), + model1.coefficients(1)) + assert(actual1 ~== expected(idx) absTol 1e-4) + + // Schema of summary.predictions should be a superset of the input dataset + assert((datasetWithWeightConstantLabel.schema.fieldNames.toSet + model1.getPredictionCol) + .subsetOf(model1.summary.predictions.schema.fieldNames.toSet)) + + val model2 = new LinearRegression() + .setFitIntercept(fitIntercept) + .setWeightCol("weight") + .setPredictionCol("myPrediction") + .setSolver(solver) + .fit(datasetWithWeightZeroLabel) + val actual2 = Vectors.dense(model2.intercept, model2.coefficients(0), + model2.coefficients(1)) + assert(actual2 ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1e-4) + + // Schema of summary.predictions should be a superset of the input dataset + assert((datasetWithWeightZeroLabel.schema.fieldNames.toSet + model2.getPredictionCol) + .subsetOf(model2.summary.predictions.schema.fieldNames.toSet)) + + idx += 1 + } + } + } + + test("regularized linear regression through origin with constant label") { + // The problem is ill-defined if fitIntercept=false, regParam is non-zero. + // An exception is thrown in this case. + Seq("auto", "l-bfgs", "normal").foreach { solver => + for (standardization <- Seq(false, true)) { + val model = new LinearRegression().setFitIntercept(false) + .setRegParam(0.1).setStandardization(standardization).setSolver(solver) intercept[IllegalArgumentException] { - trainer1.fit(datasetWithDenseFeature) - trainer2.fit(datasetWithDenseFeature) - } - } else { - val model1 = trainer1.fit(datasetWithDenseFeature) - val model2 = trainer2.fit(datasetWithDenseFeature) - - /* - coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 0.3, - lambda = 1.6, intercept=FALSE )) - > coefficients - 3 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - as.numeric.dataM.V2. 5.673348 - as.numeric.dataM.V3. 4.322251 - */ - val interceptR1 = 0.0 - val coefficientsR1 = Vectors.dense(5.673348, 4.322251) - - assert(model1.intercept ~== interceptR1 absTol 1E-3) - assert(model1.coefficients ~= coefficientsR1 relTol 1E-3) - - /* - coefficients <- coef(glmnet(features, label, family="gaussian", alpha = 0.3, - lambda = 1.6, intercept=FALSE, standardize=FALSE )) - > coefficients - 3 x 1 sparse Matrix of class "dgCMatrix" - s0 - (Intercept) . - as.numeric.data.V2. 5.477988 - as.numeric.data.V3. 4.297622 - */ - val interceptR2 = 0.0 - val coefficientsR2 = Vectors.dense(5.477988, 4.297622) - - assert(model2.intercept ~== interceptR2 absTol 1E-3) - assert(model2.coefficients ~= coefficientsR2 relTol 1E-3) - - model1.transform(datasetWithDenseFeature).select("features", "prediction") - .collect().foreach { - case Row(features: DenseVector, prediction1: Double) => - val prediction2 = - features(0) * model1.coefficients(0) + features(1) * model1.coefficients(1) + - model1.intercept - assert(prediction1 ~== prediction2 relTol 1E-5) + model.fit(datasetWithWeightConstantLabel) + } + } + } + } + + test("linear regression with l-bfgs when training is not needed") { + // When label is constant, l-bfgs solver returns results without training. + // There are two possibilities: If the label is non-zero but constant, + // and fitIntercept is true, then the model return yMean as intercept without training. + // If label is all zeros, then all coefficients are zero regardless of fitIntercept, so + // no training is needed. + for (fitIntercept <- Seq(false, true)) { + for (standardization <- Seq(false, true)) { + val model1 = new LinearRegression() + .setFitIntercept(fitIntercept) + .setStandardization(standardization) + .setWeightCol("weight") + .setSolver("l-bfgs") + .fit(datasetWithWeightConstantLabel) + if (fitIntercept) { + assert(model1.summary.objectiveHistory(0) ~== 0.0 absTol 1e-4) } + val model2 = new LinearRegression() + .setFitIntercept(fitIntercept) + .setWeightCol("weight") + .setSolver("l-bfgs") + .fit(datasetWithWeightZeroLabel) + assert(model2.summary.objectiveHistory(0) ~== 0.0 absTol 1e-4) } } } test("linear regression model training summary") { Seq("auto", "l-bfgs", "normal").foreach { solver => - val trainer = new LinearRegression().setSolver(solver) + val trainer = new LinearRegression().setSolver(solver).setPredictionCol("myPrediction") val model = trainer.fit(datasetWithDenseFeature) val trainerNoPredictionCol = trainer.setPredictionCol("") val modelNoPredictionCol = trainerNoPredictionCol.fit(datasetWithDenseFeature) @@ -567,47 +705,75 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { assert(modelNoPredictionCol.hasSummary) // Schema should be a superset of the input dataset - assert((datasetWithDenseFeature.schema.fieldNames.toSet + "prediction").subsetOf( + assert((datasetWithDenseFeature.schema.fieldNames.toSet + model.getPredictionCol).subsetOf( model.summary.predictions.schema.fieldNames.toSet)) // Validate that we re-insert a prediction column for evaluation val modelNoPredictionColFieldNames = modelNoPredictionCol.summary.predictions.schema.fieldNames - assert((datasetWithDenseFeature.schema.fieldNames.toSet).subsetOf( + assert(datasetWithDenseFeature.schema.fieldNames.toSet.subsetOf( modelNoPredictionColFieldNames.toSet)) assert(modelNoPredictionColFieldNames.exists(s => s.startsWith("prediction_"))) // Residuals in [[LinearRegressionResults]] should equal those manually computed - val expectedResiduals = datasetWithDenseFeature.select("features", "label") + datasetWithDenseFeature.select("features", "label") + .rdd .map { case Row(features: DenseVector, label: Double) => - val prediction = - features(0) * model.coefficients(0) + features(1) * model.coefficients(1) + - model.intercept - label - prediction - } - .zip(model.summary.residuals.map(_.getDouble(0))) + val prediction = + features(0) * model.coefficients(0) + features(1) * model.coefficients(1) + + model.intercept + label - prediction + } + .zip(model.summary.residuals.rdd.map(_.getDouble(0))) .collect() .foreach { case (manualResidual: Double, resultResidual: Double) => - assert(manualResidual ~== resultResidual relTol 1E-5) - } + assert(manualResidual ~== resultResidual relTol 1E-5) + } /* - Use the following R code to generate model training results. - - predictions <- predict(fit, newx=features) - residuals <- label - predictions - > mean(residuals^2) # MSE - [1] 0.009720325 - > mean(abs(residuals)) # MAD - [1] 0.07863206 - > cor(predictions, label)^2# r^2 - [,1] - s0 0.9998749 + # Use the following R code to generate model training results. + + # path/part-00000 is the file generated by running LinearDataGenerator.generateLinearInput + # as described before the beforeAll() method. + d1 <- read.csv("path/part-00000", header=FALSE, stringsAsFactors=FALSE) + fit <- glm(V1 ~ V2 + V3, data = d1, family = "gaussian") + names(f1)[1] = c("V2") + names(f1)[2] = c("V3") + f1 <- data.frame(as.numeric(d1$V2), as.numeric(d1$V3)) + predictions <- predict(fit, newdata=f1) + l1 <- as.numeric(d1$V1) + + residuals <- l1 - predictions + > mean(residuals^2) # MSE + [1] 0.00985449 + > mean(abs(residuals)) # MAD + [1] 0.07961668 + > cor(predictions, l1)^2 # r^2 + [1] 0.9998737 + + > summary(fit) + + Call: + glm(formula = V1 ~ V2 + V3, family = "gaussian", data = d1) + + Deviance Residuals: + Min 1Q Median 3Q Max + -0.47082 -0.06797 0.00002 0.06725 0.34635 + + Coefficients: + Estimate Std. Error t value Pr(>|t|) + (Intercept) 6.3022157 0.0018600 3388 <2e-16 *** + V2 4.6982442 0.0011805 3980 <2e-16 *** + V3 7.1994344 0.0009044 7961 <2e-16 *** + --- + + .... */ - assert(model.summary.meanSquaredError ~== 0.00972035 relTol 1E-5) - assert(model.summary.meanAbsoluteError ~== 0.07863206 relTol 1E-5) - assert(model.summary.r2 ~== 0.9998749 relTol 1E-5) + assert(model.summary.meanSquaredError ~== 0.00985449 relTol 1E-4) + assert(model.summary.meanAbsoluteError ~== 0.07961668 relTol 1E-4) + assert(model.summary.r2 ~== 0.9998737 relTol 1E-4) - // Normal solver uses "WeightedLeastSquares". This algorithm does not generate + // Normal solver uses "WeightedLeastSquares". If no regularization is applied or only L2 + // regularization is applied, this algorithm uses a direct solver and does not generate an // objective history because it does not run through iterations. if (solver == "l-bfgs") { // Objective function should be monotonically decreasing for linear regression @@ -617,17 +783,17 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { .sliding(2) .forall(x => x(0) >= x(1))) } else { - // To clalify that the normal solver is used here. + // To clarify that the normal solver is used here. assert(model.summary.objectiveHistory.length == 1) assert(model.summary.objectiveHistory(0) == 0.0) - val devianceResidualsR = Array(-0.35566, 0.34504) - val seCoefR = Array(0.0011756, 0.0009032, 0.0018489) - val tValsR = Array(3998, 7971, 3407) + val devianceResidualsR = Array(-0.47082, 0.34635) + val seCoefR = Array(0.0011805, 0.0009044, 0.0018600) + val tValsR = Array(3980, 7961, 3388) val pValsR = Array(0, 0, 0) model.summary.devianceResiduals.zip(devianceResidualsR).foreach { x => - assert(x._1 ~== x._2 absTol 1E-5) } - model.summary.coefficientStandardErrors.zip(seCoefR).foreach{ x => - assert(x._1 ~== x._2 absTol 1E-5) } + assert(x._1 ~== x._2 absTol 1E-4) } + model.summary.coefficientStandardErrors.zip(seCoefR).foreach { x => + assert(x._1 ~== x._2 absTol 1E-4) } model.summary.tValues.map(_.round).zip(tValsR).foreach{ x => assert(x._1 === x._2) } model.summary.pValues.map(_.round).zip(pValsR).foreach{ x => assert(x._1 === x._2) } } @@ -650,92 +816,35 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { } test("linear regression with weighted samples") { - Seq("auto", "l-bfgs", "normal").foreach { solver => - val (data, weightedData) = { - val activeData = LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 500, 1, 0.1) - - val rnd = new Random(8392) - val signedData = activeData.map { case p: LabeledPoint => - (rnd.nextGaussian() > 0.0, p) - } - - val data1 = signedData.flatMap { - case (true, p) => Iterator(p, p) - case (false, p) => Iterator(p) - } - - val weightedSignedData = signedData.flatMap { - case (true, LabeledPoint(label, features)) => - Iterator( - Instance(label, weight = 1.2, features), - Instance(label, weight = 0.8, features) - ) - case (false, LabeledPoint(label, features)) => - Iterator( - Instance(label, weight = 0.3, features), - Instance(label, weight = 0.1, features), - Instance(label, weight = 0.6, features) - ) - } - - val noiseData = LinearDataGenerator.generateLinearInput( - 2, Array(1, 3), Array(0.9, -1.3), Array(0.7, 1.2), 500, 1, 0.1) - val weightedNoiseData = noiseData.map { - case LabeledPoint(label, features) => Instance(label, weight = 0, features) - } - val data2 = weightedSignedData ++ weightedNoiseData - - (sqlContext.createDataFrame(sc.parallelize(data1, 4)), - sqlContext.createDataFrame(sc.parallelize(data2, 4))) - } - - val trainer1a = (new LinearRegression).setFitIntercept(true) - .setElasticNetParam(0.0).setRegParam(0.21).setStandardization(true).setSolver(solver) - val trainer1b = (new LinearRegression).setFitIntercept(true).setWeightCol("weight") - .setElasticNetParam(0.0).setRegParam(0.21).setStandardization(true).setSolver(solver) - - // Normal optimizer is not supported with non-zero elasticnet parameter. - val model1a0 = trainer1a.fit(data) - val model1a1 = trainer1a.fit(weightedData) - val model1b = trainer1b.fit(weightedData) - - assert(model1a0.coefficients !~= model1a1.coefficients absTol 1E-3) - assert(model1a0.intercept !~= model1a1.intercept absTol 1E-3) - assert(model1a0.coefficients ~== model1b.coefficients absTol 1E-3) - assert(model1a0.intercept ~== model1b.intercept absTol 1E-3) - - val trainer2a = (new LinearRegression).setFitIntercept(true) - .setElasticNetParam(0.0).setRegParam(0.21).setStandardization(false).setSolver(solver) - val trainer2b = (new LinearRegression).setFitIntercept(true).setWeightCol("weight") - .setElasticNetParam(0.0).setRegParam(0.21).setStandardization(false).setSolver(solver) - val model2a0 = trainer2a.fit(data) - val model2a1 = trainer2a.fit(weightedData) - val model2b = trainer2b.fit(weightedData) - assert(model2a0.coefficients !~= model2a1.coefficients absTol 1E-3) - assert(model2a0.intercept !~= model2a1.intercept absTol 1E-3) - assert(model2a0.coefficients ~== model2b.coefficients absTol 1E-3) - assert(model2a0.intercept ~== model2b.intercept absTol 1E-3) - - val trainer3a = (new LinearRegression).setFitIntercept(false) - .setElasticNetParam(0.0).setRegParam(0.21).setStandardization(true).setSolver(solver) - val trainer3b = (new LinearRegression).setFitIntercept(false).setWeightCol("weight") - .setElasticNetParam(0.0).setRegParam(0.21).setStandardization(true).setSolver(solver) - val model3a0 = trainer3a.fit(data) - val model3a1 = trainer3a.fit(weightedData) - val model3b = trainer3b.fit(weightedData) - assert(model3a0.coefficients !~= model3a1.coefficients absTol 1E-3) - assert(model3a0.coefficients ~== model3b.coefficients absTol 1E-3) - - val trainer4a = (new LinearRegression).setFitIntercept(false) - .setElasticNetParam(0.0).setRegParam(0.21).setStandardization(false).setSolver(solver) - val trainer4b = (new LinearRegression).setFitIntercept(false).setWeightCol("weight") - .setElasticNetParam(0.0).setRegParam(0.21).setStandardization(false).setSolver(solver) - val model4a0 = trainer4a.fit(data) - val model4a1 = trainer4a.fit(weightedData) - val model4b = trainer4b.fit(weightedData) - assert(model4a0.coefficients !~= model4a1.coefficients absTol 1E-3) - assert(model4a0.coefficients ~== model4b.coefficients absTol 1E-3) + val sqlContext = spark.sqlContext + import sqlContext.implicits._ + val numClasses = 0 + def modelEquals(m1: LinearRegressionModel, m2: LinearRegressionModel): Unit = { + assert(m1.coefficients ~== m2.coefficients relTol 0.01) + assert(m1.intercept ~== m2.intercept relTol 0.01) + } + val testParams = Seq( + // (elasticNetParam, regParam, fitIntercept, standardization) + (0.0, 0.21, true, true), + (0.0, 0.21, true, false), + (0.0, 0.21, false, false), + (1.0, 0.21, true, true) + ) + + for (solver <- Seq("auto", "l-bfgs", "normal"); + (elasticNetParam, regParam, fitIntercept, standardization) <- testParams) { + val estimator = new LinearRegression() + .setFitIntercept(fitIntercept) + .setStandardization(standardization) + .setRegParam(regParam) + .setElasticNetParam(elasticNetParam) + MLTestingUtils.testArbitrarilyScaledWeights[LinearRegressionModel, LinearRegression]( + datasetWithStrongNoise.as[LabeledPoint], estimator, modelEquals) + MLTestingUtils.testOutliersWithSmallWeights[LinearRegressionModel, LinearRegression]( + datasetWithStrongNoise.as[LabeledPoint], estimator, numClasses, modelEquals, + outlierRatio = 3) + MLTestingUtils.testOversamplingVsWeighting[LinearRegressionModel, LinearRegression]( + datasetWithStrongNoise.as[LabeledPoint], estimator, modelEquals, seed) } } @@ -801,6 +910,20 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { assert(x._1 ~== x._2 absTol 1E-3) } model.summary.tValues.zip(tValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } model.summary.pValues.zip(pValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } + + val modelWithL1 = new LinearRegression() + .setWeightCol("weight") + .setSolver("normal") + .setRegParam(0.5) + .setElasticNetParam(1.0) + .fit(datasetWithWeight) + + assert(modelWithL1.summary.objectiveHistory !== Array(0.0)) + assert( + modelWithL1.summary + .objectiveHistory + .sliding(2) + .forall(x => x(0) >= x(1))) } test("linear regression summary with weighted samples and w/o intercept by normal solver") { @@ -822,7 +945,7 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { V1 -3.7271 2.9032 -1.284 0.3279 V2 3.0100 0.6022 4.998 0.0378 * --- - Signif. codes: 0 ‘***’ 0.001 ‘**’ 0.01 ‘*’ 0.05 ‘.’ 0.1 ‘ ’ 1 + Signif. codes: 0 '***' 0.001 '**' 0.01 '*' 0.05 '.' 0.1 ' ' 1 (Dispersion parameter for gaussian family taken to be 17.4376) @@ -854,4 +977,44 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { model.summary.tValues.zip(tValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } model.summary.pValues.zip(pValsR).foreach{ x => assert(x._1 ~== x._2 absTol 1E-3) } } + + test("read/write") { + def checkModelData(model: LinearRegressionModel, model2: LinearRegressionModel): Unit = { + assert(model.intercept === model2.intercept) + assert(model.coefficients === model2.coefficients) + } + val lr = new LinearRegression() + testEstimatorAndModelReadWrite(lr, datasetWithWeight, LinearRegressionSuite.allParamSettings, + LinearRegressionSuite.allParamSettings, checkModelData) + } + + test("should support all NumericType labels and weights, and not support other types") { + for (solver <- Seq("auto", "l-bfgs", "normal")) { + val lr = new LinearRegression().setMaxIter(1).setSolver(solver) + MLTestingUtils.checkNumericTypes[LinearRegressionModel, LinearRegression]( + lr, spark, isClassification = false) { (expected, actual) => + assert(expected.intercept === actual.intercept) + assert(expected.coefficients === actual.coefficients) + } + } + } +} + +object LinearRegressionSuite { + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "predictionCol" -> "myPrediction", + "regParam" -> 0.01, + "elasticNetParam" -> 0.1, + "maxIter" -> 2, // intentionally small + "fitIntercept" -> true, + "tol" -> 0.8, + "standardization" -> false, + "solver" -> "l-bfgs" + ) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala index 7e751e4b553b..8b8e8a655f47 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/RandomForestRegressorSuite.scala @@ -18,10 +18,10 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.impl.TreeTests -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.tree.impl.TreeTests +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} +import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{EnsembleTestHelper, RandomForest => OldRandomForest} import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -31,7 +31,8 @@ import org.apache.spark.sql.DataFrame /** * Test suite for [[RandomForestRegressor]]. */ -class RandomForestRegressorSuite extends SparkFunSuite with MLlibTestSparkContext { +class RandomForestRegressorSuite extends SparkFunSuite with MLlibTestSparkContext + with DefaultReadWriteTest{ import RandomForestRegressorSuite.compareAPIs @@ -40,7 +41,8 @@ class RandomForestRegressorSuite extends SparkFunSuite with MLlibTestSparkContex override def beforeAll() { super.beforeAll() orderedLabeledPoints50_1000 = - sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000)) + sc.parallelize(EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) + .map(_.asML)) } ///////////////////////////////////////////////////////////////////////////// @@ -82,49 +84,51 @@ class RandomForestRegressorSuite extends SparkFunSuite with MLlibTestSparkContex .setSeed(123) // In this data, feature 1 is very important. - val data: RDD[LabeledPoint] = sc.parallelize(Seq( - new LabeledPoint(0, Vectors.dense(1, 0, 0, 0, 1)), - new LabeledPoint(1, Vectors.dense(1, 1, 0, 1, 0)), - new LabeledPoint(1, Vectors.dense(1, 1, 0, 0, 0)), - new LabeledPoint(0, Vectors.dense(1, 0, 0, 0, 0)), - new LabeledPoint(1, Vectors.dense(1, 1, 0, 0, 0)) - )) + val data: RDD[LabeledPoint] = TreeTests.featureImportanceData(sc) val categoricalFeatures = Map.empty[Int, Int] val df: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, 0) val model = rf.fit(df) - // copied model must have the same parent. - MLTestingUtils.checkCopy(model) + MLTestingUtils.checkCopyAndUids(rf, model) + val importances = model.featureImportances val mostImportantFeature = importances.argmax assert(mostImportantFeature === 1) + assert(importances.toArray.sum === 1.0) + assert(importances.toArray.forall(_ >= 0.0)) + } + + test("should support all NumericType labels and not support other types") { + val rf = new RandomForestRegressor().setMaxDepth(1) + MLTestingUtils.checkNumericTypes[RandomForestRegressionModel, RandomForestRegressor]( + rf, spark, isClassification = false) { (expected, actual) => + TreeTests.checkEqual(expected, actual) + } } ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// - // TODO: Reinstate test once save/load are implemented SPARK-6725 - /* - test("model save/load") { - val tempDir = Utils.createTempDir() - val path = tempDir.toURI.toString - - val trees = Range(0, 3).map(_ => OldDecisionTreeSuite.createModel(OldAlgo.Regression)).toArray - val oldModel = new OldRandomForestModel(OldAlgo.Regression, trees) - val newModel = RandomForestRegressionModel.fromOld(oldModel) - - // Save model, load it back, and compare. - try { - newModel.save(sc, path) - val sameNewModel = RandomForestRegressionModel.load(sc, path) - TreeTests.checkEqual(newModel, sameNewModel) - } finally { - Utils.deleteRecursively(tempDir) + test("read/write") { + def checkModelData( + model: RandomForestRegressionModel, + model2: RandomForestRegressionModel): Unit = { + TreeTests.checkEqual(model, model2) + assert(model.numFeatures === model2.numFeatures) } + + val rf = new RandomForestRegressor().setNumTrees(2) + val rdd = TreeTests.getTreeReadWriteData(sc) + + val allParamSettings = TreeTests.allParamSettings ++ Map("impurity" -> "variance") + + val continuousData: DataFrame = + TreeTests.setMetadata(rdd, Map.empty[Int, Int], numClasses = 0) + testEstimatorAndModelReadWrite(rf, continuousData, allParamSettings, + allParamSettings, checkModelData) } - */ } private object RandomForestRegressorSuite extends SparkFunSuite { @@ -140,8 +144,8 @@ private object RandomForestRegressorSuite extends SparkFunSuite { val numFeatures = data.first().features.size val oldStrategy = rf.getOldStrategy(categoricalFeatures, numClasses = 0, OldAlgo.Regression, rf.getOldImpurity) - val oldModel = OldRandomForest.trainRegressor( - data, oldStrategy, rf.getNumTrees, rf.getFeatureSubsetStrategy, rf.getSeed.toInt) + val oldModel = OldRandomForest.trainRegressor(data.map(OldLabeledPoint.fromML), oldStrategy, + rf.getNumTrees, rf.getFeatureSubsetStrategy, rf.getSeed.toInt) val newData: DataFrame = TreeTests.setMetadata(data, categoricalFeatures, numClasses = 0) val newModel = rf.fit(newData) // Use parent from newTree since this is not checked anyways. diff --git a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala index 997f574e51f6..3eabff434e8d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala @@ -17,41 +17,56 @@ package org.apache.spark.ml.source.libsvm -import java.io.File +import java.io.{File, IOException} +import java.nio.charset.StandardCharsets +import java.util.List -import com.google.common.base.Charsets import com.google.common.io.Files import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.ml.linalg.SQLDataTypes.VectorType import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{Row, SaveMode} +import org.apache.spark.sql.types.{DoubleType, StructField, StructType} import org.apache.spark.util.Utils + class LibSVMRelationSuite extends SparkFunSuite with MLlibTestSparkContext { - var tempDir: File = _ + // Path for dataset var path: String = _ override def beforeAll(): Unit = { super.beforeAll() - val lines = + val lines0 = """ |1 1:1.0 3:2.0 5:3.0 |0 + """.stripMargin + val lines1 = + """ |0 2:4.0 4:5.0 6:6.0 """.stripMargin - tempDir = Utils.createTempDir() - val file = new File(tempDir, "part-00000") - Files.write(lines, file, Charsets.US_ASCII) - path = tempDir.toURI.toString + val dir = Utils.createTempDir() + val succ = new File(dir, "_SUCCESS") + val file0 = new File(dir, "part-00000") + val file1 = new File(dir, "part-00001") + Files.write("", succ, StandardCharsets.UTF_8) + Files.write(lines0, file0, StandardCharsets.UTF_8) + Files.write(lines1, file1, StandardCharsets.UTF_8) + path = dir.getPath } override def afterAll(): Unit = { - Utils.deleteRecursively(tempDir) - super.afterAll() + try { + Utils.deleteRecursively(new File(path)) + } finally { + super.afterAll() + } } test("select as sparse vector") { - val df = sqlContext.read.format("libsvm").load(path) + val df = spark.read.format("libsvm").load(path) assert(df.columns(0) == "label") assert(df.columns(1) == "features") val row1 = df.first() @@ -61,7 +76,7 @@ class LibSVMRelationSuite extends SparkFunSuite with MLlibTestSparkContext { } test("select as dense vector") { - val df = sqlContext.read.format("libsvm").options(Map("vectorType" -> "dense")) + val df = spark.read.format("libsvm").options(Map("vectorType" -> "dense")) .load(path) assert(df.columns(0) == "label") assert(df.columns(1) == "features") @@ -72,11 +87,101 @@ class LibSVMRelationSuite extends SparkFunSuite with MLlibTestSparkContext { assert(v == Vectors.dense(1.0, 0.0, 2.0, 0.0, 3.0, 0.0)) } + test("illegal vector types") { + val e = intercept[IllegalArgumentException] { + spark.read.format("libsvm").options(Map("VectorType" -> "sparser")).load(path) + }.getMessage + assert(e.contains("Invalid value `sparser` for parameter `vectorType`. Expected " + + "types are `sparse` and `dense`.")) + } + test("select a vector with specifying the longer dimension") { - val df = sqlContext.read.option("numFeatures", "100").format("libsvm") + val df = spark.read.option("numFeatures", "100").format("libsvm") .load(path) val row1 = df.first() val v = row1.getAs[SparseVector](1) assert(v == Vectors.sparse(100, Seq((0, 1.0), (2, 2.0), (4, 3.0)))) } + + test("case insensitive option") { + val df = spark.read.option("NuMfEaTuReS", "100").format("libsvm").load(path) + assert(df.first().getAs[SparseVector](1) == + Vectors.sparse(100, Seq((0, 1.0), (2, 2.0), (4, 3.0)))) + } + + test("write libsvm data and read it again") { + val df = spark.read.format("libsvm").load(path) + val writePath = Utils.createTempDir().getPath + + // TODO: Remove requirement to coalesce by supporting multiple reads. + df.coalesce(1).write.format("libsvm").mode(SaveMode.Overwrite).save(writePath) + + val df2 = spark.read.format("libsvm").load(writePath) + val row1 = df2.first() + val v = row1.getAs[SparseVector](1) + assert(v == Vectors.sparse(6, Seq((0, 1.0), (2, 2.0), (4, 3.0)))) + } + + test("write libsvm data failed due to invalid schema") { + val df = spark.read.format("text").load(path) + intercept[IOException] { + df.write.format("libsvm").save(path + "_2") + } + } + + test("write libsvm data from scratch and read it again") { + val rawData = new java.util.ArrayList[Row]() + rawData.add(Row(1.0, Vectors.sparse(3, Seq((0, 2.0), (1, 3.0))))) + rawData.add(Row(4.0, Vectors.sparse(3, Seq((0, 5.0), (2, 6.0))))) + + val struct = StructType( + StructField("labelFoo", DoubleType, false) :: + StructField("featuresBar", VectorType, false) :: Nil + ) + val df = spark.sqlContext.createDataFrame(rawData, struct) + + val writePath = Utils.createTempDir().getPath + + df.coalesce(1).write.format("libsvm").mode(SaveMode.Overwrite).save(writePath) + + val df2 = spark.read.format("libsvm").load(writePath) + val row1 = df2.first() + val v = row1.getAs[SparseVector](1) + assert(v == Vectors.sparse(3, Seq((0, 2.0), (1, 3.0)))) + } + + test("select features from libsvm relation") { + val df = spark.read.format("libsvm").load(path) + df.select("features").rdd.map { case Row(d: Vector) => d }.first + df.select("features").collect + } + + test("create libsvmTable table without schema") { + try { + spark.sql( + s""" + |CREATE TABLE libsvmTable + |USING libsvm + |OPTIONS ( + | path '$path' + |) + """.stripMargin) + val df = spark.table("libsvmTable") + assert(df.columns(0) == "label") + assert(df.columns(1) == "features") + } finally { + spark.sql("DROP TABLE IF EXISTS libsvmTable") + } + } + + test("create libsvmTable table without schema and path") { + try { + val e = intercept[IllegalArgumentException] { + spark.sql("CREATE TABLE libsvmTable USING libsvm") + } + assert(e.getMessage.contains("No input path specified for libsvm data")) + } finally { + spark.sql("DROP TABLE IF EXISTS libsvmTable") + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/stat/ChiSquareTestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/stat/ChiSquareTestSuite.scala new file mode 100644 index 000000000000..2d6aad0808bc --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/stat/ChiSquareTestSuite.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.stat + +import java.util.Random + +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.stat.test.ChiSqTest +import org.apache.spark.mllib.util.MLlibTestSparkContext + +class ChiSquareTestSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + + test("test DataFrame of labeled points") { + // labels: 1.0 (2 / 6), 0.0 (4 / 6) + // feature1: 0.5 (1 / 6), 1.5 (2 / 6), 3.5 (3 / 6) + // feature2: 10.0 (1 / 6), 20.0 (1 / 6), 30.0 (2 / 6), 40.0 (2 / 6) + val data = Seq( + LabeledPoint(0.0, Vectors.dense(0.5, 10.0)), + LabeledPoint(0.0, Vectors.dense(1.5, 20.0)), + LabeledPoint(1.0, Vectors.dense(1.5, 30.0)), + LabeledPoint(0.0, Vectors.dense(3.5, 30.0)), + LabeledPoint(0.0, Vectors.dense(3.5, 40.0)), + LabeledPoint(1.0, Vectors.dense(3.5, 40.0))) + for (numParts <- List(2, 4, 6, 8)) { + val df = spark.createDataFrame(sc.parallelize(data, numParts)) + val chi = ChiSquareTest.test(df, "features", "label") + val (pValues: Vector, degreesOfFreedom: Array[Int], statistics: Vector) = + chi.select("pValues", "degreesOfFreedom", "statistics") + .as[(Vector, Array[Int], Vector)].head() + assert(pValues ~== Vectors.dense(0.6873, 0.6823) relTol 1e-4) + assert(degreesOfFreedom === Array(2, 3)) + assert(statistics ~== Vectors.dense(0.75, 1.5) relTol 1e-4) + } + } + + test("large number of features (SPARK-3087)") { + // Test that the right number of results is returned + val numCols = 1001 + val sparseData = Array( + LabeledPoint(0.0, Vectors.sparse(numCols, Seq((100, 2.0)))), + LabeledPoint(0.1, Vectors.sparse(numCols, Seq((200, 1.0))))) + val df = spark.createDataFrame(sparseData) + val chi = ChiSquareTest.test(df, "features", "label") + val (pValues: Vector, degreesOfFreedom: Array[Int], statistics: Vector) = + chi.select("pValues", "degreesOfFreedom", "statistics") + .as[(Vector, Array[Int], Vector)].head() + assert(pValues.size === numCols) + assert(degreesOfFreedom.length === numCols) + assert(statistics.size === numCols) + assert(pValues(1000) !== null) // SPARK-3087 + } + + test("fail on continuous features or labels") { + val tooManyCategories: Int = 100000 + assert(tooManyCategories > ChiSqTest.maxCategories, "This unit test requires that " + + "tooManyCategories be large enough to cause ChiSqTest to throw an exception.") + + val random = new Random(11L) + val continuousLabel = Seq.fill(tooManyCategories)( + LabeledPoint(random.nextDouble(), Vectors.dense(random.nextInt(2)))) + withClue("ChiSquare should throw an exception when given a continuous-valued label") { + intercept[SparkException] { + val df = spark.createDataFrame(continuousLabel) + ChiSquareTest.test(df, "features", "label") + } + } + val continuousFeature = Seq.fill(tooManyCategories)( + LabeledPoint(random.nextInt(2), Vectors.dense(random.nextDouble()))) + withClue("ChiSquare should throw an exception when given continuous-valued features") { + intercept[SparkException] { + val df = spark.createDataFrame(continuousFeature) + ChiSquareTest.test(df, "features", "label") + } + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/stat/CorrelationSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/stat/CorrelationSuite.scala new file mode 100644 index 000000000000..7d935e651f22 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/stat/CorrelationSuite.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.ml.stat + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.{Matrices, Matrix, Vectors} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{DataFrame, Row} + + +class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { + + val xData = Array(1.0, 0.0, -2.0) + val yData = Array(4.0, 5.0, 3.0) + val zeros = new Array[Double](3) + val data = Seq( + Vectors.dense(1.0, 0.0, 0.0, -2.0), + Vectors.dense(4.0, 5.0, 0.0, 3.0), + Vectors.dense(6.0, 7.0, 0.0, 8.0), + Vectors.dense(9.0, 0.0, 0.0, 1.0) + ) + + private def X = spark.createDataFrame(data.map(Tuple1.apply)).toDF("features") + + private def extract(df: DataFrame): BDM[Double] = { + val Array(Row(mat: Matrix)) = df.collect() + mat.asBreeze.toDenseMatrix + } + + + test("corr(X) default, pearson") { + val defaultMat = Correlation.corr(X, "features") + val pearsonMat = Correlation.corr(X, "features", "pearson") + // scalastyle:off + val expected = Matrices.fromBreeze(BDM( + (1.00000000, 0.05564149, Double.NaN, 0.4004714), + (0.05564149, 1.00000000, Double.NaN, 0.9135959), + (Double.NaN, Double.NaN, 1.00000000, Double.NaN), + (0.40047142, 0.91359586, Double.NaN, 1.0000000))) + // scalastyle:on + + assert(Matrices.fromBreeze(extract(defaultMat)) ~== expected absTol 1e-4) + assert(Matrices.fromBreeze(extract(pearsonMat)) ~== expected absTol 1e-4) + } + + test("corr(X) spearman") { + val spearmanMat = Correlation.corr(X, "features", "spearman") + // scalastyle:off + val expected = Matrices.fromBreeze(BDM( + (1.0000000, 0.1054093, Double.NaN, 0.4000000), + (0.1054093, 1.0000000, Double.NaN, 0.9486833), + (Double.NaN, Double.NaN, 1.00000000, Double.NaN), + (0.4000000, 0.9486833, Double.NaN, 1.0000000))) + // scalastyle:on + assert(Matrices.fromBreeze(extract(spearmanMat)) ~== expected absTol 1e-4) + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala new file mode 100644 index 000000000000..1ea851ef2d67 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/stat/SummarizerSuite.scala @@ -0,0 +1,600 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.stat + +import org.scalatest.exceptions.TestFailedException + +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} +import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, Statistics} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema + +class SummarizerSuite extends SparkFunSuite with MLlibTestSparkContext { + + import testImplicits._ + import Summarizer._ + import SummaryBuilderImpl._ + + private case class ExpectedMetrics( + mean: Seq[Double], + variance: Seq[Double], + count: Long, + numNonZeros: Seq[Long], + max: Seq[Double], + min: Seq[Double], + normL2: Seq[Double], + normL1: Seq[Double]) + + /** + * The input is expected to be either a sparse vector, a dense vector or an array of doubles + * (which will be converted to a dense vector) + * The expected is the list of all the known metrics. + * + * The tests take an list of input vectors and a list of all the summary values that + * are expected for this input. They currently test against some fixed subset of the + * metrics, but should be made fuzzy in the future. + */ + private def testExample(name: String, input: Seq[Any], exp: ExpectedMetrics): Unit = { + + def inputVec: Seq[Vector] = input.map { + case x: Array[Double @unchecked] => Vectors.dense(x) + case x: Seq[Double @unchecked] => Vectors.dense(x.toArray) + case x: Vector => x + case x => throw new Exception(x.toString) + } + + val summarizer = { + val _summarizer = new MultivariateOnlineSummarizer + inputVec.foreach(v => _summarizer.add(OldVectors.fromML(v))) + _summarizer + } + + // Because the Spark context is reset between tests, we cannot hold a reference onto it. + def wrappedInit() = { + val df = inputVec.map(Tuple1.apply).toDF("features") + val col = df.col("features") + (df, col) + } + + registerTest(s"$name - mean only") { + val (df, c) = wrappedInit() + compare(df.select(metrics("mean").summary(c), mean(c)), Seq(Row(exp.mean), summarizer.mean)) + } + + registerTest(s"$name - mean only (direct)") { + val (df, c) = wrappedInit() + compare(df.select(mean(c)), Seq(exp.mean)) + } + + registerTest(s"$name - variance only") { + val (df, c) = wrappedInit() + compare(df.select(metrics("variance").summary(c), variance(c)), + Seq(Row(exp.variance), summarizer.variance)) + } + + registerTest(s"$name - variance only (direct)") { + val (df, c) = wrappedInit() + compare(df.select(variance(c)), Seq(summarizer.variance)) + } + + registerTest(s"$name - count only") { + val (df, c) = wrappedInit() + compare(df.select(metrics("count").summary(c), count(c)), + Seq(Row(exp.count), exp.count)) + } + + registerTest(s"$name - count only (direct)") { + val (df, c) = wrappedInit() + compare(df.select(count(c)), + Seq(exp.count)) + } + + registerTest(s"$name - numNonZeros only") { + val (df, c) = wrappedInit() + compare(df.select(metrics("numNonZeros").summary(c), numNonZeros(c)), + Seq(Row(exp.numNonZeros), exp.numNonZeros)) + } + + registerTest(s"$name - numNonZeros only (direct)") { + val (df, c) = wrappedInit() + compare(df.select(numNonZeros(c)), + Seq(exp.numNonZeros)) + } + + registerTest(s"$name - min only") { + val (df, c) = wrappedInit() + compare(df.select(metrics("min").summary(c), min(c)), + Seq(Row(exp.min), exp.min)) + } + + registerTest(s"$name - max only") { + val (df, c) = wrappedInit() + compare(df.select(metrics("max").summary(c), max(c)), + Seq(Row(exp.max), exp.max)) + } + + registerTest(s"$name - normL1 only") { + val (df, c) = wrappedInit() + compare(df.select(metrics("normL1").summary(c), normL1(c)), + Seq(Row(exp.normL1), exp.normL1)) + } + + registerTest(s"$name - normL2 only") { + val (df, c) = wrappedInit() + compare(df.select(metrics("normL2").summary(c), normL2(c)), + Seq(Row(exp.normL2), exp.normL2)) + } + + registerTest(s"$name - all metrics at once") { + val (df, c) = wrappedInit() + compare(df.select( + metrics("mean", "variance", "count", "numNonZeros").summary(c), + mean(c), variance(c), count(c), numNonZeros(c)), + Seq(Row(exp.mean, exp.variance, exp.count, exp.numNonZeros), + exp.mean, exp.variance, exp.count, exp.numNonZeros)) + } + } + + private def denseData(input: Seq[Seq[Double]]): DataFrame = { + input.map(_.toArray).map(Vectors.dense).map(Tuple1.apply).toDF("features") + } + + private def compare(df: DataFrame, exp: Seq[Any]): Unit = { + val coll = df.collect().toSeq + val Seq(row) = coll + val res = row.toSeq + val names = df.schema.fieldNames.zipWithIndex.map { case (n, idx) => s"$n ($idx)" } + assert(res.size === exp.size, (res.size, exp.size)) + for (((x1, x2), name) <- res.zip(exp).zip(names)) { + compareStructures(x1, x2, name) + } + } + + // Compares structured content. + private def compareStructures(x1: Any, x2: Any, name: String): Unit = (x1, x2) match { + case (y1: Seq[Double @unchecked], v1: OldVector) => + compareStructures(y1, v1.toArray.toSeq, name) + case (d1: Double, d2: Double) => + assert2(Vectors.dense(d1) ~== Vectors.dense(d2) absTol 1e-4, name) + case (r1: GenericRowWithSchema, r2: Row) => + assert(r1.size === r2.size, (r1, r2)) + for (((fname, x1), x2) <- r1.schema.fieldNames.zip(r1.toSeq).zip(r2.toSeq)) { + compareStructures(x1, x2, s"$name.$fname") + } + case (r1: Row, r2: Row) => + assert(r1.size === r2.size, (r1, r2)) + for ((x1, x2) <- r1.toSeq.zip(r2.toSeq)) { compareStructures(x1, x2, name) } + case (v1: Vector, v2: Vector) => + assert2(v1 ~== v2 absTol 1e-4, name) + case (l1: Long, l2: Long) => assert(l1 === l2) + case (s1: Seq[_], s2: Seq[_]) => + assert(s1.size === s2.size, s"$name ${(s1, s2)}") + for (((x1, idx), x2) <- s1.zipWithIndex.zip(s2)) { + compareStructures(x1, x2, s"$name.$idx") + } + case (arr1: Array[_], arr2: Array[_]) => + assert(arr1.toSeq === arr2.toSeq) + case _ => throw new Exception(s"$name: ${x1.getClass} ${x2.getClass} $x1 $x2") + } + + private def assert2(x: => Boolean, hint: String): Unit = { + try { + assert(x, hint) + } catch { + case tfe: TestFailedException => + throw new TestFailedException(Some(s"Failure with hint $hint"), Some(tfe), 1) + } + } + + test("debugging test") { + val df = denseData(Nil) + val c = df.col("features") + val c1 = metrics("mean").summary(c) + val res = df.select(c1) + intercept[SparkException] { + compare(res, Seq.empty) + } + } + + test("basic error handling") { + val df = denseData(Nil) + val c = df.col("features") + val res = df.select(metrics("mean").summary(c), mean(c)) + intercept[SparkException] { + compare(res, Seq.empty) + } + } + + test("no element, working metrics") { + val df = denseData(Nil) + val c = df.col("features") + val res = df.select(metrics("count").summary(c), count(c)) + compare(res, Seq(Row(0L), 0L)) + } + + val singleElem = Seq(0.0, 1.0, 2.0) + testExample("single element", Seq(singleElem), ExpectedMetrics( + mean = singleElem, + variance = Seq(0.0, 0.0, 0.0), + count = 1, + numNonZeros = Seq(0, 1, 1), + max = singleElem, + min = singleElem, + normL1 = singleElem, + normL2 = singleElem + )) + + testExample("two elements", Seq(Seq(0.0, 1.0, 2.0), Seq(0.0, -1.0, -2.0)), ExpectedMetrics( + mean = Seq(0.0, 0.0, 0.0), + // TODO: I have a doubt about these values, they are not normalized. + variance = Seq(0.0, 2.0, 8.0), + count = 2, + numNonZeros = Seq(0, 2, 2), + max = Seq(0.0, 1.0, 2.0), + min = Seq(0.0, -1.0, -2.0), + normL1 = Seq(0.0, 2.0, 4.0), + normL2 = Seq(0.0, math.sqrt(2.0), math.sqrt(2.0) * 2.0) + )) + + testExample("dense vector input", + Seq(Seq(-1.0, 0.0, 6.0), Seq(3.0, -3.0, 0.0)), + ExpectedMetrics( + mean = Seq(1.0, -1.5, 3.0), + variance = Seq(8.0, 4.5, 18.0), + count = 2, + numNonZeros = Seq(2, 1, 1), + max = Seq(3.0, 0.0, 6.0), + min = Seq(-1.0, -3, 0.0), + normL1 = Seq(4.0, 3.0, 6.0), + normL2 = Seq(math.sqrt(10), 3, 6.0) + ) + ) + + test("summarizer buffer basic error handing") { + val summarizer = new SummarizerBuffer + + assert(summarizer.count === 0, "should be zero since nothing is added.") + + withClue("Getting numNonzeros from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.numNonzeros + } + } + + withClue("Getting variance from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.variance + } + } + + withClue("Getting mean from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.mean + } + } + + withClue("Getting max from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.max + } + } + + withClue("Getting min from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.min + } + } + + summarizer.add(Vectors.dense(-1.0, 2.0, 6.0)).add(Vectors.sparse(3, Seq((0, -2.0), (1, 6.0)))) + + withClue("Adding a new dense sample with different array size should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.add(Vectors.dense(3.0, 1.0)) + } + } + + withClue("Adding a new sparse sample with different array size should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.add(Vectors.sparse(5, Seq((0, -2.0), (1, 6.0)))) + } + } + + val summarizer2 = (new SummarizerBuffer).add(Vectors.dense(1.0, -2.0, 0.0, 4.0)) + withClue("Merging a new summarizer with different dimensions should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.merge(summarizer2) + } + } + } + + test("summarizer buffer dense vector input") { + // For column 2, the maximum will be 0.0, and it's not explicitly added since we ignore all + // the zeros; it's a case we need to test. For column 3, the minimum will be 0.0 which we + // need to test as well. + val summarizer = (new SummarizerBuffer) + .add(Vectors.dense(-1.0, 0.0, 6.0)) + .add(Vectors.dense(3.0, -3.0, 0.0)) + + assert(summarizer.mean ~== Vectors.dense(1.0, -1.5, 3.0) absTol 1E-5, "mean mismatch") + assert(summarizer.min ~== Vectors.dense(-1.0, -3, 0.0) absTol 1E-5, "min mismatch") + assert(summarizer.max ~== Vectors.dense(3.0, 0.0, 6.0) absTol 1E-5, "max mismatch") + assert(summarizer.numNonzeros ~== Vectors.dense(2, 1, 1) absTol 1E-5, "numNonzeros mismatch") + assert(summarizer.variance ~== Vectors.dense(8.0, 4.5, 18.0) absTol 1E-5, "variance mismatch") + assert(summarizer.count === 2) + } + + test("summarizer buffer sparse vector input") { + val summarizer = (new SummarizerBuffer) + .add(Vectors.sparse(3, Seq((0, -1.0), (2, 6.0)))) + .add(Vectors.sparse(3, Seq((0, 3.0), (1, -3.0)))) + + assert(summarizer.mean ~== Vectors.dense(1.0, -1.5, 3.0) absTol 1E-5, "mean mismatch") + assert(summarizer.min ~== Vectors.dense(-1.0, -3, 0.0) absTol 1E-5, "min mismatch") + assert(summarizer.max ~== Vectors.dense(3.0, 0.0, 6.0) absTol 1E-5, "max mismatch") + assert(summarizer.numNonzeros ~== Vectors.dense(2, 1, 1) absTol 1E-5, "numNonzeros mismatch") + assert(summarizer.variance ~== Vectors.dense(8.0, 4.5, 18.0) absTol 1E-5, "variance mismatch") + assert(summarizer.count === 2) + } + + test("summarizer buffer mixing dense and sparse vector input") { + val summarizer = (new SummarizerBuffer) + .add(Vectors.sparse(3, Seq((0, -2.0), (1, 2.3)))) + .add(Vectors.dense(0.0, -1.0, -3.0)) + .add(Vectors.sparse(3, Seq((1, -5.1)))) + .add(Vectors.dense(3.8, 0.0, 1.9)) + .add(Vectors.dense(1.7, -0.6, 0.0)) + .add(Vectors.sparse(3, Seq((1, 1.9), (2, 0.0)))) + + assert(summarizer.mean ~== + Vectors.dense(0.583333333333, -0.416666666666, -0.183333333333) absTol 1E-5, "mean mismatch") + + assert(summarizer.min ~== Vectors.dense(-2.0, -5.1, -3) absTol 1E-5, "min mismatch") + assert(summarizer.max ~== Vectors.dense(3.8, 2.3, 1.9) absTol 1E-5, "max mismatch") + assert(summarizer.numNonzeros ~== Vectors.dense(3, 5, 2) absTol 1E-5, "numNonzeros mismatch") + assert(summarizer.variance ~== + Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, + "variance mismatch") + + assert(summarizer.count === 6) + } + + test("summarizer buffer merging two summarizers") { + val summarizer1 = (new SummarizerBuffer) + .add(Vectors.sparse(3, Seq((0, -2.0), (1, 2.3)))) + .add(Vectors.dense(0.0, -1.0, -3.0)) + + val summarizer2 = (new SummarizerBuffer) + .add(Vectors.sparse(3, Seq((1, -5.1)))) + .add(Vectors.dense(3.8, 0.0, 1.9)) + .add(Vectors.dense(1.7, -0.6, 0.0)) + .add(Vectors.sparse(3, Seq((1, 1.9), (2, 0.0)))) + + val summarizer = summarizer1.merge(summarizer2) + + assert(summarizer.mean ~== + Vectors.dense(0.583333333333, -0.416666666666, -0.183333333333) absTol 1E-5, "mean mismatch") + + assert(summarizer.min ~== Vectors.dense(-2.0, -5.1, -3) absTol 1E-5, "min mismatch") + assert(summarizer.max ~== Vectors.dense(3.8, 2.3, 1.9) absTol 1E-5, "max mismatch") + assert(summarizer.numNonzeros ~== Vectors.dense(3, 5, 2) absTol 1E-5, "numNonzeros mismatch") + assert(summarizer.variance ~== + Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, + "variance mismatch") + assert(summarizer.count === 6) + } + + test("summarizer buffer zero variance test (SPARK-21818)") { + val summarizer1 = new SummarizerBuffer() + .add(Vectors.dense(3.0), 0.7) + val summarizer2 = new SummarizerBuffer() + .add(Vectors.dense(3.0), 0.4) + val summarizer3 = new SummarizerBuffer() + .add(Vectors.dense(3.0), 0.5) + val summarizer4 = new SummarizerBuffer() + .add(Vectors.dense(3.0), 0.4) + + val summarizer = summarizer1 + .merge(summarizer2) + .merge(summarizer3) + .merge(summarizer4) + + assert(summarizer.variance(0) >= 0.0) + } + + test("summarizer buffer merging summarizer with empty summarizer") { + // If one of two is non-empty, this should return the non-empty summarizer. + // If both of them are empty, then just return the empty summarizer. + val summarizer1 = (new SummarizerBuffer) + .add(Vectors.dense(0.0, -1.0, -3.0)).merge(new SummarizerBuffer) + assert(summarizer1.count === 1) + + val summarizer2 = (new SummarizerBuffer) + .merge((new SummarizerBuffer).add(Vectors.dense(0.0, -1.0, -3.0))) + assert(summarizer2.count === 1) + + val summarizer3 = (new SummarizerBuffer).merge(new SummarizerBuffer) + assert(summarizer3.count === 0) + + assert(summarizer1.mean ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "mean mismatch") + assert(summarizer2.mean ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "mean mismatch") + assert(summarizer1.min ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "min mismatch") + assert(summarizer2.min ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "min mismatch") + assert(summarizer1.max ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "max mismatch") + assert(summarizer2.max ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "max mismatch") + assert(summarizer1.numNonzeros ~== Vectors.dense(0, 1, 1) absTol 1E-5, "numNonzeros mismatch") + assert(summarizer2.numNonzeros ~== Vectors.dense(0, 1, 1) absTol 1E-5, "numNonzeros mismatch") + assert(summarizer1.variance ~== Vectors.dense(0, 0, 0) absTol 1E-5, "variance mismatch") + assert(summarizer2.variance ~== Vectors.dense(0, 0, 0) absTol 1E-5, "variance mismatch") + } + + test("summarizer buffer merging summarizer when one side has zero mean (SPARK-4355)") { + val s0 = new SummarizerBuffer() + .add(Vectors.dense(2.0)) + .add(Vectors.dense(2.0)) + val s1 = new SummarizerBuffer() + .add(Vectors.dense(1.0)) + .add(Vectors.dense(-1.0)) + s0.merge(s1) + assert(s0.mean(0) ~== 1.0 absTol 1e-14) + } + + test("summarizer buffer merging summarizer with weighted samples") { + val summarizer = (new SummarizerBuffer) + .add(Vectors.sparse(3, Seq((0, -0.8), (1, 1.7))), weight = 0.1) + .add(Vectors.dense(0.0, -1.2, -1.7), 0.2).merge( + (new SummarizerBuffer) + .add(Vectors.sparse(3, Seq((0, -0.7), (1, 0.01), (2, 1.3))), 0.15) + .add(Vectors.dense(-0.5, 0.3, -1.5), 0.05)) + + assert(summarizer.count === 4) + + // The following values are hand calculated using the formula: + // [[https://en.wikipedia.org/wiki/Weighted_arithmetic_mean#Reliability_weights]] + // which defines the reliability weight used for computing the unbiased estimation of variance + // for weighted instances. + assert(summarizer.mean ~== Vectors.dense(Array(-0.42, -0.107, -0.44)) + absTol 1E-10, "mean mismatch") + assert(summarizer.variance ~== Vectors.dense(Array(0.17657142857, 1.645115714, 2.42057142857)) + absTol 1E-8, "variance mismatch") + assert(summarizer.numNonzeros ~== Vectors.dense(Array(3.0, 4.0, 3.0)) + absTol 1E-10, "numNonzeros mismatch") + assert(summarizer.max ~== Vectors.dense(Array(0.0, 1.7, 1.3)) absTol 1E-10, "max mismatch") + assert(summarizer.min ~== Vectors.dense(Array(-0.8, -1.2, -1.7)) absTol 1E-10, "min mismatch") + assert(summarizer.normL2 ~== Vectors.dense(0.387298335, 0.762571308141, 0.9715966241192) + absTol 1E-8, "normL2 mismatch") + assert(summarizer.normL1 ~== Vectors.dense(0.21, 0.4265, 0.61) absTol 1E-10, "normL1 mismatch") + } + + test("summarizer buffer test min/max with weighted samples") { + val summarizer1 = new SummarizerBuffer() + .add(Vectors.dense(10.0, -10.0), 1e10) + .add(Vectors.dense(0.0, 0.0), 1e-7) + + val summarizer2 = new SummarizerBuffer() + summarizer2.add(Vectors.dense(10.0, -10.0), 1e10) + for (i <- 1 to 100) { + summarizer2.add(Vectors.dense(0.0, 0.0), 1e-7) + } + + val summarizer3 = new SummarizerBuffer() + for (i <- 1 to 100) { + summarizer3.add(Vectors.dense(0.0, 0.0), 1e-7) + } + summarizer3.add(Vectors.dense(10.0, -10.0), 1e10) + + assert(summarizer1.max ~== Vectors.dense(10.0, 0.0) absTol 1e-14) + assert(summarizer1.min ~== Vectors.dense(0.0, -10.0) absTol 1e-14) + assert(summarizer2.max ~== Vectors.dense(10.0, 0.0) absTol 1e-14) + assert(summarizer2.min ~== Vectors.dense(0.0, -10.0) absTol 1e-14) + assert(summarizer3.max ~== Vectors.dense(10.0, 0.0) absTol 1e-14) + assert(summarizer3.min ~== Vectors.dense(0.0, -10.0) absTol 1e-14) + } + + ignore("performance test") { + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.12 + MacBook Pro (15-inch, 2016) CPU 2.9 GHz Intel Core i7 + Use 2 partitions. tries out times= 20, warm up times = 10 + + The unit of test results is records/milliseconds (higher is better) + + Vector size/records number: 1/1E7 10/1E6 100/1E6 1E3/1E5 1E4/1E4 + ----------------------------------------------------------------------------- + DataFrame 15149 7441 2118 224 21 + RDD from DataFrame 4992 4440 2328 320 33 + Raw RDD 53931 20683 3966 528 53 + */ + import scala.util.Random + val rand = new Random() + + val genArr = (dim: Int) => { + Array.fill(dim)(rand.nextDouble()) + } + + val numPartitions = 2 + for ( (n, dim) <- Seq( + (10000000, 1), (1000000, 10), (1000000, 100), (100000, 1000), (10000, 10000)) + ) { + val rdd1 = sc.parallelize(1 to n, numPartitions).map { idx => + OldVectors.dense(genArr(dim)) + } + // scalastyle:off println + println(s"records number = $n, vector size = $dim, partition = ${rdd1.getNumPartitions}") + // scalastyle:on println + + val numOfTry = 20 + val numOfWarmUp = 10 + rdd1.cache() + rdd1.count() + val rdd2 = sc.parallelize(1 to n, numPartitions).map { idx => + Vectors.dense(genArr(dim)) + } + rdd2.cache() + rdd2.count() + val df = rdd2.map(Tuple1.apply).toDF("features") + df.cache() + df.count() + + def print(name: String, l: List[Long]): Unit = { + def f(z: Long) = (1e6 * n.toDouble) / z + val min = f(l.max) + val max = f(l.min) + val med = f(l.sorted.drop(l.size / 2).head) + // scalastyle:off println + println(s"$name = [$min ~ $med ~ $max] records / milli") + // scalastyle:on println + } + + var timeDF: List[Long] = Nil + val x = df.select( + metrics("mean", "variance", "count", "numNonZeros", "max", "min", "normL1", + "normL2").summary($"features")) + for (i <- 1 to numOfTry) { + val start = System.nanoTime() + x.head() + val end = System.nanoTime() + if (i > numOfWarmUp) timeDF ::= (end - start) + } + + var timeRDD: List[Long] = Nil + for (i <- 1 to numOfTry) { + val start = System.nanoTime() + Statistics.colStats(rdd1) + val end = System.nanoTime() + if (i > numOfWarmUp) timeRDD ::= (end - start) + } + + var timeRDDFromDF: List[Long] = Nil + val rddFromDf = df.rdd.map { case Row(v: Vector) => OldVectors.fromML(v) } + for (i <- 1 to numOfTry) { + val start = System.nanoTime() + Statistics.colStats(rddFromDf) + val end = System.nanoTime() + if (i > numOfWarmUp) timeRDDFromDF ::= (end - start) + } + + print("DataFrame : ", timeDF) + print("RDD :", timeRDD) + print("RDD from DataFrame : ", timeRDDFromDF) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BaggedPointSuite.scala similarity index 99% rename from mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala rename to mllib/src/test/scala/org/apache/spark/ml/tree/impl/BaggedPointSuite.scala index 9d756da41032..77ab3d8bb75f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BaggedPointSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.tree.impl +package org.apache.spark.ml.tree.impl import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.tree.EnsembleTestHelper diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala new file mode 100644 index 000000000000..4109a299091d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/GradientBoostedTreesSuite.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.tree.impl + +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.mllib.tree.{GradientBoostedTreesSuite => OldGBTSuite} +import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy} +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.impurity.Variance +import org.apache.spark.mllib.tree.loss.{AbsoluteError, LogLoss, SquaredError} +import org.apache.spark.mllib.util.MLlibTestSparkContext + +/** + * Test suite for [[GradientBoostedTrees]]. + */ +class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { + + import testImplicits._ + + test("runWithValidation stops early and performs better on a validation dataset") { + // Set numIterations large enough so that it stops early. + val numIterations = 20 + val trainRdd = sc.parallelize(OldGBTSuite.trainData, 2).map(_.asML) + val validateRdd = sc.parallelize(OldGBTSuite.validateData, 2).map(_.asML) + val trainDF = trainRdd.toDF() + val validateDF = validateRdd.toDF() + + val algos = Array(Regression, Regression, Classification) + val losses = Array(SquaredError, AbsoluteError, LogLoss) + algos.zip(losses).foreach { case (algo, loss) => + val treeStrategy = new Strategy(algo = algo, impurity = Variance, maxDepth = 2, + categoricalFeaturesInfo = Map.empty) + val boostingStrategy = + new BoostingStrategy(treeStrategy, loss, numIterations, validationTol = 0.0) + val (validateTrees, validateTreeWeights) = GradientBoostedTrees + .runWithValidation(trainRdd, validateRdd, boostingStrategy, 42L) + val numTrees = validateTrees.length + assert(numTrees !== numIterations) + + // Test that it performs better on the validation dataset. + val (trees, treeWeights) = GradientBoostedTrees.run(trainRdd, boostingStrategy, 42L) + val (errorWithoutValidation, errorWithValidation) = { + if (algo == Classification) { + val remappedRdd = validateRdd.map(x => new LabeledPoint(2 * x.label - 1, x.features)) + (GradientBoostedTrees.computeError(remappedRdd, trees, treeWeights, loss), + GradientBoostedTrees.computeError(remappedRdd, validateTrees, + validateTreeWeights, loss)) + } else { + (GradientBoostedTrees.computeError(validateRdd, trees, treeWeights, loss), + GradientBoostedTrees.computeError(validateRdd, validateTrees, + validateTreeWeights, loss)) + } + } + assert(errorWithValidation <= errorWithoutValidation) + + // Test that results from evaluateEachIteration comply with runWithValidation. + // Note that convergenceTol is set to 0.0 + val evaluationArray = GradientBoostedTrees + .evaluateEachIteration(validateRdd, trees, treeWeights, loss, algo) + assert(evaluationArray.length === numIterations) + assert(evaluationArray(numTrees) > evaluationArray(numTrees - 1)) + var i = 1 + while (i < numTrees) { + assert(evaluationArray(i) <= evaluationArray(i - 1)) + i += 1 + } + } + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala index d5c238e9ae16..dbe2ea931fb9 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala @@ -17,14 +17,18 @@ package org.apache.spark.ml.tree.impl +import scala.collection.mutable + import org.apache.spark.SparkFunSuite import org.apache.spark.ml.classification.DecisionTreeClassificationModel -import org.apache.spark.ml.impl.TreeTests -import org.apache.spark.ml.tree.{ContinuousSplit, DecisionTreeModel, LeafNode, Node} -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.tree.impurity.GiniCalculator +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.tree._ +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.tree.{DecisionTreeSuite => OldDTSuite, EnsembleTestHelper} +import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, QuantileStrategy, Strategy => OldStrategy} +import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, GiniCalculator, Variance} import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.collection.OpenHashMap /** @@ -34,6 +38,538 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { import RandomForestSuite.mapToVec + ///////////////////////////////////////////////////////////////////////////// + // Tests for split calculation + ///////////////////////////////////////////////////////////////////////////// + + test("Binary classification with continuous features: split calculation") { + val arr = OldDTSuite.generateOrderedLabeledPointsWithLabel1().map(_.asML) + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new OldStrategy(OldAlgo.Classification, Gini, 3, 2, 100) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + val splits = RandomForest.findSplits(rdd, metadata, seed = 42) + assert(splits.length === 2) + assert(splits(0).length === 99) + } + + test("Binary classification with binary (ordered) categorical features: split calculation") { + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 2, + maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1 -> 2)) + + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val splits = RandomForest.findSplits(rdd, metadata, seed = 42) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + assert(splits.length === 2) + // no splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + } + + test("Binary classification with 3-ary (ordered) categorical features," + + " with no samples for one category: split calculation") { + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 2, + maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val splits = RandomForest.findSplits(rdd, metadata, seed = 42) + assert(splits.length === 2) + // no splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + } + + test("find splits for a continuous feature") { + // find splits for normal case + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(6), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + val featureSamples = Array.fill(200000)(math.random) + val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + assert(splits.length === 5) + assert(fakeMetadata.numSplits(0) === 5) + assert(fakeMetadata.numBins(0) === 6) + // check returned splits are distinct + assert(splits.distinct.length === splits.length) + } + + // SPARK-16957: Use midpoints for split values. + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(3), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + + // possibleSplits <= numSplits + { + val featureSamples = Array(0, 1, 0, 0, 1, 0, 1, 1).map(_.toDouble) + val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + val expectedSplits = Array((0.0 + 1.0) / 2) + assert(splits === expectedSplits) + } + + // possibleSplits > numSplits + { + val featureSamples = Array(0, 0, 1, 1, 2, 2, 3, 3).map(_.toDouble) + val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + val expectedSplits = Array((0.0 + 1.0) / 2, (2.0 + 3.0) / 2) + assert(splits === expectedSplits) + } + } + + // find splits should not return identical splits + // when there are not enough split candidates, reduce the number of splits in metadata + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(5), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + val featureSamples = Array(1, 1, 2, 2, 2, 2, 2, 2, 2, 2, 3, 3).map(_.toDouble) + val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + val expectedSplits = Array((1.0 + 2.0) / 2, (2.0 + 3.0) / 2) + assert(splits === expectedSplits) + // check returned splits are distinct + assert(splits.distinct.length === splits.length) + } + + // find splits when most samples close to the minimum + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(3), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + val featureSamples = Array(2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 3, 4, 5) + .map(_.toDouble) + val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + val expectedSplits = Array((2.0 + 3.0) / 2, (3.0 + 4.0) / 2) + assert(splits === expectedSplits) + } + + // find splits when most samples close to the maximum + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(2), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + val featureSamples = Array(0, 1, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2).map(_.toDouble) + val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + val expectedSplits = Array((1.0 + 2.0) / 2) + assert(splits === expectedSplits) + } + + // find splits for constant feature + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(3), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + val featureSamples = Array(0, 0, 0).map(_.toDouble) + val featureSamplesEmpty = Array.empty[Double] + val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + assert(splits === Array.empty[Double]) + val splitsEmpty = + RandomForest.findSplitsForContinuousFeature(featureSamplesEmpty, fakeMetadata, 0) + assert(splitsEmpty === Array.empty[Double]) + } + } + + test("train with empty arrays") { + val lp = LabeledPoint(1.0, Vectors.dense(Array.empty[Double])) + val data = Array.fill(5)(lp) + val rdd = sc.parallelize(data) + + val strategy = new OldStrategy(OldAlgo.Regression, Gini, maxDepth = 2, + maxBins = 5) + withClue("DecisionTree requires number of features > 0," + + " but was given an empty features vector") { + intercept[IllegalArgumentException] { + RandomForest.run(rdd, strategy, 1, "all", 42L, instr = None) + } + } + } + + test("train with constant features") { + val lp = LabeledPoint(1.0, Vectors.dense(0.0, 0.0, 0.0)) + val data = Array.fill(5)(lp) + val rdd = sc.parallelize(data) + val strategy = new OldStrategy( + OldAlgo.Classification, + Gini, + maxDepth = 2, + numClasses = 2, + maxBins = 5, + categoricalFeaturesInfo = Map(0 -> 1, 1 -> 5)) + val Array(tree) = RandomForest.run(rdd, strategy, 1, "all", 42L, instr = None) + assert(tree.rootNode.impurity === -1.0) + assert(tree.depth === 0) + assert(tree.rootNode.prediction === lp.label) + + // Test with no categorical features + val strategy2 = new OldStrategy( + OldAlgo.Regression, + Variance, + maxDepth = 2, + maxBins = 5) + val Array(tree2) = RandomForest.run(rdd, strategy2, 1, "all", 42L, instr = None) + assert(tree2.rootNode.impurity === -1.0) + assert(tree2.depth === 0) + assert(tree2.rootNode.prediction === lp.label) + } + + test("Multiclass classification with unordered categorical features: split calculations") { + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new OldStrategy( + OldAlgo.Classification, + Gini, + maxDepth = 2, + numClasses = 100, + maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(metadata.isUnordered(featureIndex = 0)) + assert(metadata.isUnordered(featureIndex = 1)) + val splits = RandomForest.findSplits(rdd, metadata, seed = 42) + assert(splits.length === 2) + assert(splits(0).length === 3) + assert(metadata.numSplits(0) === 3) + assert(metadata.numBins(0) === 3) + assert(metadata.numSplits(1) === 3) + assert(metadata.numBins(1) === 3) + + // Expecting 2^2 - 1 = 3 splits per feature + def checkCategoricalSplit(s: Split, featureIndex: Int, leftCategories: Array[Double]): Unit = { + assert(s.featureIndex === featureIndex) + assert(s.isInstanceOf[CategoricalSplit]) + val s0 = s.asInstanceOf[CategoricalSplit] + assert(s0.leftCategories === leftCategories) + assert(s0.numCategories === 3) // for this unit test + } + // Feature 0 + checkCategoricalSplit(splits(0)(0), 0, Array(0.0)) + checkCategoricalSplit(splits(0)(1), 0, Array(1.0)) + checkCategoricalSplit(splits(0)(2), 0, Array(0.0, 1.0)) + // Feature 1 + checkCategoricalSplit(splits(1)(0), 1, Array(0.0)) + checkCategoricalSplit(splits(1)(1), 1, Array(1.0)) + checkCategoricalSplit(splits(1)(2), 1, Array(0.0, 1.0)) + } + + test("Multiclass classification with ordered categorical features: split calculations") { + val arr = OldDTSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures().map(_.asML) + assert(arr.length === 3000) + val rdd = sc.parallelize(arr) + val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 100, + maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) + // 2^(10-1) - 1 > 100, so categorical features will be ordered + + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val splits = RandomForest.findSplits(rdd, metadata, seed = 42) + assert(splits.length === 2) + // no splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + } + + ///////////////////////////////////////////////////////////////////////////// + // Tests of other algorithm internals + ///////////////////////////////////////////////////////////////////////////// + + test("extract categories from a number for multiclass classification") { + val l = RandomForest.extractMultiClassCategories(13, 10) + assert(l.length === 3) + assert(Seq(3.0, 2.0, 0.0) === l) + } + + test("Avoid aggregation on the last level") { + val arr = Array( + LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0))) + val input = sc.parallelize(arr) + + val strategy = new OldStrategy(algo = OldAlgo.Classification, impurity = Gini, maxDepth = 1, + numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val splits = RandomForest.findSplits(input, metadata, seed = 42) + + val treeInput = TreePoint.convertToTreeRDD(input, splits, metadata) + val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, withReplacement = false) + + val topNode = LearningNode.emptyNode(nodeIndex = 1) + assert(topNode.isLeaf === false) + assert(topNode.stats === null) + + val nodesForGroup = Map((0, Array(topNode))) + val treeToNodeToIndexInfo = Map((0, Map( + (topNode.id, new RandomForest.NodeIndexInfo(0, None)) + ))) + val nodeStack = new mutable.ArrayStack[(Int, LearningNode)] + RandomForest.findBestSplits(baggedInput, metadata, Map(0 -> topNode), + nodesForGroup, treeToNodeToIndexInfo, splits, nodeStack) + + // don't enqueue leaf nodes into node queue + assert(nodeStack.isEmpty) + + // set impurity and predict for topNode + assert(topNode.stats !== null) + assert(topNode.stats.impurity > 0.0) + + // set impurity and predict for child nodes + assert(topNode.leftChild.get.toNode.prediction === 0.0) + assert(topNode.rightChild.get.toNode.prediction === 1.0) + assert(topNode.leftChild.get.stats.impurity === 0.0) + assert(topNode.rightChild.get.stats.impurity === 0.0) + } + + test("Avoid aggregation if impurity is 0.0") { + val arr = Array( + LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0))) + val input = sc.parallelize(arr) + + val strategy = new OldStrategy(algo = OldAlgo.Classification, impurity = Gini, maxDepth = 5, + numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val splits = RandomForest.findSplits(input, metadata, seed = 42) + + val treeInput = TreePoint.convertToTreeRDD(input, splits, metadata) + val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, withReplacement = false) + + val topNode = LearningNode.emptyNode(nodeIndex = 1) + assert(topNode.isLeaf === false) + assert(topNode.stats === null) + + val nodesForGroup = Map((0, Array(topNode))) + val treeToNodeToIndexInfo = Map((0, Map( + (topNode.id, new RandomForest.NodeIndexInfo(0, None)) + ))) + val nodeStack = new mutable.ArrayStack[(Int, LearningNode)] + RandomForest.findBestSplits(baggedInput, metadata, Map(0 -> topNode), + nodesForGroup, treeToNodeToIndexInfo, splits, nodeStack) + + // don't enqueue a node into node queue if its impurity is 0.0 + assert(nodeStack.isEmpty) + + // set impurity and predict for topNode + assert(topNode.stats !== null) + assert(topNode.stats.impurity > 0.0) + + // set impurity and predict for child nodes + assert(topNode.leftChild.get.toNode.prediction === 0.0) + assert(topNode.rightChild.get.toNode.prediction === 1.0) + assert(topNode.leftChild.get.stats.impurity === 0.0) + assert(topNode.rightChild.get.stats.impurity === 0.0) + } + + test("Use soft prediction for binary classification with ordered categorical features") { + // The following dataset is set up such that the best split is {1} vs. {0, 2}. + // If the hard prediction is used to order the categories, then {0} vs. {1, 2} is chosen. + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(0.0, Vectors.dense(0.0)), + LabeledPoint(1.0, Vectors.dense(0.0)), + LabeledPoint(0.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(2.0)), + LabeledPoint(0.0, Vectors.dense(2.0)), + LabeledPoint(0.0, Vectors.dense(2.0)), + LabeledPoint(1.0, Vectors.dense(2.0))) + val input = sc.parallelize(arr) + + // Must set maxBins s.t. the feature will be treated as an ordered categorical feature. + val strategy = new OldStrategy(algo = OldAlgo.Classification, impurity = Gini, maxDepth = 1, + numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3), maxBins = 3) + + val model = RandomForest.run(input, strategy, numTrees = 1, featureSubsetStrategy = "all", + seed = 42, instr = None).head + model.rootNode match { + case n: InternalNode => n.split match { + case s: CategoricalSplit => + assert(s.leftCategories === Array(1.0)) + case _ => throw new AssertionError("model.rootNode.split was not a CategoricalSplit") + } + case _ => throw new AssertionError("model.rootNode was not an InternalNode") + } + } + + test("Second level node building with vs. without groups") { + val arr = OldDTSuite.generateOrderedLabeledPoints().map(_.asML) + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + // For tree with 1 group + val strategy1 = + new OldStrategy(OldAlgo.Classification, Entropy, 3, 2, 100, maxMemoryInMB = 1000) + // For tree with multiple groups + val strategy2 = + new OldStrategy(OldAlgo.Classification, Entropy, 3, 2, 100, maxMemoryInMB = 0) + + val tree1 = RandomForest.run(rdd, strategy1, numTrees = 1, featureSubsetStrategy = "all", + seed = 42, instr = None).head + val tree2 = RandomForest.run(rdd, strategy2, numTrees = 1, featureSubsetStrategy = "all", + seed = 42, instr = None).head + + def getChildren(rootNode: Node): Array[InternalNode] = rootNode match { + case n: InternalNode => + assert(n.leftChild.isInstanceOf[InternalNode]) + assert(n.rightChild.isInstanceOf[InternalNode]) + Array(n.leftChild.asInstanceOf[InternalNode], n.rightChild.asInstanceOf[InternalNode]) + case _ => throw new AssertionError("rootNode was not an InternalNode") + } + + // Single group second level tree construction. + val children1 = getChildren(tree1.rootNode) + val children2 = getChildren(tree2.rootNode) + + // Verify whether the splits obtained using single group and multiple group level + // construction strategies are the same. + for (i <- 0 until 2) { + assert(children1(i).gain > 0) + assert(children2(i).gain > 0) + assert(children1(i).split === children2(i).split) + assert(children1(i).impurity === children2(i).impurity) + assert(children1(i).impurityStats.stats === children2(i).impurityStats.stats) + assert(children1(i).leftChild.impurity === children2(i).leftChild.impurity) + assert(children1(i).rightChild.impurity === children2(i).rightChild.impurity) + assert(children1(i).prediction === children2(i).prediction) + } + } + + def binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy: OldStrategy) { + val numFeatures = 50 + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures, 1000) + val rdd = sc.parallelize(arr).map(_.asML) + + // Select feature subset for top nodes. Return true if OK. + def checkFeatureSubsetStrategy( + numTrees: Int, + featureSubsetStrategy: String, + numFeaturesPerNode: Int): Unit = { + val seeds = Array(123, 5354, 230, 349867, 23987) + val maxMemoryUsage: Long = 128 * 1024L * 1024L + val metadata = + DecisionTreeMetadata.buildMetadata(rdd, strategy, numTrees, featureSubsetStrategy) + seeds.foreach { seed => + val failString = s"Failed on test with:" + + s"numTrees=$numTrees, featureSubsetStrategy=$featureSubsetStrategy," + + s" numFeaturesPerNode=$numFeaturesPerNode, seed=$seed" + val nodeStack = new mutable.ArrayStack[(Int, LearningNode)] + val topNodes: Array[LearningNode] = new Array[LearningNode](numTrees) + Range(0, numTrees).foreach { treeIndex => + topNodes(treeIndex) = LearningNode.emptyNode(nodeIndex = 1) + nodeStack.push((treeIndex, topNodes(treeIndex))) + } + val rng = new scala.util.Random(seed = seed) + val (nodesForGroup: Map[Int, Array[LearningNode]], + treeToNodeToIndexInfo: Map[Int, Map[Int, RandomForest.NodeIndexInfo]]) = + RandomForest.selectNodesToSplit(nodeStack, maxMemoryUsage, metadata, rng) + + assert(nodesForGroup.size === numTrees, failString) + assert(nodesForGroup.values.forall(_.length == 1), failString) // 1 node per tree + + if (numFeaturesPerNode == numFeatures) { + // featureSubset values should all be None + assert(treeToNodeToIndexInfo.values.forall(_.values.forall(_.featureSubset.isEmpty)), + failString) + } else { + // Check number of features. + assert(treeToNodeToIndexInfo.values.forall(_.values.forall( + _.featureSubset.get.length === numFeaturesPerNode)), failString) + } + } + } + + checkFeatureSubsetStrategy(numTrees = 1, "auto", numFeatures) + checkFeatureSubsetStrategy(numTrees = 1, "all", numFeatures) + checkFeatureSubsetStrategy(numTrees = 1, "sqrt", math.sqrt(numFeatures).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 1, "log2", + (math.log(numFeatures) / math.log(2)).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 1, "onethird", (numFeatures / 3.0).ceil.toInt) + + val realStrategies = Array(".1", ".10", "0.10", "0.1", "0.9", "1.0") + for (strategy <- realStrategies) { + val expected = (strategy.toDouble * numFeatures).ceil.toInt + checkFeatureSubsetStrategy(numTrees = 1, strategy, expected) + } + + val integerStrategies = Array("1", "10", "100", "1000", "10000") + for (strategy <- integerStrategies) { + val expected = if (strategy.toInt < numFeatures) strategy.toInt else numFeatures + checkFeatureSubsetStrategy(numTrees = 1, strategy, expected) + } + + val invalidStrategies = Array("-.1", "-.10", "-0.10", ".0", "0.0", "1.1", "0") + for (invalidStrategy <- invalidStrategies) { + intercept[IllegalArgumentException]{ + val metadata = + DecisionTreeMetadata.buildMetadata(rdd, strategy, numTrees = 1, invalidStrategy) + } + } + + checkFeatureSubsetStrategy(numTrees = 2, "all", numFeatures) + checkFeatureSubsetStrategy(numTrees = 2, "auto", math.sqrt(numFeatures).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 2, "sqrt", math.sqrt(numFeatures).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 2, "log2", + (math.log(numFeatures) / math.log(2)).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 2, "onethird", (numFeatures / 3.0).ceil.toInt) + + for (strategy <- realStrategies) { + val expected = (strategy.toDouble * numFeatures).ceil.toInt + checkFeatureSubsetStrategy(numTrees = 2, strategy, expected) + } + + for (strategy <- integerStrategies) { + val expected = if (strategy.toInt < numFeatures) strategy.toInt else numFeatures + checkFeatureSubsetStrategy(numTrees = 2, strategy, expected) + } + for (invalidStrategy <- invalidStrategies) { + intercept[IllegalArgumentException]{ + val metadata = + DecisionTreeMetadata.buildMetadata(rdd, strategy, numTrees = 2, invalidStrategy) + } + } + } + + test("Binary classification with continuous features: subsampling features") { + val categoricalFeaturesInfo = Map.empty[Int, Int] + val strategy = new OldStrategy(algo = OldAlgo.Classification, impurity = Gini, maxDepth = 2, + numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy) + } + + test("Binary classification with continuous features and node Id cache: subsampling features") { + val categoricalFeaturesInfo = Map.empty[Int, Int] + val strategy = new OldStrategy(algo = OldAlgo.Classification, impurity = Gini, maxDepth = 2, + numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, + useNodeIdCache = true) + binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy) + } + test("computeFeatureImportance, featureImportances") { /* Build tree for testing, with this structure: grandParent @@ -58,7 +594,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { // Test feature importance computed at different subtrees. def testNode(node: Node, expected: Map[Int, Double]): Unit = { val map = new OpenHashMap[Int, Double]() - RandomForest.computeFeatureImportance(node, map) + TreeEnsembleModel.computeFeatureImportance(node, map) assert(mapToVec(map.toMap) ~== mapToVec(expected) relTol 0.01) } @@ -80,7 +616,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { new DecisionTreeClassificationModel(root, numFeatures = 2, numClasses = 3) .asInstanceOf[DecisionTreeModel] } - val importances: Vector = RandomForest.featureImportances(trees, 2) + val importances: Vector = TreeEnsembleModel.featureImportances(trees, 2) val tree2norm = feature0importance + feature1importance val expected = Vectors.dense((1.0 + feature0importance / tree2norm) / 2.0, (feature1importance / tree2norm) / 2.0) @@ -91,11 +627,10 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { val map = new OpenHashMap[Int, Double]() map(0) = 1.0 map(2) = 2.0 - RandomForest.normalizeMapValues(map) + TreeEnsembleModel.normalizeMapValues(map) val expected = Map(0 -> 1.0 / 3.0, 2 -> 2.0 / 3.0) assert(mapToVec(map.toMap) ~== mapToVec(expected) relTol 0.01) } - } private object RandomForestSuite { diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala new file mode 100644 index 000000000000..b6894b30b0c2 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala @@ -0,0 +1,228 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.tree.impl + +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkContext, SparkFunSuite} +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.ml.attribute.{AttributeGroup, NominalAttribute, NumericAttribute} +import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.tree._ +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, SparkSession} + +private[ml] object TreeTests extends SparkFunSuite { + + /** + * Convert the given data to a DataFrame, and set the features and label metadata. + * @param data Dataset. Categorical features and labels must already have 0-based indices. + * This must be non-empty. + * @param categoricalFeatures Map: categorical feature index to number of distinct values + * @param numClasses Number of classes label can take. If 0, mark as continuous. + * @return DataFrame with metadata + */ + def setMetadata( + data: RDD[LabeledPoint], + categoricalFeatures: Map[Int, Int], + numClasses: Int): DataFrame = { + val spark = SparkSession.builder() + .sparkContext(data.sparkContext) + .getOrCreate() + import spark.implicits._ + + val df = data.toDF() + val numFeatures = data.first().features.size + val featuresAttributes = Range(0, numFeatures).map { feature => + if (categoricalFeatures.contains(feature)) { + NominalAttribute.defaultAttr.withIndex(feature).withNumValues(categoricalFeatures(feature)) + } else { + NumericAttribute.defaultAttr.withIndex(feature) + } + }.toArray + val featuresMetadata = new AttributeGroup("features", featuresAttributes).toMetadata() + val labelAttribute = if (numClasses == 0) { + NumericAttribute.defaultAttr.withName("label") + } else { + NominalAttribute.defaultAttr.withName("label").withNumValues(numClasses) + } + val labelMetadata = labelAttribute.toMetadata() + df.select(df("features").as("features", featuresMetadata), + df("label").as("label", labelMetadata)) + } + + /** + * Java-friendly version of `setMetadata()` + */ + def setMetadata( + data: JavaRDD[LabeledPoint], + categoricalFeatures: java.util.Map[java.lang.Integer, java.lang.Integer], + numClasses: Int): DataFrame = { + setMetadata(data.rdd, categoricalFeatures.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + numClasses) + } + + /** + * Set label metadata (particularly the number of classes) on a DataFrame. + * @param data Dataset. Categorical features and labels must already have 0-based indices. + * This must be non-empty. + * @param numClasses Number of classes label can take. If 0, mark as continuous. + * @param labelColName Name of the label column on which to set the metadata. + * @param featuresColName Name of the features column + * @return DataFrame with metadata + */ + def setMetadata( + data: DataFrame, + numClasses: Int, + labelColName: String, + featuresColName: String): DataFrame = { + val labelAttribute = if (numClasses == 0) { + NumericAttribute.defaultAttr.withName(labelColName) + } else { + NominalAttribute.defaultAttr.withName(labelColName).withNumValues(numClasses) + } + val labelMetadata = labelAttribute.toMetadata() + data.select(data(featuresColName), data(labelColName).as(labelColName, labelMetadata)) + } + + /** + * Check if the two trees are exactly the same. + * Note: I hesitate to override Node.equals since it could cause problems if users + * make mistakes such as creating loops of Nodes. + * If the trees are not equal, this prints the two trees and throws an exception. + */ + def checkEqual(a: DecisionTreeModel, b: DecisionTreeModel): Unit = { + try { + checkEqual(a.rootNode, b.rootNode) + } catch { + case ex: Exception => + throw new AssertionError("checkEqual failed since the two trees were not identical.\n" + + "TREE A:\n" + a.toDebugString + "\n" + + "TREE B:\n" + b.toDebugString + "\n", ex) + } + } + + /** + * Return true iff the two nodes and their descendants are exactly the same. + * Note: I hesitate to override Node.equals since it could cause problems if users + * make mistakes such as creating loops of Nodes. + */ + private def checkEqual(a: Node, b: Node): Unit = { + assert(a.prediction === b.prediction) + assert(a.impurity === b.impurity) + (a, b) match { + case (aye: InternalNode, bee: InternalNode) => + assert(aye.split === bee.split) + checkEqual(aye.leftChild, bee.leftChild) + checkEqual(aye.rightChild, bee.rightChild) + case (aye: LeafNode, bee: LeafNode) => // do nothing + case _ => + throw new AssertionError("Found mismatched nodes") + } + } + + /** + * Check if the two models are exactly the same. + * If the models are not equal, this throws an exception. + */ + def checkEqual[M <: DecisionTreeModel](a: TreeEnsembleModel[M], b: TreeEnsembleModel[M]): Unit = { + try { + a.trees.zip(b.trees).foreach { case (treeA, treeB) => + TreeTests.checkEqual(treeA, treeB) + } + assert(a.treeWeights === b.treeWeights) + } catch { + case ex: Exception => throw new AssertionError( + "checkEqual failed since the two tree ensembles were not identical") + } + } + + /** + * Helper method for constructing a tree for testing. + * Given left, right children, construct a parent node. + * @param split Split for parent node + * @return Parent node with children attached + */ + def buildParentNode(left: Node, right: Node, split: Split): Node = { + val leftImp = left.impurityStats + val rightImp = right.impurityStats + val parentImp = leftImp.copy.add(rightImp) + val leftWeight = leftImp.count / parentImp.count.toDouble + val rightWeight = rightImp.count / parentImp.count.toDouble + val gain = parentImp.calculate() - + (leftWeight * leftImp.calculate() + rightWeight * rightImp.calculate()) + val pred = parentImp.predict + new InternalNode(pred, parentImp.calculate(), gain, left, right, split, parentImp) + } + + /** + * Create some toy data for testing feature importances. + */ + def featureImportanceData(sc: SparkContext): RDD[LabeledPoint] = sc.parallelize(Seq( + new LabeledPoint(0, Vectors.dense(1, 0, 0, 0, 1)), + new LabeledPoint(1, Vectors.dense(1, 1, 0, 1, 0)), + new LabeledPoint(1, Vectors.dense(1, 1, 0, 0, 0)), + new LabeledPoint(0, Vectors.dense(1, 0, 0, 0, 0)), + new LabeledPoint(1, Vectors.dense(1, 1, 0, 0, 0)) + )) + + /** + * Create some toy data for testing correctness of variance. + */ + def varianceData(sc: SparkContext): RDD[LabeledPoint] = sc.parallelize(Seq( + new LabeledPoint(1.0, Vectors.dense(Array(0.0))), + new LabeledPoint(2.0, Vectors.dense(Array(1.0))), + new LabeledPoint(3.0, Vectors.dense(Array(2.0))), + new LabeledPoint(10.0, Vectors.dense(Array(3.0))), + new LabeledPoint(12.0, Vectors.dense(Array(4.0))), + new LabeledPoint(14.0, Vectors.dense(Array(5.0))) + )) + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + * + * This set of Params is for all Decision Tree-based models. + */ + val allParamSettings: Map[String, Any] = Map( + "checkpointInterval" -> 7, + "seed" -> 543L, + "maxDepth" -> 2, + "maxBins" -> 20, + "minInstancesPerNode" -> 2, + "minInfoGain" -> 1e-14, + "maxMemoryInMB" -> 257, + "cacheNodeIds" -> true + ) + + /** Data for tree read/write tests which produces a non-trivial tree. */ + def getTreeReadWriteData(sc: SparkContext): RDD[LabeledPoint] = { + val arr = Array( + LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.0)), + LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), + LabeledPoint(0.0, Vectors.dense(0.0, 2.0)), + LabeledPoint(0.0, Vectors.dense(1.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(1.0, 1.0)), + LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), + LabeledPoint(1.0, Vectors.dense(1.0, 2.0))) + sc.parallelize(arr) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala index cbe09292a033..a01744f7b67f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala @@ -18,27 +18,30 @@ package org.apache.spark.ml.tuning import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.util.MLTestingUtils -import org.apache.spark.ml.{Estimator, Model} -import org.apache.spark.ml.classification.LogisticRegression -import org.apache.spark.ml.evaluation.{BinaryClassificationEvaluator, Evaluator, RegressionEvaluator} +import org.apache.spark.ml.{Estimator, Model, Pipeline} +import org.apache.spark.ml.classification.{LogisticRegression, LogisticRegressionModel, OneVsRest} +import org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInput +import org.apache.spark.ml.evaluation.{BinaryClassificationEvaluator, Evaluator, MulticlassClassificationEvaluator, RegressionEvaluator} +import org.apache.spark.ml.feature.HashingTF +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared.HasInputCol import org.apache.spark.ml.regression.LinearRegression -import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext} -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.Dataset import org.apache.spark.sql.types.StructType -class CrossValidatorSuite extends SparkFunSuite with MLlibTestSparkContext { +class CrossValidatorSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { - @transient var dataset: DataFrame = _ + import testImplicits._ + + @transient var dataset: Dataset[_] = _ override def beforeAll(): Unit = { super.beforeAll() - val sqlContext = new SQLContext(sc) - dataset = sqlContext.createDataFrame( - sc.parallelize(generateLogisticInput(1.0, 1.0, 100, 42), 2)) + dataset = sc.parallelize(generateLogisticInput(1.0, 1.0, 100, 42), 2).toDF() } test("cross validation with logistic regression") { @@ -55,8 +58,7 @@ class CrossValidatorSuite extends SparkFunSuite with MLlibTestSparkContext { .setNumFolds(3) val cvModel = cv.fit(dataset) - // copied model must have the same paren. - MLTestingUtils.checkCopy(cvModel) + MLTestingUtils.checkCopyAndUids(cv, cvModel) val parent = cvModel.bestModel.parent.asInstanceOf[LogisticRegression] assert(parent.getRegParam === 0.001) @@ -65,9 +67,10 @@ class CrossValidatorSuite extends SparkFunSuite with MLlibTestSparkContext { } test("cross validation with linear regression") { - val dataset = sqlContext.createDataFrame( - sc.parallelize(LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2)) + val dataset = sc.parallelize( + LinearDataGenerator.generateLinearInput( + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2) + .map(_.asML).toDF() val trainer = new LinearRegression().setSolver("l-bfgs") val lrParamMaps = new ParamGridBuilder() @@ -94,8 +97,8 @@ class CrossValidatorSuite extends SparkFunSuite with MLlibTestSparkContext { assert(cvModel2.avgMetrics.length === lrParamMaps.length) } - test("validateParams should check estimatorParamMaps") { - import CrossValidatorSuite._ + test("transformSchema should check estimatorParamMaps") { + import CrossValidatorSuite.{MyEstimator, MyEvaluator} val est = new MyEstimator("est") val eval = new MyEvaluator @@ -108,30 +111,303 @@ class CrossValidatorSuite extends SparkFunSuite with MLlibTestSparkContext { .setEstimatorParamMaps(paramMaps) .setEvaluator(eval) - cv.validateParams() // This should pass. + cv.transformSchema(new StructType()) // This should pass. val invalidParamMaps = paramMaps :+ ParamMap(est.inputCol -> "") cv.setEstimatorParamMaps(invalidParamMaps) intercept[IllegalArgumentException] { - cv.validateParams() + cv.transformSchema(new StructType()) + } + } + + test("cross validation with parallel evaluation") { + val lr = new LogisticRegression + val lrParamMaps = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.001, 1000.0)) + .addGrid(lr.maxIter, Array(0, 3)) + .build() + val eval = new BinaryClassificationEvaluator + val cv = new CrossValidator() + .setEstimator(lr) + .setEstimatorParamMaps(lrParamMaps) + .setEvaluator(eval) + .setNumFolds(2) + .setParallelism(1) + val cvSerialModel = cv.fit(dataset) + cv.setParallelism(2) + val cvParallelModel = cv.fit(dataset) + + val serialMetrics = cvSerialModel.avgMetrics.sorted + val parallelMetrics = cvParallelModel.avgMetrics.sorted + assert(serialMetrics === parallelMetrics) + + val parentSerial = cvSerialModel.bestModel.parent.asInstanceOf[LogisticRegression] + val parentParallel = cvParallelModel.bestModel.parent.asInstanceOf[LogisticRegression] + assert(parentSerial.getRegParam === parentParallel.getRegParam) + assert(parentSerial.getMaxIter === parentParallel.getMaxIter) + } + + test("read/write: CrossValidator with simple estimator") { + val lr = new LogisticRegression().setMaxIter(3) + val evaluator = new BinaryClassificationEvaluator() + .setMetricName("areaUnderPR") // not default metric + val paramMaps = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.1, 0.2)) + .build() + val cv = new CrossValidator() + .setEstimator(lr) + .setEvaluator(evaluator) + .setNumFolds(20) + .setEstimatorParamMaps(paramMaps) + .setSeed(42L) + .setParallelism(2) + + val cv2 = testDefaultReadWrite(cv, testParams = false) + + assert(cv.uid === cv2.uid) + assert(cv.getNumFolds === cv2.getNumFolds) + assert(cv.getSeed === cv2.getSeed) + assert(cv.getParallelism === cv2.getParallelism) + + assert(cv2.getEvaluator.isInstanceOf[BinaryClassificationEvaluator]) + val evaluator2 = cv2.getEvaluator.asInstanceOf[BinaryClassificationEvaluator] + assert(evaluator.uid === evaluator2.uid) + assert(evaluator.getMetricName === evaluator2.getMetricName) + + cv2.getEstimator match { + case lr2: LogisticRegression => + assert(lr.uid === lr2.uid) + assert(lr.getMaxIter === lr2.getMaxIter) + case other => + throw new AssertionError(s"Loaded CrossValidator expected estimator of type" + + s" LogisticRegression but found ${other.getClass.getName}") + } + + ValidatorParamsSuiteHelpers + .compareParamMaps(cv.getEstimatorParamMaps, cv2.getEstimatorParamMaps) + } + + test("read/write: CrossValidator with nested estimator") { + val ova = new OneVsRest().setClassifier(new LogisticRegression) + val evaluator = new MulticlassClassificationEvaluator() + .setMetricName("accuracy") + val classifier1 = new LogisticRegression().setRegParam(2.0) + val classifier2 = new LogisticRegression().setRegParam(3.0) + // params that are not JSON serializable must inherit from Params + val paramMaps = new ParamGridBuilder() + .addGrid(ova.classifier, Array(classifier1, classifier2)) + .build() + val cv = new CrossValidator() + .setEstimator(ova) + .setEvaluator(evaluator) + .setNumFolds(20) + .setEstimatorParamMaps(paramMaps) + + val cv2 = testDefaultReadWrite(cv, testParams = false) + + assert(cv.uid === cv2.uid) + assert(cv.getNumFolds === cv2.getNumFolds) + assert(cv.getSeed === cv2.getSeed) + + assert(cv2.getEvaluator.isInstanceOf[MulticlassClassificationEvaluator]) + val evaluator2 = cv2.getEvaluator.asInstanceOf[MulticlassClassificationEvaluator] + assert(evaluator.uid === evaluator2.uid) + assert(evaluator.getMetricName === evaluator2.getMetricName) + + cv2.getEstimator match { + case ova2: OneVsRest => + assert(ova.uid === ova2.uid) + ova2.getClassifier match { + case lr: LogisticRegression => + assert(ova.getClassifier.asInstanceOf[LogisticRegression].getMaxIter + === lr.getMaxIter) + case other => + throw new AssertionError(s"Loaded CrossValidator expected estimator of type" + + s" LogisticRegression but found ${other.getClass.getName}") + } + + case other => + throw new AssertionError(s"Loaded CrossValidator expected estimator of type" + + s" OneVsRest but found ${other.getClass.getName}") + } + + ValidatorParamsSuiteHelpers + .compareParamMaps(cv.getEstimatorParamMaps, cv2.getEstimatorParamMaps) + } + + test("read/write: Persistence of nested estimator works if parent directory changes") { + val ova = new OneVsRest().setClassifier(new LogisticRegression) + val evaluator = new MulticlassClassificationEvaluator() + .setMetricName("accuracy") + val classifier1 = new LogisticRegression().setRegParam(2.0) + val classifier2 = new LogisticRegression().setRegParam(3.0) + // params that are not JSON serializable must inherit from Params + val paramMaps = new ParamGridBuilder() + .addGrid(ova.classifier, Array(classifier1, classifier2)) + .build() + val cv = new CrossValidator() + .setEstimator(ova) + .setEvaluator(evaluator) + .setNumFolds(20) + .setEstimatorParamMaps(paramMaps) + + ValidatorParamsSuiteHelpers.testFileMove(cv, tempDir) + } + + test("read/write: CrossValidator with complex estimator") { + // workflow: CrossValidator[Pipeline[HashingTF, CrossValidator[LogisticRegression]]] + val lrEvaluator = new BinaryClassificationEvaluator() + .setMetricName("areaUnderPR") // not default metric + + val lr = new LogisticRegression().setMaxIter(3) + val lrParamMaps = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.1, 0.2)) + .build() + val lrcv = new CrossValidator() + .setEstimator(lr) + .setEvaluator(lrEvaluator) + .setEstimatorParamMaps(lrParamMaps) + + val hashingTF = new HashingTF() + val pipeline = new Pipeline().setStages(Array(hashingTF, lrcv)) + val paramMaps = new ParamGridBuilder() + .addGrid(hashingTF.numFeatures, Array(10, 20)) + .addGrid(lr.elasticNetParam, Array(0.0, 1.0)) + .build() + val evaluator = new BinaryClassificationEvaluator() + + val cv = new CrossValidator() + .setEstimator(pipeline) + .setEvaluator(evaluator) + .setNumFolds(20) + .setEstimatorParamMaps(paramMaps) + + val cv2 = testDefaultReadWrite(cv, testParams = false) + + assert(cv.uid === cv2.uid) + assert(cv.getNumFolds === cv2.getNumFolds) + assert(cv.getSeed === cv2.getSeed) + + assert(cv2.getEvaluator.isInstanceOf[BinaryClassificationEvaluator]) + assert(cv.getEvaluator.uid === cv2.getEvaluator.uid) + + ValidatorParamsSuiteHelpers + .compareParamMaps(cv.getEstimatorParamMaps, cv2.getEstimatorParamMaps) + + cv2.getEstimator match { + case pipeline2: Pipeline => + assert(pipeline.uid === pipeline2.uid) + pipeline2.getStages match { + case Array(hashingTF2: HashingTF, lrcv2: CrossValidator) => + assert(hashingTF.uid === hashingTF2.uid) + lrcv2.getEstimator match { + case lr2: LogisticRegression => + assert(lr.uid === lr2.uid) + assert(lr.getMaxIter === lr2.getMaxIter) + case other => + throw new AssertionError(s"Loaded internal CrossValidator expected to be" + + s" LogisticRegression but found type ${other.getClass.getName}") + } + assert(lrcv.uid === lrcv2.uid) + assert(lrcv2.getEvaluator.isInstanceOf[BinaryClassificationEvaluator]) + assert(lrEvaluator.uid === lrcv2.getEvaluator.uid) + ValidatorParamsSuiteHelpers + .compareParamMaps(lrParamMaps, lrcv2.getEstimatorParamMaps) + case other => + throw new AssertionError("Loaded Pipeline expected stages (HashingTF, CrossValidator)" + + " but found: " + other.map(_.getClass.getName).mkString(", ")) + } + case other => + throw new AssertionError(s"Loaded CrossValidator expected estimator of type" + + s" CrossValidator but found ${other.getClass.getName}") } } + + test("read/write: CrossValidator fails for extraneous Param") { + val lr = new LogisticRegression() + val lr2 = new LogisticRegression() + val evaluator = new BinaryClassificationEvaluator() + val paramMaps = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.1, 0.2)) + .addGrid(lr2.regParam, Array(0.1, 0.2)) + .build() + val cv = new CrossValidator() + .setEstimator(lr) + .setEvaluator(evaluator) + .setEstimatorParamMaps(paramMaps) + withClue("CrossValidator.write failed to catch extraneous Param error") { + intercept[IllegalArgumentException] { + cv.write + } + } + } + + test("read/write: CrossValidatorModel") { + val lr = new LogisticRegression() + .setThreshold(0.6) + val lrModel = new LogisticRegressionModel(lr.uid, Vectors.dense(1.0, 2.0), 1.2) + .setThreshold(0.6) + val evaluator = new BinaryClassificationEvaluator() + .setMetricName("areaUnderPR") // not default metric + val paramMaps = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.1, 0.2)) + .build() + val cv = new CrossValidatorModel("cvUid", lrModel, Array(0.3, 0.6)) + cv.set(cv.estimator, lr) + .set(cv.evaluator, evaluator) + .set(cv.numFolds, 20) + .set(cv.estimatorParamMaps, paramMaps) + + val cv2 = testDefaultReadWrite(cv, testParams = false) + + assert(cv.uid === cv2.uid) + assert(cv.getNumFolds === cv2.getNumFolds) + assert(cv.getSeed === cv2.getSeed) + + assert(cv2.getEvaluator.isInstanceOf[BinaryClassificationEvaluator]) + val evaluator2 = cv2.getEvaluator.asInstanceOf[BinaryClassificationEvaluator] + assert(evaluator.uid === evaluator2.uid) + assert(evaluator.getMetricName === evaluator2.getMetricName) + + cv2.getEstimator match { + case lr2: LogisticRegression => + assert(lr.uid === lr2.uid) + assert(lr.getThreshold === lr2.getThreshold) + case other => + throw new AssertionError(s"Loaded CrossValidator expected estimator of type" + + s" LogisticRegression but found ${other.getClass.getName}") + } + + ValidatorParamsSuiteHelpers + .compareParamMaps(cv.getEstimatorParamMaps, cv2.getEstimatorParamMaps) + + cv2.bestModel match { + case lrModel2: LogisticRegressionModel => + assert(lrModel.uid === lrModel2.uid) + assert(lrModel.getThreshold === lrModel2.getThreshold) + assert(lrModel.coefficients === lrModel2.coefficients) + assert(lrModel.intercept === lrModel2.intercept) + case other => + throw new AssertionError(s"Loaded CrossValidator expected bestModel of type" + + s" LogisticRegressionModel but found ${other.getClass.getName}") + } + assert(cv.avgMetrics === cv2.avgMetrics) + } } -object CrossValidatorSuite { +object CrossValidatorSuite extends SparkFunSuite { abstract class MyModel extends Model[MyModel] class MyEstimator(override val uid: String) extends Estimator[MyModel] with HasInputCol { - override def validateParams(): Unit = require($(inputCol).nonEmpty) - - override def fit(dataset: DataFrame): MyModel = { + override def fit(dataset: Dataset[_]): MyModel = { throw new UnsupportedOperationException } override def transformSchema(schema: StructType): StructType = { - throw new UnsupportedOperationException + require($(inputCol).nonEmpty) + schema } override def copy(extra: ParamMap): MyEstimator = defaultCopy(extra) @@ -139,7 +415,7 @@ object CrossValidatorSuite { class MyEvaluator extends Evaluator { - override def evaluate(dataset: DataFrame): Double = { + override def evaluate(dataset: Dataset[_]): Double = { throw new UnsupportedOperationException } diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala index 5fb80091d0b4..2ed4fbb601b6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala @@ -19,44 +19,56 @@ package org.apache.spark.ml.tuning import org.apache.spark.SparkFunSuite import org.apache.spark.ml.{Estimator, Model} -import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.classification.{LogisticRegression, LogisticRegressionModel, OneVsRest} +import org.apache.spark.ml.classification.LogisticRegressionSuite.generateLogisticInput import org.apache.spark.ml.evaluation.{BinaryClassificationEvaluator, Evaluator, RegressionEvaluator} +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.param.shared.HasInputCol import org.apache.spark.ml.regression.LinearRegression -import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput +import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext} -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.Dataset import org.apache.spark.sql.types.StructType -class TrainValidationSplitSuite extends SparkFunSuite with MLlibTestSparkContext { - test("train validation with logistic regression") { - val dataset = sqlContext.createDataFrame( - sc.parallelize(generateLogisticInput(1.0, 1.0, 100, 42), 2)) +class TrainValidationSplitSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + + @transient var dataset: Dataset[_] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + dataset = sc.parallelize(generateLogisticInput(1.0, 1.0, 100, 42), 2).toDF() + } + test("train validation with logistic regression") { val lr = new LogisticRegression val lrParamMaps = new ParamGridBuilder() .addGrid(lr.regParam, Array(0.001, 1000.0)) .addGrid(lr.maxIter, Array(0, 10)) .build() val eval = new BinaryClassificationEvaluator - val cv = new TrainValidationSplit() + val tvs = new TrainValidationSplit() .setEstimator(lr) .setEstimatorParamMaps(lrParamMaps) .setEvaluator(eval) .setTrainRatio(0.5) - val cvModel = cv.fit(dataset) - val parent = cvModel.bestModel.parent.asInstanceOf[LogisticRegression] - assert(cv.getTrainRatio === 0.5) + .setSeed(42L) + val tvsModel = tvs.fit(dataset) + val parent = tvsModel.bestModel.parent.asInstanceOf[LogisticRegression] + assert(tvs.getTrainRatio === 0.5) assert(parent.getRegParam === 0.001) assert(parent.getMaxIter === 10) - assert(cvModel.validationMetrics.length === lrParamMaps.length) + assert(tvsModel.validationMetrics.length === lrParamMaps.length) } test("train validation with linear regression") { - val dataset = sqlContext.createDataFrame( - sc.parallelize(LinearDataGenerator.generateLinearInput( - 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2)) + val dataset = sc.parallelize( + LinearDataGenerator.generateLinearInput( + 6.3, Array(4.7, 7.2), Array(0.9, -1.3), Array(0.7, 1.2), 100, 42, 0.1), 2) + .map(_.asML).toDF() val trainer = new LinearRegression().setSolver("l-bfgs") val lrParamMaps = new ParamGridBuilder() @@ -64,27 +76,31 @@ class TrainValidationSplitSuite extends SparkFunSuite with MLlibTestSparkContext .addGrid(trainer.maxIter, Array(0, 10)) .build() val eval = new RegressionEvaluator() - val cv = new TrainValidationSplit() + val tvs = new TrainValidationSplit() .setEstimator(trainer) .setEstimatorParamMaps(lrParamMaps) .setEvaluator(eval) .setTrainRatio(0.5) - val cvModel = cv.fit(dataset) - val parent = cvModel.bestModel.parent.asInstanceOf[LinearRegression] + .setSeed(42L) + val tvsModel = tvs.fit(dataset) + + MLTestingUtils.checkCopyAndUids(tvs, tvsModel) + + val parent = tvsModel.bestModel.parent.asInstanceOf[LinearRegression] assert(parent.getRegParam === 0.001) assert(parent.getMaxIter === 10) - assert(cvModel.validationMetrics.length === lrParamMaps.length) + assert(tvsModel.validationMetrics.length === lrParamMaps.length) eval.setMetricName("r2") - val cvModel2 = cv.fit(dataset) - val parent2 = cvModel2.bestModel.parent.asInstanceOf[LinearRegression] + val tvsModel2 = tvs.fit(dataset) + val parent2 = tvsModel2.bestModel.parent.asInstanceOf[LinearRegression] assert(parent2.getRegParam === 0.001) assert(parent2.getMaxIter === 10) - assert(cvModel2.validationMetrics.length === lrParamMaps.length) + assert(tvsModel2.validationMetrics.length === lrParamMaps.length) } - test("validateParams should check estimatorParamMaps") { - import TrainValidationSplitSuite._ + test("transformSchema should check estimatorParamMaps") { + import TrainValidationSplitSuite.{MyEstimator, MyEvaluator} val est = new MyEstimator("est") val eval = new MyEvaluator @@ -92,35 +108,179 @@ class TrainValidationSplitSuite extends SparkFunSuite with MLlibTestSparkContext .addGrid(est.inputCol, Array("input1", "input2")) .build() - val cv = new TrainValidationSplit() + val tvs = new TrainValidationSplit() .setEstimator(est) .setEstimatorParamMaps(paramMaps) .setEvaluator(eval) .setTrainRatio(0.5) - cv.validateParams() // This should pass. + tvs.transformSchema(new StructType()) // This should pass. val invalidParamMaps = paramMaps :+ ParamMap(est.inputCol -> "") - cv.setEstimatorParamMaps(invalidParamMaps) + tvs.setEstimatorParamMaps(invalidParamMaps) intercept[IllegalArgumentException] { - cv.validateParams() + tvs.transformSchema(new StructType()) } } + + test("train validation with parallel evaluation") { + val lr = new LogisticRegression + val lrParamMaps = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.001, 1000.0)) + .addGrid(lr.maxIter, Array(0, 3)) + .build() + val eval = new BinaryClassificationEvaluator + val cv = new TrainValidationSplit() + .setEstimator(lr) + .setEstimatorParamMaps(lrParamMaps) + .setEvaluator(eval) + .setParallelism(1) + val cvSerialModel = cv.fit(dataset) + cv.setParallelism(2) + val cvParallelModel = cv.fit(dataset) + + val serialMetrics = cvSerialModel.validationMetrics.sorted + val parallelMetrics = cvParallelModel.validationMetrics.sorted + assert(serialMetrics === parallelMetrics) + + val parentSerial = cvSerialModel.bestModel.parent.asInstanceOf[LogisticRegression] + val parentParallel = cvParallelModel.bestModel.parent.asInstanceOf[LogisticRegression] + assert(parentSerial.getRegParam === parentParallel.getRegParam) + assert(parentSerial.getMaxIter === parentParallel.getMaxIter) + } + + test("read/write: TrainValidationSplit") { + val lr = new LogisticRegression().setMaxIter(3) + val evaluator = new BinaryClassificationEvaluator() + val paramMaps = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.1, 0.2)) + .build() + val tvs = new TrainValidationSplit() + .setEstimator(lr) + .setEvaluator(evaluator) + .setTrainRatio(0.5) + .setEstimatorParamMaps(paramMaps) + .setSeed(42L) + .setParallelism(2) + + val tvs2 = testDefaultReadWrite(tvs, testParams = false) + + assert(tvs.getTrainRatio === tvs2.getTrainRatio) + assert(tvs.getSeed === tvs2.getSeed) + assert(tvs.getParallelism === tvs2.getParallelism) + + ValidatorParamsSuiteHelpers + .compareParamMaps(tvs.getEstimatorParamMaps, tvs2.getEstimatorParamMaps) + + tvs2.getEstimator match { + case lr2: LogisticRegression => + assert(lr.uid === lr2.uid) + assert(lr.getMaxIter === lr2.getMaxIter) + case other => + throw new AssertionError(s"Loaded TrainValidationSplit expected estimator of type" + + s" LogisticRegression but found ${other.getClass.getName}") + } + } + + test("read/write: TrainValidationSplit with nested estimator") { + val ova = new OneVsRest() + .setClassifier(new LogisticRegression) + val evaluator = new BinaryClassificationEvaluator() + .setMetricName("areaUnderPR") // not default metric + val classifier1 = new LogisticRegression().setRegParam(2.0) + val classifier2 = new LogisticRegression().setRegParam(3.0) + val paramMaps = new ParamGridBuilder() + .addGrid(ova.classifier, Array(classifier1, classifier2)) + .build() + val tvs = new TrainValidationSplit() + .setEstimator(ova) + .setEvaluator(evaluator) + .setTrainRatio(0.5) + .setEstimatorParamMaps(paramMaps) + .setSeed(42L) + + val tvs2 = testDefaultReadWrite(tvs, testParams = false) + + assert(tvs.getTrainRatio === tvs2.getTrainRatio) + assert(tvs.getSeed === tvs2.getSeed) + + tvs2.getEstimator match { + case ova2: OneVsRest => + assert(ova.uid === ova2.uid) + ova2.getClassifier match { + case lr: LogisticRegression => + assert(ova.getClassifier.asInstanceOf[LogisticRegression].getMaxIter + === lr.getMaxIter) + case other => + throw new AssertionError(s"Loaded TrainValidationSplit expected estimator of type" + + s" LogisticRegression but found ${other.getClass.getName}") + } + + case other => + throw new AssertionError(s"Loaded TrainValidationSplit expected estimator of type" + + s" OneVsRest but found ${other.getClass.getName}") + } + + ValidatorParamsSuiteHelpers + .compareParamMaps(tvs.getEstimatorParamMaps, tvs2.getEstimatorParamMaps) + } + + test("read/write: Persistence of nested estimator works if parent directory changes") { + val ova = new OneVsRest() + .setClassifier(new LogisticRegression) + val evaluator = new BinaryClassificationEvaluator() + .setMetricName("areaUnderPR") // not default metric + val classifier1 = new LogisticRegression().setRegParam(2.0) + val classifier2 = new LogisticRegression().setRegParam(3.0) + val paramMaps = new ParamGridBuilder() + .addGrid(ova.classifier, Array(classifier1, classifier2)) + .build() + val tvs = new TrainValidationSplit() + .setEstimator(ova) + .setEvaluator(evaluator) + .setTrainRatio(0.5) + .setEstimatorParamMaps(paramMaps) + .setSeed(42L) + + ValidatorParamsSuiteHelpers.testFileMove(tvs, tempDir) + } + + test("read/write: TrainValidationSplitModel") { + val lr = new LogisticRegression() + .setThreshold(0.6) + val lrModel = new LogisticRegressionModel(lr.uid, Vectors.dense(1.0, 2.0), 1.2) + .setThreshold(0.6) + val evaluator = new BinaryClassificationEvaluator() + val paramMaps = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.1, 0.2)) + .build() + val tvs = new TrainValidationSplitModel("cvUid", lrModel, Array(0.3, 0.6)) + tvs.set(tvs.estimator, lr) + .set(tvs.evaluator, evaluator) + .set(tvs.trainRatio, 0.5) + .set(tvs.estimatorParamMaps, paramMaps) + .set(tvs.seed, 42L) + + val tvs2 = testDefaultReadWrite(tvs, testParams = false) + + assert(tvs.getTrainRatio === tvs2.getTrainRatio) + assert(tvs.validationMetrics === tvs2.validationMetrics) + assert(tvs.getSeed === tvs2.getSeed) + } } -object TrainValidationSplitSuite { +object TrainValidationSplitSuite extends SparkFunSuite{ abstract class MyModel extends Model[MyModel] class MyEstimator(override val uid: String) extends Estimator[MyModel] with HasInputCol { - override def validateParams(): Unit = require($(inputCol).nonEmpty) - - override def fit(dataset: DataFrame): MyModel = { + override def fit(dataset: Dataset[_]): MyModel = { throw new UnsupportedOperationException } override def transformSchema(schema: StructType): StructType = { - throw new UnsupportedOperationException + require($(inputCol).nonEmpty) + schema } override def copy(extra: ParamMap): MyEstimator = defaultCopy(extra) @@ -128,7 +288,7 @@ object TrainValidationSplitSuite { class MyEvaluator extends Evaluator { - override def evaluate(dataset: DataFrame): Double = { + override def evaluate(dataset: Dataset[_]): Double = { throw new UnsupportedOperationException } diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/ValidatorParamsSuiteHelpers.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/ValidatorParamsSuiteHelpers.scala new file mode 100644 index 000000000000..eae1f5adc884 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/ValidatorParamsSuiteHelpers.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.tuning + +import java.io.File +import java.nio.file.{Files, StandardCopyOption} + +import org.scalatest.Assertions + +import org.apache.spark.ml.param.{ParamMap, ParamPair, Params} +import org.apache.spark.ml.util.{Identifiable, MLReader, MLWritable} + +object ValidatorParamsSuiteHelpers extends Assertions { + /** + * Assert sequences of estimatorParamMaps are identical. + * If the values for a parameter are not directly comparable with === + * and are instead Params types themselves then their corresponding paramMaps + * are compared against each other. + */ + def compareParamMaps(pMaps: Array[ParamMap], pMaps2: Array[ParamMap]): Unit = { + assert(pMaps.length === pMaps2.length) + pMaps.zip(pMaps2).foreach { case (pMap, pMap2) => + assert(pMap.size === pMap2.size) + pMap.toSeq.foreach { case ParamPair(p, v) => + assert(pMap2.contains(p)) + val otherParam = pMap2(p) + v match { + case estimator: Params => + otherParam match { + case estimator2: Params => + val estimatorParamMap = Array(estimator.extractParamMap()) + val estimatorParamMap2 = Array(estimator2.extractParamMap()) + compareParamMaps(estimatorParamMap, estimatorParamMap2) + case other => + throw new AssertionError(s"Expected parameter of type Params but" + + s" found ${otherParam.getClass.getName}") + } + case _ => + assert(otherParam === v) + } + } + } + } + + /** + * When nested estimators (ex. OneVsRest) are saved within meta-algorithms such as + * CrossValidator and TrainValidationSplit, relative paths should be used to store + * the path of the estimator so that if the parent directory changes, loading the + * model still works. + */ + def testFileMove[T <: Params with MLWritable](instance: T, tempDir: File): Unit = { + val uid = instance.uid + val subdirName = Identifiable.randomUID("test") + + val subdir = new File(tempDir, subdirName) + val subDirWithUid = new File(subdir, uid) + + instance.save(subDirWithUid.getPath) + + val newSubdirName = Identifiable.randomUID("test_moved") + val newSubdir = new File(tempDir, newSubdirName) + val newSubdirWithUid = new File(newSubdir, uid) + + Files.createDirectory(newSubdir.toPath) + Files.createDirectory(newSubdirWithUid.toPath) + Files.move(subDirWithUid.toPath, newSubdirWithUid.toPath, StandardCopyOption.ATOMIC_MOVE) + + val loader = instance.getClass.getMethod("read").invoke(null).asInstanceOf[MLReader[T]] + val newInstance = loader.load(newSubdirWithUid.getPath) + assert(uid == newInstance.uid) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala b/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.scala new file mode 100644 index 000000000000..4da95e74434e --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/util/DefaultReadWriteTest.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.ml.util + +import java.io.{File, IOException} + +import org.scalatest.Suite + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.param._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.Dataset + +trait DefaultReadWriteTest extends TempDirectory { self: Suite => + + /** + * Checks "overwrite" option and params. + * This saves to and loads from [[tempDir]], but creates a subdirectory with a random name + * in order to avoid conflicts from multiple calls to this method. + * + * @param instance ML instance to test saving/loading + * @param testParams If true, then test values of Params. Otherwise, just test overwrite option. + * @tparam T ML instance type + * @return Instance loaded from file + */ + def testDefaultReadWrite[T <: Params with MLWritable]( + instance: T, + testParams: Boolean = true): T = { + val uid = instance.uid + val subdirName = Identifiable.randomUID("test") + + val subdir = new File(tempDir, subdirName) + val path = new File(subdir, uid).getPath + + instance.save(path) + intercept[IOException] { + instance.save(path) + } + instance.write.overwrite().save(path) + val loader = instance.getClass.getMethod("read").invoke(null).asInstanceOf[MLReader[T]] + val newInstance = loader.load(path) + assert(newInstance.uid === instance.uid) + if (testParams) { + instance.params.foreach { p => + if (instance.isDefined(p)) { + (instance.getOrDefault(p), newInstance.getOrDefault(p)) match { + case (Array(values), Array(newValues)) => + assert(values === newValues, s"Values do not match on param ${p.name}.") + case (value, newValue) => + assert(value === newValue, s"Values do not match on param ${p.name}.") + } + } else { + assert(!newInstance.isDefined(p), s"Param ${p.name} shouldn't be defined.") + } + } + } + + val load = instance.getClass.getMethod("load", classOf[String]) + val another = load.invoke(instance, path).asInstanceOf[T] + assert(another.uid === instance.uid) + another + } + + /** + * Default test for Estimator, Model pairs: + * - Explicitly set Params, and train model + * - Test save/load using `testDefaultReadWrite` on Estimator and Model + * - Check Params on Estimator and Model + * - Compare model data + * + * This requires that `Model`'s `Param`s should be a subset of `Estimator`'s `Param`s. + * + * @param estimator Estimator to test + * @param dataset Dataset to pass to `Estimator.fit()` + * @param testEstimatorParams Set of `Param` values to set in estimator + * @param testModelParams Set of `Param` values to set in model + * @param checkModelData Method which takes the original and loaded `Model` and compares their + * data. This method does not need to check `Param` values. + * @tparam E Type of `Estimator` + * @tparam M Type of `Model` produced by estimator + */ + def testEstimatorAndModelReadWrite[ + E <: Estimator[M] with MLWritable, M <: Model[M] with MLWritable]( + estimator: E, + dataset: Dataset[_], + testEstimatorParams: Map[String, Any], + testModelParams: Map[String, Any], + checkModelData: (M, M) => Unit): Unit = { + // Set some Params to make sure set Params are serialized. + testEstimatorParams.foreach { case (p, v) => + estimator.set(estimator.getParam(p), v) + } + val model = estimator.fit(dataset) + + // Test Estimator save/load + val estimator2 = testDefaultReadWrite(estimator) + testEstimatorParams.foreach { case (p, v) => + val param = estimator.getParam(p) + assert(estimator.get(param).get === estimator2.get(param).get) + } + + // Test Model save/load + val model2 = testDefaultReadWrite(model) + testModelParams.foreach { case (p, v) => + val param = model.getParam(p) + assert(model.get(param).get === model2.get(param).get) + } + + checkModelData(model, model2) + } +} + +class MyParams(override val uid: String) extends Params with MLWritable { + + final val intParamWithDefault: IntParam = new IntParam(this, "intParamWithDefault", "doc") + final val intParam: IntParam = new IntParam(this, "intParam", "doc") + final val floatParam: FloatParam = new FloatParam(this, "floatParam", "doc") + final val doubleParam: DoubleParam = new DoubleParam(this, "doubleParam", "doc") + final val longParam: LongParam = new LongParam(this, "longParam", "doc") + final val stringParam: Param[String] = new Param[String](this, "stringParam", "doc") + final val intArrayParam: IntArrayParam = new IntArrayParam(this, "intArrayParam", "doc") + final val doubleArrayParam: DoubleArrayParam = + new DoubleArrayParam(this, "doubleArrayParam", "doc") + final val stringArrayParam: StringArrayParam = + new StringArrayParam(this, "stringArrayParam", "doc") + + setDefault(intParamWithDefault -> 0) + set(intParam -> 1) + set(floatParam -> 2.0f) + set(doubleParam -> 3.0) + set(longParam -> 4L) + set(stringParam -> "5") + set(intArrayParam -> Array(6, 7)) + set(doubleArrayParam -> Array(8.0, 9.0)) + set(stringArrayParam -> Array("10", "11")) + + override def copy(extra: ParamMap): Params = defaultCopy(extra) + + override def write: MLWriter = new DefaultParamsWriter(this) +} + +object MyParams extends MLReadable[MyParams] { + + override def read: MLReader[MyParams] = new DefaultParamsReader[MyParams] + + override def load(path: String): MyParams = super.load(path) +} + +class DefaultReadWriteSuite extends SparkFunSuite with MLlibTestSparkContext + with DefaultReadWriteTest { + + test("default read/write") { + val myParams = new MyParams("my_params") + testDefaultReadWrite(myParams) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala index d290cc9b06e7..aef81c8c173a 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala @@ -17,14 +17,278 @@ package org.apache.spark.ml.util -import org.apache.spark.ml.Model +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml._ +import org.apache.spark.ml.evaluation.Evaluator +import org.apache.spark.ml.feature.{Instance, LabeledPoint} +import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasLabelCol, HasWeightCol} +import org.apache.spark.ml.recommendation.{ALS, ALSModel} +import org.apache.spark.ml.tree.impl.TreeTests +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ -object MLTestingUtils { - def checkCopy(model: Model[_]): Unit = { +object MLTestingUtils extends SparkFunSuite { + + def checkCopyAndUids[T <: Estimator[_]](estimator: T, model: Model[_]): Unit = { + assert(estimator.uid === model.uid, "Model uid does not match parent estimator") + + // copied model must have the same parent val copied = model.copy(ParamMap.empty) .asInstanceOf[Model[_]] - assert(copied.parent.uid == model.parent.uid) assert(copied.parent == model.parent) + assert(copied.parent.uid == model.parent.uid) + } + + def checkNumericTypes[M <: Model[M], T <: Estimator[M]]( + estimator: T, + spark: SparkSession, + isClassification: Boolean = true)(check: (M, M) => Unit): Unit = { + val dfs = if (isClassification) { + genClassifDFWithNumericLabelCol(spark) + } else { + genRegressionDFWithNumericLabelCol(spark) + } + + val finalEstimator = estimator match { + case weighted: Estimator[M] with HasWeightCol => + weighted.set(weighted.weightCol, "weight") + weighted + case _ => estimator + } + + val expected = finalEstimator.fit(dfs(DoubleType)) + + val actuals = dfs.keys.filter(_ != DoubleType).map { t => + finalEstimator.fit(dfs(t)) + } + + actuals.foreach(actual => check(expected, actual)) + + val dfWithStringLabels = spark.createDataFrame(Seq( + ("0", 1, Vectors.dense(0, 2, 3), 0.0) + )).toDF("label", "weight", "features", "censor") + val thrown = intercept[IllegalArgumentException] { + estimator.fit(dfWithStringLabels) + } + assert(thrown.getMessage.contains( + "Column label must be of type NumericType but was actually of type StringType")) + + estimator match { + case weighted: Estimator[M] with HasWeightCol => + val dfWithStringWeights = spark.createDataFrame(Seq( + (0, "1", Vectors.dense(0, 2, 3), 0.0) + )).toDF("label", "weight", "features", "censor") + weighted.set(weighted.weightCol, "weight") + val thrown = intercept[IllegalArgumentException] { + weighted.fit(dfWithStringWeights) + } + assert(thrown.getMessage.contains( + "Column weight must be of type NumericType but was actually of type StringType")) + case _ => + } + } + + def checkNumericTypesALS( + estimator: ALS, + spark: SparkSession, + column: String, + baseType: NumericType) + (check: (ALSModel, ALSModel) => Unit) + (check2: (ALSModel, ALSModel, DataFrame) => Unit): Unit = { + val dfs = genRatingsDFWithNumericCols(spark, column) + val expected = estimator.fit(dfs(baseType)) + val actuals = dfs.keys.filter(_ != baseType).map(t => (t, estimator.fit(dfs(t)))) + actuals.foreach { case (_, actual) => check(expected, actual) } + actuals.foreach { case (t, actual) => check2(expected, actual, dfs(t)) } + + val baseDF = dfs(baseType) + val others = baseDF.columns.toSeq.diff(Seq(column)).map(col) + val cols = Seq(col(column).cast(StringType)) ++ others + val strDF = baseDF.select(cols: _*) + val thrown = intercept[IllegalArgumentException] { + estimator.fit(strDF) + } + assert(thrown.getMessage.contains( + s"$column must be of type NumericType but was actually of type StringType")) + } + + def checkNumericTypes[T <: Evaluator](evaluator: T, spark: SparkSession): Unit = { + val dfs = genEvaluatorDFWithNumericLabelCol(spark, "label", "prediction") + val expected = evaluator.evaluate(dfs(DoubleType)) + val actuals = dfs.keys.filter(_ != DoubleType).map(t => evaluator.evaluate(dfs(t))) + actuals.foreach(actual => assert(expected === actual)) + + val dfWithStringLabels = spark.createDataFrame(Seq( + ("0", 0d) + )).toDF("label", "prediction") + val thrown = intercept[IllegalArgumentException] { + evaluator.evaluate(dfWithStringLabels) + } + assert(thrown.getMessage.contains( + "Column label must be of type NumericType but was actually of type StringType")) + } + + def genClassifDFWithNumericLabelCol( + spark: SparkSession, + labelColName: String = "label", + featuresColName: String = "features", + weightColName: String = "weight"): Map[NumericType, DataFrame] = { + val df = spark.createDataFrame(Seq( + (0, Vectors.dense(0, 2, 3)), + (1, Vectors.dense(0, 3, 1)), + (0, Vectors.dense(0, 2, 2)), + (1, Vectors.dense(0, 3, 9)), + (0, Vectors.dense(0, 2, 6)) + )).toDF(labelColName, featuresColName) + + val types = + Seq(ShortType, LongType, IntegerType, FloatType, ByteType, DoubleType, DecimalType(10, 0)) + types.map { t => + val castDF = df.select(col(labelColName).cast(t), col(featuresColName)) + t -> TreeTests.setMetadata(castDF, 2, labelColName, featuresColName) + .withColumn(weightColName, round(rand(seed = 42)).cast(t)) + }.toMap + } + + def genRegressionDFWithNumericLabelCol( + spark: SparkSession, + labelColName: String = "label", + weightColName: String = "weight", + featuresColName: String = "features", + censorColName: String = "censor"): Map[NumericType, DataFrame] = { + val df = spark.createDataFrame(Seq( + (1, Vectors.dense(1)), + (2, Vectors.dense(2)), + (3, Vectors.dense(3)), + (4, Vectors.dense(4)) + )).toDF(labelColName, featuresColName) + + val types = + Seq(ShortType, LongType, IntegerType, FloatType, ByteType, DoubleType, DecimalType(10, 0)) + types.map { t => + val castDF = df.select(col(labelColName).cast(t), col(featuresColName)) + t -> TreeTests.setMetadata(castDF, 0, labelColName, featuresColName) + .withColumn(censorColName, lit(0.0)) + .withColumn(weightColName, round(rand(seed = 42)).cast(t)) + }.toMap + } + + def genRatingsDFWithNumericCols( + spark: SparkSession, + column: String): Map[NumericType, DataFrame] = { + val df = spark.createDataFrame(Seq( + (0, 10, 1.0), + (1, 20, 2.0), + (2, 30, 3.0), + (3, 40, 4.0), + (4, 50, 5.0) + )).toDF("user", "item", "rating") + + val others = df.columns.toSeq.diff(Seq(column)).map(col) + val types: Seq[NumericType] = + Seq(ShortType, LongType, IntegerType, FloatType, ByteType, DoubleType, DecimalType(10, 0)) + types.map { t => + val cols = Seq(col(column).cast(t)) ++ others + t -> df.select(cols: _*) + }.toMap + } + + def genEvaluatorDFWithNumericLabelCol( + spark: SparkSession, + labelColName: String = "label", + predictionColName: String = "prediction"): Map[NumericType, DataFrame] = { + val df = spark.createDataFrame(Seq( + (0, 0d), + (1, 1d), + (2, 2d), + (3, 3d), + (4, 4d) + )).toDF(labelColName, predictionColName) + + val types = + Seq(ShortType, LongType, IntegerType, FloatType, ByteType, DoubleType, DecimalType(10, 0)) + types + .map(t => t -> df.select(col(labelColName).cast(t), col(predictionColName))) + .toMap + } + + /** + * Given a DataFrame, generate two output DataFrames: one having the original rows oversampled + * an integer number of times, and one having the original rows but with a column of weights + * proportional to the number of oversampled instances in the oversampled DataFrames. + */ + def genEquivalentOversampledAndWeightedInstances( + data: Dataset[LabeledPoint], + seed: Long): (Dataset[Instance], Dataset[Instance]) = { + import data.sparkSession.implicits._ + val rng = new scala.util.Random(seed) + val sample: () => Int = () => rng.nextInt(10) + 1 + val sampleUDF = udf(sample) + val rawData = data.select("label", "features").withColumn("samples", sampleUDF()) + val overSampledData = rawData.rdd.flatMap { case Row(label: Double, features: Vector, n: Int) => + Iterator.fill(n)(Instance(label, 1.0, features)) + }.toDS() + rng.setSeed(seed) + val weightedData = rawData.rdd.map { case Row(label: Double, features: Vector, n: Int) => + Instance(label, n.toDouble, features) + }.toDS() + (overSampledData, weightedData) + } + + /** + * Helper function for testing sample weights. Tests that oversampling each point is equivalent + * to assigning a sample weight proportional to the number of samples for each point. + */ + def testOversamplingVsWeighting[M <: Model[M], E <: Estimator[M]]( + data: Dataset[LabeledPoint], + estimator: E with HasWeightCol, + modelEquals: (M, M) => Unit, + seed: Long): Unit = { + val (overSampledData, weightedData) = genEquivalentOversampledAndWeightedInstances( + data, seed) + val weightedModel = estimator.set(estimator.weightCol, "weight").fit(weightedData) + val overSampledModel = estimator.set(estimator.weightCol, "").fit(overSampledData) + modelEquals(weightedModel, overSampledModel) + } + + /** + * Helper function for testing sample weights. Tests that injecting a large number of outliers + * with very small sample weights does not affect fitting. The predictor should learn the true + * model despite the outliers. + */ + def testOutliersWithSmallWeights[M <: Model[M], E <: Estimator[M]]( + data: Dataset[LabeledPoint], + estimator: E with HasWeightCol, + numClasses: Int, + modelEquals: (M, M) => Unit, + outlierRatio: Int): Unit = { + import data.sqlContext.implicits._ + val outlierDS = data.withColumn("weight", lit(1.0)).as[Instance].flatMap { + case Instance(l, w, f) => + val outlierLabel = if (numClasses == 0) -l else numClasses - l - 1 + List.fill(outlierRatio)(Instance(outlierLabel, 0.0001, f)) ++ List(Instance(l, w, f)) + } + val trueModel = estimator.set(estimator.weightCol, "").fit(data) + val outlierModel = estimator.set(estimator.weightCol, "weight").fit(outlierDS) + modelEquals(trueModel, outlierModel) + } + + /** + * Helper function for testing sample weights. Tests that giving constant weights to each data + * point yields the same model, regardless of the magnitude of the weight. + */ + def testArbitrarilyScaledWeights[M <: Model[M], E <: Estimator[M]]( + data: Dataset[LabeledPoint], + estimator: E with HasWeightCol, + modelEquals: (M, M) => Unit): Unit = { + estimator.set(estimator.weightCol, "weight") + val models = Seq(0.001, 1.0, 1000.0).map { w => + val df = data.withColumn("weight", lit(w)) + estimator.fit(df) + } + models.sliding(2).foreach { case Seq(m1, m2) => modelEquals(m1, m2)} } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala index 9e6bc7193c13..54e363a8b9f2 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala @@ -60,9 +60,9 @@ class StopwatchSuite extends SparkFunSuite with MLlibTestSparkContext { test("DistributedStopwatch on executors") { val sw = new DistributedStopwatch(sc, "sw") val rdd = sc.parallelize(0 until 4, 4) - val acc = sc.accumulator(0L) + val acc = sc.longAccumulator rdd.foreach { i => - acc += checkStopwatch(sw) + acc.add(checkStopwatch(sw)) } assert(!sw.isRunning) val elapsed = sw.elapsed() @@ -88,12 +88,12 @@ class StopwatchSuite extends SparkFunSuite with MLlibTestSparkContext { assert(sw.toString === s"{\n local: ${localElapsed}ms,\n spark: ${sparkElapsed}ms\n}") val rdd = sc.parallelize(0 until 4, 4) - val acc = sc.accumulator(0L) + val acc = sc.longAccumulator rdd.foreach { i => sw("local").start() val duration = checkStopwatch(sw("spark")) sw("local").stop() - acc += duration + acc.add(duration) } val localElapsed2 = sw("local").elapsed() assert(localElapsed2 === localElapsed) @@ -105,8 +105,8 @@ class StopwatchSuite extends SparkFunSuite with MLlibTestSparkContext { private object StopwatchSuite extends SparkFunSuite { /** - * Checks the input stopwatch on a task that takes a random time (<10ms) to finish. Validates and - * returns the duration reported by the stopwatch. + * Checks the input stopwatch on a task that takes a random time (less than 10ms) to finish. + * Validates and returns the duration reported by the stopwatch. */ def checkStopwatch(sw: Stopwatch): Long = { val ubStart = now diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/TempDirectory.scala b/mllib/src/test/scala/org/apache/spark/ml/util/TempDirectory.scala new file mode 100644 index 000000000000..50b73e0e99a2 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/util/TempDirectory.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.util + +import java.io.File + +import org.scalatest.{BeforeAndAfterAll, Suite} + +import org.apache.spark.util.Utils + +/** + * Trait that creates a temporary directory before all tests and deletes it after all. + */ +trait TempDirectory extends BeforeAndAfterAll { self: Suite => + + private var _tempDir: File = _ + + /** + * Returns the temporary directory as a `File` instance. + */ + protected def tempDir: File = _tempDir + + override def beforeAll(): Unit = { + super.beforeAll() + _tempDir = Utils.createTempDir(namePrefix = this.getClass.getName) + } + + override def afterAll(): Unit = { + try { + Utils.deleteRecursively(_tempDir) + } finally { + super.afterAll() + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala index 59944416d96a..5f85c0d65ff2 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.mllib.api.python import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Vectors, SparseMatrix} -import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, SparseMatrix, Vectors} import org.apache.spark.mllib.recommendation.Rating +import org.apache.spark.mllib.regression.LabeledPoint class PythonMLLibAPISuite extends SparkFunSuite { @@ -72,7 +72,7 @@ class PythonMLLibAPISuite extends SparkFunSuite { assert(matrix === nm) // Test conversion for empty matrix - val empty = Array[Double]() + val empty = Array.empty[Double] val emptyMatrix = Matrices.dense(0, 0, empty) val ne = SerDe.loads(SerDe.dumps(emptyMatrix)).asInstanceOf[DenseMatrix] assert(emptyMatrix == ne) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 8d14bb657215..5cf437776851 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -25,9 +25,11 @@ import org.scalatest.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext} import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils @@ -170,6 +172,37 @@ object LogisticRegressionSuite { class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext with Matchers { + + @transient var binaryDataset: RDD[LabeledPoint] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + /* + Here is the instruction describing how to export the test data into CSV format + so we can validate the training accuracy compared with R's glmnet package. + + val nPoints = 10000 + val coefficients = Array(-0.57997, 0.912083, -0.371077, -0.819866, 2.688191) + val xMean = Array(5.843, 3.057, 3.758, 1.199) + val xVariance = Array(0.6856, 0.1899, 3.116, 0.581) + val data = sc.parallelize(LogisticRegressionSuite.generateMultinomialLogisticInput( + coefficients, xMean, xVariance, true, nPoints, 42), 1) + data.map(x=> x.label + ", " + x.features(0) + ", " + x.features(1) + ", " + + x.features(2) + ", " + x.features(3)).saveAsTextFile("path") + */ + binaryDataset = { + val nPoints = 10000 + val coefficients = Array(-0.57997, 0.912083, -0.371077, -0.819866, 2.688191) + val xMean = Array(5.843, 3.057, 3.758, 1.199) + val xVariance = Array(0.6856, 0.1899, 3.116, 0.581) + + val testData = LogisticRegressionSuite.generateMultinomialLogisticInput( + coefficients, xMean, xVariance, true, nPoints, 42) + + sc.parallelize(testData, 2) + } + } + def validatePrediction( predictions: Seq[Double], input: Seq[LabeledPoint], @@ -215,6 +248,11 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w // Test if we can correctly learn A, B where Y = logistic(A + B*X) test("logistic regression with LBFGS") { + val updaters: List[Updater] = List(new SquaredL2Updater(), new L1Updater()) + updaters.foreach(testLBFGS) + } + + private def testLBFGS(myUpdater: Updater): Unit = { val nPoints = 10000 val A = 2.0 val B = -1.5 @@ -223,7 +261,15 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val lr = new LogisticRegressionWithLBFGS().setIntercept(true) + + // Override the updater + class LogisticRegressionWithLBFGSCustomUpdater + extends LogisticRegressionWithLBFGS { + override val optimizer = + new LBFGS(new LogisticGradient, myUpdater) + } + + val lr = new LogisticRegressionWithLBFGSCustomUpdater().setIntercept(true) val model = lr.run(testRDD) @@ -365,10 +411,10 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w val testRDD1 = sc.parallelize(testData, 2) val testRDD2 = sc.parallelize( - testData.map(x => LabeledPoint(x.label, Vectors.fromBreeze(x.features.toBreeze * 1.0E3))), 2) + testData.map(x => LabeledPoint(x.label, Vectors.fromBreeze(x.features.asBreeze * 1.0E3))), 2) val testRDD3 = sc.parallelize( - testData.map(x => LabeledPoint(x.label, Vectors.fromBreeze(x.features.toBreeze * 1.0E6))), 2) + testData.map(x => LabeledPoint(x.label, Vectors.fromBreeze(x.features.asBreeze * 1.0E6))), 2) testRDD1.cache() testRDD2.cache() @@ -396,10 +442,11 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w assert(modelA1.weights(0) ~== modelA3.weights(0) * 1.0E6 absTol 0.01) // Training data with different scales without feature standardization - // will not yield the same result in the scaled space due to poor - // convergence rate. - assert(modelB1.weights(0) !~== modelB2.weights(0) * 1.0E3 absTol 0.1) - assert(modelB1.weights(0) !~== modelB3.weights(0) * 1.0E6 absTol 0.1) + // should still converge quickly since the model still uses standardization but + // simply modifies the regularization function. See regParamL1Fun and related + // inside of LogisticRegression + assert(modelB1.weights(0) ~== modelB2.weights(0) * 1.0E3 absTol 0.1) + assert(modelB1.weights(0) ~== modelB3.weights(0) * 1.0E6 absTol 0.1) } test("multinomial logistic regression with LBFGS") { @@ -449,7 +496,7 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w * features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) * weights = coef(glmnet(features,label, family="multinomial", alpha = 0, lambda = 0)) * - * The model weights of mutinomial logstic regression in R have `K` set of linear predictors + * The model weights of multinomial logistic regression in R have `K` set of linear predictors * for `K` classes classification problem; however, only `K-1` set is required if the first * outcome is chosen as a "pivot", and the other `K-1` outcomes are separately regressed against * the pivot outcome. This can be done by subtracting the first weights from those `K-1` set @@ -540,6 +587,322 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w } } + /** + * From Spark 2.0, MLlib LogisticRegressionWithLBFGS will call the LogisticRegression + * implementation in ML to train model. We copies test cases from ML to guarantee + * they produce the same result. + */ + test("binary logistic regression with intercept without regularization") { + val trainer1 = new LogisticRegressionWithLBFGS().setIntercept(true).setFeatureScaling(true) + val trainer2 = new LogisticRegressionWithLBFGS().setIntercept(true).setFeatureScaling(false) + + val model1 = trainer1.run(binaryDataset) + val model2 = trainer2.run(binaryDataset) + + /* + Using the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) + coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 0)) + coefficients + + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 2.8366423 + data.V2 -0.5895848 + data.V3 0.8931147 + data.V4 -0.3925051 + data.V5 -0.7996864 + */ + val interceptR = 2.8366423 + val coefficientsR = Vectors.dense(-0.5895848, 0.8931147, -0.3925051, -0.7996864) + + assert(model1.intercept ~== interceptR relTol 1E-3) + assert(model1.weights ~= coefficientsR relTol 1E-3) + + // Without regularization, with or without feature scaling will converge to the same solution. + assert(model2.intercept ~== interceptR relTol 1E-3) + assert(model2.weights ~= coefficientsR relTol 1E-3) + } + + test("binary logistic regression without intercept without regularization") { + val trainer1 = new LogisticRegressionWithLBFGS().setIntercept(false).setFeatureScaling(true) + val trainer2 = new LogisticRegressionWithLBFGS().setIntercept(false).setFeatureScaling(false) + + val model1 = trainer1.run(binaryDataset) + val model2 = trainer2.run(binaryDataset) + + /* + Using the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) + coefficients = + coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 0, intercept=FALSE)) + coefficients + + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V2 -0.3534996 + data.V3 1.2964482 + data.V4 -0.3571741 + data.V5 -0.7407946 + */ + val interceptR = 0.0 + val coefficientsR = Vectors.dense(-0.3534996, 1.2964482, -0.3571741, -0.7407946) + + assert(model1.intercept ~== interceptR relTol 1E-3) + assert(model1.weights ~= coefficientsR relTol 1E-2) + + // Without regularization, with or without feature scaling should converge to the same solution. + assert(model2.intercept ~== interceptR relTol 1E-3) + assert(model2.weights ~= coefficientsR relTol 1E-2) + } + + test("binary logistic regression with intercept with L1 regularization") { + val trainer1 = new LogisticRegressionWithLBFGS().setIntercept(true).setFeatureScaling(true) + trainer1.optimizer.setUpdater(new L1Updater).setRegParam(0.12) + val trainer2 = new LogisticRegressionWithLBFGS().setIntercept(true).setFeatureScaling(false) + trainer2.optimizer.setUpdater(new L1Updater).setRegParam(0.12) + + val model1 = trainer1.run(binaryDataset) + val model2 = trainer2.run(binaryDataset) + + /* + Using the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) + coefficients = coef(glmnet(features,label, family="binomial", alpha = 1, lambda = 0.12)) + coefficients + + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) -0.05627428 + data.V2 . + data.V3 . + data.V4 -0.04325749 + data.V5 -0.02481551 + */ + val interceptR1 = -0.05627428 + val coefficientsR1 = Vectors.dense(0.0, 0.0, -0.04325749, -0.02481551) + + assert(model1.intercept ~== interceptR1 relTol 1E-2) + assert(model1.weights ~= coefficientsR1 absTol 2E-2) + + /* + Using the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) + coefficients = coef(glmnet(features,label, family="binomial", alpha = 1, lambda = 0.12, + standardize=FALSE)) + coefficients + + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 0.3722152 + data.V2 . + data.V3 . + data.V4 -0.1665453 + data.V5 . + */ + val interceptR2 = 0.3722152 + val coefficientsR2 = Vectors.dense(0.0, 0.0, -0.1665453, 0.0) + + assert(model2.intercept ~== interceptR2 relTol 1E-2) + assert(model2.weights ~= coefficientsR2 absTol 1E-3) + } + + test("binary logistic regression without intercept with L1 regularization") { + val trainer1 = new LogisticRegressionWithLBFGS().setIntercept(false).setFeatureScaling(true) + trainer1.optimizer.setUpdater(new L1Updater).setRegParam(0.12) + val trainer2 = new LogisticRegressionWithLBFGS().setIntercept(false).setFeatureScaling(false) + trainer2.optimizer.setUpdater(new L1Updater).setRegParam(0.12) + + val model1 = trainer1.run(binaryDataset) + val model2 = trainer2.run(binaryDataset) + + /* + Using the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) + coefficients = coef(glmnet(features,label, family="binomial", alpha = 1, lambda = 0.12, + intercept=FALSE)) + coefficients + + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V2 . + data.V3 . + data.V4 -0.05189203 + data.V5 -0.03891782 + */ + val interceptR1 = 0.0 + val coefficientsR1 = Vectors.dense(0.0, 0.0, -0.05189203, -0.03891782) + + assert(model1.intercept ~== interceptR1 relTol 1E-3) + assert(model1.weights ~= coefficientsR1 absTol 1E-3) + + /* + Using the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) + coefficients = coef(glmnet(features,label, family="binomial", alpha = 1, lambda = 0.12, + intercept=FALSE, standardize=FALSE)) + coefficients + + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V2 . + data.V3 . + data.V4 -0.08420782 + data.V5 . + */ + val interceptR2 = 0.0 + val coefficientsR2 = Vectors.dense(0.0, 0.0, -0.08420782, 0.0) + + assert(model2.intercept ~== interceptR2 absTol 1E-3) + assert(model2.weights ~= coefficientsR2 absTol 1E-3) + } + + test("binary logistic regression with intercept with L2 regularization") { + val trainer1 = new LogisticRegressionWithLBFGS().setIntercept(true).setFeatureScaling(true) + trainer1.optimizer.setUpdater(new SquaredL2Updater).setRegParam(1.37) + val trainer2 = new LogisticRegressionWithLBFGS().setIntercept(true).setFeatureScaling(false) + trainer2.optimizer.setUpdater(new SquaredL2Updater).setRegParam(1.37) + + val model1 = trainer1.run(binaryDataset) + val model2 = trainer2.run(binaryDataset) + + /* + Using the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) + coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 1.37)) + coefficients + + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 0.15021751 + data.V2 -0.07251837 + data.V3 0.10724191 + data.V4 -0.04865309 + data.V5 -0.10062872 + */ + val interceptR1 = 0.15021751 + val coefficientsR1 = Vectors.dense(-0.07251837, 0.10724191, -0.04865309, -0.10062872) + + assert(model1.intercept ~== interceptR1 relTol 1E-3) + assert(model1.weights ~= coefficientsR1 relTol 1E-3) + + /* + Using the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) + coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 1.37, + standardize=FALSE)) + coefficients + + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) 0.48657516 + data.V2 -0.05155371 + data.V3 0.02301057 + data.V4 -0.11482896 + data.V5 -0.06266838 + */ + val interceptR2 = 0.48657516 + val coefficientsR2 = Vectors.dense(-0.05155371, 0.02301057, -0.11482896, -0.06266838) + + assert(model2.intercept ~== interceptR2 relTol 1E-3) + assert(model2.weights ~= coefficientsR2 relTol 1E-3) + } + + test("binary logistic regression without intercept with L2 regularization") { + val trainer1 = new LogisticRegressionWithLBFGS().setIntercept(false).setFeatureScaling(true) + trainer1.optimizer.setUpdater(new SquaredL2Updater).setRegParam(1.37) + val trainer2 = new LogisticRegressionWithLBFGS().setIntercept(false).setFeatureScaling(false) + trainer2.optimizer.setUpdater(new SquaredL2Updater).setRegParam(1.37) + + val model1 = trainer1.run(binaryDataset) + val model2 = trainer2.run(binaryDataset) + + /* + Using the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) + coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 1.37, + intercept=FALSE)) + coefficients + + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V2 -0.06099165 + data.V3 0.12857058 + data.V4 -0.04708770 + data.V5 -0.09799775 + */ + val interceptR1 = 0.0 + val coefficientsR1 = Vectors.dense(-0.06099165, 0.12857058, -0.04708770, -0.09799775) + + assert(model1.intercept ~== interceptR1 absTol 1E-3) + assert(model1.weights ~= coefficientsR1 relTol 1E-2) + + /* + Using the following R code to load the data and train the model using glmnet package. + + library("glmnet") + data <- read.csv("path", header=FALSE) + label = factor(data$V1) + features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) + coefficients = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 1.37, + intercept=FALSE, standardize=FALSE)) + coefficients + + 5 x 1 sparse Matrix of class "dgCMatrix" + s0 + (Intercept) . + data.V2 -0.005679651 + data.V3 0.048967094 + data.V4 -0.093714016 + data.V5 -0.053314311 + */ + val interceptR2 = 0.0 + val coefficientsR2 = Vectors.dense(-0.005679651, 0.048967094, -0.093714016, -0.053314311) + + assert(model2.intercept ~== interceptR2 absTol 1E-3) + assert(model2.weights ~= coefficientsR2 relTol 1E-2) + } + } class LogisticRegressionClusterSuite extends SparkFunSuite with LocalClusterSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index cffa1ab700f8..5ec4c15387e9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -21,6 +21,7 @@ import scala.util.Random import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, Vector => BV} import breeze.stats.distributions.{Multinomial => BrzMultinomial} +import org.scalatest.exceptions.TestFailedException import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -103,17 +104,24 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { piData: Array[Double], thetaData: Array[Array[Double]], model: NaiveBayesModel): Unit = { - def closeFit(d1: Double, d2: Double, precision: Double): Boolean = { - (d1 - d2).abs <= precision - } - val modelIndex = (0 until piData.length).zip(model.labels.map(_.toInt)) - for (i <- modelIndex) { - assert(closeFit(math.exp(piData(i._2)), math.exp(model.pi(i._1)), 0.05)) - } - for (i <- modelIndex) { - for (j <- 0 until thetaData(i._2).length) { - assert(closeFit(math.exp(thetaData(i._2)(j)), math.exp(model.theta(i._1)(j)), 0.05)) + val modelIndex = piData.indices.zip(model.labels.map(_.toInt)) + try { + for (i <- modelIndex) { + assert(math.exp(piData(i._2)) ~== math.exp(model.pi(i._1)) absTol 0.05) + for (j <- thetaData(i._2).indices) { + assert(math.exp(thetaData(i._2)(j)) ~== math.exp(model.theta(i._1)(j)) absTol 0.05) + } } + } catch { + case e: TestFailedException => + def arr2str(a: Array[Double]): String = a.mkString("[", ", ", "]") + def msg(orig: String): String = orig + "\nvalidateModelFit:\n" + + " piData: " + arr2str(piData) + "\n" + + " thetaData: " + thetaData.map(arr2str).mkString("\n") + "\n" + + " model.labels: " + arr2str(model.labels) + "\n" + + " model.pi: " + arr2str(model.pi) + "\n" + + " model.theta: " + model.theta.map(arr2str).mkString("\n") + throw e.modifyMessage(_.map(msg)) } } @@ -174,7 +182,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { val piVector = new BDV(model.pi) // model.theta is row-major; treat it as col-major representation of transpose, and transpose: val thetaMatrix = new BDM(model.theta(0).length, model.theta.length, model.theta.flatten).t - val logClassProbs: BV[Double] = piVector + (thetaMatrix * testData.toBreeze) + val logClassProbs: BV[Double] = piVector + (thetaMatrix * testData.asBreeze) val classProbs = logClassProbs.toArray.map(math.exp) val classProbsSum = classProbs.sum classProbs.map(_ / classProbsSum) @@ -226,7 +234,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { val thetaMatrix = new BDM(model.theta(0).length, model.theta.length, model.theta.flatten).t val negThetaMatrix = new BDM(model.theta(0).length, model.theta.length, model.theta.flatten.map(v => math.log(1.0 - math.exp(v)))).t - val testBreeze = testData.toBreeze + val testBreeze = testData.asBreeze val negTestBreeze = new BDV(Array.fill(testBreeze.size)(1.0)) - testBreeze val piTheta: BV[Double] = piVector + (thetaMatrix * testBreeze) val logClassProbs: BV[Double] = piTheta + (negThetaMatrix * negTestBreeze) @@ -299,7 +307,7 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { val tempDir = Utils.createTempDir() val path = tempDir.toURI.toString - Seq(NaiveBayesSuite.binaryBernoulliModel, NaiveBayesSuite.binaryMultinomialModel).map { + Seq(NaiveBayesSuite.binaryBernoulliModel, NaiveBayesSuite.binaryMultinomialModel).foreach { model => // Save model, load it back, and compare. try { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index ee3c85d09a46..3676d9c5debc 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.classification import scala.collection.JavaConverters._ import scala.util.Random -import org.jblas.DoubleMatrix +import breeze.linalg.{DenseVector => BDV} import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.mllib.linalg.Vectors @@ -45,12 +45,11 @@ object SVMSuite { nPoints: Int, seed: Int): Seq[LabeledPoint] = { val rnd = new Random(seed) - val weightsMat = new DoubleMatrix(1, weights.length, weights : _*) + val weightsMat = new BDV(weights) val x = Array.fill[Array[Double]](nPoints)( Array.fill[Double](weights.length)(rnd.nextDouble() * 2.0 - 1.0)) val y = x.map { xi => - val yD = new DoubleMatrix(1, xi.length, xi: _*).dot(weightsMat) + - intercept + 0.01 * rnd.nextGaussian() + val yD = new BDV(xi).dot(weightsMat) + intercept + 0.01 * rnd.nextGaussian() if (yD < 0) 0.0 else 1.0 } y.zip(x).map(p => LabeledPoint(p._1, Vectors.dense(p._2))) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala index d7b291d5a633..5f797a60f09e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala @@ -23,8 +23,8 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.TestingUtils._ -import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} +import org.apache.spark.streaming.dstream.DStream class StreamingLogisticRegressionSuite extends SparkFunSuite with TestSuiteBase { @@ -95,7 +95,7 @@ class StreamingLogisticRegressionSuite extends SparkFunSuite with TestSuiteBase // (we add a count to ensure the result is a DStream) ssc = setupStreams(input, (inputDStream: DStream[LabeledPoint]) => { model.trainOn(inputDStream) - inputDStream.foreachRDD(x => history.append(math.abs(model.latestModel().weights(0) - B))) + inputDStream.foreachRDD(x => history += math.abs(model.latestModel().weights(0) - B)) inputDStream.count() }) runStreams(ssc, numBatches, numBatches) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala new file mode 100644 index 000000000000..35f7932ae822 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.clustering + +import org.apache.spark.SparkFunSuite +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.util.Utils + +class BisectingKMeansSuite extends SparkFunSuite with MLlibTestSparkContext { + + test("default values") { + val bkm0 = new BisectingKMeans() + assert(bkm0.getK === 4) + assert(bkm0.getMaxIterations === 20) + assert(bkm0.getMinDivisibleClusterSize === 1.0) + val bkm1 = new BisectingKMeans() + assert(bkm0.getSeed === bkm1.getSeed, "The default seed should be constant.") + } + + test("setter/getter") { + val bkm = new BisectingKMeans() + + val k = 10 + assert(bkm.getK !== k) + assert(bkm.setK(k).getK === k) + val maxIter = 100 + assert(bkm.getMaxIterations !== maxIter) + assert(bkm.setMaxIterations(maxIter).getMaxIterations === maxIter) + val minSize = 2.0 + assert(bkm.getMinDivisibleClusterSize !== minSize) + assert(bkm.setMinDivisibleClusterSize(minSize).getMinDivisibleClusterSize === minSize) + val seed = 10L + assert(bkm.getSeed !== seed) + assert(bkm.setSeed(seed).getSeed === seed) + + intercept[IllegalArgumentException] { + bkm.setK(0) + } + intercept[IllegalArgumentException] { + bkm.setMaxIterations(0) + } + intercept[IllegalArgumentException] { + bkm.setMinDivisibleClusterSize(0.0) + } + } + + test("1D data") { + val points = Vectors.sparse(1, Array.empty, Array.empty) +: + (1 until 8).map(i => Vectors.dense(i)) + val data = sc.parallelize(points, 2) + val bkm = new BisectingKMeans() + .setK(4) + .setMaxIterations(1) + .setSeed(1L) + // The clusters should be + // (0, 1, 2, 3, 4, 5, 6, 7) + // - (0, 1, 2, 3) + // - (0, 1) + // - (2, 3) + // - (4, 5, 6, 7) + // - (4, 5) + // - (6, 7) + val model = bkm.run(data) + assert(model.k === 4) + // The total cost should be 8 * 0.5 * 0.5 = 2.0. + assert(model.computeCost(data) ~== 2.0 relTol 1e-12) + val predictions = data.map(v => (v(0), model.predict(v))).collectAsMap() + Range(0, 8, 2).foreach { i => + assert(predictions(i) === predictions(i + 1), + s"$i and ${i + 1} should belong to the same cluster.") + } + val root = model.root + assert(root.center(0) ~== 3.5 relTol 1e-12) + assert(root.height ~== 2.0 relTol 1e-12) + assert(root.children.length === 2) + assert(root.children(0).height ~== 1.0 relTol 1e-12) + assert(root.children(1).height ~== 1.0 relTol 1e-12) + } + + test("points are the same") { + val data = sc.parallelize(Seq.fill(8)(Vectors.dense(1.0, 1.0)), 2) + val bkm = new BisectingKMeans() + .setK(2) + .setMaxIterations(1) + .setSeed(1L) + val model = bkm.run(data) + assert(model.k === 1) + } + + test("more desired clusters than points") { + val data = sc.parallelize(Seq.tabulate(4)(i => Vectors.dense(i)), 2) + val bkm = new BisectingKMeans() + .setK(8) + .setMaxIterations(2) + .setSeed(1L) + val model = bkm.run(data) + assert(model.k === 4) + } + + test("min divisible cluster") { + val data = sc.parallelize( + Seq.tabulate(16)(i => Vectors.dense(i)) ++ Seq.tabulate(4)(i => Vectors.dense(-100.0 - i)), + 2) + val bkm = new BisectingKMeans() + .setK(4) + .setMinDivisibleClusterSize(10) + .setMaxIterations(1) + .setSeed(1L) + val model = bkm.run(data) + assert(model.k === 3) + assert(model.predict(Vectors.dense(-100)) === model.predict(Vectors.dense(-97))) + assert(model.predict(Vectors.dense(7)) !== model.predict(Vectors.dense(8))) + + bkm.setMinDivisibleClusterSize(0.5) + val sameModel = bkm.run(data) + assert(sameModel.k === 3) + } + + test("larger clusters get selected first") { + val data = sc.parallelize( + Seq.tabulate(16)(i => Vectors.dense(i)) ++ Seq.tabulate(4)(i => Vectors.dense(-100.0 - i)), + 2) + val bkm = new BisectingKMeans() + .setK(3) + .setMaxIterations(1) + .setSeed(1L) + val model = bkm.run(data) + assert(model.k === 3) + assert(model.predict(Vectors.dense(-100)) === model.predict(Vectors.dense(-97))) + assert(model.predict(Vectors.dense(7)) !== model.predict(Vectors.dense(8))) + } + + test("2D data") { + val points = Seq( + (11, 10), (9, 10), (10, 9), (10, 11), + (11, -10), (9, -10), (10, -9), (10, -11), + (0, 1), (0, -1) + ).map { case (x, y) => + if (x == 0) { + Vectors.sparse(2, Array(1), Array(y)) + } else { + Vectors.dense(x, y) + } + } + val data = sc.parallelize(points, 2) + val bkm = new BisectingKMeans() + .setK(3) + .setMaxIterations(4) + .setSeed(1L) + val model = bkm.run(data) + assert(model.k === 3) + assert(model.root.center ~== Vectors.dense(8, 0) relTol 1e-12) + model.root.leafNodes.foreach { node => + if (node.center(0) < 5) { + assert(node.size === 2) + assert(node.center ~== Vectors.dense(0, 0) relTol 1e-12) + } else if (node.center(1) > 0) { + assert(node.size === 4) + assert(node.center ~== Vectors.dense(10, 10) relTol 1e-12) + } else { + assert(node.size === 4) + assert(node.center ~== Vectors.dense(10, -10) relTol 1e-12) + } + } + } + + test("BisectingKMeans model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + val points = (1 until 8).map(i => Vectors.dense(i)) + val data = sc.parallelize(points, 2) + val model = new BisectingKMeans().run(data) + try { + model.save(sc, path) + val sameModel = BisectingKMeansModel.load(sc, path) + assert(model.k === sameModel.k) + model.clusterCenters.zip(sameModel.clusterCenters).foreach(c => c._1 === c._2) + } finally { + Utils.deleteRecursively(tempDir) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala index a72723eb00da..11189d8bd477 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala @@ -18,13 +18,27 @@ package org.apache.spark.mllib.clustering import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{Vector, Vectors, Matrices} +import org.apache.spark.mllib.linalg.{Matrices, Vector, Vectors} import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.Utils class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext { + + test("gmm fails on high dimensional data") { + val rdd = sc.parallelize(Seq( + Vectors.sparse(GaussianMixture.MAX_NUM_FEATURES + 1, Array(0, 4), Array(3.0, 8.0)), + Vectors.sparse(GaussianMixture.MAX_NUM_FEATURES + 1, Array(1, 5), Array(4.0, 9.0)))) + val gm = new GaussianMixture() + withClue(s"GMM should restrict the maximum number of features to be < " + + s"${GaussianMixture.MAX_NUM_FEATURES}") { + intercept[IllegalArgumentException] { + gm.run(rdd) + } + } + } + test("single cluster") { val data = sc.parallelize(Array( Vectors.dense(6.0, 9.0), @@ -182,7 +196,7 @@ class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext { Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734) ) - val data2: Array[Vector] = Array.tabulate(25){ i: Int => + val data2: Array[Vector] = Array.tabulate(25) { i: Int => Vectors.dense(Array.tabulate(50)(i + _.toDouble)) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 3003c62d9876..48bd41dc3e3b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -29,6 +29,8 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { import org.apache.spark.mllib.clustering.KMeans.{K_MEANS_PARALLEL, RANDOM} + private val seed = 42 + test("single cluster") { val data = sc.parallelize(Array( Vectors.dense(1.0, 2.0, 6.0), @@ -38,7 +40,7 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { val center = Vectors.dense(1.0, 3.0, 4.0) - // No matter how many runs or iterations we use, we should get one cluster, + // No matter how many iterations we use, we should get one cluster, // centered at the mean of the points var model = KMeans.train(data, k = 1, maxIterations = 1) @@ -50,44 +52,72 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { model = KMeans.train(data, k = 1, maxIterations = 5) assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) - assert(model.clusterCenters.head ~== center absTol 1E-5) - - model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) - assert(model.clusterCenters.head ~== center absTol 1E-5) - - model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) + model = KMeans.train(data, k = 1, maxIterations = 1, initializationMode = RANDOM) assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train( - data, k = 1, maxIterations = 1, runs = 1, initializationMode = K_MEANS_PARALLEL) + data, k = 1, maxIterations = 1, initializationMode = K_MEANS_PARALLEL) assert(model.clusterCenters.head ~== center absTol 1E-5) } - test("no distinct points") { + test("fewer distinct points than clusters") { val data = sc.parallelize( Array( Vectors.dense(1.0, 2.0, 3.0), Vectors.dense(1.0, 2.0, 3.0), Vectors.dense(1.0, 2.0, 3.0)), 2) - val center = Vectors.dense(1.0, 2.0, 3.0) - // Make sure code runs. - var model = KMeans.train(data, k = 2, maxIterations = 1) - assert(model.clusterCenters.size === 2) - } + var model = KMeans.train(data, k = 2, maxIterations = 1, initializationMode = "random") + assert(model.clusterCenters.length === 1) - test("more clusters than points") { - val data = sc.parallelize( - Array( - Vectors.dense(1.0, 2.0, 3.0), - Vectors.dense(1.0, 3.0, 4.0)), - 2) + model = KMeans.train(data, k = 2, maxIterations = 1, initializationMode = "k-means||") + assert(model.clusterCenters.length === 1) + } - // Make sure code runs. - var model = KMeans.train(data, k = 3, maxIterations = 1) - assert(model.clusterCenters.size === 3) + test("unique cluster centers") { + val rng = new Random(seed) + val numDistinctPoints = 10 + val points = (0 until numDistinctPoints).map(i => Vectors.dense(Array.fill(3)(rng.nextDouble))) + val data = sc.parallelize(points.flatMap(Array.fill(1 + rng.nextInt(3))(_)), 2) + val normedData = data.map(new VectorWithNorm(_)) + + // less centers than k + val km = new KMeans().setK(50) + .setMaxIterations(5) + .setInitializationMode("k-means||") + .setInitializationSteps(10) + .setSeed(seed) + val initialCenters = km.initKMeansParallel(normedData).map(_.vector) + assert(initialCenters.length === initialCenters.distinct.length) + assert(initialCenters.length <= numDistinctPoints) + + val model = km.run(data) + val finalCenters = model.clusterCenters + assert(finalCenters.length === finalCenters.distinct.length) + + // run local k-means + val k = 10 + val km2 = new KMeans().setK(k) + .setMaxIterations(5) + .setInitializationMode("k-means||") + .setInitializationSteps(10) + .setSeed(seed) + val initialCenters2 = km2.initKMeansParallel(normedData).map(_.vector) + assert(initialCenters2.length === initialCenters2.distinct.length) + assert(initialCenters2.length === k) + + val model2 = km2.run(data) + val finalCenters2 = model2.clusterCenters + assert(finalCenters2.length === finalCenters2.distinct.length) + + val km3 = new KMeans().setK(k) + .setMaxIterations(5) + .setInitializationMode("random") + .setSeed(seed) + val model3 = km3.run(data) + val finalCenters3 = model3.clusterCenters + assert(finalCenters3.length === finalCenters3.distinct.length) } test("deterministic initialization") { @@ -97,12 +127,12 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { for (initMode <- Seq(RANDOM, K_MEANS_PARALLEL)) { // Create three deterministic models and compare cluster means - val model1 = KMeans.train(rdd, k = 10, maxIterations = 2, runs = 1, - initializationMode = initMode, seed = 42) + val model1 = KMeans.train(rdd, k = 10, maxIterations = 2, + initializationMode = initMode, seed = seed) val centers1 = model1.clusterCenters - val model2 = KMeans.train(rdd, k = 10, maxIterations = 2, runs = 1, - initializationMode = initMode, seed = 42) + val model2 = KMeans.train(rdd, k = 10, maxIterations = 2, + initializationMode = initMode, seed = seed) val centers2 = model2.clusterCenters centers1.zip(centers2).foreach { case (c1, c2) => @@ -119,7 +149,7 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { ) val data = sc.parallelize((1 to 100).flatMap(_ => smallData), 4) - // No matter how many runs or iterations we use, we should get one cluster, + // No matter how many iterations we use, we should get one cluster, // centered at the mean of the points val center = Vectors.dense(1.0, 3.0, 4.0) @@ -134,17 +164,10 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { model = KMeans.train(data, k = 1, maxIterations = 5) assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) + model = KMeans.train(data, k = 1, maxIterations = 1, initializationMode = RANDOM) assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) - assert(model.clusterCenters.head ~== center absTol 1E-5) - - model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) - assert(model.clusterCenters.head ~== center absTol 1E-5) - - model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, - initializationMode = K_MEANS_PARALLEL) + model = KMeans.train(data, k = 1, maxIterations = 1, initializationMode = K_MEANS_PARALLEL) assert(model.clusterCenters.head ~== center absTol 1E-5) } @@ -165,7 +188,7 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { data.persist() - // No matter how many runs or iterations we use, we should get one cluster, + // No matter how many iterations we use, we should get one cluster, // centered at the mean of the points val center = Vectors.sparse(n, Seq((0, 1.0), (1, 3.0), (2, 4.0))) @@ -179,17 +202,10 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { model = KMeans.train(data, k = 1, maxIterations = 5) assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) + model = KMeans.train(data, k = 1, maxIterations = 1, initializationMode = RANDOM) assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) - assert(model.clusterCenters.head ~== center absTol 1E-5) - - model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) - assert(model.clusterCenters.head ~== center absTol 1E-5) - - model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, - initializationMode = K_MEANS_PARALLEL) + model = KMeans.train(data, k = 1, maxIterations = 1, initializationMode = K_MEANS_PARALLEL) assert(model.clusterCenters.head ~== center absTol 1E-5) data.unpersist() @@ -230,11 +246,6 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { model = KMeans.train(rdd, k = 5, maxIterations = 10) assert(model.clusterCenters.sortBy(VectorWithCompare(_)) .zip(points.sortBy(VectorWithCompare(_))).forall(x => x._1 ~== (x._2) absTol 1E-5)) - - // Neither should more runs - model = KMeans.train(rdd, k = 5, maxIterations = 10, runs = 5) - assert(model.clusterCenters.sortBy(VectorWithCompare(_)) - .zip(points.sortBy(VectorWithCompare(_))).forall(x => x._1 ~== (x._2) absTol 1E-5)) } test("two clusters") { @@ -250,7 +261,7 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { for (initMode <- Seq(RANDOM, K_MEANS_PARALLEL)) { // Two iterations are sufficient no matter where the initial centers are. - val model = KMeans.train(rdd, k = 2, maxIterations = 2, runs = 1, initMode) + val model = KMeans.train(rdd, k = 2, maxIterations = 2, initMode) val predicts = model.predict(rdd).collect() @@ -304,11 +315,10 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { object KMeansSuite extends SparkFunSuite { def createModel(dim: Int, k: Int, isSparse: Boolean): KMeansModel = { - val singlePoint = isSparse match { - case true => - Vectors.sparse(dim, Array.empty[Int], Array.empty[Double]) - case _ => - Vectors.dense(Array.fill[Double](dim)(0.0)) + val singlePoint = if (isSparse) { + Vectors.sparse(dim, Array.empty[Int], Array.empty[Double]) + } else { + Vectors.dense(Array.fill[Double](dim)(0.0)) } new KMeansModel(Array.fill[Vector](k)(singlePoint)) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index 37fb69d68f6b..8906e52faebe 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.clustering import java.util.{ArrayList => JArrayList} -import breeze.linalg.{DenseMatrix => BDM, argtopk, max, argmax} +import breeze.linalg.{argmax, argtopk, max, DenseMatrix => BDM} import org.apache.spark.SparkFunSuite import org.apache.spark.graphx.Edge @@ -116,10 +116,10 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { case (docId, (topicDistribution, (indices, weights))) => assert(indices.length == 2) assert(weights.length == 2) - val bdvTopicDist = topicDistribution.toBreeze + val bdvTopicDist = topicDistribution.asBreeze val top2Indices = argtopk(bdvTopicDist, 2) - assert(top2Indices.toArray === indices) - assert(bdvTopicDist(top2Indices).toArray === weights) + assert(top2Indices.toSet === indices.toSet) + assert(bdvTopicDist(top2Indices).toArray.toSet === weights.toSet) } // Check: log probabilities @@ -151,7 +151,7 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { // Check: topTopicAssignments // Make sure it assigns a topic to each term appearing in each doc. val topTopicAssignments: Map[Long, (Array[Int], Array[Int])] = - model.topicAssignments.collect().map(x => x._1 -> (x._2, x._3)).toMap + model.topicAssignments.collect().map(x => x._1 -> ((x._2, x._3))).toMap assert(topTopicAssignments.keys.max < tinyCorpus.length) tinyCorpus.foreach { case (docID: Long, doc: Vector) => if (topTopicAssignments.contains(docID)) { @@ -366,17 +366,26 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { (0, 0.99504), (1, 0.99504), (1, 0.99504), (1, 0.99504)) - val actualPredictions = ldaModel.topicDistributions(docs).map { case (id, topics) => + val actualPredictions = ldaModel.topicDistributions(docs).cache() + val topTopics = actualPredictions.map { case (id, topics) => // convert results to expectedPredictions format, which only has highest probability topic - val topicsBz = topics.toBreeze.toDenseVector + val topicsBz = topics.asBreeze.toDenseVector (id, (argmax(topicsBz), max(topicsBz))) }.sortByKey() .values .collect() - expectedPredictions.zip(actualPredictions).forall { case (expected, actual) => - expected._1 === actual._1 && (expected._2 ~== actual._2 relTol 1E-3D) + expectedPredictions.zip(topTopics).foreach { case (expected, actual) => + assert(expected._1 === actual._1 && (expected._2 ~== actual._2 relTol 1E-3D)) } + + docs.collect() + .map(doc => ldaModel.topicDistribution(doc._2)) + .zip(actualPredictions.map(_._2).collect()) + .foreach { case (single, batch) => + assert(single ~== batch relTol 1E-3D) + } + actualPredictions.unpersist() } test("OnlineLDAOptimizer with asymmetric prior") { @@ -496,6 +505,8 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { assert(distributedModel.topicConcentration === sameDistributedModel.topicConcentration) assert(distributedModel.gammaShape === sameDistributedModel.gammaShape) assert(distributedModel.globalTopicTotals === sameDistributedModel.globalTopicTotals) + assert(distributedModel.logLikelihood ~== sameDistributedModel.logLikelihood absTol 1e-6) + assert(distributedModel.logPrior ~== sameDistributedModel.logPrior absTol 1e-6) val graph = distributedModel.graph val sameGraph = sameDistributedModel.graph diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala index 189000512155..b33b86b39a42 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala @@ -30,90 +30,91 @@ class PowerIterationClusteringSuite extends SparkFunSuite with MLlibTestSparkCon import org.apache.spark.mllib.clustering.PowerIterationClustering._ + /** Generates a circle of points. */ + private def genCircle(r: Double, n: Int): Array[(Double, Double)] = { + Array.tabulate(n) { i => + val theta = 2.0 * math.Pi * i / n + (r * math.cos(theta), r * math.sin(theta)) + } + } + + /** Computes Gaussian similarity. */ + private def sim(x: (Double, Double), y: (Double, Double)): Double = { + val dist2 = (x._1 - y._1) * (x._1 - y._1) + (x._2 - y._2) * (x._2 - y._2) + math.exp(-dist2 / 2.0) + } + test("power iteration clustering") { - /* - We use the following graph to test PIC. All edges are assigned similarity 1.0 except 0.1 for - edge (3, 4). - - 15-14 -13 -12 - | | - 4 . 3 - 2 11 - | | x | | - 5 0 - 1 10 - | | - 6 - 7 - 8 - 9 - */ + // Generate two circles following the example in the PIC paper. + val r1 = 1.0 + val n1 = 10 + val r2 = 4.0 + val n2 = 10 + val n = n1 + n2 + val points = genCircle(r1, n1) ++ genCircle(r2, n2) + val similarities = for (i <- 1 until n; j <- 0 until i) yield { + (i.toLong, j.toLong, sim(points(i), points(j))) + } - val similarities = Seq[(Long, Long, Double)]((0, 1, 1.0), (0, 2, 1.0), (0, 3, 1.0), (1, 2, 1.0), - (1, 3, 1.0), (2, 3, 1.0), (3, 4, 0.1), // (3, 4) is a weak edge - (4, 5, 1.0), (4, 15, 1.0), (5, 6, 1.0), (6, 7, 1.0), (7, 8, 1.0), (8, 9, 1.0), (9, 10, 1.0), - (10, 11, 1.0), (11, 12, 1.0), (12, 13, 1.0), (13, 14, 1.0), (14, 15, 1.0)) val model = new PowerIterationClustering() .setK(2) + .setMaxIterations(40) .run(sc.parallelize(similarities, 2)) val predictions = Array.fill(2)(mutable.Set.empty[Long]) model.assignments.collect().foreach { a => predictions(a.cluster) += a.id } - assert(predictions.toSet == Set((0 to 3).toSet, (4 to 15).toSet)) + assert(predictions.toSet == Set((0 until n1).toSet, (n1 until n).toSet)) val model2 = new PowerIterationClustering() .setK(2) + .setMaxIterations(10) .setInitializationMode("degree") .run(sc.parallelize(similarities, 2)) val predictions2 = Array.fill(2)(mutable.Set.empty[Long]) model2.assignments.collect().foreach { a => predictions2(a.cluster) += a.id } - assert(predictions2.toSet == Set((0 to 3).toSet, (4 to 15).toSet)) + assert(predictions2.toSet == Set((0 until n1).toSet, (n1 until n).toSet)) } test("power iteration clustering on graph") { - /* - We use the following graph to test PIC. All edges are assigned similarity 1.0 except 0.1 for - edge (3, 4). - - 15-14 -13 -12 - | | - 4 . 3 - 2 11 - | | x | | - 5 0 - 1 10 - | | - 6 - 7 - 8 - 9 - */ - - val similarities = Seq[(Long, Long, Double)]((0, 1, 1.0), (0, 2, 1.0), (0, 3, 1.0), (1, 2, 1.0), - (1, 3, 1.0), (2, 3, 1.0), (3, 4, 0.1), // (3, 4) is a weak edge - (4, 5, 1.0), (4, 15, 1.0), (5, 6, 1.0), (6, 7, 1.0), (7, 8, 1.0), (8, 9, 1.0), (9, 10, 1.0), - (10, 11, 1.0), (11, 12, 1.0), (12, 13, 1.0), (13, 14, 1.0), (14, 15, 1.0)) + // Generate two circles following the example in the PIC paper. + val r1 = 1.0 + val n1 = 10 + val r2 = 4.0 + val n2 = 10 + val n = n1 + n2 + val points = genCircle(r1, n1) ++ genCircle(r2, n2) + val similarities = for (i <- 1 until n; j <- 0 until i) yield { + (i.toLong, j.toLong, sim(points(i), points(j))) + } val edges = similarities.flatMap { case (i, j, s) => - if (i != j) { - Seq(Edge(i, j, s), Edge(j, i, s)) - } else { - None - } + Seq(Edge(i, j, s), Edge(j, i, s)) } val graph = Graph.fromEdges(sc.parallelize(edges, 2), 0.0) val model = new PowerIterationClustering() .setK(2) + .setMaxIterations(40) .run(graph) val predictions = Array.fill(2)(mutable.Set.empty[Long]) model.assignments.collect().foreach { a => predictions(a.cluster) += a.id } - assert(predictions.toSet == Set((0 to 3).toSet, (4 to 15).toSet)) + assert(predictions.toSet == Set((0 until n1).toSet, (n1 until n).toSet)) val model2 = new PowerIterationClustering() .setK(2) + .setMaxIterations(10) .setInitializationMode("degree") .run(sc.parallelize(similarities, 2)) val predictions2 = Array.fill(2)(mutable.Set.empty[Long]) model2.assignments.collect().foreach { a => predictions2(a.cluster) += a.id } - assert(predictions2.toSet == Set((0 to 3).toSet, (4 to 15).toSet)) + assert(predictions2.toSet == Set((0 until n1).toSet, (n1 until n).toSet)) } test("normalize and powerIter") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala index 3645d29dccdb..fdaa098345d1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala @@ -67,7 +67,7 @@ class StreamingKMeansSuite extends SparkFunSuite with TestSuiteBase { // estimated center from streaming should exactly match the arithmetic mean of all data points // because the decay factor is set to 1.0 val grandMean = - input.flatten.map(x => x.toBreeze).reduce(_ + _) / (numBatches * numPoints).toDouble + input.flatten.map(x => x.asBreeze).reduce(_ + _) / (numBatches * numPoints).toDouble assert(model.latestModel().clusterCenters(0) ~== Vectors.dense(grandMean.toArray) absTol 1E-5) } @@ -98,9 +98,16 @@ class StreamingKMeansSuite extends SparkFunSuite with TestSuiteBase { runStreams(ssc, numBatches, numBatches) // check that estimated centers are close to true centers - // NOTE exact assignment depends on the initialization! - assert(centers(0) ~== kMeans.latestModel().clusterCenters(0) absTol 1E-1) - assert(centers(1) ~== kMeans.latestModel().clusterCenters(1) absTol 1E-1) + // cluster ordering is arbitrary, so choose closest cluster + val d0 = Vectors.sqdist(kMeans.latestModel().clusterCenters(0), centers(0)) + val d1 = Vectors.sqdist(kMeans.latestModel().clusterCenters(0), centers(1)) + val (c0, c1) = if (d0 < d1) { + (centers(0), centers(1)) + } else { + (centers(1), centers(0)) + } + assert(c0 ~== kMeans.latestModel().clusterCenters(0) absTol 1E-1) + assert(c1 ~== kMeans.latestModel().clusterCenters(1) absTol 1E-1) } test("detecting dying clusters") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala index 99d52fabc530..a08917ac1ebe 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala @@ -23,18 +23,16 @@ import org.apache.spark.mllib.util.TestingUtils._ class BinaryClassificationMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { - private def areWithinEpsilon(x: (Double, Double)): Boolean = x._1 ~= (x._2) absTol 1E-5 - - private def pairsWithinEpsilon(x: ((Double, Double), (Double, Double))): Boolean = - (x._1._1 ~= x._2._1 absTol 1E-5) && (x._1._2 ~= x._2._2 absTol 1E-5) - - private def assertSequencesMatch(left: Seq[Double], right: Seq[Double]): Unit = { - assert(left.zip(right).forall(areWithinEpsilon)) + private def assertSequencesMatch(actual: Seq[Double], expected: Seq[Double]): Unit = { + actual.zip(expected).foreach { case (a, e) => assert(a ~== e absTol 1.0e-5) } } - private def assertTupleSequencesMatch(left: Seq[(Double, Double)], - right: Seq[(Double, Double)]): Unit = { - assert(left.zip(right).forall(pairsWithinEpsilon)) + private def assertTupleSequencesMatch(actual: Seq[(Double, Double)], + expected: Seq[(Double, Double)]): Unit = { + actual.zip(expected).foreach { case ((ax, ay), (ex, ey)) => + assert(ax ~== ex absTol 1.0e-5) + assert(ay ~== ey absTol 1.0e-5) + } } private def validateMetrics(metrics: BinaryClassificationMetrics, @@ -44,7 +42,7 @@ class BinaryClassificationMetricsSuite extends SparkFunSuite with MLlibTestSpark expectedFMeasures1: Seq[Double], expectedFmeasures2: Seq[Double], expectedPrecisions: Seq[Double], - expectedRecalls: Seq[Double]) = { + expectedRecalls: Seq[Double]): Unit = { assertSequencesMatch(metrics.thresholds().collect(), expectedThresholds) assertTupleSequencesMatch(metrics.roc().collect(), expectedROCCurve) @@ -111,7 +109,7 @@ class BinaryClassificationMetricsSuite extends SparkFunSuite with MLlibTestSpark val fpr = Seq(1.0) val rocCurve = Seq((0.0, 0.0)) ++ fpr.zip(recalls) ++ Seq((1.0, 1.0)) val pr = recalls.zip(precisions) - val prCurve = Seq((0.0, 1.0)) ++ pr + val prCurve = Seq((0.0, 0.0)) ++ pr val f1 = pr.map { case (0, 0) => 0.0 case (r, p) => 2.0 * (p * r) / (p + r) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala index d55bc8c3ec09..142d1e9812ef 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala @@ -36,6 +36,9 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { (1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)), 2) val metrics = new MulticlassMetrics(predictionAndLabels) val delta = 0.0000001 + val tpRate0 = 2.0 / (2 + 2) + val tpRate1 = 3.0 / (3 + 1) + val tpRate2 = 1.0 / (1 + 0) val fpRate0 = 1.0 / (9 - 4) val fpRate1 = 1.0 / (9 - 4) val fpRate2 = 1.0 / (9 - 1) @@ -53,6 +56,9 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { val f2measure2 = (1 + 2 * 2) * precision2 * recall2 / (2 * 2 * precision2 + recall2) assert(metrics.confusionMatrix.toArray.sameElements(confusionMatrix.toArray)) + assert(math.abs(metrics.truePositiveRate(0.0) - tpRate0) < delta) + assert(math.abs(metrics.truePositiveRate(1.0) - tpRate1) < delta) + assert(math.abs(metrics.truePositiveRate(2.0) - tpRate2) < delta) assert(math.abs(metrics.falsePositiveRate(0.0) - fpRate0) < delta) assert(math.abs(metrics.falsePositiveRate(1.0) - fpRate1) < delta) assert(math.abs(metrics.falsePositiveRate(2.0) - fpRate2) < delta) @@ -69,11 +75,14 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { assert(math.abs(metrics.fMeasure(1.0, 2.0) - f2measure1) < delta) assert(math.abs(metrics.fMeasure(2.0, 2.0) - f2measure2) < delta) - assert(math.abs(metrics.recall - + assert(math.abs(metrics.accuracy - (2.0 + 3.0 + 1.0) / ((2 + 3 + 1) + (1 + 1 + 1))) < delta) - assert(math.abs(metrics.recall - metrics.precision) < delta) - assert(math.abs(metrics.recall - metrics.fMeasure) < delta) - assert(math.abs(metrics.recall - metrics.weightedRecall) < delta) + assert(math.abs(metrics.accuracy - metrics.precision) < delta) + assert(math.abs(metrics.accuracy - metrics.recall) < delta) + assert(math.abs(metrics.accuracy - metrics.fMeasure) < delta) + assert(math.abs(metrics.accuracy - metrics.weightedRecall) < delta) + assert(math.abs(metrics.weightedTruePositiveRate - + ((4.0 / 9) * tpRate0 + (4.0 / 9) * tpRate1 + (1.0 / 9) * tpRate2)) < delta) assert(math.abs(metrics.weightedFalsePositiveRate - ((4.0 / 9) * fpRate0 + (4.0 / 9) * fpRate1 + (1.0 / 9) * fpRate2)) < delta) assert(math.abs(metrics.weightedPrecision - diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala index f3b19aeb42f8..a660492c7ae5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala @@ -47,7 +47,7 @@ class MultilabelMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { val scoreAndLabels: RDD[(Array[Double], Array[Double])] = sc.parallelize( Seq((Array(0.0, 1.0), Array(0.0, 2.0)), (Array(0.0, 2.0), Array(0.0, 1.0)), - (Array(), Array(0.0)), + (Array.empty[Double], Array(0.0)), (Array(2.0), Array(2.0)), (Array(2.0, 0.0), Array(2.0, 0.0)), (Array(0.0, 1.0, 2.0), Array(0.0, 1.0)), diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala index c0924a213a84..f334be2c2ba8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala @@ -18,18 +18,19 @@ package org.apache.spark.mllib.evaluation import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ class RankingMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { - test("Ranking metrics: map, ndcg") { + + test("Ranking metrics: MAP, NDCG") { val predictionAndLabels = sc.parallelize( Seq( - (Array[Int](1, 6, 2, 7, 8, 3, 9, 10, 4, 5), Array[Int](1, 2, 3, 4, 5)), - (Array[Int](4, 1, 5, 6, 2, 7, 3, 8, 9, 10), Array[Int](1, 2, 3)), - (Array[Int](1, 2, 3, 4, 5), Array[Int]()) + (Array(1, 6, 2, 7, 8, 3, 9, 10, 4, 5), Array(1, 2, 3, 4, 5)), + (Array(4, 1, 5, 6, 2, 7, 3, 8, 9, 10), Array(1, 2, 3)), + (Array(1, 2, 3, 4, 5), Array.empty[Int]) ), 2) - val eps: Double = 1E-5 + val eps = 1.0E-5 val metrics = new RankingMetrics(predictionAndLabels) val map = metrics.meanAveragePrecision @@ -48,6 +49,21 @@ class RankingMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { assert(metrics.ndcgAt(5) ~== 0.328788 absTol eps) assert(metrics.ndcgAt(10) ~== 0.487913 absTol eps) assert(metrics.ndcgAt(15) ~== metrics.ndcgAt(10) absTol eps) + } + + test("MAP, NDCG with few predictions (SPARK-14886)") { + val predictionAndLabels = sc.parallelize( + Seq( + (Array(1, 6, 2), Array(1, 2, 3, 4, 5)), + (Array.empty[Int], Array(1, 2, 3)) + ), 2) + val eps = 1.0E-5 + val metrics = new RankingMetrics(predictionAndLabels) + assert(metrics.precisionAt(1) ~== 0.5 absTol eps) + assert(metrics.precisionAt(2) ~== 0.25 absTol eps) + assert(metrics.ndcgAt(1) ~== 0.5 absTol eps) + assert(metrics.ndcgAt(2) ~== 0.30657 absTol eps) } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala index 4b7f1be58f99..f1d517383643 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala @@ -22,91 +22,115 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ class RegressionMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { + val obs = List[Double](77, 85, 62, 55, 63, 88, 57, 81, 51) + val eps = 1E-5 test("regression metrics for unbiased (includes intercept term) predictor") { /* Verify results in R: - preds = c(2.25, -0.25, 1.75, 7.75) - obs = c(3.0, -0.5, 2.0, 7.0) - - SStot = sum((obs - mean(obs))^2) - SSreg = sum((preds - mean(obs))^2) - SSerr = sum((obs - preds)^2) - - explainedVariance = SSreg / length(obs) - explainedVariance - > [1] 8.796875 - meanAbsoluteError = mean(abs(preds - obs)) - meanAbsoluteError - > [1] 0.5 - meanSquaredError = mean((preds - obs)^2) - meanSquaredError - > [1] 0.3125 - rmse = sqrt(meanSquaredError) - rmse - > [1] 0.559017 - r2 = 1 - SSerr / SStot - r2 - > [1] 0.9571734 + y = c(77, 85, 62, 55, 63, 88, 57, 81, 51) + x = c(16, 22, 14, 10, 13, 19, 12, 18, 11) + df <- as.data.frame(cbind(x, y)) + model <- lm(y ~ x, data=df) + preds = signif(predict(model), digits = 4) + preds + 1 2 3 4 5 6 7 8 9 + 72.08 91.88 65.48 52.28 62.18 81.98 58.88 78.68 55.58 + options(digits=8) + explainedVariance = mean((preds - mean(y))^2) + [1] 157.3 + meanAbsoluteError = mean(abs(preds - y)) + meanAbsoluteError + [1] 3.7355556 + meanSquaredError = mean((preds - y)^2) + meanSquaredError + [1] 17.539511 + rmse = sqrt(meanSquaredError) + rmse + [1] 4.18802 + r2 = summary(model)$r.squared + r2 + [1] 0.89968225 */ - val predictionAndObservations = sc.parallelize( - Seq((2.25, 3.0), (-0.25, -0.5), (1.75, 2.0), (7.75, 7.0)), 2) + val preds = List(72.08, 91.88, 65.48, 52.28, 62.18, 81.98, 58.88, 78.68, 55.58) + val predictionAndObservations = sc.parallelize(preds.zip(obs), 2) val metrics = new RegressionMetrics(predictionAndObservations) - assert(metrics.explainedVariance ~== 8.79687 absTol 1E-5, + assert(metrics.explainedVariance ~== 157.3 absTol eps, "explained variance regression score mismatch") - assert(metrics.meanAbsoluteError ~== 0.5 absTol 1E-5, "mean absolute error mismatch") - assert(metrics.meanSquaredError ~== 0.3125 absTol 1E-5, "mean squared error mismatch") - assert(metrics.rootMeanSquaredError ~== 0.55901 absTol 1E-5, + assert(metrics.meanAbsoluteError ~== 3.7355556 absTol eps, "mean absolute error mismatch") + assert(metrics.meanSquaredError ~== 17.539511 absTol eps, "mean squared error mismatch") + assert(metrics.rootMeanSquaredError ~== 4.18802 absTol eps, "root mean squared error mismatch") - assert(metrics.r2 ~== 0.95717 absTol 1E-5, "r2 score mismatch") + assert(metrics.r2 ~== 0.89968225 absTol eps, "r2 score mismatch") } test("regression metrics for biased (no intercept term) predictor") { /* Verify results in R: - preds = c(2.5, 0.0, 2.0, 8.0) - obs = c(3.0, -0.5, 2.0, 7.0) - - SStot = sum((obs - mean(obs))^2) - SSreg = sum((preds - mean(obs))^2) - SSerr = sum((obs - preds)^2) - - explainedVariance = SSreg / length(obs) - explainedVariance - > [1] 8.859375 - meanAbsoluteError = mean(abs(preds - obs)) - meanAbsoluteError - > [1] 0.5 - meanSquaredError = mean((preds - obs)^2) - meanSquaredError - > [1] 0.375 - rmse = sqrt(meanSquaredError) - rmse - > [1] 0.6123724 - r2 = 1 - SSerr / SStot - r2 - > [1] 0.9486081 + y = c(77, 85, 62, 55, 63, 88, 57, 81, 51) + x = c(16, 22, 14, 10, 13, 19, 12, 18, 11) + df <- as.data.frame(cbind(x, y)) + model <- lm(y ~ 0 + x, data=df) + preds = signif(predict(model), digits = 4) + preds + 1 2 3 4 5 6 7 8 9 + 72.12 99.17 63.11 45.08 58.60 85.65 54.09 81.14 49.58 + options(digits=8) + explainedVariance = mean((preds - mean(y))^2) + explainedVariance + [1] 294.88167 + meanAbsoluteError = mean(abs(preds - y)) + meanAbsoluteError + [1] 4.5888889 + meanSquaredError = mean((preds - y)^2) + meanSquaredError + [1] 39.958711 + rmse = sqrt(meanSquaredError) + rmse + [1] 6.3212903 + r2 = summary(model)$r.squared + r2 + [1] 0.99185395 */ - val predictionAndObservations = sc.parallelize( - Seq((2.5, 3.0), (0.0, -0.5), (2.0, 2.0), (8.0, 7.0)), 2) - val metrics = new RegressionMetrics(predictionAndObservations) - assert(metrics.explainedVariance ~== 8.85937 absTol 1E-5, + val preds = List(72.12, 99.17, 63.11, 45.08, 58.6, 85.65, 54.09, 81.14, 49.58) + val predictionAndObservations = sc.parallelize(preds.zip(obs), 2) + val metrics = new RegressionMetrics(predictionAndObservations, true) + assert(metrics.explainedVariance ~== 294.88167 absTol eps, "explained variance regression score mismatch") - assert(metrics.meanAbsoluteError ~== 0.5 absTol 1E-5, "mean absolute error mismatch") - assert(metrics.meanSquaredError ~== 0.375 absTol 1E-5, "mean squared error mismatch") - assert(metrics.rootMeanSquaredError ~== 0.61237 absTol 1E-5, + assert(metrics.meanAbsoluteError ~== 4.5888889 absTol eps, "mean absolute error mismatch") + assert(metrics.meanSquaredError ~== 39.958711 absTol eps, "mean squared error mismatch") + assert(metrics.rootMeanSquaredError ~== 6.3212903 absTol eps, "root mean squared error mismatch") - assert(metrics.r2 ~== 0.94860 absTol 1E-5, "r2 score mismatch") + assert(metrics.r2 ~== 0.99185395 absTol eps, "r2 score mismatch") } test("regression metrics with complete fitting") { - val predictionAndObservations = sc.parallelize( - Seq((3.0, 3.0), (0.0, 0.0), (2.0, 2.0), (8.0, 8.0)), 2) + /* Verify results in R: + y = c(77, 85, 62, 55, 63, 88, 57, 81, 51) + preds = y + explainedVariance = mean((preds - mean(y))^2) + explainedVariance + [1] 174.8395 + meanAbsoluteError = mean(abs(preds - y)) + meanAbsoluteError + [1] 0 + meanSquaredError = mean((preds - y)^2) + meanSquaredError + [1] 0 + rmse = sqrt(meanSquaredError) + rmse + [1] 0 + r2 = 1 - sum((preds - y)^2)/sum((y - mean(y))^2) + r2 + [1] 1 + */ + val preds = obs + val predictionAndObservations = sc.parallelize(preds.zip(obs), 2) val metrics = new RegressionMetrics(predictionAndObservations) - assert(metrics.explainedVariance ~== 8.6875 absTol 1E-5, + assert(metrics.explainedVariance ~== 174.83951 absTol eps, "explained variance regression score mismatch") - assert(metrics.meanAbsoluteError ~== 0.0 absTol 1E-5, "mean absolute error mismatch") - assert(metrics.meanSquaredError ~== 0.0 absTol 1E-5, "mean squared error mismatch") - assert(metrics.rootMeanSquaredError ~== 0.0 absTol 1E-5, + assert(metrics.meanAbsoluteError ~== 0.0 absTol eps, "mean absolute error mismatch") + assert(metrics.meanSquaredError ~== 0.0 absTol eps, "mean squared error mismatch") + assert(metrics.rootMeanSquaredError ~== 0.0 absTol eps, "root mean squared error mismatch") - assert(metrics.r2 ~== 1.0 absTol 1E-5, "r2 score mismatch") + assert(metrics.r2 ~== 1.0 absTol eps, "r2 score mismatch") } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala index 734800a9afad..305cb4cbbdee 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala @@ -27,42 +27,144 @@ class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext { /* * Contingency tables - * feature0 = {8.0, 0.0} + * feature0 = {6.0, 0.0, 8.0} * class 0 1 2 - * 8.0||1|0|1| - * 0.0||0|2|0| + * 6.0||1|0|0| + * 0.0||0|3|0| + * 8.0||0|0|2| + * degree of freedom = 4, statistic = 12, pValue = 0.017 * * feature1 = {7.0, 9.0} * class 0 1 2 * 7.0||1|0|0| - * 9.0||0|2|1| + * 9.0||0|3|2| + * degree of freedom = 2, statistic = 6, pValue = 0.049 * - * feature2 = {0.0, 6.0, 8.0, 5.0} + * feature2 = {0.0, 6.0, 3.0, 8.0} * class 0 1 2 * 0.0||1|0|0| - * 6.0||0|1|0| + * 6.0||0|1|2| + * 3.0||0|1|0| * 8.0||0|1|0| - * 5.0||0|0|1| + * degree of freedom = 6, statistic = 8.66, pValue = 0.193 + * + * feature3 = {7.0, 0.0, 5.0, 4.0} + * class 0 1 2 + * 7.0||1|0|0| + * 0.0||0|2|0| + * 5.0||0|1|1| + * 4.0||0|0|1| + * degree of freedom = 6, statistic = 9.5, pValue = 0.147 + * + * feature4 = {6.0, 5.0, 4.0, 0.0} + * class 0 1 2 + * 6.0||1|1|0| + * 5.0||0|2|0| + * 4.0||0|0|1| + * 0.0||0|0|1| + * degree of freedom = 6, statistic = 8.0, pValue = 0.238 + * + * feature5 = {0.0, 9.0, 5.0, 4.0} + * class 0 1 2 + * 0.0||1|0|1| + * 9.0||0|1|0| + * 5.0||0|1|0| + * 4.0||0|1|1| + * degree of freedom = 6, statistic = 5, pValue = 0.54 * * Use chi-squared calculator from Internet */ - test("ChiSqSelector transform test (sparse & dense vector)") { - val labeledDiscreteData = sc.parallelize( - Seq(LabeledPoint(0.0, Vectors.sparse(3, Array((0, 8.0), (1, 7.0)))), - LabeledPoint(1.0, Vectors.sparse(3, Array((1, 9.0), (2, 6.0)))), - LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 8.0))), - LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 5.0)))), 2) + lazy val labeledDiscreteData = sc.parallelize( + Seq(LabeledPoint(0.0, Vectors.sparse(6, Array((0, 6.0), (1, 7.0), (3, 7.0), (4, 6.0)))), + LabeledPoint(1.0, Vectors.sparse(6, Array((1, 9.0), (2, 6.0), (4, 5.0), (5, 9.0)))), + LabeledPoint(1.0, Vectors.sparse(6, Array((1, 9.0), (2, 3.0), (4, 5.0), (5, 5.0)))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 8.0, 5.0, 6.0, 4.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 6.0, 5.0, 4.0, 4.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 6.0, 4.0, 0.0, 0.0)))), 2) + + test("ChiSqSelector transform by numTopFeatures test (sparse & dense vector)") { val preFilteredData = - Set(LabeledPoint(0.0, Vectors.dense(Array(0.0))), - LabeledPoint(1.0, Vectors.dense(Array(6.0))), - LabeledPoint(1.0, Vectors.dense(Array(8.0))), - LabeledPoint(2.0, Vectors.dense(Array(5.0)))) - val model = new ChiSqSelector(1).fit(labeledDiscreteData) + Set(LabeledPoint(0.0, Vectors.dense(Array(6.0, 7.0, 7.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 0.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 0.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 5.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 5.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 4.0)))) + + val model = new ChiSqSelector(3).fit(labeledDiscreteData) + val filteredData = labeledDiscreteData.map { lp => + LabeledPoint(lp.label, model.transform(lp.features)) + }.collect().toSet + assert(filteredData === preFilteredData) + } + + test("ChiSqSelector transform by Percentile test (sparse & dense vector)") { + val preFilteredData = + Set(LabeledPoint(0.0, Vectors.dense(Array(6.0, 7.0, 7.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 0.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 0.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 5.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 5.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 4.0)))) + + val model = new ChiSqSelector().setSelectorType("percentile").setPercentile(0.5) + .fit(labeledDiscreteData) + val filteredData = labeledDiscreteData.map { lp => + LabeledPoint(lp.label, model.transform(lp.features)) + }.collect().toSet + assert(filteredData === preFilteredData) + } + + test("ChiSqSelector transform by FPR test (sparse & dense vector)") { + val preFilteredData = + Set(LabeledPoint(0.0, Vectors.dense(Array(6.0, 7.0, 7.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 0.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 0.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 5.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 5.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 4.0)))) + + val model = new ChiSqSelector().setSelectorType("fpr").setFpr(0.15) + .fit(labeledDiscreteData) + val filteredData = labeledDiscreteData.map { lp => + LabeledPoint(lp.label, model.transform(lp.features)) + }.collect().toSet + assert(filteredData === preFilteredData) + } + + test("ChiSqSelector transform by FDR test (sparse & dense vector)") { + val preFilteredData = + Set(LabeledPoint(0.0, Vectors.dense(Array(6.0, 7.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0)))) + + val model = new ChiSqSelector().setSelectorType("fdr").setFdr(0.15) + .fit(labeledDiscreteData) + val filteredData = labeledDiscreteData.map { lp => + LabeledPoint(lp.label, model.transform(lp.features)) + }.collect().toSet + assert(filteredData === preFilteredData) + } + + test("ChiSqSelector transform by FWE test (sparse & dense vector)") { + val preFilteredData = + Set(LabeledPoint(0.0, Vectors.dense(Array(6.0, 7.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0)))) + + val model = new ChiSqSelector().setSelectorType("fwe").setFwe(0.3) + .fit(labeledDiscreteData) val filteredData = labeledDiscreteData.map { lp => LabeledPoint(lp.label, model.transform(lp.features)) }.collect().toSet - assert(filteredData == preFilteredData) + assert(filteredData === preFilteredData) } test("model load / save") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/HashingTFSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/HashingTFSuite.scala index cf279c02334e..6c07e3a5cef2 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/HashingTFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/HashingTFSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.feature import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ class HashingTFSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -48,4 +49,15 @@ class HashingTFSuite extends SparkFunSuite with MLlibTestSparkContext { val docs = sc.parallelize(localDocs, 2) assert(hashingTF.transform(docs).collect().toSet === localDocs.map(hashingTF.transform).toSet) } + + test("applying binary term freqs") { + val hashingTF = new HashingTF(100).setBinary(true) + val doc = "a a b c c c".split(" ") + val n = hashingTF.numFeatures + val expected = Vectors.sparse(n, Seq( + (hashingTF.indexOf("a"), 1.0), + (hashingTF.indexOf("b"), 1.0), + (hashingTF.indexOf("c"), 1.0))) + assert(hashingTF.transform(doc) ~== expected absTol 1e-14) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala index 21163633051e..5c938a61ed99 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.feature import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors, Vector} +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala index 34122d6ed2e9..10f7bafd6cf5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala @@ -51,10 +51,10 @@ class NormalizerSuite extends SparkFunSuite with MLlibTestSparkContext { assert((data1, data1RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) - assert(brzNorm(data1(0).toBreeze, 1) ~== 1.0 absTol 1E-5) - assert(brzNorm(data1(2).toBreeze, 1) ~== 1.0 absTol 1E-5) - assert(brzNorm(data1(3).toBreeze, 1) ~== 1.0 absTol 1E-5) - assert(brzNorm(data1(4).toBreeze, 1) ~== 1.0 absTol 1E-5) + assert(brzNorm(data1(0).asBreeze, 1) ~== 1.0 absTol 1E-5) + assert(brzNorm(data1(2).asBreeze, 1) ~== 1.0 absTol 1E-5) + assert(brzNorm(data1(3).asBreeze, 1) ~== 1.0 absTol 1E-5) + assert(brzNorm(data1(4).asBreeze, 1) ~== 1.0 absTol 1E-5) assert(data1(0) ~== Vectors.sparse(3, Seq((0, -0.465116279), (1, 0.53488372))) absTol 1E-5) assert(data1(1) ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) @@ -78,10 +78,10 @@ class NormalizerSuite extends SparkFunSuite with MLlibTestSparkContext { assert((data2, data2RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) - assert(brzNorm(data2(0).toBreeze, 2) ~== 1.0 absTol 1E-5) - assert(brzNorm(data2(2).toBreeze, 2) ~== 1.0 absTol 1E-5) - assert(brzNorm(data2(3).toBreeze, 2) ~== 1.0 absTol 1E-5) - assert(brzNorm(data2(4).toBreeze, 2) ~== 1.0 absTol 1E-5) + assert(brzNorm(data2(0).asBreeze, 2) ~== 1.0 absTol 1E-5) + assert(brzNorm(data2(2).asBreeze, 2) ~== 1.0 absTol 1E-5) + assert(brzNorm(data2(3).asBreeze, 2) ~== 1.0 absTol 1E-5) + assert(brzNorm(data2(4).asBreeze, 2) ~== 1.0 absTol 1E-5) assert(data2(0) ~== Vectors.sparse(3, Seq((0, -0.65617871), (1, 0.75460552))) absTol 1E-5) assert(data2(1) ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/PCASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/PCASuite.scala index e57f49191378..8eab12416a69 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/PCASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/PCASuite.scala @@ -18,9 +18,10 @@ package org.apache.spark.mllib.feature import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ class PCASuite extends SparkFunSuite with MLlibTestSparkContext { @@ -37,11 +38,20 @@ class PCASuite extends SparkFunSuite with MLlibTestSparkContext { val pca = new PCA(k).fit(dataRDD) val mat = new RowMatrix(dataRDD) - val pc = mat.computePrincipalComponents(k) + val (pc, explainedVariance) = mat.computePrincipalComponentsAndExplainedVariance(k) val pca_transform = pca.transform(dataRDD).collect() val mat_multiply = mat.multiply(pc).rows.collect() - assert(pca_transform.toSet === mat_multiply.toSet) + pca_transform.zip(mat_multiply).foreach { case (calculated, expected) => + assert(calculated ~== expected relTol 1e-8) + } + assert(pca.explainedVariance ~== explainedVariance relTol 1e-8) + } + + test("memory cost computation") { + assert(PCAUtil.memoryCost(10, 100) < Int.MaxValue) + // check overflowing + assert(PCAUtil.memoryCost(40000, 60000) > Int.MaxValue) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala index 6ab2fa677012..a5769631e510 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.mllib.feature import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, MultivariateStatisticalSummary} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ -import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, MultivariateOnlineSummarizer} import org.apache.spark.rdd.RDD class StandardScalerSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -207,23 +207,17 @@ class StandardScalerSuite extends SparkFunSuite with MLlibTestSparkContext { val equivalentModel2 = new StandardScalerModel(model2.std, model2.mean, true, false) val equivalentModel3 = new StandardScalerModel(model3.std, model3.mean, false, true) + val data1 = sparseData.map(equivalentModel1.transform) val data2 = sparseData.map(equivalentModel2.transform) + val data3 = sparseData.map(equivalentModel3.transform) - withClue("Standardization with mean can not be applied on sparse input.") { - intercept[IllegalArgumentException] { - sparseData.map(equivalentModel1.transform) - } - } - - withClue("Standardization with mean can not be applied on sparse input.") { - intercept[IllegalArgumentException] { - sparseData.map(equivalentModel3.transform) - } - } - + val data1RDD = equivalentModel1.transform(dataRDD) val data2RDD = equivalentModel2.transform(dataRDD) + val data3RDD = equivalentModel3.transform(dataRDD) - val summary = computeSummary(data2RDD) + val summary1 = computeSummary(data1RDD) + val summary2 = computeSummary(data2RDD) + val summary3 = computeSummary(data3RDD) assert((sparseData, data2, data2RDD.collect()).zipped.forall { case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true @@ -231,13 +225,23 @@ class StandardScalerSuite extends SparkFunSuite with MLlibTestSparkContext { case _ => false }, "The vector type should be preserved after standardization.") + assert((data1, data1RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) assert((data2, data2RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + assert((data3, data3RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) - assert(summary.mean !~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) - assert(summary.variance ~== Vectors.dense(1.0, 1.0, 1.0) absTol 1E-5) + assert(summary1.mean ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(summary1.variance ~== Vectors.dense(1.0, 1.0, 1.0) absTol 1E-5) + assert(summary2.mean !~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(summary2.variance ~== Vectors.dense(1.0, 1.0, 1.0) absTol 1E-5) + assert(summary3.mean ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(summary3.variance !~== Vectors.dense(1.0, 1.0, 1.0) absTol 1E-5) + assert(data1(4) ~== Vectors.dense(0.56854, -0.069068, 0.116377) absTol 1E-5) + assert(data1(5) ~== Vectors.dense(-0.296998, 0.872775, 0.116377) absTol 1E-5) assert(data2(4) ~== Vectors.sparse(3, Seq((0, 0.865538862), (1, -0.22604255))) absTol 1E-5) assert(data2(5) ~== Vectors.sparse(3, Seq((1, 0.71580142))) absTol 1E-5) + assert(data3(4) ~== Vectors.dense(1.116666, -0.183333, 0.183333) absTol 1E-5) + assert(data3(5) ~== Vectors.dense(-0.583333, 2.316666, 0.183333) absTol 1E-5) } test("Standardization with sparse input") { @@ -252,24 +256,17 @@ class StandardScalerSuite extends SparkFunSuite with MLlibTestSparkContext { val model2 = standardizer2.fit(dataRDD) val model3 = standardizer3.fit(dataRDD) + val data1 = sparseData.map(model1.transform) val data2 = sparseData.map(model2.transform) + val data3 = sparseData.map(model3.transform) - withClue("Standardization with mean can not be applied on sparse input.") { - intercept[IllegalArgumentException] { - sparseData.map(model1.transform) - } - } - - withClue("Standardization with mean can not be applied on sparse input.") { - intercept[IllegalArgumentException] { - sparseData.map(model3.transform) - } - } - + val data1RDD = model1.transform(dataRDD) val data2RDD = model2.transform(dataRDD) + val data3RDD = model3.transform(dataRDD) - - val summary = computeSummary(data2RDD) + val summary1 = computeSummary(data1RDD) + val summary2 = computeSummary(data2RDD) + val summary3 = computeSummary(data3RDD) assert((sparseData, data2, data2RDD.collect()).zipped.forall { case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true @@ -277,13 +274,23 @@ class StandardScalerSuite extends SparkFunSuite with MLlibTestSparkContext { case _ => false }, "The vector type should be preserved after standardization.") + assert((data1, data1RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) assert((data2, data2RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + assert((data3, data3RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) - assert(summary.mean !~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) - assert(summary.variance ~== Vectors.dense(1.0, 1.0, 1.0) absTol 1E-5) + assert(summary1.mean ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(summary1.variance ~== Vectors.dense(1.0, 1.0, 1.0) absTol 1E-5) + assert(summary2.mean !~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(summary2.variance ~== Vectors.dense(1.0, 1.0, 1.0) absTol 1E-5) + assert(summary3.mean ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(summary3.variance !~== Vectors.dense(1.0, 1.0, 1.0) absTol 1E-5) + assert(data1(4) ~== Vectors.dense(0.56854, -0.069068, 0.116377) absTol 1E-5) + assert(data1(5) ~== Vectors.dense(-0.296998, 0.872775, 0.116377) absTol 1E-5) assert(data2(4) ~== Vectors.sparse(3, Seq((0, 0.865538862), (1, -0.22604255))) absTol 1E-5) assert(data2(5) ~== Vectors.sparse(3, Seq((1, 0.71580142))) absTol 1E-5) + assert(data3(4) ~== Vectors.dense(1.116666, -0.183333, 0.183333) absTol 1E-5) + assert(data3(5) ~== Vectors.dense(-0.583333, 2.316666, 0.183333) absTol 1E-5) } test("Standardization with constant input when means and stds are provided") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala index a864eec460f2..f4fa216b8eba 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala @@ -18,9 +18,8 @@ package org.apache.spark.mllib.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext - -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.Utils class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -70,6 +69,21 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { assert(syms(1)._1 == "japan") } + test("findSynonyms doesn't reject similar word vectors when called with a vector") { + val num = 2 + val word2VecMap = Map( + ("china", Array(0.50f, 0.50f, 0.50f, 0.50f)), + ("japan", Array(0.40f, 0.50f, 0.50f, 0.50f)), + ("taiwan", Array(0.60f, 0.50f, 0.50f, 0.50f)), + ("korea", Array(0.45f, 0.60f, 0.60f, 0.60f)) + ) + val model = new Word2VecModel(word2VecMap) + val syms = model.findSynonyms(Vectors.dense(Array(0.52, 0.5, 0.5, 0.5)), num) + assert(syms.length == num) + assert(syms(0)._1 == "china") + assert(syms(1)._1 == "taiwan") + } + test("model load / save") { val word2VecMap = Map( @@ -92,4 +106,63 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { } } + + test("big model load / save") { + // backupping old values + val oldBufferConfValue = spark.conf.get("spark.kryoserializer.buffer.max", "64m") + val oldBufferMaxConfValue = spark.conf.get("spark.kryoserializer.buffer", "64k") + + // setting test values to trigger partitioning + spark.conf.set("spark.kryoserializer.buffer", "50b") + spark.conf.set("spark.kryoserializer.buffer.max", "50b") + + // create a model bigger than 50 Bytes + val word2VecMap = Map((0 to 10).map(i => s"$i" -> Array.fill(10)(0.1f)): _*) + val model = new Word2VecModel(word2VecMap) + + // est. size of this model, given the formula: + // (floatSize * vectorSize + 15) * numWords + // (4 * 10 + 15) * 10 = 550 + // therefore it should generate multiple partitions + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + try { + model.save(sc, path) + val sameModel = Word2VecModel.load(sc, path) + assert(sameModel.getVectors.mapValues(_.toSeq) === model.getVectors.mapValues(_.toSeq)) + } + catch { + case t: Throwable => fail("exception thrown persisting a model " + + "that spans over multiple partitions", t) + } finally { + Utils.deleteRecursively(tempDir) + spark.conf.set("spark.kryoserializer.buffer", oldBufferConfValue) + spark.conf.set("spark.kryoserializer.buffer.max", oldBufferMaxConfValue) + } + + } + + test("test similarity for word vectors with large values is not Infinity or NaN") { + val vecA = Array(-4.331467827487745E21, -5.26707742075006E21, + 5.63551690626524E21, 2.833692188614257E21, -1.9688159903619345E21, -4.933950659913092E21, + -2.7401535502536787E21, -1.418671793782632E20).map(_.toFloat) + val vecB = Array(-3.9850175451103232E16, -3.4829783883841536E16, + 9.421469251534848E15, 4.4069684466679808E16, 7.20936298872832E15, -4.2883302830374912E16, + -3.605579947835392E16, -2.8151294422155264E16).map(_.toFloat) + val vecC = Array(-1.9227381025734656E16, -3.907009342603264E16, + 2.110207626838016E15, -4.8770066610651136E16, -1.9734964555743232E16, -3.2206001247617024E16, + 2.7725358220443648E16, 3.1618718156980224E16).map(_.toFloat) + val wordMapIn = Map( + ("A", vecA), + ("B", vecB), + ("C", vecC) + ) + + val model = new Word2VecModel(wordMapIn) + model.findSynonyms("A", 5).foreach { pair => + assert(!(pair._2.isInfinite || pair._2.isNaN)) + } + } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala index 77a2773c36f5..dcb1f398b04b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala @@ -42,6 +42,7 @@ class AssociationRulesSuite extends SparkFunSuite with MLlibTestSparkContext { .collect() /* Verify results using the `R` code: + library(arules) transactions = as(sapply( list("r z h k p", "z y x w v u t s", @@ -52,7 +53,7 @@ class AssociationRulesSuite extends SparkFunSuite with MLlibTestSparkContext { FUN=function(x) strsplit(x," ",fixed=TRUE)), "transactions") ars = apriori(transactions, - parameter = list(support = 0.0, confidence = 0.5, target="rules", minlen=2)) + parameter = list(support = 0.5, confidence = 0.9, target="rules", minlen=2)) arsDF = as(ars, "data.frame") arsDF$support = arsDF$support * length(transactions) names(arsDF)[names(arsDF) == "support"] = "freq" diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala index 4a9bfdb348d9..dc44c58e97eb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala @@ -16,8 +16,11 @@ */ package org.apache.spark.mllib.fpm +import scala.language.existentials + import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.util.Utils class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -274,4 +277,71 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext { */ assert(model1.freqItemsets.count() === 65) } + + test("model save/load with String type") { + val transactions = 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(" ")) + val rdd = sc.parallelize(transactions, 2).cache() + + val model3 = new FPGrowth() + .setMinSupport(0.5) + .setNumPartitions(2) + .run(rdd) + val freqItemsets3 = model3.freqItemsets.collect().map { itemset => + (itemset.items.toSet, itemset.freq) + } + + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + try { + model3.save(sc, path) + val newModel = FPGrowthModel.load(sc, path) + val newFreqItemsets = newModel.freqItemsets.collect().map { itemset => + (itemset.items.toSet, itemset.freq) + } + assert(freqItemsets3.toSet === newFreqItemsets.toSet) + } finally { + Utils.deleteRecursively(tempDir) + } + } + + test("model save/load with Int type") { + val transactions = Seq( + "1 2 3", + "1 2 3 4", + "5 4 3 2 1", + "6 5 4 3 2 1", + "2 4", + "1 3", + "1 7") + .map(_.split(" ").map(_.toInt).toArray) + val rdd = sc.parallelize(transactions, 2).cache() + + val model3 = new FPGrowth() + .setMinSupport(0.5) + .setNumPartitions(2) + .run(rdd) + val freqItemsets3 = model3.freqItemsets.collect().map { itemset => + (itemset.items.toSet, itemset.freq) + } + + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + try { + model3.save(sc, path) + val newModel = FPGrowthModel.load(sc, path) + val newFreqItemsets = newModel.freqItemsets.collect().map { itemset => + (itemset.items.toSet, itemset.freq) + } + assert(freqItemsets3.toSet === newFreqItemsets.toSet) + } finally { + Utils.deleteRecursively(tempDir) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/PrefixSpanSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/PrefixSpanSuite.scala index a83e543859b8..c2e08d078fc1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/PrefixSpanSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/PrefixSpanSuite.scala @@ -16,8 +16,11 @@ */ package org.apache.spark.mllib.fpm +import scala.language.existentials + import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.util.Utils class PrefixSpanSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -357,6 +360,79 @@ class PrefixSpanSuite extends SparkFunSuite with MLlibTestSparkContext { compareResults(expected, model.freqSequences.collect()) } + test("PrefixSpan pre-processing's cleaning test") { + + // One item per itemSet + val itemToInt1 = (4 to 5).zipWithIndex.toMap + val sequences1 = Seq( + Array(Array(4), Array(1), Array(2), Array(5), Array(2), Array(4), Array(5)), + Array(Array(6), Array(7), Array(8))) + val rdd1 = sc.parallelize(sequences1, 2).cache() + + val cleanedSequence1 = PrefixSpan.toDatabaseInternalRepr(rdd1, itemToInt1).collect() + + val expected1 = Array(Array(0, 4, 0, 5, 0, 4, 0, 5, 0)) + .map(_.map(x => if (x == 0) 0 else itemToInt1(x) + 1)) + + compareInternalSequences(expected1, cleanedSequence1) + + // Multi-item sequence + val itemToInt2 = (4 to 6).zipWithIndex.toMap + val sequences2 = Seq( + Array(Array(4, 5), Array(1, 6, 2), Array(2), Array(5), Array(2), Array(4), Array(5, 6, 7)), + Array(Array(8, 9), Array(1, 2))) + val rdd2 = sc.parallelize(sequences2, 2).cache() + + val cleanedSequence2 = PrefixSpan.toDatabaseInternalRepr(rdd2, itemToInt2).collect() + + val expected2 = Array(Array(0, 4, 5, 0, 6, 0, 5, 0, 4, 0, 5, 6, 0)) + .map(_.map(x => if (x == 0) 0 else itemToInt2(x) + 1)) + + compareInternalSequences(expected2, cleanedSequence2) + + // Emptied sequence + val itemToInt3 = (10 to 10).zipWithIndex.toMap + val sequences3 = Seq( + Array(Array(4, 5), Array(1, 6, 2), Array(2), Array(5), Array(2), Array(4), Array(5, 6, 7)), + Array(Array(8, 9), Array(1, 2))) + val rdd3 = sc.parallelize(sequences3, 2).cache() + + val cleanedSequence3 = PrefixSpan.toDatabaseInternalRepr(rdd3, itemToInt3).collect() + val expected3 = Array[Array[Int]]() + + compareInternalSequences(expected3, cleanedSequence3) + } + + test("model save/load") { + val sequences = Seq( + Array(Array(1, 2), Array(3)), + Array(Array(1), Array(3, 2), Array(1, 2)), + Array(Array(1, 2), Array(5)), + Array(Array(6))) + val rdd = sc.parallelize(sequences, 2).cache() + + val prefixSpan = new PrefixSpan() + .setMinSupport(0.5) + .setMaxPatternLength(5) + val model = prefixSpan.run(rdd) + + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + try { + model.save(sc, path) + val newModel = PrefixSpanModel.load(sc, path) + val originalSet = model.freqSequences.collect().map { x => + (x.sequence.map(_.toSet).toSeq, x.freq) + }.toSet + val newSet = newModel.freqSequences.collect().map { x => + (x.sequence.map(_.toSet).toSeq, x.freq) + }.toSet + assert(originalSet === newSet) + } finally { + Utils.deleteRecursively(tempDir) + } + } + private def compareResults[Item]( expectedValue: Array[(Array[Array[Item]], Long)], actualValue: Array[PrefixSpan.FreqSequence[Item]]): Unit = { @@ -376,4 +452,12 @@ class PrefixSpanSuite extends SparkFunSuite with MLlibTestSparkContext { val actualSet = actualValue.map(x => (x._1.toSeq, x._2)).toSet assert(expectedSet === actualSet) } + + private def compareInternalSequences( + expectedValue: Array[Array[Int]], + actualValue: Array[Array[Int]]): Unit = { + val expectedSet = expectedValue.map(x => x.toSeq).toSet + val actualSet = actualValue.map(x => x.toSeq).toSet + assert(expectedSet === actualSet) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala deleted file mode 100644 index e331c7598918..000000000000 --- a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala +++ /dev/null @@ -1,187 +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.mllib.impl - -import org.apache.hadoop.fs.{FileSystem, Path} - -import org.apache.spark.{SparkContext, SparkFunSuite} -import org.apache.spark.graphx.{Edge, Graph} -import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.Utils - - -class PeriodicGraphCheckpointerSuite extends SparkFunSuite with MLlibTestSparkContext { - - import PeriodicGraphCheckpointerSuite._ - - test("Persisting") { - var graphsToCheck = Seq.empty[GraphToCheck] - - val graph1 = createGraph(sc) - val checkpointer = - new PeriodicGraphCheckpointer[Double, Double](10, graph1.vertices.sparkContext) - checkpointer.update(graph1) - graphsToCheck = graphsToCheck :+ GraphToCheck(graph1, 1) - checkPersistence(graphsToCheck, 1) - - var iteration = 2 - while (iteration < 9) { - val graph = createGraph(sc) - checkpointer.update(graph) - graphsToCheck = graphsToCheck :+ GraphToCheck(graph, iteration) - checkPersistence(graphsToCheck, iteration) - iteration += 1 - } - } - - test("Checkpointing") { - val tempDir = Utils.createTempDir() - val path = tempDir.toURI.toString - val checkpointInterval = 2 - var graphsToCheck = Seq.empty[GraphToCheck] - sc.setCheckpointDir(path) - val graph1 = createGraph(sc) - val checkpointer = new PeriodicGraphCheckpointer[Double, Double]( - checkpointInterval, graph1.vertices.sparkContext) - checkpointer.update(graph1) - graph1.edges.count() - graph1.vertices.count() - graphsToCheck = graphsToCheck :+ GraphToCheck(graph1, 1) - checkCheckpoint(graphsToCheck, 1, checkpointInterval) - - var iteration = 2 - while (iteration < 9) { - val graph = createGraph(sc) - checkpointer.update(graph) - graph.vertices.count() - graph.edges.count() - graphsToCheck = graphsToCheck :+ GraphToCheck(graph, iteration) - checkCheckpoint(graphsToCheck, iteration, checkpointInterval) - iteration += 1 - } - - checkpointer.deleteAllCheckpoints() - graphsToCheck.foreach { graph => - confirmCheckpointRemoved(graph.graph) - } - - Utils.deleteRecursively(tempDir) - } -} - -private object PeriodicGraphCheckpointerSuite { - - case class GraphToCheck(graph: Graph[Double, Double], gIndex: Int) - - val edges = Seq( - Edge[Double](0, 1, 0), - Edge[Double](1, 2, 0), - Edge[Double](2, 3, 0), - Edge[Double](3, 4, 0)) - - def createGraph(sc: SparkContext): Graph[Double, Double] = { - Graph.fromEdges[Double, Double](sc.parallelize(edges), 0) - } - - def checkPersistence(graphs: Seq[GraphToCheck], iteration: Int): Unit = { - graphs.foreach { g => - checkPersistence(g.graph, g.gIndex, iteration) - } - } - - /** - * Check storage level of graph. - * @param gIndex Index of graph in order inserted into checkpointer (from 1). - * @param iteration Total number of graphs inserted into checkpointer. - */ - def checkPersistence(graph: Graph[_, _], gIndex: Int, iteration: Int): Unit = { - try { - if (gIndex + 2 < iteration) { - assert(graph.vertices.getStorageLevel == StorageLevel.NONE) - assert(graph.edges.getStorageLevel == StorageLevel.NONE) - } else { - assert(graph.vertices.getStorageLevel != StorageLevel.NONE) - assert(graph.edges.getStorageLevel != StorageLevel.NONE) - } - } catch { - case _: AssertionError => - throw new Exception(s"PeriodicGraphCheckpointerSuite.checkPersistence failed with:\n" + - s"\t gIndex = $gIndex\n" + - s"\t iteration = $iteration\n" + - s"\t graph.vertices.getStorageLevel = ${graph.vertices.getStorageLevel}\n" + - s"\t graph.edges.getStorageLevel = ${graph.edges.getStorageLevel}\n") - } - } - - def checkCheckpoint(graphs: Seq[GraphToCheck], iteration: Int, checkpointInterval: Int): Unit = { - graphs.reverse.foreach { g => - checkCheckpoint(g.graph, g.gIndex, iteration, checkpointInterval) - } - } - - def confirmCheckpointRemoved(graph: Graph[_, _]): Unit = { - // Note: We cannot check graph.isCheckpointed since that value is never updated. - // Instead, we check for the presence of the checkpoint files. - // This test should continue to work even after this graph.isCheckpointed issue - // is fixed (though it can then be simplified and not look for the files). - val fs = FileSystem.get(graph.vertices.sparkContext.hadoopConfiguration) - graph.getCheckpointFiles.foreach { checkpointFile => - assert(!fs.exists(new Path(checkpointFile)), - "Graph checkpoint file should have been removed") - } - } - - /** - * Check checkpointed status of graph. - * @param gIndex Index of graph in order inserted into checkpointer (from 1). - * @param iteration Total number of graphs inserted into checkpointer. - */ - def checkCheckpoint( - graph: Graph[_, _], - gIndex: Int, - iteration: Int, - checkpointInterval: Int): Unit = { - try { - if (gIndex % checkpointInterval == 0) { - // We allow 2 checkpoint intervals since we perform an action (checkpointing a second graph) - // only AFTER PeriodicGraphCheckpointer decides whether to remove the previous checkpoint. - if (iteration - 2 * checkpointInterval < gIndex && gIndex <= iteration) { - assert(graph.isCheckpointed, "Graph should be checkpointed") - assert(graph.getCheckpointFiles.length == 2, "Graph should have 2 checkpoint files") - } else { - confirmCheckpointRemoved(graph) - } - } else { - // Graph should never be checkpointed - assert(!graph.isCheckpointed, "Graph should never have been checkpointed") - assert(graph.getCheckpointFiles.isEmpty, "Graph should not have any checkpoint files") - } - } catch { - case e: AssertionError => - throw new Exception(s"PeriodicGraphCheckpointerSuite.checkCheckpoint failed with:\n" + - s"\t gIndex = $gIndex\n" + - s"\t iteration = $iteration\n" + - s"\t checkpointInterval = $checkpointInterval\n" + - s"\t graph.isCheckpointed = ${graph.isCheckpointed}\n" + - s"\t graph.getCheckpointFiles = ${graph.getCheckpointFiles.mkString(", ")}\n" + - s" AssertionError message: ${e.getMessage}") - } - } - -} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala index 96e5ffef7a13..6e68c1c9d36c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.mllib.linalg import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.mllib.linalg.BLAS._ +import org.apache.spark.mllib.util.TestingUtils._ class BLASSuite extends SparkFunSuite { @@ -392,6 +392,23 @@ class BLASSuite extends SparkFunSuite { } } + val y17 = new DenseVector(Array(0.0, 0.0)) + val y18 = y17.copy + + val sA3 = new SparseMatrix(3, 2, Array(0, 2, 4), Array(1, 2, 0, 1), Array(2.0, 1.0, 1.0, 2.0)) + .transpose + val sA4 = + new SparseMatrix(2, 3, Array(0, 1, 3, 4), Array(1, 0, 1, 0), Array(1.0, 2.0, 2.0, 1.0)) + val sx3 = new SparseVector(3, Array(1, 2), Array(2.0, 1.0)) + + val expected4 = new DenseVector(Array(5.0, 4.0)) + + gemv(1.0, sA3, sx3, 0.0, y17) + gemv(1.0, sA4, sx3, 0.0, y18) + + assert(y17 ~== expected4 absTol 1e-15) + assert(y18 ~== expected4 absTol 1e-15) + val dAT = new DenseMatrix(3, 4, Array(0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 1.0, 0.0, 0.0, 0.0, 3.0)) val sAT = diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala index dc04258e41d2..9e4735afdd59 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala @@ -17,14 +17,14 @@ package org.apache.spark.mllib.linalg -import breeze.linalg.{DenseMatrix => BDM, CSCMatrix => BSM} +import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM} import org.apache.spark.SparkFunSuite class BreezeMatrixConversionSuite extends SparkFunSuite { test("dense matrix to breeze") { val mat = Matrices.dense(3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) - val breeze = mat.toBreeze.asInstanceOf[BDM[Double]] + val breeze = mat.asBreeze.asInstanceOf[BDM[Double]] assert(breeze.rows === mat.numRows) assert(breeze.cols === mat.numCols) assert(breeze.data.eq(mat.asInstanceOf[DenseMatrix].values), "should not copy data") @@ -48,7 +48,7 @@ class BreezeMatrixConversionSuite extends SparkFunSuite { val colPtrs = Array(0, 2, 4) val rowIndices = Array(1, 2, 1, 2) val mat = Matrices.sparse(3, 2, colPtrs, rowIndices, values) - val breeze = mat.toBreeze.asInstanceOf[BSM[Double]] + val breeze = mat.asBreeze.asInstanceOf[BSM[Double]] assert(breeze.rows === mat.numRows) assert(breeze.cols === mat.numCols) assert(breeze.data.eq(mat.asInstanceOf[SparseMatrix].values), "should not copy data") diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala index 3772c9235ad3..996f621f18c8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala @@ -33,12 +33,12 @@ class BreezeVectorConversionSuite extends SparkFunSuite { test("dense to breeze") { val vec = Vectors.dense(arr) - assert(vec.toBreeze === new BDV[Double](arr)) + assert(vec.asBreeze === new BDV[Double](arr)) } test("sparse to breeze") { val vec = Vectors.sparse(n, indices, values) - assert(vec.toBreeze === new BSV[Double](indices, values, n)) + assert(vec.asBreeze === new BSV[Double](indices, values, n)) } test("dense breeze to vector") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index 1833cf383367..c8ac92eecf40 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -19,11 +19,14 @@ package org.apache.spark.mllib.linalg import java.util.Random -import org.mockito.Mockito.when -import org.scalatest.mock.MockitoSugar._ import scala.collection.mutable.{Map => MutableMap} +import breeze.linalg.{CSCMatrix, Matrix => BM} +import org.mockito.Mockito.when +import org.scalatest.mockito.MockitoSugar._ + import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.mllib.util.TestingUtils._ class MatricesSuite extends SparkFunSuite { @@ -60,7 +63,7 @@ class MatricesSuite extends SparkFunSuite { (1, 2, 2.0), (2, 2, 2.0), (1, 2, 2.0), (0, 0, 0.0)) val mat2 = SparseMatrix.fromCOO(m, n, entries) - assert(mat.toBreeze === mat2.toBreeze) + assert(mat.asBreeze === mat2.asBreeze) assert(mat2.values.length == 4) } @@ -150,6 +153,10 @@ class MatricesSuite extends SparkFunSuite { sparseMat.update(0, 0, 10.0) } + intercept[NoSuchElementException] { + sparseMat.update(2, 1, 10.0) + } + sparseMat.update(0, 1, 10.0) assert(sparseMat(0, 1) === 10.0) assert(sparseMat.values(2) === 10.0) @@ -169,8 +176,8 @@ class MatricesSuite extends SparkFunSuite { val spMat2 = deMat1.toSparse val deMat2 = spMat1.toDense - assert(spMat1.toBreeze === spMat2.toBreeze) - assert(deMat1.toBreeze === deMat2.toBreeze) + assert(spMat1.asBreeze === spMat2.asBreeze) + assert(deMat1.asBreeze === deMat2.asBreeze) } test("map, update") { @@ -204,8 +211,8 @@ class MatricesSuite extends SparkFunSuite { val sATexpected = new SparseMatrix(3, 4, Array(0, 1, 2, 3, 4), Array(1, 0, 1, 2), Array(2.0, 1.0, 1.0, 3.0)) - assert(dAT.toBreeze === dATexpected.toBreeze) - assert(sAT.toBreeze === sATexpected.toBreeze) + assert(dAT.asBreeze === dATexpected.asBreeze) + assert(sAT.asBreeze === sATexpected.asBreeze) assert(dA(1, 0) === dAT(0, 1)) assert(dA(2, 1) === dAT(1, 2)) assert(sA(1, 0) === sAT(0, 1)) @@ -214,8 +221,8 @@ class MatricesSuite extends SparkFunSuite { assert(!dA.toArray.eq(dAT.toArray), "has to have a new array") assert(dA.values.eq(dAT.transpose.asInstanceOf[DenseMatrix].values), "should not copy array") - assert(dAT.toSparse.toBreeze === sATexpected.toBreeze) - assert(sAT.toDense.toBreeze === dATexpected.toBreeze) + assert(dAT.toSparse.asBreeze === sATexpected.asBreeze) + assert(sAT.toDense.asBreeze === dATexpected.asBreeze) } test("foreachActive") { @@ -234,22 +241,22 @@ class MatricesSuite extends SparkFunSuite { dnMap.put((i, j), value) } assert(dnMap.size === 6) - assert(dnMap(0, 0) === 1.0) - assert(dnMap(1, 0) === 2.0) - assert(dnMap(2, 0) === 0.0) - assert(dnMap(0, 1) === 0.0) - assert(dnMap(1, 1) === 4.0) - assert(dnMap(2, 1) === 5.0) + assert(dnMap((0, 0)) === 1.0) + assert(dnMap((1, 0)) === 2.0) + assert(dnMap((2, 0)) === 0.0) + assert(dnMap((0, 1)) === 0.0) + assert(dnMap((1, 1)) === 4.0) + assert(dnMap((2, 1)) === 5.0) val spMap = MutableMap[(Int, Int), Double]() sp.foreachActive { (i, j, value) => spMap.put((i, j), value) } assert(spMap.size === 4) - assert(spMap(0, 0) === 1.0) - assert(spMap(1, 0) === 2.0) - assert(spMap(1, 1) === 4.0) - assert(spMap(2, 1) === 5.0) + assert(spMap((0, 0)) === 1.0) + assert(spMap((1, 0)) === 2.0) + assert(spMap((1, 1)) === 4.0) + assert(spMap((2, 1)) === 5.0) } test("horzcat, vertcat, eye, speye") { @@ -282,7 +289,7 @@ class MatricesSuite extends SparkFunSuite { val spHorz2 = Matrices.horzcat(Array(spMat1, deMat2)) val spHorz3 = Matrices.horzcat(Array(deMat1, spMat2)) val deHorz1 = Matrices.horzcat(Array(deMat1, deMat2)) - val deHorz2 = Matrices.horzcat(Array[Matrix]()) + val deHorz2 = Matrices.horzcat(Array.empty[Matrix]) assert(deHorz1.numRows === 3) assert(spHorz2.numRows === 3) @@ -336,7 +343,7 @@ class MatricesSuite extends SparkFunSuite { val deVert1 = Matrices.vertcat(Array(deMat1, deMat3)) val spVert2 = Matrices.vertcat(Array(spMat1, deMat3)) val spVert3 = Matrices.vertcat(Array(deMat1, spMat3)) - val deVert2 = Matrices.vertcat(Array[Matrix]()) + val deVert2 = Matrices.vertcat(Array.empty[Matrix]) assert(deVert1.numRows === 5) assert(spVert2.numRows === 5) @@ -494,4 +501,134 @@ class MatricesSuite extends SparkFunSuite { assert(sm1.numNonzeros === 1) assert(sm1.numActives === 3) } + + test("fromBreeze with sparse matrix") { + // colPtr.last does NOT always equal to values.length in breeze SCSMatrix and + // invocation of compact() may be necessary. Refer to SPARK-11507 + val bm1: BM[Double] = new CSCMatrix[Double]( + Array(1.0, 1, 1), 3, 3, Array(0, 1, 2, 3), Array(0, 1, 2)) + val bm2: BM[Double] = new CSCMatrix[Double]( + Array(1.0, 2, 2, 4), 3, 3, Array(0, 0, 2, 4), Array(1, 2, 1, 2)) + val sum = bm1 + bm2 + Matrices.fromBreeze(sum) + } + + test("Test FromBreeze when Breeze.CSCMatrix.rowIndices has trailing zeros. - SPARK-20687") { + // (2, 0, 0) + // (2, 0, 0) + val mat1Brz = Matrices.sparse(2, 3, Array(0, 2, 2, 2), Array(0, 1), Array(2, 2)).asBreeze + // (2, 1E-15, 1E-15) + // (2, 1E-15, 1E-15) + val mat2Brz = Matrices.sparse(2, 3, + Array(0, 2, 4, 6), + Array(0, 0, 0, 1, 1, 1), + Array(2, 1E-15, 1E-15, 2, 1E-15, 1E-15)).asBreeze + val t1Brz = mat1Brz - mat2Brz + val t2Brz = mat2Brz - mat1Brz + // The following operations raise exceptions on un-patch Matrices.fromBreeze + val t1 = Matrices.fromBreeze(t1Brz) + val t2 = Matrices.fromBreeze(t2Brz) + // t1 == t1Brz && t2 == t2Brz + assert((t1.asBreeze - t1Brz).iterator.map((x) => math.abs(x._2)).sum < 1E-15) + assert((t2.asBreeze - t2Brz).iterator.map((x) => math.abs(x._2)).sum < 1E-15) + } + + test("row/col iterator") { + val dm = new DenseMatrix(3, 2, Array(0, 1, 2, 3, 4, 0)) + val sm = dm.toSparse + val rows = Seq(Vectors.dense(0, 3), Vectors.dense(1, 4), Vectors.dense(2, 0)) + val cols = Seq(Vectors.dense(0, 1, 2), Vectors.dense(3, 4, 0)) + for (m <- Seq(dm, sm)) { + assert(m.rowIter.toSeq === rows) + assert(m.colIter.toSeq === cols) + assert(m.transpose.rowIter.toSeq === cols) + assert(m.transpose.colIter.toSeq === rows) + } + } + + test("conversions between new local linalg and mllib linalg") { + val dm: DenseMatrix = new DenseMatrix(3, 2, Array(0.0, 0.0, 1.0, 0.0, 2.0, 3.5)) + val sm: SparseMatrix = dm.toSparse + val sm0: Matrix = sm.asInstanceOf[Matrix] + val dm0: Matrix = dm.asInstanceOf[Matrix] + + def compare(oldM: Matrix, newM: newlinalg.Matrix): Unit = { + assert(oldM.toArray === newM.toArray) + assert(oldM.numCols === newM.numCols) + assert(oldM.numRows === newM.numRows) + } + + val newSM: newlinalg.SparseMatrix = sm.asML + val newDM: newlinalg.DenseMatrix = dm.asML + val newSM0: newlinalg.Matrix = sm0.asML + val newDM0: newlinalg.Matrix = dm0.asML + assert(newSM0.isInstanceOf[newlinalg.SparseMatrix]) + assert(newDM0.isInstanceOf[newlinalg.DenseMatrix]) + compare(sm, newSM) + compare(dm, newDM) + compare(sm0, newSM0) + compare(dm0, newDM0) + + val oldSM: SparseMatrix = SparseMatrix.fromML(newSM) + val oldDM: DenseMatrix = DenseMatrix.fromML(newDM) + val oldSM0: Matrix = Matrices.fromML(newSM0) + val oldDM0: Matrix = Matrices.fromML(newDM0) + assert(oldSM0.isInstanceOf[SparseMatrix]) + assert(oldDM0.isInstanceOf[DenseMatrix]) + compare(oldSM, newSM) + compare(oldDM, newDM) + compare(oldSM0, newSM0) + compare(oldDM0, newDM0) + } + + test("implicit conversions between new local linalg and mllib linalg") { + + def mllibMatrixToTriple(m: Matrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mllibDenseMatrixToTriple(m: DenseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mllibSparseMatrixToTriple(m: SparseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mlMatrixToTriple(m: newlinalg.Matrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mlDenseMatrixToTriple(m: newlinalg.DenseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def mlSparseMatrixToTriple(m: newlinalg.SparseMatrix): (Array[Double], Int, Int) = + (m.toArray, m.numCols, m.numRows) + + def compare(m1: (Array[Double], Int, Int), m2: (Array[Double], Int, Int)): Unit = { + assert(m1._1 === m2._1) + assert(m1._2 === m2._2) + assert(m1._3 === m2._3) + } + + val dm: DenseMatrix = new DenseMatrix(3, 2, Array(0.0, 0.0, 1.0, 0.0, 2.0, 3.5)) + val sm: SparseMatrix = dm.toSparse + val sm0: Matrix = sm.asInstanceOf[Matrix] + val dm0: Matrix = dm.asInstanceOf[Matrix] + + val newSM: newlinalg.SparseMatrix = sm.asML + val newDM: newlinalg.DenseMatrix = dm.asML + val newSM0: newlinalg.Matrix = sm0.asML + val newDM0: newlinalg.Matrix = dm0.asML + + import org.apache.spark.mllib.linalg.MatrixImplicits._ + + compare(mllibMatrixToTriple(dm0), mllibMatrixToTriple(newDM0)) + compare(mllibMatrixToTriple(sm0), mllibMatrixToTriple(newSM0)) + + compare(mllibDenseMatrixToTriple(dm), mllibDenseMatrixToTriple(newDM)) + compare(mllibSparseMatrixToTriple(sm), mllibSparseMatrixToTriple(newSM)) + + compare(mlMatrixToTriple(dm0), mlMatrixToTriple(newDM)) + compare(mlMatrixToTriple(sm0), mlMatrixToTriple(newSM0)) + + compare(mlDenseMatrixToTriple(dm), mlDenseMatrixToTriple(newDM)) + compare(mlSparseMatrixToTriple(sm), mlSparseMatrixToTriple(newSM)) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala new file mode 100644 index 000000000000..5973479dfb5e --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg + +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.util.Benchmark + +/** + * Serialization benchmark for VectorUDT. + */ +object UDTSerializationBenchmark { + + def main(args: Array[String]): Unit = { + val iters = 1e2.toInt + val numRows = 1e3.toInt + + val encoder = ExpressionEncoder[Vector].resolveAndBind() + + val vectors = (1 to numRows).map { i => + Vectors.dense(Array.fill(1e5.toInt)(1.0 * i)) + }.toArray + val rows = vectors.map(encoder.toRow) + + val benchmark = new Benchmark("VectorUDT de/serialization", numRows, iters) + + benchmark.addCase("serialize") { _ => + var sum = 0 + var i = 0 + while (i < numRows) { + sum += encoder.toRow(vectors(i)).numFields + i += 1 + } + } + + benchmark.addCase("deserialize") { _ => + var sum = 0 + var i = 0 + while (i < numRows) { + sum += encoder.fromRow(rows(i)).numActives + i += 1 + } + } + + /* + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + VectorUDT de/serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + serialize 265 / 318 0.0 265138.5 1.0X + deserialize 155 / 197 0.0 154611.4 1.7X + */ + benchmark.run() + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 6508ddeba420..a1e3ee54b49f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -19,9 +19,12 @@ package org.apache.spark.mllib.linalg import scala.util.Random -import breeze.linalg.{DenseMatrix => BDM, squaredDistance => breezeSquaredDistance} +import breeze.linalg.{squaredDistance => breezeSquaredDistance, DenseMatrix => BDM} +import org.json4s.jackson.JsonMethods.{parse => parseJson} -import org.apache.spark.{Logging, SparkException, SparkFunSuite} +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.internal.Logging +import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.mllib.util.TestingUtils._ class VectorsSuite extends SparkFunSuite with Logging { @@ -119,6 +122,13 @@ class VectorsSuite extends SparkFunSuite with Logging { val vec8 = Vectors.sparse(5, Array(1, 2), Array(0.0, -1.0)) assert(vec8.argmax === 0) + + // Check for case when sparse vector is non-empty but the values are empty + val vec9 = Vectors.sparse(100, Array.empty[Int], Array.empty[Double]).asInstanceOf[SparseVector] + assert(vec9.argmax === 0) + + val vec10 = Vectors.sparse(1, Array.empty[Int], Array.empty[Double]).asInstanceOf[SparseVector] + assert(vec10.argmax === 0) } test("vector equals") { @@ -266,7 +276,7 @@ class VectorsSuite extends SparkFunSuite with Logging { val denseVector1 = Vectors.dense(sparseVector1.toArray) val denseVector2 = Vectors.dense(sparseVector2.toArray) - val squaredDist = breezeSquaredDistance(sparseVector1.toBreeze, sparseVector2.toBreeze) + val squaredDist = breezeSquaredDistance(sparseVector1.asBreeze, sparseVector2.asBreeze) // SparseVector vs. SparseVector assert(Vectors.sqdist(sparseVector1, sparseVector2) ~== squaredDist relTol 1E-8) @@ -341,11 +351,21 @@ class VectorsSuite extends SparkFunSuite with Logging { assert(dv0s.numActives === 2) assert(dv0s === dv0) + assert(dv0.toSparseWithSize(dv0.numNonzeros) === dv0) + val dv0s2 = dv0.toSparseWithSize(dv0.numNonzeros) + assert(dv0s2.numActives === 2) + assert(dv0s2 === dv0s) + val sv0 = Vectors.sparse(4, Array(0, 1, 2), Array(0.0, 2.0, 3.0)) assert(sv0.toDense === sv0) val sv0s = sv0.toSparse assert(sv0s.numActives === 2) assert(sv0s === sv0) + + assert(sv0.toSparseWithSize(sv0.numNonzeros) === sv0) + val sv0s2 = sv0.toSparseWithSize(sv0.numNonzeros) + assert(sv0s2.numActives === 2) + assert(sv0s2 === sv0s) } test("Vector.compressed") { @@ -374,4 +394,88 @@ class VectorsSuite extends SparkFunSuite with Logging { assert(v.slice(Array(2, 0)) === new SparseVector(2, Array(0), Array(2.2))) assert(v.slice(Array(2, 0, 3, 4)) === new SparseVector(4, Array(0, 3), Array(2.2, 4.4))) } + + test("toJson/fromJson") { + val sv0 = Vectors.sparse(0, Array.empty, Array.empty) + val sv1 = Vectors.sparse(1, Array.empty, Array.empty) + val sv2 = Vectors.sparse(2, Array(1), Array(2.0)) + val dv0 = Vectors.dense(Array.empty[Double]) + val dv1 = Vectors.dense(1.0) + val dv2 = Vectors.dense(0.0, 2.0) + for (v <- Seq(sv0, sv1, sv2, dv0, dv1, dv2)) { + val json = v.toJson + parseJson(json) // `json` should be a valid JSON string + val u = Vectors.fromJson(json) + assert(u.getClass === v.getClass, "toJson/fromJson should preserve vector types.") + assert(u === v, "toJson/fromJson should preserve vector values.") + } + } + + test("conversions between new local linalg and mllib linalg") { + val dv: DenseVector = new DenseVector(Array(1.0, 2.0, 3.5)) + val sv: SparseVector = new SparseVector(5, Array(1, 2, 4), Array(1.1, 2.2, 4.4)) + val sv0: Vector = sv.asInstanceOf[Vector] + val dv0: Vector = dv.asInstanceOf[Vector] + + val newSV: newlinalg.SparseVector = sv.asML + val newDV: newlinalg.DenseVector = dv.asML + val newSV0: newlinalg.Vector = sv0.asML + val newDV0: newlinalg.Vector = dv0.asML + assert(newSV0.isInstanceOf[newlinalg.SparseVector]) + assert(newDV0.isInstanceOf[newlinalg.DenseVector]) + assert(sv.toArray === newSV.toArray) + assert(dv.toArray === newDV.toArray) + assert(sv0.toArray === newSV0.toArray) + assert(dv0.toArray === newDV0.toArray) + + val oldSV: SparseVector = SparseVector.fromML(newSV) + val oldDV: DenseVector = DenseVector.fromML(newDV) + val oldSV0: Vector = Vectors.fromML(newSV0) + val oldDV0: Vector = Vectors.fromML(newDV0) + assert(oldSV0.isInstanceOf[SparseVector]) + assert(oldDV0.isInstanceOf[DenseVector]) + assert(oldSV.toArray === newSV.toArray) + assert(oldDV.toArray === newDV.toArray) + assert(oldSV0.toArray === newSV0.toArray) + assert(oldDV0.toArray === newDV0.toArray) + } + + test("implicit conversions between new local linalg and mllib linalg") { + + def mllibVectorToArray(v: Vector): Array[Double] = v.toArray + + def mllibDenseVectorToArray(v: DenseVector): Array[Double] = v.toArray + + def mllibSparseVectorToArray(v: SparseVector): Array[Double] = v.toArray + + def mlVectorToArray(v: newlinalg.Vector): Array[Double] = v.toArray + + def mlDenseVectorToArray(v: newlinalg.DenseVector): Array[Double] = v.toArray + + def mlSparseVectorToArray(v: newlinalg.SparseVector): Array[Double] = v.toArray + + val dv: DenseVector = new DenseVector(Array(1.0, 2.0, 3.5)) + val sv: SparseVector = new SparseVector(5, Array(1, 2, 4), Array(1.1, 2.2, 4.4)) + val sv0: Vector = sv.asInstanceOf[Vector] + val dv0: Vector = dv.asInstanceOf[Vector] + + val newSV: newlinalg.SparseVector = sv.asML + val newDV: newlinalg.DenseVector = dv.asML + val newSV0: newlinalg.Vector = sv0.asML + val newDV0: newlinalg.Vector = dv0.asML + + import org.apache.spark.mllib.linalg.VectorImplicits._ + + assert(mllibVectorToArray(dv0) === mllibVectorToArray(newDV0)) + assert(mllibVectorToArray(sv0) === mllibVectorToArray(newSV0)) + + assert(mllibDenseVectorToArray(dv) === mllibDenseVectorToArray(newDV)) + assert(mllibSparseVectorToArray(sv) === mllibSparseVectorToArray(newSV)) + + assert(mlVectorToArray(dv0) === mlVectorToArray(newDV0)) + assert(mlVectorToArray(sv0) === mlVectorToArray(newSV0)) + + assert(mlDenseVectorToArray(dv) === mlDenseVectorToArray(newDV)) + assert(mlSparseVectorToArray(sv) === mlSparseVectorToArray(newSV)) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala index b8eb10305801..f6a996940291 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrixSuite.scala @@ -19,10 +19,10 @@ package org.apache.spark.mllib.linalg.distributed import java.{util => ju} -import breeze.linalg.{DenseMatrix => BDM} +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, SparseVector => BSV} import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.mllib.linalg.{SparseMatrix, DenseMatrix, Matrices, Matrix} +import org.apache.spark.mllib.linalg.{DenseMatrix, DenseVector, Matrices, Matrix, SparseMatrix, SparseVector, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ @@ -134,6 +134,38 @@ class BlockMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { assert(rowMat.numRows() === m) assert(rowMat.numCols() === n) assert(rowMat.toBreeze() === gridBasedMat.toBreeze()) + + // SPARK-15922: BlockMatrix to IndexedRowMatrix throws an error" + val bmat = rowMat.toBlockMatrix + val imat = bmat.toIndexedRowMatrix + imat.rows.collect + + val rows = 1 + val cols = 10 + + val matDense = new DenseMatrix(rows, cols, + Array(1.0, 1.0, 3.0, 2.0, 5.0, 6.0, 7.0, 1.0, 2.0, 3.0)) + val matSparse = new SparseMatrix(rows, cols, + Array(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1), Array(0), Array(1.0)) + + val vectors: Seq[((Int, Int), Matrix)] = Seq( + ((0, 0), matDense), + ((1, 0), matSparse)) + + val rdd = sc.parallelize(vectors) + val B = new BlockMatrix(rdd, rows, cols) + + val C = B.toIndexedRowMatrix.rows.collect + + (C(0).vector.asBreeze, C(1).vector.asBreeze) match { + case (denseVector: BDV[Double], sparseVector: BSV[Double]) => + assert(denseVector.length === sparseVector.length) + + assert(matDense.toArray === denseVector.toArray) + assert(matSparse.toArray === sparseVector.toArray) + case _ => + throw new RuntimeException("IndexedRow returns vectors of unexpected type") + } } test("toBreeze and toLocalMatrix") { @@ -192,6 +224,58 @@ class BlockMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { assert(sparseBM.add(sparseBM).toBreeze() === sparseBM.add(denseBM).toBreeze()) } + test("subtract") { + val blocks: Seq[((Int, Int), Matrix)] = Seq( + ((0, 0), new DenseMatrix(2, 2, Array(1.0, 0.0, 0.0, 2.0))), + ((0, 1), new DenseMatrix(2, 2, Array(0.0, 1.0, 0.0, 0.0))), + ((1, 0), new DenseMatrix(2, 2, Array(3.0, 0.0, 1.0, 1.0))), + ((1, 1), new DenseMatrix(2, 2, Array(1.0, 2.0, 0.0, 1.0))), + ((2, 0), new DenseMatrix(1, 2, Array(1.0, 0.0))), // Added block that doesn't exist in A + ((2, 1), new DenseMatrix(1, 2, Array(1.0, 5.0)))) + val rdd = sc.parallelize(blocks, numPartitions) + val B = new BlockMatrix(rdd, rowPerPart, colPerPart) + + val expected = BDM( + (0.0, 0.0, 0.0, 0.0), + (0.0, 0.0, 0.0, 0.0), + (0.0, 0.0, 0.0, 0.0), + (0.0, 0.0, 0.0, 0.0), + (-1.0, 0.0, 0.0, 0.0)) + + val AsubtractB = gridBasedMat.subtract(B) + assert(AsubtractB.numRows() === m) + assert(AsubtractB.numCols() === B.numCols()) + assert(AsubtractB.toBreeze() === expected) + + val C = new BlockMatrix(rdd, rowPerPart, colPerPart, m, n + 1) // columns don't match + intercept[IllegalArgumentException] { + gridBasedMat.subtract(C) + } + val largerBlocks: Seq[((Int, Int), Matrix)] = Seq( + ((0, 0), new DenseMatrix(4, 4, new Array[Double](16))), + ((1, 0), new DenseMatrix(1, 4, Array(1.0, 0.0, 1.0, 5.0)))) + val C2 = new BlockMatrix(sc.parallelize(largerBlocks, numPartitions), 4, 4, m, n) + intercept[SparkException] { // partitioning doesn't match + gridBasedMat.subtract(C2) + } + // subtracting BlockMatrices composed of SparseMatrices + val sparseBlocks = for (i <- 0 until 4) yield ((i / 2, i % 2), SparseMatrix.speye(4)) + val denseBlocks = for (i <- 0 until 4) yield ((i / 2, i % 2), DenseMatrix.eye(4)) + val sparseBM = new BlockMatrix(sc.makeRDD(sparseBlocks, 4), 4, 4, 8, 8) + val denseBM = new BlockMatrix(sc.makeRDD(denseBlocks, 4), 4, 4, 8, 8) + + assert(sparseBM.subtract(sparseBM).toBreeze() === sparseBM.subtract(denseBM).toBreeze()) + } + + def testMultiply(A: BlockMatrix, B: BlockMatrix, expectedResult: Matrix, + numMidDimSplits: Int): Unit = { + val C = A.multiply(B, numMidDimSplits) + val localC = C.toLocalMatrix() + assert(C.numRows() === A.numRows()) + assert(C.numCols() === B.numCols()) + assert(localC ~== expectedResult absTol 1e-8) + } + test("multiply") { // identity matrix val blocks: Seq[((Int, Int), Matrix)] = Seq( @@ -227,12 +311,13 @@ class BlockMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { // Try it with increased number of partitions val largeA = new BlockMatrix(sc.parallelize(largerAblocks, 10), 6, 4) val largeB = new BlockMatrix(sc.parallelize(largerBblocks, 8), 4, 4) - val largeC = largeA.multiply(largeB) - val localC = largeC.toLocalMatrix() + val result = largeA.toLocalMatrix().multiply(largeB.toLocalMatrix().asInstanceOf[DenseMatrix]) - assert(largeC.numRows() === largeA.numRows()) - assert(largeC.numCols() === largeB.numCols()) - assert(localC ~== result absTol 1e-8) + + testMultiply(largeA, largeB, result, 1) + testMultiply(largeA, largeB, result, 2) + testMultiply(largeA, largeB, result, 3) + testMultiply(largeA, largeB, result, 4) } test("simulate multiply") { @@ -243,7 +328,7 @@ class BlockMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { val B = new BlockMatrix(rdd, colPerPart, rowPerPart) val resultPartitioner = GridPartitioner(gridBasedMat.numRowBlocks, B.numColBlocks, math.max(numPartitions, 2)) - val (destinationsA, destinationsB) = gridBasedMat.simulateMultiply(B, resultPartitioner) + val (destinationsA, destinationsB) = gridBasedMat.simulateMultiply(B, resultPartitioner, 1) assert(destinationsA((0, 0)) === Set(0)) assert(destinationsA((0, 1)) === Set(2)) assert(destinationsA((1, 0)) === Set(0)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala index f3728cd036a3..37d75103d18d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala @@ -20,8 +20,8 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLlibTestSparkContext class CoordinateMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala index 6de6cf2fa863..566ce95be084 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{diag => brzDiag, DenseMatrix => BDM, DenseVector => BDV} import org.apache.spark.SparkFunSuite +import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.mllib.linalg.{Matrices, Vectors} class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -87,19 +87,96 @@ class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { assert(coordMat.toBreeze() === idxRowMat.toBreeze()) } - test("toBlockMatrix") { - val idxRowMat = new IndexedRowMatrix(indexedRows) - val blockMat = idxRowMat.toBlockMatrix(2, 2) + test("toBlockMatrix dense backing") { + val idxRowMatDense = new IndexedRowMatrix(indexedRows) + + // Tests when n % colsPerBlock != 0 + val blockMat = idxRowMatDense.toBlockMatrix(2, 2) assert(blockMat.numRows() === m) assert(blockMat.numCols() === n) - assert(blockMat.toBreeze() === idxRowMat.toBreeze()) + assert(blockMat.toBreeze() === idxRowMatDense.toBreeze()) + + // Tests when m % rowsPerBlock != 0 + val blockMat2 = idxRowMatDense.toBlockMatrix(3, 1) + assert(blockMat2.numRows() === m) + assert(blockMat2.numCols() === n) + assert(blockMat2.toBreeze() === idxRowMatDense.toBreeze()) intercept[IllegalArgumentException] { - idxRowMat.toBlockMatrix(-1, 2) + idxRowMatDense.toBlockMatrix(-1, 2) } intercept[IllegalArgumentException] { - idxRowMat.toBlockMatrix(2, 0) + idxRowMatDense.toBlockMatrix(2, 0) } + + assert(blockMat.blocks.map { case (_, matrix: Matrix) => + matrix.isInstanceOf[DenseMatrix] + }.reduce(_ && _)) + assert(blockMat2.blocks.map { case (_, matrix: Matrix) => + matrix.isInstanceOf[DenseMatrix] + }.reduce(_ && _)) + } + + test("toBlockMatrix sparse backing") { + val sparseData = Seq( + (15L, Vectors.sparse(12, Seq((0, 4.0)))) + ).map(x => IndexedRow(x._1, x._2)) + + // Gonna make m and n larger here so the matrices can easily be completely sparse: + val m = 16 + val n = 12 + + val idxRowMatSparse = new IndexedRowMatrix(sc.parallelize(sparseData)) + + // Tests when n % colsPerBlock != 0 + val blockMat = idxRowMatSparse.toBlockMatrix(8, 8) + assert(blockMat.numRows() === m) + assert(blockMat.numCols() === n) + assert(blockMat.toBreeze() === idxRowMatSparse.toBreeze()) + + // Tests when m % rowsPerBlock != 0 + val blockMat2 = idxRowMatSparse.toBlockMatrix(6, 6) + assert(blockMat2.numRows() === m) + assert(blockMat2.numCols() === n) + assert(blockMat2.toBreeze() === idxRowMatSparse.toBreeze()) + + assert(blockMat.blocks.collect().forall{ case (_, matrix: Matrix) => + matrix.isInstanceOf[SparseMatrix] + }) + assert(blockMat2.blocks.collect().forall{ case (_, matrix: Matrix) => + matrix.isInstanceOf[SparseMatrix] + }) + } + + test("toBlockMatrix mixed backing") { + val m = 24 + val n = 18 + + val mixedData = Seq( + (0L, Vectors.dense((0 to 17).map(_.toDouble).toArray)), + (1L, Vectors.dense((0 to 17).map(_.toDouble).toArray)), + (23L, Vectors.sparse(18, Seq((0, 4.0))))) + .map(x => IndexedRow(x._1, x._2)) + + val idxRowMatMixed = new IndexedRowMatrix( + sc.parallelize(mixedData)) + + // Tests when n % colsPerBlock != 0 + val blockMat = idxRowMatMixed.toBlockMatrix(12, 12) + assert(blockMat.numRows() === m) + assert(blockMat.numCols() === n) + assert(blockMat.toBreeze() === idxRowMatMixed.toBreeze()) + + // Tests when m % rowsPerBlock != 0 + val blockMat2 = idxRowMatMixed.toBlockMatrix(18, 6) + assert(blockMat2.numRows() === m) + assert(blockMat2.numCols() === n) + assert(blockMat2.toBreeze() === idxRowMatMixed.toBreeze()) + + val blocks = blockMat.blocks.collect() + + assert(blocks.forall { case((row, col), matrix) => + if (row == 0) matrix.isInstanceOf[DenseMatrix] else matrix.isInstanceOf[SparseMatrix]}) } test("multiply a local matrix") { @@ -108,7 +185,7 @@ class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { val C = A.multiply(B) val localA = A.toBreeze() val localC = C.toBreeze() - val expected = localA * B.toBreeze.asInstanceOf[BDM[Double]] + val expected = localA * B.asBreeze.asInstanceOf[BDM[Double]] assert(localC === expected) } @@ -119,7 +196,7 @@ class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { (90.0, 12.0, 24.0), (12.0, 17.0, 22.0), (24.0, 22.0, 30.0)) - assert(G.toBreeze === expected) + assert(G.asBreeze === expected) } test("svd") { @@ -128,8 +205,8 @@ class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { assert(svd.U.isInstanceOf[IndexedRowMatrix]) val localA = A.toBreeze() val U = svd.U.toBreeze() - val s = svd.s.toBreeze.asInstanceOf[BDV[Double]] - val V = svd.V.toBreeze.asInstanceOf[BDM[Double]] + val s = svd.s.asBreeze.asInstanceOf[BDV[Double]] + val V = svd.V.asBreeze.asInstanceOf[BDM[Double]] assert(closeToZero(U.t * U - BDM.eye[Double](n))) assert(closeToZero(V.t * V - BDM.eye[Double](n))) assert(closeToZero(U * brzDiag(s) * V.t - localA)) @@ -155,7 +232,7 @@ class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { test("similar columns") { val A = new IndexedRowMatrix(indexedRows) - val gram = A.computeGramianMatrix().toBreeze.toDenseMatrix + val gram = A.computeGramianMatrix().asBreeze.toDenseMatrix val G = A.columnSimilarities().toBreeze() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index 4abb98fb6fe4..7c9e14f8cee7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -17,15 +17,18 @@ package org.apache.spark.mllib.linalg.distributed +import java.util.Arrays + import scala.util.Random +import breeze.linalg.{norm => brzNorm, svd => brzSvd, DenseMatrix => BDM, DenseVector => BDV} import breeze.numerics.abs -import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, norm => brzNorm, svd => brzSvd} import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{Matrices, Vectors, Vector} +import org.apache.spark.mllib.linalg.{Matrices, Vector, Vectors} import org.apache.spark.mllib.random.RandomRDDs import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext} +import org.apache.spark.mllib.util.TestingUtils._ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -49,6 +52,7 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { (0.0, 1.0, 0.0), (math.sqrt(2.0) / 2.0, 0.0, math.sqrt(2.0) / 2.0), (math.sqrt(2.0) / 2.0, 0.0, - math.sqrt(2.0) / 2.0)) + val explainedVariance = BDV(4.0 / 7.0, 3.0 / 7.0, 0.0) var denseMat: RowMatrix = _ var sparseMat: RowMatrix = _ @@ -93,7 +97,7 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { Matrices.dense(n, n, Array(126.0, 54.0, 72.0, 54.0, 66.0, 78.0, 72.0, 78.0, 94.0)) for (mat <- Seq(denseMat, sparseMat)) { val G = mat.computeGramianMatrix() - assert(G.toBreeze === expected.toBreeze) + assert(G.asBreeze === expected.asBreeze) } } @@ -150,8 +154,8 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { assert(V.numRows === n) assert(V.numCols === k) assertColumnEqualUpToSign(U.toBreeze(), localU, k) - assertColumnEqualUpToSign(V.toBreeze.asInstanceOf[BDM[Double]], localV, k) - assert(closeToZero(s.toBreeze.asInstanceOf[BDV[Double]] - localSigma(0 until k))) + assertColumnEqualUpToSign(V.asBreeze.asInstanceOf[BDM[Double]], localV, k) + assert(closeToZero(s.asBreeze.asInstanceOf[BDV[Double]] - localSigma(0 until k))) } } val svdWithoutU = mat.computeSVD(1, computeU = false, 1e-9, 300, 1e-10, mode) @@ -201,10 +205,15 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { test("pca") { for (mat <- Seq(denseMat, sparseMat); k <- 1 to n) { - val pc = denseMat.computePrincipalComponents(k) + val (pc, expVariance) = mat.computePrincipalComponentsAndExplainedVariance(k) assert(pc.numRows === n) assert(pc.numCols === k) - assertColumnEqualUpToSign(pc.toBreeze.asInstanceOf[BDM[Double]], principalComponents, k) + assertColumnEqualUpToSign(pc.asBreeze.asInstanceOf[BDM[Double]], principalComponents, k) + assert( + closeToZero(BDV(expVariance.toArray) - + BDV(Arrays.copyOfRange(explainedVariance.data, 0, k)))) + // Check that this method returns the same answer + assert(pc === mat.computePrincipalComponents(k)) } } @@ -248,12 +257,12 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { val calcQ = result.Q val calcR = result.R assert(closeToZero(abs(expected.q) - abs(calcQ.toBreeze()))) - assert(closeToZero(abs(expected.r) - abs(calcR.toBreeze.asInstanceOf[BDM[Double]]))) + assert(closeToZero(abs(expected.r) - abs(calcR.asBreeze.asInstanceOf[BDM[Double]]))) assert(closeToZero(calcQ.multiply(calcR).toBreeze - mat.toBreeze())) // Decomposition without computing Q val rOnly = mat.tallSkinnyQR(computeQ = false) assert(rOnly.Q == null) - assert(closeToZero(abs(expected.r) - abs(rOnly.R.toBreeze.asInstanceOf[BDM[Double]]))) + assert(closeToZero(abs(expected.r) - abs(rOnly.R.asBreeze.asInstanceOf[BDM[Double]]))) } } @@ -261,7 +270,7 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { for (mat <- Seq(denseMat, sparseMat)) { val result = mat.computeCovariance() val expected = breeze.linalg.cov(mat.toBreeze()) - assert(closeToZero(abs(expected) - abs(result.toBreeze.asInstanceOf[BDM[Double]]))) + assert(closeToZero(abs(expected) - abs(result.asBreeze.asInstanceOf[BDM[Double]]))) } } @@ -273,6 +282,22 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { assert(cov(i, j) === cov(j, i)) } } + + test("QR decomposition should aware of empty partition (SPARK-16369)") { + val mat: RowMatrix = new RowMatrix(sc.parallelize(denseData, 1)) + val qrResult = mat.tallSkinnyQR(true) + + val matWithEmptyPartition = new RowMatrix(sc.parallelize(denseData, 8)) + val qrResult2 = matWithEmptyPartition.tallSkinnyQR(true) + + assert(qrResult.Q.numCols() === qrResult2.Q.numCols(), "Q matrix ncol not match") + assert(qrResult.Q.numRows() === qrResult2.Q.numRows(), "Q matrix nrow not match") + qrResult.Q.rows.collect().zip(qrResult2.Q.rows.collect()) + .foreach(x => assert(x._1 ~== x._2 relTol 1E-8, "Q matrix not match")) + + qrResult.R.toArray.zip(qrResult2.R.toArray) + .foreach(x => assert(x._1 ~== x._2 relTol 1E-8, "R matrix not match")) + } } class RowMatrixClusterSuite extends SparkFunSuite with LocalClusterSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala index 36ac7d267243..37eb794b0c5c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.{MLUtils, LocalClusterSparkContext, MLlibTestSparkContext} +import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext, MLUtils} import org.apache.spark.mllib.util.TestingUtils._ object GradientDescentSuite { @@ -131,7 +131,7 @@ class GradientDescentSuite extends SparkFunSuite with MLlibTestSparkContext with assert( loss1(0) ~= (loss0(0) + (math.pow(initialWeightsWithIntercept(0), 2) + math.pow(initialWeightsWithIntercept(1), 2)) / 2) absTol 1E-5, - """For non-zero weights, the regVal should be \frac{1}{2}\sum_i w_i^2.""") + """For non-zero weights, the regVal should be 0.5 * sum(w_i ^ 2).""") assert( (newWeights1(0) ~= (newWeights0(0) - initialWeightsWithIntercept(0)) absTol 1E-5) && diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index 75ae0eb32fb7..69c303ee932e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -191,7 +191,7 @@ class LBFGSSuite extends SparkFunSuite with MLlibTestSparkContext with Matchers // With smaller convergenceTol, it takes more steps. assert(lossLBFGS3.length > lossLBFGS2.length) - // Based on observation, lossLBFGS2 runs 5 iterations, no theoretically guaranteed. + // Based on observation, lossLBFGS3 runs 6 iterations, no theoretically guaranteed. assert(lossLBFGS3.length == 6) assert((lossLBFGS3(4) - lossLBFGS3(5)) / lossLBFGS3(4) < convergenceTol) } @@ -230,6 +230,25 @@ class LBFGSSuite extends SparkFunSuite with MLlibTestSparkContext with Matchers (weightLBFGS(0) ~= weightGD(0) relTol 0.02) && (weightLBFGS(1) ~= weightGD(1) relTol 0.02), "The weight differences between LBFGS and GD should be within 2%.") } + + test("SPARK-18471: LBFGS aggregator on empty partitions") { + val regParam = 0 + + val initialWeightsWithIntercept = Vectors.dense(0.0) + val convergenceTol = 1e-12 + val numIterations = 1 + val dataWithEmptyPartitions = sc.parallelize(Seq((1.0, Vectors.dense(2.0))), 2) + + LBFGS.runLBFGS( + dataWithEmptyPartitions, + gradient, + simpleUpdater, + numCorrections, + convergenceTol, + numIterations, + regParam, + initialWeightsWithIntercept) + } } class LBFGSClusterSuite extends SparkFunSuite with LocalClusterSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala index d8f9b8c33963..4ec3dc0df03b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala @@ -19,28 +19,22 @@ package org.apache.spark.mllib.optimization import scala.util.Random -import org.jblas.{DoubleMatrix, SimpleBlas} +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV} import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.TestingUtils._ class NNLSSuite extends SparkFunSuite { /** Generate an NNLS problem whose optimal solution is the all-ones vector. */ - def genOnesData(n: Int, rand: Random): (DoubleMatrix, DoubleMatrix) = { - val A = new DoubleMatrix(n, n, Array.fill(n*n)(rand.nextDouble()): _*) - val b = A.mmul(DoubleMatrix.ones(n, 1)) - - val ata = A.transpose.mmul(A) - val atb = A.transpose.mmul(b) - - (ata, atb) + def genOnesData(n: Int, rand: Random): (BDM[Double], BDV[Double]) = { + val A = new BDM(n, n, Array.fill(n*n)(rand.nextDouble())) + val b = A * new BDV(Array.fill(n)(1.0)) + (A.t * A, A.t * b) } /** Compute the objective value */ - def computeObjectiveValue(ata: DoubleMatrix, atb: DoubleMatrix, x: DoubleMatrix): Double = { - val res = (x.transpose().mmul(ata).mmul(x)).mul(0.5).sub(atb.dot(x)) - res.get(0) - } + def computeObjectiveValue(ata: BDM[Double], atb: BDV[Double], x: BDV[Double]): Double = + (x.t * ata * x) / 2.0 - atb.dot(x) test("NNLS: exact solution cases") { val n = 20 @@ -54,12 +48,15 @@ class NNLSSuite extends SparkFunSuite { for (k <- 0 until 100) { val (ata, atb) = genOnesData(n, rand) - val x = new DoubleMatrix(NNLS.solve(ata.data, atb.data, ws)) + val x = new BDV(NNLS.solve(ata.data, atb.data, ws)) assert(x.length === n) - val answer = DoubleMatrix.ones(n, 1) - SimpleBlas.axpy(-1.0, answer, x) - val solved = (x.norm2 < 1e-2) && (x.normmax < 1e-3) - if (solved) numSolved = numSolved + 1 + val answer = new BDV(Array.fill(n)(1.0)) + val solved = + (breeze.linalg.norm(x - answer) < 0.01) && // L2 norm + ((x - answer).toArray.map(_.abs).max < 0.001) // inf norm + if (solved) { + numSolved += 1 + } } assert(numSolved > 50) @@ -67,20 +64,18 @@ class NNLSSuite extends SparkFunSuite { test("NNLS: nonnegativity constraint active") { val n = 5 - // scalastyle:off - val ata = new DoubleMatrix(Array( - Array( 4.377, -3.531, -1.306, -0.139, 3.418), - Array(-3.531, 4.344, 0.934, 0.305, -2.140), - Array(-1.306, 0.934, 2.644, -0.203, -0.170), - Array(-0.139, 0.305, -0.203, 5.883, 1.428), - Array( 3.418, -2.140, -0.170, 1.428, 4.684))) - // scalastyle:on - val atb = new DoubleMatrix(Array(-1.632, 2.115, 1.094, -1.025, -0.636)) + val ata = Array( + 4.377, -3.531, -1.306, -0.139, 3.418, + -3.531, 4.344, 0.934, 0.305, -2.140, + -1.306, 0.934, 2.644, -0.203, -0.170, + -0.139, 0.305, -0.203, 5.883, 1.428, + 3.418, -2.140, -0.170, 1.428, 4.684) + val atb = Array(-1.632, 2.115, 1.094, -1.025, -0.636) val goodx = Array(0.13025, 0.54506, 0.2874, 0.0, 0.028628) val ws = NNLS.createWorkspace(n) - val x = NNLS.solve(ata.data, atb.data, ws) + val x = NNLS.solve(ata, atb, ws) for (i <- 0 until n) { assert(x(i) ~== goodx(i) absTol 1E-3) assert(x(i) >= 0) @@ -89,23 +84,21 @@ class NNLSSuite extends SparkFunSuite { test("NNLS: objective value test") { val n = 5 - val ata = new DoubleMatrix(5, 5 - , 517399.13534, 242529.67289, -153644.98976, 130802.84503, -798452.29283 - , 242529.67289, 126017.69765, -75944.21743, 81785.36128, -405290.60884 - , -153644.98976, -75944.21743, 46986.44577, -45401.12659, 247059.51049 - , 130802.84503, 81785.36128, -45401.12659, 67457.31310, -253747.03819 - , -798452.29283, -405290.60884, 247059.51049, -253747.03819, 1310939.40814 - ) - val atb = new DoubleMatrix(5, 1, - -31755.05710, 13047.14813, -20191.24443, 25993.77580, 11963.55017) + val ata = new BDM(5, 5, Array( + 517399.13534, 242529.67289, -153644.98976, 130802.84503, -798452.29283, + 242529.67289, 126017.69765, -75944.21743, 81785.36128, -405290.60884, + -153644.98976, -75944.21743, 46986.44577, -45401.12659, 247059.51049, + 130802.84503, 81785.36128, -45401.12659, 67457.31310, -253747.03819, + -798452.29283, -405290.60884, 247059.51049, -253747.03819, 1310939.40814)) + val atb = new BDV(Array(-31755.05710, 13047.14813, -20191.24443, 25993.77580, 11963.55017)) /** reference solution obtained from matlab function quadprog */ - val refx = new DoubleMatrix(Array(34.90751, 103.96254, 0.00000, 27.82094, 58.79627)) + val refx = new BDV(Array(34.90751, 103.96254, 0.00000, 27.82094, 58.79627)) val refObj = computeObjectiveValue(ata, atb, refx) val ws = NNLS.createWorkspace(n) - val x = new DoubleMatrix(NNLS.solve(ata.data, atb.data, ws)) + val x = new BDV(NNLS.solve(ata.data, atb.data, ws)) val obj = computeObjectiveValue(ata, atb, x) assert(obj < refObj + 1E-5) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala index 8416771552fd..e30ad159676f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala @@ -80,7 +80,7 @@ class RandomDataGeneratorSuite extends SparkFunSuite { } test("LogNormalGenerator") { - List((0.0, 1.0), (0.0, 2.0), (2.0, 1.0), (2.0, 2.0)).map { + List((0.0, 1.0), (0.0, 2.0), (2.0, 1.0), (2.0, 2.0)).foreach { case (mean: Double, vari: Double) => val normal = new LogNormalGenerator(mean, math.sqrt(vari)) apiChecks(normal) @@ -125,7 +125,7 @@ class RandomDataGeneratorSuite extends SparkFunSuite { test("GammaGenerator") { // mean = 0.0 will not pass the API checks since 0.0 is always deterministically produced. - List((1.0, 2.0), (2.0, 2.0), (3.0, 2.0), (5.0, 1.0), (9.0, 0.5)).map { + List((1.0, 2.0), (2.0, 2.0), (3.0, 2.0), (5.0, 1.0), (9.0, 0.5)).foreach { case (shape: Double, scale: Double) => val gamma = new GammaGenerator(shape, scale) apiChecks(gamma) @@ -138,7 +138,7 @@ class RandomDataGeneratorSuite extends SparkFunSuite { } test("WeibullGenerator") { - List((1.0, 2.0), (2.0, 3.0), (2.5, 3.5), (10.4, 2.222)).map { + List((1.0, 2.0), (2.0, 3.0), (2.5, 3.5), (10.4, 2.222)).foreach { case (alpha: Double, beta: Double) => val weibull = new WeibullGenerator(alpha, beta) apiChecks(weibull) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala index 413db2000d6d..f464d25c3fbd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala @@ -20,9 +20,8 @@ package org.apache.spark.mllib.random import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkFunSuite -import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.rdd.{RandomRDDPartition, RandomRDD} +import org.apache.spark.mllib.rdd.{RandomRDD, RandomRDDPartition} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.rdd.RDD import org.apache.spark.util.StatCounter diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala index 10f5a2be48f7..56231429859e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/MLPairRDDFunctionsSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.mllib.rdd import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.rdd.MLPairRDDFunctions._ +import org.apache.spark.mllib.util.MLlibTestSparkContext class MLPairRDDFunctionsSuite extends SparkFunSuite with MLlibTestSparkContext { test("topByKey") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala index bc6417261483..0e931fca6cf0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.mllib.rdd import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.rdd.RDDFunctions._ +import org.apache.spark.mllib.util.MLlibTestSparkContext class RDDFunctionsSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -28,9 +28,12 @@ class RDDFunctionsSuite extends SparkFunSuite with MLlibTestSparkContext { for (numPartitions <- 1 to 8) { val rdd = sc.parallelize(data, numPartitions) for (windowSize <- 1 to 6) { - val sliding = rdd.sliding(windowSize).collect().map(_.toList).toList - val expected = data.sliding(windowSize).map(_.toList).toList - assert(sliding === expected) + for (step <- 1 to 3) { + val sliding = rdd.sliding(windowSize, step).collect().map(_.toList).toList + val expected = data.sliding(windowSize, step) + .map(_.toList).toList.filter(l => l.size == windowSize) + assert(sliding === expected) + } } assert(rdd.sliding(7).collect().isEmpty, "Should return an empty RDD if the window size is greater than the number of items.") @@ -40,7 +43,7 @@ class RDDFunctionsSuite extends SparkFunSuite with MLlibTestSparkContext { test("sliding with empty partitions") { val data = Seq(Seq(1, 2, 3), Seq.empty[Int], Seq(4), Seq.empty[Int], Seq(5, 6, 7)) val rdd = sc.parallelize(data, data.length).flatMap(s => s) - assert(rdd.partitions.size === data.length) + assert(rdd.partitions.length === data.length) val sliding = rdd.sliding(3).collect().toSeq.map(_.toSeq) val expected = data.flatMap(x => x).sliding(3).toSeq.map(_.toSeq) assert(sliding === expected) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index 045135f7f8d6..b08ad99f4f20 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -21,7 +21,7 @@ import scala.collection.JavaConverters._ import scala.math.abs import scala.util.Random -import org.jblas.DoubleMatrix +import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -29,16 +29,16 @@ import org.apache.spark.storage.StorageLevel object ALSSuite { - def generateRatingsAsJavaList( + def generateRatingsAsJava( users: Int, products: Int, features: Int, samplingRate: Double, implicitPrefs: Boolean, - negativeWeights: Boolean): (java.util.List[Rating], DoubleMatrix, DoubleMatrix) = { + negativeWeights: Boolean): (java.util.List[Rating], Array[Double], Array[Double]) = { val (sampledRatings, trueRatings, truePrefs) = - generateRatings(users, products, features, samplingRate, implicitPrefs) - (sampledRatings.asJava, trueRatings, truePrefs) + generateRatings(users, products, features, samplingRate, implicitPrefs, negativeWeights) + (sampledRatings.asJava, trueRatings.toArray, if (truePrefs == null) null else truePrefs.toArray) } def generateRatings( @@ -48,35 +48,36 @@ object ALSSuite { samplingRate: Double, implicitPrefs: Boolean = false, negativeWeights: Boolean = false, - negativeFactors: Boolean = true): (Seq[Rating], DoubleMatrix, DoubleMatrix) = { + negativeFactors: Boolean = true): (Seq[Rating], BDM[Double], BDM[Double]) = { val rand = new Random(42) // Create a random matrix with uniform values from -1 to 1 def randomMatrix(m: Int, n: Int) = { if (negativeFactors) { - new DoubleMatrix(m, n, Array.fill(m * n)(rand.nextDouble() * 2 - 1): _*) + new BDM(m, n, Array.fill(m * n)(rand.nextDouble() * 2 - 1)) } else { - new DoubleMatrix(m, n, Array.fill(m * n)(rand.nextDouble()): _*) + new BDM(m, n, Array.fill(m * n)(rand.nextDouble())) } } val userMatrix = randomMatrix(users, features) val productMatrix = randomMatrix(features, products) - val (trueRatings, truePrefs) = implicitPrefs match { - case true => + val (trueRatings, truePrefs) = + if (implicitPrefs) { // Generate raw values from [0,9], or if negativeWeights, from [-2,7] - val raw = new DoubleMatrix(users, products, + val raw = new BDM(users, products, Array.fill(users * products)( - (if (negativeWeights) -2 else 0) + rand.nextInt(10).toDouble): _*) + (if (negativeWeights) -2 else 0) + rand.nextInt(10).toDouble)) val prefs = - new DoubleMatrix(users, products, raw.data.map(v => if (v > 0) 1.0 else 0.0): _*) + new BDM(users, products, raw.data.map(v => if (v > 0) 1.0 else 0.0)) (raw, prefs) - case false => (userMatrix.mmul(productMatrix), null) - } + } else { + (userMatrix * productMatrix, null) + } val sampledRatings = { for (u <- 0 until users; p <- 0 until products if rand.nextDouble() < samplingRate) - yield Rating(u, p, trueRatings.get(u, p)) + yield Rating(u, p, trueRatings(u, p)) } (sampledRatings, trueRatings, truePrefs) @@ -149,8 +150,8 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext { .setSeed(1) .setFinalRDDStorageLevel(storageLevel) .run(ratings) - assert(model.productFeatures.getStorageLevel == storageLevel); - assert(model.userFeatures.getStorageLevel == storageLevel); + assert(model.productFeatures.getStorageLevel == storageLevel) + assert(model.userFeatures.getStorageLevel == storageLevel) storageLevel = StorageLevel.DISK_ONLY model = new ALS() .setRank(5) @@ -160,8 +161,8 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext { .setSeed(1) .setFinalRDDStorageLevel(storageLevel) .run(ratings) - assert(model.productFeatures.getStorageLevel == storageLevel); - assert(model.userFeatures.getStorageLevel == storageLevel); + assert(model.productFeatures.getStorageLevel == storageLevel) + assert(model.userFeatures.getStorageLevel == storageLevel) } test("negative ids") { @@ -178,7 +179,7 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext { val u = r.user + 25 val p = r.product + 25 val v = r.rating - val error = v - correct.get(u, p) + val error = v - correct(u, p) assert(math.abs(error) < 0.4) } } @@ -187,6 +188,13 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext { testALS(100, 200, 2, 15, 0.7, 0.4, false, false, false, -1, -1, false) } + test("SPARK-18268: ALS with empty RDD should fail with better message") { + val ratings = sc.parallelize(Array.empty[Rating]) + intercept[IllegalArgumentException] { + new ALS().run(ratings) + } + } + /** * Test if we can correctly factorize R = U * P where U and P are of known rank. * @@ -197,7 +205,7 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext { * @param samplingRate what fraction of the user-product pairs are known * @param matchThreshold max difference allowed to consider a predicted rating correct * @param implicitPrefs flag to test implicit feedback - * @param bulkPredict flag to test bulk predicition + * @param bulkPredict flag to test bulk prediction * @param negativeWeights whether the generated data can contain negative values * @param numUserBlocks number of user blocks to partition users into * @param numProductBlocks number of product blocks to partition products into @@ -234,30 +242,31 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext { .setNonnegative(!negativeFactors) .run(sc.parallelize(sampledRatings)) - val predictedU = new DoubleMatrix(users, features) + val predictedU = new BDM[Double](users, features) for ((u, vec) <- model.userFeatures.collect(); i <- 0 until features) { - predictedU.put(u, i, vec(i)) + predictedU(u, i) = vec(i) } - val predictedP = new DoubleMatrix(products, features) + val predictedP = new BDM[Double](products, features) for ((p, vec) <- model.productFeatures.collect(); i <- 0 until features) { - predictedP.put(p, i, vec(i)) + predictedP(p, i) = vec(i) } - val predictedRatings = bulkPredict match { - case false => predictedU.mmul(predictedP.transpose) - case true => - val allRatings = new DoubleMatrix(users, products) + val predictedRatings = + if (bulkPredict) { + val allRatings = new BDM[Double](users, products) val usersProducts = for (u <- 0 until users; p <- 0 until products) yield (u, p) val userProductsRDD = sc.parallelize(usersProducts) model.predict(userProductsRDD).collect().foreach { elem => - allRatings.put(elem.user, elem.product, elem.rating) + allRatings(elem.user, elem.product) = elem.rating } allRatings - } + } else { + predictedU * predictedP.t + } if (!implicitPrefs) { for (u <- 0 until users; p <- 0 until products) { - val prediction = predictedRatings.get(u, p) - val correct = trueRatings.get(u, p) + val prediction = predictedRatings(u, p) + val correct = trueRatings(u, p) if (math.abs(prediction - correct) > matchThreshold) { fail(("Model failed to predict (%d, %d): %f vs %f\ncorr: %s\npred: %s\nU: %s\n P: %s") .format(u, p, correct, prediction, trueRatings, predictedRatings, predictedU, @@ -269,9 +278,9 @@ class ALSSuite extends SparkFunSuite with MLlibTestSparkContext { var sqErr = 0.0 var denom = 0.0 for (u <- 0 until users; p <- 0 until products) { - val prediction = predictedRatings.get(u, p) - val truePref = truePrefs.get(u, p) - val confidence = 1 + 1.0 * abs(trueRatings.get(u, p)) + val prediction = predictedRatings(u, p) + val truePref = truePrefs(u, p) + val confidence = 1.0 + abs(trueRatings(u, p)) val err = confidence * (truePref - prediction) * (truePref - prediction) sqErr += err denom += confidence diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala index ea4f2865757c..02ea74b87f68 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/IsotonicRegressionSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.regression import org.scalatest.Matchers -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.Utils @@ -163,17 +163,27 @@ class IsotonicRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w } test("weighted isotonic regression with negative weights") { - val model = runIsotonicRegression(Seq(1, 2, 3, 2, 1), Seq(-1, 1, -3, 1, -5), true) - - assert(model.boundaries === Array(0.0, 1.0, 4.0)) - assert(model.predictions === Array(1.0, 10.0/6, 10.0/6)) + val ex = intercept[SparkException] { + runIsotonicRegression(Seq(1, 2, 3, 2, 1), Seq(-1, 1, -3, 1, -5), true) + } + assert(ex.getCause.isInstanceOf[IllegalArgumentException]) } test("weighted isotonic regression with zero weights") { - val model = runIsotonicRegression(Seq[Double](1, 2, 3, 2, 1), Seq[Double](0, 0, 0, 1, 0), true) + val model = runIsotonicRegression(Seq(1, 2, 3, 2, 1, 0), Seq(0, 0, 0, 1, 1, 0), true) + assert(model.boundaries === Array(3, 4)) + assert(model.predictions === Array(1.5, 1.5)) + } + + test("SPARK-16426 isotonic regression with duplicate features that produce NaNs") { + val trainRDD = sc.parallelize(Seq[(Double, Double, Double)]((2, 1, 1), (1, 1, 1), (0, 2, 1), + (1, 2, 1), (0.5, 3, 1), (0, 3, 1)), + 2) + + val model = new IsotonicRegression().run(trainRDD) - assert(model.boundaries === Array(0.0, 1.0, 4.0)) - assert(model.predictions === Array(1, 2, 2)) + assert(model.boundaries === Array(1.0, 3.0)) + assert(model.predictions === Array(0.75, 0.75)) } test("isotonic regression prediction") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala index f8d0af8820e6..252a068dcd72 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} import org.apache.spark.mllib.linalg.Vectors class LabeledPointSuite extends SparkFunSuite { @@ -40,4 +41,16 @@ class LabeledPointSuite extends SparkFunSuite { val point = LabeledPoint.parse("1.0,1.0 0.0 -2.0") assert(point === LabeledPoint(1.0, Vectors.dense(1.0, 0.0, -2.0))) } + + test("conversions between new ml LabeledPoint and mllib LabeledPoint") { + val points: Seq[LabeledPoint] = Seq( + LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), + LabeledPoint(0.0, Vectors.sparse(2, Array(1), Array(-1.0)))) + + val newPoints: Seq[NewLabeledPoint] = points.map(_.asML) + + points.zip(newPoints).foreach { case (p1, p2) => + assert(p1 === LabeledPoint.fromML(p2)) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index 39537e7bb4c7..d96103d01e4a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -21,7 +21,7 @@ import scala.util.Random import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, +import org.apache.spark.mllib.util.{LinearDataGenerator, LocalClusterSparkContext, MLlibTestSparkContext} import org.apache.spark.util.Utils diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index f88a1c33c9f7..0694079b9df9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -21,7 +21,7 @@ import scala.util.Random import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, +import org.apache.spark.mllib.util.{LinearDataGenerator, LocalClusterSparkContext, MLlibTestSparkContext} import org.apache.spark.util.Utils diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index 7a781fee634c..815be32d2e51 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -19,11 +19,9 @@ package org.apache.spark.mllib.regression import scala.util.Random -import org.jblas.DoubleMatrix - import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, +import org.apache.spark.mllib.util.{LinearDataGenerator, LocalClusterSparkContext, MLlibTestSparkContext} import org.apache.spark.util.Utils @@ -38,7 +36,7 @@ class RidgeRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]): Double = { predictions.zip(input).map { case (prediction, expected) => (prediction - expected.label) * (prediction - expected.label) - }.reduceLeft(_ + _) / predictions.size + }.sum / predictions.size } test("ridge regression can help avoid overfitting") { @@ -49,12 +47,12 @@ class RidgeRegressionSuite extends SparkFunSuite with MLlibTestSparkContext { val numExamples = 50 val numFeatures = 20 - org.jblas.util.Random.seed(42) // Pick weights as random values distributed uniformly in [-0.5, 0.5] - val w = DoubleMatrix.rand(numFeatures, 1).subi(0.5) + val random = new Random(42) + val w = Array.fill(numFeatures)(random.nextDouble() - 0.5) // Use half of data for training and other half for validation - val data = LinearDataGenerator.generateLinearInput(3.0, w.toArray, 2 * numExamples, 42, 10.0) + val data = LinearDataGenerator.generateLinearInput(3.0, w, 2 * numExamples, 42, 10.0) val testData = data.take(numExamples) val validationData = data.takeRight(numExamples) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index 34c07ed17081..eaeaa3fc1e68 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -109,7 +109,7 @@ class StreamingLinearRegressionSuite extends SparkFunSuite with TestSuiteBase { // (we add a count to ensure the result is a DStream) ssc = setupStreams(input, (inputDStream: DStream[LabeledPoint]) => { model.trainOn(inputDStream) - inputDStream.foreachRDD(x => history.append(math.abs(model.latestModel().weights(0) - 10.0))) + inputDStream.foreachRDD(x => history += math.abs(model.latestModel().weights(0) - 10.0)) inputDStream.count() }) runStreams(ssc, numBatches, numBatches) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala index c3eeda012571..e32767edb17a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala @@ -19,8 +19,10 @@ package org.apache.spark.mllib.stat import breeze.linalg.{DenseMatrix => BDM, Matrix => BM} -import org.apache.spark.{Logging, SparkFunSuite} +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.random.RandomRDDs import org.apache.spark.mllib.stat.correlation.{Correlations, PearsonCorrelation, SpearmanCorrelation} import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -41,10 +43,10 @@ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Log test("corr(x, y) pearson, 1 value in data") { val x = sc.parallelize(Array(1.0)) val y = sc.parallelize(Array(4.0)) - intercept[RuntimeException] { + intercept[IllegalArgumentException] { Statistics.corr(x, y, "pearson") } - intercept[RuntimeException] { + intercept[IllegalArgumentException] { Statistics.corr(x, y, "spearman") } } @@ -102,8 +104,8 @@ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Log (Double.NaN, Double.NaN, 1.00000000, Double.NaN), (0.40047142, 0.91359586, Double.NaN, 1.0000000)) // scalastyle:on - assert(matrixApproxEqual(defaultMat.toBreeze, expected)) - assert(matrixApproxEqual(pearsonMat.toBreeze, expected)) + assert(matrixApproxEqual(defaultMat.asBreeze, expected)) + assert(matrixApproxEqual(pearsonMat.asBreeze, expected)) } test("corr(X) spearman") { @@ -116,7 +118,7 @@ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Log (Double.NaN, Double.NaN, 1.00000000, Double.NaN), (0.4000000, 0.9486833, Double.NaN, 1.0000000)) // scalastyle:on - assert(matrixApproxEqual(spearmanMat.toBreeze, expected)) + assert(matrixApproxEqual(spearmanMat.asBreeze, expected)) } test("method identification") { @@ -126,15 +128,22 @@ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Log assert(Correlations.getCorrelationFromName("pearson") === pearson) assert(Correlations.getCorrelationFromName("spearman") === spearman) - // Should throw IllegalArgumentException - try { + intercept[IllegalArgumentException] { Correlations.getCorrelationFromName("kendall") - assert(false) - } catch { - case ie: IllegalArgumentException => } } + ignore("Pearson correlation of very large uncorrelated values (SPARK-14533)") { + // The two RDDs should have 0 correlation because they're random; + // this should stay the same after shifting them by any amount + // In practice a large shift produces very large values which can reveal + // round-off problems + val a = RandomRDDs.normalRDD(sc, 100000, 10).map(_ + 1000000000.0) + val b = RandomRDDs.normalRDD(sc, 100000, 10).map(_ + 1000000000.0) + val p = Statistics.corr(a, b, method = "pearson") + assert(approxEqual(p, 0.0, 0.01)) + } + def approxEqual(v1: Double, v2: Double, threshold: Double = 1e-6): Boolean = { if (v1.isNaN) { v2.isNaN diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala index 142b90e764a7..992b87656189 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala @@ -144,15 +144,18 @@ class HypothesisTestSuite extends SparkFunSuite with MLlibTestSparkContext { assert(chi.size === numCols) assert(chi(1000) != null) // SPARK-3087 - // Detect continous features or labels + // Detect continuous features or labels + val tooManyCategories: Int = 100000 + assert(tooManyCategories > ChiSqTest.maxCategories, "This unit test requires that " + + "tooManyCategories be large enough to cause ChiSqTest to throw an exception.") val random = new Random(11L) - val continuousLabel = - Seq.fill(100000)(LabeledPoint(random.nextDouble(), Vectors.dense(random.nextInt(2)))) + val continuousLabel = Seq.fill(tooManyCategories)( + LabeledPoint(random.nextDouble(), Vectors.dense(random.nextInt(2)))) intercept[SparkException] { Statistics.chiSqTest(sc.parallelize(continuousLabel, 2)) } - val continuousFeature = - Seq.fill(100000)(LabeledPoint(random.nextInt(2), Vectors.dense(random.nextDouble()))) + val continuousFeature = Seq.fill(tooManyCategories)( + LabeledPoint(random.nextInt(2), Vectors.dense(random.nextDouble()))) intercept[SparkException] { Statistics.chiSqTest(sc.parallelize(continuousFeature, 2)) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala index b6d41db69be0..c6466bc918dd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala @@ -237,7 +237,7 @@ class MultivariateOnlineSummarizerSuite extends SparkFunSuite { absTol 1E-10, "mean mismatch") assert(summarizer.variance ~== Vectors.dense(Array(0.17657142857, 1.645115714, 2.42057142857)) absTol 1E-8, "variance mismatch") - assert(summarizer.numNonzeros ~== Vectors.dense(Array(0.3, 0.5, 0.4)) + assert(summarizer.numNonzeros ~== Vectors.dense(Array(3.0, 4.0, 3.0)) absTol 1E-10, "numNonzeros mismatch") assert(summarizer.max ~== Vectors.dense(Array(0.0, 1.7, 1.3)) absTol 1E-10, "max mismatch") assert(summarizer.min ~== Vectors.dense(Array(-0.8, -1.2, -1.7)) absTol 1E-10, "min mismatch") @@ -245,4 +245,47 @@ class MultivariateOnlineSummarizerSuite extends SparkFunSuite { absTol 1E-8, "normL2 mismatch") assert(summarizer.normL1 ~== Vectors.dense(0.21, 0.4265, 0.61) absTol 1E-10, "normL1 mismatch") } + + test("test min/max with weighted samples (SPARK-16561)") { + val summarizer1 = new MultivariateOnlineSummarizer() + .add(Vectors.dense(10.0, -10.0), 1e10) + .add(Vectors.dense(0.0, 0.0), 1e-7) + + val summarizer2 = new MultivariateOnlineSummarizer() + summarizer2.add(Vectors.dense(10.0, -10.0), 1e10) + for (i <- 1 to 100) { + summarizer2.add(Vectors.dense(0.0, 0.0), 1e-7) + } + + val summarizer3 = new MultivariateOnlineSummarizer() + for (i <- 1 to 100) { + summarizer3.add(Vectors.dense(0.0, 0.0), 1e-7) + } + summarizer3.add(Vectors.dense(10.0, -10.0), 1e10) + + assert(summarizer1.max ~== Vectors.dense(10.0, 0.0) absTol 1e-14) + assert(summarizer1.min ~== Vectors.dense(0.0, -10.0) absTol 1e-14) + assert(summarizer2.max ~== Vectors.dense(10.0, 0.0) absTol 1e-14) + assert(summarizer2.min ~== Vectors.dense(0.0, -10.0) absTol 1e-14) + assert(summarizer3.max ~== Vectors.dense(10.0, 0.0) absTol 1e-14) + assert(summarizer3.min ~== Vectors.dense(0.0, -10.0) absTol 1e-14) + } + + test ("test zero variance (SPARK-21818)") { + val summarizer1 = (new MultivariateOnlineSummarizer) + .add(Vectors.dense(3.0), 0.7) + val summarizer2 = (new MultivariateOnlineSummarizer) + .add(Vectors.dense(3.0), 0.4) + val summarizer3 = (new MultivariateOnlineSummarizer) + .add(Vectors.dense(3.0), 0.5) + val summarizer4 = (new MultivariateOnlineSummarizer) + .add(Vectors.dense(3.0), 0.4) + + val summarizer = summarizer1 + .merge(summarizer2) + .merge(summarizer3) + .merge(summarizer4) + + assert(summarizer.variance(0) >= 0.0) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala index d3e9ef4ff079..0921fdba339c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala @@ -18,7 +18,8 @@ package org.apache.spark.mllib.stat import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.stat.test.{StreamingTest, StreamingTestResult, StudentTTest, WelchTTest} +import org.apache.spark.mllib.stat.test.{BinarySample, StreamingTest, StreamingTestResult, + StudentTTest, WelchTTest} import org.apache.spark.streaming.TestSuiteBase import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.StatCounter @@ -26,7 +27,7 @@ import org.apache.spark.util.random.XORShiftRandom class StreamingTestSuite extends SparkFunSuite with TestSuiteBase { - override def maxWaitTimeMillis : Int = 30000 + override def maxWaitTimeMillis: Int = 30000 test("accuracy for null hypothesis using welch t-test") { // set parameters @@ -48,7 +49,7 @@ class StreamingTestSuite extends SparkFunSuite with TestSuiteBase { // setup and run the model val ssc = setupStreams( - input, (inputDStream: DStream[(Boolean, Double)]) => model.registerStream(inputDStream)) + input, (inputDStream: DStream[BinarySample]) => model.registerStream(inputDStream)) val outputBatches = runStreams[StreamingTestResult](ssc, numBatches, numBatches) assert(outputBatches.flatten.forall(res => @@ -75,7 +76,7 @@ class StreamingTestSuite extends SparkFunSuite with TestSuiteBase { // setup and run the model val ssc = setupStreams( - input, (inputDStream: DStream[(Boolean, Double)]) => model.registerStream(inputDStream)) + input, (inputDStream: DStream[BinarySample]) => model.registerStream(inputDStream)) val outputBatches = runStreams[StreamingTestResult](ssc, numBatches, numBatches) assert(outputBatches.flatten.forall(res => @@ -102,7 +103,7 @@ class StreamingTestSuite extends SparkFunSuite with TestSuiteBase { // setup and run the model val ssc = setupStreams( - input, (inputDStream: DStream[(Boolean, Double)]) => model.registerStream(inputDStream)) + input, (inputDStream: DStream[BinarySample]) => model.registerStream(inputDStream)) val outputBatches = runStreams[StreamingTestResult](ssc, numBatches, numBatches) @@ -130,7 +131,7 @@ class StreamingTestSuite extends SparkFunSuite with TestSuiteBase { // setup and run the model val ssc = setupStreams( - input, (inputDStream: DStream[(Boolean, Double)]) => model.registerStream(inputDStream)) + input, (inputDStream: DStream[BinarySample]) => model.registerStream(inputDStream)) val outputBatches = runStreams[StreamingTestResult](ssc, numBatches, numBatches) assert(outputBatches.flatten.forall(res => @@ -157,13 +158,13 @@ class StreamingTestSuite extends SparkFunSuite with TestSuiteBase { // setup and run the model val ssc = setupStreams( input, - (inputDStream: DStream[(Boolean, Double)]) => model.summarizeByKeyAndWindow(inputDStream)) + (inputDStream: DStream[BinarySample]) => model.summarizeByKeyAndWindow(inputDStream)) val outputBatches = runStreams[(Boolean, StatCounter)](ssc, numBatches, numBatches) val outputCounts = outputBatches.flatten.map(_._2.count) // number of batches seen so far does not exceed testWindow, expect counts to continue growing for (i <- 0 until testWindow) { - assert(outputCounts.drop(2 * i).take(2).forall(_ == (i + 1) * pointsPerBatch / 2)) + assert(outputCounts.slice(2 * i, 2 * i + 2).forall(_ == (i + 1) * pointsPerBatch / 2)) } // number of batches seen exceeds testWindow, expect counts to be constant @@ -190,7 +191,7 @@ class StreamingTestSuite extends SparkFunSuite with TestSuiteBase { // setup and run the model val ssc = setupStreams( - input, (inputDStream: DStream[(Boolean, Double)]) => model.dropPeacePeriod(inputDStream)) + input, (inputDStream: DStream[BinarySample]) => model.dropPeacePeriod(inputDStream)) val outputBatches = runStreams[(Boolean, Double)](ssc, numBatches, numBatches) assert(outputBatches.flatten.length == (numBatches - peacePeriod) * pointsPerBatch) @@ -210,11 +211,11 @@ class StreamingTestSuite extends SparkFunSuite with TestSuiteBase { .setPeacePeriod(0) val input = generateTestData(numBatches, pointsPerBatch, meanA, stdevA, meanB, stdevB, 42) - .map(batch => batch.filter(_._1)) // only keep one test group + .map(batch => batch.filter(_.isExperiment)) // only keep one test group // setup and run the model val ssc = setupStreams( - input, (inputDStream: DStream[(Boolean, Double)]) => model.registerStream(inputDStream)) + input, (inputDStream: DStream[BinarySample]) => model.registerStream(inputDStream)) val outputBatches = runStreams[StreamingTestResult](ssc, numBatches, numBatches) assert(outputBatches.flatten.forall(result => (result.pValue - 1.0).abs < 0.001)) @@ -228,13 +229,13 @@ class StreamingTestSuite extends SparkFunSuite with TestSuiteBase { stdevA: Double, meanB: Double, stdevB: Double, - seed: Int): (IndexedSeq[IndexedSeq[(Boolean, Double)]]) = { + seed: Int): (IndexedSeq[IndexedSeq[BinarySample]]) = { val rand = new XORShiftRandom(seed) val numTrues = pointsPerBatch / 2 val data = (0 until numBatches).map { i => - (0 until numTrues).map { idx => (true, meanA + stdevA * rand.nextGaussian())} ++ + (0 until numTrues).map { idx => BinarySample(true, meanA + stdevA * rand.nextGaussian())} ++ (pointsPerBatch / 2 until pointsPerBatch).map { idx => - (false, meanB + stdevB * rand.nextGaussian()) + BinarySample(false, meanB + stdevB * rand.nextGaussian()) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala index 6e7a00347545..669d44223d71 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.stat.distribution import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{ Vectors, Matrices } +import org.apache.spark.mllib.linalg.{Matrices, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ 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 1a4299db4eab..441d0f7614bf 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 @@ -18,15 +18,14 @@ package org.apache.spark.mllib.tree import scala.collection.JavaConverters._ -import scala.collection.mutable import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.tree.impl.DecisionTreeMetadata import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ -import org.apache.spark.mllib.tree.configuration.{QuantileStrategy, Strategy} -import org.apache.spark.mllib.tree.impl.{BaggedPoint, DecisionTreeMetadata, TreePoint} +import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model._ import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -35,378 +34,6 @@ import org.apache.spark.util.Utils class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { - ///////////////////////////////////////////////////////////////////////////// - // Tests examining individual elements of training - ///////////////////////////////////////////////////////////////////////////// - - test("Binary classification with continuous features: split and bin calculation") { - val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() - assert(arr.length === 1000) - val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Gini, 3, 2, 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - assert(!metadata.isUnordered(featureIndex = 0)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - assert(splits.length === 2) - assert(bins.length === 2) - assert(splits(0).length === 99) - assert(bins(0).length === 100) - } - - test("Binary classification with binary (ordered) categorical features:" + - " split and bin calculation") { - val arr = DecisionTreeSuite.generateCategoricalDataPoints() - assert(arr.length === 1000) - val rdd = sc.parallelize(arr) - val strategy = new Strategy( - Classification, - Gini, - maxDepth = 2, - numClasses = 2, - maxBins = 100, - categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) - - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - assert(!metadata.isUnordered(featureIndex = 0)) - assert(!metadata.isUnordered(featureIndex = 1)) - assert(splits.length === 2) - assert(bins.length === 2) - // no bins or splits pre-computed for ordered categorical features - assert(splits(0).length === 0) - assert(bins(0).length === 0) - } - - test("Binary classification with 3-ary (ordered) categorical features," + - " with no samples for one category") { - val arr = DecisionTreeSuite.generateCategoricalDataPoints() - assert(arr.length === 1000) - val rdd = sc.parallelize(arr) - val strategy = new Strategy( - Classification, - Gini, - maxDepth = 2, - numClasses = 2, - maxBins = 100, - categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) - - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - assert(!metadata.isUnordered(featureIndex = 0)) - assert(!metadata.isUnordered(featureIndex = 1)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - assert(splits.length === 2) - assert(bins.length === 2) - // no bins or splits pre-computed for ordered categorical features - assert(splits(0).length === 0) - assert(bins(0).length === 0) - } - - test("extract categories from a number for multiclass classification") { - val l = DecisionTree.extractMultiClassCategories(13, 10) - assert(l.length === 3) - assert(List(3.0, 2.0, 0.0).toSeq === l.toSeq) - } - - test("find splits for a continuous feature") { - // find splits for normal case - { - val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, - Map(), Set(), - Array(6), Gini, QuantileStrategy.Sort, - 0, 0, 0.0, 0, 0 - ) - val featureSamples = Array.fill(200000)(math.random) - val splits = DecisionTree.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) - assert(splits.length === 5) - assert(fakeMetadata.numSplits(0) === 5) - assert(fakeMetadata.numBins(0) === 6) - // check returned splits are distinct - assert(splits.distinct.length === splits.length) - } - - // find splits should not return identical splits - // when there are not enough split candidates, reduce the number of splits in metadata - { - val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, - Map(), Set(), - Array(5), Gini, QuantileStrategy.Sort, - 0, 0, 0.0, 0, 0 - ) - val featureSamples = Array(1, 1, 2, 2, 2, 2, 2, 2, 2, 2, 3).map(_.toDouble) - val splits = DecisionTree.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) - assert(splits.length === 3) - // check returned splits are distinct - assert(splits.distinct.length === splits.length) - } - - // find splits when most samples close to the minimum - { - val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, - Map(), Set(), - Array(3), Gini, QuantileStrategy.Sort, - 0, 0, 0.0, 0, 0 - ) - val featureSamples = Array(2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 3, 4, 5).map(_.toDouble) - val splits = DecisionTree.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) - assert(splits.length === 2) - assert(splits(0) === 2.0) - assert(splits(1) === 3.0) - } - - // find splits when most samples close to the maximum - { - val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, - Map(), Set(), - Array(3), Gini, QuantileStrategy.Sort, - 0, 0, 0.0, 0, 0 - ) - val featureSamples = Array(0, 1, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2).map(_.toDouble) - val splits = DecisionTree.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) - assert(splits.length === 1) - assert(splits(0) === 1.0) - } - } - - test("Multiclass classification with unordered categorical features:" + - " split and bin calculations") { - val arr = DecisionTreeSuite.generateCategoricalDataPoints() - assert(arr.length === 1000) - val rdd = sc.parallelize(arr) - val strategy = new Strategy( - Classification, - Gini, - maxDepth = 2, - numClasses = 100, - maxBins = 100, - categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) - - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - assert(metadata.isUnordered(featureIndex = 0)) - assert(metadata.isUnordered(featureIndex = 1)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - assert(splits.length === 2) - assert(bins.length === 2) - assert(splits(0).length === 3) - assert(bins(0).length === 0) - - // Expecting 2^2 - 1 = 3 bins/splits - assert(splits(0)(0).feature === 0) - assert(splits(0)(0).threshold === Double.MinValue) - assert(splits(0)(0).featureType === Categorical) - assert(splits(0)(0).categories.length === 1) - assert(splits(0)(0).categories.contains(0.0)) - assert(splits(1)(0).feature === 1) - assert(splits(1)(0).threshold === Double.MinValue) - assert(splits(1)(0).featureType === Categorical) - assert(splits(1)(0).categories.length === 1) - assert(splits(1)(0).categories.contains(0.0)) - - assert(splits(0)(1).feature === 0) - assert(splits(0)(1).threshold === Double.MinValue) - assert(splits(0)(1).featureType === Categorical) - assert(splits(0)(1).categories.length === 1) - assert(splits(0)(1).categories.contains(1.0)) - assert(splits(1)(1).feature === 1) - assert(splits(1)(1).threshold === Double.MinValue) - assert(splits(1)(1).featureType === Categorical) - assert(splits(1)(1).categories.length === 1) - assert(splits(1)(1).categories.contains(1.0)) - - assert(splits(0)(2).feature === 0) - assert(splits(0)(2).threshold === Double.MinValue) - assert(splits(0)(2).featureType === Categorical) - assert(splits(0)(2).categories.length === 2) - assert(splits(0)(2).categories.contains(0.0)) - assert(splits(0)(2).categories.contains(1.0)) - assert(splits(1)(2).feature === 1) - assert(splits(1)(2).threshold === Double.MinValue) - assert(splits(1)(2).featureType === Categorical) - assert(splits(1)(2).categories.length === 2) - assert(splits(1)(2).categories.contains(0.0)) - assert(splits(1)(2).categories.contains(1.0)) - - } - - test("Multiclass classification with ordered categorical features: split and bin calculations") { - val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() - assert(arr.length === 3000) - val rdd = sc.parallelize(arr) - val strategy = new Strategy( - Classification, - Gini, - maxDepth = 2, - numClasses = 100, - maxBins = 100, - categoricalFeaturesInfo = Map(0 -> 10, 1-> 10)) - // 2^(10-1) - 1 > 100, so categorical features will be ordered - - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - assert(!metadata.isUnordered(featureIndex = 0)) - assert(!metadata.isUnordered(featureIndex = 1)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - assert(splits.length === 2) - assert(bins.length === 2) - // no bins or splits pre-computed for ordered categorical features - assert(splits(0).length === 0) - assert(bins(0).length === 0) - } - - test("Avoid aggregation on the last level") { - val arr = Array( - LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)), - LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)), - LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)), - LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0))) - val input = sc.parallelize(arr) - - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 1, - numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) - - val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) - - val topNode = Node.emptyNode(nodeIndex = 1) - assert(topNode.predict.predict === Double.MinValue) - assert(topNode.impurity === -1.0) - assert(topNode.isLeaf === false) - - val nodesForGroup = Map((0, Array(topNode))) - val treeToNodeToIndexInfo = Map((0, Map( - (topNode.id, new RandomForest.NodeIndexInfo(0, None)) - ))) - val nodeQueue = new mutable.Queue[(Int, Node)]() - DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), - nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) - - // don't enqueue leaf nodes into node queue - assert(nodeQueue.isEmpty) - - // set impurity and predict for topNode - assert(topNode.predict.predict !== Double.MinValue) - assert(topNode.impurity !== -1.0) - - // set impurity and predict for child nodes - assert(topNode.leftNode.get.predict.predict === 0.0) - assert(topNode.rightNode.get.predict.predict === 1.0) - assert(topNode.leftNode.get.impurity === 0.0) - assert(topNode.rightNode.get.impurity === 0.0) - } - - test("Avoid aggregation if impurity is 0.0") { - val arr = Array( - LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)), - LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)), - LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)), - LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0))) - val input = sc.parallelize(arr) - - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, - numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) - - val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) - - val topNode = Node.emptyNode(nodeIndex = 1) - assert(topNode.predict.predict === Double.MinValue) - assert(topNode.impurity === -1.0) - assert(topNode.isLeaf === false) - - val nodesForGroup = Map((0, Array(topNode))) - val treeToNodeToIndexInfo = Map((0, Map( - (topNode.id, new RandomForest.NodeIndexInfo(0, None)) - ))) - val nodeQueue = new mutable.Queue[(Int, Node)]() - DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), - nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) - - // don't enqueue a node into node queue if its impurity is 0.0 - assert(nodeQueue.isEmpty) - - // set impurity and predict for topNode - assert(topNode.predict.predict !== Double.MinValue) - assert(topNode.impurity !== -1.0) - - // set impurity and predict for child nodes - assert(topNode.leftNode.get.predict.predict === 0.0) - assert(topNode.rightNode.get.predict.predict === 1.0) - assert(topNode.leftNode.get.impurity === 0.0) - assert(topNode.rightNode.get.impurity === 0.0) - } - - test("Second level node building with vs. without groups") { - val arr = DecisionTreeSuite.generateOrderedLabeledPoints() - assert(arr.length === 1000) - val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 2, 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - assert(splits.length === 2) - assert(splits(0).length === 99) - assert(bins.length === 2) - assert(bins(0).length === 100) - - // Train a 1-node model - val strategyOneNode = new Strategy(Classification, Entropy, maxDepth = 1, - numClasses = 2, maxBins = 100) - val modelOneNode = DecisionTree.train(rdd, strategyOneNode) - val rootNode1 = modelOneNode.topNode.deepCopy() - val rootNode2 = modelOneNode.topNode.deepCopy() - assert(rootNode1.leftNode.nonEmpty) - assert(rootNode1.rightNode.nonEmpty) - - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) - - // Single group second level tree construction. - val nodesForGroup = Map((0, Array(rootNode1.leftNode.get, rootNode1.rightNode.get))) - val treeToNodeToIndexInfo = Map((0, Map( - (rootNode1.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None)), - (rootNode1.rightNode.get.id, new RandomForest.NodeIndexInfo(1, None))))) - val nodeQueue = new mutable.Queue[(Int, Node)]() - DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode1), - nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) - val children1 = new Array[Node](2) - children1(0) = rootNode1.leftNode.get - children1(1) = rootNode1.rightNode.get - - // Train one second-level node at a time. - val nodesForGroupA = Map((0, Array(rootNode2.leftNode.get))) - val treeToNodeToIndexInfoA = Map((0, Map( - (rootNode2.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) - nodeQueue.clear() - DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), - nodesForGroupA, treeToNodeToIndexInfoA, splits, bins, nodeQueue) - val nodesForGroupB = Map((0, Array(rootNode2.rightNode.get))) - val treeToNodeToIndexInfoB = Map((0, Map( - (rootNode2.rightNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) - nodeQueue.clear() - DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), - nodesForGroupB, treeToNodeToIndexInfoB, splits, bins, nodeQueue) - val children2 = new Array[Node](2) - children2(0) = rootNode2.leftNode.get - children2(1) = rootNode2.rightNode.get - - // Verify whether the splits obtained using single group and multiple group level - // construction strategies are the same. - for (i <- 0 until 2) { - assert(children1(i).stats.nonEmpty && children1(i).stats.get.gain > 0) - assert(children2(i).stats.nonEmpty && children2(i).stats.get.gain > 0) - assert(children1(i).split === children2(i).split) - assert(children1(i).stats.nonEmpty && children2(i).stats.nonEmpty) - val stats1 = children1(i).stats.get - val stats2 = children2(i).stats.get - assert(stats1.gain === stats2.gain) - assert(stats1.impurity === stats2.impurity) - assert(stats1.leftImpurity === stats2.leftImpurity) - assert(stats1.rightImpurity === stats2.rightImpurity) - assert(children1(i).predict.predict === children2(i).predict.predict) - } - } - ///////////////////////////////////////////////////////////////////////////// // Tests calling train() ///////////////////////////////////////////////////////////////////////////// @@ -421,24 +48,13 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { numClasses = 2, maxDepth = 2, maxBins = 100, - categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) - - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - assert(!metadata.isUnordered(featureIndex = 0)) - assert(!metadata.isUnordered(featureIndex = 1)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - assert(splits.length === 2) - assert(bins.length === 2) - // no bins or splits pre-computed for ordered categorical features - assert(splits(0).length === 0) - assert(bins(0).length === 0) + categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) val rootNode = DecisionTree.train(rdd, strategy).topNode val split = rootNode.split.get assert(split.categories === List(1.0)) assert(split.featureType === Categorical) - assert(split.threshold === Double.MinValue) val stats = rootNode.stats.get assert(stats.gain > 0) @@ -455,9 +71,9 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { Variance, maxDepth = 2, maxBins = 100, - categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -467,7 +83,6 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { assert(split.categories.length === 1) assert(split.categories.contains(1.0)) assert(split.featureType === Categorical) - assert(split.threshold === Double.MinValue) val stats = rootNode.stats.get assert(stats.gain > 0) @@ -484,8 +99,8 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { Variance, maxDepth = 2, maxBins = 100, - categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + categoricalFeaturesInfo = Map(0 -> 2, 1 -> 2)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -501,22 +116,15 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - assert(splits.length === 2) - assert(splits(0).length === 99) - assert(bins.length === 2) - assert(bins(0).length === 100) - val rootNode = DecisionTree.train(rdd, strategy).topNode - val stats = rootNode.stats.get - assert(stats.gain === 0) - assert(stats.leftImpurity === 0) - assert(stats.rightImpurity === 0) + assert(rootNode.impurity === 0) + assert(rootNode.stats.isEmpty) + assert(rootNode.predict.predict === 0) } test("Binary classification stump with fixed label 1 for Gini") { @@ -525,22 +133,14 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - assert(splits.length === 2) - assert(splits(0).length === 99) - assert(bins.length === 2) - assert(bins(0).length === 100) - val rootNode = DecisionTree.train(rdd, strategy).topNode - val stats = rootNode.stats.get - assert(stats.gain === 0) - assert(stats.leftImpurity === 0) - assert(stats.rightImpurity === 0) + assert(rootNode.impurity === 0) + assert(rootNode.stats.isEmpty) assert(rootNode.predict.predict === 1) } @@ -550,22 +150,14 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - assert(splits.length === 2) - assert(splits(0).length === 99) - assert(bins.length === 2) - assert(bins(0).length === 100) - val rootNode = DecisionTree.train(rdd, strategy).topNode - val stats = rootNode.stats.get - assert(stats.gain === 0) - assert(stats.leftImpurity === 0) - assert(stats.rightImpurity === 0) + assert(rootNode.impurity === 0) + assert(rootNode.stats.isEmpty) assert(rootNode.predict.predict === 0) } @@ -575,22 +167,14 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - assert(splits.length === 2) - assert(splits(0).length === 99) - assert(bins.length === 2) - assert(bins(0).length === 100) - val rootNode = DecisionTree.train(rdd, strategy).topNode - val stats = rootNode.stats.get - assert(stats.gain === 0) - assert(stats.leftImpurity === 0) - assert(stats.rightImpurity === 0) + assert(rootNode.impurity === 0) + assert(rootNode.stats.isEmpty) assert(rootNode.predict.predict === 1) } @@ -599,7 +183,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClasses = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(strategy.isMulticlassClassification) assert(metadata.isUnordered(featureIndex = 0)) assert(metadata.isUnordered(featureIndex = 1)) @@ -656,7 +240,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { numClasses = 3, maxBins = maxBins, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(metadata.isUnordered(featureIndex = 0)) assert(metadata.isUnordered(featureIndex = 1)) @@ -684,7 +268,6 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClasses = 3, maxBins = 100) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) val model = DecisionTree.train(rdd, strategy) DecisionTreeSuite.validateClassifier(model, arr, 0.9) @@ -705,7 +288,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClasses = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(metadata.isUnordered(featureIndex = 0)) val model = DecisionTree.train(rdd, strategy) @@ -727,7 +310,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { numClasses = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -773,8 +356,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { // test when no valid split can be found val rootNode = model.topNode - val gain = rootNode.stats.get - assert(gain == InformationGainStats.invalidInformationGainStats) + assert(rootNode.stats.isEmpty) } test("do not choose split that does not satisfy min instance per node requirements") { @@ -788,15 +370,16 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, - maxBins = 2, maxDepth = 2, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2), + maxBins = 2, maxDepth = 2, categoricalFeaturesInfo = Map(0 -> 2, 1 -> 2), numClasses = 2, minInstancesPerNode = 2) val rootNode = DecisionTree.train(rdd, strategy).topNode val split = rootNode.split.get - val gain = rootNode.stats.get + val gainStats = rootNode.stats.get assert(split.feature == 1) - assert(gain != InformationGainStats.invalidInformationGainStats) + assert(gainStats.gain >= 0) + assert(gainStats.impurity >= 0) } test("split must satisfy min info gain requirements") { @@ -818,10 +401,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { } // test when no valid split can be found - val rootNode = model.topNode - - val gain = rootNode.stats.get - assert(gain == InformationGainStats.invalidInformationGainStats) + assert(model.topNode.stats.isEmpty) } ///////////////////////////////////////////////////////////////////////////// @@ -1045,7 +625,7 @@ object DecisionTreeSuite extends SparkFunSuite { assert(a.isLeaf === b.isLeaf) assert(a.split === b.split) (a.stats, b.stats) match { - // TODO: Check other fields besides the infomation gain. + // TODO: Check other fields besides the information gain. case (Some(aStats), Some(bStats)) => assert(aStats.gain === bStats.gain) case (None, None) => case _ => throw new AssertionError( diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala index 3d3f80063f90..1cc8f342021a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala @@ -17,13 +17,13 @@ package org.apache.spark.mllib.tree +import scala.collection.mutable + import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel import org.apache.spark.util.StatCounter -import scala.collection.mutable - object EnsembleTestHelper { /** diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala index 6fc9e8df621d..c61f89322d35 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala @@ -17,17 +17,17 @@ package org.apache.spark.mllib.tree -import org.apache.spark.{Logging, SparkFunSuite} +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy} +import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.impurity.Variance -import org.apache.spark.mllib.tree.loss.{AbsoluteError, SquaredError, LogLoss} +import org.apache.spark.mllib.tree.loss.{AbsoluteError, LogLoss, SquaredError} import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.util.Utils - /** * Test suite for [[GradientBoostedTrees]]. */ @@ -158,49 +158,6 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext } } - test("runWithValidation stops early and performs better on a validation dataset") { - // Set numIterations large enough so that it stops early. - val numIterations = 20 - val trainRdd = sc.parallelize(GradientBoostedTreesSuite.trainData, 2) - val validateRdd = sc.parallelize(GradientBoostedTreesSuite.validateData, 2) - - val algos = Array(Regression, Regression, Classification) - val losses = Array(SquaredError, AbsoluteError, LogLoss) - algos.zip(losses).foreach { case (algo, loss) => - val treeStrategy = new Strategy(algo = algo, impurity = Variance, maxDepth = 2, - categoricalFeaturesInfo = Map.empty) - val boostingStrategy = - new BoostingStrategy(treeStrategy, loss, numIterations, validationTol = 0.0) - val gbtValidate = new GradientBoostedTrees(boostingStrategy) - .runWithValidation(trainRdd, validateRdd) - val numTrees = gbtValidate.numTrees - assert(numTrees !== numIterations) - - // Test that it performs better on the validation dataset. - val gbt = new GradientBoostedTrees(boostingStrategy).run(trainRdd) - val (errorWithoutValidation, errorWithValidation) = { - if (algo == Classification) { - val remappedRdd = validateRdd.map(x => new LabeledPoint(2 * x.label - 1, x.features)) - (loss.computeError(gbt, remappedRdd), loss.computeError(gbtValidate, remappedRdd)) - } else { - (loss.computeError(gbt, validateRdd), loss.computeError(gbtValidate, validateRdd)) - } - } - assert(errorWithValidation <= errorWithoutValidation) - - // Test that results from evaluateEachIteration comply with runWithValidation. - // Note that convergenceTol is set to 0.0 - val evaluationArray = gbt.evaluateEachIteration(validateRdd, loss) - assert(evaluationArray.length === numIterations) - assert(evaluationArray(numTrees) > evaluationArray(numTrees - 1)) - var i = 1 - while (i < numTrees) { - assert(evaluationArray(i) <= evaluationArray(i - 1)) - i += 1 - } - } - } - test("Checkpointing") { val tempDir = Utils.createTempDir() val path = tempDir.toURI.toString @@ -220,7 +177,7 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext } -private object GradientBoostedTreesSuite { +private[spark] object GradientBoostedTreesSuite { // Combinations for estimators, learning rates and subsamplingRate val testCombinations = Array((10, 1.0, 1.0), (10, 0.1, 1.0), (10, 0.5, 0.75), (10, 0.1, 0.75)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/ImpuritySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/ImpuritySuite.scala index 49aff21fe791..d0f02dd966bd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/ImpuritySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/ImpuritySuite.scala @@ -19,12 +19,11 @@ package org.apache.spark.mllib.tree import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.tree.impurity.{EntropyAggregator, GiniAggregator} -import org.apache.spark.mllib.util.MLlibTestSparkContext /** - * Test suites for [[GiniAggregator]] and [[EntropyAggregator]]. + * Test suites for `GiniAggregator` and `EntropyAggregator`. */ -class ImpuritySuite extends SparkFunSuite with MLlibTestSparkContext { +class ImpuritySuite extends SparkFunSuite { test("Gini impurity does not support negative labels") { val gini = new GiniAggregator(2) intercept[IllegalArgumentException] { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index e6df5d974bf3..bec61ba6a003 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -17,16 +17,13 @@ package org.apache.spark.mllib.tree -import scala.collection.mutable - import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.Strategy -import org.apache.spark.mllib.tree.impl.DecisionTreeMetadata import org.apache.spark.mllib.tree.impurity.{Gini, Variance} -import org.apache.spark.mllib.tree.model.{Node, RandomForestModel} +import org.apache.spark.mllib.tree.model.RandomForestModel import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.util.Utils @@ -42,7 +39,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { val rf = RandomForest.trainClassifier(rdd, strategy, numTrees = numTrees, featureSubsetStrategy = "auto", seed = 123) - assert(rf.trees.size === 1) + assert(rf.trees.length === 1) val rfTree = rf.trees(0) val dt = DecisionTree.train(rdd, strategy) @@ -78,7 +75,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { val rf = RandomForest.trainRegressor(rdd, strategy, numTrees = numTrees, featureSubsetStrategy = "auto", seed = 123) - assert(rf.trees.size === 1) + assert(rf.trees.length === 1) val rfTree = rf.trees(0) val dt = DecisionTree.train(rdd, strategy) @@ -108,80 +105,6 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { regressionTestWithContinuousFeatures(strategy) } - def binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy: Strategy) { - val numFeatures = 50 - val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures, 1000) - val rdd = sc.parallelize(arr) - - // Select feature subset for top nodes. Return true if OK. - def checkFeatureSubsetStrategy( - numTrees: Int, - featureSubsetStrategy: String, - numFeaturesPerNode: Int): Unit = { - val seeds = Array(123, 5354, 230, 349867, 23987) - val maxMemoryUsage: Long = 128 * 1024L * 1024L - val metadata = - DecisionTreeMetadata.buildMetadata(rdd, strategy, numTrees, featureSubsetStrategy) - seeds.foreach { seed => - val failString = s"Failed on test with:" + - s"numTrees=$numTrees, featureSubsetStrategy=$featureSubsetStrategy," + - s" numFeaturesPerNode=$numFeaturesPerNode, seed=$seed" - val nodeQueue = new mutable.Queue[(Int, Node)]() - val topNodes: Array[Node] = new Array[Node](numTrees) - Range(0, numTrees).foreach { treeIndex => - topNodes(treeIndex) = Node.emptyNode(nodeIndex = 1) - nodeQueue.enqueue((treeIndex, topNodes(treeIndex))) - } - val rng = new scala.util.Random(seed = seed) - val (nodesForGroup: Map[Int, Array[Node]], - treeToNodeToIndexInfo: Map[Int, Map[Int, RandomForest.NodeIndexInfo]]) = - RandomForest.selectNodesToSplit(nodeQueue, maxMemoryUsage, metadata, rng) - - assert(nodesForGroup.size === numTrees, failString) - assert(nodesForGroup.values.forall(_.size == 1), failString) // 1 node per tree - - if (numFeaturesPerNode == numFeatures) { - // featureSubset values should all be None - assert(treeToNodeToIndexInfo.values.forall(_.values.forall(_.featureSubset.isEmpty)), - failString) - } else { - // Check number of features. - assert(treeToNodeToIndexInfo.values.forall(_.values.forall( - _.featureSubset.get.size === numFeaturesPerNode)), failString) - } - } - } - - checkFeatureSubsetStrategy(numTrees = 1, "auto", numFeatures) - checkFeatureSubsetStrategy(numTrees = 1, "all", numFeatures) - checkFeatureSubsetStrategy(numTrees = 1, "sqrt", math.sqrt(numFeatures).ceil.toInt) - checkFeatureSubsetStrategy(numTrees = 1, "log2", - (math.log(numFeatures) / math.log(2)).ceil.toInt) - checkFeatureSubsetStrategy(numTrees = 1, "onethird", (numFeatures / 3.0).ceil.toInt) - - checkFeatureSubsetStrategy(numTrees = 2, "all", numFeatures) - checkFeatureSubsetStrategy(numTrees = 2, "auto", math.sqrt(numFeatures).ceil.toInt) - checkFeatureSubsetStrategy(numTrees = 2, "sqrt", math.sqrt(numFeatures).ceil.toInt) - checkFeatureSubsetStrategy(numTrees = 2, "log2", - (math.log(numFeatures) / math.log(2)).ceil.toInt) - checkFeatureSubsetStrategy(numTrees = 2, "onethird", (numFeatures / 3.0).ceil.toInt) - } - - test("Binary classification with continuous features: subsampling features") { - val categoricalFeaturesInfo = Map.empty[Int, Int] - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, - numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) - binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy) - } - - test("Binary classification with continuous features and node Id cache: subsampling features") { - val categoricalFeaturesInfo = Map.empty[Int, Int] - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, - numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, - useNodeIdCache = true) - binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy) - } - test("alternating categorical and continuous features with multiclass labels to test indexing") { val arr = new Array[LabeledPoint](4) arr(0) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0, 3.0, 1.0)) @@ -197,7 +120,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { featureSubsetStrategy = "sqrt", seed = 12345) } - test("subsampling rate in RandomForest"){ + test("subsampling rate in RandomForest") { val arr = EnsembleTestHelper.generateOrderedLabeledPoints(5, 20) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala index 525ab68c7921..95d874b8432e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.util -import org.scalatest.{Suite, BeforeAndAfterAll} +import org.scalatest.{BeforeAndAfterAll, Suite} import org.apache.spark.{SparkConf, SparkContext} @@ -25,18 +25,21 @@ trait LocalClusterSparkContext extends BeforeAndAfterAll { self: Suite => @transient var sc: SparkContext = _ override def beforeAll() { + super.beforeAll() val conf = new SparkConf() .setMaster("local-cluster[2, 1, 1024]") .setAppName("test-cluster") - .set("spark.akka.frameSize", "1") // set to 1MB to detect direct serialization of data + .set("spark.rpc.message.maxSize", "1") // set to 1MB to detect direct serialization of data sc = new SparkContext(conf) - super.beforeAll() } override def afterAll() { - if (sc != null) { - sc.stop() + try { + if (sc != null) { + sc.stop() + } + } finally { + super.afterAll() } - super.afterAll() } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 70219e9ad9d3..665708a780c4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -18,23 +18,27 @@ package org.apache.spark.mllib.util import java.io.File +import java.nio.charset.StandardCharsets import scala.io.Source import breeze.linalg.{squaredDistance => breezeSquaredDistance} -import com.google.common.base.Charsets import com.google.common.io.Files -import org.apache.spark.SparkException -import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.mllib.linalg.{DenseVector, Matrices, SparseVector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils._ import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.sql.Row +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types.MetadataBuilder import org.apache.spark.util.Utils class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { + import testImplicits._ + test("epsilon computation") { assert(1.0 + EPSILON > 1.0, s"EPSILON is too small: $EPSILON.") assert(1.0 + EPSILON / 2.0 === 1.0, s"EPSILON is too big: $EPSILON.") @@ -53,13 +57,13 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { val norm2 = Vectors.norm(v2, 2.0) val v3 = Vectors.sparse(n, indices, indices.map(i => a(i) + 0.5)) val norm3 = Vectors.norm(v3, 2.0) - val squaredDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze) + val squaredDist = breezeSquaredDistance(v1.asBreeze, v2.asBreeze) val fastSquaredDist1 = fastSquaredDistance(v1, norm1, v2, norm2, precision) assert((fastSquaredDist1 - squaredDist) <= precision * squaredDist, s"failed with m = $m") val fastSquaredDist2 = fastSquaredDistance(v1, norm1, Vectors.dense(v2.toArray), norm2, precision) assert((fastSquaredDist2 - squaredDist) <= precision * squaredDist, s"failed with m = $m") - val squaredDist2 = breezeSquaredDistance(v2.toBreeze, v3.toBreeze) + val squaredDist2 = breezeSquaredDistance(v2.asBreeze, v3.asBreeze) val fastSquaredDist3 = fastSquaredDistance(v2, norm2, v3, norm3, precision) assert((fastSquaredDist3 - squaredDist2) <= precision * squaredDist2, s"failed with m = $m") @@ -67,7 +71,7 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { val v4 = Vectors.sparse(n, indices.slice(0, m - 10), indices.map(i => a(i) + 0.5).slice(0, m - 10)) val norm4 = Vectors.norm(v4, 2.0) - val squaredDist = breezeSquaredDistance(v2.toBreeze, v4.toBreeze) + val squaredDist = breezeSquaredDistance(v2.asBreeze, v4.asBreeze) val fastSquaredDist = fastSquaredDistance(v2, norm2, v4, norm4, precision) assert((fastSquaredDist - squaredDist) <= precision * squaredDist, s"failed with m = $m") @@ -84,7 +88,7 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { """.stripMargin val tempDir = Utils.createTempDir() val file = new File(tempDir.getPath, "part-00000") - Files.write(lines, file, Charsets.US_ASCII) + Files.write(lines, file, StandardCharsets.UTF_8) val path = tempDir.toURI.toString val pointsWithNumFeatures = loadLibSVMFile(sc, path, 6).collect() @@ -117,7 +121,7 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { """.stripMargin val tempDir = Utils.createTempDir() val file = new File(tempDir.getPath, "part-00000") - Files.write(lines, file, Charsets.US_ASCII) + Files.write(lines, file, StandardCharsets.UTF_8) val path = tempDir.toURI.toString intercept[SparkException] { @@ -134,7 +138,7 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { """.stripMargin val tempDir = Utils.createTempDir() val file = new File(tempDir.getPath, "part-00000") - Files.write(lines, file, Charsets.US_ASCII) + Files.write(lines, file, StandardCharsets.UTF_8) val path = tempDir.toURI.toString intercept[SparkException] { @@ -151,13 +155,17 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { val tempDir = Utils.createTempDir() val outputDir = new File(tempDir, "output") MLUtils.saveAsLibSVMFile(examples, outputDir.toURI.toString) - val lines = outputDir.listFiles() + val sources = outputDir.listFiles() .filter(_.getName.startsWith("part-")) - .flatMap(Source.fromFile(_).getLines()) - .toSet - val expected = Set("1.1 1:1.23 3:4.56", "0.0 1:1.01 2:2.02 3:3.03") - assert(lines === expected) - Utils.deleteRecursively(tempDir) + .map(Source.fromFile) + Utils.tryWithSafeFinally { + val lines = sources.flatMap(_.getLines()).toSet + val expected = Set("1.1 1:1.23 3:4.56", "0.0 1:1.01 2:2.02 3:3.03") + assert(lines === expected) + } { + sources.foreach(_.close()) + Utils.deleteRecursively(tempDir) + } } test("appendBias") { @@ -182,8 +190,8 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { for (folds <- 2 to 10) { for (seed <- 1 to 5) { val foldedRdds = kFold(data, folds, seed) - assert(foldedRdds.size === folds) - foldedRdds.map { case (training, validation) => + assert(foldedRdds.length === folds) + foldedRdds.foreach { case (training, validation) => val result = validation.union(training).collect().sorted val validationSize = validation.collect().size.toFloat assert(validationSize > 0, "empty validation data") @@ -245,4 +253,104 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { assert(log1pExp(-13.8) ~== math.log1p(math.exp(-13.8)) absTol 1E-10) assert(log1pExp(-238423789.865) ~== math.log1p(math.exp(-238423789.865)) absTol 1E-10) } + + test("convertVectorColumnsToML") { + val x = Vectors.sparse(2, Array(1), Array(1.0)) + val metadata = new MetadataBuilder().putLong("numFeatures", 2L).build() + val y = Vectors.dense(2.0, 3.0) + val z = Vectors.dense(4.0) + val p = (5.0, z) + val w = Vectors.dense(6.0).asML + val df = Seq((0, x, y, p, w)).toDF("id", "x", "y", "p", "w") + .withColumn("x", col("x"), metadata) + val newDF1 = convertVectorColumnsToML(df) + assert(newDF1.schema("x").metadata === metadata, "Metadata should be preserved.") + val new1 = newDF1.first() + assert(new1 === Row(0, x.asML, y.asML, Row(5.0, z), w)) + val new2 = convertVectorColumnsToML(df, "x", "y").first() + assert(new2 === new1) + val new3 = convertVectorColumnsToML(df, "y", "w").first() + assert(new3 === Row(0, x, y.asML, Row(5.0, z), w)) + intercept[IllegalArgumentException] { + convertVectorColumnsToML(df, "p") + } + intercept[IllegalArgumentException] { + convertVectorColumnsToML(df, "p._2") + } + } + + test("convertVectorColumnsFromML") { + val x = Vectors.sparse(2, Array(1), Array(1.0)).asML + val metadata = new MetadataBuilder().putLong("numFeatures", 2L).build() + val y = Vectors.dense(2.0, 3.0).asML + val z = Vectors.dense(4.0).asML + val p = (5.0, z) + val w = Vectors.dense(6.0) + val df = Seq((0, x, y, p, w)).toDF("id", "x", "y", "p", "w") + .withColumn("x", col("x"), metadata) + val newDF1 = convertVectorColumnsFromML(df) + assert(newDF1.schema("x").metadata === metadata, "Metadata should be preserved.") + val new1 = newDF1.first() + assert(new1 === Row(0, Vectors.fromML(x), Vectors.fromML(y), Row(5.0, z), w)) + val new2 = convertVectorColumnsFromML(df, "x", "y").first() + assert(new2 === new1) + val new3 = convertVectorColumnsFromML(df, "y", "w").first() + assert(new3 === Row(0, x, Vectors.fromML(y), Row(5.0, z), w)) + intercept[IllegalArgumentException] { + convertVectorColumnsFromML(df, "p") + } + intercept[IllegalArgumentException] { + convertVectorColumnsFromML(df, "p._2") + } + } + + test("convertMatrixColumnsToML") { + val x = Matrices.sparse(3, 2, Array(0, 2, 3), Array(0, 2, 1), Array(0.0, -1.2, 0.0)) + val metadata = new MetadataBuilder().putLong("numFeatures", 2L).build() + val y = Matrices.dense(2, 1, Array(0.2, 1.3)) + val z = Matrices.ones(1, 1) + val p = (5.0, z) + val w = Matrices.dense(1, 1, Array(4.5)).asML + val df = Seq((0, x, y, p, w)).toDF("id", "x", "y", "p", "w") + .withColumn("x", col("x"), metadata) + val newDF1 = convertMatrixColumnsToML(df) + assert(newDF1.schema("x").metadata === metadata, "Metadata should be preserved.") + val new1 = newDF1.first() + assert(new1 === Row(0, x.asML, y.asML, Row(5.0, z), w)) + val new2 = convertMatrixColumnsToML(df, "x", "y").first() + assert(new2 === new1) + val new3 = convertMatrixColumnsToML(df, "y", "w").first() + assert(new3 === Row(0, x, y.asML, Row(5.0, z), w)) + intercept[IllegalArgumentException] { + convertMatrixColumnsToML(df, "p") + } + intercept[IllegalArgumentException] { + convertMatrixColumnsToML(df, "p._2") + } + } + + test("convertMatrixColumnsFromML") { + val x = Matrices.sparse(3, 2, Array(0, 2, 3), Array(0, 2, 1), Array(0.0, -1.2, 0.0)).asML + val metadata = new MetadataBuilder().putLong("numFeatures", 2L).build() + val y = Matrices.dense(2, 1, Array(0.2, 1.3)).asML + val z = Matrices.ones(1, 1).asML + val p = (5.0, z) + val w = Matrices.dense(1, 1, Array(4.5)) + val df = Seq((0, x, y, p, w)).toDF("id", "x", "y", "p", "w") + .withColumn("x", col("x"), metadata) + val newDF1 = convertMatrixColumnsFromML(df) + assert(newDF1.schema("x").metadata === metadata, "Metadata should be preserved.") + val new1 = newDF1.first() + assert(new1 === Row(0, Matrices.fromML(x), Matrices.fromML(y), Row(5.0, z), w)) + val new2 = convertMatrixColumnsFromML(df, "x", "y").first() + assert(new2 === new1) + val new3 = convertMatrixColumnsFromML(df, "y", "w").first() + assert(new3 === Row(0, x, Matrices.fromML(y), Row(5.0, z), w)) + intercept[IllegalArgumentException] { + convertMatrixColumnsFromML(df, "p") + } + intercept[IllegalArgumentException] { + convertMatrixColumnsFromML(df, "p._2") + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala index 5d1796ef6572..720237bd2ddd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala @@ -17,30 +17,53 @@ package org.apache.spark.mllib.util -import org.scalatest.{BeforeAndAfterAll, Suite} +import java.io.File -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.SQLContext +import org.scalatest.Suite -trait MLlibTestSparkContext extends BeforeAndAfterAll { self: Suite => +import org.apache.spark.SparkContext +import org.apache.spark.ml.util.TempDirectory +import org.apache.spark.sql.{SparkSession, SQLContext, SQLImplicits} +import org.apache.spark.util.Utils + +trait MLlibTestSparkContext extends TempDirectory { self: Suite => + @transient var spark: SparkSession = _ @transient var sc: SparkContext = _ - @transient var sqlContext: SQLContext = _ + @transient var checkpointDir: String = _ override def beforeAll() { super.beforeAll() - val conf = new SparkConf() - .setMaster("local[2]") - .setAppName("MLlibUnitTest") - sc = new SparkContext(conf) - sqlContext = new SQLContext(sc) + spark = SparkSession.builder + .master("local[2]") + .appName("MLlibUnitTest") + .getOrCreate() + sc = spark.sparkContext + + checkpointDir = Utils.createDirectory(tempDir.getCanonicalPath, "checkpoints").toString + sc.setCheckpointDir(checkpointDir) } override def afterAll() { - sqlContext = null - if (sc != null) { - sc.stop() + try { + Utils.deleteRecursively(new File(checkpointDir)) + SparkSession.clearActiveSession() + if (spark != null) { + spark.stop() + } + spark = null + } finally { + super.afterAll() } - sc = null - super.afterAll() + } + + /** + * A helper object for importing SQL implicits. + * + * Note that the alternative of importing `spark.implicits._` is not possible here. + * This is because we create the `SQLContext` immediately before the first test is run, + * but the implicits import is needed in the constructor. + */ + protected object testImplicits extends SQLImplicits { + protected override def _sqlContext: SQLContext = self.spark.sqlContext } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala index 352193a67860..d39865a19a5c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala @@ -17,9 +17,10 @@ package org.apache.spark.mllib.util -import org.apache.spark.mllib.linalg.{Matrix, Vector} import org.scalatest.exceptions.TestFailedException +import org.apache.spark.mllib.linalg.{Matrix, Vector} + object TestingUtils { val ABS_TOL_MSG = " using absolute tolerance" @@ -153,7 +154,7 @@ object TestingUtils { */ def absTol(eps: Double): CompareVectorRightSide = CompareVectorRightSide( (x: Vector, y: Vector, eps: Double) => { - x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 absTol eps) + x.size == y.size && x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 absTol eps) }, x, eps, ABS_TOL_MSG) /** @@ -163,7 +164,7 @@ object TestingUtils { */ def relTol(eps: Double): CompareVectorRightSide = CompareVectorRightSide( (x: Vector, y: Vector, eps: Double) => { - x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) + x.size == y.size && x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) }, x, eps, REL_TOL_MSG) override def toString: String = x.toString @@ -206,7 +207,7 @@ object TestingUtils { if (r.fun(x, r.y, r.eps)) { throw new TestFailedException( s"Did not expect \n$x\n and \n${r.y}\n to be within " + - "${r.eps}${r.method} for all elements.", 0) + s"${r.eps}${r.method} for all elements.", 0) } true } @@ -216,7 +217,8 @@ object TestingUtils { */ def absTol(eps: Double): CompareMatrixRightSide = CompareMatrixRightSide( (x: Matrix, y: Matrix, eps: Double) => { - x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 absTol eps) + x.numRows == y.numRows && x.numCols == y.numCols && + x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 absTol eps) }, x, eps, ABS_TOL_MSG) /** @@ -226,7 +228,8 @@ object TestingUtils { */ def relTol(eps: Double): CompareMatrixRightSide = CompareMatrixRightSide( (x: Matrix, y: Matrix, eps: Double) => { - x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) + x.numRows == y.numRows && x.numCols == y.numCols && + x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) }, x, eps, REL_TOL_MSG) override def toString: String = x.toString diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala index 8f475f30249d..3fcf1cf2c263 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala @@ -17,10 +17,11 @@ package org.apache.spark.mllib.util +import org.scalatest.exceptions.TestFailedException + import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.{Matrices, Vectors} import org.apache.spark.mllib.util.TestingUtils._ -import org.scalatest.exceptions.TestFailedException class TestingUtilsSuite extends SparkFunSuite { @@ -108,6 +109,10 @@ class TestingUtilsSuite extends SparkFunSuite { assert(Vectors.dense(Array(3.1, 3.5)) !~= Vectors.dense(Array(3.135, 3.534)) relTol 0.01) assert(!(Vectors.dense(Array(3.1, 3.5)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01)) assert(!(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.135, 3.534)) relTol 0.01)) + assert(Vectors.dense(Array(3.1)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array.empty[Double]) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array(3.1)) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array.empty[Double]) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) // Should throw exception with message when test fails. intercept[TestFailedException]( @@ -116,6 +121,12 @@ class TestingUtilsSuite extends SparkFunSuite { intercept[TestFailedException]( Vectors.dense(Array(3.1, 3.5)) ~== Vectors.dense(Array(3.135, 3.534)) relTol 0.01) + intercept[TestFailedException]( + Vectors.dense(Array(3.1)) ~== Vectors.dense(Array(3.535, 3.534)) relTol 0.01) + + intercept[TestFailedException]( + Vectors.dense(Array.empty[Double]) ~== Vectors.dense(Array(3.135)) relTol 0.01) + // Comparing against zero should fail the test and throw exception with message // saying that the relative error is meaningless in this situation. intercept[TestFailedException]( @@ -124,12 +135,18 @@ class TestingUtilsSuite extends SparkFunSuite { intercept[TestFailedException]( Vectors.dense(Array(3.1, 0.01)) ~== Vectors.sparse(2, Array(0), Array(3.13)) relTol 0.01) - // Comparisons of two sparse vectors + // Comparisons of a sparse vector and a dense vector assert(Vectors.dense(Array(3.1, 3.5)) ~== Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) assert(Vectors.dense(Array(3.1, 3.5)) !~== Vectors.sparse(2, Array(0, 1), Array(3.135, 3.534)) relTol 0.01) + + assert(Vectors.dense(Array(3.1)) !~== + Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) + + assert(Vectors.dense(Array.empty[Double]) !~== + Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) } test("Comparing vectors using absolute error.") { @@ -153,6 +170,21 @@ class TestingUtilsSuite extends SparkFunSuite { assert(!(Vectors.dense(Array(3.1, 3.5, 0.0)) ~= Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7, 1 + 1E-3)) absTol 1E-6)) + assert(Vectors.dense(Array(3.1)) !~= + Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5) + + assert(!(Vectors.dense(Array(3.1)) ~= + Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5)) + + assert(Vectors.dense(Array.empty[Double]) !~= + Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5) + + assert(!(Vectors.dense(Array.empty[Double]) ~= + Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5)) + + assert(Vectors.dense(Array.empty[Double]) ~= + Vectors.dense(Array.empty[Double]) absTol 1E-5) + // Should throw exception with message when test fails. intercept[TestFailedException](Vectors.dense(Array(3.1, 3.5, 0.0)) !~== Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6) @@ -160,6 +192,12 @@ class TestingUtilsSuite extends SparkFunSuite { intercept[TestFailedException](Vectors.dense(Array(3.1, 3.5, 0.0)) ~== Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7, 1 + 1E-3)) absTol 1E-6) + intercept[TestFailedException](Vectors.dense(Array(3.1)) ~== + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6) + + intercept[TestFailedException](Vectors.dense(Array.empty[Double]) ~== + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6) + // Comparisons of two sparse vectors assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) ~== Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-8, 2.4 + 1E-7)) absTol 1E-6) @@ -173,6 +211,12 @@ class TestingUtilsSuite extends SparkFunSuite { assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-3, 2.4)) !~== Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) absTol 1E-6) + assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-6, 2.4)) !~== + Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3) + + assert(Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) !~== + Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3) + // Comparisons of a dense vector and a sparse vector assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) ~== Vectors.dense(Array(3.1 + 1E-8, 0, 2.4 + 1E-7)) absTol 1E-6) @@ -182,5 +226,235 @@ class TestingUtilsSuite extends SparkFunSuite { assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~== Vectors.dense(Array(3.1, 1E-3, 2.4)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~== + Vectors.dense(Array(3.1)) absTol 1E-6) + + assert(Vectors.dense(Array.empty[Double]) !~== + Vectors.sparse(3, Array(0, 2), Array(0, 2.4)) absTol 1E-6) + + assert(Vectors.sparse(1, Array(0), Array(3.1)) !~== + Vectors.dense(Array(3.1, 3.2)) absTol 1E-6) + + assert(Vectors.dense(Array(3.1)) !~== + Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) absTol 1E-6) + } + + test("Comparing Matrices using absolute error.") { + + // Comparisons of two dense Matrices + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 3.5 + 2E-7, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.1 + 1E-8, 3.5 + 2E-7, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.1 + 1E-5, 3.5 + 2E-6, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.1 + 1E-5, 3.5 + 2E-6, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(!(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.1 + 1E-5, 3.5 + 2E-6, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6)) + + assert(!(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.1 + 1E-7, 3.5 + 2E-8, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6)) + + assert(Matrices.dense(2, 1, Array(3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.1 + 1E-7, 3.5 + 2E-8, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.dense(2, 1, Array(3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.1 + 1E-7, 3.5 + 2E-8, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.dense(0, 0, Array()) !~= + Matrices.dense(2, 2, Array(3.1 + 1E-7, 3.5 + 2E-8, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.dense(0, 0, Array()) !~== + Matrices.dense(2, 2, Array(3.1 + 1E-7, 3.5 + 2E-8, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + // Should throw exception with message when test fails. + intercept[TestFailedException](Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 3.5 + 2E-7, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + intercept[TestFailedException](Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 3.5 + 2E-7, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-9) + + intercept[TestFailedException](Matrices.dense(2, 1, Array(3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 3.5 + 2E-7, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-5) + + intercept[TestFailedException](Matrices.dense(0, 0, Array()) ~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 3.5 + 2E-7, 3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-5) + + // Comparisons of two sparse Matrices + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) ~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) ~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-9) + + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) !~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-9) + + assert(!(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) ~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5)) absTol 1E-9)) + + assert(!(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5)) absTol 1E-6)) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-9) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.sparse(0, 0, Array(1), Array(0), Array(0)) !~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.sparse(0, 0, Array(1), Array(0), Array(0)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1 + 1E-8, 3.5 + 1E-7)) absTol 1E-6) + + // Comparisons of a dense Matrix and a sparse Matrix + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.1 + 1E-8, 0, 0, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 0, 0, 3.5 + 1E-7)) absTol 1E-6) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 0, 0, 3.5 + 1E-7)) absTol 1E-9) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.1 + 1E-8, 0, 0, 3.5 + 1E-7)) absTol 1E-9) + + assert(!(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.1 + 1E-8, 0, 0, 3.5 + 1E-7)) absTol 1E-9)) + + assert(!(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.1 + 1E-8, 0, 0, 3.5 + 1E-7)) absTol 1E-6)) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.dense(2, 1, Array(3.1 + 1E-8, 0)) absTol 1E-6) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.dense(2, 1, Array(3.1 + 1E-8, 0)) absTol 1E-6) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.dense(0, 0, Array()) absTol 1E-6) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.dense(0, 0, Array()) absTol 1E-6) + } + + test("Comparing Matrices using relative error.") { + + // Comparisons of two dense Matrices + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.130, 3.534, 3.130, 3.534)) relTol 0.01) + + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.130, 3.534, 3.130, 3.534)) relTol 0.01) + + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.135, 3.534, 3.135, 3.534)) relTol 0.01) + + assert(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.135, 3.534, 3.135, 3.534)) relTol 0.01) + + assert(!(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.134, 3.535, 3.134, 3.535)) relTol 0.01)) + + assert(!(Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.130, 3.534, 3.130, 3.534)) relTol 0.01)) + + assert(Matrices.dense(2, 1, Array(3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) relTol 0.01) + + assert(Matrices.dense(2, 1, Array(3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) relTol 0.01) + + assert(Matrices.dense(0, 0, Array()) !~= + Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) relTol 0.01) + + assert(Matrices.dense(0, 0, Array()) !~== + Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) relTol 0.01) + + // Should throw exception with message when test fails. + intercept[TestFailedException](Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.130, 3.534, 3.130, 3.534)) relTol 0.01) + + intercept[TestFailedException](Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.135, 3.534, 3.135, 3.534)) relTol 0.01) + + intercept[TestFailedException](Matrices.dense(2, 1, Array(3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) relTol 0.01) + + intercept[TestFailedException](Matrices.dense(0, 0, Array()) ~== + Matrices.dense(2, 2, Array(3.1, 3.5, 3.1, 3.5)) relTol 0.01) + + // Comparisons of two sparse Matrices + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) ~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.130, 3.534)) relTol 0.01) + + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) ~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.130, 3.534)) relTol 0.01) + + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.135, 3.534)) relTol 0.01) + + assert(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) !~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.135, 3.534)) relTol 0.01) + + assert(!(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) ~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.135, 3.534)) relTol 0.01)) + + assert(!(Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.130, 3.534)) relTol 0.01)) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) relTol 0.01) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) relTol 0.01) + + assert(Matrices.sparse(0, 0, Array(1), Array(0), Array(0)) !~== + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) relTol 0.01) + + assert(Matrices.sparse(0, 0, Array(1), Array(0), Array(0)) !~= + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(3.1, 3.5)) relTol 0.01) + + // Comparisons of a dense Matrix and a sparse Matrix + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.130, 0, 0, 3.534)) relTol 0.01) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) ~== + Matrices.dense(2, 2, Array(3.130, 0, 0, 3.534)) relTol 0.01) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.135, 0, 0, 3.534)) relTol 0.01) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.dense(2, 2, Array(3.135, 0, 0, 3.534)) relTol 0.01) + + assert(!(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) ~= + Matrices.dense(2, 2, Array(3.135, 0, 0, 3.534)) relTol 0.01)) + + assert(!(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.dense(2, 2, Array(3.130, 0, 0, 3.534)) relTol 0.01)) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.dense(2, 1, Array(3.1, 0)) relTol 0.01) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.dense(2, 1, Array(3.1, 0)) relTol 0.01) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~== + Matrices.dense(0, 0, Array()) relTol 0.01) + + assert(Matrices.sparse(2, 2, Array(0, 1, 2), Array(0, 1), Array(3.1, 3.5)) !~= + Matrices.dense(0, 0, Array()) relTol 0.01) } } diff --git a/network/common/pom.xml b/network/common/pom.xml deleted file mode 100644 index 9af6cc5e925f..000000000000 --- a/network/common/pom.xml +++ /dev/null @@ -1,108 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-network-common_2.10 - jar - Spark Project Networking - http://spark.apache.org/ - - network-common - - - - - - io.netty - netty-all - - - - - org.slf4j - slf4j-api - provided - - - com.google.code.findbugs - jsr305 - - - - com.google.guava - guava - compile - - - - - log4j - log4j - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - org.mockito - mockito-core - test - - - org.slf4j - slf4j-log4j12 - test - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - - org.apache.maven.plugins - maven-jar-plugin - - - test-jar-on-test-compile - test-compile - - test-jar - - - - - - - diff --git a/network/common/src/main/java/org/apache/spark/network/TransportContext.java b/network/common/src/main/java/org/apache/spark/network/TransportContext.java deleted file mode 100644 index 43900e6f2c97..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/TransportContext.java +++ /dev/null @@ -1,151 +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.network; - -import java.util.List; - -import com.google.common.collect.Lists; -import io.netty.channel.Channel; -import io.netty.channel.socket.SocketChannel; -import io.netty.handler.timeout.IdleStateHandler; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.client.TransportClientBootstrap; -import org.apache.spark.network.client.TransportClientFactory; -import org.apache.spark.network.client.TransportResponseHandler; -import org.apache.spark.network.protocol.MessageDecoder; -import org.apache.spark.network.protocol.MessageEncoder; -import org.apache.spark.network.server.RpcHandler; -import org.apache.spark.network.server.TransportChannelHandler; -import org.apache.spark.network.server.TransportRequestHandler; -import org.apache.spark.network.server.TransportServer; -import org.apache.spark.network.server.TransportServerBootstrap; -import org.apache.spark.network.util.NettyUtils; -import org.apache.spark.network.util.TransportConf; -import org.apache.spark.network.util.TransportFrameDecoder; - -/** - * Contains the context to create a {@link TransportServer}, {@link TransportClientFactory}, and to - * setup Netty Channel pipelines with a {@link org.apache.spark.network.server.TransportChannelHandler}. - * - * There are two communication protocols that the TransportClient provides, control-plane RPCs and - * data-plane "chunk fetching". The handling of the RPCs is performed outside of the scope of the - * TransportContext (i.e., by a user-provided handler), and it is responsible for setting up streams - * which can be streamed through the data plane in chunks using zero-copy IO. - * - * The TransportServer and TransportClientFactory both create a TransportChannelHandler for each - * channel. As each TransportChannelHandler contains a TransportClient, this enables server - * processes to send messages back to the client on an existing channel. - */ -public class TransportContext { - private final Logger logger = LoggerFactory.getLogger(TransportContext.class); - - private final TransportConf conf; - private final RpcHandler rpcHandler; - - private final MessageEncoder encoder; - private final MessageDecoder decoder; - - public TransportContext(TransportConf conf, RpcHandler rpcHandler) { - this.conf = conf; - this.rpcHandler = rpcHandler; - this.encoder = new MessageEncoder(); - this.decoder = new MessageDecoder(); - } - - /** - * Initializes a ClientFactory which runs the given TransportClientBootstraps prior to returning - * a new Client. Bootstraps will be executed synchronously, and must run successfully in order - * to create a Client. - */ - public TransportClientFactory createClientFactory(List bootstraps) { - return new TransportClientFactory(this, bootstraps); - } - - public TransportClientFactory createClientFactory() { - return createClientFactory(Lists.newArrayList()); - } - - /** Create a server which will attempt to bind to a specific port. */ - public TransportServer createServer(int port, List bootstraps) { - return new TransportServer(this, port, rpcHandler, bootstraps); - } - - /** Creates a new server, binding to any available ephemeral port. */ - public TransportServer createServer(List bootstraps) { - return createServer(0, bootstraps); - } - - public TransportServer createServer() { - return createServer(0, Lists.newArrayList()); - } - - public TransportChannelHandler initializePipeline(SocketChannel channel) { - return initializePipeline(channel, rpcHandler); - } - - /** - * Initializes a client or server Netty Channel Pipeline which encodes/decodes messages and - * has a {@link org.apache.spark.network.server.TransportChannelHandler} to handle request or - * response messages. - * - * @param channel The channel to initialize. - * @param channelRpcHandler The RPC handler to use for the channel. - * - * @return Returns the created TransportChannelHandler, which includes a TransportClient that can - * be used to communicate on this channel. The TransportClient is directly associated with a - * ChannelHandler to ensure all users of the same channel get the same TransportClient object. - */ - public TransportChannelHandler initializePipeline( - SocketChannel channel, - RpcHandler channelRpcHandler) { - try { - TransportChannelHandler channelHandler = createChannelHandler(channel, channelRpcHandler); - channel.pipeline() - .addLast("encoder", encoder) - .addLast(TransportFrameDecoder.HANDLER_NAME, NettyUtils.createFrameDecoder()) - .addLast("decoder", decoder) - .addLast("idleStateHandler", new IdleStateHandler(0, 0, conf.connectionTimeoutMs() / 1000)) - // NOTE: Chunks are currently guaranteed to be returned in the order of request, but this - // would require more logic to guarantee if this were not part of the same event loop. - .addLast("handler", channelHandler); - return channelHandler; - } catch (RuntimeException e) { - logger.error("Error while initializing Netty pipeline", e); - throw e; - } - } - - /** - * Creates the server- and client-side handler which is used to handle both RequestMessages and - * ResponseMessages. The channel is expected to have been successfully created, though certain - * properties (such as the remoteAddress()) may not be available yet. - */ - private TransportChannelHandler createChannelHandler(Channel channel, RpcHandler rpcHandler) { - TransportResponseHandler responseHandler = new TransportResponseHandler(channel); - TransportClient client = new TransportClient(channel, responseHandler); - TransportRequestHandler requestHandler = new TransportRequestHandler(channel, client, - rpcHandler); - return new TransportChannelHandler(client, responseHandler, requestHandler, - conf.connectionTimeoutMs()); - } - - public TransportConf getConf() { return conf; } -} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/LazyFileRegion.java b/network/common/src/main/java/org/apache/spark/network/buffer/LazyFileRegion.java deleted file mode 100644 index 81bc8ec40fc8..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/buffer/LazyFileRegion.java +++ /dev/null @@ -1,111 +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.network.buffer; - -import java.io.FileInputStream; -import java.io.File; -import java.io.IOException; -import java.nio.channels.FileChannel; -import java.nio.channels.WritableByteChannel; - -import com.google.common.base.Objects; -import io.netty.channel.FileRegion; -import io.netty.util.AbstractReferenceCounted; - -import org.apache.spark.network.util.JavaUtils; - -/** - * A FileRegion implementation that only creates the file descriptor when the region is being - * transferred. This cannot be used with Epoll because there is no native support for it. - * - * This is mostly copied from DefaultFileRegion implementation in Netty. In the future, we - * should push this into Netty so the native Epoll transport can support this feature. - */ -public final class LazyFileRegion extends AbstractReferenceCounted implements FileRegion { - - private final File file; - private final long position; - private final long count; - - private FileChannel channel; - - private long numBytesTransferred = 0L; - - /** - * @param file file to transfer. - * @param position start position for the transfer. - * @param count number of bytes to transfer starting from position. - */ - public LazyFileRegion(File file, long position, long count) { - this.file = file; - this.position = position; - this.count = count; - } - - @Override - protected void deallocate() { - JavaUtils.closeQuietly(channel); - } - - @Override - public long position() { - return position; - } - - @Override - public long transfered() { - return numBytesTransferred; - } - - @Override - public long count() { - return count; - } - - @Override - public long transferTo(WritableByteChannel target, long position) throws IOException { - if (channel == null) { - channel = new FileInputStream(file).getChannel(); - } - - long count = this.count - position; - if (count < 0 || position < 0) { - throw new IllegalArgumentException( - "position out of range: " + position + " (expected: 0 - " + (count - 1) + ')'); - } - - if (count == 0) { - return 0L; - } - - long written = channel.transferTo(this.position + position, count, target); - if (written > 0) { - numBytesTransferred += written; - } - return written; - } - - @Override - public String toString() { - return Objects.toStringHelper(this) - .add("file", file) - .add("position", position) - .add("count", count) - .toString(); - } -} diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java deleted file mode 100644 index a0ba223e340a..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java +++ /dev/null @@ -1,280 +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.network.client; - -import java.io.Closeable; -import java.io.IOException; -import java.net.SocketAddress; -import java.util.UUID; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import javax.annotation.Nullable; - -import com.google.common.base.Objects; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.util.concurrent.SettableFuture; -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.protocol.ChunkFetchRequest; -import org.apache.spark.network.protocol.RpcRequest; -import org.apache.spark.network.protocol.StreamChunkId; -import org.apache.spark.network.protocol.StreamRequest; -import org.apache.spark.network.util.NettyUtils; - -/** - * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow - * efficient transfer of a large amount of data, broken up into chunks with size ranging from - * hundreds of KB to a few MB. - * - * Note that while this client deals with the fetching of chunks from a stream (i.e., data plane), - * the actual setup of the streams is done outside the scope of the transport layer. The convenience - * method "sendRPC" is provided to enable control plane communication between the client and server - * to perform this setup. - * - * For example, a typical workflow might be: - * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100 - * client.fetchChunk(streamId = 100, chunkIndex = 0, callback) - * client.fetchChunk(streamId = 100, chunkIndex = 1, callback) - * ... - * client.sendRPC(new CloseStream(100)) - * - * Construct an instance of TransportClient using {@link TransportClientFactory}. A single - * TransportClient may be used for multiple streams, but any given stream must be restricted to a - * single client, in order to avoid out-of-order responses. - * - * NB: This class is used to make requests to the server, while {@link TransportResponseHandler} is - * responsible for handling responses from the server. - * - * Concurrency: thread safe and can be called from multiple threads. - */ -public class TransportClient implements Closeable { - private final Logger logger = LoggerFactory.getLogger(TransportClient.class); - - private final Channel channel; - private final TransportResponseHandler handler; - @Nullable private String clientId; - - public TransportClient(Channel channel, TransportResponseHandler handler) { - this.channel = Preconditions.checkNotNull(channel); - this.handler = Preconditions.checkNotNull(handler); - } - - public Channel getChannel() { - return channel; - } - - public boolean isActive() { - return channel.isOpen() || channel.isActive(); - } - - public SocketAddress getSocketAddress() { - return channel.remoteAddress(); - } - - /** - * Returns the ID used by the client to authenticate itself when authentication is enabled. - * - * @return The client ID, or null if authentication is disabled. - */ - public String getClientId() { - return clientId; - } - - /** - * Sets the authenticated client ID. This is meant to be used by the authentication layer. - * - * Trying to set a different client ID after it's been set will result in an exception. - */ - public void setClientId(String id) { - Preconditions.checkState(clientId == null, "Client ID has already been set."); - this.clientId = id; - } - - /** - * Requests a single chunk from the remote side, from the pre-negotiated streamId. - * - * Chunk indices go from 0 onwards. It is valid to request the same chunk multiple times, though - * some streams may not support this. - * - * Multiple fetchChunk requests may be outstanding simultaneously, and the chunks are guaranteed - * to be returned in the same order that they were requested, assuming only a single - * TransportClient is used to fetch the chunks. - * - * @param streamId Identifier that refers to a stream in the remote StreamManager. This should - * be agreed upon by client and server beforehand. - * @param chunkIndex 0-based index of the chunk to fetch - * @param callback Callback invoked upon successful receipt of chunk, or upon any failure. - */ - public void fetchChunk( - long streamId, - final int chunkIndex, - final ChunkReceivedCallback callback) { - final String serverAddr = NettyUtils.getRemoteAddress(channel); - final long startTime = System.currentTimeMillis(); - logger.debug("Sending fetch chunk request {} to {}", chunkIndex, serverAddr); - - final StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex); - handler.addFetchRequest(streamChunkId, callback); - - channel.writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener( - new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) throws Exception { - if (future.isSuccess()) { - long timeTaken = System.currentTimeMillis() - startTime; - logger.trace("Sending request {} to {} took {} ms", streamChunkId, serverAddr, - timeTaken); - } else { - String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId, - serverAddr, future.cause()); - logger.error(errorMsg, future.cause()); - handler.removeFetchRequest(streamChunkId); - channel.close(); - try { - callback.onFailure(chunkIndex, new IOException(errorMsg, future.cause())); - } catch (Exception e) { - logger.error("Uncaught exception in RPC response callback handler!", e); - } - } - } - }); - } - - /** - * Request to stream the data with the given stream ID from the remote end. - * - * @param streamId The stream to fetch. - * @param callback Object to call with the stream data. - */ - public void stream(final String streamId, final StreamCallback callback) { - final String serverAddr = NettyUtils.getRemoteAddress(channel); - final long startTime = System.currentTimeMillis(); - logger.debug("Sending stream request for {} to {}", streamId, serverAddr); - - // Need to synchronize here so that the callback is added to the queue and the RPC is - // written to the socket atomically, so that callbacks are called in the right order - // when responses arrive. - synchronized (this) { - handler.addStreamCallback(callback); - channel.writeAndFlush(new StreamRequest(streamId)).addListener( - new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) throws Exception { - if (future.isSuccess()) { - long timeTaken = System.currentTimeMillis() - startTime; - logger.trace("Sending request for {} to {} took {} ms", streamId, serverAddr, - timeTaken); - } else { - String errorMsg = String.format("Failed to send request for %s to %s: %s", streamId, - serverAddr, future.cause()); - logger.error(errorMsg, future.cause()); - channel.close(); - try { - callback.onFailure(streamId, new IOException(errorMsg, future.cause())); - } catch (Exception e) { - logger.error("Uncaught exception in RPC response callback handler!", e); - } - } - } - }); - } - } - - /** - * Sends an opaque message to the RpcHandler on the server-side. The callback will be invoked - * with the server's response or upon any failure. - */ - public void sendRpc(byte[] message, final RpcResponseCallback callback) { - final String serverAddr = NettyUtils.getRemoteAddress(channel); - final long startTime = System.currentTimeMillis(); - logger.trace("Sending RPC to {}", serverAddr); - - final long requestId = Math.abs(UUID.randomUUID().getLeastSignificantBits()); - handler.addRpcRequest(requestId, callback); - - channel.writeAndFlush(new RpcRequest(requestId, message)).addListener( - new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) throws Exception { - if (future.isSuccess()) { - long timeTaken = System.currentTimeMillis() - startTime; - logger.trace("Sending request {} to {} took {} ms", requestId, serverAddr, timeTaken); - } else { - String errorMsg = String.format("Failed to send RPC %s to %s: %s", requestId, - serverAddr, future.cause()); - logger.error(errorMsg, future.cause()); - handler.removeRpcRequest(requestId); - channel.close(); - try { - callback.onFailure(new IOException(errorMsg, future.cause())); - } catch (Exception e) { - logger.error("Uncaught exception in RPC response callback handler!", e); - } - } - } - }); - } - - /** - * Synchronously sends an opaque message to the RpcHandler on the server-side, waiting for up to - * a specified timeout for a response. - */ - public byte[] sendRpcSync(byte[] message, long timeoutMs) { - final SettableFuture result = SettableFuture.create(); - - sendRpc(message, new RpcResponseCallback() { - @Override - public void onSuccess(byte[] response) { - result.set(response); - } - - @Override - public void onFailure(Throwable e) { - result.setException(e); - } - }); - - try { - return result.get(timeoutMs, TimeUnit.MILLISECONDS); - } catch (ExecutionException e) { - throw Throwables.propagate(e.getCause()); - } catch (Exception e) { - throw Throwables.propagate(e); - } - } - - @Override - public void close() { - // close is a local operation and should finish with milliseconds; timeout just to be safe - channel.close().awaitUninterruptibly(10, TimeUnit.SECONDS); - } - - @Override - public String toString() { - return Objects.toStringHelper(this) - .add("remoteAdress", channel.remoteAddress()) - .add("clientId", clientId) - .add("isActive", isActive()) - .toString(); - } -} diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java deleted file mode 100644 index 4952ffb44bb8..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ /dev/null @@ -1,241 +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.network.client; - -import java.io.Closeable; -import java.io.IOException; -import java.net.InetSocketAddress; -import java.net.SocketAddress; -import java.util.List; -import java.util.Random; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicReference; - -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.Lists; -import io.netty.bootstrap.Bootstrap; -import io.netty.buffer.PooledByteBufAllocator; -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelInitializer; -import io.netty.channel.ChannelOption; -import io.netty.channel.EventLoopGroup; -import io.netty.channel.socket.SocketChannel; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.TransportContext; -import org.apache.spark.network.server.TransportChannelHandler; -import org.apache.spark.network.util.IOMode; -import org.apache.spark.network.util.JavaUtils; -import org.apache.spark.network.util.NettyUtils; -import org.apache.spark.network.util.TransportConf; - -/** - * Factory for creating {@link TransportClient}s by using createClient. - * - * The factory maintains a connection pool to other hosts and should return the same - * TransportClient for the same remote host. It also shares a single worker thread pool for - * all TransportClients. - * - * TransportClients will be reused whenever possible. Prior to completing the creation of a new - * TransportClient, all given {@link TransportClientBootstrap}s will be run. - */ -public class TransportClientFactory implements Closeable { - - /** A simple data structure to track the pool of clients between two peer nodes. */ - private static class ClientPool { - TransportClient[] clients; - Object[] locks; - - public ClientPool(int size) { - clients = new TransportClient[size]; - locks = new Object[size]; - for (int i = 0; i < size; i++) { - locks[i] = new Object(); - } - } - } - - private final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class); - - private final TransportContext context; - private final TransportConf conf; - private final List clientBootstraps; - private final ConcurrentHashMap connectionPool; - - /** Random number generator for picking connections between peers. */ - private final Random rand; - private final int numConnectionsPerPeer; - - private final Class socketChannelClass; - private EventLoopGroup workerGroup; - private PooledByteBufAllocator pooledAllocator; - - public TransportClientFactory( - TransportContext context, - List clientBootstraps) { - this.context = Preconditions.checkNotNull(context); - this.conf = context.getConf(); - this.clientBootstraps = Lists.newArrayList(Preconditions.checkNotNull(clientBootstraps)); - this.connectionPool = new ConcurrentHashMap(); - this.numConnectionsPerPeer = conf.numConnectionsPerPeer(); - this.rand = new Random(); - - IOMode ioMode = IOMode.valueOf(conf.ioMode()); - this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode); - // TODO: Make thread pool name configurable. - this.workerGroup = NettyUtils.createEventLoop(ioMode, conf.clientThreads(), "shuffle-client"); - this.pooledAllocator = NettyUtils.createPooledByteBufAllocator( - conf.preferDirectBufs(), false /* allowCache */, conf.clientThreads()); - } - - /** - * Create a {@link TransportClient} connecting to the given remote host / port. - * - * We maintains an array of clients (size determined by spark.shuffle.io.numConnectionsPerPeer) - * and randomly picks one to use. If no client was previously created in the randomly selected - * spot, this function creates a new client and places it there. - * - * Prior to the creation of a new TransportClient, we will execute all - * {@link TransportClientBootstrap}s that are registered with this factory. - * - * This blocks until a connection is successfully established and fully bootstrapped. - * - * Concurrency: This method is safe to call from multiple threads. - */ - public TransportClient createClient(String remoteHost, int remotePort) throws IOException { - // Get connection from the connection pool first. - // If it is not found or not active, create a new one. - final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort); - - // Create the ClientPool if we don't have it yet. - ClientPool clientPool = connectionPool.get(address); - if (clientPool == null) { - connectionPool.putIfAbsent(address, new ClientPool(numConnectionsPerPeer)); - clientPool = connectionPool.get(address); - } - - int clientIndex = rand.nextInt(numConnectionsPerPeer); - TransportClient cachedClient = clientPool.clients[clientIndex]; - - if (cachedClient != null && cachedClient.isActive()) { - logger.trace("Returning cached connection to {}: {}", address, cachedClient); - return cachedClient; - } - - // If we reach here, we don't have an existing connection open. Let's create a new one. - // Multiple threads might race here to create new connections. Keep only one of them active. - synchronized (clientPool.locks[clientIndex]) { - cachedClient = clientPool.clients[clientIndex]; - - if (cachedClient != null) { - if (cachedClient.isActive()) { - logger.trace("Returning cached connection to {}: {}", address, cachedClient); - return cachedClient; - } else { - logger.info("Found inactive connection to {}, creating a new one.", address); - } - } - clientPool.clients[clientIndex] = createClient(address); - return clientPool.clients[clientIndex]; - } - } - - /** Create a completely new {@link TransportClient} to the remote address. */ - private TransportClient createClient(InetSocketAddress address) throws IOException { - logger.debug("Creating new connection to " + address); - - Bootstrap bootstrap = new Bootstrap(); - bootstrap.group(workerGroup) - .channel(socketChannelClass) - // Disable Nagle's Algorithm since we don't want packets to wait - .option(ChannelOption.TCP_NODELAY, true) - .option(ChannelOption.SO_KEEPALIVE, true) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs()) - .option(ChannelOption.ALLOCATOR, pooledAllocator); - - final AtomicReference clientRef = new AtomicReference(); - final AtomicReference channelRef = new AtomicReference(); - - bootstrap.handler(new ChannelInitializer() { - @Override - public void initChannel(SocketChannel ch) { - TransportChannelHandler clientHandler = context.initializePipeline(ch); - clientRef.set(clientHandler.getClient()); - channelRef.set(ch); - } - }); - - // Connect to the remote server - long preConnect = System.nanoTime(); - ChannelFuture cf = bootstrap.connect(address); - if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) { - throw new IOException( - String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs())); - } else if (cf.cause() != null) { - throw new IOException(String.format("Failed to connect to %s", address), cf.cause()); - } - - TransportClient client = clientRef.get(); - Channel channel = channelRef.get(); - assert client != null : "Channel future completed successfully with null client"; - - // Execute any client bootstraps synchronously before marking the Client as successful. - long preBootstrap = System.nanoTime(); - logger.debug("Connection to {} successful, running bootstraps...", address); - try { - for (TransportClientBootstrap clientBootstrap : clientBootstraps) { - clientBootstrap.doBootstrap(client, channel); - } - } catch (Exception e) { // catch non-RuntimeExceptions too as bootstrap may be written in Scala - long bootstrapTimeMs = (System.nanoTime() - preBootstrap) / 1000000; - logger.error("Exception while bootstrapping client after " + bootstrapTimeMs + " ms", e); - client.close(); - throw Throwables.propagate(e); - } - long postBootstrap = System.nanoTime(); - - logger.debug("Successfully created connection to {} after {} ms ({} ms spent in bootstraps)", - address, (postBootstrap - preConnect) / 1000000, (postBootstrap - preBootstrap) / 1000000); - - return client; - } - - /** Close all connections in the connection pool, and shutdown the worker thread pool. */ - @Override - public void close() { - // Go through all clients and close them if they are active. - for (ClientPool clientPool : connectionPool.values()) { - for (int i = 0; i < clientPool.clients.length; i++) { - TransportClient client = clientPool.clients[i]; - if (client != null) { - clientPool.clients[i] = null; - JavaUtils.closeQuietly(client); - } - } - } - connectionPool.clear(); - - if (workerGroup != null) { - workerGroup.shutdownGracefully(); - workerGroup = null; - } - } -} diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java deleted file mode 100644 index ed3f36af5804..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java +++ /dev/null @@ -1,219 +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.network.client; - -import java.io.IOException; -import java.util.Map; -import java.util.Queue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.atomic.AtomicLong; - -import io.netty.channel.Channel; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.protocol.ChunkFetchFailure; -import org.apache.spark.network.protocol.ChunkFetchSuccess; -import org.apache.spark.network.protocol.ResponseMessage; -import org.apache.spark.network.protocol.RpcFailure; -import org.apache.spark.network.protocol.RpcResponse; -import org.apache.spark.network.protocol.StreamChunkId; -import org.apache.spark.network.protocol.StreamFailure; -import org.apache.spark.network.protocol.StreamResponse; -import org.apache.spark.network.server.MessageHandler; -import org.apache.spark.network.util.NettyUtils; -import org.apache.spark.network.util.TransportFrameDecoder; - -/** - * Handler that processes server responses, in response to requests issued from a - * [[TransportClient]]. It works by tracking the list of outstanding requests (and their callbacks). - * - * Concurrency: thread safe and can be called from multiple threads. - */ -public class TransportResponseHandler extends MessageHandler { - private final Logger logger = LoggerFactory.getLogger(TransportResponseHandler.class); - - private final Channel channel; - - private final Map outstandingFetches; - - private final Map outstandingRpcs; - - private final Queue streamCallbacks; - - /** Records the time (in system nanoseconds) that the last fetch or RPC request was sent. */ - private final AtomicLong timeOfLastRequestNs; - - public TransportResponseHandler(Channel channel) { - this.channel = channel; - this.outstandingFetches = new ConcurrentHashMap(); - this.outstandingRpcs = new ConcurrentHashMap(); - this.streamCallbacks = new ConcurrentLinkedQueue(); - this.timeOfLastRequestNs = new AtomicLong(0); - } - - public void addFetchRequest(StreamChunkId streamChunkId, ChunkReceivedCallback callback) { - timeOfLastRequestNs.set(System.nanoTime()); - outstandingFetches.put(streamChunkId, callback); - } - - public void removeFetchRequest(StreamChunkId streamChunkId) { - outstandingFetches.remove(streamChunkId); - } - - public void addRpcRequest(long requestId, RpcResponseCallback callback) { - timeOfLastRequestNs.set(System.nanoTime()); - outstandingRpcs.put(requestId, callback); - } - - public void removeRpcRequest(long requestId) { - outstandingRpcs.remove(requestId); - } - - public void addStreamCallback(StreamCallback callback) { - streamCallbacks.offer(callback); - } - - /** - * Fire the failure callback for all outstanding requests. This is called when we have an - * uncaught exception or pre-mature connection termination. - */ - private void failOutstandingRequests(Throwable cause) { - for (Map.Entry entry : outstandingFetches.entrySet()) { - entry.getValue().onFailure(entry.getKey().chunkIndex, cause); - } - for (Map.Entry entry : outstandingRpcs.entrySet()) { - entry.getValue().onFailure(cause); - } - - // It's OK if new fetches appear, as they will fail immediately. - outstandingFetches.clear(); - outstandingRpcs.clear(); - } - - @Override - public void channelUnregistered() { - if (numOutstandingRequests() > 0) { - String remoteAddress = NettyUtils.getRemoteAddress(channel); - logger.error("Still have {} requests outstanding when connection from {} is closed", - numOutstandingRequests(), remoteAddress); - failOutstandingRequests(new IOException("Connection from " + remoteAddress + " closed")); - } - } - - @Override - public void exceptionCaught(Throwable cause) { - if (numOutstandingRequests() > 0) { - String remoteAddress = NettyUtils.getRemoteAddress(channel); - logger.error("Still have {} requests outstanding when connection from {} is closed", - numOutstandingRequests(), remoteAddress); - failOutstandingRequests(cause); - } - } - - @Override - public void handle(ResponseMessage message) { - String remoteAddress = NettyUtils.getRemoteAddress(channel); - if (message instanceof ChunkFetchSuccess) { - ChunkFetchSuccess resp = (ChunkFetchSuccess) message; - ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId); - if (listener == null) { - logger.warn("Ignoring response for block {} from {} since it is not outstanding", - resp.streamChunkId, remoteAddress); - resp.body.release(); - } else { - outstandingFetches.remove(resp.streamChunkId); - listener.onSuccess(resp.streamChunkId.chunkIndex, resp.body); - resp.body.release(); - } - } else if (message instanceof ChunkFetchFailure) { - ChunkFetchFailure resp = (ChunkFetchFailure) message; - ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId); - if (listener == null) { - logger.warn("Ignoring response for block {} from {} ({}) since it is not outstanding", - resp.streamChunkId, remoteAddress, resp.errorString); - } else { - outstandingFetches.remove(resp.streamChunkId); - listener.onFailure(resp.streamChunkId.chunkIndex, new ChunkFetchFailureException( - "Failure while fetching " + resp.streamChunkId + ": " + resp.errorString)); - } - } else if (message instanceof RpcResponse) { - RpcResponse resp = (RpcResponse) message; - RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); - if (listener == null) { - logger.warn("Ignoring response for RPC {} from {} ({} bytes) since it is not outstanding", - resp.requestId, remoteAddress, resp.response.length); - } else { - outstandingRpcs.remove(resp.requestId); - listener.onSuccess(resp.response); - } - } else if (message instanceof RpcFailure) { - RpcFailure resp = (RpcFailure) message; - RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); - if (listener == null) { - logger.warn("Ignoring response for RPC {} from {} ({}) since it is not outstanding", - resp.requestId, remoteAddress, resp.errorString); - } else { - outstandingRpcs.remove(resp.requestId); - listener.onFailure(new RuntimeException(resp.errorString)); - } - } else if (message instanceof StreamResponse) { - StreamResponse resp = (StreamResponse) message; - StreamCallback callback = streamCallbacks.poll(); - if (callback != null) { - StreamInterceptor interceptor = new StreamInterceptor(resp.streamId, resp.byteCount, - callback); - try { - TransportFrameDecoder frameDecoder = (TransportFrameDecoder) - channel.pipeline().get(TransportFrameDecoder.HANDLER_NAME); - frameDecoder.setInterceptor(interceptor); - } catch (Exception e) { - logger.error("Error installing stream handler.", e); - } - } else { - logger.error("Could not find callback for StreamResponse."); - } - } else if (message instanceof StreamFailure) { - StreamFailure resp = (StreamFailure) message; - StreamCallback callback = streamCallbacks.poll(); - if (callback != null) { - try { - callback.onFailure(resp.streamId, new RuntimeException(resp.error)); - } catch (IOException ioe) { - logger.warn("Error in stream failure handler.", ioe); - } - } else { - logger.warn("Stream failure with unknown callback: {}", resp.error); - } - } else { - throw new IllegalStateException("Unknown response type: " + message.type()); - } - } - - /** Returns total number of outstanding requests (fetch requests + rpcs) */ - public int numOutstandingRequests() { - return outstandingFetches.size() + outstandingRpcs.size(); - } - - /** Returns the time in nanoseconds of when the last request was sent out. */ - public long getTimeOfLastRequestNs() { - return timeOfLastRequestNs.get(); - } - -} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java deleted file mode 100644 index 6cce97c807dc..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.protocol; - -import java.util.List; - -import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelHandler; -import io.netty.channel.ChannelHandlerContext; -import io.netty.handler.codec.MessageToMessageEncoder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Encoder used by the server side to encode server-to-client responses. - * This encoder is stateless so it is safe to be shared by multiple threads. - */ -@ChannelHandler.Sharable -public final class MessageEncoder extends MessageToMessageEncoder { - - private final Logger logger = LoggerFactory.getLogger(MessageEncoder.class); - - /*** - * Encodes a Message by invoking its encode() method. For non-data messages, we will add one - * ByteBuf to 'out' containing the total frame length, the message type, and the message itself. - * In the case of a ChunkFetchSuccess, we will also add the ManagedBuffer corresponding to the - * data to 'out', in order to enable zero-copy transfer. - */ - @Override - public void encode(ChannelHandlerContext ctx, Message in, List out) { - Object body = null; - long bodyLength = 0; - boolean isBodyInFrame = false; - - // Detect ResponseWithBody messages and get the data buffer out of them. - // The body is used in order to enable zero-copy transfer for the payload. - if (in instanceof ResponseWithBody) { - ResponseWithBody resp = (ResponseWithBody) in; - try { - bodyLength = resp.body.size(); - body = resp.body.convertToNetty(); - isBodyInFrame = resp.isBodyInFrame; - } catch (Exception e) { - // Re-encode this message as a failure response. - String error = e.getMessage() != null ? e.getMessage() : "null"; - logger.error(String.format("Error processing %s for client %s", - resp, ctx.channel().remoteAddress()), e); - encode(ctx, resp.createFailureResponse(error), out); - return; - } - } - - Message.Type msgType = in.type(); - // All messages have the frame length, message type, and message itself. The frame length - // may optionally include the length of the body data, depending on what message is being - // sent. - int headerLength = 8 + msgType.encodedLength() + in.encodedLength(); - long frameLength = headerLength + (isBodyInFrame ? bodyLength : 0); - ByteBuf header = ctx.alloc().heapBuffer(headerLength); - header.writeLong(frameLength); - msgType.encode(header); - in.encode(header); - assert header.writableBytes() == 0; - - if (body != null && bodyLength > 0) { - out.add(new MessageWithHeader(header, body, bodyLength)); - } else { - out.add(header); - } - } - -} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java deleted file mode 100644 index d686a951467c..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java +++ /dev/null @@ -1,109 +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.network.protocol; - -import java.io.IOException; -import java.nio.channels.WritableByteChannel; - -import com.google.common.base.Preconditions; -import io.netty.buffer.ByteBuf; -import io.netty.channel.FileRegion; -import io.netty.util.AbstractReferenceCounted; -import io.netty.util.ReferenceCountUtil; - -/** - * A wrapper message that holds two separate pieces (a header and a body). - * - * The header must be a ByteBuf, while the body can be a ByteBuf or a FileRegion. - */ -class MessageWithHeader extends AbstractReferenceCounted implements FileRegion { - - private final ByteBuf header; - private final int headerLength; - private final Object body; - private final long bodyLength; - private long totalBytesTransferred; - - MessageWithHeader(ByteBuf header, Object body, long bodyLength) { - Preconditions.checkArgument(body instanceof ByteBuf || body instanceof FileRegion, - "Body must be a ByteBuf or a FileRegion."); - this.header = header; - this.headerLength = header.readableBytes(); - this.body = body; - this.bodyLength = bodyLength; - } - - @Override - public long count() { - return headerLength + bodyLength; - } - - @Override - public long position() { - return 0; - } - - @Override - public long transfered() { - return totalBytesTransferred; - } - - /** - * This code is more complicated than you would think because we might require multiple - * transferTo invocations in order to transfer a single MessageWithHeader to avoid busy waiting. - * - * The contract is that the caller will ensure position is properly set to the total number - * of bytes transferred so far (i.e. value returned by transfered()). - */ - @Override - public long transferTo(final WritableByteChannel target, final long position) throws IOException { - Preconditions.checkArgument(position == totalBytesTransferred, "Invalid position."); - // Bytes written for header in this call. - long writtenHeader = 0; - if (header.readableBytes() > 0) { - writtenHeader = copyByteBuf(header, target); - totalBytesTransferred += writtenHeader; - if (header.readableBytes() > 0) { - return writtenHeader; - } - } - - // Bytes written for body in this call. - long writtenBody = 0; - if (body instanceof FileRegion) { - writtenBody = ((FileRegion) body).transferTo(target, totalBytesTransferred - headerLength); - } else if (body instanceof ByteBuf) { - writtenBody = copyByteBuf((ByteBuf) body, target); - } - totalBytesTransferred += writtenBody; - - return writtenHeader + writtenBody; - } - - @Override - protected void deallocate() { - header.release(); - ReferenceCountUtil.release(body); - } - - private int copyByteBuf(ByteBuf buf, WritableByteChannel target) throws IOException { - int written = target.write(buf.nioBuffer()); - buf.skipBytes(written); - return written; - } -} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ResponseWithBody.java b/network/common/src/main/java/org/apache/spark/network/protocol/ResponseWithBody.java deleted file mode 100644 index 67be77e39f71..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/protocol/ResponseWithBody.java +++ /dev/null @@ -1,40 +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.network.protocol; - -import com.google.common.base.Objects; -import io.netty.buffer.ByteBuf; - -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.buffer.NettyManagedBuffer; - -/** - * Abstract class for response messages that contain a large data portion kept in a separate - * buffer. These messages are treated especially by MessageEncoder. - */ -public abstract class ResponseWithBody implements ResponseMessage { - public final ManagedBuffer body; - public final boolean isBodyInFrame; - - protected ResponseWithBody(ManagedBuffer body, boolean isBodyInFrame) { - this.body = body; - this.isBodyInFrame = isBodyInFrame; - } - - public abstract ResponseMessage createFailureResponse(String error); -} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java deleted file mode 100644 index 745039db742f..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java +++ /dev/null @@ -1,83 +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.network.protocol; - -import java.util.Arrays; - -import com.google.common.base.Objects; -import io.netty.buffer.ByteBuf; - -/** - * A generic RPC which is handled by a remote {@link org.apache.spark.network.server.RpcHandler}. - * This will correspond to a single - * {@link org.apache.spark.network.protocol.ResponseMessage} (either success or failure). - */ -public final class RpcRequest implements RequestMessage { - /** Used to link an RPC request with its response. */ - public final long requestId; - - /** Serialized message to send to remote RpcHandler. */ - public final byte[] message; - - public RpcRequest(long requestId, byte[] message) { - this.requestId = requestId; - this.message = message; - } - - @Override - public Type type() { return Type.RpcRequest; } - - @Override - public int encodedLength() { - return 8 + Encoders.ByteArrays.encodedLength(message); - } - - @Override - public void encode(ByteBuf buf) { - buf.writeLong(requestId); - Encoders.ByteArrays.encode(buf, message); - } - - public static RpcRequest decode(ByteBuf buf) { - long requestId = buf.readLong(); - byte[] message = Encoders.ByteArrays.decode(buf); - return new RpcRequest(requestId, message); - } - - @Override - public int hashCode() { - return Objects.hashCode(requestId, Arrays.hashCode(message)); - } - - @Override - public boolean equals(Object other) { - if (other instanceof RpcRequest) { - RpcRequest o = (RpcRequest) other; - return requestId == o.requestId && Arrays.equals(message, o.message); - } - return false; - } - - @Override - public String toString() { - return Objects.toStringHelper(this) - .add("requestId", requestId) - .add("message", message) - .toString(); - } -} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java deleted file mode 100644 index 1671cd444f03..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java +++ /dev/null @@ -1,74 +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.network.protocol; - -import java.util.Arrays; - -import com.google.common.base.Objects; -import io.netty.buffer.ByteBuf; - -/** Response to {@link RpcRequest} for a successful RPC. */ -public final class RpcResponse implements ResponseMessage { - public final long requestId; - public final byte[] response; - - public RpcResponse(long requestId, byte[] response) { - this.requestId = requestId; - this.response = response; - } - - @Override - public Type type() { return Type.RpcResponse; } - - @Override - public int encodedLength() { return 8 + Encoders.ByteArrays.encodedLength(response); } - - @Override - public void encode(ByteBuf buf) { - buf.writeLong(requestId); - Encoders.ByteArrays.encode(buf, response); - } - - public static RpcResponse decode(ByteBuf buf) { - long requestId = buf.readLong(); - byte[] response = Encoders.ByteArrays.decode(buf); - return new RpcResponse(requestId, response); - } - - @Override - public int hashCode() { - return Objects.hashCode(requestId, Arrays.hashCode(response)); - } - - @Override - public boolean equals(Object other) { - if (other instanceof RpcResponse) { - RpcResponse o = (RpcResponse) other; - return requestId == o.requestId && Arrays.equals(response, o.response); - } - return false; - } - - @Override - public String toString() { - return Objects.toStringHelper(this) - .add("requestId", requestId) - .add("response", response) - .toString(); - } -} diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java b/network/common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java deleted file mode 100644 index cad76ab7aa54..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java +++ /dev/null @@ -1,65 +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.network.sasl; - -import io.netty.buffer.ByteBuf; - -import org.apache.spark.network.protocol.Encodable; -import org.apache.spark.network.protocol.Encoders; - -/** - * Encodes a Sasl-related message which is attempting to authenticate using some credentials tagged - * with the given appId. This appId allows a single SaslRpcHandler to multiplex different - * applications which may be using different sets of credentials. - */ -class SaslMessage implements Encodable { - - /** Serialization tag used to catch incorrect payloads. */ - private static final byte TAG_BYTE = (byte) 0xEA; - - public final String appId; - public final byte[] payload; - - public SaslMessage(String appId, byte[] payload) { - this.appId = appId; - this.payload = payload; - } - - @Override - public int encodedLength() { - return 1 + Encoders.Strings.encodedLength(appId) + Encoders.ByteArrays.encodedLength(payload); - } - - @Override - public void encode(ByteBuf buf) { - buf.writeByte(TAG_BYTE); - Encoders.Strings.encode(buf, appId); - Encoders.ByteArrays.encode(buf, payload); - } - - public static SaslMessage decode(ByteBuf buf) { - if (buf.readByte() != TAG_BYTE) { - throw new IllegalStateException("Expected SaslMessage, received something else" - + " (maybe your client does not have SASL enabled?)"); - } - - String appId = Encoders.Strings.decode(buf); - byte[] payload = Encoders.ByteArrays.decode(buf); - return new SaslMessage(appId, payload); - } -} diff --git a/network/common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java b/network/common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java deleted file mode 100644 index 7033adb9cae6..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java +++ /dev/null @@ -1,132 +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.network.sasl; - -import javax.security.sasl.Sasl; - -import io.netty.buffer.Unpooled; -import io.netty.channel.Channel; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.client.RpcResponseCallback; -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.server.RpcHandler; -import org.apache.spark.network.server.StreamManager; -import org.apache.spark.network.util.TransportConf; - -/** - * RPC Handler which performs SASL authentication before delegating to a child RPC handler. - * The delegate will only receive messages if the given connection has been successfully - * authenticated. A connection may be authenticated at most once. - * - * Note that the authentication process consists of multiple challenge-response pairs, each of - * which are individual RPCs. - */ -class SaslRpcHandler extends RpcHandler { - private static final Logger logger = LoggerFactory.getLogger(SaslRpcHandler.class); - - /** Transport configuration. */ - private final TransportConf conf; - - /** The client channel. */ - private final Channel channel; - - /** RpcHandler we will delegate to for authenticated connections. */ - private final RpcHandler delegate; - - /** Class which provides secret keys which are shared by server and client on a per-app basis. */ - private final SecretKeyHolder secretKeyHolder; - - private SparkSaslServer saslServer; - private boolean isComplete; - - SaslRpcHandler( - TransportConf conf, - Channel channel, - RpcHandler delegate, - SecretKeyHolder secretKeyHolder) { - this.conf = conf; - this.channel = channel; - this.delegate = delegate; - this.secretKeyHolder = secretKeyHolder; - this.saslServer = null; - this.isComplete = false; - } - - @Override - public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { - if (isComplete) { - // Authentication complete, delegate to base handler. - delegate.receive(client, message, callback); - return; - } - - SaslMessage saslMessage = SaslMessage.decode(Unpooled.wrappedBuffer(message)); - - if (saslServer == null) { - // First message in the handshake, setup the necessary state. - client.setClientId(saslMessage.appId); - saslServer = new SparkSaslServer(saslMessage.appId, secretKeyHolder, - conf.saslServerAlwaysEncrypt()); - } - - byte[] response = saslServer.response(saslMessage.payload); - callback.onSuccess(response); - - // Setup encryption after the SASL response is sent, otherwise the client can't parse the - // response. It's ok to change the channel pipeline here since we are processing an incoming - // message, so the pipeline is busy and no new incoming messages will be fed to it before this - // method returns. This assumes that the code ensures, through other means, that no outbound - // messages are being written to the channel while negotiation is still going on. - if (saslServer.isComplete()) { - logger.debug("SASL authentication successful for channel {}", client); - isComplete = true; - if (SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP))) { - logger.debug("Enabling encryption for channel {}", client); - SaslEncryption.addToChannel(channel, saslServer, conf.maxSaslEncryptedBlockSize()); - saslServer = null; - } else { - saslServer.dispose(); - saslServer = null; - } - } - } - - @Override - public StreamManager getStreamManager() { - return delegate.getStreamManager(); - } - - @Override - public void connectionTerminated(TransportClient client) { - try { - delegate.connectionTerminated(client); - } finally { - if (saslServer != null) { - saslServer.dispose(); - } - } - } - - @Override - public void exceptionCaught(Throwable cause, TransportClient client) { - delegate.exceptionCaught(cause, client); - } - -} diff --git a/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java b/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java deleted file mode 100644 index e671854da1ca..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java +++ /dev/null @@ -1,144 +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.network.server; - -import java.util.Iterator; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicLong; - -import com.google.common.base.Preconditions; -import io.netty.channel.Channel; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.client.TransportClient; - -/** - * StreamManager which allows registration of an Iterator<ManagedBuffer>, which are individually - * fetched as chunks by the client. Each registered buffer is one chunk. - */ -public class OneForOneStreamManager extends StreamManager { - private final Logger logger = LoggerFactory.getLogger(OneForOneStreamManager.class); - - private final AtomicLong nextStreamId; - private final ConcurrentHashMap streams; - - /** State of a single stream. */ - private static class StreamState { - final String appId; - final Iterator buffers; - - // The channel associated to the stream - Channel associatedChannel = null; - - // Used to keep track of the index of the buffer that the user has retrieved, just to ensure - // that the caller only requests each chunk one at a time, in order. - int curChunk = 0; - - StreamState(String appId, Iterator buffers) { - this.appId = appId; - this.buffers = Preconditions.checkNotNull(buffers); - } - } - - public OneForOneStreamManager() { - // For debugging purposes, start with a random stream id to help identifying different streams. - // This does not need to be globally unique, only unique to this class. - nextStreamId = new AtomicLong((long) new Random().nextInt(Integer.MAX_VALUE) * 1000); - streams = new ConcurrentHashMap(); - } - - @Override - public void registerChannel(Channel channel, long streamId) { - if (streams.containsKey(streamId)) { - streams.get(streamId).associatedChannel = channel; - } - } - - @Override - public ManagedBuffer getChunk(long streamId, int chunkIndex) { - StreamState state = streams.get(streamId); - if (chunkIndex != state.curChunk) { - throw new IllegalStateException(String.format( - "Received out-of-order chunk index %s (expected %s)", chunkIndex, state.curChunk)); - } else if (!state.buffers.hasNext()) { - throw new IllegalStateException(String.format( - "Requested chunk index beyond end %s", chunkIndex)); - } - state.curChunk += 1; - ManagedBuffer nextChunk = state.buffers.next(); - - if (!state.buffers.hasNext()) { - logger.trace("Removing stream id {}", streamId); - streams.remove(streamId); - } - - return nextChunk; - } - - @Override - public void connectionTerminated(Channel channel) { - // Close all streams which have been associated with the channel. - for (Map.Entry entry: streams.entrySet()) { - StreamState state = entry.getValue(); - if (state.associatedChannel == channel) { - streams.remove(entry.getKey()); - - // Release all remaining buffers. - while (state.buffers.hasNext()) { - state.buffers.next().release(); - } - } - } - } - - @Override - public void checkAuthorization(TransportClient client, long streamId) { - if (client.getClientId() != null) { - StreamState state = streams.get(streamId); - Preconditions.checkArgument(state != null, "Unknown stream ID."); - if (!client.getClientId().equals(state.appId)) { - throw new SecurityException(String.format( - "Client %s not authorized to read stream %d (app %s).", - client.getClientId(), - streamId, - state.appId)); - } - } - } - - /** - * Registers a stream of ManagedBuffers which are served as individual chunks one at a time to - * callers. Each ManagedBuffer will be release()'d after it is transferred on the wire. If a - * client connection is closed before the iterator is fully drained, then the remaining buffers - * will all be release()'d. - * - * If an app ID is provided, only callers who've authenticated with the given app ID will be - * allowed to fetch from this stream. - */ - public long registerStream(String appId, Iterator buffers) { - long myStreamId = nextStreamId.getAndIncrement(); - streams.put(myStreamId, new StreamState(appId, buffers)); - return myStreamId; - } - -} diff --git a/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java b/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java deleted file mode 100644 index dbb7f95f55bc..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.server; - -import org.apache.spark.network.client.RpcResponseCallback; -import org.apache.spark.network.client.TransportClient; - -/** - * Handler for sendRPC() messages sent by {@link org.apache.spark.network.client.TransportClient}s. - */ -public abstract class RpcHandler { - /** - * Receive a single RPC message. Any exception thrown while in this method will be sent back to - * the client in string form as a standard RPC failure. - * - * This method will not be called in parallel for a single TransportClient (i.e., channel). - * - * @param client A channel client which enables the handler to make requests back to the sender - * of this RPC. This will always be the exact same object for a particular channel. - * @param message The serialized bytes of the RPC. - * @param callback Callback which should be invoked exactly once upon success or failure of the - * RPC. - */ - public abstract void receive( - TransportClient client, - byte[] message, - RpcResponseCallback callback); - - /** - * Returns the StreamManager which contains the state about which streams are currently being - * fetched by a TransportClient. - */ - public abstract StreamManager getStreamManager(); - - /** - * Invoked when the connection associated with the given client has been invalidated. - * No further requests will come from this client. - */ - public void connectionTerminated(TransportClient client) { } - - public void exceptionCaught(Throwable cause, TransportClient client) { } -} diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java b/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java deleted file mode 100644 index 8e0ee709e38e..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java +++ /dev/null @@ -1,127 +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.network.server; - -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.SimpleChannelInboundHandler; -import io.netty.handler.timeout.IdleState; -import io.netty.handler.timeout.IdleStateEvent; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.client.TransportResponseHandler; -import org.apache.spark.network.protocol.Message; -import org.apache.spark.network.protocol.RequestMessage; -import org.apache.spark.network.protocol.ResponseMessage; -import org.apache.spark.network.util.NettyUtils; - -/** - * The single Transport-level Channel handler which is used for delegating requests to the - * {@link TransportRequestHandler} and responses to the {@link TransportResponseHandler}. - * - * All channels created in the transport layer are bidirectional. When the Client initiates a Netty - * Channel with a RequestMessage (which gets handled by the Server's RequestHandler), the Server - * will produce a ResponseMessage (handled by the Client's ResponseHandler). However, the Server - * also gets a handle on the same Channel, so it may then begin to send RequestMessages to the - * Client. - * This means that the Client also needs a RequestHandler and the Server needs a ResponseHandler, - * for the Client's responses to the Server's requests. - * - * This class also handles timeouts from a {@link io.netty.handler.timeout.IdleStateHandler}. - * We consider a connection timed out if there are outstanding fetch or RPC requests but no traffic - * on the channel for at least `requestTimeoutMs`. Note that this is duplex traffic; we will not - * timeout if the client is continuously sending but getting no responses, for simplicity. - */ -public class TransportChannelHandler extends SimpleChannelInboundHandler { - private final Logger logger = LoggerFactory.getLogger(TransportChannelHandler.class); - - private final TransportClient client; - private final TransportResponseHandler responseHandler; - private final TransportRequestHandler requestHandler; - private final long requestTimeoutNs; - - public TransportChannelHandler( - TransportClient client, - TransportResponseHandler responseHandler, - TransportRequestHandler requestHandler, - long requestTimeoutMs) { - this.client = client; - this.responseHandler = responseHandler; - this.requestHandler = requestHandler; - this.requestTimeoutNs = requestTimeoutMs * 1000L * 1000; - } - - public TransportClient getClient() { - return client; - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - logger.warn("Exception in connection from " + NettyUtils.getRemoteAddress(ctx.channel()), - cause); - requestHandler.exceptionCaught(cause); - responseHandler.exceptionCaught(cause); - ctx.close(); - } - - @Override - public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { - try { - requestHandler.channelUnregistered(); - } catch (RuntimeException e) { - logger.error("Exception from request handler while unregistering channel", e); - } - try { - responseHandler.channelUnregistered(); - } catch (RuntimeException e) { - logger.error("Exception from response handler while unregistering channel", e); - } - super.channelUnregistered(ctx); - } - - @Override - public void channelRead0(ChannelHandlerContext ctx, Message request) { - if (request instanceof RequestMessage) { - requestHandler.handle((RequestMessage) request); - } else { - responseHandler.handle((ResponseMessage) request); - } - } - - /** Triggered based on events from an {@link io.netty.handler.timeout.IdleStateHandler}. */ - @Override - public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { - if (evt instanceof IdleStateEvent) { - IdleStateEvent e = (IdleStateEvent) evt; - // See class comment for timeout semantics. In addition to ensuring we only timeout while - // there are outstanding requests, we also do a secondary consistency check to ensure - // there's no race between the idle timeout and incrementing the numOutstandingRequests. - boolean hasInFlightRequests = responseHandler.numOutstandingRequests() > 0; - boolean isActuallyOverdue = - System.nanoTime() - responseHandler.getTimeOfLastRequestNs() > requestTimeoutNs; - if (e.state() == IdleState.ALL_IDLE && hasInFlightRequests && isActuallyOverdue) { - String address = NettyUtils.getRemoteAddress(ctx.channel()); - logger.error("Connection to {} has been quiet for {} ms while there are outstanding " + - "requests. Assuming connection is dead; please adjust spark.network.timeout if this " + - "is wrong.", address, requestTimeoutNs / 1000 / 1000); - ctx.close(); - } - } - } -} diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java deleted file mode 100644 index 4f67bd573be2..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ /dev/null @@ -1,180 +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.network.server; - -import com.google.common.base.Throwables; -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.client.RpcResponseCallback; -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.protocol.Encodable; -import org.apache.spark.network.protocol.RequestMessage; -import org.apache.spark.network.protocol.ChunkFetchRequest; -import org.apache.spark.network.protocol.RpcRequest; -import org.apache.spark.network.protocol.ChunkFetchFailure; -import org.apache.spark.network.protocol.ChunkFetchSuccess; -import org.apache.spark.network.protocol.RpcFailure; -import org.apache.spark.network.protocol.RpcResponse; -import org.apache.spark.network.protocol.StreamFailure; -import org.apache.spark.network.protocol.StreamRequest; -import org.apache.spark.network.protocol.StreamResponse; -import org.apache.spark.network.util.NettyUtils; - -/** - * A handler that processes requests from clients and writes chunk data back. Each handler is - * attached to a single Netty channel, and keeps track of which streams have been fetched via this - * channel, in order to clean them up if the channel is terminated (see #channelUnregistered). - * - * The messages should have been processed by the pipeline setup by {@link TransportServer}. - */ -public class TransportRequestHandler extends MessageHandler { - private final Logger logger = LoggerFactory.getLogger(TransportRequestHandler.class); - - /** The Netty channel that this handler is associated with. */ - private final Channel channel; - - /** Client on the same channel allowing us to talk back to the requester. */ - private final TransportClient reverseClient; - - /** Handles all RPC messages. */ - private final RpcHandler rpcHandler; - - /** Returns each chunk part of a stream. */ - private final StreamManager streamManager; - - public TransportRequestHandler( - Channel channel, - TransportClient reverseClient, - RpcHandler rpcHandler) { - this.channel = channel; - this.reverseClient = reverseClient; - this.rpcHandler = rpcHandler; - this.streamManager = rpcHandler.getStreamManager(); - } - - @Override - public void exceptionCaught(Throwable cause) { - rpcHandler.exceptionCaught(cause, reverseClient); - } - - @Override - public void channelUnregistered() { - if (streamManager != null) { - try { - streamManager.connectionTerminated(channel); - } catch (RuntimeException e) { - logger.error("StreamManager connectionTerminated() callback failed.", e); - } - } - rpcHandler.connectionTerminated(reverseClient); - } - - @Override - public void handle(RequestMessage request) { - if (request instanceof ChunkFetchRequest) { - processFetchRequest((ChunkFetchRequest) request); - } else if (request instanceof RpcRequest) { - processRpcRequest((RpcRequest) request); - } else if (request instanceof StreamRequest) { - processStreamRequest((StreamRequest) request); - } else { - throw new IllegalArgumentException("Unknown request type: " + request); - } - } - - private void processFetchRequest(final ChunkFetchRequest req) { - final String client = NettyUtils.getRemoteAddress(channel); - - logger.trace("Received req from {} to fetch block {}", client, req.streamChunkId); - - ManagedBuffer buf; - try { - streamManager.checkAuthorization(reverseClient, req.streamChunkId.streamId); - streamManager.registerChannel(channel, req.streamChunkId.streamId); - buf = streamManager.getChunk(req.streamChunkId.streamId, req.streamChunkId.chunkIndex); - } catch (Exception e) { - logger.error(String.format( - "Error opening block %s for request from %s", req.streamChunkId, client), e); - respond(new ChunkFetchFailure(req.streamChunkId, Throwables.getStackTraceAsString(e))); - return; - } - - respond(new ChunkFetchSuccess(req.streamChunkId, buf)); - } - - private void processStreamRequest(final StreamRequest req) { - final String client = NettyUtils.getRemoteAddress(channel); - ManagedBuffer buf; - try { - buf = streamManager.openStream(req.streamId); - } catch (Exception e) { - logger.error(String.format( - "Error opening stream %s for request from %s", req.streamId, client), e); - respond(new StreamFailure(req.streamId, Throwables.getStackTraceAsString(e))); - return; - } - - respond(new StreamResponse(req.streamId, buf.size(), buf)); - } - - private void processRpcRequest(final RpcRequest req) { - try { - rpcHandler.receive(reverseClient, req.message, new RpcResponseCallback() { - @Override - public void onSuccess(byte[] response) { - respond(new RpcResponse(req.requestId, response)); - } - - @Override - public void onFailure(Throwable e) { - respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); - } - }); - } catch (Exception e) { - logger.error("Error while invoking RpcHandler#receive() on RPC id " + req.requestId, e); - respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); - } - } - - /** - * Responds to a single message with some Encodable object. If a failure occurs while sending, - * it will be logged and the channel closed. - */ - private void respond(final Encodable result) { - final String remoteAddress = channel.remoteAddress().toString(); - channel.writeAndFlush(result).addListener( - new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) throws Exception { - if (future.isSuccess()) { - logger.trace(String.format("Sent result %s to client %s", result, remoteAddress)); - } else { - logger.error(String.format("Error sending result %s to %s; closing connection", - result, remoteAddress), future.cause()); - channel.close(); - } - } - } - ); - } -} diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java deleted file mode 100644 index 7d27439cfde7..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ /dev/null @@ -1,287 +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.network.util; - -import java.io.Closeable; -import java.io.File; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.concurrent.TimeUnit; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import com.google.common.base.Charsets; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; -import io.netty.buffer.Unpooled; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * General utilities available in the network package. Many of these are sourced from Spark's - * own Utils, just accessible within this package. - */ -public class JavaUtils { - private static final Logger logger = LoggerFactory.getLogger(JavaUtils.class); - - /** - * Define a default value for driver memory here since this value is referenced across the code - * base and nearly all files already use Utils.scala - */ - public static final long DEFAULT_DRIVER_MEM_MB = 1024; - - /** Closes the given object, ignoring IOExceptions. */ - public static void closeQuietly(Closeable closeable) { - try { - if (closeable != null) { - closeable.close(); - } - } catch (IOException e) { - logger.error("IOException should not have been thrown.", e); - } - } - - /** Returns a hash consistent with Spark's Utils.nonNegativeHash(). */ - public static int nonNegativeHash(Object obj) { - if (obj == null) { return 0; } - int hash = obj.hashCode(); - return hash != Integer.MIN_VALUE ? Math.abs(hash) : 0; - } - - /** - * Convert the given string to a byte buffer. The resulting buffer can be - * converted back to the same string through {@link #bytesToString(ByteBuffer)}. - */ - public static ByteBuffer stringToBytes(String s) { - return Unpooled.wrappedBuffer(s.getBytes(Charsets.UTF_8)).nioBuffer(); - } - - /** - * Convert the given byte buffer to a string. The resulting string can be - * converted back to the same byte buffer through {@link #stringToBytes(String)}. - */ - public static String bytesToString(ByteBuffer b) { - return Unpooled.wrappedBuffer(b).toString(Charsets.UTF_8); - } - - /* - * Delete a file or directory and its contents recursively. - * Don't follow directories if they are symlinks. - * Throws an exception if deletion is unsuccessful. - */ - public static void deleteRecursively(File file) throws IOException { - if (file == null) { return; } - - if (file.isDirectory() && !isSymlink(file)) { - IOException savedIOException = null; - for (File child : listFilesSafely(file)) { - try { - deleteRecursively(child); - } catch (IOException e) { - // In case of multiple exceptions, only last one will be thrown - savedIOException = e; - } - } - if (savedIOException != null) { - throw savedIOException; - } - } - - boolean deleted = file.delete(); - // Delete can also fail if the file simply did not exist. - if (!deleted && file.exists()) { - throw new IOException("Failed to delete: " + file.getAbsolutePath()); - } - } - - private static File[] listFilesSafely(File file) throws IOException { - if (file.exists()) { - File[] files = file.listFiles(); - if (files == null) { - throw new IOException("Failed to list files for dir: " + file); - } - return files; - } else { - return new File[0]; - } - } - - private static boolean isSymlink(File file) throws IOException { - Preconditions.checkNotNull(file); - File fileInCanonicalDir = null; - if (file.getParent() == null) { - fileInCanonicalDir = file; - } else { - fileInCanonicalDir = new File(file.getParentFile().getCanonicalFile(), file.getName()); - } - return !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()); - } - - private static final ImmutableMap timeSuffixes = - ImmutableMap.builder() - .put("us", TimeUnit.MICROSECONDS) - .put("ms", TimeUnit.MILLISECONDS) - .put("s", TimeUnit.SECONDS) - .put("m", TimeUnit.MINUTES) - .put("min", TimeUnit.MINUTES) - .put("h", TimeUnit.HOURS) - .put("d", TimeUnit.DAYS) - .build(); - - private static final ImmutableMap byteSuffixes = - ImmutableMap.builder() - .put("b", ByteUnit.BYTE) - .put("k", ByteUnit.KiB) - .put("kb", ByteUnit.KiB) - .put("m", ByteUnit.MiB) - .put("mb", ByteUnit.MiB) - .put("g", ByteUnit.GiB) - .put("gb", ByteUnit.GiB) - .put("t", ByteUnit.TiB) - .put("tb", ByteUnit.TiB) - .put("p", ByteUnit.PiB) - .put("pb", ByteUnit.PiB) - .build(); - - /** - * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a time count for - * internal use. If no suffix is provided a direct conversion is attempted. - */ - private static long parseTimeString(String str, TimeUnit unit) { - String lower = str.toLowerCase().trim(); - - try { - Matcher m = Pattern.compile("(-?[0-9]+)([a-z]+)?").matcher(lower); - if (!m.matches()) { - throw new NumberFormatException("Failed to parse time string: " + str); - } - - long val = Long.parseLong(m.group(1)); - String suffix = m.group(2); - - // Check for invalid suffixes - if (suffix != null && !timeSuffixes.containsKey(suffix)) { - throw new NumberFormatException("Invalid suffix: \"" + suffix + "\""); - } - - // If suffix is valid use that, otherwise none was provided and use the default passed - return unit.convert(val, suffix != null ? timeSuffixes.get(suffix) : unit); - } catch (NumberFormatException e) { - String timeError = "Time must be specified as seconds (s), " + - "milliseconds (ms), microseconds (us), minutes (m or min), hour (h), or day (d). " + - "E.g. 50s, 100ms, or 250us."; - - throw new NumberFormatException(timeError + "\n" + e.getMessage()); - } - } - - /** - * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. If - * no suffix is provided, the passed number is assumed to be in ms. - */ - public static long timeStringAsMs(String str) { - return parseTimeString(str, TimeUnit.MILLISECONDS); - } - - /** - * Convert a time parameter such as (50s, 100ms, or 250us) to seconds for internal use. If - * no suffix is provided, the passed number is assumed to be in seconds. - */ - public static long timeStringAsSec(String str) { - return parseTimeString(str, TimeUnit.SECONDS); - } - - /** - * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to a ByteUnit for - * internal use. If no suffix is provided a direct conversion of the provided default is - * attempted. - */ - private static long parseByteString(String str, ByteUnit unit) { - String lower = str.toLowerCase().trim(); - - try { - Matcher m = Pattern.compile("([0-9]+)([a-z]+)?").matcher(lower); - Matcher fractionMatcher = Pattern.compile("([0-9]+\\.[0-9]+)([a-z]+)?").matcher(lower); - - if (m.matches()) { - long val = Long.parseLong(m.group(1)); - String suffix = m.group(2); - - // Check for invalid suffixes - if (suffix != null && !byteSuffixes.containsKey(suffix)) { - throw new NumberFormatException("Invalid suffix: \"" + suffix + "\""); - } - - // If suffix is valid use that, otherwise none was provided and use the default passed - return unit.convertFrom(val, suffix != null ? byteSuffixes.get(suffix) : unit); - } else if (fractionMatcher.matches()) { - throw new NumberFormatException("Fractional values are not supported. Input was: " - + fractionMatcher.group(1)); - } else { - throw new NumberFormatException("Failed to parse byte string: " + str); - } - - } catch (NumberFormatException e) { - String timeError = "Size must be specified as bytes (b), " + - "kibibytes (k), mebibytes (m), gibibytes (g), tebibytes (t), or pebibytes(p). " + - "E.g. 50b, 100k, or 250m."; - - throw new NumberFormatException(timeError + "\n" + e.getMessage()); - } - } - - /** - * Convert a passed byte string (e.g. 50b, 100k, or 250m) to bytes for - * internal use. - * - * If no suffix is provided, the passed number is assumed to be in bytes. - */ - public static long byteStringAsBytes(String str) { - return parseByteString(str, ByteUnit.BYTE); - } - - /** - * Convert a passed byte string (e.g. 50b, 100k, or 250m) to kibibytes for - * internal use. - * - * If no suffix is provided, the passed number is assumed to be in kibibytes. - */ - public static long byteStringAsKb(String str) { - return parseByteString(str, ByteUnit.KiB); - } - - /** - * Convert a passed byte string (e.g. 50b, 100k, or 250m) to mebibytes for - * internal use. - * - * If no suffix is provided, the passed number is assumed to be in mebibytes. - */ - public static long byteStringAsMb(String str) { - return parseByteString(str, ByteUnit.MiB); - } - - /** - * Convert a passed byte string (e.g. 50b, 100k, or 250m) to gibibytes for - * internal use. - * - * If no suffix is provided, the passed number is assumed to be in gibibytes. - */ - public static long byteStringAsGb(String str) { - return parseByteString(str, ByteUnit.GiB); - } -} diff --git a/network/common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java b/network/common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java deleted file mode 100644 index 668d2356b955..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java +++ /dev/null @@ -1,41 +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.network.util; - -import com.google.common.collect.Maps; - -import java.util.Map; -import java.util.NoSuchElementException; - -/** ConfigProvider based on a Map (copied in the constructor). */ -public class MapConfigProvider extends ConfigProvider { - private final Map config; - - public MapConfigProvider(Map config) { - this.config = Maps.newHashMap(config); - } - - @Override - public String get(String name) { - String value = config.get(name); - if (value == null) { - throw new NoSuchElementException(name); - } - return value; - } -} diff --git a/network/common/src/main/java/org/apache/spark/network/util/SystemPropertyConfigProvider.java b/network/common/src/main/java/org/apache/spark/network/util/SystemPropertyConfigProvider.java deleted file mode 100644 index 5f20b70678d1..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/util/SystemPropertyConfigProvider.java +++ /dev/null @@ -1,34 +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.network.util; - -import java.util.NoSuchElementException; - -import org.apache.spark.network.util.ConfigProvider; - -/** Uses System properties to obtain config values. */ -public class SystemPropertyConfigProvider extends ConfigProvider { - @Override - public String get(String name) { - String value = System.getProperty(name); - if (value == null) { - throw new NoSuchElementException(name); - } - return value; - } -} diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java deleted file mode 100644 index 3b2eff377955..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ /dev/null @@ -1,133 +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.network.util; - -import com.google.common.primitives.Ints; - -/** - * A central location that tracks all the settings we expose to users. - */ -public class TransportConf { - private final ConfigProvider conf; - - public TransportConf(ConfigProvider conf) { - this.conf = conf; - } - - /** IO mode: nio or epoll */ - public String ioMode() { return conf.get("spark.shuffle.io.mode", "NIO").toUpperCase(); } - - /** If true, we will prefer allocating off-heap byte buffers within Netty. */ - public boolean preferDirectBufs() { - return conf.getBoolean("spark.shuffle.io.preferDirectBufs", true); - } - - /** Connect timeout in milliseconds. Default 120 secs. */ - public int connectionTimeoutMs() { - long defaultNetworkTimeoutS = JavaUtils.timeStringAsSec( - conf.get("spark.network.timeout", "120s")); - long defaultTimeoutMs = JavaUtils.timeStringAsSec( - conf.get("spark.shuffle.io.connectionTimeout", defaultNetworkTimeoutS + "s")) * 1000; - return (int) defaultTimeoutMs; - } - - /** Number of concurrent connections between two nodes for fetching data. */ - public int numConnectionsPerPeer() { - return conf.getInt("spark.shuffle.io.numConnectionsPerPeer", 1); - } - - /** Requested maximum length of the queue of incoming connections. Default -1 for no backlog. */ - public int backLog() { return conf.getInt("spark.shuffle.io.backLog", -1); } - - /** Number of threads used in the server thread pool. Default to 0, which is 2x#cores. */ - public int serverThreads() { return conf.getInt("spark.shuffle.io.serverThreads", 0); } - - /** Number of threads used in the client thread pool. Default to 0, which is 2x#cores. */ - public int clientThreads() { return conf.getInt("spark.shuffle.io.clientThreads", 0); } - - /** - * Receive buffer size (SO_RCVBUF). - * Note: the optimal size for receive buffer and send buffer should be - * latency * network_bandwidth. - * Assuming latency = 1ms, network_bandwidth = 10Gbps - * buffer size should be ~ 1.25MB - */ - public int receiveBuf() { return conf.getInt("spark.shuffle.io.receiveBuffer", -1); } - - /** Send buffer size (SO_SNDBUF). */ - public int sendBuf() { return conf.getInt("spark.shuffle.io.sendBuffer", -1); } - - /** Timeout for a single round trip of SASL token exchange, in milliseconds. */ - public int saslRTTimeoutMs() { - return (int) JavaUtils.timeStringAsSec(conf.get("spark.shuffle.sasl.timeout", "30s")) * 1000; - } - - /** - * Max number of times we will try IO exceptions (such as connection timeouts) per request. - * If set to 0, we will not do any retries. - */ - public int maxIORetries() { return conf.getInt("spark.shuffle.io.maxRetries", 3); } - - /** - * Time (in milliseconds) that we will wait in order to perform a retry after an IOException. - * Only relevant if maxIORetries > 0. - */ - public int ioRetryWaitTimeMs() { - return (int) JavaUtils.timeStringAsSec(conf.get("spark.shuffle.io.retryWait", "5s")) * 1000; - } - - /** - * Minimum size of a block that we should start using memory map rather than reading in through - * normal IO operations. This prevents Spark from memory mapping very small blocks. In general, - * memory mapping has high overhead for blocks close to or below the page size of the OS. - */ - public int memoryMapBytes() { - return conf.getInt("spark.storage.memoryMapThreshold", 2 * 1024 * 1024); - } - - /** - * Whether to initialize shuffle FileDescriptor lazily or not. If true, file descriptors are - * created only when data is going to be transferred. This can reduce the number of open files. - */ - public boolean lazyFileDescriptor() { - return conf.getBoolean("spark.shuffle.io.lazyFD", true); - } - - /** - * Maximum number of retries when binding to a port before giving up. - */ - public int portMaxRetries() { - return conf.getInt("spark.port.maxRetries", 16); - } - - /** - * Maximum number of bytes to be encrypted at a time when SASL encryption is enabled. - */ - public int maxSaslEncryptedBlockSize() { - return Ints.checkedCast(JavaUtils.byteStringAsBytes( - conf.get("spark.network.sasl.maxEncryptedBlockSize", "64k"))); - } - - /** - * Whether the server should enforce encryption on SASL-authenticated connections. - */ - public boolean saslServerAlwaysEncrypt() { - return conf.getBoolean("spark.network.sasl.serverAlwaysEncrypt", false); - } - -} diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java b/network/common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java deleted file mode 100644 index 272ea84e6180..000000000000 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java +++ /dev/null @@ -1,154 +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.network.util; - -import com.google.common.base.Preconditions; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.CompositeByteBuf; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelInboundHandlerAdapter; - -/** - * A customized frame decoder that allows intercepting raw data. - *

    - * This behaves like Netty's frame decoder (with harcoded parameters that match this library's - * needs), except it allows an interceptor to be installed to read data directly before it's - * framed. - *

    - * Unlike Netty's frame decoder, each frame is dispatched to child handlers as soon as it's - * decoded, instead of building as many frames as the current buffer allows and dispatching - * all of them. This allows a child handler to install an interceptor if needed. - *

    - * If an interceptor is installed, framing stops, and data is instead fed directly to the - * interceptor. When the interceptor indicates that it doesn't need to read any more data, - * framing resumes. Interceptors should not hold references to the data buffers provided - * to their handle() method. - */ -public class TransportFrameDecoder extends ChannelInboundHandlerAdapter { - - public static final String HANDLER_NAME = "frameDecoder"; - private static final int LENGTH_SIZE = 8; - private static final int MAX_FRAME_SIZE = Integer.MAX_VALUE; - - private CompositeByteBuf buffer; - private volatile Interceptor interceptor; - - @Override - public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception { - ByteBuf in = (ByteBuf) data; - - if (buffer == null) { - buffer = in.alloc().compositeBuffer(); - } - - buffer.writeBytes(in); - - while (buffer.isReadable()) { - feedInterceptor(); - if (interceptor != null) { - continue; - } - - ByteBuf frame = decodeNext(); - if (frame != null) { - ctx.fireChannelRead(frame); - } else { - break; - } - } - - // We can't discard read sub-buffers if there are other references to the buffer (e.g. - // through slices used for framing). This assumes that code that retains references - // will call retain() from the thread that called "fireChannelRead()" above, otherwise - // ref counting will go awry. - if (buffer != null && buffer.refCnt() == 1) { - buffer.discardReadComponents(); - } - } - - protected ByteBuf decodeNext() throws Exception { - if (buffer.readableBytes() < LENGTH_SIZE) { - return null; - } - - int frameLen = (int) buffer.readLong() - LENGTH_SIZE; - if (buffer.readableBytes() < frameLen) { - buffer.readerIndex(buffer.readerIndex() - LENGTH_SIZE); - return null; - } - - Preconditions.checkArgument(frameLen < MAX_FRAME_SIZE, "Too large frame: %s", frameLen); - Preconditions.checkArgument(frameLen > 0, "Frame length should be positive: %s", frameLen); - - ByteBuf frame = buffer.readSlice(frameLen); - frame.retain(); - return frame; - } - - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - if (buffer != null) { - if (buffer.isReadable()) { - feedInterceptor(); - } - buffer.release(); - } - if (interceptor != null) { - interceptor.channelInactive(); - } - super.channelInactive(ctx); - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { - if (interceptor != null) { - interceptor.exceptionCaught(cause); - } - super.exceptionCaught(ctx, cause); - } - - public void setInterceptor(Interceptor interceptor) { - Preconditions.checkState(this.interceptor == null, "Already have an interceptor."); - this.interceptor = interceptor; - } - - private void feedInterceptor() throws Exception { - if (interceptor != null && !interceptor.handle(buffer)) { - interceptor = null; - } - } - - public static interface Interceptor { - - /** - * Handles data received from the remote end. - * - * @param data Buffer containing data. - * @return "true" if the interceptor expects more data, "false" to uninstall the interceptor. - */ - boolean handle(ByteBuf data) throws Exception; - - /** Called if an exception is thrown in the channel pipeline. */ - void exceptionCaught(Throwable cause) throws Exception; - - /** Called if the channel is closed and the interceptor is still installed. */ - void channelInactive() throws Exception; - - } - -} diff --git a/network/common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java b/network/common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java deleted file mode 100644 index 84ebb337e6d5..000000000000 --- a/network/common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java +++ /dev/null @@ -1,277 +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.network; - -import com.google.common.collect.Maps; -import com.google.common.util.concurrent.Uninterruptibles; -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.buffer.NioManagedBuffer; -import org.apache.spark.network.client.ChunkReceivedCallback; -import org.apache.spark.network.client.RpcResponseCallback; -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.client.TransportClientFactory; -import org.apache.spark.network.server.RpcHandler; -import org.apache.spark.network.server.StreamManager; -import org.apache.spark.network.server.TransportServer; -import org.apache.spark.network.util.MapConfigProvider; -import org.apache.spark.network.util.TransportConf; -import org.junit.*; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.*; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; - -/** - * Suite which ensures that requests that go without a response for the network timeout period are - * failed, and the connection closed. - * - * In this suite, we use 2 seconds as the connection timeout, with some slack given in the tests, - * to ensure stability in different test environments. - */ -public class RequestTimeoutIntegrationSuite { - - private TransportServer server; - private TransportClientFactory clientFactory; - - private StreamManager defaultManager; - private TransportConf conf; - - // A large timeout that "shouldn't happen", for the sake of faulty tests not hanging forever. - private final int FOREVER = 60 * 1000; - - @Before - public void setUp() throws Exception { - Map configMap = Maps.newHashMap(); - configMap.put("spark.shuffle.io.connectionTimeout", "2s"); - conf = new TransportConf(new MapConfigProvider(configMap)); - - defaultManager = new StreamManager() { - @Override - public ManagedBuffer getChunk(long streamId, int chunkIndex) { - throw new UnsupportedOperationException(); - } - }; - } - - @After - public void tearDown() { - if (server != null) { - server.close(); - } - if (clientFactory != null) { - clientFactory.close(); - } - } - - // Basic suite: First request completes quickly, and second waits for longer than network timeout. - @Test - public void timeoutInactiveRequests() throws Exception { - final Semaphore semaphore = new Semaphore(1); - final byte[] response = new byte[16]; - RpcHandler handler = new RpcHandler() { - @Override - public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { - try { - semaphore.tryAcquire(FOREVER, TimeUnit.MILLISECONDS); - callback.onSuccess(response); - } catch (InterruptedException e) { - // do nothing - } - } - - @Override - public StreamManager getStreamManager() { - return defaultManager; - } - }; - - TransportContext context = new TransportContext(conf, handler); - server = context.createServer(); - clientFactory = context.createClientFactory(); - TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); - - // First completes quickly (semaphore starts at 1). - TestCallback callback0 = new TestCallback(); - synchronized (callback0) { - client.sendRpc(new byte[0], callback0); - callback0.wait(FOREVER); - assert (callback0.success.length == response.length); - } - - // Second times out after 2 seconds, with slack. Must be IOException. - TestCallback callback1 = new TestCallback(); - synchronized (callback1) { - client.sendRpc(new byte[0], callback1); - callback1.wait(4 * 1000); - assert (callback1.failure != null); - assert (callback1.failure instanceof IOException); - } - semaphore.release(); - } - - // A timeout will cause the connection to be closed, invalidating the current TransportClient. - // It should be the case that requesting a client from the factory produces a new, valid one. - @Test - public void timeoutCleanlyClosesClient() throws Exception { - final Semaphore semaphore = new Semaphore(0); - final byte[] response = new byte[16]; - RpcHandler handler = new RpcHandler() { - @Override - public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { - try { - semaphore.tryAcquire(FOREVER, TimeUnit.MILLISECONDS); - callback.onSuccess(response); - } catch (InterruptedException e) { - // do nothing - } - } - - @Override - public StreamManager getStreamManager() { - return defaultManager; - } - }; - - TransportContext context = new TransportContext(conf, handler); - server = context.createServer(); - clientFactory = context.createClientFactory(); - - // First request should eventually fail. - TransportClient client0 = - clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); - TestCallback callback0 = new TestCallback(); - synchronized (callback0) { - client0.sendRpc(new byte[0], callback0); - callback0.wait(FOREVER); - assert (callback0.failure instanceof IOException); - assert (!client0.isActive()); - } - - // Increment the semaphore and the second request should succeed quickly. - semaphore.release(2); - TransportClient client1 = - clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); - TestCallback callback1 = new TestCallback(); - synchronized (callback1) { - client1.sendRpc(new byte[0], callback1); - callback1.wait(FOREVER); - assert (callback1.success.length == response.length); - assert (callback1.failure == null); - } - } - - // The timeout is relative to the LAST request sent, which is kinda weird, but still. - // This test also makes sure the timeout works for Fetch requests as well as RPCs. - @Test - public void furtherRequestsDelay() throws Exception { - final byte[] response = new byte[16]; - final StreamManager manager = new StreamManager() { - @Override - public ManagedBuffer getChunk(long streamId, int chunkIndex) { - Uninterruptibles.sleepUninterruptibly(FOREVER, TimeUnit.MILLISECONDS); - return new NioManagedBuffer(ByteBuffer.wrap(response)); - } - }; - RpcHandler handler = new RpcHandler() { - @Override - public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { - throw new UnsupportedOperationException(); - } - - @Override - public StreamManager getStreamManager() { - return manager; - } - }; - - TransportContext context = new TransportContext(conf, handler); - server = context.createServer(); - clientFactory = context.createClientFactory(); - TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); - - // Send one request, which will eventually fail. - TestCallback callback0 = new TestCallback(); - client.fetchChunk(0, 0, callback0); - Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS); - - // Send a second request before the first has failed. - TestCallback callback1 = new TestCallback(); - client.fetchChunk(0, 1, callback1); - Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS); - - synchronized (callback0) { - // not complete yet, but should complete soon - assert (callback0.success == null && callback0.failure == null); - callback0.wait(2 * 1000); - assert (callback0.failure instanceof IOException); - } - - synchronized (callback1) { - // failed at same time as previous - assert (callback0.failure instanceof IOException); - } - } - - /** - * Callback which sets 'success' or 'failure' on completion. - * Additionally notifies all waiters on this callback when invoked. - */ - class TestCallback implements RpcResponseCallback, ChunkReceivedCallback { - - byte[] success; - Throwable failure; - - @Override - public void onSuccess(byte[] response) { - synchronized(this) { - success = response; - this.notifyAll(); - } - } - - @Override - public void onFailure(Throwable e) { - synchronized(this) { - failure = e; - this.notifyAll(); - } - } - - @Override - public void onSuccess(int chunkIndex, ManagedBuffer buffer) { - synchronized(this) { - try { - success = buffer.nioByteBuffer().array(); - this.notifyAll(); - } catch (IOException e) { - // weird - } - } - } - - @Override - public void onFailure(int chunkIndex, Throwable e) { - synchronized(this) { - failure = e; - this.notifyAll(); - } - } - } -} diff --git a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java deleted file mode 100644 index 35de5e57ccb9..000000000000 --- a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java +++ /dev/null @@ -1,180 +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.network; - -import java.io.IOException; -import java.util.Collections; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; - -import com.google.common.collect.Maps; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.client.TransportClientFactory; -import org.apache.spark.network.server.NoOpRpcHandler; -import org.apache.spark.network.server.RpcHandler; -import org.apache.spark.network.server.TransportServer; -import org.apache.spark.network.util.SystemPropertyConfigProvider; -import org.apache.spark.network.util.JavaUtils; -import org.apache.spark.network.util.MapConfigProvider; -import org.apache.spark.network.util.TransportConf; - -public class TransportClientFactorySuite { - private TransportConf conf; - private TransportContext context; - private TransportServer server1; - private TransportServer server2; - - @Before - public void setUp() { - conf = new TransportConf(new SystemPropertyConfigProvider()); - RpcHandler rpcHandler = new NoOpRpcHandler(); - context = new TransportContext(conf, rpcHandler); - server1 = context.createServer(); - server2 = context.createServer(); - } - - @After - public void tearDown() { - JavaUtils.closeQuietly(server1); - JavaUtils.closeQuietly(server2); - } - - /** - * Request a bunch of clients to a single server to test - * we create up to maxConnections of clients. - * - * If concurrent is true, create multiple threads to create clients in parallel. - */ - private void testClientReuse(final int maxConnections, boolean concurrent) - throws IOException, InterruptedException { - - Map configMap = Maps.newHashMap(); - configMap.put("spark.shuffle.io.numConnectionsPerPeer", Integer.toString(maxConnections)); - TransportConf conf = new TransportConf(new MapConfigProvider(configMap)); - - RpcHandler rpcHandler = new NoOpRpcHandler(); - TransportContext context = new TransportContext(conf, rpcHandler); - final TransportClientFactory factory = context.createClientFactory(); - final Set clients = Collections.synchronizedSet( - new HashSet()); - - final AtomicInteger failed = new AtomicInteger(); - Thread[] attempts = new Thread[maxConnections * 10]; - - // Launch a bunch of threads to create new clients. - for (int i = 0; i < attempts.length; i++) { - attempts[i] = new Thread() { - @Override - public void run() { - try { - TransportClient client = - factory.createClient(TestUtils.getLocalHost(), server1.getPort()); - assert (client.isActive()); - clients.add(client); - } catch (IOException e) { - failed.incrementAndGet(); - } - } - }; - - if (concurrent) { - attempts[i].start(); - } else { - attempts[i].run(); - } - } - - // Wait until all the threads complete. - for (int i = 0; i < attempts.length; i++) { - attempts[i].join(); - } - - assert(failed.get() == 0); - assert(clients.size() == maxConnections); - - for (TransportClient client : clients) { - client.close(); - } - } - - @Test - public void reuseClientsUpToConfigVariable() throws Exception { - testClientReuse(1, false); - testClientReuse(2, false); - testClientReuse(3, false); - testClientReuse(4, false); - } - - @Test - public void reuseClientsUpToConfigVariableConcurrent() throws Exception { - testClientReuse(1, true); - testClientReuse(2, true); - testClientReuse(3, true); - testClientReuse(4, true); - } - - @Test - public void returnDifferentClientsForDifferentServers() throws IOException { - TransportClientFactory factory = context.createClientFactory(); - TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); - TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); - assertTrue(c1.isActive()); - assertTrue(c2.isActive()); - assertTrue(c1 != c2); - factory.close(); - } - - @Test - public void neverReturnInactiveClients() throws IOException, InterruptedException { - TransportClientFactory factory = context.createClientFactory(); - TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); - c1.close(); - - long start = System.currentTimeMillis(); - while (c1.isActive() && (System.currentTimeMillis() - start) < 3000) { - Thread.sleep(10); - } - assertFalse(c1.isActive()); - - TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); - assertFalse(c1 == c2); - assertTrue(c2.isActive()); - factory.close(); - } - - @Test - public void closeBlockClientsWithFactory() throws IOException { - TransportClientFactory factory = context.createClientFactory(); - TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); - TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); - assertTrue(c1.isActive()); - assertTrue(c2.isActive()); - factory.close(); - assertFalse(c1.isActive()); - assertFalse(c2.isActive()); - } -} diff --git a/network/common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java b/network/common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java deleted file mode 100644 index 17a03ebe88a9..000000000000 --- a/network/common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java +++ /dev/null @@ -1,115 +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.network; - -import io.netty.channel.local.LocalChannel; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.*; - -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.client.ChunkReceivedCallback; -import org.apache.spark.network.client.RpcResponseCallback; -import org.apache.spark.network.client.TransportResponseHandler; -import org.apache.spark.network.protocol.ChunkFetchFailure; -import org.apache.spark.network.protocol.ChunkFetchSuccess; -import org.apache.spark.network.protocol.RpcFailure; -import org.apache.spark.network.protocol.RpcResponse; -import org.apache.spark.network.protocol.StreamChunkId; - -public class TransportResponseHandlerSuite { - @Test - public void handleSuccessfulFetch() { - StreamChunkId streamChunkId = new StreamChunkId(1, 0); - - TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); - ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); - handler.addFetchRequest(streamChunkId, callback); - assertEquals(1, handler.numOutstandingRequests()); - - handler.handle(new ChunkFetchSuccess(streamChunkId, new TestManagedBuffer(123))); - verify(callback, times(1)).onSuccess(eq(0), (ManagedBuffer) any()); - assertEquals(0, handler.numOutstandingRequests()); - } - - @Test - public void handleFailedFetch() { - StreamChunkId streamChunkId = new StreamChunkId(1, 0); - TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); - ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); - handler.addFetchRequest(streamChunkId, callback); - assertEquals(1, handler.numOutstandingRequests()); - - handler.handle(new ChunkFetchFailure(streamChunkId, "some error msg")); - verify(callback, times(1)).onFailure(eq(0), (Throwable) any()); - assertEquals(0, handler.numOutstandingRequests()); - } - - @Test - public void clearAllOutstandingRequests() { - TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); - ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); - handler.addFetchRequest(new StreamChunkId(1, 0), callback); - handler.addFetchRequest(new StreamChunkId(1, 1), callback); - handler.addFetchRequest(new StreamChunkId(1, 2), callback); - assertEquals(3, handler.numOutstandingRequests()); - - handler.handle(new ChunkFetchSuccess(new StreamChunkId(1, 0), new TestManagedBuffer(12))); - handler.exceptionCaught(new Exception("duh duh duhhhh")); - - // should fail both b2 and b3 - verify(callback, times(1)).onSuccess(eq(0), (ManagedBuffer) any()); - verify(callback, times(1)).onFailure(eq(1), (Throwable) any()); - verify(callback, times(1)).onFailure(eq(2), (Throwable) any()); - assertEquals(0, handler.numOutstandingRequests()); - } - - @Test - public void handleSuccessfulRPC() { - TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); - RpcResponseCallback callback = mock(RpcResponseCallback.class); - handler.addRpcRequest(12345, callback); - assertEquals(1, handler.numOutstandingRequests()); - - handler.handle(new RpcResponse(54321, new byte[7])); // should be ignored - assertEquals(1, handler.numOutstandingRequests()); - - byte[] arr = new byte[10]; - handler.handle(new RpcResponse(12345, arr)); - verify(callback, times(1)).onSuccess(eq(arr)); - assertEquals(0, handler.numOutstandingRequests()); - } - - @Test - public void handleFailedRPC() { - TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); - RpcResponseCallback callback = mock(RpcResponseCallback.class); - handler.addRpcRequest(12345, callback); - assertEquals(1, handler.numOutstandingRequests()); - - handler.handle(new RpcFailure(54321, "uh-oh!")); // should be ignored - assertEquals(1, handler.numOutstandingRequests()); - - handler.handle(new RpcFailure(12345, "oh no")); - verify(callback, times(1)).onFailure((Throwable) any()); - assertEquals(0, handler.numOutstandingRequests()); - } -} diff --git a/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java b/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java deleted file mode 100644 index 6c98e733b462..000000000000 --- a/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java +++ /dev/null @@ -1,129 +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.network.protocol; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.WritableByteChannel; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; -import io.netty.channel.FileRegion; -import io.netty.util.AbstractReferenceCounted; -import org.junit.Test; - -import static org.junit.Assert.*; - -import org.apache.spark.network.util.ByteArrayWritableChannel; - -public class MessageWithHeaderSuite { - - @Test - public void testSingleWrite() throws Exception { - testFileRegionBody(8, 8); - } - - @Test - public void testShortWrite() throws Exception { - testFileRegionBody(8, 1); - } - - @Test - public void testByteBufBody() throws Exception { - ByteBuf header = Unpooled.copyLong(42); - ByteBuf body = Unpooled.copyLong(84); - MessageWithHeader msg = new MessageWithHeader(header, body, body.readableBytes()); - - ByteBuf result = doWrite(msg, 1); - assertEquals(msg.count(), result.readableBytes()); - assertEquals(42, result.readLong()); - assertEquals(84, result.readLong()); - } - - private void testFileRegionBody(int totalWrites, int writesPerCall) throws Exception { - ByteBuf header = Unpooled.copyLong(42); - int headerLength = header.readableBytes(); - TestFileRegion region = new TestFileRegion(totalWrites, writesPerCall); - MessageWithHeader msg = new MessageWithHeader(header, region, region.count()); - - ByteBuf result = doWrite(msg, totalWrites / writesPerCall); - assertEquals(headerLength + region.count(), result.readableBytes()); - assertEquals(42, result.readLong()); - for (long i = 0; i < 8; i++) { - assertEquals(i, result.readLong()); - } - } - - private ByteBuf doWrite(MessageWithHeader msg, int minExpectedWrites) throws Exception { - int writes = 0; - ByteArrayWritableChannel channel = new ByteArrayWritableChannel((int) msg.count()); - while (msg.transfered() < msg.count()) { - msg.transferTo(channel, msg.transfered()); - writes++; - } - assertTrue("Not enough writes!", minExpectedWrites <= writes); - return Unpooled.wrappedBuffer(channel.getData()); - } - - private static class TestFileRegion extends AbstractReferenceCounted implements FileRegion { - - private final int writeCount; - private final int writesPerCall; - private int written; - - TestFileRegion(int totalWrites, int writesPerCall) { - this.writeCount = totalWrites; - this.writesPerCall = writesPerCall; - } - - @Override - public long count() { - return 8 * writeCount; - } - - @Override - public long position() { - return 0; - } - - @Override - public long transfered() { - return 8 * written; - } - - @Override - public long transferTo(WritableByteChannel target, long position) throws IOException { - for (int i = 0; i < writesPerCall; i++) { - ByteBuf buf = Unpooled.copyLong((position / 8) + i); - ByteBuffer nio = buf.nioBuffer(); - while (nio.remaining() > 0) { - target.write(nio); - } - buf.release(); - written++; - } - return 8 * writesPerCall; - } - - @Override - protected void deallocate() { - } - - } - -} diff --git a/network/common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java b/network/common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java deleted file mode 100644 index ca74f0a00cf9..000000000000 --- a/network/common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java +++ /dev/null @@ -1,142 +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.network.util; - -import java.nio.ByteBuffer; -import java.util.Random; - -import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; -import io.netty.channel.ChannelHandlerContext; -import org.junit.Test; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; - -public class TransportFrameDecoderSuite { - - @Test - public void testFrameDecoding() throws Exception { - Random rnd = new Random(); - TransportFrameDecoder decoder = new TransportFrameDecoder(); - ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); - - final int frameCount = 100; - ByteBuf data = Unpooled.buffer(); - try { - for (int i = 0; i < frameCount; i++) { - byte[] frame = new byte[1024 * (rnd.nextInt(31) + 1)]; - data.writeLong(frame.length + 8); - data.writeBytes(frame); - } - - while (data.isReadable()) { - int size = rnd.nextInt(16 * 1024) + 256; - decoder.channelRead(ctx, data.readSlice(Math.min(data.readableBytes(), size))); - } - - verify(ctx, times(frameCount)).fireChannelRead(any(ByteBuf.class)); - } finally { - data.release(); - } - } - - @Test - public void testInterception() throws Exception { - final int interceptedReads = 3; - TransportFrameDecoder decoder = new TransportFrameDecoder(); - TransportFrameDecoder.Interceptor interceptor = spy(new MockInterceptor(interceptedReads)); - ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); - - byte[] data = new byte[8]; - ByteBuf len = Unpooled.copyLong(8 + data.length); - ByteBuf dataBuf = Unpooled.wrappedBuffer(data); - - try { - decoder.setInterceptor(interceptor); - for (int i = 0; i < interceptedReads; i++) { - decoder.channelRead(ctx, dataBuf); - dataBuf.release(); - dataBuf = Unpooled.wrappedBuffer(data); - } - decoder.channelRead(ctx, len); - decoder.channelRead(ctx, dataBuf); - verify(interceptor, times(interceptedReads)).handle(any(ByteBuf.class)); - verify(ctx).fireChannelRead(any(ByteBuffer.class)); - } finally { - len.release(); - dataBuf.release(); - } - } - - @Test(expected = IllegalArgumentException.class) - public void testNegativeFrameSize() throws Exception { - testInvalidFrame(-1); - } - - @Test(expected = IllegalArgumentException.class) - public void testEmptyFrame() throws Exception { - // 8 because frame size includes the frame length. - testInvalidFrame(8); - } - - @Test(expected = IllegalArgumentException.class) - public void testLargeFrame() throws Exception { - // Frame length includes the frame size field, so need to add a few more bytes. - testInvalidFrame(Integer.MAX_VALUE + 9); - } - - private void testInvalidFrame(long size) throws Exception { - TransportFrameDecoder decoder = new TransportFrameDecoder(); - ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); - ByteBuf frame = Unpooled.copyLong(size); - try { - decoder.channelRead(ctx, frame); - } finally { - frame.release(); - } - } - - private static class MockInterceptor implements TransportFrameDecoder.Interceptor { - - private int remainingReads; - - MockInterceptor(int readCount) { - this.remainingReads = readCount; - } - - @Override - public boolean handle(ByteBuf data) throws Exception { - data.readerIndex(data.readerIndex() + data.readableBytes()); - assertFalse(data.isReadable()); - remainingReads -= 1; - return remainingReads != 0; - } - - @Override - public void exceptionCaught(Throwable cause) throws Exception { - - } - - @Override - public void channelInactive() throws Exception { - - } - - } - -} diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml deleted file mode 100644 index 70ba5cb1995b..000000000000 --- a/network/shuffle/pom.xml +++ /dev/null @@ -1,101 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-network-shuffle_2.10 - jar - Spark Project Shuffle Streaming Service - http://spark.apache.org/ - - network-shuffle - - - - - - org.apache.spark - spark-network-common_${scala.binary.version} - ${project.version} - - - - org.fusesource.leveldbjni - leveldbjni-all - 1.8 - - - - com.fasterxml.jackson.core - jackson-databind - - - - com.fasterxml.jackson.core - jackson-annotations - - - - - org.slf4j - slf4j-api - provided - - - com.google.guava - guava - - - - - org.apache.spark - spark-network-common_${scala.binary.version} - ${project.version} - test-jar - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - log4j - log4j - test - - - org.mockito - mockito-core - test - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java deleted file mode 100644 index 3ddf5c3c3918..000000000000 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ /dev/null @@ -1,139 +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.network.shuffle; - -import java.io.File; -import java.io.IOException; -import java.util.List; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.client.RpcResponseCallback; -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.server.OneForOneStreamManager; -import org.apache.spark.network.server.RpcHandler; -import org.apache.spark.network.server.StreamManager; -import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; -import org.apache.spark.network.shuffle.protocol.*; -import org.apache.spark.network.util.TransportConf; - - -/** - * RPC Handler for a server which can serve shuffle blocks from outside of an Executor process. - * - * Handles registering executors and opening shuffle blocks from them. Shuffle blocks are registered - * with the "one-for-one" strategy, meaning each Transport-layer Chunk is equivalent to one Spark- - * level shuffle block. - */ -public class ExternalShuffleBlockHandler extends RpcHandler { - private final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockHandler.class); - - @VisibleForTesting - final ExternalShuffleBlockResolver blockManager; - private final OneForOneStreamManager streamManager; - - public ExternalShuffleBlockHandler(TransportConf conf, File registeredExecutorFile) throws IOException { - this(new OneForOneStreamManager(), - new ExternalShuffleBlockResolver(conf, registeredExecutorFile)); - } - - /** Enables mocking out the StreamManager and BlockManager. */ - @VisibleForTesting - public ExternalShuffleBlockHandler( - OneForOneStreamManager streamManager, - ExternalShuffleBlockResolver blockManager) { - this.streamManager = streamManager; - this.blockManager = blockManager; - } - - @Override - public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { - BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteArray(message); - handleMessage(msgObj, client, callback); - } - - protected void handleMessage( - BlockTransferMessage msgObj, - TransportClient client, - RpcResponseCallback callback) { - if (msgObj instanceof OpenBlocks) { - OpenBlocks msg = (OpenBlocks) msgObj; - checkAuth(client, msg.appId); - - List blocks = Lists.newArrayList(); - for (String blockId : msg.blockIds) { - blocks.add(blockManager.getBlockData(msg.appId, msg.execId, blockId)); - } - long streamId = streamManager.registerStream(client.getClientId(), blocks.iterator()); - logger.trace("Registered streamId {} with {} buffers", streamId, msg.blockIds.length); - callback.onSuccess(new StreamHandle(streamId, msg.blockIds.length).toByteArray()); - - } else if (msgObj instanceof RegisterExecutor) { - RegisterExecutor msg = (RegisterExecutor) msgObj; - checkAuth(client, msg.appId); - blockManager.registerExecutor(msg.appId, msg.execId, msg.executorInfo); - callback.onSuccess(new byte[0]); - - } else { - throw new UnsupportedOperationException("Unexpected message: " + msgObj); - } - } - - @Override - public StreamManager getStreamManager() { - return streamManager; - } - - /** - * Removes an application (once it has been terminated), and optionally will clean up any - * local directories associated with the executors of that application in a separate thread. - */ - public void applicationRemoved(String appId, boolean cleanupLocalDirs) { - blockManager.applicationRemoved(appId, cleanupLocalDirs); - } - - /** - * Register an (application, executor) with the given shuffle info. - * - * The "re-" is meant to highlight the intended use of this method -- when this service is - * restarted, this is used to restore the state of executors from before the restart. Normal - * registration will happen via a message handled in receive() - * - * @param appExecId - * @param executorInfo - */ - public void reregisterExecutor(AppExecId appExecId, ExecutorShuffleInfo executorInfo) { - blockManager.registerExecutor(appExecId.appId, appExecId.execId, executorInfo); - } - - public void close() { - blockManager.close(); - } - - private void checkAuth(TransportClient client, String appId) { - if (client.getClientId() != null && !client.getClientId().equals(appId)) { - throw new SecurityException(String.format( - "Client for %s not authorized for application %s.", client.getClientId(), appId)); - } - } - -} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java deleted file mode 100644 index 0d4dd6afac76..000000000000 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ /dev/null @@ -1,450 +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.network.shuffle; - -import java.io.*; -import java.util.*; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.Executor; -import java.util.concurrent.Executors; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Charsets; -import com.google.common.base.Objects; -import com.google.common.collect.Maps; -import org.fusesource.leveldbjni.JniDBFactory; -import org.fusesource.leveldbjni.internal.NativeDB; -import org.iq80.leveldb.DB; -import org.iq80.leveldb.DBIterator; -import org.iq80.leveldb.Options; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.buffer.FileSegmentManagedBuffer; -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; -import org.apache.spark.network.util.JavaUtils; -import org.apache.spark.network.util.NettyUtils; -import org.apache.spark.network.util.TransportConf; - -/** - * Manages converting shuffle BlockIds into physical segments of local files, from a process outside - * of Executors. Each Executor must register its own configuration about where it stores its files - * (local dirs) and how (shuffle manager). The logic for retrieval of individual files is replicated - * from Spark's FileShuffleBlockResolver and IndexShuffleBlockResolver. - */ -public class ExternalShuffleBlockResolver { - private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockResolver.class); - - private static final ObjectMapper mapper = new ObjectMapper(); - /** - * This a common prefix to the key for each app registration we stick in leveldb, so they - * are easy to find, since leveldb lets you search based on prefix. - */ - private static final String APP_KEY_PREFIX = "AppExecShuffleInfo"; - private static final StoreVersion CURRENT_VERSION = new StoreVersion(1, 0); - - // Map containing all registered executors' metadata. - @VisibleForTesting - final ConcurrentMap executors; - - // Single-threaded Java executor used to perform expensive recursive directory deletion. - private final Executor directoryCleaner; - - private final TransportConf conf; - - @VisibleForTesting - final File registeredExecutorFile; - @VisibleForTesting - final DB db; - - public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) - throws IOException { - this(conf, registeredExecutorFile, Executors.newSingleThreadExecutor( - // Add `spark` prefix because it will run in NM in Yarn mode. - NettyUtils.createThreadFactory("spark-shuffle-directory-cleaner"))); - } - - // Allows tests to have more control over when directories are cleaned up. - @VisibleForTesting - ExternalShuffleBlockResolver( - TransportConf conf, - File registeredExecutorFile, - Executor directoryCleaner) throws IOException { - this.conf = conf; - this.registeredExecutorFile = registeredExecutorFile; - if (registeredExecutorFile != null) { - Options options = new Options(); - options.createIfMissing(false); - options.logger(new LevelDBLogger()); - DB tmpDb; - try { - tmpDb = JniDBFactory.factory.open(registeredExecutorFile, options); - } catch (NativeDB.DBException e) { - if (e.isNotFound() || e.getMessage().contains(" does not exist ")) { - logger.info("Creating state database at " + registeredExecutorFile); - options.createIfMissing(true); - try { - tmpDb = JniDBFactory.factory.open(registeredExecutorFile, options); - } catch (NativeDB.DBException dbExc) { - throw new IOException("Unable to create state store", dbExc); - } - } else { - // the leveldb file seems to be corrupt somehow. Lets just blow it away and create a new - // one, so we can keep processing new apps - logger.error("error opening leveldb file {}. Creating new file, will not be able to " + - "recover state for existing applications", registeredExecutorFile, e); - if (registeredExecutorFile.isDirectory()) { - for (File f : registeredExecutorFile.listFiles()) { - if (!f.delete()) { - logger.warn("error deleting {}", f.getPath()); - } - } - } - if (!registeredExecutorFile.delete()) { - logger.warn("error deleting {}", registeredExecutorFile.getPath()); - } - options.createIfMissing(true); - try { - tmpDb = JniDBFactory.factory.open(registeredExecutorFile, options); - } catch (NativeDB.DBException dbExc) { - throw new IOException("Unable to create state store", dbExc); - } - - } - } - // if there is a version mismatch, we throw an exception, which means the service is unusable - checkVersion(tmpDb); - executors = reloadRegisteredExecutors(tmpDb); - db = tmpDb; - } else { - db = null; - executors = Maps.newConcurrentMap(); - } - this.directoryCleaner = directoryCleaner; - } - - /** Registers a new Executor with all the configuration we need to find its shuffle files. */ - public void registerExecutor( - String appId, - String execId, - ExecutorShuffleInfo executorInfo) { - AppExecId fullId = new AppExecId(appId, execId); - logger.info("Registered executor {} with {}", fullId, executorInfo); - try { - if (db != null) { - byte[] key = dbAppExecKey(fullId); - byte[] value = mapper.writeValueAsString(executorInfo).getBytes(Charsets.UTF_8); - db.put(key, value); - } - } catch (Exception e) { - logger.error("Error saving registered executors", e); - } - executors.put(fullId, executorInfo); - } - - /** - * Obtains a FileSegmentManagedBuffer from a shuffle block id. We expect the blockId has the - * format "shuffle_ShuffleId_MapId_ReduceId" (from ShuffleBlockId), and additionally make - * assumptions about how the hash and sort based shuffles store their data. - */ - public ManagedBuffer getBlockData(String appId, String execId, String blockId) { - String[] blockIdParts = blockId.split("_"); - if (blockIdParts.length < 4) { - throw new IllegalArgumentException("Unexpected block id format: " + blockId); - } else if (!blockIdParts[0].equals("shuffle")) { - throw new IllegalArgumentException("Expected shuffle block id, got: " + blockId); - } - int shuffleId = Integer.parseInt(blockIdParts[1]); - int mapId = Integer.parseInt(blockIdParts[2]); - int reduceId = Integer.parseInt(blockIdParts[3]); - - ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId)); - if (executor == null) { - throw new RuntimeException( - String.format("Executor is not registered (appId=%s, execId=%s)", appId, execId)); - } - - if ("org.apache.spark.shuffle.hash.HashShuffleManager".equals(executor.shuffleManager)) { - return getHashBasedShuffleBlockData(executor, blockId); - } else if ("org.apache.spark.shuffle.sort.SortShuffleManager".equals(executor.shuffleManager) - || "org.apache.spark.shuffle.unsafe.UnsafeShuffleManager".equals(executor.shuffleManager)) { - return getSortBasedShuffleBlockData(executor, shuffleId, mapId, reduceId); - } else { - throw new UnsupportedOperationException( - "Unsupported shuffle manager: " + executor.shuffleManager); - } - } - - /** - * Removes our metadata of all executors registered for the given application, and optionally - * also deletes the local directories associated with the executors of that application in a - * separate thread. - * - * It is not valid to call registerExecutor() for an executor with this appId after invoking - * this method. - */ - public void applicationRemoved(String appId, boolean cleanupLocalDirs) { - logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); - Iterator> it = executors.entrySet().iterator(); - while (it.hasNext()) { - Map.Entry entry = it.next(); - AppExecId fullId = entry.getKey(); - final ExecutorShuffleInfo executor = entry.getValue(); - - // Only touch executors associated with the appId that was removed. - if (appId.equals(fullId.appId)) { - it.remove(); - if (db != null) { - try { - db.delete(dbAppExecKey(fullId)); - } catch (IOException e) { - logger.error("Error deleting {} from executor state db", appId, e); - } - } - - if (cleanupLocalDirs) { - logger.info("Cleaning up executor {}'s {} local dirs", fullId, executor.localDirs.length); - - // Execute the actual deletion in a different thread, as it may take some time. - directoryCleaner.execute(new Runnable() { - @Override - public void run() { - deleteExecutorDirs(executor.localDirs); - } - }); - } - } - } - } - - /** - * Synchronously deletes each directory one at a time. - * Should be executed in its own thread, as this may take a long time. - */ - private void deleteExecutorDirs(String[] dirs) { - for (String localDir : dirs) { - try { - JavaUtils.deleteRecursively(new File(localDir)); - logger.debug("Successfully cleaned up directory: " + localDir); - } catch (Exception e) { - logger.error("Failed to delete directory: " + localDir, e); - } - } - } - - /** - * Hash-based shuffle data is simply stored as one file per block. - * This logic is from FileShuffleBlockResolver. - */ - private ManagedBuffer getHashBasedShuffleBlockData(ExecutorShuffleInfo executor, String blockId) { - File shuffleFile = getFile(executor.localDirs, executor.subDirsPerLocalDir, blockId); - return new FileSegmentManagedBuffer(conf, shuffleFile, 0, shuffleFile.length()); - } - - /** - * Sort-based shuffle data uses an index called "shuffle_ShuffleId_MapId_0.index" into a data file - * called "shuffle_ShuffleId_MapId_0.data". This logic is from IndexShuffleBlockResolver, - * and the block id format is from ShuffleDataBlockId and ShuffleIndexBlockId. - */ - private ManagedBuffer getSortBasedShuffleBlockData( - ExecutorShuffleInfo executor, int shuffleId, int mapId, int reduceId) { - File indexFile = getFile(executor.localDirs, executor.subDirsPerLocalDir, - "shuffle_" + shuffleId + "_" + mapId + "_0.index"); - - DataInputStream in = null; - try { - in = new DataInputStream(new FileInputStream(indexFile)); - in.skipBytes(reduceId * 8); - long offset = in.readLong(); - long nextOffset = in.readLong(); - return new FileSegmentManagedBuffer( - conf, - getFile(executor.localDirs, executor.subDirsPerLocalDir, - "shuffle_" + shuffleId + "_" + mapId + "_0.data"), - offset, - nextOffset - offset); - } catch (IOException e) { - throw new RuntimeException("Failed to open file: " + indexFile, e); - } finally { - if (in != null) { - JavaUtils.closeQuietly(in); - } - } - } - - /** - * Hashes a filename into the corresponding local directory, in a manner consistent with - * Spark's DiskBlockManager.getFile(). - */ - @VisibleForTesting - static File getFile(String[] localDirs, int subDirsPerLocalDir, String filename) { - int hash = JavaUtils.nonNegativeHash(filename); - String localDir = localDirs[hash % localDirs.length]; - int subDirId = (hash / localDirs.length) % subDirsPerLocalDir; - return new File(new File(localDir, String.format("%02x", subDirId)), filename); - } - - void close() { - if (db != null) { - try { - db.close(); - } catch (IOException e) { - logger.error("Exception closing leveldb with registered executors", e); - } - } - } - - /** Simply encodes an executor's full ID, which is appId + execId. */ - public static class AppExecId { - public final String appId; - public final String execId; - - @JsonCreator - public AppExecId(@JsonProperty("appId") String appId, @JsonProperty("execId") String execId) { - this.appId = appId; - this.execId = execId; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - AppExecId appExecId = (AppExecId) o; - return Objects.equal(appId, appExecId.appId) && Objects.equal(execId, appExecId.execId); - } - - @Override - public int hashCode() { - return Objects.hashCode(appId, execId); - } - - @Override - public String toString() { - return Objects.toStringHelper(this) - .add("appId", appId) - .add("execId", execId) - .toString(); - } - } - - private static byte[] dbAppExecKey(AppExecId appExecId) throws IOException { - // we stick a common prefix on all the keys so we can find them in the DB - String appExecJson = mapper.writeValueAsString(appExecId); - String key = (APP_KEY_PREFIX + ";" + appExecJson); - return key.getBytes(Charsets.UTF_8); - } - - private static AppExecId parseDbAppExecKey(String s) throws IOException { - if (!s.startsWith(APP_KEY_PREFIX)) { - throw new IllegalArgumentException("expected a string starting with " + APP_KEY_PREFIX); - } - String json = s.substring(APP_KEY_PREFIX.length() + 1); - AppExecId parsed = mapper.readValue(json, AppExecId.class); - return parsed; - } - - @VisibleForTesting - static ConcurrentMap reloadRegisteredExecutors(DB db) - throws IOException { - ConcurrentMap registeredExecutors = Maps.newConcurrentMap(); - if (db != null) { - DBIterator itr = db.iterator(); - itr.seek(APP_KEY_PREFIX.getBytes(Charsets.UTF_8)); - while (itr.hasNext()) { - Map.Entry e = itr.next(); - String key = new String(e.getKey(), Charsets.UTF_8); - if (!key.startsWith(APP_KEY_PREFIX)) { - break; - } - AppExecId id = parseDbAppExecKey(key); - ExecutorShuffleInfo shuffleInfo = mapper.readValue(e.getValue(), ExecutorShuffleInfo.class); - registeredExecutors.put(id, shuffleInfo); - } - } - return registeredExecutors; - } - - private static class LevelDBLogger implements org.iq80.leveldb.Logger { - private static final Logger LOG = LoggerFactory.getLogger(LevelDBLogger.class); - - @Override - public void log(String message) { - LOG.info(message); - } - } - - /** - * Simple major.minor versioning scheme. Any incompatible changes should be across major - * versions. Minor version differences are allowed -- meaning we should be able to read - * dbs that are either earlier *or* later on the minor version. - */ - private static void checkVersion(DB db) throws IOException { - byte[] bytes = db.get(StoreVersion.KEY); - if (bytes == null) { - storeVersion(db); - } else { - StoreVersion version = mapper.readValue(bytes, StoreVersion.class); - if (version.major != CURRENT_VERSION.major) { - throw new IOException("cannot read state DB with version " + version + ", incompatible " + - "with current version " + CURRENT_VERSION); - } - storeVersion(db); - } - } - - private static void storeVersion(DB db) throws IOException { - db.put(StoreVersion.KEY, mapper.writeValueAsBytes(CURRENT_VERSION)); - } - - - public static class StoreVersion { - - final static byte[] KEY = "StoreVersion".getBytes(Charsets.UTF_8); - - public final int major; - public final int minor; - - @JsonCreator public StoreVersion(@JsonProperty("major") int major, @JsonProperty("minor") int minor) { - this.major = major; - this.minor = minor; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - StoreVersion that = (StoreVersion) o; - - return major == that.major && minor == that.minor; - } - - @Override - public int hashCode() { - int result = major; - result = 31 * result + minor; - return result; - } - } - -} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java deleted file mode 100644 index ea6d248d66be..000000000000 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java +++ /dev/null @@ -1,149 +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.network.shuffle; - -import java.io.IOException; -import java.util.List; - -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.TransportContext; -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.client.TransportClientBootstrap; -import org.apache.spark.network.client.TransportClientFactory; -import org.apache.spark.network.sasl.SaslClientBootstrap; -import org.apache.spark.network.sasl.SecretKeyHolder; -import org.apache.spark.network.server.NoOpRpcHandler; -import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; -import org.apache.spark.network.shuffle.protocol.RegisterExecutor; -import org.apache.spark.network.util.TransportConf; - -/** - * Client for reading shuffle blocks which points to an external (outside of executor) server. - * This is instead of reading shuffle blocks directly from other executors (via - * BlockTransferService), which has the downside of losing the shuffle data if we lose the - * executors. - */ -public class ExternalShuffleClient extends ShuffleClient { - private final Logger logger = LoggerFactory.getLogger(ExternalShuffleClient.class); - - private final TransportConf conf; - private final boolean saslEnabled; - private final boolean saslEncryptionEnabled; - private final SecretKeyHolder secretKeyHolder; - - protected TransportClientFactory clientFactory; - protected String appId; - - /** - * Creates an external shuffle client, with SASL optionally enabled. If SASL is not enabled, - * then secretKeyHolder may be null. - */ - public ExternalShuffleClient( - TransportConf conf, - SecretKeyHolder secretKeyHolder, - boolean saslEnabled, - boolean saslEncryptionEnabled) { - Preconditions.checkArgument( - !saslEncryptionEnabled || saslEnabled, - "SASL encryption can only be enabled if SASL is also enabled."); - this.conf = conf; - this.secretKeyHolder = secretKeyHolder; - this.saslEnabled = saslEnabled; - this.saslEncryptionEnabled = saslEncryptionEnabled; - } - - protected void checkInit() { - assert appId != null : "Called before init()"; - } - - @Override - public void init(String appId) { - this.appId = appId; - TransportContext context = new TransportContext(conf, new NoOpRpcHandler()); - List bootstraps = Lists.newArrayList(); - if (saslEnabled) { - bootstraps.add(new SaslClientBootstrap(conf, appId, secretKeyHolder, saslEncryptionEnabled)); - } - clientFactory = context.createClientFactory(bootstraps); - } - - @Override - public void fetchBlocks( - final String host, - final int port, - final String execId, - String[] blockIds, - BlockFetchingListener listener) { - checkInit(); - logger.debug("External shuffle fetch from {}:{} (executor id {})", host, port, execId); - try { - RetryingBlockFetcher.BlockFetchStarter blockFetchStarter = - new RetryingBlockFetcher.BlockFetchStarter() { - @Override - public void createAndStart(String[] blockIds, BlockFetchingListener listener) - throws IOException { - TransportClient client = clientFactory.createClient(host, port); - new OneForOneBlockFetcher(client, appId, execId, blockIds, listener).start(); - } - }; - - int maxRetries = conf.maxIORetries(); - if (maxRetries > 0) { - // Note this Fetcher will correctly handle maxRetries == 0; we avoid it just in case there's - // a bug in this code. We should remove the if statement once we're sure of the stability. - new RetryingBlockFetcher(conf, blockFetchStarter, blockIds, listener).start(); - } else { - blockFetchStarter.createAndStart(blockIds, listener); - } - } catch (Exception e) { - logger.error("Exception while beginning fetchBlocks", e); - for (String blockId : blockIds) { - listener.onBlockFetchFailure(blockId, e); - } - } - } - - /** - * Registers this executor with an external shuffle server. This registration is required to - * inform the shuffle server about where and how we store our shuffle files. - * - * @param host Host of shuffle server. - * @param port Port of shuffle server. - * @param execId This Executor's id. - * @param executorInfo Contains all info necessary for the service to find our shuffle files. - */ - public void registerWithShuffleServer( - String host, - int port, - String execId, - ExecutorShuffleInfo executorInfo) throws IOException { - checkInit(); - TransportClient client = clientFactory.createClient(host, port); - byte[] registerMessage = new RegisterExecutor(appId, execId, executorInfo).toByteArray(); - client.sendRpcSync(registerMessage, 5000 /* timeoutMs */); - } - - @Override - public void close() { - clientFactory.close(); - } -} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java deleted file mode 100644 index e653f5cb147e..000000000000 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ /dev/null @@ -1,128 +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.network.shuffle; - -import java.util.Arrays; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.client.ChunkReceivedCallback; -import org.apache.spark.network.client.RpcResponseCallback; -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; -import org.apache.spark.network.shuffle.protocol.OpenBlocks; -import org.apache.spark.network.shuffle.protocol.StreamHandle; - -/** - * Simple wrapper on top of a TransportClient which interprets each chunk as a whole block, and - * invokes the BlockFetchingListener appropriately. This class is agnostic to the actual RPC - * handler, as long as there is a single "open blocks" message which returns a ShuffleStreamHandle, - * and Java serialization is used. - * - * Note that this typically corresponds to a - * {@link org.apache.spark.network.server.OneForOneStreamManager} on the server side. - */ -public class OneForOneBlockFetcher { - private final Logger logger = LoggerFactory.getLogger(OneForOneBlockFetcher.class); - - private final TransportClient client; - private final OpenBlocks openMessage; - private final String[] blockIds; - private final BlockFetchingListener listener; - private final ChunkReceivedCallback chunkCallback; - - private StreamHandle streamHandle = null; - - public OneForOneBlockFetcher( - TransportClient client, - String appId, - String execId, - String[] blockIds, - BlockFetchingListener listener) { - this.client = client; - this.openMessage = new OpenBlocks(appId, execId, blockIds); - this.blockIds = blockIds; - this.listener = listener; - this.chunkCallback = new ChunkCallback(); - } - - /** Callback invoked on receipt of each chunk. We equate a single chunk to a single block. */ - private class ChunkCallback implements ChunkReceivedCallback { - @Override - public void onSuccess(int chunkIndex, ManagedBuffer buffer) { - // On receipt of a chunk, pass it upwards as a block. - listener.onBlockFetchSuccess(blockIds[chunkIndex], buffer); - } - - @Override - public void onFailure(int chunkIndex, Throwable e) { - // On receipt of a failure, fail every block from chunkIndex onwards. - String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); - failRemainingBlocks(remainingBlockIds, e); - } - } - - /** - * Begins the fetching process, calling the listener with every block fetched. - * The given message will be serialized with the Java serializer, and the RPC must return a - * {@link StreamHandle}. We will send all fetch requests immediately, without throttling. - */ - public void start() { - if (blockIds.length == 0) { - throw new IllegalArgumentException("Zero-sized blockIds array"); - } - - client.sendRpc(openMessage.toByteArray(), new RpcResponseCallback() { - @Override - public void onSuccess(byte[] response) { - try { - streamHandle = (StreamHandle) BlockTransferMessage.Decoder.fromByteArray(response); - logger.trace("Successfully opened blocks {}, preparing to fetch chunks.", streamHandle); - - // Immediately request all chunks -- we expect that the total size of the request is - // reasonable due to higher level chunking in [[ShuffleBlockFetcherIterator]]. - for (int i = 0; i < streamHandle.numChunks; i++) { - client.fetchChunk(streamHandle.streamId, i, chunkCallback); - } - } catch (Exception e) { - logger.error("Failed while starting block fetches after success", e); - failRemainingBlocks(blockIds, e); - } - } - - @Override - public void onFailure(Throwable e) { - logger.error("Failed while starting block fetches", e); - failRemainingBlocks(blockIds, e); - } - }); - } - - /** Invokes the "onBlockFetchFailure" callback for every listed block id. */ - private void failRemainingBlocks(String[] failedBlockIds, Throwable e) { - for (String blockId : failedBlockIds) { - try { - listener.onBlockFetchFailure(blockId, e); - } catch (Exception e2) { - logger.error("Error in block fetch failure callback", e2); - } - } - } -} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java deleted file mode 100644 index f72ab40690d0..000000000000 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java +++ /dev/null @@ -1,44 +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.network.shuffle; - -import java.io.Closeable; - -/** Provides an interface for reading shuffle files, either from an Executor or external service. */ -public abstract class ShuffleClient implements Closeable { - - /** - * Initializes the ShuffleClient, specifying this Executor's appId. - * Must be called before any other method on the ShuffleClient. - */ - public void init(String appId) { } - - /** - * Fetch a sequence of blocks from a remote node asynchronously, - * - * Note that this API takes a sequence so the implementation can batch requests, and does not - * return a future so the underlying implementation can invoke onBlockFetchSuccess as soon as - * the data of a block is fetched, rather than waiting for all blocks to be fetched. - */ - public abstract void fetchBlocks( - String host, - int port, - String execId, - String[] blockIds, - BlockFetchingListener listener); -} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java deleted file mode 100644 index 7543b6be4f2a..000000000000 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java +++ /dev/null @@ -1,72 +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.network.shuffle.mesos; - -import java.io.IOException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.client.RpcResponseCallback; -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.sasl.SecretKeyHolder; -import org.apache.spark.network.shuffle.ExternalShuffleClient; -import org.apache.spark.network.shuffle.protocol.mesos.RegisterDriver; -import org.apache.spark.network.util.TransportConf; - -/** - * A client for talking to the external shuffle service in Mesos coarse-grained mode. - * - * This is used by the Spark driver to register with each external shuffle service on the cluster. - * The reason why the driver has to talk to the service is for cleaning up shuffle files reliably - * after the application exits. Mesos does not provide a great alternative to do this, so Spark - * has to detect this itself. - */ -public class MesosExternalShuffleClient extends ExternalShuffleClient { - private final Logger logger = LoggerFactory.getLogger(MesosExternalShuffleClient.class); - - /** - * Creates an Mesos external shuffle client that wraps the {@link ExternalShuffleClient}. - * Please refer to docs on {@link ExternalShuffleClient} for more information. - */ - public MesosExternalShuffleClient( - TransportConf conf, - SecretKeyHolder secretKeyHolder, - boolean saslEnabled, - boolean saslEncryptionEnabled) { - super(conf, secretKeyHolder, saslEnabled, saslEncryptionEnabled); - } - - public void registerDriverWithShuffleService(String host, int port) throws IOException { - checkInit(); - byte[] registerDriver = new RegisterDriver(appId).toByteArray(); - TransportClient client = clientFactory.createClient(host, port); - client.sendRpc(registerDriver, new RpcResponseCallback() { - @Override - public void onSuccess(byte[] response) { - logger.info("Successfully registered app " + appId + " with external shuffle service."); - } - - @Override - public void onFailure(Throwable e) { - logger.warn("Unable to register app " + appId + " with external shuffle service. " + - "Please manually remove shuffle data after driver exit. Error: " + e); - } - }); - } -} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java deleted file mode 100644 index 94a61d6caadc..000000000000 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java +++ /dev/null @@ -1,63 +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.network.shuffle.protocol.mesos; - -import com.google.common.base.Objects; -import io.netty.buffer.ByteBuf; - -import org.apache.spark.network.protocol.Encoders; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; - -// Needed by ScalaDoc. See SPARK-7726 -import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; - -/** - * A message sent from the driver to register with the MesosExternalShuffleService. - */ -public class RegisterDriver extends BlockTransferMessage { - private final String appId; - - public RegisterDriver(String appId) { - this.appId = appId; - } - - public String getAppId() { return appId; } - - @Override - protected Type type() { return Type.REGISTER_DRIVER; } - - @Override - public int encodedLength() { - return Encoders.Strings.encodedLength(appId); - } - - @Override - public void encode(ByteBuf buf) { - Encoders.Strings.encode(buf, appId); - } - - @Override - public int hashCode() { - return Objects.hashCode(appId); - } - - public static RegisterDriver decode(ByteBuf buf) { - String appId = Encoders.Strings.decode(buf); - return new RegisterDriver(appId); - } -} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java deleted file mode 100644 index e61390cf5706..000000000000 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ /dev/null @@ -1,126 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.shuffle; - -import java.nio.ByteBuffer; -import java.util.Iterator; - -import org.junit.Before; -import org.junit.Test; -import org.mockito.ArgumentCaptor; - -import static org.junit.Assert.*; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.*; - -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.buffer.NioManagedBuffer; -import org.apache.spark.network.client.RpcResponseCallback; -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.server.OneForOneStreamManager; -import org.apache.spark.network.server.RpcHandler; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; -import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; -import org.apache.spark.network.shuffle.protocol.OpenBlocks; -import org.apache.spark.network.shuffle.protocol.RegisterExecutor; -import org.apache.spark.network.shuffle.protocol.StreamHandle; -import org.apache.spark.network.shuffle.protocol.UploadBlock; - -public class ExternalShuffleBlockHandlerSuite { - TransportClient client = mock(TransportClient.class); - - OneForOneStreamManager streamManager; - ExternalShuffleBlockResolver blockResolver; - RpcHandler handler; - - @Before - public void beforeEach() { - streamManager = mock(OneForOneStreamManager.class); - blockResolver = mock(ExternalShuffleBlockResolver.class); - handler = new ExternalShuffleBlockHandler(streamManager, blockResolver); - } - - @Test - public void testRegisterExecutor() { - RpcResponseCallback callback = mock(RpcResponseCallback.class); - - ExecutorShuffleInfo config = new ExecutorShuffleInfo(new String[] {"/a", "/b"}, 16, "sort"); - byte[] registerMessage = new RegisterExecutor("app0", "exec1", config).toByteArray(); - handler.receive(client, registerMessage, callback); - verify(blockResolver, times(1)).registerExecutor("app0", "exec1", config); - - verify(callback, times(1)).onSuccess((byte[]) any()); - verify(callback, never()).onFailure((Throwable) any()); - } - - @SuppressWarnings("unchecked") - @Test - public void testOpenShuffleBlocks() { - RpcResponseCallback callback = mock(RpcResponseCallback.class); - - ManagedBuffer block0Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[3])); - ManagedBuffer block1Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); - when(blockResolver.getBlockData("app0", "exec1", "b0")).thenReturn(block0Marker); - when(blockResolver.getBlockData("app0", "exec1", "b1")).thenReturn(block1Marker); - byte[] openBlocks = new OpenBlocks("app0", "exec1", new String[] { "b0", "b1" }).toByteArray(); - handler.receive(client, openBlocks, callback); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", "b0"); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", "b1"); - - ArgumentCaptor response = ArgumentCaptor.forClass(byte[].class); - verify(callback, times(1)).onSuccess(response.capture()); - verify(callback, never()).onFailure((Throwable) any()); - - StreamHandle handle = - (StreamHandle) BlockTransferMessage.Decoder.fromByteArray(response.getValue()); - assertEquals(2, handle.numChunks); - - @SuppressWarnings("unchecked") - ArgumentCaptor> stream = (ArgumentCaptor>) - (ArgumentCaptor) ArgumentCaptor.forClass(Iterator.class); - verify(streamManager, times(1)).registerStream(anyString(), stream.capture()); - Iterator buffers = stream.getValue(); - assertEquals(block0Marker, buffers.next()); - assertEquals(block1Marker, buffers.next()); - assertFalse(buffers.hasNext()); - } - - @Test - public void testBadMessages() { - RpcResponseCallback callback = mock(RpcResponseCallback.class); - - byte[] unserializableMsg = new byte[] { 0x12, 0x34, 0x56 }; - try { - handler.receive(client, unserializableMsg, callback); - fail("Should have thrown"); - } catch (Exception e) { - // pass - } - - byte[] unexpectedMsg = new UploadBlock("a", "e", "b", new byte[1], new byte[2]).toByteArray(); - try { - handler.receive(client, unexpectedMsg, callback); - fail("Should have thrown"); - } catch (UnsupportedOperationException e) { - // pass - } - - verify(callback, never()).onSuccess((byte[]) any()); - verify(callback, never()).onFailure((Throwable) any()); - } -} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java deleted file mode 100644 index 3c6cb367dea4..000000000000 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ /dev/null @@ -1,156 +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.network.shuffle; - -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.io.CharStreams; -import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; -import org.apache.spark.network.util.SystemPropertyConfigProvider; -import org.apache.spark.network.util.TransportConf; -import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; - -import static org.junit.Assert.*; - -public class ExternalShuffleBlockResolverSuite { - static String sortBlock0 = "Hello!"; - static String sortBlock1 = "World!"; - - static String hashBlock0 = "Elementary"; - static String hashBlock1 = "Tabular"; - - static TestShuffleDataContext dataContext; - - static TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); - - @BeforeClass - public static void beforeAll() throws IOException { - dataContext = new TestShuffleDataContext(2, 5); - - dataContext.create(); - // Write some sort and hash data. - dataContext.insertSortShuffleData(0, 0, - new byte[][] { sortBlock0.getBytes(), sortBlock1.getBytes() } ); - dataContext.insertHashShuffleData(1, 0, - new byte[][] { hashBlock0.getBytes(), hashBlock1.getBytes() } ); - } - - @AfterClass - public static void afterAll() { - dataContext.cleanup(); - } - - @Test - public void testBadRequests() throws IOException { - ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); - // Unregistered executor - try { - resolver.getBlockData("app0", "exec1", "shuffle_1_1_0"); - fail("Should have failed"); - } catch (RuntimeException e) { - assertTrue("Bad error message: " + e, e.getMessage().contains("not registered")); - } - - // Invalid shuffle manager - resolver.registerExecutor("app0", "exec2", dataContext.createExecutorInfo("foobar")); - try { - resolver.getBlockData("app0", "exec2", "shuffle_1_1_0"); - fail("Should have failed"); - } catch (UnsupportedOperationException e) { - // pass - } - - // Nonexistent shuffle block - resolver.registerExecutor("app0", "exec3", - dataContext.createExecutorInfo("org.apache.spark.shuffle.sort.SortShuffleManager")); - try { - resolver.getBlockData("app0", "exec3", "shuffle_1_1_0"); - fail("Should have failed"); - } catch (Exception e) { - // pass - } - } - - @Test - public void testSortShuffleBlocks() throws IOException { - ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); - resolver.registerExecutor("app0", "exec0", - dataContext.createExecutorInfo("org.apache.spark.shuffle.sort.SortShuffleManager")); - - InputStream block0Stream = - resolver.getBlockData("app0", "exec0", "shuffle_0_0_0").createInputStream(); - String block0 = CharStreams.toString(new InputStreamReader(block0Stream)); - block0Stream.close(); - assertEquals(sortBlock0, block0); - - InputStream block1Stream = - resolver.getBlockData("app0", "exec0", "shuffle_0_0_1").createInputStream(); - String block1 = CharStreams.toString(new InputStreamReader(block1Stream)); - block1Stream.close(); - assertEquals(sortBlock1, block1); - } - - @Test - public void testHashShuffleBlocks() throws IOException { - ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); - resolver.registerExecutor("app0", "exec0", - dataContext.createExecutorInfo("org.apache.spark.shuffle.hash.HashShuffleManager")); - - InputStream block0Stream = - resolver.getBlockData("app0", "exec0", "shuffle_1_0_0").createInputStream(); - String block0 = CharStreams.toString(new InputStreamReader(block0Stream)); - block0Stream.close(); - assertEquals(hashBlock0, block0); - - InputStream block1Stream = - resolver.getBlockData("app0", "exec0", "shuffle_1_0_1").createInputStream(); - String block1 = CharStreams.toString(new InputStreamReader(block1Stream)); - block1Stream.close(); - assertEquals(hashBlock1, block1); - } - - @Test - public void jsonSerializationOfExecutorRegistration() throws IOException { - ObjectMapper mapper = new ObjectMapper(); - AppExecId appId = new AppExecId("foo", "bar"); - String appIdJson = mapper.writeValueAsString(appId); - AppExecId parsedAppId = mapper.readValue(appIdJson, AppExecId.class); - assertEquals(parsedAppId, appId); - - ExecutorShuffleInfo shuffleInfo = - new ExecutorShuffleInfo(new String[]{"/bippy", "/flippy"}, 7, "hash"); - String shuffleJson = mapper.writeValueAsString(shuffleInfo); - ExecutorShuffleInfo parsedShuffleInfo = - mapper.readValue(shuffleJson, ExecutorShuffleInfo.class); - assertEquals(parsedShuffleInfo, shuffleInfo); - - // Intentionally keep these hard-coded strings in here, to check backwards-compatability. - // its not legacy yet, but keeping this here in case anybody changes it - String legacyAppIdJson = "{\"appId\":\"foo\", \"execId\":\"bar\"}"; - assertEquals(appId, mapper.readValue(legacyAppIdJson, AppExecId.class)); - String legacyShuffleJson = "{\"localDirs\": [\"/bippy\", \"/flippy\"], " + - "\"subDirsPerLocalDir\": 7, \"shuffleManager\": \"hash\"}"; - assertEquals(shuffleInfo, mapper.readValue(legacyShuffleJson, ExecutorShuffleInfo.class)); - } -} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java deleted file mode 100644 index a3f9a38b1aeb..000000000000 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ /dev/null @@ -1,301 +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.network.shuffle; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Collections; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; - -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; - -import static org.junit.Assert.*; - -import org.apache.spark.network.TestUtils; -import org.apache.spark.network.TransportContext; -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.buffer.NioManagedBuffer; -import org.apache.spark.network.server.TransportServer; -import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; -import org.apache.spark.network.util.SystemPropertyConfigProvider; -import org.apache.spark.network.util.TransportConf; - -public class ExternalShuffleIntegrationSuite { - - static String APP_ID = "app-id"; - static String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"; - static String HASH_MANAGER = "org.apache.spark.shuffle.hash.HashShuffleManager"; - - // Executor 0 is sort-based - static TestShuffleDataContext dataContext0; - // Executor 1 is hash-based - static TestShuffleDataContext dataContext1; - - static ExternalShuffleBlockHandler handler; - static TransportServer server; - static TransportConf conf; - - static byte[][] exec0Blocks = new byte[][] { - new byte[123], - new byte[12345], - new byte[1234567], - }; - - static byte[][] exec1Blocks = new byte[][] { - new byte[321], - new byte[54321], - }; - - @BeforeClass - public static void beforeAll() throws IOException { - Random rand = new Random(); - - for (byte[] block : exec0Blocks) { - rand.nextBytes(block); - } - for (byte[] block: exec1Blocks) { - rand.nextBytes(block); - } - - dataContext0 = new TestShuffleDataContext(2, 5); - dataContext0.create(); - dataContext0.insertSortShuffleData(0, 0, exec0Blocks); - - dataContext1 = new TestShuffleDataContext(6, 2); - dataContext1.create(); - dataContext1.insertHashShuffleData(1, 0, exec1Blocks); - - conf = new TransportConf(new SystemPropertyConfigProvider()); - handler = new ExternalShuffleBlockHandler(conf, null); - TransportContext transportContext = new TransportContext(conf, handler); - server = transportContext.createServer(); - } - - @AfterClass - public static void afterAll() { - dataContext0.cleanup(); - dataContext1.cleanup(); - server.close(); - } - - @After - public void afterEach() { - handler.applicationRemoved(APP_ID, false /* cleanupLocalDirs */); - } - - class FetchResult { - public Set successBlocks; - public Set failedBlocks; - public List buffers; - - public void releaseBuffers() { - for (ManagedBuffer buffer : buffers) { - buffer.release(); - } - } - } - - // Fetch a set of blocks from a pre-registered executor. - private FetchResult fetchBlocks(String execId, String[] blockIds) throws Exception { - return fetchBlocks(execId, blockIds, server.getPort()); - } - - // Fetch a set of blocks from a pre-registered executor. Connects to the server on the given port, - // to allow connecting to invalid servers. - private FetchResult fetchBlocks(String execId, String[] blockIds, int port) throws Exception { - final FetchResult res = new FetchResult(); - res.successBlocks = Collections.synchronizedSet(new HashSet()); - res.failedBlocks = Collections.synchronizedSet(new HashSet()); - res.buffers = Collections.synchronizedList(new LinkedList()); - - final Semaphore requestsRemaining = new Semaphore(0); - - ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false, false); - client.init(APP_ID); - client.fetchBlocks(TestUtils.getLocalHost(), port, execId, blockIds, - new BlockFetchingListener() { - @Override - public void onBlockFetchSuccess(String blockId, ManagedBuffer data) { - synchronized (this) { - if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) { - data.retain(); - res.successBlocks.add(blockId); - res.buffers.add(data); - requestsRemaining.release(); - } - } - } - - @Override - public void onBlockFetchFailure(String blockId, Throwable exception) { - synchronized (this) { - if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) { - res.failedBlocks.add(blockId); - requestsRemaining.release(); - } - } - } - }); - - if (!requestsRemaining.tryAcquire(blockIds.length, 5, TimeUnit.SECONDS)) { - fail("Timeout getting response from the server"); - } - client.close(); - return res; - } - - @Test - public void testFetchOneSort() throws Exception { - registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult exec0Fetch = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0" }); - assertEquals(Sets.newHashSet("shuffle_0_0_0"), exec0Fetch.successBlocks); - assertTrue(exec0Fetch.failedBlocks.isEmpty()); - assertBufferListsEqual(exec0Fetch.buffers, Lists.newArrayList(exec0Blocks[0])); - exec0Fetch.releaseBuffers(); - } - - @Test - public void testFetchThreeSort() throws Exception { - registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult exec0Fetch = fetchBlocks("exec-0", - new String[] { "shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2" }); - assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2"), - exec0Fetch.successBlocks); - assertTrue(exec0Fetch.failedBlocks.isEmpty()); - assertBufferListsEqual(exec0Fetch.buffers, Lists.newArrayList(exec0Blocks)); - exec0Fetch.releaseBuffers(); - } - - @Test - public void testFetchHash() throws Exception { - registerExecutor("exec-1", dataContext1.createExecutorInfo(HASH_MANAGER)); - FetchResult execFetch = fetchBlocks("exec-1", - new String[] { "shuffle_1_0_0", "shuffle_1_0_1" }); - assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.successBlocks); - assertTrue(execFetch.failedBlocks.isEmpty()); - assertBufferListsEqual(execFetch.buffers, Lists.newArrayList(exec1Blocks)); - execFetch.releaseBuffers(); - } - - @Test - public void testFetchWrongShuffle() throws Exception { - registerExecutor("exec-1", dataContext1.createExecutorInfo(SORT_MANAGER /* wrong manager */)); - FetchResult execFetch = fetchBlocks("exec-1", - new String[] { "shuffle_1_0_0", "shuffle_1_0_1" }); - assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks); - } - - @Test - public void testFetchInvalidShuffle() throws Exception { - registerExecutor("exec-1", dataContext1.createExecutorInfo("unknown sort manager")); - FetchResult execFetch = fetchBlocks("exec-1", - new String[] { "shuffle_1_0_0" }); - assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_1_0_0"), execFetch.failedBlocks); - } - - @Test - public void testFetchWrongBlockId() throws Exception { - registerExecutor("exec-1", dataContext1.createExecutorInfo(SORT_MANAGER /* wrong manager */)); - FetchResult execFetch = fetchBlocks("exec-1", - new String[] { "rdd_1_0_0" }); - assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("rdd_1_0_0"), execFetch.failedBlocks); - } - - @Test - public void testFetchNonexistent() throws Exception { - registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult execFetch = fetchBlocks("exec-0", - new String[] { "shuffle_2_0_0" }); - assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_2_0_0"), execFetch.failedBlocks); - } - - @Test - public void testFetchWrongExecutor() throws Exception { - registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult execFetch = fetchBlocks("exec-0", - new String[] { "shuffle_0_0_0" /* right */, "shuffle_1_0_0" /* wrong */ }); - // Both still fail, as we start by checking for all block. - assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_1_0_0"), execFetch.failedBlocks); - } - - @Test - public void testFetchUnregisteredExecutor() throws Exception { - registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult execFetch = fetchBlocks("exec-2", - new String[] { "shuffle_0_0_0", "shuffle_1_0_0" }); - assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_1_0_0"), execFetch.failedBlocks); - } - - @Test - public void testFetchNoServer() throws Exception { - System.setProperty("spark.shuffle.io.maxRetries", "0"); - try { - registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult execFetch = fetchBlocks("exec-0", - new String[]{"shuffle_1_0_0", "shuffle_1_0_1"}, 1 /* port */); - assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks); - } finally { - System.clearProperty("spark.shuffle.io.maxRetries"); - } - } - - private void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo) - throws IOException { - ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false, false); - client.init(APP_ID); - client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), - executorId, executorInfo); - } - - private void assertBufferListsEqual(List list0, List list1) - throws Exception { - assertEquals(list0.size(), list1.size()); - for (int i = 0; i < list0.size(); i ++) { - assertBuffersEqual(list0.get(i), new NioManagedBuffer(ByteBuffer.wrap(list1.get(i)))); - } - } - - private void assertBuffersEqual(ManagedBuffer buffer0, ManagedBuffer buffer1) throws Exception { - ByteBuffer nio0 = buffer0.nioByteBuffer(); - ByteBuffer nio1 = buffer1.nioByteBuffer(); - - int len = nio0.remaining(); - assertEquals(nio0.remaining(), nio1.remaining()); - for (int i = 0; i < len; i ++) { - assertEquals(nio0.get(), nio1.get()); - } - } -} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java deleted file mode 100644 index b35a6d685dd0..000000000000 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ /dev/null @@ -1,176 +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.network.shuffle; - -import java.nio.ByteBuffer; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.concurrent.atomic.AtomicInteger; - -import com.google.common.collect.Maps; -import io.netty.buffer.Unpooled; -import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; - -import org.apache.spark.network.buffer.ManagedBuffer; -import org.apache.spark.network.buffer.NettyManagedBuffer; -import org.apache.spark.network.buffer.NioManagedBuffer; -import org.apache.spark.network.client.ChunkReceivedCallback; -import org.apache.spark.network.client.RpcResponseCallback; -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; -import org.apache.spark.network.shuffle.protocol.OpenBlocks; -import org.apache.spark.network.shuffle.protocol.StreamHandle; - -public class OneForOneBlockFetcherSuite { - @Test - public void testFetchOne() { - LinkedHashMap blocks = Maps.newLinkedHashMap(); - blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); - - BlockFetchingListener listener = fetchBlocks(blocks); - - verify(listener).onBlockFetchSuccess("shuffle_0_0_0", blocks.get("shuffle_0_0_0")); - } - - @Test - public void testFetchThree() { - LinkedHashMap blocks = Maps.newLinkedHashMap(); - blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); - blocks.put("b1", new NioManagedBuffer(ByteBuffer.wrap(new byte[23]))); - blocks.put("b2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23]))); - - BlockFetchingListener listener = fetchBlocks(blocks); - - for (int i = 0; i < 3; i ++) { - verify(listener, times(1)).onBlockFetchSuccess("b" + i, blocks.get("b" + i)); - } - } - - @Test - public void testFailure() { - LinkedHashMap blocks = Maps.newLinkedHashMap(); - blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); - blocks.put("b1", null); - blocks.put("b2", null); - - BlockFetchingListener listener = fetchBlocks(blocks); - - // Each failure will cause a failure to be invoked in all remaining block fetches. - verify(listener, times(1)).onBlockFetchSuccess("b0", blocks.get("b0")); - verify(listener, times(1)).onBlockFetchFailure(eq("b1"), (Throwable) any()); - verify(listener, times(2)).onBlockFetchFailure(eq("b2"), (Throwable) any()); - } - - @Test - public void testFailureAndSuccess() { - LinkedHashMap blocks = Maps.newLinkedHashMap(); - blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); - blocks.put("b1", null); - blocks.put("b2", new NioManagedBuffer(ByteBuffer.wrap(new byte[21]))); - - BlockFetchingListener listener = fetchBlocks(blocks); - - // We may call both success and failure for the same block. - verify(listener, times(1)).onBlockFetchSuccess("b0", blocks.get("b0")); - verify(listener, times(1)).onBlockFetchFailure(eq("b1"), (Throwable) any()); - verify(listener, times(1)).onBlockFetchSuccess("b2", blocks.get("b2")); - verify(listener, times(1)).onBlockFetchFailure(eq("b2"), (Throwable) any()); - } - - @Test - public void testEmptyBlockFetch() { - try { - fetchBlocks(Maps.newLinkedHashMap()); - fail(); - } catch (IllegalArgumentException e) { - assertEquals("Zero-sized blockIds array", e.getMessage()); - } - } - - /** - * Begins a fetch on the given set of blocks by mocking out the server side of the RPC which - * simply returns the given (BlockId, Block) pairs. - * As "blocks" is a LinkedHashMap, the blocks are guaranteed to be returned in the same order - * that they were inserted in. - * - * If a block's buffer is "null", an exception will be thrown instead. - */ - private BlockFetchingListener fetchBlocks(final LinkedHashMap blocks) { - TransportClient client = mock(TransportClient.class); - BlockFetchingListener listener = mock(BlockFetchingListener.class); - final String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); - OneForOneBlockFetcher fetcher = - new OneForOneBlockFetcher(client, "app-id", "exec-id", blockIds, listener); - - // Respond to the "OpenBlocks" message with an appropirate ShuffleStreamHandle with streamId 123 - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocationOnMock) throws Throwable { - BlockTransferMessage message = BlockTransferMessage.Decoder.fromByteArray( - (byte[]) invocationOnMock.getArguments()[0]); - RpcResponseCallback callback = (RpcResponseCallback) invocationOnMock.getArguments()[1]; - callback.onSuccess(new StreamHandle(123, blocks.size()).toByteArray()); - assertEquals(new OpenBlocks("app-id", "exec-id", blockIds), message); - return null; - } - }).when(client).sendRpc((byte[]) any(), (RpcResponseCallback) any()); - - // Respond to each chunk request with a single buffer from our blocks array. - final AtomicInteger expectedChunkIndex = new AtomicInteger(0); - final Iterator blockIterator = blocks.values().iterator(); - doAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - try { - long streamId = (Long) invocation.getArguments()[0]; - int myChunkIndex = (Integer) invocation.getArguments()[1]; - assertEquals(123, streamId); - assertEquals(expectedChunkIndex.getAndIncrement(), myChunkIndex); - - ChunkReceivedCallback callback = (ChunkReceivedCallback) invocation.getArguments()[2]; - ManagedBuffer result = blockIterator.next(); - if (result != null) { - callback.onSuccess(myChunkIndex, result); - } else { - callback.onFailure(myChunkIndex, new RuntimeException("Failed " + myChunkIndex)); - } - } catch (Exception e) { - e.printStackTrace(); - fail("Unexpected failure"); - } - return null; - } - }).when(client).fetchChunk(anyLong(), anyInt(), (ChunkReceivedCallback) any()); - - fetcher.start(); - return listener; - } -} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java deleted file mode 100644 index 3fdde054ab6c..000000000000 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ /dev/null @@ -1,109 +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.network.shuffle; - -import java.io.DataOutputStream; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStream; - -import com.google.common.io.Files; - -import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; - -/** - * Manages some sort- and hash-based shuffle data, including the creation - * and cleanup of directories that can be read by the {@link ExternalShuffleBlockResolver}. - */ -public class TestShuffleDataContext { - public final String[] localDirs; - public final int subDirsPerLocalDir; - - public TestShuffleDataContext(int numLocalDirs, int subDirsPerLocalDir) { - this.localDirs = new String[numLocalDirs]; - this.subDirsPerLocalDir = subDirsPerLocalDir; - } - - public void create() { - for (int i = 0; i < localDirs.length; i ++) { - localDirs[i] = Files.createTempDir().getAbsolutePath(); - - for (int p = 0; p < subDirsPerLocalDir; p ++) { - new File(localDirs[i], String.format("%02x", p)).mkdirs(); - } - } - } - - public void cleanup() { - for (String localDir : localDirs) { - deleteRecursively(new File(localDir)); - } - } - - /** Creates reducer blocks in a sort-based data format within our local dirs. */ - public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) throws IOException { - String blockId = "shuffle_" + shuffleId + "_" + mapId + "_0"; - - OutputStream dataStream = new FileOutputStream( - ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId + ".data")); - DataOutputStream indexStream = new DataOutputStream(new FileOutputStream( - ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId + ".index"))); - - long offset = 0; - indexStream.writeLong(offset); - for (byte[] block : blocks) { - offset += block.length; - dataStream.write(block); - indexStream.writeLong(offset); - } - - dataStream.close(); - indexStream.close(); - } - - /** Creates reducer blocks in a hash-based data format within our local dirs. */ - public void insertHashShuffleData(int shuffleId, int mapId, byte[][] blocks) throws IOException { - for (int i = 0; i < blocks.length; i ++) { - String blockId = "shuffle_" + shuffleId + "_" + mapId + "_" + i; - Files.write(blocks[i], - ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId)); - } - } - - /** - * Creates an ExecutorShuffleInfo object based on the given shuffle manager which targets this - * context's directories. - */ - public ExecutorShuffleInfo createExecutorInfo(String shuffleManager) { - return new ExecutorShuffleInfo(localDirs, subDirsPerLocalDir, shuffleManager); - } - - private static void deleteRecursively(File f) { - assert f != null; - if (f.isDirectory()) { - File[] children = f.listFiles(); - if (children != null) { - for (File child : children) { - deleteRecursively(child); - } - } - } - f.delete(); - } -} diff --git a/network/yarn/pom.xml b/network/yarn/pom.xml deleted file mode 100644 index e2360eff5cfe..000000000000 --- a/network/yarn/pom.xml +++ /dev/null @@ -1,101 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-network-yarn_2.10 - jar - Spark Project YARN Shuffle Service - http://spark.apache.org/ - - network-yarn - - provided - - - - - - org.apache.spark - spark-network-shuffle_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - - - org.apache.hadoop - hadoop-client - - - org.slf4j - slf4j-api - provided - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-shade-plugin - - false - ${project.build.directory}/scala-${scala.binary.version}/spark-${project.version}-yarn-shuffle.jar - - - *:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - package - - shade - - - - - - - diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java deleted file mode 100644 index 11ea7f3fd3cf..000000000000 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ /dev/null @@ -1,224 +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.network.yarn; - -import java.io.File; -import java.nio.ByteBuffer; -import java.util.List; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.server.api.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.network.TransportContext; -import org.apache.spark.network.sasl.SaslServerBootstrap; -import org.apache.spark.network.sasl.ShuffleSecretManager; -import org.apache.spark.network.server.TransportServer; -import org.apache.spark.network.server.TransportServerBootstrap; -import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler; -import org.apache.spark.network.util.TransportConf; -import org.apache.spark.network.yarn.util.HadoopConfigProvider; - -/** - * An external shuffle service used by Spark on Yarn. - * - * This is intended to be a long-running auxiliary service that runs in the NodeManager process. - * A Spark application may connect to this service by setting `spark.shuffle.service.enabled`. - * The application also automatically derives the service port through `spark.shuffle.service.port` - * specified in the Yarn configuration. This is so that both the clients and the server agree on - * the same port to communicate on. - * - * The service also optionally supports authentication. This ensures that executors from one - * application cannot read the shuffle files written by those from another. This feature can be - * enabled by setting `spark.authenticate` in the Yarn configuration before starting the NM. - * Note that the Spark application must also set `spark.authenticate` manually and, unlike in - * the case of the service port, will not inherit this setting from the Yarn configuration. This - * is because an application running on the same Yarn cluster may choose to not use the external - * shuffle service, in which case its setting of `spark.authenticate` should be independent of - * the service's. - */ -public class YarnShuffleService extends AuxiliaryService { - private final Logger logger = LoggerFactory.getLogger(YarnShuffleService.class); - - // Port on which the shuffle server listens for fetch requests - private static final String SPARK_SHUFFLE_SERVICE_PORT_KEY = "spark.shuffle.service.port"; - private static final int DEFAULT_SPARK_SHUFFLE_SERVICE_PORT = 7337; - - // Whether the shuffle server should authenticate fetch requests - private static final String SPARK_AUTHENTICATE_KEY = "spark.authenticate"; - private static final boolean DEFAULT_SPARK_AUTHENTICATE = false; - - // An entity that manages the shuffle secret per application - // This is used only if authentication is enabled - private ShuffleSecretManager secretManager; - - // The actual server that serves shuffle files - private TransportServer shuffleServer = null; - - // Handles registering executors and opening shuffle blocks - @VisibleForTesting - ExternalShuffleBlockHandler blockHandler; - - // Where to store & reload executor info for recovering state after an NM restart - @VisibleForTesting - File registeredExecutorFile; - - // just for testing when you want to find an open port - @VisibleForTesting - static int boundPort = -1; - - // just for integration tests that want to look at this file -- in general not sensible as - // a static - @VisibleForTesting - static YarnShuffleService instance; - - public YarnShuffleService() { - super("spark_shuffle"); - logger.info("Initializing YARN shuffle service for Spark"); - instance = this; - } - - /** - * Return whether authentication is enabled as specified by the configuration. - * If so, fetch requests will fail unless the appropriate authentication secret - * for the application is provided. - */ - private boolean isAuthenticationEnabled() { - return secretManager != null; - } - - /** - * Start the shuffle server with the given configuration. - */ - @Override - protected void serviceInit(Configuration conf) { - - // In case this NM was killed while there were running spark applications, we need to restore - // lost state for the existing executors. We look for an existing file in the NM's local dirs. - // If we don't find one, then we choose a file to use to save the state next time. Even if - // an application was stopped while the NM was down, we expect yarn to call stopApplication() - // when it comes back - registeredExecutorFile = - findRegisteredExecutorFile(conf.getStrings("yarn.nodemanager.local-dirs")); - - TransportConf transportConf = new TransportConf(new HadoopConfigProvider(conf)); - // If authentication is enabled, set up the shuffle server to use a - // special RPC handler that filters out unauthenticated fetch requests - boolean authEnabled = conf.getBoolean(SPARK_AUTHENTICATE_KEY, DEFAULT_SPARK_AUTHENTICATE); - try { - blockHandler = new ExternalShuffleBlockHandler(transportConf, registeredExecutorFile); - } catch (Exception e) { - logger.error("Failed to initialize external shuffle service", e); - } - - List bootstraps = Lists.newArrayList(); - if (authEnabled) { - secretManager = new ShuffleSecretManager(); - bootstraps.add(new SaslServerBootstrap(transportConf, secretManager)); - } - - int port = conf.getInt( - SPARK_SHUFFLE_SERVICE_PORT_KEY, DEFAULT_SPARK_SHUFFLE_SERVICE_PORT); - TransportContext transportContext = new TransportContext(transportConf, blockHandler); - shuffleServer = transportContext.createServer(port, bootstraps); - // the port should normally be fixed, but for tests its useful to find an open port - port = shuffleServer.getPort(); - boundPort = port; - String authEnabledString = authEnabled ? "enabled" : "not enabled"; - logger.info("Started YARN shuffle service for Spark on port {}. " + - "Authentication is {}. Registered executor file is {}", port, authEnabledString, - registeredExecutorFile); - } - - @Override - public void initializeApplication(ApplicationInitializationContext context) { - String appId = context.getApplicationId().toString(); - try { - ByteBuffer shuffleSecret = context.getApplicationDataForService(); - logger.info("Initializing application {}", appId); - if (isAuthenticationEnabled()) { - secretManager.registerApp(appId, shuffleSecret); - } - } catch (Exception e) { - logger.error("Exception when initializing application {}", appId, e); - } - } - - @Override - public void stopApplication(ApplicationTerminationContext context) { - String appId = context.getApplicationId().toString(); - try { - logger.info("Stopping application {}", appId); - if (isAuthenticationEnabled()) { - secretManager.unregisterApp(appId); - } - blockHandler.applicationRemoved(appId, false /* clean up local dirs */); - } catch (Exception e) { - logger.error("Exception when stopping application {}", appId, e); - } - } - - @Override - public void initializeContainer(ContainerInitializationContext context) { - ContainerId containerId = context.getContainerId(); - logger.info("Initializing container {}", containerId); - } - - @Override - public void stopContainer(ContainerTerminationContext context) { - ContainerId containerId = context.getContainerId(); - logger.info("Stopping container {}", containerId); - } - - private File findRegisteredExecutorFile(String[] localDirs) { - for (String dir: localDirs) { - File f = new File(dir, "registeredExecutors.ldb"); - if (f.exists()) { - return f; - } - } - return new File(localDirs[0], "registeredExecutors.ldb"); - } - - /** - * Close the shuffle server to clean up any associated state. - */ - @Override - protected void serviceStop() { - try { - if (shuffleServer != null) { - shuffleServer.close(); - } - if (blockHandler != null) { - blockHandler.close(); - } - } catch (Exception e) { - logger.error("Exception when stopping service", e); - } - } - - // Not currently used - @Override - public ByteBuffer getMetaData() { - return ByteBuffer.allocate(0); - } -} diff --git a/pom.xml b/pom.xml index 4ed1c0c82dee..b0408ecca0f6 100644 --- a/pom.xml +++ b/pom.xml @@ -25,8 +25,8 @@ 14 org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT + spark-parent_2.11 + 2.3.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -58,10 +58,6 @@ https://issues.apache.org/jira/browse/SPARK - - ${maven.version} - - Dev Mailing List @@ -86,116 +82,133 @@ - tags + common/sketch + common/kvstore + common/network-common + common/network-shuffle + common/unsafe + common/tags core - bagel graphx mllib + mllib-local tools - network/common - network/shuffle streaming sql/catalyst sql/core sql/hive - unsafe assembly - external/twitter external/flume external/flume-sink external/flume-assembly - external/mqtt - external/mqtt-assembly - external/zeromq examples repl launcher - external/kafka - external/kafka-assembly + external/kafka-0-10 + external/kafka-0-10-assembly + external/kafka-0-10-sql UTF-8 UTF-8 - com.typesafe.akka - 2.3.11 - 1.7 - 3.3.3 + 1.8 + 3.3.9 spark - 0.21.1 - shaded-protobuf - 1.7.10 + 1.7.16 1.2.17 - 2.2.0 + 2.6.5 2.5.0 ${hadoop.version} - 0.98.7-hadoop2 - hbase 1.6.0 - 3.4.5 - 2.4.0 + 3.4.6 + 2.6.0 org.spark-project.hive - 1.2.1.spark + 1.2.1.spark2 1.2.1 - 10.10.1.1 - 1.7.0 + 10.12.1.1 + 1.8.2 + 1.4.0 + nohive 1.6.0 - 1.2.4 - 8.1.14.v20131031 - 3.0.0.v201112011016 - 0.5.0 + 9.3.20.v20170531 + 3.1.0 + 0.8.4 2.4.0 2.0.8 - 3.1.2 + 3.1.5 1.7.7 hadoop2 - 0.7.1 - 1.9.40 - 1.4.0 + 0.9.3 + 1.7.3 + + 1.11.76 + + 0.10.2 - 4.3.2 + 4.5.2 + 4.4.4 3.1 3.4.1 - 2.10.5 - 2.10 - ${scala.version} - org.scala-lang + + 3.2.2 + 2.11.8 + 2.11 1.9.13 - 2.4.4 - 1.1.2 + 2.6.7 + 2.6.7.1 + 1.1.2.6 1.1.2 1.2.0-incubating 1.10 + 2.4 2.6 - 3.3.2 + 3.5 3.2.10 - 2.7.8 - 1.9 - 2.9 + 3.0.0 + 2.22.2 + 2.9.3 3.5.2 1.3.9 - 0.9.2 + 0.9.3 + 4.7 + 1.1 + 2.52.0 + 2.6 + 1.8 + 1.0.0 + 0.4.0 ${java.home} + + org.spark_project + + + ${project.build.directory}/scala-${scala.binary.version}/jars + + + prepare-package + none + compile compile - compile compile + compile compile test @@ -205,8 +218,6 @@ --> ${session.executionRootDirectory} - 64m - 512m 512m @@ -222,93 +233,6 @@ false - - apache-repo - Apache Repository - https://repository.apache.org/content/repositories/releases - - true - - - false - - - - jboss-repo - JBoss Repository - https://repository.jboss.org/nexus/content/repositories/releases - - true - - - false - - - - mqtt-repo - MQTT Repository - https://repo.eclipse.org/content/repositories/paho-releases - - true - - - false - - - - cloudera-repo - Cloudera Repository - https://repository.cloudera.com/artifactory/cloudera-repos - - true - - - false - - - - spark-hive-staging - Staging Repo for Hive 1.2.1 (Spark Version) - https://oss.sonatype.org/content/repositories/orgspark-project-1113 - - true - - - - mapr-repo - MapR Repository - http://repository.mapr.com/maven/ - - true - - - false - - - - - spring-releases - Spring Release Repository - https://repo.spring.io/libs-release - - false - - - false - - - - - twttr-repo - Twttr Repository - http://maven.twttr.com - - true - - - false - - @@ -324,8 +248,15 @@ org.spark-project.spark @@ -356,39 +287,32 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} ${project.version} - test + + + org.apache.spark + spark-tags_${scala.binary.version} + ${project.version} + test-jar com.twitter chill_${scala.binary.version} ${chill.version} - - - org.ow2.asm - asm - - - org.ow2.asm - asm-commons - - com.twitter chill-java ${chill.version} - - - org.ow2.asm - asm - - - org.ow2.asm - asm-commons - - + + + + org.apache.xbean + xbean-asm5-shaded + 4.4 - org.apache.commons commons-lang3 @@ -454,6 +407,11 @@ commons-lang ${commons-lang2.version} + + commons-io + commons-io + ${commons-io.version} + commons-codec commons-codec @@ -464,6 +422,11 @@ commons-math3 ${commons.math3.version} + + commons-collections + commons-collections + ${commons.collections.version} + org.apache.ivy ivy @@ -486,13 +449,23 @@ org.apache.httpcomponents - httpcore + httpmime ${commons.httpclient.version} + + org.apache.httpcomponents + httpcore + ${commons.httpcore.version} + + + org.fusesource.leveldbjni + leveldbjni-all + 1.8 + org.seleniumhq.selenium selenium-java - 2.42.2 + ${selenium.version} test @@ -505,6 +478,12 @@ + + org.seleniumhq.selenium + selenium-htmlunit-driver + ${selenium.version} + test + xml-apis @@ -533,7 +512,7 @@ org.slf4j jcl-over-slf4j ${slf4j.version} - + log4j @@ -553,9 +532,9 @@ ${hadoop.deps.scope} - net.jpountz.lz4 - lz4 - 1.3.0 + org.lz4 + lz4-java + 1.4.0 com.clearspring.analytics @@ -581,47 +560,9 @@ ${hadoop.deps.scope} - ${akka.group} - akka-actor_${scala.binary.version} - ${akka.version} - - - ${akka.group} - akka-remote_${scala.binary.version} - ${akka.version} - - - ${akka.group} - akka-slf4j_${scala.binary.version} - ${akka.version} - - - ${akka.group} - akka-testkit_${scala.binary.version} - ${akka.version} - - - ${akka.group} - akka-zeromq_${scala.binary.version} - ${akka.version} - - - ${akka.group} - akka-actor_${scala.binary.version} - - - - - org.apache.mesos - mesos - ${mesos.version} - ${mesos.classifier} - - - com.google.protobuf - protobuf-java - - + org.roaringbitmap + RoaringBitmap + 0.5.11 commons-net @@ -631,7 +572,12 @@ io.netty netty-all - 4.0.29.Final + 4.0.47.Final + + + io.netty + netty + 3.9.9.Final org.apache.derby @@ -665,9 +611,14 @@ com.fasterxml.jackson.core - jackson-databind + jackson-core ${fasterxml.jackson.version} + + com.fasterxml.jackson.core + jackson-databind + ${fasterxml.jackson.databind.version} + com.fasterxml.jackson.core jackson-annotations @@ -678,7 +629,7 @@ com.fasterxml.jackson.module jackson-module-scala_${scala.binary.version} - ${fasterxml.jackson.version} + ${fasterxml.jackson.databind.version} com.google.guava @@ -687,28 +638,72 @@ - com.sun.jersey + com.fasterxml.jackson.module + jackson-module-paranamer + ${fasterxml.jackson.version} + + + com.fasterxml.jackson.module + jackson-module-jaxb-annotations + ${fasterxml.jackson.version} + + + org.glassfish.jersey.core jersey-server ${jersey.version} - ${hadoop.deps.scope} - com.sun.jersey - jersey-core + org.glassfish.jersey.core + jersey-common + ${jersey.version} + + + org.glassfish.jersey.core + jersey-client + ${jersey.version} + + + org.glassfish.jersey.containers + jersey-container-servlet + ${jersey.version} + + + org.glassfish.jersey.containers + jersey-container-servlet-core ${jersey.version} - ${hadoop.deps.scope} - com.sun.jersey - jersey-json + org.glassfish.jersey + jersey-client ${jersey.version} + + + javax.ws.rs + javax.ws.rs-api + 2.0.1 + + + org.scalanlp + breeze_${scala.binary.version} + 0.13.2 + - stax - stax-api + junit + junit + + + org.apache.commons + commons-math3 + + org.json4s + json4s-jackson_${scala.binary.version} + 3.2.11 + org.scala-lang scala-compiler @@ -729,6 +724,11 @@ scala-actors ${scala.version} + + org.scala-lang.modules + scala-parser-combinators_${scala.binary.version} + 1.0.4 + org.scala-lang scalap @@ -737,25 +737,25 @@ org.scalatest scalatest_${scala.binary.version} - 2.2.1 + 3.0.3 test org.mockito mockito-core - 1.9.5 + 1.10.19 test org.scalacheck scalacheck_${scala.binary.version} - 1.11.3 + 1.13.5 test junit junit - 4.11 + 4.12 test @@ -776,6 +776,34 @@ 0.11 test + + com.spotify + docker-client + 5.0.2 + test + + + guava + com.google.guava + + + commons-logging + commons-logging + + + + + mysql + mysql-connector-java + 5.1.38 + test + + + org.postgresql + postgresql + 9.4.1207.jre7 + test + org.apache.curator curator-recipes @@ -786,6 +814,10 @@ org.jboss.netty netty + + jline + jline + @@ -846,6 +878,18 @@ junit junit + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -882,6 +926,14 @@ + + + org.apache.avro + avro-ipc + tests + ${avro.version} + test + org.apache.avro avro-mapred @@ -950,6 +1002,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -978,6 +1042,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -1007,6 +1083,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -1035,6 +1123,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -1063,6 +1163,18 @@ commons-logging commons-logging + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + @@ -1070,6 +1182,12 @@ zookeeper ${zookeeper.version} ${hadoop.deps.scope} + + + org.jboss.netty + netty + + org.codehaus.jackson @@ -1338,6 +1456,19 @@ commons-logging commons-logging + + org.codehaus.groovy + groovy-all + + + jline + jline + + + + org.json + json + @@ -1363,19 +1494,11 @@ ${hive.group} - hive-service - - - ${hive.group} - hive-shims - - - org.apache.httpcomponents - httpclient + hive-service - org.apache.httpcomponents - httpcore + ${hive.group} + hive-shims org.apache.curator @@ -1409,6 +1532,10 @@ commons-logging commons-logging + + org.codehaus.groovy + groovy-all + @@ -1503,50 +1630,22 @@ commons-logging commons-logging + + org.codehaus.groovy + groovy-all + - ${hive.group} - hive-service - ${hive.version} + net.sf.jpam + jpam ${hive.deps.scope} + ${jpam.version} - ${hive.group} - hive-common - - - ${hive.group} - hive-exec - - - ${hive.group} - hive-metastore - - - ${hive.group} - hive-shims - - - commons-codec - commons-codec - - - org.apache.curator - curator-framework - - - org.apache.curator - curator-recipes - - - org.apache.thrift - libfb303 - - - org.apache.thrift - libthrift + javax.servlet + servlet-api @@ -1596,6 +1695,48 @@ commons-logging commons-logging + + org.codehaus.groovy + groovy-all + + + + + org.apache.orc + orc-core + ${orc.version} + ${orc.classifier} + ${orc.deps.scope} + + + org.apache.hadoop + hadoop-common + + + org.apache.hive + hive-storage-api + + + + + org.apache.orc + orc-mapreduce + ${orc.version} + ${orc.classifier} + ${orc.deps.scope} + + + org.apache.hadoop + hadoop-common + + + org.apache.orc + orc-core + + + org.apache.hive + hive-storage-api + @@ -1691,6 +1832,10 @@ org.codehaus.janino janino + + org.codehaus.janino + commons-compiler + @@ -1728,6 +1873,11 @@ janino ${janino.version} + + org.codehaus.janino + commons-compiler + ${janino.version} + joda-time joda-time @@ -1748,14 +1898,6 @@ libthrift ${libthrift.version} - - org.apache.httpcomponents - httpclient - - - org.apache.httpcomponents - httpcore - org.slf4j slf4j-api @@ -1768,16 +1910,53 @@ ${libthrift.version} - org.apache.httpcomponents - httpclient + org.slf4j + slf4j-api + + + + org.antlr + antlr4-runtime + ${antlr4.version} + + + jline + jline + 2.12.1 + + + org.apache.commons + commons-crypto + ${commons-crypto.version} + - org.apache.httpcomponents - httpcore + net.java.dev.jna + jna + + + + com.thoughtworks.paranamer + paranamer + ${paranamer.version} + + + org.apache.arrow + arrow-vector + ${arrow.version} + - org.slf4j - slf4j-api + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-databind + + + io.netty + netty-handler @@ -1790,7 +1969,7 @@ org.apache.maven.plugins maven-enforcer-plugin - 1.4 + 3.0.0-M1 enforce-versions @@ -1805,6 +1984,21 @@ ${java.version} + + + + org.jboss.netty + org.codehaus.groovy + *:*_2.10 + + true + @@ -1813,11 +2007,12 @@ org.codehaus.mojo build-helper-maven-plugin - 1.9.1 + 3.0.0 net.alchim31.maven scala-maven-plugin + 3.2.2 @@ -1828,25 +2023,16 @@ scala-compile-first - process-resources compile scala-test-compile-first - process-test-resources testCompile - - attach-scaladocs - verify - - doc-jar - - ${scala.version} @@ -1856,12 +2042,12 @@ -unchecked -deprecation -feature + -explaintypes + -Yno-adapted-args -Xms1024m -Xmx1024m - -XX:PermSize=${PermGen} - -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize} @@ -1869,30 +2055,31 @@ ${java.version} -target ${java.version} - -Xlint:all,-serial,-path + -Xlint:all,-serial,-path,-try org.apache.maven.plugins maven-compiler-plugin - 3.3 + 3.7.0 ${java.version} ${java.version} - UTF-8 - 1024m - true - - -Xlint:all,-serial,-path - + true + true + + org.antlr + antlr4-maven-plugin + ${antlr4.version} + org.apache.maven.plugins maven-surefire-plugin - 2.18.1 + 2.20.1 @@ -1902,7 +2089,7 @@ **/*Suite.java ${project.build.directory}/surefire-reports - -Xmx3g -Xss4096k -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m + -Xmx3g -Xss4096k -XX:ReservedCodeCacheSize=${CodeCacheSize} ${test_classpath} 1 + ${scala.binary.version} 1 ${test.java.home} + file:src/test/resources/log4j.properties test true ${project.build.directory}/tmp @@ -1922,14 +2111,22 @@ false false false - true true + true src false ${test.exclude.tags} + + + test + + test + + + @@ -1941,7 +2138,7 @@ ${project.build.directory}/surefire-reports . SparkTestSuite.txt - -ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize} + -ea -Xmx3g -XX:ReservedCodeCacheSize=${CodeCacheSize} ${test_classpath} 1 + ${scala.binary.version} 1 ${test.java.home} + file:src/test/resources/log4j.properties test true ${project.build.directory}/tmp @@ -1961,7 +2160,6 @@ 1 false false - true true __not_used__ @@ -1980,17 +2178,17 @@ org.apache.maven.plugins maven-jar-plugin - 2.6 + 3.0.2 org.apache.maven.plugins maven-antrun-plugin - 1.8 + ${maven-antrun.version} org.apache.maven.plugins maven-source-plugin - 2.4 + 3.0.1 true @@ -2007,7 +2205,7 @@ org.apache.maven.plugins maven-clean-plugin - 2.6.1 + 3.0.0 @@ -2025,22 +2223,57 @@ org.apache.maven.plugins maven-javadoc-plugin - 2.10.3 + 3.0.0-M1 + + -Xdoclint:all -Xdoclint:-missing + + + example + a + Example: + + + note + a + Note: + + + group + X + + + tparam + X + + + constructor + X + + + todo + X + + + groupname + X + + + org.codehaus.mojo exec-maven-plugin - 1.4.0 + 1.6.0 org.apache.maven.plugins maven-assembly-plugin - 2.5.5 + 3.1.0 org.apache.maven.plugins maven-shade-plugin - 2.4.1 + 3.1.0 org.apache.maven.plugins @@ -2052,6 +2285,24 @@ maven-deploy-plugin 2.8.2 + + org.apache.maven.plugins + maven-dependency-plugin + 3.0.2 + + + default-cli + + build-classpath + + + + runtime + + + + @@ -2091,7 +2342,7 @@ org.apache.maven.plugins maven-antrun-plugin - [1.8,) + [${maven-antrun.version},) run @@ -2112,9 +2363,9 @@ org.apache.maven.plugins maven-dependency-plugin - 2.10 + generate-test-classpath test-compile build-classpath @@ -2124,6 +2375,17 @@ test_classpath + + copy-module-dependencies + ${build.copyDependenciesPhase} + + copy-dependencies + + + runtime + ${jars.target.dir} + + @@ -2138,42 +2400,41 @@ false - org.spark-project.spark:unused - org.eclipse.jetty:jetty-io org.eclipse.jetty:jetty-http + org.eclipse.jetty:jetty-proxy + org.eclipse.jetty:jetty-client org.eclipse.jetty:jetty-continuation org.eclipse.jetty:jetty-servlet + org.eclipse.jetty:jetty-servlets org.eclipse.jetty:jetty-plus org.eclipse.jetty:jetty-security org.eclipse.jetty:jetty-util org.eclipse.jetty:jetty-server com.google.guava:guava + org.jpmml:* org.eclipse.jetty - org.spark-project.jetty + ${spark.shade.packageName}.jetty org.eclipse.jetty.** com.google.common - org.spark-project.guava - - - com/google/common/base/Absent* - com/google/common/base/Function - com/google/common/base/Optional* - com/google/common/base/Present* - com/google/common/base/Supplier - + ${spark.shade.packageName}.guava + + + org.dmg.pmml + ${spark.shade.packageName}.dmg.pmml + + + org.jpmml + ${spark.shade.packageName}.jpmml @@ -2201,7 +2462,7 @@ org.scalastyle scalastyle-maven-plugin - 0.7.0 + 1.0.0 false true @@ -2222,6 +2483,34 @@ + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.17 + + true + ${basedir}/src/main/java,${basedir}/src/main/scala + ${basedir}/src/test/java + dev/checkstyle.xml + ${basedir}/target/checkstyle-output.xml + ${project.build.sourceEncoding} + ${project.reporting.outputEncoding} + + + + com.puppycrawl.tools + checkstyle + 8.2 + + + + + + check + + + + org.apache.maven.plugins @@ -2258,7 +2547,7 @@ prepare-test-jar - prepare-package + ${build.testJarPhase} test-jar @@ -2276,8 +2565,8 @@ sbt @@ -2287,6 +2576,11 @@ guava compile + + org.jpmml + pmml-model + compile + @@ -2294,7 +2588,7 @@ spark-ganglia-lgpl - extras/spark-ganglia-lgpl + external/spark-ganglia-lgpl @@ -2302,53 +2596,16 @@ kinesis-asl - extras/kinesis-asl - extras/kinesis-asl-assembly + external/kinesis-asl + external/kinesis-asl-assembly - java8-tests - - - - - org.apache.maven.plugins - maven-jar-plugin - - - - test-jar - - - - - - - + docker-integration-tests - extras/java8-tests + external/docker-integration-tests - - - - - doclint-java8-disable - - [1.8,) - - - - - - org.apache.maven.plugins - maven-javadoc-plugin - - -Xdoclint:all -Xdoclint:-missing - - - - @@ -2358,84 +2615,53 @@ --> - hadoop-1 - - 1.2.1 - 2.4.1 - 0.98.7-hadoop1 - hadoop1 - 1.8.8 - org.spark-project.akka - 2.3.4-spark - - - - - hadoop-2.2 - + hadoop-2.6 + - hadoop-2.3 + hadoop-2.7 - 2.3.0 - 0.9.3 + 2.7.3 + 2.7.1 - hadoop-2.4 - - 2.4.0 - 0.9.3 - + yarn + + resource-managers/yarn + common/network-yarn + - hadoop-2.6 - - 2.6.0 - 0.9.3 - 3.4.6 - 2.6.0 - + mesos + + resource-managers/mesos + - yarn + hive-thriftserver - yarn - network/yarn + sql/hive-thriftserver - hive-thriftserver + hadoop-cloud - sql/hive-thriftserver + hadoop-cloud - scala-2.10 - - !scala-2.11 - - - 2.10.5 - 2.10 - ${scala.version} - org.scala-lang - - - - - ${jline.groupid} - jline - ${jline.version} - - - + kafka-0-8 + + external/kafka-0-8 + external/kafka-0-8-assembly + @@ -2448,15 +2674,91 @@ + scala-2.11 - - scala-2.11 - + + + + scala-2.12 + + 2.12.3 + 2.12 + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + enforce-versions + + enforce + + + + + + *:*_2.11 + *:*_2.10 + + + + + + + + + + + + + + snapshots-and-staging - 2.11.7 - 2.11 + + https://repository.apache.org/content/groups/staging/ + https://repository.apache.org/content/repositories/snapshots/ + + + + ASF Staging + ${asf.staging} + + + ASF Snapshots + ${asf.snapshots} + + true + + + false + + + + + + + ASF Staging + ${asf.staging} + + + ASF Snapshots + ${asf.snapshots} + + true + + + false + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + org.apache.xbean + xbean-asm5-shaded @@ -157,30 +170,15 @@ + - scala-2.10 - - !scala-2.11 - - - - ${jline.groupid} - jline - ${jline.version} - - - - - - scala-2.11 - - scala-2.11 - + scala-2.12 - scala-2.11/src/main/scala - scala-2.11/src/test/scala + scala-2.12/src/main/scala + scala-2.12/src/test/scala + diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/Main.scala deleted file mode 100644 index 14b448d076d8..000000000000 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/Main.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.repl - -import scala.collection.mutable.Set - -object Main { - private var _interp: SparkILoop = _ - - def interp = _interp - - def interp_=(i: SparkILoop) { _interp = i } - - def main(args: Array[String]) { - _interp = new SparkILoop - _interp.process(args) - } -} diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala deleted file mode 100644 index 24fbbc12c08d..000000000000 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala +++ /dev/null @@ -1,46 +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.repl - -import scala.tools.nsc.{Settings, CompilerCommand} -import scala.Predef._ -import org.apache.spark.annotation.DeveloperApi - -/** - * Command class enabling Spark-specific command line options (provided by - * org.apache.spark.repl.SparkRunnerSettings). - * - * @example new SparkCommandLine(Nil).settings - * - * @param args The list of command line arguments - * @param settings The underlying settings to associate with this set of - * command-line options - */ -@DeveloperApi -class SparkCommandLine(args: List[String], override val settings: Settings) - extends CompilerCommand(args, settings) { - def this(args: List[String], error: String => Unit) { - this(args, new SparkRunnerSettings(error)) - } - - def this(args: List[String]) { - // scalastyle:off println - this(args, str => Console.println("Error: " + str)) - // scalastyle:on println - } -} diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala deleted file mode 100644 index 5fb378112ef9..000000000000 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala +++ /dev/null @@ -1,114 +0,0 @@ -// scalastyle:off - -/* NSC -- new Scala compiler - * Copyright 2005-2013 LAMP/EPFL - * @author Paul Phillips - */ - -package org.apache.spark.repl - -import scala.tools.nsc._ -import scala.tools.nsc.interpreter._ - -import scala.reflect.internal.util.BatchSourceFile -import scala.tools.nsc.ast.parser.Tokens.EOF - -import org.apache.spark.Logging - -private[repl] trait SparkExprTyper extends Logging { - val repl: SparkIMain - - import repl._ - import global.{ reporter => _, Import => _, _ } - import definitions._ - import syntaxAnalyzer.{ UnitParser, UnitScanner, token2name } - import naming.freshInternalVarName - - object codeParser extends { val global: repl.global.type = repl.global } with CodeHandlers[Tree] { - def applyRule[T](code: String, rule: UnitParser => T): T = { - reporter.reset() - val scanner = newUnitParser(code) - val result = rule(scanner) - - if (!reporter.hasErrors) - scanner.accept(EOF) - - result - } - - def defns(code: String) = stmts(code) collect { case x: DefTree => x } - def expr(code: String) = applyRule(code, _.expr()) - def stmts(code: String) = applyRule(code, _.templateStats()) - def stmt(code: String) = stmts(code).last // guaranteed nonempty - } - - /** Parse a line into a sequence of trees. Returns None if the input is incomplete. */ - def parse(line: String): Option[List[Tree]] = debugging(s"""parse("$line")""") { - var isIncomplete = false - reporter.withIncompleteHandler((_, _) => isIncomplete = true) { - val trees = codeParser.stmts(line) - if (reporter.hasErrors) { - Some(Nil) - } else if (isIncomplete) { - None - } else { - Some(trees) - } - } - } - // def parsesAsExpr(line: String) = { - // import codeParser._ - // (opt expr line).isDefined - // } - - def symbolOfLine(code: String): Symbol = { - def asExpr(): Symbol = { - val name = freshInternalVarName() - // Typing it with a lazy val would give us the right type, but runs - // into compiler bugs with things like existentials, so we compile it - // behind a def and strip the NullaryMethodType which wraps the expr. - val line = "def " + name + " = {\n" + code + "\n}" - - interpretSynthetic(line) match { - case IR.Success => - val sym0 = symbolOfTerm(name) - // drop NullaryMethodType - val sym = sym0.cloneSymbol setInfo afterTyper(sym0.info.finalResultType) - if (sym.info.typeSymbol eq UnitClass) NoSymbol else sym - case _ => NoSymbol - } - } - def asDefn(): Symbol = { - val old = repl.definedSymbolList.toSet - - interpretSynthetic(code) match { - case IR.Success => - repl.definedSymbolList filterNot old match { - case Nil => NoSymbol - case sym :: Nil => sym - case syms => NoSymbol.newOverloaded(NoPrefix, syms) - } - case _ => NoSymbol - } - } - beQuietDuring(asExpr()) orElse beQuietDuring(asDefn()) - } - - private var typeOfExpressionDepth = 0 - def typeOfExpression(expr: String, silent: Boolean = true): Type = { - if (typeOfExpressionDepth > 2) { - logDebug("Terminating typeOfExpression recursion for expression: " + expr) - return NoType - } - typeOfExpressionDepth += 1 - // Don't presently have a good way to suppress undesirable success output - // while letting errors through, so it is first trying it silently: if there - // is an error, and errors are desired, then it re-evaluates non-silently - // to induce the error message. - try beSilentDuring(symbolOfLine(expr).tpe) match { - case NoType if !silent => symbolOfLine(expr).tpe // generate error - case tpe => tpe - } - finally typeOfExpressionDepth -= 1 - } -} diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala deleted file mode 100644 index 955be17a73b8..000000000000 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package scala.tools.nsc - -import org.apache.spark.annotation.DeveloperApi - -// NOTE: Forced to be public (and in scala.tools.nsc package) to access the -// settings "explicitParentLoader" method - -/** - * Provides exposure for the explicitParentLoader method on settings instances. - */ -@DeveloperApi -object SparkHelper { - /** - * Retrieves the explicit parent loader for the provided settings. - * - * @param settings The settings whose explicit parent loader to retrieve - * - * @return The Optional classloader representing the explicit parent loader - */ - @DeveloperApi - def explicitParentLoader(settings: Settings) = settings.explicitParentLoader -} 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 deleted file mode 100644 index 304b1e8cdbed..000000000000 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ /dev/null @@ -1,1142 +0,0 @@ -// scalastyle:off - -/* NSC -- new Scala compiler - * Copyright 2005-2013 LAMP/EPFL - * @author Alexander Spoon - */ - -package org.apache.spark.repl - - -import java.net.URL - -import org.apache.spark.annotation.DeveloperApi - -import scala.reflect.io.AbstractFile -import scala.tools.nsc._ -import scala.tools.nsc.backend.JavaPlatform -import scala.tools.nsc.interpreter._ - -import scala.tools.nsc.interpreter.{Results => IR} -import Predef.{println => _, _} -import java.io.{BufferedReader, FileReader} -import java.net.URI -import java.util.concurrent.locks.ReentrantLock -import scala.sys.process.Process -import scala.tools.nsc.interpreter.session._ -import scala.util.Properties.{jdkHome, javaVersion} -import scala.tools.util.{Javap} -import scala.annotation.tailrec -import scala.collection.mutable.ListBuffer -import scala.concurrent.ops -import scala.tools.nsc.util._ -import scala.tools.nsc.interpreter._ -import scala.tools.nsc.io.{File, Directory} -import scala.reflect.NameTransformer._ -import scala.tools.nsc.util.ScalaClassLoader._ -import scala.tools.util._ -import scala.language.{implicitConversions, existentials, postfixOps} -import scala.reflect.{ClassTag, classTag} -import scala.tools.reflect.StdRuntimeTags._ - -import java.lang.{Class => jClass} -import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse} - -import org.apache.spark.Logging -import org.apache.spark.SparkConf -import org.apache.spark.SparkContext -import org.apache.spark.sql.SQLContext -import org.apache.spark.util.Utils - -/** The Scala interactive shell. It provides a read-eval-print loop - * around the Interpreter class. - * After instantiation, clients should call the main() method. - * - * If no in0 is specified, then input will come from the console, and - * the class will attempt to provide input editing feature such as - * input history. - * - * @author Moez A. Abdel-Gawad - * @author Lex Spoon - * @version 1.2 - */ -@DeveloperApi -class SparkILoop( - private val in0: Option[BufferedReader], - protected val out: JPrintWriter, - val master: Option[String] -) extends AnyRef with LoopCommands with SparkILoopInit with Logging { - def this(in0: BufferedReader, out: JPrintWriter, master: String) = this(Some(in0), out, Some(master)) - def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out, None) - def this() = this(None, new JPrintWriter(Console.out, true), None) - - private var in: InteractiveReader = _ // the input stream from which commands come - - // NOTE: Exposed in package for testing - private[repl] var settings: Settings = _ - - private[repl] var intp: SparkIMain = _ - - @deprecated("Use `intp` instead.", "2.9.0") def interpreter = intp - @deprecated("Use `intp` instead.", "2.9.0") def interpreter_= (i: SparkIMain): Unit = intp = i - - /** Having inherited the difficult "var-ness" of the repl instance, - * I'm trying to work around it by moving operations into a class from - * which it will appear a stable prefix. - */ - private def onIntp[T](f: SparkIMain => T): T = f(intp) - - class IMainOps[T <: SparkIMain](val intp: T) { - import intp._ - import global._ - - def printAfterTyper(msg: => String) = - intp.reporter printMessage afterTyper(msg) - - /** Strip NullaryMethodType artifacts. */ - private def replInfo(sym: Symbol) = { - sym.info match { - case NullaryMethodType(restpe) if sym.isAccessor => restpe - case info => info - } - } - def echoTypeStructure(sym: Symbol) = - printAfterTyper("" + deconstruct.show(replInfo(sym))) - - def echoTypeSignature(sym: Symbol, verbose: Boolean) = { - if (verbose) SparkILoop.this.echo("// Type signature") - printAfterTyper("" + replInfo(sym)) - - if (verbose) { - SparkILoop.this.echo("\n// Internal Type structure") - echoTypeStructure(sym) - } - } - } - implicit def stabilizeIMain(intp: SparkIMain) = new IMainOps[intp.type](intp) - - /** TODO - - * -n normalize - * -l label with case class parameter names - * -c complete - leave nothing out - */ - private def typeCommandInternal(expr: String, verbose: Boolean): Result = { - onIntp { intp => - val sym = intp.symbolOfLine(expr) - if (sym.exists) intp.echoTypeSignature(sym, verbose) - else "" - } - } - - // NOTE: Must be public for visibility - @DeveloperApi - var sparkContext: SparkContext = _ - var sqlContext: SQLContext = _ - - override def echoCommandMessage(msg: String) { - intp.reporter printMessage msg - } - - // def isAsync = !settings.Yreplsync.value - private[repl] def isAsync = false - // lazy val power = new Power(intp, new StdReplVals(this))(tagOfStdReplVals, classTag[StdReplVals]) - private def history = in.history - - /** The context class loader at the time this object was created */ - protected val originalClassLoader = Utils.getContextOrSparkClassLoader - - // classpath entries added via :cp - private var addedClasspath: String = "" - - /** A reverse list of commands to replay if the user requests a :replay */ - private var replayCommandStack: List[String] = Nil - - /** A list of commands to replay if the user requests a :replay */ - private def replayCommands = replayCommandStack.reverse - - /** Record a command for replay should the user request a :replay */ - private def addReplay(cmd: String) = replayCommandStack ::= cmd - - private def savingReplayStack[T](body: => T): T = { - val saved = replayCommandStack - try body - finally replayCommandStack = saved - } - private def savingReader[T](body: => T): T = { - val saved = in - try body - finally in = saved - } - - - private def sparkCleanUp(){ - echo("Stopping spark context.") - intp.beQuietDuring { - command("sc.stop()") - } - } - /** Close the interpreter and set the var to null. */ - private def closeInterpreter() { - if (intp ne null) { - sparkCleanUp() - intp.close() - intp = null - } - } - - class SparkILoopInterpreter extends SparkIMain(settings, out) { - outer => - - override private[repl] lazy val formatting = new Formatting { - def prompt = SparkILoop.this.prompt - } - override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader) - } - - /** - * Constructs a new interpreter. - */ - protected def createInterpreter() { - require(settings != null) - - if (addedClasspath != "") settings.classpath.append(addedClasspath) - val addedJars = - if (Utils.isWindows) { - // Strip any URI scheme prefix so we can add the correct path to the classpath - // e.g. file:/C:/my/path.jar -> C:/my/path.jar - SparkILoop.getAddedJars.map { jar => new URI(jar).getPath.stripPrefix("/") } - } else { - // We need new URI(jar).getPath here for the case that `jar` includes encoded white space (%20). - SparkILoop.getAddedJars.map { jar => new URI(jar).getPath } - } - // work around for Scala bug - val totalClassPath = addedJars.foldLeft( - settings.classpath.value)((l, r) => ClassPath.join(l, r)) - this.settings.classpath.value = totalClassPath - - intp = new SparkILoopInterpreter - } - - /** print a friendly help message */ - private def helpCommand(line: String): Result = { - if (line == "") helpSummary() - else uniqueCommand(line) match { - case Some(lc) => echo("\n" + lc.longHelp) - case _ => ambiguousError(line) - } - } - private def helpSummary() = { - val usageWidth = commands map (_.usageMsg.length) max - val formatStr = "%-" + usageWidth + "s %s %s" - - echo("All commands can be abbreviated, e.g. :he instead of :help.") - echo("Those marked with a * have more detailed help, e.g. :help imports.\n") - - commands foreach { cmd => - val star = if (cmd.hasLongHelp) "*" else " " - echo(formatStr.format(cmd.usageMsg, star, cmd.help)) - } - } - private def ambiguousError(cmd: String): Result = { - matchingCommands(cmd) match { - case Nil => echo(cmd + ": no such command. Type :help for help.") - case xs => echo(cmd + " is ambiguous: did you mean " + xs.map(":" + _.name).mkString(" or ") + "?") - } - Result(true, None) - } - private def matchingCommands(cmd: String) = commands filter (_.name startsWith cmd) - private def uniqueCommand(cmd: String): Option[LoopCommand] = { - // this lets us add commands willy-nilly and only requires enough command to disambiguate - matchingCommands(cmd) match { - case List(x) => Some(x) - // exact match OK even if otherwise appears ambiguous - case xs => xs find (_.name == cmd) - } - } - private var fallbackMode = false - - private def toggleFallbackMode() { - val old = fallbackMode - fallbackMode = !old - System.setProperty("spark.repl.fallback", fallbackMode.toString) - echo(s""" - |Switched ${if (old) "off" else "on"} fallback mode without restarting. - | If you have defined classes in the repl, it would - |be good to redefine them incase you plan to use them. If you still run - |into issues it would be good to restart the repl and turn on `:fallback` - |mode as first command. - """.stripMargin) - } - - /** Show the history */ - private lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") { - override def usage = "[num]" - def defaultLines = 20 - - def apply(line: String): Result = { - if (history eq NoHistory) - return "No history available." - - val xs = words(line) - val current = history.index - val count = try xs.head.toInt catch { case _: Exception => defaultLines } - val lines = history.asStrings takeRight count - val offset = current - lines.size + 1 - - for ((line, index) <- lines.zipWithIndex) - echo("%3d %s".format(index + offset, line)) - } - } - - // When you know you are most likely breaking into the middle - // of a line being typed. This softens the blow. - private[repl] def echoAndRefresh(msg: String) = { - echo("\n" + msg) - in.redrawLine() - } - private[repl] def echo(msg: String) = { - out println msg - out.flush() - } - private def echoNoNL(msg: String) = { - out print msg - out.flush() - } - - /** Search the history */ - private def searchHistory(_cmdline: String) { - val cmdline = _cmdline.toLowerCase - val offset = history.index - history.size + 1 - - for ((line, index) <- history.asStrings.zipWithIndex ; if line.toLowerCase contains cmdline) - echo("%d %s".format(index + offset, line)) - } - - private var currentPrompt = Properties.shellPromptString - - /** - * Sets the prompt string used by the REPL. - * - * @param prompt The new prompt string - */ - @DeveloperApi - def setPrompt(prompt: String) = currentPrompt = prompt - - /** - * Represents the current prompt string used by the REPL. - * - * @return The current prompt string - */ - @DeveloperApi - def prompt = currentPrompt - - import LoopCommand.{ cmd, nullary } - - /** Standard commands */ - private lazy val standardCommands = List( - cmd("cp", "", "add a jar or directory to the classpath", addClasspath), - cmd("help", "[command]", "print this summary or command-specific help", helpCommand), - historyCommand, - cmd("h?", "", "search the history", searchHistory), - cmd("imports", "[name name ...]", "show import history, identifying sources of names", importsCommand), - cmd("implicits", "[-v]", "show the implicits in scope", implicitsCommand), - cmd("javap", "", "disassemble a file or class name", javapCommand), - cmd("load", "", "load and interpret a Scala file", loadCommand), - nullary("paste", "enter paste mode: all input up to ctrl-D compiled together", pasteCommand), -// nullary("power", "enable power user mode", powerCmd), - nullary("quit", "exit the repl", () => Result(false, None)), - nullary("replay", "reset execution and replay all previous commands", replay), - nullary("reset", "reset the repl to its initial state, forgetting all session entries", resetCommand), - shCommand, - nullary("silent", "disable/enable automatic printing of results", verbosity), - nullary("fallback", """ - |disable/enable advanced repl changes, these fix some issues but may introduce others. - |This mode will be removed once these fixes stablize""".stripMargin, toggleFallbackMode), - cmd("type", "[-v] ", "display the type of an expression without evaluating it", typeCommand), - nullary("warnings", "show the suppressed warnings from the most recent line which had any", warningsCommand) - ) - - /** Power user commands */ - private lazy val powerCommands: List[LoopCommand] = List( - // cmd("phase", "", "set the implicit phase for power commands", phaseCommand) - ) - - // private def dumpCommand(): Result = { - // echo("" + power) - // history.asStrings takeRight 30 foreach echo - // in.redrawLine() - // } - // private def valsCommand(): Result = power.valsDescription - - private val typeTransforms = List( - "scala.collection.immutable." -> "immutable.", - "scala.collection.mutable." -> "mutable.", - "scala.collection.generic." -> "generic.", - "java.lang." -> "jl.", - "scala.runtime." -> "runtime." - ) - - private def importsCommand(line: String): Result = { - val tokens = words(line) - val handlers = intp.languageWildcardHandlers ++ intp.importHandlers - val isVerbose = tokens contains "-v" - - handlers.filterNot(_.importedSymbols.isEmpty).zipWithIndex foreach { - case (handler, idx) => - val (types, terms) = handler.importedSymbols partition (_.name.isTypeName) - val imps = handler.implicitSymbols - val found = tokens filter (handler importsSymbolNamed _) - val typeMsg = if (types.isEmpty) "" else types.size + " types" - val termMsg = if (terms.isEmpty) "" else terms.size + " terms" - val implicitMsg = if (imps.isEmpty) "" else imps.size + " are implicit" - val foundMsg = if (found.isEmpty) "" else found.mkString(" // imports: ", ", ", "") - val statsMsg = List(typeMsg, termMsg, implicitMsg) filterNot (_ == "") mkString ("(", ", ", ")") - - intp.reporter.printMessage("%2d) %-30s %s%s".format( - idx + 1, - handler.importString, - statsMsg, - foundMsg - )) - } - } - - private def implicitsCommand(line: String): Result = onIntp { intp => - import intp._ - import global._ - - def p(x: Any) = intp.reporter.printMessage("" + x) - - // If an argument is given, only show a source with that - // in its name somewhere. - val args = line split "\\s+" - val filtered = intp.implicitSymbolsBySource filter { - case (source, syms) => - (args contains "-v") || { - if (line == "") (source.fullName.toString != "scala.Predef") - else (args exists (source.name.toString contains _)) - } - } - - if (filtered.isEmpty) - return "No implicits have been imported other than those in Predef." - - filtered foreach { - case (source, syms) => - p("/* " + syms.size + " implicit members imported from " + source.fullName + " */") - - // This groups the members by where the symbol is defined - val byOwner = syms groupBy (_.owner) - val sortedOwners = byOwner.toList sortBy { case (owner, _) => afterTyper(source.info.baseClasses indexOf owner) } - - sortedOwners foreach { - case (owner, members) => - // Within each owner, we cluster results based on the final result type - // if there are more than a couple, and sort each cluster based on name. - // This is really just trying to make the 100 or so implicits imported - // by default into something readable. - val memberGroups: List[List[Symbol]] = { - val groups = members groupBy (_.tpe.finalResultType) toList - val (big, small) = groups partition (_._2.size > 3) - val xss = ( - (big sortBy (_._1.toString) map (_._2)) :+ - (small flatMap (_._2)) - ) - - xss map (xs => xs sortBy (_.name.toString)) - } - - val ownerMessage = if (owner == source) " defined in " else " inherited from " - p(" /* " + members.size + ownerMessage + owner.fullName + " */") - - memberGroups foreach { group => - group foreach (s => p(" " + intp.symbolDefString(s))) - p("") - } - } - p("") - } - } - - private def findToolsJar() = { - val jdkPath = Directory(jdkHome) - val jar = jdkPath / "lib" / "tools.jar" toFile; - - if (jar isFile) - Some(jar) - else if (jdkPath.isDirectory) - jdkPath.deepFiles find (_.name == "tools.jar") - else None - } - private def addToolsJarToLoader() = { - val cl = findToolsJar match { - case Some(tools) => ScalaClassLoader.fromURLs(Seq(tools.toURL), intp.classLoader) - case _ => intp.classLoader - } - if (Javap.isAvailable(cl)) { - logDebug(":javap available.") - cl - } - else { - logDebug(":javap unavailable: no tools.jar at " + jdkHome) - intp.classLoader - } - } - - private def newJavap() = new JavapClass(addToolsJarToLoader(), new SparkIMain.ReplStrippingWriter(intp)) { - override def tryClass(path: String): Array[Byte] = { - val hd :: rest = path split '.' toList; - // If there are dots in the name, the first segment is the - // key to finding it. - if (rest.nonEmpty) { - intp optFlatName hd match { - case Some(flat) => - val clazz = flat :: rest mkString NAME_JOIN_STRING - val bytes = super.tryClass(clazz) - if (bytes.nonEmpty) bytes - else super.tryClass(clazz + MODULE_SUFFIX_STRING) - case _ => super.tryClass(path) - } - } - else { - // Look for Foo first, then Foo$, but if Foo$ is given explicitly, - // we have to drop the $ to find object Foo, then tack it back onto - // the end of the flattened name. - def className = intp flatName path - def moduleName = (intp flatName path.stripSuffix(MODULE_SUFFIX_STRING)) + MODULE_SUFFIX_STRING - - val bytes = super.tryClass(className) - if (bytes.nonEmpty) bytes - else super.tryClass(moduleName) - } - } - } - // private lazy val javap = substituteAndLog[Javap]("javap", NoJavap)(newJavap()) - private lazy val javap = - try newJavap() - catch { case _: Exception => null } - - // Still todo: modules. - private def typeCommand(line0: String): Result = { - line0.trim match { - case "" => ":type [-v] " - case s if s startsWith "-v " => typeCommandInternal(s stripPrefix "-v " trim, true) - case s => typeCommandInternal(s, false) - } - } - - private def warningsCommand(): Result = { - if (intp.lastWarnings.isEmpty) - "Can't find any cached warnings." - else - intp.lastWarnings foreach { case (pos, msg) => intp.reporter.warning(pos, msg) } - } - - private def javapCommand(line: String): Result = { - if (javap == null) - ":javap unavailable, no tools.jar at %s. Set JDK_HOME.".format(jdkHome) - else if (javaVersion startsWith "1.7") - ":javap not yet working with java 1.7" - else if (line == "") - ":javap [-lcsvp] [path1 path2 ...]" - else - javap(words(line)) foreach { res => - if (res.isError) return "Failed: " + res.value - else res.show() - } - } - - private def wrapCommand(line: String): Result = { - def failMsg = "Argument to :wrap must be the name of a method with signature [T](=> T): T" - onIntp { intp => - import intp._ - import global._ - - words(line) match { - case Nil => - intp.executionWrapper match { - case "" => "No execution wrapper is set." - case s => "Current execution wrapper: " + s - } - case "clear" :: Nil => - intp.executionWrapper match { - case "" => "No execution wrapper is set." - case s => intp.clearExecutionWrapper() ; "Cleared execution wrapper." - } - case wrapper :: Nil => - intp.typeOfExpression(wrapper) match { - case PolyType(List(targ), MethodType(List(arg), restpe)) => - intp setExecutionWrapper intp.pathToTerm(wrapper) - "Set wrapper to '" + wrapper + "'" - case tp => - failMsg + "\nFound: " - } - case _ => failMsg - } - } - } - - private def pathToPhaseWrapper = intp.pathToTerm("$r") + ".phased.atCurrent" - // private def phaseCommand(name: String): Result = { - // val phased: Phased = power.phased - // import phased.NoPhaseName - - // if (name == "clear") { - // phased.set(NoPhaseName) - // intp.clearExecutionWrapper() - // "Cleared active phase." - // } - // else if (name == "") phased.get match { - // case NoPhaseName => "Usage: :phase (e.g. typer, erasure.next, erasure+3)" - // case ph => "Active phase is '%s'. (To clear, :phase clear)".format(phased.get) - // } - // else { - // val what = phased.parse(name) - // if (what.isEmpty || !phased.set(what)) - // "'" + name + "' does not appear to represent a valid phase." - // else { - // intp.setExecutionWrapper(pathToPhaseWrapper) - // val activeMessage = - // if (what.toString.length == name.length) "" + what - // else "%s (%s)".format(what, name) - - // "Active phase is now: " + activeMessage - // } - // } - // } - - /** - * Provides a list of available commands. - * - * @return The list of commands - */ - @DeveloperApi - def commands: List[LoopCommand] = standardCommands /*++ ( - if (isReplPower) powerCommands else Nil - )*/ - - private val replayQuestionMessage = - """|That entry seems to have slain the compiler. Shall I replay - |your session? I can re-run each line except the last one. - |[y/n] - """.trim.stripMargin - - private def crashRecovery(ex: Throwable): Boolean = { - echo(ex.toString) - ex match { - case _: NoSuchMethodError | _: NoClassDefFoundError => - echo("\nUnrecoverable error.") - throw ex - case _ => - def fn(): Boolean = - try in.readYesOrNo(replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() }) - catch { case _: RuntimeException => false } - - if (fn()) replay() - else echo("\nAbandoning crashed session.") - } - true - } - - /** The main read-eval-print loop for the repl. It calls - * command() for each line of input, and stops when - * command() returns false. - */ - private def loop() { - def readOneLine() = { - out.flush() - in readLine prompt - } - // return false if repl should exit - def processLine(line: String): Boolean = { - if (isAsync) { - if (!awaitInitialized()) return false - runThunks() - } - if (line eq null) false // assume null means EOF - else command(line) match { - case Result(false, _) => false - case Result(_, Some(finalLine)) => addReplay(finalLine) ; true - case _ => true - } - } - def innerLoop() { - val shouldContinue = try { - processLine(readOneLine()) - } catch {case t: Throwable => crashRecovery(t)} - if (shouldContinue) - innerLoop() - } - innerLoop() - } - - /** interpret all lines from a specified file */ - private def interpretAllFrom(file: File) { - savingReader { - savingReplayStack { - file applyReader { reader => - in = SimpleReader(reader, out, false) - echo("Loading " + file + "...") - loop() - } - } - } - } - - /** create a new interpreter and replay the given commands */ - private def replay() { - reset() - if (replayCommandStack.isEmpty) - echo("Nothing to replay.") - else for (cmd <- replayCommands) { - echo("Replaying: " + cmd) // flush because maybe cmd will have its own output - command(cmd) - echo("") - } - } - private def resetCommand() { - echo("Resetting repl state.") - if (replayCommandStack.nonEmpty) { - echo("Forgetting this session history:\n") - replayCommands foreach echo - echo("") - replayCommandStack = Nil - } - if (intp.namedDefinedTerms.nonEmpty) - echo("Forgetting all expression results and named terms: " + intp.namedDefinedTerms.mkString(", ")) - if (intp.definedTypes.nonEmpty) - echo("Forgetting defined types: " + intp.definedTypes.mkString(", ")) - - reset() - } - - private def reset() { - intp.reset() - // unleashAndSetPhase() - } - - /** fork a shell and run a command */ - private lazy val shCommand = new LoopCommand("sh", "run a shell command (result is implicitly => List[String])") { - override def usage = "" - def apply(line: String): Result = line match { - case "" => showUsage() - case _ => - val toRun = classOf[ProcessResult].getName + "(" + string2codeQuoted(line) + ")" - intp interpret toRun - () - } - } - - private def withFile(filename: String)(action: File => Unit) { - val f = File(filename) - - if (f.exists) action(f) - else echo("That file does not exist") - } - - private def loadCommand(arg: String) = { - var shouldReplay: Option[String] = None - withFile(arg)(f => { - interpretAllFrom(f) - shouldReplay = Some(":load " + arg) - }) - Result(true, shouldReplay) - } - - private def addAllClasspath(args: Seq[String]): Unit = { - var added = false - var totalClasspath = "" - for (arg <- args) { - val f = File(arg).normalize - if (f.exists) { - added = true - addedClasspath = ClassPath.join(addedClasspath, f.path) - totalClasspath = ClassPath.join(settings.classpath.value, addedClasspath) - intp.addUrlsToClassPath(f.toURI.toURL) - sparkContext.addJar(f.toURI.toURL.getPath) - } - } - } - - private def addClasspath(arg: String): Unit = { - val f = File(arg).normalize - if (f.exists) { - addedClasspath = ClassPath.join(addedClasspath, f.path) - intp.addUrlsToClassPath(f.toURI.toURL) - sparkContext.addJar(f.toURI.toURL.getPath) - echo("Added '%s'. Your new classpath is:\n\"%s\"".format(f.path, intp.global.classPath.asClasspathString)) - } - else echo("The path '" + f + "' doesn't seem to exist.") - } - - - private def powerCmd(): Result = { - if (isReplPower) "Already in power mode." - else enablePowerMode(false) - } - - private[repl] def enablePowerMode(isDuringInit: Boolean) = { - // replProps.power setValue true - // unleashAndSetPhase() - // asyncEcho(isDuringInit, power.banner) - } - // private def unleashAndSetPhase() { -// if (isReplPower) { -// // power.unleash() -// // Set the phase to "typer" -// intp beSilentDuring phaseCommand("typer") -// } -// } - - private def asyncEcho(async: Boolean, msg: => String) { - if (async) asyncMessage(msg) - else echo(msg) - } - - private def verbosity() = { - // val old = intp.printResults - // intp.printResults = !old - // echo("Switched " + (if (old) "off" else "on") + " result printing.") - } - - /** Run one command submitted by the user. Two values are returned: - * (1) whether to keep running, (2) the line to record for replay, - * if any. */ - private[repl] def command(line: String): Result = { - if (line startsWith ":") { - val cmd = line.tail takeWhile (x => !x.isWhitespace) - uniqueCommand(cmd) match { - case Some(lc) => lc(line.tail stripPrefix cmd dropWhile (_.isWhitespace)) - case _ => ambiguousError(cmd) - } - } - else if (intp.global == null) Result(false, None) // Notice failure to create compiler - else Result(true, interpretStartingWith(line)) - } - - private def readWhile(cond: String => Boolean) = { - Iterator continually in.readLine("") takeWhile (x => x != null && cond(x)) - } - - private def pasteCommand(): Result = { - echo("// Entering paste mode (ctrl-D to finish)\n") - val code = readWhile(_ => true) mkString "\n" - echo("\n// Exiting paste mode, now interpreting.\n") - intp interpret code - () - } - - private object paste extends Pasted { - val ContinueString = " | " - val PromptString = "scala> " - - def interpret(line: String): Unit = { - echo(line.trim) - intp interpret line - echo("") - } - - def transcript(start: String) = { - echo("\n// Detected repl transcript paste: ctrl-D to finish.\n") - apply(Iterator(start) ++ readWhile(_.trim != PromptString.trim)) - } - } - import paste.{ ContinueString, PromptString } - - /** Interpret expressions starting with the first line. - * Read lines until a complete compilation unit is available - * or until a syntax error has been seen. If a full unit is - * read, go ahead and interpret it. Return the full string - * to be recorded for replay, if any. - */ - private def interpretStartingWith(code: String): Option[String] = { - // signal completion non-completion input has been received - in.completion.resetVerbosity() - - def reallyInterpret = { - val reallyResult = intp.interpret(code) - (reallyResult, reallyResult match { - case IR.Error => None - case IR.Success => Some(code) - case IR.Incomplete => - if (in.interactive && code.endsWith("\n\n")) { - echo("You typed two blank lines. Starting a new command.") - None - } - else in.readLine(ContinueString) match { - case null => - // we know compilation is going to fail since we're at EOF and the - // parser thinks the input is still incomplete, but since this is - // a file being read non-interactively we want to fail. So we send - // it straight to the compiler for the nice error message. - intp.compileString(code) - None - - case line => interpretStartingWith(code + "\n" + line) - } - }) - } - - /** Here we place ourselves between the user and the interpreter and examine - * the input they are ostensibly submitting. We intervene in several cases: - * - * 1) If the line starts with "scala> " it is assumed to be an interpreter paste. - * 2) If the line starts with "." (but not ".." or "./") it is treated as an invocation - * on the previous result. - * 3) If the Completion object's execute returns Some(_), we inject that value - * and avoid the interpreter, as it's likely not valid scala code. - */ - if (code == "") None - else if (!paste.running && code.trim.startsWith(PromptString)) { - paste.transcript(code) - None - } - else if (Completion.looksLikeInvocation(code) && intp.mostRecentVar != "") { - interpretStartingWith(intp.mostRecentVar + code) - } - else if (code.trim startsWith "//") { - // line comment, do nothing - None - } - else - reallyInterpret._2 - } - - // runs :load `file` on any files passed via -i - private def loadFiles(settings: Settings) = settings match { - case settings: SparkRunnerSettings => - for (filename <- settings.loadfiles.value) { - val cmd = ":load " + filename - command(cmd) - addReplay(cmd) - echo("") - } - case _ => - } - - /** Tries to create a JLineReader, falling back to SimpleReader: - * unless settings or properties are such that it should start - * with SimpleReader. - */ - private def chooseReader(settings: Settings): InteractiveReader = { - if (settings.Xnojline.value || Properties.isEmacsShell) - SimpleReader() - else try new SparkJLineReader( - if (settings.noCompletion.value) NoCompletion - else new SparkJLineCompletion(intp) - ) - catch { - case ex @ (_: Exception | _: NoClassDefFoundError) => - echo("Failed to created SparkJLineReader: " + ex + "\nFalling back to SimpleReader.") - SimpleReader() - } - } - - private val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe - private val m = u.runtimeMirror(Utils.getSparkClassLoader) - private def tagOfStaticClass[T: ClassTag]: u.TypeTag[T] = - u.TypeTag[T]( - m, - new TypeCreator { - def apply[U <: ApiUniverse with Singleton](m: Mirror[U]): U # Type = - m.staticClass(classTag[T].runtimeClass.getName).toTypeConstructor.asInstanceOf[U # Type] - }) - - private def process(settings: Settings): Boolean = savingContextLoader { - if (getMaster() == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") - - this.settings = settings - createInterpreter() - - // sets in to some kind of reader depending on environmental cues - in = in0 match { - case Some(reader) => SimpleReader(reader, out, true) - case None => - // some post-initialization - chooseReader(settings) match { - case x: SparkJLineReader => addThunk(x.consoleReader.postInit) ; x - case x => x - } - } - lazy val tagOfSparkIMain = tagOfStaticClass[org.apache.spark.repl.SparkIMain] - // Bind intp somewhere out of the regular namespace where - // we can get at it in generated code. - addThunk(intp.quietBind(NamedParam[SparkIMain]("$intp", intp)(tagOfSparkIMain, classTag[SparkIMain]))) - addThunk({ - import scala.tools.nsc.io._ - import Properties.userHome - import scala.compat.Platform.EOL - val autorun = replProps.replAutorunCode.option flatMap (f => io.File(f).safeSlurp()) - if (autorun.isDefined) intp.quietRun(autorun.get) - }) - - addThunk(printWelcome()) - addThunk(initializeSpark()) - - // it is broken on startup; go ahead and exit - if (intp.reporter.hasErrors) - return false - - // This is about the illusion of snappiness. We call initialize() - // which spins off a separate thread, then print the prompt and try - // our best to look ready. The interlocking lazy vals tend to - // inter-deadlock, so we break the cycle with a single asynchronous - // message to an rpcEndpoint. - if (isAsync) { - intp initialize initializedCallback() - createAsyncListener() // listens for signal to run postInitialization - } - else { - intp.initializeSynchronous() - postInitialization() - } - // printWelcome() - - loadFiles(settings) - - try loop() - catch AbstractOrMissingHandler() - finally closeInterpreter() - - true - } - - // NOTE: Must be public for visibility - @DeveloperApi - def createSparkContext(): SparkContext = { - val execUri = System.getenv("SPARK_EXECUTOR_URI") - val jars = SparkILoop.getAddedJars - val conf = new SparkConf() - .setMaster(getMaster()) - .setJars(jars) - .set("spark.repl.class.uri", intp.classServerUri) - .setIfMissing("spark.app.name", "Spark shell") - if (execUri != null) { - conf.set("spark.executor.uri", execUri) - } - sparkContext = new SparkContext(conf) - logInfo("Created spark context..") - sparkContext - } - - @DeveloperApi - def createSQLContext(): SQLContext = { - val name = "org.apache.spark.sql.hive.HiveContext" - val loader = Utils.getContextOrSparkClassLoader - try { - sqlContext = loader.loadClass(name).getConstructor(classOf[SparkContext]) - .newInstance(sparkContext).asInstanceOf[SQLContext] - logInfo("Created sql context (with Hive support)..") - } - catch { - case _: java.lang.ClassNotFoundException | _: java.lang.NoClassDefFoundError => - sqlContext = new SQLContext(sparkContext) - logInfo("Created sql context..") - } - sqlContext - } - - private def getMaster(): String = { - val master = this.master match { - case Some(m) => m - case None => - val envMaster = sys.env.get("MASTER") - val propMaster = sys.props.get("spark.master") - propMaster.orElse(envMaster).getOrElse("local[*]") - } - master - } - - /** process command-line arguments and do as they request */ - def process(args: Array[String]): Boolean = { - val command = new SparkCommandLine(args.toList, msg => echo(msg)) - def neededHelp(): String = - (if (command.settings.help.value) command.usageMsg + "\n" else "") + - (if (command.settings.Xhelp.value) command.xusageMsg + "\n" else "") - - // if they asked for no help and command is valid, we call the real main - neededHelp() match { - case "" => command.ok && process(command.settings) - case help => echoNoNL(help) ; true - } - } - - @deprecated("Use `process` instead", "2.9.0") - private def main(settings: Settings): Unit = process(settings) -} - -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") - 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) - } - - // Designed primarily for use by test code: take a String with a - // bunch of code, and prints out a transcript of what it would look - // like if you'd just typed it into the repl. - private[repl] def runForTranscript(code: String, settings: Settings): String = { - import java.io.{ BufferedReader, StringReader, OutputStreamWriter } - - stringFromStream { ostream => - Console.withOut(ostream) { - val output = new JPrintWriter(new OutputStreamWriter(ostream), true) { - override def write(str: String) = { - // completely skip continuation lines - if (str forall (ch => ch.isWhitespace || ch == '|')) () - // print a newline on empty scala prompts - else if ((str contains '\n') && (str.trim == "scala> ")) super.write("\n") - else super.write(str) - } - } - val input = new BufferedReader(new StringReader(code)) { - override def readLine(): String = { - val s = super.readLine() - // helping out by printing the line being interpreted. - if (s != null) - // scalastyle:off println - output.println(s) - // scalastyle:on println - s - } - } - val repl = new SparkILoop(input, output) - - if (settings.classpath.isDefault) - settings.classpath.value = sys.props("java.class.path") - - getAddedJars.map(jar => new URI(jar).getPath).foreach(settings.classpath.append(_)) - - repl process settings - } - } - } - - /** Creates an interpreter loop with default settings and feeds - * the given code to it as input. - */ - private[repl] def run(code: String, sets: Settings = new Settings): String = { - import java.io.{ BufferedReader, StringReader, OutputStreamWriter } - - stringFromStream { ostream => - Console.withOut(ostream) { - val input = new BufferedReader(new StringReader(code)) - val output = new JPrintWriter(new OutputStreamWriter(ostream), true) - val repl = new ILoop(input, output) - - if (sets.classpath.isDefault) - sets.classpath.value = sys.props("java.class.path") - - repl process sets - } - } - } - private[repl] def run(lines: List[String]): String = run(lines map (_ + "\n") mkString) -} 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 deleted file mode 100644 index bd3314d94eed..000000000000 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ /dev/null @@ -1,162 +0,0 @@ -// scalastyle:off - -/* NSC -- new Scala compiler - * Copyright 2005-2013 LAMP/EPFL - * @author Paul Phillips - */ - -package org.apache.spark.repl - -import scala.tools.nsc._ -import scala.tools.nsc.interpreter._ - -import scala.reflect.internal.util.Position -import scala.util.control.Exception.ignoring -import scala.tools.nsc.util.stackTraceString - -import org.apache.spark.SPARK_VERSION - -/** - * Machinery for the asynchronous initialization of the repl. - */ -private[repl] trait SparkILoopInit { - self: SparkILoop => - - /** Print a welcome message */ - def printWelcome() { - echo("""Welcome to - ____ __ - / __/__ ___ _____/ /__ - _\ \/ _ \/ _ `/ __/ '_/ - /___/ .__/\_,_/_/ /_/\_\ version %s - /_/ -""".format(SPARK_VERSION)) - import Properties._ - val welcomeMsg = "Using Scala %s (%s, Java %s)".format( - versionString, javaVmName, javaVersion) - echo(welcomeMsg) - echo("Type in expressions to have them evaluated.") - echo("Type :help for more information.") - } - - protected def asyncMessage(msg: String) { - if (isReplInfo || isReplPower) - echoAndRefresh(msg) - } - - private val initLock = new java.util.concurrent.locks.ReentrantLock() - private val initCompilerCondition = initLock.newCondition() // signal the compiler is initialized - private val initLoopCondition = initLock.newCondition() // signal the whole repl is initialized - private val initStart = System.nanoTime - - private def withLock[T](body: => T): T = { - initLock.lock() - try body - finally initLock.unlock() - } - // a condition used to ensure serial access to the compiler. - @volatile private var initIsComplete = false - @volatile private var initError: String = null - private def elapsed() = "%.3f".format((System.nanoTime - initStart).toDouble / 1000000000L) - - // the method to be called when the interpreter is initialized. - // Very important this method does nothing synchronous (i.e. do - // not try to use the interpreter) because until it returns, the - // repl's lazy val `global` is still locked. - protected def initializedCallback() = withLock(initCompilerCondition.signal()) - - // Spins off a thread which awaits a single message once the interpreter - // has been initialized. - protected def createAsyncListener() = { - io.spawn { - withLock(initCompilerCondition.await()) - asyncMessage("[info] compiler init time: " + elapsed() + " s.") - postInitialization() - } - } - - // called from main repl loop - protected def awaitInitialized(): Boolean = { - if (!initIsComplete) - withLock { while (!initIsComplete) initLoopCondition.await() } - if (initError != null) { - // scalastyle:off println - println(""" - |Failed to initialize the REPL due to an unexpected error. - |This is a bug, please, report it along with the error diagnostics printed below. - |%s.""".stripMargin.format(initError) - ) - // scalastyle:on println - false - } else true - } - // private def warningsThunks = List( - // () => intp.bind("lastWarnings", "" + typeTag[List[(Position, String)]], intp.lastWarnings _), - // ) - - protected def postInitThunks = List[Option[() => Unit]]( - Some(intp.setContextClassLoader _), - if (isReplPower) Some(() => enablePowerMode(true)) else None - ).flatten - // ++ ( - // warningsThunks - // ) - // called once after init condition is signalled - protected def postInitialization() { - try { - postInitThunks foreach (f => addThunk(f())) - runThunks() - } catch { - case ex: Throwable => - initError = stackTraceString(ex) - throw ex - } finally { - initIsComplete = true - - if (isAsync) { - asyncMessage("[info] total init time: " + elapsed() + " s.") - withLock(initLoopCondition.signal()) - } - } - } - - def initializeSpark() { - intp.beQuietDuring { - command(""" - @transient val sc = { - val _sc = org.apache.spark.repl.Main.interp.createSparkContext() - println("Spark context available as sc.") - _sc - } - """) - command(""" - @transient val sqlContext = { - val _sqlContext = org.apache.spark.repl.Main.interp.createSQLContext() - println("SQL context available as sqlContext.") - _sqlContext - } - """) - command("import org.apache.spark.SparkContext._") - command("import sqlContext.implicits._") - command("import sqlContext.sql") - command("import org.apache.spark.sql.functions._") - } - } - - // code to be executed only after the interpreter is initialized - // and the lazy val `global` can be accessed without risk of deadlock. - private var pendingThunks: List[() => Unit] = Nil - protected def addThunk(body: => Unit) = synchronized { - pendingThunks :+= (() => body) - } - protected def runThunks(): Unit = synchronized { - if (pendingThunks.nonEmpty) - logDebug("Clearing " + pendingThunks.size + " thunks.") - - while (pendingThunks.nonEmpty) { - val thunk = pendingThunks.head - pendingThunks = pendingThunks.tail - thunk() - } - } -} diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala deleted file mode 100644 index 4ee605fd7f11..000000000000 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ /dev/null @@ -1,1821 +0,0 @@ -// scalastyle:off - -/* NSC -- new Scala compiler - * Copyright 2005-2013 LAMP/EPFL - * @author Martin Odersky - */ - -package org.apache.spark.repl - -import java.io.File - -import scala.tools.nsc._ -import scala.tools.nsc.backend.JavaPlatform -import scala.tools.nsc.interpreter._ - -import Predef.{ println => _, _ } -import scala.tools.nsc.util.{MergedClassPath, stringFromWriter, ScalaClassLoader, stackTraceString} -import scala.reflect.internal.util._ -import java.net.URL -import scala.sys.BooleanProp -import io.{AbstractFile, PlainFile, VirtualDirectory} - -import reporters._ -import symtab.Flags -import scala.reflect.internal.Names -import scala.tools.util.PathResolver -import ScalaClassLoader.URLClassLoader -import scala.tools.nsc.util.Exceptional.unwrap -import scala.collection.{ mutable, immutable } -import scala.util.control.Exception.{ ultimately } -import SparkIMain._ -import java.util.concurrent.Future -import typechecker.Analyzer -import scala.language.implicitConversions -import scala.reflect.runtime.{ universe => ru } -import scala.reflect.{ ClassTag, classTag } -import scala.tools.reflect.StdRuntimeTags._ -import scala.util.control.ControlThrowable - -import org.apache.spark.{Logging, HttpServer, SecurityManager, SparkConf} -import org.apache.spark.util.Utils -import org.apache.spark.annotation.DeveloperApi - -// /** directory to save .class files to */ -// private class ReplVirtualDirectory(out: JPrintWriter) extends VirtualDirectory("((memory))", None) { -// private def pp(root: AbstractFile, indentLevel: Int) { -// val spaces = " " * indentLevel -// out.println(spaces + root.name) -// if (root.isDirectory) -// root.toList sortBy (_.name) foreach (x => pp(x, indentLevel + 1)) -// } -// // print the contents hierarchically -// def show() = pp(this, 0) -// } - - /** An interpreter for Scala code. - * - * The main public entry points are compile(), interpret(), and bind(). - * The compile() method loads a complete Scala file. The interpret() method - * executes one line of Scala code at the request of the user. The bind() - * method binds an object to a variable that can then be used by later - * interpreted code. - * - * The overall approach is based on compiling the requested code and then - * using a Java classloader and Java reflection to run the code - * and access its results. - * - * In more detail, a single compiler instance is used - * to accumulate all successfully compiled or interpreted Scala code. To - * "interpret" a line of code, the compiler generates a fresh object that - * includes the line of code and which has public member(s) to export - * all variables defined by that code. To extract the result of an - * interpreted line to show the user, a second "result object" is created - * which imports the variables exported by the above object and then - * exports members called "$eval" and "$print". To accomodate user expressions - * that read from variables or methods defined in previous statements, "import" - * statements are used. - * - * This interpreter shares the strengths and weaknesses of using the - * full compiler-to-Java. The main strength is that interpreted code - * behaves exactly as does compiled code, including running at full speed. - * The main weakness is that redefining classes and methods is not handled - * properly, because rebinding at the Java level is technically difficult. - * - * @author Moez A. Abdel-Gawad - * @author Lex Spoon - */ - @DeveloperApi - class SparkIMain( - initialSettings: Settings, - val out: JPrintWriter, - propagateExceptions: Boolean = false) - extends SparkImports with Logging { imain => - - private val conf = new SparkConf() - - private val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") - /** Local directory to save .class files too */ - private lazy val outputDir = { - val tmp = System.getProperty("java.io.tmpdir") - val rootDir = conf.get("spark.repl.classdir", tmp) - Utils.createTempDir(rootDir) - } - if (SPARK_DEBUG_REPL) { - echo("Output directory: " + outputDir) - } - - /** - * Returns the path to the output directory containing all generated - * class files that will be served by the REPL class server. - */ - @DeveloperApi - lazy val getClassOutputDirectory = outputDir - - private val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles - /** Jetty server that will serve our classes to worker nodes */ - private val classServerPort = conf.getInt("spark.replClassServer.port", 0) - private val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf), classServerPort, "HTTP class server") - private var currentSettings: Settings = initialSettings - private var printResults = true // whether to print result lines - private var totalSilence = false // whether to print anything - private var _initializeComplete = false // compiler is initialized - private var _isInitialized: Future[Boolean] = null // set up initialization future - private var bindExceptions = true // whether to bind the lastException variable - private var _executionWrapper = "" // code to be wrapped around all lines - - - // Start the classServer and store its URI in a spark system property - // (which will be passed to executors so that they can connect to it) - classServer.start() - if (SPARK_DEBUG_REPL) { - echo("Class server started, URI = " + classServer.uri) - } - - /** - * URI of the class server used to feed REPL compiled classes. - * - * @return The string representing the class server uri - */ - @DeveloperApi - def classServerUri = classServer.uri - - /** We're going to go to some trouble to initialize the compiler asynchronously. - * It's critical that nothing call into it until it's been initialized or we will - * run into unrecoverable issues, but the perceived repl startup time goes - * through the roof if we wait for it. So we initialize it with a future and - * use a lazy val to ensure that any attempt to use the compiler object waits - * on the future. - */ - private var _classLoader: AbstractFileClassLoader = null // active classloader - private val _compiler: Global = newCompiler(settings, reporter) // our private compiler - - private trait ExposeAddUrl extends URLClassLoader { def addNewUrl(url: URL) = this.addURL(url) } - private var _runtimeClassLoader: URLClassLoader with ExposeAddUrl = null // wrapper exposing addURL - - private val nextReqId = { - var counter = 0 - () => { counter += 1 ; counter } - } - - private def compilerClasspath: Seq[URL] = ( - if (isInitializeComplete) global.classPath.asURLs - else new PathResolver(settings).result.asURLs // the compiler's classpath - ) - // NOTE: Exposed to repl package since accessed indirectly from SparkIMain - private[repl] def settings = currentSettings - private def mostRecentLine = prevRequestList match { - case Nil => "" - case req :: _ => req.originalLine - } - // Run the code body with the given boolean settings flipped to true. - private def withoutWarnings[T](body: => T): T = beQuietDuring { - val saved = settings.nowarn.value - if (!saved) - settings.nowarn.value = true - - try body - finally if (!saved) settings.nowarn.value = false - } - - /** construct an interpreter that reports to Console */ - def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true)) - def this() = this(new Settings()) - - private lazy val repllog: Logger = new Logger { - val out: JPrintWriter = imain.out - val isInfo: Boolean = BooleanProp keyExists "scala.repl.info" - val isDebug: Boolean = BooleanProp keyExists "scala.repl.debug" - val isTrace: Boolean = BooleanProp keyExists "scala.repl.trace" - } - private[repl] lazy val formatting: Formatting = new Formatting { - val prompt = Properties.shellPromptString - } - - // NOTE: Exposed to repl package since used by SparkExprTyper and SparkILoop - private[repl] lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this) - - /** - * Determines if errors were reported (typically during compilation). - * - * @note This is not for runtime errors - * - * @return True if had errors, otherwise false - */ - @DeveloperApi - def isReportingErrors = reporter.hasErrors - - import formatting._ - import reporter.{ printMessage, withoutTruncating } - - // This exists mostly because using the reporter too early leads to deadlock. - private def echo(msg: String) { Console println msg } - private def _initSources = List(new BatchSourceFile("", "class $repl_$init { }")) - private def _initialize() = { - try { - // todo. if this crashes, REPL will hang - new _compiler.Run() compileSources _initSources - _initializeComplete = true - true - } - catch AbstractOrMissingHandler() - } - private def tquoted(s: String) = "\"\"\"" + s + "\"\"\"" - - // argument is a thunk to execute after init is done - // NOTE: Exposed to repl package since used by SparkILoop - private[repl] def initialize(postInitSignal: => Unit) { - synchronized { - if (_isInitialized == null) { - _isInitialized = io.spawn { - try _initialize() - finally postInitSignal - } - } - } - } - - /** - * Initializes the underlying compiler/interpreter in a blocking fashion. - * - * @note Must be executed before using SparkIMain! - */ - @DeveloperApi - def initializeSynchronous(): Unit = { - if (!isInitializeComplete) { - _initialize() - assert(global != null, global) - } - } - private def isInitializeComplete = _initializeComplete - - /** the public, go through the future compiler */ - - /** - * The underlying compiler used to generate ASTs and execute code. - */ - @DeveloperApi - lazy val global: Global = { - if (isInitializeComplete) _compiler - else { - // If init hasn't been called yet you're on your own. - if (_isInitialized == null) { - logWarning("Warning: compiler accessed before init set up. Assuming no postInit code.") - initialize(()) - } - // // blocks until it is ; false means catastrophic failure - if (_isInitialized.get()) _compiler - else null - } - } - @deprecated("Use `global` for access to the compiler instance.", "2.9.0") - private lazy val compiler: global.type = global - - import global._ - import definitions.{ScalaPackage, JavaLangPackage, termMember, typeMember} - import rootMirror.{RootClass, getClassIfDefined, getModuleIfDefined, getRequiredModule, getRequiredClass} - - private implicit class ReplTypeOps(tp: Type) { - def orElse(other: => Type): Type = if (tp ne NoType) tp else other - def andAlso(fn: Type => Type): Type = if (tp eq NoType) tp else fn(tp) - } - - // TODO: If we try to make naming a lazy val, we run into big time - // scalac unhappiness with what look like cycles. It has not been easy to - // reduce, but name resolution clearly takes different paths. - // NOTE: Exposed to repl package since used by SparkExprTyper - private[repl] object naming extends { - val global: imain.global.type = imain.global - } with Naming { - // make sure we don't overwrite their unwisely named res3 etc. - def freshUserTermName(): TermName = { - val name = newTermName(freshUserVarName()) - if (definedNameMap contains name) freshUserTermName() - else name - } - def isUserTermName(name: Name) = isUserVarName("" + name) - def isInternalTermName(name: Name) = isInternalVarName("" + name) - } - import naming._ - - // NOTE: Exposed to repl package since used by SparkILoop - private[repl] object deconstruct extends { - val global: imain.global.type = imain.global - } with StructuredTypeStrings - - // NOTE: Exposed to repl package since used by SparkImports - private[repl] lazy val memberHandlers = new { - val intp: imain.type = imain - } with SparkMemberHandlers - import memberHandlers._ - - /** - * Suppresses overwriting print results during the operation. - * - * @param body The block to execute - * @tparam T The return type of the block - * - * @return The result from executing the block - */ - @DeveloperApi - def beQuietDuring[T](body: => T): T = { - val saved = printResults - printResults = false - try body - finally printResults = saved - } - - /** - * Completely masks all output during the operation (minus JVM standard - * out and error). - * - * @param operation The block to execute - * @tparam T The return type of the block - * - * @return The result from executing the block - */ - @DeveloperApi - def beSilentDuring[T](operation: => T): T = { - val saved = totalSilence - totalSilence = true - try operation - finally totalSilence = saved - } - - // NOTE: Exposed to repl package since used by SparkILoop - private[repl] def quietRun[T](code: String) = beQuietDuring(interpret(code)) - - private def logAndDiscard[T](label: String, alt: => T): PartialFunction[Throwable, T] = { - case t: ControlThrowable => throw t - case t: Throwable => - logDebug(label + ": " + unwrap(t)) - logDebug(stackTraceString(unwrap(t))) - alt - } - /** takes AnyRef because it may be binding a Throwable or an Exceptional */ - - private def withLastExceptionLock[T](body: => T, alt: => T): T = { - assert(bindExceptions, "withLastExceptionLock called incorrectly.") - bindExceptions = false - - try beQuietDuring(body) - catch logAndDiscard("withLastExceptionLock", alt) - finally bindExceptions = true - } - - /** - * Contains the code (in string form) representing a wrapper around all - * code executed by this instance. - * - * @return The wrapper code as a string - */ - @DeveloperApi - def executionWrapper = _executionWrapper - - /** - * Sets the code to use as a wrapper around all code executed by this - * instance. - * - * @param code The wrapper code as a string - */ - @DeveloperApi - def setExecutionWrapper(code: String) = _executionWrapper = code - - /** - * Clears the code used as a wrapper around all code executed by - * this instance. - */ - @DeveloperApi - def clearExecutionWrapper() = _executionWrapper = "" - - /** interpreter settings */ - private lazy val isettings = new SparkISettings(this) - - /** - * Instantiates a new compiler used by SparkIMain. Overridable to provide - * own instance of a compiler. - * - * @param settings The settings to provide the compiler - * @param reporter The reporter to use for compiler output - * - * @return The compiler as a Global - */ - @DeveloperApi - protected def newCompiler(settings: Settings, reporter: Reporter): ReplGlobal = { - settings.outputDirs setSingleOutput virtualDirectory - settings.exposeEmptyPackage.value = true - new Global(settings, reporter) with ReplGlobal { - override def toString: String = "" - } - } - - /** - * Adds any specified jars to the compile and runtime classpaths. - * - * @note Currently only supports jars, not directories - * @param urls The list of items to add to the compile and runtime classpaths - */ - @DeveloperApi - def addUrlsToClassPath(urls: URL*): Unit = { - new Run // Needed to force initialization of "something" to correctly load Scala classes from jars - urls.foreach(_runtimeClassLoader.addNewUrl) // Add jars/classes to runtime for execution - updateCompilerClassPath(urls: _*) // Add jars/classes to compile time for compiling - } - - private def updateCompilerClassPath(urls: URL*): Unit = { - require(!global.forMSIL) // Only support JavaPlatform - - val platform = global.platform.asInstanceOf[JavaPlatform] - - val newClassPath = mergeUrlsIntoClassPath(platform, urls: _*) - - // NOTE: Must use reflection until this is exposed/fixed upstream in Scala - val fieldSetter = platform.getClass.getMethods - .find(_.getName.endsWith("currentClassPath_$eq")).get - fieldSetter.invoke(platform, Some(newClassPath)) - - // Reload all jars specified into our compiler - global.invalidateClassPathEntries(urls.map(_.getPath): _*) - } - - private def mergeUrlsIntoClassPath(platform: JavaPlatform, urls: URL*): MergedClassPath[AbstractFile] = { - // Collect our new jars/directories and add them to the existing set of classpaths - val allClassPaths = ( - platform.classPath.asInstanceOf[MergedClassPath[AbstractFile]].entries ++ - urls.map(url => { - platform.classPath.context.newClassPath( - if (url.getProtocol == "file") { - val f = new File(url.getPath) - if (f.isDirectory) - io.AbstractFile.getDirectory(f) - else - io.AbstractFile.getFile(f) - } else { - io.AbstractFile.getURL(url) - } - ) - }) - ).distinct - - // Combine all of our classpaths (old and new) into one merged classpath - new MergedClassPath(allClassPaths, platform.classPath.context) - } - - /** - * Represents the parent classloader used by this instance. Can be - * overridden to provide alternative classloader. - * - * @return The classloader used as the parent loader of this instance - */ - @DeveloperApi - protected def parentClassLoader: ClassLoader = - SparkHelper.explicitParentLoader(settings).getOrElse( this.getClass.getClassLoader() ) - - /* A single class loader is used for all commands interpreted by this Interpreter. - It would also be possible to create a new class loader for each command - to interpret. The advantages of the current approach are: - - - Expressions are only evaluated one time. This is especially - significant for I/O, e.g. "val x = Console.readLine" - - The main disadvantage is: - - - Objects, classes, and methods cannot be rebound. Instead, definitions - shadow the old ones, and old code objects refer to the old - definitions. - */ - private def resetClassLoader() = { - logDebug("Setting new classloader: was " + _classLoader) - _classLoader = null - ensureClassLoader() - } - private final def ensureClassLoader() { - if (_classLoader == null) - _classLoader = makeClassLoader() - } - - // NOTE: Exposed to repl package since used by SparkILoop - private[repl] def classLoader: AbstractFileClassLoader = { - ensureClassLoader() - _classLoader - } - private class TranslatingClassLoader(parent: ClassLoader) extends AbstractFileClassLoader(virtualDirectory, parent) { - /** Overridden here to try translating a simple name to the generated - * class name if the original attempt fails. This method is used by - * getResourceAsStream as well as findClass. - */ - override protected def findAbstractFile(name: String): AbstractFile = { - super.findAbstractFile(name) match { - // deadlocks on startup if we try to translate names too early - case null if isInitializeComplete => - generatedName(name) map (x => super.findAbstractFile(x)) orNull - case file => - file - } - } - } - private def makeClassLoader(): AbstractFileClassLoader = - new TranslatingClassLoader(parentClassLoader match { - case null => ScalaClassLoader fromURLs compilerClasspath - case p => - _runtimeClassLoader = new URLClassLoader(compilerClasspath, p) with ExposeAddUrl - _runtimeClassLoader - }) - - private def getInterpreterClassLoader() = classLoader - - // Set the current Java "context" class loader to this interpreter's class loader - // NOTE: Exposed to repl package since used by SparkILoopInit - private[repl] def setContextClassLoader() = classLoader.setAsContext() - - /** - * Returns the real name of a class based on its repl-defined name. - * - * ==Example== - * Given a simple repl-defined name, returns the real name of - * the class representing it, e.g. for "Bippy" it may return - * {{{ - * $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy - * }}} - * - * @param simpleName The repl-defined name whose real name to retrieve - * - * @return Some real name if the simple name exists, else None - */ - @DeveloperApi - def generatedName(simpleName: String): Option[String] = { - if (simpleName endsWith nme.MODULE_SUFFIX_STRING) optFlatName(simpleName.init) map (_ + nme.MODULE_SUFFIX_STRING) - else optFlatName(simpleName) - } - - // NOTE: Exposed to repl package since used by SparkILoop - private[repl] def flatName(id: String) = optFlatName(id) getOrElse id - // NOTE: Exposed to repl package since used by SparkILoop - private[repl] def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id) - - /** - * Retrieves all simple names contained in the current instance. - * - * @return A list of sorted names - */ - @DeveloperApi - def allDefinedNames = definedNameMap.keys.toList.sorted - - private def pathToType(id: String): String = pathToName(newTypeName(id)) - // NOTE: Exposed to repl package since used by SparkILoop - private[repl] def pathToTerm(id: String): String = pathToName(newTermName(id)) - - /** - * Retrieves the full code path to access the specified simple name - * content. - * - * @param name The simple name of the target whose path to determine - * - * @return The full path used to access the specified target (name) - */ - @DeveloperApi - def pathToName(name: Name): String = { - if (definedNameMap contains name) - definedNameMap(name) fullPath name - else name.toString - } - - /** Most recent tree handled which wasn't wholly synthetic. */ - private def mostRecentlyHandledTree: Option[Tree] = { - prevRequests.reverse foreach { req => - req.handlers.reverse foreach { - case x: MemberDefHandler if x.definesValue && !isInternalTermName(x.name) => return Some(x.member) - case _ => () - } - } - None - } - - /** Stubs for work in progress. */ - private def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = { - for (t1 <- old.simpleNameOfType(name) ; t2 <- req.simpleNameOfType(name)) { - logDebug("Redefining type '%s'\n %s -> %s".format(name, t1, t2)) - } - } - - private def handleTermRedefinition(name: TermName, old: Request, req: Request) = { - for (t1 <- old.compilerTypeOf get name ; t2 <- req.compilerTypeOf get name) { - // Printing the types here has a tendency to cause assertion errors, like - // assertion failed: fatal: has owner value x, but a class owner is required - // so DBG is by-name now to keep it in the family. (It also traps the assertion error, - // but we don't want to unnecessarily risk hosing the compiler's internal state.) - logDebug("Redefining term '%s'\n %s -> %s".format(name, t1, t2)) - } - } - - private def recordRequest(req: Request) { - if (req == null || referencedNameMap == null) - return - - prevRequests += req - req.referencedNames foreach (x => referencedNameMap(x) = req) - - // warning about serially defining companions. It'd be easy - // enough to just redefine them together but that may not always - // be what people want so I'm waiting until I can do it better. - for { - name <- req.definedNames filterNot (x => req.definedNames contains x.companionName) - oldReq <- definedNameMap get name.companionName - newSym <- req.definedSymbols get name - oldSym <- oldReq.definedSymbols get name.companionName - if Seq(oldSym, newSym).permutations exists { case Seq(s1, s2) => s1.isClass && s2.isModule } - } { - afterTyper(replwarn(s"warning: previously defined $oldSym is not a companion to $newSym.")) - replwarn("Companions must be defined together; you may wish to use :paste mode for this.") - } - - // Updating the defined name map - req.definedNames foreach { name => - if (definedNameMap contains name) { - if (name.isTypeName) handleTypeRedefinition(name.toTypeName, definedNameMap(name), req) - else handleTermRedefinition(name.toTermName, definedNameMap(name), req) - } - definedNameMap(name) = req - } - } - - private def replwarn(msg: => String) { - if (!settings.nowarnings.value) - printMessage(msg) - } - - private def isParseable(line: String): Boolean = { - beSilentDuring { - try parse(line) match { - case Some(xs) => xs.nonEmpty // parses as-is - case None => true // incomplete - } - catch { case x: Exception => // crashed the compiler - replwarn("Exception in isParseable(\"" + line + "\"): " + x) - false - } - } - } - - private def compileSourcesKeepingRun(sources: SourceFile*) = { - val run = new Run() - reporter.reset() - run compileSources sources.toList - (!reporter.hasErrors, run) - } - - /** - * Compiles specified source files. - * - * @param sources The sequence of source files to compile - * - * @return True if successful, otherwise false - */ - @DeveloperApi - def compileSources(sources: SourceFile*): Boolean = - compileSourcesKeepingRun(sources: _*)._1 - - /** - * Compiles a string of code. - * - * @param code The string of code to compile - * - * @return True if successful, otherwise false - */ - @DeveloperApi - def compileString(code: String): Boolean = - compileSources(new BatchSourceFile(" UIUtils.headerSparkPage("SQL", content, parent, Some(5000)) } } @@ -82,13 +88,19 @@ private[ui] abstract class ExecutionTable( val duration = executionUIData.completionTime.getOrElse(currentTime) - submissionTime val runningJobs = executionUIData.runningJobs.map { jobId => - {jobId.toString}
    + + [{jobId.toString}] + } val succeededJobs = executionUIData.succeededJobs.sorted.map { jobId => - {jobId.toString}
    + + [{jobId.toString}] + } val failedJobs = executionUIData.failedJobs.sorted.map { jobId => - {jobId.toString}
    + + [{jobId.toString}] + }

    }} - {detailCell(executionUIData.physicalPlanDescription)} } private def descriptionCell(execution: SQLExecutionUIData): Seq[Node] = { val details = if (execution.details.nonEmpty) { - + +details ++ - } - def toNodeSeq: Seq[Node] = {

    {tableName}

    @@ -197,7 +183,7 @@ private[ui] class RunningExecutionTable( showFailedJobs = true) { override protected def header: Seq[String] = - baseHeader ++ Seq("Running Jobs", "Succeeded Jobs", "Failed Jobs", "Detail") + baseHeader ++ Seq("Running Job IDs", "Succeeded Job IDs", "Failed Job IDs") } private[ui] class CompletedExecutionTable( @@ -215,7 +201,7 @@ private[ui] class CompletedExecutionTable( showSucceededJobs = true, showFailedJobs = false) { - override protected def header: Seq[String] = baseHeader ++ Seq("Jobs", "Detail") + override protected def header: Seq[String] = baseHeader ++ Seq("Job IDs") } private[ui] class FailedExecutionTable( @@ -234,5 +220,5 @@ private[ui] class FailedExecutionTable( showFailedJobs = true) { override protected def header: Seq[String] = - baseHeader ++ Seq("Succeeded Jobs", "Failed Jobs", "Detail") + baseHeader ++ Seq("Succeeded Job IDs", "Failed Job IDs") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala index e74d6fb396e1..460fc946c3e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/ExecutionPage.scala @@ -19,19 +19,18 @@ package org.apache.spark.sql.execution.ui import javax.servlet.http.HttpServletRequest -import scala.xml.{Node, Unparsed} +import scala.xml.Node -import org.apache.commons.lang3.StringEscapeUtils - -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ui.{UIUtils, WebUIPage} -private[sql] class ExecutionPage(parent: SQLTab) extends WebUIPage("execution") with Logging { +class ExecutionPage(parent: SQLTab) extends WebUIPage("execution") with Logging { private val listener = parent.listener override def render(request: HttpServletRequest): Seq[Node] = listener.synchronized { - val parameterExecutionId = request.getParameter("id") + // stripXSS is called first to remove suspicious characters used in XSS attacks + val parameterExecutionId = UIUtils.stripXSS(request.getParameter("id")) require(parameterExecutionId != null && parameterExecutionId.nonEmpty, "Missing execution id parameter") @@ -101,7 +100,7 @@ private[sql] class ExecutionPage(parent: SQLTab) extends WebUIPage("execution") } private def planVisualization(metrics: Map[Long, String], graph: SparkPlanGraph): Seq[Node] = { - val metadata = graph.nodes.flatMap { node => + val metadata = graph.allNodes.flatMap { node => val nodeId = s"plan-meta-data-${node.id}"
    {node.desc}
    } @@ -112,11 +111,11 @@ private[sql] class ExecutionPage(parent: SQLTab) extends WebUIPage("execution")
    {graph.makeDotFile(metrics)}
    -
    {graph.nodes.size.toString}
    +
    {graph.allNodes.size.toString}
    {metadata}
    {planVisualizationResources} - + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 5a072de400b6..8c27af374feb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -19,13 +19,85 @@ package org.apache.spark.sql.execution.ui import scala.collection.mutable -import org.apache.spark.executor.TaskMetrics +import com.fasterxml.jackson.databind.JavaType +import com.fasterxml.jackson.databind.`type`.TypeFactory +import com.fasterxml.jackson.databind.annotation.JsonDeserialize +import com.fasterxml.jackson.databind.util.Converter + +import org.apache.spark.{JobExecutionStatus, SparkConf} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ -import org.apache.spark.sql.execution.SQLExecution -import org.apache.spark.sql.execution.metric.{SQLMetricParam, SQLMetricValue} -import org.apache.spark.{JobExecutionStatus, Logging, SparkConf} +import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} +import org.apache.spark.sql.execution.metric._ +import org.apache.spark.ui.SparkUI +import org.apache.spark.util.AccumulatorContext + +@DeveloperApi +case class SparkListenerSQLExecutionStart( + executionId: Long, + description: String, + details: String, + physicalPlanDescription: String, + sparkPlanInfo: SparkPlanInfo, + time: Long) + extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) + extends SparkListenerEvent -private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Logging { +/** + * A message used to update SQL metric value for driver-side updates (which doesn't get reflected + * automatically). + * + * @param executionId The execution id for a query, so we can find the query plan. + * @param accumUpdates Map from accumulator id to the metric value (metrics are always 64-bit ints). + */ +@DeveloperApi +case class SparkListenerDriverAccumUpdates( + executionId: Long, + @JsonDeserialize(contentConverter = classOf[LongLongTupleConverter]) + accumUpdates: Seq[(Long, Long)]) + extends SparkListenerEvent + +/** + * Jackson [[Converter]] for converting an (Int, Int) tuple into a (Long, Long) tuple. + * + * This is necessary due to limitations in how Jackson's scala module deserializes primitives; + * see the "Deserializing Option[Int] and other primitive challenges" section in + * https://github.com/FasterXML/jackson-module-scala/wiki/FAQ for a discussion of this issue and + * SPARK-18462 for the specific problem that motivated this conversion. + */ +private class LongLongTupleConverter extends Converter[(Object, Object), (Long, Long)] { + + override def convert(in: (Object, Object)): (Long, Long) = { + def toLong(a: Object): Long = a match { + case i: java.lang.Integer => i.intValue() + case l: java.lang.Long => l.longValue() + } + (toLong(in._1), toLong(in._2)) + } + + override def getInputType(typeFactory: TypeFactory): JavaType = { + val objectType = typeFactory.uncheckedSimpleType(classOf[Object]) + typeFactory.constructSimpleType(classOf[(_, _)], classOf[(_, _)], Array(objectType, objectType)) + } + + override def getOutputType(typeFactory: TypeFactory): JavaType = { + val longType = typeFactory.uncheckedSimpleType(classOf[Long]) + typeFactory.constructSimpleType(classOf[(_, _)], classOf[(_, _)], Array(longType, longType)) + } +} + +class SQLHistoryListenerFactory extends SparkHistoryListenerFactory { + + override def createListeners(conf: SparkConf, sparkUI: SparkUI): Seq[SparkListener] = { + List(new SQLHistoryListener(conf, sparkUI)) + } +} + +class SQLListener(conf: SparkConf) extends SparkListener with Logging { private val retainedExecutions = conf.getInt("spark.sql.ui.retainedExecutions", 1000) @@ -117,8 +189,8 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi override def onExecutorMetricsUpdate( executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = synchronized { - for ((taskId, stageId, stageAttemptID, metrics) <- executorMetricsUpdate.taskMetrics) { - updateTaskAccumulatorValues(taskId, stageId, stageAttemptID, metrics, finishTask = false) + for ((taskId, stageId, stageAttemptID, accumUpdates) <- executorMetricsUpdate.accumUpdates) { + updateTaskAccumulatorValues(taskId, stageId, stageAttemptID, accumUpdates, finishTask = false) } } @@ -136,27 +208,28 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi } override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { - updateTaskAccumulatorValues( - taskEnd.taskInfo.taskId, - taskEnd.stageId, - taskEnd.stageAttemptId, - taskEnd.taskMetrics, - finishTask = true) + if (taskEnd.taskMetrics != null) { + updateTaskAccumulatorValues( + taskEnd.taskInfo.taskId, + taskEnd.stageId, + taskEnd.stageAttemptId, + taskEnd.taskMetrics.externalAccums.map(a => a.toInfo(Some(a.value), None)), + finishTask = true) + } } /** * Update the accumulator values of a task with the latest metrics for this task. This is called * every time we receive an executor heartbeat or when a task finishes. */ - private def updateTaskAccumulatorValues( + protected def updateTaskAccumulatorValues( taskId: Long, stageId: Int, stageAttemptID: Int, - metrics: TaskMetrics, + _accumulatorUpdates: Seq[AccumulableInfo], finishTask: Boolean): Unit = { - if (metrics == null) { - return - } + val accumulatorUpdates = + _accumulatorUpdates.filter(_.update.isDefined).map(accum => (accum.id, accum.update.get)) _stageIdToStageMetrics.get(stageId) match { case Some(stageMetrics) => @@ -174,18 +247,16 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi case Some(taskMetrics) => if (finishTask) { taskMetrics.finished = true - taskMetrics.accumulatorUpdates = metrics.accumulatorUpdates() + taskMetrics.accumulatorUpdates = accumulatorUpdates } else if (!taskMetrics.finished) { - taskMetrics.accumulatorUpdates = metrics.accumulatorUpdates() + taskMetrics.accumulatorUpdates = accumulatorUpdates } else { // If a task is finished, we should not override with accumulator updates from // heartbeat reports } case None => - // TODO Now just set attemptId to 0. Should fix here when we can get the attempt - // id from SparkListenerExecutorMetricsUpdate stageMetrics.taskIdToMetricUpdates(taskId) = new SQLTaskMetrics( - attemptId = 0, finished = finishTask, metrics.accumulatorUpdates()) + finished = finishTask, accumulatorUpdates) } } case None => @@ -193,38 +264,47 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi } } - def onExecutionStart( - executionId: Long, - description: String, - details: String, - physicalPlanDescription: String, - physicalPlanGraph: SparkPlanGraph, - time: Long): Unit = { - val sqlPlanMetrics = physicalPlanGraph.nodes.flatMap { node => - node.metrics.map(metric => metric.accumulatorId -> metric) - } - - val executionUIData = new SQLExecutionUIData(executionId, description, details, - physicalPlanDescription, physicalPlanGraph, sqlPlanMetrics.toMap, time) - synchronized { - activeExecutions(executionId) = executionUIData - _executionIdToData(executionId) = executionUIData + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { + case SparkListenerSQLExecutionStart(executionId, description, details, + physicalPlanDescription, sparkPlanInfo, time) => + val physicalPlanGraph = SparkPlanGraph(sparkPlanInfo) + val sqlPlanMetrics = physicalPlanGraph.allNodes.flatMap { node => + node.metrics.map(metric => metric.accumulatorId -> metric) + } + val executionUIData = new SQLExecutionUIData( + executionId, + description, + details, + physicalPlanDescription, + physicalPlanGraph, + sqlPlanMetrics.toMap, + time) + synchronized { + activeExecutions(executionId) = executionUIData + _executionIdToData(executionId) = executionUIData + } + case SparkListenerSQLExecutionEnd(executionId, time) => synchronized { + _executionIdToData.get(executionId).foreach { executionUIData => + executionUIData.completionTime = Some(time) + if (!executionUIData.hasRunningJobs) { + // onExecutionEnd happens after all "onJobEnd"s + // So we should update the execution lists. + markExecutionFinished(executionId) + } else { + // There are some running jobs, onExecutionEnd happens before some "onJobEnd"s. + // Then we don't if the execution is successful, so let the last onJobEnd updates the + // execution lists. + } + } } - } - - def onExecutionEnd(executionId: Long, time: Long): Unit = synchronized { - _executionIdToData.get(executionId).foreach { executionUIData => - executionUIData.completionTime = Some(time) - if (!executionUIData.hasRunningJobs) { - // onExecutionEnd happens after all "onJobEnd"s - // So we should update the execution lists. - markExecutionFinished(executionId) - } else { - // There are some running jobs, onExecutionEnd happens before some "onJobEnd"s. - // Then we don't if the execution is successful, so let the last onJobEnd updates the - // execution lists. + case SparkListenerDriverAccumUpdates(executionId, accumUpdates) => synchronized { + _executionIdToData.get(executionId).foreach { executionUIData => + for ((accId, accValue) <- accumUpdates) { + executionUIData.driverAccumUpdates(accId) = accValue + } } } + case _ => // Ignore } private def markExecutionFinished(executionId: Long): Unit = { @@ -265,12 +345,17 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi for (stageId <- executionUIData.stages; stageMetrics <- _stageIdToStageMetrics.get(stageId).toIterable; taskMetrics <- stageMetrics.taskIdToMetricUpdates.values; - accumulatorUpdate <- taskMetrics.accumulatorUpdates.toSeq) yield { - accumulatorUpdate + accumulatorUpdate <- taskMetrics.accumulatorUpdates) yield { + (accumulatorUpdate._1, accumulatorUpdate._2) } - }.filter { case (id, _) => executionUIData.accumulatorMetrics.contains(id) } - mergeAccumulatorUpdates(accumulatorUpdates, accumulatorId => - executionUIData.accumulatorMetrics(accumulatorId).metricParam) + } + + val driverUpdates = executionUIData.driverAccumUpdates.toSeq + val totalUpdates = (accumulatorUpdates ++ driverUpdates).filter { + case (id, _) => executionUIData.accumulatorMetrics.contains(id) + } + mergeAccumulatorUpdates(totalUpdates, accumulatorId => + executionUIData.accumulatorMetrics(accumulatorId).metricType) case None => // This execution has been dropped Map.empty @@ -279,16 +364,58 @@ private[sql] class SQLListener(conf: SparkConf) extends SparkListener with Loggi private def mergeAccumulatorUpdates( accumulatorUpdates: Seq[(Long, Any)], - paramFunc: Long => SQLMetricParam[SQLMetricValue[Any], Any]): Map[Long, String] = { + metricTypeFunc: Long => String): Map[Long, String] = { accumulatorUpdates.groupBy(_._1).map { case (accumulatorId, values) => - val param = paramFunc(accumulatorId) - (accumulatorId, - param.stringValue(values.map(_._2.asInstanceOf[SQLMetricValue[Any]].value))) + val metricType = metricTypeFunc(accumulatorId) + accumulatorId -> + SQLMetrics.stringValue(metricType, values.map(_._2.asInstanceOf[Long])) } } } + +/** + * A [[SQLListener]] for rendering the SQL UI in the history server. + */ +class SQLHistoryListener(conf: SparkConf, sparkUI: SparkUI) + extends SQLListener(conf) { + + private var sqlTabAttached = false + + override def onExecutorMetricsUpdate(u: SparkListenerExecutorMetricsUpdate): Unit = { + // Do nothing; these events are not logged + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { + updateTaskAccumulatorValues( + taskEnd.taskInfo.taskId, + taskEnd.stageId, + taskEnd.stageAttemptId, + taskEnd.taskInfo.accumulables.flatMap { a => + // Filter out accumulators that are not SQL metrics + // For now we assume all SQL metrics are Long's that have been JSON serialized as String's + if (a.metadata == Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) { + val newValue = a.update.map(_.toString.toLong).getOrElse(0L) + Some(a.copy(update = Some(newValue))) + } else { + None + } + }, + finishTask = true) + } + + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { + case _: SparkListenerSQLExecutionStart => + if (!sqlTabAttached) { + new SQLTab(this, sparkUI) + sqlTabAttached = true + } + super.onOtherEvent(event) + case _ => super.onOtherEvent(event) + } +} + /** * Represent all necessary data for an execution that will be used in Web UI. */ @@ -299,10 +426,15 @@ private[ui] class SQLExecutionUIData( val physicalPlanDescription: String, val physicalPlanGraph: SparkPlanGraph, val accumulatorMetrics: Map[Long, SQLPlanMetric], - val submissionTime: Long, - var completionTime: Option[Long] = None, - val jobs: mutable.HashMap[Long, JobExecutionStatus] = mutable.HashMap.empty, - val stages: mutable.ArrayBuffer[Int] = mutable.ArrayBuffer()) { + val submissionTime: Long) { + + var completionTime: Option[Long] = None + + val jobs: mutable.HashMap[Long, JobExecutionStatus] = mutable.HashMap.empty + + val stages: mutable.ArrayBuffer[Int] = mutable.ArrayBuffer() + + val driverAccumUpdates: mutable.HashMap[Long, Long] = mutable.HashMap.empty /** * Return whether there are running jobs in this execution. @@ -335,7 +467,7 @@ private[ui] class SQLExecutionUIData( private[ui] case class SQLPlanMetric( name: String, accumulatorId: Long, - metricParam: SQLMetricParam[SQLMetricValue[Any], Any]) + metricType: String) /** * Store all accumulatorUpdates for all tasks in a Spark stage. @@ -344,10 +476,11 @@ private[ui] class SQLStageMetrics( val stageAttemptId: Long, val taskIdToMetricUpdates: mutable.HashMap[Long, SQLTaskMetrics] = mutable.HashMap.empty) + +// TODO Should add attemptId here when we can get it from SparkListenerExecutorMetricsUpdate /** * Store all accumulatorUpdates for a Spark task. */ private[ui] class SQLTaskMetrics( - val attemptId: Long, // TODO not used yet var finished: Boolean, - var accumulatorUpdates: Map[Long, Any]) + var accumulatorUpdates: Seq[(Long, Any)]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala index 9c27944d42fc..d0376af3e31c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLTab.scala @@ -17,13 +17,11 @@ package org.apache.spark.sql.execution.ui -import java.util.concurrent.atomic.AtomicInteger - -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ui.{SparkUI, SparkUITab} -private[sql] class SQLTab(val listener: SQLListener, sparkUI: SparkUI) - extends SparkUITab(sparkUI, SQLTab.nextTabName) with Logging { +class SQLTab(val listener: SQLListener, sparkUI: SparkUI) + extends SparkUITab(sparkUI, "SQL") with Logging { val parent = sparkUI @@ -34,14 +32,6 @@ private[sql] class SQLTab(val listener: SQLListener, sparkUI: SparkUI) parent.addStaticHandler(SQLTab.STATIC_RESOURCE_DIR, "/static/sql") } -private[sql] object SQLTab { - +object SQLTab { private val STATIC_RESOURCE_DIR = "org/apache/spark/sql/execution/ui/static" - - private val nextTabId = new AtomicInteger(0) - - private def nextTabName: String = { - val nextId = nextTabId.getAndIncrement() - if (nextId == 0) "SQL" else s"SQL$nextId" - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index f1fce5478a3f..884f945815e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -21,8 +21,10 @@ import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.metric.{SQLMetricParam, SQLMetricValue} +import org.apache.commons.lang3.StringEscapeUtils + +import org.apache.spark.sql.execution.{SparkPlanInfo, WholeStageCodegenExec} + /** * A graph used for storing information of an executionPlan of DataFrame. @@ -30,7 +32,7 @@ import org.apache.spark.sql.execution.metric.{SQLMetricParam, SQLMetricValue} * Each graph is defined with a set of nodes and a set of edges. Each node represents a node in the * SparkPlan tree, and each edge represents a parent-child relationship between two nodes. */ -private[ui] case class SparkPlanGraph( +case class SparkPlanGraph( nodes: Seq[SparkPlanGraphNode], edges: Seq[SparkPlanGraphEdge]) { def makeDotFile(metrics: Map[Long, String]): String = { @@ -41,39 +43,92 @@ private[ui] case class SparkPlanGraph( dotFile.append("}") dotFile.toString() } + + /** + * All the SparkPlanGraphNodes, including those inside of WholeStageCodegen. + */ + val allNodes: Seq[SparkPlanGraphNode] = { + nodes.flatMap { + case cluster: SparkPlanGraphCluster => cluster.nodes :+ cluster + case node => Seq(node) + } + } } -private[sql] object SparkPlanGraph { +object SparkPlanGraph { /** * Build a SparkPlanGraph from the root of a SparkPlan tree. */ - def apply(plan: SparkPlan): SparkPlanGraph = { + def apply(planInfo: SparkPlanInfo): SparkPlanGraph = { val nodeIdGenerator = new AtomicLong(0) val nodes = mutable.ArrayBuffer[SparkPlanGraphNode]() val edges = mutable.ArrayBuffer[SparkPlanGraphEdge]() - buildSparkPlanGraphNode(plan, nodeIdGenerator, nodes, edges) + val exchanges = mutable.HashMap[SparkPlanInfo, SparkPlanGraphNode]() + buildSparkPlanGraphNode(planInfo, nodeIdGenerator, nodes, edges, null, null, exchanges) new SparkPlanGraph(nodes, edges) } private def buildSparkPlanGraphNode( - plan: SparkPlan, + planInfo: SparkPlanInfo, nodeIdGenerator: AtomicLong, nodes: mutable.ArrayBuffer[SparkPlanGraphNode], - edges: mutable.ArrayBuffer[SparkPlanGraphEdge]): SparkPlanGraphNode = { - val metrics = plan.metrics.toSeq.map { case (key, metric) => - SQLPlanMetric(metric.name.getOrElse(key), metric.id, - metric.param.asInstanceOf[SQLMetricParam[SQLMetricValue[Any], Any]]) - } - val node = SparkPlanGraphNode( - nodeIdGenerator.getAndIncrement(), plan.nodeName, plan.simpleString, metrics) - nodes += node - val childrenNodes = plan.children.map( - child => buildSparkPlanGraphNode(child, nodeIdGenerator, nodes, edges)) - for (child <- childrenNodes) { - edges += SparkPlanGraphEdge(child.id, node.id) + edges: mutable.ArrayBuffer[SparkPlanGraphEdge], + parent: SparkPlanGraphNode, + subgraph: SparkPlanGraphCluster, + exchanges: mutable.HashMap[SparkPlanInfo, SparkPlanGraphNode]): Unit = { + planInfo.nodeName match { + case "WholeStageCodegen" => + val metrics = planInfo.metrics.map { metric => + SQLPlanMetric(metric.name, metric.accumulatorId, metric.metricType) + } + + val cluster = new SparkPlanGraphCluster( + nodeIdGenerator.getAndIncrement(), + planInfo.nodeName, + planInfo.simpleString, + mutable.ArrayBuffer[SparkPlanGraphNode](), + metrics) + nodes += cluster + + buildSparkPlanGraphNode( + planInfo.children.head, nodeIdGenerator, nodes, edges, parent, cluster, exchanges) + case "InputAdapter" => + buildSparkPlanGraphNode( + planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges) + case "Subquery" if subgraph != null => + // Subquery should not be included in WholeStageCodegen + buildSparkPlanGraphNode(planInfo, nodeIdGenerator, nodes, edges, parent, null, exchanges) + case "Subquery" if exchanges.contains(planInfo) => + // Point to the re-used subquery + val node = exchanges(planInfo) + edges += SparkPlanGraphEdge(node.id, parent.id) + case "ReusedExchange" if exchanges.contains(planInfo.children.head) => + // Point to the re-used exchange + val node = exchanges(planInfo.children.head) + edges += SparkPlanGraphEdge(node.id, parent.id) + case name => + val metrics = planInfo.metrics.map { metric => + SQLPlanMetric(metric.name, metric.accumulatorId, metric.metricType) + } + val node = new SparkPlanGraphNode( + nodeIdGenerator.getAndIncrement(), planInfo.nodeName, + planInfo.simpleString, metrics) + if (subgraph == null) { + nodes += node + } else { + subgraph.nodes += node + } + if (name.contains("Exchange") || name == "Subquery") { + exchanges += planInfo -> node + } + + if (parent != null) { + edges += SparkPlanGraphEdge(node.id, parent.id) + } + planInfo.children.foreach( + buildSparkPlanGraphNode(_, nodeIdGenerator, nodes, edges, node, subgraph, exchanges)) } - node } } @@ -84,30 +139,68 @@ private[sql] object SparkPlanGraph { * @param name the name of this SparkPlan node * @param metrics metrics that this SparkPlan node will track */ -private[ui] case class SparkPlanGraphNode( - id: Long, name: String, desc: String, metrics: Seq[SQLPlanMetric]) { +private[ui] class SparkPlanGraphNode( + val id: Long, + val name: String, + val desc: String, + val metrics: Seq[SQLPlanMetric]) { def makeDotNode(metricsValue: Map[Long, String]): String = { - val values = { - for (metric <- metrics; - value <- metricsValue.get(metric.accumulatorId)) yield { - metric.name + ": " + value - } + val builder = new mutable.StringBuilder(name) + + val values = for { + metric <- metrics + value <- metricsValue.get(metric.accumulatorId) + } yield { + metric.name + ": " + value } - val label = if (values.isEmpty) { - name + + if (values.nonEmpty) { + // If there are metrics, display each entry in a separate line. + // Note: whitespace between two "\n"s is to create an empty line between the name of + // SparkPlan and metrics. If removing it, it won't display the empty line in UI. + builder ++= "\n \n" + builder ++= values.mkString("\n") + } + + s""" $id [label="${StringEscapeUtils.escapeJava(builder.toString())}"];""" + } +} + +/** + * Represent a tree of SparkPlan for WholeStageCodegen. + */ +private[ui] class SparkPlanGraphCluster( + id: Long, + name: String, + desc: String, + val nodes: mutable.ArrayBuffer[SparkPlanGraphNode], + metrics: Seq[SQLPlanMetric]) + extends SparkPlanGraphNode(id, name, desc, metrics) { + + override def makeDotNode(metricsValue: Map[Long, String]): String = { + val duration = metrics.filter(_.name.startsWith(WholeStageCodegenExec.PIPELINE_DURATION_METRIC)) + val labelStr = if (duration.nonEmpty) { + require(duration.length == 1) + val id = duration(0).accumulatorId + if (metricsValue.contains(duration(0).accumulatorId)) { + name + "\n\n" + metricsValue.get(id).get } else { - // If there are metrics, display all metrics in a separate line. We should use an escaped - // "\n" here to follow the dot syntax. - // - // Note: whitespace between two "\n"s is to create an empty line between the name of - // SparkPlan and metrics. If removing it, it won't display the empty line in UI. - name + "\\n \\n" + values.mkString("\\n") + name } - s""" $id [label="$label"];""" + } else { + name + } + s""" + | subgraph cluster${id} { + | label="${StringEscapeUtils.escapeJava(labelStr)}"; + | ${nodes.map(_.makeDotNode(metricsValue)).mkString(" \n")} + | } + """.stripMargin } } + /** * Represent an edge in the SparkPlan tree. `fromId` is the parent node id, and `toId` is the child * node id. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala new file mode 100644 index 000000000000..bc141b36e63b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.window + +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate._ + + +/** + * This class prepares and manages the processing of a number of [[AggregateFunction]]s within a + * single frame. The [[WindowFunctionFrame]] takes care of processing the frame in the correct way + * that reduces the processing of a [[AggregateWindowFunction]] to processing the underlying + * [[AggregateFunction]]. All [[AggregateFunction]]s are processed in [[Complete]] mode. + * + * [[SizeBasedWindowFunction]]s are initialized in a slightly different way. These functions + * require the size of the partition processed and this value is exposed to them when + * the processor is constructed. + * + * Processing of distinct aggregates is currently not supported. + * + * The implementation is split into an object which takes care of construction, and the actual + * processor class. + */ +private[window] object AggregateProcessor { + def apply( + functions: Array[Expression], + ordinal: Int, + inputAttributes: Seq[Attribute], + newMutableProjection: (Seq[Expression], Seq[Attribute]) => MutableProjection) + : AggregateProcessor = { + val aggBufferAttributes = mutable.Buffer.empty[AttributeReference] + val initialValues = mutable.Buffer.empty[Expression] + val updateExpressions = mutable.Buffer.empty[Expression] + val evaluateExpressions = mutable.Buffer.fill[Expression](ordinal)(NoOp) + val imperatives = mutable.Buffer.empty[ImperativeAggregate] + + // SPARK-14244: `SizeBasedWindowFunction`s are firstly created on driver side and then + // serialized to executor side. These functions all reference a global singleton window + // partition size attribute reference, i.e., `SizeBasedWindowFunction.n`. Here we must collect + // the singleton instance created on driver side instead of using executor side + // `SizeBasedWindowFunction.n` to avoid binding failure caused by mismatching expression ID. + val partitionSize: Option[AttributeReference] = { + val aggs = functions.flatMap(_.collectFirst { case f: SizeBasedWindowFunction => f }) + aggs.headOption.map(_.n) + } + + // Check if there are any SizeBasedWindowFunctions. If there are, we add the partition size to + // the aggregation buffer. Note that the ordinal of the partition size value will always be 0. + partitionSize.foreach { n => + aggBufferAttributes += n + initialValues += NoOp + updateExpressions += NoOp + } + + // Add an AggregateFunction to the AggregateProcessor. + functions.foreach { + case agg: DeclarativeAggregate => + aggBufferAttributes ++= agg.aggBufferAttributes + initialValues ++= agg.initialValues + updateExpressions ++= agg.updateExpressions + evaluateExpressions += agg.evaluateExpression + case agg: ImperativeAggregate => + val offset = aggBufferAttributes.size + val imperative = BindReferences.bindReference(agg + .withNewInputAggBufferOffset(offset) + .withNewMutableAggBufferOffset(offset), + inputAttributes) + imperatives += imperative + aggBufferAttributes ++= imperative.aggBufferAttributes + val noOps = Seq.fill(imperative.aggBufferAttributes.size)(NoOp) + initialValues ++= noOps + updateExpressions ++= noOps + evaluateExpressions += imperative + case other => + sys.error(s"Unsupported aggregate function: $other") + } + + // Create the projections. + val initialProj = newMutableProjection(initialValues, partitionSize.toSeq) + val updateProj = newMutableProjection(updateExpressions, aggBufferAttributes ++ inputAttributes) + val evalProj = newMutableProjection(evaluateExpressions, aggBufferAttributes) + + // Create the processor + new AggregateProcessor( + aggBufferAttributes.toArray, + initialProj, + updateProj, + evalProj, + imperatives.toArray, + partitionSize.isDefined) + } +} + +/** + * This class manages the processing of a number of aggregate functions. See the documentation of + * the object for more information. + */ +private[window] final class AggregateProcessor( + private[this] val bufferSchema: Array[AttributeReference], + private[this] val initialProjection: MutableProjection, + private[this] val updateProjection: MutableProjection, + private[this] val evaluateProjection: MutableProjection, + private[this] val imperatives: Array[ImperativeAggregate], + private[this] val trackPartitionSize: Boolean) { + + private[this] val join = new JoinedRow + private[this] val numImperatives = imperatives.length + private[this] val buffer = new SpecificInternalRow(bufferSchema.toSeq.map(_.dataType)) + initialProjection.target(buffer) + updateProjection.target(buffer) + + /** Create the initial state. */ + def initialize(size: Int): Unit = { + // Some initialization expressions are dependent on the partition size so we have to + // initialize the size before initializing all other fields, and we have to pass the buffer to + // the initialization projection. + if (trackPartitionSize) { + buffer.setInt(0, size) + } + initialProjection(buffer) + var i = 0 + while (i < numImperatives) { + imperatives(i).initialize(buffer) + i += 1 + } + } + + /** Update the buffer. */ + def update(input: InternalRow): Unit = { + updateProjection(join(buffer, input)) + var i = 0 + while (i < numImperatives) { + imperatives(i).update(buffer, input) + i += 1 + } + } + + /** Evaluate buffer. */ + def evaluate(target: InternalRow): Unit = { + evaluateProjection.target(target)(buffer) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/BoundOrdering.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/BoundOrdering.scala new file mode 100644 index 000000000000..d6a801954c1a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/BoundOrdering.scala @@ -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.sql.execution.window + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Projection + + +/** + * Function for comparing boundary values. + */ +private[window] abstract class BoundOrdering { + def compare(inputRow: InternalRow, inputIndex: Int, outputRow: InternalRow, outputIndex: Int): Int +} + +/** + * Compare the input index to the bound of the output index. + */ +private[window] final case class RowBoundOrdering(offset: Int) extends BoundOrdering { + override def compare( + inputRow: InternalRow, + inputIndex: Int, + outputRow: InternalRow, + outputIndex: Int): Int = + inputIndex - (outputIndex + offset) +} + +/** + * Compare the value of the input index to the value bound of the output index. + */ +private[window] final case class RangeBoundOrdering( + ordering: Ordering[InternalRow], + current: Projection, + bound: Projection) + extends BoundOrdering { + + override def compare( + inputRow: InternalRow, + inputIndex: Int, + outputRow: InternalRow, + outputIndex: Int): Int = + ordering.compare(current(inputRow), bound(outputRow)) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala new file mode 100644 index 000000000000..800a2ea3f399 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala @@ -0,0 +1,394 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.window + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.{ExternalAppendOnlyUnsafeRowArray, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.types.{CalendarIntervalType, DateType, IntegerType, TimestampType} + +/** + * This class calculates and outputs (windowed) aggregates over the rows in a single (sorted) + * partition. The aggregates are calculated for each row in the group. Special processing + * instructions, frames, are used to calculate these aggregates. Frames are processed in the order + * specified in the window specification (the ORDER BY ... clause). There are four different frame + * types: + * - Entire partition: The frame is the entire partition, i.e. + * UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING. For this case, window function will take all + * rows as inputs and be evaluated once. + * - Growing frame: We only add new rows into the frame, i.e. UNBOUNDED PRECEDING AND .... + * Every time we move to a new row to process, we add some rows to the frame. We do not remove + * rows from this frame. + * - Shrinking frame: We only remove rows from the frame, i.e. ... AND UNBOUNDED FOLLOWING. + * Every time we move to a new row to process, we remove some rows from the frame. We do not add + * rows to this frame. + * - Moving frame: Every time we move to a new row to process, we remove some rows from the frame + * and we add some rows to the frame. Examples are: + * 1 PRECEDING AND CURRENT ROW and 1 FOLLOWING AND 2 FOLLOWING. + * - Offset frame: The frame consist of one row, which is an offset number of rows away from the + * current row. Only [[OffsetWindowFunction]]s can be processed in an offset frame. + * + * Different frame boundaries can be used in Growing, Shrinking and Moving frames. A frame + * boundary can be either Row or Range based: + * - Row Based: A row based boundary is based on the position of the row within the partition. + * An offset indicates the number of rows above or below the current row, the frame for the + * current row starts or ends. For instance, given a row based sliding frame with a lower bound + * offset of -1 and a upper bound offset of +2. The frame for row with index 5 would range from + * index 4 to index 6. + * - Range based: A range based boundary is based on the actual value of the ORDER BY + * expression(s). An offset is used to alter the value of the ORDER BY expression, for + * instance if the current order by expression has a value of 10 and the lower bound offset + * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a + * number of constraints on the ORDER BY expressions: there can be only one expression and this + * expression must have a numerical data type. An exception can be made when the offset is 0, + * because no value modification is needed, in this case multiple and non-numeric ORDER BY + * expression are allowed. + * + * This is quite an expensive operator because every row for a single group must be in the same + * partition and partitions must be sorted according to the grouping and sort order. The operator + * requires the planner to take care of the partitioning and sorting. + * + * The operator is semi-blocking. The window functions and aggregates are calculated one group at + * a time, the result will only be made available after the processing for the entire group has + * finished. The operator is able to process different frame configurations at the same time. This + * is done by delegating the actual frame processing (i.e. calculation of the window functions) to + * specialized classes, see [[WindowFunctionFrame]], which take care of their own frame type: + * Entire Partition, Sliding, Growing & Shrinking. Boundary evaluation is also delegated to a pair + * of specialized classes: [[RowBoundOrdering]] & [[RangeBoundOrdering]]. + */ +case class WindowExec( + windowExpression: Seq[NamedExpression], + partitionSpec: Seq[Expression], + orderSpec: Seq[SortOrder], + child: SparkPlan) + extends UnaryExecNode { + + override def output: Seq[Attribute] = + child.output ++ windowExpression.map(_.toAttribute) + + override def requiredChildDistribution: Seq[Distribution] = { + if (partitionSpec.isEmpty) { + // Only show warning when the number of bytes is larger than 100 MB? + logWarning("No Partition Defined for Window operation! Moving all data to a single " + + "partition, this can cause serious performance degradation.") + AllTuples :: Nil + } else ClusteredDistribution(partitionSpec) :: Nil + } + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = + Seq(partitionSpec.map(SortOrder(_, Ascending)) ++ orderSpec) + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override def outputPartitioning: Partitioning = child.outputPartitioning + + /** + * Create a bound ordering object for a given frame type and offset. A bound ordering object is + * used to determine which input row lies within the frame boundaries of an output row. + * + * This method uses Code Generation. It can only be used on the executor side. + * + * @param frame to evaluate. This can either be a Row or Range frame. + * @param bound with respect to the row. + * @return a bound ordering object. + */ + private[this] def createBoundOrdering(frame: FrameType, bound: Expression): BoundOrdering = { + (frame, bound) match { + case (RowFrame, CurrentRow) => + RowBoundOrdering(0) + + case (RowFrame, IntegerLiteral(offset)) => + RowBoundOrdering(offset) + + case (RangeFrame, CurrentRow) => + val ordering = newOrdering(orderSpec, child.output) + RangeBoundOrdering(ordering, IdentityProjection, IdentityProjection) + + case (RangeFrame, offset: Expression) if orderSpec.size == 1 => + // Use only the first order expression when the offset is non-null. + val sortExpr = orderSpec.head + val expr = sortExpr.child + + // Create the projection which returns the current 'value'. + val current = newMutableProjection(expr :: Nil, child.output) + + // Flip the sign of the offset when processing the order is descending + val boundOffset = sortExpr.direction match { + case Descending => UnaryMinus(offset) + case Ascending => offset + } + + // Create the projection which returns the current 'value' modified by adding the offset. + val boundExpr = (expr.dataType, boundOffset.dataType) match { + case (DateType, IntegerType) => DateAdd(expr, boundOffset) + case (TimestampType, CalendarIntervalType) => + TimeAdd(expr, boundOffset, Some(conf.sessionLocalTimeZone)) + case (a, b) if a== b => Add(expr, boundOffset) + } + val bound = newMutableProjection(boundExpr :: Nil, child.output) + + // Construct the ordering. This is used to compare the result of current value projection + // to the result of bound value projection. This is done manually because we want to use + // Code Generation (if it is enabled). + val boundSortExprs = sortExpr.copy(BoundReference(0, expr.dataType, expr.nullable)) :: Nil + val ordering = newOrdering(boundSortExprs, Nil) + RangeBoundOrdering(ordering, current, bound) + + case (RangeFrame, _) => + sys.error("Non-Zero range offsets are not supported for windows " + + "with multiple order expressions.") + } + } + + /** + * Collection containing an entry for each window frame to process. Each entry contains a frame's + * [[WindowExpression]]s and factory function for the WindowFrameFunction. + */ + private[this] lazy val windowFrameExpressionFactoryPairs = { + type FrameKey = (String, FrameType, Expression, Expression) + type ExpressionBuffer = mutable.Buffer[Expression] + val framedFunctions = mutable.Map.empty[FrameKey, (ExpressionBuffer, ExpressionBuffer)] + + // Add a function and its function to the map for a given frame. + def collect(tpe: String, fr: SpecifiedWindowFrame, e: Expression, fn: Expression): Unit = { + val key = (tpe, fr.frameType, fr.lower, fr.upper) + val (es, fns) = framedFunctions.getOrElseUpdate( + key, (ArrayBuffer.empty[Expression], ArrayBuffer.empty[Expression])) + es += e + fns += fn + } + + // Collect all valid window functions and group them by their frame. + windowExpression.foreach { x => + x.foreach { + case e @ WindowExpression(function, spec) => + val frame = spec.frameSpecification.asInstanceOf[SpecifiedWindowFrame] + function match { + case AggregateExpression(f, _, _, _) => collect("AGGREGATE", frame, e, f) + case f: AggregateWindowFunction => collect("AGGREGATE", frame, e, f) + case f: OffsetWindowFunction => collect("OFFSET", frame, e, f) + case f => sys.error(s"Unsupported window function: $f") + } + case _ => + } + } + + // Map the groups to a (unbound) expression and frame factory pair. + var numExpressions = 0 + framedFunctions.toSeq.map { + case (key, (expressions, functionSeq)) => + val ordinal = numExpressions + val functions = functionSeq.toArray + + // Construct an aggregate processor if we need one. + def processor = AggregateProcessor( + functions, + ordinal, + child.output, + (expressions, schema) => + newMutableProjection(expressions, schema, subexpressionEliminationEnabled)) + + // Create the factory + val factory = key match { + // Offset Frame + case ("OFFSET", _, IntegerLiteral(offset), _) => + target: InternalRow => + new OffsetWindowFunctionFrame( + target, + ordinal, + // OFFSET frame functions are guaranteed be OffsetWindowFunctions. + functions.map(_.asInstanceOf[OffsetWindowFunction]), + child.output, + (expressions, schema) => + newMutableProjection(expressions, schema, subexpressionEliminationEnabled), + offset) + + // Entire Partition Frame. + case ("AGGREGATE", _, UnboundedPreceding, UnboundedFollowing) => + target: InternalRow => { + new UnboundedWindowFunctionFrame(target, processor) + } + + // Growing Frame. + case ("AGGREGATE", frameType, UnboundedPreceding, upper) => + target: InternalRow => { + new UnboundedPrecedingWindowFunctionFrame( + target, + processor, + createBoundOrdering(frameType, upper)) + } + + // Shrinking Frame. + case ("AGGREGATE", frameType, lower, UnboundedFollowing) => + target: InternalRow => { + new UnboundedFollowingWindowFunctionFrame( + target, + processor, + createBoundOrdering(frameType, lower)) + } + + // Moving Frame. + case ("AGGREGATE", frameType, lower, upper) => + target: InternalRow => { + new SlidingWindowFunctionFrame( + target, + processor, + createBoundOrdering(frameType, lower), + createBoundOrdering(frameType, upper)) + } + } + + // Keep track of the number of expressions. This is a side-effect in a map... + numExpressions += expressions.size + + // Create the Frame Expression - Factory pair. + (expressions, factory) + } + } + + /** + * Create the resulting projection. + * + * This method uses Code Generation. It can only be used on the executor side. + * + * @param expressions unbound ordered function expressions. + * @return the final resulting projection. + */ + private[this] def createResultProjection(expressions: Seq[Expression]): UnsafeProjection = { + val references = expressions.zipWithIndex.map{ case (e, i) => + // Results of window expressions will be on the right side of child's output + BoundReference(child.output.size + i, e.dataType, e.nullable) + } + val unboundToRefMap = expressions.zip(references).toMap + val patchedWindowExpression = windowExpression.map(_.transform(unboundToRefMap)) + UnsafeProjection.create( + child.output ++ patchedWindowExpression, + child.output) + } + + protected override def doExecute(): RDD[InternalRow] = { + // Unwrap the expressions and factories from the map. + val expressions = windowFrameExpressionFactoryPairs.flatMap(_._1) + val factories = windowFrameExpressionFactoryPairs.map(_._2).toArray + val inMemoryThreshold = sqlContext.conf.windowExecBufferInMemoryThreshold + val spillThreshold = sqlContext.conf.windowExecBufferSpillThreshold + + // Start processing. + child.execute().mapPartitions { stream => + new Iterator[InternalRow] { + + // Get all relevant projections. + val result = createResultProjection(expressions) + val grouping = UnsafeProjection.create(partitionSpec, child.output) + + // Manage the stream and the grouping. + var nextRow: UnsafeRow = null + var nextGroup: UnsafeRow = null + var nextRowAvailable: Boolean = false + private[this] def fetchNextRow() { + nextRowAvailable = stream.hasNext + if (nextRowAvailable) { + nextRow = stream.next().asInstanceOf[UnsafeRow] + nextGroup = grouping(nextRow) + } else { + nextRow = null + nextGroup = null + } + } + fetchNextRow() + + // Manage the current partition. + val inputFields = child.output.length + + val buffer: ExternalAppendOnlyUnsafeRowArray = + new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold) + + var bufferIterator: Iterator[UnsafeRow] = _ + + val windowFunctionResult = new SpecificInternalRow(expressions.map(_.dataType)) + val frames = factories.map(_(windowFunctionResult)) + val numFrames = frames.length + private[this] def fetchNextPartition() { + // Collect all the rows in the current partition. + // Before we start to fetch new input rows, make a copy of nextGroup. + val currentGroup = nextGroup.copy() + + // clear last partition + buffer.clear() + + while (nextRowAvailable && nextGroup == currentGroup) { + buffer.add(nextRow) + fetchNextRow() + } + + // Setup the frames. + var i = 0 + while (i < numFrames) { + frames(i).prepare(buffer) + i += 1 + } + + // Setup iteration + rowIndex = 0 + bufferIterator = buffer.generateIterator() + } + + // Iteration + var rowIndex = 0 + + override final def hasNext: Boolean = + (bufferIterator != null && bufferIterator.hasNext) || nextRowAvailable + + val join = new JoinedRow + override final def next(): InternalRow = { + // Load the next partition if we need to. + if ((bufferIterator == null || !bufferIterator.hasNext) && nextRowAvailable) { + fetchNextPartition() + } + + if (bufferIterator.hasNext) { + val current = bufferIterator.next() + + // Get the results for the window frames. + var i = 0 + while (i < numFrames) { + frames(i).write(rowIndex, current) + i += 1 + } + + // 'Merge' the input row with the window function result + join(current, windowFunctionResult) + rowIndex += 1 + + // Return the projection. + result(join) + } else { + throw new NoSuchElementException + } + } + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala new file mode 100644 index 000000000000..156002ef58fb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala @@ -0,0 +1,398 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.window + +import java.util + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp +import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray + + +/** + * A window function calculates the results of a number of window functions for a window frame. + * Before use a frame must be prepared by passing it all the rows in the current partition. After + * preparation the update method can be called to fill the output rows. + */ +private[window] abstract class WindowFunctionFrame { + /** + * Prepare the frame for calculating the results for a partition. + * + * @param rows to calculate the frame results for. + */ + def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit + + /** + * Write the current results to the target row. + */ + def write(index: Int, current: InternalRow): Unit +} + +object WindowFunctionFrame { + def getNextOrNull(iterator: Iterator[UnsafeRow]): UnsafeRow = { + if (iterator.hasNext) iterator.next() else null + } +} + +/** + * The offset window frame calculates frames containing LEAD/LAG statements. + * + * @param target to write results to. + * @param ordinal the ordinal is the starting offset at which the results of the window frame get + * written into the (shared) target row. The result of the frame expression with + * index 'i' will be written to the 'ordinal' + 'i' position in the target row. + * @param expressions to shift a number of rows. + * @param inputSchema required for creating a projection. + * @param newMutableProjection function used to create the projection. + * @param offset by which rows get moved within a partition. + */ +private[window] final class OffsetWindowFunctionFrame( + target: InternalRow, + ordinal: Int, + expressions: Array[OffsetWindowFunction], + inputSchema: Seq[Attribute], + newMutableProjection: (Seq[Expression], Seq[Attribute]) => MutableProjection, + offset: Int) + extends WindowFunctionFrame { + + /** Rows of the partition currently being processed. */ + private[this] var input: ExternalAppendOnlyUnsafeRowArray = null + + /** + * An iterator over the [[input]] + */ + private[this] var inputIterator: Iterator[UnsafeRow] = _ + + /** Index of the input row currently used for output. */ + private[this] var inputIndex = 0 + + /** + * Create the projection used when the offset row exists. + * Please note that this project always respect null input values (like PostgreSQL). + */ + private[this] val projection = { + // Collect the expressions and bind them. + val inputAttrs = inputSchema.map(_.withNullability(true)) + val boundExpressions = Seq.fill(ordinal)(NoOp) ++ expressions.toSeq.map { e => + BindReferences.bindReference(e.input, inputAttrs) + } + + // Create the projection. + newMutableProjection(boundExpressions, Nil).target(target) + } + + /** Create the projection used when the offset row DOES NOT exists. */ + private[this] val fillDefaultValue = { + // Collect the expressions and bind them. + val inputAttrs = inputSchema.map(_.withNullability(true)) + val boundExpressions = Seq.fill(ordinal)(NoOp) ++ expressions.toSeq.map { e => + if (e.default == null || e.default.foldable && e.default.eval() == null) { + // The default value is null. + Literal.create(null, e.dataType) + } else { + // The default value is an expression. + BindReferences.bindReference(e.default, inputAttrs) + } + } + + // Create the projection. + newMutableProjection(boundExpressions, Nil).target(target) + } + + override def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { + input = rows + inputIterator = input.generateIterator() + // drain the first few rows if offset is larger than zero + inputIndex = 0 + while (inputIndex < offset) { + if (inputIterator.hasNext) inputIterator.next() + inputIndex += 1 + } + inputIndex = offset + } + + override def write(index: Int, current: InternalRow): Unit = { + if (inputIndex >= 0 && inputIndex < input.length) { + val r = WindowFunctionFrame.getNextOrNull(inputIterator) + projection(r) + } else { + // Use default values since the offset row does not exist. + fillDefaultValue(current) + } + inputIndex += 1 + } +} + +/** + * The sliding window frame calculates frames with the following SQL form: + * ... BETWEEN 1 PRECEDING AND 1 FOLLOWING + * + * @param target to write results to. + * @param processor to calculate the row values with. + * @param lbound comparator used to identify the lower bound of an output row. + * @param ubound comparator used to identify the upper bound of an output row. + */ +private[window] final class SlidingWindowFunctionFrame( + target: InternalRow, + processor: AggregateProcessor, + lbound: BoundOrdering, + ubound: BoundOrdering) + extends WindowFunctionFrame { + + /** Rows of the partition currently being processed. */ + private[this] var input: ExternalAppendOnlyUnsafeRowArray = null + + /** + * An iterator over the [[input]] + */ + private[this] var inputIterator: Iterator[UnsafeRow] = _ + + /** The next row from `input`. */ + private[this] var nextRow: InternalRow = null + + /** The rows within current sliding window. */ + private[this] val buffer = new util.ArrayDeque[InternalRow]() + + /** + * Index of the first input row with a value greater than the upper bound of the current + * output row. + */ + private[this] var inputHighIndex = 0 + + /** + * Index of the first input row with a value equal to or greater than the lower bound of the + * current output row. + */ + private[this] var inputLowIndex = 0 + + /** Prepare the frame for calculating a new partition. Reset all variables. */ + override def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { + input = rows + inputIterator = input.generateIterator() + nextRow = WindowFunctionFrame.getNextOrNull(inputIterator) + inputHighIndex = 0 + inputLowIndex = 0 + buffer.clear() + } + + /** Write the frame columns for the current row to the given target row. */ + override def write(index: Int, current: InternalRow): Unit = { + var bufferUpdated = index == 0 + + // Drop all rows from the buffer for which the input row value is smaller than + // the output row lower bound. + while (!buffer.isEmpty && lbound.compare(buffer.peek(), inputLowIndex, current, index) < 0) { + buffer.remove() + inputLowIndex += 1 + bufferUpdated = true + } + + // Add all rows to the buffer for which the input row value is equal to or less than + // the output row upper bound. + while (nextRow != null && ubound.compare(nextRow, inputHighIndex, current, index) <= 0) { + if (lbound.compare(nextRow, inputLowIndex, current, index) < 0) { + inputLowIndex += 1 + } else { + buffer.add(nextRow.copy()) + bufferUpdated = true + } + nextRow = WindowFunctionFrame.getNextOrNull(inputIterator) + inputHighIndex += 1 + } + + // Only recalculate and update when the buffer changes. + if (bufferUpdated) { + processor.initialize(input.length) + val iter = buffer.iterator() + while (iter.hasNext) { + processor.update(iter.next()) + } + processor.evaluate(target) + } + } +} + +/** + * The unbounded window frame calculates frames with the following SQL forms: + * ... (No Frame Definition) + * ... BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING + * + * Its results are the same for each and every row in the partition. This class can be seen as a + * special case of a sliding window, but is optimized for the unbound case. + * + * @param target to write results to. + * @param processor to calculate the row values with. + */ +private[window] final class UnboundedWindowFunctionFrame( + target: InternalRow, + processor: AggregateProcessor) + extends WindowFunctionFrame { + + /** Prepare the frame for calculating a new partition. Process all rows eagerly. */ + override def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { + processor.initialize(rows.length) + + val iterator = rows.generateIterator() + while (iterator.hasNext) { + processor.update(iterator.next()) + } + } + + /** Write the frame columns for the current row to the given target row. */ + override def write(index: Int, current: InternalRow): Unit = { + // Unfortunately we cannot assume that evaluation is deterministic. So we need to re-evaluate + // for each row. + processor.evaluate(target) + } +} + +/** + * The UnboundPreceding window frame calculates frames with the following SQL form: + * ... BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + * + * There is only an upper bound. Very common use cases are for instance running sums or counts + * (row_number). Technically this is a special case of a sliding window. However a sliding window + * has to maintain a buffer, and it must do a full evaluation everytime the buffer changes. This + * is not the case when there is no lower bound, given the additive nature of most aggregates + * streaming updates and partial evaluation suffice and no buffering is needed. + * + * @param target to write results to. + * @param processor to calculate the row values with. + * @param ubound comparator used to identify the upper bound of an output row. + */ +private[window] final class UnboundedPrecedingWindowFunctionFrame( + target: InternalRow, + processor: AggregateProcessor, + ubound: BoundOrdering) + extends WindowFunctionFrame { + + /** Rows of the partition currently being processed. */ + private[this] var input: ExternalAppendOnlyUnsafeRowArray = null + + /** + * An iterator over the [[input]] + */ + private[this] var inputIterator: Iterator[UnsafeRow] = _ + + /** The next row from `input`. */ + private[this] var nextRow: InternalRow = null + + /** + * Index of the first input row with a value greater than the upper bound of the current + * output row. + */ + private[this] var inputIndex = 0 + + /** Prepare the frame for calculating a new partition. */ + override def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { + input = rows + inputIndex = 0 + inputIterator = input.generateIterator() + if (inputIterator.hasNext) { + nextRow = inputIterator.next() + } + + processor.initialize(input.length) + } + + /** Write the frame columns for the current row to the given target row. */ + override def write(index: Int, current: InternalRow): Unit = { + var bufferUpdated = index == 0 + + // Add all rows to the aggregates for which the input row value is equal to or less than + // the output row upper bound. + while (nextRow != null && ubound.compare(nextRow, inputIndex, current, index) <= 0) { + processor.update(nextRow) + nextRow = WindowFunctionFrame.getNextOrNull(inputIterator) + inputIndex += 1 + bufferUpdated = true + } + + // Only recalculate and update when the buffer changes. + if (bufferUpdated) { + processor.evaluate(target) + } + } +} + +/** + * The UnboundFollowing window frame calculates frames with the following SQL form: + * ... BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING + * + * There is only an upper bound. This is a slightly modified version of the sliding window. The + * sliding window operator has to check if both upper and the lower bound change when a new row + * gets processed, where as the unbounded following only has to check the lower bound. + * + * This is a very expensive operator to use, O(n * (n - 1) /2), because we need to maintain a + * buffer and must do full recalculation after each row. Reverse iteration would be possible, if + * the commutativity of the used window functions can be guaranteed. + * + * @param target to write results to. + * @param processor to calculate the row values with. + * @param lbound comparator used to identify the lower bound of an output row. + */ +private[window] final class UnboundedFollowingWindowFunctionFrame( + target: InternalRow, + processor: AggregateProcessor, + lbound: BoundOrdering) + extends WindowFunctionFrame { + + /** Rows of the partition currently being processed. */ + private[this] var input: ExternalAppendOnlyUnsafeRowArray = null + + /** + * Index of the first input row with a value equal to or greater than the lower bound of the + * current output row. + */ + private[this] var inputIndex = 0 + + /** Prepare the frame for calculating a new partition. */ + override def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { + input = rows + inputIndex = 0 + } + + /** Write the frame columns for the current row to the given target row. */ + override def write(index: Int, current: InternalRow): Unit = { + var bufferUpdated = index == 0 + + // Ignore all the rows from the buffer for which the input row value is smaller than + // the output row lower bound. + val iterator = input.generateIterator(startIndex = inputIndex) + + var nextRow = WindowFunctionFrame.getNextOrNull(iterator) + while (nextRow != null && lbound.compare(nextRow, inputIndex, current, index) < 0) { + inputIndex += 1 + bufferUpdated = true + nextRow = WindowFunctionFrame.getNextOrNull(iterator) + } + + // Only recalculate and update when the buffer changes. + if (bufferUpdated) { + processor.initialize(input.length) + if (nextRow != null) { + processor.update(nextRow) + } + while (iterator.hasNext) { + processor.update(iterator.next()) + } + processor.evaluate(target) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala new file mode 100644 index 000000000000..058c38c8cb8f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.expressions + +import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.sql.{Dataset, Encoder, TypedColumn} +import org.apache.spark.sql.catalyst.encoders.encoderFor +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete} +import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression + +/** + * :: Experimental :: + * A base class for user-defined aggregations, which can be used in `Dataset` operations to take + * all of the elements of a group and reduce them to a single value. + * + * For example, the following aggregator extracts an `int` from a specific class and adds them up: + * {{{ + * case class Data(i: Int) + * + * val customSummer = new Aggregator[Data, Int, Int] { + * def zero: Int = 0 + * def reduce(b: Int, a: Data): Int = b + a.i + * def merge(b1: Int, b2: Int): Int = b1 + b2 + * def finish(r: Int): Int = r + * }.toColumn() + * + * val ds: Dataset[Data] = ... + * val aggregated = ds.select(customSummer) + * }}} + * + * Based loosely on Aggregator from Algebird: https://github.com/twitter/algebird + * + * @tparam IN The input type for the aggregation. + * @tparam BUF The type of the intermediate value of the reduction. + * @tparam OUT The type of the final output result. + * @since 1.6.0 + */ +@Experimental +@InterfaceStability.Evolving +abstract class Aggregator[-IN, BUF, OUT] extends Serializable { + + /** + * A zero value for this aggregation. Should satisfy the property that any b + zero = b. + * @since 1.6.0 + */ + def zero: BUF + + /** + * Combine two values to produce a new value. For performance, the function may modify `b` and + * return it instead of constructing new object for b. + * @since 1.6.0 + */ + def reduce(b: BUF, a: IN): BUF + + /** + * Merge two intermediate values. + * @since 1.6.0 + */ + def merge(b1: BUF, b2: BUF): BUF + + /** + * Transform the output of the reduction. + * @since 1.6.0 + */ + def finish(reduction: BUF): OUT + + /** + * Specifies the `Encoder` for the intermediate value type. + * @since 2.0.0 + */ + def bufferEncoder: Encoder[BUF] + + /** + * Specifies the `Encoder` for the final ouput value type. + * @since 2.0.0 + */ + def outputEncoder: Encoder[OUT] + + /** + * Returns this `Aggregator` as a `TypedColumn` that can be used in `Dataset`. + * operations. + * @since 1.6.0 + */ + def toColumn: TypedColumn[IN, OUT] = { + implicit val bEncoder = bufferEncoder + implicit val cEncoder = outputEncoder + + val expr = + AggregateExpression( + TypedAggregateExpression(this), + Complete, + isDistinct = false) + + new TypedColumn[IN, OUT](expr, encoderFor[OUT]) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/ReduceAggregator.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/ReduceAggregator.scala new file mode 100644 index 000000000000..e266ae55cc4d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/ReduceAggregator.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.expressions + +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder + +/** + * An aggregator that uses a single associative and commutative reduce function. This reduce + * function can be used to go through all input values and reduces them to a single value. + * If there is no input, a null value is returned. + * + * This class currently assumes there is at least one input row. + */ +private[sql] class ReduceAggregator[T: Encoder](func: (T, T) => T) + extends Aggregator[T, (Boolean, T), T] { + + @transient private val encoder = implicitly[Encoder[T]] + + override def zero: (Boolean, T) = (false, null.asInstanceOf[T]) + + override def bufferEncoder: Encoder[(Boolean, T)] = + ExpressionEncoder.tuple( + ExpressionEncoder[Boolean](), + encoder.asInstanceOf[ExpressionEncoder[T]]) + + override def outputEncoder: Encoder[T] = encoder + + override def reduce(b: (Boolean, T), a: T): (Boolean, T) = { + if (b._1) { + (true, func(b._2, a)) + } else { + (true, a) + } + } + + override def merge(b1: (Boolean, T), b2: (Boolean, T)): (Boolean, T) = { + if (!b1._1) { + b2 + } else if (!b2._1) { + b1 + } else { + (true, func(b1._2, b2._2)) + } + } + + override def finish(reduction: (Boolean, T)): T = { + if (!reduction._1) { + throw new IllegalStateException("ReduceAggregator requires at least one input row") + } + reduction._2 + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala new file mode 100644 index 000000000000..03b654f83052 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.expressions + +import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.expressions.ScalaUDF +import org.apache.spark.sql.types.DataType + +/** + * A user-defined function. To create one, use the `udf` functions in `functions`. + * + * As an example: + * {{{ + * // Defined a UDF that returns true or false based on some numeric score. + * val predict = udf((score: Double) => if (score > 0.5) true else false) + * + * // Projects a column that adds a prediction column based on the score column. + * df.select( predict(df("score")) ) + * }}} + * + * @since 1.3.0 + */ +@InterfaceStability.Stable +case class UserDefinedFunction protected[sql] ( + f: AnyRef, + dataType: DataType, + inputTypes: Option[Seq[DataType]]) { + + private var _nameOption: Option[String] = None + private var _nullable: Boolean = true + private var _deterministic: Boolean = true + + /** + * Returns true when the UDF can return a nullable value. + * + * @since 2.3.0 + */ + def nullable: Boolean = _nullable + + /** + * Returns true iff the UDF is deterministic, i.e. the UDF produces the same output given the same + * input. + * + * @since 2.3.0 + */ + def deterministic: Boolean = _deterministic + + /** + * Returns an expression that invokes the UDF, using the given arguments. + * + * @since 1.3.0 + */ + def apply(exprs: Column*): Column = { + Column(ScalaUDF( + f, + dataType, + exprs.map(_.expr), + inputTypes.getOrElse(Nil), + udfName = _nameOption, + nullable = _nullable, + udfDeterministic = _deterministic)) + } + + private def copyAll(): UserDefinedFunction = { + val udf = copy() + udf._nameOption = _nameOption + udf._nullable = _nullable + udf._deterministic = _deterministic + udf + } + + /** + * Updates UserDefinedFunction with a given name. + * + * @since 2.3.0 + */ + def withName(name: String): UserDefinedFunction = { + val udf = copyAll() + udf._nameOption = Option(name) + udf + } + + /** + * Updates UserDefinedFunction to non-nullable. + * + * @since 2.3.0 + */ + def asNonNullable(): UserDefinedFunction = { + if (!nullable) { + this + } else { + val udf = copyAll() + udf._nullable = false + udf + } + } + + /** + * Updates UserDefinedFunction to nondeterministic. + * + * @since 2.3.0 + */ + def asNondeterministic(): UserDefinedFunction = { + if (!_deterministic) { + this + } else { + val udf = copyAll() + udf._deterministic = false + udf + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala index e9b60841fc28..1caa243f8d11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -17,17 +17,17 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.expressions._ /** - * :: Experimental :: * Utility functions for defining window in DataFrames. * * {{{ * // PARTITION BY country ORDER BY date ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW - * Window.partitionBy("country").orderBy("date").rowsBetween(Long.MinValue, 0) + * Window.partitionBy("country").orderBy("date") + * .rowsBetween(Window.unboundedPreceding, Window.currentRow) * * // PARTITION BY country ORDER BY date ROWS BETWEEN 3 PRECEDING AND 3 FOLLOWING * Window.partitionBy("country").orderBy("date").rowsBetween(-3, 3) @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.expressions._ * * @since 1.4.0 */ -@Experimental +@InterfaceStability.Stable object Window { /** @@ -74,19 +74,206 @@ object Window { spec.orderBy(cols : _*) } - private def spec: WindowSpec = { + /** + * Value representing the first row in the partition, equivalent to "UNBOUNDED PRECEDING" in SQL. + * This can be used to specify the frame boundaries: + * + * {{{ + * Window.rowsBetween(Window.unboundedPreceding, Window.currentRow) + * }}} + * + * @since 2.1.0 + */ + def unboundedPreceding: Long = Long.MinValue + + /** + * Value representing the last row in the partition, equivalent to "UNBOUNDED FOLLOWING" in SQL. + * This can be used to specify the frame boundaries: + * + * {{{ + * Window.rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing) + * }}} + * + * @since 2.1.0 + */ + def unboundedFollowing: Long = Long.MaxValue + + /** + * Value representing the current row. This can be used to specify the frame boundaries: + * + * {{{ + * Window.rowsBetween(Window.unboundedPreceding, Window.currentRow) + * }}} + * + * @since 2.1.0 + */ + def currentRow: Long = 0 + + /** + * Creates a [[WindowSpec]] with the frame boundaries defined, + * from `start` (inclusive) to `end` (inclusive). + * + * Both `start` and `end` are positions relative to the current row. For example, "0" means + * "current row", while "-1" means the row before the current row, and "5" means the fifth row + * after the current row. + * + * We recommend users use `Window.unboundedPreceding`, `Window.unboundedFollowing`, + * and `Window.currentRow` to specify special boundary values, rather than using integral + * values directly. + * + * A row based boundary is based on the position of the row within the partition. + * An offset indicates the number of rows above or below the current row, the frame for the + * current row starts or ends. For instance, given a row based sliding frame with a lower bound + * offset of -1 and a upper bound offset of +2. The frame for row with index 5 would range from + * index 4 to index 6. + * + * {{{ + * import org.apache.spark.sql.expressions.Window + * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) + * .toDF("id", "category") + * val byCategoryOrderedById = + * Window.partitionBy('category).orderBy('id).rowsBetween(Window.currentRow, 1) + * df.withColumn("sum", sum('id) over byCategoryOrderedById).show() + * + * +---+--------+---+ + * | id|category|sum| + * +---+--------+---+ + * | 1| b| 3| + * | 2| b| 5| + * | 3| b| 3| + * | 1| a| 2| + * | 1| a| 3| + * | 2| a| 2| + * +---+--------+---+ + * }}} + * + * @param start boundary start, inclusive. The frame is unbounded if this is + * the minimum long value (`Window.unboundedPreceding`). + * @param end boundary end, inclusive. The frame is unbounded if this is the + * maximum long value (`Window.unboundedFollowing`). + * @since 2.1.0 + */ + // Note: when updating the doc for this method, also update WindowSpec.rowsBetween. + def rowsBetween(start: Long, end: Long): WindowSpec = { + spec.rowsBetween(start, end) + } + + /** + * Creates a [[WindowSpec]] with the frame boundaries defined, + * from `start` (inclusive) to `end` (inclusive). + * + * Both `start` and `end` are relative to the current row. For example, "0" means "current row", + * while "-1" means one off before the current row, and "5" means the five off after the + * current row. + * + * We recommend users use `Window.unboundedPreceding`, `Window.unboundedFollowing`, + * and `Window.currentRow` to specify special boundary values, rather than using long values + * directly. + * + * A range-based boundary is based on the actual value of the ORDER BY + * expression(s). An offset is used to alter the value of the ORDER BY expression, for + * instance if the current order by expression has a value of 10 and the lower bound offset + * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a + * number of constraints on the ORDER BY expressions: there can be only one expression and this + * expression must have a numerical data type. An exception can be made when the offset is + * unbounded, because no value modification is needed, in this case multiple and non-numeric + * ORDER BY expression are allowed. + * + * {{{ + * import org.apache.spark.sql.expressions.Window + * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) + * .toDF("id", "category") + * val byCategoryOrderedById = + * Window.partitionBy('category).orderBy('id).rangeBetween(Window.currentRow, 1) + * df.withColumn("sum", sum('id) over byCategoryOrderedById).show() + * + * +---+--------+---+ + * | id|category|sum| + * +---+--------+---+ + * | 1| b| 3| + * | 2| b| 5| + * | 3| b| 3| + * | 1| a| 4| + * | 1| a| 4| + * | 2| a| 2| + * +---+--------+---+ + * }}} + * + * @param start boundary start, inclusive. The frame is unbounded if this is + * the minimum long value (`Window.unboundedPreceding`). + * @param end boundary end, inclusive. The frame is unbounded if this is the + * maximum long value (`Window.unboundedFollowing`). + * @since 2.1.0 + */ + // Note: when updating the doc for this method, also update WindowSpec.rangeBetween. + def rangeBetween(start: Long, end: Long): WindowSpec = { + spec.rangeBetween(start, end) + } + + /** + * Creates a [[WindowSpec]] with the frame boundaries defined, + * from `start` (inclusive) to `end` (inclusive). + * + * Both `start` and `end` are relative to the current row. For example, "lit(0)" means + * "current row", while "lit(-1)" means one off before the current row, and "lit(5)" means the + * five off after the current row. + * + * Users should use `unboundedPreceding()`, `unboundedFollowing()`, and `currentRow()` from + * [[org.apache.spark.sql.functions]] to specify special boundary values, literals are not + * transformed to [[org.apache.spark.sql.catalyst.expressions.SpecialFrameBoundary]]s. + * + * A range-based boundary is based on the actual value of the ORDER BY + * expression(s). An offset is used to alter the value of the ORDER BY expression, for + * instance if the current order by expression has a value of 10 and the lower bound offset + * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a + * number of constraints on the ORDER BY expressions: there can be only one expression and this + * expression must have a numerical/date/timestamp data type. An exception can be made when the + * offset is unbounded, because no value modification is needed, in this case multiple and + * non-numerical/date/timestamp data type ORDER BY expression are allowed. + * + * {{{ + * import org.apache.spark.sql.expressions.Window + * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) + * .toDF("id", "category") + * val byCategoryOrderedById = + * Window.partitionBy('category).orderBy('id).rangeBetween(currentRow(), lit(1)) + * df.withColumn("sum", sum('id) over byCategoryOrderedById).show() + * + * +---+--------+---+ + * | id|category|sum| + * +---+--------+---+ + * | 1| b| 3| + * | 2| b| 5| + * | 3| b| 3| + * | 1| a| 4| + * | 1| a| 4| + * | 2| a| 2| + * +---+--------+---+ + * }}} + * + * @param start boundary start, inclusive. The frame is unbounded if the expression is + * [[org.apache.spark.sql.catalyst.expressions.UnboundedPreceding]]. + * @param end boundary end, inclusive. The frame is unbounded if the expression is + * [[org.apache.spark.sql.catalyst.expressions.UnboundedFollowing]]. + * @since 2.3.0 + */ + def rangeBetween(start: Column, end: Column): WindowSpec = { + spec.rangeBetween(start, end) + } + + private[sql] def spec: WindowSpec = { new WindowSpec(Seq.empty, Seq.empty, UnspecifiedFrame) } } /** - * :: Experimental :: * Utility functions for defining window in DataFrames. * * {{{ * // PARTITION BY country ORDER BY date ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW - * Window.partitionBy("country").orderBy("date").rowsBetween(Long.MinValue, 0) + * Window.partitionBy("country").orderBy("date") + * .rowsBetween(Window.unboundedPreceding, Window.currentRow) * * // PARTITION BY country ORDER BY date ROWS BETWEEN 3 PRECEDING AND 3 FOLLOWING * Window.partitionBy("country").orderBy("date").rowsBetween(-3, 3) @@ -94,5 +281,5 @@ object Window { * * @since 1.4.0 */ -@Experimental +@InterfaceStability.Stable class Window private() // So we can see Window in JavaDoc. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala index 8b9247adea20..4c41aa3c5fb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala @@ -17,25 +17,22 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.types.BooleanType -import org.apache.spark.sql.{Column, catalyst} +import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.sql.{AnalysisException, Column} import org.apache.spark.sql.catalyst.expressions._ - /** - * :: Experimental :: * A window specification that defines the partitioning, ordering, and frame boundaries. * * Use the static methods in [[Window]] to create a [[WindowSpec]]. * * @since 1.4.0 */ -@Experimental +@InterfaceStability.Stable class WindowSpec private[sql]( partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], - frame: catalyst.expressions.WindowFrame) { + frame: WindowFrame) { /** * Defines the partitioning columns in a [[WindowSpec]]. @@ -88,93 +85,188 @@ class WindowSpec private[sql]( * "current row", while "-1" means the row before the current row, and "5" means the fifth row * after the current row. * - * @param start boundary start, inclusive. - * The frame is unbounded if this is the minimum long value. - * @param end boundary end, inclusive. - * The frame is unbounded if this is the maximum long value. + * We recommend users use `Window.unboundedPreceding`, `Window.unboundedFollowing`, + * and `Window.currentRow` to specify special boundary values, rather than using integral + * values directly. + * + * A row based boundary is based on the position of the row within the partition. + * An offset indicates the number of rows above or below the current row, the frame for the + * current row starts or ends. For instance, given a row based sliding frame with a lower bound + * offset of -1 and a upper bound offset of +2. The frame for row with index 5 would range from + * index 4 to index 6. + * + * {{{ + * import org.apache.spark.sql.expressions.Window + * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) + * .toDF("id", "category") + * val byCategoryOrderedById = + * Window.partitionBy('category).orderBy('id).rowsBetween(Window.currentRow, 1) + * df.withColumn("sum", sum('id) over byCategoryOrderedById).show() + * + * +---+--------+---+ + * | id|category|sum| + * +---+--------+---+ + * | 1| b| 3| + * | 2| b| 5| + * | 3| b| 3| + * | 1| a| 2| + * | 1| a| 3| + * | 2| a| 2| + * +---+--------+---+ + * }}} + * + * @param start boundary start, inclusive. The frame is unbounded if this is + * the minimum long value (`Window.unboundedPreceding`). + * @param end boundary end, inclusive. The frame is unbounded if this is the + * maximum long value (`Window.unboundedFollowing`). * @since 1.4.0 */ + // Note: when updating the doc for this method, also update Window.rowsBetween. def rowsBetween(start: Long, end: Long): WindowSpec = { - between(RowFrame, start, end) + val boundaryStart = start match { + case 0 => CurrentRow + case Long.MinValue => UnboundedPreceding + case x if Int.MinValue <= x && x <= Int.MaxValue => Literal(x.toInt) + case x => throw new AnalysisException(s"Boundary start is not a valid integer: $x") + } + + val boundaryEnd = end match { + case 0 => CurrentRow + case Long.MaxValue => UnboundedFollowing + case x if Int.MinValue <= x && x <= Int.MaxValue => Literal(x.toInt) + case x => throw new AnalysisException(s"Boundary end is not a valid integer: $x") + } + + new WindowSpec( + partitionSpec, + orderSpec, + SpecifiedWindowFrame(RowFrame, boundaryStart, boundaryEnd)) } /** * Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). * - * Both `start` and `end` are relative from the current row. For example, "0" means "current row", - * while "-1" means one off before the current row, and "5" means the five off after the - * current row. + * Both `start` and `end` are relative from the current row. For example, "0" means + * "current row", while "-1" means one off before the current row, and "5" means the five off + * after the current row. + * + * We recommend users use `Window.unboundedPreceding`, `Window.unboundedFollowing`, + * and `Window.currentRow` to specify special boundary values, rather than using long values + * directly. + * + * A range-based boundary is based on the actual value of the ORDER BY + * expression(s). An offset is used to alter the value of the ORDER BY expression, for + * instance if the current order by expression has a value of 10 and the lower bound offset + * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a + * number of constraints on the ORDER BY expressions: there can be only one expression and this + * expression must have a numerical data type. An exception can be made when the offset is + * unbounded, because no value modification is needed, in this case multiple and non-numeric + * ORDER BY expression are allowed. * - * @param start boundary start, inclusive. - * The frame is unbounded if this is the minimum long value. - * @param end boundary end, inclusive. - * The frame is unbounded if this is the maximum long value. + * {{{ + * import org.apache.spark.sql.expressions.Window + * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) + * .toDF("id", "category") + * val byCategoryOrderedById = + * Window.partitionBy('category).orderBy('id).rangeBetween(Window.currentRow, 1) + * df.withColumn("sum", sum('id) over byCategoryOrderedById).show() + * + * +---+--------+---+ + * | id|category|sum| + * +---+--------+---+ + * | 1| b| 3| + * | 2| b| 5| + * | 3| b| 3| + * | 1| a| 4| + * | 1| a| 4| + * | 2| a| 2| + * +---+--------+---+ + * }}} + * + * @param start boundary start, inclusive. The frame is unbounded if this is + * the minimum long value (`Window.unboundedPreceding`). + * @param end boundary end, inclusive. The frame is unbounded if this is the + * maximum long value (`Window.unboundedFollowing`). * @since 1.4.0 */ + // Note: when updating the doc for this method, also update Window.rangeBetween. def rangeBetween(start: Long, end: Long): WindowSpec = { - between(RangeFrame, start, end) - } - - private def between(typ: FrameType, start: Long, end: Long): WindowSpec = { val boundaryStart = start match { case 0 => CurrentRow case Long.MinValue => UnboundedPreceding - case x if x < 0 => ValuePreceding(-start.toInt) - case x if x > 0 => ValueFollowing(start.toInt) + case x => Literal(x) } val boundaryEnd = end match { case 0 => CurrentRow case Long.MaxValue => UnboundedFollowing - case x if x < 0 => ValuePreceding(-end.toInt) - case x if x > 0 => ValueFollowing(end.toInt) + case x => Literal(x) } new WindowSpec( partitionSpec, orderSpec, - SpecifiedWindowFrame(typ, boundaryStart, boundaryEnd)) + SpecifiedWindowFrame(RangeFrame, boundaryStart, boundaryEnd)) + } + + /** + * Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). + * + * Both `start` and `end` are relative to the current row. For example, "lit(0)" means + * "current row", while "lit(-1)" means one off before the current row, and "lit(5)" means the + * five off after the current row. + * + * Users should use `unboundedPreceding()`, `unboundedFollowing()`, and `currentRow()` from + * [[org.apache.spark.sql.functions]] to specify special boundary values, literals are not + * transformed to [[org.apache.spark.sql.catalyst.expressions.SpecialFrameBoundary]]s. + * + * A range-based boundary is based on the actual value of the ORDER BY + * expression(s). An offset is used to alter the value of the ORDER BY expression, for + * instance if the current order by expression has a value of 10 and the lower bound offset + * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a + * number of constraints on the ORDER BY expressions: there can be only one expression and this + * expression must have a numerical/date/timestamp data type. An exception can be made when the + * offset is unbounded, because no value modification is needed, in this case multiple and + * non-numerical/date/timestamp data type ORDER BY expression are allowed. + * + * {{{ + * import org.apache.spark.sql.expressions.Window + * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) + * .toDF("id", "category") + * val byCategoryOrderedById = + * Window.partitionBy('category).orderBy('id).rangeBetween(currentRow(), lit(1)) + * df.withColumn("sum", sum('id) over byCategoryOrderedById).show() + * + * +---+--------+---+ + * | id|category|sum| + * +---+--------+---+ + * | 1| b| 3| + * | 2| b| 5| + * | 3| b| 3| + * | 1| a| 4| + * | 1| a| 4| + * | 2| a| 2| + * +---+--------+---+ + * }}} + * + * @param start boundary start, inclusive. The frame is unbounded if the expression is + * [[org.apache.spark.sql.catalyst.expressions.UnboundedPreceding]]. + * @param end boundary end, inclusive. The frame is unbounded if the expression is + * [[org.apache.spark.sql.catalyst.expressions.UnboundedFollowing]]. + * @since 2.3.0 + */ + def rangeBetween(start: Column, end: Column): WindowSpec = { + new WindowSpec( + partitionSpec, + orderSpec, + SpecifiedWindowFrame(RangeFrame, start.expr, end.expr)) } /** * Converts this [[WindowSpec]] into a [[Column]] with an aggregate expression. */ private[sql] def withAggregate(aggregate: Column): Column = { - val windowExpr = aggregate.expr match { - case Average(child) => WindowExpression( - UnresolvedWindowFunction("avg", child :: Nil), - WindowSpecDefinition(partitionSpec, orderSpec, frame)) - case Sum(child) => WindowExpression( - UnresolvedWindowFunction("sum", child :: Nil), - WindowSpecDefinition(partitionSpec, orderSpec, frame)) - case Count(child) => WindowExpression( - UnresolvedWindowFunction("count", child :: Nil), - WindowSpecDefinition(partitionSpec, orderSpec, frame)) - case First(child, ignoreNulls) => WindowExpression( - // TODO this is a hack for Hive UDAF first_value - UnresolvedWindowFunction( - "first_value", - child :: ignoreNulls :: Nil), - WindowSpecDefinition(partitionSpec, orderSpec, frame)) - case Last(child, ignoreNulls) => WindowExpression( - // TODO this is a hack for Hive UDAF last_value - UnresolvedWindowFunction( - "last_value", - child :: ignoreNulls :: Nil), - WindowSpecDefinition(partitionSpec, orderSpec, frame)) - case Min(child) => WindowExpression( - UnresolvedWindowFunction("min", child :: Nil), - WindowSpecDefinition(partitionSpec, orderSpec, frame)) - case Max(child) => WindowExpression( - UnresolvedWindowFunction("max", child :: Nil), - WindowSpecDefinition(partitionSpec, orderSpec, frame)) - case wf: WindowFunction => WindowExpression( - wf, - WindowSpecDefinition(partitionSpec, orderSpec, frame)) - case x => - throw new UnsupportedOperationException(s"$x is not supported in window operation.") - } - new Column(windowExpr) + val spec = WindowSpecDefinition(partitionSpec, orderSpec, frame) + new Column(WindowExpression(aggregate.expr, spec)) } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala new file mode 100644 index 000000000000..650ffd458659 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.expressions.scalalang + +import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.sql._ +import org.apache.spark.sql.execution.aggregate._ + +/** + * :: Experimental :: + * Type-safe functions available for `Dataset` operations in Scala. + * + * Java users should use [[org.apache.spark.sql.expressions.javalang.typed]]. + * + * @since 2.0.0 + */ +@Experimental +@InterfaceStability.Evolving +// scalastyle:off +object typed { + // scalastyle:on + + // Note: whenever we update this file, we should update the corresponding Java version too. + // The reason we have separate files for Java and Scala is because in the Scala version, we can + // use tighter types (primitive types) for return types, whereas in the Java version we can only + // use boxed primitive types. + // For example, avg in the Scala version returns Scala primitive Double, whose bytecode + // signature is just a java.lang.Object; avg in the Java version returns java.lang.Double. + + // TODO: This is pretty hacky. Maybe we should have an object for implicit encoders. + private val implicits = new SQLImplicits { + override protected def _sqlContext: SQLContext = null + } + + import implicits._ + + /** + * Average aggregate function. + * + * @since 2.0.0 + */ + def avg[IN](f: IN => Double): TypedColumn[IN, Double] = new TypedAverage(f).toColumn + + /** + * Count aggregate function. + * + * @since 2.0.0 + */ + def count[IN](f: IN => Any): TypedColumn[IN, Long] = new TypedCount(f).toColumn + + /** + * Sum aggregate function for floating point (double) type. + * + * @since 2.0.0 + */ + def sum[IN](f: IN => Double): TypedColumn[IN, Double] = new TypedSumDouble[IN](f).toColumn + + /** + * Sum aggregate function for integral (long, i.e. 64 bit integer) type. + * + * @since 2.0.0 + */ + def sumLong[IN](f: IN => Long): TypedColumn[IN, Long] = new TypedSumLong[IN](f).toColumn + + // TODO: + // stddevOf: Double + // varianceOf: Double + // approxCountDistinct: Long + + // minOf: T + // maxOf: T + + // firstOf: T + // lastOf: T +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala index 258afadc7695..4976b875fa29 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala @@ -17,23 +17,24 @@ package org.apache.spark.sql.expressions -import org.apache.spark.sql.catalyst.expressions.aggregate.{Complete, AggregateExpression2} -import org.apache.spark.sql.execution.aggregate.ScalaUDAF +import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.{Column, Row} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete} +import org.apache.spark.sql.execution.aggregate.ScalaUDAF import org.apache.spark.sql.types._ -import org.apache.spark.annotation.Experimental /** - * :: Experimental :: * The base class for implementing user-defined aggregate functions (UDAF). + * + * @since 1.5.0 */ -@Experimental +@InterfaceStability.Stable abstract class UserDefinedAggregateFunction extends Serializable { /** - * A [[StructType]] represents data types of input arguments of this aggregate function. + * A `StructType` represents data types of input arguments of this aggregate function. * For example, if a [[UserDefinedAggregateFunction]] expects two input arguments - * with type of [[DoubleType]] and [[LongType]], the returned [[StructType]] will look like + * with type of `DoubleType` and `LongType`, the returned `StructType` will look like * * ``` * new StructType() @@ -41,16 +42,18 @@ abstract class UserDefinedAggregateFunction extends Serializable { * .add("longInput", LongType) * ``` * - * The name of a field of this [[StructType]] is only used to identify the corresponding + * The name of a field of this `StructType` is only used to identify the corresponding * input argument. Users can choose names to identify the input arguments. + * + * @since 1.5.0 */ def inputSchema: StructType /** - * A [[StructType]] represents data types of values in the aggregation buffer. + * A `StructType` represents data types of values in the aggregation buffer. * For example, if a [[UserDefinedAggregateFunction]]'s buffer has two values - * (i.e. two intermediate values) with type of [[DoubleType]] and [[LongType]], - * the returned [[StructType]] will look like + * (i.e. two intermediate values) with type of `DoubleType` and `LongType`, + * the returned `StructType` will look like * * ``` * new StructType() @@ -58,19 +61,25 @@ abstract class UserDefinedAggregateFunction extends Serializable { * .add("longInput", LongType) * ``` * - * The name of a field of this [[StructType]] is only used to identify the corresponding + * The name of a field of this `StructType` is only used to identify the corresponding * buffer value. Users can choose names to identify the input arguments. + * + * @since 1.5.0 */ def bufferSchema: StructType /** - * The [[DataType]] of the returned value of this [[UserDefinedAggregateFunction]]. + * The `DataType` of the returned value of this [[UserDefinedAggregateFunction]]. + * + * @since 1.5.0 */ def dataType: DataType /** * Returns true iff this function is deterministic, i.e. given the same input, * always return the same output. + * + * @since 1.5.0 */ def deterministic: Boolean @@ -80,6 +89,8 @@ abstract class UserDefinedAggregateFunction extends Serializable { * The contract should be that applying the merge function on two initial buffers should just * return the initial buffer itself, i.e. * `merge(initialBuffer, initialBuffer)` should equal `initialBuffer`. + * + * @since 1.5.0 */ def initialize(buffer: MutableAggregationBuffer): Unit @@ -87,6 +98,8 @@ abstract class UserDefinedAggregateFunction extends Serializable { * Updates the given aggregation buffer `buffer` with new input data from `input`. * * This is called once per input row. + * + * @since 1.5.0 */ def update(buffer: MutableAggregationBuffer, input: Row): Unit @@ -94,22 +107,28 @@ abstract class UserDefinedAggregateFunction extends Serializable { * Merges two aggregation buffers and stores the updated buffer values back to `buffer1`. * * This is called when we merge two partially aggregated data together. + * + * @since 1.5.0 */ def merge(buffer1: MutableAggregationBuffer, buffer2: Row): Unit /** * Calculates the final result of this [[UserDefinedAggregateFunction]] based on the given * aggregation buffer. + * + * @since 1.5.0 */ def evaluate(buffer: Row): Any /** - * Creates a [[Column]] for this UDAF using given [[Column]]s as input arguments. + * Creates a `Column` for this UDAF using given `Column`s as input arguments. + * + * @since 1.5.0 */ @scala.annotation.varargs def apply(exprs: Column*): Column = { val aggregateExpression = - AggregateExpression2( + AggregateExpression( ScalaUDAF(exprs.map(_.expr), this), Complete, isDistinct = false) @@ -117,13 +136,15 @@ abstract class UserDefinedAggregateFunction extends Serializable { } /** - * Creates a [[Column]] for this UDAF using the distinct values of the given - * [[Column]]s as input arguments. + * Creates a `Column` for this UDAF using the distinct values of the given + * `Column`s as input arguments. + * + * @since 1.5.0 */ @scala.annotation.varargs def distinct(exprs: Column*): Column = { val aggregateExpression = - AggregateExpression2( + AggregateExpression( ScalaUDAF(exprs.map(_.expr), this), Complete, isDistinct = true) @@ -132,12 +153,13 @@ abstract class UserDefinedAggregateFunction extends Serializable { } /** - * :: Experimental :: - * A [[Row]] representing an mutable aggregation buffer. + * A `Row` representing a mutable aggregation buffer. * * This is not meant to be extended outside of Spark. + * + * @since 1.5.0 */ -@Experimental +@InterfaceStability.Stable abstract class MutableAggregationBuffer extends Row { /** Update the ith value of this buffer. */ 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 c70c965a9b04..6bbdfa3ad189 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 @@ -17,21 +17,29 @@ package org.apache.spark.sql +import scala.collection.JavaConverters._ import scala.language.implicitConversions -import scala.reflect.runtime.universe.{TypeTag, typeTag} +import scala.reflect.runtime.universe.{typeTag, TypeTag} import scala.util.Try +import scala.util.control.NonFatal -import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.catalyst.{SqlParser, ScalaReflection} -import org.apache.spark.sql.catalyst.analysis.{UnresolvedFunction, Star} +import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedFunction} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.BroadcastHint +import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.plans.logical.{HintInfo, ResolvedHint} +import org.apache.spark.sql.execution.SparkSqlParser +import org.apache.spark.sql.expressions.UserDefinedFunction +import org.apache.spark.sql.expressions.Window +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils + /** - * :: Experimental :: - * Functions available for [[DataFrame]]. + * Functions available for DataFrame operations. * * @groupname udf_funcs UDF functions * @groupname agg_funcs Aggregate functions @@ -46,12 +54,18 @@ import org.apache.spark.util.Utils * @groupname Ungrouped Support functions for DataFrames * @since 1.3.0 */ -@Experimental +@InterfaceStability.Stable // scalastyle:off object functions { // scalastyle:on - private[this] implicit def toColumn(expr: Expression): Column = Column(expr) + private def withExpr(expr: Expression): Column = Column(expr) + + private def withAggregateFunction( + func: AggregateFunction, + isDistinct: Boolean = false): Column = { + Column(func.toAggregateExpression(isDistinct)) + } /** * Returns a [[Column]] based on the given column name. @@ -79,15 +93,24 @@ object functions { * @group normal_funcs * @since 1.3.0 */ - def lit(literal: Any): Column = { - literal match { - case c: Column => return c - case s: Symbol => return new ColumnName(literal.asInstanceOf[Symbol].name) - case _ => // continue - } + def lit(literal: Any): Column = typedLit(literal) - val literalExpr = Literal(literal) - Column(literalExpr) + /** + * Creates a [[Column]] of literal value. + * + * The passed in object is returned directly if it is already a [[Column]]. + * If the object is a Scala Symbol, it is converted into a [[Column]] also. + * Otherwise, a new [[Column]] is created to represent the literal value. + * The difference between this function and [[lit]] is that this function + * can handle parameterized scala types e.g.: List, Seq and Map. + * + * @group normal_funcs + * @since 2.2.0 + */ + def typedLit[T : TypeTag](literal: T): Column = literal match { + case c: Column => c + case s: Symbol => new ColumnName(s.name) + case _ => Column(Literal.create(literal)) } ////////////////////////////////////////////////////////////////////////////////////////////// @@ -97,7 +120,6 @@ object functions { /** * Returns a sort expression based on ascending order of the column. * {{{ - * // Sort by dept in ascending order, and then age in descending order. * df.sort(asc("dept"), desc("age")) * }}} * @@ -106,10 +128,33 @@ object functions { */ def asc(columnName: String): Column = Column(columnName).asc + /** + * Returns a sort expression based on ascending order of the column, + * and null values return before non-null values. + * {{{ + * df.sort(asc_nulls_last("dept"), desc("age")) + * }}} + * + * @group sort_funcs + * @since 2.1.0 + */ + def asc_nulls_first(columnName: String): Column = Column(columnName).asc_nulls_first + + /** + * Returns a sort expression based on ascending order of the column, + * and null values appear after non-null values. + * {{{ + * df.sort(asc_nulls_last("dept"), desc("age")) + * }}} + * + * @group sort_funcs + * @since 2.1.0 + */ + def asc_nulls_last(columnName: String): Column = Column(columnName).asc_nulls_last + /** * Returns a sort expression based on the descending order of the column. * {{{ - * // Sort by dept in ascending order, and then age in descending order. * df.sort(asc("dept"), desc("age")) * }}} * @@ -118,42 +163,105 @@ object functions { */ def desc(columnName: String): Column = Column(columnName).desc + /** + * Returns a sort expression based on the descending order of the column, + * and null values appear before non-null values. + * {{{ + * df.sort(asc("dept"), desc_nulls_first("age")) + * }}} + * + * @group sort_funcs + * @since 2.1.0 + */ + def desc_nulls_first(columnName: String): Column = Column(columnName).desc_nulls_first + + /** + * Returns a sort expression based on the descending order of the column, + * and null values appear after non-null values. + * {{{ + * df.sort(asc("dept"), desc_nulls_last("age")) + * }}} + * + * @group sort_funcs + * @since 2.1.0 + */ + def desc_nulls_last(columnName: String): Column = Column(columnName).desc_nulls_last + + ////////////////////////////////////////////////////////////////////////////////////////////// // Aggregate functions ////////////////////////////////////////////////////////////////////////////////////////////// + /** + * @group agg_funcs + * @since 1.3.0 + */ + @deprecated("Use approx_count_distinct", "2.1.0") + def approxCountDistinct(e: Column): Column = approx_count_distinct(e) + + /** + * @group agg_funcs + * @since 1.3.0 + */ + @deprecated("Use approx_count_distinct", "2.1.0") + def approxCountDistinct(columnName: String): Column = approx_count_distinct(columnName) + + /** + * @group agg_funcs + * @since 1.3.0 + */ + @deprecated("Use approx_count_distinct", "2.1.0") + def approxCountDistinct(e: Column, rsd: Double): Column = approx_count_distinct(e, rsd) + + /** + * @group agg_funcs + * @since 1.3.0 + */ + @deprecated("Use approx_count_distinct", "2.1.0") + def approxCountDistinct(columnName: String, rsd: Double): Column = { + approx_count_distinct(Column(columnName), rsd) + } + /** * Aggregate function: returns the approximate number of distinct items in a group. * * @group agg_funcs - * @since 1.3.0 + * @since 2.1.0 */ - def approxCountDistinct(e: Column): Column = ApproxCountDistinct(e.expr) + def approx_count_distinct(e: Column): Column = withAggregateFunction { + HyperLogLogPlusPlus(e.expr) + } /** * Aggregate function: returns the approximate number of distinct items in a group. * * @group agg_funcs - * @since 1.3.0 + * @since 2.1.0 */ - def approxCountDistinct(columnName: String): Column = approxCountDistinct(column(columnName)) + def approx_count_distinct(columnName: String): Column = approx_count_distinct(column(columnName)) /** * Aggregate function: returns the approximate number of distinct items in a group. * + * @param rsd maximum estimation error allowed (default = 0.05) + * * @group agg_funcs - * @since 1.3.0 + * @since 2.1.0 */ - def approxCountDistinct(e: Column, rsd: Double): Column = ApproxCountDistinct(e.expr, rsd) + def approx_count_distinct(e: Column, rsd: Double): Column = withAggregateFunction { + HyperLogLogPlusPlus(e.expr, rsd, 0, 0) + } /** * Aggregate function: returns the approximate number of distinct items in a group. * + * @param rsd maximum estimation error allowed (default = 0.05) + * * @group agg_funcs - * @since 1.3.0 + * @since 2.1.0 */ - def approxCountDistinct(columnName: String, rsd: Double): Column = { - approxCountDistinct(Column(columnName), rsd) + def approx_count_distinct(columnName: String, rsd: Double): Column = { + approx_count_distinct(Column(columnName), rsd) } /** @@ -162,7 +270,7 @@ object functions { * @group agg_funcs * @since 1.3.0 */ - def avg(e: Column): Column = Average(e.expr) + def avg(e: Column): Column = withAggregateFunction { Average(e.expr) } /** * Aggregate function: returns the average of the values in a group. @@ -172,14 +280,47 @@ object functions { */ def avg(columnName: String): Column = avg(Column(columnName)) + /** + * Aggregate function: returns a list of objects with duplicates. + * + * @group agg_funcs + * @since 1.6.0 + */ + def collect_list(e: Column): Column = withAggregateFunction { CollectList(e.expr) } + + /** + * Aggregate function: returns a list of objects with duplicates. + * + * @group agg_funcs + * @since 1.6.0 + */ + def collect_list(columnName: String): Column = collect_list(Column(columnName)) + + /** + * Aggregate function: returns a set of objects with duplicate elements eliminated. + * + * @group agg_funcs + * @since 1.6.0 + */ + def collect_set(e: Column): Column = withAggregateFunction { CollectSet(e.expr) } + + /** + * Aggregate function: returns a set of objects with duplicate elements eliminated. + * + * @group agg_funcs + * @since 1.6.0 + */ + def collect_set(columnName: String): Column = collect_set(Column(columnName)) + /** * Aggregate function: returns the Pearson Correlation Coefficient for two columns. * * @group agg_funcs * @since 1.6.0 */ - def corr(column1: Column, column2: Column): Column = + def corr(column1: Column, column2: Column): Column = withAggregateFunction { Corr(column1.expr, column2.expr) + } /** * Aggregate function: returns the Pearson Correlation Coefficient for two columns. @@ -187,8 +328,9 @@ object functions { * @group agg_funcs * @since 1.6.0 */ - def corr(columnName1: String, columnName2: String): Column = + def corr(columnName1: String, columnName2: String): Column = { corr(Column(columnName1), Column(columnName2)) + } /** * Aggregate function: returns the number of items in a group. @@ -196,10 +338,12 @@ object functions { * @group agg_funcs * @since 1.3.0 */ - def count(e: Column): Column = e.expr match { - // Turn count(*) into count(1) - case s: Star => Count(Literal(1)) - case _ => Count(e.expr) + def count(e: Column): Column = withAggregateFunction { + e.expr match { + // Turn count(*) into count(1) + case s: Star => Count(Literal(1)) + case _ => Count(e.expr) + } } /** @@ -208,7 +352,8 @@ object functions { * @group agg_funcs * @since 1.3.0 */ - def count(columnName: String): Column = count(Column(columnName)) + def count(columnName: String): TypedColumn[Any, Long] = + count(Column(columnName)).as(ExpressionEncoder[Long]()) /** * Aggregate function: returns the number of distinct items in a group. @@ -217,8 +362,9 @@ object functions { * @since 1.3.0 */ @scala.annotation.varargs - def countDistinct(expr: Column, exprs: Column*): Column = - CountDistinct((expr +: exprs).map(_.expr)) + def countDistinct(expr: Column, exprs: Column*): Column = { + withAggregateFunction(Count.apply((expr +: exprs).map(_.expr)), isDistinct = true) + } /** * Aggregate function: returns the number of distinct items in a group. @@ -230,45 +376,206 @@ object functions { def countDistinct(columnName: String, columnNames: String*): Column = countDistinct(Column(columnName), columnNames.map(Column.apply) : _*) + /** + * Aggregate function: returns the population covariance for two columns. + * + * @group agg_funcs + * @since 2.0.0 + */ + def covar_pop(column1: Column, column2: Column): Column = withAggregateFunction { + CovPopulation(column1.expr, column2.expr) + } + + /** + * Aggregate function: returns the population covariance for two columns. + * + * @group agg_funcs + * @since 2.0.0 + */ + def covar_pop(columnName1: String, columnName2: String): Column = { + covar_pop(Column(columnName1), Column(columnName2)) + } + + /** + * Aggregate function: returns the sample covariance for two columns. + * + * @group agg_funcs + * @since 2.0.0 + */ + def covar_samp(column1: Column, column2: Column): Column = withAggregateFunction { + CovSample(column1.expr, column2.expr) + } + + /** + * Aggregate function: returns the sample covariance for two columns. + * + * @group agg_funcs + * @since 2.0.0 + */ + def covar_samp(columnName1: String, columnName2: String): Column = { + covar_samp(Column(columnName1), Column(columnName2)) + } + + /** + * Aggregate function: returns the first value in a group. + * + * The function by default returns the first values it sees. It will return the first non-null + * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. + * + * @group agg_funcs + * @since 2.0.0 + */ + def first(e: Column, ignoreNulls: Boolean): Column = withAggregateFunction { + new First(e.expr, Literal(ignoreNulls)) + } + + /** + * Aggregate function: returns the first value of a column in a group. + * + * The function by default returns the first values it sees. It will return the first non-null + * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. + * + * @group agg_funcs + * @since 2.0.0 + */ + def first(columnName: String, ignoreNulls: Boolean): Column = { + first(Column(columnName), ignoreNulls) + } + /** * Aggregate function: returns the first value in a group. * + * The function by default returns the first values it sees. It will return the first non-null + * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. + * * @group agg_funcs * @since 1.3.0 */ - def first(e: Column): Column = First(e.expr) + def first(e: Column): Column = first(e, ignoreNulls = false) /** * Aggregate function: returns the first value of a column in a group. * + * The function by default returns the first values it sees. It will return the first non-null + * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. + * * @group agg_funcs * @since 1.3.0 */ def first(columnName: String): Column = first(Column(columnName)) + /** + * Aggregate function: indicates whether a specified column in a GROUP BY list is aggregated + * or not, returns 1 for aggregated or 0 for not aggregated in the result set. + * + * @group agg_funcs + * @since 2.0.0 + */ + def grouping(e: Column): Column = Column(Grouping(e.expr)) + + /** + * Aggregate function: indicates whether a specified column in a GROUP BY list is aggregated + * or not, returns 1 for aggregated or 0 for not aggregated in the result set. + * + * @group agg_funcs + * @since 2.0.0 + */ + def grouping(columnName: String): Column = grouping(Column(columnName)) + + /** + * Aggregate function: returns the level of grouping, equals to + * + * {{{ + * (grouping(c1) <<; (n-1)) + (grouping(c2) <<; (n-2)) + ... + grouping(cn) + * }}} + * + * @note The list of columns should match with grouping columns exactly, or empty (means all the + * grouping columns). + * + * @group agg_funcs + * @since 2.0.0 + */ + def grouping_id(cols: Column*): Column = Column(GroupingID(cols.map(_.expr))) + + /** + * Aggregate function: returns the level of grouping, equals to + * + * {{{ + * (grouping(c1) <<; (n-1)) + (grouping(c2) <<; (n-2)) + ... + grouping(cn) + * }}} + * + * @note The list of columns should match with grouping columns exactly. + * + * @group agg_funcs + * @since 2.0.0 + */ + def grouping_id(colName: String, colNames: String*): Column = { + grouping_id((Seq(colName) ++ colNames).map(n => Column(n)) : _*) + } + + /** + * Aggregate function: returns the kurtosis of the values in a group. + * + * @group agg_funcs + * @since 1.6.0 + */ + def kurtosis(e: Column): Column = withAggregateFunction { Kurtosis(e.expr) } + /** * Aggregate function: returns the kurtosis of the values in a group. * * @group agg_funcs * @since 1.6.0 */ - def kurtosis(e: Column): Column = Kurtosis(e.expr) + def kurtosis(columnName: String): Column = kurtosis(Column(columnName)) + + /** + * Aggregate function: returns the last value in a group. + * + * The function by default returns the last values it sees. It will return the last non-null + * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. + * + * @group agg_funcs + * @since 2.0.0 + */ + def last(e: Column, ignoreNulls: Boolean): Column = withAggregateFunction { + new Last(e.expr, Literal(ignoreNulls)) + } + + /** + * Aggregate function: returns the last value of the column in a group. + * + * The function by default returns the last values it sees. It will return the last non-null + * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. + * + * @group agg_funcs + * @since 2.0.0 + */ + def last(columnName: String, ignoreNulls: Boolean): Column = { + last(Column(columnName), ignoreNulls) + } /** * Aggregate function: returns the last value in a group. * + * The function by default returns the last values it sees. It will return the last non-null + * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. + * * @group agg_funcs * @since 1.3.0 */ - def last(e: Column): Column = Last(e.expr) + def last(e: Column): Column = last(e, ignoreNulls = false) /** * Aggregate function: returns the last value of the column in a group. * + * The function by default returns the last values it sees. It will return the last non-null + * value it sees when ignoreNulls is set to true. If all values are null, then null is returned. + * * @group agg_funcs * @since 1.3.0 */ - def last(columnName: String): Column = last(Column(columnName)) + def last(columnName: String): Column = last(Column(columnName), ignoreNulls = false) /** * Aggregate function: returns the maximum value of the expression in a group. @@ -276,7 +583,7 @@ object functions { * @group agg_funcs * @since 1.3.0 */ - def max(e: Column): Column = Max(e.expr) + def max(e: Column): Column = withAggregateFunction { Max(e.expr) } /** * Aggregate function: returns the maximum value of the column in a group. @@ -310,7 +617,7 @@ object functions { * @group agg_funcs * @since 1.3.0 */ - def min(e: Column): Column = Min(e.expr) + def min(e: Column): Column = withAggregateFunction { Min(e.expr) } /** * Aggregate function: returns the minimum value of the column in a group. @@ -326,24 +633,58 @@ object functions { * @group agg_funcs * @since 1.6.0 */ - def skewness(e: Column): Column = Skewness(e.expr) + def skewness(e: Column): Column = withAggregateFunction { Skewness(e.expr) } + + /** + * Aggregate function: returns the skewness of the values in a group. + * + * @group agg_funcs + * @since 1.6.0 + */ + def skewness(columnName: String): Column = skewness(Column(columnName)) + + /** + * Aggregate function: alias for `stddev_samp`. + * + * @group agg_funcs + * @since 1.6.0 + */ + def stddev(e: Column): Column = withAggregateFunction { StddevSamp(e.expr) } + + /** + * Aggregate function: alias for `stddev_samp`. + * + * @group agg_funcs + * @since 1.6.0 + */ + def stddev(columnName: String): Column = stddev(Column(columnName)) /** - * Aggregate function: alias for [[stddev_samp]]. + * Aggregate function: returns the sample standard deviation of + * the expression in a group. + * + * @group agg_funcs + * @since 1.6.0 + */ + def stddev_samp(e: Column): Column = withAggregateFunction { StddevSamp(e.expr) } + + /** + * Aggregate function: returns the sample standard deviation of + * the expression in a group. * * @group agg_funcs * @since 1.6.0 */ - def stddev(e: Column): Column = StddevSamp(e.expr) + def stddev_samp(columnName: String): Column = stddev_samp(Column(columnName)) /** - * Aggregate function: returns the unbiased sample standard deviation of + * Aggregate function: returns the population standard deviation of * the expression in a group. * * @group agg_funcs * @since 1.6.0 */ - def stddev_samp(e: Column): Column = StddevSamp(e.expr) + def stddev_pop(e: Column): Column = withAggregateFunction { StddevPop(e.expr) } /** * Aggregate function: returns the population standard deviation of @@ -352,7 +693,7 @@ object functions { * @group agg_funcs * @since 1.6.0 */ - def stddev_pop(e: Column): Column = StddevPop(e.expr) + def stddev_pop(columnName: String): Column = stddev_pop(Column(columnName)) /** * Aggregate function: returns the sum of all values in the expression. @@ -360,7 +701,7 @@ object functions { * @group agg_funcs * @since 1.3.0 */ - def sum(e: Column): Column = Sum(e.expr) + def sum(e: Column): Column = withAggregateFunction { Sum(e.expr) } /** * Aggregate function: returns the sum of all values in the given column. @@ -376,7 +717,7 @@ object functions { * @group agg_funcs * @since 1.3.0 */ - def sumDistinct(e: Column): Column = SumDistinct(e.expr) + def sumDistinct(e: Column): Column = withAggregateFunction(Sum(e.expr), isDistinct = true) /** * Aggregate function: returns the sum of distinct values in the expression. @@ -387,12 +728,28 @@ object functions { def sumDistinct(columnName: String): Column = sumDistinct(Column(columnName)) /** - * Aggregate function: alias for [[var_samp]]. + * Aggregate function: alias for `var_samp`. + * + * @group agg_funcs + * @since 1.6.0 + */ + def variance(e: Column): Column = withAggregateFunction { VarianceSamp(e.expr) } + + /** + * Aggregate function: alias for `var_samp`. + * + * @group agg_funcs + * @since 1.6.0 + */ + def variance(columnName: String): Column = variance(Column(columnName)) + + /** + * Aggregate function: returns the unbiased variance of the values in a group. * * @group agg_funcs * @since 1.6.0 */ - def variance(e: Column): Column = VarianceSamp(e.expr) + def var_samp(e: Column): Column = withAggregateFunction { VarianceSamp(e.expr) } /** * Aggregate function: returns the unbiased variance of the values in a group. @@ -400,7 +757,15 @@ object functions { * @group agg_funcs * @since 1.6.0 */ - def var_samp(e: Column): Column = VarianceSamp(e.expr) + def var_samp(columnName: String): Column = var_samp(Column(columnName)) + + /** + * Aggregate function: returns the population variance of the values in a group. + * + * @group agg_funcs + * @since 1.6.0 + */ + def var_pop(e: Column): Column = withAggregateFunction { VariancePop(e.expr) } /** * Aggregate function: returns the population variance of the values in a group. @@ -408,47 +773,67 @@ object functions { * @group agg_funcs * @since 1.6.0 */ - def var_pop(e: Column): Column = VariancePop(e.expr) + def var_pop(columnName: String): Column = var_pop(Column(columnName)) ////////////////////////////////////////////////////////////////////////////////////////////// // Window functions ////////////////////////////////////////////////////////////////////////////////////////////// - /** - * Window function: returns the cumulative distribution of values within a window partition, - * i.e. the fraction of rows that are below the current row. - * - * {{{ - * N = total number of rows in the partition - * cumeDist(x) = number of values before (and including) x / N - * }}} - * - * - * This is equivalent to the CUME_DIST function in SQL. + * Window function: returns the special frame boundary that represents the first row in the + * window partition. * * @group window_funcs - * @since 1.4.0 + * @since 2.3.0 */ - def cumeDist(): Column = { - UnresolvedWindowFunction("cume_dist", Nil) - } + def unboundedPreceding(): Column = Column(UnboundedPreceding) + + /** + * Window function: returns the special frame boundary that represents the last row in the + * window partition. + * + * @group window_funcs + * @since 2.3.0 + */ + def unboundedFollowing(): Column = Column(UnboundedFollowing) + + /** + * Window function: returns the special frame boundary that represents the current row in the + * window partition. + * + * @group window_funcs + * @since 2.3.0 + */ + def currentRow(): Column = Column(CurrentRow) + + /** + * Window function: returns the cumulative distribution of values within a window partition, + * i.e. the fraction of rows that are below the current row. + * + * {{{ + * N = total number of rows in the partition + * cumeDist(x) = number of values before (and including) x / N + * }}} + * + * @group window_funcs + * @since 1.6.0 + */ + def cume_dist(): Column = withExpr { new CumeDist } /** * Window function: returns the rank of rows within a window partition, without any gaps. * - * The difference between rank and denseRank is that denseRank leaves no gaps in ranking - * sequence when there are ties. That is, if you were ranking a competition using denseRank + * The difference between rank and dense_rank is that denseRank leaves no gaps in ranking + * sequence when there are ties. That is, if you were ranking a competition using dense_rank * and had three people tie for second place, you would say that all three were in second - * place and that the next person came in third. + * place and that the next person came in third. Rank would give me sequential numbers, making + * the person that came in third place (after the ties) would register as coming in fifth. * * This is equivalent to the DENSE_RANK function in SQL. * * @group window_funcs - * @since 1.4.0 + * @since 1.6.0 */ - def denseRank(): Column = { - UnresolvedWindowFunction("dense_rank", Nil) - } + def dense_rank(): Column = withExpr { new DenseRank } /** * Window function: returns the value that is `offset` rows before the current row, and @@ -460,9 +845,7 @@ object functions { * @group window_funcs * @since 1.4.0 */ - def lag(e: Column, offset: Int): Column = { - lag(e, offset, null) - } + def lag(e: Column, offset: Int): Column = lag(e, offset, null) /** * Window function: returns the value that is `offset` rows before the current row, and @@ -474,9 +857,7 @@ object functions { * @group window_funcs * @since 1.4.0 */ - def lag(columnName: String, offset: Int): Column = { - lag(columnName, offset, null) - } + def lag(columnName: String, offset: Int): Column = lag(columnName, offset, null) /** * Window function: returns the value that is `offset` rows before the current row, and @@ -502,8 +883,8 @@ object functions { * @group window_funcs * @since 1.4.0 */ - def lag(e: Column, offset: Int, defaultValue: Any): Column = { - UnresolvedWindowFunction("lag", e.expr :: Literal(offset) :: Literal(defaultValue) :: Nil) + def lag(e: Column, offset: Int, defaultValue: Any): Column = withExpr { + Lag(e.expr, Literal(offset), Literal(defaultValue)) } /** @@ -516,9 +897,7 @@ object functions { * @group window_funcs * @since 1.4.0 */ - def lead(columnName: String, offset: Int): Column = { - lead(columnName, offset, null) - } + def lead(columnName: String, offset: Int): Column = { lead(columnName, offset, null) } /** * Window function: returns the value that is `offset` rows after the current row, and @@ -530,9 +909,7 @@ object functions { * @group window_funcs * @since 1.4.0 */ - def lead(e: Column, offset: Int): Column = { - lead(e, offset, null) - } + def lead(e: Column, offset: Int): Column = { lead(e, offset, null) } /** * Window function: returns the value that is `offset` rows after the current row, and @@ -558,13 +935,13 @@ object functions { * @group window_funcs * @since 1.4.0 */ - def lead(e: Column, offset: Int, defaultValue: Any): Column = { - UnresolvedWindowFunction("lead", e.expr :: Literal(offset) :: Literal(defaultValue) :: Nil) + def lead(e: Column, offset: Int, defaultValue: Any): Column = withExpr { + Lead(e.expr, Literal(offset), Literal(defaultValue)) } /** * Window function: returns the ntile group id (from 1 to `n` inclusive) in an ordered window - * partition. Fow example, if `n` is 4, the first quarter of the rows will get value 1, the second + * partition. For example, if `n` is 4, the first quarter of the rows will get value 1, the second * quarter will get 2, the third quarter will get 3, and the last quarter will get 4. * * This is equivalent to the NTILE function in SQL. @@ -572,9 +949,7 @@ object functions { * @group window_funcs * @since 1.4.0 */ - def ntile(n: Int): Column = { - UnresolvedWindowFunction("ntile", lit(n).expr :: Nil) - } + def ntile(n: Int): Column = withExpr { new NTile(Literal(n)) } /** * Window function: returns the relative rank (i.e. percentile) of rows within a window partition. @@ -587,40 +962,33 @@ object functions { * This is equivalent to the PERCENT_RANK function in SQL. * * @group window_funcs - * @since 1.4.0 + * @since 1.6.0 */ - def percentRank(): Column = { - UnresolvedWindowFunction("percent_rank", Nil) - } + def percent_rank(): Column = withExpr { new PercentRank } /** * Window function: returns the rank of rows within a window partition. * - * The difference between rank and denseRank is that denseRank leaves no gaps in ranking - * sequence when there are ties. That is, if you were ranking a competition using denseRank + * The difference between rank and dense_rank is that dense_rank leaves no gaps in ranking + * sequence when there are ties. That is, if you were ranking a competition using dense_rank * and had three people tie for second place, you would say that all three were in second - * place and that the next person came in third. + * place and that the next person came in third. Rank would give me sequential numbers, making + * the person that came in third place (after the ties) would register as coming in fifth. * * This is equivalent to the RANK function in SQL. * * @group window_funcs * @since 1.4.0 */ - def rank(): Column = { - UnresolvedWindowFunction("rank", Nil) - } + def rank(): Column = withExpr { new Rank } /** * Window function: returns a sequential number starting at 1 within a window partition. * - * This is equivalent to the ROW_NUMBER function in SQL. - * * @group window_funcs - * @since 1.4.0 + * @since 1.6.0 */ - def rowNumber(): Column = { - UnresolvedWindowFunction("row_number", Nil) - } + def row_number(): Column = withExpr { RowNumber() } ////////////////////////////////////////////////////////////////////////////////////////////// // Non-aggregate functions @@ -632,7 +1000,7 @@ object functions { * @group normal_funcs * @since 1.3.0 */ - def abs(e: Column): Column = Abs(e.expr) + def abs(e: Column): Column = withExpr { Abs(e.expr) } /** * Creates a new array column. The input columns must all have the same data type. @@ -641,7 +1009,7 @@ object functions { * @since 1.4.0 */ @scala.annotation.varargs - def array(cols: Column*): Column = CreateArray(cols.map(_.expr)) + def array(cols: Column*): Column = withExpr { CreateArray(cols.map(_.expr)) } /** * Creates a new array column. The input columns must all have the same data type. @@ -649,10 +1017,22 @@ object functions { * @group normal_funcs * @since 1.4.0 */ + @scala.annotation.varargs def array(colName: String, colNames: String*): Column = { array((colName +: colNames).map(col) : _*) } + /** + * Creates a new map column. The input columns must be grouped as key-value pairs, e.g. + * (key1, value1, key2, value2, ...). The key columns must all have the same data type, and can't + * be null. The value columns must all have the same data type. + * + * @group normal_funcs + * @since 2.0 + */ + @scala.annotation.varargs + def map(cols: Column*): Column = withExpr { CreateMap(cols.map(_.expr)) } + /** * Marks a DataFrame as small enough for use in broadcast joins. * @@ -665,8 +1045,9 @@ object functions { * @group normal_funcs * @since 1.5.0 */ - def broadcast(df: DataFrame): DataFrame = { - DataFrame(df.sqlContext, BroadcastHint(df.logicalPlan)) + def broadcast[T](df: Dataset[T]): Dataset[T] = { + Dataset[T](df.sparkSession, + ResolvedHint(df.logicalPlan, HintInfo(broadcast = true)))(df.exprEnc) } /** @@ -679,22 +1060,31 @@ object functions { * @since 1.3.0 */ @scala.annotation.varargs - def coalesce(e: Column*): Column = Coalesce(e.map(_.expr)) + def coalesce(e: Column*): Column = withExpr { Coalesce(e.map(_.expr)) } /** * Creates a string column for the file name of the current Spark task. * * @group normal_funcs + * @since 1.6.0 */ - def inputFileName(): Column = InputFileName() + def input_file_name(): Column = withExpr { InputFileName() } /** * Return true iff the column is NaN. * * @group normal_funcs - * @since 1.5.0 + * @since 1.6.0 + */ + def isnan(e: Column): Column = withExpr { IsNaN(e.expr) } + + /** + * Return true iff the column is null. + * + * @group normal_funcs + * @since 1.6.0 */ - def isNaN(e: Column): Column = IsNaN(e.expr) + def isnull(e: Column): Column = withExpr { IsNull(e.expr) } /** * A column expression that generates monotonically increasing 64-bit integers. @@ -704,14 +1094,38 @@ object functions { * within each partition in the lower 33 bits. The assumption is that the data frame has * less than 1 billion partitions, and each partition has less than 8 billion records. * - * As an example, consider a [[DataFrame]] with two partitions, each with 3 records. + * As an example, consider a `DataFrame` with two partitions, each with 3 records. * This expression would return the following IDs: + * + * {{{ * 0, 1, 2, 8589934592 (1L << 33), 8589934593, 8589934594. + * }}} * * @group normal_funcs * @since 1.4.0 */ - def monotonicallyIncreasingId(): Column = MonotonicallyIncreasingID() + @deprecated("Use monotonically_increasing_id()", "2.0.0") + def monotonicallyIncreasingId(): Column = monotonically_increasing_id() + + /** + * A column expression that generates monotonically increasing 64-bit integers. + * + * The generated ID is guaranteed to be monotonically increasing and unique, but not consecutive. + * The current implementation puts the partition ID in the upper 31 bits, and the record number + * within each partition in the lower 33 bits. The assumption is that the data frame has + * less than 1 billion partitions, and each partition has less than 8 billion records. + * + * As an example, consider a `DataFrame` with two partitions, each with 3 records. + * This expression would return the following IDs: + * + * {{{ + * 0, 1, 2, 8589934592 (1L << 33), 8589934593, 8589934594. + * }}} + * + * @group normal_funcs + * @since 1.6.0 + */ + def monotonically_increasing_id(): Column = withExpr { MonotonicallyIncreasingID() } /** * Returns col1 if it is not NaN, or col2 if col1 is NaN. @@ -721,7 +1135,7 @@ object functions { * @group normal_funcs * @since 1.5.0 */ - def nanvl(col1: Column, col2: Column): Column = NaNvl(col1.expr, col2.expr) + def nanvl(col1: Column, col2: Column): Column = withExpr { NaNvl(col1.expr, col2.expr) } /** * Unary minus, i.e. negate the expression. @@ -755,15 +1169,19 @@ object functions { def not(e: Column): Column = !e /** - * Generate a random column with i.i.d. samples from U[0.0, 1.0]. + * Generate a random column with independent and identically distributed (i.i.d.) samples + * from U[0.0, 1.0]. + * + * @note This is indeterministic when data partitions are not fixed. * * @group normal_funcs * @since 1.4.0 */ - def rand(seed: Long): Column = Rand(seed) + def rand(seed: Long): Column = withExpr { Rand(seed) } /** - * Generate a random column with i.i.d. samples from U[0.0, 1.0]. + * Generate a random column with independent and identically distributed (i.i.d.) samples + * from U[0.0, 1.0]. * * @group normal_funcs * @since 1.4.0 @@ -771,15 +1189,19 @@ object functions { def rand(): Column = rand(Utils.random.nextLong) /** - * Generate a column with i.i.d. samples from the standard normal distribution. + * Generate a column with independent and identically distributed (i.i.d.) samples from + * the standard normal distribution. + * + * @note This is indeterministic when data partitions are not fixed. * * @group normal_funcs * @since 1.4.0 */ - def randn(seed: Long): Column = Randn(seed) + def randn(seed: Long): Column = withExpr { Randn(seed) } /** - * Generate a column with i.i.d. samples from the standard normal distribution. + * Generate a column with independent and identically distributed (i.i.d.) samples from + * the standard normal distribution. * * @group normal_funcs * @since 1.4.0 @@ -787,14 +1209,14 @@ object functions { def randn(): Column = randn(Utils.random.nextLong) /** - * Partition ID of the Spark task. + * Partition ID. * - * Note that this is indeterministic because it depends on data partitioning and task scheduling. + * @note This is indeterministic because it depends on data partitioning and task scheduling. * * @group normal_funcs - * @since 1.4.0 + * @since 1.6.0 */ - def sparkPartitionId(): Column = SparkPartitionID() + def spark_partition_id(): Column = withExpr { SparkPartitionID() } /** * Computes the square root of the specified float value. @@ -802,7 +1224,7 @@ object functions { * @group math_funcs * @since 1.3.0 */ - def sqrt(e: Column): Column = Sqrt(e.expr) + def sqrt(e: Column): Column = withExpr { Sqrt(e.expr) } /** * Computes the square root of the specified float value. @@ -814,18 +1236,16 @@ object functions { /** * Creates a new struct column. - * If the input column is a column in a [[DataFrame]], or a derived column expression - * that is named (i.e. aliased), its name would be remained as the StructField's name, - * otherwise, the newly generated StructField's name would be auto generated as col${index + 1}, - * i.e. col1, col2, col3, ... + * If the input column is a column in a `DataFrame`, or a derived column expression + * that is named (i.e. aliased), its name would be retained as the StructField's name, + * otherwise, the newly generated StructField's name would be auto generated as + * `col` with a suffix `index + 1`, i.e. col1, col2, col3, ... * * @group normal_funcs * @since 1.4.0 */ @scala.annotation.varargs - def struct(cols: Column*): Column = { - CreateStruct(cols.map(_.expr)) - } + def struct(cols: Column*): Column = withExpr { CreateStruct(cols.map(_.expr)) } /** * Creates a new struct column that composes multiple input columns. @@ -833,6 +1253,7 @@ object functions { * @group normal_funcs * @since 1.4.0 */ + @scala.annotation.varargs def struct(colName: String, colNames: String*): Column = { struct((colName +: colNames).map(col) : _*) } @@ -858,8 +1279,8 @@ object functions { * @group normal_funcs * @since 1.4.0 */ - def when(condition: Column, value: Any): Column = { - CaseWhen(Seq(condition.expr, lit(value).expr)) + def when(condition: Column, value: Any): Column = withExpr { + CaseWhen(Seq((condition.expr, lit(value).expr))) } /** @@ -868,11 +1289,11 @@ object functions { * @group normal_funcs * @since 1.4.0 */ - def bitwiseNOT(e: Column): Column = BitwiseNot(e.expr) + def bitwiseNOT(e: Column): Column = withExpr { BitwiseNot(e.expr) } /** * Parses the expression string into the column that it represents, similar to - * DataFrame.selectExpr + * [[Dataset#selectExpr]]. * {{{ * // get the number of words of each length * df.groupBy(expr("length(word)")).count() @@ -880,7 +1301,12 @@ object functions { * * @group normal_funcs */ - def expr(expr: String): Column = Column(SqlParser.parseExpression(expr)) + def expr(expr: String): Column = { + val parser = SparkSession.getActiveSession.map(_.sessionState.sqlParser).getOrElse { + new SparkSqlParser(new SQLConf) + } + Column(parser.parseExpression(expr)) + } ////////////////////////////////////////////////////////////////////////////////////////////// // Math Functions @@ -893,7 +1319,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def acos(e: Column): Column = Acos(e.expr) + def acos(e: Column): Column = withExpr { Acos(e.expr) } /** * Computes the cosine inverse of the given column; the returned angle is in the range @@ -911,7 +1337,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def asin(e: Column): Column = Asin(e.expr) + def asin(e: Column): Column = withExpr { Asin(e.expr) } /** * Computes the sine inverse of the given column; the returned angle is in the range @@ -923,15 +1349,17 @@ object functions { def asin(columnName: String): Column = asin(Column(columnName)) /** - * Computes the tangent inverse of the given value. + * Computes the tangent inverse of the given column; the returned angle is in the range + * -pi/2 through pi/2 * * @group math_funcs * @since 1.4.0 */ - def atan(e: Column): Column = Atan(e.expr) + def atan(e: Column): Column = withExpr { Atan(e.expr) } /** - * Computes the tangent inverse of the given column. + * Computes the tangent inverse of the given column; the returned angle is in the range + * -pi/2 through pi/2 * * @group math_funcs * @since 1.4.0 @@ -940,12 +1368,12 @@ object functions { /** * Returns the angle theta from the conversion of rectangular coordinates (x, y) to - * polar coordinates (r, theta). + * polar coordinates (r, theta). Units in radians. * * @group math_funcs * @since 1.4.0 */ - def atan2(l: Column, r: Column): Column = Atan2(l.expr, r.expr) + def atan2(l: Column, r: Column): Column = withExpr { Atan2(l.expr, r.expr) } /** * Returns the angle theta from the conversion of rectangular coordinates (x, y) to @@ -982,7 +1410,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def atan2(l: Column, r: Double): Column = atan2(l, lit(r).expr) + def atan2(l: Column, r: Double): Column = atan2(l, lit(r)) /** * Returns the angle theta from the conversion of rectangular coordinates (x, y) to @@ -1000,7 +1428,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def atan2(l: Double, r: Column): Column = atan2(lit(l).expr, r) + def atan2(l: Double, r: Column): Column = atan2(lit(l), r) /** * Returns the angle theta from the conversion of rectangular coordinates (x, y) to @@ -1018,7 +1446,7 @@ object functions { * @group math_funcs * @since 1.5.0 */ - def bin(e: Column): Column = Bin(e.expr) + def bin(e: Column): Column = withExpr { Bin(e.expr) } /** * An expression that returns the string representation of the binary value of the given long @@ -1035,7 +1463,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def cbrt(e: Column): Column = Cbrt(e.expr) + def cbrt(e: Column): Column = withExpr { Cbrt(e.expr) } /** * Computes the cube-root of the given column. @@ -1051,7 +1479,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def ceil(e: Column): Column = Ceil(e.expr) + def ceil(e: Column): Column = withExpr { Ceil(e.expr) } /** * Computes the ceiling of the given column. @@ -1067,16 +1495,17 @@ object functions { * @group math_funcs * @since 1.5.0 */ - def conv(num: Column, fromBase: Int, toBase: Int): Column = + def conv(num: Column, fromBase: Int, toBase: Int): Column = withExpr { Conv(num.expr, lit(fromBase).expr, lit(toBase).expr) + } /** - * Computes the cosine of the given value. + * Computes the cosine of the given value. Units in radians. * * @group math_funcs * @since 1.4.0 */ - def cos(e: Column): Column = Cos(e.expr) + def cos(e: Column): Column = withExpr { Cos(e.expr) } /** * Computes the cosine of the given column. @@ -1092,7 +1521,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def cosh(e: Column): Column = Cosh(e.expr) + def cosh(e: Column): Column = withExpr { Cosh(e.expr) } /** * Computes the hyperbolic cosine of the given column. @@ -1108,7 +1537,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def exp(e: Column): Column = Exp(e.expr) + def exp(e: Column): Column = withExpr { Exp(e.expr) } /** * Computes the exponential of the given column. @@ -1124,7 +1553,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def expm1(e: Column): Column = Expm1(e.expr) + def expm1(e: Column): Column = withExpr { Expm1(e.expr) } /** * Computes the exponential of the given column. @@ -1140,7 +1569,7 @@ object functions { * @group math_funcs * @since 1.5.0 */ - def factorial(e: Column): Column = Factorial(e.expr) + def factorial(e: Column): Column = withExpr { Factorial(e.expr) } /** * Computes the floor of the given value. @@ -1148,7 +1577,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def floor(e: Column): Column = Floor(e.expr) + def floor(e: Column): Column = withExpr { Floor(e.expr) } /** * Computes the floor of the given column. @@ -1166,10 +1595,7 @@ object functions { * @since 1.5.0 */ @scala.annotation.varargs - def greatest(exprs: Column*): Column = { - require(exprs.length > 1, "greatest requires at least 2 arguments.") - Greatest(exprs.map(_.expr)) - } + def greatest(exprs: Column*): Column = withExpr { Greatest(exprs.map(_.expr)) } /** * Returns the greatest value of the list of column names, skipping null values. @@ -1189,7 +1615,7 @@ object functions { * @group math_funcs * @since 1.5.0 */ - def hex(column: Column): Column = Hex(column.expr) + def hex(column: Column): Column = withExpr { Hex(column.expr) } /** * Inverse of hex. Interprets each pair of characters as a hexadecimal number @@ -1198,7 +1624,7 @@ object functions { * @group math_funcs * @since 1.5.0 */ - def unhex(column: Column): Column = Unhex(column.expr) + def unhex(column: Column): Column = withExpr { Unhex(column.expr) } /** * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. @@ -1206,7 +1632,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def hypot(l: Column, r: Column): Column = Hypot(l.expr, r.expr) + def hypot(l: Column, r: Column): Column = withExpr { Hypot(l.expr, r.expr) } /** * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. @@ -1239,7 +1665,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def hypot(l: Column, r: Double): Column = hypot(l, lit(r).expr) + def hypot(l: Column, r: Double): Column = hypot(l, lit(r)) /** * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. @@ -1255,7 +1681,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def hypot(l: Double, r: Column): Column = hypot(lit(l).expr, r) + def hypot(l: Double, r: Column): Column = hypot(lit(l), r) /** * Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. @@ -1273,10 +1699,7 @@ object functions { * @since 1.5.0 */ @scala.annotation.varargs - def least(exprs: Column*): Column = { - require(exprs.length > 1, "least requires at least 2 arguments.") - Least(exprs.map(_.expr)) - } + def least(exprs: Column*): Column = withExpr { Least(exprs.map(_.expr)) } /** * Returns the least value of the list of column names, skipping null values. @@ -1296,7 +1719,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def log(e: Column): Column = Log(e.expr) + def log(e: Column): Column = withExpr { Log(e.expr) } /** * Computes the natural logarithm of the given column. @@ -1312,7 +1735,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def log(base: Double, a: Column): Column = Logarithm(lit(base).expr, a.expr) + def log(base: Double, a: Column): Column = withExpr { Logarithm(lit(base).expr, a.expr) } /** * Returns the first argument-base logarithm of the second argument. @@ -1328,7 +1751,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def log10(e: Column): Column = Log10(e.expr) + def log10(e: Column): Column = withExpr { Log10(e.expr) } /** * Computes the logarithm of the given value in base 10. @@ -1344,7 +1767,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def log1p(e: Column): Column = Log1p(e.expr) + def log1p(e: Column): Column = withExpr { Log1p(e.expr) } /** * Computes the natural logarithm of the given column plus one. @@ -1360,7 +1783,7 @@ object functions { * @group math_funcs * @since 1.5.0 */ - def log2(expr: Column): Column = Log2(expr.expr) + def log2(expr: Column): Column = withExpr { Log2(expr.expr) } /** * Computes the logarithm of the given value in base 2. @@ -1376,7 +1799,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def pow(l: Column, r: Column): Column = Pow(l.expr, r.expr) + def pow(l: Column, r: Column): Column = withExpr { Pow(l.expr, r.expr) } /** * Returns the value of the first argument raised to the power of the second argument. @@ -1408,7 +1831,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def pow(l: Column, r: Double): Column = pow(l, lit(r).expr) + def pow(l: Column, r: Double): Column = pow(l, lit(r)) /** * Returns the value of the first argument raised to the power of the second argument. @@ -1424,7 +1847,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def pow(l: Double, r: Column): Column = pow(lit(l).expr, r) + def pow(l: Double, r: Column): Column = pow(lit(l), r) /** * Returns the value of the first argument raised to the power of the second argument. @@ -1440,7 +1863,9 @@ object functions { * @group math_funcs * @since 1.5.0 */ - def pmod(dividend: Column, divisor: Column): Column = Pmod(dividend.expr, divisor.expr) + def pmod(dividend: Column, divisor: Column): Column = withExpr { + Pmod(dividend.expr, divisor.expr) + } /** * Returns the double value that is closest in value to the argument and @@ -1449,7 +1874,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def rint(e: Column): Column = Rint(e.expr) + def rint(e: Column): Column = withExpr { Rint(e.expr) } /** * Returns the double value that is closest in value to the argument and @@ -1461,49 +1886,69 @@ object functions { def rint(columnName: String): Column = rint(Column(columnName)) /** - * Returns the value of the column `e` rounded to 0 decimal places. + * Returns the value of the column `e` rounded to 0 decimal places with HALF_UP round mode. * * @group math_funcs * @since 1.5.0 */ - def round(e: Column): Column = round(e.expr, 0) + def round(e: Column): Column = round(e, 0) /** - * Round the value of `e` to `scale` decimal places if `scale` >= 0 - * or at integral part when `scale` < 0. + * Round the value of `e` to `scale` decimal places with HALF_UP round mode + * if `scale` is greater than or equal to 0 or at integral part when `scale` is less than 0. * * @group math_funcs * @since 1.5.0 */ - def round(e: Column, scale: Int): Column = Round(e.expr, Literal(scale)) + def round(e: Column, scale: Int): Column = withExpr { Round(e.expr, Literal(scale)) } + + /** + * Returns the value of the column `e` rounded to 0 decimal places with HALF_EVEN round mode. + * + * @group math_funcs + * @since 2.0.0 + */ + def bround(e: Column): Column = bround(e, 0) /** - * Shift the the given value numBits left. If the given value is a long value, this function + * Round the value of `e` to `scale` decimal places with HALF_EVEN round mode + * if `scale` is greater than or equal to 0 or at integral part when `scale` is less than 0. + * + * @group math_funcs + * @since 2.0.0 + */ + def bround(e: Column, scale: Int): Column = withExpr { BRound(e.expr, Literal(scale)) } + + /** + * Shift the given value numBits left. If the given value is a long value, this function * will return a long value else it will return an integer value. * * @group math_funcs * @since 1.5.0 */ - def shiftLeft(e: Column, numBits: Int): Column = ShiftLeft(e.expr, lit(numBits).expr) + def shiftLeft(e: Column, numBits: Int): Column = withExpr { ShiftLeft(e.expr, lit(numBits).expr) } /** - * Shift the the given value numBits right. If the given value is a long value, it will return - * a long value else it will return an integer value. + * (Signed) shift the given value numBits right. If the given value is a long value, it will + * return a long value else it will return an integer value. * * @group math_funcs * @since 1.5.0 */ - def shiftRight(e: Column, numBits: Int): Column = ShiftRight(e.expr, lit(numBits).expr) + def shiftRight(e: Column, numBits: Int): Column = withExpr { + ShiftRight(e.expr, lit(numBits).expr) + } /** - * Unsigned shift the the given value numBits right. If the given value is a long value, + * Unsigned shift the given value numBits right. If the given value is a long value, * it will return a long value else it will return an integer value. * * @group math_funcs * @since 1.5.0 */ - def shiftRightUnsigned(e: Column, numBits: Int): Column = + def shiftRightUnsigned(e: Column, numBits: Int): Column = withExpr { ShiftRightUnsigned(e.expr, lit(numBits).expr) + } /** * Computes the signum of the given value. @@ -1511,7 +1956,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def signum(e: Column): Column = Signum(e.expr) + def signum(e: Column): Column = withExpr { Signum(e.expr) } /** * Computes the signum of the given column. @@ -1522,12 +1967,12 @@ object functions { def signum(columnName: String): Column = signum(Column(columnName)) /** - * Computes the sine of the given value. + * Computes the sine of the given value. Units in radians. * * @group math_funcs * @since 1.4.0 */ - def sin(e: Column): Column = Sin(e.expr) + def sin(e: Column): Column = withExpr { Sin(e.expr) } /** * Computes the sine of the given column. @@ -1543,7 +1988,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def sinh(e: Column): Column = Sinh(e.expr) + def sinh(e: Column): Column = withExpr { Sinh(e.expr) } /** * Computes the hyperbolic sine of the given column. @@ -1554,12 +1999,12 @@ object functions { def sinh(columnName: String): Column = sinh(Column(columnName)) /** - * Computes the tangent of the given value. + * Computes the tangent of the given value. Units in radians. * * @group math_funcs * @since 1.4.0 */ - def tan(e: Column): Column = Tan(e.expr) + def tan(e: Column): Column = withExpr { Tan(e.expr) } /** * Computes the tangent of the given column. @@ -1575,7 +2020,7 @@ object functions { * @group math_funcs * @since 1.4.0 */ - def tanh(e: Column): Column = Tanh(e.expr) + def tanh(e: Column): Column = withExpr { Tanh(e.expr) } /** * Computes the hyperbolic tangent of the given column. @@ -1585,37 +2030,65 @@ object functions { */ def tanh(columnName: String): Column = tanh(Column(columnName)) + /** + * @group math_funcs + * @since 1.4.0 + */ + @deprecated("Use degrees", "2.1.0") + def toDegrees(e: Column): Column = degrees(e) + + /** + * @group math_funcs + * @since 1.4.0 + */ + @deprecated("Use degrees", "2.1.0") + def toDegrees(columnName: String): Column = degrees(Column(columnName)) + /** * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. * * @group math_funcs - * @since 1.4.0 + * @since 2.1.0 */ - def toDegrees(e: Column): Column = ToDegrees(e.expr) + def degrees(e: Column): Column = withExpr { ToDegrees(e.expr) } /** * Converts an angle measured in radians to an approximately equivalent angle measured in degrees. * + * @group math_funcs + * @since 2.1.0 + */ + def degrees(columnName: String): Column = degrees(Column(columnName)) + + /** + * @group math_funcs + * @since 1.4.0 + */ + @deprecated("Use radians", "2.1.0") + def toRadians(e: Column): Column = radians(e) + + /** * @group math_funcs * @since 1.4.0 */ - def toDegrees(columnName: String): Column = toDegrees(Column(columnName)) + @deprecated("Use radians", "2.1.0") + def toRadians(columnName: String): Column = radians(Column(columnName)) /** * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. * * @group math_funcs - * @since 1.4.0 + * @since 2.1.0 */ - def toRadians(e: Column): Column = ToRadians(e.expr) + def radians(e: Column): Column = withExpr { ToRadians(e.expr) } /** * Converts an angle measured in degrees to an approximately equivalent angle measured in radians. * * @group math_funcs - * @since 1.4.0 + * @since 2.1.0 */ - def toRadians(columnName: String): Column = toRadians(Column(columnName)) + def radians(columnName: String): Column = radians(Column(columnName)) ////////////////////////////////////////////////////////////////////////////////////////////// // Misc functions @@ -1628,7 +2101,7 @@ object functions { * @group misc_funcs * @since 1.5.0 */ - def md5(e: Column): Column = Md5(e.expr) + def md5(e: Column): Column = withExpr { Md5(e.expr) } /** * Calculates the SHA-1 digest of a binary column and returns the value @@ -1637,7 +2110,7 @@ object functions { * @group misc_funcs * @since 1.5.0 */ - def sha1(e: Column): Column = Sha1(e.expr) + def sha1(e: Column): Column = withExpr { Sha1(e.expr) } /** * Calculates the SHA-2 family of hash functions of a binary column and @@ -1652,7 +2125,7 @@ object functions { def sha2(e: Column, numBits: Int): Column = { require(Seq(0, 224, 256, 384, 512).contains(numBits), s"numBits $numBits is not in the permitted values (0, 224, 256, 384, 512)") - Sha2(e.expr, lit(numBits).expr) + withExpr { Sha2(e.expr, lit(numBits).expr) } } /** @@ -1662,7 +2135,18 @@ object functions { * @group misc_funcs * @since 1.5.0 */ - def crc32(e: Column): Column = Crc32(e.expr) + def crc32(e: Column): Column = withExpr { Crc32(e.expr) } + + /** + * Calculates the hash code of given columns, and returns the result as an int column. + * + * @group misc_funcs + * @since 2.0.0 + */ + @scala.annotation.varargs + def hash(cols: Column*): Column = withExpr { + new Murmur3Hash(cols.map(_.expr)) + } ////////////////////////////////////////////////////////////////////////////////////////////// // String functions @@ -1670,12 +2154,12 @@ object functions { /** * Computes the numeric value of the first character of the string column, and returns the - * result as a int column. + * result as an int column. * * @group string_funcs * @since 1.5.0 */ - def ascii(e: Column): Column = Ascii(e.expr) + def ascii(e: Column): Column = withExpr { Ascii(e.expr) } /** * Computes the BASE64 encoding of a binary column and returns it as a string column. @@ -1684,7 +2168,7 @@ object functions { * @group string_funcs * @since 1.5.0 */ - def base64(e: Column): Column = Base64(e.expr) + def base64(e: Column): Column = withExpr { Base64(e.expr) } /** * Concatenates multiple input string columns together into a single string column. @@ -1693,7 +2177,7 @@ object functions { * @since 1.5.0 */ @scala.annotation.varargs - def concat(exprs: Column*): Column = Concat(exprs.map(_.expr)) + def concat(exprs: Column*): Column = withExpr { Concat(exprs.map(_.expr)) } /** * Concatenates multiple input string columns together into a single string column, @@ -1703,7 +2187,7 @@ object functions { * @since 1.5.0 */ @scala.annotation.varargs - def concat_ws(sep: String, exprs: Column*): Column = { + def concat_ws(sep: String, exprs: Column*): Column = withExpr { ConcatWs(Literal.create(sep, StringType) +: exprs.map(_.expr)) } @@ -1715,7 +2199,9 @@ object functions { * @group string_funcs * @since 1.5.0 */ - def decode(value: Column, charset: String): Column = Decode(value.expr, lit(charset).expr) + def decode(value: Column, charset: String): Column = withExpr { + Decode(value.expr, lit(charset).expr) + } /** * Computes the first argument into a binary from a string using the provided character set @@ -1725,19 +2211,23 @@ object functions { * @group string_funcs * @since 1.5.0 */ - def encode(value: Column, charset: String): Column = Encode(value.expr, lit(charset).expr) + def encode(value: Column, charset: String): Column = withExpr { + Encode(value.expr, lit(charset).expr) + } /** - * Formats numeric column x to a format like '#,###,###.##', rounded to d decimal places, - * and returns the result as a string column. + * Formats numeric column x to a format like '#,###,###.##', rounded to d decimal places + * with HALF_EVEN round mode, and returns the result as a string column. * * If d is 0, the result has no decimal point or fractional part. - * If d < 0, the result will be null. + * If d is less than 0, the result will be null. * * @group string_funcs * @since 1.5.0 */ - def format_number(x: Column, d: Int): Column = FormatNumber(x.expr, lit(d).expr) + def format_number(x: Column, d: Int): Column = withExpr { + FormatNumber(x.expr, lit(d).expr) + } /** * Formats the arguments in printf-style and returns the result as a string column. @@ -1746,7 +2236,7 @@ object functions { * @since 1.5.0 */ @scala.annotation.varargs - def format_string(format: String, arguments: Column*): Column = { + def format_string(format: String, arguments: Column*): Column = withExpr { FormatString((lit(format) +: arguments).map(_.expr): _*) } @@ -1759,19 +2249,21 @@ object functions { * @group string_funcs * @since 1.5.0 */ - def initcap(e: Column): Column = InitCap(e.expr) + def initcap(e: Column): Column = withExpr { InitCap(e.expr) } /** * Locate the position of the first occurrence of substr column in the given string. * Returns null if either of the arguments are null. * - * NOTE: The position is not zero based, but 1 based index, returns 0 if substr + * @note The position is not zero based, but 1 based index. Returns 0 if substr * could not be found in str. * * @group string_funcs * @since 1.5.0 */ - def instr(str: Column, substring: String): Column = StringInstr(str.expr, lit(substring).expr) + def instr(str: Column, substring: String): Column = withExpr { + StringInstr(str.expr, lit(substring).expr) + } /** * Computes the length of a given string or binary column. @@ -1779,7 +2271,7 @@ object functions { * @group string_funcs * @since 1.5.0 */ - def length(e: Column): Column = Length(e.expr) + def length(e: Column): Column = withExpr { Length(e.expr) } /** * Converts a string column to lower case. @@ -1787,47 +2279,49 @@ object functions { * @group string_funcs * @since 1.3.0 */ - def lower(e: Column): Column = Lower(e.expr) + def lower(e: Column): Column = withExpr { Lower(e.expr) } /** * Computes the Levenshtein distance of the two given string columns. * @group string_funcs * @since 1.5.0 */ - def levenshtein(l: Column, r: Column): Column = Levenshtein(l.expr, r.expr) + def levenshtein(l: Column, r: Column): Column = withExpr { Levenshtein(l.expr, r.expr) } /** * Locate the position of the first occurrence of substr. - * NOTE: The position is not zero based, but 1 based index, returns 0 if substr + * + * @note The position is not zero based, but 1 based index. Returns 0 if substr * could not be found in str. * * @group string_funcs * @since 1.5.0 */ - def locate(substr: String, str: Column): Column = { + def locate(substr: String, str: Column): Column = withExpr { new StringLocate(lit(substr).expr, str.expr) } /** * Locate the position of the first occurrence of substr in a string column, after position pos. * - * NOTE: The position is not zero based, but 1 based index. returns 0 if substr + * @note The position is not zero based, but 1 based index. returns 0 if substr * could not be found in str. * * @group string_funcs * @since 1.5.0 */ - def locate(substr: String, str: Column, pos: Int): Column = { + def locate(substr: String, str: Column, pos: Int): Column = withExpr { StringLocate(lit(substr).expr, str.expr, lit(pos).expr) } /** - * Left-pad the string column with + * Left-pad the string column with pad to a length of len. If the string column is longer + * than len, the return value is shortened to len characters. * * @group string_funcs * @since 1.5.0 */ - def lpad(str: Column, len: Int, pad: String): Column = { + def lpad(str: Column, len: Int, pad: String): Column = withExpr { StringLPad(str.expr, lit(len).expr, lit(pad).expr) } @@ -1837,15 +2331,25 @@ object functions { * @group string_funcs * @since 1.5.0 */ - def ltrim(e: Column): Column = StringTrimLeft(e.expr) + def ltrim(e: Column): Column = withExpr {StringTrimLeft(e.expr) } /** - * Extract a specific(idx) group identified by a java regex, from the specified string column. + * Trim the specified character string from left end for the specified string column. + * @group string_funcs + * @since 2.3.0 + */ + def ltrim(e: Column, trimString: String): Column = withExpr { + StringTrimLeft(e.expr, Literal(trimString)) + } + + /** + * Extract a specific group matched by a Java regex, from the specified string column. + * If the regex did not match, or the specified group did not match, an empty string is returned. * * @group string_funcs * @since 1.5.0 */ - def regexp_extract(e: Column, exp: String, groupIdx: Int): Column = { + def regexp_extract(e: Column, exp: String, groupIdx: Int): Column = withExpr { RegExpExtract(e.expr, lit(exp).expr, lit(groupIdx).expr) } @@ -1855,10 +2359,20 @@ object functions { * @group string_funcs * @since 1.5.0 */ - def regexp_replace(e: Column, pattern: String, replacement: String): Column = { + def regexp_replace(e: Column, pattern: String, replacement: String): Column = withExpr { RegExpReplace(e.expr, lit(pattern).expr, lit(replacement).expr) } + /** + * Replace all substrings of the specified string value that match regexp with rep. + * + * @group string_funcs + * @since 2.1.0 + */ + def regexp_replace(e: Column, pattern: Column, replacement: Column): Column = withExpr { + RegExpReplace(e.expr, pattern.expr, replacement.expr) + } + /** * Decodes a BASE64 encoded string column and returns it as a binary column. * This is the reverse of base64. @@ -1866,15 +2380,16 @@ object functions { * @group string_funcs * @since 1.5.0 */ - def unbase64(e: Column): Column = UnBase64(e.expr) + def unbase64(e: Column): Column = withExpr { UnBase64(e.expr) } /** - * Right-padded with pad to a length of len. + * Right-pad the string column with pad to a length of len. If the string column is longer + * than len, the return value is shortened to len characters. * * @group string_funcs * @since 1.5.0 */ - def rpad(str: Column, len: Int, pad: String): Column = { + def rpad(str: Column, len: Int, pad: String): Column = withExpr { StringRPad(str.expr, lit(len).expr, lit(pad).expr) } @@ -1884,7 +2399,7 @@ object functions { * @group string_funcs * @since 1.5.0 */ - def repeat(str: Column, n: Int): Column = { + def repeat(str: Column, n: Int): Column = withExpr { StringRepeat(str.expr, lit(n).expr) } @@ -1894,9 +2409,7 @@ object functions { * @group string_funcs * @since 1.5.0 */ - def reverse(str: Column): Column = { - StringReverse(str.expr) - } + def reverse(str: Column): Column = withExpr { StringReverse(str.expr) } /** * Trim the spaces from right end for the specified string value. @@ -1904,24 +2417,34 @@ object functions { * @group string_funcs * @since 1.5.0 */ - def rtrim(e: Column): Column = StringTrimRight(e.expr) + def rtrim(e: Column): Column = withExpr { StringTrimRight(e.expr) } /** - * * Return the soundex code for the specified expression. + * Trim the specified character string from right end for the specified string column. + * @group string_funcs + * @since 2.3.0 + */ + def rtrim(e: Column, trimString: String): Column = withExpr { + StringTrimRight(e.expr, Literal(trimString)) + } + + /** + * Returns the soundex code for the specified expression. * * @group string_funcs * @since 1.5.0 */ - def soundex(e: Column): Column = SoundEx(e.expr) + def soundex(e: Column): Column = withExpr { SoundEx(e.expr) } /** * Splits str around pattern (pattern is a regular expression). - * NOTE: pattern is a string represent the regular expression. + * + * @note Pattern is a string representation of the regular expression. * * @group string_funcs * @since 1.5.0 */ - def split(str: Column, pattern: String): Column = { + def split(str: Column, pattern: String): Column = withExpr { StringSplit(str.expr, lit(pattern).expr) } @@ -1930,11 +2453,14 @@ object functions { * returns the slice of byte array that starts at `pos` in byte and is of length `len` * when str is Binary type * + * @note The position is not zero based, but 1 based index. + * * @group string_funcs * @since 1.5.0 */ - def substring(str: Column, pos: Int, len: Int): Column = + def substring(str: Column, pos: Int, len: Int): Column = withExpr { Substring(str.expr, lit(pos).expr, lit(len).expr) + } /** * Returns the substring from string str before count occurrences of the delimiter delim. @@ -1944,20 +2470,22 @@ object functions { * * @group string_funcs */ - def substring_index(str: Column, delim: String, count: Int): Column = + def substring_index(str: Column, delim: String, count: Int): Column = withExpr { SubstringIndex(str.expr, lit(delim).expr, lit(count).expr) + } /** * Translate any character in the src by a character in replaceString. - * The characters in replaceString is corresponding to the characters in matchingString. - * The translate will happen when any character in the string matching with the character - * in the matchingString. + * The characters in replaceString correspond to the characters in matchingString. + * The translate will happen when any character in the string matches the character + * in the `matchingString`. * * @group string_funcs * @since 1.5.0 */ - def translate(src: Column, matchingString: String, replaceString: String): Column = + def translate(src: Column, matchingString: String, replaceString: String): Column = withExpr { StringTranslate(src.expr, lit(matchingString).expr, lit(replaceString).expr) + } /** * Trim the spaces from both ends for the specified string column. @@ -1965,7 +2493,16 @@ object functions { * @group string_funcs * @since 1.5.0 */ - def trim(e: Column): Column = StringTrim(e.expr) + def trim(e: Column): Column = withExpr { StringTrim(e.expr) } + + /** + * Trim the specified character from both ends for the specified string column. + * @group string_funcs + * @since 2.3.0 + */ + def trim(e: Column, trimString: String): Column = withExpr { + StringTrim(e.expr, Literal(trimString)) + } /** * Converts a string column to upper case. @@ -1973,7 +2510,7 @@ object functions { * @group string_funcs * @since 1.3.0 */ - def upper(e: Column): Column = Upper(e.expr) + def upper(e: Column): Column = withExpr { Upper(e.expr) } ////////////////////////////////////////////////////////////////////////////////////////////// // DateTime functions @@ -1985,8 +2522,9 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - def add_months(startDate: Column, numMonths: Int): Column = + def add_months(startDate: Column, numMonths: Int): Column = withExpr { AddMonths(startDate.expr, Literal(numMonths)) + } /** * Returns the current date as a date column. @@ -1994,7 +2532,7 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - def current_date(): Column = CurrentDate() + def current_date(): Column = withExpr { CurrentDate() } /** * Returns the current timestamp as a timestamp column. @@ -2002,86 +2540,87 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - def current_timestamp(): Column = CurrentTimestamp() + def current_timestamp(): Column = withExpr { CurrentTimestamp() } /** * Converts a date/timestamp/string to a value of string in the format specified by the date * format given by the second argument. * - * A pattern could be for instance `dd.MM.yyyy` and could return a string like '18.03.1993'. All - * pattern letters of [[java.text.SimpleDateFormat]] can be used. + * A pattern `dd.MM.yyyy` would return a string like `18.03.1993`. + * All pattern letters of `java.text.SimpleDateFormat` can be used. * - * NOTE: Use when ever possible specialized functions like [[year]]. These benefit from a + * @note Use specialized functions like [[year]] whenever possible as they benefit from a * specialized implementation. * * @group datetime_funcs * @since 1.5.0 */ - def date_format(dateExpr: Column, format: String): Column = + def date_format(dateExpr: Column, format: String): Column = withExpr { DateFormatClass(dateExpr.expr, Literal(format)) + } /** * Returns the date that is `days` days after `start` * @group datetime_funcs * @since 1.5.0 */ - def date_add(start: Column, days: Int): Column = DateAdd(start.expr, Literal(days)) + def date_add(start: Column, days: Int): Column = withExpr { DateAdd(start.expr, Literal(days)) } /** * Returns the date that is `days` days before `start` * @group datetime_funcs * @since 1.5.0 */ - def date_sub(start: Column, days: Int): Column = DateSub(start.expr, Literal(days)) + def date_sub(start: Column, days: Int): Column = withExpr { DateSub(start.expr, Literal(days)) } /** * Returns the number of days from `start` to `end`. * @group datetime_funcs * @since 1.5.0 */ - def datediff(end: Column, start: Column): Column = DateDiff(end.expr, start.expr) + def datediff(end: Column, start: Column): Column = withExpr { DateDiff(end.expr, start.expr) } /** * Extracts the year as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ - def year(e: Column): Column = Year(e.expr) + def year(e: Column): Column = withExpr { Year(e.expr) } /** * Extracts the quarter as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ - def quarter(e: Column): Column = Quarter(e.expr) + def quarter(e: Column): Column = withExpr { Quarter(e.expr) } /** * Extracts the month as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ - def month(e: Column): Column = Month(e.expr) + def month(e: Column): Column = withExpr { Month(e.expr) } /** * Extracts the day of the month as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ - def dayofmonth(e: Column): Column = DayOfMonth(e.expr) + def dayofmonth(e: Column): Column = withExpr { DayOfMonth(e.expr) } /** * Extracts the day of the year as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ - def dayofyear(e: Column): Column = DayOfYear(e.expr) + def dayofyear(e: Column): Column = withExpr { DayOfYear(e.expr) } /** * Extracts the hours as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ - def hour(e: Column): Column = Hour(e.expr) + def hour(e: Column): Column = withExpr { Hour(e.expr) } /** * Given a date column, returns the last day of the month which the given date belongs to. @@ -2091,21 +2630,23 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - def last_day(e: Column): Column = LastDay(e.expr) + def last_day(e: Column): Column = withExpr { LastDay(e.expr) } /** * Extracts the minutes as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ - def minute(e: Column): Column = Minute(e.expr) + def minute(e: Column): Column = withExpr { Minute(e.expr) } - /* + /** * Returns number of months between dates `date1` and `date2`. * @group datetime_funcs * @since 1.5.0 */ - def months_between(date1: Column, date2: Column): Column = MonthsBetween(date1.expr, date2.expr) + def months_between(date1: Column, date2: Column): Column = withExpr { + MonthsBetween(date1.expr, date2.expr) + } /** * Given a date column, returns the first date which is later than the value of the date column @@ -2120,21 +2661,23 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - def next_day(date: Column, dayOfWeek: String): Column = NextDay(date.expr, lit(dayOfWeek).expr) + def next_day(date: Column, dayOfWeek: String): Column = withExpr { + NextDay(date.expr, lit(dayOfWeek).expr) + } /** * Extracts the seconds as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ - def second(e: Column): Column = Second(e.expr) + def second(e: Column): Column = withExpr { Second(e.expr) } /** * Extracts the week number as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ - def weekofyear(e: Column): Column = WeekOfYear(e.expr) + def weekofyear(e: Column): Column = withExpr { WeekOfYear(e.expr) } /** * Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) to a string @@ -2143,7 +2686,9 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - def from_unixtime(ut: Column): Column = FromUnixTime(ut.expr, Literal("yyyy-MM-dd HH:mm:ss")) + def from_unixtime(ut: Column): Column = withExpr { + FromUnixTime(ut.expr, Literal("yyyy-MM-dd HH:mm:ss")) + } /** * Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) to a string @@ -2152,39 +2697,85 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - def from_unixtime(ut: Column, f: String): Column = FromUnixTime(ut.expr, Literal(f)) + def from_unixtime(ut: Column, f: String): Column = withExpr { + FromUnixTime(ut.expr, Literal(f)) + } /** - * Gets current Unix timestamp in seconds. + * Returns the current Unix timestamp (in seconds). + * + * @note All calls of `unix_timestamp` within the same query return the same value + * (i.e. the current timestamp is calculated at the start of query evaluation). + * * @group datetime_funcs * @since 1.5.0 */ - def unix_timestamp(): Column = UnixTimestamp(CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")) + def unix_timestamp(): Column = withExpr { + UnixTimestamp(CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")) + } /** * Converts time string in format yyyy-MM-dd HH:mm:ss to Unix timestamp (in seconds), - * using the default timezone and the default locale, return null if fail. + * using the default timezone and the default locale. + * Returns `null` if fails. + * * @group datetime_funcs * @since 1.5.0 */ - def unix_timestamp(s: Column): Column = UnixTimestamp(s.expr, Literal("yyyy-MM-dd HH:mm:ss")) + def unix_timestamp(s: Column): Column = withExpr { + UnixTimestamp(s.expr, Literal("yyyy-MM-dd HH:mm:ss")) + } /** - * Convert time string with given pattern - * (see [http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html]) - * to Unix time stamp (in seconds), return null if fail. + * Converts time string with given pattern to Unix timestamp (in seconds). + * Returns `null` if fails. + * + * @see + * Customizing Formats * @group datetime_funcs * @since 1.5.0 */ - def unix_timestamp(s: Column, p: String): Column = UnixTimestamp(s.expr, Literal(p)) + def unix_timestamp(s: Column, p: String): Column = withExpr { UnixTimestamp(s.expr, Literal(p)) } + + /** + * Convert time string to a Unix timestamp (in seconds) by casting rules to `TimestampType`. + * @group datetime_funcs + * @since 2.2.0 + */ + def to_timestamp(s: Column): Column = withExpr { + new ParseToTimestamp(s.expr) + } + + /** + * Convert time string to a Unix timestamp (in seconds) with a specified format + * (see [http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html]) + * to Unix timestamp (in seconds), return null if fail. + * @group datetime_funcs + * @since 2.2.0 + */ + def to_timestamp(s: Column, fmt: String): Column = withExpr { + new ParseToTimestamp(s.expr, Literal(fmt)) + } /** - * Converts the column into DateType. + * Converts the column into `DateType` by casting rules to `DateType`. * * @group datetime_funcs * @since 1.5.0 */ - def to_date(e: Column): Column = ToDate(e.expr) + def to_date(e: Column): Column = withExpr { new ParseToDate(e.expr) } + + /** + * Converts the column into a `DateType` with a specified format + * (see [http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html]) + * return null if fail. + * + * @group datetime_funcs + * @since 2.2.0 + */ + def to_date(e: Column, fmt: String): Column = withExpr { + new ParseToDate(e.expr, Literal(fmt)) + } /** * Returns date truncated to the unit specified by the format. @@ -2195,34 +2786,182 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - def trunc(date: Column, format: String): Column = TruncDate(date.expr, Literal(format)) + def trunc(date: Column, format: String): Column = withExpr { + TruncDate(date.expr, Literal(format)) + } /** - * Assumes given timestamp is UTC and converts to given timezone. + * Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in UTC, and renders + * that time as a timestamp in the given time zone. For example, 'GMT+1' would yield + * '2017-07-14 03:40:00.0'. * @group datetime_funcs * @since 1.5.0 */ - def from_utc_timestamp(ts: Column, tz: String): Column = - FromUTCTimestamp(ts.expr, Literal(tz).expr) + def from_utc_timestamp(ts: Column, tz: String): Column = withExpr { + FromUTCTimestamp(ts.expr, Literal(tz)) + } /** - * Assumes given timestamp is in given timezone and converts to UTC. + * Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in the given time + * zone, and renders that time as a timestamp in UTC. For example, 'GMT+1' would yield + * '2017-07-14 01:40:00.0'. * @group datetime_funcs * @since 1.5.0 */ - def to_utc_timestamp(ts: Column, tz: String): Column = ToUTCTimestamp(ts.expr, Literal(tz).expr) + def to_utc_timestamp(ts: Column, tz: String): Column = withExpr { + ToUTCTimestamp(ts.expr, Literal(tz)) + } + + /** + * Bucketize rows into one or more time windows given a timestamp specifying column. Window + * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window + * [12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in + * the order of months are not supported. The following example takes the average stock price for + * a one minute window every 10 seconds starting 5 seconds after the hour: + * + * {{{ + * val df = ... // schema => timestamp: TimestampType, stockId: StringType, price: DoubleType + * df.groupBy(window($"time", "1 minute", "10 seconds", "5 seconds"), $"stockId") + * .agg(mean("price")) + * }}} + * + * The windows will look like: + * + * {{{ + * 09:00:05-09:01:05 + * 09:00:15-09:01:15 + * 09:00:25-09:01:25 ... + * }}} + * + * For a streaming query, you may use the function `current_timestamp` to generate windows on + * processing time. + * + * @param timeColumn The column or the expression to use as the timestamp for windowing by time. + * The time column must be of TimestampType. + * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, + * `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for + * valid duration identifiers. Note that the duration is a fixed length of + * time, and does not vary over time according to a calendar. For example, + * `1 day` always means 86,400,000 milliseconds, not a calendar day. + * @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`. + * A new window will be generated every `slideDuration`. Must be less than + * or equal to the `windowDuration`. Check + * `org.apache.spark.unsafe.types.CalendarInterval` for valid duration + * identifiers. This duration is likewise absolute, and does not vary + * according to a calendar. + * @param startTime The offset with respect to 1970-01-01 00:00:00 UTC with which to start + * window intervals. For example, in order to have hourly tumbling windows that + * start 15 minutes past the hour, e.g. 12:15-13:15, 13:15-14:15... provide + * `startTime` as `15 minutes`. + * + * @group datetime_funcs + * @since 2.0.0 + */ + def window( + timeColumn: Column, + windowDuration: String, + slideDuration: String, + startTime: String): Column = { + withExpr { + TimeWindow(timeColumn.expr, windowDuration, slideDuration, startTime) + }.as("window") + } + + + /** + * Bucketize rows into one or more time windows given a timestamp specifying column. Window + * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window + * [12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in + * the order of months are not supported. The windows start beginning at 1970-01-01 00:00:00 UTC. + * The following example takes the average stock price for a one minute window every 10 seconds: + * + * {{{ + * val df = ... // schema => timestamp: TimestampType, stockId: StringType, price: DoubleType + * df.groupBy(window($"time", "1 minute", "10 seconds"), $"stockId") + * .agg(mean("price")) + * }}} + * + * The windows will look like: + * + * {{{ + * 09:00:00-09:01:00 + * 09:00:10-09:01:10 + * 09:00:20-09:01:20 ... + * }}} + * + * For a streaming query, you may use the function `current_timestamp` to generate windows on + * processing time. + * + * @param timeColumn The column or the expression to use as the timestamp for windowing by time. + * The time column must be of TimestampType. + * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, + * `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for + * valid duration identifiers. Note that the duration is a fixed length of + * time, and does not vary over time according to a calendar. For example, + * `1 day` always means 86,400,000 milliseconds, not a calendar day. + * @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`. + * A new window will be generated every `slideDuration`. Must be less than + * or equal to the `windowDuration`. Check + * `org.apache.spark.unsafe.types.CalendarInterval` for valid duration + * identifiers. This duration is likewise absolute, and does not vary + * according to a calendar. + * + * @group datetime_funcs + * @since 2.0.0 + */ + def window(timeColumn: Column, windowDuration: String, slideDuration: String): Column = { + window(timeColumn, windowDuration, slideDuration, "0 second") + } + + /** + * Generates tumbling time windows given a timestamp specifying column. Window + * starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window + * [12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in + * the order of months are not supported. The windows start beginning at 1970-01-01 00:00:00 UTC. + * The following example takes the average stock price for a one minute tumbling window: + * + * {{{ + * val df = ... // schema => timestamp: TimestampType, stockId: StringType, price: DoubleType + * df.groupBy(window($"time", "1 minute"), $"stockId") + * .agg(mean("price")) + * }}} + * + * The windows will look like: + * + * {{{ + * 09:00:00-09:01:00 + * 09:01:00-09:02:00 + * 09:02:00-09:03:00 ... + * }}} + * + * For a streaming query, you may use the function `current_timestamp` to generate windows on + * processing time. + * + * @param timeColumn The column or the expression to use as the timestamp for windowing by time. + * The time column must be of TimestampType. + * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, + * `1 second`. Check `org.apache.spark.unsafe.types.CalendarInterval` for + * valid duration identifiers. + * + * @group datetime_funcs + * @since 2.0.0 + */ + def window(timeColumn: Column, windowDuration: String): Column = { + window(timeColumn, windowDuration, windowDuration, "0 second") + } ////////////////////////////////////////////////////////////////////////////////////////////// // Collection functions ////////////////////////////////////////////////////////////////////////////////////////////// /** - * Returns true if the array contain the value + * Returns null if the array is null, true if the array contains `value`, and false otherwise. * @group collection_funcs * @since 1.5.0 */ - def array_contains(column: Column, value: Any): Column = + def array_contains(column: Column, value: Any): Column = withExpr { ArrayContains(column.expr, Literal(value)) + } /** * Creates a new row for each element in the given array or map column. @@ -2230,375 +2969,491 @@ object functions { * @group collection_funcs * @since 1.3.0 */ - def explode(e: Column): Column = Explode(e.expr) + def explode(e: Column): Column = withExpr { Explode(e.expr) } /** - * Returns length of array or map. + * Creates a new row for each element in the given array or map column. + * Unlike explode, if the array/map is null or empty then null is produced. * * @group collection_funcs - * @since 1.5.0 + * @since 2.2.0 */ - def size(e: Column): Column = Size(e.expr) + def explode_outer(e: Column): Column = withExpr { GeneratorOuter(Explode(e.expr)) } /** - * Sorts the input array for the given column in ascending order, - * according to the natural ordering of the array elements. + * Creates a new row for each element with position in the given array or map column. * * @group collection_funcs - * @since 1.5.0 + * @since 2.1.0 */ - def sort_array(e: Column): Column = sort_array(e, asc = true) + def posexplode(e: Column): Column = withExpr { PosExplode(e.expr) } /** - * Sorts the input array for the given column in ascending / descending order, - * according to the natural ordering of the array elements. + * Creates a new row for each element with position in the given array or map column. + * Unlike posexplode, if the array/map is null or empty then the row (null, null) is produced. * * @group collection_funcs - * @since 1.5.0 + * @since 2.2.0 */ - def sort_array(e: Column, asc: Boolean): Column = SortArray(e.expr, lit(asc).expr) - - ////////////////////////////////////////////////////////////////////////////////////////////// - ////////////////////////////////////////////////////////////////////////////////////////////// - - // scalastyle:off + def posexplode_outer(e: Column): Column = withExpr { GeneratorOuter(PosExplode(e.expr)) } - /* Use the following code to generate: - (0 to 10).map { x => - val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"}) - val typeTags = (1 to x).map(i => s"A$i: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _) - val inputTypes = (1 to x).foldRight("Nil")((i, s) => {s"ScalaReflection.schemaFor(typeTag[A$i]).dataType :: $s"}) - println(s""" - /** - * 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 - * @since 1.3.0 - */ - def udf[$typeTags](f: Function$x[$types]): UserDefinedFunction = { - val inputTypes = Try($inputTypes).getOrElse(Nil) - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, inputTypes) - }""") + /** + * Extracts json object from a json string based on json path specified, and returns json string + * of the extracted json object. It will return null if the input json string is invalid. + * + * @group collection_funcs + * @since 1.6.0 + */ + def get_json_object(e: Column, path: String): Column = withExpr { + GetJsonObject(e.expr, lit(path).expr) } - (0 to 10).map { x => - val args = (1 to x).map(i => s"arg$i: Column").mkString(", ") - val fTypes = Seq.fill(x + 1)("_").mkString(", ") - val argsInUDF = (1 to x).map(i => s"arg$i.expr").mkString(", ") - println(s""" - /** - * Call a Scala function of ${x} arguments as user-defined function (UDF). This requires - * you to specify the return data type. - * - * @group udf_funcs - * @since 1.3.0 - * @deprecated As of 1.5.0, since it's redundant with udf() - */ - @deprecated("Use udf", "1.5.0") - def callUDF(f: Function$x[$fTypes], returnType: DataType${if (args.length > 0) ", " + args else ""}): Column = { - ScalaUDF(f, returnType, Seq($argsInUDF)) - }""") - } - } - */ /** - * 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. + * Creates a new row for a json column according to the given field names. * - * @group udf_funcs - * @since 1.3.0 + * @group collection_funcs + * @since 1.6.0 */ - def udf[RT: TypeTag](f: Function0[RT]): UserDefinedFunction = { - val inputTypes = Try(Nil).getOrElse(Nil) - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, inputTypes) + @scala.annotation.varargs + def json_tuple(json: Column, fields: String*): Column = withExpr { + require(fields.nonEmpty, "at least 1 field name should be given.") + JsonTuple(json.expr +: fields.map(Literal.apply)) } /** - * 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. + * (Scala-specific) Parses a column containing a JSON string into a `StructType` with the + * specified schema. Returns `null`, in the case of an unparseable string. * - * @group udf_funcs - * @since 1.3.0 + * @param e a string column containing JSON data. + * @param schema the schema to use when parsing the json string + * @param options options to control how the json is parsed. Accepts the same options as the + * json data source. + * + * @group collection_funcs + * @since 2.1.0 */ - def udf[RT: TypeTag, A1: TypeTag](f: Function1[A1, RT]): UserDefinedFunction = { - val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: Nil).getOrElse(Nil) - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, inputTypes) - } + def from_json(e: Column, schema: StructType, options: Map[String, String]): Column = + from_json(e, schema.asInstanceOf[DataType], options) /** - * 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. + * (Scala-specific) Parses a column containing a JSON string into a `StructType` or `ArrayType` + * of `StructType`s with the specified schema. Returns `null`, in the case of an unparseable + * string. * - * @group udf_funcs - * @since 1.3.0 + * @param e a string column containing JSON data. + * @param schema the schema to use when parsing the json string + * @param options options to control how the json is parsed. accepts the same options and the + * json data source. + * + * @group collection_funcs + * @since 2.2.0 */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag](f: Function2[A1, A2, RT]): UserDefinedFunction = { - val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: Nil).getOrElse(Nil) - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, inputTypes) + def from_json(e: Column, schema: DataType, options: Map[String, String]): Column = withExpr { + JsonToStructs(schema, options, e.expr) } /** - * 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. + * (Java-specific) Parses a column containing a JSON string into a `StructType` with the + * specified schema. Returns `null`, in the case of an unparseable string. * - * @group udf_funcs - * @since 1.3.0 + * @param e a string column containing JSON data. + * @param schema the schema to use when parsing the json string + * @param options options to control how the json is parsed. accepts the same options and the + * json data source. + * + * @group collection_funcs + * @since 2.1.0 */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](f: Function3[A1, A2, A3, RT]): UserDefinedFunction = { - val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: Nil).getOrElse(Nil) - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, inputTypes) - } + def from_json(e: Column, schema: StructType, options: java.util.Map[String, String]): Column = + from_json(e, schema, options.asScala.toMap) /** - * 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. + * (Java-specific) Parses a column containing a JSON string into a `StructType` or `ArrayType` + * of `StructType`s with the specified schema. Returns `null`, in the case of an unparseable + * string. * - * @group udf_funcs - * @since 1.3.0 + * @param e a string column containing JSON data. + * @param schema the schema to use when parsing the json string + * @param options options to control how the json is parsed. accepts the same options and the + * json data source. + * + * @group collection_funcs + * @since 2.2.0 */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](f: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { - val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: Nil).getOrElse(Nil) - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, inputTypes) - } + def from_json(e: Column, schema: DataType, options: java.util.Map[String, String]): Column = + from_json(e, schema, options.asScala.toMap) /** - * 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. + * Parses a column containing a JSON string into a `StructType` with the specified schema. + * Returns `null`, in the case of an unparseable string. * - * @group udf_funcs - * @since 1.3.0 + * @param e a string column containing JSON data. + * @param schema the schema to use when parsing the json string + * + * @group collection_funcs + * @since 2.1.0 */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](f: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { - val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: Nil).getOrElse(Nil) - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, inputTypes) - } + def from_json(e: Column, schema: StructType): Column = + from_json(e, schema, Map.empty[String, String]) /** - * 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. + * Parses a column containing a JSON string into a `StructType` or `ArrayType` of `StructType`s + * with the specified schema. Returns `null`, in the case of an unparseable string. * - * @group udf_funcs - * @since 1.3.0 + * @param e a string column containing JSON data. + * @param schema the schema to use when parsing the json string + * + * @group collection_funcs + * @since 2.2.0 */ - 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 = { - val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: Nil).getOrElse(Nil) - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, inputTypes) - } + def from_json(e: Column, schema: DataType): Column = + from_json(e, schema, Map.empty[String, String]) /** - * 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. + * (Java-specific) Parses a column containing a JSON string into a `StructType` or `ArrayType` + * of `StructType`s with the specified schema. Returns `null`, in the case of an unparseable + * string. * - * @group udf_funcs - * @since 1.3.0 + * @param e a string column containing JSON data. + * @param schema the schema to use when parsing the json string as a json string. In Spark 2.1, + * the user-provided schema has to be in JSON format. Since Spark 2.2, the DDL + * format is also supported for the schema. + * + * @group collection_funcs + * @since 2.1.0 */ - 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 = { - val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: ScalaReflection.schemaFor(typeTag[A7]).dataType :: Nil).getOrElse(Nil) - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, inputTypes) + def from_json(e: Column, schema: String, options: java.util.Map[String, String]): Column = { + from_json(e, schema, options.asScala.toMap) } /** - * 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. + * (Scala-specific) Parses a column containing a JSON string into a `StructType` or `ArrayType` + * of `StructType`s with the specified schema. Returns `null`, in the case of an unparseable + * string. * - * @group udf_funcs - * @since 1.3.0 + * @param e a string column containing JSON data. + * @param schema the schema to use when parsing the json string as a json string, it could be a + * JSON format string or a DDL-formatted string. + * + * @group collection_funcs + * @since 2.3.0 */ - 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 = { - val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: ScalaReflection.schemaFor(typeTag[A7]).dataType :: ScalaReflection.schemaFor(typeTag[A8]).dataType :: Nil).getOrElse(Nil) - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, inputTypes) + def from_json(e: Column, schema: String, options: Map[String, String]): Column = { + val dataType = try { + DataType.fromJson(schema) + } catch { + case NonFatal(_) => StructType.fromDDL(schema) + } + from_json(e, dataType, options) } /** - * 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. + * (Scala-specific) Converts a column containing a `StructType`, `ArrayType` of `StructType`s, + * a `MapType` or `ArrayType` of `MapType`s into a JSON string with the specified schema. + * Throws an exception, in the case of an unsupported type. * - * @group udf_funcs - * @since 1.3.0 + * @param e a column containing a struct or array of the structs. + * @param options options to control how the struct column is converted into a json string. + * accepts the same options and the json data source. + * + * @group collection_funcs + * @since 2.1.0 */ - 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 = { - val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: ScalaReflection.schemaFor(typeTag[A7]).dataType :: ScalaReflection.schemaFor(typeTag[A8]).dataType :: ScalaReflection.schemaFor(typeTag[A9]).dataType :: Nil).getOrElse(Nil) - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, inputTypes) + def to_json(e: Column, options: Map[String, String]): Column = withExpr { + StructsToJson(options, e.expr) } /** - * 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. + * (Java-specific) Converts a column containing a `StructType`, `ArrayType` of `StructType`s, + * a `MapType` or `ArrayType` of `MapType`s into a JSON string with the specified schema. + * Throws an exception, in the case of an unsupported type. * - * @group udf_funcs - * @since 1.3.0 + * @param e a column containing a struct or array of the structs. + * @param options options to control how the struct column is converted into a json string. + * accepts the same options and the json data source. + * + * @group collection_funcs + * @since 2.1.0 */ - 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 = { - val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: ScalaReflection.schemaFor(typeTag[A7]).dataType :: ScalaReflection.schemaFor(typeTag[A8]).dataType :: ScalaReflection.schemaFor(typeTag[A9]).dataType :: ScalaReflection.schemaFor(typeTag[A10]).dataType :: Nil).getOrElse(Nil) - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, inputTypes) + def to_json(e: Column, options: java.util.Map[String, String]): Column = + to_json(e, options.asScala.toMap) + + /** + * Converts a column containing a `StructType`, `ArrayType` of `StructType`s, + * a `MapType` or `ArrayType` of `MapType`s into a JSON string with the specified schema. + * Throws an exception, in the case of an unsupported type. + * + * @param e a column containing a struct or array of the structs. + * + * @group collection_funcs + * @since 2.1.0 + */ + def to_json(e: Column): Column = + to_json(e, Map.empty[String, String]) + + /** + * Returns length of array or map. + * + * @group collection_funcs + * @since 1.5.0 + */ + def size(e: Column): Column = withExpr { Size(e.expr) } + + /** + * Sorts the input array for the given column in ascending order, + * according to the natural ordering of the array elements. + * + * @group collection_funcs + * @since 1.5.0 + */ + def sort_array(e: Column): Column = sort_array(e, asc = true) + + /** + * Sorts the input array for the given column in ascending or descending order, + * according to the natural ordering of the array elements. + * + * @group collection_funcs + * @since 1.5.0 + */ + def sort_array(e: Column, asc: Boolean): Column = withExpr { SortArray(e.expr, lit(asc).expr) } + + /** + * Returns an unordered array containing the keys of the map. + * @group collection_funcs + * @since 2.3.0 + */ + def map_keys(e: Column): Column = withExpr { MapKeys(e.expr) } + + /** + * Returns an unordered array containing the values of the map. + * @group collection_funcs + * @since 2.3.0 + */ + def map_values(e: Column): Column = withExpr { MapValues(e.expr) } + + ////////////////////////////////////////////////////////////////////////////////////////////// + ////////////////////////////////////////////////////////////////////////////////////////////// + + // scalastyle:off line.size.limit + // scalastyle:off parameter.number + + /* Use the following code to generate: + (0 to 10).map { x => + val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"}) + val typeTags = (1 to x).map(i => s"A$i: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _) + val inputTypes = (1 to x).foldRight("Nil")((i, s) => {s"ScalaReflection.schemaFor(typeTag[A$i]).dataType :: $s"}) + println(s""" + /** + * Defines a deterministic user-defined function of ${x} arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. + * + * @group udf_funcs + * @since 1.3.0 + */ + def udf[$typeTags](f: Function$x[$types]): UserDefinedFunction = { + val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val inputTypes = Try($inputTypes).toOption + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullable() + }""") } - ////////////////////////////////////////////////////////////////////////////////////////////////// + */ /** - * Call a Scala function of 0 arguments as user-defined function (UDF). This requires - * you to specify the return data type. + * Defines a deterministic user-defined function of 0 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 - * @deprecated As of 1.5.0, since it's redundant with udf() */ - @deprecated("Use udf", "1.5.0") - def callUDF(f: Function0[_], returnType: DataType): Column = { - ScalaUDF(f, returnType, Seq()) + def udf[RT: TypeTag](f: Function0[RT]): UserDefinedFunction = { + val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val inputTypes = Try(Nil).toOption + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullable() } /** - * Call a Scala function of 1 arguments as user-defined function (UDF). This requires - * you to specify the return data type. + * Defines a deterministic user-defined function of 1 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 - * @deprecated As of 1.5.0, since it's redundant with udf() */ - @deprecated("Use udf", "1.5.0") - def callUDF(f: Function1[_, _], returnType: DataType, arg1: Column): Column = { - ScalaUDF(f, returnType, Seq(arg1.expr)) + def udf[RT: TypeTag, A1: TypeTag](f: Function1[A1, RT]): UserDefinedFunction = { + val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: Nil).toOption + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullable() } /** - * Call a Scala function of 2 arguments as user-defined function (UDF). This requires - * you to specify the return data type. + * Defines a deterministic user-defined function of 2 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 - * @deprecated As of 1.5.0, since it's redundant with udf() */ - @deprecated("Use udf", "1.5.0") - def callUDF(f: Function2[_, _, _], returnType: DataType, arg1: Column, arg2: Column): Column = { - ScalaUDF(f, returnType, Seq(arg1.expr, arg2.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag](f: Function2[A1, A2, RT]): UserDefinedFunction = { + val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: Nil).toOption + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullable() } /** - * Call a Scala function of 3 arguments as user-defined function (UDF). This requires - * you to specify the return data type. + * Defines a deterministic user-defined function of 3 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 - * @deprecated As of 1.5.0, since it's redundant with udf() */ - @deprecated("Use udf", "1.5.0") - def callUDF(f: Function3[_, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column): Column = { - ScalaUDF(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](f: Function3[A1, A2, A3, RT]): UserDefinedFunction = { + val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: Nil).toOption + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullable() } /** - * Call a Scala function of 4 arguments as user-defined function (UDF). This requires - * you to specify the return data type. + * Defines a deterministic user-defined function of 4 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 - * @deprecated As of 1.5.0, since it's redundant with udf() */ - @deprecated("Use udf", "1.5.0") - 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)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](f: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { + val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: Nil).toOption + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullable() } /** - * Call a Scala function of 5 arguments as user-defined function (UDF). This requires - * you to specify the return data type. + * Defines a deterministic user-defined function of 5 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 - * @deprecated As of 1.5.0, since it's redundant with udf() */ - @deprecated("Use udf", "1.5.0") - 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)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](f: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { + val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: Nil).toOption + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullable() } /** - * Call a Scala function of 6 arguments as user-defined function (UDF). This requires - * you to specify the return data type. + * Defines a deterministic user-defined function of 6 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 - * @deprecated As of 1.5.0, since it's redundant with udf() */ - @deprecated("Use udf", "1.5.0") - 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)) + 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 = { + val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: Nil).toOption + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullable() } /** - * Call a Scala function of 7 arguments as user-defined function (UDF). This requires - * you to specify the return data type. + * Defines a deterministic user-defined function of 7 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 - * @deprecated As of 1.5.0, since it's redundant with udf() */ - @deprecated("Use udf", "1.5.0") - 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)) + 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 = { + val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: ScalaReflection.schemaFor(typeTag[A7]).dataType :: Nil).toOption + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullable() } /** - * Call a Scala function of 8 arguments as user-defined function (UDF). This requires - * you to specify the return data type. + * Defines a deterministic user-defined function of 8 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 - * @deprecated As of 1.5.0, since it's redundant with udf() */ - @deprecated("Use udf", "1.5.0") - 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)) + 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 = { + val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: ScalaReflection.schemaFor(typeTag[A7]).dataType :: ScalaReflection.schemaFor(typeTag[A8]).dataType :: Nil).toOption + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullable() } /** - * Call a Scala function of 9 arguments as user-defined function (UDF). This requires - * you to specify the return data type. + * Defines a deterministic user-defined function of 9 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 - * @deprecated As of 1.5.0, since it's redundant with udf() */ - @deprecated("Use udf", "1.5.0") - 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)) + 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 = { + val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: ScalaReflection.schemaFor(typeTag[A7]).dataType :: ScalaReflection.schemaFor(typeTag[A8]).dataType :: ScalaReflection.schemaFor(typeTag[A9]).dataType :: Nil).toOption + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullable() } /** - * Call a Scala function of 10 arguments as user-defined function (UDF). This requires - * you to specify the return data type. + * Defines a deterministic user-defined function of 10 arguments as user-defined + * function (UDF). The data types are automatically inferred based on the function's + * signature. To change a UDF to nondeterministic, call the API + * `UserDefinedFunction.asNondeterministic()`. * * @group udf_funcs * @since 1.3.0 - * @deprecated As of 1.5.0, since it's redundant with udf() */ - @deprecated("Use udf", "1.5.0") - 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)) + 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 = { + val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] + val inputTypes = Try(ScalaReflection.schemaFor(typeTag[A1]).dataType :: ScalaReflection.schemaFor(typeTag[A2]).dataType :: ScalaReflection.schemaFor(typeTag[A3]).dataType :: ScalaReflection.schemaFor(typeTag[A4]).dataType :: ScalaReflection.schemaFor(typeTag[A5]).dataType :: ScalaReflection.schemaFor(typeTag[A6]).dataType :: ScalaReflection.schemaFor(typeTag[A7]).dataType :: ScalaReflection.schemaFor(typeTag[A8]).dataType :: ScalaReflection.schemaFor(typeTag[A9]).dataType :: ScalaReflection.schemaFor(typeTag[A10]).dataType :: Nil).toOption + val udf = UserDefinedFunction(f, dataType, inputTypes) + if (nullable) udf else udf.asNonNullable() } - // scalastyle:on + // scalastyle:on parameter.number + // scalastyle:on line.size.limit /** - * Call an user-defined function. - * Example: - * {{{ - * import org.apache.spark.sql._ + * Defines a deterministic user-defined function (UDF) using a Scala closure. For this variant, + * the caller must specify the output data type, and there is no automatic input type coercion. + * To change a UDF to nondeterministic, call the API `UserDefinedFunction.asNondeterministic()`. * - * val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value") - * val sqlContext = df.sqlContext - * sqlContext.udf.register("simpleUDF", (v: Int) => v * v) - * df.select($"id", callUDF("simpleUDF", $"value")) - * }}} + * @param f A closure in Scala + * @param dataType The output data type of the UDF * * @group udf_funcs - * @since 1.5.0 + * @since 2.0.0 */ - @scala.annotation.varargs - def callUDF(udfName: String, cols: Column*): Column = { - UnresolvedFunction(udfName, cols.map(_.expr), isDistinct = false) + def udf(f: AnyRef, dataType: DataType): UserDefinedFunction = { + UserDefinedFunction(f, dataType, None) } /** @@ -2608,25 +3463,16 @@ object functions { * import org.apache.spark.sql._ * * val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value") - * val sqlContext = df.sqlContext - * sqlContext.udf.register("simpleUDF", (v: Int) => v * v) - * df.select($"id", callUdf("simpleUDF", $"value")) + * val spark = df.sparkSession + * spark.udf.register("simpleUDF", (v: Int) => v * v) + * df.select($"id", callUDF("simpleUDF", $"value")) * }}} * * @group udf_funcs - * @since 1.4.0 - * @deprecated As of 1.5.0, since it was not coherent to have two functions callUdf and callUDF - */ - @deprecated("Use callUDF", "1.5.0") - def callUdf(udfName: String, cols: Column*): Column = { - // Note: we avoid using closures here because on file systems that are case-insensitive, the - // compiled class file for the closure here will conflict with the one in callUDF (upper case). - val exprs = new Array[Expression](cols.size) - var i = 0 - while (i < cols.size) { - exprs(i) = cols(i).expr - i += 1 - } - UnresolvedFunction(udfName, exprs, isDistinct = false) + * @since 1.5.0 + */ + @scala.annotation.varargs + def callUDF(udfName: String, cols: Column*): Column = withExpr { + UnresolvedFunction(udfName, cols.map(_.expr), isDistinct = false) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala new file mode 100644 index 000000000000..4e756084bbdb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -0,0 +1,323 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.internal + +import org.apache.spark.SparkConf +import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.sql.{ExperimentalMethods, SparkSession, UDFRegistration, _} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} +import org.apache.spark.sql.catalyst.catalog.SessionCatalog +import org.apache.spark.sql.catalyst.optimizer.Optimizer +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{QueryExecution, SparkOptimizer, SparkPlanner, SparkSqlParser} +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.streaming.StreamingQueryManager +import org.apache.spark.sql.util.ExecutionListenerManager + +/** + * Builder class that coordinates construction of a new [[SessionState]]. + * + * The builder explicitly defines all components needed by the session state, and creates a session + * state when `build` is called. Components should only be initialized once. This is not a problem + * for most components as they are only used in the `build` function. However some components + * (`conf`, `catalog`, `functionRegistry`, `experimentalMethods` & `sqlParser`) are as dependencies + * for other components and are shared as a result. These components are defined as lazy vals to + * make sure the component is created only once. + * + * A developer can modify the builder by providing custom versions of components, or by using the + * hooks provided for the analyzer, optimizer & planner. There are some dependencies between the + * components (they are documented per dependency), a developer should respect these when making + * modifications in order to prevent initialization problems. + * + * A parent [[SessionState]] can be used to initialize the new [[SessionState]]. The new session + * state will clone the parent sessions state's `conf`, `functionRegistry`, `experimentalMethods` + * and `catalog` fields. Note that the state is cloned when `build` is called, and not before. + */ +@Experimental +@InterfaceStability.Unstable +abstract class BaseSessionStateBuilder( + val session: SparkSession, + val parentState: Option[SessionState] = None) { + type NewBuilder = (SparkSession, Option[SessionState]) => BaseSessionStateBuilder + + /** + * Function that produces a new instance of the `BaseSessionStateBuilder`. This is used by the + * [[SessionState]]'s clone functionality. Make sure to override this when implementing your own + * [[SessionStateBuilder]]. + */ + protected def newBuilder: NewBuilder + + /** + * Session extensions defined in the [[SparkSession]]. + */ + protected def extensions: SparkSessionExtensions = session.extensions + + /** + * Extract entries from `SparkConf` and put them in the `SQLConf` + */ + protected def mergeSparkConf(sqlConf: SQLConf, sparkConf: SparkConf): Unit = { + sparkConf.getAll.foreach { case (k, v) => + sqlConf.setConfString(k, v) + } + } + + /** + * SQL-specific key-value configurations. + * + * These either get cloned from a pre-existing instance or newly created. The conf is always + * merged with its [[SparkConf]]. + */ + protected lazy val conf: SQLConf = { + val conf = parentState.map(_.conf.clone()).getOrElse(new SQLConf) + mergeSparkConf(conf, session.sparkContext.conf) + conf + } + + /** + * Internal catalog managing functions registered by the user. + * + * This either gets cloned from a pre-existing version or cloned from the built-in registry. + */ + protected lazy val functionRegistry: FunctionRegistry = { + parentState.map(_.functionRegistry).getOrElse(FunctionRegistry.builtin).clone() + } + + /** + * Experimental methods that can be used to define custom optimization rules and custom planning + * strategies. + * + * This either gets cloned from a pre-existing version or newly created. + */ + protected lazy val experimentalMethods: ExperimentalMethods = { + parentState.map(_.experimentalMethods.clone()).getOrElse(new ExperimentalMethods) + } + + /** + * Parser that extracts expressions, plans, table identifiers etc. from SQL texts. + * + * Note: this depends on the `conf` field. + */ + protected lazy val sqlParser: ParserInterface = { + extensions.buildParser(session, new SparkSqlParser(conf)) + } + + /** + * ResourceLoader that is used to load function resources and jars. + */ + protected lazy val resourceLoader: SessionResourceLoader = new SessionResourceLoader(session) + + /** + * Catalog for managing table and database states. If there is a pre-existing catalog, the state + * of that catalog (temp tables & current database) will be copied into the new catalog. + * + * Note: this depends on the `conf`, `functionRegistry` and `sqlParser` fields. + */ + protected lazy val catalog: SessionCatalog = { + val catalog = new SessionCatalog( + session.sharedState.externalCatalog, + session.sharedState.globalTempViewManager, + functionRegistry, + conf, + SessionState.newHadoopConf(session.sparkContext.hadoopConfiguration, conf), + sqlParser, + resourceLoader) + parentState.foreach(_.catalog.copyStateTo(catalog)) + catalog + } + + /** + * Interface exposed to the user for registering user-defined functions. + * + * Note 1: The user-defined functions must be deterministic. + * Note 2: This depends on the `functionRegistry` field. + */ + protected def udfRegistration: UDFRegistration = new UDFRegistration(functionRegistry) + + /** + * Logical query plan analyzer for resolving unresolved attributes and relations. + * + * Note: this depends on the `conf` and `catalog` fields. + */ + protected def analyzer: Analyzer = new Analyzer(catalog, conf) { + override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = + new FindDataSourceTable(session) +: + new ResolveSQLOnFile(session) +: + customResolutionRules + + override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = + PreprocessTableCreation(session) +: + PreprocessTableInsertion(conf) +: + DataSourceAnalysis(conf) +: + customPostHocResolutionRules + + override val extendedCheckRules: Seq[LogicalPlan => Unit] = + PreWriteCheck +: + PreReadCheck +: + HiveOnlyCheck +: + customCheckRules + } + + /** + * Custom resolution rules to add to the Analyzer. Prefer overriding this instead of creating + * your own Analyzer. + * + * Note that this may NOT depend on the `analyzer` function. + */ + protected def customResolutionRules: Seq[Rule[LogicalPlan]] = { + extensions.buildResolutionRules(session) + } + + /** + * Custom post resolution rules to add to the Analyzer. Prefer overriding this instead of + * creating your own Analyzer. + * + * Note that this may NOT depend on the `analyzer` function. + */ + protected def customPostHocResolutionRules: Seq[Rule[LogicalPlan]] = { + extensions.buildPostHocResolutionRules(session) + } + + /** + * Custom check rules to add to the Analyzer. Prefer overriding this instead of creating + * your own Analyzer. + * + * Note that this may NOT depend on the `analyzer` function. + */ + protected def customCheckRules: Seq[LogicalPlan => Unit] = { + extensions.buildCheckRules(session) + } + + /** + * Logical query plan optimizer. + * + * Note: this depends on the `conf`, `catalog` and `experimentalMethods` fields. + */ + protected def optimizer: Optimizer = { + new SparkOptimizer(catalog, experimentalMethods) { + override def extendedOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = + super.extendedOperatorOptimizationRules ++ customOperatorOptimizationRules + } + } + + /** + * Custom operator optimization rules to add to the Optimizer. Prefer overriding this instead + * of creating your own Optimizer. + * + * Note that this may NOT depend on the `optimizer` function. + */ + protected def customOperatorOptimizationRules: Seq[Rule[LogicalPlan]] = { + extensions.buildOptimizerRules(session) + } + + /** + * Planner that converts optimized logical plans to physical plans. + * + * Note: this depends on the `conf` and `experimentalMethods` fields. + */ + protected def planner: SparkPlanner = { + new SparkPlanner(session.sparkContext, conf, experimentalMethods) { + override def extraPlanningStrategies: Seq[Strategy] = + super.extraPlanningStrategies ++ customPlanningStrategies + } + } + + /** + * Custom strategies to add to the planner. Prefer overriding this instead of creating + * your own Planner. + * + * Note that this may NOT depend on the `planner` function. + */ + protected def customPlanningStrategies: Seq[Strategy] = { + extensions.buildPlannerStrategies(session) + } + + /** + * Create a query execution object. + */ + protected def createQueryExecution: LogicalPlan => QueryExecution = { plan => + new QueryExecution(session, plan) + } + + /** + * Interface to start and stop streaming queries. + */ + protected def streamingQueryManager: StreamingQueryManager = new StreamingQueryManager(session) + + /** + * An interface to register custom [[org.apache.spark.sql.util.QueryExecutionListener]]s + * that listen for execution metrics. + * + * This gets cloned from parent if available, otherwise is a new instance is created. + */ + protected def listenerManager: ExecutionListenerManager = { + parentState.map(_.listenerManager.clone()).getOrElse(new ExecutionListenerManager) + } + + /** + * Function used to make clones of the session state. + */ + protected def createClone: (SparkSession, SessionState) => SessionState = { + val createBuilder = newBuilder + (session, state) => createBuilder(session, Option(state)).build() + } + + /** + * Build the [[SessionState]]. + */ + def build(): SessionState = { + new SessionState( + session.sharedState, + conf, + experimentalMethods, + functionRegistry, + udfRegistration, + () => catalog, + sqlParser, + () => analyzer, + () => optimizer, + planner, + streamingQueryManager, + listenerManager, + () => resourceLoader, + createQueryExecution, + createClone) + } +} + +/** + * Helper class for using SessionStateBuilders during tests. + */ +private[sql] trait WithTestConf { self: BaseSessionStateBuilder => + def overrideConfs: Map[String, String] + + override protected lazy val conf: SQLConf = { + val conf = parentState.map(_.conf.clone()).getOrElse { + new SQLConf { + clear() + override def clear(): Unit = { + super.clear() + // Make sure we start with the default test configs even after clear + overrideConfs.foreach { case (key, value) => setConfString(key, value) } + } + } + } + mergeSparkConf(conf, session.sparkContext.conf) + conf + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala new file mode 100644 index 000000000000..142b005850a4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -0,0 +1,518 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.internal + +import scala.reflect.runtime.universe.TypeTag +import scala.util.control.NonFatal + +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql._ +import org.apache.spark.sql.catalog.{Catalog, Column, Database, Function, Table} +import org.apache.spark.sql.catalyst.{DefinedByConstructorParams, FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.execution.command.AlterTableRecoverPartitionsCommand +import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource} +import org.apache.spark.sql.types.StructType +import org.apache.spark.storage.StorageLevel + + +/** + * Internal implementation of the user-facing `Catalog`. + */ +class CatalogImpl(sparkSession: SparkSession) extends Catalog { + + private def sessionCatalog: SessionCatalog = sparkSession.sessionState.catalog + + private def requireDatabaseExists(dbName: String): Unit = { + if (!sessionCatalog.databaseExists(dbName)) { + throw new AnalysisException(s"Database '$dbName' does not exist.") + } + } + + private def requireTableExists(dbName: String, tableName: String): Unit = { + if (!sessionCatalog.tableExists(TableIdentifier(tableName, Some(dbName)))) { + throw new AnalysisException(s"Table '$tableName' does not exist in database '$dbName'.") + } + } + + /** + * Returns the current default database in this session. + */ + override def currentDatabase: String = sessionCatalog.getCurrentDatabase + + /** + * Sets the current default database in this session. + */ + @throws[AnalysisException]("database does not exist") + override def setCurrentDatabase(dbName: String): Unit = { + requireDatabaseExists(dbName) + sessionCatalog.setCurrentDatabase(dbName) + } + + /** + * Returns a list of databases available across all sessions. + */ + override def listDatabases(): Dataset[Database] = { + val databases = sessionCatalog.listDatabases().map(makeDatabase) + CatalogImpl.makeDataset(databases, sparkSession) + } + + private def makeDatabase(dbName: String): Database = { + val metadata = sessionCatalog.getDatabaseMetadata(dbName) + new Database( + name = metadata.name, + description = metadata.description, + locationUri = CatalogUtils.URIToString(metadata.locationUri)) + } + + /** + * Returns a list of tables in the current database. + * This includes all temporary tables. + */ + override def listTables(): Dataset[Table] = { + listTables(currentDatabase) + } + + /** + * Returns a list of tables in the specified database. + * This includes all temporary tables. + */ + @throws[AnalysisException]("database does not exist") + override def listTables(dbName: String): Dataset[Table] = { + val tables = sessionCatalog.listTables(dbName).map(makeTable) + CatalogImpl.makeDataset(tables, sparkSession) + } + + /** + * Returns a Table for the given table/view or temporary view. + * + * Note that this function requires the table already exists in the Catalog. + * + * If the table metadata retrieval failed due to any reason (e.g., table serde class + * is not accessible or the table type is not accepted by Spark SQL), this function + * still returns the corresponding Table without the description and tableType) + */ + private def makeTable(tableIdent: TableIdentifier): Table = { + val metadata = try { + Some(sessionCatalog.getTempViewOrPermanentTableMetadata(tableIdent)) + } catch { + case NonFatal(_) => None + } + val isTemp = sessionCatalog.isTemporaryTable(tableIdent) + new Table( + name = tableIdent.table, + database = metadata.map(_.identifier.database).getOrElse(tableIdent.database).orNull, + description = metadata.map(_.comment.orNull).orNull, + tableType = if (isTemp) "TEMPORARY" else metadata.map(_.tableType.name).orNull, + isTemporary = isTemp) + } + + /** + * Returns a list of functions registered in the current database. + * This includes all temporary functions + */ + override def listFunctions(): Dataset[Function] = { + listFunctions(currentDatabase) + } + + /** + * Returns a list of functions registered in the specified database. + * This includes all temporary functions + */ + @throws[AnalysisException]("database does not exist") + override def listFunctions(dbName: String): Dataset[Function] = { + requireDatabaseExists(dbName) + val functions = sessionCatalog.listFunctions(dbName).map { case (functIdent, _) => + makeFunction(functIdent) + } + CatalogImpl.makeDataset(functions, sparkSession) + } + + private def makeFunction(funcIdent: FunctionIdentifier): Function = { + val metadata = sessionCatalog.lookupFunctionInfo(funcIdent) + new Function( + name = metadata.getName, + database = metadata.getDb, + description = null, // for now, this is always undefined + className = metadata.getClassName, + isTemporary = metadata.getDb == null) + } + + /** + * Returns a list of columns for the given table/view or temporary view. + */ + @throws[AnalysisException]("table does not exist") + override def listColumns(tableName: String): Dataset[Column] = { + val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName) + listColumns(tableIdent) + } + + /** + * Returns a list of columns for the given table/view or temporary view in the specified database. + */ + @throws[AnalysisException]("database or table does not exist") + override def listColumns(dbName: String, tableName: String): Dataset[Column] = { + requireTableExists(dbName, tableName) + listColumns(TableIdentifier(tableName, Some(dbName))) + } + + private def listColumns(tableIdentifier: TableIdentifier): Dataset[Column] = { + val tableMetadata = sessionCatalog.getTempViewOrPermanentTableMetadata(tableIdentifier) + + val partitionColumnNames = tableMetadata.partitionColumnNames.toSet + val bucketColumnNames = tableMetadata.bucketSpec.map(_.bucketColumnNames).getOrElse(Nil).toSet + val columns = tableMetadata.schema.map { c => + new Column( + name = c.name, + description = c.getComment().orNull, + dataType = c.dataType.catalogString, + nullable = c.nullable, + isPartition = partitionColumnNames.contains(c.name), + isBucket = bucketColumnNames.contains(c.name)) + } + CatalogImpl.makeDataset(columns, sparkSession) + } + + /** + * Gets the database with the specified name. This throws an `AnalysisException` when no + * `Database` can be found. + */ + override def getDatabase(dbName: String): Database = { + makeDatabase(dbName) + } + + /** + * Gets the table or view with the specified name. This table can be a temporary view or a + * table/view. This throws an `AnalysisException` when no `Table` can be found. + */ + override def getTable(tableName: String): Table = { + val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName) + getTable(tableIdent.database.orNull, tableIdent.table) + } + + /** + * Gets the table or view with the specified name in the specified database. This throws an + * `AnalysisException` when no `Table` can be found. + */ + override def getTable(dbName: String, tableName: String): Table = { + if (tableExists(dbName, tableName)) { + makeTable(TableIdentifier(tableName, Option(dbName))) + } else { + throw new AnalysisException(s"Table or view '$tableName' not found in database '$dbName'") + } + } + + /** + * Gets the function with the specified name. This function can be a temporary function or a + * function. This throws an `AnalysisException` when no `Function` can be found. + */ + override def getFunction(functionName: String): Function = { + val functionIdent = sparkSession.sessionState.sqlParser.parseFunctionIdentifier(functionName) + getFunction(functionIdent.database.orNull, functionIdent.funcName) + } + + /** + * Gets the function with the specified name. This returns `None` when no `Function` can be + * found. + */ + override def getFunction(dbName: String, functionName: String): Function = { + makeFunction(FunctionIdentifier(functionName, Option(dbName))) + } + + /** + * Checks if the database with the specified name exists. + */ + override def databaseExists(dbName: String): Boolean = { + sessionCatalog.databaseExists(dbName) + } + + /** + * Checks if the table or view with the specified name exists. This can either be a temporary + * view or a table/view. + */ + override def tableExists(tableName: String): Boolean = { + val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName) + tableExists(tableIdent.database.orNull, tableIdent.table) + } + + /** + * Checks if the table or view with the specified name exists in the specified database. + */ + override def tableExists(dbName: String, tableName: String): Boolean = { + val tableIdent = TableIdentifier(tableName, Option(dbName)) + sessionCatalog.isTemporaryTable(tableIdent) || sessionCatalog.tableExists(tableIdent) + } + + /** + * Checks if the function with the specified name exists. This can either be a temporary function + * or a function. + */ + override def functionExists(functionName: String): Boolean = { + val functionIdent = sparkSession.sessionState.sqlParser.parseFunctionIdentifier(functionName) + functionExists(functionIdent.database.orNull, functionIdent.funcName) + } + + /** + * Checks if the function with the specified name exists in the specified database. + */ + override def functionExists(dbName: String, functionName: String): Boolean = { + sessionCatalog.functionExists(FunctionIdentifier(functionName, Option(dbName))) + } + + /** + * :: Experimental :: + * Creates a 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 + * @since 2.2.0 + */ + @Experimental + override def createTable(tableName: String, path: String): DataFrame = { + val dataSourceName = sparkSession.sessionState.conf.defaultDataSourceName + createTable(tableName, path, dataSourceName) + } + + /** + * :: Experimental :: + * Creates a table from the given path and returns the corresponding + * DataFrame. + * + * @group ddl_ops + * @since 2.2.0 + */ + @Experimental + override def createTable(tableName: String, path: String, source: String): DataFrame = { + createTable(tableName, source, Map("path" -> path)) + } + + /** + * :: Experimental :: + * (Scala-specific) + * Creates a table based on the dataset in a data source and a set of options. + * Then, returns the corresponding DataFrame. + * + * @group ddl_ops + * @since 2.2.0 + */ + @Experimental + override def createTable( + tableName: String, + source: String, + options: Map[String, String]): DataFrame = { + createTable(tableName, source, new StructType, options) + } + + /** + * :: Experimental :: + * (Scala-specific) + * Creates a table based on the dataset in a data source, a schema and a set of options. + * Then, returns the corresponding DataFrame. + * + * @group ddl_ops + * @since 2.2.0 + */ + @Experimental + override def createTable( + tableName: String, + source: String, + schema: StructType, + options: Map[String, String]): DataFrame = { + val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName) + val storage = DataSource.buildStorageFormatFromOptions(options) + val tableType = if (storage.locationUri.isDefined) { + CatalogTableType.EXTERNAL + } else { + CatalogTableType.MANAGED + } + val tableDesc = CatalogTable( + identifier = tableIdent, + tableType = tableType, + storage = storage, + schema = schema, + provider = Some(source) + ) + val plan = CreateTable(tableDesc, SaveMode.ErrorIfExists, None) + sparkSession.sessionState.executePlan(plan).toRdd + sparkSession.table(tableIdent) + } + + /** + * Drops the local temporary view with the given view name in the catalog. + * If the view has been cached/persisted before, it's also unpersisted. + * + * @param viewName the identifier of the temporary view to be dropped. + * @group ddl_ops + * @since 2.0.0 + */ + override def dropTempView(viewName: String): Boolean = { + sparkSession.sessionState.catalog.getTempView(viewName).exists { viewDef => + sparkSession.sharedState.cacheManager.uncacheQuery(sparkSession, viewDef, blocking = true) + sessionCatalog.dropTempView(viewName) + } + } + + /** + * Drops the global temporary view with the given view name in the catalog. + * If the view has been cached/persisted before, it's also unpersisted. + * + * @param viewName the identifier of the global temporary view to be dropped. + * @group ddl_ops + * @since 2.1.0 + */ + override def dropGlobalTempView(viewName: String): Boolean = { + sparkSession.sessionState.catalog.getGlobalTempView(viewName).exists { viewDef => + sparkSession.sharedState.cacheManager.uncacheQuery(sparkSession, viewDef, blocking = true) + sessionCatalog.dropGlobalTempView(viewName) + } + } + + /** + * Recovers all the partitions in the directory of a table and update the catalog. + * Only works with a partitioned table, and not a temporary view. + * + * @param tableName is either a qualified or unqualified name that designates a table. + * If no database identifier is provided, it refers to a table in the + * current database. + * @group ddl_ops + * @since 2.1.1 + */ + override def recoverPartitions(tableName: String): Unit = { + val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName) + sparkSession.sessionState.executePlan( + AlterTableRecoverPartitionsCommand(tableIdent)).toRdd + } + + /** + * Returns true if the table or view is currently cached in-memory. + * + * @group cachemgmt + * @since 2.0.0 + */ + override def isCached(tableName: String): Boolean = { + sparkSession.sharedState.cacheManager.lookupCachedData(sparkSession.table(tableName)).nonEmpty + } + + /** + * Caches the specified table or view in-memory. + * + * @group cachemgmt + * @since 2.0.0 + */ + override def cacheTable(tableName: String): Unit = { + sparkSession.sharedState.cacheManager.cacheQuery(sparkSession.table(tableName), Some(tableName)) + } + + /** + * Caches the specified table or view with the given storage level. + * + * @group cachemgmt + * @since 2.3.0 + */ + override def cacheTable(tableName: String, storageLevel: StorageLevel): Unit = { + sparkSession.sharedState.cacheManager.cacheQuery( + sparkSession.table(tableName), Some(tableName), storageLevel) + } + + /** + * Removes the specified table or view from the in-memory cache. + * + * @group cachemgmt + * @since 2.0.0 + */ + override def uncacheTable(tableName: String): Unit = { + sparkSession.sharedState.cacheManager.uncacheQuery(sparkSession.table(tableName)) + } + + /** + * Removes all cached tables or views from the in-memory cache. + * + * @group cachemgmt + * @since 2.0.0 + */ + override def clearCache(): Unit = { + sparkSession.sharedState.cacheManager.clearCache() + } + + /** + * Returns true if the [[Dataset]] is currently cached in-memory. + * + * @group cachemgmt + * @since 2.0.0 + */ + protected[sql] def isCached(qName: Dataset[_]): Boolean = { + sparkSession.sharedState.cacheManager.lookupCachedData(qName).nonEmpty + } + + /** + * Invalidates and refreshes all the cached data and metadata of the given table or view. + * For Hive metastore table, the metadata is refreshed. For data source tables, the schema will + * not be inferred and refreshed. + * + * If this table is cached as an InMemoryRelation, drop the original cached version and make the + * new version cached lazily. + * + * @group cachemgmt + * @since 2.0.0 + */ + override def refreshTable(tableName: String): Unit = { + val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName) + // Temp tables: refresh (or invalidate) any metadata/data cached in the plan recursively. + // Non-temp tables: refresh the metadata cache. + sessionCatalog.refreshTable(tableIdent) + + // If this table is cached as an InMemoryRelation, drop the original + // cached version and make the new version cached lazily. + val table = sparkSession.table(tableIdent) + if (isCached(table)) { + // Uncache the logicalPlan. + sparkSession.sharedState.cacheManager.uncacheQuery(table, blocking = true) + // Cache it again. + sparkSession.sharedState.cacheManager.cacheQuery(table, Some(tableIdent.table)) + } + } + + /** + * Refreshes the cache entry and the associated metadata for all Dataset (if any), that contain + * the given data source path. Path matching is by prefix, i.e. "/" would invalidate + * everything that is cached. + * + * @group cachemgmt + * @since 2.0.0 + */ + override def refreshByPath(resourcePath: String): Unit = { + sparkSession.sharedState.cacheManager.recacheByPath(sparkSession, resourcePath) + } +} + + +private[sql] object CatalogImpl { + + def makeDataset[T <: DefinedByConstructorParams: TypeTag]( + data: Seq[T], + sparkSession: SparkSession): Dataset[T] = { + val enc = ExpressionEncoder[T]() + val encoded = data.map(d => enc.toRow(d).copy()) + val plan = new LocalRelation(enc.schema.toAttributes, encoded) + val queryExecution = sparkSession.sessionState.executePlan(plan) + new Dataset[T](sparkSession, queryExecution, enc) + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala new file mode 100644 index 000000000000..b9515ec7bca2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.internal + +import java.util.Locale + +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat + +case class HiveSerDe( + inputFormat: Option[String] = None, + outputFormat: Option[String] = None, + serde: Option[String] = None) + +object HiveSerDe { + val serdeMap = Map( + "sequencefile" -> + HiveSerDe( + inputFormat = Option("org.apache.hadoop.mapred.SequenceFileInputFormat"), + outputFormat = Option("org.apache.hadoop.mapred.SequenceFileOutputFormat")), + + "rcfile" -> + HiveSerDe( + inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat"), + serde = Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")), + + "orc" -> + HiveSerDe( + inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"), + serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde")), + + "parquet" -> + HiveSerDe( + inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"), + serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")), + + "textfile" -> + HiveSerDe( + inputFormat = Option("org.apache.hadoop.mapred.TextInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat")), + + "avro" -> + HiveSerDe( + inputFormat = Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"), + outputFormat = Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat"), + serde = Option("org.apache.hadoop.hive.serde2.avro.AvroSerDe"))) + + /** + * Get the Hive SerDe information from the data source abbreviation string or classname. + * + * @param source Currently the source abbreviation can be one of the following: + * SequenceFile, RCFile, ORC, PARQUET, and case insensitive. + * @return HiveSerDe associated with the specified source + */ + def sourceToSerDe(source: String): Option[HiveSerDe] = { + val key = source.toLowerCase(Locale.ROOT) match { + case s if s.startsWith("org.apache.spark.sql.parquet") => "parquet" + case s if s.startsWith("org.apache.spark.sql.orc") => "orc" + case s if s.equals("orcfile") => "orc" + case s if s.equals("parquetfile") => "parquet" + case s if s.equals("avrofile") => "avro" + case s => s + } + + serdeMap.get(key) + } + + def getDefaultStorage(conf: SQLConf): CatalogStorageFormat = { + val defaultStorageType = conf.getConfString("hive.default.fileformat", "textfile") + val defaultHiveSerde = sourceToSerDe(defaultStorageType) + CatalogStorageFormat.empty.copy( + inputFormat = defaultHiveSerde.flatMap(_.inputFormat) + .orElse(Some("org.apache.hadoop.mapred.TextInputFormat")), + outputFormat = defaultHiveSerde.flatMap(_.outputFormat) + .orElse(Some("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat")), + serde = defaultHiveSerde.flatMap(_.serde) + .orElse(Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"))) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala new file mode 100644 index 000000000000..accbea41b960 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.internal + +import java.io.File + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.optimizer.Optimizer +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.streaming.StreamingQueryManager +import org.apache.spark.sql.util.{ExecutionListenerManager, QueryExecutionListener} + +/** + * A class that holds all session-specific state in a given [[SparkSession]]. + * + * @param sharedState The state shared across sessions, e.g. global view manager, external catalog. + * @param conf SQL-specific key-value configurations. + * @param experimentalMethods Interface to add custom planning strategies and optimizers. + * @param functionRegistry Internal catalog for managing functions registered by the user. + * @param udfRegistration Interface exposed to the user for registering user-defined functions. + * @param catalogBuilder a function to create an internal catalog for managing table and database + * states. + * @param sqlParser Parser that extracts expressions, plans, table identifiers etc. from SQL texts. + * @param analyzerBuilder A function to create the logical query plan analyzer for resolving + * unresolved attributes and relations. + * @param optimizerBuilder a function to create the logical query plan optimizer. + * @param planner Planner that converts optimized logical plans to physical plans. + * @param streamingQueryManager Interface to start and stop streaming queries. + * @param listenerManager Interface to register custom [[QueryExecutionListener]]s. + * @param resourceLoaderBuilder a function to create a session shared resource loader to load JARs, + * files, etc. + * @param createQueryExecution Function used to create QueryExecution objects. + * @param createClone Function used to create clones of the session state. + */ +private[sql] class SessionState( + sharedState: SharedState, + val conf: SQLConf, + val experimentalMethods: ExperimentalMethods, + val functionRegistry: FunctionRegistry, + val udfRegistration: UDFRegistration, + catalogBuilder: () => SessionCatalog, + val sqlParser: ParserInterface, + analyzerBuilder: () => Analyzer, + optimizerBuilder: () => Optimizer, + val planner: SparkPlanner, + val streamingQueryManager: StreamingQueryManager, + val listenerManager: ExecutionListenerManager, + resourceLoaderBuilder: () => SessionResourceLoader, + createQueryExecution: LogicalPlan => QueryExecution, + createClone: (SparkSession, SessionState) => SessionState) { + + // The following fields are lazy to avoid creating the Hive client when creating SessionState. + lazy val catalog: SessionCatalog = catalogBuilder() + + lazy val analyzer: Analyzer = analyzerBuilder() + + lazy val optimizer: Optimizer = optimizerBuilder() + + lazy val resourceLoader: SessionResourceLoader = resourceLoaderBuilder() + + def newHadoopConf(): Configuration = SessionState.newHadoopConf( + sharedState.sparkContext.hadoopConfiguration, + conf) + + def newHadoopConfWithOptions(options: Map[String, String]): Configuration = { + val hadoopConf = newHadoopConf() + options.foreach { case (k, v) => + if ((v ne null) && k != "path" && k != "paths") { + hadoopConf.set(k, v) + } + } + hadoopConf + } + + /** + * Get an identical copy of the `SessionState` and associate it with the given `SparkSession` + */ + def clone(newSparkSession: SparkSession): SessionState = createClone(newSparkSession, this) + + // ------------------------------------------------------ + // Helper methods, partially leftover from pre-2.0 days + // ------------------------------------------------------ + + def executePlan(plan: LogicalPlan): QueryExecution = createQueryExecution(plan) + + def refreshTable(tableName: String): Unit = { + catalog.refreshTable(sqlParser.parseTableIdentifier(tableName)) + } +} + +private[sql] object SessionState { + def newHadoopConf(hadoopConf: Configuration, sqlConf: SQLConf): Configuration = { + val newHadoopConf = new Configuration(hadoopConf) + sqlConf.getAllConfs.foreach { case (k, v) => if (v ne null) newHadoopConf.set(k, v) } + newHadoopConf + } +} + +/** + * Concrete implementation of a [[BaseSessionStateBuilder]]. + */ +@Experimental +@InterfaceStability.Unstable +class SessionStateBuilder( + session: SparkSession, + parentState: Option[SessionState] = None) + extends BaseSessionStateBuilder(session, parentState) { + override protected def newBuilder: NewBuilder = new SessionStateBuilder(_, _) +} + +/** + * Session shared [[FunctionResourceLoader]]. + */ +@InterfaceStability.Unstable +class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoader { + override def loadResource(resource: FunctionResource): Unit = { + resource.resourceType match { + case JarResource => addJar(resource.uri) + case FileResource => session.sparkContext.addFile(resource.uri) + case ArchiveResource => + throw new AnalysisException( + "Archive is not allowed to be loaded. If YARN mode is used, " + + "please use --archives options while calling spark-submit.") + } + } + + /** + * Add a jar path to [[SparkContext]] and the classloader. + * + * Note: this method seems not access any session state, but a Hive based `SessionState` needs + * to add the jar to its hive client for the current session. Hence, it still needs to be in + * [[SessionState]]. + */ + def addJar(path: String): Unit = { + session.sparkContext.addJar(path) + val uri = new Path(path).toUri + val jarURL = if (uri.getScheme == null) { + // `path` is a local file path without a URL scheme + new File(path).toURI.toURL + } else { + // `path` is a URL with a scheme + uri.toURL + } + session.sharedState.jarClassLoader.addURL(jarURL) + Thread.currentThread().setContextClassLoader(session.sharedState.jarClassLoader) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala new file mode 100644 index 000000000000..ad9db308b262 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.internal + +import java.net.URL +import java.util.Locale + +import scala.reflect.ClassTag +import scala.util.control.NonFatal + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FsUrlStreamHandlerFactory + +import org.apache.spark.{SparkConf, SparkContext, SparkException} +import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.LiveListenerBus +import org.apache.spark.sql.{SparkSession, SQLContext} +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.execution.CacheManager +import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab} +import org.apache.spark.sql.internal.StaticSQLConf._ +import org.apache.spark.util.{MutableURLClassLoader, Utils} + + +/** + * A class that holds all state shared across sessions in a given [[SQLContext]]. + */ +private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { + + // Load hive-site.xml into hadoopConf and determine the warehouse path we want to use, based on + // the config from both hive and Spark SQL. Finally set the warehouse config value to sparkConf. + val warehousePath: String = { + val configFile = Utils.getContextOrSparkClassLoader.getResource("hive-site.xml") + if (configFile != null) { + logInfo(s"loading hive config file: $configFile") + sparkContext.hadoopConfiguration.addResource(configFile) + } + + // hive.metastore.warehouse.dir only stay in hadoopConf + sparkContext.conf.remove("hive.metastore.warehouse.dir") + // Set the Hive metastore warehouse path to the one we use + val hiveWarehouseDir = sparkContext.hadoopConfiguration.get("hive.metastore.warehouse.dir") + if (hiveWarehouseDir != null && !sparkContext.conf.contains(WAREHOUSE_PATH.key)) { + // If hive.metastore.warehouse.dir is set and spark.sql.warehouse.dir is not set, + // we will respect the value of hive.metastore.warehouse.dir. + sparkContext.conf.set(WAREHOUSE_PATH.key, hiveWarehouseDir) + logInfo(s"${WAREHOUSE_PATH.key} is not set, but hive.metastore.warehouse.dir " + + s"is set. Setting ${WAREHOUSE_PATH.key} to the value of " + + s"hive.metastore.warehouse.dir ('$hiveWarehouseDir').") + hiveWarehouseDir + } else { + // If spark.sql.warehouse.dir is set, we will override hive.metastore.warehouse.dir using + // the value of spark.sql.warehouse.dir. + // When neither spark.sql.warehouse.dir nor hive.metastore.warehouse.dir is set, + // we will set hive.metastore.warehouse.dir to the default value of spark.sql.warehouse.dir. + val sparkWarehouseDir = sparkContext.conf.get(WAREHOUSE_PATH) + logInfo(s"Setting hive.metastore.warehouse.dir ('$hiveWarehouseDir') to the value of " + + s"${WAREHOUSE_PATH.key} ('$sparkWarehouseDir').") + sparkContext.hadoopConfiguration.set("hive.metastore.warehouse.dir", sparkWarehouseDir) + sparkWarehouseDir + } + } + logInfo(s"Warehouse path is '$warehousePath'.") + + + /** + * Class for caching query results reused in future executions. + */ + val cacheManager: CacheManager = new CacheManager + + /** + * A listener for SQL-specific [[org.apache.spark.scheduler.SparkListenerEvent]]s. + */ + val listener: SQLListener = createListenerAndUI(sparkContext) + + /** + * A catalog that interacts with external systems. + */ + lazy val externalCatalog: ExternalCatalog = { + val externalCatalog = SharedState.reflect[ExternalCatalog, SparkConf, Configuration]( + SharedState.externalCatalogClassName(sparkContext.conf), + sparkContext.conf, + sparkContext.hadoopConfiguration) + + val defaultDbDefinition = CatalogDatabase( + SessionCatalog.DEFAULT_DATABASE, + "default database", + CatalogUtils.stringToURI(warehousePath), + Map()) + // Create default database if it doesn't exist + if (!externalCatalog.databaseExists(SessionCatalog.DEFAULT_DATABASE)) { + // There may be another Spark application creating default database at the same time, here we + // set `ignoreIfExists = true` to avoid `DatabaseAlreadyExists` exception. + externalCatalog.createDatabase(defaultDbDefinition, ignoreIfExists = true) + } + + // Make sure we propagate external catalog events to the spark listener bus + externalCatalog.addListener(new ExternalCatalogEventListener { + override def onEvent(event: ExternalCatalogEvent): Unit = { + sparkContext.listenerBus.post(event) + } + }) + + externalCatalog + } + + /** + * A manager for global temporary views. + */ + lazy val globalTempViewManager: GlobalTempViewManager = { + // System preserved database should not exists in metastore. However it's hard to guarantee it + // for every session, because case-sensitivity differs. Here we always lowercase it to make our + // life easier. + val globalTempDB = sparkContext.conf.get(GLOBAL_TEMP_DATABASE).toLowerCase(Locale.ROOT) + if (externalCatalog.databaseExists(globalTempDB)) { + throw new SparkException( + s"$globalTempDB is a system preserved database, please rename your existing database " + + "to resolve the name conflict, or set a different value for " + + s"${GLOBAL_TEMP_DATABASE.key}, and launch your Spark application again.") + } + new GlobalTempViewManager(globalTempDB) + } + + /** + * A classloader used to load all user-added jar. + */ + val jarClassLoader = new NonClosableMutableURLClassLoader( + org.apache.spark.util.Utils.getContextOrSparkClassLoader) + + /** + * Create a SQLListener then add it into SparkContext, and create a SQLTab if there is SparkUI. + */ + private def createListenerAndUI(sc: SparkContext): SQLListener = { + if (SparkSession.sqlListener.get() == null) { + val listener = new SQLListener(sc.conf) + if (SparkSession.sqlListener.compareAndSet(null, listener)) { + sc.listenerBus.addToStatusQueue(listener) + sc.ui.foreach(new SQLTab(listener, _)) + } + } + SparkSession.sqlListener.get() + } +} + +object SharedState extends Logging { + try { + URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory()) + } catch { + case e: Error => + logWarning("URL.setURLStreamHandlerFactory failed to set FsUrlStreamHandlerFactory") + } + + private val HIVE_EXTERNAL_CATALOG_CLASS_NAME = "org.apache.spark.sql.hive.HiveExternalCatalog" + + private def externalCatalogClassName(conf: SparkConf): String = { + conf.get(CATALOG_IMPLEMENTATION) match { + case "hive" => HIVE_EXTERNAL_CATALOG_CLASS_NAME + case "in-memory" => classOf[InMemoryCatalog].getCanonicalName + } + } + + /** + * Helper method to create an instance of [[T]] using a single-arg constructor that + * accepts an [[Arg1]] and an [[Arg2]]. + */ + private def reflect[T, Arg1 <: AnyRef, Arg2 <: AnyRef]( + className: String, + ctorArg1: Arg1, + ctorArg2: Arg2)( + implicit ctorArgTag1: ClassTag[Arg1], + ctorArgTag2: ClassTag[Arg2]): T = { + try { + val clazz = Utils.classForName(className) + val ctor = clazz.getDeclaredConstructor(ctorArgTag1.runtimeClass, ctorArgTag2.runtimeClass) + val args = Array[AnyRef](ctorArg1, ctorArg2) + ctor.newInstance(args: _*).asInstanceOf[T] + } catch { + case NonFatal(e) => + throw new IllegalArgumentException(s"Error while instantiating '$className':", e) + } + } +} + + +/** + * URL class loader that exposes the `addURL` and `getURLs` methods in URLClassLoader. + * This class loader cannot be closed (its `close` method is a no-op). + */ +private[sql] class NonClosableMutableURLClassLoader(parent: ClassLoader) + extends MutableURLClassLoader(Array.empty, parent) { + + override def close(): Unit = {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala new file mode 100644 index 000000000000..4e7c813be992 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.internal + +import org.apache.spark.internal.config._ + +/** + * A helper class that enables substitution using syntax like + * `${var}`, `${system:var}` and `${env:var}`. + * + * Variable substitution is controlled by `SQLConf.variableSubstituteEnabled`. + */ +class VariableSubstitution(conf: SQLConf) { + + private val provider = new ConfigProvider { + override def get(key: String): Option[String] = Option(conf.getConfString(key, "")) + } + + private val reader = new ConfigReader(provider) + .bind("spark", provider) + .bind("sparkconf", provider) + .bind("hivevar", provider) + .bind("hiveconf", provider) + + /** + * Given a query, does variable substitution and return the result. + */ + def substitute(input: String): String = { + if (conf.variableSubstituteEnabled) { + reader.substitute(input) + } else { + input + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/package-info.java b/sql/core/src/main/scala/org/apache/spark/sql/internal/package-info.java new file mode 100644 index 000000000000..1e801cb6ee2a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * All classes in this package are considered an internal API to Spark and + * are subject to change between minor releases. + */ +package org.apache.spark.sql.internal; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/package.scala new file mode 100644 index 000000000000..c2394f42e552 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/package.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 + +/** + * All classes in this package are considered an internal API to Spark and + * are subject to change between minor releases. + */ +package object internal diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala new file mode 100644 index 000000000000..1419d69f983a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/AggregatedDialect.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import org.apache.spark.sql.types.{DataType, MetadataBuilder} + +/** + * AggregatedDialect can unify multiple dialects into one virtual Dialect. + * Dialects are tried in order, and the first dialect that does not return a + * neutral element will will. + * + * @param dialects List of dialects. + */ +private class AggregatedDialect(dialects: List[JdbcDialect]) extends JdbcDialect { + + require(dialects.nonEmpty) + + override def canHandle(url : String): Boolean = + dialects.map(_.canHandle(url)).reduce(_ && _) + + override def getCatalystType( + sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { + dialects.flatMap(_.getCatalystType(sqlType, typeName, size, md)).headOption + } + + override def getJDBCType(dt: DataType): Option[JdbcType] = { + dialects.flatMap(_.getJDBCType(dt)).headOption + } + + override def isCascadingTruncateTable(): Option[Boolean] = { + // If any dialect claims cascading truncate, this dialect is also cascading truncate. + // Otherwise, if any dialect has unknown cascading truncate, this dialect is also unknown. + dialects.flatMap(_.isCascadingTruncateTable()).reduceOption(_ || _) match { + case Some(true) => Some(true) + case _ if dialects.exists(_.isCascadingTruncateTable().isEmpty) => None + case _ => Some(false) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala new file mode 100644 index 000000000000..d160ad82888a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DB2Dialect.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.sql.Types + +import org.apache.spark.sql.types._ + +private object DB2Dialect extends JdbcDialect { + + override def canHandle(url: String): Boolean = url.startsWith("jdbc:db2") + + override def getCatalystType( + sqlType: Int, + typeName: String, + size: Int, + md: MetadataBuilder): Option[DataType] = sqlType match { + case Types.REAL => Option(FloatType) + case Types.OTHER => + typeName match { + case "DECFLOAT" => Option(DecimalType(38, 18)) + case "XML" => Option(StringType) + case t if (t.startsWith("TIMESTAMP")) => Option(TimestampType) // TIMESTAMP WITH TIMEZONE + case _ => None + } + case _ => None + } + + override def getJDBCType(dt: DataType): Option[JdbcType] = dt match { + case StringType => Option(JdbcType("CLOB", java.sql.Types.CLOB)) + case BooleanType => Option(JdbcType("CHAR(1)", java.sql.Types.CHAR)) + case ShortType | ByteType => Some(JdbcType("SMALLINT", java.sql.Types.SMALLINT)) + case _ => None + } + + override def isCascadingTruncateTable(): Option[Boolean] = Some(false) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala new file mode 100644 index 000000000000..84f68e779c38 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DerbyDialect.scala @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.sql.Types + +import org.apache.spark.sql.types._ + + +private object DerbyDialect extends JdbcDialect { + + override def canHandle(url: String): Boolean = url.startsWith("jdbc:derby") + + override def getCatalystType( + sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { + if (sqlType == Types.REAL) Option(FloatType) else None + } + + override def getJDBCType(dt: DataType): Option[JdbcType] = dt match { + case StringType => Option(JdbcType("CLOB", java.sql.Types.CLOB)) + case ByteType => Option(JdbcType("SMALLINT", java.sql.Types.SMALLINT)) + case ShortType => Option(JdbcType("SMALLINT", java.sql.Types.SMALLINT)) + case BooleanType => Option(JdbcType("BOOLEAN", java.sql.Types.BOOLEAN)) + // 31 is the maximum precision and 5 is the default scale for a Derby DECIMAL + case t: DecimalType if t.precision > 31 => + Option(JdbcType("DECIMAL(31,5)", java.sql.Types.DECIMAL)) + case _ => None + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index 88ae83957a70..7c38ed68c041 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -17,10 +17,12 @@ package org.apache.spark.sql.jdbc -import java.sql.Types +import java.sql.{Connection, Date, Timestamp} +import org.apache.commons.lang3.StringUtils + +import org.apache.spark.annotation.{DeveloperApi, InterfaceStability, Since} import org.apache.spark.sql.types._ -import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: @@ -31,6 +33,7 @@ import org.apache.spark.annotation.DeveloperApi * send a null value to the database. */ @DeveloperApi +@InterfaceStability.Evolving case class JdbcType(databaseTypeDefinition : String, jdbcNullType : Int) /** @@ -39,8 +42,8 @@ case class JdbcType(databaseTypeDefinition : String, jdbcNullType : Int) * SQL dialect of a certain database or jdbc driver. * Lots of databases define types that aren't explicitly supported * by the JDBC spec. Some JDBC drivers also report inaccurate - * information---for instance, BIT(n>1) being reported as a BIT type is quite - * common, even though BIT in JDBC is meant for single-bit values. Also, there + * information---for instance, BIT(n{@literal >}1) being reported as a BIT type is quite + * common, even though BIT in JDBC is meant for single-bit values. Also, there * does not appear to be a standard name for an unbounded string or binary * type; we use BLOB and CLOB by default but override with database-specific * alternatives when these are absent or do not behave correctly. @@ -53,7 +56,8 @@ case class JdbcType(databaseTypeDefinition : String, jdbcNullType : Int) * for the given Catalyst type. */ @DeveloperApi -abstract class JdbcDialect { +@InterfaceStability.Evolving +abstract class JdbcDialect extends Serializable { /** * Check if this dialect instance can handle a certain jdbc url. * @param url the jdbc url. @@ -99,27 +103,79 @@ abstract class JdbcDialect { s"SELECT * FROM $table WHERE 1=0" } + /** + * The SQL query that should be used to discover the schema of a table. It only needs to + * ensure that the result set has the same schema as the table, such as by calling + * "SELECT * ...". Dialects can override this method to return a query that works best in a + * particular database. + * @param table The name of the table. + * @return The SQL query to use for discovering the schema. + */ + @Since("2.1.0") + def getSchemaQuery(table: String): String = { + s"SELECT * FROM $table WHERE 1=0" + } + + /** + * Override connection specific properties to run before a select is made. This is in place to + * allow dialects that need special treatment to optimize behavior. + * @param connection The connection object + * @param properties The connection properties. This is passed through from the relation. + */ + def beforeFetch(connection: Connection, properties: Map[String, String]): Unit = { + } + + /** + * Escape special characters in SQL string literals. + * @param value The string to be escaped. + * @return Escaped string. + */ + @Since("2.3.0") + protected[jdbc] def escapeSql(value: String): String = + if (value == null) null else StringUtils.replace(value, "'", "''") + + /** + * Converts value to SQL expression. + * @param value The value to be converted. + * @return Converted value. + */ + @Since("2.3.0") + def compileValue(value: Any): Any = value match { + case stringValue: String => s"'${escapeSql(stringValue)}'" + case timestampValue: Timestamp => "'" + timestampValue + "'" + case dateValue: Date => "'" + dateValue + "'" + case arrayValue: Array[Any] => arrayValue.map(compileValue).mkString(", ") + case _ => value + } + + /** + * Return Some[true] iff `TRUNCATE TABLE` causes cascading default. + * Some[true] : TRUNCATE TABLE causes cascading. + * Some[false] : TRUNCATE TABLE does not cause cascading. + * None: The behavior of TRUNCATE TABLE is unknown (default). + */ + def isCascadingTruncateTable(): Option[Boolean] = None } /** * :: DeveloperApi :: - * Registry of dialects that apply to every new jdbc [[org.apache.spark.sql.DataFrame]]. + * Registry of dialects that apply to every new jdbc `org.apache.spark.sql.DataFrame`. * * If multiple matching dialects are registered then all matching ones will be * tried in reverse order. A user-added dialect will thus be applied first, * overwriting the defaults. * - * Note that all new dialects are applied to new jdbc DataFrames only. Make + * @note All new dialects are applied to new jdbc DataFrames only. Make * sure to register your dialects first. */ @DeveloperApi +@InterfaceStability.Evolving object JdbcDialects { - private var dialects = List[JdbcDialect]() - /** - * Register a dialect for use on all new matching jdbc [[org.apache.spark.sql.DataFrame]]. - * Readding an existing dialect will cause a move-to-front. + * Register a dialect for use on all new matching jdbc `org.apache.spark.sql.DataFrame`. + * Reading an existing dialect will cause a move-to-front. + * * @param dialect The new dialect. */ def registerDialect(dialect: JdbcDialect) : Unit = { @@ -128,23 +184,27 @@ object JdbcDialects { /** * Unregister a dialect. Does nothing if the dialect is not registered. + * * @param dialect The jdbc dialect. */ def unregisterDialect(dialect : JdbcDialect) : Unit = { dialects = dialects.filterNot(_ == dialect) } + private[this] var dialects = List[JdbcDialect]() + registerDialect(MySQLDialect) registerDialect(PostgresDialect) registerDialect(DB2Dialect) registerDialect(MsSqlServerDialect) registerDialect(DerbyDialect) - + registerDialect(OracleDialect) + registerDialect(TeradataDialect) /** * Fetch the JdbcDialect class corresponding to a given database url. */ - private[sql] def get(url: String): JdbcDialect = { + def get(url: String): JdbcDialect = { val matchingDialects = dialects.filter(_.canHandle(url)) matchingDialects.length match { case 0 => NoopDialect @@ -155,163 +215,8 @@ object JdbcDialects { } /** - * :: DeveloperApi :: - * AggregatedDialect can unify multiple dialects into one virtual Dialect. - * Dialects are tried in order, and the first dialect that does not return a - * neutral element will will. - * @param dialects List of dialects. - */ -@DeveloperApi -class AggregatedDialect(dialects: List[JdbcDialect]) extends JdbcDialect { - - require(dialects.nonEmpty) - - override def canHandle(url : String): Boolean = - dialects.map(_.canHandle(url)).reduce(_ && _) - - override def getCatalystType( - sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { - dialects.flatMap(_.getCatalystType(sqlType, typeName, size, md)).headOption - } - - override def getJDBCType(dt: DataType): Option[JdbcType] = { - dialects.flatMap(_.getJDBCType(dt)).headOption - } -} - -/** - * :: DeveloperApi :: * NOOP dialect object, always returning the neutral element. */ -@DeveloperApi -case object NoopDialect extends JdbcDialect { +private object NoopDialect extends JdbcDialect { override def canHandle(url : String): Boolean = true } - -/** - * :: DeveloperApi :: - * Default postgres dialect, mapping bit/cidr/inet on read and string/binary/boolean on write. - */ -@DeveloperApi -case object PostgresDialect extends JdbcDialect { - override def canHandle(url: String): Boolean = url.startsWith("jdbc:postgresql") - override def getCatalystType( - sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { - if (sqlType == Types.BIT && typeName.equals("bit") && size != 1) { - Option(BinaryType) - } else if (sqlType == Types.OTHER && typeName.equals("cidr")) { - Option(StringType) - } else if (sqlType == Types.OTHER && typeName.equals("inet")) { - Option(StringType) - } else if (sqlType == Types.OTHER && typeName.equals("json")) { - Option(StringType) - } else if (sqlType == Types.OTHER && typeName.equals("jsonb")) { - Option(StringType) - } else None - } - - override def getJDBCType(dt: DataType): Option[JdbcType] = dt match { - case StringType => Some(JdbcType("TEXT", java.sql.Types.CHAR)) - case BinaryType => Some(JdbcType("BYTEA", java.sql.Types.BINARY)) - case BooleanType => Some(JdbcType("BOOLEAN", java.sql.Types.BOOLEAN)) - case _ => None - } - - override def getTableExistsQuery(table: String): String = { - s"SELECT 1 FROM $table LIMIT 1" - } - -} - -/** - * :: DeveloperApi :: - * Default mysql dialect to read bit/bitsets correctly. - */ -@DeveloperApi -case object MySQLDialect extends JdbcDialect { - override def canHandle(url : String): Boolean = url.startsWith("jdbc:mysql") - override def getCatalystType( - sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { - if (sqlType == Types.VARBINARY && typeName.equals("BIT") && size != 1) { - // This could instead be a BinaryType if we'd rather return bit-vectors of up to 64 bits as - // byte arrays instead of longs. - md.putLong("binarylong", 1) - Option(LongType) - } else if (sqlType == Types.BIT && typeName.equals("TINYINT")) { - Option(BooleanType) - } else None - } - - override def quoteIdentifier(colName: String): String = { - s"`$colName`" - } - - override def getTableExistsQuery(table: String): String = { - s"SELECT 1 FROM $table LIMIT 1" - } -} - -/** - * :: DeveloperApi :: - * Default DB2 dialect, mapping string/boolean on write to valid DB2 types. - * By default string, and boolean gets mapped to db2 invalid types TEXT, and BIT(1). - */ -@DeveloperApi -case object DB2Dialect extends JdbcDialect { - - override def canHandle(url: String): Boolean = url.startsWith("jdbc:db2") - - override def getJDBCType(dt: DataType): Option[JdbcType] = dt match { - case StringType => Some(JdbcType("CLOB", java.sql.Types.CLOB)) - case BooleanType => Some(JdbcType("CHAR(1)", java.sql.Types.CHAR)) - case _ => None - } -} - -/** - * :: DeveloperApi :: - * Default Microsoft SQL Server dialect, mapping the datetimeoffset types to a String on read. - */ -@DeveloperApi -case object MsSqlServerDialect extends JdbcDialect { - override def canHandle(url: String): Boolean = url.startsWith("jdbc:sqlserver") - override def getCatalystType( - sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { - if (typeName.contains("datetimeoffset")) { - // String is recommend by Microsoft SQL Server for datetimeoffset types in non-MS clients - Option(StringType) - } else None - } - - override def getJDBCType(dt: DataType): Option[JdbcType] = dt match { - case TimestampType => Some(JdbcType("DATETIME", java.sql.Types.TIMESTAMP)) - case _ => None - } -} - -/** - * :: DeveloperApi :: - * Default Apache Derby dialect, mapping real on read - * and string/byte/short/boolean/decimal on write. - */ -@DeveloperApi -case object DerbyDialect extends JdbcDialect { - override def canHandle(url: String): Boolean = url.startsWith("jdbc:derby") - override def getCatalystType( - sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { - if (sqlType == Types.REAL) Option(FloatType) else None - } - - override def getJDBCType(dt: DataType): Option[JdbcType] = dt match { - case StringType => Some(JdbcType("CLOB", java.sql.Types.CLOB)) - case ByteType => Some(JdbcType("SMALLINT", java.sql.Types.SMALLINT)) - case ShortType => Some(JdbcType("SMALLINT", java.sql.Types.SMALLINT)) - case BooleanType => Some(JdbcType("BOOLEAN", java.sql.Types.BOOLEAN)) - // 31 is the maximum precision and 5 is the default scale for a Derby DECIMAL - case (t: DecimalType) if (t.precision > 31) => - Some(JdbcType("DECIMAL(31,5)", java.sql.Types.DECIMAL)) - case _ => None - } - -} - diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala new file mode 100644 index 000000000000..da787b4859a7 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala @@ -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.jdbc + +import org.apache.spark.sql.types._ + + +private object MsSqlServerDialect extends JdbcDialect { + + override def canHandle(url: String): Boolean = url.startsWith("jdbc:sqlserver") + + override def getCatalystType( + sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { + if (typeName.contains("datetimeoffset")) { + // String is recommend by Microsoft SQL Server for datetimeoffset types in non-MS clients + Option(StringType) + } else { + None + } + } + + override def getJDBCType(dt: DataType): Option[JdbcType] = dt match { + case TimestampType => Some(JdbcType("DATETIME", java.sql.Types.TIMESTAMP)) + case StringType => Some(JdbcType("NVARCHAR(MAX)", java.sql.Types.NVARCHAR)) + case BooleanType => Some(JdbcType("BIT", java.sql.Types.BIT)) + case _ => None + } + + override def isCascadingTruncateTable(): Option[Boolean] = Some(false) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala new file mode 100644 index 000000000000..b2cff7877d8b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MySQLDialect.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.sql.Types + +import org.apache.spark.sql.types.{BooleanType, DataType, LongType, MetadataBuilder} + +private case object MySQLDialect extends JdbcDialect { + + override def canHandle(url : String): Boolean = url.startsWith("jdbc:mysql") + + override def getCatalystType( + sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { + if (sqlType == Types.VARBINARY && typeName.equals("BIT") && size != 1) { + // This could instead be a BinaryType if we'd rather return bit-vectors of up to 64 bits as + // byte arrays instead of longs. + md.putLong("binarylong", 1) + Option(LongType) + } else if (sqlType == Types.BIT && typeName.equals("TINYINT")) { + Option(BooleanType) + } else None + } + + override def quoteIdentifier(colName: String): String = { + s"`$colName`" + } + + override def getTableExistsQuery(table: String): String = { + s"SELECT 1 FROM $table LIMIT 1" + } + + override def isCascadingTruncateTable(): Option[Boolean] = Some(false) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala new file mode 100644 index 000000000000..3b44c1de93a6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/OracleDialect.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.sql.{Date, Timestamp, Types} + +import org.apache.spark.sql.types._ + + +private case object OracleDialect extends JdbcDialect { + + override def canHandle(url: String): Boolean = url.startsWith("jdbc:oracle") + + override def getCatalystType( + sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { + if (sqlType == Types.NUMERIC) { + val scale = if (null != md) md.build().getLong("scale") else 0L + size match { + // Handle NUMBER fields that have no precision/scale in special way + // because JDBC ResultSetMetaData converts this to 0 precision and -127 scale + // For more details, please see + // https://github.com/apache/spark/pull/8780#issuecomment-145598968 + // and + // https://github.com/apache/spark/pull/8780#issuecomment-144541760 + case 0 => Option(DecimalType(DecimalType.MAX_PRECISION, 10)) + // Handle FLOAT fields in a special way because JDBC ResultSetMetaData converts + // this to NUMERIC with -127 scale + // Not sure if there is a more robust way to identify the field as a float (or other + // numeric types that do not specify a scale. + case _ if scale == -127L => Option(DecimalType(DecimalType.MAX_PRECISION, 10)) + case _ => None + } + } else { + None + } + } + + override def getJDBCType(dt: DataType): Option[JdbcType] = dt match { + // For more details, please see + // https://docs.oracle.com/cd/E19501-01/819-3659/gcmaz/ + case BooleanType => Some(JdbcType("NUMBER(1)", java.sql.Types.BOOLEAN)) + case IntegerType => Some(JdbcType("NUMBER(10)", java.sql.Types.INTEGER)) + case LongType => Some(JdbcType("NUMBER(19)", java.sql.Types.BIGINT)) + case FloatType => Some(JdbcType("NUMBER(19, 4)", java.sql.Types.FLOAT)) + case DoubleType => Some(JdbcType("NUMBER(19, 4)", java.sql.Types.DOUBLE)) + case ByteType => Some(JdbcType("NUMBER(3)", java.sql.Types.SMALLINT)) + case ShortType => Some(JdbcType("NUMBER(5)", java.sql.Types.SMALLINT)) + case StringType => Some(JdbcType("VARCHAR2(255)", java.sql.Types.VARCHAR)) + case _ => None + } + + override def compileValue(value: Any): Any = value match { + // The JDBC drivers support date literals in SQL statements written in the + // format: {d 'yyyy-mm-dd'} and timestamp literals in SQL statements written + // in the format: {ts 'yyyy-mm-dd hh:mm:ss.f...'}. For details, see + // 'Oracle Database JDBC Developer’s Guide and Reference, 11g Release 1 (11.1)' + // Appendix A Reference Information. + case stringValue: String => s"'${escapeSql(stringValue)}'" + case timestampValue: Timestamp => "{ts '" + timestampValue + "'}" + case dateValue: Date => "{d '" + dateValue + "'}" + case arrayValue: Array[Any] => arrayValue.map(compileValue).mkString(", ") + case _ => value + } + + override def isCascadingTruncateTable(): Option[Boolean] = Some(false) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala new file mode 100644 index 000000000000..4f61a328f47c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.sql.{Connection, Types} + +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} +import org.apache.spark.sql.types._ + + +private object PostgresDialect extends JdbcDialect { + + override def canHandle(url: String): Boolean = url.startsWith("jdbc:postgresql") + + override def getCatalystType( + sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): Option[DataType] = { + if (sqlType == Types.REAL) { + Some(FloatType) + } else if (sqlType == Types.SMALLINT) { + Some(ShortType) + } else if (sqlType == Types.BIT && typeName.equals("bit") && size != 1) { + Some(BinaryType) + } else if (sqlType == Types.OTHER) { + Some(StringType) + } else if (sqlType == Types.ARRAY) { + val scale = md.build.getLong("scale").toInt + // postgres array type names start with underscore + toCatalystType(typeName.drop(1), size, scale).map(ArrayType(_)) + } else None + } + + private def toCatalystType( + typeName: String, + precision: Int, + scale: Int): Option[DataType] = typeName match { + case "bool" => Some(BooleanType) + case "bit" => Some(BinaryType) + case "int2" => Some(ShortType) + case "int4" => Some(IntegerType) + case "int8" | "oid" => Some(LongType) + case "float4" => Some(FloatType) + case "money" | "float8" => Some(DoubleType) + case "text" | "varchar" | "char" | "cidr" | "inet" | "json" | "jsonb" | "uuid" => + Some(StringType) + case "bytea" => Some(BinaryType) + case "timestamp" | "timestamptz" | "time" | "timetz" => Some(TimestampType) + case "date" => Some(DateType) + case "numeric" | "decimal" => Some(DecimalType.bounded(precision, scale)) + case _ => None + } + + override def getJDBCType(dt: DataType): Option[JdbcType] = dt match { + case StringType => Some(JdbcType("TEXT", Types.CHAR)) + case BinaryType => Some(JdbcType("BYTEA", Types.BINARY)) + case BooleanType => Some(JdbcType("BOOLEAN", Types.BOOLEAN)) + case FloatType => Some(JdbcType("FLOAT4", Types.FLOAT)) + case DoubleType => Some(JdbcType("FLOAT8", Types.DOUBLE)) + case ShortType => Some(JdbcType("SMALLINT", Types.SMALLINT)) + case t: DecimalType => Some( + JdbcType(s"NUMERIC(${t.precision},${t.scale})", java.sql.Types.NUMERIC)) + case ArrayType(et, _) if et.isInstanceOf[AtomicType] => + getJDBCType(et).map(_.databaseTypeDefinition) + .orElse(JdbcUtils.getCommonJDBCType(et).map(_.databaseTypeDefinition)) + .map(typeName => JdbcType(s"$typeName[]", java.sql.Types.ARRAY)) + case ByteType => throw new IllegalArgumentException(s"Unsupported type in postgresql: $dt"); + case _ => None + } + + override def getTableExistsQuery(table: String): String = { + s"SELECT 1 FROM $table LIMIT 1" + } + + override def beforeFetch(connection: Connection, properties: Map[String, String]): Unit = { + super.beforeFetch(connection, properties) + + // According to the postgres jdbc documentation we need to be in autocommit=false if we actually + // want to have fetchsize be non 0 (all the rows). This allows us to not have to cache all the + // rows inside the driver when fetching. + // + // See: https://jdbc.postgresql.org/documentation/head/query.html#query-with-cursor + // + if (properties.getOrElse(JDBCOptions.JDBC_BATCH_FETCH_SIZE, "0").toInt > 0) { + connection.setAutoCommit(false) + } + + } + + override def isCascadingTruncateTable(): Option[Boolean] = Some(true) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.scala new file mode 100644 index 000000000000..5749b791fca2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/TeradataDialect.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.jdbc + +import java.sql.Types + +import org.apache.spark.sql.types._ + + +private case object TeradataDialect extends JdbcDialect { + + override def canHandle(url: String): Boolean = { url.startsWith("jdbc:teradata") } + + override def getJDBCType(dt: DataType): Option[JdbcType] = dt match { + case StringType => Some(JdbcType("VARCHAR(255)", java.sql.Types.VARCHAR)) + case BooleanType => Option(JdbcType("CHAR(1)", java.sql.Types.CHAR)) + case _ => None + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index a9c600b139b1..161e0102f0b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -17,8 +17,8 @@ package org.apache.spark -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.annotation.{DeveloperApi, InterfaceStability} +import org.apache.spark.sql.execution.SparkStrategy /** * Allows the execution of relational queries, including those expressed in SQL using Spark. @@ -40,12 +40,8 @@ package object sql { * [[org.apache.spark.sql.sources]] */ @DeveloperApi - type Strategy = org.apache.spark.sql.catalyst.planning.GenericStrategy[SparkPlan] + @InterfaceStability.Unstable + type Strategy = SparkStrategy - /** - * Type alias for [[DataFrame]]. Kept here for backward source compatibility for Scala. - * @deprecated As of 1.3.0, replaced by `DataFrame`. - */ - @deprecated("use DataFrame", "1.3.0") - type SchemaRDD = DataFrame + type DataFrame = Dataset[Row] } 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 3780cbbcc963..2499e9b604f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.sources +import org.apache.spark.annotation.InterfaceStability + //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines all the filters that we can push down to the data sources. //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -26,7 +28,19 @@ package org.apache.spark.sql.sources * * @since 1.3.0 */ -abstract class Filter +@InterfaceStability.Stable +abstract class Filter { + /** + * List of columns that are referenced by this filter. + * @since 2.1.0 + */ + def references: Array[String] + + protected def findReferences(value: Any): Array[String] = value match { + case f: Filter => f.references + case _ => Array.empty + } +} /** * A filter that evaluates to `true` iff the attribute evaluates to a value @@ -34,7 +48,10 @@ abstract class Filter * * @since 1.3.0 */ -case class EqualTo(attribute: String, value: Any) extends Filter +@InterfaceStability.Stable +case class EqualTo(attribute: String, value: Any) extends Filter { + override def references: Array[String] = Array(attribute) ++ findReferences(value) +} /** * Performs equality comparison, similar to [[EqualTo]]. However, this differs from [[EqualTo]] @@ -43,7 +60,10 @@ case class EqualTo(attribute: String, value: Any) extends Filter * * @since 1.5.0 */ -case class EqualNullSafe(attribute: String, value: Any) extends Filter +@InterfaceStability.Stable +case class EqualNullSafe(attribute: String, value: Any) extends Filter { + override def references: Array[String] = Array(attribute) ++ findReferences(value) +} /** * A filter that evaluates to `true` iff the attribute evaluates to a value @@ -51,7 +71,10 @@ case class EqualNullSafe(attribute: String, value: Any) extends Filter * * @since 1.3.0 */ -case class GreaterThan(attribute: String, value: Any) extends Filter +@InterfaceStability.Stable +case class GreaterThan(attribute: String, value: Any) extends Filter { + override def references: Array[String] = Array(attribute) ++ findReferences(value) +} /** * A filter that evaluates to `true` iff the attribute evaluates to a value @@ -59,7 +82,10 @@ case class GreaterThan(attribute: String, value: Any) extends Filter * * @since 1.3.0 */ -case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter +@InterfaceStability.Stable +case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter { + override def references: Array[String] = Array(attribute) ++ findReferences(value) +} /** * A filter that evaluates to `true` iff the attribute evaluates to a value @@ -67,7 +93,10 @@ case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter * * @since 1.3.0 */ -case class LessThan(attribute: String, value: Any) extends Filter +@InterfaceStability.Stable +case class LessThan(attribute: String, value: Any) extends Filter { + override def references: Array[String] = Array(attribute) ++ findReferences(value) +} /** * A filter that evaluates to `true` iff the attribute evaluates to a value @@ -75,49 +104,87 @@ case class LessThan(attribute: String, value: Any) extends Filter * * @since 1.3.0 */ -case class LessThanOrEqual(attribute: String, value: Any) extends Filter +@InterfaceStability.Stable +case class LessThanOrEqual(attribute: String, value: Any) extends Filter { + override def references: Array[String] = Array(attribute) ++ findReferences(value) +} /** * A filter that evaluates to `true` iff the attribute evaluates to one of the values in the array. * * @since 1.3.0 */ -case class In(attribute: String, values: Array[Any]) extends Filter +@InterfaceStability.Stable +case class In(attribute: String, values: Array[Any]) extends Filter { + override def hashCode(): Int = { + var h = attribute.hashCode + values.foreach { v => + h *= 41 + h += v.hashCode() + } + h + } + override def equals(o: Any): Boolean = o match { + case In(a, vs) => + a == attribute && vs.length == values.length && vs.zip(values).forall(x => x._1 == x._2) + case _ => false + } + override def toString: String = { + s"In($attribute, [${values.mkString(",")}])" + } + + override def references: Array[String] = Array(attribute) ++ values.flatMap(findReferences) +} /** * A filter that evaluates to `true` iff the attribute evaluates to null. * * @since 1.3.0 */ -case class IsNull(attribute: String) extends Filter +@InterfaceStability.Stable +case class IsNull(attribute: String) extends Filter { + override def references: Array[String] = Array(attribute) +} /** * A filter that evaluates to `true` iff the attribute evaluates to a non-null value. * * @since 1.3.0 */ -case class IsNotNull(attribute: String) extends Filter +@InterfaceStability.Stable +case class IsNotNull(attribute: String) extends Filter { + override def references: Array[String] = Array(attribute) +} /** * A filter that evaluates to `true` iff both `left` or `right` evaluate to `true`. * * @since 1.3.0 */ -case class And(left: Filter, right: Filter) extends Filter +@InterfaceStability.Stable +case class And(left: Filter, right: Filter) extends Filter { + override def references: Array[String] = left.references ++ right.references +} /** * A filter that evaluates to `true` iff at least one of `left` or `right` evaluates to `true`. * * @since 1.3.0 */ -case class Or(left: Filter, right: Filter) extends Filter +@InterfaceStability.Stable +case class Or(left: Filter, right: Filter) extends Filter { + override def references: Array[String] = left.references ++ right.references +} /** * A filter that evaluates to `true` iff `child` is evaluated to `false`. * * @since 1.3.0 */ -case class Not(child: Filter) extends Filter +@InterfaceStability.Stable +case class Not(child: Filter) extends Filter { + override def references: Array[String] = child.references +} /** * A filter that evaluates to `true` iff the attribute evaluates to @@ -125,7 +192,10 @@ case class Not(child: Filter) extends Filter * * @since 1.3.1 */ -case class StringStartsWith(attribute: String, value: String) extends Filter +@InterfaceStability.Stable +case class StringStartsWith(attribute: String, value: String) extends Filter { + override def references: Array[String] = Array(attribute) +} /** * A filter that evaluates to `true` iff the attribute evaluates to @@ -133,7 +203,10 @@ case class StringStartsWith(attribute: String, value: String) extends Filter * * @since 1.3.1 */ -case class StringEndsWith(attribute: String, value: String) extends Filter +@InterfaceStability.Stable +case class StringEndsWith(attribute: String, value: String) extends Filter { + override def references: Array[String] = Array(attribute) +} /** * A filter that evaluates to `true` iff the attribute evaluates to @@ -141,4 +214,7 @@ case class StringEndsWith(attribute: String, value: String) extends Filter * * @since 1.3.1 */ -case class StringContains(attribute: String, value: String) extends Filter +@InterfaceStability.Stable +case class StringContains(attribute: String, value: String) extends Filter { + override def references: Array[String] = Array(attribute) +} 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 e296d631f0f3..6057a795c8bf 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 @@ -17,28 +17,16 @@ package org.apache.spark.sql.sources -import scala.collection.mutable -import scala.util.Try - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} -import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} - -import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.annotation.{DeveloperApi, Experimental} -import org.apache.spark.broadcast.Broadcast +import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection -import org.apache.spark.sql.execution.{FileRelation, RDDConversions} -import org.apache.spark.sql.execution.datasources.{PartitioningUtils, PartitionSpec, Partition} -import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.sql._ -import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.streaming.{Sink, Source} +import org.apache.spark.sql.streaming.OutputMode +import org.apache.spark.sql.types.StructType /** - * ::DeveloperApi:: * Data sources should implement this trait so that they can register an alias to their data source. * This allows users to give the data source alias as the format type over the fully qualified * class name. @@ -47,7 +35,7 @@ import org.apache.spark.util.SerializableConfiguration * * @since 1.5.0 */ -@DeveloperApi +@InterfaceStability.Stable trait DataSourceRegister { /** @@ -55,7 +43,7 @@ trait DataSourceRegister { * overridden by children to provide a nice alias for the data source. For example: * * {{{ - * override def format(): String = "parquet" + * override def shortName(): String = "parquet" * }}} * * @since 1.5.0 @@ -64,7 +52,6 @@ trait DataSourceRegister { } /** - * ::DeveloperApi:: * Implemented by objects that produce relations for a specific kind of data source. When * Spark SQL is given a DDL operation with a USING clause specified (to specify the implemented * RelationProvider), this interface is used to pass in the parameters specified by a user. @@ -78,18 +65,18 @@ trait DataSourceRegister { * * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable trait RelationProvider { /** * Returns a new base relation with the given parameters. - * Note: the parameters' keywords are case insensitive and this insensitivity is enforced + * + * @note The parameters' keywords are case insensitive and this insensitivity is enforced * by the Map that is passed to the function. */ def createRelation(sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation } /** - * ::DeveloperApi:: * Implemented by objects that produce relations for a specific kind of data source * with a given schema. When Spark SQL is given a DDL operation with a USING clause specified ( * to specify the implemented SchemaRelationProvider) and a user defined schema, this interface @@ -104,16 +91,17 @@ trait RelationProvider { * * The difference between a [[RelationProvider]] and a [[SchemaRelationProvider]] is that * users need to provide a schema when using a [[SchemaRelationProvider]]. - * A relation provider can inherits both [[RelationProvider]] and [[SchemaRelationProvider]] + * A relation provider can inherit both [[RelationProvider]] and [[SchemaRelationProvider]] * if it can support both schema inference and user-specified schemas. * * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable trait SchemaRelationProvider { /** * Returns a new base relation with the given parameters and user defined schema. - * Note: the parameters' keywords are case insensitive and this insensitivity is enforced + * + * @note The parameters' keywords are case insensitive and this insensitivity is enforced * by the Map that is passed to the function. */ def createRelation( @@ -124,63 +112,67 @@ trait SchemaRelationProvider { /** * ::Experimental:: - * Implemented by objects that produce relations for a specific kind of data source - * with a given schema and partitioned columns. When Spark SQL is given a DDL operation with a - * USING clause specified (to specify the implemented [[HadoopFsRelationProvider]]), a user defined - * schema, and an optional list of partition columns, this interface is used to pass in the - * parameters specified by a user. - * - * Users may specify the fully qualified class name of a given data source. When that class is - * not found Spark SQL will append the class name `DefaultSource` to the path, allowing for - * less verbose invocation. For example, 'org.apache.spark.sql.json' would resolve to the - * data source 'org.apache.spark.sql.json.DefaultSource' + * Implemented by objects that can produce a streaming `Source` for a specific format or system. * - * A new instance of this class will be instantiated each time a DDL call is made. - * - * The difference between a [[RelationProvider]] and a [[HadoopFsRelationProvider]] is - * that users need to provide a schema and a (possibly empty) list of partition columns when - * using a [[HadoopFsRelationProvider]]. A relation provider can inherits both [[RelationProvider]], - * and [[HadoopFsRelationProvider]] if it can support schema inference, user-specified - * schemas, and accessing partitioned relations. - * - * @since 1.4.0 + * @since 2.0.0 */ @Experimental -trait HadoopFsRelationProvider { +@InterfaceStability.Unstable +trait StreamSourceProvider { + /** - * Returns a new base relation with the given parameters, a user defined schema, and a list of - * partition columns. Note: the parameters' keywords are case insensitive and this insensitivity - * is enforced by the Map that is passed to the function. - * - * @param dataSchema Schema of data columns (i.e., columns that are not partition columns). + * Returns the name and schema of the source that can be used to continually read data. + * @since 2.0.0 */ - def createRelation( + def sourceSchema( sqlContext: SQLContext, - paths: Array[String], - dataSchema: Option[StructType], - partitionColumns: Option[StructType], - parameters: Map[String, String]): HadoopFsRelation + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): (String, StructType) + + /** + * @since 2.0.0 + */ + def createSource( + sqlContext: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source +} + +/** + * ::Experimental:: + * Implemented by objects that can produce a streaming `Sink` for a specific format or system. + * + * @since 2.0.0 + */ +@Experimental +@InterfaceStability.Unstable +trait StreamSinkProvider { + def createSink( + sqlContext: SQLContext, + parameters: Map[String, String], + partitionColumns: Seq[String], + outputMode: OutputMode): Sink } /** * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable 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. - * - * @since 1.3.0 - */ + * Saves a DataFrame to a destination (using data source-specific parameters) + * + * @param sqlContext SQLContext + * @param mode specifies what happens when the destination already exists + * @param parameters data source-specific parameters + * @param data DataFrame to save (i.e. the rows after executing the query) + * @return Relation with a known schema + * + * @since 1.3.0 + */ def createRelation( sqlContext: SQLContext, mode: SaveMode, @@ -189,9 +181,8 @@ trait CreatableRelationProvider { } /** - * ::DeveloperApi:: * Represents a collection of tuples with a known schema. Classes that extend BaseRelation must - * be able to produce the schema of their data in the form of a [[StructType]]. Concrete + * be able to produce the schema of their data in the form of a `StructType`. Concrete * implementation should inherit from one of the descendant `Scan` classes, which define various * abstract methods for execution. * @@ -201,7 +192,7 @@ trait CreatableRelationProvider { * * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable abstract class BaseRelation { def sqlContext: SQLContext def schema: StructType @@ -213,7 +204,7 @@ abstract class BaseRelation { * large to broadcast. This method will be called multiple times during query planning * and thus should not perform expensive operations for each invocation. * - * Note that it is always better to overestimate size than underestimate, because underestimation + * @note It is always better to overestimate size than underestimate, because underestimation * could lead to execution plans that are suboptimal (i.e. broadcasting a very large table). * * @since 1.3.0 @@ -222,12 +213,12 @@ abstract class BaseRelation { /** * Whether does it need to convert the objects in Row to internal representation, for example: - * java.lang.String -> UTF8String - * java.lang.Decimal -> Decimal + * java.lang.String to UTF8String + * java.lang.Decimal to Decimal * - * If `needConversion` is `false`, buildScan() should return an [[RDD]] of [[InternalRow]] + * If `needConversion` is `false`, buildScan() should return an `RDD` of `InternalRow` * - * Note: The internal representation is not stable across releases and thus data sources outside + * @note The internal representation is not stable across releases and thus data sources outside * of Spark SQL should leave this as true. * * @since 1.4.0 @@ -235,9 +226,11 @@ abstract class BaseRelation { def needConversion: Boolean = true /** - * Given an array of [[Filter]]s, returns an array of [[Filter]]s that this data source relation - * cannot handle. Spark SQL will apply all returned [[Filter]]s against rows returned by this - * data source relation. + * Returns the list of [[Filter]]s that this datasource may not be able to handle. + * These returned [[Filter]]s will be evaluated by Spark SQL after data is output by a scan. + * By default, this function will return all filters, as it is always safe to + * double evaluate a [[Filter]]. However, specific implementations can override this function to + * avoid double filtering when they are capable of processing a filter internally. * * @since 1.6.0 */ @@ -245,30 +238,27 @@ abstract class BaseRelation { } /** - * ::DeveloperApi:: * A BaseRelation that can produce all of its tuples as an RDD of Row objects. * * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable trait TableScan { def buildScan(): RDD[Row] } /** - * ::DeveloperApi:: * A BaseRelation that can eliminate unneeded columns before producing an RDD * containing all of its tuples as Row objects. * * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable trait PrunedScan { def buildScan(requiredColumns: Array[String]): RDD[Row] } /** - * ::DeveloperApi:: * A BaseRelation that can eliminate unneeded columns and filter using selected * predicates before producing an RDD containing all matching tuples as Row objects. * @@ -281,13 +271,12 @@ trait PrunedScan { * * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable trait PrunedFilteredScan { def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] } /** - * ::DeveloperApi:: * A BaseRelation that can be used to insert data into it through the insert method. * If overwrite in insert method is true, the old data in the relation should be overwritten with * the new data. If overwrite in insert method is false, the new data should be appended. @@ -304,7 +293,7 @@ trait PrunedFilteredScan { * * @since 1.3.0 */ -@DeveloperApi +@InterfaceStability.Stable trait InsertableRelation { def insert(data: DataFrame, overwrite: Boolean): Unit } @@ -313,550 +302,14 @@ trait InsertableRelation { * ::Experimental:: * An interface for experimenting with a more direct connection to the query planner. Compared to * [[PrunedFilteredScan]], this operator receives the raw expressions from the - * [[org.apache.spark.sql.catalyst.plans.logical.LogicalPlan]]. Unlike the other APIs this + * `org.apache.spark.sql.catalyst.plans.logical.LogicalPlan`. Unlike the other APIs this * interface is NOT designed to be binary compatible across releases and thus should only be used * for experimentation. * * @since 1.3.0 */ @Experimental +@InterfaceStability.Unstable trait CatalystScan { def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] } - -/** - * ::Experimental:: - * A factory that produces [[OutputWriter]]s. A new [[OutputWriterFactory]] is created on driver - * side for each write job issued when writing to a [[HadoopFsRelation]], and then gets serialized - * to executor side to create actual [[OutputWriter]]s on the fly. - * - * @since 1.4.0 - */ -@Experimental -abstract class OutputWriterFactory extends Serializable { - /** - * When writing to a [[HadoopFsRelation]], this method gets called by each task on executor side - * to instantiate new [[OutputWriter]]s. - * - * @param path Path of the file to which this [[OutputWriter]] is supposed to write. Note that - * this may not point to the final output file. For example, `FileOutputFormat` writes to - * temporary directories and then merge written files back to the final destination. In - * this case, `path` points to a temporary output file under the temporary directory. - * @param dataSchema Schema of the rows to be written. Partition columns are not included in the - * schema if the relation being written is partitioned. - * @param context The Hadoop MapReduce task context. - * - * @since 1.4.0 - */ - def newInstance(path: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter -} - -/** - * ::Experimental:: - * [[OutputWriter]] is used together with [[HadoopFsRelation]] for persisting rows to the - * underlying file system. Subclasses of [[OutputWriter]] must provide a zero-argument constructor. - * An [[OutputWriter]] instance is created and initialized when a new output file is opened on - * executor side. This instance is used to persist rows to this single output file. - * - * @since 1.4.0 - */ -@Experimental -abstract class OutputWriter { - /** - * Persists a single row. Invoked on the executor side. When writing to dynamically partitioned - * tables, dynamic partition columns are not included in rows to be written. - * - * @since 1.4.0 - */ - def write(row: Row): Unit - - /** - * Closes the [[OutputWriter]]. Invoked on the executor side after all rows are persisted, before - * the task output is committed. - * - * @since 1.4.0 - */ - def close(): Unit - - private var converter: InternalRow => Row = _ - - protected[sql] def initConverter(dataSchema: StructType) = { - converter = - CatalystTypeConverters.createToScalaConverter(dataSchema).asInstanceOf[InternalRow => Row] - } - - protected[sql] def writeInternal(row: InternalRow): Unit = { - write(converter(row)) - } -} - -/** - * ::Experimental:: - * A [[BaseRelation]] that provides much of the common code required for relations that store their - * data to an HDFS compatible filesystem. - * - * For the read path, similar to [[PrunedFilteredScan]], it can eliminate unneeded columns and - * filter using selected predicates before producing an RDD containing all matching tuples as - * [[Row]] objects. In addition, when reading from Hive style partitioned tables stored in file - * systems, it's able to discover partitioning information from the paths of input directories, and - * perform partition pruning before start reading the data. Subclasses of [[HadoopFsRelation()]] - * must override one of the three `buildScan` methods to implement the read path. - * - * For the write path, it provides the ability to write to both non-partitioned and partitioned - * tables. Directory layout of the partitioned tables is compatible with Hive. - * - * @constructor This constructor is for internal uses only. The [[PartitionSpec]] argument is for - * implementing metastore table conversion. - * - * @param maybePartitionSpec An [[HadoopFsRelation]] can be created with an optional - * [[PartitionSpec]], so that partition discovery can be skipped. - * - * @since 1.4.0 - */ -@Experimental -abstract class HadoopFsRelation private[sql](maybePartitionSpec: Option[PartitionSpec]) - extends BaseRelation with FileRelation with Logging { - - override def toString: String = getClass.getSimpleName + paths.mkString("[", ",", "]") - - def this() = this(None) - - private val hadoopConf = new Configuration(sqlContext.sparkContext.hadoopConfiguration) - - private val codegenEnabled = sqlContext.conf.codegenEnabled - - private var _partitionSpec: PartitionSpec = _ - - private class FileStatusCache { - var leafFiles = mutable.Map.empty[Path, FileStatus] - - var leafDirToChildrenFiles = mutable.Map.empty[Path, Array[FileStatus]] - - private def listLeafFiles(paths: Array[String]): Set[FileStatus] = { - if (paths.length >= sqlContext.conf.parallelPartitionDiscoveryThreshold) { - HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sqlContext.sparkContext) - } else { - val statuses = paths.flatMap { path => - val hdfsPath = new Path(path) - val fs = hdfsPath.getFileSystem(hadoopConf) - val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) - - logInfo(s"Listing $qualified on driver") - Try(fs.listStatus(qualified)).getOrElse(Array.empty) - }.filterNot { status => - val name = status.getPath.getName - name.toLowerCase == "_temporary" || name.startsWith(".") - } - - val (dirs, files) = statuses.partition(_.isDir) - - if (dirs.isEmpty) { - files.toSet - } else { - files.toSet ++ listLeafFiles(dirs.map(_.getPath.toString)) - } - } - } - - def refresh(): Unit = { - val files = listLeafFiles(paths) - - leafFiles.clear() - leafDirToChildrenFiles.clear() - - leafFiles ++= files.map(f => f.getPath -> f).toMap - leafDirToChildrenFiles ++= files.toArray.groupBy(_.getPath.getParent) - } - } - - private lazy val fileStatusCache = { - val cache = new FileStatusCache - cache.refresh() - cache - } - - protected def cachedLeafStatuses(): Set[FileStatus] = { - fileStatusCache.leafFiles.values.toSet - } - - final private[sql] def partitionSpec: PartitionSpec = { - if (_partitionSpec == null) { - _partitionSpec = maybePartitionSpec - .flatMap { - case spec if spec.partitions.nonEmpty => - Some(spec.copy(partitionColumns = spec.partitionColumns.asNullable)) - case _ => - None - } - .orElse { - // We only know the partition columns and their data types. We need to discover - // partition values. - userDefinedPartitionColumns.map { partitionSchema => - val spec = discoverPartitions() - val partitionColumnTypes = spec.partitionColumns.map(_.dataType) - val castedPartitions = spec.partitions.map { case p @ Partition(values, path) => - val literals = partitionColumnTypes.zipWithIndex.map { case (dt, i) => - Literal.create(values.get(i, dt), dt) - } - val castedValues = partitionSchema.zip(literals).map { case (field, literal) => - Cast(literal, field.dataType).eval() - } - p.copy(values = InternalRow.fromSeq(castedValues)) - } - PartitionSpec(partitionSchema, castedPartitions) - } - } - .getOrElse { - if (sqlContext.conf.partitionDiscoveryEnabled()) { - discoverPartitions() - } else { - PartitionSpec(StructType(Nil), Array.empty[Partition]) - } - } - } - _partitionSpec - } - - /** - * Base paths of this relation. For partitioned relations, it should be either root directories - * of all partition directories. - * - * @since 1.4.0 - */ - def paths: Array[String] - - override def inputFiles: Array[String] = cachedLeafStatuses().map(_.getPath.toString).toArray - - override def sizeInBytes: Long = cachedLeafStatuses().map(_.getLen).sum - - /** - * Partition columns. Can be either defined by [[userDefinedPartitionColumns]] or automatically - * discovered. Note that they should always be nullable. - * - * @since 1.4.0 - */ - final def partitionColumns: StructType = - userDefinedPartitionColumns.getOrElse(partitionSpec.partitionColumns) - - /** - * Optional user defined partition columns. - * - * @since 1.4.0 - */ - def userDefinedPartitionColumns: Option[StructType] = None - - private[sql] def refresh(): Unit = { - fileStatusCache.refresh() - if (sqlContext.conf.partitionDiscoveryEnabled()) { - _partitionSpec = discoverPartitions() - } - } - - private def discoverPartitions(): PartitionSpec = { - // We use leaf dirs containing data files to discover the schema. - val leafDirs = fileStatusCache.leafDirToChildrenFiles.keys.toSeq - userDefinedPartitionColumns match { - case Some(userProvidedSchema) if userProvidedSchema.nonEmpty => - val spec = PartitioningUtils.parsePartitions( - leafDirs, PartitioningUtils.DEFAULT_PARTITION_NAME, typeInference = false) - - // Without auto inference, all of value in the `row` should be null or in StringType, - // we need to cast into the data type that user specified. - def castPartitionValuesToUserSchema(row: InternalRow) = { - InternalRow((0 until row.numFields).map { i => - Cast( - Literal.create(row.getString(i), StringType), - userProvidedSchema.fields(i).dataType).eval() - }: _*) - } - - PartitionSpec(userProvidedSchema, spec.partitions.map { part => - part.copy(values = castPartitionValuesToUserSchema(part.values)) - }) - - case _ => - // user did not provide a partitioning schema - PartitioningUtils.parsePartitions(leafDirs, PartitioningUtils.DEFAULT_PARTITION_NAME, - typeInference = sqlContext.conf.partitionColumnTypeInferenceEnabled()) - } - } - - /** - * Schema of this relation. It consists of columns appearing in [[dataSchema]] and all partition - * columns not appearing in [[dataSchema]]. - * - * @since 1.4.0 - */ - override lazy val schema: StructType = { - val dataSchemaColumnNames = dataSchema.map(_.name.toLowerCase).toSet - StructType(dataSchema ++ partitionColumns.filterNot { column => - dataSchemaColumnNames.contains(column.name.toLowerCase) - }) - } - - final private[sql] def buildInternalScan( - requiredColumns: Array[String], - filters: Array[Filter], - inputPaths: Array[String], - broadcastedConf: Broadcast[SerializableConfiguration]): RDD[InternalRow] = { - val inputStatuses = inputPaths.flatMap { input => - val path = new Path(input) - - // First assumes `input` is a directory path, and tries to get all files contained in it. - fileStatusCache.leafDirToChildrenFiles.getOrElse( - path, - // Otherwise, `input` might be a file path - fileStatusCache.leafFiles.get(path).toArray - ).filter { status => - val name = status.getPath.getName - !name.startsWith("_") && !name.startsWith(".") - } - } - - buildInternalScan(requiredColumns, filters, inputStatuses, broadcastedConf) - } - - /** - * Specifies schema of actual data files. For partitioned relations, if one or more partitioned - * columns are contained in the data files, they should also appear in `dataSchema`. - * - * @since 1.4.0 - */ - def dataSchema: StructType - - /** - * For a non-partitioned relation, this method builds an `RDD[Row]` containing all rows within - * this relation. For partitioned relations, this method is called for each selected partition, - * and builds an `RDD[Row]` containing all rows within that single partition. - * - * @param inputFiles For a non-partitioned relation, it contains paths of all data files in the - * relation. For a partitioned relation, it contains paths of all data files in a single - * selected partition. - * - * @since 1.4.0 - */ - def buildScan(inputFiles: Array[FileStatus]): RDD[Row] = { - throw new UnsupportedOperationException( - "At least one buildScan() method should be overridden to read the relation.") - } - - /** - * For a non-partitioned relation, this method builds an `RDD[Row]` containing all rows within - * this relation. For partitioned relations, this method is called for each selected partition, - * and builds an `RDD[Row]` containing all rows within that single partition. - * - * @param requiredColumns Required columns. - * @param inputFiles For a non-partitioned relation, it contains paths of all data files in the - * relation. For a partitioned relation, it contains paths of all data files in a single - * selected partition. - * - * @since 1.4.0 - */ - // TODO Tries to eliminate the extra Catalyst-to-Scala conversion when `needConversion` is true - // - // PR #7626 separated `Row` and `InternalRow` completely. One of the consequences is that we can - // no longer treat an `InternalRow` containing Catalyst values as a `Row`. Thus we have to - // introduce another row value conversion for data sources whose `needConversion` is true. - def buildScan(requiredColumns: Array[String], inputFiles: Array[FileStatus]): RDD[Row] = { - // Yeah, to workaround serialization... - val dataSchema = this.dataSchema - val codegenEnabled = this.codegenEnabled - val needConversion = this.needConversion - - val requiredOutput = requiredColumns.map { col => - val field = dataSchema(col) - BoundReference(dataSchema.fieldIndex(col), field.dataType, field.nullable) - }.toSeq - - val rdd: RDD[Row] = buildScan(inputFiles) - val converted: RDD[InternalRow] = - if (needConversion) { - RDDConversions.rowToRowRdd(rdd, dataSchema.fields.map(_.dataType)) - } else { - rdd.asInstanceOf[RDD[InternalRow]] - } - - converted.mapPartitions { rows => - val buildProjection = if (codegenEnabled) { - GenerateMutableProjection.generate(requiredOutput, dataSchema.toAttributes) - } else { - () => new InterpretedMutableProjection(requiredOutput, dataSchema.toAttributes) - } - - val projectedRows = { - val mutableProjection = buildProjection() - rows.map(r => mutableProjection(r)) - } - - if (needConversion) { - val requiredSchema = StructType(requiredColumns.map(dataSchema(_))) - val toScala = CatalystTypeConverters.createToScalaConverter(requiredSchema) - projectedRows.map(toScala(_).asInstanceOf[Row]) - } else { - projectedRows - } - }.asInstanceOf[RDD[Row]] - } - - /** - * For a non-partitioned relation, this method builds an `RDD[Row]` containing all rows within - * this relation. For partitioned relations, this method is called for each selected partition, - * and builds an `RDD[Row]` containing all rows within that single partition. - * - * @param requiredColumns Required columns. - * @param filters Candidate filters to be pushed down. The actual filter should be the conjunction - * of all `filters`. The pushed down filters are currently purely an optimization as they - * will all be evaluated again. This means it is safe to use them with methods that produce - * false positives such as filtering partitions based on a bloom filter. - * @param inputFiles For a non-partitioned relation, it contains paths of all data files in the - * relation. For a partitioned relation, it contains paths of all data files in a single - * selected partition. - * - * @since 1.4.0 - */ - def buildScan( - requiredColumns: Array[String], - filters: Array[Filter], - inputFiles: Array[FileStatus]): RDD[Row] = { - buildScan(requiredColumns, inputFiles) - } - - /** - * For a non-partitioned relation, this method builds an `RDD[Row]` containing all rows within - * this relation. For partitioned relations, this method is called for each selected partition, - * and builds an `RDD[Row]` containing all rows within that single partition. - * - * Note: This interface is subject to change in future. - * - * @param requiredColumns Required columns. - * @param filters Candidate filters to be pushed down. The actual filter should be the conjunction - * of all `filters`. The pushed down filters are currently purely an optimization as they - * will all be evaluated again. This means it is safe to use them with methods that produce - * false positives such as filtering partitions based on a bloom filter. - * @param inputFiles For a non-partitioned relation, it contains paths of all data files in the - * relation. For a partitioned relation, it contains paths of all data files in a single - * selected partition. - * @param broadcastedConf A shared broadcast Hadoop Configuration, which can be used to reduce the - * overhead of broadcasting the Configuration for every Hadoop RDD. - * - * @since 1.4.0 - */ - private[sql] def buildScan( - requiredColumns: Array[String], - filters: Array[Filter], - inputFiles: Array[FileStatus], - broadcastedConf: Broadcast[SerializableConfiguration]): RDD[Row] = { - buildScan(requiredColumns, filters, inputFiles) - } - - /** - * For a non-partitioned relation, this method builds an `RDD[InternalRow]` containing all rows - * within this relation. For partitioned relations, this method is called for each selected - * partition, and builds an `RDD[InternalRow]` containing all rows within that single partition. - * - * Note: - * - * 1. Rows contained in the returned `RDD[InternalRow]` are assumed to be `UnsafeRow`s. - * 2. This interface is subject to change in future. - * - * @param requiredColumns Required columns. - * @param filters Candidate filters to be pushed down. The actual filter should be the conjunction - * of all `filters`. The pushed down filters are currently purely an optimization as they - * will all be evaluated again. This means it is safe to use them with methods that produce - * false positives such as filtering partitions based on a bloom filter. - * @param inputFiles For a non-partitioned relation, it contains paths of all data files in the - * relation. For a partitioned relation, it contains paths of all data files in a single - * selected partition. - * @param broadcastedConf A shared broadcast Hadoop Configuration, which can be used to reduce the - * overhead of broadcasting the Configuration for every Hadoop RDD. - */ - private[sql] def buildInternalScan( - requiredColumns: Array[String], - filters: Array[Filter], - inputFiles: Array[FileStatus], - broadcastedConf: Broadcast[SerializableConfiguration]): RDD[InternalRow] = { - val requiredSchema = StructType(requiredColumns.map(dataSchema.apply)) - val internalRows = { - val externalRows = buildScan(requiredColumns, filters, inputFiles, broadcastedConf) - execution.RDDConversions.rowToRowRdd(externalRows, requiredSchema.map(_.dataType)) - } - - internalRows.mapPartitions { iterator => - val unsafeProjection = UnsafeProjection.create(requiredSchema) - iterator.map(unsafeProjection) - } - } - - /** - * Prepares a write job and returns an [[OutputWriterFactory]]. Client side job preparation can - * be put here. For example, user defined output committer can be configured here - * by setting the output committer class in the conf of spark.sql.sources.outputCommitterClass. - * - * Note that the only side effect expected here is mutating `job` via its setters. Especially, - * Spark SQL caches [[BaseRelation]] instances for performance, mutating relation internal states - * may cause unexpected behaviors. - * - * @since 1.4.0 - */ - def prepareJobForWrite(job: Job): OutputWriterFactory -} - -private[sql] object HadoopFsRelation extends Logging { - // We don't filter files/directories whose name start with "_" except "_temporary" here, as - // specific data sources may take advantages over them (e.g. Parquet _metadata and - // _common_metadata files). "_temporary" directories are explicitly ignored since failed - // tasks/jobs may leave partial/corrupted data files there. Files and directories whose name - // start with "." are also ignored. - def listLeafFiles(fs: FileSystem, status: FileStatus): Array[FileStatus] = { - logInfo(s"Listing ${status.getPath}") - val name = status.getPath.getName.toLowerCase - if (name == "_temporary" || name.startsWith(".")) { - Array.empty - } else { - val (dirs, files) = fs.listStatus(status.getPath).partition(_.isDir) - files ++ dirs.flatMap(dir => listLeafFiles(fs, dir)) - } - } - - // `FileStatus` is Writable but not serializable. What make it worse, somehow it doesn't play - // well with `SerializableWritable`. So there seems to be no way to serialize a `FileStatus`. - // Here we use `FakeFileStatus` to extract key components of a `FileStatus` to serialize it from - // executor side and reconstruct it on driver side. - case class FakeFileStatus( - path: String, - length: Long, - isDir: Boolean, - blockReplication: Short, - blockSize: Long, - modificationTime: Long, - accessTime: Long) - - def listLeafFilesInParallel( - paths: Array[String], - hadoopConf: Configuration, - sparkContext: SparkContext): Set[FileStatus] = { - logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") - - val serializableConfiguration = new SerializableConfiguration(hadoopConf) - val fakeStatuses = sparkContext.parallelize(paths).flatMap { path => - val hdfsPath = new Path(path) - val fs = hdfsPath.getFileSystem(serializableConfiguration.value) - val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) - Try(listLeafFiles(fs, fs.getFileStatus(qualified))).getOrElse(Array.empty) - }.map { status => - FakeFileStatus( - status.getPath.toString, - status.getLen, - status.isDir, - status.getReplication, - status.getBlockSize, - status.getModificationTime, - status.getAccessTime) - }.collect() - - fakeStatuses.map { f => - new FileStatus( - f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)) - }.toSet - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala new file mode 100644 index 000000000000..a42e28053a96 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -0,0 +1,384 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.util.Locale + +import scala.collection.JavaConverters._ + +import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.execution.command.DDLUtils +import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.streaming.StreamingRelation +import org.apache.spark.sql.types.StructType + +/** + * Interface used to load a streaming `Dataset` from external storage systems (e.g. file systems, + * key-value stores, etc). Use `SparkSession.readStream` to access this. + * + * @since 2.0.0 + */ +@InterfaceStability.Evolving +final class DataStreamReader private[sql](sparkSession: SparkSession) extends Logging { + /** + * Specifies the input data source format. + * + * @since 2.0.0 + */ + def format(source: String): DataStreamReader = { + this.source = source + this + } + + /** + * Specifies the input schema. Some data sources (e.g. JSON) can infer the input schema + * automatically from data. By specifying the schema here, the underlying data source can + * skip the schema inference step, and thus speed up data loading. + * + * @since 2.0.0 + */ + def schema(schema: StructType): DataStreamReader = { + this.userSpecifiedSchema = Option(schema) + this + } + + /** + * Specifies the schema by using the input DDL-formatted string. Some data sources (e.g. JSON) can + * infer the input schema automatically from data. By specifying the schema here, the underlying + * data source can skip the schema inference step, and thus speed up data loading. + * + * @since 2.3.0 + */ + def schema(schemaString: String): DataStreamReader = { + this.userSpecifiedSchema = Option(StructType.fromDDL(schemaString)) + this + } + + /** + * Adds an input option for the underlying data source. + * + * You can set the following option(s): + *
      + *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone + * to be used to parse timestamps in the JSON/CSV datasources or partition values.
    • + *
    + * + * @since 2.0.0 + */ + def option(key: String, value: String): DataStreamReader = { + this.extraOptions += (key -> value) + this + } + + /** + * Adds an input option for the underlying data source. + * + * @since 2.0.0 + */ + def option(key: String, value: Boolean): DataStreamReader = option(key, value.toString) + + /** + * Adds an input option for the underlying data source. + * + * @since 2.0.0 + */ + def option(key: String, value: Long): DataStreamReader = option(key, value.toString) + + /** + * Adds an input option for the underlying data source. + * + * @since 2.0.0 + */ + def option(key: String, value: Double): DataStreamReader = option(key, value.toString) + + /** + * (Scala-specific) Adds input options for the underlying data source. + * + * You can set the following option(s): + *
      + *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone + * to be used to parse timestamps in the JSON/CSV datasources or partition values.
    • + *
    + * + * @since 2.0.0 + */ + def options(options: scala.collection.Map[String, String]): DataStreamReader = { + this.extraOptions ++= options + this + } + + /** + * Adds input options for the underlying data source. + * + * You can set the following option(s): + *
      + *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone + * to be used to parse timestamps in the JSON/CSV datasources or partition values.
    • + *
    + * + * @since 2.0.0 + */ + def options(options: java.util.Map[String, String]): DataStreamReader = { + this.options(options.asScala) + this + } + + + /** + * Loads input data stream in as a `DataFrame`, for data streams that don't require a path + * (e.g. external key-value stores). + * + * @since 2.0.0 + */ + def load(): DataFrame = { + if (source.toLowerCase(Locale.ROOT) == DDLUtils.HIVE_PROVIDER) { + throw new AnalysisException("Hive data source can only be used with tables, you can not " + + "read files of Hive data source directly.") + } + + val dataSource = + DataSource( + sparkSession, + userSpecifiedSchema = userSpecifiedSchema, + className = source, + options = extraOptions.toMap) + Dataset.ofRows(sparkSession, StreamingRelation(dataSource)) + } + + /** + * Loads input in as a `DataFrame`, for data streams that read from some path. + * + * @since 2.0.0 + */ + def load(path: String): DataFrame = { + option("path", path).load() + } + + /** + * Loads a JSON file stream and returns the results as a `DataFrame`. + * + * JSON Lines (newline-delimited JSON) is supported by + * default. For JSON (one record per file), set the `multiLine` option to true. + * + * This function goes through the input once to determine the input schema. If you know the + * schema in advance, use the version that specifies the schema to avoid the extra scan. + * + * You can set the following JSON-specific options to deal with non-standard JSON files: + *
      + *
    • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be + * considered in every trigger.
    • + *
    • `primitivesAsString` (default `false`): infers all primitive values as a string type
    • + *
    • `prefersDecimal` (default `false`): infers all floating-point values as a decimal + * type. If the values do not fit in decimal, then it infers them as doubles.
    • + *
    • `allowComments` (default `false`): ignores Java/C++ style comment in JSON records
    • + *
    • `allowUnquotedFieldNames` (default `false`): allows unquoted JSON field names
    • + *
    • `allowSingleQuotes` (default `true`): allows single quotes in addition to double quotes + *
    • + *
    • `allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers + * (e.g. 00012)
    • + *
    • `allowBackslashEscapingAnyCharacter` (default `false`): allows accepting quoting of all + * character using backslash quoting mechanism
    • + *
    • `allowUnquotedControlChars` (default `false`): allows JSON Strings to contain unquoted + * control characters (ASCII characters with value less than 32, including tab and line feed + * characters) or not.
    • + *
    • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records + * during parsing. + *
        + *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts + * the malformed string into a field configured by `columnNameOfCorruptRecord`. To keep + * corrupt records, an user can set a string type field named `columnNameOfCorruptRecord` + * in an user-defined schema. If a schema does not have the field, it drops corrupt records + * during parsing. When inferring a schema, it implicitly adds a `columnNameOfCorruptRecord` + * field in an output schema.
      • + *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • + *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • + *
      + *
    • + *
    • `columnNameOfCorruptRecord` (default is the value specified in + * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string + * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.
    • + *
    • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. + * Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to + * date type.
    • + *
    • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that + * indicates a timestamp format. Custom date formats follow the formats at + * `java.text.SimpleDateFormat`. This applies to timestamp type.
    • + *
    • `multiLine` (default `false`): parse one record, which may span multiple lines, + * per file
    • + *
    + * + * @since 2.0.0 + */ + def json(path: String): DataFrame = format("json").load(path) + + /** + * Loads a CSV file stream and returns the result as a `DataFrame`. + * + * This function will go through the input once to determine the input schema if `inferSchema` + * is enabled. To avoid going through the entire data once, disable `inferSchema` option or + * specify the schema explicitly using `schema`. + * + * You can set the following CSV-specific options to deal with CSV files: + *
      + *
    • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be + * considered in every trigger.
    • + *
    • `sep` (default `,`): sets the single character as a separator for each + * field and value.
    • + *
    • `encoding` (default `UTF-8`): decodes the CSV files by the given encoding + * type.
    • + *
    • `quote` (default `"`): sets the single character used for escaping quoted values where + * the separator can be part of the value. If you would like to turn off quotations, you need to + * set not `null` but an empty string. This behaviour is different form + * `com.databricks.spark.csv`.
    • + *
    • `escape` (default `\`): sets the single character used for escaping quotes inside + * an already quoted value.
    • + *
    • `comment` (default empty string): sets the single character used for skipping lines + * beginning with this character. By default, it is disabled.
    • + *
    • `header` (default `false`): uses the first line as names of columns.
    • + *
    • `inferSchema` (default `false`): infers the input schema automatically from data. It + * requires one extra pass over the data.
    • + *
    • `ignoreLeadingWhiteSpace` (default `false`): a flag indicating whether or not leading + * whitespaces from values being read should be skipped.
    • + *
    • `ignoreTrailingWhiteSpace` (default `false`): a flag indicating whether or not trailing + * whitespaces from values being read should be skipped.
    • + *
    • `nullValue` (default empty string): sets the string representation of a null value. Since + * 2.0.1, this applies to all supported types including the string type.
    • + *
    • `nanValue` (default `NaN`): sets the string representation of a non-number" value.
    • + *
    • `positiveInf` (default `Inf`): sets the string representation of a positive infinity + * value.
    • + *
    • `negativeInf` (default `-Inf`): sets the string representation of a negative infinity + * value.
    • + *
    • `dateFormat` (default `yyyy-MM-dd`): sets the string that indicates a date format. + * Custom date formats follow the formats at `java.text.SimpleDateFormat`. This applies to + * date type.
    • + *
    • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that + * indicates a timestamp format. Custom date formats follow the formats at + * `java.text.SimpleDateFormat`. This applies to timestamp type.
    • + *
    • `maxColumns` (default `20480`): defines a hard limit of how many columns + * a record can have.
    • + *
    • `maxCharsPerColumn` (default `-1`): defines the maximum number of characters allowed + * for any given value being read. By default, it is -1 meaning unlimited length
    • + *
    • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records + * during parsing. It supports the following case-insensitive modes. + *
        + *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts + * the malformed string into a field configured by `columnNameOfCorruptRecord`. To keep + * corrupt records, an user can set a string type field named `columnNameOfCorruptRecord` + * in an user-defined schema. If a schema does not have the field, it drops corrupt records + * during parsing. When a length of parsed CSV tokens is shorter than an expected length + * of a schema, it sets `null` for extra fields.
      • + *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • + *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • + *
      + *
    • + *
    • `columnNameOfCorruptRecord` (default is the value specified in + * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string + * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.
    • + *
    • `multiLine` (default `false`): parse one record, which may span multiple lines.
    • + *
    + * + * @since 2.0.0 + */ + def csv(path: String): DataFrame = format("csv").load(path) + + /** + * Loads a Parquet file stream, returning the result as a `DataFrame`. + * + * You can set the following Parquet-specific option(s) for reading Parquet files: + *
      + *
    • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be + * considered in every trigger.
    • + *
    • `mergeSchema` (default is the value specified in `spark.sql.parquet.mergeSchema`): sets + * whether we should merge schemas collected from all + * Parquet part-files. This will override + * `spark.sql.parquet.mergeSchema`.
    • + *
    + * + * @since 2.0.0 + */ + def parquet(path: String): DataFrame = { + format("parquet").load(path) + } + + /** + * Loads text files and returns a `DataFrame` whose schema starts with a string column named + * "value", and followed by partitioned columns if there are any. + * + * Each line in the text files is a new row in the resulting DataFrame. For example: + * {{{ + * // Scala: + * spark.readStream.text("/path/to/directory/") + * + * // Java: + * spark.readStream().text("/path/to/directory/") + * }}} + * + * You can set the following text-specific options to deal with text files: + *
      + *
    • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be + * considered in every trigger.
    • + *
    + * + * @since 2.0.0 + */ + def text(path: String): DataFrame = format("text").load(path) + + /** + * Loads text file(s) and returns a `Dataset` of String. The underlying schema of the Dataset + * contains a single string column named "value". + * + * If the directory structure of the text files contains partitioning information, those are + * ignored in the resulting Dataset. To include partitioning information as columns, use `text`. + * + * Each line in the text file is a new element in the resulting Dataset. For example: + * {{{ + * // Scala: + * spark.readStream.textFile("/path/to/spark/README.md") + * + * // Java: + * spark.readStream().textFile("/path/to/spark/README.md") + * }}} + * + * You can set the following text-specific options to deal with text files: + *
      + *
    • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be + * considered in every trigger.
    • + *
    + * + * @param path input path + * @since 2.1.0 + */ + def textFile(path: String): Dataset[String] = { + if (userSpecifiedSchema.nonEmpty) { + throw new AnalysisException("User specified schema not supported with `textFile`") + } + text(path).select("value").as[String](sparkSession.implicits.newStringEncoder) + } + + /////////////////////////////////////////////////////////////////////////////////////// + // Builder pattern config options + /////////////////////////////////////////////////////////////////////////////////////// + + private var source: String = sparkSession.sessionState.conf.defaultDataSourceName + + private var userSpecifiedSchema: Option[StructType] = None + + private var extraOptions = new scala.collection.mutable.HashMap[String, String] +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala new file mode 100644 index 000000000000..14e7df672cc5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -0,0 +1,387 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.util.Locale + +import scala.collection.JavaConverters._ + +import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.sql.{AnalysisException, Dataset, ForeachWriter} +import org.apache.spark.sql.catalyst.streaming.InternalOutputModes +import org.apache.spark.sql.execution.command.DDLUtils +import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.streaming.{ForeachSink, MemoryPlan, MemorySink} + +/** + * Interface used to write a streaming `Dataset` to external storage systems (e.g. file systems, + * key-value stores, etc). Use `Dataset.writeStream` to access this. + * + * @since 2.0.0 + */ +@InterfaceStability.Evolving +final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { + + private val df = ds.toDF() + + /** + * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. + * - `OutputMode.Append()`: only the new rows in the streaming DataFrame/Dataset will be + * written to the sink + * - `OutputMode.Complete()`: all the rows in the streaming DataFrame/Dataset will be written + * to the sink every time these is some updates + * - `OutputMode.Update()`: only the rows that were updated in the streaming DataFrame/Dataset + * will be written to the sink every time there are some updates. If + * the query doesn't contain aggregations, it will be equivalent to + * `OutputMode.Append()` mode. + * + * @since 2.0.0 + */ + def outputMode(outputMode: OutputMode): DataStreamWriter[T] = { + this.outputMode = outputMode + this + } + + /** + * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. + * - `append`: only the new rows in the streaming DataFrame/Dataset will be written to + * the sink + * - `complete`: all the rows in the streaming DataFrame/Dataset will be written to the sink + * every time these is some updates + * - `update`: only the rows that were updated in the streaming DataFrame/Dataset will + * be written to the sink every time there are some updates. If the query doesn't + * contain aggregations, it will be equivalent to `append` mode. + * @since 2.0.0 + */ + def outputMode(outputMode: String): DataStreamWriter[T] = { + this.outputMode = InternalOutputModes(outputMode) + this + } + + /** + * Set the trigger for the stream query. The default value is `ProcessingTime(0)` and it will run + * the query as fast as possible. + * + * Scala Example: + * {{{ + * df.writeStream.trigger(ProcessingTime("10 seconds")) + * + * import scala.concurrent.duration._ + * df.writeStream.trigger(ProcessingTime(10.seconds)) + * }}} + * + * Java Example: + * {{{ + * df.writeStream().trigger(ProcessingTime.create("10 seconds")) + * + * import java.util.concurrent.TimeUnit + * df.writeStream().trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) + * }}} + * + * @since 2.0.0 + */ + def trigger(trigger: Trigger): DataStreamWriter[T] = { + this.trigger = trigger + this + } + + /** + * Specifies the name of the [[StreamingQuery]] that can be started with `start()`. + * This name must be unique among all the currently active queries in the associated SQLContext. + * + * @since 2.0.0 + */ + def queryName(queryName: String): DataStreamWriter[T] = { + this.extraOptions += ("queryName" -> queryName) + this + } + + /** + * Specifies the underlying output data source. + * + * @since 2.0.0 + */ + def format(source: String): DataStreamWriter[T] = { + this.source = source + this + } + + /** + * Partitions the output by the given columns on the file system. If specified, the output is + * laid out on the file system similar to Hive's partitioning scheme. As an example, when we + * partition a dataset by year and then month, the directory layout would look like: + * + * - year=2016/month=01/ + * - year=2016/month=02/ + * + * Partitioning is one of the most widely used techniques to optimize physical data layout. + * It provides a coarse-grained index for skipping unnecessary data reads when queries have + * predicates on the partitioned columns. In order for partitioning to work well, the number + * of distinct values in each column should typically be less than tens of thousands. + * + * @since 2.0.0 + */ + @scala.annotation.varargs + def partitionBy(colNames: String*): DataStreamWriter[T] = { + this.partitioningColumns = Option(colNames) + this + } + + /** + * Adds an output option for the underlying data source. + * + * You can set the following option(s): + *
      + *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone + * to be used to format timestamps in the JSON/CSV datasources or partition values.
    • + *
    + * + * @since 2.0.0 + */ + def option(key: String, value: String): DataStreamWriter[T] = { + this.extraOptions += (key -> value) + this + } + + /** + * Adds an output option for the underlying data source. + * + * @since 2.0.0 + */ + def option(key: String, value: Boolean): DataStreamWriter[T] = option(key, value.toString) + + /** + * Adds an output option for the underlying data source. + * + * @since 2.0.0 + */ + def option(key: String, value: Long): DataStreamWriter[T] = option(key, value.toString) + + /** + * Adds an output option for the underlying data source. + * + * @since 2.0.0 + */ + def option(key: String, value: Double): DataStreamWriter[T] = option(key, value.toString) + + /** + * (Scala-specific) Adds output options for the underlying data source. + * + * You can set the following option(s): + *
      + *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone + * to be used to format timestamps in the JSON/CSV datasources or partition values.
    • + *
    + * + * @since 2.0.0 + */ + def options(options: scala.collection.Map[String, String]): DataStreamWriter[T] = { + this.extraOptions ++= options + this + } + + /** + * Adds output options for the underlying data source. + * + * You can set the following option(s): + *
      + *
    • `timeZone` (default session local timezone): sets the string that indicates a timezone + * to be used to format timestamps in the JSON/CSV datasources or partition values.
    • + *
    + * + * @since 2.0.0 + */ + def options(options: java.util.Map[String, String]): DataStreamWriter[T] = { + this.options(options.asScala) + this + } + + /** + * Starts the execution of the streaming query, which will continually output results to the given + * path as new data arrives. The returned [[StreamingQuery]] object can be used to interact with + * the stream. + * + * @since 2.0.0 + */ + def start(path: String): StreamingQuery = { + option("path", path).start() + } + + /** + * Starts the execution of the streaming query, which will continually output results to the given + * path as new data arrives. The returned [[StreamingQuery]] object can be used to interact with + * the stream. + * + * @since 2.0.0 + */ + def start(): StreamingQuery = { + if (source.toLowerCase(Locale.ROOT) == DDLUtils.HIVE_PROVIDER) { + throw new AnalysisException("Hive data source can only be used with tables, you can not " + + "write files of Hive data source directly.") + } + + if (source == "memory") { + assertNotPartitioned("memory") + if (extraOptions.get("queryName").isEmpty) { + throw new AnalysisException("queryName must be specified for memory sink") + } + val sink = new MemorySink(df.schema, outputMode) + val resultDf = Dataset.ofRows(df.sparkSession, new MemoryPlan(sink)) + val chkpointLoc = extraOptions.get("checkpointLocation") + val recoverFromChkpoint = outputMode == OutputMode.Complete() + val query = df.sparkSession.sessionState.streamingQueryManager.startQuery( + extraOptions.get("queryName"), + chkpointLoc, + df, + sink, + outputMode, + useTempCheckpointLocation = true, + recoverFromCheckpointLocation = recoverFromChkpoint, + trigger = trigger) + resultDf.createOrReplaceTempView(query.name) + query + } else if (source == "foreach") { + assertNotPartitioned("foreach") + val sink = new ForeachSink[T](foreachWriter)(ds.exprEnc) + df.sparkSession.sessionState.streamingQueryManager.startQuery( + extraOptions.get("queryName"), + extraOptions.get("checkpointLocation"), + df, + sink, + outputMode, + useTempCheckpointLocation = true, + trigger = trigger) + } else { + val (useTempCheckpointLocation, recoverFromCheckpointLocation) = + if (source == "console") { + (true, false) + } else { + (false, true) + } + val dataSource = + DataSource( + df.sparkSession, + className = source, + options = extraOptions.toMap, + partitionColumns = normalizedParCols.getOrElse(Nil)) + df.sparkSession.sessionState.streamingQueryManager.startQuery( + extraOptions.get("queryName"), + extraOptions.get("checkpointLocation"), + df, + dataSource.createSink(outputMode), + outputMode, + useTempCheckpointLocation = useTempCheckpointLocation, + recoverFromCheckpointLocation = recoverFromCheckpointLocation, + trigger = trigger) + } + } + + /** + * Starts the execution of the streaming query, which will continually send results to the given + * `ForeachWriter` as new data arrives. The `ForeachWriter` can be used to send the data + * generated by the `DataFrame`/`Dataset` to an external system. + * + * Scala example: + * {{{ + * datasetOfString.writeStream.foreach(new ForeachWriter[String] { + * + * def open(partitionId: Long, version: Long): Boolean = { + * // open connection + * } + * + * def process(record: String) = { + * // write string to connection + * } + * + * def close(errorOrNull: Throwable): Unit = { + * // close the connection + * } + * }).start() + * }}} + * + * Java example: + * {{{ + * datasetOfString.writeStream().foreach(new ForeachWriter() { + * + * @Override + * public boolean open(long partitionId, long version) { + * // open connection + * } + * + * @Override + * public void process(String value) { + * // write string to connection + * } + * + * @Override + * public void close(Throwable errorOrNull) { + * // close the connection + * } + * }).start(); + * }}} + * + * @since 2.0.0 + */ + def foreach(writer: ForeachWriter[T]): DataStreamWriter[T] = { + this.source = "foreach" + this.foreachWriter = if (writer != null) { + ds.sparkSession.sparkContext.clean(writer) + } else { + throw new IllegalArgumentException("foreach writer cannot be null") + } + this + } + + private def normalizedParCols: Option[Seq[String]] = partitioningColumns.map { cols => + cols.map(normalize(_, "Partition")) + } + + /** + * The given column name may not be equal to any of the existing column names if we were in + * case-insensitive context. Normalize the given column name to the real one so that we don't + * need to care about case sensitivity afterwards. + */ + private def normalize(columnName: String, columnType: String): String = { + val validColumnNames = df.logicalPlan.output.map(_.name) + validColumnNames.find(df.sparkSession.sessionState.analyzer.resolver(_, columnName)) + .getOrElse(throw new AnalysisException(s"$columnType column $columnName not found in " + + s"existing columns (${validColumnNames.mkString(", ")})")) + } + + private def assertNotPartitioned(operation: String): Unit = { + if (partitioningColumns.isDefined) { + throw new AnalysisException(s"'$operation' does not support partitioning") + } + } + + /////////////////////////////////////////////////////////////////////////////////////// + // Builder pattern config options + /////////////////////////////////////////////////////////////////////////////////////// + + private var source: String = df.sparkSession.sessionState.conf.defaultDataSourceName + + private var outputMode: OutputMode = OutputMode.Append + + private var trigger: Trigger = Trigger.ProcessingTime(0L) + + private var extraOptions = new scala.collection.mutable.HashMap[String, String] + + private var foreachWriter: ForeachWriter[T] = null + + private var partitioningColumns: Option[Seq[String]] = None +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala new file mode 100644 index 000000000000..04a956b70b02 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala @@ -0,0 +1,296 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming + +import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.sql.KeyValueGroupedDataset +import org.apache.spark.sql.catalyst.plans.logical.LogicalGroupState + +/** + * :: Experimental :: + * + * Wrapper class for interacting with per-group state data in `mapGroupsWithState` and + * `flatMapGroupsWithState` operations on `KeyValueGroupedDataset`. + * + * Detail description on `[map/flatMap]GroupsWithState` operation + * -------------------------------------------------------------- + * Both, `mapGroupsWithState` and `flatMapGroupsWithState` in `KeyValueGroupedDataset` + * will invoke the user-given function on each group (defined by the grouping function in + * `Dataset.groupByKey()`) while maintaining user-defined per-group state between invocations. + * For a static batch Dataset, the function will be invoked once per group. For a streaming + * Dataset, the function will be invoked for each group repeatedly in every trigger. + * That is, in every batch of the `StreamingQuery`, + * the function will be invoked once for each group that has data in the trigger. Furthermore, + * if timeout is set, then the function will invoked on timed out groups (more detail below). + * + * The function is invoked with following parameters. + * - The key of the group. + * - An iterator containing all the values for this group. + * - A user-defined state object set by previous invocations of the given function. + * + * In case of a batch Dataset, there is only one invocation and state object will be empty as + * there is no prior state. Essentially, for batch Datasets, `[map/flatMap]GroupsWithState` + * is equivalent to `[map/flatMap]Groups` and any updates to the state and/or timeouts have + * no effect. + * + * The major difference between `mapGroupsWithState` and `flatMapGroupsWithState` is that the + * former allows the function to return one and only one record, whereas the latter + * allows the function to return any number of records (including no records). Furthermore, the + * `flatMapGroupsWithState` is associated with an operation output mode, which can be either + * `Append` or `Update`. Semantically, this defines whether the output records of one trigger + * is effectively replacing the previously output records (from previous triggers) or is appending + * to the list of previously output records. Essentially, this defines how the Result Table (refer + * to the semantics in the programming guide) is updated, and allows us to reason about the + * semantics of later operations. + * + * Important points to note about the function (both mapGroupsWithState and flatMapGroupsWithState). + * - In a trigger, the function will be called only the groups present in the batch. So do not + * assume that the function will be called in every trigger for every group that has state. + * - There is no guaranteed ordering of values in the iterator in the function, neither with + * batch, nor with streaming Datasets. + * - All the data will be shuffled before applying the function. + * - If timeout is set, then the function will also be called with no values. + * See more details on `GroupStateTimeout` below. + * + * Important points to note about using `GroupState`. + * - The value of the state cannot be null. So updating state with null will throw + * `IllegalArgumentException`. + * - Operations on `GroupState` are not thread-safe. This is to avoid memory barriers. + * - If `remove()` is called, then `exists()` will return `false`, + * `get()` will throw `NoSuchElementException` and `getOption()` will return `None` + * - After that, if `update(newState)` is called, then `exists()` will again return `true`, + * `get()` and `getOption()`will return the updated value. + * + * Important points to note about using `GroupStateTimeout`. + * - The timeout type is a global param across all the groups (set as `timeout` param in + * `[map|flatMap]GroupsWithState`, but the exact timeout duration/timestamp is configurable per + * group by calling `setTimeout...()` in `GroupState`. + * - Timeouts can be either based on processing time (i.e. + * `GroupStateTimeout.ProcessingTimeTimeout`) or event time (i.e. + * `GroupStateTimeout.EventTimeTimeout`). + * - With `ProcessingTimeTimeout`, the timeout duration can be set by calling + * `GroupState.setTimeoutDuration`. The timeout will occur when the clock has advanced by the set + * duration. Guarantees provided by this timeout with a duration of D ms are as follows: + * - Timeout will never be occur before the clock time has advanced by D ms + * - Timeout will occur eventually when there is a trigger in the query + * (i.e. after D ms). So there is a no strict upper bound on when the timeout would occur. + * For example, the trigger interval of the query will affect when the timeout actually occurs. + * If there is no data in the stream (for any group) for a while, then their will not be + * any trigger and timeout function call will not occur until there is data. + * - Since the processing time timeout is based on the clock time, it is affected by the + * variations in the system clock (i.e. time zone changes, clock skew, etc.). + * - With `EventTimeTimeout`, the user also has to specify the the the event time watermark in + * the query using `Dataset.withWatermark()`. With this setting, data that is older than the + * watermark are filtered out. The timeout can be set for a group by setting a timeout timestamp + * using`GroupState.setTimeoutTimestamp()`, and the timeout would occur when the watermark + * advances beyond the set timestamp. You can control the timeout delay by two parameters - + * (i) watermark delay and an additional duration beyond the timestamp in the event (which + * is guaranteed to be newer than watermark due to the filtering). Guarantees provided by this + * timeout are as follows: + * - Timeout will never be occur before watermark has exceeded the set timeout. + * - Similar to processing time timeouts, there is a no strict upper bound on the delay when + * the timeout actually occurs. The watermark can advance only when there is data in the + * stream, and the event time of the data has actually advanced. + * - When the timeout occurs for a group, the function is called for that group with no values, and + * `GroupState.hasTimedOut()` set to true. + * - The timeout is reset every time the function is called on a group, that is, + * when the group has new data, or the group has timed out. So the user has to set the timeout + * duration every time the function is called, otherwise there will not be any timeout set. + * + * Scala example of using GroupState in `mapGroupsWithState`: + * {{{ + * // A mapping function that maintains an integer state for string keys and returns a string. + * // Additionally, it sets a timeout to remove the state if it has not received data for an hour. + * def mappingFunction(key: String, value: Iterator[Int], state: GroupState[Int]): String = { + * + * if (state.hasTimedOut) { // If called when timing out, remove the state + * state.remove() + * + * } else if (state.exists) { // If state exists, use it for processing + * val existingState = state.get // Get the existing state + * val shouldRemove = ... // Decide whether to remove the state + * if (shouldRemove) { + * state.remove() // Remove the state + * + * } else { + * val newState = ... + * state.update(newState) // Set the new state + * state.setTimeoutDuration("1 hour") // Set the timeout + * } + * + * } else { + * val initialState = ... + * state.update(initialState) // Set the initial state + * state.setTimeoutDuration("1 hour") // Set the timeout + * } + * ... + * // return something + * } + * + * dataset + * .groupByKey(...) + * .mapGroupsWithState(GroupStateTimeout.ProcessingTimeTimeout)(mappingFunction) + * }}} + * + * Java example of using `GroupState`: + * {{{ + * // A mapping function that maintains an integer state for string keys and returns a string. + * // Additionally, it sets a timeout to remove the state if it has not received data for an hour. + * MapGroupsWithStateFunction mappingFunction = + * new MapGroupsWithStateFunction() { + * + * @Override + * public String call(String key, Iterator value, GroupState state) { + * if (state.hasTimedOut()) { // If called when timing out, remove the state + * state.remove(); + * + * } else if (state.exists()) { // If state exists, use it for processing + * int existingState = state.get(); // Get the existing state + * boolean shouldRemove = ...; // Decide whether to remove the state + * if (shouldRemove) { + * state.remove(); // Remove the state + * + * } else { + * int newState = ...; + * state.update(newState); // Set the new state + * state.setTimeoutDuration("1 hour"); // Set the timeout + * } + * + * } else { + * int initialState = ...; // Set the initial state + * state.update(initialState); + * state.setTimeoutDuration("1 hour"); // Set the timeout + * } + * ... +* // return something + * } + * }; + * + * dataset + * .groupByKey(...) + * .mapGroupsWithState( + * mappingFunction, Encoders.INT, Encoders.STRING, GroupStateTimeout.ProcessingTimeTimeout); + * }}} + * + * @tparam S User-defined type of the state to be stored for each group. Must be encodable into + * Spark SQL types (see `Encoder` for more details). + * @since 2.2.0 + */ +@Experimental +@InterfaceStability.Evolving +trait GroupState[S] extends LogicalGroupState[S] { + + /** Whether state exists or not. */ + def exists: Boolean + + /** Get the state value if it exists, or throw NoSuchElementException. */ + @throws[NoSuchElementException]("when state does not exist") + def get: S + + /** Get the state value as a scala Option. */ + def getOption: Option[S] + + /** + * Update the value of the state. Note that `null` is not a valid value, and it throws + * IllegalArgumentException. + */ + @throws[IllegalArgumentException]("when updating with null") + def update(newState: S): Unit + + /** Remove this state. */ + def remove(): Unit + + /** + * Whether the function has been called because the key has timed out. + * @note This can return true only when timeouts are enabled in `[map/flatmap]GroupsWithStates`. + */ + def hasTimedOut: Boolean + + /** + * Set the timeout duration in ms for this key. + * + * @note ProcessingTimeTimeout must be enabled in `[map/flatmap]GroupsWithStates`. + */ + @throws[IllegalArgumentException]("if 'durationMs' is not positive") + @throws[IllegalStateException]("when state is either not initialized, or already removed") + @throws[UnsupportedOperationException]( + "if 'timeout' has not been enabled in [map|flatMap]GroupsWithState in a streaming query") + def setTimeoutDuration(durationMs: Long): Unit + + /** + * Set the timeout duration for this key as a string. For example, "1 hour", "2 days", etc. + * + * @note ProcessingTimeTimeout must be enabled in `[map/flatmap]GroupsWithStates`. + */ + @throws[IllegalArgumentException]("if 'duration' is not a valid duration") + @throws[IllegalStateException]("when state is either not initialized, or already removed") + @throws[UnsupportedOperationException]( + "if 'timeout' has not been enabled in [map|flatMap]GroupsWithState in a streaming query") + def setTimeoutDuration(duration: String): Unit + + @throws[IllegalArgumentException]("if 'timestampMs' is not positive") + @throws[IllegalStateException]("when state is either not initialized, or already removed") + @throws[UnsupportedOperationException]( + "if 'timeout' has not been enabled in [map|flatMap]GroupsWithState in a streaming query") + /** + * Set the timeout timestamp for this key as milliseconds in epoch time. + * This timestamp cannot be older than the current watermark. + * + * @note EventTimeTimeout must be enabled in `[map/flatmap]GroupsWithStates`. + */ + def setTimeoutTimestamp(timestampMs: Long): Unit + + @throws[IllegalArgumentException]("if 'additionalDuration' is invalid") + @throws[IllegalStateException]("when state is either not initialized, or already removed") + @throws[UnsupportedOperationException]( + "if 'timeout' has not been enabled in [map|flatMap]GroupsWithState in a streaming query") + /** + * Set the timeout timestamp for this key as milliseconds in epoch time and an additional + * duration as a string (e.g. "1 hour", "2 days", etc.). + * The final timestamp (including the additional duration) cannot be older than the + * current watermark. + * + * @note EventTimeTimeout must be enabled in `[map/flatmap]GroupsWithStates`. + */ + def setTimeoutTimestamp(timestampMs: Long, additionalDuration: String): Unit + + @throws[IllegalStateException]("when state is either not initialized, or already removed") + @throws[UnsupportedOperationException]( + "if 'timeout' has not been enabled in [map|flatMap]GroupsWithState in a streaming query") + /** + * Set the timeout timestamp for this key as a java.sql.Date. + * This timestamp cannot be older than the current watermark. + * + * @note EventTimeTimeout must be enabled in `[map/flatmap]GroupsWithStates`. + */ + def setTimeoutTimestamp(timestamp: java.sql.Date): Unit + + @throws[IllegalArgumentException]("if 'additionalDuration' is invalid") + @throws[IllegalStateException]("when state is either not initialized, or already removed") + @throws[UnsupportedOperationException]( + "if 'timeout' has not been enabled in [map|flatMap]GroupsWithState in a streaming query") + /** + * Set the timeout timestamp for this key as a java.sql.Date and an additional + * duration as a string (e.g. "1 hour", "2 days", etc.). + * The final timestamp (including the additional duration) cannot be older than the + * current watermark. + * + * @note EventTimeTimeout must be enabled in `[map/flatmap]GroupsWithStates`. + */ + def setTimeoutTimestamp(timestamp: java.sql.Date, additionalDuration: String): Unit +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala new file mode 100644 index 000000000000..a033575d3d38 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming + +import java.util.concurrent.TimeUnit + +import scala.concurrent.duration.Duration + +import org.apache.commons.lang3.StringUtils + +import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.unsafe.types.CalendarInterval + +/** + * A trigger that runs a query periodically based on the processing time. If `interval` is 0, + * the query will run as fast as possible. + * + * Scala Example: + * {{{ + * df.writeStream.trigger(ProcessingTime("10 seconds")) + * + * import scala.concurrent.duration._ + * df.writeStream.trigger(ProcessingTime(10.seconds)) + * }}} + * + * Java Example: + * {{{ + * df.writeStream.trigger(ProcessingTime.create("10 seconds")) + * + * import java.util.concurrent.TimeUnit + * df.writeStream.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) + * }}} + * + * @since 2.0.0 + */ +@InterfaceStability.Evolving +@deprecated("use Trigger.ProcessingTime(intervalMs)", "2.2.0") +case class ProcessingTime(intervalMs: Long) extends Trigger { + require(intervalMs >= 0, "the interval of trigger should not be negative") +} + +/** + * Used to create [[ProcessingTime]] triggers for [[StreamingQuery]]s. + * + * @since 2.0.0 + */ +@InterfaceStability.Evolving +@deprecated("use Trigger.ProcessingTime(intervalMs)", "2.2.0") +object ProcessingTime { + + /** + * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. + * + * Example: + * {{{ + * df.writeStream.trigger(ProcessingTime("10 seconds")) + * }}} + * + * @since 2.0.0 + * @deprecated use Trigger.ProcessingTime(interval) + */ + @deprecated("use Trigger.ProcessingTime(interval)", "2.2.0") + def apply(interval: String): ProcessingTime = { + if (StringUtils.isBlank(interval)) { + throw new IllegalArgumentException( + "interval cannot be null or blank.") + } + val cal = if (interval.startsWith("interval")) { + CalendarInterval.fromString(interval) + } else { + CalendarInterval.fromString("interval " + interval) + } + if (cal == null) { + throw new IllegalArgumentException(s"Invalid interval: $interval") + } + if (cal.months > 0) { + throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") + } + new ProcessingTime(cal.microseconds / 1000) + } + + /** + * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. + * + * Example: + * {{{ + * import scala.concurrent.duration._ + * df.writeStream.trigger(ProcessingTime(10.seconds)) + * }}} + * + * @since 2.0.0 + * @deprecated use Trigger.ProcessingTime(interval) + */ + @deprecated("use Trigger.ProcessingTime(interval)", "2.2.0") + def apply(interval: Duration): ProcessingTime = { + new ProcessingTime(interval.toMillis) + } + + /** + * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. + * + * Example: + * {{{ + * df.writeStream.trigger(ProcessingTime.create("10 seconds")) + * }}} + * + * @since 2.0.0 + * @deprecated use Trigger.ProcessingTime(interval) + */ + @deprecated("use Trigger.ProcessingTime(interval)", "2.2.0") + def create(interval: String): ProcessingTime = { + apply(interval) + } + + /** + * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. + * + * Example: + * {{{ + * import java.util.concurrent.TimeUnit + * df.writeStream.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) + * }}} + * + * @since 2.0.0 + * @deprecated use Trigger.ProcessingTime(interval, unit) + */ + @deprecated("use Trigger.ProcessingTime(interval, unit)", "2.2.0") + def create(interval: Long, unit: TimeUnit): ProcessingTime = { + new ProcessingTime(unit.toMillis(interval)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala new file mode 100644 index 000000000000..f2dfbe42260d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.util.UUID + +import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.sql.SparkSession + +/** + * A handle to a query that is executing continuously in the background as new data arrives. + * All these methods are thread-safe. + * @since 2.0.0 + */ +@InterfaceStability.Evolving +trait StreamingQuery { + + /** + * Returns the user-specified name of the query, or null if not specified. + * This name can be specified in the `org.apache.spark.sql.streaming.DataStreamWriter` + * as `dataframe.writeStream.queryName("query").start()`. + * This name, if set, must be unique across all active queries. + * + * @since 2.0.0 + */ + def name: String + + /** + * Returns the unique id of this query that persists across restarts from checkpoint data. + * That is, this id is generated when a query is started for the first time, and + * will be the same every time it is restarted from checkpoint data. Also see [[runId]]. + * + * @since 2.1.0 + */ + def id: UUID + + /** + * Returns the unique id of this run of the query. That is, every start/restart of a query will + * generated a unique runId. Therefore, every time a query is restarted from + * checkpoint, it will have the same [[id]] but different [[runId]]s. + */ + def runId: UUID + + /** + * Returns the `SparkSession` associated with `this`. + * + * @since 2.0.0 + */ + def sparkSession: SparkSession + + /** + * Returns `true` if this query is actively running. + * + * @since 2.0.0 + */ + def isActive: Boolean + + /** + * Returns the [[StreamingQueryException]] if the query was terminated by an exception. + * @since 2.0.0 + */ + def exception: Option[StreamingQueryException] + + /** + * Returns the current status of the query. + * + * @since 2.0.2 + */ + def status: StreamingQueryStatus + + /** + * Returns an array of the most recent [[StreamingQueryProgress]] updates for this query. + * The number of progress updates retained for each stream is configured by Spark session + * configuration `spark.sql.streaming.numRecentProgressUpdates`. + * + * @since 2.1.0 + */ + def recentProgress: Array[StreamingQueryProgress] + + /** + * Returns the most recent [[StreamingQueryProgress]] update of this streaming query. + * + * @since 2.1.0 + */ + def lastProgress: StreamingQueryProgress + + /** + * Waits for the termination of `this` query, either by `query.stop()` or by an exception. + * If the query has terminated with an exception, then the exception will be thrown. + * + * If the query has terminated, then all subsequent calls to this method will either return + * immediately (if the query was terminated by `stop()`), or throw the exception + * immediately (if the query has terminated with exception). + * + * @throws StreamingQueryException if the query has terminated with an exception. + * + * @since 2.0.0 + */ + @throws[StreamingQueryException] + def awaitTermination(): Unit + + /** + * Waits for the termination of `this` query, either by `query.stop()` or by an exception. + * If the query has terminated with an exception, then the exception will be thrown. + * Otherwise, it returns whether the query has terminated or not within the `timeoutMs` + * milliseconds. + * + * If the query has terminated, then all subsequent calls to this method will either return + * `true` immediately (if the query was terminated by `stop()`), or throw the exception + * immediately (if the query has terminated with exception). + * + * @throws StreamingQueryException if the query has terminated with an exception + * + * @since 2.0.0 + */ + @throws[StreamingQueryException] + def awaitTermination(timeoutMs: Long): Boolean + + /** + * Blocks until all available data in the source has been processed and committed to the sink. + * This method is intended for testing. Note that in the case of continually arriving data, this + * method may block forever. Additionally, this method is only guaranteed to block until data that + * has been synchronously appended data to a `org.apache.spark.sql.execution.streaming.Source` + * prior to invocation. (i.e. `getOffset` must immediately reflect the addition). + * @since 2.0.0 + */ + def processAllAvailable(): Unit + + /** + * Stops the execution of this query if it is running. This method blocks until the threads + * performing execution has stopped. + * @since 2.0.0 + */ + def stop(): Unit + + /** + * Prints the physical plan to the console for debugging purposes. + * @since 2.0.0 + */ + def explain(): Unit + + /** + * Prints the physical plan to the console for debugging purposes. + * + * @param extended whether to do extended explain or not + * @since 2.0.0 + */ + def explain(extended: Boolean): Unit +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala new file mode 100644 index 000000000000..03aeb14de502 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming + +import org.apache.spark.annotation.InterfaceStability + +/** + * Exception that stopped a [[StreamingQuery]]. Use `cause` get the actual exception + * that caused the failure. + * @param message Message of this exception + * @param cause Internal cause of this exception + * @param startOffset Starting offset in json of the range of data in which exception occurred + * @param endOffset Ending offset in json of the range of data in exception occurred + * @since 2.0.0 + */ +@InterfaceStability.Evolving +class StreamingQueryException private[sql]( + private val queryDebugString: String, + val message: String, + val cause: Throwable, + val startOffset: String, + val endOffset: String) + extends Exception(message, cause) { + + /** Time when the exception occurred */ + val time: Long = System.currentTimeMillis + + override def toString(): String = + s"""${classOf[StreamingQueryException].getName}: ${cause.getMessage} + |$queryDebugString""".stripMargin +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala new file mode 100644 index 000000000000..6aa82b89ede8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming + +import java.util.UUID + +import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.scheduler.SparkListenerEvent + +/** + * Interface for listening to events related to [[StreamingQuery StreamingQueries]]. + * @note The methods are not thread-safe as they may be called from different threads. + * + * @since 2.0.0 + */ +@InterfaceStability.Evolving +abstract class StreamingQueryListener { + + import StreamingQueryListener._ + + /** + * Called when a query is started. + * @note This is called synchronously with + * [[org.apache.spark.sql.streaming.DataStreamWriter `DataStreamWriter.start()`]], + * that is, `onQueryStart` will be called on all listeners before + * `DataStreamWriter.start()` returns the corresponding [[StreamingQuery]]. Please + * don't block this method as it will block your query. + * @since 2.0.0 + */ + def onQueryStarted(event: QueryStartedEvent): Unit + + /** + * Called when there is some status update (ingestion rate updated, etc.) + * + * @note This method is asynchronous. The status in [[StreamingQuery]] will always be + * latest no matter when this method is called. Therefore, the status of [[StreamingQuery]] + * may be changed before/when you process the event. E.g., you may find [[StreamingQuery]] + * is terminated when you are processing `QueryProgressEvent`. + * @since 2.0.0 + */ + def onQueryProgress(event: QueryProgressEvent): Unit + + /** + * Called when a query is stopped, with or without error. + * @since 2.0.0 + */ + def onQueryTerminated(event: QueryTerminatedEvent): Unit +} + + +/** + * Companion object of [[StreamingQueryListener]] that defines the listener events. + * @since 2.0.0 + */ +@InterfaceStability.Evolving +object StreamingQueryListener { + + /** + * Base type of [[StreamingQueryListener]] events + * @since 2.0.0 + */ + @InterfaceStability.Evolving + trait Event extends SparkListenerEvent + + /** + * Event representing the start of a query + * @param id An unique query id that persists across restarts. See `StreamingQuery.id()`. + * @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`. + * @param name User-specified name of the query, null if not specified. + * @since 2.1.0 + */ + @InterfaceStability.Evolving + class QueryStartedEvent private[sql]( + val id: UUID, + val runId: UUID, + val name: String) extends Event + + /** + * Event representing any progress updates in a query. + * @param progress The query progress updates. + * @since 2.1.0 + */ + @InterfaceStability.Evolving + class QueryProgressEvent private[sql](val progress: StreamingQueryProgress) extends Event + + /** + * Event representing that termination of a query. + * + * @param id An unique query id that persists across restarts. See `StreamingQuery.id()`. + * @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`. + * @param exception The exception message of the query if the query was terminated + * with an exception. Otherwise, it will be `None`. + * @since 2.1.0 + */ + @InterfaceStability.Evolving + class QueryTerminatedEvent private[sql]( + val id: UUID, + val runId: UUID, + val exception: Option[String]) extends Event +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala new file mode 100644 index 000000000000..48b0ea20e5da --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -0,0 +1,337 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.util.UUID +import javax.annotation.concurrent.GuardedBy + +import scala.collection.mutable + +import org.apache.hadoop.fs.Path + +import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} +import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.{Clock, SystemClock, Utils} + +/** + * A class to manage all the [[StreamingQuery]] active in a `SparkSession`. + * + * @since 2.0.0 + */ +@InterfaceStability.Evolving +class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Logging { + + private[sql] val stateStoreCoordinator = + StateStoreCoordinatorRef.forDriver(sparkSession.sparkContext.env) + private val listenerBus = new StreamingQueryListenerBus(sparkSession.sparkContext.listenerBus) + + @GuardedBy("activeQueriesLock") + private val activeQueries = new mutable.HashMap[UUID, StreamingQuery] + private val activeQueriesLock = new Object + private val awaitTerminationLock = new Object + + @GuardedBy("awaitTerminationLock") + private var lastTerminatedQuery: StreamingQuery = null + + /** + * Returns a list of active queries associated with this SQLContext + * + * @since 2.0.0 + */ + def active: Array[StreamingQuery] = activeQueriesLock.synchronized { + activeQueries.values.toArray + } + + /** + * Returns the query if there is an active query with the given id, or null. + * + * @since 2.1.0 + */ + def get(id: UUID): StreamingQuery = activeQueriesLock.synchronized { + activeQueries.get(id).orNull + } + + /** + * Returns the query if there is an active query with the given id, or null. + * + * @since 2.1.0 + */ + def get(id: String): StreamingQuery = get(UUID.fromString(id)) + + /** + * Wait until any of the queries on the associated SQLContext has terminated since the + * creation of the context, or since `resetTerminated()` was called. If any query was terminated + * with an exception, then the exception will be thrown. + * + * If a query has terminated, then subsequent calls to `awaitAnyTermination()` will either + * return immediately (if the query was terminated by `query.stop()`), + * or throw the exception immediately (if the query was terminated with exception). Use + * `resetTerminated()` to clear past terminations and wait for new terminations. + * + * In the case where multiple queries have terminated since `resetTermination()` was called, + * if any query has terminated with exception, then `awaitAnyTermination()` will + * throw any of the exception. For correctly documenting exceptions across multiple queries, + * users need to stop all of them after any of them terminates with exception, and then check the + * `query.exception()` for each query. + * + * @throws StreamingQueryException if any query has terminated with an exception + * + * @since 2.0.0 + */ + @throws[StreamingQueryException] + def awaitAnyTermination(): Unit = { + awaitTerminationLock.synchronized { + while (lastTerminatedQuery == null) { + awaitTerminationLock.wait(10) + } + if (lastTerminatedQuery != null && lastTerminatedQuery.exception.nonEmpty) { + throw lastTerminatedQuery.exception.get + } + } + } + + /** + * Wait until any of the queries on the associated SQLContext has terminated since the + * creation of the context, or since `resetTerminated()` was called. Returns whether any query + * has terminated or not (multiple may have terminated). If any query has terminated with an + * exception, then the exception will be thrown. + * + * If a query has terminated, then subsequent calls to `awaitAnyTermination()` will either + * return `true` immediately (if the query was terminated by `query.stop()`), + * or throw the exception immediately (if the query was terminated with exception). Use + * `resetTerminated()` to clear past terminations and wait for new terminations. + * + * In the case where multiple queries have terminated since `resetTermination()` was called, + * if any query has terminated with exception, then `awaitAnyTermination()` will + * throw any of the exception. For correctly documenting exceptions across multiple queries, + * users need to stop all of them after any of them terminates with exception, and then check the + * `query.exception()` for each query. + * + * @throws StreamingQueryException if any query has terminated with an exception + * + * @since 2.0.0 + */ + @throws[StreamingQueryException] + def awaitAnyTermination(timeoutMs: Long): Boolean = { + + val startTime = System.currentTimeMillis + def isTimedout = System.currentTimeMillis - startTime >= timeoutMs + + awaitTerminationLock.synchronized { + while (!isTimedout && lastTerminatedQuery == null) { + awaitTerminationLock.wait(10) + } + if (lastTerminatedQuery != null && lastTerminatedQuery.exception.nonEmpty) { + throw lastTerminatedQuery.exception.get + } + lastTerminatedQuery != null + } + } + + /** + * Forget about past terminated queries so that `awaitAnyTermination()` can be used again to + * wait for new terminations. + * + * @since 2.0.0 + */ + def resetTerminated(): Unit = { + awaitTerminationLock.synchronized { + lastTerminatedQuery = null + } + } + + /** + * Register a [[StreamingQueryListener]] to receive up-calls for life cycle events of + * [[StreamingQuery]]. + * + * @since 2.0.0 + */ + def addListener(listener: StreamingQueryListener): Unit = { + listenerBus.addListener(listener) + } + + /** + * Deregister a [[StreamingQueryListener]]. + * + * @since 2.0.0 + */ + def removeListener(listener: StreamingQueryListener): Unit = { + listenerBus.removeListener(listener) + } + + /** Post a listener event */ + private[sql] def postListenerEvent(event: StreamingQueryListener.Event): Unit = { + listenerBus.post(event) + } + + private def createQuery( + userSpecifiedName: Option[String], + userSpecifiedCheckpointLocation: Option[String], + df: DataFrame, + sink: Sink, + outputMode: OutputMode, + useTempCheckpointLocation: Boolean, + recoverFromCheckpointLocation: Boolean, + trigger: Trigger, + triggerClock: Clock): StreamingQueryWrapper = { + var deleteCheckpointOnStop = false + val checkpointLocation = userSpecifiedCheckpointLocation.map { userSpecified => + new Path(userSpecified).toUri.toString + }.orElse { + df.sparkSession.sessionState.conf.checkpointLocation.map { location => + new Path(location, userSpecifiedName.getOrElse(UUID.randomUUID().toString)).toUri.toString + } + }.getOrElse { + if (useTempCheckpointLocation) { + // Delete the temp checkpoint when a query is being stopped without errors. + deleteCheckpointOnStop = true + Utils.createTempDir(namePrefix = s"temporary").getCanonicalPath + } else { + throw new AnalysisException( + "checkpointLocation must be specified either " + + """through option("checkpointLocation", ...) or """ + + s"""SparkSession.conf.set("${SQLConf.CHECKPOINT_LOCATION.key}", ...)""") + } + } + + // If offsets have already been created, we trying to resume a query. + if (!recoverFromCheckpointLocation) { + val checkpointPath = new Path(checkpointLocation, "offsets") + val fs = checkpointPath.getFileSystem(df.sparkSession.sessionState.newHadoopConf()) + if (fs.exists(checkpointPath)) { + throw new AnalysisException( + s"This query does not support recovering from checkpoint location. " + + s"Delete $checkpointPath to start over.") + } + } + + val analyzedPlan = df.queryExecution.analyzed + df.queryExecution.assertAnalyzed() + + if (sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled) { + UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode) + } + + if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) { + logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} " + + "is not supported in streaming DataFrames/Datasets and will be disabled.") + } + + new StreamingQueryWrapper(new StreamExecution( + sparkSession, + userSpecifiedName.orNull, + checkpointLocation, + analyzedPlan, + sink, + trigger, + triggerClock, + outputMode, + deleteCheckpointOnStop)) + } + + /** + * Start a [[StreamingQuery]]. + * + * @param userSpecifiedName Query name optionally specified by the user. + * @param userSpecifiedCheckpointLocation Checkpoint location optionally specified by the user. + * @param df Streaming DataFrame. + * @param sink Sink to write the streaming outputs. + * @param outputMode Output mode for the sink. + * @param useTempCheckpointLocation Whether to use a temporary checkpoint location when the user + * has not specified one. If false, then error will be thrown. + * @param recoverFromCheckpointLocation Whether to recover query from the checkpoint location. + * If false and the checkpoint location exists, then error + * will be thrown. + * @param trigger [[Trigger]] for the query. + * @param triggerClock [[Clock]] to use for the triggering. + */ + private[sql] def startQuery( + userSpecifiedName: Option[String], + userSpecifiedCheckpointLocation: Option[String], + df: DataFrame, + sink: Sink, + outputMode: OutputMode, + useTempCheckpointLocation: Boolean = false, + recoverFromCheckpointLocation: Boolean = true, + trigger: Trigger = ProcessingTime(0), + triggerClock: Clock = new SystemClock()): StreamingQuery = { + val query = createQuery( + userSpecifiedName, + userSpecifiedCheckpointLocation, + df, + sink, + outputMode, + useTempCheckpointLocation, + recoverFromCheckpointLocation, + trigger, + triggerClock) + + activeQueriesLock.synchronized { + // Make sure no other query with same name is active + userSpecifiedName.foreach { name => + if (activeQueries.values.exists(_.name == name)) { + throw new IllegalArgumentException( + s"Cannot start query with name $name as a query with that name is already active") + } + } + + // Make sure no other query with same id is active + if (activeQueries.values.exists(_.id == query.id)) { + throw new IllegalStateException( + s"Cannot start query with id ${query.id} as another query with same id is " + + s"already active. Perhaps you are attempting to restart a query from checkpoint " + + s"that is already active.") + } + + activeQueries.put(query.id, query) + } + try { + // When starting a query, it will call `StreamingQueryListener.onQueryStarted` synchronously. + // As it's provided by the user and can run arbitrary codes, we must not hold any lock here. + // Otherwise, it's easy to cause dead-lock, or block too long if the user codes take a long + // time to finish. + query.streamingQuery.start() + } catch { + case e: Throwable => + activeQueriesLock.synchronized { + activeQueries -= query.id + } + throw e + } + query + } + + /** Notify (by the StreamingQuery) that the query has been terminated */ + private[sql] def notifyQueryTermination(terminatedQuery: StreamingQuery): Unit = { + activeQueriesLock.synchronized { + activeQueries -= terminatedQuery.id + } + awaitTerminationLock.synchronized { + if (lastTerminatedQuery == null || terminatedQuery.exception.nonEmpty) { + lastTerminatedQuery = terminatedQuery + } + awaitTerminationLock.notifyAll() + } + stateStoreCoordinator.deactivateInstances(terminatedQuery.runId) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala new file mode 100644 index 000000000000..a0c9bcc8929e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming + +import org.json4s._ +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.annotation.InterfaceStability + +/** + * Reports information about the instantaneous status of a streaming query. + * + * @param message A human readable description of what the stream is currently doing. + * @param isDataAvailable True when there is new data to be processed. + * @param isTriggerActive True when the trigger is actively firing, false when waiting for the + * next trigger time. + * + * @since 2.1.0 + */ +@InterfaceStability.Evolving +class StreamingQueryStatus protected[sql]( + val message: String, + val isDataAvailable: Boolean, + val isTriggerActive: Boolean) extends Serializable { + + /** The compact JSON representation of this status. */ + def json: String = compact(render(jsonValue)) + + /** The pretty (i.e. indented) JSON representation of this status. */ + def prettyJson: String = pretty(render(jsonValue)) + + override def toString: String = prettyJson + + private[sql] def copy( + message: String = this.message, + isDataAvailable: Boolean = this.isDataAvailable, + isTriggerActive: Boolean = this.isTriggerActive): StreamingQueryStatus = { + new StreamingQueryStatus( + message = message, + isDataAvailable = isDataAvailable, + isTriggerActive = isTriggerActive) + } + + private[sql] def jsonValue: JValue = { + ("message" -> JString(message.toString)) ~ + ("isDataAvailable" -> JBool(isDataAvailable)) ~ + ("isTriggerActive" -> JBool(isTriggerActive)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala new file mode 100644 index 000000000000..cedc1dce4a70 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala @@ -0,0 +1,218 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.{util => ju} +import java.lang.{Long => JLong} +import java.util.UUID + +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +import org.json4s._ +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.annotation.InterfaceStability + +/** + * Information about updates made to stateful operators in a [[StreamingQuery]] during a trigger. + */ +@InterfaceStability.Evolving +class StateOperatorProgress private[sql]( + val numRowsTotal: Long, + val numRowsUpdated: Long, + val memoryUsedBytes: Long + ) extends Serializable { + + /** The compact JSON representation of this progress. */ + def json: String = compact(render(jsonValue)) + + /** The pretty (i.e. indented) JSON representation of this progress. */ + def prettyJson: String = pretty(render(jsonValue)) + + private[sql] def copy(newNumRowsUpdated: Long): StateOperatorProgress = + new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes) + + private[sql] def jsonValue: JValue = { + ("numRowsTotal" -> JInt(numRowsTotal)) ~ + ("numRowsUpdated" -> JInt(numRowsUpdated)) ~ + ("memoryUsedBytes" -> JInt(memoryUsedBytes)) + } + + override def toString: String = prettyJson +} + +/** + * Information about progress made in the execution of a [[StreamingQuery]] during + * a trigger. Each event relates to processing done for a single trigger of the streaming + * query. Events are emitted even when no new data is available to be processed. + * + * @param id An unique query id that persists across restarts. See `StreamingQuery.id()`. + * @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`. + * @param name User-specified name of the query, null if not specified. + * @param timestamp Beginning time of the trigger in ISO8601 format, i.e. UTC timestamps. + * @param batchId A unique id for the current batch of data being processed. Note that in the + * case of retries after a failure a given batchId my be executed more than once. + * Similarly, when there is no data to be processed, the batchId will not be + * incremented. + * @param durationMs The amount of time taken to perform various operations in milliseconds. + * @param eventTime Statistics of event time seen in this batch. It may contain the following keys: + * {{{ + * "max" -> "2016-12-05T20:54:20.827Z" // maximum event time seen in this trigger + * "min" -> "2016-12-05T20:54:20.827Z" // minimum event time seen in this trigger + * "avg" -> "2016-12-05T20:54:20.827Z" // average event time seen in this trigger + * "watermark" -> "2016-12-05T20:54:20.827Z" // watermark used in this trigger + * }}} + * All timestamps are in ISO8601 format, i.e. UTC timestamps. + * @param stateOperators Information about operators in the query that store state. + * @param sources detailed statistics on data being read from each of the streaming sources. + * @since 2.1.0 + */ +@InterfaceStability.Evolving +class StreamingQueryProgress private[sql]( + val id: UUID, + val runId: UUID, + val name: String, + val timestamp: String, + val batchId: Long, + val durationMs: ju.Map[String, JLong], + val eventTime: ju.Map[String, String], + val stateOperators: Array[StateOperatorProgress], + val sources: Array[SourceProgress], + val sink: SinkProgress) extends Serializable { + + /** The aggregate (across all sources) number of records processed in a trigger. */ + def numInputRows: Long = sources.map(_.numInputRows).sum + + /** The aggregate (across all sources) rate of data arriving. */ + def inputRowsPerSecond: Double = sources.map(_.inputRowsPerSecond).sum + + /** The aggregate (across all sources) rate at which Spark is processing data. */ + def processedRowsPerSecond: Double = sources.map(_.processedRowsPerSecond).sum + + /** The compact JSON representation of this progress. */ + def json: String = compact(render(jsonValue)) + + /** The pretty (i.e. indented) JSON representation of this progress. */ + def prettyJson: String = pretty(render(jsonValue)) + + override def toString: String = prettyJson + + private[sql] def jsonValue: JValue = { + def safeDoubleToJValue(value: Double): JValue = { + if (value.isNaN || value.isInfinity) JNothing else JDouble(value) + } + + /** Convert map to JValue while handling empty maps. Also, this sorts the keys. */ + def safeMapToJValue[T](map: ju.Map[String, T], valueToJValue: T => JValue): JValue = { + if (map.isEmpty) return JNothing + val keys = map.asScala.keySet.toSeq.sorted + keys.map { k => k -> valueToJValue(map.get(k)) : JObject }.reduce(_ ~ _) + } + + ("id" -> JString(id.toString)) ~ + ("runId" -> JString(runId.toString)) ~ + ("name" -> JString(name)) ~ + ("timestamp" -> JString(timestamp)) ~ + ("batchId" -> JInt(batchId)) ~ + ("numInputRows" -> JInt(numInputRows)) ~ + ("inputRowsPerSecond" -> safeDoubleToJValue(inputRowsPerSecond)) ~ + ("processedRowsPerSecond" -> safeDoubleToJValue(processedRowsPerSecond)) ~ + ("durationMs" -> safeMapToJValue[JLong](durationMs, v => JInt(v.toLong))) ~ + ("eventTime" -> safeMapToJValue[String](eventTime, s => JString(s))) ~ + ("stateOperators" -> JArray(stateOperators.map(_.jsonValue).toList)) ~ + ("sources" -> JArray(sources.map(_.jsonValue).toList)) ~ + ("sink" -> sink.jsonValue) + } +} + +/** + * Information about progress made for a source in the execution of a [[StreamingQuery]] + * during a trigger. See [[StreamingQueryProgress]] for more information. + * + * @param description Description of the source. + * @param startOffset The starting offset for data being read. + * @param endOffset The ending offset for data being read. + * @param numInputRows The number of records read from this source. + * @param inputRowsPerSecond The rate at which data is arriving from this source. + * @param processedRowsPerSecond The rate at which data from this source is being procressed by + * Spark. + * @since 2.1.0 + */ +@InterfaceStability.Evolving +class SourceProgress protected[sql]( + val description: String, + val startOffset: String, + val endOffset: String, + val numInputRows: Long, + val inputRowsPerSecond: Double, + val processedRowsPerSecond: Double) extends Serializable { + + /** The compact JSON representation of this progress. */ + def json: String = compact(render(jsonValue)) + + /** The pretty (i.e. indented) JSON representation of this progress. */ + def prettyJson: String = pretty(render(jsonValue)) + + override def toString: String = prettyJson + + private[sql] def jsonValue: JValue = { + def safeDoubleToJValue(value: Double): JValue = { + if (value.isNaN || value.isInfinity) JNothing else JDouble(value) + } + + ("description" -> JString(description)) ~ + ("startOffset" -> tryParse(startOffset)) ~ + ("endOffset" -> tryParse(endOffset)) ~ + ("numInputRows" -> JInt(numInputRows)) ~ + ("inputRowsPerSecond" -> safeDoubleToJValue(inputRowsPerSecond)) ~ + ("processedRowsPerSecond" -> safeDoubleToJValue(processedRowsPerSecond)) + } + + private def tryParse(json: String) = try { + parse(json) + } catch { + case NonFatal(e) => JString(json) + } +} + +/** + * Information about progress made for a sink in the execution of a [[StreamingQuery]] + * during a trigger. See [[StreamingQueryProgress]] for more information. + * + * @param description Description of the source corresponding to this status. + * @since 2.1.0 + */ +@InterfaceStability.Evolving +class SinkProgress protected[sql]( + val description: String) extends Serializable { + + /** The compact JSON representation of this progress. */ + def json: String = compact(render(jsonValue)) + + /** The pretty (i.e. indented) JSON representation of this progress. */ + def prettyJson: String = pretty(render(jsonValue)) + + override def toString: String = prettyJson + + private[sql] def jsonValue: JValue = { + ("description" -> JString(description)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala index 8d4854b698ed..a73e4272950a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.test -import org.apache.spark.sql.catalyst.util.{GenericArrayData, ArrayData} +import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} import org.apache.spark.sql.types._ /** @@ -26,7 +26,15 @@ import org.apache.spark.sql.types._ * @param y y coordinate */ @SQLUserDefinedType(udt = classOf[ExamplePointUDT]) -private[sql] class ExamplePoint(val x: Double, val y: Double) extends Serializable +private[sql] class ExamplePoint(val x: Double, val y: Double) extends Serializable { + + override def hashCode(): Int = 31 * (31 * x.hashCode()) + y.hashCode() + + override def equals(other: Any): Boolean = other match { + case that: ExamplePoint => this.x == that.x && this.y == that.y + case _ => false + } +} /** * User-defined type for [[ExamplePoint]]. @@ -37,14 +45,11 @@ private[sql] class ExamplePointUDT extends UserDefinedType[ExamplePoint] { override def pyUDT: String = "pyspark.sql.tests.ExamplePointUDT" - override def serialize(obj: Any): GenericArrayData = { - obj match { - case p: ExamplePoint => - val output = new Array[Any](2) - output(0) = p.x - output(1) = p.y - new GenericArrayData(output) - } + override def serialize(p: ExamplePoint): GenericArrayData = { + val output = new Array[Any](2) + output(0) = p.x + output(1) = p.y + new GenericArrayData(output) } override def deserialize(datum: Any): ExamplePoint = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala index 909a8abd225b..f6240d85fba6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala @@ -18,72 +18,64 @@ package org.apache.spark.sql.util import java.util.concurrent.locks.ReentrantReadWriteLock + import scala.collection.mutable.ListBuffer +import scala.util.control.NonFatal -import org.apache.spark.annotation.{DeveloperApi, Experimental} -import org.apache.spark.Logging +import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} +import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.QueryExecution - /** + * :: Experimental :: * The interface of query execution listener that can be used to analyze execution metrics. * - * Note that implementations should guarantee thread-safety as they will be used in a non - * thread-safe way. + * @note Implementations should guarantee thread-safety as they can be invoked by + * multiple different threads. */ @Experimental +@InterfaceStability.Evolving trait QueryExecutionListener { /** * A callback function that will be called when a query executed successfully. - * Implementations should guarantee thread-safe. * - * @param funcName the name of the action that triggered this query. + * @param funcName name of the action that triggered this query. * @param qe the QueryExecution object that carries detail information like logical plan, * physical plan, etc. - * @param duration the execution time for this query in nanoseconds. + * @param durationNs the execution time for this query in nanoseconds. + * + * @note This can be invoked by multiple different threads. */ @DeveloperApi - def onSuccess(funcName: String, qe: QueryExecution, duration: Long) + def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit /** * A callback function that will be called when a query execution failed. - * Implementations should guarantee thread-safe. * * @param funcName the name of the action that triggered this query. * @param qe the QueryExecution object that carries detail information like logical plan, * physical plan, etc. * @param exception the exception that failed this query. + * + * @note This can be invoked by multiple different threads. */ @DeveloperApi - def onFailure(funcName: String, qe: QueryExecution, exception: Exception) + def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit } -@Experimental -class ExecutionListenerManager extends Logging { - private[this] val listeners = ListBuffer.empty[QueryExecutionListener] - private[this] val lock = new ReentrantReadWriteLock() - /** Acquires a read lock on the cache for the duration of `f`. */ - private def readLock[A](f: => A): A = { - val rl = lock.readLock() - rl.lock() - try f finally { - rl.unlock() - } - } - - /** Acquires a write lock on the cache for the duration of `f`. */ - private def writeLock[A](f: => A): A = { - val wl = lock.writeLock() - wl.lock() - try f finally { - wl.unlock() - } - } +/** + * :: Experimental :: + * + * Manager for [[QueryExecutionListener]]. See `org.apache.spark.sql.SQLContext.listenerManager`. + */ +@Experimental +@InterfaceStability.Evolving +class ExecutionListenerManager private[sql] () extends Logging { /** - * Registers the specified QueryExecutionListener. + * Registers the specified [[QueryExecutionListener]]. */ @DeveloperApi def register(listener: QueryExecutionListener): Unit = writeLock { @@ -91,7 +83,7 @@ class ExecutionListenerManager extends Logging { } /** - * Unregisters the specified QueryExecutionListener. + * Unregisters the specified [[QueryExecutionListener]]. */ @DeveloperApi def unregister(listener: QueryExecutionListener): Unit = writeLock { @@ -99,38 +91,69 @@ class ExecutionListenerManager extends Logging { } /** - * clears out all registered QueryExecutionListeners. + * Removes all the registered [[QueryExecutionListener]]. */ @DeveloperApi def clear(): Unit = writeLock { listeners.clear() } - private[sql] def onSuccess( - funcName: String, - qe: QueryExecution, - duration: Long): Unit = readLock { - withErrorHandling { listener => - listener.onSuccess(funcName, qe, duration) + /** + * Get an identical copy of this listener manager. + */ + @DeveloperApi + override def clone(): ExecutionListenerManager = writeLock { + val newListenerManager = new ExecutionListenerManager + listeners.foreach(newListenerManager.register) + newListenerManager + } + + private[sql] def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { + readLock { + withErrorHandling { listener => + listener.onSuccess(funcName, qe, duration) + } } } - private[sql] def onFailure( - funcName: String, - qe: QueryExecution, - exception: Exception): Unit = readLock { - withErrorHandling { listener => - listener.onFailure(funcName, qe, exception) + private[sql] def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = { + readLock { + withErrorHandling { listener => + listener.onFailure(funcName, qe, exception) + } } } + private[this] val listeners = ListBuffer.empty[QueryExecutionListener] + + /** A lock to prevent updating the list of listeners while we are traversing through them. */ + private[this] val lock = new ReentrantReadWriteLock() + private def withErrorHandling(f: QueryExecutionListener => Unit): Unit = { for (listener <- listeners) { try { f(listener) } catch { - case e: Exception => logWarning("error executing query execution listener", e) + case NonFatal(e) => logWarning("Error executing query execution listener", e) } } } + + /** Acquires a read lock on the cache for the duration of `f`. */ + private def readLock[A](f: => A): A = { + val rl = lock.readLock() + rl.lock() + try f finally { + rl.unlock() + } + } + + /** Acquires a write lock on the cache for the duration of `f`. */ + private def writeLock[A](f: => A): A = { + val wl = lock.writeLock() + wl.lock() + try f finally { + wl.unlock() + } + } } diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroArrayOfArray.java b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroArrayOfArray.java index ee327827903e..8de0b06b162c 100644 --- a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroArrayOfArray.java +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroArrayOfArray.java @@ -129,6 +129,7 @@ public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfA } @Override + @SuppressWarnings(value="unchecked") public AvroArrayOfArray build() { try { AvroArrayOfArray record = new AvroArrayOfArray(); diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroMapOfArray.java b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroMapOfArray.java index 727f6a7bf733..29f3109f83a1 100644 --- a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroMapOfArray.java +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroMapOfArray.java @@ -129,6 +129,7 @@ public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArr } @Override + @SuppressWarnings(value="unchecked") public AvroMapOfArray build() { try { AvroMapOfArray record = new AvroMapOfArray(); diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroNonNullableArrays.java b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroNonNullableArrays.java index 934793f42f9c..c5522ed1e53e 100644 --- a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroNonNullableArrays.java +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroNonNullableArrays.java @@ -182,6 +182,7 @@ public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNulla } @Override + @SuppressWarnings(value="unchecked") public AvroNonNullableArrays build() { try { AvroNonNullableArrays record = new AvroNonNullableArrays(); diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/Nested.java b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/Nested.java index a7bf4841919c..f84e3f2d61ef 100644 --- a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/Nested.java +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/Nested.java @@ -182,6 +182,7 @@ public org.apache.spark.sql.execution.datasources.parquet.test.avro.Nested.Build } @Override + @SuppressWarnings(value="unchecked") public Nested build() { try { Nested record = new Nested(); diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/ParquetAvroCompat.java b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/ParquetAvroCompat.java index ef12d193f916..46fc608398cc 100644 --- a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/ParquetAvroCompat.java +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/ParquetAvroCompat.java @@ -235,6 +235,7 @@ public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroC } @Override + @SuppressWarnings(value="unchecked") public ParquetAvroCompat build() { try { ParquetAvroCompat record = new ParquetAvroCompat(); diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/Java8DatasetAggregatorSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/Java8DatasetAggregatorSuite.java new file mode 100644 index 000000000000..6ffccee52c0f --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/Java8DatasetAggregatorSuite.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql; + +import java.util.Arrays; + +import org.junit.Assert; +import org.junit.Test; +import scala.Tuple2; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.KeyValueGroupedDataset; +import org.apache.spark.sql.expressions.javalang.typed; + +/** + * Suite that replicates tests in JavaDatasetAggregatorSuite using lambda syntax. + */ +public class Java8DatasetAggregatorSuite extends JavaDatasetAggregatorSuiteBase { + @Test + public void testTypedAggregationAverage() { + KeyValueGroupedDataset> grouped = generateGroupedDataset(); + Dataset> agged = grouped.agg(typed.avg(v -> (double)(v._2() * 2))); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 3.0), new Tuple2<>("b", 6.0)), + agged.collectAsList()); + } + + @Test + public void testTypedAggregationCount() { + KeyValueGroupedDataset> grouped = generateGroupedDataset(); + Dataset> agged = grouped.agg(typed.count(v -> v)); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 2L), new Tuple2<>("b", 1L)), + agged.collectAsList()); + } + + @Test + public void testTypedAggregationSumDouble() { + KeyValueGroupedDataset> grouped = generateGroupedDataset(); + Dataset> agged = grouped.agg(typed.sum(v -> (double)v._2())); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 3.0), new Tuple2<>("b", 3.0)), + agged.collectAsList()); + } + + @Test + public void testTypedAggregationSumLong() { + KeyValueGroupedDataset> grouped = generateGroupedDataset(); + Dataset> agged = grouped.agg(typed.sumLong(v -> (long)v._2())); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 3L), new Tuple2<>("b", 3L)), + agged.collectAsList()); + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java index 7b50aad4ad49..eb4d76c6ab03 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java @@ -28,14 +28,13 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.SparkContext; 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.sql.DataFrame; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; @@ -44,21 +43,22 @@ // serialized, as an alternative to converting these anonymous classes to static inner classes; // see http://stackoverflow.com/questions/758570/. public class JavaApplySchemaSuite implements Serializable { - private transient JavaSparkContext javaCtx; - private transient SQLContext sqlContext; + private transient SparkSession spark; + private transient JavaSparkContext jsc; @Before public void setUp() { - SparkContext context = new SparkContext("local[*]", "testing"); - javaCtx = new JavaSparkContext(context); - sqlContext = new SQLContext(context); + spark = SparkSession.builder() + .master("local[*]") + .appName("testing") + .getOrCreate(); + jsc = new JavaSparkContext(spark.sparkContext()); } @After public void tearDown() { - sqlContext.sparkContext().stop(); - sqlContext = null; - javaCtx = null; + spark.stop(); + spark = null; } public static class Person implements Serializable { @@ -94,28 +94,23 @@ public void applySchema() { person2.setAge(28); personList.add(person2); - JavaRDD rowRDD = javaCtx.parallelize(personList).map( - new Function() { - @Override - public Row call(Person person) throws Exception { - return RowFactory.create(person.getName(), person.getAge()); - } - }); + JavaRDD rowRDD = jsc.parallelize(personList).map( + person -> RowFactory.create(person.getName(), person.getAge())); List fields = new ArrayList<>(2); fields.add(DataTypes.createStructField("name", DataTypes.StringType, false)); fields.add(DataTypes.createStructField("age", DataTypes.IntegerType, false)); StructType schema = DataTypes.createStructType(fields); - DataFrame df = sqlContext.applySchema(rowRDD, schema); - df.registerTempTable("people"); - Row[] actual = sqlContext.sql("SELECT * FROM people").collect(); + Dataset df = spark.createDataFrame(rowRDD, schema); + df.createOrReplaceTempView("people"); + List actual = spark.sql("SELECT * FROM people").collectAsList(); - List expected = new ArrayList(2); + List expected = new ArrayList<>(2); expected.add(RowFactory.create("Michael", 29)); expected.add(RowFactory.create("Yin", 28)); - Assert.assertEquals(expected, Arrays.asList(actual)); + Assert.assertEquals(expected, actual); } @Test @@ -130,27 +125,18 @@ public void dataFrameRDDOperations() { person2.setAge(28); personList.add(person2); - JavaRDD rowRDD = javaCtx.parallelize(personList).map( - new Function() { - @Override - public Row call(Person person) { - return RowFactory.create(person.getName(), person.getAge()); - } - }); + JavaRDD rowRDD = jsc.parallelize(personList).map( + person -> RowFactory.create(person.getName(), person.getAge())); List fields = new ArrayList<>(2); fields.add(DataTypes.createStructField("", DataTypes.StringType, false)); fields.add(DataTypes.createStructField("age", DataTypes.IntegerType, false)); StructType schema = DataTypes.createStructType(fields); - DataFrame df = sqlContext.applySchema(rowRDD, schema); - df.registerTempTable("people"); - List actual = sqlContext.sql("SELECT * FROM people").toJavaRDD().map(new Function() { - @Override - public String call(Row row) { - return row.getString(0) + "_" + row.get(1); - } - }).collect(); + Dataset df = spark.createDataFrame(rowRDD, schema); + df.createOrReplaceTempView("people"); + List actual = spark.sql("SELECT * FROM people").toJavaRDD() + .map(row -> row.getString(0) + "_" + row.get(1)).collect(); List expected = new ArrayList<>(2); expected.add("Michael_29"); @@ -161,13 +147,13 @@ public String call(Row row) { @Test public void applySchemaToJSON() { - JavaRDD jsonRDD = javaCtx.parallelize(Arrays.asList( + Dataset jsonDS = spark.createDataset(Arrays.asList( "{\"string\":\"this is a simple string.\", \"integer\":10, \"long\":21474836470, " + "\"bigInteger\":92233720368547758070, \"double\":1.7976931348623157E308, " + "\"boolean\":true, \"null\":null}", "{\"string\":\"this is another simple string.\", \"integer\":11, \"long\":21474836469, " + "\"bigInteger\":92233720368547758069, \"double\":1.7976931348623157E305, " + - "\"boolean\":false, \"null\":null}")); + "\"boolean\":false, \"null\":null}"), Encoders.STRING()); List fields = new ArrayList<>(7); fields.add(DataTypes.createStructField("bigInteger", DataTypes.createDecimalType(20, 0), true)); @@ -198,18 +184,18 @@ public void applySchemaToJSON() { null, "this is another simple string.")); - DataFrame df1 = sqlContext.read().json(jsonRDD); + Dataset df1 = spark.read().json(jsonDS); StructType actualSchema1 = df1.schema(); Assert.assertEquals(expectedSchema, actualSchema1); - df1.registerTempTable("jsonTable1"); - List actual1 = sqlContext.sql("select * from jsonTable1").collectAsList(); + df1.createOrReplaceTempView("jsonTable1"); + List actual1 = spark.sql("select * from jsonTable1").collectAsList(); Assert.assertEquals(expectedResult, actual1); - DataFrame df2 = sqlContext.read().schema(expectedSchema).json(jsonRDD); + Dataset df2 = spark.read().schema(expectedSchema).json(jsonDS); StructType actualSchema2 = df2.schema(); Assert.assertEquals(expectedSchema, actualSchema2); - df2.registerTempTable("jsonTable2"); - List actual2 = sqlContext.sql("select * from jsonTable2").collectAsList(); + df2.createOrReplaceTempView("jsonTable2"); + List actual2 = spark.sql("select * from jsonTable2").collectAsList(); Assert.assertEquals(expectedResult, actual2); } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java new file mode 100644 index 000000000000..7babf7573c07 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameReaderWriterSuite.java @@ -0,0 +1,158 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package test.org.apache.spark.sql; + +import java.io.File; +import java.util.HashMap; + +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.test.TestSparkSession; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.util.Utils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class JavaDataFrameReaderWriterSuite { + private SparkSession spark = new TestSparkSession(); + private StructType schema = new StructType().add("s", "string"); + private transient String input; + private transient String output; + + @Before + public void setUp() { + input = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "input").toString(); + File f = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "output"); + f.delete(); + output = f.toString(); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } + + @Test + public void testFormatAPI() { + spark + .read() + .format("org.apache.spark.sql.test") + .load() + .write() + .format("org.apache.spark.sql.test") + .save(); + } + + @Test + public void testOptionsAPI() { + HashMap map = new HashMap(); + map.put("e", "1"); + spark + .read() + .option("a", "1") + .option("b", 1) + .option("c", 1.0) + .option("d", true) + .options(map) + .text() + .write() + .option("a", "1") + .option("b", 1) + .option("c", 1.0) + .option("d", true) + .options(map) + .format("org.apache.spark.sql.test") + .save(); + } + + @Test + public void testSaveModeAPI() { + spark + .range(10) + .write() + .format("org.apache.spark.sql.test") + .mode(SaveMode.ErrorIfExists) + .save(); + } + + @Test + public void testLoadAPI() { + spark.read().format("org.apache.spark.sql.test").load(); + spark.read().format("org.apache.spark.sql.test").load(input); + spark.read().format("org.apache.spark.sql.test").load(input, input, input); + spark.read().format("org.apache.spark.sql.test").load(new String[]{input, input}); + } + + @Test + public void testTextAPI() { + spark.read().text(); + spark.read().text(input); + spark.read().text(input, input, input); + spark.read().text(new String[]{input, input}) + .write().text(output); + } + + @Test + public void testTextFileAPI() { + spark.read().textFile(); + spark.read().textFile(input); + spark.read().textFile(input, input, input); + spark.read().textFile(new String[]{input, input}); + } + + @Test + public void testCsvAPI() { + spark.read().schema(schema).csv(); + spark.read().schema(schema).csv(input); + spark.read().schema(schema).csv(input, input, input); + spark.read().schema(schema).csv(new String[]{input, input}) + .write().csv(output); + } + + @Test + public void testJsonAPI() { + spark.read().schema(schema).json(); + spark.read().schema(schema).json(input); + spark.read().schema(schema).json(input, input, input); + spark.read().schema(schema).json(new String[]{input, input}) + .write().json(output); + } + + @Test + public void testParquetAPI() { + spark.read().schema(schema).parquet(); + spark.read().schema(schema).parquet(input); + spark.read().schema(schema).parquet(input, input, input); + spark.read().schema(schema).parquet(new String[] { input, input }) + .write().parquet(output); + } + + /** + * This only tests whether API compiles, but does not run it as orc() + * cannot be run without Hive classes. + */ + public void testOrcAPI() { + spark.read().schema(schema).orc(); + spark.read().schema(schema).orc(input); + spark.read().schema(schema).orc(input, input, input); + spark.read().schema(schema).orc(new String[]{input, input}) + .write().orc(output); + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index 49f516e86d75..b007093dad84 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -18,10 +18,11 @@ package test.org.apache.spark.sql; import java.io.Serializable; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; -import java.util.Map; +import java.net.URISyntaxException; +import java.net.URL; +import java.util.*; +import java.math.BigInteger; +import java.math.BigDecimal; import scala.collection.JavaConverters; import scala.collection.Seq; @@ -30,39 +31,47 @@ import com.google.common.primitives.Ints; import org.junit.*; -import org.apache.spark.SparkContext; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.*; -import static org.apache.spark.sql.functions.*; -import org.apache.spark.sql.test.TestSQLContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.test.TestSparkSession; import org.apache.spark.sql.types.*; +import org.apache.spark.util.sketch.BloomFilter; +import org.apache.spark.util.sketch.CountMinSketch; +import static org.apache.spark.sql.functions.*; import static org.apache.spark.sql.types.DataTypes.*; public class JavaDataFrameSuite { + private transient TestSparkSession spark; private transient JavaSparkContext jsc; - private transient TestSQLContext context; @Before public void setUp() { // Trigger static initializer of TestData - SparkContext sc = new SparkContext("local[*]", "testing"); - jsc = new JavaSparkContext(sc); - context = new TestSQLContext(sc); - context.loadTestData(); + spark = new TestSparkSession(); + jsc = new JavaSparkContext(spark.sparkContext()); + spark.loadTestData(); } @After public void tearDown() { - context.sparkContext().stop(); - context = null; - jsc = null; + spark.stop(); + spark = null; } @Test public void testExecution() { - DataFrame df = context.table("testData").filter("key = 1"); - Assert.assertEquals(1, df.select("key").collect()[0].get(0)); + Dataset df = spark.table("testData").filter("key = 1"); + Assert.assertEquals(1, df.select("key").collectAsList().get(0).get(0)); + } + + @Test + public void testCollectAndTake() { + Dataset df = spark.table("testData").filter("key = 1 or key = 2 or key = 3"); + Assert.assertEquals(3, df.select("key").collectAsList().size()); + Assert.assertEquals(2, df.select("key").takeAsList(2).size()); } /** @@ -70,7 +79,7 @@ public void testExecution() { */ @Test public void testVarargMethods() { - DataFrame df = context.table("testData"); + Dataset df = spark.table("testData"); df.toDF("key1", "value1"); @@ -99,7 +108,7 @@ public void testVarargMethods() { df.select(coalesce(col("key"))); // Varargs with mathfunctions - DataFrame df2 = context.table("testData2"); + Dataset df2 = spark.table("testData2"); df2.select(exp("a"), exp("b")); df2.select(exp(log("a"))); df2.select(pow("a", "a"), pow("b", 2.0)); @@ -113,7 +122,7 @@ public void testVarargMethods() { @Ignore public void testShow() { // This test case is intended ignored, but to make sure it compiles correctly - DataFrame df = context.table("testData"); + Dataset df = spark.table("testData"); df.show(); df.show(1000); } @@ -123,6 +132,7 @@ public static class Bean implements Serializable { private Integer[] b = { 0, 1 }; private Map c = ImmutableMap.of("hello", new int[] { 1, 2 }); private List d = Arrays.asList("floppy", "disk"); + private BigInteger e = new BigInteger("1234567"); public double getA() { return a; @@ -139,9 +149,11 @@ public Map getC() { public List getD() { return d; } + + public BigInteger getE() { return e; } } - void validateDataFrameWithBeans(Bean bean, DataFrame df) { + void validateDataFrameWithBeans(Bean bean, Dataset df) { StructType schema = df.schema(); Assert.assertEquals(new StructField("a", DoubleType$.MODULE$, false, Metadata.empty()), schema.apply("a")); @@ -156,7 +168,9 @@ void validateDataFrameWithBeans(Bean bean, DataFrame df) { Assert.assertEquals( new StructField("d", new ArrayType(DataTypes.StringType, true), true, Metadata.empty()), schema.apply("d")); - Row first = df.select("a", "b", "c", "d").first(); + Assert.assertEquals(new StructField("e", DataTypes.createDecimalType(38,0), true, + Metadata.empty()), schema.apply("e")); + Row first = df.select("a", "b", "c", "d", "e").first(); Assert.assertEquals(bean.getA(), first.getDouble(0), 0.0); // Now Java lists and maps are converted to Scala Seq's and Map's. Once we get a Seq below, // verify that it has the expected length, and contains expected elements. @@ -175,13 +189,15 @@ void validateDataFrameWithBeans(Bean bean, DataFrame df) { for (int i = 0; i < d.length(); i++) { Assert.assertEquals(bean.getD().get(i), d.apply(i)); } + // Java.math.BigInteger is equivalent to Spark Decimal(38,0) + Assert.assertEquals(new BigDecimal(bean.getE()), first.getDecimal(4)); } @Test public void testCreateDataFrameFromLocalJavaBeans() { Bean bean = new Bean(); List data = Arrays.asList(bean); - DataFrame df = context.createDataFrame(data, Bean.class); + Dataset df = spark.createDataFrame(data, Bean.class); validateDataFrameWithBeans(bean, df); } @@ -189,7 +205,7 @@ public void testCreateDataFrameFromLocalJavaBeans() { public void testCreateDataFrameFromJavaBeans() { Bean bean = new Bean(); JavaRDD rdd = jsc.parallelize(Arrays.asList(bean)); - DataFrame df = context.createDataFrame(rdd, Bean.class); + Dataset df = spark.createDataFrame(rdd, Bean.class); validateDataFrameWithBeans(bean, df); } @@ -197,30 +213,40 @@ public void testCreateDataFrameFromJavaBeans() { public void testCreateDataFromFromList() { StructType schema = createStructType(Arrays.asList(createStructField("i", IntegerType, true))); List rows = Arrays.asList(RowFactory.create(0)); - DataFrame df = context.createDataFrame(rows, schema); - Row[] result = df.collect(); - Assert.assertEquals(1, result.length); + Dataset df = spark.createDataFrame(rows, schema); + List result = df.collectAsList(); + Assert.assertEquals(1, result.size()); } - private static final Comparator crosstabRowComparator = new Comparator() { - @Override - public int compare(Row row1, Row row2) { - String item1 = row1.getString(0); - String item2 = row2.getString(0); - return item1.compareTo(item2); - } + @Test + public void testCreateStructTypeFromList(){ + List fields1 = new ArrayList<>(); + fields1.add(new StructField("id", DataTypes.StringType, true, Metadata.empty())); + StructType schema1 = StructType$.MODULE$.apply(fields1); + Assert.assertEquals(0, schema1.fieldIndex("id")); + + List fields2 = + Arrays.asList(new StructField("id", DataTypes.StringType, true, Metadata.empty())); + StructType schema2 = StructType$.MODULE$.apply(fields2); + Assert.assertEquals(0, schema2.fieldIndex("id")); + } + + private static final Comparator crosstabRowComparator = (row1, row2) -> { + String item1 = row1.getString(0); + String item2 = row2.getString(0); + return item1.compareTo(item2); }; @Test public void testCrosstab() { - DataFrame df = context.table("testData2"); - DataFrame crosstab = df.stat().crosstab("a", "b"); + Dataset df = spark.table("testData2"); + Dataset crosstab = df.stat().crosstab("a", "b"); String[] columnNames = crosstab.schema().fieldNames(); Assert.assertEquals("a_b", columnNames[0]); Assert.assertEquals("1", columnNames[1]); Assert.assertEquals("2", columnNames[2]); - Row[] rows = crosstab.collect(); - Arrays.sort(rows, crosstabRowComparator); + List rows = crosstab.collectAsList(); + rows.sort(crosstabRowComparator); Integer count = 1; for (Row row : rows) { Assert.assertEquals(row.get(0).toString(), count.toString()); @@ -232,32 +258,201 @@ public void testCrosstab() { @Test public void testFrequentItems() { - DataFrame df = context.table("testData2"); + Dataset df = spark.table("testData2"); String[] cols = {"a"}; - DataFrame results = df.stat().freqItems(cols, 0.2); - Assert.assertTrue(results.collect()[0].getSeq(0).contains(1)); + Dataset results = df.stat().freqItems(cols, 0.2); + Assert.assertTrue(results.collectAsList().get(0).getSeq(0).contains(1)); } @Test public void testCorrelation() { - DataFrame df = context.table("testData2"); + Dataset df = spark.table("testData2"); Double pearsonCorr = df.stat().corr("a", "b", "pearson"); Assert.assertTrue(Math.abs(pearsonCorr) < 1.0e-6); } @Test public void testCovariance() { - DataFrame df = context.table("testData2"); + Dataset df = spark.table("testData2"); Double result = df.stat().cov("a", "b"); Assert.assertTrue(Math.abs(result) < 1.0e-6); } @Test public void testSampleBy() { - DataFrame df = context.range(0, 100, 1, 2).select(col("id").mod(3).as("key")); - DataFrame sampled = df.stat().sampleBy("key", ImmutableMap.of(0, 0.1, 1, 0.2), 0L); - Row[] actual = sampled.groupBy("key").count().orderBy("key").collect(); - Row[] expected = {RowFactory.create(0, 5), RowFactory.create(1, 8)}; - Assert.assertArrayEquals(expected, actual); + Dataset df = spark.range(0, 100, 1, 2).select(col("id").mod(3).as("key")); + Dataset sampled = df.stat().sampleBy("key", ImmutableMap.of(0, 0.1, 1, 0.2), 0L); + List actual = sampled.groupBy("key").count().orderBy("key").collectAsList(); + Assert.assertEquals(0, actual.get(0).getLong(0)); + Assert.assertTrue(0 <= actual.get(0).getLong(1) && actual.get(0).getLong(1) <= 8); + Assert.assertEquals(1, actual.get(1).getLong(0)); + Assert.assertTrue(2 <= actual.get(1).getLong(1) && actual.get(1).getLong(1) <= 13); + } + + @Test + public void pivot() { + Dataset df = spark.table("courseSales"); + List actual = df.groupBy("year") + .pivot("course", Arrays.asList("dotNET", "Java")) + .agg(sum("earnings")).orderBy("year").collectAsList(); + + Assert.assertEquals(2012, actual.get(0).getInt(0)); + Assert.assertEquals(15000.0, actual.get(0).getDouble(1), 0.01); + Assert.assertEquals(20000.0, actual.get(0).getDouble(2), 0.01); + + Assert.assertEquals(2013, actual.get(1).getInt(0)); + Assert.assertEquals(48000.0, actual.get(1).getDouble(1), 0.01); + Assert.assertEquals(30000.0, actual.get(1).getDouble(2), 0.01); + } + + private String getResource(String resource) { + try { + // The following "getResource" has different behaviors in SBT and Maven. + // When running in Jenkins, the file path may contain "@" when there are multiple + // SparkPullRequestBuilders running in the same worker + // (e.g., /home/jenkins/workspace/SparkPullRequestBuilder@2) + // When running in SBT, "@" in the file path will be returned as "@", however, + // when running in Maven, "@" will be encoded as "%40". + // Therefore, we convert it to URI then call "getPath" to decode it back so that it can both + // work both in SBT and Maven. + URL url = Thread.currentThread().getContextClassLoader().getResource(resource); + return url.toURI().getPath(); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + + @Test + public void testGenericLoad() { + Dataset df1 = spark.read().format("text").load(getResource("test-data/text-suite.txt")); + Assert.assertEquals(4L, df1.count()); + + Dataset df2 = spark.read().format("text").load( + getResource("test-data/text-suite.txt"), + getResource("test-data/text-suite2.txt")); + Assert.assertEquals(5L, df2.count()); + } + + @Test + public void testTextLoad() { + Dataset ds1 = spark.read().textFile(getResource("test-data/text-suite.txt")); + Assert.assertEquals(4L, ds1.count()); + + Dataset ds2 = spark.read().textFile( + getResource("test-data/text-suite.txt"), + getResource("test-data/text-suite2.txt")); + Assert.assertEquals(5L, ds2.count()); + } + + @Test + public void testCountMinSketch() { + Dataset df = spark.range(1000); + + CountMinSketch sketch1 = df.stat().countMinSketch("id", 10, 20, 42); + Assert.assertEquals(1000, sketch1.totalCount()); + Assert.assertEquals(10, sketch1.depth()); + Assert.assertEquals(20, sketch1.width()); + + CountMinSketch sketch2 = df.stat().countMinSketch(col("id"), 10, 20, 42); + Assert.assertEquals(1000, sketch2.totalCount()); + Assert.assertEquals(10, sketch2.depth()); + Assert.assertEquals(20, sketch2.width()); + + CountMinSketch sketch3 = df.stat().countMinSketch("id", 0.001, 0.99, 42); + Assert.assertEquals(1000, sketch3.totalCount()); + Assert.assertEquals(0.001, sketch3.relativeError(), 1.0e-4); + Assert.assertEquals(0.99, sketch3.confidence(), 5.0e-3); + + CountMinSketch sketch4 = df.stat().countMinSketch(col("id"), 0.001, 0.99, 42); + Assert.assertEquals(1000, sketch4.totalCount()); + Assert.assertEquals(0.001, sketch4.relativeError(), 1.0e-4); + Assert.assertEquals(0.99, sketch4.confidence(), 5.0e-3); + } + + @Test + public void testBloomFilter() { + Dataset df = spark.range(1000); + + BloomFilter filter1 = df.stat().bloomFilter("id", 1000, 0.03); + Assert.assertTrue(filter1.expectedFpp() - 0.03 < 1e-3); + for (int i = 0; i < 1000; i++) { + Assert.assertTrue(filter1.mightContain(i)); + } + + BloomFilter filter2 = df.stat().bloomFilter(col("id").multiply(3), 1000, 0.03); + Assert.assertTrue(filter2.expectedFpp() - 0.03 < 1e-3); + for (int i = 0; i < 1000; i++) { + Assert.assertTrue(filter2.mightContain(i * 3)); + } + + BloomFilter filter3 = df.stat().bloomFilter("id", 1000, 64 * 5); + Assert.assertEquals(64 * 5, filter3.bitSize()); + for (int i = 0; i < 1000; i++) { + Assert.assertTrue(filter3.mightContain(i)); + } + + BloomFilter filter4 = df.stat().bloomFilter(col("id").multiply(3), 1000, 64 * 5); + Assert.assertEquals(64 * 5, filter4.bitSize()); + for (int i = 0; i < 1000; i++) { + Assert.assertTrue(filter4.mightContain(i * 3)); + } + } + + public static class BeanWithoutGetter implements Serializable { + private String a; + + public void setA(String a) { + this.a = a; + } + } + + @Test + public void testBeanWithoutGetter() { + BeanWithoutGetter bean = new BeanWithoutGetter(); + List data = Arrays.asList(bean); + Dataset df = spark.createDataFrame(data, BeanWithoutGetter.class); + Assert.assertEquals(df.schema().length(), 0); + Assert.assertEquals(df.collectAsList().size(), 1); + } + + @Test + public void testJsonRDDToDataFrame() { + // This is a test for the deprecated API in SPARK-15615. + JavaRDD rdd = jsc.parallelize(Arrays.asList("{\"a\": 2}")); + Dataset df = spark.read().json(rdd); + Assert.assertEquals(1L, df.count()); + Assert.assertEquals(2L, df.collectAsList().get(0).getLong(0)); + } + + public class CircularReference1Bean implements Serializable { + private CircularReference2Bean child; + + public CircularReference2Bean getChild() { + return child; + } + + public void setChild(CircularReference2Bean child) { + this.child = child; + } + } + + public class CircularReference2Bean implements Serializable { + private CircularReference1Bean child; + + public CircularReference1Bean getChild() { + return child; + } + + public void setChild(CircularReference1Bean child) { + this.child = child; + } + } + + // Checks a simple case for DataFrame here and put exhaustive tests for the issue + // of circular references in `JavaDatasetSuite`. + @Test(expected = UnsupportedOperationException.class) + public void testCircularReferenceBean() { + CircularReference1Bean bean = new CircularReference1Bean(); + spark.createDataFrame(Arrays.asList(bean), CircularReference1Bean.class); } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java new file mode 100644 index 000000000000..539976d5af46 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql; + +import java.util.Arrays; + +import scala.Tuple2; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.KeyValueGroupedDataset; +import org.apache.spark.sql.expressions.Aggregator; +import org.apache.spark.sql.expressions.javalang.typed; + +/** + * Suite for testing the aggregate functionality of Datasets in Java. + */ +public class JavaDatasetAggregatorSuite extends JavaDatasetAggregatorSuiteBase { + @Test + public void testTypedAggregationAnonClass() { + KeyValueGroupedDataset> grouped = generateGroupedDataset(); + + Dataset> agged = grouped.agg(new IntSumOf().toColumn()); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 3), new Tuple2<>("b", 3)), + agged.collectAsList()); + + Dataset> agged2 = grouped.agg(new IntSumOf().toColumn()) + .as(Encoders.tuple(Encoders.STRING(), Encoders.INT())); + Assert.assertEquals( + Arrays.asList( + new Tuple2<>("a", 3), + new Tuple2<>("b", 3)), + agged2.collectAsList()); + } + + static class IntSumOf extends Aggregator, Integer, Integer> { + @Override + public Integer zero() { + return 0; + } + + @Override + public Integer reduce(Integer l, Tuple2 t) { + return l + t._2(); + } + + @Override + public Integer merge(Integer b1, Integer b2) { + return b1 + b2; + } + + @Override + public Integer finish(Integer reduction) { + return reduction; + } + + @Override + public Encoder bufferEncoder() { + return Encoders.INT(); + } + + @Override + public Encoder outputEncoder() { + return Encoders.INT(); + } + } + + @Test + public void testTypedAggregationAverage() { + KeyValueGroupedDataset> grouped = generateGroupedDataset(); + Dataset> agged = grouped.agg(typed.avg(value -> value._2() * 2.0)); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 3.0), new Tuple2<>("b", 6.0)), + agged.collectAsList()); + } + + @Test + public void testTypedAggregationCount() { + KeyValueGroupedDataset> grouped = generateGroupedDataset(); + Dataset> agged = grouped.agg(typed.count(value -> value)); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 2L), new Tuple2<>("b", 1L)), + agged.collectAsList()); + } + + @Test + public void testTypedAggregationSumDouble() { + KeyValueGroupedDataset> grouped = generateGroupedDataset(); + Dataset> agged = grouped.agg(typed.sum(value -> (double) value._2())); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 3.0), new Tuple2<>("b", 3.0)), + agged.collectAsList()); + } + + @Test + public void testTypedAggregationSumLong() { + KeyValueGroupedDataset> grouped = generateGroupedDataset(); + Dataset> agged = grouped.agg(typed.sumLong(value -> (long) value._2())); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 3L), new Tuple2<>("b", 3L)), + agged.collectAsList()); + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuiteBase.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuiteBase.java new file mode 100644 index 000000000000..e62db7d2cff6 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuiteBase.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; + +import scala.Tuple2; + +import org.junit.After; +import org.junit.Before; + +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.KeyValueGroupedDataset; +import org.apache.spark.sql.test.TestSparkSession; + +/** + * Common test base shared across this and Java8DatasetAggregatorSuite. + */ +public class JavaDatasetAggregatorSuiteBase implements Serializable { + private transient TestSparkSession spark; + + @Before + public void setUp() { + // Trigger static initializer of TestData + spark = new TestSparkSession(); + spark.loadTestData(); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } + + protected KeyValueGroupedDataset> generateGroupedDataset() { + Encoder> encoder = Encoders.tuple(Encoders.STRING(), Encoders.INT()); + List> data = + Arrays.asList(new Tuple2<>("a", 1), new Tuple2<>("a", 2), new Tuple2<>("b", 3)); + Dataset> ds = spark.createDataset(data, encoder); + + return ds.groupByKey((MapFunction, String>) value -> value._1(), + Encoders.STRING()); + } +} + diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java new file mode 100644 index 000000000000..c132cab1b38c --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java @@ -0,0 +1,1537 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql; + +import java.io.Serializable; +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.*; + +import org.apache.spark.sql.streaming.GroupStateTimeout; +import org.apache.spark.sql.streaming.OutputMode; +import scala.Tuple2; +import scala.Tuple3; +import scala.Tuple4; +import scala.Tuple5; + +import com.google.common.base.Objects; +import org.junit.*; +import org.junit.rules.ExpectedException; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.*; +import org.apache.spark.sql.*; +import org.apache.spark.sql.catalyst.encoders.OuterScopes; +import org.apache.spark.sql.catalyst.expressions.GenericRow; +import org.apache.spark.sql.test.TestSparkSession; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.util.LongAccumulator; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.types.DataTypes.*; + +public class JavaDatasetSuite implements Serializable { + private transient TestSparkSession spark; + private transient JavaSparkContext jsc; + + @Before + public void setUp() { + // Trigger static initializer of TestData + spark = new TestSparkSession(); + jsc = new JavaSparkContext(spark.sparkContext()); + spark.loadTestData(); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } + + private Tuple2 tuple2(T1 t1, T2 t2) { + return new Tuple2<>(t1, t2); + } + + @Test + public void testCollect() { + List data = Arrays.asList("hello", "world"); + Dataset ds = spark.createDataset(data, Encoders.STRING()); + List collected = ds.collectAsList(); + Assert.assertEquals(Arrays.asList("hello", "world"), collected); + } + + @Test + public void testTake() { + List data = Arrays.asList("hello", "world"); + Dataset ds = spark.createDataset(data, Encoders.STRING()); + List collected = ds.takeAsList(1); + Assert.assertEquals(Arrays.asList("hello"), collected); + } + + @Test + public void testToLocalIterator() { + List data = Arrays.asList("hello", "world"); + Dataset ds = spark.createDataset(data, Encoders.STRING()); + Iterator iter = ds.toLocalIterator(); + Assert.assertEquals("hello", iter.next()); + Assert.assertEquals("world", iter.next()); + Assert.assertFalse(iter.hasNext()); + } + + // SPARK-15632: typed filter should preserve the underlying logical schema + @Test + public void testTypedFilterPreservingSchema() { + Dataset ds = spark.range(10); + Dataset ds2 = ds.filter((FilterFunction) value -> value > 3); + Assert.assertEquals(ds.schema(), ds2.schema()); + } + + @Test + public void testCommonOperation() { + List data = Arrays.asList("hello", "world"); + Dataset ds = spark.createDataset(data, Encoders.STRING()); + Assert.assertEquals("hello", ds.first()); + + Dataset filtered = ds.filter((FilterFunction) v -> v.startsWith("h")); + Assert.assertEquals(Arrays.asList("hello"), filtered.collectAsList()); + + + Dataset mapped = + ds.map((MapFunction) String::length, Encoders.INT()); + Assert.assertEquals(Arrays.asList(5, 5), mapped.collectAsList()); + + Dataset parMapped = ds.mapPartitions((MapPartitionsFunction) it -> { + List ls = new LinkedList<>(); + while (it.hasNext()) { + ls.add(it.next().toUpperCase(Locale.ROOT)); + } + return ls.iterator(); + }, Encoders.STRING()); + Assert.assertEquals(Arrays.asList("HELLO", "WORLD"), parMapped.collectAsList()); + + Dataset flatMapped = ds.flatMap((FlatMapFunction) s -> { + List ls = new LinkedList<>(); + for (char c : s.toCharArray()) { + ls.add(String.valueOf(c)); + } + return ls.iterator(); + }, Encoders.STRING()); + Assert.assertEquals( + Arrays.asList("h", "e", "l", "l", "o", "w", "o", "r", "l", "d"), + flatMapped.collectAsList()); + } + + @Test + public void testForeach() { + LongAccumulator accum = jsc.sc().longAccumulator(); + List data = Arrays.asList("a", "b", "c"); + Dataset ds = spark.createDataset(data, Encoders.STRING()); + + ds.foreach((ForeachFunction) s -> accum.add(1)); + Assert.assertEquals(3, accum.value().intValue()); + } + + @Test + public void testReduce() { + List data = Arrays.asList(1, 2, 3); + Dataset ds = spark.createDataset(data, Encoders.INT()); + + int reduced = ds.reduce((ReduceFunction) (v1, v2) -> v1 + v2); + Assert.assertEquals(6, reduced); + } + + @Test + public void testGroupBy() { + List data = Arrays.asList("a", "foo", "bar"); + Dataset ds = spark.createDataset(data, Encoders.STRING()); + KeyValueGroupedDataset grouped = + ds.groupByKey((MapFunction) String::length, Encoders.INT()); + + Dataset mapped = grouped.mapGroups( + (MapGroupsFunction) (key, values) -> { + StringBuilder sb = new StringBuilder(key.toString()); + while (values.hasNext()) { + sb.append(values.next()); + } + return sb.toString(); + }, Encoders.STRING()); + + Assert.assertEquals(asSet("1a", "3foobar"), toSet(mapped.collectAsList())); + + Dataset flatMapped = grouped.flatMapGroups( + (FlatMapGroupsFunction) (key, values) -> { + StringBuilder sb = new StringBuilder(key.toString()); + while (values.hasNext()) { + sb.append(values.next()); + } + return Collections.singletonList(sb.toString()).iterator(); + }, + Encoders.STRING()); + + Assert.assertEquals(asSet("1a", "3foobar"), toSet(flatMapped.collectAsList())); + + Dataset mapped2 = grouped.mapGroupsWithState( + (MapGroupsWithStateFunction) (key, values, s) -> { + StringBuilder sb = new StringBuilder(key.toString()); + while (values.hasNext()) { + sb.append(values.next()); + } + return sb.toString(); + }, + Encoders.LONG(), + Encoders.STRING()); + + Assert.assertEquals(asSet("1a", "3foobar"), toSet(mapped2.collectAsList())); + + Dataset flatMapped2 = grouped.flatMapGroupsWithState( + (FlatMapGroupsWithStateFunction) (key, values, s) -> { + StringBuilder sb = new StringBuilder(key.toString()); + while (values.hasNext()) { + sb.append(values.next()); + } + return Collections.singletonList(sb.toString()).iterator(); + }, + OutputMode.Append(), + Encoders.LONG(), + Encoders.STRING(), + GroupStateTimeout.NoTimeout()); + + Assert.assertEquals(asSet("1a", "3foobar"), toSet(flatMapped2.collectAsList())); + + Dataset> reduced = + grouped.reduceGroups((ReduceFunction) (v1, v2) -> v1 + v2); + + Assert.assertEquals( + asSet(tuple2(1, "a"), tuple2(3, "foobar")), + toSet(reduced.collectAsList())); + + List data2 = Arrays.asList(2, 6, 10); + Dataset ds2 = spark.createDataset(data2, Encoders.INT()); + KeyValueGroupedDataset grouped2 = ds2.groupByKey( + (MapFunction) v -> v / 2, + Encoders.INT()); + + Dataset cogrouped = grouped.cogroup( + grouped2, + (CoGroupFunction) (key, left, right) -> { + StringBuilder sb = new StringBuilder(key.toString()); + while (left.hasNext()) { + sb.append(left.next()); + } + sb.append("#"); + while (right.hasNext()) { + sb.append(right.next()); + } + return Collections.singletonList(sb.toString()).iterator(); + }, + Encoders.STRING()); + + Assert.assertEquals(asSet("1a#2", "3foobar#6", "5#10"), toSet(cogrouped.collectAsList())); + } + + @Test + public void testSelect() { + List data = Arrays.asList(2, 6); + Dataset ds = spark.createDataset(data, Encoders.INT()); + + Dataset> selected = ds.select( + expr("value + 1"), + col("value").cast("string")).as(Encoders.tuple(Encoders.INT(), Encoders.STRING())); + + Assert.assertEquals( + Arrays.asList(tuple2(3, "2"), tuple2(7, "6")), + selected.collectAsList()); + } + + @Test + public void testSetOperation() { + List data = Arrays.asList("abc", "abc", "xyz"); + Dataset ds = spark.createDataset(data, Encoders.STRING()); + + Assert.assertEquals(asSet("abc", "xyz"), toSet(ds.distinct().collectAsList())); + + List data2 = Arrays.asList("xyz", "foo", "foo"); + Dataset ds2 = spark.createDataset(data2, Encoders.STRING()); + + Dataset intersected = ds.intersect(ds2); + Assert.assertEquals(Arrays.asList("xyz"), intersected.collectAsList()); + + Dataset unioned = ds.union(ds2).union(ds); + Assert.assertEquals( + Arrays.asList("abc", "abc", "xyz", "xyz", "foo", "foo", "abc", "abc", "xyz"), + unioned.collectAsList()); + + Dataset subtracted = ds.except(ds2); + Assert.assertEquals(Arrays.asList("abc"), subtracted.collectAsList()); + } + + private static Set toSet(List records) { + return new HashSet<>(records); + } + + @SafeVarargs + @SuppressWarnings("varargs") + private static Set asSet(T... records) { + return toSet(Arrays.asList(records)); + } + + @Test + public void testJoin() { + List data = Arrays.asList(1, 2, 3); + Dataset ds = spark.createDataset(data, Encoders.INT()).as("a"); + List data2 = Arrays.asList(2, 3, 4); + Dataset ds2 = spark.createDataset(data2, Encoders.INT()).as("b"); + + Dataset> joined = + ds.joinWith(ds2, col("a.value").equalTo(col("b.value"))); + Assert.assertEquals( + Arrays.asList(tuple2(2, 2), tuple2(3, 3)), + joined.collectAsList()); + } + + @Test + public void testTupleEncoder() { + Encoder> encoder2 = Encoders.tuple(Encoders.INT(), Encoders.STRING()); + List> data2 = Arrays.asList(tuple2(1, "a"), tuple2(2, "b")); + Dataset> ds2 = spark.createDataset(data2, encoder2); + Assert.assertEquals(data2, ds2.collectAsList()); + + Encoder> encoder3 = + Encoders.tuple(Encoders.INT(), Encoders.LONG(), Encoders.STRING()); + List> data3 = + Arrays.asList(new Tuple3<>(1, 2L, "a")); + Dataset> ds3 = spark.createDataset(data3, encoder3); + Assert.assertEquals(data3, ds3.collectAsList()); + + Encoder> encoder4 = + Encoders.tuple(Encoders.INT(), Encoders.STRING(), Encoders.LONG(), Encoders.STRING()); + List> data4 = + Arrays.asList(new Tuple4<>(1, "b", 2L, "a")); + Dataset> ds4 = spark.createDataset(data4, encoder4); + Assert.assertEquals(data4, ds4.collectAsList()); + + Encoder> encoder5 = + Encoders.tuple(Encoders.INT(), Encoders.STRING(), Encoders.LONG(), Encoders.STRING(), + Encoders.BOOLEAN()); + List> data5 = + Arrays.asList(new Tuple5<>(1, "b", 2L, "a", true)); + Dataset> ds5 = + spark.createDataset(data5, encoder5); + Assert.assertEquals(data5, ds5.collectAsList()); + } + + @Test + public void testNestedTupleEncoder() { + // test ((int, string), string) + Encoder, String>> encoder = + Encoders.tuple(Encoders.tuple(Encoders.INT(), Encoders.STRING()), Encoders.STRING()); + List, String>> data = + Arrays.asList(tuple2(tuple2(1, "a"), "a"), tuple2(tuple2(2, "b"), "b")); + Dataset, String>> ds = spark.createDataset(data, encoder); + Assert.assertEquals(data, ds.collectAsList()); + + // test (int, (string, string, long)) + Encoder>> encoder2 = + Encoders.tuple(Encoders.INT(), + Encoders.tuple(Encoders.STRING(), Encoders.STRING(), Encoders.LONG())); + List>> data2 = + Arrays.asList(tuple2(1, new Tuple3<>("a", "b", 3L))); + Dataset>> ds2 = + spark.createDataset(data2, encoder2); + Assert.assertEquals(data2, ds2.collectAsList()); + + // test (int, ((string, long), string)) + Encoder, String>>> encoder3 = + Encoders.tuple(Encoders.INT(), + Encoders.tuple(Encoders.tuple(Encoders.STRING(), Encoders.LONG()), Encoders.STRING())); + List, String>>> data3 = + Arrays.asList(tuple2(1, tuple2(tuple2("a", 2L), "b"))); + Dataset, String>>> ds3 = + spark.createDataset(data3, encoder3); + Assert.assertEquals(data3, ds3.collectAsList()); + } + + @Test + public void testPrimitiveEncoder() { + Encoder> encoder = + Encoders.tuple(Encoders.DOUBLE(), Encoders.DECIMAL(), Encoders.DATE(), Encoders.TIMESTAMP(), + Encoders.FLOAT()); + List> data = + Arrays.asList(new Tuple5<>( + 1.7976931348623157E308, new BigDecimal("0.922337203685477589"), + Date.valueOf("1970-01-01"), new Timestamp(System.currentTimeMillis()), Float.MAX_VALUE)); + Dataset> ds = + spark.createDataset(data, encoder); + Assert.assertEquals(data, ds.collectAsList()); + } + + public static class KryoSerializable { + String value; + + KryoSerializable(String value) { + this.value = value; + } + + @Override + public boolean equals(Object other) { + if (this == other) return true; + if (other == null || getClass() != other.getClass()) return false; + + return this.value.equals(((KryoSerializable) other).value); + } + + @Override + public int hashCode() { + return this.value.hashCode(); + } + } + + public static class JavaSerializable implements Serializable { + String value; + + JavaSerializable(String value) { + this.value = value; + } + + @Override + public boolean equals(Object other) { + if (this == other) return true; + if (other == null || getClass() != other.getClass()) return false; + + return this.value.equals(((JavaSerializable) other).value); + } + + @Override + public int hashCode() { + return this.value.hashCode(); + } + } + + @Test + public void testKryoEncoder() { + Encoder encoder = Encoders.kryo(KryoSerializable.class); + List data = Arrays.asList( + new KryoSerializable("hello"), new KryoSerializable("world")); + Dataset ds = spark.createDataset(data, encoder); + Assert.assertEquals(data, ds.collectAsList()); + } + + @Test + public void testJavaEncoder() { + Encoder encoder = Encoders.javaSerialization(JavaSerializable.class); + List data = Arrays.asList( + new JavaSerializable("hello"), new JavaSerializable("world")); + Dataset ds = spark.createDataset(data, encoder); + Assert.assertEquals(data, ds.collectAsList()); + } + + @Test + public void testRandomSplit() { + List data = Arrays.asList("hello", "world", "from", "spark"); + Dataset ds = spark.createDataset(data, Encoders.STRING()); + double[] arraySplit = {1, 2, 3}; + + List> randomSplit = ds.randomSplitAsList(arraySplit, 1); + Assert.assertEquals("wrong number of splits", randomSplit.size(), 3); + } + + /** + * For testing error messages when creating an encoder on a private class. This is done + * here since we cannot create truly private classes in Scala. + */ + private static class PrivateClassTest { } + + @Test(expected = UnsupportedOperationException.class) + public void testJavaEncoderErrorMessageForPrivateClass() { + Encoders.javaSerialization(PrivateClassTest.class); + } + + @Test(expected = UnsupportedOperationException.class) + public void testKryoEncoderErrorMessageForPrivateClass() { + Encoders.kryo(PrivateClassTest.class); + } + + public static class SimpleJavaBean implements Serializable { + private boolean a; + private int b; + private byte[] c; + private String[] d; + private List e; + private List f; + private Map g; + private Map, Map> h; + + public boolean isA() { + return a; + } + + public void setA(boolean a) { + this.a = a; + } + + public int getB() { + return b; + } + + public void setB(int b) { + this.b = b; + } + + public byte[] getC() { + return c; + } + + public void setC(byte[] c) { + this.c = c; + } + + public String[] getD() { + return d; + } + + public void setD(String[] d) { + this.d = d; + } + + public List getE() { + return e; + } + + public void setE(List e) { + this.e = e; + } + + public List getF() { + return f; + } + + public void setF(List f) { + this.f = f; + } + + public Map getG() { + return g; + } + + public void setG(Map g) { + this.g = g; + } + + public Map, Map> getH() { + return h; + } + + public void setH(Map, Map> h) { + this.h = h; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + SimpleJavaBean that = (SimpleJavaBean) o; + + if (a != that.a) return false; + if (b != that.b) return false; + if (!Arrays.equals(c, that.c)) return false; + if (!Arrays.equals(d, that.d)) return false; + if (!e.equals(that.e)) return false; + if (!f.equals(that.f)) return false; + if (!g.equals(that.g)) return false; + return h.equals(that.h); + + } + + @Override + public int hashCode() { + int result = (a ? 1 : 0); + result = 31 * result + b; + result = 31 * result + Arrays.hashCode(c); + result = 31 * result + Arrays.hashCode(d); + result = 31 * result + e.hashCode(); + result = 31 * result + f.hashCode(); + result = 31 * result + g.hashCode(); + result = 31 * result + h.hashCode(); + return result; + } + } + + public static class SimpleJavaBean2 implements Serializable { + private Timestamp a; + private Date b; + private java.math.BigDecimal c; + + public Timestamp getA() { return a; } + + public void setA(Timestamp a) { this.a = a; } + + public Date getB() { return b; } + + public void setB(Date b) { this.b = b; } + + public java.math.BigDecimal getC() { return c; } + + public void setC(java.math.BigDecimal c) { this.c = c; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + SimpleJavaBean2 that = (SimpleJavaBean2) o; + + if (!a.equals(that.a)) return false; + if (!b.equals(that.b)) return false; + return c.equals(that.c); + } + + @Override + public int hashCode() { + int result = a.hashCode(); + result = 31 * result + b.hashCode(); + result = 31 * result + c.hashCode(); + return result; + } + } + + public static class NestedJavaBean implements Serializable { + private SimpleJavaBean a; + + public SimpleJavaBean getA() { + return a; + } + + public void setA(SimpleJavaBean a) { + this.a = a; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + NestedJavaBean that = (NestedJavaBean) o; + + return a.equals(that.a); + } + + @Override + public int hashCode() { + return a.hashCode(); + } + } + + @Test + public void testJavaBeanEncoder() { + OuterScopes.addOuterScope(this); + SimpleJavaBean obj1 = new SimpleJavaBean(); + obj1.setA(true); + obj1.setB(3); + obj1.setC(new byte[]{1, 2}); + obj1.setD(new String[]{"hello", null}); + obj1.setE(Arrays.asList("a", "b")); + obj1.setF(Arrays.asList(100L, null, 200L)); + Map map1 = new HashMap<>(); + map1.put(1, "a"); + map1.put(2, "b"); + obj1.setG(map1); + Map nestedMap1 = new HashMap<>(); + nestedMap1.put("x", "1"); + nestedMap1.put("y", "2"); + Map, Map> complexMap1 = new HashMap<>(); + complexMap1.put(Arrays.asList(1L, 2L), nestedMap1); + obj1.setH(complexMap1); + + SimpleJavaBean obj2 = new SimpleJavaBean(); + obj2.setA(false); + obj2.setB(30); + obj2.setC(new byte[]{3, 4}); + obj2.setD(new String[]{null, "world"}); + obj2.setE(Arrays.asList("x", "y")); + obj2.setF(Arrays.asList(300L, null, 400L)); + Map map2 = new HashMap<>(); + map2.put(3, "c"); + map2.put(4, "d"); + obj2.setG(map2); + Map nestedMap2 = new HashMap<>(); + nestedMap2.put("q", "1"); + nestedMap2.put("w", "2"); + Map, Map> complexMap2 = new HashMap<>(); + complexMap2.put(Arrays.asList(3L, 4L), nestedMap2); + obj2.setH(complexMap2); + + List data = Arrays.asList(obj1, obj2); + Dataset ds = spark.createDataset(data, Encoders.bean(SimpleJavaBean.class)); + Assert.assertEquals(data, ds.collectAsList()); + + NestedJavaBean obj3 = new NestedJavaBean(); + obj3.setA(obj1); + + List data2 = Arrays.asList(obj3); + Dataset ds2 = spark.createDataset(data2, Encoders.bean(NestedJavaBean.class)); + Assert.assertEquals(data2, ds2.collectAsList()); + + Row row1 = new GenericRow(new Object[]{ + true, + 3, + new byte[]{1, 2}, + new String[]{"hello", null}, + Arrays.asList("a", "b"), + Arrays.asList(100L, null, 200L), + map1, + complexMap1}); + Row row2 = new GenericRow(new Object[]{ + false, + 30, + new byte[]{3, 4}, + new String[]{null, "world"}, + Arrays.asList("x", "y"), + Arrays.asList(300L, null, 400L), + map2, + complexMap2}); + StructType schema = new StructType() + .add("a", BooleanType, false) + .add("b", IntegerType, false) + .add("c", BinaryType) + .add("d", createArrayType(StringType)) + .add("e", createArrayType(StringType)) + .add("f", createArrayType(LongType)) + .add("g", createMapType(IntegerType, StringType)) + .add("h",createMapType(createArrayType(LongType), createMapType(StringType, StringType))); + Dataset ds3 = spark.createDataFrame(Arrays.asList(row1, row2), schema) + .as(Encoders.bean(SimpleJavaBean.class)); + Assert.assertEquals(data, ds3.collectAsList()); + } + + @Test + public void testJavaBeanEncoder2() { + // This is a regression test of SPARK-12404 + OuterScopes.addOuterScope(this); + SimpleJavaBean2 obj = new SimpleJavaBean2(); + obj.setA(new Timestamp(0)); + obj.setB(new Date(0)); + obj.setC(java.math.BigDecimal.valueOf(1)); + Dataset ds = + spark.createDataset(Arrays.asList(obj), Encoders.bean(SimpleJavaBean2.class)); + ds.collect(); + } + + public static class SmallBean implements Serializable { + private String a; + + private int b; + + public int getB() { + return b; + } + + public void setB(int b) { + this.b = b; + } + + public String getA() { + return a; + } + + public void setA(String a) { + this.a = a; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SmallBean smallBean = (SmallBean) o; + return b == smallBean.b && com.google.common.base.Objects.equal(a, smallBean.a); + } + + @Override + public int hashCode() { + return Objects.hashCode(a, b); + } + } + + public static class NestedSmallBean implements Serializable { + private SmallBean f; + + public SmallBean getF() { + return f; + } + + public void setF(SmallBean f) { + this.f = f; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + NestedSmallBean that = (NestedSmallBean) o; + return Objects.equal(f, that.f); + } + + @Override + public int hashCode() { + return Objects.hashCode(f); + } + } + + @Rule + public transient ExpectedException nullabilityCheck = ExpectedException.none(); + + @Test + public void testRuntimeNullabilityCheck() { + OuterScopes.addOuterScope(this); + + StructType schema = new StructType() + .add("f", new StructType() + .add("a", StringType, true) + .add("b", IntegerType, true), true); + + // Shouldn't throw runtime exception since it passes nullability check. + { + Row row = new GenericRow(new Object[] { + new GenericRow(new Object[] { + "hello", 1 + }) + }); + + Dataset df = spark.createDataFrame(Collections.singletonList(row), schema); + Dataset ds = df.as(Encoders.bean(NestedSmallBean.class)); + + SmallBean smallBean = new SmallBean(); + smallBean.setA("hello"); + smallBean.setB(1); + + NestedSmallBean nestedSmallBean = new NestedSmallBean(); + nestedSmallBean.setF(smallBean); + + Assert.assertEquals(ds.collectAsList(), Collections.singletonList(nestedSmallBean)); + } + + // Shouldn't throw runtime exception when parent object (`ClassData`) is null + { + Row row = new GenericRow(new Object[] { null }); + + Dataset df = spark.createDataFrame(Collections.singletonList(row), schema); + Dataset ds = df.as(Encoders.bean(NestedSmallBean.class)); + + NestedSmallBean nestedSmallBean = new NestedSmallBean(); + Assert.assertEquals(ds.collectAsList(), Collections.singletonList(nestedSmallBean)); + } + + nullabilityCheck.expect(RuntimeException.class); + nullabilityCheck.expectMessage("Null value appeared in non-nullable field"); + + { + Row row = new GenericRow(new Object[] { + new GenericRow(new Object[] { + "hello", null + }) + }); + + Dataset df = spark.createDataFrame(Collections.singletonList(row), schema); + Dataset ds = df.as(Encoders.bean(NestedSmallBean.class)); + + ds.collect(); + } + } + + public static class Nesting3 implements Serializable { + private Integer field3_1; + private Double field3_2; + private String field3_3; + + public Nesting3() { + } + + public Nesting3(Integer field3_1, Double field3_2, String field3_3) { + this.field3_1 = field3_1; + this.field3_2 = field3_2; + this.field3_3 = field3_3; + } + + private Nesting3(Builder builder) { + setField3_1(builder.field3_1); + setField3_2(builder.field3_2); + setField3_3(builder.field3_3); + } + + public static Builder newBuilder() { + return new Builder(); + } + + public Integer getField3_1() { + return field3_1; + } + + public void setField3_1(Integer field3_1) { + this.field3_1 = field3_1; + } + + public Double getField3_2() { + return field3_2; + } + + public void setField3_2(Double field3_2) { + this.field3_2 = field3_2; + } + + public String getField3_3() { + return field3_3; + } + + public void setField3_3(String field3_3) { + this.field3_3 = field3_3; + } + + public static final class Builder { + private Integer field3_1 = 0; + private Double field3_2 = 0.0; + private String field3_3 = "value"; + + private Builder() { + } + + public Builder field3_1(Integer field3_1) { + this.field3_1 = field3_1; + return this; + } + + public Builder field3_2(Double field3_2) { + this.field3_2 = field3_2; + return this; + } + + public Builder field3_3(String field3_3) { + this.field3_3 = field3_3; + return this; + } + + public Nesting3 build() { + return new Nesting3(this); + } + } + } + + public static class Nesting2 implements Serializable { + private Nesting3 field2_1; + private Nesting3 field2_2; + private Nesting3 field2_3; + + public Nesting2() { + } + + public Nesting2(Nesting3 field2_1, Nesting3 field2_2, Nesting3 field2_3) { + this.field2_1 = field2_1; + this.field2_2 = field2_2; + this.field2_3 = field2_3; + } + + private Nesting2(Builder builder) { + setField2_1(builder.field2_1); + setField2_2(builder.field2_2); + setField2_3(builder.field2_3); + } + + public static Builder newBuilder() { + return new Builder(); + } + + public Nesting3 getField2_1() { + return field2_1; + } + + public void setField2_1(Nesting3 field2_1) { + this.field2_1 = field2_1; + } + + public Nesting3 getField2_2() { + return field2_2; + } + + public void setField2_2(Nesting3 field2_2) { + this.field2_2 = field2_2; + } + + public Nesting3 getField2_3() { + return field2_3; + } + + public void setField2_3(Nesting3 field2_3) { + this.field2_3 = field2_3; + } + + + public static final class Builder { + private Nesting3 field2_1 = Nesting3.newBuilder().build(); + private Nesting3 field2_2 = Nesting3.newBuilder().build(); + private Nesting3 field2_3 = Nesting3.newBuilder().build(); + + private Builder() { + } + + public Builder field2_1(Nesting3 field2_1) { + this.field2_1 = field2_1; + return this; + } + + public Builder field2_2(Nesting3 field2_2) { + this.field2_2 = field2_2; + return this; + } + + public Builder field2_3(Nesting3 field2_3) { + this.field2_3 = field2_3; + return this; + } + + public Nesting2 build() { + return new Nesting2(this); + } + } + } + + public static class Nesting1 implements Serializable { + private Nesting2 field1_1; + private Nesting2 field1_2; + private Nesting2 field1_3; + + public Nesting1() { + } + + public Nesting1(Nesting2 field1_1, Nesting2 field1_2, Nesting2 field1_3) { + this.field1_1 = field1_1; + this.field1_2 = field1_2; + this.field1_3 = field1_3; + } + + private Nesting1(Builder builder) { + setField1_1(builder.field1_1); + setField1_2(builder.field1_2); + setField1_3(builder.field1_3); + } + + public static Builder newBuilder() { + return new Builder(); + } + + public Nesting2 getField1_1() { + return field1_1; + } + + public void setField1_1(Nesting2 field1_1) { + this.field1_1 = field1_1; + } + + public Nesting2 getField1_2() { + return field1_2; + } + + public void setField1_2(Nesting2 field1_2) { + this.field1_2 = field1_2; + } + + public Nesting2 getField1_3() { + return field1_3; + } + + public void setField1_3(Nesting2 field1_3) { + this.field1_3 = field1_3; + } + + + public static final class Builder { + private Nesting2 field1_1 = Nesting2.newBuilder().build(); + private Nesting2 field1_2 = Nesting2.newBuilder().build(); + private Nesting2 field1_3 = Nesting2.newBuilder().build(); + + private Builder() { + } + + public Builder field1_1(Nesting2 field1_1) { + this.field1_1 = field1_1; + return this; + } + + public Builder field1_2(Nesting2 field1_2) { + this.field1_2 = field1_2; + return this; + } + + public Builder field1_3(Nesting2 field1_3) { + this.field1_3 = field1_3; + return this; + } + + public Nesting1 build() { + return new Nesting1(this); + } + } + } + + public static class NestedComplicatedJavaBean implements Serializable { + private Nesting1 field1; + private Nesting1 field2; + private Nesting1 field3; + private Nesting1 field4; + private Nesting1 field5; + private Nesting1 field6; + private Nesting1 field7; + private Nesting1 field8; + private Nesting1 field9; + private Nesting1 field10; + + public NestedComplicatedJavaBean() { + } + + private NestedComplicatedJavaBean(Builder builder) { + setField1(builder.field1); + setField2(builder.field2); + setField3(builder.field3); + setField4(builder.field4); + setField5(builder.field5); + setField6(builder.field6); + setField7(builder.field7); + setField8(builder.field8); + setField9(builder.field9); + setField10(builder.field10); + } + + public static Builder newBuilder() { + return new Builder(); + } + + public Nesting1 getField1() { + return field1; + } + + public void setField1(Nesting1 field1) { + this.field1 = field1; + } + + public Nesting1 getField2() { + return field2; + } + + public void setField2(Nesting1 field2) { + this.field2 = field2; + } + + public Nesting1 getField3() { + return field3; + } + + public void setField3(Nesting1 field3) { + this.field3 = field3; + } + + public Nesting1 getField4() { + return field4; + } + + public void setField4(Nesting1 field4) { + this.field4 = field4; + } + + public Nesting1 getField5() { + return field5; + } + + public void setField5(Nesting1 field5) { + this.field5 = field5; + } + + public Nesting1 getField6() { + return field6; + } + + public void setField6(Nesting1 field6) { + this.field6 = field6; + } + + public Nesting1 getField7() { + return field7; + } + + public void setField7(Nesting1 field7) { + this.field7 = field7; + } + + public Nesting1 getField8() { + return field8; + } + + public void setField8(Nesting1 field8) { + this.field8 = field8; + } + + public Nesting1 getField9() { + return field9; + } + + public void setField9(Nesting1 field9) { + this.field9 = field9; + } + + public Nesting1 getField10() { + return field10; + } + + public void setField10(Nesting1 field10) { + this.field10 = field10; + } + + public static final class Builder { + private Nesting1 field1 = Nesting1.newBuilder().build(); + private Nesting1 field2 = Nesting1.newBuilder().build(); + private Nesting1 field3 = Nesting1.newBuilder().build(); + private Nesting1 field4 = Nesting1.newBuilder().build(); + private Nesting1 field5 = Nesting1.newBuilder().build(); + private Nesting1 field6 = Nesting1.newBuilder().build(); + private Nesting1 field7 = Nesting1.newBuilder().build(); + private Nesting1 field8 = Nesting1.newBuilder().build(); + private Nesting1 field9 = Nesting1.newBuilder().build(); + private Nesting1 field10 = Nesting1.newBuilder().build(); + + private Builder() { + } + + public Builder field1(Nesting1 field1) { + this.field1 = field1; + return this; + } + + public Builder field2(Nesting1 field2) { + this.field2 = field2; + return this; + } + + public Builder field3(Nesting1 field3) { + this.field3 = field3; + return this; + } + + public Builder field4(Nesting1 field4) { + this.field4 = field4; + return this; + } + + public Builder field5(Nesting1 field5) { + this.field5 = field5; + return this; + } + + public Builder field6(Nesting1 field6) { + this.field6 = field6; + return this; + } + + public Builder field7(Nesting1 field7) { + this.field7 = field7; + return this; + } + + public Builder field8(Nesting1 field8) { + this.field8 = field8; + return this; + } + + public Builder field9(Nesting1 field9) { + this.field9 = field9; + return this; + } + + public Builder field10(Nesting1 field10) { + this.field10 = field10; + return this; + } + + public NestedComplicatedJavaBean build() { + return new NestedComplicatedJavaBean(this); + } + } + } + + @Test + public void test() { + /* SPARK-15285 Large numbers of Nested JavaBeans generates more than 64KB java bytecode */ + List data = new ArrayList<>(); + data.add(NestedComplicatedJavaBean.newBuilder().build()); + + NestedComplicatedJavaBean obj3 = new NestedComplicatedJavaBean(); + + Dataset ds = + spark.createDataset(data, Encoders.bean(NestedComplicatedJavaBean.class)); + ds.collectAsList(); + } + + public enum MyEnum { + A("www.elgoog.com"), + B("www.google.com"); + + private String url; + + MyEnum(String url) { + this.url = url; + } + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + } + + public static class BeanWithEnum { + MyEnum enumField; + String regularField; + + public String getRegularField() { + return regularField; + } + + public void setRegularField(String regularField) { + this.regularField = regularField; + } + + public MyEnum getEnumField() { + return enumField; + } + + public void setEnumField(MyEnum field) { + this.enumField = field; + } + + public BeanWithEnum(MyEnum enumField, String regularField) { + this.enumField = enumField; + this.regularField = regularField; + } + + public BeanWithEnum() { + } + + public String toString() { + return "BeanWithEnum(" + enumField + ", " + regularField + ")"; + } + + public int hashCode() { + return Objects.hashCode(enumField, regularField); + } + + public boolean equals(Object other) { + if (other instanceof BeanWithEnum) { + BeanWithEnum beanWithEnum = (BeanWithEnum) other; + return beanWithEnum.regularField.equals(regularField) + && beanWithEnum.enumField.equals(enumField); + } + return false; + } + } + + @Test + public void testBeanWithEnum() { + List data = Arrays.asList(new BeanWithEnum(MyEnum.A, "mira avenue"), + new BeanWithEnum(MyEnum.B, "flower boulevard")); + Encoder encoder = Encoders.bean(BeanWithEnum.class); + Dataset ds = spark.createDataset(data, encoder); + Assert.assertEquals(ds.collectAsList(), data); + } + + public static class EmptyBean implements Serializable {} + + @Test + public void testEmptyBean() { + EmptyBean bean = new EmptyBean(); + List data = Arrays.asList(bean); + Dataset df = spark.createDataset(data, Encoders.bean(EmptyBean.class)); + Assert.assertEquals(df.schema().length(), 0); + Assert.assertEquals(df.collectAsList().size(), 1); + } + + public class CircularReference1Bean implements Serializable { + private CircularReference2Bean child; + + public CircularReference2Bean getChild() { + return child; + } + + public void setChild(CircularReference2Bean child) { + this.child = child; + } + } + + public class CircularReference2Bean implements Serializable { + private CircularReference1Bean child; + + public CircularReference1Bean getChild() { + return child; + } + + public void setChild(CircularReference1Bean child) { + this.child = child; + } + } + + public class CircularReference3Bean implements Serializable { + private CircularReference3Bean[] child; + + public CircularReference3Bean[] getChild() { + return child; + } + + public void setChild(CircularReference3Bean[] child) { + this.child = child; + } + } + + public class CircularReference4Bean implements Serializable { + private Map child; + + public Map getChild() { + return child; + } + + public void setChild(Map child) { + this.child = child; + } + } + + public class CircularReference5Bean implements Serializable { + private String id; + private List child; + + public String getId() { + return id; + } + + public List getChild() { + return child; + } + + public void setId(String id) { + this.id = id; + } + + public void setChild(List child) { + this.child = child; + } + } + + @Test(expected = UnsupportedOperationException.class) + public void testCircularReferenceBean1() { + CircularReference1Bean bean = new CircularReference1Bean(); + spark.createDataset(Arrays.asList(bean), Encoders.bean(CircularReference1Bean.class)); + } + + @Test(expected = UnsupportedOperationException.class) + public void testCircularReferenceBean2() { + CircularReference3Bean bean = new CircularReference3Bean(); + spark.createDataset(Arrays.asList(bean), Encoders.bean(CircularReference3Bean.class)); + } + + @Test(expected = UnsupportedOperationException.class) + public void testCircularReferenceBean3() { + CircularReference4Bean bean = new CircularReference4Bean(); + spark.createDataset(Arrays.asList(bean), Encoders.bean(CircularReference4Bean.class)); + } + + @Test(expected = RuntimeException.class) + public void testNullInTopLevelBean() { + NestedSmallBean bean = new NestedSmallBean(); + // We cannot set null in top-level bean + spark.createDataset(Arrays.asList(bean, null), Encoders.bean(NestedSmallBean.class)); + } + + @Test + public void testSerializeNull() { + NestedSmallBean bean = new NestedSmallBean(); + Encoder encoder = Encoders.bean(NestedSmallBean.class); + List beans = Arrays.asList(bean); + Dataset ds1 = spark.createDataset(beans, encoder); + Assert.assertEquals(beans, ds1.collectAsList()); + Dataset ds2 = + ds1.map((MapFunction) b -> b, encoder); + Assert.assertEquals(beans, ds2.collectAsList()); + } + + @Test + public void testSpecificLists() { + SpecificListsBean bean = new SpecificListsBean(); + ArrayList arrayList = new ArrayList<>(); + arrayList.add(1); + bean.setArrayList(arrayList); + LinkedList linkedList = new LinkedList<>(); + linkedList.add(1); + bean.setLinkedList(linkedList); + bean.setList(Collections.singletonList(1)); + List beans = Collections.singletonList(bean); + Dataset dataset = + spark.createDataset(beans, Encoders.bean(SpecificListsBean.class)); + Assert.assertEquals(beans, dataset.collectAsList()); + } + + public static class SpecificListsBean implements Serializable { + private ArrayList arrayList; + private LinkedList linkedList; + private List list; + + public ArrayList getArrayList() { + return arrayList; + } + + public void setArrayList(ArrayList arrayList) { + this.arrayList = arrayList; + } + + public LinkedList getLinkedList() { + return linkedList; + } + + public void setLinkedList(LinkedList linkedList) { + this.linkedList = linkedList; + } + + public List getList() { + return list; + } + + public void setList(List list) { + this.list = list; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SpecificListsBean that = (SpecificListsBean) o; + return Objects.equal(arrayList, that.arrayList) && + Objects.equal(linkedList, that.linkedList) && + Objects.equal(list, that.list); + } + + @Override + public int hashCode() { + return Objects.hashCode(arrayList, linkedList, list); + } + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java new file mode 100644 index 000000000000..127d272579a6 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaSaveLoadSuite.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql; + +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.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +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 SparkSession spark; + + File path; + Dataset df; + + private static void checkAnswer(Dataset actual, List expected) { + String errorMessage = QueryTest$.MODULE$.checkAnswer(actual, expected); + if (errorMessage != null) { + Assert.fail(errorMessage); + } + } + + @Before + public void setUp() throws IOException { + spark = SparkSession.builder() + .master("local[*]") + .appName("testing") + .getOrCreate(); + + 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 + "\"}"); + } + Dataset ds = spark.createDataset(jsonObjects, Encoders.STRING()); + df = spark.read().json(ds); + df.createOrReplaceTempView("jsonTable"); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } + + @Test + public void saveAndLoad() { + Map options = new HashMap<>(); + options.put("path", path.toString()); + df.write().mode(SaveMode.ErrorIfExists).format("json").options(options).save(); + Dataset loadedDF = spark.read().format("json").options(options).load(); + checkAnswer(loadedDF, df.collectAsList()); + } + + @Test + public void saveAndLoadWithSchema() { + Map options = new HashMap<>(); + options.put("path", path.toString()); + df.write().format("json").mode(SaveMode.ErrorIfExists).options(options).save(); + + List fields = new ArrayList<>(); + fields.add(DataTypes.createStructField("b", DataTypes.StringType, true)); + StructType schema = DataTypes.createStructType(fields); + Dataset loadedDF = spark.read().format("json").schema(schema).options(options).load(); + + checkAnswer(loadedDF, spark.sql("SELECT b FROM jsonTable").collectAsList()); + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java new file mode 100644 index 000000000000..b90224f2ae39 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java @@ -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 test.org.apache.spark.sql; + +import org.apache.spark.sql.api.java.UDF1; + +/** + * It is used for register Java UDF from PySpark + */ +public class JavaStringLength implements UDF1 { + @Override + public Integer call(String str) throws Exception { + return new Integer(str.length()); + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDAFSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDAFSuite.java new file mode 100644 index 000000000000..ddbaa45a483c --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDAFSuite.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql; + +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + + +public class JavaUDAFSuite { + + private transient SparkSession spark; + + @Before + public void setUp() { + spark = SparkSession.builder() + .master("local[*]") + .appName("testing") + .getOrCreate(); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } + + @SuppressWarnings("unchecked") + @Test + public void udf1Test() { + spark.range(1, 10).toDF("value").registerTempTable("df"); + spark.udf().registerJavaUDAF("myDoubleAvg", MyDoubleAvg.class.getName()); + Row result = spark.sql("SELECT myDoubleAvg(value) as my_avg from df").head(); + Assert.assertEquals(105.0, result.getDouble(0), 1.0e-6); + } + +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java index 4a78dca7fea6..5bf188882618 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java @@ -18,76 +18,107 @@ package test.org.apache.spark.sql; import java.io.Serializable; +import java.util.List; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.apache.spark.SparkContext; +import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.api.java.UDF1; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.api.java.UDF2; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.types.DataTypes; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; // see http://stackoverflow.com/questions/758570/. public class JavaUDFSuite implements Serializable { - private transient JavaSparkContext sc; - private transient SQLContext sqlContext; + private transient SparkSession spark; @Before public void setUp() { - SparkContext _sc = new SparkContext("local[*]", "testing"); - sqlContext = new SQLContext(_sc); - sc = new JavaSparkContext(_sc); + spark = SparkSession.builder() + .master("local[*]") + .appName("testing") + .getOrCreate(); } @After public void tearDown() { - sqlContext.sparkContext().stop(); - sqlContext = null; - sc = null; + spark.stop(); + spark = null; } @SuppressWarnings("unchecked") @Test public void udf1Test() { - // With Java 8 lambdas: - // sqlContext.registerFunction( - // "stringLengthTest", (String str) -> str.length(), DataType.IntegerType); - - sqlContext.udf().register("stringLengthTest", new UDF1() { - @Override - public Integer call(String str) { - return str.length(); - } - }, DataTypes.IntegerType); - - Row result = sqlContext.sql("SELECT stringLengthTest('test')").head(); + spark.udf().register("stringLengthTest", (String str) -> str.length(), DataTypes.IntegerType); + + Row result = spark.sql("SELECT stringLengthTest('test')").head(); Assert.assertEquals(4, result.getInt(0)); } @SuppressWarnings("unchecked") @Test public void udf2Test() { - // With Java 8 lambdas: - // sqlContext.registerFunction( - // "stringLengthTest", - // (String str1, String str2) -> str1.length() + str2.length, - // DataType.IntegerType); - - sqlContext.udf().register("stringLengthTest", new UDF2() { - @Override - public Integer call(String str1, String str2) { - return str1.length() + str2.length(); - } - }, DataTypes.IntegerType); - - Row result = sqlContext.sql("SELECT stringLengthTest('test', 'test2')").head(); + spark.udf().register("stringLengthTest", + (String str1, String str2) -> str1.length() + str2.length(), DataTypes.IntegerType); + + Row result = spark.sql("SELECT stringLengthTest('test', 'test2')").head(); + Assert.assertEquals(9, result.getInt(0)); + } + + public static class StringLengthTest implements UDF2 { + @Override + public Integer call(String str1, String str2) { + return str1.length() + str2.length(); + } + } + + @SuppressWarnings("unchecked") + @Test + public void udf3Test() { + spark.udf().registerJava("stringLengthTest", StringLengthTest.class.getName(), + DataTypes.IntegerType); + Row result = spark.sql("SELECT stringLengthTest('test', 'test2')").head(); Assert.assertEquals(9, result.getInt(0)); + + // returnType is not provided + spark.udf().registerJava("stringLengthTest2", StringLengthTest.class.getName(), null); + result = spark.sql("SELECT stringLengthTest('test', 'test2')").head(); + Assert.assertEquals(9, result.getInt(0)); + } + + @SuppressWarnings("unchecked") + @Test + public void udf4Test() { + spark.udf().register("inc", (Long i) -> i + 1, DataTypes.LongType); + + spark.range(10).toDF("x").createOrReplaceTempView("tmp"); + // This tests when Java UDFs are required to be the semantically same (See SPARK-9435). + List results = spark.sql("SELECT inc(x) FROM tmp GROUP BY inc(x)").collectAsList(); + Assert.assertEquals(10, results.size()); + long sum = 0; + for (Row result : results) { + sum += result.getLong(0); + } + Assert.assertEquals(55, sum); + } + + @SuppressWarnings("unchecked") + @Test(expected = AnalysisException.class) + public void udf5Test() { + spark.udf().register("inc", (Long i) -> i + 1, DataTypes.LongType); + List results = spark.sql("SELECT inc(1, 5)").collectAsList(); + } + + @SuppressWarnings("unchecked") + @Test + public void udf6Test() { + spark.udf().register("returnOne", () -> 1, DataTypes.IntegerType); + Row result = spark.sql("SELECT returnOne()").head(); + Assert.assertEquals(1, result.getInt(0)); } } diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java b/sql/core/src/test/java/test/org/apache/spark/sql/MyDoubleAvg.java similarity index 96% rename from sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java rename to sql/core/src/test/java/test/org/apache/spark/sql/MyDoubleAvg.java index 5a167edd8959..447a71d284fb 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/MyDoubleAvg.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.hive.aggregate; +package test.org.apache.spark.sql; import java.util.ArrayList; import java.util.List; @@ -42,14 +42,14 @@ public class MyDoubleAvg extends UserDefinedAggregateFunction { private DataType _returnDataType; public MyDoubleAvg() { - List inputFields = new ArrayList(); + List inputFields = new ArrayList<>(); inputFields.add(DataTypes.createStructField("inputDouble", DataTypes.DoubleType, true)); _inputDataType = DataTypes.createStructType(inputFields); // The buffer has two values, bufferSum for storing the current sum and // bufferCount for storing the number of non-null input values that have been contribuetd // to the current sum. - List bufferFields = new ArrayList(); + List bufferFields = new ArrayList<>(); bufferFields.add(DataTypes.createStructField("bufferSum", DataTypes.DoubleType, true)); bufferFields.add(DataTypes.createStructField("bufferCount", DataTypes.LongType, true)); _bufferSchema = DataTypes.createStructType(bufferFields); diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java b/sql/core/src/test/java/test/org/apache/spark/sql/MyDoubleSum.java similarity index 95% rename from sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java rename to sql/core/src/test/java/test/org/apache/spark/sql/MyDoubleSum.java index c3b7768e71bf..93d20330c717 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/MyDoubleSum.java @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.spark.sql.hive.aggregate; +package test.org.apache.spark.sql; import java.util.ArrayList; import java.util.List; +import org.apache.spark.sql.Row; import org.apache.spark.sql.expressions.MutableAggregationBuffer; import org.apache.spark.sql.expressions.UserDefinedAggregateFunction; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; /** * An example {@link UserDefinedAggregateFunction} to calculate the sum of a @@ -41,11 +41,11 @@ public class MyDoubleSum extends UserDefinedAggregateFunction { private DataType _returnDataType; public MyDoubleSum() { - List inputFields = new ArrayList(); + List inputFields = new ArrayList<>(); inputFields.add(DataTypes.createStructField("inputDouble", DataTypes.DoubleType, true)); _inputDataType = DataTypes.createStructType(inputFields); - List bufferFields = new ArrayList(); + List bufferFields = new ArrayList<>(); bufferFields.add(DataTypes.createStructField("bufferDouble", DataTypes.DoubleType, true)); _bufferSchema = DataTypes.createStructType(bufferFields); diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java deleted file mode 100644 index 9e241f20987c..000000000000 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java +++ /dev/null @@ -1,105 +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 test.org.apache.spark.sql.sources; - -import java.io.File; -import java.io.IOException; -import java.util.*; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import org.apache.spark.SparkContext; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -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 static 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 { - SparkContext _sc = new SparkContext("local[*]", "testing"); - sqlContext = new SQLContext(_sc); - sc = new JavaSparkContext(_sc); - - 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.read().json(rdd); - df.registerTempTable("jsonTable"); - } - - @After - public void tearDown() { - sqlContext.sparkContext().stop(); - sqlContext = null; - sc = null; - } - - @Test - public void saveAndLoad() { - Map options = new HashMap<>(); - options.put("path", path.toString()); - df.write().mode(SaveMode.ErrorIfExists).format("json").options(options).save(); - DataFrame loadedDF = sqlContext.read().format("json").options(options).load(); - checkAnswer(loadedDF, df.collectAsList()); - } - - @Test - public void saveAndLoadWithSchema() { - Map options = new HashMap<>(); - options.put("path", path.toString()); - df.write().format("json").mode(SaveMode.ErrorIfExists).options(options).save(); - - List fields = new ArrayList<>(); - fields.add(DataTypes.createStructField("b", DataTypes.StringType, true)); - StructType schema = DataTypes.createStructType(fields); - DataFrame loadedDF = sqlContext.read().format("json").schema(schema).options(options).load(); - - checkAnswer(loadedDF, sqlContext.sql("SELECT b FROM jsonTable").collectAsList()); - } -} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java new file mode 100644 index 000000000000..7aacf0346d2f --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql.sources.v2; + +import java.io.IOException; +import java.util.*; + +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.expressions.GenericRow; +import org.apache.spark.sql.sources.Filter; +import org.apache.spark.sql.sources.GreaterThan; +import org.apache.spark.sql.sources.v2.DataSourceV2; +import org.apache.spark.sql.sources.v2.DataSourceV2Options; +import org.apache.spark.sql.sources.v2.ReadSupport; +import org.apache.spark.sql.sources.v2.reader.*; +import org.apache.spark.sql.types.StructType; + +public class JavaAdvancedDataSourceV2 implements DataSourceV2, ReadSupport { + + class Reader implements DataSourceV2Reader, SupportsPushDownRequiredColumns, + SupportsPushDownFilters { + + private StructType requiredSchema = new StructType().add("i", "int").add("j", "int"); + private Filter[] filters = new Filter[0]; + + @Override + public StructType readSchema() { + return requiredSchema; + } + + @Override + public void pruneColumns(StructType requiredSchema) { + this.requiredSchema = requiredSchema; + } + + @Override + public Filter[] pushFilters(Filter[] filters) { + this.filters = filters; + return new Filter[0]; + } + + @Override + public List> createReadTasks() { + List> res = new ArrayList<>(); + + Integer lowerBound = null; + for (Filter filter : filters) { + if (filter instanceof GreaterThan) { + GreaterThan f = (GreaterThan) filter; + if ("i".equals(f.attribute()) && f.value() instanceof Integer) { + lowerBound = (Integer) f.value(); + break; + } + } + } + + if (lowerBound == null) { + res.add(new JavaAdvancedReadTask(0, 5, requiredSchema)); + res.add(new JavaAdvancedReadTask(5, 10, requiredSchema)); + } else if (lowerBound < 4) { + res.add(new JavaAdvancedReadTask(lowerBound + 1, 5, requiredSchema)); + res.add(new JavaAdvancedReadTask(5, 10, requiredSchema)); + } else if (lowerBound < 9) { + res.add(new JavaAdvancedReadTask(lowerBound + 1, 10, requiredSchema)); + } + + return res; + } + } + + static class JavaAdvancedReadTask implements ReadTask, DataReader { + private int start; + private int end; + private StructType requiredSchema; + + JavaAdvancedReadTask(int start, int end, StructType requiredSchema) { + this.start = start; + this.end = end; + this.requiredSchema = requiredSchema; + } + + @Override + public DataReader createReader() { + return new JavaAdvancedReadTask(start - 1, end, requiredSchema); + } + + @Override + public boolean next() { + start += 1; + return start < end; + } + + @Override + public Row get() { + Object[] values = new Object[requiredSchema.size()]; + for (int i = 0; i < values.length; i++) { + if ("i".equals(requiredSchema.apply(i).name())) { + values[i] = start; + } else if ("j".equals(requiredSchema.apply(i).name())) { + values[i] = -start; + } + } + return new GenericRow(values); + } + + @Override + public void close() throws IOException { + + } + } + + + @Override + public DataSourceV2Reader createReader(DataSourceV2Options options) { + return new Reader(); + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java new file mode 100644 index 000000000000..a174bd8092cb --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql.sources.v2; + +import java.util.List; + +import org.apache.spark.sql.Row; +import org.apache.spark.sql.sources.v2.DataSourceV2; +import org.apache.spark.sql.sources.v2.DataSourceV2Options; +import org.apache.spark.sql.sources.v2.ReadSupportWithSchema; +import org.apache.spark.sql.sources.v2.reader.DataSourceV2Reader; +import org.apache.spark.sql.sources.v2.reader.ReadTask; +import org.apache.spark.sql.types.StructType; + +public class JavaSchemaRequiredDataSource implements DataSourceV2, ReadSupportWithSchema { + + class Reader implements DataSourceV2Reader { + private final StructType schema; + + Reader(StructType schema) { + this.schema = schema; + } + + @Override + public StructType readSchema() { + return schema; + } + + @Override + public List> createReadTasks() { + return java.util.Collections.emptyList(); + } + } + + @Override + public DataSourceV2Reader createReader(StructType schema, DataSourceV2Options options) { + return new Reader(schema); + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java new file mode 100644 index 000000000000..08469f14c257 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql.sources.v2; + +import java.io.IOException; +import java.util.List; + +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.expressions.GenericRow; +import org.apache.spark.sql.sources.v2.DataSourceV2; +import org.apache.spark.sql.sources.v2.DataSourceV2Options; +import org.apache.spark.sql.sources.v2.ReadSupport; +import org.apache.spark.sql.sources.v2.reader.DataReader; +import org.apache.spark.sql.sources.v2.reader.ReadTask; +import org.apache.spark.sql.sources.v2.reader.DataSourceV2Reader; +import org.apache.spark.sql.types.StructType; + +public class JavaSimpleDataSourceV2 implements DataSourceV2, ReadSupport { + + class Reader implements DataSourceV2Reader { + private final StructType schema = new StructType().add("i", "int").add("j", "int"); + + @Override + public StructType readSchema() { + return schema; + } + + @Override + public List> createReadTasks() { + return java.util.Arrays.asList( + new JavaSimpleReadTask(0, 5), + new JavaSimpleReadTask(5, 10)); + } + } + + static class JavaSimpleReadTask implements ReadTask, DataReader { + private int start; + private int end; + + JavaSimpleReadTask(int start, int end) { + this.start = start; + this.end = end; + } + + @Override + public DataReader createReader() { + return new JavaSimpleReadTask(start - 1, end); + } + + @Override + public boolean next() { + start += 1; + return start < end; + } + + @Override + public Row get() { + return new GenericRow(new Object[] {start, -start}); + } + + @Override + public void close() throws IOException { + + } + } + + @Override + public DataSourceV2Reader createReader(DataSourceV2Options options) { + return new Reader(); + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaUnsafeRowDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaUnsafeRowDataSourceV2.java new file mode 100644 index 000000000000..9efe7c791a93 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaUnsafeRowDataSourceV2.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql.sources.v2; + +import java.io.IOException; +import java.util.List; + +import org.apache.spark.sql.catalyst.expressions.UnsafeRow; +import org.apache.spark.sql.sources.v2.DataSourceV2; +import org.apache.spark.sql.sources.v2.DataSourceV2Options; +import org.apache.spark.sql.sources.v2.ReadSupport; +import org.apache.spark.sql.sources.v2.reader.*; +import org.apache.spark.sql.types.StructType; + +public class JavaUnsafeRowDataSourceV2 implements DataSourceV2, ReadSupport { + + class Reader implements DataSourceV2Reader, SupportsScanUnsafeRow { + private final StructType schema = new StructType().add("i", "int").add("j", "int"); + + @Override + public StructType readSchema() { + return schema; + } + + @Override + public List> createUnsafeRowReadTasks() { + return java.util.Arrays.asList( + new JavaUnsafeRowReadTask(0, 5), + new JavaUnsafeRowReadTask(5, 10)); + } + } + + static class JavaUnsafeRowReadTask implements ReadTask, DataReader { + private int start; + private int end; + private UnsafeRow row; + + JavaUnsafeRowReadTask(int start, int end) { + this.start = start; + this.end = end; + this.row = new UnsafeRow(2); + row.pointTo(new byte[8 * 3], 8 * 3); + } + + @Override + public DataReader createReader() { + return new JavaUnsafeRowReadTask(start - 1, end); + } + + @Override + public boolean next() { + start += 1; + return start < end; + } + + @Override + public UnsafeRow get() { + row.setInt(0, start); + row.setInt(1, -start); + return row; + } + + @Override + public void close() throws IOException { + + } + } + + @Override + public DataSourceV2Reader createReader(DataSourceV2Options options) { + return new Reader(); + } +} diff --git a/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index cfd7889b4ac2..c6973bf41d34 100644 --- a/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -1,3 +1,7 @@ org.apache.spark.sql.sources.FakeSourceOne org.apache.spark.sql.sources.FakeSourceTwo org.apache.spark.sql.sources.FakeSourceThree +org.apache.spark.sql.sources.FakeSourceFour +org.apache.fakesource.FakeExternalSourceOne +org.apache.fakesource.FakeExternalSourceTwo +org.apache.fakesource.FakeExternalSourceThree diff --git a/sql/core/src/test/resources/hive-site.xml b/sql/core/src/test/resources/hive-site.xml new file mode 100644 index 000000000000..17297b3e22a7 --- /dev/null +++ b/sql/core/src/test/resources/hive-site.xml @@ -0,0 +1,26 @@ + + + + + + + hive.in.test + true + Internal marker for test. + + diff --git a/sql/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties index 12fb128149d3..2e5cac12952d 100644 --- a/sql/core/src/test/resources/log4j.properties +++ b/sql/core/src/test/resources/log4j.properties @@ -16,13 +16,14 @@ # # Set everything to be logged to the file core/target/unit-tests.log -log4j.rootLogger=DEBUG, CA, FA +log4j.rootLogger=INFO, CA, FA #Console Appender log4j.appender.CA=org.apache.log4j.ConsoleAppender log4j.appender.CA.layout=org.apache.log4j.PatternLayout log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n log4j.appender.CA.Threshold = WARN +log4j.appender.CA.follow = true #File Appender @@ -52,5 +53,5 @@ log4j.additivity.hive.ql.metadata.Hive=false log4j.logger.hive.ql.metadata.Hive=OFF # Parquet related logging -log4j.logger.org.apache.parquet.hadoop=WARN -log4j.logger.org.apache.spark.sql.parquet=INFO +log4j.logger.org.apache.parquet.CorruptStatistics=ERROR +log4j.logger.parquet.CorruptStatistics=ERROR diff --git a/sql/core/src/test/resources/old-repeated.parquet b/sql/core/src/test/resources/old-repeated.parquet deleted file mode 100644 index 213f1a90291b..000000000000 Binary files a/sql/core/src/test/resources/old-repeated.parquet and /dev/null differ diff --git a/sql/core/src/test/resources/sql-tests/inputs/array.sql b/sql/core/src/test/resources/sql-tests/inputs/array.sql new file mode 100644 index 000000000000..984321ab795f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/array.sql @@ -0,0 +1,92 @@ +-- test cases for array functions + +create temporary view data as select * from values + ("one", array(11, 12, 13), array(array(111, 112, 113), array(121, 122, 123))), + ("two", array(21, 22, 23), array(array(211, 212, 213), array(221, 222, 223))) + as data(a, b, c); + +select * from data; + +-- index into array +select a, b[0], b[0] + b[1] from data; + +-- index into array of arrays +select a, c[0][0] + c[0][0 + 1] from data; + + +create temporary view primitive_arrays as select * from values ( + array(true), + array(2Y, 1Y), + array(2S, 1S), + array(2, 1), + array(2L, 1L), + array(9223372036854775809, 9223372036854775808), + array(2.0D, 1.0D), + array(float(2.0), float(1.0)), + array(date '2016-03-14', date '2016-03-13'), + array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000') +) as primitive_arrays( + boolean_array, + tinyint_array, + smallint_array, + int_array, + bigint_array, + decimal_array, + double_array, + float_array, + date_array, + timestamp_array +); + +select * from primitive_arrays; + +-- array_contains on all primitive types: result should alternate between true and false +select + array_contains(boolean_array, true), array_contains(boolean_array, false), + array_contains(tinyint_array, 2Y), array_contains(tinyint_array, 0Y), + array_contains(smallint_array, 2S), array_contains(smallint_array, 0S), + array_contains(int_array, 2), array_contains(int_array, 0), + array_contains(bigint_array, 2L), array_contains(bigint_array, 0L), + array_contains(decimal_array, 9223372036854775809), array_contains(decimal_array, 1), + array_contains(double_array, 2.0D), array_contains(double_array, 0.0D), + array_contains(float_array, float(2.0)), array_contains(float_array, float(0.0)), + array_contains(date_array, date '2016-03-14'), array_contains(date_array, date '2016-01-01'), + array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') +from primitive_arrays; + +-- array_contains on nested arrays +select array_contains(b, 11), array_contains(c, array(111, 112, 113)) from data; + +-- sort_array +select + sort_array(boolean_array), + sort_array(tinyint_array), + sort_array(smallint_array), + sort_array(int_array), + sort_array(bigint_array), + sort_array(decimal_array), + sort_array(double_array), + sort_array(float_array), + sort_array(date_array), + sort_array(timestamp_array) +from primitive_arrays; + +-- sort_array with an invalid string literal for the argument of sort order. +select sort_array(array('b', 'd'), '1'); + +-- sort_array with an invalid null literal casted as boolean for the argument of sort order. +select sort_array(array('b', 'd'), cast(NULL as boolean)); + +-- size +select + size(boolean_array), + size(tinyint_array), + size(smallint_array), + size(int_array), + size(bigint_array), + size(decimal_array), + size(double_array), + size(float_array), + size(date_array), + size(timestamp_array) +from primitive_arrays; diff --git a/sql/core/src/test/resources/sql-tests/inputs/blacklist.sql b/sql/core/src/test/resources/sql-tests/inputs/blacklist.sql new file mode 100644 index 000000000000..d69f8147a526 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/blacklist.sql @@ -0,0 +1,4 @@ +-- This is a query file that has been blacklisted. +-- It includes a query that should crash Spark. +-- If the test case is run, the whole suite would fail. +some random not working query that should crash Spark. diff --git a/sql/core/src/test/resources/sql-tests/inputs/cast.sql b/sql/core/src/test/resources/sql-tests/inputs/cast.sql new file mode 100644 index 000000000000..629df59cff8b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/cast.sql @@ -0,0 +1,45 @@ +-- cast string representing a valid fractional number to integral should truncate the number +SELECT CAST('1.23' AS int); +SELECT CAST('1.23' AS long); +SELECT CAST('-4.56' AS int); +SELECT CAST('-4.56' AS long); + +-- cast string which are not numbers to integral should return null +SELECT CAST('abc' AS int); +SELECT CAST('abc' AS long); + +-- cast string representing a very large number to integral should return null +SELECT CAST('1234567890123' AS int); +SELECT CAST('12345678901234567890123' AS long); + +-- cast empty string to integral should return null +SELECT CAST('' AS int); +SELECT CAST('' AS long); + +-- cast null to integral should return null +SELECT CAST(NULL AS int); +SELECT CAST(NULL AS long); + +-- cast invalid decimal string to integral should return null +SELECT CAST('123.a' AS int); +SELECT CAST('123.a' AS long); + +-- '-2147483648' is the smallest int value +SELECT CAST('-2147483648' AS int); +SELECT CAST('-2147483649' AS int); + +-- '2147483647' is the largest int value +SELECT CAST('2147483647' AS int); +SELECT CAST('2147483648' AS int); + +-- '-9223372036854775808' is the smallest long value +SELECT CAST('-9223372036854775808' AS long); +SELECT CAST('-9223372036854775809' AS long); + +-- '9223372036854775807' is the largest long value +SELECT CAST('9223372036854775807' AS long); +SELECT CAST('9223372036854775808' AS long); + +DESC FUNCTION boolean; +DESC FUNCTION EXTENDED boolean; +-- TODO: migrate all cast tests here. diff --git a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql new file mode 100644 index 000000000000..ad0f885f63d3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql @@ -0,0 +1,55 @@ +-- Create the origin table +CREATE TABLE test_change(a INT, b STRING, c INT) using parquet; +DESC test_change; + +-- Change column name (not supported yet) +ALTER TABLE test_change CHANGE a a1 INT; +DESC test_change; + +-- Change column dataType (not supported yet) +ALTER TABLE test_change CHANGE a a STRING; +DESC test_change; + +-- Change column position (not supported yet) +ALTER TABLE test_change CHANGE a a INT AFTER b; +ALTER TABLE test_change CHANGE b b STRING FIRST; +DESC test_change; + +-- Change column comment +ALTER TABLE test_change CHANGE a a INT COMMENT 'this is column a'; +ALTER TABLE test_change CHANGE b b STRING COMMENT '#*02?`'; +ALTER TABLE test_change CHANGE c c INT COMMENT ''; +DESC test_change; + +-- Don't change anything. +ALTER TABLE test_change CHANGE a a INT COMMENT 'this is column a'; +DESC test_change; + +-- Change a invalid column +ALTER TABLE test_change CHANGE invalid_col invalid_col INT; +DESC test_change; + +-- Change column name/dataType/position/comment together (not supported yet) +ALTER TABLE test_change CHANGE a a1 STRING COMMENT 'this is column a1' AFTER b; +DESC test_change; + +-- Check the behavior with different values of CASE_SENSITIVE +SET spark.sql.caseSensitive=false; +ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A'; +SET spark.sql.caseSensitive=true; +ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A1'; +DESC test_change; + +-- Change column can't apply to a temporary/global_temporary view +CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one"; +ALTER TABLE temp_view CHANGE a a INT COMMENT 'this is column a'; +CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one"; +ALTER TABLE global_temp.global_temp_view CHANGE a a INT COMMENT 'this is column a'; + +-- Change column in partition spec (not supported yet) +CREATE TABLE partition_table(a INT, b STRING, c INT, d STRING) USING parquet PARTITIONED BY (c, d); +ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a INT; + +-- DROP TEST TABLE +DROP TABLE test_change; +DROP TABLE partition_table; diff --git a/sql/core/src/test/resources/sql-tests/inputs/columnresolution-negative.sql b/sql/core/src/test/resources/sql-tests/inputs/columnresolution-negative.sql new file mode 100644 index 000000000000..1caa45c66749 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/columnresolution-negative.sql @@ -0,0 +1,36 @@ +-- Negative testcases for column resolution +CREATE DATABASE mydb1; +USE mydb1; +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1; + +CREATE DATABASE mydb2; +USE mydb2; +CREATE TABLE t1 USING parquet AS SELECT 20 AS i1; + +-- Negative tests: column resolution scenarios with ambiguous cases in join queries +SET spark.sql.crossJoin.enabled = true; +USE mydb1; +SELECT i1 FROM t1, mydb1.t1; +SELECT t1.i1 FROM t1, mydb1.t1; +SELECT mydb1.t1.i1 FROM t1, mydb1.t1; +SELECT i1 FROM t1, mydb2.t1; +SELECT t1.i1 FROM t1, mydb2.t1; +USE mydb2; +SELECT i1 FROM t1, mydb1.t1; +SELECT t1.i1 FROM t1, mydb1.t1; +SELECT i1 FROM t1, mydb2.t1; +SELECT t1.i1 FROM t1, mydb2.t1; +SELECT db1.t1.i1 FROM t1, mydb2.t1; +SET spark.sql.crossJoin.enabled = false; + +-- Negative tests +USE mydb1; +SELECT mydb1.t1 FROM t1; +SELECT t1.x.y.* FROM t1; +SELECT t1 FROM mydb1.t1; +USE mydb2; +SELECT mydb1.t1.i1 FROM t1; + +-- reset +DROP DATABASE mydb1 CASCADE; +DROP DATABASE mydb2 CASCADE; diff --git a/sql/core/src/test/resources/sql-tests/inputs/columnresolution-views.sql b/sql/core/src/test/resources/sql-tests/inputs/columnresolution-views.sql new file mode 100644 index 000000000000..d3f928751757 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/columnresolution-views.sql @@ -0,0 +1,25 @@ +-- Tests for qualified column names for the view code-path +-- Test scenario with Temporary view +CREATE OR REPLACE TEMPORARY VIEW view1 AS SELECT 2 AS i1; +SELECT view1.* FROM view1; +SELECT * FROM view1; +SELECT view1.i1 FROM view1; +SELECT i1 FROM view1; +SELECT a.i1 FROM view1 AS a; +SELECT i1 FROM view1 AS a; +-- cleanup +DROP VIEW view1; + +-- Test scenario with Global Temp view +CREATE OR REPLACE GLOBAL TEMPORARY VIEW view1 as SELECT 1 as i1; +SELECT * FROM global_temp.view1; +-- TODO: Support this scenario +SELECT global_temp.view1.* FROM global_temp.view1; +SELECT i1 FROM global_temp.view1; +-- TODO: Support this scenario +SELECT global_temp.view1.i1 FROM global_temp.view1; +SELECT view1.i1 FROM global_temp.view1; +SELECT a.i1 FROM global_temp.view1 AS a; +SELECT i1 FROM global_temp.view1 AS a; +-- cleanup +DROP VIEW global_temp.view1; diff --git a/sql/core/src/test/resources/sql-tests/inputs/columnresolution.sql b/sql/core/src/test/resources/sql-tests/inputs/columnresolution.sql new file mode 100644 index 000000000000..79e90ad3de91 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/columnresolution.sql @@ -0,0 +1,88 @@ +-- Tests covering different scenarios with qualified column names +-- Scenario: column resolution scenarios with datasource table +CREATE DATABASE mydb1; +USE mydb1; +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1; + +CREATE DATABASE mydb2; +USE mydb2; +CREATE TABLE t1 USING parquet AS SELECT 20 AS i1; + +USE mydb1; +SELECT i1 FROM t1; +SELECT i1 FROM mydb1.t1; +SELECT t1.i1 FROM t1; +SELECT t1.i1 FROM mydb1.t1; + +-- TODO: Support this scenario +SELECT mydb1.t1.i1 FROM t1; +-- TODO: Support this scenario +SELECT mydb1.t1.i1 FROM mydb1.t1; + +USE mydb2; +SELECT i1 FROM t1; +SELECT i1 FROM mydb1.t1; +SELECT t1.i1 FROM t1; +SELECT t1.i1 FROM mydb1.t1; +-- TODO: Support this scenario +SELECT mydb1.t1.i1 FROM mydb1.t1; + +-- Scenario: resolve fully qualified table name in star expansion +USE mydb1; +SELECT t1.* FROM t1; +SELECT mydb1.t1.* FROM mydb1.t1; +SELECT t1.* FROM mydb1.t1; +USE mydb2; +SELECT t1.* FROM t1; +-- TODO: Support this scenario +SELECT mydb1.t1.* FROM mydb1.t1; +SELECT t1.* FROM mydb1.t1; +SELECT a.* FROM mydb1.t1 AS a; + +-- Scenario: resolve in case of subquery + +USE mydb1; +CREATE TABLE t3 USING parquet AS SELECT * FROM VALUES (4,1), (3,1) AS t3(c1, c2); +CREATE TABLE t4 USING parquet AS SELECT * FROM VALUES (4,1), (2,1) AS t4(c2, c3); + +SELECT * FROM t3 WHERE c1 IN (SELECT c2 FROM t4 WHERE t4.c3 = t3.c2); + +-- TODO: Support this scenario +SELECT * FROM mydb1.t3 WHERE c1 IN + (SELECT mydb1.t4.c2 FROM mydb1.t4 WHERE mydb1.t4.c3 = mydb1.t3.c2); + +-- Scenario: column resolution scenarios in join queries +SET spark.sql.crossJoin.enabled = true; + +-- TODO: Support this scenario +SELECT mydb1.t1.i1 FROM t1, mydb2.t1; + +-- TODO: Support this scenario +SELECT mydb1.t1.i1 FROM mydb1.t1, mydb2.t1; + +USE mydb2; +-- TODO: Support this scenario +SELECT mydb1.t1.i1 FROM t1, mydb1.t1; +SET spark.sql.crossJoin.enabled = false; + +-- Scenario: Table with struct column +USE mydb1; +CREATE TABLE t5(i1 INT, t5 STRUCT) USING parquet; +INSERT INTO t5 VALUES(1, (2, 3)); +SELECT t5.i1 FROM t5; +SELECT t5.t5.i1 FROM t5; +SELECT t5.t5.i1 FROM mydb1.t5; +SELECT t5.i1 FROM mydb1.t5; +SELECT t5.* FROM mydb1.t5; +SELECT t5.t5.* FROM mydb1.t5; +-- TODO: Support this scenario +SELECT mydb1.t5.t5.i1 FROM mydb1.t5; +-- TODO: Support this scenario +SELECT mydb1.t5.t5.i2 FROM mydb1.t5; +-- TODO: Support this scenario +SELECT mydb1.t5.* FROM mydb1.t5; + +-- Cleanup and Reset +USE default; +DROP DATABASE mydb1 CASCADE; +DROP DATABASE mydb2 CASCADE; diff --git a/sql/core/src/test/resources/sql-tests/inputs/comparator.sql b/sql/core/src/test/resources/sql-tests/inputs/comparator.sql new file mode 100644 index 000000000000..3e2447723e57 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/comparator.sql @@ -0,0 +1,3 @@ +-- binary type +select x'00' < x'0f'; +select x'00' < x'ff'; diff --git a/sql/core/src/test/resources/sql-tests/inputs/cross-join.sql b/sql/core/src/test/resources/sql-tests/inputs/cross-join.sql new file mode 100644 index 000000000000..b64197e2bc70 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/cross-join.sql @@ -0,0 +1,36 @@ +-- Cross join detection and error checking is done in JoinSuite since explain output is +-- used in the error message and the ids are not stable. Only positive cases are checked here. + +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1); + +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2); + +-- Cross joins with and without predicates +SELECT * FROM nt1 cross join nt2; +SELECT * FROM nt1 cross join nt2 where nt1.k = nt2.k; +SELECT * FROM nt1 cross join nt2 on (nt1.k = nt2.k); +SELECT * FROM nt1 cross join nt2 where nt1.v1 = 1 and nt2.v2 = 22; + +SELECT a.key, b.key FROM +(SELECT k key FROM nt1 WHERE v1 < 2) a +CROSS JOIN +(SELECT k key FROM nt2 WHERE v2 = 22) b; + +-- Join reordering +create temporary view A(a, va) as select * from nt1; +create temporary view B(b, vb) as select * from nt1; +create temporary view C(c, vc) as select * from nt1; +create temporary view D(d, vd) as select * from nt1; + +-- Allowed since cross join with C is explicit +select * from ((A join B on (a = b)) cross join C) join D on (a = d); +-- Cross joins with non-equal predicates +SELECT * FROM nt1 CROSS JOIN nt2 ON (nt1.k > nt2.k); diff --git a/sql/core/src/test/resources/sql-tests/inputs/cte.sql b/sql/core/src/test/resources/sql-tests/inputs/cte.sql new file mode 100644 index 000000000000..d34d89f23575 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/cte.sql @@ -0,0 +1,29 @@ +create temporary view t as select * from values 0, 1, 2 as t(id); +create temporary view t2 as select * from values 0, 1 as t(id); + +-- WITH clause should not fall into infinite loop by referencing self +WITH s AS (SELECT 1 FROM s) SELECT * FROM s; + +-- WITH clause should reference the base table +WITH t AS (SELECT 1 FROM t) SELECT * FROM t; + +-- WITH clause should not allow cross reference +WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2; + +-- WITH clause should reference the previous CTE +WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1 cross join t2; + +-- SPARK-18609 CTE with self-join +WITH CTE1 AS ( + SELECT b.id AS id + FROM T2 a + CROSS JOIN (SELECT id AS id FROM T2) b +) +SELECT t1.id AS c1, + t2.id AS c2 +FROM CTE1 t1 + CROSS JOIN CTE1 t2; + +-- Clean up +DROP VIEW IF EXISTS t; +DROP VIEW IF EXISTS t2; diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql new file mode 100644 index 000000000000..616b6caee3f2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -0,0 +1,10 @@ +-- date time functions + +-- [SPARK-16836] current_date and current_timestamp literals +select current_date = current_date(), current_timestamp = current_timestamp(); + +select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd'); + +select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd'); + +select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), dayofweek('1582-10-15 13:10:15'); diff --git a/sql/core/src/test/resources/sql-tests/inputs/describe-part-after-analyze.sql b/sql/core/src/test/resources/sql-tests/inputs/describe-part-after-analyze.sql new file mode 100644 index 000000000000..f4239da90627 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/describe-part-after-analyze.sql @@ -0,0 +1,34 @@ +CREATE TABLE t (key STRING, value STRING, ds STRING, hr INT) USING parquet + PARTITIONED BY (ds, hr); + +INSERT INTO TABLE t PARTITION (ds='2017-08-01', hr=10) +VALUES ('k1', 100), ('k2', 200), ('k3', 300); + +INSERT INTO TABLE t PARTITION (ds='2017-08-01', hr=11) +VALUES ('k1', 101), ('k2', 201), ('k3', 301), ('k4', 401); + +INSERT INTO TABLE t PARTITION (ds='2017-09-01', hr=5) +VALUES ('k1', 102), ('k2', 202); + +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10); + +-- Collect stats for a single partition +ANALYZE TABLE t PARTITION (ds='2017-08-01', hr=10) COMPUTE STATISTICS; + +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10); + +-- Collect stats for 2 partitions +ANALYZE TABLE t PARTITION (ds='2017-08-01') COMPUTE STATISTICS; + +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10); +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11); + +-- Collect stats for all partitions +ANALYZE TABLE t PARTITION (ds, hr) COMPUTE STATISTICS; + +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10); +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11); +DESC EXTENDED t PARTITION (ds='2017-09-01', hr=5); + +-- DROP TEST TABLES/VIEWS +DROP TABLE t; diff --git a/sql/core/src/test/resources/sql-tests/inputs/describe-table-after-alter-table.sql b/sql/core/src/test/resources/sql-tests/inputs/describe-table-after-alter-table.sql new file mode 100644 index 000000000000..69bff6656c43 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/describe-table-after-alter-table.sql @@ -0,0 +1,29 @@ +CREATE TABLE table_with_comment (a STRING, b INT, c STRING, d STRING) USING parquet COMMENT 'added'; + +DESC FORMATTED table_with_comment; + +-- ALTER TABLE BY MODIFYING COMMENT +ALTER TABLE table_with_comment SET TBLPROPERTIES("comment"= "modified comment", "type"= "parquet"); + +DESC FORMATTED table_with_comment; + +-- DROP TEST TABLE +DROP TABLE table_with_comment; + +-- CREATE TABLE WITHOUT COMMENT +CREATE TABLE table_comment (a STRING, b INT) USING parquet; + +DESC FORMATTED table_comment; + +-- ALTER TABLE BY ADDING COMMENT +ALTER TABLE table_comment SET TBLPROPERTIES(comment = "added comment"); + +DESC formatted table_comment; + +-- ALTER UNSET PROPERTIES COMMENT +ALTER TABLE table_comment UNSET TBLPROPERTIES IF EXISTS ('comment'); + +DESC FORMATTED table_comment; + +-- DROP TEST TABLE +DROP TABLE table_comment; diff --git a/sql/core/src/test/resources/sql-tests/inputs/describe-table-column.sql b/sql/core/src/test/resources/sql-tests/inputs/describe-table-column.sql new file mode 100644 index 000000000000..a6ddcd999bf9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/describe-table-column.sql @@ -0,0 +1,41 @@ +-- Test temp table +CREATE TEMPORARY VIEW desc_col_temp_view (key int COMMENT 'column_comment') USING PARQUET; + +DESC desc_col_temp_view key; + +DESC EXTENDED desc_col_temp_view key; + +DESC FORMATTED desc_col_temp_view key; + +-- Describe a column with qualified name +DESC FORMATTED desc_col_temp_view desc_col_temp_view.key; + +-- Describe a non-existent column +DESC desc_col_temp_view key1; + +-- Test persistent table +CREATE TABLE desc_col_table (key int COMMENT 'column_comment') USING PARQUET; + +ANALYZE TABLE desc_col_table COMPUTE STATISTICS FOR COLUMNS key; + +DESC desc_col_table key; + +DESC EXTENDED desc_col_table key; + +DESC FORMATTED desc_col_table key; + +-- Test complex columns +CREATE TABLE desc_complex_col_table (`a.b` int, col struct) USING PARQUET; + +DESC FORMATTED desc_complex_col_table `a.b`; + +DESC FORMATTED desc_complex_col_table col; + +-- Describe a nested column +DESC FORMATTED desc_complex_col_table col.x; + +DROP VIEW desc_col_temp_view; + +DROP TABLE desc_col_table; + +DROP TABLE desc_complex_col_table; diff --git a/sql/core/src/test/resources/sql-tests/inputs/describe.sql b/sql/core/src/test/resources/sql-tests/inputs/describe.sql new file mode 100644 index 000000000000..f26d5efec076 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/describe.sql @@ -0,0 +1,90 @@ +CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet + OPTIONS (a '1', b '2') + PARTITIONED BY (c, d) CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS + COMMENT 'table_comment' + TBLPROPERTIES (t 'test'); + +CREATE TEMPORARY VIEW temp_v AS SELECT * FROM t; + +CREATE TEMPORARY VIEW temp_Data_Source_View + USING org.apache.spark.sql.sources.DDLScanSource + OPTIONS ( + From '1', + To '10', + Table 'test1'); + +CREATE VIEW v AS SELECT * FROM t; + +ALTER TABLE t SET TBLPROPERTIES (e = '3'); + +ALTER TABLE t ADD PARTITION (c='Us', d=1); + +DESCRIBE t; + +DESC default.t; + +DESC TABLE t; + +DESC FORMATTED t; + +DESC EXTENDED t; + +ALTER TABLE t UNSET TBLPROPERTIES (e); + +DESC EXTENDED t; + +ALTER TABLE t UNSET TBLPROPERTIES (comment); + +DESC EXTENDED t; + +DESC t PARTITION (c='Us', d=1); + +DESC EXTENDED t PARTITION (c='Us', d=1); + +DESC FORMATTED t PARTITION (c='Us', d=1); + +-- NoSuchPartitionException: Partition not found in table +DESC t PARTITION (c='Us', d=2); + +-- AnalysisException: Partition spec is invalid +DESC t PARTITION (c='Us'); + +-- ParseException: PARTITION specification is incomplete +DESC t PARTITION (c='Us', d); + +-- DESC Temp View + +DESC temp_v; + +DESC TABLE temp_v; + +DESC FORMATTED temp_v; + +DESC EXTENDED temp_v; + +DESC temp_Data_Source_View; + +-- AnalysisException DESC PARTITION is not allowed on a temporary view +DESC temp_v PARTITION (c='Us', d=1); + +-- DESC Persistent View + +DESC v; + +DESC TABLE v; + +DESC FORMATTED v; + +DESC EXTENDED v; + +-- AnalysisException DESC PARTITION is not allowed on a view +DESC v PARTITION (c='Us', d=1); + +-- DROP TEST TABLES/VIEWS +DROP TABLE t; + +DROP VIEW temp_v; + +DROP VIEW temp_Data_Source_View; + +DROP VIEW v; diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-analytics.sql b/sql/core/src/test/resources/sql-tests/inputs/group-analytics.sql new file mode 100644 index 000000000000..8aff4cb52419 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/group-analytics.sql @@ -0,0 +1,62 @@ +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b); + +-- CUBE on overlapping columns +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH CUBE; + +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH CUBE; + +-- ROLLUP on overlapping columns +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP; + +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH ROLLUP; + +CREATE OR REPLACE TEMPORARY VIEW courseSales AS SELECT * FROM VALUES +("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) +AS courseSales(course, year, earnings); + +-- ROLLUP +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY course, year; + +-- CUBE +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, year; + +-- GROUPING SETS +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year); +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course); +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year); + +-- GROUPING SETS with aggregate functions containing groupBy columns +SELECT course, SUM(earnings) AS sum FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum; +SELECT course, SUM(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum; + +-- GROUPING/GROUPING_ID +SELECT course, year, GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales +GROUP BY CUBE(course, year); +SELECT course, year, GROUPING(course) FROM courseSales GROUP BY course, year; +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY course, year; +SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year); + +-- GROUPING/GROUPING_ID in having clause +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) +HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0; +SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING(course) > 0; +SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0; +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0; + +-- GROUPING/GROUPING_ID in orderBy clause +SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, year; +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, year; +SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING(course); +SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING_ID(course); +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id; + +-- Aliases in SELECT could be used in ROLLUP/CUBE/GROUPING SETS +SELECT a + b AS k1, b AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2); +SELECT a + b AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b); +SELECT a + b, b AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k) diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by-ordinal.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by-ordinal.sql new file mode 100644 index 000000000000..928f766b4add --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by-ordinal.sql @@ -0,0 +1,59 @@ +-- group by ordinal positions + +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b); + +-- basic case +select a, sum(b) from data group by 1; + +-- constant case +select 1, 2, sum(b) from data group by 1, 2; + +-- duplicate group by column +select a, 1, sum(b) from data group by a, 1; +select a, 1, sum(b) from data group by 1, 2; + +-- group by a non-aggregate expression's ordinal +select a, b + 2, count(2) from data group by a, 2; + +-- with alias +select a as aa, b + 2 as bb, count(2) from data group by 1, 2; + +-- foldable non-literal: this should be the same as no grouping. +select sum(b) from data group by 1 + 0; + +-- negative cases: ordinal out of range +select a, b from data group by -1; +select a, b from data group by 0; +select a, b from data group by 3; + +-- negative case: position is an aggregate expression +select a, b, sum(b) from data group by 3; +select a, b, sum(b) + 2 from data group by 3; + +-- negative case: nondeterministic expression +select a, rand(0), sum(b) from data group by a, 2; + +-- negative case: star +select * from data group by a, b, 1; + +-- group by ordinal followed by order by +select a, count(a) from (select 1 as a) tmp group by 1 order by 1; + +-- group by ordinal followed by having +select count(a), a from (select 1 as a) tmp group by 2 having a > 0; + +-- mixed cases: group-by ordinals and aliases +select a, a AS k, count(b) from data group by k, 1; + +-- turn off group by ordinal +set spark.sql.groupByOrdinal=false; + +-- can now group by negative literal +select sum(b) from data group by -1; diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql new file mode 100644 index 000000000000..1e1384549a41 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -0,0 +1,62 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); + +-- Aggregate with empty GroupBy expressions. +SELECT a, COUNT(b) FROM testData; +SELECT COUNT(a), COUNT(b) FROM testData; + +-- Aggregate with non-empty GroupBy expressions. +SELECT a, COUNT(b) FROM testData GROUP BY a; +SELECT a, COUNT(b) FROM testData GROUP BY b; +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a; + +-- Aggregate grouped by literals. +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1; + +-- Aggregate grouped by literals (whole stage code generation). +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate grouped by literals (hash aggregate). +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate grouped by literals (sort aggregate). +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate with complex GroupBy expressions. +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b; +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1; +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1; + +-- Aggregate with nulls. +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData; + +-- Aggregate with foldable input and multiple distinct groups. +SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; + +-- Aliases in SELECT could be used in GROUP BY +SELECT a AS k, COUNT(b) FROM testData GROUP BY k; +SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1; + +-- Aggregate functions cannot be used in GROUP BY +SELECT COUNT(b) AS k FROM testData GROUP BY k; + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v); +SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a; + +-- turn off group by aliases +set spark.sql.groupByAliases=false; + +-- Check analysis exceptions +SELECT a AS k, COUNT(b) FROM testData GROUP BY k; + +-- Aggregate with empty input and non-empty GroupBy expressions. +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a; + +-- Aggregate with empty input and empty GroupBy expressions. +SELECT COUNT(1) FROM testData WHERE false; +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t; diff --git a/sql/core/src/test/resources/sql-tests/inputs/grouping_set.sql b/sql/core/src/test/resources/sql-tests/inputs/grouping_set.sql new file mode 100644 index 000000000000..359428350528 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/grouping_set.sql @@ -0,0 +1,17 @@ +CREATE TEMPORARY VIEW grouping AS SELECT * FROM VALUES + ("1", "2", "3", 1), + ("4", "5", "6", 1), + ("7", "8", "9", 1) + as grouping(a, b, c, d); + +-- SPARK-17849: grouping set throws NPE #1 +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS (()); + +-- SPARK-17849: grouping set throws NPE #2 +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((a)); + +-- SPARK-17849: grouping set throws NPE #3 +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((c)); + + + diff --git a/sql/core/src/test/resources/sql-tests/inputs/having.sql b/sql/core/src/test/resources/sql-tests/inputs/having.sql new file mode 100644 index 000000000000..868a911e787f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/having.sql @@ -0,0 +1,18 @@ +create temporary view hav as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v); + +-- having clause +SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2; + +-- having condition contains grouping column +SELECT count(k) FROM hav GROUP BY v + 1 HAVING v + 1 = 2; + +-- SPARK-11032: resolve having correctly +SELECT MIN(t.v) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(COUNT(1) > 0); + +-- SPARK-20329: make sure we handle timezones correctly +SELECT a + b FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > 1; diff --git a/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql b/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql new file mode 100644 index 000000000000..41d316444ed6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql @@ -0,0 +1,54 @@ + +-- single row, without table and column alias +select * from values ("one", 1); + +-- single row, without column alias +select * from values ("one", 1) as data; + +-- single row +select * from values ("one", 1) as data(a, b); + +-- single column multiple rows +select * from values 1, 2, 3 as data(a); + +-- three rows +select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b); + +-- null type +select * from values ("one", null), ("two", null) as data(a, b); + +-- int and long coercion +select * from values ("one", 1), ("two", 2L) as data(a, b); + +-- foldable expressions +select * from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b); + +-- complex types +select * from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b); + +-- decimal and double coercion +select * from values ("one", 2.0), ("two", 3.0D) as data(a, b); + +-- error reporting: nondeterministic function rand +select * from values ("one", rand(5)), ("two", 3.0D) as data(a, b); + +-- error reporting: different number of columns +select * from values ("one", 2.0), ("two") as data(a, b); + +-- error reporting: types that are incompatible +select * from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b); + +-- error reporting: number aliases different from number data values +select * from values ("one"), ("two") as data(a, b); + +-- error reporting: unresolved expression +select * from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b); + +-- error reporting: aggregate expression +select * from values ("one", count(1)), ("two", 2) as data(a, b); + +-- string to timestamp +select * from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b); + +-- cross-join inline tables +EXPLAIN EXTENDED SELECT * FROM VALUES ('one', 1), ('three', null) CROSS JOIN VALUES ('one', 1), ('three', null); diff --git a/sql/core/src/test/resources/sql-tests/inputs/inner-join.sql b/sql/core/src/test/resources/sql-tests/inputs/inner-join.sql new file mode 100644 index 000000000000..38739cb95058 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/inner-join.sql @@ -0,0 +1,17 @@ +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a); + +CREATE TEMPORARY VIEW ta AS +SELECT a, 'a' AS tag FROM t1 +UNION ALL +SELECT a, 'b' AS tag FROM t2; + +CREATE TEMPORARY VIEW tb AS +SELECT a, 'a' AS tag FROM t3 +UNION ALL +SELECT a, 'b' AS tag FROM t4; + +-- SPARK-19766 Constant alias columns in INNER JOIN should not be folded by FoldablePropagation rule +SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag; diff --git a/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql new file mode 100644 index 000000000000..fea069eac4d4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql @@ -0,0 +1,33 @@ +-- to_json +describe function to_json; +describe function extended to_json; +select to_json(named_struct('a', 1, 'b', 2)); +select to_json(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')); +select to_json(array(named_struct('a', 1, 'b', 2))); +select to_json(map(named_struct('a', 1, 'b', 2), named_struct('a', 1, 'b', 2))); +select to_json(map('a', named_struct('a', 1, 'b', 2))); +select to_json(map('a', 1)); +select to_json(array(map('a',1))); +select to_json(array(map('a',1), map('b',2))); +-- Check if errors handled +select to_json(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')); +select to_json(named_struct('a', 1, 'b', 2), map('mode', 1)); +select to_json(); + +-- from_json +describe function from_json; +describe function extended from_json; +select from_json('{"a":1}', 'a INT'); +select from_json('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')); +-- Check if errors handled +select from_json('{"a":1}', 1); +select from_json('{"a":1}', 'a InvalidType'); +select from_json('{"a":1}', 'a INT', named_struct('mode', 'PERMISSIVE')); +select from_json('{"a":1}', 'a INT', map('mode', 1)); +select from_json(); +-- json_tuple +SELECT json_tuple('{"a" : 1, "b" : 2}', CAST(NULL AS STRING), 'b', CAST(NULL AS STRING), 'a'); +CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a'); +SELECT json_tuple(jsonField, 'b', CAST(NULL AS STRING), a) FROM jsonTable; +-- Clean up +DROP VIEW IF EXISTS jsonTable; diff --git a/sql/core/src/test/resources/sql-tests/inputs/limit.sql b/sql/core/src/test/resources/sql-tests/inputs/limit.sql new file mode 100644 index 000000000000..f21912a04271 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/limit.sql @@ -0,0 +1,27 @@ + +-- limit on various data types +SELECT * FROM testdata LIMIT 2; +SELECT * FROM arraydata LIMIT 2; +SELECT * FROM mapdata LIMIT 2; + +-- foldable non-literal in limit +SELECT * FROM testdata LIMIT 2 + 1; + +SELECT * FROM testdata LIMIT CAST(1 AS int); + +-- limit must be non-negative +SELECT * FROM testdata LIMIT -1; +SELECT * FROM testData TABLESAMPLE (-1 ROWS); + +-- limit must be foldable +SELECT * FROM testdata LIMIT key > 3; + +-- limit must be integer +SELECT * FROM testdata LIMIT true; +SELECT * FROM testdata LIMIT 'a'; + +-- limit within a subquery +SELECT * FROM (SELECT * FROM range(10) LIMIT 5) WHERE id > 3; + +-- limit ALL +SELECT * FROM testdata WHERE key < 3 LIMIT ALL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/literals.sql new file mode 100644 index 000000000000..37b4b7606d12 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/literals.sql @@ -0,0 +1,107 @@ +-- Literal parsing + +-- null +select null, Null, nUll; + +-- boolean +select true, tRue, false, fALse; + +-- byte (tinyint) +select 1Y; +select 127Y, -128Y; + +-- out of range byte +select 128Y; + +-- short (smallint) +select 1S; +select 32767S, -32768S; + +-- out of range short +select 32768S; + +-- long (bigint) +select 1L, 2147483648L; +select 9223372036854775807L, -9223372036854775808L; + +-- out of range long +select 9223372036854775808L; + +-- integral parsing + +-- parse int +select 1, -1; + +-- parse int max and min value as int +select 2147483647, -2147483648; + +-- parse long max and min value as long +select 9223372036854775807, -9223372036854775808; + +-- parse as decimals (Long.MaxValue + 1, and Long.MinValue - 1) +select 9223372036854775808, -9223372036854775809; + +-- out of range decimal numbers +select 1234567890123456789012345678901234567890; +select 1234567890123456789012345678901234567890.0; + +-- double +select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1; +select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5; +-- negative double +select .e3; +-- very large decimals (overflowing double). +select 1E309, -1E309; + +-- decimal parsing +select 0.3, -0.8, .5, -.18, 0.1111, .1111; + +-- super large scientific notation double literals should still be valid doubles +select 123456789012345678901234567890123456789e10d, 123456789012345678901234567890123456789.1e10d; + +-- string +select "Hello Peter!", 'hello lee!'; +-- multi string +select 'hello' 'world', 'hello' " " 'lee'; +-- single quote within double quotes +select "hello 'peter'"; +select 'pattern%', 'no-pattern\%', 'pattern\\%', 'pattern\\\%'; +select '\'', '"', '\n', '\r', '\t', 'Z'; +-- "Hello!" in octals +select '\110\145\154\154\157\041'; +-- "World :)" in unicode +select '\u0057\u006F\u0072\u006C\u0064\u0020\u003A\u0029'; + +-- date +select dAte '2016-03-12'; +-- invalid date +select date 'mar 11 2016'; + +-- timestamp +select tImEstAmp '2016-03-11 20:54:00.000'; +-- invalid timestamp +select timestamp '2016-33-11 20:54:00.000'; + +-- interval +select interval 13.123456789 seconds, interval -13.123456789 second; +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond, 9 microsecond; +-- ns is not supported +select interval 10 nanoseconds; + +-- unsupported data type +select GEO '(10,-6)'; + +-- big decimal parsing +select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD; + +-- out of range big decimal +select 1.20E-38BD; + +-- hexadecimal binary literal +select x'2379ACFe'; + +-- invalid hexadecimal binary literal +select X'XuZ'; + +-- Hive literal_double test. +SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8; diff --git a/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql b/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql new file mode 100644 index 000000000000..71a50157b766 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/natural-join.sql @@ -0,0 +1,20 @@ +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1); + +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2); + + +SELECT * FROM nt1 natural join nt2 where k = "one"; + +SELECT * FROM nt1 natural left join nt2 order by v1, v2; + +SELECT * FROM nt1 natural right join nt2 order by v1, v2; + +SELECT count(*) FROM nt1 natural full outer join nt2; diff --git a/sql/core/src/test/resources/sql-tests/inputs/null-propagation.sql b/sql/core/src/test/resources/sql-tests/inputs/null-propagation.sql new file mode 100644 index 000000000000..66549da7971d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/null-propagation.sql @@ -0,0 +1,9 @@ + +-- count(null) should be 0 +SELECT COUNT(NULL) FROM VALUES 1, 2, 3; +SELECT COUNT(1 + NULL) FROM VALUES 1, 2, 3; + +-- count(null) on window should be 0 +SELECT COUNT(NULL) OVER () FROM VALUES 1, 2, 3; +SELECT COUNT(1 + NULL) OVER () FROM VALUES 1, 2, 3; + diff --git a/sql/core/src/test/resources/sql-tests/inputs/operators.sql b/sql/core/src/test/resources/sql-tests/inputs/operators.sql new file mode 100644 index 000000000000..15d981985c55 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/operators.sql @@ -0,0 +1,98 @@ + +-- unary minus and plus +select -100; +select +230; +select -5.2; +select +6.8e0; +select -key, +key from testdata where key = 2; +select -(key + 1), - key + 1, +(key + 5) from testdata where key = 1; +select -max(key), +max(key) from testdata; +select - (-10); +select + (-key) from testdata where key = 32; +select - (+max(key)) from testdata; +select - - 3; +select - + 20; +select + + 100; +select - - max(key) from testdata; +select + - key from testdata where key = 33; + +-- div +select 5 / 2; +select 5 / 0; +select 5 / null; +select null / 5; +select 5 div 2; +select 5 div 0; +select 5 div null; +select null div 5; + +-- other arithmetics +select 1 + 2; +select 1 - 2; +select 2 * 5; +select 5 % 3; +select pmod(-7, 3); + +-- check operator precedence. +-- We follow Oracle operator precedence in the table below that lists the levels of precedence +-- among SQL operators from high to low: +------------------------------------------------------------------------------------------ +-- Operator Operation +------------------------------------------------------------------------------------------ +-- +, - identity, negation +-- *, / multiplication, division +-- +, -, || addition, subtraction, concatenation +-- =, !=, <, >, <=, >=, IS NULL, LIKE, BETWEEN, IN comparison +-- NOT exponentiation, logical negation +-- AND conjunction +-- OR disjunction +------------------------------------------------------------------------------------------ +explain select 'a' || 1 + 2; +explain select 1 - 2 || 'b'; +explain select 2 * 4 + 3 || 'b'; +explain select 3 + 1 || 'a' || 4 / 2; +explain select 1 == 1 OR 'a' || 'b' == 'ab'; +explain select 'a' || 'c' == 'ac' AND 2 == 3; + +-- math functions +select cot(1); +select cot(null); +select cot(0); +select cot(-1); + +-- ceil and ceiling +select ceiling(0); +select ceiling(1); +select ceil(1234567890123456); +select ceiling(1234567890123456); +select ceil(0.01); +select ceiling(-0.10); + +-- floor +select floor(0); +select floor(1); +select floor(1234567890123456); +select floor(0.01); +select floor(-0.10); + +-- comparison operator +select 1 > 0.00001; + +-- mod +select mod(7, 2), mod(7, 0), mod(0, 2), mod(7, null), mod(null, 2), mod(null, null); + +-- length +select BIT_LENGTH('abc'); +select CHAR_LENGTH('abc'); +select CHARACTER_LENGTH('abc'); +select OCTET_LENGTH('abc'); + +-- abs +select abs(-3.13), abs('-2.19'); + +-- positive/negative +select positive('-1.11'), positive(-1.11), negative('-1.11'), negative(-1.11); + +-- pmod +select pmod(-7, 2), pmod(0, 2), pmod(7, 0), pmod(7, null), pmod(null, 2), pmod(null, null); +select pmod(cast(3.13 as decimal), cast(0 as decimal)), pmod(cast(2 as smallint), cast(0 as smallint)); diff --git a/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql b/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql new file mode 100644 index 000000000000..f7637b444b9f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql @@ -0,0 +1,83 @@ +-- Q1. testing window functions with order by +create table spark_10747(col1 int, col2 int, col3 int) using parquet; + +-- Q2. insert to tables +INSERT INTO spark_10747 VALUES (6, 12, 10), (6, 11, 4), (6, 9, 10), (6, 15, 8), +(6, 15, 8), (6, 7, 4), (6, 7, 8), (6, 13, null), (6, 10, null); + +-- Q3. windowing with order by DESC NULLS LAST +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 desc nulls last, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2; + +-- Q4. windowing with order by DESC NULLS FIRST +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 desc nulls first, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2; + +-- Q5. windowing with order by ASC NULLS LAST +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 asc nulls last, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2; + +-- Q6. windowing with order by ASC NULLS FIRST +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 asc nulls first, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2; + +-- Q7. Regular query with ORDER BY ASC NULLS FIRST +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 ASC NULLS FIRST, COL2; + +-- Q8. Regular query with ORDER BY ASC NULLS LAST +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 NULLS LAST, COL2; + +-- Q9. Regular query with ORDER BY DESC NULLS FIRST +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS FIRST, COL2; + +-- Q10. Regular query with ORDER BY DESC NULLS LAST +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS LAST, COL2; + +-- drop the test table +drop table spark_10747; + +-- Q11. mix datatype for ORDER BY NULLS FIRST|LAST +create table spark_10747_mix( +col1 string, +col2 int, +col3 double, +col4 decimal(10,2), +col5 decimal(20,1)) +using parquet; + +-- Q12. Insert to the table +INSERT INTO spark_10747_mix VALUES +('b', 2, 1.0, 1.00, 10.0), +('d', 3, 2.0, 3.00, 0.0), +('c', 3, 2.0, 2.00, 15.1), +('d', 3, 0.0, 3.00, 1.0), +(null, 3, 0.0, 3.00, 1.0), +('d', 3, null, 4.00, 1.0), +('a', 1, 1.0, 1.00, null), +('c', 3, 2.0, 2.00, null); + +-- Q13. Regular query with 2 NULLS LAST columns +select * from spark_10747_mix order by col1 nulls last, col5 nulls last; + +-- Q14. Regular query with 2 NULLS FIRST columns +select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls first; + +-- Q15. Regular query with mixed NULLS FIRST|LAST +select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls last; + +-- drop the test table +drop table spark_10747_mix; + + diff --git a/sql/core/src/test/resources/sql-tests/inputs/order-by-ordinal.sql b/sql/core/src/test/resources/sql-tests/inputs/order-by-ordinal.sql new file mode 100644 index 000000000000..8d733e77fa8d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/order-by-ordinal.sql @@ -0,0 +1,36 @@ +-- order by and sort by ordinal positions + +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b); + +select * from data order by 1 desc; + +-- mix ordinal and column name +select * from data order by 1 desc, b desc; + +-- order by multiple ordinals +select * from data order by 1 desc, 2 desc; + +-- 1 + 0 is considered a constant (not an ordinal) and thus ignored +select * from data order by 1 + 0 desc, b desc; + +-- negative cases: ordinal position out of range +select * from data order by 0; +select * from data order by -1; +select * from data order by 3; + +-- sort by ordinal +select * from data sort by 1 desc; + +-- turn off order by ordinal +set spark.sql.orderByOrdinal=false; + +-- 0 is now a valid literal +select * from data order by 0; +select * from data sort by 0; diff --git a/sql/core/src/test/resources/sql-tests/inputs/outer-join.sql b/sql/core/src/test/resources/sql-tests/inputs/outer-join.sql new file mode 100644 index 000000000000..cdc6c81e1004 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/outer-join.sql @@ -0,0 +1,39 @@ +-- SPARK-17099: Incorrect result when HAVING clause is added to group by query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1); + +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES +(-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) +as t2(int_col0, int_col1); + +SELECT + (SUM(COALESCE(t1.int_col1, t2.int_col0))), + ((COALESCE(t1.int_col1, t2.int_col0)) * 2) +FROM t1 +RIGHT JOIN t2 + ON (t2.int_col0) = (t1.int_col1) +GROUP BY GREATEST(COALESCE(t2.int_col1, 109), COALESCE(t1.int_col1, -449)), + COALESCE(t1.int_col1, t2.int_col0) +HAVING (SUM(COALESCE(t1.int_col1, t2.int_col0))) + > ((COALESCE(t1.int_col1, t2.int_col0)) * 2); + + +-- SPARK-17120: Analyzer incorrectly optimizes plan to empty LocalRelation +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1); + +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1); + +-- Set the cross join enabled flag for the LEFT JOIN test since there's no join condition. +-- Ultimately the join should be optimized away. +set spark.sql.crossJoin.enabled = true; +SELECT * +FROM ( +SELECT + COALESCE(t2.int_col1, t1.int_col1) AS int_col + FROM t1 + LEFT JOIN t2 ON false +) t where (t.int_col) is not null; +set spark.sql.crossJoin.enabled = false; + + diff --git a/sql/core/src/test/resources/sql-tests/inputs/pred-pushdown.sql b/sql/core/src/test/resources/sql-tests/inputs/pred-pushdown.sql new file mode 100644 index 000000000000..eff258a06635 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pred-pushdown.sql @@ -0,0 +1,12 @@ +CREATE OR REPLACE TEMPORARY VIEW tbl_a AS VALUES (1, 1), (2, 1), (3, 6) AS T(c1, c2); +CREATE OR REPLACE TEMPORARY VIEW tbl_b AS VALUES 1 AS T(c1); + +-- SPARK-18597: Do not push down predicates to left hand side in an anti-join +SELECT * +FROM tbl_a + LEFT ANTI JOIN tbl_b ON ((tbl_a.c1 = tbl_a.c2) IS NULL OR tbl_a.c1 = tbl_a.c2); + +-- SPARK-18614: Do not push down predicates on left table below ExistenceJoin +SELECT l.c1, l.c2 +FROM tbl_a l +WHERE EXISTS (SELECT 1 FROM tbl_b r WHERE l.c1 = l.c2) OR l.c2 < 2; diff --git a/sql/core/src/test/resources/sql-tests/inputs/predicate-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/predicate-functions.sql new file mode 100644 index 000000000000..3b3d4ad64b3e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/predicate-functions.sql @@ -0,0 +1,36 @@ +-- EqualTo +select 1 = 1; +select 1 = '1'; +select 1.0 = '1'; + +-- GreaterThan +select 1 > '1'; +select 2 > '1.0'; +select 2 > '2.0'; +select 2 > '2.2'; +select to_date('2009-07-30 04:17:52') > to_date('2009-07-30 04:17:52'); +select to_date('2009-07-30 04:17:52') > '2009-07-30 04:17:52'; + +-- GreaterThanOrEqual +select 1 >= '1'; +select 2 >= '1.0'; +select 2 >= '2.0'; +select 2.0 >= '2.2'; +select to_date('2009-07-30 04:17:52') >= to_date('2009-07-30 04:17:52'); +select to_date('2009-07-30 04:17:52') >= '2009-07-30 04:17:52'; + +-- LessThan +select 1 < '1'; +select 2 < '1.0'; +select 2 < '2.0'; +select 2.0 < '2.2'; +select to_date('2009-07-30 04:17:52') < to_date('2009-07-30 04:17:52'); +select to_date('2009-07-30 04:17:52') < '2009-07-30 04:17:52'; + +-- LessThanOrEqual +select 1 <= '1'; +select 2 <= '1.0'; +select 2 <= '2.0'; +select 2.0 <= '2.2'; +select to_date('2009-07-30 04:17:52') <= to_date('2009-07-30 04:17:52'); +select to_date('2009-07-30 04:17:52') <= '2009-07-30 04:17:52'; diff --git a/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql new file mode 100644 index 000000000000..ad96754826a4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/query_regex_column.sql @@ -0,0 +1,52 @@ +set spark.sql.parser.quotedRegexColumnNames=false; + +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66") +AS testData(key, value1, value2); + +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES +(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4) +AS testData2(A, B, c, d); + +-- AnalysisException +SELECT `(a)?+.+` FROM testData2 WHERE a = 1; +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1; +SELECT `(a|b)` FROM testData2 WHERE a = 2; +SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2; +SELECT SUM(`(a|b)?+.+`) FROM testData2; +SELECT SUM(`(a)`) FROM testData2; + +set spark.sql.parser.quotedRegexColumnNames=true; + +-- Regex columns +SELECT `(a)?+.+` FROM testData2 WHERE a = 1; +SELECT `(A)?+.+` FROM testData2 WHERE a = 1; +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1; +SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1; +SELECT `(a|B)` FROM testData2 WHERE a = 2; +SELECT `(A|b)` FROM testData2 WHERE a = 2; +SELECT `(a|B)?+.+` FROM testData2 WHERE a = 2; +SELECT `(A|b)?+.+` FROM testData2 WHERE a = 2; +SELECT `(e|f)` FROM testData2; +SELECT t.`(e|f)` FROM testData2 t; +SELECT p.`(KEY)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3; +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3; + +set spark.sql.caseSensitive=true; + +CREATE OR REPLACE TEMPORARY VIEW testdata3 AS SELECT * FROM VALUES +(0, 1), (1, 2), (2, 3), (3, 4) +AS testdata3(a, b); + +-- Regex columns +SELECT `(A)?+.+` FROM testdata3; +SELECT `(a)?+.+` FROM testdata3; +SELECT `(A)?+.+` FROM testdata3 WHERE a > 1; +SELECT `(a)?+.+` FROM testdata3 where `a` > 1; +SELECT SUM(`a`) FROM testdata3; +SELECT SUM(`(a)`) FROM testdata3; +SELECT SUM(`(a)?+.+`) FROM testdata3; +SELECT SUM(a) FROM testdata3 GROUP BY `a`; +-- AnalysisException +SELECT SUM(a) FROM testdata3 GROUP BY `(a)`; +SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+`; diff --git a/sql/core/src/test/resources/sql-tests/inputs/random.sql b/sql/core/src/test/resources/sql-tests/inputs/random.sql new file mode 100644 index 000000000000..a1aae7b8759d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/random.sql @@ -0,0 +1,17 @@ +-- rand with the seed 0 +SELECT rand(0); +SELECT rand(cast(3 / 7 AS int)); +SELECT rand(NULL); +SELECT rand(cast(NULL AS int)); + +-- rand unsupported data type +SELECT rand(1.0); + +-- randn with the seed 0 +SELECT randn(0L); +SELECT randn(cast(3 / 7 AS long)); +SELECT randn(NULL); +SELECT randn(cast(NULL AS long)); + +-- randn unsupported data type +SELECT rand('1') diff --git a/sql/core/src/test/resources/sql-tests/inputs/show-tables.sql b/sql/core/src/test/resources/sql-tests/inputs/show-tables.sql new file mode 100644 index 000000000000..3c77c9977d80 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/show-tables.sql @@ -0,0 +1,42 @@ +-- Test data. +CREATE DATABASE showdb; +USE showdb; +CREATE TABLE show_t1(a String, b Int, c String, d String) USING parquet PARTITIONED BY (c, d); +ALTER TABLE show_t1 ADD PARTITION (c='Us', d=1); +CREATE TABLE show_t2(b String, d Int) USING parquet; +CREATE TEMPORARY VIEW show_t3(e int) USING parquet; +CREATE GLOBAL TEMP VIEW show_t4 AS SELECT 1 as col1; + +-- SHOW TABLES +SHOW TABLES; +SHOW TABLES IN showdb; + +-- SHOW TABLES WITH wildcard match +SHOW TABLES 'show_t*'; +SHOW TABLES LIKE 'show_t1*|show_t2*'; +SHOW TABLES IN showdb 'show_t*'; + +-- SHOW TABLE EXTENDED +SHOW TABLE EXTENDED LIKE 'show_t*'; +SHOW TABLE EXTENDED; + +-- SHOW TABLE EXTENDED ... PARTITION +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us', d=1); +-- Throw a ParseException if table name is not specified. +SHOW TABLE EXTENDED PARTITION(c='Us', d=1); +-- Don't support regular expression for table name if a partition specification is present. +SHOW TABLE EXTENDED LIKE 'show_t*' PARTITION(c='Us', d=1); +-- Partition specification is not complete. +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us'); +-- Partition specification is invalid. +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(a='Us', d=1); +-- Partition specification doesn't exist. +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Ch', d=1); + +-- Clean Up +DROP TABLE show_t1; +DROP TABLE show_t2; +DROP VIEW show_t3; +DROP VIEW global_temp.show_t4; +USE default; +DROP DATABASE showdb; diff --git a/sql/core/src/test/resources/sql-tests/inputs/show_columns.sql b/sql/core/src/test/resources/sql-tests/inputs/show_columns.sql new file mode 100644 index 000000000000..521018e94e50 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/show_columns.sql @@ -0,0 +1,58 @@ +CREATE DATABASE showdb; + +USE showdb; + +CREATE TABLE showcolumn1 (col1 int, `col 2` int) USING json; +CREATE TABLE showcolumn2 (price int, qty int, year int, month int) USING parquet partitioned by (year, month); +CREATE TEMPORARY VIEW showColumn3 (col3 int, `col 4` int) USING json; +CREATE GLOBAL TEMP VIEW showColumn4 AS SELECT 1 as col1, 'abc' as `col 5`; + + +-- only table name +SHOW COLUMNS IN showcolumn1; + +-- qualified table name +SHOW COLUMNS IN showdb.showcolumn1; + +-- table name and database name +SHOW COLUMNS IN showcolumn1 FROM showdb; + +-- partitioned table +SHOW COLUMNS IN showcolumn2 IN showdb; + +-- Non-existent table. Raise an error in this case +SHOW COLUMNS IN badtable FROM showdb; + +-- database in table identifier and database name in different case +SHOW COLUMNS IN showdb.showcolumn1 from SHOWDB; + +-- different database name in table identifier and database name. +-- Raise an error in this case. +SHOW COLUMNS IN showdb.showcolumn1 FROM baddb; + +-- show column on temporary view +SHOW COLUMNS IN showcolumn3; + +-- error temp view can't be qualified with a database +SHOW COLUMNS IN showdb.showcolumn3; + +-- error temp view can't be qualified with a database +SHOW COLUMNS IN showcolumn3 FROM showdb; + +-- error global temp view needs to be qualified +SHOW COLUMNS IN showcolumn4; + +-- global temp view qualified with database +SHOW COLUMNS IN global_temp.showcolumn4; + +-- global temp view qualified with database +SHOW COLUMNS IN showcolumn4 FROM global_temp; + +DROP TABLE showcolumn1; +DROP TABLE showColumn2; +DROP VIEW showcolumn3; +DROP VIEW global_temp.showcolumn4; + +use default; + +DROP DATABASE showdb; diff --git a/sql/core/src/test/resources/sql-tests/inputs/sql-compatibility-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/sql-compatibility-functions.sql new file mode 100644 index 000000000000..f1461032065a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/sql-compatibility-functions.sql @@ -0,0 +1,29 @@ +-- A test suite for functions added for compatibility with other databases such as Oracle, MSSQL. +-- These functions are typically implemented using the trait RuntimeReplaceable. + +SELECT ifnull(null, 'x'), ifnull('y', 'x'), ifnull(null, null); +SELECT nullif('x', 'x'), nullif('x', 'y'); +SELECT nvl(null, 'x'), nvl('y', 'x'), nvl(null, null); +SELECT nvl2(null, 'x', 'y'), nvl2('n', 'x', 'y'), nvl2(null, null, null); + +-- type coercion +SELECT ifnull(1, 2.1d), ifnull(null, 2.1d); +SELECT nullif(1, 2.1d), nullif(1, 1.0d); +SELECT nvl(1, 2.1d), nvl(null, 2.1d); +SELECT nvl2(null, 1, 2.1d), nvl2('n', 1, 2.1d); + +-- explain for these functions; use range to avoid constant folding +explain extended +select ifnull(id, 'x'), nullif(id, 'x'), nvl(id, 'x'), nvl2(id, 'x', 'y') +from range(2); + +-- SPARK-16730 cast alias functions for Hive compatibility +SELECT boolean(1), tinyint(1), smallint(1), int(1), bigint(1); +SELECT float(1), double(1), decimal(1); +SELECT date("2014-04-04"), timestamp(date("2014-04-04")); +-- error handling: only one argument +SELECT string(1, 2); + +-- SPARK-21555: RuntimeReplaceable used in group by +CREATE TEMPORARY VIEW tempView1 AS VALUES (1, NAMED_STRUCT('col1', 'gamma', 'col2', 'delta')) AS T(id, st); +SELECT nvl(st.col1, "value"), count(*) FROM from tempView1 GROUP BY nvl(st.col1, "value"); diff --git a/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql new file mode 100644 index 000000000000..40d0c064f5c4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql @@ -0,0 +1,26 @@ +-- Argument number exception +select concat_ws(); +select format_string(); + +-- A pipe operator for string concatenation +select 'a' || 'b' || 'c'; + +-- Check if catalyst combine nested `Concat`s +EXPLAIN EXTENDED SELECT (col1 || col2 || col3 || col4) col +FROM (SELECT id col1, id col2, id col3, id col4 FROM range(10)); + +-- replace function +select replace('abc', 'b', '123'); +select replace('abc', 'b'); + +-- uuid +select length(uuid()), (uuid() <> uuid()); + +-- position +select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null); + +-- left && right +select left("abcd", 2), left("abcd", 5), left("abcd", '2'), left("abcd", null); +select left(null, -2), left("abcd", -2), left("abcd", 0), left("abcd", 'a'); +select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null); +select right(null, -2), right("abcd", -2), right("abcd", 0), right("abcd", 'a'); diff --git a/sql/core/src/test/resources/sql-tests/inputs/struct.sql b/sql/core/src/test/resources/sql-tests/inputs/struct.sql new file mode 100644 index 000000000000..93a1238ab18c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/struct.sql @@ -0,0 +1,27 @@ +CREATE TEMPORARY VIEW tbl_x AS VALUES + (1, NAMED_STRUCT('C', 'gamma', 'D', 'delta')), + (2, NAMED_STRUCT('C', 'epsilon', 'D', 'eta')), + (3, NAMED_STRUCT('C', 'theta', 'D', 'iota')) + AS T(ID, ST); + +-- Create a struct +SELECT STRUCT('alpha', 'beta') ST; + +-- Create a struct with aliases +SELECT STRUCT('alpha' AS A, 'beta' AS B) ST; + +-- Star expansion in a struct. +SELECT ID, STRUCT(ST.*) NST FROM tbl_x; + +-- Append a column to a struct +SELECT ID, STRUCT(ST.*,CAST(ID AS STRING) AS E) NST FROM tbl_x; + +-- Prepend a column to a struct +SELECT ID, STRUCT(CAST(ID AS STRING) AS AA, ST.*) NST FROM tbl_x; + +-- Select a column from a struct +SELECT ID, STRUCT(ST.*).C NST FROM tbl_x; +SELECT ID, STRUCT(ST.C, ST.D).D NST FROM tbl_x; + +-- Select an alias from a struct +SELECT ID, STRUCT(ST.C as STC, ST.D as STD).STD FROM tbl_x; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-aggregate.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-aggregate.sql new file mode 100644 index 000000000000..b5f458f2cb18 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-aggregate.sql @@ -0,0 +1,115 @@ +-- Tests aggregate expressions in outer query and EXISTS subquery. + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + +-- Aggregate in outer query block. +-- TC.01.01 +SELECT emp.dept_id, + avg(salary), + sum(salary) +FROM emp +WHERE EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id) +GROUP BY dept_id; + +-- Aggregate in inner/subquery block +-- TC.01.02 +SELECT emp_name +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id); + +-- Aggregate expression in both outer and inner query block. +-- TC.01.03 +SELECT count(*) +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id); + +-- Nested exists with aggregate expression in inner most query block. +-- TC.01.04 +SELECT * +FROM bonus +WHERE EXISTS (SELECT 1 + FROM emp + WHERE emp.emp_name = bonus.emp_name + AND EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE emp.dept_id = dept.dept_id + GROUP BY dept.dept_id)); + +-- Not exists with Aggregate expression in outer +-- TC.01.05 +SELECT emp.dept_id, + Avg(salary), + Sum(salary) +FROM emp +WHERE NOT EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id) +GROUP BY dept_id; + +-- Not exists with Aggregate expression in subquery block +-- TC.01.06 +SELECT emp_name +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id); + +-- Not exists with Aggregate expression in outer and subquery block +-- TC.01.07 +SELECT count(*) +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id); + +-- Nested not exists and exists with aggregate expression in inner most query block. +-- TC.01.08 +SELECT * +FROM bonus +WHERE NOT EXISTS (SELECT 1 + FROM emp + WHERE emp.emp_name = bonus.emp_name + AND EXISTS (SELECT Max(dept.dept_id) + FROM dept + WHERE emp.dept_id = dept.dept_id + GROUP BY dept.dept_id)); diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-basic.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-basic.sql new file mode 100644 index 000000000000..332e858800f7 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-basic.sql @@ -0,0 +1,123 @@ +-- Tests EXISTS subquery support. Tests basic form +-- of EXISTS subquery (both EXISTS and NOT EXISTS) + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + +-- uncorrelated exist query +-- TC.01.01 +SELECT * +FROM emp +WHERE EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30); + +-- simple correlated predicate in exist subquery +-- TC.01.02 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id); + +-- correlated outer isnull predicate +-- TC.01.03 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id + OR emp.dept_id IS NULL); + +-- Simple correlation with a local predicate in outer query +-- TC.01.04 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) + AND emp.id > 200; + +-- Outer references (emp.id) should not be pruned from outer plan +-- TC.01.05 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id) + AND emp.id > 200; + +-- not exists with correlated predicate +-- TC.01.06 +SELECT * +FROM dept +WHERE NOT EXISTS (SELECT emp_name + FROM emp + WHERE emp.dept_id = dept.dept_id); + +-- not exists with correlated predicate + local predicate +-- TC.01.07 +SELECT * +FROM dept +WHERE NOT EXISTS (SELECT emp_name + FROM emp + WHERE emp.dept_id = dept.dept_id + OR state = 'NJ'); + +-- not exist both equal and greaterthan predicate +-- TC.01.08 +SELECT * +FROM bonus +WHERE NOT EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary); + +-- select employees who have not received any bonus +-- TC 01.09 +SELECT emp.* +FROM emp +WHERE NOT EXISTS (SELECT NULL + FROM bonus + WHERE bonus.emp_name = emp.emp_name); + +-- Nested exists +-- TC.01.10 +SELECT * +FROM bonus +WHERE EXISTS (SELECT emp_name + FROM emp + WHERE bonus.emp_name = emp.emp_name + AND EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id)); diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-cte.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-cte.sql new file mode 100644 index 000000000000..c6784838158e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-cte.sql @@ -0,0 +1,142 @@ +-- Tests EXISTS subquery used along with +-- Common Table Expressions(CTE) + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + +-- CTE used inside subquery with correlated condition +-- TC.01.01 +WITH bonus_cte + AS (SELECT * + FROM bonus + WHERE EXISTS (SELECT dept.dept_id, + emp.emp_name, + Max(salary), + Count(*) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY dept.dept_id, + emp.emp_name + ORDER BY emp.emp_name)) +SELECT * +FROM bonus a +WHERE a.bonus_amt > 30 + AND EXISTS (SELECT 1 + FROM bonus_cte b + WHERE a.emp_name = b.emp_name); + +-- Inner join between two CTEs with correlated condition +-- TC.01.02 +WITH emp_cte + AS (SELECT * + FROM emp + WHERE id >= 100 + AND id <= 300), + dept_cte + AS (SELECT * + FROM dept + WHERE dept_id = 10) +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp_cte a + JOIN dept_cte b + ON a.dept_id = b.dept_id + WHERE bonus.emp_name = a.emp_name); + +-- Left outer join between two CTEs with correlated condition +-- TC.01.03 +WITH emp_cte + AS (SELECT * + FROM emp + WHERE id >= 100 + AND id <= 300), + dept_cte + AS (SELECT * + FROM dept + WHERE dept_id = 10) +SELECT DISTINCT b.emp_name, + b.bonus_amt +FROM bonus b, + emp_cte e, + dept d +WHERE e.dept_id = d.dept_id + AND e.emp_name = b.emp_name + AND EXISTS (SELECT * + FROM emp_cte a + LEFT JOIN dept_cte b + ON a.dept_id = b.dept_id + WHERE e.emp_name = a.emp_name); + +-- Joins inside cte and aggregation on cte referenced subquery with correlated condition +-- TC.01.04 +WITH empdept + AS (SELECT id, + salary, + emp_name, + dept.dept_id + FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id + WHERE emp.id IN ( 100, 200 )) +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE EXISTS (SELECT dept_id, + max(salary) + FROM empdept + GROUP BY dept_id + HAVING count(*) > 1) +GROUP BY emp_name; + +-- Using not exists +-- TC.01.05 +WITH empdept + AS (SELECT id, + salary, + emp_name, + dept.dept_id + FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id + WHERE emp.id IN ( 100, 200 )) +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE NOT EXISTS (SELECT dept_id, + Max(salary) + FROM empdept + GROUP BY dept_id + HAVING count(*) < 1) +GROUP BY emp_name; diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-having.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-having.sql new file mode 100644 index 000000000000..c30159039ff3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-having.sql @@ -0,0 +1,94 @@ +-- Tests HAVING clause in subquery. + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + +-- simple having in subquery. +-- TC.01.01 +SELECT dept_id, count(*) +FROM emp +GROUP BY dept_id +HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < min(emp.salary)); + +-- nested having in subquery +-- TC.01.02 +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < Min(emp.salary))); + +-- aggregation in outer and inner query block with having +-- TC.01.03 +SELECT dept_id, + Max(salary) +FROM emp gp +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp p + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < Min(p.salary))) +GROUP BY gp.dept_id; + +-- more aggregate expressions in projection list of subquery +-- TC.01.04 +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt > Min(emp.salary))); + +-- multiple aggregations in nested subquery +-- TC.01.05 +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + count(emp.dept_id) + FROM emp + WHERE dept.dept_id = dept_id + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE ( bonus_amt > min(emp.salary) + AND count(emp.dept_id) > 1 ))); diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql new file mode 100644 index 000000000000..cc4ed64affec --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql @@ -0,0 +1,228 @@ +-- Tests EXISTS subquery support. Tests Exists subquery +-- used in Joins (Both when joins occurs in outer and suquery blocks) + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + +-- Join in outer query block +-- TC.01.01 +SELECT * +FROM emp, + dept +WHERE emp.dept_id = dept.dept_id + AND EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name); + +-- Join in outer query block with ON condition +-- TC.01.02 +SELECT * +FROM emp + JOIN dept + ON emp.dept_id = dept.dept_id +WHERE EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name); + +-- Left join in outer query block with ON condition +-- TC.01.03 +SELECT * +FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id +WHERE EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name); + +-- Join in outer query block + NOT EXISTS +-- TC.01.04 +SELECT * +FROM emp, + dept +WHERE emp.dept_id = dept.dept_id + AND NOT EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name); + + +-- inner join in subquery. +-- TC.01.05 +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name); + +-- right join in subquery +-- TC.01.06 +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp + RIGHT JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name); + + +-- Aggregation and join in subquery +-- TC.01.07 +SELECT * +FROM bonus +WHERE EXISTS (SELECT dept.dept_id, + emp.emp_name, + Max(salary), + Count(*) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY dept.dept_id, + emp.emp_name + ORDER BY emp.emp_name); + +-- Aggregations in outer and subquery + join in subquery +-- TC.01.08 +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE EXISTS (SELECT emp_name, + Max(salary) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY emp_name + HAVING Count(*) > 1 + ORDER BY emp_name) +GROUP BY emp_name; + +-- TC.01.09 +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE NOT EXISTS (SELECT emp_name, + Max(salary) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY emp_name + HAVING Count(*) > 1 + ORDER BY emp_name) +GROUP BY emp_name; + +-- Set operations along with EXISTS subquery +-- union +-- TC.02.01 +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + UNION + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50); + +-- intersect +-- TC.02.02 +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50); + +-- intersect + not exists +-- TC.02.03 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50); + +-- Union all in outer query and except,intersect in subqueries. +-- TC.02.04 +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + EXCEPT + SELECT * + FROM dept + WHERE dept_id > 50) +UNION ALL +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50); + +-- Union in outer query and except,intersect in subqueries. +-- TC.02.05 +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + EXCEPT + SELECT * + FROM dept + WHERE dept_id > 50) +UNION +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50); + diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql new file mode 100644 index 000000000000..19fc18833760 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql @@ -0,0 +1,118 @@ +-- Tests EXISTS subquery support with ORDER BY and LIMIT clauses. + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + +-- order by in both outer and/or inner query block +-- TC.01.01 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate; + +-- TC.01.02 +SELECT id, + hiredate +FROM emp +WHERE EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate DESC; + +-- order by with not exists +-- TC.01.03 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate; + +-- group by + order by with not exists +-- TC.01.04 +SELECT emp_name +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state); +-- TC.01.05 +SELECT count(*) +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept_id + ORDER BY dept_id); + +-- limit in the exists subquery block. +-- TC.02.01 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + LIMIT 1); + +-- limit in the exists subquery block with aggregate. +-- TC.02.02 +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + LIMIT 1); + +-- limit in the not exists subquery block. +-- TC.02.03 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + LIMIT 1); + +-- limit in the not exists subquery block with aggregates. +-- TC.02.04 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + LIMIT 1); diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-within-and-or.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-within-and-or.sql new file mode 100644 index 000000000000..7743b5241d11 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-within-and-or.sql @@ -0,0 +1,96 @@ +-- Tests EXISTS subquery support. Tests EXISTS +-- subquery within a AND or OR expression. + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + + +-- Or used in conjunction with exists - ExistenceJoin +-- TC.02.01 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR emp.id > 200; + +-- all records from emp including the null dept_id +-- TC.02.02 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR emp.dept_id IS NULL; + +-- EXISTS subquery in both LHS and RHS of OR. +-- TC.02.03 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id + AND dept.dept_id = 20) + OR EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id + AND dept.dept_id = 30); +; + +-- not exists and exists predicate within OR +-- TC.02.04 +SELECT * +FROM bonus +WHERE ( NOT EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) + OR EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + OR bonus_amt < emp.salary) ); + +-- not exists and in predicate within AND +-- TC.02.05 +SELECT * FROM bonus WHERE NOT EXISTS +( + SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) +AND +emp_name IN +( + SELECT emp_name + FROM emp + WHERE bonus_amt < emp.salary); + diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql new file mode 100644 index 000000000000..b1d96b32c247 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-group-by.sql @@ -0,0 +1,239 @@ +-- A test suite for GROUP BY in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +create temporary view t1 as select * from values + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- GROUP BY in parent side +-- TC 01.01 +SELECT t1a, + Avg(t1b) +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2) +GROUP BY t1a; + +-- TC 01.02 +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1d; + +-- TC 01.03 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1b; + +-- TC 01.04 +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) + OR t1c IN (SELECT t3c + FROM t3 + WHERE t1a = t3a) +GROUP BY t1a, + t1c; + +-- TC 01.05 +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) + AND t1c IN (SELECT t3c + FROM t3 + WHERE t1a = t3a) +GROUP BY t1a, + t1c; + +-- TC 01.06 +SELECT t1a, + Count(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1c +HAVING t1a = "t1b"; + +-- GROUP BY in subquery +-- TC 01.07 +SELECT * +FROM t1 +WHERE t1b IN (SELECT Max(t2b) + FROM t2 + GROUP BY t2a); + +-- TC 01.08 +SELECT * +FROM (SELECT t2a, + t2b + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) + GROUP BY t2a, + t2b) t2; + +-- TC 01.09 +SELECT Count(DISTINCT( * )) +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + AND t1c = t2c + GROUP BY t2a); + +-- TC 01.10 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT Max(t2c) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a, + t2c + HAVING t2c > 8); + +-- TC 01.11 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2a IN (SELECT Min(t3a) + FROM t3 + WHERE t3a = t2a + GROUP BY t3b) + GROUP BY t2c); + +-- GROUP BY in both +-- TC 01.12 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) +GROUP BY t1a; + +-- TC 01.13 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b IN (SELECT Min(t3b) + FROM t3 + WHERE t2a = t3a + GROUP BY t3a) + GROUP BY t2c) +GROUP BY t1a, + t1d; + +-- TC 01.14 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) + AND t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d) +GROUP BY t1a; + +-- TC 01.15 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) + OR t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d) +GROUP BY t1a; + +-- TC 01.16 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a + HAVING t2a > t1a) + OR t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d + HAVING t3d = t1d) +GROUP BY t1a +HAVING Min(t1b) IS NOT NULL; + + + diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-having.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-having.sql new file mode 100644 index 000000000000..8f98ae115506 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-having.sql @@ -0,0 +1,152 @@ +-- A test suite for IN HAVING in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- HAVING in the subquery +-- TC 01.01 +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + GROUP BY t2b + HAVING t2b < 10); + +-- TC 01.02 +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2b + HAVING t2b > 1); + +-- HAVING in the parent +-- TC 01.03 +SELECT t1a, t1b, t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c < t2c) +GROUP BY t1a, t1b, t1c +HAVING t1b < 10; + +-- TC 01.04 +SELECT t1a, t1b, t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c = t2c) +GROUP BY t1a, t1b, t1c +HAVING COUNT (DISTINCT t1b) < 10; + +-- BOTH +-- TC 01.05 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a + GROUP BY t2c + HAVING t2c > 10) +GROUP BY t1b +HAVING t1b >= 8; + +-- TC 01.06 +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b > 0 +GROUP BY t1a +HAVING t1a IN (SELECT t2a + FROM t2 + WHERE t2b IN (SELECT t3b + FROM t3 + WHERE t2c = t3c) + ); + +-- HAVING clause with NOT IN +-- TC 01.07 +SELECT t1a, + t1c, + Min(t1d) +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + GROUP BY t2a + HAVING t2a > 'val2a') +GROUP BY t1a, t1c +HAVING Min(t1d) > t1c; + +-- TC 01.08 +SELECT t1a, + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1a = t2a + GROUP BY t2c, t2d + HAVING t2c > 8) +GROUP BY t1a, t1b +HAVING t1b < 10; + +-- TC 01.09 +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b > 0 +GROUP BY t1a +HAVING t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b > 3); + diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql new file mode 100644 index 000000000000..880175fd7add --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-joins.sql @@ -0,0 +1,270 @@ +-- A test suite for IN JOINS in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- different JOIN in parent side +-- TC 01.01 +SELECT t1a, t1b, t1c, t3a, t3b, t3c +FROM t1 natural JOIN t3 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1a = t2a) + AND t1b = t3b + AND t1a = t3a +ORDER BY t1a, + t1b, + t1c DESC nulls first; + +-- TC 01.02 +SELECT Count(DISTINCT(t1a)), + t1b, + t3a, + t3b, + t3c +FROM t1 natural left JOIN t3 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1d = t2d) +AND t1b > t3b +GROUP BY t1a, + t1b, + t3a, + t3b, + t3c +ORDER BY t1a DESC, t3b DESC; + +-- TC 01.03 +SELECT Count(DISTINCT(t1a)) +FROM t1 natural right JOIN t3 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1b = t2b) +AND t1d IN + ( + SELECT t2d + FROM t2 + WHERE t1c > t2c) +AND t1a = t3a +GROUP BY t1a +ORDER BY t1a; + +-- TC 01.04 +SELECT t1a, + t1b, + t1c, + t3a, + t3b, + t3c +FROM t1 FULL OUTER JOIN t3 +where t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2c IS NOT NULL) +AND t1b != t3b +AND t1a = 'val1b' +ORDER BY t1a; + +-- TC 01.05 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 RIGHT JOIN t3 +where t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2h > t3h) +AND t3a IN + ( + SELECT t2a + FROM t2 + WHERE t2c > t3c) +AND t1h >= t3h +GROUP BY t1a, + t1b +HAVING t1b > 8 +ORDER BY t1a; + +-- TC 01.06 +SELECT Count(DISTINCT(t1a)) +FROM t1 LEFT OUTER +JOIN t3 +ON t1a = t3a +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1h < t2h ) +GROUP BY t1a +ORDER BY t1a; + +-- TC 01.07 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 INNER JOIN t2 +ON t1a > t2a +WHERE t1b IN + ( + SELECT t2b + FROM t2 + WHERE t2h > t1h) +OR t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2h < t1h) +GROUP BY t1b +HAVING t1b > 6; + +-- different JOIN in the subquery +-- TC 01.08 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +AND t1h IN + ( + SELECT t2h + FROM t2 + RIGHT JOIN t3 + where t2b = t3b) +GROUP BY t1b +HAVING t1b > 8; + +-- TC 01.09 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +AND t1h IN + ( + SELECT t2h + FROM t2 + RIGHT JOIN t3 + where t2b = t3b) +AND t1b IN + ( + SELECT t2b + FROM t2 + FULL OUTER JOIN t3 + where t2b = t3b) + +GROUP BY t1b +HAVING t1b > 8; + +-- JOIN in the parent and subquery +-- TC 01.10 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +INNER JOIN t2 on t1b = t2b +RIGHT JOIN t3 ON t1a = t3a +where t1a IN + ( + SELECT t2a + FROM t2 + FULL OUTER JOIN t3 + WHERE t2b > t3b) +AND t1c IN + ( + SELECT t3c + FROM t3 + LEFT OUTER JOIN t2 + ON t3a = t2a ) +AND t1b IN + ( + SELECT t3b + FROM t3 LEFT OUTER + JOIN t1 + WHERE t3c = t1c) + +AND t1a = t2a +GROUP BY t1b +ORDER BY t1b DESC; + +-- TC 01.11 +SELECT t1a, + t1b, + t1c, + count(distinct(t2a)), + t2b, + t2c +FROM t1 +FULL JOIN t2 on t1a = t2a +RIGHT JOIN t3 on t1a = t3a +where t1a IN + ( + SELECT t2a + FROM t2 INNER + JOIN t3 + ON t2b < t3b + WHERE t2c IN + ( + SELECT t1c + FROM t1 + WHERE t1a = t2a)) +and t1a = t2a +Group By t1a, t1b, t1c, t2a, t2b, t2c +HAVING t2c IS NOT NULL +ORDER By t2b DESC nulls last; + diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql new file mode 100644 index 000000000000..a40ee082ba3b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql @@ -0,0 +1,100 @@ +-- A test suite for IN LIMIT in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- LIMIT in parent side +-- TC 01.01 +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +LIMIT 2; + +-- TC 01.02 +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + LIMIT 2) +LIMIT 4; + +-- TC 01.03 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c + LIMIT 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +LIMIT 1; + +-- LIMIT with NOT IN +-- TC 01.04 +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + LIMIT 2); + +-- TC 01.05 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first + LIMIT 1) +GROUP BY t1b +ORDER BY t1b NULLS last +LIMIT 1; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-multiple-columns.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-multiple-columns.sql new file mode 100644 index 000000000000..4643605148a0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-multiple-columns.sql @@ -0,0 +1,127 @@ +-- A test suite for multiple columns in predicate in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- TC 01.01 +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE ( t1a, t1h ) NOT IN (SELECT t2a, + t2h + FROM t2 + WHERE t2a = t1a + ORDER BY t2a) +AND t1a = 'val1a'; + +-- TC 01.02 +SELECT t1a, + t1b, + t1d +FROM t1 +WHERE ( t1b, t1d ) IN (SELECT t2b, + t2d + FROM t2 + WHERE t2i IN (SELECT t3i + FROM t3 + WHERE t2b > t3b)); + +-- TC 01.03 +SELECT t1a, + t1b, + t1d +FROM t1 +WHERE ( t1b, t1d ) NOT IN (SELECT t2b, + t2d + FROM t2 + WHERE t2h IN (SELECT t3h + FROM t3 + WHERE t2b > t3b)) +AND t1a = 'val1a'; + +-- TC 01.04 +SELECT t2a +FROM (SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t1a, + t1b + FROM t1) + UNION ALL + SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t1a, + t1b + FROM t1) + UNION DISTINCT + SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t3a, + t3b + FROM t3)) AS t4; + +-- TC 01.05 +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1 + WHERE ( + t1b, t1d) IN + ( + SELECT t2b, + t2d + FROM t2 + WHERE t1c = t2c)) +SELECT * +FROM ( + SELECT * + FROM cte1 + JOIN cte1 cte2 + on cte1.t1b = cte2.t1b) s; + diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql new file mode 100644 index 000000000000..892e39ff47c1 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-order-by.sql @@ -0,0 +1,197 @@ +-- A test suite for ORDER BY in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- ORDER BY in parent side +-- TC 01.01 +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2) +ORDER BY t1a; + +-- TC 01.02 +SELECT t1a +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +ORDER BY t1b DESC; + +-- TC 01.03 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +ORDER BY 2 DESC nulls last; + +-- TC 01.04 +SELECT Count(DISTINCT( t1a )) +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +ORDER BY Count(DISTINCT( t1a )); + +-- ORDER BY in subquery +-- TC 01.05 +SELECT * +FROM t1 +WHERE t1b IN (SELECT t2c + FROM t2 + ORDER BY t2d); + +-- ORDER BY in BOTH +-- TC 01.06 +SELECT * +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1b = t2b + ORDER BY Min(t2b)) +ORDER BY t1c DESC nulls first; + +-- TC 01.07 +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a + ORDER BY t2b DESC nulls first) + OR t1h IN (SELECT t2h + FROM t2 + WHERE t1h > t2h) +ORDER BY t1h DESC nulls last; + +-- ORDER BY with NOT IN +-- TC 01.08 +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2) +ORDER BY t1a; + +-- TC 01.09 +SELECT t1a, + t1b +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t1a = t2a) +ORDER BY t1b DESC nulls last; + +-- TC 01.10 +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + ORDER BY t2a DESC nulls first) + and t1c IN (SELECT t2c + FROM t2 + ORDER BY t2b DESC nulls last) +ORDER BY t1c DESC nulls last; + +-- GROUP BY and ORDER BY +-- TC 01.11 +SELECT * +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + GROUP BY t2a + ORDER BY t2a DESC); + +-- TC 01.12 +SELECT t1a, + Count(DISTINCT( t1b )) +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a + ORDER BY t2a) +GROUP BY t1a, + t1h +ORDER BY t1a; + +-- GROUP BY and ORDER BY with NOT IN +-- TC 01.13 +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT Min(t2b) + FROM t2 + GROUP BY t2a + ORDER BY t2a); + +-- TC 01.14 +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1b NOT IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2c + ORDER BY t2c DESC nulls last) +GROUP BY t1a; + +-- TC 01.15 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1h NOT IN (SELECT t2h + FROM t2 + where t1a = t2a + order by t2d DESC nulls first + ) +GROUP BY t1a, + t1b +ORDER BY t1b DESC nulls last; diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-set-operations.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-set-operations.sql new file mode 100644 index 000000000000..5c371d2305ac --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-set-operations.sql @@ -0,0 +1,472 @@ +-- A test suite for set-operations in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- UNION, UNION ALL, UNION DISTINCT, INTERSECT and EXCEPT in the parent +-- TC 01.01 +SELECT t2a, + t2b, + t2c, + t2h, + t2i +FROM (SELECT * + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1) + UNION ALL + SELECT * + FROM t3 + WHERE t3a IN (SELECT t1a + FROM t1)) AS t3 +WHERE t2i IS NOT NULL AND + 2 * t2b = t2c +ORDER BY t2c DESC nulls first; + +-- TC 01.02 +SELECT t2a, + t2b, + t2d, + Count(DISTINCT( t2h )), + t2i +FROM (SELECT * + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2b = t1b) + UNION + SELECT * + FROM t1 + WHERE t1a IN (SELECT t3a + FROM t3 + WHERE t1c = t3c)) AS t3 +GROUP BY t2a, + t2b, + t2d, + t2i +ORDER BY t2d DESC; + +-- TC 01.03 +SELECT t2a, + t2b, + t2c, + Min(t2d) +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, t2b, t2c +UNION ALL +SELECT t2a, + t2b, + t2c, + Max(t2d) +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, t2b, t2c +UNION +SELECT t3a, + t3b, + t3c, + Min(t3d) +FROM t3 +WHERE t3a IN (SELECT t2a + FROM t2 + WHERE t3c = t2c) +GROUP BY t3a, t3b, t3c +UNION DISTINCT +SELECT t1a, + t1b, + t1c, + Max(t1d) +FROM t1 +WHERE t1a IN (SELECT t3a + FROM t3 + WHERE t3d = t1d) +GROUP BY t1a, t1b, t1c; + +-- TC 01.04 +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +UNION +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +HAVING t2b IS NOT NULL; + +-- TC 01.05 +SELECT t2a, + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT DISTINCT(t1a) + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i + +UNION +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2b IN (SELECT Max(t1b) + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +HAVING t2b IS NOT NULL +UNION DISTINCT +SELECT t2a, + t2b, + t2c, + t2d, + t2h, + t2i +FROM t2 +WHERE t2d IN (SELECT min(t1d) + FROM t1 + WHERE t2c = t1c); + +-- TC 01.06 +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b AND + t1d < t2d) +INTERSECT +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2b IN (SELECT Max(t1b) + FROM t1 + WHERE t2c = t1c) +EXCEPT +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2d IN (SELECT Min(t3d) + FROM t3 + WHERE t2c = t3c) +UNION ALL +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2c IN (SELECT Max(t1c) + FROM t1 + WHERE t1d = t2d); + +-- UNION, UNION ALL, UNION DISTINCT, INTERSECT and EXCEPT in the subquery +-- TC 01.07 +SELECT DISTINCT(t1a), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a IN (SELECT t3a + FROM (SELECT t2a t3a + FROM t2 + UNION ALL + SELECT t2a t3a + FROM t2) AS t3 + UNION + SELECT t2a + FROM (SELECT t2a + FROM t2 + WHERE t2b > 6 + UNION + SELECT t2a + FROM t2 + WHERE t2b > 6) AS t4 + UNION DISTINCT + SELECT t2a + FROM (SELECT t2a + FROM t2 + WHERE t2b > 6 + UNION DISTINCT + SELECT t1a + FROM t1 + WHERE t1b > 6) AS t5) +GROUP BY t1a, t1b, t1c, t1d +HAVING t1c IS NOT NULL AND t1b IS NOT NULL +ORDER BY t1c DESC, t1a DESC; + +-- TC 01.08 +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM (SELECT t2b + FROM t2 + WHERE t2b > 6 + INTERSECT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t3 + WHERE t2b = t1b); + +-- TC 01.09 +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1h IN (SELECT t2h + FROM (SELECT t2h + FROM t2 + EXCEPT + SELECT t3h + FROM t3) AS t3) +ORDER BY t1b DESC NULLs first, t1c DESC NULLs last; + +-- UNION, UNION ALL, UNION DISTINCT, INTERSECT and EXCEPT in the parent and subquery +-- TC 01.10 +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b > 6 + INTERSECT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t3) +UNION DISTINCT +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b > 6 + EXCEPT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t4 + WHERE t2b = t1b) +ORDER BY t1c DESC NULLS last, t1a DESC; + +-- TC 01.11 +SELECT * +FROM (SELECT * + FROM (SELECT * + FROM t2 + WHERE t2h IN (SELECT t1h + FROM t1 + WHERE t1a = t2a) + UNION DISTINCT + SELECT * + FROM t1 + WHERE t1h IN (SELECT t3h + FROM t3 + UNION + SELECT t1h + FROM t1) + UNION + SELECT * + FROM t3 + WHERE t3a IN (SELECT t2a + FROM t2 + UNION ALL + SELECT t1a + FROM t1 + WHERE t1b > 0) + INTERSECT + SELECT * + FROM T1 + WHERE t1b IN (SELECT t3b + FROM t3 + UNION DISTINCT + SELECT t2b + FROM t2 + ) + EXCEPT + SELECT * + FROM t2 + WHERE t2h IN (SELECT t1i + FROM t1)) t4 + WHERE t4.t2b IN (SELECT Min(t3b) + FROM t3 + WHERE t4.t2a = t3a)); + +-- UNION, UNION ALL, UNION DISTINCT, INTERSECT and EXCEPT for NOT IN +-- TC 01.12 +SELECT t2a, + t2b, + t2c, + t2i +FROM (SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t1a + FROM t1 + UNION + SELECT t3a + FROM t3) + UNION ALL + SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t1a + FROM t1 + INTERSECT + SELECT t2a + FROM t2)) AS t3 +WHERE t3.t2a NOT IN (SELECT t1a + FROM t1 + INTERSECT + SELECT t2a + FROM t2) + AND t2c IS NOT NULL +ORDER BY t2a; + +-- TC 01.13 +SELECT Count(DISTINCT(t1a)), + t1b, + t1c, + t1i +FROM t1 +WHERE t1b NOT IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b NOT IN + ( + SELECT t1b + FROM t1) + UNION + SELECT t1b + FROM t1 + WHERE t1b NOT IN + ( + SELECT t3b + FROM t3) + UNION + distinct SELECT t3b + FROM t3 + WHERE t3b NOT IN + ( + SELECT t2b + FROM t2)) AS t3 + WHERE t2b = t1b) +GROUP BY t1a, + t1b, + t1c, + t1i +HAVING t1b NOT IN + ( + SELECT t2b + FROM t2 + WHERE t2c IS NULL + EXCEPT + SELECT t3b + FROM t3) +ORDER BY t1c DESC NULLS LAST, t1i; + diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-with-cte.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-with-cte.sql new file mode 100644 index 000000000000..e65cb9106c1d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-with-cte.sql @@ -0,0 +1,287 @@ +-- A test suite for in with cte in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- outside CTE +-- TC 01.01 +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1a = "val1a") +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1b IN (SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 0); + +-- TC 01.02 +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1) +SELECT count(distinct(t1a)), t1b, t1c +FROM t1 +WHERE t1b IN + ( + SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 0 + UNION + SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 5 + UNION ALL + SELECT cte1.t1b + FROM cte1 + INTERSECT + SELECT cte1.t1b + FROM cte1 + UNION + SELECT cte1.t1b + FROM cte1 ) +GROUP BY t1a, t1b, t1c +HAVING t1c IS NOT NULL; + +-- TC 01.03 +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c, + t1d, + t1e + FROM t1) +SELECT t1a, + t1b, + t1c, + t1h +FROM t1 +WHERE t1c IN + ( + SELECT cte1.t1c + FROM cte1 + JOIN cte1 cte2 + on cte1.t1b > cte2.t1b + FULL OUTER JOIN cte1 cte3 + ON cte1.t1c = cte3.t1c + LEFT JOIN cte1 cte4 + ON cte1.t1d = cte4.t1d + INNER JOIN cte1 cte5 + ON cte1.t1b < cte5.t1b + LEFT OUTER JOIN cte1 cte6 + ON cte1.t1d > cte6.t1d); + +-- CTE inside and outside +-- TC 01.04 +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1b IN (SELECT t2b + FROM t2 + RIGHT JOIN t1 + ON t1c = t2c + LEFT JOIN t3 + ON t2d = t3d) + AND t1a = "val1b") +SELECT * +FROM (SELECT * + FROM cte1 + JOIN cte1 cte2 + ON cte1.t1b > 5 + AND cte1.t1a = cte2.t1a + FULL OUTER JOIN cte1 cte3 + ON cte1.t1a = cte3.t1a + INNER JOIN cte1 cte4 + ON cte1.t1b = cte4.t1b) s; + +-- TC 01.05 +WITH cte1 AS +( + SELECT t1a, + t1b, + t1h + FROM t1 + WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1b < t2b)) +SELECT Count(DISTINCT t1a), + t1b +FROM ( + SELECT cte1.t1a, + cte1.t1b + FROM cte1 + JOIN cte1 cte2 + on cte1.t1h >= cte2.t1h) s +WHERE t1b IN + ( + SELECT t1b + FROM t1) +GROUP BY t1b; + +-- TC 01.06 +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c + FROM t1 + WHERE t1b IN + ( + SELECT t2b + FROM t2 FULL OUTER JOIN T3 on t2a = t3a + WHERE t1c = t2c) AND + t1a = "val1b") +SELECT * +FROM ( + SELECT * + FROM cte1 + INNER JOIN cte1 cte2 ON cte1.t1a = cte2.t1a + RIGHT OUTER JOIN cte1 cte3 ON cte1.t1b = cte3.t1b + LEFT OUTER JOIN cte1 cte4 ON cte1.t1c = cte4.t1c + ) s +; + +-- TC 01.07 +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c = t2c)) +SELECT Count(DISTINCT( s.t1a )), + s.t1b +FROM (SELECT cte1.t1a, + cte1.t1b + FROM cte1 + RIGHT OUTER JOIN cte1 cte2 + ON cte1.t1a = cte2.t1a) s +GROUP BY s.t1b; + +-- TC 01.08 +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1 + WHERE t1b IN + ( + SELECT t2b + FROM t2 + WHERE t1c = t2c)) +SELECT DISTINCT(s.t1b) +FROM ( + SELECT cte1.t1b + FROM cte1 + LEFT OUTER JOIN cte1 cte2 + ON cte1.t1b = cte2.t1b) s +WHERE s.t1b IN + ( + SELECT t1.t1b + FROM t1 INNER + JOIN cte1 + ON t1.t1a = cte1.t1a); + +-- CTE with NOT IN +-- TC 01.09 +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1a = "val1d") +SELECT t1a, + t1b, + t1c, + t1h +FROM t1 +WHERE t1b NOT IN (SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b < 0) AND + t1c > 10; + +-- TC 01.10 +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c, + t1d, + t1h + FROM t1 + WHERE t1d NOT IN + ( + SELECT t2d + FROM t2 + FULL OUTER JOIN t3 ON t2a = t3a + JOIN t1 on t1b = t2b)) +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1b NOT IN + ( + SELECT cte1.t1b + FROM cte1 INNER + JOIN cte1 cte2 ON cte1.t1a = cte2.t1a + RIGHT JOIN cte1 cte3 ON cte1.t1b = cte3.t1b + JOIN cte1 cte4 ON cte1.t1c = cte4.t1c) AND + t1c IS NOT NULL +ORDER BY t1c DESC; + diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-group-by.sql new file mode 100644 index 000000000000..58cf109e136c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-group-by.sql @@ -0,0 +1,101 @@ +-- A test suite for NOT IN GROUP BY in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + + +-- correlated IN subquery +-- GROUP BY in parent side +-- TC 01.01 +SELECT t1a, + Avg(t1b) +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2) +GROUP BY t1a; + +-- TC 01.02 +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1h < t2h) +GROUP BY t1a; + +-- TC 01.03 +SELECT Count(*) +FROM (SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t3a + FROM t3 + WHERE t3h != t2h)) t2 +WHERE t2b NOT IN (SELECT Min(t2b) + FROM t2 + WHERE t2b = t2b + GROUP BY t2c); + +-- TC 01.04 +SELECT t1a, + max(t1b) +FROM t1 +WHERE t1c NOT IN (SELECT Max(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a) +GROUP BY t1a; + +-- TC 01.05 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2a NOT IN (SELECT Min(t3a) + FROM t3 + WHERE t3a = t2a + GROUP BY t3b) order by t2a); diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql new file mode 100644 index 000000000000..e09b91f18de0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql @@ -0,0 +1,167 @@ +-- A test suite for not-in-joins in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- different not JOIN in parent side +-- TC 01.01 +SELECT t1a, + t1b, + t1c, + t3a, + t3b, + t3c +FROM t1 + JOIN t3 +WHERE t1a NOT IN (SELECT t2a + FROM t2) + AND t1b = t3b; + +-- TC 01.02 +SELECT t1a, + t1b, + t1c, + count(distinct(t3a)), + t3b, + t3c +FROM t1 +FULL OUTER JOIN t3 on t1b != t3b +RIGHT JOIN t2 on t1c = t2c +where t1a NOT IN + ( + SELECT t2a + FROM t2 + WHERE t2c NOT IN + ( + SELECT t1c + FROM t1 + WHERE t1a = t2a)) +AND t1b != t3b +AND t1d = t2d +GROUP BY t1a, t1b, t1c, t3a, t3b, t3c +HAVING count(distinct(t3a)) >= 1 +ORDER BY t1a, t3b; + +-- TC 01.03 +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 + LEFT JOIN t3 on t2b = t3b + WHERE t1d = t2d + ) +AND t1d NOT IN + ( + SELECT t2d + FROM t2 + RIGHT JOIN t1 on t2e = t1e + WHERE t1a = t2a); + +-- TC 01.04 +SELECT Count(DISTINCT( t1a )), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +GROUP BY t1b, + t1c, + t1d +HAVING t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1d = t2d) +ORDER BY t1b DESC; + +-- TC 01.05 +SELECT COUNT(DISTINCT(t1a)), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 INNER + JOIN t1 ON t1a = t2a) +GROUP BY t1b, + t1c, + t1d +HAVING t1b < sum(t1c); + +-- TC 01.06 +SELECT COUNT(DISTINCT(t1a)), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 INNER + JOIN t1 + ON t1a = t2a) +AND t1d NOT IN + ( + SELECT t2d + FROM t2 + INNER JOIN t3 + ON t2b = t3b ) +GROUP BY t1b, + t1c, + t1d +HAVING t1b < sum(t1c); + diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql new file mode 100644 index 000000000000..f19567d2fac2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/simple-in.sql @@ -0,0 +1,136 @@ +-- A test suite for simple IN predicate subquery +-- It includes correlated cases. + +create temporary view t1 as select * from values + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- simple select +-- TC 01.01 +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2); + +-- TC 01.02 +SELECT * +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a); + +-- TC 01.03 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t1a != t2a); + +-- TC 01.04 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t1a = t2a + OR t1b > t2b); + +-- TC 01.05 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2i IN (SELECT t3i + FROM t3 + WHERE t2c = t3c)); + +-- TC 01.06 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2a IN (SELECT t3a + FROM t3 + WHERE t2c = t3c + AND t2b IS NOT NULL)); + +-- simple select for NOT IN +-- TC 01.07 +SELECT DISTINCT( t1a ), + t1b, + t1h +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2); + +-- DDLs +create temporary view a as select * from values + (1, 1), (2, 1), (null, 1), (1, 3), (null, 3), (1, null), (null, 2) + as a(a1, a2); + +create temporary view b as select * from values + (1, 1, 2), (null, 3, 2), (1, null, 2), (1, 2, null) + as b(b1, b2, b3); + +-- TC 02.01 +SELECT a1, a2 +FROM a +WHERE a1 NOT IN (SELECT b.b1 + FROM b + WHERE a.a2 = b.b2) +; + +-- TC 02.02 +SELECT a1, a2 +FROM a +WHERE a1 NOT IN (SELECT b.b1 + FROM b + WHERE a.a2 = b.b2 + AND b.b3 > 1) +; diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/invalid-correlation.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/invalid-correlation.sql new file mode 100644 index 000000000000..e22cade93679 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/invalid-correlation.sql @@ -0,0 +1,72 @@ +-- The test file contains negative test cases +-- of invalid queries where error messages are expected. + +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c); + +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES + (1, 0, 1) +AS t2(t2a, t2b, t2c); + +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES + (3, 1, 2) +AS t3(t3a, t3b, t3c); + +-- TC 01.01 +-- The column t2b in the SELECT of the subquery is invalid +-- because it is neither an aggregate function nor a GROUP BY column. +SELECT t1a, t2b +FROM t1, t2 +WHERE t1b = t2c +AND t2b = (SELECT max(avg) + FROM (SELECT t2b, avg(t2b) avg + FROM t2 + WHERE t2a = t1.t1b + ) + ) +; + +-- TC 01.02 +-- Invalid due to the column t2b not part of the output from table t2. +SELECT * +FROM t1 +WHERE t1a IN (SELECT min(t2a) + FROM t2 + GROUP BY t2c + HAVING t2c IN (SELECT max(t3c) + FROM t3 + GROUP BY t3b + HAVING t3b > t2b )) +; + +-- TC 01.03 +-- Invalid due to mixure of outer and local references under an AggegatedExpression +-- in a correlated predicate +SELECT t1a +FROM t1 +GROUP BY 1 +HAVING EXISTS (SELECT 1 + FROM t2 + WHERE t2a < min(t1a + t2a)); + +-- TC 01.04 +-- Invalid due to mixure of outer and local references under an AggegatedExpression +SELECT t1a +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE EXISTS (SELECT 1 + FROM t3 + GROUP BY 1 + HAVING min(t2a + t3a) > 1)); + +-- TC 01.05 +-- Invalid due to outer reference appearing in projection list +SELECT t1a +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE EXISTS (SELECT min(t2a) + FROM t3)); + diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql new file mode 100644 index 000000000000..b15f4da81dd9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql @@ -0,0 +1,47 @@ +-- The test file contains negative test cases +-- of invalid queries where error messages are expected. + +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c); + +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES + (1, 0, 1) +AS t2(t2a, t2b, t2c); + +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES + (3, 1, 2) +AS t3(t3a, t3b, t3c); + +-- TC 01.01 +SELECT + ( SELECT max(t2b), min(t2b) + FROM t2 + WHERE t2.t2b = t1.t1b + GROUP BY t2.t2b + ) +FROM t1; + +-- TC 01.01 +SELECT + ( SELECT max(t2b), min(t2b) + FROM t2 + WHERE t2.t2b > 0 + GROUP BY t2.t2b + ) +FROM t1; + +-- TC 01.03 +SELECT * FROM t1 +WHERE +t1a IN (SELECT t2a, t2b + FROM t2 + WHERE t1a = t2a); + +-- TC 01.04 +SELECT * FROM T1 +WHERE +(t1a, t1b) IN (SELECT t2a + FROM t2 + WHERE t1a = t2a); + diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-predicate.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-predicate.sql new file mode 100644 index 000000000000..fb0d07fbdace --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-predicate.sql @@ -0,0 +1,271 @@ +-- A test suite for scalar subquery in predicate context + +CREATE OR REPLACE TEMPORARY VIEW p AS VALUES (1, 1) AS T(pk, pv); +CREATE OR REPLACE TEMPORARY VIEW c AS VALUES (1, 1) AS T(ck, cv); + +-- SPARK-18814.1: Simplified version of TPCDS-Q32 +SELECT pk, cv +FROM p, c +WHERE p.pk = c.ck +AND c.cv = (SELECT avg(c1.cv) + FROM c c1 + WHERE c1.ck = p.pk); + +-- SPARK-18814.2: Adding stack of aggregates +SELECT pk, cv +FROM p, c +WHERE p.pk = c.ck +AND c.cv = (SELECT max(avg) + FROM (SELECT c1.cv, avg(c1.cv) avg + FROM c c1 + WHERE c1.ck = p.pk + GROUP BY c1.cv)); + +create temporary view t1 as select * from values + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ('val1d', null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ('val1d', null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ('val3a', 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ('val3a', 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ('val3c', 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ('val3c', 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ('val1b', null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ('val3b', 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val3b', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- Group 1: scalar subquery in predicate context +-- no correlation +-- TC 01.01 +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2); + +-- TC 01.02 +SELECT t1a, t1d, t1f +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2) +AND t1b > (SELECT min(t3b) + FROM t3); + +-- TC 01.03 +SELECT t1a, t1h +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2) +OR t1b = (SELECT min(t3b) + FROM t3 + WHERE t3b > 10); + +-- TC 01.04 +-- scalar subquery over outer join +SELECT t1a, t1b, t2d +FROM t1 LEFT JOIN t2 + ON t1a = t2a +WHERE t1b = (SELECT min(t3b) + FROM t3); + +-- TC 01.05 +-- test casting +SELECT t1a, t1b, t1g +FROM t1 +WHERE t1c + 5 = (SELECT max(t2e) + FROM t2); + +-- TC 01.06 +-- test casting +SELECT t1a, t1h +FROM t1 +WHERE date(t1h) = (SELECT min(t2i) + FROM t2); + +-- TC 01.07 +-- same table, expressions in scalar subquery +SELECT t2d, t1a +FROM t1, t2 +WHERE t1b = t2b +AND t2c + 1 = (SELECT max(t2c) + 1 + FROM t2, t1 + WHERE t2b = t1b); + +-- TC 01.08 +-- same table +SELECT DISTINCT t2a, max_t1g +FROM t2, (SELECT max(t1g) max_t1g, t1a + FROM t1 + GROUP BY t1a) t1 +WHERE t2a = t1a +AND max_t1g = (SELECT max(t1g) + FROM t1); + +-- TC 01.09 +-- more than one scalar subquery +SELECT t3b, t3c +FROM t3 +WHERE (SELECT max(t3c) + FROM t3 + WHERE t3b > 10) >= + (SELECT min(t3b) + FROM t3 + WHERE t3c > 0) +AND (t3b is null or t3c is null); + +-- Group 2: scalar subquery in predicate context +-- with correlation +-- TC 02.01 +SELECT t1a +FROM t1 +WHERE t1a < (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- TC 02.02 +SELECT t1a, t1c +FROM t1 +WHERE (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) IS NULL; + +-- TC 02.03 +SELECT t1a +FROM t1 +WHERE t1a = (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c + HAVING count(*) >= 0) +OR t1i > '2014-12-31'; + +-- TC 02.04 +-- t1 on the right of an outer join +-- can be reduced to inner join +SELECT count(t1a) +FROM t1 RIGHT JOIN t2 +ON t1d = t2d +WHERE t1a < (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- TC 02.05 +SELECT t1a +FROM t1 +WHERE t1b <= (SELECT max(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +AND t1b >= (SELECT min(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- TC 02.06 +-- set op +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +INTERSECT +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- TC 02.07.01 +-- set op +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +UNION ALL +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- TC 02.07.02 +-- set op +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +UNION DISTINCT +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- TC 02.08 +-- set op +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +MINUS +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- TC 02.09 +-- in HAVING clause +SELECT t1a +FROM t1 +GROUP BY t1a, t1c +HAVING max(t1b) <= (SELECT max(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql new file mode 100644 index 000000000000..eabbd0a93225 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql @@ -0,0 +1,130 @@ +-- A test suite for scalar subquery in SELECT clause + +create temporary view t1 as select * from values + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ('val1d', null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ('val1d', null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ('val3a', 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ('val3a', 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ('val3c', 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ('val3c', 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ('val1b', null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ('val3b', 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val3b', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- Group 1: scalar subquery in SELECT clause +-- no correlation +-- TC 01.01 +-- more than one scalar subquery +SELECT (SELECT min(t3d) FROM t3) min_t3d, + (SELECT max(t2h) FROM t2) max_t2h +FROM t1 +WHERE t1a = 'val1c'; + +-- TC 01.02 +-- scalar subquery in an IN subquery +SELECT t1a, count(*) +FROM t1 +WHERE t1c IN (SELECT (SELECT min(t3c) FROM t3) + FROM t2 + GROUP BY t2g + HAVING count(*) > 1) +GROUP BY t1a; + +-- TC 01.03 +-- under a set op +SELECT (SELECT min(t3d) FROM t3) min_t3d, + null +FROM t1 +WHERE t1a = 'val1c' +UNION +SELECT null, + (SELECT max(t2h) FROM t2) max_t2h +FROM t1 +WHERE t1a = 'val1c'; + +-- TC 01.04 +SELECT (SELECT min(t3c) FROM t3) min_t3d +FROM t1 +WHERE t1a = 'val1a' +INTERSECT +SELECT (SELECT min(t2c) FROM t2) min_t2d +FROM t1 +WHERE t1a = 'val1d'; + +-- TC 01.05 +SELECT q1.t1a, q2.t2a, q1.min_t3d, q2.avg_t3d +FROM (SELECT t1a, (SELECT min(t3d) FROM t3) min_t3d + FROM t1 + WHERE t1a IN ('val1e', 'val1c')) q1 + FULL OUTER JOIN + (SELECT t2a, (SELECT avg(t3d) FROM t3) avg_t3d + FROM t2 + WHERE t2a IN ('val1c', 'val2a')) q2 +ON q1.t1a = q2.t2a +AND q1.min_t3d < q2.avg_t3d; + +-- Group 2: scalar subquery in SELECT clause +-- with correlation +-- TC 02.01 +SELECT (SELECT min(t3d) FROM t3 WHERE t3.t3a = t1.t1a) min_t3d, + (SELECT max(t2h) FROM t2 WHERE t2.t2a = t1.t1a) max_t2h +FROM t1 +WHERE t1a = 'val1b'; + +-- TC 02.02 +SELECT (SELECT min(t3d) FROM t3 WHERE t3a = t1a) min_t3d +FROM t1 +WHERE t1a = 'val1b' +MINUS +SELECT (SELECT min(t3d) FROM t3) abs_min_t3d +FROM t1 +WHERE t1a = 'val1b'; + +-- TC 02.03 +SELECT t1a, t1b +FROM t1 +WHERE NOT EXISTS (SELECT (SELECT max(t2b) + FROM t2 LEFT JOIN t1 + ON t2a = t1a + WHERE t2c = t3c) dummy + FROM t3 + WHERE t3b < (SELECT max(t2b) + FROM t2 LEFT JOIN t1 + ON t2a = t1a + WHERE t2c = t3c) + AND t3a = t1a); diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/subquery-in-from.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/subquery-in-from.sql new file mode 100644 index 000000000000..1273b56b6344 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/subquery-in-from.sql @@ -0,0 +1,14 @@ +-- Aliased subqueries in FROM clause +SELECT * FROM (SELECT * FROM testData) AS t WHERE key = 1; + +FROM (SELECT * FROM testData WHERE key = 1) AS t SELECT *; + +-- Optional `AS` keyword +SELECT * FROM (SELECT * FROM testData) t WHERE key = 1; + +FROM (SELECT * FROM testData WHERE key = 1) t SELECT *; + +-- Disallow unaliased subqueries in FROM clause +SELECT * FROM (SELECT * FROM testData) WHERE key = 1; + +FROM (SELECT * FROM testData WHERE key = 1) SELECT *; diff --git a/sql/core/src/test/resources/sql-tests/inputs/table-aliases.sql b/sql/core/src/test/resources/sql-tests/inputs/table-aliases.sql new file mode 100644 index 000000000000..4cfd5f28afda --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/table-aliases.sql @@ -0,0 +1,27 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1) AS testData(a, b); + +-- Table column aliases in FROM clause +SELECT * FROM testData AS t(col1, col2) WHERE col1 = 1; + +SELECT * FROM testData AS t(col1, col2) WHERE col1 = 2; + +SELECT col1 AS k, SUM(col2) FROM testData AS t(col1, col2) GROUP BY k; + +-- Aliasing the wrong number of columns in the FROM clause +SELECT * FROM testData AS t(col1, col2, col3); + +SELECT * FROM testData AS t(col1); + +-- Check alias duplication +SELECT a AS col1, b AS col2 FROM testData AS t(c, d); + +-- Subquery aliases in FROM clause +SELECT * FROM (SELECT 1 AS a, 1 AS b) t(col1, col2); + +-- Aliases for join relations in FROM clause +CREATE OR REPLACE TEMPORARY VIEW src1 AS SELECT * FROM VALUES (1, "a"), (2, "b"), (3, "c") AS src1(id, v1); + +CREATE OR REPLACE TEMPORARY VIEW src2 AS SELECT * FROM VALUES (2, 1.0), (3, 3.2), (1, 8.5) AS src2(id, v2); + +SELECT * FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d); diff --git a/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql new file mode 100644 index 000000000000..72cd8ca9d872 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql @@ -0,0 +1,29 @@ +-- unresolved function +select * from dummy(3); + +-- range call with end +select * from range(6 + cos(3)); + +-- range call with start and end +select * from range(5, 10); + +-- range call with step +select * from range(0, 10, 2); + +-- range call with numPartitions +select * from range(0, 10, 1, 200); + +-- range call error +select * from range(1, 1, 1, 1, 1); + +-- range call with null +select * from range(1, null); + +-- range call with a mixed-case function name +select * from RaNgE(2); + +-- Explain +EXPLAIN select * from RaNgE(2); + +-- cross-join table valued functions +EXPLAIN EXTENDED SELECT * FROM range(3) CROSS JOIN range(3); diff --git a/sql/core/src/test/resources/sql-tests/inputs/tablesample-negative.sql b/sql/core/src/test/resources/sql-tests/inputs/tablesample-negative.sql new file mode 100644 index 000000000000..72508f59bee2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/tablesample-negative.sql @@ -0,0 +1,14 @@ +-- Negative testcases for tablesample +CREATE DATABASE mydb1; +USE mydb1; +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1; + +-- Negative tests: negative percentage +SELECT mydb1.t1 FROM t1 TABLESAMPLE (-1 PERCENT); + +-- Negative tests: percentage over 100 +-- The TABLESAMPLE clause samples without replacement, so the value of PERCENT must not exceed 100 +SELECT mydb1.t1 FROM t1 TABLESAMPLE (101 PERCENT); + +-- reset +DROP DATABASE mydb1 CASCADE; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udaf.sql b/sql/core/src/test/resources/sql-tests/inputs/udaf.sql new file mode 100644 index 000000000000..2183ba23afc3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udaf.sql @@ -0,0 +1,13 @@ +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(1), (2), (3), (4) +as t1(int_col1); + +CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg'; + +SELECT default.myDoubleAvg(int_col1) as my_avg from t1; + +SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1; + +CREATE FUNCTION udaf1 AS 'test.non.existent.udaf'; + +SELECT default.udaf1(int_col1) as udaf1 from t1; diff --git a/sql/core/src/test/resources/sql-tests/inputs/union.sql b/sql/core/src/test/resources/sql-tests/inputs/union.sql new file mode 100644 index 000000000000..e57d69eaad03 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/union.sql @@ -0,0 +1,43 @@ +CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (1, 'a'), (2, 'b') tbl(c1, c2); +CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (1.0, 1), (2.0, 4) tbl(c1, c2); + +-- Simple Union +SELECT * +FROM (SELECT * FROM t1 + UNION ALL + SELECT * FROM t1); + +-- Type Coerced Union +SELECT * +FROM (SELECT * FROM t1 + UNION ALL + SELECT * FROM t2 + UNION ALL + SELECT * FROM t2); + +-- Regression test for SPARK-18622 +SELECT a +FROM (SELECT 0 a, 0 b + UNION ALL + SELECT SUM(1) a, CAST(0 AS BIGINT) b + UNION ALL SELECT 0 a, 0 b) T; + +-- Regression test for SPARK-18841 Push project through union should not be broken by redundant alias removal. +CREATE OR REPLACE TEMPORARY VIEW p1 AS VALUES 1 T(col); +CREATE OR REPLACE TEMPORARY VIEW p2 AS VALUES 1 T(col); +CREATE OR REPLACE TEMPORARY VIEW p3 AS VALUES 1 T(col); +SELECT 1 AS x, + col +FROM (SELECT col AS col + FROM (SELECT p1.col AS col + FROM p1 CROSS JOIN p2 + UNION ALL + SELECT col + FROM p3) T1) T2; + +-- Clean-up +DROP VIEW IF EXISTS t1; +DROP VIEW IF EXISTS t2; +DROP VIEW IF EXISTS p1; +DROP VIEW IF EXISTS p2; +DROP VIEW IF EXISTS p3; diff --git a/sql/core/src/test/resources/sql-tests/inputs/window.sql b/sql/core/src/test/resources/sql-tests/inputs/window.sql new file mode 100644 index 000000000000..c4bea34ec4cf --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/window.sql @@ -0,0 +1,103 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, null, null, null, null, null), +(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +AS testData(val, val_long, val_double, val_date, val_timestamp, cate); + +-- RowsBetween +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData +ORDER BY cate, val; +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val +ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long +ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long; + +-- RangeBetween +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE 1 PRECEDING) FROM testData +ORDER BY cate, val; +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long +RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long; +SELECT val_double, cate, sum(val_double) OVER(PARTITION BY cate ORDER BY val_double +RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY cate, val_double; +SELECT val_date, cate, max(val_date) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY cate, val_date; +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY cate, val_timestamp; + +-- RangeBetween with reverse OrderBy +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val DESC +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; + +-- Invalid window frame +SELECT val, cate, count(val) OVER(PARTITION BY cate +ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val, cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY current_timestamp +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val; + + +-- Window functions +SELECT val, cate, +max(val) OVER w AS max, +min(val) OVER w AS min, +min(val) OVER w AS min, +count(val) OVER w AS count, +sum(val) OVER w AS sum, +avg(val) OVER w AS avg, +stddev(val) OVER w AS stddev, +first_value(val) OVER w AS first_value, +first_value(val, true) OVER w AS first_value_ignore_null, +first_value(val, false) OVER w AS first_value_contain_null, +last_value(val) OVER w AS last_value, +last_value(val, true) OVER w AS last_value_ignore_null, +last_value(val, false) OVER w AS last_value_contain_null, +rank() OVER w AS rank, +dense_rank() OVER w AS dense_rank, +cume_dist() OVER w AS cume_dist, +percent_rank() OVER w AS percent_rank, +ntile(2) OVER w AS ntile, +row_number() OVER w AS row_number, +var_pop(val) OVER w AS var_pop, +var_samp(val) OVER w AS var_samp, +approx_count_distinct(val) OVER w AS approx_count_distinct +FROM testData +WINDOW w AS (PARTITION BY cate ORDER BY val) +ORDER BY cate, val; + +-- Null inputs +SELECT val, cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val; + +-- OrderBy not specified +SELECT val, cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, val; + +-- Over clause is empty +SELECT val, cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val; + +-- first_value()/last_value() over () +SELECT val, cate, +first_value(false) OVER w AS first_value, +first_value(true, true) OVER w AS first_value_ignore_null, +first_value(false, false) OVER w AS first_value_contain_null, +last_value(false) OVER w AS last_value, +last_value(true, true) OVER w AS last_value_ignore_null, +last_value(false, false) OVER w AS last_value_contain_null +FROM testData +WINDOW w AS () +ORDER BY cate, val; diff --git a/sql/core/src/test/resources/sql-tests/results/array.sql.out b/sql/core/src/test/resources/sql-tests/results/array.sql.out new file mode 100644 index 000000000000..981b2504bcaa --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/array.sql.out @@ -0,0 +1,162 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 12 + + +-- !query 0 +create temporary view data as select * from values + ("one", array(11, 12, 13), array(array(111, 112, 113), array(121, 122, 123))), + ("two", array(21, 22, 23), array(array(211, 212, 213), array(221, 222, 223))) + as data(a, b, c) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +select * from data +-- !query 1 schema +struct,c:array>> +-- !query 1 output +one [11,12,13] [[111,112,113],[121,122,123]] +two [21,22,23] [[211,212,213],[221,222,223]] + + +-- !query 2 +select a, b[0], b[0] + b[1] from data +-- !query 2 schema +struct +-- !query 2 output +one 11 23 +two 21 43 + + +-- !query 3 +select a, c[0][0] + c[0][0 + 1] from data +-- !query 3 schema +struct +-- !query 3 output +one 223 +two 423 + + +-- !query 4 +create temporary view primitive_arrays as select * from values ( + array(true), + array(2Y, 1Y), + array(2S, 1S), + array(2, 1), + array(2L, 1L), + array(9223372036854775809, 9223372036854775808), + array(2.0D, 1.0D), + array(float(2.0), float(1.0)), + array(date '2016-03-14', date '2016-03-13'), + array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000') +) as primitive_arrays( + boolean_array, + tinyint_array, + smallint_array, + int_array, + bigint_array, + decimal_array, + double_array, + float_array, + date_array, + timestamp_array +) +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +select * from primitive_arrays +-- !query 5 schema +struct,tinyint_array:array,smallint_array:array,int_array:array,bigint_array:array,decimal_array:array,double_array:array,float_array:array,date_array:array,timestamp_array:array> +-- !query 5 output +[true] [2,1] [2,1] [2,1] [2,1] [9223372036854775809,9223372036854775808] [2.0,1.0] [2.0,1.0] [2016-03-14,2016-03-13] [2016-11-15 20:54:00.0,2016-11-12 20:54:00.0] + + +-- !query 6 +select + array_contains(boolean_array, true), array_contains(boolean_array, false), + array_contains(tinyint_array, 2Y), array_contains(tinyint_array, 0Y), + array_contains(smallint_array, 2S), array_contains(smallint_array, 0S), + array_contains(int_array, 2), array_contains(int_array, 0), + array_contains(bigint_array, 2L), array_contains(bigint_array, 0L), + array_contains(decimal_array, 9223372036854775809), array_contains(decimal_array, 1), + array_contains(double_array, 2.0D), array_contains(double_array, 0.0D), + array_contains(float_array, float(2.0)), array_contains(float_array, float(0.0)), + array_contains(date_array, date '2016-03-14'), array_contains(date_array, date '2016-01-01'), + array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') +from primitive_arrays +-- !query 6 schema +struct +-- !query 6 output +true false true false true false true false true false true false true false true false true false true false + + +-- !query 7 +select array_contains(b, 11), array_contains(c, array(111, 112, 113)) from data +-- !query 7 schema +struct +-- !query 7 output +false false +true true + + +-- !query 8 +select + sort_array(boolean_array), + sort_array(tinyint_array), + sort_array(smallint_array), + sort_array(int_array), + sort_array(bigint_array), + sort_array(decimal_array), + sort_array(double_array), + sort_array(float_array), + sort_array(date_array), + sort_array(timestamp_array) +from primitive_arrays +-- !query 8 schema +struct,sort_array(tinyint_array, true):array,sort_array(smallint_array, true):array,sort_array(int_array, true):array,sort_array(bigint_array, true):array,sort_array(decimal_array, true):array,sort_array(double_array, true):array,sort_array(float_array, true):array,sort_array(date_array, true):array,sort_array(timestamp_array, true):array> +-- !query 8 output +[true] [1,2] [1,2] [1,2] [1,2] [9223372036854775808,9223372036854775809] [1.0,2.0] [1.0,2.0] [2016-03-13,2016-03-14] [2016-11-12 20:54:00.0,2016-11-15 20:54:00.0] + + +-- !query 9 +select sort_array(array('b', 'd'), '1') +-- !query 9 schema +struct<> +-- !query 9 output +org.apache.spark.sql.AnalysisException +cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7 + + +-- !query 10 +select sort_array(array('b', 'd'), cast(NULL as boolean)) +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +cannot resolve 'sort_array(array('b', 'd'), CAST(NULL AS BOOLEAN))' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7 + + +-- !query 11 +select + size(boolean_array), + size(tinyint_array), + size(smallint_array), + size(int_array), + size(bigint_array), + size(decimal_array), + size(double_array), + size(float_array), + size(date_array), + size(timestamp_array) +from primitive_arrays +-- !query 11 schema +struct +-- !query 11 output +1 2 2 2 2 2 2 2 2 2 diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out new file mode 100644 index 000000000000..9c5f4554d9fe --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -0,0 +1,201 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 24 + + +-- !query 0 +SELECT CAST('1.23' AS int) +-- !query 0 schema +struct +-- !query 0 output +1 + + +-- !query 1 +SELECT CAST('1.23' AS long) +-- !query 1 schema +struct +-- !query 1 output +1 + + +-- !query 2 +SELECT CAST('-4.56' AS int) +-- !query 2 schema +struct +-- !query 2 output +-4 + + +-- !query 3 +SELECT CAST('-4.56' AS long) +-- !query 3 schema +struct +-- !query 3 output +-4 + + +-- !query 4 +SELECT CAST('abc' AS int) +-- !query 4 schema +struct +-- !query 4 output +NULL + + +-- !query 5 +SELECT CAST('abc' AS long) +-- !query 5 schema +struct +-- !query 5 output +NULL + + +-- !query 6 +SELECT CAST('1234567890123' AS int) +-- !query 6 schema +struct +-- !query 6 output +NULL + + +-- !query 7 +SELECT CAST('12345678901234567890123' AS long) +-- !query 7 schema +struct +-- !query 7 output +NULL + + +-- !query 8 +SELECT CAST('' AS int) +-- !query 8 schema +struct +-- !query 8 output +NULL + + +-- !query 9 +SELECT CAST('' AS long) +-- !query 9 schema +struct +-- !query 9 output +NULL + + +-- !query 10 +SELECT CAST(NULL AS int) +-- !query 10 schema +struct +-- !query 10 output +NULL + + +-- !query 11 +SELECT CAST(NULL AS long) +-- !query 11 schema +struct +-- !query 11 output +NULL + + +-- !query 12 +SELECT CAST('123.a' AS int) +-- !query 12 schema +struct +-- !query 12 output +NULL + + +-- !query 13 +SELECT CAST('123.a' AS long) +-- !query 13 schema +struct +-- !query 13 output +NULL + + +-- !query 14 +SELECT CAST('-2147483648' AS int) +-- !query 14 schema +struct +-- !query 14 output +-2147483648 + + +-- !query 15 +SELECT CAST('-2147483649' AS int) +-- !query 15 schema +struct +-- !query 15 output +NULL + + +-- !query 16 +SELECT CAST('2147483647' AS int) +-- !query 16 schema +struct +-- !query 16 output +2147483647 + + +-- !query 17 +SELECT CAST('2147483648' AS int) +-- !query 17 schema +struct +-- !query 17 output +NULL + + +-- !query 18 +SELECT CAST('-9223372036854775808' AS long) +-- !query 18 schema +struct +-- !query 18 output +-9223372036854775808 + + +-- !query 19 +SELECT CAST('-9223372036854775809' AS long) +-- !query 19 schema +struct +-- !query 19 output +NULL + + +-- !query 20 +SELECT CAST('9223372036854775807' AS long) +-- !query 20 schema +struct +-- !query 20 output +9223372036854775807 + + +-- !query 21 +SELECT CAST('9223372036854775808' AS long) +-- !query 21 schema +struct +-- !query 21 output +NULL + + +-- !query 22 +DESC FUNCTION boolean +-- !query 22 schema +struct +-- !query 22 output +Class: org.apache.spark.sql.catalyst.expressions.Cast +Function: boolean +Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. + + +-- !query 23 +DESC FUNCTION EXTENDED boolean +-- !query 23 schema +struct +-- !query 23 output +Class: org.apache.spark.sql.catalyst.expressions.Cast +Extended Usage: + No example/argument for boolean. + +Function: boolean +Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out new file mode 100644 index 000000000000..ba8bc936f0c7 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -0,0 +1,306 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 32 + + +-- !query 0 +CREATE TABLE test_change(a INT, b STRING, c INT) using parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +DESC test_change +-- !query 1 schema +struct +-- !query 1 output +a int +b string +c int + + +-- !query 2 +ALTER TABLE test_change CHANGE a a1 INT +-- !query 2 schema +struct<> +-- !query 2 output +org.apache.spark.sql.AnalysisException +ALTER TABLE CHANGE COLUMN is not supported for changing column 'a' with type 'IntegerType' to 'a1' with type 'IntegerType'; + + +-- !query 3 +DESC test_change +-- !query 3 schema +struct +-- !query 3 output +a int +b string +c int + + +-- !query 4 +ALTER TABLE test_change CHANGE a a STRING +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +ALTER TABLE CHANGE COLUMN is not supported for changing column 'a' with type 'IntegerType' to 'a' with type 'StringType'; + + +-- !query 5 +DESC test_change +-- !query 5 schema +struct +-- !query 5 output +a int +b string +c int + + +-- !query 6 +ALTER TABLE test_change CHANGE a a INT AFTER b +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.catalyst.parser.ParseException + +Operation not allowed: ALTER TABLE table [PARTITION partition_spec] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) + +== SQL == +ALTER TABLE test_change CHANGE a a INT AFTER b +^^^ + + +-- !query 7 +ALTER TABLE test_change CHANGE b b STRING FIRST +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.catalyst.parser.ParseException + +Operation not allowed: ALTER TABLE table [PARTITION partition_spec] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) + +== SQL == +ALTER TABLE test_change CHANGE b b STRING FIRST +^^^ + + +-- !query 8 +DESC test_change +-- !query 8 schema +struct +-- !query 8 output +a int +b string +c int + + +-- !query 9 +ALTER TABLE test_change CHANGE a a INT COMMENT 'this is column a' +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +ALTER TABLE test_change CHANGE b b STRING COMMENT '#*02?`' +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +ALTER TABLE test_change CHANGE c c INT COMMENT '' +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +DESC test_change +-- !query 12 schema +struct +-- !query 12 output +a int this is column a +b string #*02?` +c int + + +-- !query 13 +ALTER TABLE test_change CHANGE a a INT COMMENT 'this is column a' +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +DESC test_change +-- !query 14 schema +struct +-- !query 14 output +a int this is column a +b string #*02?` +c int + + +-- !query 15 +ALTER TABLE test_change CHANGE invalid_col invalid_col INT +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.AnalysisException +Invalid column reference 'invalid_col', table schema is 'StructType(StructField(a,IntegerType,true), StructField(b,StringType,true), StructField(c,IntegerType,true))'; + + +-- !query 16 +DESC test_change +-- !query 16 schema +struct +-- !query 16 output +a int this is column a +b string #*02?` +c int + + +-- !query 17 +ALTER TABLE test_change CHANGE a a1 STRING COMMENT 'this is column a1' AFTER b +-- !query 17 schema +struct<> +-- !query 17 output +org.apache.spark.sql.catalyst.parser.ParseException + +Operation not allowed: ALTER TABLE table [PARTITION partition_spec] CHANGE COLUMN ... FIRST | AFTER otherCol(line 1, pos 0) + +== SQL == +ALTER TABLE test_change CHANGE a a1 STRING COMMENT 'this is column a1' AFTER b +^^^ + + +-- !query 18 +DESC test_change +-- !query 18 schema +struct +-- !query 18 output +a int this is column a +b string #*02?` +c int + + +-- !query 19 +SET spark.sql.caseSensitive=false +-- !query 19 schema +struct +-- !query 19 output +spark.sql.caseSensitive false + + +-- !query 20 +ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A' +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +SET spark.sql.caseSensitive=true +-- !query 21 schema +struct +-- !query 21 output +spark.sql.caseSensitive true + + +-- !query 22 +ALTER TABLE test_change CHANGE a A INT COMMENT 'this is column A1' +-- !query 22 schema +struct<> +-- !query 22 output +org.apache.spark.sql.AnalysisException +ALTER TABLE CHANGE COLUMN is not supported for changing column 'a' with type 'IntegerType' to 'A' with type 'IntegerType'; + + +-- !query 23 +DESC test_change +-- !query 23 schema +struct +-- !query 23 output +a int this is column A +b string #*02?` +c int + + +-- !query 24 +CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one" +-- !query 24 schema +struct<> +-- !query 24 output + + + +-- !query 25 +ALTER TABLE temp_view CHANGE a a INT COMMENT 'this is column a' +-- !query 25 schema +struct<> +-- !query 25 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'temp_view' not found in database 'default'; + + +-- !query 26 +CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one" +-- !query 26 schema +struct<> +-- !query 26 output + + + +-- !query 27 +ALTER TABLE global_temp.global_temp_view CHANGE a a INT COMMENT 'this is column a' +-- !query 27 schema +struct<> +-- !query 27 output +org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +Database 'global_temp' not found; + + +-- !query 28 +CREATE TABLE partition_table(a INT, b STRING, c INT, d STRING) USING parquet PARTITIONED BY (c, d) +-- !query 28 schema +struct<> +-- !query 28 output + + + +-- !query 29 +ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a INT +-- !query 29 schema +struct<> +-- !query 29 output +org.apache.spark.sql.catalyst.parser.ParseException + +Operation not allowed: ALTER TABLE table PARTITION partition_spec CHANGE COLUMN(line 1, pos 0) + +== SQL == +ALTER TABLE partition_table PARTITION (c = 1) CHANGE COLUMN a new_a INT +^^^ + + +-- !query 30 +DROP TABLE test_change +-- !query 30 schema +struct<> +-- !query 30 output + + + +-- !query 31 +DROP TABLE partition_table +-- !query 31 schema +struct<> +-- !query 31 output + diff --git a/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out b/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out new file mode 100644 index 000000000000..b5a4f5c2bf65 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out @@ -0,0 +1,240 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 28 + + +-- !query 0 +CREATE DATABASE mydb1 +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +USE mydb1 +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1 +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE DATABASE mydb2 +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +USE mydb2 +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +CREATE TABLE t1 USING parquet AS SELECT 20 AS i1 +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SET spark.sql.crossJoin.enabled = true +-- !query 6 schema +struct +-- !query 6 output +spark.sql.crossJoin.enabled true + + +-- !query 7 +USE mydb1 +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +SELECT i1 FROM t1, mydb1.t1 +-- !query 8 schema +struct<> +-- !query 8 output +org.apache.spark.sql.AnalysisException +Reference 'i1' is ambiguous, could be: t1.i1, t1.i1.; line 1 pos 7 + + +-- !query 9 +SELECT t1.i1 FROM t1, mydb1.t1 +-- !query 9 schema +struct<> +-- !query 9 output +org.apache.spark.sql.AnalysisException +Reference 't1.i1' is ambiguous, could be: t1.i1, t1.i1.; line 1 pos 7 + + +-- !query 10 +SELECT mydb1.t1.i1 FROM t1, mydb1.t1 +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1, t1.i1]; line 1 pos 7 + + +-- !query 11 +SELECT i1 FROM t1, mydb2.t1 +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.AnalysisException +Reference 'i1' is ambiguous, could be: t1.i1, t1.i1.; line 1 pos 7 + + +-- !query 12 +SELECT t1.i1 FROM t1, mydb2.t1 +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +Reference 't1.i1' is ambiguous, could be: t1.i1, t1.i1.; line 1 pos 7 + + +-- !query 13 +USE mydb2 +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +SELECT i1 FROM t1, mydb1.t1 +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +Reference 'i1' is ambiguous, could be: t1.i1, t1.i1.; line 1 pos 7 + + +-- !query 15 +SELECT t1.i1 FROM t1, mydb1.t1 +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.AnalysisException +Reference 't1.i1' is ambiguous, could be: t1.i1, t1.i1.; line 1 pos 7 + + +-- !query 16 +SELECT i1 FROM t1, mydb2.t1 +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.AnalysisException +Reference 'i1' is ambiguous, could be: t1.i1, t1.i1.; line 1 pos 7 + + +-- !query 17 +SELECT t1.i1 FROM t1, mydb2.t1 +-- !query 17 schema +struct<> +-- !query 17 output +org.apache.spark.sql.AnalysisException +Reference 't1.i1' is ambiguous, could be: t1.i1, t1.i1.; line 1 pos 7 + + +-- !query 18 +SELECT db1.t1.i1 FROM t1, mydb2.t1 +-- !query 18 schema +struct<> +-- !query 18 output +org.apache.spark.sql.AnalysisException +cannot resolve '`db1.t1.i1`' given input columns: [t1.i1, t1.i1]; line 1 pos 7 + + +-- !query 19 +SET spark.sql.crossJoin.enabled = false +-- !query 19 schema +struct +-- !query 19 output +spark.sql.crossJoin.enabled false + + +-- !query 20 +USE mydb1 +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +SELECT mydb1.t1 FROM t1 +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.AnalysisException +cannot resolve '`mydb1.t1`' given input columns: [t1.i1]; line 1 pos 7 + + +-- !query 22 +SELECT t1.x.y.* FROM t1 +-- !query 22 schema +struct<> +-- !query 22 output +org.apache.spark.sql.AnalysisException +cannot resolve 't1.x.y.*' give input columns 'i1'; + + +-- !query 23 +SELECT t1 FROM mydb1.t1 +-- !query 23 schema +struct<> +-- !query 23 output +org.apache.spark.sql.AnalysisException +cannot resolve '`t1`' given input columns: [t1.i1]; line 1 pos 7 + + +-- !query 24 +USE mydb2 +-- !query 24 schema +struct<> +-- !query 24 output + + + +-- !query 25 +SELECT mydb1.t1.i1 FROM t1 +-- !query 25 schema +struct<> +-- !query 25 output +org.apache.spark.sql.AnalysisException +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1]; line 1 pos 7 + + +-- !query 26 +DROP DATABASE mydb1 CASCADE +-- !query 26 schema +struct<> +-- !query 26 output + + + +-- !query 27 +DROP DATABASE mydb2 CASCADE +-- !query 27 schema +struct<> +-- !query 27 output + diff --git a/sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out b/sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out new file mode 100644 index 000000000000..7c451c2aa5b5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out @@ -0,0 +1,140 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 17 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW view1 AS SELECT 2 AS i1 +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT view1.* FROM view1 +-- !query 1 schema +struct +-- !query 1 output +2 + + +-- !query 2 +SELECT * FROM view1 +-- !query 2 schema +struct +-- !query 2 output +2 + + +-- !query 3 +SELECT view1.i1 FROM view1 +-- !query 3 schema +struct +-- !query 3 output +2 + + +-- !query 4 +SELECT i1 FROM view1 +-- !query 4 schema +struct +-- !query 4 output +2 + + +-- !query 5 +SELECT a.i1 FROM view1 AS a +-- !query 5 schema +struct +-- !query 5 output +2 + + +-- !query 6 +SELECT i1 FROM view1 AS a +-- !query 6 schema +struct +-- !query 6 output +2 + + +-- !query 7 +DROP VIEW view1 +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +CREATE OR REPLACE GLOBAL TEMPORARY VIEW view1 as SELECT 1 as i1 +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +SELECT * FROM global_temp.view1 +-- !query 9 schema +struct +-- !query 9 output +1 + + +-- !query 10 +SELECT global_temp.view1.* FROM global_temp.view1 +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +cannot resolve 'global_temp.view1.*' give input columns 'i1'; + + +-- !query 11 +SELECT i1 FROM global_temp.view1 +-- !query 11 schema +struct +-- !query 11 output +1 + + +-- !query 12 +SELECT global_temp.view1.i1 FROM global_temp.view1 +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +cannot resolve '`global_temp.view1.i1`' given input columns: [view1.i1]; line 1 pos 7 + + +-- !query 13 +SELECT view1.i1 FROM global_temp.view1 +-- !query 13 schema +struct +-- !query 13 output +1 + + +-- !query 14 +SELECT a.i1 FROM global_temp.view1 AS a +-- !query 14 schema +struct +-- !query 14 output +1 + + +-- !query 15 +SELECT i1 FROM global_temp.view1 AS a +-- !query 15 schema +struct +-- !query 15 output +1 + + +-- !query 16 +DROP VIEW global_temp.view1 +-- !query 16 schema +struct<> +-- !query 16 output + diff --git a/sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out b/sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out new file mode 100644 index 000000000000..d3ca4443cce5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out @@ -0,0 +1,447 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 54 + + +-- !query 0 +CREATE DATABASE mydb1 +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +USE mydb1 +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1 +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE DATABASE mydb2 +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +USE mydb2 +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +CREATE TABLE t1 USING parquet AS SELECT 20 AS i1 +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +USE mydb1 +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +SELECT i1 FROM t1 +-- !query 7 schema +struct +-- !query 7 output +1 + + +-- !query 8 +SELECT i1 FROM mydb1.t1 +-- !query 8 schema +struct +-- !query 8 output +1 + + +-- !query 9 +SELECT t1.i1 FROM t1 +-- !query 9 schema +struct +-- !query 9 output +1 + + +-- !query 10 +SELECT t1.i1 FROM mydb1.t1 +-- !query 10 schema +struct +-- !query 10 output +1 + + +-- !query 11 +SELECT mydb1.t1.i1 FROM t1 +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.AnalysisException +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1]; line 1 pos 7 + + +-- !query 12 +SELECT mydb1.t1.i1 FROM mydb1.t1 +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1]; line 1 pos 7 + + +-- !query 13 +USE mydb2 +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +SELECT i1 FROM t1 +-- !query 14 schema +struct +-- !query 14 output +20 + + +-- !query 15 +SELECT i1 FROM mydb1.t1 +-- !query 15 schema +struct +-- !query 15 output +1 + + +-- !query 16 +SELECT t1.i1 FROM t1 +-- !query 16 schema +struct +-- !query 16 output +20 + + +-- !query 17 +SELECT t1.i1 FROM mydb1.t1 +-- !query 17 schema +struct +-- !query 17 output +1 + + +-- !query 18 +SELECT mydb1.t1.i1 FROM mydb1.t1 +-- !query 18 schema +struct<> +-- !query 18 output +org.apache.spark.sql.AnalysisException +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1]; line 1 pos 7 + + +-- !query 19 +USE mydb1 +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +SELECT t1.* FROM t1 +-- !query 20 schema +struct +-- !query 20 output +1 + + +-- !query 21 +SELECT mydb1.t1.* FROM mydb1.t1 +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.AnalysisException +cannot resolve 'mydb1.t1.*' give input columns 'i1'; + + +-- !query 22 +SELECT t1.* FROM mydb1.t1 +-- !query 22 schema +struct +-- !query 22 output +1 + + +-- !query 23 +USE mydb2 +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +SELECT t1.* FROM t1 +-- !query 24 schema +struct +-- !query 24 output +20 + + +-- !query 25 +SELECT mydb1.t1.* FROM mydb1.t1 +-- !query 25 schema +struct<> +-- !query 25 output +org.apache.spark.sql.AnalysisException +cannot resolve 'mydb1.t1.*' give input columns 'i1'; + + +-- !query 26 +SELECT t1.* FROM mydb1.t1 +-- !query 26 schema +struct +-- !query 26 output +1 + + +-- !query 27 +SELECT a.* FROM mydb1.t1 AS a +-- !query 27 schema +struct +-- !query 27 output +1 + + +-- !query 28 +USE mydb1 +-- !query 28 schema +struct<> +-- !query 28 output + + + +-- !query 29 +CREATE TABLE t3 USING parquet AS SELECT * FROM VALUES (4,1), (3,1) AS t3(c1, c2) +-- !query 29 schema +struct<> +-- !query 29 output + + + +-- !query 30 +CREATE TABLE t4 USING parquet AS SELECT * FROM VALUES (4,1), (2,1) AS t4(c2, c3) +-- !query 30 schema +struct<> +-- !query 30 output + + + +-- !query 31 +SELECT * FROM t3 WHERE c1 IN (SELECT c2 FROM t4 WHERE t4.c3 = t3.c2) +-- !query 31 schema +struct +-- !query 31 output +4 1 + + +-- !query 32 +SELECT * FROM mydb1.t3 WHERE c1 IN + (SELECT mydb1.t4.c2 FROM mydb1.t4 WHERE mydb1.t4.c3 = mydb1.t3.c2) +-- !query 32 schema +struct<> +-- !query 32 output +org.apache.spark.sql.AnalysisException +cannot resolve '`mydb1.t4.c3`' given input columns: [t4.c2, t4.c3]; line 2 pos 42 + + +-- !query 33 +SET spark.sql.crossJoin.enabled = true +-- !query 33 schema +struct +-- !query 33 output +spark.sql.crossJoin.enabled true + + +-- !query 34 +SELECT mydb1.t1.i1 FROM t1, mydb2.t1 +-- !query 34 schema +struct<> +-- !query 34 output +org.apache.spark.sql.AnalysisException +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1, t1.i1]; line 1 pos 7 + + +-- !query 35 +SELECT mydb1.t1.i1 FROM mydb1.t1, mydb2.t1 +-- !query 35 schema +struct<> +-- !query 35 output +org.apache.spark.sql.AnalysisException +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1, t1.i1]; line 1 pos 7 + + +-- !query 36 +USE mydb2 +-- !query 36 schema +struct<> +-- !query 36 output + + + +-- !query 37 +SELECT mydb1.t1.i1 FROM t1, mydb1.t1 +-- !query 37 schema +struct<> +-- !query 37 output +org.apache.spark.sql.AnalysisException +cannot resolve '`mydb1.t1.i1`' given input columns: [t1.i1, t1.i1]; line 1 pos 7 + + +-- !query 38 +SET spark.sql.crossJoin.enabled = false +-- !query 38 schema +struct +-- !query 38 output +spark.sql.crossJoin.enabled false + + +-- !query 39 +USE mydb1 +-- !query 39 schema +struct<> +-- !query 39 output + + + +-- !query 40 +CREATE TABLE t5(i1 INT, t5 STRUCT) USING parquet +-- !query 40 schema +struct<> +-- !query 40 output + + + +-- !query 41 +INSERT INTO t5 VALUES(1, (2, 3)) +-- !query 41 schema +struct<> +-- !query 41 output + + + +-- !query 42 +SELECT t5.i1 FROM t5 +-- !query 42 schema +struct +-- !query 42 output +1 + + +-- !query 43 +SELECT t5.t5.i1 FROM t5 +-- !query 43 schema +struct +-- !query 43 output +2 + + +-- !query 44 +SELECT t5.t5.i1 FROM mydb1.t5 +-- !query 44 schema +struct +-- !query 44 output +2 + + +-- !query 45 +SELECT t5.i1 FROM mydb1.t5 +-- !query 45 schema +struct +-- !query 45 output +1 + + +-- !query 46 +SELECT t5.* FROM mydb1.t5 +-- !query 46 schema +struct> +-- !query 46 output +1 {"i1":2,"i2":3} + + +-- !query 47 +SELECT t5.t5.* FROM mydb1.t5 +-- !query 47 schema +struct +-- !query 47 output +2 3 + + +-- !query 48 +SELECT mydb1.t5.t5.i1 FROM mydb1.t5 +-- !query 48 schema +struct<> +-- !query 48 output +org.apache.spark.sql.AnalysisException +cannot resolve '`mydb1.t5.t5.i1`' given input columns: [t5.i1, t5.t5]; line 1 pos 7 + + +-- !query 49 +SELECT mydb1.t5.t5.i2 FROM mydb1.t5 +-- !query 49 schema +struct<> +-- !query 49 output +org.apache.spark.sql.AnalysisException +cannot resolve '`mydb1.t5.t5.i2`' given input columns: [t5.i1, t5.t5]; line 1 pos 7 + + +-- !query 50 +SELECT mydb1.t5.* FROM mydb1.t5 +-- !query 50 schema +struct<> +-- !query 50 output +org.apache.spark.sql.AnalysisException +cannot resolve 'mydb1.t5.*' give input columns 'i1, t5'; + + +-- !query 51 +USE default +-- !query 51 schema +struct<> +-- !query 51 output + + + +-- !query 52 +DROP DATABASE mydb1 CASCADE +-- !query 52 schema +struct<> +-- !query 52 output + + + +-- !query 53 +DROP DATABASE mydb2 CASCADE +-- !query 53 schema +struct<> +-- !query 53 output + diff --git a/sql/core/src/test/resources/sql-tests/results/comparator.sql.out b/sql/core/src/test/resources/sql-tests/results/comparator.sql.out new file mode 100644 index 000000000000..afc7b5448b7b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/comparator.sql.out @@ -0,0 +1,18 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 2 + + +-- !query 0 +select x'00' < x'0f' +-- !query 0 schema +struct<(X'00' < X'0F'):boolean> +-- !query 0 output +true + + +-- !query 1 +select x'00' < x'ff' +-- !query 1 schema +struct<(X'00' < X'FF'):boolean> +-- !query 1 output +true diff --git a/sql/core/src/test/resources/sql-tests/results/cross-join.sql.out b/sql/core/src/test/resources/sql-tests/results/cross-join.sql.out new file mode 100644 index 000000000000..3833c42bdfec --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/cross-join.sql.out @@ -0,0 +1,140 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 13 + + +-- !query 0 +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT * FROM nt1 cross join nt2 +-- !query 2 schema +struct +-- !query 2 output +one 1 one 1 +one 1 one 5 +one 1 two 22 +three 3 one 1 +three 3 one 5 +three 3 two 22 +two 2 one 1 +two 2 one 5 +two 2 two 22 + + +-- !query 3 +SELECT * FROM nt1 cross join nt2 where nt1.k = nt2.k +-- !query 3 schema +struct +-- !query 3 output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query 4 +SELECT * FROM nt1 cross join nt2 on (nt1.k = nt2.k) +-- !query 4 schema +struct +-- !query 4 output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query 5 +SELECT * FROM nt1 cross join nt2 where nt1.v1 = 1 and nt2.v2 = 22 +-- !query 5 schema +struct +-- !query 5 output +one 1 two 22 + + +-- !query 6 +SELECT a.key, b.key FROM +(SELECT k key FROM nt1 WHERE v1 < 2) a +CROSS JOIN +(SELECT k key FROM nt2 WHERE v2 = 22) b +-- !query 6 schema +struct +-- !query 6 output +one two + + +-- !query 7 +create temporary view A(a, va) as select * from nt1 +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +create temporary view B(b, vb) as select * from nt1 +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +create temporary view C(c, vc) as select * from nt1 +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +create temporary view D(d, vd) as select * from nt1 +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +select * from ((A join B on (a = b)) cross join C) join D on (a = d) +-- !query 11 schema +struct +-- !query 11 output +one 1 one 1 one 1 one 1 +one 1 one 1 three 3 one 1 +one 1 one 1 two 2 one 1 +three 3 three 3 one 1 three 3 +three 3 three 3 three 3 three 3 +three 3 three 3 two 2 three 3 +two 2 two 2 one 1 two 2 +two 2 two 2 three 3 two 2 +two 2 two 2 two 2 two 2 + + +-- !query 12 +SELECT * FROM nt1 CROSS JOIN nt2 ON (nt1.k > nt2.k) +-- !query 12 schema +struct +-- !query 12 output +three 3 one 1 +three 3 one 5 +two 2 one 1 +two 2 one 5 diff --git a/sql/core/src/test/resources/sql-tests/results/cte.sql.out b/sql/core/src/test/resources/sql-tests/results/cte.sql.out new file mode 100644 index 000000000000..a446c2cd183d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/cte.sql.out @@ -0,0 +1,104 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 9 + + +-- !query 0 +create temporary view t as select * from values 0, 1, 2 as t(id) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values 0, 1 as t(id) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +WITH s AS (SELECT 1 FROM s) SELECT * FROM s +-- !query 2 schema +struct<> +-- !query 2 output +org.apache.spark.sql.AnalysisException +Table or view not found: s; line 1 pos 25 + + +-- !query 3 +WITH t AS (SELECT 1 FROM t) SELECT * FROM t +-- !query 3 schema +struct<1:int> +-- !query 3 output +1 +1 +1 + + +-- !query 4 +WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +Table or view not found: s2; line 1 pos 26 + + +-- !query 5 +WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1 cross join t2 +-- !query 5 schema +struct +-- !query 5 output +0 2 +0 2 +1 2 +1 2 + + +-- !query 6 +WITH CTE1 AS ( + SELECT b.id AS id + FROM T2 a + CROSS JOIN (SELECT id AS id FROM T2) b +) +SELECT t1.id AS c1, + t2.id AS c2 +FROM CTE1 t1 + CROSS JOIN CTE1 t2 +-- !query 6 schema +struct +-- !query 6 output +0 0 +0 0 +0 0 +0 0 +0 1 +0 1 +0 1 +0 1 +1 0 +1 0 +1 0 +1 0 +1 1 +1 1 +1 1 +1 1 + + +-- !query 7 +DROP VIEW IF EXISTS t +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +DROP VIEW IF EXISTS t2 +-- !query 8 schema +struct<> +-- !query 8 output + diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out new file mode 100644 index 000000000000..a28b91c77324 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -0,0 +1,34 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 4 + + +-- !query 0 +select current_date = current_date(), current_timestamp = current_timestamp() +-- !query 0 schema +struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean> +-- !query 0 output +true true + + +-- !query 1 +select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') +-- !query 1 schema +struct +-- !query 1 output +NULL 2016-12-31 2016-12-31 + + +-- !query 2 +select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') +-- !query 2 schema +struct +-- !query 2 output +NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query 3 +select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), dayofweek('1582-10-15 13:10:15') +-- !query 3 schema +struct +-- !query 3 output +7 5 7 NULL 6 diff --git a/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out new file mode 100644 index 000000000000..51dac111029e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out @@ -0,0 +1,244 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 15 + + +-- !query 0 +CREATE TABLE t (key STRING, value STRING, ds STRING, hr INT) USING parquet + PARTITIONED BY (ds, hr) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO TABLE t PARTITION (ds='2017-08-01', hr=10) +VALUES ('k1', 100), ('k2', 200), ('k3', 300) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO TABLE t PARTITION (ds='2017-08-01', hr=11) +VALUES ('k1', 101), ('k2', 201), ('k3', 301), ('k4', 401) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO TABLE t PARTITION (ds='2017-09-01', hr=5) +VALUES ('k1', 102), ('k2', 202) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) +-- !query 4 schema +struct +-- !query 4 output +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=10] +Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 + +# Storage Information +Location [not included in comparison]sql/core/spark-warehouse/t + + +-- !query 5 +ANALYZE TABLE t PARTITION (ds='2017-08-01', hr=10) COMPUTE STATISTICS +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) +-- !query 6 schema +struct +-- !query 6 output +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=10] +Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 +Partition Statistics 1067 bytes, 3 rows + +# Storage Information +Location [not included in comparison]sql/core/spark-warehouse/t + + +-- !query 7 +ANALYZE TABLE t PARTITION (ds='2017-08-01') COMPUTE STATISTICS +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) +-- !query 8 schema +struct +-- !query 8 output +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=10] +Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 +Partition Statistics 1067 bytes, 3 rows + +# Storage Information +Location [not included in comparison]sql/core/spark-warehouse/t + + +-- !query 9 +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11) +-- !query 9 schema +struct +-- !query 9 output +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=11] +Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=11 +Partition Statistics 1080 bytes, 4 rows + +# Storage Information +Location [not included in comparison]sql/core/spark-warehouse/t + + +-- !query 10 +ANALYZE TABLE t PARTITION (ds, hr) COMPUTE STATISTICS +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) +-- !query 11 schema +struct +-- !query 11 output +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=10] +Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 +Partition Statistics 1067 bytes, 3 rows + +# Storage Information +Location [not included in comparison]sql/core/spark-warehouse/t + + +-- !query 12 +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11) +-- !query 12 schema +struct +-- !query 12 output +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=11] +Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=11 +Partition Statistics 1080 bytes, 4 rows + +# Storage Information +Location [not included in comparison]sql/core/spark-warehouse/t + + +-- !query 13 +DESC EXTENDED t PARTITION (ds='2017-09-01', hr=5) +-- !query 13 schema +struct +-- !query 13 output +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-09-01, hr=5] +Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-09-01/hr=5 +Partition Statistics 1054 bytes, 2 rows + +# Storage Information +Location [not included in comparison]sql/core/spark-warehouse/t + + +-- !query 14 +DROP TABLE t +-- !query 14 schema +struct<> +-- !query 14 output + diff --git a/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out new file mode 100644 index 000000000000..7873085da506 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out @@ -0,0 +1,161 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 12 + + +-- !query 0 +CREATE TABLE table_with_comment (a STRING, b INT, c STRING, d STRING) USING parquet COMMENT 'added' +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +DESC FORMATTED table_with_comment +-- !query 1 schema +struct +-- !query 1 output +a string +b int +c string +d string + +# Detailed Table Information +Database default +Table table_with_comment +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Comment added +Location [not included in comparison]sql/core/spark-warehouse/table_with_comment + + +-- !query 2 +ALTER TABLE table_with_comment SET TBLPROPERTIES("comment"= "modified comment", "type"= "parquet") +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +DESC FORMATTED table_with_comment +-- !query 3 schema +struct +-- !query 3 output +a string +b int +c string +d string + +# Detailed Table Information +Database default +Table table_with_comment +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Comment modified comment +Table Properties [type=parquet] +Location [not included in comparison]sql/core/spark-warehouse/table_with_comment + + +-- !query 4 +DROP TABLE table_with_comment +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +CREATE TABLE table_comment (a STRING, b INT) USING parquet +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +DESC FORMATTED table_comment +-- !query 6 schema +struct +-- !query 6 output +a string +b int + +# Detailed Table Information +Database default +Table table_comment +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]sql/core/spark-warehouse/table_comment + + +-- !query 7 +ALTER TABLE table_comment SET TBLPROPERTIES(comment = "added comment") +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +DESC formatted table_comment +-- !query 8 schema +struct +-- !query 8 output +a string +b int + +# Detailed Table Information +Database default +Table table_comment +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Comment added comment +Location [not included in comparison]sql/core/spark-warehouse/table_comment + + +-- !query 9 +ALTER TABLE table_comment UNSET TBLPROPERTIES IF EXISTS ('comment') +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +DESC FORMATTED table_comment +-- !query 10 schema +struct +-- !query 10 output +a string +b int + +# Detailed Table Information +Database default +Table table_comment +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]sql/core/spark-warehouse/table_comment + + +-- !query 11 +DROP TABLE table_comment +-- !query 11 schema +struct<> +-- !query 11 output + diff --git a/sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out new file mode 100644 index 000000000000..30d0a2dc5a3f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out @@ -0,0 +1,208 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 18 + + +-- !query 0 +CREATE TEMPORARY VIEW desc_col_temp_view (key int COMMENT 'column_comment') USING PARQUET +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +DESC desc_col_temp_view key +-- !query 1 schema +struct +-- !query 1 output +col_name key +data_type int +comment column_comment + + +-- !query 2 +DESC EXTENDED desc_col_temp_view key +-- !query 2 schema +struct +-- !query 2 output +col_name key +data_type int +comment column_comment +min NULL +max NULL +num_nulls NULL +distinct_count NULL +avg_col_len NULL +max_col_len NULL + + +-- !query 3 +DESC FORMATTED desc_col_temp_view key +-- !query 3 schema +struct +-- !query 3 output +col_name key +data_type int +comment column_comment +min NULL +max NULL +num_nulls NULL +distinct_count NULL +avg_col_len NULL +max_col_len NULL + + +-- !query 4 +DESC FORMATTED desc_col_temp_view desc_col_temp_view.key +-- !query 4 schema +struct +-- !query 4 output +col_name key +data_type int +comment column_comment +min NULL +max NULL +num_nulls NULL +distinct_count NULL +avg_col_len NULL +max_col_len NULL + + +-- !query 5 +DESC desc_col_temp_view key1 +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +Column key1 does not exist; + + +-- !query 6 +CREATE TABLE desc_col_table (key int COMMENT 'column_comment') USING PARQUET +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +ANALYZE TABLE desc_col_table COMPUTE STATISTICS FOR COLUMNS key +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +DESC desc_col_table key +-- !query 8 schema +struct +-- !query 8 output +col_name key +data_type int +comment column_comment + + +-- !query 9 +DESC EXTENDED desc_col_table key +-- !query 9 schema +struct +-- !query 9 output +col_name key +data_type int +comment column_comment +min NULL +max NULL +num_nulls 0 +distinct_count 0 +avg_col_len 4 +max_col_len 4 + + +-- !query 10 +DESC FORMATTED desc_col_table key +-- !query 10 schema +struct +-- !query 10 output +col_name key +data_type int +comment column_comment +min NULL +max NULL +num_nulls 0 +distinct_count 0 +avg_col_len 4 +max_col_len 4 + + +-- !query 11 +CREATE TABLE desc_complex_col_table (`a.b` int, col struct) USING PARQUET +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +DESC FORMATTED desc_complex_col_table `a.b` +-- !query 12 schema +struct +-- !query 12 output +col_name a.b +data_type int +comment NULL +min NULL +max NULL +num_nulls NULL +distinct_count NULL +avg_col_len NULL +max_col_len NULL + + +-- !query 13 +DESC FORMATTED desc_complex_col_table col +-- !query 13 schema +struct +-- !query 13 output +col_name col +data_type struct +comment NULL +min NULL +max NULL +num_nulls NULL +distinct_count NULL +avg_col_len NULL +max_col_len NULL + + +-- !query 14 +DESC FORMATTED desc_complex_col_table col.x +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +DESC TABLE COLUMN command does not support nested data types: col.x; + + +-- !query 15 +DROP VIEW desc_col_temp_view +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +DROP TABLE desc_col_table +-- !query 16 schema +struct<> +-- !query 16 output + + + +-- !query 17 +DROP TABLE desc_complex_col_table +-- !query 17 schema +struct<> +-- !query 17 output + diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out new file mode 100644 index 000000000000..8c908b762505 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -0,0 +1,539 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 36 + + +-- !query 0 +CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet + OPTIONS (a '1', b '2') + PARTITIONED BY (c, d) CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS + COMMENT 'table_comment' + TBLPROPERTIES (t 'test') +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW temp_v AS SELECT * FROM t +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW temp_Data_Source_View + USING org.apache.spark.sql.sources.DDLScanSource + OPTIONS ( + From '1', + To '10', + Table 'test1') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE VIEW v AS SELECT * FROM t +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +ALTER TABLE t SET TBLPROPERTIES (e = '3') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +ALTER TABLE t ADD PARTITION (c='Us', d=1) +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +DESCRIBE t +-- !query 6 schema +struct +-- !query 6 output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + + +-- !query 7 +DESC default.t +-- !query 7 schema +struct +-- !query 7 output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + + +-- !query 8 +DESC TABLE t +-- !query 8 schema +struct +-- !query 8 output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + + +-- !query 9 +DESC FORMATTED t +-- !query 9 schema +struct +-- !query 9 output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Table Information +Database default +Table t +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Comment table_comment +Table Properties [t=test, e=3] +Location [not included in comparison]sql/core/spark-warehouse/t +Storage Properties [a=1, b=2] +Partition Provider Catalog + + +-- !query 10 +DESC EXTENDED t +-- !query 10 schema +struct +-- !query 10 output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Table Information +Database default +Table t +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Comment table_comment +Table Properties [t=test, e=3] +Location [not included in comparison]sql/core/spark-warehouse/t +Storage Properties [a=1, b=2] +Partition Provider Catalog + + +-- !query 11 +ALTER TABLE t UNSET TBLPROPERTIES (e) +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +DESC EXTENDED t +-- !query 12 schema +struct +-- !query 12 output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Table Information +Database default +Table t +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Comment table_comment +Table Properties [t=test] +Location [not included in comparison]sql/core/spark-warehouse/t +Storage Properties [a=1, b=2] +Partition Provider Catalog + + +-- !query 13 +ALTER TABLE t UNSET TBLPROPERTIES (comment) +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +DESC EXTENDED t +-- !query 14 schema +struct +-- !query 14 output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Table Information +Database default +Table t +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Table Properties [t=test] +Location [not included in comparison]sql/core/spark-warehouse/t +Storage Properties [a=1, b=2] +Partition Provider Catalog + + +-- !query 15 +DESC t PARTITION (c='Us', d=1) +-- !query 15 schema +struct +-- !query 15 output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + + +-- !query 16 +DESC EXTENDED t PARTITION (c='Us', d=1) +-- !query 16 schema +struct +-- !query 16 output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Partition Information +Database default +Table t +Partition Values [c=Us, d=1] +Location [not included in comparison]sql/core/spark-warehouse/t/c=Us/d=1 +Storage Properties [a=1, b=2] + +# Storage Information +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Location [not included in comparison]sql/core/spark-warehouse/t +Storage Properties [a=1, b=2] + + +-- !query 17 +DESC FORMATTED t PARTITION (c='Us', d=1) +-- !query 17 schema +struct +-- !query 17 output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Partition Information +Database default +Table t +Partition Values [c=Us, d=1] +Location [not included in comparison]sql/core/spark-warehouse/t/c=Us/d=1 +Storage Properties [a=1, b=2] + +# Storage Information +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Location [not included in comparison]sql/core/spark-warehouse/t +Storage Properties [a=1, b=2] + + +-- !query 18 +DESC t PARTITION (c='Us', d=2) +-- !query 18 schema +struct<> +-- !query 18 output +org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException +Partition not found in table 't' database 'default': +c -> Us +d -> 2; + + +-- !query 19 +DESC t PARTITION (c='Us') +-- !query 19 schema +struct<> +-- !query 19 output +org.apache.spark.sql.AnalysisException +Partition spec is invalid. The spec (c) must match the partition spec (c, d) defined in table '`default`.`t`'; + + +-- !query 20 +DESC t PARTITION (c='Us', d) +-- !query 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.catalyst.parser.ParseException + +PARTITION specification is incomplete: `d`(line 1, pos 0) + +== SQL == +DESC t PARTITION (c='Us', d) +^^^ + + +-- !query 21 +DESC temp_v +-- !query 21 schema +struct +-- !query 21 output +a string +b int +c string +d string + + +-- !query 22 +DESC TABLE temp_v +-- !query 22 schema +struct +-- !query 22 output +a string +b int +c string +d string + + +-- !query 23 +DESC FORMATTED temp_v +-- !query 23 schema +struct +-- !query 23 output +a string +b int +c string +d string + + +-- !query 24 +DESC EXTENDED temp_v +-- !query 24 schema +struct +-- !query 24 output +a string +b int +c string +d string + + +-- !query 25 +DESC temp_Data_Source_View +-- !query 25 schema +struct +-- !query 25 output +intType int test comment test1 +stringType string +dateType date +timestampType timestamp +doubleType double +bigintType bigint +tinyintType tinyint +decimalType decimal(10,0) +fixedDecimalType decimal(5,1) +binaryType binary +booleanType boolean +smallIntType smallint +floatType float +mapType map +arrayType array +structType struct + + +-- !query 26 +DESC temp_v PARTITION (c='Us', d=1) +-- !query 26 schema +struct<> +-- !query 26 output +org.apache.spark.sql.AnalysisException +DESC PARTITION is not allowed on a temporary view: temp_v; + + +-- !query 27 +DESC v +-- !query 27 schema +struct +-- !query 27 output +a string +b int +c string +d string + + +-- !query 28 +DESC TABLE v +-- !query 28 schema +struct +-- !query 28 output +a string +b int +c string +d string + + +-- !query 29 +DESC FORMATTED v +-- !query 29 schema +struct +-- !query 29 output +a string +b int +c string +d string + +# Detailed Table Information +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Default Database default +View Query Output Columns [a, b, c, d] +Table Properties [view.query.out.col.3=d, view.query.out.col.0=a, view.query.out.numCols=4, view.default.database=default, view.query.out.col.1=b, view.query.out.col.2=c] + + +-- !query 30 +DESC EXTENDED v +-- !query 30 schema +struct +-- !query 30 output +a string +b int +c string +d string + +# Detailed Table Information +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Default Database default +View Query Output Columns [a, b, c, d] +Table Properties [view.query.out.col.3=d, view.query.out.col.0=a, view.query.out.numCols=4, view.default.database=default, view.query.out.col.1=b, view.query.out.col.2=c] + + +-- !query 31 +DESC v PARTITION (c='Us', d=1) +-- !query 31 schema +struct<> +-- !query 31 output +org.apache.spark.sql.AnalysisException +DESC PARTITION is not allowed on a view: v; + + +-- !query 32 +DROP TABLE t +-- !query 32 schema +struct<> +-- !query 32 output + + + +-- !query 33 +DROP VIEW temp_v +-- !query 33 schema +struct<> +-- !query 33 output + + + +-- !query 34 +DROP VIEW temp_Data_Source_View +-- !query 34 schema +struct<> +-- !query 34 output + + + +-- !query 35 +DROP VIEW v +-- !query 35 schema +struct<> +-- !query 35 output + diff --git a/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out new file mode 100644 index 000000000000..ce7a16a4d0c8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out @@ -0,0 +1,377 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 29 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH CUBE +-- !query 1 schema +struct<(a + b):int,b:int,sum((a - b)):bigint> +-- !query 1 output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL 1 3 +NULL 2 0 +NULL NULL 3 + + +-- !query 2 +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH CUBE +-- !query 2 schema +struct +-- !query 2 output +1 1 1 +1 2 2 +1 NULL 3 +2 1 1 +2 2 2 +2 NULL 3 +3 1 1 +3 2 2 +3 NULL 3 +NULL 1 3 +NULL 2 6 +NULL NULL 9 + + +-- !query 3 +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP +-- !query 3 schema +struct<(a + b):int,b:int,sum((a - b)):bigint> +-- !query 3 output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL NULL 3 + + +-- !query 4 +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH ROLLUP +-- !query 4 schema +struct +-- !query 4 output +1 1 1 +1 2 2 +1 NULL 3 +2 1 1 +2 2 2 +2 NULL 3 +3 1 1 +3 2 2 +3 NULL 3 +NULL NULL 9 + + +-- !query 5 +CREATE OR REPLACE TEMPORARY VIEW courseSales AS SELECT * FROM VALUES +("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) +AS courseSales(course, year, earnings) +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY course, year +-- !query 6 schema +struct +-- !query 6 output +NULL NULL 113000 +Java NULL 50000 +Java 2012 20000 +Java 2013 30000 +dotNET NULL 63000 +dotNET 2012 15000 +dotNET 2013 48000 + + +-- !query 7 +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, year +-- !query 7 schema +struct +-- !query 7 output +NULL NULL 113000 +NULL 2012 35000 +NULL 2013 78000 +Java NULL 50000 +Java 2012 20000 +Java 2013 30000 +dotNET NULL 63000 +dotNET 2012 15000 +dotNET 2013 48000 + + +-- !query 8 +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year) +-- !query 8 schema +struct +-- !query 8 output +Java NULL 50000 +NULL 2012 35000 +NULL 2013 78000 +dotNET NULL 63000 + + +-- !query 9 +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course) +-- !query 9 schema +struct +-- !query 9 output +Java NULL 50000 +dotNET NULL 63000 + + +-- !query 10 +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year) +-- !query 10 schema +struct +-- !query 10 output +NULL 2012 35000 +NULL 2013 78000 + + +-- !query 11 +SELECT course, SUM(earnings) AS sum FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum +-- !query 11 schema +struct +-- !query 11 output +NULL 113000 +Java 20000 +Java 30000 +Java 50000 +dotNET 5000 +dotNET 10000 +dotNET 48000 +dotNET 63000 + + +-- !query 12 +SELECT course, SUM(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum +-- !query 12 schema +struct +-- !query 12 output +NULL 113000 3 +Java 20000 0 +Java 30000 0 +Java 50000 1 +dotNET 5000 0 +dotNET 10000 0 +dotNET 48000 0 +dotNET 63000 1 + + +-- !query 13 +SELECT course, year, GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales +GROUP BY CUBE(course, year) +-- !query 13 schema +struct +-- !query 13 output +Java 2012 0 0 0 +Java 2013 0 0 0 +Java NULL 0 1 1 +NULL 2012 1 0 2 +NULL 2013 1 0 2 +NULL NULL 1 1 3 +dotNET 2012 0 0 0 +dotNET 2013 0 0 0 +dotNET NULL 0 1 1 + + +-- !query 14 +SELECT course, year, GROUPING(course) FROM courseSales GROUP BY course, year +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +grouping() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 15 +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY course, year +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.AnalysisException +grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 16 +SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.AnalysisException +grouping__id is deprecated; use grouping_id() instead; + + +-- !query 17 +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) +HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 +-- !query 17 schema +struct +-- !query 17 output +Java NULL +NULL NULL +dotNET NULL + + +-- !query 18 +SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING(course) > 0 +-- !query 18 schema +struct<> +-- !query 18 output +org.apache.spark.sql.AnalysisException +grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 19 +SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0 +-- !query 19 schema +struct<> +-- !query 19 output +org.apache.spark.sql.AnalysisException +grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 20 +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0 +-- !query 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.AnalysisException +grouping__id is deprecated; use grouping_id() instead; + + +-- !query 21 +SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, year +-- !query 21 schema +struct +-- !query 21 output +Java 2012 0 0 +Java 2013 0 0 +dotNET 2012 0 0 +dotNET 2013 0 0 +Java NULL 0 1 +dotNET NULL 0 1 +NULL 2012 1 0 +NULL 2013 1 0 +NULL NULL 1 1 + + +-- !query 22 +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, year +-- !query 22 schema +struct +-- !query 22 output +Java 2012 0 +Java 2013 0 +dotNET 2012 0 +dotNET 2013 0 +Java NULL 1 +dotNET NULL 1 +NULL 2012 2 +NULL 2013 2 +NULL NULL 3 + + +-- !query 23 +SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING(course) +-- !query 23 schema +struct<> +-- !query 23 output +org.apache.spark.sql.AnalysisException +grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 24 +SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING_ID(course) +-- !query 24 schema +struct<> +-- !query 24 output +org.apache.spark.sql.AnalysisException +grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 25 +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id +-- !query 25 schema +struct<> +-- !query 25 output +org.apache.spark.sql.AnalysisException +grouping__id is deprecated; use grouping_id() instead; + + +-- !query 26 +SELECT a + b AS k1, b AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2) +-- !query 26 schema +struct +-- !query 26 output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL 1 3 +NULL 2 0 +NULL NULL 3 + + +-- !query 27 +SELECT a + b AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b) +-- !query 27 schema +struct +-- !query 27 output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL NULL 3 + + +-- !query 28 +SELECT a + b, b AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k) +-- !query 28 schema +struct<(a + b):int,k:int,sum((a - b)):bigint> +-- !query 28 output +NULL 1 3 +NULL 2 0 diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out new file mode 100644 index 000000000000..9ecbe19078dd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out @@ -0,0 +1,198 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 20 + + +-- !query 0 +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +select a, sum(b) from data group by 1 +-- !query 1 schema +struct +-- !query 1 output +1 3 +2 3 +3 3 + + +-- !query 2 +select 1, 2, sum(b) from data group by 1, 2 +-- !query 2 schema +struct<1:int,2:int,sum(b):bigint> +-- !query 2 output +1 2 9 + + +-- !query 3 +select a, 1, sum(b) from data group by a, 1 +-- !query 3 schema +struct +-- !query 3 output +1 1 3 +2 1 3 +3 1 3 + + +-- !query 4 +select a, 1, sum(b) from data group by 1, 2 +-- !query 4 schema +struct +-- !query 4 output +1 1 3 +2 1 3 +3 1 3 + + +-- !query 5 +select a, b + 2, count(2) from data group by a, 2 +-- !query 5 schema +struct +-- !query 5 output +1 3 1 +1 4 1 +2 3 1 +2 4 1 +3 3 1 +3 4 1 + + +-- !query 6 +select a as aa, b + 2 as bb, count(2) from data group by 1, 2 +-- !query 6 schema +struct +-- !query 6 output +1 3 1 +1 4 1 +2 3 1 +2 4 1 +3 3 1 +3 4 1 + + +-- !query 7 +select sum(b) from data group by 1 + 0 +-- !query 7 schema +struct +-- !query 7 output +9 + + +-- !query 8 +select a, b from data group by -1 +-- !query 8 schema +struct<> +-- !query 8 output +org.apache.spark.sql.AnalysisException +GROUP BY position -1 is not in select list (valid range is [1, 2]); line 1 pos 31 + + +-- !query 9 +select a, b from data group by 0 +-- !query 9 schema +struct<> +-- !query 9 output +org.apache.spark.sql.AnalysisException +GROUP BY position 0 is not in select list (valid range is [1, 2]); line 1 pos 31 + + +-- !query 10 +select a, b from data group by 3 +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 31 + + +-- !query 11 +select a, b, sum(b) from data group by 3 +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.AnalysisException +aggregate functions are not allowed in GROUP BY, but found sum(CAST(data.`b` AS BIGINT)); + + +-- !query 12 +select a, b, sum(b) + 2 from data group by 3 +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +aggregate functions are not allowed in GROUP BY, but found (sum(CAST(data.`b` AS BIGINT)) + CAST(2 AS BIGINT)); + + +-- !query 13 +select a, rand(0), sum(b) from data group by a, 2 +-- !query 13 schema +struct +-- !query 13 output +1 0.4048454303385226 2 +1 0.8446490682263027 1 +2 0.5871875724155838 1 +2 0.8865128837019473 2 +3 0.742083829230211 1 +3 0.9179913208300406 2 + + +-- !query 14 +select * from data group by a, b, 1 +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +Star (*) is not allowed in select list when GROUP BY ordinal position is used; + + +-- !query 15 +select a, count(a) from (select 1 as a) tmp group by 1 order by 1 +-- !query 15 schema +struct +-- !query 15 output +1 1 + + +-- !query 16 +select count(a), a from (select 1 as a) tmp group by 2 having a > 0 +-- !query 16 schema +struct +-- !query 16 output +1 1 + + +-- !query 17 +select a, a AS k, count(b) from data group by k, 1 +-- !query 17 schema +struct +-- !query 17 output +1 1 2 +2 2 2 +3 3 2 + + +-- !query 18 +set spark.sql.groupByOrdinal=false +-- !query 18 schema +struct +-- !query 18 output +spark.sql.groupByOrdinal false + + +-- !query 19 +select sum(b) from data group by -1 +-- !query 19 schema +struct +-- !query 19 output +9 diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out new file mode 100644 index 000000000000..986bb01c13fe --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -0,0 +1,229 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 25 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT a, COUNT(b) FROM testData +-- !query 1 schema +struct<> +-- !query 1 output +org.apache.spark.sql.AnalysisException +grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(count(testdata.`b`) AS `count(b)`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.; + + +-- !query 2 +SELECT COUNT(a), COUNT(b) FROM testData +-- !query 2 schema +struct +-- !query 2 output +7 7 + + +-- !query 3 +SELECT a, COUNT(b) FROM testData GROUP BY a +-- !query 3 schema +struct +-- !query 3 output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query 4 +SELECT a, COUNT(b) FROM testData GROUP BY b +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; + + +-- !query 5 +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a +-- !query 5 schema +struct +-- !query 5 output +0 1 +2 2 +2 2 +3 2 + + +-- !query 6 +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1 +-- !query 6 schema +struct +-- !query 6 output +foo 7 + + +-- !query 7 +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1 +-- !query 7 schema +struct +-- !query 7 output + + + +-- !query 8 +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1 +-- !query 8 schema +struct +-- !query 8 output + + + +-- !query 9 +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1 +-- !query 9 schema +struct> +-- !query 9 output + + + +-- !query 10 +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b +-- !query 10 schema +struct<(a + b):int,count(b):bigint> +-- !query 10 output +2 1 +3 2 +4 2 +5 1 +NULL 1 + + +-- !query 11 +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1 +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.AnalysisException +expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; + + +-- !query 12 +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1 +-- !query 12 schema +struct<((a + 1) + 1):int,count(b):bigint> +-- !query 12 output +3 2 +4 2 +5 2 +NULL 1 + + +-- !query 13 +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData +-- !query 13 schema +struct +-- !query 13 output +-0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 + + +-- !query 14 +SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +-- !query 14 schema +struct +-- !query 14 output +1 1 + + +-- !query 15 +SELECT a AS k, COUNT(b) FROM testData GROUP BY k +-- !query 15 schema +struct +-- !query 15 output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query 16 +SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1 +-- !query 16 schema +struct +-- !query 16 output +2 2 +3 2 + + +-- !query 17 +SELECT COUNT(b) AS k FROM testData GROUP BY k +-- !query 17 schema +struct<> +-- !query 17 output +org.apache.spark.sql.AnalysisException +aggregate functions are not allowed in GROUP BY, but found count(testdata.`b`); + + +-- !query 18 +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) +-- !query 18 schema +struct<> +-- !query 18 output + + + +-- !query 19 +SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a +-- !query 19 schema +struct<> +-- !query 19 output +org.apache.spark.sql.AnalysisException +expression 'testdatahassamenamewithalias.`k`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; + + +-- !query 20 +set spark.sql.groupByAliases=false +-- !query 20 schema +struct +-- !query 20 output +spark.sql.groupByAliases false + + +-- !query 21 +SELECT a AS k, COUNT(b) FROM testData GROUP BY k +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.AnalysisException +cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 47 + + +-- !query 22 +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a +-- !query 22 schema +struct +-- !query 22 output + + + +-- !query 23 +SELECT COUNT(1) FROM testData WHERE false +-- !query 23 schema +struct +-- !query 23 output +0 + + +-- !query 24 +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t +-- !query 24 schema +struct<1:int> +-- !query 24 output +1 diff --git a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out new file mode 100644 index 000000000000..edb38a52b751 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out @@ -0,0 +1,42 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 4 + + +-- !query 0 +CREATE TEMPORARY VIEW grouping AS SELECT * FROM VALUES + ("1", "2", "3", 1), + ("4", "5", "6", 1), + ("7", "8", "9", 1) + as grouping(a, b, c, d) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS (()) +-- !query 1 schema +struct +-- !query 1 output +NULL NULL NULL 3 + + +-- !query 2 +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((a)) +-- !query 2 schema +struct +-- !query 2 output +1 NULL NULL 1 +4 NULL NULL 1 +7 NULL NULL 1 + + +-- !query 3 +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((c)) +-- !query 3 schema +struct +-- !query 3 output +NULL NULL 3 1 +NULL NULL 6 1 +NULL NULL 9 1 diff --git a/sql/core/src/test/resources/sql-tests/results/having.sql.out b/sql/core/src/test/resources/sql-tests/results/having.sql.out new file mode 100644 index 000000000000..d87ee5221647 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/having.sql.out @@ -0,0 +1,49 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 5 + + +-- !query 0 +create temporary view hav as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2 +-- !query 1 schema +struct +-- !query 1 output +one 6 +three 3 + + +-- !query 2 +SELECT count(k) FROM hav GROUP BY v + 1 HAVING v + 1 = 2 +-- !query 2 schema +struct +-- !query 2 output +1 + + +-- !query 3 +SELECT MIN(t.v) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(COUNT(1) > 0) +-- !query 3 schema +struct +-- !query 3 output +1 + + +-- !query 4 +SELECT a + b FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > 1 +-- !query 4 schema +struct<(a + CAST(b AS BIGINT)):bigint> +-- !query 4 output +3 +7 diff --git a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out new file mode 100644 index 000000000000..c065ce501292 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out @@ -0,0 +1,183 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 18 + + +-- !query 0 +select * from values ("one", 1) +-- !query 0 schema +struct +-- !query 0 output +one 1 + + +-- !query 1 +select * from values ("one", 1) as data +-- !query 1 schema +struct +-- !query 1 output +one 1 + + +-- !query 2 +select * from values ("one", 1) as data(a, b) +-- !query 2 schema +struct +-- !query 2 output +one 1 + + +-- !query 3 +select * from values 1, 2, 3 as data(a) +-- !query 3 schema +struct +-- !query 3 output +1 +2 +3 + + +-- !query 4 +select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b) +-- !query 4 schema +struct +-- !query 4 output +one 1 +three NULL +two 2 + + +-- !query 5 +select * from values ("one", null), ("two", null) as data(a, b) +-- !query 5 schema +struct +-- !query 5 output +one NULL +two NULL + + +-- !query 6 +select * from values ("one", 1), ("two", 2L) as data(a, b) +-- !query 6 schema +struct +-- !query 6 output +one 1 +two 2 + + +-- !query 7 +select * from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b) +-- !query 7 schema +struct +-- !query 7 output +one 1 +two 4 + + +-- !query 8 +select * from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b) +-- !query 8 schema +struct> +-- !query 8 output +one [0,1] +two [2,3] + + +-- !query 9 +select * from values ("one", 2.0), ("two", 3.0D) as data(a, b) +-- !query 9 schema +struct +-- !query 9 output +one 2.0 +two 3.0 + + +-- !query 10 +select * from values ("one", rand(5)), ("two", 3.0D) as data(a, b) +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +cannot evaluate expression rand(5) in inline table definition; line 1 pos 29 + + +-- !query 11 +select * from values ("one", 2.0), ("two") as data(a, b) +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.AnalysisException +expected 2 columns but found 1 columns in row 1; line 1 pos 14 + + +-- !query 12 +select * from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b) +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +incompatible types found in column b for inline table; line 1 pos 14 + + +-- !query 13 +select * from values ("one"), ("two") as data(a, b) +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.AnalysisException +expected 2 columns but found 1 columns in row 0; line 1 pos 14 + + +-- !query 14 +select * from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b) +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +Undefined function: 'random_not_exist_func'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 29 + + +-- !query 15 +select * from values ("one", count(1)), ("two", 2) as data(a, b) +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.AnalysisException +cannot evaluate expression count(1) in inline table definition; line 1 pos 29 + + +-- !query 16 +select * from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b) +-- !query 16 schema +struct> +-- !query 16 output +1991-12-06 00:00:00 [1991-12-06 01:00:00.0,1991-12-06 12:00:00.0] + + +-- !query 17 +EXPLAIN EXTENDED SELECT * FROM VALUES ('one', 1), ('three', null) CROSS JOIN VALUES ('one', 1), ('three', null) +-- !query 17 schema +struct +-- !query 17 output +== Parsed Logical Plan == +'Project [*] ++- 'Join Cross + :- 'UnresolvedInlineTable [col1, col2], [List(one, 1), List(three, null)] + +- 'UnresolvedInlineTable [col1, col2], [List(one, 1), List(three, null)] + +== Analyzed Logical Plan == +col1: string, col2: int, col1: string, col2: int +Project [col1#x, col2#x, col1#x, col2#x] ++- Join Cross + :- LocalRelation [col1#x, col2#x] + +- LocalRelation [col1#x, col2#x] + +== Optimized Logical Plan == +Join Cross +:- LocalRelation [col1#x, col2#x] ++- LocalRelation [col1#x, col2#x] + +== Physical Plan == +BroadcastNestedLoopJoin BuildRight, Cross +:- LocalTableScan [col1#x, col2#x] ++- BroadcastExchange IdentityBroadcastMode + +- LocalTableScan [col1#x, col2#x] diff --git a/sql/core/src/test/resources/sql-tests/results/inner-join.sql.out b/sql/core/src/test/resources/sql-tests/results/inner-join.sql.out new file mode 100644 index 000000000000..8d56ebe9fd3b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/inner-join.sql.out @@ -0,0 +1,67 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 7 + + +-- !query 0 +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE TEMPORARY VIEW ta AS +SELECT a, 'a' AS tag FROM t1 +UNION ALL +SELECT a, 'b' AS tag FROM t2 +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +CREATE TEMPORARY VIEW tb AS +SELECT a, 'a' AS tag FROM t3 +UNION ALL +SELECT a, 'b' AS tag FROM t4 +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag +-- !query 6 schema +struct +-- !query 6 output +1 a +1 a +1 b +1 b diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out new file mode 100644 index 000000000000..d9dc728a18e8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out @@ -0,0 +1,260 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 26 + + +-- !query 0 +describe function to_json +-- !query 0 schema +struct +-- !query 0 output +Class: org.apache.spark.sql.catalyst.expressions.StructsToJson +Function: to_json +Usage: to_json(expr[, options]) - Returns a json string with a given struct value + + +-- !query 1 +describe function extended to_json +-- !query 1 schema +struct +-- !query 1 output +Class: org.apache.spark.sql.catalyst.expressions.StructsToJson +Extended Usage: + Examples: + > SELECT to_json(named_struct('a', 1, 'b', 2)); + {"a":1,"b":2} + > SELECT to_json(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')); + {"time":"26/08/2015"} + > SELECT to_json(array(named_struct('a', 1, 'b', 2)); + [{"a":1,"b":2}] + > SELECT to_json(map('a', named_struct('b', 1))); + {"a":{"b":1}} + > SELECT to_json(map(named_struct('a', 1),named_struct('b', 2))); + {"[1]":{"b":2}} + > SELECT to_json(map('a', 1)); + {"a":1} + > SELECT to_json(array((map('a', 1)))); + [{"a":1}] + + Since: 2.2.0 + +Function: to_json +Usage: to_json(expr[, options]) - Returns a json string with a given struct value + + +-- !query 2 +select to_json(named_struct('a', 1, 'b', 2)) +-- !query 2 schema +struct +-- !query 2 output +{"a":1,"b":2} + + +-- !query 3 +select to_json(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')) +-- !query 3 schema +struct +-- !query 3 output +{"time":"26/08/2015"} + + +-- !query 4 +select to_json(array(named_struct('a', 1, 'b', 2))) +-- !query 4 schema +struct +-- !query 4 output +[{"a":1,"b":2}] + + +-- !query 5 +select to_json(map(named_struct('a', 1, 'b', 2), named_struct('a', 1, 'b', 2))) +-- !query 5 schema +struct +-- !query 5 output +{"[1,2]":{"a":1,"b":2}} + + +-- !query 6 +select to_json(map('a', named_struct('a', 1, 'b', 2))) +-- !query 6 schema +struct +-- !query 6 output +{"a":{"a":1,"b":2}} + + +-- !query 7 +select to_json(map('a', 1)) +-- !query 7 schema +struct +-- !query 7 output +{"a":1} + + +-- !query 8 +select to_json(array(map('a',1))) +-- !query 8 schema +struct +-- !query 8 output +[{"a":1}] + + +-- !query 9 +select to_json(array(map('a',1), map('b',2))) +-- !query 9 schema +struct +-- !query 9 output +[{"a":1},{"b":2}] + + +-- !query 10 +select to_json(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')) +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +Must use a map() function for options;; line 1 pos 7 + + +-- !query 11 +select to_json(named_struct('a', 1, 'b', 2), map('mode', 1)) +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.AnalysisException +A type of keys and values in map() must be string, but got MapType(StringType,IntegerType,false);; line 1 pos 7 + + +-- !query 12 +select to_json() +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function to_json; line 1 pos 7 + + +-- !query 13 +describe function from_json +-- !query 13 schema +struct +-- !query 13 output +Class: org.apache.spark.sql.catalyst.expressions.JsonToStructs +Function: from_json +Usage: from_json(jsonStr, schema[, options]) - Returns a struct value with the given `jsonStr` and `schema`. + + +-- !query 14 +describe function extended from_json +-- !query 14 schema +struct +-- !query 14 output +Class: org.apache.spark.sql.catalyst.expressions.JsonToStructs +Extended Usage: + Examples: + > SELECT from_json('{"a":1, "b":0.8}', 'a INT, b DOUBLE'); + {"a":1, "b":0.8} + > SELECT from_json('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')); + {"time":"2015-08-26 00:00:00.0"} + + Since: 2.2.0 + +Function: from_json +Usage: from_json(jsonStr, schema[, options]) - Returns a struct value with the given `jsonStr` and `schema`. + + +-- !query 15 +select from_json('{"a":1}', 'a INT') +-- !query 15 schema +struct> +-- !query 15 output +{"a":1} + + +-- !query 16 +select from_json('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')) +-- !query 16 schema +struct> +-- !query 16 output +{"time":2015-08-26 00:00:00.0} + + +-- !query 17 +select from_json('{"a":1}', 1) +-- !query 17 schema +struct<> +-- !query 17 output +org.apache.spark.sql.AnalysisException +Expected a string literal instead of 1;; line 1 pos 7 + + +-- !query 18 +select from_json('{"a":1}', 'a InvalidType') +-- !query 18 schema +struct<> +-- !query 18 output +org.apache.spark.sql.AnalysisException + +DataType invalidtype is not supported.(line 1, pos 2) + +== SQL == +a InvalidType +--^^^ +; line 1 pos 7 + + +-- !query 19 +select from_json('{"a":1}', 'a INT', named_struct('mode', 'PERMISSIVE')) +-- !query 19 schema +struct<> +-- !query 19 output +org.apache.spark.sql.AnalysisException +Must use a map() function for options;; line 1 pos 7 + + +-- !query 20 +select from_json('{"a":1}', 'a INT', map('mode', 1)) +-- !query 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.AnalysisException +A type of keys and values in map() must be string, but got MapType(StringType,IntegerType,false);; line 1 pos 7 + + +-- !query 21 +select from_json() +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function from_json; line 1 pos 7 + + +-- !query 22 +SELECT json_tuple('{"a" : 1, "b" : 2}', CAST(NULL AS STRING), 'b', CAST(NULL AS STRING), 'a') +-- !query 22 schema +struct +-- !query 22 output +NULL 2 NULL 1 + + +-- !query 23 +CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a') +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +SELECT json_tuple(jsonField, 'b', CAST(NULL AS STRING), a) FROM jsonTable +-- !query 24 schema +struct +-- !query 24 output +2 NULL 1 + + +-- !query 25 +DROP VIEW IF EXISTS jsonTable +-- !query 25 schema +struct<> +-- !query 25 output + diff --git a/sql/core/src/test/resources/sql-tests/results/limit.sql.out b/sql/core/src/test/resources/sql-tests/results/limit.sql.out new file mode 100644 index 000000000000..146abe6cbd05 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/limit.sql.out @@ -0,0 +1,109 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 12 + + +-- !query 0 +SELECT * FROM testdata LIMIT 2 +-- !query 0 schema +struct +-- !query 0 output +1 1 +2 2 + + +-- !query 1 +SELECT * FROM arraydata LIMIT 2 +-- !query 1 schema +struct,nestedarraycol:array>> +-- !query 1 output +[1,2,3] [[1,2,3]] +[2,3,4] [[2,3,4]] + + +-- !query 2 +SELECT * FROM mapdata LIMIT 2 +-- !query 2 schema +struct> +-- !query 2 output +{1:"a1",2:"b1",3:"c1",4:"d1",5:"e1"} +{1:"a2",2:"b2",3:"c2",4:"d2"} + + +-- !query 3 +SELECT * FROM testdata LIMIT 2 + 1 +-- !query 3 schema +struct +-- !query 3 output +1 1 +2 2 +3 3 + + +-- !query 4 +SELECT * FROM testdata LIMIT CAST(1 AS int) +-- !query 4 schema +struct +-- !query 4 output +1 1 + + +-- !query 5 +SELECT * FROM testdata LIMIT -1 +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +The limit expression must be equal to or greater than 0, but got -1; + + +-- !query 6 +SELECT * FROM testData TABLESAMPLE (-1 ROWS) +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.AnalysisException +The limit expression must be equal to or greater than 0, but got -1; + + +-- !query 7 +SELECT * FROM testdata LIMIT key > 3 +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.AnalysisException +The limit expression must evaluate to a constant value, but got (testdata.`key` > 3); + + +-- !query 8 +SELECT * FROM testdata LIMIT true +-- !query 8 schema +struct<> +-- !query 8 output +org.apache.spark.sql.AnalysisException +The limit expression must be integer type, but got boolean; + + +-- !query 9 +SELECT * FROM testdata LIMIT 'a' +-- !query 9 schema +struct<> +-- !query 9 output +org.apache.spark.sql.AnalysisException +The limit expression must be integer type, but got string; + + +-- !query 10 +SELECT * FROM (SELECT * FROM range(10) LIMIT 5) WHERE id > 3 +-- !query 10 schema +struct +-- !query 10 output +4 + + +-- !query 11 +SELECT * FROM testdata WHERE key < 3 LIMIT ALL +-- !query 11 schema +struct +-- !query 11 output +1 1 +2 2 diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out new file mode 100644 index 000000000000..95d4413148f6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -0,0 +1,418 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 43 + + +-- !query 0 +select null, Null, nUll +-- !query 0 schema +struct +-- !query 0 output +NULL NULL NULL + + +-- !query 1 +select true, tRue, false, fALse +-- !query 1 schema +struct +-- !query 1 output +true true false false + + +-- !query 2 +select 1Y +-- !query 2 schema +struct<1:tinyint> +-- !query 2 output +1 + + +-- !query 3 +select 127Y, -128Y +-- !query 3 schema +struct<127:tinyint,-128:tinyint> +-- !query 3 output +127 -128 + + +-- !query 4 +select 128Y +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.catalyst.parser.ParseException + +Numeric literal 128 does not fit in range [-128, 127] for type tinyint(line 1, pos 7) + +== SQL == +select 128Y +-------^^^ + + +-- !query 5 +select 1S +-- !query 5 schema +struct<1:smallint> +-- !query 5 output +1 + + +-- !query 6 +select 32767S, -32768S +-- !query 6 schema +struct<32767:smallint,-32768:smallint> +-- !query 6 output +32767 -32768 + + +-- !query 7 +select 32768S +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.catalyst.parser.ParseException + +Numeric literal 32768 does not fit in range [-32768, 32767] for type smallint(line 1, pos 7) + +== SQL == +select 32768S +-------^^^ + + +-- !query 8 +select 1L, 2147483648L +-- !query 8 schema +struct<1:bigint,2147483648:bigint> +-- !query 8 output +1 2147483648 + + +-- !query 9 +select 9223372036854775807L, -9223372036854775808L +-- !query 9 schema +struct<9223372036854775807:bigint,-9223372036854775808:bigint> +-- !query 9 output +9223372036854775807 -9223372036854775808 + + +-- !query 10 +select 9223372036854775808L +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.catalyst.parser.ParseException + +Numeric literal 9223372036854775808 does not fit in range [-9223372036854775808, 9223372036854775807] for type bigint(line 1, pos 7) + +== SQL == +select 9223372036854775808L +-------^^^ + + +-- !query 11 +select 1, -1 +-- !query 11 schema +struct<1:int,-1:int> +-- !query 11 output +1 -1 + + +-- !query 12 +select 2147483647, -2147483648 +-- !query 12 schema +struct<2147483647:int,-2147483648:int> +-- !query 12 output +2147483647 -2147483648 + + +-- !query 13 +select 9223372036854775807, -9223372036854775808 +-- !query 13 schema +struct<9223372036854775807:bigint,-9223372036854775808:bigint> +-- !query 13 output +9223372036854775807 -9223372036854775808 + + +-- !query 14 +select 9223372036854775808, -9223372036854775809 +-- !query 14 schema +struct<9223372036854775808:decimal(19,0),-9223372036854775809:decimal(19,0)> +-- !query 14 output +9223372036854775808 -9223372036854775809 + + +-- !query 15 +select 1234567890123456789012345678901234567890 +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.catalyst.parser.ParseException + +DecimalType can only support precision up to 38 +== SQL == +select 1234567890123456789012345678901234567890 + + +-- !query 16 +select 1234567890123456789012345678901234567890.0 +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.catalyst.parser.ParseException + +DecimalType can only support precision up to 38 +== SQL == +select 1234567890123456789012345678901234567890.0 + + +-- !query 17 +select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 +-- !query 17 schema +struct<1.0:double,1.2:double,1E+10:decimal(1,-10),1.5E+5:decimal(2,-4),0.1:double,0.1:double,1E+4:decimal(1,-4),9E+1:decimal(1,-1),9E+1:decimal(1,-1),90.0:decimal(3,1),9E+1:decimal(1,-1)> +-- !query 17 output +1.0 1.2 10000000000 150000 0.1 0.1 10000 90 90 90 90 + + +-- !query 18 +select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5 +-- !query 18 schema +struct<-1.0:double,-1.2:double,-1E+10:decimal(1,-10),-1.5E+5:decimal(2,-4),-0.1:double,-0.1:double,-1E+4:decimal(1,-4)> +-- !query 18 output +-1.0 -1.2 -10000000000 -150000 -0.1 -0.1 -10000 + + +-- !query 19 +select .e3 +-- !query 19 schema +struct<> +-- !query 19 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'select .'(line 1, pos 7) + +== SQL == +select .e3 +-------^^^ + + +-- !query 20 +select 1E309, -1E309 +-- !query 20 schema +struct<1E+309:decimal(1,-309),-1E+309:decimal(1,-309)> +-- !query 20 output +1000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 -1000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 + + +-- !query 21 +select 0.3, -0.8, .5, -.18, 0.1111, .1111 +-- !query 21 schema +struct<0.3:decimal(1,1),-0.8:decimal(1,1),0.5:decimal(1,1),-0.18:decimal(2,2),0.1111:decimal(4,4),0.1111:decimal(4,4)> +-- !query 21 output +0.3 -0.8 0.5 -0.18 0.1111 0.1111 + + +-- !query 22 +select 123456789012345678901234567890123456789e10d, 123456789012345678901234567890123456789.1e10d +-- !query 22 schema +struct<1.2345678901234568E48:double,1.2345678901234568E48:double> +-- !query 22 output +1.2345678901234568E48 1.2345678901234568E48 + + +-- !query 23 +select "Hello Peter!", 'hello lee!' +-- !query 23 schema +struct +-- !query 23 output +Hello Peter! hello lee! + + +-- !query 24 +select 'hello' 'world', 'hello' " " 'lee' +-- !query 24 schema +struct +-- !query 24 output +helloworld hello lee + + +-- !query 25 +select "hello 'peter'" +-- !query 25 schema +struct +-- !query 25 output +hello 'peter' + + +-- !query 26 +select 'pattern%', 'no-pattern\%', 'pattern\\%', 'pattern\\\%' +-- !query 26 schema +struct +-- !query 26 output +pattern% no-pattern\% pattern\% pattern\\% + + +-- !query 27 +select '\'', '"', '\n', '\r', '\t', 'Z' +-- !query 27 schema +struct<':string,":string, +:string, :string, :string,Z:string> +-- !query 27 output +' " + Z + + +-- !query 28 +select '\110\145\154\154\157\041' +-- !query 28 schema +struct +-- !query 28 output +Hello! + + +-- !query 29 +select '\u0057\u006F\u0072\u006C\u0064\u0020\u003A\u0029' +-- !query 29 schema +struct +-- !query 29 output +World :) + + +-- !query 30 +select dAte '2016-03-12' +-- !query 30 schema +struct +-- !query 30 output +2016-03-12 + + +-- !query 31 +select date 'mar 11 2016' +-- !query 31 schema +struct<> +-- !query 31 output +org.apache.spark.sql.catalyst.parser.ParseException + +Exception parsing DATE(line 1, pos 7) + +== SQL == +select date 'mar 11 2016' +-------^^^ + + +-- !query 32 +select tImEstAmp '2016-03-11 20:54:00.000' +-- !query 32 schema +struct +-- !query 32 output +2016-03-11 20:54:00 + + +-- !query 33 +select timestamp '2016-33-11 20:54:00.000' +-- !query 33 schema +struct<> +-- !query 33 output +org.apache.spark.sql.catalyst.parser.ParseException + +Timestamp format must be yyyy-mm-dd hh:mm:ss[.fffffffff](line 1, pos 7) + +== SQL == +select timestamp '2016-33-11 20:54:00.000' +-------^^^ + + +-- !query 34 +select interval 13.123456789 seconds, interval -13.123456789 second +-- !query 34 schema +struct<> +-- !query 34 output +scala.MatchError +(interval 13 seconds 123 milliseconds 456 microseconds,CalendarIntervalType) (of class scala.Tuple2) + + +-- !query 35 +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond, 9 microsecond +-- !query 35 schema +struct<> +-- !query 35 output +scala.MatchError +(interval 1 years 2 months 3 weeks 4 days 5 hours 6 minutes 7 seconds 8 milliseconds,CalendarIntervalType) (of class scala.Tuple2) + + +-- !query 36 +select interval 10 nanoseconds +-- !query 36 schema +struct<> +-- !query 36 output +org.apache.spark.sql.catalyst.parser.ParseException + +No interval can be constructed(line 1, pos 16) + +== SQL == +select interval 10 nanoseconds +----------------^^^ + + +-- !query 37 +select GEO '(10,-6)' +-- !query 37 schema +struct<> +-- !query 37 output +org.apache.spark.sql.catalyst.parser.ParseException + +Literals of type 'GEO' are currently not supported.(line 1, pos 7) + +== SQL == +select GEO '(10,-6)' +-------^^^ + + +-- !query 38 +select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD +-- !query 38 schema +struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> +-- !query 38 output +90912830918230182310293801923652346786 0.0000000000000000000000000123 123.08 + + +-- !query 39 +select 1.20E-38BD +-- !query 39 schema +struct<> +-- !query 39 output +org.apache.spark.sql.catalyst.parser.ParseException + +DecimalType can only support precision up to 38(line 1, pos 7) + +== SQL == +select 1.20E-38BD +-------^^^ + + +-- !query 40 +select x'2379ACFe' +-- !query 40 schema +struct +-- !query 40 output +#y�� + + +-- !query 41 +select X'XuZ' +-- !query 41 schema +struct<> +-- !query 41 output +org.apache.spark.sql.catalyst.parser.ParseException + +contains illegal character for hexBinary: 0XuZ(line 1, pos 7) + +== SQL == +select X'XuZ' +-------^^^ + + +-- !query 42 +SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 +-- !query 42 schema +struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10),-3.14E+8:decimal(3,-6),-3.14E-8:decimal(10,10),3.14E+8:decimal(3,-6),3.14E+8:decimal(3,-6),3.14E-8:decimal(10,10)> +-- !query 42 output +3.14 -3.14 314000000 0.0000000314 -314000000 -0.0000000314 314000000 314000000 0.0000000314 diff --git a/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out new file mode 100644 index 000000000000..43f2f9af61d9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out @@ -0,0 +1,64 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 6 + + +-- !query 0 +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT * FROM nt1 natural join nt2 where k = "one" +-- !query 2 schema +struct +-- !query 2 output +one 1 1 +one 1 5 + + +-- !query 3 +SELECT * FROM nt1 natural left join nt2 order by v1, v2 +-- !query 3 schema +struct +-- !query 3 output +one 1 1 +one 1 5 +two 2 22 +three 3 NULL + + +-- !query 4 +SELECT * FROM nt1 natural right join nt2 order by v1, v2 +-- !query 4 schema +struct +-- !query 4 output +one 1 1 +one 1 5 +two 2 22 + + +-- !query 5 +SELECT count(*) FROM nt1 natural full outer join nt2 +-- !query 5 schema +struct +-- !query 5 output +4 diff --git a/sql/core/src/test/resources/sql-tests/results/null-propagation.sql.out b/sql/core/src/test/resources/sql-tests/results/null-propagation.sql.out new file mode 100644 index 000000000000..ed3a651aa661 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/null-propagation.sql.out @@ -0,0 +1,38 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 4 + + +-- !query 0 +SELECT COUNT(NULL) FROM VALUES 1, 2, 3 +-- !query 0 schema +struct +-- !query 0 output +0 + + +-- !query 1 +SELECT COUNT(1 + NULL) FROM VALUES 1, 2, 3 +-- !query 1 schema +struct +-- !query 1 output +0 + + +-- !query 2 +SELECT COUNT(NULL) OVER () FROM VALUES 1, 2, 3 +-- !query 2 schema +struct +-- !query 2 output +0 +0 +0 + + +-- !query 3 +SELECT COUNT(1 + NULL) OVER () FROM VALUES 1, 2, 3 +-- !query 3 schema +struct +-- !query 3 output +0 +0 +0 diff --git a/sql/core/src/test/resources/sql-tests/results/operators.sql.out b/sql/core/src/test/resources/sql-tests/results/operators.sql.out new file mode 100644 index 000000000000..237b618a8b90 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/operators.sql.out @@ -0,0 +1,486 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 59 + + +-- !query 0 +select -100 +-- !query 0 schema +struct<-100:int> +-- !query 0 output +-100 + + +-- !query 1 +select +230 +-- !query 1 schema +struct<230:int> +-- !query 1 output +230 + + +-- !query 2 +select -5.2 +-- !query 2 schema +struct<-5.2:decimal(2,1)> +-- !query 2 output +-5.2 + + +-- !query 3 +select +6.8e0 +-- !query 3 schema +struct<6.8:decimal(2,1)> +-- !query 3 output +6.8 + + +-- !query 4 +select -key, +key from testdata where key = 2 +-- !query 4 schema +struct<(- key):int,key:int> +-- !query 4 output +-2 2 + + +-- !query 5 +select -(key + 1), - key + 1, +(key + 5) from testdata where key = 1 +-- !query 5 schema +struct<(- (key + 1)):int,((- key) + 1):int,(key + 5):int> +-- !query 5 output +-2 0 6 + + +-- !query 6 +select -max(key), +max(key) from testdata +-- !query 6 schema +struct<(- max(key)):int,max(key):int> +-- !query 6 output +-100 100 + + +-- !query 7 +select - (-10) +-- !query 7 schema +struct<(- -10):int> +-- !query 7 output +10 + + +-- !query 8 +select + (-key) from testdata where key = 32 +-- !query 8 schema +struct<(- key):int> +-- !query 8 output +-32 + + +-- !query 9 +select - (+max(key)) from testdata +-- !query 9 schema +struct<(- max(key)):int> +-- !query 9 output +-100 + + +-- !query 10 +select - - 3 +-- !query 10 schema +struct<(- -3):int> +-- !query 10 output +3 + + +-- !query 11 +select - + 20 +-- !query 11 schema +struct<(- 20):int> +-- !query 11 output +-20 + + +-- !query 12 +select + + 100 +-- !query 12 schema +struct<100:int> +-- !query 12 output +100 + + +-- !query 13 +select - - max(key) from testdata +-- !query 13 schema +struct<(- (- max(key))):int> +-- !query 13 output +100 + + +-- !query 14 +select + - key from testdata where key = 33 +-- !query 14 schema +struct<(- key):int> +-- !query 14 output +-33 + + +-- !query 15 +select 5 / 2 +-- !query 15 schema +struct<(CAST(5 AS DOUBLE) / CAST(2 AS DOUBLE)):double> +-- !query 15 output +2.5 + + +-- !query 16 +select 5 / 0 +-- !query 16 schema +struct<(CAST(5 AS DOUBLE) / CAST(0 AS DOUBLE)):double> +-- !query 16 output +NULL + + +-- !query 17 +select 5 / null +-- !query 17 schema +struct<(CAST(5 AS DOUBLE) / CAST(NULL AS DOUBLE)):double> +-- !query 17 output +NULL + + +-- !query 18 +select null / 5 +-- !query 18 schema +struct<(CAST(NULL AS DOUBLE) / CAST(5 AS DOUBLE)):double> +-- !query 18 output +NULL + + +-- !query 19 +select 5 div 2 +-- !query 19 schema +struct +-- !query 19 output +2 + + +-- !query 20 +select 5 div 0 +-- !query 20 schema +struct +-- !query 20 output +NULL + + +-- !query 21 +select 5 div null +-- !query 21 schema +struct +-- !query 21 output +NULL + + +-- !query 22 +select null div 5 +-- !query 22 schema +struct +-- !query 22 output +NULL + + +-- !query 23 +select 1 + 2 +-- !query 23 schema +struct<(1 + 2):int> +-- !query 23 output +3 + + +-- !query 24 +select 1 - 2 +-- !query 24 schema +struct<(1 - 2):int> +-- !query 24 output +-1 + + +-- !query 25 +select 2 * 5 +-- !query 25 schema +struct<(2 * 5):int> +-- !query 25 output +10 + + +-- !query 26 +select 5 % 3 +-- !query 26 schema +struct<(5 % 3):int> +-- !query 26 output +2 + + +-- !query 27 +select pmod(-7, 3) +-- !query 27 schema +struct +-- !query 27 output +2 + + +-- !query 28 +explain select 'a' || 1 + 2 +-- !query 28 schema +struct +-- !query 28 output +== Physical Plan == +*Project [null AS (CAST(concat(a, CAST(1 AS STRING)) AS DOUBLE) + CAST(2 AS DOUBLE))#x] ++- Scan OneRowRelation[] + + +-- !query 29 +explain select 1 - 2 || 'b' +-- !query 29 schema +struct +-- !query 29 output +== Physical Plan == +*Project [-1b AS concat(CAST((1 - 2) AS STRING), b)#x] ++- Scan OneRowRelation[] + + +-- !query 30 +explain select 2 * 4 + 3 || 'b' +-- !query 30 schema +struct +-- !query 30 output +== Physical Plan == +*Project [11b AS concat(CAST(((2 * 4) + 3) AS STRING), b)#x] ++- Scan OneRowRelation[] + + +-- !query 31 +explain select 3 + 1 || 'a' || 4 / 2 +-- !query 31 schema +struct +-- !query 31 output +== Physical Plan == +*Project [4a2.0 AS concat(concat(CAST((3 + 1) AS STRING), a), CAST((CAST(4 AS DOUBLE) / CAST(2 AS DOUBLE)) AS STRING))#x] ++- Scan OneRowRelation[] + + +-- !query 32 +explain select 1 == 1 OR 'a' || 'b' == 'ab' +-- !query 32 schema +struct +-- !query 32 output +== Physical Plan == +*Project [true AS ((1 = 1) OR (concat(a, b) = ab))#x] ++- Scan OneRowRelation[] + + +-- !query 33 +explain select 'a' || 'c' == 'ac' AND 2 == 3 +-- !query 33 schema +struct +-- !query 33 output +== Physical Plan == +*Project [false AS ((concat(a, c) = ac) AND (2 = 3))#x] ++- Scan OneRowRelation[] + + +-- !query 34 +select cot(1) +-- !query 34 schema +struct +-- !query 34 output +0.6420926159343306 + + +-- !query 35 +select cot(null) +-- !query 35 schema +struct +-- !query 35 output +NULL + + +-- !query 36 +select cot(0) +-- !query 36 schema +struct +-- !query 36 output +Infinity + + +-- !query 37 +select cot(-1) +-- !query 37 schema +struct +-- !query 37 output +-0.6420926159343306 + + +-- !query 38 +select ceiling(0) +-- !query 38 schema +struct +-- !query 38 output +0 + + +-- !query 39 +select ceiling(1) +-- !query 39 schema +struct +-- !query 39 output +1 + + +-- !query 40 +select ceil(1234567890123456) +-- !query 40 schema +struct +-- !query 40 output +1234567890123456 + + +-- !query 41 +select ceiling(1234567890123456) +-- !query 41 schema +struct +-- !query 41 output +1234567890123456 + + +-- !query 42 +select ceil(0.01) +-- !query 42 schema +struct +-- !query 42 output +1 + + +-- !query 43 +select ceiling(-0.10) +-- !query 43 schema +struct +-- !query 43 output +0 + + +-- !query 44 +select floor(0) +-- !query 44 schema +struct +-- !query 44 output +0 + + +-- !query 45 +select floor(1) +-- !query 45 schema +struct +-- !query 45 output +1 + + +-- !query 46 +select floor(1234567890123456) +-- !query 46 schema +struct +-- !query 46 output +1234567890123456 + + +-- !query 47 +select floor(0.01) +-- !query 47 schema +struct +-- !query 47 output +0 + + +-- !query 48 +select floor(-0.10) +-- !query 48 schema +struct +-- !query 48 output +-1 + + +-- !query 49 +select 1 > 0.00001 +-- !query 49 schema +struct<(CAST(1 AS BIGINT) > 0):boolean> +-- !query 49 output +true + + +-- !query 50 +select mod(7, 2), mod(7, 0), mod(0, 2), mod(7, null), mod(null, 2), mod(null, null) +-- !query 50 schema +struct<(7 % 2):int,(7 % 0):int,(0 % 2):int,(7 % CAST(NULL AS INT)):int,(CAST(NULL AS INT) % 2):int,(CAST(NULL AS DOUBLE) % CAST(NULL AS DOUBLE)):double> +-- !query 50 output +1 NULL 0 NULL NULL NULL + + +-- !query 51 +select BIT_LENGTH('abc') +-- !query 51 schema +struct +-- !query 51 output +24 + + +-- !query 52 +select CHAR_LENGTH('abc') +-- !query 52 schema +struct +-- !query 52 output +3 + + +-- !query 53 +select CHARACTER_LENGTH('abc') +-- !query 53 schema +struct +-- !query 53 output +3 + + +-- !query 54 +select OCTET_LENGTH('abc') +-- !query 54 schema +struct +-- !query 54 output +3 + + +-- !query 55 +select abs(-3.13), abs('-2.19') +-- !query 55 schema +struct +-- !query 55 output +3.13 2.19 + + +-- !query 56 +select positive('-1.11'), positive(-1.11), negative('-1.11'), negative(-1.11) +-- !query 56 schema +struct<(+ CAST(-1.11 AS DOUBLE)):double,(+ -1.11):decimal(3,2),(- CAST(-1.11 AS DOUBLE)):double,(- -1.11):decimal(3,2)> +-- !query 56 output +-1.11 -1.11 1.11 1.11 + + +-- !query 57 +select pmod(-7, 2), pmod(0, 2), pmod(7, 0), pmod(7, null), pmod(null, 2), pmod(null, null) +-- !query 57 schema +struct +-- !query 57 output +1 0 NULL NULL NULL NULL + + +-- !query 58 +select pmod(cast(3.13 as decimal), cast(0 as decimal)), pmod(cast(2 as smallint), cast(0 as smallint)) +-- !query 58 schema +struct +-- !query 58 output +NULL NULL diff --git a/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out b/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out new file mode 100644 index 000000000000..c1b63dfb8cae --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out @@ -0,0 +1,254 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 17 + + +-- !query 0 +create table spark_10747(col1 int, col2 int, col3 int) using parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO spark_10747 VALUES (6, 12, 10), (6, 11, 4), (6, 9, 10), (6, 15, 8), +(6, 15, 8), (6, 7, 4), (6, 7, 8), (6, 13, null), (6, 10, null) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 desc nulls last, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query 2 schema +struct +-- !query 2 output +6 9 10 28 +6 13 NULL 34 +6 10 NULL 41 +6 12 10 43 +6 15 8 55 +6 15 8 56 +6 11 4 56 +6 7 8 58 +6 7 4 58 + + +-- !query 3 +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 desc nulls first, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query 3 schema +struct +-- !query 3 output +6 10 NULL 32 +6 11 4 33 +6 13 NULL 44 +6 7 4 48 +6 9 10 51 +6 15 8 55 +6 12 10 56 +6 15 8 56 +6 7 8 58 + + +-- !query 4 +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 asc nulls last, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query 4 schema +struct +-- !query 4 output +6 7 4 25 +6 13 NULL 35 +6 11 4 40 +6 10 NULL 44 +6 7 8 55 +6 15 8 57 +6 15 8 58 +6 12 10 59 +6 9 10 61 + + +-- !query 5 +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 asc nulls first, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query 5 schema +struct +-- !query 5 output +6 10 NULL 30 +6 12 10 36 +6 13 NULL 41 +6 7 4 48 +6 9 10 51 +6 11 4 53 +6 7 8 55 +6 15 8 57 +6 15 8 58 + + +-- !query 6 +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 ASC NULLS FIRST, COL2 +-- !query 6 schema +struct +-- !query 6 output +6 10 NULL +6 13 NULL +6 7 4 +6 11 4 +6 7 8 +6 15 8 +6 15 8 +6 9 10 +6 12 10 + + +-- !query 7 +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 NULLS LAST, COL2 +-- !query 7 schema +struct +-- !query 7 output +6 7 4 +6 11 4 +6 7 8 +6 15 8 +6 15 8 +6 9 10 +6 12 10 +6 10 NULL +6 13 NULL + + +-- !query 8 +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS FIRST, COL2 +-- !query 8 schema +struct +-- !query 8 output +6 10 NULL +6 13 NULL +6 9 10 +6 12 10 +6 7 8 +6 15 8 +6 15 8 +6 7 4 +6 11 4 + + +-- !query 9 +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS LAST, COL2 +-- !query 9 schema +struct +-- !query 9 output +6 9 10 +6 12 10 +6 7 8 +6 15 8 +6 15 8 +6 7 4 +6 11 4 +6 10 NULL +6 13 NULL + + +-- !query 10 +drop table spark_10747 +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +create table spark_10747_mix( +col1 string, +col2 int, +col3 double, +col4 decimal(10,2), +col5 decimal(20,1)) +using parquet +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +INSERT INTO spark_10747_mix VALUES +('b', 2, 1.0, 1.00, 10.0), +('d', 3, 2.0, 3.00, 0.0), +('c', 3, 2.0, 2.00, 15.1), +('d', 3, 0.0, 3.00, 1.0), +(null, 3, 0.0, 3.00, 1.0), +('d', 3, null, 4.00, 1.0), +('a', 1, 1.0, 1.00, null), +('c', 3, 2.0, 2.00, null) +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +select * from spark_10747_mix order by col1 nulls last, col5 nulls last +-- !query 13 schema +struct +-- !query 13 output +a 1 1.0 1 NULL +b 2 1.0 1 10 +c 3 2.0 2 15.1 +c 3 2.0 2 NULL +d 3 2.0 3 0 +d 3 0.0 3 1 +d 3 NULL 4 1 +NULL 3 0.0 3 1 + + +-- !query 14 +select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls first +-- !query 14 schema +struct +-- !query 14 output +NULL 3 0.0 3 1 +d 3 0.0 3 1 +d 3 NULL 4 1 +d 3 2.0 3 0 +c 3 2.0 2 NULL +c 3 2.0 2 15.1 +b 2 1.0 1 10 +a 1 1.0 1 NULL + + +-- !query 15 +select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls last +-- !query 15 schema +struct +-- !query 15 output +c 3 2.0 2 NULL +a 1 1.0 1 NULL +c 3 2.0 2 15.1 +b 2 1.0 1 10 +d 3 0.0 3 1 +NULL 3 0.0 3 1 +d 3 NULL 4 1 +d 3 2.0 3 0 + + +-- !query 16 +drop table spark_10747_mix +-- !query 16 schema +struct<> +-- !query 16 output + diff --git a/sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out new file mode 100644 index 000000000000..cc47cc67c87c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out @@ -0,0 +1,143 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 12 + + +-- !query 0 +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +select * from data order by 1 desc +-- !query 1 schema +struct +-- !query 1 output +3 1 +3 2 +2 1 +2 2 +1 1 +1 2 + + +-- !query 2 +select * from data order by 1 desc, b desc +-- !query 2 schema +struct +-- !query 2 output +3 2 +3 1 +2 2 +2 1 +1 2 +1 1 + + +-- !query 3 +select * from data order by 1 desc, 2 desc +-- !query 3 schema +struct +-- !query 3 output +3 2 +3 1 +2 2 +2 1 +1 2 +1 1 + + +-- !query 4 +select * from data order by 1 + 0 desc, b desc +-- !query 4 schema +struct +-- !query 4 output +1 2 +2 2 +3 2 +1 1 +2 1 +3 1 + + +-- !query 5 +select * from data order by 0 +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +ORDER BY position 0 is not in select list (valid range is [1, 2]); line 1 pos 28 + + +-- !query 6 +select * from data order by -1 +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.AnalysisException +ORDER BY position -1 is not in select list (valid range is [1, 2]); line 1 pos 28 + + +-- !query 7 +select * from data order by 3 +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.AnalysisException +ORDER BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 28 + + +-- !query 8 +select * from data sort by 1 desc +-- !query 8 schema +struct +-- !query 8 output +1 1 +1 2 +2 1 +2 2 +3 1 +3 2 + + +-- !query 9 +set spark.sql.orderByOrdinal=false +-- !query 9 schema +struct +-- !query 9 output +spark.sql.orderByOrdinal false + + +-- !query 10 +select * from data order by 0 +-- !query 10 schema +struct +-- !query 10 output +1 1 +1 2 +2 1 +2 2 +3 1 +3 2 + + +-- !query 11 +select * from data sort by 0 +-- !query 11 schema +struct +-- !query 11 output +1 1 +1 2 +2 1 +2 2 +3 1 +3 2 diff --git a/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out b/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out new file mode 100644 index 000000000000..5db3bae5d037 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/outer-join.sql.out @@ -0,0 +1,88 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES +(-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) +as t2(int_col0, int_col1) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT + (SUM(COALESCE(t1.int_col1, t2.int_col0))), + ((COALESCE(t1.int_col1, t2.int_col0)) * 2) +FROM t1 +RIGHT JOIN t2 + ON (t2.int_col0) = (t1.int_col1) +GROUP BY GREATEST(COALESCE(t2.int_col1, 109), COALESCE(t1.int_col1, -449)), + COALESCE(t1.int_col1, t2.int_col0) +HAVING (SUM(COALESCE(t1.int_col1, t2.int_col0))) + > ((COALESCE(t1.int_col1, t2.int_col0)) * 2) +-- !query 2 schema +struct +-- !query 2 output +-367 -734 +-507 -1014 +-769 -1538 +-800 -1600 + + +-- !query 3 +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1) +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +set spark.sql.crossJoin.enabled = true +-- !query 5 schema +struct +-- !query 5 output +spark.sql.crossJoin.enabled true + + +-- !query 6 +SELECT * +FROM ( +SELECT + COALESCE(t2.int_col1, t1.int_col1) AS int_col + FROM t1 + LEFT JOIN t2 ON false +) t where (t.int_col) is not null +-- !query 6 schema +struct +-- !query 6 output +97 + + +-- !query 7 +set spark.sql.crossJoin.enabled = false +-- !query 7 schema +struct +-- !query 7 output +spark.sql.crossJoin.enabled false diff --git a/sql/core/src/test/resources/sql-tests/results/pred-pushdown.sql.out b/sql/core/src/test/resources/sql-tests/results/pred-pushdown.sql.out new file mode 100644 index 000000000000..1b8ddbe4c721 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pred-pushdown.sql.out @@ -0,0 +1,40 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 4 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW tbl_a AS VALUES (1, 1), (2, 1), (3, 6) AS T(c1, c2) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE OR REPLACE TEMPORARY VIEW tbl_b AS VALUES 1 AS T(c1) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT * +FROM tbl_a + LEFT ANTI JOIN tbl_b ON ((tbl_a.c1 = tbl_a.c2) IS NULL OR tbl_a.c1 = tbl_a.c2) +-- !query 2 schema +struct +-- !query 2 output +2 1 +3 6 + + +-- !query 3 +SELECT l.c1, l.c2 +FROM tbl_a l +WHERE EXISTS (SELECT 1 FROM tbl_b r WHERE l.c1 = l.c2) OR l.c2 < 2 +-- !query 3 schema +struct +-- !query 3 output +1 1 +2 1 diff --git a/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out new file mode 100644 index 000000000000..8e7e04c8e1c4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out @@ -0,0 +1,218 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 27 + + +-- !query 0 +select 1 = 1 +-- !query 0 schema +struct<(1 = 1):boolean> +-- !query 0 output +true + + +-- !query 1 +select 1 = '1' +-- !query 1 schema +struct<(1 = CAST(1 AS INT)):boolean> +-- !query 1 output +true + + +-- !query 2 +select 1.0 = '1' +-- !query 2 schema +struct<(1.0 = CAST(1 AS DECIMAL(2,1))):boolean> +-- !query 2 output +true + + +-- !query 3 +select 1 > '1' +-- !query 3 schema +struct<(1 > CAST(1 AS INT)):boolean> +-- !query 3 output +false + + +-- !query 4 +select 2 > '1.0' +-- !query 4 schema +struct<(2 > CAST(1.0 AS INT)):boolean> +-- !query 4 output +true + + +-- !query 5 +select 2 > '2.0' +-- !query 5 schema +struct<(2 > CAST(2.0 AS INT)):boolean> +-- !query 5 output +false + + +-- !query 6 +select 2 > '2.2' +-- !query 6 schema +struct<(2 > CAST(2.2 AS INT)):boolean> +-- !query 6 output +false + + +-- !query 7 +select to_date('2009-07-30 04:17:52') > to_date('2009-07-30 04:17:52') +-- !query 7 schema +struct<(to_date('2009-07-30 04:17:52') > to_date('2009-07-30 04:17:52')):boolean> +-- !query 7 output +false + + +-- !query 8 +select to_date('2009-07-30 04:17:52') > '2009-07-30 04:17:52' +-- !query 8 schema +struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) > 2009-07-30 04:17:52):boolean> +-- !query 8 output +false + + +-- !query 9 +select 1 >= '1' +-- !query 9 schema +struct<(1 >= CAST(1 AS INT)):boolean> +-- !query 9 output +true + + +-- !query 10 +select 2 >= '1.0' +-- !query 10 schema +struct<(2 >= CAST(1.0 AS INT)):boolean> +-- !query 10 output +true + + +-- !query 11 +select 2 >= '2.0' +-- !query 11 schema +struct<(2 >= CAST(2.0 AS INT)):boolean> +-- !query 11 output +true + + +-- !query 12 +select 2.0 >= '2.2' +-- !query 12 schema +struct<(2.0 >= CAST(2.2 AS DECIMAL(2,1))):boolean> +-- !query 12 output +false + + +-- !query 13 +select to_date('2009-07-30 04:17:52') >= to_date('2009-07-30 04:17:52') +-- !query 13 schema +struct<(to_date('2009-07-30 04:17:52') >= to_date('2009-07-30 04:17:52')):boolean> +-- !query 13 output +true + + +-- !query 14 +select to_date('2009-07-30 04:17:52') >= '2009-07-30 04:17:52' +-- !query 14 schema +struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) >= 2009-07-30 04:17:52):boolean> +-- !query 14 output +false + + +-- !query 15 +select 1 < '1' +-- !query 15 schema +struct<(1 < CAST(1 AS INT)):boolean> +-- !query 15 output +false + + +-- !query 16 +select 2 < '1.0' +-- !query 16 schema +struct<(2 < CAST(1.0 AS INT)):boolean> +-- !query 16 output +false + + +-- !query 17 +select 2 < '2.0' +-- !query 17 schema +struct<(2 < CAST(2.0 AS INT)):boolean> +-- !query 17 output +false + + +-- !query 18 +select 2.0 < '2.2' +-- !query 18 schema +struct<(2.0 < CAST(2.2 AS DECIMAL(2,1))):boolean> +-- !query 18 output +true + + +-- !query 19 +select to_date('2009-07-30 04:17:52') < to_date('2009-07-30 04:17:52') +-- !query 19 schema +struct<(to_date('2009-07-30 04:17:52') < to_date('2009-07-30 04:17:52')):boolean> +-- !query 19 output +false + + +-- !query 20 +select to_date('2009-07-30 04:17:52') < '2009-07-30 04:17:52' +-- !query 20 schema +struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) < 2009-07-30 04:17:52):boolean> +-- !query 20 output +true + + +-- !query 21 +select 1 <= '1' +-- !query 21 schema +struct<(1 <= CAST(1 AS INT)):boolean> +-- !query 21 output +true + + +-- !query 22 +select 2 <= '1.0' +-- !query 22 schema +struct<(2 <= CAST(1.0 AS INT)):boolean> +-- !query 22 output +false + + +-- !query 23 +select 2 <= '2.0' +-- !query 23 schema +struct<(2 <= CAST(2.0 AS INT)):boolean> +-- !query 23 output +true + + +-- !query 24 +select 2.0 <= '2.2' +-- !query 24 schema +struct<(2.0 <= CAST(2.2 AS DECIMAL(2,1))):boolean> +-- !query 24 output +true + + +-- !query 25 +select to_date('2009-07-30 04:17:52') <= to_date('2009-07-30 04:17:52') +-- !query 25 schema +struct<(to_date('2009-07-30 04:17:52') <= to_date('2009-07-30 04:17:52')):boolean> +-- !query 25 output +true + + +-- !query 26 +select to_date('2009-07-30 04:17:52') <= '2009-07-30 04:17:52' +-- !query 26 schema +struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) <= 2009-07-30 04:17:52):boolean> +-- !query 26 output +true diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out new file mode 100644 index 000000000000..2dade86f35df --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out @@ -0,0 +1,313 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 34 + + +-- !query 0 +set spark.sql.parser.quotedRegexColumnNames=false +-- !query 0 schema +struct +-- !query 0 output +spark.sql.parser.quotedRegexColumnNames false + + +-- !query 1 +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66") +AS testData(key, value1, value2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES +(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4) +AS testData2(A, B, c, d) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT `(a)?+.+` FROM testData2 WHERE a = 1 +-- !query 3 schema +struct<> +-- !query 3 output +org.apache.spark.sql.AnalysisException +cannot resolve '`(a)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 + + +-- !query 4 +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +cannot resolve 't.`(a)?+.+`' given input columns: [t.A, t.B, t.c, t.d]; line 1 pos 7 + + +-- !query 5 +SELECT `(a|b)` FROM testData2 WHERE a = 2 +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +cannot resolve '`(a|b)`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 + + +-- !query 6 +SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.AnalysisException +cannot resolve '`(a|b)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7 + + +-- !query 7 +SELECT SUM(`(a|b)?+.+`) FROM testData2 +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.AnalysisException +cannot resolve '`(a|b)?+.+`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 + + +-- !query 8 +SELECT SUM(`(a)`) FROM testData2 +-- !query 8 schema +struct<> +-- !query 8 output +org.apache.spark.sql.AnalysisException +cannot resolve '`(a)`' given input columns: [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11 + + +-- !query 9 +set spark.sql.parser.quotedRegexColumnNames=true +-- !query 9 schema +struct +-- !query 9 output +spark.sql.parser.quotedRegexColumnNames true + + +-- !query 10 +SELECT `(a)?+.+` FROM testData2 WHERE a = 1 +-- !query 10 schema +struct +-- !query 10 output +1 1 2 +2 1 2 + + +-- !query 11 +SELECT `(A)?+.+` FROM testData2 WHERE a = 1 +-- !query 11 schema +struct +-- !query 11 output +1 1 2 +2 1 2 + + +-- !query 12 +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 +-- !query 12 schema +struct +-- !query 12 output +1 1 2 +2 1 2 + + +-- !query 13 +SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1 +-- !query 13 schema +struct +-- !query 13 output +1 1 2 +2 1 2 + + +-- !query 14 +SELECT `(a|B)` FROM testData2 WHERE a = 2 +-- !query 14 schema +struct +-- !query 14 output +2 1 +2 2 + + +-- !query 15 +SELECT `(A|b)` FROM testData2 WHERE a = 2 +-- !query 15 schema +struct +-- !query 15 output +2 1 +2 2 + + +-- !query 16 +SELECT `(a|B)?+.+` FROM testData2 WHERE a = 2 +-- !query 16 schema +struct +-- !query 16 output +2 3 +2 3 + + +-- !query 17 +SELECT `(A|b)?+.+` FROM testData2 WHERE a = 2 +-- !query 17 schema +struct +-- !query 17 output +2 3 +2 3 + + +-- !query 18 +SELECT `(e|f)` FROM testData2 +-- !query 18 schema +struct<> +-- !query 18 output + + + +-- !query 19 +SELECT t.`(e|f)` FROM testData2 t +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +SELECT p.`(KEY)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 +-- !query 20 schema +struct +-- !query 20 output +1 11 1 1 1 2 +1 11 2 1 1 2 +2 22 1 2 2 3 +2 22 2 2 2 3 + + +-- !query 21 +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 +-- !query 21 schema +struct +-- !query 21 output +1 11 1 1 1 2 +1 11 2 1 1 2 +2 22 1 2 2 3 +2 22 2 2 2 3 + + +-- !query 22 +set spark.sql.caseSensitive=true +-- !query 22 schema +struct +-- !query 22 output +spark.sql.caseSensitive true + + +-- !query 23 +CREATE OR REPLACE TEMPORARY VIEW testdata3 AS SELECT * FROM VALUES +(0, 1), (1, 2), (2, 3), (3, 4) +AS testdata3(a, b) +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +SELECT `(A)?+.+` FROM testdata3 +-- !query 24 schema +struct +-- !query 24 output +0 1 +1 2 +2 3 +3 4 + + +-- !query 25 +SELECT `(a)?+.+` FROM testdata3 +-- !query 25 schema +struct +-- !query 25 output +1 +2 +3 +4 + + +-- !query 26 +SELECT `(A)?+.+` FROM testdata3 WHERE a > 1 +-- !query 26 schema +struct +-- !query 26 output +2 3 +3 4 + + +-- !query 27 +SELECT `(a)?+.+` FROM testdata3 where `a` > 1 +-- !query 27 schema +struct +-- !query 27 output +3 +4 + + +-- !query 28 +SELECT SUM(`a`) FROM testdata3 +-- !query 28 schema +struct +-- !query 28 output +6 + + +-- !query 29 +SELECT SUM(`(a)`) FROM testdata3 +-- !query 29 schema +struct +-- !query 29 output +6 + + +-- !query 30 +SELECT SUM(`(a)?+.+`) FROM testdata3 +-- !query 30 schema +struct +-- !query 30 output +10 + + +-- !query 31 +SELECT SUM(a) FROM testdata3 GROUP BY `a` +-- !query 31 schema +struct +-- !query 31 output +0 +1 +2 +3 + + +-- !query 32 +SELECT SUM(a) FROM testdata3 GROUP BY `(a)` +-- !query 32 schema +struct<> +-- !query 32 output +org.apache.spark.sql.AnalysisException +cannot resolve '`(a)`' given input columns: [testdata3.a, testdata3.b]; line 1 pos 38 + + +-- !query 33 +SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+` +-- !query 33 schema +struct<> +-- !query 33 output +org.apache.spark.sql.AnalysisException +cannot resolve '`(a)?+.+`' given input columns: [testdata3.a, testdata3.b]; line 1 pos 38 diff --git a/sql/core/src/test/resources/sql-tests/results/random.sql.out b/sql/core/src/test/resources/sql-tests/results/random.sql.out new file mode 100644 index 000000000000..bca67320fe7b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/random.sql.out @@ -0,0 +1,84 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 10 + + +-- !query 0 +SELECT rand(0) +-- !query 0 schema +struct +-- !query 0 output +0.8446490682263027 + + +-- !query 1 +SELECT rand(cast(3 / 7 AS int)) +-- !query 1 schema +struct +-- !query 1 output +0.8446490682263027 + + +-- !query 2 +SELECT rand(NULL) +-- !query 2 schema +struct +-- !query 2 output +0.8446490682263027 + + +-- !query 3 +SELECT rand(cast(NULL AS int)) +-- !query 3 schema +struct +-- !query 3 output +0.8446490682263027 + + +-- !query 4 +SELECT rand(1.0) +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +cannot resolve 'rand(1.0BD)' due to data type mismatch: argument 1 requires (int or bigint) type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7 + + +-- !query 5 +SELECT randn(0L) +-- !query 5 schema +struct +-- !query 5 output +1.1164209726833079 + + +-- !query 6 +SELECT randn(cast(3 / 7 AS long)) +-- !query 6 schema +struct +-- !query 6 output +1.1164209726833079 + + +-- !query 7 +SELECT randn(NULL) +-- !query 7 schema +struct +-- !query 7 output +1.1164209726833079 + + +-- !query 8 +SELECT randn(cast(NULL AS long)) +-- !query 8 schema +struct +-- !query 8 output +1.1164209726833079 + + +-- !query 9 +SELECT rand('1') +-- !query 9 schema +struct<> +-- !query 9 output +org.apache.spark.sql.AnalysisException +cannot resolve 'rand('1')' due to data type mismatch: argument 1 requires (int or bigint) type, however, ''1'' is of string type.; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out new file mode 100644 index 000000000000..975bb0612474 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out @@ -0,0 +1,280 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 26 + + +-- !query 0 +CREATE DATABASE showdb +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +USE showdb +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TABLE show_t1(a String, b Int, c String, d String) USING parquet PARTITIONED BY (c, d) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +ALTER TABLE show_t1 ADD PARTITION (c='Us', d=1) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE TABLE show_t2(b String, d Int) USING parquet +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +CREATE TEMPORARY VIEW show_t3(e int) USING parquet +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +CREATE GLOBAL TEMP VIEW show_t4 AS SELECT 1 as col1 +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +SHOW TABLES +-- !query 7 schema +struct +-- !query 7 output +arraydata +mapdata +show_t1 +show_t2 +show_t3 +testdata + + +-- !query 8 +SHOW TABLES IN showdb +-- !query 8 schema +struct +-- !query 8 output +arraydata +mapdata +show_t1 +show_t2 +show_t3 +testdata + + +-- !query 9 +SHOW TABLES 'show_t*' +-- !query 9 schema +struct +-- !query 9 output +show_t1 +show_t2 +show_t3 + + +-- !query 10 +SHOW TABLES LIKE 'show_t1*|show_t2*' +-- !query 10 schema +struct +-- !query 10 output +show_t1 +show_t2 + + +-- !query 11 +SHOW TABLES IN showdb 'show_t*' +-- !query 11 schema +struct +-- !query 11 output +show_t1 +show_t2 +show_t3 + + +-- !query 12 +SHOW TABLE EXTENDED LIKE 'show_t*' +-- !query 12 schema +struct +-- !query 12 output +show_t3 true Table: show_t3 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type: VIEW +Schema: root + |-- e: integer (nullable = true) + + +showdb show_t1 false Database: showdb +Table: show_t1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type: MANAGED +Provider: parquet +Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t1 +Partition Provider: Catalog +Partition Columns: [`c`, `d`] +Schema: root + |-- a: string (nullable = true) + |-- b: integer (nullable = true) + |-- c: string (nullable = true) + |-- d: string (nullable = true) + + +showdb show_t2 false Database: showdb +Table: show_t2 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type: MANAGED +Provider: parquet +Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t2 +Schema: root + |-- b: string (nullable = true) + |-- d: integer (nullable = true) + + +-- !query 13 +SHOW TABLE EXTENDED +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.catalyst.parser.ParseException + +mismatched input '' expecting {'FROM', 'IN', 'LIKE'}(line 1, pos 19) + +== SQL == +SHOW TABLE EXTENDED +-------------------^^^ + + +-- !query 14 +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us', d=1) +-- !query 14 schema +struct +-- !query 14 output +showdb show_t1 false Partition Values: [c=Us, d=1] +Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t1/c=Us/d=1 + + +-- !query 15 +SHOW TABLE EXTENDED PARTITION(c='Us', d=1) +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.catalyst.parser.ParseException + +mismatched input 'PARTITION' expecting {'FROM', 'IN', 'LIKE'}(line 1, pos 20) + +== SQL == +SHOW TABLE EXTENDED PARTITION(c='Us', d=1) +--------------------^^^ + + +-- !query 16 +SHOW TABLE EXTENDED LIKE 'show_t*' PARTITION(c='Us', d=1) +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'show_t*' not found in database 'showdb'; + + +-- !query 17 +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us') +-- !query 17 schema +struct<> +-- !query 17 output +org.apache.spark.sql.AnalysisException +Partition spec is invalid. The spec (c) must match the partition spec (c, d) defined in table '`showdb`.`show_t1`'; + + +-- !query 18 +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(a='Us', d=1) +-- !query 18 schema +struct<> +-- !query 18 output +org.apache.spark.sql.AnalysisException +Partition spec is invalid. The spec (a, d) must match the partition spec (c, d) defined in table '`showdb`.`show_t1`'; + + +-- !query 19 +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Ch', d=1) +-- !query 19 schema +struct<> +-- !query 19 output +org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException +Partition not found in table 'show_t1' database 'showdb': +c -> Ch +d -> 1; + + +-- !query 20 +DROP TABLE show_t1 +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +DROP TABLE show_t2 +-- !query 21 schema +struct<> +-- !query 21 output + + + +-- !query 22 +DROP VIEW show_t3 +-- !query 22 schema +struct<> +-- !query 22 output + + + +-- !query 23 +DROP VIEW global_temp.show_t4 +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +USE default +-- !query 24 schema +struct<> +-- !query 24 output + + + +-- !query 25 +DROP DATABASE showdb +-- !query 25 schema +struct<> +-- !query 25 output + diff --git a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out new file mode 100644 index 000000000000..71d6e120e894 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out @@ -0,0 +1,217 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 25 + + +-- !query 0 +CREATE DATABASE showdb +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +USE showdb +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TABLE showcolumn1 (col1 int, `col 2` int) USING json +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE TABLE showcolumn2 (price int, qty int, year int, month int) USING parquet partitioned by (year, month) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE TEMPORARY VIEW showColumn3 (col3 int, `col 4` int) USING json +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +CREATE GLOBAL TEMP VIEW showColumn4 AS SELECT 1 as col1, 'abc' as `col 5` +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SHOW COLUMNS IN showcolumn1 +-- !query 6 schema +struct +-- !query 6 output +col 2 +col1 + + +-- !query 7 +SHOW COLUMNS IN showdb.showcolumn1 +-- !query 7 schema +struct +-- !query 7 output +col 2 +col1 + + +-- !query 8 +SHOW COLUMNS IN showcolumn1 FROM showdb +-- !query 8 schema +struct +-- !query 8 output +col 2 +col1 + + +-- !query 9 +SHOW COLUMNS IN showcolumn2 IN showdb +-- !query 9 schema +struct +-- !query 9 output +month +price +qty +year + + +-- !query 10 +SHOW COLUMNS IN badtable FROM showdb +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'badtable' not found in database 'showdb'; + + +-- !query 11 +SHOW COLUMNS IN showdb.showcolumn1 from SHOWDB +-- !query 11 schema +struct +-- !query 11 output +col 2 +col1 + + +-- !query 12 +SHOW COLUMNS IN showdb.showcolumn1 FROM baddb +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +SHOW COLUMNS with conflicting databases: 'baddb' != 'showdb'; + + +-- !query 13 +SHOW COLUMNS IN showcolumn3 +-- !query 13 schema +struct +-- !query 13 output +col 4 +col3 + + +-- !query 14 +SHOW COLUMNS IN showdb.showcolumn3 +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'showcolumn3' not found in database 'showdb'; + + +-- !query 15 +SHOW COLUMNS IN showcolumn3 FROM showdb +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'showcolumn3' not found in database 'showdb'; + + +-- !query 16 +SHOW COLUMNS IN showcolumn4 +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'showcolumn4' not found in database 'showdb'; + + +-- !query 17 +SHOW COLUMNS IN global_temp.showcolumn4 +-- !query 17 schema +struct +-- !query 17 output +col 5 +col1 + + +-- !query 18 +SHOW COLUMNS IN showcolumn4 FROM global_temp +-- !query 18 schema +struct +-- !query 18 output +col 5 +col1 + + +-- !query 19 +DROP TABLE showcolumn1 +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +DROP TABLE showColumn2 +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +DROP VIEW showcolumn3 +-- !query 21 schema +struct<> +-- !query 21 output + + + +-- !query 22 +DROP VIEW global_temp.showcolumn4 +-- !query 22 schema +struct<> +-- !query 22 output + + + +-- !query 23 +use default +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +DROP DATABASE showdb +-- !query 24 schema +struct<> +-- !query 24 output + diff --git a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out new file mode 100644 index 000000000000..e035505f15d2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out @@ -0,0 +1,140 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 15 + + +-- !query 0 +SELECT ifnull(null, 'x'), ifnull('y', 'x'), ifnull(null, null) +-- !query 0 schema +struct +-- !query 0 output +x y NULL + + +-- !query 1 +SELECT nullif('x', 'x'), nullif('x', 'y') +-- !query 1 schema +struct +-- !query 1 output +NULL x + + +-- !query 2 +SELECT nvl(null, 'x'), nvl('y', 'x'), nvl(null, null) +-- !query 2 schema +struct +-- !query 2 output +x y NULL + + +-- !query 3 +SELECT nvl2(null, 'x', 'y'), nvl2('n', 'x', 'y'), nvl2(null, null, null) +-- !query 3 schema +struct +-- !query 3 output +y x NULL + + +-- !query 4 +SELECT ifnull(1, 2.1d), ifnull(null, 2.1d) +-- !query 4 schema +struct +-- !query 4 output +1.0 2.1 + + +-- !query 5 +SELECT nullif(1, 2.1d), nullif(1, 1.0d) +-- !query 5 schema +struct +-- !query 5 output +1 NULL + + +-- !query 6 +SELECT nvl(1, 2.1d), nvl(null, 2.1d) +-- !query 6 schema +struct +-- !query 6 output +1.0 2.1 + + +-- !query 7 +SELECT nvl2(null, 1, 2.1d), nvl2('n', 1, 2.1d) +-- !query 7 schema +struct +-- !query 7 output +2.1 1.0 + + +-- !query 8 +explain extended +select ifnull(id, 'x'), nullif(id, 'x'), nvl(id, 'x'), nvl2(id, 'x', 'y') +from range(2) +-- !query 8 schema +struct +-- !query 8 output +== Parsed Logical Plan == +'Project [unresolvedalias('ifnull('id, x), None), unresolvedalias('nullif('id, x), None), unresolvedalias('nvl('id, x), None), unresolvedalias('nvl2('id, x, y), None)] ++- 'UnresolvedTableValuedFunction range, [2] + +== Analyzed Logical Plan == +ifnull(`id`, 'x'): string, nullif(`id`, 'x'): bigint, nvl(`id`, 'x'): string, nvl2(`id`, 'x', 'y'): string +Project [ifnull(id#xL, x) AS ifnull(`id`, 'x')#x, nullif(id#xL, x) AS nullif(`id`, 'x')#xL, nvl(id#xL, x) AS nvl(`id`, 'x')#x, nvl2(id#xL, x, y) AS nvl2(`id`, 'x', 'y')#x] ++- Range (0, 2, step=1, splits=None) + +== Optimized Logical Plan == +Project [coalesce(cast(id#xL as string), x) AS ifnull(`id`, 'x')#x, id#xL AS nullif(`id`, 'x')#xL, coalesce(cast(id#xL as string), x) AS nvl(`id`, 'x')#x, x AS nvl2(`id`, 'x', 'y')#x] ++- Range (0, 2, step=1, splits=None) + +== Physical Plan == +*Project [coalesce(cast(id#xL as string), x) AS ifnull(`id`, 'x')#x, id#xL AS nullif(`id`, 'x')#xL, coalesce(cast(id#xL as string), x) AS nvl(`id`, 'x')#x, x AS nvl2(`id`, 'x', 'y')#x] ++- *Range (0, 2, step=1, splits=2) + + +-- !query 9 +SELECT boolean(1), tinyint(1), smallint(1), int(1), bigint(1) +-- !query 9 schema +struct +-- !query 9 output +true 1 1 1 1 + + +-- !query 10 +SELECT float(1), double(1), decimal(1) +-- !query 10 schema +struct +-- !query 10 output +1.0 1.0 1 + + +-- !query 11 +SELECT date("2014-04-04"), timestamp(date("2014-04-04")) +-- !query 11 schema +struct +-- !query 11 output +2014-04-04 2014-04-04 00:00:00 + + +-- !query 12 +SELECT string(1, 2) +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +Function string accepts only one argument; line 1 pos 7 + + +-- !query 13 +CREATE TEMPORARY VIEW tempView1 AS VALUES (1, NAMED_STRUCT('col1', 'gamma', 'col2', 'delta')) AS T(id, st) +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +SELECT nvl(st.col1, "value"), count(*) FROM from tempView1 GROUP BY nvl(st.col1, "value") +-- !query 14 schema +struct +-- !query 14 output +gamma 1 diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out new file mode 100644 index 000000000000..2d9b3d7d2ca3 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -0,0 +1,120 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 12 + + +-- !query 0 +select concat_ws() +-- !query 0 schema +struct<> +-- !query 0 output +org.apache.spark.sql.AnalysisException +requirement failed: concat_ws requires at least one argument.; line 1 pos 7 + + +-- !query 1 +select format_string() +-- !query 1 schema +struct<> +-- !query 1 output +org.apache.spark.sql.AnalysisException +requirement failed: format_string() should take at least 1 argument; line 1 pos 7 + + +-- !query 2 +select 'a' || 'b' || 'c' +-- !query 2 schema +struct +-- !query 2 output +abc + + +-- !query 3 +EXPLAIN EXTENDED SELECT (col1 || col2 || col3 || col4) col +FROM (SELECT id col1, id col2, id col3, id col4 FROM range(10)) +-- !query 3 schema +struct +-- !query 3 output +== Parsed Logical Plan == +'Project [concat(concat(concat('col1, 'col2), 'col3), 'col4) AS col#x] ++- 'SubqueryAlias __auto_generated_subquery_name + +- 'Project ['id AS col1#x, 'id AS col2#x, 'id AS col3#x, 'id AS col4#x] + +- 'UnresolvedTableValuedFunction range, [10] + +== Analyzed Logical Plan == +col: string +Project [concat(concat(concat(cast(col1#xL as string), cast(col2#xL as string)), cast(col3#xL as string)), cast(col4#xL as string)) AS col#x] ++- SubqueryAlias __auto_generated_subquery_name + +- Project [id#xL AS col1#xL, id#xL AS col2#xL, id#xL AS col3#xL, id#xL AS col4#xL] + +- Range (0, 10, step=1, splits=None) + +== Optimized Logical Plan == +Project [concat(cast(id#xL as string), cast(id#xL as string), cast(id#xL as string), cast(id#xL as string)) AS col#x] ++- Range (0, 10, step=1, splits=None) + +== Physical Plan == +*Project [concat(cast(id#xL as string), cast(id#xL as string), cast(id#xL as string), cast(id#xL as string)) AS col#x] ++- *Range (0, 10, step=1, splits=2) + + +-- !query 4 +select replace('abc', 'b', '123') +-- !query 4 schema +struct +-- !query 4 output +a123c + + +-- !query 5 +select replace('abc', 'b') +-- !query 5 schema +struct +-- !query 5 output +ac + + +-- !query 6 +select length(uuid()), (uuid() <> uuid()) +-- !query 6 schema +struct +-- !query 6 output +36 true + + +-- !query 7 +select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null) +-- !query 7 schema +struct +-- !query 7 output +4 NULL NULL + + +-- !query 8 +select left("abcd", 2), left("abcd", 5), left("abcd", '2'), left("abcd", null) +-- !query 8 schema +struct +-- !query 8 output +ab abcd ab NULL + + +-- !query 9 +select left(null, -2), left("abcd", -2), left("abcd", 0), left("abcd", 'a') +-- !query 9 schema +struct +-- !query 9 output +NULL NULL + + +-- !query 10 +select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null) +-- !query 10 schema +struct +-- !query 10 output +cd abcd cd NULL + + +-- !query 11 +select right(null, -2), right("abcd", -2), right("abcd", 0), right("abcd", 'a') +-- !query 11 schema +struct +-- !query 11 output +NULL NULL diff --git a/sql/core/src/test/resources/sql-tests/results/struct.sql.out b/sql/core/src/test/resources/sql-tests/results/struct.sql.out new file mode 100644 index 000000000000..1da33bc736f0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/struct.sql.out @@ -0,0 +1,90 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 9 + + +-- !query 0 +CREATE TEMPORARY VIEW tbl_x AS VALUES + (1, NAMED_STRUCT('C', 'gamma', 'D', 'delta')), + (2, NAMED_STRUCT('C', 'epsilon', 'D', 'eta')), + (3, NAMED_STRUCT('C', 'theta', 'D', 'iota')) + AS T(ID, ST) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT STRUCT('alpha', 'beta') ST +-- !query 1 schema +struct> +-- !query 1 output +{"col1":"alpha","col2":"beta"} + + +-- !query 2 +SELECT STRUCT('alpha' AS A, 'beta' AS B) ST +-- !query 2 schema +struct> +-- !query 2 output +{"A":"alpha","B":"beta"} + + +-- !query 3 +SELECT ID, STRUCT(ST.*) NST FROM tbl_x +-- !query 3 schema +struct> +-- !query 3 output +1 {"C":"gamma","D":"delta"} +2 {"C":"epsilon","D":"eta"} +3 {"C":"theta","D":"iota"} + + +-- !query 4 +SELECT ID, STRUCT(ST.*,CAST(ID AS STRING) AS E) NST FROM tbl_x +-- !query 4 schema +struct> +-- !query 4 output +1 {"C":"gamma","D":"delta","E":"1"} +2 {"C":"epsilon","D":"eta","E":"2"} +3 {"C":"theta","D":"iota","E":"3"} + + +-- !query 5 +SELECT ID, STRUCT(CAST(ID AS STRING) AS AA, ST.*) NST FROM tbl_x +-- !query 5 schema +struct> +-- !query 5 output +1 {"AA":"1","C":"gamma","D":"delta"} +2 {"AA":"2","C":"epsilon","D":"eta"} +3 {"AA":"3","C":"theta","D":"iota"} + + +-- !query 6 +SELECT ID, STRUCT(ST.*).C NST FROM tbl_x +-- !query 6 schema +struct +-- !query 6 output +1 gamma +2 epsilon +3 theta + + +-- !query 7 +SELECT ID, STRUCT(ST.C, ST.D).D NST FROM tbl_x +-- !query 7 schema +struct +-- !query 7 output +1 delta +2 eta +3 iota + + +-- !query 8 +SELECT ID, STRUCT(ST.C as STC, ST.D as STD).STD FROM tbl_x +-- !query 8 schema +struct +-- !query 8 output +1 delta +2 eta +3 iota diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-aggregate.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-aggregate.sql.out new file mode 100644 index 000000000000..97f494cc0506 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-aggregate.sql.out @@ -0,0 +1,183 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 11 + + +-- !query 0 +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT emp.dept_id, + avg(salary), + sum(salary) +FROM emp +WHERE EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id) +GROUP BY dept_id +-- !query 3 schema +struct +-- !query 3 output +10 133.33333333333334 400.0 +20 300.0 300.0 +30 400.0 400.0 +70 150.0 150.0 + + +-- !query 4 +SELECT emp_name +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id) +-- !query 4 schema +struct +-- !query 4 output +emp 1 +emp 1 +emp 2 +emp 3 +emp 4 +emp 8 + + +-- !query 5 +SELECT count(*) +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id) +-- !query 5 schema +struct +-- !query 5 output +6 + + +-- !query 6 +SELECT * +FROM bonus +WHERE EXISTS (SELECT 1 + FROM emp + WHERE emp.emp_name = bonus.emp_name + AND EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE emp.dept_id = dept.dept_id + GROUP BY dept.dept_id)) +-- !query 6 schema +struct +-- !query 6 output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 + + +-- !query 7 +SELECT emp.dept_id, + Avg(salary), + Sum(salary) +FROM emp +WHERE NOT EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id) +GROUP BY dept_id +-- !query 7 schema +struct +-- !query 7 output +100 400.0 800.0 +NULL 400.0 400.0 + + +-- !query 8 +SELECT emp_name +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id) +-- !query 8 schema +struct +-- !query 8 output +emp 5 +emp 6 - no dept +emp 7 + + +-- !query 9 +SELECT count(*) +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id) +-- !query 9 schema +struct +-- !query 9 output +3 + + +-- !query 10 +SELECT * +FROM bonus +WHERE NOT EXISTS (SELECT 1 + FROM emp + WHERE emp.emp_name = bonus.emp_name + AND EXISTS (SELECT Max(dept.dept_id) + FROM dept + WHERE emp.dept_id = dept.dept_id + GROUP BY dept.dept_id)) +-- !query 10 schema +struct +-- !query 10 output +emp 5 1000.0 +emp 6 - no dept 500.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-basic.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-basic.sql.out new file mode 100644 index 000000000000..900e4d573bef --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-basic.sql.out @@ -0,0 +1,214 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 13 + + +-- !query 0 +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT * +FROM emp +WHERE EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30) +-- !query 3 schema +struct +-- !query 3 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 4 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) +-- !query 4 schema +struct +-- !query 4 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 5 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id + OR emp.dept_id IS NULL) +-- !query 5 schema +struct +-- !query 5 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +800 emp 8 2016-01-01 150.0 70 + + +-- !query 6 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) + AND emp.id > 200 +-- !query 6 schema +struct +-- !query 6 output +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 7 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id) + AND emp.id > 200 +-- !query 7 schema +struct +-- !query 7 output +emp 3 +emp 4 +emp 8 + + +-- !query 8 +SELECT * +FROM dept +WHERE NOT EXISTS (SELECT emp_name + FROM emp + WHERE emp.dept_id = dept.dept_id) +-- !query 8 schema +struct +-- !query 8 output +40 dept 4 - unassigned OR +50 dept 5 - unassigned NJ + + +-- !query 9 +SELECT * +FROM dept +WHERE NOT EXISTS (SELECT emp_name + FROM emp + WHERE emp.dept_id = dept.dept_id + OR state = 'NJ') +-- !query 9 schema +struct +-- !query 9 output +40 dept 4 - unassigned OR + + +-- !query 10 +SELECT * +FROM bonus +WHERE NOT EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) +-- !query 10 schema +struct +-- !query 10 output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 4 100.0 + + +-- !query 11 +SELECT emp.* +FROM emp +WHERE NOT EXISTS (SELECT NULL + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query 11 schema +struct +-- !query 11 output +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 12 +SELECT * +FROM bonus +WHERE EXISTS (SELECT emp_name + FROM emp + WHERE bonus.emp_name = emp.emp_name + AND EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id)) +-- !query 12 schema +struct +-- !query 12 output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-cte.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-cte.sql.out new file mode 100644 index 000000000000..c6c1c04e1c73 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-cte.sql.out @@ -0,0 +1,200 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +WITH bonus_cte + AS (SELECT * + FROM bonus + WHERE EXISTS (SELECT dept.dept_id, + emp.emp_name, + Max(salary), + Count(*) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY dept.dept_id, + emp.emp_name + ORDER BY emp.emp_name)) +SELECT * +FROM bonus a +WHERE a.bonus_amt > 30 + AND EXISTS (SELECT 1 + FROM bonus_cte b + WHERE a.emp_name = b.emp_name) +-- !query 3 schema +struct +-- !query 3 output +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 + + +-- !query 4 +WITH emp_cte + AS (SELECT * + FROM emp + WHERE id >= 100 + AND id <= 300), + dept_cte + AS (SELECT * + FROM dept + WHERE dept_id = 10) +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp_cte a + JOIN dept_cte b + ON a.dept_id = b.dept_id + WHERE bonus.emp_name = a.emp_name) +-- !query 4 schema +struct +-- !query 4 output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 + + +-- !query 5 +WITH emp_cte + AS (SELECT * + FROM emp + WHERE id >= 100 + AND id <= 300), + dept_cte + AS (SELECT * + FROM dept + WHERE dept_id = 10) +SELECT DISTINCT b.emp_name, + b.bonus_amt +FROM bonus b, + emp_cte e, + dept d +WHERE e.dept_id = d.dept_id + AND e.emp_name = b.emp_name + AND EXISTS (SELECT * + FROM emp_cte a + LEFT JOIN dept_cte b + ON a.dept_id = b.dept_id + WHERE e.emp_name = a.emp_name) +-- !query 5 schema +struct +-- !query 5 output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 + + +-- !query 6 +WITH empdept + AS (SELECT id, + salary, + emp_name, + dept.dept_id + FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id + WHERE emp.id IN ( 100, 200 )) +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE EXISTS (SELECT dept_id, + max(salary) + FROM empdept + GROUP BY dept_id + HAVING count(*) > 1) +GROUP BY emp_name +-- !query 6 schema +struct +-- !query 6 output +emp 1 30.0 +emp 2 400.0 +emp 3 300.0 +emp 4 100.0 +emp 5 1000.0 +emp 6 - no dept 500.0 + + +-- !query 7 +WITH empdept + AS (SELECT id, + salary, + emp_name, + dept.dept_id + FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id + WHERE emp.id IN ( 100, 200 )) +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE NOT EXISTS (SELECT dept_id, + Max(salary) + FROM empdept + GROUP BY dept_id + HAVING count(*) < 1) +GROUP BY emp_name +-- !query 7 schema +struct +-- !query 7 output +emp 1 30.0 +emp 2 400.0 +emp 3 300.0 +emp 4 100.0 +emp 5 1000.0 +emp 6 - no dept 500.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-having.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-having.sql.out new file mode 100644 index 000000000000..de90f5e260e1 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-having.sql.out @@ -0,0 +1,153 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT dept_id, count(*) +FROM emp +GROUP BY dept_id +HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < min(emp.salary)) +-- !query 3 schema +struct +-- !query 3 output +10 3 +100 2 +20 1 +30 1 +70 1 +NULL 1 + + +-- !query 4 +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < Min(emp.salary))) +-- !query 4 schema +struct +-- !query 4 output +10 dept 1 CA +20 dept 2 NY +30 dept 3 TX +40 dept 4 - unassigned OR +50 dept 5 - unassigned NJ +70 dept 7 FL + + +-- !query 5 +SELECT dept_id, + Max(salary) +FROM emp gp +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp p + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < Min(p.salary))) +GROUP BY gp.dept_id +-- !query 5 schema +struct +-- !query 5 output +10 200.0 +100 400.0 +20 300.0 +30 400.0 +70 150.0 +NULL 400.0 + + +-- !query 6 +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt > Min(emp.salary))) +-- !query 6 schema +struct +-- !query 6 output +10 dept 1 CA +20 dept 2 NY +30 dept 3 TX +40 dept 4 - unassigned OR +50 dept 5 - unassigned NJ +70 dept 7 FL + + +-- !query 7 +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + count(emp.dept_id) + FROM emp + WHERE dept.dept_id = dept_id + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE ( bonus_amt > min(emp.salary) + AND count(emp.dept_id) > 1 ))) +-- !query 7 schema +struct +-- !query 7 output +10 dept 1 CA diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out new file mode 100644 index 000000000000..c488cba01d4d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out @@ -0,0 +1,363 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 17 + + +-- !query 0 +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT * +FROM emp, + dept +WHERE emp.dept_id = dept.dept_id + AND EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query 3 schema +struct +-- !query 3 output +100 emp 1 2005-01-01 100.0 10 10 dept 1 CA +100 emp 1 2005-01-01 100.0 10 10 dept 1 CA +200 emp 2 2003-01-01 200.0 10 10 dept 1 CA +300 emp 3 2002-01-01 300.0 20 20 dept 2 NY +400 emp 4 2005-01-01 400.0 30 30 dept 3 TX + + +-- !query 4 +SELECT * +FROM emp + JOIN dept + ON emp.dept_id = dept.dept_id +WHERE EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query 4 schema +struct +-- !query 4 output +100 emp 1 2005-01-01 100.0 10 10 dept 1 CA +100 emp 1 2005-01-01 100.0 10 10 dept 1 CA +200 emp 2 2003-01-01 200.0 10 10 dept 1 CA +300 emp 3 2002-01-01 300.0 20 20 dept 2 NY +400 emp 4 2005-01-01 400.0 30 30 dept 3 TX + + +-- !query 5 +SELECT * +FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id +WHERE EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query 5 schema +struct +-- !query 5 output +100 emp 1 2005-01-01 100.0 10 10 dept 1 CA +100 emp 1 2005-01-01 100.0 10 10 dept 1 CA +200 emp 2 2003-01-01 200.0 10 10 dept 1 CA +300 emp 3 2002-01-01 300.0 20 20 dept 2 NY +400 emp 4 2005-01-01 400.0 30 30 dept 3 TX +500 emp 5 2001-01-01 400.0 NULL NULL NULL NULL +600 emp 6 - no dept 2001-01-01 400.0 100 NULL NULL NULL + + +-- !query 6 +SELECT * +FROM emp, + dept +WHERE emp.dept_id = dept.dept_id + AND NOT EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query 6 schema +struct +-- !query 6 output +800 emp 8 2016-01-01 150.0 70 70 dept 7 FL + + +-- !query 7 +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name) +-- !query 7 schema +struct +-- !query 7 output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 + + +-- !query 8 +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp + RIGHT JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name) +-- !query 8 schema +struct +-- !query 8 output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 + + +-- !query 9 +SELECT * +FROM bonus +WHERE EXISTS (SELECT dept.dept_id, + emp.emp_name, + Max(salary), + Count(*) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY dept.dept_id, + emp.emp_name + ORDER BY emp.emp_name) +-- !query 9 schema +struct +-- !query 9 output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 + + +-- !query 10 +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE EXISTS (SELECT emp_name, + Max(salary) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY emp_name + HAVING Count(*) > 1 + ORDER BY emp_name) +GROUP BY emp_name +-- !query 10 schema +struct +-- !query 10 output +emp 1 30.0 + + +-- !query 11 +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE NOT EXISTS (SELECT emp_name, + Max(salary) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY emp_name + HAVING Count(*) > 1 + ORDER BY emp_name) +GROUP BY emp_name +-- !query 11 schema +struct +-- !query 11 output +emp 2 400.0 +emp 3 300.0 +emp 4 100.0 +emp 5 1000.0 +emp 6 - no dept 500.0 + + +-- !query 12 +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + UNION + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query 12 schema +struct +-- !query 12 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 13 +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query 13 schema +struct +-- !query 13 output + + + +-- !query 14 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query 14 schema +struct +-- !query 14 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 15 +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + EXCEPT + SELECT * + FROM dept + WHERE dept_id > 50) +UNION ALL +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query 15 schema +struct +-- !query 15 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 16 +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + EXCEPT + SELECT * + FROM dept + WHERE dept_id > 50) +UNION +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query 16 schema +struct +-- !query 16 output +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out new file mode 100644 index 000000000000..ee13ff2c4f38 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out @@ -0,0 +1,222 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 12 + + +-- !query 0 +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate +-- !query 3 schema +struct +-- !query 3 output +300 emp 3 2002-01-01 300.0 20 +200 emp 2 2003-01-01 200.0 10 +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +400 emp 4 2005-01-01 400.0 30 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 4 +SELECT id, + hiredate +FROM emp +WHERE EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate DESC +-- !query 4 schema +struct +-- !query 4 output +800 2016-01-01 +100 2005-01-01 +100 2005-01-01 +400 2005-01-01 +200 2003-01-01 +300 2002-01-01 + + +-- !query 5 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate +-- !query 5 schema +struct +-- !query 5 output +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 + + +-- !query 6 +SELECT emp_name +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state) +-- !query 6 schema +struct +-- !query 6 output +emp 5 +emp 6 - no dept +emp 7 + + +-- !query 7 +SELECT count(*) +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept_id + ORDER BY dept_id) +-- !query 7 schema +struct +-- !query 7 output +3 + + +-- !query 8 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + LIMIT 1) +-- !query 8 schema +struct +-- !query 8 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 9 +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + LIMIT 1) +-- !query 9 schema +struct +-- !query 9 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 10 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + LIMIT 1) +-- !query 10 schema +struct +-- !query 10 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query 11 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + LIMIT 1) +-- !query 11 schema +struct +-- !query 11 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-within-and-or.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-within-and-or.sql.out new file mode 100644 index 000000000000..865e4ed14e4a --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/exists-subquery/exists-within-and-or.sql.out @@ -0,0 +1,156 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR emp.id > 200 +-- !query 3 schema +struct +-- !query 3 output +emp 1 +emp 1 +emp 2 +emp 3 +emp 4 +emp 5 +emp 6 - no dept +emp 7 +emp 8 + + +-- !query 4 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR emp.dept_id IS NULL +-- !query 4 schema +struct +-- !query 4 output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +800 emp 8 2016-01-01 150.0 70 + + +-- !query 5 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id + AND dept.dept_id = 20) + OR EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id + AND dept.dept_id = 30) +-- !query 5 schema +struct +-- !query 5 output +emp 3 +emp 4 + + +-- !query 6 +SELECT * +FROM bonus +WHERE ( NOT EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) + OR EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + OR bonus_amt < emp.salary) ) +-- !query 6 schema +struct +-- !query 6 output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 +emp 5 1000.0 +emp 6 - no dept 500.0 + + +-- !query 7 +SELECT * FROM bonus WHERE NOT EXISTS +( + SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) +AND +emp_name IN +( + SELECT emp_name + FROM emp + WHERE bonus_amt < emp.salary) +-- !query 7 schema +struct +-- !query 7 output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 4 100.0 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out new file mode 100644 index 000000000000..a159aa81eff1 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-group-by.sql.out @@ -0,0 +1,357 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 19 + + +-- !query 0 +create temporary view t1 as select * from values + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view t3 as select * from values + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT t1a, + Avg(t1b) +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2) +GROUP BY t1a +-- !query 3 schema +struct +-- !query 3 output +t1b 8.0 +t1c 8.0 +t1e 10.0 + + +-- !query 4 +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1d +-- !query 4 schema +struct +-- !query 4 output +t1b 8 + + +-- !query 5 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1b +-- !query 5 schema +struct +-- !query 5 output +t1b 8 +t1c 8 + + +-- !query 6 +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) + OR t1c IN (SELECT t3c + FROM t3 + WHERE t1a = t3a) +GROUP BY t1a, + t1c +-- !query 6 schema +struct +-- !query 6 output +t1b 8 +t1c 8 + + +-- !query 7 +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) + AND t1c IN (SELECT t3c + FROM t3 + WHERE t1a = t3a) +GROUP BY t1a, + t1c +-- !query 7 schema +struct +-- !query 7 output +t1b 8 + + +-- !query 8 +SELECT t1a, + Count(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1c +HAVING t1a = "t1b" +-- !query 8 schema +struct +-- !query 8 output +t1b 1 + + +-- !query 9 +SELECT * +FROM t1 +WHERE t1b IN (SELECT Max(t2b) + FROM t2 + GROUP BY t2a) +-- !query 9 schema +struct +-- !query 9 output +t1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +t1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +t1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +t1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +t1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +t1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +t1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +t1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query 10 +SELECT * +FROM (SELECT t2a, + t2b + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) + GROUP BY t2a, + t2b) t2 +-- !query 10 schema +struct +-- !query 10 output +t1b 8 + + +-- !query 11 +SELECT Count(DISTINCT( * )) +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + AND t1c = t2c + GROUP BY t2a) +-- !query 11 schema +struct +-- !query 11 output +1 + + +-- !query 12 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT Max(t2c) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a, + t2c + HAVING t2c > 8) +-- !query 12 schema +struct +-- !query 12 output +t1b 8 +t1c 8 + + +-- !query 13 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2a IN (SELECT Min(t3a) + FROM t3 + WHERE t3a = t2a + GROUP BY t3b) + GROUP BY t2c) +-- !query 13 schema +struct +-- !query 13 output +t1a 16 +t1a 16 +t1b 8 +t1c 8 +t1d NULL +t1d NULL + + +-- !query 14 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) +GROUP BY t1a +-- !query 14 schema +struct +-- !query 14 output +t1b 8 +t1c 8 + + +-- !query 15 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b IN (SELECT Min(t3b) + FROM t3 + WHERE t2a = t3a + GROUP BY t3a) + GROUP BY t2c) +GROUP BY t1a, + t1d +-- !query 15 schema +struct +-- !query 15 output +t1b 8 +t1c 8 +t1d NULL +t1d NULL + + +-- !query 16 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) + AND t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d) +GROUP BY t1a +-- !query 16 schema +struct +-- !query 16 output +t1b 8 +t1c 8 + + +-- !query 17 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) + OR t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d) +GROUP BY t1a +-- !query 17 schema +struct +-- !query 17 output +t1a 16 +t1b 8 +t1c 8 +t1d NULL + + +-- !query 18 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a + HAVING t2a > t1a) + OR t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d + HAVING t3d = t1d) +GROUP BY t1a +HAVING Min(t1b) IS NOT NULL +-- !query 18 schema +struct +-- !query 18 output +t1a 16 +t1b 8 +t1c 8 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-having.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-having.sql.out new file mode 100644 index 000000000000..b90ebf57e739 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-having.sql.out @@ -0,0 +1,217 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 12 + + +-- !query 0 +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + GROUP BY t2b + HAVING t2b < 10) +-- !query 3 schema +struct +-- !query 3 output +val1a 6 2014-04-04 01:00:00 +val1a 6 2014-04-04 01:02:00.001 +val1b 8 2014-05-04 01:01:00 +val1c 8 2014-05-04 01:02:00.001 + + +-- !query 4 +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2b + HAVING t2b > 1) +-- !query 4 schema +struct +-- !query 4 output +val1b 8 16 + + +-- !query 5 +SELECT t1a, t1b, t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c < t2c) +GROUP BY t1a, t1b, t1c +HAVING t1b < 10 +-- !query 5 schema +struct +-- !query 5 output +val1a 6 8 + + +-- !query 6 +SELECT t1a, t1b, t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c = t2c) +GROUP BY t1a, t1b, t1c +HAVING COUNT (DISTINCT t1b) < 10 +-- !query 6 schema +struct +-- !query 6 output +val1b 8 16 +val1c 8 16 + + +-- !query 7 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a + GROUP BY t2c + HAVING t2c > 10) +GROUP BY t1b +HAVING t1b >= 8 +-- !query 7 schema +struct +-- !query 7 output +2 8 + + +-- !query 8 +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b > 0 +GROUP BY t1a +HAVING t1a IN (SELECT t2a + FROM t2 + WHERE t2b IN (SELECT t3b + FROM t3 + WHERE t2c = t3c) + ) +-- !query 8 schema +struct +-- !query 8 output +val1b 8 + + +-- !query 9 +SELECT t1a, + t1c, + Min(t1d) +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + GROUP BY t2a + HAVING t2a > 'val2a') +GROUP BY t1a, t1c +HAVING Min(t1d) > t1c +-- !query 9 schema +struct +-- !query 9 output +val1a 8 10 +val1b 16 19 +val1c 16 19 +val1d 16 19 + + +-- !query 10 +SELECT t1a, + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1a = t2a + GROUP BY t2c, t2d + HAVING t2c > 8) +GROUP BY t1a, t1b +HAVING t1b < 10 +-- !query 10 schema +struct +-- !query 10 output +val1a 6 + + +-- !query 11 +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b > 0 +GROUP BY t1a +HAVING t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b > 3) +-- !query 11 schema +struct +-- !query 11 output +val1a 16 +val1d 10 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out new file mode 100644 index 000000000000..ab6a11a2b7ef --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-joins.sql.out @@ -0,0 +1,353 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 14 + + +-- !query 0 +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT t1a, t1b, t1c, t3a, t3b, t3c +FROM t1 natural JOIN t3 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1a = t2a) + AND t1b = t3b + AND t1a = t3a +ORDER BY t1a, + t1b, + t1c DESC nulls first +-- !query 3 schema +struct +-- !query 3 output +val1b 8 16 val1b 8 16 +val1b 8 16 val1b 8 16 + + +-- !query 4 +SELECT Count(DISTINCT(t1a)), + t1b, + t3a, + t3b, + t3c +FROM t1 natural left JOIN t3 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1d = t2d) +AND t1b > t3b +GROUP BY t1a, + t1b, + t3a, + t3b, + t3c +ORDER BY t1a DESC, t3b DESC +-- !query 4 schema +struct +-- !query 4 output +1 10 val3b 8 NULL +1 10 val1b 8 16 +1 10 val3a 6 12 +1 8 val3a 6 12 +1 8 val3a 6 12 + + +-- !query 5 +SELECT Count(DISTINCT(t1a)) +FROM t1 natural right JOIN t3 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1b = t2b) +AND t1d IN + ( + SELECT t2d + FROM t2 + WHERE t1c > t2c) +AND t1a = t3a +GROUP BY t1a +ORDER BY t1a +-- !query 5 schema +struct +-- !query 5 output +1 + + +-- !query 6 +SELECT t1a, + t1b, + t1c, + t3a, + t3b, + t3c +FROM t1 FULL OUTER JOIN t3 +where t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2c IS NOT NULL) +AND t1b != t3b +AND t1a = 'val1b' +ORDER BY t1a +-- !query 6 schema +struct +-- !query 6 output +val1b 8 16 val3a 6 12 +val1b 8 16 val3a 6 12 +val1b 8 16 val1b 10 12 +val1b 8 16 val1b 10 12 +val1b 8 16 val3c 17 16 +val1b 8 16 val3c 17 16 + + +-- !query 7 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 RIGHT JOIN t3 +where t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2h > t3h) +AND t3a IN + ( + SELECT t2a + FROM t2 + WHERE t2c > t3c) +AND t1h >= t3h +GROUP BY t1a, + t1b +HAVING t1b > 8 +ORDER BY t1a +-- !query 7 schema +struct +-- !query 7 output +1 10 + + +-- !query 8 +SELECT Count(DISTINCT(t1a)) +FROM t1 LEFT OUTER +JOIN t3 +ON t1a = t3a +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1h < t2h ) +GROUP BY t1a +ORDER BY t1a +-- !query 8 schema +struct +-- !query 8 output +1 +1 +1 + + +-- !query 9 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 INNER JOIN t2 +ON t1a > t2a +WHERE t1b IN + ( + SELECT t2b + FROM t2 + WHERE t2h > t1h) +OR t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2h < t1h) +GROUP BY t1b +HAVING t1b > 6 +-- !query 9 schema +struct +-- !query 9 output +1 10 +1 8 + + +-- !query 10 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +AND t1h IN + ( + SELECT t2h + FROM t2 + RIGHT JOIN t3 + where t2b = t3b) +GROUP BY t1b +HAVING t1b > 8 +-- !query 10 schema +struct +-- !query 10 output +1 10 + + +-- !query 11 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +AND t1h IN + ( + SELECT t2h + FROM t2 + RIGHT JOIN t3 + where t2b = t3b) +AND t1b IN + ( + SELECT t2b + FROM t2 + FULL OUTER JOIN t3 + where t2b = t3b) + +GROUP BY t1b +HAVING t1b > 8 +-- !query 11 schema +struct +-- !query 11 output +1 10 + + +-- !query 12 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +INNER JOIN t2 on t1b = t2b +RIGHT JOIN t3 ON t1a = t3a +where t1a IN + ( + SELECT t2a + FROM t2 + FULL OUTER JOIN t3 + WHERE t2b > t3b) +AND t1c IN + ( + SELECT t3c + FROM t3 + LEFT OUTER JOIN t2 + ON t3a = t2a ) +AND t1b IN + ( + SELECT t3b + FROM t3 LEFT OUTER + JOIN t1 + WHERE t3c = t1c) + +AND t1a = t2a +GROUP BY t1b +ORDER BY t1b DESC +-- !query 12 schema +struct +-- !query 12 output +1 8 + + +-- !query 13 +SELECT t1a, + t1b, + t1c, + count(distinct(t2a)), + t2b, + t2c +FROM t1 +FULL JOIN t2 on t1a = t2a +RIGHT JOIN t3 on t1a = t3a +where t1a IN + ( + SELECT t2a + FROM t2 INNER + JOIN t3 + ON t2b < t3b + WHERE t2c IN + ( + SELECT t1c + FROM t1 + WHERE t1a = t2a)) +and t1a = t2a +Group By t1a, t1b, t1c, t2a, t2b, t2c +HAVING t2c IS NOT NULL +ORDER By t2b DESC nulls last +-- !query 13 schema +struct +-- !query 13 output +val1b 8 16 1 10 12 +val1b 8 16 1 8 16 +val1b 8 16 1 NULL 16 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out new file mode 100644 index 000000000000..71ca1f864947 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out @@ -0,0 +1,147 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +LIMIT 2 +-- !query 3 schema +struct +-- !query 3 output +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 + + +-- !query 4 +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + LIMIT 2) +LIMIT 4 +-- !query 4 schema +struct +-- !query 4 output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 + + +-- !query 5 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c + LIMIT 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +LIMIT 1 +-- !query 5 schema +struct +-- !query 5 output +1 NULL + + +-- !query 6 +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + LIMIT 2) +-- !query 6 schema +struct +-- !query 6 output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 + + +-- !query 7 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first + LIMIT 1) +GROUP BY t1b +ORDER BY t1b NULLS last +LIMIT 1 +-- !query 7 schema +struct +-- !query 7 output +1 6 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-multiple-columns.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-multiple-columns.sql.out new file mode 100644 index 000000000000..7a96c4bc5a30 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-multiple-columns.sql.out @@ -0,0 +1,178 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE ( t1a, t1h ) NOT IN (SELECT t2a, + t2h + FROM t2 + WHERE t2a = t1a + ORDER BY t2a) +AND t1a = 'val1a' +-- !query 3 schema +struct +-- !query 3 output +val1a 16 2014-06-04 01:02:00.001 +val1a 16 2014-07-04 01:01:00 +val1a 6 2014-04-04 01:00:00 +val1a 6 2014-04-04 01:02:00.001 + + +-- !query 4 +SELECT t1a, + t1b, + t1d +FROM t1 +WHERE ( t1b, t1d ) IN (SELECT t2b, + t2d + FROM t2 + WHERE t2i IN (SELECT t3i + FROM t3 + WHERE t2b > t3b)) +-- !query 4 schema +struct +-- !query 4 output +val1e 10 19 +val1e 10 19 + + +-- !query 5 +SELECT t1a, + t1b, + t1d +FROM t1 +WHERE ( t1b, t1d ) NOT IN (SELECT t2b, + t2d + FROM t2 + WHERE t2h IN (SELECT t3h + FROM t3 + WHERE t2b > t3b)) +AND t1a = 'val1a' +-- !query 5 schema +struct +-- !query 5 output +val1a 16 10 +val1a 16 21 +val1a 6 10 +val1a 6 10 + + +-- !query 6 +SELECT t2a +FROM (SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t1a, + t1b + FROM t1) + UNION ALL + SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t1a, + t1b + FROM t1) + UNION DISTINCT + SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t3a, + t3b + FROM t3)) AS t4 +-- !query 6 schema +struct +-- !query 6 output +val1b + + +-- !query 7 +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1 + WHERE ( + t1b, t1d) IN + ( + SELECT t2b, + t2d + FROM t2 + WHERE t1c = t2c)) +SELECT * +FROM ( + SELECT * + FROM cte1 + JOIN cte1 cte2 + on cte1.t1b = cte2.t1b) s +-- !query 7 schema +struct +-- !query 7 output +val1b 8 val1b 8 +val1b 8 val1c 8 +val1c 8 val1b 8 +val1c 8 val1c 8 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out new file mode 100644 index 000000000000..4bebd9622c3c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-order-by.sql.out @@ -0,0 +1,328 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 18 + + +-- !query 0 +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2) +ORDER BY t1a +-- !query 3 schema +struct +-- !query 3 output +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 + + +-- !query 4 +SELECT t1a +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +ORDER BY t1b DESC +-- !query 4 schema +struct +-- !query 4 output +val1b + + +-- !query 5 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +ORDER BY 2 DESC nulls last +-- !query 5 schema +struct +-- !query 5 output +val1b 8 +val1c 8 + + +-- !query 6 +SELECT Count(DISTINCT( t1a )) +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +ORDER BY Count(DISTINCT( t1a )) +-- !query 6 schema +struct +-- !query 6 output +1 + + +-- !query 7 +SELECT * +FROM t1 +WHERE t1b IN (SELECT t2c + FROM t2 + ORDER BY t2d) +-- !query 7 schema +struct +-- !query 7 output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 + + +-- !query 8 +SELECT * +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1b = t2b + ORDER BY Min(t2b)) +ORDER BY t1c DESC nulls first +-- !query 8 schema +struct +-- !query 8 output +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 + + +-- !query 9 +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a + ORDER BY t2b DESC nulls first) + OR t1h IN (SELECT t2h + FROM t2 + WHERE t1h > t2h) +ORDER BY t1h DESC nulls last +-- !query 9 schema +struct +-- !query 9 output +val1c 8 2014-05-04 01:02:00.001 +val1b 8 2014-05-04 01:01:00 + + +-- !query 10 +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2) +ORDER BY t1a +-- !query 10 schema +struct +-- !query 10 output +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 + + +-- !query 11 +SELECT t1a, + t1b +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t1a = t2a) +ORDER BY t1b DESC nulls last +-- !query 11 schema +struct +-- !query 11 output +val1a 16 +val1a 16 +val1d 10 +val1a 6 +val1a 6 +val1d NULL +val1d NULL + + +-- !query 12 +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + ORDER BY t2a DESC nulls first) + and t1c IN (SELECT t2c + FROM t2 + ORDER BY t2b DESC nulls last) +ORDER BY t1c DESC nulls last +-- !query 12 schema +struct +-- !query 12 output +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 + + +-- !query 13 +SELECT * +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + GROUP BY t2a + ORDER BY t2a DESC) +-- !query 13 schema +struct +-- !query 13 output +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 + + +-- !query 14 +SELECT t1a, + Count(DISTINCT( t1b )) +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a + ORDER BY t2a) +GROUP BY t1a, + t1h +ORDER BY t1a +-- !query 14 schema +struct +-- !query 14 output +val1b 1 + + +-- !query 15 +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT Min(t2b) + FROM t2 + GROUP BY t2a + ORDER BY t2a) +-- !query 15 schema +struct +-- !query 15 output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query 16 +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1b NOT IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2c + ORDER BY t2c DESC nulls last) +GROUP BY t1a +-- !query 16 schema +struct +-- !query 16 output +val1a 22 +val1c 8 +val1d 10 +val1e 10 + + +-- !query 17 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1h NOT IN (SELECT t2h + FROM t2 + where t1a = t2a + order by t2d DESC nulls first + ) +GROUP BY t1a, + t1b +ORDER BY t1b DESC nulls last +-- !query 17 schema +struct +-- !query 17 output +1 16 +1 10 +1 10 +1 8 +1 6 +1 NULL diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out new file mode 100644 index 000000000000..e06f9206d340 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out @@ -0,0 +1,595 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 16 + + +-- !query 0 +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT t2a, + t2b, + t2c, + t2h, + t2i +FROM (SELECT * + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1) + UNION ALL + SELECT * + FROM t3 + WHERE t3a IN (SELECT t1a + FROM t1)) AS t3 +WHERE t2i IS NOT NULL AND + 2 * t2b = t2c +ORDER BY t2c DESC nulls first +-- !query 3 schema +struct +-- !query 3 output +val1b 8 16 2015-05-04 01:01:00 2015-05-04 +val1b 8 16 2014-07-04 01:01:00 2014-07-04 +val1b 8 16 2014-06-04 01:02:00 2014-06-04 +val1b 8 16 2014-07-04 01:02:00 2014-07-04 + + +-- !query 4 +SELECT t2a, + t2b, + t2d, + Count(DISTINCT( t2h )), + t2i +FROM (SELECT * + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2b = t1b) + UNION + SELECT * + FROM t1 + WHERE t1a IN (SELECT t3a + FROM t3 + WHERE t1c = t3c)) AS t3 +GROUP BY t2a, + t2b, + t2d, + t2i +ORDER BY t2d DESC +-- !query 4 schema +struct +-- !query 4 output +val1b 8 119 1 2015-05-04 +val1b 8 19 1 2014-07-04 +val1b 8 19 1 2014-05-04 + + +-- !query 5 +SELECT t2a, + t2b, + t2c, + Min(t2d) +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, t2b, t2c +UNION ALL +SELECT t2a, + t2b, + t2c, + Max(t2d) +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, t2b, t2c +UNION +SELECT t3a, + t3b, + t3c, + Min(t3d) +FROM t3 +WHERE t3a IN (SELECT t2a + FROM t2 + WHERE t3c = t2c) +GROUP BY t3a, t3b, t3c +UNION DISTINCT +SELECT t1a, + t1b, + t1c, + Max(t1d) +FROM t1 +WHERE t1a IN (SELECT t3a + FROM t3 + WHERE t3d = t1d) +GROUP BY t1a, t1b, t1c +-- !query 5 schema +struct +-- !query 5 output +val1b 10 12 19 +val1b 8 16 119 +val1b 8 16 19 +val1b NULL 16 19 +val1b NULL 16 319 +val1c 12 16 219 + + +-- !query 6 +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +UNION +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +HAVING t2b IS NOT NULL +-- !query 6 schema +struct +-- !query 6 output +val1b 8 1 119 2015-05-04 01:01:00 2015-05-04 +val1b 8 1 19 2014-07-04 01:01:00 2014-07-04 +val1c 12 1 19 2014-08-04 01:01:00 2014-08-05 +val1c 12 1 219 2016-05-04 01:01:00 2016-05-04 + + +-- !query 7 +SELECT t2a, + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT DISTINCT(t1a) + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i + +UNION +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2b IN (SELECT Max(t1b) + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +HAVING t2b IS NOT NULL +UNION DISTINCT +SELECT t2a, + t2b, + t2c, + t2d, + t2h, + t2i +FROM t2 +WHERE t2d IN (SELECT min(t1d) + FROM t1 + WHERE t2c = t1c) +-- !query 7 schema +struct +-- !query 7 output +val1b 8 1 119 2015-05-04 01:01:00 2015-05-04 +val1b 8 1 19 2014-07-04 01:01:00 2014-07-04 +val1b 8 16 19 2014-07-04 01:01:00 2014-07-04 +val1b NULL 16 19 2014-05-04 01:01:00 NULL +val1c 12 16 19 2014-08-04 01:01:00 2014-08-05 + + +-- !query 8 +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b AND + t1d < t2d) +INTERSECT +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2b IN (SELECT Max(t1b) + FROM t1 + WHERE t2c = t1c) +EXCEPT +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2d IN (SELECT Min(t3d) + FROM t3 + WHERE t2c = t3c) +UNION ALL +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2c IN (SELECT Max(t1c) + FROM t1 + WHERE t1d = t2d) +-- !query 8 schema +struct +-- !query 8 output +val1b 8 16 119 +val1b 8 16 19 +val1b NULL 16 19 +val1c 12 16 19 + + +-- !query 9 +SELECT DISTINCT(t1a), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a IN (SELECT t3a + FROM (SELECT t2a t3a + FROM t2 + UNION ALL + SELECT t2a t3a + FROM t2) AS t3 + UNION + SELECT t2a + FROM (SELECT t2a + FROM t2 + WHERE t2b > 6 + UNION + SELECT t2a + FROM t2 + WHERE t2b > 6) AS t4 + UNION DISTINCT + SELECT t2a + FROM (SELECT t2a + FROM t2 + WHERE t2b > 6 + UNION DISTINCT + SELECT t1a + FROM t1 + WHERE t1b > 6) AS t5) +GROUP BY t1a, t1b, t1c, t1d +HAVING t1c IS NOT NULL AND t1b IS NOT NULL +ORDER BY t1c DESC, t1a DESC +-- !query 9 schema +struct +-- !query 9 output +val1c 8 16 19 +val1b 8 16 19 +val1a 16 12 21 +val1a 16 12 10 +val1a 6 8 10 + + +-- !query 10 +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM (SELECT t2b + FROM t2 + WHERE t2b > 6 + INTERSECT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t3 + WHERE t2b = t1b) +-- !query 10 schema +struct +-- !query 10 output +val1b 8 16 +val1c 8 16 +val1d 10 NULL +val1e 10 NULL +val1e 10 NULL +val1e 10 NULL + + +-- !query 11 +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1h IN (SELECT t2h + FROM (SELECT t2h + FROM t2 + EXCEPT + SELECT t3h + FROM t3) AS t3) +ORDER BY t1b DESC NULLs first, t1c DESC NULLs last +-- !query 11 schema +struct +-- !query 11 output +val1d NULL 16 +val1a 16 12 +val1e 10 NULL +val1d 10 NULL +val1e 10 NULL +val1b 8 16 + + +-- !query 12 +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b > 6 + INTERSECT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t3) +UNION DISTINCT +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b > 6 + EXCEPT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t4 + WHERE t2b = t1b) +ORDER BY t1c DESC NULLS last, t1a DESC +-- !query 12 schema +struct +-- !query 12 output +val1c 8 16 +val1b 8 16 +val1e 10 NULL +val1d 10 NULL + + +-- !query 13 +SELECT * +FROM (SELECT * + FROM (SELECT * + FROM t2 + WHERE t2h IN (SELECT t1h + FROM t1 + WHERE t1a = t2a) + UNION DISTINCT + SELECT * + FROM t1 + WHERE t1h IN (SELECT t3h + FROM t3 + UNION + SELECT t1h + FROM t1) + UNION + SELECT * + FROM t3 + WHERE t3a IN (SELECT t2a + FROM t2 + UNION ALL + SELECT t1a + FROM t1 + WHERE t1b > 0) + INTERSECT + SELECT * + FROM T1 + WHERE t1b IN (SELECT t3b + FROM t3 + UNION DISTINCT + SELECT t2b + FROM t2 + ) + EXCEPT + SELECT * + FROM t2 + WHERE t2h IN (SELECT t1i + FROM t1)) t4 + WHERE t4.t2b IN (SELECT Min(t3b) + FROM t3 + WHERE t4.t2a = t3a)) +-- !query 13 schema +struct +-- !query 13 output +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 + + +-- !query 14 +SELECT t2a, + t2b, + t2c, + t2i +FROM (SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t1a + FROM t1 + UNION + SELECT t3a + FROM t3) + UNION ALL + SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t1a + FROM t1 + INTERSECT + SELECT t2a + FROM t2)) AS t3 +WHERE t3.t2a NOT IN (SELECT t1a + FROM t1 + INTERSECT + SELECT t2a + FROM t2) + AND t2c IS NOT NULL +ORDER BY t2a +-- !query 14 schema +struct +-- !query 14 output +val2a 6 12 2014-04-04 +val2a 6 12 2014-04-04 + + +-- !query 15 +SELECT Count(DISTINCT(t1a)), + t1b, + t1c, + t1i +FROM t1 +WHERE t1b NOT IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b NOT IN + ( + SELECT t1b + FROM t1) + UNION + SELECT t1b + FROM t1 + WHERE t1b NOT IN + ( + SELECT t3b + FROM t3) + UNION + distinct SELECT t3b + FROM t3 + WHERE t3b NOT IN + ( + SELECT t2b + FROM t2)) AS t3 + WHERE t2b = t1b) +GROUP BY t1a, + t1b, + t1c, + t1i +HAVING t1b NOT IN + ( + SELECT t2b + FROM t2 + WHERE t2c IS NULL + EXCEPT + SELECT t3b + FROM t3) +ORDER BY t1c DESC NULLS LAST, t1i +-- !query 15 schema +struct +-- !query 15 output +1 8 16 2014-05-04 +1 8 16 2014-05-05 +1 16 12 2014-06-04 +1 16 12 2014-07-04 +1 6 8 2014-04-04 +1 10 NULL 2014-05-04 +1 10 NULL 2014-08-04 +1 10 NULL 2014-09-04 +1 10 NULL 2015-05-04 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-with-cte.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-with-cte.sql.out new file mode 100644 index 000000000000..7d3943e3764c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-with-cte.sql.out @@ -0,0 +1,364 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 13 + + +-- !query 0 +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1a = "val1a") +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1b IN (SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 0) +-- !query 3 schema +struct +-- !query 3 output +val1a 16 12 10 2014-07-04 01:01:00 +val1a 16 12 21 2014-06-04 01:02:00.001 +val1a 6 8 10 2014-04-04 01:00:00 +val1a 6 8 10 2014-04-04 01:02:00.001 + + +-- !query 4 +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1) +SELECT count(distinct(t1a)), t1b, t1c +FROM t1 +WHERE t1b IN + ( + SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 0 + UNION + SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 5 + UNION ALL + SELECT cte1.t1b + FROM cte1 + INTERSECT + SELECT cte1.t1b + FROM cte1 + UNION + SELECT cte1.t1b + FROM cte1 ) +GROUP BY t1a, t1b, t1c +HAVING t1c IS NOT NULL +-- !query 4 schema +struct +-- !query 4 output +1 16 12 +1 6 8 +1 8 16 +1 8 16 + + +-- !query 5 +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c, + t1d, + t1e + FROM t1) +SELECT t1a, + t1b, + t1c, + t1h +FROM t1 +WHERE t1c IN + ( + SELECT cte1.t1c + FROM cte1 + JOIN cte1 cte2 + on cte1.t1b > cte2.t1b + FULL OUTER JOIN cte1 cte3 + ON cte1.t1c = cte3.t1c + LEFT JOIN cte1 cte4 + ON cte1.t1d = cte4.t1d + INNER JOIN cte1 cte5 + ON cte1.t1b < cte5.t1b + LEFT OUTER JOIN cte1 cte6 + ON cte1.t1d > cte6.t1d) +-- !query 5 schema +struct +-- !query 5 output +val1b 8 16 2014-05-04 01:01:00 +val1c 8 16 2014-05-04 01:02:00.001 +val1d NULL 16 2014-06-04 01:01:00 +val1d NULL 16 2014-07-04 01:02:00.001 + + +-- !query 6 +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1b IN (SELECT t2b + FROM t2 + RIGHT JOIN t1 + ON t1c = t2c + LEFT JOIN t3 + ON t2d = t3d) + AND t1a = "val1b") +SELECT * +FROM (SELECT * + FROM cte1 + JOIN cte1 cte2 + ON cte1.t1b > 5 + AND cte1.t1a = cte2.t1a + FULL OUTER JOIN cte1 cte3 + ON cte1.t1a = cte3.t1a + INNER JOIN cte1 cte4 + ON cte1.t1b = cte4.t1b) s +-- !query 6 schema +struct +-- !query 6 output +val1b 8 val1b 8 val1b 8 val1b 8 + + +-- !query 7 +WITH cte1 AS +( + SELECT t1a, + t1b, + t1h + FROM t1 + WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1b < t2b)) +SELECT Count(DISTINCT t1a), + t1b +FROM ( + SELECT cte1.t1a, + cte1.t1b + FROM cte1 + JOIN cte1 cte2 + on cte1.t1h >= cte2.t1h) s +WHERE t1b IN + ( + SELECT t1b + FROM t1) +GROUP BY t1b +-- !query 7 schema +struct +-- !query 7 output +2 8 + + +-- !query 8 +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c + FROM t1 + WHERE t1b IN + ( + SELECT t2b + FROM t2 FULL OUTER JOIN T3 on t2a = t3a + WHERE t1c = t2c) AND + t1a = "val1b") +SELECT * +FROM ( + SELECT * + FROM cte1 + INNER JOIN cte1 cte2 ON cte1.t1a = cte2.t1a + RIGHT OUTER JOIN cte1 cte3 ON cte1.t1b = cte3.t1b + LEFT OUTER JOIN cte1 cte4 ON cte1.t1c = cte4.t1c + ) s +-- !query 8 schema +struct +-- !query 8 output +val1b 8 16 val1b 8 16 val1b 8 16 val1b 8 16 + + +-- !query 9 +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c = t2c)) +SELECT Count(DISTINCT( s.t1a )), + s.t1b +FROM (SELECT cte1.t1a, + cte1.t1b + FROM cte1 + RIGHT OUTER JOIN cte1 cte2 + ON cte1.t1a = cte2.t1a) s +GROUP BY s.t1b +-- !query 9 schema +struct +-- !query 9 output +2 8 + + +-- !query 10 +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1 + WHERE t1b IN + ( + SELECT t2b + FROM t2 + WHERE t1c = t2c)) +SELECT DISTINCT(s.t1b) +FROM ( + SELECT cte1.t1b + FROM cte1 + LEFT OUTER JOIN cte1 cte2 + ON cte1.t1b = cte2.t1b) s +WHERE s.t1b IN + ( + SELECT t1.t1b + FROM t1 INNER + JOIN cte1 + ON t1.t1a = cte1.t1a) +-- !query 10 schema +struct +-- !query 10 output +8 + + +-- !query 11 +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1a = "val1d") +SELECT t1a, + t1b, + t1c, + t1h +FROM t1 +WHERE t1b NOT IN (SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b < 0) AND + t1c > 10 +-- !query 11 schema +struct +-- !query 11 output +val1a 16 12 2014-06-04 01:02:00.001 +val1a 16 12 2014-07-04 01:01:00 +val1b 8 16 2014-05-04 01:01:00 +val1c 8 16 2014-05-04 01:02:00.001 +val1d NULL 16 2014-06-04 01:01:00 +val1d NULL 16 2014-07-04 01:02:00.001 + + +-- !query 12 +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c, + t1d, + t1h + FROM t1 + WHERE t1d NOT IN + ( + SELECT t2d + FROM t2 + FULL OUTER JOIN t3 ON t2a = t3a + JOIN t1 on t1b = t2b)) +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1b NOT IN + ( + SELECT cte1.t1b + FROM cte1 INNER + JOIN cte1 cte2 ON cte1.t1a = cte2.t1a + RIGHT JOIN cte1 cte3 ON cte1.t1b = cte3.t1b + JOIN cte1 cte4 ON cte1.t1c = cte4.t1c) AND + t1c IS NOT NULL +ORDER BY t1c DESC +-- !query 12 schema +struct +-- !query 12 output +val1b 8 16 19 2014-05-04 01:01:00 +val1c 8 16 19 2014-05-04 01:02:00.001 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-group-by.sql.out new file mode 100644 index 000000000000..6b86a9f6a0d0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-group-by.sql.out @@ -0,0 +1,150 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT t1a, + Avg(t1b) +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2) +GROUP BY t1a +-- !query 3 schema +struct +-- !query 3 output +val1a 11.0 +val1d 10.0 + + +-- !query 4 +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1h < t2h) +GROUP BY t1a +-- !query 4 schema +struct +-- !query 4 output +val1a 22 +val1d 10 +val1e 10 + + +-- !query 5 +SELECT Count(*) +FROM (SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t3a + FROM t3 + WHERE t3h != t2h)) t2 +WHERE t2b NOT IN (SELECT Min(t2b) + FROM t2 + WHERE t2b = t2b + GROUP BY t2c) +-- !query 5 schema +struct +-- !query 5 output +4 + + +-- !query 6 +SELECT t1a, + max(t1b) +FROM t1 +WHERE t1c NOT IN (SELECT Max(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a) +GROUP BY t1a +-- !query 6 schema +struct +-- !query 6 output +val1a 16 +val1b 8 +val1c 8 +val1d 10 + + +-- !query 7 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2a NOT IN (SELECT Min(t3a) + FROM t3 + WHERE t3a = t2a + GROUP BY t3b) order by t2a) +-- !query 7 schema +struct +-- !query 7 output +val1a 16 +val1a 16 +val1a 6 +val1a 6 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out new file mode 100644 index 000000000000..bae5d00cc863 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out @@ -0,0 +1,229 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 9 + + +-- !query 0 +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT t1a, + t1b, + t1c, + t3a, + t3b, + t3c +FROM t1 + JOIN t3 +WHERE t1a NOT IN (SELECT t2a + FROM t2) + AND t1b = t3b +-- !query 3 schema +struct +-- !query 3 output +val1a 6 8 val3a 6 12 +val1a 6 8 val3a 6 12 +val1a 6 8 val3a 6 12 +val1a 6 8 val3a 6 12 +val1d 10 NULL val1b 10 12 +val1d 10 NULL val1b 10 12 + + +-- !query 4 +SELECT t1a, + t1b, + t1c, + count(distinct(t3a)), + t3b, + t3c +FROM t1 +FULL OUTER JOIN t3 on t1b != t3b +RIGHT JOIN t2 on t1c = t2c +where t1a NOT IN + ( + SELECT t2a + FROM t2 + WHERE t2c NOT IN + ( + SELECT t1c + FROM t1 + WHERE t1a = t2a)) +AND t1b != t3b +AND t1d = t2d +GROUP BY t1a, t1b, t1c, t3a, t3b, t3c +HAVING count(distinct(t3a)) >= 1 +ORDER BY t1a, t3b +-- !query 4 schema +struct +-- !query 4 output +val1c 8 16 1 6 12 +val1c 8 16 1 10 12 +val1c 8 16 1 17 16 + + +-- !query 5 +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 + LEFT JOIN t3 on t2b = t3b + WHERE t1d = t2d + ) +AND t1d NOT IN + ( + SELECT t2d + FROM t2 + RIGHT JOIN t1 on t2e = t1e + WHERE t1a = t2a) +-- !query 5 schema +struct +-- !query 5 output +val1a 16 12 10 2014-07-04 01:01:00 +val1a 16 12 21 2014-06-04 01:02:00.001 +val1a 6 8 10 2014-04-04 01:00:00 +val1a 6 8 10 2014-04-04 01:02:00.001 +val1d 10 NULL 12 2015-05-04 01:01:00 +val1d NULL 16 22 2014-06-04 01:01:00 +val1e 10 NULL 25 2014-08-04 01:01:00 + + +-- !query 6 +SELECT Count(DISTINCT( t1a )), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +GROUP BY t1b, + t1c, + t1d +HAVING t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1d = t2d) +ORDER BY t1b DESC +-- !query 6 schema +struct +-- !query 6 output +1 16 12 10 +1 16 12 21 +1 10 NULL 12 +1 6 8 10 +1 NULL 16 22 + + +-- !query 7 +SELECT COUNT(DISTINCT(t1a)), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 INNER + JOIN t1 ON t1a = t2a) +GROUP BY t1b, + t1c, + t1d +HAVING t1b < sum(t1c) +-- !query 7 schema +struct +-- !query 7 output +1 6 8 10 + + +-- !query 8 +SELECT COUNT(DISTINCT(t1a)), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 INNER + JOIN t1 + ON t1a = t2a) +AND t1d NOT IN + ( + SELECT t2d + FROM t2 + INNER JOIN t3 + ON t2b = t3b ) +GROUP BY t1b, + t1c, + t1d +HAVING t1b < sum(t1c) +-- !query 8 schema +struct +-- !query 8 output +1 6 8 10 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out new file mode 100644 index 000000000000..d69b4bcf185c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/simple-in.sql.out @@ -0,0 +1,224 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 14 + + +-- !query 0 +create temporary view t1 as select * from values + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view t3 as select * from values + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2) +-- !query 3 schema +struct +-- !query 3 output +t1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +t1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +t1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +t1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +t1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query 4 +SELECT * +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +-- !query 4 schema +struct +-- !query 4 output +t1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 + + +-- !query 5 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t1a != t2a) +-- !query 5 schema +struct +-- !query 5 output +t1a 16 +t1a 16 +t1a 6 +t1a 6 + + +-- !query 6 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t1a = t2a + OR t1b > t2b) +-- !query 6 schema +struct +-- !query 6 output +t1a 16 +t1a 16 + + +-- !query 7 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2i IN (SELECT t3i + FROM t3 + WHERE t2c = t3c)) +-- !query 7 schema +struct +-- !query 7 output +t1a 6 +t1a 6 + + +-- !query 8 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2a IN (SELECT t3a + FROM t3 + WHERE t2c = t3c + AND t2b IS NOT NULL)) +-- !query 8 schema +struct +-- !query 8 output +t1a 6 +t1a 6 + + +-- !query 9 +SELECT DISTINCT( t1a ), + t1b, + t1h +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2) +-- !query 9 schema +struct +-- !query 9 output +t1a 16 2014-06-04 01:02:00.001 +t1a 16 2014-07-04 01:01:00 +t1a 6 2014-04-04 01:00:00 +t1a 6 2014-04-04 01:02:00.001 +t1d 10 2015-05-04 01:01:00 +t1d NULL 2014-06-04 01:01:00 +t1d NULL 2014-07-04 01:02:00.001 + + +-- !query 10 +create temporary view a as select * from values + (1, 1), (2, 1), (null, 1), (1, 3), (null, 3), (1, null), (null, 2) + as a(a1, a2) +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +create temporary view b as select * from values + (1, 1, 2), (null, 3, 2), (1, null, 2), (1, 2, null) + as b(b1, b2, b3) +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +SELECT a1, a2 +FROM a +WHERE a1 NOT IN (SELECT b.b1 + FROM b + WHERE a.a2 = b.b2) +-- !query 12 schema +struct +-- !query 12 output +1 NULL +2 1 + + +-- !query 13 +SELECT a1, a2 +FROM a +WHERE a1 NOT IN (SELECT b.b1 + FROM b + WHERE a.a2 = b.b2 + AND b.b3 > 1) +-- !query 13 schema +struct +-- !query 13 output +1 NULL +2 1 +NULL 2 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out new file mode 100644 index 000000000000..e4b1a2dbc675 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out @@ -0,0 +1,116 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES + (1, 0, 1) +AS t2(t2a, t2b, t2c) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES + (3, 1, 2) +AS t3(t3a, t3b, t3c) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT t1a, t2b +FROM t1, t2 +WHERE t1b = t2c +AND t2b = (SELECT max(avg) + FROM (SELECT t2b, avg(t2b) avg + FROM t2 + WHERE t2a = t1.t1b + ) + ) +-- !query 3 schema +struct<> +-- !query 3 output +org.apache.spark.sql.AnalysisException +grouping expressions sequence is empty, and 't2.`t2b`' is not an aggregate function. Wrap '(avg(CAST(t2.`t2b` AS BIGINT)) AS `avg`)' in windowing function(s) or wrap 't2.`t2b`' in first() (or first_value) if you don't care which value you get.; + + +-- !query 4 +SELECT * +FROM t1 +WHERE t1a IN (SELECT min(t2a) + FROM t2 + GROUP BY t2c + HAVING t2c IN (SELECT max(t3c) + FROM t3 + GROUP BY t3b + HAVING t3b > t2b )) +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +resolved attribute(s) t2b#x missing from min(t2a)#x,t2c#x in operator !Filter t2c#x IN (list#x [t2b#x]); + + +-- !query 5 +SELECT t1a +FROM t1 +GROUP BY 1 +HAVING EXISTS (SELECT 1 + FROM t2 + WHERE t2a < min(t1a + t2a)) +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +Found an aggregate expression in a correlated predicate that has both outer and local references, which is not supported yet. Aggregate expression: min((t1.`t1a` + t2.`t2a`)), Outer references: t1.`t1a`, Local references: t2.`t2a`.; + + +-- !query 6 +SELECT t1a +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE EXISTS (SELECT 1 + FROM t3 + GROUP BY 1 + HAVING min(t2a + t3a) > 1)) +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.AnalysisException +Found an aggregate expression in a correlated predicate that has both outer and local references, which is not supported yet. Aggregate expression: min((t2.`t2a` + t3.`t3a`)), Outer references: t2.`t2a`, Local references: t3.`t3a`.; + + +-- !query 7 +SELECT t1a +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE EXISTS (SELECT min(t2a) + FROM t3)) +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.AnalysisException +Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses: +Aggregate [min(outer(t2a#x)) AS min(outer())#x] ++- SubqueryAlias t3 + +- Project [t3a#x, t3b#x, t3c#x] + +- SubqueryAlias t3 + +- LocalRelation [t3a#x, t3b#x, t3c#x] +; diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out new file mode 100644 index 000000000000..70aeb9373f3c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out @@ -0,0 +1,104 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 7 + + +-- !query 0 +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES + (1, 0, 1) +AS t2(t2a, t2b, t2c) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES + (3, 1, 2) +AS t3(t3a, t3b, t3c) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT + ( SELECT max(t2b), min(t2b) + FROM t2 + WHERE t2.t2b = t1.t1b + GROUP BY t2.t2b + ) +FROM t1 +-- !query 3 schema +struct<> +-- !query 3 output +org.apache.spark.sql.AnalysisException +Scalar subquery must return only one column, but got 2; + + +-- !query 4 +SELECT + ( SELECT max(t2b), min(t2b) + FROM t2 + WHERE t2.t2b > 0 + GROUP BY t2.t2b + ) +FROM t1 +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +Scalar subquery must return only one column, but got 2; + + +-- !query 5 +SELECT * FROM t1 +WHERE +t1a IN (SELECT t2a, t2b + FROM t2 + WHERE t1a = t2a) +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +cannot resolve '(t1.`t1a` IN (listquery(t1.`t1a`)))' due to data type mismatch: +The number of columns in the left hand side of an IN subquery does not match the +number of columns in the output of subquery. +#columns in left hand side: 1. +#columns in right hand side: 2. +Left side columns: +[t1.`t1a`]. +Right side columns: +[t2.`t2a`, t2.`t2b`].; + + +-- !query 6 +SELECT * FROM T1 +WHERE +(t1a, t1b) IN (SELECT t2a + FROM t2 + WHERE t1a = t2a) +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.AnalysisException +cannot resolve '(named_struct('t1a', t1.`t1a`, 't1b', t1.`t1b`) IN (listquery(t1.`t1a`)))' due to data type mismatch: +The number of columns in the left hand side of an IN subquery does not match the +number of columns in the output of subquery. +#columns in left hand side: 2. +#columns in right hand side: 1. +Left side columns: +[t1.`t1a`, t1.`t1b`]. +Right side columns: +[t2.`t2a`].; diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out new file mode 100644 index 000000000000..8b29300e71f9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -0,0 +1,430 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 26 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW p AS VALUES (1, 1) AS T(pk, pv) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE OR REPLACE TEMPORARY VIEW c AS VALUES (1, 1) AS T(ck, cv) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT pk, cv +FROM p, c +WHERE p.pk = c.ck +AND c.cv = (SELECT avg(c1.cv) + FROM c c1 + WHERE c1.ck = p.pk) +-- !query 2 schema +struct +-- !query 2 output +1 1 + + +-- !query 3 +SELECT pk, cv +FROM p, c +WHERE p.pk = c.ck +AND c.cv = (SELECT max(avg) + FROM (SELECT c1.cv, avg(c1.cv) avg + FROM c c1 + WHERE c1.ck = p.pk + GROUP BY c1.cv)) +-- !query 3 schema +struct +-- !query 3 output +1 1 + + +-- !query 4 +create temporary view t1 as select * from values + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ('val1d', null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ('val1d', null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +create temporary view t2 as select * from values + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +create temporary view t3 as select * from values + ('val3a', 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ('val3a', 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ('val3c', 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ('val3c', 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ('val1b', null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ('val3b', 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val3b', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2) +-- !query 7 schema +struct +-- !query 7 output +val1b 8 +val1c 8 +val1d NULL +val1d NULL + + +-- !query 8 +SELECT t1a, t1d, t1f +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2) +AND t1b > (SELECT min(t3b) + FROM t3) +-- !query 8 schema +struct +-- !query 8 output +val1b 19 25.0 +val1c 19 25.0 + + +-- !query 9 +SELECT t1a, t1h +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2) +OR t1b = (SELECT min(t3b) + FROM t3 + WHERE t3b > 10) +-- !query 9 schema +struct +-- !query 9 output +val1b 2014-05-04 01:01:00 +val1c 2014-05-04 01:02:00.001 +val1d 2014-06-04 01:01:00 +val1d 2014-07-04 01:02:00.001 + + +-- !query 10 +SELECT t1a, t1b, t2d +FROM t1 LEFT JOIN t2 + ON t1a = t2a +WHERE t1b = (SELECT min(t3b) + FROM t3) +-- !query 10 schema +struct +-- !query 10 output +val1a 6 NULL +val1a 6 NULL + + +-- !query 11 +SELECT t1a, t1b, t1g +FROM t1 +WHERE t1c + 5 = (SELECT max(t2e) + FROM t2) +-- !query 11 schema +struct +-- !query 11 output +val1a 16 2000 +val1a 16 2000 + + +-- !query 12 +SELECT t1a, t1h +FROM t1 +WHERE date(t1h) = (SELECT min(t2i) + FROM t2) +-- !query 12 schema +struct +-- !query 12 output +val1a 2014-04-04 00:00:00 +val1a 2014-04-04 01:02:00.001 + + +-- !query 13 +SELECT t2d, t1a +FROM t1, t2 +WHERE t1b = t2b +AND t2c + 1 = (SELECT max(t2c) + 1 + FROM t2, t1 + WHERE t2b = t1b) +-- !query 13 schema +struct +-- !query 13 output +119 val1b +119 val1c +19 val1b +19 val1c + + +-- !query 14 +SELECT DISTINCT t2a, max_t1g +FROM t2, (SELECT max(t1g) max_t1g, t1a + FROM t1 + GROUP BY t1a) t1 +WHERE t2a = t1a +AND max_t1g = (SELECT max(t1g) + FROM t1) +-- !query 14 schema +struct +-- !query 14 output +val1b 2600 +val1c 2600 +val1e 2600 + + +-- !query 15 +SELECT t3b, t3c +FROM t3 +WHERE (SELECT max(t3c) + FROM t3 + WHERE t3b > 10) >= + (SELECT min(t3b) + FROM t3 + WHERE t3c > 0) +AND (t3b is null or t3c is null) +-- !query 15 schema +struct +-- !query 15 output +8 NULL +8 NULL +NULL 16 +NULL 16 + + +-- !query 16 +SELECT t1a +FROM t1 +WHERE t1a < (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query 16 schema +struct +-- !query 16 output +val1a +val1a +val1b + + +-- !query 17 +SELECT t1a, t1c +FROM t1 +WHERE (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) IS NULL +-- !query 17 schema +struct +-- !query 17 output +val1a 8 +val1a 8 +val1d NULL +val1e NULL +val1e NULL +val1e NULL + + +-- !query 18 +SELECT t1a +FROM t1 +WHERE t1a = (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c + HAVING count(*) >= 0) +OR t1i > '2014-12-31' +-- !query 18 schema +struct +-- !query 18 output +val1c +val1d + + +-- !query 19 +SELECT count(t1a) +FROM t1 RIGHT JOIN t2 +ON t1d = t2d +WHERE t1a < (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query 19 schema +struct +-- !query 19 output +7 + + +-- !query 20 +SELECT t1a +FROM t1 +WHERE t1b <= (SELECT max(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +AND t1b >= (SELECT min(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query 20 schema +struct +-- !query 20 output +val1b +val1c + + +-- !query 21 +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +INTERSECT +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query 21 schema +struct +-- !query 21 output +val1b +val1c + + +-- !query 22 +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +UNION ALL +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query 22 schema +struct +-- !query 22 output +val1a +val1a +val1b +val1b +val1c +val1c +val1d +val1d + + +-- !query 23 +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +UNION DISTINCT +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query 23 schema +struct +-- !query 23 output +val1a +val1b +val1c +val1d + + +-- !query 24 +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +MINUS +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query 24 schema +struct +-- !query 24 output +val1a + + +-- !query 25 +SELECT t1a +FROM t1 +GROUP BY t1a, t1c +HAVING max(t1b) <= (SELECT max(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query 25 schema +struct +-- !query 25 output +val1b +val1c diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out new file mode 100644 index 000000000000..807bb4722188 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -0,0 +1,198 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 11 + + +-- !query 0 +create temporary view t1 as select * from values + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ('val1d', null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ('val1d', null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view t3 as select * from values + ('val3a', 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ('val3a', 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ('val3c', 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ('val3c', 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ('val1b', null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ('val3b', 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val3b', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT (SELECT min(t3d) FROM t3) min_t3d, + (SELECT max(t2h) FROM t2) max_t2h +FROM t1 +WHERE t1a = 'val1c' +-- !query 3 schema +struct +-- !query 3 output +10 2017-05-04 01:01:00 + + +-- !query 4 +SELECT t1a, count(*) +FROM t1 +WHERE t1c IN (SELECT (SELECT min(t3c) FROM t3) + FROM t2 + GROUP BY t2g + HAVING count(*) > 1) +GROUP BY t1a +-- !query 4 schema +struct +-- !query 4 output +val1a 2 + + +-- !query 5 +SELECT (SELECT min(t3d) FROM t3) min_t3d, + null +FROM t1 +WHERE t1a = 'val1c' +UNION +SELECT null, + (SELECT max(t2h) FROM t2) max_t2h +FROM t1 +WHERE t1a = 'val1c' +-- !query 5 schema +struct +-- !query 5 output +10 NULL +NULL 2017-05-04 01:01:00 + + +-- !query 6 +SELECT (SELECT min(t3c) FROM t3) min_t3d +FROM t1 +WHERE t1a = 'val1a' +INTERSECT +SELECT (SELECT min(t2c) FROM t2) min_t2d +FROM t1 +WHERE t1a = 'val1d' +-- !query 6 schema +struct +-- !query 6 output +12 + + +-- !query 7 +SELECT q1.t1a, q2.t2a, q1.min_t3d, q2.avg_t3d +FROM (SELECT t1a, (SELECT min(t3d) FROM t3) min_t3d + FROM t1 + WHERE t1a IN ('val1e', 'val1c')) q1 + FULL OUTER JOIN + (SELECT t2a, (SELECT avg(t3d) FROM t3) avg_t3d + FROM t2 + WHERE t2a IN ('val1c', 'val2a')) q2 +ON q1.t1a = q2.t2a +AND q1.min_t3d < q2.avg_t3d +-- !query 7 schema +struct +-- !query 7 output +NULL val2a NULL 200.83333333333334 +val1c val1c 10 200.83333333333334 +val1c val1c 10 200.83333333333334 +val1e NULL 10 NULL +val1e NULL 10 NULL +val1e NULL 10 NULL + + +-- !query 8 +SELECT (SELECT min(t3d) FROM t3 WHERE t3.t3a = t1.t1a) min_t3d, + (SELECT max(t2h) FROM t2 WHERE t2.t2a = t1.t1a) max_t2h +FROM t1 +WHERE t1a = 'val1b' +-- !query 8 schema +struct +-- !query 8 output +19 2017-05-04 01:01:00 + + +-- !query 9 +SELECT (SELECT min(t3d) FROM t3 WHERE t3a = t1a) min_t3d +FROM t1 +WHERE t1a = 'val1b' +MINUS +SELECT (SELECT min(t3d) FROM t3) abs_min_t3d +FROM t1 +WHERE t1a = 'val1b' +-- !query 9 schema +struct +-- !query 9 output +19 + + +-- !query 10 +SELECT t1a, t1b +FROM t1 +WHERE NOT EXISTS (SELECT (SELECT max(t2b) + FROM t2 LEFT JOIN t1 + ON t2a = t1a + WHERE t2c = t3c) dummy + FROM t3 + WHERE t3b < (SELECT max(t2b) + FROM t2 LEFT JOIN t1 + ON t2a = t1a + WHERE t2c = t3c) + AND t3a = t1a) +-- !query 10 schema +struct +-- !query 10 output +val1a 16 +val1a 16 +val1a 6 +val1a 6 +val1c 8 +val1d 10 +val1d NULL +val1d NULL +val1e 10 +val1e 10 +val1e 10 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/subquery-in-from.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/subquery-in-from.sql.out new file mode 100644 index 000000000000..50370df34916 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/subquery/subquery-in-from.sql.out @@ -0,0 +1,50 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 6 + + +-- !query 0 +SELECT * FROM (SELECT * FROM testData) AS t WHERE key = 1 +-- !query 0 schema +struct +-- !query 0 output +1 1 + + +-- !query 1 +FROM (SELECT * FROM testData WHERE key = 1) AS t SELECT * +-- !query 1 schema +struct +-- !query 1 output +1 1 + + +-- !query 2 +SELECT * FROM (SELECT * FROM testData) t WHERE key = 1 +-- !query 2 schema +struct +-- !query 2 output +1 1 + + +-- !query 3 +FROM (SELECT * FROM testData WHERE key = 1) t SELECT * +-- !query 3 schema +struct +-- !query 3 output +1 1 + + +-- !query 4 +SELECT * FROM (SELECT * FROM testData) WHERE key = 1 +-- !query 4 schema +struct +-- !query 4 output +1 1 + + +-- !query 5 +FROM (SELECT * FROM testData WHERE key = 1) SELECT * +-- !query 5 schema +struct +-- !query 5 output +1 1 diff --git a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out new file mode 100644 index 000000000000..1a2bd5ea91cd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out @@ -0,0 +1,97 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 11 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1) AS testData(a, b) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT * FROM testData AS t(col1, col2) WHERE col1 = 1 +-- !query 1 schema +struct +-- !query 1 output +1 1 +1 2 + + +-- !query 2 +SELECT * FROM testData AS t(col1, col2) WHERE col1 = 2 +-- !query 2 schema +struct +-- !query 2 output +2 1 + + +-- !query 3 +SELECT col1 AS k, SUM(col2) FROM testData AS t(col1, col2) GROUP BY k +-- !query 3 schema +struct +-- !query 3 output +1 3 +2 1 + + +-- !query 4 +SELECT * FROM testData AS t(col1, col2, col3) +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +Number of column aliases does not match number of columns. Number of column aliases: 3; number of columns: 2.; line 1 pos 14 + + +-- !query 5 +SELECT * FROM testData AS t(col1) +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +Number of column aliases does not match number of columns. Number of column aliases: 1; number of columns: 2.; line 1 pos 14 + + +-- !query 6 +SELECT a AS col1, b AS col2 FROM testData AS t(c, d) +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.AnalysisException +cannot resolve '`a`' given input columns: [c, d]; line 1 pos 7 + + +-- !query 7 +SELECT * FROM (SELECT 1 AS a, 1 AS b) t(col1, col2) +-- !query 7 schema +struct +-- !query 7 output +1 1 + + +-- !query 8 +CREATE OR REPLACE TEMPORARY VIEW src1 AS SELECT * FROM VALUES (1, "a"), (2, "b"), (3, "c") AS src1(id, v1) +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +CREATE OR REPLACE TEMPORARY VIEW src2 AS SELECT * FROM VALUES (2, 1.0), (3, 3.2), (1, 8.5) AS src2(id, v2) +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +SELECT * FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d) +-- !query 10 schema +struct +-- !query 10 output +1 a 1 8.5 +2 b 2 1 +3 c 3 3.2 diff --git a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out new file mode 100644 index 000000000000..a8bc6faf1126 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out @@ -0,0 +1,135 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 10 + + +-- !query 0 +select * from dummy(3) +-- !query 0 schema +struct<> +-- !query 0 output +org.apache.spark.sql.AnalysisException +could not resolve `dummy` to a table-valued function; line 1 pos 14 + + +-- !query 1 +select * from range(6 + cos(3)) +-- !query 1 schema +struct +-- !query 1 output +0 +1 +2 +3 +4 + + +-- !query 2 +select * from range(5, 10) +-- !query 2 schema +struct +-- !query 2 output +5 +6 +7 +8 +9 + + +-- !query 3 +select * from range(0, 10, 2) +-- !query 3 schema +struct +-- !query 3 output +0 +2 +4 +6 +8 + + +-- !query 4 +select * from range(0, 10, 1, 200) +-- !query 4 schema +struct +-- !query 4 output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query 5 +select * from range(1, 1, 1, 1, 1) +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +error: table-valued function range with alternatives: + (end: long) + (start: long, end: long) + (start: long, end: long, step: long) + (start: long, end: long, step: long, numPartitions: integer) +cannot be applied to: (integer, integer, integer, integer, integer); line 1 pos 14 + + +-- !query 6 +select * from range(1, null) +-- !query 6 schema +struct<> +-- !query 6 output +java.lang.IllegalArgumentException +Invalid arguments for resolved function: 1, null + + +-- !query 7 +select * from RaNgE(2) +-- !query 7 schema +struct +-- !query 7 output +0 +1 + + +-- !query 8 +EXPLAIN select * from RaNgE(2) +-- !query 8 schema +struct +-- !query 8 output +== Physical Plan == +*Range (0, 2, step=1, splits=2) + + +-- !query 9 +EXPLAIN EXTENDED SELECT * FROM range(3) CROSS JOIN range(3) +-- !query 9 schema +struct +-- !query 9 output +== Parsed Logical Plan == +'Project [*] ++- 'Join Cross + :- 'UnresolvedTableValuedFunction range, [3] + +- 'UnresolvedTableValuedFunction range, [3] + +== Analyzed Logical Plan == +id: bigint, id: bigint +Project [id#xL, id#xL] ++- Join Cross + :- Range (0, 3, step=1, splits=None) + +- Range (0, 3, step=1, splits=None) + +== Optimized Logical Plan == +Join Cross +:- Range (0, 3, step=1, splits=None) ++- Range (0, 3, step=1, splits=None) + +== Physical Plan == +BroadcastNestedLoopJoin BuildRight, Cross +:- *Range (0, 3, step=1, splits=2) ++- BroadcastExchange IdentityBroadcastMode + +- *Range (0, 3, step=1, splits=2) diff --git a/sql/core/src/test/resources/sql-tests/results/tablesample-negative.sql.out b/sql/core/src/test/resources/sql-tests/results/tablesample-negative.sql.out new file mode 100644 index 000000000000..35f3931736b8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/tablesample-negative.sql.out @@ -0,0 +1,62 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 6 + + +-- !query 0 +CREATE DATABASE mydb1 +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +USE mydb1 +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1 +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT mydb1.t1 FROM t1 TABLESAMPLE (-1 PERCENT) +-- !query 3 schema +struct<> +-- !query 3 output +org.apache.spark.sql.catalyst.parser.ParseException + +Sampling fraction (-0.01) must be on interval [0, 1](line 1, pos 24) + +== SQL == +SELECT mydb1.t1 FROM t1 TABLESAMPLE (-1 PERCENT) +------------------------^^^ + + +-- !query 4 +SELECT mydb1.t1 FROM t1 TABLESAMPLE (101 PERCENT) +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.catalyst.parser.ParseException + +Sampling fraction (1.01) must be on interval [0, 1](line 1, pos 24) + +== SQL == +SELECT mydb1.t1 FROM t1 TABLESAMPLE (101 PERCENT) +------------------------^^^ + + +-- !query 5 +DROP DATABASE mydb1 CASCADE +-- !query 5 schema +struct<> +-- !query 5 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out new file mode 100644 index 000000000000..4815a578b102 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out @@ -0,0 +1,54 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 6 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(1), (2), (3), (4) +as t1(int_col1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT default.myDoubleAvg(int_col1) as my_avg from t1 +-- !query 2 schema +struct +-- !query 2 output +102.5 + + +-- !query 3 +SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1 +-- !query 3 schema +struct<> +-- !query 3 output +java.lang.AssertionError +assertion failed: Incorrect number of children + + +-- !query 4 +CREATE FUNCTION udaf1 AS 'test.non.existent.udaf' +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +SELECT default.udaf1(int_col1) as udaf1 from t1 +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +Can not load class 'test.non.existent.udaf' when registering the function 'default.udaf1', please make sure it is on the classpath; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/union.sql.out b/sql/core/src/test/resources/sql-tests/results/union.sql.out new file mode 100644 index 000000000000..d123b7fdbe0c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/union.sql.out @@ -0,0 +1,144 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 14 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (1, 'a'), (2, 'b') tbl(c1, c2) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (1.0, 1), (2.0, 4) tbl(c1, c2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT * +FROM (SELECT * FROM t1 + UNION ALL + SELECT * FROM t1) +-- !query 2 schema +struct +-- !query 2 output +1 a +1 a +2 b +2 b + + +-- !query 3 +SELECT * +FROM (SELECT * FROM t1 + UNION ALL + SELECT * FROM t2 + UNION ALL + SELECT * FROM t2) +-- !query 3 schema +struct +-- !query 3 output +1 1 +1 1 +1 a +2 4 +2 4 +2 b + + +-- !query 4 +SELECT a +FROM (SELECT 0 a, 0 b + UNION ALL + SELECT SUM(1) a, CAST(0 AS BIGINT) b + UNION ALL SELECT 0 a, 0 b) T +-- !query 4 schema +struct +-- !query 4 output +0 +0 +1 + + +-- !query 5 +CREATE OR REPLACE TEMPORARY VIEW p1 AS VALUES 1 T(col) +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +CREATE OR REPLACE TEMPORARY VIEW p2 AS VALUES 1 T(col) +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +CREATE OR REPLACE TEMPORARY VIEW p3 AS VALUES 1 T(col) +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +SELECT 1 AS x, + col +FROM (SELECT col AS col + FROM (SELECT p1.col AS col + FROM p1 CROSS JOIN p2 + UNION ALL + SELECT col + FROM p3) T1) T2 +-- !query 8 schema +struct +-- !query 8 output +1 1 +1 1 + + +-- !query 9 +DROP VIEW IF EXISTS t1 +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +DROP VIEW IF EXISTS t2 +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +DROP VIEW IF EXISTS p1 +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +DROP VIEW IF EXISTS p2 +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +DROP VIEW IF EXISTS p3 +-- !query 13 schema +struct<> +-- !query 13 output + diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out new file mode 100644 index 000000000000..73ad27e5bf8c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -0,0 +1,357 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 22 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, null, null, null, null, null), +(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +AS testData(val, val_long, val_double, val_date, val_timestamp, cate) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData +ORDER BY cate, val +-- !query 1 schema +struct +-- !query 1 output +NULL NULL 0 +3 NULL 1 +NULL a 0 +1 a 1 +1 a 1 +2 a 1 +1 b 1 +2 b 1 +3 b 1 + + +-- !query 2 +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val +ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query 2 schema +struct +-- !query 2 output +NULL NULL 3 +3 NULL 3 +NULL a 1 +1 a 2 +1 a 4 +2 a 4 +1 b 3 +2 b 6 +3 b 6 + + +-- !query 3 +SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long +ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long +-- !query 3 schema +struct<> +-- !query 3 output +org.apache.spark.sql.AnalysisException +cannot resolve 'ROWS BETWEEN CURRENT ROW AND 2147483648L FOLLOWING' due to data type mismatch: The data type of the upper bound 'LongType does not match the expected data type 'IntegerType'.; line 1 pos 41 + + +-- !query 4 +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE 1 PRECEDING) FROM testData +ORDER BY cate, val +-- !query 4 schema +struct +-- !query 4 output +NULL NULL 0 +3 NULL 1 +NULL a 0 +1 a 2 +1 a 2 +2 a 3 +1 b 1 +2 b 2 +3 b 2 + + +-- !query 5 +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query 5 schema +struct +-- !query 5 output +NULL NULL NULL +3 NULL 3 +NULL a NULL +1 a 4 +1 a 4 +2 a 2 +1 b 3 +2 b 5 +3 b 3 + + +-- !query 6 +SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long +RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long +-- !query 6 schema +struct +-- !query 6 output +NULL NULL NULL +1 NULL 1 +1 a 4 +1 a 4 +2 a 2147483652 +2147483650 a 2147483650 +NULL b NULL +3 b 2147483653 +2147483650 b 2147483650 + + +-- !query 7 +SELECT val_double, cate, sum(val_double) OVER(PARTITION BY cate ORDER BY val_double +RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY cate, val_double +-- !query 7 schema +struct +-- !query 7 output +NULL NULL NULL +1.0 NULL 1.0 +1.0 a 4.5 +1.0 a 4.5 +2.5 a 2.5 +100.001 a 100.001 +1.0 b 4.3 +3.3 b 3.3 +100.001 b 100.001 + + +-- !query 8 +SELECT val_date, cate, max(val_date) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY cate, val_date +-- !query 8 schema +struct +-- !query 8 output +NULL NULL NULL +2017-08-01 NULL 2017-08-01 +2017-08-01 a 2017-08-02 +2017-08-01 a 2017-08-02 +2017-08-02 a 2017-08-02 +2020-12-31 a 2020-12-31 +2017-08-01 b 2017-08-03 +2017-08-03 b 2017-08-03 +2020-12-31 b 2020-12-31 + + +-- !query 9 +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY cate, val_timestamp +-- !query 9 schema +struct +-- !query 9 output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5022728E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query 10 +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val DESC +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query 10 schema +struct +-- !query 10 output +NULL NULL NULL +3 NULL 3 +NULL a NULL +1 a 2 +1 a 2 +2 a 4 +1 b 1 +2 b 3 +3 b 5 + + +-- !query 11 +SELECT val, cate, count(val) OVER(PARTITION BY cate +ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.AnalysisException +cannot resolve 'ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING' due to data type mismatch: Window frame upper bound '1' does not followes the lower bound 'unboundedfollowing$()'.; line 1 pos 33 + + +-- !query 12 +SELECT val, cate, count(val) OVER(PARTITION BY cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +cannot resolve '(PARTITION BY testdata.`cate` RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame cannot be used in an unordered window specification.; line 1 pos 33 + + +-- !query 13 +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val, cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.AnalysisException +cannot resolve '(PARTITION BY testdata.`cate` ORDER BY testdata.`val` ASC NULLS FIRST, testdata.`cate` ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame with value boundaries cannot be used in a window specification with multiple order by expressions: val#x ASC NULLS FIRST,cate#x ASC NULLS FIRST; line 1 pos 33 + + +-- !query 14 +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY current_timestamp +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +cannot resolve '(PARTITION BY testdata.`cate` ORDER BY current_timestamp() ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: The data type 'TimestampType' used in the order specification does not match the data type 'IntegerType' which is used in the range frame.; line 1 pos 33 + + +-- !query 15 +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY cate, val +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.AnalysisException +cannot resolve 'RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING' due to data type mismatch: The lower bound of a window frame must be less than or equal to the upper bound; line 1 pos 33 + + +-- !query 16 +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.catalyst.parser.ParseException + +Frame bound value must be a literal.(line 2, pos 30) + +== SQL == +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val +------------------------------^^^ + + +-- !query 17 +SELECT val, cate, +max(val) OVER w AS max, +min(val) OVER w AS min, +min(val) OVER w AS min, +count(val) OVER w AS count, +sum(val) OVER w AS sum, +avg(val) OVER w AS avg, +stddev(val) OVER w AS stddev, +first_value(val) OVER w AS first_value, +first_value(val, true) OVER w AS first_value_ignore_null, +first_value(val, false) OVER w AS first_value_contain_null, +last_value(val) OVER w AS last_value, +last_value(val, true) OVER w AS last_value_ignore_null, +last_value(val, false) OVER w AS last_value_contain_null, +rank() OVER w AS rank, +dense_rank() OVER w AS dense_rank, +cume_dist() OVER w AS cume_dist, +percent_rank() OVER w AS percent_rank, +ntile(2) OVER w AS ntile, +row_number() OVER w AS row_number, +var_pop(val) OVER w AS var_pop, +var_samp(val) OVER w AS var_samp, +approx_count_distinct(val) OVER w AS approx_count_distinct +FROM testData +WINDOW w AS (PARTITION BY cate ORDER BY val) +ORDER BY cate, val +-- !query 17 schema +struct +-- !query 17 output +NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.5 0.0 1 1 NULL NULL 0 +3 NULL 3 3 3 1 3 3.0 NaN NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NaN 1 +NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 +1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 1 2 0.0 0.0 1 +1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 2 3 0.0 0.0 1 +2 a 2 1 1 3 4 1.3333333333333333 0.5773502691896258 NULL 1 NULL 2 2 2 4 3 1.0 1.0 2 4 0.22222222222222224 0.33333333333333337 2 +1 b 1 1 1 1 1 1.0 NaN 1 1 1 1 1 1 1 1 0.3333333333333333 0.0 1 1 0.0 NaN 1 +2 b 2 1 1 2 3 1.5 0.7071067811865476 1 1 1 2 2 2 2 2 0.6666666666666666 0.5 1 2 0.25 0.5 2 +3 b 3 1 1 3 6 2.0 1.0 1 1 1 3 3 3 3 3 1.0 1.0 2 3 0.6666666666666666 1.0 3 + + +-- !query 18 +SELECT val, cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val +-- !query 18 schema +struct +-- !query 18 output +NULL NULL NULL +3 NULL NULL +NULL a NULL +1 a NULL +1 a NULL +2 a NULL +1 b NULL +2 b NULL +3 b NULL + + +-- !query 19 +SELECT val, cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, val +-- !query 19 schema +struct<> +-- !query 19 output +org.apache.spark.sql.AnalysisException +Window function row_number() requires window to be ordered, please add ORDER BY clause. For example SELECT row_number()(value_expr) OVER (PARTITION BY window_partition ORDER BY window_ordering) from table; + + +-- !query 20 +SELECT val, cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val +-- !query 20 schema +struct +-- !query 20 output +NULL NULL 13 1.8571428571428572 +3 NULL 13 1.8571428571428572 +NULL a 13 1.8571428571428572 +1 a 13 1.8571428571428572 +1 a 13 1.8571428571428572 +2 a 13 1.8571428571428572 +1 b 13 1.8571428571428572 +2 b 13 1.8571428571428572 +3 b 13 1.8571428571428572 + + +-- !query 21 +SELECT val, cate, +first_value(false) OVER w AS first_value, +first_value(true, true) OVER w AS first_value_ignore_null, +first_value(false, false) OVER w AS first_value_contain_null, +last_value(false) OVER w AS last_value, +last_value(true, true) OVER w AS last_value_ignore_null, +last_value(false, false) OVER w AS last_value_contain_null +FROM testData +WINDOW w AS () +ORDER BY cate, val +-- !query 21 schema +struct +-- !query 21 output +NULL NULL false true false false true false +3 NULL false true false false true false +NULL a false true false false true false +1 a false true false false true false +1 a false true false false true false +2 a false true false false true false +1 b false true false false true false +2 b false true false false true false +3 b false true false false true false diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/metadata new file mode 100644 index 000000000000..3492220e36b8 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/metadata @@ -0,0 +1 @@ +{"id":"dddc5e7f-1e71-454c-8362-de184444fb5a"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/offsets/0 new file mode 100644 index 000000000000..cbde042e79af --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1489180207737} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/offsets/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/offsets/1 new file mode 100644 index 000000000000..10b5774746de --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/offsets/1 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1489180209261} +2 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/0/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/0/1.delta new file mode 100644 index 000000000000..635297805184 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/0/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/0/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/0/2.delta new file mode 100644 index 000000000000..635297805184 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/0/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/1/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/1/1.delta new file mode 100644 index 000000000000..7dc49cb3e47f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/1/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/1/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/1/2.delta new file mode 100644 index 000000000000..8b566e81f486 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/1/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/2/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/2/1.delta new file mode 100644 index 000000000000..ca2a7ed033f3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/2/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/2/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/2/2.delta new file mode 100644 index 000000000000..361f2db60502 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/2/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/3/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/3/1.delta new file mode 100644 index 000000000000..4c8804c61ad7 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/3/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/3/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/3/2.delta new file mode 100644 index 000000000000..7d3e07fe0330 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/3/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/4/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/4/1.delta new file mode 100644 index 000000000000..fe521b8c0750 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/4/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/4/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/4/2.delta new file mode 100644 index 000000000000..635297805184 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/4/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/5/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/5/1.delta new file mode 100644 index 000000000000..635297805184 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/5/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/5/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/5/2.delta new file mode 100644 index 000000000000..635297805184 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/5/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/6/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/6/1.delta new file mode 100644 index 000000000000..635297805184 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/6/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/6/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/6/2.delta new file mode 100644 index 000000000000..e69925cabaa9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/6/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/7/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/7/1.delta new file mode 100644 index 000000000000..635297805184 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/7/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/7/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/7/2.delta new file mode 100644 index 000000000000..36397a3dda24 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/7/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/8/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/8/1.delta new file mode 100644 index 000000000000..635297805184 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/8/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/8/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/8/2.delta new file mode 100644 index 000000000000..635297805184 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/8/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/9/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/9/1.delta new file mode 100644 index 000000000000..635297805184 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/9/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/9/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/9/2.delta new file mode 100644 index 000000000000..0c9b6ac5c863 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-2.1.0/state/0/9/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/file-sink-log-version-2.1.0/7.compact b/sql/core/src/test/resources/structured-streaming/file-sink-log-version-2.1.0/7.compact new file mode 100644 index 000000000000..e1ec8a74f052 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/file-sink-log-version-2.1.0/7.compact @@ -0,0 +1,9 @@ +v1 +{"path":"/a/b/0","size":1,"isDir":false,"modificationTime":1,"blockReplication":1,"blockSize":100,"action":"add"} +{"path":"/a/b/1","size":100,"isDir":false,"modificationTime":100,"blockReplication":1,"blockSize":100,"action":"add"} +{"path":"/a/b/2","size":200,"isDir":false,"modificationTime":200,"blockReplication":1,"blockSize":100,"action":"add"} +{"path":"/a/b/3","size":300,"isDir":false,"modificationTime":300,"blockReplication":1,"blockSize":100,"action":"add"} +{"path":"/a/b/4","size":400,"isDir":false,"modificationTime":400,"blockReplication":1,"blockSize":100,"action":"add"} +{"path":"/a/b/5","size":500,"isDir":false,"modificationTime":500,"blockReplication":1,"blockSize":100,"action":"add"} +{"path":"/a/b/6","size":600,"isDir":false,"modificationTime":600,"blockReplication":1,"blockSize":100,"action":"add"} +{"path":"/a/b/7","size":700,"isDir":false,"modificationTime":700,"blockReplication":1,"blockSize":100,"action":"add"} diff --git a/sql/core/src/test/resources/structured-streaming/file-sink-log-version-2.1.0/8 b/sql/core/src/test/resources/structured-streaming/file-sink-log-version-2.1.0/8 new file mode 100644 index 000000000000..e7989804e888 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/file-sink-log-version-2.1.0/8 @@ -0,0 +1,3 @@ +v1 +{"path":"/a/b/8","size":800,"isDir":false,"modificationTime":800,"blockReplication":1,"blockSize":100,"action":"add"} +{"path":"/a/b/0","size":100,"isDir":false,"modificationTime":100,"blockReplication":1,"blockSize":100,"action":"delete"} diff --git a/sql/core/src/test/resources/structured-streaming/file-sink-log-version-2.1.0/9 b/sql/core/src/test/resources/structured-streaming/file-sink-log-version-2.1.0/9 new file mode 100644 index 000000000000..42fb0ee41692 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/file-sink-log-version-2.1.0/9 @@ -0,0 +1,2 @@ +v1 +{"path":"/a/b/9","size":900,"isDir":false,"modificationTime":900,"blockReplication":3,"blockSize":200,"action":"add"} diff --git a/sql/core/src/test/resources/structured-streaming/file-source-log-version-2.1.0/2.compact b/sql/core/src/test/resources/structured-streaming/file-source-log-version-2.1.0/2.compact new file mode 100644 index 000000000000..95f78bb2620d --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/file-source-log-version-2.1.0/2.compact @@ -0,0 +1,4 @@ +v1 +{"path":"/a/b/0","timestamp":1480730949000,"batchId":0} +{"path":"/a/b/1","timestamp":1480730950000,"batchId":1} +{"path":"/a/b/2","timestamp":1480730950000,"batchId":2} diff --git a/sql/core/src/test/resources/structured-streaming/file-source-log-version-2.1.0/3 b/sql/core/src/test/resources/structured-streaming/file-source-log-version-2.1.0/3 new file mode 100644 index 000000000000..2caa5972e42e --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/file-source-log-version-2.1.0/3 @@ -0,0 +1,2 @@ +v1 +{"path":"/a/b/3","timestamp":1480730950000,"batchId":3} diff --git a/sql/core/src/test/resources/structured-streaming/file-source-log-version-2.1.0/4 b/sql/core/src/test/resources/structured-streaming/file-source-log-version-2.1.0/4 new file mode 100644 index 000000000000..e54b94322988 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/file-source-log-version-2.1.0/4 @@ -0,0 +1,2 @@ +v1 +{"path":"/a/b/4","timestamp":1480730951000,"batchId":4} diff --git a/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0-json.txt b/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0-json.txt new file mode 100644 index 000000000000..e266a47368e1 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0-json.txt @@ -0,0 +1 @@ +{"logOffset":345} diff --git a/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0-long.txt b/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0-long.txt new file mode 100644 index 000000000000..51b4008129ff --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0-long.txt @@ -0,0 +1 @@ +345 diff --git a/sql/core/src/test/resources/structured-streaming/offset-log-version-2.1.0/0 b/sql/core/src/test/resources/structured-streaming/offset-log-version-2.1.0/0 new file mode 100644 index 000000000000..988a98a7587d --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/offset-log-version-2.1.0/0 @@ -0,0 +1,4 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1480981499528} +{"logOffset":345} +{"topic-0":{"0":1}} diff --git a/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.0.txt b/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.0.txt new file mode 100644 index 000000000000..aa7e9a8c20c4 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.0.txt @@ -0,0 +1,4 @@ +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgress","queryInfo":{"name":"hello","id":0,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0"}],"sinkStatus":{"description":"org.apache.spark.sql.execution.streaming.MemorySink@2b85b3a5","offsetDesc":"[#0]"}}} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated","queryInfo":{"name":"hello","id":0,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0"}],"sinkStatus":{"description":"org.apache.spark.sql.execution.streaming.MemorySink@2b85b3a5","offsetDesc":"[#0]"}},"exception":null,"stackTrace":[]} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated","queryInfo":{"name":"hello","id":0,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0"}],"sinkStatus":{"description":"org.apache.spark.sql.execution.streaming.MemorySink@514502dc","offsetDesc":"[-]"}},"exception":"Query hello terminated with exception: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): java.lang.ArithmeticException: / by zero\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)\n\tat org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:246)\n\tat org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:240)\n\tat org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:784)\n\tat org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:784)\n\tat org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)\n\tat org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)\n\tat org.apache.spark.rdd.RDD.iterator(RDD.scala:283)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:85)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:","stackTrace":[{"methodName":"org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches","fileName":"StreamExecution.scala","lineNumber":208,"className":"org.apache.spark.sql.execution.streaming.StreamExecution","nativeMethod":false},{"methodName":"run","fileName":"StreamExecution.scala","lineNumber":120,"className":"org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1","nativeMethod":false}]} +{"Event":"SparkListenerApplicationEnd","Timestamp":1477593059313} diff --git a/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.1.txt b/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.1.txt new file mode 100644 index 000000000000..646cf107183b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.1.txt @@ -0,0 +1,4 @@ +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgress","queryInfo":{"name":"hello","id":0,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0"}],"sinkStatus":{"description":"org.apache.spark.sql.execution.streaming.MemorySink@10e5ec94","offsetDesc":"[#0]"}}} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated","queryInfo":{"name":"hello","id":0,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0"}],"sinkStatus":{"description":"org.apache.spark.sql.execution.streaming.MemorySink@10e5ec94","offsetDesc":"[#0]"}},"exception":null} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated","queryInfo":{"name":"hello","id":0,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0"}],"sinkStatus":{"description":"org.apache.spark.sql.execution.streaming.MemorySink@70c61dc8","offsetDesc":"[-]"}},"exception":"org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): java.lang.ArithmeticException: / by zero\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)\n\tat org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:246)\n\tat org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:240)\n\tat org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)\n\tat org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)\n\tat org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)\n\tat org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)\n\tat org.apache.spark.rdd.RDD.iterator(RDD.scala:283)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:86)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:\n\tat org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1454)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1442)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1441)\n\tat scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)\n\tat scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)\n\tat org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1441)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)\n\tat scala.Option.foreach(Option.scala:257)\n\tat org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:811)\n\tat org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1667)\n\tat org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1622)\n\tat org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1611)\n\tat org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)\n\tat org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:632)\n\tat org.apache.spark.SparkContext.runJob(SparkContext.scala:1890)\n\tat org.apache.spark.SparkContext.runJob(SparkContext.scala:1903)\n\tat org.apache.spark.SparkContext.runJob(SparkContext.scala:1916)\n\tat org.apache.spark.SparkContext.runJob(SparkContext.scala:1930)\n\tat org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:912)\n\tat org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)\n\tat org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)\n\tat org.apache.spark.rdd.RDD.withScope(RDD.scala:358)\n\tat org.apache.spark.rdd.RDD.collect(RDD.scala:911)\n\tat org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:290)\n\tat org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2193)\n\tat org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:57)\n\tat org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2546)\n\tat org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$execute$1(Dataset.scala:2192)\n\tat org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2197)\n\tat org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2197)\n\tat org.apache.spark.sql.Dataset.withCallback(Dataset.scala:2559)\n\tat org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collect(Dataset.scala:2197)\n\tat org.apache.spark.sql.Dataset.collect(Dataset.scala:2173)\n\tat org.apache.spark.sql.execution.streaming.MemorySink.addBatch(memory.scala:154)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatch(StreamExecution.scala:366)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1.apply$mcZ$sp(StreamExecution.scala:197)\n\tat org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:43)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:187)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:124)\nCaused by: java.lang.ArithmeticException: / by zero\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)\n\tat org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:246)\n\tat org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:240)\n\tat org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)\n\tat org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)\n\tat org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)\n\tat org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)\n\tat org.apache.spark.rdd.RDD.iterator(RDD.scala:283)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:86)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n"} +{"Event":"SparkListenerApplicationEnd","Timestamp":1477701734609} diff --git a/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.2.txt b/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.2.txt new file mode 100644 index 000000000000..57c44c862725 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.2.txt @@ -0,0 +1,5 @@ +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryStartedEvent","queryStatus":{"name":"query-1","id":1,"timestamp":1480491481350,"inputRate":0.0,"processingRate":0.0,"latency":null,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"-","inputRate":0.0,"processingRate":0.0,"triggerDetails":{}}],"sinkStatus":{"description":"FileSink[/Users/zsx/stream2]","offsetDesc":"[-]"},"triggerDetails":{}}} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgressEvent","queryStatus":{"name":"query-1","id":1,"timestamp":1480491493386,"inputRate":83.33333333333333,"processingRate":0.5773672055427251,"latency":1738.0,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0","inputRate":83.33333333333333,"processingRate":0.5773672055427251,"triggerDetails":{"latency.getBatch.source":"39","numRows.input.source":"1","latency.getOffset.source":"91","triggerId":"0"}}],"sinkStatus":{"description":"FileSink[/Users/zsx/stream2]","offsetDesc":"[#0]"},"triggerDetails":{"timestamp.afterGetBatch":"1480491491817","latency.offsetLogWrite":"26","timestamp.triggerStart":"1480491491653","triggerId":"0","timestamp.triggerFinish":"1480491493385","latency.fullTrigger":"1732","latency.getBatch.total":"44","timestamp.afterGetOffset":"1480491491772","numRows.input.total":"1","isTriggerActive":"false","latency.optimizer":"406","latency.getOffset.total":"91","isDataPresentInTrigger":"true"}}} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminatedEvent","queryStatus":{"name":"query-1","id":1,"timestamp":1480491532753,"inputRate":0.0,"processingRate":0.0,"latency":null,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0","inputRate":0.0,"processingRate":0.0,"triggerDetails":{"latency.getOffset.source":"1","triggerId":"1"}}],"sinkStatus":{"description":"FileSink[/Users/zsx/stream2]","offsetDesc":"[#0]"},"triggerDetails":{}},"exception":null} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminatedEvent","queryStatus":{"name":"query-0","id":0,"timestamp":1480491812530,"inputRate":0.0,"processingRate":0.0,"latency":null,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0","inputRate":0.0,"processingRate":0.0,"triggerDetails":{"latency.getBatch.source":"25","latency.getOffset.source":"65","triggerId":"0"}}],"sinkStatus":{"description":"FileSink[/Users/zsx/stream2]","offsetDesc":"[-]"},"triggerDetails":{}},"exception":"org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): org.apache.spark.SparkException: Task failed while writing rows.\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter.writePartitionToSingleFile(FileStreamSink.scala:183)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter$$anonfun$write$1.apply(FileStreamSink.scala:155)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter$$anonfun$write$1.apply(FileStreamSink.scala:153)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:86)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\nCaused by: java.lang.ArithmeticException: / by zero\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter.writePartitionToSingleFile(FileStreamSink.scala:172)\n\t... 8 more\n\nDriver stacktrace:\n\tat org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1454)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1442)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1441)\n\tat scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)\n\tat scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)\n\tat org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1441)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)\n\tat scala.Option.foreach(Option.scala:257)\n\tat org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:811)\n\tat org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1667)\n\tat org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1622)\n\tat org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1611)\n\tat org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)\n\tat org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:632)\n\tat org.apache.spark.SparkContext.runJob(SparkContext.scala:1873)\n\tat org.apache.spark.SparkContext.runJob(SparkContext.scala:1886)\n\tat org.apache.spark.SparkContext.runJob(SparkContext.scala:1906)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter.write(FileStreamSink.scala:151)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSink.addBatch(FileStreamSink.scala:70)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatch(StreamExecution.scala:437)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply$mcZ$sp(StreamExecution.scala:225)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply(StreamExecution.scala:213)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply(StreamExecution.scala:213)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$reportTimeTaken(StreamExecution.scala:656)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1.apply$mcZ$sp(StreamExecution.scala:212)\n\tat org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:43)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:208)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:142)\nCaused by: org.apache.spark.SparkException: Task failed while writing rows.\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter.writePartitionToSingleFile(FileStreamSink.scala:183)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter$$anonfun$write$1.apply(FileStreamSink.scala:155)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter$$anonfun$write$1.apply(FileStreamSink.scala:153)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:86)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\nCaused by: java.lang.ArithmeticException: / by zero\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)\n\tat org.apache.spark.sql.execution.streaming.FileStreamSinkWriter.writePartitionToSingleFile(FileStreamSink.scala:172)\n\t... 8 more\n"} +{"Event":"SparkListenerApplicationEnd","Timestamp":1480491541552} diff --git a/sql/core/src/test/resources/structured-streaming/query-metadata-logs-version-2.1.0.txt b/sql/core/src/test/resources/structured-streaming/query-metadata-logs-version-2.1.0.txt new file mode 100644 index 000000000000..79613e236216 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/query-metadata-logs-version-2.1.0.txt @@ -0,0 +1,3 @@ +{ + "id": "d366a8bf-db79-42ca-b5a4-d9ca0a11d63e" +} diff --git a/sql/core/src/test/resources/test-data/bool.csv b/sql/core/src/test/resources/test-data/bool.csv new file mode 100644 index 000000000000..94b2d49506e0 --- /dev/null +++ b/sql/core/src/test/resources/test-data/bool.csv @@ -0,0 +1,5 @@ +bool +"True" +"False" + +"true" diff --git a/sql/core/src/test/resources/test-data/cars-alternative.csv b/sql/core/src/test/resources/test-data/cars-alternative.csv new file mode 100644 index 000000000000..646f7c456c86 --- /dev/null +++ b/sql/core/src/test/resources/test-data/cars-alternative.csv @@ -0,0 +1,5 @@ +year|make|model|comment|blank +'2012'|'Tesla'|'S'| 'No comment'| + +1997|Ford|E350|'Go get one now they are going fast'| +2015|Chevy|Volt diff --git a/sql/core/src/test/resources/test-data/cars-blank-column-name.csv b/sql/core/src/test/resources/test-data/cars-blank-column-name.csv new file mode 100644 index 000000000000..0b804b1614d6 --- /dev/null +++ b/sql/core/src/test/resources/test-data/cars-blank-column-name.csv @@ -0,0 +1,3 @@ +"",,make,customer,comment +2012,"Tesla","S","bill","blank" +2013,"Tesla","S","c","something" diff --git a/sql/core/src/test/resources/test-data/cars-malformed.csv b/sql/core/src/test/resources/test-data/cars-malformed.csv new file mode 100644 index 000000000000..cfa378c01f1d --- /dev/null +++ b/sql/core/src/test/resources/test-data/cars-malformed.csv @@ -0,0 +1,6 @@ +~ All the rows here are malformed having tokens more than the schema (header). +year,make,model,comment,blank +"2012","Tesla","S","No comment",,null,null + +1997,Ford,E350,"Go get one now they are going fast",,null,null +2015,Chevy,,,, diff --git a/sql/core/src/test/resources/test-data/cars-null.csv b/sql/core/src/test/resources/test-data/cars-null.csv new file mode 100644 index 000000000000..130c0b40bbe7 --- /dev/null +++ b/sql/core/src/test/resources/test-data/cars-null.csv @@ -0,0 +1,6 @@ +year,make,model,comment,blank +"2012","Tesla","S",null, + +1997,Ford,E350,"Go get one now they are going fast", +null,Chevy,Volt + diff --git a/sql/core/src/test/resources/test-data/cars-unbalanced-quotes.csv b/sql/core/src/test/resources/test-data/cars-unbalanced-quotes.csv new file mode 100644 index 000000000000..5ea39fcbfadc --- /dev/null +++ b/sql/core/src/test/resources/test-data/cars-unbalanced-quotes.csv @@ -0,0 +1,4 @@ +year,make,model,comment,blank +"2012,Tesla,S,No comment +1997,Ford,E350,Go get one now they are going fast" +"2015,"Chevy",Volt, diff --git a/sql/core/src/test/resources/test-data/cars.csv b/sql/core/src/test/resources/test-data/cars.csv new file mode 100644 index 000000000000..40ded573ade5 --- /dev/null +++ b/sql/core/src/test/resources/test-data/cars.csv @@ -0,0 +1,7 @@ + +year,make,model,comment,blank +"2012","Tesla","S","No comment", + +1997,Ford,E350,"Go get one now they are going fast", +2015,Chevy,Volt + diff --git a/sql/core/src/test/resources/test-data/cars.tsv b/sql/core/src/test/resources/test-data/cars.tsv new file mode 100644 index 000000000000..a7bfa9a91f96 --- /dev/null +++ b/sql/core/src/test/resources/test-data/cars.tsv @@ -0,0 +1,4 @@ +year make model price comment blank +2012 Tesla S "80,000.65" +1997 Ford E350 35,000 "Go get one now they are going fast" +2015 Chevy Volt 5,000.10 diff --git a/sql/core/src/test/resources/test-data/cars_iso-8859-1.csv b/sql/core/src/test/resources/test-data/cars_iso-8859-1.csv new file mode 100644 index 000000000000..c51b6c59010f --- /dev/null +++ b/sql/core/src/test/resources/test-data/cars_iso-8859-1.csv @@ -0,0 +1,6 @@ +yearmakemodelcommentblank +"2012""Tesla""S""No comment" + +1997FordE350"Go get one now they are oing fast" +2015ChevyVolt + diff --git a/sql/core/src/test/resources/test-data/comments.csv b/sql/core/src/test/resources/test-data/comments.csv new file mode 100644 index 000000000000..6275be7285b3 --- /dev/null +++ b/sql/core/src/test/resources/test-data/comments.csv @@ -0,0 +1,6 @@ +~ Version 1.0 +~ Using a non-standard comment char to test CSV parser defaults are overridden +1,2,3,4,5.01,2015-08-20 15:57:00 +6,7,8,9,0,2015-08-21 16:58:01 +~0,9,8,7,6,2015-08-22 17:59:02 +1,2,3,4,5,2015-08-23 18:00:42 diff --git a/sql/core/src/test/resources/test-data/dates.csv b/sql/core/src/test/resources/test-data/dates.csv new file mode 100644 index 000000000000..9ee99c31b334 --- /dev/null +++ b/sql/core/src/test/resources/test-data/dates.csv @@ -0,0 +1,4 @@ +date +26/08/2015 18:00 +27/10/2014 18:30 +28/01/2016 20:00 diff --git a/sql/core/src/test/resources/test-data/dec-in-fixed-len.parquet b/sql/core/src/test/resources/test-data/dec-in-fixed-len.parquet new file mode 100644 index 000000000000..6ad37d563951 Binary files /dev/null and b/sql/core/src/test/resources/test-data/dec-in-fixed-len.parquet differ diff --git a/sql/core/src/test/resources/dec-in-i32.parquet b/sql/core/src/test/resources/test-data/dec-in-i32.parquet similarity index 100% rename from sql/core/src/test/resources/dec-in-i32.parquet rename to sql/core/src/test/resources/test-data/dec-in-i32.parquet diff --git a/sql/core/src/test/resources/dec-in-i64.parquet b/sql/core/src/test/resources/test-data/dec-in-i64.parquet similarity index 100% rename from sql/core/src/test/resources/dec-in-i64.parquet rename to sql/core/src/test/resources/test-data/dec-in-i64.parquet diff --git a/sql/core/src/test/resources/test-data/decimal.csv b/sql/core/src/test/resources/test-data/decimal.csv new file mode 100644 index 000000000000..870f6aaf1bb4 --- /dev/null +++ b/sql/core/src/test/resources/test-data/decimal.csv @@ -0,0 +1,7 @@ +~ decimal field has integer, integer and decimal values. The last value cannot fit to a long +~ long field has integer, long and integer values. +~ double field has double, double and decimal values. +decimal,long,double +1,1,0.1 +1,9223372036854775807,1.0 +92233720368547758070,1,92233720368547758070 diff --git a/sql/core/src/test/resources/test-data/disable_comments.csv b/sql/core/src/test/resources/test-data/disable_comments.csv new file mode 100644 index 000000000000..304d406e4d98 --- /dev/null +++ b/sql/core/src/test/resources/test-data/disable_comments.csv @@ -0,0 +1,2 @@ +#1,2,3 +4,5,6 diff --git a/core/src/test/resources/spark-events/local-1422981759269/APPLICATION_COMPLETE b/sql/core/src/test/resources/test-data/empty.csv old mode 100755 new mode 100644 similarity index 100% rename from core/src/test/resources/spark-events/local-1422981759269/APPLICATION_COMPLETE rename to sql/core/src/test/resources/test-data/empty.csv diff --git a/sql/core/src/test/resources/nested-array-struct.parquet b/sql/core/src/test/resources/test-data/nested-array-struct.parquet similarity index 100% rename from sql/core/src/test/resources/nested-array-struct.parquet rename to sql/core/src/test/resources/test-data/nested-array-struct.parquet diff --git a/sql/core/src/test/resources/test-data/numbers.csv b/sql/core/src/test/resources/test-data/numbers.csv new file mode 100644 index 000000000000..af8feac784d8 --- /dev/null +++ b/sql/core/src/test/resources/test-data/numbers.csv @@ -0,0 +1,9 @@ +int,long,float,double +8,1000000,1.042,23848545.0374 +--,34232323,98.343,184721.23987223 +34,--,98.343,184721.23987223 +34,43323123,--,184721.23987223 +34,43323123,223823.9484,-- +34,43323123,223823.NAN,NAN +34,43323123,223823.INF,INF +34,43323123,223823.-INF,-INF diff --git a/sql/core/src/test/resources/old-repeated-int.parquet b/sql/core/src/test/resources/test-data/old-repeated-int.parquet similarity index 100% rename from sql/core/src/test/resources/old-repeated-int.parquet rename to sql/core/src/test/resources/test-data/old-repeated-int.parquet diff --git a/sql/core/src/test/resources/old-repeated-message.parquet b/sql/core/src/test/resources/test-data/old-repeated-message.parquet similarity index 100% rename from sql/core/src/test/resources/old-repeated-message.parquet rename to sql/core/src/test/resources/test-data/old-repeated-message.parquet diff --git a/sql/core/src/test/resources/parquet-thrift-compat.snappy.parquet b/sql/core/src/test/resources/test-data/parquet-thrift-compat.snappy.parquet similarity index 100% rename from sql/core/src/test/resources/parquet-thrift-compat.snappy.parquet rename to sql/core/src/test/resources/test-data/parquet-thrift-compat.snappy.parquet diff --git a/sql/core/src/test/resources/proto-repeated-string.parquet b/sql/core/src/test/resources/test-data/proto-repeated-string.parquet similarity index 100% rename from sql/core/src/test/resources/proto-repeated-string.parquet rename to sql/core/src/test/resources/test-data/proto-repeated-string.parquet diff --git a/sql/core/src/test/resources/proto-repeated-struct.parquet b/sql/core/src/test/resources/test-data/proto-repeated-struct.parquet similarity index 100% rename from sql/core/src/test/resources/proto-repeated-struct.parquet rename to sql/core/src/test/resources/test-data/proto-repeated-struct.parquet diff --git a/sql/core/src/test/resources/proto-struct-with-array-many.parquet b/sql/core/src/test/resources/test-data/proto-struct-with-array-many.parquet similarity index 100% rename from sql/core/src/test/resources/proto-struct-with-array-many.parquet rename to sql/core/src/test/resources/test-data/proto-struct-with-array-many.parquet diff --git a/sql/core/src/test/resources/proto-struct-with-array.parquet b/sql/core/src/test/resources/test-data/proto-struct-with-array.parquet similarity index 100% rename from sql/core/src/test/resources/proto-struct-with-array.parquet rename to sql/core/src/test/resources/test-data/proto-struct-with-array.parquet diff --git a/sql/core/src/test/resources/test-data/simple_sparse.csv b/sql/core/src/test/resources/test-data/simple_sparse.csv new file mode 100644 index 000000000000..02d29cabf95f --- /dev/null +++ b/sql/core/src/test/resources/test-data/simple_sparse.csv @@ -0,0 +1,5 @@ +A,B,C,D +1,,, +,1,, +,,1, +,,,1 diff --git a/sql/core/src/test/resources/test-data/text-partitioned/year=2014/data.txt b/sql/core/src/test/resources/test-data/text-partitioned/year=2014/data.txt new file mode 100644 index 000000000000..e2719428bb28 --- /dev/null +++ b/sql/core/src/test/resources/test-data/text-partitioned/year=2014/data.txt @@ -0,0 +1 @@ +2014-test diff --git a/sql/core/src/test/resources/test-data/text-partitioned/year=2015/data.txt b/sql/core/src/test/resources/test-data/text-partitioned/year=2015/data.txt new file mode 100644 index 000000000000..b8c03daa8c19 --- /dev/null +++ b/sql/core/src/test/resources/test-data/text-partitioned/year=2015/data.txt @@ -0,0 +1 @@ +2015-test diff --git a/sql/core/src/test/resources/text-suite.txt b/sql/core/src/test/resources/test-data/text-suite.txt similarity index 100% rename from sql/core/src/test/resources/text-suite.txt rename to sql/core/src/test/resources/test-data/text-suite.txt diff --git a/sql/core/src/test/resources/test-data/text-suite2.txt b/sql/core/src/test/resources/test-data/text-suite2.txt new file mode 100644 index 000000000000..f9d498c80493 --- /dev/null +++ b/sql/core/src/test/resources/test-data/text-suite2.txt @@ -0,0 +1 @@ +This is another file for testing multi path loading. diff --git a/sql/core/src/test/resources/test-data/timemillis-in-i64.parquet b/sql/core/src/test/resources/test-data/timemillis-in-i64.parquet new file mode 100644 index 000000000000..d3c39e2c26ee Binary files /dev/null and b/sql/core/src/test/resources/test-data/timemillis-in-i64.parquet differ diff --git a/sql/core/src/test/resources/test-data/unescaped-quotes.csv b/sql/core/src/test/resources/test-data/unescaped-quotes.csv new file mode 100644 index 000000000000..7c68055575de --- /dev/null +++ b/sql/core/src/test/resources/test-data/unescaped-quotes.csv @@ -0,0 +1,2 @@ +"a"b,ccc,ddd +ab,cc"c,ddd" diff --git a/sql/core/src/test/resources/test-data/value-malformed.csv b/sql/core/src/test/resources/test-data/value-malformed.csv new file mode 100644 index 000000000000..8945ed73d2e8 --- /dev/null +++ b/sql/core/src/test/resources/test-data/value-malformed.csv @@ -0,0 +1,2 @@ +0,2013-111-11 12:13:14 +1,1983-08-04 diff --git a/sql/core/src/test/resources/tpcds/q1.sql b/sql/core/src/test/resources/tpcds/q1.sql new file mode 100755 index 000000000000..4d20faad8ef5 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q1.sql @@ -0,0 +1,19 @@ +WITH customer_total_return AS +( SELECT + sr_customer_sk AS ctr_customer_sk, + sr_store_sk AS ctr_store_sk, + sum(sr_return_amt) AS ctr_total_return + FROM store_returns, date_dim + WHERE sr_returned_date_sk = d_date_sk AND d_year = 2000 + GROUP BY sr_customer_sk, sr_store_sk) +SELECT c_customer_id +FROM customer_total_return ctr1, store, customer +WHERE ctr1.ctr_total_return > + (SELECT avg(ctr_total_return) * 1.2 + FROM customer_total_return ctr2 + WHERE ctr1.ctr_store_sk = ctr2.ctr_store_sk) + AND s_store_sk = ctr1.ctr_store_sk + AND s_state = 'TN' + AND ctr1.ctr_customer_sk = c_customer_sk +ORDER BY c_customer_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q10.sql b/sql/core/src/test/resources/tpcds/q10.sql new file mode 100755 index 000000000000..5500e1aea155 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q10.sql @@ -0,0 +1,57 @@ +SELECT + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3, + cd_dep_count, + count(*) cnt4, + cd_dep_employed_count, + count(*) cnt5, + cd_dep_college_count, + count(*) cnt6 +FROM + customer c, customer_address ca, customer_demographics +WHERE + c.c_current_addr_sk = ca.ca_address_sk AND + ca_county IN ('Rush County', 'Toole County', 'Jefferson County', + 'Dona Ana County', 'La Porte County') AND + cd_demo_sk = c.c_current_cdemo_sk AND + exists(SELECT * + FROM store_sales, date_dim + WHERE c.c_customer_sk = ss_customer_sk AND + ss_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_moy BETWEEN 1 AND 1 + 3) AND + (exists(SELECT * + FROM web_sales, date_dim + WHERE c.c_customer_sk = ws_bill_customer_sk AND + ws_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_moy BETWEEN 1 AND 1 + 3) OR + exists(SELECT * + FROM catalog_sales, date_dim + WHERE c.c_customer_sk = cs_ship_customer_sk AND + cs_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_moy BETWEEN 1 AND 1 + 3)) +GROUP BY cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count +ORDER BY cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q11.sql b/sql/core/src/test/resources/tpcds/q11.sql new file mode 100755 index 000000000000..3618fb14fa39 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q11.sql @@ -0,0 +1,68 @@ +WITH year_total AS ( + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum(ss_ext_list_price - ss_ext_discount_amt) year_total, + 's' sale_type + FROM customer, store_sales, date_dim + WHERE c_customer_sk = ss_customer_sk + AND ss_sold_date_sk = d_date_sk + GROUP BY c_customer_id + , c_first_name + , c_last_name + , d_year + , c_preferred_cust_flag + , c_birth_country + , c_login + , c_email_address + , d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum(ws_ext_list_price - ws_ext_discount_amt) year_total, + 'w' sale_type + FROM customer, web_sales, date_dim + WHERE c_customer_sk = ws_bill_customer_sk + AND ws_sold_date_sk = d_date_sk + GROUP BY + c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, + c_login, c_email_address, d_year) +SELECT t_s_secyear.customer_preferred_cust_flag +FROM year_total t_s_firstyear + , year_total t_s_secyear + , year_total t_w_firstyear + , year_total t_w_secyear +WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_secyear.customer_id + AND t_s_firstyear.customer_id = t_w_firstyear.customer_id + AND t_s_firstyear.sale_type = 's' + AND t_w_firstyear.sale_type = 'w' + AND t_s_secyear.sale_type = 's' + AND t_w_secyear.sale_type = 'w' + AND t_s_firstyear.dyear = 2001 + AND t_s_secyear.dyear = 2001 + 1 + AND t_w_firstyear.dyear = 2001 + AND t_w_secyear.dyear = 2001 + 1 + AND t_s_firstyear.year_total > 0 + AND t_w_firstyear.year_total > 0 + AND CASE WHEN t_w_firstyear.year_total > 0 + THEN t_w_secyear.year_total / t_w_firstyear.year_total + ELSE NULL END + > CASE WHEN t_s_firstyear.year_total > 0 + THEN t_s_secyear.year_total / t_s_firstyear.year_total + ELSE NULL END +ORDER BY t_s_secyear.customer_preferred_cust_flag +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q12.sql b/sql/core/src/test/resources/tpcds/q12.sql new file mode 100755 index 000000000000..0382737f5aa2 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q12.sql @@ -0,0 +1,22 @@ +SELECT + i_item_desc, + i_category, + i_class, + i_current_price, + sum(ws_ext_sales_price) AS itemrevenue, + sum(ws_ext_sales_price) * 100 / sum(sum(ws_ext_sales_price)) + OVER + (PARTITION BY i_class) AS revenueratio +FROM + web_sales, item, date_dim +WHERE + ws_item_sk = i_item_sk + AND i_category IN ('Sports', 'Books', 'Home') + AND ws_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('1999-02-22' AS DATE) + AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) +GROUP BY + i_item_id, i_item_desc, i_category, i_class, i_current_price +ORDER BY + i_category, i_class, i_item_id, i_item_desc, revenueratio +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q13.sql b/sql/core/src/test/resources/tpcds/q13.sql new file mode 100755 index 000000000000..32dc9e26097b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q13.sql @@ -0,0 +1,49 @@ +SELECT + avg(ss_quantity), + avg(ss_ext_sales_price), + avg(ss_ext_wholesale_cost), + sum(ss_ext_wholesale_cost) +FROM store_sales + , store + , customer_demographics + , household_demographics + , customer_address + , date_dim +WHERE s_store_sk = ss_store_sk + AND ss_sold_date_sk = d_date_sk AND d_year = 2001 + AND ((ss_hdemo_sk = hd_demo_sk + AND cd_demo_sk = ss_cdemo_sk + AND cd_marital_status = 'M' + AND cd_education_status = 'Advanced Degree' + AND ss_sales_price BETWEEN 100.00 AND 150.00 + AND hd_dep_count = 3 +) OR + (ss_hdemo_sk = hd_demo_sk + AND cd_demo_sk = ss_cdemo_sk + AND cd_marital_status = 'S' + AND cd_education_status = 'College' + AND ss_sales_price BETWEEN 50.00 AND 100.00 + AND hd_dep_count = 1 + ) OR + (ss_hdemo_sk = hd_demo_sk + AND cd_demo_sk = ss_cdemo_sk + AND cd_marital_status = 'W' + AND cd_education_status = '2 yr Degree' + AND ss_sales_price BETWEEN 150.00 AND 200.00 + AND hd_dep_count = 1 + )) + AND ((ss_addr_sk = ca_address_sk + AND ca_country = 'United States' + AND ca_state IN ('TX', 'OH', 'TX') + AND ss_net_profit BETWEEN 100 AND 200 +) OR + (ss_addr_sk = ca_address_sk + AND ca_country = 'United States' + AND ca_state IN ('OR', 'NM', 'KY') + AND ss_net_profit BETWEEN 150 AND 300 + ) OR + (ss_addr_sk = ca_address_sk + AND ca_country = 'United States' + AND ca_state IN ('VA', 'TX', 'MS') + AND ss_net_profit BETWEEN 50 AND 250 + )) diff --git a/sql/core/src/test/resources/tpcds/q14a.sql b/sql/core/src/test/resources/tpcds/q14a.sql new file mode 100755 index 000000000000..954ddd41be0e --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q14a.sql @@ -0,0 +1,120 @@ +WITH cross_items AS +(SELECT i_item_sk ss_item_sk + FROM item, + (SELECT + iss.i_brand_id brand_id, + iss.i_class_id class_id, + iss.i_category_id category_id + FROM store_sales, item iss, date_dim d1 + WHERE ss_item_sk = iss.i_item_sk + AND ss_sold_date_sk = d1.d_date_sk + AND d1.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + ics.i_brand_id, + ics.i_class_id, + ics.i_category_id + FROM catalog_sales, item ics, date_dim d2 + WHERE cs_item_sk = ics.i_item_sk + AND cs_sold_date_sk = d2.d_date_sk + AND d2.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + iws.i_brand_id, + iws.i_class_id, + iws.i_category_id + FROM web_sales, item iws, date_dim d3 + WHERE ws_item_sk = iws.i_item_sk + AND ws_sold_date_sk = d3.d_date_sk + AND d3.d_year BETWEEN 1999 AND 1999 + 2) x + WHERE i_brand_id = brand_id + AND i_class_id = class_id + AND i_category_id = category_id +), + avg_sales AS + (SELECT avg(quantity * list_price) average_sales + FROM ( + SELECT + ss_quantity quantity, + ss_list_price list_price + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk + AND d_year BETWEEN 1999 AND 2001 + UNION ALL + SELECT + cs_quantity quantity, + cs_list_price list_price + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk + AND d_year BETWEEN 1999 AND 1999 + 2 + UNION ALL + SELECT + ws_quantity quantity, + ws_list_price list_price + FROM web_sales, date_dim + WHERE ws_sold_date_sk = d_date_sk + AND d_year BETWEEN 1999 AND 1999 + 2) x) +SELECT + channel, + i_brand_id, + i_class_id, + i_category_id, + sum(sales), + sum(number_sales) +FROM ( + SELECT + 'store' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ss_quantity * ss_list_price) sales, + count(*) number_sales + FROM store_sales, item, date_dim + WHERE ss_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 1999 + 2 + AND d_moy = 11 + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales + FROM avg_sales) + UNION ALL + SELECT + 'catalog' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(cs_quantity * cs_list_price) sales, + count(*) number_sales + FROM catalog_sales, item, date_dim + WHERE cs_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 1999 + 2 + AND d_moy = 11 + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(cs_quantity * cs_list_price) > (SELECT average_sales FROM avg_sales) + UNION ALL + SELECT + 'web' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ws_quantity * ws_list_price) sales, + count(*) number_sales + FROM web_sales, item, date_dim + WHERE ws_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 1999 + 2 + AND d_moy = 11 + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ws_quantity * ws_list_price) > (SELECT average_sales + FROM avg_sales) + ) y +GROUP BY ROLLUP (channel, i_brand_id, i_class_id, i_category_id) +ORDER BY channel, i_brand_id, i_class_id, i_category_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q14b.sql b/sql/core/src/test/resources/tpcds/q14b.sql new file mode 100755 index 000000000000..929a8484bf9b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q14b.sql @@ -0,0 +1,95 @@ +WITH cross_items AS +(SELECT i_item_sk ss_item_sk + FROM item, + (SELECT + iss.i_brand_id brand_id, + iss.i_class_id class_id, + iss.i_category_id category_id + FROM store_sales, item iss, date_dim d1 + WHERE ss_item_sk = iss.i_item_sk + AND ss_sold_date_sk = d1.d_date_sk + AND d1.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + ics.i_brand_id, + ics.i_class_id, + ics.i_category_id + FROM catalog_sales, item ics, date_dim d2 + WHERE cs_item_sk = ics.i_item_sk + AND cs_sold_date_sk = d2.d_date_sk + AND d2.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + iws.i_brand_id, + iws.i_class_id, + iws.i_category_id + FROM web_sales, item iws, date_dim d3 + WHERE ws_item_sk = iws.i_item_sk + AND ws_sold_date_sk = d3.d_date_sk + AND d3.d_year BETWEEN 1999 AND 1999 + 2) x + WHERE i_brand_id = brand_id + AND i_class_id = class_id + AND i_category_id = category_id +), + avg_sales AS + (SELECT avg(quantity * list_price) average_sales + FROM (SELECT + ss_quantity quantity, + ss_list_price list_price + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2 + UNION ALL + SELECT + cs_quantity quantity, + cs_list_price list_price + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2 + UNION ALL + SELECT + ws_quantity quantity, + ws_list_price list_price + FROM web_sales, date_dim + WHERE ws_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2) x) +SELECT * +FROM + (SELECT + 'store' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ss_quantity * ss_list_price) sales, + count(*) number_sales + FROM store_sales, item, date_dim + WHERE ss_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_year = 1999 + 1 AND d_moy = 12 AND d_dom = 11) + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales + FROM avg_sales)) this_year, + (SELECT + 'store' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ss_quantity * ss_list_price) sales, + count(*) number_sales + FROM store_sales, item, date_dim + WHERE ss_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_year = 1999 AND d_moy = 12 AND d_dom = 11) + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales + FROM avg_sales)) last_year +WHERE this_year.i_brand_id = last_year.i_brand_id + AND this_year.i_class_id = last_year.i_class_id + AND this_year.i_category_id = last_year.i_category_id +ORDER BY this_year.channel, this_year.i_brand_id, this_year.i_class_id, this_year.i_category_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q15.sql b/sql/core/src/test/resources/tpcds/q15.sql new file mode 100755 index 000000000000..b8182e23b019 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q15.sql @@ -0,0 +1,15 @@ +SELECT + ca_zip, + sum(cs_sales_price) +FROM catalog_sales, customer, customer_address, date_dim +WHERE cs_bill_customer_sk = c_customer_sk + AND c_current_addr_sk = ca_address_sk + AND (substr(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', + '85392', '85460', '80348', '81792') + OR ca_state IN ('CA', 'WA', 'GA') + OR cs_sales_price > 500) + AND cs_sold_date_sk = d_date_sk + AND d_qoy = 2 AND d_year = 2001 +GROUP BY ca_zip +ORDER BY ca_zip +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q16.sql b/sql/core/src/test/resources/tpcds/q16.sql new file mode 100755 index 000000000000..732ad0d84807 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q16.sql @@ -0,0 +1,23 @@ +SELECT + count(DISTINCT cs_order_number) AS `order count `, + sum(cs_ext_ship_cost) AS `total shipping cost `, + sum(cs_net_profit) AS `total net profit ` +FROM + catalog_sales cs1, date_dim, customer_address, call_center +WHERE + d_date BETWEEN '2002-02-01' AND (CAST('2002-02-01' AS DATE) + INTERVAL 60 days) + AND cs1.cs_ship_date_sk = d_date_sk + AND cs1.cs_ship_addr_sk = ca_address_sk + AND ca_state = 'GA' + AND cs1.cs_call_center_sk = cc_call_center_sk + AND cc_county IN + ('Williamson County', 'Williamson County', 'Williamson County', 'Williamson County', 'Williamson County') + AND EXISTS(SELECT * + FROM catalog_sales cs2 + WHERE cs1.cs_order_number = cs2.cs_order_number + AND cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk) + AND NOT EXISTS(SELECT * + FROM catalog_returns cr1 + WHERE cs1.cs_order_number = cr1.cr_order_number) +ORDER BY count(DISTINCT cs_order_number) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q17.sql b/sql/core/src/test/resources/tpcds/q17.sql new file mode 100755 index 000000000000..4d647f795600 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q17.sql @@ -0,0 +1,33 @@ +SELECT + i_item_id, + i_item_desc, + s_state, + count(ss_quantity) AS store_sales_quantitycount, + avg(ss_quantity) AS store_sales_quantityave, + stddev_samp(ss_quantity) AS store_sales_quantitystdev, + stddev_samp(ss_quantity) / avg(ss_quantity) AS store_sales_quantitycov, + count(sr_return_quantity) as_store_returns_quantitycount, + avg(sr_return_quantity) as_store_returns_quantityave, + stddev_samp(sr_return_quantity) as_store_returns_quantitystdev, + stddev_samp(sr_return_quantity) / avg(sr_return_quantity) AS store_returns_quantitycov, + count(cs_quantity) AS catalog_sales_quantitycount, + avg(cs_quantity) AS catalog_sales_quantityave, + stddev_samp(cs_quantity) / avg(cs_quantity) AS catalog_sales_quantitystdev, + stddev_samp(cs_quantity) / avg(cs_quantity) AS catalog_sales_quantitycov +FROM store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, date_dim d3, store, item +WHERE d1.d_quarter_name = '2001Q1' + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND ss_customer_sk = sr_customer_sk + AND ss_item_sk = sr_item_sk + AND ss_ticket_number = sr_ticket_number + AND sr_returned_date_sk = d2.d_date_sk + AND d2.d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3') + AND sr_customer_sk = cs_bill_customer_sk + AND sr_item_sk = cs_item_sk + AND cs_sold_date_sk = d3.d_date_sk + AND d3.d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3') +GROUP BY i_item_id, i_item_desc, s_state +ORDER BY i_item_id, i_item_desc, s_state +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q18.sql b/sql/core/src/test/resources/tpcds/q18.sql new file mode 100755 index 000000000000..4055c80fdef5 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q18.sql @@ -0,0 +1,28 @@ +SELECT + i_item_id, + ca_country, + ca_state, + ca_county, + avg(cast(cs_quantity AS DECIMAL(12, 2))) agg1, + avg(cast(cs_list_price AS DECIMAL(12, 2))) agg2, + avg(cast(cs_coupon_amt AS DECIMAL(12, 2))) agg3, + avg(cast(cs_sales_price AS DECIMAL(12, 2))) agg4, + avg(cast(cs_net_profit AS DECIMAL(12, 2))) agg5, + avg(cast(c_birth_year AS DECIMAL(12, 2))) agg6, + avg(cast(cd1.cd_dep_count AS DECIMAL(12, 2))) agg7 +FROM catalog_sales, customer_demographics cd1, + customer_demographics cd2, customer, customer_address, date_dim, item +WHERE cs_sold_date_sk = d_date_sk AND + cs_item_sk = i_item_sk AND + cs_bill_cdemo_sk = cd1.cd_demo_sk AND + cs_bill_customer_sk = c_customer_sk AND + cd1.cd_gender = 'F' AND + cd1.cd_education_status = 'Unknown' AND + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_addr_sk = ca_address_sk AND + c_birth_month IN (1, 6, 8, 9, 12, 2) AND + d_year = 1998 AND + ca_state IN ('MS', 'IN', 'ND', 'OK', 'NM', 'VA', 'MS') +GROUP BY ROLLUP (i_item_id, ca_country, ca_state, ca_county) +ORDER BY ca_country, ca_state, ca_county, i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q19.sql b/sql/core/src/test/resources/tpcds/q19.sql new file mode 100755 index 000000000000..e38ab7f2683f --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q19.sql @@ -0,0 +1,19 @@ +SELECT + i_brand_id brand_id, + i_brand brand, + i_manufact_id, + i_manufact, + sum(ss_ext_sales_price) ext_price +FROM date_dim, store_sales, item, customer, customer_address, store +WHERE d_date_sk = ss_sold_date_sk + AND ss_item_sk = i_item_sk + AND i_manager_id = 8 + AND d_moy = 11 + AND d_year = 1998 + AND ss_customer_sk = c_customer_sk + AND c_current_addr_sk = ca_address_sk + AND substr(ca_zip, 1, 5) <> substr(s_zip, 1, 5) + AND ss_store_sk = s_store_sk +GROUP BY i_brand, i_brand_id, i_manufact_id, i_manufact +ORDER BY ext_price DESC, brand, brand_id, i_manufact_id, i_manufact +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q2.sql b/sql/core/src/test/resources/tpcds/q2.sql new file mode 100755 index 000000000000..52c0e90c4674 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q2.sql @@ -0,0 +1,81 @@ +WITH wscs AS +( SELECT + sold_date_sk, + sales_price + FROM (SELECT + ws_sold_date_sk sold_date_sk, + ws_ext_sales_price sales_price + FROM web_sales) x + UNION ALL + (SELECT + cs_sold_date_sk sold_date_sk, + cs_ext_sales_price sales_price + FROM catalog_sales)), + wswscs AS + ( SELECT + d_week_seq, + sum(CASE WHEN (d_day_name = 'Sunday') + THEN sales_price + ELSE NULL END) + sun_sales, + sum(CASE WHEN (d_day_name = 'Monday') + THEN sales_price + ELSE NULL END) + mon_sales, + sum(CASE WHEN (d_day_name = 'Tuesday') + THEN sales_price + ELSE NULL END) + tue_sales, + sum(CASE WHEN (d_day_name = 'Wednesday') + THEN sales_price + ELSE NULL END) + wed_sales, + sum(CASE WHEN (d_day_name = 'Thursday') + THEN sales_price + ELSE NULL END) + thu_sales, + sum(CASE WHEN (d_day_name = 'Friday') + THEN sales_price + ELSE NULL END) + fri_sales, + sum(CASE WHEN (d_day_name = 'Saturday') + THEN sales_price + ELSE NULL END) + sat_sales + FROM wscs, date_dim + WHERE d_date_sk = sold_date_sk + GROUP BY d_week_seq) +SELECT + d_week_seq1, + round(sun_sales1 / sun_sales2, 2), + round(mon_sales1 / mon_sales2, 2), + round(tue_sales1 / tue_sales2, 2), + round(wed_sales1 / wed_sales2, 2), + round(thu_sales1 / thu_sales2, 2), + round(fri_sales1 / fri_sales2, 2), + round(sat_sales1 / sat_sales2, 2) +FROM + (SELECT + wswscs.d_week_seq d_week_seq1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 + FROM wswscs, date_dim + WHERE date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001) y, + (SELECT + wswscs.d_week_seq d_week_seq2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + FROM wswscs, date_dim + WHERE date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 + 1) z +WHERE d_week_seq1 = d_week_seq2 - 53 +ORDER BY d_week_seq1 diff --git a/sql/core/src/test/resources/tpcds/q20.sql b/sql/core/src/test/resources/tpcds/q20.sql new file mode 100755 index 000000000000..7ac6c7a75d8e --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q20.sql @@ -0,0 +1,18 @@ +SELECT + i_item_desc, + i_category, + i_class, + i_current_price, + sum(cs_ext_sales_price) AS itemrevenue, + sum(cs_ext_sales_price) * 100 / sum(sum(cs_ext_sales_price)) + OVER + (PARTITION BY i_class) AS revenueratio +FROM catalog_sales, item, date_dim +WHERE cs_item_sk = i_item_sk + AND i_category IN ('Sports', 'Books', 'Home') + AND cs_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('1999-02-22' AS DATE) +AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) +GROUP BY i_item_id, i_item_desc, i_category, i_class, i_current_price +ORDER BY i_category, i_class, i_item_id, i_item_desc, revenueratio +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q21.sql b/sql/core/src/test/resources/tpcds/q21.sql new file mode 100755 index 000000000000..550881143f80 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q21.sql @@ -0,0 +1,25 @@ +SELECT * +FROM ( + SELECT + w_warehouse_name, + i_item_id, + sum(CASE WHEN (cast(d_date AS DATE) < cast('2000-03-11' AS DATE)) + THEN inv_quantity_on_hand + ELSE 0 END) AS inv_before, + sum(CASE WHEN (cast(d_date AS DATE) >= cast('2000-03-11' AS DATE)) + THEN inv_quantity_on_hand + ELSE 0 END) AS inv_after + FROM inventory, warehouse, item, date_dim + WHERE i_current_price BETWEEN 0.99 AND 1.49 + AND i_item_sk = inv_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND inv_date_sk = d_date_sk + AND d_date BETWEEN (cast('2000-03-11' AS DATE) - INTERVAL 30 days) + AND (cast('2000-03-11' AS DATE) + INTERVAL 30 days) + GROUP BY w_warehouse_name, i_item_id) x +WHERE (CASE WHEN inv_before > 0 + THEN inv_after / inv_before + ELSE NULL + END) BETWEEN 2.0 / 3.0 AND 3.0 / 2.0 +ORDER BY w_warehouse_name, i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q22.sql b/sql/core/src/test/resources/tpcds/q22.sql new file mode 100755 index 000000000000..add3b41f7c76 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q22.sql @@ -0,0 +1,14 @@ +SELECT + i_product_name, + i_brand, + i_class, + i_category, + avg(inv_quantity_on_hand) qoh +FROM inventory, date_dim, item, warehouse +WHERE inv_date_sk = d_date_sk + AND inv_item_sk = i_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 +GROUP BY ROLLUP (i_product_name, i_brand, i_class, i_category) +ORDER BY qoh, i_product_name, i_brand, i_class, i_category +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q23a.sql b/sql/core/src/test/resources/tpcds/q23a.sql new file mode 100755 index 000000000000..37791f643375 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q23a.sql @@ -0,0 +1,53 @@ +WITH frequent_ss_items AS +(SELECT + substr(i_item_desc, 1, 30) itemdesc, + i_item_sk item_sk, + d_date solddate, + count(*) cnt + FROM store_sales, date_dim, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY substr(i_item_desc, 1, 30), i_item_sk, d_date + HAVING count(*) > 4), + max_store_sales AS + (SELECT max(csales) tpcds_cmax + FROM (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) csales + FROM store_sales, customer, date_dim + WHERE ss_customer_sk = c_customer_sk + AND ss_sold_date_sk = d_date_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY c_customer_sk) x), + best_ss_customer AS + (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) ssales + FROM store_sales, customer + WHERE ss_customer_sk = c_customer_sk + GROUP BY c_customer_sk + HAVING sum(ss_quantity * ss_sales_price) > (50 / 100.0) * + (SELECT * + FROM max_store_sales)) +SELECT sum(sales) +FROM ((SELECT cs_quantity * cs_list_price sales +FROM catalog_sales, date_dim +WHERE d_year = 2000 + AND d_moy = 2 + AND cs_sold_date_sk = d_date_sk + AND cs_item_sk IN (SELECT item_sk +FROM frequent_ss_items) + AND cs_bill_customer_sk IN (SELECT c_customer_sk +FROM best_ss_customer)) + UNION ALL + (SELECT ws_quantity * ws_list_price sales + FROM web_sales, date_dim + WHERE d_year = 2000 + AND d_moy = 2 + AND ws_sold_date_sk = d_date_sk + AND ws_item_sk IN (SELECT item_sk + FROM frequent_ss_items) + AND ws_bill_customer_sk IN (SELECT c_customer_sk + FROM best_ss_customer))) y +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q23b.sql b/sql/core/src/test/resources/tpcds/q23b.sql new file mode 100755 index 000000000000..01150197af2b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q23b.sql @@ -0,0 +1,68 @@ +WITH frequent_ss_items AS +(SELECT + substr(i_item_desc, 1, 30) itemdesc, + i_item_sk item_sk, + d_date solddate, + count(*) cnt + FROM store_sales, date_dim, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY substr(i_item_desc, 1, 30), i_item_sk, d_date + HAVING count(*) > 4), + max_store_sales AS + (SELECT max(csales) tpcds_cmax + FROM (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) csales + FROM store_sales, customer, date_dim + WHERE ss_customer_sk = c_customer_sk + AND ss_sold_date_sk = d_date_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY c_customer_sk) x), + best_ss_customer AS + (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) ssales + FROM store_sales + , customer + WHERE ss_customer_sk = c_customer_sk + GROUP BY c_customer_sk + HAVING sum(ss_quantity * ss_sales_price) > (50 / 100.0) * + (SELECT * + FROM max_store_sales)) +SELECT + c_last_name, + c_first_name, + sales +FROM ((SELECT + c_last_name, + c_first_name, + sum(cs_quantity * cs_list_price) sales +FROM catalog_sales, customer, date_dim +WHERE d_year = 2000 + AND d_moy = 2 + AND cs_sold_date_sk = d_date_sk + AND cs_item_sk IN (SELECT item_sk +FROM frequent_ss_items) + AND cs_bill_customer_sk IN (SELECT c_customer_sk +FROM best_ss_customer) + AND cs_bill_customer_sk = c_customer_sk +GROUP BY c_last_name, c_first_name) + UNION ALL + (SELECT + c_last_name, + c_first_name, + sum(ws_quantity * ws_list_price) sales + FROM web_sales, customer, date_dim + WHERE d_year = 2000 + AND d_moy = 2 + AND ws_sold_date_sk = d_date_sk + AND ws_item_sk IN (SELECT item_sk + FROM frequent_ss_items) + AND ws_bill_customer_sk IN (SELECT c_customer_sk + FROM best_ss_customer) + AND ws_bill_customer_sk = c_customer_sk + GROUP BY c_last_name, c_first_name)) y +ORDER BY c_last_name, c_first_name, sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q24a.sql b/sql/core/src/test/resources/tpcds/q24a.sql new file mode 100755 index 000000000000..bcc189486634 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q24a.sql @@ -0,0 +1,34 @@ +WITH ssales AS +(SELECT + c_last_name, + c_first_name, + s_store_name, + ca_state, + s_state, + i_color, + i_current_price, + i_manager_id, + i_units, + i_size, + sum(ss_net_paid) netpaid + FROM store_sales, store_returns, store, item, customer, customer_address + WHERE ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk + AND ss_customer_sk = c_customer_sk + AND ss_item_sk = i_item_sk + AND ss_store_sk = s_store_sk + AND c_birth_country = upper(ca_country) + AND s_zip = ca_zip + AND s_market_id = 8 + GROUP BY c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, + i_current_price, i_manager_id, i_units, i_size) +SELECT + c_last_name, + c_first_name, + s_store_name, + sum(netpaid) paid +FROM ssales +WHERE i_color = 'pale' +GROUP BY c_last_name, c_first_name, s_store_name +HAVING sum(netpaid) > (SELECT 0.05 * avg(netpaid) +FROM ssales) diff --git a/sql/core/src/test/resources/tpcds/q24b.sql b/sql/core/src/test/resources/tpcds/q24b.sql new file mode 100755 index 000000000000..830eb670bcdd --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q24b.sql @@ -0,0 +1,34 @@ +WITH ssales AS +(SELECT + c_last_name, + c_first_name, + s_store_name, + ca_state, + s_state, + i_color, + i_current_price, + i_manager_id, + i_units, + i_size, + sum(ss_net_paid) netpaid + FROM store_sales, store_returns, store, item, customer, customer_address + WHERE ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk + AND ss_customer_sk = c_customer_sk + AND ss_item_sk = i_item_sk + AND ss_store_sk = s_store_sk + AND c_birth_country = upper(ca_country) + AND s_zip = ca_zip + AND s_market_id = 8 + GROUP BY c_last_name, c_first_name, s_store_name, ca_state, s_state, + i_color, i_current_price, i_manager_id, i_units, i_size) +SELECT + c_last_name, + c_first_name, + s_store_name, + sum(netpaid) paid +FROM ssales +WHERE i_color = 'chiffon' +GROUP BY c_last_name, c_first_name, s_store_name +HAVING sum(netpaid) > (SELECT 0.05 * avg(netpaid) +FROM ssales) diff --git a/sql/core/src/test/resources/tpcds/q25.sql b/sql/core/src/test/resources/tpcds/q25.sql new file mode 100755 index 000000000000..a4d78a3c56ad --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q25.sql @@ -0,0 +1,33 @@ +SELECT + i_item_id, + i_item_desc, + s_store_id, + s_store_name, + sum(ss_net_profit) AS store_sales_profit, + sum(sr_net_loss) AS store_returns_loss, + sum(cs_net_profit) AS catalog_sales_profit +FROM + store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, date_dim d3, + store, item +WHERE + d1.d_moy = 4 + AND d1.d_year = 2001 + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND ss_customer_sk = sr_customer_sk + AND ss_item_sk = sr_item_sk + AND ss_ticket_number = sr_ticket_number + AND sr_returned_date_sk = d2.d_date_sk + AND d2.d_moy BETWEEN 4 AND 10 + AND d2.d_year = 2001 + AND sr_customer_sk = cs_bill_customer_sk + AND sr_item_sk = cs_item_sk + AND cs_sold_date_sk = d3.d_date_sk + AND d3.d_moy BETWEEN 4 AND 10 + AND d3.d_year = 2001 +GROUP BY + i_item_id, i_item_desc, s_store_id, s_store_name +ORDER BY + i_item_id, i_item_desc, s_store_id, s_store_name +LIMIT 100 \ No newline at end of file diff --git a/sql/core/src/test/resources/tpcds/q26.sql b/sql/core/src/test/resources/tpcds/q26.sql new file mode 100755 index 000000000000..6d395a1d791d --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q26.sql @@ -0,0 +1,19 @@ +SELECT + i_item_id, + avg(cs_quantity) agg1, + avg(cs_list_price) agg2, + avg(cs_coupon_amt) agg3, + avg(cs_sales_price) agg4 +FROM catalog_sales, customer_demographics, date_dim, item, promotion +WHERE cs_sold_date_sk = d_date_sk AND + cs_item_sk = i_item_sk AND + cs_bill_cdemo_sk = cd_demo_sk AND + cs_promo_sk = p_promo_sk AND + cd_gender = 'M' AND + cd_marital_status = 'S' AND + cd_education_status = 'College' AND + (p_channel_email = 'N' OR p_channel_event = 'N') AND + d_year = 2000 +GROUP BY i_item_id +ORDER BY i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q27.sql b/sql/core/src/test/resources/tpcds/q27.sql new file mode 100755 index 000000000000..b0e2fd95fd15 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q27.sql @@ -0,0 +1,21 @@ +SELECT + i_item_id, + s_state, + grouping(s_state) g_state, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 +FROM store_sales, customer_demographics, date_dim, store, item +WHERE ss_sold_date_sk = d_date_sk AND + ss_item_sk = i_item_sk AND + ss_store_sk = s_store_sk AND + ss_cdemo_sk = cd_demo_sk AND + cd_gender = 'M' AND + cd_marital_status = 'S' AND + cd_education_status = 'College' AND + d_year = 2002 AND + s_state IN ('TN', 'TN', 'TN', 'TN', 'TN', 'TN') +GROUP BY ROLLUP (i_item_id, s_state) +ORDER BY i_item_id, s_state +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q28.sql b/sql/core/src/test/resources/tpcds/q28.sql new file mode 100755 index 000000000000..f34c2bb0e34e --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q28.sql @@ -0,0 +1,56 @@ +SELECT * +FROM (SELECT + avg(ss_list_price) B1_LP, + count(ss_list_price) B1_CNT, + count(DISTINCT ss_list_price) B1_CNTD +FROM store_sales +WHERE ss_quantity BETWEEN 0 AND 5 + AND (ss_list_price BETWEEN 8 AND 8 + 10 + OR ss_coupon_amt BETWEEN 459 AND 459 + 1000 + OR ss_wholesale_cost BETWEEN 57 AND 57 + 20)) B1, + (SELECT + avg(ss_list_price) B2_LP, + count(ss_list_price) B2_CNT, + count(DISTINCT ss_list_price) B2_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 6 AND 10 + AND (ss_list_price BETWEEN 90 AND 90 + 10 + OR ss_coupon_amt BETWEEN 2323 AND 2323 + 1000 + OR ss_wholesale_cost BETWEEN 31 AND 31 + 20)) B2, + (SELECT + avg(ss_list_price) B3_LP, + count(ss_list_price) B3_CNT, + count(DISTINCT ss_list_price) B3_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 11 AND 15 + AND (ss_list_price BETWEEN 142 AND 142 + 10 + OR ss_coupon_amt BETWEEN 12214 AND 12214 + 1000 + OR ss_wholesale_cost BETWEEN 79 AND 79 + 20)) B3, + (SELECT + avg(ss_list_price) B4_LP, + count(ss_list_price) B4_CNT, + count(DISTINCT ss_list_price) B4_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 16 AND 20 + AND (ss_list_price BETWEEN 135 AND 135 + 10 + OR ss_coupon_amt BETWEEN 6071 AND 6071 + 1000 + OR ss_wholesale_cost BETWEEN 38 AND 38 + 20)) B4, + (SELECT + avg(ss_list_price) B5_LP, + count(ss_list_price) B5_CNT, + count(DISTINCT ss_list_price) B5_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 25 + AND (ss_list_price BETWEEN 122 AND 122 + 10 + OR ss_coupon_amt BETWEEN 836 AND 836 + 1000 + OR ss_wholesale_cost BETWEEN 17 AND 17 + 20)) B5, + (SELECT + avg(ss_list_price) B6_LP, + count(ss_list_price) B6_CNT, + count(DISTINCT ss_list_price) B6_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 26 AND 30 + AND (ss_list_price BETWEEN 154 AND 154 + 10 + OR ss_coupon_amt BETWEEN 7326 AND 7326 + 1000 + OR ss_wholesale_cost BETWEEN 7 AND 7 + 20)) B6 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q29.sql b/sql/core/src/test/resources/tpcds/q29.sql new file mode 100755 index 000000000000..3f1fd553f6da --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q29.sql @@ -0,0 +1,32 @@ +SELECT + i_item_id, + i_item_desc, + s_store_id, + s_store_name, + sum(ss_quantity) AS store_sales_quantity, + sum(sr_return_quantity) AS store_returns_quantity, + sum(cs_quantity) AS catalog_sales_quantity +FROM + store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, + date_dim d3, store, item +WHERE + d1.d_moy = 9 + AND d1.d_year = 1999 + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND ss_customer_sk = sr_customer_sk + AND ss_item_sk = sr_item_sk + AND ss_ticket_number = sr_ticket_number + AND sr_returned_date_sk = d2.d_date_sk + AND d2.d_moy BETWEEN 9 AND 9 + 3 + AND d2.d_year = 1999 + AND sr_customer_sk = cs_bill_customer_sk + AND sr_item_sk = cs_item_sk + AND cs_sold_date_sk = d3.d_date_sk + AND d3.d_year IN (1999, 1999 + 1, 1999 + 2) +GROUP BY + i_item_id, i_item_desc, s_store_id, s_store_name +ORDER BY + i_item_id, i_item_desc, s_store_id, s_store_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q3.sql b/sql/core/src/test/resources/tpcds/q3.sql new file mode 100755 index 000000000000..181509df9deb --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q3.sql @@ -0,0 +1,13 @@ +SELECT + dt.d_year, + item.i_brand_id brand_id, + item.i_brand brand, + SUM(ss_ext_sales_price) sum_agg +FROM date_dim dt, store_sales, item +WHERE dt.d_date_sk = store_sales.ss_sold_date_sk + AND store_sales.ss_item_sk = item.i_item_sk + AND item.i_manufact_id = 128 + AND dt.d_moy = 11 +GROUP BY dt.d_year, item.i_brand, item.i_brand_id +ORDER BY dt.d_year, sum_agg DESC, brand_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q30.sql b/sql/core/src/test/resources/tpcds/q30.sql new file mode 100755 index 000000000000..986bef566d2c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q30.sql @@ -0,0 +1,35 @@ +WITH customer_total_return AS +(SELECT + wr_returning_customer_sk AS ctr_customer_sk, + ca_state AS ctr_state, + sum(wr_return_amt) AS ctr_total_return + FROM web_returns, date_dim, customer_address + WHERE wr_returned_date_sk = d_date_sk + AND d_year = 2002 + AND wr_returning_addr_sk = ca_address_sk + GROUP BY wr_returning_customer_sk, ca_state) +SELECT + c_customer_id, + c_salutation, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_day, + c_birth_month, + c_birth_year, + c_birth_country, + c_login, + c_email_address, + c_last_review_date, + ctr_total_return +FROM customer_total_return ctr1, customer_address, customer +WHERE ctr1.ctr_total_return > (SELECT avg(ctr_total_return) * 1.2 +FROM customer_total_return ctr2 +WHERE ctr1.ctr_state = ctr2.ctr_state) + AND ca_address_sk = c_current_addr_sk + AND ca_state = 'GA' + AND ctr1.ctr_customer_sk = c_customer_sk +ORDER BY c_customer_id, c_salutation, c_first_name, c_last_name, c_preferred_cust_flag + , c_birth_day, c_birth_month, c_birth_year, c_birth_country, c_login, c_email_address + , c_last_review_date, ctr_total_return +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q31.sql b/sql/core/src/test/resources/tpcds/q31.sql new file mode 100755 index 000000000000..3e543d543640 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q31.sql @@ -0,0 +1,60 @@ +WITH ss AS +(SELECT + ca_county, + d_qoy, + d_year, + sum(ss_ext_sales_price) AS store_sales + FROM store_sales, date_dim, customer_address + WHERE ss_sold_date_sk = d_date_sk + AND ss_addr_sk = ca_address_sk + GROUP BY ca_county, d_qoy, d_year), + ws AS + (SELECT + ca_county, + d_qoy, + d_year, + sum(ws_ext_sales_price) AS web_sales + FROM web_sales, date_dim, customer_address + WHERE ws_sold_date_sk = d_date_sk + AND ws_bill_addr_sk = ca_address_sk + GROUP BY ca_county, d_qoy, d_year) +SELECT + ss1.ca_county, + ss1.d_year, + ws2.web_sales / ws1.web_sales web_q1_q2_increase, + ss2.store_sales / ss1.store_sales store_q1_q2_increase, + ws3.web_sales / ws2.web_sales web_q2_q3_increase, + ss3.store_sales / ss2.store_sales store_q2_q3_increase +FROM + ss ss1, ss ss2, ss ss3, ws ws1, ws ws2, ws ws3 +WHERE + ss1.d_qoy = 1 + AND ss1.d_year = 2000 + AND ss1.ca_county = ss2.ca_county + AND ss2.d_qoy = 2 + AND ss2.d_year = 2000 + AND ss2.ca_county = ss3.ca_county + AND ss3.d_qoy = 3 + AND ss3.d_year = 2000 + AND ss1.ca_county = ws1.ca_county + AND ws1.d_qoy = 1 + AND ws1.d_year = 2000 + AND ws1.ca_county = ws2.ca_county + AND ws2.d_qoy = 2 + AND ws2.d_year = 2000 + AND ws1.ca_county = ws3.ca_county + AND ws3.d_qoy = 3 + AND ws3.d_year = 2000 + AND CASE WHEN ws1.web_sales > 0 + THEN ws2.web_sales / ws1.web_sales + ELSE NULL END + > CASE WHEN ss1.store_sales > 0 + THEN ss2.store_sales / ss1.store_sales + ELSE NULL END + AND CASE WHEN ws2.web_sales > 0 + THEN ws3.web_sales / ws2.web_sales + ELSE NULL END + > CASE WHEN ss2.store_sales > 0 + THEN ss3.store_sales / ss2.store_sales + ELSE NULL END +ORDER BY ss1.ca_county diff --git a/sql/core/src/test/resources/tpcds/q32.sql b/sql/core/src/test/resources/tpcds/q32.sql new file mode 100755 index 000000000000..1a907961e74b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q32.sql @@ -0,0 +1,15 @@ +SELECT 1 AS `excess discount amount ` +FROM + catalog_sales, item, date_dim +WHERE + i_manufact_id = 977 + AND i_item_sk = cs_item_sk + AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + interval 90 days) + AND d_date_sk = cs_sold_date_sk + AND cs_ext_discount_amt > ( + SELECT 1.3 * avg(cs_ext_discount_amt) + FROM catalog_sales, date_dim + WHERE cs_item_sk = i_item_sk + AND d_date BETWEEN '2000-01-27]' AND (cast('2000-01-27' AS DATE) + interval 90 days) + AND d_date_sk = cs_sold_date_sk) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q33.sql b/sql/core/src/test/resources/tpcds/q33.sql new file mode 100755 index 000000000000..d24856aa5c1e --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q33.sql @@ -0,0 +1,65 @@ +WITH ss AS ( + SELECT + i_manufact_id, + sum(ss_ext_sales_price) total_sales + FROM + store_sales, date_dim, customer_address, item + WHERE + i_manufact_id IN (SELECT i_manufact_id + FROM item + WHERE i_category IN ('Electronics')) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 5 + AND ss_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_manufact_id), cs AS +(SELECT + i_manufact_id, + sum(cs_ext_sales_price) total_sales + FROM catalog_sales, date_dim, customer_address, item + WHERE + i_manufact_id IN ( + SELECT i_manufact_id + FROM item + WHERE + i_category IN ('Electronics')) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 5 + AND cs_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_manufact_id), + ws AS ( + SELECT + i_manufact_id, + sum(ws_ext_sales_price) total_sales + FROM + web_sales, date_dim, customer_address, item + WHERE + i_manufact_id IN (SELECT i_manufact_id + FROM item + WHERE i_category IN ('Electronics')) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 5 + AND ws_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_manufact_id) +SELECT + i_manufact_id, + sum(total_sales) total_sales +FROM (SELECT * + FROM ss + UNION ALL + SELECT * + FROM cs + UNION ALL + SELECT * + FROM ws) tmp1 +GROUP BY i_manufact_id +ORDER BY total_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q34.sql b/sql/core/src/test/resources/tpcds/q34.sql new file mode 100755 index 000000000000..33396bf16e57 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q34.sql @@ -0,0 +1,32 @@ +SELECT + c_last_name, + c_first_name, + c_salutation, + c_preferred_cust_flag, + ss_ticket_number, + cnt +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + count(*) cnt + FROM store_sales, date_dim, store, household_demographics + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND (date_dim.d_dom BETWEEN 1 AND 3 OR date_dim.d_dom BETWEEN 25 AND 28) + AND (household_demographics.hd_buy_potential = '>10000' OR + household_demographics.hd_buy_potential = 'unknown') + AND household_demographics.hd_vehicle_count > 0 + AND (CASE WHEN household_demographics.hd_vehicle_count > 0 + THEN household_demographics.hd_dep_count / household_demographics.hd_vehicle_count + ELSE NULL + END) > 1.2 + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_county IN + ('Williamson County', 'Williamson County', 'Williamson County', 'Williamson County', + 'Williamson County', 'Williamson County', 'Williamson County', 'Williamson County') + GROUP BY ss_ticket_number, ss_customer_sk) dn, customer +WHERE ss_customer_sk = c_customer_sk + AND cnt BETWEEN 15 AND 20 +ORDER BY c_last_name, c_first_name, c_salutation, c_preferred_cust_flag DESC diff --git a/sql/core/src/test/resources/tpcds/q35.sql b/sql/core/src/test/resources/tpcds/q35.sql new file mode 100755 index 000000000000..cfe4342d8be8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q35.sql @@ -0,0 +1,46 @@ +SELECT + ca_state, + cd_gender, + cd_marital_status, + count(*) cnt1, + min(cd_dep_count), + max(cd_dep_count), + avg(cd_dep_count), + cd_dep_employed_count, + count(*) cnt2, + min(cd_dep_employed_count), + max(cd_dep_employed_count), + avg(cd_dep_employed_count), + cd_dep_college_count, + count(*) cnt3, + min(cd_dep_college_count), + max(cd_dep_college_count), + avg(cd_dep_college_count) +FROM + customer c, customer_address ca, customer_demographics +WHERE + c.c_current_addr_sk = ca.ca_address_sk AND + cd_demo_sk = c.c_current_cdemo_sk AND + exists(SELECT * + FROM store_sales, date_dim + WHERE c.c_customer_sk = ss_customer_sk AND + ss_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_qoy < 4) AND + (exists(SELECT * + FROM web_sales, date_dim + WHERE c.c_customer_sk = ws_bill_customer_sk AND + ws_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_qoy < 4) OR + exists(SELECT * + FROM catalog_sales, date_dim + WHERE c.c_customer_sk = cs_ship_customer_sk AND + cs_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_qoy < 4)) +GROUP BY ca_state, cd_gender, cd_marital_status, cd_dep_count, + cd_dep_employed_count, cd_dep_college_count +ORDER BY ca_state, cd_gender, cd_marital_status, cd_dep_count, + cd_dep_employed_count, cd_dep_college_count +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q36.sql b/sql/core/src/test/resources/tpcds/q36.sql new file mode 100755 index 000000000000..a8f93df76a34 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q36.sql @@ -0,0 +1,26 @@ +SELECT + sum(ss_net_profit) / sum(ss_ext_sales_price) AS gross_margin, + i_category, + i_class, + grouping(i_category) + grouping(i_class) AS lochierarchy, + rank() + OVER ( + PARTITION BY grouping(i_category) + grouping(i_class), + CASE WHEN grouping(i_class) = 0 + THEN i_category END + ORDER BY sum(ss_net_profit) / sum(ss_ext_sales_price) ASC) AS rank_within_parent +FROM + store_sales, date_dim d1, item, store +WHERE + d1.d_year = 2001 + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND s_state IN ('TN', 'TN', 'TN', 'TN', 'TN', 'TN', 'TN', 'TN') +GROUP BY ROLLUP (i_category, i_class) +ORDER BY + lochierarchy DESC + , CASE WHEN lochierarchy = 0 + THEN i_category END + , rank_within_parent +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q37.sql b/sql/core/src/test/resources/tpcds/q37.sql new file mode 100755 index 000000000000..11b3821fa48b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q37.sql @@ -0,0 +1,15 @@ +SELECT + i_item_id, + i_item_desc, + i_current_price +FROM item, inventory, date_dim, catalog_sales +WHERE i_current_price BETWEEN 68 AND 68 + 30 + AND inv_item_sk = i_item_sk + AND d_date_sk = inv_date_sk + AND d_date BETWEEN cast('2000-02-01' AS DATE) AND (cast('2000-02-01' AS DATE) + INTERVAL 60 days) + AND i_manufact_id IN (677, 940, 694, 808) + AND inv_quantity_on_hand BETWEEN 100 AND 500 + AND cs_item_sk = i_item_sk +GROUP BY i_item_id, i_item_desc, i_current_price +ORDER BY i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q38.sql b/sql/core/src/test/resources/tpcds/q38.sql new file mode 100755 index 000000000000..1c8d53ee2bbf --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q38.sql @@ -0,0 +1,30 @@ +SELECT count(*) +FROM ( + SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM store_sales, date_dim, customer + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + INTERSECT + SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM catalog_sales, date_dim, customer + WHERE catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + INTERSECT + SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM web_sales, date_dim, customer + WHERE web_sales.ws_sold_date_sk = date_dim.d_date_sk + AND web_sales.ws_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + ) hot_cust +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q39a.sql b/sql/core/src/test/resources/tpcds/q39a.sql new file mode 100755 index 000000000000..9fc4c1701cf2 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q39a.sql @@ -0,0 +1,47 @@ +WITH inv AS +(SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stdev, + mean, + CASE mean + WHEN 0 + THEN NULL + ELSE stdev / mean END cov + FROM (SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stddev_samp(inv_quantity_on_hand) stdev, + avg(inv_quantity_on_hand) mean + FROM inventory, item, warehouse, date_dim + WHERE inv_item_sk = i_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND inv_date_sk = d_date_sk + AND d_year = 2001 + GROUP BY w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo + WHERE CASE mean + WHEN 0 + THEN 0 + ELSE stdev / mean END > 1) +SELECT + inv1.w_warehouse_sk, + inv1.i_item_sk, + inv1.d_moy, + inv1.mean, + inv1.cov, + inv2.w_warehouse_sk, + inv2.i_item_sk, + inv2.d_moy, + inv2.mean, + inv2.cov +FROM inv inv1, inv inv2 +WHERE inv1.i_item_sk = inv2.i_item_sk + AND inv1.w_warehouse_sk = inv2.w_warehouse_sk + AND inv1.d_moy = 1 + AND inv2.d_moy = 1 + 1 +ORDER BY inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov + , inv2.d_moy, inv2.mean, inv2.cov diff --git a/sql/core/src/test/resources/tpcds/q39b.sql b/sql/core/src/test/resources/tpcds/q39b.sql new file mode 100755 index 000000000000..6f8493029fab --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q39b.sql @@ -0,0 +1,48 @@ +WITH inv AS +(SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stdev, + mean, + CASE mean + WHEN 0 + THEN NULL + ELSE stdev / mean END cov + FROM (SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stddev_samp(inv_quantity_on_hand) stdev, + avg(inv_quantity_on_hand) mean + FROM inventory, item, warehouse, date_dim + WHERE inv_item_sk = i_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND inv_date_sk = d_date_sk + AND d_year = 2001 + GROUP BY w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo + WHERE CASE mean + WHEN 0 + THEN 0 + ELSE stdev / mean END > 1) +SELECT + inv1.w_warehouse_sk, + inv1.i_item_sk, + inv1.d_moy, + inv1.mean, + inv1.cov, + inv2.w_warehouse_sk, + inv2.i_item_sk, + inv2.d_moy, + inv2.mean, + inv2.cov +FROM inv inv1, inv inv2 +WHERE inv1.i_item_sk = inv2.i_item_sk + AND inv1.w_warehouse_sk = inv2.w_warehouse_sk + AND inv1.d_moy = 1 + AND inv2.d_moy = 1 + 1 + AND inv1.cov > 1.5 +ORDER BY inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov + , inv2.d_moy, inv2.mean, inv2.cov diff --git a/sql/core/src/test/resources/tpcds/q4.sql b/sql/core/src/test/resources/tpcds/q4.sql new file mode 100755 index 000000000000..b9f27fbc9a4a --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q4.sql @@ -0,0 +1,120 @@ +WITH year_total AS ( + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + + ss_ext_sales_price) / 2) year_total, + 's' sale_type + FROM customer, store_sales, date_dim + WHERE c_customer_sk = ss_customer_sk AND ss_sold_date_sk = d_date_sk + GROUP BY c_customer_id, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_country, + c_login, + c_email_address, + d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + + cs_ext_sales_price) / 2)) year_total, + 'c' sale_type + FROM customer, catalog_sales, date_dim + WHERE c_customer_sk = cs_bill_customer_sk AND cs_sold_date_sk = d_date_sk + GROUP BY c_customer_id, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_country, + c_login, + c_email_address, + d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / + 2)) year_total, + 'w' sale_type + FROM customer, web_sales, date_dim + WHERE c_customer_sk = ws_bill_customer_sk AND ws_sold_date_sk = d_date_sk + GROUP BY c_customer_id, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_country, + c_login, + c_email_address, + d_year) +SELECT + t_s_secyear.customer_id, + t_s_secyear.customer_first_name, + t_s_secyear.customer_last_name, + t_s_secyear.customer_preferred_cust_flag, + t_s_secyear.customer_birth_country, + t_s_secyear.customer_login, + t_s_secyear.customer_email_address +FROM year_total t_s_firstyear, year_total t_s_secyear, year_total t_c_firstyear, + year_total t_c_secyear, year_total t_w_firstyear, year_total t_w_secyear +WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id + AND t_s_firstyear.customer_id = t_c_secyear.customer_id + AND t_s_firstyear.customer_id = t_c_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_secyear.customer_id + AND t_s_firstyear.sale_type = 's' + AND t_c_firstyear.sale_type = 'c' + AND t_w_firstyear.sale_type = 'w' + AND t_s_secyear.sale_type = 's' + AND t_c_secyear.sale_type = 'c' + AND t_w_secyear.sale_type = 'w' + AND t_s_firstyear.dyear = 2001 + AND t_s_secyear.dyear = 2001 + 1 + AND t_c_firstyear.dyear = 2001 + AND t_c_secyear.dyear = 2001 + 1 + AND t_w_firstyear.dyear = 2001 + AND t_w_secyear.dyear = 2001 + 1 + AND t_s_firstyear.year_total > 0 + AND t_c_firstyear.year_total > 0 + AND t_w_firstyear.year_total > 0 + AND CASE WHEN t_c_firstyear.year_total > 0 + THEN t_c_secyear.year_total / t_c_firstyear.year_total + ELSE NULL END + > CASE WHEN t_s_firstyear.year_total > 0 + THEN t_s_secyear.year_total / t_s_firstyear.year_total + ELSE NULL END + AND CASE WHEN t_c_firstyear.year_total > 0 + THEN t_c_secyear.year_total / t_c_firstyear.year_total + ELSE NULL END + > CASE WHEN t_w_firstyear.year_total > 0 + THEN t_w_secyear.year_total / t_w_firstyear.year_total + ELSE NULL END +ORDER BY + t_s_secyear.customer_id, + t_s_secyear.customer_first_name, + t_s_secyear.customer_last_name, + t_s_secyear.customer_preferred_cust_flag, + t_s_secyear.customer_birth_country, + t_s_secyear.customer_login, + t_s_secyear.customer_email_address +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q40.sql b/sql/core/src/test/resources/tpcds/q40.sql new file mode 100755 index 000000000000..66d8b73ac1c1 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q40.sql @@ -0,0 +1,25 @@ +SELECT + w_state, + i_item_id, + sum(CASE WHEN (cast(d_date AS DATE) < cast('2000-03-11' AS DATE)) + THEN cs_sales_price - coalesce(cr_refunded_cash, 0) + ELSE 0 END) AS sales_before, + sum(CASE WHEN (cast(d_date AS DATE) >= cast('2000-03-11' AS DATE)) + THEN cs_sales_price - coalesce(cr_refunded_cash, 0) + ELSE 0 END) AS sales_after +FROM + catalog_sales + LEFT OUTER JOIN catalog_returns ON + (cs_order_number = cr_order_number + AND cs_item_sk = cr_item_sk) + , warehouse, item, date_dim +WHERE + i_current_price BETWEEN 0.99 AND 1.49 + AND i_item_sk = cs_item_sk + AND cs_warehouse_sk = w_warehouse_sk + AND cs_sold_date_sk = d_date_sk + AND d_date BETWEEN (cast('2000-03-11' AS DATE) - INTERVAL 30 days) + AND (cast('2000-03-11' AS DATE) + INTERVAL 30 days) +GROUP BY w_state, i_item_id +ORDER BY w_state, i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q41.sql b/sql/core/src/test/resources/tpcds/q41.sql new file mode 100755 index 000000000000..25e317e0e201 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q41.sql @@ -0,0 +1,49 @@ +SELECT DISTINCT (i_product_name) +FROM item i1 +WHERE i_manufact_id BETWEEN 738 AND 738 + 40 + AND (SELECT count(*) AS item_cnt +FROM item +WHERE (i_manufact = i1.i_manufact AND + ((i_category = 'Women' AND + (i_color = 'powder' OR i_color = 'khaki') AND + (i_units = 'Ounce' OR i_units = 'Oz') AND + (i_size = 'medium' OR i_size = 'extra large') + ) OR + (i_category = 'Women' AND + (i_color = 'brown' OR i_color = 'honeydew') AND + (i_units = 'Bunch' OR i_units = 'Ton') AND + (i_size = 'N/A' OR i_size = 'small') + ) OR + (i_category = 'Men' AND + (i_color = 'floral' OR i_color = 'deep') AND + (i_units = 'N/A' OR i_units = 'Dozen') AND + (i_size = 'petite' OR i_size = 'large') + ) OR + (i_category = 'Men' AND + (i_color = 'light' OR i_color = 'cornflower') AND + (i_units = 'Box' OR i_units = 'Pound') AND + (i_size = 'medium' OR i_size = 'extra large') + ))) OR + (i_manufact = i1.i_manufact AND + ((i_category = 'Women' AND + (i_color = 'midnight' OR i_color = 'snow') AND + (i_units = 'Pallet' OR i_units = 'Gross') AND + (i_size = 'medium' OR i_size = 'extra large') + ) OR + (i_category = 'Women' AND + (i_color = 'cyan' OR i_color = 'papaya') AND + (i_units = 'Cup' OR i_units = 'Dram') AND + (i_size = 'N/A' OR i_size = 'small') + ) OR + (i_category = 'Men' AND + (i_color = 'orange' OR i_color = 'frosted') AND + (i_units = 'Each' OR i_units = 'Tbl') AND + (i_size = 'petite' OR i_size = 'large') + ) OR + (i_category = 'Men' AND + (i_color = 'forest' OR i_color = 'ghost') AND + (i_units = 'Lb' OR i_units = 'Bundle') AND + (i_size = 'medium' OR i_size = 'extra large') + )))) > 0 +ORDER BY i_product_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q42.sql b/sql/core/src/test/resources/tpcds/q42.sql new file mode 100755 index 000000000000..4d2e71760d87 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q42.sql @@ -0,0 +1,18 @@ +SELECT + dt.d_year, + item.i_category_id, + item.i_category, + sum(ss_ext_sales_price) +FROM date_dim dt, store_sales, item +WHERE dt.d_date_sk = store_sales.ss_sold_date_sk + AND store_sales.ss_item_sk = item.i_item_sk + AND item.i_manager_id = 1 + AND dt.d_moy = 11 + AND dt.d_year = 2000 +GROUP BY dt.d_year + , item.i_category_id + , item.i_category +ORDER BY sum(ss_ext_sales_price) DESC, dt.d_year + , item.i_category_id + , item.i_category +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q43.sql b/sql/core/src/test/resources/tpcds/q43.sql new file mode 100755 index 000000000000..45411772c1b5 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q43.sql @@ -0,0 +1,33 @@ +SELECT + s_store_name, + s_store_id, + sum(CASE WHEN (d_day_name = 'Sunday') + THEN ss_sales_price + ELSE NULL END) sun_sales, + sum(CASE WHEN (d_day_name = 'Monday') + THEN ss_sales_price + ELSE NULL END) mon_sales, + sum(CASE WHEN (d_day_name = 'Tuesday') + THEN ss_sales_price + ELSE NULL END) tue_sales, + sum(CASE WHEN (d_day_name = 'Wednesday') + THEN ss_sales_price + ELSE NULL END) wed_sales, + sum(CASE WHEN (d_day_name = 'Thursday') + THEN ss_sales_price + ELSE NULL END) thu_sales, + sum(CASE WHEN (d_day_name = 'Friday') + THEN ss_sales_price + ELSE NULL END) fri_sales, + sum(CASE WHEN (d_day_name = 'Saturday') + THEN ss_sales_price + ELSE NULL END) sat_sales +FROM date_dim, store_sales, store +WHERE d_date_sk = ss_sold_date_sk AND + s_store_sk = ss_store_sk AND + s_gmt_offset = -5 AND + d_year = 2000 +GROUP BY s_store_name, s_store_id +ORDER BY s_store_name, s_store_id, sun_sales, mon_sales, tue_sales, wed_sales, + thu_sales, fri_sales, sat_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q44.sql b/sql/core/src/test/resources/tpcds/q44.sql new file mode 100755 index 000000000000..379e60478862 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q44.sql @@ -0,0 +1,46 @@ +SELECT + asceding.rnk, + i1.i_product_name best_performing, + i2.i_product_name worst_performing +FROM (SELECT * +FROM (SELECT + item_sk, + rank() + OVER ( + ORDER BY rank_col ASC) rnk +FROM (SELECT + ss_item_sk item_sk, + avg(ss_net_profit) rank_col +FROM store_sales ss1 +WHERE ss_store_sk = 4 +GROUP BY ss_item_sk +HAVING avg(ss_net_profit) > 0.9 * (SELECT avg(ss_net_profit) rank_col +FROM store_sales +WHERE ss_store_sk = 4 + AND ss_addr_sk IS NULL +GROUP BY ss_store_sk)) V1) V11 +WHERE rnk < 11) asceding, + (SELECT * + FROM (SELECT + item_sk, + rank() + OVER ( + ORDER BY rank_col DESC) rnk + FROM (SELECT + ss_item_sk item_sk, + avg(ss_net_profit) rank_col + FROM store_sales ss1 + WHERE ss_store_sk = 4 + GROUP BY ss_item_sk + HAVING avg(ss_net_profit) > 0.9 * (SELECT avg(ss_net_profit) rank_col + FROM store_sales + WHERE ss_store_sk = 4 + AND ss_addr_sk IS NULL + GROUP BY ss_store_sk)) V2) V21 + WHERE rnk < 11) descending, + item i1, item i2 +WHERE asceding.rnk = descending.rnk + AND i1.i_item_sk = asceding.item_sk + AND i2.i_item_sk = descending.item_sk +ORDER BY asceding.rnk +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q45.sql b/sql/core/src/test/resources/tpcds/q45.sql new file mode 100755 index 000000000000..907438f196c4 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q45.sql @@ -0,0 +1,21 @@ +SELECT + ca_zip, + ca_city, + sum(ws_sales_price) +FROM web_sales, customer, customer_address, date_dim, item +WHERE ws_bill_customer_sk = c_customer_sk + AND c_current_addr_sk = ca_address_sk + AND ws_item_sk = i_item_sk + AND (substr(ca_zip, 1, 5) IN + ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') + OR + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_item_sk IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) + ) +) + AND ws_sold_date_sk = d_date_sk + AND d_qoy = 2 AND d_year = 2001 +GROUP BY ca_zip, ca_city +ORDER BY ca_zip, ca_city +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q46.sql b/sql/core/src/test/resources/tpcds/q46.sql new file mode 100755 index 000000000000..0911677dff20 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q46.sql @@ -0,0 +1,32 @@ +SELECT + c_last_name, + c_first_name, + ca_city, + bought_city, + ss_ticket_number, + amt, + profit +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + ca_city bought_city, + sum(ss_coupon_amt) amt, + sum(ss_net_profit) profit + FROM store_sales, date_dim, store, household_demographics, customer_address + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND store_sales.ss_addr_sk = customer_address.ca_address_sk + AND (household_demographics.hd_dep_count = 4 OR + household_demographics.hd_vehicle_count = 3) + AND date_dim.d_dow IN (6, 0) + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_city IN ('Fairview', 'Midway', 'Fairview', 'Fairview', 'Fairview') + GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city) dn, customer, + customer_address current_addr +WHERE ss_customer_sk = c_customer_sk + AND customer.c_current_addr_sk = current_addr.ca_address_sk + AND current_addr.ca_city <> bought_city +ORDER BY c_last_name, c_first_name, ca_city, bought_city, ss_ticket_number +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q47.sql b/sql/core/src/test/resources/tpcds/q47.sql new file mode 100755 index 000000000000..cfc37a4cece6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q47.sql @@ -0,0 +1,63 @@ +WITH v1 AS ( + SELECT + i_category, + i_brand, + s_store_name, + s_company_name, + d_year, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER + (PARTITION BY i_category, i_brand, + s_store_name, s_company_name, d_year) + avg_monthly_sales, + rank() + OVER + (PARTITION BY i_category, i_brand, + s_store_name, s_company_name + ORDER BY d_year, d_moy) rn + FROM item, store_sales, date_dim, store + WHERE ss_item_sk = i_item_sk AND + ss_sold_date_sk = d_date_sk AND + ss_store_sk = s_store_sk AND + ( + d_year = 1999 OR + (d_year = 1999 - 1 AND d_moy = 12) OR + (d_year = 1999 + 1 AND d_moy = 1) + ) + GROUP BY i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy), + v2 AS ( + SELECT + v1.i_category, + v1.i_brand, + v1.s_store_name, + v1.s_company_name, + v1.d_year, + v1.d_moy, + v1.avg_monthly_sales, + v1.sum_sales, + v1_lag.sum_sales psum, + v1_lead.sum_sales nsum + FROM v1, v1 v1_lag, v1 v1_lead + WHERE v1.i_category = v1_lag.i_category AND + v1.i_category = v1_lead.i_category AND + v1.i_brand = v1_lag.i_brand AND + v1.i_brand = v1_lead.i_brand AND + v1.s_store_name = v1_lag.s_store_name AND + v1.s_store_name = v1_lead.s_store_name AND + v1.s_company_name = v1_lag.s_company_name AND + v1.s_company_name = v1_lead.s_company_name AND + v1.rn = v1_lag.rn + 1 AND + v1.rn = v1_lead.rn - 1) +SELECT * +FROM v2 +WHERE d_year = 1999 AND + avg_monthly_sales > 0 AND + CASE WHEN avg_monthly_sales > 0 + THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales + ELSE NULL END > 0.1 +ORDER BY sum_sales - avg_monthly_sales, 3 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q48.sql b/sql/core/src/test/resources/tpcds/q48.sql new file mode 100755 index 000000000000..fdb9f38e294f --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q48.sql @@ -0,0 +1,63 @@ +SELECT sum(ss_quantity) +FROM store_sales, store, customer_demographics, customer_address, date_dim +WHERE s_store_sk = ss_store_sk + AND ss_sold_date_sk = d_date_sk AND d_year = 2001 + AND + ( + ( + cd_demo_sk = ss_cdemo_sk + AND + cd_marital_status = 'M' + AND + cd_education_status = '4 yr Degree' + AND + ss_sales_price BETWEEN 100.00 AND 150.00 + ) + OR + ( + cd_demo_sk = ss_cdemo_sk + AND + cd_marital_status = 'D' + AND + cd_education_status = '2 yr Degree' + AND + ss_sales_price BETWEEN 50.00 AND 100.00 + ) + OR + ( + cd_demo_sk = ss_cdemo_sk + AND + cd_marital_status = 'S' + AND + cd_education_status = 'College' + AND + ss_sales_price BETWEEN 150.00 AND 200.00 + ) + ) + AND + ( + ( + ss_addr_sk = ca_address_sk + AND + ca_country = 'United States' + AND + ca_state IN ('CO', 'OH', 'TX') + AND ss_net_profit BETWEEN 0 AND 2000 + ) + OR + (ss_addr_sk = ca_address_sk + AND + ca_country = 'United States' + AND + ca_state IN ('OR', 'MN', 'KY') + AND ss_net_profit BETWEEN 150 AND 3000 + ) + OR + (ss_addr_sk = ca_address_sk + AND + ca_country = 'United States' + AND + ca_state IN ('VA', 'CA', 'MS') + AND ss_net_profit BETWEEN 50 AND 25000 + ) + ) diff --git a/sql/core/src/test/resources/tpcds/q49.sql b/sql/core/src/test/resources/tpcds/q49.sql new file mode 100755 index 000000000000..9568d8b92d10 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q49.sql @@ -0,0 +1,126 @@ +SELECT + 'web' AS channel, + web.item, + web.return_ratio, + web.return_rank, + web.currency_rank +FROM ( + SELECT + item, + return_ratio, + currency_ratio, + rank() + OVER ( + ORDER BY return_ratio) AS return_rank, + rank() + OVER ( + ORDER BY currency_ratio) AS currency_rank + FROM + (SELECT + ws.ws_item_sk AS item, + (cast(sum(coalesce(wr.wr_return_quantity, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(ws.ws_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, + (cast(sum(coalesce(wr.wr_return_amt, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(ws.ws_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio + FROM + web_sales ws LEFT OUTER JOIN web_returns wr + ON (ws.ws_order_number = wr.wr_order_number AND + ws.ws_item_sk = wr.wr_item_sk) + , date_dim + WHERE + wr.wr_return_amt > 10000 + AND ws.ws_net_profit > 1 + AND ws.ws_net_paid > 0 + AND ws.ws_quantity > 0 + AND ws_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 12 + GROUP BY ws.ws_item_sk + ) in_web + ) web +WHERE (web.return_rank <= 10 OR web.currency_rank <= 10) +UNION +SELECT + 'catalog' AS channel, + catalog.item, + catalog.return_ratio, + catalog.return_rank, + catalog.currency_rank +FROM ( + SELECT + item, + return_ratio, + currency_ratio, + rank() + OVER ( + ORDER BY return_ratio) AS return_rank, + rank() + OVER ( + ORDER BY currency_ratio) AS currency_rank + FROM + (SELECT + cs.cs_item_sk AS item, + (cast(sum(coalesce(cr.cr_return_quantity, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(cs.cs_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, + (cast(sum(coalesce(cr.cr_return_amount, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(cs.cs_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio + FROM + catalog_sales cs LEFT OUTER JOIN catalog_returns cr + ON (cs.cs_order_number = cr.cr_order_number AND + cs.cs_item_sk = cr.cr_item_sk) + , date_dim + WHERE + cr.cr_return_amount > 10000 + AND cs.cs_net_profit > 1 + AND cs.cs_net_paid > 0 + AND cs.cs_quantity > 0 + AND cs_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 12 + GROUP BY cs.cs_item_sk + ) in_cat + ) catalog +WHERE (catalog.return_rank <= 10 OR catalog.currency_rank <= 10) +UNION +SELECT + 'store' AS channel, + store.item, + store.return_ratio, + store.return_rank, + store.currency_rank +FROM ( + SELECT + item, + return_ratio, + currency_ratio, + rank() + OVER ( + ORDER BY return_ratio) AS return_rank, + rank() + OVER ( + ORDER BY currency_ratio) AS currency_rank + FROM + (SELECT + sts.ss_item_sk AS item, + (cast(sum(coalesce(sr.sr_return_quantity, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(sts.ss_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, + (cast(sum(coalesce(sr.sr_return_amt, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(sts.ss_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio + FROM + store_sales sts LEFT OUTER JOIN store_returns sr + ON (sts.ss_ticket_number = sr.sr_ticket_number AND sts.ss_item_sk = sr.sr_item_sk) + , date_dim + WHERE + sr.sr_return_amt > 10000 + AND sts.ss_net_profit > 1 + AND sts.ss_net_paid > 0 + AND sts.ss_quantity > 0 + AND ss_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 12 + GROUP BY sts.ss_item_sk + ) in_store + ) store +WHERE (store.return_rank <= 10 OR store.currency_rank <= 10) +ORDER BY 1, 4, 5 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q5.sql b/sql/core/src/test/resources/tpcds/q5.sql new file mode 100755 index 000000000000..b87cf3a44827 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q5.sql @@ -0,0 +1,131 @@ +WITH ssr AS +( SELECT + s_store_id, + sum(sales_price) AS sales, + sum(profit) AS profit, + sum(return_amt) AS RETURNS, + sum(net_loss) AS profit_loss + FROM + (SELECT + ss_store_sk AS store_sk, + ss_sold_date_sk AS date_sk, + ss_ext_sales_price AS sales_price, + ss_net_profit AS profit, + cast(0 AS DECIMAL(7, 2)) AS return_amt, + cast(0 AS DECIMAL(7, 2)) AS net_loss + FROM store_sales + UNION ALL + SELECT + sr_store_sk AS store_sk, + sr_returned_date_sk AS date_sk, + cast(0 AS DECIMAL(7, 2)) AS sales_price, + cast(0 AS DECIMAL(7, 2)) AS profit, + sr_return_amt AS return_amt, + sr_net_loss AS net_loss + FROM store_returns) + salesreturns, date_dim, store + WHERE date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) + AND store_sk = s_store_sk + GROUP BY s_store_id), + csr AS + ( SELECT + cp_catalog_page_id, + sum(sales_price) AS sales, + sum(profit) AS profit, + sum(return_amt) AS RETURNS, + sum(net_loss) AS profit_loss + FROM + (SELECT + cs_catalog_page_sk AS page_sk, + cs_sold_date_sk AS date_sk, + cs_ext_sales_price AS sales_price, + cs_net_profit AS profit, + cast(0 AS DECIMAL(7, 2)) AS return_amt, + cast(0 AS DECIMAL(7, 2)) AS net_loss + FROM catalog_sales + UNION ALL + SELECT + cr_catalog_page_sk AS page_sk, + cr_returned_date_sk AS date_sk, + cast(0 AS DECIMAL(7, 2)) AS sales_price, + cast(0 AS DECIMAL(7, 2)) AS profit, + cr_return_amount AS return_amt, + cr_net_loss AS net_loss + FROM catalog_returns + ) salesreturns, date_dim, catalog_page + WHERE date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) + AND page_sk = cp_catalog_page_sk + GROUP BY cp_catalog_page_id) + , + wsr AS + ( SELECT + web_site_id, + sum(sales_price) AS sales, + sum(profit) AS profit, + sum(return_amt) AS RETURNS, + sum(net_loss) AS profit_loss + FROM + (SELECT + ws_web_site_sk AS wsr_web_site_sk, + ws_sold_date_sk AS date_sk, + ws_ext_sales_price AS sales_price, + ws_net_profit AS profit, + cast(0 AS DECIMAL(7, 2)) AS return_amt, + cast(0 AS DECIMAL(7, 2)) AS net_loss + FROM web_sales + UNION ALL + SELECT + ws_web_site_sk AS wsr_web_site_sk, + wr_returned_date_sk AS date_sk, + cast(0 AS DECIMAL(7, 2)) AS sales_price, + cast(0 AS DECIMAL(7, 2)) AS profit, + wr_return_amt AS return_amt, + wr_net_loss AS net_loss + FROM web_returns + LEFT OUTER JOIN web_sales ON + (wr_item_sk = ws_item_sk + AND wr_order_number = ws_order_number) + ) salesreturns, date_dim, web_site + WHERE date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) + AND wsr_web_site_sk = web_site_sk + GROUP BY web_site_id) +SELECT + channel, + id, + sum(sales) AS sales, + sum(returns) AS returns, + sum(profit) AS profit +FROM + (SELECT + 'store channel' AS channel, + concat('store', s_store_id) AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM ssr + UNION ALL + SELECT + 'catalog channel' AS channel, + concat('catalog_page', cp_catalog_page_id) AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM csr + UNION ALL + SELECT + 'web channel' AS channel, + concat('web_site', web_site_id) AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM wsr + ) x +GROUP BY ROLLUP (channel, id) +ORDER BY channel, id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q50.sql b/sql/core/src/test/resources/tpcds/q50.sql new file mode 100755 index 000000000000..f1d4b15449ed --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q50.sql @@ -0,0 +1,47 @@ +SELECT + s_store_name, + s_company_id, + s_street_number, + s_street_name, + s_street_type, + s_suite_number, + s_city, + s_county, + s_state, + s_zip, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk <= 30) + THEN 1 + ELSE 0 END) AS `30 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 30) AND + (sr_returned_date_sk - ss_sold_date_sk <= 60) + THEN 1 + ELSE 0 END) AS `31 - 60 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 60) AND + (sr_returned_date_sk - ss_sold_date_sk <= 90) + THEN 1 + ELSE 0 END) AS `61 - 90 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 90) AND + (sr_returned_date_sk - ss_sold_date_sk <= 120) + THEN 1 + ELSE 0 END) AS `91 - 120 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 120) + THEN 1 + ELSE 0 END) AS `>120 days ` +FROM + store_sales, store_returns, store, date_dim d1, date_dim d2 +WHERE + d2.d_year = 2001 + AND d2.d_moy = 8 + AND ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk + AND ss_sold_date_sk = d1.d_date_sk + AND sr_returned_date_sk = d2.d_date_sk + AND ss_customer_sk = sr_customer_sk + AND ss_store_sk = s_store_sk +GROUP BY + s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, + s_suite_number, s_city, s_county, s_state, s_zip +ORDER BY + s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, + s_suite_number, s_city, s_county, s_state, s_zip +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q51.sql b/sql/core/src/test/resources/tpcds/q51.sql new file mode 100755 index 000000000000..62b003eb67b9 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q51.sql @@ -0,0 +1,55 @@ +WITH web_v1 AS ( + SELECT + ws_item_sk item_sk, + d_date, + sum(sum(ws_sales_price)) + OVER (PARTITION BY ws_item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) cume_sales + FROM web_sales, date_dim + WHERE ws_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + AND ws_item_sk IS NOT NULL + GROUP BY ws_item_sk, d_date), + store_v1 AS ( + SELECT + ss_item_sk item_sk, + d_date, + sum(sum(ss_sales_price)) + OVER (PARTITION BY ss_item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) cume_sales + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + AND ss_item_sk IS NOT NULL + GROUP BY ss_item_sk, d_date) +SELECT * +FROM (SELECT + item_sk, + d_date, + web_sales, + store_sales, + max(web_sales) + OVER (PARTITION BY item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) web_cumulative, + max(store_sales) + OVER (PARTITION BY item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) store_cumulative +FROM (SELECT + CASE WHEN web.item_sk IS NOT NULL + THEN web.item_sk + ELSE store.item_sk END item_sk, + CASE WHEN web.d_date IS NOT NULL + THEN web.d_date + ELSE store.d_date END d_date, + web.cume_sales web_sales, + store.cume_sales store_sales +FROM web_v1 web FULL OUTER JOIN store_v1 store ON (web.item_sk = store.item_sk + AND web.d_date = store.d_date) + ) x) y +WHERE web_cumulative > store_cumulative +ORDER BY item_sk, d_date +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q52.sql b/sql/core/src/test/resources/tpcds/q52.sql new file mode 100755 index 000000000000..467d1ae05045 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q52.sql @@ -0,0 +1,14 @@ +SELECT + dt.d_year, + item.i_brand_id brand_id, + item.i_brand brand, + sum(ss_ext_sales_price) ext_price +FROM date_dim dt, store_sales, item +WHERE dt.d_date_sk = store_sales.ss_sold_date_sk + AND store_sales.ss_item_sk = item.i_item_sk + AND item.i_manager_id = 1 + AND dt.d_moy = 11 + AND dt.d_year = 2000 +GROUP BY dt.d_year, item.i_brand, item.i_brand_id +ORDER BY dt.d_year, ext_price DESC, brand_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q53.sql b/sql/core/src/test/resources/tpcds/q53.sql new file mode 100755 index 000000000000..b42c68dcf871 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q53.sql @@ -0,0 +1,30 @@ +SELECT * +FROM + (SELECT + i_manufact_id, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER (PARTITION BY i_manufact_id) avg_quarterly_sales + FROM item, store_sales, date_dim, store + WHERE ss_item_sk = i_item_sk AND + ss_sold_date_sk = d_date_sk AND + ss_store_sk = s_store_sk AND + d_month_seq IN (1200, 1200 + 1, 1200 + 2, 1200 + 3, 1200 + 4, 1200 + 5, 1200 + 6, + 1200 + 7, 1200 + 8, 1200 + 9, 1200 + 10, 1200 + 11) AND + ((i_category IN ('Books', 'Children', 'Electronics') AND + i_class IN ('personal', 'portable', 'reference', 'self-help') AND + i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', + 'exportiunivamalg #9', 'scholaramalgamalg #9')) + OR + (i_category IN ('Women', 'Music', 'Men') AND + i_class IN ('accessories', 'classical', 'fragrances', 'pants') AND + i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', + 'importoamalg #1'))) + GROUP BY i_manufact_id, d_qoy) tmp1 +WHERE CASE WHEN avg_quarterly_sales > 0 + THEN abs(sum_sales - avg_quarterly_sales) / avg_quarterly_sales + ELSE NULL END > 0.1 +ORDER BY avg_quarterly_sales, + sum_sales, + i_manufact_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q54.sql b/sql/core/src/test/resources/tpcds/q54.sql new file mode 100755 index 000000000000..897237fb6e10 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q54.sql @@ -0,0 +1,61 @@ +WITH my_customers AS ( + SELECT DISTINCT + c_customer_sk, + c_current_addr_sk + FROM + (SELECT + cs_sold_date_sk sold_date_sk, + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + FROM catalog_sales + UNION ALL + SELECT + ws_sold_date_sk sold_date_sk, + ws_bill_customer_sk customer_sk, + ws_item_sk item_sk + FROM web_sales + ) cs_or_ws_sales, + item, + date_dim, + customer + WHERE sold_date_sk = d_date_sk + AND item_sk = i_item_sk + AND i_category = 'Women' + AND i_class = 'maternity' + AND c_customer_sk = cs_or_ws_sales.customer_sk + AND d_moy = 12 + AND d_year = 1998 +) + , my_revenue AS ( + SELECT + c_customer_sk, + sum(ss_ext_sales_price) AS revenue + FROM my_customers, + store_sales, + customer_address, + store, + date_dim + WHERE c_current_addr_sk = ca_address_sk + AND ca_county = s_county + AND ca_state = s_state + AND ss_sold_date_sk = d_date_sk + AND c_customer_sk = ss_customer_sk + AND d_month_seq BETWEEN (SELECT DISTINCT d_month_seq + 1 + FROM date_dim + WHERE d_year = 1998 AND d_moy = 12) + AND (SELECT DISTINCT d_month_seq + 3 + FROM date_dim + WHERE d_year = 1998 AND d_moy = 12) + GROUP BY c_customer_sk +) + , segments AS +(SELECT cast((revenue / 50) AS INT) AS segment + FROM my_revenue) +SELECT + segment, + count(*) AS num_customers, + segment * 50 AS segment_base +FROM segments +GROUP BY segment +ORDER BY segment, num_customers +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q55.sql b/sql/core/src/test/resources/tpcds/q55.sql new file mode 100755 index 000000000000..bc5d888c9ac5 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q55.sql @@ -0,0 +1,13 @@ +SELECT + i_brand_id brand_id, + i_brand brand, + sum(ss_ext_sales_price) ext_price +FROM date_dim, store_sales, item +WHERE d_date_sk = ss_sold_date_sk + AND ss_item_sk = i_item_sk + AND i_manager_id = 28 + AND d_moy = 11 + AND d_year = 1999 +GROUP BY i_brand, i_brand_id +ORDER BY ext_price DESC, brand_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q56.sql b/sql/core/src/test/resources/tpcds/q56.sql new file mode 100755 index 000000000000..2fa1738dcfee --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q56.sql @@ -0,0 +1,65 @@ +WITH ss AS ( + SELECT + i_item_id, + sum(ss_ext_sales_price) total_sales + FROM + store_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_color IN ('slate', 'blanched', 'burnished')) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 2 + AND ss_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + cs AS ( + SELECT + i_item_id, + sum(cs_ext_sales_price) total_sales + FROM + catalog_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_color IN ('slate', 'blanched', 'burnished')) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 2 + AND cs_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + ws AS ( + SELECT + i_item_id, + sum(ws_ext_sales_price) total_sales + FROM + web_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_color IN ('slate', 'blanched', 'burnished')) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 2 + AND ws_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id) +SELECT + i_item_id, + sum(total_sales) total_sales +FROM (SELECT * + FROM ss + UNION ALL + SELECT * + FROM cs + UNION ALL + SELECT * + FROM ws) tmp1 +GROUP BY i_item_id +ORDER BY total_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q57.sql b/sql/core/src/test/resources/tpcds/q57.sql new file mode 100755 index 000000000000..cf70d4b905b5 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q57.sql @@ -0,0 +1,56 @@ +WITH v1 AS ( + SELECT + i_category, + i_brand, + cc_name, + d_year, + d_moy, + sum(cs_sales_price) sum_sales, + avg(sum(cs_sales_price)) + OVER + (PARTITION BY i_category, i_brand, cc_name, d_year) + avg_monthly_sales, + rank() + OVER + (PARTITION BY i_category, i_brand, cc_name + ORDER BY d_year, d_moy) rn + FROM item, catalog_sales, date_dim, call_center + WHERE cs_item_sk = i_item_sk AND + cs_sold_date_sk = d_date_sk AND + cc_call_center_sk = cs_call_center_sk AND + ( + d_year = 1999 OR + (d_year = 1999 - 1 AND d_moy = 12) OR + (d_year = 1999 + 1 AND d_moy = 1) + ) + GROUP BY i_category, i_brand, + cc_name, d_year, d_moy), + v2 AS ( + SELECT + v1.i_category, + v1.i_brand, + v1.cc_name, + v1.d_year, + v1.d_moy, + v1.avg_monthly_sales, + v1.sum_sales, + v1_lag.sum_sales psum, + v1_lead.sum_sales nsum + FROM v1, v1 v1_lag, v1 v1_lead + WHERE v1.i_category = v1_lag.i_category AND + v1.i_category = v1_lead.i_category AND + v1.i_brand = v1_lag.i_brand AND + v1.i_brand = v1_lead.i_brand AND + v1.cc_name = v1_lag.cc_name AND + v1.cc_name = v1_lead.cc_name AND + v1.rn = v1_lag.rn + 1 AND + v1.rn = v1_lead.rn - 1) +SELECT * +FROM v2 +WHERE d_year = 1999 AND + avg_monthly_sales > 0 AND + CASE WHEN avg_monthly_sales > 0 + THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales + ELSE NULL END > 0.1 +ORDER BY sum_sales - avg_monthly_sales, 3 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q58.sql b/sql/core/src/test/resources/tpcds/q58.sql new file mode 100755 index 000000000000..5f63f33dc927 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q58.sql @@ -0,0 +1,59 @@ +WITH ss_items AS +(SELECT + i_item_id item_id, + sum(ss_ext_sales_price) ss_item_rev + FROM store_sales, item, date_dim + WHERE ss_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_date = '2000-01-03')) + AND ss_sold_date_sk = d_date_sk + GROUP BY i_item_id), + cs_items AS + (SELECT + i_item_id item_id, + sum(cs_ext_sales_price) cs_item_rev + FROM catalog_sales, item, date_dim + WHERE cs_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_date = '2000-01-03')) + AND cs_sold_date_sk = d_date_sk + GROUP BY i_item_id), + ws_items AS + (SELECT + i_item_id item_id, + sum(ws_ext_sales_price) ws_item_rev + FROM web_sales, item, date_dim + WHERE ws_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_date = '2000-01-03')) + AND ws_sold_date_sk = d_date_sk + GROUP BY i_item_id) +SELECT + ss_items.item_id, + ss_item_rev, + ss_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 ss_dev, + cs_item_rev, + cs_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 cs_dev, + ws_item_rev, + ws_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 ws_dev, + (ss_item_rev + cs_item_rev + ws_item_rev) / 3 average +FROM ss_items, cs_items, ws_items +WHERE ss_items.item_id = cs_items.item_id + AND ss_items.item_id = ws_items.item_id + AND ss_item_rev BETWEEN 0.9 * cs_item_rev AND 1.1 * cs_item_rev + AND ss_item_rev BETWEEN 0.9 * ws_item_rev AND 1.1 * ws_item_rev + AND cs_item_rev BETWEEN 0.9 * ss_item_rev AND 1.1 * ss_item_rev + AND cs_item_rev BETWEEN 0.9 * ws_item_rev AND 1.1 * ws_item_rev + AND ws_item_rev BETWEEN 0.9 * ss_item_rev AND 1.1 * ss_item_rev + AND ws_item_rev BETWEEN 0.9 * cs_item_rev AND 1.1 * cs_item_rev +ORDER BY item_id, ss_item_rev +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q59.sql b/sql/core/src/test/resources/tpcds/q59.sql new file mode 100755 index 000000000000..3cef2027680b --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q59.sql @@ -0,0 +1,75 @@ +WITH wss AS +(SELECT + d_week_seq, + ss_store_sk, + sum(CASE WHEN (d_day_name = 'Sunday') + THEN ss_sales_price + ELSE NULL END) sun_sales, + sum(CASE WHEN (d_day_name = 'Monday') + THEN ss_sales_price + ELSE NULL END) mon_sales, + sum(CASE WHEN (d_day_name = 'Tuesday') + THEN ss_sales_price + ELSE NULL END) tue_sales, + sum(CASE WHEN (d_day_name = 'Wednesday') + THEN ss_sales_price + ELSE NULL END) wed_sales, + sum(CASE WHEN (d_day_name = 'Thursday') + THEN ss_sales_price + ELSE NULL END) thu_sales, + sum(CASE WHEN (d_day_name = 'Friday') + THEN ss_sales_price + ELSE NULL END) fri_sales, + sum(CASE WHEN (d_day_name = 'Saturday') + THEN ss_sales_price + ELSE NULL END) sat_sales + FROM store_sales, date_dim + WHERE d_date_sk = ss_sold_date_sk + GROUP BY d_week_seq, ss_store_sk +) +SELECT + s_store_name1, + s_store_id1, + d_week_seq1, + sun_sales1 / sun_sales2, + mon_sales1 / mon_sales2, + tue_sales1 / tue_sales2, + wed_sales1 / wed_sales2, + thu_sales1 / thu_sales2, + fri_sales1 / fri_sales2, + sat_sales1 / sat_sales2 +FROM + (SELECT + s_store_name s_store_name1, + wss.d_week_seq d_week_seq1, + s_store_id s_store_id1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 + FROM wss, store, date_dim d + WHERE d.d_week_seq = wss.d_week_seq AND + ss_store_sk = s_store_sk AND + d_month_seq BETWEEN 1212 AND 1212 + 11) y, + (SELECT + s_store_name s_store_name2, + wss.d_week_seq d_week_seq2, + s_store_id s_store_id2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + FROM wss, store, date_dim d + WHERE d.d_week_seq = wss.d_week_seq AND + ss_store_sk = s_store_sk AND + d_month_seq BETWEEN 1212 + 12 AND 1212 + 23) x +WHERE s_store_id1 = s_store_id2 + AND d_week_seq1 = d_week_seq2 - 52 +ORDER BY s_store_name1, s_store_id1, d_week_seq1 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q6.sql b/sql/core/src/test/resources/tpcds/q6.sql new file mode 100755 index 000000000000..f0f5cf05aebd --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q6.sql @@ -0,0 +1,21 @@ +SELECT + a.ca_state state, + count(*) cnt +FROM + customer_address a, customer c, store_sales s, date_dim d, item i +WHERE a.ca_address_sk = c.c_current_addr_sk + AND c.c_customer_sk = s.ss_customer_sk + AND s.ss_sold_date_sk = d.d_date_sk + AND s.ss_item_sk = i.i_item_sk + AND d.d_month_seq = + (SELECT DISTINCT (d_month_seq) + FROM date_dim + WHERE d_year = 2000 AND d_moy = 1) + AND i.i_current_price > 1.2 * + (SELECT avg(j.i_current_price) + FROM item j + WHERE j.i_category = i.i_category) +GROUP BY a.ca_state +HAVING count(*) >= 10 +ORDER BY cnt +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q60.sql b/sql/core/src/test/resources/tpcds/q60.sql new file mode 100755 index 000000000000..41b963f44ba1 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q60.sql @@ -0,0 +1,62 @@ +WITH ss AS ( + SELECT + i_item_id, + sum(ss_ext_sales_price) total_sales + FROM store_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_category IN ('Music')) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 9 + AND ss_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + cs AS ( + SELECT + i_item_id, + sum(cs_ext_sales_price) total_sales + FROM catalog_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_category IN ('Music')) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 9 + AND cs_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + ws AS ( + SELECT + i_item_id, + sum(ws_ext_sales_price) total_sales + FROM web_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_category IN ('Music')) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 9 + AND ws_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id) +SELECT + i_item_id, + sum(total_sales) total_sales +FROM (SELECT * + FROM ss + UNION ALL + SELECT * + FROM cs + UNION ALL + SELECT * + FROM ws) tmp1 +GROUP BY i_item_id +ORDER BY i_item_id, total_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q61.sql b/sql/core/src/test/resources/tpcds/q61.sql new file mode 100755 index 000000000000..b0a872b4b80e --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q61.sql @@ -0,0 +1,33 @@ +SELECT + promotions, + total, + cast(promotions AS DECIMAL(15, 4)) / cast(total AS DECIMAL(15, 4)) * 100 +FROM + (SELECT sum(ss_ext_sales_price) promotions + FROM store_sales, store, promotion, date_dim, customer, customer_address, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_store_sk = s_store_sk + AND ss_promo_sk = p_promo_sk + AND ss_customer_sk = c_customer_sk + AND ca_address_sk = c_current_addr_sk + AND ss_item_sk = i_item_sk + AND ca_gmt_offset = -5 + AND i_category = 'Jewelry' + AND (p_channel_dmail = 'Y' OR p_channel_email = 'Y' OR p_channel_tv = 'Y') + AND s_gmt_offset = -5 + AND d_year = 1998 + AND d_moy = 11) promotional_sales, + (SELECT sum(ss_ext_sales_price) total + FROM store_sales, store, date_dim, customer, customer_address, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_store_sk = s_store_sk + AND ss_customer_sk = c_customer_sk + AND ca_address_sk = c_current_addr_sk + AND ss_item_sk = i_item_sk + AND ca_gmt_offset = -5 + AND i_category = 'Jewelry' + AND s_gmt_offset = -5 + AND d_year = 1998 + AND d_moy = 11) all_sales +ORDER BY promotions, total +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q62.sql b/sql/core/src/test/resources/tpcds/q62.sql new file mode 100755 index 000000000000..8a414f154bdc --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q62.sql @@ -0,0 +1,35 @@ +SELECT + substr(w_warehouse_name, 1, 20), + sm_type, + web_name, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk <= 30) + THEN 1 + ELSE 0 END) AS `30 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 30) AND + (ws_ship_date_sk - ws_sold_date_sk <= 60) + THEN 1 + ELSE 0 END) AS `31 - 60 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 60) AND + (ws_ship_date_sk - ws_sold_date_sk <= 90) + THEN 1 + ELSE 0 END) AS `61 - 90 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 90) AND + (ws_ship_date_sk - ws_sold_date_sk <= 120) + THEN 1 + ELSE 0 END) AS `91 - 120 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 120) + THEN 1 + ELSE 0 END) AS `>120 days ` +FROM + web_sales, warehouse, ship_mode, web_site, date_dim +WHERE + d_month_seq BETWEEN 1200 AND 1200 + 11 + AND ws_ship_date_sk = d_date_sk + AND ws_warehouse_sk = w_warehouse_sk + AND ws_ship_mode_sk = sm_ship_mode_sk + AND ws_web_site_sk = web_site_sk +GROUP BY + substr(w_warehouse_name, 1, 20), sm_type, web_name +ORDER BY + substr(w_warehouse_name, 1, 20), sm_type, web_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q63.sql b/sql/core/src/test/resources/tpcds/q63.sql new file mode 100755 index 000000000000..ef6867e0a945 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q63.sql @@ -0,0 +1,31 @@ +SELECT * +FROM (SELECT + i_manager_id, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER (PARTITION BY i_manager_id) avg_monthly_sales +FROM item + , store_sales + , date_dim + , store +WHERE ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND ss_store_sk = s_store_sk + AND d_month_seq IN (1200, 1200 + 1, 1200 + 2, 1200 + 3, 1200 + 4, 1200 + 5, 1200 + 6, 1200 + 7, + 1200 + 8, 1200 + 9, 1200 + 10, 1200 + 11) + AND ((i_category IN ('Books', 'Children', 'Electronics') + AND i_class IN ('personal', 'portable', 'refernece', 'self-help') + AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', + 'exportiunivamalg #9', 'scholaramalgamalg #9')) + OR (i_category IN ('Women', 'Music', 'Men') + AND i_class IN ('accessories', 'classical', 'fragrances', 'pants') + AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', + 'importoamalg #1'))) +GROUP BY i_manager_id, d_moy) tmp1 +WHERE CASE WHEN avg_monthly_sales > 0 + THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales + ELSE NULL END > 0.1 +ORDER BY i_manager_id + , avg_monthly_sales + , sum_sales +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q64.sql b/sql/core/src/test/resources/tpcds/q64.sql new file mode 100755 index 000000000000..8ec1d31b61af --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q64.sql @@ -0,0 +1,92 @@ +WITH cs_ui AS +(SELECT + cs_item_sk, + sum(cs_ext_list_price) AS sale, + sum(cr_refunded_cash + cr_reversed_charge + cr_store_credit) AS refund + FROM catalog_sales + , catalog_returns + WHERE cs_item_sk = cr_item_sk + AND cs_order_number = cr_order_number + GROUP BY cs_item_sk + HAVING sum(cs_ext_list_price) > 2 * sum(cr_refunded_cash + cr_reversed_charge + cr_store_credit)), + cross_sales AS + (SELECT + i_product_name product_name, + i_item_sk item_sk, + s_store_name store_name, + s_zip store_zip, + ad1.ca_street_number b_street_number, + ad1.ca_street_name b_streen_name, + ad1.ca_city b_city, + ad1.ca_zip b_zip, + ad2.ca_street_number c_street_number, + ad2.ca_street_name c_street_name, + ad2.ca_city c_city, + ad2.ca_zip c_zip, + d1.d_year AS syear, + d2.d_year AS fsyear, + d3.d_year s2year, + count(*) cnt, + sum(ss_wholesale_cost) s1, + sum(ss_list_price) s2, + sum(ss_coupon_amt) s3 + FROM store_sales, store_returns, cs_ui, date_dim d1, date_dim d2, date_dim d3, + store, customer, customer_demographics cd1, customer_demographics cd2, + promotion, household_demographics hd1, household_demographics hd2, + customer_address ad1, customer_address ad2, income_band ib1, income_band ib2, item + WHERE ss_store_sk = s_store_sk AND + ss_sold_date_sk = d1.d_date_sk AND + ss_customer_sk = c_customer_sk AND + ss_cdemo_sk = cd1.cd_demo_sk AND + ss_hdemo_sk = hd1.hd_demo_sk AND + ss_addr_sk = ad1.ca_address_sk AND + ss_item_sk = i_item_sk AND + ss_item_sk = sr_item_sk AND + ss_ticket_number = sr_ticket_number AND + ss_item_sk = cs_ui.cs_item_sk AND + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_hdemo_sk = hd2.hd_demo_sk AND + c_current_addr_sk = ad2.ca_address_sk AND + c_first_sales_date_sk = d2.d_date_sk AND + c_first_shipto_date_sk = d3.d_date_sk AND + ss_promo_sk = p_promo_sk AND + hd1.hd_income_band_sk = ib1.ib_income_band_sk AND + hd2.hd_income_band_sk = ib2.ib_income_band_sk AND + cd1.cd_marital_status <> cd2.cd_marital_status AND + i_color IN ('purple', 'burlywood', 'indian', 'spring', 'floral', 'medium') AND + i_current_price BETWEEN 64 AND 64 + 10 AND + i_current_price BETWEEN 64 + 1 AND 64 + 15 + GROUP BY i_product_name, i_item_sk, s_store_name, s_zip, ad1.ca_street_number, + ad1.ca_street_name, ad1.ca_city, ad1.ca_zip, ad2.ca_street_number, + ad2.ca_street_name, ad2.ca_city, ad2.ca_zip, d1.d_year, d2.d_year, d3.d_year + ) +SELECT + cs1.product_name, + cs1.store_name, + cs1.store_zip, + cs1.b_street_number, + cs1.b_streen_name, + cs1.b_city, + cs1.b_zip, + cs1.c_street_number, + cs1.c_street_name, + cs1.c_city, + cs1.c_zip, + cs1.syear, + cs1.cnt, + cs1.s1, + cs1.s2, + cs1.s3, + cs2.s1, + cs2.s2, + cs2.s3, + cs2.syear, + cs2.cnt +FROM cross_sales cs1, cross_sales cs2 +WHERE cs1.item_sk = cs2.item_sk AND + cs1.syear = 1999 AND + cs2.syear = 1999 + 1 AND + cs2.cnt <= cs1.cnt AND + cs1.store_name = cs2.store_name AND + cs1.store_zip = cs2.store_zip +ORDER BY cs1.product_name, cs1.store_name, cs2.cnt diff --git a/sql/core/src/test/resources/tpcds/q65.sql b/sql/core/src/test/resources/tpcds/q65.sql new file mode 100755 index 000000000000..aad04be1bcdf --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q65.sql @@ -0,0 +1,33 @@ +SELECT + s_store_name, + i_item_desc, + sc.revenue, + i_current_price, + i_wholesale_cost, + i_brand +FROM store, item, + (SELECT + ss_store_sk, + avg(revenue) AS ave + FROM + (SELECT + ss_store_sk, + ss_item_sk, + sum(ss_sales_price) AS revenue + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk AND d_month_seq BETWEEN 1176 AND 1176 + 11 + GROUP BY ss_store_sk, ss_item_sk) sa + GROUP BY ss_store_sk) sb, + (SELECT + ss_store_sk, + ss_item_sk, + sum(ss_sales_price) AS revenue + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk AND d_month_seq BETWEEN 1176 AND 1176 + 11 + GROUP BY ss_store_sk, ss_item_sk) sc +WHERE sb.ss_store_sk = sc.ss_store_sk AND + sc.revenue <= 0.1 * sb.ave AND + s_store_sk = sc.ss_store_sk AND + i_item_sk = sc.ss_item_sk +ORDER BY s_store_name, i_item_desc +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q66.sql b/sql/core/src/test/resources/tpcds/q66.sql new file mode 100755 index 000000000000..f826b4164372 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q66.sql @@ -0,0 +1,240 @@ +SELECT + w_warehouse_name, + w_warehouse_sq_ft, + w_city, + w_county, + w_state, + w_country, + ship_carriers, + year, + sum(jan_sales) AS jan_sales, + sum(feb_sales) AS feb_sales, + sum(mar_sales) AS mar_sales, + sum(apr_sales) AS apr_sales, + sum(may_sales) AS may_sales, + sum(jun_sales) AS jun_sales, + sum(jul_sales) AS jul_sales, + sum(aug_sales) AS aug_sales, + sum(sep_sales) AS sep_sales, + sum(oct_sales) AS oct_sales, + sum(nov_sales) AS nov_sales, + sum(dec_sales) AS dec_sales, + sum(jan_sales / w_warehouse_sq_ft) AS jan_sales_per_sq_foot, + sum(feb_sales / w_warehouse_sq_ft) AS feb_sales_per_sq_foot, + sum(mar_sales / w_warehouse_sq_ft) AS mar_sales_per_sq_foot, + sum(apr_sales / w_warehouse_sq_ft) AS apr_sales_per_sq_foot, + sum(may_sales / w_warehouse_sq_ft) AS may_sales_per_sq_foot, + sum(jun_sales / w_warehouse_sq_ft) AS jun_sales_per_sq_foot, + sum(jul_sales / w_warehouse_sq_ft) AS jul_sales_per_sq_foot, + sum(aug_sales / w_warehouse_sq_ft) AS aug_sales_per_sq_foot, + sum(sep_sales / w_warehouse_sq_ft) AS sep_sales_per_sq_foot, + sum(oct_sales / w_warehouse_sq_ft) AS oct_sales_per_sq_foot, + sum(nov_sales / w_warehouse_sq_ft) AS nov_sales_per_sq_foot, + sum(dec_sales / w_warehouse_sq_ft) AS dec_sales_per_sq_foot, + sum(jan_net) AS jan_net, + sum(feb_net) AS feb_net, + sum(mar_net) AS mar_net, + sum(apr_net) AS apr_net, + sum(may_net) AS may_net, + sum(jun_net) AS jun_net, + sum(jul_net) AS jul_net, + sum(aug_net) AS aug_net, + sum(sep_net) AS sep_net, + sum(oct_net) AS oct_net, + sum(nov_net) AS nov_net, + sum(dec_net) AS dec_net +FROM ( + (SELECT + w_warehouse_name, + w_warehouse_sq_ft, + w_city, + w_county, + w_state, + w_country, + concat('DHL', ',', 'BARIAN') AS ship_carriers, + d_year AS year, + sum(CASE WHEN d_moy = 1 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS jan_sales, + sum(CASE WHEN d_moy = 2 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS feb_sales, + sum(CASE WHEN d_moy = 3 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS mar_sales, + sum(CASE WHEN d_moy = 4 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS apr_sales, + sum(CASE WHEN d_moy = 5 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS may_sales, + sum(CASE WHEN d_moy = 6 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS jun_sales, + sum(CASE WHEN d_moy = 7 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS jul_sales, + sum(CASE WHEN d_moy = 8 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS aug_sales, + sum(CASE WHEN d_moy = 9 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS sep_sales, + sum(CASE WHEN d_moy = 10 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS oct_sales, + sum(CASE WHEN d_moy = 11 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS nov_sales, + sum(CASE WHEN d_moy = 12 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS dec_sales, + sum(CASE WHEN d_moy = 1 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS jan_net, + sum(CASE WHEN d_moy = 2 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS feb_net, + sum(CASE WHEN d_moy = 3 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS mar_net, + sum(CASE WHEN d_moy = 4 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS apr_net, + sum(CASE WHEN d_moy = 5 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS may_net, + sum(CASE WHEN d_moy = 6 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS jun_net, + sum(CASE WHEN d_moy = 7 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS jul_net, + sum(CASE WHEN d_moy = 8 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS aug_net, + sum(CASE WHEN d_moy = 9 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS sep_net, + sum(CASE WHEN d_moy = 10 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS oct_net, + sum(CASE WHEN d_moy = 11 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS nov_net, + sum(CASE WHEN d_moy = 12 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS dec_net + FROM + web_sales, warehouse, date_dim, time_dim, ship_mode + WHERE + ws_warehouse_sk = w_warehouse_sk + AND ws_sold_date_sk = d_date_sk + AND ws_sold_time_sk = t_time_sk + AND ws_ship_mode_sk = sm_ship_mode_sk + AND d_year = 2001 + AND t_time BETWEEN 30838 AND 30838 + 28800 + AND sm_carrier IN ('DHL', 'BARIAN') + GROUP BY + w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, d_year) + UNION ALL + (SELECT + w_warehouse_name, + w_warehouse_sq_ft, + w_city, + w_county, + w_state, + w_country, + concat('DHL', ',', 'BARIAN') AS ship_carriers, + d_year AS year, + sum(CASE WHEN d_moy = 1 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS jan_sales, + sum(CASE WHEN d_moy = 2 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS feb_sales, + sum(CASE WHEN d_moy = 3 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS mar_sales, + sum(CASE WHEN d_moy = 4 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS apr_sales, + sum(CASE WHEN d_moy = 5 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS may_sales, + sum(CASE WHEN d_moy = 6 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS jun_sales, + sum(CASE WHEN d_moy = 7 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS jul_sales, + sum(CASE WHEN d_moy = 8 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS aug_sales, + sum(CASE WHEN d_moy = 9 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS sep_sales, + sum(CASE WHEN d_moy = 10 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS oct_sales, + sum(CASE WHEN d_moy = 11 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS nov_sales, + sum(CASE WHEN d_moy = 12 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS dec_sales, + sum(CASE WHEN d_moy = 1 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS jan_net, + sum(CASE WHEN d_moy = 2 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS feb_net, + sum(CASE WHEN d_moy = 3 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS mar_net, + sum(CASE WHEN d_moy = 4 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS apr_net, + sum(CASE WHEN d_moy = 5 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS may_net, + sum(CASE WHEN d_moy = 6 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS jun_net, + sum(CASE WHEN d_moy = 7 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS jul_net, + sum(CASE WHEN d_moy = 8 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS aug_net, + sum(CASE WHEN d_moy = 9 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS sep_net, + sum(CASE WHEN d_moy = 10 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS oct_net, + sum(CASE WHEN d_moy = 11 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS nov_net, + sum(CASE WHEN d_moy = 12 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS dec_net + FROM + catalog_sales, warehouse, date_dim, time_dim, ship_mode + WHERE + cs_warehouse_sk = w_warehouse_sk + AND cs_sold_date_sk = d_date_sk + AND cs_sold_time_sk = t_time_sk + AND cs_ship_mode_sk = sm_ship_mode_sk + AND d_year = 2001 + AND t_time BETWEEN 30838 AND 30838 + 28800 + AND sm_carrier IN ('DHL', 'BARIAN') + GROUP BY + w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, d_year + ) + ) x +GROUP BY + w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, + ship_carriers, year +ORDER BY w_warehouse_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q67.sql b/sql/core/src/test/resources/tpcds/q67.sql new file mode 100755 index 000000000000..f66e2252bdbd --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q67.sql @@ -0,0 +1,38 @@ +SELECT * +FROM + (SELECT + i_category, + i_class, + i_brand, + i_product_name, + d_year, + d_qoy, + d_moy, + s_store_id, + sumsales, + rank() + OVER (PARTITION BY i_category + ORDER BY sumsales DESC) rk + FROM + (SELECT + i_category, + i_class, + i_brand, + i_product_name, + d_year, + d_qoy, + d_moy, + s_store_id, + sum(coalesce(ss_sales_price * ss_quantity, 0)) sumsales + FROM store_sales, date_dim, store, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + AND ss_store_sk = s_store_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + GROUP BY ROLLUP (i_category, i_class, i_brand, i_product_name, d_year, d_qoy, + d_moy, s_store_id)) dw1) dw2 +WHERE rk <= 100 +ORDER BY + i_category, i_class, i_brand, i_product_name, d_year, + d_qoy, d_moy, s_store_id, sumsales, rk +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q68.sql b/sql/core/src/test/resources/tpcds/q68.sql new file mode 100755 index 000000000000..adb8a7189dad --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q68.sql @@ -0,0 +1,34 @@ +SELECT + c_last_name, + c_first_name, + ca_city, + bought_city, + ss_ticket_number, + extended_price, + extended_tax, + list_price +FROM (SELECT + ss_ticket_number, + ss_customer_sk, + ca_city bought_city, + sum(ss_ext_sales_price) extended_price, + sum(ss_ext_list_price) list_price, + sum(ss_ext_tax) extended_tax +FROM store_sales, date_dim, store, household_demographics, customer_address +WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND store_sales.ss_addr_sk = customer_address.ca_address_sk + AND date_dim.d_dom BETWEEN 1 AND 2 + AND (household_demographics.hd_dep_count = 4 OR + household_demographics.hd_vehicle_count = 3) + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_city IN ('Midway', 'Fairview') +GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city) dn, + customer, + customer_address current_addr +WHERE ss_customer_sk = c_customer_sk + AND customer.c_current_addr_sk = current_addr.ca_address_sk + AND current_addr.ca_city <> bought_city +ORDER BY c_last_name, ss_ticket_number +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q69.sql b/sql/core/src/test/resources/tpcds/q69.sql new file mode 100755 index 000000000000..1f0ee64f565a --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q69.sql @@ -0,0 +1,38 @@ +SELECT + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3 +FROM + customer c, customer_address ca, customer_demographics +WHERE + c.c_current_addr_sk = ca.ca_address_sk AND + ca_state IN ('KY', 'GA', 'NM') AND + cd_demo_sk = c.c_current_cdemo_sk AND + exists(SELECT * + FROM store_sales, date_dim + WHERE c.c_customer_sk = ss_customer_sk AND + ss_sold_date_sk = d_date_sk AND + d_year = 2001 AND + d_moy BETWEEN 4 AND 4 + 2) AND + (NOT exists(SELECT * + FROM web_sales, date_dim + WHERE c.c_customer_sk = ws_bill_customer_sk AND + ws_sold_date_sk = d_date_sk AND + d_year = 2001 AND + d_moy BETWEEN 4 AND 4 + 2) AND + NOT exists(SELECT * + FROM catalog_sales, date_dim + WHERE c.c_customer_sk = cs_ship_customer_sk AND + cs_sold_date_sk = d_date_sk AND + d_year = 2001 AND + d_moy BETWEEN 4 AND 4 + 2)) +GROUP BY cd_gender, cd_marital_status, cd_education_status, + cd_purchase_estimate, cd_credit_rating +ORDER BY cd_gender, cd_marital_status, cd_education_status, + cd_purchase_estimate, cd_credit_rating +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q7.sql b/sql/core/src/test/resources/tpcds/q7.sql new file mode 100755 index 000000000000..6630a0054840 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q7.sql @@ -0,0 +1,19 @@ +SELECT + i_item_id, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 +FROM store_sales, customer_demographics, date_dim, item, promotion +WHERE ss_sold_date_sk = d_date_sk AND + ss_item_sk = i_item_sk AND + ss_cdemo_sk = cd_demo_sk AND + ss_promo_sk = p_promo_sk AND + cd_gender = 'M' AND + cd_marital_status = 'S' AND + cd_education_status = 'College' AND + (p_channel_email = 'N' OR p_channel_event = 'N') AND + d_year = 2000 +GROUP BY i_item_id +ORDER BY i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q70.sql b/sql/core/src/test/resources/tpcds/q70.sql new file mode 100755 index 000000000000..625011b212fe --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q70.sql @@ -0,0 +1,38 @@ +SELECT + sum(ss_net_profit) AS total_sum, + s_state, + s_county, + grouping(s_state) + grouping(s_county) AS lochierarchy, + rank() + OVER ( + PARTITION BY grouping(s_state) + grouping(s_county), + CASE WHEN grouping(s_county) = 0 + THEN s_state END + ORDER BY sum(ss_net_profit) DESC) AS rank_within_parent +FROM + store_sales, date_dim d1, store +WHERE + d1.d_month_seq BETWEEN 1200 AND 1200 + 11 + AND d1.d_date_sk = ss_sold_date_sk + AND s_store_sk = ss_store_sk + AND s_state IN + (SELECT s_state + FROM + (SELECT + s_state AS s_state, + rank() + OVER (PARTITION BY s_state + ORDER BY sum(ss_net_profit) DESC) AS ranking + FROM store_sales, store, date_dim + WHERE d_month_seq BETWEEN 1200 AND 1200 + 11 + AND d_date_sk = ss_sold_date_sk + AND s_store_sk = ss_store_sk + GROUP BY s_state) tmp1 + WHERE ranking <= 5) +GROUP BY ROLLUP (s_state, s_county) +ORDER BY + lochierarchy DESC + , CASE WHEN lochierarchy = 0 + THEN s_state END + , rank_within_parent +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q71.sql b/sql/core/src/test/resources/tpcds/q71.sql new file mode 100755 index 000000000000..8d724b9244e1 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q71.sql @@ -0,0 +1,44 @@ +SELECT + i_brand_id brand_id, + i_brand brand, + t_hour, + t_minute, + sum(ext_price) ext_price +FROM item, + (SELECT + ws_ext_sales_price AS ext_price, + ws_sold_date_sk AS sold_date_sk, + ws_item_sk AS sold_item_sk, + ws_sold_time_sk AS time_sk + FROM web_sales, date_dim + WHERE d_date_sk = ws_sold_date_sk + AND d_moy = 11 + AND d_year = 1999 + UNION ALL + SELECT + cs_ext_sales_price AS ext_price, + cs_sold_date_sk AS sold_date_sk, + cs_item_sk AS sold_item_sk, + cs_sold_time_sk AS time_sk + FROM catalog_sales, date_dim + WHERE d_date_sk = cs_sold_date_sk + AND d_moy = 11 + AND d_year = 1999 + UNION ALL + SELECT + ss_ext_sales_price AS ext_price, + ss_sold_date_sk AS sold_date_sk, + ss_item_sk AS sold_item_sk, + ss_sold_time_sk AS time_sk + FROM store_sales, date_dim + WHERE d_date_sk = ss_sold_date_sk + AND d_moy = 11 + AND d_year = 1999 + ) AS tmp, time_dim +WHERE + sold_item_sk = i_item_sk + AND i_manager_id = 1 + AND time_sk = t_time_sk + AND (t_meal_time = 'breakfast' OR t_meal_time = 'dinner') +GROUP BY i_brand, i_brand_id, t_hour, t_minute +ORDER BY ext_price DESC, brand_id diff --git a/sql/core/src/test/resources/tpcds/q72.sql b/sql/core/src/test/resources/tpcds/q72.sql new file mode 100755 index 000000000000..99b3eee54aa1 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q72.sql @@ -0,0 +1,33 @@ +SELECT + i_item_desc, + w_warehouse_name, + d1.d_week_seq, + count(CASE WHEN p_promo_sk IS NULL + THEN 1 + ELSE 0 END) no_promo, + count(CASE WHEN p_promo_sk IS NOT NULL + THEN 1 + ELSE 0 END) promo, + count(*) total_cnt +FROM catalog_sales + JOIN inventory ON (cs_item_sk = inv_item_sk) + JOIN warehouse ON (w_warehouse_sk = inv_warehouse_sk) + JOIN item ON (i_item_sk = cs_item_sk) + JOIN customer_demographics ON (cs_bill_cdemo_sk = cd_demo_sk) + JOIN household_demographics ON (cs_bill_hdemo_sk = hd_demo_sk) + JOIN date_dim d1 ON (cs_sold_date_sk = d1.d_date_sk) + JOIN date_dim d2 ON (inv_date_sk = d2.d_date_sk) + JOIN date_dim d3 ON (cs_ship_date_sk = d3.d_date_sk) + LEFT OUTER JOIN promotion ON (cs_promo_sk = p_promo_sk) + LEFT OUTER JOIN catalog_returns ON (cr_item_sk = cs_item_sk AND cr_order_number = cs_order_number) +WHERE d1.d_week_seq = d2.d_week_seq + AND inv_quantity_on_hand < cs_quantity + AND d3.d_date > (cast(d1.d_date AS DATE) + interval 5 days) + AND hd_buy_potential = '>10000' + AND d1.d_year = 1999 + AND hd_buy_potential = '>10000' + AND cd_marital_status = 'D' + AND d1.d_year = 1999 +GROUP BY i_item_desc, w_warehouse_name, d1.d_week_seq +ORDER BY total_cnt DESC, i_item_desc, w_warehouse_name, d_week_seq +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q73.sql b/sql/core/src/test/resources/tpcds/q73.sql new file mode 100755 index 000000000000..881be2e9024d --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q73.sql @@ -0,0 +1,30 @@ +SELECT + c_last_name, + c_first_name, + c_salutation, + c_preferred_cust_flag, + ss_ticket_number, + cnt +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + count(*) cnt + FROM store_sales, date_dim, store, household_demographics + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND date_dim.d_dom BETWEEN 1 AND 2 + AND (household_demographics.hd_buy_potential = '>10000' OR + household_demographics.hd_buy_potential = 'unknown') + AND household_demographics.hd_vehicle_count > 0 + AND CASE WHEN household_demographics.hd_vehicle_count > 0 + THEN + household_demographics.hd_dep_count / household_demographics.hd_vehicle_count + ELSE NULL END > 1 + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_county IN ('Williamson County', 'Franklin Parish', 'Bronx County', 'Orange County') + GROUP BY ss_ticket_number, ss_customer_sk) dj, customer +WHERE ss_customer_sk = c_customer_sk + AND cnt BETWEEN 1 AND 5 +ORDER BY cnt DESC diff --git a/sql/core/src/test/resources/tpcds/q74.sql b/sql/core/src/test/resources/tpcds/q74.sql new file mode 100755 index 000000000000..154b26d6802a --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q74.sql @@ -0,0 +1,58 @@ +WITH year_total AS ( + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + d_year AS year, + sum(ss_net_paid) year_total, + 's' sale_type + FROM + customer, store_sales, date_dim + WHERE c_customer_sk = ss_customer_sk + AND ss_sold_date_sk = d_date_sk + AND d_year IN (2001, 2001 + 1) + GROUP BY + c_customer_id, c_first_name, c_last_name, d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + d_year AS year, + sum(ws_net_paid) year_total, + 'w' sale_type + FROM + customer, web_sales, date_dim + WHERE c_customer_sk = ws_bill_customer_sk + AND ws_sold_date_sk = d_date_sk + AND d_year IN (2001, 2001 + 1) + GROUP BY + c_customer_id, c_first_name, c_last_name, d_year) +SELECT + t_s_secyear.customer_id, + t_s_secyear.customer_first_name, + t_s_secyear.customer_last_name +FROM + year_total t_s_firstyear, year_total t_s_secyear, + year_total t_w_firstyear, year_total t_w_secyear +WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_secyear.customer_id + AND t_s_firstyear.customer_id = t_w_firstyear.customer_id + AND t_s_firstyear.sale_type = 's' + AND t_w_firstyear.sale_type = 'w' + AND t_s_secyear.sale_type = 's' + AND t_w_secyear.sale_type = 'w' + AND t_s_firstyear.year = 2001 + AND t_s_secyear.year = 2001 + 1 + AND t_w_firstyear.year = 2001 + AND t_w_secyear.year = 2001 + 1 + AND t_s_firstyear.year_total > 0 + AND t_w_firstyear.year_total > 0 + AND CASE WHEN t_w_firstyear.year_total > 0 + THEN t_w_secyear.year_total / t_w_firstyear.year_total + ELSE NULL END + > CASE WHEN t_s_firstyear.year_total > 0 + THEN t_s_secyear.year_total / t_s_firstyear.year_total + ELSE NULL END +ORDER BY 1, 1, 1 +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q75.sql b/sql/core/src/test/resources/tpcds/q75.sql new file mode 100755 index 000000000000..2a143232b519 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q75.sql @@ -0,0 +1,76 @@ +WITH all_sales AS ( + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + SUM(sales_cnt) AS sales_cnt, + SUM(sales_amt) AS sales_amt + FROM ( + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + cs_quantity - COALESCE(cr_return_quantity, 0) AS sales_cnt, + cs_ext_sales_price - COALESCE(cr_return_amount, 0.0) AS sales_amt + FROM catalog_sales + JOIN item ON i_item_sk = cs_item_sk + JOIN date_dim ON d_date_sk = cs_sold_date_sk + LEFT JOIN catalog_returns ON (cs_order_number = cr_order_number + AND cs_item_sk = cr_item_sk) + WHERE i_category = 'Books' + UNION + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + ss_quantity - COALESCE(sr_return_quantity, 0) AS sales_cnt, + ss_ext_sales_price - COALESCE(sr_return_amt, 0.0) AS sales_amt + FROM store_sales + JOIN item ON i_item_sk = ss_item_sk + JOIN date_dim ON d_date_sk = ss_sold_date_sk + LEFT JOIN store_returns ON (ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk) + WHERE i_category = 'Books' + UNION + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + ws_quantity - COALESCE(wr_return_quantity, 0) AS sales_cnt, + ws_ext_sales_price - COALESCE(wr_return_amt, 0.0) AS sales_amt + FROM web_sales + JOIN item ON i_item_sk = ws_item_sk + JOIN date_dim ON d_date_sk = ws_sold_date_sk + LEFT JOIN web_returns ON (ws_order_number = wr_order_number + AND ws_item_sk = wr_item_sk) + WHERE i_category = 'Books') sales_detail + GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id) +SELECT + prev_yr.d_year AS prev_year, + curr_yr.d_year AS year, + curr_yr.i_brand_id, + curr_yr.i_class_id, + curr_yr.i_category_id, + curr_yr.i_manufact_id, + prev_yr.sales_cnt AS prev_yr_cnt, + curr_yr.sales_cnt AS curr_yr_cnt, + curr_yr.sales_cnt - prev_yr.sales_cnt AS sales_cnt_diff, + curr_yr.sales_amt - prev_yr.sales_amt AS sales_amt_diff +FROM all_sales curr_yr, all_sales prev_yr +WHERE curr_yr.i_brand_id = prev_yr.i_brand_id + AND curr_yr.i_class_id = prev_yr.i_class_id + AND curr_yr.i_category_id = prev_yr.i_category_id + AND curr_yr.i_manufact_id = prev_yr.i_manufact_id + AND curr_yr.d_year = 2002 + AND prev_yr.d_year = 2002 - 1 + AND CAST(curr_yr.sales_cnt AS DECIMAL(17, 2)) / CAST(prev_yr.sales_cnt AS DECIMAL(17, 2)) < 0.9 +ORDER BY sales_cnt_diff +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q76.sql b/sql/core/src/test/resources/tpcds/q76.sql new file mode 100755 index 000000000000..815fa922be19 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q76.sql @@ -0,0 +1,47 @@ +SELECT + channel, + col_name, + d_year, + d_qoy, + i_category, + COUNT(*) sales_cnt, + SUM(ext_sales_price) sales_amt +FROM ( + SELECT + 'store' AS channel, + ss_store_sk col_name, + d_year, + d_qoy, + i_category, + ss_ext_sales_price ext_sales_price + FROM store_sales, item, date_dim + WHERE ss_store_sk IS NULL + AND ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + UNION ALL + SELECT + 'web' AS channel, + ws_ship_customer_sk col_name, + d_year, + d_qoy, + i_category, + ws_ext_sales_price ext_sales_price + FROM web_sales, item, date_dim + WHERE ws_ship_customer_sk IS NULL + AND ws_sold_date_sk = d_date_sk + AND ws_item_sk = i_item_sk + UNION ALL + SELECT + 'catalog' AS channel, + cs_ship_addr_sk col_name, + d_year, + d_qoy, + i_category, + cs_ext_sales_price ext_sales_price + FROM catalog_sales, item, date_dim + WHERE cs_ship_addr_sk IS NULL + AND cs_sold_date_sk = d_date_sk + AND cs_item_sk = i_item_sk) foo +GROUP BY channel, col_name, d_year, d_qoy, i_category +ORDER BY channel, col_name, d_year, d_qoy, i_category +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q77.sql b/sql/core/src/test/resources/tpcds/q77.sql new file mode 100755 index 000000000000..a69df9fbcd36 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q77.sql @@ -0,0 +1,100 @@ +WITH ss AS +(SELECT + s_store_sk, + sum(ss_ext_sales_price) AS sales, + sum(ss_net_profit) AS profit + FROM store_sales, date_dim, store + WHERE ss_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND ss_store_sk = s_store_sk + GROUP BY s_store_sk), + sr AS + (SELECT + s_store_sk, + sum(sr_return_amt) AS returns, + sum(sr_net_loss) AS profit_loss + FROM store_returns, date_dim, store + WHERE sr_returned_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND sr_store_sk = s_store_sk + GROUP BY s_store_sk), + cs AS + (SELECT + cs_call_center_sk, + sum(cs_ext_sales_price) AS sales, + sum(cs_net_profit) AS profit + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + GROUP BY cs_call_center_sk), + cr AS + (SELECT + sum(cr_return_amount) AS returns, + sum(cr_net_loss) AS profit_loss + FROM catalog_returns, date_dim + WHERE cr_returned_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days)), + ws AS + (SELECT + wp_web_page_sk, + sum(ws_ext_sales_price) AS sales, + sum(ws_net_profit) AS profit + FROM web_sales, date_dim, web_page + WHERE ws_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND ws_web_page_sk = wp_web_page_sk + GROUP BY wp_web_page_sk), + wr AS + (SELECT + wp_web_page_sk, + sum(wr_return_amt) AS returns, + sum(wr_net_loss) AS profit_loss + FROM web_returns, date_dim, web_page + WHERE wr_returned_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND wr_web_page_sk = wp_web_page_sk + GROUP BY wp_web_page_sk) +SELECT + channel, + id, + sum(sales) AS sales, + sum(returns) AS returns, + sum(profit) AS profit +FROM + (SELECT + 'store channel' AS channel, + ss.s_store_sk AS id, + sales, + coalesce(returns, 0) AS returns, + (profit - coalesce(profit_loss, 0)) AS profit + FROM ss + LEFT JOIN sr + ON ss.s_store_sk = sr.s_store_sk + UNION ALL + SELECT + 'catalog channel' AS channel, + cs_call_center_sk AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM cs, cr + UNION ALL + SELECT + 'web channel' AS channel, + ws.wp_web_page_sk AS id, + sales, + coalesce(returns, 0) returns, + (profit - coalesce(profit_loss, 0)) AS profit + FROM ws + LEFT JOIN wr + ON ws.wp_web_page_sk = wr.wp_web_page_sk + ) x +GROUP BY ROLLUP (channel, id) +ORDER BY channel, id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q78.sql b/sql/core/src/test/resources/tpcds/q78.sql new file mode 100755 index 000000000000..07b0940e2688 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q78.sql @@ -0,0 +1,64 @@ +WITH ws AS +(SELECT + d_year AS ws_sold_year, + ws_item_sk, + ws_bill_customer_sk ws_customer_sk, + sum(ws_quantity) ws_qty, + sum(ws_wholesale_cost) ws_wc, + sum(ws_sales_price) ws_sp + FROM web_sales + LEFT JOIN web_returns ON wr_order_number = ws_order_number AND ws_item_sk = wr_item_sk + JOIN date_dim ON ws_sold_date_sk = d_date_sk + WHERE wr_order_number IS NULL + GROUP BY d_year, ws_item_sk, ws_bill_customer_sk +), + cs AS + (SELECT + d_year AS cs_sold_year, + cs_item_sk, + cs_bill_customer_sk cs_customer_sk, + sum(cs_quantity) cs_qty, + sum(cs_wholesale_cost) cs_wc, + sum(cs_sales_price) cs_sp + FROM catalog_sales + LEFT JOIN catalog_returns ON cr_order_number = cs_order_number AND cs_item_sk = cr_item_sk + JOIN date_dim ON cs_sold_date_sk = d_date_sk + WHERE cr_order_number IS NULL + GROUP BY d_year, cs_item_sk, cs_bill_customer_sk + ), + ss AS + (SELECT + d_year AS ss_sold_year, + ss_item_sk, + ss_customer_sk, + sum(ss_quantity) ss_qty, + sum(ss_wholesale_cost) ss_wc, + sum(ss_sales_price) ss_sp + FROM store_sales + LEFT JOIN store_returns ON sr_ticket_number = ss_ticket_number AND ss_item_sk = sr_item_sk + JOIN date_dim ON ss_sold_date_sk = d_date_sk + WHERE sr_ticket_number IS NULL + GROUP BY d_year, ss_item_sk, ss_customer_sk + ) +SELECT + round(ss_qty / (coalesce(ws_qty + cs_qty, 1)), 2) ratio, + ss_qty store_qty, + ss_wc store_wholesale_cost, + ss_sp store_sales_price, + coalesce(ws_qty, 0) + coalesce(cs_qty, 0) other_chan_qty, + coalesce(ws_wc, 0) + coalesce(cs_wc, 0) other_chan_wholesale_cost, + coalesce(ws_sp, 0) + coalesce(cs_sp, 0) other_chan_sales_price +FROM ss + LEFT JOIN ws + ON (ws_sold_year = ss_sold_year AND ws_item_sk = ss_item_sk AND ws_customer_sk = ss_customer_sk) + LEFT JOIN cs + ON (cs_sold_year = ss_sold_year AND cs_item_sk = ss_item_sk AND cs_customer_sk = ss_customer_sk) +WHERE coalesce(ws_qty, 0) > 0 AND coalesce(cs_qty, 0) > 0 AND ss_sold_year = 2000 +ORDER BY + ratio, + ss_qty DESC, ss_wc DESC, ss_sp DESC, + other_chan_qty, + other_chan_wholesale_cost, + other_chan_sales_price, + round(ss_qty / (coalesce(ws_qty + cs_qty, 1)), 2) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q79.sql b/sql/core/src/test/resources/tpcds/q79.sql new file mode 100755 index 000000000000..08f86dc2032a --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q79.sql @@ -0,0 +1,27 @@ +SELECT + c_last_name, + c_first_name, + substr(s_city, 1, 30), + ss_ticket_number, + amt, + profit +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + store.s_city, + sum(ss_coupon_amt) amt, + sum(ss_net_profit) profit + FROM store_sales, date_dim, store, household_demographics + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND (household_demographics.hd_dep_count = 6 OR + household_demographics.hd_vehicle_count > 2) + AND date_dim.d_dow = 1 + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_number_employees BETWEEN 200 AND 295 + GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, store.s_city) ms, customer +WHERE ss_customer_sk = c_customer_sk +ORDER BY c_last_name, c_first_name, substr(s_city, 1, 30), profit +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q8.sql b/sql/core/src/test/resources/tpcds/q8.sql new file mode 100755 index 000000000000..497725111f4f --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q8.sql @@ -0,0 +1,87 @@ +SELECT + s_store_name, + sum(ss_net_profit) +FROM store_sales, date_dim, store, + (SELECT ca_zip + FROM ( + (SELECT substr(ca_zip, 1, 5) ca_zip + FROM customer_address + WHERE substr(ca_zip, 1, 5) IN ( + '24128','76232','65084','87816','83926','77556','20548', + '26231','43848','15126','91137','61265','98294','25782', + '17920','18426','98235','40081','84093','28577','55565', + '17183','54601','67897','22752','86284','18376','38607', + '45200','21756','29741','96765','23932','89360','29839', + '25989','28898','91068','72550','10390','18845','47770', + '82636','41367','76638','86198','81312','37126','39192', + '88424','72175','81426','53672','10445','42666','66864', + '66708','41248','48583','82276','18842','78890','49448', + '14089','38122','34425','79077','19849','43285','39861', + '66162','77610','13695','99543','83444','83041','12305', + '57665','68341','25003','57834','62878','49130','81096', + '18840','27700','23470','50412','21195','16021','76107', + '71954','68309','18119','98359','64544','10336','86379', + '27068','39736','98569','28915','24206','56529','57647', + '54917','42961','91110','63981','14922','36420','23006', + '67467','32754','30903','20260','31671','51798','72325', + '85816','68621','13955','36446','41766','68806','16725', + '15146','22744','35850','88086','51649','18270','52867', + '39972','96976','63792','11376','94898','13595','10516', + '90225','58943','39371','94945','28587','96576','57855', + '28488','26105','83933','25858','34322','44438','73171', + '30122','34102','22685','71256','78451','54364','13354', + '45375','40558','56458','28286','45266','47305','69399', + '83921','26233','11101','15371','69913','35942','15882', + '25631','24610','44165','99076','33786','70738','26653', + '14328','72305','62496','22152','10144','64147','48425', + '14663','21076','18799','30450','63089','81019','68893', + '24996','51200','51211','45692','92712','70466','79994', + '22437','25280','38935','71791','73134','56571','14060', + '19505','72425','56575','74351','68786','51650','20004', + '18383','76614','11634','18906','15765','41368','73241', + '76698','78567','97189','28545','76231','75691','22246', + '51061','90578','56691','68014','51103','94167','57047', + '14867','73520','15734','63435','25733','35474','24676', + '94627','53535','17879','15559','53268','59166','11928', + '59402','33282','45721','43933','68101','33515','36634', + '71286','19736','58058','55253','67473','41918','19515', + '36495','19430','22351','77191','91393','49156','50298', + '87501','18652','53179','18767','63193','23968','65164', + '68880','21286','72823','58470','67301','13394','31016', + '70372','67030','40604','24317','45748','39127','26065', + '77721','31029','31880','60576','24671','45549','13376', + '50016','33123','19769','22927','97789','46081','72151', + '15723','46136','51949','68100','96888','64528','14171', + '79777','28709','11489','25103','32213','78668','22245', + '15798','27156','37930','62971','21337','51622','67853', + '10567','38415','15455','58263','42029','60279','37125', + '56240','88190','50308','26859','64457','89091','82136', + '62377','36233','63837','58078','17043','30010','60099', + '28810','98025','29178','87343','73273','30469','64034', + '39516','86057','21309','90257','67875','40162','11356', + '73650','61810','72013','30431','22461','19512','13375', + '55307','30625','83849','68908','26689','96451','38193', + '46820','88885','84935','69035','83144','47537','56616', + '94983','48033','69952','25486','61547','27385','61860', + '58048','56910','16807','17871','35258','31387','35458', + '35576')) + INTERSECT + (SELECT ca_zip + FROM + (SELECT + substr(ca_zip, 1, 5) ca_zip, + count(*) cnt + FROM customer_address, customer + WHERE ca_address_sk = c_current_addr_sk AND + c_preferred_cust_flag = 'Y' + GROUP BY ca_zip + HAVING count(*) > 10) A1) + ) A2 + ) V1 +WHERE ss_store_sk = s_store_sk + AND ss_sold_date_sk = d_date_sk + AND d_qoy = 2 AND d_year = 1998 + AND (substr(s_zip, 1, 2) = substr(V1.ca_zip, 1, 2)) +GROUP BY s_store_name +ORDER BY s_store_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q80.sql b/sql/core/src/test/resources/tpcds/q80.sql new file mode 100755 index 000000000000..433db87d2a85 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q80.sql @@ -0,0 +1,94 @@ +WITH ssr AS +(SELECT + s_store_id AS store_id, + sum(ss_ext_sales_price) AS sales, + sum(coalesce(sr_return_amt, 0)) AS returns, + sum(ss_net_profit - coalesce(sr_net_loss, 0)) AS profit + FROM store_sales + LEFT OUTER JOIN store_returns ON + (ss_item_sk = sr_item_sk AND + ss_ticket_number = sr_ticket_number) + , + date_dim, store, item, promotion + WHERE ss_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) + AND ss_store_sk = s_store_sk + AND ss_item_sk = i_item_sk + AND i_current_price > 50 + AND ss_promo_sk = p_promo_sk + AND p_channel_tv = 'N' + GROUP BY s_store_id), + csr AS + (SELECT + cp_catalog_page_id AS catalog_page_id, + sum(cs_ext_sales_price) AS sales, + sum(coalesce(cr_return_amount, 0)) AS returns, + sum(cs_net_profit - coalesce(cr_net_loss, 0)) AS profit + FROM catalog_sales + LEFT OUTER JOIN catalog_returns ON + (cs_item_sk = cr_item_sk AND + cs_order_number = cr_order_number) + , + date_dim, catalog_page, item, promotion + WHERE cs_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) + AND cs_catalog_page_sk = cp_catalog_page_sk + AND cs_item_sk = i_item_sk + AND i_current_price > 50 + AND cs_promo_sk = p_promo_sk + AND p_channel_tv = 'N' + GROUP BY cp_catalog_page_id), + wsr AS + (SELECT + web_site_id, + sum(ws_ext_sales_price) AS sales, + sum(coalesce(wr_return_amt, 0)) AS returns, + sum(ws_net_profit - coalesce(wr_net_loss, 0)) AS profit + FROM web_sales + LEFT OUTER JOIN web_returns ON + (ws_item_sk = wr_item_sk AND ws_order_number = wr_order_number) + , + date_dim, web_site, item, promotion + WHERE ws_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) + AND ws_web_site_sk = web_site_sk + AND ws_item_sk = i_item_sk + AND i_current_price > 50 + AND ws_promo_sk = p_promo_sk + AND p_channel_tv = 'N' + GROUP BY web_site_id) +SELECT + channel, + id, + sum(sales) AS sales, + sum(returns) AS returns, + sum(profit) AS profit +FROM (SELECT + 'store channel' AS channel, + concat('store', store_id) AS id, + sales, + returns, + profit + FROM ssr + UNION ALL + SELECT + 'catalog channel' AS channel, + concat('catalog_page', catalog_page_id) AS id, + sales, + returns, + profit + FROM csr + UNION ALL + SELECT + 'web channel' AS channel, + concat('web_site', web_site_id) AS id, + sales, + returns, + profit + FROM wsr) x +GROUP BY ROLLUP (channel, id) +ORDER BY channel, id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q81.sql b/sql/core/src/test/resources/tpcds/q81.sql new file mode 100755 index 000000000000..18f0ffa7e8f4 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q81.sql @@ -0,0 +1,38 @@ +WITH customer_total_return AS +(SELECT + cr_returning_customer_sk AS ctr_customer_sk, + ca_state AS ctr_state, + sum(cr_return_amt_inc_tax) AS ctr_total_return + FROM catalog_returns, date_dim, customer_address + WHERE cr_returned_date_sk = d_date_sk + AND d_year = 2000 + AND cr_returning_addr_sk = ca_address_sk + GROUP BY cr_returning_customer_sk, ca_state ) +SELECT + c_customer_id, + c_salutation, + c_first_name, + c_last_name, + ca_street_number, + ca_street_name, + ca_street_type, + ca_suite_number, + ca_city, + ca_county, + ca_state, + ca_zip, + ca_country, + ca_gmt_offset, + ca_location_type, + ctr_total_return +FROM customer_total_return ctr1, customer_address, customer +WHERE ctr1.ctr_total_return > (SELECT avg(ctr_total_return) * 1.2 +FROM customer_total_return ctr2 +WHERE ctr1.ctr_state = ctr2.ctr_state) + AND ca_address_sk = c_current_addr_sk + AND ca_state = 'GA' + AND ctr1.ctr_customer_sk = c_customer_sk +ORDER BY c_customer_id, c_salutation, c_first_name, c_last_name, ca_street_number, ca_street_name + , ca_street_type, ca_suite_number, ca_city, ca_county, ca_state, ca_zip, ca_country, ca_gmt_offset + , ca_location_type, ctr_total_return +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q82.sql b/sql/core/src/test/resources/tpcds/q82.sql new file mode 100755 index 000000000000..20942cfeb078 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q82.sql @@ -0,0 +1,15 @@ +SELECT + i_item_id, + i_item_desc, + i_current_price +FROM item, inventory, date_dim, store_sales +WHERE i_current_price BETWEEN 62 AND 62 + 30 + AND inv_item_sk = i_item_sk + AND d_date_sk = inv_date_sk + AND d_date BETWEEN cast('2000-05-25' AS DATE) AND (cast('2000-05-25' AS DATE) + INTERVAL 60 days) + AND i_manufact_id IN (129, 270, 821, 423) + AND inv_quantity_on_hand BETWEEN 100 AND 500 + AND ss_item_sk = i_item_sk +GROUP BY i_item_id, i_item_desc, i_current_price +ORDER BY i_item_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q83.sql b/sql/core/src/test/resources/tpcds/q83.sql new file mode 100755 index 000000000000..53c10c7ded6c --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q83.sql @@ -0,0 +1,56 @@ +WITH sr_items AS +(SELECT + i_item_id item_id, + sum(sr_return_quantity) sr_item_qty + FROM store_returns, item, date_dim + WHERE sr_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq IN + (SELECT d_week_seq + FROM date_dim + WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) + AND sr_returned_date_sk = d_date_sk + GROUP BY i_item_id), + cr_items AS + (SELECT + i_item_id item_id, + sum(cr_return_quantity) cr_item_qty + FROM catalog_returns, item, date_dim + WHERE cr_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq IN + (SELECT d_week_seq + FROM date_dim + WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) + AND cr_returned_date_sk = d_date_sk + GROUP BY i_item_id), + wr_items AS + (SELECT + i_item_id item_id, + sum(wr_return_quantity) wr_item_qty + FROM web_returns, item, date_dim + WHERE wr_item_sk = i_item_sk AND d_date IN + (SELECT d_date + FROM date_dim + WHERE d_week_seq IN + (SELECT d_week_seq + FROM date_dim + WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) + AND wr_returned_date_sk = d_date_sk + GROUP BY i_item_id) +SELECT + sr_items.item_id, + sr_item_qty, + sr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 sr_dev, + cr_item_qty, + cr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 cr_dev, + wr_item_qty, + wr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 wr_dev, + (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 average +FROM sr_items, cr_items, wr_items +WHERE sr_items.item_id = cr_items.item_id + AND sr_items.item_id = wr_items.item_id +ORDER BY sr_items.item_id, sr_item_qty +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q84.sql b/sql/core/src/test/resources/tpcds/q84.sql new file mode 100755 index 000000000000..a1076b57ced5 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q84.sql @@ -0,0 +1,19 @@ +SELECT + c_customer_id AS customer_id, + concat(c_last_name, ', ', c_first_name) AS customername +FROM customer + , customer_address + , customer_demographics + , household_demographics + , income_band + , store_returns +WHERE ca_city = 'Edgewood' + AND c_current_addr_sk = ca_address_sk + AND ib_lower_bound >= 38128 + AND ib_upper_bound <= 38128 + 50000 + AND ib_income_band_sk = hd_income_band_sk + AND cd_demo_sk = c_current_cdemo_sk + AND hd_demo_sk = c_current_hdemo_sk + AND sr_cdemo_sk = cd_demo_sk +ORDER BY c_customer_id +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q85.sql b/sql/core/src/test/resources/tpcds/q85.sql new file mode 100755 index 000000000000..cf718b0f8ade --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q85.sql @@ -0,0 +1,82 @@ +SELECT + substr(r_reason_desc, 1, 20), + avg(ws_quantity), + avg(wr_refunded_cash), + avg(wr_fee) +FROM web_sales, web_returns, web_page, customer_demographics cd1, + customer_demographics cd2, customer_address, date_dim, reason +WHERE ws_web_page_sk = wp_web_page_sk + AND ws_item_sk = wr_item_sk + AND ws_order_number = wr_order_number + AND ws_sold_date_sk = d_date_sk AND d_year = 2000 + AND cd1.cd_demo_sk = wr_refunded_cdemo_sk + AND cd2.cd_demo_sk = wr_returning_cdemo_sk + AND ca_address_sk = wr_refunded_addr_sk + AND r_reason_sk = wr_reason_sk + AND + ( + ( + cd1.cd_marital_status = 'M' + AND + cd1.cd_marital_status = cd2.cd_marital_status + AND + cd1.cd_education_status = 'Advanced Degree' + AND + cd1.cd_education_status = cd2.cd_education_status + AND + ws_sales_price BETWEEN 100.00 AND 150.00 + ) + OR + ( + cd1.cd_marital_status = 'S' + AND + cd1.cd_marital_status = cd2.cd_marital_status + AND + cd1.cd_education_status = 'College' + AND + cd1.cd_education_status = cd2.cd_education_status + AND + ws_sales_price BETWEEN 50.00 AND 100.00 + ) + OR + ( + cd1.cd_marital_status = 'W' + AND + cd1.cd_marital_status = cd2.cd_marital_status + AND + cd1.cd_education_status = '2 yr Degree' + AND + cd1.cd_education_status = cd2.cd_education_status + AND + ws_sales_price BETWEEN 150.00 AND 200.00 + ) + ) + AND + ( + ( + ca_country = 'United States' + AND + ca_state IN ('IN', 'OH', 'NJ') + AND ws_net_profit BETWEEN 100 AND 200 + ) + OR + ( + ca_country = 'United States' + AND + ca_state IN ('WI', 'CT', 'KY') + AND ws_net_profit BETWEEN 150 AND 300 + ) + OR + ( + ca_country = 'United States' + AND + ca_state IN ('LA', 'IA', 'AR') + AND ws_net_profit BETWEEN 50 AND 250 + ) + ) +GROUP BY r_reason_desc +ORDER BY substr(r_reason_desc, 1, 20) + , avg(ws_quantity) + , avg(wr_refunded_cash) + , avg(wr_fee) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q86.sql b/sql/core/src/test/resources/tpcds/q86.sql new file mode 100755 index 000000000000..789a4abf7b5f --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q86.sql @@ -0,0 +1,24 @@ +SELECT + sum(ws_net_paid) AS total_sum, + i_category, + i_class, + grouping(i_category) + grouping(i_class) AS lochierarchy, + rank() + OVER ( + PARTITION BY grouping(i_category) + grouping(i_class), + CASE WHEN grouping(i_class) = 0 + THEN i_category END + ORDER BY sum(ws_net_paid) DESC) AS rank_within_parent +FROM + web_sales, date_dim d1, item +WHERE + d1.d_month_seq BETWEEN 1200 AND 1200 + 11 + AND d1.d_date_sk = ws_sold_date_sk + AND i_item_sk = ws_item_sk +GROUP BY ROLLUP (i_category, i_class) +ORDER BY + lochierarchy DESC, + CASE WHEN lochierarchy = 0 + THEN i_category END, + rank_within_parent +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q87.sql b/sql/core/src/test/resources/tpcds/q87.sql new file mode 100755 index 000000000000..4aaa9f39dce9 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q87.sql @@ -0,0 +1,28 @@ +SELECT count(*) +FROM ((SELECT DISTINCT + c_last_name, + c_first_name, + d_date +FROM store_sales, date_dim, customer +WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11) + EXCEPT + (SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM catalog_sales, date_dim, customer + WHERE catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11) + EXCEPT + (SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM web_sales, date_dim, customer + WHERE web_sales.ws_sold_date_sk = date_dim.d_date_sk + AND web_sales.ws_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11) + ) cool_cust diff --git a/sql/core/src/test/resources/tpcds/q88.sql b/sql/core/src/test/resources/tpcds/q88.sql new file mode 100755 index 000000000000..25bcd90f41ab --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q88.sql @@ -0,0 +1,122 @@ +SELECT * +FROM + (SELECT count(*) h8_30_to_9 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 8 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s1, + (SELECT count(*) h9_to_9_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 9 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s2, + (SELECT count(*) h9_30_to_10 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 9 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s3, + (SELECT count(*) h10_to_10_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 10 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s4, + (SELECT count(*) h10_30_to_11 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 10 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s5, + (SELECT count(*) h11_to_11_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 11 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s6, + (SELECT count(*) h11_30_to_12 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 11 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s7, + (SELECT count(*) h12_to_12_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 12 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s8 diff --git a/sql/core/src/test/resources/tpcds/q89.sql b/sql/core/src/test/resources/tpcds/q89.sql new file mode 100755 index 000000000000..75408cb0323f --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q89.sql @@ -0,0 +1,30 @@ +SELECT * +FROM ( + SELECT + i_category, + i_class, + i_brand, + s_store_name, + s_company_name, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER + (PARTITION BY i_category, i_brand, s_store_name, s_company_name) + avg_monthly_sales + FROM item, store_sales, date_dim, store + WHERE ss_item_sk = i_item_sk AND + ss_sold_date_sk = d_date_sk AND + ss_store_sk = s_store_sk AND + d_year IN (1999) AND + ((i_category IN ('Books', 'Electronics', 'Sports') AND + i_class IN ('computers', 'stereo', 'football')) + OR (i_category IN ('Men', 'Jewelry', 'Women') AND + i_class IN ('shirts', 'birdal', 'dresses'))) + GROUP BY i_category, i_class, i_brand, + s_store_name, s_company_name, d_moy) tmp1 +WHERE CASE WHEN (avg_monthly_sales <> 0) + THEN (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) + ELSE NULL END > 0.1 +ORDER BY sum_sales - avg_monthly_sales, s_store_name +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q9.sql b/sql/core/src/test/resources/tpcds/q9.sql new file mode 100755 index 000000000000..de3db9d988f1 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q9.sql @@ -0,0 +1,48 @@ +SELECT + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 1 AND 20) > 62316685 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 1 AND 20) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 1 AND 20) END bucket1, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 40) > 19045798 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 40) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 40) END bucket2, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 41 AND 60) > 365541424 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 41 AND 60) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 41 AND 60) END bucket3, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 61 AND 80) > 216357808 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 61 AND 80) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 61 AND 80) END bucket4, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 81 AND 100) > 184483884 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 81 AND 100) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 81 AND 100) END bucket5 +FROM reason +WHERE r_reason_sk = 1 diff --git a/sql/core/src/test/resources/tpcds/q90.sql b/sql/core/src/test/resources/tpcds/q90.sql new file mode 100755 index 000000000000..85e35bf8bf8e --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q90.sql @@ -0,0 +1,19 @@ +SELECT cast(amc AS DECIMAL(15, 4)) / cast(pmc AS DECIMAL(15, 4)) am_pm_ratio +FROM (SELECT count(*) amc +FROM web_sales, household_demographics, time_dim, web_page +WHERE ws_sold_time_sk = time_dim.t_time_sk + AND ws_ship_hdemo_sk = household_demographics.hd_demo_sk + AND ws_web_page_sk = web_page.wp_web_page_sk + AND time_dim.t_hour BETWEEN 8 AND 8 + 1 + AND household_demographics.hd_dep_count = 6 + AND web_page.wp_char_count BETWEEN 5000 AND 5200) at, + (SELECT count(*) pmc + FROM web_sales, household_demographics, time_dim, web_page + WHERE ws_sold_time_sk = time_dim.t_time_sk + AND ws_ship_hdemo_sk = household_demographics.hd_demo_sk + AND ws_web_page_sk = web_page.wp_web_page_sk + AND time_dim.t_hour BETWEEN 19 AND 19 + 1 + AND household_demographics.hd_dep_count = 6 + AND web_page.wp_char_count BETWEEN 5000 AND 5200) pt +ORDER BY am_pm_ratio +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q91.sql b/sql/core/src/test/resources/tpcds/q91.sql new file mode 100755 index 000000000000..9ca7ce00ac77 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q91.sql @@ -0,0 +1,23 @@ +SELECT + cc_call_center_id Call_Center, + cc_name Call_Center_Name, + cc_manager Manager, + sum(cr_net_loss) Returns_Loss +FROM + call_center, catalog_returns, date_dim, customer, customer_address, + customer_demographics, household_demographics +WHERE + cr_call_center_sk = cc_call_center_sk + AND cr_returned_date_sk = d_date_sk + AND cr_returning_customer_sk = c_customer_sk + AND cd_demo_sk = c_current_cdemo_sk + AND hd_demo_sk = c_current_hdemo_sk + AND ca_address_sk = c_current_addr_sk + AND d_year = 1998 + AND d_moy = 11 + AND ((cd_marital_status = 'M' AND cd_education_status = 'Unknown') + OR (cd_marital_status = 'W' AND cd_education_status = 'Advanced Degree')) + AND hd_buy_potential LIKE 'Unknown%' + AND ca_gmt_offset = -7 +GROUP BY cc_call_center_id, cc_name, cc_manager, cd_marital_status, cd_education_status +ORDER BY sum(cr_net_loss) DESC diff --git a/sql/core/src/test/resources/tpcds/q92.sql b/sql/core/src/test/resources/tpcds/q92.sql new file mode 100755 index 000000000000..99129c3bd9e5 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q92.sql @@ -0,0 +1,16 @@ +SELECT sum(ws_ext_discount_amt) AS `Excess Discount Amount ` +FROM web_sales, item, date_dim +WHERE i_manufact_id = 350 + AND i_item_sk = ws_item_sk + AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + INTERVAL 90 days) + AND d_date_sk = ws_sold_date_sk + AND ws_ext_discount_amt > + ( + SELECT 1.3 * avg(ws_ext_discount_amt) + FROM web_sales, date_dim + WHERE ws_item_sk = i_item_sk + AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + INTERVAL 90 days) + AND d_date_sk = ws_sold_date_sk + ) +ORDER BY sum(ws_ext_discount_amt) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q93.sql b/sql/core/src/test/resources/tpcds/q93.sql new file mode 100755 index 000000000000..222dc31c1f56 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q93.sql @@ -0,0 +1,19 @@ +SELECT + ss_customer_sk, + sum(act_sales) sumsales +FROM (SELECT + ss_item_sk, + ss_ticket_number, + ss_customer_sk, + CASE WHEN sr_return_quantity IS NOT NULL + THEN (ss_quantity - sr_return_quantity) * ss_sales_price + ELSE (ss_quantity * ss_sales_price) END act_sales +FROM store_sales + LEFT OUTER JOIN store_returns + ON (sr_item_sk = ss_item_sk AND sr_ticket_number = ss_ticket_number) + , + reason +WHERE sr_reason_sk = r_reason_sk AND r_reason_desc = 'reason 28') t +GROUP BY ss_customer_sk +ORDER BY sumsales, ss_customer_sk +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q94.sql b/sql/core/src/test/resources/tpcds/q94.sql new file mode 100755 index 000000000000..d6de3d75b82d --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q94.sql @@ -0,0 +1,23 @@ +SELECT + count(DISTINCT ws_order_number) AS `order count `, + sum(ws_ext_ship_cost) AS `total shipping cost `, + sum(ws_net_profit) AS `total net profit ` +FROM + web_sales ws1, date_dim, customer_address, web_site +WHERE + d_date BETWEEN '1999-02-01' AND + (CAST('1999-02-01' AS DATE) + INTERVAL 60 days) + AND ws1.ws_ship_date_sk = d_date_sk + AND ws1.ws_ship_addr_sk = ca_address_sk + AND ca_state = 'IL' + AND ws1.ws_web_site_sk = web_site_sk + AND web_company_name = 'pri' + AND EXISTS(SELECT * + FROM web_sales ws2 + WHERE ws1.ws_order_number = ws2.ws_order_number + AND ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) + AND NOT EXISTS(SELECT * + FROM web_returns wr1 + WHERE ws1.ws_order_number = wr1.wr_order_number) +ORDER BY count(DISTINCT ws_order_number) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q95.sql b/sql/core/src/test/resources/tpcds/q95.sql new file mode 100755 index 000000000000..df71f00bd6c0 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q95.sql @@ -0,0 +1,29 @@ +WITH ws_wh AS +(SELECT + ws1.ws_order_number, + ws1.ws_warehouse_sk wh1, + ws2.ws_warehouse_sk wh2 + FROM web_sales ws1, web_sales ws2 + WHERE ws1.ws_order_number = ws2.ws_order_number + AND ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) +SELECT + count(DISTINCT ws_order_number) AS `order count `, + sum(ws_ext_ship_cost) AS `total shipping cost `, + sum(ws_net_profit) AS `total net profit ` +FROM + web_sales ws1, date_dim, customer_address, web_site +WHERE + d_date BETWEEN '1999-02-01' AND + (CAST('1999-02-01' AS DATE) + INTERVAL 60 DAY) + AND ws1.ws_ship_date_sk = d_date_sk + AND ws1.ws_ship_addr_sk = ca_address_sk + AND ca_state = 'IL' + AND ws1.ws_web_site_sk = web_site_sk + AND web_company_name = 'pri' + AND ws1.ws_order_number IN (SELECT ws_order_number + FROM ws_wh) + AND ws1.ws_order_number IN (SELECT wr_order_number + FROM web_returns, ws_wh + WHERE wr_order_number = ws_wh.ws_order_number) +ORDER BY count(DISTINCT ws_order_number) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q96.sql b/sql/core/src/test/resources/tpcds/q96.sql new file mode 100755 index 000000000000..7ab17e7bc459 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q96.sql @@ -0,0 +1,11 @@ +SELECT count(*) +FROM store_sales, household_demographics, time_dim, store +WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 20 + AND time_dim.t_minute >= 30 + AND household_demographics.hd_dep_count = 7 + AND store.s_store_name = 'ese' +ORDER BY count(*) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q97.sql b/sql/core/src/test/resources/tpcds/q97.sql new file mode 100755 index 000000000000..e7e0b1a05259 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q97.sql @@ -0,0 +1,30 @@ +WITH ssci AS ( + SELECT + ss_customer_sk customer_sk, + ss_item_sk item_sk + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + GROUP BY ss_customer_sk, ss_item_sk), + csci AS ( + SELECT + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + GROUP BY cs_bill_customer_sk, cs_item_sk) +SELECT + sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NULL + THEN 1 + ELSE 0 END) store_only, + sum(CASE WHEN ssci.customer_sk IS NULL AND csci.customer_sk IS NOT NULL + THEN 1 + ELSE 0 END) catalog_only, + sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NOT NULL + THEN 1 + ELSE 0 END) store_and_catalog +FROM ssci + FULL OUTER JOIN csci ON (ssci.customer_sk = csci.customer_sk + AND ssci.item_sk = csci.item_sk) +LIMIT 100 diff --git a/sql/core/src/test/resources/tpcds/q98.sql b/sql/core/src/test/resources/tpcds/q98.sql new file mode 100755 index 000000000000..bb10d4bf8da2 --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q98.sql @@ -0,0 +1,21 @@ +SELECT + i_item_desc, + i_category, + i_class, + i_current_price, + sum(ss_ext_sales_price) AS itemrevenue, + sum(ss_ext_sales_price) * 100 / sum(sum(ss_ext_sales_price)) + OVER + (PARTITION BY i_class) AS revenueratio +FROM + store_sales, item, date_dim +WHERE + ss_item_sk = i_item_sk + AND i_category IN ('Sports', 'Books', 'Home') + AND ss_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('1999-02-22' AS DATE) + AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) +GROUP BY + i_item_id, i_item_desc, i_category, i_class, i_current_price +ORDER BY + i_category, i_class, i_item_id, i_item_desc, revenueratio diff --git a/sql/core/src/test/resources/tpcds/q99.sql b/sql/core/src/test/resources/tpcds/q99.sql new file mode 100755 index 000000000000..f1a3d4d2b7fe --- /dev/null +++ b/sql/core/src/test/resources/tpcds/q99.sql @@ -0,0 +1,34 @@ +SELECT + substr(w_warehouse_name, 1, 20), + sm_type, + cc_name, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk <= 30) + THEN 1 + ELSE 0 END) AS `30 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 30) AND + (cs_ship_date_sk - cs_sold_date_sk <= 60) + THEN 1 + ELSE 0 END) AS `31 - 60 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 60) AND + (cs_ship_date_sk - cs_sold_date_sk <= 90) + THEN 1 + ELSE 0 END) AS `61 - 90 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 90) AND + (cs_ship_date_sk - cs_sold_date_sk <= 120) + THEN 1 + ELSE 0 END) AS `91 - 120 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 120) + THEN 1 + ELSE 0 END) AS `>120 days ` +FROM + catalog_sales, warehouse, ship_mode, call_center, date_dim +WHERE + d_month_seq BETWEEN 1200 AND 1200 + 11 + AND cs_ship_date_sk = d_date_sk + AND cs_warehouse_sk = w_warehouse_sk + AND cs_ship_mode_sk = sm_ship_mode_sk + AND cs_call_center_sk = cc_call_center_sk +GROUP BY + substr(w_warehouse_name, 1, 20), sm_type, cc_name +ORDER BY substr(w_warehouse_name, 1, 20), sm_type, cc_name +LIMIT 100 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala new file mode 100644 index 000000000000..7e61a6802515 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.SparkConf + +class SingleLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { + override protected def sparkConf: SparkConf = super.sparkConf + .set("spark.sql.codegen.fallback", "false") + .set("spark.sql.codegen.aggregate.map.twolevel.enable", "false") + + // adding some checking after each test is run, assuring that the configs are not changed + // in test code + after { + assert(sparkConf.get("spark.sql.codegen.fallback") == "false", + "configuration parameter changed in test body") + assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enable") == "false", + "configuration parameter changed in test body") + } +} + +class TwoLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { + override protected def sparkConf: SparkConf = super.sparkConf + .set("spark.sql.codegen.fallback", "false") + .set("spark.sql.codegen.aggregate.map.twolevel.enable", "true") + + // adding some checking after each test is run, assuring that the configs are not changed + // in test code + after { + assert(sparkConf.get("spark.sql.codegen.fallback") == "false", + "configuration parameter changed in test body") + assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enable") == "true", + "configuration parameter changed in test body") + } +} + +class TwoLevelAggregateHashMapWithVectorizedMapSuite + extends DataFrameAggregateSuite + with BeforeAndAfter { + + override protected def sparkConf: SparkConf = super.sparkConf + .set("spark.sql.codegen.fallback", "false") + .set("spark.sql.codegen.aggregate.map.twolevel.enable", "true") + .set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + + // adding some checking after each test is run, assuring that the configs are not changed + // in test code + after { + assert(sparkConf.get("spark.sql.codegen.fallback") == "false", + "configuration parameter changed in test body") + assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enable") == "true", + "configuration parameter changed in test body") + assert(sparkConf.get("spark.sql.codegen.aggregate.map.vectorized.enable") == "true", + "configuration parameter changed in test body") + } +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala new file mode 100644 index 000000000000..1aea33766407 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.sql.{Date, Timestamp} + +import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile.DEFAULT_PERCENTILE_ACCURACY +import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile.PercentileDigest +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.test.SharedSQLContext + +/** + * End-to-end tests for approximate percentile aggregate function. + */ +class ApproximatePercentileQuerySuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + private val table = "percentile_test" + + test("percentile_approx, single percentile value") { + withTempView(table) { + (1 to 1000).toDF("col").createOrReplaceTempView(table) + checkAnswer( + spark.sql( + s""" + |SELECT + | percentile_approx(col, 0.25), + | percentile_approx(col, 0.5), + | percentile_approx(col, 0.75d), + | percentile_approx(col, 0.0), + | percentile_approx(col, 1.0), + | percentile_approx(col, 0), + | percentile_approx(col, 1) + |FROM $table + """.stripMargin), + Row(250D, 500D, 750D, 1D, 1000D, 1D, 1000D) + ) + } + } + + test("percentile_approx, array of percentile value") { + withTempView(table) { + (1 to 1000).toDF("col").createOrReplaceTempView(table) + checkAnswer( + spark.sql( + s"""SELECT + | percentile_approx(col, array(0.25, 0.5, 0.75D)), + | count(col), + | percentile_approx(col, array(0.0, 1.0)), + | sum(col) + |FROM $table + """.stripMargin), + Row(Seq(250D, 500D, 750D), 1000, Seq(1D, 1000D), 500500) + ) + } + } + + test("percentile_approx, different column types") { + withTempView(table) { + val intSeq = 1 to 1000 + val data: Seq[(java.math.BigDecimal, Date, Timestamp)] = intSeq.map { i => + (new java.math.BigDecimal(i), DateTimeUtils.toJavaDate(i), DateTimeUtils.toJavaTimestamp(i)) + } + data.toDF("cdecimal", "cdate", "ctimestamp").createOrReplaceTempView(table) + checkAnswer( + spark.sql( + s"""SELECT + | percentile_approx(cdecimal, array(0.25, 0.5, 0.75D)), + | percentile_approx(cdate, array(0.25, 0.5, 0.75D)), + | percentile_approx(ctimestamp, array(0.25, 0.5, 0.75D)) + |FROM $table + """.stripMargin), + Row( + Seq("250.000000000000000000", "500.000000000000000000", "750.000000000000000000") + .map(i => new java.math.BigDecimal(i)), + Seq(250, 500, 750).map(DateTimeUtils.toJavaDate), + Seq(250, 500, 750).map(i => DateTimeUtils.toJavaTimestamp(i.toLong))) + ) + } + } + + test("percentile_approx, multiple records with the minimum value in a partition") { + withTempView(table) { + spark.sparkContext.makeRDD(Seq(1, 1, 2, 1, 1, 3, 1, 1, 4, 1, 1, 5), 4).toDF("col") + .createOrReplaceTempView(table) + checkAnswer( + spark.sql(s"SELECT percentile_approx(col, array(0.5)) FROM $table"), + Row(Seq(1.0D)) + ) + } + } + + test("percentile_approx, with different accuracies") { + + withTempView(table) { + (1 to 1000).toDF("col").createOrReplaceTempView(table) + + // With different accuracies + val expectedPercentile = 250D + val accuracies = Array(1, 10, 100, 1000, 10000) + val errors = accuracies.map { accuracy => + val df = spark.sql(s"SELECT percentile_approx(col, 0.25, $accuracy) FROM $table") + val approximatePercentile = df.collect().head.getInt(0) + val error = Math.abs(approximatePercentile - expectedPercentile) + error + } + + // The larger accuracy value we use, the smaller error we get + assert(errors.sorted.sameElements(errors.reverse)) + } + } + + test("percentile_approx, supports constant folding for parameter accuracy and percentages") { + withTempView(table) { + (1 to 1000).toDF("col").createOrReplaceTempView(table) + checkAnswer( + spark.sql(s"SELECT percentile_approx(col, array(0.25 + 0.25D), 200 + 800D) FROM $table"), + Row(Seq(500D)) + ) + } + } + + test("percentile_approx(), aggregation on empty input table, no group by") { + withTempView(table) { + Seq.empty[Int].toDF("col").createOrReplaceTempView(table) + checkAnswer( + spark.sql(s"SELECT sum(col), percentile_approx(col, 0.5) FROM $table"), + Row(null, null) + ) + } + } + + test("percentile_approx(), aggregation on empty input table, with group by") { + withTempView(table) { + Seq.empty[Int].toDF("col").createOrReplaceTempView(table) + checkAnswer( + spark.sql(s"SELECT sum(col), percentile_approx(col, 0.5) FROM $table GROUP BY col"), + Seq.empty[Row] + ) + } + } + + test("percentile_approx(null), aggregation with group by") { + withTempView(table) { + (1 to 1000).map(x => (x % 3, x)).toDF("key", "value").createOrReplaceTempView(table) + checkAnswer( + spark.sql( + s"""SELECT + | key, + | percentile_approx(null, 0.5) + |FROM $table + |GROUP BY key + """.stripMargin), + Seq( + Row(0, null), + Row(1, null), + Row(2, null)) + ) + } + } + + test("percentile_approx(null), aggregation without group by") { + withTempView(table) { + (1 to 1000).map(x => (x % 3, x)).toDF("key", "value").createOrReplaceTempView(table) + checkAnswer( + spark.sql( + s"""SELECT + | percentile_approx(null, 0.5), + | sum(null), + | percentile_approx(null, 0.5) + |FROM $table + """.stripMargin), + Row(null, null, null) + ) + } + } + + test("percentile_approx(col, ...), input rows contains null, with out group by") { + withTempView(table) { + (1 to 1000).map(new Integer(_)).flatMap(Seq(null: Integer, _)).toDF("col") + .createOrReplaceTempView(table) + checkAnswer( + spark.sql( + s"""SELECT + | percentile_approx(col, 0.5), + | sum(null), + | percentile_approx(col, 0.5) + |FROM $table + """.stripMargin), + Row(500D, null, 500D)) + } + } + + test("percentile_approx(col, ...), input rows contains null, with group by") { + withTempView(table) { + val rand = new java.util.Random() + (1 to 1000) + .map(new Integer(_)) + .map(v => (new Integer(v % 2), v)) + // Add some nulls + .flatMap(Seq(_, (null: Integer, null: Integer))) + .toDF("key", "value").createOrReplaceTempView(table) + checkAnswer( + spark.sql( + s"""SELECT + | percentile_approx(value, 0.5), + | sum(value), + | percentile_approx(value, 0.5) + |FROM $table + |GROUP BY key + """.stripMargin), + Seq( + Row(499.0D, 250000, 499.0D), + Row(500.0D, 250500, 500.0D), + Row(null, null, null)) + ) + } + } + + test("percentile_approx(col, ...) works in window function") { + withTempView(table) { + val data = (1 to 10).map(v => (v % 2, v)) + data.toDF("key", "value").createOrReplaceTempView(table) + + val query = spark.sql( + s""" + |SElECT percentile_approx(value, 0.5) + |OVER + | (PARTITION BY key ORDER BY value ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) + | AS percentile + |FROM $table + """.stripMargin) + + val expected = data.groupBy(_._1).toSeq.flatMap { group => + val (key, values) = group + val sortedValues = values.map(_._2).sorted + + var outputRows = Seq.empty[Row] + var i = 0 + + val percentile = new PercentileDigest(1.0 / DEFAULT_PERCENTILE_ACCURACY) + sortedValues.foreach { value => + percentile.add(value) + outputRows :+= Row(percentile.getPercentiles(Array(0.5D)).head) + } + outputRows + } + + checkAnswer(query, expected) + } + } +} 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 dbcb011f603f..3e4f61943159 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -17,38 +17,70 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException -import org.apache.spark.sql.execution.Exchange -import org.apache.spark.sql.execution.PhysicalRDD - +import scala.collection.mutable.HashSet import scala.concurrent.duration._ import scala.language.postfixOps import org.scalatest.concurrent.Eventually._ -import org.apache.spark.Accumulators -import org.apache.spark.sql.columnar._ +import org.apache.spark.CleanerListener +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.execution.{RDDScanExec, SparkPlan} +import org.apache.spark.sql.execution.columnar._ +import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.storage.{StorageLevel, RDDBlockId} +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.storage.{RDDBlockId, StorageLevel} +import org.apache.spark.util.{AccumulatorContext, Utils} private case class BigData(s: String) -class CachedTableSuite extends QueryTest with SharedSQLContext { +class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext { import testImplicits._ + setupTestData() + + override def afterEach(): Unit = { + try { + spark.catalog.clearCache() + } finally { + super.afterEach() + } + } + def rddIdOf(tableName: String): Int = { - val executedPlan = sqlContext.table(tableName).queryExecution.executedPlan - executedPlan.collect { - case InMemoryColumnarTableScan(_, _, relation) => + val plan = spark.table(tableName).queryExecution.sparkPlan + plan.collect { + case InMemoryTableScanExec(_, _, relation) => relation.cachedColumnBuffers.id case _ => - fail(s"Table $tableName is not cached\n" + executedPlan) + fail(s"Table $tableName is not cached\n" + plan) }.head } def isMaterialized(rddId: Int): Boolean = { - sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)).nonEmpty + val maybeBlock = sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)) + maybeBlock.foreach(_ => sparkContext.env.blockManager.releaseLock(RDDBlockId(rddId, 0))) + maybeBlock.nonEmpty + } + + private def getNumInMemoryRelations(ds: Dataset[_]): Int = { + val plan = ds.queryExecution.withCachedData + var sum = plan.collect { case _: InMemoryRelation => 1 }.sum + plan.transformAllExpressions { + case e: SubqueryExpression => + sum += getNumInMemoryRelations(e.plan) + e + } + sum + } + + private def getNumInMemoryTablesRecursively(plan: SparkPlan): Int = { + plan.collect { + case InMemoryTableScanExec(_, _, relation) => + getNumInMemoryTablesRecursively(relation.child) + 1 + }.sum } test("withColumn doesn't invalidate cached dataframe") { @@ -71,43 +103,47 @@ class CachedTableSuite extends QueryTest with SharedSQLContext { } test("cache temp table") { - testData.select('key).registerTempTable("tempTable") - assertCached(sql("SELECT COUNT(*) FROM tempTable"), 0) - sqlContext.cacheTable("tempTable") - assertCached(sql("SELECT COUNT(*) FROM tempTable")) - sqlContext.uncacheTable("tempTable") + withTempView("tempTable") { + testData.select('key).createOrReplaceTempView("tempTable") + assertCached(sql("SELECT COUNT(*) FROM tempTable"), 0) + spark.catalog.cacheTable("tempTable") + assertCached(sql("SELECT COUNT(*) FROM tempTable")) + spark.catalog.uncacheTable("tempTable") + } } test("unpersist an uncached table will not raise exception") { - assert(None == sqlContext.cacheManager.lookupCachedData(testData)) + assert(None == spark.sharedState.cacheManager.lookupCachedData(testData)) testData.unpersist(blocking = true) - assert(None == sqlContext.cacheManager.lookupCachedData(testData)) + assert(None == spark.sharedState.cacheManager.lookupCachedData(testData)) testData.unpersist(blocking = false) - assert(None == sqlContext.cacheManager.lookupCachedData(testData)) + assert(None == spark.sharedState.cacheManager.lookupCachedData(testData)) testData.persist() - assert(None != sqlContext.cacheManager.lookupCachedData(testData)) + assert(None != spark.sharedState.cacheManager.lookupCachedData(testData)) testData.unpersist(blocking = true) - assert(None == sqlContext.cacheManager.lookupCachedData(testData)) + assert(None == spark.sharedState.cacheManager.lookupCachedData(testData)) testData.unpersist(blocking = false) - assert(None == sqlContext.cacheManager.lookupCachedData(testData)) + assert(None == spark.sharedState.cacheManager.lookupCachedData(testData)) } test("cache table as select") { - sql("CACHE TABLE tempTable AS SELECT key FROM testData") - assertCached(sql("SELECT COUNT(*) FROM tempTable")) - sqlContext.uncacheTable("tempTable") + withTempView("tempTable") { + sql("CACHE TABLE tempTable AS SELECT key FROM testData") + assertCached(sql("SELECT COUNT(*) FROM tempTable")) + spark.catalog.uncacheTable("tempTable") + } } test("uncaching temp table") { - testData.select('key).registerTempTable("tempTable1") - testData.select('key).registerTempTable("tempTable2") - sqlContext.cacheTable("tempTable1") + testData.select('key).createOrReplaceTempView("tempTable1") + testData.select('key).createOrReplaceTempView("tempTable2") + spark.catalog.cacheTable("tempTable1") assertCached(sql("SELECT COUNT(*) FROM tempTable1")) assertCached(sql("SELECT COUNT(*) FROM tempTable2")) // Is this valid? - sqlContext.uncacheTable("tempTable2") + spark.catalog.uncacheTable("tempTable2") // Should this be cached? assertCached(sql("SELECT COUNT(*) FROM tempTable1"), 0) @@ -116,102 +152,94 @@ class CachedTableSuite extends QueryTest with SharedSQLContext { test("too big for memory") { val data = "*" * 1000 sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).toDF() - .registerTempTable("bigData") - sqlContext.table("bigData").persist(StorageLevel.MEMORY_AND_DISK) - assert(sqlContext.table("bigData").count() === 200000L) - sqlContext.table("bigData").unpersist(blocking = true) + .createOrReplaceTempView("bigData") + spark.table("bigData").persist(StorageLevel.MEMORY_AND_DISK) + assert(spark.table("bigData").count() === 200000L) + spark.table("bigData").unpersist(blocking = true) } test("calling .cache() should use in-memory columnar caching") { - sqlContext.table("testData").cache() - assertCached(sqlContext.table("testData")) - sqlContext.table("testData").unpersist(blocking = true) + spark.table("testData").cache() + assertCached(spark.table("testData")) + spark.table("testData").unpersist(blocking = true) } test("calling .unpersist() should drop in-memory columnar cache") { - sqlContext.table("testData").cache() - sqlContext.table("testData").count() - sqlContext.table("testData").unpersist(blocking = true) - assertCached(sqlContext.table("testData"), 0) + spark.table("testData").cache() + spark.table("testData").count() + spark.table("testData").unpersist(blocking = true) + assertCached(spark.table("testData"), 0) } test("isCached") { - sqlContext.cacheTable("testData") + spark.catalog.cacheTable("testData") - assertCached(sqlContext.table("testData")) - assert(sqlContext.table("testData").queryExecution.withCachedData match { + assertCached(spark.table("testData")) + assert(spark.table("testData").queryExecution.withCachedData match { case _: InMemoryRelation => true case _ => false }) - sqlContext.uncacheTable("testData") - assert(!sqlContext.isCached("testData")) - assert(sqlContext.table("testData").queryExecution.withCachedData match { + spark.catalog.uncacheTable("testData") + assert(!spark.catalog.isCached("testData")) + assert(spark.table("testData").queryExecution.withCachedData match { case _: InMemoryRelation => false case _ => true }) } test("SPARK-1669: cacheTable should be idempotent") { - assume(!sqlContext.table("testData").logicalPlan.isInstanceOf[InMemoryRelation]) + assume(!spark.table("testData").logicalPlan.isInstanceOf[InMemoryRelation]) - sqlContext.cacheTable("testData") - assertCached(sqlContext.table("testData")) + spark.catalog.cacheTable("testData") + assertCached(spark.table("testData")) assertResult(1, "InMemoryRelation not found, testData should have been cached") { - sqlContext.table("testData").queryExecution.withCachedData.collect { - case r: InMemoryRelation => r - }.size + getNumInMemoryRelations(spark.table("testData")) } - sqlContext.cacheTable("testData") + spark.catalog.cacheTable("testData") assertResult(0, "Double InMemoryRelations found, cacheTable() is not idempotent") { - sqlContext.table("testData").queryExecution.withCachedData.collect { - case r @ InMemoryRelation(_, _, _, _, _: InMemoryColumnarTableScan, _) => r + spark.table("testData").queryExecution.withCachedData.collect { + case r @ InMemoryRelation(_, _, _, _, _: InMemoryTableScanExec, _) => r }.size } - sqlContext.uncacheTable("testData") + spark.catalog.uncacheTable("testData") } test("read from cached table and uncache") { - sqlContext.cacheTable("testData") - checkAnswer(sqlContext.table("testData"), testData.collect().toSeq) - assertCached(sqlContext.table("testData")) - - sqlContext.uncacheTable("testData") - checkAnswer(sqlContext.table("testData"), testData.collect().toSeq) - assertCached(sqlContext.table("testData"), 0) - } + spark.catalog.cacheTable("testData") + checkAnswer(spark.table("testData"), testData.collect().toSeq) + assertCached(spark.table("testData")) - test("correct error on uncache of non-cached table") { - intercept[IllegalArgumentException] { - sqlContext.uncacheTable("testData") - } + spark.catalog.uncacheTable("testData") + checkAnswer(spark.table("testData"), testData.collect().toSeq) + assertCached(spark.table("testData"), 0) } test("SELECT star from cached table") { - sql("SELECT * FROM testData").registerTempTable("selectStar") - sqlContext.cacheTable("selectStar") + sql("SELECT * FROM testData").createOrReplaceTempView("selectStar") + spark.catalog.cacheTable("selectStar") checkAnswer( sql("SELECT * FROM selectStar WHERE key = 1"), Seq(Row(1, "1"))) - sqlContext.uncacheTable("selectStar") + spark.catalog.uncacheTable("selectStar") } test("Self-join cached") { val unCachedAnswer = sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key").collect() - sqlContext.cacheTable("testData") + spark.catalog.cacheTable("testData") checkAnswer( sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key"), unCachedAnswer.toSeq) - sqlContext.uncacheTable("testData") + spark.catalog.uncacheTable("testData") } test("'CACHE TABLE' and 'UNCACHE TABLE' SQL statement") { sql("CACHE TABLE testData") - assertCached(sqlContext.table("testData")) + assertCached(spark.table("testData")) val rddId = rddIdOf("testData") assert( @@ -219,7 +247,7 @@ class CachedTableSuite extends QueryTest with SharedSQLContext { "Eagerly cached in-memory table should have already been materialized") sql("UNCACHE TABLE testData") - assert(!sqlContext.isCached("testData"), "Table 'testData' should not be cached") + assert(!spark.catalog.isCached("testData"), "Table 'testData' should not be cached") eventually(timeout(10 seconds)) { assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") @@ -227,38 +255,42 @@ class CachedTableSuite extends QueryTest with SharedSQLContext { } test("CACHE TABLE tableName AS SELECT * FROM anotherTable") { - sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") - assertCached(sqlContext.table("testCacheTable")) - - val rddId = rddIdOf("testCacheTable") - assert( - isMaterialized(rddId), - "Eagerly cached in-memory table should have already been materialized") - - sqlContext.uncacheTable("testCacheTable") - eventually(timeout(10 seconds)) { - assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + withTempView("testCacheTable") { + sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") + assertCached(spark.table("testCacheTable")) + + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") + + spark.catalog.uncacheTable("testCacheTable") + eventually(timeout(10 seconds)) { + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } } } test("CACHE TABLE tableName AS SELECT ...") { - sql("CACHE TABLE testCacheTable AS SELECT key FROM testData LIMIT 10") - assertCached(sqlContext.table("testCacheTable")) - - val rddId = rddIdOf("testCacheTable") - assert( - isMaterialized(rddId), - "Eagerly cached in-memory table should have already been materialized") - - sqlContext.uncacheTable("testCacheTable") - eventually(timeout(10 seconds)) { - assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + withTempView("testCacheTable") { + sql("CACHE TABLE testCacheTable AS SELECT key FROM testData LIMIT 10") + assertCached(spark.table("testCacheTable")) + + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") + + spark.catalog.uncacheTable("testCacheTable") + eventually(timeout(10 seconds)) { + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } } } test("CACHE LAZY TABLE tableName") { sql("CACHE LAZY TABLE testData") - assertCached(sqlContext.table("testData")) + assertCached(spark.table("testData")) val rddId = rddIdOf("testData") assert( @@ -270,7 +302,7 @@ class CachedTableSuite extends QueryTest with SharedSQLContext { isMaterialized(rddId), "Lazily cached in-memory table should have been materialized") - sqlContext.uncacheTable("testData") + spark.catalog.uncacheTable("testData") eventually(timeout(10 seconds)) { assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") } @@ -278,150 +310,475 @@ class CachedTableSuite extends QueryTest with SharedSQLContext { test("InMemoryRelation statistics") { sql("CACHE TABLE testData") - sqlContext.table("testData").queryExecution.withCachedData.collect { + spark.table("testData").queryExecution.withCachedData.collect { case cached: InMemoryRelation => - val actualSizeInBytes = (1 to 100).map(i => INT.defaultSize + i.toString.length + 4).sum - assert(cached.statistics.sizeInBytes === actualSizeInBytes) + val actualSizeInBytes = (1 to 100).map(i => 4 + i.toString.length + 4).sum + assert(cached.stats.sizeInBytes === actualSizeInBytes) } } test("Drops temporary table") { - testData.select('key).registerTempTable("t1") - sqlContext.table("t1") - sqlContext.dropTempTable("t1") - intercept[NoSuchTableException](sqlContext.table("t1")) + testData.select('key).createOrReplaceTempView("t1") + spark.table("t1") + spark.catalog.dropTempView("t1") + intercept[AnalysisException](spark.table("t1")) } test("Drops cached temporary table") { - testData.select('key).registerTempTable("t1") - testData.select('key).registerTempTable("t2") - sqlContext.cacheTable("t1") + testData.select('key).createOrReplaceTempView("t1") + testData.select('key).createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") - assert(sqlContext.isCached("t1")) - assert(sqlContext.isCached("t2")) + assert(spark.catalog.isCached("t1")) + assert(spark.catalog.isCached("t2")) - sqlContext.dropTempTable("t1") - intercept[NoSuchTableException](sqlContext.table("t1")) - assert(!sqlContext.isCached("t2")) + spark.catalog.dropTempView("t1") + intercept[AnalysisException](spark.table("t1")) + assert(!spark.catalog.isCached("t2")) } test("Clear all cache") { - sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") - sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") - sqlContext.cacheTable("t1") - sqlContext.cacheTable("t2") - sqlContext.clearCache() - assert(sqlContext.cacheManager.isEmpty) - - sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") - sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") - sqlContext.cacheTable("t1") - sqlContext.cacheTable("t2") + sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1") + sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + spark.catalog.clearCache() + assert(spark.sharedState.cacheManager.isEmpty) + + sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1") + sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") sql("Clear CACHE") - assert(sqlContext.cacheManager.isEmpty) + assert(spark.sharedState.cacheManager.isEmpty) } - test("Clear accumulators when uncacheTable to prevent memory leaking") { - sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") - sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + test("Ensure accumulators to be cleared after GC when uncacheTable") { + sql("SELECT key FROM testData LIMIT 10").createOrReplaceTempView("t1") + sql("SELECT key FROM testData LIMIT 5").createOrReplaceTempView("t2") - sqlContext.cacheTable("t1") - sqlContext.cacheTable("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") sql("SELECT * FROM t1").count() sql("SELECT * FROM t2").count() sql("SELECT * FROM t1").count() sql("SELECT * FROM t2").count() - Accumulators.synchronized { - val accsSize = Accumulators.originals.size - sqlContext.uncacheTable("t1") - sqlContext.uncacheTable("t2") - assert((accsSize - 2) == Accumulators.originals.size) + val toBeCleanedAccIds = new HashSet[Long] + + val accId1 = spark.table("t1").queryExecution.withCachedData.collect { + case i: InMemoryRelation => i.batchStats.id + }.head + toBeCleanedAccIds += accId1 + + val accId2 = spark.table("t1").queryExecution.withCachedData.collect { + case i: InMemoryRelation => i.batchStats.id + }.head + toBeCleanedAccIds += accId2 + + val cleanerListener = new CleanerListener { + def rddCleaned(rddId: Int): Unit = {} + def shuffleCleaned(shuffleId: Int): Unit = {} + def broadcastCleaned(broadcastId: Long): Unit = {} + def accumCleaned(accId: Long): Unit = { + toBeCleanedAccIds.synchronized { toBeCleanedAccIds -= accId } + } + def checkpointCleaned(rddId: Long): Unit = {} + } + spark.sparkContext.cleaner.get.attachListener(cleanerListener) + + spark.catalog.uncacheTable("t1") + spark.catalog.uncacheTable("t2") + + System.gc() + + eventually(timeout(10 seconds)) { + assert(toBeCleanedAccIds.synchronized { toBeCleanedAccIds.isEmpty }, + "batchStats accumulators should be cleared after GC when uncacheTable") } + + assert(AccumulatorContext.get(accId1).isEmpty) + assert(AccumulatorContext.get(accId2).isEmpty) } test("SPARK-10327 Cache Table is not working while subquery has alias in its project list") { sparkContext.parallelize((1, 1) :: (2, 2) :: Nil) - .toDF("key", "value").selectExpr("key", "value", "key+1").registerTempTable("abc") - sqlContext.cacheTable("abc") + .toDF("key", "value").selectExpr("key", "value", "key+1").createOrReplaceTempView("abc") + spark.catalog.cacheTable("abc") val sparkPlan = sql( """select a.key, b.key, c.key from |abc a join abc b on a.key=b.key |join abc c on a.key=c.key""".stripMargin).queryExecution.sparkPlan - assert(sparkPlan.collect { case e: InMemoryColumnarTableScan => e }.size === 3) - assert(sparkPlan.collect { case e: PhysicalRDD => e }.size === 0) + assert(sparkPlan.collect { case e: InMemoryTableScanExec => e }.size === 3) + assert(sparkPlan.collect { case e: RDDScanExec => e }.size === 0) } /** * Verifies that the plan for `df` contains `expected` number of Exchange operators. */ private def verifyNumExchanges(df: DataFrame, expected: Int): Unit = { - assert(df.queryExecution.executedPlan.collect { case e: Exchange => e }.size == expected) + assert(df.queryExecution.executedPlan.collect { case e: ShuffleExchange => e }.size == expected) } test("A cached table preserves the partitioning and ordering of its cached SparkPlan") { - val table3x = testData.unionAll(testData).unionAll(testData) - table3x.registerTempTable("testData3x") + val table3x = testData.union(testData).union(testData) + table3x.createOrReplaceTempView("testData3x") - sql("SELECT key, value FROM testData3x ORDER BY key").registerTempTable("orderedTable") - sqlContext.cacheTable("orderedTable") - assertCached(sqlContext.table("orderedTable")) + sql("SELECT key, value FROM testData3x ORDER BY key").createOrReplaceTempView("orderedTable") + spark.catalog.cacheTable("orderedTable") + assertCached(spark.table("orderedTable")) // Should not have an exchange as the query is already sorted on the group by key. verifyNumExchanges(sql("SELECT key, count(*) FROM orderedTable GROUP BY key"), 0) checkAnswer( sql("SELECT key, count(*) FROM orderedTable GROUP BY key ORDER BY key"), sql("SELECT key, count(*) FROM testData3x GROUP BY key ORDER BY key").collect()) - sqlContext.uncacheTable("orderedTable") + spark.catalog.uncacheTable("orderedTable") + spark.catalog.dropTempView("orderedTable") // Set up two tables distributed in the same way. Try this with the data distributed into // different number of partitions. for (numPartitions <- 1 until 10 by 4) { - testData.repartition(numPartitions, $"key").registerTempTable("t1") - testData2.repartition(numPartitions, $"a").registerTempTable("t2") - sqlContext.cacheTable("t1") - sqlContext.cacheTable("t2") - - // Joining them should result in no exchanges. - verifyNumExchanges(sql("SELECT * FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a"), 0) - checkAnswer(sql("SELECT * FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a"), - sql("SELECT * FROM testData t1 JOIN testData2 t2 ON t1.key = t2.a")) - - // Grouping on the partition key should result in no exchanges - verifyNumExchanges(sql("SELECT count(*) FROM t1 GROUP BY key"), 0) - checkAnswer(sql("SELECT count(*) FROM t1 GROUP BY key"), - sql("SELECT count(*) FROM testData GROUP BY key")) - - sqlContext.uncacheTable("t1") - sqlContext.uncacheTable("t2") - sqlContext.dropTempTable("t1") - sqlContext.dropTempTable("t2") + withTempView("t1", "t2") { + testData.repartition(numPartitions, $"key").createOrReplaceTempView("t1") + testData2.repartition(numPartitions, $"a").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + + // Joining them should result in no exchanges. + verifyNumExchanges(sql("SELECT * FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a"), 0) + checkAnswer(sql("SELECT * FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a"), + sql("SELECT * FROM testData t1 JOIN testData2 t2 ON t1.key = t2.a")) + + // Grouping on the partition key should result in no exchanges + verifyNumExchanges(sql("SELECT count(*) FROM t1 GROUP BY key"), 0) + checkAnswer(sql("SELECT count(*) FROM t1 GROUP BY key"), + sql("SELECT count(*) FROM testData GROUP BY key")) + + spark.catalog.uncacheTable("t1") + spark.catalog.uncacheTable("t2") + } + } + + // Distribute the tables into non-matching number of partitions. Need to shuffle one side. + withTempView("t1", "t2") { + testData.repartition(6, $"key").createOrReplaceTempView("t1") + testData2.repartition(3, $"a").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + + val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") + verifyNumExchanges(query, 1) + assert(query.queryExecution.executedPlan.outputPartitioning.numPartitions === 6) + checkAnswer( + query, + testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) + spark.catalog.uncacheTable("t1") + spark.catalog.uncacheTable("t2") + } + + // One side of join is not partitioned in the desired way. Need to shuffle one side. + withTempView("t1", "t2") { + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(6, $"a").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + + val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") + verifyNumExchanges(query, 1) + assert(query.queryExecution.executedPlan.outputPartitioning.numPartitions === 6) + checkAnswer( + query, + testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) + spark.catalog.uncacheTable("t1") + spark.catalog.uncacheTable("t2") } - // Distribute the tables into non-matching number of partitions. Need to shuffle. - testData.repartition(6, $"key").registerTempTable("t1") - testData2.repartition(3, $"a").registerTempTable("t2") - sqlContext.cacheTable("t1") - sqlContext.cacheTable("t2") - - verifyNumExchanges(sql("SELECT * FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a"), 2) - sqlContext.uncacheTable("t1") - sqlContext.uncacheTable("t2") - sqlContext.dropTempTable("t1") - sqlContext.dropTempTable("t2") - - // One side of join is not partitioned in the desired way. Need to shuffle. - testData.repartition(6, $"value").registerTempTable("t1") - testData2.repartition(6, $"a").registerTempTable("t2") - sqlContext.cacheTable("t1") - sqlContext.cacheTable("t2") - - verifyNumExchanges(sql("SELECT * FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a"), 2) - sqlContext.uncacheTable("t1") - sqlContext.uncacheTable("t2") - sqlContext.dropTempTable("t1") - sqlContext.dropTempTable("t2") + withTempView("t1", "t2") { + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(12, $"a").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + + val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") + verifyNumExchanges(query, 1) + assert(query.queryExecution.executedPlan.outputPartitioning.numPartitions === 12) + checkAnswer( + query, + testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) + spark.catalog.uncacheTable("t1") + spark.catalog.uncacheTable("t2") + } + + // One side of join is not partitioned in the desired way. Since the number of partitions of + // the side that has already partitioned is smaller than the side that is not partitioned, + // we shuffle both side. + withTempView("t1", "t2") { + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(3, $"a").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + + val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") + verifyNumExchanges(query, 2) + checkAnswer( + query, + testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) + spark.catalog.uncacheTable("t1") + spark.catalog.uncacheTable("t2") + } + + // repartition's column ordering is different from group by column ordering. + // But they use the same set of columns. + withTempView("t1") { + testData.repartition(6, $"value", $"key").createOrReplaceTempView("t1") + spark.catalog.cacheTable("t1") + + val query = sql("SELECT value, key from t1 group by key, value") + verifyNumExchanges(query, 0) + checkAnswer( + query, + testData.distinct().select($"value", $"key")) + spark.catalog.uncacheTable("t1") + } + + // repartition's column ordering is different from join condition's column ordering. + // We will still shuffle because hashcodes of a row depend on the column ordering. + // If we do not shuffle, we may actually partition two tables in totally two different way. + // See PartitioningSuite for more details. + withTempView("t1", "t2") { + val df1 = testData + df1.repartition(6, $"value", $"key").createOrReplaceTempView("t1") + val df2 = testData2.select($"a", $"b".cast("string")) + df2.repartition(6, $"a", $"b").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + + val query = + sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a and t1.value = t2.b") + verifyNumExchanges(query, 1) + assert(query.queryExecution.executedPlan.outputPartitioning.numPartitions === 6) + checkAnswer( + query, + df1.join(df2, $"key" === $"a" && $"value" === $"b").select($"key", $"value", $"a", $"b")) + spark.catalog.uncacheTable("t1") + spark.catalog.uncacheTable("t2") + } + } + + test("SPARK-15870 DataFrame can't execute after uncacheTable") { + val selectStar = sql("SELECT * FROM testData WHERE key = 1") + selectStar.createOrReplaceTempView("selectStar") + + spark.catalog.cacheTable("selectStar") + checkAnswer( + selectStar, + Seq(Row(1, "1"))) + + spark.catalog.uncacheTable("selectStar") + checkAnswer( + selectStar, + Seq(Row(1, "1"))) + } + + test("SPARK-15915 Logical plans should use canonicalized plan when override sameResult") { + val localRelation = Seq(1, 2, 3).toDF() + localRelation.createOrReplaceTempView("localRelation") + + spark.catalog.cacheTable("localRelation") + assert(getNumInMemoryRelations(localRelation) == 1) + } + + test("SPARK-19093 Caching in side subquery") { + withTempView("t1") { + Seq(1).toDF("c1").createOrReplaceTempView("t1") + spark.catalog.cacheTable("t1") + val ds = + sql( + """ + |SELECT * FROM t1 + |WHERE + |NOT EXISTS (SELECT * FROM t1) + """.stripMargin) + assert(getNumInMemoryRelations(ds) == 2) + } + } + + test("SPARK-19093 scalar and nested predicate query") { + withTempView("t1", "t2", "t3", "t4") { + Seq(1).toDF("c1").createOrReplaceTempView("t1") + Seq(2).toDF("c1").createOrReplaceTempView("t2") + Seq(1).toDF("c1").createOrReplaceTempView("t3") + Seq(1).toDF("c1").createOrReplaceTempView("t4") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + spark.catalog.cacheTable("t3") + spark.catalog.cacheTable("t4") + + // Nested predicate subquery + val ds = + sql( + """ + |SELECT * FROM t1 + |WHERE + |c1 IN (SELECT c1 FROM t2 WHERE c1 IN (SELECT c1 FROM t3 WHERE c1 = 1)) + """.stripMargin) + assert(getNumInMemoryRelations(ds) == 3) + + // Scalar subquery and predicate subquery + val ds2 = + sql( + """ + |SELECT * FROM (SELECT c1, max(c1) FROM t1 GROUP BY c1) + |WHERE + |c1 = (SELECT max(c1) FROM t2 GROUP BY c1) + |OR + |EXISTS (SELECT c1 FROM t3) + |OR + |c1 IN (SELECT c1 FROM t4) + """.stripMargin) + assert(getNumInMemoryRelations(ds2) == 4) + } + } + + test("SPARK-19765: UNCACHE TABLE should un-cache all cached plans that refer to this table") { + withTable("t") { + withTempPath { path => + Seq(1 -> "a").toDF("i", "j").write.parquet(path.getCanonicalPath) + sql(s"CREATE TABLE t USING parquet LOCATION '${path.toURI}'") + spark.catalog.cacheTable("t") + spark.table("t").select($"i").cache() + checkAnswer(spark.table("t").select($"i"), Row(1)) + assertCached(spark.table("t").select($"i")) + + Utils.deleteRecursively(path) + spark.sessionState.catalog.refreshTable(TableIdentifier("t")) + spark.catalog.uncacheTable("t") + assert(spark.table("t").select($"i").count() == 0) + assert(getNumInMemoryRelations(spark.table("t").select($"i")) == 0) + } + } + } + + test("refreshByPath should refresh all cached plans with the specified path") { + withTempDir { dir => + val path = dir.getCanonicalPath() + + spark.range(10).write.mode("overwrite").parquet(path) + spark.read.parquet(path).cache() + spark.read.parquet(path).filter($"id" > 4).cache() + assert(spark.read.parquet(path).filter($"id" > 4).count() == 5) + + spark.range(20).write.mode("overwrite").parquet(path) + spark.catalog.refreshByPath(path) + assert(spark.read.parquet(path).count() == 20) + assert(spark.read.parquet(path).filter($"id" > 4).count() == 15) + } + } + + test("SPARK-19993 simple subquery caching") { + withTempView("t1", "t2") { + Seq(1).toDF("c1").createOrReplaceTempView("t1") + Seq(2).toDF("c1").createOrReplaceTempView("t2") + + val sql1 = + """ + |SELECT * FROM t1 + |WHERE + |NOT EXISTS (SELECT * FROM t2) + """.stripMargin + sql(sql1).cache() + + val cachedDs = sql(sql1) + assert(getNumInMemoryRelations(cachedDs) == 1) + + // Additional predicate in the subquery plan should cause a cache miss + val cachedMissDs = + sql( + """ + |SELECT * FROM t1 + |WHERE + |NOT EXISTS (SELECT * FROM t2 where c1 = 0) + """.stripMargin) + assert(getNumInMemoryRelations(cachedMissDs) == 0) + } + } + + test("SPARK-19993 subquery caching with correlated predicates") { + withTempView("t1", "t2") { + Seq(1).toDF("c1").createOrReplaceTempView("t1") + Seq(1).toDF("c1").createOrReplaceTempView("t2") + + // Simple correlated predicate in subquery + val sqlText = + """ + |SELECT * FROM t1 + |WHERE + |t1.c1 in (SELECT t2.c1 FROM t2 where t1.c1 = t2.c1) + """.stripMargin + sql(sqlText).cache() + + val cachedDs = sql(sqlText) + assert(getNumInMemoryRelations(cachedDs) == 1) + } + } + + test("SPARK-19993 subquery with cached underlying relation") { + withTempView("t1") { + Seq(1).toDF("c1").createOrReplaceTempView("t1") + spark.catalog.cacheTable("t1") + + // underlying table t1 is cached as well as the query that refers to it. + val sqlText = + """ + |SELECT * FROM t1 + |WHERE + |NOT EXISTS (SELECT * FROM t1) + """.stripMargin + val ds = sql(sqlText) + assert(getNumInMemoryRelations(ds) == 2) + + val cachedDs = sql(sqlText).cache() + assert(getNumInMemoryTablesRecursively(cachedDs.queryExecution.sparkPlan) == 3) + } + } + + test("SPARK-19993 nested subquery caching and scalar + predicate subqueris") { + withTempView("t1", "t2", "t3", "t4") { + Seq(1).toDF("c1").createOrReplaceTempView("t1") + Seq(2).toDF("c1").createOrReplaceTempView("t2") + Seq(1).toDF("c1").createOrReplaceTempView("t3") + Seq(1).toDF("c1").createOrReplaceTempView("t4") + + // Nested predicate subquery + val sql1 = + """ + |SELECT * FROM t1 + |WHERE + |c1 IN (SELECT c1 FROM t2 WHERE c1 IN (SELECT c1 FROM t3 WHERE c1 = 1)) + """.stripMargin + sql(sql1).cache() + + val cachedDs = sql(sql1) + assert(getNumInMemoryRelations(cachedDs) == 1) + + // Scalar subquery and predicate subquery + val sql2 = + """ + |SELECT * FROM (SELECT c1, max(c1) FROM t1 GROUP BY c1) + |WHERE + |c1 = (SELECT max(c1) FROM t2 GROUP BY c1) + |OR + |EXISTS (SELECT c1 FROM t3) + |OR + |c1 IN (SELECT c1 FROM t4) + """.stripMargin + sql(sql2).cache() + + val cachedDs2 = sql(sql2) + assert(getNumInMemoryRelations(cachedDs2) == 1) + } } } 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 fa559c9c6400..7c45be21961d 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,10 +17,12 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.expressions.NamedExpression +import org.apache.hadoop.io.{LongWritable, Text} +import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat} import org.scalatest.Matchers._ -import org.apache.spark.sql.execution.{Project, TungstenProject} +import org.apache.spark.sql.catalyst.expressions.NamedExpression +import org.apache.spark.sql.execution.ProjectExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -29,7 +31,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { import testImplicits._ private lazy val booleanData = { - sqlContext.createDataFrame(sparkContext.parallelize( + spark.createDataFrame(sparkContext.parallelize( Row(false, false) :: Row(false, true) :: Row(true, false) :: @@ -37,6 +39,9 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { StructType(Seq(StructField("a", BooleanType), StructField("b", BooleanType)))) } + private lazy val nullData = Seq( + (Some(1), Some(1)), (Some(1), Some(2)), (Some(1), None), (None, None)).toDF("a", "b") + test("column names with space") { val df = Seq((1, "a")).toDF("name with space", "name.with.dot") @@ -105,10 +110,11 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { Row("a") :: Nil) } - test("alias") { + test("alias and name") { val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") assert(df.select(df("a").as("b")).columns.head === "b") assert(df.select(df("a").alias("b")).columns.head === "b") + assert(df.select(df("a").name("b")).columns.head === "b") } test("as propagates metadata") { @@ -119,66 +125,6 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { assert(newCol.expr.asInstanceOf[NamedExpression].metadata.getString("key") === "value") } - test("single explode") { - val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") - checkAnswer( - df.select(explode('intList)), - Row(1) :: Row(2) :: Row(3) :: Nil) - } - - test("explode and other columns") { - val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") - - checkAnswer( - df.select($"a", explode('intList)), - Row(1, 1) :: - Row(1, 2) :: - Row(1, 3) :: Nil) - - checkAnswer( - df.select($"*", explode('intList)), - Row(1, Seq(1, 2, 3), 1) :: - Row(1, Seq(1, 2, 3), 2) :: - Row(1, Seq(1, 2, 3), 3) :: Nil) - } - - test("aliased explode") { - val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") - - checkAnswer( - df.select(explode('intList).as('int)).select('int), - Row(1) :: Row(2) :: Row(3) :: Nil) - - checkAnswer( - df.select(explode('intList).as('int)).select(sum('int)), - Row(6) :: Nil) - } - - test("explode on map") { - val df = Seq((1, Map("a" -> "b"))).toDF("a", "map") - - checkAnswer( - df.select(explode('map)), - Row("a", "b")) - } - - test("explode on map with aliases") { - val df = Seq((1, Map("a" -> "b"))).toDF("a", "map") - - checkAnswer( - df.select(explode('map).as("key1" :: "value1" :: Nil)).select("key1", "value1"), - Row("a", "b")) - } - - test("self join explode") { - val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") - val exploded = df.select(explode('intList).as('i)) - - checkAnswer( - exploded.join(exploded, exploded("i") === exploded("i")).agg(count("*")), - Row(3) :: Nil) - } - test("collect on column produced by a binary operator") { val df = Seq((1, 2, 3)).toDF("a", "b", "c") checkAnswer(df.select(df("a") + df("b")), Seq(Row(3))) @@ -286,7 +232,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { } test("isNaN") { - val testData = sqlContext.createDataFrame(sparkContext.parallelize( + val testData = spark.createDataFrame(sparkContext.parallelize( Row(Double.NaN, Float.NaN) :: Row(math.log(-1), math.log(-3).toFloat) :: Row(null, null) :: @@ -298,7 +244,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { Row(true, true) :: Row(true, true) :: Row(false, false) :: Row(false, false) :: Nil) checkAnswer( - testData.select(isNaN($"a"), isNaN($"b")), + testData.select(isnan($"a"), isnan($"b")), Row(true, true) :: Row(true, true) :: Row(false, false) :: Row(false, false) :: Nil) checkAnswer( @@ -307,7 +253,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { } test("nanvl") { - val testData = sqlContext.createDataFrame(sparkContext.parallelize( + val testData = spark.createDataFrame(sparkContext.parallelize( Row(null, 3.0, Double.NaN, Double.PositiveInfinity, 1.0f, 4) :: Nil), StructType(Seq(StructField("a", DoubleType), StructField("b", DoubleType), StructField("c", DoubleType), StructField("d", DoubleType), @@ -320,7 +266,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { nanvl($"b", $"e"), nanvl($"e", $"f")), Row(null, 3.0, 10.0, null, Double.PositiveInfinity, 3.0, 1.0) ) - testData.registerTempTable("t") + testData.createOrReplaceTempView("t") checkAnswer( sql( "select nanvl(a, 5), nanvl(b, 10), nanvl(10, b), nanvl(c, null), nanvl(d, 10), " + @@ -340,23 +286,6 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { } test("<=>") { - checkAnswer( - testData2.filter($"a" === 1), - testData2.collect().toSeq.filter(r => r.getInt(0) == 1)) - - checkAnswer( - testData2.filter($"a" === $"b"), - testData2.collect().toSeq.filter(r => r.getInt(0) == r.getInt(1))) - } - - test("!==") { - val nullData = sqlContext.createDataFrame(sparkContext.parallelize( - Row(1, 1) :: - Row(1, 2) :: - Row(1, null) :: - Row(null, null) :: Nil), - StructType(Seq(StructField("a", IntegerType), StructField("b", IntegerType)))) - checkAnswer( nullData.filter($"b" <=> 1), Row(1, 1) :: Nil) @@ -368,6 +297,28 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { checkAnswer( nullData.filter($"a" <=> $"b"), Row(1, 1) :: Row(null, null) :: Nil) + + val nullData2 = spark.createDataFrame(sparkContext.parallelize( + Row("abc") :: + Row(null) :: + Row("xyz") :: Nil), + StructType(Seq(StructField("a", StringType, true)))) + + checkAnswer( + nullData2.filter($"a" <=> null), + Row(null) :: Nil) + } + + test("=!=") { + checkAnswer( + nullData.filter($"b" =!= 1), + Row(1, 2) :: Nil) + + checkAnswer(nullData.filter($"b" =!= null), Nil) + + checkAnswer( + nullData.filter($"a" =!= $"b"), + Row(1, 2) :: Nil) } test(">") { @@ -554,37 +505,161 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { Row("ab", "cde")) } - test("monotonicallyIncreasingId") { + test("monotonically_increasing_id") { // Make sure we have 2 partitions, each with 2 records. val df = sparkContext.parallelize(Seq[Int](), 2).mapPartitions { _ => Iterator(Tuple1(1), Tuple1(2)) }.toDF("a") checkAnswer( - df.select(monotonicallyIncreasingId()), - Row(0L) :: Row(1L) :: Row((1L << 33) + 0L) :: Row((1L << 33) + 1L) :: Nil + df.select(monotonically_increasing_id(), expr("monotonically_increasing_id()")), + Row(0L, 0L) :: + Row(1L, 1L) :: + Row((1L << 33) + 0L, (1L << 33) + 0L) :: + Row((1L << 33) + 1L, (1L << 33) + 1L) :: Nil ) } - test("sparkPartitionId") { + test("spark_partition_id") { // Make sure we have 2 partitions, each with 2 records. val df = sparkContext.parallelize(Seq[Int](), 2).mapPartitions { _ => Iterator(Tuple1(1), Tuple1(2)) }.toDF("a") checkAnswer( - df.select(sparkPartitionId()), + df.select(spark_partition_id()), Row(0) :: Row(0) :: Row(1) :: Row(1) :: Nil ) } - test("InputFileName") { + test("input_file_name, input_file_block_start, input_file_block_length - more than one source") { + withTempView("tempView1") { + withTable("tab1", "tab2") { + val data = sparkContext.parallelize(0 to 9).toDF("id") + data.write.saveAsTable("tab1") + data.write.saveAsTable("tab2") + data.createOrReplaceTempView("tempView1") + Seq("input_file_name", "input_file_block_start", "input_file_block_length").foreach { f => + val e = intercept[AnalysisException] { + sql(s"SELECT *, $f() FROM tab1 JOIN tab2 ON tab1.id = tab2.id") + }.getMessage + assert(e.contains(s"'$f' does not support more than one source")) + } + + def checkResult( + fromClause: String, + exceptionExpected: Boolean, + numExpectedRows: Int = 0): Unit = { + val stmt = s"SELECT *, input_file_name() FROM ($fromClause)" + if (exceptionExpected) { + val e = intercept[AnalysisException](sql(stmt)).getMessage + assert(e.contains("'input_file_name' does not support more than one source")) + } else { + assert(sql(stmt).count() == numExpectedRows) + } + } + + checkResult( + "SELECT * FROM tab1 UNION ALL SELECT * FROM tab2 UNION ALL SELECT * FROM tab2", + exceptionExpected = false, + numExpectedRows = 30) + + checkResult( + "(SELECT * FROM tempView1 NATURAL JOIN tab2) UNION ALL SELECT * FROM tab2", + exceptionExpected = false, + numExpectedRows = 20) + + checkResult( + "(SELECT * FROM tab1 UNION ALL SELECT * FROM tab2) NATURAL JOIN tempView1", + exceptionExpected = false, + numExpectedRows = 20) + + checkResult( + "(SELECT * FROM tempView1 UNION ALL SELECT * FROM tab2) NATURAL JOIN tab2", + exceptionExpected = true) + + checkResult( + "(SELECT * FROM tab1 NATURAL JOIN tab2) UNION ALL SELECT * FROM tab2", + exceptionExpected = true) + + checkResult( + "(SELECT * FROM tab1 UNION ALL SELECT * FROM tab2) NATURAL JOIN tab2", + exceptionExpected = true) + } + } + } + + test("input_file_name, input_file_block_start, input_file_block_length - FileScanRDD") { withTempPath { dir => val data = sparkContext.parallelize(0 to 10).toDF("id") data.write.parquet(dir.getCanonicalPath) - val answer = sqlContext.read.parquet(dir.getCanonicalPath).select(inputFileName()) - .head.getString(0) - assert(answer.contains(dir.getCanonicalPath)) - checkAnswer(data.select(inputFileName()).limit(1), Row("")) + // Test the 3 expressions when reading from files + val q = spark.read.parquet(dir.getCanonicalPath).select( + input_file_name(), expr("input_file_block_start()"), expr("input_file_block_length()")) + val firstRow = q.head() + assert(firstRow.getString(0).contains(dir.toURI.getPath)) + assert(firstRow.getLong(1) == 0) + assert(firstRow.getLong(2) > 0) + + // Now read directly from the original RDD without going through any files to make sure + // we are returning empty string, -1, and -1. + checkAnswer( + data.select( + input_file_name(), expr("input_file_block_start()"), expr("input_file_block_length()") + ).limit(1), + Row("", -1L, -1L)) + } + } + + test("input_file_name, input_file_block_start, input_file_block_length - HadoopRDD") { + withTempPath { dir => + val data = sparkContext.parallelize((0 to 10).map(_.toString)).toDF() + data.write.text(dir.getCanonicalPath) + val df = spark.sparkContext.textFile(dir.getCanonicalPath).toDF() + + // Test the 3 expressions when reading from files + val q = df.select( + input_file_name(), expr("input_file_block_start()"), expr("input_file_block_length()")) + val firstRow = q.head() + assert(firstRow.getString(0).contains(dir.toURI.getPath)) + assert(firstRow.getLong(1) == 0) + assert(firstRow.getLong(2) > 0) + + // Now read directly from the original RDD without going through any files to make sure + // we are returning empty string, -1, and -1. + checkAnswer( + data.select( + input_file_name(), expr("input_file_block_start()"), expr("input_file_block_length()") + ).limit(1), + Row("", -1L, -1L)) + } + } + + test("input_file_name, input_file_block_start, input_file_block_length - NewHadoopRDD") { + withTempPath { dir => + val data = sparkContext.parallelize((0 to 10).map(_.toString)).toDF() + data.write.text(dir.getCanonicalPath) + val rdd = spark.sparkContext.newAPIHadoopFile( + dir.getCanonicalPath, + classOf[NewTextInputFormat], + classOf[LongWritable], + classOf[Text]) + val df = rdd.map(pair => pair._2.toString).toDF() + + // Test the 3 expressions when reading from files + val q = df.select( + input_file_name(), expr("input_file_block_start()"), expr("input_file_block_length()")) + val firstRow = q.head() + assert(firstRow.getString(0).contains(dir.toURI.getPath)) + assert(firstRow.getLong(1) == 0) + assert(firstRow.getLong(2) > 0) + + // Now read directly from the original RDD without going through any files to make sure + // we are returning empty string, -1, and -1. + checkAnswer( + data.select( + input_file_name(), expr("input_file_block_start()"), expr("input_file_block_length()") + ).limit(1), + Row("", -1L, -1L)) } } @@ -614,9 +689,8 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { } def checkNumProjects(df: DataFrame, expectedNumProjects: Int): Unit = { - val projects = df.queryExecution.executedPlan.collect { - case project: Project => project - case tungstenProject: TungstenProject => tungstenProject + val projects = df.queryExecution.sparkPlan.collect { + case tungstenProject: ProjectExec => tungstenProject } assert(projects.size === expectedNumProjects) } @@ -693,4 +767,17 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { testData2.collect().toSeq.map(r => Row(r.getInt(0) ^ r.getInt(1) ^ 39))) } + test("typedLit") { + val df = Seq(Tuple1(0)).toDF("a") + // Only check the types `lit` cannot handle + checkAnswer( + df.select(typedLit(Seq(1, 2, 3))), + Row(Seq(1, 2, 3)) :: Nil) + checkAnswer( + df.select(typedLit(Map("a" -> 1, "b" -> 2))), + Row(Map("a" -> 1, "b" -> 2)) :: Nil) + checkAnswer( + df.select(typedLit(("a", 2, 1.0))), + Row(Row("a", 2, 1.0)) :: Nil) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CountMinSketchAggQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CountMinSketchAggQuerySuite.scala new file mode 100644 index 000000000000..dea0d4c0c6d4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/CountMinSketchAggQuerySuite.scala @@ -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 + +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.util.sketch.CountMinSketch + +/** + * End-to-end test suite for count_min_sketch. + */ +class CountMinSketchAggQuerySuite extends QueryTest with SharedSQLContext { + + test("count-min sketch") { + import testImplicits._ + + val eps = 0.1 + val confidence = 0.95 + val seed = 11 + + val items = Seq(1, 1, 2, 2, 2, 2, 3, 4, 5) + val sketch = CountMinSketch.readFrom(items.toDF("id") + .selectExpr(s"count_min_sketch(id, ${eps}d, ${confidence}d, $seed)") + .head().get(0).asInstanceOf[Array[Byte]]) + + val reference = CountMinSketch.create(eps, confidence, seed) + items.foreach(reference.add) + + assert(sketch == reference) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 2e679e7bc4e0..8549eac58ee9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -17,10 +17,18 @@ package org.apache.spark.sql +import scala.util.Random + +import org.apache.spark.sql.execution.WholeStageCodegenExec +import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.DecimalType +import org.apache.spark.sql.test.SQLTestData.DecimalData +import org.apache.spark.sql.types.{Decimal, DecimalType} +case class Fact(date: Int, hour: Int, minute: Int, room_name: String, temp: Double) class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -58,6 +66,178 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { df1.groupBy("key").min("value2"), Seq(Row("a", 0), Row("b", 4)) ) + + checkAnswer( + decimalData.groupBy("a").agg(sum("b")), + Seq(Row(new java.math.BigDecimal(1), new java.math.BigDecimal(3)), + Row(new java.math.BigDecimal(2), new java.math.BigDecimal(3)), + Row(new java.math.BigDecimal(3), new java.math.BigDecimal(3))) + ) + + val decimalDataWithNulls = spark.sparkContext.parallelize( + DecimalData(1, 1) :: + DecimalData(1, null) :: + DecimalData(2, 1) :: + DecimalData(2, null) :: + DecimalData(3, 1) :: + DecimalData(3, 2) :: + DecimalData(null, 2) :: Nil).toDF() + checkAnswer( + decimalDataWithNulls.groupBy("a").agg(sum("b")), + Seq(Row(new java.math.BigDecimal(1), new java.math.BigDecimal(1)), + Row(new java.math.BigDecimal(2), new java.math.BigDecimal(1)), + Row(new java.math.BigDecimal(3), new java.math.BigDecimal(3)), + Row(null, new java.math.BigDecimal(2))) + ) + } + + test("SPARK-17124 agg should be ordering preserving") { + val df = spark.range(2) + val ret = df.groupBy("id").agg("id" -> "sum", "id" -> "count", "id" -> "min") + assert(ret.schema.map(_.name) == Seq("id", "sum(id)", "count(id)", "min(id)")) + checkAnswer( + ret, + Row(0, 0, 1, 0) :: Row(1, 1, 1, 1) :: Nil + ) + } + + test("SPARK-18952: regexes fail codegen when used as keys due to bad forward-slash escapes") { + val df = Seq(("some[thing]", "random-string")).toDF("key", "val") + + checkAnswer( + df.groupBy(regexp_extract('key, "([a-z]+)\\[", 1)).count(), + Row("some", 1) :: Nil + ) + } + + test("rollup") { + checkAnswer( + courseSales.rollup("course", "year").sum("earnings"), + Row("Java", 2012, 20000.0) :: + Row("Java", 2013, 30000.0) :: + Row("Java", null, 50000.0) :: + Row("dotNET", 2012, 15000.0) :: + Row("dotNET", 2013, 48000.0) :: + Row("dotNET", null, 63000.0) :: + Row(null, null, 113000.0) :: Nil + ) + } + + test("cube") { + checkAnswer( + courseSales.cube("course", "year").sum("earnings"), + Row("Java", 2012, 20000.0) :: + Row("Java", 2013, 30000.0) :: + Row("Java", null, 50000.0) :: + Row("dotNET", 2012, 15000.0) :: + Row("dotNET", 2013, 48000.0) :: + Row("dotNET", null, 63000.0) :: + Row(null, 2012, 35000.0) :: + Row(null, 2013, 78000.0) :: + Row(null, null, 113000.0) :: Nil + ) + + val df0 = spark.sparkContext.parallelize(Seq( + Fact(20151123, 18, 35, "room1", 18.6), + Fact(20151123, 18, 35, "room2", 22.4), + Fact(20151123, 18, 36, "room1", 17.4), + Fact(20151123, 18, 36, "room2", 25.6))).toDF() + + val cube0 = df0.cube("date", "hour", "minute", "room_name").agg(Map("temp" -> "avg")) + assert(cube0.where("date IS NULL").count > 0) + } + + test("grouping and grouping_id") { + checkAnswer( + courseSales.cube("course", "year") + .agg(grouping("course"), grouping("year"), grouping_id("course", "year")), + Row("Java", 2012, 0, 0, 0) :: + Row("Java", 2013, 0, 0, 0) :: + Row("Java", null, 0, 1, 1) :: + Row("dotNET", 2012, 0, 0, 0) :: + Row("dotNET", 2013, 0, 0, 0) :: + Row("dotNET", null, 0, 1, 1) :: + Row(null, 2012, 1, 0, 2) :: + Row(null, 2013, 1, 0, 2) :: + Row(null, null, 1, 1, 3) :: Nil + ) + + intercept[AnalysisException] { + courseSales.groupBy().agg(grouping("course")).explain() + } + intercept[AnalysisException] { + courseSales.groupBy().agg(grouping_id("course")).explain() + } + } + + test("grouping/grouping_id inside window function") { + + val w = Window.orderBy(sum("earnings")) + checkAnswer( + courseSales.cube("course", "year") + .agg(sum("earnings"), + grouping_id("course", "year"), + rank().over(Window.partitionBy(grouping_id("course", "year")).orderBy(sum("earnings")))), + Row("Java", 2012, 20000.0, 0, 2) :: + Row("Java", 2013, 30000.0, 0, 3) :: + Row("Java", null, 50000.0, 1, 1) :: + Row("dotNET", 2012, 15000.0, 0, 1) :: + Row("dotNET", 2013, 48000.0, 0, 4) :: + Row("dotNET", null, 63000.0, 1, 2) :: + Row(null, 2012, 35000.0, 2, 1) :: + Row(null, 2013, 78000.0, 2, 2) :: + Row(null, null, 113000.0, 3, 1) :: Nil + ) + } + + test("SPARK-21980: References in grouping functions should be indexed with semanticEquals") { + checkAnswer( + courseSales.cube("course", "year") + .agg(grouping("CouRse"), grouping("year")), + Row("Java", 2012, 0, 0) :: + Row("Java", 2013, 0, 0) :: + Row("Java", null, 0, 1) :: + Row("dotNET", 2012, 0, 0) :: + Row("dotNET", 2013, 0, 0) :: + Row("dotNET", null, 0, 1) :: + Row(null, 2012, 1, 0) :: + Row(null, 2013, 1, 0) :: + Row(null, null, 1, 1) :: Nil + ) + } + + test("rollup overlapping columns") { + checkAnswer( + testData2.rollup($"a" + $"b" as "foo", $"b" as "bar").agg(sum($"a" - $"b") as "foo"), + Row(2, 1, 0) :: Row(3, 2, -1) :: Row(3, 1, 1) :: Row(4, 2, 0) :: Row(4, 1, 2) :: Row(5, 2, 1) + :: Row(2, null, 0) :: Row(3, null, 0) :: Row(4, null, 2) :: Row(5, null, 1) + :: Row(null, null, 3) :: Nil + ) + + checkAnswer( + testData2.rollup("a", "b").agg(sum("b")), + Row(1, 1, 1) :: Row(1, 2, 2) :: Row(2, 1, 1) :: Row(2, 2, 2) :: Row(3, 1, 1) :: Row(3, 2, 2) + :: Row(1, null, 3) :: Row(2, null, 3) :: Row(3, null, 3) + :: Row(null, null, 9) :: Nil + ) + } + + test("cube overlapping columns") { + checkAnswer( + testData2.cube($"a" + $"b", $"b").agg(sum($"a" - $"b")), + Row(2, 1, 0) :: Row(3, 2, -1) :: Row(3, 1, 1) :: Row(4, 2, 0) :: Row(4, 1, 2) :: Row(5, 2, 1) + :: Row(2, null, 0) :: Row(3, null, 0) :: Row(4, null, 2) :: Row(5, null, 1) + :: Row(null, 1, 3) :: Row(null, 2, 0) + :: Row(null, null, 3) :: Nil + ) + + checkAnswer( + testData2.cube("a", "b").agg(sum("b")), + Row(1, 1, 1) :: Row(1, 2, 2) :: Row(2, 1, 1) :: Row(2, 2, 2) :: Row(3, 1, 1) :: Row(3, 2, 2) + :: Row(1, null, 3) :: Row(2, null, 3) :: Row(3, null, 3) + :: Row(null, 1, 3) :: Row(null, 2, 6) + :: Row(null, null, 9) :: Nil + ) } test("spark.sql.retainGroupColumns config") { @@ -66,12 +246,12 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { Seq(Row(1, 3), Row(2, 3), Row(3, 3)) ) - sqlContext.conf.setConf(SQLConf.DATAFRAME_RETAIN_GROUP_COLUMNS, false) + spark.conf.set(SQLConf.DATAFRAME_RETAIN_GROUP_COLUMNS.key, false) checkAnswer( testData2.groupBy("a").agg(sum($"b")), Seq(Row(3), Row(3), Row(3)) ) - sqlContext.conf.setConf(SQLConf.DATAFRAME_RETAIN_GROUP_COLUMNS, true) + spark.conf.set(SQLConf.DATAFRAME_RETAIN_GROUP_COLUMNS.key, true) } test("agg without groups") { @@ -81,6 +261,13 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { ) } + test("agg without groups and functions") { + checkAnswer( + testData2.agg(lit(1)), + Row(1) + ) + } + test("average") { checkAnswer( testData2.agg(avg('a), mean('a)), @@ -92,19 +279,19 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { checkAnswer( decimalData.agg(avg('a)), - Row(new java.math.BigDecimal(2.0))) + Row(new java.math.BigDecimal(2))) checkAnswer( decimalData.agg(avg('a), sumDistinct('a)), // non-partial - Row(new java.math.BigDecimal(2.0), new java.math.BigDecimal(6)) :: Nil) + Row(new java.math.BigDecimal(2), new java.math.BigDecimal(6)) :: Nil) checkAnswer( decimalData.agg(avg('a cast DecimalType(10, 2))), - Row(new java.math.BigDecimal(2.0))) + Row(new java.math.BigDecimal(2))) // non-partial checkAnswer( decimalData.agg(avg('a cast DecimalType(10, 2)), sumDistinct('a cast DecimalType(10, 2))), - Row(new java.math.BigDecimal(2.0), new java.math.BigDecimal(6)) :: Nil) + Row(new java.math.BigDecimal(2), new java.math.BigDecimal(6)) :: Nil) } test("null average") { @@ -133,7 +320,7 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { } test("count") { - assert(testData2.count() === testData2.map(_ => 1).count()) + assert(testData2.count() === testData2.rdd.map(_ => 1).count()) checkAnswer( testData2.agg(count('a), sumDistinct('a)), // non-partial @@ -162,6 +349,31 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { ) } + test("multiple column distinct count") { + val df1 = Seq( + ("a", "b", "c"), + ("a", "b", "c"), + ("a", "b", "d"), + ("x", "y", "z"), + ("x", "q", null.asInstanceOf[String])) + .toDF("key1", "key2", "key3") + + checkAnswer( + df1.agg(countDistinct('key1, 'key2)), + Row(3) + ) + + checkAnswer( + df1.agg(countDistinct('key1, 'key2, 'key3)), + Row(3) + ) + + checkAnswer( + df1.groupBy('key1).agg(countDistinct('key2, 'key3)), + Seq(Row("a", 2), Row("x", 1)) + ) + } + test("zero count") { val emptyTableData = Seq.empty[(Int, Int)].toDF("a", "b") checkAnswer( @@ -170,10 +382,13 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { } test("stddev") { - val testData2ADev = math.sqrt(4 / 5.0) + val testData2ADev = math.sqrt(4.0 / 5.0) checkAnswer( testData2.agg(stddev('a), stddev_pop('a), stddev_samp('a)), Row(testData2ADev, math.sqrt(4 / 6.0), testData2ADev)) + checkAnswer( + testData2.agg(stddev("a"), stddev_pop("a"), stddev_samp("a")), + Row(testData2ADev, math.sqrt(4 / 6.0), testData2ADev)) } test("zero stddev") { @@ -219,17 +434,23 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { test("zero moments") { val input = Seq((1, 2)).toDF("a", "b") checkAnswer( - input.agg(variance('a), var_samp('a), var_pop('a), skewness('a), kurtosis('a)), - Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN)) + input.agg(stddev('a), stddev_samp('a), stddev_pop('a), variance('a), + var_samp('a), var_pop('a), skewness('a), kurtosis('a)), + Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, + Double.NaN, Double.NaN)) checkAnswer( input.agg( + expr("stddev(a)"), + expr("stddev_samp(a)"), + expr("stddev_pop(a)"), expr("variance(a)"), expr("var_samp(a)"), expr("var_pop(a)"), expr("skewness(a)"), expr("kurtosis(a)")), - Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN)) + Row(Double.NaN, Double.NaN, 0.0, Double.NaN, Double.NaN, 0.0, + Double.NaN, Double.NaN)) } test("null moments") { @@ -237,7 +458,7 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { checkAnswer( emptyTableData.agg(variance('a), var_samp('a), var_pop('a), skewness('a), kurtosis('a)), - Row(Double.NaN, Double.NaN, Double.NaN, Double.NaN, Double.NaN)) + Row(null, null, null, null, null)) checkAnswer( emptyTableData.agg( @@ -246,6 +467,173 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { expr("var_pop(a)"), expr("skewness(a)"), expr("kurtosis(a)")), - Row(Double.NaN, Double.NaN, Double.NaN, Double.NaN, Double.NaN)) + Row(null, null, null, null, null)) + } + + test("collect functions") { + val df = Seq((1, 2), (2, 2), (3, 4)).toDF("a", "b") + checkAnswer( + df.select(collect_list($"a"), collect_list($"b")), + Seq(Row(Seq(1, 2, 3), Seq(2, 2, 4))) + ) + checkAnswer( + df.select(collect_set($"a"), collect_set($"b")), + Seq(Row(Seq(1, 2, 3), Seq(2, 4))) + ) + + checkDataset( + df.select(collect_set($"a").as("aSet")).as[Set[Int]], + Set(1, 2, 3)) + checkDataset( + df.select(collect_set($"b").as("bSet")).as[Set[Int]], + Set(2, 4)) + checkDataset( + df.select(collect_set($"a"), collect_set($"b")).as[(Set[Int], Set[Int])], + Seq(Set(1, 2, 3) -> Set(2, 4)): _*) + } + + test("collect functions structs") { + val df = Seq((1, 2, 2), (2, 2, 2), (3, 4, 1)) + .toDF("a", "x", "y") + .select($"a", struct($"x", $"y").as("b")) + checkAnswer( + df.select(collect_list($"a"), sort_array(collect_list($"b"))), + Seq(Row(Seq(1, 2, 3), Seq(Row(2, 2), Row(2, 2), Row(4, 1)))) + ) + checkAnswer( + df.select(collect_set($"a"), sort_array(collect_set($"b"))), + Seq(Row(Seq(1, 2, 3), Seq(Row(2, 2), Row(4, 1)))) + ) + } + + test("collect_set functions cannot have maps") { + val df = Seq((1, 3, 0), (2, 3, 0), (3, 4, 1)) + .toDF("a", "x", "y") + .select($"a", map($"x", $"y").as("b")) + val error = intercept[AnalysisException] { + df.select(collect_set($"a"), collect_set($"b")) + } + assert(error.message.contains("collect_set() cannot have map type data")) + } + + test("SPARK-17641: collect functions should not collect null values") { + val df = Seq(("1", 2), (null, 2), ("1", 4)).toDF("a", "b") + checkAnswer( + df.select(collect_list($"a"), collect_list($"b")), + Seq(Row(Seq("1", "1"), Seq(2, 2, 4))) + ) + checkAnswer( + df.select(collect_set($"a"), collect_set($"b")), + Seq(Row(Seq("1"), Seq(2, 4))) + ) + } + + test("SPARK-14664: Decimal sum/avg over window should work.") { + checkAnswer( + spark.sql("select sum(a) over () from values 1.0, 2.0, 3.0 T(a)"), + Row(6.0) :: Row(6.0) :: Row(6.0) :: Nil) + checkAnswer( + spark.sql("select avg(a) over () from values 1.0, 2.0, 3.0 T(a)"), + Row(2.0) :: Row(2.0) :: Row(2.0) :: Nil) + } + + test("SQL decimal test (used for catching certain decimal handling bugs in aggregates)") { + checkAnswer( + decimalData.groupBy('a cast DecimalType(10, 2)).agg(avg('b cast DecimalType(10, 2))), + Seq(Row(new java.math.BigDecimal(1), new java.math.BigDecimal("1.5")), + Row(new java.math.BigDecimal(2), new java.math.BigDecimal("1.5")), + Row(new java.math.BigDecimal(3), new java.math.BigDecimal("1.5")))) + } + + test("SPARK-17616: distinct aggregate combined with a non-partial aggregate") { + val df = Seq((1, 3, "a"), (1, 2, "b"), (3, 4, "c"), (3, 4, "c"), (3, 5, "d")) + .toDF("x", "y", "z") + checkAnswer( + df.groupBy($"x").agg(countDistinct($"y"), sort_array(collect_list($"z"))), + Seq(Row(1, 2, Seq("a", "b")), Row(3, 2, Seq("c", "c", "d")))) + } + + test("SPARK-18004 limit + aggregates") { + val df = Seq(("a", 1), ("b", 2), ("c", 1), ("d", 5)).toDF("id", "value") + val limit2Df = df.limit(2) + checkAnswer( + limit2Df.groupBy("id").count().select($"id"), + limit2Df.select($"id")) + } + + test("SPARK-17237 remove backticks in a pivot result schema") { + val df = Seq((2, 3, 4), (3, 4, 5)).toDF("a", "x", "y") + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + checkAnswer( + df.groupBy("a").pivot("x").agg(count("y"), avg("y")).na.fill(0), + Seq(Row(3, 0, 0.0, 1, 5.0), Row(2, 1, 4.0, 0, 0.0)) + ) + } + } + + test("aggregate function in GROUP BY") { + val e = intercept[AnalysisException] { + testData.groupBy(sum($"key")).count() + } + assert(e.message.contains("aggregate functions are not allowed in GROUP BY")) + } + + private def assertNoExceptions(c: Column): Unit = { + for ((wholeStage, useObjectHashAgg) <- + Seq((true, true), (true, false), (false, true), (false, false))) { + withSQLConf( + (SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, wholeStage.toString), + (SQLConf.USE_OBJECT_HASH_AGG.key, useObjectHashAgg.toString)) { + + val df = Seq(("1", 1), ("1", 2), ("2", 3), ("2", 4)).toDF("x", "y") + + // test case for HashAggregate + val hashAggDF = df.groupBy("x").agg(c, sum("y")) + val hashAggPlan = hashAggDF.queryExecution.executedPlan + if (wholeStage) { + assert(hashAggPlan.find { + case WholeStageCodegenExec(_: HashAggregateExec) => true + case _ => false + }.isDefined) + } else { + assert(hashAggPlan.isInstanceOf[HashAggregateExec]) + } + hashAggDF.collect() + + // test case for ObjectHashAggregate and SortAggregate + val objHashAggOrSortAggDF = df.groupBy("x").agg(c, collect_list("y")) + val objHashAggOrSortAggPlan = objHashAggOrSortAggDF.queryExecution.executedPlan + if (useObjectHashAgg) { + assert(objHashAggOrSortAggPlan.isInstanceOf[ObjectHashAggregateExec]) + } else { + assert(objHashAggOrSortAggPlan.isInstanceOf[SortAggregateExec]) + } + objHashAggOrSortAggDF.collect() + } + } + } + + test("SPARK-19471: AggregationIterator does not initialize the generated result projection" + + " before using it") { + Seq( + monotonically_increasing_id(), spark_partition_id(), + rand(Random.nextLong()), randn(Random.nextLong()) + ).foreach(assertNoExceptions) + } + + test("SPARK-21580 ints in aggregation expressions are taken as group-by ordinal.") { + checkAnswer( + testData2.groupBy(lit(3), lit(4)).agg(lit(6), lit(7), sum("b")), + Seq(Row(3, 4, 6, 7, 9))) + checkAnswer( + testData2.groupBy(lit(3), lit(4)).agg(lit(6), 'b, sum("b")), + Seq(Row(3, 4, 6, 1, 3), Row(3, 4, 6, 2, 6))) + + checkAnswer( + spark.sql("SELECT 3, 4, SUM(b) FROM testData2 GROUP BY 1, 2"), + Seq(Row(3, 4, 9))) + checkAnswer( + spark.sql("SELECT 3 AS c, 4 AS d, SUM(b) FROM testData2 GROUP BY c, d"), + Seq(Row(3, 4, 9))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala index 09f7b507670c..1230b921aa27 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.sql.catalyst.DefinedByConstructorParams import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext @@ -41,6 +42,60 @@ class DataFrameComplexTypeSuite extends QueryTest with SharedSQLContext { test("UDF on array") { val f = udf((a: String) => a) val df = sparkContext.parallelize(Seq((1, 1))).toDF("a", "b") - df.select(array($"a").as("s")).select(f(expr("s[0]"))).collect() + df.select(array($"a").as("s")).select(f($"s".getItem(0))).collect() + } + + test("UDF on map") { + val f = udf((a: String) => a) + val df = Seq("a" -> 1).toDF("a", "b") + df.select(map($"a", $"b").as("s")).select(f($"s".getItem("a"))).collect() + } + + test("SPARK-12477 accessing null element in array field") { + val df = sparkContext.parallelize(Seq((Seq("val1", null, "val2"), + Seq(Some(1), None, Some(2))))).toDF("s", "i") + val nullStringRow = df.selectExpr("s[1]").collect()(0) + assert(nullStringRow == org.apache.spark.sql.Row(null)) + val nullIntRow = df.selectExpr("i[1]").collect()(0) + assert(nullIntRow == org.apache.spark.sql.Row(null)) + } + + test("SPARK-15285 Generated SpecificSafeProjection.apply method grows beyond 64KB") { + val ds100_5 = Seq(S100_5()).toDS() + ds100_5.rdd.count } } + +class S100( + val s1: String = "1", val s2: String = "2", val s3: String = "3", val s4: String = "4", + val s5: String = "5", val s6: String = "6", val s7: String = "7", val s8: String = "8", + val s9: String = "9", val s10: String = "10", val s11: String = "11", val s12: String = "12", + val s13: String = "13", val s14: String = "14", val s15: String = "15", val s16: String = "16", + val s17: String = "17", val s18: String = "18", val s19: String = "19", val s20: String = "20", + val s21: String = "21", val s22: String = "22", val s23: String = "23", val s24: String = "24", + val s25: String = "25", val s26: String = "26", val s27: String = "27", val s28: String = "28", + val s29: String = "29", val s30: String = "30", val s31: String = "31", val s32: String = "32", + val s33: String = "33", val s34: String = "34", val s35: String = "35", val s36: String = "36", + val s37: String = "37", val s38: String = "38", val s39: String = "39", val s40: String = "40", + val s41: String = "41", val s42: String = "42", val s43: String = "43", val s44: String = "44", + val s45: String = "45", val s46: String = "46", val s47: String = "47", val s48: String = "48", + val s49: String = "49", val s50: String = "50", val s51: String = "51", val s52: String = "52", + val s53: String = "53", val s54: String = "54", val s55: String = "55", val s56: String = "56", + val s57: String = "57", val s58: String = "58", val s59: String = "59", val s60: String = "60", + val s61: String = "61", val s62: String = "62", val s63: String = "63", val s64: String = "64", + val s65: String = "65", val s66: String = "66", val s67: String = "67", val s68: String = "68", + val s69: String = "69", val s70: String = "70", val s71: String = "71", val s72: String = "72", + val s73: String = "73", val s74: String = "74", val s75: String = "75", val s76: String = "76", + val s77: String = "77", val s78: String = "78", val s79: String = "79", val s80: String = "80", + val s81: String = "81", val s82: String = "82", val s83: String = "83", val s84: String = "84", + val s85: String = "85", val s86: String = "86", val s87: String = "87", val s88: String = "88", + val s89: String = "89", val s90: String = "90", val s91: String = "91", val s92: String = "92", + val s93: String = "93", val s94: String = "94", val s95: String = "95", val s96: String = "96", + val s97: String = "97", val s98: String = "98", val s99: String = "99", val s100: String = "100") +extends DefinedByConstructorParams + +case class S100_5( + s1: S100 = new S100(), s2: S100 = new S100(), s3: S100 = new S100(), + s4: S100 = new S100(), s5: S100 = new S100()) + + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 3a3f19af1473..50e475984f45 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -17,7 +17,15 @@ package org.apache.spark.sql +import java.nio.charset.StandardCharsets + +import scala.util.Random + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -42,15 +50,16 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { val expectedType = ArrayType(IntegerType, containsNull = false) assert(row.schema(0).dataType === expectedType) - assert(row.getAs[Seq[Int]](0) === Seq(0, 2)) + assert(row.getSeq[Int](0) === Seq(0, 2)) } - // Turn this on once we add a rule to the analyzer to throw a friendly exception - ignore("array: throw exception if putting columns of different types into an array") { - val df = Seq((0, "str")).toDF("a", "b") - intercept[AnalysisException] { - df.select(array("a", "b")) - } + test("map with column expressions") { + val df = Seq(1 -> "a").toDF("a", "b") + val row = df.select(map($"a" + 1, $"b")).first() + + val expectedType = MapType(IntegerType, StringType, valueContainsNull = true) + assert(row.schema(0).dataType === expectedType) + assert(row.getMap[Int, String](0) === Map(2 -> "a")) } test("struct with column name") { @@ -149,12 +158,6 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { Row("one", "not_one")) } - test("nvl function") { - checkAnswer( - sql("SELECT nvl(null, 'x'), nvl('y', 'x'), nvl(null, null)"), - Row("x", "y", null)) - } - test("misc md5 function") { val df = Seq(("ABC", Array[Byte](1, 2, 3, 4, 5, 6))).toDF("a", "b") checkAnswer( @@ -167,12 +170,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { } test("misc sha1 function") { - val df = Seq(("ABC", "ABC".getBytes)).toDF("a", "b") + val df = Seq(("ABC", "ABC".getBytes(StandardCharsets.UTF_8))).toDF("a", "b") checkAnswer( df.select(sha1($"a"), sha1($"b")), Row("3c01bdbb26f358bab27f267924aa2c9a03fcfdb8", "3c01bdbb26f358bab27f267924aa2c9a03fcfdb8")) - val dfEmpty = Seq(("", "".getBytes)).toDF("a", "b") + val dfEmpty = Seq(("", "".getBytes(StandardCharsets.UTF_8))).toDF("a", "b") checkAnswer( dfEmpty.selectExpr("sha1(a)", "sha1(b)"), Row("da39a3ee5e6b4b0d3255bfef95601890afd80709", "da39a3ee5e6b4b0d3255bfef95601890afd80709")) @@ -276,7 +279,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { test("sort_array function") { val df = Seq( (Array[Int](2, 1, 3), Array("b", "c", "a")), - (Array[Int](), Array[String]()), + (Array.empty[Int], Array.empty[String]), (null, null) ).toDF("a", "b") checkAnswer( @@ -308,10 +311,14 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { Row(null, null)) ) - val df2 = Seq((Array[Array[Int]](Array(2)), "x")).toDF("a", "b") - assert(intercept[AnalysisException] { - df2.selectExpr("sort_array(a)").collect() - }.getMessage().contains("does not support sorting array of type array")) + val df2 = Seq((Array[Array[Int]](Array(2), Array(1), Array(2, 4), null), "x")).toDF("a", "b") + checkAnswer( + df2.selectExpr("sort_array(a, true)", "sort_array(a, false)"), + Seq( + Row( + Seq[Seq[Int]](null, Seq(1), Seq(2), Seq(2, 4)), + Seq[Seq[Int]](Seq(2, 4), Seq(2), Seq(1), null))) + ) val df3 = Seq(("xxx", "x")).toDF("a", "b") assert(intercept[AnalysisException] { @@ -323,15 +330,16 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { val df = Seq( (Seq[Int](1, 2), "x"), (Seq[Int](), "y"), - (Seq[Int](1, 2, 3), "z") + (Seq[Int](1, 2, 3), "z"), + (null, "empty") ).toDF("a", "b") checkAnswer( df.select(size($"a")), - Seq(Row(2), Row(0), Row(3)) + Seq(Row(2), Row(0), Row(3), Row(-1)) ) checkAnswer( df.selectExpr("size(a)"), - Seq(Row(2), Row(0), Row(3)) + Seq(Row(2), Row(0), Row(3), Row(-1)) ) } @@ -339,15 +347,32 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { val df = Seq( (Map[Int, Int](1 -> 1, 2 -> 2), "x"), (Map[Int, Int](), "y"), - (Map[Int, Int](1 -> 1, 2 -> 2, 3 -> 3), "z") + (Map[Int, Int](1 -> 1, 2 -> 2, 3 -> 3), "z"), + (null, "empty") ).toDF("a", "b") checkAnswer( df.select(size($"a")), - Seq(Row(2), Row(0), Row(3)) + Seq(Row(2), Row(0), Row(3), Row(-1)) ) checkAnswer( df.selectExpr("size(a)"), - Seq(Row(2), Row(0), Row(3)) + Seq(Row(2), Row(0), Row(3), Row(-1)) + ) + } + + test("map_keys/map_values function") { + val df = Seq( + (Map[Int, Int](1 -> 100, 2 -> 200), "x"), + (Map[Int, Int](), "y"), + (Map[Int, Int](1 -> 100, 2 -> 200, 3 -> 300), "z") + ).toDF("a", "b") + checkAnswer( + df.selectExpr("map_keys(a)"), + Seq(Row(Seq(1, 2)), Row(Seq.empty), Row(Seq(1, 2, 3))) + ) + checkAnswer( + df.selectExpr("map_values(a)"), + Seq(Row(Seq(100, 200)), Row(Seq.empty), Row(Seq(100, 200, 300))) ) } @@ -387,4 +412,86 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { Seq(Row(true), Row(true)) ) } + + private def assertValuesDoNotChangeAfterCoalesceOrUnion(v: Column): Unit = { + import DataFrameFunctionsSuite.CodegenFallbackExpr + for ((codegenFallback, wholeStage) <- Seq((true, false), (false, false), (false, true))) { + val c = if (codegenFallback) { + Column(CodegenFallbackExpr(v.expr)) + } else { + v + } + withSQLConf( + (SQLConf.CODEGEN_FALLBACK.key, codegenFallback.toString), + (SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, wholeStage.toString)) { + val df = spark.range(0, 4, 1, 4).withColumn("c", c) + val rows = df.collect() + val rowsAfterCoalesce = df.coalesce(2).collect() + assert(rows === rowsAfterCoalesce, "Values changed after coalesce when " + + s"codegenFallback=$codegenFallback and wholeStage=$wholeStage.") + + val df1 = spark.range(0, 2, 1, 2).withColumn("c", c) + val rows1 = df1.collect() + val df2 = spark.range(2, 4, 1, 2).withColumn("c", c) + val rows2 = df2.collect() + val rowsAfterUnion = df1.union(df2).collect() + assert(rowsAfterUnion === rows1 ++ rows2, "Values changed after union when " + + s"codegenFallback=$codegenFallback and wholeStage=$wholeStage.") + } + } + } + + test("SPARK-14393: values generated by non-deterministic functions shouldn't change after " + + "coalesce or union") { + Seq( + monotonically_increasing_id(), spark_partition_id(), + rand(Random.nextLong()), randn(Random.nextLong()) + ).foreach(assertValuesDoNotChangeAfterCoalesceOrUnion(_)) + } + + test("SPARK-21281 use string types by default if array and map have no argument") { + val ds = spark.range(1) + var expectedSchema = new StructType() + .add("x", ArrayType(StringType, containsNull = false), nullable = false) + assert(ds.select(array().as("x")).schema == expectedSchema) + expectedSchema = new StructType() + .add("x", MapType(StringType, StringType, valueContainsNull = false), nullable = false) + assert(ds.select(map().as("x")).schema == expectedSchema) + } + + test("SPARK-21281 fails if functions have no argument") { + val df = Seq(1).toDF("a") + + val funcsMustHaveAtLeastOneArg = + ("coalesce", (df: DataFrame) => df.select(coalesce())) :: + ("coalesce", (df: DataFrame) => df.selectExpr("coalesce()")) :: + ("named_struct", (df: DataFrame) => df.select(struct())) :: + ("named_struct", (df: DataFrame) => df.selectExpr("named_struct()")) :: + ("hash", (df: DataFrame) => df.select(hash())) :: + ("hash", (df: DataFrame) => df.selectExpr("hash()")) :: Nil + funcsMustHaveAtLeastOneArg.foreach { case (name, func) => + val errMsg = intercept[AnalysisException] { func(df) }.getMessage + assert(errMsg.contains(s"input to function $name requires at least one argument")) + } + + val funcsMustHaveAtLeastTwoArgs = + ("greatest", (df: DataFrame) => df.select(greatest())) :: + ("greatest", (df: DataFrame) => df.selectExpr("greatest()")) :: + ("least", (df: DataFrame) => df.select(least())) :: + ("least", (df: DataFrame) => df.selectExpr("least()")) :: Nil + funcsMustHaveAtLeastTwoArgs.foreach { case (name, func) => + val errMsg = intercept[AnalysisException] { func(df) }.getMessage + assert(errMsg.contains(s"input to function $name requires at least two arguments")) + } + } +} + +object DataFrameFunctionsSuite { + case class CodegenFallbackExpr(child: Expression) extends Expression with CodegenFallback { + override def children: Seq[Expression] = Seq(child) + override def nullable: Boolean = child.nullable + override def dataType: DataType = child.dataType + override lazy val resolved = true + override def eval(input: InternalRow): Any = child.eval(input) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala new file mode 100644 index 000000000000..0dd5bdcba2e4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameHintSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.analysis.AnalysisTest +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.test.SharedSQLContext + +class DataFrameHintSuite extends AnalysisTest with SharedSQLContext { + import testImplicits._ + lazy val df = spark.range(10) + + private def check(df: Dataset[_], expected: LogicalPlan) = { + comparePlans( + df.queryExecution.logical, + expected + ) + } + + test("various hint parameters") { + check( + df.hint("hint1"), + UnresolvedHint("hint1", Seq(), + df.logicalPlan + ) + ) + + check( + df.hint("hint1", 1, "a"), + UnresolvedHint("hint1", Seq(1, "a"), df.logicalPlan) + ) + + check( + df.hint("hint1", 1, $"a"), + UnresolvedHint("hint1", Seq(1, $"a"), + df.logicalPlan + ) + ) + + check( + df.hint("hint1", Seq(1, 2, 3), Seq($"a", $"b", $"c")), + UnresolvedHint("hint1", Seq(Seq(1, 2, 3), Seq($"a", $"b", $"c")), + df.logicalPlan + ) + ) + } +} 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 094efbaeadcd..25e1d93ff092 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 @@ -1,19 +1,19 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 @@ -51,4 +51,15 @@ class DataFrameImplicitsSuite extends QueryTest with SharedSQLContext { sparkContext.parallelize(1 to 10).map(_.toString).toDF("stringCol"), (1 to 10).map(i => Row(i.toString))) } + + test("SPARK-19959: df[java.lang.Long].collect includes null throws NullPointerException") { + checkAnswer(sparkContext.parallelize(Seq[java.lang.Integer](0, null, 2), 1).toDF, + Seq(Row(0), Row(null), Row(2))) + checkAnswer(sparkContext.parallelize(Seq[java.lang.Long](0L, null, 2L), 1).toDF, + Seq(Row(0L), Row(null), Row(2L))) + checkAnswer(sparkContext.parallelize(Seq[java.lang.Float](0.0F, null, 2.0F), 1).toDF, + Seq(Row(0.0F), Row(null), Row(2.0F))) + checkAnswer(sparkContext.parallelize(Seq[java.lang.Double](0.0D, null, 2.0D), 1).toDF, + Seq(Row(0.0D), Row(null), Row(2.0D))) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index 56ad71ea4f48..aef0d7f3e425 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -17,7 +17,9 @@ package org.apache.spark.sql -import org.apache.spark.sql.execution.joins.BroadcastHashJoin +import org.apache.spark.sql.catalyst.plans.{Inner, LeftOuter, RightOuter} +import org.apache.spark.sql.catalyst.plans.logical.Join +import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext @@ -42,17 +44,45 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { Row(1, 2, "1", "2") :: Row(2, 3, "2", "3") :: Row(3, 4, "3", "4") :: Nil) } + test("join - sorted columns not in join's outputSet") { + val df = Seq((1, 2, "1"), (3, 4, "3")).toDF("int", "int2", "str_sort").as('df1) + val df2 = Seq((1, 3, "1"), (5, 6, "5")).toDF("int", "int2", "str").as('df2) + val df3 = Seq((1, 3, "1"), (5, 6, "5")).toDF("int", "int2", "str").as('df3) + + checkAnswer( + df.join(df2, $"df1.int" === $"df2.int", "outer").select($"df1.int", $"df2.int2") + .orderBy('str_sort.asc, 'str.asc), + Row(null, 6) :: Row(1, 3) :: Row(3, null) :: Nil) + + checkAnswer( + df2.join(df3, $"df2.int" === $"df3.int", "inner") + .select($"df2.int", $"df3.int").orderBy($"df2.str".desc), + Row(5, 5) :: Row(1, 1) :: Nil) + } + test("join - join using multiple columns and specifying join type") { - val df = Seq(1, 2, 3).map(i => (i, i + 1, i.toString)).toDF("int", "int2", "str") - val df2 = Seq(1, 2, 3).map(i => (i, i + 1, (i + 1).toString)).toDF("int", "int2", "str") + val df = Seq((1, 2, "1"), (3, 4, "3")).toDF("int", "int2", "str") + val df2 = Seq((1, 3, "1"), (5, 6, "5")).toDF("int", "int2", "str") + + checkAnswer( + df.join(df2, Seq("int", "str"), "inner"), + Row(1, "1", 2, 3) :: Nil) checkAnswer( df.join(df2, Seq("int", "str"), "left"), - Row(1, 2, "1", null) :: Row(2, 3, "2", null) :: Row(3, 4, "3", null) :: Nil) + Row(1, "1", 2, 3) :: Row(3, "3", 4, null) :: Nil) checkAnswer( df.join(df2, Seq("int", "str"), "right"), - Row(null, null, null, 2) :: Row(null, null, null, 3) :: Row(null, null, null, 4) :: Nil) + Row(1, "1", 2, 3) :: Row(5, "5", null, 6) :: Nil) + + checkAnswer( + df.join(df2, Seq("int", "str"), "outer"), + Row(1, "1", 2, 3) :: Row(3, "3", 4, null) :: Row(5, "5", null, 6) :: Nil) + + checkAnswer( + df.join(df2, Seq("int", "str"), "left_semi"), + Row(1, "1", 2) :: Nil) } test("join - join using self join") { @@ -74,6 +104,21 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { .collect().toSeq) } + test("join - cross join") { + val df1 = Seq((1, "1"), (3, "3")).toDF("int", "str") + val df2 = Seq((2, "2"), (4, "4")).toDF("int", "str") + + checkAnswer( + df1.crossJoin(df2), + Row(1, "1", 2, "2") :: Row(1, "1", 4, "4") :: + Row(3, "3", 2, "2") :: Row(3, "3", 4, "4") :: Nil) + + checkAnswer( + df2.crossJoin(df1), + Row(2, "2", 1, "1") :: Row(2, "2", 3, "3") :: + Row(4, "4", 1, "1") :: Row(4, "4", 3, "3") :: Nil) + } + test("join - using aliases after self join") { val df = Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str") checkAnswer( @@ -106,19 +151,127 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { Row(1, 1, 1, 1) :: Row(2, 1, 2, 2) :: Nil) } - test("broadcast join hint") { + test("broadcast join hint using broadcast function") { val df1 = Seq((1, "1"), (2, "2")).toDF("key", "value") val df2 = Seq((1, "1"), (2, "2")).toDF("key", "value") // equijoin - should be converted into broadcast join - val plan1 = df1.join(broadcast(df2), "key").queryExecution.executedPlan - assert(plan1.collect { case p: BroadcastHashJoin => p }.size === 1) + val plan1 = df1.join(broadcast(df2), "key").queryExecution.sparkPlan + assert(plan1.collect { case p: BroadcastHashJoinExec => p }.size === 1) // no join key -- should not be a broadcast join - val plan2 = df1.join(broadcast(df2)).queryExecution.executedPlan - assert(plan2.collect { case p: BroadcastHashJoin => p }.size === 0) + val plan2 = df1.crossJoin(broadcast(df2)).queryExecution.sparkPlan + assert(plan2.collect { case p: BroadcastHashJoinExec => p }.size === 0) // planner should not crash without a join - broadcast(df1).queryExecution.executedPlan + broadcast(df1).queryExecution.sparkPlan + + // SPARK-12275: no physical plan for BroadcastHint in some condition + withTempPath { path => + df1.write.parquet(path.getCanonicalPath) + val pf1 = spark.read.parquet(path.getCanonicalPath) + assert(df1.crossJoin(broadcast(pf1)).count() === 4) + } + } + + test("broadcast join hint using Dataset.hint") { + // make sure a giant join is not broadcastable + val plan1 = + spark.range(10e10.toLong) + .join(spark.range(10e10.toLong), "id") + .queryExecution.executedPlan + assert(plan1.collect { case p: BroadcastHashJoinExec => p }.size == 0) + + // now with a hint it should be broadcasted + val plan2 = + spark.range(10e10.toLong) + .join(spark.range(10e10.toLong).hint("broadcast"), "id") + .queryExecution.executedPlan + assert(plan2.collect { case p: BroadcastHashJoinExec => p }.size == 1) } + + test("join - outer join conversion") { + val df = Seq((1, 2, "1"), (3, 4, "3")).toDF("int", "int2", "str").as("a") + val df2 = Seq((1, 3, "1"), (5, 6, "5")).toDF("int", "int2", "str").as("b") + + // outer -> left + val outerJoin2Left = df.join(df2, $"a.int" === $"b.int", "outer").where($"a.int" === 3) + assert(outerJoin2Left.queryExecution.optimizedPlan.collect { + case j @ Join(_, _, LeftOuter, _) => j }.size === 1) + checkAnswer( + outerJoin2Left, + Row(3, 4, "3", null, null, null) :: Nil) + + // outer -> right + val outerJoin2Right = df.join(df2, $"a.int" === $"b.int", "outer").where($"b.int" === 5) + assert(outerJoin2Right.queryExecution.optimizedPlan.collect { + case j @ Join(_, _, RightOuter, _) => j }.size === 1) + checkAnswer( + outerJoin2Right, + Row(null, null, null, 5, 6, "5") :: Nil) + + // outer -> inner + val outerJoin2Inner = df.join(df2, $"a.int" === $"b.int", "outer"). + where($"a.int" === 1 && $"b.int2" === 3) + assert(outerJoin2Inner.queryExecution.optimizedPlan.collect { + case j @ Join(_, _, Inner, _) => j }.size === 1) + checkAnswer( + outerJoin2Inner, + Row(1, 2, "1", 1, 3, "1") :: Nil) + + // right -> inner + val rightJoin2Inner = df.join(df2, $"a.int" === $"b.int", "right").where($"a.int" === 1) + assert(rightJoin2Inner.queryExecution.optimizedPlan.collect { + case j @ Join(_, _, Inner, _) => j }.size === 1) + checkAnswer( + rightJoin2Inner, + Row(1, 2, "1", 1, 3, "1") :: Nil) + + // left -> inner + val leftJoin2Inner = df.join(df2, $"a.int" === $"b.int", "left").where($"b.int2" === 3) + assert(leftJoin2Inner.queryExecution.optimizedPlan.collect { + case j @ Join(_, _, Inner, _) => j }.size === 1) + checkAnswer( + leftJoin2Inner, + Row(1, 2, "1", 1, 3, "1") :: Nil) + } + + test("process outer join results using the non-nullable columns in the join input") { + // Filter data using a non-nullable column from a right table + val df1 = Seq((0, 0), (1, 0), (2, 0), (3, 0), (4, 0)).toDF("id", "count") + val df2 = Seq(Tuple1(0), Tuple1(1)).toDF("id").groupBy("id").count + checkAnswer( + df1.join(df2, df1("id") === df2("id"), "left_outer").filter(df2("count").isNull), + Row(2, 0, null, null) :: + Row(3, 0, null, null) :: + Row(4, 0, null, null) :: Nil + ) + + // Coalesce data using non-nullable columns in input tables + val df3 = Seq((1, 1)).toDF("a", "b") + val df4 = Seq((2, 2)).toDF("a", "b") + checkAnswer( + df3.join(df4, df3("a") === df4("a"), "outer") + .select(coalesce(df3("a"), df3("b")), coalesce(df4("a"), df4("b"))), + Row(1, null) :: Row(null, 2) :: Nil + ) + } + + test("SPARK-16991: Full outer join followed by inner join produces wrong results") { + val a = Seq((1, 2), (2, 3)).toDF("a", "b") + val b = Seq((2, 5), (3, 4)).toDF("a", "c") + val c = Seq((3, 1)).toDF("a", "d") + val ab = a.join(b, Seq("a"), "fullouter") + checkAnswer(ab.join(c, "a"), Row(3, null, 4, 1) :: Nil) + } + + test("SPARK-17685: WholeStageCodegenExec throws IndexOutOfBoundsException") { + val df = Seq((1, 1, "1"), (2, 2, "3")).toDF("int", "int2", "str") + val df2 = Seq((1, 1, "1"), (2, 3, "5")).toDF("int", "int2", "str") + val limit = 1310721 + val innerJoin = df.limit(limit).join(df2.limit(limit), Seq("int", "int2"), "inner") + .agg(count($"int")) + checkAnswer(innerJoin, Row(1) :: Nil) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index e34875471f09..e6983b6be555 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext - class DataFrameNaFunctionsSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -57,7 +57,7 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSQLContext { rows(0)) // dropna on an a dataframe with no column should return an empty data frame. - val empty = input.sqlContext.emptyDataFrame.select() + val empty = input.sparkSession.emptyDataFrame.select() assert(empty.na.drop().count() === 0L) // Make sure the columns are properly named. @@ -104,63 +104,131 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSQLContext { test("fill") { val input = createDF() - val fillNumeric = input.na.fill(50.6) - checkAnswer( - fillNumeric, - Row("Bob", 16, 176.5) :: - Row("Alice", 50, 164.3) :: - Row("David", 60, 50.6) :: - Row("Nina", 25, 50.6) :: - Row("Amy", 50, 50.6) :: - Row(null, 50, 50.6) :: Nil) - - // Make sure the columns are properly named. - assert(fillNumeric.columns.toSeq === input.columns.toSeq) - - // string - checkAnswer( - input.na.fill("unknown").select("name"), - Row("Bob") :: Row("Alice") :: Row("David") :: - Row("Nina") :: Row("Amy") :: Row("unknown") :: Nil) - assert(input.na.fill("unknown").columns.toSeq === input.columns.toSeq) - - // fill double with subset columns - checkAnswer( - input.na.fill(50.6, "age" :: Nil).select("name", "age"), - Row("Bob", 16) :: - Row("Alice", 50) :: - Row("David", 60) :: - Row("Nina", 25) :: - Row("Amy", 50) :: - Row(null, 50) :: Nil) - - // fill string with subset columns - checkAnswer( - Seq[(String, String)]((null, null)).toDF("col1", "col2").na.fill("test", "col1" :: Nil), - Row("test", null)) + val boolInput = Seq[(String, java.lang.Boolean)]( + ("Bob", false), + ("Alice", null), + ("Mallory", true), + (null, null) + ).toDF("name", "spy") + + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val fillNumeric = input.na.fill(50.6) + checkAnswer( + fillNumeric, + Row("Bob", 16, 176.5) :: + Row("Alice", 50, 164.3) :: + Row("David", 60, 50.6) :: + Row("Nina", 25, 50.6) :: + Row("Amy", 50, 50.6) :: + Row(null, 50, 50.6) :: Nil) + + // Make sure the columns are properly named. + assert(fillNumeric.columns.toSeq === input.columns.toSeq) + + // string + checkAnswer( + input.na.fill("unknown").select("name"), + Row("Bob") :: Row("Alice") :: Row("David") :: + Row("Nina") :: Row("Amy") :: Row("unknown") :: Nil) + assert(input.na.fill("unknown").columns.toSeq === input.columns.toSeq) + + // boolean + checkAnswer( + boolInput.na.fill(true).select("spy"), + Row(false) :: Row(true) :: Row(true) :: Row(true) :: Nil) + assert(boolInput.na.fill(true).columns.toSeq === boolInput.columns.toSeq) + + // fill double with subset columns + checkAnswer( + input.na.fill(50.6, "age" :: Nil).select("name", "age"), + Row("Bob", 16) :: + Row("Alice", 50) :: + Row("David", 60) :: + Row("Nina", 25) :: + Row("Amy", 50) :: + Row(null, 50) :: Nil) + + // fill boolean with subset columns + checkAnswer( + boolInput.na.fill(true, "spy" :: Nil).select("name", "spy"), + Row("Bob", false) :: + Row("Alice", true) :: + Row("Mallory", true) :: + Row(null, true) :: Nil) + + // fill string with subset columns + checkAnswer( + Seq[(String, String)]((null, null)).toDF("col1", "col2").na.fill("test", "col1" :: Nil), + Row("test", null)) + + checkAnswer( + Seq[(Long, Long)]((1, 2), (-1, -2), (9123146099426677101L, 9123146560113991650L)) + .toDF("a", "b").na.fill(0), + Row(1, 2) :: Row(-1, -2) :: Row(9123146099426677101L, 9123146560113991650L) :: Nil + ) + + checkAnswer( + Seq[(java.lang.Long, java.lang.Double)]((null, 3.14), (9123146099426677101L, null), + (9123146560113991650L, 1.6), (null, null)).toDF("a", "b").na.fill(0.2), + Row(0, 3.14) :: Row(9123146099426677101L, 0.2) :: Row(9123146560113991650L, 1.6) + :: Row(0, 0.2) :: Nil + ) + + checkAnswer( + Seq[(java.lang.Long, java.lang.Float)]((null, 3.14f), (9123146099426677101L, null), + (9123146560113991650L, 1.6f), (null, null)).toDF("a", "b").na.fill(0.2), + Row(0, 3.14f) :: Row(9123146099426677101L, 0.2f) :: Row(9123146560113991650L, 1.6f) + :: Row(0, 0.2f) :: Nil + ) + + checkAnswer( + Seq[(java.lang.Long, java.lang.Double)]((null, 1.23), (3L, null), (4L, 3.45)) + .toDF("a", "b").na.fill(2.34), + Row(2, 1.23) :: Row(3, 2.34) :: Row(4, 3.45) :: Nil + ) + + checkAnswer( + Seq[(java.lang.Long, java.lang.Double)]((null, 1.23), (3L, null), (4L, 3.45)) + .toDF("a", "b").na.fill(5), + Row(5, 1.23) :: Row(3, 5.0) :: Row(4, 3.45) :: Nil + ) + } } test("fill with map") { - val df = Seq[(String, String, java.lang.Long, java.lang.Double, java.lang.Boolean)]( - (null, null, null, null, null)).toDF("a", "b", "c", "d", "e") - checkAnswer( - df.na.fill(Map( - "a" -> "test", - "c" -> 1, - "d" -> 2.2, - "e" -> false - )), - Row("test", null, 1, 2.2, false)) - - // Test Java version - checkAnswer( - df.na.fill(Map( - "a" -> "test", - "c" -> 1, - "d" -> 2.2, - "e" -> false - ).asJava), - Row("test", null, 1, 2.2, false)) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val df = Seq[(String, String, java.lang.Integer, java.lang.Long, + java.lang.Float, java.lang.Double, java.lang.Boolean)]( + (null, null, null, null, null, null, null)) + .toDF("stringFieldA", "stringFieldB", "integerField", "longField", + "floatField", "doubleField", "booleanField") + + val fillMap = Map( + "stringFieldA" -> "test", + "integerField" -> 1, + "longField" -> 2L, + "floatField" -> 3.3f, + "doubleField" -> 4.4d, + "booleanField" -> false) + + val expectedRow = Row("test", null, 1, 2L, 3.3f, 4.4d, false) + + + checkAnswer(df.na.fill(fillMap), expectedRow) + checkAnswer(df.na.fill(fillMap.asJava), expectedRow) // Test Java version + + // Ensure replacement values are cast to the column data type. + checkAnswer(df.na.fill(Map( + "integerField" -> 1d, + "longField" -> 2d, + "floatField" -> 3d, + "doubleField" -> 4d)), + Row(null, null, 1, 2L, 3f, 4d, null)) + + // Ensure column types do not change. Columns that have null values replaced + // will no longer be flagged as nullable, so do not compare schemas directly. + assert(df.na.fill(fillMap).schema.fields.map(_.dataType) === df.schema.fields.map(_.dataType)) + } } test("replace") { @@ -194,4 +262,47 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSQLContext { assert(out1(4) === Row("Amy", null, null)) assert(out1(5) === Row(null, null, null)) } + + test("replace with null") { + val input = Seq[(String, java.lang.Double, java.lang.Boolean)]( + ("Bob", 176.5, true), + ("Alice", 164.3, false), + ("David", null, true) + ).toDF("name", "height", "married") + + // Replace String with String and null + checkAnswer( + input.na.replace("name", Map( + "Bob" -> "Bravo", + "Alice" -> null + )), + Row("Bravo", 176.5, true) :: + Row(null, 164.3, false) :: + Row("David", null, true) :: Nil) + + // Replace Double with null + checkAnswer( + input.na.replace("height", Map[Any, Any]( + 164.3 -> null + )), + Row("Bob", 176.5, true) :: + Row("Alice", null, false) :: + Row("David", null, true) :: Nil) + + // Replace Boolean with null + checkAnswer( + input.na.replace("*", Map[Any, Any]( + false -> null + )), + Row("Bob", 176.5, true) :: + Row("Alice", 164.3, null) :: + Row("David", null, true) :: Nil) + + // Replace String with null and then drop rows containing null + checkAnswer( + input.na.replace("name", Map( + "Bob" -> null + )).na.drop("name" :: Nil).select("name"), + Row("Alice") :: Row("David") :: Nil) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala new file mode 100644 index 000000000000..6ca9ee57e8f4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFramePivotSuite.scala @@ -0,0 +1,249 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.expressions.aggregate.PivotFirst +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ + +class DataFramePivotSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("pivot courses") { + checkAnswer( + courseSales.groupBy("year").pivot("course", Seq("dotNET", "Java")) + .agg(sum($"earnings")), + Row(2012, 15000.0, 20000.0) :: Row(2013, 48000.0, 30000.0) :: Nil + ) + } + + test("pivot year") { + checkAnswer( + courseSales.groupBy("course").pivot("year", Seq(2012, 2013)).agg(sum($"earnings")), + Row("dotNET", 15000.0, 48000.0) :: Row("Java", 20000.0, 30000.0) :: Nil + ) + } + + test("pivot courses with multiple aggregations") { + checkAnswer( + courseSales.groupBy($"year") + .pivot("course", Seq("dotNET", "Java")) + .agg(sum($"earnings"), avg($"earnings")), + Row(2012, 15000.0, 7500.0, 20000.0, 20000.0) :: + Row(2013, 48000.0, 48000.0, 30000.0, 30000.0) :: Nil + ) + } + + test("pivot year with string values (cast)") { + checkAnswer( + courseSales.groupBy("course").pivot("year", Seq("2012", "2013")).sum("earnings"), + Row("dotNET", 15000.0, 48000.0) :: Row("Java", 20000.0, 30000.0) :: Nil + ) + } + + test("pivot year with int values") { + checkAnswer( + courseSales.groupBy("course").pivot("year", Seq(2012, 2013)).sum("earnings"), + Row("dotNET", 15000.0, 48000.0) :: Row("Java", 20000.0, 30000.0) :: Nil + ) + } + + test("pivot courses with no values") { + // Note Java comes before dotNet in sorted order + checkAnswer( + courseSales.groupBy("year").pivot("course").agg(sum($"earnings")), + Row(2012, 20000.0, 15000.0) :: Row(2013, 30000.0, 48000.0) :: Nil + ) + } + + test("pivot year with no values") { + checkAnswer( + courseSales.groupBy("course").pivot("year").agg(sum($"earnings")), + Row("dotNET", 15000.0, 48000.0) :: Row("Java", 20000.0, 30000.0) :: Nil + ) + } + + test("pivot max values enforced") { + spark.conf.set(SQLConf.DATAFRAME_PIVOT_MAX_VALUES.key, 1) + intercept[AnalysisException]( + courseSales.groupBy("year").pivot("course") + ) + spark.conf.set(SQLConf.DATAFRAME_PIVOT_MAX_VALUES.key, + SQLConf.DATAFRAME_PIVOT_MAX_VALUES.defaultValue.get) + } + + test("pivot with UnresolvedFunction") { + checkAnswer( + courseSales.groupBy("year").pivot("course", Seq("dotNET", "Java")) + .agg("earnings" -> "sum"), + Row(2012, 15000.0, 20000.0) :: Row(2013, 48000.0, 30000.0) :: Nil + ) + } + + // Tests for optimized pivot (with PivotFirst) below + + test("optimized pivot planned") { + val df = courseSales.groupBy("year") + // pivot with extra columns to trigger optimization + .pivot("course", Seq("dotNET", "Java") ++ (1 to 10).map(_.toString)) + .agg(sum($"earnings")) + val queryExecution = spark.sessionState.executePlan(df.queryExecution.logical) + assert(queryExecution.simpleString.contains("pivotfirst")) + } + + + test("optimized pivot courses with literals") { + checkAnswer( + courseSales.groupBy("year") + // pivot with extra columns to trigger optimization + .pivot("course", Seq("dotNET", "Java") ++ (1 to 10).map(_.toString)) + .agg(sum($"earnings")) + .select("year", "dotNET", "Java"), + Row(2012, 15000.0, 20000.0) :: Row(2013, 48000.0, 30000.0) :: Nil + ) + } + + test("optimized pivot year with literals") { + checkAnswer( + courseSales.groupBy($"course") + // pivot with extra columns to trigger optimization + .pivot("year", Seq(2012, 2013) ++ (1 to 10)) + .agg(sum($"earnings")) + .select("course", "2012", "2013"), + Row("dotNET", 15000.0, 48000.0) :: Row("Java", 20000.0, 30000.0) :: Nil + ) + } + + test("optimized pivot year with string values (cast)") { + checkAnswer( + courseSales.groupBy("course") + // pivot with extra columns to trigger optimization + .pivot("year", Seq("2012", "2013") ++ (1 to 10).map(_.toString)) + .sum("earnings") + .select("course", "2012", "2013"), + Row("dotNET", 15000.0, 48000.0) :: Row("Java", 20000.0, 30000.0) :: Nil + ) + } + + test("optimized pivot DecimalType") { + val df = courseSales.select($"course", $"year", $"earnings".cast(DecimalType(10, 2))) + .groupBy("year") + // pivot with extra columns to trigger optimization + .pivot("course", Seq("dotNET", "Java") ++ (1 to 10).map(_.toString)) + .agg(sum($"earnings")) + .select("year", "dotNET", "Java") + + assertResult(IntegerType)(df.schema("year").dataType) + assertResult(DecimalType(20, 2))(df.schema("Java").dataType) + assertResult(DecimalType(20, 2))(df.schema("dotNET").dataType) + + checkAnswer(df, Row(2012, BigDecimal(1500000, 2), BigDecimal(2000000, 2)) :: + Row(2013, BigDecimal(4800000, 2), BigDecimal(3000000, 2)) :: Nil) + } + + test("PivotFirst supported datatypes") { + val supportedDataTypes: Seq[DataType] = DoubleType :: IntegerType :: LongType :: FloatType :: + BooleanType :: ShortType :: ByteType :: Nil + for (datatype <- supportedDataTypes) { + assertResult(true)(PivotFirst.supportsDataType(datatype)) + } + assertResult(true)(PivotFirst.supportsDataType(DecimalType(10, 1))) + assertResult(false)(PivotFirst.supportsDataType(null)) + assertResult(false)(PivotFirst.supportsDataType(ArrayType(IntegerType))) + } + + test("optimized pivot with multiple aggregations") { + checkAnswer( + courseSales.groupBy($"year") + // pivot with extra columns to trigger optimization + .pivot("course", Seq("dotNET", "Java") ++ (1 to 10).map(_.toString)) + .agg(sum($"earnings"), avg($"earnings")), + Row(Seq(2012, 15000.0, 7500.0, 20000.0, 20000.0) ++ Seq.fill(20)(null): _*) :: + Row(Seq(2013, 48000.0, 48000.0, 30000.0, 30000.0) ++ Seq.fill(20)(null): _*) :: Nil + ) + } + + test("pivot with datatype not supported by PivotFirst") { + checkAnswer( + complexData.groupBy().pivot("b", Seq(true, false)).agg(max("a")), + Row(Seq(1, 1, 1), Seq(2, 2, 2)) :: Nil + ) + } + + test("pivot with datatype not supported by PivotFirst 2") { + checkAnswer( + courseSales.withColumn("e", expr("array(earnings, 7.0d)")) + .groupBy("year") + .pivot("course", Seq("dotNET", "Java")) + .agg(min($"e")), + Row(2012, Seq(5000.0, 7.0), Seq(20000.0, 7.0)) :: + Row(2013, Seq(48000.0, 7.0), Seq(30000.0, 7.0)) :: Nil + ) + } + + test("pivot preserves aliases if given") { + assertResult( + Array("year", "dotNET_foo", "dotNET_avg(earnings)", "Java_foo", "Java_avg(earnings)") + )( + courseSales.groupBy($"year") + .pivot("course", Seq("dotNET", "Java")) + .agg(sum($"earnings").as("foo"), avg($"earnings")).columns + ) + } + + test("pivot with column definition in groupby") { + checkAnswer( + courseSales.groupBy(substring(col("course"), 0, 1).as("foo")) + .pivot("year", Seq(2012, 2013)) + .sum("earnings"), + Row("d", 15000.0, 48000.0) :: Row("J", 20000.0, 30000.0) :: Nil + ) + } + + test("pivot with null should not throw NPE") { + checkAnswer( + Seq(Tuple1(None), Tuple1(Some(1))).toDF("a").groupBy($"a").pivot("a").count(), + Row(null, 1, null) :: Row(1, null, 1) :: Nil) + } + + test("pivot with null and aggregate type not supported by PivotFirst returns correct result") { + checkAnswer( + Seq(Tuple1(None), Tuple1(Some(1))).toDF("a") + .withColumn("b", expr("array(a, 7)")) + .groupBy($"a").pivot("a").agg(min($"b")), + Row(null, Seq(null, 7), null) :: Row(1, null, Seq(1, 7)) :: Nil) + } + + test("pivot with timestamp and count should not print internal representation") { + val ts = "2012-12-31 16:00:10.011" + val tsWithZone = "2013-01-01 00:00:10.011" + + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") { + val df = Seq(java.sql.Timestamp.valueOf(ts)).toDF("a").groupBy("a").pivot("a").count() + val expected = StructType( + StructField("a", TimestampType) :: + StructField(tsWithZone, LongType) :: Nil) + assert(df.schema == expected) + // String representation of timestamp with timezone should take the time difference + // into account. + checkAnswer(df.select($"a".cast(StringType)), Row(tsWithZone)) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala new file mode 100644 index 000000000000..45afbd29d190 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import scala.concurrent.duration._ +import scala.math.abs +import scala.util.Random + +import org.scalatest.concurrent.Eventually + +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + + +class DataFrameRangeSuite extends QueryTest with SharedSQLContext with Eventually { + import testImplicits._ + + test("SPARK-7150 range api") { + // numSlice is greater than length + val res1 = spark.range(0, 10, 1, 15).select("id") + assert(res1.count == 10) + assert(res1.agg(sum("id")).as("sumid").collect() === Seq(Row(45))) + + val res2 = spark.range(3, 15, 3, 2).select("id") + assert(res2.count == 4) + assert(res2.agg(sum("id")).as("sumid").collect() === Seq(Row(30))) + + val res3 = spark.range(1, -2).select("id") + assert(res3.count == 0) + + // start is positive, end is negative, step is negative + val res4 = spark.range(1, -2, -2, 6).select("id") + assert(res4.count == 2) + assert(res4.agg(sum("id")).as("sumid").collect() === Seq(Row(0))) + + // start, end, step are negative + val res5 = spark.range(-3, -8, -2, 1).select("id") + assert(res5.count == 3) + assert(res5.agg(sum("id")).as("sumid").collect() === Seq(Row(-15))) + + // start, end are negative, step is positive + val res6 = spark.range(-8, -4, 2, 1).select("id") + assert(res6.count == 2) + assert(res6.agg(sum("id")).as("sumid").collect() === Seq(Row(-14))) + + val res7 = spark.range(-10, -9, -20, 1).select("id") + assert(res7.count == 0) + + val res8 = spark.range(Long.MinValue, Long.MaxValue, Long.MaxValue, 100).select("id") + assert(res8.count == 3) + assert(res8.agg(sum("id")).as("sumid").collect() === Seq(Row(-3))) + + val res9 = spark.range(Long.MaxValue, Long.MinValue, Long.MinValue, 100).select("id") + assert(res9.count == 2) + assert(res9.agg(sum("id")).as("sumid").collect() === Seq(Row(Long.MaxValue - 1))) + + // only end provided as argument + val res10 = spark.range(10).select("id") + assert(res10.count == 10) + assert(res10.agg(sum("id")).as("sumid").collect() === Seq(Row(45))) + + val res11 = spark.range(-1).select("id") + assert(res11.count == 0) + + // using the default slice number + val res12 = spark.range(3, 15, 3).select("id") + assert(res12.count == 4) + assert(res12.agg(sum("id")).as("sumid").collect() === Seq(Row(30))) + + // difference between range start and end does not fit in a 64-bit integer + val n = 9L * 1000 * 1000 * 1000 * 1000 * 1000 * 1000 + val res13 = spark.range(-n, n, n / 9).select("id") + assert(res13.count == 18) + + // range with non aggregation operation + val res14 = spark.range(0, 100, 2).toDF.filter("50 <= id") + val len14 = res14.collect.length + assert(len14 == 25) + + val res15 = spark.range(100, -100, -2).toDF.filter("id <= 0") + val len15 = res15.collect.length + assert(len15 == 50) + + val res16 = spark.range(-1500, 1500, 3).toDF.filter("0 <= id") + val len16 = res16.collect.length + assert(len16 == 500) + + val res17 = spark.range(10, 0, -1, 1).toDF.sortWithinPartitions("id") + assert(res17.collect === (1 to 10).map(i => Row(i)).toArray) + } + + test("Range with randomized parameters") { + val MAX_NUM_STEPS = 10L * 1000 + + val seed = System.currentTimeMillis() + val random = new Random(seed) + + def randomBound(): Long = { + val n = if (random.nextBoolean()) { + random.nextLong() % (Long.MaxValue / (100 * MAX_NUM_STEPS)) + } else { + random.nextLong() / 2 + } + if (random.nextBoolean()) n else -n + } + + for (l <- 1 to 10) { + val start = randomBound() + val end = randomBound() + val numSteps = (abs(random.nextLong()) % MAX_NUM_STEPS) + 1 + val stepAbs = (abs(end - start) / numSteps) + 1 + val step = if (start < end) stepAbs else -stepAbs + val partitions = random.nextInt(20) + 1 + + val expCount = (start until end by step).size + val expSum = (start until end by step).sum + + for (codegen <- List(false, true)) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> codegen.toString()) { + val res = spark.range(start, end, step, partitions).toDF("id"). + agg(count("id"), sum("id")).collect() + + withClue(s"seed = $seed start = $start end = $end step = $step partitions = " + + s"$partitions codegen = $codegen") { + assert(!res.isEmpty) + assert(res.head.getLong(0) == expCount) + if (expCount > 0) { + assert(res.head.getLong(1) == expSum) + } + } + } + } + } + } + + test("Cancelling stage in a query with Range.") { + val listener = new SparkListener { + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + eventually(timeout(10.seconds)) { + assert(DataFrameRangeSuite.stageToKill > 0) + } + sparkContext.cancelStage(DataFrameRangeSuite.stageToKill) + } + } + + sparkContext.addSparkListener(listener) + for (codegen <- Seq(true, false)) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> codegen.toString()) { + DataFrameRangeSuite.stageToKill = -1 + val ex = intercept[SparkException] { + spark.range(1000000000L).map { x => + DataFrameRangeSuite.stageToKill = TaskContext.get().stageId() + x + }.toDF("id").agg(sum("id")).collect() + } + ex.getCause() match { + case null => + assert(ex.getMessage().contains("cancelled")) + case cause: SparkException => + assert(cause.getMessage().contains("cancelled")) + case cause: Throwable => + fail("Expected the cause to be SparkException, got " + cause.toString() + " instead.") + } + } + eventually(timeout(20.seconds)) { + assert(sparkContext.statusTracker.getExecutorInfos.map(_.numRunningTasks()).sum == 0) + } + } + } + + test("SPARK-20430 Initialize Range parameters in a driver side") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + checkAnswer(sql("SELECT * FROM range(3)"), Row(0) :: Row(1) :: Row(2) :: Nil) + } + } + + test("SPARK-21041 SparkSession.range()'s behavior is inconsistent with SparkContext.range()") { + val start = java.lang.Long.MAX_VALUE - 3 + val end = java.lang.Long.MIN_VALUE + 2 + Seq("false", "true").foreach { value => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> value) { + assert(spark.range(start, end, 1).collect.length == 0) + assert(spark.range(start, start, 1).collect.length == 0) + } + } + } +} + +object DataFrameRangeSuite { + @volatile var stageToKill = -1 +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index 6524abcf5e97..247c30e2ee65 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -19,8 +19,14 @@ package org.apache.spark.sql import java.util.Random +import org.scalatest.Matchers._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.stat.StatFunctions import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{DoubleType, StructField, StructType} class DataFrameStatSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -41,7 +47,7 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { val data = sparkContext.parallelize(1 to n, 2).toDF("id") checkAnswer( data.sample(withReplacement = false, 0.05, seed = 13), - Seq(16, 23, 88, 100).map(Row(_)) + Seq(3, 17, 27, 58, 62).map(Row(_)) ) } @@ -52,7 +58,7 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { val splits = data.randomSplit(Array[Double](1, 2, 3), seed) assert(splits.length == 3, "wrong number of splits") - assert(splits.reduce((a, b) => a.unionAll(b)).sort("id").collect().toList == + assert(splits.reduce((a, b) => a.union(b)).sort("id").collect().toList == data.collect().toList, "incomplete or wrong split") val s = splits.map(_.count()) @@ -62,6 +68,41 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { } } + test("randomSplit on reordered partitions") { + + def testNonOverlappingSplits(data: DataFrame): Unit = { + val splits = data.randomSplit(Array[Double](2, 3), seed = 1) + assert(splits.length == 2, "wrong number of splits") + + // Verify that the splits span the entire dataset + assert(splits.flatMap(_.collect()).toSet == data.collect().toSet) + + // Verify that the splits don't overlap + assert(splits(0).collect().toSeq.intersect(splits(1).collect().toSeq).isEmpty) + + // Verify that the results are deterministic across multiple runs + val firstRun = splits.toSeq.map(_.collect().toSeq) + val secondRun = data.randomSplit(Array[Double](2, 3), seed = 1).toSeq.map(_.collect().toSeq) + assert(firstRun == secondRun) + } + + // This test ensures that randomSplit does not create overlapping splits even when the + // underlying dataframe (such as the one below) doesn't guarantee a deterministic ordering of + // rows in each partition. + val dataWithInts = sparkContext.parallelize(1 to 600, 2) + .mapPartitions(scala.util.Random.shuffle(_)).toDF("int") + val dataWithMaps = sparkContext.parallelize(1 to 600, 2) + .map(i => (i, Map(i -> i.toString))) + .mapPartitions(scala.util.Random.shuffle(_)).toDF("int", "map") + val dataWithArrayOfMaps = sparkContext.parallelize(1 to 600, 2) + .map(i => (i, Array(Map(i -> i.toString)))) + .mapPartitions(scala.util.Random.shuffle(_)).toDF("int", "arrayOfMaps") + + testNonOverlappingSplits(dataWithInts) + testNonOverlappingSplits(dataWithMaps) + testNonOverlappingSplits(dataWithArrayOfMaps) + } + test("pearson correlation") { val df = Seq.tabulate(10)(i => (i, 2 * i, i * -1.0)).toDF("a", "b", "c") val corr1 = df.stat.corr("a", "b", "pearson") @@ -98,53 +139,178 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { assert(math.abs(decimalRes) < 1e-12) } + test("approximate quantile") { + val n = 1000 + val df = Seq.tabulate(n)(i => (i, 2.0 * i)).toDF("singles", "doubles") + + val q1 = 0.5 + val q2 = 0.8 + val epsilons = List(0.1, 0.05, 0.001) + + for (epsilon <- epsilons) { + val Array(single1) = df.stat.approxQuantile("singles", Array(q1), epsilon) + val Array(double2) = df.stat.approxQuantile("doubles", Array(q2), epsilon) + // Also make sure there is no regression by computing multiple quantiles at once. + val Array(d1, d2) = df.stat.approxQuantile("doubles", Array(q1, q2), epsilon) + val Array(s1, s2) = df.stat.approxQuantile("singles", Array(q1, q2), epsilon) + + val error_single = 2 * 1000 * epsilon + val error_double = 2 * 2000 * epsilon + + assert(math.abs(single1 - q1 * n) < error_single) + assert(math.abs(double2 - 2 * q2 * n) < error_double) + assert(math.abs(s1 - q1 * n) < error_single) + assert(math.abs(s2 - q2 * n) < error_single) + assert(math.abs(d1 - 2 * q1 * n) < error_double) + assert(math.abs(d2 - 2 * q2 * n) < error_double) + + // Multiple columns + val Array(Array(ms1, ms2), Array(md1, md2)) = + df.stat.approxQuantile(Array("singles", "doubles"), Array(q1, q2), epsilon) + + assert(math.abs(ms1 - q1 * n) < error_single) + assert(math.abs(ms2 - q2 * n) < error_single) + assert(math.abs(md1 - 2 * q1 * n) < error_double) + assert(math.abs(md2 - 2 * q2 * n) < error_double) + } + + // quantile should be in the range [0.0, 1.0] + val e = intercept[IllegalArgumentException] { + df.stat.approxQuantile(Array("singles", "doubles"), Array(q1, q2, -0.1), epsilons.head) + } + assert(e.getMessage.contains("quantile should be in the range [0.0, 1.0]")) + + // relativeError should be non-negative + val e2 = intercept[IllegalArgumentException] { + df.stat.approxQuantile(Array("singles", "doubles"), Array(q1, q2), -1.0) + } + assert(e2.getMessage.contains("Relative Error must be non-negative")) + } + + test("approximate quantile 2: test relativeError greater than 1 return the same result as 1") { + val n = 1000 + val df = Seq.tabulate(n)(i => (i, 2.0 * i)).toDF("singles", "doubles") + + val q1 = 0.5 + val q2 = 0.8 + val epsilons = List(2.0, 5.0, 100.0) + + val Array(single1_1) = df.stat.approxQuantile("singles", Array(q1), 1.0) + val Array(s1_1, s2_1) = df.stat.approxQuantile("singles", Array(q1, q2), 1.0) + val Array(Array(ms1_1, ms2_1), Array(md1_1, md2_1)) = + df.stat.approxQuantile(Array("singles", "doubles"), Array(q1, q2), 1.0) + + for (epsilon <- epsilons) { + val Array(single1) = df.stat.approxQuantile("singles", Array(q1), epsilon) + val Array(s1, s2) = df.stat.approxQuantile("singles", Array(q1, q2), epsilon) + val Array(Array(ms1, ms2), Array(md1, md2)) = + df.stat.approxQuantile(Array("singles", "doubles"), Array(q1, q2), epsilon) + assert(single1_1 === single1) + assert(s1_1 === s1) + assert(s2_1 === s2) + assert(ms1_1 === ms1) + assert(ms2_1 === ms2) + assert(md1_1 === md1) + assert(md2_1 === md2) + } + } + + test("approximate quantile 3: test on NaN and null values") { + val q1 = 0.5 + val q2 = 0.8 + val epsilon = 0.1 + val rows = spark.sparkContext.parallelize(Seq(Row(Double.NaN, 1.0, Double.NaN), + Row(1.0, -1.0, null), Row(-1.0, Double.NaN, null), Row(Double.NaN, Double.NaN, null), + Row(null, null, Double.NaN), Row(null, 1.0, null), Row(-1.0, null, Double.NaN), + Row(Double.NaN, null, null))) + val schema = StructType(Seq(StructField("input1", DoubleType, nullable = true), + StructField("input2", DoubleType, nullable = true), + StructField("input3", DoubleType, nullable = true))) + val dfNaN = spark.createDataFrame(rows, schema) + + val resNaN1 = dfNaN.stat.approxQuantile("input1", Array(q1, q2), epsilon) + assert(resNaN1.count(_.isNaN) === 0) + + val resNaN2 = dfNaN.stat.approxQuantile("input2", Array(q1, q2), epsilon) + assert(resNaN2.count(_.isNaN) === 0) + + val resNaN3 = dfNaN.stat.approxQuantile("input3", Array(q1, q2), epsilon) + assert(resNaN3.isEmpty) + + val resNaNAll = dfNaN.stat.approxQuantile(Array("input1", "input2", "input3"), + Array(q1, q2), epsilon) + assert(resNaNAll.flatten.count(_.isNaN) === 0) + + assert(resNaN1(0) === resNaNAll(0)(0)) + assert(resNaN1(1) === resNaNAll(0)(1)) + assert(resNaN2(0) === resNaNAll(1)(0)) + assert(resNaN2(1) === resNaNAll(1)(1)) + + // return empty array for columns only containing null or NaN values + assert(resNaNAll(2).isEmpty) + + // return empty array if the dataset is empty + val res1 = dfNaN.selectExpr("*").limit(0) + .stat.approxQuantile("input1", Array(q1, q2), epsilon) + assert(res1.isEmpty) + + val res2 = dfNaN.selectExpr("*").limit(0) + .stat.approxQuantile(Array("input1", "input2"), Array(q1, q2), epsilon) + assert(res2(0).isEmpty) + assert(res2(1).isEmpty) + } + test("crosstab") { - val rng = new Random() - val data = Seq.tabulate(25)(i => (rng.nextInt(5), rng.nextInt(10))) - val df = data.toDF("a", "b") - val crosstab = df.stat.crosstab("a", "b") - val columnNames = crosstab.schema.fieldNames - assert(columnNames(0) === "a_b") - // reduce by key - val expected = data.map(t => (t, 1)).groupBy(_._1).mapValues(_.length) - val rows = crosstab.collect() - rows.foreach { row => - val i = row.getString(0).toInt - for (col <- 1 until columnNames.length) { - val j = columnNames(col).toInt - assert(row.getLong(col) === expected.getOrElse((i, j), 0).toLong) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val rng = new Random() + val data = Seq.tabulate(25)(i => (rng.nextInt(5), rng.nextInt(10))) + val df = data.toDF("a", "b") + val crosstab = df.stat.crosstab("a", "b") + val columnNames = crosstab.schema.fieldNames + assert(columnNames(0) === "a_b") + // reduce by key + val expected = data.map(t => (t, 1)).groupBy(_._1).mapValues(_.length) + val rows = crosstab.collect() + rows.foreach { row => + val i = row.getString(0).toInt + for (col <- 1 until columnNames.length) { + val j = columnNames(col).toInt + assert(row.getLong(col) === expected.getOrElse((i, j), 0).toLong) + } } } } test("special crosstab elements (., '', null, ``)") { - val data = Seq( - ("a", Double.NaN, "ho"), - (null, 2.0, "ho"), - ("a.b", Double.NegativeInfinity, ""), - ("b", Double.PositiveInfinity, "`ha`"), - ("a", 1.0, null) - ) - val df = data.toDF("1", "2", "3") - val ct1 = df.stat.crosstab("1", "2") - // column fields should be 1 + distinct elements of second column - assert(ct1.schema.fields.length === 6) - assert(ct1.collect().length === 4) - val ct2 = df.stat.crosstab("1", "3") - assert(ct2.schema.fields.length === 5) - assert(ct2.schema.fieldNames.contains("ha")) - assert(ct2.collect().length === 4) - val ct3 = df.stat.crosstab("3", "2") - assert(ct3.schema.fields.length === 6) - assert(ct3.schema.fieldNames.contains("NaN")) - assert(ct3.schema.fieldNames.contains("Infinity")) - assert(ct3.schema.fieldNames.contains("-Infinity")) - assert(ct3.collect().length === 4) - val ct4 = df.stat.crosstab("3", "1") - assert(ct4.schema.fields.length === 5) - assert(ct4.schema.fieldNames.contains("null")) - assert(ct4.schema.fieldNames.contains("a.b")) - assert(ct4.collect().length === 4) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val data = Seq( + ("a", Double.NaN, "ho"), + (null, 2.0, "ho"), + ("a.b", Double.NegativeInfinity, ""), + ("b", Double.PositiveInfinity, "`ha`"), + ("a", 1.0, null) + ) + val df = data.toDF("1", "2", "3") + val ct1 = df.stat.crosstab("1", "2") + // column fields should be 1 + distinct elements of second column + assert(ct1.schema.fields.length === 6) + assert(ct1.collect().length === 4) + val ct2 = df.stat.crosstab("1", "3") + assert(ct2.schema.fields.length === 5) + assert(ct2.schema.fieldNames.contains("ha")) + assert(ct2.collect().length === 4) + val ct3 = df.stat.crosstab("3", "2") + assert(ct3.schema.fields.length === 6) + assert(ct3.schema.fieldNames.contains("NaN")) + assert(ct3.schema.fieldNames.contains("Infinity")) + assert(ct3.schema.fieldNames.contains("-Infinity")) + assert(ct3.collect().length === 4) + val ct4 = df.stat.crosstab("3", "1") + assert(ct4.schema.fields.length === 5) + assert(ct4.schema.fieldNames.contains("null")) + assert(ct4.schema.fieldNames.contains("a.b")) + assert(ct4.collect().length === 4) + } } test("Frequent Items") { @@ -181,11 +347,114 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { assert(items.length === 1) } + test("SPARK-15709: Prevent `UnsupportedOperationException: empty.min` in `freqItems`") { + val ds = spark.createDataset(Seq(1, 2, 2, 3, 3, 3)) + + intercept[IllegalArgumentException] { + ds.stat.freqItems(Seq("value"), 0) + } + intercept[IllegalArgumentException] { + ds.stat.freqItems(Seq("value"), 2) + } + } + test("sampleBy") { - val df = sqlContext.range(0, 100).select((col("id") % 3).as("key")) + val df = spark.range(0, 100).select((col("id") % 3).as("key")) val sampled = df.stat.sampleBy("key", Map(0 -> 0.1, 1 -> 0.2), 0L) checkAnswer( sampled.groupBy("key").count().orderBy("key"), - Seq(Row(0, 5), Row(1, 8))) + Seq(Row(0, 6), Row(1, 11))) } + + // This test case only verifies that `DataFrame.countMinSketch()` methods do return + // `CountMinSketch`es that meet required specs. Test cases for `CountMinSketch` can be found in + // `CountMinSketchSuite` in project spark-sketch. + test("countMinSketch") { + val df = spark.range(1000) + + val sketch1 = df.stat.countMinSketch("id", depth = 10, width = 20, seed = 42) + assert(sketch1.totalCount() === 1000) + assert(sketch1.depth() === 10) + assert(sketch1.width() === 20) + + val sketch2 = df.stat.countMinSketch($"id", depth = 10, width = 20, seed = 42) + assert(sketch2.totalCount() === 1000) + assert(sketch2.depth() === 10) + assert(sketch2.width() === 20) + + val sketch3 = df.stat.countMinSketch("id", eps = 0.001, confidence = 0.99, seed = 42) + assert(sketch3.totalCount() === 1000) + assert(sketch3.relativeError() === 0.001) + assert(sketch3.confidence() === 0.99 +- 5e-3) + + val sketch4 = df.stat.countMinSketch($"id", eps = 0.001, confidence = 0.99, seed = 42) + assert(sketch4.totalCount() === 1000) + assert(sketch4.relativeError() === 0.001 +- 1e04) + assert(sketch4.confidence() === 0.99 +- 5e-3) + + intercept[IllegalArgumentException] { + df.select('id cast DoubleType as 'id) + .stat + .countMinSketch('id, depth = 10, width = 20, seed = 42) + } + } + + // This test only verifies some basic requirements, more correctness tests can be found in + // `BloomFilterSuite` in project spark-sketch. + test("Bloom filter") { + val df = spark.range(1000) + + val filter1 = df.stat.bloomFilter("id", 1000, 0.03) + assert(filter1.expectedFpp() - 0.03 < 1e-3) + assert(0.until(1000).forall(filter1.mightContain)) + + val filter2 = df.stat.bloomFilter($"id" * 3, 1000, 0.03) + assert(filter2.expectedFpp() - 0.03 < 1e-3) + assert(0.until(1000).forall(i => filter2.mightContain(i * 3))) + + val filter3 = df.stat.bloomFilter("id", 1000, 64 * 5) + assert(filter3.bitSize() == 64 * 5) + assert(0.until(1000).forall(filter3.mightContain)) + + val filter4 = df.stat.bloomFilter($"id" * 3, 1000, 64 * 5) + assert(filter4.bitSize() == 64 * 5) + assert(0.until(1000).forall(i => filter4.mightContain(i * 3))) + } +} + + +class DataFrameStatPerfSuite extends QueryTest with SharedSQLContext with Logging { + + // Turn on this test if you want to test the performance of approximate quantiles. + ignore("computing quantiles should not take much longer than describe()") { + val df = spark.range(5000000L).toDF("col1").cache() + def seconds(f: => Any): Double = { + // Do some warmup + logDebug("warmup...") + for (i <- 1 to 10) { + df.count() + f + } + logDebug("execute...") + // Do it 10 times and report median + val times = (1 to 10).map { i => + val start = System.nanoTime() + f + val end = System.nanoTime() + (end - start) / 1e9 + } + logDebug("execute done") + times.sum / times.length.toDouble + } + + logDebug("*** Normal describe ***") + val t1 = seconds { df.describe() } + logDebug(s"T1 = $t1") + logDebug("*** Just quantiles ***") + val t2 = seconds { + StatFunctions.multipleApproxQuantiles(df, Seq("col1"), Seq(0.1, 0.25, 0.5, 0.75, 0.9), 0.01) + } + logDebug(s"T1 = $t1, T2 = $t2") + } + } 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 84a616d0b908..6178661cf7b2 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 @@ -18,42 +18,40 @@ package org.apache.spark.sql import java.io.File +import java.nio.charset.StandardCharsets +import java.sql.{Date, Timestamp} +import java.util.UUID -import scala.language.postfixOps import scala.util.Random import org.scalatest.Matchers._ import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation -import org.apache.spark.sql.execution.Exchange -import org.apache.spark.sql.execution.aggregate.TungstenAggregate +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.plans.logical.{Filter, OneRowRelation, Union} +import org.apache.spark.sql.execution.{FilterExec, QueryExecution} +import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchange} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SQLTestData.TestData2 +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext} +import org.apache.spark.sql.test.SQLTestData.TestData2 import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils class DataFrameSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("analysis error should be eagerly reported") { - // Eager analysis. - withSQLConf(SQLConf.DATAFRAME_EAGER_ANALYSIS.key -> "true") { - intercept[Exception] { testData.select('nonExistentName) } - intercept[Exception] { - testData.groupBy('key).agg(Map("nonExistentName" -> "sum")) - } - intercept[Exception] { - testData.groupBy("nonExistentName").agg(Map("key" -> "sum")) - } - intercept[Exception] { - testData.groupBy($"abcd").agg(Map("key" -> "sum")) - } + intercept[Exception] { testData.select('nonExistentName) } + intercept[Exception] { + testData.groupBy('key).agg(Map("nonExistentName" -> "sum")) } - - // No more eager analysis once the flag is turned off - withSQLConf(SQLConf.DATAFRAME_EAGER_ANALYSIS.key -> "false") { - testData.select('nonExistentName) + intercept[Exception] { + testData.groupBy("nonExistentName").agg(Map("key" -> "sum")) + } + intercept[Exception] { + testData.groupBy($"abcd").agg(Map("key" -> "sum")) } } @@ -71,21 +69,6 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { Row(1, 1) :: Nil) } - test("invalid plan toString, debug mode") { - // Turn on debug mode so we can see invalid query plans. - import org.apache.spark.sql.execution.debug._ - - withSQLConf(SQLConf.DATAFRAME_EAGER_ANALYSIS.key -> "true") { - sqlContext.debug() - - val badPlan = testData.select('badColumn) - - assert(badPlan.toString contains badPlan.queryExecution.toString, - "toString on bad query plans should include the query execution but was:\n" + - badPlan.toString) - } - } - test("access complex data") { assert(complexData.filter(complexData("a").getItem(0) === 2).count() == 1) assert(complexData.filter(complexData("m").getItem("1") === 1).count() == 1) @@ -98,9 +81,126 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { testData.collect().toSeq) } + test("union all") { + val unionDF = testData.union(testData).union(testData) + .union(testData).union(testData) + + // Before optimizer, Union should be combined. + assert(unionDF.queryExecution.analyzed.collect { + case j: Union if j.children.size == 5 => j }.size === 1) + + checkAnswer( + unionDF.agg(avg('key), max('key), min('key), sum('key)), + Row(50.5, 100, 1, 25250) :: Nil + ) + } + + test("union should union DataFrames with UDTs (SPARK-13410)") { + val rowRDD1 = sparkContext.parallelize(Seq(Row(1, new ExamplePoint(1.0, 2.0)))) + val schema1 = StructType(Array(StructField("label", IntegerType, false), + StructField("point", new ExamplePointUDT(), false))) + val rowRDD2 = sparkContext.parallelize(Seq(Row(2, new ExamplePoint(3.0, 4.0)))) + val schema2 = StructType(Array(StructField("label", IntegerType, false), + StructField("point", new ExamplePointUDT(), false))) + val df1 = spark.createDataFrame(rowRDD1, schema1) + val df2 = spark.createDataFrame(rowRDD2, schema2) + + checkAnswer( + df1.union(df2).orderBy("label"), + Seq(Row(1, new ExamplePoint(1.0, 2.0)), Row(2, new ExamplePoint(3.0, 4.0))) + ) + } + + test("union by name") { + var df1 = Seq((1, 2, 3)).toDF("a", "b", "c") + var df2 = Seq((3, 1, 2)).toDF("c", "a", "b") + val df3 = Seq((2, 3, 1)).toDF("b", "c", "a") + val unionDf = df1.unionByName(df2.unionByName(df3)) + checkAnswer(unionDf, + Row(1, 2, 3) :: Row(1, 2, 3) :: Row(1, 2, 3) :: Nil + ) + + // Check if adjacent unions are combined into a single one + assert(unionDf.queryExecution.optimizedPlan.collect { case u: Union => true }.size == 1) + + // Check failure cases + df1 = Seq((1, 2)).toDF("a", "c") + df2 = Seq((3, 4, 5)).toDF("a", "b", "c") + var errMsg = intercept[AnalysisException] { + df1.unionByName(df2) + }.getMessage + assert(errMsg.contains( + "Union can only be performed on tables with the same number of columns, " + + "but the first table has 2 columns and the second table has 3 columns")) + + df1 = Seq((1, 2, 3)).toDF("a", "b", "c") + df2 = Seq((4, 5, 6)).toDF("a", "c", "d") + errMsg = intercept[AnalysisException] { + df1.unionByName(df2) + }.getMessage + assert(errMsg.contains("""Cannot resolve column name "b" among (a, c, d)""")) + } + + test("union by name - type coercion") { + var df1 = Seq((1, "a")).toDF("c0", "c1") + var df2 = Seq((3, 1L)).toDF("c1", "c0") + checkAnswer(df1.unionByName(df2), Row(1L, "a") :: Row(1L, "3") :: Nil) + + df1 = Seq((1, 1.0)).toDF("c0", "c1") + df2 = Seq((8L, 3.0)).toDF("c1", "c0") + checkAnswer(df1.unionByName(df2), Row(1.0, 1.0) :: Row(3.0, 8.0) :: Nil) + + df1 = Seq((2.0f, 7.4)).toDF("c0", "c1") + df2 = Seq(("a", 4.0)).toDF("c1", "c0") + checkAnswer(df1.unionByName(df2), Row(2.0, "7.4") :: Row(4.0, "a") :: Nil) + + df1 = Seq((1, "a", 3.0)).toDF("c0", "c1", "c2") + df2 = Seq((1.2, 2, "bc")).toDF("c2", "c0", "c1") + val df3 = Seq(("def", 1.2, 3)).toDF("c1", "c2", "c0") + checkAnswer(df1.unionByName(df2.unionByName(df3)), + Row(1, "a", 3.0) :: Row(2, "bc", 1.2) :: Row(3, "def", 1.2) :: Nil + ) + } + + test("union by name - check case sensitivity") { + def checkCaseSensitiveTest(): Unit = { + val df1 = Seq((1, 2, 3)).toDF("ab", "cd", "ef") + val df2 = Seq((4, 5, 6)).toDF("cd", "ef", "AB") + checkAnswer(df1.unionByName(df2), Row(1, 2, 3) :: Row(6, 4, 5) :: Nil) + } + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val errMsg2 = intercept[AnalysisException] { + checkCaseSensitiveTest() + }.getMessage + assert(errMsg2.contains("""Cannot resolve column name "ab" among (cd, ef, AB)""")) + } + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + checkCaseSensitiveTest() + } + } + + test("union by name - check name duplication") { + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + var df1 = Seq((1, 1)).toDF(c0, c1) + var df2 = Seq((1, 1)).toDF("c0", "c1") + var errMsg = intercept[AnalysisException] { + df1.unionByName(df2) + }.getMessage + assert(errMsg.contains("Found duplicate column(s) in the left attributes:")) + df1 = Seq((1, 1)).toDF("c0", "c1") + df2 = Seq((1, 1)).toDF(c0, c1) + errMsg = intercept[AnalysisException] { + df1.unionByName(df2) + }.getMessage + assert(errMsg.contains("Found duplicate column(s) in the right attributes:")) + } + } + } + test("empty data frame") { - assert(sqlContext.emptyDataFrame.columns.toSeq === Seq.empty[String]) - assert(sqlContext.emptyDataFrame.count() === 0) + assert(spark.emptyDataFrame.columns.toSeq === Seq.empty[String]) + assert(spark.emptyDataFrame.count() === 0) } test("head and take") { @@ -141,22 +241,62 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { ) } - test("SPARK-8930: explode should fail with a meaningful message if it takes a star") { + test("Star Expansion - CreateStruct and CreateArray") { + val structDf = testData2.select("a", "b").as("record") + // CreateStruct and CreateArray in aggregateExpressions + assert(structDf.groupBy($"a").agg(min(struct($"record.*"))).first() == Row(3, Row(3, 1))) + assert(structDf.groupBy($"a").agg(min(array($"record.*"))).first() == Row(3, Seq(3, 1))) + + // CreateStruct and CreateArray in project list (unresolved alias) + assert(structDf.select(struct($"record.*")).first() == Row(Row(1, 1))) + assert(structDf.select(array($"record.*")).first().getAs[Seq[Int]](0) === Seq(1, 1)) + + // CreateStruct and CreateArray in project list (alias) + assert(structDf.select(struct($"record.*").as("a")).first() == Row(Row(1, 1))) + assert(structDf.select(array($"record.*").as("a")).first().getAs[Seq[Int]](0) === Seq(1, 1)) + } + + test("Star Expansion - hash") { + val structDf = testData2.select("a", "b").as("record") + checkAnswer( + structDf.groupBy($"a", $"b").agg(min(hash($"a", $"*"))), + structDf.groupBy($"a", $"b").agg(min(hash($"a", $"a", $"b")))) + + checkAnswer( + structDf.groupBy($"a", $"b").agg(hash($"a", $"*")), + structDf.groupBy($"a", $"b").agg(hash($"a", $"a", $"b"))) + + checkAnswer( + structDf.select(hash($"*")), + structDf.select(hash($"record.*"))) + + checkAnswer( + structDf.select(hash($"a", $"*")), + structDf.select(hash($"a", $"record.*"))) + } + + test("Star Expansion - explode should fail with a meaningful message if it takes a star") { val df = Seq(("1", "1,2"), ("2", "4"), ("3", "7,8,9")).toDF("prefix", "csv") val e = intercept[AnalysisException] { df.explode($"*") { case Row(prefix: String, csv: String) => csv.split(",").map(v => Tuple1(prefix + ":" + v)).toSeq }.queryExecution.assertAnalyzed() } - assert(e.getMessage.contains( - "Cannot explode *, explode can only be applied on a specific column.")) + assert(e.getMessage.contains("Invalid usage of '*' in explode/json_tuple/UDTF")) - df.explode('prefix, 'csv) { case Row(prefix: String, csv: String) => - csv.split(",").map(v => Tuple1(prefix + ":" + v)).toSeq - }.queryExecution.assertAnalyzed() + checkAnswer( + df.explode('prefix, 'csv) { case Row(prefix: String, csv: String) => + csv.split(",").map(v => Tuple1(prefix + ":" + v)).toSeq + }, + Row("1", "1,2", "1:1") :: + Row("1", "1,2", "1:2") :: + Row("2", "4", "2:4") :: + Row("3", "7,8,9", "3:7") :: + Row("3", "7,8,9", "3:8") :: + Row("3", "7,8,9", "3:9") :: Nil) } - test("explode alias and star") { + test("Star Expansion - explode alias and star") { val df = Seq((Array("a"), 1)).toDF("a", "b") checkAnswer( @@ -164,6 +304,14 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { Row("a", Seq("a"), 1) :: Nil) } + test("sort after generate with join=true") { + val df = Seq((Array("a"), 1)).toDF("a", "b") + + checkAnswer( + df.select($"*", explode($"a").as("c")).sortWithinPartitions("b", "c"), + Row(Seq("a"), 1, "a") :: Nil) + } + test("selectExpr") { checkAnswer( testData.selectExpr("abs(key)", "value"), @@ -176,6 +324,13 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { testData.select("key").collect().toSeq) } + test("selectExpr with udtf") { + val df = Seq((Map("1" -> 1), 1)).toDF("a", "b") + checkAnswer( + df.selectExpr("explode(a)"), + Row("1", 1) :: Nil) + } + test("filterExpr") { val res = testData.collect().filter(_.getInt(0) > 90).toSeq checkAnswer(testData.filter("key > 90"), res) @@ -194,12 +349,20 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("repartition") { + intercept[IllegalArgumentException] { + testData.select('key).repartition(0) + } + checkAnswer( testData.select('key).repartition(10).select('key), testData.select('key).collect().toSeq) } test("coalesce") { + intercept[IllegalArgumentException] { + testData.select('key).coalesce(0) + } + assert(testData.select('key).coalesce(1).rdd.partitions.size === 1) checkAnswer( @@ -251,6 +414,24 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { Row(6)) } + test("sorting with null ordering") { + val data = Seq[java.lang.Integer](2, 1, null).toDF("key") + + checkAnswer(data.orderBy('key.asc), Row(null) :: Row(1) :: Row(2) :: Nil) + checkAnswer(data.orderBy(asc("key")), Row(null) :: Row(1) :: Row(2) :: Nil) + checkAnswer(data.orderBy('key.asc_nulls_first), Row(null) :: Row(1) :: Row(2) :: Nil) + checkAnswer(data.orderBy(asc_nulls_first("key")), Row(null) :: Row(1) :: Row(2) :: Nil) + checkAnswer(data.orderBy('key.asc_nulls_last), Row(1) :: Row(2) :: Row(null) :: Nil) + checkAnswer(data.orderBy(asc_nulls_last("key")), Row(1) :: Row(2) :: Row(null) :: Nil) + + checkAnswer(data.orderBy('key.desc), Row(2) :: Row(1) :: Row(null) :: Nil) + checkAnswer(data.orderBy(desc("key")), Row(2) :: Row(1) :: Row(null) :: Nil) + checkAnswer(data.orderBy('key.desc_nulls_first), Row(null) :: Row(2) :: Row(1) :: Nil) + checkAnswer(data.orderBy(desc_nulls_first("key")), Row(null) :: Row(2) :: Row(1) :: Nil) + checkAnswer(data.orderBy('key.desc_nulls_last), Row(2) :: Row(1) :: Row(null) :: Nil) + checkAnswer(data.orderBy(desc_nulls_last("key")), Row(2) :: Row(1) :: Row(null) :: Nil) + } + test("global sorting") { checkAnswer( testData2.orderBy('a.asc, 'b.asc), @@ -301,6 +482,12 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { checkAnswer( mapData.toDF().limit(1), mapData.take(1).map(r => Row.fromSeq(r.productIterator.toSeq))) + + // SPARK-12340: overstep the bounds of Int in SparkPlan.executeTake + checkAnswer( + spark.range(2).toDF().limit(2147483638), + Row(0) :: Row(1) :: Nil + ) } test("except") { @@ -312,6 +499,66 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { Row(4, "d") :: Nil) checkAnswer(lowerCaseData.except(lowerCaseData), Nil) checkAnswer(upperCaseData.except(upperCaseData), Nil) + + // check null equality + checkAnswer( + nullInts.except(nullInts.filter("0 = 1")), + nullInts) + checkAnswer( + nullInts.except(nullInts), + Nil) + + // check if values are de-duplicated + checkAnswer( + allNulls.except(allNulls.filter("0 = 1")), + Row(null) :: Nil) + checkAnswer( + allNulls.except(allNulls), + Nil) + + // check if values are de-duplicated + val df = Seq(("id1", 1), ("id1", 1), ("id", 1), ("id1", 2)).toDF("id", "value") + checkAnswer( + df.except(df.filter("0 = 1")), + Row("id1", 1) :: + Row("id", 1) :: + Row("id1", 2) :: Nil) + + // check if the empty set on the left side works + checkAnswer( + allNulls.filter("0 = 1").except(allNulls), + Nil) + } + + test("except distinct - SQL compliance") { + val df_left = Seq(1, 2, 2, 3, 3, 4).toDF("id") + val df_right = Seq(1, 3).toDF("id") + + checkAnswer( + df_left.except(df_right), + Row(2) :: Row(4) :: Nil + ) + } + + test("except - nullability") { + val nonNullableInts = Seq(Tuple1(11), Tuple1(3)).toDF() + assert(nonNullableInts.schema.forall(!_.nullable)) + + val df1 = nonNullableInts.except(nullInts) + checkAnswer(df1, Row(11) :: Nil) + assert(df1.schema.forall(!_.nullable)) + + val df2 = nullInts.except(nonNullableInts) + checkAnswer(df2, Row(1) :: Row(2) :: Row(null) :: Nil) + assert(df2.schema.forall(_.nullable)) + + val df3 = nullInts.except(nullInts) + checkAnswer(df3, Nil) + assert(df3.schema.forall(_.nullable)) + + val df4 = nonNullableInts.except(nonNullableInts) + checkAnswer(df4, Nil) + assert(df4.schema.forall(!_.nullable)) } test("intersect") { @@ -322,6 +569,48 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { Row(3, "c") :: Row(4, "d") :: Nil) checkAnswer(lowerCaseData.intersect(upperCaseData), Nil) + + // check null equality + checkAnswer( + nullInts.intersect(nullInts), + Row(1) :: + Row(2) :: + Row(3) :: + Row(null) :: Nil) + + // check if values are de-duplicated + checkAnswer( + allNulls.intersect(allNulls), + Row(null) :: Nil) + + // check if values are de-duplicated + val df = Seq(("id1", 1), ("id1", 1), ("id", 1), ("id1", 2)).toDF("id", "value") + checkAnswer( + df.intersect(df), + Row("id1", 1) :: + Row("id", 1) :: + Row("id1", 2) :: Nil) + } + + test("intersect - nullability") { + val nonNullableInts = Seq(Tuple1(1), Tuple1(3)).toDF() + assert(nonNullableInts.schema.forall(!_.nullable)) + + val df1 = nonNullableInts.intersect(nullInts) + checkAnswer(df1, Row(1) :: Row(3) :: Nil) + assert(df1.schema.forall(!_.nullable)) + + val df2 = nullInts.intersect(nonNullableInts) + checkAnswer(df2, Row(1) :: Row(3) :: Nil) + assert(df2.schema.forall(!_.nullable)) + + val df3 = nullInts.intersect(nullInts) + checkAnswer(df3, Row(1) :: Row(2) :: Row(3) :: Row(null) :: Nil) + assert(df3.schema.forall(_.nullable)) + + val df4 = nonNullableInts.intersect(nonNullableInts) + checkAnswer(df4, Row(1) :: Row(3) :: Nil) + assert(df4.schema.forall(!_.nullable)) } test("udf") { @@ -334,19 +623,9 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { ) } - test("deprecated callUdf in SQLContext") { - val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value") - val sqlctx = df.sqlContext - sqlctx.udf.register("simpleUdf", (v: Int) => v * v) - checkAnswer( - df.select($"id", callUdf("simpleUdf", $"value")), - Row("id1", 1) :: Row("id2", 16) :: Row("id3", 25) :: Nil) - } - - test("callUDF in SQLContext") { + test("callUDF without Hive Support") { val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value") - val sqlctx = df.sqlContext - sqlctx.udf.register("simpleUDF", (v: Int) => v * v) + df.sparkSession.udf.register("simpleUDF", (v: Int) => v * v) checkAnswer( df.select($"id", callUDF("simpleUDF", $"value")), Row("id1", 1) :: Row("id2", 16) :: Row("id3", 25) :: Nil) @@ -378,6 +657,13 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(df.schema.map(_.name) === Seq("value")) } + test("drop columns using drop") { + val src = Seq((0, 2, 3)).toDF("a", "b", "c") + val df = src.drop("a", "b") + checkAnswer(df, Row(3)) + assert(df.schema.map(_.name) === Seq("c")) + } + test("drop unknown column (no-op)") { val df = testData.drop("random") checkAnswer( @@ -431,6 +717,27 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(df("id") == person("id")) } + test("drop top level columns that contains dot") { + val df1 = Seq((1, 2)).toDF("a.b", "a.c") + checkAnswer(df1.drop("a.b"), Row(2)) + + // Creates data set: {"a.b": 1, "a": {"b": 3}} + val df2 = Seq((1)).toDF("a.b").withColumn("a", struct(lit(3) as "b")) + // Not like select(), drop() parses the column name "a.b" literally without interpreting "." + checkAnswer(df2.drop("a.b").select("a.b"), Row(3)) + + // "`" is treated as a normal char here with no interpreting, "`a`b" is a valid column name. + assert(df2.drop("`a.b`").columns.size == 2) + } + + test("drop(name: String) search and drop all top level columns that matchs the name") { + val df1 = Seq((1, 2)).toDF("a", "b") + val df2 = Seq((3, 4)).toDF("a", "b") + checkAnswer(df1.crossJoin(df2), Row(1, 2, 3, 4)) + // Finds and drops all columns that match the name (case insensitive). + checkAnswer(df1.crossJoin(df2).drop("A"), Row(2, 4)) + } + test("withColumnRenamed") { val df = testData.toDF().withColumn("newCol", col("key") + 1) .withColumnRenamed("value", "valueRenamed") @@ -442,57 +749,124 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(df.schema.map(_.name) === Seq("key", "valueRenamed", "newCol")) } - test("describe") { - val describeTestData = Seq( - ("Bob", 16, 176), - ("Alice", 32, 164), - ("David", 60, 192), - ("Amy", 24, 180)).toDF("name", "age", "height") + private lazy val person2: DataFrame = Seq( + ("Bob", 16, 176), + ("Alice", 32, 164), + ("David", 60, 192), + ("Amy", 24, 180)).toDF("name", "age", "height") + test("describe") { val describeResult = Seq( - Row("count", "4", "4"), - Row("mean", "33.0", "178.0"), - Row("stddev", "19.148542155126762", "11.547005383792516"), - Row("min", "16", "164"), - Row("max", "60", "192")) + Row("count", "4", "4", "4"), + Row("mean", null, "33.0", "178.0"), + Row("stddev", null, "19.148542155126762", "11.547005383792516"), + Row("min", "Alice", "16", "164"), + Row("max", "David", "60", "192")) val emptyDescribeResult = Seq( - Row("count", "0", "0"), - Row("mean", null, null), - Row("stddev", null, null), - Row("min", null, null), - Row("max", null, null)) + Row("count", "0", "0", "0"), + Row("mean", null, null, null), + Row("stddev", null, null, null), + Row("min", null, null, null), + Row("max", null, null, null)) def getSchemaAsSeq(df: DataFrame): Seq[String] = df.schema.map(_.name) - val describeTwoCols = describeTestData.describe("age", "height") - assert(getSchemaAsSeq(describeTwoCols) === Seq("summary", "age", "height")) - checkAnswer(describeTwoCols, describeResult) + val describeAllCols = person2.describe() + assert(getSchemaAsSeq(describeAllCols) === Seq("summary", "name", "age", "height")) + checkAnswer(describeAllCols, describeResult) // All aggregate value should have been cast to string - describeTwoCols.collect().foreach { row => - assert(row.get(1).isInstanceOf[String], "expected string but found " + row.get(1).getClass) - assert(row.get(2).isInstanceOf[String], "expected string but found " + row.get(2).getClass) + describeAllCols.collect().foreach { row => + row.toSeq.foreach { value => + if (value != null) { + assert(value.isInstanceOf[String], "expected string but found " + value.getClass) + } + } } - val describeAllCols = describeTestData.describe() - assert(getSchemaAsSeq(describeAllCols) === Seq("summary", "age", "height")) - checkAnswer(describeAllCols, describeResult) - - val describeOneCol = describeTestData.describe("age") + val describeOneCol = person2.describe("age") assert(getSchemaAsSeq(describeOneCol) === Seq("summary", "age")) - checkAnswer(describeOneCol, describeResult.map { case Row(s, d, _) => Row(s, d)} ) + checkAnswer(describeOneCol, describeResult.map { case Row(s, _, d, _) => Row(s, d)} ) - val describeNoCol = describeTestData.select("name").describe() + val describeNoCol = person2.select().describe() assert(getSchemaAsSeq(describeNoCol) === Seq("summary")) - checkAnswer(describeNoCol, describeResult.map { case Row(s, _, _) => Row(s)} ) + checkAnswer(describeNoCol, describeResult.map { case Row(s, _, _, _) => Row(s)} ) - val emptyDescription = describeTestData.limit(0).describe() - assert(getSchemaAsSeq(emptyDescription) === Seq("summary", "age", "height")) + val emptyDescription = person2.limit(0).describe() + assert(getSchemaAsSeq(emptyDescription) === Seq("summary", "name", "age", "height")) checkAnswer(emptyDescription, emptyDescribeResult) } + test("summary") { + val summaryResult = Seq( + Row("count", "4", "4", "4"), + Row("mean", null, "33.0", "178.0"), + Row("stddev", null, "19.148542155126762", "11.547005383792516"), + Row("min", "Alice", "16", "164"), + Row("25%", null, "24", "176"), + Row("50%", null, "24", "176"), + Row("75%", null, "32", "180"), + Row("max", "David", "60", "192")) + + val emptySummaryResult = Seq( + Row("count", "0", "0", "0"), + Row("mean", null, null, null), + Row("stddev", null, null, null), + Row("min", null, null, null), + Row("25%", null, null, null), + Row("50%", null, null, null), + Row("75%", null, null, null), + Row("max", null, null, null)) + + def getSchemaAsSeq(df: DataFrame): Seq[String] = df.schema.map(_.name) + + val summaryAllCols = person2.summary() + + assert(getSchemaAsSeq(summaryAllCols) === Seq("summary", "name", "age", "height")) + checkAnswer(summaryAllCols, summaryResult) + // All aggregate value should have been cast to string + summaryAllCols.collect().foreach { row => + row.toSeq.foreach { value => + if (value != null) { + assert(value.isInstanceOf[String], "expected string but found " + value.getClass) + } + } + } + + val summaryOneCol = person2.select("age").summary() + assert(getSchemaAsSeq(summaryOneCol) === Seq("summary", "age")) + checkAnswer(summaryOneCol, summaryResult.map { case Row(s, _, d, _) => Row(s, d)} ) + + val summaryNoCol = person2.select().summary() + assert(getSchemaAsSeq(summaryNoCol) === Seq("summary")) + checkAnswer(summaryNoCol, summaryResult.map { case Row(s, _, _, _) => Row(s)} ) + + val emptyDescription = person2.limit(0).summary() + assert(getSchemaAsSeq(emptyDescription) === Seq("summary", "name", "age", "height")) + checkAnswer(emptyDescription, emptySummaryResult) + } + + test("summary advanced") { + val stats = Array("count", "50.01%", "max", "mean", "min", "25%") + val orderMatters = person2.summary(stats: _*) + assert(orderMatters.collect().map(_.getString(0)) === stats) + + val onlyPercentiles = person2.summary("0.1%", "99.9%") + assert(onlyPercentiles.count() === 2) + + val fooE = intercept[IllegalArgumentException] { + person2.summary("foo") + } + assert(fooE.getMessage === "foo is not a recognised statistic") + + val parseE = intercept[IllegalArgumentException] { + person2.summary("foo%") + } + assert(parseE.getMessage === "Unable to parse foo% as a percentile") + } + test("apply on query results (SPARK-5462)") { - val df = testData.sqlContext.sql("select key from testData") + val df = testData.sparkSession.sql("select key from testData") checkAnswer(df.select(df("key")), testData.select('key).collect().toSeq) } @@ -502,16 +876,16 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val parquetDir = new File(dir, "parquet").getCanonicalPath df.write.parquet(parquetDir) - val parquetDF = sqlContext.read.parquet(parquetDir) + val parquetDF = spark.read.parquet(parquetDir) assert(parquetDF.inputFiles.nonEmpty) val jsonDir = new File(dir, "json").getCanonicalPath df.write.json(jsonDir) - val jsonDF = sqlContext.read.json(jsonDir) + val jsonDF = spark.read.json(jsonDir) assert(parquetDF.inputFiles.nonEmpty) - val unioned = jsonDF.unionAll(parquetDF).inputFiles.sorted - val allFiles = (jsonDF.inputFiles ++ parquetDF.inputFiles).toSet.toArray.sorted + val unioned = jsonDF.union(parquetDF).inputFiles.sorted + val allFiles = (jsonDF.inputFiles ++ parquetDF.inputFiles).distinct.sorted assert(unioned === allFiles) } } @@ -522,25 +896,76 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { testData.select($"*").show(1000) } - test("showString: truncate = [true, false]") { + test("showString: truncate = [0, 20]") { val longString = Array.fill(21)("1").mkString val df = sparkContext.parallelize(Seq("1", longString)).toDF() val expectedAnswerForFalse = """+---------------------+ - ||_1 | + ||value | |+---------------------+ ||1 | ||111111111111111111111| |+---------------------+ |""".stripMargin - assert(df.showString(10, false) === expectedAnswerForFalse) + assert(df.showString(10, truncate = 0) === expectedAnswerForFalse) val expectedAnswerForTrue = """+--------------------+ - || _1| + || value| |+--------------------+ || 1| ||11111111111111111...| |+--------------------+ |""".stripMargin - assert(df.showString(10, true) === expectedAnswerForTrue) + assert(df.showString(10, truncate = 20) === expectedAnswerForTrue) + } + + test("showString: truncate = [0, 20], vertical = true") { + val longString = Array.fill(21)("1").mkString + val df = sparkContext.parallelize(Seq("1", longString)).toDF() + val expectedAnswerForFalse = "-RECORD 0----------------------\n" + + " value | 1 \n" + + "-RECORD 1----------------------\n" + + " value | 111111111111111111111 \n" + assert(df.showString(10, truncate = 0, vertical = true) === expectedAnswerForFalse) + val expectedAnswerForTrue = "-RECORD 0---------------------\n" + + " value | 1 \n" + + "-RECORD 1---------------------\n" + + " value | 11111111111111111... \n" + assert(df.showString(10, truncate = 20, vertical = true) === expectedAnswerForTrue) + } + + test("showString: truncate = [3, 17]") { + val longString = Array.fill(21)("1").mkString + val df = sparkContext.parallelize(Seq("1", longString)).toDF() + val expectedAnswerForFalse = """+-----+ + ||value| + |+-----+ + || 1| + || 111| + |+-----+ + |""".stripMargin + assert(df.showString(10, truncate = 3) === expectedAnswerForFalse) + val expectedAnswerForTrue = """+-----------------+ + || value| + |+-----------------+ + || 1| + ||11111111111111...| + |+-----------------+ + |""".stripMargin + assert(df.showString(10, truncate = 17) === expectedAnswerForTrue) + } + + test("showString: truncate = [3, 17], vertical = true") { + val longString = Array.fill(21)("1").mkString + val df = sparkContext.parallelize(Seq("1", longString)).toDF() + val expectedAnswerForFalse = "-RECORD 0----\n" + + " value | 1 \n" + + "-RECORD 1----\n" + + " value | 111 \n" + assert(df.showString(10, truncate = 3, vertical = true) === expectedAnswerForFalse) + val expectedAnswerForTrue = "-RECORD 0------------------\n" + + " value | 1 \n" + + "-RECORD 1------------------\n" + + " value | 11111111111111... \n" + assert(df.showString(10, truncate = 17, vertical = true) === expectedAnswerForTrue) } test("showString(negative)") { @@ -553,6 +978,11 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(testData.select($"*").showString(-1) === expectedAnswer) } + test("showString(negative), vertical = true") { + val expectedAnswer = "(0 rows)\n" + assert(testData.select($"*").showString(-1, vertical = true) === expectedAnswer) + } + test("showString(0)") { val expectedAnswer = """+---+-----+ ||key|value| @@ -563,6 +993,11 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(testData.select($"*").showString(0) === expectedAnswer) } + test("showString(0), vertical = true") { + val expectedAnswer = "(0 rows)\n" + assert(testData.select($"*").showString(0, vertical = true) === expectedAnswer) + } + test("showString: array") { val df = Seq( (Array(1, 2, 3), Array(1, 2, 3)), @@ -578,6 +1013,49 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(df.showString(10) === expectedAnswer) } + test("showString: array, vertical = true") { + val df = Seq( + (Array(1, 2, 3), Array(1, 2, 3)), + (Array(2, 3, 4), Array(2, 3, 4)) + ).toDF() + val expectedAnswer = "-RECORD 0--------\n" + + " _1 | [1, 2, 3] \n" + + " _2 | [1, 2, 3] \n" + + "-RECORD 1--------\n" + + " _1 | [2, 3, 4] \n" + + " _2 | [2, 3, 4] \n" + assert(df.showString(10, vertical = true) === expectedAnswer) + } + + test("showString: binary") { + val df = Seq( + ("12".getBytes(StandardCharsets.UTF_8), "ABC.".getBytes(StandardCharsets.UTF_8)), + ("34".getBytes(StandardCharsets.UTF_8), "12346".getBytes(StandardCharsets.UTF_8)) + ).toDF() + val expectedAnswer = """+-------+----------------+ + || _1| _2| + |+-------+----------------+ + ||[31 32]| [41 42 43 2E]| + ||[33 34]|[31 32 33 34 36]| + |+-------+----------------+ + |""".stripMargin + assert(df.showString(10) === expectedAnswer) + } + + test("showString: binary, vertical = true") { + val df = Seq( + ("12".getBytes(StandardCharsets.UTF_8), "ABC.".getBytes(StandardCharsets.UTF_8)), + ("34".getBytes(StandardCharsets.UTF_8), "12346".getBytes(StandardCharsets.UTF_8)) + ).toDF() + val expectedAnswer = "-RECORD 0---------------\n" + + " _1 | [31 32] \n" + + " _2 | [41 42 43 2E] \n" + + "-RECORD 1---------------\n" + + " _1 | [33 34] \n" + + " _2 | [31 32 33 34 36] \n" + assert(df.showString(10, vertical = true) === expectedAnswer) + } + test("showString: minimum column width") { val df = Seq( (1, 1), @@ -593,6 +1071,20 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(df.showString(10) === expectedAnswer) } + test("showString: minimum column width, vertical = true") { + val df = Seq( + (1, 1), + (2, 2) + ).toDF() + val expectedAnswer = "-RECORD 0--\n" + + " _1 | 1 \n" + + " _2 | 1 \n" + + "-RECORD 1--\n" + + " _1 | 2 \n" + + " _2 | 2 \n" + assert(df.showString(10, vertical = true) === expectedAnswer) + } + test("SPARK-7319 showString") { val expectedAnswer = """+---+-----+ ||key|value| @@ -604,6 +1096,14 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(testData.select($"*").showString(1) === expectedAnswer) } + test("SPARK-7319 showString, vertical = true") { + val expectedAnswer = "-RECORD 0----\n" + + " key | 1 \n" + + " value | 1 \n" + + "only showing top 1 row\n" + assert(testData.select($"*").showString(1, vertical = true) === expectedAnswer) + } + test("SPARK-7327 show with empty dataFrame") { val expectedAnswer = """+---+-----+ ||key|value| @@ -613,19 +1113,61 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(testData.select($"*").filter($"key" < 0).showString(1) === expectedAnswer) } + test("SPARK-7327 show with empty dataFrame, vertical = true") { + assert(testData.select($"*").filter($"key" < 0).showString(1, vertical = true) === "(0 rows)\n") + } + + test("SPARK-18350 show with session local timezone") { + val d = Date.valueOf("2016-12-01") + val ts = Timestamp.valueOf("2016-12-01 00:00:00") + val df = Seq((d, ts)).toDF("d", "ts") + val expectedAnswer = """+----------+-------------------+ + ||d |ts | + |+----------+-------------------+ + ||2016-12-01|2016-12-01 00:00:00| + |+----------+-------------------+ + |""".stripMargin + assert(df.showString(1, truncate = 0) === expectedAnswer) + + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") { + + val expectedAnswer = """+----------+-------------------+ + ||d |ts | + |+----------+-------------------+ + ||2016-12-01|2016-12-01 08:00:00| + |+----------+-------------------+ + |""".stripMargin + assert(df.showString(1, truncate = 0) === expectedAnswer) + } + } + + test("SPARK-18350 show with session local timezone, vertical = true") { + val d = Date.valueOf("2016-12-01") + val ts = Timestamp.valueOf("2016-12-01 00:00:00") + val df = Seq((d, ts)).toDF("d", "ts") + val expectedAnswer = "-RECORD 0------------------\n" + + " d | 2016-12-01 \n" + + " ts | 2016-12-01 00:00:00 \n" + assert(df.showString(1, truncate = 0, vertical = true) === expectedAnswer) + + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") { + + val expectedAnswer = "-RECORD 0------------------\n" + + " d | 2016-12-01 \n" + + " ts | 2016-12-01 08:00:00 \n" + assert(df.showString(1, truncate = 0, vertical = true) === expectedAnswer) + } + } + test("createDataFrame(RDD[Row], StructType) should convert UDTs (SPARK-6672)") { val rowRDD = sparkContext.parallelize(Seq(Row(new ExamplePoint(1.0, 2.0)))) val schema = StructType(Array(StructField("point", new ExamplePointUDT(), false))) - val df = sqlContext.createDataFrame(rowRDD, schema) + val df = spark.createDataFrame(rowRDD, schema) df.rdd.collect() } test("SPARK-6899: type should match when using codegen") { - withSQLConf(SQLConf.CODEGEN_ENABLED.key -> "true") { - checkAnswer( - decimalData.agg(avg('a)), - Row(new java.math.BigDecimal(2.0))) - } + checkAnswer(decimalData.agg(avg('a)), Row(new java.math.BigDecimal(2))) } test("SPARK-7133: Implement struct, array, and map field accessor") { @@ -637,30 +1179,31 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("SPARK-7551: support backticks for DataFrame attribute resolution") { - val df = sqlContext.read.json(sparkContext.makeRDD( - """{"a.b": {"c": {"d..e": {"f": 1}}}}""" :: Nil)) - checkAnswer( - df.select(df("`a.b`.c.`d..e`.`f`")), - Row(1) - ) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val df = spark.read.json(Seq("""{"a.b": {"c": {"d..e": {"f": 1}}}}""").toDS()) + checkAnswer( + df.select(df("`a.b`.c.`d..e`.`f`")), + Row(1) + ) - val df2 = sqlContext.read.json(sparkContext.makeRDD( - """{"a b": {"c": {"d e": {"f": 1}}}}""" :: Nil)) - checkAnswer( - df2.select(df2("`a b`.c.d e.f")), - Row(1) - ) + val df2 = spark.read.json(Seq("""{"a b": {"c": {"d e": {"f": 1}}}}""").toDS()) + checkAnswer( + df2.select(df2("`a b`.c.d e.f")), + Row(1) + ) - def checkError(testFun: => Unit): Unit = { - val e = intercept[org.apache.spark.sql.AnalysisException] { - testFun + def checkError(testFun: => Unit): Unit = { + val e = intercept[org.apache.spark.sql.AnalysisException] { + testFun + } + assert(e.getMessage.contains("syntax error in attribute name:")) } - assert(e.getMessage.contains("syntax error in attribute name:")) + + checkError(df("`abc.`c`")) + checkError(df("`abc`..d")) + checkError(df("`a`.b.")) + checkError(df("`a.b`.c.`d")) } - checkError(df("`abc.`c`")) - checkError(df("`abc`..d")) - checkError(df("`a`.b.")) - checkError(df("`a.b`.c.`d")) } test("SPARK-7324 dropDuplicates") { @@ -696,54 +1239,10 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { checkAnswer( testData.dropDuplicates(Seq("value2")), Seq(Row(2, 1, 2), Row(1, 1, 1))) - } - - test("SPARK-7150 range api") { - // numSlice is greater than length - val res1 = sqlContext.range(0, 10, 1, 15).select("id") - assert(res1.count == 10) - assert(res1.agg(sum("id")).as("sumid").collect() === Seq(Row(45))) - - val res2 = sqlContext.range(3, 15, 3, 2).select("id") - assert(res2.count == 4) - assert(res2.agg(sum("id")).as("sumid").collect() === Seq(Row(30))) - - val res3 = sqlContext.range(1, -2).select("id") - assert(res3.count == 0) - - // start is positive, end is negative, step is negative - val res4 = sqlContext.range(1, -2, -2, 6).select("id") - assert(res4.count == 2) - assert(res4.agg(sum("id")).as("sumid").collect() === Seq(Row(0))) - - // start, end, step are negative - val res5 = sqlContext.range(-3, -8, -2, 1).select("id") - assert(res5.count == 3) - assert(res5.agg(sum("id")).as("sumid").collect() === Seq(Row(-15))) - // start, end are negative, step is positive - val res6 = sqlContext.range(-8, -4, 2, 1).select("id") - assert(res6.count == 2) - assert(res6.agg(sum("id")).as("sumid").collect() === Seq(Row(-14))) - - val res7 = sqlContext.range(-10, -9, -20, 1).select("id") - assert(res7.count == 0) - - val res8 = sqlContext.range(Long.MinValue, Long.MaxValue, Long.MaxValue, 100).select("id") - assert(res8.count == 3) - assert(res8.agg(sum("id")).as("sumid").collect() === Seq(Row(-3))) - - val res9 = sqlContext.range(Long.MaxValue, Long.MinValue, Long.MinValue, 100).select("id") - assert(res9.count == 2) - assert(res9.agg(sum("id")).as("sumid").collect() === Seq(Row(Long.MaxValue - 1))) - - // only end provided as argument - val res10 = sqlContext.range(10).select("id") - assert(res10.count == 10) - assert(res10.agg(sum("id")).as("sumid").collect() === Seq(Row(45))) - - val res11 = sqlContext.range(-1).select("id") - assert(res11.count == 0) + checkAnswer( + testData.dropDuplicates("key", "value1"), + Seq(Row(2, 1, 2), Row(1, 2, 1), Row(1, 1, 1), Row(2, 2, 2))) } test("SPARK-8621: support empty string column name") { @@ -780,8 +1279,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { Seq((1, 2, 3), (2, 3, 4), (3, 4, 5)).toDF("column1", "column2", "column1") .write.format("parquet").save("temp") } - assert(e.getMessage.contains("Duplicate column(s)")) - assert(e.getMessage.contains("parquet")) + assert(e.getMessage.contains("Found duplicate column(s) when inserting into")) assert(e.getMessage.contains("column1")) assert(!e.getMessage.contains("column2")) @@ -791,8 +1289,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { .toDF("column1", "column2", "column3", "column1", "column3") .write.format("json").save("temp") } - assert(f.getMessage.contains("Duplicate column(s)")) - assert(f.getMessage.contains("JSON")) + assert(f.getMessage.contains("Found duplicate column(s) when inserting into")) assert(f.getMessage.contains("column1")) assert(f.getMessage.contains("column3")) assert(!f.getMessage.contains("column2")) @@ -809,18 +1306,19 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { // pass case: parquet table (HadoopFsRelation) df.write.mode(SaveMode.Overwrite).parquet(tempParquetFile.getCanonicalPath) - val pdf = sqlContext.read.parquet(tempParquetFile.getCanonicalPath) - pdf.registerTempTable("parquet_base") + val pdf = spark.read.parquet(tempParquetFile.getCanonicalPath) + pdf.createOrReplaceTempView("parquet_base") + insertion.write.insertInto("parquet_base") // pass case: json table (InsertableRelation) df.write.mode(SaveMode.Overwrite).json(tempJsonFile.getCanonicalPath) - val jdf = sqlContext.read.json(tempJsonFile.getCanonicalPath) - jdf.registerTempTable("json_base") + val jdf = spark.read.json(tempJsonFile.getCanonicalPath) + jdf.createOrReplaceTempView("json_base") insertion.write.mode(SaveMode.Overwrite).insertInto("json_base") // error cases: insert into an RDD - df.registerTempTable("rdd_base") + df.createOrReplaceTempView("rdd_base") val e1 = intercept[AnalysisException] { insertion.write.insertInto("rdd_base") } @@ -828,14 +1326,14 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { // error case: insert into a logical plan that is not a LeafNode val indirectDS = pdf.select("_1").filter($"_1" > 5) - indirectDS.registerTempTable("indirect_ds") + indirectDS.createOrReplaceTempView("indirect_ds") val e2 = intercept[AnalysisException] { insertion.write.insertInto("indirect_ds") } assert(e2.getMessage.contains("Inserting into an RDD-based table is not allowed.")) // error case: insert into an OneRowRelation - new DataFrame(sqlContext, OneRowRelation).registerTempTable("one_row") + Dataset.ofRows(spark, OneRowRelation()).createOrReplaceTempView("one_row") val e3 = intercept[AnalysisException] { insertion.write.insertInto("one_row") } @@ -844,31 +1342,16 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("SPARK-8608: call `show` on local DataFrame with random columns should return same value") { - // Make sure we can pass this test for both codegen mode and interpreted mode. - withSQLConf(SQLConf.CODEGEN_ENABLED.key -> "true") { - val df = testData.select(rand(33)) - assert(df.showString(5) == df.showString(5)) - } - - withSQLConf(SQLConf.CODEGEN_ENABLED.key -> "false") { - val df = testData.select(rand(33)) - assert(df.showString(5) == df.showString(5)) - } + val df = testData.select(rand(33)) + assert(df.showString(5) == df.showString(5)) // We will reuse the same Expression object for LocalRelation. - val df = (1 to 10).map(Tuple1.apply).toDF().select(rand(33)) - assert(df.showString(5) == df.showString(5)) + val df1 = (1 to 10).map(Tuple1.apply).toDF().select(rand(33)) + assert(df1.showString(5) == df1.showString(5)) } test("SPARK-8609: local DataFrame with random columns should return same value after sort") { - // Make sure we can pass this test for both codegen mode and interpreted mode. - withSQLConf(SQLConf.CODEGEN_ENABLED.key -> "true") { - checkAnswer(testData.sort(rand(33)), testData.sort(rand(33))) - } - - withSQLConf(SQLConf.CODEGEN_ENABLED.key -> "false") { - checkAnswer(testData.sort(rand(33)), testData.sort(rand(33))) - } + checkAnswer(testData.sort(rand(33)), testData.sort(rand(33))) // We will reuse the same Expression object for LocalRelation. val df = (1 to 10).map(Tuple1.apply).toDF() @@ -886,9 +1369,14 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(expected === actual) } + test("Sorting columns are not in Filter and Project") { + checkAnswer( + upperCaseData.filter('N > 1).select('N).filter('N < 6).orderBy('L.asc), + Row(2) :: Row(3) :: Row(4) :: Row(5) :: Nil) + } + test("SPARK-9323: DataFrame.orderBy should support nested column name") { - val df = sqlContext.read.json(sparkContext.makeRDD( - """{"a": {"b": 1}}""" :: Nil)) + val df = spark.read.json(Seq("""{"a": {"b": 1}}""").toDS()) checkAnswer(df.orderBy("a.b"), Row(Row(1))) } @@ -916,25 +1404,32 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val dir2 = new File(dir, "dir2").getCanonicalPath df2.write.format("json").save(dir2) - checkAnswer(sqlContext.read.format("json").load(Array(dir1, dir2)), + checkAnswer(spark.read.format("json").load(dir1, dir2), Row(1, 22) :: Row(2, 23) :: Nil) - checkAnswer(sqlContext.read.format("json").load(dir1), + checkAnswer(spark.read.format("json").load(dir1), Row(1, 22) :: Nil) } } - test("SPARK-10034: Sort on Aggregate with aggregation expression named 'aggOrdering'") { + test("Alias uses internally generated names 'aggOrder' and 'havingCondition'") { val df = Seq(1 -> 2).toDF("i", "j") - val query = df.groupBy('i) - .agg(max('j).as("aggOrdering")) + val query1 = df.groupBy('i) + .agg(max('j).as("aggOrder")) .orderBy(sum('j)) - checkAnswer(query, Row(1, 2)) + checkAnswer(query1, Row(1, 2)) + + // In the plan, there are two attributes having the same name 'havingCondition' + // One is a user-provided alias name; another is an internally generated one. + val query2 = df.groupBy('i) + .agg(max('j).as("havingCondition")) + .where(sum('j) > 0) + .orderBy('havingCondition.asc) + checkAnswer(query2, Row(1, 2)) } test("SPARK-10316: respect non-deterministic expressions in PhysicalOperation") { - val input = sqlContext.read.json(sqlContext.sparkContext.makeRDD( - (1 to 10).map(i => s"""{"id": $i}"""))) + val input = spark.read.json((1 to 10).map(i => s"""{"id": $i}""").toDS()) val df = input.select($"id", rand(0).as('r)) df.as("a").join(df.filter($"r" < 0.5).as("b"), $"a.id" === $"b.id").collect().foreach { row => @@ -965,7 +1460,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } } - val union = df1.unionAll(df2) + val union = df1.union(df2) checkAnswer( union.filter('i < rand(7) * 10), expected(union) @@ -995,13 +1490,14 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { test("SPARK-10743: keep the name of expression if possible when do cast") { val df = (1 to 10).map(Tuple1.apply).toDF("i").as("src") assert(df.select($"src.i".cast(StringType)).columns.head === "i") + assert(df.select($"src.i".cast(StringType).cast(IntegerType)).columns.head === "i") } test("SPARK-11301: fix case sensitivity for filter on partitioned columns") { withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { withTempPath { path => Seq(2012 -> "a").toDF("year", "val").write.partitionBy("year").parquet(path.getAbsolutePath) - val df = sqlContext.read.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) checkAnswer(df.filter($"yEAr" > 2000).select($"val"), Row("a")) } } @@ -1013,14 +1509,12 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { private def verifyNonExchangingAgg(df: DataFrame) = { var atFirstAgg: Boolean = false df.queryExecution.executedPlan.foreach { - case agg: TungstenAggregate => { + case agg: HashAggregateExec => atFirstAgg = !atFirstAgg - } - case _ => { + case _ => if (atFirstAgg) { fail("Should not have operators between the two aggregations") } - } } } @@ -1030,13 +1524,12 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { private def verifyExchangingAgg(df: DataFrame) = { var atFirstAgg: Boolean = false df.queryExecution.executedPlan.foreach { - case agg: TungstenAggregate => { + case agg: HashAggregateExec => if (atFirstAgg) { fail("Should not have back to back Aggregates") } atFirstAgg = true - } - case e: Exchange => atFirstAgg = false + case e: ShuffleExchange => atFirstAgg = false case _ => } } @@ -1063,7 +1556,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { verifyExchangingAgg(testData.repartition($"key", $"value") .groupBy("key").count()) - val data = sqlContext.sparkContext.parallelize( + val data = spark.sparkContext.parallelize( (1 to 100).map(i => TestData2(i % 10, i))).toDF() // Distribute and order by. @@ -1071,17 +1564,20 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { // Walk each partition and verify that it is sorted descending and does not contain all // the values. df4.rdd.foreachPartition { p => - var previousValue: Int = -1 - var allSequential: Boolean = true - p.foreach { r => - val v: Int = r.getInt(1) - if (previousValue != -1) { - if (previousValue < v) throw new SparkException("Partition is not ordered.") - if (v + 1 != previousValue) allSequential = false + // Skip empty partition + if (p.hasNext) { + var previousValue: Int = -1 + var allSequential: Boolean = true + p.foreach { r => + val v: Int = r.getInt(1) + if (previousValue != -1) { + if (previousValue < v) throw new SparkException("Partition is not ordered.") + if (v + 1 != previousValue) allSequential = false + } + previousValue = v } - previousValue = v + if (allSequential) throw new SparkException("Partition should not be globally ordered") } - if (allSequential) throw new SparkException("Partition should not be globally ordered") } // Distribute and order by with multiple order bys @@ -1102,8 +1598,8 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } // Distribute into one partition and order by. This partition should contain all the values. - val df6 = data.repartition(1, $"a").sortWithinPartitions($"b".asc) - // Walk each partition and verify that it is sorted descending and not globally sorted. + val df6 = data.repartition(1, $"a").sortWithinPartitions("b") + // Walk each partition and verify that it is sorted ascending and not globally sorted. df6.rdd.foreachPartition { p => var previousValue: Int = -1 var allSequential: Boolean = true @@ -1124,8 +1620,423 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { withTempPath { path => val p = path.getAbsolutePath Seq(2012 -> "a").toDF("year", "val").write.partitionBy("yEAr").parquet(p) - checkAnswer(sqlContext.read.parquet(p).select("YeaR"), Row(2012)) + checkAnswer(spark.read.parquet(p).select("YeaR"), Row(2012)) } } } + + // This test case is to verify a bug when making a new instance of LogicalRDD. + test("SPARK-11633: LogicalRDD throws TreeNode Exception: Failed to Copy Node") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val rdd = sparkContext.makeRDD(Seq(Row(1, 3), Row(2, 1))) + val df = spark.createDataFrame( + rdd, + new StructType().add("f1", IntegerType).add("f2", IntegerType), + needsConversion = false).select($"F1", $"f2".as("f2")) + val df1 = df.as("a") + val df2 = df.as("b") + checkAnswer(df1.join(df2, $"a.f2" === $"b.f2"), Row(1, 3, 1, 3) :: Row(2, 1, 2, 1) :: Nil) + } + } + + test("SPARK-10656: completely support special chars") { + val df = Seq(1 -> "a").toDF("i_$.a", "d^'a.") + checkAnswer(df.select(df("*")), Row(1, "a")) + checkAnswer(df.withColumnRenamed("d^'a.", "a"), Row(1, "a")) + } + + test("SPARK-11725: correctly handle null inputs for ScalaUDF") { + val df = sparkContext.parallelize(Seq( + new java.lang.Integer(22) -> "John", + null.asInstanceOf[java.lang.Integer] -> "Lucy")).toDF("age", "name") + + // passing null into the UDF that could handle it + val boxedUDF = udf[java.lang.Integer, java.lang.Integer] { + (i: java.lang.Integer) => if (i == null) -10 else null + } + checkAnswer(df.select(boxedUDF($"age")), Row(null) :: Row(-10) :: Nil) + + spark.udf.register("boxedUDF", + (i: java.lang.Integer) => (if (i == null) -10 else null): java.lang.Integer) + checkAnswer(sql("select boxedUDF(null), boxedUDF(-1)"), Row(-10, null) :: Nil) + + val primitiveUDF = udf((i: Int) => i * 2) + checkAnswer(df.select(primitiveUDF($"age")), Row(44) :: Row(null) :: Nil) + } + + test("SPARK-12398 truncated toString") { + val df1 = Seq((1L, "row1")).toDF("id", "name") + assert(df1.toString() === "[id: bigint, name: string]") + + val df2 = Seq((1L, "c2", false)).toDF("c1", "c2", "c3") + assert(df2.toString === "[c1: bigint, c2: string ... 1 more field]") + + val df3 = Seq((1L, "c2", false, 10)).toDF("c1", "c2", "c3", "c4") + assert(df3.toString === "[c1: bigint, c2: string ... 2 more fields]") + + val df4 = Seq((1L, Tuple2(1L, "val"))).toDF("c1", "c2") + assert(df4.toString === "[c1: bigint, c2: struct<_1: bigint, _2: string>]") + + val df5 = Seq((1L, Tuple2(1L, "val"), 20.0)).toDF("c1", "c2", "c3") + assert(df5.toString === "[c1: bigint, c2: struct<_1: bigint, _2: string> ... 1 more field]") + + val df6 = Seq((1L, Tuple2(1L, "val"), 20.0, 1)).toDF("c1", "c2", "c3", "c4") + assert(df6.toString === "[c1: bigint, c2: struct<_1: bigint, _2: string> ... 2 more fields]") + + val df7 = Seq((1L, Tuple3(1L, "val", 2), 20.0, 1)).toDF("c1", "c2", "c3", "c4") + assert( + df7.toString === + "[c1: bigint, c2: struct<_1: bigint, _2: string ... 1 more field> ... 2 more fields]") + + val df8 = Seq((1L, Tuple7(1L, "val", 2, 3, 4, 5, 6), 20.0, 1)).toDF("c1", "c2", "c3", "c4") + assert( + df8.toString === + "[c1: bigint, c2: struct<_1: bigint, _2: string ... 5 more fields> ... 2 more fields]") + + val df9 = + Seq((1L, Tuple4(1L, Tuple4(1L, 2L, 3L, 4L), 2L, 3L), 20.0, 1)).toDF("c1", "c2", "c3", "c4") + assert( + df9.toString === + "[c1: bigint, c2: struct<_1: bigint," + + " _2: struct<_1: bigint," + + " _2: bigint ... 2 more fields> ... 2 more fields> ... 2 more fields]") + + } + + test("reuse exchange") { + withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "2") { + val df = spark.range(100).toDF() + val join = df.join(df, "id") + val plan = join.queryExecution.executedPlan + checkAnswer(join, df) + assert( + join.queryExecution.executedPlan.collect { case e: ShuffleExchange => true }.size === 1) + assert( + join.queryExecution.executedPlan.collect { case e: ReusedExchangeExec => true }.size === 1) + val broadcasted = broadcast(join) + val join2 = join.join(broadcasted, "id").join(broadcasted, "id") + checkAnswer(join2, df) + assert( + join2.queryExecution.executedPlan.collect { case e: ShuffleExchange => true }.size === 1) + assert( + join2.queryExecution.executedPlan + .collect { case e: BroadcastExchangeExec => true }.size === 1) + assert( + join2.queryExecution.executedPlan.collect { case e: ReusedExchangeExec => true }.size === 4) + } + } + + test("sameResult() on aggregate") { + val df = spark.range(100) + val agg1 = df.groupBy().count() + val agg2 = df.groupBy().count() + // two aggregates with different ExprId within them should have same result + assert(agg1.queryExecution.executedPlan.sameResult(agg2.queryExecution.executedPlan)) + val agg3 = df.groupBy().sum() + assert(!agg1.queryExecution.executedPlan.sameResult(agg3.queryExecution.executedPlan)) + val df2 = spark.range(101) + val agg4 = df2.groupBy().count() + assert(!agg1.queryExecution.executedPlan.sameResult(agg4.queryExecution.executedPlan)) + } + + test("SPARK-12512: support `.` in column name for withColumn()") { + val df = Seq("a" -> "b").toDF("col.a", "col.b") + checkAnswer(df.select(df("*")), Row("a", "b")) + checkAnswer(df.withColumn("col.a", lit("c")), Row("c", "b")) + checkAnswer(df.withColumn("col.c", lit("c")), Row("a", "b", "c")) + } + + test("SPARK-12841: cast in filter") { + checkAnswer( + Seq(1 -> "a").toDF("i", "j").filter($"i".cast(StringType) === "1"), + Row(1, "a")) + } + + test("SPARK-12982: Add table name validation in temp table registration") { + val df = Seq("foo", "bar").map(Tuple1.apply).toDF("col") + // invalid table names + Seq("11111", "t~", "#$@sum", "table!#").foreach { name => + val m = intercept[AnalysisException](df.createOrReplaceTempView(name)).getMessage + assert(m.contains(s"Invalid view name: $name")) + } + + // valid table names + Seq("table1", "`11111`", "`t~`", "`#$@sum`", "`table!#`").foreach { name => + df.createOrReplaceTempView(name) + } + } + + test("assertAnalyzed shouldn't replace original stack trace") { + val e = intercept[AnalysisException] { + spark.range(1).select('id as 'a, 'id as 'b).groupBy('a).agg('b) + } + + assert(e.getStackTrace.head.getClassName != classOf[QueryExecution].getName) + } + + test("SPARK-13774: Check error message for non existent path without globbed paths") { + val uuid = UUID.randomUUID().toString + val baseDir = Utils.createTempDir() + try { + val e = intercept[AnalysisException] { + spark.read.format("csv").load( + new File(baseDir, "file").getAbsolutePath, + new File(baseDir, "file2").getAbsolutePath, + new File(uuid, "file3").getAbsolutePath, + uuid).rdd + } + assert(e.getMessage.startsWith("Path does not exist")) + } finally { + + } + + } + + test("SPARK-13774: Check error message for not existent globbed paths") { + // Non-existent initial path component: + val nonExistentBasePath = "/" + UUID.randomUUID().toString + assert(!new File(nonExistentBasePath).exists()) + val e = intercept[AnalysisException] { + spark.read.format("text").load(s"$nonExistentBasePath/*") + } + assert(e.getMessage.startsWith("Path does not exist")) + + // Existent initial path component, but no matching files: + val baseDir = Utils.createTempDir() + val childDir = Utils.createTempDir(baseDir.getAbsolutePath) + assert(childDir.exists()) + try { + val e1 = intercept[AnalysisException] { + spark.read.json(s"${baseDir.getAbsolutePath}/*/*-xyz.json").rdd + } + assert(e1.getMessage.startsWith("Path does not exist")) + } finally { + Utils.deleteRecursively(baseDir) + } + } + + test("SPARK-15230: distinct() does not handle column name with dot properly") { + val df = Seq(1, 1, 2).toDF("column.with.dot") + checkAnswer(df.distinct(), Row(1) :: Row(2) :: Nil) + } + + test("SPARK-16181: outer join with isNull filter") { + val left = Seq("x").toDF("col") + val right = Seq("y").toDF("col").withColumn("new", lit(true)) + val joined = left.join(right, left("col") === right("col"), "left_outer") + + checkAnswer(joined, Row("x", null, null)) + checkAnswer(joined.filter($"new".isNull), Row("x", null, null)) + } + + test("SPARK-16664: persist with more than 200 columns") { + val size = 201L + val rdd = sparkContext.makeRDD(Seq(Row.fromSeq(Seq.range(0, size)))) + val schemas = List.range(0, size).map(a => StructField("name" + a, LongType, true)) + val df = spark.createDataFrame(rdd, StructType(schemas), false) + assert(df.persist.take(1).apply(0).toSeq(100).asInstanceOf[Long] == 100) + } + + test("SPARK-17409: Do Not Optimize Query in CTAS (Data source tables) More Than Once") { + withTable("bar") { + withTempView("foo") { + withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "json") { + sql("select 0 as id").createOrReplaceTempView("foo") + val df = sql("select * from foo group by id") + // If we optimize the query in CTAS more than once, the following saveAsTable will fail + // with the error: `GROUP BY position 0 is not in select list (valid range is [1, 1])` + df.write.mode("overwrite").saveAsTable("bar") + checkAnswer(spark.table("bar"), Row(0) :: Nil) + val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier("bar")) + assert(tableMetadata.provider == Some("json"), + "the expected table is a data source table using json") + } + } + } + } + + test("copy results for sampling with replacement") { + val df = Seq((1, 0), (2, 0), (3, 0)).toDF("a", "b") + val sampleDf = df.sample(true, 2.00) + val d = sampleDf.withColumn("c", monotonically_increasing_id).select($"c").collect + assert(d.size == d.distinct.size) + } + + private def verifyNullabilityInFilterExec( + df: DataFrame, + expr: String, + expectedNonNullableColumns: Seq[String]): Unit = { + val dfWithFilter = df.where(s"isnotnull($expr)").selectExpr(expr) + // In the logical plan, all the output columns of input dataframe are nullable + dfWithFilter.queryExecution.optimizedPlan.collect { + case e: Filter => assert(e.output.forall(_.nullable)) + } + + dfWithFilter.queryExecution.executedPlan.collect { + // When the child expression in isnotnull is null-intolerant (i.e. any null input will + // result in null output), the involved columns are converted to not nullable; + // otherwise, no change should be made. + case e: FilterExec => + assert(e.output.forall { o => + if (expectedNonNullableColumns.contains(o.name)) !o.nullable else o.nullable + }) + } + } + + test("SPARK-17957: no change on nullability in FilterExec output") { + val df = sparkContext.parallelize(Seq( + null.asInstanceOf[java.lang.Integer] -> new java.lang.Integer(3), + new java.lang.Integer(1) -> null.asInstanceOf[java.lang.Integer], + new java.lang.Integer(2) -> new java.lang.Integer(4))).toDF() + + verifyNullabilityInFilterExec(df, + expr = "Rand()", expectedNonNullableColumns = Seq.empty[String]) + verifyNullabilityInFilterExec(df, + expr = "coalesce(_1, _2)", expectedNonNullableColumns = Seq.empty[String]) + verifyNullabilityInFilterExec(df, + expr = "coalesce(_1, 0) + Rand()", expectedNonNullableColumns = Seq.empty[String]) + verifyNullabilityInFilterExec(df, + expr = "cast(coalesce(cast(coalesce(_1, _2) as double), 0.0) as int)", + expectedNonNullableColumns = Seq.empty[String]) + } + + test("SPARK-17957: set nullability to false in FilterExec output") { + val df = sparkContext.parallelize(Seq( + null.asInstanceOf[java.lang.Integer] -> new java.lang.Integer(3), + new java.lang.Integer(1) -> null.asInstanceOf[java.lang.Integer], + new java.lang.Integer(2) -> new java.lang.Integer(4))).toDF() + + verifyNullabilityInFilterExec(df, + expr = "_1 + _2 * 3", expectedNonNullableColumns = Seq("_1", "_2")) + verifyNullabilityInFilterExec(df, + expr = "_1 + _2", expectedNonNullableColumns = Seq("_1", "_2")) + verifyNullabilityInFilterExec(df, + expr = "_1", expectedNonNullableColumns = Seq("_1")) + // `constructIsNotNullConstraints` infers the IsNotNull(_2) from IsNotNull(_2 + Rand()) + // Thus, we are able to set nullability of _2 to false. + // If IsNotNull(_2) is not given from `constructIsNotNullConstraints`, the impl of + // isNullIntolerant in `FilterExec` needs an update for more advanced inference. + verifyNullabilityInFilterExec(df, + expr = "_2 + Rand()", expectedNonNullableColumns = Seq("_2")) + verifyNullabilityInFilterExec(df, + expr = "_2 * 3 + coalesce(_1, 0)", expectedNonNullableColumns = Seq("_2")) + verifyNullabilityInFilterExec(df, + expr = "cast((_1 + _2) as boolean)", expectedNonNullableColumns = Seq("_1", "_2")) + } + + test("SPARK-17897: Fixed IsNotNull Constraint Inference Rule") { + val data = Seq[java.lang.Integer](1, null).toDF("key") + checkAnswer(data.filter(!$"key".isNotNull), Row(null)) + checkAnswer(data.filter(!(- $"key").isNotNull), Row(null)) + } + + test("SPARK-17957: outer join + na.fill") { + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val df1 = Seq((1, 2), (2, 3)).toDF("a", "b") + val df2 = Seq((2, 5), (3, 4)).toDF("a", "c") + val joinedDf = df1.join(df2, Seq("a"), "outer").na.fill(0) + val df3 = Seq((3, 1)).toDF("a", "d") + checkAnswer(joinedDf.join(df3, "a"), Row(3, 0, 4, 1)) + } + } + + test("SPARK-17123: Performing set operations that combine non-scala native types") { + val dates = Seq( + (new Date(0), BigDecimal.valueOf(1), new Timestamp(2)), + (new Date(3), BigDecimal.valueOf(4), new Timestamp(5)) + ).toDF("date", "timestamp", "decimal") + + val widenTypedRows = Seq( + (new Timestamp(2), 10.5D, "string") + ).toDF("date", "timestamp", "decimal") + + dates.union(widenTypedRows).collect() + dates.except(widenTypedRows).collect() + dates.intersect(widenTypedRows).collect() + } + + test("SPARK-18070 binary operator should not consider nullability when comparing input types") { + val rows = Seq(Row(Seq(1), Seq(1))) + val schema = new StructType() + .add("array1", ArrayType(IntegerType)) + .add("array2", ArrayType(IntegerType, containsNull = false)) + val df = spark.createDataFrame(spark.sparkContext.makeRDD(rows), schema) + assert(df.filter($"array1" === $"array2").count() == 1) + } + + test("SPARK-17913: compare long and string type column may return confusing result") { + val df = Seq(123L -> "123", 19157170390056973L -> "19157170390056971").toDF("i", "j") + checkAnswer(df.select($"i" === $"j"), Row(true) :: Row(false) :: Nil) + } + + test("SPARK-19691 Calculating percentile of decimal column fails with ClassCastException") { + val df = spark.range(1).selectExpr("CAST(id as DECIMAL) as x").selectExpr("percentile(x, 0.5)") + checkAnswer(df, Row(BigDecimal(0)) :: Nil) + } + + test("SPARK-19893: cannot run set operations with map type") { + val df = spark.range(1).select(map(lit("key"), $"id").as("m")) + val e = intercept[AnalysisException](df.intersect(df)) + assert(e.message.contains( + "Cannot have map type columns in DataFrame which calls set operations")) + val e2 = intercept[AnalysisException](df.except(df)) + assert(e2.message.contains( + "Cannot have map type columns in DataFrame which calls set operations")) + val e3 = intercept[AnalysisException](df.distinct()) + assert(e3.message.contains( + "Cannot have map type columns in DataFrame which calls set operations")) + withTempView("v") { + df.createOrReplaceTempView("v") + val e4 = intercept[AnalysisException](sql("SELECT DISTINCT m FROM v")) + assert(e4.message.contains( + "Cannot have map type columns in DataFrame which calls set operations")) + } + } + + test("SPARK-20359: catalyst outer join optimization should not throw npe") { + val df1 = Seq("a", "b", "c").toDF("x") + .withColumn("y", udf{ (x: String) => x.substring(0, 1) + "!" }.apply($"x")) + val df2 = Seq("a", "b").toDF("x1") + df1 + .join(df2, df1("x") === df2("x1"), "left_outer") + .filter($"x1".isNotNull || !$"y".isin("a!")) + .count + } + + testQuietly("SPARK-19372: Filter can be executed w/o generated code due to JVM code size limit") { + val N = 400 + val rows = Seq(Row.fromSeq(Seq.fill(N)("string"))) + val schema = StructType(Seq.tabulate(N)(i => StructField(s"_c$i", StringType))) + val df = spark.createDataFrame(spark.sparkContext.makeRDD(rows), schema) + + val filter = (0 until N) + .foldLeft(lit(false))((e, index) => e.or(df.col(df.columns(index)) =!= "string")) + + withSQLConf(SQLConf.CODEGEN_FALLBACK.key -> "true") { + df.filter(filter).count() + } + + withSQLConf(SQLConf.CODEGEN_FALLBACK.key -> "false") { + val e = intercept[SparkException] { + df.filter(filter).count() + }.getMessage + assert(e.contains("grows beyond 64 KB")) + } + } + + test("SPARK-20897: cached self-join should not fail") { + // force to plan sort merge join + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0") { + val df = Seq(1 -> "a").toDF("i", "j") + val df1 = df.as("t1") + val df2 = df.as("t2") + assert(df1.join(df2, $"t1.i" === $"t2.i").cache().count() == 1) + } + } + + test("order-by ordinal.") { + checkAnswer( + testData2.select(lit(7), 'a, 'b).orderBy(lit(1), lit(2), lit(3)), + Seq(Row(7, 1, 1), Row(7, 1, 2), Row(7, 2, 1), Row(7, 2, 2), Row(7, 3, 1), Row(7, 3, 2))) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala new file mode 100644 index 000000000000..6fe356877c26 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala @@ -0,0 +1,312 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.BeforeAndAfterEach + +import org.apache.spark.sql.catalyst.plans.logical.Expand +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StringType + +class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { + + import testImplicits._ + + test("simple tumbling window with record at window start") { + val df = Seq( + ("2016-03-27 19:39:30", 1, "a")).toDF("time", "value", "id") + + checkAnswer( + df.groupBy(window($"time", "10 seconds")) + .agg(count("*").as("counts")) + .orderBy($"window.start".asc) + .select($"window.start".cast("string"), $"window.end".cast("string"), $"counts"), + Seq( + Row("2016-03-27 19:39:30", "2016-03-27 19:39:40", 1) + ) + ) + } + + test("tumbling window groupBy statement") { + val df = Seq( + ("2016-03-27 19:39:34", 1, "a"), + ("2016-03-27 19:39:56", 2, "a"), + ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") + + checkAnswer( + df.groupBy(window($"time", "10 seconds")) + .agg(count("*").as("counts")) + .orderBy($"window.start".asc) + .select("counts"), + Seq(Row(1), Row(1), Row(1)) + ) + } + + test("tumbling window groupBy statement with startTime") { + val df = Seq( + ("2016-03-27 19:39:34", 1, "a"), + ("2016-03-27 19:39:56", 2, "a"), + ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") + + checkAnswer( + df.groupBy(window($"time", "10 seconds", "10 seconds", "5 seconds"), $"id") + .agg(count("*").as("counts")) + .orderBy($"window.start".asc) + .select("counts"), + Seq(Row(1), Row(1), Row(1))) + } + + test("tumbling window with multi-column projection") { + val df = Seq( + ("2016-03-27 19:39:34", 1, "a"), + ("2016-03-27 19:39:56", 2, "a"), + ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") + .select(window($"time", "10 seconds"), $"value") + .orderBy($"window.start".asc) + .select($"window.start".cast("string"), $"window.end".cast("string"), $"value") + + val expands = df.queryExecution.optimizedPlan.find(_.isInstanceOf[Expand]) + assert(expands.isEmpty, "Tumbling windows shouldn't require expand") + + checkAnswer( + df, + Seq( + Row("2016-03-27 19:39:20", "2016-03-27 19:39:30", 4), + Row("2016-03-27 19:39:30", "2016-03-27 19:39:40", 1), + Row("2016-03-27 19:39:50", "2016-03-27 19:40:00", 2) + ) + ) + } + + test("sliding window grouping") { + val df = Seq( + ("2016-03-27 19:39:34", 1, "a"), + ("2016-03-27 19:39:56", 2, "a"), + ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") + + checkAnswer( + df.groupBy(window($"time", "10 seconds", "3 seconds", "0 second")) + .agg(count("*").as("counts")) + .orderBy($"window.start".asc) + .select($"window.start".cast("string"), $"window.end".cast("string"), $"counts"), + // 2016-03-27 19:39:27 UTC -> 4 bins + // 2016-03-27 19:39:34 UTC -> 3 bins + // 2016-03-27 19:39:56 UTC -> 3 bins + Seq( + Row("2016-03-27 19:39:18", "2016-03-27 19:39:28", 1), + Row("2016-03-27 19:39:21", "2016-03-27 19:39:31", 1), + Row("2016-03-27 19:39:24", "2016-03-27 19:39:34", 1), + Row("2016-03-27 19:39:27", "2016-03-27 19:39:37", 2), + Row("2016-03-27 19:39:30", "2016-03-27 19:39:40", 1), + Row("2016-03-27 19:39:33", "2016-03-27 19:39:43", 1), + Row("2016-03-27 19:39:48", "2016-03-27 19:39:58", 1), + Row("2016-03-27 19:39:51", "2016-03-27 19:40:01", 1), + Row("2016-03-27 19:39:54", "2016-03-27 19:40:04", 1)) + ) + } + + test("sliding window projection") { + val df = Seq( + ("2016-03-27 19:39:34", 1, "a"), + ("2016-03-27 19:39:56", 2, "a"), + ("2016-03-27 19:39:27", 4, "b")).toDF("time", "value", "id") + .select(window($"time", "10 seconds", "3 seconds", "0 second"), $"value") + .orderBy($"window.start".asc, $"value".desc).select("value") + + val expands = df.queryExecution.optimizedPlan.find(_.isInstanceOf[Expand]) + assert(expands.nonEmpty, "Sliding windows require expand") + + checkAnswer( + df, + // 2016-03-27 19:39:27 UTC -> 4 bins + // 2016-03-27 19:39:34 UTC -> 3 bins + // 2016-03-27 19:39:56 UTC -> 3 bins + Seq(Row(4), Row(4), Row(4), Row(4), Row(1), Row(1), Row(1), Row(2), Row(2), Row(2)) + ) + } + + test("windowing combined with explode expression") { + val df = Seq( + ("2016-03-27 19:39:34", 1, Seq("a", "b")), + ("2016-03-27 19:39:56", 2, Seq("a", "c", "d"))).toDF("time", "value", "ids") + + checkAnswer( + df.select(window($"time", "10 seconds"), $"value", explode($"ids")) + .orderBy($"window.start".asc).select("value"), + // first window exploded to two rows for "a", and "b", second window exploded to 3 rows + Seq(Row(1), Row(1), Row(2), Row(2), Row(2)) + ) + } + + test("null timestamps") { + val df = Seq( + ("2016-03-27 09:00:05", 1), + ("2016-03-27 09:00:32", 2), + (null, 3), + (null, 4)).toDF("time", "value") + + checkDataset( + df.select(window($"time", "10 seconds"), $"value") + .orderBy($"window.start".asc) + .select("value") + .as[Int], + 1, 2) // null columns are dropped + } + + test("time window joins") { + val df = Seq( + ("2016-03-27 09:00:05", 1), + ("2016-03-27 09:00:32", 2), + (null, 3), + (null, 4)).toDF("time", "value") + + val df2 = Seq( + ("2016-03-27 09:00:02", 3), + ("2016-03-27 09:00:35", 6)).toDF("time", "othervalue") + + checkAnswer( + df.select(window($"time", "10 seconds"), $"value").join( + df2.select(window($"time", "10 seconds"), $"othervalue"), Seq("window")) + .groupBy("window") + .agg((sum("value") + sum("othervalue")).as("total")) + .orderBy($"window.start".asc).select("total"), + Seq(Row(4), Row(8))) + } + + test("negative timestamps") { + val df4 = Seq( + ("1970-01-01 00:00:02", 1), + ("1970-01-01 00:00:12", 2)).toDF("time", "value") + checkAnswer( + df4.select(window($"time", "10 seconds", "10 seconds", "5 seconds"), $"value") + .orderBy($"window.start".asc) + .select($"window.start".cast(StringType), $"window.end".cast(StringType), $"value"), + Seq( + Row("1969-12-31 23:59:55", "1970-01-01 00:00:05", 1), + Row("1970-01-01 00:00:05", "1970-01-01 00:00:15", 2)) + ) + } + + test("multiple time windows in a single operator throws nice exception") { + val df = Seq( + ("2016-03-27 09:00:02", 3), + ("2016-03-27 09:00:35", 6)).toDF("time", "value") + val e = intercept[AnalysisException] { + df.select(window($"time", "10 second"), window($"time", "15 second")).collect() + } + assert(e.getMessage.contains( + "Multiple time window expressions would result in a cartesian product")) + } + + test("aliased windows") { + val df = Seq( + ("2016-03-27 19:39:34", 1, Seq("a", "b")), + ("2016-03-27 19:39:56", 2, Seq("a", "c", "d"))).toDF("time", "value", "ids") + + checkAnswer( + df.select(window($"time", "10 seconds").as("time_window"), $"value") + .orderBy($"time_window.start".asc) + .select("value"), + Seq(Row(1), Row(2)) + ) + } + + test("millisecond precision sliding windows") { + val df = Seq( + ("2016-03-27 09:00:00.41", 3), + ("2016-03-27 09:00:00.62", 6), + ("2016-03-27 09:00:00.715", 8)).toDF("time", "value") + checkAnswer( + df.groupBy(window($"time", "200 milliseconds", "40 milliseconds", "0 milliseconds")) + .agg(count("*").as("counts")) + .orderBy($"window.start".asc) + .select($"window.start".cast(StringType), $"window.end".cast(StringType), $"counts"), + Seq( + Row("2016-03-27 09:00:00.24", "2016-03-27 09:00:00.44", 1), + Row("2016-03-27 09:00:00.28", "2016-03-27 09:00:00.48", 1), + Row("2016-03-27 09:00:00.32", "2016-03-27 09:00:00.52", 1), + Row("2016-03-27 09:00:00.36", "2016-03-27 09:00:00.56", 1), + Row("2016-03-27 09:00:00.4", "2016-03-27 09:00:00.6", 1), + Row("2016-03-27 09:00:00.44", "2016-03-27 09:00:00.64", 1), + Row("2016-03-27 09:00:00.48", "2016-03-27 09:00:00.68", 1), + Row("2016-03-27 09:00:00.52", "2016-03-27 09:00:00.72", 2), + Row("2016-03-27 09:00:00.56", "2016-03-27 09:00:00.76", 2), + Row("2016-03-27 09:00:00.6", "2016-03-27 09:00:00.8", 2), + Row("2016-03-27 09:00:00.64", "2016-03-27 09:00:00.84", 1), + Row("2016-03-27 09:00:00.68", "2016-03-27 09:00:00.88", 1)) + ) + } + + private def withTempTable(f: String => Unit): Unit = { + val tableName = "temp" + Seq( + ("2016-03-27 19:39:34", 1), + ("2016-03-27 19:39:56", 2), + ("2016-03-27 19:39:27", 4)).toDF("time", "value").createOrReplaceTempView(tableName) + try { + f(tableName) + } finally { + spark.catalog.dropTempView(tableName) + } + } + + test("time window in SQL with single string expression") { + withTempTable { table => + checkAnswer( + spark.sql(s"""select window(time, "10 seconds"), value from $table""") + .select($"window.start".cast(StringType), $"window.end".cast(StringType), $"value"), + Seq( + Row("2016-03-27 19:39:20", "2016-03-27 19:39:30", 4), + Row("2016-03-27 19:39:30", "2016-03-27 19:39:40", 1), + Row("2016-03-27 19:39:50", "2016-03-27 19:40:00", 2) + ) + ) + } + } + + test("time window in SQL with two expressions") { + withTempTable { table => + checkAnswer( + spark.sql( + s"""select window(time, "10 seconds", 10000000), value from $table""") + .select($"window.start".cast(StringType), $"window.end".cast(StringType), $"value"), + Seq( + Row("2016-03-27 19:39:20", "2016-03-27 19:39:30", 4), + Row("2016-03-27 19:39:30", "2016-03-27 19:39:40", 1), + Row("2016-03-27 19:39:50", "2016-03-27 19:40:00", 2) + ) + ) + } + } + + test("time window in SQL with three expressions") { + withTempTable { table => + checkAnswer( + spark.sql( + s"""select window(time, "10 seconds", 10000000, "5 seconds"), value from $table""") + .select($"window.start".cast(StringType), $"window.end".cast(StringType), $"value"), + Seq( + Row("2016-03-27 19:39:25", "2016-03-27 19:39:35", 1), + Row("2016-03-27 19:39:25", "2016-03-27 19:39:35", 4), + Row("2016-03-27 19:39:55", "2016-03-27 19:40:05", 2) + ) + ) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala index 7ae12a7895f7..fe6ba83b4cbf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTungstenSuite.scala @@ -31,52 +31,46 @@ class DataFrameTungstenSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("test simple types") { - withSQLConf(SQLConf.UNSAFE_ENABLED.key -> "true") { - val df = sparkContext.parallelize(Seq((1, 2))).toDF("a", "b") - assert(df.select(struct("a", "b")).first().getStruct(0) === Row(1, 2)) - } + val df = sparkContext.parallelize(Seq((1, 2))).toDF("a", "b") + assert(df.select(struct("a", "b")).first().getStruct(0) === Row(1, 2)) } test("test struct type") { - withSQLConf(SQLConf.UNSAFE_ENABLED.key -> "true") { - val struct = Row(1, 2L, 3.0F, 3.0) - val data = sparkContext.parallelize(Seq(Row(1, struct))) + val struct = Row(1, 2L, 3.0F, 3.0) + val data = sparkContext.parallelize(Seq(Row(1, struct))) - val schema = new StructType() - .add("a", IntegerType) - .add("b", - new StructType() - .add("b1", IntegerType) - .add("b2", LongType) - .add("b3", FloatType) - .add("b4", DoubleType)) + val schema = new StructType() + .add("a", IntegerType) + .add("b", + new StructType() + .add("b1", IntegerType) + .add("b2", LongType) + .add("b3", FloatType) + .add("b4", DoubleType)) - val df = sqlContext.createDataFrame(data, schema) - assert(df.select("b").first() === Row(struct)) - } + val df = spark.createDataFrame(data, schema) + assert(df.select("b").first() === Row(struct)) } test("test nested struct type") { - withSQLConf(SQLConf.UNSAFE_ENABLED.key -> "true") { - val innerStruct = Row(1, "abcd") - val outerStruct = Row(1, 2L, 3.0F, 3.0, innerStruct, "efg") - val data = sparkContext.parallelize(Seq(Row(1, outerStruct))) + val innerStruct = Row(1, "abcd") + val outerStruct = Row(1, 2L, 3.0F, 3.0, innerStruct, "efg") + val data = sparkContext.parallelize(Seq(Row(1, outerStruct))) - val schema = new StructType() - .add("a", IntegerType) - .add("b", - new StructType() - .add("b1", IntegerType) - .add("b2", LongType) - .add("b3", FloatType) - .add("b4", DoubleType) - .add("b5", new StructType() - .add("b5a", IntegerType) - .add("b5b", StringType)) - .add("b6", StringType)) + val schema = new StructType() + .add("a", IntegerType) + .add("b", + new StructType() + .add("b1", IntegerType) + .add("b2", LongType) + .add("b3", FloatType) + .add("b4", DoubleType) + .add("b5", new StructType() + .add("b5a", IntegerType) + .add("b5b", StringType)) + .add("b6", StringType)) - val df = sqlContext.createDataFrame(data, schema) - assert(df.select("b").first() === Row(outerStruct)) - } + val df = spark.createDataFrame(data, schema) + assert(df.select("b").first() === Row(outerStruct)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala new file mode 100644 index 000000000000..ea725af8d1ad --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -0,0 +1,564 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.sql.{Date, Timestamp} + +import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction, Window} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval + +/** + * Window function testing for DataFrame API. + */ +class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("reuse window partitionBy") { + val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") + val w = Window.partitionBy("key").orderBy("value") + + checkAnswer( + df.select( + lead("key", 1).over(w), + lead("value", 1).over(w)), + Row(1, "1") :: Row(2, "2") :: Row(null, null) :: Row(null, null) :: Nil) + } + + test("reuse window orderBy") { + val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") + val w = Window.orderBy("value").partitionBy("key") + + checkAnswer( + df.select( + lead("key", 1).over(w), + lead("value", 1).over(w)), + Row(1, "1") :: Row(2, "2") :: Row(null, null) :: Row(null, null) :: Nil) + } + + test("Window.rowsBetween") { + val df = Seq(("one", 1), ("two", 2)).toDF("key", "value") + // Running (cumulative) sum + checkAnswer( + df.select('key, sum("value").over( + Window.rowsBetween(Window.unboundedPreceding, Window.currentRow))), + Row("one", 1) :: Row("two", 3) :: Nil + ) + } + + test("lead") { + val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") + df.createOrReplaceTempView("window_table") + + checkAnswer( + df.select( + lead("value", 1).over(Window.partitionBy($"key").orderBy($"value"))), + Row("1") :: Row(null) :: Row("2") :: Row(null) :: Nil) + } + + test("lag") { + val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") + df.createOrReplaceTempView("window_table") + + checkAnswer( + df.select( + lag("value", 1).over(Window.partitionBy($"key").orderBy($"value"))), + Row(null) :: Row("1") :: Row(null) :: Row("2") :: Nil) + } + + test("lead with default value") { + val df = Seq((1, "1"), (1, "1"), (2, "2"), (1, "1"), + (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select( + lead("value", 2, "n/a").over(Window.partitionBy("key").orderBy("value"))), + Seq(Row("1"), Row("1"), Row("n/a"), Row("n/a"), Row("2"), Row("n/a"), Row("n/a"))) + } + + test("lag with default value") { + val df = Seq((1, "1"), (1, "1"), (2, "2"), (1, "1"), + (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select( + lag("value", 2, "n/a").over(Window.partitionBy($"key").orderBy($"value"))), + Seq(Row("n/a"), Row("n/a"), Row("1"), Row("1"), Row("n/a"), Row("n/a"), Row("2"))) + } + + test("rank functions in unspecific window") { + val df = Seq((1, "1"), (2, "2"), (1, "2"), (2, "2")).toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select( + $"key", + max("key").over(Window.partitionBy("value").orderBy("key")), + min("key").over(Window.partitionBy("value").orderBy("key")), + mean("key").over(Window.partitionBy("value").orderBy("key")), + count("key").over(Window.partitionBy("value").orderBy("key")), + sum("key").over(Window.partitionBy("value").orderBy("key")), + ntile(2).over(Window.partitionBy("value").orderBy("key")), + row_number().over(Window.partitionBy("value").orderBy("key")), + dense_rank().over(Window.partitionBy("value").orderBy("key")), + rank().over(Window.partitionBy("value").orderBy("key")), + cume_dist().over(Window.partitionBy("value").orderBy("key")), + percent_rank().over(Window.partitionBy("value").orderBy("key"))), + Row(1, 1, 1, 1.0d, 1, 1, 1, 1, 1, 1, 1.0d, 0.0d) :: + Row(1, 1, 1, 1.0d, 1, 1, 1, 1, 1, 1, 1.0d / 3.0d, 0.0d) :: + Row(2, 2, 1, 5.0d / 3.0d, 3, 5, 1, 2, 2, 2, 1.0d, 0.5d) :: + Row(2, 2, 1, 5.0d / 3.0d, 3, 5, 2, 3, 2, 2, 1.0d, 0.5d) :: Nil) + } + + test("window function should fail if order by clause is not specified") { + val df = Seq((1, "1"), (2, "2"), (1, "2"), (2, "2")).toDF("key", "value") + val e = intercept[AnalysisException]( + // Here we missed .orderBy("key")! + df.select(row_number().over(Window.partitionBy("value"))).collect()) + assert(e.message.contains("requires window to be ordered")) + } + + test("aggregation and rows between") { + val df = Seq((1, "1"), (2, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select( + avg("key").over(Window.partitionBy($"value").orderBy($"key").rowsBetween(-1, 2))), + Seq(Row(4.0d / 3.0d), Row(4.0d / 3.0d), Row(3.0d / 2.0d), Row(2.0d), Row(2.0d))) + } + + test("aggregation and range between") { + val df = Seq((1, "1"), (1, "1"), (3, "1"), (2, "2"), (2, "1"), (2, "2")).toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select( + avg("key").over(Window.partitionBy($"value").orderBy($"key").rangeBetween(-1, 1))), + Seq(Row(4.0d / 3.0d), Row(4.0d / 3.0d), Row(7.0d / 4.0d), Row(5.0d / 2.0d), + Row(2.0d), Row(2.0d))) + } + + test("row between should accept integer values as boundary") { + val df = Seq((1L, "1"), (1L, "1"), (2147483650L, "1"), + (3L, "2"), (2L, "1"), (2147483650L, "2")) + .toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select( + $"key", + count("key").over( + Window.partitionBy($"value").orderBy($"key").rowsBetween(0, 2147483647))), + Seq(Row(1, 3), Row(1, 4), Row(2, 2), Row(3, 2), Row(2147483650L, 1), Row(2147483650L, 1)) + ) + + val e = intercept[AnalysisException]( + df.select( + $"key", + count("key").over( + Window.partitionBy($"value").orderBy($"key").rowsBetween(0, 2147483648L)))) + assert(e.message.contains("Boundary end is not a valid integer: 2147483648")) + } + + test("range between should accept int/long values as boundary") { + val df = Seq((1L, "1"), (1L, "1"), (2147483650L, "1"), + (3L, "2"), (2L, "1"), (2147483650L, "2")) + .toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select( + $"key", + count("key").over( + Window.partitionBy($"value").orderBy($"key").rangeBetween(0, 2147483648L))), + Seq(Row(1, 3), Row(1, 3), Row(2, 2), Row(3, 2), Row(2147483650L, 1), Row(2147483650L, 1)) + ) + checkAnswer( + df.select( + $"key", + count("key").over( + Window.partitionBy($"value").orderBy($"key").rangeBetween(-2147483649L, 0))), + Seq(Row(1, 2), Row(1, 2), Row(2, 3), Row(2147483650L, 2), Row(2147483650L, 4), Row(3, 1)) + ) + + def dt(date: String): Date = Date.valueOf(date) + + val df2 = Seq((dt("2017-08-01"), "1"), (dt("2017-08-01"), "1"), (dt("2020-12-31"), "1"), + (dt("2017-08-03"), "2"), (dt("2017-08-02"), "1"), (dt("2020-12-31"), "2")) + .toDF("key", "value") + checkAnswer( + df2.select( + $"key", + count("key").over( + Window.partitionBy($"value").orderBy($"key").rangeBetween(lit(0), lit(2)))), + Seq(Row(dt("2017-08-01"), 3), Row(dt("2017-08-01"), 3), Row(dt("2020-12-31"), 1), + Row(dt("2017-08-03"), 1), Row(dt("2017-08-02"), 1), Row(dt("2020-12-31"), 1)) + ) + } + + test("range between should accept double values as boundary") { + val df = Seq((1.0D, "1"), (1.0D, "1"), (100.001D, "1"), + (3.3D, "2"), (2.02D, "1"), (100.001D, "2")) + .toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select( + $"key", + count("key").over( + Window.partitionBy($"value").orderBy($"key") + .rangeBetween(currentRow, lit(2.5D)))), + Seq(Row(1.0, 3), Row(1.0, 3), Row(100.001, 1), Row(3.3, 1), Row(2.02, 1), Row(100.001, 1)) + ) + } + + test("range between should accept interval values as boundary") { + def ts(timestamp: Long): Timestamp = new Timestamp(timestamp * 1000) + + val df = Seq((ts(1501545600), "1"), (ts(1501545600), "1"), (ts(1609372800), "1"), + (ts(1503000000), "2"), (ts(1502000000), "1"), (ts(1609372800), "2")) + .toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select( + $"key", + count("key").over( + Window.partitionBy($"value").orderBy($"key") + .rangeBetween(currentRow, + lit(CalendarInterval.fromString("interval 23 days 4 hours"))))), + Seq(Row(ts(1501545600), 3), Row(ts(1501545600), 3), Row(ts(1609372800), 1), + Row(ts(1503000000), 1), Row(ts(1502000000), 1), Row(ts(1609372800), 1)) + ) + } + + test("aggregation and rows between with unbounded") { + val df = Seq((1, "1"), (2, "2"), (2, "3"), (1, "3"), (3, "2"), (4, "3")).toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select( + $"key", + last("key").over( + Window.partitionBy($"value").orderBy($"key") + .rowsBetween(Window.currentRow, Window.unboundedFollowing)), + last("key").over( + Window.partitionBy($"value").orderBy($"key") + .rowsBetween(Window.unboundedPreceding, Window.currentRow)), + last("key").over(Window.partitionBy($"value").orderBy($"key").rowsBetween(-1, 1))), + Seq(Row(1, 1, 1, 1), Row(2, 3, 2, 3), Row(3, 3, 3, 3), Row(1, 4, 1, 2), Row(2, 4, 2, 4), + Row(4, 4, 4, 4))) + } + + test("aggregation and range between with unbounded") { + val df = Seq((5, "1"), (5, "2"), (4, "2"), (6, "2"), (3, "1"), (2, "2")).toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select( + $"key", + last("value").over( + Window.partitionBy($"value").orderBy($"key").rangeBetween(-2, -1)) + .equalTo("2") + .as("last_v"), + avg("key").over(Window.partitionBy("value").orderBy("key").rangeBetween(Long.MinValue, 1)) + .as("avg_key1"), + avg("key").over(Window.partitionBy("value").orderBy("key").rangeBetween(0, Long.MaxValue)) + .as("avg_key2"), + avg("key").over(Window.partitionBy("value").orderBy("key").rangeBetween(-1, 0)) + .as("avg_key3") + ), + Seq(Row(3, null, 3.0d, 4.0d, 3.0d), + Row(5, false, 4.0d, 5.0d, 5.0d), + Row(2, null, 2.0d, 17.0d / 4.0d, 2.0d), + Row(4, true, 11.0d / 3.0d, 5.0d, 4.0d), + Row(5, true, 17.0d / 4.0d, 11.0d / 2.0d, 4.5d), + Row(6, true, 17.0d / 4.0d, 6.0d, 11.0d / 2.0d))) + } + + test("reverse sliding range frame") { + val df = Seq( + (1, "Thin", "Cell Phone", 6000), + (2, "Normal", "Tablet", 1500), + (3, "Mini", "Tablet", 5500), + (4, "Ultra thin", "Cell Phone", 5500), + (5, "Very thin", "Cell Phone", 6000), + (6, "Big", "Tablet", 2500), + (7, "Bendable", "Cell Phone", 3000), + (8, "Foldable", "Cell Phone", 3000), + (9, "Pro", "Tablet", 4500), + (10, "Pro2", "Tablet", 6500)). + toDF("id", "product", "category", "revenue") + val window = Window. + partitionBy($"category"). + orderBy($"revenue".desc). + rangeBetween(-2000L, 1000L) + checkAnswer( + df.select( + $"id", + avg($"revenue").over(window).cast("int")), + Row(1, 5833) :: Row(2, 2000) :: Row(3, 5500) :: + Row(4, 5833) :: Row(5, 5833) :: Row(6, 2833) :: + Row(7, 3000) :: Row(8, 3000) :: Row(9, 5500) :: + Row(10, 6000) :: Nil) + } + + // This is here to illustrate the fact that reverse order also reverses offsets. + test("reverse unbounded range frame") { + val df = Seq(1, 2, 4, 3, 2, 1). + map(Tuple1.apply). + toDF("value") + val window = Window.orderBy($"value".desc) + checkAnswer( + df.select( + $"value", + sum($"value").over(window.rangeBetween(Long.MinValue, 1)), + sum($"value").over(window.rangeBetween(1, Long.MaxValue))), + Row(1, 13, null) :: Row(2, 13, 2) :: Row(4, 7, 9) :: + Row(3, 11, 6) :: Row(2, 13, 2) :: Row(1, 13, null) :: Nil) + } + + test("statistical functions") { + val df = Seq(("a", 1), ("a", 1), ("a", 2), ("a", 2), ("b", 4), ("b", 3), ("b", 2)). + toDF("key", "value") + val window = Window.partitionBy($"key") + checkAnswer( + df.select( + $"key", + var_pop($"value").over(window), + var_samp($"value").over(window), + approx_count_distinct($"value").over(window)), + Seq.fill(4)(Row("a", 1.0d / 4.0d, 1.0d / 3.0d, 2)) + ++ Seq.fill(3)(Row("b", 2.0d / 3.0d, 1.0d, 3))) + } + + test("window function with aggregates") { + val df = Seq(("a", 1), ("a", 1), ("a", 2), ("a", 2), ("b", 4), ("b", 3), ("b", 2)). + toDF("key", "value") + val window = Window.orderBy() + checkAnswer( + df.groupBy($"key") + .agg( + sum($"value"), + sum(sum($"value")).over(window) - sum($"value")), + Seq(Row("a", 6, 9), Row("b", 9, 6))) + } + + test("SPARK-16195 empty over spec") { + val df = Seq(("a", 1), ("a", 1), ("a", 2), ("b", 2)). + toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select($"key", $"value", sum($"value").over(), avg($"value").over()), + Seq(Row("a", 1, 6, 1.5), Row("a", 1, 6, 1.5), Row("a", 2, 6, 1.5), Row("b", 2, 6, 1.5))) + checkAnswer( + sql("select key, value, sum(value) over(), avg(value) over() from window_table"), + Seq(Row("a", 1, 6, 1.5), Row("a", 1, 6, 1.5), Row("a", 2, 6, 1.5), Row("b", 2, 6, 1.5))) + } + + test("window function with udaf") { + val udaf = new UserDefinedAggregateFunction { + def inputSchema: StructType = new StructType() + .add("a", LongType) + .add("b", LongType) + + def bufferSchema: StructType = new StructType() + .add("product", LongType) + + def dataType: DataType = LongType + + def deterministic: Boolean = true + + def initialize(buffer: MutableAggregationBuffer): Unit = { + buffer(0) = 0L + } + + def update(buffer: MutableAggregationBuffer, input: Row): Unit = { + if (!(input.isNullAt(0) || input.isNullAt(1))) { + buffer(0) = buffer.getLong(0) + input.getLong(0) * input.getLong(1) + } + } + + def merge(buffer1: MutableAggregationBuffer, buffer2: Row): Unit = { + buffer1(0) = buffer1.getLong(0) + buffer2.getLong(0) + } + + def evaluate(buffer: Row): Any = + buffer.getLong(0) + } + val df = Seq( + ("a", 1, 1), + ("a", 1, 5), + ("a", 2, 10), + ("a", 2, -1), + ("b", 4, 7), + ("b", 3, 8), + ("b", 2, 4)) + .toDF("key", "a", "b") + val window = Window.partitionBy($"key").orderBy($"a").rangeBetween(Long.MinValue, 0L) + checkAnswer( + df.select( + $"key", + $"a", + $"b", + udaf($"a", $"b").over(window)), + Seq( + Row("a", 1, 1, 6), + Row("a", 1, 5, 6), + Row("a", 2, 10, 24), + Row("a", 2, -1, 24), + Row("b", 4, 7, 60), + Row("b", 3, 8, 32), + Row("b", 2, 4, 8))) + } + + test("null inputs") { + val df = Seq(("a", 1), ("a", 1), ("a", 2), ("a", 2), ("b", 4), ("b", 3), ("b", 2)) + .toDF("key", "value") + val window = Window.orderBy() + checkAnswer( + df.select( + $"key", + $"value", + avg(lit(null)).over(window), + sum(lit(null)).over(window)), + Seq( + Row("a", 1, null, null), + Row("a", 1, null, null), + Row("a", 2, null, null), + Row("a", 2, null, null), + Row("b", 4, null, null), + Row("b", 3, null, null), + Row("b", 2, null, null))) + } + + test("last/first with ignoreNulls") { + val nullStr: String = null + val df = Seq( + ("a", 0, nullStr), + ("a", 1, "x"), + ("a", 2, "y"), + ("a", 3, "z"), + ("a", 4, nullStr), + ("b", 1, nullStr), + ("b", 2, nullStr)). + toDF("key", "order", "value") + val window = Window.partitionBy($"key").orderBy($"order") + checkAnswer( + df.select( + $"key", + $"order", + first($"value").over(window), + first($"value", ignoreNulls = false).over(window), + first($"value", ignoreNulls = true).over(window), + last($"value").over(window), + last($"value", ignoreNulls = false).over(window), + last($"value", ignoreNulls = true).over(window)), + Seq( + Row("a", 0, null, null, null, null, null, null), + Row("a", 1, null, null, "x", "x", "x", "x"), + Row("a", 2, null, null, "x", "y", "y", "y"), + Row("a", 3, null, null, "x", "z", "z", "z"), + Row("a", 4, null, null, "x", null, null, "z"), + Row("b", 1, null, null, null, null, null, null), + Row("b", 2, null, null, null, null, null, null))) + } + + test("SPARK-12989 ExtractWindowExpressions treats alias as regular attribute") { + val src = Seq((0, 3, 5)).toDF("a", "b", "c") + .withColumn("Data", struct("a", "b")) + .drop("a") + .drop("b") + val winSpec = Window.partitionBy("Data.a", "Data.b").orderBy($"c".desc) + val df = src.select($"*", max("c").over(winSpec) as "max") + checkAnswer(df, Row(5, Row(0, 3), 5)) + } + + test("aggregation and rows between with unbounded + predicate pushdown") { + val df = Seq((1, "1"), (2, "2"), (2, "3"), (1, "3"), (3, "2"), (4, "3")).toDF("key", "value") + df.createOrReplaceTempView("window_table") + val selectList = Seq($"key", $"value", + last("key").over( + Window.partitionBy($"value").orderBy($"key").rowsBetween(0, Long.MaxValue)), + last("key").over( + Window.partitionBy($"value").orderBy($"key").rowsBetween(Long.MinValue, 0)), + last("key").over(Window.partitionBy($"value").orderBy($"key").rowsBetween(-1, 1))) + + checkAnswer( + df.select(selectList: _*).where($"value" < "3"), + Seq(Row(1, "1", 1, 1, 1), Row(2, "2", 3, 2, 3), Row(3, "2", 3, 3, 3))) + } + + test("aggregation and range between with unbounded + predicate pushdown") { + val df = Seq((5, "1"), (5, "2"), (4, "2"), (6, "2"), (3, "1"), (2, "2")).toDF("key", "value") + df.createOrReplaceTempView("window_table") + val selectList = Seq($"key", $"value", + last("value").over( + Window.partitionBy($"value").orderBy($"key").rangeBetween(-2, -1)).equalTo("2") + .as("last_v"), + avg("key").over(Window.partitionBy("value").orderBy("key").rangeBetween(Long.MinValue, 1)) + .as("avg_key1"), + avg("key").over(Window.partitionBy("value").orderBy("key").rangeBetween(0, Long.MaxValue)) + .as("avg_key2"), + avg("key").over(Window.partitionBy("value").orderBy("key").rangeBetween(-1, 1)) + .as("avg_key3")) + + checkAnswer( + df.select(selectList: _*).where($"value" < 2), + Seq(Row(3, "1", null, 3.0, 4.0, 3.0), Row(5, "1", false, 4.0, 5.0, 5.0))) + } + + test("SPARK-21258: complex object in combination with spilling") { + // Make sure we trigger the spilling path. + withSQLConf(SQLConf.WINDOW_EXEC_BUFFER_SPILL_THRESHOLD.key -> "17") { + val sampleSchema = new StructType(). + add("f0", StringType). + add("f1", LongType). + add("f2", ArrayType(new StructType(). + add("f20", StringType))). + add("f3", ArrayType(new StructType(). + add("f30", StringType))) + + val w0 = Window.partitionBy("f0").orderBy("f1") + val w1 = w0.rowsBetween(Long.MinValue, Long.MaxValue) + + val c0 = first(struct($"f2", $"f3")).over(w0) as "c0" + val c1 = last(struct($"f2", $"f3")).over(w1) as "c1" + + val input = + """{"f1":1497820153720,"f2":[{"f20":"x","f21":0}],"f3":[{"f30":"x","f31":0}]} + |{"f1":1497802179638} + |{"f1":1497802189347} + |{"f1":1497802189593} + |{"f1":1497802189597} + |{"f1":1497802189599} + |{"f1":1497802192103} + |{"f1":1497802193414} + |{"f1":1497802193577} + |{"f1":1497802193709} + |{"f1":1497802202883} + |{"f1":1497802203006} + |{"f1":1497802203743} + |{"f1":1497802203834} + |{"f1":1497802203887} + |{"f1":1497802203893} + |{"f1":1497802203976} + |{"f1":1497820168098} + |""".stripMargin.split("\n").toSeq + + import testImplicits._ + + spark.read.schema(sampleSchema).json(input.toDS()).select(c0, c1).foreach { _ => () } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala new file mode 100644 index 000000000000..0e7eaa9e88d5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala @@ -0,0 +1,336 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.expressions.Aggregator +import org.apache.spark.sql.expressions.scalalang.typed +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StringType + + +object ComplexResultAgg extends Aggregator[(String, Int), (Long, Long), (Long, Long)] { + override def zero: (Long, Long) = (0, 0) + override def reduce(countAndSum: (Long, Long), input: (String, Int)): (Long, Long) = { + (countAndSum._1 + 1, countAndSum._2 + input._2) + } + override def merge(b1: (Long, Long), b2: (Long, Long)): (Long, Long) = { + (b1._1 + b2._1, b1._2 + b2._2) + } + override def finish(reduction: (Long, Long)): (Long, Long) = reduction + override def bufferEncoder: Encoder[(Long, Long)] = Encoders.product[(Long, Long)] + override def outputEncoder: Encoder[(Long, Long)] = Encoders.product[(Long, Long)] +} + + +case class AggData(a: Int, b: String) + +object ClassInputAgg extends Aggregator[AggData, Int, Int] { + override def zero: Int = 0 + override def reduce(b: Int, a: AggData): Int = b + a.a + override def finish(reduction: Int): Int = reduction + override def merge(b1: Int, b2: Int): Int = b1 + b2 + override def bufferEncoder: Encoder[Int] = Encoders.scalaInt + override def outputEncoder: Encoder[Int] = Encoders.scalaInt +} + + +object ClassBufferAggregator extends Aggregator[AggData, AggData, Int] { + override def zero: AggData = AggData(0, "") + override def reduce(b: AggData, a: AggData): AggData = AggData(b.a + a.a, "") + override def finish(reduction: AggData): Int = reduction.a + override def merge(b1: AggData, b2: AggData): AggData = AggData(b1.a + b2.a, "") + override def bufferEncoder: Encoder[AggData] = Encoders.product[AggData] + override def outputEncoder: Encoder[Int] = Encoders.scalaInt +} + + +object ComplexBufferAgg extends Aggregator[AggData, (Int, AggData), Int] { + override def zero: (Int, AggData) = 0 -> AggData(0, "0") + override def reduce(b: (Int, AggData), a: AggData): (Int, AggData) = (b._1 + 1, a) + override def finish(reduction: (Int, AggData)): Int = reduction._1 + override def merge(b1: (Int, AggData), b2: (Int, AggData)): (Int, AggData) = + (b1._1 + b2._1, b1._2) + override def bufferEncoder: Encoder[(Int, AggData)] = Encoders.product[(Int, AggData)] + override def outputEncoder: Encoder[Int] = Encoders.scalaInt +} + + +object MapTypeBufferAgg extends Aggregator[Int, Map[Int, Int], Int] { + override def zero: Map[Int, Int] = Map.empty + override def reduce(b: Map[Int, Int], a: Int): Map[Int, Int] = b + override def finish(reduction: Map[Int, Int]): Int = 1 + override def merge(b1: Map[Int, Int], b2: Map[Int, Int]): Map[Int, Int] = b1 + override def bufferEncoder: Encoder[Map[Int, Int]] = ExpressionEncoder() + override def outputEncoder: Encoder[Int] = ExpressionEncoder() +} + + +object NameAgg extends Aggregator[AggData, String, String] { + def zero: String = "" + def reduce(b: String, a: AggData): String = a.b + b + def merge(b1: String, b2: String): String = b1 + b2 + def finish(r: String): String = r + override def bufferEncoder: Encoder[String] = Encoders.STRING + override def outputEncoder: Encoder[String] = Encoders.STRING +} + + +object SeqAgg extends Aggregator[AggData, Seq[Int], Seq[(Int, Int)]] { + def zero: Seq[Int] = Nil + def reduce(b: Seq[Int], a: AggData): Seq[Int] = a.a +: b + def merge(b1: Seq[Int], b2: Seq[Int]): Seq[Int] = b1 ++ b2 + def finish(r: Seq[Int]): Seq[(Int, Int)] = r.map(i => i -> i) + override def bufferEncoder: Encoder[Seq[Int]] = ExpressionEncoder() + override def outputEncoder: Encoder[Seq[(Int, Int)]] = ExpressionEncoder() +} + + +class ParameterizedTypeSum[IN, OUT : Numeric : Encoder](f: IN => OUT) + extends Aggregator[IN, OUT, OUT] { + + private val numeric = implicitly[Numeric[OUT]] + override def zero: OUT = numeric.zero + override def reduce(b: OUT, a: IN): OUT = numeric.plus(b, f(a)) + override def merge(b1: OUT, b2: OUT): OUT = numeric.plus(b1, b2) + override def finish(reduction: OUT): OUT = reduction + override def bufferEncoder: Encoder[OUT] = implicitly[Encoder[OUT]] + override def outputEncoder: Encoder[OUT] = implicitly[Encoder[OUT]] +} + +object RowAgg extends Aggregator[Row, Int, Int] { + def zero: Int = 0 + def reduce(b: Int, a: Row): Int = a.getInt(0) + b + def merge(b1: Int, b2: Int): Int = b1 + b2 + def finish(r: Int): Int = r + override def bufferEncoder: Encoder[Int] = Encoders.scalaInt + override def outputEncoder: Encoder[Int] = Encoders.scalaInt +} + +object NullResultAgg extends Aggregator[AggData, AggData, AggData] { + override def zero: AggData = AggData(0, "") + override def reduce(b: AggData, a: AggData): AggData = AggData(b.a + a.a, b.b + a.b) + override def finish(reduction: AggData): AggData = { + if (reduction.a % 2 == 0) null else reduction + } + override def merge(b1: AggData, b2: AggData): AggData = AggData(b1.a + b2.a, b1.b + b2.b) + override def bufferEncoder: Encoder[AggData] = Encoders.product[AggData] + override def outputEncoder: Encoder[AggData] = Encoders.product[AggData] +} + +case class ComplexAggData(d1: AggData, d2: AggData) + +object VeryComplexResultAgg extends Aggregator[Row, String, ComplexAggData] { + override def zero: String = "" + override def reduce(buffer: String, input: Row): String = buffer + input.getString(1) + override def merge(b1: String, b2: String): String = b1 + b2 + override def finish(reduction: String): ComplexAggData = { + ComplexAggData(AggData(reduction.length, reduction), AggData(reduction.length, reduction)) + } + override def bufferEncoder: Encoder[String] = Encoders.STRING + override def outputEncoder: Encoder[ComplexAggData] = Encoders.product[ComplexAggData] +} + + +class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + private implicit val ordering = Ordering.by((c: AggData) => c.a -> c.b) + + test("typed aggregation: TypedAggregator") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDataset( + ds.groupByKey(_._1).agg(typed.sum(_._2)), + ("a", 30.0), ("b", 3.0), ("c", 1.0)) + } + + test("typed aggregation: TypedAggregator, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDataset( + ds.groupByKey(_._1).agg( + typed.sum(_._2), + expr("sum(_2)").as[Long], + count("*")), + ("a", 30.0, 30L, 2L), ("b", 3.0, 3L, 2L), ("c", 1.0, 1L, 1L)) + } + + test("typed aggregation: complex result type") { + val ds = Seq("a" -> 1, "a" -> 3, "b" -> 3).toDS() + + checkDataset( + ds.groupByKey(_._1).agg( + expr("avg(_2)").as[Double], + ComplexResultAgg.toColumn), + ("a", 2.0, (2L, 4L)), ("b", 3.0, (1L, 3L))) + } + + test("typed aggregation: in project list") { + val ds = Seq(1, 3, 2, 5).toDS() + + checkDataset( + ds.select(typed.sum((i: Int) => i)), + 11.0) + checkDataset( + ds.select(typed.sum((i: Int) => i), typed.sum((i: Int) => i * 2)), + 11.0 -> 22.0) + } + + test("typed aggregation: class input") { + val ds = Seq(AggData(1, "one"), AggData(2, "two")).toDS() + + checkDataset( + ds.select(ClassInputAgg.toColumn), + 3) + } + + test("typed aggregation: class input with reordering") { + val ds = sql("SELECT 'one' AS b, 1 as a").as[AggData] + + checkDataset( + ds.select(ClassInputAgg.toColumn), + 1) + + checkDataset( + ds.select(expr("avg(a)").as[Double], ClassInputAgg.toColumn), + (1.0, 1)) + + checkDataset( + ds.groupByKey(_.b).agg(ClassInputAgg.toColumn), + ("one", 1)) + } + + test("Typed aggregation using aggregator") { + // based on Dataset complex Aggregator test of DatasetBenchmark + val ds = Seq(AggData(1, "x"), AggData(2, "y"), AggData(3, "z")).toDS() + checkDataset( + ds.select(ClassBufferAggregator.toColumn), + 6) + } + + test("typed aggregation: complex input") { + val ds = Seq(AggData(1, "one"), AggData(2, "two")).toDS() + + checkDataset( + ds.select(ComplexBufferAgg.toColumn), + 2 + ) + + checkDataset( + ds.select(expr("avg(a)").as[Double], ComplexBufferAgg.toColumn), + (1.5, 2)) + + checkDatasetUnorderly( + ds.groupByKey(_.b).agg(ComplexBufferAgg.toColumn), + ("one", 1), ("two", 1)) + } + + test("typed aggregate: avg, count, sum") { + val ds = Seq("a" -> 1, "a" -> 3, "b" -> 3).toDS() + checkDataset( + ds.groupByKey(_._1).agg( + typed.avg(_._2), typed.count(_._2), typed.sum(_._2), typed.sumLong(_._2)), + ("a", 2.0, 2L, 4.0, 4L), ("b", 3.0, 1L, 3.0, 3L)) + } + + test("generic typed sum") { + val ds = Seq("a" -> 1, "a" -> 3, "b" -> 3).toDS() + checkDataset( + ds.groupByKey(_._1) + .agg(new ParameterizedTypeSum[(String, Int), Double](_._2.toDouble).toColumn), + ("a", 4.0), ("b", 3.0)) + + checkDataset( + ds.groupByKey(_._1) + .agg(new ParameterizedTypeSum((x: (String, Int)) => x._2.toInt).toColumn), + ("a", 4), ("b", 3)) + } + + test("SPARK-12555 - result should not be corrupted after input columns are reordered") { + val ds = sql("SELECT 'Some String' AS b, 1279869254 AS a").as[AggData] + + checkDataset( + ds.groupByKey(_.a).agg(NameAgg.toColumn), + (1279869254, "Some String")) + } + + test("aggregator in DataFrame/Dataset[Row]") { + val df = Seq(1 -> "a", 2 -> "b", 3 -> "b").toDF("i", "j") + checkAnswer(df.groupBy($"j").agg(RowAgg.toColumn), Row("a", 1) :: Row("b", 5) :: Nil) + } + + test("SPARK-14675: ClassFormatError when use Seq as Aggregator buffer type") { + val ds = Seq(AggData(1, "a"), AggData(2, "a")).toDS() + + checkDataset( + ds.groupByKey(_.b).agg(SeqAgg.toColumn), + "a" -> Seq(1 -> 1, 2 -> 2) + ) + } + + test("spark-15051 alias of aggregator in DataFrame/Dataset[Row]") { + val df1 = Seq(1 -> "a", 2 -> "b", 3 -> "b").toDF("i", "j") + checkAnswer(df1.agg(RowAgg.toColumn as "b"), Row(6) :: Nil) + + val df2 = Seq(1 -> "a", 2 -> "b", 3 -> "b").toDF("i", "j") + checkAnswer(df2.agg(RowAgg.toColumn as "b").select("b"), Row(6) :: Nil) + } + + test("spark-15114 shorter system generated alias names") { + val ds = Seq(1, 3, 2, 5).toDS() + assert(ds.select(typed.sum((i: Int) => i)).columns.head === "TypedSumDouble(int)") + val ds2 = ds.select(typed.sum((i: Int) => i), typed.avg((i: Int) => i)) + assert(ds2.columns.head === "TypedSumDouble(int)") + assert(ds2.columns.last === "TypedAverage(int)") + val df = Seq(1 -> "a", 2 -> "b", 3 -> "b").toDF("i", "j") + assert(df.groupBy($"j").agg(RowAgg.toColumn).columns.last == + "RowAgg(org.apache.spark.sql.Row)") + assert(df.groupBy($"j").agg(RowAgg.toColumn as "agg1").columns.last == "agg1") + } + + test("SPARK-15814 Aggregator can return null result") { + val ds = Seq(AggData(1, "one"), AggData(2, "two")).toDS() + checkDatasetUnorderly( + ds.groupByKey(_.a).agg(NullResultAgg.toColumn), + 1 -> AggData(1, "one"), 2 -> null) + } + + test("SPARK-16100: use Map as the buffer type of Aggregator") { + val ds = Seq(1, 2, 3).toDS() + checkDataset(ds.select(MapTypeBufferAgg.toColumn), 1) + } + + test("SPARK-15204 improve nullability inference for Aggregator") { + val ds1 = Seq(1, 3, 2, 5).toDS() + assert(ds1.select(typed.sum((i: Int) => i)).schema.head.nullable === false) + val ds2 = Seq(AggData(1, "a"), AggData(2, "a")).toDS() + assert(ds2.select(SeqAgg.toColumn).schema.head.nullable === true) + val ds3 = sql("SELECT 'Some String' AS b, 1279869254 AS a").as[AggData] + assert(ds3.select(NameAgg.toColumn).schema.head.nullable === true) + } + + test("SPARK-18147: very complex aggregator result type") { + val df = Seq(1 -> "a", 2 -> "b", 2 -> "c").toDF("i", "j") + + checkAnswer( + df.groupBy($"i").agg(VeryComplexResultAgg.toColumn), + Row(1, Row(Row(1, "a"), Row(1, "a"))) :: Row(2, Row(Row(2, "bc"), Row(2, "bc"))) :: Nil) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala new file mode 100644 index 000000000000..1a0672b8876d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala @@ -0,0 +1,316 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.{SparkConf, SparkContext} +import org.apache.spark.sql.expressions.Aggregator +import org.apache.spark.sql.expressions.scalalang.typed +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.StringType +import org.apache.spark.util.Benchmark + +/** + * Benchmark for Dataset typed operations comparing with DataFrame and RDD versions. + */ +object DatasetBenchmark { + + case class Data(l: Long, s: String) + + def backToBackMapLong(spark: SparkSession, numRows: Long, numChains: Int): Benchmark = { + import spark.implicits._ + + val rdd = spark.sparkContext.range(0, numRows) + val ds = spark.range(0, numRows) + val df = ds.toDF("l") + val func = (l: Long) => l + 1 + + val benchmark = new Benchmark("back-to-back map long", numRows) + + benchmark.addCase("RDD") { iter => + var res = rdd + var i = 0 + while (i < numChains) { + res = res.map(func) + i += 1 + } + res.foreach(_ => Unit) + } + + benchmark.addCase("DataFrame") { iter => + var res = df + var i = 0 + while (i < numChains) { + res = res.select($"l" + 1 as "l") + i += 1 + } + res.queryExecution.toRdd.foreach(_ => Unit) + } + + benchmark.addCase("Dataset") { iter => + var res = ds.as[Long] + var i = 0 + while (i < numChains) { + res = res.map(func) + i += 1 + } + res.queryExecution.toRdd.foreach(_ => Unit) + } + + benchmark + } + + def backToBackMap(spark: SparkSession, numRows: Long, numChains: Int): Benchmark = { + import spark.implicits._ + + val df = spark.range(1, numRows).select($"id".as("l"), $"id".cast(StringType).as("s")) + val benchmark = new Benchmark("back-to-back map", numRows) + val func = (d: Data) => Data(d.l + 1, d.s) + + val rdd = spark.sparkContext.range(1, numRows).map(l => Data(l, l.toString)) + benchmark.addCase("RDD") { iter => + var res = rdd + var i = 0 + while (i < numChains) { + res = res.map(func) + i += 1 + } + res.foreach(_ => Unit) + } + + benchmark.addCase("DataFrame") { iter => + var res = df + var i = 0 + while (i < numChains) { + res = res.select($"l" + 1 as "l", $"s") + i += 1 + } + res.queryExecution.toRdd.foreach(_ => Unit) + } + + benchmark.addCase("Dataset") { iter => + var res = df.as[Data] + var i = 0 + while (i < numChains) { + res = res.map(func) + i += 1 + } + res.queryExecution.toRdd.foreach(_ => Unit) + } + + benchmark + } + + def backToBackFilterLong(spark: SparkSession, numRows: Long, numChains: Int): Benchmark = { + import spark.implicits._ + + val rdd = spark.sparkContext.range(1, numRows) + val ds = spark.range(1, numRows) + val df = ds.toDF("l") + val func = (l: Long) => l % 2L == 0L + + val benchmark = new Benchmark("back-to-back filter Long", numRows) + + benchmark.addCase("RDD") { iter => + var res = rdd + var i = 0 + while (i < numChains) { + res = res.filter(func) + i += 1 + } + res.foreach(_ => Unit) + } + + benchmark.addCase("DataFrame") { iter => + var res = df + var i = 0 + while (i < numChains) { + res = res.filter($"l" % 2L === 0L) + i += 1 + } + res.queryExecution.toRdd.foreach(_ => Unit) + } + + benchmark.addCase("Dataset") { iter => + var res = ds.as[Long] + var i = 0 + while (i < numChains) { + res = res.filter(func) + i += 1 + } + res.queryExecution.toRdd.foreach(_ => Unit) + } + + benchmark + } + + def backToBackFilter(spark: SparkSession, numRows: Long, numChains: Int): Benchmark = { + import spark.implicits._ + + val df = spark.range(1, numRows).select($"id".as("l"), $"id".cast(StringType).as("s")) + val benchmark = new Benchmark("back-to-back filter", numRows) + val func = (d: Data, i: Int) => d.l % (100L + i) == 0L + val funcs = 0.until(numChains).map { i => + (d: Data) => func(d, i) + } + + val rdd = spark.sparkContext.range(1, numRows).map(l => Data(l, l.toString)) + benchmark.addCase("RDD") { iter => + var res = rdd + var i = 0 + while (i < numChains) { + res = res.filter(funcs(i)) + i += 1 + } + res.foreach(_ => Unit) + } + + benchmark.addCase("DataFrame") { iter => + var res = df + var i = 0 + while (i < numChains) { + res = res.filter($"l" % (100L + i) === 0L) + i += 1 + } + res.queryExecution.toRdd.foreach(_ => Unit) + } + + benchmark.addCase("Dataset") { iter => + var res = df.as[Data] + var i = 0 + while (i < numChains) { + res = res.filter(funcs(i)) + i += 1 + } + res.queryExecution.toRdd.foreach(_ => Unit) + } + + benchmark + } + + object ComplexAggregator extends Aggregator[Data, Data, Long] { + override def zero: Data = Data(0, "") + + override def reduce(b: Data, a: Data): Data = Data(b.l + a.l, "") + + override def finish(reduction: Data): Long = reduction.l + + override def merge(b1: Data, b2: Data): Data = Data(b1.l + b2.l, "") + + override def bufferEncoder: Encoder[Data] = Encoders.product[Data] + + override def outputEncoder: Encoder[Long] = Encoders.scalaLong + } + + def aggregate(spark: SparkSession, numRows: Long): Benchmark = { + import spark.implicits._ + + val df = spark.range(1, numRows).select($"id".as("l"), $"id".cast(StringType).as("s")) + val benchmark = new Benchmark("aggregate", numRows) + + val rdd = spark.sparkContext.range(1, numRows).map(l => Data(l, l.toString)) + benchmark.addCase("RDD sum") { iter => + rdd.aggregate(0L)(_ + _.l, _ + _) + } + + benchmark.addCase("DataFrame sum") { iter => + df.select(sum($"l")).queryExecution.toRdd.foreach(_ => Unit) + } + + benchmark.addCase("Dataset sum using Aggregator") { iter => + df.as[Data].select(typed.sumLong((d: Data) => d.l)).queryExecution.toRdd.foreach(_ => Unit) + } + + benchmark.addCase("Dataset complex Aggregator") { iter => + df.as[Data].select(ComplexAggregator.toColumn).queryExecution.toRdd.foreach(_ => Unit) + } + + benchmark + } + + def main(args: Array[String]): Unit = { + val spark = SparkSession.builder + .master("local[*]") + .appName("Dataset benchmark") + .getOrCreate() + + val numRows = 100000000 + val numChains = 10 + + val benchmark0 = backToBackMapLong(spark, numRows, numChains) + val benchmark1 = backToBackMap(spark, numRows, numChains) + val benchmark2 = backToBackFilterLong(spark, numRows, numChains) + val benchmark3 = backToBackFilter(spark, numRows, numChains) + val benchmark4 = aggregate(spark, numRows) + + /* + OpenJDK 64-Bit Server VM 1.8.0_111-8u111-b14-2ubuntu0.16.04.2-b14 on Linux 4.4.0-47-generic + Intel(R) Xeon(R) CPU E5-2667 v3 @ 3.20GHz + back-to-back map long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + RDD 1883 / 1892 53.1 18.8 1.0X + DataFrame 502 / 642 199.1 5.0 3.7X + Dataset 657 / 784 152.2 6.6 2.9X + */ + benchmark0.run() + + /* + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 3.10.0-327.18.2.el7.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + back-to-back map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + RDD 3448 / 3646 29.0 34.5 1.0X + DataFrame 2647 / 3116 37.8 26.5 1.3X + Dataset 4781 / 5155 20.9 47.8 0.7X + */ + benchmark1.run() + + /* + OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-47-generic + Intel(R) Xeon(R) CPU E5-2667 v3 @ 3.20GHz + back-to-back filter Long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + RDD 846 / 1120 118.1 8.5 1.0X + DataFrame 270 / 329 370.9 2.7 3.1X + Dataset 545 / 789 183.5 5.4 1.6X + */ + benchmark2.run() + + /* + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 3.10.0-327.18.2.el7.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + back-to-back filter: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + RDD 1346 / 1618 74.3 13.5 1.0X + DataFrame 59 / 72 1695.4 0.6 22.8X + Dataset 2777 / 2805 36.0 27.8 0.5X + */ + benchmark3.run() + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.12.1 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + aggregate: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + RDD sum 1913 / 1942 52.3 19.1 1.0X + DataFrame sum 46 / 61 2157.7 0.5 41.3X + Dataset sum using Aggregator 4656 / 4758 21.5 46.6 0.4X + Dataset complex Aggregator 6636 / 7039 15.1 66.4 0.3X + */ + benchmark4.run() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala new file mode 100644 index 000000000000..e0561ee2797a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.storage.StorageLevel + + +class DatasetCacheSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("get storage level") { + val ds1 = Seq("1", "2").toDS().as("a") + val ds2 = Seq(2, 3).toDS().as("b") + + // default storage level + ds1.persist() + ds2.cache() + assert(ds1.storageLevel == StorageLevel.MEMORY_AND_DISK) + assert(ds2.storageLevel == StorageLevel.MEMORY_AND_DISK) + // unpersist + ds1.unpersist() + assert(ds1.storageLevel == StorageLevel.NONE) + // non-default storage level + ds1.persist(StorageLevel.MEMORY_ONLY_2) + assert(ds1.storageLevel == StorageLevel.MEMORY_ONLY_2) + // joined Dataset should not be persisted + val joined = ds1.joinWith(ds2, $"a.value" === $"b.value") + assert(joined.storageLevel == StorageLevel.NONE) + } + + test("persist and unpersist") { + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS().select(expr("_2 + 1").as[Int]) + val cached = ds.cache() + // count triggers the caching action. It should not throw. + cached.count() + // Make sure, the Dataset is indeed cached. + assertCached(cached) + // Check result. + checkDataset( + cached, + 2, 3, 4) + // Drop the cache. + cached.unpersist() + assert(cached.storageLevel == StorageLevel.NONE, "The Dataset should not be cached.") + } + + test("persist and then rebind right encoder when join 2 datasets") { + val ds1 = Seq("1", "2").toDS().as("a") + val ds2 = Seq(2, 3).toDS().as("b") + + ds1.persist() + assertCached(ds1) + ds2.persist() + assertCached(ds2) + + val joined = ds1.joinWith(ds2, $"a.value" === $"b.value") + checkDataset(joined, ("2", 2)) + assertCached(joined, 2) + + ds1.unpersist() + assert(ds1.storageLevel == StorageLevel.NONE, "The Dataset ds1 should not be cached.") + ds2.unpersist() + assert(ds2.storageLevel == StorageLevel.NONE, "The Dataset ds2 should not be cached.") + } + + test("persist and then groupBy columns asKey, map") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + val grouped = ds.groupByKey(_._1) + val agged = grouped.mapGroups { case (g, iter) => (g, iter.map(_._2).sum) } + agged.persist() + + checkDataset( + agged.filter(_._1 == "b"), + ("b", 3)) + assertCached(agged.filter(_._1 == "b")) + + ds.unpersist() + assert(ds.storageLevel == StorageLevel.NONE, "The Dataset ds should not be cached.") + agged.unpersist() + assert(agged.storageLevel == StorageLevel.NONE, "The Dataset agged should not be cached.") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala index 32443557fb8e..edcdd77908d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala @@ -17,25 +17,49 @@ package org.apache.spark.sql -import scala.language.postfixOps +import scala.collection.immutable.{HashSet => HSet} +import scala.collection.immutable.Queue +import scala.collection.mutable.{LinkedHashMap => LHMap} +import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.test.SharedSQLContext case class IntClass(value: Int) +case class SeqClass(s: Seq[Int]) + +case class ListClass(l: List[Int]) + +case class QueueClass(q: Queue[Int]) + +case class MapClass(m: Map[Int, Int]) + +case class LHMapClass(m: LHMap[Int, Int]) + +case class ComplexClass(seq: SeqClass, list: ListClass, queue: QueueClass) + +case class ComplexMapClass(map: MapClass, lhmap: LHMapClass) + +case class InnerData(name: String, value: Int) +case class NestedData(id: Int, param: Map[String, InnerData]) + +package object packageobject { + case class PackageClass(value: Int) +} + class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("toDS") { val data = Seq(1, 2, 3, 4, 5, 6) - checkAnswer( + checkDataset( data.toDS(), data: _*) } test("as case class / collect") { val ds = Seq(1, 2, 3).toDS().as[IntClass] - checkAnswer( + checkDataset( ds, IntClass(1), IntClass(2), IntClass(3)) @@ -44,29 +68,90 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext { test("map") { val ds = Seq(1, 2, 3).toDS() - checkAnswer( + checkDataset( ds.map(_ + 1), 2, 3, 4) } + test("mapPrimitive") { + val dsInt = Seq(1, 2, 3).toDS() + checkDataset(dsInt.map(_ > 1), false, true, true) + checkDataset(dsInt.map(_ + 1), 2, 3, 4) + checkDataset(dsInt.map(_ + 8589934592L), 8589934593L, 8589934594L, 8589934595L) + checkDataset(dsInt.map(_ + 1.1F), 2.1F, 3.1F, 4.1F) + checkDataset(dsInt.map(_ + 1.23D), 2.23D, 3.23D, 4.23D) + + val dsLong = Seq(1L, 2L, 3L).toDS() + checkDataset(dsLong.map(_ > 1), false, true, true) + checkDataset(dsLong.map(e => (e + 1).toInt), 2, 3, 4) + checkDataset(dsLong.map(_ + 8589934592L), 8589934593L, 8589934594L, 8589934595L) + checkDataset(dsLong.map(_ + 1.1F), 2.1F, 3.1F, 4.1F) + checkDataset(dsLong.map(_ + 1.23D), 2.23D, 3.23D, 4.23D) + + val dsFloat = Seq(1F, 2F, 3F).toDS() + checkDataset(dsFloat.map(_ > 1), false, true, true) + checkDataset(dsFloat.map(e => (e + 1).toInt), 2, 3, 4) + checkDataset(dsFloat.map(e => (e + 123456L).toLong), 123457L, 123458L, 123459L) + checkDataset(dsFloat.map(_ + 1.1F), 2.1F, 3.1F, 4.1F) + checkDataset(dsFloat.map(_ + 1.23D), 2.23D, 3.23D, 4.23D) + + val dsDouble = Seq(1D, 2D, 3D).toDS() + checkDataset(dsDouble.map(_ > 1), false, true, true) + checkDataset(dsDouble.map(e => (e + 1).toInt), 2, 3, 4) + checkDataset(dsDouble.map(e => (e + 8589934592L).toLong), + 8589934593L, 8589934594L, 8589934595L) + checkDataset(dsDouble.map(e => (e + 1.1F).toFloat), 2.1F, 3.1F, 4.1F) + checkDataset(dsDouble.map(_ + 1.23D), 2.23D, 3.23D, 4.23D) + + val dsBoolean = Seq(true, false).toDS() + checkDataset(dsBoolean.map(e => !e), false, true) + } + + test("mapPrimitiveArray") { + val dsInt = Seq(Array(1, 2), Array(3, 4)).toDS() + checkDataset(dsInt.map(e => e), Array(1, 2), Array(3, 4)) + checkDataset(dsInt.map(e => null: Array[Int]), null, null) + + val dsDouble = Seq(Array(1D, 2D), Array(3D, 4D)).toDS() + checkDataset(dsDouble.map(e => e), Array(1D, 2D), Array(3D, 4D)) + checkDataset(dsDouble.map(e => null: Array[Double]), null, null) + } + test("filter") { val ds = Seq(1, 2, 3, 4).toDS() - checkAnswer( + checkDataset( ds.filter(_ % 2 == 0), 2, 4) } + test("filterPrimitive") { + val dsInt = Seq(1, 2, 3).toDS() + checkDataset(dsInt.filter(_ > 1), 2, 3) + + val dsLong = Seq(1L, 2L, 3L).toDS() + checkDataset(dsLong.filter(_ > 1), 2L, 3L) + + val dsFloat = Seq(1F, 2F, 3F).toDS() + checkDataset(dsFloat.filter(_ > 1), 2F, 3F) + + val dsDouble = Seq(1D, 2D, 3D).toDS() + checkDataset(dsDouble.filter(_ > 1), 2D, 3D) + + val dsBoolean = Seq(true, false).toDS() + checkDataset(dsBoolean.filter(e => !e), false) + } + test("foreach") { val ds = Seq(1, 2, 3).toDS() - val acc = sparkContext.accumulator(0) - ds.foreach(acc +=) + val acc = sparkContext.longAccumulator + ds.foreach(acc.add(_)) assert(acc.value == 6) } test("foreachPartition") { val ds = Seq(1, 2, 3).toDS() - val acc = sparkContext.accumulator(0) - ds.foreachPartition(_.foreach(acc +=)) + val acc = sparkContext.longAccumulator + ds.foreachPartition((it: Iterator[Int]) => it.foreach(acc.add(_))) assert(acc.value == 6) } @@ -75,29 +160,237 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext { assert(ds.reduce(_ + _) == 6) } - test("fold") { - val ds = Seq(1, 2, 3).toDS() - assert(ds.fold(0)(_ + _) == 6) - } - test("groupBy function, keys") { val ds = Seq(1, 2, 3, 4, 5).toDS() - val grouped = ds.groupBy(_ % 2) - checkAnswer( + val grouped = ds.groupByKey(_ % 2) + checkDatasetUnorderly( grouped.keys, 0, 1) } - test("groupBy function, mapGroups") { + test("groupBy function, map") { val ds = Seq(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11).toDS() - val grouped = ds.groupBy(_ % 2) + val grouped = ds.groupByKey(_ % 2) val agged = grouped.mapGroups { case (g, iter) => val name = if (g == 0) "even" else "odd" - Iterator((name, iter.size)) + (name, iter.size) } - checkAnswer( + checkDatasetUnorderly( agged, ("even", 5), ("odd", 6)) } + + test("groupBy function, flatMap") { + val ds = Seq("a", "b", "c", "xyz", "hello").toDS() + val grouped = ds.groupByKey(_.length) + val agged = grouped.flatMapGroups { case (g, iter) => Iterator(g.toString, iter.mkString) } + + checkDatasetUnorderly( + agged, + "1", "abc", "3", "xyz", "5", "hello") + } + + test("Arrays and Lists") { + checkDataset(Seq(Seq(1)).toDS(), Seq(1)) + checkDataset(Seq(Seq(1.toLong)).toDS(), Seq(1.toLong)) + checkDataset(Seq(Seq(1.toDouble)).toDS(), Seq(1.toDouble)) + checkDataset(Seq(Seq(1.toFloat)).toDS(), Seq(1.toFloat)) + checkDataset(Seq(Seq(1.toByte)).toDS(), Seq(1.toByte)) + checkDataset(Seq(Seq(1.toShort)).toDS(), Seq(1.toShort)) + checkDataset(Seq(Seq(true)).toDS(), Seq(true)) + checkDataset(Seq(Seq("test")).toDS(), Seq("test")) + checkDataset(Seq(Seq(Tuple1(1))).toDS(), Seq(Tuple1(1))) + + checkDataset(Seq(Array(1)).toDS(), Array(1)) + checkDataset(Seq(Array(1.toLong)).toDS(), Array(1.toLong)) + checkDataset(Seq(Array(1.toDouble)).toDS(), Array(1.toDouble)) + checkDataset(Seq(Array(1.toFloat)).toDS(), Array(1.toFloat)) + checkDataset(Seq(Array(1.toByte)).toDS(), Array(1.toByte)) + checkDataset(Seq(Array(1.toShort)).toDS(), Array(1.toShort)) + checkDataset(Seq(Array(true)).toDS(), Array(true)) + checkDataset(Seq(Array("test")).toDS(), Array("test")) + checkDataset(Seq(Array(Tuple1(1))).toDS(), Array(Tuple1(1))) + } + + test("arbitrary sequences") { + checkDataset(Seq(Queue(1)).toDS(), Queue(1)) + checkDataset(Seq(Queue(1.toLong)).toDS(), Queue(1.toLong)) + checkDataset(Seq(Queue(1.toDouble)).toDS(), Queue(1.toDouble)) + checkDataset(Seq(Queue(1.toFloat)).toDS(), Queue(1.toFloat)) + checkDataset(Seq(Queue(1.toByte)).toDS(), Queue(1.toByte)) + checkDataset(Seq(Queue(1.toShort)).toDS(), Queue(1.toShort)) + checkDataset(Seq(Queue(true)).toDS(), Queue(true)) + checkDataset(Seq(Queue("test")).toDS(), Queue("test")) + checkDataset(Seq(Queue(Tuple1(1))).toDS(), Queue(Tuple1(1))) + + checkDataset(Seq(ArrayBuffer(1)).toDS(), ArrayBuffer(1)) + checkDataset(Seq(ArrayBuffer(1.toLong)).toDS(), ArrayBuffer(1.toLong)) + checkDataset(Seq(ArrayBuffer(1.toDouble)).toDS(), ArrayBuffer(1.toDouble)) + checkDataset(Seq(ArrayBuffer(1.toFloat)).toDS(), ArrayBuffer(1.toFloat)) + checkDataset(Seq(ArrayBuffer(1.toByte)).toDS(), ArrayBuffer(1.toByte)) + checkDataset(Seq(ArrayBuffer(1.toShort)).toDS(), ArrayBuffer(1.toShort)) + checkDataset(Seq(ArrayBuffer(true)).toDS(), ArrayBuffer(true)) + checkDataset(Seq(ArrayBuffer("test")).toDS(), ArrayBuffer("test")) + checkDataset(Seq(ArrayBuffer(Tuple1(1))).toDS(), ArrayBuffer(Tuple1(1))) + } + + test("sequence and product combinations") { + // Case classes + checkDataset(Seq(SeqClass(Seq(1))).toDS(), SeqClass(Seq(1))) + checkDataset(Seq(Seq(SeqClass(Seq(1)))).toDS(), Seq(SeqClass(Seq(1)))) + checkDataset(Seq(List(SeqClass(Seq(1)))).toDS(), List(SeqClass(Seq(1)))) + checkDataset(Seq(Queue(SeqClass(Seq(1)))).toDS(), Queue(SeqClass(Seq(1)))) + + checkDataset(Seq(ListClass(List(1))).toDS(), ListClass(List(1))) + checkDataset(Seq(Seq(ListClass(List(1)))).toDS(), Seq(ListClass(List(1)))) + checkDataset(Seq(List(ListClass(List(1)))).toDS(), List(ListClass(List(1)))) + checkDataset(Seq(Queue(ListClass(List(1)))).toDS(), Queue(ListClass(List(1)))) + + checkDataset(Seq(QueueClass(Queue(1))).toDS(), QueueClass(Queue(1))) + checkDataset(Seq(Seq(QueueClass(Queue(1)))).toDS(), Seq(QueueClass(Queue(1)))) + checkDataset(Seq(List(QueueClass(Queue(1)))).toDS(), List(QueueClass(Queue(1)))) + checkDataset(Seq(Queue(QueueClass(Queue(1)))).toDS(), Queue(QueueClass(Queue(1)))) + + val complex = ComplexClass(SeqClass(Seq(1)), ListClass(List(2)), QueueClass(Queue(3))) + checkDataset(Seq(complex).toDS(), complex) + checkDataset(Seq(Seq(complex)).toDS(), Seq(complex)) + checkDataset(Seq(List(complex)).toDS(), List(complex)) + checkDataset(Seq(Queue(complex)).toDS(), Queue(complex)) + + // Tuples + checkDataset(Seq(Seq(1) -> Seq(2)).toDS(), Seq(1) -> Seq(2)) + checkDataset(Seq(List(1) -> Queue(2)).toDS(), List(1) -> Queue(2)) + checkDataset(Seq(List(Seq("test1") -> List(Queue("test2")))).toDS(), + List(Seq("test1") -> List(Queue("test2")))) + + // Complex + checkDataset(Seq(ListClass(List(1)) -> Queue("test" -> SeqClass(Seq(2)))).toDS(), + ListClass(List(1)) -> Queue("test" -> SeqClass(Seq(2)))) + } + + test("arbitrary maps") { + checkDataset(Seq(Map(1 -> 2)).toDS(), Map(1 -> 2)) + checkDataset(Seq(Map(1.toLong -> 2.toLong)).toDS(), Map(1.toLong -> 2.toLong)) + checkDataset(Seq(Map(1.toDouble -> 2.toDouble)).toDS(), Map(1.toDouble -> 2.toDouble)) + checkDataset(Seq(Map(1.toFloat -> 2.toFloat)).toDS(), Map(1.toFloat -> 2.toFloat)) + checkDataset(Seq(Map(1.toByte -> 2.toByte)).toDS(), Map(1.toByte -> 2.toByte)) + checkDataset(Seq(Map(1.toShort -> 2.toShort)).toDS(), Map(1.toShort -> 2.toShort)) + checkDataset(Seq(Map(true -> false)).toDS(), Map(true -> false)) + checkDataset(Seq(Map("test1" -> "test2")).toDS(), Map("test1" -> "test2")) + checkDataset(Seq(Map(Tuple1(1) -> Tuple1(2))).toDS(), Map(Tuple1(1) -> Tuple1(2))) + checkDataset(Seq(Map(1 -> Tuple1(2))).toDS(), Map(1 -> Tuple1(2))) + checkDataset(Seq(Map("test" -> 2.toLong)).toDS(), Map("test" -> 2.toLong)) + + checkDataset(Seq(LHMap(1 -> 2)).toDS(), LHMap(1 -> 2)) + checkDataset(Seq(LHMap(1.toLong -> 2.toLong)).toDS(), LHMap(1.toLong -> 2.toLong)) + checkDataset(Seq(LHMap(1.toDouble -> 2.toDouble)).toDS(), LHMap(1.toDouble -> 2.toDouble)) + checkDataset(Seq(LHMap(1.toFloat -> 2.toFloat)).toDS(), LHMap(1.toFloat -> 2.toFloat)) + checkDataset(Seq(LHMap(1.toByte -> 2.toByte)).toDS(), LHMap(1.toByte -> 2.toByte)) + checkDataset(Seq(LHMap(1.toShort -> 2.toShort)).toDS(), LHMap(1.toShort -> 2.toShort)) + checkDataset(Seq(LHMap(true -> false)).toDS(), LHMap(true -> false)) + checkDataset(Seq(LHMap("test1" -> "test2")).toDS(), LHMap("test1" -> "test2")) + checkDataset(Seq(LHMap(Tuple1(1) -> Tuple1(2))).toDS(), LHMap(Tuple1(1) -> Tuple1(2))) + checkDataset(Seq(LHMap(1 -> Tuple1(2))).toDS(), LHMap(1 -> Tuple1(2))) + checkDataset(Seq(LHMap("test" -> 2.toLong)).toDS(), LHMap("test" -> 2.toLong)) + } + + ignore("SPARK-19104: map and product combinations") { + // Case classes + checkDataset(Seq(MapClass(Map(1 -> 2))).toDS(), MapClass(Map(1 -> 2))) + checkDataset(Seq(Map(1 -> MapClass(Map(2 -> 3)))).toDS(), Map(1 -> MapClass(Map(2 -> 3)))) + checkDataset(Seq(Map(MapClass(Map(1 -> 2)) -> 3)).toDS(), Map(MapClass(Map(1 -> 2)) -> 3)) + checkDataset(Seq(Map(MapClass(Map(1 -> 2)) -> MapClass(Map(3 -> 4)))).toDS(), + Map(MapClass(Map(1 -> 2)) -> MapClass(Map(3 -> 4)))) + checkDataset(Seq(LHMap(1 -> MapClass(Map(2 -> 3)))).toDS(), LHMap(1 -> MapClass(Map(2 -> 3)))) + checkDataset(Seq(LHMap(MapClass(Map(1 -> 2)) -> 3)).toDS(), LHMap(MapClass(Map(1 -> 2)) -> 3)) + checkDataset(Seq(LHMap(MapClass(Map(1 -> 2)) -> MapClass(Map(3 -> 4)))).toDS(), + LHMap(MapClass(Map(1 -> 2)) -> MapClass(Map(3 -> 4)))) + + checkDataset(Seq(LHMapClass(LHMap(1 -> 2))).toDS(), LHMapClass(LHMap(1 -> 2))) + checkDataset(Seq(Map(1 -> LHMapClass(LHMap(2 -> 3)))).toDS(), + Map(1 -> LHMapClass(LHMap(2 -> 3)))) + checkDataset(Seq(Map(LHMapClass(LHMap(1 -> 2)) -> 3)).toDS(), + Map(LHMapClass(LHMap(1 -> 2)) -> 3)) + checkDataset(Seq(Map(LHMapClass(LHMap(1 -> 2)) -> LHMapClass(LHMap(3 -> 4)))).toDS(), + Map(LHMapClass(LHMap(1 -> 2)) -> LHMapClass(LHMap(3 -> 4)))) + checkDataset(Seq(LHMap(1 -> LHMapClass(LHMap(2 -> 3)))).toDS(), + LHMap(1 -> LHMapClass(LHMap(2 -> 3)))) + checkDataset(Seq(LHMap(LHMapClass(LHMap(1 -> 2)) -> 3)).toDS(), + LHMap(LHMapClass(LHMap(1 -> 2)) -> 3)) + checkDataset(Seq(LHMap(LHMapClass(LHMap(1 -> 2)) -> LHMapClass(LHMap(3 -> 4)))).toDS(), + LHMap(LHMapClass(LHMap(1 -> 2)) -> LHMapClass(LHMap(3 -> 4)))) + + val complex = ComplexMapClass(MapClass(Map(1 -> 2)), LHMapClass(LHMap(3 -> 4))) + checkDataset(Seq(complex).toDS(), complex) + checkDataset(Seq(Map(1 -> complex)).toDS(), Map(1 -> complex)) + checkDataset(Seq(Map(complex -> 5)).toDS(), Map(complex -> 5)) + checkDataset(Seq(Map(complex -> complex)).toDS(), Map(complex -> complex)) + checkDataset(Seq(LHMap(1 -> complex)).toDS(), LHMap(1 -> complex)) + checkDataset(Seq(LHMap(complex -> 5)).toDS(), LHMap(complex -> 5)) + checkDataset(Seq(LHMap(complex -> complex)).toDS(), LHMap(complex -> complex)) + + // Tuples + checkDataset(Seq(Map(1 -> 2) -> Map(3 -> 4)).toDS(), Map(1 -> 2) -> Map(3 -> 4)) + checkDataset(Seq(LHMap(1 -> 2) -> Map(3 -> 4)).toDS(), LHMap(1 -> 2) -> Map(3 -> 4)) + checkDataset(Seq(Map(1 -> 2) -> LHMap(3 -> 4)).toDS(), Map(1 -> 2) -> LHMap(3 -> 4)) + checkDataset(Seq(LHMap(1 -> 2) -> LHMap(3 -> 4)).toDS(), LHMap(1 -> 2) -> LHMap(3 -> 4)) + checkDataset(Seq(LHMap((Map("test1" -> 1) -> 2) -> (3 -> LHMap(4 -> "test2")))).toDS(), + LHMap((Map("test1" -> 1) -> 2) -> (3 -> LHMap(4 -> "test2")))) + + // Complex + checkDataset(Seq(LHMapClass(LHMap(1 -> 2)) -> LHMap("test" -> MapClass(Map(3 -> 4)))).toDS(), + LHMapClass(LHMap(1 -> 2)) -> LHMap("test" -> MapClass(Map(3 -> 4)))) + } + + test("arbitrary sets") { + checkDataset(Seq(Set(1, 2, 3, 4)).toDS(), Set(1, 2, 3, 4)) + checkDataset(Seq(Set(1.toLong, 2.toLong)).toDS(), Set(1.toLong, 2.toLong)) + checkDataset(Seq(Set(1.toDouble, 2.toDouble)).toDS(), Set(1.toDouble, 2.toDouble)) + checkDataset(Seq(Set(1.toFloat, 2.toFloat)).toDS(), Set(1.toFloat, 2.toFloat)) + checkDataset(Seq(Set(1.toByte, 2.toByte)).toDS(), Set(1.toByte, 2.toByte)) + checkDataset(Seq(Set(1.toShort, 2.toShort)).toDS(), Set(1.toShort, 2.toShort)) + checkDataset(Seq(Set(true, false)).toDS(), Set(true, false)) + checkDataset(Seq(Set("test1", "test2")).toDS(), Set("test1", "test2")) + checkDataset(Seq(Set(Tuple1(1), Tuple1(2))).toDS(), Set(Tuple1(1), Tuple1(2))) + + checkDataset(Seq(HSet(1, 2)).toDS(), HSet(1, 2)) + checkDataset(Seq(HSet(1.toLong, 2.toLong)).toDS(), HSet(1.toLong, 2.toLong)) + checkDataset(Seq(HSet(1.toDouble, 2.toDouble)).toDS(), HSet(1.toDouble, 2.toDouble)) + checkDataset(Seq(HSet(1.toFloat, 2.toFloat)).toDS(), HSet(1.toFloat, 2.toFloat)) + checkDataset(Seq(HSet(1.toByte, 2.toByte)).toDS(), HSet(1.toByte, 2.toByte)) + checkDataset(Seq(HSet(1.toShort, 2.toShort)).toDS(), HSet(1.toShort, 2.toShort)) + checkDataset(Seq(HSet(true, false)).toDS(), HSet(true, false)) + checkDataset(Seq(HSet("test1", "test2")).toDS(), HSet("test1", "test2")) + checkDataset(Seq(HSet(Tuple1(1), Tuple1(2))).toDS(), HSet(Tuple1(1), Tuple1(2))) + + checkDataset(Seq(Seq(Some(1), None), Seq(Some(2))).toDF("c").as[Set[Integer]], + Seq(Set[Integer](1, null), Set[Integer](2)): _*) + } + + test("nested sequences") { + checkDataset(Seq(Seq(Seq(1))).toDS(), Seq(Seq(1))) + checkDataset(Seq(List(Queue(1))).toDS(), List(Queue(1))) + } + + test("nested maps") { + checkDataset(Seq(Map(1 -> LHMap(2 -> 3))).toDS(), Map(1 -> LHMap(2 -> 3))) + checkDataset(Seq(LHMap(Map(1 -> 2) -> 3)).toDS(), LHMap(Map(1 -> 2) -> 3)) + } + + test("nested set") { + checkDataset(Seq(Set(HSet(1, 2), HSet(3, 4))).toDS(), Set(HSet(1, 2), HSet(3, 4))) + checkDataset(Seq(HSet(Set(1, 2), Set(3, 4))).toDS(), HSet(Set(1, 2), Set(3, 4))) + } + + test("package objects") { + import packageobject._ + checkDataset(Seq(PackageClass(1)).toDS(), PackageClass(1)) + } + + test("SPARK-19104: Lambda variables in ExternalMapToCatalyst should be global") { + val data = Seq.tabulate(10)(i => NestedData(1, Map("key" -> InnerData("name", i + 100)))) + val ds = spark.createDataset(data) + checkDataset(ds, data: _*) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala new file mode 100644 index 000000000000..68f7de047b39 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import com.esotericsoftware.kryo.{Kryo, Serializer} +import com.esotericsoftware.kryo.io.{Input, Output} + +import org.apache.spark.SparkConf +import org.apache.spark.serializer.KryoRegistrator +import org.apache.spark.sql.test.SharedSQLContext + +/** + * Test suite to test Kryo custom registrators. + */ +class DatasetSerializerRegistratorSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + + override protected def sparkConf: SparkConf = { + // Make sure we use the KryoRegistrator + super.sparkConf.set("spark.kryo.registrator", TestRegistrator().getClass.getCanonicalName) + } + + test("Kryo registrator") { + implicit val kryoEncoder = Encoders.kryo[KryoData] + val ds = Seq(KryoData(1), KryoData(2)).toDS() + assert(ds.collect().toSet == Set(KryoData(0), KryoData(0))) + } + +} + +/** Used to test user provided registrator. */ +class TestRegistrator extends KryoRegistrator { + override def registerClasses(kryo: Kryo): Unit = + kryo.register(classOf[KryoData], new ZeroKryoDataSerializer()) +} + +object TestRegistrator { + def apply(): TestRegistrator = new TestRegistrator() +} + +/** + * A `Serializer` that takes a [[KryoData]] and serializes it as KryoData(0). + */ +class ZeroKryoDataSerializer extends Serializer[KryoData] { + override def write(kryo: Kryo, output: Output, t: KryoData): Unit = { + output.writeInt(0) + } + + override def read(kryo: Kryo, input: Input, aClass: Class[KryoData]): KryoData = { + KryoData(input.readInt()) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 3e9b621cfd67..5015f3709f13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -17,40 +17,128 @@ package org.apache.spark.sql -import scala.language.postfixOps +import java.io.{Externalizable, ObjectInput, ObjectOutput} +import java.sql.{Date, Timestamp} +import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder} +import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} +import org.apache.spark.sql.catalyst.util.sideBySide +import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec} +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchange} +import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ -case class ClassData(a: String, b: Int) +case class TestDataPoint(x: Int, y: Double, s: String, t: TestDataPoint2) +case class TestDataPoint2(x: Int, s: String) + +object TestForTypeAlias { + type TwoInt = (Int, Int) + type ThreeInt = (TwoInt, Int) + type SeqOfTwoInt = Seq[TwoInt] + + def tupleTypeAlias: TwoInt = (1, 1) + def nestedTupleTypeAlias: ThreeInt = ((1, 1), 2) + def seqOfTupleTypeAlias: SeqOfTwoInt = Seq((1, 1), (2, 2)) +} class DatasetSuite extends QueryTest with SharedSQLContext { import testImplicits._ - test("toDS") { - val data = Seq(("a", 1) , ("b", 2), ("c", 3)) + private implicit val ordering = Ordering.by((c: ClassData) => c.a -> c.b) + + test("checkAnswer should compare map correctly") { + val data = Seq((1, "2", Map(1 -> 2, 2 -> 1))) checkAnswer( + data.toDF(), + Seq(Row(1, "2", Map(2 -> 1, 1 -> 2)))) + } + + test("toDS") { + val data = Seq(("a", 1), ("b", 2), ("c", 3)) + checkDataset( data.toDS(), data: _*) } test("toDS with RDD") { val ds = sparkContext.makeRDD(Seq("a", "b", "c"), 3).toDS() - checkAnswer( + checkDataset( ds.mapPartitions(_ => Iterator(1)), 1, 1, 1) } + test("emptyDataset") { + val ds = spark.emptyDataset[Int] + assert(ds.count() == 0L) + assert(ds.collect() sameElements Array.empty[Int]) + } + + test("range") { + assert(spark.range(10).map(_ + 1).reduce(_ + _) == 55) + assert(spark.range(10).map{ case i: java.lang.Long => i + 1 }.reduce(_ + _) == 55) + assert(spark.range(0, 10).map(_ + 1).reduce(_ + _) == 55) + assert(spark.range(0, 10).map{ case i: java.lang.Long => i + 1 }.reduce(_ + _) == 55) + assert(spark.range(0, 10, 1, 2).map(_ + 1).reduce(_ + _) == 55) + assert(spark.range(0, 10, 1, 2).map{ case i: java.lang.Long => i + 1 }.reduce(_ + _) == 55) + } + + test("SPARK-12404: Datatype Helper Serializability") { + val ds = sparkContext.parallelize(( + new Timestamp(0), + new Date(0), + java.math.BigDecimal.valueOf(1), + scala.math.BigDecimal(1)) :: Nil).toDS() + + ds.collect() + } + + test("collect, first, and take should use encoders for serialization") { + val item = NonSerializableCaseClass("abcd") + val ds = Seq(item).toDS() + assert(ds.collect().head == item) + assert(ds.collectAsList().get(0) == item) + assert(ds.first() == item) + assert(ds.take(1).head == item) + assert(ds.takeAsList(1).get(0) == item) + assert(ds.toLocalIterator().next() === item) + } + + test("coalesce, repartition") { + val data = (1 to 100).map(i => ClassData(i.toString, i)) + val ds = data.toDS() + + intercept[IllegalArgumentException] { + ds.coalesce(0) + } + + intercept[IllegalArgumentException] { + ds.repartition(0) + } + + assert(ds.repartition(10).rdd.partitions.length == 10) + checkDatasetUnorderly( + ds.repartition(10), + data: _*) + + assert(ds.coalesce(1).rdd.partitions.length == 1) + checkDatasetUnorderly( + ds.coalesce(1), + data: _*) + } + test("as tuple") { val data = Seq(("a", 1), ("b", 2)).toDF("a", "b") - checkAnswer( + checkDataset( data.as[(String, Int)], ("a", 1), ("b", 2)) } test("as case class / collect") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDF("a", "b").as[ClassData] - checkAnswer( + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDF("a", "b").as[ClassData] + checkDataset( ds, ClassData("a", 1), ClassData("b", 2), ClassData("c", 3)) assert(ds.collect().head == ClassData("a", 1)) @@ -61,23 +149,79 @@ class DatasetSuite extends QueryTest with SharedSQLContext { assert(ds.collect() === Array(ClassData("a", 1), ClassData("b", 2), ClassData("c", 3))) } + test("as case class - take") { + val ds = Seq((1, "a"), (2, "b"), (3, "c")).toDF("b", "a").as[ClassData] + assert(ds.take(2) === Array(ClassData("a", 1), ClassData("b", 2))) + } + + test("as seq of case class - reorder fields by name") { + val df = spark.range(3).select(array(struct($"id".cast("int").as("b"), lit("a").as("a")))) + val ds = df.as[Seq[ClassData]] + assert(ds.collect() === Array( + Seq(ClassData("a", 0)), + Seq(ClassData("a", 1)), + Seq(ClassData("a", 2)))) + } + test("map") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() - checkAnswer( + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + checkDataset( ds.map(v => (v._1, v._2 + 1)), ("a", 2), ("b", 3), ("c", 4)) } + test("map with type change with the exact matched number of attributes") { + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + + checkDataset( + ds.map(identity[(String, Int)]) + .as[OtherTuple] + .map(identity[OtherTuple]), + OtherTuple("a", 1), OtherTuple("b", 2), OtherTuple("c", 3)) + } + + test("map with type change with less attributes") { + val ds = Seq(("a", 1, 3), ("b", 2, 4), ("c", 3, 5)).toDS() + + checkDataset( + ds.as[OtherTuple] + .map(identity[OtherTuple]), + OtherTuple("a", 1), OtherTuple("b", 2), OtherTuple("c", 3)) + } + + test("map and group by with class data") { + // We inject a group by here to make sure this test case is future proof + // when we implement better pipelining and local execution mode. + val ds: Dataset[(ClassData, Long)] = Seq(ClassData("one", 1), ClassData("two", 2)).toDS() + .map(c => ClassData(c.a, c.b + 1)) + .groupByKey(p => p).count() + + checkDatasetUnorderly( + ds, + (ClassData("one", 2), 1L), (ClassData("two", 3), 1L)) + } + test("select") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() - checkAnswer( + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + checkDataset( ds.select(expr("_2 + 1").as[Int]), 2, 3, 4) } + test("SPARK-16853: select, case class and tuple") { + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + checkDataset( + ds.select(expr("struct(_2, _2)").as[(Int, Int)]): Dataset[(Int, Int)], + (1, 1), (2, 2), (3, 3)) + + checkDataset( + ds.select(expr("named_struct('a', _1, 'b', _2)").as[ClassData]): Dataset[ClassData], + ClassData("a", 1), ClassData("b", 2), ClassData("c", 3)) + } + test("select 2") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() - checkAnswer( + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + checkDataset( ds.select( expr("_1").as[String], expr("_2").as[Int]) : Dataset[(String, Int)], @@ -85,8 +229,8 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select 2, primitive and tuple") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() - checkAnswer( + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + checkDataset( ds.select( expr("_1").as[String], expr("struct(_2, _2)").as[(Int, Int)]), @@ -94,8 +238,8 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select 2, primitive and class") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() - checkAnswer( + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + checkDataset( ds.select( expr("_1").as[String], expr("named_struct('a', _1, 'b', _2)").as[ClassData]), @@ -103,68 +247,146 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select 2, primitive and class, fields reordered") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() - checkDecoding( + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + checkDataset( ds.select( expr("_1").as[String], expr("named_struct('b', _2, 'a', _1)").as[ClassData]), ("a", ClassData("a", 1)), ("b", ClassData("b", 2)), ("c", ClassData("c", 3))) } + test("REGEX column specification") { + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + var e = intercept[AnalysisException] { + ds.select(expr("`(_1)?+.+`").as[Int]) + }.getMessage + assert(e.contains("cannot resolve '`(_1)?+.+`'")) + + e = intercept[AnalysisException] { + ds.select(expr("`(_1|_2)`").as[Int]) + }.getMessage + assert(e.contains("cannot resolve '`(_1|_2)`'")) + + e = intercept[AnalysisException] { + ds.select(ds("`(_1)?+.+`")) + }.getMessage + assert(e.contains("Cannot resolve column name \"`(_1)?+.+`\"")) + + e = intercept[AnalysisException] { + ds.select(ds("`(_1|_2)`")) + }.getMessage + assert(e.contains("Cannot resolve column name \"`(_1|_2)`\"")) + } + + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "true") { + checkDataset( + ds.select(ds.col("_2")).as[Int], + 1, 2, 3) + + checkDataset( + ds.select(ds.colRegex("`(_1)?+.+`")).as[Int], + 1, 2, 3) + + checkDataset( + ds.select(ds("`(_1|_2)`")) + .select(expr("named_struct('a', _1, 'b', _2)").as[ClassData]), + ClassData("a", 1), ClassData("b", 2), ClassData("c", 3)) + + checkDataset( + ds.alias("g") + .select(ds("g.`(_1|_2)`")) + .select(expr("named_struct('a', _1, 'b', _2)").as[ClassData]), + ClassData("a", 1), ClassData("b", 2), ClassData("c", 3)) + + checkDataset( + ds.select(ds("`(_1)?+.+`")) + .select(expr("_2").as[Int]), + 1, 2, 3) + + checkDataset( + ds.alias("g") + .select(ds("g.`(_1)?+.+`")) + .select(expr("_2").as[Int]), + 1, 2, 3) + + checkDataset( + ds.select(expr("`(_1)?+.+`").as[Int]), + 1, 2, 3) + val m = ds.select(expr("`(_1|_2)`")) + + checkDataset( + ds.select(expr("`(_1|_2)`")) + .select(expr("named_struct('a', _1, 'b', _2)").as[ClassData]), + ClassData("a", 1), ClassData("b", 2), ClassData("c", 3)) + + checkDataset( + ds.alias("g") + .select(expr("g.`(_1)?+.+`").as[Int]), + 1, 2, 3) + + checkDataset( + ds.alias("g") + .select(expr("g.`(_1|_2)`")) + .select(expr("named_struct('a', _1, 'b', _2)").as[ClassData]), + ClassData("a", 1), ClassData("b", 2), ClassData("c", 3)) + } + } + test("filter") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() - checkAnswer( + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + checkDataset( ds.filter(_._1 == "b"), ("b", 2)) } + test("filter and then select") { + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + checkDataset( + ds.filter(_._1 == "b").select(expr("_1").as[String]), + "b") + } + + test("SPARK-15632: typed filter should preserve the underlying logical schema") { + val ds = spark.range(10) + val ds2 = ds.filter(_ > 3) + assert(ds.schema.equals(ds2.schema)) + } + test("foreach") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() - val acc = sparkContext.accumulator(0) - ds.foreach(v => acc += v._2) + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + val acc = sparkContext.longAccumulator + ds.foreach(v => acc.add(v._2)) assert(acc.value == 6) } test("foreachPartition") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() - val acc = sparkContext.accumulator(0) - ds.foreachPartition(_.foreach(v => acc += v._2)) + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + val acc = sparkContext.longAccumulator + ds.foreachPartition((it: Iterator[(String, Int)]) => it.foreach(v => acc.add(v._2))) assert(acc.value == 6) } test("reduce") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() - assert(ds.reduce((a, b) => ("sum", a._2 + b._2)) == ("sum", 6)) - } - - test("fold") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() - assert(ds.fold(("", 0))((a, b) => ("sum", a._2 + b._2)) == ("sum", 6)) + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + assert(ds.reduce((a, b) => ("sum", a._2 + b._2)) == (("sum", 6))) } test("joinWith, flat schema") { val ds1 = Seq(1, 2, 3).toDS().as("a") val ds2 = Seq(1, 2).toDS().as("b") - checkAnswer( - ds1.joinWith(ds2, $"a.value" === $"b.value"), + checkDataset( + ds1.joinWith(ds2, $"a.value" === $"b.value", "inner"), (1, 1), (2, 2)) } - test("joinWith, expression condition") { - val ds1 = Seq(ClassData("a", 1), ClassData("b", 2)).toDS() - val ds2 = Seq(("a", 1), ("b", 2)).toDS() - - checkAnswer( - ds1.joinWith(ds2, $"_1" === $"a"), - (ClassData("a", 1), ("a", 1)), (ClassData("b", 2), ("b", 2))) - } - test("joinWith tuple with primitive, expression") { val ds1 = Seq(1, 1, 2).toDS() val ds2 = Seq(("a", 1), ("b", 2)).toDS() - checkAnswer( + checkDataset( ds1.joinWith(ds2, $"value" === $"_2"), (1, ("a", 1)), (1, ("a", 1)), (2, ("b", 2))) } @@ -183,98 +405,1024 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val ds2 = Seq(("a", 1), ("b", 2)).toDS().as("b") val ds3 = Seq(("a", 1), ("b", 2)).toDS().as("c") - checkAnswer( + checkDataset( ds1.joinWith(ds2, $"a._2" === $"b._2").as("ab").joinWith(ds3, $"ab._1._2" === $"c._2"), ((("a", 1), ("a", 1)), ("a", 1)), ((("b", 2), ("b", 2)), ("b", 2))) + } + + test("joinWith join types") { + val ds1 = Seq(1, 2, 3).toDS().as("a") + val ds2 = Seq(1, 2).toDS().as("b") + val e1 = intercept[AnalysisException] { + ds1.joinWith(ds2, $"a.value" === $"b.value", "left_semi") + }.getMessage + assert(e1.contains("Invalid join type in joinWith: " + LeftSemi.sql)) + + val e2 = intercept[AnalysisException] { + ds1.joinWith(ds2, $"a.value" === $"b.value", "left_anti") + }.getMessage + assert(e2.contains("Invalid join type in joinWith: " + LeftAnti.sql)) } test("groupBy function, keys") { val ds = Seq(("a", 1), ("b", 1)).toDS() - val grouped = ds.groupBy(v => (1, v._2)) - checkAnswer( + val grouped = ds.groupByKey(v => (1, v._2)) + checkDatasetUnorderly( grouped.keys, (1, 1)) } - test("groupBy function, mapGroups") { + test("groupBy function, map") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() - val grouped = ds.groupBy(v => (v._1, "word")) - val agged = grouped.mapGroups { case (g, iter) => - Iterator((g._1, iter.map(_._2).sum)) - } + val grouped = ds.groupByKey(v => (v._1, "word")) + val agged = grouped.mapGroups { case (g, iter) => (g._1, iter.map(_._2).sum) } - checkAnswer( + checkDatasetUnorderly( agged, ("a", 30), ("b", 3), ("c", 1)) } - test("groupBy columns, mapGroups") { + test("groupBy function, flatMap") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() - val grouped = ds.groupBy($"_1") - val agged = grouped.mapGroups { case (g, iter) => - Iterator((g.getString(0), iter.map(_._2).sum)) + val grouped = ds.groupByKey(v => (v._1, "word")) + val agged = grouped.flatMapGroups { case (g, iter) => + Iterator(g._1, iter.map(_._2).sum.toString) } - checkAnswer( + checkDatasetUnorderly( agged, - ("a", 30), ("b", 3), ("c", 1)) + "a", "30", "b", "3", "c", "1") } - test("groupBy columns asKey, mapGroups") { + test("groupBy function, mapValues, flatMap") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() - val grouped = ds.groupBy($"_1").asKey[String] - val agged = grouped.mapGroups { case (g, iter) => - Iterator((g, iter.map(_._2).sum)) - } + val keyValue = ds.groupByKey(_._1).mapValues(_._2) + val agged = keyValue.mapGroups { case (g, iter) => (g, iter.sum) } + checkDataset(agged, ("a", 30), ("b", 3), ("c", 1)) - checkAnswer( + val keyValue1 = ds.groupByKey(t => (t._1, "key")).mapValues(t => (t._2, "value")) + val agged1 = keyValue1.mapGroups { case (g, iter) => (g._1, iter.map(_._1).sum) } + checkDataset(agged, ("a", 30), ("b", 3), ("c", 1)) + } + + test("groupBy function, reduce") { + val ds = Seq("abc", "xyz", "hello").toDS() + val agged = ds.groupByKey(_.length).reduceGroups(_ + _) + + checkDatasetUnorderly( agged, - ("a", 30), ("b", 3), ("c", 1)) + 3 -> "abcxyz", 5 -> "hello") } - test("groupBy columns asKey tuple, mapGroups") { + test("groupBy single field class, count") { + val ds = Seq("abc", "xyz", "hello").toDS() + val count = ds.groupByKey(s => Tuple1(s.length)).count() + + checkDataset( + count, + (Tuple1(3), 2L), (Tuple1(5), 1L) + ) + } + + test("typed aggregation: expr") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() - val grouped = ds.groupBy($"_1", lit(1)).asKey[(String, Int)] - val agged = grouped.mapGroups { case (g, iter) => - Iterator((g, iter.map(_._2).sum)) - } - checkAnswer( - agged, - (("a", 1), 30), (("b", 1), 3), (("c", 1), 1)) + checkDatasetUnorderly( + ds.groupByKey(_._1).agg(sum("_2").as[Long]), + ("a", 30L), ("b", 3L), ("c", 1L)) } - test("groupBy columns asKey class, mapGroups") { + test("typed aggregation: expr, expr") { val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() - val grouped = ds.groupBy($"_1".as("a"), lit(1).as("b")).asKey[ClassData] - val agged = grouped.mapGroups { case (g, iter) => - Iterator((g, iter.map(_._2).sum)) - } - checkAnswer( - agged, - (ClassData("a", 1), 30), (ClassData("b", 1), 3), (ClassData("c", 1), 1)) + checkDatasetUnorderly( + ds.groupByKey(_._1).agg(sum("_2").as[Long], sum($"_2" + 1).as[Long]), + ("a", 30L, 32L), ("b", 3L, 5L), ("c", 1L, 2L)) + } + + test("typed aggregation: expr, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDatasetUnorderly( + ds.groupByKey(_._1).agg(sum("_2").as[Long], sum($"_2" + 1).as[Long], count("*")), + ("a", 30L, 32L, 2L), ("b", 3L, 5L, 2L), ("c", 1L, 2L, 1L)) + } + + test("typed aggregation: expr, expr, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDatasetUnorderly( + ds.groupByKey(_._1).agg( + sum("_2").as[Long], + sum($"_2" + 1).as[Long], + count("*").as[Long], + avg("_2").as[Double]), + ("a", 30L, 32L, 2L, 15.0), ("b", 3L, 5L, 2L, 1.5), ("c", 1L, 2L, 1L, 1.0)) } test("cogroup") { val ds1 = Seq(1 -> "a", 3 -> "abc", 5 -> "hello", 3 -> "foo").toDS() val ds2 = Seq(2 -> "q", 3 -> "w", 5 -> "e", 5 -> "r").toDS() - val cogrouped = ds1.groupBy(_._1).cogroup(ds2.groupBy(_._1)) { case (key, data1, data2) => + val cogrouped = ds1.groupByKey(_._1).cogroup(ds2.groupByKey(_._1)) { case (key, data1, data2) => Iterator(key -> (data1.map(_._2).mkString + "#" + data2.map(_._2).mkString)) } - checkAnswer( + checkDatasetUnorderly( cogrouped, 1 -> "a#", 2 -> "#q", 3 -> "abcfoo#w", 5 -> "hello#er") } + test("cogroup with complex data") { + val ds1 = Seq(1 -> ClassData("a", 1), 2 -> ClassData("b", 2)).toDS() + val ds2 = Seq(2 -> ClassData("c", 3), 3 -> ClassData("d", 4)).toDS() + val cogrouped = ds1.groupByKey(_._1).cogroup(ds2.groupByKey(_._1)) { case (key, data1, data2) => + Iterator(key -> (data1.map(_._2.a).mkString + data2.map(_._2.a).mkString)) + } + + checkDatasetUnorderly( + cogrouped, + 1 -> "a", 2 -> "bc", 3 -> "d") + } + + test("sample with replacement") { + val n = 100 + val data = sparkContext.parallelize(1 to n, 2).toDS() + checkDataset( + data.sample(withReplacement = true, 0.05, seed = 13), + 5, 10, 52, 73) + } + + test("sample without replacement") { + val n = 100 + val data = sparkContext.parallelize(1 to n, 2).toDS() + checkDataset( + data.sample(withReplacement = false, 0.05, seed = 13), + 3, 17, 27, 58, 62) + } + + test("sample fraction should not be negative with replacement") { + val data = sparkContext.parallelize(1 to 2, 1).toDS() + val errMsg = intercept[IllegalArgumentException] { + data.sample(withReplacement = true, -0.1, 0) + }.getMessage + assert(errMsg.contains("Sampling fraction (-0.1) must be nonnegative with replacement")) + + // Sampling fraction can be greater than 1 with replacement. + checkDataset( + data.sample(withReplacement = true, 1.05, seed = 13), + 1, 2) + } + + test("sample fraction should be on interval [0, 1] without replacement") { + val data = sparkContext.parallelize(1 to 2, 1).toDS() + val errMsg1 = intercept[IllegalArgumentException] { + data.sample(withReplacement = false, -0.1, 0) + }.getMessage() + assert(errMsg1.contains( + "Sampling fraction (-0.1) must be on interval [0, 1] without replacement")) + + val errMsg2 = intercept[IllegalArgumentException] { + data.sample(withReplacement = false, 1.1, 0) + }.getMessage() + assert(errMsg2.contains( + "Sampling fraction (1.1) must be on interval [0, 1] without replacement")) + } + + test("SPARK-16686: Dataset.sample with seed results shouldn't depend on downstream usage") { + val simpleUdf = udf((n: Int) => { + require(n != 1, "simpleUdf shouldn't see id=1!") + 1 + }) + + val df = Seq( + (0, "string0"), + (1, "string1"), + (2, "string2"), + (3, "string3"), + (4, "string4"), + (5, "string5"), + (6, "string6"), + (7, "string7"), + (8, "string8"), + (9, "string9") + ).toDF("id", "stringData") + val sampleDF = df.sample(false, 0.7, 50) + // After sampling, sampleDF doesn't contain id=1. + assert(!sampleDF.select("id").collect.contains(1)) + // simpleUdf should not encounter id=1. + checkAnswer(sampleDF.select(simpleUdf($"id")), List.fill(sampleDF.count.toInt)(Row(1))) + } + test("SPARK-11436: we should rebind right encoder when join 2 datasets") { val ds1 = Seq("1", "2").toDS().as("a") val ds2 = Seq(2, 3).toDS().as("b") val joined = ds1.joinWith(ds2, $"a.value" === $"b.value") - checkAnswer(joined, ("2", 2)) + checkDataset(joined, ("2", 2)) + } + + test("self join") { + val ds = Seq("1", "2").toDS().as("a") + val joined = ds.joinWith(ds, lit(true), "cross") + checkDataset(joined, ("1", "1"), ("1", "2"), ("2", "1"), ("2", "2")) + } + + test("toString") { + val ds = Seq((1, 2)).toDS() + assert(ds.toString == "[_1: int, _2: int]") + } + + test("Kryo encoder") { + implicit val kryoEncoder = Encoders.kryo[KryoData] + val ds = Seq(KryoData(1), KryoData(2)).toDS() + + assert(ds.groupByKey(p => p).count().collect().toSet == + Set((KryoData(1), 1L), (KryoData(2), 1L))) + } + + test("Kryo encoder self join") { + implicit val kryoEncoder = Encoders.kryo[KryoData] + val ds = Seq(KryoData(1), KryoData(2)).toDS() + assert(ds.joinWith(ds, lit(true), "cross").collect().toSet == + Set( + (KryoData(1), KryoData(1)), + (KryoData(1), KryoData(2)), + (KryoData(2), KryoData(1)), + (KryoData(2), KryoData(2)))) + } + + test("Kryo encoder: check the schema mismatch when converting DataFrame to Dataset") { + implicit val kryoEncoder = Encoders.kryo[KryoData] + val df = Seq((1)).toDF("a") + val e = intercept[AnalysisException] { + df.as[KryoData] + }.message + assert(e.contains("cannot cast IntegerType to BinaryType")) + } + + test("Java encoder") { + implicit val kryoEncoder = Encoders.javaSerialization[JavaData] + val ds = Seq(JavaData(1), JavaData(2)).toDS() + + assert(ds.groupByKey(p => p).count().collect().toSet == + Set((JavaData(1), 1L), (JavaData(2), 1L))) + } + + test("Java encoder self join") { + implicit val kryoEncoder = Encoders.javaSerialization[JavaData] + val ds = Seq(JavaData(1), JavaData(2)).toDS() + assert(ds.joinWith(ds, lit(true), "cross").collect().toSet == + Set( + (JavaData(1), JavaData(1)), + (JavaData(1), JavaData(2)), + (JavaData(2), JavaData(1)), + (JavaData(2), JavaData(2)))) + } + + test("SPARK-14696: implicit encoders for boxed types") { + assert(spark.range(1).map { i => i : java.lang.Long }.head == 0L) + } + + test("SPARK-11894: Incorrect results are returned when using null") { + val nullInt = null.asInstanceOf[java.lang.Integer] + val ds1 = Seq((nullInt, "1"), (new java.lang.Integer(22), "2")).toDS() + val ds2 = Seq((nullInt, "1"), (new java.lang.Integer(22), "2")).toDS() + + checkDataset( + ds1.joinWith(ds2, lit(true), "cross"), + ((nullInt, "1"), (nullInt, "1")), + ((nullInt, "1"), (new java.lang.Integer(22), "2")), + ((new java.lang.Integer(22), "2"), (nullInt, "1")), + ((new java.lang.Integer(22), "2"), (new java.lang.Integer(22), "2"))) + } + + test("change encoder with compatible schema") { + val ds = Seq(2 -> 2.toByte, 3 -> 3.toByte).toDF("a", "b").as[ClassData] + assert(ds.collect().toSeq == Seq(ClassData("2", 2), ClassData("3", 3))) + } + + test("verify mismatching field names fail with a good error") { + val ds = Seq(ClassData("a", 1)).toDS() + val e = intercept[AnalysisException] { + ds.as[ClassData2] + } + assert(e.getMessage.contains("cannot resolve '`c`' given input columns: [a, b]"), e.getMessage) + } + + test("runtime nullability check") { + val schema = StructType(Seq( + StructField("f", StructType(Seq( + StructField("a", StringType, nullable = true), + StructField("b", IntegerType, nullable = true) + )), nullable = true) + )) + + def buildDataset(rows: Row*): Dataset[NestedStruct] = { + val rowRDD = spark.sparkContext.parallelize(rows) + spark.createDataFrame(rowRDD, schema).as[NestedStruct] + } + + checkDataset( + buildDataset(Row(Row("hello", 1))), + NestedStruct(ClassData("hello", 1)) + ) + + // Shouldn't throw runtime exception when parent object (`ClassData`) is null + assert(buildDataset(Row(null)).collect() === Array(NestedStruct(null))) + + val message = intercept[RuntimeException] { + buildDataset(Row(Row("hello", null))).collect() + }.getMessage + + assert(message.contains("Null value appeared in non-nullable field")) + } + + test("SPARK-12478: top level null field") { + val ds0 = Seq(NestedStruct(null)).toDS() + checkDataset(ds0, NestedStruct(null)) + checkAnswer(ds0.toDF(), Row(null)) + + val ds1 = Seq(DeepNestedStruct(NestedStruct(null))).toDS() + checkDataset(ds1, DeepNestedStruct(NestedStruct(null))) + checkAnswer(ds1.toDF(), Row(Row(null))) + } + + test("support inner class in Dataset") { + val outer = new OuterClass + OuterScopes.addOuterScope(outer) + val ds = Seq(outer.InnerClass("1"), outer.InnerClass("2")).toDS() + checkDataset(ds.map(_.a), "1", "2") + } + + test("grouping key and grouped value has field with same name") { + val ds = Seq(ClassData("a", 1), ClassData("a", 2)).toDS() + val agged = ds.groupByKey(d => ClassNullableData(d.a, null)).mapGroups { + case (key, values) => key.a + values.map(_.b).sum + } + + checkDataset(agged, "a3") + } + + test("cogroup's left and right side has field with same name") { + val left = Seq(ClassData("a", 1), ClassData("b", 2)).toDS() + val right = Seq(ClassNullableData("a", 3), ClassNullableData("b", 4)).toDS() + val cogrouped = left.groupByKey(_.a).cogroup(right.groupByKey(_.a)) { + case (key, lData, rData) => Iterator(key + lData.map(_.b).sum + rData.map(_.b.toInt).sum) + } + + checkDataset(cogrouped, "a13", "b24") + } + + test("give nice error message when the real number of fields doesn't match encoder schema") { + val ds = Seq(ClassData("a", 1), ClassData("b", 2)).toDS() + + val message = intercept[AnalysisException] { + ds.as[(String, Int, Long)] + }.message + assert(message == + "Try to map struct to Tuple3, " + + "but failed as the number of fields does not line up.") + + val message2 = intercept[AnalysisException] { + ds.as[Tuple1[String]] + }.message + assert(message2 == + "Try to map struct to Tuple1, " + + "but failed as the number of fields does not line up.") + } + + test("SPARK-13440: Resolving option fields") { + val df = Seq(1, 2, 3).toDS() + val ds = df.as[Option[Int]] + checkDataset( + ds.filter(_ => true), + Some(1), Some(2), Some(3)) + } + + test("SPARK-13540 Dataset of nested class defined in Scala object") { + checkDataset( + Seq(OuterObject.InnerClass("foo")).toDS(), + OuterObject.InnerClass("foo")) + } + + test("SPARK-14000: case class with tuple type field") { + checkDataset( + Seq(TupleClass((1, "a"))).toDS(), + TupleClass((1, "a")) + ) + } + + test("isStreaming returns false for static Dataset") { + val data = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + assert(!data.isStreaming, "static Dataset returned true for 'isStreaming'.") + } + + test("isStreaming returns true for streaming Dataset") { + val data = MemoryStream[Int].toDS() + assert(data.isStreaming, "streaming Dataset returned false for 'isStreaming'.") + } + + test("isStreaming returns true after static and streaming Dataset join") { + val static = Seq(("a", 1), ("b", 2), ("c", 3)).toDF("a", "b") + val streaming = MemoryStream[Int].toDS().toDF("b") + val df = streaming.join(static, Seq("b")) + assert(df.isStreaming, "streaming Dataset returned false for 'isStreaming'.") + } + + test("SPARK-14554: Dataset.map may generate wrong java code for wide table") { + val wideDF = spark.range(10).select(Seq.tabulate(1000) {i => ('id + i).as(s"c$i")} : _*) + // Make sure the generated code for this plan can compile and execute. + checkDataset(wideDF.map(_.getLong(0)), 0L until 10 : _*) + } + + test("SPARK-14838: estimating sizeInBytes in operators with ObjectProducer shouldn't fail") { + val dataset = Seq( + (0, 3, 54f), + (0, 4, 44f), + (0, 5, 42f), + (1, 3, 39f), + (1, 5, 33f), + (1, 4, 26f), + (2, 3, 51f), + (2, 5, 45f), + (2, 4, 30f) + ).toDF("user", "item", "rating") + + val actual = dataset + .select("user", "item") + .as[(Int, Int)] + .groupByKey(_._1) + .mapGroups { case (src, ids) => (src, ids.map(_._2).toArray) } + .toDF("id", "actual") + + dataset.join(actual, dataset("user") === actual("id")).collect() + } + + test("SPARK-15097: implicits on dataset's spark can be imported") { + val dataset = Seq(1, 2, 3).toDS() + checkDataset(DatasetTransform.addOne(dataset), 2, 3, 4) + } + + test("dataset.rdd with generic case class") { + val ds = Seq(Generic(1, 1.0), Generic(2, 2.0)).toDS() + val ds2 = ds.map(g => Generic(g.id, g.value)) + assert(ds.rdd.map(r => r.id).count === 2) + assert(ds2.rdd.map(r => r.id).count === 2) + + val ds3 = ds.map(g => new java.lang.Long(g.id)) + assert(ds3.rdd.map(r => r).count === 2) + } + + test("runtime null check for RowEncoder") { + val schema = new StructType().add("i", IntegerType, nullable = false) + val df = spark.range(10).map(l => { + if (l % 5 == 0) { + Row(null) + } else { + Row(l) + } + })(RowEncoder(schema)) + + val message = intercept[Exception] { + df.collect() + }.getMessage + assert(message.contains("The 0th field 'i' of input row cannot be null")) + } + + test("row nullability mismatch") { + val schema = new StructType().add("a", StringType, true).add("b", StringType, false) + val rdd = spark.sparkContext.parallelize(Row(null, "123") :: Row("234", null) :: Nil) + val message = intercept[Exception] { + spark.createDataFrame(rdd, schema).collect() + }.getMessage + assert(message.contains("The 1th field 'b' of input row cannot be null")) + } + + test("createTempView") { + val dataset = Seq(1, 2, 3).toDS() + dataset.createOrReplaceTempView("tempView") + + // Overrides the existing temporary view with same name + // No exception should be thrown here. + dataset.createOrReplaceTempView("tempView") + + // Throws AnalysisException if temp view with same name already exists + val e = intercept[AnalysisException]( + dataset.createTempView("tempView")) + intercept[AnalysisException](dataset.createTempView("tempView")) + assert(e.message.contains("already exists")) + dataset.sparkSession.catalog.dropTempView("tempView") + } + + test("SPARK-15381: physical object operator should define `reference` correctly") { + val df = Seq(1 -> 2).toDF("a", "b") + checkAnswer(df.map(row => row)(RowEncoder(df.schema)).select("b", "a"), Row(2, 1)) + } + + private def checkShowString[T](ds: Dataset[T], expected: String): Unit = { + val numRows = expected.split("\n").length - 4 + val actual = ds.showString(numRows, truncate = 20) + + if (expected != actual) { + fail( + "Dataset.showString() gives wrong result:\n\n" + sideBySide( + "== Expected ==\n" + expected, + "== Actual ==\n" + actual + ).mkString("\n") + ) + } + } + + test("SPARK-15550 Dataset.show() should show contents of the underlying logical plan") { + val df = Seq((1, "foo", "extra"), (2, "bar", "extra")).toDF("b", "a", "c") + val ds = df.as[ClassData] + val expected = + """+---+---+-----+ + || b| a| c| + |+---+---+-----+ + || 1|foo|extra| + || 2|bar|extra| + |+---+---+-----+ + |""".stripMargin + + checkShowString(ds, expected) + } + + test("SPARK-15550 Dataset.show() should show inner nested products as rows") { + val ds = Seq( + NestedStruct(ClassData("foo", 1)), + NestedStruct(ClassData("bar", 2)) + ).toDS() + + val expected = + """+-------+ + || f| + |+-------+ + ||[foo,1]| + ||[bar,2]| + |+-------+ + |""".stripMargin + + checkShowString(ds, expected) + } + + test( + "SPARK-15112: EmbedDeserializerInFilter should not optimize plan fragment that changes schema" + ) { + val ds = Seq(1 -> "foo", 2 -> "bar").toDF("b", "a").as[ClassData] + + assertResult(Seq(ClassData("foo", 1), ClassData("bar", 2))) { + ds.collect().toSeq + } + + assertResult(Seq(ClassData("bar", 2))) { + ds.filter(_.b > 1).collect().toSeq + } + } + + test("mapped dataset should resolve duplicated attributes for self join") { + val ds = Seq(1, 2, 3).toDS().map(_ + 1) + val ds1 = ds.as("d1") + val ds2 = ds.as("d2") + + checkDatasetUnorderly(ds1.joinWith(ds2, $"d1.value" === $"d2.value"), (2, 2), (3, 3), (4, 4)) + checkDatasetUnorderly(ds1.intersect(ds2), 2, 3, 4) + checkDatasetUnorderly(ds1.except(ds1)) + } + + test("SPARK-15441: Dataset outer join") { + val left = Seq(ClassData("a", 1), ClassData("b", 2)).toDS().as("left") + val right = Seq(ClassData("x", 2), ClassData("y", 3)).toDS().as("right") + val joined = left.joinWith(right, $"left.b" === $"right.b", "left") + val result = joined.collect().toSet + assert(result == Set(ClassData("a", 1) -> null, ClassData("b", 2) -> ClassData("x", 2))) + } + + test("better error message when use java reserved keyword as field name") { + val e = intercept[UnsupportedOperationException] { + Seq(InvalidInJava(1)).toDS() + } + assert(e.getMessage.contains( + "`abstract` is a reserved keyword and cannot be used as field name")) + } + + test("Dataset should support flat input object to be null") { + checkDataset(Seq("a", null).toDS(), "a", null) + } + + test("Dataset should throw RuntimeException if top-level product input object is null") { + val e = intercept[RuntimeException](Seq(ClassData("a", 1), null).toDS()) + assert(e.getMessage.contains("Null value appeared in non-nullable field")) + assert(e.getMessage.contains("top level Product input object")) + } + + test("dropDuplicates") { + val ds = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() + checkDataset( + ds.dropDuplicates("_1"), + ("a", 1), ("b", 1)) + checkDataset( + ds.dropDuplicates("_2"), + ("a", 1), ("a", 2)) + checkDataset( + ds.dropDuplicates("_1", "_2"), + ("a", 1), ("a", 2), ("b", 1)) + } + + test("dropDuplicates: columns with same column name") { + val ds1 = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() + val ds2 = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() + // The dataset joined has two columns of the same name "_2". + val joined = ds1.join(ds2, "_1").select(ds1("_2").as[Int], ds2("_2").as[Int]) + checkDataset( + joined.dropDuplicates(), + (1, 2), (1, 1), (2, 1), (2, 2)) + } + + test("SPARK-16097: Encoders.tuple should handle null object correctly") { + val enc = Encoders.tuple(Encoders.tuple(Encoders.STRING, Encoders.STRING), Encoders.STRING) + val data = Seq((("a", "b"), "c"), (null, "d")) + val ds = spark.createDataset(data)(enc) + checkDataset(ds, (("a", "b"), "c"), (null, "d")) + } + + test("SPARK-16995: flat mapping on Dataset containing a column created with lit/expr") { + val df = Seq("1").toDF("a") + + import df.sparkSession.implicits._ + + checkDataset( + df.withColumn("b", lit(0)).as[ClassData] + .groupByKey(_.a).flatMapGroups { case (x, iter) => List[Int]() }) + checkDataset( + df.withColumn("b", expr("0")).as[ClassData] + .groupByKey(_.a).flatMapGroups { case (x, iter) => List[Int]() }) + } + + test("SPARK-18125: Spark generated code causes CompileException") { + val data = Array( + Route("a", "b", 1), + Route("a", "b", 2), + Route("a", "c", 2), + Route("a", "d", 10), + Route("b", "a", 1), + Route("b", "a", 5), + Route("b", "c", 6)) + val ds = sparkContext.parallelize(data).toDF.as[Route] + + val grped = ds.map(r => GroupedRoutes(r.src, r.dest, Seq(r))) + .groupByKey(r => (r.src, r.dest)) + .reduceGroups { (g1: GroupedRoutes, g2: GroupedRoutes) => + GroupedRoutes(g1.src, g1.dest, g1.routes ++ g2.routes) + }.map(_._2) + + val expected = Seq( + GroupedRoutes("a", "d", Seq(Route("a", "d", 10))), + GroupedRoutes("b", "c", Seq(Route("b", "c", 6))), + GroupedRoutes("a", "b", Seq(Route("a", "b", 1), Route("a", "b", 2))), + GroupedRoutes("b", "a", Seq(Route("b", "a", 1), Route("b", "a", 5))), + GroupedRoutes("a", "c", Seq(Route("a", "c", 2))) + ) + + implicit def ordering[GroupedRoutes]: Ordering[GroupedRoutes] = new Ordering[GroupedRoutes] { + override def compare(x: GroupedRoutes, y: GroupedRoutes): Int = { + x.toString.compareTo(y.toString) + } + } + + checkDatasetUnorderly(grped, expected: _*) + } + + test("SPARK-18189: Fix serialization issue in KeyValueGroupedDataset") { + val resultValue = 12345 + val keyValueGrouped = Seq((1, 2), (3, 4)).toDS().groupByKey(_._1) + val mapGroups = keyValueGrouped.mapGroups((k, v) => (k, 1)) + val broadcasted = spark.sparkContext.broadcast(resultValue) + + // Using broadcast triggers serialization issue in KeyValueGroupedDataset + val dataset = mapGroups.map(_ => broadcasted.value) + + assert(dataset.collect() sameElements Array(resultValue, resultValue)) + } + + test("SPARK-18284: Serializer should have correct nullable value") { + val df1 = Seq(1, 2, 3, 4).toDF + assert(df1.schema(0).nullable == false) + val df2 = Seq(Integer.valueOf(1), Integer.valueOf(2)).toDF + assert(df2.schema(0).nullable == true) + + val df3 = Seq(Seq(1, 2), Seq(3, 4)).toDF + assert(df3.schema(0).nullable == true) + assert(df3.schema(0).dataType.asInstanceOf[ArrayType].containsNull == false) + val df4 = Seq(Seq("a", "b"), Seq("c", "d")).toDF + assert(df4.schema(0).nullable == true) + assert(df4.schema(0).dataType.asInstanceOf[ArrayType].containsNull == true) + + val df5 = Seq((0, 1.0), (2, 2.0)).toDF("id", "v") + assert(df5.schema(0).nullable == false) + assert(df5.schema(1).nullable == false) + val df6 = Seq((0, 1.0, "a"), (2, 2.0, "b")).toDF("id", "v1", "v2") + assert(df6.schema(0).nullable == false) + assert(df6.schema(1).nullable == false) + assert(df6.schema(2).nullable == true) + + val df7 = (Tuple1(Array(1, 2, 3)) :: Nil).toDF("a") + assert(df7.schema(0).nullable == true) + assert(df7.schema(0).dataType.asInstanceOf[ArrayType].containsNull == false) + + val df8 = (Tuple1(Array((null: Integer), (null: Integer))) :: Nil).toDF("a") + assert(df8.schema(0).nullable == true) + assert(df8.schema(0).dataType.asInstanceOf[ArrayType].containsNull == true) + + val df9 = (Tuple1(Map(2 -> 3)) :: Nil).toDF("m") + assert(df9.schema(0).nullable == true) + assert(df9.schema(0).dataType.asInstanceOf[MapType].valueContainsNull == false) + + val df10 = (Tuple1(Map(1 -> (null: Integer))) :: Nil).toDF("m") + assert(df10.schema(0).nullable == true) + assert(df10.schema(0).dataType.asInstanceOf[MapType].valueContainsNull == true) + + val df11 = Seq(TestDataPoint(1, 2.2, "a", null), + TestDataPoint(3, 4.4, "null", (TestDataPoint2(33, "b")))).toDF + assert(df11.schema(0).nullable == false) + assert(df11.schema(1).nullable == false) + assert(df11.schema(2).nullable == true) + assert(df11.schema(3).nullable == true) + assert(df11.schema(3).dataType.asInstanceOf[StructType].fields(0).nullable == false) + assert(df11.schema(3).dataType.asInstanceOf[StructType].fields(1).nullable == true) + } + + Seq(true, false).foreach { eager => + def testCheckpointing(testName: String)(f: => Unit): Unit = { + test(s"Dataset.checkpoint() - $testName (eager = $eager)") { + withTempDir { dir => + val originalCheckpointDir = spark.sparkContext.checkpointDir + + try { + spark.sparkContext.setCheckpointDir(dir.getCanonicalPath) + f + } finally { + // Since the original checkpointDir can be None, we need + // to set the variable directly. + spark.sparkContext.checkpointDir = originalCheckpointDir + } + } + } + } + + testCheckpointing("basic") { + val ds = spark.range(10).repartition('id % 2).filter('id > 5).orderBy('id.desc) + val cp = ds.checkpoint(eager) + + val logicalRDD = cp.logicalPlan match { + case plan: LogicalRDD => plan + case _ => + val treeString = cp.logicalPlan.treeString(verbose = true) + fail(s"Expecting a LogicalRDD, but got\n$treeString") + } + + val dsPhysicalPlan = ds.queryExecution.executedPlan + val cpPhysicalPlan = cp.queryExecution.executedPlan + + assertResult(dsPhysicalPlan.outputPartitioning) { logicalRDD.outputPartitioning } + assertResult(dsPhysicalPlan.outputOrdering) { logicalRDD.outputOrdering } + + assertResult(dsPhysicalPlan.outputPartitioning) { cpPhysicalPlan.outputPartitioning } + assertResult(dsPhysicalPlan.outputOrdering) { cpPhysicalPlan.outputOrdering } + + // For a lazy checkpoint() call, the first check also materializes the checkpoint. + checkDataset(cp, (9L to 6L by -1L).map(java.lang.Long.valueOf): _*) + + // Reads back from checkpointed data and check again. + checkDataset(cp, (9L to 6L by -1L).map(java.lang.Long.valueOf): _*) + } + + testCheckpointing("should preserve partitioning information") { + val ds = spark.range(10).repartition('id % 2) + val cp = ds.checkpoint(eager) + + val agg = cp.groupBy('id % 2).agg(count('id)) + + agg.queryExecution.executedPlan.collectFirst { + case ShuffleExchange(_, _: RDDScanExec, _) => + case BroadcastExchangeExec(_, _: RDDScanExec) => + }.foreach { _ => + fail( + "No Exchange should be inserted above RDDScanExec since the checkpointed Dataset " + + "preserves partitioning information:\n\n" + agg.queryExecution + ) + } + + checkAnswer(agg, ds.groupBy('id % 2).agg(count('id))) + } + } + + test("identity map for primitive arrays") { + val arrayByte = Array(1.toByte, 2.toByte, 3.toByte) + val arrayInt = Array(1, 2, 3) + val arrayLong = Array(1.toLong, 2.toLong, 3.toLong) + val arrayDouble = Array(1.1, 2.2, 3.3) + val arrayString = Array("a", "b", "c") + val dsByte = sparkContext.parallelize(Seq(arrayByte), 1).toDS.map(e => e) + val dsInt = sparkContext.parallelize(Seq(arrayInt), 1).toDS.map(e => e) + val dsLong = sparkContext.parallelize(Seq(arrayLong), 1).toDS.map(e => e) + val dsDouble = sparkContext.parallelize(Seq(arrayDouble), 1).toDS.map(e => e) + val dsString = sparkContext.parallelize(Seq(arrayString), 1).toDS.map(e => e) + checkDataset(dsByte, arrayByte) + checkDataset(dsInt, arrayInt) + checkDataset(dsLong, arrayLong) + checkDataset(dsDouble, arrayDouble) + checkDataset(dsString, arrayString) + } + + test("SPARK-18251: the type of Dataset can't be Option of Product type") { + checkDataset(Seq(Some(1), None).toDS(), Some(1), None) + + val e = intercept[UnsupportedOperationException] { + Seq(Some(1 -> "a"), None).toDS() + } + assert(e.getMessage.contains("Cannot create encoder for Option of Product type")) + } + + test ("SPARK-17460: the sizeInBytes in Statistics shouldn't overflow to a negative number") { + // Since the sizeInBytes in Statistics could exceed the limit of an Int, we should use BigInt + // instead of Int for avoiding possible overflow. + val ds = (0 to 10000).map( i => + (i, Seq((i, Seq((i, "This is really not that long of a string")))))).toDS() + val sizeInBytes = ds.logicalPlan.stats.sizeInBytes + // sizeInBytes is 2404280404, before the fix, it overflows to a negative number + assert(sizeInBytes > 0) + } + + test("SPARK-18717: code generation works for both scala.collection.Map" + + " and scala.collection.imutable.Map") { + val ds = Seq(WithImmutableMap("hi", Map(42L -> "foo"))).toDS + checkDataset(ds.map(t => t), WithImmutableMap("hi", Map(42L -> "foo"))) + + val ds2 = Seq(WithMap("hi", Map(42L -> "foo"))).toDS + checkDataset(ds2.map(t => t), WithMap("hi", Map(42L -> "foo"))) + } + + test("SPARK-18746: add implicit encoder for BigDecimal, date, timestamp") { + // For this implicit encoder, 18 is the default scale + assert(spark.range(1).map { x => new java.math.BigDecimal(1) }.head == + new java.math.BigDecimal(1).setScale(18)) + + assert(spark.range(1).map { x => scala.math.BigDecimal(1, 18) }.head == + scala.math.BigDecimal(1, 18)) + + assert(spark.range(1).map { x => new java.sql.Date(2016, 12, 12) }.head == + new java.sql.Date(2016, 12, 12)) + + assert(spark.range(1).map { x => new java.sql.Timestamp(100000) }.head == + new java.sql.Timestamp(100000)) + } + + test("SPARK-19896: cannot have circular references in in case class") { + val errMsg1 = intercept[UnsupportedOperationException] { + Seq(CircularReferenceClassA(null)).toDS + } + assert(errMsg1.getMessage.startsWith("cannot have circular references in class, but got the " + + "circular reference of class")) + val errMsg2 = intercept[UnsupportedOperationException] { + Seq(CircularReferenceClassC(null)).toDS + } + assert(errMsg2.getMessage.startsWith("cannot have circular references in class, but got the " + + "circular reference of class")) + val errMsg3 = intercept[UnsupportedOperationException] { + Seq(CircularReferenceClassD(null)).toDS + } + assert(errMsg3.getMessage.startsWith("cannot have circular references in class, but got the " + + "circular reference of class")) + } + + test("SPARK-20125: option of map") { + val ds = Seq(WithMapInOption(Some(Map(1 -> 1)))).toDS() + checkDataset(ds, WithMapInOption(Some(Map(1 -> 1)))) + } + + test("SPARK-20399: do not unescaped regex pattern when ESCAPED_STRING_LITERALS is enabled") { + withSQLConf(SQLConf.ESCAPED_STRING_LITERALS.key -> "true") { + val data = Seq("\u0020\u0021\u0023", "abc") + val df = data.toDF() + val rlike1 = df.filter("value rlike '^\\x20[\\x20-\\x23]+$'") + val rlike2 = df.filter($"value".rlike("^\\x20[\\x20-\\x23]+$")) + val rlike3 = df.filter("value rlike '^\\\\x20[\\\\x20-\\\\x23]+$'") + checkAnswer(rlike1, rlike2) + assert(rlike3.count() == 0) + } + } + + test("SPARK-21538: Attribute resolution inconsistency in Dataset API") { + val df = spark.range(3).withColumnRenamed("id", "x") + val expected = Row(0) :: Row(1) :: Row (2) :: Nil + checkAnswer(df.sort("id"), expected) + checkAnswer(df.sort(col("id")), expected) + checkAnswer(df.sort($"id"), expected) + checkAnswer(df.sort('id), expected) + checkAnswer(df.orderBy("id"), expected) + checkAnswer(df.orderBy(col("id")), expected) + checkAnswer(df.orderBy($"id"), expected) + checkAnswer(df.orderBy('id), expected) + } + + test("SPARK-21567: Dataset should work with type alias") { + checkDataset( + Seq(1).toDS().map(_ => ("", TestForTypeAlias.tupleTypeAlias)), + ("", (1, 1))) + + checkDataset( + Seq(1).toDS().map(_ => ("", TestForTypeAlias.nestedTupleTypeAlias)), + ("", ((1, 1), 2))) + + checkDataset( + Seq(1).toDS().map(_ => ("", TestForTypeAlias.seqOfTupleTypeAlias)), + ("", Seq((1, 1), (2, 2)))) } } + +case class WithImmutableMap(id: String, map_test: scala.collection.immutable.Map[Long, String]) +case class WithMap(id: String, map_test: scala.collection.Map[Long, String]) +case class WithMapInOption(m: Option[scala.collection.Map[Int, Int]]) + +case class Generic[T](id: T, value: Double) + +case class OtherTuple(_1: String, _2: Int) + +case class TupleClass(data: (Int, String)) + +class OuterClass extends Serializable { + case class InnerClass(a: String) +} + +object OuterObject { + case class InnerClass(a: String) +} + +case class ClassData(a: String, b: Int) +case class ClassData2(c: String, d: Int) +case class ClassNullableData(a: String, b: Integer) + +case class NestedStruct(f: ClassData) +case class DeepNestedStruct(f: NestedStruct) + +case class InvalidInJava(`abstract`: Int) + +/** + * A class used to test serialization using encoders. This class throws exceptions when using + * Java serialization -- so the only way it can be "serialized" is through our encoders. + */ +case class NonSerializableCaseClass(value: String) extends Externalizable { + override def readExternal(in: ObjectInput): Unit = { + throw new UnsupportedOperationException + } + + override def writeExternal(out: ObjectOutput): Unit = { + throw new UnsupportedOperationException + } +} + +/** Used to test Kryo encoder. */ +class KryoData(val a: Int) { + override def equals(other: Any): Boolean = { + a == other.asInstanceOf[KryoData].a + } + override def hashCode: Int = a + override def toString: String = s"KryoData($a)" +} + +object KryoData { + def apply(a: Int): KryoData = new KryoData(a) +} + +/** Used to test Java encoder. */ +class JavaData(val a: Int) extends Serializable { + override def equals(other: Any): Boolean = { + a == other.asInstanceOf[JavaData].a + } + override def hashCode: Int = a + override def toString: String = s"JavaData($a)" +} + +object JavaData { + def apply(a: Int): JavaData = new JavaData(a) +} + +/** Used to test importing dataset.spark.implicits._ */ +object DatasetTransform { + def addOne(ds: Dataset[Int]): Dataset[Int] = { + import ds.sparkSession.implicits._ + ds.map(_ + 1) + } +} + +case class Route(src: String, dest: String, cost: Int) +case class GroupedRoutes(src: String, dest: String, routes: Seq[Route]) + +case class CircularReferenceClassA(cls: CircularReferenceClassB) +case class CircularReferenceClassB(cls: CircularReferenceClassA) +case class CircularReferenceClassC(ar: Array[CircularReferenceClassC]) +case class CircularReferenceClassD(map: Map[String, CircularReferenceClassE]) +case class CircularReferenceClassE(id: String, list: List[CircularReferenceClassD]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 9080c53c491a..3a8694839bb2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -17,8 +17,9 @@ package org.apache.spark.sql -import java.sql.{Timestamp, Date} +import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat +import java.util.Locale import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.functions._ @@ -38,19 +39,25 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { assert(d0 <= d1 && d1 <= d2 && d2 <= d3 && d3 - d0 <= 1) } - // This is a bad test. SPARK-9196 will fix it and re-enable it. - ignore("function current_timestamp") { + test("function current_timestamp and now") { val df1 = Seq((1, 2), (3, 1)).toDF("a", "b") checkAnswer(df1.select(countDistinct(current_timestamp())), Row(1)) + // Execution in one query should return the same value - checkAnswer(sql("""SELECT CURRENT_TIMESTAMP() = CURRENT_TIMESTAMP()"""), - Row(true)) - assert(math.abs(sql("""SELECT CURRENT_TIMESTAMP()""").collect().head.getTimestamp( - 0).getTime - System.currentTimeMillis()) < 5000) + checkAnswer(sql("""SELECT CURRENT_TIMESTAMP() = CURRENT_TIMESTAMP()"""), Row(true)) + + // Current timestamp should return the current timestamp ... + val before = System.currentTimeMillis + val got = sql("SELECT CURRENT_TIMESTAMP()").collect().head.getTimestamp(0).getTime + val after = System.currentTimeMillis + assert(got >= before && got <= after) + + // Now alias + checkAnswer(sql("""SELECT CURRENT_TIMESTAMP() = NOW()"""), Row(true)) } - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val sdfDate = new SimpleDateFormat("yyyy-MM-dd") + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + val sdfDate = new SimpleDateFormat("yyyy-MM-dd", Locale.US) val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) @@ -347,31 +354,71 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { test("function to_date") { val d1 = Date.valueOf("2015-07-22") val d2 = Date.valueOf("2015-07-01") + val d3 = Date.valueOf("2014-12-31") val t1 = Timestamp.valueOf("2015-07-22 10:00:00") val t2 = Timestamp.valueOf("2014-12-31 23:59:59") + val t3 = Timestamp.valueOf("2014-12-31 23:59:59") val s1 = "2015-07-22 10:00:00" val s2 = "2014-12-31" - val df = Seq((d1, t1, s1), (d2, t2, s2)).toDF("d", "t", "s") + val s3 = "2014-31-12" + val df = Seq((d1, t1, s1), (d2, t2, s2), (d3, t3, s3)).toDF("d", "t", "s") checkAnswer( df.select(to_date(col("t"))), - Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")))) + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), + Row(Date.valueOf("2014-12-31")))) checkAnswer( df.select(to_date(col("d"))), - Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2015-07-01")))) + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2015-07-01")), + Row(Date.valueOf("2014-12-31")))) checkAnswer( df.select(to_date(col("s"))), - Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")))) + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), Row(null))) checkAnswer( df.selectExpr("to_date(t)"), - Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")))) + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), + Row(Date.valueOf("2014-12-31")))) checkAnswer( df.selectExpr("to_date(d)"), - Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2015-07-01")))) + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2015-07-01")), + Row(Date.valueOf("2014-12-31")))) checkAnswer( df.selectExpr("to_date(s)"), - Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")))) + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), Row(null))) + + // now with format + checkAnswer( + df.select(to_date(col("t"), "yyyy-MM-dd")), + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), + Row(Date.valueOf("2014-12-31")))) + checkAnswer( + df.select(to_date(col("d"), "yyyy-MM-dd")), + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2015-07-01")), + Row(Date.valueOf("2014-12-31")))) + checkAnswer( + df.select(to_date(col("s"), "yyyy-MM-dd")), + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), Row(null))) + + // now switch format + checkAnswer( + df.select(to_date(col("s"), "yyyy-dd-MM")), + Seq(Row(null), Row(null), Row(Date.valueOf("2014-12-31")))) + + // invalid format + checkAnswer( + df.select(to_date(col("s"), "yyyy-hh-MM")), + Seq(Row(null), Row(null), Row(null))) + checkAnswer( + df.select(to_date(col("s"), "yyyy-dd-aa")), + Seq(Row(null), Row(null), Row(null))) + + // february + val x1 = "2016-02-29" + val x2 = "2017-02-29" + val df1 = Seq(x1, x2).toDF("x") + checkAnswer( + df1.select(to_date(col("x"))), Row(Date.valueOf("2016-02-29")) :: Row(null) :: Nil) } test("function trunc") { @@ -389,11 +436,11 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { } test("from_unixtime") { - val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" - val sdf2 = new SimpleDateFormat(fmt2) + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) val fmt3 = "yy-MM-dd HH-mm-ss" - val sdf3 = new SimpleDateFormat(fmt3) + val sdf3 = new SimpleDateFormat(fmt3, Locale.US) val df = Seq((1000, "yyyy-MM-dd HH:mm:ss.SSS"), (-1000, "yy-MM-dd HH-mm-ss")).toDF("a", "b") checkAnswer( df.select(from_unixtime(col("a"))), @@ -442,6 +489,111 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { Row(date1.getTime / 1000L), Row(date2.getTime / 1000L))) checkAnswer(df.selectExpr(s"unix_timestamp(s, '$fmt')"), Seq( Row(ts1.getTime / 1000L), Row(ts2.getTime / 1000L))) + + val x1 = "2015-07-24 10:00:00" + val x2 = "2015-25-07 02:02:02" + val x3 = "2015-07-24 25:02:02" + val x4 = "2015-24-07 26:02:02" + val ts3 = Timestamp.valueOf("2015-07-24 02:25:02") + val ts4 = Timestamp.valueOf("2015-07-24 00:10:00") + + val df1 = Seq(x1, x2, x3, x4).toDF("x") + checkAnswer(df1.select(unix_timestamp(col("x"))), Seq( + Row(ts1.getTime / 1000L), Row(null), Row(null), Row(null))) + checkAnswer(df1.selectExpr("unix_timestamp(x)"), Seq( + Row(ts1.getTime / 1000L), Row(null), Row(null), Row(null))) + checkAnswer(df1.select(unix_timestamp(col("x"), "yyyy-dd-MM HH:mm:ss")), Seq( + Row(null), Row(ts2.getTime / 1000L), Row(null), Row(null))) + checkAnswer(df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd mm:HH:ss')"), Seq( + Row(ts4.getTime / 1000L), Row(null), Row(ts3.getTime / 1000L), Row(null))) + + // invalid format + checkAnswer(df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd aa:HH:ss')"), Seq( + Row(null), Row(null), Row(null), Row(null))) + + // february + val y1 = "2016-02-29" + val y2 = "2017-02-29" + val ts5 = Timestamp.valueOf("2016-02-29 00:00:00") + val df2 = Seq(y1, y2).toDF("y") + checkAnswer(df2.select(unix_timestamp(col("y"), "yyyy-MM-dd")), Seq( + Row(ts5.getTime / 1000L), Row(null))) + + val now = sql("select unix_timestamp()").collect().head.getLong(0) + checkAnswer(sql(s"select cast ($now as timestamp)"), Row(new java.util.Date(now * 1000))) + } + + test("to_unix_timestamp") { + val date1 = Date.valueOf("2015-07-24") + val date2 = Date.valueOf("2015-07-25") + val ts1 = Timestamp.valueOf("2015-07-24 10:00:00.3") + val ts2 = Timestamp.valueOf("2015-07-25 02:02:02.2") + val s1 = "2015/07/24 10:00:00.5" + val s2 = "2015/07/25 02:02:02.6" + val ss1 = "2015-07-24 10:00:00" + val ss2 = "2015-07-25 02:02:02" + val fmt = "yyyy/MM/dd HH:mm:ss.S" + val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") + checkAnswer(df.selectExpr("to_unix_timestamp(ts)"), Seq( + Row(ts1.getTime / 1000L), Row(ts2.getTime / 1000L))) + checkAnswer(df.selectExpr("to_unix_timestamp(ss)"), Seq( + Row(ts1.getTime / 1000L), Row(ts2.getTime / 1000L))) + checkAnswer(df.selectExpr(s"to_unix_timestamp(d, '$fmt')"), Seq( + Row(date1.getTime / 1000L), Row(date2.getTime / 1000L))) + checkAnswer(df.selectExpr(s"to_unix_timestamp(s, '$fmt')"), Seq( + Row(ts1.getTime / 1000L), Row(ts2.getTime / 1000L))) + + val x1 = "2015-07-24 10:00:00" + val x2 = "2015-25-07 02:02:02" + val x3 = "2015-07-24 25:02:02" + val x4 = "2015-24-07 26:02:02" + val ts3 = Timestamp.valueOf("2015-07-24 02:25:02") + val ts4 = Timestamp.valueOf("2015-07-24 00:10:00") + + val df1 = Seq(x1, x2, x3, x4).toDF("x") + checkAnswer(df1.selectExpr("to_unix_timestamp(x)"), Seq( + Row(ts1.getTime / 1000L), Row(null), Row(null), Row(null))) + checkAnswer(df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd mm:HH:ss')"), Seq( + Row(ts4.getTime / 1000L), Row(null), Row(ts3.getTime / 1000L), Row(null))) + + // february + val y1 = "2016-02-29" + val y2 = "2017-02-29" + val ts5 = Timestamp.valueOf("2016-02-29 00:00:00") + val df2 = Seq(y1, y2).toDF("y") + checkAnswer(df2.select(unix_timestamp(col("y"), "yyyy-MM-dd")), Seq( + Row(ts5.getTime / 1000L), Row(null))) + + // invalid format + checkAnswer(df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd bb:HH:ss')"), Seq( + Row(null), Row(null), Row(null), Row(null))) + } + + + test("to_timestamp") { + val date1 = Date.valueOf("2015-07-24") + val date2 = Date.valueOf("2015-07-25") + val ts_date1 = Timestamp.valueOf("2015-07-24 00:00:00") + val ts_date2 = Timestamp.valueOf("2015-07-25 00:00:00") + val ts1 = Timestamp.valueOf("2015-07-24 10:00:00") + val ts2 = Timestamp.valueOf("2015-07-25 02:02:02") + val s1 = "2015/07/24 10:00:00.5" + val s2 = "2015/07/25 02:02:02.6" + val ss1 = "2015-07-24 10:00:00" + val ss2 = "2015-07-25 02:02:02" + val fmt = "yyyy/MM/dd HH:mm:ss.S" + val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") + + checkAnswer(df.select(to_timestamp(col("ss"))), + df.select(unix_timestamp(col("ss")).cast("timestamp"))) + checkAnswer(df.select(to_timestamp(col("ss"))), Seq( + Row(ts1), Row(ts2))) + checkAnswer(df.select(to_timestamp(col("s"), fmt)), Seq( + Row(ts1), Row(ts2))) + checkAnswer(df.select(to_timestamp(col("ts"), fmt)), Seq( + Row(ts1), Row(ts2))) + checkAnswer(df.select(to_timestamp(col("d"), "yyyy-MM-dd")), Seq( + Row(ts_date1), Row(ts_date2))) } test("datediff") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala index 78a98798eff6..a41b46554862 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala @@ -15,25 +15,26 @@ * limitations under the License. */ -package test.org.apache.spark.sql +package org.apache.spark.sql import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Literal, GenericInternalRow, Attribute} -import org.apache.spark.sql.catalyst.plans.logical.{Project, LogicalPlan} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.{Row, Strategy, QueryTest} import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.unsafe.types.UTF8String case class FastOperator(output: Seq[Attribute]) extends SparkPlan { override protected def doExecute(): RDD[InternalRow] = { val str = Literal("so fast").value val row = new GenericInternalRow(Array[Any](str)) - sparkContext.parallelize(Seq(row)) + val unsafeProj = UnsafeProjection.create(schema) + val unsafeRow = unsafeProj(row).copy() + sparkContext.parallelize(Seq(unsafeRow)) } + override def producedAttributes: AttributeSet = outputSet override def children: Seq[SparkPlan] = Nil } @@ -50,7 +51,7 @@ class ExtraStrategiesSuite extends QueryTest with SharedSQLContext { test("insert an extraStrategy") { try { - sqlContext.experimental.extraStrategies = TestStrategy :: Nil + spark.experimental.extraStrategies = TestStrategy :: Nil val df = sparkContext.parallelize(Seq(("so slow", 1))).toDF("a", "b") checkAnswer( @@ -61,7 +62,7 @@ class ExtraStrategiesSuite extends QueryTest with SharedSQLContext { df.select("a", "b"), Row("so slow", 1)) } finally { - sqlContext.experimental.extraStrategies = Nil + spark.experimental.extraStrategies = Nil } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala new file mode 100644 index 000000000000..6b98209fd49b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/GeneratorFunctionSuite.scala @@ -0,0 +1,320 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Expression, Generator} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{IntegerType, StructType} + +class GeneratorFunctionSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("stack") { + val df = spark.range(1) + + // Empty DataFrame suppress the result generation + checkAnswer(spark.emptyDataFrame.selectExpr("stack(1, 1, 2, 3)"), Nil) + + // Rows & columns + checkAnswer(df.selectExpr("stack(1, 1, 2, 3)"), Row(1, 2, 3) :: Nil) + checkAnswer(df.selectExpr("stack(2, 1, 2, 3)"), Row(1, 2) :: Row(3, null) :: Nil) + checkAnswer(df.selectExpr("stack(3, 1, 2, 3)"), Row(1) :: Row(2) :: Row(3) :: Nil) + checkAnswer(df.selectExpr("stack(4, 1, 2, 3)"), Row(1) :: Row(2) :: Row(3) :: Row(null) :: Nil) + + // Various column types + checkAnswer(df.selectExpr("stack(3, 1, 1.1, 'a', 2, 2.2, 'b', 3, 3.3, 'c')"), + Row(1, 1.1, "a") :: Row(2, 2.2, "b") :: Row(3, 3.3, "c") :: Nil) + + // Null values + checkAnswer(df.selectExpr("stack(3, 1, 1.1, null, 2, null, 'b', null, 3.3, 'c')"), + Row(1, 1.1, null) :: Row(2, null, "b") :: Row(null, 3.3, "c") :: Nil) + + // Repeat generation at every input row + checkAnswer(spark.range(2).selectExpr("stack(2, 1, 2, 3)"), + Row(1, 2) :: Row(3, null) :: Row(1, 2) :: Row(3, null) :: Nil) + + // The first argument must be a positive constant integer. + val m = intercept[AnalysisException] { + df.selectExpr("stack(1.1, 1, 2, 3)") + }.getMessage + assert(m.contains("The number of rows must be a positive constant integer.")) + val m2 = intercept[AnalysisException] { + df.selectExpr("stack(-1, 1, 2, 3)") + }.getMessage + assert(m2.contains("The number of rows must be a positive constant integer.")) + + // The data for the same column should have the same type. + val m3 = intercept[AnalysisException] { + df.selectExpr("stack(2, 1, '2.2')") + }.getMessage + assert(m3.contains("data type mismatch: Argument 1 (IntegerType) != Argument 2 (StringType)")) + + // stack on column data + val df2 = Seq((2, 1, 2, 3)).toDF("n", "a", "b", "c") + checkAnswer(df2.selectExpr("stack(2, a, b, c)"), Row(1, 2) :: Row(3, null) :: Nil) + + val m4 = intercept[AnalysisException] { + df2.selectExpr("stack(n, a, b, c)") + }.getMessage + assert(m4.contains("The number of rows must be a positive constant integer.")) + + val df3 = Seq((2, 1, 2.0)).toDF("n", "a", "b") + val m5 = intercept[AnalysisException] { + df3.selectExpr("stack(2, a, b)") + }.getMessage + assert(m5.contains("data type mismatch: Argument 1 (IntegerType) != Argument 2 (DoubleType)")) + + } + + test("single explode") { + val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") + checkAnswer( + df.select(explode('intList)), + Row(1) :: Row(2) :: Row(3) :: Nil) + } + + test("single explode_outer") { + val df = Seq((1, Seq(1, 2, 3)), (2, Seq())).toDF("a", "intList") + checkAnswer( + df.select(explode_outer('intList)), + Row(1) :: Row(2) :: Row(3) :: Row(null) :: Nil) + } + + test("single posexplode") { + val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") + checkAnswer( + df.select(posexplode('intList)), + Row(0, 1) :: Row(1, 2) :: Row(2, 3) :: Nil) + } + + test("single posexplode_outer") { + val df = Seq((1, Seq(1, 2, 3)), (2, Seq())).toDF("a", "intList") + checkAnswer( + df.select(posexplode_outer('intList)), + Row(0, 1) :: Row(1, 2) :: Row(2, 3) :: Row(null, null) :: Nil) + } + + test("explode and other columns") { + val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") + + checkAnswer( + df.select($"a", explode('intList)), + Row(1, 1) :: + Row(1, 2) :: + Row(1, 3) :: Nil) + + checkAnswer( + df.select($"*", explode('intList)), + Row(1, Seq(1, 2, 3), 1) :: + Row(1, Seq(1, 2, 3), 2) :: + Row(1, Seq(1, 2, 3), 3) :: Nil) + } + + test("explode_outer and other columns") { + val df = Seq((1, Seq(1, 2, 3)), (2, Seq())).toDF("a", "intList") + + checkAnswer( + df.select($"a", explode_outer('intList)), + Row(1, 1) :: + Row(1, 2) :: + Row(1, 3) :: + Row(2, null) :: + Nil) + + checkAnswer( + df.select($"*", explode_outer('intList)), + Row(1, Seq(1, 2, 3), 1) :: + Row(1, Seq(1, 2, 3), 2) :: + Row(1, Seq(1, 2, 3), 3) :: + Row(2, Seq(), null) :: + Nil) + } + + test("aliased explode") { + val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") + + checkAnswer( + df.select(explode('intList).as('int)).select('int), + Row(1) :: Row(2) :: Row(3) :: Nil) + + checkAnswer( + df.select(explode('intList).as('int)).select(sum('int)), + Row(6) :: Nil) + } + + test("aliased explode_outer") { + val df = Seq((1, Seq(1, 2, 3)), (2, Seq())).toDF("a", "intList") + + checkAnswer( + df.select(explode_outer('intList).as('int)).select('int), + Row(1) :: Row(2) :: Row(3) :: Row(null) :: Nil) + + checkAnswer( + df.select(explode('intList).as('int)).select(sum('int)), + Row(6) :: Nil) + } + + test("explode on map") { + val df = Seq((1, Map("a" -> "b"))).toDF("a", "map") + + checkAnswer( + df.select(explode('map)), + Row("a", "b")) + } + + test("explode_outer on map") { + val df = Seq((1, Map("a" -> "b")), (2, Map[String, String]()), + (3, Map("c" -> "d"))).toDF("a", "map") + + checkAnswer( + df.select(explode_outer('map)), + Row("a", "b") :: Row(null, null) :: Row("c", "d") :: Nil) + } + + test("explode on map with aliases") { + val df = Seq((1, Map("a" -> "b"))).toDF("a", "map") + + checkAnswer( + df.select(explode('map).as("key1" :: "value1" :: Nil)).select("key1", "value1"), + Row("a", "b")) + } + + test("explode_outer on map with aliases") { + val df = Seq((3, None), (1, Some(Map("a" -> "b")))).toDF("a", "map") + + checkAnswer( + df.select(explode_outer('map).as("key1" :: "value1" :: Nil)).select("key1", "value1"), + Row("a", "b") :: Row(null, null) :: Nil) + } + + test("self join explode") { + val df = Seq((1, Seq(1, 2, 3))).toDF("a", "intList") + val exploded = df.select(explode('intList).as('i)) + + checkAnswer( + exploded.join(exploded, exploded("i") === exploded("i")).agg(count("*")), + Row(3) :: Nil) + } + + test("inline raises exception on array of null type") { + val m = intercept[AnalysisException] { + spark.range(2).selectExpr("inline(array())") + }.getMessage + assert(m.contains("data type mismatch")) + } + + test("inline with empty table") { + checkAnswer( + spark.range(0).selectExpr("inline(array(struct(10, 100)))"), + Nil) + } + + test("inline on literal") { + checkAnswer( + spark.range(2).selectExpr("inline(array(struct(10, 100), struct(20, 200), struct(30, 300)))"), + Row(10, 100) :: Row(20, 200) :: Row(30, 300) :: + Row(10, 100) :: Row(20, 200) :: Row(30, 300) :: Nil) + } + + test("inline on column") { + val df = Seq((1, 2)).toDF("a", "b") + + checkAnswer( + df.selectExpr("inline(array(struct(a), struct(a)))"), + Row(1) :: Row(1) :: Nil) + + checkAnswer( + df.selectExpr("inline(array(struct(a, b), struct(a, b)))"), + Row(1, 2) :: Row(1, 2) :: Nil) + + // Spark think [struct, struct] is heterogeneous due to name difference. + val m = intercept[AnalysisException] { + df.selectExpr("inline(array(struct(a), struct(b)))") + }.getMessage + assert(m.contains("data type mismatch")) + + checkAnswer( + df.selectExpr("inline(array(struct(a), named_struct('a', b)))"), + Row(1) :: Row(2) :: Nil) + + // Spark think [struct, struct] is heterogeneous due to name difference. + val m2 = intercept[AnalysisException] { + df.selectExpr("inline(array(struct(a), struct(2)))") + }.getMessage + assert(m2.contains("data type mismatch")) + + checkAnswer( + df.selectExpr("inline(array(struct(a), named_struct('a', 2)))"), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + df.selectExpr("struct(a)").selectExpr("inline(array(*))"), + Row(1) :: Nil) + + checkAnswer( + df.selectExpr("array(struct(a), named_struct('a', b))").selectExpr("inline(*)"), + Row(1) :: Row(2) :: Nil) + } + + test("inline_outer") { + val df = Seq((1, "2"), (3, "4"), (5, "6")).toDF("col1", "col2") + val df2 = df.select(when('col1 === 1, null).otherwise(array(struct('col1, 'col2))).as("col1")) + checkAnswer( + df2.selectExpr("inline(col1)"), + Row(3, "4") :: Row(5, "6") :: Nil + ) + checkAnswer( + df2.selectExpr("inline_outer(col1)"), + Row(null, null) :: Row(3, "4") :: Row(5, "6") :: Nil + ) + } + + test("SPARK-14986: Outer lateral view with empty generate expression") { + checkAnswer( + sql("select nil from values 1 lateral view outer explode(array()) n as nil"), + Row(null) :: Nil + ) + } + + test("outer explode()") { + checkAnswer( + sql("select * from values 1, 2 lateral view outer explode(array()) a as b"), + Row(1, null) :: Row(2, null) :: Nil) + } + + test("outer generator()") { + spark.sessionState.functionRegistry + .createOrReplaceTempFunction("empty_gen", _ => EmptyGenerator()) + checkAnswer( + sql("select * from values 1, 2 lateral view outer empty_gen() a as b"), + Row(1, null) :: Row(2, null) :: Nil) + } +} + +case class EmptyGenerator() extends Generator { + override def children: Seq[Expression] = Nil + override def elementSchema: StructType = new StructType().add("id", IntegerType) + override def eval(input: InternalRow): TraversableOnce[InternalRow] = Seq.empty + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val iteratorClass = classOf[Iterator[_]].getName + ev.copy(code = s"$iteratorClass ${ev.value} = $iteratorClass$$.MODULE$$.empty();") + } +} 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 a9ca46cab067..9d50e8be6089 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 @@ -17,11 +17,19 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import scala.collection.JavaConverters._ +import scala.collection.mutable.ListBuffer +import scala.language.existentials + +import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.expressions.{Ascending, SortOrder} +import org.apache.spark.sql.execution.SortExec import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext - +import org.apache.spark.sql.types.StructType class JoinSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -29,173 +37,147 @@ class JoinSuite extends QueryTest with SharedSQLContext { setupTestData() def statisticSizeInByte(df: DataFrame): BigInt = { - df.queryExecution.optimizedPlan.statistics.sizeInBytes + df.queryExecution.optimizedPlan.stats.sizeInBytes } test("equi-join is hash-join") { val x = testData2.as("x") val y = testData2.as("y") val join = x.join(y, $"x.a" === $"y.a", "inner").queryExecution.optimizedPlan - val planned = sqlContext.planner.EquiJoinSelection(join) + val planned = spark.sessionState.planner.JoinSelection(join) assert(planned.size === 1) } - def assertJoin(sqlString: String, c: Class[_]): Any = { + def assertJoin(pair: (String, Class[_])): Any = { + val (sqlString, c) = pair val df = sql(sqlString) val physical = df.queryExecution.sparkPlan val operators = physical.collect { - case j: ShuffledHashJoin => j - case j: ShuffledHashOuterJoin => j - case j: LeftSemiJoinHash => j - case j: BroadcastHashJoin => j - case j: BroadcastHashOuterJoin => j - case j: LeftSemiJoinBNL => j - case j: CartesianProduct => j - case j: BroadcastNestedLoopJoin => j - case j: BroadcastLeftSemiJoinHash => j - case j: SortMergeJoin => j - case j: SortMergeOuterJoin => j + case j: BroadcastHashJoinExec => j + case j: ShuffledHashJoinExec => j + case j: CartesianProductExec => j + case j: BroadcastNestedLoopJoinExec => j + case j: SortMergeJoinExec => j } assert(operators.size === 1) - if (operators(0).getClass() != c) { - fail(s"$sqlString expected operator: $c, but got ${operators(0)}\n physical: \n$physical") + if (operators.head.getClass != c) { + fail(s"$sqlString expected operator: $c, but got ${operators.head}\n physical: \n$physical") } } test("join operator selection") { - sqlContext.cacheManager.clearCache() + spark.sharedState.cacheManager.clearCache() - Seq( - ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[LeftSemiJoinHash]), - ("SELECT * FROM testData LEFT SEMI JOIN testData2", classOf[LeftSemiJoinBNL]), - ("SELECT * FROM testData JOIN testData2", classOf[CartesianProduct]), - ("SELECT * FROM testData JOIN testData2 WHERE key = 2", classOf[CartesianProduct]), - ("SELECT * FROM testData LEFT JOIN testData2", classOf[CartesianProduct]), - ("SELECT * FROM testData RIGHT JOIN testData2", classOf[CartesianProduct]), - ("SELECT * FROM testData FULL OUTER JOIN testData2", classOf[CartesianProduct]), - ("SELECT * FROM testData LEFT JOIN testData2 WHERE key = 2", classOf[CartesianProduct]), - ("SELECT * FROM testData RIGHT JOIN testData2 WHERE key = 2", classOf[CartesianProduct]), - ("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key = 2", classOf[CartesianProduct]), - ("SELECT * FROM testData JOIN testData2 WHERE key > a", classOf[CartesianProduct]), - ("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key > a", classOf[CartesianProduct]), - ("SELECT * FROM testData JOIN testData2 ON key = a", classOf[SortMergeJoin]), - ("SELECT * FROM testData JOIN testData2 ON key = a and key = 2", classOf[SortMergeJoin]), - ("SELECT * FROM testData JOIN testData2 ON key = a where key = 2", classOf[SortMergeJoin]), - ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", classOf[SortMergeOuterJoin]), - ("SELECT * FROM testData RIGHT JOIN testData2 ON key = a where key = 2", - classOf[SortMergeOuterJoin]), - ("SELECT * FROM testData right join testData2 ON key = a and key = 2", - classOf[SortMergeOuterJoin]), - ("SELECT * FROM testData full outer join testData2 ON key = a", - classOf[SortMergeOuterJoin]), - ("SELECT * FROM testData left JOIN testData2 ON (key * a != key + a)", - classOf[BroadcastNestedLoopJoin]), - ("SELECT * FROM testData right JOIN testData2 ON (key * a != key + a)", - classOf[BroadcastNestedLoopJoin]), - ("SELECT * FROM testData full JOIN testData2 ON (key * a != key + a)", - classOf[BroadcastNestedLoopJoin]) - ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> "false") { + withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "0", + SQLConf.CROSS_JOINS_ENABLED.key -> "true") { Seq( - ("SELECT * FROM testData JOIN testData2 ON key = a", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", + classOf[SortMergeJoinExec]), + ("SELECT * FROM testData LEFT SEMI JOIN testData2", classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData JOIN testData2", classOf[CartesianProductExec]), + ("SELECT * FROM testData JOIN testData2 WHERE key = 2", classOf[CartesianProductExec]), + ("SELECT * FROM testData LEFT JOIN testData2", classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData RIGHT JOIN testData2", classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData FULL OUTER JOIN testData2", classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData LEFT JOIN testData2 WHERE key = 2", + classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData RIGHT JOIN testData2 WHERE key = 2", + classOf[CartesianProductExec]), + ("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key = 2", + classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData JOIN testData2 WHERE key > a", classOf[CartesianProductExec]), + ("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key > a", + classOf[CartesianProductExec]), + ("SELECT * FROM testData JOIN testData2 ON key = a", classOf[SortMergeJoinExec]), ("SELECT * FROM testData JOIN testData2 ON key = a and key = 2", - classOf[ShuffledHashJoin]), + classOf[SortMergeJoinExec]), ("SELECT * FROM testData JOIN testData2 ON key = a where key = 2", - classOf[ShuffledHashJoin]), - ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", classOf[ShuffledHashOuterJoin]), + classOf[SortMergeJoinExec]), + ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", classOf[SortMergeJoinExec]), ("SELECT * FROM testData RIGHT JOIN testData2 ON key = a where key = 2", - classOf[ShuffledHashOuterJoin]), + classOf[SortMergeJoinExec]), ("SELECT * FROM testData right join testData2 ON key = a and key = 2", - classOf[ShuffledHashOuterJoin]), + classOf[SortMergeJoinExec]), ("SELECT * FROM testData full outer join testData2 ON key = a", - classOf[ShuffledHashOuterJoin]) - ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + classOf[SortMergeJoinExec]), + ("SELECT * FROM testData left JOIN testData2 ON (key * a != key + a)", + classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData right JOIN testData2 ON (key * a != key + a)", + classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData full JOIN testData2 ON (key * a != key + a)", + classOf[BroadcastNestedLoopJoinExec]), + ("SELECT * FROM testData ANTI JOIN testData2 ON key = a", classOf[SortMergeJoinExec]), + ("SELECT * FROM testData LEFT ANTI JOIN testData2", classOf[BroadcastNestedLoopJoinExec]) + ).foreach(assertJoin) } } - test("SortMergeJoin shouldn't work on unsortable columns") { - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> "true") { - Seq( - ("SELECT * FROM arrayData JOIN complexData ON data = a", classOf[ShuffledHashJoin]) - ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } - } - } +// ignore("SortMergeJoin shouldn't work on unsortable columns") { +// Seq( +// ("SELECT * FROM arrayData JOIN complexData ON data = a", classOf[ShuffledHashJoin]) +// ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } +// } test("broadcasted hash join operator selection") { - sqlContext.cacheManager.clearCache() + spark.sharedState.cacheManager.clearCache() sql("CACHE TABLE testData") - for (sortMergeJoinEnabled <- Seq(true, false)) { - withClue(s"sortMergeJoinEnabled=$sortMergeJoinEnabled") { - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> s"$sortMergeJoinEnabled") { - Seq( - ("SELECT * FROM testData join testData2 ON key = a", - classOf[BroadcastHashJoin]), - ("SELECT * FROM testData join testData2 ON key = a and key = 2", - classOf[BroadcastHashJoin]), - ("SELECT * FROM testData join testData2 ON key = a where key = 2", - classOf[BroadcastHashJoin]) - ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } - } - } - } - sql("UNCACHE TABLE testData") + Seq( + ("SELECT * FROM testData join testData2 ON key = a", + classOf[BroadcastHashJoinExec]), + ("SELECT * FROM testData join testData2 ON key = a and key = 2", + classOf[BroadcastHashJoinExec]), + ("SELECT * FROM testData join testData2 ON key = a where key = 2", + classOf[BroadcastHashJoinExec]) + ).foreach(assertJoin) } test("broadcasted hash outer join operator selection") { - sqlContext.cacheManager.clearCache() + spark.sharedState.cacheManager.clearCache() sql("CACHE TABLE testData") - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> "true") { - Seq( - ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", - classOf[SortMergeOuterJoin]), - ("SELECT * FROM testData RIGHT JOIN testData2 ON key = a where key = 2", - classOf[BroadcastHashOuterJoin]), - ("SELECT * FROM testData right join testData2 ON key = a and key = 2", - classOf[BroadcastHashOuterJoin]) - ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } - } - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> "false") { - Seq( - ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", - classOf[ShuffledHashOuterJoin]), - ("SELECT * FROM testData RIGHT JOIN testData2 ON key = a where key = 2", - classOf[BroadcastHashOuterJoin]), - ("SELECT * FROM testData right join testData2 ON key = a and key = 2", - classOf[BroadcastHashOuterJoin]) - ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } - } - sql("UNCACHE TABLE testData") + sql("CACHE TABLE testData2") + Seq( + ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", + classOf[BroadcastHashJoinExec]), + ("SELECT * FROM testData RIGHT JOIN testData2 ON key = a where key = 2", + classOf[BroadcastHashJoinExec]), + ("SELECT * FROM testData right join testData2 ON key = a and key = 2", + classOf[BroadcastHashJoinExec]) + ).foreach(assertJoin) } test("multiple-key equi-join is hash-join") { val x = testData2.as("x") val y = testData2.as("y") val join = x.join(y, ($"x.a" === $"y.a") && ($"x.b" === $"y.b")).queryExecution.optimizedPlan - val planned = sqlContext.planner.EquiJoinSelection(join) + val planned = spark.sessionState.planner.JoinSelection(join) assert(planned.size === 1) } test("inner join where, one match per row") { - checkAnswer( - upperCaseData.join(lowerCaseData).where('n === 'N), - Seq( - Row(1, "A", 1, "a"), - Row(2, "B", 2, "b"), - Row(3, "C", 3, "c"), - Row(4, "D", 4, "d") - )) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer( + upperCaseData.join(lowerCaseData).where('n === 'N), + Seq( + Row(1, "A", 1, "a"), + Row(2, "B", 2, "b"), + Row(3, "C", 3, "c"), + Row(4, "D", 4, "d") + )) + } } test("inner join ON, one match per row") { - checkAnswer( - upperCaseData.join(lowerCaseData, $"n" === $"N"), - Seq( - Row(1, "A", 1, "a"), - Row(2, "B", 2, "b"), - Row(3, "C", 3, "c"), - Row(4, "D", 4, "d") - )) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer( + upperCaseData.join(lowerCaseData, $"n" === $"N"), + Seq( + Row(1, "A", 1, "a"), + Row(2, "B", 2, "b"), + Row(3, "C", 3, "c"), + Row(4, "D", 4, "d") + )) + } } test("inner join, where, multiple matches") { @@ -219,7 +201,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { } test("big inner join, 4 matches per row") { - val bigData = testData.unionAll(testData).unionAll(testData).unionAll(testData) + val bigData = testData.union(testData).union(testData).union(testData) val bigDataX = bigData.as("x") val bigDataY = bigData.as("y") @@ -228,144 +210,168 @@ class JoinSuite extends QueryTest with SharedSQLContext { testData.rdd.flatMap(row => Seq.fill(16)(Row.merge(row, row))).collect().toSeq) } - test("cartisian product join") { - checkAnswer( - testData3.join(testData3), - Row(1, null, 1, null) :: - Row(1, null, 2, 2) :: - Row(2, 2, 1, null) :: - Row(2, 2, 2, 2) :: Nil) + test("cartesian product join") { + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + checkAnswer( + testData3.join(testData3), + Row(1, null, 1, null) :: + Row(1, null, 2, 2) :: + Row(2, 2, 1, null) :: + Row(2, 2, 2, 2) :: Nil) + checkAnswer( + testData3.as("x").join(testData3.as("y"), $"x.a" > $"y.a"), + Row(2, 2, 1, null) :: Nil) + } + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "false") { + val e = intercept[Exception] { + checkAnswer( + testData3.join(testData3), + Row(1, null, 1, null) :: + Row(1, null, 2, 2) :: + Row(2, 2, 1, null) :: + Row(2, 2, 2, 2) :: Nil) + } + assert(e.getMessage.contains("Detected cartesian product for INNER join " + + "between logical plans")) + } } test("left outer join") { - checkAnswer( - upperCaseData.join(lowerCaseData, $"n" === $"N", "left"), - Row(1, "A", 1, "a") :: - Row(2, "B", 2, "b") :: - Row(3, "C", 3, "c") :: - Row(4, "D", 4, "d") :: - Row(5, "E", null, null) :: - Row(6, "F", null, null) :: Nil) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer( + upperCaseData.join(lowerCaseData, $"n" === $"N", "left"), + Row(1, "A", 1, "a") :: + Row(2, "B", 2, "b") :: + Row(3, "C", 3, "c") :: + Row(4, "D", 4, "d") :: + Row(5, "E", null, null) :: + Row(6, "F", null, null) :: Nil) - checkAnswer( - upperCaseData.join(lowerCaseData, $"n" === $"N" && $"n" > 1, "left"), - Row(1, "A", null, null) :: - Row(2, "B", 2, "b") :: - Row(3, "C", 3, "c") :: - Row(4, "D", 4, "d") :: - Row(5, "E", null, null) :: - Row(6, "F", null, null) :: Nil) + checkAnswer( + upperCaseData.join(lowerCaseData, $"n" === $"N" && $"n" > 1, "left"), + Row(1, "A", null, null) :: + Row(2, "B", 2, "b") :: + Row(3, "C", 3, "c") :: + Row(4, "D", 4, "d") :: + Row(5, "E", null, null) :: + Row(6, "F", null, null) :: Nil) - checkAnswer( - upperCaseData.join(lowerCaseData, $"n" === $"N" && $"N" > 1, "left"), - Row(1, "A", null, null) :: - Row(2, "B", 2, "b") :: - Row(3, "C", 3, "c") :: - Row(4, "D", 4, "d") :: - Row(5, "E", null, null) :: - Row(6, "F", null, null) :: Nil) + checkAnswer( + upperCaseData.join(lowerCaseData, $"n" === $"N" && $"N" > 1, "left"), + Row(1, "A", null, null) :: + Row(2, "B", 2, "b") :: + Row(3, "C", 3, "c") :: + Row(4, "D", 4, "d") :: + Row(5, "E", null, null) :: + Row(6, "F", null, null) :: Nil) - checkAnswer( - upperCaseData.join(lowerCaseData, $"n" === $"N" && $"l" > $"L", "left"), - Row(1, "A", 1, "a") :: - Row(2, "B", 2, "b") :: - Row(3, "C", 3, "c") :: - Row(4, "D", 4, "d") :: - Row(5, "E", null, null) :: - Row(6, "F", null, null) :: Nil) - - // Make sure we are choosing left.outputPartitioning as the - // outputPartitioning for the outer join operator. - checkAnswer( - sql( - """ + checkAnswer( + upperCaseData.join(lowerCaseData, $"n" === $"N" && $"l" > $"L", "left"), + Row(1, "A", 1, "a") :: + Row(2, "B", 2, "b") :: + Row(3, "C", 3, "c") :: + Row(4, "D", 4, "d") :: + Row(5, "E", null, null) :: + Row(6, "F", null, null) :: Nil) + + // Make sure we are choosing left.outputPartitioning as the + // outputPartitioning for the outer join operator. + checkAnswer( + sql( + """ |SELECT l.N, count(*) - |FROM upperCaseData l LEFT OUTER JOIN allNulls r ON (l.N = r.a) + |FROM uppercasedata l LEFT OUTER JOIN allnulls r ON (l.N = r.a) |GROUP BY l.N - """.stripMargin), - Row(1, 1) :: + """.stripMargin), + Row( + 1, 1) :: Row(2, 1) :: Row(3, 1) :: Row(4, 1) :: Row(5, 1) :: Row(6, 1) :: Nil) - checkAnswer( - sql( - """ - |SELECT r.a, count(*) - |FROM upperCaseData l LEFT OUTER JOIN allNulls r ON (l.N = r.a) - |GROUP BY r.a - """.stripMargin), - Row(null, 6) :: Nil) + checkAnswer( + sql( + """ + |SELECT r.a, count(*) + |FROM uppercasedata l LEFT OUTER JOIN allnulls r ON (l.N = r.a) + |GROUP BY r.a + """.stripMargin), + Row(null, 6) :: Nil) + } } test("right outer join") { - checkAnswer( - lowerCaseData.join(upperCaseData, $"n" === $"N", "right"), - Row(1, "a", 1, "A") :: - Row(2, "b", 2, "B") :: - Row(3, "c", 3, "C") :: - Row(4, "d", 4, "D") :: - Row(null, null, 5, "E") :: - Row(null, null, 6, "F") :: Nil) - checkAnswer( - lowerCaseData.join(upperCaseData, $"n" === $"N" && $"n" > 1, "right"), - Row(null, null, 1, "A") :: - Row(2, "b", 2, "B") :: - Row(3, "c", 3, "C") :: - Row(4, "d", 4, "D") :: - Row(null, null, 5, "E") :: - Row(null, null, 6, "F") :: Nil) - checkAnswer( - lowerCaseData.join(upperCaseData, $"n" === $"N" && $"N" > 1, "right"), - Row(null, null, 1, "A") :: - Row(2, "b", 2, "B") :: - Row(3, "c", 3, "C") :: - Row(4, "d", 4, "D") :: - Row(null, null, 5, "E") :: - Row(null, null, 6, "F") :: Nil) - checkAnswer( - lowerCaseData.join(upperCaseData, $"n" === $"N" && $"l" > $"L", "right"), - Row(1, "a", 1, "A") :: - Row(2, "b", 2, "B") :: - Row(3, "c", 3, "C") :: - Row(4, "d", 4, "D") :: - Row(null, null, 5, "E") :: - Row(null, null, 6, "F") :: Nil) - - // Make sure we are choosing right.outputPartitioning as the - // outputPartitioning for the outer join operator. - checkAnswer( - sql( - """ - |SELECT l.a, count(*) - |FROM allNulls l RIGHT OUTER JOIN upperCaseData r ON (l.a = r.N) - |GROUP BY l.a - """.stripMargin), - Row(null, 6)) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer( + lowerCaseData.join(upperCaseData, $"n" === $"N", "right"), + Row(1, "a", 1, "A") :: + Row(2, "b", 2, "B") :: + Row(3, "c", 3, "C") :: + Row(4, "d", 4, "D") :: + Row(null, null, 5, "E") :: + Row(null, null, 6, "F") :: Nil) + checkAnswer( + lowerCaseData.join(upperCaseData, $"n" === $"N" && $"n" > 1, "right"), + Row(null, null, 1, "A") :: + Row(2, "b", 2, "B") :: + Row(3, "c", 3, "C") :: + Row(4, "d", 4, "D") :: + Row(null, null, 5, "E") :: + Row(null, null, 6, "F") :: Nil) + checkAnswer( + lowerCaseData.join(upperCaseData, $"n" === $"N" && $"N" > 1, "right"), + Row(null, null, 1, "A") :: + Row(2, "b", 2, "B") :: + Row(3, "c", 3, "C") :: + Row(4, "d", 4, "D") :: + Row(null, null, 5, "E") :: + Row(null, null, 6, "F") :: Nil) + checkAnswer( + lowerCaseData.join(upperCaseData, $"n" === $"N" && $"l" > $"L", "right"), + Row(1, "a", 1, "A") :: + Row(2, "b", 2, "B") :: + Row(3, "c", 3, "C") :: + Row(4, "d", 4, "D") :: + Row(null, null, 5, "E") :: + Row(null, null, 6, "F") :: Nil) + + // Make sure we are choosing right.outputPartitioning as the + // outputPartitioning for the outer join operator. + checkAnswer( + sql( + """ + |SELECT l.a, count(*) + |FROM allnulls l RIGHT OUTER JOIN uppercasedata r ON (l.a = r.N) + |GROUP BY l.a + """.stripMargin), + Row(null, + 6)) - checkAnswer( - sql( - """ - |SELECT r.N, count(*) - |FROM allNulls l RIGHT OUTER JOIN upperCaseData r ON (l.a = r.N) - |GROUP BY r.N - """.stripMargin), - Row(1, 1) :: - Row(2, 1) :: - Row(3, 1) :: - Row(4, 1) :: - Row(5, 1) :: - Row(6, 1) :: Nil) + checkAnswer( + sql( + """ + |SELECT r.N, count(*) + |FROM allnulls l RIGHT OUTER JOIN uppercasedata r ON (l.a = r.N) + |GROUP BY r.N + """.stripMargin), + Row(1 + , 1) :: + Row(2, 1) :: + Row(3, 1) :: + Row(4, 1) :: + Row(5, 1) :: + Row(6, 1) :: Nil) + } } test("full outer join") { - upperCaseData.where('N <= 4).registerTempTable("left") - upperCaseData.where('N >= 3).registerTempTable("right") + upperCaseData.where('N <= 4).createOrReplaceTempView("`left`") + upperCaseData.where('N >= 3).createOrReplaceTempView("`right`") - val left = UnresolvedRelation(TableIdentifier("left"), None) - val right = UnresolvedRelation(TableIdentifier("right"), None) + val left = UnresolvedRelation(TableIdentifier("left")) + val right = UnresolvedRelation(TableIdentifier("right")) checkAnswer( left.join(right, $"left.N" === $"right.N", "full"), @@ -377,7 +383,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { Row(null, null, 6, "F") :: Nil) checkAnswer( - left.join(right, ($"left.N" === $"right.N") && ($"left.N" !== 3), "full"), + left.join(right, ($"left.N" === $"right.N") && ($"left.N" =!= 3), "full"), Row(1, "A", null, null) :: Row(2, "B", null, null) :: Row(3, "C", null, null) :: @@ -387,7 +393,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { Row(null, null, 6, "F") :: Nil) checkAnswer( - left.join(right, ($"left.N" === $"right.N") && ($"right.N" !== 3), "full"), + left.join(right, ($"left.N" === $"right.N") && ($"right.N" =!= 3), "full"), Row(1, "A", null, null) :: Row(2, "B", null, null) :: Row(3, "C", null, null) :: @@ -396,14 +402,16 @@ class JoinSuite extends QueryTest with SharedSQLContext { Row(null, null, 5, "E") :: Row(null, null, 6, "F") :: Nil) - // Make sure we are UnknownPartitioning as the outputPartitioning for the outer join operator. + // Make sure we are UnknownPartitioning as the outputPartitioning for the outer join + // operator. checkAnswer( sql( """ - |SELECT l.a, count(*) - |FROM allNulls l FULL OUTER JOIN upperCaseData r ON (l.a = r.N) - |GROUP BY l.a - """.stripMargin), + |SELECT l.a, count(*) + |FROM allNulls l FULL OUTER JOIN upperCaseData r ON (l.a = r.N) + |GROUP BY l.a + """. + stripMargin), Row(null, 10)) checkAnswer( @@ -413,7 +421,8 @@ class JoinSuite extends QueryTest with SharedSQLContext { |FROM allNulls l FULL OUTER JOIN upperCaseData r ON (l.a = r.N) |GROUP BY r.N """.stripMargin), - Row(1, 1) :: + Row + (1, 1) :: Row(2, 1) :: Row(3, 1) :: Row(4, 1) :: @@ -428,7 +437,8 @@ class JoinSuite extends QueryTest with SharedSQLContext { |FROM upperCaseData l FULL OUTER JOIN allNulls r ON (l.N = r.a) |GROUP BY l.N """.stripMargin), - Row(1, 1) :: + Row(1 + , 1) :: Row(2, 1) :: Row(3, 1) :: Row(4, 1) :: @@ -439,84 +449,85 @@ class JoinSuite extends QueryTest with SharedSQLContext { checkAnswer( sql( """ - |SELECT r.a, count(*) - |FROM upperCaseData l FULL OUTER JOIN allNulls r ON (l.N = r.a) - |GROUP BY r.a - """.stripMargin), + |SELECT r.a, count(*) + |FROM upperCaseData l FULL OUTER JOIN allNulls r ON (l.N = r.a) + |GROUP BY r.a + """. + stripMargin), Row(null, 10)) } - test("broadcasted left semi join operator selection") { - sqlContext.cacheManager.clearCache() + test("broadcasted existence join operator selection") { + spark.sharedState.cacheManager.clearCache() sql("CACHE TABLE testData") - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1000000000") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Long.MaxValue.toString) { Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", - classOf[BroadcastLeftSemiJoinHash]) - ).foreach { - case (query, joinClass) => assertJoin(query, joinClass) - } + classOf[BroadcastHashJoinExec]), + ("SELECT * FROM testData ANT JOIN testData2 ON key = a", classOf[BroadcastHashJoinExec]) + ).foreach(assertJoin) } withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { Seq( - ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[LeftSemiJoinHash]) - ).foreach { - case (query, joinClass) => assertJoin(query, joinClass) - } + ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", + classOf[SortMergeJoinExec]), + ("SELECT * FROM testData LEFT ANTI JOIN testData2 ON key = a", + classOf[SortMergeJoinExec]) + ).foreach(assertJoin) } - sql("UNCACHE TABLE testData") } test("cross join with broadcast") { sql("CACHE TABLE testData") - val sizeInByteOfTestData = statisticSizeInByte(sqlContext.table("testData")) + val sizeInByteOfTestData = statisticSizeInByte(spark.table("testData")) // we set the threshold is greater than statistic of the cached table testData withSQLConf( - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> (sizeInByteOfTestData + 1).toString()) { + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> (sizeInByteOfTestData + 1).toString(), + SQLConf.CROSS_JOINS_ENABLED.key -> "true") { - assert(statisticSizeInByte(sqlContext.table("testData2")) > - sqlContext.conf.autoBroadcastJoinThreshold) + assert(statisticSizeInByte(spark.table("testData2")) > + spark.conf.get(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD)) - assert(statisticSizeInByte(sqlContext.table("testData")) < - sqlContext.conf.autoBroadcastJoinThreshold) + assert(statisticSizeInByte(spark.table("testData")) < + spark.conf.get(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD)) Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", - classOf[LeftSemiJoinHash]), + classOf[SortMergeJoinExec]), ("SELECT * FROM testData LEFT SEMI JOIN testData2", - classOf[LeftSemiJoinBNL]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData JOIN testData2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData JOIN testData2 WHERE key = 2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData LEFT JOIN testData2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData RIGHT JOIN testData2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData FULL OUTER JOIN testData2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData LEFT JOIN testData2 WHERE key = 2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData RIGHT JOIN testData2 WHERE key = 2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key = 2", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData JOIN testData2 WHERE key > a", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key > a", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData left JOIN testData2 WHERE (key * a != key + a)", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData right JOIN testData2 WHERE (key * a != key + a)", - classOf[BroadcastNestedLoopJoin]), + classOf[BroadcastNestedLoopJoinExec]), ("SELECT * FROM testData full JOIN testData2 WHERE (key * a != key + a)", - classOf[BroadcastNestedLoopJoin]) - ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + classOf[BroadcastNestedLoopJoinExec]) + ).foreach(assertJoin) checkAnswer( sql( @@ -555,7 +566,6 @@ class JoinSuite extends QueryTest with SharedSQLContext { Row("2", 3, 2) :: Nil) } - sql("UNCACHE TABLE testData") } test("left semi join") { @@ -568,4 +578,275 @@ class JoinSuite extends QueryTest with SharedSQLContext { Row(3, 1) :: Row(3, 2) :: Nil) } + + test("cross join detection") { + testData.createOrReplaceTempView("A") + testData.createOrReplaceTempView("B") + testData2.createOrReplaceTempView("C") + testData3.createOrReplaceTempView("D") + upperCaseData.where('N >= 3).createOrReplaceTempView("`right`") + val cartesianQueries = Seq( + /** The following should error out since there is no explicit cross join */ + "SELECT * FROM testData inner join testData2", + "SELECT * FROM testData left outer join testData2", + "SELECT * FROM testData right outer join testData2", + "SELECT * FROM testData full outer join testData2", + "SELECT * FROM testData, testData2", + "SELECT * FROM testData, testData2 where testData.key = 1 and testData2.a = 22", + /** The following should fail because after reordering there are cartesian products */ + "select * from (A join B on (A.key = B.key)) join D on (A.key=D.a) join C", + "select * from ((A join B on (A.key = B.key)) join C) join D on (A.key = D.a)", + /** Cartesian product involving C, which is not involved in a CROSS join */ + "select * from ((A join B on (A.key = B.key)) cross join D) join C on (A.key = D.a)"); + + def checkCartesianDetection(query: String): Unit = { + val e = intercept[Exception] { + checkAnswer(sql(query), Nil); + } + assert(e.getMessage.contains("Detected cartesian product")) + } + + cartesianQueries.foreach(checkCartesianDetection) + + // Check that left_semi, left_anti, existence joins without conditions do not throw + // an exception if cross joins are disabled + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "false") { + checkAnswer( + sql("SELECT * FROM testData3 LEFT SEMI JOIN testData2"), + Row(1, null) :: Row (2, 2) :: Nil) + checkAnswer( + sql("SELECT * FROM testData3 LEFT ANTI JOIN testData2"), + Nil) + checkAnswer( + sql( + """ + |SELECT a FROM testData3 + |WHERE + | EXISTS (SELECT * FROM testData) + |OR + | EXISTS (SELECT * FROM testData2)""".stripMargin), + Row(1) :: Row(2) :: Nil) + checkAnswer( + sql( + """ + |SELECT key FROM testData + |WHERE + | key IN (SELECT a FROM testData2) + |OR + | key IN (SELECT a FROM testData3)""".stripMargin), + Row(1) :: Row(2) :: Row(3) :: Nil) + } + } + + test("test SortMergeJoin (without spill)") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", + "spark.sql.sortMergeJoinExec.buffer.spill.threshold" -> Int.MaxValue.toString) { + + assertNotSpilled(sparkContext, "inner join") { + checkAnswer( + sql("SELECT * FROM testData JOIN testData2 ON key = a where key = 2"), + Row(2, "2", 2, 1) :: Row(2, "2", 2, 2) :: Nil + ) + } + + val expected = new ListBuffer[Row]() + expected.append( + Row(1, "1", 1, 1), Row(1, "1", 1, 2), + Row(2, "2", 2, 1), Row(2, "2", 2, 2), + Row(3, "3", 3, 1), Row(3, "3", 3, 2) + ) + for (i <- 4 to 100) { + expected.append(Row(i, i.toString, null, null)) + } + + assertNotSpilled(sparkContext, "left outer join") { + checkAnswer( + sql( + """ + |SELECT + | big.key, big.value, small.a, small.b + |FROM + | testData big + |LEFT OUTER JOIN + | testData2 small + |ON + | big.key = small.a + """.stripMargin), + expected + ) + } + + assertNotSpilled(sparkContext, "right outer join") { + checkAnswer( + sql( + """ + |SELECT + | big.key, big.value, small.a, small.b + |FROM + | testData2 small + |RIGHT OUTER JOIN + | testData big + |ON + | big.key = small.a + """.stripMargin), + expected + ) + } + } + } + + test("test SortMergeJoin (with spill)") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", + "spark.sql.sortMergeJoinExec.buffer.in.memory.threshold" -> "0", + "spark.sql.sortMergeJoinExec.buffer.spill.threshold" -> "1") { + + assertSpilled(sparkContext, "inner join") { + checkAnswer( + sql("SELECT * FROM testData JOIN testData2 ON key = a where key = 2"), + Row(2, "2", 2, 1) :: Row(2, "2", 2, 2) :: Nil + ) + } + + val expected = new ListBuffer[Row]() + expected.append( + Row(1, "1", 1, 1), Row(1, "1", 1, 2), + Row(2, "2", 2, 1), Row(2, "2", 2, 2), + Row(3, "3", 3, 1), Row(3, "3", 3, 2) + ) + for (i <- 4 to 100) { + expected.append(Row(i, i.toString, null, null)) + } + + assertSpilled(sparkContext, "left outer join") { + checkAnswer( + sql( + """ + |SELECT + | big.key, big.value, small.a, small.b + |FROM + | testData big + |LEFT OUTER JOIN + | testData2 small + |ON + | big.key = small.a + """.stripMargin), + expected + ) + } + + assertSpilled(sparkContext, "right outer join") { + checkAnswer( + sql( + """ + |SELECT + | big.key, big.value, small.a, small.b + |FROM + | testData2 small + |RIGHT OUTER JOIN + | testData big + |ON + | big.key = small.a + """.stripMargin), + expected + ) + } + + // FULL OUTER JOIN still does not use [[ExternalAppendOnlyUnsafeRowArray]] + // so should not cause any spill + assertNotSpilled(sparkContext, "full outer join") { + checkAnswer( + sql( + """ + |SELECT + | big.key, big.value, small.a, small.b + |FROM + | testData2 small + |FULL OUTER JOIN + | testData big + |ON + | big.key = small.a + """.stripMargin), + expected + ) + } + } + } + + test("outer broadcast hash join should not throw NPE") { + withTempView("v1", "v2") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + Seq(2 -> 2).toDF("x", "y").createTempView("v1") + + spark.createDataFrame( + Seq(Row(1, "a")).asJava, + new StructType().add("i", "int", nullable = false).add("j", "string", nullable = false) + ).createTempView("v2") + + checkAnswer( + sql("select x, y, i, j from v1 left join v2 on x = i and y < length(j)"), + Row(2, 2, null, null) + ) + } + } + } + + test("test SortMergeJoin output ordering") { + val joinQueries = Seq( + "SELECT * FROM testData JOIN testData2 ON key = a", + "SELECT * FROM testData t1 JOIN " + + "testData2 t2 ON t1.key = t2.a JOIN testData3 t3 ON t2.a = t3.a", + "SELECT * FROM testData t1 JOIN " + + "testData2 t2 ON t1.key = t2.a JOIN " + + "testData3 t3 ON t2.a = t3.a JOIN " + + "testData t4 ON t1.key = t4.key") + + def assertJoinOrdering(sqlString: String): Unit = { + val df = sql(sqlString) + val physical = df.queryExecution.sparkPlan + val physicalJoins = physical.collect { + case j: SortMergeJoinExec => j + } + val executed = df.queryExecution.executedPlan + val executedJoins = executed.collect { + case j: SortMergeJoinExec => j + } + // This only applies to the above tested queries, in which a child SortMergeJoin always + // contains the SortOrder required by its parent SortMergeJoin. Thus, SortExec should never + // appear as parent of SortMergeJoin. + executed.foreach { + case s: SortExec => s.foreach { + case j: SortMergeJoinExec => fail( + s"No extra sort should be added since $j already satisfies the required ordering" + ) + case _ => + } + case _ => + } + val joinPairs = physicalJoins.zip(executedJoins) + val numOfJoins = sqlString.split(" ").count(_.toUpperCase == "JOIN") + assert(joinPairs.size == numOfJoins) + + joinPairs.foreach { + case(join1, join2) => + val leftKeys = join1.leftKeys + val rightKeys = join1.rightKeys + val outputOrderingPhysical = join1.outputOrdering + val outputOrderingExecuted = join2.outputOrdering + + // outputOrdering should always contain join keys + assert( + SortOrder.orderingSatisfies( + outputOrderingPhysical, leftKeys.map(SortOrder(_, Ascending)))) + assert( + SortOrder.orderingSatisfies( + outputOrderingPhysical, rightKeys.map(SortOrder(_, Ascending)))) + // outputOrdering should be consistent between physical plan and executed plan + assert(outputOrderingPhysical == outputOrderingExecuted, + s"Operator $join1 did not have the same output ordering in the physical plan as in " + + s"the executed plan.") + } + } + + joinQueries.foreach(assertJoinOrdering) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index e3531d0d6d79..00d2acc4a1d8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -17,7 +17,9 @@ package org.apache.spark.sql +import org.apache.spark.sql.functions.{from_json, lit, map, struct, to_json} import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -29,7 +31,6 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { Row("alice", "5")) } - val tuples: Seq[(String, String)] = ("1", """{"f1": "value1", "f2": "value2", "f3": 3, "f5": 5.23}""") :: ("2", """{"f1": "value12", "f3": "value3", "f2": 2, "f4": 4.01}""") :: @@ -39,25 +40,52 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { ("6", "[invalid JSON string]") :: Nil + test("function get_json_object - null") { + val df: DataFrame = tuples.toDF("key", "jstring") + val expected = + Row("1", "value1", "value2", "3", null, "5.23") :: + Row("2", "value12", "2", "value3", "4.01", null) :: + Row("3", "value13", "2", "value33", "value44", "5.01") :: + Row("4", null, null, null, null, null) :: + Row("5", "", null, null, null, null) :: + Row("6", null, null, null, null, null) :: + Nil + + checkAnswer( + df.select($"key", functions.get_json_object($"jstring", "$.f1"), + functions.get_json_object($"jstring", "$.f2"), + functions.get_json_object($"jstring", "$.f3"), + functions.get_json_object($"jstring", "$.f4"), + functions.get_json_object($"jstring", "$.f5")), + expected) + } + test("json_tuple select") { val df: DataFrame = tuples.toDF("key", "jstring") - val expected = Row("1", Row("value1", "value2", "3", null, "5.23")) :: - Row("2", Row("value12", "2", "value3", "4.01", null)) :: - Row("3", Row("value13", "2", "value33", "value44", "5.01")) :: - Row("4", Row(null, null, null, null, null)) :: - Row("5", Row("", null, null, null, null)) :: - Row("6", Row(null, null, null, null, null)) :: + val expected = + Row("1", "value1", "value2", "3", null, "5.23") :: + Row("2", "value12", "2", "value3", "4.01", null) :: + Row("3", "value13", "2", "value33", "value44", "5.01") :: + Row("4", null, null, null, null, null) :: + Row("5", "", null, null, null, null) :: + Row("6", null, null, null, null, null) :: Nil - checkAnswer(df.selectExpr("key", "json_tuple(jstring, 'f1', 'f2', 'f3', 'f4', 'f5')"), expected) + checkAnswer( + df.select($"key", functions.json_tuple($"jstring", "f1", "f2", "f3", "f4", "f5")), + expected) + + checkAnswer( + df.selectExpr("key", "json_tuple(jstring, 'f1', 'f2', 'f3', 'f4', 'f5')"), + expected) } test("json_tuple filter and group") { val df: DataFrame = tuples.toDF("key", "jstring") val expr = df - .selectExpr("json_tuple(jstring, 'f1', 'f2') as jt") - .where($"jt.c0".isNotNull) - .groupBy($"jt.c1") + .select(functions.json_tuple($"jstring", "f1", "f2")) + .where($"c0".isNotNull) + .groupBy($"c1") .count() val expected = Row(null, 1) :: @@ -67,4 +95,235 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { checkAnswer(expr, expected) } + + test("from_json") { + val df = Seq("""{"a": 1}""").toDS() + val schema = new StructType().add("a", IntegerType) + + checkAnswer( + df.select(from_json($"value", schema)), + Row(Row(1)) :: Nil) + } + + test("from_json with option") { + val df = Seq("""{"time": "26/08/2015 18:00"}""").toDS() + val schema = new StructType().add("time", TimestampType) + val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm") + + checkAnswer( + df.select(from_json($"value", schema, options)), + Row(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))) + } + + test("from_json missing columns") { + val df = Seq("""{"a": 1}""").toDS() + val schema = new StructType().add("b", IntegerType) + + checkAnswer( + df.select(from_json($"value", schema)), + Row(Row(null)) :: Nil) + } + + test("from_json invalid json") { + val df = Seq("""{"a" 1}""").toDS() + val schema = new StructType().add("a", IntegerType) + + checkAnswer( + df.select(from_json($"value", schema)), + Row(null) :: Nil) + } + + test("from_json invalid schema") { + val df = Seq("""{"a" 1}""").toDS() + val schema = ArrayType(StringType) + val message = intercept[AnalysisException] { + df.select(from_json($"value", schema)) + }.getMessage + + assert(message.contains( + "Input schema array must be a struct or an array of structs.")) + } + + test("from_json array support") { + val df = Seq("""[{"a": 1, "b": "a"}, {"a": 2}, { }]""").toDS() + val schema = ArrayType( + StructType( + StructField("a", IntegerType) :: + StructField("b", StringType) :: Nil)) + + checkAnswer( + df.select(from_json($"value", schema)), + Row(Seq(Row(1, "a"), Row(2, null), Row(null, null)))) + } + + test("from_json uses DDL strings for defining a schema - java") { + val df = Seq("""{"a": 1, "b": "haa"}""").toDS() + checkAnswer( + df.select(from_json($"value", "a INT, b STRING", new java.util.HashMap[String, String]())), + Row(Row(1, "haa")) :: Nil) + } + + test("from_json uses DDL strings for defining a schema - scala") { + val df = Seq("""{"a": 1, "b": "haa"}""").toDS() + checkAnswer( + df.select(from_json($"value", "a INT, b STRING", Map[String, String]())), + Row(Row(1, "haa")) :: Nil) + } + + test("to_json - struct") { + val df = Seq(Tuple1(Tuple1(1))).toDF("a") + + checkAnswer( + df.select(to_json($"a")), + Row("""{"_1":1}""") :: Nil) + } + + test("to_json - array") { + val df = Seq(Tuple1(Tuple1(1) :: Nil)).toDF("a") + val df2 = Seq(Tuple1(Map("a" -> 1) :: Nil)).toDF("a") + + checkAnswer( + df.select(to_json($"a")), + Row("""[{"_1":1}]""") :: Nil) + checkAnswer( + df2.select(to_json($"a")), + Row("""[{"a":1}]""") :: Nil) + } + + test("to_json - map") { + val df1 = Seq(Map("a" -> Tuple1(1))).toDF("a") + val df2 = Seq(Map("a" -> 1)).toDF("a") + + checkAnswer( + df1.select(to_json($"a")), + Row("""{"a":{"_1":1}}""") :: Nil) + checkAnswer( + df2.select(to_json($"a")), + Row("""{"a":1}""") :: Nil) + } + + test("to_json with option") { + val df = Seq(Tuple1(Tuple1(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))).toDF("a") + val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm") + + checkAnswer( + df.select(to_json($"a", options)), + Row("""{"_1":"26/08/2015 18:00"}""") :: Nil) + } + + test("to_json - key types of map don't matter") { + // interval type is invalid for converting to JSON. However, the keys of a map are treated + // as strings, so its type doesn't matter. + val df = Seq(Tuple1(Tuple1("interval -3 month 7 hours"))).toDF("a") + .select(struct(map($"a._1".cast(CalendarIntervalType), lit("a")).as("col1")).as("c")) + checkAnswer( + df.select(to_json($"c")), + Row("""{"col1":{"interval -3 months 7 hours":"a"}}""") :: Nil) + } + + test("to_json unsupported type") { + val baseDf = Seq(Tuple1(Tuple1("interval -3 month 7 hours"))).toDF("a") + val df = baseDf.select(struct($"a._1".cast(CalendarIntervalType).as("a")).as("c")) + val e = intercept[AnalysisException]{ + // Unsupported type throws an exception + df.select(to_json($"c")).collect() + } + assert(e.getMessage.contains( + "Unable to convert column a of type calendarinterval to JSON.")) + + // interval type is invalid for converting to JSON. We can't use it as value type of a map. + val df2 = baseDf + .select(struct(map(lit("a"), $"a._1".cast(CalendarIntervalType)).as("col1")).as("c")) + val e2 = intercept[AnalysisException] { + df2.select(to_json($"c")).collect() + } + assert(e2.getMessage.contains("Unable to convert column col1 of type calendarinterval to JSON")) + } + + test("roundtrip in to_json and from_json - struct") { + val dfOne = Seq(Tuple1(Tuple1(1)), Tuple1(null)).toDF("struct") + val schemaOne = dfOne.schema(0).dataType.asInstanceOf[StructType] + val readBackOne = dfOne.select(to_json($"struct").as("json")) + .select(from_json($"json", schemaOne).as("struct")) + checkAnswer(dfOne, readBackOne) + + val dfTwo = Seq(Some("""{"a":1}"""), None).toDF("json") + val schemaTwo = new StructType().add("a", IntegerType) + val readBackTwo = dfTwo.select(from_json($"json", schemaTwo).as("struct")) + .select(to_json($"struct").as("json")) + checkAnswer(dfTwo, readBackTwo) + } + + test("roundtrip in to_json and from_json - array") { + val dfOne = Seq(Tuple1(Tuple1(1) :: Nil), Tuple1(null :: Nil)).toDF("array") + val schemaOne = dfOne.schema(0).dataType + val readBackOne = dfOne.select(to_json($"array").as("json")) + .select(from_json($"json", schemaOne).as("array")) + checkAnswer(dfOne, readBackOne) + + val dfTwo = Seq(Some("""[{"a":1}]"""), None).toDF("json") + val schemaTwo = ArrayType(StructType(StructField("a", IntegerType) :: Nil)) + val readBackTwo = dfTwo.select(from_json($"json", schemaTwo).as("array")) + .select(to_json($"array").as("json")) + checkAnswer(dfTwo, readBackTwo) + } + + test("SPARK-19637 Support to_json in SQL") { + val df1 = Seq(Tuple1(Tuple1(1))).toDF("a") + checkAnswer( + df1.selectExpr("to_json(a)"), + Row("""{"_1":1}""") :: Nil) + + val df2 = Seq(Tuple1(Tuple1(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))).toDF("a") + checkAnswer( + df2.selectExpr("to_json(a, map('timestampFormat', 'dd/MM/yyyy HH:mm'))"), + Row("""{"_1":"26/08/2015 18:00"}""") :: Nil) + + val errMsg1 = intercept[AnalysisException] { + df2.selectExpr("to_json(a, named_struct('a', 1))") + } + assert(errMsg1.getMessage.startsWith("Must use a map() function for options")) + + val errMsg2 = intercept[AnalysisException] { + df2.selectExpr("to_json(a, map('a', 1))") + } + assert(errMsg2.getMessage.startsWith( + "A type of keys and values in map() must be string, but got")) + } + + test("SPARK-19967 Support from_json in SQL") { + val df1 = Seq("""{"a": 1}""").toDS() + checkAnswer( + df1.selectExpr("from_json(value, 'a INT')"), + Row(Row(1)) :: Nil) + + val df2 = Seq("""{"c0": "a", "c1": 1, "c2": {"c20": 3.8, "c21": 8}}""").toDS() + checkAnswer( + df2.selectExpr("from_json(value, 'c0 STRING, c1 INT, c2 STRUCT')"), + Row(Row("a", 1, Row(3.8, 8))) :: Nil) + + val df3 = Seq("""{"time": "26/08/2015 18:00"}""").toDS() + checkAnswer( + df3.selectExpr( + "from_json(value, 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy HH:mm'))"), + Row(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))) + + val errMsg1 = intercept[AnalysisException] { + df3.selectExpr("from_json(value, 1)") + } + assert(errMsg1.getMessage.startsWith("Expected a string literal instead of")) + val errMsg2 = intercept[AnalysisException] { + df3.selectExpr("""from_json(value, 'time InvalidType')""") + } + assert(errMsg2.getMessage.contains("DataType invalidtype is not supported")) + val errMsg3 = intercept[AnalysisException] { + df3.selectExpr("from_json(value, 'time Timestamp', named_struct('a', 1))") + } + assert(errMsg3.getMessage.startsWith("Must use a map() function for options")) + val errMsg4 = intercept[AnalysisException] { + df3.selectExpr("from_json(value, 'time Timestamp', map('a', 1))") + } + assert(errMsg4.getMessage.startsWith( + "A type of keys and values in map() must be string, but got")) + } } 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 deleted file mode 100644 index 5688f46e5e3d..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/ListTablesSuite.scala +++ /dev/null @@ -1,86 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ - -package org.apache.spark.sql - -import org.scalatest.BeforeAndAfter - -import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{BooleanType, StringType, StructField, StructType} -import org.apache.spark.sql.catalyst.TableIdentifier - -class ListTablesSuite extends QueryTest with BeforeAndAfter with SharedSQLContext { - import testImplicits._ - - private lazy val df = (1 to 10).map(i => (i, s"str$i")).toDF("key", "value") - - before { - df.registerTempTable("ListTablesSuiteTable") - } - - after { - sqlContext.catalog.unregisterTable(TableIdentifier("ListTablesSuiteTable")) - } - - test("get all tables") { - checkAnswer( - sqlContext.tables().filter("tableName = 'ListTablesSuiteTable'"), - Row("ListTablesSuiteTable", true)) - - checkAnswer( - sql("SHOW tables").filter("tableName = 'ListTablesSuiteTable'"), - Row("ListTablesSuiteTable", true)) - - sqlContext.catalog.unregisterTable(TableIdentifier("ListTablesSuiteTable")) - assert(sqlContext.tables().filter("tableName = 'ListTablesSuiteTable'").count() === 0) - } - - test("getting all Tables with a database name has no impact on returned table names") { - checkAnswer( - sqlContext.tables("DB").filter("tableName = 'ListTablesSuiteTable'"), - Row("ListTablesSuiteTable", true)) - - checkAnswer( - sql("show TABLES in DB").filter("tableName = 'ListTablesSuiteTable'"), - Row("ListTablesSuiteTable", true)) - - sqlContext.catalog.unregisterTable(TableIdentifier("ListTablesSuiteTable")) - assert(sqlContext.tables().filter("tableName = 'ListTablesSuiteTable'").count() === 0) - } - - test("query the returned DataFrame of tables") { - val expectedSchema = StructType( - StructField("tableName", StringType, false) :: - StructField("isTemporary", BooleanType, false) :: Nil) - - Seq(sqlContext.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( - sqlContext.tables().filter("tableName = 'tables'").select("tableName", "isTemporary"), - Row("tables", true)) - sqlContext.dropTempTable("tables") - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/LocalSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/LocalSparkSession.scala new file mode 100644 index 000000000000..d66a6902b051 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/LocalSparkSession.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import _root_.io.netty.util.internal.logging.{InternalLoggerFactory, Slf4JLoggerFactory} +import org.scalatest.BeforeAndAfterAll +import org.scalatest.BeforeAndAfterEach +import org.scalatest.Suite + +/** Manages a local `spark` {@link SparkSession} variable, correctly stopping it after each test. */ +trait LocalSparkSession extends BeforeAndAfterEach with BeforeAndAfterAll { self: Suite => + + @transient var spark: SparkSession = _ + + override def beforeAll() { + super.beforeAll() + InternalLoggerFactory.setDefaultFactory(Slf4JLoggerFactory.INSTANCE) + } + + override def afterEach() { + try { + resetSparkContext() + } finally { + super.afterEach() + } + } + + def resetSparkContext(): Unit = { + LocalSparkSession.stop(spark) + spark = null + } + +} + +object LocalSparkSession { + def stop(spark: SparkSession) { + if (spark != null) { + spark.stop() + } + // To avoid RPC rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port") + } + + /** Runs `f` by passing in `sc` and ensures that `sc` is stopped. */ + def withSparkSession[T](sc: SparkSession)(f: SparkSession => T): T = { + try { + f(sc) + } finally { + stop(sc) + } + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala deleted file mode 100644 index 58f982c2bc93..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala +++ /dev/null @@ -1,402 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql - -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.functions.{log => logarithm} -import org.apache.spark.sql.test.SharedSQLContext - -private object MathExpressionsTestData { - case class DoubleData(a: java.lang.Double, b: java.lang.Double) - case class NullDoubles(a: java.lang.Double) -} - -class MathExpressionsSuite extends QueryTest with SharedSQLContext { - import MathExpressionsTestData._ - import testImplicits._ - - private lazy val doubleData = (1 to 10).map(i => DoubleData(i * 0.2 - 1, i * -0.2 + 1)).toDF() - - private lazy val nnDoubleData = (1 to 10).map(i => DoubleData(i * 0.1, i * -0.1)).toDF() - - private lazy val nullDoubles = - Seq(NullDoubles(1.0), NullDoubles(2.0), NullDoubles(3.0), NullDoubles(null)).toDF() - - private def testOneToOneMathFunction[ - @specialized(Int, Long, Float, Double) T, - @specialized(Int, Long, Float, Double) U]( - c: Column => Column, - f: T => U): Unit = { - checkAnswer( - doubleData.select(c('a)), - (1 to 10).map(n => Row(f((n * 0.2 - 1).asInstanceOf[T]))) - ) - - checkAnswer( - doubleData.select(c('b)), - (1 to 10).map(n => Row(f((-n * 0.2 + 1).asInstanceOf[T]))) - ) - - checkAnswer( - doubleData.select(c(lit(null))), - (1 to 10).map(_ => Row(null)) - ) - } - - private def testOneToOneNonNegativeMathFunction(c: Column => Column, f: Double => Double): Unit = - { - checkAnswer( - nnDoubleData.select(c('a)), - (1 to 10).map(n => Row(f(n * 0.1))) - ) - - if (f(-1) === math.log1p(-1)) { - checkAnswer( - nnDoubleData.select(c('b)), - (1 to 9).map(n => Row(f(n * -0.1))) :+ Row(null) - ) - } - - checkAnswer( - nnDoubleData.select(c(lit(null))), - (1 to 10).map(_ => Row(null)) - ) - } - - private def testTwoToOneMathFunction( - c: (Column, Column) => Column, - d: (Column, Double) => Column, - f: (Double, Double) => Double): Unit = { - checkAnswer( - nnDoubleData.select(c('a, 'a)), - nnDoubleData.collect().toSeq.map(r => Row(f(r.getDouble(0), r.getDouble(0)))) - ) - - checkAnswer( - nnDoubleData.select(c('a, 'b)), - nnDoubleData.collect().toSeq.map(r => Row(f(r.getDouble(0), r.getDouble(1)))) - ) - - checkAnswer( - nnDoubleData.select(d('a, 2.0)), - nnDoubleData.collect().toSeq.map(r => Row(f(r.getDouble(0), 2.0))) - ) - - checkAnswer( - nnDoubleData.select(d('a, -0.5)), - nnDoubleData.collect().toSeq.map(r => Row(f(r.getDouble(0), -0.5))) - ) - - val nonNull = nullDoubles.collect().toSeq.filter(r => r.get(0) != null) - - checkAnswer( - nullDoubles.select(c('a, 'a)).orderBy('a.asc), - Row(null) +: nonNull.map(r => Row(f(r.getDouble(0), r.getDouble(0)))) - ) - } - - test("sin") { - testOneToOneMathFunction(sin, math.sin) - } - - test("asin") { - testOneToOneMathFunction(asin, math.asin) - } - - test("sinh") { - testOneToOneMathFunction(sinh, math.sinh) - } - - test("cos") { - testOneToOneMathFunction(cos, math.cos) - } - - test("acos") { - testOneToOneMathFunction(acos, math.acos) - } - - test("cosh") { - testOneToOneMathFunction(cosh, math.cosh) - } - - test("tan") { - testOneToOneMathFunction(tan, math.tan) - } - - test("atan") { - testOneToOneMathFunction(atan, math.atan) - } - - test("tanh") { - testOneToOneMathFunction(tanh, math.tanh) - } - - test("toDegrees") { - testOneToOneMathFunction(toDegrees, math.toDegrees) - checkAnswer( - sql("SELECT degrees(0), degrees(1), degrees(1.5)"), - Seq((1, 2)).toDF().select(toDegrees(lit(0)), toDegrees(lit(1)), toDegrees(lit(1.5))) - ) - } - - test("toRadians") { - testOneToOneMathFunction(toRadians, math.toRadians) - checkAnswer( - sql("SELECT radians(0), radians(1), radians(1.5)"), - Seq((1, 2)).toDF().select(toRadians(lit(0)), toRadians(lit(1)), toRadians(lit(1.5))) - ) - } - - test("cbrt") { - testOneToOneMathFunction(cbrt, math.cbrt) - } - - test("ceil and ceiling") { - testOneToOneMathFunction(ceil, (d: Double) => math.ceil(d).toLong) - checkAnswer( - sql("SELECT ceiling(0), ceiling(1), ceiling(1.5)"), - Row(0L, 1L, 2L)) - } - - test("conv") { - val df = Seq(("333", 10, 2)).toDF("num", "fromBase", "toBase") - checkAnswer(df.select(conv('num, 10, 16)), Row("14D")) - checkAnswer(df.select(conv(lit(100), 2, 16)), Row("4")) - checkAnswer(df.select(conv(lit(3122234455L), 10, 16)), Row("BA198457")) - checkAnswer(df.selectExpr("conv(num, fromBase, toBase)"), Row("101001101")) - checkAnswer(df.selectExpr("""conv("100", 2, 10)"""), Row("4")) - checkAnswer(df.selectExpr("""conv("-10", 16, -10)"""), Row("-16")) - checkAnswer( - df.selectExpr("""conv("9223372036854775807", 36, -16)"""), Row("-1")) // for overflow - } - - test("floor") { - testOneToOneMathFunction(floor, (d: Double) => math.floor(d).toLong) - } - - test("factorial") { - val df = (0 to 5).map(i => (i, i)).toDF("a", "b") - checkAnswer( - df.select(factorial('a)), - Seq(Row(1), Row(1), Row(2), Row(6), Row(24), Row(120)) - ) - checkAnswer( - df.selectExpr("factorial(a)"), - Seq(Row(1), Row(1), Row(2), Row(6), Row(24), Row(120)) - ) - } - - test("rint") { - testOneToOneMathFunction(rint, math.rint) - } - - test("round") { - val df = Seq(5, 55, 555).map(Tuple1(_)).toDF("a") - checkAnswer( - df.select(round('a), round('a, -1), round('a, -2)), - Seq(Row(5, 10, 0), Row(55, 60, 100), Row(555, 560, 600)) - ) - - val pi = 3.1415 - checkAnswer( - sql(s"SELECT round($pi, -3), round($pi, -2), round($pi, -1), " + - s"round($pi, 0), round($pi, 1), round($pi, 2), round($pi, 3)"), - Seq(Row(BigDecimal("0E3"), BigDecimal("0E2"), BigDecimal("0E1"), BigDecimal(3), - BigDecimal("3.1"), BigDecimal("3.14"), BigDecimal("3.142"))) - ) - } - - test("exp") { - testOneToOneMathFunction(exp, math.exp) - } - - test("expm1") { - testOneToOneMathFunction(expm1, math.expm1) - } - - test("signum / sign") { - testOneToOneMathFunction[Double, Double](signum, math.signum) - - checkAnswer( - sql("SELECT sign(10), signum(-11)"), - Row(1, -1)) - } - - test("pow / power") { - testTwoToOneMathFunction(pow, pow, math.pow) - - checkAnswer( - sql("SELECT pow(1, 2), power(2, 1)"), - Seq((1, 2)).toDF().select(pow(lit(1), lit(2)), pow(lit(2), lit(1))) - ) - } - - test("hex") { - val data = Seq((28, -28, 100800200404L, "hello")).toDF("a", "b", "c", "d") - checkAnswer(data.select(hex('a)), Seq(Row("1C"))) - checkAnswer(data.select(hex('b)), Seq(Row("FFFFFFFFFFFFFFE4"))) - checkAnswer(data.select(hex('c)), Seq(Row("177828FED4"))) - checkAnswer(data.select(hex('d)), Seq(Row("68656C6C6F"))) - checkAnswer(data.selectExpr("hex(a)"), Seq(Row("1C"))) - checkAnswer(data.selectExpr("hex(b)"), Seq(Row("FFFFFFFFFFFFFFE4"))) - checkAnswer(data.selectExpr("hex(c)"), Seq(Row("177828FED4"))) - checkAnswer(data.selectExpr("hex(d)"), Seq(Row("68656C6C6F"))) - checkAnswer(data.selectExpr("hex(cast(d as binary))"), Seq(Row("68656C6C6F"))) - } - - test("unhex") { - val data = Seq(("1C", "737472696E67")).toDF("a", "b") - checkAnswer(data.select(unhex('a)), Row(Array[Byte](28.toByte))) - checkAnswer(data.select(unhex('b)), Row("string".getBytes)) - checkAnswer(data.selectExpr("unhex(a)"), Row(Array[Byte](28.toByte))) - checkAnswer(data.selectExpr("unhex(b)"), Row("string".getBytes)) - checkAnswer(data.selectExpr("""unhex("##")"""), Row(null)) - checkAnswer(data.selectExpr("""unhex("G123")"""), Row(null)) - } - - test("hypot") { - testTwoToOneMathFunction(hypot, hypot, math.hypot) - } - - test("atan2") { - testTwoToOneMathFunction(atan2, atan2, math.atan2) - } - - test("log / ln") { - testOneToOneNonNegativeMathFunction(org.apache.spark.sql.functions.log, math.log) - checkAnswer( - sql("SELECT ln(0), ln(1), ln(1.5)"), - Seq((1, 2)).toDF().select(logarithm(lit(0)), logarithm(lit(1)), logarithm(lit(1.5))) - ) - } - - test("log10") { - testOneToOneNonNegativeMathFunction(log10, math.log10) - } - - test("log1p") { - testOneToOneNonNegativeMathFunction(log1p, math.log1p) - } - - test("shift left") { - val df = Seq[(Long, Integer, Short, Byte, Integer, Integer)]((21, 21, 21, 21, 21, null)) - .toDF("a", "b", "c", "d", "e", "f") - - checkAnswer( - df.select( - shiftLeft('a, 1), shiftLeft('b, 1), shiftLeft('c, 1), shiftLeft('d, 1), - shiftLeft('f, 1)), - Row(42.toLong, 42, 42.toShort, 42.toByte, null)) - - checkAnswer( - df.selectExpr( - "shiftLeft(a, 1)", "shiftLeft(b, 1)", "shiftLeft(b, 1)", "shiftLeft(d, 1)", - "shiftLeft(f, 1)"), - Row(42.toLong, 42, 42.toShort, 42.toByte, null)) - } - - test("shift right") { - val df = Seq[(Long, Integer, Short, Byte, Integer, Integer)]((42, 42, 42, 42, 42, null)) - .toDF("a", "b", "c", "d", "e", "f") - - checkAnswer( - df.select( - shiftRight('a, 1), shiftRight('b, 1), shiftRight('c, 1), shiftRight('d, 1), - shiftRight('f, 1)), - Row(21.toLong, 21, 21.toShort, 21.toByte, null)) - - checkAnswer( - df.selectExpr( - "shiftRight(a, 1)", "shiftRight(b, 1)", "shiftRight(c, 1)", "shiftRight(d, 1)", - "shiftRight(f, 1)"), - Row(21.toLong, 21, 21.toShort, 21.toByte, null)) - } - - test("shift right unsigned") { - val df = Seq[(Long, Integer, Short, Byte, Integer, Integer)]((-42, 42, 42, 42, 42, null)) - .toDF("a", "b", "c", "d", "e", "f") - - checkAnswer( - df.select( - shiftRightUnsigned('a, 1), shiftRightUnsigned('b, 1), shiftRightUnsigned('c, 1), - shiftRightUnsigned('d, 1), shiftRightUnsigned('f, 1)), - Row(9223372036854775787L, 21, 21.toShort, 21.toByte, null)) - - checkAnswer( - df.selectExpr( - "shiftRightUnsigned(a, 1)", "shiftRightUnsigned(b, 1)", "shiftRightUnsigned(c, 1)", - "shiftRightUnsigned(d, 1)", "shiftRightUnsigned(f, 1)"), - Row(9223372036854775787L, 21, 21.toShort, 21.toByte, null)) - } - - test("binary log") { - val df = Seq[(Integer, Integer)]((123, null)).toDF("a", "b") - checkAnswer( - df.select(org.apache.spark.sql.functions.log("a"), - org.apache.spark.sql.functions.log(2.0, "a"), - org.apache.spark.sql.functions.log("b")), - Row(math.log(123), math.log(123) / math.log(2), null)) - - checkAnswer( - df.selectExpr("log(a)", "log(2.0, a)", "log(b)"), - Row(math.log(123), math.log(123) / math.log(2), null)) - } - - test("abs") { - val input = - Seq[(java.lang.Double, java.lang.Double)]((null, null), (0.0, 0.0), (1.5, 1.5), (-2.5, 2.5)) - checkAnswer( - input.toDF("key", "value").select(abs($"key").alias("a")).sort("a"), - input.map(pair => Row(pair._2))) - - checkAnswer( - input.toDF("key", "value").selectExpr("abs(key) a").sort("a"), - input.map(pair => Row(pair._2))) - } - - test("log2") { - val df = Seq((1, 2)).toDF("a", "b") - checkAnswer( - df.select(log2("b") + log2("a")), - Row(1)) - - checkAnswer(sql("SELECT LOG2(8), LOG2(null)"), Row(3, null)) - } - - test("sqrt") { - val df = Seq((1, 4)).toDF("a", "b") - checkAnswer( - df.select(sqrt("a"), sqrt("b")), - Row(1.0, 2.0)) - - checkAnswer(sql("SELECT SQRT(4.0), SQRT(null)"), Row(2.0, null)) - checkAnswer(df.selectExpr("sqrt(a)", "sqrt(b)", "sqrt(null)"), Row(1.0, 2.0, null)) - } - - test("negative") { - checkAnswer( - sql("SELECT negative(1), negative(0), negative(-1)"), - Row(-1, 0, 1)) - } - - test("positive") { - val df = Seq((1, -1, "abc")).toDF("a", "b", "c") - checkAnswer(df.selectExpr("positive(a)"), Row(1)) - checkAnswer(df.selectExpr("positive(b)"), Row(-1)) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala new file mode 100644 index 000000000000..c2d08a06569b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala @@ -0,0 +1,449 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.nio.charset.StandardCharsets + +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.functions.{log => logarithm} +import org.apache.spark.sql.test.SharedSQLContext + +private object MathFunctionsTestData { + case class DoubleData(a: java.lang.Double, b: java.lang.Double) + case class NullDoubles(a: java.lang.Double) +} + +class MathFunctionsSuite extends QueryTest with SharedSQLContext { + import MathFunctionsTestData._ + import testImplicits._ + + private lazy val doubleData = (1 to 10).map(i => DoubleData(i * 0.2 - 1, i * -0.2 + 1)).toDF() + + private lazy val nnDoubleData = (1 to 10).map(i => DoubleData(i * 0.1, i * -0.1)).toDF() + + private lazy val nullDoubles = + Seq(NullDoubles(1.0), NullDoubles(2.0), NullDoubles(3.0), NullDoubles(null)).toDF() + + private def testOneToOneMathFunction[ + @specialized(Int, Long, Float, Double) T, + @specialized(Int, Long, Float, Double) U]( + c: Column => Column, + f: T => U): Unit = { + checkAnswer( + doubleData.select(c('a)), + (1 to 10).map(n => Row(f((n * 0.2 - 1).asInstanceOf[T]))) + ) + + checkAnswer( + doubleData.select(c('b)), + (1 to 10).map(n => Row(f((-n * 0.2 + 1).asInstanceOf[T]))) + ) + + checkAnswer( + doubleData.select(c(lit(null))), + (1 to 10).map(_ => Row(null)) + ) + } + + private def testOneToOneNonNegativeMathFunction(c: Column => Column, f: Double => Double): Unit = + { + checkAnswer( + nnDoubleData.select(c('a)), + (1 to 10).map(n => Row(f(n * 0.1))) + ) + + if (f(-1) === math.log1p(-1)) { + checkAnswer( + nnDoubleData.select(c('b)), + (1 to 9).map(n => Row(f(n * -0.1))) :+ Row(null) + ) + } + + checkAnswer( + nnDoubleData.select(c(lit(null))), + (1 to 10).map(_ => Row(null)) + ) + } + + private def testTwoToOneMathFunction( + c: (Column, Column) => Column, + d: (Column, Double) => Column, + f: (Double, Double) => Double): Unit = { + checkAnswer( + nnDoubleData.select(c('a, 'a)), + nnDoubleData.collect().toSeq.map(r => Row(f(r.getDouble(0), r.getDouble(0)))) + ) + + checkAnswer( + nnDoubleData.select(c('a, 'b)), + nnDoubleData.collect().toSeq.map(r => Row(f(r.getDouble(0), r.getDouble(1)))) + ) + + checkAnswer( + nnDoubleData.select(d('a, 2.0)), + nnDoubleData.collect().toSeq.map(r => Row(f(r.getDouble(0), 2.0))) + ) + + checkAnswer( + nnDoubleData.select(d('a, -0.5)), + nnDoubleData.collect().toSeq.map(r => Row(f(r.getDouble(0), -0.5))) + ) + + val nonNull = nullDoubles.collect().toSeq.filter(r => r.get(0) != null) + + checkAnswer( + nullDoubles.select(c('a, 'a)).orderBy('a.asc), + Row(null) +: nonNull.map(r => Row(f(r.getDouble(0), r.getDouble(0)))) + ) + } + + test("sin") { + testOneToOneMathFunction(sin, math.sin) + } + + test("asin") { + testOneToOneMathFunction(asin, math.asin) + } + + test("sinh") { + testOneToOneMathFunction(sinh, math.sinh) + } + + test("cos") { + testOneToOneMathFunction(cos, math.cos) + } + + test("acos") { + testOneToOneMathFunction(acos, math.acos) + } + + test("cosh") { + testOneToOneMathFunction(cosh, math.cosh) + } + + test("tan") { + testOneToOneMathFunction(tan, math.tan) + } + + test("atan") { + testOneToOneMathFunction(atan, math.atan) + } + + test("tanh") { + testOneToOneMathFunction(tanh, math.tanh) + } + + test("degrees") { + testOneToOneMathFunction(degrees, math.toDegrees) + checkAnswer( + sql("SELECT degrees(0), degrees(1), degrees(1.5)"), + Seq((1, 2)).toDF().select(degrees(lit(0)), degrees(lit(1)), degrees(lit(1.5))) + ) + } + + test("radians") { + testOneToOneMathFunction(radians, math.toRadians) + checkAnswer( + sql("SELECT radians(0), radians(1), radians(1.5)"), + Seq((1, 2)).toDF().select(radians(lit(0)), radians(lit(1)), radians(lit(1.5))) + ) + } + + test("cbrt") { + testOneToOneMathFunction(cbrt, math.cbrt) + } + + test("ceil and ceiling") { + testOneToOneMathFunction(ceil, (d: Double) => math.ceil(d).toLong) + checkAnswer( + sql("SELECT ceiling(0), ceiling(1), ceiling(1.5)"), + Row(0L, 1L, 2L)) + } + + test("conv") { + val df = Seq(("333", 10, 2)).toDF("num", "fromBase", "toBase") + checkAnswer(df.select(conv('num, 10, 16)), Row("14D")) + checkAnswer(df.select(conv(lit(100), 2, 16)), Row("4")) + checkAnswer(df.select(conv(lit(3122234455L), 10, 16)), Row("BA198457")) + checkAnswer(df.selectExpr("conv(num, fromBase, toBase)"), Row("101001101")) + checkAnswer(df.selectExpr("""conv("100", 2, 10)"""), Row("4")) + checkAnswer(df.selectExpr("""conv("-10", 16, -10)"""), Row("-16")) + checkAnswer( + df.selectExpr("""conv("9223372036854775807", 36, -16)"""), Row("-1")) // for overflow + } + + test("floor") { + testOneToOneMathFunction(floor, (d: Double) => math.floor(d).toLong) + } + + test("factorial") { + val df = (0 to 5).map(i => (i, i)).toDF("a", "b") + checkAnswer( + df.select(factorial('a)), + Seq(Row(1), Row(1), Row(2), Row(6), Row(24), Row(120)) + ) + checkAnswer( + df.selectExpr("factorial(a)"), + Seq(Row(1), Row(1), Row(2), Row(6), Row(24), Row(120)) + ) + } + + test("rint") { + testOneToOneMathFunction(rint, math.rint) + } + + test("round/bround") { + val df = Seq(5, 55, 555).map(Tuple1(_)).toDF("a") + checkAnswer( + df.select(round('a), round('a, -1), round('a, -2)), + Seq(Row(5, 10, 0), Row(55, 60, 100), Row(555, 560, 600)) + ) + checkAnswer( + df.select(bround('a), bround('a, -1), bround('a, -2)), + Seq(Row(5, 0, 0), Row(55, 60, 100), Row(555, 560, 600)) + ) + + val pi = "3.1415" + checkAnswer( + sql(s"SELECT round($pi, -3), round($pi, -2), round($pi, -1), " + + s"round($pi, 0), round($pi, 1), round($pi, 2), round($pi, 3)"), + Seq(Row(BigDecimal("0E3"), BigDecimal("0E2"), BigDecimal("0E1"), BigDecimal(3), + BigDecimal("3.1"), BigDecimal("3.14"), BigDecimal("3.142"))) + ) + checkAnswer( + sql(s"SELECT bround($pi, -3), bround($pi, -2), bround($pi, -1), " + + s"bround($pi, 0), bround($pi, 1), bround($pi, 2), bround($pi, 3)"), + Seq(Row(BigDecimal("0E3"), BigDecimal("0E2"), BigDecimal("0E1"), BigDecimal(3), + BigDecimal("3.1"), BigDecimal("3.14"), BigDecimal("3.142"))) + ) + + val bdPi: BigDecimal = BigDecimal(31415925L, 7) + checkAnswer( + sql(s"SELECT round($bdPi, 7), round($bdPi, 8), round($bdPi, 9), round($bdPi, 10), " + + s"round($bdPi, 100), round($bdPi, 6), round(null, 8)"), + Seq(Row(bdPi, bdPi, bdPi, bdPi, bdPi, BigDecimal("3.141593"), null)) + ) + + checkAnswer( + sql(s"SELECT bround($bdPi, 7), bround($bdPi, 8), bround($bdPi, 9), bround($bdPi, 10), " + + s"bround($bdPi, 100), bround($bdPi, 6), bround(null, 8)"), + Seq(Row(bdPi, bdPi, bdPi, bdPi, bdPi, BigDecimal("3.141592"), null)) + ) + } + + test("round/bround with data frame from a local Seq of Product") { + val df = spark.createDataFrame(Seq(Tuple1(BigDecimal("5.9")))).toDF("value") + checkAnswer( + df.withColumn("value_rounded", round('value)), + Seq(Row(BigDecimal("5.9"), BigDecimal("6"))) + ) + checkAnswer( + df.withColumn("value_brounded", bround('value)), + Seq(Row(BigDecimal("5.9"), BigDecimal("6"))) + ) + } + + test("exp") { + testOneToOneMathFunction(exp, math.exp) + } + + test("expm1") { + testOneToOneMathFunction(expm1, math.expm1) + } + + test("signum / sign") { + testOneToOneMathFunction[Double, Double](signum, math.signum) + + checkAnswer( + sql("SELECT sign(10), signum(-11)"), + Row(1, -1)) + } + + test("pow / power") { + testTwoToOneMathFunction(pow, pow, math.pow) + + checkAnswer( + sql("SELECT pow(1, 2), power(2, 1)"), + Seq((1, 2)).toDF().select(pow(lit(1), lit(2)), pow(lit(2), lit(1))) + ) + } + + test("hex") { + val data = Seq((28, -28, 100800200404L, "hello")).toDF("a", "b", "c", "d") + checkAnswer(data.select(hex('a)), Seq(Row("1C"))) + checkAnswer(data.select(hex('b)), Seq(Row("FFFFFFFFFFFFFFE4"))) + checkAnswer(data.select(hex('c)), Seq(Row("177828FED4"))) + checkAnswer(data.select(hex('d)), Seq(Row("68656C6C6F"))) + checkAnswer(data.selectExpr("hex(a)"), Seq(Row("1C"))) + checkAnswer(data.selectExpr("hex(b)"), Seq(Row("FFFFFFFFFFFFFFE4"))) + checkAnswer(data.selectExpr("hex(c)"), Seq(Row("177828FED4"))) + checkAnswer(data.selectExpr("hex(d)"), Seq(Row("68656C6C6F"))) + checkAnswer(data.selectExpr("hex(cast(d as binary))"), Seq(Row("68656C6C6F"))) + } + + test("unhex") { + val data = Seq(("1C", "737472696E67")).toDF("a", "b") + checkAnswer(data.select(unhex('a)), Row(Array[Byte](28.toByte))) + checkAnswer(data.select(unhex('b)), Row("string".getBytes(StandardCharsets.UTF_8))) + checkAnswer(data.selectExpr("unhex(a)"), Row(Array[Byte](28.toByte))) + checkAnswer(data.selectExpr("unhex(b)"), Row("string".getBytes(StandardCharsets.UTF_8))) + checkAnswer(data.selectExpr("""unhex("##")"""), Row(null)) + checkAnswer(data.selectExpr("""unhex("G123")"""), Row(null)) + } + + test("hypot") { + testTwoToOneMathFunction(hypot, hypot, math.hypot) + } + + test("atan2") { + testTwoToOneMathFunction(atan2, atan2, math.atan2) + } + + test("log / ln") { + testOneToOneNonNegativeMathFunction(org.apache.spark.sql.functions.log, math.log) + checkAnswer( + sql("SELECT ln(0), ln(1), ln(1.5)"), + Seq((1, 2)).toDF().select(logarithm(lit(0)), logarithm(lit(1)), logarithm(lit(1.5))) + ) + } + + test("log10") { + testOneToOneNonNegativeMathFunction(log10, math.log10) + } + + test("log1p") { + testOneToOneNonNegativeMathFunction(log1p, math.log1p) + } + + test("shift left") { + val df = Seq[(Long, Integer, Short, Byte, Integer, Integer)]((21, 21, 21, 21, 21, null)) + .toDF("a", "b", "c", "d", "e", "f") + + checkAnswer( + df.select( + shiftLeft('a, 1), shiftLeft('b, 1), shiftLeft('c, 1), shiftLeft('d, 1), + shiftLeft('f, 1)), + Row(42.toLong, 42, 42.toShort, 42.toByte, null)) + + checkAnswer( + df.selectExpr( + "shiftLeft(a, 1)", "shiftLeft(b, 1)", "shiftLeft(b, 1)", "shiftLeft(d, 1)", + "shiftLeft(f, 1)"), + Row(42.toLong, 42, 42.toShort, 42.toByte, null)) + } + + test("shift right") { + val df = Seq[(Long, Integer, Short, Byte, Integer, Integer)]((42, 42, 42, 42, 42, null)) + .toDF("a", "b", "c", "d", "e", "f") + + checkAnswer( + df.select( + shiftRight('a, 1), shiftRight('b, 1), shiftRight('c, 1), shiftRight('d, 1), + shiftRight('f, 1)), + Row(21.toLong, 21, 21.toShort, 21.toByte, null)) + + checkAnswer( + df.selectExpr( + "shiftRight(a, 1)", "shiftRight(b, 1)", "shiftRight(c, 1)", "shiftRight(d, 1)", + "shiftRight(f, 1)"), + Row(21.toLong, 21, 21.toShort, 21.toByte, null)) + } + + test("shift right unsigned") { + val df = Seq[(Long, Integer, Short, Byte, Integer, Integer)]((-42, 42, 42, 42, 42, null)) + .toDF("a", "b", "c", "d", "e", "f") + + checkAnswer( + df.select( + shiftRightUnsigned('a, 1), shiftRightUnsigned('b, 1), shiftRightUnsigned('c, 1), + shiftRightUnsigned('d, 1), shiftRightUnsigned('f, 1)), + Row(9223372036854775787L, 21, 21.toShort, 21.toByte, null)) + + checkAnswer( + df.selectExpr( + "shiftRightUnsigned(a, 1)", "shiftRightUnsigned(b, 1)", "shiftRightUnsigned(c, 1)", + "shiftRightUnsigned(d, 1)", "shiftRightUnsigned(f, 1)"), + Row(9223372036854775787L, 21, 21.toShort, 21.toByte, null)) + } + + test("binary log") { + val df = Seq[(Integer, Integer)]((123, null)).toDF("a", "b") + checkAnswer( + df.select(org.apache.spark.sql.functions.log("a"), + org.apache.spark.sql.functions.log(2.0, "a"), + org.apache.spark.sql.functions.log("b")), + Row(math.log(123), math.log(123) / math.log(2), null)) + + checkAnswer( + df.selectExpr("log(a)", "log(2.0, a)", "log(b)"), + Row(math.log(123), math.log(123) / math.log(2), null)) + } + + test("abs") { + val input = + Seq[(java.lang.Double, java.lang.Double)]((null, null), (0.0, 0.0), (1.5, 1.5), (-2.5, 2.5)) + checkAnswer( + input.toDF("key", "value").select(abs($"key").alias("a")).sort("a"), + input.map(pair => Row(pair._2))) + + checkAnswer( + input.toDF("key", "value").selectExpr("abs(key) a").sort("a"), + input.map(pair => Row(pair._2))) + + checkAnswer( + sql("select abs(0), abs(-1), abs(123), abs(-9223372036854775807), abs(9223372036854775807)"), + Row(0, 1, 123, 9223372036854775807L, 9223372036854775807L) + ) + + checkAnswer( + sql("select abs(0.0), abs(-3.14159265), abs(3.14159265)"), + Row(BigDecimal("0.0"), BigDecimal("3.14159265"), BigDecimal("3.14159265")) + ) + } + + test("log2") { + val df = Seq((1, 2)).toDF("a", "b") + checkAnswer( + df.select(log2("b") + log2("a")), + Row(1)) + + checkAnswer(sql("SELECT LOG2(8), LOG2(null)"), Row(3, null)) + } + + test("sqrt") { + val df = Seq((1, 4)).toDF("a", "b") + checkAnswer( + df.select(sqrt("a"), sqrt("b")), + Row(1.0, 2.0)) + + checkAnswer(sql("SELECT SQRT(4.0), SQRT(null)"), Row(2.0, null)) + checkAnswer(df.selectExpr("sqrt(a)", "sqrt(b)", "sqrt(null)"), Row(1.0, 2.0, null)) + } + + test("negative") { + checkAnswer( + sql("SELECT negative(1), negative(0), negative(-1)"), + Row(-1, 0, 1)) + } + + test("positive") { + val df = Seq((1, -1, "abc")).toDF("a", "b", "c") + checkAnswer(df.selectExpr("positive(a)"), Row(1)) + checkAnswer(df.selectExpr("positive(b)"), Row(-1)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala new file mode 100644 index 000000000000..98aa447fc056 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/MetadataCacheSuite.scala @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.io.File + +import org.apache.spark.SparkException +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +/** + * Test suite to handle metadata cache related. + */ +class MetadataCacheSuite extends QueryTest with SharedSQLContext { + + /** Removes one data file in the given directory. */ + private def deleteOneFileInDirectory(dir: File): Unit = { + assert(dir.isDirectory) + val oneFile = dir.listFiles().find { file => + !file.getName.startsWith("_") && !file.getName.startsWith(".") + } + assert(oneFile.isDefined) + oneFile.foreach(_.delete()) + } + + test("SPARK-16336 Suggest doing table refresh when encountering FileNotFoundException") { + withTempPath { (location: File) => + // Create a Parquet directory + spark.range(start = 0, end = 100, step = 1, numPartitions = 3) + .write.parquet(location.getAbsolutePath) + + // Read the directory in + val df = spark.read.parquet(location.getAbsolutePath) + assert(df.count() == 100) + + // Delete a file + deleteOneFileInDirectory(location) + + // Read it again and now we should see a FileNotFoundException + val e = intercept[SparkException] { + df.count() + } + assert(e.getMessage.contains("FileNotFoundException")) + assert(e.getMessage.contains("REFRESH")) + } + } + + test("SPARK-16337 temporary view refresh") { + withTempView("view_refresh") { withTempPath { (location: File) => + // Create a Parquet directory + spark.range(start = 0, end = 100, step = 1, numPartitions = 3) + .write.parquet(location.getAbsolutePath) + + // Read the directory in + spark.read.parquet(location.getAbsolutePath).createOrReplaceTempView("view_refresh") + assert(sql("select count(*) from view_refresh").first().getLong(0) == 100) + + // Delete a file + deleteOneFileInDirectory(location) + + // Read it again and now we should see a FileNotFoundException + val e = intercept[SparkException] { + sql("select count(*) from view_refresh").first() + } + assert(e.getMessage.contains("FileNotFoundException")) + assert(e.getMessage.contains("REFRESH")) + + // Refresh and we should be able to read it again. + spark.catalog.refreshTable("view_refresh") + val newCount = sql("select count(*) from view_refresh").first().getLong(0) + assert(newCount > 0 && newCount < 100) + }} + } + + test("case sensitivity support in temporary view refresh") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + withTempView("view_refresh") { + withTempPath { (location: File) => + // Create a Parquet directory + spark.range(start = 0, end = 100, step = 1, numPartitions = 3) + .write.parquet(location.getAbsolutePath) + + // Read the directory in + spark.read.parquet(location.getAbsolutePath).createOrReplaceTempView("view_refresh") + + // Delete a file + deleteOneFileInDirectory(location) + intercept[SparkException](sql("select count(*) from view_refresh").first()) + + // Refresh and we should be able to read it again. + spark.catalog.refreshTable("vIeW_reFrEsH") + val newCount = sql("select count(*) from view_refresh").first().getLong(0) + assert(newCount > 0 && newCount < 100) + } + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala new file mode 100644 index 000000000000..a5b08f717767 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/MiscFunctionsSuite.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.SharedSQLContext + +class MiscFunctionsSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("reflect and java_method") { + val df = Seq((1, "one")).toDF("a", "b") + val className = ReflectClass.getClass.getName.stripSuffix("$") + checkAnswer( + df.selectExpr( + s"reflect('$className', 'method1', a, b)", + s"java_method('$className', 'method1', a, b)"), + Row("m1one", "m1one")) + } +} + +object ReflectClass { + def method1(v1: Int, v2: String): String = "m" + v1 + v2 +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala deleted file mode 100644 index 0e8fcb6a858b..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/MultiSQLContextsSuite.scala +++ /dev/null @@ -1,99 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ - -package org.apache.spark.sql - -import org.apache.spark._ -import org.scalatest.BeforeAndAfterAll - -class MultiSQLContextsSuite extends SparkFunSuite with BeforeAndAfterAll { - - private var originalActiveSQLContext: Option[SQLContext] = _ - private var originalInstantiatedSQLContext: Option[SQLContext] = _ - private var sparkConf: SparkConf = _ - - override protected def beforeAll(): Unit = { - originalActiveSQLContext = SQLContext.getActiveContextOption() - originalInstantiatedSQLContext = SQLContext.getInstantiatedContextOption() - - SQLContext.clearActive() - originalInstantiatedSQLContext.foreach(ctx => SQLContext.clearInstantiatedContext(ctx)) - sparkConf = - new SparkConf(false) - .setMaster("local[*]") - .setAppName("test") - .set("spark.ui.enabled", "false") - .set("spark.driver.allowMultipleContexts", "true") - } - - override protected def afterAll(): Unit = { - // Set these states back. - originalActiveSQLContext.foreach(ctx => SQLContext.setActive(ctx)) - originalInstantiatedSQLContext.foreach(ctx => SQLContext.setInstantiatedContext(ctx)) - } - - def testNewSession(rootSQLContext: SQLContext): Unit = { - // Make sure we can successfully create new Session. - rootSQLContext.newSession() - - // Reset the state. It is always safe to clear the active context. - SQLContext.clearActive() - } - - def testCreatingNewSQLContext(allowsMultipleContexts: Boolean): Unit = { - val conf = - sparkConf - .clone - .set(SQLConf.ALLOW_MULTIPLE_CONTEXTS.key, allowsMultipleContexts.toString) - val sparkContext = new SparkContext(conf) - - try { - if (allowsMultipleContexts) { - new SQLContext(sparkContext) - SQLContext.clearActive() - } else { - // If allowsMultipleContexts is false, make sure we can get the error. - val message = intercept[SparkException] { - new SQLContext(sparkContext) - }.getMessage - assert(message.contains("Only one SQLContext/HiveContext may be running")) - } - } finally { - sparkContext.stop() - } - } - - test("test the flag to disallow creating multiple root SQLContext") { - Seq(false, true).foreach { allowMultipleSQLContexts => - val conf = - sparkConf - .clone - .set(SQLConf.ALLOW_MULTIPLE_CONTEXTS.key, allowMultipleSQLContexts.toString) - val sc = new SparkContext(conf) - try { - val rootSQLContext = new SQLContext(sc) - testNewSession(rootSQLContext) - testNewSession(rootSQLContext) - testCreatingNewSQLContext(allowMultipleSQLContexts) - - SQLContext.clearInstantiatedContext(rootSQLContext) - } finally { - sc.stop() - } - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala new file mode 100644 index 000000000000..623a1b6f854c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.util.concurrent.TimeUnit + +import scala.concurrent.duration._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.streaming.{ProcessingTime, Trigger} + +class ProcessingTimeSuite extends SparkFunSuite { + + test("create") { + def getIntervalMs(trigger: Trigger): Long = trigger.asInstanceOf[ProcessingTime].intervalMs + + assert(getIntervalMs(Trigger.ProcessingTime(10.seconds)) === 10 * 1000) + assert(getIntervalMs(Trigger.ProcessingTime(10, TimeUnit.SECONDS)) === 10 * 1000) + assert(getIntervalMs(Trigger.ProcessingTime("1 minute")) === 60 * 1000) + assert(getIntervalMs(Trigger.ProcessingTime("interval 1 minute")) === 60 * 1000) + + intercept[IllegalArgumentException] { Trigger.ProcessingTime(null: String) } + intercept[IllegalArgumentException] { Trigger.ProcessingTime("") } + intercept[IllegalArgumentException] { Trigger.ProcessingTime("invalid") } + intercept[IllegalArgumentException] { Trigger.ProcessingTime("1 month") } + intercept[IllegalArgumentException] { Trigger.ProcessingTime("1 year") } + } +} 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 3c174efe73ff..f9808834df4a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -17,18 +17,28 @@ package org.apache.spark.sql -import java.util.{Locale, TimeZone} +import java.util.{ArrayDeque, Locale, TimeZone} import scala.collection.JavaConverters._ +import scala.util.control.NonFatal +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.ImperativeAggregate import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.columnar.InMemoryRelation -import org.apache.spark.sql.catalyst.encoders.Encoder +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression +import org.apache.spark.sql.execution.columnar.InMemoryRelation +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.streaming.MemoryPlan +import org.apache.spark.sql.types.{Metadata, ObjectType} + abstract class QueryTest extends PlanTest { - protected def sqlContext: SQLContext + protected def spark: SparkSession // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) @@ -36,90 +46,129 @@ abstract class QueryTest extends PlanTest { Locale.setDefault(Locale.US) /** - * Runs the plan and makes sure the answer contains all of the keywords, or the - * none of keywords are listed in the answer - * @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 + * Runs the plan and makes sure the answer contains all of the keywords. */ - def checkExistence(df: DataFrame, exists: Boolean, keywords: String*) { + def checkKeywordsExist(df: DataFrame, keywords: String*): Unit = { val outputs = df.collect().map(_.mkString).mkString for (key <- keywords) { - if (exists) { - assert(outputs.contains(key), s"Failed for $df ($key doesn't exist in result)") - } else { - assert(!outputs.contains(key), s"Failed for $df ($key existed in the result)") - } + assert(outputs.contains(key), s"Failed for $df ($key doesn't exist in result)") + } + } + + /** + * Runs the plan and makes sure the answer does NOT contain any of the keywords. + */ + def checkKeywordsNotExist(df: DataFrame, keywords: String*): Unit = { + val outputs = df.collect().map(_.mkString).mkString + for (key <- keywords) { + assert(!outputs.contains(key), s"Failed for $df ($key existed in the result)") } } /** * Evaluates a dataset to make sure that the result of calling collect matches the given * expected answer. - * - Special handling is done based on whether the query plan should be expected to return - * the results in sorted order. - * - This function also checks to make sure that the schema for serializing the expected answer - * matches that produced by the dataset (i.e. does manual construction of object match - * the constructed encoder for cases like joins, etc). Note that this means that it will fail - * for cases where reordering is done on fields. For such tests, user `checkDecoding` instead - * which performs a subset of the checks done by this function. */ - protected def checkAnswer[T : Encoder]( + protected def checkDataset[T]( ds: => Dataset[T], expectedAnswer: T*): Unit = { - checkAnswer( - ds.toDF(), - sqlContext.createDataset(expectedAnswer).toDF().collect().toSeq) + val result = getResult(ds) - checkDecoding(ds, expectedAnswer: _*) + if (!compare(result.toSeq, expectedAnswer)) { + fail( + s""" + |Decoded objects do not match expected objects: + |expected: $expectedAnswer + |actual: ${result.toSeq} + |${ds.exprEnc.deserializer.treeString} + """.stripMargin) + } } - protected def checkDecoding[T]( + /** + * Evaluates a dataset to make sure that the result of calling collect matches the given + * expected answer, after sort. + */ + protected def checkDatasetUnorderly[T : Ordering]( ds: => Dataset[T], expectedAnswer: T*): Unit = { - val decoded = try ds.collect().toSet catch { + val result = getResult(ds) + + if (!compare(result.toSeq.sorted, expectedAnswer.sorted)) { + fail( + s""" + |Decoded objects do not match expected objects: + |expected: $expectedAnswer + |actual: ${result.toSeq} + |${ds.exprEnc.deserializer.treeString} + """.stripMargin) + } + } + + private def getResult[T](ds: => Dataset[T]): Array[T] = { + val analyzedDS = try ds catch { + case ae: AnalysisException => + if (ae.plan.isDefined) { + fail( + s""" + |Failed to analyze query: $ae + |${ae.plan.get} + | + |${stackTraceToString(ae)} + """.stripMargin) + } else { + throw ae + } + } + assertEmptyMissingInput(analyzedDS) + + try ds.collect() catch { case e: Exception => fail( s""" |Exception collecting dataset as objects - |${ds.encoder} - |${ds.encoder.constructExpression.treeString} + |${ds.exprEnc} + |${ds.exprEnc.deserializer.treeString} |${ds.queryExecution} """.stripMargin, e) } + } - if (decoded != expectedAnswer.toSet) { - fail( - s"""Decoded objects do not match expected objects: - |Expected: ${expectedAnswer.toSet.toSeq.map((a: Any) => a.toString).sorted} - |Actual ${decoded.toSet.toSeq.map((a: Any) => a.toString).sorted} - |${ds.encoder.constructExpression.treeString} - """.stripMargin) - } + private def compare(obj1: Any, obj2: Any): Boolean = (obj1, obj2) match { + case (null, null) => true + case (null, _) => false + case (_, null) => false + case (a: Array[_], b: Array[_]) => + a.length == b.length && a.zip(b).forall { case (l, r) => compare(l, r)} + case (a: Iterable[_], b: Iterable[_]) => + a.size == b.size && a.zip(b).forall { case (l, r) => compare(l, r)} + case (a, b) => a == b } /** * Runs the plan and makes sure the answer matches the expected result. + * * @param df the [[DataFrame]] to be executed * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. */ protected def checkAnswer(df: => DataFrame, expectedAnswer: Seq[Row]): Unit = { val analyzedDF = try df catch { case ae: AnalysisException => - val currentValue = sqlContext.conf.dataFrameEagerAnalysis - sqlContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, false) - val partiallyAnalzyedPlan = df.queryExecution.analyzed - sqlContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, currentValue) - fail( - s""" - |Failed to analyze query: $ae - |$partiallyAnalzyedPlan - | - |${stackTraceToString(ae)} - |""".stripMargin) + if (ae.plan.isDefined) { + fail( + s""" + |Failed to analyze query: $ae + |${ae.plan.get} + | + |${stackTraceToString(ae)} + |""".stripMargin) + } else { + throw ae + } } + assertEmptyMissingInput(analyzedDF) + QueryTest.checkAnswer(analyzedDF, expectedAnswer) match { case Some(errorMessage) => fail(errorMessage) case None => @@ -136,6 +185,7 @@ abstract class QueryTest extends PlanTest { /** * Runs the plan and makes sure the answer is within absTol of the expected result. + * * @param dataFrame the [[DataFrame]] to be executed * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. * @param absTol the absolute tolerance between actual and expected answers. @@ -161,9 +211,9 @@ abstract class QueryTest extends PlanTest { } /** - * Asserts that a given [[DataFrame]] will be executed using the given number of cached results. + * Asserts that a given [[Dataset]] will be executed using the given number of cached results. */ - def assertCached(query: DataFrame, numCachedTables: Int = 1): Unit = { + def assertCached(query: Dataset[_], numCachedTables: Int = 1): Unit = { val planWithCaching = query.queryExecution.withCachedData val cachedData = planWithCaching collect { case cached: InMemoryRelation => cached @@ -174,6 +224,18 @@ abstract class QueryTest extends PlanTest { s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + planWithCaching) } + + /** + * Asserts that a given [[Dataset]] does not have missing inputs in all the analyzed plans. + */ + def assertEmptyMissingInput(query: Dataset[_]): Unit = { + assert(query.queryExecution.analyzed.missingInput.isEmpty, + s"The analyzed logical plan has missing inputs:\n${query.queryExecution.analyzed}") + assert(query.queryExecution.optimizedPlan.missingInput.isEmpty, + s"The optimized logical plan has missing inputs:\n${query.queryExecution.optimizedPlan}") + assert(query.queryExecution.executedPlan.missingInput.isEmpty, + s"The physical plan has missing inputs:\n${query.queryExecution.executedPlan}") + } } object QueryTest { @@ -182,33 +244,20 @@ 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 df the [[DataFrame]] to be executed * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. + * @param checkToRDD whether to verify deserialization to an RDD. This runs the query twice. */ - def checkAnswer(df: DataFrame, expectedAnswer: Seq[Row]): Option[String] = { + def checkAnswer( + df: DataFrame, + expectedAnswer: Seq[Row], + checkToRDD: Boolean = true): Option[String] = { val isSorted = df.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty - - // We need to call prepareRow recursively to handle schemas with struct types. - def prepareRow(row: Row): Row = { - Row.fromSeq(row.toSeq.map { - case null => null - case d: java.math.BigDecimal => BigDecimal(d) - // Convert array to Seq for easy equality check. - case b: Array[_] => b.toSeq - case r: Row => prepareRow(r) - case o => o - }) + if (checkToRDD) { + df.rdd.count() // Also attempt to deserialize as an RDD [SPARK-15791] } - def prepareAnswer(answer: Seq[Row]): Seq[Row] = { - // Converts data to types that we can do equality comparison using Scala collections. - // For BigDecimal type, the Scala type has a better definition of equality test (similar to - // Java's java.math.BigDecimal.compareTo). - // For binary arrays, we convert it to Seq to avoid of calling java.util.Arrays.equals for - // equality test. - val converted: Seq[Row] = answer.map(prepareRow) - if (!isSorted) converted.sortBy(_.toString()) else converted - } val sparkAnswer = try df.collect().toSeq catch { case e: Exception => val errorMessage = @@ -222,26 +271,74 @@ object QueryTest { return Some(errorMessage) } - if (prepareAnswer(expectedAnswer) != prepareAnswer(sparkAnswer)) { - val errorMessage = + sameRows(expectedAnswer, sparkAnswer, isSorted).map { results => s""" |Results do not match for query: + |Timezone: ${TimeZone.getDefault} + |Timezone Env: ${sys.env.getOrElse("TZ", "")} + | |${df.queryExecution} |== Results == - |${sideBySide( - s"== Correct Answer - ${expectedAnswer.size} ==" +: - prepareAnswer(expectedAnswer).map(_.toString()), - s"== Spark Answer - ${sparkAnswer.size} ==" +: - prepareAnswer(sparkAnswer).map(_.toString())).mkString("\n")} - """.stripMargin - return Some(errorMessage) + |$results + """.stripMargin } + } + - return None + def prepareAnswer(answer: Seq[Row], isSorted: Boolean): Seq[Row] = { + // Converts data to types that we can do equality comparison using Scala collections. + // For BigDecimal type, the Scala type has a better definition of equality test (similar to + // Java's java.math.BigDecimal.compareTo). + // For binary arrays, we convert it to Seq to avoid of calling java.util.Arrays.equals for + // equality test. + val converted: Seq[Row] = answer.map(prepareRow) + if (!isSorted) converted.sortBy(_.toString()) else converted + } + + // We need to call prepareRow recursively to handle schemas with struct types. + def prepareRow(row: Row): Row = { + Row.fromSeq(row.toSeq.map { + case null => null + case d: java.math.BigDecimal => BigDecimal(d) + // Convert array to Seq for easy equality check. + case b: Array[_] => b.toSeq + case r: Row => prepareRow(r) + case o => o + }) + } + + def sameRows( + expectedAnswer: Seq[Row], + sparkAnswer: Seq[Row], + isSorted: Boolean = false): Option[String] = { + if (prepareAnswer(expectedAnswer, isSorted) != prepareAnswer(sparkAnswer, isSorted)) { + val getRowType: Option[Row] => String = row => + row.map(row => + if (row.schema == null) { + "struct<>" + } else { + s"${row.schema.catalogString}" + }).getOrElse("struct<>") + + val errorMessage = + s""" + |== Results == + |${sideBySide( + s"== Correct Answer - ${expectedAnswer.size} ==" +: + getRowType(expectedAnswer.headOption) +: + prepareAnswer(expectedAnswer, isSorted).map(_.toString()), + s"== Spark Answer - ${sparkAnswer.size} ==" +: + getRowType(sparkAnswer.headOption) +: + prepareAnswer(sparkAnswer, isSorted).map(_.toString())).mkString("\n")} + """.stripMargin + return Some(errorMessage) + } + None } /** * Runs the plan and makes sure the answer is within absTol of the expected result. + * * @param actualAnswer the actual result in a [[Row]]. * @param expectedAnswer the expected result in a[[Row]]. * @param absTol the absolute tolerance between actual and expected answers. @@ -269,3 +366,11 @@ object QueryTest { } } } + +class QueryTestSuite extends QueryTest with test.SharedSQLContext { + test("SPARK-16940: checkAnswer should raise TestFailedException for wrong results") { + intercept[org.scalatest.exceptions.TestFailedException] { + checkAnswer(sql("SELECT 1"), Row(2) :: Nil) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 3ba14d7602a6..57b5f5e4ab99 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -1,25 +1,24 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.SparkFunSuite -import org.apache.spark.sql.execution.SparkSqlSerializer -import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, SpecificMutableRow} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, SpecificInternalRow} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -28,7 +27,7 @@ class RowSuite extends SparkFunSuite with SharedSQLContext { import testImplicits._ test("create row") { - val expected = new GenericMutableRow(4) + val expected = new GenericInternalRow(4) expected.setInt(0, 2147483647) expected.update(1, UTF8String.fromString("this is a string")) expected.setBoolean(2, false) @@ -50,20 +49,11 @@ class RowSuite extends SparkFunSuite with SharedSQLContext { } test("SpecificMutableRow.update with null") { - val row = new SpecificMutableRow(Seq(IntegerType)) + val row = new SpecificInternalRow(Seq(IntegerType)) row(0) = null assert(row.isNullAt(0)) } - test("serialize w/ kryo") { - val row = Seq((1, Seq(1), Map(1 -> 1), BigDecimal(1))).toDF().first() - val serializer = new SparkSqlSerializer(sparkContext.getConf) - val instance = serializer.newInstance() - val ser = instance.serialize(row) - val de = instance.deserialize(ser).asInstanceOf[Row] - assert(de === row) - } - test("get values by field name on Row created via .toDF") { val row = Seq((1, Seq(1))).toDF("a", "b").first() assert(row.getAs[Int]("a") === 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala new file mode 100644 index 000000000000..cfe2e9f2dbc4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.SparkFunSuite + +class RuntimeConfigSuite extends SparkFunSuite { + + private def newConf(): RuntimeConfig = new RuntimeConfig + + test("set and get") { + val conf = newConf() + conf.set("k1", "v1") + conf.set("k2", 2) + conf.set("k3", value = false) + + assert(conf.get("k1") == "v1") + assert(conf.get("k2") == "2") + assert(conf.get("k3") == "false") + + intercept[NoSuchElementException] { + conf.get("notset") + } + } + + test("getOption") { + val conf = newConf() + conf.set("k1", "v1") + assert(conf.getOption("k1") == Some("v1")) + assert(conf.getOption("notset") == None) + } + + test("unset") { + val conf = newConf() + conf.set("k1", "v1") + assert(conf.get("k1") == "v1") + conf.unset("k1") + intercept[NoSuchElementException] { + conf.get("k1") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfEntrySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfEntrySuite.scala deleted file mode 100644 index 2e33777f14ad..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfEntrySuite.scala +++ /dev/null @@ -1,150 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.SQLConf._ - -class SQLConfEntrySuite extends SparkFunSuite { - - val conf = new SQLConf - - test("intConf") { - val key = "spark.sql.SQLConfEntrySuite.int" - val confEntry = SQLConfEntry.intConf(key) - assert(conf.getConf(confEntry, 5) === 5) - - conf.setConf(confEntry, 10) - assert(conf.getConf(confEntry, 5) === 10) - - conf.setConfString(key, "20") - assert(conf.getConfString(key, "5") === "20") - assert(conf.getConfString(key) === "20") - assert(conf.getConf(confEntry, 5) === 20) - - val e = intercept[IllegalArgumentException] { - conf.setConfString(key, "abc") - } - assert(e.getMessage === s"$key should be int, but was abc") - } - - test("longConf") { - val key = "spark.sql.SQLConfEntrySuite.long" - val confEntry = SQLConfEntry.longConf(key) - assert(conf.getConf(confEntry, 5L) === 5L) - - conf.setConf(confEntry, 10L) - assert(conf.getConf(confEntry, 5L) === 10L) - - conf.setConfString(key, "20") - assert(conf.getConfString(key, "5") === "20") - assert(conf.getConfString(key) === "20") - assert(conf.getConf(confEntry, 5L) === 20L) - - val e = intercept[IllegalArgumentException] { - conf.setConfString(key, "abc") - } - assert(e.getMessage === s"$key should be long, but was abc") - } - - test("booleanConf") { - val key = "spark.sql.SQLConfEntrySuite.boolean" - val confEntry = SQLConfEntry.booleanConf(key) - assert(conf.getConf(confEntry, false) === false) - - conf.setConf(confEntry, true) - assert(conf.getConf(confEntry, false) === true) - - conf.setConfString(key, "true") - assert(conf.getConfString(key, "false") === "true") - assert(conf.getConfString(key) === "true") - assert(conf.getConf(confEntry, false) === true) - - val e = intercept[IllegalArgumentException] { - conf.setConfString(key, "abc") - } - assert(e.getMessage === s"$key should be boolean, but was abc") - } - - test("doubleConf") { - val key = "spark.sql.SQLConfEntrySuite.double" - val confEntry = SQLConfEntry.doubleConf(key) - assert(conf.getConf(confEntry, 5.0) === 5.0) - - conf.setConf(confEntry, 10.0) - assert(conf.getConf(confEntry, 5.0) === 10.0) - - conf.setConfString(key, "20.0") - assert(conf.getConfString(key, "5.0") === "20.0") - assert(conf.getConfString(key) === "20.0") - assert(conf.getConf(confEntry, 5.0) === 20.0) - - val e = intercept[IllegalArgumentException] { - conf.setConfString(key, "abc") - } - assert(e.getMessage === s"$key should be double, but was abc") - } - - test("stringConf") { - val key = "spark.sql.SQLConfEntrySuite.string" - val confEntry = SQLConfEntry.stringConf(key) - assert(conf.getConf(confEntry, "abc") === "abc") - - conf.setConf(confEntry, "abcd") - assert(conf.getConf(confEntry, "abc") === "abcd") - - conf.setConfString(key, "abcde") - assert(conf.getConfString(key, "abc") === "abcde") - assert(conf.getConfString(key) === "abcde") - assert(conf.getConf(confEntry, "abc") === "abcde") - } - - test("enumConf") { - val key = "spark.sql.SQLConfEntrySuite.enum" - val confEntry = SQLConfEntry.enumConf(key, v => v, Set("a", "b", "c"), defaultValue = Some("a")) - assert(conf.getConf(confEntry) === "a") - - conf.setConf(confEntry, "b") - assert(conf.getConf(confEntry) === "b") - - conf.setConfString(key, "c") - assert(conf.getConfString(key, "a") === "c") - assert(conf.getConfString(key) === "c") - assert(conf.getConf(confEntry) === "c") - - val e = intercept[IllegalArgumentException] { - conf.setConfString(key, "d") - } - assert(e.getMessage === s"The value of $key should be one of a, b, c, but was d") - } - - test("stringSeqConf") { - val key = "spark.sql.SQLConfEntrySuite.stringSeq" - val confEntry = SQLConfEntry.stringSeqConf("spark.sql.SQLConfEntrySuite.stringSeq", - defaultValue = Some(Nil)) - assert(conf.getConf(confEntry, Seq("a", "b", "c")) === Seq("a", "b", "c")) - - conf.setConf(confEntry, Seq("a", "b", "c", "d")) - assert(conf.getConf(confEntry, Seq("a", "b", "c")) === Seq("a", "b", "c", "d")) - - conf.setConfString(key, "a,b,c,d,e") - assert(conf.getConfString(key, "a,b,c") === "a,b,c,d,e") - assert(conf.getConfString(key) === "a,b,c,d,e") - assert(conf.getConf(confEntry, Seq("a", "b", "c")) === Seq("a", "b", "c", "d", "e")) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala deleted file mode 100644 index 3d2bd236ceea..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala +++ /dev/null @@ -1,96 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ - -package org.apache.spark.sql - -import org.apache.spark.sql.test.{TestSQLContext, SharedSQLContext} - - -class SQLConfSuite extends QueryTest with SharedSQLContext { - private val testKey = "test.key.0" - private val testVal = "test.val.0" - - test("propagate from spark conf") { - // We create a new context here to avoid order dependence with other tests that might call - // clear(). - val newContext = new SQLContext(sparkContext) - assert(newContext.getConf("spark.sql.testkey", "false") === "true") - } - - test("programmatic ways of basic setting and getting") { - // Set a conf first. - sqlContext.setConf(testKey, testVal) - // Clear the conf. - sqlContext.conf.clear() - // After clear, only overrideConfs used by unit test should be in the SQLConf. - assert(sqlContext.getAllConfs === TestSQLContext.overrideConfs) - - sqlContext.setConf(testKey, testVal) - assert(sqlContext.getConf(testKey) === testVal) - assert(sqlContext.getConf(testKey, testVal + "_") === testVal) - assert(sqlContext.getAllConfs.contains(testKey)) - - // Tests SQLConf as accessed from a SQLContext is mutable after - // the latter is initialized, unlike SparkConf inside a SparkContext. - assert(sqlContext.getConf(testKey) === testVal) - assert(sqlContext.getConf(testKey, testVal + "_") === testVal) - assert(sqlContext.getAllConfs.contains(testKey)) - - sqlContext.conf.clear() - } - - test("parse SQL set commands") { - sqlContext.conf.clear() - sql(s"set $testKey=$testVal") - assert(sqlContext.getConf(testKey, testVal + "_") === testVal) - assert(sqlContext.getConf(testKey, testVal + "_") === testVal) - - sql("set some.property=20") - assert(sqlContext.getConf("some.property", "0") === "20") - sql("set some.property = 40") - assert(sqlContext.getConf("some.property", "0") === "40") - - val key = "spark.sql.key" - val vs = "val0,val_1,val2.3,my_table" - sql(s"set $key=$vs") - assert(sqlContext.getConf(key, "0") === vs) - - sql(s"set $key=") - assert(sqlContext.getConf(key, "0") === "") - - sqlContext.conf.clear() - } - - test("deprecated property") { - sqlContext.conf.clear() - val original = sqlContext.conf.numShufflePartitions - try{ - sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") - assert(sqlContext.conf.numShufflePartitions === 10) - } finally { - sql(s"set ${SQLConf.SHUFFLE_PARTITIONS}=$original") - } - } - - test("invalid conf value") { - sqlContext.conf.clear() - val e = intercept[IllegalArgumentException] { - sql(s"set ${SQLConf.CASE_SENSITIVE.key}=10") - } - assert(e.getMessage === s"${SQLConf.CASE_SENSITIVE.key} should be boolean, but was 10") - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index 1994dacfc4df..a1799829932b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -1,25 +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. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.{SharedSparkContext, SparkFunSuite} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{BooleanType, StringType, StructField, StructType} -class SQLContextSuite extends SparkFunSuite with SharedSparkContext{ +@deprecated("This suite is deprecated to silent compiler deprecation warnings", "2.0.0") +class SQLContextSuite extends SparkFunSuite with SharedSparkContext { + + object DummyRule extends Rule[LogicalPlan] { + def apply(p: LogicalPlan): LogicalPlan = p + } test("getOrCreate instantiates SQLContext") { val sqlContext = SQLContext.getOrCreate(sc) @@ -33,7 +43,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext{ val newSession = sqlContext.newSession() assert(SQLContext.getOrCreate(sc).eq(sqlContext), "SQLContext.getOrCreate after explicitly created SQLContext did not return the context") - SQLContext.setActive(newSession) + SparkSession.setActiveSession(newSession.sparkSession) assert(SQLContext.getOrCreate(sc).eq(newSession), "SQLContext.getOrCreate after explicitly setActive() did not return the active context") } @@ -53,7 +63,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext{ // temporary table should not be shared val df = session1.range(10) - df.registerTempTable("test1") + df.createOrReplaceTempView("test1") assert(session1.tableNames().contains("test1")) assert(!session2.tableNames().contains("test1")) @@ -65,4 +75,71 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext{ session2.sql("select myadd(1, 2)").explain() } } + + test("Catalyst optimization passes are modifiable at runtime") { + val sqlContext = SQLContext.getOrCreate(sc) + sqlContext.experimental.extraOptimizations = Seq(DummyRule) + assert(sqlContext.sessionState.optimizer.batches.flatMap(_.rules).contains(DummyRule)) + } + + test("get all tables") { + val sqlContext = SQLContext.getOrCreate(sc) + val df = sqlContext.range(10) + df.createOrReplaceTempView("listtablessuitetable") + assert( + sqlContext.tables().filter("tableName = 'listtablessuitetable'").collect().toSeq == + Row("", "listtablessuitetable", true) :: Nil) + + assert( + sqlContext.sql("SHOW tables").filter("tableName = 'listtablessuitetable'").collect().toSeq == + Row("", "listtablessuitetable", true) :: Nil) + + sqlContext.sessionState.catalog.dropTable( + TableIdentifier("listtablessuitetable"), ignoreIfNotExists = true, purge = false) + assert(sqlContext.tables().filter("tableName = 'listtablessuitetable'").count() === 0) + } + + test("getting all tables with a database name has no impact on returned table names") { + val sqlContext = SQLContext.getOrCreate(sc) + val df = sqlContext.range(10) + df.createOrReplaceTempView("listtablessuitetable") + assert( + sqlContext.tables("default").filter("tableName = 'listtablessuitetable'").collect().toSeq == + Row("", "listtablessuitetable", true) :: Nil) + + assert( + sqlContext.sql("show TABLES in default").filter("tableName = 'listtablessuitetable'") + .collect().toSeq == Row("", "listtablessuitetable", true) :: Nil) + + sqlContext.sessionState.catalog.dropTable( + TableIdentifier("listtablessuitetable"), ignoreIfNotExists = true, purge = false) + assert(sqlContext.tables().filter("tableName = 'listtablessuitetable'").count() === 0) + } + + test("query the returned DataFrame of tables") { + val sqlContext = SQLContext.getOrCreate(sc) + val df = sqlContext.range(10) + df.createOrReplaceTempView("listtablessuitetable") + + val expectedSchema = StructType( + StructField("database", StringType, false) :: + StructField("tableName", StringType, false) :: + StructField("isTemporary", BooleanType, false) :: Nil) + + Seq(sqlContext.tables(), sqlContext.sql("SHOW TABLes")).foreach { + case tableDF => + assert(expectedSchema === tableDF.schema) + + tableDF.createOrReplaceTempView("tables") + assert( + sqlContext.sql( + "SELECT isTemporary, tableName from tables WHERE tableName = 'listtablessuitetable'") + .collect().toSeq == Row(true, "listtablessuitetable") :: Nil) + assert( + sqlContext.tables().filter("tableName = 'tables'").select("tableName", "isTemporary") + .collect().toSeq == Row("tables", true) :: Nil) + sqlContext.dropTempTable("tables") + } + } + } 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 3de277a79a52..93a7777b70b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -17,38 +17,31 @@ package org.apache.spark.sql +import java.io.File import java.math.MathContext +import java.net.{MalformedURLException, URL} import java.sql.Timestamp +import java.util.concurrent.atomic.AtomicBoolean -import org.apache.spark.AccumulatorSuite -import org.apache.spark.sql.catalyst.DefaultParserDialect -import org.apache.spark.sql.catalyst.analysis.FunctionRegistry -import org.apache.spark.sql.catalyst.errors.DialectException +import org.apache.spark.{AccumulatorSuite, SparkException} +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} +import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.execution.aggregate -import org.apache.spark.sql.execution.joins.{CartesianProduct, SortMergeJoin} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, CartesianProductExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SQLTestData._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} +import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ -/** A SQL Dialect for testing purpose, and it can not be nested type */ -class MyDialect extends DefaultParserDialect - class SQLQuerySuite extends QueryTest with SharedSQLContext { import testImplicits._ setupTestData() - test("having clause") { - Seq(("one", 1), ("two", 2), ("three", 3), ("one", 5)).toDF("k", "v").registerTempTable("hav") - checkAnswer( - sql("SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2"), - Row("one", 6) :: Row("three", 3) :: Nil) - } - test("SPARK-8010: promote numeric to string") { val df = Seq((1, 1)).toDF("key", "value") - df.registerTempTable("src") + df.createOrReplaceTempView("src") val queryCaseWhen = sql("select case when true then 1.0 else '1' end from src ") val queryCoalesce = sql("select coalesce(null, 1, '1') from src ") @@ -57,29 +50,66 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("show functions") { - checkAnswer(sql("SHOW functions"), - FunctionRegistry.builtin.listFunction().sorted.map(Row(_))) + def getFunctions(pattern: String): Seq[Row] = { + StringUtils.filterPattern( + spark.sessionState.catalog.listFunctions("default").map(_._1.funcName), pattern) + .map(Row(_)) + } + + def createFunction(names: Seq[String]): Unit = { + names.foreach { name => + spark.udf.register(name, (arg1: Int, arg2: String) => arg2 + arg1) + } + } + + def dropFunction(names: Seq[String]): Unit = { + names.foreach { name => + spark.sessionState.catalog.dropTempFunction(name, false) + } + } + + val functions = Array("ilog", "logi", "logii", "logiii", "crc32i", "cubei", "cume_disti", + "isize", "ispace", "to_datei", "date_addi", "current_datei") + + createFunction(functions) + + checkAnswer(sql("SHOW functions"), getFunctions("*")) + assert(sql("SHOW functions").collect().size > 200) + + Seq("^c*", "*e$", "log*", "*date*").foreach { pattern => + // For the pattern part, only '*' and '|' are allowed as wildcards. + // For '*', we need to replace it to '.*'. + checkAnswer(sql(s"SHOW FUNCTIONS '$pattern'"), getFunctions(pattern)) + } + dropFunction(functions) } test("describe functions") { - checkExistence(sql("describe function extended upper"), true, + checkKeywordsExist(sql("describe function extended upper"), "Function: upper", "Class: org.apache.spark.sql.catalyst.expressions.Upper", - "Usage: upper(str) - Returns str with all characters changed to uppercase", + "Usage: upper(str) - Returns `str` with all characters changed to uppercase", "Extended Usage:", + "Examples:", "> SELECT upper('SparkSql');", - "'SPARKSQL'") + "SPARKSQL") - checkExistence(sql("describe functioN Upper"), true, + checkKeywordsExist(sql("describe functioN Upper"), "Function: upper", "Class: org.apache.spark.sql.catalyst.expressions.Upper", - "Usage: upper(str) - Returns str with all characters changed to uppercase") + "Usage: upper(str) - Returns `str` with all characters changed to uppercase") - checkExistence(sql("describe functioN Upper"), false, - "Extended Usage") + checkKeywordsNotExist(sql("describe functioN Upper"), "Extended Usage") - checkExistence(sql("describe functioN abcadf"), true, - "Function: abcadf is not found.") + checkKeywordsExist(sql("describe functioN abcadf"), "Function: abcadf not found.") + } + + test("SPARK-14415: All functions should have own descriptions") { + for (f <- spark.sessionState.functionRegistry.listFunction()) { + if (!Seq("cube", "grouping", "grouping_id", "rollup", "window").contains(f.unquotedString)) { + checkKeywordsNotExist(sql(s"describe function `$f`"), "N/A.") + } + } } test("SPARK-6743: no columns from cache") { @@ -87,16 +117,18 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { (83, 0, 38), (26, 0, 79), (43, 81, 24) - ).toDF("a", "b", "c").registerTempTable("cachedData") + ).toDF("a", "b", "c").createOrReplaceTempView("cachedData") - sqlContext.cacheTable("cachedData") - checkAnswer( - sql("SELECT t1.b FROM cachedData, cachedData t1 GROUP BY t1.b"), - Row(0) :: Row(81) :: Nil) + spark.catalog.cacheTable("cachedData") + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + checkAnswer( + sql("SELECT t1.b FROM cachedData, cachedData t1 GROUP BY t1.b"), + Row(0) :: Row(81) :: Nil) + } } test("self join with aliases") { - Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str").registerTempTable("df") + Seq(1, 2, 3).map(i => (i, i.toString)).toDF("int", "str").createOrReplaceTempView("df") checkAnswer( sql( @@ -124,7 +156,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { .toDF("int", "str") .groupBy("str") .agg($"str", count("str").as("strCount")) - .registerTempTable("df") + .createOrReplaceTempView("df") checkAnswer( sql( @@ -147,23 +179,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { .count(), Row(24, 1) :: Row(14, 1) :: Nil) } - test("SQL Dialect Switching to a new SQL parser") { - val newContext = new SQLContext(sparkContext) - newContext.setConf("spark.sql.dialect", classOf[MyDialect].getCanonicalName()) - assert(newContext.getSQLDialect().getClass === classOf[MyDialect]) - assert(newContext.sql("SELECT 1").collect() === Array(Row(1))) - } - - test("SQL Dialect Switch to an invalid parser with alias") { - val newContext = new SQLContext(sparkContext) - newContext.sql("SET spark.sql.dialect=MyTestClass") - intercept[DialectException] { - newContext.sql("SELECT 1") - } - // test if the dialect set back to DefaultSQLDialect - assert(newContext.getSQLDialect().getClass === classOf[DefaultParserDialect]) - } - test("SPARK-4625 support SORT BY in SimpleSQLParser & DSL") { checkAnswer( sql("SELECT a FROM testData2 SORT BY a"), @@ -197,9 +212,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("grouping on nested fields") { - sqlContext.read.json(sparkContext.parallelize( - """{"nested": {"attribute": 1}, "value": 2}""" :: Nil)) - .registerTempTable("rows") + spark.read + .json(Seq("""{"nested": {"attribute": 1}, "value": 2}""").toDS()) + .createOrReplaceTempView("rows") checkAnswer( sql( @@ -215,10 +230,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-6201 IN type conversion") { - sqlContext.read.json( - sparkContext.parallelize( - Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}"))) - .registerTempTable("d") + spark.read + .json(Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}").toDS()) + .createOrReplaceTempView("d") checkAnswer( sql("select * from d where d.a in (1,2)"), @@ -226,10 +240,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-11226 Skip empty line in json file") { - sqlContext.read.json( - sparkContext.parallelize( - Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}", ""))) - .registerTempTable("d") + spark.read + .json(Seq("{\"a\": \"1\"}}", "{\"a\": \"2\"}}", "{\"a\": \"3\"}}", "").toDS()) + .createOrReplaceTempView("d") checkAnswer( sql("select count(1) from d"), @@ -237,46 +250,22 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-8828 sum should return null if all input values are null") { - withSQLConf(SQLConf.USE_SQL_AGGREGATE2.key -> "true") { - withSQLConf(SQLConf.CODEGEN_ENABLED.key -> "true") { - checkAnswer( - sql("select sum(a), avg(a) from allNulls"), - Seq(Row(null, null)) - ) - } - withSQLConf(SQLConf.CODEGEN_ENABLED.key -> "false") { - checkAnswer( - sql("select sum(a), avg(a) from allNulls"), - Seq(Row(null, null)) - ) - } - } - withSQLConf(SQLConf.USE_SQL_AGGREGATE2.key -> "false") { - withSQLConf(SQLConf.CODEGEN_ENABLED.key -> "true") { - checkAnswer( - sql("select sum(a), avg(a) from allNulls"), - Seq(Row(null, null)) - ) - } - withSQLConf(SQLConf.CODEGEN_ENABLED.key -> "false") { - checkAnswer( - sql("select sum(a), avg(a) from allNulls"), - Seq(Row(null, null)) - ) - } - } + checkAnswer( + sql("select sum(a), avg(a) from allNulls"), + Seq(Row(null, null)) + ) } private def testCodeGen(sqlText: String, expectedResults: Seq[Row]): Unit = { val df = sql(sqlText) // First, check if we have GeneratedAggregate. - val hasGeneratedAgg = df.queryExecution.executedPlan - .collect { case _: aggregate.TungstenAggregate => true } + val hasGeneratedAgg = df.queryExecution.sparkPlan + .collect { case _: aggregate.HashAggregateExec => true } .nonEmpty if (!hasGeneratedAgg) { fail( s""" - |Codegen is enabled, but query $sqlText does not have TungstenAggregate in the plan. + |Codegen is enabled, but query $sqlText does not have HashAggregate in the plan. |${df.queryExecution.simpleString} """.stripMargin) } @@ -285,13 +274,11 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("aggregation with codegen") { - val originalValue = sqlContext.conf.codegenEnabled - sqlContext.setConf(SQLConf.CODEGEN_ENABLED, true) // Prepare a table that we can group some rows. - sqlContext.table("testData") - .unionAll(sqlContext.table("testData")) - .unionAll(sqlContext.table("testData")) - .registerTempTable("testData3x") + spark.table("testData") + .union(spark.table("testData")) + .union(spark.table("testData")) + .createOrReplaceTempView("testData3x") try { // Just to group rows. @@ -340,13 +327,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { testCodeGen( "SELECT min(key) FROM testData3x", Row(1) :: Nil) - // STDDEV - testCodeGen( - "SELECT a, stddev(b), stddev_pop(b) FROM testData2 GROUP BY a", - (1 to 3).map(i => Row(i, math.sqrt(0.5), math.sqrt(0.25)))) - testCodeGen( - "SELECT stddev(b), stddev_pop(b), stddev_samp(b) FROM testData2", - Row(math.sqrt(1.5 / 5), math.sqrt(1.5 / 6), math.sqrt(1.5 / 5)) :: Nil) // Some combinations. testCodeGen( """ @@ -367,11 +347,10 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(100, 1, 50.5, 300, 100) :: Nil) // Aggregate with Code generation handling all null values testCodeGen( - "SELECT sum('a'), avg('a'), stddev('a'), count(null) FROM testData", - Row(null, null, null, 0) :: Nil) + "SELECT sum('a'), avg('a'), count(null) FROM testData", + Row(null, null, 0) :: Nil) } finally { - sqlContext.dropTempTable("testData3x") - sqlContext.setConf(SQLConf.CODEGEN_ENABLED, originalValue) + spark.catalog.dropTempView("testData3x") } } @@ -429,7 +408,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-3173 Timestamp support in the parser") { - (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time").registerTempTable("timestamps") + (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time").createOrReplaceTempView("timestamps") checkAnswer(sql( "SELECT time FROM timestamps WHERE time='1969-12-31 16:00:00.0'"), @@ -465,17 +444,13 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Nil) } - test("index into array") { - checkAnswer( - sql("SELECT data, data[0], data[0] + data[1], data[0 + 1] FROM arrayData"), - arrayData.map(d => Row(d.data, d.data(0), d.data(0) + d.data(1), d.data(1))).collect()) - } - test("left semi greater than predicate") { - checkAnswer( - sql("SELECT * FROM testData2 x LEFT SEMI JOIN testData2 y ON x.a >= y.a + 2"), - Seq(Row(3, 1), Row(3, 2)) - ) + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + checkAnswer( + sql("SELECT * FROM testData2 x LEFT SEMI JOIN testData2 y ON x.a >= y.a + 2"), + Seq(Row(3, 1), Row(3, 2)) + ) + } } test("left semi greater than predicate and equal operator") { @@ -490,56 +465,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { ) } - test("index into array of arrays") { - checkAnswer( - sql( - "SELECT nestedData, nestedData[0][0], nestedData[0][0] + nestedData[0][1] FROM arrayData"), - arrayData.map(d => - Row(d.nestedData, - d.nestedData(0)(0), - d.nestedData(0)(0) + d.nestedData(0)(1))).collect().toSeq) - } - - test("agg") { - checkAnswer( - sql("SELECT a, SUM(b) FROM testData2 GROUP BY a"), - Seq(Row(1, 3), Row(2, 3), Row(3, 3))) - } - - test("literal in agg grouping expressions") { - def literalInAggTest(): Unit = { - checkAnswer( - sql("SELECT a, count(1) FROM testData2 GROUP BY a, 1"), - Seq(Row(1, 2), Row(2, 2), Row(3, 2))) - checkAnswer( - sql("SELECT a, count(2) FROM testData2 GROUP BY a, 2"), - Seq(Row(1, 2), Row(2, 2), Row(3, 2))) - - checkAnswer( - sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY a, 1"), - sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY a")) - checkAnswer( - sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY a, 1 + 2"), - sql("SELECT a, 1, sum(b) FROM testData2 GROUP BY a")) - checkAnswer( - sql("SELECT 1, 2, sum(b) FROM testData2 GROUP BY 1, 2"), - sql("SELECT 1, 2, sum(b) FROM testData2")) - } - - literalInAggTest() - withSQLConf(SQLConf.USE_SQL_AGGREGATE2.key -> "false") { - literalInAggTest() - } - } - - test("aggregates with nulls") { - checkAnswer( - sql("SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a)," + - "AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) FROM nullInts"), - Row(0, -1.5, 1, 3, 2, 1.0, 1, 6, 3) - ) - } - test("select *") { checkAnswer( sql("SELECT * FROM testData"), @@ -598,26 +523,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sortTest() } - test("SPARK-6927 external sorting with codegen on") { - withSQLConf(SQLConf.CODEGEN_ENABLED.key -> "true") { - sortTest() - } - } - - test("limit") { - checkAnswer( - sql("SELECT * FROM testData LIMIT 10"), - testData.take(10).toSeq) - - checkAnswer( - sql("SELECT * FROM arrayData LIMIT 1"), - arrayData.collect().take(1).map(Row.fromTuple).toSeq) - - checkAnswer( - sql("SELECT * FROM mapData LIMIT 1"), - mapData.collect().take(1).map(Row.fromTuple).toSeq) - } - test("CTE feature") { checkAnswer( sql("with q1 as (select * from testData limit 10) select * from q1"), @@ -633,7 +538,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("Allow only a single WITH clause per query") { - intercept[RuntimeException] { + intercept[AnalysisException] { sql( "with q1 as (select * from testData) with q2 as (select * from q1) select * from q2") } @@ -649,8 +554,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("from follow multiple brackets") { checkAnswer(sql( """ - |select key from ((select * from testData limit 1) - | union all (select * from testData limit 1)) x limit 1 + |select key from ((select * from testData) + | union all (select * from testData)) x limit 1 """.stripMargin), Row(1) ) @@ -663,7 +568,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { checkAnswer(sql( """ |select key from - | (select * from testData limit 1 union all select * from testData limit 1) x + | (select * from testData union all select * from testData) x | limit 1 """.stripMargin), Row(1) @@ -696,13 +601,13 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("approximate count distinct") { checkAnswer( - sql("SELECT APPROXIMATE COUNT(DISTINCT a) FROM testData2"), + sql("SELECT APPROX_COUNT_DISTINCT(a) FROM testData2"), Row(3)) } test("approximate count distinct with user provided standard deviation") { checkAnswer( - sql("SELECT APPROXIMATE(0.04) COUNT(DISTINCT a) FROM testData2"), + sql("SELECT APPROX_COUNT_DISTINCT(a, 0.04) FROM testData2"), Row(3)) } @@ -718,8 +623,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("count of empty table") { - withTempTable("t") { - Seq.empty[(Int, Int)].toDF("a", "b").registerTempTable("t") + withTempView("t") { + Seq.empty[(Int, Int)].toDF("a", "b").createOrReplaceTempView("t") checkAnswer( sql("select count(a) from t"), Row(0)) @@ -727,36 +632,43 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("inner join where, one match per row") { - checkAnswer( - sql("SELECT * FROM upperCaseData JOIN lowerCaseData WHERE n = N"), - Seq( - Row(1, "A", 1, "a"), - Row(2, "B", 2, "b"), - Row(3, "C", 3, "c"), - Row(4, "D", 4, "d"))) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer( + sql("SELECT * FROM uppercasedata JOIN lowercasedata WHERE n = N"), + Seq( + Row(1, "A", 1, "a"), + Row(2, "B", 2, "b"), + Row(3, "C", 3, "c"), + Row(4, "D", 4, "d"))) + } } test("inner join ON, one match per row") { - checkAnswer( - sql("SELECT * FROM upperCaseData JOIN lowerCaseData ON n = N"), - Seq( - Row(1, "A", 1, "a"), - Row(2, "B", 2, "b"), - Row(3, "C", 3, "c"), - Row(4, "D", 4, "d"))) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer( + sql("SELECT * FROM uppercasedata JOIN lowercasedata ON n = N"), + Seq( + Row(1, "A", 1, "a"), + Row(2, "B", 2, "b"), + Row(3, "C", 3, "c"), + Row(4, "D", 4, "d"))) + } } test("inner join, where, multiple matches") { - checkAnswer( - sql(""" - |SELECT * FROM - | (SELECT * FROM testData2 WHERE a = 1) x JOIN - | (SELECT * FROM testData2 WHERE a = 1) y - |WHERE x.a = y.a""".stripMargin), - Row(1, 1, 1, 1) :: - Row(1, 1, 1, 2) :: - Row(1, 2, 1, 1) :: - Row(1, 2, 1, 2) :: Nil) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer( + sql( + """ + |SELECT * FROM + | (SELECT * FROM testdata2 WHERE a = 1) x JOIN + | (SELECT * FROM testdata2 WHERE a = 1) y + |WHERE x.a = y.a""".stripMargin), + Row(1, 1, 1, 1) :: + Row(1, 1, 1, 2) :: + Row(1, 2, 1, 1) :: + Row(1, 2, 1, 2) :: Nil) + } } test("inner join, no matches") { @@ -789,34 +701,40 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("cartesian product join") { - checkAnswer( - testData3.join(testData3), - Row(1, null, 1, null) :: - Row(1, null, 2, 2) :: - Row(2, 2, 1, null) :: - Row(2, 2, 2, 2) :: Nil) + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + checkAnswer( + testData3.join(testData3), + Row(1, null, 1, null) :: + Row(1, null, 2, 2) :: + Row(2, 2, 1, null) :: + Row(2, 2, 2, 2) :: Nil) + } } test("left outer join") { - checkAnswer( - sql("SELECT * FROM upperCaseData LEFT OUTER JOIN lowerCaseData ON n = N"), - Row(1, "A", 1, "a") :: - Row(2, "B", 2, "b") :: - Row(3, "C", 3, "c") :: - Row(4, "D", 4, "d") :: - Row(5, "E", null, null) :: - Row(6, "F", null, null) :: Nil) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer( + sql("SELECT * FROM uppercasedata LEFT OUTER JOIN lowercasedata ON n = N"), + Row(1, "A", 1, "a") :: + Row(2, "B", 2, "b") :: + Row(3, "C", 3, "c") :: + Row(4, "D", 4, "d") :: + Row(5, "E", null, null) :: + Row(6, "F", null, null) :: Nil) + } } test("right outer join") { - checkAnswer( - sql("SELECT * FROM lowerCaseData RIGHT OUTER JOIN upperCaseData ON n = N"), - Row(1, "a", 1, "A") :: - Row(2, "b", 2, "B") :: - Row(3, "c", 3, "C") :: - Row(4, "d", 4, "D") :: - Row(null, null, 5, "E") :: - Row(null, null, 6, "F") :: Nil) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer( + sql("SELECT * FROM lowercasedata RIGHT OUTER JOIN uppercasedata ON n = N"), + Row(1, "a", 1, "A") :: + Row(2, "b", 2, "B") :: + Row(3, "c", 3, "C") :: + Row(4, "d", 4, "D") :: + Row(null, null, 5, "E") :: + Row(null, null, 6, "F") :: Nil) + } } test("full outer join") { @@ -838,24 +756,25 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SPARK-11111 null-safe join should not use cartesian product") { val df = sql("select count(*) from testData a join testData b on (a.key <=> b.key)") - val cp = df.queryExecution.executedPlan.collect { - case cp: CartesianProduct => cp + val cp = df.queryExecution.sparkPlan.collect { + case cp: CartesianProductExec => cp } assert(cp.isEmpty, "should not use CartesianProduct for null-safe join") - val smj = df.queryExecution.executedPlan.collect { - case smj: SortMergeJoin => smj + val smj = df.queryExecution.sparkPlan.collect { + case smj: SortMergeJoinExec => smj + case j: BroadcastHashJoinExec => j } - assert(smj.size > 0, "should use SortMergeJoin") + assert(smj.size > 0, "should use SortMergeJoin or BroadcastHashJoin") checkAnswer(df, Row(100) :: Nil) } test("SPARK-3349 partitioning after limit") { sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") .limit(2) - .registerTempTable("subset1") - sql("SELECT DISTINCT n FROM lowerCaseData") + .createOrReplaceTempView("subset1") + sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n ASC") .limit(2) - .registerTempTable("subset2") + .createOrReplaceTempView("subset2") checkAnswer( sql("SELECT * FROM lowerCaseData INNER JOIN subset1 ON subset1.n = lowerCaseData.n"), Row(3, "c", 3) :: @@ -990,6 +909,16 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sql("SELECT * FROM upperCaseData EXCEPT SELECT * FROM upperCaseData"), Nil) } + test("MINUS") { + checkAnswer( + sql("SELECT * FROM lowerCaseData MINUS SELECT * FROM upperCaseData"), + Row(1, "a") :: Row(2, "b") :: Row(3, "c") :: Row(4, "d") :: Nil) + checkAnswer( + sql("SELECT * FROM lowerCaseData MINUS SELECT * FROM lowerCaseData"), Nil) + checkAnswer( + sql("SELECT * FROM upperCaseData MINUS SELECT * FROM upperCaseData"), Nil) + } + test("INTERSECT") { checkAnswer( sql("SELECT * FROM lowerCaseData INTERSECT SELECT * FROM lowerCaseData"), @@ -1002,7 +931,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SET commands semantics using sql()") { - sqlContext.conf.clear() + spark.sessionState.conf.clear() val testKey = "test.key.0" val testVal = "test.val.0" val nonexistentKey = "nonexistent" @@ -1043,17 +972,56 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sql(s"SET $nonexistentKey"), Row(nonexistentKey, "") ) - sqlContext.conf.clear() + spark.sessionState.conf.clear() + } + + test("SPARK-19218 SET command should show a result in a sorted order") { + val overrideConfs = sql("SET").collect() + sql(s"SET test.key3=1") + sql(s"SET test.key2=2") + sql(s"SET test.key1=3") + val result = sql("SET").collect() + assert(result === + (overrideConfs ++ Seq( + Row("test.key1", "3"), + Row("test.key2", "2"), + Row("test.key3", "1"))).sortBy(_.getString(0)) + ) + spark.sessionState.conf.clear() + } + + test("SPARK-19218 `SET -v` should not fail with null value configuration") { + import SQLConf._ + val confEntry = buildConf("spark.test").doc("doc").stringConf.createWithDefault(null) + + try { + val result = sql("SET -v").collect() + assert(result === result.sortBy(_.getString(0))) + } finally { + SQLConf.unregister(confEntry) + } } test("SET commands with illegal or inappropriate argument") { - sqlContext.conf.clear() - // Set negative mapred.reduce.tasks for automatically determing + spark.sessionState.conf.clear() + // Set negative mapred.reduce.tasks for automatically determining // the number of reducers is not supported intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-1")) intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-01")) intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-2")) - sqlContext.conf.clear() + spark.sessionState.conf.clear() + } + + test("SET mapreduce.job.reduces automatically converted to spark.sql.shuffle.partitions") { + spark.sessionState.conf.clear() + val before = spark.conf.get(SQLConf.SHUFFLE_PARTITIONS.key).toInt + val newConf = before + 1 + sql(s"SET mapreduce.job.reduces=${newConf.toString}") + val after = spark.conf.get(SQLConf.SHUFFLE_PARTITIONS.key).toInt + assert(before != after) + assert(newConf === after) + intercept[IllegalArgumentException](sql(s"SET mapreduce.job.reduces=-1")) + spark.sessionState.conf.clear() } test("apply schema") { @@ -1071,8 +1039,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(values(0).toInt, values(1), values(2).toBoolean, v4) } - val df1 = sqlContext.createDataFrame(rowRDD1, schema1) - df1.registerTempTable("applySchema1") + val df1 = spark.createDataFrame(rowRDD1, schema1) + df1.createOrReplaceTempView("applySchema1") checkAnswer( sql("SELECT * FROM applySchema1"), Row(1, "A1", true, null) :: @@ -1101,8 +1069,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(Row(values(0).toInt, values(2).toBoolean), Map(values(1) -> v4)) } - val df2 = sqlContext.createDataFrame(rowRDD2, schema2) - df2.registerTempTable("applySchema2") + val df2 = spark.createDataFrame(rowRDD2, schema2) + df2.createOrReplaceTempView("applySchema2") checkAnswer( sql("SELECT * FROM applySchema2"), Row(Row(1, true), Map("A1" -> null)) :: @@ -1126,8 +1094,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(Row(values(0).toInt, values(2).toBoolean), scala.collection.mutable.Map(values(1) -> v4)) } - val df3 = sqlContext.createDataFrame(rowRDD3, schema2) - df3.registerTempTable("applySchema3") + val df3 = spark.createDataFrame(rowRDD3, schema2) + df3.createOrReplaceTempView("applySchema3") checkAnswer( sql("SELECT f1.f11, f2['D4'] FROM applySchema3"), @@ -1154,6 +1122,30 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { ) } + test("SPARK-17863: SELECT distinct does not work correctly if order by missing attribute") { + checkAnswer( + sql("""select distinct struct.a, struct.b + |from ( + | select named_struct('a', 1, 'b', 2, 'c', 3) as struct + | union all + | select named_struct('a', 1, 'b', 2, 'c', 4) as struct) tmp + |order by a, b + |""".stripMargin), + Row(1, 2) :: Nil) + + val error = intercept[AnalysisException] { + sql("""select distinct struct.a, struct.b + |from ( + | select named_struct('a', 1, 'b', 2, 'c', 3) as struct + | union all + | select named_struct('a', 1, 'b', 2, 'c', 4) as struct) tmp + |order by struct.a, struct.b + |""".stripMargin) + } + assert(error.message contains "cannot resolve '`struct.a`' given input columns: [a, b]") + + } + test("cast boolean to string") { // TODO Ensure true/false string letter casing is consistent with Hive in all cases. checkAnswer( @@ -1171,11 +1163,11 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { .build() val schemaWithMeta = new StructType(Array( schema("id"), schema("name").copy(metadata = metadata), schema("age"))) - val personWithMeta = sqlContext.createDataFrame(person.rdd, schemaWithMeta) + val personWithMeta = spark.createDataFrame(person.rdd, schemaWithMeta) def validateMetadata(rdd: DataFrame): Unit = { assert(rdd.schema("name").metadata.getString(docKey) == docValue) } - personWithMeta.registerTempTable("personWithMeta") + personWithMeta.createOrReplaceTempView("personWithMeta") validateMetadata(personWithMeta.select($"name")) validateMetadata(personWithMeta.select($"name")) validateMetadata(personWithMeta.select($"id", $"name")) @@ -1187,7 +1179,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-3371 Renaming a function expression with group by gives error") { - sqlContext.udf.register("len", (s: String) => s.length) + spark.udf.register("len", (s: String) => s.length) checkAnswer( sql("SELECT len(value) as temp FROM testData WHERE key = 1 group by len(value)"), Row(1)) @@ -1205,155 +1197,12 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(1)) } - test("throw errors for non-aggregate attributes with aggregation") { - def checkAggregation(query: String, isInvalidQuery: Boolean = true) { - if (isInvalidQuery) { - val e = intercept[AnalysisException](sql(query).queryExecution.analyzed) - assert(e.getMessage contains "group by") - } else { - // Should not throw - sql(query).queryExecution.analyzed - } + testQuietly( + "SPARK-16748: SparkExceptions during planning should not wrapped in TreeNodeException") { + intercept[SparkException] { + val df = spark.range(0, 5).map(x => (1 / x).toString).toDF("a").orderBy("a") + df.queryExecution.toRdd // force physical planning, but not execution of the plan } - - checkAggregation("SELECT key, COUNT(*) FROM testData") - checkAggregation("SELECT COUNT(key), COUNT(*) FROM testData", isInvalidQuery = false) - - checkAggregation("SELECT value, COUNT(*) FROM testData GROUP BY key") - checkAggregation("SELECT COUNT(value), SUM(key) FROM testData GROUP BY key", false) - - checkAggregation("SELECT key + 2, COUNT(*) FROM testData GROUP BY key + 1") - checkAggregation("SELECT key + 1 + 1, COUNT(*) FROM testData GROUP BY key + 1", false) - } - - test("Test to check we can use Long.MinValue") { - checkAnswer( - sql(s"SELECT ${Long.MinValue} FROM testData ORDER BY key LIMIT 1"), Row(Long.MinValue) - ) - - checkAnswer( - sql(s"SELECT key FROM testData WHERE key > ${Long.MinValue}"), - (1 to 100).map(Row(_)).toSeq - ) - } - - test("Floating point number format") { - checkAnswer( - sql("SELECT 0.3"), Row(BigDecimal(0.3).underlying()) - ) - - checkAnswer( - sql("SELECT -0.8"), Row(BigDecimal(-0.8).underlying()) - ) - - checkAnswer( - sql("SELECT .5"), Row(BigDecimal(0.5)) - ) - - checkAnswer( - sql("SELECT -.18"), Row(BigDecimal(-0.18)) - ) - } - - test("Auto cast integer type") { - checkAnswer( - sql(s"SELECT ${Int.MaxValue + 1L}"), Row(Int.MaxValue + 1L) - ) - - checkAnswer( - sql(s"SELECT ${Int.MinValue - 1L}"), Row(Int.MinValue - 1L) - ) - - checkAnswer( - sql("SELECT 9223372036854775808"), Row(new java.math.BigDecimal("9223372036854775808")) - ) - - checkAnswer( - sql("SELECT -9223372036854775809"), Row(new java.math.BigDecimal("-9223372036854775809")) - ) - } - - test("Test to check we can apply sign to expression") { - - checkAnswer( - sql("SELECT -100"), Row(-100) - ) - - checkAnswer( - sql("SELECT +230"), Row(230) - ) - - checkAnswer( - sql("SELECT -5.2"), Row(BigDecimal(-5.2)) - ) - - checkAnswer( - sql("SELECT +6.8"), Row(BigDecimal(6.8)) - ) - - checkAnswer( - sql("SELECT -key FROM testData WHERE key = 2"), Row(-2) - ) - - checkAnswer( - sql("SELECT +key FROM testData WHERE key = 3"), Row(3) - ) - - checkAnswer( - sql("SELECT -(key + 1) FROM testData WHERE key = 1"), Row(-2) - ) - - checkAnswer( - sql("SELECT - key + 1 FROM testData WHERE key = 10"), Row(-9) - ) - - checkAnswer( - sql("SELECT +(key + 5) FROM testData WHERE key = 5"), Row(10) - ) - - checkAnswer( - sql("SELECT -MAX(key) FROM testData"), Row(-100) - ) - - checkAnswer( - sql("SELECT +MAX(key) FROM testData"), Row(100) - ) - - checkAnswer( - sql("SELECT - (-10)"), Row(10) - ) - - checkAnswer( - sql("SELECT + (-key) FROM testData WHERE key = 32"), Row(-32) - ) - - checkAnswer( - sql("SELECT - (+Max(key)) FROM testData"), Row(-100) - ) - - checkAnswer( - sql("SELECT - - 3"), Row(3) - ) - - checkAnswer( - sql("SELECT - + 20"), Row(-20) - ) - - checkAnswer( - sql("SELEcT - + 45"), Row(-45) - ) - - checkAnswer( - sql("SELECT + + 100"), Row(100) - ) - - checkAnswer( - sql("SELECT - - Max(key) FROM testData"), Row(100) - ) - - checkAnswer( - sql("SELECT + - key FROM testData WHERE key = 33"), Row(-33) - ) } test("Multiple join") { @@ -1368,10 +1217,11 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-3483 Special chars in column names") { - val data = sparkContext.parallelize( - Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) - sqlContext.read.json(data).registerTempTable("records") - sql("SELECT `key?number1`, `key.number2` FROM records") + val data = Seq("""{"key?number1": "value1", "key.number2": "value2"}""").toDS() + spark.read.json(data).createOrReplaceTempView("records") + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + sql("SELECT `key?number1`, `key.number2` FROM records") + } } test("SPARK-3814 Support Bitwise & operator") { @@ -1411,15 +1261,15 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-4322 Grouping field with struct field as sub expression") { - sqlContext.read.json(sparkContext.makeRDD("""{"a": {"b": [{"c": 1}]}}""" :: Nil)) - .registerTempTable("data") + spark.read.json(Seq("""{"a": {"b": [{"c": 1}]}}""").toDS()) + .createOrReplaceTempView("data") checkAnswer(sql("SELECT a.b[0].c FROM data GROUP BY a.b[0].c"), Row(1)) - sqlContext.dropTempTable("data") + spark.catalog.dropTempView("data") - sqlContext.read.json( - sparkContext.makeRDD("""{"a": {"b": 1}}""" :: Nil)).registerTempTable("data") + spark.read.json(Seq("""{"a": {"b": 1}}""").toDS()) + .createOrReplaceTempView("data") checkAnswer(sql("SELECT a.b + 1 FROM data GROUP BY a.b + 1"), Row(2)) - sqlContext.dropTempTable("data") + spark.catalog.dropTempView("data") } test("SPARK-4432 Fix attribute reference resolution error when using ORDER BY") { @@ -1439,10 +1289,10 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { 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().createOrReplaceTempView("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().createOrReplaceTempView("nulldata2") checkAnswer(sql("SELECT nulldata1.key FROM nulldata1 join " + "nulldata2 on nulldata1.value <=> nulldata2.value"), (1 to 2).map(i => Row(i))) @@ -1451,23 +1301,23 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { 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().createOrReplaceTempView("distinctData") checkAnswer(sql("SELECT COUNT(DISTINCT key,value) FROM distinctData"), Row(2)) } test("SPARK-4699 case sensitivity SQL query") { - sqlContext.setConf(SQLConf.CASE_SENSITIVE, false) - 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("testTable1") - checkAnswer(sql("SELECT VALUE FROM TESTTABLE1 where KEY = 1"), Row("val_1")) - sqlContext.setConf(SQLConf.CASE_SENSITIVE, true) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val data = TestData(1, "val_1") :: TestData(2, "val_2") :: Nil + val rdd = sparkContext.parallelize((0 to 1).map(i => data(i))) + rdd.toDF().createOrReplaceTempView("testTable1") + checkAnswer(sql("SELECT VALUE FROM TESTTABLE1 where KEY = 1"), Row("val_1")) + } } test("SPARK-6145: ORDER BY test for nested fields") { - sqlContext.read.json(sparkContext.makeRDD( - """{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""" :: Nil)) - .registerTempTable("nestedOrder") + spark.read + .json(Seq("""{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""").toDS()) + .createOrReplaceTempView("nestedOrder") checkAnswer(sql("SELECT 1 FROM nestedOrder ORDER BY a.b"), Row(1)) checkAnswer(sql("SELECT a.b FROM nestedOrder ORDER BY a.b"), Row(1)) @@ -1478,23 +1328,27 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-6145: special cases") { - sqlContext.read.json(sparkContext.makeRDD( - """{"a": {"b": [1]}, "b": [{"a": 1}], "_c0": {"a": 1}}""" :: Nil)).registerTempTable("t") + spark.read + .json(Seq("""{"a": {"b": [1]}, "b": [{"a": 1}], "_c0": {"a": 1}}""").toDS()) + .createOrReplaceTempView("t") + checkAnswer(sql("SELECT a.b[0] FROM t ORDER BY _c0.a"), Row(1)) checkAnswer(sql("SELECT b[0].a FROM t ORDER BY _c0.a"), Row(1)) } test("SPARK-6898: complete support for special chars in column names") { - sqlContext.read.json(sparkContext.makeRDD( - """{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""" :: Nil)) - .registerTempTable("t") + spark.read + .json(Seq("""{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""").toDS()) + .createOrReplaceTempView("t") - checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) + } } test("SPARK-6583 order by aggregated function") { Seq("1" -> 3, "1" -> 4, "2" -> 7, "2" -> 8, "3" -> 5, "3" -> 6, "4" -> 1, "4" -> 2) - .toDF("a", "b").registerTempTable("orderByData") + .toDF("a", "b").createOrReplaceTempView("orderByData") checkAnswer( sql( @@ -1568,7 +1422,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-7952: fix the equality check between boolean and numeric types") { - withTempTable("t") { + withTempView("t") { // numeric field i, boolean field j, result of i = j, result of i <=> j Seq[(Integer, java.lang.Boolean, java.lang.Boolean, java.lang.Boolean)]( (1, true, true, true), @@ -1580,7 +1434,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { (0, null, null, false), (1, null, null, false), (null, null, null, true) - ).toDF("i", "b", "r1", "r2").registerTempTable("t") + ).toDF("i", "b", "r1", "r2").createOrReplaceTempView("t") checkAnswer(sql("select i = b from t"), sql("select r1 from t")) checkAnswer(sql("select i <=> b from t"), sql("select r2 from t")) @@ -1588,25 +1442,26 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-7067: order by queries for complex ExtractValue chain") { - withTempTable("t") { - sqlContext.read.json(sparkContext.makeRDD( - """{"a": {"b": [{"c": 1}]}, "b": [{"d": 1}]}""" :: Nil)).registerTempTable("t") + withTempView("t") { + spark.read + .json(Seq("""{"a": {"b": [{"c": 1}]}, "b": [{"d": 1}]}""").toDS()) + .createOrReplaceTempView("t") checkAnswer(sql("SELECT a.b FROM t ORDER BY b[0].d"), Row(Seq(Row(1)))) } } test("SPARK-8782: ORDER BY NULL") { - withTempTable("t") { - Seq((1, 2), (1, 2)).toDF("a", "b").registerTempTable("t") + withTempView("t") { + Seq((1, 2), (1, 2)).toDF("a", "b").createOrReplaceTempView("t") checkAnswer(sql("SELECT * FROM t ORDER BY NULL"), Seq(Row(1, 2), Row(1, 2))) } } test("SPARK-8837: use keyword in column name") { - withTempTable("t") { + withTempView("t") { val df = Seq(1 -> "a").toDF("count", "sort") checkAnswer(df.filter("count > 0"), Row(1, "a")) - df.registerTempTable("t") + df.createOrReplaceTempView("t") checkAnswer(sql("select count, sort from t"), Row(1, "a")) } } @@ -1624,16 +1479,15 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { e.message.contains("Cannot save interval data type into external storage") }) - def checkIntervalParseError(s: String): Unit = { - val e = intercept[AnalysisException] { - sql(s) - } - e.message.contains("at least one time unit should be given for interval literal") + val e1 = intercept[AnalysisException] { + sql("select interval") } - - checkIntervalParseError("select interval") + assert(e1.message.contains("at least one time unit should be given for interval literal")) // Currently we don't yet support nanosecond - checkIntervalParseError("select interval 23 nanosecond") + val e2 = intercept[AnalysisException] { + sql("select interval 23 nanosecond") + } + assert(e2.message.contains("No interval can be constructed")) } test("SPARK-8945: add and subtract expressions for interval type") { @@ -1655,12 +1509,10 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("aggregation with codegen updates peak execution memory") { - withSQLConf((SQLConf.CODEGEN_ENABLED.key, "true")) { - AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "aggregation with codegen") { - testCodeGen( - "SELECT key, count(value) FROM testData GROUP BY key", - (1 to 100).map(i => Row(i, 1))) - } + AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "aggregation with codegen") { + testCodeGen( + "SELECT key, count(value) FROM testData GROUP BY key", + (1 to 100).map(i => Row(i, 1))) } } @@ -1683,24 +1535,24 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-10215 Div of Decimal returns null") { - val d = Decimal(1.12321) + val d = Decimal(1.12321).toBigDecimal val df = Seq((d, 1)).toDF("a", "b") checkAnswer( df.selectExpr("b * a / b"), - Seq(Row(d.toBigDecimal))) + Seq(Row(d))) checkAnswer( df.selectExpr("b * a / b / b"), - Seq(Row(d.toBigDecimal))) + Seq(Row(d))) checkAnswer( df.selectExpr("b * a + b"), - Seq(Row(BigDecimal(2.12321)))) + Seq(Row(BigDecimal("2.12321")))) checkAnswer( df.selectExpr("b * a - b"), - Seq(Row(BigDecimal(0.12321)))) + Seq(Row(BigDecimal("0.12321")))) checkAnswer( df.selectExpr("b * a * b"), - Seq(Row(d.toBigDecimal))) + Seq(Row(d))) } test("precision smaller than scale") { @@ -1715,22 +1567,22 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("external sorting updates peak execution memory") { AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "external sort") { - sortTest() + sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC").collect() } } test("SPARK-9511: error with table starting with number") { - withTempTable("1one") { + withTempView("1one") { sparkContext.parallelize(1 to 10).map(i => (i, i.toString)) .toDF("num", "str") - .registerTempTable("1one") + .createOrReplaceTempView("1one") checkAnswer(sql("select count(num) from 1one"), Row(10)) } } - test("specifying database name for a temporary table is not allowed") { + test("specifying database name for a temporary view is not allowed") { withTempPath { dir => - val path = dir.getCanonicalPath + val path = dir.toURI.toString val df = sparkContext.parallelize(1 to 10).map(i => (i, i.toString)).toDF("num", "str") df @@ -1738,42 +1590,42 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { .format("parquet") .save(path) - val message = intercept[AnalysisException] { - sqlContext.sql( + // We don't support creating a temporary table while specifying a database + intercept[AnalysisException] { + spark.sql( s""" - |CREATE TEMPORARY TABLE db.t - |USING parquet - |OPTIONS ( - | path '$path' - |) - """.stripMargin) + |CREATE TEMPORARY VIEW db.t + |USING parquet + |OPTIONS ( + | path '$path' + |) + """.stripMargin) }.getMessage - assert(message.contains("Specifying database name or other qualifiers are not allowed")) - // If you use backticks to quote the name of a temporary table having dot in it. - sqlContext.sql( + // If you use backticks to quote the name then it's OK. + spark.sql( s""" - |CREATE TEMPORARY TABLE `db.t` + |CREATE TEMPORARY VIEW `db.t` |USING parquet |OPTIONS ( | path '$path' |) - """.stripMargin) - checkAnswer(sqlContext.table("`db.t`"), df) + """.stripMargin) + checkAnswer(spark.table("`db.t`"), df) } } test("SPARK-10130 type coercion for IF should have children resolved first") { - withTempTable("src") { - Seq((1, 1), (-1, 1)).toDF("key", "value").registerTempTable("src") + withTempView("src") { + Seq((1, 1), (-1, 1)).toDF("key", "value").createOrReplaceTempView("src") checkAnswer( sql("SELECT IF(a > 0, a, 0) FROM (SELECT key a FROM src) temp"), Seq(Row(1), Row(0))) } } test("SPARK-10389: order by non-attribute grouping expression on Aggregate") { - withTempTable("src") { - Seq((1, 1), (-1, 1)).toDF("key", "value").registerTempTable("src") + withTempView("src") { + Seq((1, 1), (-1, 1)).toDF("key", "value").createOrReplaceTempView("src") checkAnswer(sql("SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY key + 1"), Seq(Row(1), Row(1))) checkAnswer(sql("SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY (key + 1) * 2"), @@ -1782,7 +1634,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("run sql directly on files") { - val df = sqlContext.range(100) + val df = spark.range(100).toDF() withTempPath(f => { df.write.json(f.getCanonicalPath) checkAnswer(sql(s"select id from json.`${f.getCanonicalPath}`"), @@ -1793,30 +1645,65 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { df) }) - val e1 = intercept[AnalysisException] { + var e = intercept[AnalysisException] { sql("select * from in_valid_table") } - assert(e1.message.contains("Table not found")) + assert(e.message.contains("Table or view not found")) - val e2 = intercept[AnalysisException] { - sql("select * from no_db.no_table") + e = intercept[AnalysisException] { + sql("select * from no_db.no_table").show() } - assert(e2.message.contains("Table not found")) + assert(e.message.contains("Table or view not found")) - val e3 = intercept[AnalysisException] { + e = intercept[AnalysisException] { sql("select * from json.invalid_file") } - assert(e3.message.contains("No input paths specified")) + assert(e.message.contains("Path does not exist")) + + e = intercept[AnalysisException] { + sql(s"select id from `org.apache.spark.sql.hive.orc`.`file_path`") + } + assert(e.message.contains("The ORC data source must be used with Hive support enabled")) + + e = intercept[AnalysisException] { + sql(s"select id from `com.databricks.spark.avro`.`file_path`") + } + assert(e.message.contains("Failed to find data source: com.databricks.spark.avro.")) + + // data source type is case insensitive + e = intercept[AnalysisException] { + sql(s"select id from Avro.`file_path`") + } + assert(e.message.contains("Failed to find data source: avro.")) + + e = intercept[AnalysisException] { + sql(s"select id from avro.`file_path`") + } + assert(e.message.contains("Failed to find data source: avro.")) + + e = intercept[AnalysisException] { + sql(s"select id from `org.apache.spark.sql.sources.HadoopFsRelationProvider`.`file_path`") + } + assert(e.message.contains("Table or view not found: " + + "`org.apache.spark.sql.sources.HadoopFsRelationProvider`.`file_path`")) + + e = intercept[AnalysisException] { + sql(s"select id from `Jdbc`.`file_path`") + } + assert(e.message.contains("Unsupported data source type for direct query on files: Jdbc")) + + e = intercept[AnalysisException] { + sql(s"select id from `org.apache.spark.sql.execution.datasources.jdbc`.`file_path`") + } + assert(e.message.contains("Unsupported data source type for direct query on files: " + + "org.apache.spark.sql.execution.datasources.jdbc")) } test("SortMergeJoin returns wrong results when using UnsafeRows") { // This test is for the fix of https://issues.apache.org/jira/browse/SPARK-10737. // This bug will be triggered when Tungsten is enabled and there are multiple // SortMergeJoin operators executed in the same task. - val confs = - SQLConf.SORTMERGE_JOIN.key -> "true" :: - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1" :: - SQLConf.TUNGSTEN_ENABLED.key -> "true" :: Nil + val confs = SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1" :: Nil withSQLConf(confs: _*) { val df1 = (1 to 50).map(i => (s"str_$i", i)).toDF("i", "j") val df2 = @@ -1837,17 +1724,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } - test("SPARK-11032: resolve having correctly") { - withTempTable("src") { - Seq(1 -> "a").toDF("i", "j").registerTempTable("src") - checkAnswer( - sql("SELECT MIN(t.i) FROM (SELECT * FROM src WHERE i > 0) t HAVING(COUNT(1) > 0)"), - Row(1)) - } - } - test("SPARK-11303: filter should not be pushed down into sample") { - val df = sqlContext.range(100) + val df = spark.range(100) List(true, false).foreach { withReplacement => val sampled = df.sample(withReplacement, 0.1, 1) val sampledOdd = sampled.filter("id % 2 != 0") @@ -1877,8 +1755,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(1, 1, 1, 1) :: Row(1, 2, 2, 1) :: Row(2, 1, 1, 2) :: Row(2, 2, 2, 2) :: Row(3, 1, 1, 3) :: Row(3, 2, 2, 3) :: Nil) - // Try with a registered table. - sql("select struct(a, b) as record from testData2").registerTempTable("structTable") + // Try with a temporary view + sql("select struct(a, b) as record from testData2").createOrReplaceTempView("structTable") checkAnswer( sql("SELECT record.* FROM structTable"), Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil) @@ -1942,9 +1820,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { nestedStructData.select($"record.r1.*"), Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil) - // Try with a registered table - withTempTable("nestedStructTable") { - nestedStructData.registerTempTable("nestedStructTable") + // Try with a temporary view + withTempView("nestedStructTable") { + nestedStructData.createOrReplaceTempView("nestedStructTable") checkAnswer( sql("SELECT record.* FROM nestedStructTable"), nestedStructData.select($"record.*")) @@ -1961,25 +1839,28 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } // Create paths with unusual characters - val specialCharacterPath = sql( - """ + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + val specialCharacterPath = sql( + """ | SELECT struct(`col$.a_`, `a.b.c.`) as `r&&b.c` FROM | (SELECT struct(a, b) as `col$.a_`, struct(b, a) as `a.b.c.` FROM testData2) tmp """.stripMargin) - withTempTable("specialCharacterTable") { - specialCharacterPath.registerTempTable("specialCharacterTable") - checkAnswer( - specialCharacterPath.select($"`r&&b.c`.*"), - nestedStructData.select($"record.*")) - checkAnswer( - sql("SELECT `r&&b.c`.`col$.a_` FROM specialCharacterTable"), + withTempView("specialCharacterTable") { + specialCharacterPath.createOrReplaceTempView("specialCharacterTable") + checkAnswer( + specialCharacterPath.select($"`r&&b.c`.*"), + nestedStructData.select($"record.*")) + checkAnswer( + sql( + "SELECT `r&&b.c`.`col$.a_` FROM specialCharacterTable"), nestedStructData.select($"record.r1")) - checkAnswer( - sql("SELECT `r&&b.c`.`a.b.c.` FROM specialCharacterTable"), - nestedStructData.select($"record.r2")) - checkAnswer( - sql("SELECT `r&&b.c`.`col$.a_`.* FROM specialCharacterTable"), - nestedStructData.select($"record.r1.*")) + checkAnswer( + sql("SELECT `r&&b.c`.`a.b.c.` FROM specialCharacterTable"), + nestedStructData.select($"record.r2")) + checkAnswer( + sql("SELECT `r&&b.c`.`col$.a_`.* FROM specialCharacterTable"), + nestedStructData.select($"record.r1.*")) + } } // Try star expanding a scalar. This should fail. @@ -1990,8 +1871,8 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("Struct Star Expansion - Name conflict") { // Create a data set that contains a naming conflict val nameConflict = sql("SELECT struct(a, b) as nameConflict, a as a FROM testData2") - withTempTable("nameConflict") { - nameConflict.registerTempTable("nameConflict") + withTempView("nameConflict") { + nameConflict.createOrReplaceTempView("nameConflict") // Unqualified should resolve to table. checkAnswer(sql("SELECT nameConflict.* FROM nameConflict"), Row(Row(1, 1), 1) :: Row(Row(1, 2), 1) :: Row(Row(2, 1), 2) :: Row(Row(2, 2), 2) :: @@ -2001,4 +1882,799 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(1, 1) :: Row(1, 2) :: Row(2, 1) :: Row(2, 2) :: Row(3, 1) :: Row(3, 2) :: Nil) } } + + test("Star Expansion - group by") { + withSQLConf("spark.sql.retainGroupColumns" -> "false") { + checkAnswer( + testData2.groupBy($"a", $"b").agg($"*"), + sql("SELECT * FROM testData2 group by a, b")) + } + } + + test("Star Expansion - table with zero column") { + withTempView("temp_table_no_cols") { + val rddNoCols = sparkContext.parallelize(1 to 10).map(_ => Row.empty) + val dfNoCols = spark.createDataFrame(rddNoCols, StructType(Seq.empty)) + dfNoCols.createTempView("temp_table_no_cols") + + // ResolvedStar + checkAnswer( + dfNoCols, + dfNoCols.select(dfNoCols.col("*"))) + + // UnresolvedStar + checkAnswer( + dfNoCols, + sql("SELECT * FROM temp_table_no_cols")) + checkAnswer( + dfNoCols, + dfNoCols.select($"*")) + + var e = intercept[AnalysisException] { + sql("SELECT a.* FROM temp_table_no_cols a") + }.getMessage + assert(e.contains("cannot resolve 'a.*' give input columns ''")) + + e = intercept[AnalysisException] { + dfNoCols.select($"b.*") + }.getMessage + assert(e.contains("cannot resolve 'b.*' give input columns ''")) + } + } + + test("Common subexpression elimination") { + // TODO: support subexpression elimination in whole stage codegen + withSQLConf("spark.sql.codegen.wholeStage" -> "false") { + // select from a table to prevent constant folding. + val df = sql("SELECT a, b from testData2 limit 1") + checkAnswer(df, Row(1, 1)) + + checkAnswer(df.selectExpr("a + 1", "a + 1"), Row(2, 2)) + checkAnswer(df.selectExpr("a + 1", "a + 1 + 1"), Row(2, 3)) + + // This does not work because the expressions get grouped like (a + a) + 1 + checkAnswer(df.selectExpr("a + 1", "a + a + 1"), Row(2, 3)) + checkAnswer(df.selectExpr("a + 1", "a + (a + 1)"), Row(2, 3)) + + // Identity udf that tracks the number of times it is called. + val countAcc = sparkContext.longAccumulator("CallCount") + spark.udf.register("testUdf", (x: Int) => { + countAcc.add(1) + x + }) + + // Evaluates df, verifying it is equal to the expectedResult and the accumulator's value + // is correct. + def verifyCallCount(df: DataFrame, expectedResult: Row, expectedCount: Int): Unit = { + countAcc.setValue(0) + QueryTest.checkAnswer( + df, Seq(expectedResult), checkToRDD = false /* avoid duplicate exec */) + assert(countAcc.value == expectedCount) + } + + verifyCallCount(df.selectExpr("testUdf(a)"), Row(1), 1) + verifyCallCount(df.selectExpr("testUdf(a)", "testUdf(a)"), Row(1, 1), 1) + verifyCallCount(df.selectExpr("testUdf(a + 1)", "testUdf(a + 1)"), Row(2, 2), 1) + verifyCallCount(df.selectExpr("testUdf(a + 1)", "testUdf(a)"), Row(2, 1), 2) + verifyCallCount( + df.selectExpr("testUdf(a + 1) + testUdf(a + 1)", "testUdf(a + 1)"), Row(4, 2), 1) + + verifyCallCount( + df.selectExpr("testUdf(a + 1) + testUdf(1 + b)", "testUdf(a + 1)"), Row(4, 2), 2) + + val testUdf = functions.udf((x: Int) => { + countAcc.add(1) + x + }) + verifyCallCount( + df.groupBy().agg(sum(testUdf($"b") + testUdf($"b") + testUdf($"b"))), Row(3.0), 1) + + verifyCallCount( + df.selectExpr("testUdf(a + 1) + testUdf(1 + a)", "testUdf(a + 1)"), Row(4, 2), 1) + + // Try disabling it via configuration. + spark.conf.set("spark.sql.subexpressionElimination.enabled", "false") + verifyCallCount(df.selectExpr("testUdf(a)", "testUdf(a)"), Row(1, 1), 2) + spark.conf.set("spark.sql.subexpressionElimination.enabled", "true") + verifyCallCount(df.selectExpr("testUdf(a)", "testUdf(a)"), Row(1, 1), 1) + } + } + + test("SPARK-10707: nullability should be correctly propagated through set operations (1)") { + // This test produced an incorrect result of 1 before the SPARK-10707 fix because of the + // NullPropagation rule: COUNT(v) got replaced with COUNT(1) because the output column of + // UNION was incorrectly considered non-nullable: + checkAnswer( + sql("""SELECT count(v) FROM ( + | SELECT v FROM ( + | SELECT 'foo' AS v UNION ALL + | SELECT NULL AS v + | ) my_union WHERE isnull(v) + |) my_subview""".stripMargin), + Seq(Row(0))) + } + + test("SPARK-10707: nullability should be correctly propagated through set operations (2)") { + // This test uses RAND() to stop column pruning for Union and checks the resulting isnull + // value. This would produce an incorrect result before the fix in SPARK-10707 because the "v" + // column of the union was considered non-nullable. + checkAnswer( + sql( + """ + |SELECT a FROM ( + | SELECT ISNULL(v) AS a, RAND() FROM ( + | SELECT 'foo' AS v UNION ALL SELECT null AS v + | ) my_union + |) my_view + """.stripMargin), + Row(false) :: Row(true) :: Nil) + } + + test("filter on a grouping column that is not presented in SELECT") { + checkAnswer( + sql("select count(1) from (select 1 as a) t group by a having a > 0"), + Row(1) :: Nil) + } + + test("SPARK-13056: Null in map value causes NPE") { + val df = Seq(1 -> Map("abc" -> "somestring", "cba" -> null)).toDF("key", "value") + withTempView("maptest") { + df.createOrReplaceTempView("maptest") + // local optimization will by pass codegen code, so we should keep the filter `key=1` + checkAnswer(sql("SELECT value['abc'] FROM maptest where key = 1"), Row("somestring")) + checkAnswer(sql("SELECT value['cba'] FROM maptest where key = 1"), Row(null)) + } + } + + test("hash function") { + val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") + withTempView("tbl") { + df.createOrReplaceTempView("tbl") + checkAnswer( + df.select(hash($"i", $"j")), + sql("SELECT hash(i, j) from tbl") + ) + } + } + + test("join with using clause") { + val df1 = Seq(("r1c1", "r1c2", "t1r1c3"), + ("r2c1", "r2c2", "t1r2c3"), ("r3c1x", "r3c2", "t1r3c3")).toDF("c1", "c2", "c3") + val df2 = Seq(("r1c1", "r1c2", "t2r1c3"), + ("r2c1", "r2c2", "t2r2c3"), ("r3c1y", "r3c2", "t2r3c3")).toDF("c1", "c2", "c3") + val df3 = Seq((null, "r1c2", "t3r1c3"), + ("r2c1", "r2c2", "t3r2c3"), ("r3c1y", "r3c2", "t3r3c3")).toDF("c1", "c2", "c3") + withTempView("t1", "t2", "t3") { + df1.createOrReplaceTempView("t1") + df2.createOrReplaceTempView("t2") + df3.createOrReplaceTempView("t3") + // inner join with one using column + checkAnswer( + sql("SELECT * FROM t1 join t2 using (c1)"), + Row("r1c1", "r1c2", "t1r1c3", "r1c2", "t2r1c3") :: + Row("r2c1", "r2c2", "t1r2c3", "r2c2", "t2r2c3") :: Nil) + + // inner join with two using columns + checkAnswer( + sql("SELECT * FROM t1 join t2 using (c1, c2)"), + Row("r1c1", "r1c2", "t1r1c3", "t2r1c3") :: + Row("r2c1", "r2c2", "t1r2c3", "t2r2c3") :: Nil) + + // Left outer join with one using column. + checkAnswer( + sql("SELECT * FROM t1 left join t2 using (c1)"), + Row("r1c1", "r1c2", "t1r1c3", "r1c2", "t2r1c3") :: + Row("r2c1", "r2c2", "t1r2c3", "r2c2", "t2r2c3") :: + Row("r3c1x", "r3c2", "t1r3c3", null, null) :: Nil) + + // Right outer join with one using column. + checkAnswer( + sql("SELECT * FROM t1 right join t2 using (c1)"), + Row("r1c1", "r1c2", "t1r1c3", "r1c2", "t2r1c3") :: + Row("r2c1", "r2c2", "t1r2c3", "r2c2", "t2r2c3") :: + Row("r3c1y", null, null, "r3c2", "t2r3c3") :: Nil) + + // Full outer join with one using column. + checkAnswer( + sql("SELECT * FROM t1 full outer join t2 using (c1)"), + Row("r1c1", "r1c2", "t1r1c3", "r1c2", "t2r1c3") :: + Row("r2c1", "r2c2", "t1r2c3", "r2c2", "t2r2c3") :: + Row("r3c1x", "r3c2", "t1r3c3", null, null) :: + Row("r3c1y", null, + null, "r3c2", "t2r3c3") :: Nil) + + // Full outer join with null value in join column. + checkAnswer( + sql("SELECT * FROM t1 full outer join t3 using (c1)"), + Row("r1c1", "r1c2", "t1r1c3", null, null) :: + Row("r2c1", "r2c2", "t1r2c3", "r2c2", "t3r2c3") :: + Row("r3c1x", "r3c2", "t1r3c3", null, null) :: + Row("r3c1y", null, null, "r3c2", "t3r3c3") :: + Row(null, null, null, "r1c2", "t3r1c3") :: Nil) + + // Self join with using columns. + checkAnswer( + sql("SELECT * FROM t1 join t1 using (c1)"), + Row("r1c1", "r1c2", "t1r1c3", "r1c2", "t1r1c3") :: + Row("r2c1", "r2c2", "t1r2c3", "r2c2", "t1r2c3") :: + Row("r3c1x", "r3c2", "t1r3c3", "r3c2", "t1r3c3") :: Nil) + } + } + + test("SPARK-15327: fail to compile generated code with complex data structure") { + withTempDir{ dir => + val json = + """ + |{"h": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "d": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"e": "testing", "count": 3}], "b": [{"e": "test", "count": 1}]}}, + |"c": {"b": {"c": [{"e": "adfgd"}], "a": [{"count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "a": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"count": 3}], "b": [{"e": "test", "count": 1}]}}, + |"e": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "g": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"e": "testing", "count": 3}], "b": [{"e": "test", "count": 1}]}}, + |"f": {"b": {"c": [{"e": "adfgd"}], "a": [{"e": "testing", "count": 3}], + |"b": [{"e": "test", "count": 1}]}}, "b": {"b": {"c": [{"e": "adfgd"}], + |"a": [{"count": 3}], "b": [{"e": "test", "count": 1}]}}}' + | + """.stripMargin + spark.read.json(Seq(json).toDS()).write.mode("overwrite").parquet(dir.toString) + spark.read.parquet(dir.toString).collect() + } + } + + test("data source table created in InMemoryCatalog should be able to read/write") { + withTable("tbl") { + sql("CREATE TABLE tbl(i INT, j STRING) USING parquet") + checkAnswer(sql("SELECT i, j FROM tbl"), Nil) + + Seq(1 -> "a", 2 -> "b").toDF("i", "j").write.mode("overwrite").insertInto("tbl") + checkAnswer(sql("SELECT i, j FROM tbl"), Row(1, "a") :: Row(2, "b") :: Nil) + + Seq(3 -> "c", 4 -> "d").toDF("i", "j").write.mode("append").saveAsTable("tbl") + checkAnswer( + sql("SELECT i, j FROM tbl"), + Row(1, "a") :: Row(2, "b") :: Row(3, "c") :: Row(4, "d") :: Nil) + } + } + + test("Eliminate noop ordinal ORDER BY") { + withSQLConf(SQLConf.ORDER_BY_ORDINAL.key -> "true") { + val plan1 = sql("SELECT 1.0, 'abc', year(current_date()) ORDER BY 1, 2, 3") + val plan2 = sql("SELECT 1.0, 'abc', year(current_date())") + comparePlans(plan1.queryExecution.optimizedPlan, plan2.queryExecution.optimizedPlan) + } + } + + test("check code injection is prevented") { + // The end of comment (*/) should be escaped. + var literal = + """|*/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + var expected = + """|*/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + // `\u002A` is `*` and `\u002F` is `/` + // so if the end of comment consists of those characters in queries, we need to escape them. + literal = + """|\\u002A/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\u002A/"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\\\u002A/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|\\u002A/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\u002a/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\u002a/"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\\\u002a/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|\\u002a/ + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|*\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"*\\u002F"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|*\\\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|*\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|*\\u002f + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"*\\u002f"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|*\\\\u002f + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|*\\u002f + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\u002A\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\u002A\\u002F"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\\\u002A\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\\\u002A\\u002F"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\u002A\\\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + s"""|${"\\u002A\\\\u002F"} + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + + literal = + """|\\\\u002A\\\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + expected = + """|\\u002A\\u002F + |{ + | new Object() { + | void f() { throw new RuntimeException("This exception is injected."); } + | }.f(); + |} + |/*""".stripMargin + checkAnswer( + sql(s"SELECT '$literal' AS DUMMY"), + Row(s"$expected") :: Nil) + } + + test("SPARK-15752 optimize metadata only query for datasource table") { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + withTable("srcpart_15752") { + val data = (1 to 10).map(i => (i, s"data-$i", i % 2, if ((i % 2) == 0) "a" else "b")) + .toDF("col1", "col2", "partcol1", "partcol2") + data.write.partitionBy("partcol1", "partcol2").mode("append").saveAsTable("srcpart_15752") + checkAnswer( + sql("select partcol1 from srcpart_15752 group by partcol1"), + Row(0) :: Row(1) :: Nil) + checkAnswer( + sql("select partcol1 from srcpart_15752 where partcol1 = 1 group by partcol1"), + Row(1)) + checkAnswer( + sql("select partcol1, count(distinct partcol2) from srcpart_15752 group by partcol1"), + Row(0, 1) :: Row(1, 1) :: Nil) + checkAnswer( + sql("select partcol1, count(distinct partcol2) from srcpart_15752 where partcol1 = 1 " + + "group by partcol1"), + Row(1, 1) :: Nil) + checkAnswer(sql("select distinct partcol1 from srcpart_15752"), Row(0) :: Row(1) :: Nil) + checkAnswer(sql("select distinct partcol1 from srcpart_15752 where partcol1 = 1"), Row(1)) + checkAnswer( + sql("select distinct col from (select partcol1 + 1 as col from srcpart_15752 " + + "where partcol1 = 1) t"), + Row(2)) + checkAnswer(sql("select max(partcol1) from srcpart_15752"), Row(1)) + checkAnswer(sql("select max(partcol1) from srcpart_15752 where partcol1 = 1"), Row(1)) + checkAnswer(sql("select max(partcol1) from (select partcol1 from srcpart_15752) t"), Row(1)) + checkAnswer( + sql("select max(col) from (select partcol1 + 1 as col from srcpart_15752 " + + "where partcol1 = 1) t"), + Row(2)) + } + } + } + + test("SPARK-16975: Column-partition path starting '_' should be handled correctly") { + withTempDir { dir => + val parquetDir = new File(dir, "parquet").getCanonicalPath + spark.range(10).withColumn("_col", $"id").write.partitionBy("_col").save(parquetDir) + spark.read.parquet(parquetDir) + } + } + + test("SPARK-16644: Aggregate should not put aggregate expressions to constraints") { + withTable("tbl") { + sql("CREATE TABLE tbl(a INT, b INT) USING parquet") + checkAnswer(sql( + """ + |SELECT + | a, + | MAX(b) AS c1, + | b AS c2 + |FROM tbl + |WHERE a = b + |GROUP BY a, b + |HAVING c1 = 1 + """.stripMargin), Nil) + } + } + + test("SPARK-16674: field names containing dots for both fields and partitioned fields") { + withTempPath { path => + val data = (1 to 10).map(i => (i, s"data-$i", i % 2, if ((i % 2) == 0) "a" else "b")) + .toDF("col.1", "col.2", "part.col1", "part.col2") + data.write + .format("parquet") + .partitionBy("part.col1", "part.col2") + .save(path.getCanonicalPath) + val readBack = spark.read.format("parquet").load(path.getCanonicalPath) + checkAnswer( + readBack.selectExpr("`part.col1`", "`col.1`"), + data.selectExpr("`part.col1`", "`col.1`")) + } + } + + test("SPARK-17515: CollectLimit.execute() should perform per-partition limits") { + val numRecordsRead = spark.sparkContext.longAccumulator + spark.range(1, 100, 1, numPartitions = 10).map { x => + numRecordsRead.add(1) + x + }.limit(1).queryExecution.toRdd.count() + assert(numRecordsRead.value === 10) + } + + test("CREATE TABLE USING should not fail if a same-name temp view exists") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + sql("CREATE TABLE same_name(i int) USING json") + checkAnswer(spark.table("same_name"), spark.range(10).toDF()) + assert(spark.table("default.same_name").collect().isEmpty) + } + } + } + + test("SPARK-18053: ARRAY equality is broken") { + withTable("array_tbl") { + spark.range(10).select(array($"id").as("arr")).write.saveAsTable("array_tbl") + assert(sql("SELECT * FROM array_tbl where arr = ARRAY(1L)").count == 1) + } + } + + test("SPARK-19157: should be able to change spark.sql.runSQLOnFiles at runtime") { + withTempPath { path => + Seq(1 -> "a").toDF("i", "j").write.parquet(path.getCanonicalPath) + + val newSession = spark.newSession() + val originalValue = newSession.sessionState.conf.runSQLonFile + + try { + newSession.sessionState.conf.setConf(SQLConf.RUN_SQL_ON_FILES, false) + intercept[AnalysisException] { + newSession.sql(s"SELECT i, j FROM parquet.`${path.getCanonicalPath}`") + } + + newSession.sessionState.conf.setConf(SQLConf.RUN_SQL_ON_FILES, true) + checkAnswer( + newSession.sql(s"SELECT i, j FROM parquet.`${path.getCanonicalPath}`"), + Row(1, "a")) + } finally { + newSession.sessionState.conf.setConf(SQLConf.RUN_SQL_ON_FILES, originalValue) + } + } + } + + test("should be able to resolve a persistent view") { + withTable("t1", "t2") { + withView("v1") { + sql("CREATE TABLE `t1` USING parquet AS SELECT * FROM VALUES(1, 1) AS t1(a, b)") + sql("CREATE TABLE `t2` USING parquet AS SELECT * FROM VALUES('a', 2, 1.0) AS t2(d, e, f)") + sql("CREATE VIEW `v1`(x, y) AS SELECT * FROM t1") + checkAnswer(spark.table("v1").orderBy("x"), Row(1, 1)) + + sql("ALTER VIEW `v1` AS SELECT * FROM t2") + checkAnswer(spark.table("v1").orderBy("f"), Row("a", 2, 1.0)) + } + } + } + + test("SPARK-19059: read file based table whose name starts with underscore") { + withTable("_tbl") { + sql("CREATE TABLE `_tbl`(i INT) USING parquet") + sql("INSERT INTO `_tbl` VALUES (1), (2), (3)") + checkAnswer( sql("SELECT * FROM `_tbl`"), Row(1) :: Row(2) :: Row(3) :: Nil) + } + } + + test("SPARK-19334: check code injection is prevented") { + // The end of comment (*/) should be escaped. + val badQuery = + """|SELECT inline(array(cast(struct(1) AS + | struct<`= + | new Object() { + | {f();} + | public void f() {throw new RuntimeException("This exception is injected.");} + | public int x; + | }.x + | `:int>)))""".stripMargin.replaceAll("\n", "") + + checkAnswer(sql(badQuery), Row(1) :: Nil) + } + + test("SPARK-19650: An action on a Command should not trigger a Spark job") { + // Create a listener that checks if new jobs have started. + val jobStarted = new AtomicBoolean(false) + val listener = new SparkListener { + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + jobStarted.set(true) + } + } + + // Make sure no spurious job starts are pending in the listener bus. + sparkContext.listenerBus.waitUntilEmpty(500) + sparkContext.addSparkListener(listener) + try { + // Execute the command. + sql("show databases").head() + + // Make sure we have seen all events triggered by DataFrame.show() + sparkContext.listenerBus.waitUntilEmpty(500) + } finally { + sparkContext.removeSparkListener(listener) + } + assert(!jobStarted.get(), "Command should not trigger a Spark job.") + } + + test("SPARK-20164: AnalysisException should be tolerant to null query plan") { + try { + throw new AnalysisException("", None, None, plan = null) + } catch { + case ae: AnalysisException => assert(ae.plan == null && ae.getMessage == ae.getSimpleMessage) + } + } + + test("SPARK-12868: Allow adding jars from hdfs ") { + val jarFromHdfs = "hdfs://doesnotmatter/test.jar" + val jarFromInvalidFs = "fffs://doesnotmatter/test.jar" + + // if 'hdfs' is not supported, MalformedURLException will be thrown + new URL(jarFromHdfs) + + intercept[MalformedURLException] { + new URL(jarFromInvalidFs) + } + } + + test("RuntimeReplaceable functions should not take extra parameters") { + val e = intercept[AnalysisException](sql("SELECT nvl(1, 2, 3)")) + assert(e.message.contains("Invalid number of arguments")) + } + + test("SPARK-21228: InSet incorrect handling of structs") { + withTempView("A") { + // reduce this from the default of 10 so the repro query text is not too long + withSQLConf((SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> "3")) { + // a relation that has 1 column of struct type with values (1,1), ..., (9, 9) + spark.range(1, 10).selectExpr("named_struct('a', id, 'b', id) as a") + .createOrReplaceTempView("A") + val df = sql( + """ + |SELECT * from + | (SELECT MIN(a) as minA FROM A) AA -- this Aggregate will return UnsafeRows + | -- the IN will become InSet with a Set of GenericInternalRows + | -- a GenericInternalRow is never equal to an UnsafeRow so the query would + | -- returns 0 results, which is incorrect + | WHERE minA IN (NAMED_STRUCT('a', 1L, 'b', 1L), NAMED_STRUCT('a', 2L, 'b', 2L), + | NAMED_STRUCT('a', 3L, 'b', 3L)) + """.stripMargin) + checkAnswer(df, Row(Row(1, 1))) + } + } + } + + test("SPARK-21335: support un-aliased subquery") { + withTempView("v") { + Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("v") + checkAnswer(sql("SELECT i from (SELECT i FROM v)"), Row(1)) + + val e = intercept[AnalysisException](sql("SELECT v.i from (SELECT i FROM v)")) + assert(e.message == + "cannot resolve '`v.i`' given input columns: [__auto_generated_subquery_name.i]") + + checkAnswer(sql("SELECT __auto_generated_subquery_name.i from (SELECT i FROM v)"), Row(1)) + } + } + + test("SPARK-21743: top-most limit should not cause memory leak") { + // In unit test, Spark will fail the query if memory leak detected. + spark.range(100).groupBy("id").count().limit(1).collect() + } + + test("SPARK-21652: rule confliction of InferFiltersFromConstraints and ConstantPropagation") { + withTempView("t1", "t2") { + Seq((1, 1)).toDF("col1", "col2").createOrReplaceTempView("t1") + Seq(1, 2).toDF("col").createOrReplaceTempView("t2") + val df = sql( + """ + |SELECT * + |FROM t1, t2 + |WHERE t1.col1 = 1 AND 1 = t1.col2 AND t1.col1 = t2.col AND t1.col2 = t2.col + """.stripMargin) + checkAnswer(df, Row(1, 1, 1)) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala new file mode 100644 index 000000000000..e3901af4b998 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -0,0 +1,308 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.io.File +import java.util.{Locale, TimeZone} + +import scala.util.control.NonFatal + +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile} +import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeTableCommand} +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType + +/** + * End-to-end test cases for SQL queries. + * + * Each case is loaded from a file in "spark/sql/core/src/test/resources/sql-tests/inputs". + * Each case has a golden result file in "spark/sql/core/src/test/resources/sql-tests/results". + * + * To run the entire test suite: + * {{{ + * build/sbt "sql/test-only *SQLQueryTestSuite" + * }}} + * + * To run a single test file upon change: + * {{{ + * build/sbt "~sql/test-only *SQLQueryTestSuite -- -z inline-table.sql" + * }}} + * + * To re-generate golden files, run: + * {{{ + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *SQLQueryTestSuite" + * }}} + * + * The format for input files is simple: + * 1. A list of SQL queries separated by semicolon. + * 2. Lines starting with -- are treated as comments and ignored. + * + * For example: + * {{{ + * -- this is a comment + * select 1, -1; + * select current_date; + * }}} + * + * The format for golden result files look roughly like: + * {{{ + * -- some header information + * + * -- !query 0 + * select 1, -1 + * -- !query 0 schema + * struct<...schema...> + * -- !query 0 output + * ... data row 1 ... + * ... data row 2 ... + * ... + * + * -- !query 1 + * ... + * }}} + */ +class SQLQueryTestSuite extends QueryTest with SharedSQLContext { + + private val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" + + private val baseResourcePath = { + // If regenerateGoldenFiles is true, we must be running this in SBT and we use hard-coded + // relative path. Otherwise, we use classloader's getResource to find the location. + if (regenerateGoldenFiles) { + java.nio.file.Paths.get("src", "test", "resources", "sql-tests").toFile + } else { + val res = getClass.getClassLoader.getResource("sql-tests") + new File(res.getFile) + } + } + + private val inputFilePath = new File(baseResourcePath, "inputs").getAbsolutePath + private val goldenFilePath = new File(baseResourcePath, "results").getAbsolutePath + + /** List of test cases to ignore, in lower cases. */ + private val blackList = Set( + "blacklist.sql", // Do NOT remove this one. It is here to test the blacklist functionality. + ".DS_Store" // A meta-file that may be created on Mac by Finder App. + // We should ignore this file from processing. + ) + + // Create all the test cases. + listTestCases().foreach(createScalaTestCase) + + /** A test case. */ + private case class TestCase(name: String, inputFile: String, resultFile: String) + + /** A single SQL query's output. */ + private case class QueryOutput(sql: String, schema: String, output: String) { + def toString(queryIndex: Int): String = { + // We are explicitly not using multi-line string due to stripMargin removing "|" in output. + s"-- !query $queryIndex\n" + + sql + "\n" + + s"-- !query $queryIndex schema\n" + + schema + "\n" + + s"-- !query $queryIndex output\n" + + output + } + } + + private def createScalaTestCase(testCase: TestCase): Unit = { + if (blackList.exists(t => + testCase.name.toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT)))) { + // Create a test case to ignore this case. + ignore(testCase.name) { /* Do nothing */ } + } else { + // Create a test case to run this case. + test(testCase.name) { runTest(testCase) } + } + } + + /** Run a test case. */ + private def runTest(testCase: TestCase): Unit = { + val input = fileToString(new File(testCase.inputFile)) + + // List of SQL queries to run + val queries: Seq[String] = { + val cleaned = input.split("\n").filterNot(_.startsWith("--")).mkString("\n") + // note: this is not a robust way to split queries using semicolon, but works for now. + cleaned.split("(?<=[^\\\\]);").map(_.trim).filter(_ != "").toSeq + } + + // Create a local SparkSession to have stronger isolation between different test cases. + // This does not isolate catalog changes. + val localSparkSession = spark.newSession() + loadTestData(localSparkSession) + + // Run the SQL queries preparing them for comparison. + val outputs: Seq[QueryOutput] = queries.map { sql => + val (schema, output) = getNormalizedResult(localSparkSession, sql) + // We might need to do some query canonicalization in the future. + QueryOutput( + sql = sql, + schema = schema.catalogString, + output = output.mkString("\n").trim) + } + + if (regenerateGoldenFiles) { + // Again, we are explicitly not using multi-line string due to stripMargin removing "|". + val goldenOutput = { + s"-- Automatically generated by ${getClass.getSimpleName}\n" + + s"-- Number of queries: ${outputs.size}\n\n\n" + + outputs.zipWithIndex.map{case (qr, i) => qr.toString(i)}.mkString("\n\n\n") + "\n" + } + val resultFile = new File(testCase.resultFile) + val parent = resultFile.getParentFile + if (!parent.exists()) { + assert(parent.mkdirs(), "Could not create directory: " + parent) + } + stringToFile(resultFile, goldenOutput) + } + + // Read back the golden file. + val expectedOutputs: Seq[QueryOutput] = { + val goldenOutput = fileToString(new File(testCase.resultFile)) + val segments = goldenOutput.split("-- !query.+\n") + + // each query has 3 segments, plus the header + assert(segments.size == outputs.size * 3 + 1, + s"Expected ${outputs.size * 3 + 1} blocks in result file but got ${segments.size}. " + + s"Try regenerate the result files.") + Seq.tabulate(outputs.size) { i => + QueryOutput( + sql = segments(i * 3 + 1).trim, + schema = segments(i * 3 + 2).trim, + output = segments(i * 3 + 3).trim + ) + } + } + + // Compare results. + assertResult(expectedOutputs.size, s"Number of queries should be ${expectedOutputs.size}") { + outputs.size + } + + outputs.zip(expectedOutputs).zipWithIndex.foreach { case ((output, expected), i) => + assertResult(expected.sql, s"SQL query did not match for query #$i\n${expected.sql}") { + output.sql + } + assertResult(expected.schema, s"Schema did not match for query #$i\n${expected.sql}") { + output.schema + } + assertResult(expected.output, s"Result did not match for query #$i\n${expected.sql}") { + output.output + } + } + } + + /** Executes a query and returns the result as (schema of the output, normalized output). */ + private def getNormalizedResult(session: SparkSession, sql: String): (StructType, Seq[String]) = { + // Returns true if the plan is supposed to be sorted. + def isSorted(plan: LogicalPlan): Boolean = plan match { + case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false + case _: DescribeTableCommand | _: DescribeColumnCommand => true + case PhysicalOperation(_, _, Sort(_, true, _)) => true + case _ => plan.children.iterator.exists(isSorted) + } + + try { + val df = session.sql(sql) + val schema = df.schema + val notIncludedMsg = "[not included in comparison]" + // Get answer, but also get rid of the #1234 expression ids that show up in explain plans + val answer = df.queryExecution.hiveResultString().map(_.replaceAll("#\\d+", "#x") + .replaceAll("Location.*/sql/core/", s"Location ${notIncludedMsg}sql/core/") + .replaceAll("Created By.*", s"Created By $notIncludedMsg") + .replaceAll("Created Time.*", s"Created Time $notIncludedMsg") + .replaceAll("Last Access.*", s"Last Access $notIncludedMsg")) + + // If the output is not pre-sorted, sort it. + if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted) + + } catch { + case a: AnalysisException => + // Do not output the logical plan tree which contains expression IDs. + // Also implement a crude way of masking expression IDs in the error message + // with a generic pattern "###". + val msg = if (a.plan.nonEmpty) a.getSimpleMessage else a.getMessage + (StructType(Seq.empty), Seq(a.getClass.getName, msg.replaceAll("#\\d+", "#x"))) + case NonFatal(e) => + // If there is an exception, put the exception class followed by the message. + (StructType(Seq.empty), Seq(e.getClass.getName, e.getMessage)) + } + } + + private def listTestCases(): Seq[TestCase] = { + listFilesRecursively(new File(inputFilePath)).map { file => + val resultFile = file.getAbsolutePath.replace(inputFilePath, goldenFilePath) + ".out" + val absPath = file.getAbsolutePath + val testCaseName = absPath.stripPrefix(inputFilePath).stripPrefix(File.separator) + TestCase(testCaseName, absPath, resultFile) + } + } + + /** Returns all the files (not directories) in a directory, recursively. */ + private def listFilesRecursively(path: File): Seq[File] = { + val (dirs, files) = path.listFiles().partition(_.isDirectory) + files ++ dirs.flatMap(listFilesRecursively) + } + + /** Load built-in test tables into the SparkSession. */ + private def loadTestData(session: SparkSession): Unit = { + import session.implicits._ + + (1 to 100).map(i => (i, i.toString)).toDF("key", "value").createOrReplaceTempView("testdata") + + ((Seq(1, 2, 3), Seq(Seq(1, 2, 3))) :: (Seq(2, 3, 4), Seq(Seq(2, 3, 4))) :: Nil) + .toDF("arraycol", "nestedarraycol") + .createOrReplaceTempView("arraydata") + + (Tuple1(Map(1 -> "a1", 2 -> "b1", 3 -> "c1", 4 -> "d1", 5 -> "e1")) :: + Tuple1(Map(1 -> "a2", 2 -> "b2", 3 -> "c2", 4 -> "d2")) :: + Tuple1(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: + Tuple1(Map(1 -> "a4", 2 -> "b4")) :: + Tuple1(Map(1 -> "a5")) :: Nil) + .toDF("mapcol") + .createOrReplaceTempView("mapdata") + } + + private val originalTimeZone = TimeZone.getDefault + private val originalLocale = Locale.getDefault + + override def beforeAll(): Unit = { + super.beforeAll() + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + RuleExecutor.resetTime() + } + + override def afterAll(): Unit = { + try { + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + + // For debugging dump some statistics about how much time was spent in various optimizer rules + logWarning(RuleExecutor.dumpTimeSpent()) + } finally { + super.afterAll() + } + } +} 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 295f02f9a7b5..c9bd05d0e4e3 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 @@ -34,7 +34,9 @@ case class ReflectData( decimalField: java.math.BigDecimal, date: Date, timestampField: Timestamp, - seqInt: Seq[Int]) + seqInt: Seq[Int], + javaBigInt: java.math.BigInteger, + scalaBigInt: scala.math.BigInt) case class NullReflectData( intField: java.lang.Integer, @@ -77,18 +79,20 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, - new java.math.BigDecimal(1), Date.valueOf("1970-01-01"), new Timestamp(12345), Seq(1, 2, 3)) - Seq(data).toDF().registerTempTable("reflectData") + new java.math.BigDecimal(1), Date.valueOf("1970-01-01"), new Timestamp(12345), Seq(1, 2, 3), + new java.math.BigInteger("1"), scala.math.BigInt(1)) + Seq(data).toDF().createOrReplaceTempView("reflectData") assert(sql("SELECT * FROM reflectData").collect().head === Row("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, new java.math.BigDecimal(1), Date.valueOf("1970-01-01"), - new Timestamp(12345), Seq(1, 2, 3))) + new Timestamp(12345), Seq(1, 2, 3), new java.math.BigDecimal(1), + new java.math.BigDecimal(1))) } test("query case class RDD with nulls") { val data = NullReflectData(null, null, null, null, null, null, null) - Seq(data).toDF().registerTempTable("reflectNullData") + Seq(data).toDF().createOrReplaceTempView("reflectNullData") assert(sql("SELECT * FROM reflectNullData").collect().head === Row.fromSeq(Seq.fill(7)(null))) @@ -96,7 +100,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { test("query case class RDD with Nones") { val data = OptionalReflectData(None, None, None, None, None, None, None) - Seq(data).toDF().registerTempTable("reflectOptionalData") + Seq(data).toDF().createOrReplaceTempView("reflectOptionalData") assert(sql("SELECT * FROM reflectOptionalData").collect().head === Row.fromSeq(Seq.fill(7)(null))) @@ -104,7 +108,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { // Equality is broken for Arrays, so we test that separately. test("query binary data") { - Seq(ReflectBinary(Array[Byte](1))).toDF().registerTempTable("reflectBinary") + Seq(ReflectBinary(Array[Byte](1))).toDF().createOrReplaceTempView("reflectBinary") val result = sql("SELECT data FROM reflectBinary") .collect().head(0).asInstanceOf[Array[Byte]] @@ -124,7 +128,7 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext { Map(10 -> Some(100L), 20 -> Some(200L), 30 -> None), Nested(None, "abc"))) - Seq(data).toDF().registerTempTable("reflectComplexData") + Seq(data).toDF().createOrReplaceTempView("reflectComplexData") assert(sql("SELECT * FROM reflectComplexData").collect().head === Row( Seq(1, 2, 3), 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 index ddab91862964..cd6b2647e0be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.test.SharedSQLContext class SerializationSuite extends SparkFunSuite with SharedSQLContext { test("[SPARK-5235] SQLContext should be serializable") { - val _sqlContext = new SQLContext(sparkContext) - new JavaSerializer(new SparkConf()).newInstance().serialize(_sqlContext) + val spark = SparkSession.builder.getOrCreate() + new JavaSerializer(new SparkConf()).newInstance().serialize(spark.sqlContext) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala new file mode 100644 index 000000000000..c01666770720 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala @@ -0,0 +1,216 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.BeforeAndAfterAll +import org.scalatest.BeforeAndAfterEach +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.util.QueryExecutionListener + +class SessionStateSuite extends SparkFunSuite + with BeforeAndAfterEach with BeforeAndAfterAll { + + /** + * A shared SparkSession for all tests in this suite. Make sure you reset any changes to this + * session as this is a singleton HiveSparkSession in HiveSessionStateSuite and it's shared + * with all Hive test suites. + */ + protected var activeSession: SparkSession = _ + + override def beforeAll(): Unit = { + activeSession = SparkSession.builder().master("local").getOrCreate() + } + + override def afterAll(): Unit = { + if (activeSession != null) { + activeSession.stop() + activeSession = null + } + super.afterAll() + } + + test("fork new session and inherit RuntimeConfig options") { + val key = "spark-config-clone" + try { + activeSession.conf.set(key, "active") + + // inheritance + val forkedSession = activeSession.cloneSession() + assert(forkedSession ne activeSession) + assert(forkedSession.conf ne activeSession.conf) + assert(forkedSession.conf.get(key) == "active") + + // independence + forkedSession.conf.set(key, "forked") + assert(activeSession.conf.get(key) == "active") + activeSession.conf.set(key, "dontcopyme") + assert(forkedSession.conf.get(key) == "forked") + } finally { + activeSession.conf.unset(key) + } + } + + test("fork new session and inherit function registry and udf") { + val testFuncName1 = FunctionIdentifier("strlenScala") + val testFuncName2 = FunctionIdentifier("addone") + try { + activeSession.udf.register(testFuncName1.funcName, (_: String).length + (_: Int)) + val forkedSession = activeSession.cloneSession() + + // inheritance + assert(forkedSession ne activeSession) + assert(forkedSession.sessionState.functionRegistry ne + activeSession.sessionState.functionRegistry) + assert(forkedSession.sessionState.functionRegistry.lookupFunction(testFuncName1).nonEmpty) + + // independence + forkedSession.sessionState.functionRegistry.dropFunction(testFuncName1) + assert(activeSession.sessionState.functionRegistry.lookupFunction(testFuncName1).nonEmpty) + activeSession.udf.register(testFuncName2.funcName, (_: Int) + 1) + assert(forkedSession.sessionState.functionRegistry.lookupFunction(testFuncName2).isEmpty) + } finally { + activeSession.sessionState.functionRegistry.dropFunction(testFuncName1) + activeSession.sessionState.functionRegistry.dropFunction(testFuncName2) + } + } + + test("fork new session and inherit experimental methods") { + val originalExtraOptimizations = activeSession.experimental.extraOptimizations + val originalExtraStrategies = activeSession.experimental.extraStrategies + try { + object DummyRule1 extends Rule[LogicalPlan] { + def apply(p: LogicalPlan): LogicalPlan = p + } + object DummyRule2 extends Rule[LogicalPlan] { + def apply(p: LogicalPlan): LogicalPlan = p + } + val optimizations = List(DummyRule1, DummyRule2) + activeSession.experimental.extraOptimizations = optimizations + val forkedSession = activeSession.cloneSession() + + // inheritance + assert(forkedSession ne activeSession) + assert(forkedSession.experimental ne activeSession.experimental) + assert(forkedSession.experimental.extraOptimizations.toSet == + activeSession.experimental.extraOptimizations.toSet) + + // independence + forkedSession.experimental.extraOptimizations = List(DummyRule2) + assert(activeSession.experimental.extraOptimizations == optimizations) + activeSession.experimental.extraOptimizations = List(DummyRule1) + assert(forkedSession.experimental.extraOptimizations == List(DummyRule2)) + } finally { + activeSession.experimental.extraOptimizations = originalExtraOptimizations + activeSession.experimental.extraStrategies = originalExtraStrategies + } + } + + test("fork new session and inherit listener manager") { + class CommandCollector extends QueryExecutionListener { + val commands: ArrayBuffer[String] = ArrayBuffer.empty[String] + override def onFailure(funcName: String, qe: QueryExecution, ex: Exception) : Unit = {} + override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { + commands += funcName + } + } + val collectorA = new CommandCollector + val collectorB = new CommandCollector + val collectorC = new CommandCollector + + try { + def runCollectQueryOn(sparkSession: SparkSession): Unit = { + val tupleEncoder = Encoders.tuple(Encoders.scalaInt, Encoders.STRING) + val df = sparkSession.createDataset(Seq(1 -> "a"))(tupleEncoder).toDF("i", "j") + df.select("i").collect() + } + + activeSession.listenerManager.register(collectorA) + val forkedSession = activeSession.cloneSession() + + // inheritance + assert(forkedSession ne activeSession) + assert(forkedSession.listenerManager ne activeSession.listenerManager) + runCollectQueryOn(forkedSession) + assert(collectorA.commands.length == 1) // forked should callback to A + assert(collectorA.commands(0) == "collect") + + // independence + // => changes to forked do not affect original + forkedSession.listenerManager.register(collectorB) + runCollectQueryOn(activeSession) + assert(collectorB.commands.isEmpty) // original should not callback to B + assert(collectorA.commands.length == 2) // original should still callback to A + assert(collectorA.commands(1) == "collect") + // <= changes to original do not affect forked + activeSession.listenerManager.register(collectorC) + runCollectQueryOn(forkedSession) + assert(collectorC.commands.isEmpty) // forked should not callback to C + assert(collectorA.commands.length == 3) // forked should still callback to A + assert(collectorB.commands.length == 1) // forked should still callback to B + assert(collectorA.commands(2) == "collect") + assert(collectorB.commands(0) == "collect") + } finally { + activeSession.listenerManager.unregister(collectorA) + activeSession.listenerManager.unregister(collectorC) + } + } + + test("fork new sessions and run query on inherited table") { + def checkTableExists(sparkSession: SparkSession): Unit = { + QueryTest.checkAnswer(sparkSession.sql( + """ + |SELECT x.str, COUNT(*) + |FROM df x JOIN df y ON x.str = y.str + |GROUP BY x.str + """.stripMargin), + Row("1", 1) :: Row("2", 1) :: Row("3", 1) :: Nil) + } + + val spark = activeSession + // Cannot use `import activeSession.implicits._` due to the compiler limitation. + import spark.implicits._ + + try { + activeSession + .createDataset[(Int, String)](Seq(1, 2, 3).map(i => (i, i.toString))) + .toDF("int", "str") + .createOrReplaceTempView("df") + checkTableExists(activeSession) + + val forkedSession = activeSession.cloneSession() + assert(forkedSession ne activeSession) + assert(forkedSession.sessionState ne activeSession.sessionState) + checkTableExists(forkedSession) + checkTableExists(activeSession.cloneSession()) // ability to clone multiple times + checkTableExists(forkedSession.cloneSession()) // clone of clone + } finally { + activeSession.sql("drop table df") + } + } + + test("fork new session and inherit reference to SharedState") { + val forkedSession = activeSession.cloneSession() + assert(activeSession.sharedState eq forkedSession.sharedState) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala new file mode 100644 index 000000000000..c0301f2ce2d6 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.sql.internal.SQLConf + +/** + * Test cases for the builder pattern of [[SparkSession]]. + */ +class SparkSessionBuilderSuite extends SparkFunSuite with BeforeAndAfterEach { + + override def afterEach(): Unit = { + // This suite should not interfere with the other test suites. + SparkSession.getActiveSession.foreach(_.stop()) + SparkSession.clearActiveSession() + SparkSession.getDefaultSession.foreach(_.stop()) + SparkSession.clearDefaultSession() + } + + test("create with config options and propagate them to SparkContext and SparkSession") { + val session = SparkSession.builder() + .master("local") + .config("spark.ui.enabled", value = false) + .config("some-config", "v2") + .getOrCreate() + assert(session.sparkContext.conf.get("some-config") == "v2") + assert(session.conf.get("some-config") == "v2") + } + + test("use global default session") { + val session = SparkSession.builder().master("local").getOrCreate() + assert(SparkSession.builder().getOrCreate() == session) + } + + test("config options are propagated to existing SparkSession") { + val session1 = SparkSession.builder().master("local").config("spark-config1", "a").getOrCreate() + assert(session1.conf.get("spark-config1") == "a") + val session2 = SparkSession.builder().config("spark-config1", "b").getOrCreate() + assert(session1 == session2) + assert(session1.conf.get("spark-config1") == "b") + } + + test("use session from active thread session and propagate config options") { + val defaultSession = SparkSession.builder().master("local").getOrCreate() + val activeSession = defaultSession.newSession() + SparkSession.setActiveSession(activeSession) + val session = SparkSession.builder().config("spark-config2", "a").getOrCreate() + + assert(activeSession != defaultSession) + assert(session == activeSession) + assert(session.conf.get("spark-config2") == "a") + assert(session.sessionState.conf == SQLConf.get) + assert(SQLConf.get.getConfString("spark-config2") == "a") + SparkSession.clearActiveSession() + + assert(SparkSession.builder().getOrCreate() == defaultSession) + } + + test("create a new session if the default session has been stopped") { + val defaultSession = SparkSession.builder().master("local").getOrCreate() + SparkSession.setDefaultSession(defaultSession) + defaultSession.stop() + val newSession = SparkSession.builder().master("local").getOrCreate() + assert(newSession != defaultSession) + } + + test("create a new session if the active thread session has been stopped") { + val activeSession = SparkSession.builder().master("local").getOrCreate() + SparkSession.setActiveSession(activeSession) + activeSession.stop() + val newSession = SparkSession.builder().master("local").getOrCreate() + assert(newSession != activeSession) + } + + test("create SparkContext first then SparkSession") { + val conf = new SparkConf().setAppName("test").setMaster("local").set("key1", "value1") + val sparkContext2 = new SparkContext(conf) + val session = SparkSession.builder().config("key2", "value2").getOrCreate() + assert(session.conf.get("key1") == "value1") + assert(session.conf.get("key2") == "value2") + assert(session.sparkContext == sparkContext2) + // We won't update conf for existing `SparkContext` + assert(!sparkContext2.conf.contains("key2")) + assert(sparkContext2.conf.get("key1") == "value1") + } + + test("create SparkContext first then pass context to SparkSession") { + val conf = new SparkConf().setAppName("test").setMaster("local").set("key1", "value1") + val newSC = new SparkContext(conf) + val session = SparkSession.builder().sparkContext(newSC).config("key2", "value2").getOrCreate() + assert(session.conf.get("key1") == "value1") + assert(session.conf.get("key2") == "value2") + assert(session.sparkContext == newSC) + assert(session.sparkContext.conf.get("key1") == "value1") + // If the created sparkContext is passed through the Builder's API sparkContext, + // the conf of this sparkContext will not contain the conf set through the API config. + assert(!session.sparkContext.conf.contains("key2")) + assert(session.sparkContext.conf.get("spark.app.name") == "test") + } + + test("SPARK-15887: hive-site.xml should be loaded") { + val session = SparkSession.builder().master("local").getOrCreate() + assert(session.sessionState.newHadoopConf().get("hive.in.test") == "true") + assert(session.sparkContext.hadoopConfiguration.get("hive.in.test") == "true") + } + + test("SPARK-15991: Set global Hadoop conf") { + val session = SparkSession.builder().master("local").getOrCreate() + val mySpecialKey = "my.special.key.15991" + val mySpecialValue = "msv" + try { + session.sparkContext.hadoopConfiguration.set(mySpecialKey, mySpecialValue) + assert(session.sessionState.newHadoopConf().get(mySpecialKey) == mySpecialValue) + } finally { + session.sparkContext.hadoopConfiguration.unset(mySpecialKey) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala new file mode 100644 index 000000000000..43db79663322 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.SparkFunSuite +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy} +import org.apache.spark.sql.types.{DataType, StructType} + +/** + * Test cases for the [[SparkSessionExtensions]]. + */ +class SparkSessionExtensionSuite extends SparkFunSuite { + type ExtensionsBuilder = SparkSessionExtensions => Unit + private def create(builder: ExtensionsBuilder): ExtensionsBuilder = builder + + private def stop(spark: SparkSession): Unit = { + spark.stop() + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() + } + + private def withSession(builder: ExtensionsBuilder)(f: SparkSession => Unit): Unit = { + val spark = SparkSession.builder().master("local[1]").withExtensions(builder).getOrCreate() + try f(spark) finally { + stop(spark) + } + } + + test("inject analyzer rule") { + withSession(_.injectResolutionRule(MyRule)) { session => + assert(session.sessionState.analyzer.extendedResolutionRules.contains(MyRule(session))) + } + } + + test("inject check analysis rule") { + withSession(_.injectCheckRule(MyCheckRule)) { session => + assert(session.sessionState.analyzer.extendedCheckRules.contains(MyCheckRule(session))) + } + } + + test("inject optimizer rule") { + withSession(_.injectOptimizerRule(MyRule)) { session => + assert(session.sessionState.optimizer.batches.flatMap(_.rules).contains(MyRule(session))) + } + } + + test("inject spark planner strategy") { + withSession(_.injectPlannerStrategy(MySparkStrategy)) { session => + assert(session.sessionState.planner.strategies.contains(MySparkStrategy(session))) + } + } + + test("inject parser") { + val extension = create { extensions => + extensions.injectParser((_, _) => CatalystSqlParser) + } + withSession(extension) { session => + assert(session.sessionState.sqlParser == CatalystSqlParser) + } + } + + test("inject stacked parsers") { + val extension = create { extensions => + extensions.injectParser((_, _) => CatalystSqlParser) + extensions.injectParser(MyParser) + extensions.injectParser(MyParser) + } + withSession(extension) { session => + val parser = MyParser(session, MyParser(session, CatalystSqlParser)) + assert(session.sessionState.sqlParser == parser) + } + } + + test("use custom class for extensions") { + val session = SparkSession.builder() + .master("local[1]") + .config("spark.sql.extensions", classOf[MyExtensions].getCanonicalName) + .getOrCreate() + try { + assert(session.sessionState.planner.strategies.contains(MySparkStrategy(session))) + assert(session.sessionState.analyzer.extendedResolutionRules.contains(MyRule(session))) + } finally { + stop(session) + } + } +} + +case class MyRule(spark: SparkSession) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan +} + +case class MyCheckRule(spark: SparkSession) extends (LogicalPlan => Unit) { + override def apply(plan: LogicalPlan): Unit = { } +} + +case class MySparkStrategy(spark: SparkSession) extends SparkStrategy { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = Seq.empty +} + +case class MyParser(spark: SparkSession, delegate: ParserInterface) extends ParserInterface { + override def parsePlan(sqlText: String): LogicalPlan = + delegate.parsePlan(sqlText) + + override def parseExpression(sqlText: String): Expression = + delegate.parseExpression(sqlText) + + override def parseTableIdentifier(sqlText: String): TableIdentifier = + delegate.parseTableIdentifier(sqlText) + + override def parseFunctionIdentifier(sqlText: String): FunctionIdentifier = + delegate.parseFunctionIdentifier(sqlText) + + override def parseTableSchema(sqlText: String): StructType = + delegate.parseTableSchema(sqlText) + + override def parseDataType(sqlText: String): DataType = + delegate.parseDataType(sqlText) +} + +class MyExtensions extends (SparkSessionExtensions => Unit) { + def apply(e: SparkSessionExtensions): Unit = { + e.injectPlannerStrategy(MySparkStrategy) + e.injectResolutionRule(MyRule) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala new file mode 100644 index 000000000000..2fc92f4aff92 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -0,0 +1,351 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SQLTestData.ArrayData +import org.apache.spark.sql.types._ + + +/** + * End-to-end suite testing statistics collection and use on both entire table and columns. + */ +class StatisticsCollectionSuite extends StatisticsCollectionTestBase with SharedSQLContext { + import testImplicits._ + + test("estimates the size of a limit 0 on outer join") { + withTempView("test") { + Seq(("one", 1), ("two", 2), ("three", 3), ("four", 4)).toDF("k", "v") + .createOrReplaceTempView("test") + val df1 = spark.table("test") + val df2 = spark.table("test").limit(0) + val df = df1.join(df2, Seq("k"), "left") + + val sizes = df.queryExecution.analyzed.collect { case g: Join => + g.stats.sizeInBytes + } + + assert(sizes.size === 1, s"number of Join nodes is wrong:\n ${df.queryExecution}") + assert(sizes.head === BigInt(96), + s"expected exact size 96 for table 'test', got: ${sizes.head}") + } + } + + test("analyzing views is not supported") { + def assertAnalyzeUnsupported(analyzeCommand: String): Unit = { + val err = intercept[AnalysisException] { + sql(analyzeCommand) + } + assert(err.message.contains("ANALYZE TABLE is not supported")) + } + + val tableName = "tbl" + withTable(tableName) { + spark.range(10).write.saveAsTable(tableName) + val viewName = "view" + withView(viewName) { + sql(s"CREATE VIEW $viewName AS SELECT * FROM $tableName") + assertAnalyzeUnsupported(s"ANALYZE TABLE $viewName COMPUTE STATISTICS") + assertAnalyzeUnsupported(s"ANALYZE TABLE $viewName COMPUTE STATISTICS FOR COLUMNS id") + } + } + } + + test("statistics collection of a table with zero column") { + val table_no_cols = "table_no_cols" + withTable(table_no_cols) { + val rddNoCols = sparkContext.parallelize(1 to 10).map(_ => Row.empty) + val dfNoCols = spark.createDataFrame(rddNoCols, StructType(Seq.empty)) + dfNoCols.write.format("json").saveAsTable(table_no_cols) + sql(s"ANALYZE TABLE $table_no_cols COMPUTE STATISTICS") + checkTableStats(table_no_cols, hasSizeInBytes = true, expectedRowCounts = Some(10)) + } + } + + test("analyze empty table") { + val table = "emptyTable" + withTable(table) { + sql(s"CREATE TABLE $table (key STRING, value STRING) USING PARQUET") + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS noscan") + val fetchedStats1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetchedStats1.get.sizeInBytes == 0) + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS") + val fetchedStats2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) + assert(fetchedStats2.get.sizeInBytes == 0) + } + } + + test("analyze column command - unsupported types and invalid columns") { + val tableName = "column_stats_test1" + withTable(tableName) { + Seq(ArrayData(Seq(1, 2, 3), Seq(Seq(1, 2, 3)))).toDF().write.saveAsTable(tableName) + + // Test unsupported data types + val err1 = intercept[AnalysisException] { + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS data") + } + assert(err1.message.contains("does not support statistics collection")) + + // Test invalid columns + val err2 = intercept[AnalysisException] { + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS some_random_column") + } + assert(err2.message.contains("does not exist")) + } + } + + test("test table-level statistics for data source table") { + val tableName = "tbl" + withTable(tableName) { + sql(s"CREATE TABLE $tableName(i INT, j STRING) USING parquet") + Seq(1 -> "a", 2 -> "b").toDF("i", "j").write.mode("overwrite").insertInto(tableName) + + // noscan won't count the number of rows + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS noscan") + checkTableStats(tableName, hasSizeInBytes = true, expectedRowCounts = None) + + // without noscan, we count the number of rows + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS") + checkTableStats(tableName, hasSizeInBytes = true, expectedRowCounts = Some(2)) + } + } + + test("SPARK-15392: DataFrame created from RDD should not be broadcasted") { + val rdd = sparkContext.range(1, 100).map(i => Row(i, i)) + val df = spark.createDataFrame(rdd, new StructType().add("a", LongType).add("b", LongType)) + assert(df.queryExecution.analyzed.stats.sizeInBytes > + spark.sessionState.conf.autoBroadcastJoinThreshold) + assert(df.selectExpr("a").queryExecution.analyzed.stats.sizeInBytes > + spark.sessionState.conf.autoBroadcastJoinThreshold) + } + + test("column stats round trip serialization") { + // Make sure we serialize and then deserialize and we will get the result data + val df = data.toDF(stats.keys.toSeq :+ "carray" : _*) + stats.zip(df.schema).foreach { case ((k, v), field) => + withClue(s"column $k with type ${field.dataType}") { + val roundtrip = ColumnStat.fromMap("table_is_foo", field, v.toMap(k, field.dataType)) + assert(roundtrip == Some(v)) + } + } + } + + test("analyze column command - result verification") { + // (data.head.productArity - 1) because the last column does not support stats collection. + assert(stats.size == data.head.productArity - 1) + val df = data.toDF(stats.keys.toSeq :+ "carray" : _*) + checkColStats(df, stats) + } + + test("column stats collection for null columns") { + val dataTypes: Seq[(DataType, Int)] = Seq( + BooleanType, ByteType, ShortType, IntegerType, LongType, + DoubleType, FloatType, DecimalType.SYSTEM_DEFAULT, + StringType, BinaryType, DateType, TimestampType + ).zipWithIndex + + val df = sql("select " + dataTypes.map { case (tpe, idx) => + s"cast(null as ${tpe.sql}) as col$idx" + }.mkString(", ")) + + val expectedColStats = dataTypes.map { case (tpe, idx) => + (s"col$idx", ColumnStat(0, None, None, 1, tpe.defaultSize.toLong, tpe.defaultSize.toLong)) + } + checkColStats(df, mutable.LinkedHashMap(expectedColStats: _*)) + } + + test("number format in statistics") { + val numbers = Seq( + BigInt(0) -> (("0.0 B", "0")), + BigInt(100) -> (("100.0 B", "100")), + BigInt(2047) -> (("2047.0 B", "2.05E+3")), + BigInt(2048) -> (("2.0 KB", "2.05E+3")), + BigInt(3333333) -> (("3.2 MB", "3.33E+6")), + BigInt(4444444444L) -> (("4.1 GB", "4.44E+9")), + BigInt(5555555555555L) -> (("5.1 TB", "5.56E+12")), + BigInt(6666666666666666L) -> (("5.9 PB", "6.67E+15")), + BigInt(1L << 10 ) * (1L << 60) -> (("1024.0 EB", "1.18E+21")), + BigInt(1L << 11) * (1L << 60) -> (("2.36E+21 B", "2.36E+21")) + ) + numbers.foreach { case (input, (expectedSize, expectedRows)) => + val stats = Statistics(sizeInBytes = input, rowCount = Some(input)) + val expectedString = s"sizeInBytes=$expectedSize, rowCount=$expectedRows," + + s" hints=none" + assert(stats.simpleString == expectedString) + } + } + + test("change stats after truncate command") { + val table = "change_stats_truncate_table" + withTable(table) { + spark.range(100).select($"id", $"id" % 5 as "value").write.saveAsTable(table) + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS id, value") + val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(100)) + assert(fetched1.get.sizeInBytes > 0) + assert(fetched1.get.colStats.size == 2) + + // truncate table command + sql(s"TRUNCATE TABLE $table") + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) + assert(fetched2.get.sizeInBytes == 0) + assert(fetched2.get.colStats.isEmpty) + } + } + + test("change stats after set location command") { + val table = "change_stats_set_location_table" + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + spark.range(100).select($"id", $"id" % 5 as "value").write.saveAsTable(table) + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS id, value") + val fetched1 = checkTableStats( + table, hasSizeInBytes = true, expectedRowCounts = Some(100)) + assert(fetched1.get.sizeInBytes > 0) + assert(fetched1.get.colStats.size == 2) + + // set location command + val initLocation = spark.sessionState.catalog.getTableMetadata(TableIdentifier(table)) + .storage.locationUri.get.toString + withTempDir { newLocation => + sql(s"ALTER TABLE $table SET LOCATION '${newLocation.toURI.toString}'") + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes == 0) + assert(fetched2.get.colStats.isEmpty) + + // set back to the initial location + sql(s"ALTER TABLE $table SET LOCATION '$initLocation'") + val fetched3 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched3.get.sizeInBytes == fetched1.get.sizeInBytes) + } else { + checkTableStats(table, hasSizeInBytes = false, expectedRowCounts = None) + } + } + } + } + } + } + + test("change stats after insert command for datasource table") { + val table = "change_stats_insert_datasource_table" + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + sql(s"CREATE TABLE $table (i int, j string) USING PARQUET") + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) + assert(fetched1.get.sizeInBytes == 0) + assert(fetched1.get.colStats.size == 2) + + // table lookup will make the table cached + spark.table(table) + assert(isTableInCatalogCache(table)) + + // insert into command + sql(s"INSERT INTO TABLE $table SELECT 1, 'abc'") + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes > 0) + assert(fetched2.get.colStats.isEmpty) + } else { + checkTableStats(table, hasSizeInBytes = false, expectedRowCounts = None) + } + + // check that tableRelationCache inside the catalog was invalidated after insert + assert(!isTableInCatalogCache(table)) + } + } + } + } + + test("invalidation of tableRelationCache after inserts") { + val table = "invalidate_catalog_cache_table" + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + spark.range(100).write.saveAsTable(table) + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS") + spark.table(table) + val initialSizeInBytes = getTableFromCatalogCache(table).stats.sizeInBytes + spark.range(100).write.mode(SaveMode.Append).saveAsTable(table) + spark.table(table) + assert(getTableFromCatalogCache(table).stats.sizeInBytes == 2 * initialSizeInBytes) + } + } + } + } + + test("invalidation of tableRelationCache after table truncation") { + val table = "invalidate_catalog_cache_table" + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + spark.range(100).write.saveAsTable(table) + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS") + spark.table(table) + sql(s"TRUNCATE TABLE $table") + spark.table(table) + assert(getTableFromCatalogCache(table).stats.sizeInBytes == 0) + } + } + } + } + + test("invalidation of tableRelationCache after alter table add partition") { + val table = "invalidate_catalog_cache_table" + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTempDir { dir => + withTable(table) { + val path = dir.getCanonicalPath + sql(s""" + |CREATE TABLE $table (col1 int, col2 int) + |USING PARQUET + |PARTITIONED BY (col2) + |LOCATION '${dir.toURI}'""".stripMargin) + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS") + spark.table(table) + assert(getTableFromCatalogCache(table).stats.sizeInBytes == 0) + spark.catalog.recoverPartitions(table) + val df = Seq((1, 2), (1, 2)).toDF("col2", "col1") + df.write.parquet(s"$path/col2=1") + sql(s"ALTER TABLE $table ADD PARTITION (col2=1) LOCATION '${dir.toURI}'") + spark.table(table) + val cachedTable = getTableFromCatalogCache(table) + val cachedTableSizeInBytes = cachedTable.stats.sizeInBytes + val defaultSizeInBytes = conf.defaultSizeInBytes + if (autoUpdate) { + assert(cachedTableSizeInBytes != defaultSizeInBytes && cachedTableSizeInBytes > 0) + } else { + assert(cachedTableSizeInBytes == defaultSizeInBytes) + } + } + } + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala new file mode 100644 index 000000000000..a2f63edd786b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.{lang => jl} +import java.sql.{Date, Timestamp} + +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} +import org.apache.spark.sql.catalyst.catalog.{CatalogStatistics, CatalogTable, HiveTableRelation} +import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan} +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.internal.StaticSQLConf +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.Decimal + + +/** + * The base for statistics test cases that we want to include in both the hive module (for + * verifying behavior when using the Hive external catalog) as well as in the sql/core module. + */ +abstract class StatisticsCollectionTestBase extends QueryTest with SQLTestUtils { + import testImplicits._ + + private val dec1 = new java.math.BigDecimal("1.000000000000000000") + private val dec2 = new java.math.BigDecimal("8.000000000000000000") + private val d1 = Date.valueOf("2016-05-08") + private val d2 = Date.valueOf("2016-05-09") + private val t1 = Timestamp.valueOf("2016-05-08 00:00:01") + private val t2 = Timestamp.valueOf("2016-05-09 00:00:02") + + /** + * Define a very simple 3 row table used for testing column serialization. + * Note: last column is seq[int] which doesn't support stats collection. + */ + protected val data = Seq[ + (jl.Boolean, jl.Byte, jl.Short, jl.Integer, jl.Long, + jl.Double, jl.Float, java.math.BigDecimal, + String, Array[Byte], Date, Timestamp, + Seq[Int])]( + (false, 1.toByte, 1.toShort, 1, 1L, 1.0, 1.0f, dec1, "s1", "b1".getBytes, d1, t1, null), + (true, 2.toByte, 3.toShort, 4, 5L, 6.0, 7.0f, dec2, "ss9", "bb0".getBytes, d2, t2, null), + (null, null, null, null, null, null, null, null, null, null, null, null, null) + ) + + /** A mapping from column to the stats collected. */ + protected val stats = mutable.LinkedHashMap( + "cbool" -> ColumnStat(2, Some(false), Some(true), 1, 1, 1), + "cbyte" -> ColumnStat(2, Some(1.toByte), Some(2.toByte), 1, 1, 1), + "cshort" -> ColumnStat(2, Some(1.toShort), Some(3.toShort), 1, 2, 2), + "cint" -> ColumnStat(2, Some(1), Some(4), 1, 4, 4), + "clong" -> ColumnStat(2, Some(1L), Some(5L), 1, 8, 8), + "cdouble" -> ColumnStat(2, Some(1.0), Some(6.0), 1, 8, 8), + "cfloat" -> ColumnStat(2, Some(1.0f), Some(7.0f), 1, 4, 4), + "cdecimal" -> ColumnStat(2, Some(Decimal(dec1)), Some(Decimal(dec2)), 1, 16, 16), + "cstring" -> ColumnStat(2, None, None, 1, 3, 3), + "cbinary" -> ColumnStat(2, None, None, 1, 3, 3), + "cdate" -> ColumnStat(2, Some(DateTimeUtils.fromJavaDate(d1)), + Some(DateTimeUtils.fromJavaDate(d2)), 1, 4, 4), + "ctimestamp" -> ColumnStat(2, Some(DateTimeUtils.fromJavaTimestamp(t1)), + Some(DateTimeUtils.fromJavaTimestamp(t2)), 1, 8, 8) + ) + + private val randomName = new Random(31) + + def getCatalogTable(tableName: String): CatalogTable = { + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) + } + + def getTableFromCatalogCache(tableName: String): LogicalPlan = { + val catalog = spark.sessionState.catalog + val qualifiedTableName = QualifiedTableName(catalog.getCurrentDatabase, tableName) + catalog.getCachedTable(qualifiedTableName) + } + + def isTableInCatalogCache(tableName: String): Boolean = { + getTableFromCatalogCache(tableName) != null + } + + def getCatalogStatistics(tableName: String): CatalogStatistics = { + getCatalogTable(tableName).stats.get + } + + def checkTableStats( + tableName: String, + hasSizeInBytes: Boolean, + expectedRowCounts: Option[Int]): Option[CatalogStatistics] = { + val stats = getCatalogTable(tableName).stats + if (hasSizeInBytes || expectedRowCounts.nonEmpty) { + assert(stats.isDefined) + assert(stats.get.sizeInBytes >= 0) + assert(stats.get.rowCount === expectedRowCounts) + } else { + assert(stats.isEmpty) + } + + stats + } + + /** + * Compute column stats for the given DataFrame and compare it with colStats. + */ + def checkColStats( + df: DataFrame, + colStats: mutable.LinkedHashMap[String, ColumnStat]): Unit = { + val tableName = "column_stats_test_" + randomName.nextInt(1000) + withTable(tableName) { + df.write.saveAsTable(tableName) + + // Collect statistics + sql(s"analyze table $tableName compute STATISTICS FOR COLUMNS " + + colStats.keys.mkString(", ")) + + // Validate statistics + val table = getCatalogTable(tableName) + assert(table.stats.isDefined) + assert(table.stats.get.colStats.size == colStats.size) + + colStats.foreach { case (k, v) => + withClue(s"column $k") { + assert(table.stats.get.colStats(k) == v) + } + } + } + } + + // This test will be run twice: with and without Hive support + test("SPARK-18856: non-empty partitioned table should not report zero size") { + withTable("ds_tbl", "hive_tbl") { + spark.range(100).select($"id", $"id" % 5 as "p").write.partitionBy("p").saveAsTable("ds_tbl") + val stats = spark.table("ds_tbl").queryExecution.optimizedPlan.stats + assert(stats.sizeInBytes > 0, "non-empty partitioned table should not report zero size.") + + if (spark.conf.get(StaticSQLConf.CATALOG_IMPLEMENTATION) == "hive") { + sql("CREATE TABLE hive_tbl(i int) PARTITIONED BY (j int)") + sql("INSERT INTO hive_tbl PARTITION(j=1) SELECT 1") + val stats2 = spark.table("hive_tbl").queryExecution.optimizedPlan.stats + assert(stats2.sizeInBytes > 0, "non-empty partitioned table should not report zero size.") + } + } + } + + // This test will be run twice: with and without Hive support + test("conversion from CatalogStatistics to Statistics") { + withTable("ds_tbl", "hive_tbl") { + // Test data source table + checkStatsConversion(tableName = "ds_tbl", isDatasourceTable = true) + // Test hive serde table + if (spark.conf.get(StaticSQLConf.CATALOG_IMPLEMENTATION) == "hive") { + checkStatsConversion(tableName = "hive_tbl", isDatasourceTable = false) + } + } + } + + private def checkStatsConversion(tableName: String, isDatasourceTable: Boolean): Unit = { + // Create an empty table and run analyze command on it. + val createTableSql = if (isDatasourceTable) { + s"CREATE TABLE $tableName (c1 INT, c2 STRING) USING PARQUET" + } else { + s"CREATE TABLE $tableName (c1 INT, c2 STRING)" + } + sql(createTableSql) + // Analyze only one column. + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS c1") + val (relation, catalogTable) = spark.table(tableName).queryExecution.analyzed.collect { + case catalogRel: HiveTableRelation => (catalogRel, catalogRel.tableMeta) + case logicalRel: LogicalRelation => (logicalRel, logicalRel.catalogTable.get) + }.head + val emptyColStat = ColumnStat(0, None, None, 0, 4, 4) + // Check catalog statistics + assert(catalogTable.stats.isDefined) + assert(catalogTable.stats.get.sizeInBytes == 0) + assert(catalogTable.stats.get.rowCount == Some(0)) + assert(catalogTable.stats.get.colStats == Map("c1" -> emptyColStat)) + + // Check relation statistics + assert(relation.stats.sizeInBytes == 0) + assert(relation.stats.rowCount == Some(0)) + assert(relation.stats.attributeStats.size == 1) + val (attribute, colStat) = relation.stats.attributeStats.head + assert(attribute.name == "c1") + assert(colStat == emptyColStat) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index e2090b0a83ce..3d76b9ac33e5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -48,6 +48,20 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { Row("a||b")) } + test("string elt") { + val df = Seq[(String, String, String, Int)](("hello", "world", null, 15)) + .toDF("a", "b", "c", "d") + + checkAnswer( + df.selectExpr("elt(0, a, b, c)", "elt(1, a, b, c)", "elt(4, a, b, c)"), + Row(null, "hello", null)) + + // check implicit type cast + checkAnswer( + df.selectExpr("elt(4, a, b, c, d)", "elt('2', a, b, c, d)"), + Row("15", "world")) + } + test("string Levenshtein distance") { val df = Seq(("kitten", "sitting"), ("frog", "fog")).toDF("l", "r") checkAnswer(df.select(levenshtein($"l", $"r")), Seq(Row(3), Row(1))) @@ -63,8 +77,10 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { checkAnswer( df.select( regexp_replace($"a", "(\\d+)", "num"), + regexp_replace($"a", $"b", $"c"), regexp_extract($"a", "(\\d+)-(\\d+)", 1)), - Row("num-num", "100") :: Row("num-num", "100") :: Row("num-num", "100") :: Nil) + Row("num-num", "300", "100") :: Row("num-num", "400", "100") :: + Row("num-num", "400-400", "100") :: Nil) // for testing the mutable state of the expression in code gen. // This is a hack way to enable the codegen, thus the codegen is enable by default, @@ -78,6 +94,18 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { Row("300", "100") :: Row("400", "100") :: Row("400-400", "100") :: Nil) } + test("non-matching optional group") { + val df = Seq(Tuple1("aaaac")).toDF("s") + checkAnswer( + df.select(regexp_extract($"s", "(foo)", 1)), + Row("") + ) + checkAnswer( + df.select(regexp_extract($"s", "(a+)(b)?(c)", 2)), + Row("") + ) + } + test("string ascii function") { val df = Seq(("abc", "")).toDF("a", "b") checkAnswer( @@ -133,12 +161,24 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { } test("string trim functions") { - val df = Seq((" example ", "")).toDF("a", "b") + val df = Seq((" example ", "", "example")).toDF("a", "b", "c") checkAnswer( df.select(ltrim($"a"), rtrim($"a"), trim($"a")), Row("example ", " example", "example")) + checkAnswer( + df.select(ltrim($"c", "e"), rtrim($"c", "e"), trim($"c", "e")), + Row("xample", "exampl", "xampl")) + + checkAnswer( + df.select(ltrim($"c", "xe"), rtrim($"c", "emlp"), trim($"c", "elxp")), + Row("ample", "exa", "am")) + + checkAnswer( + df.select(trim($"c", "xyz")), + Row("example")) + checkAnswer( df.selectExpr("ltrim(a)", "rtrim(a)", "trim(a)"), Row("example ", " example", "example")) @@ -189,15 +229,15 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { } test("string locate function") { - val df = Seq(("aaads", "aa", "zz", 1)).toDF("a", "b", "c", "d") + val df = Seq(("aaads", "aa", "zz", 2)).toDF("a", "b", "c", "d") checkAnswer( - df.select(locate("aa", $"a"), locate("aa", $"a", 1)), - Row(1, 2)) + df.select(locate("aa", $"a"), locate("aa", $"a", 2), locate("aa", $"a", 0)), + Row(1, 2, 0)) checkAnswer( - df.selectExpr("locate(b, a)", "locate(b, a, d)"), - Row(1, 2)) + df.selectExpr("locate(b, a)", "locate(b, a, d)", "locate(b, a, 3)"), + Row(1, 2, 0)) } test("string padding functions") { @@ -212,6 +252,51 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { Row("???hi", "hi???", "h", "h")) } + test("string parse_url function") { + + def testUrl(url: String, expected: Row) { + checkAnswer(Seq[String]((url)).toDF("url").selectExpr( + "parse_url(url, 'HOST')", "parse_url(url, 'PATH')", + "parse_url(url, 'QUERY')", "parse_url(url, 'REF')", + "parse_url(url, 'PROTOCOL')", "parse_url(url, 'FILE')", + "parse_url(url, 'AUTHORITY')", "parse_url(url, 'USERINFO')", + "parse_url(url, 'QUERY', 'query')"), expected) + } + + testUrl( + "http://userinfo@spark.apache.org/path?query=1#Ref", + Row("spark.apache.org", "/path", "query=1", "Ref", + "http", "/path?query=1", "userinfo@spark.apache.org", "userinfo", "1")) + + testUrl( + "https://use%20r:pas%20s@example.com/dir%20/pa%20th.HTML?query=x%20y&q2=2#Ref%20two", + Row("example.com", "/dir%20/pa%20th.HTML", "query=x%20y&q2=2", "Ref%20two", + "https", "/dir%20/pa%20th.HTML?query=x%20y&q2=2", "use%20r:pas%20s@example.com", + "use%20r:pas%20s", "x%20y")) + + testUrl( + "http://user:pass@host", + Row("host", "", null, null, "http", "", "user:pass@host", "user:pass", null)) + + testUrl( + "http://user:pass@host/", + Row("host", "/", null, null, "http", "/", "user:pass@host", "user:pass", null)) + + testUrl( + "http://user:pass@host/?#", + Row("host", "/", "", "", "http", "/?", "user:pass@host", "user:pass", null)) + + testUrl( + "http://user:pass@host/file;param?query;p2", + Row("host", "/file;param", "query;p2", null, "http", "/file;param?query;p2", + "user:pass@host", "user:pass", null)) + + testUrl( + "inva lid://user:pass@host/file;param?query;p2", + Row(null, null, null, null, null, null, null, null, null)) + + } + test("string repeat function") { val df = Seq(("hi", 2)).toDF("a", "b") @@ -257,7 +342,8 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { } test("string / binary length function") { - val df = Seq(("123", Array[Byte](1, 2, 3, 4), 123)).toDF("a", "b", "c") + val df = Seq(("123", Array[Byte](1, 2, 3, 4), 123, 2.0f, 3.015)) + .toDF("a", "b", "c", "d", "e") checkAnswer( df.select(length($"a"), length($"b")), Row(3, 4)) @@ -266,22 +352,23 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { df.selectExpr("length(a)", "length(b)"), Row(3, 4)) - intercept[AnalysisException] { - df.selectExpr("length(c)") // int type of the argument is unacceptable - } + checkAnswer( + df.selectExpr("length(c)", "length(d)", "length(e)"), + Row(3, 3, 5) + ) } test("initcap function") { - val df = Seq(("ab", "a B")).toDF("l", "r") + val df = Seq(("ab", "a B", "sParK")).toDF("x", "y", "z") checkAnswer( - df.select(initcap($"l"), initcap($"r")), Row("Ab", "A B")) + df.select(initcap($"x"), initcap($"y"), initcap($"z")), Row("Ab", "A B", "Spark")) checkAnswer( - df.selectExpr("InitCap(l)", "InitCap(r)"), Row("Ab", "A B")) + df.selectExpr("InitCap(x)", "InitCap(y)", "InitCap(z)"), Row("Ab", "A B", "Spark")) } test("number format function") { - val df = sqlContext.range(1) + val df = spark.range(1) checkAnswer( df.select(format_number(lit(5L), 4)), @@ -312,7 +399,7 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { Row("6.4817")) checkAnswer( - df.select(format_number(lit(BigDecimal(7.128381)), 4)), // not convert anything + df.select(format_number(lit(BigDecimal("7.128381")), 4)), // not convert anything Row("7.1284")) intercept[AnalysisException] { @@ -333,4 +420,47 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { df2.filter("b>0").selectExpr("format_number(a, b)"), Row("5.0000") :: Row("4.000") :: Row("4.000") :: Row("4.000") :: Row("3.00") :: Nil) } + + test("string sentences function") { + val df = Seq(("Hi there! The price was $1,234.56.... But, not now.", "en", "US")) + .toDF("str", "language", "country") + + checkAnswer( + df.selectExpr("sentences(str, language, country)"), + Row(Seq(Seq("Hi", "there"), Seq("The", "price", "was"), Seq("But", "not", "now")))) + + // Type coercion + checkAnswer( + df.selectExpr("sentences(null)", "sentences(10)", "sentences(3.14)"), + Row(null, Seq(Seq("10")), Seq(Seq("3.14")))) + + // Argument number exception + val m = intercept[AnalysisException] { + df.selectExpr("sentences()") + }.getMessage + assert(m.contains("Invalid number of arguments for function sentences")) + } + + test("str_to_map function") { + val df1 = Seq( + ("a=1,b=2", "y"), + ("a=1,b=2,c=3", "y") + ).toDF("a", "b") + + checkAnswer( + df1.selectExpr("str_to_map(a,',','=')"), + Seq( + Row(Map("a" -> "1", "b" -> "2")), + Row(Map("a" -> "1", "b" -> "2", "c" -> "3")) + ) + ) + + val df2 = Seq(("a:1,b:2,c:3", "y")).toDF("a", "b") + + checkAnswer( + df2.selectExpr("str_to_map(a)"), + Seq(Row(Map("a" -> "1", "b" -> "2", "c" -> "3"))) + ) + + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala new file mode 100644 index 000000000000..8673dc14f759 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -0,0 +1,953 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.plans.logical.Join +import org.apache.spark.sql.test.SharedSQLContext + +class SubquerySuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + setupTestData() + + val row = identity[(java.lang.Integer, java.lang.Double)](_) + + lazy val l = Seq( + row((1, 2.0)), + row((1, 2.0)), + row((2, 1.0)), + row((2, 1.0)), + row((3, 3.0)), + row((null, null)), + row((null, 5.0)), + row((6, null))).toDF("a", "b") + + lazy val r = Seq( + row((2, 3.0)), + row((2, 3.0)), + row((3, 2.0)), + row((4, 1.0)), + row((null, null)), + row((null, 5.0)), + row((6, null))).toDF("c", "d") + + lazy val t = r.filter($"c".isNotNull && $"d".isNotNull) + + protected override def beforeAll(): Unit = { + super.beforeAll() + l.createOrReplaceTempView("l") + r.createOrReplaceTempView("r") + t.createOrReplaceTempView("t") + } + + test("SPARK-18854 numberedTreeString for subquery") { + val df = sql("select * from range(10) where id not in " + + "(select id from range(2) union all select id from range(2))") + + // The depth first traversal of the plan tree + val dfs = Seq("Project", "Filter", "Union", "Project", "Range", "Project", "Range", "Range") + val numbered = df.queryExecution.analyzed.numberedTreeString.split("\n") + + // There should be 8 plan nodes in total + assert(numbered.size == dfs.size) + + for (i <- dfs.indices) { + val node = df.queryExecution.analyzed(i) + assert(node.nodeName == dfs(i)) + assert(numbered(i).contains(node.nodeName)) + } + } + + test("SPARK-15791: rdd deserialization does not crash") { + sql("select (select 1 as b) as b").rdd.count() + } + + test("simple uncorrelated scalar subquery") { + checkAnswer( + sql("select (select 1 as b) as b"), + Array(Row(1)) + ) + + checkAnswer( + sql("select (select (select 1) + 1) + 1"), + Array(Row(3)) + ) + + // string type + checkAnswer( + sql("select (select 's' as s) as b"), + Array(Row("s")) + ) + } + + test("define CTE in CTE subquery") { + checkAnswer( + sql( + """ + | with t2 as (with t1 as (select 1 as b, 2 as c) select b, c from t1) + | select a from (select 1 as a union all select 2 as a) t + | where a = (select max(b) from t2) + """.stripMargin), + Array(Row(1)) + ) + checkAnswer( + sql( + """ + | with t2 as (with t1 as (select 1 as b, 2 as c) select b, c from t1), + | t3 as ( + | with t4 as (select 1 as d, 3 as e) + | select * from t4 cross join t2 where t2.b = t4.d + | ) + | select a from (select 1 as a union all select 2 as a) + | where a = (select max(d) from t3) + """.stripMargin), + Array(Row(1)) + ) + } + + test("uncorrelated scalar subquery in CTE") { + checkAnswer( + sql("with t2 as (select 1 as b, 2 as c) " + + "select a from (select 1 as a union all select 2 as a) t " + + "where a = (select max(b) from t2) "), + Array(Row(1)) + ) + } + + test("uncorrelated scalar subquery should return null if there is 0 rows") { + checkAnswer( + sql("select (select 's' as s limit 0) as b"), + Array(Row(null)) + ) + } + + test("runtime error when the number of rows is greater than 1") { + val error2 = intercept[RuntimeException] { + sql("select (select a from (select 1 as a union all select 2 as a) t) as b").collect() + } + assert(error2.getMessage.contains( + "more than one row returned by a subquery used as an expression") + ) + } + + test("uncorrelated scalar subquery on a DataFrame generated query") { + val df = Seq((1, "one"), (2, "two"), (3, "three")).toDF("key", "value") + df.createOrReplaceTempView("subqueryData") + + checkAnswer( + sql("select (select key from subqueryData where key > 2 order by key limit 1) + 1"), + Array(Row(4)) + ) + + checkAnswer( + sql("select -(select max(key) from subqueryData)"), + Array(Row(-3)) + ) + + checkAnswer( + sql("select (select value from subqueryData limit 0)"), + Array(Row(null)) + ) + + checkAnswer( + sql("select (select min(value) from subqueryData" + + " where key = (select max(key) from subqueryData) - 1)"), + Array(Row("two")) + ) + } + + test("SPARK-15677: Queries against local relations with scalar subquery in Select list") { + withTempView("t1", "t2") { + Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t1") + Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2") + + checkAnswer( + sql("SELECT (select 1 as col) from t1"), + Row(1) :: Row(1) :: Nil) + + checkAnswer( + sql("SELECT (select max(c1) from t2) from t1"), + Row(2) :: Row(2) :: Nil) + + checkAnswer( + sql("SELECT 1 + (select 1 as col) from t1"), + Row(2) :: Row(2) :: Nil) + + checkAnswer( + sql("SELECT c1, (select max(c1) from t2) + c2 from t1"), + Row(1, 3) :: Row(2, 4) :: Nil) + + checkAnswer( + sql("SELECT c1, (select max(c1) from t2 where t1.c2 = t2.c2) from t1"), + Row(1, 1) :: Row(2, 2) :: Nil) + } + } + + test("SPARK-14791: scalar subquery inside broadcast join") { + val df = sql("select a, sum(b) as s from l group by a having a > (select avg(a) from l)") + val expected = Row(3, 2.0, 3, 3.0) :: Row(6, null, 6, null) :: Nil + (1 to 10).foreach { _ => + checkAnswer(r.join(df, $"c" === $"a"), expected) + } + } + + test("EXISTS predicate subquery") { + checkAnswer( + sql("select * from l where exists (select * from r where l.a = r.c)"), + Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Row(6, null) :: Nil) + + checkAnswer( + sql("select * from l where exists (select * from r where l.a = r.c) and l.a <= 2"), + Row(2, 1.0) :: Row(2, 1.0) :: Nil) + } + + test("NOT EXISTS predicate subquery") { + checkAnswer( + sql("select * from l where not exists (select * from r where l.a = r.c)"), + Row(1, 2.0) :: Row(1, 2.0) :: Row(null, null) :: Row(null, 5.0) :: Nil) + + checkAnswer( + sql("select * from l where not exists (select * from r where l.a = r.c and l.b < r.d)"), + Row(1, 2.0) :: Row(1, 2.0) :: Row(3, 3.0) :: + Row(null, null) :: Row(null, 5.0) :: Row(6, null) :: Nil) + } + + test("EXISTS predicate subquery within OR") { + checkAnswer( + sql("select * from l where exists (select * from r where l.a = r.c)" + + " or exists (select * from r where l.a = r.c)"), + Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Row(6, null) :: Nil) + + checkAnswer( + sql("select * from l where not exists (select * from r where l.a = r.c and l.b < r.d)" + + " or not exists (select * from r where l.a = r.c)"), + Row(1, 2.0) :: Row(1, 2.0) :: Row(3, 3.0) :: + Row(null, null) :: Row(null, 5.0) :: Row(6, null) :: Nil) + } + + test("IN predicate subquery") { + checkAnswer( + sql("select * from l where l.a in (select c from r)"), + Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Row(6, null) :: Nil) + + checkAnswer( + sql("select * from l where l.a in (select c from r where l.b < r.d)"), + Row(2, 1.0) :: Row(2, 1.0) :: Nil) + + checkAnswer( + sql("select * from l where l.a in (select c from r) and l.a > 2 and l.b is not null"), + Row(3, 3.0) :: Nil) + } + + test("NOT IN predicate subquery") { + checkAnswer( + sql("select * from l where a not in (select c from r)"), + Nil) + + checkAnswer( + sql("select * from l where a not in (select c from r where c is not null)"), + Row(1, 2.0) :: Row(1, 2.0) :: Nil) + + checkAnswer( + sql("select * from l where (a, b) not in (select c, d from t) and a < 4"), + Row(1, 2.0) :: Row(1, 2.0) :: Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Nil) + + // Empty sub-query + checkAnswer( + sql("select * from l where (a, b) not in (select c, d from r where c > 10)"), + Row(1, 2.0) :: Row(1, 2.0) :: Row(2, 1.0) :: Row(2, 1.0) :: + Row(3, 3.0) :: Row(null, null) :: Row(null, 5.0) :: Row(6, null) :: Nil) + + } + + test("IN predicate subquery within OR") { + checkAnswer( + sql("select * from l where l.a in (select c from r)" + + " or l.a in (select c from r where l.b < r.d)"), + Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Row(6, null) :: Nil) + + intercept[AnalysisException] { + sql("select * from l where a not in (select c from r)" + + " or a not in (select c from r where c is not null)") + } + } + + test("complex IN predicate subquery") { + checkAnswer( + sql("select * from l where (a, b) not in (select c, d from r)"), + Nil) + + checkAnswer( + sql("select * from l where (a, b) not in (select c, d from t) and (a + b) is not null"), + Row(1, 2.0) :: Row(1, 2.0) :: Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Nil) + } + + test("same column in subquery and outer table") { + checkAnswer( + sql("select a from l l1 where a in (select a from l where a < 3 group by a)"), + Row(1) :: Row(1) :: Row(2) :: Row(2) :: Nil + ) + } + + test("having with function in subquery") { + checkAnswer( + sql("select a from l group by 1 having exists (select 1 from r where d < min(b))"), + Row(null) :: Row(1) :: Row(3) :: Nil) + } + + test("SPARK-15832: Test embedded existential predicate sub-queries") { + withTempView("t1", "t2", "t3", "t4", "t5") { + Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t1") + Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2") + Seq((1, 1), (2, 2), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t3") + + checkAnswer( + sql( + """ + | select c1 from t1 + | where c2 IN (select c2 from t2) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where c2 NOT IN (select c2 from t2) + | + """.stripMargin), + Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where EXISTS (select c2 from t2) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where NOT EXISTS (select c2 from t2) + | + """.stripMargin), + Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where NOT EXISTS (select c2 from t2) and + | c2 IN (select c2 from t3) + | + """.stripMargin), + Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where (case when c2 IN (select 1 as one) then 1 + | else 2 end) = c1 + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where (case when c2 IN (select 1 as one) then 1 + | else 2 end) + | IN (select c2 from t2) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where (case when c2 IN (select c2 from t2) then 1 + | else 2 end) + | IN (select c2 from t3) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where (case when c2 IN (select c2 from t2) then 1 + | when c2 IN (select c2 from t3) then 2 + | else 3 end) + | IN (select c2 from t1) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where (c1, (case when c2 IN (select c2 from t2) then 1 + | when c2 IN (select c2 from t3) then 2 + | else 3 end)) + | IN (select c1, c2 from t1) + | + """.stripMargin), + Row(1) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t3 + | where ((case when c2 IN (select c2 from t2) then 1 else 2 end), + | (case when c2 IN (select c2 from t3) then 2 else 3 end)) + | IN (select c1, c2 from t3) + | + """.stripMargin), + Row(1) :: Row(2) :: Row(1) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where ((case when EXISTS (select c2 from t2) then 1 else 2 end), + | (case when c2 IN (select c2 from t3) then 2 else 3 end)) + | IN (select c1, c2 from t3) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where (case when c2 IN (select c2 from t2) then 3 + | else 2 end) + | NOT IN (select c2 from t3) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where ((case when c2 IN (select c2 from t2) then 1 else 2 end), + | (case when NOT EXISTS (select c2 from t3) then 2 + | when EXISTS (select c2 from t2) then 3 + | else 3 end)) + | NOT IN (select c1, c2 from t3) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + + checkAnswer( + sql( + """ + | select c1 from t1 + | where (select max(c1) from t2 where c2 IN (select c2 from t3)) + | IN (select c2 from t2) + | + """.stripMargin), + Row(1) :: Row(2) :: Nil) + } + } + + test("correlated scalar subquery in where") { + checkAnswer( + sql("select * from l where b < (select max(d) from r where a = c)"), + Row(2, 1.0) :: Row(2, 1.0) :: Nil) + } + + test("correlated scalar subquery in select") { + checkAnswer( + sql("select a, (select sum(b) from l l2 where l2.a = l1.a) sum_b from l l1"), + Row(1, 4.0) :: Row(1, 4.0) :: Row(2, 2.0) :: Row(2, 2.0) :: Row(3, 3.0) :: + Row(null, null) :: Row(null, null) :: Row(6, null) :: Nil) + } + + test("correlated scalar subquery in select (null safe)") { + checkAnswer( + sql("select a, (select sum(b) from l l2 where l2.a <=> l1.a) sum_b from l l1"), + Row(1, 4.0) :: Row(1, 4.0) :: Row(2, 2.0) :: Row(2, 2.0) :: Row(3, 3.0) :: + Row(null, 5.0) :: Row(null, 5.0) :: Row(6, null) :: Nil) + } + + test("correlated scalar subquery in aggregate") { + checkAnswer( + sql("select a, (select sum(d) from r where a = c) sum_d from l l1 group by 1, 2"), + Row(1, null) :: Row(2, 6.0) :: Row(3, 2.0) :: Row(null, null) :: Row(6, null) :: Nil) + } + + test("SPARK-18504 extra GROUP BY column in correlated scalar subquery is not permitted") { + withTempView("t") { + Seq((1, 1), (1, 2)).toDF("c1", "c2").createOrReplaceTempView("t") + + val errMsg = intercept[AnalysisException] { + sql("select (select sum(-1) from t t2 where t1.c2 = t2.c1 group by t2.c2) sum from t t1") + } + assert(errMsg.getMessage.contains( + "A GROUP BY clause in a scalar correlated subquery cannot contain non-correlated columns:")) + } + } + + test("non-aggregated correlated scalar subquery") { + val msg1 = intercept[AnalysisException] { + sql("select a, (select b from l l2 where l2.a = l1.a) sum_b from l l1") + } + assert(msg1.getMessage.contains("Correlated scalar subqueries must be aggregated")) + + val msg2 = intercept[AnalysisException] { + sql("select a, (select b from l l2 where l2.a = l1.a group by 1) sum_b from l l1") + } + assert(msg2.getMessage.contains( + "The output of a correlated scalar subquery must be aggregated")) + } + + test("non-equal correlated scalar subquery") { + val msg1 = intercept[AnalysisException] { + sql("select a, (select sum(b) from l l2 where l2.a < l1.a) sum_b from l l1") + } + assert(msg1.getMessage.contains( + "Correlated column is not allowed in a non-equality predicate:")) + } + + test("disjunctive correlated scalar subquery") { + checkAnswer( + sql(""" + |select a + |from l + |where (select count(*) + | from r + | where (a = c and d = 2.0) or (a = c and d = 1.0)) > 0 + """.stripMargin), + Row(3) :: Nil) + } + + test("SPARK-15370: COUNT bug in WHERE clause (Filter)") { + // Case 1: Canonical example of the COUNT bug + checkAnswer( + sql("select l.a from l where (select count(*) from r where l.a = r.c) < l.a"), + Row(1) :: Row(1) :: Row(3) :: Row(6) :: Nil) + // Case 2: count(*) = 0; could be rewritten to NOT EXISTS but currently uses + // a rewrite that is vulnerable to the COUNT bug + checkAnswer( + sql("select l.a from l where (select count(*) from r where l.a = r.c) = 0"), + Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil) + // Case 3: COUNT bug without a COUNT aggregate + checkAnswer( + sql("select l.a from l where (select sum(r.d) is null from r where l.a = r.c)"), + Row(1) :: Row(1) ::Row(null) :: Row(null) :: Row(6) :: Nil) + } + + test("SPARK-15370: COUNT bug in SELECT clause (Project)") { + checkAnswer( + sql("select a, (select count(*) from r where l.a = r.c) as cnt from l"), + Row(1, 0) :: Row(1, 0) :: Row(2, 2) :: Row(2, 2) :: Row(3, 1) :: Row(null, 0) + :: Row(null, 0) :: Row(6, 1) :: Nil) + } + + test("SPARK-15370: COUNT bug in HAVING clause (Filter)") { + checkAnswer( + sql("select l.a as grp_a from l group by l.a " + + "having (select count(*) from r where grp_a = r.c) = 0 " + + "order by grp_a"), + Row(null) :: Row(1) :: Nil) + } + + test("SPARK-15370: COUNT bug in Aggregate") { + checkAnswer( + sql("select l.a as aval, sum((select count(*) from r where l.a = r.c)) as cnt " + + "from l group by l.a order by aval"), + Row(null, 0) :: Row(1, 0) :: Row(2, 4) :: Row(3, 1) :: Row(6, 1) :: Nil) + } + + test("SPARK-15370: COUNT bug negative examples") { + // Case 1: Potential COUNT bug case that was working correctly prior to the fix + checkAnswer( + sql("select l.a from l where (select sum(r.d) from r where l.a = r.c) is null"), + Row(1) :: Row(1) :: Row(null) :: Row(null) :: Row(6) :: Nil) + // Case 2: COUNT aggregate but no COUNT bug due to > 0 test. + checkAnswer( + sql("select l.a from l where (select count(*) from r where l.a = r.c) > 0"), + Row(2) :: Row(2) :: Row(3) :: Row(6) :: Nil) + // Case 3: COUNT inside aggregate expression but no COUNT bug. + checkAnswer( + sql("select l.a from l where (select count(*) + sum(r.d) from r where l.a = r.c) = 0"), + Nil) + } + + test("SPARK-15370: COUNT bug in subquery in subquery in subquery") { + checkAnswer( + sql("""select l.a from l + |where ( + | select cntPlusOne + 1 as cntPlusTwo from ( + | select cnt + 1 as cntPlusOne from ( + | select sum(r.c) s, count(*) cnt from r where l.a = r.c having cnt = 0 + | ) + | ) + |) = 2""".stripMargin), + Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil) + } + + test("SPARK-15370: COUNT bug with nasty predicate expr") { + checkAnswer( + sql("select l.a from l where " + + "(select case when count(*) = 1 then null else count(*) end as cnt " + + "from r where l.a = r.c) = 0"), + Row(1) :: Row(1) :: Row(null) :: Row(null) :: Nil) + } + + test("SPARK-15370: COUNT bug with attribute ref in subquery input and output ") { + checkAnswer( + sql( + """ + |select l.b, (select (r.c + count(*)) is null + |from r + |where l.a = r.c group by r.c) from l + """.stripMargin), + Row(1.0, false) :: Row(1.0, false) :: Row(2.0, true) :: Row(2.0, true) :: + Row(3.0, false) :: Row(5.0, true) :: Row(null, false) :: Row(null, true) :: Nil) + } + + test("SPARK-16804: Correlated subqueries containing LIMIT - 1") { + withTempView("onerow") { + Seq(1).toDF("c1").createOrReplaceTempView("onerow") + + checkAnswer( + sql( + """ + | select c1 from onerow t1 + | where exists (select 1 from onerow t2 where t1.c1=t2.c1) + | and exists (select 1 from onerow LIMIT 1)""".stripMargin), + Row(1) :: Nil) + } + } + + test("SPARK-16804: Correlated subqueries containing LIMIT - 2") { + withTempView("onerow") { + Seq(1).toDF("c1").createOrReplaceTempView("onerow") + + checkAnswer( + sql( + """ + | select c1 from onerow t1 + | where exists (select 1 + | from (select c1 from onerow t2 LIMIT 1) t2 + | where t1.c1=t2.c1)""".stripMargin), + Row(1) :: Nil) + } + } + + test("SPARK-17337: Incorrect column resolution leads to incorrect results") { + withTempView("t1", "t2") { + Seq(1, 2).toDF("c1").createOrReplaceTempView("t1") + Seq(1).toDF("c2").createOrReplaceTempView("t2") + + checkAnswer( + sql( + """ + | select * + | from (select t2.c2+1 as c3 + | from t1 left join t2 on t1.c1=t2.c2) t3 + | where c3 not in (select c2 from t2)""".stripMargin), + Row(2) :: Nil) + } + } + + test("SPARK-17348: Correlated subqueries with non-equality predicate (good case)") { + withTempView("t1", "t2") { + Seq((1, 1)).toDF("c1", "c2").createOrReplaceTempView("t1") + Seq((1, 1), (2, 0)).toDF("c1", "c2").createOrReplaceTempView("t2") + + // Simple case + checkAnswer( + sql( + """ + | select c1 + | from t1 + | where c1 in (select t2.c1 + | from t2 + | where t1.c2 >= t2.c2)""".stripMargin), + Row(1) :: Nil) + + // More complex case with OR predicate + checkAnswer( + sql( + """ + | select t1.c1 + | from t1, t1 as t3 + | where t1.c1 = t3.c1 + | and (t1.c1 in (select t2.c1 + | from t2 + | where t1.c2 >= t2.c2 + | or t3.c2 < t2.c2) + | or t1.c2 >= 0)""".stripMargin), + Row(1) :: Nil) + } + } + + test("SPARK-17348: Correlated subqueries with non-equality predicate (error case)") { + withTempView("t1", "t2", "t3", "t4") { + Seq((1, 1)).toDF("c1", "c2").createOrReplaceTempView("t1") + Seq((1, 1), (2, 0)).toDF("c1", "c2").createOrReplaceTempView("t2") + Seq((2, 1)).toDF("c1", "c2").createOrReplaceTempView("t3") + Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t4") + + // Simplest case + intercept[AnalysisException] { + sql( + """ + | select t1.c1 + | from t1 + | where t1.c1 in (select max(t2.c1) + | from t2 + | where t1.c2 >= t2.c2)""".stripMargin).collect() + } + + // Add a HAVING on top and augmented within an OR predicate + intercept[AnalysisException] { + sql( + """ + | select t1.c1 + | from t1 + | where t1.c1 in (select max(t2.c1) + | from t2 + | where t1.c2 >= t2.c2 + | having count(*) > 0 ) + | or t1.c2 >= 0""".stripMargin).collect() + } + + // Add a HAVING on top and augmented within an OR predicate + intercept[AnalysisException] { + sql( + """ + | select t1.c1 + | from t1, t1 as t3 + | where t1.c1 = t3.c1 + | and (t1.c1 in (select max(t2.c1) + | from t2 + | where t1.c2 = t2.c2 + | or t3.c2 = t2.c2) + | )""".stripMargin).collect() + } + + // In Window expression: changing the data set to + // demonstrate if this query ran, it would return incorrect result. + intercept[AnalysisException] { + sql( + """ + | select c1 + | from t3 + | where c1 in (select max(t4.c1) over () + | from t4 + | where t3.c2 >= t4.c2)""".stripMargin).collect() + } + } + } + // This restriction applies to + // the permutation of { LOJ, ROJ, FOJ } x { EXISTS, IN, scalar subquery } + // where correlated predicates appears in right operand of LOJ, + // or in left operand of ROJ, or in either operand of FOJ. + // The test cases below cover the representatives of the patterns + test("Correlated subqueries in outer joins") { + withTempView("t1", "t2", "t3") { + Seq(1).toDF("c1").createOrReplaceTempView("t1") + Seq(2).toDF("c1").createOrReplaceTempView("t2") + Seq(1).toDF("c1").createOrReplaceTempView("t3") + + // Left outer join (LOJ) in IN subquery context + intercept[AnalysisException] { + sql( + """ + | select t1.c1 + | from t1 + | where 1 IN (select 1 + | from t3 left outer join + | (select c1 from t2 where t1.c1 = 2) t2 + | on t2.c1 = t3.c1)""".stripMargin).collect() + } + // Right outer join (ROJ) in EXISTS subquery context + intercept[AnalysisException] { + sql( + """ + | select t1.c1 + | from t1 + | where exists (select 1 + | from (select c1 from t2 where t1.c1 = 2) t2 + | right outer join t3 + | on t2.c1 = t3.c1)""".stripMargin).collect() + } + // SPARK-18578: Full outer join (FOJ) in scalar subquery context + intercept[AnalysisException] { + sql( + """ + | select (select max(1) + | from (select c1 from t2 where t1.c1 = 2 and t1.c1=t2.c1) t2 + | full join t3 + | on t2.c1=t3.c1) + | from t1""".stripMargin).collect() + } + } + } + + // Generate operator + test("Correlated subqueries in LATERAL VIEW") { + withTempView("t1", "t2") { + Seq((1, 1), (2, 0)).toDF("c1", "c2").createOrReplaceTempView("t1") + Seq[(Int, Array[Int])]((1, Array(1, 2)), (2, Array(-1, -3))) + .toDF("c1", "arr_c2").createTempView("t2") + checkAnswer( + sql( + """ + | SELECT c2 + | FROM t1 + | WHERE EXISTS (SELECT * + | FROM t2 LATERAL VIEW explode(arr_c2) q AS c2 + WHERE t1.c1 = t2.c1)""".stripMargin), + Row(1) :: Row(0) :: Nil) + + val msg1 = intercept[AnalysisException] { + sql( + """ + | SELECT c1 + | FROM t2 + | WHERE EXISTS (SELECT * + | FROM t1 LATERAL VIEW explode(t2.arr_c2) q AS c2 + | WHERE t1.c1 = t2.c1) + """.stripMargin) + } + assert(msg1.getMessage.contains( + "Expressions referencing the outer query are not supported outside of WHERE/HAVING")) + } + } + + test("SPARK-19933 Do not eliminate top-level aliases in sub-queries") { + withTempView("t1", "t2") { + spark.range(4).createOrReplaceTempView("t1") + checkAnswer( + sql("select * from t1 where id in (select id as id from t1)"), + Row(0) :: Row(1) :: Row(2) :: Row(3) :: Nil) + + spark.range(2).createOrReplaceTempView("t2") + checkAnswer( + sql("select * from t1 where id in (select id as id from t2)"), + Row(0) :: Row(1) :: Nil) + } + } + + test("ListQuery and Exists should work even no correlated references") { + checkAnswer( + sql("select * from l, r where l.a = r.c AND (r.d in (select d from r) OR l.a >= 1)"), + Row(2, 1.0, 2, 3.0) :: Row(2, 1.0, 2, 3.0) :: Row(2, 1.0, 2, 3.0) :: + Row(2, 1.0, 2, 3.0) :: Row(3.0, 3.0, 3, 2.0) :: Row(6, null, 6, null) :: Nil) + checkAnswer( + sql("select * from l, r where l.a = r.c + 1 AND (exists (select * from r) OR l.a = r.c)"), + Row(3, 3.0, 2, 3.0) :: Row(3, 3.0, 2, 3.0) :: Nil) + } + + test("SPARK-20688: correctly check analysis for scalar sub-queries") { + withTempView("t") { + Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("t") + val e = intercept[AnalysisException](sql("SELECT (SELECT count(*) FROM t WHERE a = 1)")) + assert(e.message.contains("cannot resolve '`a`' given input columns: [t.i, t.j]")) + } + } + + test("SPARK-21835: Join in correlated subquery should be duplicateResolved: case 1") { + withTable("t1") { + withTempPath { path => + Seq(1 -> "a").toDF("i", "j").write.parquet(path.getCanonicalPath) + sql(s"CREATE TABLE t1 USING parquet LOCATION '${path.toURI}'") + + val sqlText = + """ + |SELECT * FROM t1 + |WHERE + |NOT EXISTS (SELECT * FROM t1) + """.stripMargin + val optimizedPlan = sql(sqlText).queryExecution.optimizedPlan + val join = optimizedPlan.collectFirst { case j: Join => j }.get + assert(join.duplicateResolved) + assert(optimizedPlan.resolved) + } + } + } + + test("SPARK-21835: Join in correlated subquery should be duplicateResolved: case 2") { + withTable("t1", "t2", "t3") { + withTempPath { path => + val data = Seq((1, 1, 1), (2, 0, 2)) + + data.toDF("t1a", "t1b", "t1c").write.parquet(path.getCanonicalPath + "/t1") + data.toDF("t2a", "t2b", "t2c").write.parquet(path.getCanonicalPath + "/t2") + data.toDF("t3a", "t3b", "t3c").write.parquet(path.getCanonicalPath + "/t3") + + sql(s"CREATE TABLE t1 USING parquet LOCATION '${path.toURI}/t1'") + sql(s"CREATE TABLE t2 USING parquet LOCATION '${path.toURI}/t2'") + sql(s"CREATE TABLE t3 USING parquet LOCATION '${path.toURI}/t3'") + + val sqlText = + s""" + |SELECT * + |FROM (SELECT * + | FROM t2 + | WHERE t2c IN (SELECT t1c + | FROM t1 + | WHERE t1a = t2a) + | UNION + | SELECT * + | FROM t3 + | WHERE t3a IN (SELECT t2a + | FROM t2 + | UNION ALL + | SELECT t1a + | FROM t1 + | WHERE t1b > 0)) t4 + |WHERE t4.t2b IN (SELECT Min(t3b) + | FROM t3 + | WHERE t4.t2a = t3a) + """.stripMargin + val optimizedPlan = sql(sqlText).queryExecution.optimizedPlan + val joinNodes = optimizedPlan.collect { case j: Join => j } + joinNodes.foreach(j => assert(j.duplicateResolved)) + assert(optimizedPlan.resolved) + } + } + } + + test("SPARK-21835: Join in correlated subquery should be duplicateResolved: case 3") { + val sqlText = + """ + |SELECT * FROM l, r WHERE l.a = r.c + 1 AND + |(EXISTS (SELECT * FROM r) OR l.a = r.c) + """.stripMargin + val optimizedPlan = sql(sqlText).queryExecution.optimizedPlan + val join = optimizedPlan.collectFirst { case j: Join => j }.get + assert(join.duplicateResolved) + assert(optimizedPlan.resolved) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala new file mode 100644 index 000000000000..b76f168220d8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/TypedImperativeAggregateSuite.scala @@ -0,0 +1,303 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} + +import org.apache.spark.sql.TypedImperativeAggregateSuite.TypedMax +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{BoundReference, Expression, GenericInternalRow, ImplicitCastInputTypes, SpecificInternalRow} +import org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate +import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.expressions.Window +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ + +class TypedImperativeAggregateSuite extends QueryTest with SharedSQLContext { + + import testImplicits._ + + private val random = new java.util.Random() + + private val data = (0 until 1000).map { _ => + (random.nextInt(10), random.nextInt(100)) + } + + test("aggregate with object aggregate buffer") { + val agg = new TypedMax(BoundReference(0, IntegerType, nullable = false)) + + val group1 = (0 until data.length / 2) + val group1Buffer = agg.createAggregationBuffer() + group1.foreach { index => + val input = InternalRow(data(index)._1, data(index)._2) + agg.update(group1Buffer, input) + } + + val group2 = (data.length / 2 until data.length) + val group2Buffer = agg.createAggregationBuffer() + group2.foreach { index => + val input = InternalRow(data(index)._1, data(index)._2) + agg.update(group2Buffer, input) + } + + val mergeBuffer = agg.createAggregationBuffer() + agg.merge(mergeBuffer, group1Buffer) + agg.merge(mergeBuffer, group2Buffer) + + assert(mergeBuffer.value == data.map(_._1).max) + assert(agg.eval(mergeBuffer) == data.map(_._1).max) + + // Tests low level eval(row: InternalRow) API. + val row = new GenericInternalRow(Array(mergeBuffer): Array[Any]) + + // Evaluates directly on row consist of aggregation buffer object. + assert(agg.eval(row) == data.map(_._1).max) + } + + test("supports SpecificMutableRow as mutable row") { + val aggregationBufferSchema = Seq(IntegerType, LongType, BinaryType, IntegerType) + val aggBufferOffset = 2 + val buffer = new SpecificInternalRow(aggregationBufferSchema) + val agg = new TypedMax(BoundReference(ordinal = 1, dataType = IntegerType, nullable = false)) + .withNewMutableAggBufferOffset(aggBufferOffset) + + agg.initialize(buffer) + data.foreach { kv => + val input = InternalRow(kv._1, kv._2) + agg.update(buffer, input) + } + assert(agg.eval(buffer) == data.map(_._2).max) + } + + test("dataframe aggregate with object aggregate buffer, should not use HashAggregate") { + val df = data.toDF("a", "b") + val max = TypedMax($"a".expr) + + // Always uses SortAggregateExec + val sparkPlan = df.select(Column(max.toAggregateExpression())).queryExecution.sparkPlan + assert(!sparkPlan.isInstanceOf[HashAggregateExec]) + } + + test("dataframe aggregate with object aggregate buffer, no group by") { + val df = data.toDF("key", "value").coalesce(2) + val query = df.select(typedMax($"key"), count($"key"), typedMax($"value"), count($"value")) + val maxKey = data.map(_._1).max + val countKey = data.size + val maxValue = data.map(_._2).max + val countValue = data.size + val expected = Seq(Row(maxKey, countKey, maxValue, countValue)) + checkAnswer(query, expected) + } + + test("dataframe aggregate with object aggregate buffer, non-nullable aggregator") { + val df = data.toDF("key", "value").coalesce(2) + + // Test non-nullable typedMax + val query = df.select(typedMax(lit(null)), count($"key"), typedMax(lit(null)), + count($"value")) + + // typedMax is not nullable + val maxNull = Int.MinValue + val countKey = data.size + val countValue = data.size + val expected = Seq(Row(maxNull, countKey, maxNull, countValue)) + checkAnswer(query, expected) + } + + test("dataframe aggregate with object aggregate buffer, nullable aggregator") { + val df = data.toDF("key", "value").coalesce(2) + + // Test nullable nullableTypedMax + val query = df.select(nullableTypedMax(lit(null)), count($"key"), nullableTypedMax(lit(null)), + count($"value")) + + // nullableTypedMax is nullable + val maxNull = null + val countKey = data.size + val countValue = data.size + val expected = Seq(Row(maxNull, countKey, maxNull, countValue)) + checkAnswer(query, expected) + } + + test("dataframe aggregation with object aggregate buffer, input row contains null") { + + val nullableData = (0 until 1000).map {id => + val nullableKey: Integer = if (random.nextBoolean()) null else random.nextInt(100) + val nullableValue: Integer = if (random.nextBoolean()) null else random.nextInt(100) + (nullableKey, nullableValue) + } + + val df = nullableData.toDF("key", "value").coalesce(2) + val query = df.select(typedMax($"key"), count($"key"), typedMax($"value"), + count($"value")) + val maxKey = nullableData.map(_._1).filter(_ != null).max + val countKey = nullableData.map(_._1).filter(_ != null).size + val maxValue = nullableData.map(_._2).filter(_ != null).max + val countValue = nullableData.map(_._2).filter(_ != null).size + val expected = Seq(Row(maxKey, countKey, maxValue, countValue)) + checkAnswer(query, expected) + } + + test("dataframe aggregate with object aggregate buffer, with group by") { + val df = data.toDF("value", "key").coalesce(2) + val query = df.groupBy($"key").agg(typedMax($"value"), count($"value"), typedMax($"value")) + val expected = data.groupBy(_._2).toSeq.map { group => + val (key, values) = group + val valueMax = values.map(_._1).max + val countValue = values.size + Row(key, valueMax, countValue, valueMax) + } + checkAnswer(query, expected) + } + + test("dataframe aggregate with object aggregate buffer, empty inputs, no group by") { + val empty = Seq.empty[(Int, Int)].toDF("a", "b") + checkAnswer( + empty.select(typedMax($"a"), count($"a"), typedMax($"b"), count($"b")), + Seq(Row(Int.MinValue, 0, Int.MinValue, 0))) + } + + test("dataframe aggregate with object aggregate buffer, empty inputs, with group by") { + val empty = Seq.empty[(Int, Int)].toDF("a", "b") + checkAnswer( + empty.groupBy($"b").agg(typedMax($"a"), count($"a"), typedMax($"a")), + Seq.empty[Row]) + } + + test("TypedImperativeAggregate should not break Window function") { + val df = data.toDF("key", "value") + // OVER (PARTITION BY a ORDER BY b ROW BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) + val w = Window.orderBy("value").partitionBy("key").rowsBetween(Long.MinValue, 0) + + val query = df.select(sum($"key").over(w), typedMax($"key").over(w), sum($"value").over(w), + typedMax($"value").over(w)) + + val expected = data.groupBy(_._1).toSeq.flatMap { group => + val (key, values) = group + val sortedValues = values.map(_._2).sorted + + var outputRows = Seq.empty[Row] + var i = 0 + while (i < sortedValues.size) { + val unboundedPrecedingAndCurrent = sortedValues.slice(0, i + 1) + val sumKey = key * unboundedPrecedingAndCurrent.size + val maxKey = key + val sumValue = unboundedPrecedingAndCurrent.sum + val maxValue = unboundedPrecedingAndCurrent.max + + outputRows :+= Row(sumKey, maxKey, sumValue, maxValue) + i += 1 + } + + outputRows + } + checkAnswer(query, expected) + } + + private def typedMax(column: Column): Column = { + val max = TypedMax(column.expr, nullable = false) + Column(max.toAggregateExpression()) + } + + private def nullableTypedMax(column: Column): Column = { + val max = TypedMax(column.expr, nullable = true) + Column(max.toAggregateExpression()) + } +} + +object TypedImperativeAggregateSuite { + + /** + * Calculate the max value with object aggregation buffer. This stores class MaxValue + * in aggregation buffer. + */ + private case class TypedMax( + child: Expression, + nullable: Boolean = false, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) + extends TypedImperativeAggregate[MaxValue] with ImplicitCastInputTypes { + + + override def createAggregationBuffer(): MaxValue = { + // Returns Int.MinValue if all inputs are null + new MaxValue(Int.MinValue) + } + + override def update(buffer: MaxValue, input: InternalRow): MaxValue = { + child.eval(input) match { + case inputValue: Int => + if (inputValue > buffer.value) { + buffer.value = inputValue + buffer.isValueSet = true + } + case null => // skip + } + buffer + } + + override def merge(bufferMax: MaxValue, inputMax: MaxValue): MaxValue = { + if (inputMax.value > bufferMax.value) { + bufferMax.value = inputMax.value + bufferMax.isValueSet = bufferMax.isValueSet || inputMax.isValueSet + } + bufferMax + } + + override def eval(bufferMax: MaxValue): Any = { + if (nullable && bufferMax.isValueSet == false) { + null + } else { + bufferMax.value + } + } + + override def deterministic: Boolean = true + + override def children: Seq[Expression] = Seq(child) + + override def inputTypes: Seq[AbstractDataType] = Seq(IntegerType) + + override def dataType: DataType = IntegerType + + override def withNewMutableAggBufferOffset(newOffset: Int): TypedImperativeAggregate[MaxValue] = + copy(mutableAggBufferOffset = newOffset) + + override def withNewInputAggBufferOffset(newOffset: Int): TypedImperativeAggregate[MaxValue] = + copy(inputAggBufferOffset = newOffset) + + override def serialize(buffer: MaxValue): Array[Byte] = { + val out = new ByteArrayOutputStream() + val stream = new DataOutputStream(out) + stream.writeBoolean(buffer.isValueSet) + stream.writeInt(buffer.value) + out.toByteArray + } + + override def deserialize(storageFormat: Array[Byte]): MaxValue = { + val in = new ByteArrayInputStream(storageFormat) + val stream = new DataInputStream(in) + val isValueSet = stream.readBoolean() + val value = stream.readInt() + new MaxValue(value, isValueSet) + } + } + + private class MaxValue(var value: Int, var isValueSet: Boolean = false) +} 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 6e510f0b8aff..7f1c009ca6e7 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,10 +17,12 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.expressions.ScalaUDF -import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.execution.command.ExplainCommand +import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SQLTestData._ +import org.apache.spark.sql.types.DataTypes private case class FunctionResult(f1: String, f2: String) @@ -28,7 +30,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("built-in fixed arity expressions") { - val df = sqlContext.emptyDataFrame + val df = spark.emptyDataFrame df.selectExpr("rand()", "randn()", "rand(5)", "randn(50)") } @@ -55,60 +57,90 @@ class UDFSuite extends QueryTest with SharedSQLContext { test("SPARK-8003 spark_partition_id") { val df = Seq((1, "Tearing down the walls that divide us")).toDF("id", "saying") - df.registerTempTable("tmp_table") + df.createOrReplaceTempView("tmp_table") checkAnswer(sql("select spark_partition_id() from tmp_table").toDF(), Row(0)) - sqlContext.dropTempTable("tmp_table") + spark.catalog.dropTempView("tmp_table") } test("SPARK-8005 input_file_name") { withTempPath { dir => val data = sparkContext.parallelize(0 to 10, 2).toDF("id") data.write.parquet(dir.getCanonicalPath) - sqlContext.read.parquet(dir.getCanonicalPath).registerTempTable("test_table") + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("test_table") val answer = sql("select input_file_name() from test_table").head().getString(0) - assert(answer.contains(dir.getCanonicalPath)) + assert(answer.contains(dir.toURI.getPath)) assert(sql("select input_file_name() from test_table").distinct().collect().length >= 2) - sqlContext.dropTempTable("test_table") + spark.catalog.dropTempView("test_table") } } - test("error reporting for incorrect number of arguments") { - val df = sqlContext.emptyDataFrame + test("error reporting for incorrect number of arguments - builtin function") { + val df = spark.emptyDataFrame val e = intercept[AnalysisException] { df.selectExpr("substr('abcd', 2, 3, 4)") } - assert(e.getMessage.contains("arguments")) + assert(e.getMessage.contains("Invalid number of arguments for function substr")) + } + + test("error reporting for incorrect number of arguments - udf") { + val df = spark.emptyDataFrame + val e = intercept[AnalysisException] { + spark.udf.register("foo", (_: String).length) + df.selectExpr("foo(2, 3, 4)") + } + assert(e.getMessage.contains("Invalid number of arguments for function foo")) } test("error reporting for undefined functions") { - val df = sqlContext.emptyDataFrame + val df = spark.emptyDataFrame val e = intercept[AnalysisException] { df.selectExpr("a_function_that_does_not_exist()") } - assert(e.getMessage.contains("undefined function")) + assert(e.getMessage.contains("Undefined function")) + assert(e.getMessage.contains("a_function_that_does_not_exist")) } test("Simple UDF") { - sqlContext.udf.register("strLenScala", (_: String).length) + spark.udf.register("strLenScala", (_: String).length) assert(sql("SELECT strLenScala('test')").head().getInt(0) === 4) } - test("ZeroArgument UDF") { - sqlContext.udf.register("random0", () => { Math.random()}) - assert(sql("SELECT random0()").head().getDouble(0) >= 0.0) + test("UDF defined using UserDefinedFunction") { + import functions.udf + val foo = udf((x: Int) => x + 1) + spark.udf.register("foo", foo) + assert(sql("select foo(5)").head().getInt(0) == 6) + } + + test("ZeroArgument non-deterministic UDF") { + val foo = udf(() => Math.random()) + spark.udf.register("random0", foo.asNondeterministic()) + val df = sql("SELECT random0()") + assert(df.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) + assert(df.head().getDouble(0) >= 0.0) + + val foo1 = foo.asNondeterministic() + val df1 = testData.select(foo1()) + assert(df1.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) + assert(df1.head().getDouble(0) >= 0.0) + + val bar = udf(() => Math.random(), DataTypes.DoubleType).asNondeterministic() + val df2 = testData.select(bar()) + assert(df2.logicalPlan.asInstanceOf[Project].projectList.forall(!_.deterministic)) + assert(df2.head().getDouble(0) >= 0.0) } test("TwoArgument UDF") { - sqlContext.udf.register("strLenScala", (_: String).length + (_: Int)) + spark.udf.register("strLenScala", (_: String).length + (_: Int)) assert(sql("SELECT strLenScala('test', 1)").head().getInt(0) === 5) } test("UDF in a WHERE") { - sqlContext.udf.register("oneArgFilter", (n: Int) => { n > 80 }) + spark.udf.register("oneArgFilter", (n: Int) => { n > 80 }) val df = sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))).toDF() - df.registerTempTable("integerData") + df.createOrReplaceTempView("integerData") val result = sql("SELECT * FROM integerData WHERE oneArgFilter(key)") @@ -116,11 +148,11 @@ class UDFSuite extends QueryTest with SharedSQLContext { } test("UDF in a HAVING") { - sqlContext.udf.register("havingFilter", (n: Long) => { n > 5 }) + spark.udf.register("havingFilter", (n: Long) => { n > 5 }) val df = Seq(("red", 1), ("red", 2), ("blue", 10), ("green", 100), ("green", 200)).toDF("g", "v") - df.registerTempTable("groupData") + df.createOrReplaceTempView("groupData") val result = sql( @@ -135,11 +167,11 @@ class UDFSuite extends QueryTest with SharedSQLContext { } test("UDF in a GROUP BY") { - sqlContext.udf.register("groupFunction", (n: Int) => { n > 10 }) + spark.udf.register("groupFunction", (n: Int) => { n > 10 }) val df = Seq(("red", 1), ("red", 2), ("blue", 10), ("green", 100), ("green", 200)).toDF("g", "v") - df.registerTempTable("groupData") + df.createOrReplaceTempView("groupData") val result = sql( @@ -152,14 +184,14 @@ class UDFSuite extends QueryTest with SharedSQLContext { } test("UDFs everywhere") { - sqlContext.udf.register("groupFunction", (n: Int) => { n > 10 }) - sqlContext.udf.register("havingFilter", (n: Long) => { n > 2000 }) - sqlContext.udf.register("whereFilter", (n: Int) => { n < 150 }) - sqlContext.udf.register("timesHundred", (n: Long) => { n * 100 }) + spark.udf.register("groupFunction", (n: Int) => { n > 10 }) + spark.udf.register("havingFilter", (n: Long) => { n > 2000 }) + spark.udf.register("whereFilter", (n: Int) => { n < 150 }) + spark.udf.register("timesHundred", (n: Long) => { n * 100 }) val df = Seq(("red", 1), ("red", 2), ("blue", 10), ("green", 100), ("green", 200)).toDF("g", "v") - df.registerTempTable("groupData") + df.createOrReplaceTempView("groupData") val result = sql( @@ -174,7 +206,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { } test("struct UDF") { - sqlContext.udf.register("returnStruct", (f1: String, f2: String) => FunctionResult(f1, f2)) + spark.udf.register("returnStruct", (f1: String, f2: String) => FunctionResult(f1, f2)) val result = sql("SELECT returnStruct('test', 'test2') as ret") @@ -183,117 +215,85 @@ class UDFSuite extends QueryTest with SharedSQLContext { } test("udf that is transformed") { - sqlContext.udf.register("makeStruct", (x: Int, y: Int) => (x, y)) + spark.udf.register("makeStruct", (x: Int, y: Int) => (x, y)) // 1 + 1 is constant folded causing a transformation. assert(sql("SELECT makeStruct(1 + 1, 2)").first().getAs[Row](0) === Row(2, 2)) } test("type coercion for udf inputs") { - sqlContext.udf.register("intExpected", (x: Int) => x) + spark.udf.register("intExpected", (x: Int) => x) // pass a decimal to intExpected. assert(sql("SELECT intExpected(1.0)").head().getInt(0) === 1) } - private def checkNumUDFs(df: DataFrame, expectedNumUDFs: Int): Unit = { - val udfs = df.queryExecution.optimizedPlan.collect { - case p: logical.Project => p.projectList.flatMap { - case e => e.collect { - case udf: ScalaUDF => udf - } - } - }.flatten - assert(udfs.length === expectedNumUDFs) - } - - test("foldable udf") { - import org.apache.spark.sql.functions._ - - val myUDF = udf((x: Int) => x + 1) - - { - val df = sql("SELECT 1 as a") - .select(col("a"), myUDF(col("a")).as("b")) - .select(col("a"), col("b"), myUDF(col("b")).as("c")) - checkNumUDFs(df, 0) - checkAnswer(df, Row(1, 2, 3)) - } + test("udf in different types") { + spark.udf.register("testDataFunc", (n: Int, s: String) => { (n, s) }) + spark.udf.register("decimalDataFunc", + (a: java.math.BigDecimal, b: java.math.BigDecimal) => { (a, b) }) + spark.udf.register("binaryDataFunc", (a: Array[Byte], b: Int) => { (a, b) }) + spark.udf.register("arrayDataFunc", + (data: Seq[Int], nestedData: Seq[Seq[Int]]) => { (data, nestedData) }) + spark.udf.register("mapDataFunc", + (data: scala.collection.Map[Int, String]) => { data }) + spark.udf.register("complexDataFunc", + (m: Map[String, Int], a: Seq[Int], b: Boolean) => { (m, a, b) } ) + + checkAnswer( + sql("SELECT tmp.t.* FROM (SELECT testDataFunc(key, value) AS t from testData) tmp").toDF(), + testData) + checkAnswer( + sql(""" + | SELECT tmp.t.* FROM + | (SELECT decimalDataFunc(a, b) AS t FROM decimalData) tmp + """.stripMargin).toDF(), decimalData) + checkAnswer( + sql(""" + | SELECT tmp.t.* FROM + | (SELECT binaryDataFunc(a, b) AS t FROM binaryData) tmp + """.stripMargin).toDF(), binaryData) + checkAnswer( + sql(""" + | SELECT tmp.t.* FROM + | (SELECT arrayDataFunc(data, nestedData) AS t FROM arrayData) tmp + """.stripMargin).toDF(), arrayData.toDF()) + checkAnswer( + sql(""" + | SELECT mapDataFunc(data) AS t FROM mapData + """.stripMargin).toDF(), mapData.toDF()) + checkAnswer( + sql(""" + | SELECT tmp.t.* FROM + | (SELECT complexDataFunc(m, a, b) AS t FROM complexData) tmp + """.stripMargin).toDF(), complexData.select("m", "a", "b")) } - test("nondeterministic udf: using UDFRegistration") { - import org.apache.spark.sql.functions._ - - val myUDF = sqlContext.udf.register("plusOne1", (x: Int) => x + 1) - sqlContext.udf.register("plusOne2", myUDF.nondeterministic) - - { - val df = sqlContext.range(1, 2).select(col("id").as("a")) - .select(col("a"), myUDF(col("a")).as("b")) - .select(col("a"), col("b"), myUDF(col("b")).as("c")) - checkNumUDFs(df, 3) - checkAnswer(df, Row(1, 2, 3)) - } - - { - val df = sqlContext.range(1, 2).select(col("id").as("a")) - .select(col("a"), callUDF("plusOne1", col("a")).as("b")) - .select(col("a"), col("b"), callUDF("plusOne1", col("b")).as("c")) - checkNumUDFs(df, 3) - checkAnswer(df, Row(1, 2, 3)) - } + test("SPARK-11716 UDFRegistration does not include the input data type in returned UDF") { + val myUDF = spark.udf.register("testDataFunc", (n: Int, s: String) => { (n, s.toInt) }) - { - val df = sqlContext.range(1, 2).select(col("id").as("a")) - .select(col("a"), myUDF.nondeterministic(col("a")).as("b")) - .select(col("a"), col("b"), myUDF.nondeterministic(col("b")).as("c")) - checkNumUDFs(df, 2) - checkAnswer(df, Row(1, 2, 3)) - } + // Without the fix, this will fail because we fail to cast data type of b to string + // because myUDF does not know its input data type. With the fix, this query should not + // fail. + checkAnswer( + testData2.select(myUDF($"a", $"b").as("t")), + testData2.selectExpr("struct(a, b)")) - { - val df = sqlContext.range(1, 2).select(col("id").as("a")) - .select(col("a"), callUDF("plusOne2", col("a")).as("b")) - .select(col("a"), col("b"), callUDF("plusOne2", col("b")).as("c")) - checkNumUDFs(df, 2) - checkAnswer(df, Row(1, 2, 3)) - } + checkAnswer( + sql("SELECT tmp.t.* FROM (SELECT testDataFunc(a, b) AS t from testData2) tmp").toDF(), + testData2) } - test("nondeterministic udf: using udf function") { - import org.apache.spark.sql.functions._ - - val myUDF = udf((x: Int) => x + 1) - - { - val df = sqlContext.range(1, 2).select(col("id").as("a")) - .select(col("a"), myUDF(col("a")).as("b")) - .select(col("a"), col("b"), myUDF(col("b")).as("c")) - checkNumUDFs(df, 3) - checkAnswer(df, Row(1, 2, 3)) - } - - { - val df = sqlContext.range(1, 2).select(col("id").as("a")) - .select(col("a"), myUDF.nondeterministic(col("a")).as("b")) - .select(col("a"), col("b"), myUDF.nondeterministic(col("b")).as("c")) - checkNumUDFs(df, 2) - checkAnswer(df, Row(1, 2, 3)) - } - - { - // nondeterministicUDF will not be foldable. - val df = sql("SELECT 1 as a") - .select(col("a"), myUDF.nondeterministic(col("a")).as("b")) - .select(col("a"), col("b"), myUDF.nondeterministic(col("b")).as("c")) - checkNumUDFs(df, 2) - checkAnswer(df, Row(1, 2, 3)) + test("SPARK-19338 Provide identical names for UDFs in the EXPLAIN output") { + def explainStr(df: DataFrame): String = { + val explain = ExplainCommand(df.queryExecution.logical, extended = false) + val sparkPlan = spark.sessionState.executePlan(explain).executedPlan + sparkPlan.executeCollect().map(_.getString(0).trim).headOption.getOrElse("") } - } - - test("override a registered udf") { - sqlContext.udf.register("intExpected", (x: Int) => x) - assert(sql("SELECT intExpected(1.0)").head().getInt(0) === 1) - - sqlContext.udf.register("intExpected", (x: Int) => x + 1) - assert(sql("SELECT intExpected(1.0)").head().getInt(0) === 2) + val udf1Name = "myUdf1" + val udf2Name = "myUdf2" + val udf1 = spark.udf.register(udf1Name, (n: Int) => n + 1) + val udf2 = spark.udf.register(udf2Name, (n: Int) => n * 1) + assert(explainStr(sql("SELECT myUdf1(myUdf2(1))")).contains(s"UDF:$udf1Name(UDF:$udf2Name(1))")) + assert(explainStr(spark.range(1).select(udf1(udf2(functions.lit(1))))) + .contains(s"UDF:$udf1Name(UDF:$udf2Name(1))")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDTRegistrationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDTRegistrationSuite.scala new file mode 100644 index 000000000000..d61ede780a74 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDTRegistrationSuite.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.sql.types._ + +private[sql] class TestUserClass { +} + +private[sql] class TestUserClass2 { +} + +private[sql] class TestUserClass3 { +} + +private[sql] class NonUserDefinedType { +} + +private[sql] class TestUserClassUDT extends UserDefinedType[TestUserClass] { + + override def sqlType: DataType = IntegerType + override def serialize(input: TestUserClass): Int = 1 + + override def deserialize(datum: Any): TestUserClass = new TestUserClass + + override def userClass: Class[TestUserClass] = classOf[TestUserClass] + + private[spark] override def asNullable: TestUserClassUDT = this + + override def hashCode(): Int = classOf[TestUserClassUDT].getName.hashCode() + + override def equals(other: Any): Boolean = other match { + case _: TestUserClassUDT => true + case _ => false + } +} + +class UDTRegistrationSuite extends SparkFunSuite { + + test("register non-UserDefinedType") { + UDTRegistration.register(classOf[TestUserClass].getName, + "org.apache.spark.sql.NonUserDefinedType") + intercept[SparkException] { + UDTRegistration.getUDTFor(classOf[TestUserClass].getName) + } + } + + test("default UDTs") { + val userClasses = Seq( + "org.apache.spark.ml.linalg.Vector", + "org.apache.spark.ml.linalg.DenseVector", + "org.apache.spark.ml.linalg.SparseVector", + "org.apache.spark.ml.linalg.Matrix", + "org.apache.spark.ml.linalg.DenseMatrix", + "org.apache.spark.ml.linalg.SparseMatrix") + userClasses.foreach { c => + assert(UDTRegistration.exists(c)) + } + } + + test("query registered user class") { + UDTRegistration.register(classOf[TestUserClass2].getName, classOf[TestUserClassUDT].getName) + assert(UDTRegistration.exists(classOf[TestUserClass2].getName)) + assert( + classOf[UserDefinedType[_]].isAssignableFrom(( + UDTRegistration.getUDTFor(classOf[TestUserClass2].getName).get))) + } + + test("query unregistered user class") { + assert(!UDTRegistration.exists(classOf[TestUserClass3].getName)) + assert(!UDTRegistration.getUDTFor(classOf[TestUserClass3].getName).isDefined) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala index 00f1526576cc..a5f904c621e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala @@ -34,8 +34,8 @@ class UnsafeRowSuite extends SparkFunSuite { test("UnsafeRow Java serialization") { // serializing an UnsafeRow pointing to a large buffer should only serialize the relevant data val data = new Array[Byte](1024) - val row = new UnsafeRow - row.pointTo(data, 1, 16) + val row = new UnsafeRow(1) + row.pointTo(data, 16) row.setLong(0, 19285) val ser = new JavaSerializer(new SparkConf).newInstance() @@ -47,8 +47,8 @@ class UnsafeRowSuite extends SparkFunSuite { test("UnsafeRow Kryo serialization") { // serializing an UnsafeRow pointing to a large buffer should only serialize the relevant data val data = new Array[Byte](1024) - val row = new UnsafeRow - row.pointTo(data, 1, 16) + val row = new UnsafeRow(1) + row.pointTo(data, 16) row.setLong(0, 19285) val ser = new KryoSerializer(new SparkConf).newInstance() @@ -86,11 +86,10 @@ class UnsafeRowSuite extends SparkFunSuite { offheapRowPage.getBaseOffset, arrayBackedUnsafeRow.getSizeInBytes ) - val offheapUnsafeRow: UnsafeRow = new UnsafeRow() + val offheapUnsafeRow: UnsafeRow = new UnsafeRow(3) offheapUnsafeRow.pointTo( offheapRowPage.getBaseObject, offheapRowPage.getBaseOffset, - 3, // num fields arrayBackedUnsafeRow.getSizeInBytes ) assert(offheapUnsafeRow.getBaseObject === null) @@ -102,9 +101,22 @@ class UnsafeRowSuite extends SparkFunSuite { MemoryAllocator.UNSAFE.free(offheapRowPage) } } + val (bytesFromArrayBackedRowWithOffset, field0StringFromArrayBackedRowWithOffset) = { + val baos = new ByteArrayOutputStream() + val numBytes = arrayBackedUnsafeRow.getSizeInBytes + val bytesWithOffset = new Array[Byte](numBytes + 100) + System.arraycopy(arrayBackedUnsafeRow.getBaseObject.asInstanceOf[Array[Byte]], 0, + bytesWithOffset, 100, numBytes) + val arrayBackedRow = new UnsafeRow(arrayBackedUnsafeRow.numFields()) + arrayBackedRow.pointTo(bytesWithOffset, Platform.BYTE_ARRAY_OFFSET + 100, numBytes) + arrayBackedRow.writeToStream(baos, null) + (baos.toByteArray, arrayBackedRow.getString(0)) + } assert(bytesFromArrayBackedRow === bytesFromOffheapRow) assert(field0StringFromArrayBackedRow === field0StringFromOffheapRow) + assert(bytesFromArrayBackedRow === bytesFromArrayBackedRowWithOffset) + assert(field0StringFromArrayBackedRow === field0StringFromArrayBackedRowWithOffset) } test("calling getDouble() and getFloat() on null columns") { 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 a229e5814df8..a08433ba794d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -17,66 +17,142 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.util.{GenericArrayData, ArrayData} - import scala.beans.{BeanInfo, BeanProperty} -import com.clearspring.analytics.stream.cardinality.HyperLogLog - import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.CatalystTypeConverters -import org.apache.spark.sql.catalyst.expressions.{OpenHashSetUDT, HyperLogLogUDT} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} import org.apache.spark.sql.execution.datasources.parquet.ParquetTest import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils -import org.apache.spark.util.collection.OpenHashSet +@BeanInfo +private[sql] case class MyLabeledPoint( + @BeanProperty label: Double, + @BeanProperty features: UDT.MyDenseVector) + +// Wrapped in an object to check Scala compatibility. See SPARK-13929 +object UDT { + + @SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) + private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { + override def hashCode(): Int = java.util.Arrays.hashCode(data) -@SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) -private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { - override def equals(other: Any): Boolean = other match { - case v: MyDenseVector => - java.util.Arrays.equals(this.data, v.data) - case _ => false + override def equals(other: Any): Boolean = other match { + case v: MyDenseVector => java.util.Arrays.equals(this.data, v.data) + case _ => false + } } -} -@BeanInfo -private[sql] case class MyLabeledPoint( - @BeanProperty label: Double, - @BeanProperty features: MyDenseVector) + private[sql] class MyDenseVectorUDT extends UserDefinedType[MyDenseVector] { -private[sql] class MyDenseVectorUDT extends UserDefinedType[MyDenseVector] { + override def sqlType: DataType = ArrayType(DoubleType, containsNull = false) - override def sqlType: DataType = ArrayType(DoubleType, containsNull = false) + override def serialize(features: MyDenseVector): ArrayData = { + new GenericArrayData(features.data.map(_.asInstanceOf[Any])) + } - override def serialize(obj: Any): ArrayData = { - obj match { - case features: MyDenseVector => - new GenericArrayData(features.data.map(_.asInstanceOf[Any])) + override def deserialize(datum: Any): MyDenseVector = { + datum match { + case data: ArrayData => + new MyDenseVector(data.toDoubleArray()) + } } + + override def userClass: Class[MyDenseVector] = classOf[MyDenseVector] + + private[spark] override def asNullable: MyDenseVectorUDT = this + + override def hashCode(): Int = getClass.hashCode() + + override def equals(other: Any): Boolean = other.isInstanceOf[MyDenseVectorUDT] + } + +} + +// object and classes to test SPARK-19311 + +// Trait/Interface for base type +sealed trait IExampleBaseType extends Serializable { + def field: Int +} + +// Trait/Interface for derived type +sealed trait IExampleSubType extends IExampleBaseType + +// a base class +class ExampleBaseClass(override val field: Int) extends IExampleBaseType + +// a derived class +class ExampleSubClass(override val field: Int) + extends ExampleBaseClass(field) with IExampleSubType + +// UDT for base class +class ExampleBaseTypeUDT extends UserDefinedType[IExampleBaseType] { + + override def sqlType: StructType = { + StructType(Seq( + StructField("intfield", IntegerType, nullable = false))) + } + + override def serialize(obj: IExampleBaseType): InternalRow = { + val row = new GenericInternalRow(1) + row.setInt(0, obj.field) + row } - override def deserialize(datum: Any): MyDenseVector = { + override def deserialize(datum: Any): IExampleBaseType = { datum match { - case data: ArrayData => - new MyDenseVector(data.toDoubleArray()) + case row: InternalRow => + require(row.numFields == 1, + "ExampleBaseTypeUDT requires row with length == 1") + val field = row.getInt(0) + new ExampleBaseClass(field) } } - override def userClass: Class[MyDenseVector] = classOf[MyDenseVector] + override def userClass: Class[IExampleBaseType] = classOf[IExampleBaseType] +} + +// UDT for derived class +private[spark] class ExampleSubTypeUDT extends UserDefinedType[IExampleSubType] { + + override def sqlType: StructType = { + StructType(Seq( + StructField("intfield", IntegerType, nullable = false))) + } - private[spark] override def asNullable: MyDenseVectorUDT = this + override def serialize(obj: IExampleSubType): InternalRow = { + val row = new GenericInternalRow(1) + row.setInt(0, obj.field) + row + } + + override def deserialize(datum: Any): IExampleSubType = { + datum match { + case row: InternalRow => + require(row.numFields == 1, + "ExampleSubTypeUDT requires row with length == 1") + val field = row.getInt(0) + new ExampleSubClass(field) + } + } + + override def userClass: Class[IExampleSubType] = classOf[IExampleSubType] } class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetTest { import testImplicits._ private lazy val pointsRDD = Seq( - MyLabeledPoint(1.0, new MyDenseVector(Array(0.1, 1.0))), - MyLabeledPoint(0.0, new MyDenseVector(Array(0.2, 2.0)))).toDF() + MyLabeledPoint(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), + MyLabeledPoint(0.0, new UDT.MyDenseVector(Array(0.2, 2.0)))).toDF() + + private lazy val pointsRDD2 = Seq( + MyLabeledPoint(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), + MyLabeledPoint(0.0, new UDT.MyDenseVector(Array(0.3, 3.0)))).toDF() test("register user type: MyDenseVector for MyLabeledPoint") { val labels: RDD[Double] = pointsRDD.select('label).rdd.map { case Row(v: Double) => v } @@ -85,17 +161,17 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT assert(labelsArrays.contains(1.0)) assert(labelsArrays.contains(0.0)) - val features: RDD[MyDenseVector] = - pointsRDD.select('features).rdd.map { case Row(v: MyDenseVector) => v } - val featuresArrays: Array[MyDenseVector] = features.collect() + val features: RDD[UDT.MyDenseVector] = + pointsRDD.select('features).rdd.map { case Row(v: UDT.MyDenseVector) => v } + val featuresArrays: Array[UDT.MyDenseVector] = features.collect() assert(featuresArrays.size === 2) - assert(featuresArrays.contains(new MyDenseVector(Array(0.1, 1.0)))) - assert(featuresArrays.contains(new MyDenseVector(Array(0.2, 2.0)))) + assert(featuresArrays.contains(new UDT.MyDenseVector(Array(0.1, 1.0)))) + assert(featuresArrays.contains(new UDT.MyDenseVector(Array(0.2, 2.0)))) } test("UDTs and UDFs") { - sqlContext.udf.register("testType", (d: MyDenseVector) => d.isInstanceOf[MyDenseVector]) - pointsRDD.registerTempTable("points") + spark.udf.register("testType", (d: UDT.MyDenseVector) => d.isInstanceOf[UDT.MyDenseVector]) + pointsRDD.createOrReplaceTempView("points") checkAnswer( sql("SELECT testType(features) from points"), Seq(Row(true), Row(true))) @@ -106,10 +182,10 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT val path = dir.getCanonicalPath pointsRDD.write.parquet(path) checkAnswer( - sqlContext.read.parquet(path), + spark.read.parquet(path), Seq( - Row(1.0, new MyDenseVector(Array(0.1, 1.0))), - Row(0.0, new MyDenseVector(Array(0.2, 2.0))))) + Row(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), + Row(0.0, new UDT.MyDenseVector(Array(0.2, 2.0))))) } } @@ -118,39 +194,21 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT val path = dir.getCanonicalPath pointsRDD.repartition(1).write.parquet(path) checkAnswer( - sqlContext.read.parquet(path), + spark.read.parquet(path), Seq( - Row(1.0, new MyDenseVector(Array(0.1, 1.0))), - Row(0.0, new MyDenseVector(Array(0.2, 2.0))))) + Row(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), + Row(0.0, new UDT.MyDenseVector(Array(0.2, 2.0))))) } } // Tests to make sure that all operators correctly convert types on the way out. test("Local UDTs") { - val df = Seq((1, new MyDenseVector(Array(0.1, 1.0)))).toDF("int", "vec") - df.collect()(0).getAs[MyDenseVector](1) - df.take(1)(0).getAs[MyDenseVector](1) - df.limit(1).groupBy('int).agg(first('vec)).collect()(0).getAs[MyDenseVector](0) - df.orderBy('int).limit(1).groupBy('int).agg(first('vec)).collect()(0).getAs[MyDenseVector](0) - } - - test("HyperLogLogUDT") { - val hyperLogLogUDT = HyperLogLogUDT - val hyperLogLog = new HyperLogLog(0.4) - (1 to 10).foreach(i => hyperLogLog.offer(Row(i))) - - val actual = hyperLogLogUDT.deserialize(hyperLogLogUDT.serialize(hyperLogLog)) - assert(actual.cardinality() === hyperLogLog.cardinality()) - assert(java.util.Arrays.equals(actual.getBytes, hyperLogLog.getBytes)) - } - - test("OpenHashSetUDT") { - val openHashSetUDT = new OpenHashSetUDT(IntegerType) - val set = new OpenHashSet[Int] - (1 to 10).foreach(i => set.add(i)) - - val actual = openHashSetUDT.deserialize(openHashSetUDT.serialize(set)) - assert(actual.iterator.toSet === set.iterator.toSet) + val vec = new UDT.MyDenseVector(Array(0.1, 1.0)) + val df = Seq((1, vec)).toDF("int", "vec") + assert(vec === df.collect()(0).getAs[UDT.MyDenseVector](1)) + assert(vec === df.take(1)(0).getAs[UDT.MyDenseVector](1)) + checkAnswer(df.limit(1).groupBy('int).agg(first('vec)), Row(1, vec)) + checkAnswer(df.orderBy('int).limit(1).groupBy('int).agg(first('vec)), Row(1, vec)) } test("UDTs with JSON") { @@ -160,32 +218,90 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT ) val schema = StructType(Seq( StructField("id", IntegerType, false), - StructField("vec", new MyDenseVectorUDT, false) + StructField("vec", new UDT.MyDenseVectorUDT, false) )) - val stringRDD = sparkContext.parallelize(data) - val jsonRDD = sqlContext.read.schema(schema).json(stringRDD) + val jsonRDD = spark.read.schema(schema).json(data.toDS()) checkAnswer( jsonRDD, - Row(1, new MyDenseVector(Array(1.1, 2.2, 3.3, 4.4))) :: - Row(2, new MyDenseVector(Array(2.25, 4.5, 8.75))) :: + Row(1, new UDT.MyDenseVector(Array(1.1, 2.2, 3.3, 4.4))) :: + Row(2, new UDT.MyDenseVector(Array(2.25, 4.5, 8.75))) :: Nil ) } + test("UDTs with JSON and Dataset") { + val data = Seq( + "{\"id\":1,\"vec\":[1.1,2.2,3.3,4.4]}", + "{\"id\":2,\"vec\":[2.25,4.5,8.75]}" + ) + + val schema = StructType(Seq( + StructField("id", IntegerType, false), + StructField("vec", new UDT.MyDenseVectorUDT, false) + )) + + val jsonDataset = spark.read.schema(schema).json(data.toDS()) + .as[(Int, UDT.MyDenseVector)] + checkDataset( + jsonDataset, + (1, new UDT.MyDenseVector(Array(1.1, 2.2, 3.3, 4.4))), + (2, new UDT.MyDenseVector(Array(2.25, 4.5, 8.75))) + ) + } + test("SPARK-10472 UserDefinedType.typeName") { assert(IntegerType.typeName === "integer") - assert(new MyDenseVectorUDT().typeName === "mydensevector") - assert(new OpenHashSetUDT(IntegerType).typeName === "openhashset") + assert(new UDT.MyDenseVectorUDT().typeName === "mydensevector") } test("Catalyst type converter null handling for UDTs") { - val udt = new MyDenseVectorUDT() + val udt = new UDT.MyDenseVectorUDT() val toScalaConverter = CatalystTypeConverters.createToScalaConverter(udt) assert(toScalaConverter(null) === null) val toCatalystConverter = CatalystTypeConverters.createToCatalystConverter(udt) assert(toCatalystConverter(null) === null) + } + + test("SPARK-15658: Analysis exception if Dataset.map returns UDT object") { + // call `collect` to make sure this query can pass analysis. + pointsRDD.as[MyLabeledPoint].map(_.copy(label = 2.0)).collect() + } + test("SPARK-19311: UDFs disregard UDT type hierarchy") { + UDTRegistration.register(classOf[IExampleBaseType].getName, + classOf[ExampleBaseTypeUDT].getName) + UDTRegistration.register(classOf[IExampleSubType].getName, + classOf[ExampleSubTypeUDT].getName) + + // UDF that returns a base class object + sqlContext.udf.register("doUDF", (param: Int) => { + new ExampleBaseClass(param) + }: IExampleBaseType) + + // UDF that returns a derived class object + sqlContext.udf.register("doSubTypeUDF", (param: Int) => { + new ExampleSubClass(param) + }: IExampleSubType) + + // UDF that takes a base class object as parameter + sqlContext.udf.register("doOtherUDF", (obj: IExampleBaseType) => { + obj.field + }: Int) + + // this worked already before the fix SPARK-19311: + // return type of doUDF equals parameter type of doOtherUDF + sql("SELECT doOtherUDF(doUDF(41))") + + // this one passes only with the fix SPARK-19311: + // return type of doSubUDF is a subtype of the parameter type of doOtherUDF + sql("SELECT doOtherUDF(doSubTypeUDF(42))") + } + + test("except on UDT") { + checkAnswer( + pointsRDD.except(pointsRDD2), + Seq(Row(0.0, new UDT.MyDenseVector(Array(0.2, 2.0))))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala new file mode 100644 index 000000000000..1d33e7970be8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/XPathFunctionsSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.test.SharedSQLContext + +/** + * End-to-end tests for xpath expressions. + */ +class XPathFunctionsSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("xpath_boolean") { + val df = Seq("b").toDF("xml") + checkAnswer(df.selectExpr("xpath_boolean(xml, 'a/b')"), Row(true)) + } + + test("xpath_short, xpath_int, xpath_long") { + val df = Seq("12").toDF("xml") + checkAnswer( + df.selectExpr( + "xpath_short(xml, 'sum(a/b)')", + "xpath_int(xml, 'sum(a/b)')", + "xpath_long(xml, 'sum(a/b)')"), + Row(3.toShort, 3, 3L)) + } + + test("xpath_float, xpath_double, xpath_number") { + val df = Seq("1.02.1").toDF("xml") + checkAnswer( + df.selectExpr( + "xpath_float(xml, 'sum(a/b)')", + "xpath_double(xml, 'sum(a/b)')", + "xpath_number(xml, 'sum(a/b)')"), + Row(3.1.toFloat, 3.1, 3.1)) + } + + test("xpath_string") { + val df = Seq("bcc").toDF("xml") + checkAnswer(df.selectExpr("xpath_string(xml, 'a/c')"), Row("cc")) + } + + test("xpath") { + val df = Seq("b1b2b3c1c2").toDF("xml") + checkAnswer(df.selectExpr("xpath(xml, 'a/*/text()')"), Row(Seq("b1", "b2", "b3", "c1", "c2"))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala index f54e23e3aa6c..7cfee4957557 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/r/SQLUtilsSuite.scala @@ -1,19 +1,19 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.sql.api.r diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala deleted file mode 100644 index 89a664001bdd..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ /dev/null @@ -1,110 +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.columnar - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow -import org.apache.spark.sql.types._ - -class ColumnStatsSuite extends SparkFunSuite { - testColumnStats(classOf[BooleanColumnStats], BOOLEAN, createRow(true, false, 0)) - testColumnStats(classOf[ByteColumnStats], BYTE, createRow(Byte.MaxValue, Byte.MinValue, 0)) - testColumnStats(classOf[ShortColumnStats], SHORT, createRow(Short.MaxValue, Short.MinValue, 0)) - testColumnStats(classOf[IntColumnStats], INT, createRow(Int.MaxValue, Int.MinValue, 0)) - testColumnStats(classOf[LongColumnStats], LONG, createRow(Long.MaxValue, Long.MinValue, 0)) - testColumnStats(classOf[FloatColumnStats], FLOAT, createRow(Float.MaxValue, Float.MinValue, 0)) - testColumnStats(classOf[DoubleColumnStats], DOUBLE, - createRow(Double.MaxValue, Double.MinValue, 0)) - testColumnStats(classOf[StringColumnStats], STRING, createRow(null, null, 0)) - testDecimalColumnStats(createRow(null, null, 0)) - - def createRow(values: Any*): GenericInternalRow = new GenericInternalRow(values.toArray) - - def testColumnStats[T <: AtomicType, U <: ColumnStats]( - columnStatsClass: Class[U], - columnType: NativeColumnType[T], - initialStatistics: GenericInternalRow): Unit = { - - val columnStatsName = columnStatsClass.getSimpleName - - test(s"$columnStatsName: empty") { - val columnStats = columnStatsClass.newInstance() - columnStats.collectedStatistics.values.zip(initialStatistics.values).foreach { - case (actual, expected) => assert(actual === expected) - } - } - - test(s"$columnStatsName: non-empty") { - import org.apache.spark.sql.columnar.ColumnarTestUtils._ - - val columnStats = columnStatsClass.newInstance() - val rows = Seq.fill(10)(makeRandomRow(columnType)) ++ Seq.fill(10)(makeNullRow(1)) - rows.foreach(columnStats.gatherStats(_, 0)) - - val values = rows.take(10).map(_.get(0, columnType.dataType).asInstanceOf[T#InternalType]) - val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#InternalType]] - val stats = columnStats.collectedStatistics - - assertResult(values.min(ordering), "Wrong lower bound")(stats.values(0)) - assertResult(values.max(ordering), "Wrong upper bound")(stats.values(1)) - assertResult(10, "Wrong null count")(stats.values(2)) - assertResult(20, "Wrong row count")(stats.values(3)) - assertResult(stats.values(4), "Wrong size in bytes") { - rows.map { row => - if (row.isNullAt(0)) 4 else columnType.actualSize(row, 0) - }.sum - } - } - } - - def testDecimalColumnStats[T <: AtomicType, U <: ColumnStats]( - initialStatistics: GenericInternalRow): Unit = { - - val columnStatsName = classOf[DecimalColumnStats].getSimpleName - val columnType = COMPACT_DECIMAL(15, 10) - - test(s"$columnStatsName: empty") { - val columnStats = new DecimalColumnStats(15, 10) - columnStats.collectedStatistics.values.zip(initialStatistics.values).foreach { - case (actual, expected) => assert(actual === expected) - } - } - - test(s"$columnStatsName: non-empty") { - import org.apache.spark.sql.columnar.ColumnarTestUtils._ - - val columnStats = new DecimalColumnStats(15, 10) - val rows = Seq.fill(10)(makeRandomRow(columnType)) ++ Seq.fill(10)(makeNullRow(1)) - rows.foreach(columnStats.gatherStats(_, 0)) - - val values = rows.take(10).map(_.get(0, columnType.dataType).asInstanceOf[T#InternalType]) - val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#InternalType]] - val stats = columnStats.collectedStatistics - - assertResult(values.min(ordering), "Wrong lower bound")(stats.values(0)) - assertResult(values.max(ordering), "Wrong upper bound")(stats.values(1)) - assertResult(10, "Wrong null count")(stats.values(2)) - assertResult(20, "Wrong row count")(stats.values(3)) - assertResult(stats.values(4), "Wrong size in bytes") { - rows.map { row => - if (row.isNullAt(0)) 4 else columnType.actualSize(row, 0) - }.sum - } - } - } -} 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 deleted file mode 100644 index 6265e40a0a07..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ /dev/null @@ -1,222 +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.columnar - -import java.sql.{Date, Timestamp} - -import org.apache.spark.sql.{QueryTest, Row} -import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.test.SQLTestData._ -import org.apache.spark.sql.types._ -import org.apache.spark.storage.StorageLevel.MEMORY_ONLY - -class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { - import testImplicits._ - - setupTestData() - - test("simple columnar query") { - val plan = sqlContext.executePlan(testData.logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None) - - checkAnswer(scan, testData.collect().toSeq) - } - - 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)) - .toDF().registerTempTable("sizeTst") - sqlContext.cacheTable("sizeTst") - assert( - sqlContext.table("sizeTst").queryExecution.analyzed.statistics.sizeInBytes > - sqlContext.conf.autoBroadcastJoinThreshold) - } - - test("projection") { - val plan = sqlContext.executePlan(testData.select('value, 'key).logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None) - - checkAnswer(scan, testData.collect().map { - case Row(key: Int, value: String) => value -> key - }.map(Row.fromTuple)) - } - - test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { - val plan = sqlContext.executePlan(testData.logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None) - - checkAnswer(scan, testData.collect().toSeq) - checkAnswer(scan, testData.collect().toSeq) - } - - test("SPARK-1678 regression: compression must not lose repeated values") { - checkAnswer( - sql("SELECT * FROM repeatedData"), - repeatedData.collect().toSeq.map(Row.fromTuple)) - - sqlContext.cacheTable("repeatedData") - - checkAnswer( - sql("SELECT * FROM repeatedData"), - repeatedData.collect().toSeq.map(Row.fromTuple)) - } - - test("with null values") { - checkAnswer( - sql("SELECT * FROM nullableRepeatedData"), - nullableRepeatedData.collect().toSeq.map(Row.fromTuple)) - - sqlContext.cacheTable("nullableRepeatedData") - - checkAnswer( - sql("SELECT * FROM nullableRepeatedData"), - nullableRepeatedData.collect().toSeq.map(Row.fromTuple)) - } - - test("SPARK-2729 regression: timestamp data type") { - val timestamps = (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time") - timestamps.registerTempTable("timestamps") - - checkAnswer( - sql("SELECT time FROM timestamps"), - timestamps.collect().toSeq) - - sqlContext.cacheTable("timestamps") - - checkAnswer( - sql("SELECT time FROM timestamps"), - timestamps.collect().toSeq) - } - - test("SPARK-3320 regression: batched column buffer building should work with empty partitions") { - checkAnswer( - sql("SELECT * FROM withEmptyParts"), - withEmptyParts.collect().toSeq.map(Row.fromTuple)) - - sqlContext.cacheTable("withEmptyParts") - - checkAnswer( - sql("SELECT * FROM withEmptyParts"), - withEmptyParts.collect().toSeq.map(Row.fromTuple)) - } - - test("SPARK-4182 Caching complex types") { - complexData.cache().count() - // Shouldn't throw - complexData.count() - complexData.unpersist() - } - - test("decimal type") { - // Casting is required here because ScalaReflection can't capture decimal precision information. - val df = (1 to 10) - .map(i => Tuple1(Decimal(i, 15, 10))) - .toDF("dec") - .select($"dec" cast DecimalType(15, 10)) - - assert(df.schema.head.dataType === DecimalType(15, 10)) - - df.cache().registerTempTable("test_fixed_decimal") - checkAnswer( - sql("SELECT * FROM test_fixed_decimal"), - (1 to 10).map(i => Row(Decimal(i, 15, 10).toJavaBigDecimal))) - } - - test("test different data types") { - // Create the schema. - val struct = - StructType( - StructField("f1", FloatType, true) :: - StructField("f2", ArrayType(BooleanType), true) :: Nil) - val dataTypes = - Seq(StringType, BinaryType, NullType, BooleanType, - ByteType, ShortType, IntegerType, LongType, - FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), - DateType, TimestampType, - ArrayType(IntegerType), MapType(StringType, LongType), struct) - val fields = dataTypes.zipWithIndex.map { case (dataType, index) => - StructField(s"col$index", dataType, true) - } - val allColumns = fields.map(_.name).mkString(",") - val schema = StructType(fields) - - // Create a RDD for the schema - val rdd = - sparkContext.parallelize((1 to 10000), 10).map { i => - Row( - s"str${i}: test cache.", - s"binary${i}: test cache.".getBytes("UTF-8"), - null, - i % 2 == 0, - i.toByte, - i.toShort, - i, - Long.MaxValue - i.toLong, - (i + 0.25).toFloat, - (i + 0.75), - BigDecimal(Long.MaxValue.toString + ".12345"), - new java.math.BigDecimal(s"${i % 9 + 1}" + ".23456"), - new Date(i), - new Timestamp(i * 1000000L), - (i to i + 10).toSeq, - (i to i + 10).map(j => s"map_key_$j" -> (Long.MaxValue - j)).toMap, - Row((i - 0.25).toFloat, Seq(true, false, null))) - } - sqlContext.createDataFrame(rdd, schema).registerTempTable("InMemoryCache_different_data_types") - // Cache the table. - sql("cache table InMemoryCache_different_data_types") - // Make sure the table is indeed cached. - sqlContext.table("InMemoryCache_different_data_types").queryExecution.executedPlan - assert( - sqlContext.isCached("InMemoryCache_different_data_types"), - "InMemoryCache_different_data_types should be cached.") - // Issue a query and check the results. - checkAnswer( - sql(s"SELECT DISTINCT ${allColumns} FROM InMemoryCache_different_data_types"), - sqlContext.table("InMemoryCache_different_data_types").collect()) - sqlContext.dropTempTable("InMemoryCache_different_data_types") - } - - test("SPARK-10422: String column in InMemoryColumnarCache needs to override clone method") { - val df = sqlContext.range(1, 100).selectExpr("id % 10 as id") - .rdd.map(id => Tuple1(s"str_$id")).toDF("i") - val cached = df.cache() - // count triggers the caching action. It should not throw. - cached.count() - - // Make sure, the DataFrame is indeed cached. - assert(sqlContext.cacheManager.lookupCachedData(cached).nonEmpty) - - // Check result. - checkAnswer( - cached, - sqlContext.range(1, 100).selectExpr("id % 10 as id") - .rdd.map(id => Tuple1(s"str_$id")).toDF("i") - ) - - // Drop the cache. - cached.unpersist() - } - - test("SPARK-10859: Predicates pushed to InMemoryColumnarTableScan are not evaluated correctly") { - val data = sqlContext.range(10).selectExpr("id", "cast(id as string) as s") - data.cache() - assert(data.count() === 10) - assert(data.filter($"s" === "3").count() === 1) - } -} 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 deleted file mode 100644 index 6b7401464f46..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ /dev/null @@ -1,127 +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.columnar - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql._ -import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.test.SQLTestData._ - -class PartitionBatchPruningSuite extends SparkFunSuite with SharedSQLContext { - import testImplicits._ - - private lazy val originalColumnBatchSize = sqlContext.conf.columnBatchSize - private lazy val originalInMemoryPartitionPruning = sqlContext.conf.inMemoryPartitionPruning - - override protected def beforeAll(): Unit = { - super.beforeAll() - // Make a table with 5 partitions, 2 batches per partition, 10 elements per batch - sqlContext.setConf(SQLConf.COLUMN_BATCH_SIZE, 10) - - 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).toDF() - pruningData.registerTempTable("pruningData") - - // Enable in-memory partition pruning - sqlContext.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, true) - // Enable in-memory table scan accumulators - sqlContext.setConf("spark.sql.inMemoryTableScanStatistics.enable", "true") - sqlContext.cacheTable("pruningData") - } - - override protected def afterAll(): Unit = { - try { - sqlContext.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize) - sqlContext.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning) - sqlContext.uncacheTable("pruningData") - } finally { - super.afterAll() - } - } - - // Comparisons - checkBatchPruning("SELECT key FROM pruningData WHERE key = 1", 1, 1)(Seq(1)) - checkBatchPruning("SELECT key FROM pruningData WHERE 1 = key", 1, 1)(Seq(1)) - checkBatchPruning("SELECT key FROM pruningData WHERE key < 12", 1, 2)(1 to 11) - checkBatchPruning("SELECT key FROM pruningData WHERE key <= 11", 1, 2)(1 to 11) - checkBatchPruning("SELECT key FROM pruningData WHERE key > 88", 1, 2)(89 to 100) - checkBatchPruning("SELECT key FROM pruningData WHERE key >= 89", 1, 2)(89 to 100) - checkBatchPruning("SELECT key FROM pruningData WHERE 12 > key", 1, 2)(1 to 11) - checkBatchPruning("SELECT key FROM pruningData WHERE 11 >= key", 1, 2)(1 to 11) - checkBatchPruning("SELECT key FROM pruningData WHERE 88 < key", 1, 2)(89 to 100) - checkBatchPruning("SELECT key FROM pruningData WHERE 89 <= key", 1, 2)(89 to 100) - - // IS NULL - checkBatchPruning("SELECT key FROM pruningData WHERE value IS NULL", 5, 5) { - (1 to 10) ++ (21 to 30) ++ (41 to 50) ++ (61 to 70) ++ (81 to 90) - } - - // IS NOT NULL - checkBatchPruning("SELECT key FROM pruningData WHERE value IS NOT NULL", 5, 5) { - (11 to 20) ++ (31 to 40) ++ (51 to 60) ++ (71 to 80) ++ (91 to 100) - } - - // Conjunction and disjunction - checkBatchPruning("SELECT key FROM pruningData WHERE key > 8 AND key <= 21", 2, 3)(9 to 21) - checkBatchPruning("SELECT key FROM pruningData WHERE key < 2 OR key > 99", 2, 2)(Seq(1, 100)) - checkBatchPruning("SELECT key FROM pruningData WHERE key < 12 AND key IS NOT NULL", 1, 2)(1 to 11) - checkBatchPruning("SELECT key FROM pruningData WHERE key < 2 OR (key > 78 AND key < 92)", 3, 4) { - Seq(1) ++ (79 to 91) - } - checkBatchPruning("SELECT key FROM pruningData WHERE NOT (key < 88)", 1, 2) { - // Although the `NOT` operator isn't supported directly, the optimizer can transform - // `NOT (a < b)` to `b >= a` - 88 to 100 - } - - // With unsupported predicate - { - val seq = (1 to 30).mkString(", ") - checkBatchPruning(s"SELECT key FROM pruningData WHERE NOT (key IN ($seq))", 5, 10)(31 to 100) - checkBatchPruning(s"SELECT key FROM pruningData WHERE NOT (key IN ($seq)) AND key > 88", 1, 2) { - 89 to 100 - } - } - - def checkBatchPruning( - query: String, - expectedReadPartitions: Int, - expectedReadBatches: Int)( - expectedQueryResult: => Seq[Int]): Unit = { - - test(query) { - val df = sql(query) - val queryExecution = df.queryExecution - - assertResult(expectedQueryResult.toArray, s"Wrong query result: $queryExecution") { - df.collect().map(_(0)).toArray - } - - val (readPartitions, readBatches) = df.queryExecution.executedPlan.collect { - case in: InMemoryColumnarTableScan => (in.readPartitions.value, in.readBatches.value) - }.head - - assert(readBatches === expectedReadBatches, s"Wrong number of read batches: $queryExecution") - assert( - readPartitions === expectedReadPartitions, - s"Wrong number of read partitions: $queryExecution") - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala new file mode 100644 index 000000000000..423e1288e8dc --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.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.execution + +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkConf +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.SharedSQLContext + +/** + * Suite that tests the redaction of DataSourceScanExec + */ +class DataSourceScanExecRedactionSuite extends QueryTest with SharedSQLContext { + + override protected def sparkConf: SparkConf = super.sparkConf + .set("spark.redaction.string.regex", "file:/[\\w_]+") + + test("treeString is redacted") { + withTempDir { dir => + val basePath = dir.getCanonicalPath + spark.range(0, 10).toDF("a").write.parquet(new Path(basePath, "foo=1").toString) + val df = spark.read.parquet(basePath) + + val rootPath = df.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get + .asInstanceOf[FileSourceScanExec].relation.location.rootPaths.head + assert(rootPath.toString.contains(dir.toURI.getPath.stripSuffix("/"))) + + assert(!df.queryExecution.sparkPlan.treeString(verbose = true).contains(rootPath.getName)) + assert(!df.queryExecution.executedPlan.treeString(verbose = true).contains(rootPath.getName)) + assert(!df.queryExecution.toString.contains(rootPath.getName)) + assert(!df.queryExecution.simpleString.contains(rootPath.getName)) + + val replacement = "*********" + assert(df.queryExecution.sparkPlan.treeString(verbose = true).contains(replacement)) + assert(df.queryExecution.executedPlan.treeString(verbose = true).contains(replacement)) + assert(df.queryExecution.toString.contains(replacement)) + assert(df.queryExecution.simpleString.contains(replacement)) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 25f2f5caeed1..f1b5e3be5b63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -19,28 +19,29 @@ package org.apache.spark.sql.execution import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.{MapOutputStatistics, SparkConf, SparkFunSuite} import org.apache.spark.sql._ -import org.apache.spark.{SparkFunSuite, SparkContext, SparkConf, MapOutputStatistics} +import org.apache.spark.sql.execution.exchange.{ExchangeCoordinator, ShuffleExchange} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { - private var originalActiveSQLContext: Option[SQLContext] = _ - private var originalInstantiatedSQLContext: Option[SQLContext] = _ + private var originalActiveSparkSession: Option[SparkSession] = _ + private var originalInstantiatedSparkSession: Option[SparkSession] = _ override protected def beforeAll(): Unit = { - originalActiveSQLContext = SQLContext.getActiveContextOption() - originalInstantiatedSQLContext = SQLContext.getInstantiatedContextOption() + originalActiveSparkSession = SparkSession.getActiveSession + originalInstantiatedSparkSession = SparkSession.getDefaultSession - SQLContext.clearActive() - originalInstantiatedSQLContext.foreach(ctx => SQLContext.clearInstantiatedContext(ctx)) + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() } override protected def afterAll(): Unit = { // Set these states back. - originalActiveSQLContext.foreach(ctx => SQLContext.setActive(ctx)) - originalInstantiatedSQLContext.foreach(ctx => SQLContext.setInstantiatedContext(ctx)) + originalActiveSparkSession.foreach(ctx => SparkSession.setActiveSession(ctx)) + originalInstantiatedSparkSession.foreach(ctx => SparkSession.setDefaultSession(ctx)) } private def checkEstimation( @@ -84,7 +85,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { { // There are a few large pre-shuffle partitions. val bytesByPartitionId = Array[Long](110, 10, 100, 110, 0) - val expectedPartitionStartIndices = Array[Int](0, 1, 3, 4) + val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) checkEstimation(coordinator, Array(bytesByPartitionId), expectedPartitionStartIndices) } @@ -145,7 +146,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // 2 post-shuffle partition are needed. val bytesByPartitionId1 = Array[Long](0, 10, 0, 20, 0) val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) - val expectedPartitionStartIndices = Array[Int](0, 3) + val expectedPartitionStartIndices = Array[Int](0, 2, 4) checkEstimation( coordinator, Array(bytesByPartitionId1, bytesByPartitionId2), @@ -153,10 +154,10 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { } { - // 2 post-shuffle partition are needed. + // 4 post-shuffle partition are needed. val bytesByPartitionId1 = Array[Long](0, 99, 0, 20, 0) val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) - val expectedPartitionStartIndices = Array[Int](0, 2) + val expectedPartitionStartIndices = Array[Int](0, 1, 2, 4) checkEstimation( coordinator, Array(bytesByPartitionId1, bytesByPartitionId2), @@ -167,7 +168,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // 2 post-shuffle partition are needed. val bytesByPartitionId1 = Array[Long](0, 100, 0, 30, 0) val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) - val expectedPartitionStartIndices = Array[Int](0, 2, 4) + val expectedPartitionStartIndices = Array[Int](0, 1, 2, 4) checkEstimation( coordinator, Array(bytesByPartitionId1, bytesByPartitionId2), @@ -178,7 +179,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // There are a few large pre-shuffle partitions. val bytesByPartitionId1 = Array[Long](0, 100, 40, 30, 0) val bytesByPartitionId2 = Array[Long](30, 0, 60, 0, 110) - val expectedPartitionStartIndices = Array[Int](0, 2, 3) + val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) checkEstimation( coordinator, Array(bytesByPartitionId1, bytesByPartitionId2), @@ -227,7 +228,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // The number of post-shuffle partitions is determined by the coordinator. val bytesByPartitionId1 = Array[Long](10, 50, 20, 80, 20) val bytesByPartitionId2 = Array[Long](40, 10, 0, 10, 30) - val expectedPartitionStartIndices = Array[Int](0, 2, 4) + val expectedPartitionStartIndices = Array[Int](0, 1, 3, 4) checkEstimation( coordinator, Array(bytesByPartitionId1, bytesByPartitionId2), @@ -248,8 +249,8 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { } } - def withSQLContext( - f: SQLContext => Unit, + def withSparkSession( + f: SparkSession => Unit, targetNumPostShufflePartitions: Int, minNumPostShufflePartitions: Option[Int]): Unit = { val sparkConf = @@ -260,6 +261,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { .set("spark.driver.allowMultipleContexts", "true") .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") + .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") .set( SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, targetNumPostShufflePartitions.toString) @@ -269,66 +271,68 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { case None => sparkConf.set(SQLConf.SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS.key, "-1") } - val sparkContext = new SparkContext(sparkConf) - val sqlContext = new TestSQLContext(sparkContext) - try f(sqlContext) finally sparkContext.stop() + + val spark = SparkSession.builder() + .config(sparkConf) + .getOrCreate() + try f(spark) finally spark.stop() } - Seq(Some(3), None).foreach { minNumPostShufflePartitions => + Seq(Some(5), None).foreach { minNumPostShufflePartitions => val testNameNote = minNumPostShufflePartitions match { - case Some(numPartitions) => "(minNumPostShufflePartitions: 3)" + case Some(numPartitions) => "(minNumPostShufflePartitions: " + numPartitions + ")" case None => "" } test(s"determining the number of reducers: aggregate operator$testNameNote") { - val test = { sqlContext: SQLContext => + val test = { spark: SparkSession => val df = - sqlContext + spark .range(0, 1000, 1, numInputPartitions) .selectExpr("id % 20 as key", "id as value") - val agg = df.groupBy("key").count + val agg = df.groupBy("key").count() // Check the answer first. checkAnswer( agg, - sqlContext.range(0, 20).selectExpr("id", "50 as cnt").collect()) + spark.range(0, 20).selectExpr("id", "50 as cnt").collect()) // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val exchanges = agg.queryExecution.executedPlan.collect { - case e: Exchange => e + case e: ShuffleExchange => e } assert(exchanges.length === 1) minNumPostShufflePartitions match { case Some(numPartitions) => exchanges.foreach { - case e: Exchange => + case e: ShuffleExchange => assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 3) + assert(e.outputPartitioning.numPartitions === 5) case o => } case None => exchanges.foreach { - case e: Exchange => + case e: ShuffleExchange => assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 2) + assert(e.outputPartitioning.numPartitions === 3) case o => } } } - withSQLContext(test, 1536, minNumPostShufflePartitions) + withSparkSession(test, 2000, minNumPostShufflePartitions) } test(s"determining the number of reducers: join operator$testNameNote") { - val test = { sqlContext: SQLContext => + val test = { spark: SparkSession => val df1 = - sqlContext + spark .range(0, 1000, 1, numInputPartitions) .selectExpr("id % 500 as key1", "id as value1") val df2 = - sqlContext + spark .range(0, 1000, 1, numInputPartitions) .selectExpr("id % 500 as key2", "id as value2") @@ -336,10 +340,10 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Check the answer first. val expectedAnswer = - sqlContext + spark .range(0, 1000) .selectExpr("id % 500 as key", "id as value") - .unionAll(sqlContext.range(0, 1000).selectExpr("id % 500 as key", "id as value")) + .union(spark.range(0, 1000).selectExpr("id % 500 as key", "id as value")) checkAnswer( join, expectedAnswer.collect()) @@ -347,21 +351,21 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val exchanges = join.queryExecution.executedPlan.collect { - case e: Exchange => e + case e: ShuffleExchange => e } assert(exchanges.length === 2) minNumPostShufflePartitions match { case Some(numPartitions) => exchanges.foreach { - case e: Exchange => + case e: ShuffleExchange => assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 3) + assert(e.outputPartitioning.numPartitions === 5) case o => } case None => exchanges.foreach { - case e: Exchange => + case e: ShuffleExchange => assert(e.coordinator.isDefined) assert(e.outputPartitioning.numPartitions === 2) case o => @@ -369,31 +373,31 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { } } - withSQLContext(test, 16384, minNumPostShufflePartitions) + withSparkSession(test, 16384, minNumPostShufflePartitions) } test(s"determining the number of reducers: complex query 1$testNameNote") { - val test = { sqlContext: SQLContext => + val test: (SparkSession) => Unit = { spark: SparkSession => val df1 = - sqlContext + spark .range(0, 1000, 1, numInputPartitions) .selectExpr("id % 500 as key1", "id as value1") .groupBy("key1") - .count + .count() .toDF("key1", "cnt1") val df2 = - sqlContext + spark .range(0, 1000, 1, numInputPartitions) .selectExpr("id % 500 as key2", "id as value2") .groupBy("key2") - .count + .count() .toDF("key2", "cnt2") val join = df1.join(df2, col("key1") === col("key2")).select(col("key1"), col("cnt2")) // Check the answer first. val expectedAnswer = - sqlContext + spark .range(0, 500) .selectExpr("id", "2 as cnt") checkAnswer( @@ -403,38 +407,38 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val exchanges = join.queryExecution.executedPlan.collect { - case e: Exchange => e + case e: ShuffleExchange => e } assert(exchanges.length === 4) minNumPostShufflePartitions match { case Some(numPartitions) => exchanges.foreach { - case e: Exchange => + case e: ShuffleExchange => assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 3) + assert(e.outputPartitioning.numPartitions === 5) case o => } case None => assert(exchanges.forall(_.coordinator.isDefined)) - assert(exchanges.map(_.outputPartitioning.numPartitions).toSeq.toSet === Set(1, 2)) + assert(exchanges.map(_.outputPartitioning.numPartitions).toSet === Set(2, 3)) } } - withSQLContext(test, 6144, minNumPostShufflePartitions) + withSparkSession(test, 6644, minNumPostShufflePartitions) } test(s"determining the number of reducers: complex query 2$testNameNote") { - val test = { sqlContext: SQLContext => + val test: (SparkSession) => Unit = { spark: SparkSession => val df1 = - sqlContext + spark .range(0, 1000, 1, numInputPartitions) .selectExpr("id % 500 as key1", "id as value1") .groupBy("key1") - .count + .count() .toDF("key1", "cnt1") val df2 = - sqlContext + spark .range(0, 1000, 1, numInputPartitions) .selectExpr("id % 500 as key2", "id as value2") @@ -445,7 +449,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Check the answer first. val expectedAnswer = - sqlContext + spark .range(0, 1000) .selectExpr("id % 500 as key", "2 as cnt", "id as value") checkAnswer( @@ -455,25 +459,25 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. val exchanges = join.queryExecution.executedPlan.collect { - case e: Exchange => e + case e: ShuffleExchange => e } assert(exchanges.length === 3) minNumPostShufflePartitions match { case Some(numPartitions) => exchanges.foreach { - case e: Exchange => + case e: ShuffleExchange => assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 3) + assert(e.outputPartitioning.numPartitions === 5) case o => } case None => assert(exchanges.forall(_.coordinator.isDefined)) - assert(exchanges.map(_.outputPartitioning.numPartitions).toSeq.toSet === Set(2, 3)) + assert(exchanges.map(_.outputPartitioning.numPartitions).toSet === Set(5, 3)) } } - withSQLContext(test, 6144, minNumPostShufflePartitions) + withSparkSession(test, 6144, minNumPostShufflePartitions) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala index 911d12e93e50..59eaf4d1c29b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeSuite.scala @@ -18,18 +18,87 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.plans.physical.SinglePartition +import org.apache.spark.sql.catalyst.expressions.{Alias, Literal} +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, IdentityBroadcastMode, SinglePartition} +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchange} +import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode import org.apache.spark.sql.test.SharedSQLContext class ExchangeSuite extends SparkPlanTest with SharedSQLContext { - import testImplicits.localSeqToDataFrameHolder + import testImplicits._ test("shuffling UnsafeRows in exchange") { val input = (1 to 1000).map(Tuple1.apply) checkAnswer( input.toDF(), - plan => ConvertToSafe(Exchange(SinglePartition, ConvertToUnsafe(plan))), + plan => ShuffleExchange(SinglePartition, plan), input.map(Row.fromTuple) ) } + + test("BroadcastMode.canonicalized") { + val mode1 = IdentityBroadcastMode + val mode2 = HashedRelationBroadcastMode(Literal(1L) :: Nil) + val mode3 = HashedRelationBroadcastMode(Literal("s") :: Nil) + + assert(mode1.canonicalized == mode1.canonicalized) + assert(mode1.canonicalized != mode2.canonicalized) + assert(mode2.canonicalized != mode1.canonicalized) + assert(mode2.canonicalized == mode2.canonicalized) + assert(mode2.canonicalized != mode3.canonicalized) + assert(mode3.canonicalized == mode3.canonicalized) + } + + test("BroadcastExchange same result") { + val df = spark.range(10) + val plan = df.queryExecution.executedPlan + val output = plan.output + assert(plan sameResult plan) + + val exchange1 = BroadcastExchangeExec(IdentityBroadcastMode, plan) + val hashMode = HashedRelationBroadcastMode(output) + val exchange2 = BroadcastExchangeExec(hashMode, plan) + val hashMode2 = + HashedRelationBroadcastMode(Alias(output.head, "id2")() :: Nil) + val exchange3 = BroadcastExchangeExec(hashMode2, plan) + val exchange4 = ReusedExchangeExec(output, exchange3) + + assert(exchange1 sameResult exchange1) + assert(exchange2 sameResult exchange2) + assert(exchange3 sameResult exchange3) + assert(exchange4 sameResult exchange4) + + assert(!exchange1.sameResult(exchange2)) + assert(!exchange2.sameResult(exchange3)) + assert(exchange3.sameResult(exchange4)) + assert(exchange4 sameResult exchange3) + } + + test("ShuffleExchange same result") { + val df = spark.range(10) + val plan = df.queryExecution.executedPlan + val output = plan.output + assert(plan sameResult plan) + + val part1 = HashPartitioning(output, 1) + val exchange1 = ShuffleExchange(part1, plan) + val exchange2 = ShuffleExchange(part1, plan) + val part2 = HashPartitioning(output, 2) + val exchange3 = ShuffleExchange(part2, plan) + val part3 = HashPartitioning(output ++ output, 2) + val exchange4 = ShuffleExchange(part3, plan) + val exchange5 = ReusedExchangeExec(output, exchange4) + + assert(exchange1 sameResult exchange1) + assert(exchange2 sameResult exchange2) + assert(exchange3 sameResult exchange3) + assert(exchange4 sameResult exchange4) + assert(exchange5 sameResult exchange5) + + assert(exchange1 sameResult exchange2) + assert(!exchange2.sameResult(exchange3)) + assert(!exchange3.sameResult(exchange4)) + assert(exchange4.sameResult(exchange5)) + assert(exchange5 sameResult exchange4) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala new file mode 100644 index 000000000000..efe28afab08e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala @@ -0,0 +1,236 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{SparkConf, SparkContext, SparkEnv, TaskContext} +import org.apache.spark.memory.MemoryTestingUtils +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.util.Benchmark +import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter + +object ExternalAppendOnlyUnsafeRowArrayBenchmark { + + def testAgainstRawArrayBuffer(numSpillThreshold: Int, numRows: Int, iterations: Int): Unit = { + val random = new java.util.Random() + val rows = (1 to numRows).map(_ => { + val row = new UnsafeRow(1) + row.pointTo(new Array[Byte](64), 16) + row.setLong(0, random.nextLong()) + row + }) + + val benchmark = new Benchmark(s"Array with $numRows rows", iterations * numRows) + + // Internally, `ExternalAppendOnlyUnsafeRowArray` will create an + // in-memory buffer of size `numSpillThreshold`. This will mimic that + val initialSize = + Math.min( + ExternalAppendOnlyUnsafeRowArray.DefaultInitialSizeOfInMemoryBuffer, + numSpillThreshold) + + benchmark.addCase("ArrayBuffer") { _: Int => + var sum = 0L + for (_ <- 0L until iterations) { + val array = new ArrayBuffer[UnsafeRow](initialSize) + + // Internally, `ExternalAppendOnlyUnsafeRowArray` will create a + // copy of the row. This will mimic that + rows.foreach(x => array += x.copy()) + + var i = 0 + val n = array.length + while (i < n) { + sum = sum + array(i).getLong(0) + i += 1 + } + array.clear() + } + } + + benchmark.addCase("ExternalAppendOnlyUnsafeRowArray") { _: Int => + var sum = 0L + for (_ <- 0L until iterations) { + val array = new ExternalAppendOnlyUnsafeRowArray( + ExternalAppendOnlyUnsafeRowArray.DefaultInitialSizeOfInMemoryBuffer, + numSpillThreshold) + + rows.foreach(x => array.add(x)) + + val iterator = array.generateIterator() + while (iterator.hasNext) { + sum = sum + iterator.next().getLong(0) + } + array.clear() + } + } + + val conf = new SparkConf(false) + // Make the Java serializer write a reset instruction (TC_RESET) after each object to test + // for a bug we had with bytes written past the last object in a batch (SPARK-2792) + conf.set("spark.serializer.objectStreamReset", "1") + conf.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") + + val sc = new SparkContext("local", "test", conf) + val taskContext = MemoryTestingUtils.fakeTaskContext(SparkEnv.get) + TaskContext.setTaskContext(taskContext) + benchmark.run() + sc.stop() + } + + def testAgainstRawUnsafeExternalSorter( + numSpillThreshold: Int, + numRows: Int, + iterations: Int): Unit = { + + val random = new java.util.Random() + val rows = (1 to numRows).map(_ => { + val row = new UnsafeRow(1) + row.pointTo(new Array[Byte](64), 16) + row.setLong(0, random.nextLong()) + row + }) + + val benchmark = new Benchmark(s"Spilling with $numRows rows", iterations * numRows) + + benchmark.addCase("UnsafeExternalSorter") { _: Int => + var sum = 0L + for (_ <- 0L until iterations) { + val array = UnsafeExternalSorter.create( + TaskContext.get().taskMemoryManager(), + SparkEnv.get.blockManager, + SparkEnv.get.serializerManager, + TaskContext.get(), + null, + null, + 1024, + SparkEnv.get.memoryManager.pageSizeBytes, + numSpillThreshold, + false) + + rows.foreach(x => + array.insertRecord( + x.getBaseObject, + x.getBaseOffset, + x.getSizeInBytes, + 0, + false)) + + val unsafeRow = new UnsafeRow(1) + val iter = array.getIterator(0) + while (iter.hasNext) { + iter.loadNext() + unsafeRow.pointTo(iter.getBaseObject, iter.getBaseOffset, iter.getRecordLength) + sum = sum + unsafeRow.getLong(0) + } + array.cleanupResources() + } + } + + benchmark.addCase("ExternalAppendOnlyUnsafeRowArray") { _: Int => + var sum = 0L + for (_ <- 0L until iterations) { + val array = new ExternalAppendOnlyUnsafeRowArray(numSpillThreshold, numSpillThreshold) + rows.foreach(x => array.add(x)) + + val iterator = array.generateIterator() + while (iterator.hasNext) { + sum = sum + iterator.next().getLong(0) + } + array.clear() + } + } + + val conf = new SparkConf(false) + // Make the Java serializer write a reset instruction (TC_RESET) after each object to test + // for a bug we had with bytes written past the last object in a batch (SPARK-2792) + conf.set("spark.serializer.objectStreamReset", "1") + conf.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") + + val sc = new SparkContext("local", "test", conf) + val taskContext = MemoryTestingUtils.fakeTaskContext(SparkEnv.get) + TaskContext.setTaskContext(taskContext) + benchmark.run() + sc.stop() + } + + def main(args: Array[String]): Unit = { + + // ========================================================================================= // + // WITHOUT SPILL + // ========================================================================================= // + + val spillThreshold = 100 * 1000 + + /* + Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz + + Array with 1000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + ArrayBuffer 7821 / 7941 33.5 29.8 1.0X + ExternalAppendOnlyUnsafeRowArray 8798 / 8819 29.8 33.6 0.9X + */ + testAgainstRawArrayBuffer(spillThreshold, 1000, 1 << 18) + + /* + Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz + + Array with 30000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + ArrayBuffer 19200 / 19206 25.6 39.1 1.0X + ExternalAppendOnlyUnsafeRowArray 19558 / 19562 25.1 39.8 1.0X + */ + testAgainstRawArrayBuffer(spillThreshold, 30 * 1000, 1 << 14) + + /* + Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz + + Array with 100000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + ArrayBuffer 5949 / 6028 17.2 58.1 1.0X + ExternalAppendOnlyUnsafeRowArray 6078 / 6138 16.8 59.4 1.0X + */ + testAgainstRawArrayBuffer(spillThreshold, 100 * 1000, 1 << 10) + + // ========================================================================================= // + // WITH SPILL + // ========================================================================================= // + + /* + Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz + + Spilling with 1000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + UnsafeExternalSorter 9239 / 9470 28.4 35.2 1.0X + ExternalAppendOnlyUnsafeRowArray 8857 / 8909 29.6 33.8 1.0X + */ + testAgainstRawUnsafeExternalSorter(100 * 1000, 1000, 1 << 18) + + /* + Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz + + Spilling with 10000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + UnsafeExternalSorter 4 / 5 39.3 25.5 1.0X + ExternalAppendOnlyUnsafeRowArray 5 / 6 29.8 33.5 0.8X + */ + testAgainstRawUnsafeExternalSorter( + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt, 10 * 1000, 1 << 4) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArraySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArraySuite.scala new file mode 100644 index 000000000000..ecc7264d7944 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArraySuite.scala @@ -0,0 +1,374 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import java.util.ConcurrentModificationException + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark._ +import org.apache.spark.memory.MemoryTestingUtils +import org.apache.spark.sql.catalyst.expressions.UnsafeRow + +class ExternalAppendOnlyUnsafeRowArraySuite extends SparkFunSuite with LocalSparkContext { + private val random = new java.util.Random() + private var taskContext: TaskContext = _ + + override def afterAll(): Unit = TaskContext.unset() + + private def withExternalArray(inMemoryThreshold: Int, spillThreshold: Int) + (f: ExternalAppendOnlyUnsafeRowArray => Unit): Unit = { + sc = new SparkContext("local", "test", new SparkConf(false)) + + taskContext = MemoryTestingUtils.fakeTaskContext(SparkEnv.get) + TaskContext.setTaskContext(taskContext) + + val array = new ExternalAppendOnlyUnsafeRowArray( + taskContext.taskMemoryManager(), + SparkEnv.get.blockManager, + SparkEnv.get.serializerManager, + taskContext, + 1024, + SparkEnv.get.memoryManager.pageSizeBytes, + inMemoryThreshold, + spillThreshold) + try f(array) finally { + array.clear() + } + } + + private def insertRow(array: ExternalAppendOnlyUnsafeRowArray): Long = { + val valueInserted = random.nextLong() + + val row = new UnsafeRow(1) + row.pointTo(new Array[Byte](64), 16) + row.setLong(0, valueInserted) + array.add(row) + valueInserted + } + + private def checkIfValueExists(iterator: Iterator[UnsafeRow], expectedValue: Long): Unit = { + assert(iterator.hasNext) + val actualRow = iterator.next() + assert(actualRow.getLong(0) == expectedValue) + assert(actualRow.getSizeInBytes == 16) + } + + private def validateData( + array: ExternalAppendOnlyUnsafeRowArray, + expectedValues: ArrayBuffer[Long]): Iterator[UnsafeRow] = { + val iterator = array.generateIterator() + for (value <- expectedValues) { + checkIfValueExists(iterator, value) + } + + assert(!iterator.hasNext) + iterator + } + + private def populateRows( + array: ExternalAppendOnlyUnsafeRowArray, + numRowsToBePopulated: Int): ArrayBuffer[Long] = { + val populatedValues = new ArrayBuffer[Long] + populateRows(array, numRowsToBePopulated, populatedValues) + } + + private def populateRows( + array: ExternalAppendOnlyUnsafeRowArray, + numRowsToBePopulated: Int, + populatedValues: ArrayBuffer[Long]): ArrayBuffer[Long] = { + for (_ <- 0 until numRowsToBePopulated) { + populatedValues.append(insertRow(array)) + } + populatedValues + } + + private def getNumBytesSpilled: Long = { + TaskContext.get().taskMetrics().memoryBytesSpilled + } + + private def assertNoSpill(): Unit = { + assert(getNumBytesSpilled == 0) + } + + private def assertSpill(): Unit = { + assert(getNumBytesSpilled > 0) + } + + test("insert rows less than the inMemoryThreshold") { + val (inMemoryThreshold, spillThreshold) = (100, 50) + withExternalArray(inMemoryThreshold, spillThreshold) { array => + assert(array.isEmpty) + + val expectedValues = populateRows(array, 1) + assert(!array.isEmpty) + assert(array.length == 1) + + val iterator1 = validateData(array, expectedValues) + + // Add more rows (but not too many to trigger switch to [[UnsafeExternalSorter]]) + // Verify that NO spill has happened + populateRows(array, inMemoryThreshold - 1, expectedValues) + assert(array.length == inMemoryThreshold) + assertNoSpill() + + val iterator2 = validateData(array, expectedValues) + + assert(!iterator1.hasNext) + assert(!iterator2.hasNext) + } + } + + test("insert rows more than the inMemoryThreshold but less than spillThreshold") { + val (inMemoryThreshold, spillThreshold) = (10, 50) + withExternalArray(inMemoryThreshold, spillThreshold) { array => + assert(array.isEmpty) + val expectedValues = populateRows(array, inMemoryThreshold - 1) + assert(array.length == (inMemoryThreshold - 1)) + val iterator1 = validateData(array, expectedValues) + assertNoSpill() + + // Add more rows to trigger switch to [[UnsafeExternalSorter]] but not too many to cause a + // spill to happen. Verify that NO spill has happened + populateRows(array, spillThreshold - expectedValues.length - 1, expectedValues) + assert(array.length == spillThreshold - 1) + assertNoSpill() + + val iterator2 = validateData(array, expectedValues) + assert(!iterator2.hasNext) + + assert(!iterator1.hasNext) + intercept[ConcurrentModificationException](iterator1.next()) + } + } + + test("insert rows enough to force spill") { + val (inMemoryThreshold, spillThreshold) = (20, 10) + withExternalArray(inMemoryThreshold, spillThreshold) { array => + assert(array.isEmpty) + val expectedValues = populateRows(array, inMemoryThreshold - 1) + assert(array.length == (inMemoryThreshold - 1)) + val iterator1 = validateData(array, expectedValues) + assertNoSpill() + + // Add more rows to trigger switch to [[UnsafeExternalSorter]] and cause a spill to happen. + // Verify that spill has happened + populateRows(array, 2, expectedValues) + assert(array.length == inMemoryThreshold + 1) + assertSpill() + + val iterator2 = validateData(array, expectedValues) + assert(!iterator2.hasNext) + + assert(!iterator1.hasNext) + intercept[ConcurrentModificationException](iterator1.next()) + } + } + + test("iterator on an empty array should be empty") { + withExternalArray(inMemoryThreshold = 4, spillThreshold = 10) { array => + val iterator = array.generateIterator() + assert(array.isEmpty) + assert(array.length == 0) + assert(!iterator.hasNext) + } + } + + test("generate iterator with negative start index") { + withExternalArray(inMemoryThreshold = 100, spillThreshold = 56) { array => + val exception = + intercept[ArrayIndexOutOfBoundsException](array.generateIterator(startIndex = -10)) + + assert(exception.getMessage.contains( + "Invalid `startIndex` provided for generating iterator over the array") + ) + } + } + + test("generate iterator with start index exceeding array's size (without spill)") { + val (inMemoryThreshold, spillThreshold) = (20, 100) + withExternalArray(inMemoryThreshold, spillThreshold) { array => + populateRows(array, spillThreshold / 2) + + val exception = + intercept[ArrayIndexOutOfBoundsException]( + array.generateIterator(startIndex = spillThreshold * 10)) + assert(exception.getMessage.contains( + "Invalid `startIndex` provided for generating iterator over the array")) + } + } + + test("generate iterator with start index exceeding array's size (with spill)") { + val (inMemoryThreshold, spillThreshold) = (20, 100) + withExternalArray(inMemoryThreshold, spillThreshold) { array => + populateRows(array, spillThreshold * 2) + + val exception = + intercept[ArrayIndexOutOfBoundsException]( + array.generateIterator(startIndex = spillThreshold * 10)) + + assert(exception.getMessage.contains( + "Invalid `startIndex` provided for generating iterator over the array")) + } + } + + test("generate iterator with custom start index (without spill)") { + val (inMemoryThreshold, spillThreshold) = (20, 100) + withExternalArray(inMemoryThreshold, spillThreshold) { array => + val expectedValues = populateRows(array, inMemoryThreshold) + val startIndex = inMemoryThreshold / 2 + val iterator = array.generateIterator(startIndex = startIndex) + for (i <- startIndex until expectedValues.length) { + checkIfValueExists(iterator, expectedValues(i)) + } + } + } + + test("generate iterator with custom start index (with spill)") { + val (inMemoryThreshold, spillThreshold) = (20, 100) + withExternalArray(inMemoryThreshold, spillThreshold) { array => + val expectedValues = populateRows(array, spillThreshold * 10) + val startIndex = spillThreshold * 2 + val iterator = array.generateIterator(startIndex = startIndex) + for (i <- startIndex until expectedValues.length) { + checkIfValueExists(iterator, expectedValues(i)) + } + } + } + + test("test iterator invalidation (without spill)") { + withExternalArray(inMemoryThreshold = 10, spillThreshold = 100) { array => + // insert 2 rows, iterate until the first row + populateRows(array, 2) + + var iterator = array.generateIterator() + assert(iterator.hasNext) + iterator.next() + + // Adding more row(s) should invalidate any old iterators + populateRows(array, 1) + assert(!iterator.hasNext) + intercept[ConcurrentModificationException](iterator.next()) + + // Clearing the array should also invalidate any old iterators + iterator = array.generateIterator() + assert(iterator.hasNext) + iterator.next() + + array.clear() + assert(!iterator.hasNext) + intercept[ConcurrentModificationException](iterator.next()) + } + } + + test("test iterator invalidation (with spill)") { + val (inMemoryThreshold, spillThreshold) = (2, 10) + withExternalArray(inMemoryThreshold, spillThreshold) { array => + // Populate enough rows so that spill happens + populateRows(array, spillThreshold * 2) + assertSpill() + + var iterator = array.generateIterator() + assert(iterator.hasNext) + iterator.next() + + // Adding more row(s) should invalidate any old iterators + populateRows(array, 1) + assert(!iterator.hasNext) + intercept[ConcurrentModificationException](iterator.next()) + + // Clearing the array should also invalidate any old iterators + iterator = array.generateIterator() + assert(iterator.hasNext) + iterator.next() + + array.clear() + assert(!iterator.hasNext) + intercept[ConcurrentModificationException](iterator.next()) + } + } + + test("clear on an empty the array") { + withExternalArray(inMemoryThreshold = 2, spillThreshold = 3) { array => + val iterator = array.generateIterator() + assert(!iterator.hasNext) + + // multiple clear'ing should not have an side-effect + array.clear() + array.clear() + array.clear() + assert(array.isEmpty) + assert(array.length == 0) + + // Clearing an empty array should also invalidate any old iterators + assert(!iterator.hasNext) + intercept[ConcurrentModificationException](iterator.next()) + } + } + + test("clear array (without spill)") { + val (inMemoryThreshold, spillThreshold) = (10, 100) + withExternalArray(inMemoryThreshold, spillThreshold) { array => + // Populate rows ... but not enough to trigger spill + populateRows(array, inMemoryThreshold / 2) + assertNoSpill() + + // Clear the array + array.clear() + assert(array.isEmpty) + + // Re-populate few rows so that there is no spill + // Verify the data. Verify that there was no spill + val expectedValues = populateRows(array, inMemoryThreshold / 2) + validateData(array, expectedValues) + assertNoSpill() + + // Populate more rows .. enough to not trigger a spill. + // Verify the data. Verify that there was no spill + populateRows(array, inMemoryThreshold / 2, expectedValues) + validateData(array, expectedValues) + assertNoSpill() + } + } + + test("clear array (with spill)") { + val (inMemoryThreshold, spillThreshold) = (10, 20) + withExternalArray(inMemoryThreshold, spillThreshold) { array => + // Populate enough rows to trigger spill + populateRows(array, spillThreshold * 2) + val bytesSpilled = getNumBytesSpilled + assert(bytesSpilled > 0) + + // Clear the array + array.clear() + assert(array.isEmpty) + + // Re-populate the array ... but NOT upto the point that there is spill. + // Verify data. Verify that there was NO "extra" spill + val expectedValues = populateRows(array, spillThreshold / 2) + validateData(array, expectedValues) + assert(getNumBytesSpilled == bytesSpilled) + + // Populate more rows to trigger spill + // Verify the data. Verify that there was "extra" spill + populateRows(array, spillThreshold * 2, expectedValues) + validateData(array, expectedValues) + assert(getNumBytesSpilled > bytesSpilled) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala new file mode 100644 index 000000000000..a3d75b221ec3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/GlobalTempViewSuite.scala @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalog.Table +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType + +class GlobalTempViewSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + override protected def beforeAll(): Unit = { + super.beforeAll() + globalTempDB = spark.sharedState.globalTempViewManager.database + } + + private var globalTempDB: String = _ + + test("basic semantic") { + try { + sql("CREATE GLOBAL TEMP VIEW src AS SELECT 1, 'a'") + + // If there is no database in table name, we should try local temp view first, if not found, + // try table/view in current database, which is "default" in this case. So we expect + // NoSuchTableException here. + intercept[NoSuchTableException](spark.table("src")) + + // Use qualified name to refer to the global temp view explicitly. + checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, "a")) + + // Table name without database will never refer to a global temp view. + intercept[NoSuchTableException](sql("DROP VIEW src")) + + sql(s"DROP VIEW $globalTempDB.src") + // The global temp view should be dropped successfully. + intercept[NoSuchTableException](spark.table(s"$globalTempDB.src")) + + // We can also use Dataset API to create global temp view + Seq(1 -> "a").toDF("i", "j").createGlobalTempView("src") + checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, "a")) + + // Use qualified name to rename a global temp view. + sql(s"ALTER VIEW $globalTempDB.src RENAME TO src2") + intercept[NoSuchTableException](spark.table(s"$globalTempDB.src")) + checkAnswer(spark.table(s"$globalTempDB.src2"), Row(1, "a")) + + // Use qualified name to alter a global temp view. + sql(s"ALTER VIEW $globalTempDB.src2 AS SELECT 2, 'b'") + checkAnswer(spark.table(s"$globalTempDB.src2"), Row(2, "b")) + + // We can also use Catalog API to drop global temp view + spark.catalog.dropGlobalTempView("src2") + intercept[NoSuchTableException](spark.table(s"$globalTempDB.src2")) + + // We can also use Dataset API to replace global temp view + Seq(2 -> "b").toDF("i", "j").createOrReplaceGlobalTempView("src") + checkAnswer(spark.table(s"$globalTempDB.src"), Row(2, "b")) + } finally { + spark.catalog.dropGlobalTempView("src") + } + } + + test("global temp view is shared among all sessions") { + try { + sql("CREATE GLOBAL TEMP VIEW src AS SELECT 1, 2") + checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, 2)) + val newSession = spark.newSession() + checkAnswer(newSession.table(s"$globalTempDB.src"), Row(1, 2)) + } finally { + spark.catalog.dropGlobalTempView("src") + } + } + + test("global temp view database should be preserved") { + val e = intercept[AnalysisException](sql(s"CREATE DATABASE $globalTempDB")) + assert(e.message.contains("system preserved database")) + + val e2 = intercept[AnalysisException](sql(s"USE $globalTempDB")) + assert(e2.message.contains("system preserved database")) + } + + test("CREATE GLOBAL TEMP VIEW USING") { + withTempPath { path => + try { + Seq(1 -> "a").toDF("i", "j").write.parquet(path.getAbsolutePath) + sql(s"CREATE GLOBAL TEMP VIEW src USING parquet OPTIONS (PATH '${path.toURI}')") + checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, "a")) + sql(s"INSERT INTO $globalTempDB.src SELECT 2, 'b'") + checkAnswer(spark.table(s"$globalTempDB.src"), Row(1, "a") :: Row(2, "b") :: Nil) + } finally { + spark.catalog.dropGlobalTempView("src") + } + } + } + + test("CREATE TABLE LIKE should work for global temp view") { + try { + sql("CREATE GLOBAL TEMP VIEW src AS SELECT 1 AS a, '2' AS b") + sql(s"CREATE TABLE cloned LIKE $globalTempDB.src") + val tableMeta = spark.sessionState.catalog.getTableMetadata(TableIdentifier("cloned")) + assert(tableMeta.schema == new StructType().add("a", "int", false).add("b", "string", false)) + } finally { + spark.catalog.dropGlobalTempView("src") + sql("DROP TABLE default.cloned") + } + } + + test("list global temp views") { + try { + sql("CREATE GLOBAL TEMP VIEW v1 AS SELECT 3, 4") + sql("CREATE TEMP VIEW v2 AS SELECT 1, 2") + + checkAnswer(sql(s"SHOW TABLES IN $globalTempDB"), + Row(globalTempDB, "v1", true) :: + Row("", "v2", true) :: Nil) + + assert(spark.catalog.listTables(globalTempDB).collect().toSeq.map(_.name) == Seq("v1", "v2")) + } finally { + spark.catalog.dropTempView("v1") + spark.catalog.dropGlobalTempView("v2") + } + } + + test("should lookup global temp view if and only if global temp db is specified") { + try { + sql("CREATE GLOBAL TEMP VIEW same_name AS SELECT 3, 4") + sql("CREATE TEMP VIEW same_name AS SELECT 1, 2") + + checkAnswer(sql("SELECT * FROM same_name"), Row(1, 2)) + + // we never lookup global temp views if database is not specified in table name + spark.catalog.dropTempView("same_name") + intercept[AnalysisException](sql("SELECT * FROM same_name")) + + // Use qualified name to lookup a global temp view. + checkAnswer(sql(s"SELECT * FROM $globalTempDB.same_name"), Row(3, 4)) + } finally { + spark.catalog.dropTempView("same_name") + spark.catalog.dropGlobalTempView("same_name") + } + } + + test("public Catalog should recognize global temp view") { + try { + sql("CREATE GLOBAL TEMP VIEW src AS SELECT 1, 2") + + assert(spark.catalog.tableExists(globalTempDB, "src")) + assert(spark.catalog.getTable(globalTempDB, "src").toString == new Table( + name = "src", + database = globalTempDB, + description = null, + tableType = "TEMPORARY", + isTemporary = true).toString) + } finally { + spark.catalog.dropGlobalTempView("src") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/GroupedIteratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/GroupedIteratorSuite.scala index e7a08481cfa8..80340b5552c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/GroupedIteratorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/GroupedIteratorSuite.scala @@ -21,13 +21,13 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.types.{LongType, StringType, IntegerType, StructType} +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} class GroupedIteratorSuite extends SparkFunSuite { test("basic") { val schema = new StructType().add("i", IntegerType).add("s", StringType) - val encoder = RowEncoder(schema) + val encoder = RowEncoder(schema).resolveAndBind() val input = Seq(Row(1, "a"), Row(1, "b"), Row(2, "c")) val grouped = GroupedIterator(input.iterator.map(encoder.toRow), Seq('i.int.at(0)), schema.toAttributes) @@ -45,7 +45,7 @@ class GroupedIteratorSuite extends SparkFunSuite { test("group by 2 columns") { val schema = new StructType().add("i", IntegerType).add("l", LongType).add("s", StringType) - val encoder = RowEncoder(schema) + val encoder = RowEncoder(schema).resolveAndBind() val input = Seq( Row(1, 2L, "a"), @@ -72,7 +72,7 @@ class GroupedIteratorSuite extends SparkFunSuite { test("do nothing to the value iterator") { val schema = new StructType().add("i", IntegerType).add("s", StringType) - val encoder = RowEncoder(schema) + val encoder = RowEncoder(schema).resolveAndBind() val input = Seq(Row(1, "a"), Row(1, "b"), Row(2, "c")) val grouped = GroupedIterator(input.iterator.map(encoder.toRow), Seq('i.int.at(0)), schema.toAttributes) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala new file mode 100644 index 000000000000..78c1e5dae566 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + override def beforeAll(): Unit = { + super.beforeAll() + val data = (1 to 10).map(i => (i, s"data-$i", i % 2, if ((i % 2) == 0) "even" else "odd")) + .toDF("col1", "col2", "partcol1", "partcol2") + data.write.partitionBy("partcol1", "partcol2").mode("append").saveAsTable("srcpart") + } + + override protected def afterAll(): Unit = { + try { + sql("DROP TABLE IF EXISTS srcpart") + } finally { + super.afterAll() + } + } + + private def assertMetadataOnlyQuery(df: DataFrame): Unit = { + val localRelations = df.queryExecution.optimizedPlan.collect { + case l @ LocalRelation(_, _, _) => l + } + assert(localRelations.size == 1) + } + + private def assertNotMetadataOnlyQuery(df: DataFrame): Unit = { + val localRelations = df.queryExecution.optimizedPlan.collect { + case l @ LocalRelation(_, _, _) => l + } + assert(localRelations.size == 0) + } + + private def testMetadataOnly(name: String, sqls: String*): Unit = { + test(name) { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + sqls.foreach { case q => assertMetadataOnlyQuery(sql(q)) } + } + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") { + sqls.foreach { case q => assertNotMetadataOnlyQuery(sql(q)) } + } + } + } + + private def testNotMetadataOnly(name: String, sqls: String*): Unit = { + test(name) { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + sqls.foreach { case q => assertNotMetadataOnlyQuery(sql(q)) } + } + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") { + sqls.foreach { case q => assertNotMetadataOnlyQuery(sql(q)) } + } + } + } + + testMetadataOnly( + "Aggregate expression is partition columns", + "select partcol1 from srcpart group by partcol1", + "select partcol2 from srcpart where partcol1 = 0 group by partcol2") + + testMetadataOnly( + "Distinct aggregate function on partition columns", + "SELECT partcol1, count(distinct partcol2) FROM srcpart group by partcol1", + "SELECT partcol1, count(distinct partcol2) FROM srcpart where partcol1 = 0 group by partcol1") + + testMetadataOnly( + "Distinct on partition columns", + "select distinct partcol1, partcol2 from srcpart", + "select distinct c1 from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t") + + testMetadataOnly( + "Aggregate function on partition columns which have same result w or w/o DISTINCT keyword", + "select max(partcol1) from srcpart", + "select min(partcol1) from srcpart where partcol1 = 0", + "select first(partcol1) from srcpart", + "select last(partcol1) from srcpart where partcol1 = 0", + "select partcol2, min(partcol1) from srcpart where partcol1 = 0 group by partcol2", + "select max(c1) from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t") + + testNotMetadataOnly( + "Don't optimize metadata only query for non-partition columns", + "select col1 from srcpart group by col1", + "select partcol1, max(col1) from srcpart group by partcol1", + "select partcol1, count(distinct col1) from srcpart group by partcol1", + "select distinct partcol1, col1 from srcpart") + + testNotMetadataOnly( + "Don't optimize metadata only query for non-distinct aggregate function on partition columns", + "select partcol1, sum(partcol2) from srcpart group by partcol1", + "select partcol1, count(partcol2) from srcpart group by partcol1") + + testNotMetadataOnly( + "Don't optimize metadata only query for GroupingSet/Union operator", + "select partcol1, max(partcol2) from srcpart where partcol1 = 0 group by rollup (partcol1)", + "select partcol2 from (select partcol2 from srcpart where partcol1 = 0 union all " + + "select partcol2 from srcpart where partcol1 = 1) t group by partcol2") + + test("SPARK-21884 Fix StackOverflowError on MetadataOnlyQuery") { + withTable("t_1000") { + sql("CREATE TABLE t_1000 (a INT, p INT) USING PARQUET PARTITIONED BY (p)") + (1 to 1000).foreach(p => sql(s"ALTER TABLE t_1000 ADD PARTITION (p=$p)")) + sql("SELECT COUNT(DISTINCT p) FROM t_1000").collect() + } + } +} 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 2076c573b56c..63e17c7f372b 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 @@ -18,51 +18,41 @@ package org.apache.spark.sql.execution import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{execution, Row, SQLConf} +import org.apache.spark.sql.{execution, Row} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Literal, SortOrder} -import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, LeftOuter, RightOuter} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Repartition} import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin} +import org.apache.spark.sql.execution.columnar.InMemoryRelation +import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReusedExchangeExec, ReuseExchange, ShuffleExchange} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ - class PlannerSuite extends SharedSQLContext { import testImplicits._ setupTestData() private def testPartialAggregationPlan(query: LogicalPlan): Unit = { - val planner = sqlContext.planner + val planner = spark.sessionState.planner import planner._ - val plannedOption = HashAggregation(query).headOption.orElse(Aggregation(query).headOption) + val plannedOption = Aggregation(query).headOption val planned = plannedOption.getOrElse( fail(s"Could query play aggregation query $query. Is it an aggregation query?")) val aggregations = planned.collect { case n if n.nodeName contains "Aggregate" => n } - // For the new aggregation code path, there will be three aggregate operator for + // For the new aggregation code path, there will be four aggregate operator for // distinct aggregations. assert( - aggregations.size == 2 || aggregations.size == 3, + aggregations.size == 2 || aggregations.size == 4, s"The plan of query $query does not have partial aggregations.") } - test("unions are collapsed") { - val planner = sqlContext.planner - import planner._ - val query = testData.unionAll(testData).unionAll(testData).logicalPlan - val planned = BasicOperators(query).head - val logicalUnions = query collect { case u: logical.Union => u } - val physicalUnions = planned collect { case u: execution.Union => u } - - assert(logicalUnions.size === 2) - assert(physicalUnions.size === 1) - } - test("count is partially aggregated") { val query = testData.groupBy('value).agg(count('key)).queryExecution.analyzed testPartialAggregationPlan(query) @@ -81,26 +71,26 @@ class PlannerSuite extends SharedSQLContext { test("sizeInBytes estimation of limit operator for broadcast hash join optimization") { def checkPlan(fieldTypes: Seq[DataType]): Unit = { - withTempTable("testLimit") { + withTempView("testLimit") { val fields = fieldTypes.zipWithIndex.map { case (dataType, index) => StructField(s"c${index}", dataType, true) } :+ StructField("key", IntegerType, true) val schema = StructType(fields) val row = Row.fromSeq(Seq.fill(fields.size)(null)) val rowRDD = sparkContext.parallelize(row :: Nil) - sqlContext.createDataFrame(rowRDD, schema).registerTempTable("testLimit") + spark.createDataFrame(rowRDD, schema).createOrReplaceTempView("testLimit") val planned = sql( """ |SELECT l.a, l.b |FROM testData2 l JOIN (SELECT * FROM testLimit LIMIT 1) r ON (l.a = r.key) - """.stripMargin).queryExecution.executedPlan + """.stripMargin).queryExecution.sparkPlan - val broadcastHashJoins = planned.collect { case join: BroadcastHashJoin => join } - val shuffledHashJoins = planned.collect { case join: ShuffledHashJoin => join } + val broadcastHashJoins = planned.collect { case join: BroadcastHashJoinExec => join } + val sortMergeJoins = planned.collect { case join: SortMergeJoinExec => join } assert(broadcastHashJoins.size === 1, "Should use broadcast hash join") - assert(shuffledHashJoins.isEmpty, "Should not use shuffled hash join") + assert(sortMergeJoins.isEmpty, "Should not use sort merge join") } } @@ -141,50 +131,77 @@ class PlannerSuite extends SharedSQLContext { test("InMemoryRelation statistics propagation") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "81920") { - withTempTable("tiny") { - testData.limit(3).registerTempTable("tiny") + withTempView("tiny") { + testData.limit(3).createOrReplaceTempView("tiny") sql("CACHE TABLE tiny") val a = testData.as("a") - val b = sqlContext.table("tiny").as("b") - val planned = a.join(b, $"a.key" === $"b.key").queryExecution.executedPlan + val b = spark.table("tiny").as("b") + val planned = a.join(b, $"a.key" === $"b.key").queryExecution.sparkPlan - val broadcastHashJoins = planned.collect { case join: BroadcastHashJoin => join } - val shuffledHashJoins = planned.collect { case join: ShuffledHashJoin => join } + val broadcastHashJoins = planned.collect { case join: BroadcastHashJoinExec => join } + val sortMergeJoins = planned.collect { case join: SortMergeJoinExec => join } assert(broadcastHashJoins.size === 1, "Should use broadcast hash join") - assert(shuffledHashJoins.isEmpty, "Should not use shuffled hash join") + assert(sortMergeJoins.isEmpty, "Should not use shuffled hash join") - sqlContext.clearCache() + spark.catalog.clearCache() } } } - test("efficient limit -> project -> sort") { - { - val query = - testData.select('key, 'value).sort('key).limit(2).logicalPlan - val planned = sqlContext.planner.TakeOrderedAndProject(query) - assert(planned.head.isInstanceOf[execution.TakeOrderedAndProject]) - assert(planned.head.output === testData.select('key, 'value).logicalPlan.output) - } + test("SPARK-11390 explain should print PushedFilters of PhysicalRDD") { + withTempPath { file => + val path = file.getCanonicalPath + testData.write.parquet(path) + val df = spark.read.parquet(path) + df.createOrReplaceTempView("testPushed") - { - // We need to make sure TakeOrderedAndProject's output is correct when we push a project - // into it. - val query = - testData.select('key, 'value).sort('key).select('value, 'key).limit(2).logicalPlan - val planned = sqlContext.planner.TakeOrderedAndProject(query) - assert(planned.head.isInstanceOf[execution.TakeOrderedAndProject]) - assert(planned.head.output === testData.select('value, 'key).logicalPlan.output) + withTempView("testPushed") { + val exp = sql("select * from testPushed where key = 15").queryExecution.sparkPlan + assert(exp.toString.contains("PushedFilters: [IsNotNull(key), EqualTo(key,15)]")) + } } } + test("efficient terminal limit -> sort should use TakeOrderedAndProject") { + val query = testData.select('key, 'value).sort('key).limit(2) + val planned = query.queryExecution.executedPlan + assert(planned.isInstanceOf[execution.TakeOrderedAndProjectExec]) + assert(planned.output === testData.select('key, 'value).logicalPlan.output) + } + + test("terminal limit -> project -> sort should use TakeOrderedAndProject") { + val query = testData.select('key, 'value).sort('key).select('value, 'key).limit(2) + val planned = query.queryExecution.executedPlan + assert(planned.isInstanceOf[execution.TakeOrderedAndProjectExec]) + assert(planned.output === testData.select('value, 'key).logicalPlan.output) + } + + test("terminal limits that are not handled by TakeOrderedAndProject should use CollectLimit") { + val query = testData.select('value).limit(2) + val planned = query.queryExecution.sparkPlan + assert(planned.isInstanceOf[CollectLimitExec]) + assert(planned.output === testData.select('value).logicalPlan.output) + } + + test("TakeOrderedAndProject can appear in the middle of plans") { + val query = testData.select('key, 'value).sort('key).limit(2).filter('key === 3) + val planned = query.queryExecution.executedPlan + assert(planned.find(_.isInstanceOf[TakeOrderedAndProjectExec]).isDefined) + } + + test("CollectLimit can appear in the middle of a plan when caching is used") { + val query = testData.select('key, 'value).limit(2).cache() + val planned = query.queryExecution.optimizedPlan.asInstanceOf[InMemoryRelation] + assert(planned.child.isInstanceOf[CollectLimitExec]) + } + test("PartitioningCollection") { - withTempTable("normal", "small", "tiny") { - testData.registerTempTable("normal") - testData.limit(10).registerTempTable("small") - testData.limit(3).registerTempTable("tiny") + withTempView("normal", "small", "tiny") { + testData.createOrReplaceTempView("normal") + testData.limit(10).createOrReplaceTempView("small") + testData.limit(3).createOrReplaceTempView("tiny") // Disable broadcast join withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { @@ -197,9 +214,9 @@ class PlannerSuite extends SharedSQLContext { | JOIN tiny ON (small.key = tiny.key) """.stripMargin ).queryExecution.executedPlan.collect { - case exchange: Exchange => exchange + case exchange: ShuffleExchange => exchange }.length - assert(numExchanges === 3) + assert(numExchanges === 5) } { @@ -212,16 +229,31 @@ class PlannerSuite extends SharedSQLContext { | JOIN tiny ON (normal.key = tiny.key) """.stripMargin ).queryExecution.executedPlan.collect { - case exchange: Exchange => exchange + case exchange: ShuffleExchange => exchange }.length - assert(numExchanges === 3) + assert(numExchanges === 5) } } } } - // --- Unit tests of EnsureRequirements --------------------------------------------------------- + test("collapse adjacent repartitions") { + val doubleRepartitioned = testData.repartition(10).repartition(20).coalesce(5) + def countRepartitions(plan: LogicalPlan): Int = plan.collect { case r: Repartition => r }.length + assert(countRepartitions(doubleRepartitioned.queryExecution.logical) === 3) + assert(countRepartitions(doubleRepartitioned.queryExecution.optimizedPlan) === 2) + doubleRepartitioned.queryExecution.optimizedPlan match { + case Repartition (numPartitions, shuffle, Repartition(_, shuffleChild, _)) => + assert(numPartitions === 5) + assert(shuffle === false) + assert(shuffleChild === true) + } + } + + /////////////////////////////////////////////////////////////////////////// + // Unit tests of EnsureRequirements for Exchange + /////////////////////////////////////////////////////////////////////////// // When it comes to testing whether EnsureRequirements properly ensures distribution requirements, // there two dimensions that need to be considered: are the child partitionings compatible and @@ -266,9 +298,9 @@ class PlannerSuite extends SharedSQLContext { requiredChildDistribution = Seq(distribution, distribution), requiredChildOrdering = Seq(Seq.empty, Seq.empty) ) - val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan) + val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case e: Exchange => true }.isEmpty) { + if (outputPlan.collect { case e: ShuffleExchange => true }.isEmpty) { fail(s"Exchange should have been added:\n$outputPlan") } } @@ -286,7 +318,7 @@ class PlannerSuite extends SharedSQLContext { requiredChildDistribution = Seq(distribution, distribution), requiredChildOrdering = Seq(Seq.empty, Seq.empty) ) - val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan) + val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) } @@ -304,9 +336,9 @@ class PlannerSuite extends SharedSQLContext { requiredChildDistribution = Seq(distribution, distribution), requiredChildOrdering = Seq(Seq.empty, Seq.empty) ) - val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan) + val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case e: Exchange => true }.isEmpty) { + if (outputPlan.collect { case e: ShuffleExchange => true }.isEmpty) { fail(s"Exchange should have been added:\n$outputPlan") } } @@ -324,9 +356,9 @@ class PlannerSuite extends SharedSQLContext { requiredChildDistribution = Seq(distribution, distribution), requiredChildOrdering = Seq(Seq.empty, Seq.empty) ) - val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan) + val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case e: Exchange => true }.nonEmpty) { + if (outputPlan.collect { case e: ShuffleExchange => true }.nonEmpty) { fail(s"Exchange should not have been added:\n$outputPlan") } } @@ -334,7 +366,7 @@ class PlannerSuite extends SharedSQLContext { // This is a regression test for SPARK-9703 test("EnsureRequirements should not repartition if only ordering requirement is unsatisfied") { // Consider an operator that imposes both output distribution and ordering requirements on its - // children, such as sort sort merge join. If the distribution requirements are satisfied but + // children, such as sort merge join. If the distribution requirements are satisfied but // the output ordering requirements are unsatisfied, then the planner should only add sorts and // should not need to add additional shuffles / exchanges. val outputOrdering = Seq(SortOrder(Literal(1), Ascending)) @@ -347,63 +379,253 @@ class PlannerSuite extends SharedSQLContext { requiredChildDistribution = Seq(distribution, distribution), requiredChildOrdering = Seq(outputOrdering, outputOrdering) ) - val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan) + val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case e: Exchange => true }.nonEmpty) { + if (outputPlan.collect { case e: ShuffleExchange => true }.nonEmpty) { fail(s"No Exchanges should have been added:\n$outputPlan") } } - test("EnsureRequirements adds sort when there is no existing ordering") { - val orderingA = SortOrder(Literal(1), Ascending) - val orderingB = SortOrder(Literal(2), Ascending) - assert(orderingA != orderingB) - val inputPlan = DummySparkPlan( - children = DummySparkPlan(outputOrdering = Seq.empty) :: Nil, - requiredChildOrdering = Seq(Seq(orderingB)), - requiredChildDistribution = Seq(UnspecifiedDistribution) - ) - val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan) + test("EnsureRequirements eliminates Exchange if child has Exchange with same partitioning") { + val distribution = ClusteredDistribution(Literal(1) :: Nil) + val finalPartitioning = HashPartitioning(Literal(1) :: Nil, 5) + val childPartitioning = HashPartitioning(Literal(2) :: Nil, 5) + assert(!childPartitioning.satisfies(distribution)) + val inputPlan = ShuffleExchange(finalPartitioning, + DummySparkPlan( + children = DummySparkPlan(outputPartitioning = childPartitioning) :: Nil, + requiredChildDistribution = Seq(distribution), + requiredChildOrdering = Seq(Seq.empty)), + None) + + val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case s: TungstenSort => true; case s: Sort => true }.isEmpty) { - fail(s"Sort should have been added:\n$outputPlan") + if (outputPlan.collect { case e: ShuffleExchange => true }.size == 2) { + fail(s"Topmost Exchange should have been eliminated:\n$outputPlan") } } - test("EnsureRequirements skips sort when required ordering is prefix of existing ordering") { - val orderingA = SortOrder(Literal(1), Ascending) - val orderingB = SortOrder(Literal(2), Ascending) - assert(orderingA != orderingB) - val inputPlan = DummySparkPlan( - children = DummySparkPlan(outputOrdering = Seq(orderingA, orderingB)) :: Nil, - requiredChildOrdering = Seq(Seq(orderingA)), - requiredChildDistribution = Seq(UnspecifiedDistribution) - ) - val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan) + test("EnsureRequirements does not eliminate Exchange with different partitioning") { + val distribution = ClusteredDistribution(Literal(1) :: Nil) + // Number of partitions differ + val finalPartitioning = HashPartitioning(Literal(1) :: Nil, 8) + val childPartitioning = HashPartitioning(Literal(2) :: Nil, 5) + assert(!childPartitioning.satisfies(distribution)) + val inputPlan = ShuffleExchange(finalPartitioning, + DummySparkPlan( + children = DummySparkPlan(outputPartitioning = childPartitioning) :: Nil, + requiredChildDistribution = Seq(distribution), + requiredChildOrdering = Seq(Seq.empty)), + None) + + val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case s: TungstenSort => true; case s: Sort => true }.nonEmpty) { - fail(s"No sorts should have been added:\n$outputPlan") + if (outputPlan.collect { case e: ShuffleExchange => true }.size == 1) { + fail(s"Topmost Exchange should not have been eliminated:\n$outputPlan") } } - // This is a regression test for SPARK-11135 - test("EnsureRequirements adds sort when required ordering isn't a prefix of existing ordering") { - val orderingA = SortOrder(Literal(1), Ascending) - val orderingB = SortOrder(Literal(2), Ascending) - assert(orderingA != orderingB) + test("Reuse exchanges") { + val distribution = ClusteredDistribution(Literal(1) :: Nil) + val finalPartitioning = HashPartitioning(Literal(1) :: Nil, 5) + val childPartitioning = HashPartitioning(Literal(2) :: Nil, 5) + assert(!childPartitioning.satisfies(distribution)) + val shuffle = ShuffleExchange(finalPartitioning, + DummySparkPlan( + children = DummySparkPlan(outputPartitioning = childPartitioning) :: Nil, + requiredChildDistribution = Seq(distribution), + requiredChildOrdering = Seq(Seq.empty)), + None) + + val inputPlan = SortMergeJoinExec( + Literal(1) :: Nil, + Literal(1) :: Nil, + Inner, + None, + shuffle, + shuffle) + + val outputPlan = ReuseExchange(spark.sessionState.conf).apply(inputPlan) + if (outputPlan.collect { case e: ReusedExchangeExec => true }.size != 1) { + fail(s"Should re-use the shuffle:\n$outputPlan") + } + if (outputPlan.collect { case e: ShuffleExchange => true }.size != 1) { + fail(s"Should have only one shuffle:\n$outputPlan") + } + + // nested exchanges + val inputPlan2 = SortMergeJoinExec( + Literal(1) :: Nil, + Literal(1) :: Nil, + Inner, + None, + ShuffleExchange(finalPartitioning, inputPlan), + ShuffleExchange(finalPartitioning, inputPlan)) + + val outputPlan2 = ReuseExchange(spark.sessionState.conf).apply(inputPlan2) + if (outputPlan2.collect { case e: ReusedExchangeExec => true }.size != 2) { + fail(s"Should re-use the two shuffles:\n$outputPlan2") + } + if (outputPlan2.collect { case e: ShuffleExchange => true }.size != 2) { + fail(s"Should have only two shuffles:\n$outputPlan") + } + } + + /////////////////////////////////////////////////////////////////////////// + // Unit tests of EnsureRequirements for Sort + /////////////////////////////////////////////////////////////////////////// + + private val exprA = Literal(1) + private val exprB = Literal(2) + private val exprC = Literal(3) + private val orderingA = SortOrder(exprA, Ascending) + private val orderingB = SortOrder(exprB, Ascending) + private val orderingC = SortOrder(exprC, Ascending) + private val planA = DummySparkPlan(outputOrdering = Seq(orderingA), + outputPartitioning = HashPartitioning(exprA :: Nil, 5)) + private val planB = DummySparkPlan(outputOrdering = Seq(orderingB), + outputPartitioning = HashPartitioning(exprB :: Nil, 5)) + private val planC = DummySparkPlan(outputOrdering = Seq(orderingC), + outputPartitioning = HashPartitioning(exprC :: Nil, 5)) + + assert(orderingA != orderingB && orderingA != orderingC && orderingB != orderingC) + + private def assertSortRequirementsAreSatisfied( + childPlan: SparkPlan, + requiredOrdering: Seq[SortOrder], + shouldHaveSort: Boolean): Unit = { val inputPlan = DummySparkPlan( - children = DummySparkPlan(outputOrdering = Seq(orderingA)) :: Nil, - requiredChildOrdering = Seq(Seq(orderingA, orderingB)), + children = childPlan :: Nil, + requiredChildOrdering = Seq(requiredOrdering), requiredChildDistribution = Seq(UnspecifiedDistribution) ) - val outputPlan = EnsureRequirements(sqlContext).apply(inputPlan) + val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) - if (outputPlan.collect { case s: TungstenSort => true; case s: Sort => true }.isEmpty) { - fail(s"Sort should have been added:\n$outputPlan") + if (shouldHaveSort) { + if (outputPlan.collect { case s: SortExec => true }.isEmpty) { + fail(s"Sort should have been added:\n$outputPlan") + } + } else { + if (outputPlan.collect { case s: SortExec => true }.nonEmpty) { + fail(s"No sorts should have been added:\n$outputPlan") + } + } + } + + test("EnsureRequirements skips sort when either side of join keys is required after inner SMJ") { + Seq(Inner, Cross).foreach { joinType => + val innerSmj = SortMergeJoinExec(exprA :: Nil, exprB :: Nil, joinType, None, planA, planB) + // Both left and right keys should be sorted after the SMJ. + Seq(orderingA, orderingB).foreach { ordering => + assertSortRequirementsAreSatisfied( + childPlan = innerSmj, + requiredOrdering = Seq(ordering), + shouldHaveSort = false) + } + } + } + + test("EnsureRequirements skips sort when key order of a parent SMJ is propagated from its " + + "child SMJ") { + Seq(Inner, Cross).foreach { joinType => + val childSmj = SortMergeJoinExec(exprA :: Nil, exprB :: Nil, joinType, None, planA, planB) + val parentSmj = SortMergeJoinExec(exprB :: Nil, exprC :: Nil, joinType, None, childSmj, planC) + // After the second SMJ, exprA, exprB and exprC should all be sorted. + Seq(orderingA, orderingB, orderingC).foreach { ordering => + assertSortRequirementsAreSatisfied( + childPlan = parentSmj, + requiredOrdering = Seq(ordering), + shouldHaveSort = false) + } + } + } + + test("EnsureRequirements for sort operator after left outer sort merge join") { + // Only left key is sorted after left outer SMJ (thus doesn't need a sort). + val leftSmj = SortMergeJoinExec(exprA :: Nil, exprB :: Nil, LeftOuter, None, planA, planB) + Seq((orderingA, false), (orderingB, true)).foreach { case (ordering, needSort) => + assertSortRequirementsAreSatisfied( + childPlan = leftSmj, + requiredOrdering = Seq(ordering), + shouldHaveSort = needSort) } } - // --------------------------------------------------------------------------------------------- + test("EnsureRequirements for sort operator after right outer sort merge join") { + // Only right key is sorted after right outer SMJ (thus doesn't need a sort). + val rightSmj = SortMergeJoinExec(exprA :: Nil, exprB :: Nil, RightOuter, None, planA, planB) + Seq((orderingA, true), (orderingB, false)).foreach { case (ordering, needSort) => + assertSortRequirementsAreSatisfied( + childPlan = rightSmj, + requiredOrdering = Seq(ordering), + shouldHaveSort = needSort) + } + } + + test("EnsureRequirements adds sort after full outer sort merge join") { + // Neither keys is sorted after full outer SMJ, so they both need sorts. + val fullSmj = SortMergeJoinExec(exprA :: Nil, exprB :: Nil, FullOuter, None, planA, planB) + Seq(orderingA, orderingB).foreach { ordering => + assertSortRequirementsAreSatisfied( + childPlan = fullSmj, + requiredOrdering = Seq(ordering), + shouldHaveSort = true) + } + } + + test("EnsureRequirements adds sort when there is no existing ordering") { + assertSortRequirementsAreSatisfied( + childPlan = DummySparkPlan(outputOrdering = Seq.empty), + requiredOrdering = Seq(orderingB), + shouldHaveSort = true) + } + + test("EnsureRequirements skips sort when required ordering is prefix of existing ordering") { + assertSortRequirementsAreSatisfied( + childPlan = DummySparkPlan(outputOrdering = Seq(orderingA, orderingB)), + requiredOrdering = Seq(orderingA), + shouldHaveSort = false) + } + + test("EnsureRequirements skips sort when required ordering is semantically equal to " + + "existing ordering") { + val exprId: ExprId = NamedExpression.newExprId + val attribute1 = + AttributeReference( + name = "col1", + dataType = LongType, + nullable = false + ) (exprId = exprId, + qualifier = Some("col1_qualifier") + ) + + val attribute2 = + AttributeReference( + name = "col1", + dataType = LongType, + nullable = false + ) (exprId = exprId) + + val orderingA1 = SortOrder(attribute1, Ascending) + val orderingA2 = SortOrder(attribute2, Ascending) + + assert(orderingA1 != orderingA2, s"$orderingA1 should NOT equal to $orderingA2") + assert(orderingA1.semanticEquals(orderingA2), + s"$orderingA1 should be semantically equal to $orderingA2") + + assertSortRequirementsAreSatisfied( + childPlan = DummySparkPlan(outputOrdering = Seq(orderingA1)), + requiredOrdering = Seq(orderingA2), + shouldHaveSort = false) + } + + // This is a regression test for SPARK-11135 + test("EnsureRequirements adds sort when required ordering isn't a prefix of existing ordering") { + assertSortRequirementsAreSatisfied( + childPlan = DummySparkPlan(outputOrdering = Seq(orderingA)), + requiredOrdering = Seq(orderingA, orderingB), + shouldHaveSort = true) + } } // Used for unit-testing EnsureRequirements diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala new file mode 100644 index 000000000000..964440346deb --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} +import org.apache.spark.sql.test.SharedSQLContext + +class QueryExecutionSuite extends SharedSQLContext { + test("toString() exception/error handling") { + spark.experimental.extraStrategies = Seq( + new SparkStrategy { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = Nil + }) + + def qe: QueryExecution = new QueryExecution(spark, OneRowRelation()) + + // Nothing! + assert(qe.toString.contains("OneRowRelation")) + + // Throw an AnalysisException - this should be captured. + spark.experimental.extraStrategies = Seq( + new SparkStrategy { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = + throw new AnalysisException("exception") + }) + assert(qe.toString.contains("org.apache.spark.sql.AnalysisException")) + + // Throw an Error - this should not be captured. + spark.experimental.extraStrategies = Seq( + new SparkStrategy { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = + throw new Error("error") + }) + val error = intercept[Error](qe.toString) + assert(error.getMessage.contains("error")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala new file mode 100644 index 000000000000..6abcb1f06796 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.errors._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.util.CompletionIterator +import org.apache.spark.util.collection.ExternalSorter + + +/** + * A reference sort implementation used to compare against our normal sort. + */ +case class ReferenceSort( + sortOrder: Seq[SortOrder], + global: Boolean, + child: SparkPlan) + extends UnaryExecNode { + + override def requiredChildDistribution: Seq[Distribution] = + if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil + + protected override def doExecute(): RDD[InternalRow] = attachTree(this, "sort") { + child.execute().mapPartitions( { iterator => + val ordering = newOrdering(sortOrder, child.output) + val sorter = new ExternalSorter[InternalRow, Null, InternalRow]( + TaskContext.get(), ordering = Some(ordering)) + sorter.insertAll(iterator.map(r => (r.copy(), null))) + val baseIterator = sorter.iterator.map(_._1) + val context = TaskContext.get() + context.taskMetrics().incDiskBytesSpilled(sorter.diskBytesSpilled) + context.taskMetrics().incMemoryBytesSpilled(sorter.memoryBytesSpilled) + context.taskMetrics().incPeakExecutionMemory(sorter.peakMemoryUsedBytes) + CompletionIterator[InternalRow, Iterator[InternalRow]](baseIterator, sorter.stop()) + }, preservesPartitioning = true) + } + + override def output: Seq[Attribute] = child.output + + override def outputOrdering: Seq[SortOrder] = sortOrder + + override def outputPartitioning: Partitioning = child.outputPartitioning +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RowFormatConvertersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RowFormatConvertersSuite.scala deleted file mode 100644 index b3fceeab64cf..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RowFormatConvertersSuite.scala +++ /dev/null @@ -1,129 +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.execution - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute, Literal, IsNull} -import org.apache.spark.sql.catalyst.util.GenericArrayData -import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{ArrayType, StringType} -import org.apache.spark.unsafe.types.UTF8String - -class RowFormatConvertersSuite extends SparkPlanTest with SharedSQLContext { - - private def getConverters(plan: SparkPlan): Seq[SparkPlan] = plan.collect { - case c: ConvertToUnsafe => c - case c: ConvertToSafe => c - } - - private val outputsSafe = Sort(Nil, false, PhysicalRDD(Seq.empty, null, "name")) - assert(!outputsSafe.outputsUnsafeRows) - private val outputsUnsafe = TungstenSort(Nil, false, PhysicalRDD(Seq.empty, null, "name")) - assert(outputsUnsafe.outputsUnsafeRows) - - test("planner should insert unsafe->safe conversions when required") { - val plan = Limit(10, outputsUnsafe) - val preparedPlan = sqlContext.prepareForExecution.execute(plan) - assert(preparedPlan.children.head.isInstanceOf[ConvertToSafe]) - } - - test("filter can process unsafe rows") { - val plan = Filter(IsNull(IsNull(Literal(1))), outputsUnsafe) - val preparedPlan = sqlContext.prepareForExecution.execute(plan) - assert(getConverters(preparedPlan).size === 1) - assert(preparedPlan.outputsUnsafeRows) - } - - test("filter can process safe rows") { - val plan = Filter(IsNull(IsNull(Literal(1))), outputsSafe) - val preparedPlan = sqlContext.prepareForExecution.execute(plan) - assert(getConverters(preparedPlan).isEmpty) - assert(!preparedPlan.outputsUnsafeRows) - } - - test("execute() fails an assertion if inputs rows are of different formats") { - val e = intercept[AssertionError] { - Union(Seq(outputsSafe, outputsUnsafe)).execute() - } - assert(e.getMessage.contains("format")) - } - - test("union requires all of its input rows' formats to agree") { - val plan = Union(Seq(outputsSafe, outputsUnsafe)) - assert(plan.canProcessSafeRows && plan.canProcessUnsafeRows) - val preparedPlan = sqlContext.prepareForExecution.execute(plan) - assert(preparedPlan.outputsUnsafeRows) - } - - test("union can process safe rows") { - val plan = Union(Seq(outputsSafe, outputsSafe)) - val preparedPlan = sqlContext.prepareForExecution.execute(plan) - assert(!preparedPlan.outputsUnsafeRows) - } - - test("union can process unsafe rows") { - val plan = Union(Seq(outputsUnsafe, outputsUnsafe)) - val preparedPlan = sqlContext.prepareForExecution.execute(plan) - assert(preparedPlan.outputsUnsafeRows) - } - - test("round trip with ConvertToUnsafe and ConvertToSafe") { - val input = Seq(("hello", 1), ("world", 2)) - checkAnswer( - sqlContext.createDataFrame(input), - plan => ConvertToSafe(ConvertToUnsafe(plan)), - input.map(Row.fromTuple) - ) - } - - test("SPARK-9683: copy UTF8String when convert unsafe array/map to safe") { - SparkPlan.currentContext.set(sqlContext) - val schema = ArrayType(StringType) - val rows = (1 to 100).map { i => - InternalRow(new GenericArrayData(Array[Any](UTF8String.fromString(i.toString)))) - } - val relation = LocalTableScan(Seq(AttributeReference("t", schema)()), rows) - - val plan = - DummyPlan( - ConvertToSafe( - ConvertToUnsafe(relation))) - assert(plan.execute().collect().map(_.getUTF8String(0).toString) === (1 to 100).map(_.toString)) - } -} - -case class DummyPlan(child: SparkPlan) extends UnaryNode { - - override protected def doExecute(): RDD[InternalRow] = { - child.execute().mapPartitions { iter => - // This `DummyPlan` is in safe mode, so we don't need to do copy even we hold some - // values gotten from the incoming rows. - // we cache all strings here to make sure we have deep copied UTF8String inside incoming - // safe InternalRow. - val strings = new scala.collection.mutable.ArrayBuffer[UTF8String] - iter.foreach { row => - strings += row.getArray(0).getUTF8String(0) - } - strings.map(InternalRow(_)).iterator - } - } - - override def output: Seq[Attribute] = Seq(AttributeReference("a", StringType)()) -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala index 63639681ef80..f6b006b98edd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala @@ -19,35 +19,38 @@ package org.apache.spark.sql.execution import java.util.Properties -import scala.collection.parallel.CompositeThrowable - import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} -import org.apache.spark.sql.SQLContext +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} +import org.apache.spark.sql.SparkSession class SQLExecutionSuite extends SparkFunSuite { test("concurrent query execution (SPARK-10548)") { - // Try to reproduce the issue with the old SparkContext val conf = new SparkConf() .setMaster("local[*]") .setAppName("test") - val badSparkContext = new BadSparkContext(conf) + val goodSparkContext = new SparkContext(conf) try { - testConcurrentQueryExecution(badSparkContext) - fail("unable to reproduce SPARK-10548") - } catch { - case e: IllegalArgumentException => - assert(e.getMessage.contains(SQLExecution.EXECUTION_ID_KEY)) + testConcurrentQueryExecution(goodSparkContext) } finally { - badSparkContext.stop() + goodSparkContext.stop() } + } - // Verify that the issue is fixed with the latest SparkContext - val goodSparkContext = new SparkContext(conf) + test("concurrent query execution with fork-join pool (SPARK-13747)") { + val spark = SparkSession.builder + .master("local[*]") + .appName("test") + .getOrCreate() + + import spark.implicits._ try { - testConcurrentQueryExecution(goodSparkContext) + // Should not throw IllegalArgumentException + (1 to 100).par.foreach { _ => + spark.sparkContext.parallelize(1 to 5).map { i => (i, i) }.toDF("a", "b").count() + } } finally { - goodSparkContext.stop() + spark.sparkContext.stop() } } @@ -55,8 +58,8 @@ class SQLExecutionSuite extends SparkFunSuite { * Trigger SPARK-10548 by mocking a parent and its child thread executing queries concurrently. */ private def testConcurrentQueryExecution(sc: SparkContext): Unit = { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ + val spark = SparkSession.builder.getOrCreate() + import spark.implicits._ // Initialize local properties. This is necessary for the test to pass. sc.getLocalProperties @@ -87,15 +90,37 @@ class SQLExecutionSuite extends SparkFunSuite { } } -} -/** - * A bad [[SparkContext]] that does not clone the inheritable thread local properties - * when passing them to children threads. - */ -private class BadSparkContext(conf: SparkConf) extends SparkContext(conf) { - protected[spark] override val localProperties = new InheritableThreadLocal[Properties] { - override protected def childValue(parent: Properties): Properties = new Properties(parent) - override protected def initialValue(): Properties = new Properties() + test("Finding QueryExecution for given executionId") { + val spark = SparkSession.builder.master("local[*]").appName("test").getOrCreate() + import spark.implicits._ + + var queryExecution: QueryExecution = null + + spark.sparkContext.addSparkListener(new SparkListener { + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + val executionIdStr = jobStart.properties.getProperty(SQLExecution.EXECUTION_ID_KEY) + if (executionIdStr != null) { + queryExecution = SQLExecution.getQueryExecution(executionIdStr.toLong) + } + SQLExecutionSuite.canProgress = true + } + }) + + val df = spark.range(1).map { x => + while (!SQLExecutionSuite.canProgress) { + Thread.sleep(1) + } + x + } + df.collect() + + assert(df.queryExecution === queryExecution) + + spark.stop() } } + +object SQLExecutionSuite { + @volatile var canProgress = false +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala new file mode 100644 index 000000000000..c2e62b987e0c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.json4s.jackson.JsonMethods.parse + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart +import org.apache.spark.util.JsonProtocol + +class SQLJsonProtocolSuite extends SparkFunSuite { + + test("SparkPlanGraph backward compatibility: metadata") { + val SQLExecutionStartJsonString = + """ + |{ + | "Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart", + | "executionId":0, + | "description":"test desc", + | "details":"test detail", + | "physicalPlanDescription":"test plan", + | "sparkPlanInfo": { + | "nodeName":"TestNode", + | "simpleString":"test string", + | "children":[], + | "metadata":{}, + | "metrics":[] + | }, + | "time":0 + |} + """.stripMargin + val reconstructedEvent = JsonProtocol.sparkEventFromJson(parse(SQLExecutionStartJsonString)) + val expectedEvent = SparkListenerSQLExecutionStart(0, "test desc", "test detail", "test plan", + new SparkPlanInfo("TestNode", "test string", Nil, Nil), 0) + assert(reconstructedEvent == expectedEvent) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala new file mode 100644 index 000000000000..6761f05bb462 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -0,0 +1,682 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} + +class SimpleSQLViewSuite extends SQLViewSuite with SharedSQLContext + +/** + * A suite for testing view related functionality. + */ +abstract class SQLViewSuite extends QueryTest with SQLTestUtils { + import testImplicits._ + + protected override def beforeAll(): Unit = { + super.beforeAll() + // Create a simple table with two columns: id and id1 + spark.range(1, 10).selectExpr("id", "id id1").write.format("json").saveAsTable("jt") + } + + protected override def afterAll(): Unit = { + try { + spark.sql(s"DROP TABLE IF EXISTS jt") + } finally { + super.afterAll() + } + } + + test("create a permanent view on a permanent view") { + withView("jtv1", "jtv2") { + sql("CREATE VIEW jtv1 AS SELECT * FROM jt WHERE id > 3") + sql("CREATE VIEW jtv2 AS SELECT * FROM jtv1 WHERE id < 6") + checkAnswer(sql("select count(*) FROM jtv2"), Row(2)) + } + } + + test("create a temp view on a permanent view") { + withView("jtv1", "temp_jtv1") { + sql("CREATE VIEW jtv1 AS SELECT * FROM jt WHERE id > 3") + sql("CREATE TEMPORARY VIEW temp_jtv1 AS SELECT * FROM jtv1 WHERE id < 6") + checkAnswer(sql("select count(*) FROM temp_jtv1"), Row(2)) + } + } + + test("create a temp view on a temp view") { + withView("temp_jtv1", "temp_jtv2") { + sql("CREATE TEMPORARY VIEW temp_jtv1 AS SELECT * FROM jt WHERE id > 3") + sql("CREATE TEMPORARY VIEW temp_jtv2 AS SELECT * FROM temp_jtv1 WHERE id < 6") + checkAnswer(sql("select count(*) FROM temp_jtv2"), Row(2)) + } + } + + test("create a permanent view on a temp view") { + withView("jtv1", "temp_jtv1", "global_temp_jtv1") { + sql("CREATE TEMPORARY VIEW temp_jtv1 AS SELECT * FROM jt WHERE id > 3") + var e = intercept[AnalysisException] { + sql("CREATE VIEW jtv1 AS SELECT * FROM temp_jtv1 WHERE id < 6") + }.getMessage + assert(e.contains("Not allowed to create a permanent view `jtv1` by " + + "referencing a temporary view `temp_jtv1`")) + + val globalTempDB = spark.sharedState.globalTempViewManager.database + sql("CREATE GLOBAL TEMP VIEW global_temp_jtv1 AS SELECT * FROM jt WHERE id > 0") + e = intercept[AnalysisException] { + sql(s"CREATE VIEW jtv1 AS SELECT * FROM $globalTempDB.global_temp_jtv1 WHERE id < 6") + }.getMessage + assert(e.contains(s"Not allowed to create a permanent view `jtv1` by referencing " + + s"a temporary view `global_temp`.`global_temp_jtv1`")) + } + } + + test("error handling: existing a table with the duplicate name when creating/altering a view") { + withTable("tab1") { + sql("CREATE TABLE tab1 (id int) USING parquet") + var e = intercept[AnalysisException] { + sql("CREATE OR REPLACE VIEW tab1 AS SELECT * FROM jt") + }.getMessage + assert(e.contains("`tab1` is not a view")) + e = intercept[AnalysisException] { + sql("CREATE VIEW tab1 AS SELECT * FROM jt") + }.getMessage + assert(e.contains("`tab1` is not a view")) + e = intercept[AnalysisException] { + sql("ALTER VIEW tab1 AS SELECT * FROM jt") + }.getMessage + assert(e.contains("`tab1` is not a view")) + } + } + + test("existing a table with the duplicate name when CREATE VIEW IF NOT EXISTS") { + withTable("tab1") { + sql("CREATE TABLE tab1 (id int) USING parquet") + sql("CREATE VIEW IF NOT EXISTS tab1 AS SELECT * FROM jt") + checkAnswer(sql("select count(*) FROM tab1"), Row(0)) + } + } + + test("Issue exceptions for ALTER VIEW on the temporary view") { + val viewName = "testView" + withTempView(viewName) { + spark.range(10).createTempView(viewName) + assertNoSuchTable(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')") + assertNoSuchTable(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") + } + } + + test("Issue exceptions for ALTER TABLE on the temporary view") { + val viewName = "testView" + withTempView(viewName) { + spark.range(10).createTempView(viewName) + assertNoSuchTable(s"ALTER TABLE $viewName SET SERDE 'whatever'") + assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a=1, b=2) SET SERDE 'whatever'") + assertNoSuchTable(s"ALTER TABLE $viewName SET SERDEPROPERTIES ('p' = 'an')") + assertNoSuchTable(s"ALTER TABLE $viewName SET LOCATION '/path/to/your/lovely/heart'") + assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') SET LOCATION '/path/to/home'") + assertNoSuchTable(s"ALTER TABLE $viewName ADD IF NOT EXISTS PARTITION (a='4', b='8')") + assertNoSuchTable(s"ALTER TABLE $viewName DROP PARTITION (a='4', b='8')") + assertNoSuchTable(s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION (a='5')") + assertNoSuchTable(s"ALTER TABLE $viewName RECOVER PARTITIONS") + } + } + + test("Issue exceptions for other table DDL on the temporary view") { + val viewName = "testView" + withTempView(viewName) { + spark.range(10).createTempView(viewName) + + val e = intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $viewName SELECT 1") + }.getMessage + assert(e.contains("Inserting into an RDD-based table is not allowed")) + + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource("data/files/employee.dat") + assertNoSuchTable(s"""LOAD DATA LOCAL INPATH "$dataFilePath" INTO TABLE $viewName""") + assertNoSuchTable(s"TRUNCATE TABLE $viewName") + assertNoSuchTable(s"SHOW CREATE TABLE $viewName") + assertNoSuchTable(s"SHOW PARTITIONS $viewName") + assertNoSuchTable(s"ANALYZE TABLE $viewName COMPUTE STATISTICS") + assertNoSuchTable(s"ANALYZE TABLE $viewName COMPUTE STATISTICS FOR COLUMNS id") + } + } + + private def assertNoSuchTable(query: String): Unit = { + intercept[NoSuchTableException] { + sql(query) + } + } + + test("error handling: insert/load/truncate table commands against a view") { + val viewName = "testView" + withView(viewName) { + sql(s"CREATE VIEW $viewName AS SELECT id FROM jt") + var e = intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $viewName SELECT 1") + }.getMessage + assert(e.contains("Inserting into a view is not allowed. View: `default`.`testview`")) + + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource("data/files/employee.dat") + e = intercept[AnalysisException] { + sql(s"""LOAD DATA LOCAL INPATH "$dataFilePath" INTO TABLE $viewName""") + }.getMessage + assert(e.contains(s"Target table in LOAD DATA cannot be a view: `default`.`testview`")) + + e = intercept[AnalysisException] { + sql(s"TRUNCATE TABLE $viewName") + }.getMessage + assert(e.contains(s"Operation not allowed: TRUNCATE TABLE on views: `default`.`testview`")) + } + } + + test("error handling: fail if the view sql itself is invalid") { + // A database that does not exist + assertInvalidReference("CREATE OR REPLACE VIEW myabcdview AS SELECT * FROM db_not_exist234.jt") + + // A table that does not exist + assertInvalidReference("CREATE OR REPLACE VIEW myabcdview AS SELECT * FROM table_not_exist345") + + // A column that does not exist + intercept[AnalysisException] { + sql("CREATE OR REPLACE VIEW myabcdview AS SELECT random1234 FROM jt").collect() + } + } + + private def assertInvalidReference(query: String): Unit = { + val e = intercept[AnalysisException] { + sql(query) + }.getMessage + assert(e.contains("Table or view not found")) + } + + + test("error handling: fail if the temp view name contains the database prefix") { + // Fully qualified table name like "database.table" is not allowed for temporary view + val e = intercept[AnalysisException] { + sql("CREATE OR REPLACE TEMPORARY VIEW default.myabcdview AS SELECT * FROM jt") + } + assert(e.message.contains("It is not allowed to add database prefix")) + } + + test("error handling: disallow IF NOT EXISTS for CREATE TEMPORARY VIEW") { + val e = intercept[AnalysisException] { + sql("CREATE TEMPORARY VIEW IF NOT EXISTS myabcdview AS SELECT * FROM jt") + } + assert(e.message.contains("It is not allowed to define a TEMPORARY view with IF NOT EXISTS")) + } + + test("error handling: fail if the temp view sql itself is invalid") { + // A database that does not exist + assertInvalidReference( + "CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT * FROM db_not_exist234.jt") + + // A table that does not exist + assertInvalidReference( + "CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT * FROM table_not_exist1345") + + // A column that does not exist, for temporary view + intercept[AnalysisException] { + sql("CREATE OR REPLACE TEMPORARY VIEW myabcdview AS SELECT random1234 FROM jt") + } + } + + test("correctly parse CREATE VIEW statement") { + withView("testView") { + sql( + """CREATE VIEW IF NOT EXISTS + |default.testView (c1 COMMENT 'blabla', c2 COMMENT 'blabla') + |TBLPROPERTIES ('a' = 'b') + |AS SELECT * FROM jt + |""".stripMargin) + checkAnswer(sql("SELECT c1, c2 FROM testView ORDER BY c1"), (1 to 9).map(i => Row(i, i))) + } + } + + test("correctly parse a nested view") { + withTempDatabase { db => + withView("view1", "view2", s"$db.view3") { + sql("CREATE VIEW view1(x, y) AS SELECT * FROM jt") + + // Create a nested view in the same database. + sql("CREATE VIEW view2(id, id1) AS SELECT * FROM view1") + checkAnswer(sql("SELECT * FROM view2 ORDER BY id"), (1 to 9).map(i => Row(i, i))) + + // Create a nested view in a different database. + activateDatabase(db) { + sql(s"CREATE VIEW $db.view3(id, id1) AS SELECT * FROM default.view1") + checkAnswer(sql("SELECT * FROM view3 ORDER BY id"), (1 to 9).map(i => Row(i, i))) + } + } + } + } + + test("correctly parse CREATE TEMPORARY VIEW statement") { + withView("testView") { + sql( + """CREATE TEMPORARY VIEW + |testView (c1 COMMENT 'blabla', c2 COMMENT 'blabla') + |TBLPROPERTIES ('a' = 'b') + |AS SELECT * FROM jt + |""".stripMargin) + checkAnswer(sql("SELECT c1, c2 FROM testView ORDER BY c1"), (1 to 9).map(i => Row(i, i))) + } + } + + test("should NOT allow CREATE TEMPORARY VIEW when TEMPORARY VIEW with same name exists") { + withView("testView") { + sql("CREATE TEMPORARY VIEW testView AS SELECT id FROM jt") + + val e = intercept[AnalysisException] { + sql("CREATE TEMPORARY VIEW testView AS SELECT id FROM jt") + } + + assert(e.message.contains("Temporary table") && e.message.contains("already exists")) + } + } + + test("should allow CREATE TEMPORARY VIEW when a permanent VIEW with same name exists") { + withView("testView", "default.testView") { + sql("CREATE VIEW testView AS SELECT id FROM jt") + sql("CREATE TEMPORARY VIEW testView AS SELECT id FROM jt") + } + } + + test("should allow CREATE permanent VIEW when a TEMPORARY VIEW with same name exists") { + withView("testView", "default.testView") { + sql("CREATE TEMPORARY VIEW testView AS SELECT id FROM jt") + sql("CREATE VIEW testView AS SELECT id FROM jt") + } + } + + test("correctly handle CREATE VIEW IF NOT EXISTS") { + withTable("jt2") { + withView("testView") { + sql("CREATE VIEW testView AS SELECT id FROM jt") + + val df = (1 until 10).map(i => i -> i).toDF("i", "j") + df.write.format("json").saveAsTable("jt2") + sql("CREATE VIEW IF NOT EXISTS testView AS SELECT * FROM jt2") + + // make sure our view doesn't change. + checkAnswer(sql("SELECT * FROM testView ORDER BY id"), (1 to 9).map(i => Row(i))) + } + } + } + + test(s"correctly handle CREATE OR REPLACE TEMPORARY VIEW") { + withTable("jt2") { + withView("testView") { + sql("CREATE OR REPLACE TEMPORARY VIEW testView AS SELECT id FROM jt") + checkAnswer(sql("SELECT * FROM testView ORDER BY id"), (1 to 9).map(i => Row(i))) + + sql("CREATE OR REPLACE TEMPORARY VIEW testView AS SELECT id AS i, id AS j FROM jt") + // make sure the view has been changed. + checkAnswer(sql("SELECT * FROM testView ORDER BY i"), (1 to 9).map(i => Row(i, i))) + } + } + } + + test("correctly handle CREATE OR REPLACE VIEW") { + withTable("jt2") { + sql("CREATE OR REPLACE VIEW testView AS SELECT id FROM jt") + checkAnswer(sql("SELECT * FROM testView ORDER BY id"), (1 to 9).map(i => Row(i))) + + val df = (1 until 10).map(i => i -> i).toDF("i", "j") + df.write.format("json").saveAsTable("jt2") + sql("CREATE OR REPLACE VIEW testView AS SELECT * FROM jt2") + // make sure the view has been changed. + checkAnswer(sql("SELECT * FROM testView ORDER BY i"), (1 to 9).map(i => Row(i, i))) + + sql("DROP VIEW testView") + + val e = intercept[AnalysisException] { + sql("CREATE OR REPLACE VIEW IF NOT EXISTS testView AS SELECT id FROM jt") + } + assert(e.message.contains( + "CREATE VIEW with both IF NOT EXISTS and REPLACE is not allowed")) + } + } + + test("correctly handle ALTER VIEW") { + withTable("jt2") { + withView("testView") { + sql("CREATE VIEW testView AS SELECT id FROM jt") + + val df = (1 until 10).map(i => i -> i).toDF("i", "j") + df.write.format("json").saveAsTable("jt2") + sql("ALTER VIEW testView AS SELECT * FROM jt2") + // make sure the view has been changed. + checkAnswer(sql("SELECT * FROM testView ORDER BY i"), (1 to 9).map(i => Row(i, i))) + } + } + } + + test("correctly handle ALTER VIEW on a referenced view") { + withView("view1", "view2") { + sql("CREATE VIEW view1(x, y) AS SELECT * FROM jt") + + // Create a nested view. + sql("CREATE VIEW view2(id, id1) AS SELECT * FROM view1") + checkAnswer(sql("SELECT * FROM view2 ORDER BY id"), (1 to 9).map(i => Row(i, i))) + + // Alter the referenced view. + sql("ALTER VIEW view1 AS SELECT id AS x, id1 + 1 As y FROM jt") + checkAnswer(sql("SELECT * FROM view2 ORDER BY id"), (1 to 9).map(i => Row(i, i + 1))) + } + } + + test("should not allow ALTER VIEW AS when the view does not exist") { + assertNoSuchTable("ALTER VIEW testView AS SELECT 1, 2") + assertNoSuchTable("ALTER VIEW default.testView AS SELECT 1, 2") + } + + test("ALTER VIEW AS should try to alter temp view first if view name has no database part") { + withView("test_view") { + withTempView("test_view") { + sql("CREATE VIEW test_view AS SELECT 1 AS a, 2 AS b") + sql("CREATE TEMP VIEW test_view AS SELECT 1 AS a, 2 AS b") + + sql("ALTER VIEW test_view AS SELECT 3 AS i, 4 AS j") + + // The temporary view should be updated. + checkAnswer(spark.table("test_view"), Row(3, 4)) + + // The permanent view should stay same. + checkAnswer(spark.table("default.test_view"), Row(1, 2)) + } + } + } + + test("ALTER VIEW AS should alter permanent view if view name has database part") { + withView("test_view") { + withTempView("test_view") { + sql("CREATE VIEW test_view AS SELECT 1 AS a, 2 AS b") + sql("CREATE TEMP VIEW test_view AS SELECT 1 AS a, 2 AS b") + + sql("ALTER VIEW default.test_view AS SELECT 3 AS i, 4 AS j") + + // The temporary view should stay same. + checkAnswer(spark.table("test_view"), Row(1, 2)) + + // The permanent view should be updated. + checkAnswer(spark.table("default.test_view"), Row(3, 4)) + } + } + } + + test("ALTER VIEW AS should keep the previous table properties, comment, create_time, etc.") { + withView("test_view") { + sql( + """ + |CREATE VIEW test_view + |COMMENT 'test' + |TBLPROPERTIES ('key' = 'a') + |AS SELECT 1 AS a, 2 AS b + """.stripMargin) + + val catalog = spark.sessionState.catalog + val viewMeta = catalog.getTableMetadata(TableIdentifier("test_view")) + assert(viewMeta.comment == Some("test")) + assert(viewMeta.properties("key") == "a") + + sql("ALTER VIEW test_view AS SELECT 3 AS i, 4 AS j") + val updatedViewMeta = catalog.getTableMetadata(TableIdentifier("test_view")) + assert(updatedViewMeta.comment == Some("test")) + assert(updatedViewMeta.properties("key") == "a") + assert(updatedViewMeta.createTime == viewMeta.createTime) + // The view should be updated. + checkAnswer(spark.table("test_view"), Row(3, 4)) + } + } + + test("create view for json table") { + // json table is not hive-compatible, make sure the new flag fix it. + withView("testView") { + sql("CREATE VIEW testView AS SELECT id FROM jt") + checkAnswer(sql("SELECT * FROM testView ORDER BY id"), (1 to 9).map(i => Row(i))) + } + } + + test("create view for partitioned parquet table") { + // partitioned parquet table is not hive-compatible, make sure the new flag fix it. + withTable("parTable") { + withView("testView") { + val df = Seq(1 -> "a").toDF("i", "j") + df.write.format("parquet").partitionBy("i").saveAsTable("parTable") + sql("CREATE VIEW testView AS SELECT i, j FROM parTable") + checkAnswer(sql("SELECT * FROM testView"), Row(1, "a")) + } + } + } + + test("create view for joined tables") { + // make sure the new flag can handle some complex cases like join and schema change. + withTable("jt1", "jt2") { + spark.range(1, 10).toDF("id1").write.format("json").saveAsTable("jt1") + spark.range(1, 10).toDF("id2").write.format("json").saveAsTable("jt2") + withView("testView") { + sql("CREATE VIEW testView AS SELECT * FROM jt1 JOIN jt2 ON id1 == id2") + checkAnswer(sql("SELECT * FROM testView ORDER BY id1"), (1 to 9).map(i => Row(i, i))) + + val df = (1 until 10).map(i => i -> i).toDF("id1", "newCol") + df.write.format("json").mode(SaveMode.Overwrite).saveAsTable("jt1") + checkAnswer(sql("SELECT * FROM testView ORDER BY id1"), (1 to 9).map(i => Row(i, i))) + } + } + } + + test("CTE within view") { + withView("cte_view") { + sql("CREATE VIEW cte_view AS WITH w AS (SELECT 1 AS n) SELECT n FROM w") + checkAnswer(sql("SELECT * FROM cte_view"), Row(1)) + } + } + + test("Using view after switching current database") { + withView("v") { + sql("CREATE VIEW v AS SELECT * FROM jt") + withTempDatabase { db => + activateDatabase(db) { + // Should look up table `jt` in database `default`. + checkAnswer(sql("SELECT * FROM default.v"), sql("SELECT * FROM default.jt")) + + // The new `jt` table shouldn't be scanned. + sql("CREATE TABLE jt(key INT, value STRING) USING parquet") + checkAnswer(sql("SELECT * FROM default.v"), sql("SELECT * FROM default.jt")) + } + } + } + } + + test("Using view after adding more columns") { + withTable("add_col") { + spark.range(10).write.saveAsTable("add_col") + withView("v") { + sql("CREATE VIEW v AS SELECT * FROM add_col") + spark.range(10).select('id, 'id as 'a).write.mode("overwrite").saveAsTable("add_col") + checkAnswer(sql("SELECT * FROM v"), spark.range(10).toDF()) + } + } + } + + test("error handling: fail if the referenced table or view is invalid") { + withView("view1", "view2", "view3") { + // Fail if the referenced table is defined in a invalid database. + withTempDatabase { db => + withTable(s"$db.table1") { + activateDatabase(db) { + sql("CREATE TABLE table1(a int, b string) USING parquet") + sql("CREATE VIEW default.view1 AS SELECT * FROM table1") + } + } + } + assertInvalidReference("SELECT * FROM view1") + + // Fail if the referenced table is invalid. + withTable("table2") { + sql("CREATE TABLE table2(a int, b string) USING parquet") + sql("CREATE VIEW view2 AS SELECT * FROM table2") + } + assertInvalidReference("SELECT * FROM view2") + + // Fail if the referenced view is invalid. + withView("testView") { + sql("CREATE VIEW testView AS SELECT * FROM jt") + sql("CREATE VIEW view3 AS SELECT * FROM testView") + } + assertInvalidReference("SELECT * FROM view3") + } + } + + test("correctly resolve a view in a self join") { + withView("testView") { + sql("CREATE VIEW testView AS SELECT * FROM jt") + checkAnswer( + sql("SELECT * FROM testView t1 JOIN testView t2 ON t1.id = t2.id ORDER BY t1.id"), + (1 to 9).map(i => Row(i, i, i, i))) + } + } + + test("correctly handle a view with custom column names") { + withTable("tab1") { + spark.range(1, 10).selectExpr("id", "id + 1 id1").write.saveAsTable("tab1") + withView("testView", "testView2") { + sql("CREATE VIEW testView(x, y) AS SELECT * FROM tab1") + + // Correctly resolve a view with custom column names. + checkAnswer(sql("SELECT * FROM testView ORDER BY x"), (1 to 9).map(i => Row(i, i + 1))) + + // Throw an AnalysisException if the number of columns don't match up. + val e = intercept[AnalysisException] { + sql("CREATE VIEW testView2(x, y, z) AS SELECT * FROM tab1") + }.getMessage + assert(e.contains("The number of columns produced by the SELECT clause (num: `2`) does " + + "not match the number of column names specified by CREATE VIEW (num: `3`).")) + + // Correctly resolve a view when the referenced table schema changes. + spark.range(1, 10).selectExpr("id", "id + id dummy", "id + 1 id1") + .write.mode(SaveMode.Overwrite).saveAsTable("tab1") + checkAnswer(sql("SELECT * FROM testView ORDER BY x"), (1 to 9).map(i => Row(i, i + 1))) + + // Throw an AnalysisException if the column name is not found. + spark.range(1, 10).selectExpr("id", "id + 1 dummy") + .write.mode(SaveMode.Overwrite).saveAsTable("tab1") + intercept[AnalysisException](sql("SELECT * FROM testView")) + } + } + } + + test("resolve a view when the dataTypes of referenced table columns changed") { + withTable("tab1") { + spark.range(1, 10).selectExpr("id", "id + 1 id1").write.saveAsTable("tab1") + withView("testView") { + sql("CREATE VIEW testView AS SELECT * FROM tab1") + + // Allow casting from IntegerType to LongType + val df = (1 until 10).map(i => (i, i + 1)).toDF("id", "id1") + df.write.format("json").mode(SaveMode.Overwrite).saveAsTable("tab1") + checkAnswer(sql("SELECT * FROM testView ORDER BY id1"), (1 to 9).map(i => Row(i, i + 1))) + + // Casting from DoubleType to LongType might truncate, throw an AnalysisException. + val df2 = (1 until 10).map(i => (i.toDouble, i.toDouble)).toDF("id", "id1") + df2.write.format("json").mode(SaveMode.Overwrite).saveAsTable("tab1") + intercept[AnalysisException](sql("SELECT * FROM testView")) + + // Can't cast from ArrayType to LongType, throw an AnalysisException. + val df3 = (1 until 10).map(i => (i, Seq(i))).toDF("id", "id1") + df3.write.format("json").mode(SaveMode.Overwrite).saveAsTable("tab1") + intercept[AnalysisException](sql("SELECT * FROM testView")) + } + } + } + + test("correctly handle a cyclic view reference") { + withView("view1", "view2", "view3") { + sql("CREATE VIEW view1 AS SELECT * FROM jt") + sql("CREATE VIEW view2 AS SELECT * FROM view1") + sql("CREATE VIEW view3 AS SELECT * FROM view2") + + // Detect cyclic view reference on ALTER VIEW. + val e1 = intercept[AnalysisException] { + sql("ALTER VIEW view1 AS SELECT * FROM view2") + }.getMessage + assert(e1.contains("Recursive view `default`.`view1` detected (cycle: `default`.`view1` " + + "-> `default`.`view2` -> `default`.`view1`)")) + + // Detect the most left cycle when there exists multiple cyclic view references. + val e2 = intercept[AnalysisException] { + sql("ALTER VIEW view1 AS SELECT * FROM view3 JOIN view2") + }.getMessage + assert(e2.contains("Recursive view `default`.`view1` detected (cycle: `default`.`view1` " + + "-> `default`.`view3` -> `default`.`view2` -> `default`.`view1`)")) + + // Detect cyclic view reference on CREATE OR REPLACE VIEW. + val e3 = intercept[AnalysisException] { + sql("CREATE OR REPLACE VIEW view1 AS SELECT * FROM view2") + }.getMessage + assert(e3.contains("Recursive view `default`.`view1` detected (cycle: `default`.`view1` " + + "-> `default`.`view2` -> `default`.`view1`)")) + + // Detect cyclic view reference from subqueries. + val e4 = intercept[AnalysisException] { + sql("ALTER VIEW view1 AS SELECT * FROM jt WHERE EXISTS (SELECT 1 FROM view2)") + }.getMessage + assert(e4.contains("Recursive view `default`.`view1` detected (cycle: `default`.`view1` " + + "-> `default`.`view2` -> `default`.`view1`)")) + } + } + + test("restrict the nested level of a view") { + val viewNames = Array.range(0, 11).map(idx => s"view$idx") + withView(viewNames: _*) { + sql("CREATE VIEW view0 AS SELECT * FROM jt") + Array.range(0, 10).foreach { idx => + sql(s"CREATE VIEW view${idx + 1} AS SELECT * FROM view$idx") + } + + withSQLConf("spark.sql.view.maxNestedViewDepth" -> "10") { + val e = intercept[AnalysisException] { + sql("SELECT * FROM view10") + }.getMessage + assert(e.contains("The depth of view `default`.`view0` exceeds the maximum view " + + "resolution depth (10). Analysis is aborted to avoid errors. Increase the value " + + "of spark.sql.view.maxNestedViewDepth to work aroud this.")) + } + + val e = intercept[IllegalArgumentException] { + withSQLConf("spark.sql.view.maxNestedViewDepth" -> "0") {} + }.getMessage + assert(e.contains("The maximum depth of a view reference in a nested view must be " + + "positive.")) + } + } + + test("permanent view should be case-preserving") { + withView("v") { + sql("CREATE VIEW v AS SELECT 1 as aBc") + assert(spark.table("v").schema.head.name == "aBc") + + sql("CREATE OR REPLACE VIEW v AS SELECT 2 as cBa") + assert(spark.table("v").schema.head.name == "cBa") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala new file mode 100644 index 000000000000..1c6fc3530cbe --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala @@ -0,0 +1,489 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.TestUtils.assertSpilled +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.test.SharedSQLContext + +case class WindowData(month: Int, area: String, product: Int) + + +/** + * Test suite for SQL window functions. + */ +class SQLWindowFunctionSuite extends QueryTest with SharedSQLContext { + + import testImplicits._ + + test("window function: udaf with aggregate expression") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") + + checkAnswer( + sql( + """ + |select area, sum(product), sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 11), + ("a", 6, 11), + ("b", 7, 15), + ("b", 8, 15), + ("c", 9, 19), + ("c", 10, 19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product) - 1, sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 4, 11), + ("a", 5, 11), + ("b", 6, 15), + ("b", 7, 15), + ("c", 8, 19), + ("c", 9, 19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product), sum(product) / sum(sum(product)) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 5d/11), + ("a", 6, 6d/11), + ("b", 7, 7d/15), + ("b", 8, 8d/15), + ("c", 10, 10d/19), + ("c", 9, 9d/19) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, sum(product), sum(product) / sum(sum(product) - 1) over (partition by area) + |from windowData group by month, area + """.stripMargin), + Seq( + ("a", 5, 5d/9), + ("a", 6, 6d/9), + ("b", 7, 7d/13), + ("b", 8, 8d/13), + ("c", 10, 10d/17), + ("c", 9, 9d/17) + ).map(i => Row(i._1, i._2, i._3))) + } + + test("window function: refer column in inner select block") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") + + checkAnswer( + sql( + """ + |select area, rank() over (partition by area order by tmp.month) + tmp.tmp1 as c1 + |from (select month, area, product, 1 as tmp1 from windowData) tmp + """.stripMargin), + Seq( + ("a", 2), + ("a", 3), + ("b", 2), + ("b", 3), + ("c", 2), + ("c", 3) + ).map(i => Row(i._1, i._2))) + } + + test("window function: partition and order expressions") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") + + checkAnswer( + sql( + """ + |select month, area, product, sum(product + 1) over (partition by 1 order by 2) + |from windowData + """.stripMargin), + Seq( + (1, "a", 5, 51), + (2, "a", 6, 51), + (3, "b", 7, 51), + (4, "b", 8, 51), + (5, "c", 9, 51), + (6, "c", 10, 51) + ).map(i => Row(i._1, i._2, i._3, i._4))) + + checkAnswer( + sql( + """ + |select month, area, product, sum(product) + |over (partition by month % 2 order by 10 - product) + |from windowData + """.stripMargin), + Seq( + (1, "a", 5, 21), + (2, "a", 6, 24), + (3, "b", 7, 16), + (4, "b", 8, 18), + (5, "c", 9, 9), + (6, "c", 10, 10) + ).map(i => Row(i._1, i._2, i._3, i._4))) + } + + test("window function: distinct should not be silently ignored") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") + + val e = intercept[AnalysisException] { + sql( + """ + |select month, area, product, sum(distinct product + 1) over (partition by 1 order by 2) + |from windowData + """.stripMargin) + } + assert(e.getMessage.contains("Distinct window functions are not supported")) + } + + test("window function: expressions in arguments of a window functions") { + val data = Seq( + WindowData(1, "a", 5), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 10) + ) + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") + + checkAnswer( + sql( + """ + |select month, area, month % 2, + |lag(product, 1 + 1, product) over (partition by month % 2 order by area) + |from windowData + """.stripMargin), + Seq( + (1, "a", 1, 5), + (2, "a", 0, 6), + (3, "b", 1, 7), + (4, "b", 0, 8), + (5, "c", 1, 5), + (6, "c", 0, 6) + ).map(i => Row(i._1, i._2, i._3, i._4))) + } + + + test("window function: Sorting columns are not in Project") { + val data = Seq( + WindowData(1, "d", 10), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 11) + ) + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") + + checkAnswer( + sql("select month, product, sum(product + 1) over() from windowData order by area"), + Seq( + (2, 6, 57), + (3, 7, 57), + (4, 8, 57), + (5, 9, 57), + (6, 11, 57), + (1, 10, 57) + ).map(i => Row(i._1, i._2, i._3))) + + checkAnswer( + sql( + """ + |select area, rank() over (partition by area order by tmp.month) + tmp.tmp1 as c1 + |from (select month, area, product as p, 1 as tmp1 from windowData) tmp order by p + """.stripMargin), + Seq( + ("a", 2), + ("b", 2), + ("b", 3), + ("c", 2), + ("d", 2), + ("c", 3) + ).map(i => Row(i._1, i._2))) + + checkAnswer( + sql( + """ + |select area, rank() over (partition by area order by month) as c1 + |from windowData group by product, area, month order by product, area + """.stripMargin), + Seq( + ("a", 1), + ("b", 1), + ("b", 2), + ("c", 1), + ("d", 1), + ("c", 2) + ).map(i => Row(i._1, i._2))) + + checkAnswer( + sql( + """ + |select area, sum(product) / sum(sum(product)) over (partition by area) as c1 + |from windowData group by area, month order by month, c1 + """.stripMargin), + Seq( + ("d", 1.0), + ("a", 1.0), + ("b", 0.4666666666666667), + ("b", 0.5333333333333333), + ("c", 0.45), + ("c", 0.55) + ).map(i => Row(i._1, i._2))) + } + + // todo: fix this test case by reimplementing the function ResolveAggregateFunctions + ignore("window function: Pushing aggregate Expressions in Sort to Aggregate") { + val data = Seq( + WindowData(1, "d", 10), + WindowData(2, "a", 6), + WindowData(3, "b", 7), + WindowData(4, "b", 8), + WindowData(5, "c", 9), + WindowData(6, "c", 11) + ) + sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData") + + checkAnswer( + sql( + """ + |select area, sum(product) over () as c from windowData + |where product > 3 group by area, product + |having avg(month) > 0 order by avg(month), product + """.stripMargin), + Seq( + ("a", 51), + ("b", 51), + ("b", 51), + ("c", 51), + ("c", 51), + ("d", 51) + ).map(i => Row(i._1, i._2))) + } + + test("window function: multiple window expressions in a single expression") { + val nums = sparkContext.parallelize(1 to 10).map(x => (x, x % 2)).toDF("x", "y") + nums.createOrReplaceTempView("nums") + + val expected = + Row(1, 1, 1, 55, 1, 57) :: + Row(0, 2, 3, 55, 2, 60) :: + Row(1, 3, 6, 55, 4, 65) :: + Row(0, 4, 10, 55, 6, 71) :: + Row(1, 5, 15, 55, 9, 79) :: + Row(0, 6, 21, 55, 12, 88) :: + Row(1, 7, 28, 55, 16, 99) :: + Row(0, 8, 36, 55, 20, 111) :: + Row(1, 9, 45, 55, 25, 125) :: + Row(0, 10, 55, 55, 30, 140) :: Nil + + val actual = sql( + """ + |SELECT + | y, + | x, + | sum(x) OVER w1 AS running_sum, + | sum(x) OVER w2 AS total_sum, + | sum(x) OVER w3 AS running_sum_per_y, + | ((sum(x) OVER w1) + (sum(x) OVER w2) + (sum(x) OVER w3)) as combined2 + |FROM nums + |WINDOW w1 AS (ORDER BY x ROWS BETWEEN UnBOUNDED PRECEDiNG AND CuRRENT RoW), + | w2 AS (ORDER BY x ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOuNDED FoLLOWING), + | w3 AS (PARTITION BY y ORDER BY x ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) + """.stripMargin) + + checkAnswer(actual, expected) + + spark.catalog.dropTempView("nums") + } + + test("window function: mutiple window expressions specified by range in a single expression") { + val nums = sparkContext.parallelize(1 to 10).map(x => (x, x % 2)).toDF("x", "y") + nums.createOrReplaceTempView("nums") + withTempView("nums") { + val expected = + Row(1, 1, 1, 4, null, 8, 25) :: + Row(1, 3, 4, 9, 1, 12, 24) :: + Row(1, 5, 9, 15, 4, 16, 21) :: + Row(1, 7, 16, 21, 8, 9, 16) :: + Row(1, 9, 25, 16, 12, null, 9) :: + Row(0, 2, 2, 6, null, 10, 30) :: + Row(0, 4, 6, 12, 2, 14, 28) :: + Row(0, 6, 12, 18, 6, 18, 24) :: + Row(0, 8, 20, 24, 10, 10, 18) :: + Row(0, 10, 30, 18, 14, null, 10) :: + Nil + + val actual = sql( + """ + |SELECT + | y, + | x, + | sum(x) over w1 as history_sum, + | sum(x) over w2 as period_sum1, + | sum(x) over w3 as period_sum2, + | sum(x) over w4 as period_sum3, + | sum(x) over w5 as future_sum + |FROM nums + |WINDOW + | w1 AS (PARTITION BY y ORDER BY x RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), + | w2 AS (PARTITION BY y ORDER BY x RANGE BETWEEN 2 PRECEDING AND 2 FOLLOWING), + | w3 AS (PARTITION BY y ORDER BY x RANGE BETWEEN 4 PRECEDING AND 2 PRECEDING ), + | w4 AS (PARTITION BY y ORDER BY x RANGE BETWEEN 2 FOLLOWING AND 4 FOLLOWING), + | w5 AS (PARTITION BY y ORDER BY x RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + """.stripMargin + ) + checkAnswer(actual, expected) + } + } + + test("SPARK-7595: Window will cause resolve failed with self join") { + checkAnswer(sql( + """ + |with + | v0 as (select 0 as key, 1 as value), + | v1 as (select key, count(value) over (partition by key) cnt_val from v0), + | v2 as (select v1.key, v1_lag.cnt_val from v1 cross join v1 v1_lag + | where v1.key = v1_lag.key) + | select key, cnt_val from v2 order by key limit 1 + """.stripMargin), Row(0, 1)) + } + + test("SPARK-16633: lead/lag should return the default value if the offset row does not exist") { + checkAnswer(sql( + """ + |SELECT + | lag(123, 100, 321) OVER (ORDER BY id) as lag, + | lead(123, 100, 321) OVER (ORDER BY id) as lead + |FROM (SELECT 1 as id) tmp + """.stripMargin), + Row(321, 321)) + + checkAnswer(sql( + """ + |SELECT + | lag(123, 100, a) OVER (ORDER BY id) as lag, + | lead(123, 100, a) OVER (ORDER BY id) as lead + |FROM (SELECT 1 as id, 2 as a) tmp + """.stripMargin), + Row(2, 2)) + } + + test("lead/lag should respect null values") { + checkAnswer(sql( + """ + |SELECT + | b, + | lag(a, 1, 321) OVER (ORDER BY b) as lag, + | lead(a, 1, 321) OVER (ORDER BY b) as lead + |FROM (SELECT cast(null as int) as a, 1 as b + | UNION ALL + | select cast(null as int) as id, 2 as b) tmp + """.stripMargin), + Row(1, 321, null) :: Row(2, null, 321) :: Nil) + + checkAnswer(sql( + """ + |SELECT + | b, + | lag(a, 1, c) OVER (ORDER BY b) as lag, + | lead(a, 1, c) OVER (ORDER BY b) as lead + |FROM (SELECT cast(null as int) as a, 1 as b, 3 as c + | UNION ALL + | select cast(null as int) as id, 2 as b, 4 as c) tmp + """.stripMargin), + Row(1, 3, null) :: Row(2, null, 4) :: Nil) + } + + test("test with low buffer spill threshold") { + val nums = sparkContext.parallelize(1 to 10).map(x => (x, x % 2)).toDF("x", "y") + nums.createOrReplaceTempView("nums") + + val expected = + Row(1, 1, 1) :: + Row(0, 2, 3) :: + Row(1, 3, 6) :: + Row(0, 4, 10) :: + Row(1, 5, 15) :: + Row(0, 6, 21) :: + Row(1, 7, 28) :: + Row(0, 8, 36) :: + Row(1, 9, 45) :: + Row(0, 10, 55) :: Nil + + val actual = sql( + """ + |SELECT y, x, sum(x) OVER w1 AS running_sum + |FROM nums + |WINDOW w1 AS (ORDER BY x ROWS BETWEEN UNBOUNDED PRECEDiNG AND CURRENT RoW) + """.stripMargin) + + withSQLConf("spark.sql.windowExec.buffer.in.memory.threshold" -> "1", + "spark.sql.windowExec.buffer.spill.threshold" -> "2") { + assertSpilled(sparkContext, "test with low buffer spill threshold") { + checkAnswer(actual, expected) + } + } + + spark.catalog.dropTempView("nums") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala new file mode 100644 index 000000000000..aaf51b5b9011 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.{DataFrame, QueryTest} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SharedSQLContext + +/** + * Tests for the sameResult function for [[SparkPlan]]s. + */ +class SameResultSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("FileSourceScanExec: different orders of data filters and partition filters") { + withTempPath { path => + val tmpDir = path.getCanonicalPath + spark.range(10) + .selectExpr("id as a", "id + 1 as b", "id + 2 as c", "id + 3 as d") + .write + .partitionBy("a", "b") + .parquet(tmpDir) + val df = spark.read.parquet(tmpDir) + // partition filters: a > 1 AND b < 9 + // data filters: c > 1 AND d < 9 + val plan1 = getFileSourceScanExec(df.where("a > 1 AND b < 9 AND c > 1 AND d < 9")) + val plan2 = getFileSourceScanExec(df.where("b < 9 AND a > 1 AND d < 9 AND c > 1")) + assert(plan1.sameResult(plan2)) + } + } + + private def getFileSourceScanExec(df: DataFrame): FileSourceScanExec = { + df.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get + .asInstanceOf[FileSourceScanExec] + } + + test("SPARK-20725: partial aggregate should behave correctly for sameResult") { + val df1 = spark.range(10).agg(sum($"id")) + val df2 = spark.range(10).agg(sum($"id")) + assert(df1.queryExecution.executedPlan.sameResult(df2.queryExecution.executedPlan)) + + val df3 = spark.range(10).agg(sumDistinct($"id")) + val df4 = spark.range(10).agg(sumDistinct($"id")) + assert(df3.queryExecution.executedPlan.sameResult(df4.queryExecution.executedPlan)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala index 847c188a3033..a7bbe34f4eed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala @@ -17,15 +17,22 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.Row +import scala.util.Random + +import org.apache.spark.AccumulatorSuite +import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ +/** + * Test sorting. Many of the test cases generate random data and compares the sorted result with one + * sorted by a reference implementation ([[ReferenceSort]]). + */ class SortSuite extends SparkPlanTest with SharedSQLContext { - import testImplicits.localSeqToDataFrameHolder + import testImplicits.newProductEncoder + import testImplicits.localSeqToDatasetHolder - // This test was originally added as an example of how to use [[SparkPlanTest]]; - // it's not designed to be a comprehensive test of ExternalSort. test("basic sorting using ExternalSort") { val input = Seq( @@ -36,14 +43,80 @@ class SortSuite extends SparkPlanTest with SharedSQLContext { checkAnswer( input.toDF("a", "b", "c"), - Sort('a.asc :: 'b.asc :: Nil, global = true, _: SparkPlan), + (child: SparkPlan) => SortExec('a.asc :: 'b.asc :: Nil, global = true, child = child), input.sortBy(t => (t._1, t._2)).map(Row.fromTuple), sortAnswers = false) checkAnswer( input.toDF("a", "b", "c"), - Sort('b.asc :: 'a.asc :: Nil, global = true, _: SparkPlan), + (child: SparkPlan) => SortExec('b.asc :: 'a.asc :: Nil, global = true, child = child), input.sortBy(t => (t._2, t._1)).map(Row.fromTuple), sortAnswers = false) } + + test("sorting all nulls") { + checkThatPlansAgree( + (1 to 100).map(v => Tuple1(v)).toDF().selectExpr("NULL as a"), + (child: SparkPlan) => + GlobalLimitExec(10, SortExec('a.asc :: Nil, global = true, child = child)), + (child: SparkPlan) => + GlobalLimitExec(10, ReferenceSort('a.asc :: Nil, global = true, child)), + sortAnswers = false + ) + } + + test("sort followed by limit") { + checkThatPlansAgree( + (1 to 100).map(v => Tuple1(v)).toDF("a"), + (child: SparkPlan) => + GlobalLimitExec(10, SortExec('a.asc :: Nil, global = true, child = child)), + (child: SparkPlan) => + GlobalLimitExec(10, ReferenceSort('a.asc :: Nil, global = true, child)), + sortAnswers = false + ) + } + + test("sorting does not crash for large inputs") { + val sortOrder = 'a.asc :: Nil + val stringLength = 1024 * 1024 * 2 + checkThatPlansAgree( + Seq(Tuple1("a" * stringLength), Tuple1("b" * stringLength)).toDF("a").repartition(1), + SortExec(sortOrder, global = true, _: SparkPlan, testSpillFrequency = 1), + ReferenceSort(sortOrder, global = true, _: SparkPlan), + sortAnswers = false + ) + } + + test("sorting updates peak execution memory") { + AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "unsafe external sort") { + checkThatPlansAgree( + (1 to 100).map(v => Tuple1(v)).toDF("a"), + (child: SparkPlan) => SortExec('a.asc :: Nil, global = true, child = child), + (child: SparkPlan) => ReferenceSort('a.asc :: Nil, global = true, child), + sortAnswers = false) + } + } + + // Test sorting on different data types + for ( + dataType <- DataTypeTestUtils.atomicTypes ++ Set(NullType); + nullable <- Seq(true, false); + sortOrder <- + Seq('a.asc :: Nil, 'a.asc_nullsLast :: Nil, 'a.desc :: Nil, 'a.desc_nullsFirst :: Nil); + randomDataGenerator <- RandomDataGenerator.forType(dataType, nullable) + ) { + test(s"sorting on $dataType with nullable=$nullable, sortOrder=$sortOrder") { + val inputData = Seq.fill(1000)(randomDataGenerator()) + val inputDf = spark.createDataFrame( + sparkContext.parallelize(Random.shuffle(inputData).map(v => Row(v))), + StructType(StructField("a", dataType, nullable = true) :: Nil) + ) + checkThatPlansAgree( + inputDf, + p => SortExec(sortOrder, global = true, p: SparkPlan, testSpillFrequency = 23), + ReferenceSort(sortOrder, global = true, _: SparkPlan), + sortAnswers = false + ) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala index 8549a6a0f664..b29e822add8b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala @@ -18,11 +18,10 @@ package org.apache.spark.sql.execution import scala.language.implicitConversions -import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.{DataFrame, DataFrameHolder, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.test.SQLTestUtils @@ -31,7 +30,7 @@ import org.apache.spark.sql.test.SQLTestUtils * class's test helper methods can be used, see [[SortSuite]]. */ private[sql] abstract class SparkPlanTest extends SparkFunSuite { - protected def sqlContext: SQLContext + protected def spark: SparkSession /** * Runs the plan and makes sure the answer matches the expected result. @@ -91,9 +90,10 @@ private[sql] abstract class SparkPlanTest extends SparkFunSuite { planFunction: Seq[SparkPlan] => SparkPlan, expectedAnswer: Seq[Row], sortAnswers: Boolean = true): Unit = { - SparkPlanTest.checkAnswer(input, planFunction, expectedAnswer, sortAnswers, sqlContext) match { - case Some(errorMessage) => fail(errorMessage) - case None => + SparkPlanTest + .checkAnswer(input, planFunction, expectedAnswer, sortAnswers, spark.sqlContext) match { + case Some(errorMessage) => fail(errorMessage) + case None => } } @@ -115,7 +115,7 @@ private[sql] abstract class SparkPlanTest extends SparkFunSuite { expectedPlanFunction: SparkPlan => SparkPlan, sortAnswers: Boolean = true): Unit = { SparkPlanTest.checkAnswer( - input, planFunction, expectedPlanFunction, sortAnswers, sqlContext) match { + input, planFunction, expectedPlanFunction, sortAnswers, spark.sqlContext) match { case Some(errorMessage) => fail(errorMessage) case None => } @@ -142,13 +142,13 @@ object SparkPlanTest { planFunction: SparkPlan => SparkPlan, expectedPlanFunction: SparkPlan => SparkPlan, sortAnswers: Boolean, - sqlContext: SQLContext): Option[String] = { + spark: SQLContext): Option[String] = { val outputPlan = planFunction(input.queryExecution.sparkPlan) val expectedOutputPlan = expectedPlanFunction(input.queryExecution.sparkPlan) val expectedAnswer: Seq[Row] = try { - executePlan(expectedOutputPlan, sqlContext) + executePlan(expectedOutputPlan, spark) } catch { case NonFatal(e) => val errorMessage = @@ -163,7 +163,7 @@ object SparkPlanTest { } val actualAnswer: Seq[Row] = try { - executePlan(outputPlan, sqlContext) + executePlan(outputPlan, spark) } catch { case NonFatal(e) => val errorMessage = @@ -203,12 +203,12 @@ object SparkPlanTest { planFunction: Seq[SparkPlan] => SparkPlan, expectedAnswer: Seq[Row], sortAnswers: Boolean, - sqlContext: SQLContext): Option[String] = { + spark: SQLContext): Option[String] = { val outputPlan = planFunction(input.map(_.queryExecution.sparkPlan)) val sparkAnswer: Seq[Row] = try { - executePlan(outputPlan, sqlContext) + executePlan(outputPlan, spark) } catch { case NonFatal(e) => val errorMessage = @@ -231,11 +231,14 @@ object SparkPlanTest { } } - private def executePlan(outputPlan: SparkPlan, sqlContext: SQLContext): Seq[Row] = { - // A very simple resolver to make writing tests easier. In contrast to the real resolver - // this is always case sensitive and does not try to handle scoping or complex type resolution. - val resolvedPlan = sqlContext.prepareForExecution.execute( - outputPlan transform { + /** + * Runs the plan + * @param outputPlan SparkPlan to be executed + * @param spark SqlContext used for execution of the plan + */ + def executePlan(outputPlan: SparkPlan, spark: SQLContext): Seq[Row] = { + val execution = new QueryExecution(spark.sparkSession, null) { + override lazy val sparkPlan: SparkPlan = outputPlan transform { case plan: SparkPlan => val inputMap = plan.children.flatMap(_.output).map(a => (a.name, a)).toMap plan transformExpressions { @@ -244,8 +247,8 @@ object SparkPlanTest { sys.error(s"Invalid Test: Cannot resolve $u given input $inputMap")) } } - ) - resolvedPlan.executeCollectPublic().toSeq + } + execution.executedPlan.executeCollectPublic().toSeq } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala new file mode 100644 index 000000000000..5828f9783da4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlannerSuite.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.Strategy +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LocalRelation, LogicalPlan, ReturnAnswer, Union} +import org.apache.spark.sql.test.SharedSQLContext + +class SparkPlannerSuite extends SharedSQLContext { + import testImplicits._ + + test("Ensure to go down only the first branch, not any other possible branches") { + + case object NeverPlanned extends LeafNode { + override def output: Seq[Attribute] = Nil + } + + var planned = 0 + object TestStrategy extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case ReturnAnswer(child) => + planned += 1 + planLater(child) :: planLater(NeverPlanned) :: Nil + case Union(children) => + planned += 1 + UnionExec(children.map(planLater)) :: planLater(NeverPlanned) :: Nil + case LocalRelation(output, data, _) => + planned += 1 + LocalTableScanExec(output, data) :: planLater(NeverPlanned) :: Nil + case NeverPlanned => + fail("QueryPlanner should not go down to this branch.") + case _ => Nil + } + } + + try { + spark.experimental.extraStrategies = TestStrategy :: Nil + + val ds = Seq("a", "b", "c").toDS().union(Seq("d", "e", "f").toDS()) + + assert(ds.collect().toSeq === Seq("a", "b", "c", "d", "e", "f")) + assert(planned === 4) + } finally { + spark.experimental.extraStrategies = Nil + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala new file mode 100644 index 000000000000..107a2f710979 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -0,0 +1,369 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.SaveMode +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAlias, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.expressions.{Ascending, Concat, SortOrder} +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RepartitionByExpression, Sort} +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.datasources.{CreateTable, RefreshResource} +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} + +/** + * Parser test cases for rules defined in [[SparkSqlParser]]. + * + * See [[org.apache.spark.sql.catalyst.parser.PlanParserSuite]] for rules + * defined in the Catalyst module. + */ +class SparkSqlParserSuite extends AnalysisTest { + + val newConf = new SQLConf + private lazy val parser = new SparkSqlParser(newConf) + + /** + * Normalizes plans: + * - CreateTable the createTime in tableDesc will replaced by -1L. + */ + override def normalizePlan(plan: LogicalPlan): LogicalPlan = { + plan match { + case CreateTable(tableDesc, mode, query) => + val newTableDesc = tableDesc.copy(createTime = -1L) + CreateTable(newTableDesc, mode, query) + case _ => plan // Don't transform + } + } + + private def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = { + val normalized1 = normalizePlan(parser.parsePlan(sqlCommand)) + val normalized2 = normalizePlan(plan) + comparePlans(normalized1, normalized2) + } + + private def intercept(sqlCommand: String, messages: String*): Unit = { + val e = intercept[ParseException](parser.parsePlan(sqlCommand)) + messages.foreach { message => + assert(e.message.contains(message)) + } + } + + test("refresh resource") { + assertEqual("REFRESH prefix_path", RefreshResource("prefix_path")) + assertEqual("REFRESH /", RefreshResource("/")) + assertEqual("REFRESH /path///a", RefreshResource("/path///a")) + assertEqual("REFRESH pat1h/112/_1a", RefreshResource("pat1h/112/_1a")) + assertEqual("REFRESH pat1h/112/_1a/a-1", RefreshResource("pat1h/112/_1a/a-1")) + assertEqual("REFRESH path-with-dash", RefreshResource("path-with-dash")) + assertEqual("REFRESH \'path with space\'", RefreshResource("path with space")) + assertEqual("REFRESH \"path with space 2\"", RefreshResource("path with space 2")) + intercept("REFRESH a b", "REFRESH statements cannot contain") + intercept("REFRESH a\tb", "REFRESH statements cannot contain") + intercept("REFRESH a\nb", "REFRESH statements cannot contain") + intercept("REFRESH a\rb", "REFRESH statements cannot contain") + intercept("REFRESH a\r\nb", "REFRESH statements cannot contain") + intercept("REFRESH @ $a$", "REFRESH statements cannot contain") + intercept("REFRESH ", "Resource paths cannot be empty in REFRESH statements") + intercept("REFRESH", "Resource paths cannot be empty in REFRESH statements") + } + + test("show functions") { + assertEqual("show functions", ShowFunctionsCommand(None, None, true, true)) + assertEqual("show all functions", ShowFunctionsCommand(None, None, true, true)) + assertEqual("show user functions", ShowFunctionsCommand(None, None, true, false)) + assertEqual("show system functions", ShowFunctionsCommand(None, None, false, true)) + intercept("show special functions", "SHOW special FUNCTIONS") + assertEqual("show functions foo", + ShowFunctionsCommand(None, Some("foo"), true, true)) + assertEqual("show functions foo.bar", + ShowFunctionsCommand(Some("foo"), Some("bar"), true, true)) + assertEqual("show functions 'foo\\\\.*'", + ShowFunctionsCommand(None, Some("foo\\.*"), true, true)) + intercept("show functions foo.bar.baz", "Unsupported function name") + } + + test("describe function") { + assertEqual("describe function bar", + DescribeFunctionCommand(FunctionIdentifier("bar", database = None), isExtended = false)) + assertEqual("describe function extended bar", + DescribeFunctionCommand(FunctionIdentifier("bar", database = None), isExtended = true)) + assertEqual("describe function foo.bar", + DescribeFunctionCommand( + FunctionIdentifier("bar", database = Some("foo")), isExtended = false)) + assertEqual("describe function extended f.bar", + DescribeFunctionCommand(FunctionIdentifier("bar", database = Some("f")), isExtended = true)) + } + + private def createTableUsing( + table: String, + database: Option[String] = None, + tableType: CatalogTableType = CatalogTableType.MANAGED, + storage: CatalogStorageFormat = CatalogStorageFormat.empty, + schema: StructType = new StructType, + provider: Option[String] = Some("parquet"), + partitionColumnNames: Seq[String] = Seq.empty, + bucketSpec: Option[BucketSpec] = None, + mode: SaveMode = SaveMode.ErrorIfExists, + query: Option[LogicalPlan] = None): CreateTable = { + CreateTable( + CatalogTable( + identifier = TableIdentifier(table, database), + tableType = tableType, + storage = storage, + schema = schema, + provider = provider, + partitionColumnNames = partitionColumnNames, + bucketSpec = bucketSpec + ), mode, query + ) + } + + private def createTable( + table: String, + database: Option[String] = None, + tableType: CatalogTableType = CatalogTableType.MANAGED, + storage: CatalogStorageFormat = CatalogStorageFormat.empty.copy( + inputFormat = HiveSerDe.sourceToSerDe("textfile").get.inputFormat, + outputFormat = HiveSerDe.sourceToSerDe("textfile").get.outputFormat, + serde = Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")), + schema: StructType = new StructType, + provider: Option[String] = Some("hive"), + partitionColumnNames: Seq[String] = Seq.empty, + comment: Option[String] = None, + mode: SaveMode = SaveMode.ErrorIfExists, + query: Option[LogicalPlan] = None): CreateTable = { + CreateTable( + CatalogTable( + identifier = TableIdentifier(table, database), + tableType = tableType, + storage = storage, + schema = schema, + provider = provider, + partitionColumnNames = partitionColumnNames, + comment = comment + ), mode, query + ) + } + + test("create table - schema") { + assertEqual("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING)", + createTable( + table = "my_tab", + schema = (new StructType) + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType) + ) + ) + assertEqual("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) " + + "PARTITIONED BY (c INT, d STRING COMMENT 'test2')", + createTable( + table = "my_tab", + schema = (new StructType) + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType) + .add("c", IntegerType) + .add("d", StringType, nullable = true, "test2"), + partitionColumnNames = Seq("c", "d") + ) + ) + assertEqual("CREATE TABLE my_tab(id BIGINT, nested STRUCT)", + createTable( + table = "my_tab", + schema = (new StructType) + .add("id", LongType) + .add("nested", (new StructType) + .add("col1", StringType) + .add("col2", IntegerType) + ) + ) + ) + // Partitioned by a StructType should be accepted by `SparkSqlParser` but will fail an analyze + // rule in `AnalyzeCreateTable`. + assertEqual("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) " + + "PARTITIONED BY (nested STRUCT)", + createTable( + table = "my_tab", + schema = (new StructType) + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType) + .add("nested", (new StructType) + .add("col1", StringType) + .add("col2", IntegerType) + ), + partitionColumnNames = Seq("nested") + ) + ) + intercept("CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING)", + "no viable alternative at input") + } + + test("create table using - schema") { + assertEqual("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet", + createTableUsing( + table = "my_tab", + schema = (new StructType) + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType) + ) + ) + intercept("CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING) USING parquet", + "no viable alternative at input") + } + + test("create view as insert into table") { + // Single insert query + intercept("CREATE VIEW testView AS INSERT INTO jt VALUES(1, 1)", + "Operation not allowed: CREATE VIEW ... AS INSERT INTO") + + // Multi insert query + intercept("CREATE VIEW testView AS FROM jt INSERT INTO tbl1 SELECT * WHERE jt.id < 5 " + + "INSERT INTO tbl2 SELECT * WHERE jt.id > 4", + "Operation not allowed: CREATE VIEW ... AS FROM ... [INSERT INTO ...]+") + } + + test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") { + assertEqual("describe table t", + DescribeTableCommand( + TableIdentifier("t"), Map.empty, isExtended = false)) + assertEqual("describe table extended t", + DescribeTableCommand( + TableIdentifier("t"), Map.empty, isExtended = true)) + assertEqual("describe table formatted t", + DescribeTableCommand( + TableIdentifier("t"), Map.empty, isExtended = true)) + } + + test("describe table column") { + assertEqual("DESCRIBE t col", + DescribeColumnCommand( + TableIdentifier("t"), Seq("col"), isExtended = false)) + assertEqual("DESCRIBE t `abc.xyz`", + DescribeColumnCommand( + TableIdentifier("t"), Seq("abc.xyz"), isExtended = false)) + assertEqual("DESCRIBE t abc.xyz", + DescribeColumnCommand( + TableIdentifier("t"), Seq("abc", "xyz"), isExtended = false)) + assertEqual("DESCRIBE t `a.b`.`x.y`", + DescribeColumnCommand( + TableIdentifier("t"), Seq("a.b", "x.y"), isExtended = false)) + + assertEqual("DESCRIBE TABLE t col", + DescribeColumnCommand( + TableIdentifier("t"), Seq("col"), isExtended = false)) + assertEqual("DESCRIBE TABLE EXTENDED t col", + DescribeColumnCommand( + TableIdentifier("t"), Seq("col"), isExtended = true)) + assertEqual("DESCRIBE TABLE FORMATTED t col", + DescribeColumnCommand( + TableIdentifier("t"), Seq("col"), isExtended = true)) + + intercept("DESCRIBE TABLE t PARTITION (ds='1970-01-01') col", + "DESC TABLE COLUMN for a specific partition is not supported") + } + + test("analyze table statistics") { + assertEqual("analyze table t compute statistics", + AnalyzeTableCommand(TableIdentifier("t"), noscan = false)) + assertEqual("analyze table t compute statistics noscan", + AnalyzeTableCommand(TableIdentifier("t"), noscan = true)) + assertEqual("analyze table t partition (a) compute statistics nOscAn", + AnalyzePartitionCommand(TableIdentifier("t"), Map("a" -> None), noscan = true)) + + // Partitions specified + assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS", + AnalyzePartitionCommand(TableIdentifier("t"), noscan = false, + partitionSpec = Map("ds" -> Some("2008-04-09"), "hr" -> Some("11")))) + assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS noscan", + AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, + partitionSpec = Map("ds" -> Some("2008-04-09"), "hr" -> Some("11")))) + assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09') COMPUTE STATISTICS noscan", + AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, + partitionSpec = Map("ds" -> Some("2008-04-09")))) + assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09', hr) COMPUTE STATISTICS", + AnalyzePartitionCommand(TableIdentifier("t"), noscan = false, + partitionSpec = Map("ds" -> Some("2008-04-09"), "hr" -> None))) + assertEqual("ANALYZE TABLE t PARTITION(ds='2008-04-09', hr) COMPUTE STATISTICS noscan", + AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, + partitionSpec = Map("ds" -> Some("2008-04-09"), "hr" -> None))) + assertEqual("ANALYZE TABLE t PARTITION(ds, hr=11) COMPUTE STATISTICS noscan", + AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, + partitionSpec = Map("ds" -> None, "hr" -> Some("11")))) + assertEqual("ANALYZE TABLE t PARTITION(ds, hr) COMPUTE STATISTICS", + AnalyzePartitionCommand(TableIdentifier("t"), noscan = false, + partitionSpec = Map("ds" -> None, "hr" -> None))) + assertEqual("ANALYZE TABLE t PARTITION(ds, hr) COMPUTE STATISTICS noscan", + AnalyzePartitionCommand(TableIdentifier("t"), noscan = true, + partitionSpec = Map("ds" -> None, "hr" -> None))) + + intercept("analyze table t compute statistics xxxx", + "Expected `NOSCAN` instead of `xxxx`") + intercept("analyze table t partition (a) compute statistics xxxx", + "Expected `NOSCAN` instead of `xxxx`") + } + + test("analyze table column statistics") { + intercept("ANALYZE TABLE t COMPUTE STATISTICS FOR COLUMNS", "") + + assertEqual("ANALYZE TABLE t COMPUTE STATISTICS FOR COLUMNS key, value", + AnalyzeColumnCommand(TableIdentifier("t"), Seq("key", "value"))) + + // Partition specified - should be ignored + assertEqual("ANALYZE TABLE t PARTITION(ds='2017-06-10') " + + "COMPUTE STATISTICS FOR COLUMNS key, value", + AnalyzeColumnCommand(TableIdentifier("t"), Seq("key", "value"))) + } + + test("query organization") { + // Test all valid combinations of order by/sort by/distribute by/cluster by/limit/windows + val baseSql = "select * from t" + val basePlan = + Project(Seq(UnresolvedStar(None)), UnresolvedRelation(TableIdentifier("t"))) + + assertEqual(s"$baseSql distribute by a, b", + RepartitionByExpression(UnresolvedAttribute("a") :: UnresolvedAttribute("b") :: Nil, + basePlan, + numPartitions = newConf.numShufflePartitions)) + assertEqual(s"$baseSql distribute by a sort by b", + Sort(SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, + global = false, + RepartitionByExpression(UnresolvedAttribute("a") :: Nil, + basePlan, + numPartitions = newConf.numShufflePartitions))) + assertEqual(s"$baseSql cluster by a, b", + Sort(SortOrder(UnresolvedAttribute("a"), Ascending) :: + SortOrder(UnresolvedAttribute("b"), Ascending) :: Nil, + global = false, + RepartitionByExpression(UnresolvedAttribute("a") :: UnresolvedAttribute("b") :: Nil, + basePlan, + numPartitions = newConf.numShufflePartitions))) + } + + test("pipeline concatenation") { + val concat = Concat( + Concat(UnresolvedAttribute("a") :: UnresolvedAttribute("b") :: Nil) :: + UnresolvedAttribute("c") :: + Nil + ) + assertEqual( + "SELECT a || b || c FROM t", + Project(UnresolvedAlias(concat) :: Nil, UnresolvedRelation(TableIdentifier("t")))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala new file mode 100644 index 000000000000..7e317a4d8026 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import scala.util.Random + +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ + + +class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext { + + private var rand: Random = _ + private var seed: Long = 0 + + protected override def beforeAll(): Unit = { + super.beforeAll() + seed = System.currentTimeMillis() + rand = new Random(seed) + } + + private def generateRandomInputData(): DataFrame = { + val schema = new StructType() + .add("a", IntegerType, nullable = false) + .add("b", IntegerType, nullable = false) + val inputData = Seq.fill(10000)(Row(rand.nextInt(), rand.nextInt())) + spark.createDataFrame(sparkContext.parallelize(Random.shuffle(inputData), 10), schema) + } + + /** + * Adds a no-op filter to the child plan in order to prevent executeCollect() from being + * called directly on the child plan. + */ + private def noOpFilter(plan: SparkPlan): SparkPlan = FilterExec(Literal(true), plan) + + val limit = 250 + val sortOrder = 'a.desc :: 'b.desc :: Nil + + test("TakeOrderedAndProject.doExecute without project") { + withClue(s"seed = $seed") { + checkThatPlansAgree( + generateRandomInputData(), + input => + noOpFilter(TakeOrderedAndProjectExec(limit, sortOrder, input.output, input)), + input => + GlobalLimitExec(limit, + LocalLimitExec(limit, + SortExec(sortOrder, true, input))), + sortAnswers = false) + } + } + + test("TakeOrderedAndProject.doExecute with project") { + withClue(s"seed = $seed") { + checkThatPlansAgree( + generateRandomInputData(), + input => + noOpFilter( + TakeOrderedAndProjectExec(limit, sortOrder, Seq(input.output.last), input)), + input => + GlobalLimitExec(limit, + LocalLimitExec(limit, + ProjectExec(Seq(input.output.last), + SortExec(sortOrder, true, input)))), + sortAnswers = false) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TungstenSortSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TungstenSortSuite.scala deleted file mode 100644 index 7a0f0dfd2b7f..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TungstenSortSuite.scala +++ /dev/null @@ -1,100 +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.execution - -import scala.util.Random - -import org.apache.spark.AccumulatorSuite -import org.apache.spark.sql.{RandomDataGenerator, Row, SQLConf} -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types._ - -/** - * A test suite that generates randomized data to test the [[TungstenSort]] operator. - */ -class TungstenSortSuite extends SparkPlanTest with SharedSQLContext { - import testImplicits.localSeqToDataFrameHolder - - override def beforeAll(): Unit = { - super.beforeAll() - sqlContext.conf.setConf(SQLConf.CODEGEN_ENABLED, true) - } - - override def afterAll(): Unit = { - try { - sqlContext.conf.unsetConf(SQLConf.CODEGEN_ENABLED) - } finally { - super.afterAll() - } - } - - test("sort followed by limit") { - checkThatPlansAgree( - (1 to 100).map(v => Tuple1(v)).toDF("a"), - (child: SparkPlan) => Limit(10, TungstenSort('a.asc :: Nil, true, child)), - (child: SparkPlan) => Limit(10, Sort('a.asc :: Nil, global = true, child)), - sortAnswers = false - ) - } - - test("sorting does not crash for large inputs") { - val sortOrder = 'a.asc :: Nil - val stringLength = 1024 * 1024 * 2 - checkThatPlansAgree( - Seq(Tuple1("a" * stringLength), Tuple1("b" * stringLength)).toDF("a").repartition(1), - TungstenSort(sortOrder, global = true, _: SparkPlan, testSpillFrequency = 1), - Sort(sortOrder, global = true, _: SparkPlan), - sortAnswers = false - ) - } - - test("sorting updates peak execution memory") { - AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "unsafe external sort") { - checkThatPlansAgree( - (1 to 100).map(v => Tuple1(v)).toDF("a"), - (child: SparkPlan) => TungstenSort('a.asc :: Nil, true, child), - (child: SparkPlan) => Sort('a.asc :: Nil, global = true, child), - sortAnswers = false) - } - } - - // Test sorting on different data types - for ( - dataType <- DataTypeTestUtils.atomicTypes ++ Set(NullType); - nullable <- Seq(true, false); - sortOrder <- Seq('a.asc :: Nil, 'a.desc :: Nil); - randomDataGenerator <- RandomDataGenerator.forType(dataType, nullable) - ) { - test(s"sorting on $dataType with nullable=$nullable, sortOrder=$sortOrder") { - val inputData = Seq.fill(1000)(randomDataGenerator()) - val inputDf = sqlContext.createDataFrame( - sparkContext.parallelize(Random.shuffle(inputData).map(v => Row(v))), - StructType(StructField("a", dataType, nullable = true) :: Nil) - ) - assert(TungstenSort.supportsSchema(inputDf.schema)) - checkThatPlansAgree( - inputDf, - plan => ConvertToSafe( - TungstenSort(sortOrder, global = true, plan: SparkPlan, testSpillFrequency = 23)), - Sort(sortOrder, global = true, _: SparkPlan), - sortAnswers = false - ) - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala index 7ceaee38d131..d194f58cd1cd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala @@ -17,16 +17,18 @@ package org.apache.spark.sql.execution -import scala.util.control.NonFatal +import java.util.Properties + import scala.collection.mutable -import scala.util.{Try, Random} +import scala.util.{Random, Try} +import scala.util.control.NonFatal import org.scalatest.Matchers -import org.apache.spark.{SparkConf, TaskContextImpl, TaskContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkFunSuite, TaskContext, TaskContextImpl} import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{UnsafeRow, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -61,7 +63,7 @@ class UnsafeFixedWidthAggregationMapSuite } test(name) { - val conf = new SparkConf().set("spark.unsafe.offHeap", "false") + val conf = new SparkConf().set("spark.memory.offHeap.enabled", "false") memoryManager = new TestMemoryManager(conf) taskMemoryManager = new TaskMemoryManager(memoryManager, 0) @@ -71,8 +73,8 @@ class UnsafeFixedWidthAggregationMapSuite taskAttemptId = Random.nextInt(10000), attemptNumber = 0, taskMemoryManager = taskMemoryManager, - metricsSystem = null, - internalAccumulators = Seq.empty)) + localProperties = new Properties, + metricsSystem = null)) try { f @@ -109,8 +111,7 @@ class UnsafeFixedWidthAggregationMapSuite groupKeySchema, taskMemoryManager, 1024, // initial capacity, - PAGE_SIZE_BYTES, - false // disable perf metrics + PAGE_SIZE_BYTES ) assert(!map.iterator().next()) map.free() @@ -123,13 +124,13 @@ class UnsafeFixedWidthAggregationMapSuite groupKeySchema, taskMemoryManager, 1024, // initial capacity - PAGE_SIZE_BYTES, - false // disable perf metrics + PAGE_SIZE_BYTES ) val groupKey = InternalRow(UTF8String.fromString("cats")) + val row = map.getAggregationBuffer(groupKey) // Looking up a key stores a zero-entry in the map (like Python Counters or DefaultDicts) - assert(map.getAggregationBuffer(groupKey) != null) + assert(row != null) val iter = map.iterator() assert(iter.next()) iter.getKey.getString(0) should be ("cats") @@ -138,7 +139,7 @@ class UnsafeFixedWidthAggregationMapSuite // Modifications to rows retrieved from the map should update the values in the map iter.getValue.setInt(0, 42) - map.getAggregationBuffer(groupKey).getInt(0) should be (42) + row.getInt(0) should be (42) map.free() } @@ -150,8 +151,7 @@ class UnsafeFixedWidthAggregationMapSuite groupKeySchema, taskMemoryManager, 128, // initial capacity - PAGE_SIZE_BYTES, - false // disable perf metrics + PAGE_SIZE_BYTES ) val rand = new Random(42) val groupKeys: Set[String] = Seq.fill(512)(rand.nextString(1024)).toSet @@ -176,8 +176,7 @@ class UnsafeFixedWidthAggregationMapSuite groupKeySchema, taskMemoryManager, 128, // initial capacity - PAGE_SIZE_BYTES, - false // disable perf metrics + PAGE_SIZE_BYTES ) val keys = randomStrings(1024).take(512) @@ -224,8 +223,7 @@ class UnsafeFixedWidthAggregationMapSuite groupKeySchema, taskMemoryManager, 128, // initial capacity - PAGE_SIZE_BYTES, - false // disable perf metrics + PAGE_SIZE_BYTES ) val sorter = map.destructAndCreateExternalSorter() @@ -265,8 +263,7 @@ class UnsafeFixedWidthAggregationMapSuite StructType(Nil), taskMemoryManager, 128, // initial capacity - PAGE_SIZE_BYTES, - false // disable perf metrics + PAGE_SIZE_BYTES ) (1 to 10).foreach { i => val buf = map.getAggregationBuffer(UnsafeRow.createFromByteArray(0, 0)) @@ -310,8 +307,7 @@ class UnsafeFixedWidthAggregationMapSuite groupKeySchema, taskMemoryManager, 128, // initial capacity - pageSize, - false // disable perf metrics + pageSize ) val rand = new Random(42) @@ -340,4 +336,43 @@ class UnsafeFixedWidthAggregationMapSuite } } + testWithMemoryLeakDetection("convert to external sorter after fail to grow (SPARK-19500)") { + val pageSize = 4096000 + val map = new UnsafeFixedWidthAggregationMap( + emptyAggregationBuffer, + aggBufferSchema, + groupKeySchema, + taskMemoryManager, + 128, // initial capacity + pageSize + ) + + val rand = new Random(42) + for (i <- 1 to 63) { + val str = rand.nextString(1024) + val buf = map.getAggregationBuffer(InternalRow(UTF8String.fromString(str))) + buf.setInt(0, str.length) + } + // Simulate running out of space + memoryManager.limit(0) + var str = rand.nextString(1024) + var buf = map.getAggregationBuffer(InternalRow(UTF8String.fromString(str))) + assert(buf != null) + str = rand.nextString(1024) + buf = map.getAggregationBuffer(InternalRow(UTF8String.fromString(str))) + assert(buf == null) + + // Convert the map into a sorter. This used to fail before the fix for SPARK-10474 + // because we would try to acquire space for the in-memory sorter pointer array before + // actually releasing the pages despite having spilled all of them. + var sorter: UnsafeKVExternalSorter = null + try { + sorter = map.destructAndCreateExternalSorter() + map.free() + } finally { + if (sorter != null) { + sorter.cleanupResources() + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala index 7b80963ec870..3d869c77e960 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala @@ -17,15 +17,18 @@ package org.apache.spark.sql.execution +import java.util.Properties + import scala.util.Random import org.apache.spark._ import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{InterpretedOrdering, UnsafeRow, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{InterpretedOrdering, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ +import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter /** * Test suite for [[UnsafeKVExternalSorter]], with randomly generated test data. @@ -40,8 +43,8 @@ class UnsafeKVExternalSorterSuite extends SparkFunSuite with SharedSQLContext { private val rand = new Random(42) for (i <- 0 until 6) { - val keySchema = RandomDataGenerator.randomSchema(rand.nextInt(10) + 1, keyTypes) - val valueSchema = RandomDataGenerator.randomSchema(rand.nextInt(10) + 1, valueTypes) + val keySchema = RandomDataGenerator.randomSchema(rand, rand.nextInt(10) + 1, keyTypes) + val valueSchema = RandomDataGenerator.randomSchema(rand, rand.nextInt(10) + 1, valueTypes) testKVSorter(keySchema, valueSchema, spill = i > 3) } @@ -109,7 +112,7 @@ class UnsafeKVExternalSorterSuite extends SparkFunSuite with SharedSQLContext { pageSize: Long, spill: Boolean): Unit = { val memoryManager = - new TestMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")) + new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")) val taskMemMgr = new TaskMemoryManager(memoryManager, 0) TaskContext.setTaskContext(new TaskContextImpl( stageId = 0, @@ -117,11 +120,12 @@ class UnsafeKVExternalSorterSuite extends SparkFunSuite with SharedSQLContext { taskAttemptId = 98456, attemptNumber = 0, taskMemoryManager = taskMemMgr, - metricsSystem = null, - internalAccumulators = Seq.empty)) + localProperties = new Properties, + metricsSystem = null)) val sorter = new UnsafeKVExternalSorter( - keySchema, valueSchema, SparkEnv.get.blockManager, pageSize) + keySchema, valueSchema, SparkEnv.get.blockManager, SparkEnv.get.serializerManager, + pageSize, UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD) // Insert the keys and values into the sorter inputData.foreach { case (k, v) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala index 09e258299de5..dff88ce7f1b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala @@ -17,20 +17,20 @@ package org.apache.spark.sql.execution -import java.io.{File, ByteArrayInputStream, ByteArrayOutputStream} +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File} +import java.util.Properties -import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark._ +import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rdd.RDD -import org.apache.spark.storage.ShuffleBlockId -import org.apache.spark.util.collection.ExternalSorter -import org.apache.spark.util.Utils import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} import org.apache.spark.sql.types._ -import org.apache.spark._ - +import org.apache.spark.storage.ShuffleBlockId +import org.apache.spark.util.Utils +import org.apache.spark.util.collection.ExternalSorter /** * used to test close InputStream in UnsafeRowSerializer @@ -114,13 +114,12 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkContext { (i, converter(Row(i))) } val taskMemoryManager = new TaskMemoryManager(sc.env.memoryManager, 0) - val taskContext = new TaskContextImpl( - 0, 0, 0, 0, taskMemoryManager, null, InternalAccumulator.create(sc)) + val taskContext = new TaskContextImpl(0, 0, 0, 0, taskMemoryManager, new Properties, null) val sorter = new ExternalSorter[Int, UnsafeRow, UnsafeRow]( taskContext, partitioner = Some(new HashPartitioner(10)), - serializer = Some(new UnsafeRowSerializer(numFields = 1))) + serializer = new UnsafeRowSerializer(numFields = 1)) // Ensure we spilled something and have to merge them later assert(sorter.numSpills === 0) @@ -128,7 +127,6 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkContext { assert(sorter.numSpills > 0) // Merging spilled files should not throw assertion error - taskContext.taskMetrics.shuffleWriteMetrics = Some(new ShuffleWriteMetrics) sorter.writePartitionedFile(ShuffleBlockId(0, 0, 0), outputFile) } { // Clean up @@ -156,7 +154,7 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkContext { new ShuffleDependency[Int, InternalRow, InternalRow]( rowsRDD, new PartitionIdPassthrough(2), - Some(new UnsafeRowSerializer(2))) + new UnsafeRowSerializer(2)) val shuffled = new ShuffledRowRDD(dependency) shuffled.count() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala new file mode 100644 index 000000000000..beeee6a97c8d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.{Column, Dataset, Row} +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{Add, Literal, Stack} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext +import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec +import org.apache.spark.sql.execution.joins.SortMergeJoinExec +import org.apache.spark.sql.expressions.scalalang.typed +import org.apache.spark.sql.functions.{avg, broadcast, col, max} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{IntegerType, StringType, StructType} + +class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { + + test("range/filter should be combined") { + val df = spark.range(10).filter("id = 1").selectExpr("id + 1") + val plan = df.queryExecution.executedPlan + assert(plan.find(_.isInstanceOf[WholeStageCodegenExec]).isDefined) + assert(df.collect() === Array(Row(2))) + } + + test("Aggregate should be included in WholeStageCodegen") { + val df = spark.range(10).groupBy().agg(max(col("id")), avg(col("id"))) + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) + assert(df.collect() === Array(Row(9, 4.5))) + } + + test("Aggregate with grouping keys should be included in WholeStageCodegen") { + val df = spark.range(3).groupBy("id").count().orderBy("id") + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) + assert(df.collect() === Array(Row(0, 1), Row(1, 1), Row(2, 1))) + } + + test("BroadcastHashJoin should be included in WholeStageCodegen") { + val rdd = spark.sparkContext.makeRDD(Seq(Row(1, "1"), Row(1, "1"), Row(2, "2"))) + val schema = new StructType().add("k", IntegerType).add("v", StringType) + val smallDF = spark.createDataFrame(rdd, schema) + val df = spark.range(10).join(broadcast(smallDF), col("k") === col("id")) + assert(df.queryExecution.executedPlan.find(p => + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[BroadcastHashJoinExec]).isDefined) + assert(df.collect() === Array(Row(1, 1, "1"), Row(1, 1, "1"), Row(2, 2, "2"))) + } + + test("Sort should be included in WholeStageCodegen") { + val df = spark.range(3, 0, -1).toDF().sort(col("id")) + val plan = df.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SortExec]).isDefined) + assert(df.collect() === Array(Row(1), Row(2), Row(3))) + } + + test("MapElements should be included in WholeStageCodegen") { + import testImplicits._ + + val ds = spark.range(10).map(_.toString) + val plan = ds.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SerializeFromObjectExec]).isDefined) + assert(ds.collect() === 0.until(10).map(_.toString).toArray) + } + + test("typed filter should be included in WholeStageCodegen") { + val ds = spark.range(10).filter(_ % 2 == 0) + val plan = ds.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[FilterExec]).isDefined) + assert(ds.collect() === Array(0, 2, 4, 6, 8)) + } + + test("back-to-back typed filter should be included in WholeStageCodegen") { + val ds = spark.range(10).filter(_ % 2 == 0).filter(_ % 3 == 0) + val plan = ds.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[FilterExec]).isDefined) + assert(ds.collect() === Array(0, 6)) + } + + test("simple typed UDAF should be included in WholeStageCodegen") { + import testImplicits._ + + val ds = Seq(("a", 10), ("b", 1), ("b", 2), ("c", 1)).toDS() + .groupByKey(_._1).agg(typed.sum(_._2)) + + val plan = ds.queryExecution.executedPlan + assert(plan.find(p => + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[HashAggregateExec]).isDefined) + assert(ds.collect() === Array(("a", 10.0), ("b", 3.0), ("c", 1.0))) + } + + test("SPARK-19512 codegen for comparing structs is incorrect") { + // this would raise CompileException before the fix + spark.range(10) + .selectExpr("named_struct('a', id) as col1", "named_struct('a', id+2) as col2") + .filter("col1 = col2").count() + // this would raise java.lang.IndexOutOfBoundsException before the fix + spark.range(10) + .selectExpr("named_struct('a', id, 'b', id) as col1", + "named_struct('a',id+2, 'b',id+2) as col2") + .filter("col1 = col2").count() + } + + test("SPARK-21441 SortMergeJoin codegen with CodegenFallback expressions should be disabled") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1") { + import testImplicits._ + + val df1 = Seq((1, 1), (2, 2), (3, 3)).toDF("key", "int") + val df2 = Seq((1, "1"), (2, "2"), (3, "3")).toDF("key", "str") + + val df = df1.join(df2, df1("key") === df2("key")) + .filter("int = 2 or reflect('java.lang.Integer', 'valueOf', str) = 1") + .select("int") + + val plan = df.queryExecution.executedPlan + assert(!plan.find(p => + p.isInstanceOf[WholeStageCodegenExec] && + p.asInstanceOf[WholeStageCodegenExec].child.children(0) + .isInstanceOf[SortMergeJoinExec]).isDefined) + assert(df.collect() === Array(Row(1), Row(2))) + } + } + + def genGroupByCodeGenContext(caseNum: Int): CodegenContext = { + val caseExp = (1 to caseNum).map { i => + s"case when id > $i and id <= ${i + 1} then 1 else 0 end as v$i" + }.toList + val keyExp = List( + "id", + "(id & 1023) as k1", + "cast(id & 1023 as double) as k2", + "cast(id & 1023 as int) as k3") + + val ds = spark.range(10) + .selectExpr(keyExp:::caseExp: _*) + .groupBy("k1", "k2", "k3") + .sum() + val plan = ds.queryExecution.executedPlan + + val wholeStageCodeGenExec = plan.find(p => p match { + case wp: WholeStageCodegenExec => wp.child match { + case hp: HashAggregateExec if (hp.child.isInstanceOf[ProjectExec]) => true + case _ => false + } + case _ => false + }) + + assert(wholeStageCodeGenExec.isDefined) + wholeStageCodeGenExec.get.asInstanceOf[WholeStageCodegenExec].doCodeGen()._1 + } + + test("SPARK-21603 check there is a too long generated function") { + withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> "1500") { + val ctx = genGroupByCodeGenContext(30) + assert(ctx.isTooLongGeneratedFunction === true) + } + } + + test("SPARK-21603 check there is not a too long generated function") { + withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> "1500") { + val ctx = genGroupByCodeGenContext(1) + assert(ctx.isTooLongGeneratedFunction === false) + } + } + + test("SPARK-21603 check there is not a too long generated function when threshold is Int.Max") { + withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> Int.MaxValue.toString) { + val ctx = genGroupByCodeGenContext(30) + assert(ctx.isTooLongGeneratedFunction === false) + } + } + + test("SPARK-21603 check there is a too long generated function when threshold is 0") { + withSQLConf(SQLConf.WHOLESTAGE_MAX_LINES_PER_FUNCTION.key -> "0") { + val ctx = genGroupByCodeGenContext(1) + assert(ctx.isTooLongGeneratedFunction === true) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala new file mode 100644 index 000000000000..10f1ee279bed --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.aggregate + +import java.util.Properties + +import scala.collection.mutable + +import org.apache.spark._ +import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.unsafe.KVIterator + +class SortBasedAggregationStoreSuite extends SparkFunSuite with LocalSparkContext { + + override def beforeAll(): Unit = { + super.beforeAll() + val conf = new SparkConf() + sc = new SparkContext("local[2, 4]", "test", conf) + val taskManager = new TaskMemoryManager(new TestMemoryManager(conf), 0) + TaskContext.setTaskContext(new TaskContextImpl(0, 0, 0, 0, taskManager, new Properties, null)) + } + + override def afterAll(): Unit = TaskContext.unset() + + private val rand = new java.util.Random() + + // In this test, the aggregator is XOR checksum. + test("merge input kv iterator and aggregation buffer iterator") { + + val inputSchema = StructType(Seq(StructField("a", IntegerType), StructField("b", IntegerType))) + val groupingSchema = StructType(Seq(StructField("b", IntegerType))) + + // Schema: a: Int, b: Int + val inputRow: UnsafeRow = createUnsafeRow(2) + + // Schema: group: Int + val group: UnsafeRow = createUnsafeRow(1) + + val expected = new mutable.HashMap[Int, Int]() + val hashMap = new ObjectAggregationMap + (0 to 5000).foreach { _ => + randomKV(inputRow, group) + + // XOR aggregate on first column of input row + expected.put(group.getInt(0), expected.getOrElse(group.getInt(0), 0) ^ inputRow.getInt(0)) + if (hashMap.getAggregationBuffer(group) == null) { + hashMap.putAggregationBuffer(group.copy, createNewAggregationBuffer()) + } + updateInputRow(hashMap.getAggregationBuffer(group), inputRow) + } + + val store = new SortBasedAggregator( + createSortedAggBufferIterator(hashMap), + inputSchema, + groupingSchema, + updateInputRow, + mergeAggBuffer, + createNewAggregationBuffer) + + (5000 to 100000).foreach { _ => + randomKV(inputRow, group) + // XOR aggregate on first column of input row + expected.put(group.getInt(0), expected.getOrElse(group.getInt(0), 0) ^ inputRow.getInt(0)) + store.addInput(group, inputRow) + } + + val iter = store.destructiveIterator() + while(iter.hasNext) { + val agg = iter.next() + assert(agg.aggregationBuffer.getInt(0) == expected(agg.groupingKey.getInt(0))) + } + } + + private def createNewAggregationBuffer(): InternalRow = { + val buffer = createUnsafeRow(1) + buffer.setInt(0, 0) + buffer + } + + private def updateInputRow: (InternalRow, InternalRow) => Unit = { + (buffer: InternalRow, input: InternalRow) => { + buffer.setInt(0, buffer.getInt(0) ^ input.getInt(0)) + } + } + + private def mergeAggBuffer: (InternalRow, InternalRow) => Unit = updateInputRow + + private def createUnsafeRow(numOfField: Int): UnsafeRow = { + val buffer: Array[Byte] = new Array(1024) + val row: UnsafeRow = new UnsafeRow(numOfField) + row.pointTo(buffer, 1024) + row + } + + private def randomKV(inputRow: UnsafeRow, group: UnsafeRow): Unit = { + inputRow.setInt(0, rand.nextInt(100000)) + inputRow.setInt(1, rand.nextInt(10000)) + group.setInt(0, inputRow.getInt(1) % 100) + } + + def createSortedAggBufferIterator( + hashMap: ObjectAggregationMap): KVIterator[UnsafeRow, UnsafeRow] = { + + val sortedIterator = hashMap.iterator.toList.sortBy(_.groupingKey.getInt(0)).iterator + new KVIterator[UnsafeRow, UnsafeRow] { + var key: UnsafeRow = null + var value: UnsafeRow = null + override def next: Boolean = { + if (sortedIterator.hasNext) { + val kv = sortedIterator.next() + key = kv.groupingKey + value = kv.aggregationBuffer.asInstanceOf[UnsafeRow] + true + } else { + false + } + } + override def getKey(): UnsafeRow = key + override def getValue(): UnsafeRow = value + override def close(): Unit = Unit + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala new file mode 100644 index 000000000000..30422b657742 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala @@ -0,0 +1,1692 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.arrow + +import java.io.File +import java.nio.charset.StandardCharsets +import java.sql.{Date, Timestamp} +import java.text.SimpleDateFormat +import java.util.Locale + +import com.google.common.io.Files +import org.apache.arrow.memory.RootAllocator +import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot} +import org.apache.arrow.vector.file.json.JsonFileReader +import org.apache.arrow.vector.util.Validator +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{BinaryType, IntegerType, StructField, StructType} +import org.apache.spark.util.Utils + + +class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { + import testImplicits._ + + private var tempDataPath: String = _ + + override def beforeAll(): Unit = { + super.beforeAll() + tempDataPath = Utils.createTempDir(namePrefix = "arrow").getAbsolutePath + } + + test("collect to arrow record batch") { + val indexData = (1 to 6).toDF("i") + val arrowPayloads = indexData.toArrowPayload.collect() + assert(arrowPayloads.nonEmpty) + assert(arrowPayloads.length == indexData.rdd.getNumPartitions) + val allocator = new RootAllocator(Long.MaxValue) + val arrowRecordBatches = arrowPayloads.map(_.loadBatch(allocator)) + val rowCount = arrowRecordBatches.map(_.getLength).sum + assert(rowCount === indexData.count()) + arrowRecordBatches.foreach(batch => assert(batch.getNodes.size() > 0)) + arrowRecordBatches.foreach(_.close()) + allocator.close() + } + + test("short conversion") { + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "a_s", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 16 + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 16 + | } ] + | } + | }, { + | "name" : "b_s", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 16 + | }, + | "nullable" : true, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 16 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 6, + | "columns" : [ { + | "name" : "a_s", + | "count" : 6, + | "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ], + | "DATA" : [ 1, -1, 2, -2, 32767, -32768 ] + | }, { + | "name" : "b_s", + | "count" : 6, + | "VALIDITY" : [ 1, 0, 0, 1, 0, 1 ], + | "DATA" : [ 1, 0, 0, -2, 0, -32768 ] + | } ] + | } ] + |} + """.stripMargin + + val a_s = List[Short](1, -1, 2, -2, 32767, -32768) + val b_s = List[Option[Short]](Some(1), None, None, Some(-2), None, Some(-32768)) + val df = a_s.zip(b_s).toDF("a_s", "b_s") + + collectAndValidate(df, json, "integer-16bit.json") + } + + test("int conversion") { + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "a_i", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 32 + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | }, { + | "name" : "b_i", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 32 + | }, + | "nullable" : true, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 6, + | "columns" : [ { + | "name" : "a_i", + | "count" : 6, + | "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ], + | "DATA" : [ 1, -1, 2, -2, 2147483647, -2147483648 ] + | }, { + | "name" : "b_i", + | "count" : 6, + | "VALIDITY" : [ 1, 0, 0, 1, 0, 1 ], + | "DATA" : [ 1, 0, 0, -2, 0, -2147483648 ] + | } ] + | } ] + |} + """.stripMargin + + val a_i = List[Int](1, -1, 2, -2, 2147483647, -2147483648) + val b_i = List[Option[Int]](Some(1), None, None, Some(-2), None, Some(-2147483648)) + val df = a_i.zip(b_i).toDF("a_i", "b_i") + + collectAndValidate(df, json, "integer-32bit.json") + } + + test("long conversion") { + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "a_l", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 64 + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 64 + | } ] + | } + | }, { + | "name" : "b_l", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 64 + | }, + | "nullable" : true, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 64 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 6, + | "columns" : [ { + | "name" : "a_l", + | "count" : 6, + | "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ], + | "DATA" : [ 1, -1, 2, -2, 9223372036854775807, -9223372036854775808 ] + | }, { + | "name" : "b_l", + | "count" : 6, + | "VALIDITY" : [ 1, 0, 0, 1, 0, 1 ], + | "DATA" : [ 1, 0, 0, -2, 0, -9223372036854775808 ] + | } ] + | } ] + |} + """.stripMargin + + val a_l = List[Long](1, -1, 2, -2, 9223372036854775807L, -9223372036854775808L) + val b_l = List[Option[Long]](Some(1), None, None, Some(-2), None, Some(-9223372036854775808L)) + val df = a_l.zip(b_l).toDF("a_l", "b_l") + + collectAndValidate(df, json, "integer-64bit.json") + } + + test("float conversion") { + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "a_f", + | "type" : { + | "name" : "floatingpoint", + | "precision" : "SINGLE" + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | }, { + | "name" : "b_f", + | "type" : { + | "name" : "floatingpoint", + | "precision" : "SINGLE" + | }, + | "nullable" : true, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 6, + | "columns" : [ { + | "name" : "a_f", + | "count" : 6, + | "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ], + | "DATA" : [ 1.0, 2.0, 0.01, 200.0, 0.0001, 20000.0 ] + | }, { + | "name" : "b_f", + | "count" : 6, + | "VALIDITY" : [ 1, 0, 0, 1, 0, 1 ], + | "DATA" : [ 1.1, 0.0, 0.0, 2.2, 0.0, 3.3 ] + | } ] + | } ] + |} + """.stripMargin + + val a_f = List(1.0f, 2.0f, 0.01f, 200.0f, 0.0001f, 20000.0f) + val b_f = List[Option[Float]](Some(1.1f), None, None, Some(2.2f), None, Some(3.3f)) + val df = a_f.zip(b_f).toDF("a_f", "b_f") + + collectAndValidate(df, json, "floating_point-single_precision.json") + } + + test("double conversion") { + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "a_d", + | "type" : { + | "name" : "floatingpoint", + | "precision" : "DOUBLE" + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 64 + | } ] + | } + | }, { + | "name" : "b_d", + | "type" : { + | "name" : "floatingpoint", + | "precision" : "DOUBLE" + | }, + | "nullable" : true, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 64 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 6, + | "columns" : [ { + | "name" : "a_d", + | "count" : 6, + | "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ], + | "DATA" : [ 1.0, 2.0, 0.01, 200.0, 1.0E-4, 20000.0 ] + | }, { + | "name" : "b_d", + | "count" : 6, + | "VALIDITY" : [ 1, 0, 0, 1, 0, 1 ], + | "DATA" : [ 1.1, 0.0, 0.0, 2.2, 0.0, 3.3 ] + | } ] + | } ] + |} + """.stripMargin + + val a_d = List(1.0, 2.0, 0.01, 200.0, 0.0001, 20000.0) + val b_d = List[Option[Double]](Some(1.1), None, None, Some(2.2), None, Some(3.3)) + val df = a_d.zip(b_d).toDF("a_d", "b_d") + + collectAndValidate(df, json, "floating_point-double_precision.json") + } + + test("index conversion") { + val data = List[Int](1, 2, 3, 4, 5, 6) + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "i", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 32 + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 6, + | "columns" : [ { + | "name" : "i", + | "count" : 6, + | "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ], + | "DATA" : [ 1, 2, 3, 4, 5, 6 ] + | } ] + | } ] + |} + """.stripMargin + val df = data.toDF("i") + + collectAndValidate(df, json, "indexData-ints.json") + } + + test("mixed numeric type conversion") { + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "a", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 16 + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 16 + | } ] + | } + | }, { + | "name" : "b", + | "type" : { + | "name" : "floatingpoint", + | "precision" : "SINGLE" + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | }, { + | "name" : "c", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 32 + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | }, { + | "name" : "d", + | "type" : { + | "name" : "floatingpoint", + | "precision" : "DOUBLE" + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 64 + | } ] + | } + | }, { + | "name" : "e", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 64 + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 64 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 6, + | "columns" : [ { + | "name" : "a", + | "count" : 6, + | "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ], + | "DATA" : [ 1, 2, 3, 4, 5, 6 ] + | }, { + | "name" : "b", + | "count" : 6, + | "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ], + | "DATA" : [ 1.0, 2.0, 3.0, 4.0, 5.0, 6.0 ] + | }, { + | "name" : "c", + | "count" : 6, + | "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ], + | "DATA" : [ 1, 2, 3, 4, 5, 6 ] + | }, { + | "name" : "d", + | "count" : 6, + | "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ], + | "DATA" : [ 1.0, 2.0, 3.0, 4.0, 5.0, 6.0 ] + | }, { + | "name" : "e", + | "count" : 6, + | "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ], + | "DATA" : [ 1, 2, 3, 4, 5, 6 ] + | } ] + | } ] + |} + """.stripMargin + + val data = List(1, 2, 3, 4, 5, 6) + val data_tuples = for (d <- data) yield { + (d.toShort, d.toFloat, d.toInt, d.toDouble, d.toLong) + } + val df = data_tuples.toDF("a", "b", "c", "d", "e") + + collectAndValidate(df, json, "mixed_numeric_types.json") + } + + test("string type conversion") { + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "upper_case", + | "type" : { + | "name" : "utf8" + | }, + | "nullable" : true, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "OFFSET", + | "typeBitWidth" : 32 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 8 + | } ] + | } + | }, { + | "name" : "lower_case", + | "type" : { + | "name" : "utf8" + | }, + | "nullable" : true, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "OFFSET", + | "typeBitWidth" : 32 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 8 + | } ] + | } + | }, { + | "name" : "null_str", + | "type" : { + | "name" : "utf8" + | }, + | "nullable" : true, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "OFFSET", + | "typeBitWidth" : 32 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 8 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 3, + | "columns" : [ { + | "name" : "upper_case", + | "count" : 3, + | "VALIDITY" : [ 1, 1, 1 ], + | "OFFSET" : [ 0, 1, 2, 3 ], + | "DATA" : [ "A", "B", "C" ] + | }, { + | "name" : "lower_case", + | "count" : 3, + | "VALIDITY" : [ 1, 1, 1 ], + | "OFFSET" : [ 0, 1, 2, 3 ], + | "DATA" : [ "a", "b", "c" ] + | }, { + | "name" : "null_str", + | "count" : 3, + | "VALIDITY" : [ 1, 1, 0 ], + | "OFFSET" : [ 0, 2, 5, 5 ], + | "DATA" : [ "ab", "CDE", "" ] + | } ] + | } ] + |} + """.stripMargin + + val upperCase = Seq("A", "B", "C") + val lowerCase = Seq("a", "b", "c") + val nullStr = Seq("ab", "CDE", null) + val df = (upperCase, lowerCase, nullStr).zipped.toList + .toDF("upper_case", "lower_case", "null_str") + + collectAndValidate(df, json, "stringData.json") + } + + test("boolean type conversion") { + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "a_bool", + | "type" : { + | "name" : "bool" + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 1 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 4, + | "columns" : [ { + | "name" : "a_bool", + | "count" : 4, + | "VALIDITY" : [ 1, 1, 1, 1 ], + | "DATA" : [ true, true, false, true ] + | } ] + | } ] + |} + """.stripMargin + val df = Seq(true, true, false, true).toDF("a_bool") + collectAndValidate(df, json, "boolData.json") + } + + test("byte type conversion") { + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "a_byte", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 8 + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 8 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 4, + | "columns" : [ { + | "name" : "a_byte", + | "count" : 4, + | "VALIDITY" : [ 1, 1, 1, 1 ], + | "DATA" : [ 1, -1, 64, 127 ] + | } ] + | } ] + |} + | + """.stripMargin + val df = List[Byte](1.toByte, (-1).toByte, 64.toByte, Byte.MaxValue).toDF("a_byte") + collectAndValidate(df, json, "byteData.json") + } + + test("binary type conversion") { + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "a_binary", + | "type" : { + | "name" : "binary" + | }, + | "nullable" : true, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "OFFSET", + | "typeBitWidth" : 32 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 8 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 3, + | "columns" : [ { + | "name" : "a_binary", + | "count" : 3, + | "VALIDITY" : [ 1, 1, 1 ], + | "OFFSET" : [ 0, 3, 4, 6 ], + | "DATA" : [ "616263", "64", "6566" ] + | } ] + | } ] + |} + """.stripMargin + + val data = Seq("abc", "d", "ef") + val rdd = sparkContext.parallelize(data.map(s => Row(s.getBytes("utf-8")))) + val df = spark.createDataFrame(rdd, StructType(Seq(StructField("a_binary", BinaryType)))) + + collectAndValidate(df, json, "binaryData.json") + } + + test("floating-point NaN") { + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "NaN_f", + | "type" : { + | "name" : "floatingpoint", + | "precision" : "SINGLE" + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | }, { + | "name" : "NaN_d", + | "type" : { + | "name" : "floatingpoint", + | "precision" : "DOUBLE" + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 64 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 2, + | "columns" : [ { + | "name" : "NaN_f", + | "count" : 2, + | "VALIDITY" : [ 1, 1 ], + | "DATA" : [ 1.2000000476837158, "NaN" ] + | }, { + | "name" : "NaN_d", + | "count" : 2, + | "VALIDITY" : [ 1, 1 ], + | "DATA" : [ "NaN", 1.2 ] + | } ] + | } ] + |} + """.stripMargin + + val fnan = Seq(1.2F, Float.NaN) + val dnan = Seq(Double.NaN, 1.2) + val df = fnan.zip(dnan).toDF("NaN_f", "NaN_d") + + collectAndValidate(df, json, "nanData-floating_point.json") + } + + test("array type conversion") { + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "a_arr", + | "nullable" : true, + | "type" : { + | "name" : "list" + | }, + | "children" : [ { + | "name" : "element", + | "nullable" : false, + | "type" : { + | "name" : "int", + | "bitWidth" : 32, + | "isSigned" : true + | }, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "OFFSET", + | "typeBitWidth" : 32 + | } ] + | } + | }, { + | "name" : "b_arr", + | "nullable" : true, + | "type" : { + | "name" : "list" + | }, + | "children" : [ { + | "name" : "element", + | "nullable" : false, + | "type" : { + | "name" : "int", + | "bitWidth" : 32, + | "isSigned" : true + | }, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "OFFSET", + | "typeBitWidth" : 32 + | } ] + | } + | }, { + | "name" : "c_arr", + | "nullable" : true, + | "type" : { + | "name" : "list" + | }, + | "children" : [ { + | "name" : "element", + | "nullable" : true, + | "type" : { + | "name" : "int", + | "bitWidth" : 32, + | "isSigned" : true + | }, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "OFFSET", + | "typeBitWidth" : 32 + | } ] + | } + | }, { + | "name" : "d_arr", + | "nullable" : true, + | "type" : { + | "name" : "list" + | }, + | "children" : [ { + | "name" : "element", + | "nullable" : true, + | "type" : { + | "name" : "list" + | }, + | "children" : [ { + | "name" : "element", + | "nullable" : false, + | "type" : { + | "name" : "int", + | "bitWidth" : 32, + | "isSigned" : true + | }, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "OFFSET", + | "typeBitWidth" : 32 + | } ] + | } + | } ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "OFFSET", + | "typeBitWidth" : 32 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 4, + | "columns" : [ { + | "name" : "a_arr", + | "count" : 4, + | "VALIDITY" : [ 1, 1, 1, 1 ], + | "OFFSET" : [ 0, 2, 4, 4, 5 ], + | "children" : [ { + | "name" : "element", + | "count" : 5, + | "VALIDITY" : [ 1, 1, 1, 1, 1 ], + | "DATA" : [ 1, 2, 3, 4, 5 ] + | } ] + | }, { + | "name" : "b_arr", + | "count" : 4, + | "VALIDITY" : [ 1, 0, 1, 0 ], + | "OFFSET" : [ 0, 2, 2, 2, 2 ], + | "children" : [ { + | "name" : "element", + | "count" : 2, + | "VALIDITY" : [ 1, 1 ], + | "DATA" : [ 1, 2 ] + | } ] + | }, { + | "name" : "c_arr", + | "count" : 4, + | "VALIDITY" : [ 1, 1, 1, 1 ], + | "OFFSET" : [ 0, 2, 4, 4, 5 ], + | "children" : [ { + | "name" : "element", + | "count" : 5, + | "VALIDITY" : [ 1, 1, 1, 0, 1 ], + | "DATA" : [ 1, 2, 3, 0, 5 ] + | } ] + | }, { + | "name" : "d_arr", + | "count" : 4, + | "VALIDITY" : [ 1, 1, 1, 1 ], + | "OFFSET" : [ 0, 1, 3, 3, 4 ], + | "children" : [ { + | "name" : "element", + | "count" : 4, + | "VALIDITY" : [ 1, 1, 1, 1 ], + | "OFFSET" : [ 0, 2, 3, 3, 4 ], + | "children" : [ { + | "name" : "element", + | "count" : 4, + | "VALIDITY" : [ 1, 1, 1, 1 ], + | "DATA" : [ 1, 2, 3, 5 ] + | } ] + | } ] + | } ] + | } ] + |} + """.stripMargin + + val aArr = Seq(Seq(1, 2), Seq(3, 4), Seq(), Seq(5)) + val bArr = Seq(Some(Seq(1, 2)), None, Some(Seq()), None) + val cArr = Seq(Seq(Some(1), Some(2)), Seq(Some(3), None), Seq(), Seq(Some(5))) + val dArr = Seq(Seq(Seq(1, 2)), Seq(Seq(3), Seq()), Seq(), Seq(Seq(5))) + + val df = aArr.zip(bArr).zip(cArr).zip(dArr).map { + case (((a, b), c), d) => (a, b, c, d) + }.toDF("a_arr", "b_arr", "c_arr", "d_arr") + + collectAndValidate(df, json, "arrayData.json") + } + + test("struct type conversion") { + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "a_struct", + | "nullable" : false, + | "type" : { + | "name" : "struct" + | }, + | "children" : [ { + | "name" : "i", + | "nullable" : false, + | "type" : { + | "name" : "int", + | "bitWidth" : 32, + | "isSigned" : true + | }, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | } ] + | } + | }, { + | "name" : "b_struct", + | "nullable" : true, + | "type" : { + | "name" : "struct" + | }, + | "children" : [ { + | "name" : "i", + | "nullable" : false, + | "type" : { + | "name" : "int", + | "bitWidth" : 32, + | "isSigned" : true + | }, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | } ] + | } + | }, { + | "name" : "c_struct", + | "nullable" : false, + | "type" : { + | "name" : "struct" + | }, + | "children" : [ { + | "name" : "i", + | "nullable" : true, + | "type" : { + | "name" : "int", + | "bitWidth" : 32, + | "isSigned" : true + | }, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | } ] + | } + | }, { + | "name" : "d_struct", + | "nullable" : true, + | "type" : { + | "name" : "struct" + | }, + | "children" : [ { + | "name" : "nested", + | "nullable" : true, + | "type" : { + | "name" : "struct" + | }, + | "children" : [ { + | "name" : "i", + | "nullable" : true, + | "type" : { + | "name" : "int", + | "bitWidth" : 32, + | "isSigned" : true + | }, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | } ] + | } + | } ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 3, + | "columns" : [ { + | "name" : "a_struct", + | "count" : 3, + | "VALIDITY" : [ 1, 1, 1 ], + | "children" : [ { + | "name" : "i", + | "count" : 3, + | "VALIDITY" : [ 1, 1, 1 ], + | "DATA" : [ 1, 2, 3 ] + | } ] + | }, { + | "name" : "b_struct", + | "count" : 3, + | "VALIDITY" : [ 1, 0, 1 ], + | "children" : [ { + | "name" : "i", + | "count" : 3, + | "VALIDITY" : [ 1, 0, 1 ], + | "DATA" : [ 1, 2, 3 ] + | } ] + | }, { + | "name" : "c_struct", + | "count" : 3, + | "VALIDITY" : [ 1, 1, 1 ], + | "children" : [ { + | "name" : "i", + | "count" : 3, + | "VALIDITY" : [ 1, 0, 1 ], + | "DATA" : [ 1, 2, 3 ] + | } ] + | }, { + | "name" : "d_struct", + | "count" : 3, + | "VALIDITY" : [ 1, 0, 1 ], + | "children" : [ { + | "name" : "nested", + | "count" : 3, + | "VALIDITY" : [ 1, 0, 0 ], + | "children" : [ { + | "name" : "i", + | "count" : 3, + | "VALIDITY" : [ 1, 0, 0 ], + | "DATA" : [ 1, 2, 0 ] + | } ] + | } ] + | } ] + | } ] + |} + """.stripMargin + + val aStruct = Seq(Row(1), Row(2), Row(3)) + val bStruct = Seq(Row(1), null, Row(3)) + val cStruct = Seq(Row(1), Row(null), Row(3)) + val dStruct = Seq(Row(Row(1)), null, Row(null)) + val data = aStruct.zip(bStruct).zip(cStruct).zip(dStruct).map { + case (((a, b), c), d) => Row(a, b, c, d) + } + + val rdd = sparkContext.parallelize(data) + val schema = new StructType() + .add("a_struct", new StructType().add("i", IntegerType, nullable = false), nullable = false) + .add("b_struct", new StructType().add("i", IntegerType, nullable = false), nullable = true) + .add("c_struct", new StructType().add("i", IntegerType, nullable = true), nullable = false) + .add("d_struct", new StructType().add("nested", new StructType().add("i", IntegerType))) + val df = spark.createDataFrame(rdd, schema) + + collectAndValidate(df, json, "structData.json") + } + + test("partitioned DataFrame") { + val json1 = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "a", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 32 + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | }, { + | "name" : "b", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 32 + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 3, + | "columns" : [ { + | "name" : "a", + | "count" : 3, + | "VALIDITY" : [ 1, 1, 1 ], + | "DATA" : [ 1, 1, 2 ] + | }, { + | "name" : "b", + | "count" : 3, + | "VALIDITY" : [ 1, 1, 1 ], + | "DATA" : [ 1, 2, 1 ] + | } ] + | } ] + |} + """.stripMargin + val json2 = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "a", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 32 + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | }, { + | "name" : "b", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 32 + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 3, + | "columns" : [ { + | "name" : "a", + | "count" : 3, + | "VALIDITY" : [ 1, 1, 1 ], + | "DATA" : [ 2, 3, 3 ] + | }, { + | "name" : "b", + | "count" : 3, + | "VALIDITY" : [ 1, 1, 1 ], + | "DATA" : [ 2, 1, 2 ] + | } ] + | } ] + |} + """.stripMargin + + val arrowPayloads = testData2.toArrowPayload.collect() + // NOTE: testData2 should have 2 partitions -> 2 arrow batches in payload + assert(arrowPayloads.length === 2) + val schema = testData2.schema + + val tempFile1 = new File(tempDataPath, "testData2-ints-part1.json") + val tempFile2 = new File(tempDataPath, "testData2-ints-part2.json") + Files.write(json1, tempFile1, StandardCharsets.UTF_8) + Files.write(json2, tempFile2, StandardCharsets.UTF_8) + + validateConversion(schema, arrowPayloads(0), tempFile1) + validateConversion(schema, arrowPayloads(1), tempFile2) + } + + test("empty frame collect") { + val arrowPayload = spark.emptyDataFrame.toArrowPayload.collect() + assert(arrowPayload.isEmpty) + + val filteredDF = List[Int](1, 2, 3, 4, 5, 6).toDF("i") + val filteredArrowPayload = filteredDF.filter("i < 0").toArrowPayload.collect() + assert(filteredArrowPayload.isEmpty) + } + + test("empty partition collect") { + val emptyPart = spark.sparkContext.parallelize(Seq(1), 2).toDF("i") + val arrowPayloads = emptyPart.toArrowPayload.collect() + assert(arrowPayloads.length === 1) + val allocator = new RootAllocator(Long.MaxValue) + val arrowRecordBatches = arrowPayloads.map(_.loadBatch(allocator)) + assert(arrowRecordBatches.head.getLength == 1) + arrowRecordBatches.foreach(_.close()) + allocator.close() + } + + test("max records in batch conf") { + val totalRecords = 10 + val maxRecordsPerBatch = 3 + spark.conf.set("spark.sql.execution.arrow.maxRecordsPerBatch", maxRecordsPerBatch) + val df = spark.sparkContext.parallelize(1 to totalRecords, 2).toDF("i") + val arrowPayloads = df.toArrowPayload.collect() + assert(arrowPayloads.length >= 4) + val allocator = new RootAllocator(Long.MaxValue) + val arrowRecordBatches = arrowPayloads.map(_.loadBatch(allocator)) + var recordCount = 0 + arrowRecordBatches.foreach { batch => + assert(batch.getLength > 0) + assert(batch.getLength <= maxRecordsPerBatch) + recordCount += batch.getLength + batch.close() + } + assert(recordCount == totalRecords) + allocator.close() + spark.conf.unset("spark.sql.execution.arrow.maxRecordsPerBatch") + } + + testQuietly("unsupported types") { + def runUnsupported(block: => Unit): Unit = { + val msg = intercept[SparkException] { + block + } + assert(msg.getMessage.contains("Unsupported data type")) + assert(msg.getCause.getClass === classOf[UnsupportedOperationException]) + } + + runUnsupported { decimalData.toArrowPayload.collect() } + runUnsupported { mapData.toDF().toArrowPayload.collect() } + runUnsupported { complexData.toArrowPayload.collect() } + + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS z", Locale.US) + val d1 = new Date(sdf.parse("2015-04-08 13:10:15.000 UTC").getTime) + val d2 = new Date(sdf.parse("2016-05-09 13:10:15.000 UTC").getTime) + runUnsupported { Seq(d1, d2).toDF("date").toArrowPayload.collect() } + + val ts1 = new Timestamp(sdf.parse("2013-04-08 01:10:15.567 UTC").getTime) + val ts2 = new Timestamp(sdf.parse("2013-04-08 13:10:10.789 UTC").getTime) + runUnsupported { Seq(ts1, ts2).toDF("timestamp").toArrowPayload.collect() } + } + + test("test Arrow Validator") { + val json = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "a_i", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 32 + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | }, { + | "name" : "b_i", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 32 + | }, + | "nullable" : true, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 6, + | "columns" : [ { + | "name" : "a_i", + | "count" : 6, + | "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ], + | "DATA" : [ 1, -1, 2, -2, 2147483647, -2147483648 ] + | }, { + | "name" : "b_i", + | "count" : 6, + | "VALIDITY" : [ 1, 0, 0, 1, 0, 1 ], + | "DATA" : [ 1, 0, 0, -2, 0, -2147483648 ] + | } ] + | } ] + |} + """.stripMargin + val json_diff_col_order = + s""" + |{ + | "schema" : { + | "fields" : [ { + | "name" : "b_i", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 32 + | }, + | "nullable" : true, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | }, { + | "name" : "a_i", + | "type" : { + | "name" : "int", + | "isSigned" : true, + | "bitWidth" : 32 + | }, + | "nullable" : false, + | "children" : [ ], + | "typeLayout" : { + | "vectors" : [ { + | "type" : "VALIDITY", + | "typeBitWidth" : 1 + | }, { + | "type" : "DATA", + | "typeBitWidth" : 32 + | } ] + | } + | } ] + | }, + | "batches" : [ { + | "count" : 6, + | "columns" : [ { + | "name" : "a_i", + | "count" : 6, + | "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ], + | "DATA" : [ 1, -1, 2, -2, 2147483647, -2147483648 ] + | }, { + | "name" : "b_i", + | "count" : 6, + | "VALIDITY" : [ 1, 0, 0, 1, 0, 1 ], + | "DATA" : [ 1, 0, 0, -2, 0, -2147483648 ] + | } ] + | } ] + |} + """.stripMargin + + val a_i = List[Int](1, -1, 2, -2, 2147483647, -2147483648) + val b_i = List[Option[Int]](Some(1), None, None, Some(-2), None, Some(-2147483648)) + val df = a_i.zip(b_i).toDF("a_i", "b_i") + + // Different schema + intercept[IllegalArgumentException] { + collectAndValidate(df, json_diff_col_order, "validator_diff_schema.json") + } + + // Different values + intercept[IllegalArgumentException] { + collectAndValidate(df.sort($"a_i".desc), json, "validator_diff_values.json") + } + } + + test("roundtrip payloads") { + val inputRows = (0 until 9).map { i => + InternalRow(i) + } :+ InternalRow(null) + + val schema = StructType(Seq(StructField("int", IntegerType, nullable = true))) + + val ctx = TaskContext.empty() + val payloadIter = ArrowConverters.toPayloadIterator(inputRows.toIterator, schema, 0, ctx) + val outputRowIter = ArrowConverters.fromPayloadIterator(payloadIter, ctx) + + assert(schema.equals(outputRowIter.schema)) + + var count = 0 + outputRowIter.zipWithIndex.foreach { case (row, i) => + if (i != 9) { + assert(row.getInt(0) == i) + } else { + assert(row.isNullAt(0)) + } + count += 1 + } + + assert(count == inputRows.length) + } + + /** Test that a converted DataFrame to Arrow record batch equals batch read from JSON file */ + private def collectAndValidate(df: DataFrame, json: String, file: String): Unit = { + // NOTE: coalesce to single partition because can only load 1 batch in validator + val arrowPayload = df.coalesce(1).toArrowPayload.collect().head + val tempFile = new File(tempDataPath, file) + Files.write(json, tempFile, StandardCharsets.UTF_8) + validateConversion(df.schema, arrowPayload, tempFile) + } + + private def validateConversion( + sparkSchema: StructType, + arrowPayload: ArrowPayload, + jsonFile: File): Unit = { + val allocator = new RootAllocator(Long.MaxValue) + val jsonReader = new JsonFileReader(jsonFile, allocator) + + val arrowSchema = ArrowUtils.toArrowSchema(sparkSchema) + val jsonSchema = jsonReader.start() + Validator.compareSchemas(arrowSchema, jsonSchema) + + val arrowRoot = VectorSchemaRoot.create(arrowSchema, allocator) + val vectorLoader = new VectorLoader(arrowRoot) + val arrowRecordBatch = arrowPayload.loadBatch(allocator) + vectorLoader.load(arrowRecordBatch) + val jsonRoot = jsonReader.read() + Validator.compareVectorSchemaRoot(arrowRoot, jsonRoot) + + jsonRoot.close() + jsonReader.close() + arrowRecordBatch.close() + arrowRoot.close() + allocator.close() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowUtilsSuite.scala new file mode 100644 index 000000000000..638619fd39d0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowUtilsSuite.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.arrow + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types._ + +class ArrowUtilsSuite extends SparkFunSuite { + + def roundtrip(dt: DataType): Unit = { + dt match { + case schema: StructType => + assert(ArrowUtils.fromArrowSchema(ArrowUtils.toArrowSchema(schema)) === schema) + case _ => + roundtrip(new StructType().add("value", dt)) + } + } + + test("simple") { + roundtrip(BooleanType) + roundtrip(ByteType) + roundtrip(ShortType) + roundtrip(IntegerType) + roundtrip(LongType) + roundtrip(FloatType) + roundtrip(DoubleType) + roundtrip(StringType) + roundtrip(BinaryType) + roundtrip(DecimalType.SYSTEM_DEFAULT) + } + + test("array") { + roundtrip(ArrayType(IntegerType, containsNull = true)) + roundtrip(ArrayType(IntegerType, containsNull = false)) + roundtrip(ArrayType(ArrayType(IntegerType, containsNull = true), containsNull = true)) + roundtrip(ArrayType(ArrayType(IntegerType, containsNull = false), containsNull = true)) + roundtrip(ArrayType(ArrayType(IntegerType, containsNull = true), containsNull = false)) + roundtrip(ArrayType(ArrayType(IntegerType, containsNull = false), containsNull = false)) + } + + test("struct") { + roundtrip(new StructType()) + roundtrip(new StructType().add("i", IntegerType)) + roundtrip(new StructType().add("arr", ArrayType(IntegerType))) + roundtrip(new StructType().add("i", IntegerType).add("arr", ArrayType(IntegerType))) + roundtrip(new StructType().add( + "struct", + new StructType().add("i", IntegerType).add("arr", ArrayType(IntegerType)))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala new file mode 100644 index 000000000000..e9a629315f5f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala @@ -0,0 +1,260 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.arrow + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.execution.vectorized.ArrowColumnVector +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +class ArrowWriterSuite extends SparkFunSuite { + + test("simple") { + def check(dt: DataType, data: Seq[Any]): Unit = { + val schema = new StructType().add("value", dt, nullable = true) + val writer = ArrowWriter.create(schema) + assert(writer.schema === schema) + + data.foreach { datum => + writer.write(InternalRow(datum)) + } + writer.finish() + + val reader = new ArrowColumnVector(writer.root.getFieldVectors().get(0)) + data.zipWithIndex.foreach { + case (null, rowId) => assert(reader.isNullAt(rowId)) + case (datum, rowId) => + val value = dt match { + case BooleanType => reader.getBoolean(rowId) + case ByteType => reader.getByte(rowId) + case ShortType => reader.getShort(rowId) + case IntegerType => reader.getInt(rowId) + case LongType => reader.getLong(rowId) + case FloatType => reader.getFloat(rowId) + case DoubleType => reader.getDouble(rowId) + case StringType => reader.getUTF8String(rowId) + case BinaryType => reader.getBinary(rowId) + } + assert(value === datum) + } + + writer.root.close() + } + check(BooleanType, Seq(true, null, false)) + check(ByteType, Seq(1.toByte, 2.toByte, null, 4.toByte)) + check(ShortType, Seq(1.toShort, 2.toShort, null, 4.toShort)) + check(IntegerType, Seq(1, 2, null, 4)) + check(LongType, Seq(1L, 2L, null, 4L)) + check(FloatType, Seq(1.0f, 2.0f, null, 4.0f)) + check(DoubleType, Seq(1.0d, 2.0d, null, 4.0d)) + check(StringType, Seq("a", "b", null, "d").map(UTF8String.fromString)) + check(BinaryType, Seq("a".getBytes(), "b".getBytes(), null, "d".getBytes())) + } + + test("get multiple") { + def check(dt: DataType, data: Seq[Any]): Unit = { + val schema = new StructType().add("value", dt, nullable = false) + val writer = ArrowWriter.create(schema) + assert(writer.schema === schema) + + data.foreach { datum => + writer.write(InternalRow(datum)) + } + writer.finish() + + val reader = new ArrowColumnVector(writer.root.getFieldVectors().get(0)) + val values = dt match { + case BooleanType => reader.getBooleans(0, data.size) + case ByteType => reader.getBytes(0, data.size) + case ShortType => reader.getShorts(0, data.size) + case IntegerType => reader.getInts(0, data.size) + case LongType => reader.getLongs(0, data.size) + case FloatType => reader.getFloats(0, data.size) + case DoubleType => reader.getDoubles(0, data.size) + } + assert(values === data) + + writer.root.close() + } + check(BooleanType, Seq(true, false)) + check(ByteType, (0 until 10).map(_.toByte)) + check(ShortType, (0 until 10).map(_.toShort)) + check(IntegerType, (0 until 10)) + check(LongType, (0 until 10).map(_.toLong)) + check(FloatType, (0 until 10).map(_.toFloat)) + check(DoubleType, (0 until 10).map(_.toDouble)) + } + + test("array") { + val schema = new StructType() + .add("arr", ArrayType(IntegerType, containsNull = true), nullable = true) + val writer = ArrowWriter.create(schema) + assert(writer.schema === schema) + + writer.write(InternalRow(ArrayData.toArrayData(Array(1, 2, 3)))) + writer.write(InternalRow(ArrayData.toArrayData(Array(4, 5)))) + writer.write(InternalRow(null)) + writer.write(InternalRow(ArrayData.toArrayData(Array.empty[Int]))) + writer.write(InternalRow(ArrayData.toArrayData(Array(6, null, 8)))) + writer.finish() + + val reader = new ArrowColumnVector(writer.root.getFieldVectors().get(0)) + + val array0 = reader.getArray(0) + assert(array0.numElements() === 3) + assert(array0.getInt(0) === 1) + assert(array0.getInt(1) === 2) + assert(array0.getInt(2) === 3) + + val array1 = reader.getArray(1) + assert(array1.numElements() === 2) + assert(array1.getInt(0) === 4) + assert(array1.getInt(1) === 5) + + assert(reader.isNullAt(2)) + + val array3 = reader.getArray(3) + assert(array3.numElements() === 0) + + val array4 = reader.getArray(4) + assert(array4.numElements() === 3) + assert(array4.getInt(0) === 6) + assert(array4.isNullAt(1)) + assert(array4.getInt(2) === 8) + + writer.root.close() + } + + test("nested array") { + val schema = new StructType().add("nested", ArrayType(ArrayType(IntegerType))) + val writer = ArrowWriter.create(schema) + assert(writer.schema === schema) + + writer.write(InternalRow(ArrayData.toArrayData(Array( + ArrayData.toArrayData(Array(1, 2, 3)), + ArrayData.toArrayData(Array(4, 5)), + null, + ArrayData.toArrayData(Array.empty[Int]), + ArrayData.toArrayData(Array(6, null, 8)))))) + writer.write(InternalRow(null)) + writer.write(InternalRow(ArrayData.toArrayData(Array.empty))) + writer.finish() + + val reader = new ArrowColumnVector(writer.root.getFieldVectors().get(0)) + + val array0 = reader.getArray(0) + assert(array0.numElements() === 5) + + val array00 = array0.getArray(0) + assert(array00.numElements() === 3) + assert(array00.getInt(0) === 1) + assert(array00.getInt(1) === 2) + assert(array00.getInt(2) === 3) + + val array01 = array0.getArray(1) + assert(array01.numElements() === 2) + assert(array01.getInt(0) === 4) + assert(array01.getInt(1) === 5) + + assert(array0.isNullAt(2)) + + val array03 = array0.getArray(3) + assert(array03.numElements() === 0) + + val array04 = array0.getArray(4) + assert(array04.numElements() === 3) + assert(array04.getInt(0) === 6) + assert(array04.isNullAt(1)) + assert(array04.getInt(2) === 8) + + assert(reader.isNullAt(1)) + + val array2 = reader.getArray(2) + assert(array2.numElements() === 0) + + writer.root.close() + } + + test("struct") { + val schema = new StructType() + .add("struct", new StructType().add("i", IntegerType).add("str", StringType)) + val writer = ArrowWriter.create(schema) + assert(writer.schema === schema) + + writer.write(InternalRow(InternalRow(1, UTF8String.fromString("str1")))) + writer.write(InternalRow(InternalRow(null, null))) + writer.write(InternalRow(null)) + writer.write(InternalRow(InternalRow(4, null))) + writer.write(InternalRow(InternalRow(null, UTF8String.fromString("str5")))) + writer.finish() + + val reader = new ArrowColumnVector(writer.root.getFieldVectors().get(0)) + + val struct0 = reader.getStruct(0, 2) + assert(struct0.getInt(0) === 1) + assert(struct0.getUTF8String(1) === UTF8String.fromString("str1")) + + val struct1 = reader.getStruct(1, 2) + assert(struct1.isNullAt(0)) + assert(struct1.isNullAt(1)) + + assert(reader.isNullAt(2)) + + val struct3 = reader.getStruct(3, 2) + assert(struct3.getInt(0) === 4) + assert(struct3.isNullAt(1)) + + val struct4 = reader.getStruct(4, 2) + assert(struct4.isNullAt(0)) + assert(struct4.getUTF8String(1) === UTF8String.fromString("str5")) + + writer.root.close() + } + + test("nested struct") { + val schema = new StructType().add("struct", + new StructType().add("nested", new StructType().add("i", IntegerType).add("str", StringType))) + val writer = ArrowWriter.create(schema) + assert(writer.schema === schema) + + writer.write(InternalRow(InternalRow(InternalRow(1, UTF8String.fromString("str1"))))) + writer.write(InternalRow(InternalRow(InternalRow(null, null)))) + writer.write(InternalRow(InternalRow(null))) + writer.write(InternalRow(null)) + writer.finish() + + val reader = new ArrowColumnVector(writer.root.getFieldVectors().get(0)) + + val struct00 = reader.getStruct(0, 1).getStruct(0, 2) + assert(struct00.getInt(0) === 1) + assert(struct00.getUTF8String(1) === UTF8String.fromString("str1")) + + val struct10 = reader.getStruct(1, 1).getStruct(0, 2) + assert(struct10.isNullAt(0)) + assert(struct10.isNullAt(1)) + + val struct2 = reader.getStruct(2, 1) + assert(struct2.isNullAt(0)) + + assert(reader.isNullAt(3)) + + writer.root.close() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala new file mode 100644 index 000000000000..691fa9ac5e1e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -0,0 +1,651 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import java.util.HashMap + +import org.apache.spark.SparkConf +import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.execution.joins.LongToUnsafeRowMap +import org.apache.spark.sql.execution.vectorized.AggregateHashMap +import org.apache.spark.sql.types.{LongType, StructType} +import org.apache.spark.unsafe.Platform +import org.apache.spark.unsafe.hash.Murmur3_x86_32 +import org.apache.spark.unsafe.map.BytesToBytesMap +import org.apache.spark.util.Benchmark + +/** + * Benchmark to measure performance for aggregate primitives. + * To run this: + * build/sbt "sql/test-only *benchmark.AggregateBenchmark" + * + * Benchmarks in this file are skipped in normal builds. + */ +class AggregateBenchmark extends BenchmarkBase { + + ignore("aggregate without grouping") { + val N = 500L << 22 + val benchmark = new Benchmark("agg without grouping", N) + runBenchmark("agg w/o group", N) { + sparkSession.range(N).selectExpr("sum(id)").collect() + } + /* + agg w/o group: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + agg w/o group wholestage off 30136 / 31885 69.6 14.4 1.0X + agg w/o group wholestage on 1851 / 1860 1132.9 0.9 16.3X + */ + } + + ignore("stat functions") { + val N = 100L << 20 + + runBenchmark("stddev", N) { + sparkSession.range(N).groupBy().agg("id" -> "stddev").collect() + } + + runBenchmark("kurtosis", N) { + sparkSession.range(N).groupBy().agg("id" -> "kurtosis").collect() + } + + /* + Using ImperativeAggregate (as implemented in Spark 1.6): + + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + stddev: Avg Time(ms) Avg Rate(M/s) Relative Rate + ------------------------------------------------------------------------------- + stddev w/o codegen 2019.04 10.39 1.00 X + stddev w codegen 2097.29 10.00 0.96 X + kurtosis w/o codegen 2108.99 9.94 0.96 X + kurtosis w codegen 2090.69 10.03 0.97 X + + Using DeclarativeAggregate: + + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + stddev: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + stddev codegen=false 5630 / 5776 18.0 55.6 1.0X + stddev codegen=true 1259 / 1314 83.0 12.0 4.5X + + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + kurtosis: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + kurtosis codegen=false 14847 / 15084 7.0 142.9 1.0X + kurtosis codegen=true 1652 / 2124 63.0 15.9 9.0X + */ + } + + ignore("aggregate with linear keys") { + val N = 20 << 22 + + val benchmark = new Benchmark("Aggregate w keys", N) + def f(): Unit = { + sparkSession.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().collect() + } + + benchmark.addCase(s"codegen = F", numIters = 2) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") + f() + } + + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "false") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + f() + } + + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + f() + } + + benchmark.run() + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + codegen = F 6619 / 6780 12.7 78.9 1.0X + codegen = T hashmap = F 3935 / 4059 21.3 46.9 1.7X + codegen = T hashmap = T 897 / 971 93.5 10.7 7.4X + */ + } + + ignore("aggregate with randomized keys") { + val N = 20 << 22 + + val benchmark = new Benchmark("Aggregate w keys", N) + sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k") + .createOrReplaceTempView("test") + + def f(): Unit = sparkSession.sql("select k, k, sum(id) from test group by k, k").collect() + + benchmark.addCase(s"codegen = F", numIters = 2) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) + f() + } + + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "false") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + f() + } + + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + f() + } + + benchmark.run() + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + codegen = F 7445 / 7517 11.3 88.7 1.0X + codegen = T hashmap = F 4672 / 4703 18.0 55.7 1.6X + codegen = T hashmap = T 1764 / 1958 47.6 21.0 4.2X + */ + } + + ignore("aggregate with string key") { + val N = 20 << 20 + + val benchmark = new Benchmark("Aggregate w string key", N) + def f(): Unit = sparkSession.range(N).selectExpr("id", "cast(id & 1023 as string) as k") + .groupBy("k").count().collect() + + benchmark.addCase(s"codegen = F", numIters = 2) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") + f() + } + + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "false") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + f() + } + + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + f() + } + + benchmark.run() + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + Aggregate w string key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + codegen = F 3307 / 3376 6.3 157.7 1.0X + codegen = T hashmap = F 2364 / 2471 8.9 112.7 1.4X + codegen = T hashmap = T 1740 / 1841 12.0 83.0 1.9X + */ + } + + ignore("aggregate with decimal key") { + val N = 20 << 20 + + val benchmark = new Benchmark("Aggregate w decimal key", N) + def f(): Unit = sparkSession.range(N).selectExpr("id", "cast(id & 65535 as decimal) as k") + .groupBy("k").count().collect() + + benchmark.addCase(s"codegen = F") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") + f() + } + + benchmark.addCase(s"codegen = T hashmap = F") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "false") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + f() + } + + benchmark.addCase(s"codegen = T hashmap = T") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + f() + } + + benchmark.run() + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + Aggregate w decimal key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + codegen = F 2756 / 2817 7.6 131.4 1.0X + codegen = T hashmap = F 1580 / 1647 13.3 75.4 1.7X + codegen = T hashmap = T 641 / 662 32.7 30.6 4.3X + */ + } + + ignore("aggregate with multiple key types") { + val N = 20 << 20 + + val benchmark = new Benchmark("Aggregate w multiple keys", N) + def f(): Unit = sparkSession.range(N) + .selectExpr( + "id", + "(id & 1023) as k1", + "cast(id & 1023 as string) as k2", + "cast(id & 1023 as int) as k3", + "cast(id & 1023 as double) as k4", + "cast(id & 1023 as float) as k5", + "id > 1023 as k6") + .groupBy("k1", "k2", "k3", "k4", "k5", "k6") + .sum() + .collect() + + benchmark.addCase(s"codegen = F") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") + f() + } + + benchmark.addCase(s"codegen = T hashmap = F") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "false") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + f() + } + + benchmark.addCase(s"codegen = T hashmap = T") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + f() + } + + benchmark.run() + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + Aggregate w decimal key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + codegen = F 5885 / 6091 3.6 280.6 1.0X + codegen = T hashmap = F 3625 / 4009 5.8 172.8 1.6X + codegen = T hashmap = T 3204 / 3271 6.5 152.8 1.8X + */ + } + + ignore("max function length of wholestagecodegen") { + val N = 20 << 15 + + val benchmark = new Benchmark("max function length of wholestagecodegen", N) + def f(): Unit = sparkSession.range(N) + .selectExpr( + "id", + "(id & 1023) as k1", + "cast(id & 1023 as double) as k2", + "cast(id & 1023 as int) as k3", + "case when id > 100 and id <= 200 then 1 else 0 end as v1", + "case when id > 200 and id <= 300 then 1 else 0 end as v2", + "case when id > 300 and id <= 400 then 1 else 0 end as v3", + "case when id > 400 and id <= 500 then 1 else 0 end as v4", + "case when id > 500 and id <= 600 then 1 else 0 end as v5", + "case when id > 600 and id <= 700 then 1 else 0 end as v6", + "case when id > 700 and id <= 800 then 1 else 0 end as v7", + "case when id > 800 and id <= 900 then 1 else 0 end as v8", + "case when id > 900 and id <= 1000 then 1 else 0 end as v9", + "case when id > 1000 and id <= 1100 then 1 else 0 end as v10", + "case when id > 1100 and id <= 1200 then 1 else 0 end as v11", + "case when id > 1200 and id <= 1300 then 1 else 0 end as v12", + "case when id > 1300 and id <= 1400 then 1 else 0 end as v13", + "case when id > 1400 and id <= 1500 then 1 else 0 end as v14", + "case when id > 1500 and id <= 1600 then 1 else 0 end as v15", + "case when id > 1600 and id <= 1700 then 1 else 0 end as v16", + "case when id > 1700 and id <= 1800 then 1 else 0 end as v17", + "case when id > 1800 and id <= 1900 then 1 else 0 end as v18") + .groupBy("k1", "k2", "k3") + .sum() + .collect() + + benchmark.addCase(s"codegen = F") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") + f() + } + + benchmark.addCase(s"codegen = T maxLinesPerFunction = 10000") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.maxLinesPerFunction", "10000") + f() + } + + benchmark.addCase(s"codegen = T maxLinesPerFunction = 1500") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.maxLinesPerFunction", "1500") + f() + } + + benchmark.run() + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_111-b14 on Windows 7 6.1 + Intel64 Family 6 Model 58 Stepping 9, GenuineIntel + max function length of wholestagecodegen: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ---------------------------------------------------------------------------------------------- + codegen = F 462 / 533 1.4 704.4 1.0X + codegen = T maxLinesPerFunction = 10000 3444 / 3447 0.2 5255.3 0.1X + codegen = T maxLinesPerFunction = 1500 447 / 478 1.5 682.1 1.0X + */ + } + + + ignore("cube") { + val N = 5 << 20 + + runBenchmark("cube", N) { + sparkSession.range(N).selectExpr("id", "id % 1000 as k1", "id & 256 as k2") + .cube("k1", "k2").sum("id").collect() + } + + /** + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + cube: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + cube codegen=false 3188 / 3392 1.6 608.2 1.0X + cube codegen=true 1239 / 1394 4.2 236.3 2.6X + */ + } + + ignore("hash and BytesToBytesMap") { + val N = 20 << 20 + + val benchmark = new Benchmark("BytesToBytesMap", N) + + benchmark.addCase("UnsafeRowhash") { iter => + var i = 0 + val keyBytes = new Array[Byte](16) + val key = new UnsafeRow(1) + key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) + var s = 0 + while (i < N) { + key.setInt(0, i % 1000) + val h = Murmur3_x86_32.hashUnsafeWords( + key.getBaseObject, key.getBaseOffset, key.getSizeInBytes, 42) + s += h + i += 1 + } + } + + benchmark.addCase("murmur3 hash") { iter => + var i = 0 + val keyBytes = new Array[Byte](16) + val key = new UnsafeRow(1) + key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) + var p = 524283 + var s = 0 + while (i < N) { + var h = Murmur3_x86_32.hashLong(i, 42) + key.setInt(0, h) + s += h + i += 1 + } + } + + benchmark.addCase("fast hash") { iter => + var i = 0 + val keyBytes = new Array[Byte](16) + val key = new UnsafeRow(1) + key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) + var p = 524283 + var s = 0 + while (i < N) { + var h = i % p + if (h < 0) { + h += p + } + key.setInt(0, h) + s += h + i += 1 + } + } + + benchmark.addCase("arrayEqual") { iter => + var i = 0 + val keyBytes = new Array[Byte](16) + val valueBytes = new Array[Byte](16) + val key = new UnsafeRow(1) + key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) + val value = new UnsafeRow(1) + value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) + value.setInt(0, 555) + var s = 0 + while (i < N) { + key.setInt(0, i % 1000) + if (key.equals(value)) { + s += 1 + } + i += 1 + } + } + + benchmark.addCase("Java HashMap (Long)") { iter => + var i = 0 + val keyBytes = new Array[Byte](16) + val valueBytes = new Array[Byte](16) + val value = new UnsafeRow(1) + value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) + value.setInt(0, 555) + val map = new HashMap[Long, UnsafeRow]() + while (i < 65536) { + value.setInt(0, i) + map.put(i.toLong, value) + i += 1 + } + var s = 0 + i = 0 + while (i < N) { + if (map.get(i % 100000) != null) { + s += 1 + } + i += 1 + } + } + + benchmark.addCase("Java HashMap (two ints) ") { iter => + var i = 0 + val valueBytes = new Array[Byte](16) + val value = new UnsafeRow(1) + value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) + value.setInt(0, 555) + val map = new HashMap[Long, UnsafeRow]() + while (i < 65536) { + value.setInt(0, i) + val key = (i.toLong << 32) + Integer.rotateRight(i, 15) + map.put(key, value) + i += 1 + } + var s = 0 + i = 0 + while (i < N) { + val key = ((i & 100000).toLong << 32) + Integer.rotateRight(i & 100000, 15) + if (map.get(key) != null) { + s += 1 + } + i += 1 + } + } + + benchmark.addCase("Java HashMap (UnsafeRow)") { iter => + var i = 0 + val keyBytes = new Array[Byte](16) + val valueBytes = new Array[Byte](16) + val key = new UnsafeRow(1) + key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) + val value = new UnsafeRow(1) + value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) + value.setInt(0, 555) + val map = new HashMap[UnsafeRow, UnsafeRow]() + while (i < 65536) { + key.setInt(0, i) + value.setInt(0, i) + map.put(key, value.copy()) + i += 1 + } + var s = 0 + i = 0 + while (i < N) { + key.setInt(0, i % 100000) + if (map.get(key) != null) { + s += 1 + } + i += 1 + } + } + + Seq(false, true).foreach { optimized => + benchmark.addCase(s"LongToUnsafeRowMap (opt=$optimized)") { iter => + var i = 0 + val valueBytes = new Array[Byte](16) + val value = new UnsafeRow(1) + value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) + value.setInt(0, 555) + val taskMemoryManager = new TaskMemoryManager( + new StaticMemoryManager( + new SparkConf().set("spark.memory.offHeap.enabled", "false"), + Long.MaxValue, + Long.MaxValue, + 1), + 0) + val map = new LongToUnsafeRowMap(taskMemoryManager, 64) + while (i < 65536) { + value.setInt(0, i) + val key = i % 100000 + map.append(key, value) + i += 1 + } + if (optimized) { + map.optimize() + } + var s = 0 + i = 0 + while (i < N) { + val key = i % 100000 + if (map.getValue(key, value) != null) { + s += 1 + } + i += 1 + } + } + } + + Seq("off", "on").foreach { heap => + benchmark.addCase(s"BytesToBytesMap ($heap Heap)") { iter => + val taskMemoryManager = new TaskMemoryManager( + new StaticMemoryManager( + new SparkConf().set("spark.memory.offHeap.enabled", s"${heap == "off"}") + .set("spark.memory.offHeap.size", "102400000"), + Long.MaxValue, + Long.MaxValue, + 1), + 0) + val map = new BytesToBytesMap(taskMemoryManager, 1024, 64L<<20) + val keyBytes = new Array[Byte](16) + val valueBytes = new Array[Byte](16) + val key = new UnsafeRow(1) + key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) + val value = new UnsafeRow(1) + value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) + var i = 0 + val numKeys = 65536 + while (i < numKeys) { + key.setInt(0, i % 65536) + val loc = map.lookup(key.getBaseObject, key.getBaseOffset, key.getSizeInBytes, + Murmur3_x86_32.hashLong(i % 65536, 42)) + if (!loc.isDefined) { + loc.append(key.getBaseObject, key.getBaseOffset, key.getSizeInBytes, + value.getBaseObject, value.getBaseOffset, value.getSizeInBytes) + } + i += 1 + } + i = 0 + var s = 0 + while (i < N) { + key.setInt(0, i % 100000) + val loc = map.lookup(key.getBaseObject, key.getBaseOffset, key.getSizeInBytes, + Murmur3_x86_32.hashLong(i % 100000, 42)) + if (loc.isDefined) { + s += 1 + } + i += 1 + } + } + } + + benchmark.addCase("Aggregate HashMap") { iter => + var i = 0 + val numKeys = 65536 + val schema = new StructType() + .add("key", LongType) + .add("value", LongType) + val map = new AggregateHashMap(schema) + while (i < numKeys) { + val row = map.findOrInsert(i.toLong) + row.setLong(1, row.getLong(1) + 1) + i += 1 + } + var s = 0 + i = 0 + while (i < N) { + if (map.find(i % 100000) != -1) { + s += 1 + } + i += 1 + } + } + + /* + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + BytesToBytesMap: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + UnsafeRow hash 267 / 284 78.4 12.8 1.0X + murmur3 hash 102 / 129 205.5 4.9 2.6X + fast hash 79 / 96 263.8 3.8 3.4X + arrayEqual 164 / 172 128.2 7.8 1.6X + Java HashMap (Long) 321 / 399 65.4 15.3 0.8X + Java HashMap (two ints) 328 / 363 63.9 15.7 0.8X + Java HashMap (UnsafeRow) 1140 / 1200 18.4 54.3 0.2X + LongToUnsafeRowMap (opt=false) 378 / 400 55.5 18.0 0.7X + LongToUnsafeRowMap (opt=true) 144 / 152 145.2 6.9 1.9X + BytesToBytesMap (off Heap) 1300 / 1616 16.1 62.0 0.2X + BytesToBytesMap (on Heap) 1165 / 1202 18.0 55.5 0.2X + Aggregate HashMap 121 / 131 173.3 5.8 2.2X + */ + benchmark.run() + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala new file mode 100644 index 000000000000..c99a5aec1cd6 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.SparkSession +import org.apache.spark.util.Benchmark + +/** + * Common base trait for micro benchmarks that are supposed to run standalone (i.e. not together + * with other test suites). + */ +private[benchmark] trait BenchmarkBase extends SparkFunSuite { + + lazy val sparkSession = SparkSession.builder + .master("local[1]") + .appName("microbenchmark") + .config("spark.sql.shuffle.partitions", 1) + .config("spark.sql.autoBroadcastJoinThreshold", 1) + .getOrCreate() + + /** Runs function `f` with whole stage codegen on and off. */ + def runBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { + val benchmark = new Benchmark(name, cardinality) + + benchmark.addCase(s"$name wholestage off", numIters = 2) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) + f + } + + benchmark.addCase(s"$name wholestage on", numIters = 5) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + f + } + + benchmark.run() + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkWideTable.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkWideTable.scala new file mode 100644 index 000000000000..9dcaca0ca93e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkWideTable.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.util.Benchmark + + +/** + * Benchmark to measure performance for wide table. + * To run this: + * build/sbt "sql/test-only *benchmark.BenchmarkWideTable" + * + * Benchmarks in this file are skipped in normal builds. + */ +class BenchmarkWideTable extends BenchmarkBase { + + ignore("project on wide table") { + val N = 1 << 20 + val df = sparkSession.range(N) + val columns = (0 until 400).map{ i => s"id as id$i"} + val benchmark = new Benchmark("projection on wide table", N) + benchmark.addCase("wide table", numIters = 5) { iter => + df.selectExpr(columns : _*).queryExecution.toRdd.count() + } + benchmark.run() + + /** + * Here are some numbers with different split threshold: + * + * Split threshold methods Rate(M/s) Per Row(ns) + * 10 400 0.4 2279 + * 100 200 0.6 1554 + * 1k 37 0.9 1116 + * 8k 5 0.5 2025 + * 64k 1 0.0 21649 + */ + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala new file mode 100644 index 000000000000..5a25d7230837 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala @@ -0,0 +1,247 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.IntegerType + +/** + * Benchmark to measure performance for aggregate primitives. + * To run this: + * build/sbt "sql/test-only *benchmark.JoinBenchmark" + * + * Benchmarks in this file are skipped in normal builds. + */ +class JoinBenchmark extends BenchmarkBase { + + ignore("broadcast hash join, long key") { + val N = 20 << 20 + val M = 1 << 16 + + val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) + runBenchmark("Join w long", N) { + val df = sparkSession.range(N).join(dim, (col("id") % M) === col("k")) + assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) + df.count() + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + Join w long codegen=false 3002 / 3262 7.0 143.2 1.0X + Join w long codegen=true 321 / 371 65.3 15.3 9.3X + */ + } + + ignore("broadcast hash join, long key with duplicates") { + val N = 20 << 20 + val M = 1 << 16 + + val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) + runBenchmark("Join w long duplicated", N) { + val dim = broadcast(sparkSession.range(M).selectExpr("cast(id/10 as long) as k")) + val df = sparkSession.range(N).join(dim, (col("id") % M) === col("k")) + assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) + df.count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *Join w long duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *Join w long duplicated codegen=false 3446 / 3478 6.1 164.3 1.0X + *Join w long duplicated codegen=true 322 / 351 65.2 15.3 10.7X + */ + } + + ignore("broadcast hash join, two int key") { + val N = 20 << 20 + val M = 1 << 16 + val dim2 = broadcast(sparkSession.range(M) + .selectExpr("cast(id as int) as k1", "cast(id as int) as k2", "cast(id as string) as v")) + + runBenchmark("Join w 2 ints", N) { + val df = sparkSession.range(N).join(dim2, + (col("id") % M).cast(IntegerType) === col("k1") + && (col("id") % M).cast(IntegerType) === col("k2")) + assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) + df.count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *Join w 2 ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *Join w 2 ints codegen=false 4426 / 4501 4.7 211.1 1.0X + *Join w 2 ints codegen=true 791 / 818 26.5 37.7 5.6X + */ + } + + ignore("broadcast hash join, two long key") { + val N = 20 << 20 + val M = 1 << 16 + val dim3 = broadcast(sparkSession.range(M) + .selectExpr("id as k1", "id as k2", "cast(id as string) as v")) + + runBenchmark("Join w 2 longs", N) { + val df = sparkSession.range(N).join(dim3, + (col("id") % M) === col("k1") && (col("id") % M) === col("k2")) + assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) + df.count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *Join w 2 longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *Join w 2 longs codegen=false 5905 / 6123 3.6 281.6 1.0X + *Join w 2 longs codegen=true 2230 / 2529 9.4 106.3 2.6X + */ + } + + ignore("broadcast hash join, two long key with duplicates") { + val N = 20 << 20 + val M = 1 << 16 + val dim4 = broadcast(sparkSession.range(M) + .selectExpr("cast(id/10 as long) as k1", "cast(id/10 as long) as k2")) + + runBenchmark("Join w 2 longs duplicated", N) { + val df = sparkSession.range(N).join(dim4, + (col("id") bitwiseAND M) === col("k1") && (col("id") bitwiseAND M) === col("k2")) + assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) + df.count() + } + + /* + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *Join w 2 longs duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *Join w 2 longs duplicated codegen=false 6420 / 6587 3.3 306.1 1.0X + *Join w 2 longs duplicated codegen=true 2080 / 2139 10.1 99.2 3.1X + */ + } + + ignore("broadcast hash join, outer join long key") { + val N = 20 << 20 + val M = 1 << 16 + val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) + runBenchmark("outer join w long", N) { + val df = sparkSession.range(N).join(dim, (col("id") % M) === col("k"), "left") + assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) + df.count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *outer join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *outer join w long codegen=false 3055 / 3189 6.9 145.7 1.0X + *outer join w long codegen=true 261 / 276 80.5 12.4 11.7X + */ + } + + ignore("broadcast hash join, semi join long key") { + val N = 20 << 20 + val M = 1 << 16 + val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) + runBenchmark("semi join w long", N) { + val df = sparkSession.range(N).join(dim, (col("id") % M) === col("k"), "leftsemi") + assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) + df.count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *semi join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *semi join w long codegen=false 1912 / 1990 11.0 91.2 1.0X + *semi join w long codegen=true 237 / 244 88.3 11.3 8.1X + */ + } + + ignore("sort merge join") { + val N = 2 << 20 + runBenchmark("merge join", N) { + val df1 = sparkSession.range(N).selectExpr(s"id * 2 as k1") + val df2 = sparkSession.range(N).selectExpr(s"id * 3 as k2") + val df = df1.join(df2, col("k1") === col("k2")) + assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[SortMergeJoinExec]).isDefined) + df.count() + } + + /* + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *merge join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *merge join codegen=false 1588 / 1880 1.3 757.1 1.0X + *merge join codegen=true 1477 / 1531 1.4 704.2 1.1X + */ + } + + ignore("sort merge join with duplicates") { + val N = 2 << 20 + runBenchmark("sort merge join", N) { + val df1 = sparkSession.range(N) + .selectExpr(s"(id * 15485863) % ${N*10} as k1") + val df2 = sparkSession.range(N) + .selectExpr(s"(id * 15485867) % ${N*10} as k2") + val df = df1.join(df2, col("k1") === col("k2")) + assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[SortMergeJoinExec]).isDefined) + df.count() + } + + /* + *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + *sort merge join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *sort merge join codegen=false 3626 / 3667 0.6 1728.9 1.0X + *sort merge join codegen=true 3405 / 3438 0.6 1623.8 1.1X + */ + } + + ignore("shuffle hash join") { + val N = 4 << 20 + sparkSession.conf.set("spark.sql.shuffle.partitions", "2") + sparkSession.conf.set("spark.sql.autoBroadcastJoinThreshold", "10000000") + sparkSession.conf.set("spark.sql.join.preferSortMergeJoin", "false") + runBenchmark("shuffle hash join", N) { + val df1 = sparkSession.range(N).selectExpr(s"id as k1") + val df2 = sparkSession.range(N / 3).selectExpr(s"id * 3 as k2") + val df = df1.join(df2, col("k1") === col("k2")) + assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[ShuffledHashJoinExec]).isDefined) + df.count() + } + + /* + *Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Windows 7 6.1 + *Intel64 Family 6 Model 94 Stepping 3, GenuineIntel + *shuffle hash join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + *------------------------------------------------------------------------------------------- + *shuffle hash join codegen=false 2005 / 2010 2.1 478.0 1.0X + *shuffle hash join codegen=true 1773 / 1792 2.4 422.7 1.1X + */ + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala new file mode 100644 index 000000000000..01773c238b0d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.util.Benchmark + +/** + * Benchmark to measure whole stage codegen performance. + * To run this: + * build/sbt "sql/test-only *benchmark.MiscBenchmark" + * + * Benchmarks in this file are skipped in normal builds. + */ +class MiscBenchmark extends BenchmarkBase { + + ignore("filter & aggregate without group") { + val N = 500L << 22 + runBenchmark("range/filter/sum", N) { + sparkSession.range(N).filter("(id & 1) = 1").groupBy().sum().collect() + } + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + range/filter/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + range/filter/sum codegen=false 30663 / 31216 68.4 14.6 1.0X + range/filter/sum codegen=true 2399 / 2409 874.1 1.1 12.8X + */ + } + + ignore("range/limit/sum") { + val N = 500L << 20 + runBenchmark("range/limit/sum", N) { + sparkSession.range(N).limit(1000000).groupBy().sum().collect() + } + /* + Westmere E56xx/L56xx/X56xx (Nehalem-C) + range/limit/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + range/limit/sum codegen=false 609 / 672 861.6 1.2 1.0X + range/limit/sum codegen=true 561 / 621 935.3 1.1 1.1X + */ + } + + ignore("sample") { + val N = 500 << 18 + runBenchmark("sample with replacement", N) { + sparkSession.range(N).sample(withReplacement = true, 0.01).groupBy().sum().collect() + } + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + sample with replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + sample with replacement codegen=false 7073 / 7227 18.5 54.0 1.0X + sample with replacement codegen=true 5199 / 5203 25.2 39.7 1.4X + */ + + runBenchmark("sample without replacement", N) { + sparkSession.range(N).sample(withReplacement = false, 0.01).groupBy().sum().collect() + } + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + sample without replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + sample without replacement codegen=false 1508 / 1529 86.9 11.5 1.0X + sample without replacement codegen=true 644 / 662 203.5 4.9 2.3X + */ + } + + ignore("collect") { + val N = 1 << 20 + + val benchmark = new Benchmark("collect", N) + benchmark.addCase("collect 1 million") { iter => + sparkSession.range(N).collect() + } + benchmark.addCase("collect 2 millions") { iter => + sparkSession.range(N * 2).collect() + } + benchmark.addCase("collect 4 millions") { iter => + sparkSession.range(N * 4).collect() + } + benchmark.run() + + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + collect: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + collect 1 million 439 / 654 2.4 418.7 1.0X + collect 2 millions 961 / 1907 1.1 916.4 0.5X + collect 4 millions 3193 / 3895 0.3 3044.7 0.1X + */ + } + + ignore("collect limit") { + val N = 1 << 20 + + val benchmark = new Benchmark("collect limit", N) + benchmark.addCase("collect limit 1 million") { iter => + sparkSession.range(N * 4).limit(N).collect() + } + benchmark.addCase("collect limit 2 millions") { iter => + sparkSession.range(N * 4).limit(N * 2).collect() + } + benchmark.run() + + /* + model name : Westmere E56xx/L56xx/X56xx (Nehalem-C) + collect limit: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + collect limit 1 million 833 / 1284 1.3 794.4 1.0X + collect limit 2 millions 3348 / 4005 0.3 3193.3 0.2X + */ + } + + ignore("generate explode") { + val N = 1 << 24 + runBenchmark("generate explode array", N) { + val df = sparkSession.range(N).selectExpr( + "id as key", + "array(rand(), rand(), rand(), rand(), rand()) as values") + df.selectExpr("key", "explode(values) value").count() + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 + Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz + + generate explode array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + generate explode array wholestage off 6920 / 7129 2.4 412.5 1.0X + generate explode array wholestage on 623 / 646 26.9 37.1 11.1X + */ + + runBenchmark("generate explode map", N) { + val df = sparkSession.range(N).selectExpr( + "id as key", + "map('a', rand(), 'b', rand(), 'c', rand(), 'd', rand(), 'e', rand()) pairs") + df.selectExpr("key", "explode(pairs) as (k, v)").count() + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 + Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz + + generate explode map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + generate explode map wholestage off 11978 / 11993 1.4 714.0 1.0X + generate explode map wholestage on 866 / 919 19.4 51.6 13.8X + */ + + runBenchmark("generate posexplode array", N) { + val df = sparkSession.range(N).selectExpr( + "id as key", + "array(rand(), rand(), rand(), rand(), rand()) as values") + df.selectExpr("key", "posexplode(values) as (idx, value)").count() + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 + Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz + + generate posexplode array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + generate posexplode array wholestage off 7502 / 7513 2.2 447.1 1.0X + generate posexplode array wholestage on 617 / 623 27.2 36.8 12.2X + */ + + runBenchmark("generate inline array", N) { + val df = sparkSession.range(N).selectExpr( + "id as key", + "array((rand(), rand()), (rand(), rand()), (rand(), 0.0d)) as values") + df.selectExpr("key", "inline(values) as (r1, r2)").count() + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 + Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz + + generate inline array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + generate inline array wholestage off 6901 / 6928 2.4 411.3 1.0X + generate inline array wholestage on 1001 / 1010 16.8 59.7 6.9X + */ + } + + ignore("generate regular generator") { + val N = 1 << 24 + runBenchmark("generate stack", N) { + val df = sparkSession.range(N).selectExpr( + "id as key", + "id % 2 as t1", + "id % 3 as t2", + "id % 5 as t3", + "id % 7 as t4", + "id % 13 as t5") + df.selectExpr("key", "stack(4, t1, t2, t3, t4, t5)").count() + } + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 + Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz + + generate stack: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + generate stack wholestage off 12953 / 13070 1.3 772.1 1.0X + generate stack wholestage on 836 / 847 20.1 49.8 15.5X + */ + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala new file mode 100644 index 000000000000..e7c8f2717fd7 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import scala.concurrent.duration._ + +import org.apache.spark.SparkConf +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.util.Benchmark + +/** + * Benchmark [[PrimitiveArray]] for DataFrame and Dataset program using primitive array + * To run this: + * 1. replace ignore(...) with test(...) + * 2. build/sbt "sql/test-only *benchmark.PrimitiveArrayBenchmark" + * + * Benchmarks in this file are skipped in normal builds. + */ +class PrimitiveArrayBenchmark extends BenchmarkBase { + + def writeDatasetArray(iters: Int): Unit = { + import sparkSession.implicits._ + + val count = 1024 * 1024 * 2 + + val sc = sparkSession.sparkContext + val primitiveIntArray = Array.fill[Int](count)(65535) + val dsInt = sc.parallelize(Seq(primitiveIntArray), 1).toDS + dsInt.count // force to build dataset + val intArray = { i: Int => + var n = 0 + var len = 0 + while (n < iters) { + len += dsInt.map(e => e).queryExecution.toRdd.collect.length + n += 1 + } + } + val primitiveDoubleArray = Array.fill[Double](count)(65535.0) + val dsDouble = sc.parallelize(Seq(primitiveDoubleArray), 1).toDS + dsDouble.count // force to build dataset + val doubleArray = { i: Int => + var n = 0 + var len = 0 + while (n < iters) { + len += dsDouble.map(e => e).queryExecution.toRdd.collect.length + n += 1 + } + } + + val benchmark = new Benchmark("Write an array in Dataset", count * iters) + benchmark.addCase("Int ")(intArray) + benchmark.addCase("Double")(doubleArray) + benchmark.run + /* + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Write an array in Dataset: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Int 352 / 401 23.8 42.0 1.0X + Double 821 / 885 10.2 97.9 0.4X + */ + } + + ignore("Write an array in Dataset") { + writeDatasetArray(4) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala new file mode 100644 index 000000000000..50ae26a3ff9d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import java.util.{Arrays, Comparator} + +import org.apache.spark.unsafe.array.LongArray +import org.apache.spark.unsafe.memory.MemoryBlock +import org.apache.spark.util.Benchmark +import org.apache.spark.util.collection.Sorter +import org.apache.spark.util.collection.unsafe.sort._ +import org.apache.spark.util.random.XORShiftRandom + +/** + * Benchmark to measure performance for aggregate primitives. + * To run this: + * build/sbt "sql/test-only *benchmark.SortBenchmark" + * + * Benchmarks in this file are skipped in normal builds. + */ +class SortBenchmark extends BenchmarkBase { + + private def referenceKeyPrefixSort(buf: LongArray, lo: Int, hi: Int, refCmp: PrefixComparator) { + val sortBuffer = new LongArray(MemoryBlock.fromLongArray(new Array[Long](buf.size().toInt))) + new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort( + buf, lo, hi, new Comparator[RecordPointerAndKeyPrefix] { + override def compare( + r1: RecordPointerAndKeyPrefix, + r2: RecordPointerAndKeyPrefix): Int = { + refCmp.compare(r1.keyPrefix, r2.keyPrefix) + } + }) + } + + private def generateKeyPrefixTestData(size: Int, rand: => Long): (LongArray, LongArray) = { + val ref = Array.tabulate[Long](size * 2) { i => rand } + val extended = ref ++ Array.fill[Long](size * 2)(0) + (new LongArray(MemoryBlock.fromLongArray(ref)), + new LongArray(MemoryBlock.fromLongArray(extended))) + } + + ignore("sort") { + val size = 25000000 + val rand = new XORShiftRandom(123) + val benchmark = new Benchmark("radix sort " + size, size) + benchmark.addTimerCase("reference TimSort key prefix array") { timer => + val array = Array.tabulate[Long](size * 2) { i => rand.nextLong } + val buf = new LongArray(MemoryBlock.fromLongArray(array)) + timer.startTiming() + referenceKeyPrefixSort(buf, 0, size, PrefixComparators.BINARY) + timer.stopTiming() + } + benchmark.addTimerCase("reference Arrays.sort") { timer => + val ref = Array.tabulate[Long](size) { i => rand.nextLong } + timer.startTiming() + Arrays.sort(ref) + timer.stopTiming() + } + benchmark.addTimerCase("radix sort one byte") { timer => + val array = new Array[Long](size * 2) + var i = 0 + while (i < size) { + array(i) = rand.nextLong & 0xff + i += 1 + } + val buf = new LongArray(MemoryBlock.fromLongArray(array)) + timer.startTiming() + RadixSort.sort(buf, size, 0, 7, false, false) + timer.stopTiming() + } + benchmark.addTimerCase("radix sort two bytes") { timer => + val array = new Array[Long](size * 2) + var i = 0 + while (i < size) { + array(i) = rand.nextLong & 0xffff + i += 1 + } + val buf = new LongArray(MemoryBlock.fromLongArray(array)) + timer.startTiming() + RadixSort.sort(buf, size, 0, 7, false, false) + timer.stopTiming() + } + benchmark.addTimerCase("radix sort eight bytes") { timer => + val array = new Array[Long](size * 2) + var i = 0 + while (i < size) { + array(i) = rand.nextLong + i += 1 + } + val buf = new LongArray(MemoryBlock.fromLongArray(array)) + timer.startTiming() + RadixSort.sort(buf, size, 0, 7, false, false) + timer.stopTiming() + } + benchmark.addTimerCase("radix sort key prefix array") { timer => + val (_, buf2) = generateKeyPrefixTestData(size, rand.nextLong) + timer.startTiming() + RadixSort.sortKeyPrefixArray(buf2, 0, size, 0, 7, false, false) + timer.stopTiming() + } + benchmark.run() + + /* + Running benchmark: radix sort 25000000 + Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 3.13.0-44-generic + Intel(R) Core(TM) i7-4600U CPU @ 2.10GHz + + radix sort 25000000: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + reference TimSort key prefix array 15546 / 15859 1.6 621.9 1.0X + reference Arrays.sort 2416 / 2446 10.3 96.6 6.4X + radix sort one byte 133 / 137 188.4 5.3 117.2X + radix sort two bytes 255 / 258 98.2 10.2 61.1X + radix sort eight bytes 991 / 997 25.2 39.6 15.7X + radix sort key prefix array 1540 / 1563 16.2 61.6 10.1X + */ + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala new file mode 100644 index 000000000000..99c6df738920 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.util.Benchmark + +/** + * Benchmark to measure TPCDS query performance. + * To run this: + * spark-submit --class --data-location + */ +object TPCDSQueryBenchmark extends Logging { + val conf = + new SparkConf() + .setMaster("local[1]") + .setAppName("test-sql-context") + .set("spark.sql.parquet.compression.codec", "snappy") + .set("spark.sql.shuffle.partitions", "4") + .set("spark.driver.memory", "3g") + .set("spark.executor.memory", "3g") + .set("spark.sql.autoBroadcastJoinThreshold", (20 * 1024 * 1024).toString) + .set("spark.sql.crossJoin.enabled", "true") + + val spark = SparkSession.builder.config(conf).getOrCreate() + + val tables = Seq("catalog_page", "catalog_returns", "customer", "customer_address", + "customer_demographics", "date_dim", "household_demographics", "inventory", "item", + "promotion", "store", "store_returns", "catalog_sales", "web_sales", "store_sales", + "web_returns", "web_site", "reason", "call_center", "warehouse", "ship_mode", "income_band", + "time_dim", "web_page") + + def setupTables(dataLocation: String): Map[String, Long] = { + tables.map { tableName => + spark.read.parquet(s"$dataLocation/$tableName").createOrReplaceTempView(tableName) + tableName -> spark.table(tableName).count() + }.toMap + } + + def tpcdsAll(dataLocation: String, queries: Seq[String]): Unit = { + val tableSizes = setupTables(dataLocation) + queries.foreach { name => + val queryString = resourceToString(s"tpcds/$name.sql", + classLoader = Thread.currentThread().getContextClassLoader) + + // This is an indirect hack to estimate the size of each query's input by traversing the + // logical plan and adding up the sizes of all tables that appear in the plan. Note that this + // currently doesn't take WITH subqueries into account which might lead to fairly inaccurate + // per-row processing time for those cases. + val queryRelations = scala.collection.mutable.HashSet[String]() + spark.sql(queryString).queryExecution.logical.map { + case UnresolvedRelation(t: TableIdentifier) => + queryRelations.add(t.table) + case lp: LogicalPlan => + lp.expressions.foreach { _ foreach { + case subquery: SubqueryExpression => + subquery.plan.foreach { + case UnresolvedRelation(t: TableIdentifier) => + queryRelations.add(t.table) + case _ => + } + case _ => + } + } + case _ => + } + val numRows = queryRelations.map(tableSizes.getOrElse(_, 0L)).sum + val benchmark = new Benchmark(s"TPCDS Snappy", numRows, 5) + benchmark.addCase(name) { i => + spark.sql(queryString).collect() + } + logInfo(s"\n\n===== TPCDS QUERY BENCHMARK OUTPUT FOR $name =====\n") + benchmark.run() + logInfo(s"\n\n===== FINISHED $name =====\n") + } + } + + def main(args: Array[String]): Unit = { + val benchmarkArgs = new TPCDSQueryBenchmarkArguments(args) + + // List of all TPC-DS queries + val tpcdsQueries = Seq( + "q1", "q2", "q3", "q4", "q5", "q6", "q7", "q8", "q9", "q10", "q11", + "q12", "q13", "q14a", "q14b", "q15", "q16", "q17", "q18", "q19", "q20", + "q21", "q22", "q23a", "q23b", "q24a", "q24b", "q25", "q26", "q27", "q28", "q29", "q30", + "q31", "q32", "q33", "q34", "q35", "q36", "q37", "q38", "q39a", "q39b", "q40", + "q41", "q42", "q43", "q44", "q45", "q46", "q47", "q48", "q49", "q50", + "q51", "q52", "q53", "q54", "q55", "q56", "q57", "q58", "q59", "q60", + "q61", "q62", "q63", "q64", "q65", "q66", "q67", "q68", "q69", "q70", + "q71", "q72", "q73", "q74", "q75", "q76", "q77", "q78", "q79", "q80", + "q81", "q82", "q83", "q84", "q85", "q86", "q87", "q88", "q89", "q90", + "q91", "q92", "q93", "q94", "q95", "q96", "q97", "q98", "q99") + + // If `--query-filter` defined, filters the queries that this option selects + val queriesToRun = if (benchmarkArgs.queryFilter.nonEmpty) { + val queries = tpcdsQueries.filter { case queryName => + benchmarkArgs.queryFilter.contains(queryName) + } + if (queries.isEmpty) { + throw new RuntimeException( + s"Empty queries to run. Bad query name filter: ${benchmarkArgs.queryFilter}") + } + queries + } else { + tpcdsQueries + } + + tpcdsAll(benchmarkArgs.dataLocation, queries = queriesToRun) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmarkArguments.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmarkArguments.scala new file mode 100644 index 000000000000..184ffff94298 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmarkArguments.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import java.util.Locale + + +class TPCDSQueryBenchmarkArguments(val args: Array[String]) { + var dataLocation: String = null + var queryFilter: Set[String] = Set.empty + + parseArgs(args.toList) + validateArguments() + + private def optionMatch(optionName: String, s: String): Boolean = { + optionName == s.toLowerCase(Locale.ROOT) + } + + private def parseArgs(inputArgs: List[String]): Unit = { + var args = inputArgs + + while (args.nonEmpty) { + args match { + case optName :: value :: tail if optionMatch("--data-location", optName) => + dataLocation = value + args = tail + + case optName :: value :: tail if optionMatch("--query-filter", optName) => + queryFilter = value.toLowerCase(Locale.ROOT).split(",").map(_.trim).toSet + args = tail + + case _ => + // scalastyle:off println + System.err.println("Unknown/unsupported param " + args) + // scalastyle:on println + printUsageAndExit(1) + } + } + } + + private def printUsageAndExit(exitCode: Int): Unit = { + // scalastyle:off + System.err.println(""" + |Usage: spark-submit --class [Options] + |Options: + | --data-location Path to TPCDS data + | --query-filter Queries to filter, e.g., q3,q5,q13 + | + |------------------------------------------------------------------------------------------------------------------ + |In order to run this benchmark, please follow the instructions at + |https://github.com/databricks/spark-sql-perf/blob/master/README.md + |to generate the TPCDS data locally (preferably with a scale factor of 5 for benchmarking). + |Thereafter, the value of needs to be set to the location where the generated data is stored. + """.stripMargin) + // scalastyle:on + System.exit(exitCode) + } + + private def validateArguments(): Unit = { + if (dataLocation == null) { + // scalastyle:off println + System.err.println("Must specify a data location") + // scalastyle:on println + printUsageAndExit(-1) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala new file mode 100644 index 000000000000..6c7779b5790d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala @@ -0,0 +1,232 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import scala.util.Random + +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.expressions.{UnsafeArrayData, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, UnsafeArrayWriter} +import org.apache.spark.util.Benchmark + +/** + * Benchmark [[UnsafeArrayDataBenchmark]] for UnsafeArrayData + * To run this: + * 1. replace ignore(...) with test(...) + * 2. build/sbt "sql/test-only *benchmark.UnsafeArrayDataBenchmark" + * + * Benchmarks in this file are skipped in normal builds. + */ +class UnsafeArrayDataBenchmark extends BenchmarkBase { + + def calculateHeaderPortionInBytes(count: Int) : Int = { + /* 4 + 4 * count // Use this expression for SPARK-15962 */ + UnsafeArrayData.calculateHeaderPortionInBytes(count) + } + + def readUnsafeArray(iters: Int): Unit = { + val count = 1024 * 1024 * 16 + val rand = new Random(42) + + val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt } + val intEncoder = ExpressionEncoder[Array[Int]].resolveAndBind() + val intUnsafeArray = intEncoder.toRow(intPrimitiveArray).getArray(0) + val readIntArray = { i: Int => + var n = 0 + while (n < iters) { + val len = intUnsafeArray.numElements + var sum = 0 + var i = 0 + while (i < len) { + sum += intUnsafeArray.getInt(i) + i += 1 + } + n += 1 + } + } + + val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble } + val doubleEncoder = ExpressionEncoder[Array[Double]].resolveAndBind() + val doubleUnsafeArray = doubleEncoder.toRow(doublePrimitiveArray).getArray(0) + val readDoubleArray = { i: Int => + var n = 0 + while (n < iters) { + val len = doubleUnsafeArray.numElements + var sum = 0.0 + var i = 0 + while (i < len) { + sum += doubleUnsafeArray.getDouble(i) + i += 1 + } + n += 1 + } + } + + val benchmark = new Benchmark("Read UnsafeArrayData", count * iters) + benchmark.addCase("Int")(readIntArray) + benchmark.addCase("Double")(readDoubleArray) + benchmark.run + /* + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Read UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Int 252 / 260 666.1 1.5 1.0X + Double 281 / 292 597.7 1.7 0.9X + */ + } + + def writeUnsafeArray(iters: Int): Unit = { + val count = 1024 * 1024 * 2 + val rand = new Random(42) + + var intTotalLength: Int = 0 + val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt } + val intEncoder = ExpressionEncoder[Array[Int]].resolveAndBind() + val writeIntArray = { i: Int => + var len = 0 + var n = 0 + while (n < iters) { + len += intEncoder.toRow(intPrimitiveArray).getArray(0).numElements() + n += 1 + } + intTotalLength = len + } + + var doubleTotalLength: Int = 0 + val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble } + val doubleEncoder = ExpressionEncoder[Array[Double]].resolveAndBind() + val writeDoubleArray = { i: Int => + var len = 0 + var n = 0 + while (n < iters) { + len += doubleEncoder.toRow(doublePrimitiveArray).getArray(0).numElements() + n += 1 + } + doubleTotalLength = len + } + + val benchmark = new Benchmark("Write UnsafeArrayData", count * iters) + benchmark.addCase("Int")(writeIntArray) + benchmark.addCase("Double")(writeDoubleArray) + benchmark.run + /* + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Write UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Int 196 / 249 107.0 9.3 1.0X + Double 227 / 367 92.3 10.8 0.9X + */ + } + + def getPrimitiveArray(iters: Int): Unit = { + val count = 1024 * 1024 * 12 + val rand = new Random(42) + + var intTotalLength: Int = 0 + val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt } + val intEncoder = ExpressionEncoder[Array[Int]].resolveAndBind() + val intUnsafeArray = intEncoder.toRow(intPrimitiveArray).getArray(0) + val readIntArray = { i: Int => + var len = 0 + var n = 0 + while (n < iters) { + len += intUnsafeArray.toIntArray.length + n += 1 + } + intTotalLength = len + } + + var doubleTotalLength: Int = 0 + val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble } + val doubleEncoder = ExpressionEncoder[Array[Double]].resolveAndBind() + val doubleUnsafeArray = doubleEncoder.toRow(doublePrimitiveArray).getArray(0) + val readDoubleArray = { i: Int => + var len = 0 + var n = 0 + while (n < iters) { + len += doubleUnsafeArray.toDoubleArray.length + n += 1 + } + doubleTotalLength = len + } + + val benchmark = new Benchmark("Get primitive array from UnsafeArrayData", count * iters) + benchmark.addCase("Int")(readIntArray) + benchmark.addCase("Double")(readDoubleArray) + benchmark.run + /* + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Get primitive array from UnsafeArrayData: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Int 151 / 198 415.8 2.4 1.0X + Double 214 / 394 293.6 3.4 0.7X + */ + } + + def putPrimitiveArray(iters: Int): Unit = { + val count = 1024 * 1024 * 12 + val rand = new Random(42) + + var intTotalLen: Int = 0 + val intPrimitiveArray = Array.fill[Int](count) { rand.nextInt } + val createIntArray = { i: Int => + var len = 0 + var n = 0 + while (n < iters) { + len += UnsafeArrayData.fromPrimitiveArray(intPrimitiveArray).numElements + n += 1 + } + intTotalLen = len + } + + var doubleTotalLen: Int = 0 + val doublePrimitiveArray = Array.fill[Double](count) { rand.nextDouble } + val createDoubleArray = { i: Int => + var len = 0 + var n = 0 + while (n < iters) { + len += UnsafeArrayData.fromPrimitiveArray(doublePrimitiveArray).numElements + n += 1 + } + doubleTotalLen = len + } + + val benchmark = new Benchmark("Create UnsafeArrayData from primitive array", count * iters) + benchmark.addCase("Int")(createIntArray) + benchmark.addCase("Double")(createDoubleArray) + benchmark.run + /* + OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64 + Intel Xeon E3-12xx v2 (Ivy Bridge) + Create UnsafeArrayData from primitive array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Int 206 / 211 306.0 3.3 1.0X + Double 232 / 406 271.6 3.7 0.9X + */ + } + + ignore("Benchmark UnsafeArrayData") { + readUnsafeArray(10) + writeUnsafeArray(10) + getPrimitiveArray(5) + putPrimitiveArray(5) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala new file mode 100644 index 000000000000..a42891e55a18 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala @@ -0,0 +1,221 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.io.{File, FileOutputStream, OutputStream} + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.functions._ +import org.apache.spark.util.{Benchmark, Utils} + +/** + * Benchmark for performance with very wide and nested DataFrames. + * To run this: + * build/sbt "sql/test-only *WideSchemaBenchmark" + * + * Results will be written to "sql/core/benchmarks/WideSchemaBenchmark-results.txt". + */ +class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { + private val scaleFactor = 100000 + private val widthsToTest = Seq(1, 100, 2500) + private val depthsToTest = Seq(1, 100, 250) + assert(scaleFactor > widthsToTest.max) + + private lazy val sparkSession = SparkSession.builder + .master("local[1]") + .appName("microbenchmark") + .getOrCreate() + + import sparkSession.implicits._ + + private var tmpFiles: List[File] = Nil + private var out: OutputStream = null + + override def beforeAll() { + super.beforeAll() + out = new FileOutputStream(new File("benchmarks/WideSchemaBenchmark-results.txt")) + } + + override def afterAll() { + super.afterAll() + out.close() + } + + override def afterEach() { + super.afterEach() + for (tmpFile <- tmpFiles) { + Utils.deleteRecursively(tmpFile) + } + } + + /** + * Writes the given DataFrame to parquet at a temporary location, and returns a DataFrame + * backed by the written parquet files. + */ + private def saveAsParquet(df: DataFrame): DataFrame = { + val tmpFile = File.createTempFile("WideSchemaBenchmark", "tmp") + tmpFiles ::= tmpFile + tmpFile.delete() + df.write.parquet(tmpFile.getAbsolutePath) + assert(tmpFile.isDirectory()) + sparkSession.read.parquet(tmpFile.getAbsolutePath) + } + + /** + * Adds standard set of cases to a benchmark given a dataframe and field to select. + */ + private def addCases( + benchmark: Benchmark, + df: DataFrame, + desc: String, + selector: String): Unit = { + benchmark.addCase(desc + " (read in-mem)") { iter => + df.selectExpr(s"sum($selector)").collect() + } + benchmark.addCase(desc + " (exec in-mem)") { iter => + df.selectExpr("*", s"hash($selector) as f").selectExpr(s"sum($selector)", "sum(f)").collect() + } + val parquet = saveAsParquet(df) + benchmark.addCase(desc + " (read parquet)") { iter => + parquet.selectExpr(s"sum($selector) as f").collect() + } + benchmark.addCase(desc + " (write parquet)") { iter => + saveAsParquet(df.selectExpr(s"sum($selector) as f")) + } + } + + ignore("parsing large select expressions") { + val benchmark = new Benchmark("parsing large select", 1, output = Some(out)) + for (width <- widthsToTest) { + val selectExpr = (1 to width).map(i => s"id as a_$i") + benchmark.addCase(s"$width select expressions") { iter => + sparkSession.range(1).toDF.selectExpr(selectExpr: _*) + } + } + benchmark.run() + } + + ignore("many column field read and write") { + val benchmark = new Benchmark("many column field r/w", scaleFactor, output = Some(out)) + for (width <- widthsToTest) { + // normalize by width to keep constant data size + val numRows = scaleFactor / width + val selectExpr = (1 to width).map(i => s"id as a_$i") + val df = sparkSession.range(numRows).toDF.selectExpr(selectExpr: _*).cache() + df.count() // force caching + addCases(benchmark, df, s"$width cols x $numRows rows", "a_1") + } + benchmark.run() + } + + ignore("wide shallowly nested struct field read and write") { + val benchmark = new Benchmark( + "wide shallowly nested struct field r/w", scaleFactor, output = Some(out)) + for (width <- widthsToTest) { + val numRows = scaleFactor / width + var datum: String = "{" + for (i <- 1 to width) { + if (i == 1) { + datum += s""""value_$i": 1""" + } else { + datum += s""", "value_$i": 1""" + } + } + datum += "}" + datum = s"""{"a": {"b": {"c": $datum, "d": $datum}, "e": $datum}}""" + val df = sparkSession.read.json(sparkSession.range(numRows).map(_ => datum)).cache() + df.count() // force caching + addCases(benchmark, df, s"$width wide x $numRows rows", "a.b.c.value_1") + } + benchmark.run() + } + + ignore("deeply nested struct field read and write") { + val benchmark = new Benchmark("deeply nested struct field r/w", scaleFactor, output = Some(out)) + for (depth <- depthsToTest) { + val numRows = scaleFactor / depth + var datum: String = "{\"value\": 1}" + var selector: String = "value" + for (i <- 1 to depth) { + datum = "{\"value\": " + datum + "}" + selector = selector + ".value" + } + val df = sparkSession.read.json(sparkSession.range(numRows).map(_ => datum)).cache() + df.count() // force caching + addCases(benchmark, df, s"$depth deep x $numRows rows", selector) + } + benchmark.run() + } + + ignore("bushy struct field read and write") { + val benchmark = new Benchmark("bushy struct field r/w", scaleFactor, output = Some(out)) + for (width <- Seq(1, 100, 1000)) { + val numRows = scaleFactor / width + var numNodes = 1 + var datum: String = "{\"value\": 1}" + var selector: String = "value" + var depth = 1 + while (numNodes < width) { + numNodes *= 2 + datum = s"""{"left_$depth": $datum, "right_$depth": $datum}""" + selector = s"left_$depth." + selector + depth += 1 + } + // TODO(ekl) seems like the json parsing is actually the majority of the time, perhaps + // we should benchmark that too separately. + val df = sparkSession.read.json(sparkSession.range(numRows).map(_ => datum)).cache() + df.count() // force caching + addCases(benchmark, df, s"$numNodes x $depth deep x $numRows rows", selector) + } + benchmark.run() + } + + ignore("wide array field read and write") { + val benchmark = new Benchmark("wide array field r/w", scaleFactor, output = Some(out)) + for (width <- widthsToTest) { + val numRows = scaleFactor / width + var datum: String = "{\"value\": [" + for (i <- 1 to width) { + if (i == 1) { + datum += "1" + } else { + datum += ", 1" + } + } + datum += "]}" + val df = sparkSession.read.json(sparkSession.range(numRows).map(_ => datum)).cache() + df.count() // force caching + addCases(benchmark, df, s"$width wide x $numRows rows", "value[0]") + } + benchmark.run() + } + + ignore("wide map field read and write") { + val benchmark = new Benchmark("wide map field r/w", scaleFactor, output = Some(out)) + for (width <- widthsToTest) { + val numRows = scaleFactor / width + val datum = Tuple1((1 to width).map(i => ("value_" + i -> 1)).toMap) + val df = sparkSession.range(numRows).map(_ => datum).toDF.cache() + df.count() // force caching + addCases(benchmark, df, s"$width wide x $numRows rows", "_1[\"value_1\"]") + } + benchmark.run() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala new file mode 100644 index 000000000000..d4e7e362c6c8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.columnar + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types._ + +class ColumnStatsSuite extends SparkFunSuite { + testColumnStats(classOf[BooleanColumnStats], BOOLEAN, Array(true, false, 0)) + testColumnStats(classOf[ByteColumnStats], BYTE, Array(Byte.MaxValue, Byte.MinValue, 0)) + testColumnStats(classOf[ShortColumnStats], SHORT, Array(Short.MaxValue, Short.MinValue, 0)) + testColumnStats(classOf[IntColumnStats], INT, Array(Int.MaxValue, Int.MinValue, 0)) + testColumnStats(classOf[LongColumnStats], LONG, Array(Long.MaxValue, Long.MinValue, 0)) + testColumnStats(classOf[FloatColumnStats], FLOAT, Array(Float.MaxValue, Float.MinValue, 0)) + testColumnStats(classOf[DoubleColumnStats], DOUBLE, Array(Double.MaxValue, Double.MinValue, 0)) + testColumnStats(classOf[StringColumnStats], STRING, Array(null, null, 0)) + testDecimalColumnStats(Array(null, null, 0)) + + def testColumnStats[T <: AtomicType, U <: ColumnStats]( + columnStatsClass: Class[U], + columnType: NativeColumnType[T], + initialStatistics: Array[Any]): Unit = { + + val columnStatsName = columnStatsClass.getSimpleName + + test(s"$columnStatsName: empty") { + val columnStats = columnStatsClass.newInstance() + columnStats.collectedStatistics.zip(initialStatistics).foreach { + case (actual, expected) => assert(actual === expected) + } + } + + test(s"$columnStatsName: non-empty") { + import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ + + val columnStats = columnStatsClass.newInstance() + val rows = Seq.fill(10)(makeRandomRow(columnType)) ++ Seq.fill(10)(makeNullRow(1)) + rows.foreach(columnStats.gatherStats(_, 0)) + + val values = rows.take(10).map(_.get(0, columnType.dataType).asInstanceOf[T#InternalType]) + val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#InternalType]] + val stats = columnStats.collectedStatistics + + assertResult(values.min(ordering), "Wrong lower bound")(stats(0)) + assertResult(values.max(ordering), "Wrong upper bound")(stats(1)) + assertResult(10, "Wrong null count")(stats(2)) + assertResult(20, "Wrong row count")(stats(3)) + assertResult(stats(4), "Wrong size in bytes") { + rows.map { row => + if (row.isNullAt(0)) 4 else columnType.actualSize(row, 0) + }.sum + } + } + } + + def testDecimalColumnStats[T <: AtomicType, U <: ColumnStats]( + initialStatistics: Array[Any]): Unit = { + + val columnStatsName = classOf[DecimalColumnStats].getSimpleName + val columnType = COMPACT_DECIMAL(15, 10) + + test(s"$columnStatsName: empty") { + val columnStats = new DecimalColumnStats(15, 10) + columnStats.collectedStatistics.zip(initialStatistics).foreach { + case (actual, expected) => assert(actual === expected) + } + } + + test(s"$columnStatsName: non-empty") { + import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ + + val columnStats = new DecimalColumnStats(15, 10) + val rows = Seq.fill(10)(makeRandomRow(columnType)) ++ Seq.fill(10)(makeNullRow(1)) + rows.foreach(columnStats.gatherStats(_, 0)) + + val values = rows.take(10).map(_.get(0, columnType.dataType).asInstanceOf[T#InternalType]) + val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#InternalType]] + val stats = columnStats.collectedStatistics + + assertResult(values.min(ordering), "Wrong lower bound")(stats(0)) + assertResult(values.max(ordering), "Wrong upper bound")(stats(1)) + assertResult(10, "Wrong null count")(stats(2)) + assertResult(20, "Wrong row count")(stats(3)) + assertResult(stats(4), "Wrong size in bytes") { + rows.map { row => + if (row.isNullAt(0)) 4 else columnType.actualSize(row, 0) + }.sum + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala similarity index 76% rename from sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala index 63bc39bfa030..ff05049551dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala @@ -15,17 +15,18 @@ * limitations under the License. */ -package org.apache.spark.sql.columnar +package org.apache.spark.sql.execution.columnar -import java.nio.{ByteOrder, ByteBuffer} +import java.nio.{ByteBuffer, ByteOrder} +import java.nio.charset.StandardCharsets +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.CatalystTypeConverters -import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, GenericMutableRow} -import org.apache.spark.sql.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} +import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ import org.apache.spark.sql.types._ -import org.apache.spark.{Logging, SparkFunSuite} - class ColumnTypeSuite extends SparkFunSuite with Logging { private val DEFAULT_BUFFER_SIZE = 512 @@ -35,9 +36,9 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { test("defaultSize") { val checks = Map( - NULL-> 0, BOOLEAN -> 1, BYTE -> 1, SHORT -> 2, INT -> 4, LONG -> 8, + NULL -> 0, BOOLEAN -> 1, BYTE -> 1, SHORT -> 2, INT -> 4, LONG -> 8, FLOAT -> 4, DOUBLE -> 8, COMPACT_DECIMAL(15, 10) -> 8, LARGE_DECIMAL(20, 10) -> 12, - STRING -> 8, BINARY -> 16, STRUCT_TYPE -> 20, ARRAY_TYPE -> 16, MAP_TYPE -> 32) + STRING -> 8, BINARY -> 16, STRUCT_TYPE -> 20, ARRAY_TYPE -> 28, MAP_TYPE -> 68) checks.foreach { case (columnType, expectedSize) => assertResult(expectedSize, s"Wrong defaultSize for $columnType") { @@ -53,7 +54,7 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { expected: Int): Unit = { assertResult(expected, s"Wrong actualSize for $columnType") { - val row = new GenericMutableRow(1) + val row = new GenericInternalRow(1) row.update(0, CatalystTypeConverters.convertToCatalyst(value)) val proj = UnsafeProjection.create(Array[DataType](columnType.dataType)) columnType.actualSize(proj(row), 0) @@ -68,12 +69,12 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { checkActualSize(LONG, Long.MaxValue, 8) checkActualSize(FLOAT, Float.MaxValue, 4) checkActualSize(DOUBLE, Double.MaxValue, 8) - checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) + checkActualSize(STRING, "hello", 4 + "hello".getBytes(StandardCharsets.UTF_8).length) checkActualSize(BINARY, Array.fill[Byte](4)(0.toByte), 4 + 4) checkActualSize(COMPACT_DECIMAL(15, 10), Decimal(0, 15, 10), 8) checkActualSize(LARGE_DECIMAL(20, 10), Decimal(0, 20, 10), 5) - checkActualSize(ARRAY_TYPE, Array[Any](1), 16) - checkActualSize(MAP_TYPE, Map(1 -> "a"), 29) + checkActualSize(ARRAY_TYPE, Array[Any](1), 4 + 8 + 8 + 8) + checkActualSize(MAP_TYPE, Map(1 -> "a"), 4 + (8 + 8 + 8 + 8) + (8 + 8 + 8 + 8)) checkActualSize(STRUCT_TYPE, Row("hello"), 28) } @@ -100,14 +101,15 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { def testColumnType[JvmType](columnType: ColumnType[JvmType]): Unit = { - val buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE).order(ByteOrder.nativeOrder()) val proj = UnsafeProjection.create(Array[DataType](columnType.dataType)) val converter = CatalystTypeConverters.createToScalaConverter(columnType.dataType) val seq = (0 until 4).map(_ => proj(makeRandomRow(columnType)).copy()) + val totalSize = seq.map(_.getSizeInBytes).sum + val bufferSize = Math.max(DEFAULT_BUFFER_SIZE, totalSize) test(s"$columnType append/extract") { - buffer.rewind() - seq.foreach(columnType.append(_, 0, buffer)) + val buffer = ByteBuffer.allocate(bufferSize).order(ByteOrder.nativeOrder()) + seq.foreach(r => columnType.append(columnType.getField(r, 0), buffer)) buffer.rewind() seq.foreach { row => @@ -142,4 +144,18 @@ class ColumnTypeSuite extends SparkFunSuite with Logging { ColumnType(DecimalType(19, 0)) } } + + test("show type name in type mismatch error") { + val invalidType = new DataType { + override def defaultSize: Int = 1 + override private[spark] def asNullable: DataType = this + override def typeName: String = "invalid type name" + } + + val message = intercept[java.lang.Exception] { + ColumnType(invalidType) + }.getMessage + + assert(message.contains("Unsupported type: invalid type name")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala similarity index 86% rename from sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala index a5882f7870e3..686c8fa6f5fa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala @@ -15,20 +15,20 @@ * limitations under the License. */ -package org.apache.spark.sql.columnar +package org.apache.spark.sql.execution.columnar import scala.collection.immutable.HashSet import scala.util.Random import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, GenericMutableRow} -import org.apache.spark.sql.catalyst.util.{GenericArrayData, ArrayBasedMapData} +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} import org.apache.spark.sql.types.{AtomicType, Decimal} import org.apache.spark.unsafe.types.UTF8String object ColumnarTestUtils { - def makeNullRow(length: Int): GenericMutableRow = { - val row = new GenericMutableRow(length) + def makeNullRow(length: Int): GenericInternalRow = { + val row = new GenericInternalRow(length) (0 until length).foreach(row.setNullAt) row } @@ -60,6 +60,7 @@ object ColumnarTestUtils { case MAP(_) => ArrayBasedMapData( Map(Random.nextInt() -> UTF8String.fromString(Random.nextString(Random.nextInt(32))))) + case _ => throw new IllegalArgumentException(s"Unknown column type $columnType") }).asInstanceOf[JvmType] } @@ -85,7 +86,7 @@ object ColumnarTestUtils { tail: ColumnType[_]*): InternalRow = makeRandomRow(Seq(head) ++ tail) def makeRandomRow(columnTypes: Seq[ColumnType[_]]): InternalRow = { - val row = new GenericMutableRow(columnTypes.length) + val row = new GenericInternalRow(columnTypes.length) makeRandomValues(columnTypes).zipWithIndex.foreach { case (value, index) => row(index) = value } @@ -94,11 +95,11 @@ object ColumnarTestUtils { def makeUniqueValuesAndSingleValueRows[T <: AtomicType]( columnType: NativeColumnType[T], - count: Int): (Seq[T#InternalType], Seq[GenericMutableRow]) = { + count: Int): (Seq[T#InternalType], Seq[GenericInternalRow]) = { val values = makeUniqueRandomValues(columnType, count) val rows = values.map { value => - val row = new GenericMutableRow(1) + val row = new GenericInternalRow(1) row(0) = value row } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala new file mode 100644 index 000000000000..8d411eb191cd --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -0,0 +1,432 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.columnar + +import java.nio.charset.StandardCharsets +import java.sql.{Date, Timestamp} + +import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.catalyst.expressions.AttributeSet +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SQLTestData._ +import org.apache.spark.sql.types._ +import org.apache.spark.storage.StorageLevel._ + +class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + setupTestData() + + private def cachePrimitiveTest(data: DataFrame, dataType: String) { + data.createOrReplaceTempView(s"testData$dataType") + val storageLevel = MEMORY_ONLY + val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan + val inMemoryRelation = InMemoryRelation(useCompression = true, 5, storageLevel, plan, None) + + assert(inMemoryRelation.cachedColumnBuffers.getStorageLevel == storageLevel) + inMemoryRelation.cachedColumnBuffers.collect().head match { + case _: CachedBatch => + case other => fail(s"Unexpected cached batch type: ${other.getClass.getName}") + } + checkAnswer(inMemoryRelation, data.collect().toSeq) + } + + private def testPrimitiveType(nullability: Boolean): Unit = { + val dataTypes = Seq(BooleanType, ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DateType, TimestampType, DecimalType(25, 5), DecimalType(6, 5)) + val schema = StructType(dataTypes.zipWithIndex.map { case (dataType, index) => + StructField(s"col$index", dataType, nullability) + }) + val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( + if (nullability && i % 3 == 0) null else if (i % 2 == 0) true else false, + if (nullability && i % 3 == 0) null else i.toByte, + if (nullability && i % 3 == 0) null else i.toShort, + if (nullability && i % 3 == 0) null else i.toInt, + if (nullability && i % 3 == 0) null else i.toLong, + if (nullability && i % 3 == 0) null else (i + 0.25).toFloat, + if (nullability && i % 3 == 0) null else (i + 0.75).toDouble, + if (nullability && i % 3 == 0) null else new Date(i), + if (nullability && i % 3 == 0) null else new Timestamp(i * 1000000L), + if (nullability && i % 3 == 0) null else BigDecimal(Long.MaxValue.toString + ".12345"), + if (nullability && i % 3 == 0) null + else new java.math.BigDecimal(s"${i % 9 + 1}" + ".23456") + ))) + cachePrimitiveTest(spark.createDataFrame(rdd, schema), "primitivesDateTimeStamp") + } + + private def tesNonPrimitiveType(nullability: Boolean): Unit = { + val struct = StructType(StructField("f1", FloatType, false) :: + StructField("f2", ArrayType(BooleanType), true) :: Nil) + val schema = StructType(Seq( + StructField("col0", StringType, nullability), + StructField("col1", ArrayType(IntegerType), nullability), + StructField("col2", ArrayType(ArrayType(IntegerType)), nullability), + StructField("col3", MapType(StringType, IntegerType), nullability), + StructField("col4", struct, nullability) + )) + val rdd = spark.sparkContext.parallelize((1 to 10).map(i => Row( + if (nullability && i % 3 == 0) null else s"str${i}: test cache.", + if (nullability && i % 3 == 0) null else (i * 100 to i * 100 + i).toArray, + if (nullability && i % 3 == 0) null + else Array(Array(i, i + 1), Array(i * 100 + 1, i * 100, i * 100 + 2)), + if (nullability && i % 3 == 0) null else (i to i + i).map(j => s"key$j" -> j).toMap, + if (nullability && i % 3 == 0) null else Row((i + 0.25).toFloat, Seq(true, false, null)) + ))) + cachePrimitiveTest(spark.createDataFrame(rdd, schema), "StringArrayMapStruct") + } + + test("primitive type with nullability:true") { + testPrimitiveType(true) + } + + test("primitive type with nullability:false") { + testPrimitiveType(false) + } + + test("non-primitive type with nullability:true") { + val schemaNull = StructType(Seq(StructField("col", NullType, true))) + val rddNull = spark.sparkContext.parallelize((1 to 10).map(i => Row(null))) + cachePrimitiveTest(spark.createDataFrame(rddNull, schemaNull), "Null") + + tesNonPrimitiveType(true) + } + + test("non-primitive type with nullability:false") { + tesNonPrimitiveType(false) + } + + test("simple columnar query") { + val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan + val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None) + + checkAnswer(scan, testData.collect().toSeq) + } + + 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)) + .toDF().createOrReplaceTempView("sizeTst") + spark.catalog.cacheTable("sizeTst") + assert( + spark.table("sizeTst").queryExecution.analyzed.stats.sizeInBytes > + spark.conf.get(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD)) + } + + test("projection") { + val plan = spark.sessionState.executePlan(testData.select('value, 'key).logicalPlan).sparkPlan + val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None) + + checkAnswer(scan, testData.collect().map { + case Row(key: Int, value: String) => value -> key + }.map(Row.fromTuple)) + } + + test("access only some column of the all of columns") { + val df = spark.range(1, 100).map(i => (i, (i + 1).toFloat)).toDF("i", "f") + df.cache + df.count // forced to build cache + assert(df.filter("f <= 10.0").count == 9) + } + + test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { + val plan = spark.sessionState.executePlan(testData.logicalPlan).sparkPlan + val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan, None) + + checkAnswer(scan, testData.collect().toSeq) + checkAnswer(scan, testData.collect().toSeq) + } + + test("SPARK-1678 regression: compression must not lose repeated values") { + checkAnswer( + sql("SELECT * FROM repeatedData"), + repeatedData.collect().toSeq.map(Row.fromTuple)) + + spark.catalog.cacheTable("repeatedData") + + checkAnswer( + sql("SELECT * FROM repeatedData"), + repeatedData.collect().toSeq.map(Row.fromTuple)) + } + + test("with null values") { + checkAnswer( + sql("SELECT * FROM nullableRepeatedData"), + nullableRepeatedData.collect().toSeq.map(Row.fromTuple)) + + spark.catalog.cacheTable("nullableRepeatedData") + + checkAnswer( + sql("SELECT * FROM nullableRepeatedData"), + nullableRepeatedData.collect().toSeq.map(Row.fromTuple)) + } + + test("SPARK-2729 regression: timestamp data type") { + val timestamps = (0 to 3).map(i => Tuple1(new Timestamp(i))).toDF("time") + timestamps.createOrReplaceTempView("timestamps") + + checkAnswer( + sql("SELECT time FROM timestamps"), + timestamps.collect().toSeq) + + spark.catalog.cacheTable("timestamps") + + checkAnswer( + sql("SELECT time FROM timestamps"), + timestamps.collect().toSeq) + } + + test("SPARK-3320 regression: batched column buffer building should work with empty partitions") { + checkAnswer( + sql("SELECT * FROM withEmptyParts"), + withEmptyParts.collect().toSeq.map(Row.fromTuple)) + + spark.catalog.cacheTable("withEmptyParts") + + checkAnswer( + sql("SELECT * FROM withEmptyParts"), + withEmptyParts.collect().toSeq.map(Row.fromTuple)) + } + + test("SPARK-4182 Caching complex types") { + complexData.cache().count() + // Shouldn't throw + complexData.count() + complexData.unpersist() + } + + test("decimal type") { + // Casting is required here because ScalaReflection can't capture decimal precision information. + val df = (1 to 10) + .map(i => Tuple1(Decimal(i, 15, 10).toJavaBigDecimal)) + .toDF("dec") + .select($"dec" cast DecimalType(15, 10)) + + assert(df.schema.head.dataType === DecimalType(15, 10)) + + df.cache().createOrReplaceTempView("test_fixed_decimal") + checkAnswer( + sql("SELECT * FROM test_fixed_decimal"), + (1 to 10).map(i => Row(Decimal(i, 15, 10).toJavaBigDecimal))) + } + + test("test different data types") { + // Create the schema. + val struct = + StructType( + StructField("f1", FloatType, true) :: + StructField("f2", ArrayType(BooleanType), true) :: Nil) + val dataTypes = + Seq(StringType, BinaryType, NullType, BooleanType, + ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), + DateType, TimestampType, ArrayType(IntegerType), struct) + val fields = dataTypes.zipWithIndex.map { case (dataType, index) => + StructField(s"col$index", dataType, true) + } + val allColumns = fields.map(_.name).mkString(",") + val schema = StructType(fields) + + // Create an RDD for the schema + val rdd = + sparkContext.parallelize(1 to 10000, 10).map { i => + Row( + s"str$i: test cache.", + s"binary$i: test cache.".getBytes(StandardCharsets.UTF_8), + null, + i % 2 == 0, + i.toByte, + i.toShort, + i, + Long.MaxValue - i.toLong, + (i + 0.25).toFloat, + i + 0.75, + BigDecimal(Long.MaxValue.toString + ".12345"), + new java.math.BigDecimal(s"${i % 9 + 1}" + ".23456"), + new Date(i), + new Timestamp(i * 1000000L), + i to i + 10, + Row((i - 0.25).toFloat, Seq(true, false, null))) + } + spark.createDataFrame(rdd, schema).createOrReplaceTempView("InMemoryCache_different_data_types") + // Cache the table. + sql("cache table InMemoryCache_different_data_types") + // Make sure the table is indeed cached. + spark.table("InMemoryCache_different_data_types").queryExecution.executedPlan + assert( + spark.catalog.isCached("InMemoryCache_different_data_types"), + "InMemoryCache_different_data_types should be cached.") + // Issue a query and check the results. + checkAnswer( + sql(s"SELECT DISTINCT ${allColumns} FROM InMemoryCache_different_data_types"), + spark.table("InMemoryCache_different_data_types").collect()) + spark.catalog.dropTempView("InMemoryCache_different_data_types") + } + + test("SPARK-10422: String column in InMemoryColumnarCache needs to override clone method") { + val df = spark.range(1, 100).selectExpr("id % 10 as id") + .rdd.map(id => Tuple1(s"str_$id")).toDF("i") + val cached = df.cache() + // count triggers the caching action. It should not throw. + cached.count() + + // Make sure, the DataFrame is indeed cached. + assert(spark.sharedState.cacheManager.lookupCachedData(cached).nonEmpty) + + // Check result. + checkAnswer( + cached, + spark.range(1, 100).selectExpr("id % 10 as id") + .rdd.map(id => Tuple1(s"str_$id")).toDF("i") + ) + + // Drop the cache. + cached.unpersist() + } + + test("SPARK-10859: Predicates pushed to InMemoryColumnarTableScan are not evaluated correctly") { + val data = spark.range(10).selectExpr("id", "cast(id as string) as s") + data.cache() + assert(data.count() === 10) + assert(data.filter($"s" === "3").count() === 1) + } + + test("SPARK-14138: Generated SpecificColumnarIterator can exceed JVM size limit for cached DF") { + val length1 = 3999 + val columnTypes1 = List.fill(length1)(IntegerType) + val columnarIterator1 = GenerateColumnAccessor.generate(columnTypes1) + + // SPARK-16664: the limit of janino is 8117 + val length2 = 8117 + val columnTypes2 = List.fill(length2)(IntegerType) + val columnarIterator2 = GenerateColumnAccessor.generate(columnTypes2) + } + + test("SPARK-17549: cached table size should be correctly calculated") { + val data = spark.sparkContext.parallelize(1 to 10, 5).toDF() + val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan + val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None) + + // Materialize the data. + val expectedAnswer = data.collect() + checkAnswer(cached, expectedAnswer) + + // Check that the right size was calculated. + assert(cached.batchStats.value === expectedAnswer.size * INT.defaultSize) + } + + test("access primitive-type columns in CachedBatch without whole stage codegen") { + // whole stage codegen is not applied to a row with more than WHOLESTAGE_MAX_NUM_FIELDS fields + withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "2") { + val data = Seq(null, true, 1.toByte, 3.toShort, 7, 15.toLong, + 31.25.toFloat, 63.75, new Date(127), new Timestamp(255000000L), null) + val dataTypes = Seq(NullType, BooleanType, ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DateType, TimestampType, IntegerType) + val schemas = dataTypes.zipWithIndex.map { case (dataType, index) => + StructField(s"col$index", dataType, true) + } + val rdd = sparkContext.makeRDD(Seq(Row.fromSeq(data))) + val df = spark.createDataFrame(rdd, StructType(schemas)) + val row = df.persist.take(1).apply(0) + checkAnswer(df, row) + } + } + + test("access decimal/string-type columns in CachedBatch without whole stage codegen") { + withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "2") { + val data = Seq(BigDecimal(Long.MaxValue.toString + ".12345"), + new java.math.BigDecimal("1234567890.12345"), + new java.math.BigDecimal("1.23456"), + "test123" + ) + val schemas = Seq( + StructField("col0", DecimalType(25, 5), true), + StructField("col1", DecimalType(15, 5), true), + StructField("col2", DecimalType(6, 5), true), + StructField("col3", StringType, true) + ) + val rdd = sparkContext.makeRDD(Seq(Row.fromSeq(data))) + val df = spark.createDataFrame(rdd, StructType(schemas)) + val row = df.persist.take(1).apply(0) + checkAnswer(df, row) + } + } + + test("access non-primitive-type columns in CachedBatch without whole stage codegen") { + withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "2") { + val data = Seq((1 to 10).toArray, + Array(Array(10, 11), Array(100, 111, 123)), + Map("key1" -> 111, "key2" -> 222), + Row(1.25.toFloat, Seq(true, false, null)) + ) + val struct = StructType(StructField("f1", FloatType, false) :: + StructField("f2", ArrayType(BooleanType), true) :: Nil) + val schemas = Seq( + StructField("col0", ArrayType(IntegerType), true), + StructField("col1", ArrayType(ArrayType(IntegerType)), true), + StructField("col2", MapType(StringType, IntegerType), true), + StructField("col3", struct, true) + ) + val rdd = sparkContext.makeRDD(Seq(Row.fromSeq(data))) + val df = spark.createDataFrame(rdd, StructType(schemas)) + val row = df.persist.take(1).apply(0) + checkAnswer(df, row) + } + } + + test("InMemoryTableScanExec should return correct output ordering and partitioning") { + val df1 = Seq((0, 0), (1, 1)).toDF + .repartition(col("_1")).sortWithinPartitions(col("_1")).persist + val df2 = Seq((0, 0), (1, 1)).toDF + .repartition(col("_1")).sortWithinPartitions(col("_1")).persist + + // Because two cached dataframes have the same logical plan, this is a self-join actually. + // So we force one of in-memory relation to alias its output. Then we can test if original and + // aliased in-memory relations have correct ordering and partitioning. + val joined = df1.joinWith(df2, df1("_1") === df2("_1")) + + val inMemoryScans = joined.queryExecution.executedPlan.collect { + case m: InMemoryTableScanExec => m + } + inMemoryScans.foreach { inMemoryScan => + val sortedAttrs = AttributeSet(inMemoryScan.outputOrdering.flatMap(_.references)) + assert(sortedAttrs.subsetOf(inMemoryScan.outputSet)) + + val partitionedAttrs = + inMemoryScan.outputPartitioning.asInstanceOf[HashPartitioning].references + assert(partitionedAttrs.subsetOf(inMemoryScan.outputSet)) + } + } + + test("SPARK-20356: pruned InMemoryTableScanExec should have correct ordering and partitioning") { + withSQLConf("spark.sql.shuffle.partitions" -> "200") { + val df1 = Seq(("a", 1), ("b", 1), ("c", 2)).toDF("item", "group") + val df2 = Seq(("a", 1), ("b", 2), ("c", 3)).toDF("item", "id") + val df3 = df1.join(df2, Seq("item")).select($"id", $"group".as("item")).distinct() + + df3.unpersist() + val agg_without_cache = df3.groupBy($"item").count() + + df3.cache() + val agg_with_cache = df3.groupBy($"item").count() + checkAnswer(agg_without_cache, agg_with_cache) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessorSuite.scala similarity index 92% rename from sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessorSuite.scala index aa1605fee8c7..8f4ca3cea77a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnAccessorSuite.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql.columnar +package org.apache.spark.sql.execution.columnar import java.nio.ByteBuffer import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.CatalystTypeConverters -import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, GenericMutableRow} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} import org.apache.spark.sql.types._ class TestNullableColumnAccessor[JvmType]( @@ -38,7 +38,7 @@ object TestNullableColumnAccessor { } class NullableColumnAccessorSuite extends SparkFunSuite { - import org.apache.spark.sql.columnar.ColumnarTestUtils._ + import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ Seq( NULL, BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, @@ -72,7 +72,7 @@ class NullableColumnAccessorSuite extends SparkFunSuite { } val accessor = TestNullableColumnAccessor(builder.build(), columnType) - val row = new GenericMutableRow(1) + val row = new GenericInternalRow(1) val converter = CatalystTypeConverters.createToScalaConverter(columnType.dataType) (0 until 4).foreach { _ => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnBuilderSuite.scala similarity index 93% rename from sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnBuilderSuite.scala index 91404577832a..b2b6e92e9a05 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/NullableColumnBuilderSuite.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.columnar +package org.apache.spark.sql.execution.columnar import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.CatalystTypeConverters -import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, GenericMutableRow} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} import org.apache.spark.sql.types._ class TestNullableColumnBuilder[JvmType](columnType: ColumnType[JvmType]) @@ -36,7 +36,7 @@ object TestNullableColumnBuilder { } class NullableColumnBuilderSuite extends SparkFunSuite { - import org.apache.spark.sql.columnar.ColumnarTestUtils._ + import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ Seq( BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, @@ -94,7 +94,7 @@ class NullableColumnBuilderSuite extends SparkFunSuite { (1 to 7 by 2).foreach(assertResult(_, "Wrong null position")(buffer.getInt())) // For non-null values - val actual = new GenericMutableRow(new Array[Any](1)) + val actual = new GenericInternalRow(new Array[Any](1)) (0 until 4).foreach { _ => columnType.extract(buffer, actual, 0) assert(converter(actual.get(0, dataType)) === converter(randomRow.get(0, dataType)), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala new file mode 100644 index 000000000000..9d862cfdecb2 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.columnar + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SQLTestData._ + + +class PartitionBatchPruningSuite + extends SparkFunSuite + with BeforeAndAfterEach + with SharedSQLContext { + + import testImplicits._ + + private lazy val originalColumnBatchSize = spark.conf.get(SQLConf.COLUMN_BATCH_SIZE) + private lazy val originalInMemoryPartitionPruning = + spark.conf.get(SQLConf.IN_MEMORY_PARTITION_PRUNING) + + override protected def beforeAll(): Unit = { + super.beforeAll() + // Make a table with 5 partitions, 2 batches per partition, 10 elements per batch + spark.conf.set(SQLConf.COLUMN_BATCH_SIZE.key, 10) + // Enable in-memory partition pruning + spark.conf.set(SQLConf.IN_MEMORY_PARTITION_PRUNING.key, true) + // Enable in-memory table scan accumulators + spark.conf.set("spark.sql.inMemoryTableScanStatistics.enable", "true") + } + + override protected def afterAll(): Unit = { + try { + spark.conf.set(SQLConf.COLUMN_BATCH_SIZE.key, originalColumnBatchSize) + spark.conf.set(SQLConf.IN_MEMORY_PARTITION_PRUNING.key, originalInMemoryPartitionPruning) + } finally { + super.afterAll() + } + } + + override protected def beforeEach(): Unit = { + super.beforeEach() + // This creates accumulators, which get cleaned up after every single test, + // so we need to do this before every test. + 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).toDF() + pruningData.createOrReplaceTempView("pruningData") + spark.catalog.cacheTable("pruningData") + + val pruningStringData = sparkContext.makeRDD((100 to 200).map { key => + StringData(key.toString) + }, 5).toDF() + pruningStringData.createOrReplaceTempView("pruningStringData") + spark.catalog.cacheTable("pruningStringData") + } + + override protected def afterEach(): Unit = { + try { + spark.catalog.uncacheTable("pruningData") + spark.catalog.uncacheTable("pruningStringData") + } finally { + super.afterEach() + } + } + + // Comparisons + checkBatchPruning("SELECT key FROM pruningData WHERE key = 1", 1, 1)(Seq(1)) + checkBatchPruning("SELECT key FROM pruningData WHERE 1 = key", 1, 1)(Seq(1)) + checkBatchPruning("SELECT key FROM pruningData WHERE key <=> 1", 1, 1)(Seq(1)) + checkBatchPruning("SELECT key FROM pruningData WHERE 1 <=> key", 1, 1)(Seq(1)) + checkBatchPruning("SELECT key FROM pruningData WHERE key < 12", 1, 2)(1 to 11) + checkBatchPruning("SELECT key FROM pruningData WHERE key <= 11", 1, 2)(1 to 11) + checkBatchPruning("SELECT key FROM pruningData WHERE key > 88", 1, 2)(89 to 100) + checkBatchPruning("SELECT key FROM pruningData WHERE key >= 89", 1, 2)(89 to 100) + checkBatchPruning("SELECT key FROM pruningData WHERE 12 > key", 1, 2)(1 to 11) + checkBatchPruning("SELECT key FROM pruningData WHERE 11 >= key", 1, 2)(1 to 11) + checkBatchPruning("SELECT key FROM pruningData WHERE 88 < key", 1, 2)(89 to 100) + checkBatchPruning("SELECT key FROM pruningData WHERE 89 <= key", 1, 2)(89 to 100) + + // IS NULL + checkBatchPruning("SELECT key FROM pruningData WHERE value IS NULL", 5, 5) { + (1 to 10) ++ (21 to 30) ++ (41 to 50) ++ (61 to 70) ++ (81 to 90) + } + + // IS NOT NULL + checkBatchPruning("SELECT key FROM pruningData WHERE value IS NOT NULL", 5, 5) { + (11 to 20) ++ (31 to 40) ++ (51 to 60) ++ (71 to 80) ++ (91 to 100) + } + + // Conjunction and disjunction + checkBatchPruning("SELECT key FROM pruningData WHERE key > 8 AND key <= 21", 2, 3)(9 to 21) + checkBatchPruning("SELECT key FROM pruningData WHERE key < 2 OR key > 99", 2, 2)(Seq(1, 100)) + checkBatchPruning("SELECT key FROM pruningData WHERE key < 12 AND key IS NOT NULL", 1, 2)(1 to 11) + checkBatchPruning("SELECT key FROM pruningData WHERE key < 2 OR (key > 78 AND key < 92)", 3, 4) { + Seq(1) ++ (79 to 91) + } + checkBatchPruning("SELECT key FROM pruningData WHERE NOT (key < 88)", 1, 2) { + // Although the `NOT` operator isn't supported directly, the optimizer can transform + // `NOT (a < b)` to `b >= a` + 88 to 100 + } + + // Support `IN` predicate + checkBatchPruning("SELECT key FROM pruningData WHERE key IN (1)", 1, 1)(Seq(1)) + checkBatchPruning("SELECT key FROM pruningData WHERE key IN (1, 2)", 1, 1)(Seq(1, 2)) + checkBatchPruning("SELECT key FROM pruningData WHERE key IN (1, 11)", 1, 2)(Seq(1, 11)) + checkBatchPruning("SELECT key FROM pruningData WHERE key IN (1, 21, 41, 61, 81)", 5, 5)( + Seq(1, 21, 41, 61, 81)) + checkBatchPruning("SELECT CAST(s AS INT) FROM pruningStringData WHERE s = '100'", 1, 1)(Seq(100)) + checkBatchPruning("SELECT CAST(s AS INT) FROM pruningStringData WHERE s < '102'", 1, 1)( + Seq(100, 101)) + checkBatchPruning( + "SELECT CAST(s AS INT) FROM pruningStringData WHERE s IN ('99', '150', '201')", 1, 1)( + Seq(150)) + + // With unsupported `InSet` predicate + { + val seq = (1 to 30).mkString(", ") + checkBatchPruning(s"SELECT key FROM pruningData WHERE key IN ($seq)", 5, 10)(1 to 30) + checkBatchPruning(s"SELECT key FROM pruningData WHERE NOT (key IN ($seq))", 5, 10)(31 to 100) + checkBatchPruning(s"SELECT key FROM pruningData WHERE NOT (key IN ($seq)) AND key > 88", 1, 2) { + 89 to 100 + } + } + + // With disable IN_MEMORY_PARTITION_PRUNING option + test("disable IN_MEMORY_PARTITION_PRUNING") { + spark.conf.set(SQLConf.IN_MEMORY_PARTITION_PRUNING.key, false) + + val df = sql("SELECT key FROM pruningData WHERE key = 1") + val result = df.collect().map(_(0)).toArray + assert(result.length === 1) + + val (readPartitions, readBatches) = df.queryExecution.sparkPlan.collect { + case in: InMemoryTableScanExec => (in.readPartitions.value, in.readBatches.value) + }.head + assert(readPartitions === 5) + assert(readBatches === 10) + } + + def checkBatchPruning( + query: String, + expectedReadPartitions: Int, + expectedReadBatches: Int)( + expectedQueryResult: => Seq[Int]): Unit = { + + test(query) { + val df = sql(query) + val queryExecution = df.queryExecution + + assertResult(expectedQueryResult.toArray, s"Wrong query result: $queryExecution") { + df.collect().map(_(0)).toArray + } + + val (readPartitions, readBatches) = df.queryExecution.sparkPlan.collect { + case in: InMemoryTableScanExec => (in.readPartitions.value, in.readBatches.value) + }.head + + assert(readBatches === expectedReadBatches, s"Wrong number of read batches: $queryExecution") + assert( + readPartitions === expectedReadPartitions, + s"Wrong number of read partitions: $queryExecution") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala similarity index 91% rename from sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala index 9a2948c59ba4..d01bf911e3a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql.columnar.compression +package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.columnar.ColumnarTestUtils._ -import org.apache.spark.sql.columnar.{BOOLEAN, NoopColumnStats} +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.execution.columnar.{BOOLEAN, NoopColumnStats} +import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ class BooleanBitSetSuite extends SparkFunSuite { import BooleanBitSet._ @@ -72,7 +72,7 @@ class BooleanBitSetSuite extends SparkFunSuite { buffer.rewind().position(headerSize + 4) val decoder = BooleanBitSet.decoder(buffer, BOOLEAN) - val mutableRow = new GenericMutableRow(1) + val mutableRow = new GenericInternalRow(1) if (values.nonEmpty) { values.foreach { assert(decoder.hasNext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala new file mode 100644 index 000000000000..9005ec93e786 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala @@ -0,0 +1,345 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.columnar.compression + +import java.nio.{ByteBuffer, ByteOrder} +import java.nio.charset.StandardCharsets + +import org.apache.commons.lang3.RandomStringUtils +import org.apache.commons.math3.distribution.LogNormalDistribution + +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.execution.columnar.{BOOLEAN, INT, LONG, NativeColumnType, SHORT, STRING} +import org.apache.spark.sql.types.AtomicType +import org.apache.spark.util.Benchmark +import org.apache.spark.util.Utils._ + +/** + * Benchmark to decoders using various compression schemes. + */ +object CompressionSchemeBenchmark extends AllCompressionSchemes { + + private[this] def allocateLocal(size: Int): ByteBuffer = { + ByteBuffer.allocate(size).order(ByteOrder.nativeOrder) + } + + private[this] def genLowerSkewData() = { + val rng = new LogNormalDistribution(0.0, 0.01) + () => rng.sample + } + + private[this] def genHigherSkewData() = { + val rng = new LogNormalDistribution(0.0, 1.0) + () => rng.sample + } + + private[this] def prepareEncodeInternal[T <: AtomicType]( + count: Int, + tpe: NativeColumnType[T], + supportedScheme: CompressionScheme, + input: ByteBuffer): ((ByteBuffer, ByteBuffer) => ByteBuffer, Double, ByteBuffer) = { + assert(supportedScheme.supports(tpe)) + + def toRow(d: Any) = new GenericInternalRow(Array[Any](d)) + val encoder = supportedScheme.encoder(tpe) + for (i <- 0 until count) { + encoder.gatherCompressibilityStats(toRow(tpe.extract(input)), 0) + } + input.rewind() + + val compressedSize = if (encoder.compressedSize == 0) { + input.remaining() + } else { + encoder.compressedSize + } + + (encoder.compress, encoder.compressionRatio, allocateLocal(4 + compressedSize)) + } + + private[this] def runEncodeBenchmark[T <: AtomicType]( + name: String, + iters: Int, + count: Int, + tpe: NativeColumnType[T], + input: ByteBuffer): Unit = { + val benchmark = new Benchmark(name, iters * count) + + schemes.filter(_.supports(tpe)).foreach { scheme => + val (compressFunc, compressionRatio, buf) = prepareEncodeInternal(count, tpe, scheme, input) + val label = s"${getFormattedClassName(scheme)}(${compressionRatio.formatted("%.3f")})" + + benchmark.addCase(label)({ i: Int => + for (n <- 0L until iters) { + compressFunc(input, buf) + input.rewind() + buf.rewind() + } + }) + } + + benchmark.run() + } + + private[this] def runDecodeBenchmark[T <: AtomicType]( + name: String, + iters: Int, + count: Int, + tpe: NativeColumnType[T], + input: ByteBuffer): Unit = { + val benchmark = new Benchmark(name, iters * count) + + schemes.filter(_.supports(tpe)).foreach { scheme => + val (compressFunc, _, buf) = prepareEncodeInternal(count, tpe, scheme, input) + val compressedBuf = compressFunc(input, buf) + val label = s"${getFormattedClassName(scheme)}" + + input.rewind() + + benchmark.addCase(label)({ i: Int => + val rowBuf = new GenericInternalRow(1) + + for (n <- 0L until iters) { + compressedBuf.rewind.position(4) + val decoder = scheme.decoder(compressedBuf, tpe) + while (decoder.hasNext) { + decoder.next(rowBuf, 0) + } + } + }) + } + + benchmark.run() + } + + def bitEncodingBenchmark(iters: Int): Unit = { + val count = 65536 + val testData = allocateLocal(count * BOOLEAN.defaultSize) + + val g = { + val rng = genLowerSkewData() + () => (rng().toInt % 2).toByte + } + for (i <- 0 until count) { + testData.put(i * BOOLEAN.defaultSize, g()) + } + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // BOOLEAN Encode: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough(1.000) 3 / 4 19300.2 0.1 1.0X + // RunLengthEncoding(2.491) 923 / 939 72.7 13.8 0.0X + // BooleanBitSet(0.125) 359 / 363 187.1 5.3 0.0X + runEncodeBenchmark("BOOLEAN Encode", iters, count, BOOLEAN, testData) + + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // BOOLEAN Decode: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough 129 / 136 519.8 1.9 1.0X + // RunLengthEncoding 613 / 623 109.4 9.1 0.2X + // BooleanBitSet 1196 / 1222 56.1 17.8 0.1X + runDecodeBenchmark("BOOLEAN Decode", iters, count, BOOLEAN, testData) + } + + def shortEncodingBenchmark(iters: Int): Unit = { + val count = 65536 + val testData = allocateLocal(count * SHORT.defaultSize) + + val g1 = genLowerSkewData() + for (i <- 0 until count) { + testData.putShort(i * SHORT.defaultSize, g1().toShort) + } + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // SHORT Encode (Lower Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough(1.000) 6 / 7 10971.4 0.1 1.0X + // RunLengthEncoding(1.510) 1526 / 1542 44.0 22.7 0.0X + runEncodeBenchmark("SHORT Encode (Lower Skew)", iters, count, SHORT, testData) + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // SHORT Decode (Lower Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough 811 / 837 82.8 12.1 1.0X + // RunLengthEncoding 1219 / 1266 55.1 18.2 0.7X + runDecodeBenchmark("SHORT Decode (Lower Skew)", iters, count, SHORT, testData) + + val g2 = genHigherSkewData() + for (i <- 0 until count) { + testData.putShort(i * SHORT.defaultSize, g2().toShort) + } + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // SHORT Encode (Higher Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough(1.000) 7 / 7 10112.4 0.1 1.0X + // RunLengthEncoding(2.009) 1623 / 1661 41.4 24.2 0.0X + runEncodeBenchmark("SHORT Encode (Higher Skew)", iters, count, SHORT, testData) + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // SHORT Decode (Higher Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough 818 / 827 82.0 12.2 1.0X + // RunLengthEncoding 1202 / 1237 55.8 17.9 0.7X + runDecodeBenchmark("SHORT Decode (Higher Skew)", iters, count, SHORT, testData) + } + + def intEncodingBenchmark(iters: Int): Unit = { + val count = 65536 + val testData = allocateLocal(count * INT.defaultSize) + + val g1 = genLowerSkewData() + for (i <- 0 until count) { + testData.putInt(i * INT.defaultSize, g1().toInt) + } + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // INT Encode (Lower Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough(1.000) 18 / 19 3716.4 0.3 1.0X + // RunLengthEncoding(1.001) 1992 / 2056 33.7 29.7 0.0X + // DictionaryEncoding(0.500) 723 / 739 92.8 10.8 0.0X + // IntDelta(0.250) 368 / 377 182.2 5.5 0.0X + runEncodeBenchmark("INT Encode (Lower Skew)", iters, count, INT, testData) + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // INT Decode (Lower Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough 821 / 845 81.8 12.2 1.0X + // RunLengthEncoding 1246 / 1256 53.9 18.6 0.7X + // DictionaryEncoding 757 / 766 88.6 11.3 1.1X + // IntDelta 680 / 689 98.7 10.1 1.2X + runDecodeBenchmark("INT Decode (Lower Skew)", iters, count, INT, testData) + + val g2 = genHigherSkewData() + for (i <- 0 until count) { + testData.putInt(i * INT.defaultSize, g2().toInt) + } + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // INT Encode (Higher Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough(1.000) 17 / 19 3888.4 0.3 1.0X + // RunLengthEncoding(1.339) 2127 / 2148 31.5 31.7 0.0X + // DictionaryEncoding(0.501) 960 / 972 69.9 14.3 0.0X + // IntDelta(0.250) 362 / 366 185.5 5.4 0.0X + runEncodeBenchmark("INT Encode (Higher Skew)", iters, count, INT, testData) + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // INT Decode (Higher Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough 838 / 884 80.1 12.5 1.0X + // RunLengthEncoding 1287 / 1311 52.1 19.2 0.7X + // DictionaryEncoding 844 / 859 79.5 12.6 1.0X + // IntDelta 764 / 784 87.8 11.4 1.1X + runDecodeBenchmark("INT Decode (Higher Skew)", iters, count, INT, testData) + } + + def longEncodingBenchmark(iters: Int): Unit = { + val count = 65536 + val testData = allocateLocal(count * LONG.defaultSize) + + val g1 = genLowerSkewData() + for (i <- 0 until count) { + testData.putLong(i * LONG.defaultSize, g1().toLong) + } + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // LONG Encode (Lower Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough(1.000) 37 / 38 1804.8 0.6 1.0X + // RunLengthEncoding(0.748) 2065 / 2094 32.5 30.8 0.0X + // DictionaryEncoding(0.250) 950 / 962 70.6 14.2 0.0X + // LongDelta(0.125) 475 / 482 141.2 7.1 0.1X + runEncodeBenchmark("LONG Encode (Lower Skew)", iters, count, LONG, testData) + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // LONG Decode (Lower Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough 888 / 894 75.5 13.2 1.0X + // RunLengthEncoding 1301 / 1311 51.6 19.4 0.7X + // DictionaryEncoding 887 / 904 75.7 13.2 1.0X + // LongDelta 693 / 735 96.8 10.3 1.3X + runDecodeBenchmark("LONG Decode (Lower Skew)", iters, count, LONG, testData) + + val g2 = genHigherSkewData() + for (i <- 0 until count) { + testData.putLong(i * LONG.defaultSize, g2().toLong) + } + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // LONG Encode (Higher Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough(1.000) 34 / 35 1963.9 0.5 1.0X + // RunLengthEncoding(0.999) 2260 / 3021 29.7 33.7 0.0X + // DictionaryEncoding(0.251) 1270 / 1438 52.8 18.9 0.0X + // LongDelta(0.125) 496 / 509 135.3 7.4 0.1X + runEncodeBenchmark("LONG Encode (Higher Skew)", iters, count, LONG, testData) + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // LONG Decode (Higher Skew): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough 965 / 1494 69.5 14.4 1.0X + // RunLengthEncoding 1350 / 1378 49.7 20.1 0.7X + // DictionaryEncoding 892 / 924 75.2 13.3 1.1X + // LongDelta 817 / 847 82.2 12.2 1.2X + runDecodeBenchmark("LONG Decode (Higher Skew)", iters, count, LONG, testData) + } + + def stringEncodingBenchmark(iters: Int): Unit = { + val count = 65536 + val strLen = 8 + val tableSize = 16 + val testData = allocateLocal(count * (4 + strLen)) + + val g = { + val dataTable = (0 until tableSize).map(_ => RandomStringUtils.randomAlphabetic(strLen)) + val rng = genHigherSkewData() + () => dataTable(rng().toInt % tableSize) + } + for (i <- 0 until count) { + testData.putInt(strLen) + testData.put(g().getBytes(StandardCharsets.UTF_8)) + } + testData.rewind() + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // STRING Encode: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough(1.000) 56 / 57 1197.9 0.8 1.0X + // RunLengthEncoding(0.893) 4892 / 4937 13.7 72.9 0.0X + // DictionaryEncoding(0.167) 2968 / 2992 22.6 44.2 0.0X + runEncodeBenchmark("STRING Encode", iters, count, STRING, testData) + + // Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + // STRING Decode: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + // ------------------------------------------------------------------------------------------- + // PassThrough 2422 / 2449 27.7 36.1 1.0X + // RunLengthEncoding 2885 / 3018 23.3 43.0 0.8X + // DictionaryEncoding 2716 / 2752 24.7 40.5 0.9X + runDecodeBenchmark("STRING Decode", iters, count, STRING, testData) + } + + def main(args: Array[String]): Unit = { + bitEncodingBenchmark(1024) + shortEncodingBenchmark(1024) + intEncodingBenchmark(1024) + longEncodingBenchmark(1024) + stringEncodingBenchmark(1024) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala similarity index 93% rename from sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala index acfab6586c0d..67139b13d788 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.spark.sql.columnar.compression +package org.apache.spark.sql.execution.columnar.compression import java.nio.ByteBuffer import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.columnar._ -import org.apache.spark.sql.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.execution.columnar._ +import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ import org.apache.spark.sql.types.AtomicType class DictionaryEncodingSuite extends SparkFunSuite { @@ -97,7 +97,7 @@ class DictionaryEncodingSuite extends SparkFunSuite { buffer.rewind().position(headerSize + 4) val decoder = DictionaryEncoding.decoder(buffer, columnType) - val mutableRow = new GenericMutableRow(1) + val mutableRow = new GenericInternalRow(1) if (inputSeq.nonEmpty) { inputSeq.foreach { i => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala similarity index 91% rename from sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala index 2111e9fbe62c..411d31fa0e29 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql.columnar.compression +package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.columnar._ -import org.apache.spark.sql.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.execution.columnar._ +import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ import org.apache.spark.sql.types.IntegralType class IntegralDeltaSuite extends SparkFunSuite { @@ -47,8 +47,8 @@ class IntegralDeltaSuite extends SparkFunSuite { } } - input.map { value => - val row = new GenericMutableRow(1) + input.foreach { value => + val row = new GenericInternalRow(1) columnType.setField(row, 0, value) builder.appendFrom(row, 0) } @@ -95,7 +95,7 @@ class IntegralDeltaSuite extends SparkFunSuite { buffer.rewind().position(headerSize + 4) val decoder = scheme.decoder(buffer, columnType) - val mutableRow = new GenericMutableRow(1) + val mutableRow = new GenericInternalRow(1) if (input.nonEmpty) { input.foreach{ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala similarity index 91% rename from sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala index 67ec08f594a4..dffa9b364ebf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql.columnar.compression +package org.apache.spark.sql.execution.columnar.compression import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.columnar._ -import org.apache.spark.sql.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.execution.columnar._ +import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ import org.apache.spark.sql.types.AtomicType class RunLengthEncodingSuite extends SparkFunSuite { @@ -80,7 +80,7 @@ class RunLengthEncodingSuite extends SparkFunSuite { buffer.rewind().position(headerSize + 4) val decoder = RunLengthEncoding.decoder(buffer, columnType) - val mutableRow = new GenericMutableRow(1) + val mutableRow = new GenericInternalRow(1) if (inputSeq.nonEmpty) { inputSeq.foreach { i => @@ -100,11 +100,11 @@ class RunLengthEncodingSuite extends SparkFunSuite { } test(s"$RunLengthEncoding with $typeName: simple case") { - skeleton(2, Seq(0 -> 2, 1 ->2)) + skeleton(2, Seq(0 -> 2, 1 -> 2)) } test(s"$RunLengthEncoding with $typeName: run length == 1") { - skeleton(2, Seq(0 -> 1, 1 ->1)) + skeleton(2, Seq(0 -> 1, 1 -> 1)) } test(s"$RunLengthEncoding with $typeName: single long run") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/TestCompressibleColumnBuilder.scala similarity index 93% rename from sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/TestCompressibleColumnBuilder.scala index 5268dfe0aa03..5e078f251375 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/TestCompressibleColumnBuilder.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.columnar.compression +package org.apache.spark.sql.execution.columnar.compression -import org.apache.spark.sql.columnar._ +import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.types.AtomicType class TestCompressibleColumnBuilder[T <: AtomicType]( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala new file mode 100644 index 000000000000..fa5172ca8a3e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -0,0 +1,1685 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import java.net.URI +import java.util.Locale + +import scala.reflect.{classTag, ClassTag} + +import org.apache.spark.sql.{AnalysisException, SaveMode} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans +import org.apache.spark.sql.catalyst.dsl.plans.DslLogicalPlan +import org.apache.spark.sql.catalyst.expressions.JsonTuple +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{Generate, InsertIntoDir, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{Project, ScriptTransformation} +import org.apache.spark.sql.execution.SparkSqlParser +import org.apache.spark.sql.execution.datasources.CreateTable +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} + + +class DDLParserSuite extends PlanTest with SharedSQLContext { + private lazy val parser = new SparkSqlParser(new SQLConf) + + private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = { + val e = intercept[ParseException] { + parser.parsePlan(sql) + } + assert(e.getMessage.toLowerCase(Locale.ROOT).contains("operation not allowed")) + containsThesePhrases.foreach { p => + assert(e.getMessage.toLowerCase(Locale.ROOT).contains(p.toLowerCase(Locale.ROOT))) + } + } + + private def parseAs[T: ClassTag](query: String): T = { + parser.parsePlan(query) match { + case t: T => t + case other => + fail(s"Expected to parse ${classTag[T].runtimeClass} from query," + + s"got ${other.getClass.getName}: $query") + } + } + + private def compareTransformQuery(sql: String, expected: LogicalPlan): Unit = { + val plan = parser.parsePlan(sql).asInstanceOf[ScriptTransformation].copy(ioschema = null) + comparePlans(plan, expected, checkAnalysis = false) + } + + private def extractTableDesc(sql: String): (CatalogTable, Boolean) = { + parser.parsePlan(sql).collect { + case CreateTable(tableDesc, mode, _) => (tableDesc, mode == SaveMode.Ignore) + }.head + } + + test("create database") { + val sql = + """ + |CREATE DATABASE IF NOT EXISTS database_name + |COMMENT 'database_comment' LOCATION '/home/user/db' + |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') + """.stripMargin + val parsed = parser.parsePlan(sql) + val expected = CreateDatabaseCommand( + "database_name", + ifNotExists = true, + Some("/home/user/db"), + Some("database_comment"), + Map("a" -> "a", "b" -> "b", "c" -> "c")) + comparePlans(parsed, expected) + } + + test("create database - property values must be set") { + assertUnsupported( + sql = "CREATE DATABASE my_db WITH DBPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + + test("drop database") { + val sql1 = "DROP DATABASE IF EXISTS database_name RESTRICT" + val sql2 = "DROP DATABASE IF EXISTS database_name CASCADE" + val sql3 = "DROP SCHEMA IF EXISTS database_name RESTRICT" + val sql4 = "DROP SCHEMA IF EXISTS database_name CASCADE" + // The default is restrict=true + val sql5 = "DROP DATABASE IF EXISTS database_name" + // The default is ifExists=false + val sql6 = "DROP DATABASE database_name" + val sql7 = "DROP DATABASE database_name CASCADE" + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + val parsed4 = parser.parsePlan(sql4) + val parsed5 = parser.parsePlan(sql5) + val parsed6 = parser.parsePlan(sql6) + val parsed7 = parser.parsePlan(sql7) + + val expected1 = DropDatabaseCommand( + "database_name", + ifExists = true, + cascade = false) + val expected2 = DropDatabaseCommand( + "database_name", + ifExists = true, + cascade = true) + val expected3 = DropDatabaseCommand( + "database_name", + ifExists = false, + cascade = false) + val expected4 = DropDatabaseCommand( + "database_name", + ifExists = false, + cascade = true) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected1) + comparePlans(parsed4, expected2) + comparePlans(parsed5, expected1) + comparePlans(parsed6, expected3) + comparePlans(parsed7, expected4) + } + + test("alter database set dbproperties") { + // ALTER (DATABASE|SCHEMA) database_name SET DBPROPERTIES (property_name=property_value, ...) + val sql1 = "ALTER DATABASE database_name SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')" + val sql2 = "ALTER SCHEMA database_name SET DBPROPERTIES ('a'='a')" + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + + val expected1 = AlterDatabasePropertiesCommand( + "database_name", + Map("a" -> "a", "b" -> "b", "c" -> "c")) + val expected2 = AlterDatabasePropertiesCommand( + "database_name", + Map("a" -> "a")) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter database - property values must be set") { + assertUnsupported( + sql = "ALTER DATABASE my_db SET DBPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + + test("describe database") { + // DESCRIBE DATABASE [EXTENDED] db_name; + val sql1 = "DESCRIBE DATABASE EXTENDED db_name" + val sql2 = "DESCRIBE DATABASE db_name" + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + + val expected1 = DescribeDatabaseCommand( + "db_name", + extended = true) + val expected2 = DescribeDatabaseCommand( + "db_name", + extended = false) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("create function") { + val sql1 = + """ + |CREATE TEMPORARY FUNCTION helloworld as + |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', + |JAR '/path/to/jar2' + """.stripMargin + val sql2 = + """ + |CREATE FUNCTION hello.world as + |'com.matthewrathbone.example.SimpleUDFExample' USING ARCHIVE '/path/to/archive', + |FILE '/path/to/file' + """.stripMargin + val sql3 = + """ + |CREATE OR REPLACE TEMPORARY FUNCTION helloworld3 as + |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', + |JAR '/path/to/jar2' + """.stripMargin + val sql4 = + """ + |CREATE OR REPLACE FUNCTION hello.world1 as + |'com.matthewrathbone.example.SimpleUDFExample' USING ARCHIVE '/path/to/archive', + |FILE '/path/to/file' + """.stripMargin + val sql5 = + """ + |CREATE FUNCTION IF NOT EXISTS hello.world2 as + |'com.matthewrathbone.example.SimpleUDFExample' USING ARCHIVE '/path/to/archive', + |FILE '/path/to/file' + """.stripMargin + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + val parsed4 = parser.parsePlan(sql4) + val parsed5 = parser.parsePlan(sql5) + val expected1 = CreateFunctionCommand( + None, + "helloworld", + "com.matthewrathbone.example.SimpleUDFExample", + Seq( + FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar1"), + FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar2")), + isTemp = true, ifNotExists = false, replace = false) + val expected2 = CreateFunctionCommand( + Some("hello"), + "world", + "com.matthewrathbone.example.SimpleUDFExample", + Seq( + FunctionResource(FunctionResourceType.fromString("archive"), "/path/to/archive"), + FunctionResource(FunctionResourceType.fromString("file"), "/path/to/file")), + isTemp = false, ifNotExists = false, replace = false) + val expected3 = CreateFunctionCommand( + None, + "helloworld3", + "com.matthewrathbone.example.SimpleUDFExample", + Seq( + FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar1"), + FunctionResource(FunctionResourceType.fromString("jar"), "/path/to/jar2")), + isTemp = true, ifNotExists = false, replace = true) + val expected4 = CreateFunctionCommand( + Some("hello"), + "world1", + "com.matthewrathbone.example.SimpleUDFExample", + Seq( + FunctionResource(FunctionResourceType.fromString("archive"), "/path/to/archive"), + FunctionResource(FunctionResourceType.fromString("file"), "/path/to/file")), + isTemp = false, ifNotExists = false, replace = true) + val expected5 = CreateFunctionCommand( + Some("hello"), + "world2", + "com.matthewrathbone.example.SimpleUDFExample", + Seq( + FunctionResource(FunctionResourceType.fromString("archive"), "/path/to/archive"), + FunctionResource(FunctionResourceType.fromString("file"), "/path/to/file")), + isTemp = false, ifNotExists = true, replace = false) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + comparePlans(parsed5, expected5) + } + + test("drop function") { + val sql1 = "DROP TEMPORARY FUNCTION helloworld" + val sql2 = "DROP TEMPORARY FUNCTION IF EXISTS helloworld" + val sql3 = "DROP FUNCTION hello.world" + val sql4 = "DROP FUNCTION IF EXISTS hello.world" + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + val parsed4 = parser.parsePlan(sql4) + + val expected1 = DropFunctionCommand( + None, + "helloworld", + ifExists = false, + isTemp = true) + val expected2 = DropFunctionCommand( + None, + "helloworld", + ifExists = true, + isTemp = true) + val expected3 = DropFunctionCommand( + Some("hello"), + "world", + ifExists = false, + isTemp = false) + val expected4 = DropFunctionCommand( + Some("hello"), + "world", + ifExists = true, + isTemp = false) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + } + + test("create hive table - table file format") { + val allSources = Seq("parquet", "parquetfile", "orc", "orcfile", "avro", "avrofile", + "sequencefile", "rcfile", "textfile") + + allSources.foreach { s => + val query = s"CREATE TABLE my_tab STORED AS $s" + val ct = parseAs[CreateTable](query) + val hiveSerde = HiveSerDe.sourceToSerDe(s) + assert(hiveSerde.isDefined) + assert(ct.tableDesc.storage.serde == + hiveSerde.get.serde.orElse(Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"))) + assert(ct.tableDesc.storage.inputFormat == hiveSerde.get.inputFormat) + assert(ct.tableDesc.storage.outputFormat == hiveSerde.get.outputFormat) + } + } + + test("create hive table - row format and table file format") { + val createTableStart = "CREATE TABLE my_tab ROW FORMAT" + val fileFormat = s"STORED AS INPUTFORMAT 'inputfmt' OUTPUTFORMAT 'outputfmt'" + val query1 = s"$createTableStart SERDE 'anything' $fileFormat" + val query2 = s"$createTableStart DELIMITED FIELDS TERMINATED BY ' ' $fileFormat" + + // No conflicting serdes here, OK + val parsed1 = parseAs[CreateTable](query1) + assert(parsed1.tableDesc.storage.serde == Some("anything")) + assert(parsed1.tableDesc.storage.inputFormat == Some("inputfmt")) + assert(parsed1.tableDesc.storage.outputFormat == Some("outputfmt")) + + val parsed2 = parseAs[CreateTable](query2) + assert(parsed2.tableDesc.storage.serde == + Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) + assert(parsed2.tableDesc.storage.inputFormat == Some("inputfmt")) + assert(parsed2.tableDesc.storage.outputFormat == Some("outputfmt")) + } + + test("create hive table - row format serde and generic file format") { + val allSources = Seq("parquet", "orc", "avro", "sequencefile", "rcfile", "textfile") + val supportedSources = Set("sequencefile", "rcfile", "textfile") + + allSources.foreach { s => + val query = s"CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS $s" + if (supportedSources.contains(s)) { + val ct = parseAs[CreateTable](query) + val hiveSerde = HiveSerDe.sourceToSerDe(s) + assert(hiveSerde.isDefined) + assert(ct.tableDesc.storage.serde == Some("anything")) + assert(ct.tableDesc.storage.inputFormat == hiveSerde.get.inputFormat) + assert(ct.tableDesc.storage.outputFormat == hiveSerde.get.outputFormat) + } else { + assertUnsupported(query, Seq("row format serde", "incompatible", s)) + } + } + } + + test("create hive table - row format delimited and generic file format") { + val allSources = Seq("parquet", "orc", "avro", "sequencefile", "rcfile", "textfile") + val supportedSources = Set("textfile") + + allSources.foreach { s => + val query = s"CREATE TABLE my_tab ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS $s" + if (supportedSources.contains(s)) { + val ct = parseAs[CreateTable](query) + val hiveSerde = HiveSerDe.sourceToSerDe(s) + assert(hiveSerde.isDefined) + assert(ct.tableDesc.storage.serde == + hiveSerde.get.serde.orElse(Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"))) + assert(ct.tableDesc.storage.inputFormat == hiveSerde.get.inputFormat) + assert(ct.tableDesc.storage.outputFormat == hiveSerde.get.outputFormat) + } else { + assertUnsupported(query, Seq("row format delimited", "only compatible with 'textfile'", s)) + } + } + } + + test("create hive external table - location must be specified") { + assertUnsupported( + sql = "CREATE EXTERNAL TABLE my_tab", + containsThesePhrases = Seq("create external table", "location")) + val query = "CREATE EXTERNAL TABLE my_tab LOCATION '/something/anything'" + val ct = parseAs[CreateTable](query) + assert(ct.tableDesc.tableType == CatalogTableType.EXTERNAL) + assert(ct.tableDesc.storage.locationUri == Some(new URI("/something/anything"))) + } + + test("create hive table - property values must be set") { + assertUnsupported( + sql = "CREATE TABLE my_tab TBLPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + assertUnsupported( + sql = "CREATE TABLE my_tab ROW FORMAT SERDE 'serde' " + + "WITH SERDEPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + + test("create hive table - location implies external") { + val query = "CREATE TABLE my_tab LOCATION '/something/anything'" + val ct = parseAs[CreateTable](query) + assert(ct.tableDesc.tableType == CatalogTableType.EXTERNAL) + assert(ct.tableDesc.storage.locationUri == Some(new URI("/something/anything"))) + } + + test("create table - with partitioned by") { + val query = "CREATE TABLE my_tab(a INT comment 'test', b STRING) " + + "USING parquet PARTITIONED BY (a)" + + val expectedTableDesc = CatalogTable( + identifier = TableIdentifier("my_tab"), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType() + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType), + provider = Some("parquet"), + partitionColumnNames = Seq("a") + ) + + parser.parsePlan(query) match { + case CreateTable(tableDesc, _, None) => + assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) + case other => + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + + s"got ${other.getClass.getName}: $query") + } + } + + test("create table - with bucket") { + val query = "CREATE TABLE my_tab(a INT, b STRING) USING parquet " + + "CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS" + + val expectedTableDesc = CatalogTable( + identifier = TableIdentifier("my_tab"), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType().add("a", IntegerType).add("b", StringType), + provider = Some("parquet"), + bucketSpec = Some(BucketSpec(5, Seq("a"), Seq("b"))) + ) + + parser.parsePlan(query) match { + case CreateTable(tableDesc, _, None) => + assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) + case other => + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + + s"got ${other.getClass.getName}: $query") + } + } + + test("create table - with comment") { + val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet COMMENT 'abc'" + + val expectedTableDesc = CatalogTable( + identifier = TableIdentifier("my_tab"), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType().add("a", IntegerType).add("b", StringType), + provider = Some("parquet"), + comment = Some("abc")) + + parser.parsePlan(sql) match { + case CreateTable(tableDesc, _, None) => + assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) + case other => + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("create table - with table properties") { + val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet TBLPROPERTIES('test' = 'test')" + + val expectedTableDesc = CatalogTable( + identifier = TableIdentifier("my_tab"), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType().add("a", IntegerType).add("b", StringType), + provider = Some("parquet"), + properties = Map("test" -> "test")) + + parser.parsePlan(sql) match { + case CreateTable(tableDesc, _, None) => + assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) + case other => + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("create table - with location") { + val v1 = "CREATE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" + + val expectedTableDesc = CatalogTable( + identifier = TableIdentifier("my_tab"), + tableType = CatalogTableType.EXTERNAL, + storage = CatalogStorageFormat.empty.copy(locationUri = Some(new URI("/tmp/file"))), + schema = new StructType().add("a", IntegerType).add("b", StringType), + provider = Some("parquet")) + + parser.parsePlan(v1) match { + case CreateTable(tableDesc, _, None) => + assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) + case other => + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + + s"got ${other.getClass.getName}: $v1") + } + + val v2 = + """ + |CREATE TABLE my_tab(a INT, b STRING) + |USING parquet + |OPTIONS (path '/tmp/file') + |LOCATION '/tmp/file' + """.stripMargin + val e = intercept[ParseException] { + parser.parsePlan(v2) + } + assert(e.message.contains("you can only specify one of them.")) + } + + test("insert overwrite directory") { + val v1 = "INSERT OVERWRITE DIRECTORY '/tmp/file' USING parquet SELECT 1 as a" + parser.parsePlan(v1) match { + case InsertIntoDir(_, storage, provider, query, overwrite) => + assert(storage.locationUri.isDefined && storage.locationUri.get.toString == "/tmp/file") + case other => + fail(s"Expected to parse ${classOf[InsertIntoDataSourceDirCommand].getClass.getName}" + + " from query," + s" got ${other.getClass.getName}: $v1") + } + + val v2 = "INSERT OVERWRITE DIRECTORY USING parquet SELECT 1 as a" + val e2 = intercept[ParseException] { + parser.parsePlan(v2) + } + assert(e2.message.contains( + "Directory path and 'path' in OPTIONS should be specified one, but not both")) + + val v3 = + """ + | INSERT OVERWRITE DIRECTORY USING json + | OPTIONS ('path' '/tmp/file', a 1, b 0.1, c TRUE) + | SELECT 1 as a + """.stripMargin + parser.parsePlan(v3) match { + case InsertIntoDir(_, storage, provider, query, overwrite) => + assert(storage.locationUri.isDefined && provider == Some("json")) + assert(storage.properties.get("a") == Some("1")) + assert(storage.properties.get("b") == Some("0.1")) + assert(storage.properties.get("c") == Some("true")) + assert(!storage.properties.contains("abc")) + assert(!storage.properties.contains("path")) + case other => + fail(s"Expected to parse ${classOf[InsertIntoDataSourceDirCommand].getClass.getName}" + + " from query," + s"got ${other.getClass.getName}: $v1") + } + + val v4 = + """ + | INSERT OVERWRITE DIRECTORY '/tmp/file' USING json + | OPTIONS ('path' '/tmp/file', a 1, b 0.1, c TRUE) + | SELECT 1 as a + """.stripMargin + val e4 = intercept[ParseException] { + parser.parsePlan(v4) + } + assert(e4.message.contains( + "Directory path and 'path' in OPTIONS should be specified one, but not both")) + } + + // ALTER TABLE table_name RENAME TO new_table_name; + // ALTER VIEW view_name RENAME TO new_view_name; + test("alter table/view: rename table/view") { + val sql_table = "ALTER TABLE table_name RENAME TO new_table_name" + val sql_view = sql_table.replace("TABLE", "VIEW") + val parsed_table = parser.parsePlan(sql_table) + val parsed_view = parser.parsePlan(sql_view) + val expected_table = AlterTableRenameCommand( + TableIdentifier("table_name"), + TableIdentifier("new_table_name"), + isView = false) + val expected_view = AlterTableRenameCommand( + TableIdentifier("table_name"), + TableIdentifier("new_table_name"), + isView = true) + comparePlans(parsed_table, expected_table) + comparePlans(parsed_view, expected_view) + } + + test("alter table: rename table with database") { + val query = "ALTER TABLE db1.tbl RENAME TO db1.tbl2" + val plan = parseAs[AlterTableRenameCommand](query) + assert(plan.oldName == TableIdentifier("tbl", Some("db1"))) + assert(plan.newName == TableIdentifier("tbl2", Some("db1"))) + } + + // ALTER TABLE table_name SET TBLPROPERTIES ('comment' = new_comment); + // ALTER TABLE table_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); + // ALTER VIEW view_name SET TBLPROPERTIES ('comment' = new_comment); + // ALTER VIEW view_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); + test("alter table/view: alter table/view properties") { + val sql1_table = "ALTER TABLE table_name SET TBLPROPERTIES ('test' = 'test', " + + "'comment' = 'new_comment')" + val sql2_table = "ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 'test')" + val sql3_table = "ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')" + val sql1_view = sql1_table.replace("TABLE", "VIEW") + val sql2_view = sql2_table.replace("TABLE", "VIEW") + val sql3_view = sql3_table.replace("TABLE", "VIEW") + + val parsed1_table = parser.parsePlan(sql1_table) + val parsed2_table = parser.parsePlan(sql2_table) + val parsed3_table = parser.parsePlan(sql3_table) + val parsed1_view = parser.parsePlan(sql1_view) + val parsed2_view = parser.parsePlan(sql2_view) + val parsed3_view = parser.parsePlan(sql3_view) + + val tableIdent = TableIdentifier("table_name", None) + val expected1_table = AlterTableSetPropertiesCommand( + tableIdent, Map("test" -> "test", "comment" -> "new_comment"), isView = false) + val expected2_table = AlterTableUnsetPropertiesCommand( + tableIdent, Seq("comment", "test"), ifExists = false, isView = false) + val expected3_table = AlterTableUnsetPropertiesCommand( + tableIdent, Seq("comment", "test"), ifExists = true, isView = false) + val expected1_view = expected1_table.copy(isView = true) + val expected2_view = expected2_table.copy(isView = true) + val expected3_view = expected3_table.copy(isView = true) + + comparePlans(parsed1_table, expected1_table) + comparePlans(parsed2_table, expected2_table) + comparePlans(parsed3_table, expected3_table) + comparePlans(parsed1_view, expected1_view) + comparePlans(parsed2_view, expected2_view) + comparePlans(parsed3_view, expected3_view) + } + + test("alter table - property values must be set") { + assertUnsupported( + sql = "ALTER TABLE my_tab SET TBLPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + + test("alter table unset properties - property values must NOT be set") { + assertUnsupported( + sql = "ALTER TABLE my_tab UNSET TBLPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_with_value")) + } + + test("alter table: SerDe properties") { + val sql1 = "ALTER TABLE table_name SET SERDE 'org.apache.class'" + val sql2 = + """ + |ALTER TABLE table_name SET SERDE 'org.apache.class' + |WITH SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',') + """.stripMargin + val sql3 = + """ + |ALTER TABLE table_name SET SERDEPROPERTIES ('columns'='foo,bar', + |'field.delim' = ',') + """.stripMargin + val sql4 = + """ + |ALTER TABLE table_name PARTITION (test=1, dt='2008-08-08', + |country='us') SET SERDE 'org.apache.class' WITH SERDEPROPERTIES ('columns'='foo,bar', + |'field.delim' = ',') + """.stripMargin + val sql5 = + """ + |ALTER TABLE table_name PARTITION (test=1, dt='2008-08-08', + |country='us') SET SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',') + """.stripMargin + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + val parsed4 = parser.parsePlan(sql4) + val parsed5 = parser.parsePlan(sql5) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableSerDePropertiesCommand( + tableIdent, Some("org.apache.class"), None, None) + val expected2 = AlterTableSerDePropertiesCommand( + tableIdent, + Some("org.apache.class"), + Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), + None) + val expected3 = AlterTableSerDePropertiesCommand( + tableIdent, None, Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), None) + val expected4 = AlterTableSerDePropertiesCommand( + tableIdent, + Some("org.apache.class"), + Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), + Some(Map("test" -> "1", "dt" -> "2008-08-08", "country" -> "us"))) + val expected5 = AlterTableSerDePropertiesCommand( + tableIdent, + None, + Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), + Some(Map("test" -> "1", "dt" -> "2008-08-08", "country" -> "us"))) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + comparePlans(parsed5, expected5) + } + + test("alter table - SerDe property values must be set") { + assertUnsupported( + sql = "ALTER TABLE my_tab SET SERDE 'serde' " + + "WITH SERDEPROPERTIES('key_without_value', 'key_with_value'='x')", + containsThesePhrases = Seq("key_without_value")) + } + + // ALTER TABLE table_name ADD [IF NOT EXISTS] PARTITION partition_spec + // [LOCATION 'location1'] partition_spec [LOCATION 'location2'] ...; + test("alter table: add partition") { + val sql1 = + """ + |ALTER TABLE table_name ADD IF NOT EXISTS PARTITION + |(dt='2008-08-08', country='us') LOCATION 'location1' PARTITION + |(dt='2009-09-09', country='uk') + """.stripMargin + val sql2 = "ALTER TABLE table_name ADD PARTITION (dt='2008-08-08') LOCATION 'loc'" + + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + + val expected1 = AlterTableAddPartitionCommand( + TableIdentifier("table_name", None), + Seq( + (Map("dt" -> "2008-08-08", "country" -> "us"), Some("location1")), + (Map("dt" -> "2009-09-09", "country" -> "uk"), None)), + ifNotExists = true) + val expected2 = AlterTableAddPartitionCommand( + TableIdentifier("table_name", None), + Seq((Map("dt" -> "2008-08-08"), Some("loc"))), + ifNotExists = false) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: recover partitions") { + val sql = "ALTER TABLE table_name RECOVER PARTITIONS" + val parsed = parser.parsePlan(sql) + val expected = AlterTableRecoverPartitionsCommand( + TableIdentifier("table_name", None)) + comparePlans(parsed, expected) + } + + test("alter view: add partition (not supported)") { + assertUnsupported( + """ + |ALTER VIEW view_name ADD IF NOT EXISTS PARTITION + |(dt='2008-08-08', country='us') PARTITION + |(dt='2009-09-09', country='uk') + """.stripMargin) + } + + test("alter table: rename partition") { + val sql = + """ + |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') + |RENAME TO PARTITION (dt='2008-09-09', country='uk') + """.stripMargin + val parsed = parser.parsePlan(sql) + val expected = AlterTableRenamePartitionCommand( + TableIdentifier("table_name", None), + Map("dt" -> "2008-08-08", "country" -> "us"), + Map("dt" -> "2008-09-09", "country" -> "uk")) + comparePlans(parsed, expected) + } + + test("alter table: exchange partition (not supported)") { + assertUnsupported( + """ + |ALTER TABLE table_name_1 EXCHANGE PARTITION + |(dt='2008-08-08', country='us') WITH TABLE table_name_2 + """.stripMargin) + } + + // ALTER TABLE table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] + // ALTER VIEW table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] + test("alter table/view: drop partitions") { + val sql1_table = + """ + |ALTER TABLE table_name DROP IF EXISTS PARTITION + |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') + """.stripMargin + val sql2_table = + """ + |ALTER TABLE table_name DROP PARTITION + |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') + """.stripMargin + val sql1_view = sql1_table.replace("TABLE", "VIEW") + val sql2_view = sql2_table.replace("TABLE", "VIEW") + + val parsed1_table = parser.parsePlan(sql1_table) + val parsed2_table = parser.parsePlan(sql2_table) + val parsed1_purge = parser.parsePlan(sql1_table + " PURGE") + assertUnsupported(sql1_view) + assertUnsupported(sql2_view) + + val tableIdent = TableIdentifier("table_name", None) + val expected1_table = AlterTableDropPartitionCommand( + tableIdent, + Seq( + Map("dt" -> "2008-08-08", "country" -> "us"), + Map("dt" -> "2009-09-09", "country" -> "uk")), + ifExists = true, + purge = false, + retainData = false) + val expected2_table = expected1_table.copy(ifExists = false) + val expected1_purge = expected1_table.copy(purge = true) + + comparePlans(parsed1_table, expected1_table) + comparePlans(parsed2_table, expected2_table) + comparePlans(parsed1_purge, expected1_purge) + } + + test("alter table: archive partition (not supported)") { + assertUnsupported("ALTER TABLE table_name ARCHIVE PARTITION (dt='2008-08-08', country='us')") + } + + test("alter table: unarchive partition (not supported)") { + assertUnsupported("ALTER TABLE table_name UNARCHIVE PARTITION (dt='2008-08-08', country='us')") + } + + test("alter table: set file format (not allowed)") { + assertUnsupported( + "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' OUTPUTFORMAT 'test'") + assertUnsupported( + "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + + "SET FILEFORMAT PARQUET") + } + + test("alter table: set location") { + val sql1 = "ALTER TABLE table_name SET LOCATION 'new location'" + val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + + "SET LOCATION 'new location'" + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableSetLocationCommand( + tableIdent, + None, + "new location") + val expected2 = AlterTableSetLocationCommand( + tableIdent, + Some(Map("dt" -> "2008-08-08", "country" -> "us")), + "new location") + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: change column name/type/comment") { + val sql1 = "ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT" + val sql2 = "ALTER TABLE table_name CHANGE COLUMN col_name col_name INT COMMENT 'new_comment'" + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableChangeColumnCommand( + tableIdent, + "col_old_name", + StructField("col_new_name", IntegerType)) + val expected2 = AlterTableChangeColumnCommand( + tableIdent, + "col_name", + StructField("col_name", IntegerType).withComment("new_comment")) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: change column position (not supported)") { + assertUnsupported("ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT FIRST") + assertUnsupported( + "ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT AFTER other_col") + } + + test("alter table: change column in partition spec") { + assertUnsupported("ALTER TABLE table_name PARTITION (a='1', a='2') CHANGE COLUMN a new_a INT") + } + + test("alter table: touch (not supported)") { + assertUnsupported("ALTER TABLE table_name TOUCH") + assertUnsupported("ALTER TABLE table_name TOUCH PARTITION (dt='2008-08-08', country='us')") + } + + test("alter table: compact (not supported)") { + assertUnsupported("ALTER TABLE table_name COMPACT 'compaction_type'") + assertUnsupported( + """ + |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') + |COMPACT 'MAJOR' + """.stripMargin) + } + + test("alter table: concatenate (not supported)") { + assertUnsupported("ALTER TABLE table_name CONCATENATE") + assertUnsupported( + "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') CONCATENATE") + } + + test("alter table: cluster by (not supported)") { + assertUnsupported( + "ALTER TABLE table_name CLUSTERED BY (col_name) SORTED BY (col2_name) INTO 3 BUCKETS") + assertUnsupported("ALTER TABLE table_name CLUSTERED BY (col_name) INTO 3 BUCKETS") + assertUnsupported("ALTER TABLE table_name NOT CLUSTERED") + assertUnsupported("ALTER TABLE table_name NOT SORTED") + } + + test("alter table: skewed by (not supported)") { + assertUnsupported("ALTER TABLE table_name NOT SKEWED") + assertUnsupported("ALTER TABLE table_name NOT STORED AS DIRECTORIES") + assertUnsupported("ALTER TABLE table_name SET SKEWED LOCATION (col_name1=\"location1\"") + assertUnsupported("ALTER TABLE table_name SKEWED BY (key) ON (1,5,6) STORED AS DIRECTORIES") + } + + test("alter table: replace columns (not allowed)") { + assertUnsupported( + """ + |ALTER TABLE table_name REPLACE COLUMNS (new_col1 INT + |COMMENT 'test_comment', new_col2 LONG COMMENT 'test_comment2') RESTRICT + """.stripMargin) + } + + test("show databases") { + val sql1 = "SHOW DATABASES" + val sql2 = "SHOW DATABASES LIKE 'defau*'" + val parsed1 = parser.parsePlan(sql1) + val expected1 = ShowDatabasesCommand(None) + val parsed2 = parser.parsePlan(sql2) + val expected2 = ShowDatabasesCommand(Some("defau*")) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("show tblproperties") { + val parsed1 = parser.parsePlan("SHOW TBLPROPERTIES tab1") + val expected1 = ShowTablePropertiesCommand(TableIdentifier("tab1", None), None) + val parsed2 = parser.parsePlan("SHOW TBLPROPERTIES tab1('propKey1')") + val expected2 = ShowTablePropertiesCommand(TableIdentifier("tab1", None), Some("propKey1")) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("SPARK-14383: DISTRIBUTE and UNSET as non-keywords") { + val sql = "SELECT distribute, unset FROM x" + val parsed = parser.parsePlan(sql) + assert(parsed.isInstanceOf[Project]) + } + + test("duplicate keys in table properties") { + val e = intercept[ParseException] { + parser.parsePlan("ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('key1' = '1', 'key1' = '2')") + }.getMessage + assert(e.contains("Found duplicate keys 'key1'")) + } + + test("duplicate columns in partition specs") { + val e = intercept[ParseException] { + parser.parsePlan( + "ALTER TABLE dbx.tab1 PARTITION (a='1', a='2') RENAME TO PARTITION (a='100', a='200')") + }.getMessage + assert(e.contains("Found duplicate keys 'a'")) + } + + test("empty values in non-optional partition specs") { + val e = intercept[ParseException] { + parser.parsePlan( + "SHOW PARTITIONS dbx.tab1 PARTITION (a='1', b)") + }.getMessage + assert(e.contains("Found an empty partition key 'b'")) + } + + test("drop table") { + val tableName1 = "db.tab" + val tableName2 = "tab" + + val parsed = Seq( + s"DROP TABLE $tableName1", + s"DROP TABLE IF EXISTS $tableName1", + s"DROP TABLE $tableName2", + s"DROP TABLE IF EXISTS $tableName2", + s"DROP TABLE $tableName2 PURGE", + s"DROP TABLE IF EXISTS $tableName2 PURGE" + ).map(parser.parsePlan) + + val expected = Seq( + DropTableCommand(TableIdentifier("tab", Option("db")), ifExists = false, isView = false, + purge = false), + DropTableCommand(TableIdentifier("tab", Option("db")), ifExists = true, isView = false, + purge = false), + DropTableCommand(TableIdentifier("tab", None), ifExists = false, isView = false, + purge = false), + DropTableCommand(TableIdentifier("tab", None), ifExists = true, isView = false, + purge = false), + DropTableCommand(TableIdentifier("tab", None), ifExists = false, isView = false, + purge = true), + DropTableCommand(TableIdentifier("tab", None), ifExists = true, isView = false, + purge = true)) + + parsed.zip(expected).foreach { case (p, e) => comparePlans(p, e) } + } + + test("drop view") { + val viewName1 = "db.view" + val viewName2 = "view" + + val parsed1 = parser.parsePlan(s"DROP VIEW $viewName1") + val parsed2 = parser.parsePlan(s"DROP VIEW IF EXISTS $viewName1") + val parsed3 = parser.parsePlan(s"DROP VIEW $viewName2") + val parsed4 = parser.parsePlan(s"DROP VIEW IF EXISTS $viewName2") + + val expected1 = + DropTableCommand(TableIdentifier("view", Option("db")), ifExists = false, isView = true, + purge = false) + val expected2 = + DropTableCommand(TableIdentifier("view", Option("db")), ifExists = true, isView = true, + purge = false) + val expected3 = + DropTableCommand(TableIdentifier("view", None), ifExists = false, isView = true, + purge = false) + val expected4 = + DropTableCommand(TableIdentifier("view", None), ifExists = true, isView = true, + purge = false) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + } + + test("show columns") { + val sql1 = "SHOW COLUMNS FROM t1" + val sql2 = "SHOW COLUMNS IN db1.t1" + val sql3 = "SHOW COLUMNS FROM t1 IN db1" + val sql4 = "SHOW COLUMNS FROM db1.t1 IN db2" + + val parsed1 = parser.parsePlan(sql1) + val expected1 = ShowColumnsCommand(None, TableIdentifier("t1", None)) + val parsed2 = parser.parsePlan(sql2) + val expected2 = ShowColumnsCommand(None, TableIdentifier("t1", Some("db1"))) + val parsed3 = parser.parsePlan(sql3) + val expected3 = ShowColumnsCommand(Some("db1"), TableIdentifier("t1", None)) + val parsed4 = parser.parsePlan(sql4) + val expected4 = ShowColumnsCommand(Some("db2"), TableIdentifier("t1", Some("db1"))) + + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + } + + + test("show partitions") { + val sql1 = "SHOW PARTITIONS t1" + val sql2 = "SHOW PARTITIONS db1.t1" + val sql3 = "SHOW PARTITIONS t1 PARTITION(partcol1='partvalue', partcol2='partvalue')" + + val parsed1 = parser.parsePlan(sql1) + val expected1 = + ShowPartitionsCommand(TableIdentifier("t1", None), None) + val parsed2 = parser.parsePlan(sql2) + val expected2 = + ShowPartitionsCommand(TableIdentifier("t1", Some("db1")), None) + val expected3 = + ShowPartitionsCommand(TableIdentifier("t1", None), + Some(Map("partcol1" -> "partvalue", "partcol2" -> "partvalue"))) + val parsed3 = parser.parsePlan(sql3) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + } + + test("support for other types in DBPROPERTIES") { + val sql = + """ + |CREATE DATABASE database_name + |LOCATION '/home/user/db' + |WITH DBPROPERTIES ('a'=1, 'b'=0.1, 'c'=TRUE) + """.stripMargin + val parsed = parser.parsePlan(sql) + val expected = CreateDatabaseCommand( + "database_name", + ifNotExists = false, + Some("/home/user/db"), + None, + Map("a" -> "1", "b" -> "0.1", "c" -> "true")) + + comparePlans(parsed, expected) + } + + test("support for other types in TBLPROPERTIES") { + val sql = + """ + |ALTER TABLE table_name + |SET TBLPROPERTIES ('a' = 1, 'b' = 0.1, 'c' = TRUE) + """.stripMargin + val parsed = parser.parsePlan(sql) + val expected = AlterTableSetPropertiesCommand( + TableIdentifier("table_name"), + Map("a" -> "1", "b" -> "0.1", "c" -> "true"), + isView = false) + + comparePlans(parsed, expected) + } + + test("support for other types in OPTIONS") { + val sql = + """ + |CREATE TABLE table_name USING json + |OPTIONS (a 1, b 0.1, c TRUE) + """.stripMargin + + val expectedTableDesc = CatalogTable( + identifier = TableIdentifier("table_name"), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty.copy( + properties = Map("a" -> "1", "b" -> "0.1", "c" -> "true") + ), + schema = new StructType, + provider = Some("json") + ) + + parser.parsePlan(sql) match { + case CreateTable(tableDesc, _, None) => + assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) + case other => + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("Test CTAS #1") { + val s1 = + """CREATE EXTERNAL TABLE IF NOT EXISTS mydb.page_view + |COMMENT 'This is the staging page view table' + |STORED AS RCFILE + |LOCATION '/user/external/page_view' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |AS SELECT * FROM src""".stripMargin + + val (desc, exists) = extractTableDesc(s1) + assert(exists) + assert(desc.identifier.database == Some("mydb")) + assert(desc.identifier.table == "page_view") + assert(desc.tableType == CatalogTableType.EXTERNAL) + assert(desc.storage.locationUri == Some(new URI("/user/external/page_view"))) + assert(desc.schema.isEmpty) // will be populated later when the table is actually created + assert(desc.comment == Some("This is the staging page view table")) + // TODO will be SQLText + assert(desc.viewText.isEmpty) + assert(desc.viewDefaultDatabase.isEmpty) + assert(desc.viewQueryColumnNames.isEmpty) + assert(desc.partitionColumnNames.isEmpty) + assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) + assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) + assert(desc.storage.serde == + Some("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) + assert(desc.properties == Map("p1" -> "v1", "p2" -> "v2")) + } + + test("Test CTAS #2") { + val s2 = + """CREATE EXTERNAL TABLE IF NOT EXISTS mydb.page_view + |COMMENT 'This is the staging page view table' + |ROW FORMAT SERDE 'parquet.hive.serde.ParquetHiveSerDe' + | STORED AS + | INPUTFORMAT 'parquet.hive.DeprecatedParquetInputFormat' + | OUTPUTFORMAT 'parquet.hive.DeprecatedParquetOutputFormat' + |LOCATION '/user/external/page_view' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |AS SELECT * FROM src""".stripMargin + + val (desc, exists) = extractTableDesc(s2) + assert(exists) + assert(desc.identifier.database == Some("mydb")) + assert(desc.identifier.table == "page_view") + assert(desc.tableType == CatalogTableType.EXTERNAL) + assert(desc.storage.locationUri == Some(new URI("/user/external/page_view"))) + assert(desc.schema.isEmpty) // will be populated later when the table is actually created + // TODO will be SQLText + assert(desc.comment == Some("This is the staging page view table")) + assert(desc.viewText.isEmpty) + assert(desc.viewDefaultDatabase.isEmpty) + assert(desc.viewQueryColumnNames.isEmpty) + assert(desc.partitionColumnNames.isEmpty) + assert(desc.storage.properties == Map()) + assert(desc.storage.inputFormat == Some("parquet.hive.DeprecatedParquetInputFormat")) + assert(desc.storage.outputFormat == Some("parquet.hive.DeprecatedParquetOutputFormat")) + assert(desc.storage.serde == Some("parquet.hive.serde.ParquetHiveSerDe")) + assert(desc.properties == Map("p1" -> "v1", "p2" -> "v2")) + } + + test("Test CTAS #3") { + val s3 = """CREATE TABLE page_view AS SELECT * FROM src""" + val (desc, exists) = extractTableDesc(s3) + assert(exists == false) + assert(desc.identifier.database == None) + assert(desc.identifier.table == "page_view") + assert(desc.tableType == CatalogTableType.MANAGED) + assert(desc.storage.locationUri == None) + assert(desc.schema.isEmpty) + assert(desc.viewText == None) // TODO will be SQLText + assert(desc.viewDefaultDatabase.isEmpty) + assert(desc.viewQueryColumnNames.isEmpty) + assert(desc.storage.properties == Map()) + assert(desc.storage.inputFormat == Some("org.apache.hadoop.mapred.TextInputFormat")) + assert(desc.storage.outputFormat == + Some("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat")) + assert(desc.storage.serde == Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) + assert(desc.properties == Map()) + } + + test("Test CTAS #4") { + val s4 = + """CREATE TABLE page_view + |STORED BY 'storage.handler.class.name' AS SELECT * FROM src""".stripMargin + intercept[AnalysisException] { + extractTableDesc(s4) + } + } + + test("Test CTAS #5") { + val s5 = """CREATE TABLE ctas2 + | ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" + | WITH SERDEPROPERTIES("serde_p1"="p1","serde_p2"="p2") + | STORED AS RCFile + | TBLPROPERTIES("tbl_p1"="p11", "tbl_p2"="p22") + | AS + | SELECT key, value + | FROM src + | ORDER BY key, value""".stripMargin + val (desc, exists) = extractTableDesc(s5) + assert(exists == false) + assert(desc.identifier.database == None) + assert(desc.identifier.table == "ctas2") + assert(desc.tableType == CatalogTableType.MANAGED) + assert(desc.storage.locationUri == None) + assert(desc.schema.isEmpty) + assert(desc.viewText == None) // TODO will be SQLText + assert(desc.viewDefaultDatabase.isEmpty) + assert(desc.viewQueryColumnNames.isEmpty) + assert(desc.storage.properties == Map(("serde_p1" -> "p1"), ("serde_p2" -> "p2"))) + assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) + assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) + assert(desc.storage.serde == Some("org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe")) + assert(desc.properties == Map(("tbl_p1" -> "p11"), ("tbl_p2" -> "p22"))) + } + + test("CTAS statement with a PARTITIONED BY clause is not allowed") { + assertUnsupported(s"CREATE TABLE ctas1 PARTITIONED BY (k int)" + + " AS SELECT key, value FROM (SELECT 1 as key, 2 as value) tmp") + } + + test("CTAS statement with schema") { + assertUnsupported(s"CREATE TABLE ctas1 (age INT, name STRING) AS SELECT * FROM src") + assertUnsupported(s"CREATE TABLE ctas1 (age INT, name STRING) AS SELECT 1, 'hello'") + } + + test("unsupported operations") { + intercept[ParseException] { + parser.parsePlan( + """ + |CREATE TEMPORARY TABLE ctas2 + |ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" + |WITH SERDEPROPERTIES("serde_p1"="p1","serde_p2"="p2") + |STORED AS RCFile + |TBLPROPERTIES("tbl_p1"="p11", "tbl_p2"="p22") + |AS SELECT key, value FROM src ORDER BY key, value + """.stripMargin) + } + intercept[ParseException] { + parser.parsePlan( + """ + |CREATE TABLE user_info_bucketed(user_id BIGINT, firstname STRING, lastname STRING) + |CLUSTERED BY(user_id) INTO 256 BUCKETS + |AS SELECT key, value FROM src ORDER BY key, value + """.stripMargin) + } + intercept[ParseException] { + parser.parsePlan( + """ + |CREATE TABLE user_info_bucketed(user_id BIGINT, firstname STRING, lastname STRING) + |SKEWED BY (key) ON (1,5,6) + |AS SELECT key, value FROM src ORDER BY key, value + """.stripMargin) + } + intercept[ParseException] { + parser.parsePlan( + """ + |SELECT TRANSFORM (key, value) USING 'cat' AS (tKey, tValue) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.TypedBytesSerDe' + |RECORDREADER 'org.apache.hadoop.hive.contrib.util.typedbytes.TypedBytesRecordReader' + |FROM testData + """.stripMargin) + } + } + + test("Invalid interval term should throw AnalysisException") { + def assertError(sql: String, errorMessage: String): Unit = { + val e = intercept[AnalysisException] { + parser.parsePlan(sql) + } + assert(e.getMessage.contains(errorMessage)) + } + assertError("select interval '42-32' year to month", + "month 32 outside range [0, 11]") + assertError("select interval '5 49:12:15' day to second", + "hour 49 outside range [0, 23]") + assertError("select interval '.1111111111' second", + "nanosecond 1111111111 outside range") + } + + test("use native json_tuple instead of hive's UDTF in LATERAL VIEW") { + val analyzer = spark.sessionState.analyzer + val plan = analyzer.execute(parser.parsePlan( + """ + |SELECT * + |FROM (SELECT '{"f1": "value1", "f2": 12}' json) test + |LATERAL VIEW json_tuple(json, 'f1', 'f2') jt AS a, b + """.stripMargin)) + + assert(plan.children.head.asInstanceOf[Generate].generator.isInstanceOf[JsonTuple]) + } + + test("transform query spec") { + val p = ScriptTransformation( + Seq(UnresolvedAttribute("a"), UnresolvedAttribute("b")), + "func", Seq.empty, plans.table("e"), null) + + compareTransformQuery("select transform(a, b) using 'func' from e where f < 10", + p.copy(child = p.child.where('f < 10), output = Seq('key.string, 'value.string))) + compareTransformQuery("map a, b using 'func' as c, d from e", + p.copy(output = Seq('c.string, 'd.string))) + compareTransformQuery("reduce a, b using 'func' as (c int, d decimal(10, 0)) from e", + p.copy(output = Seq('c.int, 'd.decimal(10, 0)))) + } + + test("use backticks in output of Script Transform") { + parser.parsePlan( + """SELECT `t`.`thing1` + |FROM (SELECT TRANSFORM (`parquet_t1`.`key`, `parquet_t1`.`value`) + |USING 'cat' AS (`thing1` int, `thing2` string) FROM `default`.`parquet_t1`) AS t + """.stripMargin) + } + + test("use backticks in output of Generator") { + parser.parsePlan( + """ + |SELECT `gentab2`.`gencol2` + |FROM `default`.`src` + |LATERAL VIEW explode(array(array(1, 2, 3))) `gentab1` AS `gencol1` + |LATERAL VIEW explode(`gentab1`.`gencol1`) `gentab2` AS `gencol2` + """.stripMargin) + } + + test("use escaped backticks in output of Generator") { + parser.parsePlan( + """ + |SELECT `gen``tab2`.`gen``col2` + |FROM `default`.`src` + |LATERAL VIEW explode(array(array(1, 2, 3))) `gen``tab1` AS `gen``col1` + |LATERAL VIEW explode(`gen``tab1`.`gen``col1`) `gen``tab2` AS `gen``col2` + """.stripMargin) + } + + test("create table - basic") { + val query = "CREATE TABLE my_table (id int, name string)" + val (desc, allowExisting) = extractTableDesc(query) + assert(!allowExisting) + assert(desc.identifier.database.isEmpty) + assert(desc.identifier.table == "my_table") + assert(desc.tableType == CatalogTableType.MANAGED) + assert(desc.schema == new StructType().add("id", "int").add("name", "string")) + assert(desc.partitionColumnNames.isEmpty) + assert(desc.bucketSpec.isEmpty) + assert(desc.viewText.isEmpty) + assert(desc.viewDefaultDatabase.isEmpty) + assert(desc.viewQueryColumnNames.isEmpty) + assert(desc.storage.locationUri.isEmpty) + assert(desc.storage.inputFormat == + Some("org.apache.hadoop.mapred.TextInputFormat")) + assert(desc.storage.outputFormat == + Some("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat")) + assert(desc.storage.serde == Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) + assert(desc.storage.properties.isEmpty) + assert(desc.properties.isEmpty) + assert(desc.comment.isEmpty) + } + + test("create table - with database name") { + val query = "CREATE TABLE dbx.my_table (id int, name string)" + val (desc, _) = extractTableDesc(query) + assert(desc.identifier.database == Some("dbx")) + assert(desc.identifier.table == "my_table") + } + + test("create table - temporary") { + val query = "CREATE TEMPORARY TABLE tab1 (id int, name string)" + val e = intercept[ParseException] { parser.parsePlan(query) } + assert(e.message.contains("CREATE TEMPORARY TABLE is not supported yet")) + } + + test("create table - external") { + val query = "CREATE EXTERNAL TABLE tab1 (id int, name string) LOCATION '/path/to/nowhere'" + val (desc, _) = extractTableDesc(query) + assert(desc.tableType == CatalogTableType.EXTERNAL) + assert(desc.storage.locationUri == Some(new URI("/path/to/nowhere"))) + } + + test("create table - if not exists") { + val query = "CREATE TABLE IF NOT EXISTS tab1 (id int, name string)" + val (_, allowExisting) = extractTableDesc(query) + assert(allowExisting) + } + + test("create table - comment") { + val query = "CREATE TABLE my_table (id int, name string) COMMENT 'its hot as hell below'" + val (desc, _) = extractTableDesc(query) + assert(desc.comment == Some("its hot as hell below")) + } + + test("create table - partitioned columns") { + val query = "CREATE TABLE my_table (id int, name string) PARTITIONED BY (month int)" + val (desc, _) = extractTableDesc(query) + assert(desc.schema == new StructType() + .add("id", "int") + .add("name", "string") + .add("month", "int")) + assert(desc.partitionColumnNames == Seq("month")) + } + + test("create table - clustered by") { + val numBuckets = 10 + val bucketedColumn = "id" + val sortColumn = "id" + val baseQuery = + s""" + CREATE TABLE my_table ( + $bucketedColumn int, + name string) + CLUSTERED BY($bucketedColumn) + """ + + val query1 = s"$baseQuery INTO $numBuckets BUCKETS" + val (desc1, _) = extractTableDesc(query1) + assert(desc1.bucketSpec.isDefined) + val bucketSpec1 = desc1.bucketSpec.get + assert(bucketSpec1.numBuckets == numBuckets) + assert(bucketSpec1.bucketColumnNames.head.equals(bucketedColumn)) + assert(bucketSpec1.sortColumnNames.isEmpty) + + val query2 = s"$baseQuery SORTED BY($sortColumn) INTO $numBuckets BUCKETS" + val (desc2, _) = extractTableDesc(query2) + assert(desc2.bucketSpec.isDefined) + val bucketSpec2 = desc2.bucketSpec.get + assert(bucketSpec2.numBuckets == numBuckets) + assert(bucketSpec2.bucketColumnNames.head.equals(bucketedColumn)) + assert(bucketSpec2.sortColumnNames.head.equals(sortColumn)) + } + + test("create table - skewed by") { + val baseQuery = "CREATE TABLE my_table (id int, name string) SKEWED BY" + val query1 = s"$baseQuery(id) ON (1, 10, 100)" + val query2 = s"$baseQuery(id, name) ON ((1, 'x'), (2, 'y'), (3, 'z'))" + val query3 = s"$baseQuery(id, name) ON ((1, 'x'), (2, 'y'), (3, 'z')) STORED AS DIRECTORIES" + val e1 = intercept[ParseException] { parser.parsePlan(query1) } + val e2 = intercept[ParseException] { parser.parsePlan(query2) } + val e3 = intercept[ParseException] { parser.parsePlan(query3) } + assert(e1.getMessage.contains("Operation not allowed")) + assert(e2.getMessage.contains("Operation not allowed")) + assert(e3.getMessage.contains("Operation not allowed")) + } + + test("create table - row format") { + val baseQuery = "CREATE TABLE my_table (id int, name string) ROW FORMAT" + val query1 = s"$baseQuery SERDE 'org.apache.poof.serde.Baff'" + val query2 = s"$baseQuery SERDE 'org.apache.poof.serde.Baff' WITH SERDEPROPERTIES ('k1'='v1')" + val query3 = + s""" + |$baseQuery DELIMITED FIELDS TERMINATED BY 'x' ESCAPED BY 'y' + |COLLECTION ITEMS TERMINATED BY 'a' + |MAP KEYS TERMINATED BY 'b' + |LINES TERMINATED BY '\n' + |NULL DEFINED AS 'c' + """.stripMargin + val (desc1, _) = extractTableDesc(query1) + val (desc2, _) = extractTableDesc(query2) + val (desc3, _) = extractTableDesc(query3) + assert(desc1.storage.serde == Some("org.apache.poof.serde.Baff")) + assert(desc1.storage.properties.isEmpty) + assert(desc2.storage.serde == Some("org.apache.poof.serde.Baff")) + assert(desc2.storage.properties == Map("k1" -> "v1")) + assert(desc3.storage.properties == Map( + "field.delim" -> "x", + "escape.delim" -> "y", + "serialization.format" -> "x", + "line.delim" -> "\n", + "colelction.delim" -> "a", // yes, it's a typo from Hive :) + "mapkey.delim" -> "b")) + } + + test("create table - file format") { + val baseQuery = "CREATE TABLE my_table (id int, name string) STORED AS" + val query1 = s"$baseQuery INPUTFORMAT 'winput' OUTPUTFORMAT 'wowput'" + val query2 = s"$baseQuery ORC" + val (desc1, _) = extractTableDesc(query1) + val (desc2, _) = extractTableDesc(query2) + assert(desc1.storage.inputFormat == Some("winput")) + assert(desc1.storage.outputFormat == Some("wowput")) + assert(desc1.storage.serde == Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) + assert(desc2.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat")) + assert(desc2.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) + assert(desc2.storage.serde == Some("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + } + + test("create table - storage handler") { + val baseQuery = "CREATE TABLE my_table (id int, name string) STORED BY" + val query1 = s"$baseQuery 'org.papachi.StorageHandler'" + val query2 = s"$baseQuery 'org.mamachi.StorageHandler' WITH SERDEPROPERTIES ('k1'='v1')" + val e1 = intercept[ParseException] { parser.parsePlan(query1) } + val e2 = intercept[ParseException] { parser.parsePlan(query2) } + assert(e1.getMessage.contains("Operation not allowed")) + assert(e2.getMessage.contains("Operation not allowed")) + } + + test("create table - properties") { + val query = "CREATE TABLE my_table (id int, name string) TBLPROPERTIES ('k1'='v1', 'k2'='v2')" + val (desc, _) = extractTableDesc(query) + assert(desc.properties == Map("k1" -> "v1", "k2" -> "v2")) + } + + test("create table - everything!") { + val query = + """ + |CREATE EXTERNAL TABLE IF NOT EXISTS dbx.my_table (id int, name string) + |COMMENT 'no comment' + |PARTITIONED BY (month int) + |ROW FORMAT SERDE 'org.apache.poof.serde.Baff' WITH SERDEPROPERTIES ('k1'='v1') + |STORED AS INPUTFORMAT 'winput' OUTPUTFORMAT 'wowput' + |LOCATION '/path/to/mercury' + |TBLPROPERTIES ('k1'='v1', 'k2'='v2') + """.stripMargin + val (desc, allowExisting) = extractTableDesc(query) + assert(allowExisting) + assert(desc.identifier.database == Some("dbx")) + assert(desc.identifier.table == "my_table") + assert(desc.tableType == CatalogTableType.EXTERNAL) + assert(desc.schema == new StructType() + .add("id", "int") + .add("name", "string") + .add("month", "int")) + assert(desc.partitionColumnNames == Seq("month")) + assert(desc.bucketSpec.isEmpty) + assert(desc.viewText.isEmpty) + assert(desc.viewDefaultDatabase.isEmpty) + assert(desc.viewQueryColumnNames.isEmpty) + assert(desc.storage.locationUri == Some(new URI("/path/to/mercury"))) + assert(desc.storage.inputFormat == Some("winput")) + assert(desc.storage.outputFormat == Some("wowput")) + assert(desc.storage.serde == Some("org.apache.poof.serde.Baff")) + assert(desc.storage.properties == Map("k1" -> "v1")) + assert(desc.properties == Map("k1" -> "v1", "k2" -> "v2")) + assert(desc.comment == Some("no comment")) + } + + test("create view -- basic") { + val v1 = "CREATE VIEW view1 AS SELECT * FROM tab1" + val command = parser.parsePlan(v1).asInstanceOf[CreateViewCommand] + assert(!command.allowExisting) + assert(command.name.database.isEmpty) + assert(command.name.table == "view1") + assert(command.originalText == Some("SELECT * FROM tab1")) + assert(command.userSpecifiedColumns.isEmpty) + } + + test("create view - full") { + val v1 = + """ + |CREATE OR REPLACE VIEW view1 + |(col1, col3 COMMENT 'hello') + |COMMENT 'BLABLA' + |TBLPROPERTIES('prop1Key'="prop1Val") + |AS SELECT * FROM tab1 + """.stripMargin + val command = parser.parsePlan(v1).asInstanceOf[CreateViewCommand] + assert(command.name.database.isEmpty) + assert(command.name.table == "view1") + assert(command.userSpecifiedColumns == Seq("col1" -> None, "col3" -> Some("hello"))) + assert(command.originalText == Some("SELECT * FROM tab1")) + assert(command.properties == Map("prop1Key" -> "prop1Val")) + assert(command.comment == Some("BLABLA")) + } + + test("create view -- partitioned view") { + val v1 = "CREATE VIEW view1 partitioned on (ds, hr) as select * from srcpart" + intercept[ParseException] { + parser.parsePlan(v1) + } + } + + test("MSCK REPAIR table") { + val sql = "MSCK REPAIR TABLE tab1" + val parsed = parser.parsePlan(sql) + val expected = AlterTableRecoverPartitionsCommand( + TableIdentifier("tab1", None), + "MSCK REPAIR TABLE") + comparePlans(parsed, expected) + } + + test("create table like") { + val v1 = "CREATE TABLE table1 LIKE table2" + val (target, source, location, exists) = parser.parsePlan(v1).collect { + case CreateTableLikeCommand(t, s, l, allowExisting) => (t, s, l, allowExisting) + }.head + assert(exists == false) + assert(target.database.isEmpty) + assert(target.table == "table1") + assert(source.database.isEmpty) + assert(source.table == "table2") + assert(location.isEmpty) + + val v2 = "CREATE TABLE IF NOT EXISTS table1 LIKE table2" + val (target2, source2, location2, exists2) = parser.parsePlan(v2).collect { + case CreateTableLikeCommand(t, s, l, allowExisting) => (t, s, l, allowExisting) + }.head + assert(exists2) + assert(target2.database.isEmpty) + assert(target2.table == "table1") + assert(source2.database.isEmpty) + assert(source2.table == "table2") + assert(location2.isEmpty) + + val v3 = "CREATE TABLE table1 LIKE table2 LOCATION '/spark/warehouse'" + val (target3, source3, location3, exists3) = parser.parsePlan(v3).collect { + case CreateTableLikeCommand(t, s, l, allowExisting) => (t, s, l, allowExisting) + }.head + assert(!exists3) + assert(target3.database.isEmpty) + assert(target3.table == "table1") + assert(source3.database.isEmpty) + assert(source3.table == "table2") + assert(location3 == Some("/spark/warehouse")) + + val v4 = "CREATE TABLE IF NOT EXISTS table1 LIKE table2 LOCATION '/spark/warehouse'" + val (target4, source4, location4, exists4) = parser.parsePlan(v4).collect { + case CreateTableLikeCommand(t, s, l, allowExisting) => (t, s, l, allowExisting) + }.head + assert(exists4) + assert(target4.database.isEmpty) + assert(target4.table == "table1") + assert(source4.database.isEmpty) + assert(source4.table == "table2") + assert(location4 == Some("/spark/warehouse")) + } + + test("load data") { + val v1 = "LOAD DATA INPATH 'path' INTO TABLE table1" + val (table, path, isLocal, isOverwrite, partition) = parser.parsePlan(v1).collect { + case LoadDataCommand(t, path, l, o, partition) => (t, path, l, o, partition) + }.head + assert(table.database.isEmpty) + assert(table.table == "table1") + assert(path == "path") + assert(!isLocal) + assert(!isOverwrite) + assert(partition.isEmpty) + + val v2 = "LOAD DATA LOCAL INPATH 'path' OVERWRITE INTO TABLE table1 PARTITION(c='1', d='2')" + val (table2, path2, isLocal2, isOverwrite2, partition2) = parser.parsePlan(v2).collect { + case LoadDataCommand(t, path, l, o, partition) => (t, path, l, o, partition) + }.head + assert(table2.database.isEmpty) + assert(table2.table == "table1") + assert(path2 == "path") + assert(isLocal2) + assert(isOverwrite2) + assert(partition2.nonEmpty) + assert(partition2.get.apply("c") == "1" && partition2.get.apply("d") == "2") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala new file mode 100644 index 000000000000..d19cfeef7d19 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -0,0 +1,2387 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import java.io.File +import java.net.URI +import java.util.Locale + +import org.apache.hadoop.fs.Path +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + + +class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with BeforeAndAfterEach { + override def afterEach(): Unit = { + try { + // drop all databases, tables and functions after each test + spark.sessionState.catalog.reset() + } finally { + Utils.deleteRecursively(new File(spark.sessionState.conf.warehousePath)) + super.afterEach() + } + } + + protected override def generateTable( + catalog: SessionCatalog, + name: TableIdentifier, + isDataSource: Boolean = true): CatalogTable = { + val storage = + CatalogStorageFormat.empty.copy(locationUri = Some(catalog.defaultTablePath(name))) + val metadata = new MetadataBuilder() + .putString("key", "value") + .build() + CatalogTable( + identifier = name, + tableType = CatalogTableType.EXTERNAL, + storage = storage, + schema = new StructType() + .add("col1", "int", nullable = true, metadata = metadata) + .add("col2", "string") + .add("a", "int") + .add("b", "int"), + provider = Some("parquet"), + partitionColumnNames = Seq("a", "b"), + createTime = 0L, + createVersion = org.apache.spark.SPARK_VERSION, + tracksPartitionsInCatalog = true) + } + + test("create a managed Hive source table") { + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") + val tabName = "tbl" + withTable(tabName) { + val e = intercept[AnalysisException] { + sql(s"CREATE TABLE $tabName (i INT, j STRING)") + }.getMessage + assert(e.contains("Hive support is required to CREATE Hive TABLE")) + } + } + + test("create an external Hive source table") { + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") + withTempDir { tempDir => + val tabName = "tbl" + withTable(tabName) { + val e = intercept[AnalysisException] { + sql( + s""" + |CREATE EXTERNAL TABLE $tabName (i INT, j STRING) + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |LOCATION '${tempDir.toURI}' + """.stripMargin) + }.getMessage + assert(e.contains("Hive support is required to CREATE Hive TABLE")) + } + } + } + + test("Create Hive Table As Select") { + import testImplicits._ + withTable("t", "t1") { + var e = intercept[AnalysisException] { + sql("CREATE TABLE t SELECT 1 as a, 1 as b") + }.getMessage + assert(e.contains("Hive support is required to CREATE Hive TABLE (AS SELECT)")) + + spark.range(1).select('id as 'a, 'id as 'b).write.saveAsTable("t1") + e = intercept[AnalysisException] { + sql("CREATE TABLE t SELECT a, b from t1") + }.getMessage + assert(e.contains("Hive support is required to CREATE Hive TABLE (AS SELECT)")) + } + } + +} + +abstract class DDLSuite extends QueryTest with SQLTestUtils { + + protected def isUsingHiveMetastore: Boolean = { + spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive" + } + + protected def generateTable( + catalog: SessionCatalog, + name: TableIdentifier, + isDataSource: Boolean = true): CatalogTable + + private val escapedIdentifier = "`(.+)`".r + + protected def normalizeCatalogTable(table: CatalogTable): CatalogTable = table + + private def normalizeSerdeProp(props: Map[String, String]): Map[String, String] = { + props.filterNot(p => Seq("serialization.format", "path").contains(p._1)) + } + + private def checkCatalogTables(expected: CatalogTable, actual: CatalogTable): Unit = { + assert(normalizeCatalogTable(actual) == normalizeCatalogTable(expected)) + } + + /** + * Strip backticks, if any, from the string. + */ + private def cleanIdentifier(ident: String): String = { + ident match { + case escapedIdentifier(i) => i + case plainIdent => plainIdent + } + } + + private def assertUnsupported(query: String): Unit = { + val e = intercept[AnalysisException] { + sql(query) + } + assert(e.getMessage.toLowerCase(Locale.ROOT).contains("operation not allowed")) + } + + private def maybeWrapException[T](expectException: Boolean)(body: => T): Unit = { + if (expectException) intercept[AnalysisException] { body } else body + } + + private def createDatabase(catalog: SessionCatalog, name: String): Unit = { + catalog.createDatabase( + CatalogDatabase( + name, "", CatalogUtils.stringToURI(spark.sessionState.conf.warehousePath), Map()), + ignoreIfExists = false) + } + + private def createTable( + catalog: SessionCatalog, + name: TableIdentifier, + isDataSource: Boolean = true): Unit = { + catalog.createTable(generateTable(catalog, name, isDataSource), ignoreIfExists = false) + } + + private def createTablePartition( + catalog: SessionCatalog, + spec: TablePartitionSpec, + tableName: TableIdentifier): Unit = { + val part = CatalogTablePartition( + spec, CatalogStorageFormat(None, None, None, None, false, Map())) + catalog.createPartitions(tableName, Seq(part), ignoreIfExists = false) + } + + private def getDBPath(dbName: String): URI = { + val warehousePath = makeQualifiedPath(spark.sessionState.conf.warehousePath) + new Path(CatalogUtils.URIToString(warehousePath), s"$dbName.db").toUri + } + + test("alter table: set location (datasource table)") { + testSetLocation(isDatasourceTable = true) + } + + test("alter table: set properties (datasource table)") { + testSetProperties(isDatasourceTable = true) + } + + test("alter table: unset properties (datasource table)") { + testUnsetProperties(isDatasourceTable = true) + } + + test("alter table: set serde (datasource table)") { + testSetSerde(isDatasourceTable = true) + } + + test("alter table: set serde partition (datasource table)") { + testSetSerdePartition(isDatasourceTable = true) + } + + test("alter table: change column (datasource table)") { + testChangeColumn(isDatasourceTable = true) + } + + test("alter table: add partition (datasource table)") { + testAddPartitions(isDatasourceTable = true) + } + + test("alter table: drop partition (datasource table)") { + testDropPartitions(isDatasourceTable = true) + } + + test("alter table: rename partition (datasource table)") { + testRenamePartitions(isDatasourceTable = true) + } + + test("drop table - data source table") { + testDropTable(isDatasourceTable = true) + } + + test("the qualified path of a database is stored in the catalog") { + val catalog = spark.sessionState.catalog + + withTempDir { tmpDir => + val path = tmpDir.getCanonicalPath + // The generated temp path is not qualified. + assert(!path.startsWith("file:/")) + val uri = tmpDir.toURI + sql(s"CREATE DATABASE db1 LOCATION '$uri'") + val pathInCatalog = new Path(catalog.getDatabaseMetadata("db1").locationUri).toUri + assert("file" === pathInCatalog.getScheme) + val expectedPath = new Path(path).toUri + assert(expectedPath.getPath === pathInCatalog.getPath) + sql("DROP DATABASE db1") + } + } + + test("Create Database using Default Warehouse Path") { + val catalog = spark.sessionState.catalog + val dbName = "db1" + try { + sql(s"CREATE DATABASE $dbName") + val db1 = catalog.getDatabaseMetadata(dbName) + assert(db1 == CatalogDatabase( + dbName, + "", + getDBPath(dbName), + Map.empty)) + sql(s"DROP DATABASE $dbName CASCADE") + assert(!catalog.databaseExists(dbName)) + } finally { + catalog.reset() + } + } + + test("Create/Drop Database - location") { + val catalog = spark.sessionState.catalog + val databaseNames = Seq("db1", "`database`") + withTempDir { tmpDir => + val path = new Path(tmpDir.getCanonicalPath).toUri + databaseNames.foreach { dbName => + try { + val dbNameWithoutBackTicks = cleanIdentifier(dbName) + sql(s"CREATE DATABASE $dbName Location '$path'") + val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) + val expPath = makeQualifiedPath(tmpDir.toString) + assert(db1 == CatalogDatabase( + dbNameWithoutBackTicks, + "", + expPath, + Map.empty)) + sql(s"DROP DATABASE $dbName CASCADE") + assert(!catalog.databaseExists(dbNameWithoutBackTicks)) + } finally { + catalog.reset() + } + } + } + } + + test("Create Database - database already exists") { + val catalog = spark.sessionState.catalog + val databaseNames = Seq("db1", "`database`") + + databaseNames.foreach { dbName => + try { + val dbNameWithoutBackTicks = cleanIdentifier(dbName) + sql(s"CREATE DATABASE $dbName") + val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) + assert(db1 == CatalogDatabase( + dbNameWithoutBackTicks, + "", + getDBPath(dbNameWithoutBackTicks), + Map.empty)) + + // TODO: HiveExternalCatalog should throw DatabaseAlreadyExistsException + val e = intercept[AnalysisException] { + sql(s"CREATE DATABASE $dbName") + }.getMessage + assert(e.contains(s"already exists")) + } finally { + catalog.reset() + } + } + } + + private def checkSchemaInCreatedDataSourceTable( + path: File, + userSpecifiedSchema: Option[String], + userSpecifiedPartitionCols: Option[String], + expectedSchema: StructType, + expectedPartitionCols: Seq[String]): Unit = { + val tabName = "tab1" + withTable(tabName) { + val partitionClause = + userSpecifiedPartitionCols.map(p => s"PARTITIONED BY ($p)").getOrElse("") + val schemaClause = userSpecifiedSchema.map(s => s"($s)").getOrElse("") + val uri = path.toURI + val sqlCreateTable = + s""" + |CREATE TABLE $tabName $schemaClause + |USING parquet + |OPTIONS ( + | path '$uri' + |) + |$partitionClause + """.stripMargin + if (userSpecifiedSchema.isEmpty && userSpecifiedPartitionCols.nonEmpty) { + val e = intercept[AnalysisException](sql(sqlCreateTable)).getMessage + assert(e.contains( + "not allowed to specify partition columns when the table schema is not defined")) + } else { + sql(sqlCreateTable) + val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName)) + + assert(expectedSchema == tableMetadata.schema) + assert(expectedPartitionCols == tableMetadata.partitionColumnNames) + } + } + } + + test("Create partitioned data source table without user specified schema") { + import testImplicits._ + val df = sparkContext.parallelize(1 to 10).map(i => (i, i.toString)).toDF("num", "str") + + // Case 1: with partitioning columns but no schema: Option("inexistentColumns") + // Case 2: without schema and partitioning columns: None + Seq(Option("inexistentColumns"), None).foreach { partitionCols => + withTempPath { pathToPartitionedTable => + df.write.format("parquet").partitionBy("num") + .save(pathToPartitionedTable.getCanonicalPath) + checkSchemaInCreatedDataSourceTable( + pathToPartitionedTable, + userSpecifiedSchema = None, + userSpecifiedPartitionCols = partitionCols, + expectedSchema = new StructType().add("str", StringType).add("num", IntegerType), + expectedPartitionCols = Seq("num")) + } + } + } + + test("Create partitioned data source table with user specified schema") { + import testImplicits._ + val df = sparkContext.parallelize(1 to 10).map(i => (i, i.toString)).toDF("num", "str") + + // Case 1: with partitioning columns but no schema: Option("num") + // Case 2: without schema and partitioning columns: None + Seq(Option("num"), None).foreach { partitionCols => + withTempPath { pathToPartitionedTable => + df.write.format("parquet").partitionBy("num") + .save(pathToPartitionedTable.getCanonicalPath) + checkSchemaInCreatedDataSourceTable( + pathToPartitionedTable, + userSpecifiedSchema = Option("num int, str string"), + userSpecifiedPartitionCols = partitionCols, + expectedSchema = new StructType().add("str", StringType).add("num", IntegerType), + expectedPartitionCols = partitionCols.map(Seq(_)).getOrElse(Seq.empty[String])) + } + } + } + + test("Create non-partitioned data source table without user specified schema") { + import testImplicits._ + val df = sparkContext.parallelize(1 to 10).map(i => (i, i.toString)).toDF("num", "str") + + // Case 1: with partitioning columns but no schema: Option("inexistentColumns") + // Case 2: without schema and partitioning columns: None + Seq(Option("inexistentColumns"), None).foreach { partitionCols => + withTempPath { pathToNonPartitionedTable => + df.write.format("parquet").save(pathToNonPartitionedTable.getCanonicalPath) + checkSchemaInCreatedDataSourceTable( + pathToNonPartitionedTable, + userSpecifiedSchema = None, + userSpecifiedPartitionCols = partitionCols, + expectedSchema = new StructType().add("num", IntegerType).add("str", StringType), + expectedPartitionCols = Seq.empty[String]) + } + } + } + + test("Create non-partitioned data source table with user specified schema") { + import testImplicits._ + val df = sparkContext.parallelize(1 to 10).map(i => (i, i.toString)).toDF("num", "str") + + // Case 1: with partitioning columns but no schema: Option("inexistentColumns") + // Case 2: without schema and partitioning columns: None + Seq(Option("num"), None).foreach { partitionCols => + withTempPath { pathToNonPartitionedTable => + df.write.format("parquet").save(pathToNonPartitionedTable.getCanonicalPath) + checkSchemaInCreatedDataSourceTable( + pathToNonPartitionedTable, + userSpecifiedSchema = Option("num int, str string"), + userSpecifiedPartitionCols = partitionCols, + expectedSchema = if (partitionCols.isDefined) { + // we skipped inference, so the partition col is ordered at the end + new StructType().add("str", StringType).add("num", IntegerType) + } else { + // no inferred partitioning, so schema is in original order + new StructType().add("num", IntegerType).add("str", StringType) + }, + expectedPartitionCols = partitionCols.map(Seq(_)).getOrElse(Seq.empty[String])) + } + } + } + + test("create table - duplicate column names in the table definition") { + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val errMsg = intercept[AnalysisException] { + sql(s"CREATE TABLE t($c0 INT, $c1 INT) USING parquet") + }.getMessage + assert(errMsg.contains("Found duplicate column(s) in the table definition of `t`")) + } + } + } + + test("create table - partition column names not in table definition") { + val e = intercept[AnalysisException] { + sql("CREATE TABLE tbl(a int, b string) USING json PARTITIONED BY (c)") + } + assert(e.message == "partition column c is not defined in table tbl, " + + "defined table columns are: a, b") + } + + test("create table - bucket column names not in table definition") { + val e = intercept[AnalysisException] { + sql("CREATE TABLE tbl(a int, b string) USING json CLUSTERED BY (c) INTO 4 BUCKETS") + } + assert(e.message == "bucket column c is not defined in table tbl, " + + "defined table columns are: a, b") + } + + test("create table - column repeated in partition columns") { + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val errMsg = intercept[AnalysisException] { + sql(s"CREATE TABLE t($c0 INT) USING parquet PARTITIONED BY ($c0, $c1)") + }.getMessage + assert(errMsg.contains("Found duplicate column(s) in the partition schema")) + } + } + } + + test("create table - column repeated in bucket/sort columns") { + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + var errMsg = intercept[AnalysisException] { + sql(s"CREATE TABLE t($c0 INT) USING parquet CLUSTERED BY ($c0, $c1) INTO 2 BUCKETS") + }.getMessage + assert(errMsg.contains("Found duplicate column(s) in the bucket definition")) + + errMsg = intercept[AnalysisException] { + sql(s""" + |CREATE TABLE t($c0 INT, col INT) USING parquet CLUSTERED BY (col) + | SORTED BY ($c0, $c1) INTO 2 BUCKETS + """.stripMargin) + }.getMessage + assert(errMsg.contains("Found duplicate column(s) in the sort definition")) + } + } + } + + test("Refresh table after changing the data source table partitioning") { + import testImplicits._ + + val tabName = "tab1" + val catalog = spark.sessionState.catalog + withTempPath { dir => + val path = dir.getCanonicalPath + val df = sparkContext.parallelize(1 to 10).map(i => (i, i.toString, i, i)) + .toDF("col1", "col2", "col3", "col4") + df.write.format("json").partitionBy("col1", "col3").save(path) + val schema = new StructType() + .add("col2", StringType).add("col4", LongType) + .add("col1", IntegerType).add("col3", IntegerType) + val partitionCols = Seq("col1", "col3") + val uri = dir.toURI + + withTable(tabName) { + spark.sql( + s""" + |CREATE TABLE $tabName + |USING json + |OPTIONS ( + | path '$uri' + |) + """.stripMargin) + val tableMetadata = catalog.getTableMetadata(TableIdentifier(tabName)) + assert(tableMetadata.schema == schema) + assert(tableMetadata.partitionColumnNames == partitionCols) + + // Change the schema + val newDF = sparkContext.parallelize(1 to 10).map(i => (i, i.toString)) + .toDF("newCol1", "newCol2") + newDF.write.format("json").partitionBy("newCol1").mode(SaveMode.Overwrite).save(path) + + // No change on the schema + val tableMetadataBeforeRefresh = catalog.getTableMetadata(TableIdentifier(tabName)) + assert(tableMetadataBeforeRefresh.schema == schema) + assert(tableMetadataBeforeRefresh.partitionColumnNames == partitionCols) + + // Refresh does not affect the schema + spark.catalog.refreshTable(tabName) + + val tableMetadataAfterRefresh = catalog.getTableMetadata(TableIdentifier(tabName)) + assert(tableMetadataAfterRefresh.schema == schema) + assert(tableMetadataAfterRefresh.partitionColumnNames == partitionCols) + } + } + } + + test("create view - duplicate column names in the view definition") { + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val errMsg = intercept[AnalysisException] { + sql(s"CREATE VIEW t AS SELECT * FROM VALUES (1, 1) AS t($c0, $c1)") + }.getMessage + assert(errMsg.contains("Found duplicate column(s) in the view definition")) + } + } + } + + test("Alter/Describe Database") { + val catalog = spark.sessionState.catalog + val databaseNames = Seq("db1", "`database`") + + databaseNames.foreach { dbName => + try { + val dbNameWithoutBackTicks = cleanIdentifier(dbName) + val location = getDBPath(dbNameWithoutBackTicks) + + sql(s"CREATE DATABASE $dbName") + + checkAnswer( + sql(s"DESCRIBE DATABASE EXTENDED $dbName"), + Row("Database Name", dbNameWithoutBackTicks) :: + Row("Description", "") :: + Row("Location", CatalogUtils.URIToString(location)) :: + Row("Properties", "") :: Nil) + + sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") + + checkAnswer( + sql(s"DESCRIBE DATABASE EXTENDED $dbName"), + Row("Database Name", dbNameWithoutBackTicks) :: + Row("Description", "") :: + Row("Location", CatalogUtils.URIToString(location)) :: + Row("Properties", "((a,a), (b,b), (c,c))") :: Nil) + + sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('d'='d')") + + checkAnswer( + sql(s"DESCRIBE DATABASE EXTENDED $dbName"), + Row("Database Name", dbNameWithoutBackTicks) :: + Row("Description", "") :: + Row("Location", CatalogUtils.URIToString(location)) :: + Row("Properties", "((a,a), (b,b), (c,c), (d,d))") :: Nil) + } finally { + catalog.reset() + } + } + } + + test("Drop/Alter/Describe Database - database does not exists") { + val databaseNames = Seq("db1", "`database`") + + databaseNames.foreach { dbName => + val dbNameWithoutBackTicks = cleanIdentifier(dbName) + assert(!spark.sessionState.catalog.databaseExists(dbNameWithoutBackTicks)) + + var message = intercept[AnalysisException] { + sql(s"DROP DATABASE $dbName") + }.getMessage + // TODO: Unify the exception. + if (isUsingHiveMetastore) { + assert(message.contains(s"NoSuchObjectException: $dbNameWithoutBackTicks")) + } else { + assert(message.contains(s"Database '$dbNameWithoutBackTicks' not found")) + } + + message = intercept[AnalysisException] { + sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('d'='d')") + }.getMessage + assert(message.contains(s"Database '$dbNameWithoutBackTicks' not found")) + + message = intercept[AnalysisException] { + sql(s"DESCRIBE DATABASE EXTENDED $dbName") + }.getMessage + assert(message.contains(s"Database '$dbNameWithoutBackTicks' not found")) + + sql(s"DROP DATABASE IF EXISTS $dbName") + } + } + + test("drop non-empty database in restrict mode") { + val catalog = spark.sessionState.catalog + val dbName = "db1" + sql(s"CREATE DATABASE $dbName") + + // create a table in database + val tableIdent1 = TableIdentifier("tab1", Some(dbName)) + createTable(catalog, tableIdent1) + + // drop a non-empty database in Restrict mode + val message = intercept[AnalysisException] { + sql(s"DROP DATABASE $dbName RESTRICT") + }.getMessage + assert(message.contains(s"Database $dbName is not empty. One or more tables exist")) + + + catalog.dropTable(tableIdent1, ignoreIfNotExists = false, purge = false) + + assert(catalog.listDatabases().contains(dbName)) + sql(s"DROP DATABASE $dbName RESTRICT") + assert(!catalog.listDatabases().contains(dbName)) + } + + test("drop non-empty database in cascade mode") { + val catalog = spark.sessionState.catalog + val dbName = "db1" + sql(s"CREATE DATABASE $dbName") + + // create a table in database + val tableIdent1 = TableIdentifier("tab1", Some(dbName)) + createTable(catalog, tableIdent1) + + // drop a non-empty database in CASCADE mode + assert(catalog.listTables(dbName).contains(tableIdent1)) + assert(catalog.listDatabases().contains(dbName)) + sql(s"DROP DATABASE $dbName CASCADE") + assert(!catalog.listDatabases().contains(dbName)) + } + + test("create table in default db") { + val catalog = spark.sessionState.catalog + val tableIdent1 = TableIdentifier("tab1", None) + createTable(catalog, tableIdent1) + val expectedTableIdent = tableIdent1.copy(database = Some("default")) + val expectedTable = generateTable(catalog, expectedTableIdent) + checkCatalogTables(expectedTable, catalog.getTableMetadata(tableIdent1)) + } + + test("create table in a specific db") { + val catalog = spark.sessionState.catalog + createDatabase(catalog, "dbx") + val tableIdent1 = TableIdentifier("tab1", Some("dbx")) + createTable(catalog, tableIdent1) + val expectedTable = generateTable(catalog, tableIdent1) + checkCatalogTables(expectedTable, catalog.getTableMetadata(tableIdent1)) + } + + test("create table using") { + val catalog = spark.sessionState.catalog + withTable("tbl") { + sql("CREATE TABLE tbl(a INT, b INT) USING parquet") + val table = catalog.getTableMetadata(TableIdentifier("tbl")) + assert(table.tableType == CatalogTableType.MANAGED) + assert(table.schema == new StructType().add("a", "int").add("b", "int")) + assert(table.provider == Some("parquet")) + } + } + + test("create table using - with partitioned by") { + val catalog = spark.sessionState.catalog + withTable("tbl") { + sql("CREATE TABLE tbl(a INT, b INT) USING parquet PARTITIONED BY (a)") + val table = catalog.getTableMetadata(TableIdentifier("tbl")) + assert(table.tableType == CatalogTableType.MANAGED) + assert(table.provider == Some("parquet")) + // a is ordered last since it is a user-specified partitioning column + assert(table.schema == new StructType().add("b", IntegerType).add("a", IntegerType)) + assert(table.partitionColumnNames == Seq("a")) + } + } + + test("create table using - with bucket") { + val catalog = spark.sessionState.catalog + withTable("tbl") { + sql("CREATE TABLE tbl(a INT, b INT) USING parquet " + + "CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS") + val table = catalog.getTableMetadata(TableIdentifier("tbl")) + assert(table.tableType == CatalogTableType.MANAGED) + assert(table.provider == Some("parquet")) + assert(table.schema == new StructType().add("a", IntegerType).add("b", IntegerType)) + assert(table.bucketSpec == Some(BucketSpec(5, Seq("a"), Seq("b")))) + } + } + + test("create temporary view using") { + // when we test the HiveCatalogedDDLSuite, it will failed because the csvFile path above + // starts with 'jar:', and it is an illegal parameter for Path, so here we copy it + // to a temp file by withResourceTempPath + withResourceTempPath("test-data/cars.csv") { tmpFile => + withView("testview") { + sql(s"CREATE OR REPLACE TEMPORARY VIEW testview (c1 String, c2 String) USING " + + "org.apache.spark.sql.execution.datasources.csv.CSVFileFormat " + + s"OPTIONS (PATH '${tmpFile.toURI}')") + + checkAnswer( + sql("select c1, c2 from testview order by c1 limit 1"), + Row("1997", "Ford") :: Nil) + + // Fails if creating a new view with the same name + intercept[TempTableAlreadyExistsException] { + sql( + s""" + |CREATE TEMPORARY VIEW testview + |USING org.apache.spark.sql.execution.datasources.csv.CSVFileFormat + |OPTIONS (PATH '${tmpFile.toURI}') + """.stripMargin) + } + } + } + } + + test("alter table: rename") { + val catalog = spark.sessionState.catalog + val tableIdent1 = TableIdentifier("tab1", Some("dbx")) + val tableIdent2 = TableIdentifier("tab2", Some("dbx")) + createDatabase(catalog, "dbx") + createDatabase(catalog, "dby") + createTable(catalog, tableIdent1) + + assert(catalog.listTables("dbx") == Seq(tableIdent1)) + sql("ALTER TABLE dbx.tab1 RENAME TO dbx.tab2") + assert(catalog.listTables("dbx") == Seq(tableIdent2)) + + // The database in destination table name can be omitted, and we will use the database of source + // table for it. + sql("ALTER TABLE dbx.tab2 RENAME TO tab1") + assert(catalog.listTables("dbx") == Seq(tableIdent1)) + + catalog.setCurrentDatabase("dbx") + // rename without explicitly specifying database + sql("ALTER TABLE tab1 RENAME TO tab2") + assert(catalog.listTables("dbx") == Seq(tableIdent2)) + // table to rename does not exist + intercept[AnalysisException] { + sql("ALTER TABLE dbx.does_not_exist RENAME TO dbx.tab2") + } + // destination database is different + intercept[AnalysisException] { + sql("ALTER TABLE dbx.tab1 RENAME TO dby.tab2") + } + } + + test("alter table: rename cached table") { + import testImplicits._ + sql("CREATE TABLE students (age INT, name STRING) USING parquet") + val df = (1 to 2).map { i => (i, i.toString) }.toDF("age", "name") + df.write.insertInto("students") + spark.catalog.cacheTable("students") + checkAnswer(spark.table("students"), df) + assume(spark.catalog.isCached("students"), "bad test: table was not cached in the first place") + sql("ALTER TABLE students RENAME TO teachers") + sql("CREATE TABLE students (age INT, name STRING) USING parquet") + // Now we have both students and teachers. + // The cached data for the old students table should not be read by the new students table. + assert(!spark.catalog.isCached("students")) + assert(spark.catalog.isCached("teachers")) + assert(spark.table("students").collect().isEmpty) + checkAnswer(spark.table("teachers"), df) + } + + test("rename temporary table - destination table with database name") { + withTempView("tab1") { + sql( + """ + |CREATE TEMPORARY TABLE tab1 + |USING org.apache.spark.sql.sources.DDLScanSource + |OPTIONS ( + | From '1', + | To '10', + | Table 'test1' + |) + """.stripMargin) + + val e = intercept[AnalysisException] { + sql("ALTER TABLE tab1 RENAME TO default.tab2") + } + assert(e.getMessage.contains( + "RENAME TEMPORARY TABLE from '`tab1`' to '`default`.`tab2`': " + + "cannot specify database name 'default' in the destination table")) + + val catalog = spark.sessionState.catalog + assert(catalog.listTables("default") == Seq(TableIdentifier("tab1"))) + } + } + + test("rename temporary table") { + withTempView("tab1", "tab2") { + spark.range(10).createOrReplaceTempView("tab1") + sql("ALTER TABLE tab1 RENAME TO tab2") + checkAnswer(spark.table("tab2"), spark.range(10).toDF()) + intercept[NoSuchTableException] { spark.table("tab1") } + sql("ALTER VIEW tab2 RENAME TO tab1") + checkAnswer(spark.table("tab1"), spark.range(10).toDF()) + intercept[NoSuchTableException] { spark.table("tab2") } + } + } + + test("rename temporary table - destination table already exists") { + withTempView("tab1", "tab2") { + sql( + """ + |CREATE TEMPORARY TABLE tab1 + |USING org.apache.spark.sql.sources.DDLScanSource + |OPTIONS ( + | From '1', + | To '10', + | Table 'test1' + |) + """.stripMargin) + + sql( + """ + |CREATE TEMPORARY TABLE tab2 + |USING org.apache.spark.sql.sources.DDLScanSource + |OPTIONS ( + | From '1', + | To '10', + | Table 'test1' + |) + """.stripMargin) + + val e = intercept[AnalysisException] { + sql("ALTER TABLE tab1 RENAME TO tab2") + } + assert(e.getMessage.contains( + "RENAME TEMPORARY TABLE from '`tab1`' to '`tab2`': destination table already exists")) + + val catalog = spark.sessionState.catalog + assert(catalog.listTables("default") == Seq(TableIdentifier("tab1"), TableIdentifier("tab2"))) + } + } + + test("alter table: bucketing is not supported") { + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent) + assertUnsupported("ALTER TABLE dbx.tab1 CLUSTERED BY (blood, lemon, grape) INTO 11 BUCKETS") + assertUnsupported("ALTER TABLE dbx.tab1 CLUSTERED BY (fuji) SORTED BY (grape) INTO 5 BUCKETS") + assertUnsupported("ALTER TABLE dbx.tab1 NOT CLUSTERED") + assertUnsupported("ALTER TABLE dbx.tab1 NOT SORTED") + } + + test("alter table: skew is not supported") { + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent) + assertUnsupported("ALTER TABLE dbx.tab1 SKEWED BY (dt, country) ON " + + "(('2008-08-08', 'us'), ('2009-09-09', 'uk'), ('2010-10-10', 'cn'))") + assertUnsupported("ALTER TABLE dbx.tab1 SKEWED BY (dt, country) ON " + + "(('2008-08-08', 'us'), ('2009-09-09', 'uk')) STORED AS DIRECTORIES") + assertUnsupported("ALTER TABLE dbx.tab1 NOT SKEWED") + assertUnsupported("ALTER TABLE dbx.tab1 NOT STORED AS DIRECTORIES") + } + + test("alter table: recover partitions (sequential)") { + withSQLConf("spark.rdd.parallelListingThreshold" -> "10") { + testRecoverPartitions() + } + } + + test("alter table: recover partition (parallel)") { + withSQLConf("spark.rdd.parallelListingThreshold" -> "1") { + testRecoverPartitions() + } + } + + protected def testRecoverPartitions() { + val catalog = spark.sessionState.catalog + // table to alter does not exist + intercept[AnalysisException] { + sql("ALTER TABLE does_not_exist RECOVER PARTITIONS") + } + + val tableIdent = TableIdentifier("tab1") + createTable(catalog, tableIdent) + val part1 = Map("a" -> "1", "b" -> "5") + createTablePartition(catalog, part1, tableIdent) + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part1)) + + val part2 = Map("a" -> "2", "b" -> "6") + val root = new Path(catalog.getTableMetadata(tableIdent).location) + val fs = root.getFileSystem(spark.sparkContext.hadoopConfiguration) + // valid + fs.mkdirs(new Path(new Path(root, "a=1"), "b=5")) + fs.createNewFile(new Path(new Path(root, "a=1/b=5"), "a.csv")) // file + fs.createNewFile(new Path(new Path(root, "a=1/b=5"), "_SUCCESS")) // file + fs.mkdirs(new Path(new Path(root, "A=2"), "B=6")) + fs.createNewFile(new Path(new Path(root, "A=2/B=6"), "b.csv")) // file + fs.createNewFile(new Path(new Path(root, "A=2/B=6"), "c.csv")) // file + fs.createNewFile(new Path(new Path(root, "A=2/B=6"), ".hiddenFile")) // file + fs.mkdirs(new Path(new Path(root, "A=2/B=6"), "_temporary")) + + // invalid + fs.mkdirs(new Path(new Path(root, "a"), "b")) // bad name + fs.mkdirs(new Path(new Path(root, "b=1"), "a=1")) // wrong order + fs.mkdirs(new Path(root, "a=4")) // not enough columns + fs.createNewFile(new Path(new Path(root, "a=1"), "b=4")) // file + fs.createNewFile(new Path(new Path(root, "a=1"), "_SUCCESS")) // _SUCCESS + fs.mkdirs(new Path(new Path(root, "a=1"), "_temporary")) // _temporary + fs.mkdirs(new Path(new Path(root, "a=1"), ".b=4")) // start with . + + try { + sql("ALTER TABLE tab1 RECOVER PARTITIONS") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == + Set(part1, part2)) + if (!isUsingHiveMetastore) { + assert(catalog.getPartition(tableIdent, part1).parameters("numFiles") == "1") + assert(catalog.getPartition(tableIdent, part2).parameters("numFiles") == "2") + } else { + // After ALTER TABLE, the statistics of the first partition is removed by Hive megastore + assert(catalog.getPartition(tableIdent, part1).parameters.get("numFiles").isEmpty) + assert(catalog.getPartition(tableIdent, part2).parameters("numFiles") == "2") + } + } finally { + fs.delete(root, true) + } + } + + test("alter table: add partition is not supported for views") { + assertUnsupported("ALTER VIEW dbx.tab1 ADD IF NOT EXISTS PARTITION (b='2')") + } + + test("alter table: drop partition is not supported for views") { + assertUnsupported("ALTER VIEW dbx.tab1 DROP IF EXISTS PARTITION (b='2')") + } + + + test("show databases") { + sql("CREATE DATABASE showdb2B") + sql("CREATE DATABASE showdb1A") + + // check the result as well as its order + checkDataset(sql("SHOW DATABASES"), Row("default"), Row("showdb1a"), Row("showdb2b")) + + checkAnswer( + sql("SHOW DATABASES LIKE '*db1A'"), + Row("showdb1a") :: Nil) + + checkAnswer( + sql("SHOW DATABASES LIKE 'showdb1A'"), + Row("showdb1a") :: Nil) + + checkAnswer( + sql("SHOW DATABASES LIKE '*db1A|*db2B'"), + Row("showdb1a") :: + Row("showdb2b") :: Nil) + + checkAnswer( + sql("SHOW DATABASES LIKE 'non-existentdb'"), + Nil) + } + + test("drop view - temporary view") { + val catalog = spark.sessionState.catalog + sql( + """ + |CREATE TEMPORARY VIEW tab1 + |USING org.apache.spark.sql.sources.DDLScanSource + |OPTIONS ( + | From '1', + | To '10', + | Table 'test1' + |) + """.stripMargin) + assert(catalog.listTables("default") == Seq(TableIdentifier("tab1"))) + sql("DROP VIEW tab1") + assert(catalog.listTables("default") == Nil) + } + + protected def testDropTable(isDatasourceTable: Boolean): Unit = { + if (!isUsingHiveMetastore) { + assert(isDatasourceTable, "InMemoryCatalog only supports data source tables") + } + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent, isDatasourceTable) + assert(catalog.listTables("dbx") == Seq(tableIdent)) + sql("DROP TABLE dbx.tab1") + assert(catalog.listTables("dbx") == Nil) + sql("DROP TABLE IF EXISTS dbx.tab1") + intercept[AnalysisException] { + sql("DROP TABLE dbx.tab1") + } + } + + test("drop view") { + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent) + assert(catalog.listTables("dbx") == Seq(tableIdent)) + + val e = intercept[AnalysisException] { + sql("DROP VIEW dbx.tab1") + } + assert( + e.getMessage.contains("Cannot drop a table with DROP VIEW. Please use DROP TABLE instead")) + } + + protected def testSetProperties(isDatasourceTable: Boolean): Unit = { + if (!isUsingHiveMetastore) { + assert(isDatasourceTable, "InMemoryCatalog only supports data source tables") + } + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent, isDatasourceTable) + def getProps: Map[String, String] = { + if (isUsingHiveMetastore) { + normalizeCatalogTable(catalog.getTableMetadata(tableIdent)).properties + } else { + catalog.getTableMetadata(tableIdent).properties + } + } + assert(getProps.isEmpty) + // set table properties + sql("ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('andrew' = 'or14', 'kor' = 'bel')") + assert(getProps == Map("andrew" -> "or14", "kor" -> "bel")) + // set table properties without explicitly specifying database + catalog.setCurrentDatabase("dbx") + sql("ALTER TABLE tab1 SET TBLPROPERTIES ('kor' = 'belle', 'kar' = 'bol')") + assert(getProps == Map("andrew" -> "or14", "kor" -> "belle", "kar" -> "bol")) + // table to alter does not exist + intercept[AnalysisException] { + sql("ALTER TABLE does_not_exist SET TBLPROPERTIES ('winner' = 'loser')") + } + } + + protected def testUnsetProperties(isDatasourceTable: Boolean): Unit = { + if (!isUsingHiveMetastore) { + assert(isDatasourceTable, "InMemoryCatalog only supports data source tables") + } + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent, isDatasourceTable) + def getProps: Map[String, String] = { + if (isUsingHiveMetastore) { + normalizeCatalogTable(catalog.getTableMetadata(tableIdent)).properties + } else { + catalog.getTableMetadata(tableIdent).properties + } + } + // unset table properties + sql("ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('j' = 'am', 'p' = 'an', 'c' = 'lan', 'x' = 'y')") + sql("ALTER TABLE dbx.tab1 UNSET TBLPROPERTIES ('j')") + assert(getProps == Map("p" -> "an", "c" -> "lan", "x" -> "y")) + // unset table properties without explicitly specifying database + catalog.setCurrentDatabase("dbx") + sql("ALTER TABLE tab1 UNSET TBLPROPERTIES ('p')") + assert(getProps == Map("c" -> "lan", "x" -> "y")) + // table to alter does not exist + intercept[AnalysisException] { + sql("ALTER TABLE does_not_exist UNSET TBLPROPERTIES ('c' = 'lan')") + } + // property to unset does not exist + val e = intercept[AnalysisException] { + sql("ALTER TABLE tab1 UNSET TBLPROPERTIES ('c', 'xyz')") + } + assert(e.getMessage.contains("xyz")) + // property to unset does not exist, but "IF EXISTS" is specified + sql("ALTER TABLE tab1 UNSET TBLPROPERTIES IF EXISTS ('c', 'xyz')") + assert(getProps == Map("x" -> "y")) + } + + protected def testSetLocation(isDatasourceTable: Boolean): Unit = { + if (!isUsingHiveMetastore) { + assert(isDatasourceTable, "InMemoryCatalog only supports data source tables") + } + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + val partSpec = Map("a" -> "1", "b" -> "2") + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent, isDatasourceTable) + createTablePartition(catalog, partSpec, tableIdent) + assert(catalog.getTableMetadata(tableIdent).storage.locationUri.isDefined) + assert(normalizeSerdeProp(catalog.getTableMetadata(tableIdent).storage.properties).isEmpty) + assert(catalog.getPartition(tableIdent, partSpec).storage.locationUri.isDefined) + assert( + normalizeSerdeProp(catalog.getPartition(tableIdent, partSpec).storage.properties).isEmpty) + + // Verify that the location is set to the expected string + def verifyLocation(expected: URI, spec: Option[TablePartitionSpec] = None): Unit = { + val storageFormat = spec + .map { s => catalog.getPartition(tableIdent, s).storage } + .getOrElse { catalog.getTableMetadata(tableIdent).storage } + // TODO(gatorsmile): fix the bug in alter table set location. + // if (isUsingHiveMetastore) { + // assert(storageFormat.properties.get("path") === expected) + // } + assert(storageFormat.locationUri === Some(expected)) + } + // set table location + sql("ALTER TABLE dbx.tab1 SET LOCATION '/path/to/your/lovely/heart'") + verifyLocation(new URI("/path/to/your/lovely/heart")) + // set table partition location + sql("ALTER TABLE dbx.tab1 PARTITION (a='1', b='2') SET LOCATION '/path/to/part/ways'") + verifyLocation(new URI("/path/to/part/ways"), Some(partSpec)) + // set table location without explicitly specifying database + catalog.setCurrentDatabase("dbx") + sql("ALTER TABLE tab1 SET LOCATION '/swanky/steak/place'") + verifyLocation(new URI("/swanky/steak/place")) + // set table partition location without explicitly specifying database + sql("ALTER TABLE tab1 PARTITION (a='1', b='2') SET LOCATION 'vienna'") + verifyLocation(new URI("vienna"), Some(partSpec)) + // table to alter does not exist + intercept[AnalysisException] { + sql("ALTER TABLE dbx.does_not_exist SET LOCATION '/mister/spark'") + } + // partition to alter does not exist + intercept[AnalysisException] { + sql("ALTER TABLE dbx.tab1 PARTITION (b='2') SET LOCATION '/mister/spark'") + } + } + + protected def testSetSerde(isDatasourceTable: Boolean): Unit = { + if (!isUsingHiveMetastore) { + assert(isDatasourceTable, "InMemoryCatalog only supports data source tables") + } + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent, isDatasourceTable) + def checkSerdeProps(expectedSerdeProps: Map[String, String]): Unit = { + val serdeProp = catalog.getTableMetadata(tableIdent).storage.properties + if (isUsingHiveMetastore) { + assert(normalizeSerdeProp(serdeProp) == expectedSerdeProps) + } else { + assert(serdeProp == expectedSerdeProps) + } + } + if (isUsingHiveMetastore) { + val expectedSerde = if (isDatasourceTable) { + "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe" + } else { + "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe" + } + assert(catalog.getTableMetadata(tableIdent).storage.serde == Some(expectedSerde)) + } else { + assert(catalog.getTableMetadata(tableIdent).storage.serde.isEmpty) + } + checkSerdeProps(Map.empty[String, String]) + // set table serde and/or properties (should fail on datasource tables) + if (isDatasourceTable) { + val e1 = intercept[AnalysisException] { + sql("ALTER TABLE dbx.tab1 SET SERDE 'whatever'") + } + val e2 = intercept[AnalysisException] { + sql("ALTER TABLE dbx.tab1 SET SERDE 'org.apache.madoop' " + + "WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee')") + } + assert(e1.getMessage.contains("datasource")) + assert(e2.getMessage.contains("datasource")) + } else { + val newSerde = "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe" + sql(s"ALTER TABLE dbx.tab1 SET SERDE '$newSerde'") + assert(catalog.getTableMetadata(tableIdent).storage.serde == Some(newSerde)) + checkSerdeProps(Map.empty[String, String]) + val serde2 = "org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe" + sql(s"ALTER TABLE dbx.tab1 SET SERDE '$serde2' " + + "WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee')") + assert(catalog.getTableMetadata(tableIdent).storage.serde == Some(serde2)) + checkSerdeProps(Map("k" -> "v", "kay" -> "vee")) + } + // set serde properties only + sql("ALTER TABLE dbx.tab1 SET SERDEPROPERTIES ('k' = 'vvv', 'kay' = 'vee')") + checkSerdeProps(Map("k" -> "vvv", "kay" -> "vee")) + // set things without explicitly specifying database + catalog.setCurrentDatabase("dbx") + sql("ALTER TABLE tab1 SET SERDEPROPERTIES ('kay' = 'veee')") + checkSerdeProps(Map("k" -> "vvv", "kay" -> "veee")) + // table to alter does not exist + intercept[AnalysisException] { + sql("ALTER TABLE does_not_exist SET SERDEPROPERTIES ('x' = 'y')") + } + } + + protected def testSetSerdePartition(isDatasourceTable: Boolean): Unit = { + if (!isUsingHiveMetastore) { + assert(isDatasourceTable, "InMemoryCatalog only supports data source tables") + } + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + val spec = Map("a" -> "1", "b" -> "2") + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent, isDatasourceTable) + createTablePartition(catalog, spec, tableIdent) + createTablePartition(catalog, Map("a" -> "1", "b" -> "3"), tableIdent) + createTablePartition(catalog, Map("a" -> "2", "b" -> "2"), tableIdent) + createTablePartition(catalog, Map("a" -> "2", "b" -> "3"), tableIdent) + def checkPartitionSerdeProps(expectedSerdeProps: Map[String, String]): Unit = { + val serdeProp = catalog.getPartition(tableIdent, spec).storage.properties + if (isUsingHiveMetastore) { + assert(normalizeSerdeProp(serdeProp) == expectedSerdeProps) + } else { + assert(serdeProp == expectedSerdeProps) + } + } + if (isUsingHiveMetastore) { + val expectedSerde = if (isDatasourceTable) { + "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe" + } else { + "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe" + } + assert(catalog.getPartition(tableIdent, spec).storage.serde == Some(expectedSerde)) + } else { + assert(catalog.getPartition(tableIdent, spec).storage.serde.isEmpty) + } + checkPartitionSerdeProps(Map.empty[String, String]) + // set table serde and/or properties (should fail on datasource tables) + if (isDatasourceTable) { + val e1 = intercept[AnalysisException] { + sql("ALTER TABLE dbx.tab1 PARTITION (a=1, b=2) SET SERDE 'whatever'") + } + val e2 = intercept[AnalysisException] { + sql("ALTER TABLE dbx.tab1 PARTITION (a=1, b=2) SET SERDE 'org.apache.madoop' " + + "WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee')") + } + assert(e1.getMessage.contains("datasource")) + assert(e2.getMessage.contains("datasource")) + } else { + sql("ALTER TABLE dbx.tab1 PARTITION (a=1, b=2) SET SERDE 'org.apache.jadoop'") + assert(catalog.getPartition(tableIdent, spec).storage.serde == Some("org.apache.jadoop")) + checkPartitionSerdeProps(Map.empty[String, String]) + sql("ALTER TABLE dbx.tab1 PARTITION (a=1, b=2) SET SERDE 'org.apache.madoop' " + + "WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee')") + assert(catalog.getPartition(tableIdent, spec).storage.serde == Some("org.apache.madoop")) + checkPartitionSerdeProps(Map("k" -> "v", "kay" -> "vee")) + } + // set serde properties only + maybeWrapException(isDatasourceTable) { + sql("ALTER TABLE dbx.tab1 PARTITION (a=1, b=2) " + + "SET SERDEPROPERTIES ('k' = 'vvv', 'kay' = 'vee')") + checkPartitionSerdeProps(Map("k" -> "vvv", "kay" -> "vee")) + } + // set things without explicitly specifying database + catalog.setCurrentDatabase("dbx") + maybeWrapException(isDatasourceTable) { + sql("ALTER TABLE tab1 PARTITION (a=1, b=2) SET SERDEPROPERTIES ('kay' = 'veee')") + checkPartitionSerdeProps(Map("k" -> "vvv", "kay" -> "veee")) + } + // table to alter does not exist + intercept[AnalysisException] { + sql("ALTER TABLE does_not_exist PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y')") + } + } + + protected def testAddPartitions(isDatasourceTable: Boolean): Unit = { + if (!isUsingHiveMetastore) { + assert(isDatasourceTable, "InMemoryCatalog only supports data source tables") + } + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + val part1 = Map("a" -> "1", "b" -> "5") + val part2 = Map("a" -> "2", "b" -> "6") + val part3 = Map("a" -> "3", "b" -> "7") + val part4 = Map("a" -> "4", "b" -> "8") + val part5 = Map("a" -> "9", "b" -> "9") + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent, isDatasourceTable) + createTablePartition(catalog, part1, tableIdent) + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part1)) + + // basic add partition + sql("ALTER TABLE dbx.tab1 ADD IF NOT EXISTS " + + "PARTITION (a='2', b='6') LOCATION 'paris' PARTITION (a='3', b='7')") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part1, part2, part3)) + assert(catalog.getPartition(tableIdent, part1).storage.locationUri.isDefined) + val partitionLocation = if (isUsingHiveMetastore) { + val tableLocation = catalog.getTableMetadata(tableIdent).storage.locationUri + assert(tableLocation.isDefined) + makeQualifiedPath(new Path(tableLocation.get.toString, "paris").toString) + } else { + new URI("paris") + } + + assert(catalog.getPartition(tableIdent, part2).storage.locationUri == Option(partitionLocation)) + assert(catalog.getPartition(tableIdent, part3).storage.locationUri.isDefined) + + // add partitions without explicitly specifying database + catalog.setCurrentDatabase("dbx") + sql("ALTER TABLE tab1 ADD IF NOT EXISTS PARTITION (a='4', b='8')") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == + Set(part1, part2, part3, part4)) + + // table to alter does not exist + intercept[AnalysisException] { + sql("ALTER TABLE does_not_exist ADD IF NOT EXISTS PARTITION (a='4', b='9')") + } + + // partition to add already exists + intercept[AnalysisException] { + sql("ALTER TABLE tab1 ADD PARTITION (a='4', b='8')") + } + + // partition to add already exists when using IF NOT EXISTS + sql("ALTER TABLE tab1 ADD IF NOT EXISTS PARTITION (a='4', b='8')") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == + Set(part1, part2, part3, part4)) + + // partition spec in ADD PARTITION should be case insensitive by default + sql("ALTER TABLE tab1 ADD PARTITION (A='9', B='9')") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == + Set(part1, part2, part3, part4, part5)) + } + + protected def testDropPartitions(isDatasourceTable: Boolean): Unit = { + if (!isUsingHiveMetastore) { + assert(isDatasourceTable, "InMemoryCatalog only supports data source tables") + } + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + val part1 = Map("a" -> "1", "b" -> "5") + val part2 = Map("a" -> "2", "b" -> "6") + val part3 = Map("a" -> "3", "b" -> "7") + val part4 = Map("a" -> "4", "b" -> "8") + val part5 = Map("a" -> "9", "b" -> "9") + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent, isDatasourceTable) + createTablePartition(catalog, part1, tableIdent) + createTablePartition(catalog, part2, tableIdent) + createTablePartition(catalog, part3, tableIdent) + createTablePartition(catalog, part4, tableIdent) + createTablePartition(catalog, part5, tableIdent) + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == + Set(part1, part2, part3, part4, part5)) + + // basic drop partition + sql("ALTER TABLE dbx.tab1 DROP IF EXISTS PARTITION (a='4', b='8'), PARTITION (a='3', b='7')") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part1, part2, part5)) + + // drop partitions without explicitly specifying database + catalog.setCurrentDatabase("dbx") + sql("ALTER TABLE tab1 DROP IF EXISTS PARTITION (a='2', b ='6')") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part1, part5)) + + // table to alter does not exist + intercept[AnalysisException] { + sql("ALTER TABLE does_not_exist DROP IF EXISTS PARTITION (a='2')") + } + + // partition to drop does not exist + intercept[AnalysisException] { + sql("ALTER TABLE tab1 DROP PARTITION (a='300')") + } + + // partition to drop does not exist when using IF EXISTS + sql("ALTER TABLE tab1 DROP IF EXISTS PARTITION (a='300')") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part1, part5)) + + // partition spec in DROP PARTITION should be case insensitive by default + sql("ALTER TABLE tab1 DROP PARTITION (A='1', B='5')") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part5)) + + // use int literal as partition value for int type partition column + sql("ALTER TABLE tab1 DROP PARTITION (a=9, b=9)") + assert(catalog.listPartitions(tableIdent).isEmpty) + } + + protected def testRenamePartitions(isDatasourceTable: Boolean): Unit = { + if (!isUsingHiveMetastore) { + assert(isDatasourceTable, "InMemoryCatalog only supports data source tables") + } + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1", Some("dbx")) + val part1 = Map("a" -> "1", "b" -> "q") + val part2 = Map("a" -> "2", "b" -> "c") + val part3 = Map("a" -> "3", "b" -> "p") + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent, isDatasourceTable) + createTablePartition(catalog, part1, tableIdent) + createTablePartition(catalog, part2, tableIdent) + createTablePartition(catalog, part3, tableIdent) + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part1, part2, part3)) + + // basic rename partition + sql("ALTER TABLE dbx.tab1 PARTITION (a='1', b='q') RENAME TO PARTITION (a='100', b='p')") + sql("ALTER TABLE dbx.tab1 PARTITION (a='2', b='c') RENAME TO PARTITION (a='20', b='c')") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == + Set(Map("a" -> "100", "b" -> "p"), Map("a" -> "20", "b" -> "c"), Map("a" -> "3", "b" -> "p"))) + + // rename without explicitly specifying database + catalog.setCurrentDatabase("dbx") + sql("ALTER TABLE tab1 PARTITION (a='100', b='p') RENAME TO PARTITION (a='10', b='p')") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == + Set(Map("a" -> "10", "b" -> "p"), Map("a" -> "20", "b" -> "c"), Map("a" -> "3", "b" -> "p"))) + + // table to alter does not exist + intercept[NoSuchTableException] { + sql("ALTER TABLE does_not_exist PARTITION (c='3') RENAME TO PARTITION (c='333')") + } + + // partition to rename does not exist + intercept[NoSuchPartitionException] { + sql("ALTER TABLE tab1 PARTITION (a='not_found', b='1') RENAME TO PARTITION (a='1', b='2')") + } + + // partition spec in RENAME PARTITION should be case insensitive by default + sql("ALTER TABLE tab1 PARTITION (A='10', B='p') RENAME TO PARTITION (A='1', B='p')") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == + Set(Map("a" -> "1", "b" -> "p"), Map("a" -> "20", "b" -> "c"), Map("a" -> "3", "b" -> "p"))) + } + + protected def testChangeColumn(isDatasourceTable: Boolean): Unit = { + if (!isUsingHiveMetastore) { + assert(isDatasourceTable, "InMemoryCatalog only supports data source tables") + } + val catalog = spark.sessionState.catalog + val resolver = spark.sessionState.conf.resolver + val tableIdent = TableIdentifier("tab1", Some("dbx")) + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent, isDatasourceTable) + def getMetadata(colName: String): Metadata = { + val column = catalog.getTableMetadata(tableIdent).schema.fields.find { field => + resolver(field.name, colName) + } + column.map(_.metadata).getOrElse(Metadata.empty) + } + // Ensure that change column will preserve other metadata fields. + sql("ALTER TABLE dbx.tab1 CHANGE COLUMN col1 col1 INT COMMENT 'this is col1'") + assert(getMetadata("col1").getString("key") == "value") + } + + test("drop build-in function") { + Seq("true", "false").foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { + // partition to add already exists + var e = intercept[AnalysisException] { + sql("DROP TEMPORARY FUNCTION year") + } + assert(e.getMessage.contains("Cannot drop native function 'year'")) + + e = intercept[AnalysisException] { + sql("DROP TEMPORARY FUNCTION YeAr") + } + assert(e.getMessage.contains("Cannot drop native function 'YeAr'")) + + e = intercept[AnalysisException] { + sql("DROP TEMPORARY FUNCTION `YeAr`") + } + assert(e.getMessage.contains("Cannot drop native function 'YeAr'")) + } + } + } + + test("describe function") { + checkAnswer( + sql("DESCRIBE FUNCTION log"), + Row("Class: org.apache.spark.sql.catalyst.expressions.Logarithm") :: + Row("Function: log") :: + Row("Usage: log(base, expr) - Returns the logarithm of `expr` with `base`.") :: Nil + ) + // predicate operator + checkAnswer( + sql("DESCRIBE FUNCTION or"), + Row("Class: org.apache.spark.sql.catalyst.expressions.Or") :: + Row("Function: or") :: + Row("Usage: expr1 or expr2 - Logical OR.") :: Nil + ) + checkAnswer( + sql("DESCRIBE FUNCTION !"), + Row("Class: org.apache.spark.sql.catalyst.expressions.Not") :: + Row("Function: !") :: + Row("Usage: ! expr - Logical not.") :: Nil + ) + // arithmetic operators + checkAnswer( + sql("DESCRIBE FUNCTION +"), + Row("Class: org.apache.spark.sql.catalyst.expressions.Add") :: + Row("Function: +") :: + Row("Usage: expr1 + expr2 - Returns `expr1`+`expr2`.") :: Nil + ) + // comparison operators + checkAnswer( + sql("DESCRIBE FUNCTION <"), + Row("Class: org.apache.spark.sql.catalyst.expressions.LessThan") :: + Row("Function: <") :: + Row("Usage: expr1 < expr2 - Returns true if `expr1` is less than `expr2`.") :: Nil + ) + // STRING + checkAnswer( + sql("DESCRIBE FUNCTION 'concat'"), + Row("Class: org.apache.spark.sql.catalyst.expressions.Concat") :: + Row("Function: concat") :: + Row("Usage: concat(str1, str2, ..., strN) - " + + "Returns the concatenation of str1, str2, ..., strN.") :: Nil + ) + // extended mode + checkAnswer( + sql("DESCRIBE FUNCTION EXTENDED ^"), + Row("Class: org.apache.spark.sql.catalyst.expressions.BitwiseXor") :: + Row( + """Extended Usage: + | Examples: + | > SELECT 3 ^ 5; + | 2 + | """.stripMargin) :: + Row("Function: ^") :: + Row("Usage: expr1 ^ expr2 - Returns the result of " + + "bitwise exclusive OR of `expr1` and `expr2`.") :: Nil + ) + } + + test("create a data source table without schema") { + import testImplicits._ + withTempPath { tempDir => + withTable("tab1", "tab2") { + (("a", "b") :: Nil).toDF().write.json(tempDir.getCanonicalPath) + + val e = intercept[AnalysisException] { sql("CREATE TABLE tab1 USING json") }.getMessage + assert(e.contains("Unable to infer schema for JSON. It must be specified manually")) + + sql(s"CREATE TABLE tab2 using json location '${tempDir.toURI}'") + checkAnswer(spark.table("tab2"), Row("a", "b")) + } + } + } + + test("create table using CLUSTERED BY without schema specification") { + import testImplicits._ + withTempPath { tempDir => + withTable("jsonTable") { + (("a", "b") :: Nil).toDF().write.json(tempDir.getCanonicalPath) + + val e = intercept[AnalysisException] { + sql( + s""" + |CREATE TABLE jsonTable + |USING org.apache.spark.sql.json + |OPTIONS ( + | path '${tempDir.getCanonicalPath}' + |) + |CLUSTERED BY (inexistentColumnA) SORTED BY (inexistentColumnB) INTO 2 BUCKETS + """.stripMargin) + } + assert(e.message == "Cannot specify bucketing information if the table schema is not " + + "specified when creating and will be inferred at runtime") + } + } + } + + test("Create Data Source Table As Select") { + import testImplicits._ + withTable("t", "t1", "t2") { + sql("CREATE TABLE t USING parquet SELECT 1 as a, 1 as b") + checkAnswer(spark.table("t"), Row(1, 1) :: Nil) + + spark.range(1).select('id as 'a, 'id as 'b).write.saveAsTable("t1") + sql("CREATE TABLE t2 USING parquet SELECT a, b from t1") + checkAnswer(spark.table("t2"), spark.table("t1")) + } + } + + test("drop current database") { + withDatabase("temp") { + sql("CREATE DATABASE temp") + sql("USE temp") + sql("DROP DATABASE temp") + val e = intercept[AnalysisException] { + sql("CREATE TABLE t (a INT, b INT) USING parquet") + }.getMessage + assert(e.contains("Database 'temp' not found")) + } + } + + test("drop default database") { + val caseSensitiveOptions = if (isUsingHiveMetastore) Seq("false") else Seq("true", "false") + caseSensitiveOptions.foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { + var message = intercept[AnalysisException] { + sql("DROP DATABASE default") + }.getMessage + assert(message.contains("Can not drop default database")) + + message = intercept[AnalysisException] { + sql("DROP DATABASE DeFault") + }.getMessage + if (caseSensitive == "true") { + assert(message.contains("Database 'DeFault' not found")) + } else { + assert(message.contains("Can not drop default database")) + } + } + } + } + + test("truncate table - datasource table") { + import testImplicits._ + + val data = (1 to 10).map { i => (i, i) }.toDF("width", "length") + // Test both a Hive compatible and incompatible code path. + Seq("json", "parquet").foreach { format => + withTable("rectangles") { + data.write.format(format).saveAsTable("rectangles") + assume(spark.table("rectangles").collect().nonEmpty, + "bad test; table was empty to begin with") + + sql("TRUNCATE TABLE rectangles") + assert(spark.table("rectangles").collect().isEmpty) + + // not supported since the table is not partitioned + assertUnsupported("TRUNCATE TABLE rectangles PARTITION (width=1)") + } + } + } + + test("truncate partitioned table - datasource table") { + import testImplicits._ + + val data = (1 to 10).map { i => (i % 3, i % 5, i) }.toDF("width", "length", "height") + + withTable("partTable") { + data.write.partitionBy("width", "length").saveAsTable("partTable") + // supported since partitions are stored in the metastore + sql("TRUNCATE TABLE partTable PARTITION (width=1, length=1)") + assert(spark.table("partTable").filter($"width" === 1).collect().nonEmpty) + assert(spark.table("partTable").filter($"width" === 1 && $"length" === 1).collect().isEmpty) + } + + withTable("partTable") { + data.write.partitionBy("width", "length").saveAsTable("partTable") + // support partial partition spec + sql("TRUNCATE TABLE partTable PARTITION (width=1)") + assert(spark.table("partTable").collect().nonEmpty) + assert(spark.table("partTable").filter($"width" === 1).collect().isEmpty) + } + + withTable("partTable") { + data.write.partitionBy("width", "length").saveAsTable("partTable") + // do nothing if no partition is matched for the given partial partition spec + sql("TRUNCATE TABLE partTable PARTITION (width=100)") + assert(spark.table("partTable").count() == data.count()) + + // throw exception if no partition is matched for the given non-partial partition spec. + intercept[NoSuchPartitionException] { + sql("TRUNCATE TABLE partTable PARTITION (width=100, length=100)") + } + + // throw exception if the column in partition spec is not a partition column. + val e = intercept[AnalysisException] { + sql("TRUNCATE TABLE partTable PARTITION (unknown=1)") + } + assert(e.message.contains("unknown is not a valid partition column")) + } + } + + test("create temporary view with mismatched schema") { + withTable("tab1") { + spark.range(10).write.saveAsTable("tab1") + withView("view1") { + val e = intercept[AnalysisException] { + sql("CREATE TEMPORARY VIEW view1 (col1, col3) AS SELECT * FROM tab1") + }.getMessage + assert(e.contains("the SELECT clause (num: `1`) does not match") + && e.contains("CREATE VIEW (num: `2`)")) + } + } + } + + test("create temporary view with specified schema") { + withView("view1") { + sql("CREATE TEMPORARY VIEW view1 (col1, col2) AS SELECT 1, 2") + checkAnswer( + sql("SELECT * FROM view1"), + Row(1, 2) :: Nil + ) + } + } + + test("block creating duplicate temp table") { + withView("t_temp") { + sql("CREATE TEMPORARY VIEW t_temp AS SELECT 1, 2") + val e = intercept[TempTableAlreadyExistsException] { + sql("CREATE TEMPORARY TABLE t_temp (c3 int, c4 string) USING JSON") + }.getMessage + assert(e.contains("Temporary table 't_temp' already exists")) + } + } + + test("truncate table - external table, temporary table, view (not allowed)") { + import testImplicits._ + withTempPath { tempDir => + withTable("my_ext_tab") { + (("a", "b") :: Nil).toDF().write.parquet(tempDir.getCanonicalPath) + (1 to 10).map { i => (i, i) }.toDF("a", "b").createTempView("my_temp_tab") + sql(s"CREATE TABLE my_ext_tab using parquet LOCATION '${tempDir.toURI}'") + sql(s"CREATE VIEW my_view AS SELECT 1") + intercept[NoSuchTableException] { + sql("TRUNCATE TABLE my_temp_tab") + } + assertUnsupported("TRUNCATE TABLE my_ext_tab") + assertUnsupported("TRUNCATE TABLE my_view") + } + } + } + + test("truncate table - non-partitioned table (not allowed)") { + withTable("my_tab") { + sql("CREATE TABLE my_tab (age INT, name STRING) using parquet") + sql("INSERT INTO my_tab values (10, 'a')") + assertUnsupported("TRUNCATE TABLE my_tab PARTITION (age=10)") + } + } + + test("SPARK-16034 Partition columns should match when appending to existing data source tables") { + import testImplicits._ + val df = Seq((1, 2, 3)).toDF("a", "b", "c") + withTable("partitionedTable") { + df.write.mode("overwrite").partitionBy("a", "b").saveAsTable("partitionedTable") + // Misses some partition columns + intercept[AnalysisException] { + df.write.mode("append").partitionBy("a").saveAsTable("partitionedTable") + } + // Wrong order + intercept[AnalysisException] { + df.write.mode("append").partitionBy("b", "a").saveAsTable("partitionedTable") + } + // Partition columns not specified + intercept[AnalysisException] { + df.write.mode("append").saveAsTable("partitionedTable") + } + assert(sql("select * from partitionedTable").collect().size == 1) + // Inserts new data successfully when partition columns are correctly specified in + // partitionBy(...). + // TODO: Right now, partition columns are always treated in a case-insensitive way. + // See the write method in DataSource.scala. + Seq((4, 5, 6)).toDF("a", "B", "c") + .write + .mode("append") + .partitionBy("a", "B") + .saveAsTable("partitionedTable") + + Seq((7, 8, 9)).toDF("a", "b", "c") + .write + .mode("append") + .partitionBy("a", "b") + .saveAsTable("partitionedTable") + + checkAnswer( + sql("select a, b, c from partitionedTable"), + Row(1, 2, 3) :: Row(4, 5, 6) :: Row(7, 8, 9) :: Nil + ) + } + } + + test("show functions") { + withUserDefinedFunction("add_one" -> true) { + val numFunctions = FunctionRegistry.functionSet.size.toLong + assert(sql("show functions").count() === numFunctions) + assert(sql("show system functions").count() === numFunctions) + assert(sql("show all functions").count() === numFunctions) + assert(sql("show user functions").count() === 0L) + spark.udf.register("add_one", (x: Long) => x + 1) + assert(sql("show functions").count() === numFunctions + 1L) + assert(sql("show system functions").count() === numFunctions) + assert(sql("show all functions").count() === numFunctions + 1L) + assert(sql("show user functions").count() === 1L) + } + } + + test("show columns - negative test") { + // When case sensitivity is true, the user supplied database name in table identifier + // should match the supplied database name in case sensitive way. + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withTempDatabase { db => + val tabName = s"$db.showcolumn" + withTable(tabName) { + sql(s"CREATE TABLE $tabName(col1 int, col2 string) USING parquet ") + val message = intercept[AnalysisException] { + sql(s"SHOW COLUMNS IN $db.showcolumn FROM ${db.toUpperCase(Locale.ROOT)}") + }.getMessage + assert(message.contains("SHOW COLUMNS with conflicting databases")) + } + } + } + } + + test("SPARK-18009 calling toLocalIterator on commands") { + import scala.collection.JavaConverters._ + val df = sql("show databases") + val rows: Seq[Row] = df.toLocalIterator().asScala.toSeq + assert(rows.length > 0) + } + + test("SET LOCATION for managed table") { + withTable("tbl") { + withTempDir { dir => + sql("CREATE TABLE tbl(i INT) USING parquet") + sql("INSERT INTO tbl SELECT 1") + checkAnswer(spark.table("tbl"), Row(1)) + val defaultTablePath = spark.sessionState.catalog + .getTableMetadata(TableIdentifier("tbl")).storage.locationUri.get + try { + sql(s"ALTER TABLE tbl SET LOCATION '${dir.toURI}'") + spark.catalog.refreshTable("tbl") + // SET LOCATION won't move data from previous table path to new table path. + assert(spark.table("tbl").count() == 0) + // the previous table path should be still there. + assert(new File(defaultTablePath).exists()) + + sql("INSERT INTO tbl SELECT 2") + checkAnswer(spark.table("tbl"), Row(2)) + // newly inserted data will go to the new table path. + assert(dir.listFiles().nonEmpty) + + sql("DROP TABLE tbl") + // the new table path will be removed after DROP TABLE. + assert(!dir.exists()) + } finally { + Utils.deleteRecursively(new File(defaultTablePath)) + } + } + } + } + + test("insert data to a data source table which has a non-existing location should succeed") { + withTable("t") { + withTempDir { dir => + spark.sql( + s""" + |CREATE TABLE t(a string, b int) + |USING parquet + |OPTIONS(path "${dir.toURI}") + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + dir.delete + assert(!dir.exists) + spark.sql("INSERT INTO TABLE t SELECT 'c', 1") + assert(dir.exists) + checkAnswer(spark.table("t"), Row("c", 1) :: Nil) + + Utils.deleteRecursively(dir) + assert(!dir.exists) + spark.sql("INSERT OVERWRITE TABLE t SELECT 'c', 1") + assert(dir.exists) + checkAnswer(spark.table("t"), Row("c", 1) :: Nil) + + val newDirFile = new File(dir, "x") + val newDir = newDirFile.toURI + spark.sql(s"ALTER TABLE t SET LOCATION '$newDir'") + spark.sessionState.catalog.refreshTable(TableIdentifier("t")) + + val table1 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table1.location == newDir) + assert(!newDirFile.exists) + + spark.sql("INSERT INTO TABLE t SELECT 'c', 1") + assert(newDirFile.exists) + checkAnswer(spark.table("t"), Row("c", 1) :: Nil) + } + } + } + + test("insert into a data source table with a non-existing partition location should succeed") { + withTable("t") { + withTempDir { dir => + spark.sql( + s""" + |CREATE TABLE t(a int, b int, c int, d int) + |USING parquet + |PARTITIONED BY(a, b) + |LOCATION "${dir.toURI}" + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + spark.sql("INSERT INTO TABLE t PARTITION(a=1, b=2) SELECT 3, 4") + checkAnswer(spark.table("t"), Row(3, 4, 1, 2) :: Nil) + + val partLoc = new File(s"${dir.getAbsolutePath}/a=1") + Utils.deleteRecursively(partLoc) + assert(!partLoc.exists()) + // insert overwrite into a partition which location has been deleted. + spark.sql("INSERT OVERWRITE TABLE t PARTITION(a=1, b=2) SELECT 7, 8") + assert(partLoc.exists()) + checkAnswer(spark.table("t"), Row(7, 8, 1, 2) :: Nil) + } + } + } + + test("read data from a data source table which has a non-existing location should succeed") { + withTable("t") { + withTempDir { dir => + spark.sql( + s""" + |CREATE TABLE t(a string, b int) + |USING parquet + |OPTIONS(path "${dir.toURI}") + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + dir.delete() + checkAnswer(spark.table("t"), Nil) + + val newDirFile = new File(dir, "x") + val newDir = newDirFile.toURI + spark.sql(s"ALTER TABLE t SET LOCATION '$newDir'") + + val table1 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table1.location == newDir) + assert(!newDirFile.exists()) + checkAnswer(spark.table("t"), Nil) + } + } + } + + test("read data from a data source table with non-existing partition location should succeed") { + withTable("t") { + withTempDir { dir => + spark.sql( + s""" + |CREATE TABLE t(a int, b int, c int, d int) + |USING parquet + |PARTITIONED BY(a, b) + |LOCATION "${dir.toURI}" + """.stripMargin) + spark.sql("INSERT INTO TABLE t PARTITION(a=1, b=2) SELECT 3, 4") + checkAnswer(spark.table("t"), Row(3, 4, 1, 2) :: Nil) + + // select from a partition which location has been deleted. + Utils.deleteRecursively(dir) + assert(!dir.exists()) + spark.sql("REFRESH TABLE t") + checkAnswer(spark.sql("select * from t where a=1 and b=2"), Nil) + } + } + } + + test("create datasource table with a non-existing location") { + withTable("t", "t1") { + withTempPath { dir => + spark.sql(s"CREATE TABLE t(a int, b int) USING parquet LOCATION '${dir.toURI}'") + + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + spark.sql("INSERT INTO TABLE t SELECT 1, 2") + assert(dir.exists()) + + checkAnswer(spark.table("t"), Row(1, 2)) + } + // partition table + withTempPath { dir => + spark.sql( + s"CREATE TABLE t1(a int, b int) USING parquet PARTITIONED BY(a) LOCATION '${dir.toURI}'") + + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1")) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + spark.sql("INSERT INTO TABLE t1 PARTITION(a=1) SELECT 2") + + val partDir = new File(dir, "a=1") + assert(partDir.exists()) + + checkAnswer(spark.table("t1"), Row(2, 1)) + } + } + } + + Seq(true, false).foreach { shouldDelete => + val tcName = if (shouldDelete) "non-existing" else "existed" + test(s"CTAS for external data source table with a $tcName location") { + withTable("t", "t1") { + withTempDir { dir => + if (shouldDelete) dir.delete() + spark.sql( + s""" + |CREATE TABLE t + |USING parquet + |LOCATION '${dir.toURI}' + |AS SELECT 3 as a, 4 as b, 1 as c, 2 as d + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + checkAnswer(spark.table("t"), Row(3, 4, 1, 2)) + } + // partition table + withTempDir { dir => + if (shouldDelete) dir.delete() + spark.sql( + s""" + |CREATE TABLE t1 + |USING parquet + |PARTITIONED BY(a, b) + |LOCATION '${dir.toURI}' + |AS SELECT 3 as a, 4 as b, 1 as c, 2 as d + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1")) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + val partDir = new File(dir, "a=3") + assert(partDir.exists()) + + checkAnswer(spark.table("t1"), Row(1, 2, 3, 4)) + } + } + } + } + + Seq("a b", "a:b", "a%b", "a,b").foreach { specialChars => + test(s"data source table:partition column name containing $specialChars") { + // On Windows, it looks colon in the file name is illegal by default. See + // https://support.microsoft.com/en-us/help/289627 + assume(!Utils.isWindows || specialChars != "a:b") + + withTable("t") { + withTempDir { dir => + spark.sql( + s""" + |CREATE TABLE t(a string, `$specialChars` string) + |USING parquet + |PARTITIONED BY(`$specialChars`) + |LOCATION '${dir.toURI}' + """.stripMargin) + + assert(dir.listFiles().isEmpty) + spark.sql(s"INSERT INTO TABLE t PARTITION(`$specialChars`=2) SELECT 1") + val partEscaped = s"${ExternalCatalogUtils.escapePathName(specialChars)}=2" + val partFile = new File(dir, partEscaped) + assert(partFile.listFiles().nonEmpty) + checkAnswer(spark.table("t"), Row("1", "2") :: Nil) + } + } + } + } + + Seq("a b", "a:b", "a%b").foreach { specialChars => + test(s"location uri contains $specialChars for datasource table") { + // On Windows, it looks colon in the file name is illegal by default. See + // https://support.microsoft.com/en-us/help/289627 + assume(!Utils.isWindows || specialChars != "a:b") + + withTable("t", "t1") { + withTempDir { dir => + val loc = new File(dir, specialChars) + loc.mkdir() + // The parser does not recognize the backslashes on Windows as they are. + // These currently should be escaped. + val escapedLoc = loc.getAbsolutePath.replace("\\", "\\\\") + spark.sql( + s""" + |CREATE TABLE t(a string) + |USING parquet + |LOCATION '$escapedLoc' + """.stripMargin) + + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.location == makeQualifiedPath(loc.getAbsolutePath)) + assert(new Path(table.location).toString.contains(specialChars)) + + assert(loc.listFiles().isEmpty) + spark.sql("INSERT INTO TABLE t SELECT 1") + assert(loc.listFiles().nonEmpty) + checkAnswer(spark.table("t"), Row("1") :: Nil) + } + + withTempDir { dir => + val loc = new File(dir, specialChars) + loc.mkdir() + // The parser does not recognize the backslashes on Windows as they are. + // These currently should be escaped. + val escapedLoc = loc.getAbsolutePath.replace("\\", "\\\\") + spark.sql( + s""" + |CREATE TABLE t1(a string, b string) + |USING parquet + |PARTITIONED BY(b) + |LOCATION '$escapedLoc' + """.stripMargin) + + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1")) + assert(table.location == makeQualifiedPath(loc.getAbsolutePath)) + assert(new Path(table.location).toString.contains(specialChars)) + + assert(loc.listFiles().isEmpty) + spark.sql("INSERT INTO TABLE t1 PARTITION(b=2) SELECT 1") + val partFile = new File(loc, "b=2") + assert(partFile.listFiles().nonEmpty) + checkAnswer(spark.table("t1"), Row("1", "2") :: Nil) + + spark.sql("INSERT INTO TABLE t1 PARTITION(b='2017-03-03 12:13%3A14') SELECT 1") + val partFile1 = new File(loc, "b=2017-03-03 12:13%3A14") + assert(!partFile1.exists()) + + if (!Utils.isWindows) { + // Actual path becomes "b=2017-03-03%2012%3A13%253A14" on Windows. + val partFile2 = new File(loc, "b=2017-03-03 12%3A13%253A14") + assert(partFile2.listFiles().nonEmpty) + checkAnswer( + spark.table("t1"), Row("1", "2") :: Row("1", "2017-03-03 12:13%3A14") :: Nil) + } + } + } + } + } + + Seq("a b", "a:b", "a%b").foreach { specialChars => + test(s"location uri contains $specialChars for database") { + // On Windows, it looks colon in the file name is illegal by default. See + // https://support.microsoft.com/en-us/help/289627 + assume(!Utils.isWindows || specialChars != "a:b") + + withDatabase ("tmpdb") { + withTable("t") { + withTempDir { dir => + val loc = new File(dir, specialChars) + // The parser does not recognize the backslashes on Windows as they are. + // These currently should be escaped. + val escapedLoc = loc.getAbsolutePath.replace("\\", "\\\\") + spark.sql(s"CREATE DATABASE tmpdb LOCATION '$escapedLoc'") + spark.sql("USE tmpdb") + + import testImplicits._ + Seq(1).toDF("a").write.saveAsTable("t") + val tblloc = new File(loc, "t") + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.location == makeQualifiedPath(tblloc.getAbsolutePath)) + assert(tblloc.listFiles().nonEmpty) + } + } + } + } + } + + test("the qualified path of a datasource table is stored in the catalog") { + withTable("t", "t1") { + withTempDir { dir => + assert(!dir.getAbsolutePath.startsWith("file:/")) + // The parser does not recognize the backslashes on Windows as they are. + // These currently should be escaped. + val escapedDir = dir.getAbsolutePath.replace("\\", "\\\\") + spark.sql( + s""" + |CREATE TABLE t(a string) + |USING parquet + |LOCATION '$escapedDir' + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.location.toString.startsWith("file:/")) + } + + withTempDir { dir => + assert(!dir.getAbsolutePath.startsWith("file:/")) + // The parser does not recognize the backslashes on Windows as they are. + // These currently should be escaped. + val escapedDir = dir.getAbsolutePath.replace("\\", "\\\\") + spark.sql( + s""" + |CREATE TABLE t1(a string, b string) + |USING parquet + |PARTITIONED BY(b) + |LOCATION '$escapedDir' + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1")) + assert(table.location.toString.startsWith("file:/")) + } + } + } + + val supportedNativeFileFormatsForAlterTableAddColumns = Seq("parquet", "json", "csv") + + supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider => + test(s"alter datasource table add columns - $provider") { + withTable("t1") { + sql(s"CREATE TABLE t1 (c1 int) USING $provider") + sql("INSERT INTO t1 VALUES (1)") + sql("ALTER TABLE t1 ADD COLUMNS (c2 int)") + checkAnswer( + spark.table("t1"), + Seq(Row(1, null)) + ) + checkAnswer( + sql("SELECT * FROM t1 WHERE c2 is null"), + Seq(Row(1, null)) + ) + + sql("INSERT INTO t1 VALUES (3, 2)") + checkAnswer( + sql("SELECT * FROM t1 WHERE c2 = 2"), + Seq(Row(3, 2)) + ) + } + } + } + + supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider => + test(s"alter datasource table add columns - partitioned - $provider") { + withTable("t1") { + sql(s"CREATE TABLE t1 (c1 int, c2 int) USING $provider PARTITIONED BY (c2)") + sql("INSERT INTO t1 PARTITION(c2 = 2) VALUES (1)") + sql("ALTER TABLE t1 ADD COLUMNS (c3 int)") + checkAnswer( + spark.table("t1"), + Seq(Row(1, null, 2)) + ) + checkAnswer( + sql("SELECT * FROM t1 WHERE c3 is null"), + Seq(Row(1, null, 2)) + ) + sql("INSERT INTO t1 PARTITION(c2 =1) VALUES (2, 3)") + checkAnswer( + sql("SELECT * FROM t1 WHERE c3 = 3"), + Seq(Row(2, 3, 1)) + ) + checkAnswer( + sql("SELECT * FROM t1 WHERE c2 = 1"), + Seq(Row(2, 3, 1)) + ) + } + } + } + + test("alter datasource table add columns - text format not supported") { + withTable("t1") { + sql("CREATE TABLE t1 (c1 int) USING text") + val e = intercept[AnalysisException] { + sql("ALTER TABLE t1 ADD COLUMNS (c2 int)") + }.getMessage + assert(e.contains("ALTER ADD COLUMNS does not support datasource table with type")) + } + } + + test("alter table add columns -- not support temp view") { + withTempView("tmp_v") { + sql("CREATE TEMPORARY VIEW tmp_v AS SELECT 1 AS c1, 2 AS c2") + val e = intercept[AnalysisException] { + sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)") + } + assert(e.message.contains("ALTER ADD COLUMNS does not support views")) + } + } + + test("alter table add columns -- not support view") { + withView("v1") { + sql("CREATE VIEW v1 AS SELECT 1 AS c1, 2 AS c2") + val e = intercept[AnalysisException] { + sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)") + } + assert(e.message.contains("ALTER ADD COLUMNS does not support views")) + } + } + + test("alter table add columns with existing column name") { + withTable("t1") { + sql("CREATE TABLE t1 (c1 int) USING PARQUET") + val e = intercept[AnalysisException] { + sql("ALTER TABLE t1 ADD COLUMNS (c1 string)") + }.getMessage + assert(e.contains("Found duplicate column(s)")) + } + } + + test("create temporary function with if not exists") { + withUserDefinedFunction("func1" -> true) { + val sql1 = + """ + |CREATE TEMPORARY FUNCTION IF NOT EXISTS func1 as + |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', + |JAR '/path/to/jar2' + """.stripMargin + val e = intercept[AnalysisException] { + sql(sql1) + }.getMessage + assert(e.contains("It is not allowed to define a TEMPORARY function with IF NOT EXISTS")) + } + } + + test("create function with both if not exists and replace") { + withUserDefinedFunction("func1" -> false) { + val sql1 = + """ + |CREATE OR REPLACE FUNCTION IF NOT EXISTS func1 as + |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', + |JAR '/path/to/jar2' + """.stripMargin + val e = intercept[AnalysisException] { + sql(sql1) + }.getMessage + assert(e.contains("CREATE FUNCTION with both IF NOT EXISTS and REPLACE is not allowed")) + } + } + + test("create temporary function by specifying a database") { + val dbName = "mydb" + withDatabase(dbName) { + sql(s"CREATE DATABASE $dbName") + sql(s"USE $dbName") + withUserDefinedFunction("func1" -> true) { + val sql1 = + s""" + |CREATE TEMPORARY FUNCTION $dbName.func1 as + |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', + |JAR '/path/to/jar2' + """.stripMargin + val e = intercept[AnalysisException] { + sql(sql1) + }.getMessage + assert(e.contains(s"Specifying a database in CREATE TEMPORARY FUNCTION " + + s"is not allowed: '$dbName'")) + } + } + } + + Seq(true, false).foreach { caseSensitive => + test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> s"$caseSensitive") { + withTable("t1") { + sql("CREATE TABLE t1 (c1 int) USING PARQUET") + if (!caseSensitive) { + val e = intercept[AnalysisException] { + sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") + }.getMessage + assert(e.contains("Found duplicate column(s)")) + } else { + sql("ALTER TABLE t1 ADD COLUMNS (C1 string)") + assert(spark.table("t1").schema == + new StructType().add("c1", IntegerType).add("C1", StringType)) + } + } + } + } + + test(s"basic DDL using locale tr - caseSensitive $caseSensitive") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> s"$caseSensitive") { + withLocale("tr") { + val dbName = "DaTaBaSe_I" + withDatabase(dbName) { + sql(s"CREATE DATABASE $dbName") + sql(s"USE $dbName") + + val tabName = "tAb_I" + withTable(tabName) { + sql(s"CREATE TABLE $tabName(col_I int) USING PARQUET") + sql(s"INSERT OVERWRITE TABLE $tabName SELECT 1") + checkAnswer(sql(s"SELECT col_I FROM $tabName"), Row(1) :: Nil) + } + } + } + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BucketingUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BucketingUtilsSuite.scala new file mode 100644 index 000000000000..9d892bbdba4c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BucketingUtilsSuite.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.SparkFunSuite + +class BucketingUtilsSuite extends SparkFunSuite { + + test("generate bucket id") { + assert(BucketingUtils.bucketIdToString(0) == "_00000") + assert(BucketingUtils.bucketIdToString(10) == "_00010") + assert(BucketingUtils.bucketIdToString(999999) == "_999999") + } + + test("match bucket ids") { + def testCase(filename: String, expected: Option[Int]): Unit = withClue(s"name: $filename") { + assert(BucketingUtils.getBucketId(filename) == expected) + } + + testCase("a_1", Some(1)) + testCase("a_1.txt", Some(1)) + testCase("a_9999999", Some(9999999)) + testCase("a_9999999.txt", Some(9999999)) + testCase("a_1.c2.txt", Some(1)) + testCase("a_1.", Some(1)) + + testCase("a_1:txt", None) + testCase("a_1-c2.txt", None) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala new file mode 100644 index 000000000000..a0c1ea63d382 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.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.execution.datasources + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.SharedSQLContext + +class FileFormatWriterSuite extends QueryTest with SharedSQLContext { + + test("empty file should be skipped while write to file") { + withTempPath { path => + spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) + val partFiles = path.listFiles() + .filter(f => f.isFile && !f.getName.startsWith(".") && !f.getName.startsWith("_")) + assert(partFiles.length === 2) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala new file mode 100644 index 000000000000..b4616826e40b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala @@ -0,0 +1,259 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.io.File +import java.net.URI + +import scala.collection.mutable +import scala.language.reflectiveCalls + +import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem} + +import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.util.{KnownSizeEstimation, SizeEstimator} + +class FileIndexSuite extends SharedSQLContext { + + test("InMemoryFileIndex: leaf files are qualified paths") { + withTempDir { dir => + val file = new File(dir, "text.txt") + stringToFile(file, "text") + + val path = new Path(file.getCanonicalPath) + val catalog = new InMemoryFileIndex(spark, Seq(path), Map.empty, None) { + def leafFilePaths: Seq[Path] = leafFiles.keys.toSeq + def leafDirPaths: Seq[Path] = leafDirToChildrenFiles.keys.toSeq + } + assert(catalog.leafFilePaths.forall(p => p.toString.startsWith("file:/"))) + assert(catalog.leafDirPaths.forall(p => p.toString.startsWith("file:/"))) + } + } + + test("InMemoryFileIndex: input paths are converted to qualified paths") { + withTempDir { dir => + val file = new File(dir, "text.txt") + stringToFile(file, "text") + + val unqualifiedDirPath = new Path(dir.getCanonicalPath) + val unqualifiedFilePath = new Path(file.getCanonicalPath) + require(!unqualifiedDirPath.toString.contains("file:")) + require(!unqualifiedFilePath.toString.contains("file:")) + + val fs = unqualifiedDirPath.getFileSystem(sparkContext.hadoopConfiguration) + val qualifiedFilePath = fs.makeQualified(new Path(file.getCanonicalPath)) + require(qualifiedFilePath.toString.startsWith("file:")) + + val catalog1 = new InMemoryFileIndex( + spark, Seq(unqualifiedDirPath), Map.empty, None) + assert(catalog1.allFiles.map(_.getPath) === Seq(qualifiedFilePath)) + + val catalog2 = new InMemoryFileIndex( + spark, Seq(unqualifiedFilePath), Map.empty, None) + assert(catalog2.allFiles.map(_.getPath) === Seq(qualifiedFilePath)) + + } + } + + test("InMemoryFileIndex: folders that don't exist don't throw exceptions") { + withTempDir { dir => + val deletedFolder = new File(dir, "deleted") + assert(!deletedFolder.exists()) + val catalog1 = new InMemoryFileIndex( + spark, Seq(new Path(deletedFolder.getCanonicalPath)), Map.empty, None) + // doesn't throw an exception + assert(catalog1.listLeafFiles(catalog1.rootPaths).isEmpty) + } + } + + test("PartitioningAwareFileIndex listing parallelized with many top level dirs") { + for ((scale, expectedNumPar) <- Seq((10, 0), (50, 1))) { + withTempDir { dir => + val topLevelDirs = (1 to scale).map { i => + val tmp = new File(dir, s"foo=$i.txt") + tmp.mkdir() + new Path(tmp.getCanonicalPath) + } + HiveCatalogMetrics.reset() + assert(HiveCatalogMetrics.METRIC_PARALLEL_LISTING_JOB_COUNT.getCount() == 0) + new InMemoryFileIndex(spark, topLevelDirs, Map.empty, None) + assert(HiveCatalogMetrics.METRIC_PARALLEL_LISTING_JOB_COUNT.getCount() == expectedNumPar) + } + } + } + + test("PartitioningAwareFileIndex listing parallelized with large child dirs") { + for ((scale, expectedNumPar) <- Seq((10, 0), (50, 1))) { + withTempDir { dir => + for (i <- 1 to scale) { + new File(dir, s"foo=$i.txt").mkdir() + } + HiveCatalogMetrics.reset() + assert(HiveCatalogMetrics.METRIC_PARALLEL_LISTING_JOB_COUNT.getCount() == 0) + new InMemoryFileIndex(spark, Seq(new Path(dir.getCanonicalPath)), Map.empty, None) + assert(HiveCatalogMetrics.METRIC_PARALLEL_LISTING_JOB_COUNT.getCount() == expectedNumPar) + } + } + } + + test("PartitioningAwareFileIndex listing parallelized with large, deeply nested child dirs") { + for ((scale, expectedNumPar) <- Seq((10, 0), (50, 4))) { + withTempDir { dir => + for (i <- 1 to 2) { + val subdirA = new File(dir, s"a=$i") + subdirA.mkdir() + for (j <- 1 to 2) { + val subdirB = new File(subdirA, s"b=$j") + subdirB.mkdir() + for (k <- 1 to scale) { + new File(subdirB, s"foo=$k.txt").mkdir() + } + } + } + HiveCatalogMetrics.reset() + assert(HiveCatalogMetrics.METRIC_PARALLEL_LISTING_JOB_COUNT.getCount() == 0) + new InMemoryFileIndex(spark, Seq(new Path(dir.getCanonicalPath)), Map.empty, None) + assert(HiveCatalogMetrics.METRIC_PARALLEL_LISTING_JOB_COUNT.getCount() == expectedNumPar) + } + } + } + + test("InMemoryFileIndex - file filtering") { + assert(!InMemoryFileIndex.shouldFilterOut("abcd")) + assert(InMemoryFileIndex.shouldFilterOut(".ab")) + assert(InMemoryFileIndex.shouldFilterOut("_cd")) + assert(!InMemoryFileIndex.shouldFilterOut("_metadata")) + assert(!InMemoryFileIndex.shouldFilterOut("_common_metadata")) + assert(InMemoryFileIndex.shouldFilterOut("_ab_metadata")) + assert(InMemoryFileIndex.shouldFilterOut("_cd_common_metadata")) + assert(InMemoryFileIndex.shouldFilterOut("a._COPYING_")) + } + + test("SPARK-17613 - PartitioningAwareFileIndex: base path w/o '/' at end") { + class MockCatalog( + override val rootPaths: Seq[Path]) + extends PartitioningAwareFileIndex(spark, Map.empty, None) { + + override def refresh(): Unit = {} + + override def leafFiles: mutable.LinkedHashMap[Path, FileStatus] = mutable.LinkedHashMap( + new Path("mockFs://some-bucket/file1.json") -> new FileStatus() + ) + + override def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] = Map( + new Path("mockFs://some-bucket/") -> Array(new FileStatus()) + ) + + override def partitionSpec(): PartitionSpec = { + PartitionSpec.emptySpec + } + } + + withSQLConf( + "fs.mockFs.impl" -> classOf[FakeParentPathFileSystem].getName, + "fs.mockFs.impl.disable.cache" -> "true") { + val pathWithSlash = new Path("mockFs://some-bucket/") + assert(pathWithSlash.getParent === null) + val pathWithoutSlash = new Path("mockFs://some-bucket") + assert(pathWithoutSlash.getParent === null) + val catalog1 = new MockCatalog(Seq(pathWithSlash)) + val catalog2 = new MockCatalog(Seq(pathWithoutSlash)) + assert(catalog1.allFiles().nonEmpty) + assert(catalog2.allFiles().nonEmpty) + } + } + + test("InMemoryFileIndex with empty rootPaths when PARALLEL_PARTITION_DISCOVERY_THRESHOLD" + + "is a nonpositive number") { + withSQLConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD.key -> "0") { + new InMemoryFileIndex(spark, Seq.empty, Map.empty, None) + } + + val e = intercept[IllegalArgumentException] { + withSQLConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD.key -> "-1") { + new InMemoryFileIndex(spark, Seq.empty, Map.empty, None) + } + }.getMessage + assert(e.contains("The maximum number of paths allowed for listing files at " + + "driver side must not be negative")) + } + + test("refresh for InMemoryFileIndex with FileStatusCache") { + withTempDir { dir => + val fileStatusCache = FileStatusCache.getOrCreate(spark) + val dirPath = new Path(dir.getAbsolutePath) + val fs = dirPath.getFileSystem(spark.sessionState.newHadoopConf()) + val catalog = + new InMemoryFileIndex(spark, Seq(dirPath), Map.empty, None, fileStatusCache) { + def leafFilePaths: Seq[Path] = leafFiles.keys.toSeq + def leafDirPaths: Seq[Path] = leafDirToChildrenFiles.keys.toSeq + } + + val file = new File(dir, "text.txt") + stringToFile(file, "text") + assert(catalog.leafDirPaths.isEmpty) + assert(catalog.leafFilePaths.isEmpty) + + catalog.refresh() + + assert(catalog.leafFilePaths.size == 1) + assert(catalog.leafFilePaths.head == fs.makeQualified(new Path(file.getAbsolutePath))) + + assert(catalog.leafDirPaths.size == 1) + assert(catalog.leafDirPaths.head == fs.makeQualified(dirPath)) + } + } + + test("SPARK-20280 - FileStatusCache with a partition with very many files") { + /* fake the size, otherwise we need to allocate 2GB of data to trigger this bug */ + class MyFileStatus extends FileStatus with KnownSizeEstimation { + override def estimatedSize: Long = 1000 * 1000 * 1000 + } + /* files * MyFileStatus.estimatedSize should overflow to negative integer + * so, make it between 2bn and 4bn + */ + val files = (1 to 3).map { i => + new MyFileStatus() + } + val fileStatusCache = FileStatusCache.getOrCreate(spark) + fileStatusCache.putLeafFiles(new Path("/tmp", "abc"), files.toArray) + } + + test("SPARK-20367 - properly unescape column names in inferPartitioning") { + withTempPath { path => + val colToUnescape = "Column/#%'?" + spark + .range(1) + .select(col("id").as(colToUnescape), col("id")) + .write.partitionBy(colToUnescape).parquet(path.getAbsolutePath) + assert(spark.read.parquet(path.getAbsolutePath).schema.exists(_.name == colToUnescape)) + } + } +} + +class FakeParentPathFileSystem extends RawLocalFileSystem { + override def getScheme: String = "mockFs" + + override def getUri: URI = { + URI.create("mockFs://some-bucket") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala new file mode 100644 index 000000000000..c1d61b843d89 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -0,0 +1,660 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.io._ +import java.util.concurrent.atomic.AtomicInteger +import java.util.zip.GZIPOutputStream + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{BlockLocation, FileStatus, Path, RawLocalFileSystem} +import org.apache.hadoop.mapreduce.Job + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionSet, PredicateHelper} +import org.apache.spark.sql.catalyst.util +import org.apache.spark.sql.execution.{DataSourceScanExec, SparkPlan} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.util.Utils + +class FileSourceStrategySuite extends QueryTest with SharedSQLContext with PredicateHelper { + import testImplicits._ + + protected override def sparkConf = super.sparkConf.set("spark.default.parallelism", "1") + + test("unpartitioned table, single partition") { + val table = + createTable( + files = Seq( + "file1" -> 1, + "file2" -> 1, + "file3" -> 1, + "file4" -> 1, + "file5" -> 1, + "file6" -> 1, + "file7" -> 1, + "file8" -> 1, + "file9" -> 1, + "file10" -> 1)) + + checkScan(table.select('c1)) { partitions => + // 10 one byte files should fit in a single partition with 10 files. + assert(partitions.size == 1, "when checking partitions") + assert(partitions.head.files.size == 10, "when checking partition 1") + // 1 byte files are too small to split so we should read the whole thing. + assert(partitions.head.files.head.start == 0) + assert(partitions.head.files.head.length == 1) + } + + checkPartitionSchema(StructType(Nil)) + checkDataSchema(StructType(Nil).add("c1", IntegerType)) + } + + test("unpartitioned table, multiple partitions") { + val table = + createTable( + files = Seq( + "file1" -> 5, + "file2" -> 5, + "file3" -> 5)) + + withSQLConf(SQLConf.FILES_MAX_PARTITION_BYTES.key -> "11", + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> "1") { + checkScan(table.select('c1)) { partitions => + // 5 byte files should be laid out [(5, 5), (5)] + assert(partitions.size == 2, "when checking partitions") + assert(partitions(0).files.size == 2, "when checking partition 1") + assert(partitions(1).files.size == 1, "when checking partition 2") + + // 5 byte files are too small to split so we should read the whole thing. + assert(partitions.head.files.head.start == 0) + assert(partitions.head.files.head.length == 5) + } + + checkPartitionSchema(StructType(Nil)) + checkDataSchema(StructType(Nil).add("c1", IntegerType)) + } + } + + test("Unpartitioned table, large file that gets split") { + val table = + createTable( + files = Seq( + "file1" -> 15, + "file2" -> 3)) + + withSQLConf(SQLConf.FILES_MAX_PARTITION_BYTES.key -> "10", + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> "1") { + checkScan(table.select('c1)) { partitions => + // Files should be laid out [(0-10), (10-15, 4)] + assert(partitions.size == 2, "when checking partitions") + assert(partitions(0).files.size == 1, "when checking partition 1") + assert(partitions(1).files.size == 2, "when checking partition 2") + + // Start by reading 10 bytes of the first file + assert(partitions.head.files.head.start == 0) + assert(partitions.head.files.head.length == 10) + + // Second partition reads the remaining 5 + assert(partitions(1).files.head.start == 10) + assert(partitions(1).files.head.length == 5) + } + + checkPartitionSchema(StructType(Nil)) + checkDataSchema(StructType(Nil).add("c1", IntegerType)) + } + } + + test("Unpartitioned table, many files that get split") { + val table = + createTable( + files = Seq( + "file1" -> 2, + "file2" -> 2, + "file3" -> 1, + "file4" -> 1, + "file5" -> 1, + "file6" -> 1)) + + withSQLConf(SQLConf.FILES_MAX_PARTITION_BYTES.key -> "4", + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> "1") { + checkScan(table.select('c1)) { partitions => + // Files should be laid out [(file1), (file2, file3), (file4, file5), (file6)] + assert(partitions.size == 4, "when checking partitions") + assert(partitions(0).files.size == 1, "when checking partition 1") + assert(partitions(1).files.size == 2, "when checking partition 2") + assert(partitions(2).files.size == 2, "when checking partition 3") + assert(partitions(3).files.size == 1, "when checking partition 4") + + // First partition reads (file1) + assert(partitions(0).files(0).start == 0) + assert(partitions(0).files(0).length == 2) + + // Second partition reads (file2, file3) + assert(partitions(1).files(0).start == 0) + assert(partitions(1).files(0).length == 2) + assert(partitions(1).files(1).start == 0) + assert(partitions(1).files(1).length == 1) + + // Third partition reads (file4, file5) + assert(partitions(2).files(0).start == 0) + assert(partitions(2).files(0).length == 1) + assert(partitions(2).files(1).start == 0) + assert(partitions(2).files(1).length == 1) + + // Final partition reads (file6) + assert(partitions(3).files(0).start == 0) + assert(partitions(3).files(0).length == 1) + } + + checkPartitionSchema(StructType(Nil)) + checkDataSchema(StructType(Nil).add("c1", IntegerType)) + } + } + + test("partitioned table") { + val table = + createTable( + files = Seq( + "p1=1/file1" -> 10, + "p1=2/file2" -> 10)) + + // Only one file should be read. + checkScan(table.where("p1 = 1")) { partitions => + assert(partitions.size == 1, "when checking partitions") + assert(partitions.head.files.size == 1, "when files in partition 1") + } + // We don't need to reevaluate filters that are only on partitions. + checkDataFilters(Set.empty) + + // Only one file should be read. + checkScan(table.where("p1 = 1 AND c1 = 1 AND (p1 + c1) = 2")) { partitions => + assert(partitions.size == 1, "when checking partitions") + assert(partitions.head.files.size == 1, "when checking files in partition 1") + assert(partitions.head.files.head.partitionValues.getInt(0) == 1, + "when checking partition values") + } + // Only the filters that do not contain the partition column should be pushed down + checkDataFilters(Set(IsNotNull("c1"), EqualTo("c1", 1))) + } + + test("partitioned table - case insensitive") { + withSQLConf("spark.sql.caseSensitive" -> "false") { + val table = + createTable( + files = Seq( + "p1=1/file1" -> 10, + "p1=2/file2" -> 10)) + + // Only one file should be read. + checkScan(table.where("P1 = 1")) { partitions => + assert(partitions.size == 1, "when checking partitions") + assert(partitions.head.files.size == 1, "when files in partition 1") + } + // We don't need to reevaluate filters that are only on partitions. + checkDataFilters(Set.empty) + + // Only one file should be read. + checkScan(table.where("P1 = 1 AND C1 = 1 AND (P1 + C1) = 2")) { partitions => + assert(partitions.size == 1, "when checking partitions") + assert(partitions.head.files.size == 1, "when checking files in partition 1") + assert(partitions.head.files.head.partitionValues.getInt(0) == 1, + "when checking partition values") + } + // Only the filters that do not contain the partition column should be pushed down + checkDataFilters(Set(IsNotNull("c1"), EqualTo("c1", 1))) + } + } + + test("partitioned table - after scan filters") { + val table = + createTable( + files = Seq( + "p1=1/file1" -> 10, + "p1=2/file2" -> 10)) + + val df1 = table.where("p1 = 1 AND (p1 + c1) = 2 AND c1 = 1") + // Filter on data only are advisory so we have to reevaluate. + assert(getPhysicalFilters(df1) contains resolve(df1, "c1 = 1")) + // Don't reevaluate partition only filters. + assert(!(getPhysicalFilters(df1) contains resolve(df1, "p1 = 1"))) + + val df2 = table.where("(p1 + c2) = 2 AND c1 = 1") + // Filter on data only are advisory so we have to reevaluate. + assert(getPhysicalFilters(df2) contains resolve(df2, "c1 = 1")) + // Need to evaluate filters that are not pushed down. + assert(getPhysicalFilters(df2) contains resolve(df2, "(p1 + c2) = 2")) + } + + test("bucketed table") { + val table = + createTable( + files = Seq( + "p1=1/file1_0000" -> 1, + "p1=1/file2_0000" -> 1, + "p1=1/file3_0002" -> 1, + "p1=2/file4_0002" -> 1, + "p1=2/file5_0000" -> 1, + "p1=2/file6_0000" -> 1, + "p1=2/file7_0000" -> 1), + buckets = 3) + + // No partition pruning + checkScan(table) { partitions => + assert(partitions.size == 3) + assert(partitions(0).files.size == 5) + assert(partitions(1).files.size == 0) + assert(partitions(2).files.size == 2) + } + + // With partition pruning + checkScan(table.where("p1=2")) { partitions => + assert(partitions.size == 3) + assert(partitions(0).files.size == 3) + assert(partitions(1).files.size == 0) + assert(partitions(2).files.size == 1) + } + } + + test("Locality support for FileScanRDD") { + val partition = FilePartition(0, Seq( + PartitionedFile(InternalRow.empty, "fakePath0", 0, 10, Array("host0", "host1")), + PartitionedFile(InternalRow.empty, "fakePath0", 10, 20, Array("host1", "host2")), + PartitionedFile(InternalRow.empty, "fakePath1", 0, 5, Array("host3")), + PartitionedFile(InternalRow.empty, "fakePath2", 0, 5, Array("host4")) + )) + + val fakeRDD = new FileScanRDD( + spark, + (file: PartitionedFile) => Iterator.empty, + Seq(partition) + ) + + assertResult(Set("host0", "host1", "host2")) { + fakeRDD.preferredLocations(partition).toSet + } + } + + test("Locality support for FileScanRDD - one file per partition") { + withSQLConf( + SQLConf.FILES_MAX_PARTITION_BYTES.key -> "10", + "fs.file.impl" -> classOf[LocalityTestFileSystem].getName, + "fs.file.impl.disable.cache" -> "true") { + val table = + createTable(files = Seq( + "file1" -> 10, + "file2" -> 10 + )) + + checkScan(table) { partitions => + val Seq(p1, p2) = partitions + assert(p1.files.length == 1) + assert(p1.files.flatMap(_.locations).length == 1) + assert(p2.files.length == 1) + assert(p2.files.flatMap(_.locations).length == 1) + + val fileScanRDD = getFileScanRDD(table) + assert(partitions.flatMap(fileScanRDD.preferredLocations).length == 2) + } + } + } + + test("Locality support for FileScanRDD - large file") { + withSQLConf( + SQLConf.FILES_MAX_PARTITION_BYTES.key -> "10", + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> "0", + "fs.file.impl" -> classOf[LocalityTestFileSystem].getName, + "fs.file.impl.disable.cache" -> "true") { + val table = + createTable(files = Seq( + "file1" -> 15, + "file2" -> 5 + )) + + checkScan(table) { partitions => + val Seq(p1, p2) = partitions + assert(p1.files.length == 1) + assert(p1.files.flatMap(_.locations).length == 1) + assert(p2.files.length == 2) + assert(p2.files.flatMap(_.locations).length == 2) + + val fileScanRDD = getFileScanRDD(table) + assert(partitions.flatMap(fileScanRDD.preferredLocations).length == 3) + } + } + } + + test("SPARK-15654 do not split non-splittable files") { + // Check if a non-splittable file is not assigned into partitions + Seq("gz", "snappy", "lz4").foreach { suffix => + val table = createTable( + files = Seq(s"file1.${suffix}" -> 3, s"file2.${suffix}" -> 1, s"file3.${suffix}" -> 1) + ) + withSQLConf( + SQLConf.FILES_MAX_PARTITION_BYTES.key -> "2", + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> "0") { + checkScan(table.select('c1)) { partitions => + assert(partitions.size == 2) + assert(partitions(0).files.size == 1) + assert(partitions(1).files.size == 2) + } + } + } + + // Check if a splittable compressed file is assigned into multiple partitions + Seq("bz2").foreach { suffix => + val table = createTable( + files = Seq(s"file1.${suffix}" -> 3, s"file2.${suffix}" -> 1, s"file3.${suffix}" -> 1) + ) + withSQLConf( + SQLConf.FILES_MAX_PARTITION_BYTES.key -> "2", + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> "0") { + checkScan(table.select('c1)) { partitions => + assert(partitions.size == 3) + assert(partitions(0).files.size == 1) + assert(partitions(1).files.size == 2) + assert(partitions(2).files.size == 1) + } + } + } + } + + test("SPARK-14959: Do not call getFileBlockLocations on directories") { + // Setting PARALLEL_PARTITION_DISCOVERY_THRESHOLD to 2. So we will first + // list file statues at driver side and then for the level of p2, we will list + // file statues in parallel. + withSQLConf( + "fs.file.impl" -> classOf[MockDistributedFileSystem].getName, + SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD.key -> "2") { + withTempPath { path => + val tempDir = path.getCanonicalPath + + Seq("p1=1/p2=2/p3=3/file1", "p1=1/p2=3/p3=3/file1").foreach { fileName => + val file = new File(tempDir, fileName) + assert(file.getParentFile.exists() || file.getParentFile.mkdirs()) + util.stringToFile(file, fileName) + } + + val fileCatalog = new InMemoryFileIndex( + sparkSession = spark, + rootPathsSpecified = Seq(new Path(tempDir)), + parameters = Map.empty[String, String], + partitionSchema = None) + // This should not fail. + fileCatalog.listLeafFiles(Seq(new Path(tempDir))) + + // Also have an integration test. + checkAnswer( + spark.read.text(tempDir).select("p1", "p2", "p3", "value"), + Row(1, 2, 3, "p1=1/p2=2/p3=3/file1") :: Row(1, 3, 3, "p1=1/p2=3/p3=3/file1") :: Nil) + } + } + } + + test("[SPARK-16818] partition pruned file scans implement sameResult correctly") { + withTempPath { path => + val tempDir = path.getCanonicalPath + spark.range(100) + .selectExpr("id", "id as b") + .write + .partitionBy("id") + .parquet(tempDir) + val df = spark.read.parquet(tempDir) + def getPlan(df: DataFrame): SparkPlan = { + df.queryExecution.executedPlan + } + assert(getPlan(df.where("id = 2")).sameResult(getPlan(df.where("id = 2")))) + assert(!getPlan(df.where("id = 2")).sameResult(getPlan(df.where("id = 3")))) + } + } + + test("[SPARK-16818] exchange reuse respects differences in partition pruning") { + spark.conf.set("spark.sql.exchange.reuse", true) + withTempPath { path => + val tempDir = path.getCanonicalPath + spark.range(10) + .selectExpr("id % 2 as a", "id % 3 as b", "id as c") + .write + .partitionBy("a") + .parquet(tempDir) + val df = spark.read.parquet(tempDir) + val df1 = df.where("a = 0").groupBy("b").agg("c" -> "sum") + val df2 = df.where("a = 1").groupBy("b").agg("c" -> "sum") + checkAnswer(df1.join(df2, "b"), Row(0, 6, 12) :: Row(1, 4, 8) :: Row(2, 10, 5) :: Nil) + } + } + + test("spark.files.ignoreCorruptFiles should work in SQL") { + val inputFile = File.createTempFile("input-", ".gz") + try { + // Create a corrupt gzip file + val byteOutput = new ByteArrayOutputStream() + val gzip = new GZIPOutputStream(byteOutput) + try { + gzip.write(Array[Byte](1, 2, 3, 4)) + } finally { + gzip.close() + } + val bytes = byteOutput.toByteArray + val o = new FileOutputStream(inputFile) + try { + // It's corrupt since we only write half of bytes into the file. + o.write(bytes.take(bytes.length / 2)) + } finally { + o.close() + } + withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") { + val e = intercept[SparkException] { + spark.read.text(inputFile.toURI.toString).collect() + } + assert(e.getCause.isInstanceOf[EOFException]) + assert(e.getCause.getMessage === "Unexpected end of input stream") + } + withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") { + assert(spark.read.text(inputFile.toURI.toString).collect().isEmpty) + } + } finally { + inputFile.delete() + } + } + + test("[SPARK-18753] keep pushed-down null literal as a filter in Spark-side post-filter") { + val ds = Seq(Tuple1(Some(true)), Tuple1(None), Tuple1(Some(false))).toDS() + withTempPath { p => + val path = p.getAbsolutePath + ds.write.parquet(path) + val readBack = spark.read.parquet(path).filter($"_1" === "true") + val filtered = ds.filter($"_1" === "true").toDF() + checkAnswer(readBack, filtered) + } + } + + // Helpers for checking the arguments passed to the FileFormat. + + protected val checkPartitionSchema = + checkArgument("partition schema", _.partitionSchema, _: StructType) + protected val checkDataSchema = + checkArgument("data schema", _.dataSchema, _: StructType) + protected val checkDataFilters = + checkArgument("data filters", _.filters.toSet, _: Set[Filter]) + + /** Helper for building checks on the arguments passed to the reader. */ + protected def checkArgument[T](name: String, arg: LastArguments.type => T, expected: T): Unit = { + if (arg(LastArguments) != expected) { + fail( + s""" + |Wrong $name + |expected: $expected + |actual: ${arg(LastArguments)} + """.stripMargin) + } + } + + /** Returns a resolved expression for `str` in the context of `df`. */ + def resolve(df: DataFrame, str: String): Expression = { + df.select(expr(str)).queryExecution.analyzed.expressions.head.children.head + } + + /** Returns a set with all the filters present in the physical plan. */ + def getPhysicalFilters(df: DataFrame): ExpressionSet = { + ExpressionSet( + df.queryExecution.executedPlan.collect { + case execution.FilterExec(f, _) => splitConjunctivePredicates(f) + }.flatten) + } + + /** Plans the query and calls the provided validation function with the planned partitioning. */ + def checkScan(df: DataFrame)(func: Seq[FilePartition] => Unit): Unit = { + func(getFileScanRDD(df).filePartitions) + } + + /** + * Constructs a new table given a list of file names and sizes expressed in bytes. The table + * is written out in a temporary directory and any nested directories in the files names + * are automatically created. + * + * When `buckets` is > 0 the returned [[DataFrame]] will have metadata specifying that number of + * buckets. However, it is the responsibility of the caller to assign files to each bucket + * by appending the bucket id to the file names. + */ + def createTable( + files: Seq[(String, Int)], + buckets: Int = 0): DataFrame = { + val tempDir = Utils.createTempDir() + files.foreach { + case (name, size) => + val file = new File(tempDir, name) + assert(file.getParentFile.exists() || file.getParentFile.mkdirs()) + util.stringToFile(file, "*" * size) + } + + val df = spark.read + .format(classOf[TestFileFormat].getName) + .load(tempDir.getCanonicalPath) + + if (buckets > 0) { + val bucketed = df.queryExecution.analyzed transform { + case l @ LogicalRelation(r: HadoopFsRelation, _, _, _) => + l.copy(relation = + r.copy(bucketSpec = + Some(BucketSpec(numBuckets = buckets, "c1" :: Nil, Nil)))(r.sparkSession)) + } + Dataset.ofRows(spark, bucketed) + } else { + df + } + } + + def getFileScanRDD(df: DataFrame): FileScanRDD = { + df.queryExecution.executedPlan.collect { + case scan: DataSourceScanExec if scan.inputRDDs().head.isInstanceOf[FileScanRDD] => + scan.inputRDDs().head.asInstanceOf[FileScanRDD] + }.headOption.getOrElse { + fail(s"No FileScan in query\n${df.queryExecution}") + } + } +} + +/** Holds the last arguments passed to [[TestFileFormat]]. */ +object LastArguments { + var partitionSchema: StructType = _ + var dataSchema: StructType = _ + var filters: Seq[Filter] = _ + var options: Map[String, String] = _ +} + +/** A test [[FileFormat]] that records the arguments passed to buildReader, and returns nothing. */ +class TestFileFormat extends TextBasedFileFormat { + + override def toString: String = "TestFileFormat" + + /** + * When possible, this method should return the schema of the given `files`. When the format + * does not support inference, or no valid files are given should return None. In these cases + * Spark will require that user specify the schema manually. + */ + override def inferSchema( + sparkSession: SparkSession, + options: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = + Some( + StructType(Nil) + .add("c1", IntegerType) + .add("c2", IntegerType)) + + /** + * Prepares a write job and returns an [[OutputWriterFactory]]. Client side job preparation can + * be put here. For example, user defined output committer can be configured here + * by setting the output committer class in the conf of spark.sql.sources.outputCommitterClass. + */ + override def prepareWrite( + sparkSession: SparkSession, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + throw new NotImplementedError("JUST FOR TESTING") + } + + override def buildReader( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { + + // Record the arguments so they can be checked in the test case. + LastArguments.partitionSchema = partitionSchema + LastArguments.dataSchema = requiredSchema + LastArguments.filters = filters + LastArguments.options = options + + (file: PartitionedFile) => { Iterator.empty } + } +} + + +class LocalityTestFileSystem extends RawLocalFileSystem { + private val invocations = new AtomicInteger(0) + + override def getFileBlockLocations( + file: FileStatus, start: Long, len: Long): Array[BlockLocation] = { + require(!file.isDirectory, "The file path can not be a directory.") + val count = invocations.getAndAdd(1) + Array(new BlockLocation(Array(s"host$count:50010"), Array(s"host$count"), 0, len)) + } +} + +// This file system is for SPARK-14959 (DistributedFileSystem will throw an exception +// if we call getFileBlockLocations on a dir). +class MockDistributedFileSystem extends RawLocalFileSystem { + + override def getFileBlockLocations( + file: FileStatus, start: Long, len: Long): Array[BlockLocation] = { + require(!file.isDirectory, "The file path can not be a directory.") + super.getFileBlockLocations(file, start, len) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala new file mode 100644 index 000000000000..caf03885e387 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.io.{File, FilenameFilter} + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.SharedSQLContext + +class HadoopFsRelationSuite extends QueryTest with SharedSQLContext { + + test("sizeInBytes should be the total size of all files") { + withTempDir{ dir => + dir.delete() + spark.range(1000).write.parquet(dir.toString) + // ignore hidden files + val allFiles = dir.listFiles(new FilenameFilter { + override def accept(dir: File, name: String): Boolean = { + !name.startsWith(".") && !name.startsWith("_") + } + }) + val totalSize = allFiles.map(_.length()).sum + val df = spark.read.parquet(dir.toString) + assert(df.queryExecution.logical.stats.sizeInBytes === BigInt(totalSize)) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/RowDataSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/RowDataSourceStrategySuite.scala new file mode 100644 index 000000000000..e8bf21a2a9db --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/RowDataSourceStrategySuite.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.sql.DriverManager +import java.util.Properties + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + +class RowDataSourceStrategySuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext { + import testImplicits._ + + val url = "jdbc:h2:mem:testdb0" + val urlWithUserAndPass = "jdbc:h2:mem:testdb0;user=testUser;password=testPass" + var conn: java.sql.Connection = null + + before { + Utils.classForName("org.h2.Driver") + // Extra properties that will be specified for our database. We need these to test + // usage of parameters from OPTIONS clause in queries. + val properties = new Properties() + properties.setProperty("user", "testUser") + properties.setProperty("password", "testPass") + properties.setProperty("rowId", "false") + + conn = DriverManager.getConnection(url, properties) + conn.prepareStatement("create schema test").executeUpdate() + conn.prepareStatement("create table test.inttypes (a INT, b INT, c INT)").executeUpdate() + conn.prepareStatement("insert into test.inttypes values (1, 2, 3)").executeUpdate() + conn.commit() + sql( + s""" + |CREATE OR REPLACE TEMPORARY VIEW inttypes + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.INTTYPES', user 'testUser', password 'testPass') + """.stripMargin.replaceAll("\n", " ")) + } + + after { + conn.close() + } + + test("SPARK-17673: Exchange reuse respects differences in output schema") { + val df = sql("SELECT * FROM inttypes") + val df1 = df.groupBy("a").agg("b" -> "min") + val df2 = df.groupBy("a").agg("c" -> "min") + val res = df1.union(df2) + assert(res.distinct().count() == 2) // would be 1 if the exchange was incorrectly reused + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala new file mode 100644 index 000000000000..661742087112 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.csv + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types._ + +class CSVInferSchemaSuite extends SparkFunSuite { + + test("String fields types are inferred correctly from null types") { + val options = new CSVOptions(Map.empty[String, String], "GMT") + assert(CSVInferSchema.inferField(NullType, "", options) == NullType) + assert(CSVInferSchema.inferField(NullType, null, options) == NullType) + assert(CSVInferSchema.inferField(NullType, "100000000000", options) == LongType) + assert(CSVInferSchema.inferField(NullType, "60", options) == IntegerType) + assert(CSVInferSchema.inferField(NullType, "3.5", options) == DoubleType) + assert(CSVInferSchema.inferField(NullType, "test", options) == StringType) + assert(CSVInferSchema.inferField(NullType, "2015-08-20 15:57:00", options) == TimestampType) + assert(CSVInferSchema.inferField(NullType, "True", options) == BooleanType) + assert(CSVInferSchema.inferField(NullType, "FAlSE", options) == BooleanType) + + val textValueOne = Long.MaxValue.toString + "0" + val decimalValueOne = new java.math.BigDecimal(textValueOne) + val expectedTypeOne = DecimalType(decimalValueOne.precision, decimalValueOne.scale) + assert(CSVInferSchema.inferField(NullType, textValueOne, options) == expectedTypeOne) + } + + test("String fields types are inferred correctly from other types") { + val options = new CSVOptions(Map.empty[String, String], "GMT") + assert(CSVInferSchema.inferField(LongType, "1.0", options) == DoubleType) + assert(CSVInferSchema.inferField(LongType, "test", options) == StringType) + assert(CSVInferSchema.inferField(IntegerType, "1.0", options) == DoubleType) + assert(CSVInferSchema.inferField(DoubleType, null, options) == DoubleType) + assert(CSVInferSchema.inferField(DoubleType, "test", options) == StringType) + assert(CSVInferSchema.inferField(LongType, "2015-08-20 14:57:00", options) == TimestampType) + assert(CSVInferSchema.inferField(DoubleType, "2015-08-20 15:57:00", options) == TimestampType) + assert(CSVInferSchema.inferField(LongType, "True", options) == BooleanType) + assert(CSVInferSchema.inferField(IntegerType, "FALSE", options) == BooleanType) + assert(CSVInferSchema.inferField(TimestampType, "FALSE", options) == BooleanType) + + val textValueOne = Long.MaxValue.toString + "0" + val decimalValueOne = new java.math.BigDecimal(textValueOne) + val expectedTypeOne = DecimalType(decimalValueOne.precision, decimalValueOne.scale) + assert(CSVInferSchema.inferField(IntegerType, textValueOne, options) == expectedTypeOne) + } + + test("Timestamp field types are inferred correctly via custom data format") { + var options = new CSVOptions(Map("timestampFormat" -> "yyyy-mm"), "GMT") + assert(CSVInferSchema.inferField(TimestampType, "2015-08", options) == TimestampType) + options = new CSVOptions(Map("timestampFormat" -> "yyyy"), "GMT") + assert(CSVInferSchema.inferField(TimestampType, "2015", options) == TimestampType) + } + + test("Timestamp field types are inferred correctly from other types") { + val options = new CSVOptions(Map.empty[String, String], "GMT") + assert(CSVInferSchema.inferField(IntegerType, "2015-08-20 14", options) == StringType) + assert(CSVInferSchema.inferField(DoubleType, "2015-08-20 14:10", options) == StringType) + assert(CSVInferSchema.inferField(LongType, "2015-08 14:49:00", options) == StringType) + } + + test("Boolean fields types are inferred correctly from other types") { + val options = new CSVOptions(Map.empty[String, String], "GMT") + assert(CSVInferSchema.inferField(LongType, "Fale", options) == StringType) + assert(CSVInferSchema.inferField(DoubleType, "TRUEe", options) == StringType) + } + + test("Type arrays are merged to highest common type") { + assert( + CSVInferSchema.mergeRowTypes(Array(StringType), + Array(DoubleType)).deep == Array(StringType).deep) + assert( + CSVInferSchema.mergeRowTypes(Array(IntegerType), + Array(LongType)).deep == Array(LongType).deep) + assert( + CSVInferSchema.mergeRowTypes(Array(DoubleType), + Array(LongType)).deep == Array(DoubleType).deep) + } + + test("Null fields are handled properly when a nullValue is specified") { + var options = new CSVOptions(Map("nullValue" -> "null"), "GMT") + assert(CSVInferSchema.inferField(NullType, "null", options) == NullType) + assert(CSVInferSchema.inferField(StringType, "null", options) == StringType) + assert(CSVInferSchema.inferField(LongType, "null", options) == LongType) + + options = new CSVOptions(Map("nullValue" -> "\\N"), "GMT") + assert(CSVInferSchema.inferField(IntegerType, "\\N", options) == IntegerType) + assert(CSVInferSchema.inferField(DoubleType, "\\N", options) == DoubleType) + assert(CSVInferSchema.inferField(TimestampType, "\\N", options) == TimestampType) + assert(CSVInferSchema.inferField(BooleanType, "\\N", options) == BooleanType) + assert(CSVInferSchema.inferField(DecimalType(1, 1), "\\N", options) == DecimalType(1, 1)) + } + + test("Merging Nulltypes should yield Nulltype.") { + val mergedNullTypes = CSVInferSchema.mergeRowTypes(Array(NullType), Array(NullType)) + assert(mergedNullTypes.deep == Array(NullType).deep) + } + + test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") { + val options = new CSVOptions(Map("TiMeStampFormat" -> "yyyy-mm"), "GMT") + assert(CSVInferSchema.inferField(TimestampType, "2015-08", options) == TimestampType) + } + + test("SPARK-18877: `inferField` on DecimalType should find a common type with `typeSoFar`") { + val options = new CSVOptions(Map.empty[String, String], "GMT") + + // 9.03E+12 is Decimal(3, -10) and 1.19E+11 is Decimal(3, -9). + assert(CSVInferSchema.inferField(DecimalType(3, -10), "1.19E+11", options) == + DecimalType(4, -9)) + + // BigDecimal("12345678901234567890.01234567890123456789") is precision 40 and scale 20. + val value = "12345678901234567890.01234567890123456789" + assert(CSVInferSchema.inferField(DecimalType(3, -10), value, options) == DoubleType) + + // Seq(s"${Long.MaxValue}1", "2015-12-01 00:00:00") should be StringType + assert(CSVInferSchema.inferField(NullType, s"${Long.MaxValue}1", options) == DecimalType(20, 0)) + assert(CSVInferSchema.inferField(DecimalType(20, 0), "2015-12-01 00:00:00", options) + == StringType) + } + + test("DoubleType should be infered when user defined nan/inf are provided") { + val options = new CSVOptions(Map("nanValue" -> "nan", "negativeInf" -> "-inf", + "positiveInf" -> "inf"), "GMT") + assert(CSVInferSchema.inferField(NullType, "nan", options) == DoubleType) + assert(CSVInferSchema.inferField(NullType, "inf", options) == DoubleType) + assert(CSVInferSchema.inferField(NullType, "-inf", options) == DoubleType) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala new file mode 100644 index 000000000000..e439699605ab --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -0,0 +1,1248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.csv + +import java.io.File +import java.nio.charset.UnsupportedCharsetException +import java.sql.{Date, Timestamp} +import java.text.SimpleDateFormat +import java.util.Locale + +import org.apache.commons.lang3.time.FastDateFormat +import org.apache.hadoop.io.SequenceFile.CompressionType +import org.apache.hadoop.io.compress.GzipCodec + +import org.apache.spark.SparkException +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, UDT} +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.functions.{col, regexp_replace} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.types._ + +class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { + import testImplicits._ + + private val carsFile = "test-data/cars.csv" + private val carsMalformedFile = "test-data/cars-malformed.csv" + private val carsFile8859 = "test-data/cars_iso-8859-1.csv" + private val carsTsvFile = "test-data/cars.tsv" + private val carsAltFile = "test-data/cars-alternative.csv" + private val carsUnbalancedQuotesFile = "test-data/cars-unbalanced-quotes.csv" + private val carsNullFile = "test-data/cars-null.csv" + private val carsBlankColName = "test-data/cars-blank-column-name.csv" + private val emptyFile = "test-data/empty.csv" + private val commentsFile = "test-data/comments.csv" + private val disableCommentsFile = "test-data/disable_comments.csv" + private val boolFile = "test-data/bool.csv" + private val decimalFile = "test-data/decimal.csv" + private val simpleSparseFile = "test-data/simple_sparse.csv" + private val numbersFile = "test-data/numbers.csv" + private val datesFile = "test-data/dates.csv" + private val unescapedQuotesFile = "test-data/unescaped-quotes.csv" + private val valueMalformedFile = "test-data/value-malformed.csv" + + private def testFile(fileName: String): String = { + Thread.currentThread().getContextClassLoader.getResource(fileName).toString + } + + /** Verifies data and schema. */ + private def verifyCars( + df: DataFrame, + withHeader: Boolean, + numCars: Int = 3, + numFields: Int = 5, + checkHeader: Boolean = true, + checkValues: Boolean = true, + checkTypes: Boolean = false): Unit = { + + val numColumns = numFields + val numRows = if (withHeader) numCars else numCars + 1 + // schema + assert(df.schema.fieldNames.length === numColumns) + assert(df.count === numRows) + + if (checkHeader) { + if (withHeader) { + assert(df.schema.fieldNames === Array("year", "make", "model", "comment", "blank")) + } else { + assert(df.schema.fieldNames === Array("_c0", "_c1", "_c2", "_c3", "_c4")) + } + } + + if (checkValues) { + val yearValues = List("2012", "1997", "2015") + val actualYears = if (!withHeader) "year" :: yearValues else yearValues + val years = if (withHeader) df.select("year").collect() else df.select("_c0").collect() + + years.zipWithIndex.foreach { case (year, index) => + if (checkTypes) { + assert(year === Row(actualYears(index).toInt)) + } else { + assert(year === Row(actualYears(index))) + } + } + } + } + + test("simple csv test") { + val cars = spark + .read + .format("csv") + .option("header", "false") + .load(testFile(carsFile)) + + verifyCars(cars, withHeader = false, checkTypes = false) + } + + test("simple csv test with calling another function to load") { + val cars = spark + .read + .option("header", "false") + .csv(testFile(carsFile)) + + verifyCars(cars, withHeader = false, checkTypes = false) + } + + test("simple csv test with type inference") { + val cars = spark + .read + .format("csv") + .option("header", "true") + .option("inferSchema", "true") + .load(testFile(carsFile)) + + verifyCars(cars, withHeader = true, checkTypes = true) + } + + test("simple csv test with string dataset") { + val csvDataset = spark.read.text(testFile(carsFile)).as[String] + val cars = spark.read + .option("header", "true") + .option("inferSchema", "true") + .csv(csvDataset) + + verifyCars(cars, withHeader = true, checkTypes = true) + + val carsWithoutHeader = spark.read + .option("header", "false") + .csv(csvDataset) + + verifyCars(carsWithoutHeader, withHeader = false, checkTypes = false) + } + + test("test inferring booleans") { + val result = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "true") + .load(testFile(boolFile)) + + val expectedSchema = StructType(List( + StructField("bool", BooleanType, nullable = true))) + assert(result.schema === expectedSchema) + } + + test("test inferring decimals") { + val result = spark.read + .format("csv") + .option("comment", "~") + .option("header", "true") + .option("inferSchema", "true") + .load(testFile(decimalFile)) + val expectedSchema = StructType(List( + StructField("decimal", DecimalType(20, 0), nullable = true), + StructField("long", LongType, nullable = true), + StructField("double", DoubleType, nullable = true))) + assert(result.schema === expectedSchema) + } + + test("test with alternative delimiter and quote") { + val cars = spark.read + .format("csv") + .options(Map("quote" -> "\'", "delimiter" -> "|", "header" -> "true")) + .load(testFile(carsAltFile)) + + verifyCars(cars, withHeader = true) + } + + test("parse unescaped quotes with maxCharsPerColumn") { + val rows = spark.read + .format("csv") + .option("maxCharsPerColumn", "4") + .load(testFile(unescapedQuotesFile)) + + val expectedRows = Seq(Row("\"a\"b", "ccc", "ddd"), Row("ab", "cc\"c", "ddd\"")) + + checkAnswer(rows, expectedRows) + } + + test("bad encoding name") { + val exception = intercept[UnsupportedCharsetException] { + spark + .read + .format("csv") + .option("charset", "1-9588-osi") + .load(testFile(carsFile8859)) + } + + assert(exception.getMessage.contains("1-9588-osi")) + } + + test("test different encoding") { + withView("carsTable") { + // scalastyle:off + spark.sql( + s""" + |CREATE TEMPORARY VIEW carsTable USING csv + |OPTIONS (path "${testFile(carsFile8859)}", header "true", + |charset "iso-8859-1", delimiter "þ") + """.stripMargin.replaceAll("\n", " ")) + // scalastyle:on + verifyCars(spark.table("carsTable"), withHeader = true) + } + } + + test("test aliases sep and encoding for delimiter and charset") { + // scalastyle:off + val cars = spark + .read + .format("csv") + .option("header", "true") + .option("encoding", "iso-8859-1") + .option("sep", "þ") + .load(testFile(carsFile8859)) + // scalastyle:on + + verifyCars(cars, withHeader = true) + } + + test("DDL test with tab separated file") { + withView("carsTable") { + spark.sql( + s""" + |CREATE TEMPORARY VIEW carsTable USING csv + |OPTIONS (path "${testFile(carsTsvFile)}", header "true", delimiter "\t") + """.stripMargin.replaceAll("\n", " ")) + + verifyCars(spark.table("carsTable"), numFields = 6, withHeader = true, checkHeader = false) + } + } + + test("DDL test parsing decimal type") { + withView("carsTable") { + spark.sql( + s""" + |CREATE TEMPORARY VIEW carsTable + |(yearMade double, makeName string, modelName string, priceTag decimal, + | comments string, grp string) + |USING csv + |OPTIONS (path "${testFile(carsTsvFile)}", header "true", delimiter "\t") + """.stripMargin.replaceAll("\n", " ")) + + assert( + spark.sql("SELECT makeName FROM carsTable where priceTag > 60000").collect().size === 1) + } + } + + test("test for DROPMALFORMED parsing mode") { + Seq(false, true).foreach { multiLine => + val cars = spark.read + .format("csv") + .option("multiLine", multiLine) + .options(Map("header" -> "true", "mode" -> "dropmalformed")) + .load(testFile(carsFile)) + + assert(cars.select("year").collect().size === 2) + } + } + + test("test for blank column names on read and select columns") { + val cars = spark.read + .format("csv") + .options(Map("header" -> "true", "inferSchema" -> "true")) + .load(testFile(carsBlankColName)) + + assert(cars.select("customer").collect().size == 2) + assert(cars.select("_c0").collect().size == 2) + assert(cars.select("_c1").collect().size == 2) + } + + test("test for FAILFAST parsing mode") { + Seq(false, true).foreach { multiLine => + val exception = intercept[SparkException] { + spark.read + .format("csv") + .option("multiLine", multiLine) + .options(Map("header" -> "true", "mode" -> "failfast")) + .load(testFile(carsFile)).collect() + } + + assert(exception.getMessage.contains("Malformed CSV record")) + } + } + + test("test for tokens more than the fields in the schema") { + val cars = spark + .read + .format("csv") + .option("header", "false") + .option("comment", "~") + .load(testFile(carsMalformedFile)) + + verifyCars(cars, withHeader = false, checkTypes = false) + } + + test("test with null quote character") { + val cars = spark.read + .format("csv") + .option("header", "true") + .option("quote", "") + .load(testFile(carsUnbalancedQuotesFile)) + + verifyCars(cars, withHeader = true, checkValues = false) + + } + + test("test with empty file and known schema") { + val result = spark.read + .format("csv") + .schema(StructType(List(StructField("column", StringType, false)))) + .load(testFile(emptyFile)) + + assert(result.collect.size === 0) + assert(result.schema.fieldNames.size === 1) + } + + test("DDL test with empty file") { + withView("carsTable") { + spark.sql( + s""" + |CREATE TEMPORARY VIEW carsTable + |(yearMade double, makeName string, modelName string, comments string, grp string) + |USING csv + |OPTIONS (path "${testFile(emptyFile)}", header "false") + """.stripMargin.replaceAll("\n", " ")) + + assert(spark.sql("SELECT count(*) FROM carsTable").collect().head(0) === 0) + } + } + + test("DDL test with schema") { + withView("carsTable") { + spark.sql( + s""" + |CREATE TEMPORARY VIEW carsTable + |(yearMade double, makeName string, modelName string, comments string, blank string) + |USING csv + |OPTIONS (path "${testFile(carsFile)}", header "true") + """.stripMargin.replaceAll("\n", " ")) + + val cars = spark.table("carsTable") + verifyCars(cars, withHeader = true, checkHeader = false, checkValues = false) + assert( + cars.schema.fieldNames === Array("yearMade", "makeName", "modelName", "comments", "blank")) + } + } + + test("save csv") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + val cars = spark.read + .format("csv") + .option("header", "true") + .load(testFile(carsFile)) + + cars.coalesce(1).write + .option("header", "true") + .csv(csvDir) + + val carsCopy = spark.read + .format("csv") + .option("header", "true") + .load(csvDir) + + verifyCars(carsCopy, withHeader = true) + } + } + + test("save csv with quote") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + val cars = spark.read + .format("csv") + .option("header", "true") + .load(testFile(carsFile)) + + cars.coalesce(1).write + .format("csv") + .option("header", "true") + .option("quote", "\"") + .save(csvDir) + + val carsCopy = spark.read + .format("csv") + .option("header", "true") + .option("quote", "\"") + .load(csvDir) + + verifyCars(carsCopy, withHeader = true) + } + } + + test("save csv with quoteAll enabled") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + + val data = Seq(("test \"quote\"", 123, "it \"works\"!", "\"very\" well")) + val df = spark.createDataFrame(data) + + // escapeQuotes should be true by default + df.coalesce(1).write + .format("csv") + .option("quote", "\"") + .option("escape", "\"") + .option("quoteAll", "true") + .save(csvDir) + + val results = spark.read + .format("text") + .load(csvDir) + .collect() + + val expected = "\"test \"\"quote\"\"\",\"123\",\"it \"\"works\"\"!\",\"\"\"very\"\" well\"" + + assert(results.toSeq.map(_.toSeq) === Seq(Seq(expected))) + } + } + + test("save csv with quote escaping enabled") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + + val data = Seq(("test \"quote\"", 123, "it \"works\"!", "\"very\" well")) + val df = spark.createDataFrame(data) + + // escapeQuotes should be true by default + df.coalesce(1).write + .format("csv") + .option("quote", "\"") + .option("escape", "\"") + .save(csvDir) + + val results = spark.read + .format("text") + .load(csvDir) + .collect() + + val expected = "\"test \"\"quote\"\"\",123,\"it \"\"works\"\"!\",\"\"\"very\"\" well\"" + + assert(results.toSeq.map(_.toSeq) === Seq(Seq(expected))) + } + } + + test("save csv with quote escaping disabled") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + + val data = Seq(("test \"quote\"", 123, "it \"works\"!", "\"very\" well")) + val df = spark.createDataFrame(data) + + // escapeQuotes should be true by default + df.coalesce(1).write + .format("csv") + .option("quote", "\"") + .option("escapeQuotes", "false") + .option("escape", "\"") + .save(csvDir) + + val results = spark.read + .format("text") + .load(csvDir) + .collect() + + val expected = "test \"quote\",123,it \"works\"!,\"\"\"very\"\" well\"" + + assert(results.toSeq.map(_.toSeq) === Seq(Seq(expected))) + } + } + + test("commented lines in CSV data") { + val results = spark.read + .format("csv") + .options(Map("comment" -> "~", "header" -> "false")) + .load(testFile(commentsFile)) + .collect() + + val expected = + Seq(Seq("1", "2", "3", "4", "5.01", "2015-08-20 15:57:00"), + Seq("6", "7", "8", "9", "0", "2015-08-21 16:58:01"), + Seq("1", "2", "3", "4", "5", "2015-08-23 18:00:42")) + + assert(results.toSeq.map(_.toSeq) === expected) + } + + test("inferring schema with commented lines in CSV data") { + val results = spark.read + .format("csv") + .options(Map("comment" -> "~", "header" -> "false", "inferSchema" -> "true")) + .load(testFile(commentsFile)) + .collect() + + val expected = + Seq(Seq(1, 2, 3, 4, 5.01D, Timestamp.valueOf("2015-08-20 15:57:00")), + Seq(6, 7, 8, 9, 0, Timestamp.valueOf("2015-08-21 16:58:01")), + Seq(1, 2, 3, 4, 5, Timestamp.valueOf("2015-08-23 18:00:42"))) + + assert(results.toSeq.map(_.toSeq) === expected) + } + + test("inferring timestamp types via custom date format") { + val options = Map( + "header" -> "true", + "inferSchema" -> "true", + "timestampFormat" -> "dd/MM/yyyy HH:mm") + val results = spark.read + .format("csv") + .options(options) + .load(testFile(datesFile)) + .select("date") + .collect() + + val dateFormat = new SimpleDateFormat("dd/MM/yyyy HH:mm", Locale.US) + val expected = + Seq(Seq(new Timestamp(dateFormat.parse("26/08/2015 18:00").getTime)), + Seq(new Timestamp(dateFormat.parse("27/10/2014 18:30").getTime)), + Seq(new Timestamp(dateFormat.parse("28/01/2016 20:00").getTime))) + assert(results.toSeq.map(_.toSeq) === expected) + } + + test("load date types via custom date format") { + val customSchema = new StructType(Array(StructField("date", DateType, true))) + val options = Map( + "header" -> "true", + "inferSchema" -> "false", + "dateFormat" -> "dd/MM/yyyy hh:mm") + val results = spark.read + .format("csv") + .options(options) + .schema(customSchema) + .load(testFile(datesFile)) + .select("date") + .collect() + + val dateFormat = new SimpleDateFormat("dd/MM/yyyy hh:mm", Locale.US) + val expected = Seq( + new Date(dateFormat.parse("26/08/2015 18:00").getTime), + new Date(dateFormat.parse("27/10/2014 18:30").getTime), + new Date(dateFormat.parse("28/01/2016 20:00").getTime)) + val dates = results.toSeq.map(_.toSeq.head) + expected.zip(dates).foreach { + case (expectedDate, date) => + // As it truncates the hours, minutes and etc., we only check + // if the dates (days, months and years) are the same via `toString()`. + assert(expectedDate.toString === date.toString) + } + } + + test("setting comment to null disables comment support") { + val results = spark.read + .format("csv") + .options(Map("comment" -> "", "header" -> "false")) + .load(testFile(disableCommentsFile)) + .collect() + + val expected = + Seq( + Seq("#1", "2", "3"), + Seq("4", "5", "6")) + + assert(results.toSeq.map(_.toSeq) === expected) + } + + test("nullable fields with user defined null value of \"null\"") { + + // year,make,model,comment,blank + val dataSchema = StructType(List( + StructField("year", IntegerType, nullable = true), + StructField("make", StringType, nullable = false), + StructField("model", StringType, nullable = false), + StructField("comment", StringType, nullable = true), + StructField("blank", StringType, nullable = true))) + val cars = spark.read + .format("csv") + .schema(dataSchema) + .options(Map("header" -> "true", "nullValue" -> "null")) + .load(testFile(carsNullFile)) + + verifyCars(cars, withHeader = true, checkValues = false) + val results = cars.collect() + assert(results(0).toSeq === Array(2012, "Tesla", "S", null, null)) + assert(results(2).toSeq === Array(null, "Chevy", "Volt", null, null)) + } + + test("save csv with compression codec option") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + val cars = spark.read + .format("csv") + .option("header", "true") + .load(testFile(carsFile)) + + cars.coalesce(1).write + .format("csv") + .option("header", "true") + .option("compression", "gZiP") + .save(csvDir) + + val compressedFiles = new File(csvDir).listFiles() + assert(compressedFiles.exists(_.getName.endsWith(".csv.gz"))) + + val carsCopy = spark.read + .format("csv") + .option("header", "true") + .load(csvDir) + + verifyCars(carsCopy, withHeader = true) + } + } + + test("SPARK-13543 Write the output as uncompressed via option()") { + val extraOptions = Map( + "mapreduce.output.fileoutputformat.compress" -> "true", + "mapreduce.output.fileoutputformat.compress.type" -> CompressionType.BLOCK.toString, + "mapreduce.map.output.compress" -> "true", + "mapreduce.map.output.compress.codec" -> classOf[GzipCodec].getName + ) + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + val cars = spark.read + .format("csv") + .option("header", "true") + .options(extraOptions) + .load(testFile(carsFile)) + + cars.coalesce(1).write + .format("csv") + .option("header", "true") + .option("compression", "none") + .options(extraOptions) + .save(csvDir) + + val compressedFiles = new File(csvDir).listFiles() + assert(compressedFiles.exists(!_.getName.endsWith(".csv.gz"))) + + val carsCopy = spark.read + .format("csv") + .option("header", "true") + .options(extraOptions) + .load(csvDir) + + verifyCars(carsCopy, withHeader = true) + } + } + + test("Schema inference correctly identifies the datatype when data is sparse.") { + val df = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "true") + .load(testFile(simpleSparseFile)) + + assert( + df.schema.fields.map(field => field.dataType).deep == + Array(IntegerType, IntegerType, IntegerType, IntegerType).deep) + } + + test("old csv data source name works") { + val cars = spark + .read + .format("com.databricks.spark.csv") + .option("header", "false") + .load(testFile(carsFile)) + + verifyCars(cars, withHeader = false, checkTypes = false) + } + + test("nulls, NaNs and Infinity values can be parsed") { + val numbers = spark + .read + .format("csv") + .schema(StructType(List( + StructField("int", IntegerType, true), + StructField("long", LongType, true), + StructField("float", FloatType, true), + StructField("double", DoubleType, true) + ))) + .options(Map( + "header" -> "true", + "mode" -> "DROPMALFORMED", + "nullValue" -> "--", + "nanValue" -> "NAN", + "negativeInf" -> "-INF", + "positiveInf" -> "INF")) + .load(testFile(numbersFile)) + + assert(numbers.count() == 8) + } + + test("error handling for unsupported data types.") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + var msg = intercept[UnsupportedOperationException] { + Seq((1, "Tesla")).toDF("a", "b").selectExpr("struct(a, b)").write.csv(csvDir) + }.getMessage + assert(msg.contains("CSV data source does not support struct data type")) + + msg = intercept[UnsupportedOperationException] { + Seq((1, Map("Tesla" -> 3))).toDF("id", "cars").write.csv(csvDir) + }.getMessage + assert(msg.contains("CSV data source does not support map data type")) + + msg = intercept[UnsupportedOperationException] { + Seq((1, Array("Tesla", "Chevy", "Ford"))).toDF("id", "brands").write.csv(csvDir) + }.getMessage + assert(msg.contains("CSV data source does not support array data type")) + + msg = intercept[UnsupportedOperationException] { + Seq((1, new UDT.MyDenseVector(Array(0.25, 2.25, 4.25)))).toDF("id", "vectors") + .write.csv(csvDir) + }.getMessage + assert(msg.contains("CSV data source does not support array data type")) + + msg = intercept[UnsupportedOperationException] { + val schema = StructType(StructField("a", new UDT.MyDenseVectorUDT(), true) :: Nil) + spark.range(1).write.csv(csvDir) + spark.read.schema(schema).csv(csvDir).collect() + }.getMessage + assert(msg.contains("CSV data source does not support array data type.")) + } + } + + test("SPARK-15585 turn off quotations") { + val cars = spark.read + .format("csv") + .option("header", "true") + .option("quote", "") + .load(testFile(carsUnbalancedQuotesFile)) + + verifyCars(cars, withHeader = true, checkValues = false) + } + + test("Write timestamps correctly in ISO8601 format by default") { + withTempDir { dir => + val iso8601timestampsPath = s"${dir.getCanonicalPath}/iso8601timestamps.csv" + val timestamps = spark.read + .format("csv") + .option("inferSchema", "true") + .option("header", "true") + .option("timestampFormat", "dd/MM/yyyy HH:mm") + .load(testFile(datesFile)) + timestamps.write + .format("csv") + .option("header", "true") + .save(iso8601timestampsPath) + + // This will load back the timestamps as string. + val stringSchema = StructType(StructField("date", StringType, true) :: Nil) + val iso8601Timestamps = spark.read + .format("csv") + .schema(stringSchema) + .option("header", "true") + .load(iso8601timestampsPath) + + val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX", Locale.US) + val expectedTimestamps = timestamps.collect().map { r => + // This should be ISO8601 formatted string. + Row(iso8501.format(r.toSeq.head)) + } + + checkAnswer(iso8601Timestamps, expectedTimestamps) + } + } + + test("Write dates correctly in ISO8601 format by default") { + withTempDir { dir => + val customSchema = new StructType(Array(StructField("date", DateType, true))) + val iso8601datesPath = s"${dir.getCanonicalPath}/iso8601dates.csv" + val dates = spark.read + .format("csv") + .schema(customSchema) + .option("header", "true") + .option("inferSchema", "false") + .option("dateFormat", "dd/MM/yyyy HH:mm") + .load(testFile(datesFile)) + dates.write + .format("csv") + .option("header", "true") + .save(iso8601datesPath) + + // This will load back the dates as string. + val stringSchema = StructType(StructField("date", StringType, true) :: Nil) + val iso8601dates = spark.read + .format("csv") + .schema(stringSchema) + .option("header", "true") + .load(iso8601datesPath) + + val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd", Locale.US) + val expectedDates = dates.collect().map { r => + // This should be ISO8601 formatted string. + Row(iso8501.format(r.toSeq.head)) + } + + checkAnswer(iso8601dates, expectedDates) + } + } + + test("Roundtrip in reading and writing timestamps") { + withTempDir { dir => + val iso8601timestampsPath = s"${dir.getCanonicalPath}/iso8601timestamps.csv" + val timestamps = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "true") + .load(testFile(datesFile)) + + timestamps.write + .format("csv") + .option("header", "true") + .save(iso8601timestampsPath) + + val iso8601timestamps = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "true") + .load(iso8601timestampsPath) + + checkAnswer(iso8601timestamps, timestamps) + } + } + + test("Write dates correctly with dateFormat option") { + val customSchema = new StructType(Array(StructField("date", DateType, true))) + withTempDir { dir => + // With dateFormat option. + val datesWithFormatPath = s"${dir.getCanonicalPath}/datesWithFormat.csv" + val datesWithFormat = spark.read + .format("csv") + .schema(customSchema) + .option("header", "true") + .option("dateFormat", "dd/MM/yyyy HH:mm") + .load(testFile(datesFile)) + datesWithFormat.write + .format("csv") + .option("header", "true") + .option("dateFormat", "yyyy/MM/dd") + .save(datesWithFormatPath) + + // This will load back the dates as string. + val stringSchema = StructType(StructField("date", StringType, true) :: Nil) + val stringDatesWithFormat = spark.read + .format("csv") + .schema(stringSchema) + .option("header", "true") + .load(datesWithFormatPath) + val expectedStringDatesWithFormat = Seq( + Row("2015/08/26"), + Row("2014/10/27"), + Row("2016/01/28")) + + checkAnswer(stringDatesWithFormat, expectedStringDatesWithFormat) + } + } + + test("Write timestamps correctly with timestampFormat option") { + withTempDir { dir => + // With dateFormat option. + val timestampsWithFormatPath = s"${dir.getCanonicalPath}/timestampsWithFormat.csv" + val timestampsWithFormat = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "true") + .option("timestampFormat", "dd/MM/yyyy HH:mm") + .load(testFile(datesFile)) + timestampsWithFormat.write + .format("csv") + .option("header", "true") + .option("timestampFormat", "yyyy/MM/dd HH:mm") + .save(timestampsWithFormatPath) + + // This will load back the timestamps as string. + val stringSchema = StructType(StructField("date", StringType, true) :: Nil) + val stringTimestampsWithFormat = spark.read + .format("csv") + .schema(stringSchema) + .option("header", "true") + .load(timestampsWithFormatPath) + val expectedStringTimestampsWithFormat = Seq( + Row("2015/08/26 18:00"), + Row("2014/10/27 18:30"), + Row("2016/01/28 20:00")) + + checkAnswer(stringTimestampsWithFormat, expectedStringTimestampsWithFormat) + } + } + + test("Write timestamps correctly with timestampFormat option and timeZone option") { + withTempDir { dir => + // With dateFormat option and timeZone option. + val timestampsWithFormatPath = s"${dir.getCanonicalPath}/timestampsWithFormat.csv" + val timestampsWithFormat = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "true") + .option("timestampFormat", "dd/MM/yyyy HH:mm") + .load(testFile(datesFile)) + timestampsWithFormat.write + .format("csv") + .option("header", "true") + .option("timestampFormat", "yyyy/MM/dd HH:mm") + .option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + .save(timestampsWithFormatPath) + + // This will load back the timestamps as string. + val stringSchema = StructType(StructField("date", StringType, true) :: Nil) + val stringTimestampsWithFormat = spark.read + .format("csv") + .schema(stringSchema) + .option("header", "true") + .load(timestampsWithFormatPath) + val expectedStringTimestampsWithFormat = Seq( + Row("2015/08/27 01:00"), + Row("2014/10/28 01:30"), + Row("2016/01/29 04:00")) + + checkAnswer(stringTimestampsWithFormat, expectedStringTimestampsWithFormat) + + val readBack = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "true") + .option("timestampFormat", "yyyy/MM/dd HH:mm") + .option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + .load(timestampsWithFormatPath) + + checkAnswer(readBack, timestampsWithFormat) + } + } + + test("load duplicated field names consistently with null or empty strings - case sensitive") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withTempPath { path => + Seq("a,a,c,A,b,B").toDF().write.text(path.getAbsolutePath) + val actualSchema = spark.read + .format("csv") + .option("header", true) + .load(path.getAbsolutePath) + .schema + val fields = Seq("a0", "a1", "c", "A", "b", "B").map(StructField(_, StringType, true)) + val expectedSchema = StructType(fields) + assert(actualSchema == expectedSchema) + } + } + } + + test("load duplicated field names consistently with null or empty strings - case insensitive") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + withTempPath { path => + Seq("a,A,c,A,b,B").toDF().write.text(path.getAbsolutePath) + val actualSchema = spark.read + .format("csv") + .option("header", true) + .load(path.getAbsolutePath) + .schema + val fields = Seq("a0", "A1", "c", "A3", "b4", "B5").map(StructField(_, StringType, true)) + val expectedSchema = StructType(fields) + assert(actualSchema == expectedSchema) + } + } + } + + test("load null when the schema is larger than parsed tokens ") { + withTempPath { path => + Seq("1").toDF().write.text(path.getAbsolutePath) + val schema = StructType( + StructField("a", IntegerType, true) :: + StructField("b", IntegerType, true) :: Nil) + val df = spark.read + .schema(schema) + .option("header", "false") + .csv(path.getAbsolutePath) + + checkAnswer(df, Row(1, null)) + } + } + + test("SPARK-18699 put malformed records in a `columnNameOfCorruptRecord` field") { + Seq(false, true).foreach { multiLine => + val schema = new StructType().add("a", IntegerType).add("b", TimestampType) + // We use `PERMISSIVE` mode by default if invalid string is given. + val df1 = spark + .read + .option("mode", "abcd") + .option("multiLine", multiLine) + .schema(schema) + .csv(testFile(valueMalformedFile)) + checkAnswer(df1, + Row(null, null) :: + Row(1, java.sql.Date.valueOf("1983-08-04")) :: + Nil) + + // If `schema` has `columnNameOfCorruptRecord`, it should handle corrupt records + val columnNameOfCorruptRecord = "_unparsed" + val schemaWithCorrField1 = schema.add(columnNameOfCorruptRecord, StringType) + val df2 = spark + .read + .option("mode", "Permissive") + .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) + .option("multiLine", multiLine) + .schema(schemaWithCorrField1) + .csv(testFile(valueMalformedFile)) + checkAnswer(df2, + Row(null, null, "0,2013-111-11 12:13:14") :: + Row(1, java.sql.Date.valueOf("1983-08-04"), null) :: + Nil) + + // We put a `columnNameOfCorruptRecord` field in the middle of a schema + val schemaWithCorrField2 = new StructType() + .add("a", IntegerType) + .add(columnNameOfCorruptRecord, StringType) + .add("b", TimestampType) + val df3 = spark + .read + .option("mode", "permissive") + .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) + .option("multiLine", multiLine) + .schema(schemaWithCorrField2) + .csv(testFile(valueMalformedFile)) + checkAnswer(df3, + Row(null, "0,2013-111-11 12:13:14", null) :: + Row(1, null, java.sql.Date.valueOf("1983-08-04")) :: + Nil) + + val errMsg = intercept[AnalysisException] { + spark + .read + .option("mode", "PERMISSIVE") + .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) + .option("multiLine", multiLine) + .schema(schema.add(columnNameOfCorruptRecord, IntegerType)) + .csv(testFile(valueMalformedFile)) + .collect + }.getMessage + assert(errMsg.startsWith("The field for corrupt records must be string type and nullable")) + } + } + + test("SPARK-19610: Parse normal multi-line CSV files") { + val primitiveFieldAndType = Seq( + """" + |string","integer + | + | + |","long + | + |","bigInteger",double,boolean,null""".stripMargin, + """"this is a + |simple + |string."," + | + |10"," + |21474836470","92233720368547758070"," + | + |1.7976931348623157E308",true,""".stripMargin) + + withTempPath { path => + primitiveFieldAndType.toDF("value").coalesce(1).write.text(path.getAbsolutePath) + + val df = spark.read + .option("header", true) + .option("multiLine", true) + .csv(path.getAbsolutePath) + + // Check if headers have new lines in the names. + val actualFields = df.schema.fieldNames.toSeq + val expectedFields = + Seq("\nstring", "integer\n\n\n", "long\n\n", "bigInteger", "double", "boolean", "null") + assert(actualFields === expectedFields) + + // Check if the rows have new lines in the values. + val expected = Row( + "this is a\nsimple\nstring.", + "\n\n10", + "\n21474836470", + "92233720368547758070", + "\n\n1.7976931348623157E308", + "true", + null) + checkAnswer(df, expected) + } + } + + test("Empty file produces empty dataframe with empty schema") { + Seq(false, true).foreach { multiLine => + val df = spark.read.format("csv") + .option("header", true) + .option("multiLine", multiLine) + .load(testFile(emptyFile)) + + assert(df.schema === spark.emptyDataFrame.schema) + checkAnswer(df, spark.emptyDataFrame) + } + } + + test("Empty string dataset produces empty dataframe and keep user-defined schema") { + val df1 = spark.read.csv(spark.emptyDataset[String]) + assert(df1.schema === spark.emptyDataFrame.schema) + checkAnswer(df1, spark.emptyDataFrame) + + val schema = StructType(StructField("a", StringType) :: Nil) + val df2 = spark.read.schema(schema).csv(spark.emptyDataset[String]) + assert(df2.schema === schema) + } + + test("ignoreLeadingWhiteSpace and ignoreTrailingWhiteSpace options - read") { + val input = " a,b , c " + + // For reading, default of both `ignoreLeadingWhiteSpace` and`ignoreTrailingWhiteSpace` + // are `false`. So, these are excluded. + val combinations = Seq( + (true, true), + (false, true), + (true, false)) + + // Check if read rows ignore whitespaces as configured. + val expectedRows = Seq( + Row("a", "b", "c"), + Row(" a", "b", " c"), + Row("a", "b ", "c ")) + + combinations.zip(expectedRows) + .foreach { case ((ignoreLeadingWhiteSpace, ignoreTrailingWhiteSpace), expected) => + val df = spark.read + .option("ignoreLeadingWhiteSpace", ignoreLeadingWhiteSpace) + .option("ignoreTrailingWhiteSpace", ignoreTrailingWhiteSpace) + .csv(Seq(input).toDS()) + + checkAnswer(df, expected) + } + } + + test("SPARK-18579: ignoreLeadingWhiteSpace and ignoreTrailingWhiteSpace options - write") { + val df = Seq((" a", "b ", " c ")).toDF() + + // For writing, default of both `ignoreLeadingWhiteSpace` and `ignoreTrailingWhiteSpace` + // are `true`. So, these are excluded. + val combinations = Seq( + (false, false), + (false, true), + (true, false)) + + // Check if written lines ignore each whitespaces as configured. + val expectedLines = Seq( + " a,b , c ", + " a,b, c", + "a,b ,c ") + + combinations.zip(expectedLines) + .foreach { case ((ignoreLeadingWhiteSpace, ignoreTrailingWhiteSpace), expected) => + withTempPath { path => + df.write + .option("ignoreLeadingWhiteSpace", ignoreLeadingWhiteSpace) + .option("ignoreTrailingWhiteSpace", ignoreTrailingWhiteSpace) + .csv(path.getAbsolutePath) + + // Read back the written lines. + val readBack = spark.read.text(path.getAbsolutePath) + checkAnswer(readBack, Row(expected)) + } + } + } + + test("SPARK-21263: Invalid float and double are handled correctly in different modes") { + val exception = intercept[SparkException] { + spark.read.schema("a DOUBLE") + .option("mode", "FAILFAST") + .csv(Seq("10u12").toDS()) + .collect() + } + assert(exception.getMessage.contains("""input string: "10u12"""")) + + val count = spark.read.schema("a FLOAT") + .option("mode", "DROPMALFORMED") + .csv(Seq("10u12").toDS()) + .count() + assert(count == 0) + + val results = spark.read.schema("a FLOAT") + .option("mode", "PERMISSIVE") + .csv(Seq("10u12").toDS()) + checkAnswer(results, Row(null)) + } + + test("SPARK-20978: Fill the malformed column when the number of tokens is less than schema") { + val df = spark.read + .schema("a string, b string, unparsed string") + .option("columnNameOfCorruptRecord", "unparsed") + .csv(Seq("a").toDS()) + checkAnswer(df, Row("a", null, "a")) + } + + test("SPARK-21610: Corrupt records are not handled properly when creating a dataframe " + + "from a file") { + val columnNameOfCorruptRecord = "_corrupt_record" + val schema = new StructType() + .add("a", IntegerType) + .add("b", TimestampType) + .add(columnNameOfCorruptRecord, StringType) + // negative cases + val msg = intercept[AnalysisException] { + spark + .read + .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) + .schema(schema) + .csv(testFile(valueMalformedFile)) + .select(columnNameOfCorruptRecord) + .collect() + }.getMessage + assert(msg.contains("only include the internal corrupt record column")) + intercept[org.apache.spark.sql.catalyst.errors.TreeNodeException[_]] { + spark + .read + .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) + .schema(schema) + .csv(testFile(valueMalformedFile)) + .filter($"_corrupt_record".isNotNull) + .count() + } + // workaround + val df = spark + .read + .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) + .schema(schema) + .csv(testFile(valueMalformedFile)) + .cache() + assert(df.filter($"_corrupt_record".isNotNull).count() == 1) + assert(df.filter($"_corrupt_record".isNull).count() == 1) + checkAnswer( + df.select(columnNameOfCorruptRecord), + Row("0,2013-111-11 12:13:14") :: Row(null) :: Nil + ) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtilsSuite.scala new file mode 100644 index 000000000000..221e44ce2cff --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtilsSuite.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.csv + +import org.apache.spark.SparkFunSuite + +class CSVUtilsSuite extends SparkFunSuite { + test("Can parse escaped characters") { + assert(CSVUtils.toChar("""\t""") === '\t') + assert(CSVUtils.toChar("""\r""") === '\r') + assert(CSVUtils.toChar("""\b""") === '\b') + assert(CSVUtils.toChar("""\f""") === '\f') + assert(CSVUtils.toChar("""\"""") === '\"') + assert(CSVUtils.toChar("""\'""") === '\'') + assert(CSVUtils.toChar("""\u0000""") === '\u0000') + } + + test("Does not accept delimiter larger than one character") { + val exception = intercept[IllegalArgumentException]{ + CSVUtils.toChar("ab") + } + assert(exception.getMessage.contains("cannot be more than one character")) + } + + test("Throws exception for unsupported escaped characters") { + val exception = intercept[IllegalArgumentException]{ + CSVUtils.toChar("""\1""") + } + assert(exception.getMessage.contains("Unsupported special character for delimiter")) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParserSuite.scala new file mode 100644 index 000000000000..efbf73534bd1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParserSuite.scala @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.csv + +import java.math.BigDecimal +import java.util.Locale + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +class UnivocityParserSuite extends SparkFunSuite { + private val parser = + new UnivocityParser(StructType(Seq.empty), new CSVOptions(Map.empty[String, String], "GMT")) + + private def assertNull(v: Any) = assert(v == null) + + test("Can parse decimal type values") { + val stringValues = Seq("10.05", "1,000.01", "158,058,049.001") + val decimalValues = Seq(10.05, 1000.01, 158058049.001) + val decimalType = new DecimalType() + + stringValues.zip(decimalValues).foreach { case (strVal, decimalVal) => + val decimalValue = new BigDecimal(decimalVal.toString) + val options = new CSVOptions(Map.empty[String, String], "GMT") + assert(parser.makeConverter("_1", decimalType, options = options).apply(strVal) === + Decimal(decimalValue, decimalType.precision, decimalType.scale)) + } + } + + test("Nullable types are handled") { + val types = Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, + BooleanType, DecimalType.DoubleDecimal, TimestampType, DateType, StringType) + + // Nullable field with nullValue option. + types.foreach { t => + // Tests that a custom nullValue. + val nullValueOptions = new CSVOptions(Map("nullValue" -> "-"), "GMT") + val converter = + parser.makeConverter("_1", t, nullable = true, options = nullValueOptions) + assertNull(converter.apply("-")) + assertNull(converter.apply(null)) + + // Tests that the default nullValue is empty string. + val options = new CSVOptions(Map.empty[String, String], "GMT") + assertNull(parser.makeConverter("_1", t, nullable = true, options = options).apply("")) + } + + // Not nullable field with nullValue option. + types.foreach { t => + // Casts a null to not nullable field should throw an exception. + val options = new CSVOptions(Map("nullValue" -> "-"), "GMT") + val converter = + parser.makeConverter("_1", t, nullable = false, options = options) + var message = intercept[RuntimeException] { + converter.apply("-") + }.getMessage + assert(message.contains("null value found but field _1 is not nullable.")) + message = intercept[RuntimeException] { + converter.apply(null) + }.getMessage + assert(message.contains("null value found but field _1 is not nullable.")) + } + + // If nullValue is different with empty string, then, empty string should not be casted into + // null. + Seq(true, false).foreach { b => + val options = new CSVOptions(Map("nullValue" -> "null"), "GMT") + val converter = + parser.makeConverter("_1", StringType, nullable = b, options = options) + assert(converter.apply("") == UTF8String.fromString("")) + } + } + + test("Throws exception for empty string with non null type") { + val options = new CSVOptions(Map.empty[String, String], "GMT") + val exception = intercept[RuntimeException]{ + parser.makeConverter("_1", IntegerType, nullable = false, options = options).apply("") + } + assert(exception.getMessage.contains("null value found but field _1 is not nullable.")) + } + + test("Types are cast correctly") { + val options = new CSVOptions(Map.empty[String, String], "GMT") + assert(parser.makeConverter("_1", ByteType, options = options).apply("10") == 10) + assert(parser.makeConverter("_1", ShortType, options = options).apply("10") == 10) + assert(parser.makeConverter("_1", IntegerType, options = options).apply("10") == 10) + assert(parser.makeConverter("_1", LongType, options = options).apply("10") == 10) + assert(parser.makeConverter("_1", FloatType, options = options).apply("1.00") == 1.0) + assert(parser.makeConverter("_1", DoubleType, options = options).apply("1.00") == 1.0) + assert(parser.makeConverter("_1", BooleanType, options = options).apply("true") == true) + + val timestampsOptions = + new CSVOptions(Map("timestampFormat" -> "dd/MM/yyyy hh:mm"), "GMT") + val customTimestamp = "31/01/2015 00:00" + val expectedTime = timestampsOptions.timestampFormat.parse(customTimestamp).getTime + val castedTimestamp = + parser.makeConverter("_1", TimestampType, nullable = true, options = timestampsOptions) + .apply(customTimestamp) + assert(castedTimestamp == expectedTime * 1000L) + + val customDate = "31/01/2015" + val dateOptions = new CSVOptions(Map("dateFormat" -> "dd/MM/yyyy"), "GMT") + val expectedDate = dateOptions.dateFormat.parse(customDate).getTime + val castedDate = + parser.makeConverter("_1", DateType, nullable = true, options = dateOptions) + .apply(customTimestamp) + assert(castedDate == DateTimeUtils.millisToDays(expectedDate)) + + val timestamp = "2015-01-01 00:00:00" + assert(parser.makeConverter("_1", TimestampType, options = options).apply(timestamp) == + DateTimeUtils.stringToTime(timestamp).getTime * 1000L) + assert(parser.makeConverter("_1", DateType, options = options).apply("2015-01-01") == + DateTimeUtils.millisToDays(DateTimeUtils.stringToTime("2015-01-01").getTime)) + } + + test("Throws exception for casting an invalid string to Float and Double Types") { + val options = new CSVOptions(Map.empty[String, String], "GMT") + val types = Seq(DoubleType, FloatType) + val input = Seq("10u000", "abc", "1 2/3") + types.foreach { dt => + input.foreach { v => + val message = intercept[NumberFormatException] { + parser.makeConverter("_1", dt, options = options).apply(v) + }.getMessage + assert(message.contains(v)) + } + } + } + + test("Float NaN values are parsed correctly") { + val options = new CSVOptions(Map("nanValue" -> "nn"), "GMT") + val floatVal: Float = parser.makeConverter( + "_1", FloatType, nullable = true, options = options + ).apply("nn").asInstanceOf[Float] + + // Java implements the IEEE-754 floating point standard which guarantees that any comparison + // against NaN will return false (except != which returns true) + assert(floatVal != floatVal) + } + + test("Double NaN values are parsed correctly") { + val options = new CSVOptions(Map("nanValue" -> "-"), "GMT") + val doubleVal: Double = parser.makeConverter( + "_1", DoubleType, nullable = true, options = options + ).apply("-").asInstanceOf[Double] + + assert(doubleVal.isNaN) + } + + test("Float infinite values can be parsed") { + val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), "GMT") + val floatVal1 = parser.makeConverter( + "_1", FloatType, nullable = true, options = negativeInfOptions + ).apply("max").asInstanceOf[Float] + + assert(floatVal1 == Float.NegativeInfinity) + + val positiveInfOptions = new CSVOptions(Map("positiveInf" -> "max"), "GMT") + val floatVal2 = parser.makeConverter( + "_1", FloatType, nullable = true, options = positiveInfOptions + ).apply("max").asInstanceOf[Float] + + assert(floatVal2 == Float.PositiveInfinity) + } + + test("Double infinite values can be parsed") { + val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), "GMT") + val doubleVal1 = parser.makeConverter( + "_1", DoubleType, nullable = true, options = negativeInfOptions + ).apply("max").asInstanceOf[Double] + + assert(doubleVal1 == Double.NegativeInfinity) + + val positiveInfOptions = new CSVOptions(Map("positiveInf" -> "max"), "GMT") + val doubleVal2 = parser.makeConverter( + "_1", DoubleType, nullable = true, options = positiveInfOptions + ).apply("max").asInstanceOf[Double] + + assert(doubleVal2 == Double.PositiveInfinity) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala new file mode 100644 index 000000000000..7d277c1ffaff --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.jdbc + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.types._ + +class JdbcUtilsSuite extends SparkFunSuite { + + val tableSchema = StructType(Seq( + StructField("C1", StringType, false), StructField("C2", IntegerType, false))) + val caseSensitive = org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution + val caseInsensitive = org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution + + test("Parse user specified column types") { + assert(JdbcUtils.getCustomSchema(tableSchema, null, caseInsensitive) === tableSchema) + assert(JdbcUtils.getCustomSchema(tableSchema, "", caseInsensitive) === tableSchema) + + assert(JdbcUtils.getCustomSchema(tableSchema, "c1 DATE", caseInsensitive) === + StructType(Seq(StructField("C1", DateType, false), StructField("C2", IntegerType, false)))) + assert(JdbcUtils.getCustomSchema(tableSchema, "c1 DATE", caseSensitive) === + StructType(Seq(StructField("C1", StringType, false), StructField("C2", IntegerType, false)))) + + assert( + JdbcUtils.getCustomSchema(tableSchema, "c1 DATE, C2 STRING", caseInsensitive) === + StructType(Seq(StructField("C1", DateType, false), StructField("C2", StringType, false)))) + assert(JdbcUtils.getCustomSchema(tableSchema, "c1 DATE, C2 STRING", caseSensitive) === + StructType(Seq(StructField("C1", StringType, false), StructField("C2", StringType, false)))) + + // Throw AnalysisException + val duplicate = intercept[AnalysisException]{ + JdbcUtils.getCustomSchema(tableSchema, "c1 DATE, c1 STRING", caseInsensitive) === + StructType(Seq(StructField("c1", DateType, false), StructField("c1", StringType, false))) + } + assert(duplicate.getMessage.contains( + "Found duplicate column(s) in the customSchema option value")) + + // Throw ParseException + val dataTypeNotSupported = intercept[ParseException]{ + JdbcUtils.getCustomSchema(tableSchema, "c3 DATEE, C2 STRING", caseInsensitive) === + StructType(Seq(StructField("c3", DateType, false), StructField("C2", StringType, false))) + } + assert(dataTypeNotSupported.getMessage.contains("DataType datee is not supported")) + + val mismatchedInput = intercept[ParseException]{ + JdbcUtils.getCustomSchema(tableSchema, "c3 DATE. C2 STRING", caseInsensitive) === + StructType(Seq(StructField("c3", DateType, false), StructField("C2", StringType, false))) + } + assert(mismatchedInput.getMessage.contains("mismatched input '.' expecting")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala new file mode 100644 index 000000000000..316c5183fddf --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.json + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.json.JSONOptions +import org.apache.spark.sql.test.SharedSQLContext + +/** + * Test cases for various [[JSONOptions]]. + */ +class JsonParsingOptionsSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("allowComments off") { + val str = """{'name': /* hello */ 'Reynold Xin'}""" + val df = spark.read.json(Seq(str).toDS()) + + assert(df.schema.head.name == "_corrupt_record") + } + + test("allowComments on") { + val str = """{'name': /* hello */ 'Reynold Xin'}""" + val df = spark.read.option("allowComments", "true").json(Seq(str).toDS()) + + assert(df.schema.head.name == "name") + assert(df.first().getString(0) == "Reynold Xin") + } + + test("allowSingleQuotes off") { + val str = """{'name': 'Reynold Xin'}""" + val df = spark.read.option("allowSingleQuotes", "false").json(Seq(str).toDS()) + + assert(df.schema.head.name == "_corrupt_record") + } + + test("allowSingleQuotes on") { + val str = """{'name': 'Reynold Xin'}""" + val df = spark.read.json(Seq(str).toDS()) + + assert(df.schema.head.name == "name") + assert(df.first().getString(0) == "Reynold Xin") + } + + test("allowUnquotedFieldNames off") { + val str = """{name: 'Reynold Xin'}""" + val df = spark.read.json(Seq(str).toDS()) + + assert(df.schema.head.name == "_corrupt_record") + } + + test("allowUnquotedFieldNames on") { + val str = """{name: 'Reynold Xin'}""" + val df = spark.read.option("allowUnquotedFieldNames", "true").json(Seq(str).toDS()) + + assert(df.schema.head.name == "name") + assert(df.first().getString(0) == "Reynold Xin") + } + + test("allowUnquotedControlChars off") { + val str = """{"name": "a\u0001b"}""" + val df = spark.read.json(Seq(str).toDS()) + + assert(df.schema.head.name == "_corrupt_record") + } + + test("allowUnquotedControlChars on") { + val str = """{"name": "a\u0001b"}""" + val df = spark.read.option("allowUnquotedControlChars", "true").json(Seq(str).toDS()) + + assert(df.schema.head.name == "name") + assert(df.first().getString(0) == "a\u0001b") + } + + test("allowNumericLeadingZeros off") { + val str = """{"age": 0018}""" + val df = spark.read.json(Seq(str).toDS()) + + assert(df.schema.head.name == "_corrupt_record") + } + + test("allowNumericLeadingZeros on") { + val str = """{"age": 0018}""" + val df = spark.read.option("allowNumericLeadingZeros", "true").json(Seq(str).toDS()) + + assert(df.schema.head.name == "age") + assert(df.first().getLong(0) == 18) + } + + // The following two tests are not really working - need to look into Jackson's + // JsonParser.Feature.ALLOW_NON_NUMERIC_NUMBERS. + ignore("allowNonNumericNumbers off") { + val str = """{"age": NaN}""" + val df = spark.read.json(Seq(str).toDS()) + + assert(df.schema.head.name == "_corrupt_record") + } + + ignore("allowNonNumericNumbers on") { + val str = """{"age": NaN}""" + val df = spark.read.option("allowNonNumericNumbers", "true").json(Seq(str).toDS()) + + assert(df.schema.head.name == "age") + assert(df.first().getDouble(0).isNaN) + } + + test("allowBackslashEscapingAnyCharacter off") { + val str = """{"name": "Cazen Lee", "price": "\$10"}""" + val df = spark.read.option("allowBackslashEscapingAnyCharacter", "false").json(Seq(str).toDS()) + + assert(df.schema.head.name == "_corrupt_record") + } + + test("allowBackslashEscapingAnyCharacter on") { + val str = """{"name": "Cazen Lee", "price": "\$10"}""" + val df = spark.read.option("allowBackslashEscapingAnyCharacter", "true").json(Seq(str).toDS()) + + assert(df.schema.head.name == "name") + assert(df.schema.last.name == "price") + assert(df.first().getString(0) == "Cazen Lee") + assert(df.first().getString(1) == "$10") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index 28b8f02bdf87..8c8d41ebf115 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -18,20 +18,32 @@ package org.apache.spark.sql.execution.datasources.json import java.io.{File, StringWriter} +import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} +import java.util.Locale import com.fasterxml.jackson.core.JsonFactory -import org.apache.spark.rdd.RDD -import org.scalactic.Tolerance._ +import org.apache.hadoop.fs.{Path, PathFilter} +import org.apache.hadoop.io.SequenceFile.CompressionType +import org.apache.hadoop.io.compress.GzipCodec -import org.apache.spark.sql._ +import org.apache.spark.SparkException +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{functions => F, _} +import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions} import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.datasources.{ResolvedDataSource, LogicalRelation} -import org.apache.spark.sql.execution.datasources.json.InferSchema.compatibleType +import org.apache.spark.sql.execution.ExternalRDD +import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.datasources.json.JsonInferSchema.compatibleType +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.util.Utils +class TestFileFilter extends PathFilter { + override def accept(path: Path): Boolean = path.getParent.getName != "p=2" +} + class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { import testImplicits._ @@ -52,9 +64,14 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { generator.flush() } - Utils.tryWithResource(factory.createParser(writer.toString)) { parser => - parser.nextToken() - JacksonParser.convertField(factory, parser, dataType) + val dummyOption = new JSONOptions(Map.empty[String, String], "GMT") + val dummySchema = StructType(Seq.empty) + val parser = new JacksonParser(dummySchema, dummyOption) + + Utils.tryWithResource(factory.createParser(writer.toString)) { jsonParser => + jsonParser.nextToken() + val converter = parser.makeConverter(dataType) + converter.apply(jsonParser) } } @@ -74,9 +91,9 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val doubleNumber: Double = 1.7976931348623157E308d checkTypePromotion(doubleNumber.toDouble, enforceCorrectType(doubleNumber, DoubleType)) - checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber)), + checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber * 1000L)), enforceCorrectType(intNumber, TimestampType)) - checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber.toLong)), + checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber.toLong * 1000L)), enforceCorrectType(intNumber.toLong, TimestampType)) val strTime = "2014-09-30 12:34:56" checkTypePromotion(DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf(strTime)), @@ -87,15 +104,15 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { DateTimeUtils.fromJavaDate(Date.valueOf(strDate)), enforceCorrectType(strDate, DateType)) val ISO8601Time1 = "1970-01-01T01:00:01.0Z" + val ISO8601Time2 = "1970-01-01T02:00:01-01:00" checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(3601000)), enforceCorrectType(ISO8601Time1, TimestampType)) - checkTypePromotion(DateTimeUtils.millisToDays(3601000), - enforceCorrectType(ISO8601Time1, DateType)) - val ISO8601Time2 = "1970-01-01T02:00:01-01:00" checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(10801000)), enforceCorrectType(ISO8601Time2, TimestampType)) - checkTypePromotion(DateTimeUtils.millisToDays(10801000), - enforceCorrectType(ISO8601Time2, DateType)) + + val ISO8601Date = "1970-01-01" + checkTypePromotion(DateTimeUtils.millisToDays(32400000), + enforceCorrectType(ISO8601Date, DateType)) } test("Get compatible type") { @@ -197,7 +214,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { StructType( StructField("f1", IntegerType, true) :: StructField("f2", IntegerType, true) :: Nil), - StructType(StructField("f1", LongType, true) :: Nil) , + StructType(StructField("f1", LongType, true) :: Nil), StructType( StructField("f1", LongType, true) :: StructField("f2", IntegerType, true) :: Nil)) @@ -217,7 +234,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("Complex field and type inferring with null in sampling") { - val jsonDF = sqlContext.read.json(jsonNullStruct) + val jsonDF = spark.read.json(jsonNullStruct) val expectedSchema = StructType( StructField("headers", StructType( StructField("Charset", StringType, true) :: @@ -227,7 +244,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { StructField("nullstr", StringType, true):: Nil) assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select nullstr, headers.Host from jsonTable"), @@ -236,7 +253,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("Primitive field and type inferring") { - val jsonDF = sqlContext.read.json(primitiveFieldAndType) + val jsonDF = spark.read.json(primitiveFieldAndType) val expectedSchema = StructType( StructField("bigInteger", DecimalType(20, 0), true) :: @@ -249,7 +266,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -264,7 +281,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("Complex field and type inferring") { - val jsonDF = sqlContext.read.json(complexFieldAndType1) + val jsonDF = spark.read.json(complexFieldAndType1) val expectedSchema = StructType( StructField("arrayOfArray1", ArrayType(ArrayType(StringType, true), true), true) :: @@ -290,7 +307,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") // Access elements of a primitive array. checkAnswer( @@ -363,8 +380,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("GetField operation on complex data type") { - val jsonDF = sqlContext.read.json(complexFieldAndType1) - jsonDF.registerTempTable("jsonTable") + val jsonDF = spark.read.json(complexFieldAndType1) + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), @@ -379,7 +396,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("Type conflict in primitive field values") { - val jsonDF = sqlContext.read.json(primitiveFieldValueTypeConflict) + val jsonDF = spark.read.json(primitiveFieldValueTypeConflict) val expectedSchema = StructType( StructField("num_bool", StringType, true) :: @@ -391,7 +408,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -433,14 +450,14 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // Number and String conflict: resolve the type as number in this query. checkAnswer( - sql("select num_str + 1.2 from jsonTable where num_str > 14"), - Row(BigDecimal("92233720368547758071.2")) + sql("select num_str + 1.2 from jsonTable where num_str > 14d"), + Row(92233720368547758071.2) ) // Number and String conflict: resolve the type as number in this query. checkAnswer( sql("select num_str + 1.2 from jsonTable where num_str >= 92233720368547758060"), - Row(new java.math.BigDecimal("92233720368547758071.2")) + Row(new java.math.BigDecimal("92233720368547758071.2").doubleValue) ) // String and Boolean conflict: resolve the type as string. @@ -451,8 +468,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } ignore("Type conflict in primitive field values (Ignored)") { - val jsonDF = sqlContext.read.json(primitiveFieldValueTypeConflict) - jsonDF.registerTempTable("jsonTable") + val jsonDF = spark.read.json(primitiveFieldValueTypeConflict) + jsonDF.createOrReplaceTempView("jsonTable") // Right now, the analyzer does not promote strings in a boolean expression. // Number and Boolean conflict: resolve the type as boolean in this query. @@ -504,7 +521,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("Type conflict in complex field values") { - val jsonDF = sqlContext.read.json(complexFieldValueTypeConflict) + val jsonDF = spark.read.json(complexFieldValueTypeConflict) val expectedSchema = StructType( StructField("array", ArrayType(LongType, true), true) :: @@ -516,7 +533,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -528,7 +545,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("Type conflict in array elements") { - val jsonDF = sqlContext.read.json(arrayElementTypeConflict) + val jsonDF = spark.read.json(arrayElementTypeConflict) val expectedSchema = StructType( StructField("array1", ArrayType(StringType, true), true) :: @@ -538,7 +555,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -556,7 +573,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("Handling missing fields") { - val jsonDF = sqlContext.read.json(missingFields) + val jsonDF = spark.read.json(missingFields) val expectedSchema = StructType( StructField("a", BooleanType, true) :: @@ -568,44 +585,15 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") - } - - test("jsonFile should be based on JSONRelation") { - val dir = Utils.createTempDir() - dir.delete() - val path = dir.getCanonicalFile.toURI.toString - sparkContext.parallelize(1 to 100) - .map(i => s"""{"a": 1, "b": "str$i"}""").saveAsTextFile(path) - val jsonDF = sqlContext.read.option("samplingRatio", "0.49").json(path) - - val analyzed = jsonDF.queryExecution.analyzed - assert( - analyzed.isInstanceOf[LogicalRelation], - "The DataFrame returned by jsonFile should be based on LogicalRelation.") - val relation = analyzed.asInstanceOf[LogicalRelation].relation - assert( - relation.isInstanceOf[JSONRelation], - "The DataFrame returned by jsonFile should be based on JSONRelation.") - assert(relation.asInstanceOf[JSONRelation].paths === Array(path)) - assert(relation.asInstanceOf[JSONRelation].samplingRatio === (0.49 +- 0.001)) - - val schema = StructType(StructField("a", LongType, true) :: Nil) - val logicalRelation = - sqlContext.read.schema(schema).json(path) - .queryExecution.analyzed.asInstanceOf[LogicalRelation] - val relationWithSchema = logicalRelation.relation.asInstanceOf[JSONRelation] - assert(relationWithSchema.paths === Array(path)) - assert(relationWithSchema.schema === schema) - assert(relationWithSchema.samplingRatio > 0.99) + jsonDF.createOrReplaceTempView("jsonTable") } test("Loading a JSON dataset from a text file") { val dir = Utils.createTempDir() dir.delete() val path = dir.getCanonicalPath - primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path) - val jsonDF = sqlContext.read.json(path) + primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).write.text(path) + val jsonDF = spark.read.json(path) val expectedSchema = StructType( StructField("bigInteger", DecimalType(20, 0), true) :: @@ -618,7 +606,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -636,8 +624,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val dir = Utils.createTempDir() dir.delete() val path = dir.getCanonicalPath - primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path) - val jsonDF = sqlContext.read.option("primitivesAsString", "true").json(path) + primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).write.text(path) + val jsonDF = spark.read.option("primitivesAsString", "true").json(path) val expectedSchema = StructType( StructField("bigInteger", StringType, true) :: @@ -650,7 +638,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -665,7 +653,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("Loading a JSON dataset primitivesAsString returns complex fields as strings") { - val jsonDF = sqlContext.read.option("primitivesAsString", "true").json(complexFieldAndType1) + val jsonDF = spark.read.option("primitivesAsString", "true").json(complexFieldAndType1) val expectedSchema = StructType( StructField("arrayOfArray1", ArrayType(ArrayType(StringType, true), true), true) :: @@ -691,7 +679,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { assert(expectedSchema === jsonDF.schema) - jsonDF.registerTempTable("jsonTable") + jsonDF.createOrReplaceTempView("jsonTable") // Access elements of a primitive array. checkAnswer( @@ -762,15 +750,109 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ) } + test("Loading a JSON dataset prefersDecimal returns schema with float types as BigDecimal") { + val jsonDF = spark.read.option("prefersDecimal", "true").json(primitiveFieldAndType) + + val expectedSchema = StructType( + StructField("bigInteger", DecimalType(20, 0), true) :: + StructField("boolean", BooleanType, true) :: + StructField("double", DecimalType(17, -292), true) :: + StructField("integer", LongType, true) :: + StructField("long", LongType, true) :: + StructField("null", StringType, true) :: + StructField("string", StringType, true) :: Nil) + + assert(expectedSchema === jsonDF.schema) + + jsonDF.createOrReplaceTempView("jsonTable") + + checkAnswer( + sql("select * from jsonTable"), + Row(BigDecimal("92233720368547758070"), + true, + BigDecimal("1.7976931348623157E308"), + 10, + 21474836470L, + null, + "this is a simple string.") + ) + } + + test("Find compatible types even if inferred DecimalType is not capable of other IntegralType") { + val mixedIntegerAndDoubleRecords = Seq( + """{"a": 3, "b": 1.1}""", + s"""{"a": 3.1, "b": 0.${"0" * 38}1}""").toDS() + val jsonDF = spark.read + .option("prefersDecimal", "true") + .json(mixedIntegerAndDoubleRecords) + + // The values in `a` field will be decimals as they fit in decimal. For `b` field, + // they will be doubles as `1.0E-39D` does not fit. + val expectedSchema = StructType( + StructField("a", DecimalType(21, 1), true) :: + StructField("b", DoubleType, true) :: Nil) + + assert(expectedSchema === jsonDF.schema) + checkAnswer( + jsonDF, + Row(BigDecimal("3"), 1.1D) :: + Row(BigDecimal("3.1"), 1.0E-39D) :: Nil + ) + } + + test("Infer big integers correctly even when it does not fit in decimal") { + val jsonDF = spark.read + .json(bigIntegerRecords) + + // The value in `a` field will be a double as it does not fit in decimal. For `b` field, + // it will be a decimal as `92233720368547758070`. + val expectedSchema = StructType( + StructField("a", DoubleType, true) :: + StructField("b", DecimalType(20, 0), true) :: Nil) + + assert(expectedSchema === jsonDF.schema) + checkAnswer(jsonDF, Row(1.0E38D, BigDecimal("92233720368547758070"))) + } + + test("Infer floating-point values correctly even when it does not fit in decimal") { + val jsonDF = spark.read + .option("prefersDecimal", "true") + .json(floatingValueRecords) + + // The value in `a` field will be a double as it does not fit in decimal. For `b` field, + // it will be a decimal as `0.01` by having a precision equal to the scale. + val expectedSchema = StructType( + StructField("a", DoubleType, true) :: + StructField("b", DecimalType(2, 2), true):: Nil) + + assert(expectedSchema === jsonDF.schema) + checkAnswer(jsonDF, Row(1.0E-39D, BigDecimal("0.01"))) + + val mergedJsonDF = spark.read + .option("prefersDecimal", "true") + .json(floatingValueRecords.union(bigIntegerRecords)) + + val expectedMergedSchema = StructType( + StructField("a", DoubleType, true) :: + StructField("b", DecimalType(22, 2), true):: Nil) + + assert(expectedMergedSchema === mergedJsonDF.schema) + checkAnswer( + mergedJsonDF, + Row(1.0E-39D, BigDecimal("0.01")) :: + Row(1.0E38D, BigDecimal("92233720368547758070")) :: Nil + ) + } + test("Loading a JSON dataset from a text file with SQL") { val dir = Utils.createTempDir() dir.delete() - val path = dir.getCanonicalPath - primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path) + val path = dir.toURI.toString + primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).write.text(path) sql( s""" - |CREATE TEMPORARY TABLE jsonTableSQL + |CREATE TEMPORARY VIEW jsonTableSQL |USING org.apache.spark.sql.json |OPTIONS ( | path '$path' @@ -793,7 +875,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val dir = Utils.createTempDir() dir.delete() val path = dir.getCanonicalPath - primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path) + primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).write.text(path) val schema = StructType( StructField("bigInteger", DecimalType.SYSTEM_DEFAULT, true) :: @@ -804,11 +886,11 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { StructField("null", StringType, true) :: StructField("string", StringType, true) :: Nil) - val jsonDF1 = sqlContext.read.schema(schema).json(path) + val jsonDF1 = spark.read.schema(schema).json(path) assert(schema === jsonDF1.schema) - jsonDF1.registerTempTable("jsonTable1") + jsonDF1.createOrReplaceTempView("jsonTable1") checkAnswer( sql("select * from jsonTable1"), @@ -821,11 +903,11 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { "this is a simple string.") ) - val jsonDF2 = sqlContext.read.schema(schema).json(primitiveFieldAndType) + val jsonDF2 = spark.read.schema(schema).json(primitiveFieldAndType) assert(schema === jsonDF2.schema) - jsonDF2.registerTempTable("jsonTable2") + jsonDF2.createOrReplaceTempView("jsonTable2") checkAnswer( sql("select * from jsonTable2"), @@ -842,12 +924,12 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("Applying schemas with MapType") { val schemaWithSimpleMap = StructType( StructField("map", MapType(StringType, IntegerType, true), false) :: Nil) - val jsonWithSimpleMap = sqlContext.read.schema(schemaWithSimpleMap).json(mapType1) + val jsonWithSimpleMap = spark.read.schema(schemaWithSimpleMap).json(mapType1) - jsonWithSimpleMap.registerTempTable("jsonWithSimpleMap") + jsonWithSimpleMap.createOrReplaceTempView("jsonWithSimpleMap") checkAnswer( - sql("select map from jsonWithSimpleMap"), + sql("select `map` from jsonWithSimpleMap"), Row(Map("a" -> 1)) :: Row(Map("b" -> 2)) :: Row(Map("c" -> 3)) :: @@ -855,14 +937,16 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { Row(Map("e" -> null)) :: Nil ) - checkAnswer( - sql("select map['c'] from jsonWithSimpleMap"), - Row(null) :: - Row(null) :: - Row(3) :: - Row(1) :: - Row(null) :: Nil - ) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + checkAnswer( + sql("select `map`['c'] from jsonWithSimpleMap"), + Row(null) :: + Row(null) :: + Row(3) :: + Row(1) :: + Row(null) :: Nil + ) + } val innerStruct = StructType( StructField("field1", ArrayType(IntegerType, true), true) :: @@ -870,12 +954,12 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val schemaWithComplexMap = StructType( StructField("map", MapType(StringType, innerStruct, true), false) :: Nil) - val jsonWithComplexMap = sqlContext.read.schema(schemaWithComplexMap).json(mapType2) + val jsonWithComplexMap = spark.read.schema(schemaWithComplexMap).json(mapType2) - jsonWithComplexMap.registerTempTable("jsonWithComplexMap") + jsonWithComplexMap.createOrReplaceTempView("jsonWithComplexMap") checkAnswer( - sql("select map from jsonWithComplexMap"), + sql("select `map` from jsonWithComplexMap"), Row(Map("a" -> Row(Seq(1, 2, 3, null), null))) :: Row(Map("b" -> Row(null, 2))) :: Row(Map("c" -> Row(Seq(), 4))) :: @@ -884,20 +968,22 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { Row(Map("f" -> Row(null, null))) :: Nil ) - checkAnswer( - sql("select map['a'].field1, map['c'].field2 from jsonWithComplexMap"), - Row(Seq(1, 2, 3, null), null) :: - Row(null, null) :: - Row(null, 4) :: - Row(null, 3) :: - Row(null, null) :: - Row(null, null) :: Nil - ) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + checkAnswer( + sql("select `map`['a'].field1, `map`['c'].field2 from jsonWithComplexMap"), + Row(Seq(1, 2, 3, null), null) :: + Row(null, null) :: + Row(null, 4) :: + Row(null, 3) :: + Row(null, null) :: + Row(null, null) :: Nil + ) + } } test("SPARK-2096 Correctly parse dot notations") { - val jsonDF = sqlContext.read.json(complexFieldAndType2) - jsonDF.registerTempTable("jsonTable") + val jsonDF = spark.read.json(complexFieldAndType2) + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), @@ -914,8 +1000,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("SPARK-3390 Complex arrays") { - val jsonDF = sqlContext.read.json(complexFieldAndType2) - jsonDF.registerTempTable("jsonTable") + val jsonDF = spark.read.json(complexFieldAndType2) + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql( @@ -937,8 +1023,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("SPARK-3308 Read top level JSON arrays") { - val jsonDF = sqlContext.read.json(jsonArray) - jsonDF.registerTempTable("jsonTable") + val jsonDF = spark.read.json(jsonArray) + jsonDF.createOrReplaceTempView("jsonTable") checkAnswer( sql( @@ -953,63 +1039,147 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ) } - test("Corrupt records") { + test("Corrupt records: FAILFAST mode") { + // `FAILFAST` mode should throw an exception for corrupt records. + val exceptionOne = intercept[SparkException] { + spark.read + .option("mode", "FAILFAST") + .json(corruptRecords) + }.getMessage + assert(exceptionOne.contains( + "Malformed records are detected in schema inference. Parse Mode: FAILFAST.")) + + val exceptionTwo = intercept[SparkException] { + spark.read + .option("mode", "FAILFAST") + .schema("a string") + .json(corruptRecords) + .collect() + }.getMessage + assert(exceptionTwo.contains( + "Malformed records are detected in record parsing. Parse Mode: FAILFAST.")) + } + + test("Corrupt records: DROPMALFORMED mode") { + val schemaOne = StructType( + StructField("a", StringType, true) :: + StructField("b", StringType, true) :: + StructField("c", StringType, true) :: Nil) + val schemaTwo = StructType( + StructField("a", StringType, true) :: Nil) + // `DROPMALFORMED` mode should skip corrupt records + val jsonDFOne = spark.read + .option("mode", "DROPMALFORMED") + .json(corruptRecords) + checkAnswer( + jsonDFOne, + Row("str_a_4", "str_b_4", "str_c_4") :: Nil + ) + assert(jsonDFOne.schema === schemaOne) + + val jsonDFTwo = spark.read + .option("mode", "DROPMALFORMED") + .schema(schemaTwo) + .json(corruptRecords) + checkAnswer( + jsonDFTwo, + Row("str_a_4") :: Nil) + assert(jsonDFTwo.schema === schemaTwo) + } + + test("SPARK-19641: Additional corrupt records: DROPMALFORMED mode") { + val schema = new StructType().add("dummy", StringType) + // `DROPMALFORMED` mode should skip corrupt records + val jsonDF = spark.read + .option("mode", "DROPMALFORMED") + .json(additionalCorruptRecords) + checkAnswer( + jsonDF, + Row("test")) + assert(jsonDF.schema === schema) + } + + test("Corrupt records: PERMISSIVE mode, without designated column for malformed records") { + val schema = StructType( + StructField("a", StringType, true) :: + StructField("b", StringType, true) :: + StructField("c", StringType, true) :: Nil) + + val jsonDF = spark.read.schema(schema).json(corruptRecords) + + checkAnswer( + jsonDF.select($"a", $"b", $"c"), + Seq( + // Corrupted records are replaced with null + Row(null, null, null), + Row(null, null, null), + Row(null, null, null), + Row("str_a_4", "str_b_4", "str_c_4"), + Row(null, null, null)) + ) + } + + test("Corrupt records: PERMISSIVE mode, with designated column for malformed records") { // Test if we can query corrupt records. withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") { - withTempTable("jsonTable") { - val jsonDF = sqlContext.read.json(corruptRecords) - jsonDF.registerTempTable("jsonTable") - val schema = StructType( - StructField("_unparsed", StringType, true) :: + val jsonDF = spark.read.json(corruptRecords) + val schema = StructType( + StructField("_unparsed", StringType, true) :: StructField("a", StringType, true) :: StructField("b", StringType, true) :: StructField("c", StringType, true) :: Nil) - assert(schema === jsonDF.schema) - - // In HiveContext, backticks should be used to access columns starting with a underscore. - checkAnswer( - sql( - """ - |SELECT a, b, c, _unparsed - |FROM jsonTable - """.stripMargin), - Row(null, null, null, "{") :: - Row(null, null, null, """{"a":1, b:2}""") :: - Row(null, null, null, """{"a":{, b:3}""") :: - Row("str_a_4", "str_b_4", "str_c_4", null) :: - Row(null, null, null, "]") :: Nil - ) - - checkAnswer( - sql( - """ - |SELECT a, b, c - |FROM jsonTable - |WHERE _unparsed IS NULL - """.stripMargin), - Row("str_a_4", "str_b_4", "str_c_4") - ) - - checkAnswer( - sql( - """ - |SELECT _unparsed - |FROM jsonTable - |WHERE _unparsed IS NOT NULL - """.stripMargin), - Row("{") :: - Row("""{"a":1, b:2}""") :: - Row("""{"a":{, b:3}""") :: - Row("]") :: Nil - ) - } + assert(schema === jsonDF.schema) + + // In HiveContext, backticks should be used to access columns starting with a underscore. + checkAnswer( + jsonDF.select($"a", $"b", $"c", $"_unparsed"), + Row(null, null, null, "{") :: + Row(null, null, null, """{"a":1, b:2}""") :: + Row(null, null, null, """{"a":{, b:3}""") :: + Row("str_a_4", "str_b_4", "str_c_4", null) :: + Row(null, null, null, "]") :: Nil + ) + + checkAnswer( + jsonDF.filter($"_unparsed".isNull).select($"a", $"b", $"c"), + Row("str_a_4", "str_b_4", "str_c_4") + ) + + checkAnswer( + jsonDF.filter($"_unparsed".isNotNull).select($"_unparsed"), + Row("{") :: + Row("""{"a":1, b:2}""") :: + Row("""{"a":{, b:3}""") :: + Row("]") :: Nil + ) } } + test("SPARK-13953 Rename the corrupt record field via option") { + val jsonDF = spark.read + .option("columnNameOfCorruptRecord", "_malformed") + .json(corruptRecords) + val schema = StructType( + StructField("_malformed", StringType, true) :: + StructField("a", StringType, true) :: + StructField("b", StringType, true) :: + StructField("c", StringType, true) :: Nil) + + assert(schema === jsonDF.schema) + checkAnswer( + jsonDF.selectExpr("a", "b", "c", "_malformed"), + Row(null, null, null, "{") :: + Row(null, null, null, """{"a":1, b:2}""") :: + Row(null, null, null, """{"a":{, b:3}""") :: + Row("str_a_4", "str_b_4", "str_c_4", null) :: + Row(null, null, null, "]") :: Nil + ) + } + test("SPARK-4068: nulls in arrays") { - val jsonDF = sqlContext.read.json(nullsInArrays) - jsonDF.registerTempTable("jsonTable") + val jsonDF = spark.read.json(nullsInArrays) + jsonDF.createOrReplaceTempView("jsonTable") val schema = StructType( StructField("field1", @@ -1054,8 +1224,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { Row(values(0).toInt, values(1), values(2).toBoolean, r.split(",").toList, v5) } - val df1 = sqlContext.createDataFrame(rowRDD1, schema1) - df1.registerTempTable("applySchema1") + val df1 = spark.createDataFrame(rowRDD1, schema1) + df1.createOrReplaceTempView("applySchema1") val df2 = df1.toDF val result = df2.toJSON.collect() // scalastyle:off @@ -1077,17 +1247,17 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { Row(Row(values(0).toInt, values(2).toBoolean), Map(values(1) -> v4)) } - val df3 = sqlContext.createDataFrame(rowRDD2, schema2) - df3.registerTempTable("applySchema2") + val df3 = spark.createDataFrame(rowRDD2, schema2) + df3.createOrReplaceTempView("applySchema2") val df4 = df3.toDF 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}}") - val jsonDF = sqlContext.read.json(primitiveFieldAndType) - val primTable = sqlContext.read.json(jsonDF.toJSON) - primTable.registerTempTable("primitiveTable") + val jsonDF = spark.read.json(primitiveFieldAndType) + val primTable = spark.read.json(jsonDF.toJSON) + primTable.createOrReplaceTempView("primitiveTable") checkAnswer( sql("select * from primitiveTable"), Row(new java.math.BigDecimal("92233720368547758070"), @@ -1098,9 +1268,9 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { "this is a simple string.") ) - val complexJsonDF = sqlContext.read.json(complexFieldAndType1) - val compTable = sqlContext.read.json(complexJsonDF.toJSON) - compTable.registerTempTable("complexTable") + val complexJsonDF = spark.read.json(complexFieldAndType1) + val compTable = spark.read.json(complexJsonDF.toJSON) + compTable.createOrReplaceTempView("complexTable") // Access elements of a primitive array. checkAnswer( sql("select arrayOfString[0], arrayOfString[1], arrayOfString[2] from complexTable"), @@ -1162,77 +1332,46 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { ) } - test("JSONRelation equality test") { - val relation0 = new JSONRelation( - Some(empty), - 1.0, - false, - Some(StructType(StructField("a", IntegerType, true) :: Nil)), - None, None)(sqlContext) - val logicalRelation0 = LogicalRelation(relation0) - val relation1 = new JSONRelation( - Some(singleRow), - 1.0, - false, - Some(StructType(StructField("a", IntegerType, true) :: Nil)), - None, None)(sqlContext) - val logicalRelation1 = LogicalRelation(relation1) - val relation2 = new JSONRelation( - Some(singleRow), - 0.5, - false, - Some(StructType(StructField("a", IntegerType, true) :: Nil)), - None, None)(sqlContext) - val logicalRelation2 = LogicalRelation(relation2) - val relation3 = new JSONRelation( - Some(singleRow), - 1.0, - false, - Some(StructType(StructField("b", IntegerType, true) :: Nil)), - None, None)(sqlContext) - val logicalRelation3 = LogicalRelation(relation3) - - assert(relation0 !== relation1) - assert(!logicalRelation0.sameResult(logicalRelation1), - s"$logicalRelation0 and $logicalRelation1 should be considered not having the same result.") - - assert(relation1 === relation2) - assert(logicalRelation1.sameResult(logicalRelation2), - s"$logicalRelation1 and $logicalRelation2 should be considered having the same result.") - - assert(relation1 !== relation3) - assert(!logicalRelation1.sameResult(logicalRelation3), - s"$logicalRelation1 and $logicalRelation3 should be considered not having the same result.") - - assert(relation2 !== relation3) - assert(!logicalRelation2.sameResult(logicalRelation3), - s"$logicalRelation2 and $logicalRelation3 should be considered not having the same result.") + test("Dataset toJSON doesn't construct rdd") { + val containsRDD = spark.emptyDataFrame.toJSON.queryExecution.logical.find { + case ExternalRDD(_, _) => true + case _ => false + } + assert(containsRDD.isEmpty, "Expected logical plan of toJSON to not contain an RDD") + } + + test("JSONRelation equality test") { withTempPath(dir => { val path = dir.getCanonicalFile.toURI.toString sparkContext.parallelize(1 to 100) .map(i => s"""{"a": 1, "b": "str$i"}""").saveAsTextFile(path) - val d1 = ResolvedDataSource( - sqlContext, + val d1 = DataSource( + spark, userSpecifiedSchema = None, partitionColumns = Array.empty[String], - provider = classOf[DefaultSource].getCanonicalName, - options = Map("path" -> path)) + bucketSpec = None, + className = classOf[JsonFileFormat].getCanonicalName, + options = Map("path" -> path)).resolveRelation() - val d2 = ResolvedDataSource( - sqlContext, + val d2 = DataSource( + spark, userSpecifiedSchema = None, partitionColumns = Array.empty[String], - provider = classOf[DefaultSource].getCanonicalName, - options = Map("path" -> path)) + bucketSpec = None, + className = classOf[JsonFileFormat].getCanonicalName, + options = Map("path" -> path)).resolveRelation() assert(d1 === d2) }) } - test("SPARK-6245 JsonRDD.inferSchema on empty RDD") { + test("SPARK-6245 JsonInferSchema.infer on empty RDD") { // This is really a test that it doesn't throw an exception - val emptySchema = InferSchema(empty, 1.0, "") + val emptySchema = JsonInferSchema.infer( + empty.rdd, + new JSONOptions(Map.empty[String, String], "GMT"), + CreateJacksonParser.string) assert(StructType(Seq()) === emptySchema) } @@ -1241,22 +1380,25 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { withTempDir { dir => val schemaWithSimpleMap = StructType( StructField("map", MapType(StringType, IntegerType, true), false) :: Nil) - val df = sqlContext.read.schema(schemaWithSimpleMap).json(mapType1) + val df = spark.read.schema(schemaWithSimpleMap).json(mapType1) val path = dir.getAbsolutePath df.write.mode("overwrite").parquet(path) // order of MapType is not defined - assert(sqlContext.read.parquet(path).count() == 5) + assert(spark.read.parquet(path).count() == 5) - val df2 = sqlContext.read.json(corruptRecords) + val df2 = spark.read.json(corruptRecords) df2.write.mode("overwrite").parquet(path) - checkAnswer(sqlContext.read.parquet(path), df2.collect()) + checkAnswer(spark.read.parquet(path), df2.collect()) } } } test("SPARK-8093 Erase empty structs") { - val emptySchema = InferSchema(emptyRecords, 1.0, "") + val emptySchema = JsonInferSchema.infer( + emptyRecords.rdd, + new JSONOptions(Map.empty[String, String], "GMT"), + CreateJacksonParser.string) assert(StructType(Seq()) === emptySchema) } @@ -1283,7 +1425,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { "col1", "abd") - sqlContext.read.json(root.getAbsolutePath).registerTempTable("test_myjson_with_part") + spark.read.json(root.getAbsolutePath).createOrReplaceTempView("test_myjson_with_part") checkAnswer(sql( "SELECT count(a) FROM test_myjson_with_part where d1 = 1 and col1='abc'"), Row(4)) checkAnswer(sql( @@ -1317,7 +1459,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), DateType, TimestampType, ArrayType(IntegerType), MapType(StringType, LongType), struct, - new MyDenseVectorUDT()) + new UDT.MyDenseVectorUDT()) val fields = dataTypes.zipWithIndex.map { case (dataType, index) => StructField(s"col$index", dataType, nullable = true) } @@ -1325,7 +1467,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val constantValues = Seq( - "a string in binary".getBytes("UTF-8"), + "a string in binary".getBytes(StandardCharsets.UTF_8), null, true, 1.toByte, @@ -1341,9 +1483,9 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { Seq(2, 3, 4), Map("a string" -> 2000L), Row(4.75.toFloat, Seq(false, true)), - new MyDenseVector(Array(0.25, 2.25, 4.25))) + new UDT.MyDenseVector(Array(0.25, 2.25, 4.25))) val data = - Row.fromSeq(Seq("Spark " + sqlContext.sparkContext.version) ++ constantValues) :: Nil + Row.fromSeq(Seq("Spark " + spark.sparkContext.version) ++ constantValues) :: Nil // Data generated by previous versions. // scalastyle:off @@ -1358,7 +1500,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // scalastyle:on // Generate data for the current version. - val df = sqlContext.createDataFrame(sqlContext.sparkContext.parallelize(data, 1), schema) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data, 1), schema) withTempPath { path => df.write.format("json").mode("overwrite").save(path.getCanonicalPath) @@ -1382,15 +1524,543 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { "Spark 1.4.1", "Spark 1.5.0", "Spark 1.5.0", - "Spark " + sqlContext.sparkContext.version, - "Spark " + sqlContext.sparkContext.version) + "Spark " + spark.sparkContext.version, + "Spark " + spark.sparkContext.version) val expectedResult = col0Values.map { v => Row.fromSeq(Seq(v) ++ constantValues) } checkAnswer( - sqlContext.read.format("json").schema(schema).load(path.getCanonicalPath), + spark.read.format("json").schema(schema).load(path.getCanonicalPath), expectedResult ) } } + + test("SPARK-11544 test pathfilter") { + withTempPath { dir => + val path = dir.getCanonicalPath + + val df = spark.range(2) + df.write.json(path + "/p=1") + df.write.json(path + "/p=2") + assert(spark.read.json(path).count() === 4) + + val extraOptions = Map( + "mapred.input.pathFilter.class" -> classOf[TestFileFilter].getName, + "mapreduce.input.pathFilter.class" -> classOf[TestFileFilter].getName + ) + assert(spark.read.options(extraOptions).json(path).count() === 2) + } + } + + test("SPARK-12057 additional corrupt records do not throw exceptions") { + // Test if we can query corrupt records. + withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") { + withTempView("jsonTable") { + val schema = StructType( + StructField("_unparsed", StringType, true) :: + StructField("dummy", StringType, true) :: Nil) + + { + // We need to make sure we can infer the schema. + val jsonDF = spark.read.json(additionalCorruptRecords) + assert(jsonDF.schema === schema) + } + + { + val jsonDF = spark.read.schema(schema).json(additionalCorruptRecords) + jsonDF.createOrReplaceTempView("jsonTable") + + // In HiveContext, backticks should be used to access columns starting with a underscore. + checkAnswer( + sql( + """ + |SELECT dummy, _unparsed + |FROM jsonTable + """.stripMargin), + Row("test", null) :: + Row(null, """[1,2,3]""") :: + Row(null, """":"test", "a":1}""") :: + Row(null, """42""") :: + Row(null, """ ","ian":"test"}""") :: Nil + ) + } + } + } + } + + test("Parse JSON rows having an array type and a struct type in the same field.") { + withTempDir { dir => + val dir = Utils.createTempDir() + dir.delete() + val path = dir.getCanonicalPath + arrayAndStructRecords.map(record => record.replaceAll("\n", " ")).write.text(path) + + val schema = + StructType( + StructField("a", StructType( + StructField("b", StringType) :: Nil + )) :: Nil) + val jsonDF = spark.read.schema(schema).json(path) + assert(jsonDF.count() == 2) + } + } + + test("SPARK-12872 Support to specify the option for compression codec") { + withTempDir { dir => + val dir = Utils.createTempDir() + dir.delete() + val path = dir.getCanonicalPath + primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).write.text(path) + + val jsonDF = spark.read.json(path) + val jsonDir = new File(dir, "json").getCanonicalPath + jsonDF.coalesce(1).write + .format("json") + .option("compression", "gZiP") + .save(jsonDir) + + val compressedFiles = new File(jsonDir).listFiles() + assert(compressedFiles.exists(_.getName.endsWith(".json.gz"))) + + val jsonCopy = spark.read + .format("json") + .load(jsonDir) + + assert(jsonCopy.count == jsonDF.count) + val jsonCopySome = jsonCopy.selectExpr("string", "long", "boolean") + val jsonDFSome = jsonDF.selectExpr("string", "long", "boolean") + checkAnswer(jsonCopySome, jsonDFSome) + } + } + + test("SPARK-13543 Write the output as uncompressed via option()") { + val extraOptions = Map[String, String]( + "mapreduce.output.fileoutputformat.compress" -> "true", + "mapreduce.output.fileoutputformat.compress.type" -> CompressionType.BLOCK.toString, + "mapreduce.output.fileoutputformat.compress.codec" -> classOf[GzipCodec].getName, + "mapreduce.map.output.compress" -> "true", + "mapreduce.map.output.compress.codec" -> classOf[GzipCodec].getName + ) + withTempDir { dir => + val dir = Utils.createTempDir() + dir.delete() + + val path = dir.getCanonicalPath + primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).write.text(path) + + val jsonDF = spark.read.json(path) + val jsonDir = new File(dir, "json").getCanonicalPath + jsonDF.coalesce(1).write + .format("json") + .option("compression", "none") + .options(extraOptions) + .save(jsonDir) + + val compressedFiles = new File(jsonDir).listFiles() + assert(compressedFiles.exists(!_.getName.endsWith(".json.gz"))) + + val jsonCopy = spark.read + .format("json") + .options(extraOptions) + .load(jsonDir) + + assert(jsonCopy.count == jsonDF.count) + val jsonCopySome = jsonCopy.selectExpr("string", "long", "boolean") + val jsonDFSome = jsonDF.selectExpr("string", "long", "boolean") + checkAnswer(jsonCopySome, jsonDFSome) + } + } + + test("Casting long as timestamp") { + withTempView("jsonTable") { + val schema = (new StructType).add("ts", TimestampType) + val jsonDF = spark.read.schema(schema).json(timestampAsLong) + + jsonDF.createOrReplaceTempView("jsonTable") + + checkAnswer( + sql("select ts from jsonTable"), + Row(java.sql.Timestamp.valueOf("2016-01-02 03:04:05")) + ) + } + } + + test("wide nested json table") { + val nested = (1 to 100).map { i => + s""" + |"c$i": $i + """.stripMargin + }.mkString(", ") + val json = s""" + |{"a": [{$nested}], "b": [{$nested}]} + """.stripMargin + val df = spark.read.json(Seq(json).toDS()) + assert(df.schema.size === 2) + df.collect() + } + + test("Write dates correctly with dateFormat option") { + val customSchema = new StructType(Array(StructField("date", DateType, true))) + withTempDir { dir => + // With dateFormat option. + val datesWithFormatPath = s"${dir.getCanonicalPath}/datesWithFormat.json" + val datesWithFormat = spark.read + .schema(customSchema) + .option("dateFormat", "dd/MM/yyyy HH:mm") + .json(datesRecords) + + datesWithFormat.write + .format("json") + .option("dateFormat", "yyyy/MM/dd") + .save(datesWithFormatPath) + + // This will load back the dates as string. + val stringSchema = StructType(StructField("date", StringType, true) :: Nil) + val stringDatesWithFormat = spark.read + .schema(stringSchema) + .json(datesWithFormatPath) + val expectedStringDatesWithFormat = Seq( + Row("2015/08/26"), + Row("2014/10/27"), + Row("2016/01/28")) + + checkAnswer(stringDatesWithFormat, expectedStringDatesWithFormat) + } + } + + test("Write timestamps correctly with timestampFormat option") { + val customSchema = new StructType(Array(StructField("date", TimestampType, true))) + withTempDir { dir => + // With dateFormat option. + val timestampsWithFormatPath = s"${dir.getCanonicalPath}/timestampsWithFormat.json" + val timestampsWithFormat = spark.read + .schema(customSchema) + .option("timestampFormat", "dd/MM/yyyy HH:mm") + .json(datesRecords) + timestampsWithFormat.write + .format("json") + .option("timestampFormat", "yyyy/MM/dd HH:mm") + .save(timestampsWithFormatPath) + + // This will load back the timestamps as string. + val stringSchema = StructType(StructField("date", StringType, true) :: Nil) + val stringTimestampsWithFormat = spark.read + .schema(stringSchema) + .json(timestampsWithFormatPath) + val expectedStringDatesWithFormat = Seq( + Row("2015/08/26 18:00"), + Row("2014/10/27 18:30"), + Row("2016/01/28 20:00")) + + checkAnswer(stringTimestampsWithFormat, expectedStringDatesWithFormat) + } + } + + test("Write timestamps correctly with timestampFormat option and timeZone option") { + val customSchema = new StructType(Array(StructField("date", TimestampType, true))) + withTempDir { dir => + // With dateFormat option and timeZone option. + val timestampsWithFormatPath = s"${dir.getCanonicalPath}/timestampsWithFormat.json" + val timestampsWithFormat = spark.read + .schema(customSchema) + .option("timestampFormat", "dd/MM/yyyy HH:mm") + .json(datesRecords) + timestampsWithFormat.write + .format("json") + .option("timestampFormat", "yyyy/MM/dd HH:mm") + .option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + .save(timestampsWithFormatPath) + + // This will load back the timestamps as string. + val stringSchema = StructType(StructField("date", StringType, true) :: Nil) + val stringTimestampsWithFormat = spark.read + .schema(stringSchema) + .json(timestampsWithFormatPath) + val expectedStringDatesWithFormat = Seq( + Row("2015/08/27 01:00"), + Row("2014/10/28 01:30"), + Row("2016/01/29 04:00")) + + checkAnswer(stringTimestampsWithFormat, expectedStringDatesWithFormat) + + val readBack = spark.read + .schema(customSchema) + .option("timestampFormat", "yyyy/MM/dd HH:mm") + .option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + .json(timestampsWithFormatPath) + + checkAnswer(readBack, timestampsWithFormat) + } + } + + test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") { + val records = Seq("""{"a": 3, "b": 1.1}""", """{"a": 3.1, "b": 0.000001}""").toDS() + + val schema = StructType( + StructField("a", DecimalType(21, 1), true) :: + StructField("b", DecimalType(7, 6), true) :: Nil) + + val df1 = spark.read.option("prefersDecimal", "true").json(records) + assert(df1.schema == schema) + val df2 = spark.read.option("PREfersdecimaL", "true").json(records) + assert(df2.schema == schema) + } + + test("SPARK-18352: Parse normal multi-line JSON files (compressed)") { + withTempPath { dir => + val path = dir.getCanonicalPath + primitiveFieldAndType + .toDF("value") + .write + .option("compression", "GzIp") + .text(path) + + assert(new File(path).listFiles().exists(_.getName.endsWith(".gz"))) + + val jsonDF = spark.read.option("multiLine", true).json(path) + val jsonDir = new File(dir, "json").getCanonicalPath + jsonDF.coalesce(1).write + .option("compression", "gZiP") + .json(jsonDir) + + assert(new File(jsonDir).listFiles().exists(_.getName.endsWith(".json.gz"))) + + val originalData = spark.read.json(primitiveFieldAndType) + checkAnswer(jsonDF, originalData) + checkAnswer(spark.read.schema(originalData.schema).json(jsonDir), originalData) + } + } + + test("SPARK-18352: Parse normal multi-line JSON files (uncompressed)") { + withTempPath { dir => + val path = dir.getCanonicalPath + primitiveFieldAndType + .toDF("value") + .write + .text(path) + + val jsonDF = spark.read.option("multiLine", true).json(path) + val jsonDir = new File(dir, "json").getCanonicalPath + jsonDF.coalesce(1).write.json(jsonDir) + + val compressedFiles = new File(jsonDir).listFiles() + assert(compressedFiles.exists(_.getName.endsWith(".json"))) + + val originalData = spark.read.json(primitiveFieldAndType) + checkAnswer(jsonDF, originalData) + checkAnswer(spark.read.schema(originalData.schema).json(jsonDir), originalData) + } + } + + test("SPARK-18352: Expect one JSON document per file") { + // the json parser terminates as soon as it sees a matching END_OBJECT or END_ARRAY token. + // this might not be the optimal behavior but this test verifies that only the first value + // is parsed and the rest are discarded. + + // alternatively the parser could continue parsing following objects, which may further reduce + // allocations by skipping the line reader entirely + + withTempPath { dir => + val path = dir.getCanonicalPath + spark + .createDataFrame(Seq(Tuple1("{}{invalid}"))) + .coalesce(1) + .write + .text(path) + + val jsonDF = spark.read.option("multiLine", true).json(path) + // no corrupt record column should be created + assert(jsonDF.schema === StructType(Seq())) + // only the first object should be read + assert(jsonDF.count() === 1) + } + } + + test("SPARK-18352: Handle multi-line corrupt documents (PERMISSIVE)") { + withTempPath { dir => + val path = dir.getCanonicalPath + val corruptRecordCount = additionalCorruptRecords.count().toInt + assert(corruptRecordCount === 5) + + additionalCorruptRecords + .toDF("value") + // this is the minimum partition count that avoids hash collisions + .repartition(corruptRecordCount * 4, F.hash($"value")) + .write + .text(path) + + val jsonDF = spark.read.option("multiLine", true).option("mode", "PERMISSIVE").json(path) + assert(jsonDF.count() === corruptRecordCount) + assert(jsonDF.schema === new StructType() + .add("_corrupt_record", StringType) + .add("dummy", StringType)) + val counts = jsonDF + .join( + additionalCorruptRecords.toDF("value"), + F.regexp_replace($"_corrupt_record", "(^\\s+|\\s+$)", "") === F.trim($"value"), + "outer") + .agg( + F.count($"dummy").as("valid"), + F.count($"_corrupt_record").as("corrupt"), + F.count("*").as("count")) + checkAnswer(counts, Row(1, 4, 6)) + } + } + + test("SPARK-19641: Handle multi-line corrupt documents (DROPMALFORMED)") { + withTempPath { dir => + val path = dir.getCanonicalPath + val corruptRecordCount = additionalCorruptRecords.count().toInt + assert(corruptRecordCount === 5) + + additionalCorruptRecords + .toDF("value") + // this is the minimum partition count that avoids hash collisions + .repartition(corruptRecordCount * 4, F.hash($"value")) + .write + .text(path) + + val jsonDF = spark.read.option("multiLine", true).option("mode", "DROPMALFORMED").json(path) + checkAnswer(jsonDF, Seq(Row("test"))) + } + } + + test("SPARK-18352: Handle multi-line corrupt documents (FAILFAST)") { + withTempPath { dir => + val path = dir.getCanonicalPath + val corruptRecordCount = additionalCorruptRecords.count().toInt + assert(corruptRecordCount === 5) + + additionalCorruptRecords + .toDF("value") + // this is the minimum partition count that avoids hash collisions + .repartition(corruptRecordCount * 4, F.hash($"value")) + .write + .text(path) + + val schema = new StructType().add("dummy", StringType) + + // `FAILFAST` mode should throw an exception for corrupt records. + val exceptionOne = intercept[SparkException] { + spark.read + .option("multiLine", true) + .option("mode", "FAILFAST") + .json(path) + } + assert(exceptionOne.getMessage.contains("Malformed records are detected in schema " + + "inference. Parse Mode: FAILFAST.")) + + val exceptionTwo = intercept[SparkException] { + spark.read + .option("multiLine", true) + .option("mode", "FAILFAST") + .schema(schema) + .json(path) + .collect() + } + assert(exceptionTwo.getMessage.contains("Malformed records are detected in record " + + "parsing. Parse Mode: FAILFAST.")) + } + } + + test("Throw an exception if a `columnNameOfCorruptRecord` field violates requirements") { + val columnNameOfCorruptRecord = "_unparsed" + val schema = StructType( + StructField(columnNameOfCorruptRecord, IntegerType, true) :: + StructField("a", StringType, true) :: + StructField("b", StringType, true) :: + StructField("c", StringType, true) :: Nil) + val errMsg = intercept[AnalysisException] { + spark.read + .option("mode", "Permissive") + .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) + .schema(schema) + .json(corruptRecords) + }.getMessage + assert(errMsg.startsWith("The field for corrupt records must be string type and nullable")) + + // We use `PERMISSIVE` mode by default if invalid string is given. + withTempPath { dir => + val path = dir.getCanonicalPath + corruptRecords.toDF("value").write.text(path) + val errMsg = intercept[AnalysisException] { + spark.read + .option("mode", "permm") + .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) + .schema(schema) + .json(path) + .collect + }.getMessage + assert(errMsg.startsWith("The field for corrupt records must be string type and nullable")) + } + } + + test("SPARK-18772: Parse special floats correctly") { + val jsons = Seq( + """{"a": "NaN"}""", + """{"a": "Infinity"}""", + """{"a": "-Infinity"}""") + + // positive cases + val checks: Seq[Double => Boolean] = Seq( + _.isNaN, + _.isPosInfinity, + _.isNegInfinity) + + Seq(FloatType, DoubleType).foreach { dt => + jsons.zip(checks).foreach { case (json, check) => + val ds = spark.read + .schema(StructType(Seq(StructField("a", dt)))) + .json(Seq(json).toDS()) + .select($"a".cast(DoubleType)).as[Double] + assert(check(ds.first())) + } + } + + // negative cases + Seq(FloatType, DoubleType).foreach { dt => + val lowerCasedJsons = jsons.map(_.toLowerCase(Locale.ROOT)) + // The special floats are case-sensitive so these cases below throw exceptions. + lowerCasedJsons.foreach { lowerCasedJson => + val e = intercept[SparkException] { + spark.read + .option("mode", "FAILFAST") + .schema(StructType(Seq(StructField("a", dt)))) + .json(Seq(lowerCasedJson).toDS()) + .collect() + } + assert(e.getMessage.contains("Cannot parse")) + } + } + } + + test("SPARK-21610: Corrupt records are not handled properly when creating a dataframe " + + "from a file") { + withTempPath { dir => + val path = dir.getCanonicalPath + val data = + """{"field": 1} + |{"field": 2} + |{"field": "3"}""".stripMargin + Seq(data).toDF().repartition(1).write.text(path) + val schema = new StructType().add("field", ByteType).add("_corrupt_record", StringType) + // negative cases + val msg = intercept[AnalysisException] { + spark.read.schema(schema).json(path).select("_corrupt_record").collect() + }.getMessage + assert(msg.contains("only include the internal corrupt record column")) + intercept[catalyst.errors.TreeNodeException[_]] { + spark.read.schema(schema).json(path).filter($"_corrupt_record".isNotNull).count() + } + // workaround + val df = spark.read.schema(schema).json(path).cache() + assert(df.filter($"_corrupt_record".isNotNull).count() == 1) + assert(df.filter($"_corrupt_record".isNull).count() == 2) + checkAnswer( + df.select("_corrupt_record"), + Row(null) :: Row(null) :: Row("{\"field\": \"3\"}") :: Nil + ) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala index 713d1da1cb51..13084ba4a7f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala @@ -17,14 +17,13 @@ package org.apache.spark.sql.execution.datasources.json -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{Dataset, Encoders, SparkSession} private[json] trait TestJsonData { - protected def sqlContext: SQLContext + protected def spark: SparkSession - def primitiveFieldAndType: RDD[String] = - sqlContext.sparkContext.parallelize( + def primitiveFieldAndType: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( """{"string":"this is a simple string.", "integer":10, "long":21474836470, @@ -32,10 +31,10 @@ private[json] trait TestJsonData { "double":1.7976931348623157E308, "boolean":true, "null":null - }""" :: Nil) + }""" :: Nil))(Encoders.STRING) - def primitiveFieldValueTypeConflict: RDD[String] = - sqlContext.sparkContext.parallelize( + def primitiveFieldValueTypeConflict: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( """{"num_num_1":11, "num_num_2":null, "num_num_3": 1.1, "num_bool":true, "num_str":13.1, "str_bool":"str1"}""" :: """{"num_num_1":null, "num_num_2":21474836470.9, "num_num_3": null, @@ -44,16 +43,17 @@ private[json] trait TestJsonData { "num_bool":false, "num_str":"str1", "str_bool":false}""" :: """{"num_num_1":21474836570, "num_num_2":1.1, "num_num_3": 21474836470, "num_bool":null, "num_str":92233720368547758070, "str_bool":null}""" :: Nil) + )(Encoders.STRING) - def jsonNullStruct: RDD[String] = - sqlContext.sparkContext.parallelize( + def jsonNullStruct: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( """{"nullstr":"","ip":"27.31.100.29","headers":{"Host":"1.abc.com","Charset":"UTF-8"}}""" :: """{"nullstr":"","ip":"27.31.100.29","headers":{}}""" :: """{"nullstr":"","ip":"27.31.100.29","headers":""}""" :: - """{"nullstr":null,"ip":"27.31.100.29","headers":null}""" :: Nil) + """{"nullstr":null,"ip":"27.31.100.29","headers":null}""" :: Nil))(Encoders.STRING) - def complexFieldValueTypeConflict: RDD[String] = - sqlContext.sparkContext.parallelize( + def complexFieldValueTypeConflict: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( """{"num_struct":11, "str_array":[1, 2, 3], "array":[], "struct_array":[], "struct": {}}""" :: """{"num_struct":{"field":false}, "str_array":null, @@ -62,24 +62,25 @@ private[json] trait TestJsonData { "array":[4, 5, 6], "struct_array":[7, 8, 9], "struct": {"field":null}}""" :: """{"num_struct":{}, "str_array":["str1", "str2", 33], "array":[7], "struct_array":{"field": true}, "struct": {"field": "str"}}""" :: Nil) + )(Encoders.STRING) - def arrayElementTypeConflict: RDD[String] = - sqlContext.sparkContext.parallelize( + def arrayElementTypeConflict: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( """{"array1": [1, 1.1, true, null, [], {}, [2,3,4], {"field":"str"}], "array2": [{"field":214748364700}, {"field":1}]}""" :: """{"array3": [{"field":"str"}, {"field":1}]}""" :: - """{"array3": [1, 2, 3]}""" :: Nil) + """{"array3": [1, 2, 3]}""" :: Nil))(Encoders.STRING) - def missingFields: RDD[String] = - sqlContext.sparkContext.parallelize( + def missingFields: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( """{"a":true}""" :: """{"b":21474836470}""" :: """{"c":[33, 44]}""" :: """{"d":{"field":true}}""" :: - """{"e":"str"}""" :: Nil) + """{"e":"str"}""" :: Nil))(Encoders.STRING) - def complexFieldAndType1: RDD[String] = - sqlContext.sparkContext.parallelize( + def complexFieldAndType1: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( """{"struct":{"field1": true, "field2": 92233720368547758070}, "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, "arrayOfString":["str1", "str2"], @@ -92,10 +93,10 @@ private[json] trait TestJsonData { "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] - }""" :: Nil) + }""" :: Nil))(Encoders.STRING) - def complexFieldAndType2: RDD[String] = - sqlContext.sparkContext.parallelize( + def complexFieldAndType2: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( """{"arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], "complexArrayOfStruct": [ { @@ -146,59 +147,90 @@ private[json] trait TestJsonData { {"inner3": [[{"inner4": 2}]]} ] ]] - }""" :: Nil) + }""" :: Nil))(Encoders.STRING) - def mapType1: RDD[String] = - sqlContext.sparkContext.parallelize( + def mapType1: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( """{"map": {"a": 1}}""" :: """{"map": {"b": 2}}""" :: """{"map": {"c": 3}}""" :: """{"map": {"c": 1, "d": 4}}""" :: - """{"map": {"e": null}}""" :: Nil) + """{"map": {"e": null}}""" :: Nil))(Encoders.STRING) - def mapType2: RDD[String] = - sqlContext.sparkContext.parallelize( + def mapType2: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( """{"map": {"a": {"field1": [1, 2, 3, null]}}}""" :: """{"map": {"b": {"field2": 2}}}""" :: """{"map": {"c": {"field1": [], "field2": 4}}}""" :: """{"map": {"c": {"field2": 3}, "d": {"field1": [null]}}}""" :: """{"map": {"e": null}}""" :: - """{"map": {"f": {"field1": null}}}""" :: Nil) + """{"map": {"f": {"field1": null}}}""" :: Nil))(Encoders.STRING) - def nullsInArrays: RDD[String] = - sqlContext.sparkContext.parallelize( + def nullsInArrays: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( """{"field1":[[null], [[["Test"]]]]}""" :: """{"field2":[null, [{"Test":1}]]}""" :: """{"field3":[[null], [{"Test":"2"}]]}""" :: - """{"field4":[[null, [1,2,3]]]}""" :: Nil) + """{"field4":[[null, [1,2,3]]]}""" :: Nil))(Encoders.STRING) - def jsonArray: RDD[String] = - sqlContext.sparkContext.parallelize( + def jsonArray: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( """[{"a":"str_a_1"}]""" :: """[{"a":"str_a_2"}, {"b":"str_b_3"}]""" :: """{"b":"str_b_4", "a":"str_a_4", "c":"str_c_4"}""" :: - """[]""" :: Nil) + """[]""" :: Nil))(Encoders.STRING) - def corruptRecords: RDD[String] = - sqlContext.sparkContext.parallelize( + def corruptRecords: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( """{""" :: """""" :: """{"a":1, b:2}""" :: """{"a":{, b:3}""" :: """{"b":"str_b_4", "a":"str_a_4", "c":"str_c_4"}""" :: - """]""" :: Nil) - - def emptyRecords: RDD[String] = - sqlContext.sparkContext.parallelize( + """]""" :: Nil))(Encoders.STRING) + + def additionalCorruptRecords: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( + """{"dummy":"test"}""" :: + """[1,2,3]""" :: + """":"test", "a":1}""" :: + """42""" :: + """ ","ian":"test"}""" :: Nil))(Encoders.STRING) + + def emptyRecords: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( """{""" :: """""" :: """{"a": {}}""" :: """{"a": {"b": {}}}""" :: """{"b": [{"c": {}}]}""" :: - """]""" :: Nil) + """]""" :: Nil))(Encoders.STRING) + + def timestampAsLong: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( + """{"ts":1451732645}""" :: Nil))(Encoders.STRING) + + def arrayAndStructRecords: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( + """{"a": {"b": 1}}""" :: + """{"a": []}""" :: Nil))(Encoders.STRING) + + def floatingValueRecords: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( + s"""{"a": 0.${"0" * 38}1, "b": 0.01}""" :: Nil))(Encoders.STRING) + + def bigIntegerRecords: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( + s"""{"a": 1${"0" * 38}, "b": 92233720368547758070}""" :: Nil))(Encoders.STRING) + def datesRecords: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize( + """{"date": "26/08/2015 18:00"}""" :: + """{"date": "27/10/2014 18:30"}""" :: + """{"date": "28/01/2016 20:00"}""" :: Nil))(Encoders.STRING) - lazy val singleRow: RDD[String] = sqlContext.sparkContext.parallelize("""{"a":123}""" :: Nil) + lazy val singleRow: Dataset[String] = + spark.createDataset(spark.sparkContext.parallelize("""{"a":123}""" :: Nil))(Encoders.STRING) - def empty: RDD[String] = sqlContext.sparkContext.parallelize(Seq[String]()) + def empty: Dataset[String] = spark.emptyDataset(Encoders.STRING) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala index 36b929ee1f40..1b99fbedca04 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.execution.datasources.parquet -import java.io.File import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets import java.util.{List => JList, Map => JMap} import scala.collection.JavaConverters._ @@ -27,6 +27,7 @@ import org.apache.avro.Schema import org.apache.avro.generic.IndexedRecord import org.apache.hadoop.fs.Path import org.apache.parquet.avro.AvroParquetWriter +import org.apache.parquet.hadoop.ParquetWriter import org.apache.spark.sql.Row import org.apache.spark.sql.execution.datasources.parquet.test.avro._ @@ -35,14 +36,14 @@ import org.apache.spark.sql.test.SharedSQLContext class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with SharedSQLContext { private def withWriter[T <: IndexedRecord] (path: String, schema: Schema) - (f: AvroParquetWriter[T] => Unit): Unit = { + (f: ParquetWriter[T] => Unit): Unit = { logInfo( s"""Writing Avro records with the following Avro schema into Parquet file: | |${schema.toString(true)} """.stripMargin) - val writer = new AvroParquetWriter[T](new Path(path), schema) + val writer = AvroParquetWriter.builder[T](new Path(path)).withSchema(schema).build() try f(writer) finally writer.close() } @@ -59,7 +60,7 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared .setLongColumn(i.toLong * 10) .setFloatColumn(i.toFloat + 0.1f) .setDoubleColumn(i.toDouble + 0.2d) - .setBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes("UTF-8"))) + .setBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes(StandardCharsets.UTF_8))) .setStringColumn(s"val_$i") .build()) } @@ -67,14 +68,14 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared logParquetSchema(path) - checkAnswer(sqlContext.read.parquet(path), (0 until 10).map { i => + checkAnswer(spark.read.parquet(path), (0 until 10).map { i => Row( i % 2 == 0, i, i.toLong * 10, i.toFloat + 0.1f, i.toDouble + 0.2d, - s"val_$i".getBytes("UTF-8"), + s"val_$i".getBytes(StandardCharsets.UTF_8), s"val_$i") }) } @@ -103,7 +104,7 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared .setMaybeLongColumn(i.toLong * 10) .setMaybeFloatColumn(i.toFloat + 0.1f) .setMaybeDoubleColumn(i.toDouble + 0.2d) - .setMaybeBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes("UTF-8"))) + .setMaybeBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes(StandardCharsets.UTF_8))) .setMaybeStringColumn(s"val_$i") .build() } @@ -114,7 +115,7 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared logParquetSchema(path) - checkAnswer(sqlContext.read.parquet(path), (0 until 10).map { i => + checkAnswer(spark.read.parquet(path), (0 until 10).map { i => if (i % 3 == 0) { Row.apply(Seq.fill(7)(null): _*) } else { @@ -124,7 +125,7 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared i.toLong * 10, i.toFloat + 0.1f, i.toDouble + 0.2d, - s"val_$i".getBytes("UTF-8"), + s"val_$i".getBytes(StandardCharsets.UTF_8), s"val_$i") } }) @@ -155,7 +156,7 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared logParquetSchema(path) - checkAnswer(sqlContext.read.parquet(path), (0 until 10).map { i => + checkAnswer(spark.read.parquet(path), (0 until 10).map { i => Row( Seq.tabulate(3)(i => s"val_$i"), if (i % 3 == 0) null else Seq.tabulate(3)(identity)) @@ -182,7 +183,7 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared logParquetSchema(path) - checkAnswer(sqlContext.read.parquet(path), (0 until 10).map { i => + checkAnswer(spark.read.parquet(path), (0 until 10).map { i => Row(Seq.tabulate(3, 3)((i, j) => i * 3 + j)) }) } @@ -205,7 +206,7 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared logParquetSchema(path) - checkAnswer(sqlContext.read.parquet(path), (0 until 10).map { i => + checkAnswer(spark.read.parquet(path), (0 until 10).map { i => Row(Seq.tabulate(3)(i => i.toString -> Seq.tabulate(3)(j => i + j)).toMap) }) } @@ -221,7 +222,7 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared logParquetSchema(path) - checkAnswer(sqlContext.read.parquet(path), (0 until 10).map { i => + checkAnswer(spark.read.parquet(path), (0 until 10).map { i => Row( Seq.tabulate(3)(n => s"arr_${i + n}"), Seq.tabulate(3)(n => n.toString -> (i + n: Integer)).toMap, @@ -267,7 +268,7 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared } } - checkAnswer(sqlContext.read.parquet(path).filter('suit === "SPADES"), Row("SPADES")) + checkAnswer(spark.read.parquet(path).filter('suit === "SPADES"), Row("SPADES")) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala index 0835bd123049..a43a856d16ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala @@ -21,9 +21,9 @@ import scala.collection.JavaConverters.{collectionAsScalaIterableConverter, mapA import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, PathFilter} +import org.apache.parquet.hadoop.{ParquetFileReader, ParquetWriter} import org.apache.parquet.hadoop.api.WriteSupport import org.apache.parquet.hadoop.api.WriteSupport.WriteContext -import org.apache.parquet.hadoop.{ParquetFileReader, ParquetWriter} import org.apache.parquet.io.api.RecordConsumer import org.apache.parquet.schema.{MessageType, MessageTypeParser} @@ -38,14 +38,15 @@ private[sql] abstract class ParquetCompatibilityTest extends QueryTest with Parq } protected def readParquetSchema(path: String, pathFilter: Path => Boolean): MessageType = { + val hadoopConf = spark.sessionState.newHadoopConf() val fsPath = new Path(path) - val fs = fsPath.getFileSystem(hadoopConfiguration) + val fs = fsPath.getFileSystem(hadoopConf) val parquetFiles = fs.listStatus(fsPath, new PathFilter { override def accept(path: Path): Boolean = pathFilter(path) }).toSeq.asJava val footers = - ParquetFileReader.readAllFootersInParallel(hadoopConfiguration, parquetFiles, true) + ParquetFileReader.readAllFootersInParallel(hadoopConf, parquetFiles, true) footers.asScala.head.getParquetMetadata.getFileMetaData.getSchema } @@ -118,8 +119,18 @@ private[sql] object ParquetCompatibilityTest { metadata: Map[String, String], recordWriters: (RecordConsumer => Unit)*): Unit = { val messageType = MessageTypeParser.parseMessageType(schema) - val writeSupport = new DirectWriteSupport(messageType, metadata) - val parquetWriter = new ParquetWriter[RecordConsumer => Unit](new Path(path), writeSupport) + val testWriteSupport = new DirectWriteSupport(messageType, metadata) + /** + * Provide a builder for constructing a parquet writer - after PARQUET-248 directly constructing + * the writer is deprecated and should be done through a builder. The default builders include + * Avro - but for raw Parquet writing we must create our own builder. + */ + class ParquetWriterBuilder() extends + ParquetWriter.Builder[RecordConsumer => Unit, ParquetWriterBuilder](new Path(path)) { + override def getWriteSupport(conf: Configuration) = testWriteSupport + override def self() = this + } + val parquetWriter = new ParquetWriterBuilder().build() try recordWriters.foreach(parquetWriter.write) finally parquetWriter.close() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala new file mode 100644 index 000000000000..00799301ca8d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import scala.collection.JavaConverters._ + +import org.apache.parquet.hadoop.ParquetOutputFormat + +import org.apache.spark.sql.test.SharedSQLContext + +// TODO: this needs a lot more testing but it's currently not easy to test with the parquet +// writer abstractions. Revisit. +class ParquetEncodingSuite extends ParquetCompatibilityTest with SharedSQLContext { + import testImplicits._ + + val ROW = ((1).toByte, 2, 3L, "abc") + val NULL_ROW = ( + null.asInstanceOf[java.lang.Byte], + null.asInstanceOf[Integer], + null.asInstanceOf[java.lang.Long], + null.asInstanceOf[String]) + + test("All Types Dictionary") { + (1 :: 1000 :: Nil).foreach { n => { + withTempPath { dir => + List.fill(n)(ROW).toDF.repartition(1).write.parquet(dir.getCanonicalPath) + val file = SpecificParquetRecordReaderBase.listDirectory(dir).toArray.head + + val reader = new VectorizedParquetRecordReader + reader.initialize(file.asInstanceOf[String], null) + val batch = reader.resultBatch() + assert(reader.nextBatch()) + assert(batch.numRows() == n) + var i = 0 + while (i < n) { + assert(batch.column(0).getByte(i) == 1) + assert(batch.column(1).getInt(i) == 2) + assert(batch.column(2).getLong(i) == 3) + assert(batch.column(3).getUTF8String(i).toString == "abc") + i += 1 + } + reader.close() + } + }} + } + + test("All Types Null") { + (1 :: 100 :: Nil).foreach { n => { + withTempPath { dir => + val data = List.fill(n)(NULL_ROW).toDF + data.repartition(1).write.parquet(dir.getCanonicalPath) + val file = SpecificParquetRecordReaderBase.listDirectory(dir).toArray.head + + val reader = new VectorizedParquetRecordReader + reader.initialize(file.asInstanceOf[String], null) + val batch = reader.resultBatch() + assert(reader.nextBatch()) + assert(batch.numRows() == n) + var i = 0 + while (i < n) { + assert(batch.column(0).isNullAt(i)) + assert(batch.column(1).isNullAt(i)) + assert(batch.column(2).isNullAt(i)) + assert(batch.column(3).isNullAt(i)) + i += 1 + } + reader.close() + }} + } + } + + test("Read row group containing both dictionary and plain encoded pages") { + withSQLConf(ParquetOutputFormat.DICTIONARY_PAGE_SIZE -> "2048", + ParquetOutputFormat.PAGE_SIZE -> "4096") { + withTempPath { dir => + // In order to explicitly test for SPARK-14217, we set the parquet dictionary and page size + // such that the following data spans across 3 pages (within a single row group) where the + // first page is dictionary encoded and the remaining two are plain encoded. + val data = (0 until 512).flatMap(i => Seq.fill(3)(i.toString)) + data.toDF("f").coalesce(1).write.parquet(dir.getCanonicalPath) + val file = SpecificParquetRecordReaderBase.listDirectory(dir).asScala.head + + val reader = new VectorizedParquetRecordReader + reader.initialize(file, null /* set columns to null to project all columns */) + val column = reader.resultBatch().column(0) + assert(reader.nextBatch()) + + (0 until 512).foreach { i => + assert(column.getUTF8String(3 * i).toString == i.toString) + assert(column.getUTF8String(3 * i + 1).toString == i.toString) + assert(column.getUTF8String(3 * i + 2).toString == i.toString) + } + reader.close() + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala new file mode 100644 index 000000000000..ccb34355f1ba --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.hadoop.fs.{FileSystem, Path} + +import org.apache.spark.SparkException +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class ParquetFileFormatSuite extends QueryTest with ParquetTest with SharedSQLContext { + + test("read parquet footers in parallel") { + def testReadFooters(ignoreCorruptFiles: Boolean): Unit = { + withTempDir { dir => + val fs = FileSystem.get(sparkContext.hadoopConfiguration) + val basePath = dir.getCanonicalPath + + val path1 = new Path(basePath, "first") + val path2 = new Path(basePath, "second") + val path3 = new Path(basePath, "third") + + spark.range(1).toDF("a").coalesce(1).write.parquet(path1.toString) + spark.range(1, 2).toDF("a").coalesce(1).write.parquet(path2.toString) + spark.range(2, 3).toDF("a").coalesce(1).write.json(path3.toString) + + val fileStatuses = + Seq(fs.listStatus(path1), fs.listStatus(path2), fs.listStatus(path3)).flatten + + val footers = ParquetFileFormat.readParquetFootersInParallel( + sparkContext.hadoopConfiguration, fileStatuses, ignoreCorruptFiles) + + assert(footers.size == 2) + } + } + + testReadFooters(true) + val exception = intercept[java.io.IOException] { + testReadFooters(false) + } + assert(exception.getMessage().contains("Could not read footer for file")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index c24c9f025dad..90f6620d990c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -17,15 +17,22 @@ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.parquet.filter2.predicate.Operators._ +import java.nio.charset.StandardCharsets + import org.apache.parquet.filter2.predicate.{FilterPredicate, Operators} +import org.apache.parquet.filter2.predicate.FilterApi._ +import org.apache.parquet.filter2.predicate.Operators.{Column => _, _} -import org.apache.spark.sql.{Column, DataFrame, QueryTest, Row, SQLConf} +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ +import org.apache.spark.util.{AccumulatorContext, AccumulatorV2} /** * A test suite that tests Parquet filter2 API based filter pushdown optimization. @@ -40,7 +47,6 @@ import org.apache.spark.sql.test.SharedSQLContext * data type is nullable. */ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContext { - private def checkFilterPredicate( df: DataFrame, predicate: Predicate, @@ -50,27 +56,32 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex val output = predicate.collect { case a: Attribute => a }.distinct withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { - val query = df - .select(output.map(e => Column(e)): _*) - .where(Column(predicate)) - - val analyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(_: ParquetRelation, _)) => filters - }.flatten - assert(analyzedPredicate.nonEmpty) - - val selectedFilters = analyzedPredicate.flatMap(DataSourceStrategy.translateFilter) - assert(selectedFilters.nonEmpty) - - selectedFilters.foreach { pred => - val maybeFilter = ParquetFilters.createFilter(df.schema, pred) - assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pred") - maybeFilter.foreach { f => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { + case PhysicalOperation(_, filters, + LogicalRelation(relation: HadoopFsRelation, _, _, _)) => + maybeRelation = Some(relation) + filters + }.flatten.reduceLeftOption(_ && _) + assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") + + val (_, selectedFilters, _) = + DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) + assert(selectedFilters.nonEmpty, "No filter is pushed down") + + selectedFilters.foreach { pred => + val maybeFilter = ParquetFilters.createFilter(df.schema, pred) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pred") // Doesn't bother checking type parameters here (e.g. `Eq[Integer]`) - assert(f.getClass === filterClass) + maybeFilter.exists(_.getClass === filterClass) } + checker(stripSparkFilter(query), expected) } - checker(query, expected) } } @@ -91,7 +102,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex (implicit df: DataFrame): Unit = { def checkBinaryAnswer(df: DataFrame, expected: Seq[Row]) = { assertResult(expected.map(_.getAs[Array[Byte]](0).mkString(",")).sorted) { - df.map(_.getAs[Array[Byte]](0).mkString(",")).collect().toSeq.sorted + df.rdd.map(_.getAs[Array[Byte]](0).mkString(",")).collect().toSeq.sorted } } @@ -111,7 +122,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1 === true, classOf[Eq[_]], true) checkFilterPredicate('_1 <=> true, classOf[Eq[_]], true) - checkFilterPredicate('_1 !== true, classOf[NotEq[_]], false) + checkFilterPredicate('_1 =!= true, classOf[NotEq[_]], false) } } @@ -122,7 +133,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) + checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) @@ -148,7 +159,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) + checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) @@ -174,7 +185,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) + checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) @@ -200,7 +211,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1) - checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) + checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4) @@ -219,8 +230,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex } } - // See https://issues.apache.org/jira/browse/SPARK-11153 - ignore("filter pushdown - string") { + test("filter pushdown - string") { withParquetDataFrame((1 to 4).map(i => Tuple1(i.toString))) { implicit df => checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) checkFilterPredicate( @@ -229,7 +239,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1 === "1", classOf[Eq[_]], "1") checkFilterPredicate('_1 <=> "1", classOf[Eq[_]], "1") checkFilterPredicate( - '_1 !== "1", classOf[NotEq[_]], (2 to 4).map(i => Row.apply(i.toString))) + '_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") @@ -248,10 +258,9 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex } } - // See https://issues.apache.org/jira/browse/SPARK-11153 - ignore("filter pushdown - binary") { + test("filter pushdown - binary") { implicit class IntToBinary(int: Int) { - def b: Array[Byte] = int.toString.getBytes("UTF-8") + def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8) } withParquetDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => @@ -263,7 +272,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex '_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) + '_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) @@ -294,7 +303,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex // If the "part = 1" filter gets pushed down, this query will throw an exception since // "part" is not a valid column in the actual Parquet file checkAnswer( - sqlContext.read.parquet(path).filter("part = 1"), + spark.read.parquet(dir.getCanonicalPath).filter("part = 1"), (1 to 3).map(i => Row(i, i.toString, 1))) } } @@ -311,29 +320,263 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex // If the "part = 1" filter gets pushed down, this query will throw an exception since // "part" is not a valid column in the actual Parquet file checkAnswer( - sqlContext.read.parquet(path).filter("a > 0 and (part = 0 or a > 1)"), + spark.read.parquet(dir.getCanonicalPath).filter("a > 0 and (part = 0 or a > 1)"), (2 to 3).map(i => Row(i, i.toString, 1))) } } } - test("SPARK-11103: Filter applied on merged Parquet schema with new column fails") { + test("SPARK-12231: test the filter and empty project in partitioned DataSource scan") { import testImplicits._ - withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", - SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true") { + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { withTempPath { dir => - val pathOne = s"${dir.getCanonicalPath}/table1" - (1 to 3).map(i => (i, i.toString)).toDF("a", "b").write.parquet(pathOne) - val pathTwo = s"${dir.getCanonicalPath}/table2" - (1 to 3).map(i => (i, i.toString)).toDF("c", "b").write.parquet(pathTwo) + val path = s"${dir.getCanonicalPath}" + (1 to 3).map(i => (i, i + 1, i + 2, i + 3)).toDF("a", "b", "c", "d"). + write.partitionBy("a").parquet(path) + + // The filter "a > 1 or b < 2" will not get pushed down, and the projection is empty, + // this query will throw an exception since the project from combinedFilter expect + // two projection while the + val df1 = spark.read.parquet(dir.getCanonicalPath) + + assert(df1.filter("a > 1 or b < 2").count() == 2) + } + } + } + + test("SPARK-12231: test the new projection in partitioned DataSource scan") { + import testImplicits._ + + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}" + (1 to 3).map(i => (i, i + 1, i + 2, i + 3)).toDF("a", "b", "c", "d"). + write.partitionBy("a").parquet(path) + + // test the generate new projection case + // when projects != partitionAndNormalColumnProjs + + val df1 = spark.read.parquet(dir.getCanonicalPath) + + checkAnswer( + df1.filter("a > 1 or b > 2").orderBy("a").selectExpr("a", "b", "c", "d"), + (2 to 3).map(i => Row(i, i + 1, i + 2, i + 3))) + } + } + } + + + test("Filter applied on merged Parquet schema with new column should work") { + import testImplicits._ + Seq("true", "false").map { vectorized => + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", + SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true", + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized) { + withTempPath { dir => + val path1 = s"${dir.getCanonicalPath}/table1" + (1 to 3).map(i => (i, i.toString)).toDF("a", "b").write.parquet(path1) + val path2 = s"${dir.getCanonicalPath}/table2" + (1 to 3).map(i => (i, i.toString)).toDF("c", "b").write.parquet(path2) + + // No matter "c = 1" gets pushed down or not, this query should work without exception. + val df = spark.read.parquet(path1, path2).filter("c = 1").selectExpr("c", "b", "a") + checkAnswer( + df, + Row(1, "1", null)) + + val path3 = s"${dir.getCanonicalPath}/table3" + val dfStruct = sparkContext.parallelize(Seq((1, 1))).toDF("a", "b") + dfStruct.select(struct("a").as("s")).write.parquet(path3) + + val path4 = s"${dir.getCanonicalPath}/table4" + val dfStruct2 = sparkContext.parallelize(Seq((1, 1))).toDF("c", "b") + dfStruct2.select(struct("c").as("s")).write.parquet(path4) + + // No matter "s.c = 1" gets pushed down or not, this query should work without exception. + val dfStruct3 = spark.read.parquet(path3, path4).filter("s.c = 1") + .selectExpr("s") + checkAnswer(dfStruct3, Row(Row(null, 1))) + } + } + } + } + + // The unsafe row RecordReader does not support row by row filtering so run it with it disabled. + test("SPARK-11661 Still pushdown filters returned by unhandledFilters") { + import testImplicits._ + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/part=1" + (1 to 3).map(i => (i, i.toString)).toDF("a", "b").write.parquet(path) + val df = spark.read.parquet(path).filter("a = 2") + + // The result should be single row. + // When a filter is pushed to Parquet, Parquet can apply it to every row. + // So, we can check the number of rows returned from the Parquet + // to make sure our filter pushdown work. + assert(stripSparkFilter(df).count == 1) + } + } + } + } + + test("SPARK-12218: 'Not' is included in Parquet filter pushdown") { + import testImplicits._ + + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/table1" + (1 to 5).map(i => (i, (i % 2).toString)).toDF("a", "b").write.parquet(path) + + checkAnswer( + spark.read.parquet(path).where("not (a = 2) or not(b in ('1'))"), + (1 to 5).map(i => Row(i, (i % 2).toString))) - // If the "c = 1" filter gets pushed down, this query will throw an exception which - // Parquet emits. This is a Parquet issue (PARQUET-389). checkAnswer( - sqlContext.read.parquet(pathOne, pathTwo).filter("c = 1").selectExpr("c", "b", "a"), - (1 to 1).map(i => Row(i, i.toString, null))) + spark.read.parquet(path).where("not (a = 2 and b in ('1'))"), + (1 to 5).map(i => Row(i, (i % 2).toString))) } } } + + test("SPARK-12218 Converting conjunctions into Parquet filter predicates") { + val schema = StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", StringType, nullable = true), + StructField("c", DoubleType, nullable = true) + )) + + assertResult(Some(and( + lt(intColumn("a"), 10: Integer), + gt(doubleColumn("c"), 1.5: java.lang.Double))) + ) { + ParquetFilters.createFilter( + schema, + sources.And( + sources.LessThan("a", 10), + sources.GreaterThan("c", 1.5D))) + } + + assertResult(None) { + ParquetFilters.createFilter( + schema, + sources.And( + sources.LessThan("a", 10), + sources.StringContains("b", "prefix"))) + } + + assertResult(None) { + ParquetFilters.createFilter( + schema, + sources.Not( + sources.And( + sources.GreaterThan("a", 1), + sources.StringContains("b", "prefix")))) + } + } + + test("SPARK-16371 Do not push down filters when inner name and outer name are the same") { + withParquetDataFrame((1 to 4).map(i => Tuple1(Tuple1(i)))) { implicit df => + // Here the schema becomes as below: + // + // root + // |-- _1: struct (nullable = true) + // | |-- _1: integer (nullable = true) + // + // The inner column name, `_1` and outer column name `_1` are the same. + // Obviously this should not push down filters because the outer column is struct. + assert(df.filter("_1 IS NOT NULL").count() === 4) + } + } + + test("Fiters should be pushed down for vectorized Parquet reader at row group level") { + import testImplicits._ + + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/table" + (1 to 1024).map(i => (101, i)).toDF("a", "b").write.parquet(path) + + Seq(true, false).foreach { enablePushDown => + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> enablePushDown.toString) { + val accu = new NumRowGroupsAcc + sparkContext.register(accu) + + val df = spark.read.parquet(path).filter("a < 100") + df.foreachPartition((it: Iterator[Row]) => it.foreach(v => accu.add(0))) + df.collect + + if (enablePushDown) { + assert(accu.value == 0) + } else { + assert(accu.value > 0) + } + AccumulatorContext.remove(accu.id) + } + } + } + } + } + + test("SPARK-17213: Broken Parquet filter push-down for string columns") { + withTempPath { dir => + import testImplicits._ + + val path = dir.getCanonicalPath + // scalastyle:off nonascii + Seq("a", "é").toDF("name").write.parquet(path) + // scalastyle:on nonascii + + assert(spark.read.parquet(path).where("name > 'a'").count() == 1) + assert(spark.read.parquet(path).where("name >= 'a'").count() == 2) + + // scalastyle:off nonascii + assert(spark.read.parquet(path).where("name < 'é'").count() == 1) + assert(spark.read.parquet(path).where("name <= 'é'").count() == 2) + // scalastyle:on nonascii + } + } + + test("SPARK-20364: Disable Parquet predicate pushdown for fields having dots in the names") { + import testImplicits._ + + Seq(true, false).foreach { vectorized => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized.toString, + SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> true.toString, + SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + withTempPath { path => + Seq(Some(1), None).toDF("col.dots").write.parquet(path.getAbsolutePath) + val readBack = spark.read.parquet(path.getAbsolutePath).where("`col.dots` IS NOT NULL") + assert(readBack.count() == 1) + } + } + } + } +} + +class NumRowGroupsAcc extends AccumulatorV2[Integer, Integer] { + private var _sum = 0 + + override def isZero: Boolean = _sum == 0 + + override def copy(): AccumulatorV2[Integer, Integer] = { + val acc = new NumRowGroupsAcc() + acc._sum = _sum + acc + } + + override def reset(): Unit = _sum = 0 + + override def add(v: Integer): Unit = _sum += v + + override def merge(other: AccumulatorV2[Integer, Integer]): Unit = other match { + case a: NumRowGroupsAcc => _sum += a._sum + case _ => throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + } + + override def value: Integer = _sum } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index f14b2886a9ec..d76990b482db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -17,30 +17,36 @@ package org.apache.spark.sql.execution.datasources.parquet -import java.util.Collections +import java.util.Locale import scala.collection.JavaConverters._ +import scala.collection.mutable 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.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} -import org.apache.parquet.example.data.simple.SimpleGroup +import org.apache.parquet.column.{Encoding, ParquetProperties} import org.apache.parquet.example.data.{Group, GroupWriter} +import org.apache.parquet.example.data.simple.SimpleGroup import org.apache.parquet.hadoop._ import org.apache.parquet.hadoop.api.WriteSupport import org.apache.parquet.hadoop.api.WriteSupport.WriteContext -import org.apache.parquet.hadoop.metadata.{CompressionCodecName, FileMetaData, ParquetMetadata} +import org.apache.parquet.hadoop.metadata.CompressionCodecName import org.apache.parquet.io.api.RecordConsumer import org.apache.parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.SparkException import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.{InternalRow, ScalaReflection} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow} import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String // 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?) @@ -91,6 +97,37 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { } } + test("SPARK-11694 Parquet logical types are not being tested properly") { + val parquetSchema = MessageTypeParser.parseMessageType( + """message root { + | required int32 a(INT_8); + | required int32 b(INT_16); + | required int32 c(DATE); + | required int32 d(DECIMAL(1,0)); + | required int64 e(DECIMAL(10,0)); + | required binary f(UTF8); + | required binary g(ENUM); + | required binary h(DECIMAL(32,0)); + | required fixed_len_byte_array(32) i(DECIMAL(32,0)); + | required int64 j(TIMESTAMP_MILLIS); + |} + """.stripMargin) + + val expectedSparkTypes = Seq(ByteType, ShortType, DateType, DecimalType(1, 0), + DecimalType(10, 0), StringType, StringType, DecimalType(32, 0), DecimalType(32, 0), + TimestampType) + + withTempPath { location => + val path = new Path(location.getCanonicalPath) + val conf = spark.sessionState.newHadoopConf() + writeMetadata(parquetSchema, path, conf) + readParquetFile(path.toString)(df => { + val sparkTypes = df.schema.map(_.dataType) + assert(sparkTypes === expectedSparkTypes) + }) + } + } + 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 @@ -101,7 +138,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { testStandardAndLegacyModes("fixed-length decimals") { def makeDecimalRDD(decimal: DecimalType): DataFrame = { - sqlContext + spark .range(1000) // Parquet doesn't allow column names with spaces, have to add an alias here. // Minus 500 here so that negative decimals are also tested. @@ -114,7 +151,9 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { withTempPath { dir => val data = makeDecimalRDD(DecimalType(precision, scale)) data.write.parquet(dir.getCanonicalPath) - checkAnswer(sqlContext.read.parquet(dir.getCanonicalPath), data.collect().toSeq) + readParquetFile(dir.getCanonicalPath) { df => { + checkAnswer(df, data.collect().toSeq) + }} } } } @@ -130,7 +169,9 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { withTempPath { dir => val data = makeDateRDD() data.write.parquet(dir.getCanonicalPath) - checkAnswer(sqlContext.read.parquet(dir.getCanonicalPath), data.collect().toSeq) + readParquetFile(dir.getCanonicalPath) { df => + checkAnswer(df, data.collect().toSeq) + } } } @@ -170,7 +211,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { } testStandardAndLegacyModes("nested map with struct as value type") { - val data = (1 to 4).map(i => Tuple1(Map(i -> (i, s"val_$i")))) + val data = (1 to 4).map(i => Tuple1(Map(i -> ((i, s"val_$i"))))) withParquetDataFrame(data) { df => checkAnswer(df, data.map { case Tuple1(m) => Row(m.mapValues(struct => Row(struct.productIterator.toSeq: _*))) @@ -206,10 +247,48 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { } } + test("SPARK-10113 Support for unsigned Parquet logical types") { + val parquetSchema = MessageTypeParser.parseMessageType( + """message root { + | required int32 c(UINT_32); + |} + """.stripMargin) + + withTempPath { location => + val path = new Path(location.getCanonicalPath) + val conf = spark.sessionState.newHadoopConf() + writeMetadata(parquetSchema, path, conf) + val errorMessage = intercept[Throwable] { + spark.read.parquet(path.toString).printSchema() + }.toString + assert(errorMessage.contains("Parquet type not supported")) + } + } + + test("SPARK-11692 Support for Parquet logical types, JSON and BSON (embedded types)") { + val parquetSchema = MessageTypeParser.parseMessageType( + """message root { + | required binary a(JSON); + | required binary b(BSON); + |} + """.stripMargin) + + val expectedSparkTypes = Seq(StringType, BinaryType) + + withTempPath { location => + val path = new Path(location.getCanonicalPath) + val conf = spark.sessionState.newHadoopConf() + writeMetadata(parquetSchema, path, conf) + val sparkTypes = spark.read.parquet(path.toString).schema.map(_.dataType) + assert(sparkTypes === expectedSparkTypes) + } + } + test("compression codec") { + val hadoopConf = spark.sessionState.newHadoopConf() def compressionCodecFor(path: String, codecName: String): String = { val codecs = for { - footer <- readAllFootersWithoutSummaryFiles(new Path(path), hadoopConfiguration) + footer <- readAllFootersWithoutSummaryFiles(new Path(path), hadoopConf) block <- footer.getParquetMetadata.getBlocks.asScala column <- block.getColumns.asScala } yield column.getCodec.name() @@ -223,7 +302,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { def checkCompressionCodec(codec: CompressionCodecName): Unit = { withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> codec.name()) { withParquetFile(data) { path => - assertResult(sqlContext.conf.parquetCompressionCodec.toUpperCase) { + assertResult(spark.conf.get(SQLConf.PARQUET_COMPRESSION).toUpperCase(Locale.ROOT)) { compressionCodecFor(path, codec.name()) } } @@ -231,7 +310,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { } // Checks default compression codec - checkCompressionCodec(CompressionCodecName.fromConf(sqlContext.conf.parquetCompressionCodec)) + checkCompressionCodec( + CompressionCodecName.fromConf(spark.conf.get(SQLConf.PARQUET_COMPRESSION))) checkCompressionCodec(CompressionCodecName.UNCOMPRESSED) checkCompressionCodec(CompressionCodecName.GZIP) @@ -251,8 +331,20 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { |} """.stripMargin) - val writeSupport = new TestGroupWriteSupport(schema) - val writer = new ParquetWriter[Group](path, writeSupport) + val testWriteSupport = new TestGroupWriteSupport(schema) + /** + * Provide a builder for constructing a parquet writer - after PARQUET-248 directly + * constructing the writer is deprecated and should be done through a builder. The default + * builders include Avro - but for raw Parquet writing we must create our own builder. + */ + class ParquetWriterBuilder() extends + ParquetWriter.Builder[Group, ParquetWriterBuilder](path) { + override def getWriteSupport(conf: Configuration) = testWriteSupport + + override def self() = this + } + + val writer = new ParquetWriterBuilder().build() (0 until 10).foreach { i => val record = new SimpleGroup(schema) @@ -270,24 +362,26 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { withTempDir { dir => val path = new Path(dir.toURI.toString, "part-r-0.parquet") makeRawParquetFile(path) - checkAnswer(sqlContext.read.parquet(path.toString), (0 until 10).map { i => - Row(i % 2 == 0, i, i.toLong, i.toFloat, i.toDouble) - }) + readParquetFile(path.toString) { df => + checkAnswer(df, (0 until 10).map { i => + Row(i % 2 == 0, i, i.toLong, i.toFloat, i.toDouble) }) + } } } test("write metadata") { + val hadoopConf = spark.sessionState.newHadoopConf() withTempPath { file => val path = new Path(file.toURI.toString) - val fs = FileSystem.getLocal(hadoopConfiguration) + val fs = FileSystem.getLocal(hadoopConf) val schema = StructType.fromAttributes(ScalaReflection.attributesFor[(Int, String)]) - writeMetadata(schema, path, hadoopConfiguration) + writeMetadata(schema, path, hadoopConf) assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_COMMON_METADATA_FILE))) assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE))) - val expectedSchema = new CatalystSchemaConverter().convert(schema) - val actualSchema = readFooter(path, hadoopConfiguration).getFileMetaData.getSchema + val expectedSchema = new ParquetSchemaConverter().convert(schema) + val actualSchema = readFooter(path, hadoopConf).getFileMetaData.getSchema actualSchema.checkContains(expectedSchema) expectedSchema.checkContains(actualSchema) @@ -298,7 +392,9 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { withParquetFile((1 to 10).map(i => (i, i.toString))) { file => val newData = (11 to 20).map(i => (i, i.toString)) newData.toDF().write.format("parquet").mode(SaveMode.Overwrite).save(file) - checkAnswer(sqlContext.read.parquet(file), newData.map(Row.fromTuple)) + readParquetFile(file) { df => + checkAnswer(df, newData.map(Row.fromTuple)) + } } } @@ -307,7 +403,9 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { withParquetFile(data) { file => val newData = (11 to 20).map(i => (i, i.toString)) newData.toDF().write.format("parquet").mode(SaveMode.Ignore).save(file) - checkAnswer(sqlContext.read.parquet(file), data.map(Row.fromTuple)) + readParquetFile(file) { df => + checkAnswer(df, data.map(Row.fromTuple)) + } } } @@ -327,7 +425,9 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { withParquetFile(data) { file => val newData = (11 to 20).map(i => (i, i.toString)) newData.toDF().write.format("parquet").mode(SaveMode.Append).save(file) - checkAnswer(sqlContext.read.parquet(file), (data ++ newData).map(Row.fromTuple)) + readParquetFile(file) { df => + checkAnswer(df, (data ++ newData).map(Row.fromTuple)) + } } } @@ -350,95 +450,35 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { """.stripMargin) withTempPath { location => - val extraMetadata = Collections.singletonMap( - CatalystReadSupport.SPARK_METADATA_KEY, sparkSchema.toString) - val fileMetadata = new FileMetaData(parquetSchema, extraMetadata, "Spark") + val extraMetadata = Map(ParquetReadSupport.SPARK_METADATA_KEY -> sparkSchema.toString) val path = new Path(location.getCanonicalPath) - - ParquetFileWriter.writeMetadataFile( - sparkContext.hadoopConfiguration, - path, - Collections.singletonList( - new Footer(path, new ParquetMetadata(fileMetadata, Collections.emptyList())))) - - assertResult(sqlContext.read.parquet(path.toString).schema) { - StructType( - StructField("a", BooleanType, nullable = false) :: - StructField("b", IntegerType, nullable = false) :: - Nil) - } - } - } - - test("SPARK-6352 DirectParquetOutputCommitter") { - val clonedConf = new Configuration(hadoopConfiguration) - - // Write to a parquet file and let it fail. - // _temporary should be missing if direct output committer works. - try { - hadoopConfiguration.set("spark.sql.parquet.output.committer.class", - classOf[DirectParquetOutputCommitter].getCanonicalName) - sqlContext.udf.register("div0", (x: Int) => x / 0) - withTempPath { dir => - intercept[org.apache.spark.SparkException] { - sqlContext.range(1, 2).selectExpr("div0(id) as a").write.parquet(dir.getCanonicalPath) + val conf = spark.sessionState.newHadoopConf() + writeMetadata(parquetSchema, path, conf, extraMetadata) + + readParquetFile(path.toString) { df => + assertResult(df.schema) { + StructType( + StructField("a", BooleanType, nullable = true) :: + StructField("b", IntegerType, nullable = true) :: + Nil) } - val path = new Path(dir.getCanonicalPath, "_temporary") - val fs = path.getFileSystem(hadoopConfiguration) - assert(!fs.exists(path)) } - } finally { - // Hadoop 1 doesn't have `Configuration.unset` - hadoopConfiguration.clear() - clonedConf.asScala.foreach(entry => hadoopConfiguration.set(entry.getKey, entry.getValue)) } } - test("SPARK-9849 DirectParquetOutputCommitter qualified name should be backward compatible") { - val clonedConf = new Configuration(hadoopConfiguration) - - // Write to a parquet file and let it fail. - // _temporary should be missing if direct output committer works. - try { - hadoopConfiguration.set("spark.sql.parquet.output.committer.class", - "org.apache.spark.sql.parquet.DirectParquetOutputCommitter") - sqlContext.udf.register("div0", (x: Int) => x / 0) - withTempPath { dir => - intercept[org.apache.spark.SparkException] { - sqlContext.range(1, 2).selectExpr("div0(id) as a").write.parquet(dir.getCanonicalPath) - } - val path = new Path(dir.getCanonicalPath, "_temporary") - val fs = path.getFileSystem(hadoopConfiguration) - assert(!fs.exists(path)) - } - } finally { - // Hadoop 1 doesn't have `Configuration.unset` - hadoopConfiguration.clear() - clonedConf.asScala.foreach(entry => hadoopConfiguration.set(entry.getKey, entry.getValue)) - } - } - - test("SPARK-8121: spark.sql.parquet.output.committer.class shouldn't be overridden") { - withTempPath { dir => - val clonedConf = new Configuration(hadoopConfiguration) - - hadoopConfiguration.set( - SQLConf.OUTPUT_COMMITTER_CLASS.key, classOf[ParquetOutputCommitter].getCanonicalName) - - hadoopConfiguration.set( - "spark.sql.parquet.output.committer.class", - classOf[JobCommitFailureParquetOutputCommitter].getCanonicalName) - - try { + withSQLConf(SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> + classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName) { + val extraOptions = Map( + SQLConf.OUTPUT_COMMITTER_CLASS.key -> classOf[ParquetOutputCommitter].getCanonicalName, + "spark.sql.parquet.output.committer.class" -> + classOf[JobCommitFailureParquetOutputCommitter].getCanonicalName + ) + withTempPath { dir => val message = intercept[SparkException] { - sqlContext.range(0, 1).write.parquet(dir.getCanonicalPath) + spark.range(0, 1).write.options(extraOptions).parquet(dir.getCanonicalPath) }.getCause.getMessage assert(message === "Intentional exception for testing purposes") - } finally { - // Hadoop 1 doesn't have `Configuration.unset` - hadoopConfiguration.clear() - clonedConf.asScala.foreach(entry => hadoopConfiguration.set(entry.getKey, entry.getValue)) } } } @@ -447,66 +487,288 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { // In 1.3.0, save to fs other than file: without configuring core-site.xml would get: // IllegalArgumentException: Wrong FS: hdfs://..., expected: file:/// intercept[Throwable] { - sqlContext.read.parquet("file:///nonexistent") + spark.read.parquet("file:///nonexistent") } val errorMessage = intercept[Throwable] { - sqlContext.read.parquet("hdfs://nonexistent") + spark.read.parquet("hdfs://nonexistent") }.toString assert(errorMessage.contains("UnknownHostException")) } test("SPARK-7837 Do not close output writer twice when commitTask() fails") { - val clonedConf = new Configuration(hadoopConfiguration) + withSQLConf(SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> + classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName) { + // Using a output committer that always fail when committing a task, so that both + // `commitTask()` and `abortTask()` are invoked. + val extraOptions = Map[String, String]( + "spark.sql.parquet.output.committer.class" -> + classOf[TaskCommitFailureParquetOutputCommitter].getCanonicalName + ) - // Using a output committer that always fail when committing a task, so that both - // `commitTask()` and `abortTask()` are invoked. - hadoopConfiguration.set( - "spark.sql.parquet.output.committer.class", - classOf[TaskCommitFailureParquetOutputCommitter].getCanonicalName) - - try { // Before fixing SPARK-7837, the following code results in an NPE because both // `commitTask()` and `abortTask()` try to close output writers. withTempPath { dir => val m1 = intercept[SparkException] { - sqlContext.range(1).coalesce(1).write.parquet(dir.getCanonicalPath) + spark.range(1).coalesce(1).write.options(extraOptions).parquet(dir.getCanonicalPath) }.getCause.getMessage assert(m1.contains("Intentional exception for testing purposes")) } withTempPath { dir => val m2 = intercept[SparkException] { - val df = sqlContext.range(1).select('id as 'a, 'id as 'b).coalesce(1) - df.write.partitionBy("a").parquet(dir.getCanonicalPath) + val df = spark.range(1).select('id as 'a, 'id as 'b).coalesce(1) + df.write.partitionBy("a").options(extraOptions).parquet(dir.getCanonicalPath) }.getCause.getMessage assert(m2.contains("Intentional exception for testing purposes")) } - } finally { - // Hadoop 1 doesn't have `Configuration.unset` - hadoopConfiguration.clear() - clonedConf.asScala.foreach(entry => hadoopConfiguration.set(entry.getKey, entry.getValue)) + } + } + + test("SPARK-11044 Parquet writer version fixed as version1 ") { + withSQLConf(SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> + classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName) { + // For dictionary encoding, Parquet changes the encoding types according to its writer + // version. So, this test checks one of the encoding types in order to ensure that + // the file is written with writer version2. + val extraOptions = Map[String, String]( + // Write a Parquet file with writer version2. + ParquetOutputFormat.WRITER_VERSION -> ParquetProperties.WriterVersion.PARQUET_2_0.toString, + // By default, dictionary encoding is enabled from Parquet 1.2.0 but + // it is enabled just in case. + ParquetOutputFormat.ENABLE_DICTIONARY -> "true" + ) + + val hadoopConf = spark.sessionState.newHadoopConfWithOptions(extraOptions) + + withSQLConf(ParquetOutputFormat.ENABLE_JOB_SUMMARY -> "true") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/part-r-0.parquet" + spark.range(1 << 16).selectExpr("(id % 4) AS i") + .coalesce(1).write.options(extraOptions).mode("overwrite").parquet(path) + + val blockMetadata = readFooter(new Path(path), hadoopConf).getBlocks.asScala.head + val columnChunkMetadata = blockMetadata.getColumns.asScala.head + + // If the file is written with version2, this should include + // Encoding.RLE_DICTIONARY type. For version1, it is Encoding.PLAIN_DICTIONARY + assert(columnChunkMetadata.getEncodings.contains(Encoding.RLE_DICTIONARY)) + } + } + } + } + + test("null and non-null strings") { + // Create a dataset where the first values are NULL and then some non-null values. The + // number of non-nulls needs to be bigger than the ParquetReader batch size. + val data: Dataset[String] = spark.range(200).map (i => + if (i < 150) null + else "a" + ) + val df = data.toDF("col") + assert(df.agg("col" -> "count").collect().head.getLong(0) == 50) + + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/data" + df.write.parquet(path) + + readParquetFile(path) { df2 => + assert(df2.agg("col" -> "count").collect().head.getLong(0) == 50) + } } } test("read dictionary encoded decimals written as INT32") { - checkAnswer( - // Decimal column in this file is encoded using plain dictionary - readResourceParquetFile("dec-in-i32.parquet"), - sqlContext.range(1 << 4).select('id % 10 cast DecimalType(5, 2) as 'i32_dec)) + ("true" :: "false" :: Nil).foreach { vectorized => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized) { + checkAnswer( + // Decimal column in this file is encoded using plain dictionary + readResourceParquetFile("test-data/dec-in-i32.parquet"), + spark.range(1 << 4).select('id % 10 cast DecimalType(5, 2) as 'i32_dec)) + } + } } test("read dictionary encoded decimals written as INT64") { - checkAnswer( - // Decimal column in this file is encoded using plain dictionary - readResourceParquetFile("dec-in-i64.parquet"), - sqlContext.range(1 << 4).select('id % 10 cast DecimalType(10, 2) as 'i64_dec)) + ("true" :: "false" :: Nil).foreach { vectorized => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized) { + checkAnswer( + // Decimal column in this file is encoded using plain dictionary + readResourceParquetFile("test-data/dec-in-i64.parquet"), + spark.range(1 << 4).select('id % 10 cast DecimalType(10, 2) as 'i64_dec)) + } + } + } + + test("read dictionary encoded decimals written as FIXED_LEN_BYTE_ARRAY") { + ("true" :: "false" :: Nil).foreach { vectorized => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized) { + checkAnswer( + // Decimal column in this file is encoded using plain dictionary + readResourceParquetFile("test-data/dec-in-fixed-len.parquet"), + spark.range(1 << 4).select('id % 10 cast DecimalType(10, 2) as 'fixed_len_dec)) + } + } } - // TODO Adds test case for reading dictionary encoded decimals written as `FIXED_LEN_BYTE_ARRAY` - // The Parquet writer version Spark 1.6 and prior versions use is `PARQUET_1_0`, which doesn't - // provide dictionary encoding support for `FIXED_LEN_BYTE_ARRAY`. Should add a test here once - // we upgrade to `PARQUET_2_0`. + test("read dictionary and plain encoded timestamp_millis written as INT64") { + ("true" :: "false" :: Nil).foreach { vectorized => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized) { + checkAnswer( + // timestamp column in this file is encoded using combination of plain + // and dictionary encodings. + readResourceParquetFile("test-data/timemillis-in-i64.parquet"), + (1 to 3).map(i => Row(new java.sql.Timestamp(10)))) + } + } + } + + test("SPARK-12589 copy() on rows returned from reader works for strings") { + withTempPath { dir => + val data = (1, "abc") ::(2, "helloabcde") :: Nil + data.toDF().write.parquet(dir.getCanonicalPath) + var hash1: Int = 0 + var hash2: Int = 0 + (false :: true :: Nil).foreach { v => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> v.toString) { + val df = spark.read.parquet(dir.getCanonicalPath) + val rows = df.queryExecution.toRdd.map(_.copy()).collect() + val unsafeRows = rows.map(_.asInstanceOf[UnsafeRow]) + if (!v) { + hash1 = unsafeRows(0).hashCode() + hash2 = unsafeRows(1).hashCode() + } else { + assert(hash1 == unsafeRows(0).hashCode()) + assert(hash2 == unsafeRows(1).hashCode()) + } + } + } + } + } + + test("VectorizedParquetRecordReader - direct path read") { + val data = (0 to 10).map(i => (i, (i + 'a').toChar.toString)) + withTempPath { dir => + spark.createDataFrame(data).repartition(1).write.parquet(dir.getCanonicalPath) + val file = SpecificParquetRecordReaderBase.listDirectory(dir).get(0); + { + val reader = new VectorizedParquetRecordReader + try { + reader.initialize(file, null) + val result = mutable.ArrayBuffer.empty[(Int, String)] + while (reader.nextKeyValue()) { + val row = reader.getCurrentValue.asInstanceOf[InternalRow] + val v = (row.getInt(0), row.getString(1)) + result += v + } + assert(data == result) + } finally { + reader.close() + } + } + + // Project just one column + { + val reader = new VectorizedParquetRecordReader + try { + reader.initialize(file, ("_2" :: Nil).asJava) + val result = mutable.ArrayBuffer.empty[(String)] + while (reader.nextKeyValue()) { + val row = reader.getCurrentValue.asInstanceOf[InternalRow] + result += row.getString(0) + } + assert(data.map(_._2) == result) + } finally { + reader.close() + } + } + + // Project columns in opposite order + { + val reader = new VectorizedParquetRecordReader + try { + reader.initialize(file, ("_2" :: "_1" :: Nil).asJava) + val result = mutable.ArrayBuffer.empty[(String, Int)] + while (reader.nextKeyValue()) { + val row = reader.getCurrentValue.asInstanceOf[InternalRow] + val v = (row.getString(0), row.getInt(1)) + result += v + } + assert(data.map { x => (x._2, x._1) } == result) + } finally { + reader.close() + } + } + + // Empty projection + { + val reader = new VectorizedParquetRecordReader + try { + reader.initialize(file, List[String]().asJava) + var result = 0 + while (reader.nextKeyValue()) { + result += 1 + } + assert(result == data.length) + } finally { + reader.close() + } + } + } + } + + test("VectorizedParquetRecordReader - partition column types") { + withTempPath { dir => + Seq(1).toDF().repartition(1).write.parquet(dir.getCanonicalPath) + + val dataTypes = + Seq(StringType, BooleanType, ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DecimalType(25, 5), DateType, TimestampType) + + val constantValues = + Seq( + UTF8String.fromString("a string"), + true, + 1.toByte, + 2.toShort, + 3, + Long.MaxValue, + 0.25.toFloat, + 0.75D, + Decimal("1234.23456"), + DateTimeUtils.fromJavaDate(java.sql.Date.valueOf("2015-01-01")), + DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123"))) + + dataTypes.zip(constantValues).foreach { case (dt, v) => + val schema = StructType(StructField("pcol", dt) :: Nil) + val vectorizedReader = new VectorizedParquetRecordReader + val partitionValues = new GenericInternalRow(Array(v)) + val file = SpecificParquetRecordReaderBase.listDirectory(dir).get(0) + + try { + vectorizedReader.initialize(file, null) + vectorizedReader.initBatch(schema, partitionValues) + vectorizedReader.nextKeyValue() + val row = vectorizedReader.getCurrentValue.asInstanceOf[InternalRow] + + // Use `GenericMutableRow` by explicitly copying rather than `ColumnarBatch` + // in order to use get(...) method which is not implemented in `ColumnarBatch`. + val actual = row.copy().get(1, dt) + val expected = v + assert(actual == expected) + } finally { + vectorizedReader.close() + } + } + } + } + + test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") { + withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> "snappy") { + val option = new ParquetOptions(Map("Compression" -> "uncompressed"), spark.sessionState.conf) + assert(option.compressionCodecClassName == "UNCOMPRESSED") + } + } } class JobCommitFailureParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala index 83b65fb419ed..9dc56292c372 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -81,7 +81,7 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS logParquetSchema(protobufStylePath) checkAnswer( - sqlContext.read.parquet(dir.getCanonicalPath), + spark.read.parquet(dir.getCanonicalPath), Seq( Row(Seq(0, 1)), Row(Seq(2, 3)))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 61cc0da50865..f79b92b804c7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -19,17 +19,25 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.File import java.math.BigInteger -import java.sql.Timestamp +import java.sql.{Date, Timestamp} +import java.util.{Calendar, Locale, TimeZone} import scala.collection.mutable.ArrayBuffer import com.google.common.io.Files import org.apache.hadoop.fs.Path +import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.execution.datasources.{LogicalRelation, PartitionSpec, Partition, PartitioningUtils} +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.{PartitionPath => Partition} +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -44,17 +52,34 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha import PartitioningUtils._ import testImplicits._ - val defaultPartitionName = "__HIVE_DEFAULT_PARTITION__" + val defaultPartitionName = ExternalCatalogUtils.DEFAULT_PARTITION_NAME + + val timeZone = TimeZone.getDefault() + val timeZoneId = timeZone.getID test("column type inference") { - def check(raw: String, literal: Literal): Unit = { - assert(inferPartitionColumnValue(raw, defaultPartitionName, true) === literal) + def check(raw: String, literal: Literal, timeZone: TimeZone = timeZone): Unit = { + assert(inferPartitionColumnValue(raw, true, timeZone) === literal) } check("10", Literal.create(10, IntegerType)) check("1000000000000000", Literal.create(1000000000000000L, LongType)) + val decimal = Decimal("1" * 20) + check("1" * 20, + Literal.create(decimal, DecimalType(decimal.precision, decimal.scale))) check("1.5", Literal.create(1.5, DoubleType)) check("hello", Literal.create("hello", StringType)) + check("1990-02-24", Literal.create(Date.valueOf("1990-02-24"), DateType)) + check("1990-02-24 12:00:30", + Literal.create(Timestamp.valueOf("1990-02-24 12:00:30"), TimestampType)) + + val c = Calendar.getInstance(TimeZone.getTimeZone("GMT")) + c.set(1990, 1, 24, 12, 0, 30) + c.set(Calendar.MILLISECOND, 0) + check("1990-02-24 12:00:30", + Literal.create(new Timestamp(c.getTimeInMillis), TimestampType), + TimeZone.getTimeZone("GMT")) + check(defaultPartitionName, Literal.create(null, NullType)) } @@ -66,7 +91,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha "hdfs://host:9000/path/a=10.5/b=hello") var exception = intercept[AssertionError] { - parsePartitions(paths.map(new Path(_)), defaultPartitionName, true) + parsePartitions(paths.map(new Path(_)), true, Set.empty[Path], timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -76,7 +101,37 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha "hdfs://host:9000/path/a=10/b=20", "hdfs://host:9000/path/_temporary/path") - parsePartitions(paths.map(new Path(_)), defaultPartitionName, true) + parsePartitions( + paths.map(new Path(_)), + true, + Set(new Path("hdfs://host:9000/path/")), + timeZoneId) + + // Valid + paths = Seq( + "hdfs://host:9000/path/something=true/table/", + "hdfs://host:9000/path/something=true/table/_temporary", + "hdfs://host:9000/path/something=true/table/a=10/b=20", + "hdfs://host:9000/path/something=true/table/_temporary/path") + + parsePartitions( + paths.map(new Path(_)), + true, + Set(new Path("hdfs://host:9000/path/something=true/table")), + timeZoneId) + + // Valid + paths = Seq( + "hdfs://host:9000/path/table=true/", + "hdfs://host:9000/path/table=true/_temporary", + "hdfs://host:9000/path/table=true/a=10/b=20", + "hdfs://host:9000/path/table=true/_temporary/path") + + parsePartitions( + paths.map(new Path(_)), + true, + Set(new Path("hdfs://host:9000/path/table=true")), + timeZoneId) // Invalid paths = Seq( @@ -85,7 +140,11 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha "hdfs://host:9000/path/path1") exception = intercept[AssertionError] { - parsePartitions(paths.map(new Path(_)), defaultPartitionName, true) + parsePartitions( + paths.map(new Path(_)), + true, + Set(new Path("hdfs://host:9000/path/")), + timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -101,19 +160,24 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha "hdfs://host:9000/tmp/tables/nonPartitionedTable2") exception = intercept[AssertionError] { - parsePartitions(paths.map(new Path(_)), defaultPartitionName, true) + parsePartitions( + paths.map(new Path(_)), + true, + Set(new Path("hdfs://host:9000/tmp/tables/")), + timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) } test("parse partition") { def check(path: String, expected: Option[PartitionValues]): Unit = { - assert(expected === parsePartition(new Path(path), defaultPartitionName, true)._1) + val actual = parsePartition(new Path(path), true, Set.empty[Path], timeZone)._1 + assert(expected === actual) } def checkThrows[T <: Throwable: Manifest](path: String, expected: String): Unit = { val message = intercept[T] { - parsePartition(new Path(path), defaultPartitionName, true) + parsePartition(new Path(path), true, Set.empty[Path], timeZone) }.getMessage assert(message.contains(expected)) @@ -151,9 +215,41 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha checkThrows[AssertionError]("file://path/a=", "Empty partition column value") } + test("parse partition with base paths") { + // when the basePaths is the same as the path to a leaf directory + val partitionSpec1: Option[PartitionValues] = parsePartition( + path = new Path("file://path/a=10"), + typeInference = true, + basePaths = Set(new Path("file://path/a=10")), + timeZone = timeZone)._1 + + assert(partitionSpec1.isEmpty) + + // when the basePaths is the path to a base directory of leaf directories + val partitionSpec2: Option[PartitionValues] = parsePartition( + path = new Path("file://path/a=10"), + typeInference = true, + basePaths = Set(new Path("file://path")), + timeZone = timeZone)._1 + + assert(partitionSpec2 == + Option(PartitionValues( + ArrayBuffer("a"), + ArrayBuffer(Literal.create(10, IntegerType))))) + } + test("parse partitions") { - def check(paths: Seq[String], spec: PartitionSpec): Unit = { - assert(parsePartitions(paths.map(new Path(_)), defaultPartitionName, true) === spec) + def check( + paths: Seq[String], + spec: PartitionSpec, + rootPaths: Set[Path] = Set.empty[Path]): Unit = { + val actualSpec = + parsePartitions( + paths.map(new Path(_)), + true, + rootPaths, + timeZoneId) + assert(actualSpec === spec) } check(Seq( @@ -232,7 +328,9 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha test("parse partitions with type inference disabled") { def check(paths: Seq[String], spec: PartitionSpec): Unit = { - assert(parsePartitions(paths.map(new Path(_)), defaultPartitionName, false) === spec) + val actualSpec = + parsePartitions(paths.map(new Path(_)), false, Set.empty[Path], timeZoneId) + assert(actualSpec === spec) } check(Seq( @@ -326,9 +424,9 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha // Introduce _temporary dir to the base dir the robustness of the schema discovery process. new File(base.getCanonicalPath, "_temporary").mkdir() - sqlContext.read.parquet(base.getCanonicalPath).registerTempTable("t") + spark.read.parquet(base.getCanonicalPath).createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { checkAnswer( sql("SELECT * FROM t"), for { @@ -362,6 +460,45 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } } + test("read partitioned table using different path options") { + withTempDir { base => + val pi = 1 + val ps = "foo" + val path = makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps) + makeParquetFile( + (1 to 10).map(i => ParquetData(i, i.toString)), path) + + // when the input is the base path containing partitioning directories + val baseDf = spark.read.parquet(base.getCanonicalPath) + assert(baseDf.schema.map(_.name) === Seq("intField", "stringField", "pi", "ps")) + + // when the input is a path to the leaf directory containing a parquet file + val partDf = spark.read.parquet(path.getCanonicalPath) + assert(partDf.schema.map(_.name) === Seq("intField", "stringField")) + + path.listFiles().foreach { f => + if (!f.getName.startsWith("_") && + f.getName.toLowerCase(Locale.ROOT).endsWith(".parquet")) { + // when the input is a path to a parquet file + val df = spark.read.parquet(f.getCanonicalPath) + assert(df.schema.map(_.name) === Seq("intField", "stringField")) + } + } + + path.listFiles().foreach { f => + if (!f.getName.startsWith("_") && + f.getName.toLowerCase(Locale.ROOT).endsWith(".parquet")) { + // when the input is a path to a parquet file but `basePath` is overridden to + // the base path containing partitioning directories + val df = spark + .read.option("basePath", base.getCanonicalPath) + .parquet(f.getCanonicalPath) + assert(df.schema.map(_.name) === Seq("intField", "stringField", "pi", "ps")) + } + } + } + } + test("read partitioned table - partition key included in Parquet file") { withTempDir { base => for { @@ -373,9 +510,9 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) } - sqlContext.read.parquet(base.getCanonicalPath).registerTempTable("t") + spark.read.parquet(base.getCanonicalPath).createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { checkAnswer( sql("SELECT * FROM t"), for { @@ -421,10 +558,10 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) } - val parquetRelation = sqlContext.read.format("parquet").load(base.getCanonicalPath) - parquetRelation.registerTempTable("t") + val parquetRelation = spark.read.format("parquet").load(base.getCanonicalPath) + parquetRelation.createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { checkAnswer( sql("SELECT * FROM t"), for { @@ -461,10 +598,10 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) } - val parquetRelation = sqlContext.read.format("parquet").load(base.getCanonicalPath) - parquetRelation.registerTempTable("t") + val parquetRelation = spark.read.format("parquet").load(base.getCanonicalPath) + parquetRelation.createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { checkAnswer( sql("SELECT * FROM t"), for { @@ -493,14 +630,14 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha (1 to 10).map(i => (i, i.toString)).toDF("intField", "stringField"), makePartitionDir(base, defaultPartitionName, "pi" -> 2)) - sqlContext + spark .read .option("mergeSchema", "true") .format("parquet") .load(base.getCanonicalPath) - .registerTempTable("t") + .createOrReplaceTempView("t") - withTempTable("t") { + withTempView("t") { checkAnswer( sql("SELECT * FROM t"), (1 to 10).map(i => Row(i, null, 1)) ++ (1 to 10).map(i => Row(i, i.toString, 2))) @@ -511,12 +648,13 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha test("SPARK-7749 Non-partitioned table should have empty partition spec") { withTempPath { dir => (1 to 10).map(i => (i, i.toString)).toDF("a", "b").write.parquet(dir.getCanonicalPath) - val queryExecution = sqlContext.read.parquet(dir.getCanonicalPath).queryExecution + val queryExecution = spark.read.parquet(dir.getCanonicalPath).queryExecution queryExecution.analyzed.collectFirst { - case LogicalRelation(relation: ParquetRelation, _) => - assert(relation.partitionSpec === PartitionSpec.emptySpec) + case LogicalRelation( + HadoopFsRelation(location: PartitioningAwareFileIndex, _, _, _, _, _), _, _, _) => + assert(location.partitionSpec() === PartitionSpec.emptySpec) }.getOrElse { - fail(s"Expecting a ParquetRelation2, but got:\n$queryExecution") + fail(s"Expecting a matching HadoopFsRelation, but got:\n$queryExecution") } } } @@ -525,7 +663,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha withTempPath { dir => val df = Seq("/", "[]", "?").zipWithIndex.map(_.swap).toDF("i", "s") df.write.format("parquet").partitionBy("s").save(dir.getCanonicalPath) - checkAnswer(sqlContext.read.parquet(dir.getCanonicalPath), df.collect()) + checkAnswer(spark.read.parquet(dir.getCanonicalPath), df.collect()) } } @@ -539,7 +677,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha 1.5.toFloat, 4.5, new java.math.BigDecimal(new BigInteger("212500"), 5), - new java.math.BigDecimal(2.125), + new java.math.BigDecimal("2.125"), java.sql.Date.valueOf("2015-05-23"), new Timestamp(0), "This is a string, /[]?=:", @@ -565,12 +703,62 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } val schema = StructType(partitionColumns :+ StructField(s"i", StringType)) - val df = sqlContext.createDataFrame(sparkContext.parallelize(row :: Nil), schema) + val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema) withTempPath { dir => df.write.format("parquet").partitionBy(partitionColumns.map(_.name): _*).save(dir.toString) val fields = schema.map(f => Column(f.name).cast(f.dataType)) - checkAnswer(sqlContext.read.load(dir.toString).select(fields: _*), row) + checkAnswer(spark.read.load(dir.toString).select(fields: _*), row) + } + + withTempPath { dir => + df.write.option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + .format("parquet").partitionBy(partitionColumns.map(_.name): _*).save(dir.toString) + val fields = schema.map(f => Column(f.name).cast(f.dataType)) + checkAnswer(spark.read.option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + .load(dir.toString).select(fields: _*), row) + } + } + + test("Various inferred partition value types") { + val row = + Row( + Long.MaxValue, + 4.5, + new java.math.BigDecimal(new BigInteger("1" * 20)), + java.sql.Date.valueOf("2015-05-23"), + java.sql.Timestamp.valueOf("1990-02-24 12:00:30"), + "This is a string, /[]?=:", + "This is not a partition column") + + val partitionColumnTypes = + Seq( + LongType, + DoubleType, + DecimalType(20, 0), + DateType, + TimestampType, + StringType) + + val partitionColumns = partitionColumnTypes.zipWithIndex.map { + case (t, index) => StructField(s"p_$index", t) + } + + val schema = StructType(partitionColumns :+ StructField(s"i", StringType)) + val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema) + + withTempPath { dir => + df.write.format("parquet").partitionBy(partitionColumns.map(_.name): _*).save(dir.toString) + val fields = schema.map(f => Column(f.name)) + checkAnswer(spark.read.load(dir.toString).select(fields: _*), row) + } + + withTempPath { dir => + df.write.option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + .format("parquet").partitionBy(partitionColumns.map(_.name): _*).save(dir.toString) + val fields = schema.map(f => Column(f.name)) + checkAnswer(spark.read.option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + .load(dir.toString).select(fields: _*), row) } } @@ -586,7 +774,141 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha Files.touch(new File(s"${dir.getCanonicalPath}/b=1", ".DS_Store")) Files.createParentDirs(new File(s"${dir.getCanonicalPath}/b=1/c=1/.foo/bar")) - checkAnswer(sqlContext.read.format("parquet").load(dir.getCanonicalPath), df) + checkAnswer(spark.read.format("parquet").load(dir.getCanonicalPath), df) + } + } + + test("SPARK-11678: Partition discovery stops at the root path of the dataset") { + withTempPath { dir => + val tablePath = new File(dir, "key=value") + val df = (1 to 3).map(i => (i, i, i, i)).toDF("a", "b", "c", "d") + + df.write + .format("parquet") + .partitionBy("b", "c", "d") + .save(tablePath.getCanonicalPath) + + Files.touch(new File(s"${tablePath.getCanonicalPath}/", "_SUCCESS")) + Files.createParentDirs(new File(s"${dir.getCanonicalPath}/b=1/c=1/.foo/bar")) + + checkAnswer(spark.read.format("parquet").load(tablePath.getCanonicalPath), df) + } + + withTempPath { dir => + val path = new File(dir, "key=value") + val tablePath = new File(path, "table") + + val df = (1 to 3).map(i => (i, i, i, i)).toDF("a", "b", "c", "d") + + df.write + .format("parquet") + .partitionBy("b", "c", "d") + .save(tablePath.getCanonicalPath) + + Files.touch(new File(s"${tablePath.getCanonicalPath}/", "_SUCCESS")) + Files.createParentDirs(new File(s"${dir.getCanonicalPath}/b=1/c=1/.foo/bar")) + + checkAnswer(spark.read.format("parquet").load(tablePath.getCanonicalPath), df) + } + } + + test("use basePath to specify the root dir of a partitioned table.") { + withTempPath { dir => + val tablePath = new File(dir, "table") + val df = (1 to 3).map(i => (i, i, i, i)).toDF("a", "b", "c", "d") + + df.write + .format("parquet") + .partitionBy("b", "c", "d") + .save(tablePath.getCanonicalPath) + + val twoPartitionsDF = + spark + .read + .option("basePath", tablePath.getCanonicalPath) + .parquet( + s"${tablePath.getCanonicalPath}/b=1", + s"${tablePath.getCanonicalPath}/b=2") + + checkAnswer(twoPartitionsDF, df.filter("b != 3")) + + intercept[AssertionError] { + spark + .read + .parquet( + s"${tablePath.getCanonicalPath}/b=1", + s"${tablePath.getCanonicalPath}/b=2") + } + } + } + + test("use basePath and file globbing to selectively load partitioned table") { + withTempPath { dir => + + val df = Seq( + (1, "foo", 100), + (1, "bar", 200), + (2, "foo", 300), + (2, "bar", 400) + ).toDF("p1", "p2", "v") + df.write + .mode(SaveMode.Overwrite) + .partitionBy("p1", "p2") + .parquet(dir.getCanonicalPath) + + def check(path: String, basePath: String, expectedDf: DataFrame): Unit = { + val testDf = spark.read + .option("basePath", basePath) + .parquet(path) + checkAnswer(testDf, expectedDf) + } + + // Should find all the data with partitioning columns when base path is set to the root + val resultDf = df.select("v", "p1", "p2") + check(path = s"$dir", basePath = s"$dir", resultDf) + check(path = s"$dir/*", basePath = s"$dir", resultDf) + check(path = s"$dir/*/*", basePath = s"$dir", resultDf) + check(path = s"$dir/*/*/*", basePath = s"$dir", resultDf) + + // Should find selective partitions of the data if the base path is not set to root + + check( // read from ../p1=1 with base ../p1=1, should not infer p1 col + path = s"$dir/p1=1/*", + basePath = s"$dir/p1=1/", + resultDf.filter("p1 = 1").drop("p1")) + + check( // red from ../p1=1/p2=foo with base ../p1=1/ should not infer p1 + path = s"$dir/p1=1/p2=foo/*", + basePath = s"$dir/p1=1/", + resultDf.filter("p1 = 1").filter("p2 = 'foo'").drop("p1")) + + check( // red from ../p1=1/p2=foo with base ../p1=1/p2=foo, should not infer p1, p2 + path = s"$dir/p1=1/p2=foo/*", + basePath = s"$dir/p1=1/p2=foo/", + resultDf.filter("p1 = 1").filter("p2 = 'foo'").drop("p1", "p2")) + } + } + + test("_SUCCESS should not break partitioning discovery") { + Seq(1, 32).foreach { threshold => + // We have two paths to list files, one at driver side, another one that we use + // a Spark job. We need to test both ways. + withSQLConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD.key -> threshold.toString) { + withTempPath { dir => + val tablePath = new File(dir, "table") + val df = (1 to 3).map(i => (i, i, i, i)).toDF("a", "b", "c", "d") + + df.write + .format("parquet") + .partitionBy("b", "c", "d") + .save(tablePath.getCanonicalPath) + + Files.touch(new File(s"${tablePath.getCanonicalPath}/b=1", "_SUCCESS")) + Files.touch(new File(s"${tablePath.getCanonicalPath}/b=1/c=1", "_SUCCESS")) + Files.touch(new File(s"${tablePath.getCanonicalPath}/b=1/c=1/d=1", "_SUCCESS")) + checkAnswer(spark.read.format("parquet").load(tablePath.getCanonicalPath), df) + } + } } } @@ -639,10 +961,110 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha withTempPath { dir => withSQLConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD.key -> "1") { val path = dir.getCanonicalPath - val df = sqlContext.range(5).select('id as 'a, 'id as 'b, 'id as 'c).coalesce(1) + val df = spark.range(5).select('id as 'a, 'id as 'b, 'id as 'c).coalesce(1) df.write.partitionBy("b", "c").parquet(path) - checkAnswer(sqlContext.read.parquet(path), df) + checkAnswer(spark.read.parquet(path), df) + } + } + } + + test("SPARK-15895 summary files in non-leaf partition directories") { + withTempPath { dir => + val path = dir.getCanonicalPath + + withSQLConf( + ParquetOutputFormat.ENABLE_JOB_SUMMARY -> "true", + "spark.sql.sources.commitProtocolClass" -> + classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName) { + spark.range(3).write.parquet(s"$path/p0=0/p1=0") } + + val p0 = new File(path, "p0=0") + val p1 = new File(p0, "p1=0") + + // Builds the following directory layout by: + // + // 1. copying Parquet summary files we just wrote into `p0=0`, and + // 2. touching a dot-file `.dummy` under `p0=0`. + // + // + // +- p0=0 + // |- _metadata + // |- _common_metadata + // |- .dummy + // +- p1=0 + // |- _metadata + // |- _common_metadata + // |- part-00000.parquet + // |- part-00001.parquet + // +- ... + // + // The summary files and the dot-file under `p0=0` should not fail partition discovery. + + Files.copy(new File(p1, "_metadata"), new File(p0, "_metadata")) + Files.copy(new File(p1, "_common_metadata"), new File(p0, "_common_metadata")) + Files.touch(new File(p0, ".dummy")) + + checkAnswer(spark.read.parquet(s"$path"), Seq( + Row(0, 0, 0), + Row(1, 0, 0), + Row(2, 0, 0) + )) + } + } + + test("SPARK-18108 Parquet reader fails when data column types conflict with partition ones") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { + withTempPath { dir => + val path = dir.getCanonicalPath + val df = Seq((1L, 2.0)).toDF("a", "b") + df.write.parquet(s"$path/a=1") + checkAnswer(spark.read.parquet(s"$path"), Seq(Row(1, 2.0))) + } + } + } + + test("SPARK-21463: MetadataLogFileIndex should respect userSpecifiedSchema for partition cols") { + withTempDir { tempDir => + val output = new File(tempDir, "output").toString + val checkpoint = new File(tempDir, "chkpoint").toString + try { + val stream = MemoryStream[(String, Int)] + val df = stream.toDS().toDF("time", "value") + val sq = df.writeStream + .option("checkpointLocation", checkpoint) + .format("parquet") + .partitionBy("time") + .start(output) + + stream.addData(("2017-01-01-00", 1), ("2017-01-01-01", 2)) + sq.processAllAvailable() + + val schema = new StructType() + .add("time", StringType) + .add("value", IntegerType) + val readBack = spark.read.schema(schema).parquet(output) + assert(readBack.schema.toSet === schema.toSet) + + checkAnswer( + readBack, + Seq(Row("2017-01-01-00", 1), Row("2017-01-01-01", 2)) + ) + } finally { + spark.streams.active.foreach(_.stop()) + } + } + } + + test("SPARK-22109: Resolve type conflicts between strings and timestamps in partition column") { + val df = Seq( + (1, "2015-01-01 00:00:00"), + (2, "2014-01-01 00:00:00"), + (3, "blah")).toDF("i", "str") + + withTempPath { path => + df.write.format("parquet").partitionBy("str").save(path.getAbsolutePath) + checkAnswer(spark.read.load(path.getAbsolutePath), df) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala index 98333e58cada..fa88019298a6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala @@ -22,12 +22,12 @@ import org.apache.spark.sql.test.SharedSQLContext class ParquetProtobufCompatibilitySuite extends ParquetCompatibilityTest with SharedSQLContext { test("unannotated array of primitive type") { - checkAnswer(readResourceParquetFile("old-repeated-int.parquet"), Row(Seq(1, 2, 3))) + checkAnswer(readResourceParquetFile("test-data/old-repeated-int.parquet"), Row(Seq(1, 2, 3))) } test("unannotated array of struct") { checkAnswer( - readResourceParquetFile("old-repeated-message.parquet"), + readResourceParquetFile("test-data/old-repeated-message.parquet"), Row( Seq( Row("First inner", null, null), @@ -35,14 +35,14 @@ class ParquetProtobufCompatibilitySuite extends ParquetCompatibilityTest with Sh Row(null, null, "Third inner")))) checkAnswer( - readResourceParquetFile("proto-repeated-struct.parquet"), + readResourceParquetFile("test-data/proto-repeated-struct.parquet"), Row( Seq( Row("0 - 1", "0 - 2", "0 - 3"), Row("1 - 1", "1 - 2", "1 - 3")))) checkAnswer( - readResourceParquetFile("proto-struct-with-array-many.parquet"), + readResourceParquetFile("test-data/proto-struct-with-array-many.parquet"), Seq( Row( Seq( @@ -60,13 +60,13 @@ class ParquetProtobufCompatibilitySuite extends ParquetCompatibilityTest with Sh test("struct with unannotated array") { checkAnswer( - readResourceParquetFile("proto-struct-with-array.parquet"), + readResourceParquetFile("test-data/proto-struct-with-array.parquet"), Row(10, 9, Seq.empty, null, Row(9), Seq(Row(9), Row(10)))) } test("unannotated array of struct with unannotated array") { checkAnswer( - readResourceParquetFile("nested-array-struct.parquet"), + readResourceParquetFile("test-data/nested-array-struct.parquet"), Seq( Row(2, Seq(Row(1, Seq(Row(3))))), Row(5, Seq(Row(4, Seq(Row(6))))), @@ -75,7 +75,7 @@ class ParquetProtobufCompatibilitySuite extends ParquetCompatibilityTest with Sh test("unannotated array of string") { checkAnswer( - readResourceParquetFile("proto-repeated-string.parquet"), + readResourceParquetFile("test-data/proto-repeated-string.parquet"), Seq( Row(Seq("hello", "world")), Row(Seq("good", "bye")), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 70fae32b7e7a..2efff3f57d7d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -18,13 +18,19 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.File +import java.sql.Timestamp -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.parquet.hadoop.ParquetOutputFormat +import org.apache.spark.{DebugFilesystem, SparkException} import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.SpecificMutableRow import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} -import org.apache.spark.sql.execution.datasources.parquet.TestingUDT.{NestedStruct, NestedStructUDT} +import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol +import org.apache.spark.sql.execution.datasources.parquet.TestingUDT.{NestedStruct, NestedStructUDT, SingleElement} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -44,22 +50,49 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext test("appending") { val data = (0 until 10).map(i => (i, i.toString)) - sqlContext.createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") - withParquetTable(data, "t") { + spark.createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") + // Query appends, don't test with both read modes. + withParquetTable(data, "t", false) { sql("INSERT INTO TABLE t SELECT * FROM tmp") - checkAnswer(sqlContext.table("t"), (data ++ data).map(Row.fromTuple)) + checkAnswer(spark.table("t"), (data ++ data).map(Row.fromTuple)) } - sqlContext.catalog.unregisterTable(TableIdentifier("tmp")) + spark.sessionState.catalog.dropTable( + TableIdentifier("tmp"), ignoreIfNotExists = true, purge = false) } test("overwriting") { val data = (0 until 10).map(i => (i, i.toString)) - sqlContext.createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") + spark.createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") withParquetTable(data, "t") { sql("INSERT OVERWRITE TABLE t SELECT * FROM tmp") - checkAnswer(sqlContext.table("t"), data.map(Row.fromTuple)) + checkAnswer(spark.table("t"), data.map(Row.fromTuple)) + } + spark.sessionState.catalog.dropTable( + TableIdentifier("tmp"), ignoreIfNotExists = true, purge = false) + } + + test("SPARK-15678: not use cache on overwrite") { + withTempDir { dir => + val path = dir.toString + spark.range(1000).write.mode("overwrite").parquet(path) + val df = spark.read.parquet(path).cache() + assert(df.count() == 1000) + spark.range(10).write.mode("overwrite").parquet(path) + assert(df.count() == 10) + assert(spark.read.parquet(path).count() == 10) + } + } + + test("SPARK-15678: not use cache on append") { + withTempDir { dir => + val path = dir.toString + spark.range(1000).write.mode("append").parquet(path) + val df = spark.read.parquet(path).cache() + assert(df.count() == 1000) + spark.range(10).write.mode("append").parquet(path) + assert(df.count() == 1010) + assert(spark.read.parquet(path).count() == 1010) } - sqlContext.catalog.unregisterTable(TableIdentifier("tmp")) } test("self-join") { @@ -69,7 +102,8 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext (maybeInt, i.toString) } - withParquetTable(data, "t") { + // TODO: vectorized doesn't work here because it requires UnsafeRows + withParquetTable(data, "t", false) { val selfJoin = sql("SELECT * FROM t x JOIN t y WHERE x._1 = y._1") val queryOutput = selfJoin.queryExecution.analyzed.output @@ -122,33 +156,114 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext val schema = StructType(List(StructField("d", DecimalType(18, 0), false), StructField("time", TimestampType, false)).toArray) withTempPath { file => - val df = sqlContext.createDataFrame(sparkContext.parallelize(data), schema) + val df = spark.createDataFrame(sparkContext.parallelize(data), schema) df.write.parquet(file.getCanonicalPath) - val df2 = sqlContext.read.parquet(file.getCanonicalPath) + val df2 = spark.read.parquet(file.getCanonicalPath) checkAnswer(df2, df.collect().toSeq) } } + test("SPARK-10634 timestamp written and read as INT64 - TIMESTAMP_MILLIS") { + val data = (1 to 10).map(i => Row(i, new java.sql.Timestamp(i))) + val schema = StructType(List(StructField("d", IntegerType, false), + StructField("time", TimestampType, false)).toArray) + withSQLConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key -> "true") { + withTempPath { file => + val df = spark.createDataFrame(sparkContext.parallelize(data), schema) + df.write.parquet(file.getCanonicalPath) + ("true" :: "false" :: Nil).foreach { vectorized => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized) { + val df2 = spark.read.parquet(file.getCanonicalPath) + checkAnswer(df2, df.collect().toSeq) + } + } + } + } + } + + test("SPARK-10634 timestamp written and read as INT64 - truncation") { + withTable("ts") { + sql("create table ts (c1 int, c2 timestamp) using parquet") + sql("insert into ts values (1, '2016-01-01 10:11:12.123456')") + sql("insert into ts values (2, null)") + sql("insert into ts values (3, '1965-01-01 10:11:12.123456')") + checkAnswer( + sql("select * from ts"), + Seq( + Row(1, Timestamp.valueOf("2016-01-01 10:11:12.123456")), + Row(2, null), + Row(3, Timestamp.valueOf("1965-01-01 10:11:12.123456")))) + } + + // The microsecond portion is truncated when written as TIMESTAMP_MILLIS. + withTable("ts") { + withSQLConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key -> "true") { + sql("create table ts (c1 int, c2 timestamp) using parquet") + sql("insert into ts values (1, '2016-01-01 10:11:12.123456')") + sql("insert into ts values (2, null)") + sql("insert into ts values (3, '1965-01-01 10:11:12.125456')") + sql("insert into ts values (4, '1965-01-01 10:11:12.125')") + sql("insert into ts values (5, '1965-01-01 10:11:12.1')") + sql("insert into ts values (6, '1965-01-01 10:11:12.123456789')") + sql("insert into ts values (7, '0001-01-01 00:00:00.000000')") + checkAnswer( + sql("select * from ts"), + Seq( + Row(1, Timestamp.valueOf("2016-01-01 10:11:12.123")), + Row(2, null), + Row(3, Timestamp.valueOf("1965-01-01 10:11:12.125")), + Row(4, Timestamp.valueOf("1965-01-01 10:11:12.125")), + Row(5, Timestamp.valueOf("1965-01-01 10:11:12.1")), + Row(6, Timestamp.valueOf("1965-01-01 10:11:12.123")), + Row(7, Timestamp.valueOf("0001-01-01 00:00:00.000")))) + + // Read timestamps that were encoded as TIMESTAMP_MILLIS annotated as INT64 + // with PARQUET_INT64_AS_TIMESTAMP_MILLIS set to false. + withSQLConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key -> "false") { + checkAnswer( + sql("select * from ts"), + Seq( + Row(1, Timestamp.valueOf("2016-01-01 10:11:12.123")), + Row(2, null), + Row(3, Timestamp.valueOf("1965-01-01 10:11:12.125")), + Row(4, Timestamp.valueOf("1965-01-01 10:11:12.125")), + Row(5, Timestamp.valueOf("1965-01-01 10:11:12.1")), + Row(6, Timestamp.valueOf("1965-01-01 10:11:12.123")), + Row(7, Timestamp.valueOf("0001-01-01 00:00:00.000")))) + } + } + } + } + test("Enabling/disabling merging partfiles when merging parquet schema") { def testSchemaMerging(expectedColumnNumber: Int): Unit = { withTempDir { dir => val basePath = dir.getCanonicalPath - sqlContext.range(0, 10).toDF("a").write.parquet(new Path(basePath, "foo=1").toString) - sqlContext.range(0, 10).toDF("b").write.parquet(new Path(basePath, "foo=2").toString) + spark.range(0, 10).toDF("a").write.parquet(new Path(basePath, "foo=1").toString) + spark.range(0, 10).toDF("b").write.parquet(new Path(basePath, "foo=2").toString) // delete summary files, so if we don't merge part-files, one column will not be included. Utils.deleteRecursively(new File(basePath + "/foo=1/_metadata")) Utils.deleteRecursively(new File(basePath + "/foo=1/_common_metadata")) - assert(sqlContext.read.parquet(basePath).columns.length === expectedColumnNumber) + assert(spark.read.parquet(basePath).columns.length === expectedColumnNumber) } } - withSQLConf(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true", - SQLConf.PARQUET_SCHEMA_RESPECT_SUMMARIES.key -> "true") { + withSQLConf( + SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> + classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName, + SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true", + SQLConf.PARQUET_SCHEMA_RESPECT_SUMMARIES.key -> "true", + ParquetOutputFormat.ENABLE_JOB_SUMMARY -> "true" + ) { testSchemaMerging(2) } - withSQLConf(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true", - SQLConf.PARQUET_SCHEMA_RESPECT_SUMMARIES.key -> "false") { + withSQLConf( + SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> + classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName, + SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true", + SQLConf.PARQUET_SCHEMA_RESPECT_SUMMARIES.key -> "false" + ) { testSchemaMerging(3) } } @@ -157,9 +272,9 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext def testSchemaMerging(expectedColumnNumber: Int): Unit = { withTempDir { dir => val basePath = dir.getCanonicalPath - sqlContext.range(0, 10).toDF("a").write.parquet(new Path(basePath, "foo=1").toString) - sqlContext.range(0, 10).toDF("b").write.parquet(new Path(basePath, "foo=2").toString) - assert(sqlContext.read.parquet(basePath).columns.length === expectedColumnNumber) + spark.range(0, 10).toDF("a").write.parquet(new Path(basePath, "foo=1").toString) + spark.range(0, 10).toDF("b").write.parquet(new Path(basePath, "foo=2").toString) + assert(spark.read.parquet(basePath).columns.length === expectedColumnNumber) } } @@ -172,22 +287,84 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext } } + test("Enabling/disabling ignoreCorruptFiles") { + def testIgnoreCorruptFiles(): Unit = { + withTempDir { dir => + val basePath = dir.getCanonicalPath + spark.range(1).toDF("a").write.parquet(new Path(basePath, "first").toString) + spark.range(1, 2).toDF("a").write.parquet(new Path(basePath, "second").toString) + spark.range(2, 3).toDF("a").write.json(new Path(basePath, "third").toString) + val df = spark.read.parquet( + new Path(basePath, "first").toString, + new Path(basePath, "second").toString, + new Path(basePath, "third").toString) + checkAnswer( + df, + Seq(Row(0), Row(1))) + } + } + + withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") { + testIgnoreCorruptFiles() + } + + withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") { + val exception = intercept[SparkException] { + testIgnoreCorruptFiles() + } + assert(exception.getMessage().contains("is not a Parquet file")) + } + } + + /** + * this is part of test 'Enabling/disabling ignoreCorruptFiles' but run in a loop + * to increase the chance of failure + */ + ignore("SPARK-20407 ParquetQuerySuite 'Enabling/disabling ignoreCorruptFiles' flaky test") { + def testIgnoreCorruptFiles(): Unit = { + withTempDir { dir => + val basePath = dir.getCanonicalPath + spark.range(1).toDF("a").write.parquet(new Path(basePath, "first").toString) + spark.range(1, 2).toDF("a").write.parquet(new Path(basePath, "second").toString) + spark.range(2, 3).toDF("a").write.json(new Path(basePath, "third").toString) + val df = spark.read.parquet( + new Path(basePath, "first").toString, + new Path(basePath, "second").toString, + new Path(basePath, "third").toString) + checkAnswer( + df, + Seq(Row(0), Row(1))) + } + } + + for (i <- 1 to 100) { + DebugFilesystem.clearOpenStreams() + withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") { + val exception = intercept[SparkException] { + testIgnoreCorruptFiles() + } + assert(exception.getMessage().contains("is not a Parquet file")) + } + DebugFilesystem.assertNoOpenStreams() + } + } + test("SPARK-8990 DataFrameReader.parquet() should respect user specified options") { withTempPath { dir => val basePath = dir.getCanonicalPath - sqlContext.range(0, 10).toDF("a").write.parquet(new Path(basePath, "foo=1").toString) - sqlContext.range(0, 10).toDF("b").write.parquet(new Path(basePath, "foo=a").toString) + spark.range(0, 10).toDF("a").write.parquet(new Path(basePath, "foo=1").toString) + spark.range(0, 10).toDF("b").write.parquet(new Path(basePath, "foo=a").toString) // Disables the global SQL option for schema merging withSQLConf(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "false") { assertResult(2) { // Disables schema merging via data source option - sqlContext.read.option("mergeSchema", "false").parquet(basePath).columns.length + spark.read.option("mergeSchema", "false").parquet(basePath).columns.length } assertResult(3) { // Enables schema merging via data source option - sqlContext.read.option("mergeSchema", "true").parquet(basePath).columns.length + spark.read.option("mergeSchema", "true").parquet(basePath).columns.length } } } @@ -198,10 +375,10 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext val basePath = dir.getCanonicalPath val schema = StructType(Array(StructField("name", DecimalType(10, 5), false))) val rowRDD = sparkContext.parallelize(Array(Row(Decimal("67123.45")))) - val df = sqlContext.createDataFrame(rowRDD, schema) + val df = spark.createDataFrame(rowRDD, schema) df.write.parquet(basePath) - val decimal = sqlContext.read.parquet(basePath).first().getDecimal(0) + val decimal = spark.read.parquet(basePath).first().getDecimal(0) assert(Decimal("67123.45") === Decimal(decimal)) } } @@ -221,7 +398,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING.key -> "true") { checkAnswer( - sqlContext.read.option("mergeSchema", "true").parquet(path), + spark.read.option("mergeSchema", "true").parquet(path), Seq( Row(Row(1, 1, null)), Row(Row(2, 2, null)), @@ -234,7 +411,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext test("SPARK-10301 requested schema clipping - same schema") { withTempPath { dir => val path = dir.getCanonicalPath - val df = sqlContext.range(1).selectExpr("NAMED_STRUCT('a', id, 'b', id + 1) AS s").coalesce(1) + val df = spark.range(1).selectExpr("NAMED_STRUCT('a', id, 'b', id + 1) AS s").coalesce(1) df.write.parquet(path) val userDefinedSchema = @@ -247,16 +424,29 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext nullable = true) checkAnswer( - sqlContext.read.schema(userDefinedSchema).parquet(path), + spark.read.schema(userDefinedSchema).parquet(path), Row(Row(0L, 1L))) } } + test("SPARK-11997 parquet with null partition values") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.range(1, 3) + .selectExpr("if(id % 2 = 0, null, id) AS n", "id") + .write.partitionBy("n").parquet(path) + + checkAnswer( + spark.read.parquet(path).filter("n is null"), + Row(2, null)) + } + } + // This test case is ignored because of parquet-mr bug PARQUET-370 ignore("SPARK-10301 requested schema clipping - schemas with disjoint sets of fields") { withTempPath { dir => val path = dir.getCanonicalPath - val df = sqlContext.range(1).selectExpr("NAMED_STRUCT('a', id, 'b', id + 1) AS s").coalesce(1) + val df = spark.range(1).selectExpr("NAMED_STRUCT('a', id, 'b', id + 1) AS s").coalesce(1) df.write.parquet(path) val userDefinedSchema = @@ -269,7 +459,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext nullable = true) checkAnswer( - sqlContext.read.schema(userDefinedSchema).parquet(path), + spark.read.schema(userDefinedSchema).parquet(path), Row(Row(null, null))) } } @@ -277,7 +467,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext test("SPARK-10301 requested schema clipping - requested schema contains physical schema") { withTempPath { dir => val path = dir.getCanonicalPath - val df = sqlContext.range(1).selectExpr("NAMED_STRUCT('a', id, 'b', id + 1) AS s").coalesce(1) + val df = spark.range(1).selectExpr("NAMED_STRUCT('a', id, 'b', id + 1) AS s").coalesce(1) df.write.parquet(path) val userDefinedSchema = @@ -292,13 +482,13 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext nullable = true) checkAnswer( - sqlContext.read.schema(userDefinedSchema).parquet(path), + spark.read.schema(userDefinedSchema).parquet(path), Row(Row(0L, 1L, null, null))) } withTempPath { dir => val path = dir.getCanonicalPath - val df = sqlContext.range(1).selectExpr("NAMED_STRUCT('a', id, 'd', id + 3) AS s").coalesce(1) + val df = spark.range(1).selectExpr("NAMED_STRUCT('a', id, 'd', id + 3) AS s").coalesce(1) df.write.parquet(path) val userDefinedSchema = @@ -313,7 +503,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext nullable = true) checkAnswer( - sqlContext.read.schema(userDefinedSchema).parquet(path), + spark.read.schema(userDefinedSchema).parquet(path), Row(Row(0L, null, null, 3L))) } } @@ -321,7 +511,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext test("SPARK-10301 requested schema clipping - physical schema contains requested schema") { withTempPath { dir => val path = dir.getCanonicalPath - val df = sqlContext + val df = spark .range(1) .selectExpr("NAMED_STRUCT('a', id, 'b', id + 1, 'c', id + 2, 'd', id + 3) AS s") .coalesce(1) @@ -338,13 +528,13 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext nullable = true) checkAnswer( - sqlContext.read.schema(userDefinedSchema).parquet(path), + spark.read.schema(userDefinedSchema).parquet(path), Row(Row(0L, 1L))) } withTempPath { dir => val path = dir.getCanonicalPath - val df = sqlContext + val df = spark .range(1) .selectExpr("NAMED_STRUCT('a', id, 'b', id + 1, 'c', id + 2, 'd', id + 3) AS s") .coalesce(1) @@ -361,7 +551,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext nullable = true) checkAnswer( - sqlContext.read.schema(userDefinedSchema).parquet(path), + spark.read.schema(userDefinedSchema).parquet(path), Row(Row(0L, 3L))) } } @@ -369,7 +559,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext test("SPARK-10301 requested schema clipping - schemas overlap but don't contain each other") { withTempPath { dir => val path = dir.getCanonicalPath - val df = sqlContext + val df = spark .range(1) .selectExpr("NAMED_STRUCT('a', id, 'b', id + 1, 'c', id + 2) AS s") .coalesce(1) @@ -387,7 +577,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext nullable = true) checkAnswer( - sqlContext.read.schema(userDefinedSchema).parquet(path), + spark.read.schema(userDefinedSchema).parquet(path), Row(Row(1L, 2L, null))) } } @@ -396,7 +586,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext withTempPath { dir => val path = dir.getCanonicalPath - val df = sqlContext + val df = spark .range(1) .selectExpr("NAMED_STRUCT('a', ARRAY(NAMED_STRUCT('b', id, 'c', id))) AS s") .coalesce(1) @@ -417,7 +607,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext nullable = true) checkAnswer( - sqlContext.read.schema(userDefinedSchema).parquet(path), + spark.read.schema(userDefinedSchema).parquet(path), Row(Row(Seq(Row(0, null))))) } } @@ -426,12 +616,12 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext withTempPath { dir => val path = dir.getCanonicalPath - val df1 = sqlContext + val df1 = spark .range(1) .selectExpr("NAMED_STRUCT('a', id, 'b', id + 1, 'c', id + 2) AS s") .coalesce(1) - val df2 = sqlContext + val df2 = spark .range(1, 2) .selectExpr("NAMED_STRUCT('c', id + 2, 'b', id + 1, 'd', id + 3) AS s") .coalesce(1) @@ -448,7 +638,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext nullable = true) checkAnswer( - sqlContext.read.schema(userDefinedSchema).parquet(path), + spark.read.schema(userDefinedSchema).parquet(path), Seq( Row(Row(0, 1, null)), Row(Row(null, 2, 4)))) @@ -459,12 +649,12 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext withTempPath { dir => val path = dir.getCanonicalPath - val df1 = sqlContext + val df1 = spark .range(1) .selectExpr("NAMED_STRUCT('a', id, 'c', id + 2) AS s") .coalesce(1) - val df2 = sqlContext + val df2 = spark .range(1, 2) .selectExpr("NAMED_STRUCT('a', id, 'b', id + 1, 'c', id + 2) AS s") .coalesce(1) @@ -473,7 +663,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext df2.write.mode(SaveMode.Append).parquet(path) checkAnswer( - sqlContext + spark .read .option("mergeSchema", "true") .parquet(path) @@ -488,7 +678,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext withTempPath { dir => val path = dir.getCanonicalPath - val df = sqlContext + val df = spark .range(1) .selectExpr( """NAMED_STRUCT( @@ -513,7 +703,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext nullable = true) checkAnswer( - sqlContext.read.schema(userDefinedSchema).parquet(path), + spark.read.schema(userDefinedSchema).parquet(path), Row(Row(NestedStruct(1, 2L, 3.5D)))) } } @@ -521,7 +711,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext test("expand UDT in StructType") { val schema = new StructType().add("n", new NestedStructUDT, nullable = true) val expected = new StructType().add("n", new NestedStructUDT().sqlType, nullable = true) - assert(CatalystReadSupport.expandUDT(schema) === expected) + assert(ParquetReadSupport.expandUDT(schema) === expected) } test("expand UDT in ArrayType") { @@ -539,7 +729,7 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext containsNull = false), nullable = true) - assert(CatalystReadSupport.expandUDT(schema) === expected) + assert(ParquetReadSupport.expandUDT(schema) === expected) } test("expand UDT in MapType") { @@ -559,11 +749,102 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext valueContainsNull = false), nullable = true) - assert(CatalystReadSupport.expandUDT(schema) === expected) + assert(ParquetReadSupport.expandUDT(schema) === expected) + } + + test("returning batch for wide table") { + withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "10") { + withTempPath { dir => + val path = dir.getCanonicalPath + val df = spark.range(10).select(Seq.tabulate(11) {i => ('id + i).as(s"c$i")} : _*) + df.write.mode(SaveMode.Overwrite).parquet(path) + + // donot return batch, because whole stage codegen is disabled for wide table (>200 columns) + val df2 = spark.read.parquet(path) + val fileScan2 = df2.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get + assert(!fileScan2.asInstanceOf[FileSourceScanExec].supportsBatch) + checkAnswer(df2, df) + + // return batch + val columns = Seq.tabulate(9) {i => s"c$i"} + val df3 = df2.selectExpr(columns : _*) + val fileScan3 = df3.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get + assert(fileScan3.asInstanceOf[FileSourceScanExec].supportsBatch) + checkAnswer(df3, df.selectExpr(columns : _*)) + } + } + } + + test("SPARK-15719: disable writing summary files by default") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.range(3).write.parquet(path) + + val fs = FileSystem.get(sparkContext.hadoopConfiguration) + val files = fs.listFiles(new Path(path), true) + + while (files.hasNext) { + val file = files.next + assert(!file.getPath.getName.contains("_metadata")) + } + } + } + + test("SPARK-15804: write out the metadata to parquet file") { + val df = Seq((1, "abc"), (2, "hello")).toDF("a", "b") + val md = new MetadataBuilder().putString("key", "value").build() + val dfWithmeta = df.select('a, 'b.as("b", md)) + + withTempPath { dir => + val path = dir.getCanonicalPath + dfWithmeta.write.parquet(path) + + readParquetFile(path) { df => + assert(df.schema.last.metadata.getString("key") == "value") + } + } + } + + test("SPARK-16344: array of struct with a single field named 'element'") { + withTempPath { dir => + val path = dir.getCanonicalPath + Seq(Tuple1(Array(SingleElement(42)))).toDF("f").write.parquet(path) + + checkAnswer( + sqlContext.read.parquet(path), + Row(Array(Row(42))) + ) + } + } + + test("SPARK-16632: read Parquet int32 as ByteType and ShortType") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { + withTempPath { dir => + val path = dir.getCanonicalPath + + // When being written to Parquet, `TINYINT` and `SMALLINT` should be converted into + // `int32 (INT_8)` and `int32 (INT_16)` respectively. However, Hive doesn't add the `INT_8` + // and `INT_16` annotation properly (HIVE-14294). Thus, when reading files written by Hive + // using Spark with the vectorized Parquet reader enabled, we may hit error due to type + // mismatch. + // + // Here we are simulating Hive's behavior by writing a single `INT` field and then read it + // back as `TINYINT` and `SMALLINT` in Spark to verify this issue. + Seq(1).toDF("f").write.parquet(path) + + val withByteField = new StructType().add("f", ByteType) + checkAnswer(spark.read.schema(withByteField).parquet(path), Row(1: Byte)) + + val withShortField = new StructType().add("f", ShortType) + checkAnswer(spark.read.schema(withShortField).parquet(path), Row(1: Short)) + } + } } } object TestingUDT { + case class SingleElement(element: Long) + @SQLUserDefinedType(udt = classOf[NestedStructUDT]) case class NestedStruct(a: Integer, b: Long, c: Double) @@ -574,14 +855,11 @@ object TestingUDT { .add("b", LongType, nullable = false) .add("c", DoubleType, nullable = false) - override def serialize(obj: Any): Any = { - val row = new SpecificMutableRow(sqlType.asInstanceOf[StructType].map(_.dataType)) - obj match { - case n: NestedStruct => - row.setInt(0, n.a) - row.setLong(1, n.b) - row.setDouble(2, n.c) - } + override def serialize(n: NestedStruct): Any = { + val row = new SpecificInternalRow(sqlType.asInstanceOf[StructType].map(_.dataType)) + row.setInt(0, n.a) + row.setLong(1, n.b) + row.setDouble(2, n.c) } override def userClass: Class[NestedStruct] = classOf[NestedStruct] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala new file mode 100644 index 000000000000..0917f188b979 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala @@ -0,0 +1,356 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import java.io.File + +import scala.collection.JavaConverters._ +import scala.util.Try + +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.{Benchmark, Utils} + +/** + * Benchmark to measure parquet read performance. + * To run this: + * spark-submit --class --jars + */ +object ParquetReadBenchmark { + val conf = new SparkConf() + conf.set("spark.sql.parquet.compression.codec", "snappy") + + val spark = SparkSession.builder + .master("local[1]") + .appName("test-sql-context") + .config(conf) + .getOrCreate() + + // Set default configs. Individual cases will change them if necessary. + spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") + spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + + def withTempPath(f: File => Unit): Unit = { + val path = Utils.createTempDir() + path.delete() + try f(path) finally Utils.deleteRecursively(path) + } + + def withTempTable(tableNames: String*)(f: => Unit): Unit = { + try f finally tableNames.foreach(spark.catalog.dropTempView) + } + + def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + val (keys, values) = pairs.unzip + val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption) + (keys, values).zipped.foreach(spark.conf.set) + try f finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => spark.conf.set(key, value) + case (key, None) => spark.conf.unset(key) + } + } + } + + def intScanBenchmark(values: Int): Unit = { + // Benchmarks running through spark sql. + val sqlBenchmark = new Benchmark("SQL Single Int Column Scan", values) + // Benchmarks driving reader component directly. + val parquetReaderBenchmark = new Benchmark("Parquet Reader Single Int Column Scan", values) + + withTempPath { dir => + withTempTable("t1", "tempTable") { + spark.range(values).createOrReplaceTempView("t1") + spark.sql("select cast(id as INT) as id from t1") + .write.parquet(dir.getCanonicalPath) + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") + + sqlBenchmark.addCase("SQL Parquet Vectorized") { iter => + spark.sql("select sum(id) from tempTable").collect() + } + + sqlBenchmark.addCase("SQL Parquet MR") { iter => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { + spark.sql("select sum(id) from tempTable").collect() + } + } + + val files = SpecificParquetRecordReaderBase.listDirectory(dir).toArray + // Driving the parquet reader in batch mode directly. + parquetReaderBenchmark.addCase("ParquetReader Vectorized") { num => + var sum = 0L + files.map(_.asInstanceOf[String]).foreach { p => + val reader = new VectorizedParquetRecordReader + try { + reader.initialize(p, ("id" :: Nil).asJava) + val batch = reader.resultBatch() + val col = batch.column(0) + while (reader.nextBatch()) { + val numRows = batch.numRows() + var i = 0 + while (i < numRows) { + if (!col.isNullAt(i)) sum += col.getInt(i) + i += 1 + } + } + } finally { + reader.close() + } + } + } + + // Decoding in vectorized but having the reader return rows. + parquetReaderBenchmark.addCase("ParquetReader Vectorized -> Row") { num => + var sum = 0L + files.map(_.asInstanceOf[String]).foreach { p => + val reader = new VectorizedParquetRecordReader + try { + reader.initialize(p, ("id" :: Nil).asJava) + val batch = reader.resultBatch() + while (reader.nextBatch()) { + val it = batch.rowIterator() + while (it.hasNext) { + val record = it.next() + if (!record.isNullAt(0)) sum += record.getInt(0) + } + } + } finally { + reader.close() + } + } + } + + /* + Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz + SQL Single Int Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + SQL Parquet Vectorized 215 / 262 73.0 13.7 1.0X + SQL Parquet MR 1946 / 2083 8.1 123.7 0.1X + */ + sqlBenchmark.run() + + /* + Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz + Parquet Reader Single Int Column Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + ParquetReader Vectorized 123 / 152 127.8 7.8 1.0X + ParquetReader Vectorized -> Row 165 / 180 95.2 10.5 0.7X + */ + parquetReaderBenchmark.run() + } + } + } + + def intStringScanBenchmark(values: Int): Unit = { + withTempPath { dir => + withTempTable("t1", "tempTable") { + spark.range(values).createOrReplaceTempView("t1") + spark.sql("select cast(id as INT) as c1, cast(id as STRING) as c2 from t1") + .write.parquet(dir.getCanonicalPath) + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") + + val benchmark = new Benchmark("Int and String Scan", values) + + benchmark.addCase("SQL Parquet Vectorized") { iter => + spark.sql("select sum(c1), sum(length(c2)) from tempTable").collect + } + + benchmark.addCase("SQL Parquet MR") { iter => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { + spark.sql("select sum(c1), sum(length(c2)) from tempTable").collect + } + } + + val files = SpecificParquetRecordReaderBase.listDirectory(dir).toArray + + /* + Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz + Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + SQL Parquet Vectorized 628 / 720 16.7 59.9 1.0X + SQL Parquet MR 1905 / 2239 5.5 181.7 0.3X + */ + benchmark.run() + } + } + } + + def stringDictionaryScanBenchmark(values: Int): Unit = { + withTempPath { dir => + withTempTable("t1", "tempTable") { + spark.range(values).createOrReplaceTempView("t1") + spark.sql("select cast((id % 200) + 10000 as STRING) as c1 from t1") + .write.parquet(dir.getCanonicalPath) + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") + + val benchmark = new Benchmark("String Dictionary", values) + + benchmark.addCase("SQL Parquet Vectorized") { iter => + spark.sql("select sum(length(c1)) from tempTable").collect + } + + benchmark.addCase("SQL Parquet MR") { iter => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { + spark.sql("select sum(length(c1)) from tempTable").collect + } + } + + /* + Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz + String Dictionary: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + SQL Parquet Vectorized 329 / 337 31.9 31.4 1.0X + SQL Parquet MR 1131 / 1325 9.3 107.8 0.3X + */ + benchmark.run() + } + } + } + + def partitionTableScanBenchmark(values: Int): Unit = { + withTempPath { dir => + withTempTable("t1", "tempTable") { + spark.range(values).createOrReplaceTempView("t1") + spark.sql("select id % 2 as p, cast(id as INT) as id from t1") + .write.partitionBy("p").parquet(dir.getCanonicalPath) + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") + + val benchmark = new Benchmark("Partitioned Table", values) + + benchmark.addCase("Read data column") { iter => + spark.sql("select sum(id) from tempTable").collect + } + + benchmark.addCase("Read partition column") { iter => + spark.sql("select sum(p) from tempTable").collect + } + + benchmark.addCase("Read both columns") { iter => + spark.sql("select sum(p), sum(id) from tempTable").collect + } + + /* + Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz + Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + Read data column 191 / 250 82.1 12.2 1.0X + Read partition column 82 / 86 192.4 5.2 2.3X + Read both columns 220 / 248 71.5 14.0 0.9X + */ + benchmark.run() + } + } + } + + def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = { + withTempPath { dir => + withTempTable("t1", "tempTable") { + spark.range(values).createOrReplaceTempView("t1") + spark.sql(s"select IF(rand(1) < $fractionOfNulls, NULL, cast(id as STRING)) as c1, " + + s"IF(rand(2) < $fractionOfNulls, NULL, cast(id as STRING)) as c2 from t1") + .write.parquet(dir.getCanonicalPath) + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("tempTable") + + val benchmark = new Benchmark("String with Nulls Scan", values) + + benchmark.addCase("SQL Parquet Vectorized") { iter => + spark.sql("select sum(length(c2)) from tempTable where c1 is " + + "not NULL and c2 is not NULL").collect() + } + + val files = SpecificParquetRecordReaderBase.listDirectory(dir).toArray + benchmark.addCase("PR Vectorized") { num => + var sum = 0 + files.map(_.asInstanceOf[String]).foreach { p => + val reader = new VectorizedParquetRecordReader + try { + reader.initialize(p, ("c1" :: "c2" :: Nil).asJava) + val batch = reader.resultBatch() + while (reader.nextBatch()) { + val rowIterator = batch.rowIterator() + while (rowIterator.hasNext) { + val row = rowIterator.next() + val value = row.getUTF8String(0) + if (!row.isNullAt(0) && !row.isNullAt(1)) sum += value.numBytes() + } + } + } finally { + reader.close() + } + } + } + + benchmark.addCase("PR Vectorized (Null Filtering)") { num => + var sum = 0L + files.map(_.asInstanceOf[String]).foreach { p => + val reader = new VectorizedParquetRecordReader + try { + reader.initialize(p, ("c1" :: "c2" :: Nil).asJava) + val batch = reader.resultBatch() + batch.filterNullsInColumn(0) + batch.filterNullsInColumn(1) + while (reader.nextBatch()) { + val rowIterator = batch.rowIterator() + while (rowIterator.hasNext) { + sum += rowIterator.next().getUTF8String(0).numBytes() + } + } + } finally { + reader.close() + } + } + } + + /* + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + String with Nulls Scan (0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + SQL Parquet Vectorized 1229 / 1648 8.5 117.2 1.0X + PR Vectorized 833 / 846 12.6 79.4 1.5X + PR Vectorized (Null Filtering) 732 / 782 14.3 69.8 1.7X + + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + String with Nulls Scan (50%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + SQL Parquet Vectorized 995 / 1053 10.5 94.9 1.0X + PR Vectorized 732 / 772 14.3 69.8 1.4X + PR Vectorized (Null Filtering) 725 / 790 14.5 69.1 1.4X + + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + String with Nulls Scan (95%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------- + SQL Parquet Vectorized 326 / 333 32.2 31.1 1.0X + PR Vectorized 190 / 200 55.1 18.2 1.7X + PR Vectorized (Null Filtering) 168 / 172 62.2 16.1 1.9X + */ + + benchmark.run() + } + } + } + + def main(args: Array[String]): Unit = { + intScanBenchmark(1024 * 1024 * 15) + intStringScanBenchmark(1024 * 1024 * 10) + stringDictionaryScanBenchmark(1024 * 1024 * 10) + partitionTableScanBenchmark(1024 * 1024 * 15) + for (fractionOfNulls <- List(0.0, 0.50, 0.95)) { + stringWithNullsScanBenchmark(1024 * 1024 * 10, fractionOfNulls) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index 60fa81b1ab81..ce992674d719 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -20,8 +20,9 @@ package org.apache.spark.sql.execution.datasources.parquet import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag -import org.apache.parquet.schema.MessageTypeParser +import org.apache.parquet.schema.{MessageType, MessageTypeParser} +import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -52,11 +53,13 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSQLContext { parquetSchema: String, binaryAsString: Boolean, int96AsTimestamp: Boolean, - writeLegacyParquetFormat: Boolean): Unit = { - val converter = new CatalystSchemaConverter( + writeLegacyParquetFormat: Boolean, + int64AsTimestampMillis: Boolean = false): Unit = { + val converter = new ParquetSchemaConverter( assumeBinaryIsString = binaryAsString, assumeInt96IsTimestamp = int96AsTimestamp, - writeLegacyParquetFormat = writeLegacyParquetFormat) + writeLegacyParquetFormat = writeLegacyParquetFormat, + writeTimestampInMillis = int64AsTimestampMillis) test(s"sql <= parquet: $testName") { val actual = converter.convert(MessageTypeParser.parseMessageType(parquetSchema)) @@ -76,11 +79,13 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSQLContext { parquetSchema: String, binaryAsString: Boolean, int96AsTimestamp: Boolean, - writeLegacyParquetFormat: Boolean): Unit = { - val converter = new CatalystSchemaConverter( + writeLegacyParquetFormat: Boolean, + int64AsTimestampMillis: Boolean = false): Unit = { + val converter = new ParquetSchemaConverter( assumeBinaryIsString = binaryAsString, assumeInt96IsTimestamp = int96AsTimestamp, - writeLegacyParquetFormat = writeLegacyParquetFormat) + writeLegacyParquetFormat = writeLegacyParquetFormat, + writeTimestampInMillis = int64AsTimestampMillis) test(s"sql => parquet: $testName") { val actual = converter.convert(sqlSchema) @@ -96,7 +101,8 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSQLContext { parquetSchema: String, binaryAsString: Boolean, int96AsTimestamp: Boolean, - writeLegacyParquetFormat: Boolean): Unit = { + writeLegacyParquetFormat: Boolean, + int64AsTimestampMillis: Boolean = false): Unit = { testCatalystToParquet( testName, @@ -104,7 +110,8 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSQLContext { parquetSchema, binaryAsString, int96AsTimestamp, - writeLegacyParquetFormat) + writeLegacyParquetFormat, + int64AsTimestampMillis) testParquetToCatalyst( testName, @@ -112,7 +119,8 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSQLContext { parquetSchema, binaryAsString, int96AsTimestamp, - writeLegacyParquetFormat) + writeLegacyParquetFormat, + int64AsTimestampMillis) } } @@ -260,7 +268,7 @@ class ParquetSchemaInferenceSuite extends ParquetSchemaTest { int96AsTimestamp = true, writeLegacyParquetFormat = true) - testSchemaInference[Tuple1[Pair[Int, String]]]( + testSchemaInference[Tuple1[(Int, String)]]( "struct", """ |message root { @@ -367,86 +375,20 @@ class ParquetSchemaSuite extends ParquetSchemaTest { } } - test("merge with metastore schema") { - // Field type conflict resolution - assertResult( - StructType(Seq( - StructField("lowerCase", StringType), - StructField("UPPERCase", DoubleType, nullable = false)))) { - - ParquetRelation.mergeMetastoreParquetSchema( - StructType(Seq( - StructField("lowercase", StringType), - StructField("uppercase", DoubleType, nullable = false))), - - StructType(Seq( - StructField("lowerCase", BinaryType), - StructField("UPPERCase", IntegerType, nullable = true)))) - } - - // MetaStore schema is subset of parquet schema - assertResult( - StructType(Seq( - StructField("UPPERCase", DoubleType, nullable = false)))) { - - ParquetRelation.mergeMetastoreParquetSchema( - StructType(Seq( - StructField("uppercase", DoubleType, nullable = false))), + test("schema merging failure error message") { + import testImplicits._ - StructType(Seq( - StructField("lowerCase", BinaryType), - StructField("UPPERCase", IntegerType, nullable = true)))) - } + withTempPath { dir => + val path = dir.getCanonicalPath + spark.range(3).write.parquet(s"$path/p=1") + spark.range(3).select('id cast IntegerType as 'id).write.parquet(s"$path/p=2") - // Metastore schema contains additional non-nullable fields. - assert(intercept[Throwable] { - ParquetRelation.mergeMetastoreParquetSchema( - StructType(Seq( - StructField("uppercase", DoubleType, nullable = false), - StructField("lowerCase", BinaryType, nullable = false))), - - StructType(Seq( - StructField("UPPERCase", IntegerType, nullable = true)))) - }.getMessage.contains("detected conflicting schemas")) - - // Conflicting non-nullable field names - intercept[Throwable] { - ParquetRelation.mergeMetastoreParquetSchema( - StructType(Seq(StructField("lower", StringType, nullable = false))), - StructType(Seq(StructField("lowerCase", BinaryType)))) - } - } + val message = intercept[SparkException] { + spark.read.option("mergeSchema", "true").parquet(path).schema + }.getMessage - test("merge missing nullable fields from Metastore schema") { - // Standard case: Metastore schema contains additional nullable fields not present - // in the Parquet file schema. - assertResult( - StructType(Seq( - StructField("firstField", StringType, nullable = true), - StructField("secondField", StringType, nullable = true), - StructField("thirdfield", StringType, nullable = true)))) { - ParquetRelation.mergeMetastoreParquetSchema( - StructType(Seq( - StructField("firstfield", StringType, nullable = true), - StructField("secondfield", StringType, nullable = true), - StructField("thirdfield", StringType, nullable = true))), - StructType(Seq( - StructField("firstField", StringType, nullable = true), - StructField("secondField", StringType, nullable = true)))) + assert(message.contains("Failed merging schema")) } - - // Merge should fail if the Metastore contains any additional fields that are not - // nullable. - assert(intercept[Throwable] { - ParquetRelation.mergeMetastoreParquetSchema( - StructType(Seq( - StructField("firstfield", StringType, nullable = true), - StructField("secondfield", StringType, nullable = true), - StructField("thirdfield", StringType, nullable = false))), - StructType(Seq( - StructField("firstField", StringType, nullable = true), - StructField("secondField", StringType, nullable = true)))) - }.getMessage.contains("detected conflicting schemas")) } // ======================================================= @@ -1030,23 +972,43 @@ class ParquetSchemaSuite extends ParquetSchemaTest { int96AsTimestamp = true, writeLegacyParquetFormat = true) + testSchema( + "Timestamp written and read as INT64 with TIMESTAMP_MILLIS", + StructType(Seq(StructField("f1", TimestampType))), + """message root { + | optional INT64 f1 (TIMESTAMP_MILLIS); + |} + """.stripMargin, + binaryAsString = true, + int96AsTimestamp = false, + writeLegacyParquetFormat = true, + int64AsTimestampMillis = true) + private def testSchemaClipping( testName: String, parquetSchema: String, catalystSchema: StructType, expectedSchema: String): Unit = { + testSchemaClipping(testName, parquetSchema, catalystSchema, + MessageTypeParser.parseMessageType(expectedSchema)) + } + + private def testSchemaClipping( + testName: String, + parquetSchema: String, + catalystSchema: StructType, + expectedSchema: MessageType): Unit = { test(s"Clipping - $testName") { - val expected = MessageTypeParser.parseMessageType(expectedSchema) - val actual = CatalystReadSupport.clipParquetSchema( + val actual = ParquetReadSupport.clipParquetSchema( MessageTypeParser.parseMessageType(parquetSchema), catalystSchema) try { - expected.checkContains(actual) - actual.checkContains(expected) + expectedSchema.checkContains(actual) + actual.checkContains(expectedSchema) } catch { case cause: Throwable => fail( s"""Expected clipped schema: - |$expected + |$expectedSchema |Actual clipped schema: |$actual """.stripMargin, @@ -1399,7 +1361,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { catalystSchema = new StructType(), - expectedSchema = "message root {}") + expectedSchema = ParquetSchemaConverter.EMPTY_MESSAGE) testSchemaClipping( "disjoint field sets", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala index 8ffb01fc5b58..85efca3c4b24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala @@ -26,12 +26,14 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.parquet.format.converter.ParquetMetadataConverter -import org.apache.parquet.hadoop.metadata.{BlockMetaData, FileMetaData, ParquetMetadata} import org.apache.parquet.hadoop.{Footer, ParquetFileReader, ParquetFileWriter} +import org.apache.parquet.hadoop.metadata.{BlockMetaData, FileMetaData, ParquetMetadata} +import org.apache.parquet.schema.MessageType +import org.apache.spark.sql.{DataFrame, SaveMode} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{DataFrame, SQLConf, SaveMode} /** * A helper trait that provides convenient facilities for Parquet testing. @@ -42,6 +44,20 @@ import org.apache.spark.sql.{DataFrame, SQLConf, SaveMode} */ private[sql] trait ParquetTest extends SQLTestUtils { + /** + * Reads the parquet file at `path` + */ + protected def readParquetFile(path: String, testVectorized: Boolean = true) + (f: DataFrame => Unit) = { + (true :: false :: Nil).foreach { vectorized => + if (!vectorized || testVectorized) { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + f(spark.read.parquet(path.toString)) + } + } + } + } + /** * Writes `data` to a Parquet file, which is then passed to `f` and will be deleted after `f` * returns. @@ -50,7 +66,7 @@ private[sql] trait ParquetTest extends SQLTestUtils { (data: Seq[T]) (f: String => Unit): Unit = { withTempPath { file => - sqlContext.createDataFrame(data).write.parquet(file.getCanonicalPath) + spark.createDataFrame(data).write.parquet(file.getCanonicalPath) f(file.getCanonicalPath) } } @@ -60,9 +76,9 @@ private[sql] trait ParquetTest extends SQLTestUtils { * which is then passed to `f`. The Parquet file will be deleted after `f` returns. */ protected def withParquetDataFrame[T <: Product: ClassTag: TypeTag] - (data: Seq[T]) + (data: Seq[T], testVectorized: Boolean = true) (f: DataFrame => Unit): Unit = { - withParquetFile(data)(path => f(sqlContext.read.parquet(path))) + withParquetFile(data)(path => readParquetFile(path.toString, testVectorized)(f)) } /** @@ -71,17 +87,17 @@ private[sql] trait ParquetTest extends SQLTestUtils { * Parquet file will be dropped/deleted after `f` returns. */ protected def withParquetTable[T <: Product: ClassTag: TypeTag] - (data: Seq[T], tableName: String) + (data: Seq[T], tableName: String, testVectorized: Boolean = true) (f: => Unit): Unit = { - withParquetDataFrame(data) { df => - sqlContext.registerDataFrameAsTable(df, tableName) - withTempTable(tableName)(f) + withParquetDataFrame(data, testVectorized) { df => + df.createOrReplaceTempView(tableName) + withTempView(tableName)(f) } } protected def makeParquetFile[T <: Product: ClassTag: TypeTag]( data: Seq[T], path: File): Unit = { - sqlContext.createDataFrame(data).write.mode(SaveMode.Overwrite).parquet(path.getCanonicalPath) + spark.createDataFrame(data).write.mode(SaveMode.Overwrite).parquet(path.getCanonicalPath) } protected def makeParquetFile[T <: Product: ClassTag: TypeTag]( @@ -108,8 +124,8 @@ private[sql] trait ParquetTest extends SQLTestUtils { protected def writeMetadata( schema: StructType, path: Path, configuration: Configuration): Unit = { - val parquetSchema = new CatalystSchemaConverter().convert(schema) - val extraMetadata = Map(CatalystReadSupport.SPARK_METADATA_KEY -> schema.json).asJava + val parquetSchema = new ParquetSchemaConverter().convert(schema) + val extraMetadata = Map(ParquetReadSupport.SPARK_METADATA_KEY -> schema.json).asJava val createdBy = s"Apache Spark ${org.apache.spark.SPARK_VERSION}" val fileMetadata = new FileMetaData(parquetSchema, extraMetadata, createdBy) val parquetMetadata = new ParquetMetadata(fileMetadata, Seq.empty[BlockMetaData].asJava) @@ -117,6 +133,21 @@ private[sql] trait ParquetTest extends SQLTestUtils { ParquetFileWriter.writeMetadataFile(configuration, path, Seq(footer).asJava) } + /** + * This is an overloaded version of `writeMetadata` above to allow writing customized + * Parquet schema. + */ + protected def writeMetadata( + parquetSchema: MessageType, path: Path, configuration: Configuration, + extraMetadata: Map[String, String] = Map.empty[String, String]): Unit = { + val extraMetadataAsJava = extraMetadata.asJava + val createdBy = s"Apache Spark ${org.apache.spark.SPARK_VERSION}" + val fileMetadata = new FileMetaData(parquetSchema, extraMetadataAsJava, createdBy) + val parquetMetadata = new ParquetMetadata(fileMetadata, Seq.empty[BlockMetaData].asJava) + val footer = new Footer(path, parquetMetadata) + ParquetFileWriter.writeMetadataFile(configuration, path, Seq(footer).asJava) + } + protected def readAllFootersWithoutSummaryFiles( path: Path, configuration: Configuration): Seq[Footer] = { val fs = path.getFileSystem(configuration) @@ -142,6 +173,6 @@ private[sql] trait ParquetTest extends SQLTestUtils { protected def readResourceParquetFile(name: String): DataFrame = { val url = Thread.currentThread().getContextClassLoader.getResource(name) - sqlContext.read.parquet(url.toString) + spark.read.parquet(url.toString) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala index 88a3d878f97f..4157a5b46dc4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala @@ -23,8 +23,8 @@ import org.apache.spark.sql.test.SharedSQLContext class ParquetThriftCompatibilitySuite extends ParquetCompatibilityTest with SharedSQLContext { import ParquetCompatibilityTest._ - private val parquetFilePath = - Thread.currentThread().getContextClassLoader.getResource("parquet-thrift-compat.snappy.parquet") + private val parquetFilePath = Thread.currentThread().getContextClassLoader.getResource( + "test-data/parquet-thrift-compat.snappy.parquet") test("Read Parquet file generated by parquet-thrift") { logInfo( @@ -32,7 +32,7 @@ class ParquetThriftCompatibilitySuite extends ParquetCompatibilityTest with Shar |${readParquetSchema(parquetFilePath.toString)} """.stripMargin) - checkAnswer(sqlContext.read.parquet(parquetFilePath.toString), (0 until 10).map { i => + checkAnswer(spark.read.parquet(parquetFilePath.toString), (0 until 10).map { i => val suits = Array("SPADES", "HEARTS", "DIAMONDS", "CLUBS") val nonNullablePrimitiveValues = Seq( @@ -139,7 +139,7 @@ class ParquetThriftCompatibilitySuite extends ParquetCompatibilityTest with Shar logParquetSchema(path) checkAnswer( - sqlContext.read.parquet(path), + spark.read.parquet(path), Seq( Row(Seq(Seq(0, 1), Seq(2, 3))), Row(Seq(Seq(4, 5), Seq(6, 7))))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index 0a2306c06646..cb7393cdd2b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -17,29 +17,35 @@ package org.apache.spark.sql.execution.datasources.text +import java.io.File + +import org.apache.hadoop.io.SequenceFile.CompressionType +import org.apache.hadoop.io.compress.GzipCodec + +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{StringType, StructType} -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.util.Utils - class TextSuite extends QueryTest with SharedSQLContext { + import testImplicits._ test("reading text file") { - verifyFrame(sqlContext.read.format("text").load(testFile)) + verifyFrame(spark.read.format("text").load(testFile)) } test("SQLContext.read.text() API") { - verifyFrame(sqlContext.read.text(testFile)) + verifyFrame(spark.read.text(testFile)) } - test("writing") { - val df = sqlContext.read.text(testFile) + test("SPARK-12562 verify write.text() can handle column name beyond `value`") { + val df = spark.read.text(testFile).withColumnRenamed("value", "adwrasdf") val tempFile = Utils.createTempDir() tempFile.delete() df.write.text(tempFile.getCanonicalPath) - verifyFrame(sqlContext.read.text(tempFile.getCanonicalPath)) + verifyFrame(spark.read.text(tempFile.getCanonicalPath)) Utils.deleteRecursively(tempFile) } @@ -48,24 +54,132 @@ class TextSuite extends QueryTest with SharedSQLContext { val tempFile = Utils.createTempDir() tempFile.delete() - val df = sqlContext.range(2) + val df = spark.range(2) intercept[AnalysisException] { df.write.text(tempFile.getCanonicalPath) } intercept[AnalysisException] { - sqlContext.range(2).select(df("id"), df("id") + 1).write.text(tempFile.getCanonicalPath) + spark.range(2).select(df("id"), df("id") + 1).write.text(tempFile.getCanonicalPath) + } + } + + test("reading partitioned data using read.textFile()") { + val partitionedData = Thread.currentThread().getContextClassLoader + .getResource("test-data/text-partitioned").toString + val ds = spark.read.textFile(partitionedData) + val data = ds.collect() + + assert(ds.schema == new StructType().add("value", StringType)) + assert(data.length == 2) + } + + test("support for partitioned reading using read.text()") { + val partitionedData = Thread.currentThread().getContextClassLoader + .getResource("test-data/text-partitioned").toString + val df = spark.read.text(partitionedData) + val data = df.filter("year = '2015'").select("value").collect() + + assert(data(0) == Row("2015-test")) + assert(data.length == 1) + } + + test("SPARK-13503 Support to specify the option for compression codec for TEXT") { + val testDf = spark.read.text(testFile) + val extensionNameMap = Map("bzip2" -> ".bz2", "deflate" -> ".deflate", "gzip" -> ".gz") + extensionNameMap.foreach { + case (codecName, extension) => + val tempDir = Utils.createTempDir() + val tempDirPath = tempDir.getAbsolutePath + testDf.write.option("compression", codecName).mode(SaveMode.Overwrite).text(tempDirPath) + val compressedFiles = new File(tempDirPath).listFiles() + assert(compressedFiles.exists(_.getName.endsWith(s".txt$extension"))) + verifyFrame(spark.read.text(tempDirPath)) + } + + val errMsg = intercept[IllegalArgumentException] { + val tempDirPath = Utils.createTempDir().getAbsolutePath + testDf.write.option("compression", "illegal").mode(SaveMode.Overwrite).text(tempDirPath) + } + assert(errMsg.getMessage.contains("Codec [illegal] is not available. " + + "Known codecs are")) + } + + test("SPARK-13543 Write the output as uncompressed via option()") { + val extraOptions = Map[String, String]( + "mapreduce.output.fileoutputformat.compress" -> "true", + "mapreduce.output.fileoutputformat.compress.type" -> CompressionType.BLOCK.toString, + "mapreduce.map.output.compress" -> "true", + "mapreduce.output.fileoutputformat.compress.codec" -> classOf[GzipCodec].getName, + "mapreduce.map.output.compress.codec" -> classOf[GzipCodec].getName + ) + withTempDir { dir => + val testDf = spark.read.text(testFile) + val tempDirPath = dir.getAbsolutePath + testDf.write.option("compression", "none") + .options(extraOptions).mode(SaveMode.Overwrite).text(tempDirPath) + val compressedFiles = new File(tempDirPath).listFiles() + assert(compressedFiles.exists(!_.getName.endsWith(".txt.gz"))) + verifyFrame(spark.read.options(extraOptions).text(tempDirPath)) + } + } + + test("case insensitive option") { + val extraOptions = Map[String, String]( + "mApReDuCe.output.fileoutputformat.compress" -> "true", + "mApReDuCe.output.fileoutputformat.compress.type" -> CompressionType.BLOCK.toString, + "mApReDuCe.map.output.compress" -> "true", + "mApReDuCe.output.fileoutputformat.compress.codec" -> classOf[GzipCodec].getName, + "mApReDuCe.map.output.compress.codec" -> classOf[GzipCodec].getName + ) + withTempDir { dir => + val testDf = spark.read.text(testFile) + val tempDirPath = dir.getAbsolutePath + testDf.write.option("CoMpReSsIoN", "none") + .options(extraOptions).mode(SaveMode.Overwrite).text(tempDirPath) + val compressedFiles = new File(tempDirPath).listFiles() + assert(compressedFiles.exists(!_.getName.endsWith(".txt.gz"))) + verifyFrame(spark.read.options(extraOptions).text(tempDirPath)) + } + } + + test("SPARK-14343: select partitioning column") { + withTempPath { dir => + val path = dir.getCanonicalPath + val ds1 = spark.range(1).selectExpr("CONCAT('val_', id)") + ds1.write.text(s"$path/part=a") + ds1.write.text(s"$path/part=b") + + checkAnswer( + spark.read.format("text").load(path).select($"part"), + Row("a") :: Row("b") :: Nil) + } + } + + test("SPARK-15654: should not split gz files") { + withTempDir { dir => + val path = dir.getCanonicalPath + val df1 = spark.range(0, 1000).selectExpr("CAST(id AS STRING) AS s") + df1.write.option("compression", "gzip").mode("overwrite").text(path) + + val expected = df1.collect() + Seq(10, 100, 1000).foreach { bytes => + withSQLConf(SQLConf.FILES_MAX_PARTITION_BYTES.key -> bytes.toString) { + val df2 = spark.read.format("text").load(path) + checkAnswer(df2, expected) + } + } } } private def testFile: String = { - Thread.currentThread().getContextClassLoader.getResource("text-suite.txt").toString + Thread.currentThread().getContextClassLoader.getResource("test-data/text-suite.txt").toString } /** Verifies data and schema. */ private def verifyFrame(df: DataFrame): Unit = { // schema - assert(df.schema == new StructType().add("text", StringType)) + assert(df.schema == new StructType().add("value", StringType)) // verify content val data = df.collect() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index 22189477d277..adcaf2d76519 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -19,10 +19,30 @@ package org.apache.spark.sql.execution.debug import org.apache.spark.SparkFunSuite import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SQLTestData.TestData class DebuggingSuite extends SparkFunSuite with SharedSQLContext { test("DataFrame.debug()") { testData.debug() } + + test("Dataset.debug()") { + import testImplicits._ + testData.as[TestData].debug() + } + + test("debugCodegen") { + val res = codegenString(spark.range(10).groupBy("id").count().queryExecution.executedPlan) + assert(res.contains("Subtree 1 / 2")) + assert(res.contains("Subtree 2 / 2")) + assert(res.contains("Object[]")) + } + + test("debugCodegenStringSeq") { + val res = codegenStringSeq(spark.range(10).groupBy("id").count().queryExecution.executedPlan) + assert(res.length == 2) + assert(res.forall{ case (subtree, code) => + subtree.contains("Range") && code.contains("Object[]")}) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index dcbfdca71acb..a0fad862b44c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -1,85 +1,226 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.sql.execution.joins import scala.reflect.ClassTag -import org.scalatest.BeforeAndAfterAll - -import org.apache.spark.{AccumulatorSuite, SparkConf, SparkContext} +import org.apache.spark.AccumulatorSuite +import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{BitwiseAnd, BitwiseOr, Cast, Literal, ShiftLeft} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.functions._ -import org.apache.spark.sql.{SQLConf, SQLContext, QueryTest} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.{LongType, ShortType} /** - * Test various broadcast join operators with unsafe enabled. + * Test various broadcast join operators. * * Tests in this suite we need to run Spark in local-cluster mode. In particular, the use of * unsafe map in [[org.apache.spark.sql.execution.joins.UnsafeHashedRelation]] is not triggered * without serializing the hashed relation, which does not happen in local mode. */ -class BroadcastJoinSuite extends QueryTest with BeforeAndAfterAll { - protected var sqlContext: SQLContext = null +class BroadcastJoinSuite extends QueryTest with SQLTestUtils { + import testImplicits._ + + protected var spark: SparkSession = null /** - * Create a new [[SQLContext]] running in local-cluster mode with unsafe and codegen enabled. + * Create a new [[SparkSession]] running in local-cluster mode with unsafe and codegen enabled. */ override def beforeAll(): Unit = { super.beforeAll() - val conf = new SparkConf() - .setMaster("local-cluster[2,1,1024]") - .setAppName("testing") - val sc = new SparkContext(conf) - sqlContext = new SQLContext(sc) - sqlContext.setConf(SQLConf.UNSAFE_ENABLED, true) - sqlContext.setConf(SQLConf.CODEGEN_ENABLED, true) + spark = SparkSession.builder() + .master("local-cluster[2,1,1024]") + .appName("testing") + .getOrCreate() } override def afterAll(): Unit = { - sqlContext.sparkContext.stop() - sqlContext = null + spark.stop() + spark = null } /** * Test whether the specified broadcast join updates the peak execution memory accumulator. */ - private def testBroadcastJoin[T: ClassTag](name: String, joinType: String): Unit = { - AccumulatorSuite.verifyPeakExecutionMemorySet(sqlContext.sparkContext, name) { - val df1 = sqlContext.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value") - val df2 = sqlContext.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value") - // Comparison at the end is for broadcast left semi join - val joinExpression = df1("key") === df2("key") && df1("value") > df2("value") - val df3 = df1.join(broadcast(df2), joinExpression, joinType) - val plan = df3.queryExecution.executedPlan - assert(plan.collect { case p: T => p }.size === 1) + private def testBroadcastJoinPeak[T: ClassTag](name: String, joinType: String): Unit = { + AccumulatorSuite.verifyPeakExecutionMemorySet(spark.sparkContext, name) { + val plan = testBroadcastJoin[T](joinType) plan.executeCollect() } } + private def testBroadcastJoin[T: ClassTag]( + joinType: String, + forceBroadcast: Boolean = false): SparkPlan = { + val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value") + val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value") + + // Comparison at the end is for broadcast left semi join + val joinExpression = df1("key") === df2("key") && df1("value") > df2("value") + val df3 = if (forceBroadcast) { + df1.join(broadcast(df2), joinExpression, joinType) + } else { + df1.join(df2, joinExpression, joinType) + } + val plan = EnsureRequirements(spark.sessionState.conf).apply(df3.queryExecution.sparkPlan) + assert(plan.collect { case p: T => p }.size === 1) + plan + } + test("unsafe broadcast hash join updates peak execution memory") { - testBroadcastJoin[BroadcastHashJoin]("unsafe broadcast hash join", "inner") + testBroadcastJoinPeak[BroadcastHashJoinExec]("unsafe broadcast hash join", "inner") } test("unsafe broadcast hash outer join updates peak execution memory") { - testBroadcastJoin[BroadcastHashOuterJoin]("unsafe broadcast hash outer join", "left_outer") + testBroadcastJoinPeak[BroadcastHashJoinExec]("unsafe broadcast hash outer join", "left_outer") } test("unsafe broadcast left semi join updates peak execution memory") { - testBroadcastJoin[BroadcastLeftSemiJoinHash]("unsafe broadcast left semi join", "leftsemi") + testBroadcastJoinPeak[BroadcastHashJoinExec]("unsafe broadcast left semi join", "leftsemi") + } + + test("broadcast hint isn't bothered by authBroadcastJoinThreshold set to low values") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0") { + testBroadcastJoin[BroadcastHashJoinExec]("inner", true) + } + } + + test("broadcast hint isn't bothered by a disabled authBroadcastJoinThreshold") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + testBroadcastJoin[BroadcastHashJoinExec]("inner", true) + } + } + + test("broadcast hint isn't propagated after a join") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value") + val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value") + val df3 = df1.join(broadcast(df2), Seq("key"), "inner").drop(df2("key")) + + val df4 = spark.createDataFrame(Seq((1, "5"), (2, "5"))).toDF("key", "value") + val df5 = df4.join(df3, Seq("key"), "inner") + + val plan = + EnsureRequirements(spark.sessionState.conf).apply(df5.queryExecution.sparkPlan) + + assert(plan.collect { case p: BroadcastHashJoinExec => p }.size === 1) + assert(plan.collect { case p: SortMergeJoinExec => p }.size === 1) + } } + private def assertBroadcastJoin(df : Dataset[Row]) : Unit = { + val df1 = spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value") + val joined = df1.join(df, Seq("key"), "inner") + + val plan = + EnsureRequirements(spark.sessionState.conf).apply(joined.queryExecution.sparkPlan) + + assert(plan.collect { case p: BroadcastHashJoinExec => p }.size === 1) + } + + test("broadcast hint programming API") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"), (3, "2"))).toDF("key", "value") + val broadcasted = broadcast(df2) + val df3 = spark.createDataFrame(Seq((2, "2"), (3, "3"))).toDF("key", "value") + + val cases = Seq(broadcasted.limit(2), + broadcasted.filter("value < 10"), + broadcasted.sample(true, 0.5), + broadcasted.distinct(), + broadcasted.groupBy("value").agg(min($"key").as("key")), + // except and intersect are semi/anti-joins which won't return more data then + // their left argument, so the broadcast hint should be propagated here + broadcasted.except(df3), + broadcasted.intersect(df3)) + + cases.foreach(assertBroadcastJoin) + } + } + + test("broadcast hint in SQL") { + import org.apache.spark.sql.catalyst.plans.logical.{ResolvedHint, Join} + + spark.range(10).createOrReplaceTempView("t") + spark.range(10).createOrReplaceTempView("u") + + for (name <- Seq("BROADCAST", "BROADCASTJOIN", "MAPJOIN")) { + val plan1 = sql(s"SELECT /*+ $name(t) */ * FROM t JOIN u ON t.id = u.id").queryExecution + .optimizedPlan + val plan2 = sql(s"SELECT /*+ $name(u) */ * FROM t JOIN u ON t.id = u.id").queryExecution + .optimizedPlan + val plan3 = sql(s"SELECT /*+ $name(v) */ * FROM t JOIN u ON t.id = u.id").queryExecution + .optimizedPlan + + assert(plan1.asInstanceOf[Join].left.isInstanceOf[ResolvedHint]) + assert(!plan1.asInstanceOf[Join].right.isInstanceOf[ResolvedHint]) + assert(!plan2.asInstanceOf[Join].left.isInstanceOf[ResolvedHint]) + assert(plan2.asInstanceOf[Join].right.isInstanceOf[ResolvedHint]) + assert(!plan3.asInstanceOf[Join].left.isInstanceOf[ResolvedHint]) + assert(!plan3.asInstanceOf[Join].right.isInstanceOf[ResolvedHint]) + } + } + + test("join key rewritten") { + val l = Literal(1L) + val i = Literal(2) + val s = Literal.create(3, ShortType) + val ss = Literal("hello") + + assert(HashJoin.rewriteKeyExpr(l :: Nil) === l :: Nil) + assert(HashJoin.rewriteKeyExpr(l :: l :: Nil) === l :: l :: Nil) + assert(HashJoin.rewriteKeyExpr(l :: i :: Nil) === l :: i :: Nil) + + assert(HashJoin.rewriteKeyExpr(i :: Nil) === Cast(i, LongType) :: Nil) + assert(HashJoin.rewriteKeyExpr(i :: l :: Nil) === i :: l :: Nil) + assert(HashJoin.rewriteKeyExpr(i :: i :: Nil) === + BitwiseOr(ShiftLeft(Cast(i, LongType), Literal(32)), + BitwiseAnd(Cast(i, LongType), Literal((1L << 32) - 1))) :: Nil) + assert(HashJoin.rewriteKeyExpr(i :: i :: i :: Nil) === i :: i :: i :: Nil) + + assert(HashJoin.rewriteKeyExpr(s :: Nil) === Cast(s, LongType) :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: l :: Nil) === s :: l :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: s :: Nil) === + BitwiseOr(ShiftLeft(Cast(s, LongType), Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))) :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: s :: s :: Nil) === + BitwiseOr(ShiftLeft( + BitwiseOr(ShiftLeft(Cast(s, LongType), Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))), + Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))) :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: s :: s :: s :: Nil) === + BitwiseOr(ShiftLeft( + BitwiseOr(ShiftLeft( + BitwiseOr(ShiftLeft(Cast(s, LongType), Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))), + Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))), + Literal(16)), + BitwiseAnd(Cast(s, LongType), Literal((1L << 16) - 1))) :: Nil) + assert(HashJoin.rewriteKeyExpr(s :: s :: s :: s :: s :: Nil) === + s :: s :: s :: s :: s :: Nil) + + assert(HashJoin.rewriteKeyExpr(ss :: Nil) === ss :: Nil) + assert(HashJoin.rewriteKeyExpr(l :: ss :: Nil) === l :: ss :: Nil) + assert(HashJoin.rewriteKeyExpr(i :: ss :: Nil) === i :: ss :: Nil) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala new file mode 100644 index 000000000000..38377164c10e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.joins + +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.Join +import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan, SparkPlanTest} +import org.apache.spark.sql.execution.exchange.EnsureRequirements +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{BooleanType, DoubleType, IntegerType, StructType} + +class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { + + private lazy val left = spark.createDataFrame( + sparkContext.parallelize(Seq( + Row(1, 2.0), + Row(1, 2.0), + Row(2, 1.0), + Row(2, 1.0), + Row(3, 3.0), + Row(null, null), + Row(null, 5.0), + Row(6, null) + )), new StructType().add("a", IntegerType).add("b", DoubleType)) + + private lazy val right = spark.createDataFrame( + sparkContext.parallelize(Seq( + Row(2, 3.0), + Row(2, 3.0), + Row(3, 2.0), + Row(4, 1.0), + Row(null, null), + Row(null, 5.0), + Row(6, null) + )), new StructType().add("c", IntegerType).add("d", DoubleType)) + + private lazy val rightUniqueKey = spark.createDataFrame( + sparkContext.parallelize(Seq( + Row(2, 3.0), + Row(3, 2.0), + Row(4, 1.0), + Row(null, 5.0), + Row(6, null) + )), new StructType().add("c", IntegerType).add("d", DoubleType)) + + private lazy val singleConditionEQ = (left.col("a") === right.col("c")).expr + + private lazy val composedConditionEQ = { + And((left.col("a") === right.col("c")).expr, + LessThan(left.col("b").expr, right.col("d").expr)) + } + + private lazy val composedConditionNEQ = { + And((left.col("a") < right.col("c")).expr, + LessThan(left.col("b").expr, right.col("d").expr)) + } + + // Note: the input dataframes and expression must be evaluated lazily because + // the SQLContext should be used only within a test to keep SQL tests stable + private def testExistenceJoin( + testName: String, + joinType: JoinType, + leftRows: => DataFrame, + rightRows: => DataFrame, + condition: => Expression, + expectedAnswer: Seq[Row]): Unit = { + + def extractJoinParts(): Option[ExtractEquiJoinKeys.ReturnType] = { + val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, Inner, Some(condition)) + ExtractEquiJoinKeys.unapply(join) + } + + val existsAttr = AttributeReference("exists", BooleanType, false)() + val leftSemiPlus = ExistenceJoin(existsAttr) + def createLeftSemiPlusJoin(join: SparkPlan): SparkPlan = { + val output = join.output.dropRight(1) + val condition = if (joinType == LeftSemi) { + existsAttr + } else { + Not(existsAttr) + } + ProjectExec(output, FilterExec(condition, join)) + } + + test(s"$testName using ShuffledHashJoin") { + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + ShuffledHashJoinExec( + leftKeys, rightKeys, joinType, BuildRight, boundCondition, left, right)), + expectedAnswer, + sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(ShuffledHashJoinExec( + leftKeys, rightKeys, leftSemiPlus, BuildRight, boundCondition, left, right))), + expectedAnswer, + sortAnswers = true) + } + } + } + + test(s"$testName using BroadcastHashJoin") { + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + BroadcastHashJoinExec( + leftKeys, rightKeys, joinType, BuildRight, boundCondition, left, right)), + expectedAnswer, + sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(BroadcastHashJoinExec( + leftKeys, rightKeys, leftSemiPlus, BuildRight, boundCondition, left, right))), + expectedAnswer, + sortAnswers = true) + } + } + } + + test(s"$testName using SortMergeJoin") { + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + SortMergeJoinExec(leftKeys, rightKeys, joinType, boundCondition, left, right)), + expectedAnswer, + sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(SortMergeJoinExec( + leftKeys, rightKeys, leftSemiPlus, boundCondition, left, right))), + expectedAnswer, + sortAnswers = true) + } + } + } + + test(s"$testName using BroadcastNestedLoopJoin build left") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + BroadcastNestedLoopJoinExec(left, right, BuildLeft, joinType, Some(condition))), + expectedAnswer, + sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(BroadcastNestedLoopJoinExec( + left, right, BuildLeft, leftSemiPlus, Some(condition)))), + expectedAnswer, + sortAnswers = true) + } + } + + test(s"$testName using BroadcastNestedLoopJoin build right") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + BroadcastNestedLoopJoinExec(left, right, BuildRight, joinType, Some(condition))), + expectedAnswer, + sortAnswers = true) + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(left.sqlContext.sessionState.conf).apply( + createLeftSemiPlusJoin(BroadcastNestedLoopJoinExec( + left, right, BuildRight, leftSemiPlus, Some(condition)))), + expectedAnswer, + sortAnswers = true) + } + } + } + + testExistenceJoin( + "test single condition (equal) for left semi join", + LeftSemi, + left, + right, + singleConditionEQ, + Seq(Row(2, 1.0), Row(2, 1.0), Row(3, 3.0), Row(6, null))) + + testExistenceJoin( + "test composed condition (equal & non-equal) for left semi join", + LeftSemi, + left, + right, + composedConditionEQ, + Seq(Row(2, 1.0), Row(2, 1.0))) + + testExistenceJoin( + "test composed condition (both non-equal) for left semi join", + LeftSemi, + left, + right, + composedConditionNEQ, + Seq(Row(1, 2.0), Row(1, 2.0), Row(2, 1.0), Row(2, 1.0))) + + testExistenceJoin( + "test single condition (equal) for left Anti join", + LeftAnti, + left, + right, + singleConditionEQ, + Seq(Row(1, 2.0), Row(1, 2.0), Row(null, null), Row(null, 5.0))) + + + testExistenceJoin( + "test single unique condition (equal) for left Anti join", + LeftAnti, + left, + right.select(right.col("c")).distinct(), /* Trigger BHJs unique key code path! */ + singleConditionEQ, + Seq(Row(1, 2.0), Row(1, 2.0), Row(null, null), Row(null, 5.0))) + + testExistenceJoin( + "test composed condition (equal & non-equal) test for anti join", + LeftAnti, + left, + right, + composedConditionEQ, + Seq(Row(1, 2.0), Row(1, 2.0), Row(3, 3.0), Row(6, null), Row(null, 5.0), Row(null, null))) + + testExistenceJoin( + "test composed condition (both non-equal) for anti join", + LeftAnti, + left, + right, + composedConditionNEQ, + Seq(Row(3, 3.0), Row(6, null), Row(null, 5.0), Row(null, null))) + + testExistenceJoin( + "test composed unique condition (both non-equal) for anti join", + LeftAnti, + left, + rightUniqueKey, + (left.col("a") === rightUniqueKey.col("c") && left.col("b") < rightUniqueKey.col("d")).expr, + Seq(Row(1, 2.0), Row(1, 2.0), Row(3, 3.0), Row(null, null), Row(null, 5.0), Row(6, null))) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index e5fd9e277fc6..ede63fea9606 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -19,76 +19,47 @@ package org.apache.spark.sql.execution.joins import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream} -import org.apache.spark.SparkFunSuite +import scala.util.Random + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} +import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructField, StructType} +import org.apache.spark.unsafe.map.BytesToBytesMap +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.collection.CompactBuffer - class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { - // Key is simply the record itself - private val keyProjection = new Projection { - override def apply(row: InternalRow): InternalRow = row - } - - test("GeneralHashedRelation") { - val data = Array(InternalRow(0), InternalRow(1), InternalRow(2), InternalRow(2)) - val numDataRows = SQLMetrics.createLongMetric(sparkContext, "data") - val hashed = HashedRelation(data.iterator, numDataRows, keyProjection) - assert(hashed.isInstanceOf[GeneralHashedRelation]) - - assert(hashed.get(data(0)) === CompactBuffer[InternalRow](data(0))) - assert(hashed.get(data(1)) === CompactBuffer[InternalRow](data(1))) - assert(hashed.get(InternalRow(10)) === null) - - val data2 = CompactBuffer[InternalRow](data(2)) - data2 += data(2) - assert(hashed.get(data(2)) === data2) - assert(numDataRows.value.value === data.length) - } - - test("UniqueKeyHashedRelation") { - val data = Array(InternalRow(0), InternalRow(1), InternalRow(2)) - val numDataRows = SQLMetrics.createLongMetric(sparkContext, "data") - val hashed = HashedRelation(data.iterator, numDataRows, keyProjection) - assert(hashed.isInstanceOf[UniqueKeyHashedRelation]) - - assert(hashed.get(data(0)) === CompactBuffer[InternalRow](data(0))) - assert(hashed.get(data(1)) === CompactBuffer[InternalRow](data(1))) - assert(hashed.get(data(2)) === CompactBuffer[InternalRow](data(2))) - assert(hashed.get(InternalRow(10)) === null) - - val uniqHashed = hashed.asInstanceOf[UniqueKeyHashedRelation] - assert(uniqHashed.getValue(data(0)) === data(0)) - assert(uniqHashed.getValue(data(1)) === data(1)) - assert(uniqHashed.getValue(data(2)) === data(2)) - assert(uniqHashed.getValue(InternalRow(10)) === null) - assert(numDataRows.value.value === data.length) - } + val mm = new TaskMemoryManager( + new StaticMemoryManager( + new SparkConf().set("spark.memory.offHeap.enabled", "false"), + Long.MaxValue, + Long.MaxValue, + 1), + 0) test("UnsafeHashedRelation") { val schema = StructType(StructField("a", IntegerType, true) :: Nil) val data = Array(InternalRow(0), InternalRow(1), InternalRow(2), InternalRow(2)) - val numDataRows = SQLMetrics.createLongMetric(sparkContext, "data") val toUnsafe = UnsafeProjection.create(schema) - val unsafeData = data.map(toUnsafe(_).copy()).toArray + val unsafeData = data.map(toUnsafe(_).copy()) + val buildKey = Seq(BoundReference(0, IntegerType, false)) - val keyGenerator = UnsafeProjection.create(buildKey) - val hashed = UnsafeHashedRelation(unsafeData.iterator, numDataRows, keyGenerator, 1) + val hashed = UnsafeHashedRelation(unsafeData.iterator, buildKey, 1, mm) assert(hashed.isInstanceOf[UnsafeHashedRelation]) - assert(hashed.get(unsafeData(0)) === CompactBuffer[InternalRow](unsafeData(0))) - assert(hashed.get(unsafeData(1)) === CompactBuffer[InternalRow](unsafeData(1))) + assert(hashed.get(unsafeData(0)).toArray === Array(unsafeData(0))) + assert(hashed.get(unsafeData(1)).toArray === Array(unsafeData(1))) assert(hashed.get(toUnsafe(InternalRow(10))) === null) val data2 = CompactBuffer[InternalRow](unsafeData(2).copy()) data2 += unsafeData(2).copy() - assert(hashed.get(unsafeData(2)) === data2) + assert(hashed.get(unsafeData(2)).toArray === data2.toArray) val os = new ByteArrayOutputStream() val out = new ObjectOutputStream(os) @@ -97,11 +68,10 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { val in = new ObjectInputStream(new ByteArrayInputStream(os.toByteArray)) val hashed2 = new UnsafeHashedRelation() hashed2.readExternal(in) - assert(hashed2.get(unsafeData(0)) === CompactBuffer[InternalRow](unsafeData(0))) - assert(hashed2.get(unsafeData(1)) === CompactBuffer[InternalRow](unsafeData(1))) + assert(hashed2.get(unsafeData(0)).toArray === Array(unsafeData(0))) + assert(hashed2.get(unsafeData(1)).toArray === Array(unsafeData(1))) assert(hashed2.get(toUnsafe(InternalRow(10))) === null) - assert(hashed2.get(unsafeData(2)) === data2) - assert(numDataRows.value.value === data.length) + assert(hashed2.get(unsafeData(2)).toArray === data2) val os2 = new ByteArrayOutputStream() val out2 = new ObjectOutputStream(os2) @@ -113,10 +83,17 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { } test("test serialization empty hash map") { + val taskMemoryManager = new TaskMemoryManager( + new StaticMemoryManager( + new SparkConf().set("spark.memory.offHeap.enabled", "false"), + Long.MaxValue, + Long.MaxValue, + 1), + 0) + val binaryMap = new BytesToBytesMap(taskMemoryManager, 1, 1) val os = new ByteArrayOutputStream() val out = new ObjectOutputStream(os) - val hashed = new UnsafeHashedRelation( - new java.util.HashMap[UnsafeRow, CompactBuffer[UnsafeRow]]) + val hashed = new UnsafeHashedRelation(1, binaryMap) hashed.writeExternal(out) out.flush() val in = new ObjectInputStream(new ByteArrayInputStream(os.toByteArray)) @@ -134,4 +111,222 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { out2.flush() assert(java.util.Arrays.equals(os2.toByteArray, os.toByteArray)) } + + test("LongToUnsafeRowMap") { + val unsafeProj = UnsafeProjection.create( + Seq(BoundReference(0, LongType, false), BoundReference(1, IntegerType, true))) + val rows = (0 until 100).map(i => unsafeProj(InternalRow(Int.int2long(i), i + 1)).copy()) + val key = Seq(BoundReference(0, LongType, false)) + val longRelation = LongHashedRelation(rows.iterator, key, 10, mm) + assert(longRelation.keyIsUnique) + (0 until 100).foreach { i => + val row = longRelation.getValue(i) + assert(row.getLong(0) === i) + assert(row.getInt(1) === i + 1) + } + + val longRelation2 = LongHashedRelation(rows.iterator ++ rows.iterator, key, 100, mm) + assert(!longRelation2.keyIsUnique) + (0 until 100).foreach { i => + val rows = longRelation2.get(i).toArray + assert(rows.length === 2) + assert(rows(0).getLong(0) === i) + assert(rows(0).getInt(1) === i + 1) + assert(rows(1).getLong(0) === i) + assert(rows(1).getInt(1) === i + 1) + } + + val os = new ByteArrayOutputStream() + val out = new ObjectOutputStream(os) + longRelation2.writeExternal(out) + out.flush() + val in = new ObjectInputStream(new ByteArrayInputStream(os.toByteArray)) + val relation = new LongHashedRelation() + relation.readExternal(in) + assert(!relation.keyIsUnique) + (0 until 100).foreach { i => + val rows = relation.get(i).toArray + assert(rows.length === 2) + assert(rows(0).getLong(0) === i) + assert(rows(0).getInt(1) === i + 1) + assert(rows(1).getLong(0) === i) + assert(rows(1).getInt(1) === i + 1) + } + } + + test("LongToUnsafeRowMap with very wide range") { + val taskMemoryManager = new TaskMemoryManager( + new StaticMemoryManager( + new SparkConf().set("spark.memory.offHeap.enabled", "false"), + Long.MaxValue, + Long.MaxValue, + 1), + 0) + val unsafeProj = UnsafeProjection.create(Seq(BoundReference(0, LongType, false))) + + { + // SPARK-16740 + val keys = Seq(0L, Long.MaxValue, Long.MaxValue) + val map = new LongToUnsafeRowMap(taskMemoryManager, 1) + keys.foreach { k => + map.append(k, unsafeProj(InternalRow(k))) + } + map.optimize() + val row = unsafeProj(InternalRow(0L)).copy() + keys.foreach { k => + assert(map.getValue(k, row) eq row) + assert(row.getLong(0) === k) + } + map.free() + } + + + { + // SPARK-16802 + val keys = Seq(Long.MaxValue, Long.MaxValue - 10) + val map = new LongToUnsafeRowMap(taskMemoryManager, 1) + keys.foreach { k => + map.append(k, unsafeProj(InternalRow(k))) + } + map.optimize() + val row = unsafeProj(InternalRow(0L)).copy() + keys.foreach { k => + assert(map.getValue(k, row) eq row) + assert(row.getLong(0) === k) + } + assert(map.getValue(Long.MinValue, row) eq null) + map.free() + } + } + + test("LongToUnsafeRowMap with random keys") { + val taskMemoryManager = new TaskMemoryManager( + new StaticMemoryManager( + new SparkConf().set("spark.memory.offHeap.enabled", "false"), + Long.MaxValue, + Long.MaxValue, + 1), + 0) + val unsafeProj = UnsafeProjection.create(Seq(BoundReference(0, LongType, false))) + + val N = 1000000 + val rand = new Random + val keys = (0 to N).map(x => rand.nextLong()).toArray + + val map = new LongToUnsafeRowMap(taskMemoryManager, 10) + keys.foreach { k => + map.append(k, unsafeProj(InternalRow(k))) + } + map.optimize() + + val os = new ByteArrayOutputStream() + val out = new ObjectOutputStream(os) + map.writeExternal(out) + out.flush() + val in = new ObjectInputStream(new ByteArrayInputStream(os.toByteArray)) + val map2 = new LongToUnsafeRowMap(taskMemoryManager, 1) + map2.readExternal(in) + + val row = unsafeProj(InternalRow(0L)).copy() + keys.foreach { k => + val r = map2.get(k, row) + assert(r.hasNext) + var c = 0 + while (r.hasNext) { + val rr = r.next() + assert(rr.getLong(0) === k) + c += 1 + } + } + var i = 0 + while (i < N * 10) { + val k = rand.nextLong() + val r = map2.get(k, row) + if (r != null) { + assert(r.hasNext) + while (r.hasNext) { + assert(r.next().getLong(0) === k) + } + } + i += 1 + } + map.free() + } + + test("Spark-14521") { + val ser = new KryoSerializer( + (new SparkConf).set("spark.kryo.referenceTracking", "false")).newInstance() + val key = Seq(BoundReference(0, LongType, false)) + + // Testing Kryo serialization of HashedRelation + val unsafeProj = UnsafeProjection.create( + Seq(BoundReference(0, LongType, false), BoundReference(1, IntegerType, true))) + val rows = (0 until 100).map(i => unsafeProj(InternalRow(Int.int2long(i), i + 1)).copy()) + val longRelation = LongHashedRelation(rows.iterator ++ rows.iterator, key, 100, mm) + val longRelation2 = ser.deserialize[LongHashedRelation](ser.serialize(longRelation)) + (0 until 100).foreach { i => + val rows = longRelation2.get(i).toArray + assert(rows.length === 2) + assert(rows(0).getLong(0) === i) + assert(rows(0).getInt(1) === i + 1) + assert(rows(1).getLong(0) === i) + assert(rows(1).getInt(1) === i + 1) + } + + // Testing Kryo serialization of UnsafeHashedRelation + val unsafeHashed = UnsafeHashedRelation(rows.iterator, key, 1, mm) + val os = new ByteArrayOutputStream() + val out = new ObjectOutputStream(os) + unsafeHashed.asInstanceOf[UnsafeHashedRelation].writeExternal(out) + out.flush() + val unsafeHashed2 = ser.deserialize[UnsafeHashedRelation](ser.serialize(unsafeHashed)) + val os2 = new ByteArrayOutputStream() + val out2 = new ObjectOutputStream(os2) + unsafeHashed2.writeExternal(out2) + out2.flush() + assert(java.util.Arrays.equals(os.toByteArray, os2.toByteArray)) + } + + // This test require 4G heap to run, should run it manually + ignore("build HashedRelation that is larger than 1G") { + val unsafeProj = UnsafeProjection.create( + Seq(BoundReference(0, IntegerType, false), + BoundReference(1, StringType, true))) + val unsafeRow = unsafeProj(InternalRow(0, UTF8String.fromString(" " * 100))) + val key = Seq(BoundReference(0, IntegerType, false)) + val rows = (0 until (1 << 24)).iterator.map { i => + unsafeRow.setInt(0, i % 1000000) + unsafeRow.setInt(1, i) + unsafeRow + } + + val unsafeRelation = UnsafeHashedRelation(rows, key, 1000, mm) + assert(unsafeRelation.estimatedSize > (2L << 30)) + unsafeRelation.close() + + val rows2 = (0 until (1 << 24)).iterator.map { i => + unsafeRow.setInt(0, i % 1000000) + unsafeRow.setInt(1, i) + unsafeRow + } + val longRelation = LongHashedRelation(rows2, key, 1000, mm) + assert(longRelation.estimatedSize > (2L << 30)) + longRelation.close() + } + + // This test require 4G heap to run, should run it manually + ignore("build HashedRelation with more than 100 millions rows") { + val unsafeProj = UnsafeProjection.create( + Seq(BoundReference(0, IntegerType, false), + BoundReference(1, StringType, true))) + val unsafeRow = unsafeProj(InternalRow(0, UTF8String.fromString(" " * 100))) + val key = Seq(BoundReference(0, IntegerType, false)) + val rows = (0 until (1 << 10)).iterator.map { i => + unsafeRow.setInt(0, i % 1000000) + unsafeRow.setInt(1, i) + unsafeRow + } + val m = LongHashedRelation(rows, key, 100 << 20, mm) + m.close() + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala index 066c16e535c7..4408ece11225 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala @@ -17,19 +17,22 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.sql.{DataFrame, execution, Row, SQLConf} +import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.logical.Join import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.exchange.EnsureRequirements +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StringType, StructType} class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { - import testImplicits.localSeqToDataFrameHolder + import testImplicits.newProductEncoder + import testImplicits.localSeqToDatasetHolder - private lazy val myUpperCaseData = sqlContext.createDataFrame( + private lazy val myUpperCaseData = spark.createDataFrame( sparkContext.parallelize(Seq( Row(1, "A"), Row(2, "B"), @@ -40,7 +43,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { Row(null, "G") )), new StructType().add("N", IntegerType).add("L", StringType)) - private lazy val myLowerCaseData = sqlContext.createDataFrame( + private lazy val myLowerCaseData = spark.createDataFrame( sparkContext.parallelize(Seq( Row(1, "a"), Row(2, "b"), @@ -88,9 +91,15 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { leftPlan: SparkPlan, rightPlan: SparkPlan, side: BuildSide) = { - val broadcastHashJoin = - execution.joins.BroadcastHashJoin(leftKeys, rightKeys, side, leftPlan, rightPlan) - boundCondition.map(Filter(_, broadcastHashJoin)).getOrElse(broadcastHashJoin) + val broadcastJoin = joins.BroadcastHashJoinExec( + leftKeys, + rightKeys, + Inner, + side, + boundCondition, + leftPlan, + rightPlan) + EnsureRequirements(spark.sessionState.conf).apply(broadcastJoin) } def makeShuffledHashJoin( @@ -100,11 +109,11 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { leftPlan: SparkPlan, rightPlan: SparkPlan, side: BuildSide) = { - val shuffledHashJoin = - execution.joins.ShuffledHashJoin(leftKeys, rightKeys, side, leftPlan, rightPlan) + val shuffledHashJoin = joins.ShuffledHashJoinExec(leftKeys, rightKeys, Inner, + side, None, leftPlan, rightPlan) val filteredJoin = - boundCondition.map(Filter(_, shuffledHashJoin)).getOrElse(shuffledHashJoin) - EnsureRequirements(sqlContext).apply(filteredJoin) + boundCondition.map(FilterExec(_, shuffledHashJoin)).getOrElse(shuffledHashJoin) + EnsureRequirements(spark.sessionState.conf).apply(filteredJoin) } def makeSortMergeJoin( @@ -113,10 +122,9 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { boundCondition: Option[Expression], leftPlan: SparkPlan, rightPlan: SparkPlan) = { - val sortMergeJoin = - execution.joins.SortMergeJoin(leftKeys, rightKeys, leftPlan, rightPlan) - val filteredJoin = boundCondition.map(Filter(_, sortMergeJoin)).getOrElse(sortMergeJoin) - EnsureRequirements(sqlContext).apply(filteredJoin) + val sortMergeJoin = joins.SortMergeJoinExec(leftKeys, rightKeys, Inner, boundCondition, + leftPlan, rightPlan) + EnsureRequirements(spark.sessionState.conf).apply(sortMergeJoin) } test(s"$testName using BroadcastHashJoin (build=left)") { @@ -177,6 +185,34 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { } } } + + test(s"$testName using CartesianProduct") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1", + SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + CartesianProductExec(left, right, Some(condition())), + expectedAnswer.map(Row.fromTuple), + sortAnswers = true) + } + } + + test(s"$testName using BroadcastNestedLoopJoin build left") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + BroadcastNestedLoopJoinExec(left, right, BuildLeft, Inner, Some(condition())), + expectedAnswer.map(Row.fromTuple), + sortAnswers = true) + } + } + + test(s"$testName using BroadcastNestedLoopJoin build right") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + BroadcastNestedLoopJoinExec(left, right, BuildRight, Inner, Some(condition())), + expectedAnswer.map(Row.fromTuple), + sortAnswers = true) + } + } } testInnerJoin( @@ -235,4 +271,19 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { ) ) } + + { + def df: DataFrame = spark.range(3).selectExpr("struct(id, id) as key", "id as value") + lazy val left = df.selectExpr("key", "concat('L', value) as value").alias("left") + lazy val right = df.selectExpr("key", "concat('R', value) as value").alias("right") + testInnerJoin( + "SPARK-15822 - test structs as keys", + left, + right, + () => (left.col("key") === right.col("key")).expr, + Seq( + (Row(0, 0), "L0", Row(0, 0), "R0"), + (Row(1, 1), "L1", Row(1, 1), "R1"), + (Row(2, 2), "L2", Row(2, 2), "R2"))) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala index 09e0237a7cc5..001feb0f2b39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala @@ -17,18 +17,20 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.sql.{DataFrame, Row, SQLConf} +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.catalyst.expressions.{And, Expression, LessThan} import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.Join -import org.apache.spark.sql.catalyst.expressions.{And, Expression, LessThan} -import org.apache.spark.sql.execution.{EnsureRequirements, SparkPlan, SparkPlanTest} +import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest} +import org.apache.spark.sql.execution.exchange.EnsureRequirements +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{IntegerType, DoubleType, StructType} +import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { - private lazy val left = sqlContext.createDataFrame( + private lazy val left = spark.createDataFrame( sparkContext.parallelize(Seq( Row(1, 2.0), Row(2, 100.0), @@ -40,7 +42,7 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { Row(null, null) )), new StructType().add("a", IntegerType).add("b", DoubleType)) - private lazy val right = sqlContext.createDataFrame( + private lazy val right = spark.createDataFrame( sparkContext.parallelize(Seq( Row(0, 0.0), Row(2, 3.0), // This row is duplicated to ensure that we will have multiple buffered matches @@ -74,24 +76,34 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { ExtractEquiJoinKeys.unapply(join) } - test(s"$testName using ShuffledHashOuterJoin") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => - withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { - checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - EnsureRequirements(sqlContext).apply( - ShuffledHashOuterJoin(leftKeys, rightKeys, joinType, boundCondition, left, right)), - expectedAnswer.map(Row.fromTuple), - sortAnswers = true) + if (joinType != FullOuter) { + test(s"$testName using ShuffledHashJoin") { + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + val buildSide = if (joinType == LeftOuter) BuildRight else BuildLeft + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements(spark.sessionState.conf).apply( + ShuffledHashJoinExec( + leftKeys, rightKeys, joinType, buildSide, boundCondition, left, right)), + expectedAnswer.map(Row.fromTuple), + sortAnswers = true) + } } } } if (joinType != FullOuter) { - test(s"$testName using BroadcastHashOuterJoin") { + test(s"$testName using BroadcastHashJoin") { + val buildSide = joinType match { + case LeftOuter => BuildRight + case RightOuter => BuildLeft + case _ => fail(s"Unsupported join type $joinType") + } extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - BroadcastHashOuterJoin(leftKeys, rightKeys, joinType, boundCondition, left, right), + BroadcastHashJoinExec( + leftKeys, rightKeys, joinType, buildSide, boundCondition, left, right), expectedAnswer.map(Row.fromTuple), sortAnswers = true) } @@ -99,17 +111,35 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { } } - test(s"$testName using SortMergeOuterJoin") { + test(s"$testName using SortMergeJoin") { extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - EnsureRequirements(sqlContext).apply( - SortMergeOuterJoin(leftKeys, rightKeys, joinType, boundCondition, left, right)), + EnsureRequirements(spark.sessionState.conf).apply( + SortMergeJoinExec(leftKeys, rightKeys, joinType, boundCondition, left, right)), expectedAnswer.map(Row.fromTuple), sortAnswers = true) } } } + + test(s"$testName using BroadcastNestedLoopJoin build left") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + BroadcastNestedLoopJoinExec(left, right, BuildLeft, joinType, Some(condition)), + expectedAnswer.map(Row.fromTuple), + sortAnswers = true) + } + } + + test(s"$testName using BroadcastNestedLoopJoin build right") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + BroadcastNestedLoopJoinExec(left, right, BuildRight, joinType, Some(condition)), + expectedAnswer.map(Row.fromTuple), + sortAnswers = true) + } + } } // --- Basic outer joins ------------------------------------------------------------------------ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala deleted file mode 100644 index 3afd762942bc..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/SemiJoinSuite.scala +++ /dev/null @@ -1,116 +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.execution.joins - -import org.apache.spark.sql.{SQLConf, DataFrame, Row} -import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys -import org.apache.spark.sql.catalyst.plans.Inner -import org.apache.spark.sql.catalyst.plans.logical.Join -import org.apache.spark.sql.catalyst.expressions.{And, LessThan, Expression} -import org.apache.spark.sql.execution.{EnsureRequirements, SparkPlan, SparkPlanTest} -import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} - -class SemiJoinSuite extends SparkPlanTest with SharedSQLContext { - - private lazy val left = sqlContext.createDataFrame( - sparkContext.parallelize(Seq( - Row(1, 2.0), - Row(1, 2.0), - Row(2, 1.0), - Row(2, 1.0), - Row(3, 3.0), - Row(null, null), - Row(null, 5.0), - Row(6, null) - )), new StructType().add("a", IntegerType).add("b", DoubleType)) - - private lazy val right = sqlContext.createDataFrame( - sparkContext.parallelize(Seq( - Row(2, 3.0), - Row(2, 3.0), - Row(3, 2.0), - Row(4, 1.0), - Row(null, null), - Row(null, 5.0), - Row(6, null) - )), new StructType().add("c", IntegerType).add("d", DoubleType)) - - private lazy val condition = { - And((left.col("a") === right.col("c")).expr, - LessThan(left.col("b").expr, right.col("d").expr)) - } - - // Note: the input dataframes and expression must be evaluated lazily because - // the SQLContext should be used only within a test to keep SQL tests stable - private def testLeftSemiJoin( - testName: String, - leftRows: => DataFrame, - rightRows: => DataFrame, - condition: => Expression, - expectedAnswer: Seq[Product]): Unit = { - - def extractJoinParts(): Option[ExtractEquiJoinKeys.ReturnType] = { - val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, Inner, Some(condition)) - ExtractEquiJoinKeys.unapply(join) - } - - test(s"$testName using LeftSemiJoinHash") { - extractJoinParts().foreach { case (joinType, leftKeys, rightKeys, boundCondition, _, _) => - withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { - checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - EnsureRequirements(left.sqlContext).apply( - LeftSemiJoinHash(leftKeys, rightKeys, left, right, boundCondition)), - expectedAnswer.map(Row.fromTuple), - sortAnswers = true) - } - } - } - - test(s"$testName using BroadcastLeftSemiJoinHash") { - extractJoinParts().foreach { case (joinType, leftKeys, rightKeys, boundCondition, _, _) => - withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { - checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - BroadcastLeftSemiJoinHash(leftKeys, rightKeys, left, right, boundCondition), - expectedAnswer.map(Row.fromTuple), - sortAnswers = true) - } - } - } - - test(s"$testName using LeftSemiJoinBNL") { - withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { - checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - LeftSemiJoinBNL(left, right, Some(condition)), - expectedAnswer.map(Row.fromTuple), - sortAnswers = true) - } - } - } - - testLeftSemiJoin( - "basic test", - left, - right, - condition, - Seq( - (2, 1.0), - (2, 1.0) - ) - ) -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/DummyNode.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/DummyNode.scala deleted file mode 100644 index efc3227dd60d..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/DummyNode.scala +++ /dev/null @@ -1,68 +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.execution.local - -import org.apache.spark.sql.SQLConf -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LocalRelation - -/** - * A dummy [[LocalNode]] that just returns rows from a [[LocalRelation]]. - */ -private[local] case class DummyNode( - output: Seq[Attribute], - relation: LocalRelation, - conf: SQLConf) - extends LocalNode(conf) { - - import DummyNode._ - - private var index: Int = CLOSED - private val input: Seq[InternalRow] = relation.data - - def this(output: Seq[Attribute], data: Seq[Product], conf: SQLConf = new SQLConf) { - this(output, LocalRelation.fromProduct(output, data), conf) - } - - def isOpen: Boolean = index != CLOSED - - override def children: Seq[LocalNode] = Seq.empty - - override def open(): Unit = { - index = -1 - } - - override def next(): Boolean = { - index += 1 - index < input.size - } - - override def fetch(): InternalRow = { - assert(index >= 0 && index < input.size) - input(index) - } - - override def close(): Unit = { - index = CLOSED - } -} - -private object DummyNode { - val CLOSED: Int = Int.MinValue -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/ExpandNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/ExpandNodeSuite.scala deleted file mode 100644 index bbd94d8da2d1..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/ExpandNodeSuite.scala +++ /dev/null @@ -1,49 +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.execution.local - -import org.apache.spark.sql.catalyst.dsl.expressions._ - - -class ExpandNodeSuite extends LocalNodeTest { - - private def testExpand(inputData: Array[(Int, Int)] = Array.empty): Unit = { - val inputNode = new DummyNode(kvIntAttributes, inputData) - val projections = Seq(Seq('k + 'v, 'k - 'v), Seq('k * 'v, 'k / 'v)) - val expandNode = new ExpandNode(conf, projections, inputNode.output, inputNode) - val resolvedNode = resolveExpressions(expandNode) - val expectedOutput = { - val firstHalf = inputData.map { case (k, v) => (k + v, k - v) } - val secondHalf = inputData.map { case (k, v) => (k * v, k / v) } - firstHalf ++ secondHalf - } - val actualOutput = resolvedNode.collect().map { case row => - (row.getInt(0), row.getInt(1)) - } - assert(actualOutput.toSet === expectedOutput.toSet) - } - - test("empty") { - testExpand() - } - - test("basic") { - testExpand((1 to 100).map { i => (i, i * 1000) }.toArray) - } - -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/FilterNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/FilterNodeSuite.scala deleted file mode 100644 index 4eadce646d37..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/FilterNodeSuite.scala +++ /dev/null @@ -1,45 +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.execution.local - -import org.apache.spark.sql.catalyst.dsl.expressions._ - - -class FilterNodeSuite extends LocalNodeTest { - - private def testFilter(inputData: Array[(Int, Int)] = Array.empty): Unit = { - val cond = 'k % 2 === 0 - val inputNode = new DummyNode(kvIntAttributes, inputData) - val filterNode = new FilterNode(conf, cond, inputNode) - val resolvedNode = resolveExpressions(filterNode) - val expectedOutput = inputData.filter { case (k, _) => k % 2 == 0 } - val actualOutput = resolvedNode.collect().map { case row => - (row.getInt(0), row.getInt(1)) - } - assert(actualOutput === expectedOutput) - } - - test("empty") { - testFilter() - } - - test("basic") { - testFilter((1 to 100).map { i => (i, i) }.toArray) - } - -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/HashJoinNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/HashJoinNodeSuite.scala deleted file mode 100644 index 8c2e78b2a9db..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/HashJoinNodeSuite.scala +++ /dev/null @@ -1,160 +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.execution.local - -import org.mockito.Mockito.{mock, when} - -import org.apache.spark.broadcast.TorrentBroadcast -import org.apache.spark.sql.SQLConf -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions.{InterpretedMutableProjection, UnsafeProjection, Expression} -import org.apache.spark.sql.execution.joins.{HashedRelation, BuildLeft, BuildRight, BuildSide} - -class HashJoinNodeSuite extends LocalNodeTest { - - // Test all combinations of the two dimensions: with/out unsafe and build sides - private val maybeUnsafeAndCodegen = Seq(false, true) - private val buildSides = Seq(BuildLeft, BuildRight) - maybeUnsafeAndCodegen.foreach { unsafeAndCodegen => - buildSides.foreach { buildSide => - testJoin(unsafeAndCodegen, buildSide) - } - } - - /** - * Builds a [[HashedRelation]] based on a resolved `buildKeys` - * and a resolved `buildNode`. - */ - private def buildHashedRelation( - conf: SQLConf, - buildKeys: Seq[Expression], - buildNode: LocalNode): HashedRelation = { - - val isUnsafeMode = - conf.codegenEnabled && - conf.unsafeEnabled && - UnsafeProjection.canSupport(buildKeys) - - val buildSideKeyGenerator = - if (isUnsafeMode) { - UnsafeProjection.create(buildKeys, buildNode.output) - } else { - new InterpretedMutableProjection(buildKeys, buildNode.output) - } - - buildNode.prepare() - buildNode.open() - val hashedRelation = HashedRelation(buildNode, buildSideKeyGenerator) - buildNode.close() - - hashedRelation - } - - /** - * Test inner hash join with varying degrees of matches. - */ - private def testJoin( - unsafeAndCodegen: Boolean, - buildSide: BuildSide): Unit = { - val simpleOrUnsafe = if (!unsafeAndCodegen) "simple" else "unsafe" - val testNamePrefix = s"$simpleOrUnsafe / $buildSide" - val someData = (1 to 100).map { i => (i, "burger" + i) }.toArray - val conf = new SQLConf - conf.setConf(SQLConf.UNSAFE_ENABLED, unsafeAndCodegen) - conf.setConf(SQLConf.CODEGEN_ENABLED, unsafeAndCodegen) - - // Actual test body - def runTest(leftInput: Array[(Int, String)], rightInput: Array[(Int, String)]): Unit = { - val rightInputMap = rightInput.toMap - val leftNode = new DummyNode(joinNameAttributes, leftInput) - val rightNode = new DummyNode(joinNicknameAttributes, rightInput) - val makeBinaryHashJoinNode = (node1: LocalNode, node2: LocalNode) => { - val binaryHashJoinNode = - BinaryHashJoinNode(conf, Seq('id1), Seq('id2), buildSide, node1, node2) - resolveExpressions(binaryHashJoinNode) - } - val makeBroadcastJoinNode = (node1: LocalNode, node2: LocalNode) => { - val leftKeys = Seq('id1.attr) - val rightKeys = Seq('id2.attr) - // Figure out the build side and stream side. - val (buildNode, buildKeys, streamedNode, streamedKeys) = buildSide match { - case BuildLeft => (node1, leftKeys, node2, rightKeys) - case BuildRight => (node2, rightKeys, node1, leftKeys) - } - // Resolve the expressions of the build side and then create a HashedRelation. - val resolvedBuildNode = resolveExpressions(buildNode) - val resolvedBuildKeys = resolveExpressions(buildKeys, resolvedBuildNode) - val hashedRelation = buildHashedRelation(conf, resolvedBuildKeys, resolvedBuildNode) - val broadcastHashedRelation = mock(classOf[TorrentBroadcast[HashedRelation]]) - when(broadcastHashedRelation.value).thenReturn(hashedRelation) - - val hashJoinNode = - BroadcastHashJoinNode( - conf, - streamedKeys, - streamedNode, - buildSide, - resolvedBuildNode.output, - broadcastHashedRelation) - resolveExpressions(hashJoinNode) - } - - val expectedOutput = leftInput - .filter { case (k, _) => rightInputMap.contains(k) } - .map { case (k, v) => (k, v, k, rightInputMap(k)) } - - Seq(makeBinaryHashJoinNode, makeBroadcastJoinNode).foreach { makeNode => - val makeUnsafeNode = if (unsafeAndCodegen) wrapForUnsafe(makeNode) else makeNode - val hashJoinNode = makeUnsafeNode(leftNode, rightNode) - - val actualOutput = hashJoinNode.collect().map { row => - // (id, name, id, nickname) - (row.getInt(0), row.getString(1), row.getInt(2), row.getString(3)) - } - assert(actualOutput === expectedOutput) - } - } - - test(s"$testNamePrefix: empty") { - runTest(Array.empty, Array.empty) - runTest(someData, Array.empty) - runTest(Array.empty, someData) - } - - test(s"$testNamePrefix: no matches") { - val someIrrelevantData = (10000 to 100100).map { i => (i, "piper" + i) }.toArray - runTest(someData, Array.empty) - runTest(Array.empty, someData) - runTest(someData, someIrrelevantData) - runTest(someIrrelevantData, someData) - } - - test(s"$testNamePrefix: partial matches") { - val someOtherData = (50 to 150).map { i => (i, "finnegan" + i) }.toArray - runTest(someData, someOtherData) - runTest(someOtherData, someData) - } - - test(s"$testNamePrefix: full matches") { - val someSuperRelevantData = someData.map { case (k, v) => (k, "cooper" + v) }.toArray - runTest(someData, someSuperRelevantData) - runTest(someSuperRelevantData, someData) - } - } - -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/IntersectNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/IntersectNodeSuite.scala deleted file mode 100644 index c0ad2021b204..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/IntersectNodeSuite.scala +++ /dev/null @@ -1,37 +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.execution.local - - -class IntersectNodeSuite extends LocalNodeTest { - - test("basic") { - val n = 100 - val leftData = (1 to n).filter { i => i % 2 == 0 }.map { i => (i, i) }.toArray - val rightData = (1 to n).filter { i => i % 3 == 0 }.map { i => (i, i) }.toArray - val leftNode = new DummyNode(kvIntAttributes, leftData) - val rightNode = new DummyNode(kvIntAttributes, rightData) - val intersectNode = new IntersectNode(conf, leftNode, rightNode) - val expectedOutput = leftData.intersect(rightData) - val actualOutput = intersectNode.collect().map { case row => - (row.getInt(0), row.getInt(1)) - } - assert(actualOutput === expectedOutput) - } - -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LimitNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LimitNodeSuite.scala deleted file mode 100644 index fb790636a368..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LimitNodeSuite.scala +++ /dev/null @@ -1,41 +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.execution.local - - -class LimitNodeSuite extends LocalNodeTest { - - private def testLimit(inputData: Array[(Int, Int)] = Array.empty, limit: Int = 10): Unit = { - val inputNode = new DummyNode(kvIntAttributes, inputData) - val limitNode = new LimitNode(conf, limit, inputNode) - val expectedOutput = inputData.take(limit) - val actualOutput = limitNode.collect().map { case row => - (row.getInt(0), row.getInt(1)) - } - assert(actualOutput === expectedOutput) - } - - test("empty") { - testLimit() - } - - test("basic") { - testLimit((1 to 100).map { i => (i, i) }.toArray, 20) - } - -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LocalNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LocalNodeSuite.scala deleted file mode 100644 index 0d1ed99eec6c..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LocalNodeSuite.scala +++ /dev/null @@ -1,73 +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.execution.local - - -class LocalNodeSuite extends LocalNodeTest { - private val data = (1 to 100).map { i => (i, i) }.toArray - - test("basic open, next, fetch, close") { - val node = new DummyNode(kvIntAttributes, data) - assert(!node.isOpen) - node.open() - assert(node.isOpen) - data.foreach { case (k, v) => - assert(node.next()) - // fetch should be idempotent - val fetched = node.fetch() - assert(node.fetch() === fetched) - assert(node.fetch() === fetched) - assert(node.fetch().numFields === 2) - assert(node.fetch().getInt(0) === k) - assert(node.fetch().getInt(1) === v) - } - assert(!node.next()) - node.close() - assert(!node.isOpen) - } - - test("asIterator") { - val node = new DummyNode(kvIntAttributes, data) - val iter = node.asIterator - node.open() - data.foreach { case (k, v) => - // hasNext should be idempotent - assert(iter.hasNext) - assert(iter.hasNext) - val item = iter.next() - assert(item.numFields === 2) - assert(item.getInt(0) === k) - assert(item.getInt(1) === v) - } - intercept[NoSuchElementException] { - iter.next() - } - node.close() - } - - test("collect") { - val node = new DummyNode(kvIntAttributes, data) - node.open() - val collected = node.collect() - assert(collected.size === data.size) - assert(collected.forall(_.size === 2)) - assert(collected.map { case row => (row.getInt(0), row.getInt(0)) } === data) - node.close() - } - -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LocalNodeTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LocalNodeTest.scala deleted file mode 100644 index 615c41709361..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LocalNodeTest.scala +++ /dev/null @@ -1,88 +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.execution.local - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.SQLConf -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.expressions.{Expression, AttributeReference} -import org.apache.spark.sql.types.{IntegerType, StringType} - - -class LocalNodeTest extends SparkFunSuite { - - protected val conf: SQLConf = new SQLConf - protected val kvIntAttributes = Seq( - AttributeReference("k", IntegerType)(), - AttributeReference("v", IntegerType)()) - protected val joinNameAttributes = Seq( - AttributeReference("id1", IntegerType)(), - AttributeReference("name", StringType)()) - protected val joinNicknameAttributes = Seq( - AttributeReference("id2", IntegerType)(), - AttributeReference("nickname", StringType)()) - - /** - * Wrap a function processing two [[LocalNode]]s such that: - * (1) all input rows are automatically converted to unsafe rows - * (2) all output rows are automatically converted back to safe rows - */ - protected def wrapForUnsafe( - f: (LocalNode, LocalNode) => LocalNode): (LocalNode, LocalNode) => LocalNode = { - (left: LocalNode, right: LocalNode) => { - val _left = ConvertToUnsafeNode(conf, left) - val _right = ConvertToUnsafeNode(conf, right) - val r = f(_left, _right) - ConvertToSafeNode(conf, r) - } - } - - /** - * Recursively resolve all expressions in a [[LocalNode]] using the node's attributes. - */ - protected def resolveExpressions(outputNode: LocalNode): LocalNode = { - outputNode transform { - case node: LocalNode => - val inputMap = node.output.map { a => (a.name, a) }.toMap - node transformExpressions { - case UnresolvedAttribute(Seq(u)) => - inputMap.getOrElse(u, - sys.error(s"Invalid Test: Cannot resolve $u given input $inputMap")) - } - } - } - - /** - * Resolve all expressions in `expressions` based on the `output` of `localNode`. - * It assumes that all expressions in the `localNode` are resolved. - */ - protected def resolveExpressions( - expressions: Seq[Expression], - localNode: LocalNode): Seq[Expression] = { - require(localNode.expressions.forall(_.resolved)) - val inputMap = localNode.output.map { a => (a.name, a) }.toMap - expressions.map { expression => - expression.transformUp { - case UnresolvedAttribute(Seq(u)) => - inputMap.getOrElse(u, - sys.error(s"Invalid Test: Cannot resolve $u given input $inputMap")) - } - } - } - -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/NestedLoopJoinNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/NestedLoopJoinNodeSuite.scala deleted file mode 100644 index 40299d9d5ee3..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/NestedLoopJoinNodeSuite.scala +++ /dev/null @@ -1,145 +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.execution.local - -import org.apache.spark.sql.SQLConf -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} -import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} - - -class NestedLoopJoinNodeSuite extends LocalNodeTest { - - // Test all combinations of the three dimensions: with/out unsafe, build sides, and join types - private val maybeUnsafeAndCodegen = Seq(false, true) - private val buildSides = Seq(BuildLeft, BuildRight) - private val joinTypes = Seq(LeftOuter, RightOuter, FullOuter) - maybeUnsafeAndCodegen.foreach { unsafeAndCodegen => - buildSides.foreach { buildSide => - joinTypes.foreach { joinType => - testJoin(unsafeAndCodegen, buildSide, joinType) - } - } - } - - /** - * Test outer nested loop joins with varying degrees of matches. - */ - private def testJoin( - unsafeAndCodegen: Boolean, - buildSide: BuildSide, - joinType: JoinType): Unit = { - val simpleOrUnsafe = if (!unsafeAndCodegen) "simple" else "unsafe" - val testNamePrefix = s"$simpleOrUnsafe / $buildSide / $joinType" - val someData = (1 to 100).map { i => (i, "burger" + i) }.toArray - val conf = new SQLConf - conf.setConf(SQLConf.UNSAFE_ENABLED, unsafeAndCodegen) - conf.setConf(SQLConf.CODEGEN_ENABLED, unsafeAndCodegen) - - // Actual test body - def runTest( - joinType: JoinType, - leftInput: Array[(Int, String)], - rightInput: Array[(Int, String)]): Unit = { - val leftNode = new DummyNode(joinNameAttributes, leftInput) - val rightNode = new DummyNode(joinNicknameAttributes, rightInput) - val cond = 'id1 === 'id2 - val makeNode = (node1: LocalNode, node2: LocalNode) => { - resolveExpressions( - new NestedLoopJoinNode(conf, node1, node2, buildSide, joinType, Some(cond))) - } - val makeUnsafeNode = if (unsafeAndCodegen) wrapForUnsafe(makeNode) else makeNode - val hashJoinNode = makeUnsafeNode(leftNode, rightNode) - val expectedOutput = generateExpectedOutput(leftInput, rightInput, joinType) - val actualOutput = hashJoinNode.collect().map { row => - // (id, name, id, nickname) - (row.getInt(0), row.getString(1), row.getInt(2), row.getString(3)) - } - assert(actualOutput.toSet === expectedOutput.toSet) - } - - test(s"$testNamePrefix: empty") { - runTest(joinType, Array.empty, Array.empty) - } - - test(s"$testNamePrefix: no matches") { - val someIrrelevantData = (10000 to 10100).map { i => (i, "piper" + i) }.toArray - runTest(joinType, someData, Array.empty) - runTest(joinType, Array.empty, someData) - runTest(joinType, someData, someIrrelevantData) - runTest(joinType, someIrrelevantData, someData) - } - - test(s"$testNamePrefix: partial matches") { - val someOtherData = (50 to 150).map { i => (i, "finnegan" + i) }.toArray - runTest(joinType, someData, someOtherData) - runTest(joinType, someOtherData, someData) - } - - test(s"$testNamePrefix: full matches") { - val someSuperRelevantData = someData.map { case (k, v) => (k, "cooper" + v) } - runTest(joinType, someData, someSuperRelevantData) - runTest(joinType, someSuperRelevantData, someData) - } - } - - /** - * Helper method to generate the expected output of a test based on the join type. - */ - private def generateExpectedOutput( - leftInput: Array[(Int, String)], - rightInput: Array[(Int, String)], - joinType: JoinType): Array[(Int, String, Int, String)] = { - joinType match { - case LeftOuter => - val rightInputMap = rightInput.toMap - leftInput.map { case (k, v) => - val rightKey = rightInputMap.get(k).map { _ => k }.getOrElse(0) - val rightValue = rightInputMap.getOrElse(k, null) - (k, v, rightKey, rightValue) - } - - case RightOuter => - val leftInputMap = leftInput.toMap - rightInput.map { case (k, v) => - val leftKey = leftInputMap.get(k).map { _ => k }.getOrElse(0) - val leftValue = leftInputMap.getOrElse(k, null) - (leftKey, leftValue, k, v) - } - - case FullOuter => - val leftInputMap = leftInput.toMap - val rightInputMap = rightInput.toMap - val leftOutput = leftInput.map { case (k, v) => - val rightKey = rightInputMap.get(k).map { _ => k }.getOrElse(0) - val rightValue = rightInputMap.getOrElse(k, null) - (k, v, rightKey, rightValue) - } - val rightOutput = rightInput.map { case (k, v) => - val leftKey = leftInputMap.get(k).map { _ => k }.getOrElse(0) - val leftValue = leftInputMap.getOrElse(k, null) - (leftKey, leftValue, k, v) - } - (leftOutput ++ rightOutput).distinct - - case other => - throw new IllegalArgumentException(s"Join type $other is not applicable") - } - } - -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/ProjectNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/ProjectNodeSuite.scala deleted file mode 100644 index 02ecb23d34b2..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/ProjectNodeSuite.scala +++ /dev/null @@ -1,49 +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.execution.local - -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, NamedExpression} -import org.apache.spark.sql.types.{IntegerType, StringType} - - -class ProjectNodeSuite extends LocalNodeTest { - private val pieAttributes = Seq( - AttributeReference("id", IntegerType)(), - AttributeReference("age", IntegerType)(), - AttributeReference("name", StringType)()) - - private def testProject(inputData: Array[(Int, Int, String)] = Array.empty): Unit = { - val inputNode = new DummyNode(pieAttributes, inputData) - val columns = Seq[NamedExpression](inputNode.output(0), inputNode.output(2)) - val projectNode = new ProjectNode(conf, columns, inputNode) - val expectedOutput = inputData.map { case (id, age, name) => (id, name) } - val actualOutput = projectNode.collect().map { case row => - (row.getInt(0), row.getString(1)) - } - assert(actualOutput === expectedOutput) - } - - test("empty") { - testProject() - } - - test("basic") { - testProject((1 to 100).map { i => (i, i + 1, "pie" + i) }.toArray) - } - -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/SampleNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/SampleNodeSuite.scala deleted file mode 100644 index a3e83bbd5145..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/SampleNodeSuite.scala +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.local - -import org.apache.spark.util.random.{BernoulliCellSampler, PoissonSampler} - - -class SampleNodeSuite extends LocalNodeTest { - - private def testSample(withReplacement: Boolean): Unit = { - val seed = 0L - val lowerb = 0.0 - val upperb = 0.3 - val maybeOut = if (withReplacement) "" else "out" - test(s"with$maybeOut replacement") { - val inputData = (1 to 1000).map { i => (i, i) }.toArray - val inputNode = new DummyNode(kvIntAttributes, inputData) - val sampleNode = new SampleNode(conf, lowerb, upperb, withReplacement, seed, inputNode) - val sampler = - if (withReplacement) { - new PoissonSampler[(Int, Int)](upperb - lowerb, useGapSamplingIfPossible = false) - } else { - new BernoulliCellSampler[(Int, Int)](lowerb, upperb) - } - sampler.setSeed(seed) - val expectedOutput = sampler.sample(inputData.iterator).toArray - val actualOutput = sampleNode.collect().map { case row => - (row.getInt(0), row.getInt(1)) - } - assert(actualOutput === expectedOutput) - } - } - - testSample(withReplacement = true) - testSample(withReplacement = false) -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/TakeOrderedAndProjectNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/TakeOrderedAndProjectNodeSuite.scala deleted file mode 100644 index 42ebc7bfcaad..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/TakeOrderedAndProjectNodeSuite.scala +++ /dev/null @@ -1,50 +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.execution.local - -import scala.util.Random - -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.SortOrder - - -class TakeOrderedAndProjectNodeSuite extends LocalNodeTest { - - private def testTakeOrderedAndProject(desc: Boolean): Unit = { - val limit = 10 - val ascOrDesc = if (desc) "desc" else "asc" - test(ascOrDesc) { - val inputData = Random.shuffle((1 to 100).toList).map { i => (i, i) }.toArray - val inputNode = new DummyNode(kvIntAttributes, inputData) - val firstColumn = inputNode.output(0) - val sortDirection = if (desc) Descending else Ascending - val sortOrder = SortOrder(firstColumn, sortDirection) - val takeOrderAndProjectNode = new TakeOrderedAndProjectNode( - conf, limit, Seq(sortOrder), Some(Seq(firstColumn)), inputNode) - val expectedOutput = inputData - .map { case (k, _) => k } - .sortBy { k => k * (if (desc) -1 else 1) } - .take(limit) - val actualOutput = takeOrderAndProjectNode.collect().map { row => row.getInt(0) } - assert(actualOutput === expectedOutput) - } - } - - testTakeOrderedAndProject(desc = false) - testTakeOrderedAndProject(desc = true) -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/UnionNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/UnionNodeSuite.scala deleted file mode 100644 index 666b0235c061..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/UnionNodeSuite.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ - -package org.apache.spark.sql.execution.local - - -class UnionNodeSuite extends LocalNodeTest { - - private def testUnion(inputData: Seq[Array[(Int, Int)]]): Unit = { - val inputNodes = inputData.map { data => - new DummyNode(kvIntAttributes, data) - } - val unionNode = new UnionNode(conf, inputNodes) - val expectedOutput = inputData.flatten - val actualOutput = unionNode.collect().map { case row => - (row.getInt(0), row.getInt(1)) - } - assert(actualOutput === expectedOutput) - } - - test("empty") { - testUnion(Seq(Array.empty)) - testUnion(Seq(Array.empty, Array.empty)) - } - - test("self") { - val data = (1 to 100).map { i => (i, i) }.toArray - testUnion(Seq(data)) - testUnion(Seq(data, data)) - testUnion(Seq(data, data, data)) - } - - test("basic") { - val zero = Array.empty[(Int, Int)] - val one = (1 to 100).map { i => (i, i) }.toArray - val two = (50 to 150).map { i => (i, i) }.toArray - val three = (800 to 900).map { i => (i, i) }.toArray - testUnion(Seq(zero, one, two, three)) - } - -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index cdd885ba1420..0dc612ef735f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -1,144 +1,66 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.sql.execution.metric -import java.io.{ByteArrayInputStream, ByteArrayOutputStream} +import java.io.File -import scala.collection.mutable - -import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm._ -import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ +import scala.util.Random import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ -import org.apache.spark.sql.execution.ui.SparkPlanGraph +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.util.Utils - +import org.apache.spark.util.{AccumulatorContext, JsonProtocol} -class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { +class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with SharedSQLContext { import testImplicits._ - test("LongSQLMetric should not box Long") { - val l = SQLMetrics.createLongMetric(sparkContext, "long") - val f = () => { - l += 1L - l.add(1L) - } - BoxingFinder.getClassReader(f.getClass).foreach { cl => - val boxingFinder = new BoxingFinder() - cl.accept(boxingFinder, 0) - assert(boxingFinder.boxingInvokes.isEmpty, s"Found boxing: ${boxingFinder.boxingInvokes}") - } - } - - test("Normal accumulator should do boxing") { - // We need this test to make sure BoxingFinder works. - val l = sparkContext.accumulator(0L) - val f = () => { l += 1L } - BoxingFinder.getClassReader(f.getClass).foreach { cl => - val boxingFinder = new BoxingFinder() - cl.accept(boxingFinder, 0) - assert(boxingFinder.boxingInvokes.nonEmpty, "Found find boxing in this test") - } - } /** - * Call `df.collect()` and verify if the collected metrics are same as "expectedMetrics". - * - * @param df `DataFrame` to run - * @param expectedNumOfJobs number of jobs that will run - * @param expectedMetrics the expected metrics. The format is - * `nodeId -> (operatorName, metric name -> metric value)`. + * Generates a `DataFrame` by filling randomly generated bytes for hash collision. */ - private def testSparkPlanMetrics( - df: DataFrame, - expectedNumOfJobs: Int, - expectedMetrics: Map[Long, (String, Map[String, Any])]): Unit = { - val previousExecutionIds = sqlContext.listener.executionIdToData.keySet - df.collect() - sparkContext.listenerBus.waitUntilEmpty(10000) - val executionIds = sqlContext.listener.executionIdToData.keySet.diff(previousExecutionIds) - assert(executionIds.size === 1) - val executionId = executionIds.head - val jobs = sqlContext.listener.getExecution(executionId).get.jobs - // Use "<=" because there is a race condition that we may miss some jobs - // TODO Change it to "=" once we fix the race condition that missing the JobStarted event. - assert(jobs.size <= expectedNumOfJobs) - if (jobs.size == expectedNumOfJobs) { - // If we can track all jobs, check the metric values - val metricValues = sqlContext.listener.getExecutionMetrics(executionId) - val actualMetrics = SparkPlanGraph(df.queryExecution.executedPlan).nodes.filter { node => - expectedMetrics.contains(node.id) - }.map { node => - val nodeMetrics = node.metrics.map { metric => - val metricValue = metricValues(metric.accumulatorId) - (metric.name, metricValue) - }.toMap - (node.id, node.name -> nodeMetrics) - }.toMap - - assert(expectedMetrics.keySet === actualMetrics.keySet) - for (nodeId <- expectedMetrics.keySet) { - val (expectedNodeName, expectedMetricsMap) = expectedMetrics(nodeId) - val (actualNodeName, actualMetricsMap) = actualMetrics(nodeId) - assert(expectedNodeName === actualNodeName) - for (metricName <- expectedMetricsMap.keySet) { - assert(expectedMetricsMap(metricName).toString === actualMetricsMap(metricName)) - } - } - } else { - // TODO Remove this "else" once we fix the race condition that missing the JobStarted event. - // Since we cannot track all jobs, the metric values could be wrong and we should not check - // them. - logWarning("Due to a race condition, we miss some jobs and cannot verify the metric values") - } - } - - test("Project metrics") { - withSQLConf( - SQLConf.UNSAFE_ENABLED.key -> "false", - SQLConf.CODEGEN_ENABLED.key -> "false", - SQLConf.TUNGSTEN_ENABLED.key -> "false") { - // Assume the execution plan is - // PhysicalRDD(nodeId = 1) -> Project(nodeId = 0) - val df = person.select('name) - testSparkPlanMetrics(df, 1, Map( - 0L ->("Project", Map( - "number of rows" -> 2L))) - ) + private def generateRandomBytesDF(numRows: Int = 65535): DataFrame = { + val random = new Random() + val manyBytes = (0 until numRows).map { _ => + val byteArrSize = random.nextInt(100) + val bytes = new Array[Byte](byteArrSize) + random.nextBytes(bytes) + (bytes, random.nextInt(100)) } + manyBytes.toSeq.toDF("a", "b") } - test("TungstenProject metrics") { - withSQLConf( - SQLConf.UNSAFE_ENABLED.key -> "true", - SQLConf.CODEGEN_ENABLED.key -> "true", - SQLConf.TUNGSTEN_ENABLED.key -> "true") { - // Assume the execution plan is - // PhysicalRDD(nodeId = 1) -> TungstenProject(nodeId = 0) - val df = person.select('name) - testSparkPlanMetrics(df, 1, Map( - 0L ->("TungstenProject", Map( - "number of rows" -> 2L))) - ) - } + test("LocalTableScanExec computes metrics in collect and take") { + val df1 = spark.createDataset(Seq(1, 2, 3)) + val logical = df1.queryExecution.logical + require(logical.isInstanceOf[LocalRelation]) + df1.collect() + val metrics1 = df1.queryExecution.executedPlan.collectLeaves().head.metrics + assert(metrics1.contains("numOutputRows")) + assert(metrics1("numOutputRows").value === 3) + + val df2 = spark.createDataset(Seq(1, 2, 3)).limit(2) + df2.collect() + val metrics2 = df2.queryExecution.executedPlan.collectLeaves().head.metrics + assert(metrics2.contains("numOutputRows")) + assert(metrics2("numOutputRows").value === 2) } test("Filter metrics") { @@ -146,441 +68,436 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { // PhysicalRDD(nodeId = 1) -> Filter(nodeId = 0) val df = person.filter('age < 25) testSparkPlanMetrics(df, 1, Map( - 0L -> ("Filter", Map( - "number of input rows" -> 2L, - "number of output rows" -> 1L))) + 0L -> (("Filter", Map( + "number of output rows" -> 1L)))) ) } + test("WholeStageCodegen metrics") { + // Assume the execution plan is + // WholeStageCodegen(nodeId = 0, Range(nodeId = 2) -> Filter(nodeId = 1)) + // TODO: update metrics in generated operators + val ds = spark.range(10).filter('id < 5) + testSparkPlanMetrics(ds.toDF(), 1, Map.empty) + } + test("Aggregate metrics") { - withSQLConf( - SQLConf.UNSAFE_ENABLED.key -> "false", - SQLConf.CODEGEN_ENABLED.key -> "false", - SQLConf.TUNGSTEN_ENABLED.key -> "false") { - // Assume the execution plan is - // ... -> Aggregate(nodeId = 2) -> TungstenExchange(nodeId = 1) -> Aggregate(nodeId = 0) - val df = testData2.groupBy().count() // 2 partitions - testSparkPlanMetrics(df, 1, Map( - 2L -> ("Aggregate", Map( - "number of input rows" -> 6L, - "number of output rows" -> 2L)), - 0L -> ("Aggregate", Map( - "number of input rows" -> 2L, - "number of output rows" -> 1L))) - ) + // Assume the execution plan is + // ... -> HashAggregate(nodeId = 2) -> Exchange(nodeId = 1) + // -> HashAggregate(nodeId = 0) + val df = testData2.groupBy().count() // 2 partitions + val expected1 = Seq( + Map("number of output rows" -> 2L, + "avg hash probe (min, med, max)" -> "\n(1, 1, 1)"), + Map("number of output rows" -> 1L, + "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")) + testSparkPlanMetrics(df, 1, Map( + 2L -> (("HashAggregate", expected1(0))), + 0L -> (("HashAggregate", expected1(1)))) + ) - // 2 partitions and each partition contains 2 keys - val df2 = testData2.groupBy('a).count() - testSparkPlanMetrics(df2, 1, Map( - 2L -> ("Aggregate", Map( - "number of input rows" -> 6L, - "number of output rows" -> 4L)), - 0L -> ("Aggregate", Map( - "number of input rows" -> 4L, - "number of output rows" -> 3L))) - ) + // 2 partitions and each partition contains 2 keys + val df2 = testData2.groupBy('a).count() + val expected2 = Seq( + Map("number of output rows" -> 4L, + "avg hash probe (min, med, max)" -> "\n(1, 1, 1)"), + Map("number of output rows" -> 3L, + "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")) + testSparkPlanMetrics(df2, 1, Map( + 2L -> (("HashAggregate", expected2(0))), + 0L -> (("HashAggregate", expected2(1)))) + ) + } + + test("Aggregate metrics: track avg probe") { + // The executed plan looks like: + // HashAggregate(keys=[a#61], functions=[count(1)], output=[a#61, count#71L]) + // +- Exchange hashpartitioning(a#61, 5) + // +- HashAggregate(keys=[a#61], functions=[partial_count(1)], output=[a#61, count#76L]) + // +- Exchange RoundRobinPartitioning(1) + // +- LocalTableScan [a#61] + // + // Assume the execution plan with node id is: + // Wholestage disabled: + // HashAggregate(nodeId = 0) + // Exchange(nodeId = 1) + // HashAggregate(nodeId = 2) + // Exchange (nodeId = 3) + // LocalTableScan(nodeId = 4) + // + // Wholestage enabled: + // WholeStageCodegen(nodeId = 0) + // HashAggregate(nodeId = 1) + // Exchange(nodeId = 2) + // WholeStageCodegen(nodeId = 3) + // HashAggregate(nodeId = 4) + // Exchange(nodeId = 5) + // LocalTableScan(nodeId = 6) + Seq(true, false).foreach { enableWholeStage => + val df = generateRandomBytesDF().repartition(1).groupBy('a).count() + val nodeIds = if (enableWholeStage) { + Set(4L, 1L) + } else { + Set(2L, 0L) + } + val metrics = getSparkPlanMetrics(df, 1, nodeIds, enableWholeStage).get + nodeIds.foreach { nodeId => + val probes = metrics(nodeId)._2("avg hash probe (min, med, max)") + probes.toString.stripPrefix("\n(").stripSuffix(")").split(", ").foreach { probe => + assert(probe.toDouble > 1.0) + } + } } } - test("SortBasedAggregate metrics") { - // Because SortBasedAggregate may skip different rows if the number of partitions is different, - // this test should use the deterministic number of partitions. - withSQLConf( - SQLConf.UNSAFE_ENABLED.key -> "false", - SQLConf.CODEGEN_ENABLED.key -> "true", - SQLConf.TUNGSTEN_ENABLED.key -> "true") { - // Assume the execution plan is - // ... -> SortBasedAggregate(nodeId = 2) -> TungstenExchange(nodeId = 1) -> - // SortBasedAggregate(nodeId = 0) - val df = testData2.groupBy().count() // 2 partitions - testSparkPlanMetrics(df, 1, Map( - 2L -> ("SortBasedAggregate", Map( - "number of input rows" -> 6L, - "number of output rows" -> 2L)), - 0L -> ("SortBasedAggregate", Map( - "number of input rows" -> 2L, - "number of output rows" -> 1L))) - ) + test("ObjectHashAggregate metrics") { + // Assume the execution plan is + // ... -> ObjectHashAggregate(nodeId = 2) -> Exchange(nodeId = 1) + // -> ObjectHashAggregate(nodeId = 0) + val df = testData2.groupBy().agg(collect_set('a)) // 2 partitions + testSparkPlanMetrics(df, 1, Map( + 2L -> (("ObjectHashAggregate", Map("number of output rows" -> 2L))), + 0L -> (("ObjectHashAggregate", Map("number of output rows" -> 1L)))) + ) + + // 2 partitions and each partition contains 2 keys + val df2 = testData2.groupBy('a).agg(collect_set('a)) + testSparkPlanMetrics(df2, 1, Map( + 2L -> (("ObjectHashAggregate", Map("number of output rows" -> 4L))), + 0L -> (("ObjectHashAggregate", Map("number of output rows" -> 3L)))) + ) + } + test("Sort metrics") { + // Assume the execution plan is + // WholeStageCodegen(nodeId = 0, Range(nodeId = 2) -> Sort(nodeId = 1)) + val ds = spark.range(10).sort('id) + testSparkPlanMetrics(ds.toDF(), 2, Map.empty) + } + + test("SortMergeJoin metrics") { + // Because SortMergeJoin may skip different rows if the number of partitions is different, this + // test should use the deterministic number of partitions. + val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) + testDataForJoin.createOrReplaceTempView("testDataForJoin") + withTempView("testDataForJoin") { // Assume the execution plan is - // ... -> SortBasedAggregate(nodeId = 3) -> TungstenExchange(nodeId = 2) - // -> ExternalSort(nodeId = 1)-> SortBasedAggregate(nodeId = 0) - // 2 partitions and each partition contains 2 keys - val df2 = testData2.groupBy('a).count() - testSparkPlanMetrics(df2, 1, Map( - 3L -> ("SortBasedAggregate", Map( - "number of input rows" -> 6L, - "number of output rows" -> 4L)), - 0L -> ("SortBasedAggregate", Map( - "number of input rows" -> 4L, - "number of output rows" -> 3L))) + // ... -> SortMergeJoin(nodeId = 1) -> TungstenProject(nodeId = 0) + val df = spark.sql( + "SELECT * FROM testData2 JOIN testDataForJoin ON testData2.a = testDataForJoin.a") + testSparkPlanMetrics(df, 1, Map( + 0L -> (("SortMergeJoin", Map( + // It's 4 because we only read 3 rows in the first partition and 1 row in the second one + "number of output rows" -> 4L)))) ) } } - test("TungstenAggregate metrics") { - withSQLConf( - SQLConf.UNSAFE_ENABLED.key -> "true", - SQLConf.CODEGEN_ENABLED.key -> "true", - SQLConf.TUNGSTEN_ENABLED.key -> "true") { + test("SortMergeJoin(outer) metrics") { + // Because SortMergeJoin may skip different rows if the number of partitions is different, + // this test should use the deterministic number of partitions. + val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) + testDataForJoin.createOrReplaceTempView("testDataForJoin") + withTempView("testDataForJoin") { // Assume the execution plan is - // ... -> TungstenAggregate(nodeId = 2) -> Exchange(nodeId = 1) - // -> TungstenAggregate(nodeId = 0) - val df = testData2.groupBy().count() // 2 partitions + // ... -> SortMergeJoin(nodeId = 1) -> TungstenProject(nodeId = 0) + val df = spark.sql( + "SELECT * FROM testData2 left JOIN testDataForJoin ON testData2.a = testDataForJoin.a") testSparkPlanMetrics(df, 1, Map( - 2L -> ("TungstenAggregate", Map( - "number of input rows" -> 6L, - "number of output rows" -> 2L)), - 0L -> ("TungstenAggregate", Map( - "number of input rows" -> 2L, - "number of output rows" -> 1L))) + 0L -> (("SortMergeJoin", Map( + // It's 4 because we only read 3 rows in the first partition and 1 row in the second one + "number of output rows" -> 8L)))) ) - // 2 partitions and each partition contains 2 keys - val df2 = testData2.groupBy('a).count() + val df2 = spark.sql( + "SELECT * FROM testDataForJoin right JOIN testData2 ON testData2.a = testDataForJoin.a") testSparkPlanMetrics(df2, 1, Map( - 2L -> ("TungstenAggregate", Map( - "number of input rows" -> 6L, - "number of output rows" -> 4L)), - 0L -> ("TungstenAggregate", Map( - "number of input rows" -> 4L, - "number of output rows" -> 3L))) + 0L -> (("SortMergeJoin", Map( + // It's 4 because we only read 3 rows in the first partition and 1 row in the second one + "number of output rows" -> 8L)))) ) } } - test("SortMergeJoin metrics") { - // Because SortMergeJoin may skip different rows if the number of partitions is different, this - // test should use the deterministic number of partitions. - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> "true") { - val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") - withTempTable("testDataForJoin") { - // Assume the execution plan is - // ... -> SortMergeJoin(nodeId = 1) -> TungstenProject(nodeId = 0) - val df = sqlContext.sql( - "SELECT * FROM testData2 JOIN testDataForJoin ON testData2.a = testDataForJoin.a") - testSparkPlanMetrics(df, 1, Map( - 1L -> ("SortMergeJoin", Map( - // It's 4 because we only read 3 rows in the first partition and 1 row in the second one - "number of left rows" -> 4L, - "number of right rows" -> 2L, - "number of output rows" -> 4L))) - ) - } - } + test("BroadcastHashJoin metrics") { + val df1 = Seq((1, "1"), (2, "2")).toDF("key", "value") + val df2 = Seq((1, "1"), (2, "2"), (3, "3"), (4, "4")).toDF("key", "value") + // Assume the execution plan is + // ... -> BroadcastHashJoin(nodeId = 1) -> TungstenProject(nodeId = 0) + val df = df1.join(broadcast(df2), "key") + testSparkPlanMetrics(df, 2, Map( + 1L -> (("BroadcastHashJoin", Map( + "number of output rows" -> 2L, + "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")))) + ) } - test("SortMergeOuterJoin metrics") { - // Because SortMergeOuterJoin may skip different rows if the number of partitions is different, - // this test should use the deterministic number of partitions. - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> "true") { - val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") - withTempTable("testDataForJoin") { - // Assume the execution plan is - // ... -> SortMergeOuterJoin(nodeId = 1) -> TungstenProject(nodeId = 0) - val df = sqlContext.sql( - "SELECT * FROM testData2 left JOIN testDataForJoin ON testData2.a = testDataForJoin.a") - testSparkPlanMetrics(df, 1, Map( - 1L -> ("SortMergeOuterJoin", Map( - // It's 4 because we only read 3 rows in the first partition and 1 row in the second one - "number of left rows" -> 6L, - "number of right rows" -> 2L, - "number of output rows" -> 8L))) - ) - - val df2 = sqlContext.sql( - "SELECT * FROM testDataForJoin right JOIN testData2 ON testData2.a = testDataForJoin.a") - testSparkPlanMetrics(df2, 1, Map( - 1L -> ("SortMergeOuterJoin", Map( - // It's 4 because we only read 3 rows in the first partition and 1 row in the second one - "number of left rows" -> 2L, - "number of right rows" -> 6L, - "number of output rows" -> 8L))) - ) + test("BroadcastHashJoin metrics: track avg probe") { + // The executed plan looks like: + // Project [a#210, b#211, b#221] + // +- BroadcastHashJoin [a#210], [a#220], Inner, BuildRight + // :- Project [_1#207 AS a#210, _2#208 AS b#211] + // : +- Filter isnotnull(_1#207) + // : +- LocalTableScan [_1#207, _2#208] + // +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, binary, true])) + // +- Project [_1#217 AS a#220, _2#218 AS b#221] + // +- Filter isnotnull(_1#217) + // +- LocalTableScan [_1#217, _2#218] + // + // Assume the execution plan with node id is + // WholeStageCodegen disabled: + // Project(nodeId = 0) + // BroadcastHashJoin(nodeId = 1) + // ...(ignored) + // + // WholeStageCodegen enabled: + // WholeStageCodegen(nodeId = 0) + // Project(nodeId = 1) + // BroadcastHashJoin(nodeId = 2) + // Project(nodeId = 3) + // Filter(nodeId = 4) + // ...(ignored) + Seq(true, false).foreach { enableWholeStage => + val df1 = generateRandomBytesDF() + val df2 = generateRandomBytesDF() + val df = df1.join(broadcast(df2), "a") + val nodeIds = if (enableWholeStage) { + Set(2L) + } else { + Set(1L) + } + val metrics = getSparkPlanMetrics(df, 2, nodeIds, enableWholeStage).get + nodeIds.foreach { nodeId => + val probes = metrics(nodeId)._2("avg hash probe (min, med, max)") + probes.toString.stripPrefix("\n(").stripSuffix(")").split(", ").foreach { probe => + assert(probe.toDouble > 1.0) + } } } } - test("BroadcastHashJoin metrics") { - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> "false") { + test("ShuffledHashJoin metrics") { + withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "40", + "spark.sql.shuffle.partitions" -> "2", + "spark.sql.join.preferSortMergeJoin" -> "false") { val df1 = Seq((1, "1"), (2, "2")).toDF("key", "value") - val df2 = Seq((1, "1"), (2, "2"), (3, "3"), (4, "4")).toDF("key", "value") + val df2 = (1 to 10).map(i => (i, i.toString)).toSeq.toDF("key", "value") // Assume the execution plan is - // ... -> BroadcastHashJoin(nodeId = 1) -> TungstenProject(nodeId = 0) - val df = df1.join(broadcast(df2), "key") - testSparkPlanMetrics(df, 2, Map( - 1L -> ("BroadcastHashJoin", Map( - "number of left rows" -> 2L, - "number of right rows" -> 4L, - "number of output rows" -> 2L))) + // ... -> ShuffledHashJoin(nodeId = 1) -> Project(nodeId = 0) + val df = df1.join(df2, "key") + val metrics = getSparkPlanMetrics(df, 1, Set(1L)) + testSparkPlanMetrics(df, 1, Map( + 1L -> (("ShuffledHashJoin", Map( + "number of output rows" -> 2L, + "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")))) ) } } - test("ShuffledHashJoin metrics") { - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> "false") { - val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") - withTempTable("testDataForJoin") { - // Assume the execution plan is - // ... -> ShuffledHashJoin(nodeId = 1) -> TungstenProject(nodeId = 0) - val df = sqlContext.sql( - "SELECT * FROM testData2 JOIN testDataForJoin ON testData2.a = testDataForJoin.a") - testSparkPlanMetrics(df, 1, Map( - 1L -> ("ShuffledHashJoin", Map( - "number of left rows" -> 6L, - "number of right rows" -> 2L, - "number of output rows" -> 4L))) - ) + test("ShuffledHashJoin metrics: track avg probe") { + // The executed plan looks like: + // Project [a#308, b#309, b#319] + // +- ShuffledHashJoin [a#308], [a#318], Inner, BuildRight + // :- Exchange hashpartitioning(a#308, 2) + // : +- Project [_1#305 AS a#308, _2#306 AS b#309] + // : +- Filter isnotnull(_1#305) + // : +- LocalTableScan [_1#305, _2#306] + // +- Exchange hashpartitioning(a#318, 2) + // +- Project [_1#315 AS a#318, _2#316 AS b#319] + // +- Filter isnotnull(_1#315) + // +- LocalTableScan [_1#315, _2#316] + // + // Assume the execution plan with node id is + // WholeStageCodegen disabled: + // Project(nodeId = 0) + // ShuffledHashJoin(nodeId = 1) + // ...(ignored) + // + // WholeStageCodegen enabled: + // WholeStageCodegen(nodeId = 0) + // Project(nodeId = 1) + // ShuffledHashJoin(nodeId = 2) + // ...(ignored) + withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "5000000", + "spark.sql.shuffle.partitions" -> "2", + "spark.sql.join.preferSortMergeJoin" -> "false") { + Seq(true, false).foreach { enableWholeStage => + val df1 = generateRandomBytesDF(65535 * 5) + val df2 = generateRandomBytesDF(65535) + val df = df1.join(df2, "a") + val nodeIds = if (enableWholeStage) { + Set(2L) + } else { + Set(1L) + } + val metrics = getSparkPlanMetrics(df, 1, nodeIds, enableWholeStage).get + nodeIds.foreach { nodeId => + val probes = metrics(nodeId)._2("avg hash probe (min, med, max)") + probes.toString.stripPrefix("\n(").stripSuffix(")").split(", ").foreach { probe => + assert(probe.toDouble > 1.0) + } + } } } } - test("ShuffledHashOuterJoin metrics") { - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> "false", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0") { - val df1 = Seq((1, "a"), (1, "b"), (4, "c")).toDF("key", "value") - val df2 = Seq((1, "a"), (1, "b"), (2, "c"), (3, "d")).toDF("key2", "value") - // Assume the execution plan is - // ... -> ShuffledHashOuterJoin(nodeId = 0) - val df = df1.join(df2, $"key" === $"key2", "left_outer") - testSparkPlanMetrics(df, 1, Map( - 0L -> ("ShuffledHashOuterJoin", Map( - "number of left rows" -> 3L, - "number of right rows" -> 4L, - "number of output rows" -> 5L))) - ) - - val df3 = df1.join(df2, $"key" === $"key2", "right_outer") - testSparkPlanMetrics(df3, 1, Map( - 0L -> ("ShuffledHashOuterJoin", Map( - "number of left rows" -> 3L, - "number of right rows" -> 4L, - "number of output rows" -> 6L))) - ) - - val df4 = df1.join(df2, $"key" === $"key2", "outer") - testSparkPlanMetrics(df4, 1, Map( - 0L -> ("ShuffledHashOuterJoin", Map( - "number of left rows" -> 3L, - "number of right rows" -> 4L, - "number of output rows" -> 7L))) - ) - } - } - - test("BroadcastHashOuterJoin metrics") { - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> "false") { - val df1 = Seq((1, "a"), (1, "b"), (4, "c")).toDF("key", "value") - val df2 = Seq((1, "a"), (1, "b"), (2, "c"), (3, "d")).toDF("key2", "value") - // Assume the execution plan is - // ... -> BroadcastHashOuterJoin(nodeId = 0) - val df = df1.join(broadcast(df2), $"key" === $"key2", "left_outer") - testSparkPlanMetrics(df, 2, Map( - 0L -> ("BroadcastHashOuterJoin", Map( - "number of left rows" -> 3L, - "number of right rows" -> 4L, - "number of output rows" -> 5L))) - ) + test("BroadcastHashJoin(outer) metrics") { + val df1 = Seq((1, "a"), (1, "b"), (4, "c")).toDF("key", "value") + val df2 = Seq((1, "a"), (1, "b"), (2, "c"), (3, "d")).toDF("key2", "value") + // Assume the execution plan is + // ... -> BroadcastHashJoin(nodeId = 0) + val df = df1.join(broadcast(df2), $"key" === $"key2", "left_outer") + testSparkPlanMetrics(df, 2, Map( + 0L -> (("BroadcastHashJoin", Map( + "number of output rows" -> 5L)))) + ) - val df3 = df1.join(broadcast(df2), $"key" === $"key2", "right_outer") - testSparkPlanMetrics(df3, 2, Map( - 0L -> ("BroadcastHashOuterJoin", Map( - "number of left rows" -> 3L, - "number of right rows" -> 4L, - "number of output rows" -> 6L))) - ) - } + val df3 = df1.join(broadcast(df2), $"key" === $"key2", "right_outer") + testSparkPlanMetrics(df3, 2, Map( + 0L -> (("BroadcastHashJoin", Map( + "number of output rows" -> 6L)))) + ) } test("BroadcastNestedLoopJoin metrics") { - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> "true") { - val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") - withTempTable("testDataForJoin") { + val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) + testDataForJoin.createOrReplaceTempView("testDataForJoin") + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + withTempView("testDataForJoin") { // Assume the execution plan is // ... -> BroadcastNestedLoopJoin(nodeId = 1) -> TungstenProject(nodeId = 0) - val df = sqlContext.sql( + val df = spark.sql( "SELECT * FROM testData2 left JOIN testDataForJoin ON " + "testData2.a * testDataForJoin.a != testData2.a + testDataForJoin.a") testSparkPlanMetrics(df, 3, Map( - 1L -> ("BroadcastNestedLoopJoin", Map( - "number of left rows" -> 12L, // left needs to be scanned twice - "number of right rows" -> 2L, - "number of output rows" -> 12L))) + 1L -> (("BroadcastNestedLoopJoin", Map( + "number of output rows" -> 12L)))) ) } } } test("BroadcastLeftSemiJoinHash metrics") { - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> "false") { - val df1 = Seq((1, "1"), (2, "2")).toDF("key", "value") - val df2 = Seq((1, "1"), (2, "2"), (3, "3"), (4, "4")).toDF("key2", "value") - // Assume the execution plan is - // ... -> BroadcastLeftSemiJoinHash(nodeId = 0) - val df = df1.join(broadcast(df2), $"key" === $"key2", "leftsemi") - testSparkPlanMetrics(df, 2, Map( - 0L -> ("BroadcastLeftSemiJoinHash", Map( - "number of left rows" -> 2L, - "number of right rows" -> 4L, - "number of output rows" -> 2L))) - ) - } - } - - test("LeftSemiJoinHash metrics") { - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0") { - val df1 = Seq((1, "1"), (2, "2")).toDF("key", "value") - val df2 = Seq((1, "1"), (2, "2"), (3, "3"), (4, "4")).toDF("key2", "value") - // Assume the execution plan is - // ... -> LeftSemiJoinHash(nodeId = 0) - val df = df1.join(df2, $"key" === $"key2", "leftsemi") - testSparkPlanMetrics(df, 1, Map( - 0L -> ("LeftSemiJoinHash", Map( - "number of left rows" -> 2L, - "number of right rows" -> 4L, - "number of output rows" -> 2L))) - ) - } + val df1 = Seq((1, "1"), (2, "2")).toDF("key", "value") + val df2 = Seq((1, "1"), (2, "2"), (3, "3"), (4, "4")).toDF("key2", "value") + // Assume the execution plan is + // ... -> BroadcastHashJoin(nodeId = 0) + val df = df1.join(broadcast(df2), $"key" === $"key2", "leftsemi") + testSparkPlanMetrics(df, 2, Map( + 0L -> (("BroadcastHashJoin", Map( + "number of output rows" -> 2L)))) + ) } - test("LeftSemiJoinBNL metrics") { - withSQLConf(SQLConf.SORTMERGE_JOIN.key -> "false") { - val df1 = Seq((1, "1"), (2, "2")).toDF("key", "value") - val df2 = Seq((1, "1"), (2, "2"), (3, "3"), (4, "4")).toDF("key2", "value") - // Assume the execution plan is - // ... -> LeftSemiJoinBNL(nodeId = 0) - val df = df1.join(df2, $"key" < $"key2", "leftsemi") - testSparkPlanMetrics(df, 2, Map( - 0L -> ("LeftSemiJoinBNL", Map( - "number of left rows" -> 2L, - "number of right rows" -> 4L, - "number of output rows" -> 2L))) - ) + test("CartesianProduct metrics") { + withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) + testDataForJoin.createOrReplaceTempView("testDataForJoin") + withTempView("testDataForJoin") { + // Assume the execution plan is + // ... -> CartesianProduct(nodeId = 1) -> TungstenProject(nodeId = 0) + val df = spark.sql( + "SELECT * FROM testData2 JOIN testDataForJoin") + testSparkPlanMetrics(df, 1, Map( + 0L -> (("CartesianProduct", Map("number of output rows" -> 12L)))) + ) + } } } - test("CartesianProduct metrics") { - val testDataForJoin = testData2.filter('a < 2) // TestData2(1, 1) :: TestData2(1, 2) - testDataForJoin.registerTempTable("testDataForJoin") - withTempTable("testDataForJoin") { - // Assume the execution plan is - // ... -> CartesianProduct(nodeId = 1) -> TungstenProject(nodeId = 0) - val df = sqlContext.sql( - "SELECT * FROM testData2 JOIN testDataForJoin") + test("SortMergeJoin(left-anti) metrics") { + val anti = testData2.filter("a > 2") + withTempView("antiData") { + anti.createOrReplaceTempView("antiData") + val df = spark.sql( + "SELECT * FROM testData2 ANTI JOIN antiData ON testData2.a = antiData.a") testSparkPlanMetrics(df, 1, Map( - 1L -> ("CartesianProduct", Map( - "number of left rows" -> 12L, // left needs to be scanned twice - "number of right rows" -> 12L, // right is read 6 times - "number of output rows" -> 12L))) + 0L -> (("SortMergeJoin", Map("number of output rows" -> 4L)))) ) } } test("save metrics") { withTempPath { file => - val previousExecutionIds = sqlContext.listener.executionIdToData.keySet + // person creates a temporary view. get the DF before listing previous execution IDs + val data = person.select('name) + sparkContext.listenerBus.waitUntilEmpty(10000) + val previousExecutionIds = spark.sharedState.listener.executionIdToData.keySet // Assume the execution plan is // PhysicalRDD(nodeId = 0) - person.select('name).write.format("json").save(file.getAbsolutePath) + data.write.format("json").save(file.getAbsolutePath) sparkContext.listenerBus.waitUntilEmpty(10000) - val executionIds = sqlContext.listener.executionIdToData.keySet.diff(previousExecutionIds) + val executionIds = + spark.sharedState.listener.executionIdToData.keySet.diff(previousExecutionIds) assert(executionIds.size === 1) val executionId = executionIds.head - val jobs = sqlContext.listener.getExecution(executionId).get.jobs + val jobs = spark.sharedState.listener.getExecution(executionId).get.jobs // Use "<=" because there is a race condition that we may miss some jobs // TODO Change "<=" to "=" once we fix the race condition that missing the JobStarted event. assert(jobs.size <= 1) - val metricValues = sqlContext.listener.getExecutionMetrics(executionId) + val metricValues = spark.sharedState.listener.getExecutionMetrics(executionId) // Because "save" will create a new DataFrame internally, we cannot get the real metric id. // However, we still can check the value. - assert(metricValues.values.toSeq === Seq("2")) + assert(metricValues.values.toSeq.exists(_ === "2")) } } -} + test("metrics can be loaded by history server") { + val metric = SQLMetrics.createMetric(sparkContext, "zanzibar") + metric += 10L + val metricInfo = metric.toInfo(Some(metric.value), None) + metricInfo.update match { + case Some(v: Long) => assert(v === 10L) + case Some(v) => fail(s"metric value was not a Long: ${v.getClass.getName}") + case _ => fail("metric update is missing") + } + assert(metricInfo.metadata === Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) + // After serializing to JSON, the original value type is lost, but we can still + // identify that it's a SQL metric from the metadata + val metricInfoJson = JsonProtocol.accumulableInfoToJson(metricInfo) + val metricInfoDeser = JsonProtocol.accumulableInfoFromJson(metricInfoJson) + metricInfoDeser.update match { + case Some(v: String) => assert(v.toLong === 10L) + case Some(v) => fail(s"deserialized metric value was not a string: ${v.getClass.getName}") + case _ => fail("deserialized metric update is missing") + } + assert(metricInfoDeser.metadata === Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) + } -private case class MethodIdentifier[T](cls: Class[T], name: String, desc: String) + test("range metrics") { + val res1 = InputOutputMetricsHelper.run( + spark.range(30).filter(x => x % 3 == 0).toDF() + ) + assert(res1 === (30L, 0L, 30L) :: Nil) -/** - * If `method` is null, search all methods of this class recursively to find if they do some boxing. - * If `method` is specified, only search this method of the class to speed up the searching. - * - * This method will skip the methods in `visitedMethods` to avoid potential infinite cycles. - */ -private class BoxingFinder( - method: MethodIdentifier[_] = null, - val boxingInvokes: mutable.Set[String] = mutable.Set.empty, - visitedMethods: mutable.Set[MethodIdentifier[_]] = mutable.Set.empty) - extends ClassVisitor(ASM4) { - - private val primitiveBoxingClassName = - Set("java/lang/Long", - "java/lang/Double", - "java/lang/Integer", - "java/lang/Float", - "java/lang/Short", - "java/lang/Character", - "java/lang/Byte", - "java/lang/Boolean") - - override def visitMethod( - access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): - MethodVisitor = { - if (method != null && (method.name != name || method.desc != desc)) { - // If method is specified, skip other methods. - return new MethodVisitor(ASM4) {} - } + val res2 = InputOutputMetricsHelper.run( + spark.range(150).repartition(4).filter(x => x < 10).toDF() + ) + assert(res2 === (150L, 0L, 150L) :: (0L, 150L, 10L) :: Nil) - new MethodVisitor(ASM4) { - override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { - if (op == INVOKESPECIAL && name == "" || op == INVOKESTATIC && name == "valueOf") { - if (primitiveBoxingClassName.contains(owner)) { - // Find boxing methods, e.g, new java.lang.Long(l) or java.lang.Long.valueOf(l) - boxingInvokes.add(s"$owner.$name") - } - } else { - // scalastyle:off classforname - val classOfMethodOwner = Class.forName(owner.replace('/', '.'), false, - Thread.currentThread.getContextClassLoader) - // scalastyle:on classforname - val m = MethodIdentifier(classOfMethodOwner, name, desc) - if (!visitedMethods.contains(m)) { - // Keep track of visited methods to avoid potential infinite cycles - visitedMethods += m - BoxingFinder.getClassReader(classOfMethodOwner).foreach { cl => - visitedMethods += m - cl.accept(new BoxingFinder(m, boxingInvokes, visitedMethods), 0) - } - } - } - } + withTempDir { tempDir => + val dir = new File(tempDir, "pqS").getCanonicalPath + + spark.range(10).write.parquet(dir) + spark.read.parquet(dir).createOrReplaceTempView("pqS") + + val res3 = InputOutputMetricsHelper.run( + spark.range(30).repartition(3).crossJoin(sql("select * from pqS")).repartition(2).toDF() + ) + // The query above is executed in the following stages: + // 1. sql("select * from pqS") => (10, 0, 10) + // 2. range(30) => (30, 0, 30) + // 3. crossJoin(...) of 1. and 2. => (0, 30, 300) + // 4. shuffle & return results => (0, 300, 0) + assert(res3 === (10L, 0L, 10L) :: (30L, 0L, 30L) :: (0L, 30L, 300L) :: (0L, 300L, 0L) :: Nil) } } -} -private object BoxingFinder { - - def getClassReader(cls: Class[_]): Option[ClassReader] = { - val className = cls.getName.replaceFirst("^.*\\.", "") + ".class" - val resourceStream = cls.getResourceAsStream(className) - val baos = new ByteArrayOutputStream(128) - // Copy data over, before delegating to ClassReader - - // else we can run out of open file handles. - Utils.copyStream(resourceStream, baos, true) - // ASM4 doesn't support Java 8 classes, which requires ASM5. - // So if the class is ASM5 (E.g., java.lang.Long when using JDK8 runtime to run these codes), - // then ClassReader will throw IllegalArgumentException, - // However, since this is only for testing, it's safe to skip these classes. - try { - Some(new ClassReader(new ByteArrayInputStream(baos.toByteArray))) - } catch { - case _: IllegalArgumentException => None - } + test("writing data out metrics: parquet") { + testMetricsNonDynamicPartition("parquet", "t1") } + test("writing data out metrics with dynamic partition: parquet") { + testMetricsDynamicPartition("parquet", "parquet", "t1") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala new file mode 100644 index 000000000000..3966e98c1ce0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala @@ -0,0 +1,270 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.metric + +import java.io.File + +import scala.collection.mutable.HashMap + +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.SparkPlanInfo +import org.apache.spark.sql.execution.ui.SparkPlanGraph +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.util.Utils + + +trait SQLMetricsTestUtils extends SQLTestUtils { + + import testImplicits._ + + /** + * Get execution metrics for the SQL execution and verify metrics values. + * + * @param metricsValues the expected metric values (numFiles, numPartitions, numOutputRows). + * @param func the function can produce execution id after running. + */ + private def verifyWriteDataMetrics(metricsValues: Seq[Int])(func: => Unit): Unit = { + val previousExecutionIds = spark.sharedState.listener.executionIdToData.keySet + // Run the given function to trigger query execution. + func + spark.sparkContext.listenerBus.waitUntilEmpty(10000) + val executionIds = + spark.sharedState.listener.executionIdToData.keySet.diff(previousExecutionIds) + assert(executionIds.size == 1) + val executionId = executionIds.head + + val executionData = spark.sharedState.listener.getExecution(executionId).get + val executedNode = executionData.physicalPlanGraph.nodes.head + + val metricsNames = Seq( + "number of written files", + "number of dynamic part", + "number of output rows") + + val metrics = spark.sharedState.listener.getExecutionMetrics(executionId) + + metricsNames.zip(metricsValues).foreach { case (metricsName, expected) => + val sqlMetric = executedNode.metrics.find(_.name == metricsName) + assert(sqlMetric.isDefined) + val accumulatorId = sqlMetric.get.accumulatorId + val metricValue = metrics(accumulatorId).replaceAll(",", "").toInt + assert(metricValue == expected) + } + + val totalNumBytesMetric = executedNode.metrics.find(_.name == "bytes of written output").get + val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "").toInt + assert(totalNumBytes > 0) + } + + protected def testMetricsNonDynamicPartition( + dataFormat: String, + tableName: String): Unit = { + withTable(tableName) { + Seq((1, 2)).toDF("i", "j") + .write.format(dataFormat).mode("overwrite").saveAsTable(tableName) + + val tableLocation = + new File(spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).location) + + // 2 files, 100 rows, 0 dynamic partition. + verifyWriteDataMetrics(Seq(2, 0, 100)) { + (0 until 100).map(i => (i, i + 1)).toDF("i", "j").repartition(2) + .write.format(dataFormat).mode("overwrite").insertInto(tableName) + } + assert(Utils.recursiveList(tableLocation).count(_.getName.startsWith("part-")) == 2) + } + } + + protected def testMetricsDynamicPartition( + provider: String, + dataFormat: String, + tableName: String): Unit = { + withTempPath { dir => + spark.sql( + s""" + |CREATE TABLE $tableName(a int, b int) + |USING $provider + |PARTITIONED BY(a) + |LOCATION '${dir.toURI}' + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + val df = spark.range(start = 0, end = 40, step = 1, numPartitions = 1) + .selectExpr("id a", "id b") + + // 40 files, 80 rows, 40 dynamic partitions. + verifyWriteDataMetrics(Seq(40, 40, 80)) { + df.union(df).repartition(2, $"a") + .write + .format(dataFormat) + .mode("overwrite") + .insertInto(tableName) + } + assert(Utils.recursiveList(dir).count(_.getName.startsWith("part-")) == 40) + } + } + + /** + * Call `df.collect()` and collect necessary metrics from execution data. + * + * @param df `DataFrame` to run + * @param expectedNumOfJobs number of jobs that will run + * @param expectedNodeIds the node ids of the metrics to collect from execution data. + */ + protected def getSparkPlanMetrics( + df: DataFrame, + expectedNumOfJobs: Int, + expectedNodeIds: Set[Long], + enableWholeStage: Boolean = false): Option[Map[Long, (String, Map[String, Any])]] = { + val previousExecutionIds = spark.sharedState.listener.executionIdToData.keySet + withSQLConf("spark.sql.codegen.wholeStage" -> enableWholeStage.toString) { + df.collect() + } + sparkContext.listenerBus.waitUntilEmpty(10000) + val executionIds = + spark.sharedState.listener.executionIdToData.keySet.diff(previousExecutionIds) + assert(executionIds.size === 1) + val executionId = executionIds.head + val jobs = spark.sharedState.listener.getExecution(executionId).get.jobs + // Use "<=" because there is a race condition that we may miss some jobs + // TODO Change it to "=" once we fix the race condition that missing the JobStarted event. + assert(jobs.size <= expectedNumOfJobs) + if (jobs.size == expectedNumOfJobs) { + // If we can track all jobs, check the metric values + val metricValues = spark.sharedState.listener.getExecutionMetrics(executionId) + val metrics = SparkPlanGraph(SparkPlanInfo.fromSparkPlan( + df.queryExecution.executedPlan)).allNodes.filter { node => + expectedNodeIds.contains(node.id) + }.map { node => + val nodeMetrics = node.metrics.map { metric => + val metricValue = metricValues(metric.accumulatorId) + (metric.name, metricValue) + }.toMap + (node.id, node.name -> nodeMetrics) + }.toMap + Some(metrics) + } else { + // TODO Remove this "else" once we fix the race condition that missing the JobStarted event. + // Since we cannot track all jobs, the metric values could be wrong and we should not check + // them. + logWarning("Due to a race condition, we miss some jobs and cannot verify the metric values") + None + } + } + + /** + * Call `df.collect()` and verify if the collected metrics are same as "expectedMetrics". + * + * @param df `DataFrame` to run + * @param expectedNumOfJobs number of jobs that will run + * @param expectedMetrics the expected metrics. The format is + * `nodeId -> (operatorName, metric name -> metric value)`. + */ + protected def testSparkPlanMetrics( + df: DataFrame, + expectedNumOfJobs: Int, + expectedMetrics: Map[Long, (String, Map[String, Any])]): Unit = { + val optActualMetrics = getSparkPlanMetrics(df, expectedNumOfJobs, expectedMetrics.keySet) + optActualMetrics.foreach { actualMetrics => + assert(expectedMetrics.keySet === actualMetrics.keySet) + for (nodeId <- expectedMetrics.keySet) { + val (expectedNodeName, expectedMetricsMap) = expectedMetrics(nodeId) + val (actualNodeName, actualMetricsMap) = actualMetrics(nodeId) + assert(expectedNodeName === actualNodeName) + for (metricName <- expectedMetricsMap.keySet) { + assert(expectedMetricsMap(metricName).toString === actualMetricsMap(metricName)) + } + } + } + } +} + + +object InputOutputMetricsHelper { + private class InputOutputMetricsListener extends SparkListener { + private case class MetricsResult( + var recordsRead: Long = 0L, + var shuffleRecordsRead: Long = 0L, + var sumMaxOutputRows: Long = 0L) + + private[this] val stageIdToMetricsResult = HashMap.empty[Int, MetricsResult] + + def reset(): Unit = { + stageIdToMetricsResult.clear() + } + + /** + * Return a list of recorded metrics aggregated per stage. + * + * The list is sorted in the ascending order on the stageId. + * For each recorded stage, the following tuple is returned: + * - sum of inputMetrics.recordsRead for all the tasks in the stage + * - sum of shuffleReadMetrics.recordsRead for all the tasks in the stage + * - sum of the highest values of "number of output rows" metric for all the tasks in the stage + */ + def getResults(): List[(Long, Long, Long)] = { + stageIdToMetricsResult.keySet.toList.sorted.map { stageId => + val res = stageIdToMetricsResult(stageId) + (res.recordsRead, res.shuffleRecordsRead, res.sumMaxOutputRows) + } + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { + val res = stageIdToMetricsResult.getOrElseUpdate(taskEnd.stageId, MetricsResult()) + + res.recordsRead += taskEnd.taskMetrics.inputMetrics.recordsRead + res.shuffleRecordsRead += taskEnd.taskMetrics.shuffleReadMetrics.recordsRead + + var maxOutputRows = 0L + for (accum <- taskEnd.taskMetrics.externalAccums) { + val info = accum.toInfo(Some(accum.value), None) + if (info.name.toString.contains("number of output rows")) { + info.update match { + case Some(n: Number) => + if (n.longValue() > maxOutputRows) { + maxOutputRows = n.longValue() + } + case _ => // Ignore. + } + } + } + res.sumMaxOutputRows += maxOutputRows + } + } + + // Run df.collect() and return aggregated metrics for each stage. + def run(df: DataFrame): List[(Long, Long, Long)] = { + val spark = df.sparkSession + val sparkContext = spark.sparkContext + val listener = new InputOutputMetricsListener() + sparkContext.addSparkListener(listener) + + try { + sparkContext.listenerBus.waitUntilEmpty(5000) + listener.reset() + df.collect() + sparkContext.listenerBus.waitUntilEmpty(5000) + } finally { + sparkContext.removeSparkListener(listener) + } + listener.getResults() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala new file mode 100644 index 000000000000..153e6e1f88c7 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.python + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.api.python.PythonFunction +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, GreaterThan, In} +import org.apache.spark.sql.execution.{FilterExec, InputAdapter, SparkPlanTest, WholeStageCodegenExec} +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.BooleanType + +class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSQLContext { + import testImplicits.newProductEncoder + import testImplicits.localSeqToDatasetHolder + + override def beforeAll(): Unit = { + super.beforeAll() + spark.udf.registerPython("dummyPythonUDF", new MyDummyPythonUDF) + } + + override def afterAll(): Unit = { + spark.sessionState.functionRegistry.dropFunction(FunctionIdentifier("dummyPythonUDF")) + super.afterAll() + } + + test("Python UDF: push down deterministic FilterExec predicates") { + val df = Seq(("Hello", 4)).toDF("a", "b") + .where("dummyPythonUDF(b) and dummyPythonUDF(a) and a in (3, 4)") + val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { + case f @ FilterExec( + And(_: AttributeReference, _: AttributeReference), + InputAdapter(_: BatchEvalPythonExec)) => f + case b @ BatchEvalPythonExec(_, _, WholeStageCodegenExec(FilterExec(_: In, _))) => b + } + assert(qualifiedPlanNodes.size == 2) + } + + test("Nested Python UDF: push down deterministic FilterExec predicates") { + val df = Seq(("Hello", 4)).toDF("a", "b") + .where("dummyPythonUDF(a, dummyPythonUDF(a, b)) and a in (3, 4)") + val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { + case f @ FilterExec(_: AttributeReference, InputAdapter(_: BatchEvalPythonExec)) => f + case b @ BatchEvalPythonExec(_, _, WholeStageCodegenExec(FilterExec(_: In, _))) => b + } + assert(qualifiedPlanNodes.size == 2) + } + + test("Python UDF: no push down on non-deterministic") { + val df = Seq(("Hello", 4)).toDF("a", "b") + .where("b > 4 and dummyPythonUDF(a) and rand() > 0.3") + val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { + case f @ FilterExec( + And(_: AttributeReference, _: GreaterThan), + InputAdapter(_: BatchEvalPythonExec)) => f + case b @ BatchEvalPythonExec(_, _, WholeStageCodegenExec(_: FilterExec)) => b + } + assert(qualifiedPlanNodes.size == 2) + } + + test("Python UDF: no push down on predicates starting from the first non-deterministic") { + val df = Seq(("Hello", 4)).toDF("a", "b") + .where("dummyPythonUDF(a) and rand() > 0.3 and b > 4") + val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { + case f @ FilterExec(And(_: And, _: GreaterThan), InputAdapter(_: BatchEvalPythonExec)) => f + } + assert(qualifiedPlanNodes.size == 1) + } + + test("Python UDF refers to the attributes from more than one child") { + val df = Seq(("Hello", 4)).toDF("a", "b") + val df2 = Seq(("Hello", 4)).toDF("c", "d") + val joinDF = df.crossJoin(df2).where("dummyPythonUDF(a, c) == dummyPythonUDF(d, c)") + val qualifiedPlanNodes = joinDF.queryExecution.executedPlan.collect { + case b: BatchEvalPythonExec => b + } + assert(qualifiedPlanNodes.size == 1) + } +} + +// This Python UDF is dummy and just for testing. Unable to execute. +class DummyUDF extends PythonFunction( + command = Array[Byte](), + envVars = Map("" -> "").asJava, + pythonIncludes = ArrayBuffer("").asJava, + pythonExec = "", + pythonVer = "", + broadcastVars = null, + accumulator = null) + +class MyDummyPythonUDF extends UserDefinedPythonFunction( + name = "dummyUDF", + func = new DummyUDF, + dataType = BooleanType, + vectorized = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala new file mode 100644 index 000000000000..ffda33cf906c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.python + +import java.io.File + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.memory.{MemoryManager, TaskMemoryManager, TestMemoryManager} +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.unsafe.memory.MemoryBlock +import org.apache.spark.util.Utils + +class RowQueueSuite extends SparkFunSuite { + + test("in-memory queue") { + val page = MemoryBlock.fromLongArray(new Array[Long](1<<10)) + val queue = new InMemoryRowQueue(page, 1) { + override def close() {} + } + val row = new UnsafeRow(1) + row.pointTo(new Array[Byte](16), 16) + val n = page.size() / (4 + row.getSizeInBytes) + var i = 0 + while (i < n) { + row.setLong(0, i) + assert(queue.add(row), "fail to add") + i += 1 + } + assert(!queue.add(row), "should not add more") + i = 0 + while (i < n) { + val row = queue.remove() + assert(row != null, "fail to poll") + assert(row.getLong(0) == i, "does not match") + i += 1 + } + assert(queue.remove() == null, "should be empty") + queue.close() + } + + test("disk queue") { + val dir = Utils.createTempDir().getCanonicalFile + dir.mkdirs() + val queue = DiskRowQueue(new File(dir, "buffer"), 1) + val row = new UnsafeRow(1) + row.pointTo(new Array[Byte](16), 16) + val n = 1000 + var i = 0 + while (i < n) { + row.setLong(0, i) + assert(queue.add(row), "fail to add") + i += 1 + } + val first = queue.remove() + assert(first != null, "first should not be null") + assert(first.getLong(0) == 0, "first should be 0") + assert(!queue.add(row), "should not add more") + i = 1 + while (i < n) { + val row = queue.remove() + assert(row != null, "fail to poll") + assert(row.getLong(0) == i, "does not match") + i += 1 + } + assert(queue.remove() == null, "should be empty") + queue.close() + } + + test("hybrid queue") { + val mem = new TestMemoryManager(new SparkConf()) + mem.limit(4<<10) + val taskM = new TaskMemoryManager(mem, 0) + val queue = HybridRowQueue(taskM, Utils.createTempDir().getCanonicalFile, 1) + val row = new UnsafeRow(1) + row.pointTo(new Array[Byte](16), 16) + val n = (4<<10) / 16 * 3 + var i = 0 + while (i < n) { + row.setLong(0, i) + assert(queue.add(row), "fail to add") + i += 1 + } + assert(queue.numQueues() > 1, "should have more than one queue") + queue.spill(1<<20, null) + i = 0 + while (i < n) { + val row = queue.remove() + assert(row != null, "fail to poll") + assert(row.getLong(0) == i, "does not match") + i += 1 + } + + // fill again and spill + i = 0 + while (i < n) { + row.setLong(0, i) + assert(queue.add(row), "fail to add") + i += 1 + } + assert(queue.numQueues() > 1, "should have more than one queue") + queue.spill(1<<20, null) + assert(queue.numQueues() > 1, "should have more than one queue") + i = 0 + while (i < n) { + val row = queue.remove() + assert(row != null, "fail to poll") + assert(row.getLong(0) == i, "does not match") + i += 1 + } + queue.close() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala new file mode 100644 index 000000000000..83018f95aa55 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala @@ -0,0 +1,285 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.io._ +import java.nio.charset.StandardCharsets._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.streaming.FakeFileSystem._ +import org.apache.spark.sql.test.SharedSQLContext + +class CompactibleFileStreamLogSuite extends SparkFunSuite with SharedSQLContext { + + /** To avoid caching of FS objects */ + override protected def sparkConf = + super.sparkConf.set(s"spark.hadoop.fs.$scheme.impl.disable.cache", "true") + + import CompactibleFileStreamLog._ + + /** -- testing of `object CompactibleFileStreamLog` begins -- */ + + test("getBatchIdFromFileName") { + assert(1234L === getBatchIdFromFileName("1234")) + assert(1234L === getBatchIdFromFileName("1234.compact")) + intercept[NumberFormatException] { + getBatchIdFromFileName("1234a") + } + } + + test("isCompactionBatch") { + assert(false === isCompactionBatch(0, compactInterval = 3)) + assert(false === isCompactionBatch(1, compactInterval = 3)) + assert(true === isCompactionBatch(2, compactInterval = 3)) + assert(false === isCompactionBatch(3, compactInterval = 3)) + assert(false === isCompactionBatch(4, compactInterval = 3)) + assert(true === isCompactionBatch(5, compactInterval = 3)) + } + + test("nextCompactionBatchId") { + assert(2 === nextCompactionBatchId(0, compactInterval = 3)) + assert(2 === nextCompactionBatchId(1, compactInterval = 3)) + assert(5 === nextCompactionBatchId(2, compactInterval = 3)) + assert(5 === nextCompactionBatchId(3, compactInterval = 3)) + assert(5 === nextCompactionBatchId(4, compactInterval = 3)) + assert(8 === nextCompactionBatchId(5, compactInterval = 3)) + } + + test("getValidBatchesBeforeCompactionBatch") { + intercept[AssertionError] { + getValidBatchesBeforeCompactionBatch(0, compactInterval = 3) + } + intercept[AssertionError] { + getValidBatchesBeforeCompactionBatch(1, compactInterval = 3) + } + assert(Seq(0, 1) === getValidBatchesBeforeCompactionBatch(2, compactInterval = 3)) + intercept[AssertionError] { + getValidBatchesBeforeCompactionBatch(3, compactInterval = 3) + } + intercept[AssertionError] { + getValidBatchesBeforeCompactionBatch(4, compactInterval = 3) + } + assert(Seq(2, 3, 4) === getValidBatchesBeforeCompactionBatch(5, compactInterval = 3)) + } + + test("getAllValidBatches") { + assert(Seq(0) === getAllValidBatches(0, compactInterval = 3)) + assert(Seq(0, 1) === getAllValidBatches(1, compactInterval = 3)) + assert(Seq(2) === getAllValidBatches(2, compactInterval = 3)) + assert(Seq(2, 3) === getAllValidBatches(3, compactInterval = 3)) + assert(Seq(2, 3, 4) === getAllValidBatches(4, compactInterval = 3)) + assert(Seq(5) === getAllValidBatches(5, compactInterval = 3)) + assert(Seq(5, 6) === getAllValidBatches(6, compactInterval = 3)) + assert(Seq(5, 6, 7) === getAllValidBatches(7, compactInterval = 3)) + assert(Seq(8) === getAllValidBatches(8, compactInterval = 3)) + } + + test("deriveCompactInterval") { + // latestCompactBatchId(4) + 1 <= default(5) + // then use latestestCompactBatchId + 1 === 5 + assert(5 === deriveCompactInterval(5, 4)) + // First divisor of 10 greater than 4 === 5 + assert(5 === deriveCompactInterval(4, 9)) + } + + /** -- testing of `object CompactibleFileStreamLog` ends -- */ + + test("batchIdToPath") { + withFakeCompactibleFileStreamLog( + fileCleanupDelayMs = Long.MaxValue, + defaultCompactInterval = 3, + defaultMinBatchesToRetain = 1, + compactibleLog => { + assert("0" === compactibleLog.batchIdToPath(0).getName) + assert("1" === compactibleLog.batchIdToPath(1).getName) + assert("2.compact" === compactibleLog.batchIdToPath(2).getName) + assert("3" === compactibleLog.batchIdToPath(3).getName) + assert("4" === compactibleLog.batchIdToPath(4).getName) + assert("5.compact" === compactibleLog.batchIdToPath(5).getName) + }) + } + + test("serialize") { + withFakeCompactibleFileStreamLog( + fileCleanupDelayMs = Long.MaxValue, + defaultCompactInterval = 3, + defaultMinBatchesToRetain = 1, + compactibleLog => { + val logs = Array("entry_1", "entry_2", "entry_3") + val expected = s"""v${FakeCompactibleFileStreamLog.VERSION} + |"entry_1" + |"entry_2" + |"entry_3"""".stripMargin + val baos = new ByteArrayOutputStream() + compactibleLog.serialize(logs, baos) + assert(expected === baos.toString(UTF_8.name())) + + baos.reset() + compactibleLog.serialize(Array(), baos) + assert(s"v${FakeCompactibleFileStreamLog.VERSION}" === baos.toString(UTF_8.name())) + }) + } + + test("deserialize") { + withFakeCompactibleFileStreamLog( + fileCleanupDelayMs = Long.MaxValue, + defaultCompactInterval = 3, + defaultMinBatchesToRetain = 1, + compactibleLog => { + val logs = s"""v${FakeCompactibleFileStreamLog.VERSION} + |"entry_1" + |"entry_2" + |"entry_3"""".stripMargin + val expected = Array("entry_1", "entry_2", "entry_3") + assert(expected === + compactibleLog.deserialize(new ByteArrayInputStream(logs.getBytes(UTF_8)))) + + assert(Nil === + compactibleLog.deserialize( + new ByteArrayInputStream(s"v${FakeCompactibleFileStreamLog.VERSION}".getBytes(UTF_8)))) + }) + } + + test("deserialization log written by future version") { + withTempDir { dir => + def newFakeCompactibleFileStreamLog(version: Int): FakeCompactibleFileStreamLog = + new FakeCompactibleFileStreamLog( + version, + _fileCleanupDelayMs = Long.MaxValue, // this param does not matter here in this test case + _defaultCompactInterval = 3, // this param does not matter here in this test case + _defaultMinBatchesToRetain = 1, // this param does not matter here in this test case + spark, + dir.getCanonicalPath) + + val writer = newFakeCompactibleFileStreamLog(version = 2) + val reader = newFakeCompactibleFileStreamLog(version = 1) + writer.add(0, Array("entry")) + val e = intercept[IllegalStateException] { + reader.get(0) + } + Seq( + "maximum supported log version is v1, but encountered v2", + "produced by a newer version of Spark and cannot be read by this version" + ).foreach { message => + assert(e.getMessage.contains(message)) + } + } + } + + test("compact") { + withFakeCompactibleFileStreamLog( + fileCleanupDelayMs = Long.MaxValue, + defaultCompactInterval = 3, + defaultMinBatchesToRetain = 1, + compactibleLog => { + for (batchId <- 0 to 10) { + compactibleLog.add(batchId, Array("some_path_" + batchId)) + val expectedFiles = (0 to batchId).map { id => "some_path_" + id } + assert(compactibleLog.allFiles() === expectedFiles) + if (isCompactionBatch(batchId, 3)) { + // Since batchId is a compaction batch, the batch log file should contain all logs + assert(compactibleLog.get(batchId).getOrElse(Nil) === expectedFiles) + } + } + }) + } + + test("delete expired file") { + // Set `fileCleanupDelayMs` to 0 so that we can detect the deleting behaviour deterministically + withFakeCompactibleFileStreamLog( + fileCleanupDelayMs = 0, + defaultCompactInterval = 3, + defaultMinBatchesToRetain = 1, + compactibleLog => { + val fs = compactibleLog.metadataPath.getFileSystem(spark.sessionState.newHadoopConf()) + + def listBatchFiles(): Set[String] = { + fs.listStatus(compactibleLog.metadataPath).map(_.getPath.getName).filter { fileName => + try { + getBatchIdFromFileName(fileName) + true + } catch { + case _: NumberFormatException => false + } + }.toSet + } + + compactibleLog.add(0, Array("some_path_0")) + assert(Set("0") === listBatchFiles()) + compactibleLog.add(1, Array("some_path_1")) + assert(Set("0", "1") === listBatchFiles()) + compactibleLog.add(2, Array("some_path_2")) + assert(Set("0", "1", "2.compact") === listBatchFiles()) + compactibleLog.add(3, Array("some_path_3")) + assert(Set("2.compact", "3") === listBatchFiles()) + compactibleLog.add(4, Array("some_path_4")) + assert(Set("2.compact", "3", "4") === listBatchFiles()) + compactibleLog.add(5, Array("some_path_5")) + assert(Set("2.compact", "3", "4", "5.compact") === listBatchFiles()) + compactibleLog.add(6, Array("some_path_6")) + assert(Set("5.compact", "6") === listBatchFiles()) + }) + } + + private def withFakeCompactibleFileStreamLog( + fileCleanupDelayMs: Long, + defaultCompactInterval: Int, + defaultMinBatchesToRetain: Int, + f: FakeCompactibleFileStreamLog => Unit + ): Unit = { + withTempDir { file => + val compactibleLog = new FakeCompactibleFileStreamLog( + FakeCompactibleFileStreamLog.VERSION, + fileCleanupDelayMs, + defaultCompactInterval, + defaultMinBatchesToRetain, + spark, + file.getCanonicalPath) + f(compactibleLog) + } + } +} + +object FakeCompactibleFileStreamLog { + val VERSION = 1 +} + +class FakeCompactibleFileStreamLog( + metadataLogVersion: Int, + _fileCleanupDelayMs: Long, + _defaultCompactInterval: Int, + _defaultMinBatchesToRetain: Int, + sparkSession: SparkSession, + path: String) + extends CompactibleFileStreamLog[String]( + metadataLogVersion, + sparkSession, + path + ) { + + override protected def fileCleanupDelayMs: Long = _fileCleanupDelayMs + + override protected def isDeletingExpiredLog: Boolean = true + + override protected def defaultCompactInterval: Int = _defaultCompactInterval + + override protected val minBatchesToRetain: Int = _defaultMinBatchesToRetain + + override def compactLogs(logs: Seq[String]): Seq[String] = logs +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala new file mode 100644 index 000000000000..dd3a414659c2 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala @@ -0,0 +1,270 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} +import java.nio.charset.StandardCharsets.UTF_8 + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class FileStreamSinkLogSuite extends SparkFunSuite with SharedSQLContext { + + import CompactibleFileStreamLog._ + import FileStreamSinkLog._ + + test("compactLogs") { + withFileStreamSinkLog { sinkLog => + val logs = Seq( + newFakeSinkFileStatus("/a/b/x", FileStreamSinkLog.ADD_ACTION), + newFakeSinkFileStatus("/a/b/y", FileStreamSinkLog.ADD_ACTION), + newFakeSinkFileStatus("/a/b/z", FileStreamSinkLog.ADD_ACTION)) + assert(logs === sinkLog.compactLogs(logs)) + + val logs2 = Seq( + newFakeSinkFileStatus("/a/b/m", FileStreamSinkLog.ADD_ACTION), + newFakeSinkFileStatus("/a/b/n", FileStreamSinkLog.ADD_ACTION), + newFakeSinkFileStatus("/a/b/z", FileStreamSinkLog.DELETE_ACTION)) + assert(logs.dropRight(1) ++ logs2.dropRight(1) === sinkLog.compactLogs(logs ++ logs2)) + } + } + + test("serialize") { + withFileStreamSinkLog { sinkLog => + val logs = Array( + SinkFileStatus( + path = "/a/b/x", + size = 100L, + isDir = false, + modificationTime = 1000L, + blockReplication = 1, + blockSize = 10000L, + action = FileStreamSinkLog.ADD_ACTION), + SinkFileStatus( + path = "/a/b/y", + size = 200L, + isDir = false, + modificationTime = 2000L, + blockReplication = 2, + blockSize = 20000L, + action = FileStreamSinkLog.DELETE_ACTION), + SinkFileStatus( + path = "/a/b/z", + size = 300L, + isDir = false, + modificationTime = 3000L, + blockReplication = 3, + blockSize = 30000L, + action = FileStreamSinkLog.ADD_ACTION)) + + // scalastyle:off + val expected = s"""v$VERSION + |{"path":"/a/b/x","size":100,"isDir":false,"modificationTime":1000,"blockReplication":1,"blockSize":10000,"action":"add"} + |{"path":"/a/b/y","size":200,"isDir":false,"modificationTime":2000,"blockReplication":2,"blockSize":20000,"action":"delete"} + |{"path":"/a/b/z","size":300,"isDir":false,"modificationTime":3000,"blockReplication":3,"blockSize":30000,"action":"add"}""".stripMargin + // scalastyle:on + val baos = new ByteArrayOutputStream() + sinkLog.serialize(logs, baos) + assert(expected === baos.toString(UTF_8.name())) + baos.reset() + sinkLog.serialize(Array(), baos) + assert(s"v$VERSION" === baos.toString(UTF_8.name())) + } + } + + test("deserialize") { + withFileStreamSinkLog { sinkLog => + // scalastyle:off + val logs = s"""v$VERSION + |{"path":"/a/b/x","size":100,"isDir":false,"modificationTime":1000,"blockReplication":1,"blockSize":10000,"action":"add"} + |{"path":"/a/b/y","size":200,"isDir":false,"modificationTime":2000,"blockReplication":2,"blockSize":20000,"action":"delete"} + |{"path":"/a/b/z","size":300,"isDir":false,"modificationTime":3000,"blockReplication":3,"blockSize":30000,"action":"add"}""".stripMargin + // scalastyle:on + + val expected = Seq( + SinkFileStatus( + path = "/a/b/x", + size = 100L, + isDir = false, + modificationTime = 1000L, + blockReplication = 1, + blockSize = 10000L, + action = FileStreamSinkLog.ADD_ACTION), + SinkFileStatus( + path = "/a/b/y", + size = 200L, + isDir = false, + modificationTime = 2000L, + blockReplication = 2, + blockSize = 20000L, + action = FileStreamSinkLog.DELETE_ACTION), + SinkFileStatus( + path = "/a/b/z", + size = 300L, + isDir = false, + modificationTime = 3000L, + blockReplication = 3, + blockSize = 30000L, + action = FileStreamSinkLog.ADD_ACTION)) + + assert(expected === sinkLog.deserialize(new ByteArrayInputStream(logs.getBytes(UTF_8)))) + + assert(Nil === sinkLog.deserialize(new ByteArrayInputStream(s"v$VERSION".getBytes(UTF_8)))) + } + } + + test("compact") { + withSQLConf(SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL.key -> "3") { + withFileStreamSinkLog { sinkLog => + for (batchId <- 0 to 10) { + sinkLog.add( + batchId, + Array(newFakeSinkFileStatus("/a/b/" + batchId, FileStreamSinkLog.ADD_ACTION))) + val expectedFiles = (0 to batchId).map { + id => newFakeSinkFileStatus("/a/b/" + id, FileStreamSinkLog.ADD_ACTION) + } + assert(sinkLog.allFiles() === expectedFiles) + if (isCompactionBatch(batchId, 3)) { + // Since batchId is a compaction batch, the batch log file should contain all logs + assert(sinkLog.get(batchId).getOrElse(Nil) === expectedFiles) + } + } + } + } + } + + test("delete expired file") { + // Set FILE_SINK_LOG_CLEANUP_DELAY to 0 so that we can detect the deleting behaviour + // deterministically and one min batches to retain + withSQLConf( + SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL.key -> "3", + SQLConf.FILE_SINK_LOG_CLEANUP_DELAY.key -> "0", + SQLConf.MIN_BATCHES_TO_RETAIN.key -> "1") { + withFileStreamSinkLog { sinkLog => + val fs = sinkLog.metadataPath.getFileSystem(spark.sessionState.newHadoopConf()) + + def listBatchFiles(): Set[String] = { + fs.listStatus(sinkLog.metadataPath).map(_.getPath.getName).filter { fileName => + try { + getBatchIdFromFileName(fileName) + true + } catch { + case _: NumberFormatException => false + } + }.toSet + } + + sinkLog.add(0, Array(newFakeSinkFileStatus("/a/b/0", FileStreamSinkLog.ADD_ACTION))) + assert(Set("0") === listBatchFiles()) + sinkLog.add(1, Array(newFakeSinkFileStatus("/a/b/1", FileStreamSinkLog.ADD_ACTION))) + assert(Set("0", "1") === listBatchFiles()) + sinkLog.add(2, Array(newFakeSinkFileStatus("/a/b/2", FileStreamSinkLog.ADD_ACTION))) + assert(Set("0", "1", "2.compact") === listBatchFiles()) + sinkLog.add(3, Array(newFakeSinkFileStatus("/a/b/3", FileStreamSinkLog.ADD_ACTION))) + assert(Set("2.compact", "3") === listBatchFiles()) + sinkLog.add(4, Array(newFakeSinkFileStatus("/a/b/4", FileStreamSinkLog.ADD_ACTION))) + assert(Set("2.compact", "3", "4") === listBatchFiles()) + sinkLog.add(5, Array(newFakeSinkFileStatus("/a/b/5", FileStreamSinkLog.ADD_ACTION))) + assert(Set("2.compact", "3", "4", "5.compact") === listBatchFiles()) + sinkLog.add(6, Array(newFakeSinkFileStatus("/a/b/6", FileStreamSinkLog.ADD_ACTION))) + assert(Set("5.compact", "6") === listBatchFiles()) + } + } + + withSQLConf( + SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL.key -> "3", + SQLConf.FILE_SINK_LOG_CLEANUP_DELAY.key -> "0", + SQLConf.MIN_BATCHES_TO_RETAIN.key -> "2") { + withFileStreamSinkLog { sinkLog => + val fs = sinkLog.metadataPath.getFileSystem(spark.sessionState.newHadoopConf()) + + def listBatchFiles(): Set[String] = { + fs.listStatus(sinkLog.metadataPath).map(_.getPath.getName).filter { fileName => + try { + getBatchIdFromFileName(fileName) + true + } catch { + case _: NumberFormatException => false + } + }.toSet + } + + sinkLog.add(0, Array(newFakeSinkFileStatus("/a/b/0", FileStreamSinkLog.ADD_ACTION))) + assert(Set("0") === listBatchFiles()) + sinkLog.add(1, Array(newFakeSinkFileStatus("/a/b/1", FileStreamSinkLog.ADD_ACTION))) + assert(Set("0", "1") === listBatchFiles()) + sinkLog.add(2, Array(newFakeSinkFileStatus("/a/b/2", FileStreamSinkLog.ADD_ACTION))) + assert(Set("0", "1", "2.compact") === listBatchFiles()) + sinkLog.add(3, Array(newFakeSinkFileStatus("/a/b/3", FileStreamSinkLog.ADD_ACTION))) + assert(Set("0", "1", "2.compact", "3") === listBatchFiles()) + sinkLog.add(4, Array(newFakeSinkFileStatus("/a/b/4", FileStreamSinkLog.ADD_ACTION))) + assert(Set("2.compact", "3", "4") === listBatchFiles()) + sinkLog.add(5, Array(newFakeSinkFileStatus("/a/b/5", FileStreamSinkLog.ADD_ACTION))) + assert(Set("2.compact", "3", "4", "5.compact") === listBatchFiles()) + sinkLog.add(6, Array(newFakeSinkFileStatus("/a/b/6", FileStreamSinkLog.ADD_ACTION))) + assert(Set("2.compact", "3", "4", "5.compact", "6") === listBatchFiles()) + sinkLog.add(7, Array(newFakeSinkFileStatus("/a/b/7", FileStreamSinkLog.ADD_ACTION))) + assert(Set("5.compact", "6", "7") === listBatchFiles()) + } + } + } + + test("read Spark 2.1.0 log format") { + assert(readFromResource("file-sink-log-version-2.1.0") === Seq( + // SinkFileStatus("/a/b/0", 100, false, 100, 1, 100, FileStreamSinkLog.ADD_ACTION), -> deleted + SinkFileStatus("/a/b/1", 100, false, 100, 1, 100, FileStreamSinkLog.ADD_ACTION), + SinkFileStatus("/a/b/2", 200, false, 200, 1, 100, FileStreamSinkLog.ADD_ACTION), + SinkFileStatus("/a/b/3", 300, false, 300, 1, 100, FileStreamSinkLog.ADD_ACTION), + SinkFileStatus("/a/b/4", 400, false, 400, 1, 100, FileStreamSinkLog.ADD_ACTION), + SinkFileStatus("/a/b/5", 500, false, 500, 1, 100, FileStreamSinkLog.ADD_ACTION), + SinkFileStatus("/a/b/6", 600, false, 600, 1, 100, FileStreamSinkLog.ADD_ACTION), + SinkFileStatus("/a/b/7", 700, false, 700, 1, 100, FileStreamSinkLog.ADD_ACTION), + SinkFileStatus("/a/b/8", 800, false, 800, 1, 100, FileStreamSinkLog.ADD_ACTION), + SinkFileStatus("/a/b/9", 900, false, 900, 3, 200, FileStreamSinkLog.ADD_ACTION) + )) + } + + /** + * Create a fake SinkFileStatus using path and action. Most of tests don't care about other fields + * in SinkFileStatus. + */ + private def newFakeSinkFileStatus(path: String, action: String): SinkFileStatus = { + SinkFileStatus( + path = path, + size = 100L, + isDir = false, + modificationTime = 100L, + blockReplication = 1, + blockSize = 100L, + action = action) + } + + private def withFileStreamSinkLog(f: FileStreamSinkLog => Unit): Unit = { + withTempDir { file => + val sinkLog = new FileStreamSinkLog(FileStreamSinkLog.VERSION, spark, file.getCanonicalPath) + f(sinkLog) + } + } + + private def readFromResource(dir: String): Seq[SinkFileStatus] = { + val input = getClass.getResource(s"/structured-streaming/$dir") + val log = new FileStreamSinkLog(FileStreamSinkLog.VERSION, spark, input.toString) + log.allFiles() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala new file mode 100644 index 000000000000..9137d650e906 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ForeachSinkSuite.scala @@ -0,0 +1,320 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.collection.mutable + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkException +import org.apache.spark.sql.ForeachWriter +import org.apache.spark.sql.functions.{count, window} +import org.apache.spark.sql.streaming.{OutputMode, StreamingQueryException, StreamTest} +import org.apache.spark.sql.test.SharedSQLContext + +class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAfter { + + import testImplicits._ + + after { + sqlContext.streams.active.foreach(_.stop()) + } + + test("foreach() with `append` output mode") { + withTempDir { checkpointDir => + val input = MemoryStream[Int] + val query = input.toDS().repartition(2).writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .outputMode(OutputMode.Append) + .foreach(new TestForeachWriter()) + .start() + + // -- batch 0 --------------------------------------- + input.addData(1, 2, 3, 4) + query.processAllAvailable() + + var expectedEventsForPartition0 = Seq( + ForeachSinkSuite.Open(partition = 0, version = 0), + ForeachSinkSuite.Process(value = 1), + ForeachSinkSuite.Process(value = 3), + ForeachSinkSuite.Close(None) + ) + var expectedEventsForPartition1 = Seq( + ForeachSinkSuite.Open(partition = 1, version = 0), + ForeachSinkSuite.Process(value = 2), + ForeachSinkSuite.Process(value = 4), + ForeachSinkSuite.Close(None) + ) + + var allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 2) + assert(allEvents.toSet === Set(expectedEventsForPartition0, expectedEventsForPartition1)) + + ForeachSinkSuite.clear() + + // -- batch 1 --------------------------------------- + input.addData(5, 6, 7, 8) + query.processAllAvailable() + + expectedEventsForPartition0 = Seq( + ForeachSinkSuite.Open(partition = 0, version = 1), + ForeachSinkSuite.Process(value = 5), + ForeachSinkSuite.Process(value = 7), + ForeachSinkSuite.Close(None) + ) + expectedEventsForPartition1 = Seq( + ForeachSinkSuite.Open(partition = 1, version = 1), + ForeachSinkSuite.Process(value = 6), + ForeachSinkSuite.Process(value = 8), + ForeachSinkSuite.Close(None) + ) + + allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 2) + assert(allEvents.toSet === Set(expectedEventsForPartition0, expectedEventsForPartition1)) + + query.stop() + } + } + + test("foreach() with `complete` output mode") { + withTempDir { checkpointDir => + val input = MemoryStream[Int] + + val query = input.toDS() + .groupBy().count().as[Long].map(_.toInt) + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .outputMode(OutputMode.Complete) + .foreach(new TestForeachWriter()) + .start() + + // -- batch 0 --------------------------------------- + input.addData(1, 2, 3, 4) + query.processAllAvailable() + + var allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 1) + var expectedEvents = Seq( + ForeachSinkSuite.Open(partition = 0, version = 0), + ForeachSinkSuite.Process(value = 4), + ForeachSinkSuite.Close(None) + ) + assert(allEvents === Seq(expectedEvents)) + + ForeachSinkSuite.clear() + + // -- batch 1 --------------------------------------- + input.addData(5, 6, 7, 8) + query.processAllAvailable() + + allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 1) + expectedEvents = Seq( + ForeachSinkSuite.Open(partition = 0, version = 1), + ForeachSinkSuite.Process(value = 8), + ForeachSinkSuite.Close(None) + ) + assert(allEvents === Seq(expectedEvents)) + + query.stop() + } + } + + testQuietly("foreach with error") { + withTempDir { checkpointDir => + val input = MemoryStream[Int] + val query = input.toDS().repartition(1).writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .foreach(new TestForeachWriter() { + override def process(value: Int): Unit = { + super.process(value) + throw new RuntimeException("error") + } + }).start() + input.addData(1, 2, 3, 4) + + // Error in `process` should fail the Spark job + val e = intercept[StreamingQueryException] { + query.processAllAvailable() + } + assert(e.getCause.isInstanceOf[SparkException]) + assert(e.getCause.getCause.getMessage === "error") + assert(query.isActive === false) + + val allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 1) + assert(allEvents(0)(0) === ForeachSinkSuite.Open(partition = 0, version = 0)) + assert(allEvents(0)(1) === ForeachSinkSuite.Process(value = 1)) + + // `close` should be called with the error + val errorEvent = allEvents(0)(2).asInstanceOf[ForeachSinkSuite.Close] + assert(errorEvent.error.get.isInstanceOf[RuntimeException]) + assert(errorEvent.error.get.getMessage === "error") + } + } + + test("foreach with watermark: complete") { + val inputData = MemoryStream[Int] + + val windowedAggregation = inputData.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"count".as[Long]) + .map(_.toInt) + .repartition(1) + + val query = windowedAggregation + .writeStream + .outputMode(OutputMode.Complete) + .foreach(new TestForeachWriter()) + .start() + try { + inputData.addData(10, 11, 12) + query.processAllAvailable() + + val allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 1) + val expectedEvents = Seq( + ForeachSinkSuite.Open(partition = 0, version = 0), + ForeachSinkSuite.Process(value = 3), + ForeachSinkSuite.Close(None) + ) + assert(allEvents === Seq(expectedEvents)) + } finally { + query.stop() + } + } + + test("foreach with watermark: append") { + val inputData = MemoryStream[Int] + + val windowedAggregation = inputData.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"count".as[Long]) + .map(_.toInt) + .repartition(1) + + val query = windowedAggregation + .writeStream + .outputMode(OutputMode.Append) + .foreach(new TestForeachWriter()) + .start() + try { + inputData.addData(10, 11, 12) + query.processAllAvailable() + inputData.addData(25) // Advance watermark to 15 seconds + query.processAllAvailable() + inputData.addData(25) // Evict items less than previous watermark + query.processAllAvailable() + + // There should be 3 batches and only does the last batch contain a value. + val allEvents = ForeachSinkSuite.allEvents() + assert(allEvents.size === 3) + val expectedEvents = Seq( + Seq( + ForeachSinkSuite.Open(partition = 0, version = 0), + ForeachSinkSuite.Close(None) + ), + Seq( + ForeachSinkSuite.Open(partition = 0, version = 1), + ForeachSinkSuite.Close(None) + ), + Seq( + ForeachSinkSuite.Open(partition = 0, version = 2), + ForeachSinkSuite.Process(value = 3), + ForeachSinkSuite.Close(None) + ) + ) + assert(allEvents === expectedEvents) + } finally { + query.stop() + } + } + + test("foreach sink should support metrics") { + val inputData = MemoryStream[Int] + val query = inputData.toDS() + .writeStream + .foreach(new TestForeachWriter()) + .start() + try { + inputData.addData(10, 11, 12) + query.processAllAvailable() + val recentProgress = query.recentProgress.filter(_.numInputRows != 0).headOption + assert(recentProgress.isDefined && recentProgress.get.numInputRows === 3, + s"recentProgress[${query.recentProgress.toList}] doesn't contain correct metrics") + } finally { + query.stop() + } + } +} + +/** A global object to collect events in the executor */ +object ForeachSinkSuite { + + trait Event + + case class Open(partition: Long, version: Long) extends Event + + case class Process[T](value: T) extends Event + + case class Close(error: Option[Throwable]) extends Event + + private val _allEvents = new ConcurrentLinkedQueue[Seq[Event]]() + + def addEvents(events: Seq[Event]): Unit = { + _allEvents.add(events) + } + + def allEvents(): Seq[Seq[Event]] = { + _allEvents.toArray(new Array[Seq[Event]](_allEvents.size())) + } + + def clear(): Unit = { + _allEvents.clear() + } +} + +/** A [[ForeachWriter]] that writes collected events to ForeachSinkSuite */ +class TestForeachWriter extends ForeachWriter[Int] { + ForeachSinkSuite.clear() + + private val events = mutable.ArrayBuffer[ForeachSinkSuite.Event]() + + override def open(partitionId: Long, version: Long): Boolean = { + events += ForeachSinkSuite.Open(partition = partitionId, version = version) + true + } + + override def process(value: Int): Unit = { + events += ForeachSinkSuite.Process(value) + } + + override def close(errorOrNull: Throwable): Unit = { + events += ForeachSinkSuite.Close(error = Option(errorOrNull)) + ForeachSinkSuite.addEvents(events) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala new file mode 100644 index 000000000000..48e70e48b179 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala @@ -0,0 +1,290 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.io.{File, FileNotFoundException, IOException} +import java.net.URI +import java.util.ConcurrentModificationException + +import scala.language.implicitConversions +import scala.util.Random + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.scalatest.concurrent.AsyncAssertions._ +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.sql.execution.streaming.FakeFileSystem._ +import org.apache.spark.sql.execution.streaming.HDFSMetadataLog.{FileContextManager, FileManager, FileSystemManager} +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.util.UninterruptibleThread + +class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { + + /** To avoid caching of FS objects */ + override protected def sparkConf = + super.sparkConf.set(s"spark.hadoop.fs.$scheme.impl.disable.cache", "true") + + private implicit def toOption[A](a: A): Option[A] = Option(a) + + test("FileManager: FileContextManager") { + withTempDir { temp => + val path = new Path(temp.getAbsolutePath) + testFileManager(path, new FileContextManager(path, new Configuration)) + } + } + + test("FileManager: FileSystemManager") { + withTempDir { temp => + val path = new Path(temp.getAbsolutePath) + testFileManager(path, new FileSystemManager(path, new Configuration)) + } + } + + test("HDFSMetadataLog: basic") { + withTempDir { temp => + val dir = new File(temp, "dir") // use non-existent directory to test whether log make the dir + val metadataLog = new HDFSMetadataLog[String](spark, dir.getAbsolutePath) + assert(metadataLog.add(0, "batch0")) + assert(metadataLog.getLatest() === Some(0 -> "batch0")) + assert(metadataLog.get(0) === Some("batch0")) + assert(metadataLog.getLatest() === Some(0 -> "batch0")) + assert(metadataLog.get(None, Some(0)) === Array(0 -> "batch0")) + + assert(metadataLog.add(1, "batch1")) + assert(metadataLog.get(0) === Some("batch0")) + assert(metadataLog.get(1) === Some("batch1")) + assert(metadataLog.getLatest() === Some(1 -> "batch1")) + assert(metadataLog.get(None, Some(1)) === Array(0 -> "batch0", 1 -> "batch1")) + + // Adding the same batch does nothing + metadataLog.add(1, "batch1-duplicated") + assert(metadataLog.get(0) === Some("batch0")) + assert(metadataLog.get(1) === Some("batch1")) + assert(metadataLog.getLatest() === Some(1 -> "batch1")) + assert(metadataLog.get(None, Some(1)) === Array(0 -> "batch0", 1 -> "batch1")) + } + } + + testQuietly("HDFSMetadataLog: fallback from FileContext to FileSystem") { + spark.conf.set( + s"fs.$scheme.impl", + classOf[FakeFileSystem].getName) + withTempDir { temp => + val metadataLog = new HDFSMetadataLog[String](spark, s"$scheme://${temp.toURI.getPath}") + assert(metadataLog.add(0, "batch0")) + assert(metadataLog.getLatest() === Some(0 -> "batch0")) + assert(metadataLog.get(0) === Some("batch0")) + assert(metadataLog.get(None, Some(0)) === Array(0 -> "batch0")) + + + val metadataLog2 = new HDFSMetadataLog[String](spark, s"$scheme://${temp.toURI.getPath}") + assert(metadataLog2.get(0) === Some("batch0")) + assert(metadataLog2.getLatest() === Some(0 -> "batch0")) + assert(metadataLog2.get(None, Some(0)) === Array(0 -> "batch0")) + + } + } + + test("HDFSMetadataLog: purge") { + withTempDir { temp => + val metadataLog = new HDFSMetadataLog[String](spark, temp.getAbsolutePath) + assert(metadataLog.add(0, "batch0")) + assert(metadataLog.add(1, "batch1")) + assert(metadataLog.add(2, "batch2")) + assert(metadataLog.get(0).isDefined) + assert(metadataLog.get(1).isDefined) + assert(metadataLog.get(2).isDefined) + assert(metadataLog.getLatest().get._1 == 2) + + metadataLog.purge(2) + assert(metadataLog.get(0).isEmpty) + assert(metadataLog.get(1).isEmpty) + assert(metadataLog.get(2).isDefined) + assert(metadataLog.getLatest().get._1 == 2) + + // There should be exactly one file, called "2", in the metadata directory. + // This check also tests for regressions of SPARK-17475 + val allFiles = new File(metadataLog.metadataPath.toString).listFiles().toSeq + assert(allFiles.size == 1) + assert(allFiles(0).getName() == "2") + } + } + + test("HDFSMetadataLog: parseVersion") { + withTempDir { dir => + val metadataLog = new HDFSMetadataLog[String](spark, dir.getAbsolutePath) + def assertLogFileMalformed(func: => Int): Unit = { + val e = intercept[IllegalStateException] { func } + assert(e.getMessage.contains(s"Log file was malformed: failed to read correct log version")) + } + assertLogFileMalformed { metadataLog.parseVersion("", 100) } + assertLogFileMalformed { metadataLog.parseVersion("xyz", 100) } + assertLogFileMalformed { metadataLog.parseVersion("v10.x", 100) } + assertLogFileMalformed { metadataLog.parseVersion("10", 100) } + assertLogFileMalformed { metadataLog.parseVersion("v0", 100) } + assertLogFileMalformed { metadataLog.parseVersion("v-10", 100) } + + assert(metadataLog.parseVersion("v10", 10) === 10) + assert(metadataLog.parseVersion("v10", 100) === 10) + + val e = intercept[IllegalStateException] { metadataLog.parseVersion("v200", 100) } + Seq( + "maximum supported log version is v100, but encountered v200", + "produced by a newer version of Spark and cannot be read by this version" + ).foreach { message => + assert(e.getMessage.contains(message)) + } + } + } + + test("HDFSMetadataLog: restart") { + withTempDir { temp => + val metadataLog = new HDFSMetadataLog[String](spark, temp.getAbsolutePath) + assert(metadataLog.add(0, "batch0")) + assert(metadataLog.add(1, "batch1")) + assert(metadataLog.get(0) === Some("batch0")) + assert(metadataLog.get(1) === Some("batch1")) + assert(metadataLog.getLatest() === Some(1 -> "batch1")) + assert(metadataLog.get(None, Some(1)) === Array(0 -> "batch0", 1 -> "batch1")) + + val metadataLog2 = new HDFSMetadataLog[String](spark, temp.getAbsolutePath) + assert(metadataLog2.get(0) === Some("batch0")) + assert(metadataLog2.get(1) === Some("batch1")) + assert(metadataLog2.getLatest() === Some(1 -> "batch1")) + assert(metadataLog2.get(None, Some(1)) === Array(0 -> "batch0", 1 -> "batch1")) + } + } + + test("HDFSMetadataLog: metadata directory collision") { + withTempDir { temp => + val waiter = new Waiter + val maxBatchId = 100 + for (id <- 0 until 10) { + new UninterruptibleThread(s"HDFSMetadataLog: metadata directory collision - thread $id") { + override def run(): Unit = waiter { + val metadataLog = + new HDFSMetadataLog[String](spark, temp.getAbsolutePath) + try { + var nextBatchId = metadataLog.getLatest().map(_._1).getOrElse(-1L) + nextBatchId += 1 + while (nextBatchId <= maxBatchId) { + metadataLog.add(nextBatchId, nextBatchId.toString) + nextBatchId += 1 + } + } catch { + case e: ConcurrentModificationException => + // This is expected since there are multiple writers + } finally { + waiter.dismiss() + } + } + }.start() + } + + waiter.await(timeout(10.seconds), dismissals(10)) + val metadataLog = new HDFSMetadataLog[String](spark, temp.getAbsolutePath) + assert(metadataLog.getLatest() === Some(maxBatchId -> maxBatchId.toString)) + assert( + metadataLog.get(None, Some(maxBatchId)) === (0 to maxBatchId).map(i => (i, i.toString))) + } + } + + /** Basic test case for [[FileManager]] implementation. */ + private def testFileManager(basePath: Path, fm: FileManager): Unit = { + // Mkdirs + val dir = new Path(s"$basePath/dir/subdir/subsubdir") + assert(!fm.exists(dir)) + fm.mkdirs(dir) + assert(fm.exists(dir)) + fm.mkdirs(dir) + + // List + val acceptAllFilter = new PathFilter { + override def accept(path: Path): Boolean = true + } + val rejectAllFilter = new PathFilter { + override def accept(path: Path): Boolean = false + } + assert(fm.list(basePath, acceptAllFilter).exists(_.getPath.getName == "dir")) + assert(fm.list(basePath, rejectAllFilter).length === 0) + + // Create + val path = new Path(s"$dir/file") + assert(!fm.exists(path)) + fm.create(path).close() + assert(fm.exists(path)) + intercept[IOException] { + fm.create(path) + } + + // Open and delete + fm.open(path).close() + fm.delete(path) + assert(!fm.exists(path)) + intercept[IOException] { + fm.open(path) + } + fm.delete(path) // should not throw exception + + // Rename + val path1 = new Path(s"$dir/file1") + val path2 = new Path(s"$dir/file2") + fm.create(path1).close() + assert(fm.exists(path1)) + fm.rename(path1, path2) + intercept[FileNotFoundException] { + fm.rename(path1, path2) + } + val path3 = new Path(s"$dir/file3") + fm.create(path3).close() + assert(fm.exists(path3)) + intercept[FileAlreadyExistsException] { + fm.rename(path2, path3) + } + } + + test("verifyBatchIds") { + import HDFSMetadataLog.verifyBatchIds + verifyBatchIds(Seq(1L, 2L, 3L), Some(1L), Some(3L)) + verifyBatchIds(Seq(1L), Some(1L), Some(1L)) + verifyBatchIds(Seq(1L, 2L, 3L), None, Some(3L)) + verifyBatchIds(Seq(1L, 2L, 3L), Some(1L), None) + verifyBatchIds(Seq(1L, 2L, 3L), None, None) + + intercept[IllegalStateException](verifyBatchIds(Seq(), Some(1L), None)) + intercept[IllegalStateException](verifyBatchIds(Seq(), None, Some(1L))) + intercept[IllegalStateException](verifyBatchIds(Seq(), Some(1L), Some(1L))) + intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), Some(1L), None)) + intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), None, Some(5L))) + intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), Some(1L), Some(5L))) + intercept[IllegalStateException](verifyBatchIds(Seq(1, 2, 4, 5), Some(1L), Some(5L))) + } +} + +/** FakeFileSystem to test fallback of the HDFSMetadataLog from FileContext to FileSystem API */ +class FakeFileSystem extends RawLocalFileSystem { + override def getUri: URI = { + URI.create(s"$scheme:///") + } +} + +object FakeFileSystem { + val scheme = s"HDFSMetadataLogSuite${math.abs(Random.nextInt)}" +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala new file mode 100644 index 000000000000..e8420eee7fe9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala @@ -0,0 +1,298 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import scala.language.implicitConversions + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql._ +import org.apache.spark.sql.streaming.{OutputMode, StreamTest} +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.util.Utils + +class MemorySinkSuite extends StreamTest with BeforeAndAfter { + + import testImplicits._ + + after { + sqlContext.streams.active.foreach(_.stop()) + } + + test("directly add data in Append output mode") { + implicit val schema = new StructType().add(new StructField("value", IntegerType)) + val sink = new MemorySink(schema, OutputMode.Append) + + // Before adding data, check output + assert(sink.latestBatchId === None) + checkAnswer(sink.latestBatchData, Seq.empty) + checkAnswer(sink.allData, Seq.empty) + + // Add batch 0 and check outputs + sink.addBatch(0, 1 to 3) + assert(sink.latestBatchId === Some(0)) + checkAnswer(sink.latestBatchData, 1 to 3) + checkAnswer(sink.allData, 1 to 3) + + // Add batch 1 and check outputs + sink.addBatch(1, 4 to 6) + assert(sink.latestBatchId === Some(1)) + checkAnswer(sink.latestBatchData, 4 to 6) + checkAnswer(sink.allData, 1 to 6) // new data should get appended to old data + + // Re-add batch 1 with different data, should not be added and outputs should not be changed + sink.addBatch(1, 7 to 9) + assert(sink.latestBatchId === Some(1)) + checkAnswer(sink.latestBatchData, 4 to 6) + checkAnswer(sink.allData, 1 to 6) + + // Add batch 2 and check outputs + sink.addBatch(2, 7 to 9) + assert(sink.latestBatchId === Some(2)) + checkAnswer(sink.latestBatchData, 7 to 9) + checkAnswer(sink.allData, 1 to 9) + } + + test("directly add data in Update output mode") { + implicit val schema = new StructType().add(new StructField("value", IntegerType)) + val sink = new MemorySink(schema, OutputMode.Update) + + // Before adding data, check output + assert(sink.latestBatchId === None) + checkAnswer(sink.latestBatchData, Seq.empty) + checkAnswer(sink.allData, Seq.empty) + + // Add batch 0 and check outputs + sink.addBatch(0, 1 to 3) + assert(sink.latestBatchId === Some(0)) + checkAnswer(sink.latestBatchData, 1 to 3) + checkAnswer(sink.allData, 1 to 3) + + // Add batch 1 and check outputs + sink.addBatch(1, 4 to 6) + assert(sink.latestBatchId === Some(1)) + checkAnswer(sink.latestBatchData, 4 to 6) + checkAnswer(sink.allData, 1 to 6) // new data should get appended to old data + + // Re-add batch 1 with different data, should not be added and outputs should not be changed + sink.addBatch(1, 7 to 9) + assert(sink.latestBatchId === Some(1)) + checkAnswer(sink.latestBatchData, 4 to 6) + checkAnswer(sink.allData, 1 to 6) + + // Add batch 2 and check outputs + sink.addBatch(2, 7 to 9) + assert(sink.latestBatchId === Some(2)) + checkAnswer(sink.latestBatchData, 7 to 9) + checkAnswer(sink.allData, 1 to 9) + } + + test("directly add data in Complete output mode") { + implicit val schema = new StructType().add(new StructField("value", IntegerType)) + val sink = new MemorySink(schema, OutputMode.Complete) + + // Before adding data, check output + assert(sink.latestBatchId === None) + checkAnswer(sink.latestBatchData, Seq.empty) + checkAnswer(sink.allData, Seq.empty) + + // Add batch 0 and check outputs + sink.addBatch(0, 1 to 3) + assert(sink.latestBatchId === Some(0)) + checkAnswer(sink.latestBatchData, 1 to 3) + checkAnswer(sink.allData, 1 to 3) + + // Add batch 1 and check outputs + sink.addBatch(1, 4 to 6) + assert(sink.latestBatchId === Some(1)) + checkAnswer(sink.latestBatchData, 4 to 6) + checkAnswer(sink.allData, 4 to 6) // new data should replace old data + + // Re-add batch 1 with different data, should not be added and outputs should not be changed + sink.addBatch(1, 7 to 9) + assert(sink.latestBatchId === Some(1)) + checkAnswer(sink.latestBatchData, 4 to 6) + checkAnswer(sink.allData, 4 to 6) + + // Add batch 2 and check outputs + sink.addBatch(2, 7 to 9) + assert(sink.latestBatchId === Some(2)) + checkAnswer(sink.latestBatchData, 7 to 9) + checkAnswer(sink.allData, 7 to 9) + } + + + test("registering as a table in Append output mode") { + val input = MemoryStream[Int] + val query = input.toDF().writeStream + .format("memory") + .outputMode("append") + .queryName("memStream") + .start() + input.addData(1, 2, 3) + query.processAllAvailable() + + checkDataset( + spark.table("memStream").as[Int], + 1, 2, 3) + + input.addData(4, 5, 6) + query.processAllAvailable() + checkDataset( + spark.table("memStream").as[Int], + 1, 2, 3, 4, 5, 6) + + query.stop() + } + + test("registering as a table in Complete output mode") { + val input = MemoryStream[Int] + val query = input.toDF() + .groupBy("value") + .count() + .writeStream + .format("memory") + .outputMode("complete") + .queryName("memStream") + .start() + input.addData(1, 2, 3) + query.processAllAvailable() + + checkDatasetUnorderly( + spark.table("memStream").as[(Int, Long)], + (1, 1L), (2, 1L), (3, 1L)) + + input.addData(4, 5, 6) + query.processAllAvailable() + checkDatasetUnorderly( + spark.table("memStream").as[(Int, Long)], + (1, 1L), (2, 1L), (3, 1L), (4, 1L), (5, 1L), (6, 1L)) + + query.stop() + } + + test("registering as a table in Update output mode") { + val input = MemoryStream[Int] + val query = input.toDF().writeStream + .format("memory") + .outputMode("update") + .queryName("memStream") + .start() + input.addData(1, 2, 3) + query.processAllAvailable() + + checkDataset( + spark.table("memStream").as[Int], + 1, 2, 3) + + input.addData(4, 5, 6) + query.processAllAvailable() + checkDataset( + spark.table("memStream").as[Int], + 1, 2, 3, 4, 5, 6) + + query.stop() + } + + test("MemoryPlan statistics") { + implicit val schema = new StructType().add(new StructField("value", IntegerType)) + val sink = new MemorySink(schema, OutputMode.Append) + val plan = new MemoryPlan(sink) + + // Before adding data, check output + checkAnswer(sink.allData, Seq.empty) + assert(plan.stats.sizeInBytes === 0) + + sink.addBatch(0, 1 to 3) + plan.invalidateStatsCache() + assert(plan.stats.sizeInBytes === 12) + + sink.addBatch(1, 4 to 6) + plan.invalidateStatsCache() + assert(plan.stats.sizeInBytes === 24) + } + + ignore("stress test") { + // Ignore the stress test as it takes several minutes to run + (0 until 1000).foreach { _ => + val input = MemoryStream[Int] + val query = input.toDF().writeStream + .format("memory") + .queryName("memStream") + .start() + input.addData(1, 2, 3) + query.processAllAvailable() + + checkDataset( + spark.table("memStream").as[Int], + 1, 2, 3) + + input.addData(4, 5, 6) + query.processAllAvailable() + checkDataset( + spark.table("memStream").as[Int], + 1, 2, 3, 4, 5, 6) + + query.stop() + } + } + + test("error when no name is specified") { + val error = intercept[AnalysisException] { + val input = MemoryStream[Int] + val query = input.toDF().writeStream + .format("memory") + .start() + } + + assert(error.message contains "queryName must be specified") + } + + test("error if attempting to resume specific checkpoint") { + val location = Utils.createTempDir(namePrefix = "steaming.checkpoint").getCanonicalPath + + val input = MemoryStream[Int] + val query = input.toDF().writeStream + .format("memory") + .queryName("memStream") + .option("checkpointLocation", location) + .start() + input.addData(1, 2, 3) + query.processAllAvailable() + query.stop() + + intercept[AnalysisException] { + input.toDF().writeStream + .format("memory") + .queryName("memStream") + .option("checkpointLocation", location) + .start() + } + } + + private def checkAnswer(rows: Seq[Row], expected: Seq[Int])(implicit schema: StructType): Unit = { + checkAnswer( + sqlContext.createDataFrame(sparkContext.makeRDD(rows), schema), + intsToDF(expected)(schema)) + } + + private implicit def intsToDF(seq: Seq[Int])(implicit schema: StructType): DataFrame = { + require(schema.fields.size === 1) + sqlContext.createDataset(seq).toDF(schema.fieldNames.head) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala new file mode 100644 index 000000000000..e6cdc063c4e9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.io.File + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.stringToFile +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class OffsetSeqLogSuite extends SparkFunSuite with SharedSQLContext { + + /** test string offset type */ + case class StringOffset(override val json: String) extends Offset + + test("OffsetSeqMetadata - deserialization") { + val key = SQLConf.SHUFFLE_PARTITIONS.key + + def getConfWith(shufflePartitions: Int): Map[String, String] = { + Map(key -> shufflePartitions.toString) + } + + // None set + assert(new OffsetSeqMetadata(0, 0, Map.empty) === OffsetSeqMetadata("""{}""")) + + // One set + assert(new OffsetSeqMetadata(1, 0, Map.empty) === + OffsetSeqMetadata("""{"batchWatermarkMs":1}""")) + assert(new OffsetSeqMetadata(0, 2, Map.empty) === + OffsetSeqMetadata("""{"batchTimestampMs":2}""")) + assert(OffsetSeqMetadata(0, 0, getConfWith(shufflePartitions = 2)) === + OffsetSeqMetadata(s"""{"conf": {"$key":2}}""")) + + // Two set + assert(new OffsetSeqMetadata(1, 2, Map.empty) === + OffsetSeqMetadata("""{"batchWatermarkMs":1,"batchTimestampMs":2}""")) + assert(OffsetSeqMetadata(1, 0, getConfWith(shufflePartitions = 3)) === + OffsetSeqMetadata(s"""{"batchWatermarkMs":1,"conf": {"$key":3}}""")) + assert(OffsetSeqMetadata(0, 2, getConfWith(shufflePartitions = 3)) === + OffsetSeqMetadata(s"""{"batchTimestampMs":2,"conf": {"$key":3}}""")) + + // All set + assert(OffsetSeqMetadata(1, 2, getConfWith(shufflePartitions = 3)) === + OffsetSeqMetadata(s"""{"batchWatermarkMs":1,"batchTimestampMs":2,"conf": {"$key":3}}""")) + + // Drop unknown fields + assert(OffsetSeqMetadata(1, 2, getConfWith(shufflePartitions = 3)) === + OffsetSeqMetadata( + s"""{"batchWatermarkMs":1,"batchTimestampMs":2,"conf": {"$key":3}},"unknown":1""")) + } + + test("OffsetSeqLog - serialization - deserialization") { + withTempDir { temp => + val dir = new File(temp, "dir") // use non-existent directory to test whether log make the dir + val metadataLog = new OffsetSeqLog(spark, dir.getAbsolutePath) + val batch0 = OffsetSeq.fill(LongOffset(0), LongOffset(1), LongOffset(2)) + val batch1 = OffsetSeq.fill(StringOffset("one"), StringOffset("two"), StringOffset("three")) + + val batch0Serialized = OffsetSeq.fill(batch0.offsets.flatMap(_.map(o => + SerializedOffset(o.json))): _*) + + val batch1Serialized = OffsetSeq.fill(batch1.offsets.flatMap(_.map(o => + SerializedOffset(o.json))): _*) + + assert(metadataLog.add(0, batch0)) + assert(metadataLog.getLatest() === Some(0 -> batch0Serialized)) + assert(metadataLog.get(0) === Some(batch0Serialized)) + + assert(metadataLog.add(1, batch1)) + assert(metadataLog.get(0) === Some(batch0Serialized)) + assert(metadataLog.get(1) === Some(batch1Serialized)) + assert(metadataLog.getLatest() === Some(1 -> batch1Serialized)) + assert(metadataLog.get(None, Some(1)) === + Array(0 -> batch0Serialized, 1 -> batch1Serialized)) + + // Adding the same batch does nothing + metadataLog.add(1, OffsetSeq.fill(LongOffset(3))) + assert(metadataLog.get(0) === Some(batch0Serialized)) + assert(metadataLog.get(1) === Some(batch1Serialized)) + assert(metadataLog.getLatest() === Some(1 -> batch1Serialized)) + assert(metadataLog.get(None, Some(1)) === + Array(0 -> batch0Serialized, 1 -> batch1Serialized)) + } + } + + test("deserialization log written by future version") { + withTempDir { dir => + stringToFile(new File(dir, "0"), "v99999") + val log = new OffsetSeqLog(spark, dir.getCanonicalPath) + val e = intercept[IllegalStateException] { + log.get(0) + } + Seq( + s"maximum supported log version is v${OffsetSeqLog.VERSION}, but encountered v99999", + "produced by a newer version of Spark and cannot be read by this version" + ).foreach { message => + assert(e.getMessage.contains(message)) + } + } + } + + test("read Spark 2.1.0 log format") { + val (batchId, offsetSeq) = readFromResource("offset-log-version-2.1.0") + assert(batchId === 0) + assert(offsetSeq.offsets === Seq( + Some(SerializedOffset("""{"logOffset":345}""")), + Some(SerializedOffset("""{"topic-0":{"0":1}}""")) + )) + assert(offsetSeq.metadata === Some(OffsetSeqMetadata(0L, 1480981499528L))) + } + + private def readFromResource(dir: String): (Long, OffsetSeq) = { + val input = getClass.getResource(s"/structured-streaming/$dir") + val log = new OffsetSeqLog(spark, input.toString) + log.getLatest().get + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala new file mode 100644 index 000000000000..519e3c01afe8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.mutable + +import org.eclipse.jetty.util.ConcurrentHashSet +import org.scalatest.concurrent.Eventually +import org.scalatest.concurrent.PatienceConfiguration.Timeout +import org.scalatest.concurrent.TimeLimits._ +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.streaming.ProcessingTime +import org.apache.spark.sql.streaming.util.StreamManualClock + +class ProcessingTimeExecutorSuite extends SparkFunSuite { + + val timeout = 10.seconds + + test("nextBatchTime") { + val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(100)) + assert(processingTimeExecutor.nextBatchTime(0) === 100) + assert(processingTimeExecutor.nextBatchTime(1) === 100) + assert(processingTimeExecutor.nextBatchTime(99) === 100) + assert(processingTimeExecutor.nextBatchTime(100) === 200) + assert(processingTimeExecutor.nextBatchTime(101) === 200) + assert(processingTimeExecutor.nextBatchTime(150) === 200) + } + + test("trigger timing") { + val triggerTimes = new ConcurrentHashSet[Int] + val clock = new StreamManualClock() + @volatile var continueExecuting = true + @volatile var clockIncrementInTrigger = 0L + val executor = ProcessingTimeExecutor(ProcessingTime("1000 milliseconds"), clock) + val executorThread = new Thread() { + override def run(): Unit = { + executor.execute(() => { + // Record the trigger time, increment clock if needed and + triggerTimes.add(clock.getTimeMillis.toInt) + clock.advance(clockIncrementInTrigger) + clockIncrementInTrigger = 0 // reset this so that there are no runaway triggers + continueExecuting + }) + } + } + executorThread.start() + // First batch should execute immediately, then executor should wait for next one + eventually { + assert(triggerTimes.contains(0)) + assert(clock.isStreamWaitingAt(0)) + assert(clock.isStreamWaitingFor(1000)) + } + + // Second batch should execute when clock reaches the next trigger time. + // If next trigger takes less than the trigger interval, executor should wait for next one + clockIncrementInTrigger = 500 + clock.setTime(1000) + eventually { + assert(triggerTimes.contains(1000)) + assert(clock.isStreamWaitingAt(1500)) + assert(clock.isStreamWaitingFor(2000)) + } + + // If next trigger takes less than the trigger interval, executor should immediately execute + // another one + clockIncrementInTrigger = 1500 + clock.setTime(2000) // allow another trigger by setting clock to 2000 + eventually { + // Since the next trigger will take 1500 (which is more than trigger interval of 1000) + // executor will immediately execute another trigger + assert(triggerTimes.contains(2000) && triggerTimes.contains(3500)) + assert(clock.isStreamWaitingAt(3500)) + assert(clock.isStreamWaitingFor(4000)) + } + continueExecuting = false + clock.advance(1000) + waitForThreadJoin(executorThread) + } + + test("calling nextBatchTime with the result of a previous call should return the next interval") { + val intervalMS = 100 + val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(intervalMS)) + + val ITERATION = 10 + var nextBatchTime: Long = 0 + for (it <- 1 to ITERATION) { + nextBatchTime = processingTimeExecutor.nextBatchTime(nextBatchTime) + } + + // nextBatchTime should be 1000 + assert(nextBatchTime === intervalMS * ITERATION) + } + + private def testBatchTermination(intervalMs: Long): Unit = { + var batchCounts = 0 + val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(intervalMs)) + processingTimeExecutor.execute(() => { + batchCounts += 1 + // If the batch termination works correctly, batchCounts should be 3 after `execute` + batchCounts < 3 + }) + assert(batchCounts === 3) + } + + test("batch termination") { + testBatchTermination(0) + testBatchTermination(10) + } + + test("notifyBatchFallingBehind") { + val clock = new StreamManualClock() + @volatile var batchFallingBehindCalled = false + val t = new Thread() { + override def run(): Unit = { + val processingTimeExecutor = new ProcessingTimeExecutor(ProcessingTime(100), clock) { + override def notifyBatchFallingBehind(realElapsedTimeMs: Long): Unit = { + batchFallingBehindCalled = true + } + } + processingTimeExecutor.execute(() => { + clock.waitTillTime(200) + false + }) + } + } + t.start() + // Wait until the batch is running so that we don't call `advance` too early + eventually { assert(clock.isStreamWaitingFor(200)) } + clock.advance(200) + waitForThreadJoin(t) + assert(batchFallingBehindCalled === true) + } + + private def eventually(body: => Unit): Unit = { + Eventually.eventually(Timeout(timeout)) { body } + } + + private def waitForThreadJoin(thread: Thread): Unit = { + failAfter(timeout) { thread.join() } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceSuite.scala new file mode 100644 index 000000000000..03d0f63fa4d7 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/RateSourceSuite.scala @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.util.concurrent.TimeUnit + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.streaming.{StreamingQueryException, StreamTest} +import org.apache.spark.util.ManualClock + +class RateSourceSuite extends StreamTest { + + import testImplicits._ + + case class AdvanceRateManualClock(seconds: Long) extends AddData { + override def addData(query: Option[StreamExecution]): (Source, Offset) = { + assert(query.nonEmpty) + val rateSource = query.get.logicalPlan.collect { + case StreamingExecutionRelation(source, _) if source.isInstanceOf[RateStreamSource] => + source.asInstanceOf[RateStreamSource] + }.head + rateSource.clock.asInstanceOf[ManualClock].advance(TimeUnit.SECONDS.toMillis(seconds)) + (rateSource, rateSource.getOffset.get) + } + } + + test("basic") { + val input = spark.readStream + .format("rate") + .option("rowsPerSecond", "10") + .option("useManualClock", "true") + .load() + testStream(input)( + AdvanceRateManualClock(seconds = 1), + CheckLastBatch((0 until 10).map(v => new java.sql.Timestamp(v * 100L) -> v): _*), + StopStream, + StartStream(), + // Advance 2 seconds because creating a new RateSource will also create a new ManualClock + AdvanceRateManualClock(seconds = 2), + CheckLastBatch((10 until 20).map(v => new java.sql.Timestamp(v * 100L) -> v): _*) + ) + } + + test("uniform distribution of event timestamps") { + val input = spark.readStream + .format("rate") + .option("rowsPerSecond", "1500") + .option("useManualClock", "true") + .load() + .as[(java.sql.Timestamp, Long)] + .map(v => (v._1.getTime, v._2)) + val expectedAnswer = (0 until 1500).map { v => + (math.round(v * (1000.0 / 1500)), v) + } + testStream(input)( + AdvanceRateManualClock(seconds = 1), + CheckLastBatch(expectedAnswer: _*) + ) + } + + test("valueAtSecond") { + import RateStreamSource._ + + assert(valueAtSecond(seconds = 0, rowsPerSecond = 5, rampUpTimeSeconds = 0) === 0) + assert(valueAtSecond(seconds = 1, rowsPerSecond = 5, rampUpTimeSeconds = 0) === 5) + + assert(valueAtSecond(seconds = 0, rowsPerSecond = 5, rampUpTimeSeconds = 2) === 0) + assert(valueAtSecond(seconds = 1, rowsPerSecond = 5, rampUpTimeSeconds = 2) === 1) + assert(valueAtSecond(seconds = 2, rowsPerSecond = 5, rampUpTimeSeconds = 2) === 3) + assert(valueAtSecond(seconds = 3, rowsPerSecond = 5, rampUpTimeSeconds = 2) === 8) + + assert(valueAtSecond(seconds = 0, rowsPerSecond = 10, rampUpTimeSeconds = 4) === 0) + assert(valueAtSecond(seconds = 1, rowsPerSecond = 10, rampUpTimeSeconds = 4) === 2) + assert(valueAtSecond(seconds = 2, rowsPerSecond = 10, rampUpTimeSeconds = 4) === 6) + assert(valueAtSecond(seconds = 3, rowsPerSecond = 10, rampUpTimeSeconds = 4) === 12) + assert(valueAtSecond(seconds = 4, rowsPerSecond = 10, rampUpTimeSeconds = 4) === 20) + assert(valueAtSecond(seconds = 5, rowsPerSecond = 10, rampUpTimeSeconds = 4) === 30) + } + + test("rampUpTime") { + val input = spark.readStream + .format("rate") + .option("rowsPerSecond", "10") + .option("rampUpTime", "4s") + .option("useManualClock", "true") + .load() + .as[(java.sql.Timestamp, Long)] + .map(v => (v._1.getTime, v._2)) + testStream(input)( + AdvanceRateManualClock(seconds = 1), + CheckLastBatch((0 until 2).map(v => v * 500 -> v): _*), // speed = 2 + AdvanceRateManualClock(seconds = 1), + CheckLastBatch((2 until 6).map(v => 1000 + (v - 2) * 250 -> v): _*), // speed = 4 + AdvanceRateManualClock(seconds = 1), + CheckLastBatch({ + Seq(2000 -> 6, 2167 -> 7, 2333 -> 8, 2500 -> 9, 2667 -> 10, 2833 -> 11) + }: _*), // speed = 6 + AdvanceRateManualClock(seconds = 1), + CheckLastBatch((12 until 20).map(v => 3000 + (v - 12) * 125 -> v): _*), // speed = 8 + AdvanceRateManualClock(seconds = 1), + // Now we should reach full speed + CheckLastBatch((20 until 30).map(v => 4000 + (v - 20) * 100 -> v): _*), // speed = 10 + AdvanceRateManualClock(seconds = 1), + CheckLastBatch((30 until 40).map(v => 5000 + (v - 30) * 100 -> v): _*), // speed = 10 + AdvanceRateManualClock(seconds = 1), + CheckLastBatch((40 until 50).map(v => 6000 + (v - 40) * 100 -> v): _*) // speed = 10 + ) + } + + test("numPartitions") { + val input = spark.readStream + .format("rate") + .option("rowsPerSecond", "10") + .option("numPartitions", "6") + .option("useManualClock", "true") + .load() + .select(spark_partition_id()) + .distinct() + testStream(input)( + AdvanceRateManualClock(1), + CheckLastBatch((0 until 6): _*) + ) + } + + testQuietly("overflow") { + val input = spark.readStream + .format("rate") + .option("rowsPerSecond", Long.MaxValue.toString) + .option("useManualClock", "true") + .load() + .select(spark_partition_id()) + .distinct() + testStream(input)( + AdvanceRateManualClock(2), + ExpectFailure[ArithmeticException](t => { + Seq("overflow", "rowsPerSecond").foreach { msg => + assert(t.getMessage.contains(msg)) + } + }) + ) + } + + testQuietly("illegal option values") { + def testIllegalOptionValue( + option: String, + value: String, + expectedMessages: Seq[String]): Unit = { + val e = intercept[StreamingQueryException] { + spark.readStream + .format("rate") + .option(option, value) + .load() + .writeStream + .format("console") + .start() + .awaitTermination() + } + assert(e.getCause.isInstanceOf[IllegalArgumentException]) + for (msg <- expectedMessages) { + assert(e.getCause.getMessage.contains(msg)) + } + } + + testIllegalOptionValue("rowsPerSecond", "-1", Seq("-1", "rowsPerSecond", "positive")) + testIllegalOptionValue("numPartitions", "-1", Seq("-1", "numPartitions", "positive")) + } + + test("user-specified schema given") { + val exception = intercept[AnalysisException] { + spark.readStream + .format("rate") + .schema(spark.range(1).schema) + .load() + } + assert(exception.getMessage.contains( + "rate source does not support a user-specified schema")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetadataSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetadataSuite.scala new file mode 100644 index 000000000000..87f8004ab958 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetadataSuite.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.execution.streaming + +import java.io.File +import java.util.UUID + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.streaming.StreamTest + +class StreamMetadataSuite extends StreamTest { + + test("writing and reading") { + withTempDir { dir => + val id = UUID.randomUUID.toString + val metadata = StreamMetadata(id) + val file = new Path(new File(dir, "test").toString) + StreamMetadata.write(metadata, file, hadoopConf) + val readMetadata = StreamMetadata.read(file, hadoopConf) + assert(readMetadata.nonEmpty) + assert(readMetadata.get.id === id) + } + } + + test("read Spark 2.1.0 format") { + // query-metadata-logs-version-2.1.0.txt has the execution metadata generated by Spark 2.1.0 + assert( + readForResource("query-metadata-logs-version-2.1.0.txt") === + StreamMetadata("d366a8bf-db79-42ca-b5a4-d9ca0a11d63e")) + } + + private def readForResource(fileName: String): StreamMetadata = { + val input = getClass.getResource(s"/structured-streaming/$fileName") + StreamMetadata.read(new Path(input.toString), hadoopConf).get + } + + private val hadoopConf = new Configuration() +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala new file mode 100644 index 000000000000..ec1154907365 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala @@ -0,0 +1,231 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.io.{IOException, OutputStreamWriter} +import java.net.ServerSocket +import java.sql.Timestamp +import java.util.concurrent.LinkedBlockingQueue + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.streaming.StreamTest +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} + +class TextSocketStreamSuite extends StreamTest with SharedSQLContext with BeforeAndAfterEach { + import testImplicits._ + + override def afterEach() { + sqlContext.streams.active.foreach(_.stop()) + if (serverThread != null) { + serverThread.interrupt() + serverThread.join() + serverThread = null + } + if (source != null) { + source.stop() + source = null + } + } + + private var serverThread: ServerThread = null + private var source: Source = null + + test("basic usage") { + serverThread = new ServerThread() + serverThread.start() + + val provider = new TextSocketSourceProvider + val parameters = Map("host" -> "localhost", "port" -> serverThread.port.toString) + val schema = provider.sourceSchema(sqlContext, None, "", parameters)._2 + assert(schema === StructType(StructField("value", StringType) :: Nil)) + + source = provider.createSource(sqlContext, "", None, "", parameters) + + failAfter(streamingTimeout) { + serverThread.enqueue("hello") + while (source.getOffset.isEmpty) { + Thread.sleep(10) + } + withSQLConf("spark.sql.streaming.unsupportedOperationCheck" -> "false") { + val offset1 = source.getOffset.get + val batch1 = source.getBatch(None, offset1) + assert(batch1.as[String].collect().toSeq === Seq("hello")) + + serverThread.enqueue("world") + while (source.getOffset.get === offset1) { + Thread.sleep(10) + } + val offset2 = source.getOffset.get + val batch2 = source.getBatch(Some(offset1), offset2) + assert(batch2.as[String].collect().toSeq === Seq("world")) + + val both = source.getBatch(None, offset2) + assert(both.as[String].collect().sorted.toSeq === Seq("hello", "world")) + } + + // Try stopping the source to make sure this does not block forever. + source.stop() + source = null + } + } + + test("timestamped usage") { + serverThread = new ServerThread() + serverThread.start() + + val provider = new TextSocketSourceProvider + val parameters = Map("host" -> "localhost", "port" -> serverThread.port.toString, + "includeTimestamp" -> "true") + val schema = provider.sourceSchema(sqlContext, None, "", parameters)._2 + assert(schema === StructType(StructField("value", StringType) :: + StructField("timestamp", TimestampType) :: Nil)) + + source = provider.createSource(sqlContext, "", None, "", parameters) + + failAfter(streamingTimeout) { + serverThread.enqueue("hello") + while (source.getOffset.isEmpty) { + Thread.sleep(10) + } + withSQLConf("spark.sql.streaming.unsupportedOperationCheck" -> "false") { + val offset1 = source.getOffset.get + val batch1 = source.getBatch(None, offset1) + val batch1Seq = batch1.as[(String, Timestamp)].collect().toSeq + assert(batch1Seq.map(_._1) === Seq("hello")) + val batch1Stamp = batch1Seq(0)._2 + + serverThread.enqueue("world") + while (source.getOffset.get === offset1) { + Thread.sleep(10) + } + val offset2 = source.getOffset.get + val batch2 = source.getBatch(Some(offset1), offset2) + val batch2Seq = batch2.as[(String, Timestamp)].collect().toSeq + assert(batch2Seq.map(_._1) === Seq("world")) + val batch2Stamp = batch2Seq(0)._2 + assert(!batch2Stamp.before(batch1Stamp)) + } + + // Try stopping the source to make sure this does not block forever. + source.stop() + source = null + } + } + + test("params not given") { + val provider = new TextSocketSourceProvider + intercept[AnalysisException] { + provider.sourceSchema(sqlContext, None, "", Map()) + } + intercept[AnalysisException] { + provider.sourceSchema(sqlContext, None, "", Map("host" -> "localhost")) + } + intercept[AnalysisException] { + provider.sourceSchema(sqlContext, None, "", Map("port" -> "1234")) + } + } + + test("non-boolean includeTimestamp") { + val provider = new TextSocketSourceProvider + intercept[AnalysisException] { + provider.sourceSchema(sqlContext, None, "", Map("host" -> "localhost", + "port" -> "1234", "includeTimestamp" -> "fasle")) + } + } + + test("user-specified schema given") { + val provider = new TextSocketSourceProvider + val userSpecifiedSchema = StructType( + StructField("name", StringType) :: + StructField("area", StringType) :: Nil) + val exception = intercept[AnalysisException] { + provider.sourceSchema( + sqlContext, Some(userSpecifiedSchema), + "", + Map("host" -> "localhost", "port" -> "1234")) + } + assert(exception.getMessage.contains( + "socket source does not support a user-specified schema")) + } + + test("no server up") { + val provider = new TextSocketSourceProvider + val parameters = Map("host" -> "localhost", "port" -> "0") + intercept[IOException] { + source = provider.createSource(sqlContext, "", None, "", parameters) + } + } + + test("input row metrics") { + serverThread = new ServerThread() + serverThread.start() + + val provider = new TextSocketSourceProvider + val parameters = Map("host" -> "localhost", "port" -> serverThread.port.toString) + source = provider.createSource(sqlContext, "", None, "", parameters) + + failAfter(streamingTimeout) { + serverThread.enqueue("hello") + while (source.getOffset.isEmpty) { + Thread.sleep(10) + } + withSQLConf("spark.sql.streaming.unsupportedOperationCheck" -> "false") { + val batch = source.getBatch(None, source.getOffset.get).as[String] + batch.collect() + val numRowsMetric = + batch.queryExecution.executedPlan.collectLeaves().head.metrics.get("numOutputRows") + assert(numRowsMetric.nonEmpty) + assert(numRowsMetric.get.value === 1) + } + source.stop() + source = null + } + } + + private class ServerThread extends Thread with Logging { + private val serverSocket = new ServerSocket(0) + private val messageQueue = new LinkedBlockingQueue[String]() + + val port = serverSocket.getLocalPort + + override def run(): Unit = { + try { + val clientSocket = serverSocket.accept() + clientSocket.setTcpNoDelay(true) + val out = new OutputStreamWriter(clientSocket.getOutputStream) + while (true) { + val line = messageQueue.take() + out.write(line + "\n") + out.flush() + } + } catch { + case e: InterruptedException => + } finally { + serverSocket.close() + } + } + + def enqueue(line: String): Unit = { + messageQueue.put(line) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala new file mode 100644 index 000000000000..9a7595eee7bd --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming.state + +import java.util.UUID + +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.{SharedSparkContext, SparkContext, SparkFunSuite} +import org.apache.spark.scheduler.ExecutorCacheTaskLocation +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingQueryWrapper} +import org.apache.spark.sql.functions.count +import org.apache.spark.util.Utils + +class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { + + import StateStoreCoordinatorSuite._ + + test("report, verify, getLocation") { + withCoordinatorRef(sc) { coordinatorRef => + val id = StateStoreProviderId(StateStoreId("x", 0, 0), UUID.randomUUID) + + assert(coordinatorRef.verifyIfInstanceActive(id, "exec1") === false) + assert(coordinatorRef.getLocation(id) === None) + + coordinatorRef.reportActiveInstance(id, "hostX", "exec1") + eventually(timeout(5 seconds)) { + assert(coordinatorRef.verifyIfInstanceActive(id, "exec1") === true) + assert( + coordinatorRef.getLocation(id) === + Some(ExecutorCacheTaskLocation("hostX", "exec1").toString)) + } + + coordinatorRef.reportActiveInstance(id, "hostX", "exec2") + + eventually(timeout(5 seconds)) { + assert(coordinatorRef.verifyIfInstanceActive(id, "exec1") === false) + assert(coordinatorRef.verifyIfInstanceActive(id, "exec2") === true) + + assert( + coordinatorRef.getLocation(id) === + Some(ExecutorCacheTaskLocation("hostX", "exec2").toString)) + } + } + } + + test("make inactive") { + withCoordinatorRef(sc) { coordinatorRef => + val runId1 = UUID.randomUUID + val runId2 = UUID.randomUUID + val id1 = StateStoreProviderId(StateStoreId("x", 0, 0), runId1) + val id2 = StateStoreProviderId(StateStoreId("y", 1, 0), runId2) + val id3 = StateStoreProviderId(StateStoreId("x", 0, 1), runId1) + val host = "hostX" + val exec = "exec1" + + coordinatorRef.reportActiveInstance(id1, host, exec) + coordinatorRef.reportActiveInstance(id2, host, exec) + coordinatorRef.reportActiveInstance(id3, host, exec) + + eventually(timeout(5 seconds)) { + assert(coordinatorRef.verifyIfInstanceActive(id1, exec) === true) + assert(coordinatorRef.verifyIfInstanceActive(id2, exec) === true) + assert(coordinatorRef.verifyIfInstanceActive(id3, exec) === true) + } + + coordinatorRef.deactivateInstances(runId1) + + assert(coordinatorRef.verifyIfInstanceActive(id1, exec) === false) + assert(coordinatorRef.verifyIfInstanceActive(id2, exec) === true) + assert(coordinatorRef.verifyIfInstanceActive(id3, exec) === false) + + assert(coordinatorRef.getLocation(id1) === None) + assert( + coordinatorRef.getLocation(id2) === + Some(ExecutorCacheTaskLocation(host, exec).toString)) + assert(coordinatorRef.getLocation(id3) === None) + + coordinatorRef.deactivateInstances(runId2) + assert(coordinatorRef.verifyIfInstanceActive(id2, exec) === false) + assert(coordinatorRef.getLocation(id2) === None) + } + } + + test("multiple references have same underlying coordinator") { + withCoordinatorRef(sc) { coordRef1 => + val coordRef2 = StateStoreCoordinatorRef.forDriver(sc.env) + + val id = StateStoreProviderId(StateStoreId("x", 0, 0), UUID.randomUUID) + + coordRef1.reportActiveInstance(id, "hostX", "exec1") + + eventually(timeout(5 seconds)) { + assert(coordRef2.verifyIfInstanceActive(id, "exec1") === true) + assert( + coordRef2.getLocation(id) === + Some(ExecutorCacheTaskLocation("hostX", "exec1").toString)) + } + } + } + + test("query stop deactivates related store providers") { + var coordRef: StateStoreCoordinatorRef = null + try { + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + SparkSession.setActiveSession(spark) + import spark.implicits._ + coordRef = spark.streams.stateStoreCoordinator + implicit val sqlContext = spark.sqlContext + spark.conf.set("spark.sql.shuffle.partitions", "1") + + // Start a query and run a batch to load state stores + val inputData = MemoryStream[Int] + val aggregated = inputData.toDF().groupBy("value").agg(count("*")) // stateful query + val checkpointLocation = Utils.createTempDir().getAbsoluteFile + val query = aggregated.writeStream + .format("memory") + .outputMode("update") + .queryName("query") + .option("checkpointLocation", checkpointLocation.toString) + .start() + inputData.addData(1, 2, 3) + query.processAllAvailable() + + // Verify state store has been loaded + val stateCheckpointDir = + query.asInstanceOf[StreamingQueryWrapper].streamingQuery.lastExecution.checkpointLocation + val providerId = StateStoreProviderId(StateStoreId(stateCheckpointDir, 0, 0), query.runId) + assert(coordRef.getLocation(providerId).nonEmpty) + + // Stop and verify whether the stores are deactivated in the coordinator + query.stop() + assert(coordRef.getLocation(providerId).isEmpty) + } finally { + SparkSession.getActiveSession.foreach(_.streams.active.foreach(_.stop())) + if (coordRef != null) coordRef.stop() + StateStore.stop() + } + } +} + +object StateStoreCoordinatorSuite { + def withCoordinatorRef(sc: SparkContext)(body: StateStoreCoordinatorRef => Unit): Unit = { + var coordinatorRef: StateStoreCoordinatorRef = null + try { + coordinatorRef = StateStoreCoordinatorRef.forDriver(sc.env) + body(coordinatorRef) + } finally { + if (coordinatorRef != null) coordinatorRef.stop() + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala new file mode 100644 index 000000000000..defb9ed63a88 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming.state + +import java.io.File +import java.nio.file.Files +import java.util.UUID + +import scala.util.Random + +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.ExecutorCacheTaskLocation +import org.apache.spark.sql.LocalSparkSession._ +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.util.quietly +import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.apache.spark.util.{CompletionIterator, Utils} + +class StateStoreRDDSuite extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterAll { + + import StateStoreTestsHelper._ + + private val sparkConf = new SparkConf().setMaster("local").setAppName(this.getClass.getSimpleName) + private val tempDir = Files.createTempDirectory("StateStoreRDDSuite").toString + private val keySchema = StructType(Seq(StructField("key", StringType, true))) + private val valueSchema = StructType(Seq(StructField("value", IntegerType, true))) + + after { + StateStore.stop() + } + + override def afterAll(): Unit = { + super.afterAll() + Utils.deleteRecursively(new File(tempDir)) + } + + test("versioning and immutability") { + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + val path = Utils.createDirectory(tempDir, Random.nextString(10)).toString + val rdd1 = makeRDD(spark.sparkContext, Seq("a", "b", "a")).mapPartitionsWithStateStore( + spark.sqlContext, operatorStateInfo(path, version = 0), keySchema, valueSchema, None)( + increment) + assert(rdd1.collect().toSet === Set("a" -> 2, "b" -> 1)) + + // Generate next version of stores + val rdd2 = makeRDD(spark.sparkContext, Seq("a", "c")).mapPartitionsWithStateStore( + spark.sqlContext, operatorStateInfo(path, version = 1), keySchema, valueSchema, None)( + increment) + assert(rdd2.collect().toSet === Set("a" -> 3, "b" -> 1, "c" -> 1)) + + // Make sure the previous RDD still has the same data. + assert(rdd1.collect().toSet === Set("a" -> 2, "b" -> 1)) + } + } + + test("recovering from files") { + val path = Utils.createDirectory(tempDir, Random.nextString(10)).toString + + def makeStoreRDD( + spark: SparkSession, + seq: Seq[String], + storeVersion: Int): RDD[(String, Int)] = { + implicit val sqlContext = spark.sqlContext + makeRDD(spark.sparkContext, Seq("a")).mapPartitionsWithStateStore( + sqlContext, operatorStateInfo(path, version = storeVersion), + keySchema, valueSchema, None)(increment) + } + + // Generate RDDs and state store data + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + for (i <- 1 to 20) { + require(makeStoreRDD(spark, Seq("a"), i - 1).collect().toSet === Set("a" -> i)) + } + } + + // With a new context, try using the earlier state store data + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + assert(makeStoreRDD(spark, Seq("a"), 20).collect().toSet === Set("a" -> 21)) + } + } + + test("usage with iterators - only gets and only puts") { + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + implicit val sqlContext = spark.sqlContext + val path = Utils.createDirectory(tempDir, Random.nextString(10)).toString + val opId = 0 + + // Returns an iterator of the incremented value made into the store + def iteratorOfPuts(store: StateStore, iter: Iterator[String]): Iterator[(String, Int)] = { + val resIterator = iter.map { s => + val key = stringToRow(s) + val oldValue = Option(store.get(key)).map(rowToInt).getOrElse(0) + val newValue = oldValue + 1 + store.put(key, intToRow(newValue)) + (s, newValue) + } + CompletionIterator[(String, Int), Iterator[(String, Int)]](resIterator, { + store.commit() + }) + } + + def iteratorOfGets( + store: StateStore, + iter: Iterator[String]): Iterator[(String, Option[Int])] = { + iter.map { s => + val key = stringToRow(s) + val value = Option(store.get(key)).map(rowToInt) + (s, value) + } + } + + val rddOfGets1 = makeRDD(spark.sparkContext, Seq("a", "b", "c")).mapPartitionsWithStateStore( + spark.sqlContext, operatorStateInfo(path, version = 0), keySchema, valueSchema, None)( + iteratorOfGets) + assert(rddOfGets1.collect().toSet === Set("a" -> None, "b" -> None, "c" -> None)) + + val rddOfPuts = makeRDD(spark.sparkContext, Seq("a", "b", "a")).mapPartitionsWithStateStore( + sqlContext, operatorStateInfo(path, version = 0), keySchema, valueSchema, None)( + iteratorOfPuts) + assert(rddOfPuts.collect().toSet === Set("a" -> 1, "a" -> 2, "b" -> 1)) + + val rddOfGets2 = makeRDD(spark.sparkContext, Seq("a", "b", "c")).mapPartitionsWithStateStore( + sqlContext, operatorStateInfo(path, version = 1), keySchema, valueSchema, None)( + iteratorOfGets) + assert(rddOfGets2.collect().toSet === Set("a" -> Some(2), "b" -> Some(1), "c" -> None)) + } + } + + test("preferred locations using StateStoreCoordinator") { + quietly { + val queryRunId = UUID.randomUUID + val opId = 0 + val path = Utils.createDirectory(tempDir, Random.nextString(10)).toString + + withSparkSession(SparkSession.builder.config(sparkConf).getOrCreate()) { spark => + implicit val sqlContext = spark.sqlContext + val coordinatorRef = sqlContext.streams.stateStoreCoordinator + val storeProviderId1 = StateStoreProviderId(StateStoreId(path, opId, 0), queryRunId) + val storeProviderId2 = StateStoreProviderId(StateStoreId(path, opId, 1), queryRunId) + coordinatorRef.reportActiveInstance(storeProviderId1, "host1", "exec1") + coordinatorRef.reportActiveInstance(storeProviderId2, "host2", "exec2") + + require( + coordinatorRef.getLocation(storeProviderId1) === + Some(ExecutorCacheTaskLocation("host1", "exec1").toString)) + + val rdd = makeRDD(spark.sparkContext, Seq("a", "b", "a")).mapPartitionsWithStateStore( + sqlContext, operatorStateInfo(path, queryRunId = queryRunId), + keySchema, valueSchema, None)(increment) + require(rdd.partitions.length === 2) + + assert( + rdd.preferredLocations(rdd.partitions(0)) === + Seq(ExecutorCacheTaskLocation("host1", "exec1").toString)) + + assert( + rdd.preferredLocations(rdd.partitions(1)) === + Seq(ExecutorCacheTaskLocation("host2", "exec2").toString)) + + rdd.collect() + } + } + } + + test("distributed test") { + quietly { + + withSparkSession( + SparkSession.builder + .config(sparkConf.setMaster("local-cluster[2, 1, 1024]")) + .getOrCreate()) { spark => + implicit val sqlContext = spark.sqlContext + val path = Utils.createDirectory(tempDir, Random.nextString(10)).toString + val opId = 0 + val rdd1 = makeRDD(spark.sparkContext, Seq("a", "b", "a")).mapPartitionsWithStateStore( + sqlContext, operatorStateInfo(path, version = 0), keySchema, valueSchema, None)(increment) + assert(rdd1.collect().toSet === Set("a" -> 2, "b" -> 1)) + + // Generate next version of stores + val rdd2 = makeRDD(spark.sparkContext, Seq("a", "c")).mapPartitionsWithStateStore( + sqlContext, operatorStateInfo(path, version = 1), keySchema, valueSchema, None)(increment) + assert(rdd2.collect().toSet === Set("a" -> 3, "b" -> 1, "c" -> 1)) + + // Make sure the previous RDD still has the same data. + assert(rdd1.collect().toSet === Set("a" -> 2, "b" -> 1)) + } + } + } + + private def makeRDD(sc: SparkContext, seq: Seq[String]): RDD[String] = { + sc.makeRDD(seq, 2).groupBy(x => x).flatMap(_._2) + } + + private def operatorStateInfo( + path: String, + queryRunId: UUID = UUID.randomUUID, + version: Int = 0): StatefulOperatorStateInfo = { + StatefulOperatorStateInfo(path, queryRunId, operatorId = 0, version) + } + + private val increment = (store: StateStore, iter: Iterator[String]) => { + iter.foreach { s => + val key = stringToRow(s) + val oldValue = Option(store.get(key)).map(rowToInt).getOrElse(0) + store.put(key, intToRow(oldValue + 1)) + } + store.commit() + store.iterator().map(rowsToStringInt) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala new file mode 100644 index 000000000000..c843b65020d8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -0,0 +1,801 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming.state + +import java.io.{File, IOException} +import java.net.URI +import java.util.UUID +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util.Random + +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem} +import org.scalatest.{BeforeAndAfter, PrivateMethodTester} +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkFunSuite} +import org.apache.spark.LocalSparkContext._ +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.util.quietly +import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingQueryWrapper} +import org.apache.spark.sql.functions.count +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.util.Utils + +class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] + with BeforeAndAfter with PrivateMethodTester { + type MapType = mutable.HashMap[UnsafeRow, UnsafeRow] + + import StateStoreCoordinatorSuite._ + import StateStoreTestsHelper._ + + val keySchema = StructType(Seq(StructField("key", StringType, true))) + val valueSchema = StructType(Seq(StructField("value", IntegerType, true))) + + before { + StateStore.stop() + require(!StateStore.isMaintenanceRunning) + } + + after { + StateStore.stop() + require(!StateStore.isMaintenanceRunning) + } + + test("snapshotting") { + val provider = newStoreProvider(opId = Random.nextInt, partition = 0, minDeltasForSnapshot = 5) + + var currentVersion = 0 + def updateVersionTo(targetVersion: Int): Unit = { + for (i <- currentVersion + 1 to targetVersion) { + val store = provider.getStore(currentVersion) + put(store, "a", i) + store.commit() + currentVersion += 1 + } + require(currentVersion === targetVersion) + } + + updateVersionTo(2) + require(getData(provider) === Set("a" -> 2)) + provider.doMaintenance() // should not generate snapshot files + assert(getData(provider) === Set("a" -> 2)) + + for (i <- 1 to currentVersion) { + assert(fileExists(provider, i, isSnapshot = false)) // all delta files present + assert(!fileExists(provider, i, isSnapshot = true)) // no snapshot files present + } + + // After version 6, snapshotting should generate one snapshot file + updateVersionTo(6) + require(getData(provider) === Set("a" -> 6), "store not updated correctly") + provider.doMaintenance() // should generate snapshot files + + val snapshotVersion = (0 to 6).find(version => fileExists(provider, version, isSnapshot = true)) + assert(snapshotVersion.nonEmpty, "snapshot file not generated") + deleteFilesEarlierThanVersion(provider, snapshotVersion.get) + assert( + getData(provider, snapshotVersion.get) === Set("a" -> snapshotVersion.get), + "snapshotting messed up the data of the snapshotted version") + assert( + getData(provider) === Set("a" -> 6), + "snapshotting messed up the data of the final version") + + // After version 20, snapshotting should generate newer snapshot files + updateVersionTo(20) + require(getData(provider) === Set("a" -> 20), "store not updated correctly") + provider.doMaintenance() // do snapshot + + val latestSnapshotVersion = (0 to 20).filter(version => + fileExists(provider, version, isSnapshot = true)).lastOption + assert(latestSnapshotVersion.nonEmpty, "no snapshot file found") + assert(latestSnapshotVersion.get > snapshotVersion.get, "newer snapshot not generated") + + deleteFilesEarlierThanVersion(provider, latestSnapshotVersion.get) + assert(getData(provider) === Set("a" -> 20), "snapshotting messed up the data") + } + + test("cleaning") { + val provider = newStoreProvider(opId = Random.nextInt, partition = 0, minDeltasForSnapshot = 5) + + for (i <- 1 to 20) { + val store = provider.getStore(i - 1) + put(store, "a", i) + store.commit() + provider.doMaintenance() // do cleanup + } + require( + rowsToSet(provider.latestIterator()) === Set("a" -> 20), + "store not updated correctly") + + assert(!fileExists(provider, version = 1, isSnapshot = false)) // first file should be deleted + + // last couple of versions should be retrievable + assert(getData(provider, 20) === Set("a" -> 20)) + assert(getData(provider, 19) === Set("a" -> 19)) + } + + test("SPARK-19677: Committing a delta file atop an existing one should not fail on HDFS") { + val conf = new Configuration() + conf.set("fs.fake.impl", classOf[RenameLikeHDFSFileSystem].getName) + conf.set("fs.defaultFS", "fake:///") + + val provider = newStoreProvider(opId = Random.nextInt, partition = 0, hadoopConf = conf) + provider.getStore(0).commit() + provider.getStore(0).commit() + + // Verify we don't leak temp files + val tempFiles = FileUtils.listFiles(new File(provider.stateStoreId.checkpointRootLocation), + null, true).asScala.filter(_.getName.startsWith("temp-")) + assert(tempFiles.isEmpty) + } + + test("corrupted file handling") { + val provider = newStoreProvider(opId = Random.nextInt, partition = 0, minDeltasForSnapshot = 5) + for (i <- 1 to 6) { + val store = provider.getStore(i - 1) + put(store, "a", i) + store.commit() + provider.doMaintenance() // do cleanup + } + val snapshotVersion = (0 to 10).find( version => + fileExists(provider, version, isSnapshot = true)).getOrElse(fail("snapshot file not found")) + + // Corrupt snapshot file and verify that it throws error + assert(getData(provider, snapshotVersion) === Set("a" -> snapshotVersion)) + corruptFile(provider, snapshotVersion, isSnapshot = true) + intercept[Exception] { + getData(provider, snapshotVersion) + } + + // Corrupt delta file and verify that it throws error + assert(getData(provider, snapshotVersion - 1) === Set("a" -> (snapshotVersion - 1))) + corruptFile(provider, snapshotVersion - 1, isSnapshot = false) + intercept[Exception] { + getData(provider, snapshotVersion - 1) + } + + // Delete delta file and verify that it throws error + deleteFilesEarlierThanVersion(provider, snapshotVersion) + intercept[Exception] { + getData(provider, snapshotVersion - 1) + } + } + + test("reports memory usage") { + val provider = newStoreProvider() + val store = provider.getStore(0) + val noDataMemoryUsed = store.metrics.memoryUsedBytes + put(store, "a", 1) + store.commit() + assert(store.metrics.memoryUsedBytes > noDataMemoryUsed) + } + + test("StateStore.get") { + quietly { + val dir = newDir() + val storeId = StateStoreProviderId(StateStoreId(dir, 0, 0), UUID.randomUUID) + val storeConf = StateStoreConf.empty + val hadoopConf = new Configuration() + + // Verify that trying to get incorrect versions throw errors + intercept[IllegalArgumentException] { + StateStore.get( + storeId, keySchema, valueSchema, None, -1, storeConf, hadoopConf) + } + assert(!StateStore.isLoaded(storeId)) // version -1 should not attempt to load the store + + intercept[IllegalStateException] { + StateStore.get( + storeId, keySchema, valueSchema, None, 1, storeConf, hadoopConf) + } + + // Increase version of the store and try to get again + val store0 = StateStore.get( + storeId, keySchema, valueSchema, None, 0, storeConf, hadoopConf) + assert(store0.version === 0) + put(store0, "a", 1) + store0.commit() + + val store1 = StateStore.get( + storeId, keySchema, valueSchema, None, 1, storeConf, hadoopConf) + assert(StateStore.isLoaded(storeId)) + assert(store1.version === 1) + assert(rowsToSet(store1.iterator()) === Set("a" -> 1)) + + // Verify that you can also load older version + val store0reloaded = StateStore.get( + storeId, keySchema, valueSchema, None, 0, storeConf, hadoopConf) + assert(store0reloaded.version === 0) + assert(rowsToSet(store0reloaded.iterator()) === Set.empty) + + // Verify that you can remove the store and still reload and use it + StateStore.unload(storeId) + assert(!StateStore.isLoaded(storeId)) + + val store1reloaded = StateStore.get( + storeId, keySchema, valueSchema, None, 1, storeConf, hadoopConf) + assert(StateStore.isLoaded(storeId)) + assert(store1reloaded.version === 1) + put(store1reloaded, "a", 2) + assert(store1reloaded.commit() === 2) + assert(rowsToSet(store1reloaded.iterator()) === Set("a" -> 2)) + } + } + + test("maintenance") { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + // Make maintenance thread do snapshots and cleanups very fast + .set(StateStore.MAINTENANCE_INTERVAL_CONFIG, "10ms") + // Make sure that when SparkContext stops, the StateStore maintenance thread 'quickly' + // fails to talk to the StateStoreCoordinator and unloads all the StateStores + .set("spark.rpc.numRetries", "1") + val opId = 0 + val dir = newDir() + val storeProviderId = StateStoreProviderId(StateStoreId(dir, opId, 0), UUID.randomUUID) + val sqlConf = new SQLConf() + sqlConf.setConf(SQLConf.MIN_BATCHES_TO_RETAIN, 2) + val storeConf = StateStoreConf(sqlConf) + val hadoopConf = new Configuration() + val provider = newStoreProvider(storeProviderId.storeId) + + var latestStoreVersion = 0 + + def generateStoreVersions() { + for (i <- 1 to 20) { + val store = StateStore.get(storeProviderId, keySchema, valueSchema, None, + latestStoreVersion, storeConf, hadoopConf) + put(store, "a", i) + store.commit() + latestStoreVersion += 1 + } + } + + val timeoutDuration = 60 seconds + + quietly { + withSpark(new SparkContext(conf)) { sc => + withCoordinatorRef(sc) { coordinatorRef => + require(!StateStore.isMaintenanceRunning, "StateStore is unexpectedly running") + + // Generate sufficient versions of store for snapshots + generateStoreVersions() + + eventually(timeout(timeoutDuration)) { + // Store should have been reported to the coordinator + assert(coordinatorRef.getLocation(storeProviderId).nonEmpty, + "active instance was not reported") + + // Background maintenance should clean up and generate snapshots + assert(StateStore.isMaintenanceRunning, "Maintenance task is not running") + + // Some snapshots should have been generated + val snapshotVersions = (1 to latestStoreVersion).filter { version => + fileExists(provider, version, isSnapshot = true) + } + assert(snapshotVersions.nonEmpty, "no snapshot file found") + } + + // Generate more versions such that there is another snapshot and + // the earliest delta file will be cleaned up + generateStoreVersions() + + // Earliest delta file should get cleaned up + eventually(timeout(timeoutDuration)) { + assert(!fileExists(provider, 1, isSnapshot = false), "earliest file not deleted") + } + + // If driver decides to deactivate all stores related to a query run, + // then this instance should be unloaded + coordinatorRef.deactivateInstances(storeProviderId.queryRunId) + eventually(timeout(timeoutDuration)) { + assert(!StateStore.isLoaded(storeProviderId)) + } + + // Reload the store and verify + StateStore.get(storeProviderId, keySchema, valueSchema, indexOrdinal = None, + latestStoreVersion, storeConf, hadoopConf) + assert(StateStore.isLoaded(storeProviderId)) + + // If some other executor loads the store, then this instance should be unloaded + coordinatorRef.reportActiveInstance(storeProviderId, "other-host", "other-exec") + eventually(timeout(timeoutDuration)) { + assert(!StateStore.isLoaded(storeProviderId)) + } + + // Reload the store and verify + StateStore.get(storeProviderId, keySchema, valueSchema, indexOrdinal = None, + latestStoreVersion, storeConf, hadoopConf) + assert(StateStore.isLoaded(storeProviderId)) + } + } + + // Verify if instance is unloaded if SparkContext is stopped + eventually(timeout(timeoutDuration)) { + require(SparkEnv.get === null) + assert(!StateStore.isLoaded(storeProviderId)) + assert(!StateStore.isMaintenanceRunning) + } + } + } + + test("SPARK-18342: commit fails when rename fails") { + import RenameReturnsFalseFileSystem._ + val dir = scheme + "://" + newDir() + val conf = new Configuration() + conf.set(s"fs.$scheme.impl", classOf[RenameReturnsFalseFileSystem].getName) + val provider = newStoreProvider( + opId = Random.nextInt, partition = 0, dir = dir, hadoopConf = conf) + val store = provider.getStore(0) + put(store, "a", 0) + val e = intercept[IllegalStateException](store.commit()) + assert(e.getCause.getMessage.contains("Failed to rename")) + } + + test("SPARK-18416: do not create temp delta file until the store is updated") { + val dir = newDir() + val storeId = StateStoreProviderId(StateStoreId(dir, 0, 0), UUID.randomUUID) + val storeConf = StateStoreConf.empty + val hadoopConf = new Configuration() + val deltaFileDir = new File(s"$dir/0/0/") + + def numTempFiles: Int = { + if (deltaFileDir.exists) { + deltaFileDir.listFiles.map(_.getName).count(n => n.contains("temp") && !n.startsWith(".")) + } else 0 + } + + def numDeltaFiles: Int = { + if (deltaFileDir.exists) { + deltaFileDir.listFiles.map(_.getName).count(n => n.contains(".delta") && !n.startsWith(".")) + } else 0 + } + + def shouldNotCreateTempFile[T](body: => T): T = { + val before = numTempFiles + val result = body + assert(numTempFiles === before) + result + } + + // Getting the store should not create temp file + val store0 = shouldNotCreateTempFile { + StateStore.get( + storeId, keySchema, valueSchema, indexOrdinal = None, version = 0, storeConf, hadoopConf) + } + + // Put should create a temp file + put(store0, "a", 1) + assert(numTempFiles === 1) + assert(numDeltaFiles === 0) + + // Commit should remove temp file and create a delta file + store0.commit() + assert(numTempFiles === 0) + assert(numDeltaFiles === 1) + + // Remove should create a temp file + val store1 = shouldNotCreateTempFile { + StateStore.get( + storeId, keySchema, valueSchema, indexOrdinal = None, version = 1, storeConf, hadoopConf) + } + remove(store1, _ == "a") + assert(numTempFiles === 1) + assert(numDeltaFiles === 1) + + // Commit should remove temp file and create a delta file + store1.commit() + assert(numTempFiles === 0) + assert(numDeltaFiles === 2) + + // Commit without any updates should create a delta file + val store2 = shouldNotCreateTempFile { + StateStore.get( + storeId, keySchema, valueSchema, indexOrdinal = None, version = 2, storeConf, hadoopConf) + } + store2.commit() + assert(numTempFiles === 0) + assert(numDeltaFiles === 3) + } + + test("SPARK-21145: Restarted queries create new provider instances") { + try { + val checkpointLocation = Utils.createTempDir().getAbsoluteFile + val spark = SparkSession.builder().master("local[2]").getOrCreate() + SparkSession.setActiveSession(spark) + implicit val sqlContext = spark.sqlContext + spark.conf.set("spark.sql.shuffle.partitions", "1") + import spark.implicits._ + val inputData = MemoryStream[Int] + + def runQueryAndGetLoadedProviders(): Seq[StateStoreProvider] = { + val aggregated = inputData.toDF().groupBy("value").agg(count("*")) + // stateful query + val query = aggregated.writeStream + .format("memory") + .outputMode("complete") + .queryName("query") + .option("checkpointLocation", checkpointLocation.toString) + .start() + inputData.addData(1, 2, 3) + query.processAllAvailable() + require(query.lastProgress != null) // at least one batch processed after start + val loadedProvidersMethod = + PrivateMethod[mutable.HashMap[StateStoreProviderId, StateStoreProvider]]('loadedProviders) + val loadedProvidersMap = StateStore invokePrivate loadedProvidersMethod() + val loadedProviders = loadedProvidersMap.synchronized { loadedProvidersMap.values.toSeq } + query.stop() + loadedProviders + } + + val loadedProvidersAfterRun1 = runQueryAndGetLoadedProviders() + require(loadedProvidersAfterRun1.length === 1) + + val loadedProvidersAfterRun2 = runQueryAndGetLoadedProviders() + assert(loadedProvidersAfterRun2.length === 2) // two providers loaded for 2 runs + + // Both providers should have the same StateStoreId, but the should be different objects + assert(loadedProvidersAfterRun2(0).stateStoreId === loadedProvidersAfterRun2(1).stateStoreId) + assert(loadedProvidersAfterRun2(0) ne loadedProvidersAfterRun2(1)) + + } finally { + SparkSession.getActiveSession.foreach { spark => + spark.streams.active.foreach(_.stop()) + spark.stop() + } + } + } + + override def newStoreProvider(): HDFSBackedStateStoreProvider = { + newStoreProvider(opId = Random.nextInt(), partition = 0) + } + + override def newStoreProvider(storeId: StateStoreId): HDFSBackedStateStoreProvider = { + newStoreProvider(storeId.operatorId, storeId.partitionId, dir = storeId.checkpointRootLocation) + } + + override def getLatestData(storeProvider: HDFSBackedStateStoreProvider): Set[(String, Int)] = { + getData(storeProvider) + } + + override def getData( + provider: HDFSBackedStateStoreProvider, + version: Int = -1): Set[(String, Int)] = { + val reloadedProvider = newStoreProvider(provider.stateStoreId) + if (version < 0) { + reloadedProvider.latestIterator().map(rowsToStringInt).toSet + } else { + reloadedProvider.getStore(version).iterator().map(rowsToStringInt).toSet + } + } + + def newStoreProvider( + opId: Long, + partition: Int, + dir: String = newDir(), + minDeltasForSnapshot: Int = SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT.defaultValue.get, + hadoopConf: Configuration = new Configuration): HDFSBackedStateStoreProvider = { + val sqlConf = new SQLConf() + sqlConf.setConf(SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT, minDeltasForSnapshot) + sqlConf.setConf(SQLConf.MIN_BATCHES_TO_RETAIN, 2) + val provider = new HDFSBackedStateStoreProvider() + provider.init( + StateStoreId(dir, opId, partition), + keySchema, + valueSchema, + indexOrdinal = None, + new StateStoreConf(sqlConf), + hadoopConf) + provider + } + + def fileExists( + provider: HDFSBackedStateStoreProvider, + version: Long, + isSnapshot: Boolean): Boolean = { + val method = PrivateMethod[Path]('baseDir) + val basePath = provider invokePrivate method() + val fileName = if (isSnapshot) s"$version.snapshot" else s"$version.delta" + val filePath = new File(basePath.toString, fileName) + filePath.exists + } + + def deleteFilesEarlierThanVersion(provider: HDFSBackedStateStoreProvider, version: Long): Unit = { + val method = PrivateMethod[Path]('baseDir) + val basePath = provider invokePrivate method() + for (version <- 0 until version.toInt) { + for (isSnapshot <- Seq(false, true)) { + val fileName = if (isSnapshot) s"$version.snapshot" else s"$version.delta" + val filePath = new File(basePath.toString, fileName) + if (filePath.exists) filePath.delete() + } + } + } + + def corruptFile( + provider: HDFSBackedStateStoreProvider, + version: Long, + isSnapshot: Boolean): Unit = { + val method = PrivateMethod[Path]('baseDir) + val basePath = provider invokePrivate method() + val fileName = if (isSnapshot) s"$version.snapshot" else s"$version.delta" + val filePath = new File(basePath.toString, fileName) + filePath.delete() + filePath.createNewFile() + } +} + +abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] + extends SparkFunSuite { + import StateStoreTestsHelper._ + + test("get, put, remove, commit, and all data iterator") { + val provider = newStoreProvider() + + // Verify state before starting a new set of updates + assert(getLatestData(provider).isEmpty) + + val store = provider.getStore(0) + assert(!store.hasCommitted) + assert(get(store, "a") === None) + assert(store.iterator().isEmpty) + assert(store.metrics.numKeys === 0) + + // Verify state after updating + put(store, "a", 1) + assert(get(store, "a") === Some(1)) + assert(store.metrics.numKeys === 1) + + assert(store.iterator().nonEmpty) + assert(getLatestData(provider).isEmpty) + + // Make updates, commit and then verify state + put(store, "b", 2) + put(store, "aa", 3) + assert(store.metrics.numKeys === 3) + remove(store, _.startsWith("a")) + assert(store.metrics.numKeys === 1) + assert(store.commit() === 1) + + assert(store.hasCommitted) + assert(rowsToSet(store.iterator()) === Set("b" -> 2)) + assert(getLatestData(provider) === Set("b" -> 2)) + + // Trying to get newer versions should fail + intercept[Exception] { + provider.getStore(2) + } + intercept[Exception] { + getData(provider, 2) + } + + // New updates to the reloaded store with new version, and does not change old version + val reloadedProvider = newStoreProvider(store.id) + val reloadedStore = reloadedProvider.getStore(1) + assert(reloadedStore.metrics.numKeys === 1) + put(reloadedStore, "c", 4) + assert(reloadedStore.metrics.numKeys === 2) + assert(reloadedStore.commit() === 2) + assert(rowsToSet(reloadedStore.iterator()) === Set("b" -> 2, "c" -> 4)) + assert(getLatestData(provider) === Set("b" -> 2, "c" -> 4)) + assert(getData(provider, version = 1) === Set("b" -> 2)) + } + + test("removing while iterating") { + val provider = newStoreProvider() + + // Verify state before starting a new set of updates + assert(getLatestData(provider).isEmpty) + val store = provider.getStore(0) + put(store, "a", 1) + put(store, "b", 2) + + // Updates should work while iterating of filtered entries + val filtered = store.iterator.filter { tuple => rowToString(tuple.key) == "a" } + filtered.foreach { tuple => + store.put(tuple.key, intToRow(rowToInt(tuple.value) + 1)) + } + assert(get(store, "a") === Some(2)) + + // Removes should work while iterating of filtered entries + val filtered2 = store.iterator.filter { tuple => rowToString(tuple.key) == "b" } + filtered2.foreach { tuple => store.remove(tuple.key) } + assert(get(store, "b") === None) + } + + test("abort") { + val provider = newStoreProvider() + val store = provider.getStore(0) + put(store, "a", 1) + store.commit() + assert(rowsToSet(store.iterator()) === Set("a" -> 1)) + + // cancelUpdates should not change the data in the files + val store1 = provider.getStore(1) + put(store1, "b", 1) + store1.abort() + } + + test("getStore with invalid versions") { + val provider = newStoreProvider() + + def checkInvalidVersion(version: Int): Unit = { + intercept[Exception] { + provider.getStore(version) + } + } + + checkInvalidVersion(-1) + checkInvalidVersion(1) + + val store = provider.getStore(0) + put(store, "a", 1) + assert(store.commit() === 1) + assert(rowsToSet(store.iterator()) === Set("a" -> 1)) + + val store1_ = provider.getStore(1) + assert(rowsToSet(store1_.iterator()) === Set("a" -> 1)) + + checkInvalidVersion(-1) + checkInvalidVersion(2) + + // Update store version with some data + val store1 = provider.getStore(1) + assert(rowsToSet(store1.iterator()) === Set("a" -> 1)) + put(store1, "b", 1) + assert(store1.commit() === 2) + assert(rowsToSet(store1.iterator()) === Set("a" -> 1, "b" -> 1)) + + checkInvalidVersion(-1) + checkInvalidVersion(3) + } + + test("two concurrent StateStores - one for read-only and one for read-write") { + // During Streaming Aggregation, we have two StateStores per task, one used as read-only in + // `StateStoreRestoreExec`, and one read-write used in `StateStoreSaveExec`. `StateStore.abort` + // will be called for these StateStores if they haven't committed their results. We need to + // make sure that `abort` in read-only store after a `commit` in the read-write store doesn't + // accidentally lead to the deletion of state. + val dir = newDir() + val storeId = StateStoreId(dir, 0L, 1) + val provider0 = newStoreProvider(storeId) + // prime state + val store = provider0.getStore(0) + val key = "a" + put(store, key, 1) + store.commit() + assert(rowsToSet(store.iterator()) === Set(key -> 1)) + + // two state stores + val provider1 = newStoreProvider(storeId) + val restoreStore = provider1.getStore(1) + val saveStore = provider1.getStore(1) + + put(saveStore, key, get(restoreStore, key).get + 1) + saveStore.commit() + restoreStore.abort() + + // check that state is correct for next batch + val provider2 = newStoreProvider(storeId) + val finalStore = provider2.getStore(2) + assert(rowsToSet(finalStore.iterator()) === Set(key -> 2)) + } + + /** Return a new provider with a random id */ + def newStoreProvider(): ProviderClass + + /** Return a new provider with the given id */ + def newStoreProvider(storeId: StateStoreId): ProviderClass + + /** Get the latest data referred to by the given provider but not using this provider */ + def getLatestData(storeProvider: ProviderClass): Set[(String, Int)] + + /** + * Get a specific version of data referred to by the given provider but not using + * this provider + */ + def getData(storeProvider: ProviderClass, version: Int): Set[(String, Int)] +} + +object StateStoreTestsHelper { + + val strProj = UnsafeProjection.create(Array[DataType](StringType)) + val intProj = UnsafeProjection.create(Array[DataType](IntegerType)) + + def stringToRow(s: String): UnsafeRow = { + strProj.apply(new GenericInternalRow(Array[Any](UTF8String.fromString(s)))).copy() + } + + def intToRow(i: Int): UnsafeRow = { + intProj.apply(new GenericInternalRow(Array[Any](i))).copy() + } + + def rowToString(row: UnsafeRow): String = { + row.getUTF8String(0).toString + } + + def rowToInt(row: UnsafeRow): Int = { + row.getInt(0) + } + + def rowsToStringInt(row: UnsafeRowPair): (String, Int) = { + (rowToString(row.key), rowToInt(row.value)) + } + + def rowsToSet(iterator: Iterator[UnsafeRowPair]): Set[(String, Int)] = { + iterator.map(rowsToStringInt).toSet + } + + def remove(store: StateStore, condition: String => Boolean): Unit = { + store.getRange(None, None).foreach { rowPair => + if (condition(rowToString(rowPair.key))) store.remove(rowPair.key) + } + } + + def put(store: StateStore, key: String, value: Int): Unit = { + store.put(stringToRow(key), intToRow(value)) + } + + def get(store: StateStore, key: String): Option[Int] = { + Option(store.get(stringToRow(key))).map(rowToInt) + } + + def newDir(): String = Utils.createTempDir().toString +} + +/** + * Fake FileSystem that simulates HDFS rename semantic, i.e. renaming a file atop an existing + * one should return false. + * See hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-common/filesystem/filesystem.html + */ +class RenameLikeHDFSFileSystem extends RawLocalFileSystem { + override def rename(src: Path, dst: Path): Boolean = { + if (exists(dst)) { + return false + } else { + return super.rename(src, dst) + } + } +} + +/** + * Fake FileSystem to test that the StateStore throws an exception while committing the + * delta file, when `fs.rename` returns `false`. + */ +class RenameReturnsFalseFileSystem extends RawLocalFileSystem { + import RenameReturnsFalseFileSystem._ + override def getUri: URI = { + URI.create(s"$scheme:///") + } + + override def rename(src: Path, dst: Path): Boolean = false +} + +object RenameReturnsFalseFileSystem { + val scheme = s"StateStoreSuite${math.abs(Random.nextInt)}fs" +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManagerSuite.scala new file mode 100644 index 000000000000..ffa4c3c22a19 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManagerSuite.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.sql.execution.streaming.state + +import java.util.UUID + +import org.apache.hadoop.conf.Configuration +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Expression, GenericInternalRow, LessThanOrEqual, Literal, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate +import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark +import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo +import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.LeftSide +import org.apache.spark.sql.streaming.StreamTest +import org.apache.spark.sql.types._ + +class SymmetricHashJoinStateManagerSuite extends StreamTest with BeforeAndAfter { + + before { + SparkSession.setActiveSession(spark) // set this before force initializing 'joinExec' + spark.streams.stateStoreCoordinator // initialize the lazy coordinator + } + + + test("SymmetricHashJoinStateManager - all operations") { + withJoinStateManager(inputValueAttribs, joinKeyExprs) { manager => + implicit val mgr = manager + + assert(get(20) === Seq.empty) // initially empty + append(20, 2) + assert(get(20) === Seq(2)) // should first value correctly + assert(numRows === 1) + + append(20, 3) + assert(get(20) === Seq(2, 3)) // should append new values + append(20, 3) + assert(get(20) === Seq(2, 3, 3)) // should append another copy if same value added again + assert(numRows === 3) + + assert(get(30) === Seq.empty) + append(30, 1) + assert(get(30) === Seq(1)) + assert(get(20) === Seq(2, 3, 3)) // add another key-value should not affect existing ones + assert(numRows === 4) + + removeByKey(25) + assert(get(20) === Seq.empty) + assert(get(30) === Seq(1)) // should remove 20, not 30 + assert(numRows === 1) + + removeByKey(30) + assert(get(30) === Seq.empty) // should remove 30 + assert(numRows === 0) + + def appendAndTest(key: Int, values: Int*): Unit = { + values.foreach { value => append(key, value)} + require(get(key) === values) + } + + appendAndTest(40, 100, 200, 300) + appendAndTest(50, 125) + appendAndTest(60, 275) // prepare for testing removeByValue + assert(numRows === 5) + + removeByValue(125) + assert(get(40) === Seq(200, 300)) + assert(get(50) === Seq.empty) + assert(get(60) === Seq(275)) // should remove only some values, not all + assert(numRows === 3) + + append(40, 50) + assert(get(40) === Seq(50, 200, 300)) + assert(numRows === 4) + + removeByValue(200) + assert(get(40) === Seq(300)) + assert(get(60) === Seq(275)) // should remove only some values, not all + assert(numRows === 2) + + removeByValue(300) + assert(get(40) === Seq.empty) + assert(get(60) === Seq.empty) // should remove all values now + assert(numRows === 0) + } + } + val watermarkMetadata = new MetadataBuilder().putLong(EventTimeWatermark.delayKey, 10).build() + val inputValueSchema = new StructType() + .add(StructField("time", IntegerType, metadata = watermarkMetadata)) + .add(StructField("value", BooleanType)) + val inputValueAttribs = inputValueSchema.toAttributes + val inputValueAttribWithWatermark = inputValueAttribs(0) + val joinKeyExprs = Seq[Expression](Literal(false), inputValueAttribWithWatermark, Literal(10.0)) + + val inputValueGen = UnsafeProjection.create(inputValueAttribs.map(_.dataType).toArray) + val joinKeyGen = UnsafeProjection.create(joinKeyExprs.map(_.dataType).toArray) + + + def toInputValue(i: Int): UnsafeRow = { + inputValueGen.apply(new GenericInternalRow(Array[Any](i, false))) + } + + def toJoinKeyRow(i: Int): UnsafeRow = { + joinKeyGen.apply(new GenericInternalRow(Array[Any](false, i, 10.0))) + } + + def toValueInt(inputValueRow: UnsafeRow): Int = inputValueRow.getInt(0) + + def append(key: Int, value: Int)(implicit manager: SymmetricHashJoinStateManager): Unit = { + manager.append(toJoinKeyRow(key), toInputValue(value)) + } + + def get(key: Int)(implicit manager: SymmetricHashJoinStateManager): Seq[Int] = { + manager.get(toJoinKeyRow(key)).map(toValueInt).toSeq.sorted + } + + /** Remove keys (and corresponding values) where `time <= threshold` */ + def removeByKey(threshold: Long)(implicit manager: SymmetricHashJoinStateManager): Unit = { + val expr = + LessThanOrEqual( + BoundReference( + 1, inputValueAttribWithWatermark.dataType, inputValueAttribWithWatermark.nullable), + Literal(threshold)) + manager.removeByKeyCondition(GeneratePredicate.generate(expr).eval _) + } + + /** Remove values where `time <= threshold` */ + def removeByValue(watermark: Long)(implicit manager: SymmetricHashJoinStateManager): Unit = { + val expr = LessThanOrEqual(inputValueAttribWithWatermark, Literal(watermark)) + manager.removeByValueCondition( + GeneratePredicate.generate(expr, inputValueAttribs).eval _) + } + + def numRows(implicit manager: SymmetricHashJoinStateManager): Long = { + manager.metrics.numKeys + } + + + def withJoinStateManager( + inputValueAttribs: Seq[Attribute], + joinKeyExprs: Seq[Expression])(f: SymmetricHashJoinStateManager => Unit): Unit = { + + withTempDir { file => + val storeConf = new StateStoreConf() + val stateInfo = StatefulOperatorStateInfo(file.getAbsolutePath, UUID.randomUUID, 0, 0) + val manager = new SymmetricHashJoinStateManager( + LeftSide, inputValueAttribs, joinKeyExprs, Some(stateInfo), storeConf, new Configuration) + try { + f(manager) + } finally { + manager.abortIfNeeded() + } + } + StateStore.stop() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index c15aac775096..1055f09f5411 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -19,16 +19,30 @@ package org.apache.spark.sql.execution.ui import java.util.Properties -import org.apache.spark.{SparkException, SparkContext, SparkConf, SparkFunSuite} +import org.json4s.jackson.JsonMethods._ +import org.mockito.Mockito.mock + +import org.apache.spark._ +import org.apache.spark.LocalSparkContext._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.sql.execution.metric.LongSQLMetricValue +import org.apache.spark.internal.config +import org.apache.spark.rdd.RDD import org.apache.spark.scheduler._ -import org.apache.spark.sql.{DataFrame, SQLContext} -import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.catalyst.util.quietly +import org.apache.spark.sql.execution.{LeafExecNode, QueryExecution, SparkPlanInfo, SQLExecution} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.ui.SparkUI +import org.apache.spark.util.{AccumulatorMetadata, JsonProtocol, LongAccumulator} + -class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { +class SQLListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTestUtils { import testImplicits._ + import org.apache.spark.AccumulatorSuite.makeInfo private def createTestDataFrame: DataFrame = { Seq( @@ -67,22 +81,36 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { ) private def createTaskMetrics(accumulatorUpdates: Map[Long, Long]): TaskMetrics = { - val metrics = new TaskMetrics - metrics.setAccumulatorsUpdater(() => accumulatorUpdates.mapValues(new LongSQLMetricValue(_))) - metrics.updateAccumulators() + val metrics = TaskMetrics.empty + accumulatorUpdates.foreach { case (id, update) => + val acc = new LongAccumulator + acc.metadata = AccumulatorMetadata(id, Some(""), true) + acc.add(update) + metrics.registerAccumulator(acc) + } metrics } test("basic") { def checkAnswer(actual: Map[Long, String], expected: Map[Long, Long]): Unit = { - assert(actual === expected.mapValues(_.toString)) + assert(actual.size == expected.size) + expected.foreach { e => + // The values in actual can be SQL metrics meaning that they contain additional formatting + // when converted to string. Verify that they start with the expected value. + // TODO: this is brittle. There is no requirement that the actual string needs to start + // with the accumulator value. + assert(actual.contains(e._1)) + val v = actual.get(e._1).get.trim + assert(v.startsWith(e._2.toString)) + } } - val listener = new SQLListener(sqlContext.sparkContext.conf) + val listener = new SQLListener(spark.sparkContext.conf) val executionId = 0 val df = createTestDataFrame val accumulatorIds = - SparkPlanGraph(df.queryExecution.executedPlan).nodes.flatMap(_.metrics.map(_.accumulatorId)) + SparkPlanGraph(SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan)) + .allNodes.flatMap(_.metrics.map(_.accumulatorId)) // Assume all accumulators are long var accumulatorValue = 0L val accumulatorUpdates = accumulatorIds.map { id => @@ -90,13 +118,13 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { (id, accumulatorValue) }.toMap - listener.onExecutionStart( + listener.onOtherEvent(SparkListenerSQLExecutionStart( executionId, "test", "test", df.queryExecution.toString, - SparkPlanGraph(df.queryExecution.executedPlan), - System.currentTimeMillis()) + SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), + System.currentTimeMillis())) val executionUIData = listener.executionIdToData(0) @@ -113,17 +141,23 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { assert(listener.getExecutionMetrics(0).isEmpty) listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( - // (task id, stage id, stage attempt, metrics) - (0L, 0, 0, createTaskMetrics(accumulatorUpdates)), - (1L, 0, 0, createTaskMetrics(accumulatorUpdates)) + // (task id, stage id, stage attempt, accum updates) + (0L, 0, 0, createTaskMetrics(accumulatorUpdates).accumulators().map(makeInfo)), + (1L, 0, 0, createTaskMetrics(accumulatorUpdates).accumulators().map(makeInfo)) ))) checkAnswer(listener.getExecutionMetrics(0), accumulatorUpdates.mapValues(_ * 2)) + // Driver accumulator updates don't belong to this execution should be filtered and no + // exception will be thrown. + listener.onOtherEvent(SparkListenerDriverAccumUpdates(0, Seq((999L, 2L)))) + checkAnswer(listener.getExecutionMetrics(0), accumulatorUpdates.mapValues(_ * 2)) + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( - // (task id, stage id, stage attempt, metrics) - (0L, 0, 0, createTaskMetrics(accumulatorUpdates)), - (1L, 0, 0, createTaskMetrics(accumulatorUpdates.mapValues(_ * 2))) + // (task id, stage id, stage attempt, accum updates) + (0L, 0, 0, createTaskMetrics(accumulatorUpdates).accumulators().map(makeInfo)), + (1L, 0, 0, + createTaskMetrics(accumulatorUpdates.mapValues(_ * 2)).accumulators().map(makeInfo)) ))) checkAnswer(listener.getExecutionMetrics(0), accumulatorUpdates.mapValues(_ * 3)) @@ -132,9 +166,9 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(0, 1))) listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( - // (task id, stage id, stage attempt, metrics) - (0L, 0, 1, createTaskMetrics(accumulatorUpdates)), - (1L, 0, 1, createTaskMetrics(accumulatorUpdates)) + // (task id, stage id, stage attempt, accum updates) + (0L, 0, 1, createTaskMetrics(accumulatorUpdates).accumulators().map(makeInfo)), + (1L, 0, 1, createTaskMetrics(accumulatorUpdates).accumulators().map(makeInfo)) ))) checkAnswer(listener.getExecutionMetrics(0), accumulatorUpdates.mapValues(_ * 2)) @@ -172,9 +206,9 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(1, 0))) listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( - // (task id, stage id, stage attempt, metrics) - (0L, 1, 0, createTaskMetrics(accumulatorUpdates)), - (1L, 1, 0, createTaskMetrics(accumulatorUpdates)) + // (task id, stage id, stage attempt, accum updates) + (0L, 1, 0, createTaskMetrics(accumulatorUpdates).accumulators().map(makeInfo)), + (1L, 1, 0, createTaskMetrics(accumulatorUpdates).accumulators().map(makeInfo)) ))) checkAnswer(listener.getExecutionMetrics(0), accumulatorUpdates.mapValues(_ * 7)) @@ -206,7 +240,8 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { time = System.currentTimeMillis(), JobSucceeded )) - listener.onExecutionEnd(executionId, System.currentTimeMillis()) + listener.onOtherEvent(SparkListenerSQLExecutionEnd( + executionId, System.currentTimeMillis())) assert(executionUIData.runningJobs.isEmpty) assert(executionUIData.succeededJobs === Seq(0)) @@ -216,22 +251,23 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { } test("onExecutionEnd happens before onJobEnd(JobSucceeded)") { - val listener = new SQLListener(sqlContext.sparkContext.conf) + val listener = new SQLListener(spark.sparkContext.conf) val executionId = 0 val df = createTestDataFrame - listener.onExecutionStart( + listener.onOtherEvent(SparkListenerSQLExecutionStart( executionId, "test", "test", df.queryExecution.toString, - SparkPlanGraph(df.queryExecution.executedPlan), - System.currentTimeMillis()) + SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), + System.currentTimeMillis())) listener.onJobStart(SparkListenerJobStart( jobId = 0, time = System.currentTimeMillis(), stageInfos = Nil, createProperties(executionId))) - listener.onExecutionEnd(executionId, System.currentTimeMillis()) + listener.onOtherEvent(SparkListenerSQLExecutionEnd( + executionId, System.currentTimeMillis())) listener.onJobEnd(SparkListenerJobEnd( jobId = 0, time = System.currentTimeMillis(), @@ -245,16 +281,16 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { } test("onExecutionEnd happens before multiple onJobEnd(JobSucceeded)s") { - val listener = new SQLListener(sqlContext.sparkContext.conf) + val listener = new SQLListener(spark.sparkContext.conf) val executionId = 0 val df = createTestDataFrame - listener.onExecutionStart( + listener.onOtherEvent(SparkListenerSQLExecutionStart( executionId, "test", "test", df.queryExecution.toString, - SparkPlanGraph(df.queryExecution.executedPlan), - System.currentTimeMillis()) + SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), + System.currentTimeMillis())) listener.onJobStart(SparkListenerJobStart( jobId = 0, time = System.currentTimeMillis(), @@ -271,7 +307,8 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { time = System.currentTimeMillis(), stageInfos = Nil, createProperties(executionId))) - listener.onExecutionEnd(executionId, System.currentTimeMillis()) + listener.onOtherEvent(SparkListenerSQLExecutionEnd( + executionId, System.currentTimeMillis())) listener.onJobEnd(SparkListenerJobEnd( jobId = 1, time = System.currentTimeMillis(), @@ -285,22 +322,23 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { } test("onExecutionEnd happens before onJobEnd(JobFailed)") { - val listener = new SQLListener(sqlContext.sparkContext.conf) + val listener = new SQLListener(spark.sparkContext.conf) val executionId = 0 val df = createTestDataFrame - listener.onExecutionStart( + listener.onOtherEvent(SparkListenerSQLExecutionStart( executionId, "test", "test", df.queryExecution.toString, - SparkPlanGraph(df.queryExecution.executedPlan), - System.currentTimeMillis()) + SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), + System.currentTimeMillis())) listener.onJobStart(SparkListenerJobStart( jobId = 0, time = System.currentTimeMillis(), stageInfos = Seq.empty, createProperties(executionId))) - listener.onExecutionEnd(executionId, System.currentTimeMillis()) + listener.onOtherEvent(SparkListenerSQLExecutionEnd( + executionId, System.currentTimeMillis())) listener.onJobEnd(SparkListenerJobEnd( jobId = 0, time = System.currentTimeMillis(), @@ -314,55 +352,177 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { } test("SPARK-11126: no memory leak when running non SQL jobs") { - val previousStageNumber = sqlContext.listener.stageIdToStageMetrics.size - sqlContext.sparkContext.parallelize(1 to 10).foreach(i => ()) - sqlContext.sparkContext.listenerBus.waitUntilEmpty(10000) + val previousStageNumber = spark.sharedState.listener.stageIdToStageMetrics.size + spark.sparkContext.parallelize(1 to 10).foreach(i => ()) + spark.sparkContext.listenerBus.waitUntilEmpty(10000) // listener should ignore the non SQL stage - assert(sqlContext.listener.stageIdToStageMetrics.size == previousStageNumber) + assert(spark.sharedState.listener.stageIdToStageMetrics.size == previousStageNumber) - sqlContext.sparkContext.parallelize(1 to 10).toDF().foreach(i => ()) - sqlContext.sparkContext.listenerBus.waitUntilEmpty(10000) + spark.sparkContext.parallelize(1 to 10).toDF().foreach(i => ()) + spark.sparkContext.listenerBus.waitUntilEmpty(10000) // listener should save the SQL stage - assert(sqlContext.listener.stageIdToStageMetrics.size == previousStageNumber + 1) + assert(spark.sharedState.listener.stageIdToStageMetrics.size == previousStageNumber + 1) + } + + test("SPARK-13055: history listener only tracks SQL metrics") { + val listener = new SQLHistoryListener(sparkContext.conf, mock(classOf[SparkUI])) + // We need to post other events for the listener to track our accumulators. + // These are largely just boilerplate unrelated to what we're trying to test. + val df = createTestDataFrame + val executionStart = SparkListenerSQLExecutionStart( + 0, "", "", "", SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), 0) + val stageInfo = createStageInfo(0, 0) + val jobStart = SparkListenerJobStart(0, 0, Seq(stageInfo), createProperties(0)) + val stageSubmitted = SparkListenerStageSubmitted(stageInfo) + // This task has both accumulators that are SQL metrics and accumulators that are not. + // The listener should only track the ones that are actually SQL metrics. + val sqlMetric = SQLMetrics.createMetric(sparkContext, "beach umbrella") + val nonSqlMetric = sparkContext.longAccumulator("baseball") + val sqlMetricInfo = sqlMetric.toInfo(Some(sqlMetric.value), None) + val nonSqlMetricInfo = nonSqlMetric.toInfo(Some(nonSqlMetric.value), None) + val taskInfo = createTaskInfo(0, 0) + taskInfo.setAccumulables(List(sqlMetricInfo, nonSqlMetricInfo)) + val taskEnd = SparkListenerTaskEnd(0, 0, "just-a-task", null, taskInfo, null) + listener.onOtherEvent(executionStart) + listener.onJobStart(jobStart) + listener.onStageSubmitted(stageSubmitted) + // Before SPARK-13055, this throws ClassCastException because the history listener would + // assume that the accumulator value is of type Long, but this may not be true for + // accumulators that are not SQL metrics. + listener.onTaskEnd(taskEnd) + val trackedAccums = listener.stageIdToStageMetrics.values.flatMap { stageMetrics => + stageMetrics.taskIdToMetricUpdates.values.flatMap(_.accumulatorUpdates) + } + // Listener tracks only SQL metrics, not other accumulators + assert(trackedAccums.size === 1) + assert(trackedAccums.head === ((sqlMetricInfo.id, sqlMetricInfo.update.get))) + } + + test("driver side SQL metrics") { + val listener = new SQLListener(spark.sparkContext.conf) + val expectedAccumValue = 12345 + val physicalPlan = MyPlan(sqlContext.sparkContext, expectedAccumValue) + sqlContext.sparkContext.addSparkListener(listener) + val dummyQueryExecution = new QueryExecution(spark, LocalRelation()) { + override lazy val sparkPlan = physicalPlan + override lazy val executedPlan = physicalPlan + } + SQLExecution.withNewExecutionId(spark, dummyQueryExecution) { + physicalPlan.execute().collect() + } + + def waitTillExecutionFinished(): Unit = { + while (listener.getCompletedExecutions.isEmpty) { + Thread.sleep(100) + } + } + waitTillExecutionFinished() + + val driverUpdates = listener.getCompletedExecutions.head.driverAccumUpdates + assert(driverUpdates.size == 1) + assert(driverUpdates(physicalPlan.longMetric("dummy").id) == expectedAccumValue) + } + + test("roundtripping SparkListenerDriverAccumUpdates through JsonProtocol (SPARK-18462)") { + val event = SparkListenerDriverAccumUpdates(1L, Seq((2L, 3L))) + val json = JsonProtocol.sparkEventToJson(event) + assertValidDataInJson(json, + parse(""" + |{ + | "Event": "org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates", + | "executionId": 1, + | "accumUpdates": [[2,3]] + |} + """.stripMargin)) + JsonProtocol.sparkEventFromJson(json) match { + case SparkListenerDriverAccumUpdates(executionId, accums) => + assert(executionId == 1L) + accums.foreach { case (a, b) => + assert(a == 2L) + assert(b == 3L) + } + } + + // Test a case where the numbers in the JSON can only fit in longs: + val longJson = parse( + """ + |{ + | "Event": "org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates", + | "executionId": 4294967294, + | "accumUpdates": [[4294967294,3]] + |} + """.stripMargin) + JsonProtocol.sparkEventFromJson(longJson) match { + case SparkListenerDriverAccumUpdates(executionId, accums) => + assert(executionId == 4294967294L) + accums.foreach { case (a, b) => + assert(a == 4294967294L) + assert(b == 3L) + } + } } } + +/** + * A dummy [[org.apache.spark.sql.execution.SparkPlan]] that updates a [[SQLMetrics]] + * on the driver. + */ +private case class MyPlan(sc: SparkContext, expectedValue: Long) extends LeafExecNode { + override def sparkContext: SparkContext = sc + override def output: Seq[Attribute] = Seq() + + override val metrics: Map[String, SQLMetric] = Map( + "dummy" -> SQLMetrics.createMetric(sc, "dummy")) + + override def doExecute(): RDD[InternalRow] = { + longMetric("dummy") += expectedValue + + SQLMetrics.postDriverMetricUpdates( + sc, + sc.getLocalProperty(SQLExecution.EXECUTION_ID_KEY), + metrics.values.toSeq) + sc.emptyRDD + } +} + + class SQLListenerMemoryLeakSuite extends SparkFunSuite { test("no memory leak") { - val conf = new SparkConf() - .setMaster("local") - .setAppName("test") - .set("spark.task.maxFailures", "1") // Don't retry the tasks to run this test quickly - .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run this test quickly - val sc = new SparkContext(conf) - try { - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ - // Run 100 successful executions and 100 failed executions. - // Each execution only has one job and one stage. - for (i <- 0 until 100) { - val df = Seq( - (1, 1), - (2, 2) - ).toDF() - df.collect() - try { - df.foreach(_ => throw new RuntimeException("Oops")) - } catch { - case e: SparkException => // This is expected for a failed job + quietly { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set(config.MAX_TASK_FAILURES, 1) // Don't retry the tasks to run this test quickly + .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run this test quickly + withSpark(new SparkContext(conf)) { sc => + SparkSession.sqlListener.set(null) + val spark = new SparkSession(sc) + import spark.implicits._ + // Run 100 successful executions and 100 failed executions. + // Each execution only has one job and one stage. + for (i <- 0 until 100) { + val df = Seq( + (1, 1), + (2, 2) + ).toDF() + df.collect() + try { + df.foreach(_ => throw new RuntimeException("Oops")) + } catch { + case e: SparkException => // This is expected for a failed job + } } + sc.listenerBus.waitUntilEmpty(10000) + assert(spark.sharedState.listener.getCompletedExecutions.size <= 50) + assert(spark.sharedState.listener.getFailedExecutions.size <= 50) + // 50 for successful executions and 50 for failed executions + assert(spark.sharedState.listener.executionIdToData.size <= 100) + assert(spark.sharedState.listener.jobIdToExecutionId.size <= 100) + assert(spark.sharedState.listener.stageIdToStageMetrics.size <= 100) } - sc.listenerBus.waitUntilEmpty(10000) - assert(sqlContext.listener.getCompletedExecutions.size <= 50) - assert(sqlContext.listener.getFailedExecutions.size <= 50) - // 50 for successful executions and 50 for failed executions - assert(sqlContext.listener.executionIdToData.size <= 100) - assert(sqlContext.listener.jobIdToExecutionId.size <= 100) - assert(sqlContext.listener.stageIdToStageMetrics.size <= 100) - } finally { - sc.stop() } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ArrowColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ArrowColumnVectorSuite.scala new file mode 100644 index 000000000000..d24a9e1f4bd1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ArrowColumnVectorSuite.scala @@ -0,0 +1,410 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.vectorized + +import org.apache.arrow.vector._ +import org.apache.arrow.vector.complex._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.execution.arrow.ArrowUtils +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +class ArrowColumnVectorSuite extends SparkFunSuite { + + test("boolean") { + val allocator = ArrowUtils.rootAllocator.newChildAllocator("boolean", 0, Long.MaxValue) + val vector = ArrowUtils.toArrowField("boolean", BooleanType, nullable = true) + .createVector(allocator).asInstanceOf[NullableBitVector] + vector.allocateNew() + val mutator = vector.getMutator() + + (0 until 10).foreach { i => + mutator.setSafe(i, if (i % 2 == 0) 1 else 0) + } + mutator.setNull(10) + mutator.setValueCount(11) + + val columnVector = new ArrowColumnVector(vector) + assert(columnVector.dataType === BooleanType) + assert(columnVector.anyNullsSet) + assert(columnVector.numNulls === 1) + + (0 until 10).foreach { i => + assert(columnVector.getBoolean(i) === (i % 2 == 0)) + } + assert(columnVector.isNullAt(10)) + + assert(columnVector.getBooleans(0, 10) === (0 until 10).map(i => (i % 2 == 0))) + + columnVector.close() + allocator.close() + } + + test("byte") { + val allocator = ArrowUtils.rootAllocator.newChildAllocator("byte", 0, Long.MaxValue) + val vector = ArrowUtils.toArrowField("byte", ByteType, nullable = true) + .createVector(allocator).asInstanceOf[NullableTinyIntVector] + vector.allocateNew() + val mutator = vector.getMutator() + + (0 until 10).foreach { i => + mutator.setSafe(i, i.toByte) + } + mutator.setNull(10) + mutator.setValueCount(11) + + val columnVector = new ArrowColumnVector(vector) + assert(columnVector.dataType === ByteType) + assert(columnVector.anyNullsSet) + assert(columnVector.numNulls === 1) + + (0 until 10).foreach { i => + assert(columnVector.getByte(i) === i.toByte) + } + assert(columnVector.isNullAt(10)) + + assert(columnVector.getBytes(0, 10) === (0 until 10).map(i => i.toByte)) + + columnVector.close() + allocator.close() + } + + test("short") { + val allocator = ArrowUtils.rootAllocator.newChildAllocator("short", 0, Long.MaxValue) + val vector = ArrowUtils.toArrowField("short", ShortType, nullable = true) + .createVector(allocator).asInstanceOf[NullableSmallIntVector] + vector.allocateNew() + val mutator = vector.getMutator() + + (0 until 10).foreach { i => + mutator.setSafe(i, i.toShort) + } + mutator.setNull(10) + mutator.setValueCount(11) + + val columnVector = new ArrowColumnVector(vector) + assert(columnVector.dataType === ShortType) + assert(columnVector.anyNullsSet) + assert(columnVector.numNulls === 1) + + (0 until 10).foreach { i => + assert(columnVector.getShort(i) === i.toShort) + } + assert(columnVector.isNullAt(10)) + + assert(columnVector.getShorts(0, 10) === (0 until 10).map(i => i.toShort)) + + columnVector.close() + allocator.close() + } + + test("int") { + val allocator = ArrowUtils.rootAllocator.newChildAllocator("int", 0, Long.MaxValue) + val vector = ArrowUtils.toArrowField("int", IntegerType, nullable = true) + .createVector(allocator).asInstanceOf[NullableIntVector] + vector.allocateNew() + val mutator = vector.getMutator() + + (0 until 10).foreach { i => + mutator.setSafe(i, i) + } + mutator.setNull(10) + mutator.setValueCount(11) + + val columnVector = new ArrowColumnVector(vector) + assert(columnVector.dataType === IntegerType) + assert(columnVector.anyNullsSet) + assert(columnVector.numNulls === 1) + + (0 until 10).foreach { i => + assert(columnVector.getInt(i) === i) + } + assert(columnVector.isNullAt(10)) + + assert(columnVector.getInts(0, 10) === (0 until 10)) + + columnVector.close() + allocator.close() + } + + test("long") { + val allocator = ArrowUtils.rootAllocator.newChildAllocator("long", 0, Long.MaxValue) + val vector = ArrowUtils.toArrowField("long", LongType, nullable = true) + .createVector(allocator).asInstanceOf[NullableBigIntVector] + vector.allocateNew() + val mutator = vector.getMutator() + + (0 until 10).foreach { i => + mutator.setSafe(i, i.toLong) + } + mutator.setNull(10) + mutator.setValueCount(11) + + val columnVector = new ArrowColumnVector(vector) + assert(columnVector.dataType === LongType) + assert(columnVector.anyNullsSet) + assert(columnVector.numNulls === 1) + + (0 until 10).foreach { i => + assert(columnVector.getLong(i) === i.toLong) + } + assert(columnVector.isNullAt(10)) + + assert(columnVector.getLongs(0, 10) === (0 until 10).map(i => i.toLong)) + + columnVector.close() + allocator.close() + } + + test("float") { + val allocator = ArrowUtils.rootAllocator.newChildAllocator("float", 0, Long.MaxValue) + val vector = ArrowUtils.toArrowField("float", FloatType, nullable = true) + .createVector(allocator).asInstanceOf[NullableFloat4Vector] + vector.allocateNew() + val mutator = vector.getMutator() + + (0 until 10).foreach { i => + mutator.setSafe(i, i.toFloat) + } + mutator.setNull(10) + mutator.setValueCount(11) + + val columnVector = new ArrowColumnVector(vector) + assert(columnVector.dataType === FloatType) + assert(columnVector.anyNullsSet) + assert(columnVector.numNulls === 1) + + (0 until 10).foreach { i => + assert(columnVector.getFloat(i) === i.toFloat) + } + assert(columnVector.isNullAt(10)) + + assert(columnVector.getFloats(0, 10) === (0 until 10).map(i => i.toFloat)) + + columnVector.close() + allocator.close() + } + + test("double") { + val allocator = ArrowUtils.rootAllocator.newChildAllocator("double", 0, Long.MaxValue) + val vector = ArrowUtils.toArrowField("double", DoubleType, nullable = true) + .createVector(allocator).asInstanceOf[NullableFloat8Vector] + vector.allocateNew() + val mutator = vector.getMutator() + + (0 until 10).foreach { i => + mutator.setSafe(i, i.toDouble) + } + mutator.setNull(10) + mutator.setValueCount(11) + + val columnVector = new ArrowColumnVector(vector) + assert(columnVector.dataType === DoubleType) + assert(columnVector.anyNullsSet) + assert(columnVector.numNulls === 1) + + (0 until 10).foreach { i => + assert(columnVector.getDouble(i) === i.toDouble) + } + assert(columnVector.isNullAt(10)) + + assert(columnVector.getDoubles(0, 10) === (0 until 10).map(i => i.toDouble)) + + columnVector.close() + allocator.close() + } + + test("string") { + val allocator = ArrowUtils.rootAllocator.newChildAllocator("string", 0, Long.MaxValue) + val vector = ArrowUtils.toArrowField("string", StringType, nullable = true) + .createVector(allocator).asInstanceOf[NullableVarCharVector] + vector.allocateNew() + val mutator = vector.getMutator() + + (0 until 10).foreach { i => + val utf8 = s"str$i".getBytes("utf8") + mutator.setSafe(i, utf8, 0, utf8.length) + } + mutator.setNull(10) + mutator.setValueCount(11) + + val columnVector = new ArrowColumnVector(vector) + assert(columnVector.dataType === StringType) + assert(columnVector.anyNullsSet) + assert(columnVector.numNulls === 1) + + (0 until 10).foreach { i => + assert(columnVector.getUTF8String(i) === UTF8String.fromString(s"str$i")) + } + assert(columnVector.isNullAt(10)) + + columnVector.close() + allocator.close() + } + + test("binary") { + val allocator = ArrowUtils.rootAllocator.newChildAllocator("binary", 0, Long.MaxValue) + val vector = ArrowUtils.toArrowField("binary", BinaryType, nullable = true) + .createVector(allocator).asInstanceOf[NullableVarBinaryVector] + vector.allocateNew() + val mutator = vector.getMutator() + + (0 until 10).foreach { i => + val utf8 = s"str$i".getBytes("utf8") + mutator.setSafe(i, utf8, 0, utf8.length) + } + mutator.setNull(10) + mutator.setValueCount(11) + + val columnVector = new ArrowColumnVector(vector) + assert(columnVector.dataType === BinaryType) + assert(columnVector.anyNullsSet) + assert(columnVector.numNulls === 1) + + (0 until 10).foreach { i => + assert(columnVector.getBinary(i) === s"str$i".getBytes("utf8")) + } + assert(columnVector.isNullAt(10)) + + columnVector.close() + allocator.close() + } + + test("array") { + val allocator = ArrowUtils.rootAllocator.newChildAllocator("array", 0, Long.MaxValue) + val vector = ArrowUtils.toArrowField("array", ArrayType(IntegerType), nullable = true) + .createVector(allocator).asInstanceOf[ListVector] + vector.allocateNew() + val mutator = vector.getMutator() + val elementVector = vector.getDataVector().asInstanceOf[NullableIntVector] + val elementMutator = elementVector.getMutator() + + // [1, 2] + mutator.startNewValue(0) + elementMutator.setSafe(0, 1) + elementMutator.setSafe(1, 2) + mutator.endValue(0, 2) + + // [3, null, 5] + mutator.startNewValue(1) + elementMutator.setSafe(2, 3) + elementMutator.setNull(3) + elementMutator.setSafe(4, 5) + mutator.endValue(1, 3) + + // null + + // [] + mutator.startNewValue(3) + mutator.endValue(3, 0) + + elementMutator.setValueCount(5) + mutator.setValueCount(4) + + val columnVector = new ArrowColumnVector(vector) + assert(columnVector.dataType === ArrayType(IntegerType)) + assert(columnVector.anyNullsSet) + assert(columnVector.numNulls === 1) + + val array0 = columnVector.getArray(0) + assert(array0.numElements() === 2) + assert(array0.getInt(0) === 1) + assert(array0.getInt(1) === 2) + + val array1 = columnVector.getArray(1) + assert(array1.numElements() === 3) + assert(array1.getInt(0) === 3) + assert(array1.isNullAt(1)) + assert(array1.getInt(2) === 5) + + assert(columnVector.isNullAt(2)) + + val array3 = columnVector.getArray(3) + assert(array3.numElements() === 0) + + columnVector.close() + allocator.close() + } + + test("struct") { + val allocator = ArrowUtils.rootAllocator.newChildAllocator("struct", 0, Long.MaxValue) + val schema = new StructType().add("int", IntegerType).add("long", LongType) + val vector = ArrowUtils.toArrowField("struct", schema, nullable = true) + .createVector(allocator).asInstanceOf[NullableMapVector] + vector.allocateNew() + val mutator = vector.getMutator() + val intVector = vector.getChildByOrdinal(0).asInstanceOf[NullableIntVector] + val intMutator = intVector.getMutator() + val longVector = vector.getChildByOrdinal(1).asInstanceOf[NullableBigIntVector] + val longMutator = longVector.getMutator() + + // (1, 1L) + mutator.setIndexDefined(0) + intMutator.setSafe(0, 1) + longMutator.setSafe(0, 1L) + + // (2, null) + mutator.setIndexDefined(1) + intMutator.setSafe(1, 2) + longMutator.setNull(1) + + // (null, 3L) + mutator.setIndexDefined(2) + intMutator.setNull(2) + longMutator.setSafe(2, 3L) + + // null + mutator.setNull(3) + + // (5, 5L) + mutator.setIndexDefined(4) + intMutator.setSafe(4, 5) + longMutator.setSafe(4, 5L) + + intMutator.setValueCount(5) + longMutator.setValueCount(5) + mutator.setValueCount(5) + + val columnVector = new ArrowColumnVector(vector) + assert(columnVector.dataType === schema) + assert(columnVector.anyNullsSet) + assert(columnVector.numNulls === 1) + + val row0 = columnVector.getStruct(0, 2) + assert(row0.getInt(0) === 1) + assert(row0.getLong(1) === 1L) + + val row1 = columnVector.getStruct(1, 2) + assert(row1.getInt(0) === 2) + assert(row1.isNullAt(1)) + + val row2 = columnVector.getStruct(2, 2) + assert(row2.isNullAt(0)) + assert(row2.getLong(1) === 3L) + + assert(columnVector.isNullAt(3)) + + val row4 = columnVector.getStruct(4, 2) + assert(row4.getInt(0) === 5) + assert(row4.getLong(1) === 5L) + + columnVector.close() + allocator.close() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala new file mode 100644 index 000000000000..f7b06c97f9db --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala @@ -0,0 +1,227 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.vectorized + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { + + var testVector: WritableColumnVector = _ + + private def allocate(capacity: Int, dt: DataType): WritableColumnVector = { + new OnHeapColumnVector(capacity, dt) + } + + override def afterEach(): Unit = { + testVector.close() + } + + test("boolean") { + testVector = allocate(10, BooleanType) + (0 until 10).foreach { i => + testVector.appendBoolean(i % 2 == 0) + } + + val array = new ColumnVector.Array(testVector) + + (0 until 10).foreach { i => + assert(array.get(i, BooleanType) === (i % 2 == 0)) + } + } + + test("byte") { + testVector = allocate(10, ByteType) + (0 until 10).foreach { i => + testVector.appendByte(i.toByte) + } + + val array = new ColumnVector.Array(testVector) + + (0 until 10).foreach { i => + assert(array.get(i, ByteType) === (i.toByte)) + } + } + + test("short") { + testVector = allocate(10, ShortType) + (0 until 10).foreach { i => + testVector.appendShort(i.toShort) + } + + val array = new ColumnVector.Array(testVector) + + (0 until 10).foreach { i => + assert(array.get(i, ShortType) === (i.toShort)) + } + } + + test("int") { + testVector = allocate(10, IntegerType) + (0 until 10).foreach { i => + testVector.appendInt(i) + } + + val array = new ColumnVector.Array(testVector) + + (0 until 10).foreach { i => + assert(array.get(i, IntegerType) === i) + } + } + + test("long") { + testVector = allocate(10, LongType) + (0 until 10).foreach { i => + testVector.appendLong(i) + } + + val array = new ColumnVector.Array(testVector) + + (0 until 10).foreach { i => + assert(array.get(i, LongType) === i) + } + } + + test("float") { + testVector = allocate(10, FloatType) + (0 until 10).foreach { i => + testVector.appendFloat(i.toFloat) + } + + val array = new ColumnVector.Array(testVector) + + (0 until 10).foreach { i => + assert(array.get(i, FloatType) === i.toFloat) + } + } + + test("double") { + testVector = allocate(10, DoubleType) + (0 until 10).foreach { i => + testVector.appendDouble(i.toDouble) + } + + val array = new ColumnVector.Array(testVector) + + (0 until 10).foreach { i => + assert(array.get(i, DoubleType) === i.toDouble) + } + } + + test("string") { + testVector = allocate(10, StringType) + (0 until 10).map { i => + val utf8 = s"str$i".getBytes("utf8") + testVector.appendByteArray(utf8, 0, utf8.length) + } + + val array = new ColumnVector.Array(testVector) + + (0 until 10).foreach { i => + assert(array.get(i, StringType) === UTF8String.fromString(s"str$i")) + } + } + + test("binary") { + testVector = allocate(10, BinaryType) + (0 until 10).map { i => + val utf8 = s"str$i".getBytes("utf8") + testVector.appendByteArray(utf8, 0, utf8.length) + } + + val array = new ColumnVector.Array(testVector) + + (0 until 10).foreach { i => + val utf8 = s"str$i".getBytes("utf8") + assert(array.get(i, BinaryType) === utf8) + } + } + + test("array") { + val arrayType = ArrayType(IntegerType, true) + testVector = allocate(10, arrayType) + + val data = testVector.arrayData() + var i = 0 + while (i < 6) { + data.putInt(i, i) + i += 1 + } + + // Populate it with arrays [0], [1, 2], [], [3, 4, 5] + testVector.putArray(0, 0, 1) + testVector.putArray(1, 1, 2) + testVector.putArray(2, 3, 0) + testVector.putArray(3, 3, 3) + + val array = new ColumnVector.Array(testVector) + + assert(array.get(0, arrayType).asInstanceOf[ArrayData].toIntArray() === Array(0)) + assert(array.get(1, arrayType).asInstanceOf[ArrayData].toIntArray() === Array(1, 2)) + assert(array.get(2, arrayType).asInstanceOf[ArrayData].toIntArray() === Array.empty[Int]) + assert(array.get(3, arrayType).asInstanceOf[ArrayData].toIntArray() === Array(3, 4, 5)) + } + + test("struct") { + val schema = new StructType().add("int", IntegerType).add("double", DoubleType) + testVector = allocate(10, schema) + val c1 = testVector.getChildColumn(0) + val c2 = testVector.getChildColumn(1) + c1.putInt(0, 123) + c2.putDouble(0, 3.45) + c1.putInt(1, 456) + c2.putDouble(1, 5.67) + + val array = new ColumnVector.Array(testVector) + + assert(array.get(0, schema).asInstanceOf[ColumnarBatch.Row].get(0, IntegerType) === 123) + assert(array.get(0, schema).asInstanceOf[ColumnarBatch.Row].get(1, DoubleType) === 3.45) + assert(array.get(1, schema).asInstanceOf[ColumnarBatch.Row].get(0, IntegerType) === 456) + assert(array.get(1, schema).asInstanceOf[ColumnarBatch.Row].get(1, DoubleType) === 5.67) + } + + test("[SPARK-22092] off-heap column vector reallocation corrupts array data") { + val arrayType = ArrayType(IntegerType, true) + testVector = new OffHeapColumnVector(8, arrayType) + + val data = testVector.arrayData() + (0 until 8).foreach(i => data.putInt(i, i)) + (0 until 8).foreach(i => testVector.putArray(i, i, 1)) + + // Increase vector's capacity and reallocate the data to new bigger buffers. + testVector.reserve(16) + + // Check that none of the values got lost/overwritten. + val array = new ColumnVector.Array(testVector) + (0 until 8).foreach { i => + assert(array.get(i, arrayType).asInstanceOf[ArrayData].toIntArray() === Array(i)) + } + } + + test("[SPARK-22092] off-heap column vector reallocation corrupts struct nullability") { + val structType = new StructType().add("int", IntegerType).add("double", DoubleType) + testVector = new OffHeapColumnVector(8, structType) + (0 until 8).foreach(i => if (i % 2 == 0) testVector.putNull(i) else testVector.putNotNull(i)) + testVector.reserve(16) + (0 until 8).foreach(i => assert(testVector.isNullAt(i) == (i % 2 == 0))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala new file mode 100644 index 000000000000..1331f157363b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala @@ -0,0 +1,410 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets + +import scala.util.Random + +import org.apache.spark.memory.MemoryMode +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector +import org.apache.spark.sql.execution.vectorized.WritableColumnVector +import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType} +import org.apache.spark.unsafe.Platform +import org.apache.spark.util.Benchmark +import org.apache.spark.util.collection.BitSet + +/** + * Benchmark to low level memory access using different ways to manage buffers. + */ +object ColumnarBatchBenchmark { + + def allocate(capacity: Int, dt: DataType, memMode: MemoryMode): WritableColumnVector = { + if (memMode == MemoryMode.OFF_HEAP) { + new OffHeapColumnVector(capacity, dt) + } else { + new OnHeapColumnVector(capacity, dt) + } + } + + // This benchmark reads and writes an array of ints. + // TODO: there is a big (2x) penalty for a random access API for off heap. + // Note: carefully if modifying this code. It's hard to reason about the JIT. + def intAccess(iters: Long): Unit = { + val count = 8 * 1000 + + // Accessing a java array. + val javaArray = { i: Int => + val data = new Array[Int](count) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + data(i) = i + i += 1 + } + i = 0 + while (i < count) { + sum += data(i) + i += 1 + } + } + } + + // Accessing ByteBuffers + val byteBufferUnsafe = { i: Int => + val data = ByteBuffer.allocate(count * 4) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + Platform.putInt(data.array(), Platform.BYTE_ARRAY_OFFSET + i * 4, i) + i += 1 + } + i = 0 + while (i < count) { + sum += Platform.getInt(data.array(), Platform.BYTE_ARRAY_OFFSET + i * 4) + i += 1 + } + } + } + + // Accessing offheap byte buffers + val directByteBuffer = { i: Int => + val data = ByteBuffer.allocateDirect(count * 4).asIntBuffer() + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + data.put(i) + i += 1 + } + data.rewind() + i = 0 + while (i < count) { + sum += data.get() + i += 1 + } + data.rewind() + } + } + + // Accessing ByteBuffer using the typed APIs + val byteBufferApi = { i: Int => + val data = ByteBuffer.allocate(count * 4) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + data.putInt(i) + i += 1 + } + data.rewind() + i = 0 + while (i < count) { + sum += data.getInt() + i += 1 + } + data.rewind() + } + } + + // Using unsafe memory + val unsafeBuffer = { i: Int => + val data: Long = Platform.allocateMemory(count * 4) + var sum = 0L + for (n <- 0L until iters) { + var ptr = data + var i = 0 + while (i < count) { + Platform.putInt(null, ptr, i) + ptr += 4 + i += 1 + } + ptr = data + i = 0 + while (i < count) { + sum += Platform.getInt(null, ptr) + ptr += 4 + i += 1 + } + } + } + + // Access through the column API with on heap memory + val columnOnHeap = { i: Int => + val col = allocate(count, IntegerType, MemoryMode.ON_HEAP) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + col.putInt(i, i) + i += 1 + } + i = 0 + while (i < count) { + sum += col.getInt(i) + i += 1 + } + } + col.close + } + + // Access through the column API with off heap memory + def columnOffHeap = { i: Int => { + val col = allocate(count, IntegerType, MemoryMode.OFF_HEAP) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + col.putInt(i, i) + i += 1 + } + i = 0 + while (i < count) { + sum += col.getInt(i) + i += 1 + } + } + col.close + }} + + // Access by directly getting the buffer backing the column. + val columnOffheapDirect = { i: Int => + val col = allocate(count, IntegerType, MemoryMode.OFF_HEAP) + var sum = 0L + for (n <- 0L until iters) { + var addr = col.valuesNativeAddress() + var i = 0 + while (i < count) { + Platform.putInt(null, addr, i) + addr += 4 + i += 1 + } + i = 0 + addr = col.valuesNativeAddress() + while (i < count) { + sum += Platform.getInt(null, addr) + addr += 4 + i += 1 + } + } + col.close + } + + // Access by going through a batch of unsafe rows. + val unsafeRowOnheap = { i: Int => + val buffer = new Array[Byte](count * 16) + var sum = 0L + for (n <- 0L until iters) { + val row = new UnsafeRow(1) + var i = 0 + while (i < count) { + row.pointTo(buffer, Platform.BYTE_ARRAY_OFFSET + i * 16, 16) + row.setInt(0, i) + i += 1 + } + i = 0 + while (i < count) { + row.pointTo(buffer, Platform.BYTE_ARRAY_OFFSET + i * 16, 16) + sum += row.getInt(0) + i += 1 + } + } + } + + // Access by going through a batch of unsafe rows. + val unsafeRowOffheap = { i: Int => + val buffer = Platform.allocateMemory(count * 16) + var sum = 0L + for (n <- 0L until iters) { + val row = new UnsafeRow(1) + var i = 0 + while (i < count) { + row.pointTo(null, buffer + i * 16, 16) + row.setInt(0, i) + i += 1 + } + i = 0 + while (i < count) { + row.pointTo(null, buffer + i * 16, 16) + sum += row.getInt(0) + i += 1 + } + } + Platform.freeMemory(buffer) + } + + // Adding values by appending, instead of putting. + val onHeapAppend = { i: Int => + val col = allocate(count, IntegerType, MemoryMode.ON_HEAP) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + col.appendInt(i) + i += 1 + } + i = 0 + while (i < count) { + sum += col.getInt(i) + i += 1 + } + col.reset() + } + col.close + } + + /* + Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz + Int Read/Write: Avg Time(ms) Avg Rate(M/s) Relative Rate + ------------------------------------------------------------------------- + Java Array 248.8 1317.04 1.00 X + ByteBuffer Unsafe 435.6 752.25 0.57 X + ByteBuffer API 1752.0 187.03 0.14 X + DirectByteBuffer 595.4 550.35 0.42 X + Unsafe Buffer 235.2 1393.20 1.06 X + Column(on heap) 189.8 1726.45 1.31 X + Column(off heap) 408.4 802.35 0.61 X + Column(off heap direct) 237.6 1379.12 1.05 X + UnsafeRow (on heap) 414.6 790.35 0.60 X + UnsafeRow (off heap) 487.2 672.58 0.51 X + Column On Heap Append 530.1 618.14 0.59 X + */ + val benchmark = new Benchmark("Int Read/Write", count * iters) + benchmark.addCase("Java Array")(javaArray) + benchmark.addCase("ByteBuffer Unsafe")(byteBufferUnsafe) + benchmark.addCase("ByteBuffer API")(byteBufferApi) + benchmark.addCase("DirectByteBuffer")(directByteBuffer) + benchmark.addCase("Unsafe Buffer")(unsafeBuffer) + benchmark.addCase("Column(on heap)")(columnOnHeap) + benchmark.addCase("Column(off heap)")(columnOffHeap) + benchmark.addCase("Column(off heap direct)")(columnOffheapDirect) + benchmark.addCase("UnsafeRow (on heap)")(unsafeRowOnheap) + benchmark.addCase("UnsafeRow (off heap)")(unsafeRowOffheap) + benchmark.addCase("Column On Heap Append")(onHeapAppend) + benchmark.run() + } + + def booleanAccess(iters: Int): Unit = { + val count = 8 * 1024 + val benchmark = new Benchmark("Boolean Read/Write", iters * count) + benchmark.addCase("Bitset") { i: Int => { + val b = new BitSet(count) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + if (i % 2 == 0) b.set(i) + i += 1 + } + i = 0 + while (i < count) { + if (b.get(i)) sum += 1 + i += 1 + } + } + }} + + benchmark.addCase("Byte Array") { i: Int => { + val b = new Array[Byte](count) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + if (i % 2 == 0) b(i) = 1; + i += 1 + } + i = 0 + while (i < count) { + if (b(i) == 1) sum += 1 + i += 1 + } + } + }} + /* + Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz + Boolean Read/Write: Avg Time(ms) Avg Rate(M/s) Relative Rate + ------------------------------------------------------------------------- + Bitset 895.88 374.54 1.00 X + Byte Array 578.96 579.56 1.55 X + */ + benchmark.run() + } + + def stringAccess(iters: Long): Unit = { + val chars = "0123456789ABCDEFGHIJKLMNOPQRSTUVWXYZ"; + val random = new Random(0) + + def randomString(min: Int, max: Int): String = { + val len = random.nextInt(max - min) + min + val sb = new StringBuilder(len) + var i = 0 + while (i < len) { + sb.append(chars.charAt(random.nextInt(chars.length()))); + i += 1 + } + return sb.toString + } + + val minString = 3 + val maxString = 32 + val count = 4 * 1000 + + val data = Seq.fill(count)(randomString(minString, maxString)) + .map(_.getBytes(StandardCharsets.UTF_8)).toArray + + def column(memoryMode: MemoryMode) = { i: Int => + val column = allocate(count, BinaryType, memoryMode) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + column.putByteArray(i, data(i)) + i += 1 + } + i = 0 + while (i < count) { + sum += column.getUTF8String(i).numBytes() + i += 1 + } + column.reset() + } + } + + /* + String Read/Write: Avg Time(ms) Avg Rate(M/s) Relative Rate + ------------------------------------------------------------------------------------- + On Heap 457.0 35.85 1.00 X + Off Heap 1206.0 13.59 0.38 X + */ + val benchmark = new Benchmark("String Read/Write", count * iters) + benchmark.addCase("On Heap")(column(MemoryMode.ON_HEAP)) + benchmark.addCase("Off Heap")(column(MemoryMode.OFF_HEAP)) + benchmark.run + } + + def main(args: Array[String]): Unit = { + intAccess(1024 * 40) + booleanAccess(1024 * 40) + stringAccess(1024 * 4) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala new file mode 100644 index 000000000000..ebf76613343b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -0,0 +1,1314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.vectorized + +import java.nio.ByteBuffer +import java.nio.ByteOrder +import java.nio.charset.StandardCharsets + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util.Random + +import org.apache.arrow.vector.NullableIntVector + +import org.apache.spark.SparkFunSuite +import org.apache.spark.memory.MemoryMode +import org.apache.spark.sql.{RandomDataGenerator, Row} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.arrow.ArrowUtils +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.Platform +import org.apache.spark.unsafe.types.CalendarInterval + +class ColumnarBatchSuite extends SparkFunSuite { + + def allocate(capacity: Int, dt: DataType, memMode: MemoryMode): WritableColumnVector = { + if (memMode == MemoryMode.OFF_HEAP) { + new OffHeapColumnVector(capacity, dt) + } else { + new OnHeapColumnVector(capacity, dt) + } + } + + test("Null Apis") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + val reference = mutable.ArrayBuffer.empty[Boolean] + + val column = allocate(1024, IntegerType, memMode) + var idx = 0 + assert(column.anyNullsSet() == false) + assert(column.numNulls() == 0) + + column.appendNotNull() + reference += false + assert(column.anyNullsSet() == false) + assert(column.numNulls() == 0) + + column.appendNotNulls(3) + (1 to 3).foreach(_ => reference += false) + assert(column.anyNullsSet() == false) + assert(column.numNulls() == 0) + + column.appendNull() + reference += true + assert(column.anyNullsSet()) + assert(column.numNulls() == 1) + + column.appendNulls(3) + (1 to 3).foreach(_ => reference += true) + assert(column.anyNullsSet()) + assert(column.numNulls() == 4) + + idx = column.elementsAppended + + column.putNotNull(idx) + reference += false + idx += 1 + assert(column.anyNullsSet()) + assert(column.numNulls() == 4) + + column.putNull(idx) + reference += true + idx += 1 + assert(column.anyNullsSet()) + assert(column.numNulls() == 5) + + column.putNulls(idx, 3) + reference += true + reference += true + reference += true + idx += 3 + assert(column.anyNullsSet()) + assert(column.numNulls() == 8) + + column.putNotNulls(idx, 4) + reference += false + reference += false + reference += false + reference += false + idx += 4 + assert(column.anyNullsSet()) + assert(column.numNulls() == 8) + + reference.zipWithIndex.foreach { v => + assert(v._1 == column.isNullAt(v._2)) + if (memMode == MemoryMode.OFF_HEAP) { + val addr = column.nullsNativeAddress() + assert(v._1 == (Platform.getByte(null, addr + v._2) == 1), "index=" + v._2) + } + } + column.close + }} + } + + test("Byte Apis") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + val reference = mutable.ArrayBuffer.empty[Byte] + + val column = allocate(1024, ByteType, memMode) + + var values = (10 :: 20 :: 30 :: 40 :: 50 :: Nil).map(_.toByte).toArray + column.appendBytes(2, values, 0) + reference += 10.toByte + reference += 20.toByte + + column.appendBytes(3, values, 2) + reference += 30.toByte + reference += 40.toByte + reference += 50.toByte + + column.appendBytes(6, 60.toByte) + (1 to 6).foreach(_ => reference += 60.toByte) + + column.appendByte(70.toByte) + reference += 70.toByte + + var idx = column.elementsAppended + + values = (1 :: 2 :: 3 :: 4 :: 5 :: Nil).map(_.toByte).toArray + column.putBytes(idx, 2, values, 0) + reference += 1 + reference += 2 + idx += 2 + + column.putBytes(idx, 3, values, 2) + reference += 3 + reference += 4 + reference += 5 + idx += 3 + + column.putByte(idx, 9) + reference += 9 + idx += 1 + + column.putBytes(idx, 3, 4) + reference += 4 + reference += 4 + reference += 4 + idx += 3 + + reference.zipWithIndex.foreach { v => + assert(v._1 == column.getByte(v._2), "MemoryMode" + memMode) + if (memMode == MemoryMode.OFF_HEAP) { + val addr = column.valuesNativeAddress() + assert(v._1 == Platform.getByte(null, addr + v._2)) + } + } + }} + } + + test("Short Apis") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + val seed = System.currentTimeMillis() + val random = new Random(seed) + val reference = mutable.ArrayBuffer.empty[Short] + + val column = allocate(1024, ShortType, memMode) + + var values = (10 :: 20 :: 30 :: 40 :: 50 :: Nil).map(_.toShort).toArray + column.appendShorts(2, values, 0) + reference += 10.toShort + reference += 20.toShort + + column.appendShorts(3, values, 2) + reference += 30.toShort + reference += 40.toShort + reference += 50.toShort + + column.appendShorts(6, 60.toShort) + (1 to 6).foreach(_ => reference += 60.toShort) + + column.appendShort(70.toShort) + reference += 70.toShort + + var idx = column.elementsAppended + + values = (1 :: 2 :: 3 :: 4 :: 5 :: Nil).map(_.toShort).toArray + column.putShorts(idx, 2, values, 0) + reference += 1 + reference += 2 + idx += 2 + + column.putShorts(idx, 3, values, 2) + reference += 3 + reference += 4 + reference += 5 + idx += 3 + + column.putShort(idx, 9) + reference += 9 + idx += 1 + + column.putShorts(idx, 3, 4) + reference += 4 + reference += 4 + reference += 4 + idx += 3 + + while (idx < column.capacity) { + val single = random.nextBoolean() + if (single) { + val v = random.nextInt().toShort + column.putShort(idx, v) + reference += v + idx += 1 + } else { + val n = math.min(random.nextInt(column.capacity / 20), column.capacity - idx) + val v = (n + 1).toShort + column.putShorts(idx, n, v) + var i = 0 + while (i < n) { + reference += v + i += 1 + } + idx += n + } + } + + reference.zipWithIndex.foreach { v => + assert(v._1 == column.getShort(v._2), "Seed = " + seed + " Mem Mode=" + memMode) + if (memMode == MemoryMode.OFF_HEAP) { + val addr = column.valuesNativeAddress() + assert(v._1 == Platform.getShort(null, addr + 2 * v._2)) + } + } + + column.close + }} + } + + test("Int Apis") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + val seed = System.currentTimeMillis() + val random = new Random(seed) + val reference = mutable.ArrayBuffer.empty[Int] + + val column = allocate(1024, IntegerType, memMode) + + var values = (10 :: 20 :: 30 :: 40 :: 50 :: Nil).toArray + column.appendInts(2, values, 0) + reference += 10 + reference += 20 + + column.appendInts(3, values, 2) + reference += 30 + reference += 40 + reference += 50 + + column.appendInts(6, 60) + (1 to 6).foreach(_ => reference += 60) + + column.appendInt(70) + reference += 70 + + var idx = column.elementsAppended + + values = (1 :: 2 :: 3 :: 4 :: 5 :: Nil).toArray + column.putInts(idx, 2, values, 0) + reference += 1 + reference += 2 + idx += 2 + + column.putInts(idx, 3, values, 2) + reference += 3 + reference += 4 + reference += 5 + idx += 3 + + val littleEndian = new Array[Byte](8) + littleEndian(0) = 7 + littleEndian(1) = 1 + littleEndian(4) = 6 + littleEndian(6) = 1 + + column.putIntsLittleEndian(idx, 1, littleEndian, 4) + column.putIntsLittleEndian(idx + 1, 1, littleEndian, 0) + reference += 6 + (1 << 16) + reference += 7 + (1 << 8) + idx += 2 + + column.putIntsLittleEndian(idx, 2, littleEndian, 0) + reference += 7 + (1 << 8) + reference += 6 + (1 << 16) + idx += 2 + + while (idx < column.capacity) { + val single = random.nextBoolean() + if (single) { + val v = random.nextInt() + column.putInt(idx, v) + reference += v + idx += 1 + } else { + val n = math.min(random.nextInt(column.capacity / 20), column.capacity - idx) + column.putInts(idx, n, n + 1) + var i = 0 + while (i < n) { + reference += (n + 1) + i += 1 + } + idx += n + } + } + + reference.zipWithIndex.foreach { v => + assert(v._1 == column.getInt(v._2), "Seed = " + seed + " Mem Mode=" + memMode) + if (memMode == MemoryMode.OFF_HEAP) { + val addr = column.valuesNativeAddress() + assert(v._1 == Platform.getInt(null, addr + 4 * v._2)) + } + } + column.close + }} + } + + test("Long Apis") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + val seed = System.currentTimeMillis() + val random = new Random(seed) + val reference = mutable.ArrayBuffer.empty[Long] + + val column = allocate(1024, LongType, memMode) + + var values = (10L :: 20L :: 30L :: 40L :: 50L :: Nil).toArray + column.appendLongs(2, values, 0) + reference += 10L + reference += 20L + + column.appendLongs(3, values, 2) + reference += 30L + reference += 40L + reference += 50L + + column.appendLongs(6, 60L) + (1 to 6).foreach(_ => reference += 60L) + + column.appendLong(70L) + reference += 70L + + var idx = column.elementsAppended + + values = (1L :: 2L :: 3L :: 4L :: 5L :: Nil).toArray + column.putLongs(idx, 2, values, 0) + reference += 1 + reference += 2 + idx += 2 + + column.putLongs(idx, 3, values, 2) + reference += 3 + reference += 4 + reference += 5 + idx += 3 + + val littleEndian = new Array[Byte](16) + littleEndian(0) = 7 + littleEndian(1) = 1 + littleEndian(8) = 6 + littleEndian(10) = 1 + + column.putLongsLittleEndian(idx, 1, littleEndian, 8) + column.putLongsLittleEndian(idx + 1, 1, littleEndian, 0) + reference += 6 + (1 << 16) + reference += 7 + (1 << 8) + idx += 2 + + column.putLongsLittleEndian(idx, 2, littleEndian, 0) + reference += 7 + (1 << 8) + reference += 6 + (1 << 16) + idx += 2 + + while (idx < column.capacity) { + val single = random.nextBoolean() + if (single) { + val v = random.nextLong() + column.putLong(idx, v) + reference += v + idx += 1 + } else { + + val n = math.min(random.nextInt(column.capacity / 20), column.capacity - idx) + column.putLongs(idx, n, n + 1) + var i = 0 + while (i < n) { + reference += (n + 1) + i += 1 + } + idx += n + } + } + + + reference.zipWithIndex.foreach { v => + assert(v._1 == column.getLong(v._2), "idx=" + v._2 + + " Seed = " + seed + " MemMode=" + memMode) + if (memMode == MemoryMode.OFF_HEAP) { + val addr = column.valuesNativeAddress() + assert(v._1 == Platform.getLong(null, addr + 8 * v._2)) + } + } + }} + } + + test("Float APIs") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + val seed = System.currentTimeMillis() + val random = new Random(seed) + val reference = mutable.ArrayBuffer.empty[Float] + + val column = allocate(1024, FloatType, memMode) + + var values = (.1f :: .2f :: .3f :: .4f :: .5f :: Nil).toArray + column.appendFloats(2, values, 0) + reference += .1f + reference += .2f + + column.appendFloats(3, values, 2) + reference += .3f + reference += .4f + reference += .5f + + column.appendFloats(6, .6f) + (1 to 6).foreach(_ => reference += .6f) + + column.appendFloat(.7f) + reference += .7f + + var idx = column.elementsAppended + + values = (1.0f :: 2.0f :: 3.0f :: 4.0f :: 5.0f :: Nil).toArray + column.putFloats(idx, 2, values, 0) + reference += 1.0f + reference += 2.0f + idx += 2 + + column.putFloats(idx, 3, values, 2) + reference += 3.0f + reference += 4.0f + reference += 5.0f + idx += 3 + + val buffer = new Array[Byte](8) + Platform.putFloat(buffer, Platform.BYTE_ARRAY_OFFSET, 2.234f) + Platform.putFloat(buffer, Platform.BYTE_ARRAY_OFFSET + 4, 1.123f) + + if (ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN)) { + // Ensure array contains Little Endian floats + val bb = ByteBuffer.wrap(buffer).order(ByteOrder.LITTLE_ENDIAN) + Platform.putFloat(buffer, Platform.BYTE_ARRAY_OFFSET, bb.getFloat(0)) + Platform.putFloat(buffer, Platform.BYTE_ARRAY_OFFSET + 4, bb.getFloat(4)) + } + + column.putFloats(idx, 1, buffer, 4) + column.putFloats(idx + 1, 1, buffer, 0) + reference += 1.123f + reference += 2.234f + idx += 2 + + column.putFloats(idx, 2, buffer, 0) + reference += 2.234f + reference += 1.123f + idx += 2 + + while (idx < column.capacity) { + val single = random.nextBoolean() + if (single) { + val v = random.nextFloat() + column.putFloat(idx, v) + reference += v + idx += 1 + } else { + val n = math.min(random.nextInt(column.capacity / 20), column.capacity - idx) + val v = random.nextFloat() + column.putFloats(idx, n, v) + var i = 0 + while (i < n) { + reference += v + i += 1 + } + idx += n + } + } + + reference.zipWithIndex.foreach { v => + assert(v._1 == column.getFloat(v._2), "Seed = " + seed + " MemMode=" + memMode) + if (memMode == MemoryMode.OFF_HEAP) { + val addr = column.valuesNativeAddress() + assert(v._1 == Platform.getFloat(null, addr + 4 * v._2)) + } + } + column.close + }} + } + + test("Double APIs") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + val seed = System.currentTimeMillis() + val random = new Random(seed) + val reference = mutable.ArrayBuffer.empty[Double] + + val column = allocate(1024, DoubleType, memMode) + + var values = (.1 :: .2 :: .3 :: .4 :: .5 :: Nil).toArray + column.appendDoubles(2, values, 0) + reference += .1 + reference += .2 + + column.appendDoubles(3, values, 2) + reference += .3 + reference += .4 + reference += .5 + + column.appendDoubles(6, .6) + (1 to 6).foreach(_ => reference += .6) + + column.appendDouble(.7) + reference += .7 + + var idx = column.elementsAppended + + values = (1.0 :: 2.0 :: 3.0 :: 4.0 :: 5.0 :: Nil).toArray + column.putDoubles(idx, 2, values, 0) + reference += 1.0 + reference += 2.0 + idx += 2 + + column.putDoubles(idx, 3, values, 2) + reference += 3.0 + reference += 4.0 + reference += 5.0 + idx += 3 + + val buffer = new Array[Byte](16) + Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET, 2.234) + Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET + 8, 1.123) + + if (ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN)) { + // Ensure array contains Little Endian doubles + val bb = ByteBuffer.wrap(buffer).order(ByteOrder.LITTLE_ENDIAN) + Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET, bb.getDouble(0)) + Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET + 8, bb.getDouble(8)) + } + + column.putDoubles(idx, 1, buffer, 8) + column.putDoubles(idx + 1, 1, buffer, 0) + reference += 1.123 + reference += 2.234 + idx += 2 + + column.putDoubles(idx, 2, buffer, 0) + reference += 2.234 + reference += 1.123 + idx += 2 + + while (idx < column.capacity) { + val single = random.nextBoolean() + if (single) { + val v = random.nextDouble() + column.putDouble(idx, v) + reference += v + idx += 1 + } else { + val n = math.min(random.nextInt(column.capacity / 20), column.capacity - idx) + val v = random.nextDouble() + column.putDoubles(idx, n, v) + var i = 0 + while (i < n) { + reference += v + i += 1 + } + idx += n + } + } + + reference.zipWithIndex.foreach { v => + assert(v._1 == column.getDouble(v._2), "Seed = " + seed + " MemMode=" + memMode) + if (memMode == MemoryMode.OFF_HEAP) { + val addr = column.valuesNativeAddress() + assert(v._1 == Platform.getDouble(null, addr + 8 * v._2)) + } + } + column.close + }} + } + + test("String APIs") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + val reference = mutable.ArrayBuffer.empty[String] + + val column = allocate(6, BinaryType, memMode) + assert(column.arrayData().elementsAppended == 0) + + val str = "string" + column.appendByteArray(str.getBytes(StandardCharsets.UTF_8), + 0, str.getBytes(StandardCharsets.UTF_8).length) + reference += str + assert(column.arrayData().elementsAppended == 6) + + var idx = column.elementsAppended + + val values = ("Hello" :: "abc" :: Nil).toArray + column.putByteArray(idx, values(0).getBytes(StandardCharsets.UTF_8), + 0, values(0).getBytes(StandardCharsets.UTF_8).length) + reference += values(0) + idx += 1 + assert(column.arrayData().elementsAppended == 11) + + column.putByteArray(idx, values(1).getBytes(StandardCharsets.UTF_8), + 0, values(1).getBytes(StandardCharsets.UTF_8).length) + reference += values(1) + idx += 1 + assert(column.arrayData().elementsAppended == 14) + + // Just put llo + val offset = column.putByteArray(idx, values(0).getBytes(StandardCharsets.UTF_8), + 2, values(0).getBytes(StandardCharsets.UTF_8).length - 2) + reference += "llo" + idx += 1 + assert(column.arrayData().elementsAppended == 17) + + // Put the same "ll" at offset. This should not allocate more memory in the column. + column.putArray(idx, offset, 2) + reference += "ll" + idx += 1 + assert(column.arrayData().elementsAppended == 17) + + // Put a long string + val s = "abcdefghijklmnopqrstuvwxyz" + column.putByteArray(idx, (s + s).getBytes(StandardCharsets.UTF_8)) + reference += (s + s) + idx += 1 + assert(column.arrayData().elementsAppended == 17 + (s + s).length) + + reference.zipWithIndex.foreach { v => + assert(v._1.length == column.getArrayLength(v._2), "MemoryMode=" + memMode) + assert(v._1 == column.getUTF8String(v._2).toString, + "MemoryMode" + memMode) + } + + column.reset() + assert(column.arrayData().elementsAppended == 0) + }} + } + + test("Int Array") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + val column = allocate(10, new ArrayType(IntegerType, true), memMode) + + // Fill the underlying data with all the arrays back to back. + val data = column.arrayData(); + var i = 0 + while (i < 6) { + data.putInt(i, i) + i += 1 + } + + // Populate it with arrays [0], [1, 2], [], [3, 4, 5] + column.putArray(0, 0, 1) + column.putArray(1, 1, 2) + column.putArray(2, 2, 0) + column.putArray(3, 3, 3) + + val a1 = ColumnVectorUtils.toPrimitiveJavaArray(column.getArray(0)).asInstanceOf[Array[Int]] + val a2 = ColumnVectorUtils.toPrimitiveJavaArray(column.getArray(1)).asInstanceOf[Array[Int]] + val a3 = ColumnVectorUtils.toPrimitiveJavaArray(column.getArray(2)).asInstanceOf[Array[Int]] + val a4 = ColumnVectorUtils.toPrimitiveJavaArray(column.getArray(3)).asInstanceOf[Array[Int]] + assert(a1 === Array(0)) + assert(a2 === Array(1, 2)) + assert(a3 === Array.empty[Int]) + assert(a4 === Array(3, 4, 5)) + + // Verify the ArrayData APIs + assert(column.getArray(0).length == 1) + assert(column.getArray(0).getInt(0) == 0) + + assert(column.getArray(1).length == 2) + assert(column.getArray(1).getInt(0) == 1) + assert(column.getArray(1).getInt(1) == 2) + + assert(column.getArray(2).length == 0) + + assert(column.getArray(3).length == 3) + assert(column.getArray(3).getInt(0) == 3) + assert(column.getArray(3).getInt(1) == 4) + assert(column.getArray(3).getInt(2) == 5) + + // Add a longer array which requires resizing + column.reset + val array = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) + assert(data.capacity == 10) + data.reserve(array.length) + assert(data.capacity == array.length * 2) + data.putInts(0, array.length, array, 0) + column.putArray(0, 0, array.length) + assert(ColumnVectorUtils.toPrimitiveJavaArray(column.getArray(0)).asInstanceOf[Array[Int]] + === array) + }} + } + + test("toArray for primitive types") { + // (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + (MemoryMode.ON_HEAP :: Nil).foreach { memMode => { + val len = 4 + + val columnBool = allocate(len, new ArrayType(BooleanType, false), memMode) + val boolArray = Array(false, true, false, true) + boolArray.zipWithIndex.map { case (v, i) => columnBool.arrayData.putBoolean(i, v) } + columnBool.putArray(0, 0, len) + assert(columnBool.getArray(0).toBooleanArray === boolArray) + + val columnByte = allocate(len, new ArrayType(ByteType, false), memMode) + val byteArray = Array[Byte](0, 1, 2, 3) + byteArray.zipWithIndex.map { case (v, i) => columnByte.arrayData.putByte(i, v) } + columnByte.putArray(0, 0, len) + assert(columnByte.getArray(0).toByteArray === byteArray) + + val columnShort = allocate(len, new ArrayType(ShortType, false), memMode) + val shortArray = Array[Short](0, 1, 2, 3) + shortArray.zipWithIndex.map { case (v, i) => columnShort.arrayData.putShort(i, v) } + columnShort.putArray(0, 0, len) + assert(columnShort.getArray(0).toShortArray === shortArray) + + val columnInt = allocate(len, new ArrayType(IntegerType, false), memMode) + val intArray = Array(0, 1, 2, 3) + intArray.zipWithIndex.map { case (v, i) => columnInt.arrayData.putInt(i, v) } + columnInt.putArray(0, 0, len) + assert(columnInt.getArray(0).toIntArray === intArray) + + val columnLong = allocate(len, new ArrayType(LongType, false), memMode) + val longArray = Array[Long](0, 1, 2, 3) + longArray.zipWithIndex.map { case (v, i) => columnLong.arrayData.putLong(i, v) } + columnLong.putArray(0, 0, len) + assert(columnLong.getArray(0).toLongArray === longArray) + + val columnFloat = allocate(len, new ArrayType(FloatType, false), memMode) + val floatArray = Array(0.0F, 1.1F, 2.2F, 3.3F) + floatArray.zipWithIndex.map { case (v, i) => columnFloat.arrayData.putFloat(i, v) } + columnFloat.putArray(0, 0, len) + assert(columnFloat.getArray(0).toFloatArray === floatArray) + + val columnDouble = allocate(len, new ArrayType(DoubleType, false), memMode) + val doubleArray = Array(0.0, 1.1, 2.2, 3.3) + doubleArray.zipWithIndex.map { case (v, i) => columnDouble.arrayData.putDouble(i, v) } + columnDouble.putArray(0, 0, len) + assert(columnDouble.getArray(0).toDoubleArray === doubleArray) + }} + } + + test("Struct Column") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + val schema = new StructType().add("int", IntegerType).add("double", DoubleType) + val column = allocate(1024, schema, memMode) + + val c1 = column.getChildColumn(0) + val c2 = column.getChildColumn(1) + assert(c1.dataType() == IntegerType) + assert(c2.dataType() == DoubleType) + + c1.putInt(0, 123) + c2.putDouble(0, 3.45) + c1.putInt(1, 456) + c2.putDouble(1, 5.67) + + val s = column.getStruct(0) + assert(s.columns()(0).getInt(0) == 123) + assert(s.columns()(0).getInt(1) == 456) + assert(s.columns()(1).getDouble(0) == 3.45) + assert(s.columns()(1).getDouble(1) == 5.67) + + assert(s.getInt(0) == 123) + assert(s.getDouble(1) == 3.45) + + val s2 = column.getStruct(1) + assert(s2.getInt(0) == 456) + assert(s2.getDouble(1) == 5.67) + }} + } + + test("Nest Array in Array.") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => + val column = allocate(10, new ArrayType(new ArrayType(IntegerType, true), true), + memMode) + val childColumn = column.arrayData() + val data = column.arrayData().arrayData() + (0 until 6).foreach { + case 3 => data.putNull(3) + case i => data.putInt(i, i) + } + // Arrays in child column: [0], [1, 2], [], [null, 4, 5] + childColumn.putArray(0, 0, 1) + childColumn.putArray(1, 1, 2) + childColumn.putArray(2, 2, 0) + childColumn.putArray(3, 3, 3) + // Arrays in column: [[0]], [[1, 2], []], [[], [null, 4, 5]], null + column.putArray(0, 0, 1) + column.putArray(1, 1, 2) + column.putArray(2, 2, 2) + column.putNull(3) + + assert(column.getArray(0).getArray(0).toIntArray() === Array(0)) + assert(column.getArray(1).getArray(0).toIntArray() === Array(1, 2)) + assert(column.getArray(1).getArray(1).toIntArray() === Array()) + assert(column.getArray(2).getArray(0).toIntArray() === Array()) + assert(column.getArray(2).getArray(1).isNullAt(0)) + assert(column.getArray(2).getArray(1).getInt(1) === 4) + assert(column.getArray(2).getArray(1).getInt(2) === 5) + assert(column.isNullAt(3)) + } + } + + test("Nest Struct in Array.") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => + val schema = new StructType().add("int", IntegerType).add("long", LongType) + val column = allocate(10, new ArrayType(schema, true), memMode) + val data = column.arrayData() + val c0 = data.getChildColumn(0) + val c1 = data.getChildColumn(1) + // Structs in child column: (0, 0), (1, 10), (2, 20), (3, 30), (4, 40), (5, 50) + (0 until 6).foreach { i => + c0.putInt(i, i) + c1.putLong(i, i * 10) + } + // Arrays in column: [(0, 0), (1, 10)], [(1, 10), (2, 20), (3, 30)], + // [(4, 40), (5, 50)] + column.putArray(0, 0, 2) + column.putArray(1, 1, 3) + column.putArray(2, 4, 2) + + assert(column.getArray(0).getStruct(0, 2).toSeq(schema) === Seq(0, 0)) + assert(column.getArray(0).getStruct(1, 2).toSeq(schema) === Seq(1, 10)) + assert(column.getArray(1).getStruct(0, 2).toSeq(schema) === Seq(1, 10)) + assert(column.getArray(1).getStruct(1, 2).toSeq(schema) === Seq(2, 20)) + assert(column.getArray(1).getStruct(2, 2).toSeq(schema) === Seq(3, 30)) + assert(column.getArray(2).getStruct(0, 2).toSeq(schema) === Seq(4, 40)) + assert(column.getArray(2).getStruct(1, 2).toSeq(schema) === Seq(5, 50)) + } + } + + test("Nest Array in Struct.") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => + val schema = new StructType() + .add("int", IntegerType) + .add("array", new ArrayType(IntegerType, true)) + val column = allocate(10, schema, memMode) + val c0 = column.getChildColumn(0) + val c1 = column.getChildColumn(1) + c0.putInt(0, 0) + c0.putInt(1, 1) + c0.putInt(2, 2) + val c1Child = c1.arrayData() + (0 until 6).foreach { i => + c1Child.putInt(i, i) + } + // Arrays in c1: [0, 1], [2], [3, 4, 5] + c1.putArray(0, 0, 2) + c1.putArray(1, 2, 1) + c1.putArray(2, 3, 3) + + assert(column.getStruct(0).getInt(0) === 0) + assert(column.getStruct(0).getArray(1).toIntArray() === Array(0, 1)) + assert(column.getStruct(1).getInt(0) === 1) + assert(column.getStruct(1).getArray(1).toIntArray() === Array(2)) + assert(column.getStruct(2).getInt(0) === 2) + assert(column.getStruct(2).getArray(1).toIntArray() === Array(3, 4, 5)) + } + } + + test("Nest Struct in Struct.") { + (MemoryMode.ON_HEAP :: Nil).foreach { memMode => + val subSchema = new StructType() + .add("int", IntegerType) + .add("int", IntegerType) + val schema = new StructType() + .add("int", IntegerType) + .add("struct", subSchema) + val column = allocate(10, schema, memMode) + val c0 = column.getChildColumn(0) + val c1 = column.getChildColumn(1) + c0.putInt(0, 0) + c0.putInt(1, 1) + c0.putInt(2, 2) + val c1c0 = c1.getChildColumn(0) + val c1c1 = c1.getChildColumn(1) + // Structs in c1: (7, 70), (8, 80), (9, 90) + c1c0.putInt(0, 7) + c1c0.putInt(1, 8) + c1c0.putInt(2, 9) + c1c1.putInt(0, 70) + c1c1.putInt(1, 80) + c1c1.putInt(2, 90) + + assert(column.getStruct(0).getInt(0) === 0) + assert(column.getStruct(0).getStruct(1, 2).toSeq(subSchema) === Seq(7, 70)) + assert(column.getStruct(1).getInt(0) === 1) + assert(column.getStruct(1).getStruct(1, 2).toSeq(subSchema) === Seq(8, 80)) + assert(column.getStruct(2).getInt(0) === 2) + assert(column.getStruct(2).getStruct(1, 2).toSeq(subSchema) === Seq(9, 90)) + } + } + + test("ColumnarBatch basic") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + val schema = new StructType() + .add("intCol", IntegerType) + .add("doubleCol", DoubleType) + .add("intCol2", IntegerType) + .add("string", BinaryType) + + val capacity = ColumnarBatch.DEFAULT_BATCH_SIZE + val columns = schema.fields.map { field => + allocate(capacity, field.dataType, memMode) + } + val batch = new ColumnarBatch(schema, columns.toArray, ColumnarBatch.DEFAULT_BATCH_SIZE) + assert(batch.numCols() == 4) + assert(batch.numRows() == 0) + assert(batch.numValidRows() == 0) + assert(batch.capacity() > 0) + assert(batch.rowIterator().hasNext == false) + + // Add a row [1, 1.1, NULL] + columns(0).putInt(0, 1) + columns(1).putDouble(0, 1.1) + columns(2).putNull(0) + columns(3).putByteArray(0, "Hello".getBytes(StandardCharsets.UTF_8)) + batch.setNumRows(1) + + // Verify the results of the row. + assert(batch.numCols() == 4) + assert(batch.numRows() == 1) + assert(batch.numValidRows() == 1) + assert(batch.rowIterator().hasNext == true) + assert(batch.rowIterator().hasNext == true) + + assert(columns(0).getInt(0) == 1) + assert(columns(0).isNullAt(0) == false) + assert(columns(1).getDouble(0) == 1.1) + assert(columns(1).isNullAt(0) == false) + assert(columns(2).isNullAt(0) == true) + assert(columns(3).getUTF8String(0).toString == "Hello") + + // Verify the iterator works correctly. + val it = batch.rowIterator() + assert(it.hasNext()) + val row = it.next() + assert(row.getInt(0) == 1) + assert(row.isNullAt(0) == false) + assert(row.getDouble(1) == 1.1) + assert(row.isNullAt(1) == false) + assert(row.isNullAt(2) == true) + assert(columns(3).getUTF8String(0).toString == "Hello") + assert(it.hasNext == false) + assert(it.hasNext == false) + + // Filter out the row. + row.markFiltered() + assert(batch.numRows() == 1) + assert(batch.numValidRows() == 0) + assert(batch.rowIterator().hasNext == false) + + // Reset and add 3 rows + batch.reset() + assert(batch.numRows() == 0) + assert(batch.numValidRows() == 0) + assert(batch.rowIterator().hasNext == false) + + // Add rows [NULL, 2.2, 2, "abc"], [3, NULL, 3, ""], [4, 4.4, 4, "world] + columns(0).putNull(0) + columns(1).putDouble(0, 2.2) + columns(2).putInt(0, 2) + columns(3).putByteArray(0, "abc".getBytes(StandardCharsets.UTF_8)) + + columns(0).putInt(1, 3) + columns(1).putNull(1) + columns(2).putInt(1, 3) + columns(3).putByteArray(1, "".getBytes(StandardCharsets.UTF_8)) + + columns(0).putInt(2, 4) + columns(1).putDouble(2, 4.4) + columns(2).putInt(2, 4) + columns(3).putByteArray(2, "world".getBytes(StandardCharsets.UTF_8)) + batch.setNumRows(3) + + def rowEquals(x: InternalRow, y: Row): Unit = { + assert(x.isNullAt(0) == y.isNullAt(0)) + if (!x.isNullAt(0)) assert(x.getInt(0) == y.getInt(0)) + + assert(x.isNullAt(1) == y.isNullAt(1)) + if (!x.isNullAt(1)) assert(x.getDouble(1) == y.getDouble(1)) + + assert(x.isNullAt(2) == y.isNullAt(2)) + if (!x.isNullAt(2)) assert(x.getInt(2) == y.getInt(2)) + + assert(x.isNullAt(3) == y.isNullAt(3)) + if (!x.isNullAt(3)) assert(x.getString(3) == y.getString(3)) + } + + // Verify + assert(batch.numRows() == 3) + assert(batch.numValidRows() == 3) + val it2 = batch.rowIterator() + rowEquals(it2.next(), Row(null, 2.2, 2, "abc")) + rowEquals(it2.next(), Row(3, null, 3, "")) + rowEquals(it2.next(), Row(4, 4.4, 4, "world")) + assert(!it.hasNext) + + // Filter out some rows and verify + batch.markFiltered(1) + assert(batch.numValidRows() == 2) + val it3 = batch.rowIterator() + rowEquals(it3.next(), Row(null, 2.2, 2, "abc")) + rowEquals(it3.next(), Row(4, 4.4, 4, "world")) + assert(!it.hasNext) + + batch.markFiltered(2) + assert(batch.numValidRows() == 1) + val it4 = batch.rowIterator() + rowEquals(it4.next(), Row(null, 2.2, 2, "abc")) + + batch.close + }} + } + + private def doubleEquals(d1: Double, d2: Double): Boolean = { + if (d1.isNaN && d2.isNaN) { + true + } else { + d1 == d2 + } + } + + private def compareStruct(fields: Seq[StructField], r1: InternalRow, r2: Row, seed: Long) { + fields.zipWithIndex.foreach { case (field: StructField, ordinal: Int) => + assert(r1.isNullAt(ordinal) == r2.isNullAt(ordinal), "Seed = " + seed) + if (!r1.isNullAt(ordinal)) { + field.dataType match { + case BooleanType => assert(r1.getBoolean(ordinal) == r2.getBoolean(ordinal), + "Seed = " + seed) + case ByteType => assert(r1.getByte(ordinal) == r2.getByte(ordinal), "Seed = " + seed) + case ShortType => assert(r1.getShort(ordinal) == r2.getShort(ordinal), "Seed = " + seed) + case IntegerType => assert(r1.getInt(ordinal) == r2.getInt(ordinal), "Seed = " + seed) + case LongType => assert(r1.getLong(ordinal) == r2.getLong(ordinal), "Seed = " + seed) + case FloatType => assert(doubleEquals(r1.getFloat(ordinal), r2.getFloat(ordinal)), + "Seed = " + seed) + case DoubleType => assert(doubleEquals(r1.getDouble(ordinal), r2.getDouble(ordinal)), + "Seed = " + seed) + case t: DecimalType => + val d1 = r1.getDecimal(ordinal, t.precision, t.scale).toBigDecimal + val d2 = r2.getDecimal(ordinal) + assert(d1.compare(d2) == 0, "Seed = " + seed) + case StringType => + assert(r1.getString(ordinal) == r2.getString(ordinal), "Seed = " + seed) + case CalendarIntervalType => + assert(r1.getInterval(ordinal) === r2.get(ordinal).asInstanceOf[CalendarInterval]) + case ArrayType(childType, n) => + val a1 = r1.getArray(ordinal).array + val a2 = r2.getList(ordinal).toArray + assert(a1.length == a2.length, "Seed = " + seed) + childType match { + case DoubleType => + var i = 0 + while (i < a1.length) { + assert(doubleEquals(a1(i).asInstanceOf[Double], a2(i).asInstanceOf[Double]), + "Seed = " + seed) + i += 1 + } + case FloatType => + var i = 0 + while (i < a1.length) { + assert(doubleEquals(a1(i).asInstanceOf[Float], a2(i).asInstanceOf[Float]), + "Seed = " + seed) + i += 1 + } + case t: DecimalType => + var i = 0 + while (i < a1.length) { + assert((a1(i) == null) == (a2(i) == null), "Seed = " + seed) + if (a1(i) != null) { + val d1 = a1(i).asInstanceOf[Decimal].toBigDecimal + val d2 = a2(i).asInstanceOf[java.math.BigDecimal] + assert(d1.compare(d2) == 0, "Seed = " + seed) + } + i += 1 + } + case _ => assert(a1 === a2, "Seed = " + seed) + } + case StructType(childFields) => + compareStruct(childFields, r1.getStruct(ordinal, fields.length), + r2.getStruct(ordinal), seed) + case _ => + throw new NotImplementedError("Not implemented " + field.dataType) + } + } + } + } + + test("Convert rows") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + val rows = Row(1, 2L, "a", 1.2, 'b'.toByte) :: Row(4, 5L, "cd", 2.3, 'a'.toByte) :: Nil + val schema = new StructType() + .add("i1", IntegerType) + .add("l2", LongType) + .add("string", StringType) + .add("d", DoubleType) + .add("b", ByteType) + + val batch = ColumnVectorUtils.toBatch(schema, memMode, rows.iterator.asJava) + assert(batch.numRows() == 2) + assert(batch.numCols() == 5) + + val it = batch.rowIterator() + val referenceIt = rows.iterator + while (it.hasNext) { + compareStruct(schema, it.next(), referenceIt.next(), 0) + } + batch.close() + } + }} + + /** + * This test generates a random schema data, serializes it to column batches and verifies the + * results. + */ + def testRandomRows(flatSchema: Boolean, numFields: Int) { + // TODO: Figure out why StringType doesn't work on jenkins. + val types = Array( + BooleanType, ByteType, FloatType, DoubleType, IntegerType, LongType, ShortType, + DecimalType.ShortDecimal, DecimalType.IntDecimal, DecimalType.ByteDecimal, + DecimalType.FloatDecimal, DecimalType.LongDecimal, new DecimalType(5, 2), + new DecimalType(12, 2), new DecimalType(30, 10), CalendarIntervalType) + val seed = System.nanoTime() + val NUM_ROWS = 200 + val NUM_ITERS = 1000 + val random = new Random(seed) + var i = 0 + while (i < NUM_ITERS) { + val schema = if (flatSchema) { + RandomDataGenerator.randomSchema(random, numFields, types) + } else { + RandomDataGenerator.randomNestedSchema(random, numFields, types) + } + val rows = mutable.ArrayBuffer.empty[Row] + var j = 0 + while (j < NUM_ROWS) { + val row = RandomDataGenerator.randomRow(random, schema) + rows += row + j += 1 + } + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + val batch = ColumnVectorUtils.toBatch(schema, memMode, rows.iterator.asJava) + assert(batch.numRows() == NUM_ROWS) + + val it = batch.rowIterator() + val referenceIt = rows.iterator + var k = 0 + while (it.hasNext) { + compareStruct(schema, it.next(), referenceIt.next(), seed) + k += 1 + } + batch.close() + }} + i += 1 + } + } + + test("Random flat schema") { + testRandomRows(true, 15) + } + + test("Random nested schema") { + testRandomRows(false, 30) + } + + test("null filtered columns") { + val NUM_ROWS = 10 + val schema = new StructType() + .add("key", IntegerType, nullable = false) + .add("value", StringType, nullable = true) + for (numNulls <- List(0, NUM_ROWS / 2, NUM_ROWS)) { + val rows = mutable.ArrayBuffer.empty[Row] + for (i <- 0 until NUM_ROWS) { + val row = if (i < numNulls) Row.fromSeq(Seq(i, null)) else Row.fromSeq(Seq(i, i.toString)) + rows += row + } + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => { + val batch = ColumnVectorUtils.toBatch(schema, memMode, rows.iterator.asJava) + batch.filterNullsInColumn(1) + batch.setNumRows(NUM_ROWS) + assert(batch.numRows() == NUM_ROWS) + val it = batch.rowIterator() + // Top numNulls rows should be filtered + var k = numNulls + while (it.hasNext) { + assert(it.next().getInt(0) == k) + k += 1 + } + assert(k == NUM_ROWS) + batch.close() + }} + } + } + + test("mutable ColumnarBatch rows") { + val NUM_ITERS = 10 + val types = Array( + BooleanType, FloatType, DoubleType, IntegerType, LongType, ShortType, + DecimalType.ShortDecimal, DecimalType.IntDecimal, DecimalType.ByteDecimal, + DecimalType.FloatDecimal, DecimalType.LongDecimal, new DecimalType(5, 2), + new DecimalType(12, 2), new DecimalType(30, 10)) + for (i <- 0 to NUM_ITERS) { + val random = new Random(System.nanoTime()) + val schema = RandomDataGenerator.randomSchema(random, numFields = 20, types) + val oldRow = RandomDataGenerator.randomRow(random, schema) + val newRow = RandomDataGenerator.randomRow(random, schema) + + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => + val batch = ColumnVectorUtils.toBatch(schema, memMode, (oldRow :: Nil).iterator.asJava) + val columnarBatchRow = batch.getRow(0) + newRow.toSeq.zipWithIndex.foreach(i => columnarBatchRow.update(i._2, i._1)) + compareStruct(schema, columnarBatchRow, newRow, 0) + batch.close() + } + } + } + + test("exceeding maximum capacity should throw an error") { + (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => + val column = allocate(1, ByteType, memMode) + column.MAX_CAPACITY = 15 + column.appendBytes(5, 0.toByte) + // Successfully allocate twice the requested capacity + assert(column.capacity == 10) + column.appendBytes(10, 0.toByte) + // Allocated capacity doesn't exceed MAX_CAPACITY + assert(column.capacity == 15) + val ex = intercept[RuntimeException] { + // Over-allocating beyond MAX_CAPACITY throws an exception + column.appendBytes(10, 0.toByte) + } + assert(ex.getMessage.contains(s"Cannot reserve additional contiguous bytes in the " + + s"vectorized reader")) + } + } + + test("create columnar batch from Arrow column vectors") { + val allocator = ArrowUtils.rootAllocator.newChildAllocator("int", 0, Long.MaxValue) + val vector1 = ArrowUtils.toArrowField("int1", IntegerType, nullable = true) + .createVector(allocator).asInstanceOf[NullableIntVector] + vector1.allocateNew() + val mutator1 = vector1.getMutator() + val vector2 = ArrowUtils.toArrowField("int2", IntegerType, nullable = true) + .createVector(allocator).asInstanceOf[NullableIntVector] + vector2.allocateNew() + val mutator2 = vector2.getMutator() + + (0 until 10).foreach { i => + mutator1.setSafe(i, i) + mutator2.setSafe(i + 1, i) + } + mutator1.setNull(10) + mutator1.setValueCount(11) + mutator2.setNull(0) + mutator2.setValueCount(11) + + val columnVectors = Seq(new ArrowColumnVector(vector1), new ArrowColumnVector(vector2)) + + val schema = StructType(Seq(StructField("int1", IntegerType), StructField("int2", IntegerType))) + val batch = new ColumnarBatch(schema, columnVectors.toArray[ColumnVector], 11) + batch.setNumRows(11) + + assert(batch.numCols() == 2) + assert(batch.numRows() == 11) + + val rowIter = batch.rowIterator().asScala + rowIter.zipWithIndex.foreach { case (row, i) => + if (i == 10) { + assert(row.isNullAt(0)) + } else { + assert(row.getInt(0) == i) + } + if (i == 0) { + assert(row.isNullAt(1)) + } else { + assert(row.getInt(1) == i - 1) + } + } + + batch.close() + allocator.close() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ReduceAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ReduceAggregatorSuite.scala new file mode 100644 index 000000000000..f65dcdf119c3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ReduceAggregatorSuite.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.Encoders +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder + +class ReduceAggregatorSuite extends SparkFunSuite { + + test("zero value") { + val encoder: ExpressionEncoder[Int] = ExpressionEncoder() + val func = (v1: Int, v2: Int) => v1 + v2 + val aggregator: ReduceAggregator[Int] = new ReduceAggregator(func)(Encoders.scalaInt) + assert(aggregator.zero == (false, null).asInstanceOf[(Boolean, Int)]) + } + + test("reduce, merge and finish") { + val encoder: ExpressionEncoder[Int] = ExpressionEncoder() + val func = (v1: Int, v2: Int) => v1 + v2 + val aggregator: ReduceAggregator[Int] = new ReduceAggregator(func)(Encoders.scalaInt) + + val firstReduce = aggregator.reduce(aggregator.zero, 1) + assert(firstReduce == ((true, 1))) + + val secondReduce = aggregator.reduce(firstReduce, 2) + assert(secondReduce == ((true, 3))) + + val thirdReduce = aggregator.reduce(secondReduce, 3) + assert(thirdReduce == ((true, 6))) + + val mergeWithZero1 = aggregator.merge(aggregator.zero, firstReduce) + assert(mergeWithZero1 == ((true, 1))) + + val mergeWithZero2 = aggregator.merge(secondReduce, aggregator.zero) + assert(mergeWithZero2 == ((true, 3))) + + val mergeTwoReduced = aggregator.merge(firstReduce, secondReduce) + assert(mergeTwoReduced == ((true, 4))) + + assert(aggregator.finish(firstReduce)== 1) + assert(aggregator.finish(secondReduce) == 3) + assert(aggregator.finish(thirdReduce) == 6) + assert(aggregator.finish(mergeWithZero1) == 1) + assert(aggregator.finish(mergeWithZero2) == 3) + assert(aggregator.finish(mergeTwoReduced) == 4) + } + + test("requires at least one input row") { + val encoder: ExpressionEncoder[Int] = ExpressionEncoder() + val func = (v1: Int, v2: Int) => v1 + v2 + val aggregator: ReduceAggregator[Int] = new ReduceAggregator(func)(Encoders.scalaInt) + + intercept[IllegalStateException] { + aggregator.finish(aggregator.zero) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala new file mode 100644 index 000000000000..6acac1a9aa31 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -0,0 +1,550 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.internal + +import java.io.File + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalog.{Column, Database, Function, Table} +import org.apache.spark.sql.catalyst.{FunctionIdentifier, ScalaReflection, TableIdentifier} +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} +import org.apache.spark.sql.catalyst.plans.logical.Range +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType +import org.apache.spark.storage.StorageLevel + + +/** + * Tests for the user-facing [[org.apache.spark.sql.catalog.Catalog]]. + */ +class CatalogSuite + extends SparkFunSuite + with BeforeAndAfterEach + with SharedSQLContext { + import testImplicits._ + + private def sessionCatalog: SessionCatalog = spark.sessionState.catalog + + private val utils = new CatalogTestUtils { + override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat" + override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat" + override val defaultProvider: String = "parquet" + override def newEmptyCatalog(): ExternalCatalog = spark.sharedState.externalCatalog + } + + private def createDatabase(name: String): Unit = { + sessionCatalog.createDatabase(utils.newDb(name), ignoreIfExists = false) + } + + private def dropDatabase(name: String): Unit = { + sessionCatalog.dropDatabase(name, ignoreIfNotExists = false, cascade = true) + } + + private def createTable(name: String, db: Option[String] = None): Unit = { + sessionCatalog.createTable(utils.newTable(name, db), ignoreIfExists = false) + } + + private def createTempTable(name: String): Unit = { + sessionCatalog.createTempView(name, Range(1, 2, 3, 4), overrideIfExists = true) + } + + private def dropTable(name: String, db: Option[String] = None): Unit = { + sessionCatalog.dropTable(TableIdentifier(name, db), ignoreIfNotExists = false, purge = false) + } + + private def createFunction(name: String, db: Option[String] = None): Unit = { + sessionCatalog.createFunction(utils.newFunc(name, db), ignoreIfExists = false) + } + + private def createTempFunction(name: String): Unit = { + val tempFunc = (e: Seq[Expression]) => e.head + val funcMeta = CatalogFunction(FunctionIdentifier(name, None), "className", Nil) + sessionCatalog.registerFunction( + funcMeta, overrideIfExists = false, functionBuilder = Some(tempFunc)) + } + + private def dropFunction(name: String, db: Option[String] = None): Unit = { + sessionCatalog.dropFunction(FunctionIdentifier(name, db), ignoreIfNotExists = false) + } + + private def dropTempFunction(name: String): Unit = { + sessionCatalog.dropTempFunction(name, ignoreIfNotExists = false) + } + + private def testListColumns(tableName: String, dbName: Option[String]): Unit = { + val tableMetadata = sessionCatalog.getTableMetadata(TableIdentifier(tableName, dbName)) + val columns = dbName + .map { db => spark.catalog.listColumns(db, tableName) } + .getOrElse { spark.catalog.listColumns(tableName) } + assume(tableMetadata.schema.nonEmpty, "bad test") + assume(tableMetadata.partitionColumnNames.nonEmpty, "bad test") + assume(tableMetadata.bucketSpec.isDefined, "bad test") + assert(columns.collect().map(_.name).toSet == tableMetadata.schema.map(_.name).toSet) + val bucketColumnNames = tableMetadata.bucketSpec.map(_.bucketColumnNames).getOrElse(Nil).toSet + columns.collect().foreach { col => + assert(col.isPartition == tableMetadata.partitionColumnNames.contains(col.name)) + assert(col.isBucket == bucketColumnNames.contains(col.name)) + } + + dbName.foreach { db => + val expected = columns.collect().map(_.name).toSet + assert(spark.catalog.listColumns(s"$db.$tableName").collect().map(_.name).toSet == expected) + } + } + + override def afterEach(): Unit = { + try { + sessionCatalog.reset() + } finally { + super.afterEach() + } + } + + test("current database") { + assert(spark.catalog.currentDatabase == "default") + assert(sessionCatalog.getCurrentDatabase == "default") + createDatabase("my_db") + spark.catalog.setCurrentDatabase("my_db") + assert(spark.catalog.currentDatabase == "my_db") + assert(sessionCatalog.getCurrentDatabase == "my_db") + val e = intercept[AnalysisException] { + spark.catalog.setCurrentDatabase("unknown_db") + } + assert(e.getMessage.contains("unknown_db")) + } + + test("list databases") { + assert(spark.catalog.listDatabases().collect().map(_.name).toSet == Set("default")) + createDatabase("my_db1") + createDatabase("my_db2") + assert(spark.catalog.listDatabases().collect().map(_.name).toSet == + Set("default", "my_db1", "my_db2")) + dropDatabase("my_db1") + assert(spark.catalog.listDatabases().collect().map(_.name).toSet == + Set("default", "my_db2")) + } + + test("list tables") { + assert(spark.catalog.listTables().collect().isEmpty) + createTable("my_table1") + createTable("my_table2") + createTempTable("my_temp_table") + assert(spark.catalog.listTables().collect().map(_.name).toSet == + Set("my_table1", "my_table2", "my_temp_table")) + dropTable("my_table1") + assert(spark.catalog.listTables().collect().map(_.name).toSet == + Set("my_table2", "my_temp_table")) + dropTable("my_temp_table") + assert(spark.catalog.listTables().collect().map(_.name).toSet == Set("my_table2")) + } + + test("list tables with database") { + assert(spark.catalog.listTables("default").collect().isEmpty) + createDatabase("my_db1") + createDatabase("my_db2") + createTable("my_table1", Some("my_db1")) + createTable("my_table2", Some("my_db2")) + createTempTable("my_temp_table") + assert(spark.catalog.listTables("default").collect().map(_.name).toSet == + Set("my_temp_table")) + assert(spark.catalog.listTables("my_db1").collect().map(_.name).toSet == + Set("my_table1", "my_temp_table")) + assert(spark.catalog.listTables("my_db2").collect().map(_.name).toSet == + Set("my_table2", "my_temp_table")) + dropTable("my_table1", Some("my_db1")) + assert(spark.catalog.listTables("my_db1").collect().map(_.name).toSet == + Set("my_temp_table")) + assert(spark.catalog.listTables("my_db2").collect().map(_.name).toSet == + Set("my_table2", "my_temp_table")) + dropTable("my_temp_table") + assert(spark.catalog.listTables("default").collect().map(_.name).isEmpty) + assert(spark.catalog.listTables("my_db1").collect().map(_.name).isEmpty) + assert(spark.catalog.listTables("my_db2").collect().map(_.name).toSet == + Set("my_table2")) + val e = intercept[AnalysisException] { + spark.catalog.listTables("unknown_db") + } + assert(e.getMessage.contains("unknown_db")) + } + + test("list functions") { + assert(Set("+", "current_database", "window").subsetOf( + spark.catalog.listFunctions().collect().map(_.name).toSet)) + createFunction("my_func1") + createFunction("my_func2") + createTempFunction("my_temp_func") + val funcNames1 = spark.catalog.listFunctions().collect().map(_.name).toSet + assert(funcNames1.contains("my_func1")) + assert(funcNames1.contains("my_func2")) + assert(funcNames1.contains("my_temp_func")) + dropFunction("my_func1") + dropTempFunction("my_temp_func") + val funcNames2 = spark.catalog.listFunctions().collect().map(_.name).toSet + assert(!funcNames2.contains("my_func1")) + assert(funcNames2.contains("my_func2")) + assert(!funcNames2.contains("my_temp_func")) + } + + test("list functions with database") { + assert(Set("+", "current_database", "window").subsetOf( + spark.catalog.listFunctions().collect().map(_.name).toSet)) + createDatabase("my_db1") + createDatabase("my_db2") + createFunction("my_func1", Some("my_db1")) + createFunction("my_func2", Some("my_db2")) + createTempFunction("my_temp_func") + val funcNames1 = spark.catalog.listFunctions("my_db1").collect().map(_.name).toSet + val funcNames2 = spark.catalog.listFunctions("my_db2").collect().map(_.name).toSet + assert(funcNames1.contains("my_func1")) + assert(!funcNames1.contains("my_func2")) + assert(funcNames1.contains("my_temp_func")) + assert(!funcNames2.contains("my_func1")) + assert(funcNames2.contains("my_func2")) + assert(funcNames2.contains("my_temp_func")) + + // Make sure database is set properly. + assert( + spark.catalog.listFunctions("my_db1").collect().map(_.database).toSet == Set("my_db1", null)) + assert( + spark.catalog.listFunctions("my_db2").collect().map(_.database).toSet == Set("my_db2", null)) + + // Remove the function and make sure they no longer appear. + dropFunction("my_func1", Some("my_db1")) + dropTempFunction("my_temp_func") + val funcNames1b = spark.catalog.listFunctions("my_db1").collect().map(_.name).toSet + val funcNames2b = spark.catalog.listFunctions("my_db2").collect().map(_.name).toSet + assert(!funcNames1b.contains("my_func1")) + assert(!funcNames1b.contains("my_temp_func")) + assert(funcNames2b.contains("my_func2")) + assert(!funcNames2b.contains("my_temp_func")) + val e = intercept[AnalysisException] { + spark.catalog.listFunctions("unknown_db") + } + assert(e.getMessage.contains("unknown_db")) + } + + test("list columns") { + createTable("tab1") + testListColumns("tab1", dbName = None) + } + + test("list columns in temporary table") { + createTempTable("temp1") + spark.catalog.listColumns("temp1") + } + + test("list columns in database") { + createDatabase("db1") + createTable("tab1", Some("db1")) + testListColumns("tab1", dbName = Some("db1")) + } + + test("Database.toString") { + assert(new Database("cool_db", "cool_desc", "cool_path").toString == + "Database[name='cool_db', description='cool_desc', path='cool_path']") + assert(new Database("cool_db", null, "cool_path").toString == + "Database[name='cool_db', path='cool_path']") + } + + test("Table.toString") { + assert(new Table("volley", "databasa", "one", "world", isTemporary = true).toString == + "Table[name='volley', database='databasa', description='one', " + + "tableType='world', isTemporary='true']") + assert(new Table("volley", null, null, "world", isTemporary = true).toString == + "Table[name='volley', tableType='world', isTemporary='true']") + } + + test("Function.toString") { + assert( + new Function("nama", "databasa", "commenta", "classNameAh", isTemporary = true).toString == + "Function[name='nama', database='databasa', description='commenta', " + + "className='classNameAh', isTemporary='true']") + assert(new Function("nama", null, null, "classNameAh", isTemporary = false).toString == + "Function[name='nama', className='classNameAh', isTemporary='false']") + } + + test("Column.toString") { + assert(new Column("namama", "descaca", "datatapa", + nullable = true, isPartition = false, isBucket = true).toString == + "Column[name='namama', description='descaca', dataType='datatapa', " + + "nullable='true', isPartition='false', isBucket='true']") + assert(new Column("namama", null, "datatapa", + nullable = false, isPartition = true, isBucket = true).toString == + "Column[name='namama', dataType='datatapa', " + + "nullable='false', isPartition='true', isBucket='true']") + } + + test("catalog classes format in Dataset.show") { + val db = new Database("nama", "descripta", "locata") + val table = new Table("nama", "databasa", "descripta", "typa", isTemporary = false) + val function = new Function("nama", "databasa", "descripta", "classa", isTemporary = false) + val column = new Column( + "nama", "descripta", "typa", nullable = false, isPartition = true, isBucket = true) + val dbFields = ScalaReflection.getConstructorParameterValues(db) + val tableFields = ScalaReflection.getConstructorParameterValues(table) + val functionFields = ScalaReflection.getConstructorParameterValues(function) + val columnFields = ScalaReflection.getConstructorParameterValues(column) + assert(dbFields == Seq("nama", "descripta", "locata")) + assert(tableFields == Seq("nama", "databasa", "descripta", "typa", false)) + assert(functionFields == Seq("nama", "databasa", "descripta", "classa", false)) + assert(columnFields == Seq("nama", "descripta", "typa", false, true, true)) + val dbString = CatalogImpl.makeDataset(Seq(db), spark).showString(10) + val tableString = CatalogImpl.makeDataset(Seq(table), spark).showString(10) + val functionString = CatalogImpl.makeDataset(Seq(function), spark).showString(10) + val columnString = CatalogImpl.makeDataset(Seq(column), spark).showString(10) + dbFields.foreach { f => assert(dbString.contains(f.toString)) } + tableFields.foreach { f => assert(tableString.contains(f.toString)) } + functionFields.foreach { f => assert(functionString.contains(f.toString)) } + columnFields.foreach { f => assert(columnString.contains(f.toString)) } + } + + test("dropTempView should not un-cache and drop metastore table if a same-name table exists") { + withTable("same_name") { + spark.range(10).write.saveAsTable("same_name") + sql("CACHE TABLE same_name") + assert(spark.catalog.isCached("default.same_name")) + spark.catalog.dropTempView("same_name") + assert(spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + assert(spark.catalog.isCached("default.same_name")) + } + } + + test("get database") { + intercept[AnalysisException](spark.catalog.getDatabase("db10")) + withTempDatabase { db => + assert(spark.catalog.getDatabase(db).name === db) + } + } + + test("get table") { + withTempDatabase { db => + withTable(s"tbl_x", s"$db.tbl_y") { + // Try to find non existing tables. + intercept[AnalysisException](spark.catalog.getTable("tbl_x")) + intercept[AnalysisException](spark.catalog.getTable("tbl_y")) + intercept[AnalysisException](spark.catalog.getTable(db, "tbl_y")) + + // Create objects. + createTempTable("tbl_x") + createTable("tbl_y", Some(db)) + + // Find a temporary table + assert(spark.catalog.getTable("tbl_x").name === "tbl_x") + + // Find a qualified table + assert(spark.catalog.getTable(db, "tbl_y").name === "tbl_y") + assert(spark.catalog.getTable(s"$db.tbl_y").name === "tbl_y") + + // Find an unqualified table using the current database + intercept[AnalysisException](spark.catalog.getTable("tbl_y")) + spark.catalog.setCurrentDatabase(db) + assert(spark.catalog.getTable("tbl_y").name === "tbl_y") + } + } + } + + test("get function") { + withTempDatabase { db => + withUserDefinedFunction("fn1" -> true, s"$db.fn2" -> false) { + // Try to find non existing functions. + intercept[AnalysisException](spark.catalog.getFunction("fn1")) + intercept[AnalysisException](spark.catalog.getFunction(db, "fn1")) + intercept[AnalysisException](spark.catalog.getFunction("fn2")) + intercept[AnalysisException](spark.catalog.getFunction(db, "fn2")) + + // Create objects. + createTempFunction("fn1") + createFunction("fn2", Some(db)) + + // Find a temporary function + val fn1 = spark.catalog.getFunction("fn1") + assert(fn1.name === "fn1") + assert(fn1.database === null) + assert(fn1.isTemporary) + // Find a temporary function with database + intercept[AnalysisException](spark.catalog.getFunction(db, "fn1")) + + // Find a qualified function + val fn2 = spark.catalog.getFunction(db, "fn2") + assert(fn2.name === "fn2") + assert(fn2.database === db) + assert(!fn2.isTemporary) + + val fn2WithQualifiedName = spark.catalog.getFunction(s"$db.fn2") + assert(fn2WithQualifiedName.name === "fn2") + assert(fn2WithQualifiedName.database === db) + assert(!fn2WithQualifiedName.isTemporary) + + // Find an unqualified function using the current database + intercept[AnalysisException](spark.catalog.getFunction("fn2")) + spark.catalog.setCurrentDatabase(db) + val unqualified = spark.catalog.getFunction("fn2") + assert(unqualified.name === "fn2") + assert(unqualified.database === db) + assert(!unqualified.isTemporary) + } + } + } + + test("database exists") { + assert(!spark.catalog.databaseExists("db10")) + createDatabase("db10") + assert(spark.catalog.databaseExists("db10")) + dropDatabase("db10") + } + + test("table exists") { + withTempDatabase { db => + withTable(s"tbl_x", s"$db.tbl_y") { + // Try to find non existing tables. + assert(!spark.catalog.tableExists("tbl_x")) + assert(!spark.catalog.tableExists("tbl_y")) + assert(!spark.catalog.tableExists(db, "tbl_y")) + assert(!spark.catalog.tableExists(s"$db.tbl_y")) + + // Create objects. + createTempTable("tbl_x") + createTable("tbl_y", Some(db)) + + // Find a temporary table + assert(spark.catalog.tableExists("tbl_x")) + + // Find a qualified table + assert(spark.catalog.tableExists(db, "tbl_y")) + assert(spark.catalog.tableExists(s"$db.tbl_y")) + + // Find an unqualified table using the current database + assert(!spark.catalog.tableExists("tbl_y")) + spark.catalog.setCurrentDatabase(db) + assert(spark.catalog.tableExists("tbl_y")) + + // Unable to find the table, although the temp view with the given name exists + assert(!spark.catalog.tableExists(db, "tbl_x")) + } + } + } + + test("function exists") { + withTempDatabase { db => + withUserDefinedFunction("fn1" -> true, s"$db.fn2" -> false) { + // Try to find non existing functions. + assert(!spark.catalog.functionExists("fn1")) + assert(!spark.catalog.functionExists("fn2")) + assert(!spark.catalog.functionExists(db, "fn2")) + assert(!spark.catalog.functionExists(s"$db.fn2")) + + // Create objects. + createTempFunction("fn1") + createFunction("fn2", Some(db)) + + // Find a temporary function + assert(spark.catalog.functionExists("fn1")) + assert(!spark.catalog.functionExists(db, "fn1")) + + // Find a qualified function + assert(spark.catalog.functionExists(db, "fn2")) + assert(spark.catalog.functionExists(s"$db.fn2")) + + // Find an unqualified function using the current database + assert(!spark.catalog.functionExists("fn2")) + spark.catalog.setCurrentDatabase(db) + assert(spark.catalog.functionExists("fn2")) + + // Unable to find the function, although the temp function with the given name exists + assert(!spark.catalog.functionExists(db, "fn1")) + } + } + } + + test("createTable with 'path' in options") { + withTable("t") { + withTempDir { dir => + spark.catalog.createTable( + tableName = "t", + source = "json", + schema = new StructType().add("i", "int"), + options = Map("path" -> dir.getAbsolutePath)) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.tableType == CatalogTableType.EXTERNAL) + assert(table.storage.locationUri.get == makeQualifiedPath(dir.getAbsolutePath)) + + Seq((1)).toDF("i").write.insertInto("t") + assert(dir.exists() && dir.listFiles().nonEmpty) + + sql("DROP TABLE t") + // the table path and data files are still there after DROP TABLE, if custom table path is + // specified. + assert(dir.exists() && dir.listFiles().nonEmpty) + } + } + } + + test("createTable without 'path' in options") { + withTable("t") { + spark.catalog.createTable( + tableName = "t", + source = "json", + schema = new StructType().add("i", "int"), + options = Map.empty[String, String]) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.tableType == CatalogTableType.MANAGED) + val tablePath = new File(table.storage.locationUri.get) + assert(tablePath.exists() && tablePath.listFiles().isEmpty) + + Seq((1)).toDF("i").write.insertInto("t") + assert(tablePath.listFiles().nonEmpty) + + sql("DROP TABLE t") + // the table path is removed after DROP TABLE, if custom table path is not specified. + assert(!tablePath.exists()) + } + } + + test("clone Catalog") { + // need to test tempTables are cloned + assert(spark.catalog.listTables().collect().isEmpty) + + createTempTable("my_temp_table") + assert(spark.catalog.listTables().collect().map(_.name).toSet == Set("my_temp_table")) + + // inheritance + val forkedSession = spark.cloneSession() + assert(spark ne forkedSession) + assert(spark.catalog ne forkedSession.catalog) + assert(forkedSession.catalog.listTables().collect().map(_.name).toSet == Set("my_temp_table")) + + // independence + dropTable("my_temp_table") // drop table in original session + assert(spark.catalog.listTables().collect().map(_.name).toSet == Set()) + assert(forkedSession.catalog.listTables().collect().map(_.name).toSet == Set("my_temp_table")) + forkedSession.sessionState.catalog + .createTempView("fork_table", Range(1, 2, 3, 4), overrideIfExists = true) + assert(spark.catalog.listTables().collect().map(_.name).toSet == Set()) + } + + test("cacheTable with storage level") { + createTempTable("my_temp_table") + spark.catalog.cacheTable("my_temp_table", StorageLevel.DISK_ONLY) + assert(spark.table("my_temp_table").storageLevel == StorageLevel.DISK_ONLY) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala new file mode 100644 index 000000000000..135370bd1d67 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.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.sql.internal + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.internal.SQLConf._ + +class SQLConfEntrySuite extends SparkFunSuite { + + val conf = new SQLConf + + test("intConf") { + val key = "spark.sql.SQLConfEntrySuite.int" + val confEntry = buildConf(key).intConf.createWithDefault(1) + assert(conf.getConf(confEntry, 5) === 5) + + conf.setConf(confEntry, 10) + assert(conf.getConf(confEntry, 5) === 10) + + conf.setConfString(key, "20") + assert(conf.getConfString(key, "5") === "20") + assert(conf.getConfString(key) === "20") + assert(conf.getConf(confEntry, 5) === 20) + + conf.setConfString(key, " 20") + assert(conf.getConf(confEntry, 5) === 20) + + val e = intercept[IllegalArgumentException] { + conf.setConfString(key, "abc") + } + assert(e.getMessage === s"$key should be int, but was abc") + } + + test("longConf") { + val key = "spark.sql.SQLConfEntrySuite.long" + val confEntry = buildConf(key).longConf.createWithDefault(1L) + assert(conf.getConf(confEntry, 5L) === 5L) + + conf.setConf(confEntry, 10L) + assert(conf.getConf(confEntry, 5L) === 10L) + + conf.setConfString(key, "20") + assert(conf.getConfString(key, "5") === "20") + assert(conf.getConfString(key) === "20") + assert(conf.getConf(confEntry, 5L) === 20L) + + val e = intercept[IllegalArgumentException] { + conf.setConfString(key, "abc") + } + assert(e.getMessage === s"$key should be long, but was abc") + } + + test("booleanConf") { + val key = "spark.sql.SQLConfEntrySuite.boolean" + val confEntry = buildConf(key).booleanConf.createWithDefault(true) + assert(conf.getConf(confEntry, false) === false) + + conf.setConf(confEntry, true) + assert(conf.getConf(confEntry, false) === true) + + conf.setConfString(key, "true") + assert(conf.getConfString(key, "false") === "true") + assert(conf.getConfString(key) === "true") + assert(conf.getConf(confEntry, false) === true) + + conf.setConfString(key, " true ") + assert(conf.getConf(confEntry, false) === true) + val e = intercept[IllegalArgumentException] { + conf.setConfString(key, "abc") + } + assert(e.getMessage === s"$key should be boolean, but was abc") + } + + test("doubleConf") { + val key = "spark.sql.SQLConfEntrySuite.double" + val confEntry = buildConf(key).doubleConf.createWithDefault(1d) + assert(conf.getConf(confEntry, 5.0) === 5.0) + + conf.setConf(confEntry, 10.0) + assert(conf.getConf(confEntry, 5.0) === 10.0) + + conf.setConfString(key, "20.0") + assert(conf.getConfString(key, "5.0") === "20.0") + assert(conf.getConfString(key) === "20.0") + assert(conf.getConf(confEntry, 5.0) === 20.0) + + val e = intercept[IllegalArgumentException] { + conf.setConfString(key, "abc") + } + assert(e.getMessage === s"$key should be double, but was abc") + } + + test("stringConf") { + val key = "spark.sql.SQLConfEntrySuite.string" + val confEntry = buildConf(key).stringConf.createWithDefault(null) + assert(conf.getConf(confEntry, "abc") === "abc") + + conf.setConf(confEntry, "abcd") + assert(conf.getConf(confEntry, "abc") === "abcd") + + conf.setConfString(key, "abcde") + assert(conf.getConfString(key, "abc") === "abcde") + assert(conf.getConfString(key) === "abcde") + assert(conf.getConf(confEntry, "abc") === "abcde") + } + + test("enumConf") { + val key = "spark.sql.SQLConfEntrySuite.enum" + val confEntry = buildConf(key) + .stringConf + .checkValues(Set("a", "b", "c")) + .createWithDefault("a") + assert(conf.getConf(confEntry) === "a") + + conf.setConf(confEntry, "b") + assert(conf.getConf(confEntry) === "b") + + conf.setConfString(key, "c") + assert(conf.getConfString(key, "a") === "c") + assert(conf.getConfString(key) === "c") + assert(conf.getConf(confEntry) === "c") + + val e = intercept[IllegalArgumentException] { + conf.setConfString(key, "d") + } + assert(e.getMessage === s"The value of $key should be one of a, b, c, but was d") + } + + test("stringSeqConf") { + val key = "spark.sql.SQLConfEntrySuite.stringSeq" + val confEntry = buildConf(key) + .stringConf + .toSequence + .createWithDefault(Nil) + assert(conf.getConf(confEntry, Seq("a", "b", "c")) === Seq("a", "b", "c")) + + conf.setConf(confEntry, Seq("a", "b", "c", "d")) + assert(conf.getConf(confEntry, Seq("a", "b", "c")) === Seq("a", "b", "c", "d")) + + conf.setConfString(key, "a,b,c,d,e") + assert(conf.getConfString(key, "a,b,c") === "a,b,c,d,e") + assert(conf.getConfString(key) === "a,b,c,d,e") + assert(conf.getConf(confEntry, Seq("a", "b", "c")) === Seq("a", "b", "c", "d", "e")) + } + + test("optionalConf") { + val key = "spark.sql.SQLConfEntrySuite.optional" + val confEntry = buildConf(key) + .stringConf + .createOptional + + assert(conf.getConf(confEntry) === None) + conf.setConfString(key, "a") + assert(conf.getConf(confEntry) === Some("a")) + } + + test("duplicate entry") { + val key = "spark.sql.SQLConfEntrySuite.duplicate" + buildConf(key).stringConf.createOptional + intercept[IllegalArgumentException] { + buildConf(key).stringConf.createOptional + } + } + + test("StaticSQLConf.FILESOURCE_TABLE_RELATION_CACHE_SIZE") { + val confEntry = StaticSQLConf.FILESOURCE_TABLE_RELATION_CACHE_SIZE + assert(conf.getConf(confEntry) === 1000) + + conf.setConf(confEntry, -1) + val e1 = intercept[IllegalArgumentException] { + conf.getConf(confEntry) + } + assert(e1.getMessage === "The maximum size of the cache must not be negative") + + val e2 = intercept[IllegalArgumentException] { + conf.setConfString(confEntry.key, "-1") + } + assert(e2.getMessage === "The maximum size of the cache must not be negative") + } + + test("clone SQLConf") { + val original = new SQLConf + val key = "spark.sql.SQLConfEntrySuite.clone" + assert(original.getConfString(key, "noentry") === "noentry") + + // inheritance + original.setConfString(key, "orig") + val clone = original.clone() + assert(original ne clone) + assert(clone.getConfString(key, "noentry") === "orig") + + // independence + clone.setConfString(key, "clone") + assert(original.getConfString(key, "noentry") === "orig") + original.setConfString(key, "dontcopyme") + assert(clone.getConfString(key, "noentry") === "clone") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala new file mode 100644 index 000000000000..205c303b6cc4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -0,0 +1,284 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.internal + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql._ +import org.apache.spark.sql.execution.WholeStageCodegenExec +import org.apache.spark.sql.internal.StaticSQLConf._ +import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} +import org.apache.spark.util.Utils + +class SQLConfSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + private val testKey = "test.key.0" + private val testVal = "test.val.0" + + test("propagate from spark conf") { + // We create a new context here to avoid order dependence with other tests that might call + // clear(). + val newContext = new SQLContext(SparkSession.builder().sparkContext(sparkContext).getOrCreate()) + assert(newContext.getConf("spark.sql.testkey", "false") === "true") + } + + test("programmatic ways of basic setting and getting") { + // Set a conf first. + spark.conf.set(testKey, testVal) + // Clear the conf. + spark.sessionState.conf.clear() + // After clear, only overrideConfs used by unit test should be in the SQLConf. + assert(spark.conf.getAll === TestSQLContext.overrideConfs) + + spark.conf.set(testKey, testVal) + assert(spark.conf.get(testKey) === testVal) + assert(spark.conf.get(testKey, testVal + "_") === testVal) + assert(spark.conf.getAll.contains(testKey)) + + // Tests SQLConf as accessed from a SQLContext is mutable after + // the latter is initialized, unlike SparkConf inside a SparkContext. + assert(spark.conf.get(testKey) === testVal) + assert(spark.conf.get(testKey, testVal + "_") === testVal) + assert(spark.conf.getAll.contains(testKey)) + + spark.sessionState.conf.clear() + } + + test("parse SQL set commands") { + spark.sessionState.conf.clear() + sql(s"set $testKey=$testVal") + assert(spark.conf.get(testKey, testVal + "_") === testVal) + assert(spark.conf.get(testKey, testVal + "_") === testVal) + + sql("set some.property=20") + assert(spark.conf.get("some.property", "0") === "20") + sql("set some.property = 40") + assert(spark.conf.get("some.property", "0") === "40") + + val key = "spark.sql.key" + val vs = "val0,val_1,val2.3,my_table" + sql(s"set $key=$vs") + assert(spark.conf.get(key, "0") === vs) + + sql(s"set $key=") + assert(spark.conf.get(key, "0") === "") + + spark.sessionState.conf.clear() + } + + test("set command for display") { + spark.sessionState.conf.clear() + checkAnswer( + sql("SET").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Nil) + + checkAnswer( + sql("SET -v").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Row("spark.sql.groupByOrdinal", "true")) + + sql("SET spark.sql.groupByOrdinal=false") + + checkAnswer( + sql("SET").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Row("spark.sql.groupByOrdinal", "false")) + + checkAnswer( + sql("SET -v").where("key = 'spark.sql.groupByOrdinal'").select("key", "value"), + Row("spark.sql.groupByOrdinal", "false")) + } + + test("deprecated property") { + spark.sessionState.conf.clear() + val original = spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) + try { + sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") + assert(spark.conf.get(SQLConf.SHUFFLE_PARTITIONS) === 10) + } finally { + sql(s"set ${SQLConf.SHUFFLE_PARTITIONS}=$original") + } + } + + test("reset - public conf") { + spark.sessionState.conf.clear() + val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) + try { + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) + sql(s"set ${SQLConf.GROUP_BY_ORDINAL.key}=false") + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === false) + assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 1) + sql(s"reset") + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) + assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 0) + } finally { + sql(s"set ${SQLConf.GROUP_BY_ORDINAL}=$original") + } + } + + test("reset - internal conf") { + spark.sessionState.conf.clear() + val original = spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) + try { + assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 100) + sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}=10") + assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 10) + assert(sql(s"set").where(s"key = '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}'").count() == 1) + sql(s"reset") + assert(spark.conf.get(SQLConf.OPTIMIZER_MAX_ITERATIONS) === 100) + assert(sql(s"set").where(s"key = '${SQLConf.OPTIMIZER_MAX_ITERATIONS.key}'").count() == 0) + } finally { + sql(s"set ${SQLConf.OPTIMIZER_MAX_ITERATIONS}=$original") + } + } + + test("reset - user-defined conf") { + spark.sessionState.conf.clear() + val userDefinedConf = "x.y.z.reset" + try { + assert(spark.conf.getOption(userDefinedConf).isEmpty) + sql(s"set $userDefinedConf=false") + assert(spark.conf.get(userDefinedConf) === "false") + assert(sql(s"set").where(s"key = '$userDefinedConf'").count() == 1) + sql(s"reset") + assert(spark.conf.getOption(userDefinedConf).isEmpty) + } finally { + spark.conf.unset(userDefinedConf) + } + } + + test("invalid conf value") { + spark.sessionState.conf.clear() + val e = intercept[IllegalArgumentException] { + sql(s"set ${SQLConf.CASE_SENSITIVE.key}=10") + } + assert(e.getMessage === s"${SQLConf.CASE_SENSITIVE.key} should be boolean, but was 10") + } + + test("Test SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE's method") { + spark.sessionState.conf.clear() + + spark.conf.set(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "100") + assert(spark.conf.get(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) === 100) + + spark.conf.set(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "1k") + assert(spark.conf.get(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) === 1024) + + spark.conf.set(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "1M") + assert(spark.conf.get(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) === 1048576) + + spark.conf.set(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "1g") + assert(spark.conf.get(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) === 1073741824) + + spark.conf.set(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "-1") + assert(spark.conf.get(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE) === -1) + + // Test overflow exception + intercept[IllegalArgumentException] { + // This value exceeds Long.MaxValue + spark.conf.set(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "90000000000g") + } + + intercept[IllegalArgumentException] { + // This value less than Long.MinValue + spark.conf.set(SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, "-90000000000g") + } + + spark.sessionState.conf.clear() + } + + test("SparkSession can access configs set in SparkConf") { + try { + sparkContext.conf.set("spark.to.be.or.not.to.be", "my love") + sparkContext.conf.set("spark.sql.with.or.without.you", "my love") + val spark = new SparkSession(sparkContext) + assert(spark.conf.get("spark.to.be.or.not.to.be") == "my love") + assert(spark.conf.get("spark.sql.with.or.without.you") == "my love") + } finally { + sparkContext.conf.remove("spark.to.be.or.not.to.be") + sparkContext.conf.remove("spark.sql.with.or.without.you") + } + } + + test("default value of WAREHOUSE_PATH") { + // JVM adds a trailing slash if the directory exists and leaves it as-is, if it doesn't + // In our comparison, strip trailing slash off of both sides, to account for such cases + assert(new Path(Utils.resolveURI("spark-warehouse")).toString.stripSuffix("/") === spark + .sessionState.conf.warehousePath.stripSuffix("/")) + } + + test("MAX_CASES_BRANCHES") { + withTable("tab1") { + spark.range(10).write.saveAsTable("tab1") + val sql_one_branch_caseWhen = "SELECT CASE WHEN id = 1 THEN 1 END FROM tab1" + val sql_two_branch_caseWhen = "SELECT CASE WHEN id = 1 THEN 1 ELSE 0 END FROM tab1" + + withSQLConf(SQLConf.MAX_CASES_BRANCHES.key -> "0") { + assert(!sql(sql_one_branch_caseWhen) + .queryExecution.executedPlan.isInstanceOf[WholeStageCodegenExec]) + assert(!sql(sql_two_branch_caseWhen) + .queryExecution.executedPlan.isInstanceOf[WholeStageCodegenExec]) + } + + withSQLConf(SQLConf.MAX_CASES_BRANCHES.key -> "1") { + assert(sql(sql_one_branch_caseWhen) + .queryExecution.executedPlan.isInstanceOf[WholeStageCodegenExec]) + assert(!sql(sql_two_branch_caseWhen) + .queryExecution.executedPlan.isInstanceOf[WholeStageCodegenExec]) + } + + withSQLConf(SQLConf.MAX_CASES_BRANCHES.key -> "2") { + assert(sql(sql_one_branch_caseWhen) + .queryExecution.executedPlan.isInstanceOf[WholeStageCodegenExec]) + assert(sql(sql_two_branch_caseWhen) + .queryExecution.executedPlan.isInstanceOf[WholeStageCodegenExec]) + } + } + } + + test("static SQL conf comes from SparkConf") { + val previousValue = sparkContext.conf.get(SCHEMA_STRING_LENGTH_THRESHOLD) + try { + sparkContext.conf.set(SCHEMA_STRING_LENGTH_THRESHOLD, 2000) + val newSession = new SparkSession(sparkContext) + assert(newSession.conf.get(SCHEMA_STRING_LENGTH_THRESHOLD) == 2000) + checkAnswer( + newSession.sql(s"SET ${SCHEMA_STRING_LENGTH_THRESHOLD.key}"), + Row(SCHEMA_STRING_LENGTH_THRESHOLD.key, "2000")) + } finally { + sparkContext.conf.set(SCHEMA_STRING_LENGTH_THRESHOLD, previousValue) + } + } + + test("cannot set/unset static SQL conf") { + val e1 = intercept[AnalysisException](sql(s"SET ${SCHEMA_STRING_LENGTH_THRESHOLD.key}=10")) + assert(e1.message.contains("Cannot modify the value of a static config")) + val e2 = intercept[AnalysisException](spark.conf.unset(SCHEMA_STRING_LENGTH_THRESHOLD.key)) + assert(e2.message.contains("Cannot modify the value of a static config")) + } + + test("SPARK-21588 SQLContext.getConf(key, null) should return null") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + assert("1" == spark.conf.get(SQLConf.SHUFFLE_PARTITIONS.key, null)) + assert("1" == spark.conf.get(SQLConf.SHUFFLE_PARTITIONS.key, "")) + } + + assert(spark.conf.getOption("spark.sql.nonexistent").isEmpty) + assert(null == spark.conf.get("spark.sql.nonexistent", null)) + assert("" == spark.conf.get("spark.sql.nonexistent", "")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/VariableSubstitutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/VariableSubstitutionSuite.scala new file mode 100644 index 000000000000..d5a946aeaac3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/VariableSubstitutionSuite.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.internal + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException + +class VariableSubstitutionSuite extends SparkFunSuite { + + private lazy val conf = new SQLConf + private lazy val sub = new VariableSubstitution(conf) + + test("system property") { + System.setProperty("varSubSuite.var", "abcd") + assert(sub.substitute("${system:varSubSuite.var}") == "abcd") + } + + test("environmental variables") { + assert(sub.substitute("${env:SPARK_TESTING}") == "1") + } + + test("Spark configuration variable") { + conf.setConfString("some-random-string-abcd", "1234abcd") + assert(sub.substitute("${hiveconf:some-random-string-abcd}") == "1234abcd") + assert(sub.substitute("${sparkconf:some-random-string-abcd}") == "1234abcd") + assert(sub.substitute("${spark:some-random-string-abcd}") == "1234abcd") + assert(sub.substitute("${some-random-string-abcd}") == "1234abcd") + } + + test("multiple substitutes") { + val q = "select ${bar} ${foo} ${doo} this is great" + conf.setConfString("bar", "1") + conf.setConfString("foo", "2") + conf.setConfString("doo", "3") + assert(sub.substitute(q) == "select 1 2 3 this is great") + } + + test("test nested substitutes") { + val q = "select ${bar} ${foo} this is great" + conf.setConfString("bar", "1") + conf.setConfString("foo", "${bar}") + assert(sub.substitute(q) == "select 1 1 this is great") + } + +} 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 d530b1a469ce..34205e0b2bf0 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,18 +18,28 @@ package org.apache.spark.sql.jdbc import java.math.BigDecimal -import java.sql.DriverManager +import java.sql.{Date, DriverManager, SQLException, Timestamp} import java.util.{Calendar, GregorianCalendar, Properties} import org.h2.jdbc.JdbcSQLException -import org.scalatest.BeforeAndAfter +import org.scalatest.{BeforeAndAfter, PrivateMethodTester} -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.sql.{AnalysisException, DataFrame, Row} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.execution.DataSourceScanExec +import org.apache.spark.sql.execution.command.ExplainCommand +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JDBCRDD, JDBCRelation, JdbcUtils} +import org.apache.spark.sql.execution.metric.InputOutputMetricsHelper +import org.apache.spark.sql.sources._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.util.Utils -class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext { +class JDBCSuite extends SparkFunSuite + with BeforeAndAfter with PrivateMethodTester with SharedSQLContext { import testImplicits._ val url = "jdbc:h2:mem:testdb0" @@ -66,26 +76,35 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext sql( s""" - |CREATE TEMPORARY TABLE foobar + |CREATE OR REPLACE TEMPORARY VIEW foobar |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass') - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin.replaceAll("\n", " ")) sql( s""" - |CREATE TEMPORARY TABLE fetchtwo + |CREATE OR REPLACE TEMPORARY VIEW fetchtwo |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass', - | fetchSize '2') - """.stripMargin.replaceAll("\n", " ")) + | ${JDBCOptions.JDBC_BATCH_FETCH_SIZE} '2') + """.stripMargin.replaceAll("\n", " ")) sql( s""" - |CREATE TEMPORARY TABLE parts + |CREATE OR REPLACE TEMPORARY VIEW parts |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass', | partitionColumn 'THEID', lowerBound '1', upperBound '4', numPartitions '3') - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin.replaceAll("\n", " ")) + + sql( + s""" + |CREATE OR REPLACE TEMPORARY VIEW partsoverflow + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass', + | partitionColumn 'THEID', lowerBound '-9223372036854775808', + | upperBound '9223372036854775807', numPartitions '3') + """.stripMargin.replaceAll("\n", " ")) conn.prepareStatement("create table test.inttypes (a INT, b BOOLEAN, c TINYINT, " + "d SMALLINT, e BIGINT)").executeUpdate() @@ -96,10 +115,10 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext conn.commit() sql( s""" - |CREATE TEMPORARY TABLE inttypes + |CREATE OR REPLACE TEMPORARY VIEW inttypes |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url', dbtable 'TEST.INTTYPES', user 'testUser', password 'testPass') - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin.replaceAll("\n", " ")) conn.prepareStatement("create table test.strtypes (a BINARY(20), b VARCHAR(20), " + "c VARCHAR_IGNORECASE(20), d CHAR(20), e BLOB, f CLOB)").executeUpdate() @@ -113,10 +132,10 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext stmt.executeUpdate() sql( s""" - |CREATE TEMPORARY TABLE strtypes + |CREATE OR REPLACE TEMPORARY VIEW strtypes |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url', dbtable 'TEST.STRTYPES', user 'testUser', password 'testPass') - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin.replaceAll("\n", " ")) conn.prepareStatement("create table test.timetypes (a TIME, b DATE, c TIMESTAMP)" ).executeUpdate() @@ -127,11 +146,20 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext conn.commit() sql( s""" - |CREATE TEMPORARY TABLE timetypes + |CREATE OR REPLACE TEMPORARY VIEW timetypes |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url', dbtable 'TEST.TIMETYPES', user 'testUser', password 'testPass') - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin.replaceAll("\n", " ")) + conn.prepareStatement("CREATE TABLE test.timezone (tz TIMESTAMP WITH TIME ZONE) " + + "AS SELECT '1999-01-08 04:05:06.543543543 GMT-08:00'") + .executeUpdate() + conn.commit() + + conn.prepareStatement("CREATE TABLE test.array (ar ARRAY) " + + "AS SELECT '(1, 2, 3)'") + .executeUpdate() + conn.commit() conn.prepareStatement("create table test.flttypes (a DOUBLE, b REAL, c DECIMAL(38, 18))" ).executeUpdate() @@ -142,27 +170,73 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext conn.commit() sql( s""" - |CREATE TEMPORARY TABLE flttypes + |CREATE OR REPLACE TEMPORARY VIEW flttypes |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url', dbtable 'TEST.FLTTYPES', user 'testUser', password 'testPass') - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin.replaceAll("\n", " ")) conn.prepareStatement( s""" |create table test.nulltypes (a INT, b BOOLEAN, c TINYINT, d BINARY(20), e VARCHAR(20), |f VARCHAR_IGNORECASE(20), g CHAR(20), h BLOB, i CLOB, j TIME, k DATE, l TIMESTAMP, |m DOUBLE, n REAL, o DECIMAL(38, 18)) - """.stripMargin.replaceAll("\n", " ")).executeUpdate() + """.stripMargin.replaceAll("\n", " ")).executeUpdate() conn.prepareStatement("insert into test.nulltypes values (" + "null, null, null, null, null, null, null, null, null, " + "null, null, null, null, null, null)").executeUpdate() conn.commit() sql( s""" - |CREATE TEMPORARY TABLE nulltypes + |CREATE OR REPLACE TEMPORARY VIEW nulltypes |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url', dbtable 'TEST.NULLTYPES', user 'testUser', password 'testPass') - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin.replaceAll("\n", " ")) + + conn.prepareStatement( + "create table test.emp(name TEXT(32) NOT NULL," + + " theid INTEGER, \"Dept\" INTEGER)").executeUpdate() + conn.prepareStatement( + "insert into test.emp values ('fred', 1, 10)").executeUpdate() + conn.prepareStatement( + "insert into test.emp values ('mary', 2, null)").executeUpdate() + conn.prepareStatement( + "insert into test.emp values ('joe ''foo'' \"bar\"', 3, 30)").executeUpdate() + conn.prepareStatement( + "insert into test.emp values ('kathy', null, null)").executeUpdate() + conn.commit() + + conn.prepareStatement( + "create table test.seq(id INTEGER)").executeUpdate() + (0 to 6).foreach { value => + conn.prepareStatement( + s"insert into test.seq values ($value)").executeUpdate() + } + conn.prepareStatement( + "insert into test.seq values (null)").executeUpdate() + conn.commit() + + sql( + s""" + |CREATE OR REPLACE TEMPORARY VIEW nullparts + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.EMP', user 'testUser', password 'testPass', + |partitionColumn '"Dept"', lowerBound '1', upperBound '4', numPartitions '3') + """.stripMargin.replaceAll("\n", " ")) + + conn.prepareStatement( + """create table test."mixedCaseCols" ("Name" TEXT(32), "Id" INTEGER NOT NULL)""") + .executeUpdate() + conn.prepareStatement("""insert into test."mixedCaseCols" values ('fred', 1)""").executeUpdate() + conn.prepareStatement("""insert into test."mixedCaseCols" values ('mary', 2)""").executeUpdate() + conn.prepareStatement("""insert into test."mixedCaseCols" values (null, 3)""").executeUpdate() + conn.commit() + + sql( + s""" + |CREATE OR REPLACE TEMPORARY VIEW mixedCaseCols + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST."mixedCaseCols"', user 'testUser', password 'testPass') + """.stripMargin.replaceAll("\n", " ")) // Untested: IDENTITY, OTHER, UUID, ARRAY, and GEOMETRY types. } @@ -171,17 +245,81 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext conn.close() } + // Check whether the tables are fetched in the expected degree of parallelism + def checkNumPartitions(df: DataFrame, expectedNumPartitions: Int): Unit = { + val jdbcRelations = df.queryExecution.analyzed.collect { + case LogicalRelation(r: JDBCRelation, _, _, _) => r + } + assert(jdbcRelations.length == 1) + assert(jdbcRelations.head.parts.length == expectedNumPartitions, + s"Expecting a JDBCRelation with $expectedNumPartitions partitions, but got:`$jdbcRelations`") + } + test("SELECT *") { assert(sql("SELECT * FROM foobar").collect().size === 3) } test("SELECT * WHERE (simple predicates)") { - assert(sql("SELECT * FROM foobar WHERE THEID < 1").collect().size === 0) - assert(sql("SELECT * FROM foobar WHERE THEID != 2").collect().size === 2) - assert(sql("SELECT * FROM foobar WHERE THEID = 1").collect().size === 1) - assert(sql("SELECT * FROM foobar WHERE NAME = 'fred'").collect().size === 1) - assert(sql("SELECT * FROM foobar WHERE NAME > 'fred'").collect().size === 2) - assert(sql("SELECT * FROM foobar WHERE NAME != 'fred'").collect().size === 2) + def checkPushdown(df: DataFrame): DataFrame = { + val parentPlan = df.queryExecution.executedPlan + // Check if SparkPlan Filter is removed in a physical plan and + // the plan only has PhysicalRDD to scan JDBCRelation. + assert(parentPlan.isInstanceOf[org.apache.spark.sql.execution.WholeStageCodegenExec]) + val node = parentPlan.asInstanceOf[org.apache.spark.sql.execution.WholeStageCodegenExec] + assert(node.child.isInstanceOf[org.apache.spark.sql.execution.DataSourceScanExec]) + assert(node.child.asInstanceOf[DataSourceScanExec].nodeName.contains("JDBCRelation")) + df + } + assert(checkPushdown(sql("SELECT * FROM foobar WHERE THEID < 1")).collect().size == 0) + assert(checkPushdown(sql("SELECT * FROM foobar WHERE THEID != 2")).collect().size == 2) + assert(checkPushdown(sql("SELECT * FROM foobar WHERE THEID = 1")).collect().size == 1) + assert(checkPushdown(sql("SELECT * FROM foobar WHERE NAME = 'fred'")).collect().size == 1) + assert(checkPushdown(sql("SELECT * FROM foobar WHERE NAME <=> 'fred'")).collect().size == 1) + assert(checkPushdown(sql("SELECT * FROM foobar WHERE NAME > 'fred'")).collect().size == 2) + assert(checkPushdown(sql("SELECT * FROM foobar WHERE NAME != 'fred'")).collect().size == 2) + + assert(checkPushdown(sql("SELECT * FROM foobar WHERE NAME IN ('mary', 'fred')")) + .collect().size == 2) + assert(checkPushdown(sql("SELECT * FROM foobar WHERE NAME NOT IN ('fred')")) + .collect().size == 2) + assert(checkPushdown(sql("SELECT * FROM foobar WHERE THEID = 1 OR NAME = 'mary'")) + .collect().size == 2) + assert(checkPushdown(sql("SELECT * FROM foobar WHERE THEID = 1 OR NAME = 'mary' " + + "AND THEID = 2")).collect().size == 2) + assert(checkPushdown(sql("SELECT * FROM foobar WHERE NAME LIKE 'fr%'")).collect().size == 1) + assert(checkPushdown(sql("SELECT * FROM foobar WHERE NAME LIKE '%ed'")).collect().size == 1) + assert(checkPushdown(sql("SELECT * FROM foobar WHERE NAME LIKE '%re%'")).collect().size == 1) + assert(checkPushdown(sql("SELECT * FROM nulltypes WHERE A IS NULL")).collect().size == 1) + assert(checkPushdown(sql("SELECT * FROM nulltypes WHERE A IS NOT NULL")).collect().size == 0) + + // This is a test to reflect discussion in SPARK-12218. + // The older versions of spark have this kind of bugs in parquet data source. + val df1 = sql("SELECT * FROM foobar WHERE NOT (THEID != 2 AND NAME != 'mary')") + val df2 = sql("SELECT * FROM foobar WHERE NOT (THEID != 2) OR NOT (NAME != 'mary')") + assert(df1.collect.toSet === Set(Row("mary", 2))) + assert(df2.collect.toSet === Set(Row("mary", 2))) + + def checkNotPushdown(df: DataFrame): DataFrame = { + val parentPlan = df.queryExecution.executedPlan + // Check if SparkPlan Filter is not removed in a physical plan because JDBCRDD + // cannot compile given predicates. + assert(parentPlan.isInstanceOf[org.apache.spark.sql.execution.WholeStageCodegenExec]) + val node = parentPlan.asInstanceOf[org.apache.spark.sql.execution.WholeStageCodegenExec] + assert(node.child.isInstanceOf[org.apache.spark.sql.execution.FilterExec]) + df + } + assert(checkNotPushdown(sql("SELECT * FROM foobar WHERE (THEID + 1) < 2")).collect().size == 0) + assert(checkNotPushdown(sql("SELECT * FROM foobar WHERE (THEID + 2) != 4")).collect().size == 2) + } + + test("SELECT COUNT(1) WHERE (predicates)") { + // Check if an answer is correct when Filter is removed from operations such as count() which + // does not require any columns. In some data sources, e.g., Parquet, `requiredColumns` in + // org.apache.spark.sql.sources.interfaces is not given in logical plans, but some filters + // are applied for columns with Filter producing wrong results. On the other hand, JDBCRDD + // correctly handles this case by assigning `requiredColumns` properly. See PR 10427 for more + // discussions. + assert(sql("SELECT COUNT(1) FROM foobar WHERE NAME = 'mary'").collect.toSet === Set(Row(1))) } test("SELECT * WHERE (quoted strings)") { @@ -196,7 +334,7 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext assert(names(2).equals("mary")) } - test("SELECT first field when fetchSize is two") { + test("SELECT first field when fetchsize is two") { val names = sql("SELECT NAME FROM fetchtwo").collect().map(x => x.getString(0)).sortWith(_ < _) assert(names.size === 3) assert(names(0).equals("fred")) @@ -212,7 +350,7 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext assert(ids(2) === 3) } - test("SELECT second field when fetchSize is two") { + test("SELECT second field when fetchsize is two") { val ids = sql("SELECT THEID FROM fetchtwo").collect().map(x => x.getInt(0)).sortWith(_ < _) assert(ids.size === 3) assert(ids(0) === 1) @@ -221,13 +359,23 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext } test("SELECT * partitioned") { - assert(sql("SELECT * FROM parts").collect().size == 3) + val df = sql("SELECT * FROM parts") + checkNumPartitions(df, expectedNumPartitions = 3) + assert(df.collect().length == 3) } test("SELECT WHERE (simple predicates) partitioned") { - assert(sql("SELECT * FROM parts WHERE THEID < 1").collect().size === 0) - assert(sql("SELECT * FROM parts WHERE THEID != 2").collect().size === 2) - assert(sql("SELECT THEID FROM parts WHERE THEID = 1").collect().size === 1) + val df1 = sql("SELECT * FROM parts WHERE THEID < 1") + checkNumPartitions(df1, expectedNumPartitions = 3) + assert(df1.collect().length === 0) + + val df2 = sql("SELECT * FROM parts WHERE THEID != 2") + checkNumPartitions(df2, expectedNumPartitions = 3) + assert(df2.collect().length === 2) + + val df3 = sql("SELECT THEID FROM parts WHERE THEID = 1") + checkNumPartitions(df3, expectedNumPartitions = 3) + assert(df3.collect().length === 1) } test("SELECT second field partitioned") { @@ -238,15 +386,21 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext assert(ids(2) === 3) } + test("overflow of partition bound difference does not give negative stride") { + val df = sql("SELECT * FROM partsoverflow") + checkNumPartitions(df, expectedNumPartitions = 3) + assert(df.collect().length == 3) + } + test("Register JDBC query with renamed fields") { // Regression test for bug SPARK-7345 sql( s""" - |CREATE TEMPORARY TABLE renamed + |CREATE OR REPLACE TEMPORARY VIEW renamed |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url', dbtable '(select NAME as NAME1, NAME as NAME2 from TEST.PEOPLE)', |user 'testUser', password 'testPass') - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin.replaceAll("\n", " ")) val df = sql("SELECT * FROM renamed") assert(df.schema.fields.size == 2) @@ -255,27 +409,140 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext } test("Basic API") { - assert(sqlContext.read.jdbc( - urlWithUserAndPass, "TEST.PEOPLE", new Properties).collect().length === 3) + assert(spark.read.jdbc( + urlWithUserAndPass, "TEST.PEOPLE", new Properties()).collect().length === 3) + } + + test("Basic API with illegal fetchsize") { + val properties = new Properties() + properties.setProperty(JDBCOptions.JDBC_BATCH_FETCH_SIZE, "-1") + val e = intercept[IllegalArgumentException] { + spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", properties).collect() + }.getMessage + assert(e.contains("Invalid value `-1` for parameter `fetchsize`")) + } + + test("Missing partition columns") { + withView("tempPeople") { + val e = intercept[IllegalArgumentException] { + sql( + s""" + |CREATE OR REPLACE TEMPORARY VIEW tempPeople + |USING org.apache.spark.sql.jdbc + |OPTIONS ( + | url 'jdbc:h2:mem:testdb0;user=testUser;password=testPass', + | dbtable 'TEST.PEOPLE', + | lowerBound '0', + | upperBound '52', + | numPartitions '53', + | fetchSize '10000' ) + """.stripMargin.replaceAll("\n", " ")) + }.getMessage + assert(e.contains("When reading JDBC data sources, users need to specify all or none " + + "for the following options: 'partitionColumn', 'lowerBound', 'upperBound', and " + + "'numPartitions'")) + } } test("Basic API with FetchSize") { - val properties = new Properties - properties.setProperty("fetchSize", "2") - assert(sqlContext.read.jdbc( - urlWithUserAndPass, "TEST.PEOPLE", properties).collect().length === 3) + (0 to 4).foreach { size => + val properties = new Properties() + properties.setProperty(JDBCOptions.JDBC_BATCH_FETCH_SIZE, size.toString) + assert(spark.read.jdbc( + urlWithUserAndPass, "TEST.PEOPLE", properties).collect().length === 3) + } } test("Partitioning via JDBCPartitioningInfo API") { - assert( - sqlContext.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", "THEID", 0, 4, 3, new Properties) - .collect().length === 3) + val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", "THEID", 0, 4, 3, new Properties()) + checkNumPartitions(df, expectedNumPartitions = 3) + assert(df.collect().length === 3) } test("Partitioning via list-of-where-clauses API") { val parts = Array[String]("THEID < 2", "THEID >= 2") - assert(sqlContext.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, new Properties) - .collect().length === 3) + val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, new Properties()) + checkNumPartitions(df, expectedNumPartitions = 2) + assert(df.collect().length === 3) + } + + test("Partitioning on column that might have null values.") { + val df = spark.read.jdbc(urlWithUserAndPass, "TEST.EMP", "theid", 0, 4, 3, new Properties()) + checkNumPartitions(df, expectedNumPartitions = 3) + assert(df.collect().length === 4) + + val df2 = spark.read.jdbc(urlWithUserAndPass, "TEST.EMP", "THEID", 0, 4, 3, new Properties()) + checkNumPartitions(df2, expectedNumPartitions = 3) + assert(df2.collect().length === 4) + + // partitioning on a nullable quoted column + assert( + spark.read.jdbc(urlWithUserAndPass, "TEST.EMP", """"Dept"""", 0, 4, 3, new Properties()) + .collect().length === 4) + } + + test("Partitioning on column where numPartitions is zero") { + val res = spark.read.jdbc( + url = urlWithUserAndPass, + table = "TEST.seq", + columnName = "id", + lowerBound = 0, + upperBound = 4, + numPartitions = 0, + connectionProperties = new Properties() + ) + checkNumPartitions(res, expectedNumPartitions = 1) + assert(res.count() === 8) + } + + test("Partitioning on column where numPartitions are more than the number of total rows") { + val res = spark.read.jdbc( + url = urlWithUserAndPass, + table = "TEST.seq", + columnName = "id", + lowerBound = 1, + upperBound = 5, + numPartitions = 10, + connectionProperties = new Properties() + ) + checkNumPartitions(res, expectedNumPartitions = 4) + assert(res.count() === 8) + } + + test("Partitioning on column where lowerBound is equal to upperBound") { + val res = spark.read.jdbc( + url = urlWithUserAndPass, + table = "TEST.seq", + columnName = "id", + lowerBound = 5, + upperBound = 5, + numPartitions = 4, + connectionProperties = new Properties() + ) + checkNumPartitions(res, expectedNumPartitions = 1) + assert(res.count() === 8) + } + + test("Partitioning on column where lowerBound is larger than upperBound") { + val e = intercept[IllegalArgumentException] { + spark.read.jdbc( + url = urlWithUserAndPass, + table = "TEST.seq", + columnName = "id", + lowerBound = 5, + upperBound = 1, + numPartitions = 3, + connectionProperties = new Properties() + ) + }.getMessage + assert(e.contains("Operation not allowed: the lower bound of partitioning column " + + "is larger than the upper bound. Lower bound: 5; Upper bound: 1")) + } + + test("SELECT * on partitioned table with a nullable partition column") { + val df = sql("SELECT * FROM nullparts") + checkNumPartitions(df, expectedNumPartitions = 3) + assert(df.collect().length == 4) } test("H2 integral types") { @@ -330,9 +597,9 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext } test("test DATE types") { - val rows = sqlContext.read.jdbc( - urlWithUserAndPass, "TEST.TIMETYPES", new Properties).collect() - val cachedRows = sqlContext.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties) + val rows = spark.read.jdbc( + urlWithUserAndPass, "TEST.TIMETYPES", new Properties()).collect() + val cachedRows = spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties()) .cache().collect() assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) assert(rows(1).getAs[java.sql.Date](1) === null) @@ -340,17 +607,17 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext } test("test DATE types in cache") { - val rows = sqlContext.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties).collect() - sqlContext.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties) - .cache().registerTempTable("mycached_date") + val rows = spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties()).collect() + spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties()) + .cache().createOrReplaceTempView("mycached_date") val cachedRows = sql("select * from mycached_date").collect() assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) assert(cachedRows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) } test("test types for null value") { - val rows = sqlContext.read.jdbc( - urlWithUserAndPass, "TEST.NULLTYPES", new Properties).collect() + val rows = spark.read.jdbc( + urlWithUserAndPass, "TEST.NULLTYPES", new Properties()).collect() assert((0 to 14).forall(i => rows(0).isNullAt(i))) } @@ -369,11 +636,11 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext test("SQL query as table name") { sql( s""" - |CREATE TEMPORARY TABLE hack + |CREATE OR REPLACE TEMPORARY VIEW hack |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url', dbtable '(SELECT B, B*B FROM TEST.FLTTYPES)', | user 'testUser', password 'testPass') - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin.replaceAll("\n", " ")) val rows = sql("SELECT * FROM hack").collect() assert(rows(0).getDouble(0) === 1.00000011920928955) // Yes, I meant ==. // For some reason, H2 computes this square incorrectly... @@ -386,17 +653,17 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext intercept[JdbcSQLException] { sql( s""" - |CREATE TEMPORARY TABLE abc + |CREATE OR REPLACE TEMPORARY VIEW abc |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url', dbtable '(SELECT _ROWID_ FROM test.people)', | user 'testUser', password 'testPass') - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin.replaceAll("\n", " ")) } } test("Remap types via JdbcDialects") { JdbcDialects.registerDialect(testH2Dialect) - val df = sqlContext.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties) + val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties()) assert(df.schema.filter(_.dataType != org.apache.spark.sql.types.StringType).isEmpty) val rows = df.collect() assert(rows(0).get(0).isInstanceOf[String]) @@ -427,6 +694,36 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext assert(DerbyColumns === Seq(""""abc"""", """"key"""")) } + test("compile filters") { + val compileFilter = PrivateMethod[Option[String]]('compileFilter) + def doCompileFilter(f: Filter): String = + JDBCRDD invokePrivate compileFilter(f, JdbcDialects.get("jdbc:")) getOrElse("") + assert(doCompileFilter(EqualTo("col0", 3)) === """"col0" = 3""") + assert(doCompileFilter(Not(EqualTo("col1", "abc"))) === """(NOT ("col1" = 'abc'))""") + assert(doCompileFilter(And(EqualTo("col0", 0), EqualTo("col1", "def"))) + === """("col0" = 0) AND ("col1" = 'def')""") + assert(doCompileFilter(Or(EqualTo("col0", 2), EqualTo("col1", "ghi"))) + === """("col0" = 2) OR ("col1" = 'ghi')""") + assert(doCompileFilter(LessThan("col0", 5)) === """"col0" < 5""") + assert(doCompileFilter(LessThan("col3", + Timestamp.valueOf("1995-11-21 00:00:00.0"))) === """"col3" < '1995-11-21 00:00:00.0'""") + assert(doCompileFilter(LessThan("col4", Date.valueOf("1983-08-04"))) + === """"col4" < '1983-08-04'""") + assert(doCompileFilter(LessThanOrEqual("col0", 5)) === """"col0" <= 5""") + assert(doCompileFilter(GreaterThan("col0", 3)) === """"col0" > 3""") + assert(doCompileFilter(GreaterThanOrEqual("col0", 3)) === """"col0" >= 3""") + assert(doCompileFilter(In("col1", Array("jkl"))) === """"col1" IN ('jkl')""") + assert(doCompileFilter(In("col1", Array.empty)) === + """CASE WHEN "col1" IS NULL THEN NULL ELSE FALSE END""") + assert(doCompileFilter(Not(In("col1", Array("mno", "pqr")))) + === """(NOT ("col1" IN ('mno', 'pqr')))""") + assert(doCompileFilter(IsNull("col1")) === """"col1" IS NULL""") + assert(doCompileFilter(IsNotNull("col1")) === """"col1" IS NOT NULL""") + assert(doCompileFilter(And(EqualNullSafe("col0", "abc"), EqualTo("col1", "def"))) + === """((NOT ("col0" != 'abc' OR "col0" IS NULL OR 'abc' IS NULL) """ + + """OR ("col0" IS NULL AND 'abc' IS NULL))) AND ("col1" = 'def')""") + } + test("Dialect unregister") { JdbcDialects.registerDialect(testH2Dialect) JdbcDialects.unregisterDialect(testH2Dialect) @@ -443,23 +740,65 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext } else { None } + override def isCascadingTruncateTable(): Option[Boolean] = Some(true) }, testH2Dialect)) assert(agg.canHandle("jdbc:h2:xxx")) assert(!agg.canHandle("jdbc:h2")) assert(agg.getCatalystType(0, "", 1, null) === Some(LongType)) assert(agg.getCatalystType(1, "", 1, null) === Some(StringType)) + assert(agg.isCascadingTruncateTable() === Some(true)) + } + + test("Aggregated dialects: isCascadingTruncateTable") { + def genDialect(cascadingTruncateTable: Option[Boolean]): JdbcDialect = new JdbcDialect { + override def canHandle(url: String): Boolean = true + override def getCatalystType( + sqlType: Int, + typeName: String, + size: Int, + md: MetadataBuilder): Option[DataType] = None + override def isCascadingTruncateTable(): Option[Boolean] = cascadingTruncateTable + } + + def testDialects(cascadings: List[Option[Boolean]], expected: Option[Boolean]): Unit = { + val dialects = cascadings.map(genDialect(_)) + val agg = new AggregatedDialect(dialects) + assert(agg.isCascadingTruncateTable() === expected) + } + + testDialects(List(Some(true), Some(false), None), Some(true)) + testDialects(List(Some(true), Some(true), None), Some(true)) + testDialects(List(Some(false), Some(false), None), None) + testDialects(List(Some(true), Some(true)), Some(true)) + testDialects(List(Some(false), Some(false)), Some(false)) + testDialects(List(None, None), None) } test("DB2Dialect type mapping") { val db2Dialect = JdbcDialects.get("jdbc:db2://127.0.0.1/db") assert(db2Dialect.getJDBCType(StringType).map(_.databaseTypeDefinition).get == "CLOB") assert(db2Dialect.getJDBCType(BooleanType).map(_.databaseTypeDefinition).get == "CHAR(1)") + assert(db2Dialect.getJDBCType(ShortType).map(_.databaseTypeDefinition).get == "SMALLINT") + assert(db2Dialect.getJDBCType(ByteType).map(_.databaseTypeDefinition).get == "SMALLINT") + // test db2 dialect mappings on read + assert(db2Dialect.getCatalystType(java.sql.Types.REAL, "REAL", 1, null) == Option(FloatType)) + assert(db2Dialect.getCatalystType(java.sql.Types.OTHER, "DECFLOAT", 1, null) == + Option(DecimalType(38, 18))) + assert(db2Dialect.getCatalystType(java.sql.Types.OTHER, "XML", 1, null) == Option(StringType)) + assert(db2Dialect.getCatalystType(java.sql.Types.OTHER, "TIMESTAMP WITH TIME ZONE", 1, null) == + Option(TimestampType)) } test("PostgresDialect type mapping") { val Postgres = JdbcDialects.get("jdbc:postgresql://127.0.0.1/db") assert(Postgres.getCatalystType(java.sql.Types.OTHER, "json", 1, null) === Some(StringType)) assert(Postgres.getCatalystType(java.sql.Types.OTHER, "jsonb", 1, null) === Some(StringType)) + assert(Postgres.getJDBCType(FloatType).map(_.databaseTypeDefinition).get == "FLOAT4") + assert(Postgres.getJDBCType(DoubleType).map(_.databaseTypeDefinition).get == "FLOAT8") + val errMsg = intercept[IllegalArgumentException] { + Postgres.getJDBCType(ByteType) + } + assert(errMsg.getMessage contains "Unsupported type in postgresql: ByteType") } test("DerbyDialect jdbc type mapping") { @@ -469,6 +808,15 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext assert(derbyDialect.getJDBCType(BooleanType).map(_.databaseTypeDefinition).get == "BOOLEAN") } + test("OracleDialect jdbc type mapping") { + val oracleDialect = JdbcDialects.get("jdbc:oracle") + val metadata = new MetadataBuilder().putString("name", "test_column").putLong("scale", -127) + assert(oracleDialect.getCatalystType(java.sql.Types.NUMERIC, "float", 1, metadata) == + Some(DecimalType(DecimalType.MAX_PRECISION, 10))) + assert(oracleDialect.getCatalystType(java.sql.Types.NUMERIC, "numeric", 0, null) == + Some(DecimalType(DecimalType.MAX_PRECISION, 10))) + } + test("table exists query by jdbc dialect") { val MySQL = JdbcDialects.get("jdbc:mysql://127.0.0.1/db") val Postgres = JdbcDialects.get("jdbc:postgresql://127.0.0.1/db") @@ -484,4 +832,303 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext assert(h2.getTableExistsQuery(table) == defaultQuery) assert(derby.getTableExistsQuery(table) == defaultQuery) } + + test("Test DataFrame.where for Date and Timestamp") { + // Regression test for bug SPARK-11788 + val timestamp = java.sql.Timestamp.valueOf("2001-02-20 11:22:33.543543"); + val date = java.sql.Date.valueOf("1995-01-01") + val jdbcDf = spark.read.jdbc(urlWithUserAndPass, "TEST.TIMETYPES", new Properties()) + val rows = jdbcDf.where($"B" > date && $"C" > timestamp).collect() + assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf("1996-01-01")) + assert(rows(0).getAs[java.sql.Timestamp](2) + === java.sql.Timestamp.valueOf("2002-02-20 11:22:33.543543")) + } + + test("test credentials in the properties are not in plan output") { + val df = sql("SELECT * FROM parts") + val explain = ExplainCommand(df.queryExecution.logical, extended = true) + spark.sessionState.executePlan(explain).executedPlan.executeCollect().foreach { + r => assert(!List("testPass", "testUser").exists(r.toString.contains)) + } + // test the JdbcRelation toString output + df.queryExecution.analyzed.collect { + case r: LogicalRelation => + assert(r.relation.toString == "JDBCRelation(TEST.PEOPLE) [numPartitions=3]") + } + } + + test("test credentials in the connection url are not in the plan output") { + val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties()) + val explain = ExplainCommand(df.queryExecution.logical, extended = true) + spark.sessionState.executePlan(explain).executedPlan.executeCollect().foreach { + r => assert(!List("testPass", "testUser").exists(r.toString.contains)) + } + } + + test("hide credentials in create and describe a persistent/temp table") { + val password = "testPass" + val tableName = "tab1" + Seq("TABLE", "TEMPORARY VIEW").foreach { tableType => + withTable(tableName) { + val df = sql( + s""" + |CREATE $tableType $tableName + |USING org.apache.spark.sql.jdbc + |OPTIONS ( + | url '$urlWithUserAndPass', + | dbtable 'TEST.PEOPLE', + | user 'testUser', + | password '$password') + """.stripMargin) + + val explain = ExplainCommand(df.queryExecution.logical, extended = true) + spark.sessionState.executePlan(explain).executedPlan.executeCollect().foreach { r => + assert(!r.toString.contains(password)) + } + + sql(s"DESC FORMATTED $tableName").collect().foreach { r => + assert(!r.toString().contains(password)) + } + } + } + } + + test("SPARK 12941: The data type mapping for StringType to Oracle") { + val oracleDialect = JdbcDialects.get("jdbc:oracle://127.0.0.1/db") + assert(oracleDialect.getJDBCType(StringType). + map(_.databaseTypeDefinition).get == "VARCHAR2(255)") + } + + test("SPARK-16625: General data types to be mapped to Oracle") { + + def getJdbcType(dialect: JdbcDialect, dt: DataType): String = { + dialect.getJDBCType(dt).orElse(JdbcUtils.getCommonJDBCType(dt)). + map(_.databaseTypeDefinition).get + } + + val oracleDialect = JdbcDialects.get("jdbc:oracle://127.0.0.1/db") + assert(getJdbcType(oracleDialect, BooleanType) == "NUMBER(1)") + assert(getJdbcType(oracleDialect, IntegerType) == "NUMBER(10)") + assert(getJdbcType(oracleDialect, LongType) == "NUMBER(19)") + assert(getJdbcType(oracleDialect, FloatType) == "NUMBER(19, 4)") + assert(getJdbcType(oracleDialect, DoubleType) == "NUMBER(19, 4)") + assert(getJdbcType(oracleDialect, ByteType) == "NUMBER(3)") + assert(getJdbcType(oracleDialect, ShortType) == "NUMBER(5)") + assert(getJdbcType(oracleDialect, StringType) == "VARCHAR2(255)") + assert(getJdbcType(oracleDialect, BinaryType) == "BLOB") + assert(getJdbcType(oracleDialect, DateType) == "DATE") + assert(getJdbcType(oracleDialect, TimestampType) == "TIMESTAMP") + } + + private def assertEmptyQuery(sqlString: String): Unit = { + assert(sql(sqlString).collect().isEmpty) + } + + test("SPARK-15916: JDBC filter operator push down should respect operator precedence") { + val TRUE = "NAME != 'non_exists'" + val FALSE1 = "THEID > 1000000000" + val FALSE2 = "THEID < -1000000000" + + assertEmptyQuery(s"SELECT * FROM foobar WHERE ($TRUE OR $FALSE1) AND $FALSE2") + assertEmptyQuery(s"SELECT * FROM foobar WHERE $FALSE1 AND ($FALSE2 OR $TRUE)") + + // Tests JDBCPartition whereClause clause push down. + withTempView("tempFrame") { + val jdbcPartitionWhereClause = s"$FALSE1 OR $TRUE" + val df = spark.read.jdbc( + urlWithUserAndPass, + "TEST.PEOPLE", + predicates = Array[String](jdbcPartitionWhereClause), + new Properties()) + + df.createOrReplaceTempView("tempFrame") + assertEmptyQuery(s"SELECT * FROM tempFrame where $FALSE2") + } + } + + test("SPARK-16387: Reserved SQL words are not escaped by JDBC writer") { + val df = spark.createDataset(Seq("a", "b", "c")).toDF("order") + val schema = JdbcUtils.schemaString(df, "jdbc:mysql://localhost:3306/temp") + assert(schema.contains("`order` TEXT")) + } + + test("SPARK-18141: Predicates on quoted column names in the jdbc data source") { + assert(sql("SELECT * FROM mixedCaseCols WHERE Id < 1").collect().size == 0) + assert(sql("SELECT * FROM mixedCaseCols WHERE Id <= 1").collect().size == 1) + assert(sql("SELECT * FROM mixedCaseCols WHERE Id > 1").collect().size == 2) + assert(sql("SELECT * FROM mixedCaseCols WHERE Id >= 1").collect().size == 3) + assert(sql("SELECT * FROM mixedCaseCols WHERE Id = 1").collect().size == 1) + assert(sql("SELECT * FROM mixedCaseCols WHERE Id != 2").collect().size == 2) + assert(sql("SELECT * FROM mixedCaseCols WHERE Id <=> 2").collect().size == 1) + assert(sql("SELECT * FROM mixedCaseCols WHERE Name LIKE 'fr%'").collect().size == 1) + assert(sql("SELECT * FROM mixedCaseCols WHERE Name LIKE '%ed'").collect().size == 1) + assert(sql("SELECT * FROM mixedCaseCols WHERE Name LIKE '%re%'").collect().size == 1) + assert(sql("SELECT * FROM mixedCaseCols WHERE Name IS NULL").collect().size == 1) + assert(sql("SELECT * FROM mixedCaseCols WHERE Name IS NOT NULL").collect().size == 2) + assert(sql("SELECT * FROM mixedCaseCols").filter($"Name".isin()).collect().size == 0) + assert(sql("SELECT * FROM mixedCaseCols WHERE Name IN ('mary', 'fred')").collect().size == 2) + assert(sql("SELECT * FROM mixedCaseCols WHERE Name NOT IN ('fred')").collect().size == 1) + assert(sql("SELECT * FROM mixedCaseCols WHERE Id = 1 OR Name = 'mary'").collect().size == 2) + assert(sql("SELECT * FROM mixedCaseCols WHERE Name = 'mary' AND Id = 2").collect().size == 1) + } + + test("SPARK-18419: Fix `asConnectionProperties` to filter case-insensitively") { + val parameters = Map( + "url" -> "jdbc:mysql://localhost:3306/temp", + "dbtable" -> "t1", + "numPartitions" -> "10") + assert(new JDBCOptions(parameters).asConnectionProperties.isEmpty) + assert(new JDBCOptions(CaseInsensitiveMap(parameters)).asConnectionProperties.isEmpty) + } + + test("SPARK-16848: jdbc API throws an exception for user specified schema") { + val schema = StructType(Seq( + StructField("name", StringType, false), StructField("theid", IntegerType, false))) + val parts = Array[String]("THEID < 2", "THEID >= 2") + val e1 = intercept[AnalysisException] { + spark.read.schema(schema).jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, new Properties()) + }.getMessage + assert(e1.contains("User specified schema not supported with `jdbc`")) + + val e2 = intercept[AnalysisException] { + spark.read.schema(schema).jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties()) + }.getMessage + assert(e2.contains("User specified schema not supported with `jdbc`")) + } + + test("jdbc API support custom schema") { + val parts = Array[String]("THEID < 2", "THEID >= 2") + val customSchema = "NAME STRING, THEID INT" + val props = new Properties() + props.put("customSchema", customSchema) + val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts, props) + assert(df.schema.size === 2) + assert(df.schema === CatalystSqlParser.parseTableSchema(customSchema)) + assert(df.count() === 3) + } + + test("jdbc API custom schema DDL-like strings.") { + withTempView("people_view") { + val customSchema = "NAME STRING, THEID INT" + sql( + s""" + |CREATE TEMPORARY VIEW people_view + |USING org.apache.spark.sql.jdbc + |OPTIONS (uRl '$url', DbTaBlE 'TEST.PEOPLE', User 'testUser', PassWord 'testPass', + |customSchema '$customSchema') + """.stripMargin.replaceAll("\n", " ")) + val df = sql("select * from people_view") + assert(df.schema.length === 2) + assert(df.schema === CatalystSqlParser.parseTableSchema(customSchema)) + assert(df.count() === 3) + } + } + + test("SPARK-15648: teradataDialect StringType data mapping") { + val teradataDialect = JdbcDialects.get("jdbc:teradata://127.0.0.1/db") + assert(teradataDialect.getJDBCType(StringType). + map(_.databaseTypeDefinition).get == "VARCHAR(255)") + } + + test("SPARK-15648: teradataDialect BooleanType data mapping") { + val teradataDialect = JdbcDialects.get("jdbc:teradata://127.0.0.1/db") + assert(teradataDialect.getJDBCType(BooleanType). + map(_.databaseTypeDefinition).get == "CHAR(1)") + } + + test("Checking metrics correctness with JDBC") { + val foobarCnt = spark.table("foobar").count() + val res = InputOutputMetricsHelper.run(sql("SELECT * FROM foobar").toDF()) + assert(res === (foobarCnt, 0L, foobarCnt) :: Nil) + } + + test("unsupported types") { + var e = intercept[SparkException] { + spark.read.jdbc(urlWithUserAndPass, "TEST.TIMEZONE", new Properties()).collect() + }.getMessage + assert(e.contains("java.lang.UnsupportedOperationException: unimplemented")) + e = intercept[SQLException] { + spark.read.jdbc(urlWithUserAndPass, "TEST.ARRAY", new Properties()).collect() + }.getMessage + assert(e.contains("Unsupported type ARRAY")) + } + + test("SPARK-19318: Connection properties keys should be case-sensitive.") { + def testJdbcOptions(options: JDBCOptions): Unit = { + // Spark JDBC data source options are case-insensitive + assert(options.table == "t1") + // When we convert it to properties, it should be case-sensitive. + assert(options.asProperties.size == 3) + assert(options.asProperties.get("customkey") == null) + assert(options.asProperties.get("customKey") == "a-value") + assert(options.asConnectionProperties.size == 1) + assert(options.asConnectionProperties.get("customkey") == null) + assert(options.asConnectionProperties.get("customKey") == "a-value") + } + + val parameters = Map("url" -> url, "dbTAblE" -> "t1", "customKey" -> "a-value") + testJdbcOptions(new JDBCOptions(parameters)) + testJdbcOptions(new JDBCOptions(CaseInsensitiveMap(parameters))) + // test add/remove key-value from the case-insensitive map + var modifiedParameters = CaseInsensitiveMap(Map.empty) ++ parameters + testJdbcOptions(new JDBCOptions(modifiedParameters)) + modifiedParameters -= "dbtable" + assert(modifiedParameters.get("dbTAblE").isEmpty) + modifiedParameters -= "customkey" + assert(modifiedParameters.get("customKey").isEmpty) + modifiedParameters += ("customKey" -> "a-value") + modifiedParameters += ("dbTable" -> "t1") + testJdbcOptions(new JDBCOptions(modifiedParameters)) + assert ((modifiedParameters -- parameters.keys).size == 0) + } + + test("SPARK-19318: jdbc data source options should be treated case-insensitive.") { + val df = spark.read.format("jdbc") + .option("Url", urlWithUserAndPass) + .option("DbTaBle", "TEST.PEOPLE") + .load() + assert(df.count() == 3) + + withTempView("people_view") { + sql( + s""" + |CREATE TEMPORARY VIEW people_view + |USING org.apache.spark.sql.jdbc + |OPTIONS (uRl '$url', DbTaBlE 'TEST.PEOPLE', User 'testUser', PassWord 'testPass') + """.stripMargin.replaceAll("\n", " ")) + + assert(sql("select * from people_view").count() == 3) + } + } + + test("SPARK-21519: option sessionInitStatement, run SQL to initialize the database session.") { + val initSQL1 = "SET @MYTESTVAR 21519" + val df1 = spark.read.format("jdbc") + .option("url", urlWithUserAndPass) + .option("dbtable", "(SELECT NVL(@MYTESTVAR, -1))") + .option("sessionInitStatement", initSQL1) + .load() + assert(df1.collect() === Array(Row(21519))) + + val initSQL2 = "SET SCHEMA DUMMY" + val df2 = spark.read.format("jdbc") + .option("url", urlWithUserAndPass) + .option("dbtable", "TEST.PEOPLE") + .option("sessionInitStatement", initSQL2) + .load() + val e = intercept[SparkException] {df2.collect()}.getMessage + assert(e.contains("""Schema "DUMMY" not found""")) + + sql( + s""" + |CREATE OR REPLACE TEMPORARY VIEW test_sessionInitStatement + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$urlWithUserAndPass', + |dbtable '(SELECT NVL(@MYTESTVAR1, -1), NVL(@MYTESTVAR2, -1))', + |sessionInitStatement 'SET @MYTESTVAR1 21519; SET @MYTESTVAR2 1234') + """.stripMargin) + + val df3 = sql("SELECT * FROM test_sessionInitStatement") + assert(df3.collect() === Array(Row(21519, 1234))) + } } 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 e23ee6693133..1985b1dc8287 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 @@ -20,9 +20,15 @@ package org.apache.spark.sql.jdbc import java.sql.DriverManager import java.util.Properties +import scala.collection.JavaConverters.propertiesAsScalaMapConverter + import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.{Row, SaveMode} +import org.apache.spark.SparkException +import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode} +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -38,6 +44,11 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { properties.setProperty("password", "testPass") properties.setProperty("rowId", "false") + val testH2Dialect = new JdbcDialect { + override def canHandle(url: String) : Boolean = url.startsWith("jdbc:h2") + override def isCascadingTruncateTable(): Option[Boolean] = Some(false) + } + before { Utils.classForName("org.h2.Driver") conn = DriverManager.getConnection(url) @@ -57,14 +68,14 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { sql( s""" - |CREATE TEMPORARY TABLE PEOPLE + |CREATE OR REPLACE TEMPORARY VIEW PEOPLE |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url1', dbtable 'TEST.PEOPLE', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) sql( s""" - |CREATE TEMPORARY TABLE PEOPLE1 + |CREATE OR REPLACE TEMPORARY VIEW PEOPLE1 |USING org.apache.spark.sql.jdbc |OPTIONS (url '$url1', dbtable 'TEST.PEOPLE1', user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) @@ -87,68 +98,421 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { StructField("id", IntegerType) :: StructField("seq", IntegerType) :: Nil) + private lazy val schema4 = StructType( + StructField("NAME", StringType) :: + StructField("ID", IntegerType) :: Nil) + test("Basic CREATE") { - val df = sqlContext.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) - df.write.jdbc(url, "TEST.BASICCREATETEST", new Properties) - assert(2 === sqlContext.read.jdbc(url, "TEST.BASICCREATETEST", new Properties).count) + df.write.jdbc(url, "TEST.BASICCREATETEST", new Properties()) + assert(2 === spark.read.jdbc(url, "TEST.BASICCREATETEST", new Properties()).count()) assert( - 2 === sqlContext.read.jdbc(url, "TEST.BASICCREATETEST", new Properties).collect()(0).length) + 2 === spark.read.jdbc(url, "TEST.BASICCREATETEST", new Properties()).collect()(0).length) + } + + test("Basic CREATE with illegal batchsize") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + + (-1 to 0).foreach { size => + val properties = new Properties() + properties.setProperty(JDBCOptions.JDBC_BATCH_INSERT_SIZE, size.toString) + val e = intercept[IllegalArgumentException] { + df.write.mode(SaveMode.Overwrite).jdbc(url, "TEST.BASICCREATETEST", properties) + }.getMessage + assert(e.contains(s"Invalid value `$size` for parameter `batchsize`")) + } + } + + test("Basic CREATE with batchsize") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + + (1 to 3).foreach { size => + val properties = new Properties() + properties.setProperty(JDBCOptions.JDBC_BATCH_INSERT_SIZE, size.toString) + df.write.mode(SaveMode.Overwrite).jdbc(url, "TEST.BASICCREATETEST", properties) + assert(2 === spark.read.jdbc(url, "TEST.BASICCREATETEST", new Properties()).count()) + } + } + + test("CREATE with ignore") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x3), schema3) + val df2 = spark.createDataFrame(sparkContext.parallelize(arr1x2), schema2) + + df.write.mode(SaveMode.Ignore).jdbc(url1, "TEST.DROPTEST", properties) + assert(2 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).count()) + assert(3 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).collect()(0).length) + + df2.write.mode(SaveMode.Ignore).jdbc(url1, "TEST.DROPTEST", properties) + assert(2 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).count()) + assert(3 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).collect()(0).length) } test("CREATE with overwrite") { - val df = sqlContext.createDataFrame(sparkContext.parallelize(arr2x3), schema3) - val df2 = sqlContext.createDataFrame(sparkContext.parallelize(arr1x2), schema2) + val df = spark.createDataFrame(sparkContext.parallelize(arr2x3), schema3) + val df2 = spark.createDataFrame(sparkContext.parallelize(arr1x2), schema2) df.write.jdbc(url1, "TEST.DROPTEST", properties) - assert(2 === sqlContext.read.jdbc(url1, "TEST.DROPTEST", properties).count) - assert(3 === sqlContext.read.jdbc(url1, "TEST.DROPTEST", properties).collect()(0).length) + assert(2 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).count()) + assert(3 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).collect()(0).length) df2.write.mode(SaveMode.Overwrite).jdbc(url1, "TEST.DROPTEST", properties) - assert(1 === sqlContext.read.jdbc(url1, "TEST.DROPTEST", properties).count) - assert(2 === sqlContext.read.jdbc(url1, "TEST.DROPTEST", properties).collect()(0).length) + assert(1 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).count()) + assert(2 === spark.read.jdbc(url1, "TEST.DROPTEST", properties).collect()(0).length) } test("CREATE then INSERT to append") { - val df = sqlContext.createDataFrame(sparkContext.parallelize(arr2x2), schema2) - val df2 = sqlContext.createDataFrame(sparkContext.parallelize(arr1x2), schema2) + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + val df2 = spark.createDataFrame(sparkContext.parallelize(arr1x2), schema2) - df.write.jdbc(url, "TEST.APPENDTEST", new Properties) - df2.write.mode(SaveMode.Append).jdbc(url, "TEST.APPENDTEST", new Properties) - assert(3 === sqlContext.read.jdbc(url, "TEST.APPENDTEST", new Properties).count) - assert(2 === sqlContext.read.jdbc(url, "TEST.APPENDTEST", new Properties).collect()(0).length) + df.write.jdbc(url, "TEST.APPENDTEST", new Properties()) + df2.write.mode(SaveMode.Append).jdbc(url, "TEST.APPENDTEST", new Properties()) + assert(3 === spark.read.jdbc(url, "TEST.APPENDTEST", new Properties()).count()) + assert(2 === spark.read.jdbc(url, "TEST.APPENDTEST", new Properties()).collect()(0).length) } - test("CREATE then INSERT to truncate") { - val df = sqlContext.createDataFrame(sparkContext.parallelize(arr2x2), schema2) - val df2 = sqlContext.createDataFrame(sparkContext.parallelize(arr1x2), schema2) + test("SPARK-18123 Append with column names with different cases") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + val df2 = spark.createDataFrame(sparkContext.parallelize(arr1x2), schema4) + + df.write.jdbc(url, "TEST.APPENDTEST", new Properties()) + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val m = intercept[AnalysisException] { + df2.write.mode(SaveMode.Append).jdbc(url, "TEST.APPENDTEST", new Properties()) + }.getMessage + assert(m.contains("Column \"NAME\" not found")) + } + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + df2.write.mode(SaveMode.Append).jdbc(url, "TEST.APPENDTEST", new Properties()) + assert(3 === spark.read.jdbc(url, "TEST.APPENDTEST", new Properties()).count()) + assert(2 === spark.read.jdbc(url, "TEST.APPENDTEST", new Properties()).collect()(0).length) + } + } + + test("Truncate") { + JdbcDialects.registerDialect(testH2Dialect) + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + val df2 = spark.createDataFrame(sparkContext.parallelize(arr1x2), schema2) + val df3 = spark.createDataFrame(sparkContext.parallelize(arr2x3), schema3) df.write.jdbc(url1, "TEST.TRUNCATETEST", properties) - df2.write.mode(SaveMode.Overwrite).jdbc(url1, "TEST.TRUNCATETEST", properties) - assert(1 === sqlContext.read.jdbc(url1, "TEST.TRUNCATETEST", properties).count) - assert(2 === sqlContext.read.jdbc(url1, "TEST.TRUNCATETEST", properties).collect()(0).length) + df2.write.mode(SaveMode.Overwrite).option("truncate", true) + .jdbc(url1, "TEST.TRUNCATETEST", properties) + assert(1 === spark.read.jdbc(url1, "TEST.TRUNCATETEST", properties).count()) + assert(2 === spark.read.jdbc(url1, "TEST.TRUNCATETEST", properties).collect()(0).length) + + val m = intercept[AnalysisException] { + df3.write.mode(SaveMode.Overwrite).option("truncate", true) + .jdbc(url1, "TEST.TRUNCATETEST", properties) + }.getMessage + assert(m.contains("Column \"seq\" not found")) + assert(0 === spark.read.jdbc(url1, "TEST.TRUNCATETEST", properties).count()) + JdbcDialects.unregisterDialect(testH2Dialect) + } + + test("createTableOptions") { + JdbcDialects.registerDialect(testH2Dialect) + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + + val m = intercept[org.h2.jdbc.JdbcSQLException] { + df.write.option("createTableOptions", "ENGINE tableEngineName") + .jdbc(url1, "TEST.CREATETBLOPTS", properties) + }.getMessage + assert(m.contains("Class \"TABLEENGINENAME\" not found")) + JdbcDialects.unregisterDialect(testH2Dialect) } test("Incompatible INSERT to append") { - val df = sqlContext.createDataFrame(sparkContext.parallelize(arr2x2), schema2) - val df2 = sqlContext.createDataFrame(sparkContext.parallelize(arr2x3), schema3) + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + val df2 = spark.createDataFrame(sparkContext.parallelize(arr2x3), schema3) - df.write.jdbc(url, "TEST.INCOMPATIBLETEST", new Properties) - intercept[org.apache.spark.SparkException] { - df2.write.mode(SaveMode.Append).jdbc(url, "TEST.INCOMPATIBLETEST", new Properties) - } + df.write.jdbc(url, "TEST.INCOMPATIBLETEST", new Properties()) + val m = intercept[AnalysisException] { + df2.write.mode(SaveMode.Append).jdbc(url, "TEST.INCOMPATIBLETEST", new Properties()) + }.getMessage + assert(m.contains("Column \"seq\" not found")) } test("INSERT to JDBC Datasource") { sql("INSERT INTO TABLE PEOPLE1 SELECT * FROM PEOPLE") - assert(2 === sqlContext.read.jdbc(url1, "TEST.PEOPLE1", properties).count) - assert(2 === sqlContext.read.jdbc(url1, "TEST.PEOPLE1", properties).collect()(0).length) + assert(2 === spark.read.jdbc(url1, "TEST.PEOPLE1", properties).count()) + assert(2 === spark.read.jdbc(url1, "TEST.PEOPLE1", properties).collect()(0).length) } test("INSERT to JDBC Datasource with overwrite") { sql("INSERT INTO TABLE PEOPLE1 SELECT * FROM PEOPLE") sql("INSERT OVERWRITE TABLE PEOPLE1 SELECT * FROM PEOPLE") - assert(2 === sqlContext.read.jdbc(url1, "TEST.PEOPLE1", properties).count) - assert(2 === sqlContext.read.jdbc(url1, "TEST.PEOPLE1", properties).collect()(0).length) + assert(2 === spark.read.jdbc(url1, "TEST.PEOPLE1", properties).count()) + assert(2 === spark.read.jdbc(url1, "TEST.PEOPLE1", properties).collect()(0).length) + } + + test("save works for format(\"jdbc\") if url and dbtable are set") { + val df = sqlContext.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + + df.write.format("jdbc") + .options(Map("url" -> url, "dbtable" -> "TEST.SAVETEST")) + .save() + + assert(2 === sqlContext.read.jdbc(url, "TEST.SAVETEST", new Properties).count) + assert( + 2 === sqlContext.read.jdbc(url, "TEST.SAVETEST", new Properties).collect()(0).length) + } + + test("save API with SaveMode.Overwrite") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + val df2 = spark.createDataFrame(sparkContext.parallelize(arr1x2), schema2) + + df.write.format("jdbc") + .option("url", url1) + .option("dbtable", "TEST.SAVETEST") + .options(properties.asScala) + .save() + df2.write.mode(SaveMode.Overwrite).format("jdbc") + .option("url", url1) + .option("dbtable", "TEST.SAVETEST") + .options(properties.asScala) + .save() + assert(1 === spark.read.jdbc(url1, "TEST.SAVETEST", properties).count()) + assert(2 === spark.read.jdbc(url1, "TEST.SAVETEST", properties).collect()(0).length) + } + + test("save errors if url is not specified") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + + val e = intercept[RuntimeException] { + df.write.format("jdbc") + .option("dbtable", "TEST.SAVETEST") + .options(properties.asScala) + .save() + }.getMessage + assert(e.contains("Option 'url' is required")) + } + + test("save errors if dbtable is not specified") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + + val e = intercept[RuntimeException] { + df.write.format("jdbc") + .option("url", url1) + .options(properties.asScala) + .save() + }.getMessage + assert(e.contains("Option 'dbtable' is required")) + } + + test("save errors if wrong user/password combination") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + + val e = intercept[org.h2.jdbc.JdbcSQLException] { + df.write.format("jdbc") + .option("dbtable", "TEST.SAVETEST") + .option("url", url1) + .save() + }.getMessage + assert(e.contains("Wrong user name or password")) + } + + test("save errors if partitionColumn and numPartitions and bounds not set") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + + val e = intercept[java.lang.IllegalArgumentException] { + df.write.format("jdbc") + .option("dbtable", "TEST.SAVETEST") + .option("url", url1) + .option("partitionColumn", "foo") + .save() + }.getMessage + assert(e.contains("When reading JDBC data sources, users need to specify all or none " + + "for the following options: 'partitionColumn', 'lowerBound', 'upperBound', and " + + "'numPartitions'")) + } + + test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + df.write.format("jdbc") + .option("Url", url1) + .option("dbtable", "TEST.SAVETEST") + .options(properties.asScala) + .save() + } + + test("SPARK-18413: Use `numPartitions` JDBCOption") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + val e = intercept[IllegalArgumentException] { + df.write.format("jdbc") + .option("dbtable", "TEST.SAVETEST") + .option("url", url1) + .option("user", "testUser") + .option("password", "testPass") + .option(s"${JDBCOptions.JDBC_NUM_PARTITIONS}", "0") + .save() + }.getMessage + assert(e.contains("Invalid value `0` for parameter `numPartitions` in table writing " + + "via JDBC. The minimum value is 1.")) + } + + test("SPARK-19318 temporary view data source option keys should be case-insensitive") { + withTempView("people_view") { + sql( + s""" + |CREATE TEMPORARY VIEW people_view + |USING org.apache.spark.sql.jdbc + |OPTIONS (uRl '$url1', DbTaBlE 'TEST.PEOPLE1', User 'testUser', PassWord 'testPass') + """.stripMargin.replaceAll("\n", " ")) + sql("INSERT OVERWRITE TABLE PEOPLE_VIEW SELECT * FROM PEOPLE") + assert(sql("select * from people_view").count() == 2) + } + } + + test("SPARK-10849: test schemaString - from createTableColumnTypes option values") { + def testCreateTableColDataTypes(types: Seq[String]): Unit = { + val colTypes = types.zipWithIndex.map { case (t, i) => (s"col$i", t) } + val schema = colTypes + .foldLeft(new StructType())((schema, colType) => schema.add(colType._1, colType._2)) + val createTableColTypes = + colTypes.map { case (col, dataType) => s"$col $dataType" }.mkString(", ") + val df = spark.createDataFrame(sparkContext.parallelize(Seq(Row.empty)), schema) + + val expectedSchemaStr = + colTypes.map { case (col, dataType) => s""""$col" $dataType """ }.mkString(", ") + + assert(JdbcUtils.schemaString(df, url1, Option(createTableColTypes)) == expectedSchemaStr) + } + + testCreateTableColDataTypes(Seq("boolean")) + testCreateTableColDataTypes(Seq("tinyint", "smallint", "int", "bigint")) + testCreateTableColDataTypes(Seq("float", "double")) + testCreateTableColDataTypes(Seq("string", "char(10)", "varchar(20)")) + testCreateTableColDataTypes(Seq("decimal(10,0)", "decimal(10,5)")) + testCreateTableColDataTypes(Seq("date", "timestamp")) + testCreateTableColDataTypes(Seq("binary")) + } + + test("SPARK-10849: create table using user specified column type and verify on target table") { + def testUserSpecifiedColTypes( + df: DataFrame, + createTableColTypes: String, + expectedTypes: Map[String, String]): Unit = { + df.write + .mode(SaveMode.Overwrite) + .option("createTableColumnTypes", createTableColTypes) + .jdbc(url1, "TEST.DBCOLTYPETEST", properties) + + // verify the data types of the created table by reading the database catalog of H2 + val query = + """ + |(SELECT column_name, type_name, character_maximum_length + | FROM information_schema.columns WHERE table_name = 'DBCOLTYPETEST') + """.stripMargin + val rows = spark.read.jdbc(url1, query, properties).collect() + + rows.foreach { row => + val typeName = row.getString(1) + // For CHAR and VARCHAR, we also compare the max length + if (typeName.contains("CHAR")) { + val charMaxLength = row.getInt(2) + assert(expectedTypes(row.getString(0)) == s"$typeName($charMaxLength)") + } else { + assert(expectedTypes(row.getString(0)) == typeName) + } + } + } + + val data = Seq[Row](Row(1, "dave", "Boston")) + val schema = StructType( + StructField("id", IntegerType) :: + StructField("first#name", StringType) :: + StructField("city", StringType) :: Nil) + val df = spark.createDataFrame(sparkContext.parallelize(data), schema) + + // out-of-order + val expected1 = Map("id" -> "BIGINT", "first#name" -> "VARCHAR(123)", "city" -> "CHAR(20)") + testUserSpecifiedColTypes(df, "`first#name` VARCHAR(123), id BIGINT, city CHAR(20)", expected1) + // partial schema + val expected2 = Map("id" -> "INTEGER", "first#name" -> "VARCHAR(123)", "city" -> "CHAR(20)") + testUserSpecifiedColTypes(df, "`first#name` VARCHAR(123), city CHAR(20)", expected2) + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + // should still respect the original column names + val expected = Map("id" -> "INTEGER", "first#name" -> "VARCHAR(123)", "city" -> "CLOB") + testUserSpecifiedColTypes(df, "`FiRsT#NaMe` VARCHAR(123)", expected) + } + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val schema = StructType( + StructField("id", IntegerType) :: + StructField("First#Name", StringType) :: + StructField("city", StringType) :: Nil) + val df = spark.createDataFrame(sparkContext.parallelize(data), schema) + val expected = Map("id" -> "INTEGER", "First#Name" -> "VARCHAR(123)", "city" -> "CLOB") + testUserSpecifiedColTypes(df, "`First#Name` VARCHAR(123)", expected) + } + } + + test("SPARK-10849: jdbc CreateTableColumnTypes option with invalid data type") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + val msg = intercept[ParseException] { + df.write.mode(SaveMode.Overwrite) + .option("createTableColumnTypes", "name CLOB(2000)") + .jdbc(url1, "TEST.USERDBTYPETEST", properties) + }.getMessage() + assert(msg.contains("DataType clob(2000) is not supported.")) + } + + test("SPARK-10849: jdbc CreateTableColumnTypes option with invalid syntax") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + val msg = intercept[ParseException] { + df.write.mode(SaveMode.Overwrite) + .option("createTableColumnTypes", "`name char(20)") // incorrectly quoted column + .jdbc(url1, "TEST.USERDBTYPETEST", properties) + }.getMessage() + assert(msg.contains("extraneous input")) + } + + test("SPARK-10849: jdbc CreateTableColumnTypes duplicate columns") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + val msg = intercept[AnalysisException] { + df.write.mode(SaveMode.Overwrite) + .option("createTableColumnTypes", "name CHAR(20), id int, NaMe VARCHAR(100)") + .jdbc(url1, "TEST.USERDBTYPETEST", properties) + }.getMessage() + assert(msg.contains( + "Found duplicate column(s) in the createTableColumnTypes option value: `name`")) + } + } + + test("SPARK-10849: jdbc CreateTableColumnTypes invalid columns") { + // schema2 has the column "id" and "name" + val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2) + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val msg = intercept[AnalysisException] { + df.write.mode(SaveMode.Overwrite) + .option("createTableColumnTypes", "firstName CHAR(20), id int") + .jdbc(url1, "TEST.USERDBTYPETEST", properties) + }.getMessage() + assert(msg.contains("createTableColumnTypes option column firstName not found in " + + "schema struct")) + } + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val msg = intercept[AnalysisException] { + df.write.mode(SaveMode.Overwrite) + .option("createTableColumnTypes", "id int, Name VARCHAR(100)") + .jdbc(url1, "TEST.USERDBTYPETEST", properties) + }.getMessage() + assert(msg.contains("createTableColumnTypes option column Name not found in " + + "schema struct")) + } + } + + test("SPARK-19726: INSERT null to a NOT NULL column") { + val e = intercept[SparkException] { + sql("INSERT INTO PEOPLE1 values (null, null)") + }.getMessage + assert(e.contains("NULL not allowed for column \"NAME\"")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala new file mode 100644 index 000000000000..eb9e6458fc61 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -0,0 +1,632 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.net.URI + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +import org.apache.spark.sql.execution.{DataSourceScanExec, SortExec} +import org.apache.spark.sql.execution.datasources.DataSourceStrategy +import org.apache.spark.sql.execution.exchange.ShuffleExchange +import org.apache.spark.sql.execution.joins.SortMergeJoinExec +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.util.Utils +import org.apache.spark.util.collection.BitSet + +class BucketedReadWithoutHiveSupportSuite extends BucketedReadSuite with SharedSQLContext { + protected override def beforeAll(): Unit = { + super.beforeAll() + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") + } +} + + +abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { + import testImplicits._ + + private lazy val df = (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k") + private lazy val nullDF = (for { + i <- 0 to 50 + s <- Seq(null, "a", "b", "c", "d", "e", "f", null, "g") + } yield (i % 5, s, i % 13)).toDF("i", "j", "k") + + test("read bucketed data") { + withTable("bucketed_table") { + df.write + .format("parquet") + .partitionBy("i") + .bucketBy(8, "j", "k") + .saveAsTable("bucketed_table") + + for (i <- 0 until 5) { + val table = spark.table("bucketed_table").filter($"i" === i) + val query = table.queryExecution + val output = query.analyzed.output + val rdd = query.toRdd + + assert(rdd.partitions.length == 8) + + val attrs = table.select("j", "k").queryExecution.analyzed.output + val checkBucketId = rdd.mapPartitionsWithIndex((index, rows) => { + val getBucketId = UnsafeProjection.create( + HashPartitioning(attrs, 8).partitionIdExpression :: Nil, + output) + rows.map(row => getBucketId(row).getInt(0) -> index) + }) + checkBucketId.collect().foreach(r => assert(r._1 == r._2)) + } + } + } + + // To verify if the bucket pruning works, this function checks two conditions: + // 1) Check if the pruned buckets (before filtering) are empty. + // 2) Verify the final result is the same as the expected one + private def checkPrunedAnswers( + bucketSpec: BucketSpec, + bucketValues: Seq[Integer], + filterCondition: Column, + originalDataFrame: DataFrame): Unit = { + // This test verifies parts of the plan. Disable whole stage codegen. + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val strategy = DataSourceStrategy(spark.sessionState.conf) + val bucketedDataFrame = spark.table("bucketed_table").select("i", "j", "k") + val BucketSpec(numBuckets, bucketColumnNames, _) = bucketSpec + // Limit: bucket pruning only works when the bucket column has one and only one column + assert(bucketColumnNames.length == 1) + val bucketColumnIndex = bucketedDataFrame.schema.fieldIndex(bucketColumnNames.head) + val bucketColumn = bucketedDataFrame.schema.toAttributes(bucketColumnIndex) + val matchedBuckets = new BitSet(numBuckets) + bucketValues.foreach { value => + matchedBuckets.set(strategy.getBucketId(bucketColumn, numBuckets, value)) + } + + // Filter could hide the bug in bucket pruning. Thus, skipping all the filters + val plan = bucketedDataFrame.filter(filterCondition).queryExecution.executedPlan + val rdd = plan.find(_.isInstanceOf[DataSourceScanExec]) + assert(rdd.isDefined, plan) + + val checkedResult = rdd.get.execute().mapPartitionsWithIndex { case (index, iter) => + if (matchedBuckets.get(index % numBuckets) && iter.nonEmpty) Iterator(index) else Iterator() + } + // TODO: These tests are not testing the right columns. +// // checking if all the pruned buckets are empty +// val invalidBuckets = checkedResult.collect().toList +// if (invalidBuckets.nonEmpty) { +// fail(s"Buckets $invalidBuckets should have been pruned from:\n$plan") +// } + + checkAnswer( + bucketedDataFrame.filter(filterCondition).orderBy("i", "j", "k"), + originalDataFrame.filter(filterCondition).orderBy("i", "j", "k")) + } + } + + test("read partitioning bucketed tables with bucket pruning filters") { + withTable("bucketed_table") { + val numBuckets = 8 + val bucketSpec = BucketSpec(numBuckets, Seq("j"), Nil) + // json does not support predicate push-down, and thus json is used here + df.write + .format("json") + .partitionBy("i") + .bucketBy(numBuckets, "j") + .saveAsTable("bucketed_table") + + for (j <- 0 until 13) { + // Case 1: EqualTo + checkPrunedAnswers( + bucketSpec, + bucketValues = j :: Nil, + filterCondition = $"j" === j, + df) + + // Case 2: EqualNullSafe + checkPrunedAnswers( + bucketSpec, + bucketValues = j :: Nil, + filterCondition = $"j" <=> j, + df) + + // Case 3: In + checkPrunedAnswers( + bucketSpec, + bucketValues = Seq(j, j + 1, j + 2, j + 3), + filterCondition = $"j".isin(j, j + 1, j + 2, j + 3), + df) + } + } + } + + test("read non-partitioning bucketed tables with bucket pruning filters") { + withTable("bucketed_table") { + val numBuckets = 8 + val bucketSpec = BucketSpec(numBuckets, Seq("j"), Nil) + // json does not support predicate push-down, and thus json is used here + df.write + .format("json") + .bucketBy(numBuckets, "j") + .saveAsTable("bucketed_table") + + for (j <- 0 until 13) { + checkPrunedAnswers( + bucketSpec, + bucketValues = j :: Nil, + filterCondition = $"j" === j, + df) + } + } + } + + test("read partitioning bucketed tables having null in bucketing key") { + withTable("bucketed_table") { + val numBuckets = 8 + val bucketSpec = BucketSpec(numBuckets, Seq("j"), Nil) + // json does not support predicate push-down, and thus json is used here + nullDF.write + .format("json") + .partitionBy("i") + .bucketBy(numBuckets, "j") + .saveAsTable("bucketed_table") + + // Case 1: isNull + checkPrunedAnswers( + bucketSpec, + bucketValues = null :: Nil, + filterCondition = $"j".isNull, + nullDF) + + // Case 2: <=> null + checkPrunedAnswers( + bucketSpec, + bucketValues = null :: Nil, + filterCondition = $"j" <=> null, + nullDF) + } + } + + test("read partitioning bucketed tables having composite filters") { + withTable("bucketed_table") { + val numBuckets = 8 + val bucketSpec = BucketSpec(numBuckets, Seq("j"), Nil) + // json does not support predicate push-down, and thus json is used here + df.write + .format("json") + .partitionBy("i") + .bucketBy(numBuckets, "j") + .saveAsTable("bucketed_table") + + for (j <- 0 until 13) { + checkPrunedAnswers( + bucketSpec, + bucketValues = j :: Nil, + filterCondition = $"j" === j && $"k" > $"j", + df) + + checkPrunedAnswers( + bucketSpec, + bucketValues = j :: Nil, + filterCondition = $"j" === j && $"i" > j % 5, + df) + } + } + } + + private lazy val df1 = + (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k").as("df1") + private lazy val df2 = + (0 until 50).map(i => (i % 7, i % 11, i.toString)).toDF("i", "j", "k").as("df2") + + case class BucketedTableTestSpec( + bucketSpec: Option[BucketSpec], + numPartitions: Int = 10, + expectedShuffle: Boolean = true, + expectedSort: Boolean = true) + + /** + * A helper method to test the bucket read functionality using join. It will save `df1` and `df2` + * to hive tables, bucketed or not, according to the given bucket specifics. Next we will join + * these 2 tables, and firstly make sure the answer is corrected, and then check if the shuffle + * exists as user expected according to the `shuffleLeft` and `shuffleRight`. + */ + private def testBucketing( + bucketedTableTestSpecLeft: BucketedTableTestSpec, + bucketedTableTestSpecRight: BucketedTableTestSpec, + joinType: String = "inner", + joinCondition: (DataFrame, DataFrame) => Column): Unit = { + val BucketedTableTestSpec(bucketSpecLeft, numPartitionsLeft, shuffleLeft, sortLeft) = + bucketedTableTestSpecLeft + val BucketedTableTestSpec(bucketSpecRight, numPartitionsRight, shuffleRight, sortRight) = + bucketedTableTestSpecRight + + withTable("bucketed_table1", "bucketed_table2") { + def withBucket( + writer: DataFrameWriter[Row], + bucketSpec: Option[BucketSpec]): DataFrameWriter[Row] = { + bucketSpec.map { spec => + writer.bucketBy( + spec.numBuckets, + spec.bucketColumnNames.head, + spec.bucketColumnNames.tail: _*) + + if (spec.sortColumnNames.nonEmpty) { + writer.sortBy( + spec.sortColumnNames.head, + spec.sortColumnNames.tail: _* + ) + } else { + writer + } + }.getOrElse(writer) + } + + withBucket(df1.repartition(numPartitionsLeft).write.format("parquet"), bucketSpecLeft) + .saveAsTable("bucketed_table1") + withBucket(df2.repartition(numPartitionsRight).write.format("parquet"), bucketSpecRight) + .saveAsTable("bucketed_table2") + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val t1 = spark.table("bucketed_table1") + val t2 = spark.table("bucketed_table2") + val joined = t1.join(t2, joinCondition(t1, t2), joinType) + + // First check the result is corrected. + checkAnswer( + joined.sort("bucketed_table1.k", "bucketed_table2.k"), + df1.join(df2, joinCondition(df1, df2), joinType).sort("df1.k", "df2.k")) + + assert(joined.queryExecution.executedPlan.isInstanceOf[SortMergeJoinExec]) + val joinOperator = joined.queryExecution.executedPlan.asInstanceOf[SortMergeJoinExec] + + // check existence of shuffle + assert( + joinOperator.left.find(_.isInstanceOf[ShuffleExchange]).isDefined == shuffleLeft, + s"expected shuffle in plan to be $shuffleLeft but found\n${joinOperator.left}") + assert( + joinOperator.right.find(_.isInstanceOf[ShuffleExchange]).isDefined == shuffleRight, + s"expected shuffle in plan to be $shuffleRight but found\n${joinOperator.right}") + + // check existence of sort + assert( + joinOperator.left.find(_.isInstanceOf[SortExec]).isDefined == sortLeft, + s"expected sort in the left child to be $sortLeft but found\n${joinOperator.left}") + assert( + joinOperator.right.find(_.isInstanceOf[SortExec]).isDefined == sortRight, + s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") + } + } + } + + private def joinCondition(joinCols: Seq[String]) (left: DataFrame, right: DataFrame): Column = { + joinCols.map(col => left(col) === right(col)).reduce(_ && _) + } + + test("avoid shuffle when join 2 bucketed tables") { + val bucketSpec = Some(BucketSpec(8, Seq("i", "j"), Nil)) + val bucketedTableTestSpecLeft = BucketedTableTestSpec(bucketSpec, expectedShuffle = false) + val bucketedTableTestSpecRight = BucketedTableTestSpec(bucketSpec, expectedShuffle = false) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinCondition = joinCondition(Seq("i", "j")) + ) + } + + // Enable it after fix https://issues.apache.org/jira/browse/SPARK-12704 + ignore("avoid shuffle when join keys are a super-set of bucket keys") { + val bucketSpec = Some(BucketSpec(8, Seq("i"), Nil)) + val bucketedTableTestSpecLeft = BucketedTableTestSpec(bucketSpec, expectedShuffle = false) + val bucketedTableTestSpecRight = BucketedTableTestSpec(bucketSpec, expectedShuffle = false) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinCondition = joinCondition(Seq("i", "j")) + ) + } + + test("only shuffle one side when join bucketed table and non-bucketed table") { + val bucketSpec = Some(BucketSpec(8, Seq("i", "j"), Nil)) + val bucketedTableTestSpecLeft = BucketedTableTestSpec(bucketSpec, expectedShuffle = false) + val bucketedTableTestSpecRight = BucketedTableTestSpec(None, expectedShuffle = true) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinCondition = joinCondition(Seq("i", "j")) + ) + } + + test("only shuffle one side when 2 bucketed tables have different bucket number") { + val bucketSpecLeft = Some(BucketSpec(8, Seq("i", "j"), Nil)) + val bucketSpecRight = Some(BucketSpec(5, Seq("i", "j"), Nil)) + val bucketedTableTestSpecLeft = BucketedTableTestSpec(bucketSpecLeft, expectedShuffle = false) + val bucketedTableTestSpecRight = BucketedTableTestSpec(bucketSpecRight, expectedShuffle = true) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinCondition = joinCondition(Seq("i", "j")) + ) + } + + test("only shuffle one side when 2 bucketed tables have different bucket keys") { + val bucketSpecLeft = Some(BucketSpec(8, Seq("i"), Nil)) + val bucketSpecRight = Some(BucketSpec(8, Seq("j"), Nil)) + val bucketedTableTestSpecLeft = BucketedTableTestSpec(bucketSpecLeft, expectedShuffle = false) + val bucketedTableTestSpecRight = BucketedTableTestSpec(bucketSpecRight, expectedShuffle = true) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinCondition = joinCondition(Seq("i")) + ) + } + + test("shuffle when join keys are not equal to bucket keys") { + val bucketSpec = Some(BucketSpec(8, Seq("i"), Nil)) + val bucketedTableTestSpecLeft = BucketedTableTestSpec(bucketSpec, expectedShuffle = true) + val bucketedTableTestSpecRight = BucketedTableTestSpec(bucketSpec, expectedShuffle = true) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinCondition = joinCondition(Seq("j")) + ) + } + + test("shuffle when join 2 bucketed tables with bucketing disabled") { + val bucketSpec = Some(BucketSpec(8, Seq("i", "j"), Nil)) + val bucketedTableTestSpecLeft = BucketedTableTestSpec(bucketSpec, expectedShuffle = true) + val bucketedTableTestSpecRight = BucketedTableTestSpec(bucketSpec, expectedShuffle = true) + withSQLConf(SQLConf.BUCKETING_ENABLED.key -> "false") { + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinCondition = joinCondition(Seq("i", "j")) + ) + } + } + + test("check sort and shuffle when bucket and sort columns are join keys") { + // In case of bucketing, its possible to have multiple files belonging to the + // same bucket in a given relation. Each of these files are locally sorted + // but those files combined together are not globally sorted. Given that, + // the RDD partition will not be sorted even if the relation has sort columns set + // Therefore, we still need to keep the Sort in both sides. + val bucketSpec = Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))) + + val bucketedTableTestSpecLeft1 = BucketedTableTestSpec( + bucketSpec, numPartitions = 50, expectedShuffle = false, expectedSort = true) + val bucketedTableTestSpecRight1 = BucketedTableTestSpec( + bucketSpec, numPartitions = 1, expectedShuffle = false, expectedSort = false) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft1, + bucketedTableTestSpecRight = bucketedTableTestSpecRight1, + joinCondition = joinCondition(Seq("i", "j")) + ) + + val bucketedTableTestSpecLeft2 = BucketedTableTestSpec( + bucketSpec, numPartitions = 1, expectedShuffle = false, expectedSort = false) + val bucketedTableTestSpecRight2 = BucketedTableTestSpec( + bucketSpec, numPartitions = 50, expectedShuffle = false, expectedSort = true) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft2, + bucketedTableTestSpecRight = bucketedTableTestSpecRight2, + joinCondition = joinCondition(Seq("i", "j")) + ) + + val bucketedTableTestSpecLeft3 = BucketedTableTestSpec( + bucketSpec, numPartitions = 50, expectedShuffle = false, expectedSort = true) + val bucketedTableTestSpecRight3 = BucketedTableTestSpec( + bucketSpec, numPartitions = 50, expectedShuffle = false, expectedSort = true) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft3, + bucketedTableTestSpecRight = bucketedTableTestSpecRight3, + joinCondition = joinCondition(Seq("i", "j")) + ) + + val bucketedTableTestSpecLeft4 = BucketedTableTestSpec( + bucketSpec, numPartitions = 1, expectedShuffle = false, expectedSort = false) + val bucketedTableTestSpecRight4 = BucketedTableTestSpec( + bucketSpec, numPartitions = 1, expectedShuffle = false, expectedSort = false) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft4, + bucketedTableTestSpecRight = bucketedTableTestSpecRight4, + joinCondition = joinCondition(Seq("i", "j")) + ) + } + + test("avoid shuffle and sort when sort columns are a super set of join keys") { + val bucketSpecLeft = Some(BucketSpec(8, Seq("i"), Seq("i", "j"))) + val bucketSpecRight = Some(BucketSpec(8, Seq("i"), Seq("i", "k"))) + val bucketedTableTestSpecLeft = BucketedTableTestSpec( + bucketSpecLeft, numPartitions = 1, expectedShuffle = false, expectedSort = false) + val bucketedTableTestSpecRight = BucketedTableTestSpec( + bucketSpecRight, numPartitions = 1, expectedShuffle = false, expectedSort = false) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinCondition = joinCondition(Seq("i")) + ) + } + + test("only sort one side when sort columns are different") { + val bucketSpecLeft = Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))) + val bucketSpecRight = Some(BucketSpec(8, Seq("i", "j"), Seq("k"))) + val bucketedTableTestSpecLeft = BucketedTableTestSpec( + bucketSpecLeft, numPartitions = 1, expectedShuffle = false, expectedSort = false) + val bucketedTableTestSpecRight = BucketedTableTestSpec( + bucketSpecRight, numPartitions = 1, expectedShuffle = false, expectedSort = true) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinCondition = joinCondition(Seq("i", "j")) + ) + } + + test("only sort one side when sort columns are same but their ordering is different") { + val bucketSpecLeft = Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))) + val bucketSpecRight = Some(BucketSpec(8, Seq("i", "j"), Seq("j", "i"))) + val bucketedTableTestSpecLeft = BucketedTableTestSpec( + bucketSpecLeft, numPartitions = 1, expectedShuffle = false, expectedSort = false) + val bucketedTableTestSpecRight = BucketedTableTestSpec( + bucketSpecRight, numPartitions = 1, expectedShuffle = false, expectedSort = true) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinCondition = joinCondition(Seq("i", "j")) + ) + } + + test("avoid shuffle when grouping keys are equal to bucket keys") { + withTable("bucketed_table") { + df1.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("bucketed_table") + val tbl = spark.table("bucketed_table") + val agged = tbl.groupBy("i", "j").agg(max("k")) + + checkAnswer( + agged.sort("i", "j"), + df1.groupBy("i", "j").agg(max("k")).sort("i", "j")) + + assert(agged.queryExecution.executedPlan.find(_.isInstanceOf[ShuffleExchange]).isEmpty) + } + } + + test("avoid shuffle when grouping keys are a super-set of bucket keys") { + withTable("bucketed_table") { + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") + val tbl = spark.table("bucketed_table") + val agged = tbl.groupBy("i", "j").agg(max("k")) + + checkAnswer( + agged.sort("i", "j"), + df1.groupBy("i", "j").agg(max("k")).sort("i", "j")) + + assert(agged.queryExecution.executedPlan.find(_.isInstanceOf[ShuffleExchange]).isEmpty) + } + } + + test("SPARK-17698 Join predicates should not contain filter clauses") { + val bucketSpec = Some(BucketSpec(8, Seq("i"), Seq("i"))) + val bucketedTableTestSpecLeft = BucketedTableTestSpec( + bucketSpec, numPartitions = 1, expectedShuffle = false, expectedSort = false) + val bucketedTableTestSpecRight = BucketedTableTestSpec( + bucketSpec, numPartitions = 1, expectedShuffle = false, expectedSort = false) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpecLeft, + bucketedTableTestSpecRight = bucketedTableTestSpecRight, + joinType = "fullouter", + joinCondition = (left: DataFrame, right: DataFrame) => { + val joinPredicates = Seq("i").map(col => left(col) === right(col)).reduce(_ && _) + val filterLeft = left("i") === Literal("1") + val filterRight = right("i") === Literal("1") + joinPredicates && filterLeft && filterRight + } + ) + } + + test("SPARK-19122 Re-order join predicates if they match with the child's output partitioning") { + val bucketedTableTestSpec = BucketedTableTestSpec( + Some(BucketSpec(8, Seq("i", "j", "k"), Seq("i", "j", "k"))), + numPartitions = 1, + expectedShuffle = false, + expectedSort = false) + + // If the set of join columns is equal to the set of bucketed + sort columns, then + // the order of join keys in the query should not matter and there should not be any shuffle + // and sort added in the query plan + Seq( + Seq("i", "j", "k"), + Seq("i", "k", "j"), + Seq("j", "k", "i"), + Seq("j", "i", "k"), + Seq("k", "j", "i"), + Seq("k", "i", "j") + ).foreach(joinKeys => { + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpec, + bucketedTableTestSpecRight = bucketedTableTestSpec, + joinCondition = joinCondition(joinKeys) + ) + }) + } + + test("SPARK-19122 No re-ordering should happen if set of join columns != set of child's " + + "partitioning columns") { + + // join predicates is a super set of child's partitioning columns + val bucketedTableTestSpec1 = + BucketedTableTestSpec(Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), numPartitions = 1) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpec1, + bucketedTableTestSpecRight = bucketedTableTestSpec1, + joinCondition = joinCondition(Seq("i", "j", "k")) + ) + + // child's partitioning columns is a super set of join predicates + val bucketedTableTestSpec2 = + BucketedTableTestSpec(Some(BucketSpec(8, Seq("i", "j", "k"), Seq("i", "j", "k"))), + numPartitions = 1) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpec2, + bucketedTableTestSpecRight = bucketedTableTestSpec2, + joinCondition = joinCondition(Seq("i", "j")) + ) + + // set of child's partitioning columns != set join predicates (despite the lengths of the + // sets are same) + val bucketedTableTestSpec3 = + BucketedTableTestSpec(Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))), numPartitions = 1) + testBucketing( + bucketedTableTestSpecLeft = bucketedTableTestSpec3, + bucketedTableTestSpecRight = bucketedTableTestSpec3, + joinCondition = joinCondition(Seq("j", "k")) + ) + } + + test("error if there exists any malformed bucket files") { + withTable("bucketed_table") { + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") + val warehouseFilePath = new URI(spark.sessionState.conf.warehousePath).getPath + val tableDir = new File(warehouseFilePath, "bucketed_table") + Utils.deleteRecursively(tableDir) + df1.write.parquet(tableDir.getAbsolutePath) + + val agged = spark.table("bucketed_table").groupBy("i").count() + val error = intercept[Exception] { + agged.count() + } + + assert(error.getCause().toString contains "Invalid bucket file") + } + } + + test("disable bucketing when the output doesn't contain all bucketing columns") { + withTable("bucketed_table") { + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") + + checkAnswer(spark.table("bucketed_table").select("j"), df1.select("j")) + + checkAnswer(spark.table("bucketed_table").groupBy("j").agg(max("k")), + df1.groupBy("j").agg(max("k"))) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala new file mode 100644 index 000000000000..93f3efe2ccc4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala @@ -0,0 +1,249 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.net.URI + +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +import org.apache.spark.sql.execution.datasources.BucketingUtils +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} + +class BucketedWriteWithoutHiveSupportSuite extends BucketedWriteSuite with SharedSQLContext { + protected override def beforeAll(): Unit = { + super.beforeAll() + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "in-memory") + } + + override protected def fileFormatsToTest: Seq[String] = Seq("parquet", "json") +} + +abstract class BucketedWriteSuite extends QueryTest with SQLTestUtils { + import testImplicits._ + + protected def fileFormatsToTest: Seq[String] + + test("bucketed by non-existing column") { + val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") + intercept[AnalysisException](df.write.bucketBy(2, "k").saveAsTable("tt")) + } + + test("numBuckets be greater than 0 but less than 100000") { + val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") + + Seq(-1, 0, 100000).foreach(numBuckets => { + val e = intercept[AnalysisException](df.write.bucketBy(numBuckets, "i").saveAsTable("tt")) + assert( + e.getMessage.contains("Number of buckets should be greater than 0 but less than 100000")) + }) + } + + test("specify sorting columns without bucketing columns") { + val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") + intercept[IllegalArgumentException](df.write.sortBy("j").saveAsTable("tt")) + } + + test("sorting by non-orderable column") { + val df = Seq("a" -> Map(1 -> 1), "b" -> Map(2 -> 2)).toDF("i", "j") + intercept[AnalysisException](df.write.bucketBy(2, "i").sortBy("j").saveAsTable("tt")) + } + + test("write bucketed data using save()") { + val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") + + val e = intercept[AnalysisException] { + df.write.bucketBy(2, "i").parquet("/tmp/path") + } + assert(e.getMessage == "'save' does not support bucketing right now;") + } + + test("write bucketed data using insertInto()") { + val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") + + val e = intercept[AnalysisException] { + df.write.bucketBy(2, "i").insertInto("tt") + } + assert(e.getMessage == "'insertInto' does not support bucketing right now;") + } + + private lazy val df = { + (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k") + } + + def tableDir: File = { + val identifier = spark.sessionState.sqlParser.parseTableIdentifier("bucketed_table") + new File(spark.sessionState.catalog.defaultTablePath(identifier)) + } + + /** + * A helper method to check the bucket write functionality in low level, i.e. check the written + * bucket files to see if the data are correct. User should pass in a data dir that these bucket + * files are written to, and the format of data(parquet, json, etc.), and the bucketing + * information. + */ + private def testBucketing( + dataDir: File, + source: String, + numBuckets: Int, + bucketCols: Seq[String], + sortCols: Seq[String] = Nil): Unit = { + val allBucketFiles = dataDir.listFiles().filterNot(f => + f.getName.startsWith(".") || f.getName.startsWith("_") + ) + + for (bucketFile <- allBucketFiles) { + val bucketId = BucketingUtils.getBucketId(bucketFile.getName).getOrElse { + fail(s"Unable to find the related bucket files.") + } + + // Remove the duplicate columns in bucketCols and sortCols; + // Otherwise, we got analysis errors due to duplicate names + val selectedColumns = (bucketCols ++ sortCols).distinct + // We may lose the type information after write(e.g. json format doesn't keep schema + // information), here we get the types from the original dataframe. + val types = df.select(selectedColumns.map(col): _*).schema.map(_.dataType) + val columns = selectedColumns.zip(types).map { + case (colName, dt) => col(colName).cast(dt) + } + + // Read the bucket file into a dataframe, so that it's easier to test. + val readBack = spark.read.format(source) + .load(bucketFile.getAbsolutePath) + .select(columns: _*) + + // If we specified sort columns while writing bucket table, make sure the data in this + // bucket file is already sorted. + if (sortCols.nonEmpty) { + checkAnswer(readBack.sort(sortCols.map(col): _*), readBack.collect()) + } + + // Go through all rows in this bucket file, calculate bucket id according to bucket column + // values, and make sure it equals to the expected bucket id that inferred from file name. + val qe = readBack.select(bucketCols.map(col): _*).queryExecution + val rows = qe.toRdd.map(_.copy()).collect() + val getBucketId = UnsafeProjection.create( + HashPartitioning(qe.analyzed.output, numBuckets).partitionIdExpression :: Nil, + qe.analyzed.output) + + for (row <- rows) { + val actualBucketId = getBucketId(row).getInt(0) + assert(actualBucketId == bucketId) + } + } + } + + test("write bucketed data") { + for (source <- fileFormatsToTest) { + withTable("bucketed_table") { + df.write + .format(source) + .partitionBy("i") + .bucketBy(8, "j", "k") + .saveAsTable("bucketed_table") + + for (i <- 0 until 5) { + testBucketing(new File(tableDir, s"i=$i"), source, 8, Seq("j", "k")) + } + } + } + } + + test("write bucketed data with sortBy") { + for (source <- fileFormatsToTest) { + withTable("bucketed_table") { + df.write + .format(source) + .partitionBy("i") + .bucketBy(8, "j") + .sortBy("k") + .saveAsTable("bucketed_table") + + for (i <- 0 until 5) { + testBucketing(new File(tableDir, s"i=$i"), source, 8, Seq("j"), Seq("k")) + } + } + } + } + + test("write bucketed data with the overlapping bucketBy/sortBy and partitionBy columns") { + val e1 = intercept[AnalysisException](df.write + .partitionBy("i", "j") + .bucketBy(8, "j", "k") + .sortBy("k") + .saveAsTable("bucketed_table")) + assert(e1.message.contains("bucketing column 'j' should not be part of partition columns")) + + val e2 = intercept[AnalysisException](df.write + .partitionBy("i", "j") + .bucketBy(8, "k") + .sortBy("i") + .saveAsTable("bucketed_table")) + assert(e2.message.contains("bucket sorting column 'i' should not be part of partition columns")) + } + + test("write bucketed data without partitionBy") { + for (source <- fileFormatsToTest) { + withTable("bucketed_table") { + df.write + .format(source) + .bucketBy(8, "i", "j") + .saveAsTable("bucketed_table") + + testBucketing(tableDir, source, 8, Seq("i", "j")) + } + } + } + + test("write bucketed data without partitionBy with sortBy") { + for (source <- fileFormatsToTest) { + withTable("bucketed_table") { + df.write + .format(source) + .bucketBy(8, "i", "j") + .sortBy("k") + .saveAsTable("bucketed_table") + + testBucketing(tableDir, source, 8, Seq("i", "j"), Seq("k")) + } + } + } + + test("write bucketed data with bucketing disabled") { + // The configuration BUCKETING_ENABLED does not affect the writing path + withSQLConf(SQLConf.BUCKETING_ENABLED.key -> "false") { + for (source <- fileFormatsToTest) { + withTable("bucketed_table") { + df.write + .format(source) + .partitionBy("i") + .bucketBy(8, "j", "k") + .saveAsTable("bucketed_table") + + for (i <- 0 until 5) { + testBucketing(new File(tableDir, s"i=$i"), source, 8, Seq("j", "k")) + } + } + } + } + } +} 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 6fc9febe4970..916a01ee0ca8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -17,199 +17,248 @@ package org.apache.spark.sql.sources -import java.io.{File, IOException} +import java.io.File -import org.scalatest.BeforeAndAfter +import org.scalatest.BeforeAndAfterEach +import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.execution.datasources.DDLException +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils -class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with BeforeAndAfter { - protected override lazy val sql = caseInsensitiveContext.sql _ +class CreateTableAsSelectSuite + extends DataSourceTest + with SharedSQLContext + with BeforeAndAfterEach { + import testImplicits._ + + protected override lazy val sql = spark.sql _ private var path: File = null override def beforeAll(): Unit = { super.beforeAll() - path = Utils.createTempDir() - val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) - caseInsensitiveContext.read.json(rdd).registerTempTable("jt") + val ds = (1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""").toDS() + spark.read.json(ds).createOrReplaceTempView("jt") } override def afterAll(): Unit = { try { - caseInsensitiveContext.dropTempTable("jt") + spark.catalog.dropTempView("jt") + Utils.deleteRecursively(path) } finally { super.afterAll() } } - after { + override def beforeEach(): Unit = { + super.beforeEach() + path = Utils.createTempDir() + path.delete() + } + + override def afterEach(): Unit = { Utils.deleteRecursively(path) + super.afterEach() } - test("CREATE TEMPORARY TABLE AS SELECT") { - sql( - s""" - |CREATE TEMPORARY TABLE jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT a, b FROM jt - """.stripMargin) - - checkAnswer( - sql("SELECT a, b FROM jsonTable"), - sql("SELECT a, b FROM jt").collect()) - - caseInsensitiveContext.dropTempTable("jsonTable") + test("CREATE TABLE USING AS SELECT") { + withTable("jsonTable") { + sql( + s""" + |CREATE TABLE jsonTable + |USING json + |OPTIONS ( + | path '${path.toURI}' + |) AS + |SELECT a, b FROM jt + """.stripMargin) + + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + sql("SELECT a, b FROM jt")) + } } - test("CREATE TEMPORARY TABLE AS SELECT based on the file without write permission") { + test("CREATE TABLE USING AS SELECT based on the file without write permission") { + // setWritable(...) does not work on Windows. Please refer JDK-6728842. + assume(!Utils.isWindows) val childPath = new File(path.toString, "child") path.mkdir() - childPath.createNewFile() path.setWritable(false) - val e = intercept[IOException] { + val e = intercept[SparkException] { sql( s""" - |CREATE TEMPORARY TABLE jsonTable + |CREATE TABLE jsonTable |USING json |OPTIONS ( - | path '${path.toString}' + | path '${childPath.toURI}' |) AS |SELECT a, b FROM jt - """.stripMargin) + """.stripMargin) sql("SELECT a, b FROM jsonTable").collect() } - assert(e.getMessage().contains("Unable to clear output directory")) + assert(e.getMessage().contains("Job aborted")) path.setWritable(true) } test("create a table, drop it and create another one with the same name") { - sql( - s""" - |CREATE TEMPORARY TABLE jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT a, b FROM jt - """.stripMargin) - - checkAnswer( - sql("SELECT a, b FROM jsonTable"), - sql("SELECT a, b FROM jt").collect()) - - val message = intercept[DDLException]{ + withTable("jsonTable") { sql( s""" - |CREATE TEMPORARY TABLE IF NOT EXISTS jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT a * 4 FROM jt - """.stripMargin) - }.getMessage - assert( - 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.") - - // Overwrite the temporary table. - sql( - s""" - |CREATE TEMPORARY TABLE jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT a * 4 FROM jt - """.stripMargin) - checkAnswer( - sql("SELECT * FROM jsonTable"), - sql("SELECT a * 4 FROM jt").collect()) - - caseInsensitiveContext.dropTempTable("jsonTable") - // Explicitly delete the data. - if (path.exists()) Utils.deleteRecursively(path) - - sql( - s""" - |CREATE TEMPORARY TABLE jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT b FROM jt - """.stripMargin) - - checkAnswer( - sql("SELECT * FROM jsonTable"), - sql("SELECT b FROM jt").collect()) - - caseInsensitiveContext.dropTempTable("jsonTable") - } + |CREATE TABLE jsonTable + |USING json + |OPTIONS ( + | path '${path.toURI}' + |) AS + |SELECT a, b FROM jt + """.stripMargin) + + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + sql("SELECT a, b FROM jt")) + + // Creates a table of the same name with flag "if not exists", nothing happens + sql( + s""" + |CREATE TABLE IF NOT EXISTS jsonTable + |USING json + |OPTIONS ( + | path '${path.toURI}' + |) AS + |SELECT a * 4 FROM jt + """.stripMargin) + checkAnswer( + sql("SELECT * FROM jsonTable"), + sql("SELECT a, b FROM jt")) + + // Explicitly drops the table and deletes the underlying data. + sql("DROP TABLE jsonTable") + if (path.exists()) Utils.deleteRecursively(path) - test("CREATE TEMPORARY TABLE AS SELECT with IF NOT EXISTS is not allowed") { - val message = intercept[DDLException]{ + // Creates a table of the same name again, this time we succeed. sql( s""" - |CREATE TEMPORARY TABLE IF NOT EXISTS jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT b FROM jt - """.stripMargin) - }.getMessage - assert( - message.contains("a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause."), - "CREATE TEMPORARY TABLE IF NOT EXISTS should not be allowed.") + |CREATE TABLE jsonTable + |USING json + |OPTIONS ( + | path '${path.toURI}' + |) AS + |SELECT b FROM jt + """.stripMargin) + + checkAnswer( + sql("SELECT * FROM jsonTable"), + sql("SELECT b FROM jt")) + } + } + + test("disallows CREATE TEMPORARY TABLE ... USING ... AS query") { + withTable("t") { + val error = intercept[ParseException] { + sql( + s""" + |CREATE TEMPORARY TABLE t USING PARQUET + |OPTIONS (PATH '${path.toURI}') + |PARTITIONED BY (a) + |AS SELECT 1 AS a, 2 AS b + """.stripMargin + ) + }.getMessage + assert(error.contains("Operation not allowed") && + error.contains("CREATE TEMPORARY TABLE ... USING ... AS query")) + } + } + + test("disallows CREATE EXTERNAL TABLE ... USING ... AS query") { + withTable("t") { + val error = intercept[ParseException] { + sql( + s""" + |CREATE EXTERNAL TABLE t USING PARQUET + |OPTIONS (PATH '${path.toURI}') + |AS SELECT 1 AS a, 2 AS b + """.stripMargin + ) + }.getMessage + + assert(error.contains("Operation not allowed") && + error.contains("CREATE EXTERNAL TABLE ... USING")) + } } - test("a CTAS statement with column definitions is not allowed") { - intercept[DDLException]{ + test("create table using as select - with partitioned by") { + val catalog = spark.sessionState.catalog + withTable("t") { sql( s""" - |CREATE TEMPORARY TABLE jsonTable (a int, b string) - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT a, b FROM jt - """.stripMargin) + |CREATE TABLE t USING PARQUET + |OPTIONS (PATH '${path.toURI}') + |PARTITIONED BY (a) + |AS SELECT 1 AS a, 2 AS b + """.stripMargin + ) + val table = catalog.getTableMetadata(TableIdentifier("t")) + assert(table.partitionColumnNames == Seq("a")) } } - test("it is not allowed to write to a table while querying it.") { - sql( - s""" - |CREATE TEMPORARY TABLE jsonTable - |USING json - |OPTIONS ( - | path '${path.toString}' - |) AS - |SELECT a, b FROM jt - """.stripMargin) - - val message = intercept[AnalysisException] { + test("create table using as select - with valid number of buckets") { + val catalog = spark.sessionState.catalog + withTable("t") { sql( s""" - |CREATE TEMPORARY TABLE jsonTable - |USING json - |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.") + |CREATE TABLE t USING PARQUET + |OPTIONS (PATH '${path.toURI}') + |CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS + |AS SELECT 1 AS a, 2 AS b + """.stripMargin + ) + val table = catalog.getTableMetadata(TableIdentifier("t")) + assert(table.bucketSpec == Option(BucketSpec(5, Seq("a"), Seq("b")))) + } + } + + test("create table using as select - with invalid number of buckets") { + withTable("t") { + Seq(0, 100000).foreach(numBuckets => { + val e = intercept[AnalysisException] { + sql( + s""" + |CREATE TABLE t USING PARQUET + |OPTIONS (PATH '${path.toURI}') + |CLUSTERED BY (a) SORTED BY (b) INTO $numBuckets BUCKETS + |AS SELECT 1 AS a, 2 AS b + """.stripMargin + ) + }.getMessage + assert(e.contains("Number of buckets should be greater than 0 but less than 100000")) + }) + } + } + + test("SPARK-17409: CTAS of decimal calculation") { + withTable("tab2") { + withTempView("tab1") { + spark.range(99, 101).createOrReplaceTempView("tab1") + val sqlStmt = + "SELECT id, cast(id as long) * cast('1.0' as decimal(38, 18)) as num FROM tab1" + sql(s"CREATE TABLE tab2 USING PARQUET AS $sqlStmt") + checkAnswer(spark.table("tab2"), sql(sqlStmt)) + } + } + } + + test("specifying the column list for CTAS") { + withTable("t") { + val e = intercept[ParseException] { + sql("CREATE TABLE t (a int, b int) USING parquet AS SELECT 1, 2") + }.getMessage + assert(e.contains("Schema may not be specified in a Create Table As Select (CTAS)")) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala index 853707c036c9..3ce6ae3c5292 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala @@ -1,50 +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. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.sql.sources -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{AnalysisException, SQLContext} import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.sql.types._ // please note that the META-INF/services had to be modified for the test directory for this to work class DDLSourceLoadSuite extends DataSourceTest with SharedSQLContext { - test("data sources with the same name") { - intercept[RuntimeException] { - caseInsensitiveContext.read.format("Fluet da Bomb").load() + test("data sources with the same name - internal data sources") { + val e = intercept[AnalysisException] { + spark.read.format("Fluet da Bomb").load() } + assert(e.getMessage.contains("Multiple sources found for Fluet da Bomb")) + } + + test("data sources with the same name - internal data source/external data source") { + assert(spark.read.format("datasource").load().schema == + StructType(Seq(StructField("longType", LongType, nullable = false)))) + } + + test("data sources with the same name - external data sources") { + val e = intercept[AnalysisException] { + spark.read.format("Fake external source").load() + } + assert(e.getMessage.contains("Multiple sources found for Fake external source")) } test("load data source from format alias") { - caseInsensitiveContext.read.format("gathering quorum").load().schema == - StructType(Seq(StructField("stringType", StringType, nullable = false))) + assert(spark.read.format("gathering quorum").load().schema == + StructType(Seq(StructField("stringType", StringType, nullable = false)))) } test("specify full classname with duplicate formats") { - caseInsensitiveContext.read.format("org.apache.spark.sql.sources.FakeSourceOne") - .load().schema == StructType(Seq(StructField("stringType", StringType, nullable = false))) + assert(spark.read.format("org.apache.spark.sql.sources.FakeSourceOne") + .load().schema == StructType(Seq(StructField("stringType", StringType, nullable = false)))) } - test("should fail to load ORC without HiveContext") { - intercept[ClassNotFoundException] { - caseInsensitiveContext.read.format("orc").load() + test("should fail to load ORC without Hive Support") { + val e = intercept[AnalysisException] { + spark.read.format("orc").load() } + assert(e.message.contains("The ORC data source must be used with Hive support enabled")) } } @@ -62,7 +76,7 @@ class FakeSourceOne extends RelationProvider with DataSourceRegister { } } -class FakeSourceTwo extends RelationProvider with DataSourceRegister { +class FakeSourceTwo extends RelationProvider with DataSourceRegister { def shortName(): String = "Fluet da Bomb" @@ -71,7 +85,7 @@ class FakeSourceTwo extends RelationProvider with DataSourceRegister { override def sqlContext: SQLContext = cont override def schema: StructType = - StructType(Seq(StructField("stringType", StringType, nullable = false))) + StructType(Seq(StructField("integerType", IntegerType, nullable = false))) } } @@ -87,3 +101,16 @@ class FakeSourceThree extends RelationProvider with DataSourceRegister { StructType(Seq(StructField("stringType", StringType, nullable = false))) } } + +class FakeSourceFour extends RelationProvider with DataSourceRegister { + + def shortName(): String = "datasource" + + override def createRelation(cont: SQLContext, param: Map[String, String]): BaseRelation = + new BaseRelation { + override def sqlContext: SQLContext = cont + + override def schema: StructType = + StructType(Seq(StructField("longType", LongType, nullable = false))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala deleted file mode 100644 index 5f8514e1a241..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala +++ /dev/null @@ -1,116 +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.sources - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String - -class DDLScanSource extends RelationProvider { - override def createRelation( - sqlContext: SQLContext, - parameters: Map[String, String]): BaseRelation = { - SimpleDDLScan(parameters("from").toInt, parameters("TO").toInt, parameters("Table"))(sqlContext) - } -} - -case class SimpleDDLScan(from: Int, to: Int, table: String)(@transient val sqlContext: SQLContext) - extends BaseRelation with TableScan { - - override def schema: StructType = - StructType(Seq( - StructField("intType", IntegerType, nullable = false, - new MetadataBuilder().putString("comment", s"test comment $table").build()), - StructField("stringType", StringType, nullable = false), - StructField("dateType", DateType, nullable = false), - StructField("timestampType", TimestampType, nullable = false), - StructField("doubleType", DoubleType, nullable = false), - StructField("bigintType", LongType, nullable = false), - StructField("tinyintType", ByteType, nullable = false), - StructField("decimalType", DecimalType.USER_DEFAULT, nullable = false), - StructField("fixedDecimalType", DecimalType(5, 1), nullable = false), - StructField("binaryType", BinaryType, nullable = false), - StructField("booleanType", BooleanType, nullable = false), - StructField("smallIntType", ShortType, nullable = false), - StructField("floatType", FloatType, nullable = false), - StructField("mapType", MapType(StringType, StringType)), - StructField("arrayType", ArrayType(StringType)), - StructField("structType", - StructType(StructField("f1", StringType) :: StructField("f2", IntegerType) :: Nil - ) - ) - )) - - override def needConversion: Boolean = false - - override def buildScan(): RDD[Row] = { - // Rely on a type erasure hack to pass RDD[InternalRow] back as RDD[Row] - sqlContext.sparkContext.parallelize(from to to).map { e => - InternalRow(UTF8String.fromString(s"people$e"), e * 2) - }.asInstanceOf[RDD[Row]] - } -} - -class DDLTestSuite extends DataSourceTest with SharedSQLContext { - protected override lazy val sql = caseInsensitiveContext.sql _ - - override def beforeAll(): Unit = { - super.beforeAll() - sql( - """ - |CREATE TEMPORARY TABLE ddlPeople - |USING org.apache.spark.sql.sources.DDLScanSource - |OPTIONS ( - | From '1', - | To '10', - | Table 'test1' - |) - """.stripMargin) - } - - sqlTest( - "describe ddlPeople", - Seq( - Row("intType", "int", "test comment test1"), - Row("stringType", "string", ""), - Row("dateType", "date", ""), - Row("timestampType", "timestamp", ""), - Row("doubleType", "double", ""), - Row("bigintType", "bigint", ""), - Row("tinyintType", "tinyint", ""), - Row("decimalType", "decimal(10,0)", ""), - Row("fixedDecimalType", "decimal(5,1)", ""), - Row("binaryType", "binary", ""), - Row("booleanType", "boolean", ""), - Row("smallIntType", "smallint", ""), - Row("floatType", "float", ""), - Row("mapType", "map", ""), - Row("arrayType", "array", ""), - Row("structType", "struct", "") - )) - - test("SPARK-7686 DescribeCommand should have correct physical plan output attributes") { - val attributes = sql("describe ddlPeople") - .queryExecution.executedPlan.output - assert(attributes.map(_.name) === Seq("col_name", "data_type", "comment")) - assert(attributes.map(_.dataType).toSet === Set(StringType)) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala new file mode 100644 index 000000000000..735e07c21373 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, Expression, Literal} +import org.apache.spark.sql.execution.datasources.DataSourceAnalysis +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DataType, IntegerType, StructType} + +class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll { + + private var targetAttributes: Seq[Attribute] = _ + private var targetPartitionSchema: StructType = _ + + override def beforeAll(): Unit = { + targetAttributes = Seq('a.int, 'd.int, 'b.int, 'c.int) + targetPartitionSchema = new StructType() + .add("b", IntegerType) + .add("c", IntegerType) + } + + private def checkProjectList(actual: Seq[Expression], expected: Seq[Expression]): Unit = { + // Remove aliases since we have no control on their exprId. + val withoutAliases = actual.map { + case alias: Alias => alias.child + case other => other + } + assert(withoutAliases === expected) + } + + Seq(true, false).foreach { caseSensitive => + val conf = new SQLConf().copy(SQLConf.CASE_SENSITIVE -> caseSensitive) + def cast(e: Expression, dt: DataType): Expression = { + Cast(e, dt, Option(conf.sessionLocalTimeZone)) + } + val rule = DataSourceAnalysis(conf) + test( + s"convertStaticPartitions only handle INSERT having at least static partitions " + + s"(caseSensitive: $caseSensitive)") { + intercept[AssertionError] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int, 'f.int), + providedPartitions = Map("b" -> None, "c" -> None), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + } + + test(s"Missing columns (caseSensitive: $caseSensitive)") { + // Missing columns. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int), + providedPartitions = Map("b" -> Some("1"), "c" -> None), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + } + + test(s"Missing partitioning columns (caseSensitive: $caseSensitive)") { + // Missing partitioning columns. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int, 'f.int), + providedPartitions = Map("b" -> Some("1")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + + // Missing partitioning columns. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int, 'f.int, 'g.int), + providedPartitions = Map("b" -> Some("1")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + + // Wrong partitioning columns. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int, 'f.int), + providedPartitions = Map("b" -> Some("1"), "d" -> None), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + } + + test(s"Wrong partitioning columns (caseSensitive: $caseSensitive)") { + // Wrong partitioning columns. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int, 'f.int), + providedPartitions = Map("b" -> Some("1"), "d" -> Some("2")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + + // Wrong partitioning columns. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int), + providedPartitions = Map("b" -> Some("1"), "c" -> Some("3"), "d" -> Some("2")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + + if (caseSensitive) { + // Wrong partitioning columns. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int, 'f.int), + providedPartitions = Map("b" -> Some("1"), "C" -> Some("3")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + } + } + + test( + s"Static partitions need to appear before dynamic partitions" + + s" (caseSensitive: $caseSensitive)") { + // Static partitions need to appear before dynamic partitions. + intercept[AnalysisException] { + rule.convertStaticPartitions( + sourceAttributes = Seq('e.int, 'f.int), + providedPartitions = Map("b" -> None, "c" -> Some("3")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + } + } + + test(s"All static partitions (caseSensitive: $caseSensitive)") { + if (!caseSensitive) { + val nonPartitionedAttributes = Seq('e.int, 'f.int) + val expected = nonPartitionedAttributes ++ + Seq(cast(Literal("1"), IntegerType), cast(Literal("3"), IntegerType)) + val actual = rule.convertStaticPartitions( + sourceAttributes = nonPartitionedAttributes, + providedPartitions = Map("b" -> Some("1"), "C" -> Some("3")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + checkProjectList(actual, expected) + } + + { + val nonPartitionedAttributes = Seq('e.int, 'f.int) + val expected = nonPartitionedAttributes ++ + Seq(cast(Literal("1"), IntegerType), cast(Literal("3"), IntegerType)) + val actual = rule.convertStaticPartitions( + sourceAttributes = nonPartitionedAttributes, + providedPartitions = Map("b" -> Some("1"), "c" -> Some("3")), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + checkProjectList(actual, expected) + } + + // Test the case having a single static partition column. + { + val nonPartitionedAttributes = Seq('e.int, 'f.int) + val expected = nonPartitionedAttributes ++ Seq(cast(Literal("1"), IntegerType)) + val actual = rule.convertStaticPartitions( + sourceAttributes = nonPartitionedAttributes, + providedPartitions = Map("b" -> Some("1")), + targetAttributes = Seq('a.int, 'd.int, 'b.int), + targetPartitionSchema = new StructType().add("b", IntegerType)) + checkProjectList(actual, expected) + } + } + + test(s"Static partition and dynamic partition (caseSensitive: $caseSensitive)") { + val nonPartitionedAttributes = Seq('e.int, 'f.int) + val dynamicPartitionAttributes = Seq('g.int) + val expected = + nonPartitionedAttributes ++ + Seq(cast(Literal("1"), IntegerType)) ++ + dynamicPartitionAttributes + val actual = rule.convertStaticPartitions( + sourceAttributes = nonPartitionedAttributes ++ dynamicPartitionAttributes, + providedPartitions = Map("b" -> Some("1"), "c" -> None), + targetAttributes = targetAttributes, + targetPartitionSchema = targetPartitionSchema) + checkProjectList(actual, expected) + } + } +} 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 af04079ec895..1ece98aa7eb3 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 @@ -1,37 +1,89 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.sql.sources +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String private[sql] abstract class DataSourceTest extends QueryTest { - // We want to test some edge cases. - protected lazy val caseInsensitiveContext: SQLContext = { - val ctx = new SQLContext(sqlContext.sparkContext) - ctx.setConf(SQLConf.CASE_SENSITIVE, false) - ctx - } - - protected def sqlTest(sqlString: String, expectedAnswer: Seq[Row]) { + protected def sqlTest(sqlString: String, expectedAnswer: Seq[Row], enableRegex: Boolean = false) { test(sqlString) { - checkAnswer(caseInsensitiveContext.sql(sqlString), expectedAnswer) + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> enableRegex.toString) { + checkAnswer(spark.sql(sqlString), expectedAnswer) + } } } } + +class DDLScanSource extends RelationProvider { + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + SimpleDDLScan( + parameters("from").toInt, + parameters("TO").toInt, + parameters("Table"))(sqlContext.sparkSession) + } +} + +case class SimpleDDLScan( + from: Int, + to: Int, + table: String)(@transient val sparkSession: SparkSession) + extends BaseRelation with TableScan { + + override def sqlContext: SQLContext = sparkSession.sqlContext + + override def schema: StructType = + StructType(Seq( + StructField("intType", IntegerType, nullable = false).withComment(s"test comment $table"), + StructField("stringType", StringType, nullable = false), + StructField("dateType", DateType, nullable = false), + StructField("timestampType", TimestampType, nullable = false), + StructField("doubleType", DoubleType, nullable = false), + StructField("bigintType", LongType, nullable = false), + StructField("tinyintType", ByteType, nullable = false), + StructField("decimalType", DecimalType.USER_DEFAULT, nullable = false), + StructField("fixedDecimalType", DecimalType(5, 1), nullable = false), + StructField("binaryType", BinaryType, nullable = false), + StructField("booleanType", BooleanType, nullable = false), + StructField("smallIntType", ShortType, nullable = false), + StructField("floatType", FloatType, nullable = false), + StructField("mapType", MapType(StringType, StringType)), + StructField("arrayType", ArrayType(StringType)), + StructField("structType", + StructType(StructField("f1", StringType) :: StructField("f2", IntegerType) :: Nil + ) + ) + )) + + override def needConversion: Boolean = false + + override def buildScan(): RDD[Row] = { + // Rely on a type erasure hack to pass RDD[InternalRow] back as RDD[Row] + sparkSession.sparkContext.parallelize(from to to).map { e => + InternalRow(UTF8String.fromString(s"people$e"), e * 2) + }.asInstanceOf[RDD[Row]] + } +} 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 7541e723029b..c45b507d2b48 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 @@ -1,45 +1,49 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.sql.sources -import org.apache.spark.sql.execution.datasources.LogicalRelation +import java.util.Locale import scala.language.existentials import org.apache.spark.rdd.RDD -import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.expressions.PredicateHelper +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ - +import org.apache.spark.unsafe.types.UTF8String class FilteredScanSource extends RelationProvider { override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - SimpleFilteredScan(parameters("from").toInt, parameters("to").toInt)(sqlContext) + SimpleFilteredScan(parameters("from").toInt, parameters("to").toInt)(sqlContext.sparkSession) } } -case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) +case class SimpleFilteredScan(from: Int, to: Int)(@transient val sparkSession: SparkSession) extends BaseRelation with PrunedFilteredScan { + override def sqlContext: SQLContext = sparkSession.sqlContext + override def schema: StructType = StructType( StructField("a", IntegerType, nullable = false) :: @@ -74,7 +78,7 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQL case "b" => (i: Int) => Seq(i * 2) case "c" => (i: Int) => val c = (i - 1 + 'a').toChar.toString - Seq(c * 5 + c.toUpperCase * 5) + Seq(c * 5 + c.toUpperCase(Locale.ROOT) * 5) } FiltersPushed.list = filters @@ -111,11 +115,12 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQL } def eval(a: Int) = { - val c = (a - 1 + 'a').toChar.toString * 5 + (a - 1 + 'a').toChar.toString.toUpperCase * 5 + val c = (a - 1 + 'a').toChar.toString * 5 + + (a - 1 + 'a').toChar.toString.toUpperCase(Locale.ROOT) * 5 filters.forall(translateFilterOnA(_)(a)) && filters.forall(translateFilterOnC(_)(c)) } - sqlContext.sparkContext.parallelize(from to to).filter(eval).map(i => + sparkSession.sparkContext.parallelize(from to to).filter(eval).map(i => Row.fromSeq(rowBuilders.map(_(i)).reduceOption(_ ++ _).getOrElse(Seq.empty))) } } @@ -130,29 +135,26 @@ object ColumnsRequired { var set: Set[String] = Set.empty } -class FilteredScanSuite extends DataSourceTest with SharedSQLContext { - protected override lazy val sql = caseInsensitiveContext.sql _ +class FilteredScanSuite extends DataSourceTest with SharedSQLContext with PredicateHelper { + protected override lazy val sql = spark.sql _ override def beforeAll(): Unit = { super.beforeAll() sql( """ - |CREATE TEMPORARY TABLE oneToTenFiltered + |CREATE TEMPORARY VIEW oneToTenFiltered |USING org.apache.spark.sql.sources.FilteredScanSource |OPTIONS ( | from '1', | to '10' |) """.stripMargin) - - // UDF for testing filter push-down - caseInsensitiveContext.udf.register("udf_gt3", (_: Int) > 3) } sqlTest( "SELECT * FROM oneToTenFiltered", (1 to 10).map(i => Row(i, i * 2, (i - 1 + 'a').toChar.toString * 5 - + (i - 1 + 'a').toChar.toString.toUpperCase * 5)).toSeq) + + (i - 1 + 'a').toChar.toString.toUpperCase(Locale.ROOT) * 5)).toSeq) sqlTest( "SELECT a, b FROM oneToTenFiltered", @@ -258,7 +260,11 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext { testPushDown("SELECT * FROM oneToTenFiltered WHERE a IN (1,3,5)", 3, Set("a", "b", "c")) testPushDown("SELECT * FROM oneToTenFiltered WHERE a = 20", 0, Set("a", "b", "c")) - testPushDown("SELECT * FROM oneToTenFiltered WHERE b = 1", 10, Set("a", "b", "c")) + testPushDown( + "SELECT * FROM oneToTenFiltered WHERE b = 1", + 10, + Set("a", "b", "c"), + Set(EqualTo("b", 1))) testPushDown("SELECT * FROM oneToTenFiltered WHERE a < 5 AND a > 1", 3, Set("a", "b", "c")) testPushDown("SELECT * FROM oneToTenFiltered WHERE a < 3 OR a > 8", 4, Set("a", "b", "c")) @@ -276,48 +282,66 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext { testPushDown("SELECT c FROM oneToTenFiltered WHERE c = 'aaaaaAAAAA'", 1, Set("c")) testPushDown("SELECT c FROM oneToTenFiltered WHERE c IN ('aaaaaAAAAA', 'foo')", 1, Set("c")) - // Columns only referenced by UDF filter must be required, as UDF filters can't be pushed down. - testPushDown("SELECT c FROM oneToTenFiltered WHERE udf_gt3(A)", 10, Set("a", "c")) + // Filters referencing multiple columns are not convertible, all referenced columns must be + // required. + testPushDown("SELECT c FROM oneToTenFiltered WHERE A + b > 9", 10, Set("a", "b", "c")) - // A query with an unconvertible filter, an unhandled filter, and a handled filter. + // A query with an inconvertible filter, an unhandled filter, and a handled filter. testPushDown( """SELECT a | FROM oneToTenFiltered - | WHERE udf_gt3(b) + | WHERE a + b > 9 | AND b < 16 | AND c IN ('bbbbbBBBBB', 'cccccCCCCC', 'dddddDDDDD', 'foo') - """.stripMargin.split("\n").map(_.trim).mkString(" "), 3, Set("a", "b")) + """.stripMargin.split("\n").map(_.trim).mkString(" "), + 3, + Set("a", "b"), + Set(LessThan("b", 16))) def testPushDown( - sqlString: String, - expectedCount: Int, - requiredColumnNames: Set[String]): Unit = { - test(s"PushDown Returns $expectedCount: $sqlString") { - val queryExecution = sql(sqlString).queryExecution - val rawPlan = queryExecution.executedPlan.collect { - case p: execution.PhysicalRDD => p - } match { - case Seq(p) => p - case _ => fail(s"More than one PhysicalRDD found\n$queryExecution") - } - val rawCount = rawPlan.execute().count() - assert(ColumnsRequired.set === requiredColumnNames) + sqlString: String, + expectedCount: Int, + requiredColumnNames: Set[String]): Unit = { + testPushDown(sqlString, expectedCount, requiredColumnNames, Set.empty[Filter]) + } + + def testPushDown( + sqlString: String, + expectedCount: Int, + requiredColumnNames: Set[String], + expectedUnhandledFilters: Set[Filter]): Unit = { - assert { - val table = caseInsensitiveContext.table("oneToTenFiltered") + test(s"PushDown Returns $expectedCount: $sqlString") { + // These tests check a particular plan, disable whole stage codegen. + spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, false) + try { + val queryExecution = sql(sqlString).queryExecution + val rawPlan = queryExecution.executedPlan.collect { + case p: execution.DataSourceScanExec => p + } match { + case Seq(p) => p + case _ => fail(s"More than one PhysicalRDD found\n$queryExecution") + } + val rawCount = rawPlan.execute().count() + assert(ColumnsRequired.set === requiredColumnNames) + + val table = spark.table("oneToTenFiltered") val relation = table.queryExecution.logical.collectFirst { - case LogicalRelation(r, _) => r + case LogicalRelation(r, _, _, _) => r }.get - // `relation` should be able to handle all pushed filters - relation.unhandledFilters(FiltersPushed.list.toArray).isEmpty - } - - if (rawCount != expectedCount) { - fail( - s"Wrong # of results for pushed filter. Got $rawCount, Expected $expectedCount\n" + - s"Filters pushed: ${FiltersPushed.list.mkString(",")}\n" + - queryExecution) + assert( + relation.unhandledFilters(FiltersPushed.list.toArray).toSet === expectedUnhandledFilters) + + if (rawCount != expectedCount) { + fail( + s"Wrong # of results for pushed filter. Got $rawCount, Expected $expectedCount\n" + + s"Filters pushed: ${FiltersPushed.list.mkString(",")}\n" + + queryExecution) + } + } finally { + spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.defaultValue.get) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FiltersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FiltersSuite.scala new file mode 100644 index 000000000000..1cb7a2156c3d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FiltersSuite.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import org.apache.spark.SparkFunSuite + +/** + * Unit test suites for data source filters. + */ +class FiltersSuite extends SparkFunSuite { + + test("EqualTo references") { + assert(EqualTo("a", "1").references.toSeq == Seq("a")) + assert(EqualTo("a", EqualTo("b", "2")).references.toSeq == Seq("a", "b")) + } + + test("EqualNullSafe references") { + assert(EqualNullSafe("a", "1").references.toSeq == Seq("a")) + assert(EqualNullSafe("a", EqualTo("b", "2")).references.toSeq == Seq("a", "b")) + } + + test("GreaterThan references") { + assert(GreaterThan("a", "1").references.toSeq == Seq("a")) + assert(GreaterThan("a", EqualTo("b", "2")).references.toSeq == Seq("a", "b")) + } + + test("GreaterThanOrEqual references") { + assert(GreaterThanOrEqual("a", "1").references.toSeq == Seq("a")) + assert(GreaterThanOrEqual("a", EqualTo("b", "2")).references.toSeq == Seq("a", "b")) + } + + test("LessThan references") { + assert(LessThan("a", "1").references.toSeq == Seq("a")) + assert(LessThan("a", EqualTo("b", "2")).references.toSeq == Seq("a", "b")) + } + + test("LessThanOrEqual references") { + assert(LessThanOrEqual("a", "1").references.toSeq == Seq("a")) + assert(LessThanOrEqual("a", EqualTo("b", "2")).references.toSeq == Seq("a", "b")) + } + + test("In references") { + assert(In("a", Array("1")).references.toSeq == Seq("a")) + assert(In("a", Array("1", EqualTo("b", "2"))).references.toSeq == Seq("a", "b")) + } + + test("IsNull references") { + assert(IsNull("a").references.toSeq == Seq("a")) + } + + test("IsNotNull references") { + assert(IsNotNull("a").references.toSeq == Seq("a")) + } + + test("And references") { + assert(And(EqualTo("a", "1"), EqualTo("b", "1")).references.toSeq == Seq("a", "b")) + } + + test("Or references") { + assert(Or(EqualTo("a", "1"), EqualTo("b", "1")).references.toSeq == Seq("a", "b")) + } + + test("StringStartsWith references") { + assert(StringStartsWith("a", "str").references.toSeq == Seq("a")) + } + + test("StringEndsWith references") { + assert(StringEndsWith("a", "str").references.toSeq == Seq("a")) + } + + test("StringContains references") { + assert(StringContains("a", "str").references.toSeq == Seq("a")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 5b70d258d6ce..875b74551add 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -19,33 +19,36 @@ package org.apache.spark.sql.sources import java.io.File +import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils class InsertSuite extends DataSourceTest with SharedSQLContext { - protected override lazy val sql = caseInsensitiveContext.sql _ + import testImplicits._ + + protected override lazy val sql = spark.sql _ private var path: File = null override def beforeAll(): Unit = { super.beforeAll() path = Utils.createTempDir() - val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""")) - caseInsensitiveContext.read.json(rdd).registerTempTable("jt") + val ds = (1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""").toDS() + spark.read.json(ds).createOrReplaceTempView("jt") sql( s""" - |CREATE TEMPORARY TABLE jsonTable (a int, b string) + |CREATE TEMPORARY VIEW jsonTable (a int, b string) |USING org.apache.spark.sql.json.DefaultSource |OPTIONS ( - | path '${path.toString}' + | path '${path.toURI.toString}' |) """.stripMargin) } override def afterAll(): Unit = { try { - caseInsensitiveContext.dropTempTable("jsonTable") - caseInsensitiveContext.dropTempTable("jt") + spark.catalog.dropTempView("jsonTable") + spark.catalog.dropTempView("jt") Utils.deleteRecursively(path) } finally { super.afterAll() @@ -64,6 +67,26 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { ) } + test("insert into a temp view that does not point to an insertable data source") { + import testImplicits._ + withTempView("t1", "t2") { + sql( + """ + |CREATE TEMPORARY VIEW t1 + |USING org.apache.spark.sql.sources.SimpleScanSource + |OPTIONS ( + | From '1', + | To '10') + """.stripMargin) + sparkContext.parallelize(1 to 10).toDF("a").createOrReplaceTempView("t2") + + val message = intercept[AnalysisException] { + sql("INSERT INTO TABLE t1 SELECT a FROM t2") + }.getMessage + assert(message.contains("does not allow insertion")) + } + } + test("PreInsert casting and renaming") { sql( s""" @@ -87,15 +110,13 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { } test("SELECT clause generating a different number of columns is not allowed.") { - val message = intercept[RuntimeException] { + val message = intercept[AnalysisException] { sql( s""" |INSERT OVERWRITE TABLE jsonTable SELECT a FROM jt """.stripMargin) }.getMessage - assert( - message.contains("generates the same number of columns as its schema"), - "SELECT clause generating a different number of columns should not be not allowed." + assert(message.contains("target table has 2 column(s) but the inserted data has 1 column(s)") ) } @@ -111,7 +132,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { // Writing the table to less part files. val rdd1 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}"""), 5) - caseInsensitiveContext.read.json(rdd1).registerTempTable("jt1") + spark.read.json(rdd1.toDS()).createOrReplaceTempView("jt1") sql( s""" |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt1 @@ -123,7 +144,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { // Writing the table to more part files. val rdd2 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}"""), 10) - caseInsensitiveContext.read.json(rdd2).registerTempTable("jt2") + spark.read.json(rdd1.toDS()).createOrReplaceTempView("jt2") sql( s""" |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt2 @@ -142,8 +163,8 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { (1 to 10).map(i => Row(i * 10, s"str$i")) ) - caseInsensitiveContext.dropTempTable("jt1") - caseInsensitiveContext.dropTempTable("jt2") + spark.catalog.dropTempView("jt1") + spark.catalog.dropTempView("jt2") } test("INSERT INTO JSONRelation for now") { @@ -166,6 +187,48 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { ) } + test("INSERT INTO TABLE with Comment in columns") { + val tabName = "tab1" + withTable(tabName) { + sql( + s""" + |CREATE TABLE $tabName(col1 int COMMENT 'a', col2 int) + |USING parquet + """.stripMargin) + sql(s"INSERT INTO TABLE $tabName SELECT 1, 2") + + checkAnswer( + sql(s"SELECT col1, col2 FROM $tabName"), + Row(1, 2) :: Nil + ) + } + } + + test("INSERT INTO TABLE - complex type but different names") { + val tab1 = "tab1" + val tab2 = "tab2" + withTable(tab1, tab2) { + sql( + s""" + |CREATE TABLE $tab1 (s struct) + |USING parquet + """.stripMargin) + sql(s"INSERT INTO TABLE $tab1 SELECT named_struct('col1','1','col2','2')") + + sql( + s""" + |CREATE TABLE $tab2 (p struct) + |USING parquet + """.stripMargin) + sql(s"INSERT INTO TABLE $tab2 SELECT * FROM $tab1") + + checkAnswer( + spark.table(tab1), + spark.table(tab2) + ) + } + } + test("it is not allowed to write to a table while querying it.") { val message = intercept[AnalysisException] { sql( @@ -174,7 +237,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { """.stripMargin) }.getMessage assert( - message.contains("Cannot insert overwrite into table that is also being read from."), + message.contains("Cannot overwrite a path that is also being read from."), "INSERT OVERWRITE to a table while querying it should not be allowed.") } @@ -185,7 +248,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt """.stripMargin) // Cached Query Execution - caseInsensitiveContext.cacheTable("jsonTable") + spark.catalog.cacheTable("jsonTable") assertCached(sql("SELECT * FROM jsonTable")) checkAnswer( sql("SELECT * FROM jsonTable"), @@ -219,21 +282,21 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { """.stripMargin) // jsonTable should be recached. assertCached(sql("SELECT * FROM jsonTable")) - // TODO we need to invalidate the cached data in InsertIntoHadoopFsRelation -// // The cached data is the new data. -// checkAnswer( -// sql("SELECT a, b FROM jsonTable"), -// sql("SELECT a * 2, b FROM jt").collect()) -// -// // Verify uncaching -// caseInsensitiveContext.uncacheTable("jsonTable") -// assertCached(sql("SELECT * FROM jsonTable"), 0) + + // The cached data is the new data. + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + sql("SELECT a * 2, b FROM jt").collect()) + + // Verify uncaching + spark.catalog.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 + |CREATE TEMPORARY VIEW oneToTen |USING org.apache.spark.sql.sources.SimpleScanSource |OPTIONS ( | From '1', @@ -257,6 +320,110 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { "It is not allowed to insert into a table that is not an InsertableRelation." ) - caseInsensitiveContext.dropTempTable("oneToTen") + spark.catalog.dropTempView("oneToTen") + } + + test("SPARK-15824 - Execute an INSERT wrapped in a WITH statement immediately") { + withTable("target", "target2") { + sql(s"CREATE TABLE target(a INT, b STRING) USING JSON") + sql("WITH tbl AS (SELECT * FROM jt) INSERT OVERWRITE TABLE target SELECT a, b FROM tbl") + checkAnswer( + sql("SELECT a, b FROM target"), + sql("SELECT a, b FROM jt") + ) + + sql(s"CREATE TABLE target2(a INT, b STRING) USING JSON") + val e = sql( + """ + |WITH tbl AS (SELECT * FROM jt) + |FROM tbl + |INSERT INTO target2 SELECT a, b WHERE a <= 5 + |INSERT INTO target2 SELECT a, b WHERE a > 5 + """.stripMargin) + checkAnswer( + sql("SELECT a, b FROM target2"), + sql("SELECT a, b FROM jt") + ) + } + } + + test("SPARK-21203 wrong results of insertion of Array of Struct") { + val tabName = "tab1" + withTable(tabName) { + spark.sql( + """ + |CREATE TABLE `tab1` + |(`custom_fields` ARRAY>) + |USING parquet + """.stripMargin) + spark.sql( + """ + |INSERT INTO `tab1` + |SELECT ARRAY(named_struct('id', 1, 'value', 'a'), named_struct('id', 2, 'value', 'b')) + """.stripMargin) + + checkAnswer( + spark.sql("SELECT custom_fields.id, custom_fields.value FROM tab1"), + Row(Array(1, 2), Array("a", "b"))) + } + } + + test("insert overwrite directory") { + withTempDir { dir => + val path = dir.toURI.getPath + + val v1 = + s""" + | INSERT OVERWRITE DIRECTORY '${path}' + | USING json + | OPTIONS (a 1, b 0.1, c TRUE) + | SELECT 1 as a, 'c' as b + """.stripMargin + + spark.sql(v1) + + checkAnswer( + spark.read.json(dir.getCanonicalPath), + sql("SELECT 1 as a, 'c' as b")) + } + } + + test("insert overwrite directory with path in options") { + withTempDir { dir => + val path = dir.toURI.getPath + + val v1 = + s""" + | INSERT OVERWRITE DIRECTORY + | USING json + | OPTIONS ('path' '${path}') + | SELECT 1 as a, 'c' as b + """.stripMargin + + spark.sql(v1) + + checkAnswer( + spark.read.json(dir.getCanonicalPath), + sql("SELECT 1 as a, 'c' as b")) + } + } + + test("insert overwrite directory to data source not providing FileFormat") { + withTempDir { dir => + val path = dir.toURI.getPath + + val v1 = + s""" + | INSERT OVERWRITE DIRECTORY '${path}' + | USING JDBC + | OPTIONS (a 1, b 0.1, c TRUE) + | SELECT 1 as a, 'c' as b + """.stripMargin + val e = intercept[SparkException] { + spark.sql(v1) + }.getMessage + + assert(e.contains("Only Data Sources providing FileFormat are supported")) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala index c9791879ec74..0fe33e87318a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PartitionedWriteSuite.scala @@ -17,11 +17,31 @@ package org.apache.spark.sql.sources -import org.apache.spark.sql.{Row, QueryTest} +import java.io.File +import java.sql.Timestamp + +import org.apache.hadoop.mapreduce.TaskAttemptContext + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils +private class OnlyDetectCustomPathFileCommitProtocol(jobId: String, path: String) + extends SQLHadoopMapReduceCommitProtocol(jobId, path) + with Serializable with Logging { + + override def newTaskTempFileAbsPath( + taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { + throw new Exception("there should be no custom partition path") + } +} + class PartitionedWriteSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -29,11 +49,11 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext { val path = Utils.createTempDir() path.delete() - val df = sqlContext.range(100).select($"id", lit(1).as("data")) + val df = spark.range(100).select($"id", lit(1).as("data")) df.write.partitionBy("id").save(path.getCanonicalPath) checkAnswer( - sqlContext.read.load(path.getCanonicalPath), + spark.read.load(path.getCanonicalPath), (0 to 99).map(Row(1, _)).toSeq) Utils.deleteRecursively(path) @@ -43,14 +63,96 @@ class PartitionedWriteSuite extends QueryTest with SharedSQLContext { val path = Utils.createTempDir() path.delete() - val base = sqlContext.range(100) - val df = base.unionAll(base).select($"id", lit(1).as("data")) + val base = spark.range(100) + val df = base.union(base).select($"id", lit(1).as("data")) df.write.partitionBy("id").save(path.getCanonicalPath) checkAnswer( - sqlContext.read.load(path.getCanonicalPath), + spark.read.load(path.getCanonicalPath), (0 to 99).map(Row(1, _)).toSeq ++ (0 to 99).map(Row(1, _)).toSeq) Utils.deleteRecursively(path) } + + test("partitioned columns should appear at the end of schema") { + withTempPath { f => + val path = f.getAbsolutePath + Seq(1 -> "a").toDF("i", "j").write.partitionBy("i").parquet(path) + assert(spark.read.parquet(path).schema.map(_.name) == Seq("j", "i")) + } + } + + test("maxRecordsPerFile setting in non-partitioned write path") { + withTempDir { f => + spark.range(start = 0, end = 4, step = 1, numPartitions = 1) + .write.option("maxRecordsPerFile", 1).mode("overwrite").parquet(f.getAbsolutePath) + assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) + + spark.range(start = 0, end = 4, step = 1, numPartitions = 1) + .write.option("maxRecordsPerFile", 2).mode("overwrite").parquet(f.getAbsolutePath) + assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 2) + + spark.range(start = 0, end = 4, step = 1, numPartitions = 1) + .write.option("maxRecordsPerFile", -1).mode("overwrite").parquet(f.getAbsolutePath) + assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 1) + } + } + + test("maxRecordsPerFile setting in dynamic partition writes") { + withTempDir { f => + spark.range(start = 0, end = 4, step = 1, numPartitions = 1).selectExpr("id", "id id1") + .write + .partitionBy("id") + .option("maxRecordsPerFile", 1) + .mode("overwrite") + .parquet(f.getAbsolutePath) + assert(Utils.recursiveList(f).count(_.getAbsolutePath.endsWith("parquet")) == 4) + } + } + + test("append data to an existing partitioned table without custom partition path") { + withTable("t") { + withSQLConf(SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> + classOf[OnlyDetectCustomPathFileCommitProtocol].getName) { + Seq((1, 2)).toDF("a", "b").write.partitionBy("b").saveAsTable("t") + // if custom partition path is detected by the task, it will throw an Exception + // from OnlyDetectCustomPathFileCommitProtocol above. + Seq((3, 2)).toDF("a", "b").write.mode("append").partitionBy("b").saveAsTable("t") + } + } + } + + test("timeZone setting in dynamic partition writes") { + def checkPartitionValues(file: File, expected: String): Unit = { + val dir = file.getParentFile() + val value = ExternalCatalogUtils.unescapePathName( + dir.getName.substring(dir.getName.indexOf("=") + 1)) + assert(value == expected) + } + val ts = Timestamp.valueOf("2016-12-01 00:00:00") + val df = Seq((1, ts)).toDF("i", "ts") + withTempPath { f => + df.write.partitionBy("ts").parquet(f.getAbsolutePath) + val files = Utils.recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) + assert(files.length == 1) + checkPartitionValues(files.head, "2016-12-01 00:00:00") + } + withTempPath { f => + df.write.option(DateTimeUtils.TIMEZONE_OPTION, "GMT") + .partitionBy("ts").parquet(f.getAbsolutePath) + val files = Utils.recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) + assert(files.length == 1) + // use timeZone option "GMT" to format partition value. + checkPartitionValues(files.head, "2016-12-01 08:00:00") + } + withTempPath { f => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "GMT") { + df.write.partitionBy("ts").parquet(f.getAbsolutePath) + val files = Utils.recursiveList(f).filter(_.getAbsolutePath.endsWith("parquet")) + assert(files.length == 1) + // if there isn't timeZone option, then use session local timezone. + checkPartitionValues(files.head, "2016-12-01 08:00:00") + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala new file mode 100644 index 000000000000..85da3f0e3846 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PathOptionSuite.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import java.net.URI + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession, SQLContext} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.CatalogUtils +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{IntegerType, Metadata, MetadataBuilder, StructType} + +class TestOptionsSource extends SchemaRelationProvider with CreatableRelationProvider { + + // This is used in the read path. + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String], + schema: StructType): BaseRelation = { + new TestOptionsRelation(parameters)(sqlContext.sparkSession) + } + + // This is used in the write path. + override def createRelation( + sqlContext: SQLContext, + mode: SaveMode, + parameters: Map[String, String], + data: DataFrame): BaseRelation = { + new TestOptionsRelation(parameters)(sqlContext.sparkSession) + } +} + +class TestOptionsRelation(val options: Map[String, String])(@transient val session: SparkSession) + extends BaseRelation { + + override def sqlContext: SQLContext = session.sqlContext + + def pathOption: Option[String] = options.get("path") + + // We can't get the relation directly for write path, here we put the path option in schema + // metadata, so that we can test it later. + override def schema: StructType = { + val metadataWithPath = pathOption.map { path => + new MetadataBuilder().putString("path", path).build() + } + new StructType().add("i", IntegerType, true, metadataWithPath.getOrElse(Metadata.empty)) + } +} + +class PathOptionSuite extends DataSourceTest with SharedSQLContext { + + test("path option always exist") { + withTable("src") { + sql( + s""" + |CREATE TABLE src(i int) + |USING ${classOf[TestOptionsSource].getCanonicalName} + |OPTIONS (PATH '/tmp/path') + """.stripMargin) + assert(getPathOption("src").map(makeQualifiedPath) == Some(makeQualifiedPath("/tmp/path"))) + } + + // should exist even path option is not specified when creating table + withTable("src") { + sql(s"CREATE TABLE src(i int) USING ${classOf[TestOptionsSource].getCanonicalName}") + assert(getPathOption("src").map(makeQualifiedPath) == Some(defaultTablePath("src"))) + } + } + + test("path option also exist for write path") { + withTable("src") { + withTempPath { p => + sql( + s""" + |CREATE TABLE src + |USING ${classOf[TestOptionsSource].getCanonicalName} + |OPTIONS (PATH '${p.toURI}') + |AS SELECT 1 + """.stripMargin) + assert( + spark.table("src").schema.head.metadata.getString("path") == + p.toURI.toString) + } + } + + // should exist even path option is not specified when creating table + withTable("src") { + sql( + s""" + |CREATE TABLE src + |USING ${classOf[TestOptionsSource].getCanonicalName} + |AS SELECT 1 + """.stripMargin) + assert( + makeQualifiedPath(spark.table("src").schema.head.metadata.getString("path")) == + defaultTablePath("src")) + } + } + + test("path option always represent the value of table location") { + withTable("src") { + sql( + s""" + |CREATE TABLE src(i int) + |USING ${classOf[TestOptionsSource].getCanonicalName} + |OPTIONS (PATH '/tmp/path')""".stripMargin) + sql("ALTER TABLE src SET LOCATION '/tmp/path2'") + assert(getPathOption("src").map(makeQualifiedPath) == Some(makeQualifiedPath("/tmp/path2"))) + } + + withTable("src", "src2") { + sql(s"CREATE TABLE src(i int) USING ${classOf[TestOptionsSource].getCanonicalName}") + sql("ALTER TABLE src RENAME TO src2") + assert(getPathOption("src2").map(makeQualifiedPath) == Some(defaultTablePath("src2"))) + } + } + + private def getPathOption(tableName: String): Option[String] = { + spark.table(tableName).queryExecution.analyzed.collect { + case LogicalRelation(r: TestOptionsRelation, _, _, _) => r.pathOption + }.head + } + + private def defaultTablePath(tableName: String): URI = { + spark.sessionState.catalog.defaultTablePath(TableIdentifier(tableName)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index a89c5f8007e7..c1eaf948a4b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -1,19 +1,19 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 @@ -21,6 +21,7 @@ import scala.language.existentials import org.apache.spark.rdd.RDD import org.apache.spark.sql._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -28,14 +29,16 @@ class PrunedScanSource extends RelationProvider { override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - SimplePrunedScan(parameters("from").toInt, parameters("to").toInt)(sqlContext) + SimplePrunedScan(parameters("from").toInt, parameters("to").toInt)(sqlContext.sparkSession) } } -case class SimplePrunedScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) +case class SimplePrunedScan(from: Int, to: Int)(@transient val sparkSession: SparkSession) extends BaseRelation with PrunedScan { + override def sqlContext: SQLContext = sparkSession.sqlContext + override def schema: StructType = StructType( StructField("a", IntegerType, nullable = false) :: @@ -47,19 +50,19 @@ case class SimplePrunedScan(from: Int, to: Int)(@transient val sqlContext: SQLCo case "b" => (i: Int) => Seq(i * 2) } - sqlContext.sparkContext.parallelize(from to to).map(i => + sparkSession.sparkContext.parallelize(from to to).map(i => Row.fromSeq(rowBuilders.map(_(i)).reduceOption(_ ++ _).getOrElse(Seq.empty))) } } class PrunedScanSuite extends DataSourceTest with SharedSQLContext { - protected override lazy val sql = caseInsensitiveContext.sql _ + protected override lazy val sql = spark.sql _ override def beforeAll(): Unit = { super.beforeAll() sql( """ - |CREATE TEMPORARY TABLE oneToTenPruned + |CREATE TEMPORARY VIEW oneToTenPruned |USING org.apache.spark.sql.sources.PrunedScanSource |OPTIONS ( | from '1', @@ -117,28 +120,35 @@ class PrunedScanSuite extends DataSourceTest with SharedSQLContext { def testPruning(sqlString: String, expectedColumns: String*): Unit = { test(s"Columns output ${expectedColumns.mkString(",")}: $sqlString") { - val queryExecution = sql(sqlString).queryExecution - val rawPlan = queryExecution.executedPlan.collect { - case p: execution.PhysicalRDD => p - } match { - case Seq(p) => p - case _ => fail(s"More than one PhysicalRDD found\n$queryExecution") - } - val rawColumns = rawPlan.output.map(_.name) - val rawOutput = rawPlan.execute().first() - - if (rawColumns != expectedColumns) { - fail( - s"Wrong column names. Got $rawColumns, Expected $expectedColumns\n" + - s"Filters pushed: ${FiltersPushed.list.mkString(",")}\n" + - queryExecution) - } - if (rawOutput.numFields != expectedColumns.size) { - fail(s"Wrong output row. Got $rawOutput\n$queryExecution") + // These tests check a particular plan, disable whole stage codegen. + spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, false) + try { + val queryExecution = sql(sqlString).queryExecution + val rawPlan = queryExecution.executedPlan.collect { + case p: execution.DataSourceScanExec => p + } match { + case Seq(p) => p + case _ => fail(s"More than one PhysicalRDD found\n$queryExecution") + } + val rawColumns = rawPlan.output.map(_.name) + val rawOutput = rawPlan.execute().first() + + if (rawColumns != expectedColumns) { + fail( + s"Wrong column names. Got $rawColumns, Expected $expectedColumns\n" + + s"Filters pushed: ${FiltersPushed.list.mkString(",")}\n" + + queryExecution) + } + + if (rawOutput.numFields != expectedColumns.size) { + fail(s"Wrong output row. Got $rawOutput\n$queryExecution") + } + } finally { + spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.defaultValue.get) } } } - } 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 index 27d1cd92fca1..4adbff5c663b 100644 --- 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 @@ -1,60 +1,95 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.sql.sources import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.execution.datasources.ResolvedDataSource +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.test.SharedSQLContext -class ResolvedDataSourceSuite extends SparkFunSuite { +class ResolvedDataSourceSuite extends SparkFunSuite with SharedSQLContext { + private def getProvidingClass(name: String): Class[_] = + DataSource( + sparkSession = spark, + className = name, + options = Map(DateTimeUtils.TIMEZONE_OPTION -> DateTimeUtils.defaultTimeZone().getID) + ).providingClass test("jdbc") { assert( - ResolvedDataSource.lookupDataSource("jdbc") === - classOf[org.apache.spark.sql.execution.datasources.jdbc.DefaultSource]) + getProvidingClass("jdbc") === + classOf[org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider]) assert( - ResolvedDataSource.lookupDataSource("org.apache.spark.sql.execution.datasources.jdbc") === - classOf[org.apache.spark.sql.execution.datasources.jdbc.DefaultSource]) + getProvidingClass("org.apache.spark.sql.execution.datasources.jdbc") === + classOf[org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider]) assert( - ResolvedDataSource.lookupDataSource("org.apache.spark.sql.jdbc") === - classOf[org.apache.spark.sql.execution.datasources.jdbc.DefaultSource]) + getProvidingClass("org.apache.spark.sql.jdbc") === + classOf[org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider]) } test("json") { assert( - ResolvedDataSource.lookupDataSource("json") === - classOf[org.apache.spark.sql.execution.datasources.json.DefaultSource]) + getProvidingClass("json") === + classOf[org.apache.spark.sql.execution.datasources.json.JsonFileFormat]) assert( - ResolvedDataSource.lookupDataSource("org.apache.spark.sql.execution.datasources.json") === - classOf[org.apache.spark.sql.execution.datasources.json.DefaultSource]) + getProvidingClass("org.apache.spark.sql.execution.datasources.json") === + classOf[org.apache.spark.sql.execution.datasources.json.JsonFileFormat]) assert( - ResolvedDataSource.lookupDataSource("org.apache.spark.sql.json") === - classOf[org.apache.spark.sql.execution.datasources.json.DefaultSource]) + getProvidingClass("org.apache.spark.sql.json") === + classOf[org.apache.spark.sql.execution.datasources.json.JsonFileFormat]) } test("parquet") { assert( - ResolvedDataSource.lookupDataSource("parquet") === - classOf[org.apache.spark.sql.execution.datasources.parquet.DefaultSource]) + getProvidingClass("parquet") === + classOf[org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat]) assert( - ResolvedDataSource.lookupDataSource("org.apache.spark.sql.execution.datasources.parquet") === - classOf[org.apache.spark.sql.execution.datasources.parquet.DefaultSource]) + getProvidingClass("org.apache.spark.sql.execution.datasources.parquet") === + classOf[org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat]) assert( - ResolvedDataSource.lookupDataSource("org.apache.spark.sql.parquet") === - classOf[org.apache.spark.sql.execution.datasources.parquet.DefaultSource]) + getProvidingClass("org.apache.spark.sql.parquet") === + classOf[org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat]) + } + + test("csv") { + assert( + getProvidingClass("csv") === + classOf[org.apache.spark.sql.execution.datasources.csv.CSVFileFormat]) + assert( + getProvidingClass("com.databricks.spark.csv") === + classOf[org.apache.spark.sql.execution.datasources.csv.CSVFileFormat]) + } + + test("error message for unknown data sources") { + val error1 = intercept[AnalysisException] { + getProvidingClass("avro") + } + assert(error1.getMessage.contains("Failed to find data source: avro.")) + + val error2 = intercept[AnalysisException] { + getProvidingClass("com.databricks.spark.avro") + } + assert(error2.getMessage.contains("Failed to find data source: com.databricks.spark.avro.")) + + val error3 = intercept[ClassNotFoundException] { + getProvidingClass("asfdwefasdfasdf") + } + assert(error3.getMessage.contains("Failed to find data source: asfdwefasdfasdf.")) } } 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 10d261368993..773d34dfaf9a 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,32 +21,35 @@ import java.io.File import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.{AnalysisException, SaveMode, SQLConf, DataFrame} +import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.util.Utils class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndAfter { - protected override lazy val sql = caseInsensitiveContext.sql _ + import testImplicits._ + + protected override lazy val sql = spark.sql _ private var originalDefaultSource: String = null private var path: File = null private var df: DataFrame = null override def beforeAll(): Unit = { super.beforeAll() - originalDefaultSource = caseInsensitiveContext.conf.defaultDataSourceName + originalDefaultSource = spark.sessionState.conf.defaultDataSourceName path = Utils.createTempDir() path.delete() - val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) - df = caseInsensitiveContext.read.json(rdd) - df.registerTempTable("jsonTable") + val ds = (1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""").toDS() + df = spark.read.json(ds) + df.createOrReplaceTempView("jsonTable") } override def afterAll(): Unit = { try { - caseInsensitiveContext.conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, originalDefaultSource) + spark.conf.set(SQLConf.DEFAULT_DATA_SOURCE_NAME.key, originalDefaultSource) } finally { super.afterAll() } @@ -57,45 +60,42 @@ class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndA } def checkLoad(expectedDF: DataFrame = df, tbl: String = "jsonTable"): Unit = { - caseInsensitiveContext.conf.setConf( - SQLConf.DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.json") - checkAnswer(caseInsensitiveContext.read.load(path.toString), expectedDF.collect()) + spark.conf.set(SQLConf.DEFAULT_DATA_SOURCE_NAME.key, "org.apache.spark.sql.json") + checkAnswer(spark.read.load(path.toString), expectedDF.collect()) // Test if we can pick up the data source name passed in load. - caseInsensitiveContext.conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name") - checkAnswer(caseInsensitiveContext.read.format("json").load(path.toString), + spark.conf.set(SQLConf.DEFAULT_DATA_SOURCE_NAME.key, "not a source name") + checkAnswer(spark.read.format("json").load(path.toString), expectedDF.collect()) - checkAnswer(caseInsensitiveContext.read.format("json").load(path.toString), + checkAnswer(spark.read.format("json").load(path.toString), expectedDF.collect()) val schema = StructType(StructField("b", StringType, true) :: Nil) checkAnswer( - caseInsensitiveContext.read.format("json").schema(schema).load(path.toString), + spark.read.format("json").schema(schema).load(path.toString), sql(s"SELECT b FROM $tbl").collect()) } test("save with path and load") { - caseInsensitiveContext.conf.setConf( - SQLConf.DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.json") + spark.conf.set(SQLConf.DEFAULT_DATA_SOURCE_NAME.key, "org.apache.spark.sql.json") df.write.save(path.toString) checkLoad() } test("save with string mode and path, and load") { - caseInsensitiveContext.conf.setConf( - SQLConf.DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.json") + spark.conf.set(SQLConf.DEFAULT_DATA_SOURCE_NAME.key, "org.apache.spark.sql.json") path.createNewFile() df.write.mode("overwrite").save(path.toString) checkLoad() } test("save with path and datasource, and load") { - caseInsensitiveContext.conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name") + spark.conf.set(SQLConf.DEFAULT_DATA_SOURCE_NAME.key, "not a source name") df.write.json(path.toString) checkLoad() } test("save with data source and options, and load") { - caseInsensitiveContext.conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name") + spark.conf.set(SQLConf.DEFAULT_DATA_SOURCE_NAME.key, "not a source name") df.write.mode(SaveMode.ErrorIfExists).json(path.toString) checkLoad() } @@ -121,8 +121,8 @@ class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndA // verify the append mode df.write.mode(SaveMode.Append).json(path.toString) - val df2 = df.unionAll(df) - df2.registerTempTable("jsonTable2") + val df2 = df.union(df) + df2.createOrReplaceTempView("jsonTable2") checkLoad(df2, "jsonTable2") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 12af8068c398..17690e3df915 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -1,19 +1,19 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 @@ -22,6 +22,7 @@ import java.sql.{Date, Timestamp} import org.apache.spark.rdd.RDD import org.apache.spark.sql._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -31,17 +32,21 @@ class SimpleScanSource extends RelationProvider { override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - SimpleScan(parameters("from").toInt, parameters("TO").toInt)(sqlContext) + SimpleScan(parameters("from").toInt, parameters("TO").toInt)(sqlContext.sparkSession) } } -case class SimpleScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) +case class SimpleScan(from: Int, to: Int)(@transient val sparkSession: SparkSession) extends BaseRelation with TableScan { + override def sqlContext: SQLContext = sparkSession.sqlContext + override def schema: StructType = StructType(StructField("i", IntegerType, nullable = false) :: Nil) - override def buildScan(): RDD[Row] = sqlContext.sparkContext.parallelize(from to to).map(Row(_)) + override def buildScan(): RDD[Row] = { + sparkSession.sparkContext.parallelize(from to to).map(Row(_)) + } } class AllDataTypesScanSource extends SchemaRelationProvider { @@ -53,23 +58,27 @@ class AllDataTypesScanSource extends SchemaRelationProvider { parameters("option_with_underscores") parameters("option.with.dots") - AllDataTypesScan(parameters("from").toInt, parameters("TO").toInt, schema)(sqlContext) + AllDataTypesScan( + parameters("from").toInt, + parameters("TO").toInt, schema)(sqlContext.sparkSession) } } case class AllDataTypesScan( from: Int, to: Int, - userSpecifiedSchema: StructType)(@transient val sqlContext: SQLContext) + userSpecifiedSchema: StructType)(@transient val sparkSession: SparkSession) extends BaseRelation with TableScan { + override def sqlContext: SQLContext = sparkSession.sqlContext + override def schema: StructType = userSpecifiedSchema override def needConversion: Boolean = true override def buildScan(): RDD[Row] = { - sqlContext.sparkContext.parallelize(from to to).map { i => + sparkSession.sparkContext.parallelize(from to to).map { i => Row( s"str_$i", s"str_$i".getBytes(StandardCharsets.UTF_8), @@ -85,6 +94,7 @@ case class AllDataTypesScan( Date.valueOf("1970-01-01"), new Timestamp(20000 + i), s"varchar_$i", + s"char_$i", Seq(i, i + 1), Seq(Map(s"str_$i" -> Row(i.toLong))), Map(i -> i.toString), @@ -97,7 +107,7 @@ case class AllDataTypesScan( } class TableScanSuite extends DataSourceTest with SharedSQLContext { - protected override lazy val sql = caseInsensitiveContext.sql _ + protected override lazy val sql = spark.sql _ private lazy val tableWithSchemaExpected = (1 to 10).map { i => Row( @@ -115,6 +125,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { Date.valueOf("1970-01-01"), new Timestamp(20000 + i), s"varchar_$i", + s"char_$i", Seq(i, i + 1), Seq(Map(s"str_$i" -> Row(i.toLong))), Map(i -> i.toString), @@ -127,7 +138,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { super.beforeAll() sql( """ - |CREATE TEMPORARY TABLE oneToTen + |CREATE TEMPORARY VIEW oneToTen |USING org.apache.spark.sql.sources.SimpleScanSource |OPTIONS ( | From '1', @@ -139,7 +150,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { sql( """ - |CREATE TEMPORARY TABLE tableWithSchema ( + |CREATE TEMPORARY VIEW tableWithSchema ( |`string$%Field` stRIng, |binaryField binary, |`booleanField` boolean, @@ -154,6 +165,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { |dateField dAte, |timestampField tiMestamp, |varcharField varchaR(12), + |charField ChaR(18), |arrayFieldSimple Array, |arrayFieldComplex Array>>, |mapFieldSimple MAP, @@ -192,6 +204,10 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { (2 to 10).map(i => Row(i, i - 1)).toSeq) test("Schema and all fields") { + def hiveMetadata(dt: String): Metadata = { + new MetadataBuilder().putString(HIVE_TYPE_STRING, dt).build() + } + val expectedSchema = StructType( StructField("string$%Field", StringType, true) :: StructField("binaryField", BinaryType, true) :: @@ -206,7 +222,8 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { StructField("decimalField2", DecimalType(9, 2), true) :: StructField("dateField", DateType, true) :: StructField("timestampField", TimestampType, true) :: - StructField("varcharField", StringType, true) :: + StructField("varcharField", StringType, true, hiveMetadata("varchar(12)")) :: + StructField("charField", StringType, true, hiveMetadata("char(18)")) :: StructField("arrayFieldSimple", ArrayType(IntegerType), true) :: StructField("arrayFieldComplex", ArrayType( @@ -229,33 +246,36 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { Nil ) - assert(expectedSchema == caseInsensitiveContext.table("tableWithSchema").schema) - - checkAnswer( - sql( - """SELECT - | `string$%Field`, - | cast(binaryField as string), - | booleanField, - | byteField, - | shortField, - | int_Field, - | `longField_:,<>=+/~^`, - | floatField, - | doubleField, - | decimalField1, - | decimalField2, - | dateField, - | timestampField, - | varcharField, - | arrayFieldSimple, - | arrayFieldComplex, - | mapFieldSimple, - | mapFieldComplex, - | structFieldSimple, - | structFieldComplex FROM tableWithSchema""".stripMargin), - tableWithSchemaExpected - ) + assert(expectedSchema == spark.table("tableWithSchema").schema) + + withSQLConf(SQLConf.SUPPORT_QUOTED_REGEX_COLUMN_NAME.key -> "false") { + checkAnswer( + sql( + """SELECT + | `string$%Field`, + | cast(binaryField as string), + | booleanField, + | byteField, + | shortField, + | int_Field, + | `longField_:,<>=+/~^`, + | floatField, + | doubleField, + | decimalField1, + | decimalField2, + | dateField, + | timestampField, + | varcharField, + | charField, + | arrayFieldSimple, + | arrayFieldComplex, + | mapFieldSimple, + | mapFieldComplex, + | structFieldSimple, + | structFieldComplex FROM tableWithSchema""".stripMargin), + tableWithSchemaExpected + ) + } } sqlTest( @@ -284,7 +304,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { test("Caching") { // Cached Query Execution - caseInsensitiveContext.cacheTable("oneToTen") + spark.catalog.cacheTable("oneToTen") assertCached(sql("SELECT * FROM oneToTen")) checkAnswer( sql("SELECT * FROM oneToTen"), @@ -312,14 +332,14 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { (2 to 10).map(i => Row(i, i - 1)).toSeq) // Verify uncaching - caseInsensitiveContext.uncacheTable("oneToTen") + spark.catalog.uncacheTable("oneToTen") assertCached(sql("SELECT * FROM oneToTen"), 0) } test("defaultSource") { sql( """ - |CREATE TEMPORARY TABLE oneToTenDef + |CREATE TEMPORARY VIEW oneToTenDef |USING org.apache.spark.sql.sources |OPTIONS ( | from '1', @@ -334,38 +354,58 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { test("exceptions") { // Make sure we do throw correct exception when users use a relation provider that - // only implements the RelationProvier or the SchemaRelationProvider. - val schemaNotAllowed = intercept[Exception] { - sql( - """ - |CREATE TEMPORARY TABLE relationProvierWithSchema (i int) - |USING org.apache.spark.sql.sources.SimpleScanSource - |OPTIONS ( - | From '1', - | To '10' - |) - """.stripMargin) + // only implements the RelationProvider or the SchemaRelationProvider. + Seq("TEMPORARY VIEW", "TABLE").foreach { tableType => + val schemaNotAllowed = intercept[Exception] { + sql( + s""" + |CREATE $tableType relationProvierWithSchema (i int) + |USING org.apache.spark.sql.sources.SimpleScanSource + |OPTIONS ( + | From '1', + | To '10' + |) + """.stripMargin) + } + assert(schemaNotAllowed.getMessage.contains("does not allow user-specified schemas")) + + val schemaNeeded = intercept[Exception] { + sql( + s""" + |CREATE $tableType schemaRelationProvierWithoutSchema + |USING org.apache.spark.sql.sources.AllDataTypesScanSource + |OPTIONS ( + | From '1', + | To '10' + |) + """.stripMargin) + } + assert(schemaNeeded.getMessage.contains("A schema needs to be specified when using")) } - assert(schemaNotAllowed.getMessage.contains("does not allow user-specified schemas")) - - val schemaNeeded = intercept[Exception] { - sql( - """ - |CREATE TEMPORARY TABLE schemaRelationProvierWithoutSchema - |USING org.apache.spark.sql.sources.AllDataTypesScanSource - |OPTIONS ( - | From '1', - | To '10' - |) - """.stripMargin) + } + + test("read the data source tables that do not extend SchemaRelationProvider") { + Seq("TEMPORARY VIEW", "TABLE").foreach { tableType => + val tableName = "relationProvierWithSchema" + withTable (tableName) { + sql( + s""" + |CREATE $tableType $tableName + |USING org.apache.spark.sql.sources.SimpleScanSource + |OPTIONS ( + | From '1', + | To '10' + |) + """.stripMargin) + checkAnswer(spark.table(tableName), spark.range(1, 11).toDF()) + } } - assert(schemaNeeded.getMessage.contains("A schema needs to be specified when using")) } test("SPARK-5196 schema field with comment") { sql( """ - |CREATE TEMPORARY TABLE student(name string comment "SN", age int comment "SA", grade int) + |CREATE TEMPORARY VIEW student(name string comment "SN", age int comment "SA", grade int) |USING org.apache.spark.sql.sources.AllDataTypesScanSource |OPTIONS ( | from '1', @@ -375,12 +415,8 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { |) """.stripMargin) - val planned = sql("SELECT * FROM student").queryExecution.executedPlan - val comments = planned.schema.fields.map { field => - if (field.metadata.contains("comment")) field.metadata.getString("comment") - else "NO_COMMENT" - }.mkString(",") - + val planned = sql("SELECT * FROM student").queryExecution.executedPlan + val comments = planned.schema.fields.map(_.getComment().getOrElse("NO_COMMENT")).mkString(",") assert(comments === "SN,SA,NO_COMMENT") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/fakeExternalSources.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/fakeExternalSources.scala new file mode 100644 index 000000000000..bf43de597a7a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/fakeExternalSources.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.fakesource + +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister, RelationProvider} +import org.apache.spark.sql.types._ + + +// Note that the package name is intendedly mismatched in order to resemble external data sources +// and test the detection for them. +class FakeExternalSourceOne extends RelationProvider with DataSourceRegister { + + def shortName(): String = "Fake external source" + + override def createRelation(cont: SQLContext, param: Map[String, String]): BaseRelation = + new BaseRelation { + override def sqlContext: SQLContext = cont + + override def schema: StructType = + StructType(Seq(StructField("stringType", StringType, nullable = false))) + } +} + +class FakeExternalSourceTwo extends RelationProvider with DataSourceRegister { + + def shortName(): String = "Fake external source" + + override def createRelation(cont: SQLContext, param: Map[String, String]): BaseRelation = + new BaseRelation { + override def sqlContext: SQLContext = cont + + override def schema: StructType = + StructType(Seq(StructField("integerType", IntegerType, nullable = false))) + } +} + +class FakeExternalSourceThree extends RelationProvider with DataSourceRegister { + + def shortName(): String = "datasource" + + override def createRelation(cont: SQLContext, param: Map[String, String]): BaseRelation = + new BaseRelation { + override def sqlContext: SQLContext = cont + + override def schema: StructType = + StructType(Seq(StructField("byteType", ByteType, nullable = false))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2OptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2OptionsSuite.scala new file mode 100644 index 000000000000..933f4075bcc8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2OptionsSuite.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2 + +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite + +/** + * A simple test suite to verify `DataSourceV2Options`. + */ +class DataSourceV2OptionsSuite extends SparkFunSuite { + + test("key is case-insensitive") { + val options = new DataSourceV2Options(Map("foo" -> "bar").asJava) + assert(options.get("foo").get() == "bar") + assert(options.get("FoO").get() == "bar") + assert(!options.get("abc").isPresent) + } + + test("value is case-sensitive") { + val options = new DataSourceV2Options(Map("foo" -> "bAr").asJava) + assert(options.get("foo").get == "bAr") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala new file mode 100644 index 000000000000..9ce93d7ae926 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.v2 + +import java.util.{ArrayList, List => JList} + +import test.org.apache.spark.sql.sources.v2._ + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.sources.{Filter, GreaterThan} +import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType + +class DataSourceV2Suite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("simplest implementation") { + Seq(classOf[SimpleDataSourceV2], classOf[JavaSimpleDataSourceV2]).foreach { cls => + withClue(cls.getName) { + val df = spark.read.format(cls.getName).load() + checkAnswer(df, (0 until 10).map(i => Row(i, -i))) + checkAnswer(df.select('j), (0 until 10).map(i => Row(-i))) + checkAnswer(df.filter('i > 5), (6 until 10).map(i => Row(i, -i))) + } + } + } + + test("advanced implementation") { + Seq(classOf[AdvancedDataSourceV2], classOf[JavaAdvancedDataSourceV2]).foreach { cls => + withClue(cls.getName) { + val df = spark.read.format(cls.getName).load() + checkAnswer(df, (0 until 10).map(i => Row(i, -i))) + checkAnswer(df.select('j), (0 until 10).map(i => Row(-i))) + checkAnswer(df.filter('i > 3), (4 until 10).map(i => Row(i, -i))) + checkAnswer(df.select('j).filter('i > 6), (7 until 10).map(i => Row(-i))) + checkAnswer(df.select('i).filter('i > 10), Nil) + } + } + } + + test("unsafe row implementation") { + Seq(classOf[UnsafeRowDataSourceV2], classOf[JavaUnsafeRowDataSourceV2]).foreach { cls => + withClue(cls.getName) { + val df = spark.read.format(cls.getName).load() + checkAnswer(df, (0 until 10).map(i => Row(i, -i))) + checkAnswer(df.select('j), (0 until 10).map(i => Row(-i))) + checkAnswer(df.filter('i > 5), (6 until 10).map(i => Row(i, -i))) + } + } + } + + test("schema required data source") { + Seq(classOf[SchemaRequiredDataSource], classOf[JavaSchemaRequiredDataSource]).foreach { cls => + withClue(cls.getName) { + val e = intercept[AnalysisException](spark.read.format(cls.getName).load()) + assert(e.message.contains("A schema needs to be specified")) + + val schema = new StructType().add("i", "int").add("s", "string") + val df = spark.read.format(cls.getName).schema(schema).load() + + assert(df.schema == schema) + assert(df.collect().isEmpty) + } + } + } +} + +class SimpleDataSourceV2 extends DataSourceV2 with ReadSupport { + + class Reader extends DataSourceV2Reader { + override def readSchema(): StructType = new StructType().add("i", "int").add("j", "int") + + override def createReadTasks(): JList[ReadTask[Row]] = { + java.util.Arrays.asList(new SimpleReadTask(0, 5), new SimpleReadTask(5, 10)) + } + } + + override def createReader(options: DataSourceV2Options): DataSourceV2Reader = new Reader +} + +class SimpleReadTask(start: Int, end: Int) extends ReadTask[Row] with DataReader[Row] { + private var current = start - 1 + + override def createReader(): DataReader[Row] = new SimpleReadTask(start, end) + + override def next(): Boolean = { + current += 1 + current < end + } + + override def get(): Row = Row(current, -current) + + override def close(): Unit = {} +} + + + +class AdvancedDataSourceV2 extends DataSourceV2 with ReadSupport { + + class Reader extends DataSourceV2Reader + with SupportsPushDownRequiredColumns with SupportsPushDownFilters { + + var requiredSchema = new StructType().add("i", "int").add("j", "int") + var filters = Array.empty[Filter] + + override def pruneColumns(requiredSchema: StructType): Unit = { + this.requiredSchema = requiredSchema + } + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + this.filters = filters + Array.empty + } + + override def readSchema(): StructType = { + requiredSchema + } + + override def createReadTasks(): JList[ReadTask[Row]] = { + val lowerBound = filters.collect { + case GreaterThan("i", v: Int) => v + }.headOption + + val res = new ArrayList[ReadTask[Row]] + + if (lowerBound.isEmpty) { + res.add(new AdvancedReadTask(0, 5, requiredSchema)) + res.add(new AdvancedReadTask(5, 10, requiredSchema)) + } else if (lowerBound.get < 4) { + res.add(new AdvancedReadTask(lowerBound.get + 1, 5, requiredSchema)) + res.add(new AdvancedReadTask(5, 10, requiredSchema)) + } else if (lowerBound.get < 9) { + res.add(new AdvancedReadTask(lowerBound.get + 1, 10, requiredSchema)) + } + + res + } + } + + override def createReader(options: DataSourceV2Options): DataSourceV2Reader = new Reader +} + +class AdvancedReadTask(start: Int, end: Int, requiredSchema: StructType) + extends ReadTask[Row] with DataReader[Row] { + + private var current = start - 1 + + override def createReader(): DataReader[Row] = new AdvancedReadTask(start, end, requiredSchema) + + override def close(): Unit = {} + + override def next(): Boolean = { + current += 1 + current < end + } + + override def get(): Row = { + val values = requiredSchema.map(_.name).map { + case "i" => current + case "j" => -current + } + Row.fromSeq(values) + } +} + + +class UnsafeRowDataSourceV2 extends DataSourceV2 with ReadSupport { + + class Reader extends DataSourceV2Reader with SupportsScanUnsafeRow { + override def readSchema(): StructType = new StructType().add("i", "int").add("j", "int") + + override def createUnsafeRowReadTasks(): JList[ReadTask[UnsafeRow]] = { + java.util.Arrays.asList(new UnsafeRowReadTask(0, 5), new UnsafeRowReadTask(5, 10)) + } + } + + override def createReader(options: DataSourceV2Options): DataSourceV2Reader = new Reader +} + +class UnsafeRowReadTask(start: Int, end: Int) + extends ReadTask[UnsafeRow] with DataReader[UnsafeRow] { + + private val row = new UnsafeRow(2) + row.pointTo(new Array[Byte](8 * 3), 8 * 3) + + private var current = start - 1 + + override def createReader(): DataReader[UnsafeRow] = new UnsafeRowReadTask(start, end) + + override def next(): Boolean = { + current += 1 + current < end + } + override def get(): UnsafeRow = { + row.setInt(0, current) + row.setInt(1, -current) + row + } + + override def close(): Unit = {} +} + +class SchemaRequiredDataSource extends DataSourceV2 with ReadSupportWithSchema { + + class Reader(val readSchema: StructType) extends DataSourceV2Reader { + override def createReadTasks(): JList[ReadTask[Row]] = + java.util.Collections.emptyList() + } + + override def createReader(schema: StructType, options: DataSourceV2Options): DataSourceV2Reader = + new Reader(schema) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DeduplicateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DeduplicateSuite.scala new file mode 100644 index 000000000000..e858b7d9998a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DeduplicateSuite.scala @@ -0,0 +1,284 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.state.StateStore +import org.apache.spark.sql.functions._ + +class DeduplicateSuite extends StateStoreMetricsTest with BeforeAndAfterAll { + + import testImplicits._ + + override def afterAll(): Unit = { + super.afterAll() + StateStore.stop() + } + + test("deduplicate with all columns") { + val inputData = MemoryStream[String] + val result = inputData.toDS().dropDuplicates() + + testStream(result, Append)( + AddData(inputData, "a"), + CheckLastBatch("a"), + assertNumStateRows(total = 1, updated = 1), + AddData(inputData, "a"), + CheckLastBatch(), + assertNumStateRows(total = 1, updated = 0), + AddData(inputData, "b"), + CheckLastBatch("b"), + assertNumStateRows(total = 2, updated = 1) + ) + } + + test("deduplicate with some columns") { + val inputData = MemoryStream[(String, Int)] + val result = inputData.toDS().dropDuplicates("_1") + + testStream(result, Append)( + AddData(inputData, "a" -> 1), + CheckLastBatch("a" -> 1), + assertNumStateRows(total = 1, updated = 1), + AddData(inputData, "a" -> 2), // Dropped + CheckLastBatch(), + assertNumStateRows(total = 1, updated = 0), + AddData(inputData, "b" -> 1), + CheckLastBatch("b" -> 1), + assertNumStateRows(total = 2, updated = 1) + ) + } + + test("multiple deduplicates") { + val inputData = MemoryStream[(String, Int)] + val result = inputData.toDS().dropDuplicates().dropDuplicates("_1") + + testStream(result, Append)( + AddData(inputData, "a" -> 1), + CheckLastBatch("a" -> 1), + assertNumStateRows(total = Seq(1L, 1L), updated = Seq(1L, 1L)), + + AddData(inputData, "a" -> 2), // Dropped from the second `dropDuplicates` + CheckLastBatch(), + assertNumStateRows(total = Seq(1L, 2L), updated = Seq(0L, 1L)), + + AddData(inputData, "b" -> 1), + CheckLastBatch("b" -> 1), + assertNumStateRows(total = Seq(2L, 3L), updated = Seq(1L, 1L)) + ) + } + + test("deduplicate with watermark") { + val inputData = MemoryStream[Int] + val result = inputData.toDS() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .dropDuplicates() + .select($"eventTime".cast("long").as[Long]) + + testStream(result, Append)( + AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*), + CheckLastBatch(10 to 15: _*), + assertNumStateRows(total = 6, updated = 6), + + AddData(inputData, 25), // Advance watermark to 15 seconds + CheckLastBatch(25), + assertNumStateRows(total = 7, updated = 1), + + AddData(inputData, 25), // Drop states less than watermark + CheckLastBatch(), + assertNumStateRows(total = 1, updated = 0), + + AddData(inputData, 10), // Should not emit anything as data less than watermark + CheckLastBatch(), + assertNumStateRows(total = 1, updated = 0), + + AddData(inputData, 45), // Advance watermark to 35 seconds + CheckLastBatch(45), + assertNumStateRows(total = 2, updated = 1), + + AddData(inputData, 45), // Drop states less than watermark + CheckLastBatch(), + assertNumStateRows(total = 1, updated = 0) + ) + } + + test("deduplicate with aggregate - append mode") { + val inputData = MemoryStream[Int] + val windowedaggregate = inputData.toDS() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .dropDuplicates() + .withWatermark("eventTime", "10 seconds") + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) + + testStream(windowedaggregate)( + AddData(inputData, (1 to 5).flatMap(_ => (10 to 15)): _*), + CheckLastBatch(), + // states in aggregate in [10, 14), [15, 20) (2 windows) + // states in deduplicate is 10 to 15 + assertNumStateRows(total = Seq(2L, 6L), updated = Seq(2L, 6L)), + + AddData(inputData, 25), // Advance watermark to 15 seconds + CheckLastBatch(), + // states in aggregate in [10, 14), [15, 20) and [25, 30) (3 windows) + // states in deduplicate is 10 to 15 and 25 + assertNumStateRows(total = Seq(3L, 7L), updated = Seq(1L, 1L)), + + AddData(inputData, 25), // Emit items less than watermark and drop their state + CheckLastBatch((10 -> 5)), // 5 items (10 to 14) after deduplicate + // states in aggregate in [15, 20) and [25, 30) (2 windows, note aggregate uses the end of + // window to evict items, so [15, 20) is still in the state store) + // states in deduplicate is 25 + assertNumStateRows(total = Seq(2L, 1L), updated = Seq(0L, 0L)), + + AddData(inputData, 10), // Should not emit anything as data less than watermark + CheckLastBatch(), + assertNumStateRows(total = Seq(2L, 1L), updated = Seq(0L, 0L)), + + AddData(inputData, 40), // Advance watermark to 30 seconds + CheckLastBatch(), + // states in aggregate in [15, 20), [25, 30) and [40, 45) + // states in deduplicate is 25 and 40, + assertNumStateRows(total = Seq(3L, 2L), updated = Seq(1L, 1L)), + + AddData(inputData, 40), // Emit items less than watermark and drop their state + CheckLastBatch((15 -> 1), (25 -> 1)), + // states in aggregate in [40, 45) + // states in deduplicate is 40, + assertNumStateRows(total = Seq(1L, 1L), updated = Seq(0L, 0L)) + ) + } + + test("deduplicate with aggregate - update mode") { + val inputData = MemoryStream[(String, Int)] + val result = inputData.toDS() + .select($"_1" as "str", $"_2" as "num") + .dropDuplicates() + .groupBy("str") + .agg(sum("num")) + .as[(String, Long)] + + testStream(result, Update)( + AddData(inputData, "a" -> 1), + CheckLastBatch("a" -> 1L), + assertNumStateRows(total = Seq(1L, 1L), updated = Seq(1L, 1L)), + AddData(inputData, "a" -> 1), // Dropped + CheckLastBatch(), + assertNumStateRows(total = Seq(1L, 1L), updated = Seq(0L, 0L)), + AddData(inputData, "a" -> 2), + CheckLastBatch("a" -> 3L), + assertNumStateRows(total = Seq(1L, 2L), updated = Seq(1L, 1L)), + AddData(inputData, "b" -> 1), + CheckLastBatch("b" -> 1L), + assertNumStateRows(total = Seq(2L, 3L), updated = Seq(1L, 1L)) + ) + } + + test("deduplicate with aggregate - complete mode") { + val inputData = MemoryStream[(String, Int)] + val result = inputData.toDS() + .select($"_1" as "str", $"_2" as "num") + .dropDuplicates() + .groupBy("str") + .agg(sum("num")) + .as[(String, Long)] + + testStream(result, Complete)( + AddData(inputData, "a" -> 1), + CheckLastBatch("a" -> 1L), + assertNumStateRows(total = Seq(1L, 1L), updated = Seq(1L, 1L)), + AddData(inputData, "a" -> 1), // Dropped + CheckLastBatch("a" -> 1L), + assertNumStateRows(total = Seq(1L, 1L), updated = Seq(0L, 0L)), + AddData(inputData, "a" -> 2), + CheckLastBatch("a" -> 3L), + assertNumStateRows(total = Seq(1L, 2L), updated = Seq(1L, 1L)), + AddData(inputData, "b" -> 1), + CheckLastBatch("a" -> 3L, "b" -> 1L), + assertNumStateRows(total = Seq(2L, 3L), updated = Seq(1L, 1L)) + ) + } + + test("deduplicate with file sink") { + withTempDir { output => + withTempDir { checkpointDir => + val outputPath = output.getAbsolutePath + val inputData = MemoryStream[String] + val result = inputData.toDS().dropDuplicates() + val q = result.writeStream + .format("parquet") + .outputMode(Append) + .option("checkpointLocation", checkpointDir.getPath) + .start(outputPath) + try { + inputData.addData("a") + q.processAllAvailable() + checkDataset(spark.read.parquet(outputPath).as[String], "a") + + inputData.addData("a") // Dropped + q.processAllAvailable() + checkDataset(spark.read.parquet(outputPath).as[String], "a") + + inputData.addData("b") + q.processAllAvailable() + checkDataset(spark.read.parquet(outputPath).as[String], "a", "b") + } finally { + q.stop() + } + } + } + } + + test("SPARK-19841: watermarkPredicate should filter based on keys") { + val input = MemoryStream[(Int, Int)] + val df = input.toDS.toDF("time", "id") + .withColumn("time", $"time".cast("timestamp")) + .withWatermark("time", "1 second") + .dropDuplicates("id", "time") // Change the column positions + .select($"id") + testStream(df)( + AddData(input, 1 -> 1, 1 -> 1, 1 -> 2), + CheckLastBatch(1, 2), + AddData(input, 1 -> 1, 2 -> 3, 2 -> 4), + CheckLastBatch(3, 4), + AddData(input, 1 -> 0, 1 -> 1, 3 -> 5, 3 -> 6), // Drop (1 -> 0, 1 -> 1) due to watermark + CheckLastBatch(5, 6), + AddData(input, 1 -> 0, 4 -> 7), // Drop (1 -> 0) due to watermark + CheckLastBatch(7) + ) + } + + test("SPARK-21546: dropDuplicates should ignore watermark when it's not a key") { + val input = MemoryStream[(Int, Int)] + val df = input.toDS.toDF("id", "time") + .withColumn("time", $"time".cast("timestamp")) + .withWatermark("time", "1 second") + .dropDuplicates("id") + .select($"id", $"time".cast("long")) + testStream(df)( + AddData(input, 1 -> 1, 1 -> 2, 2 -> 2), + CheckLastBatch(1 -> 1, 2 -> 2) + ) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EnsureStatefulOpPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EnsureStatefulOpPartitioningSuite.scala new file mode 100644 index 000000000000..044bb03480aa --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EnsureStatefulOpPartitioningSuite.scala @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.util.UUID + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest, UnaryExecNode} +import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchange} +import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata, StatefulOperator, StatefulOperatorStateInfo} +import org.apache.spark.sql.test.SharedSQLContext + +class EnsureStatefulOpPartitioningSuite extends SparkPlanTest with SharedSQLContext { + + import testImplicits._ + + private var baseDf: DataFrame = null + + override def beforeAll(): Unit = { + super.beforeAll() + baseDf = Seq((1, "A"), (2, "b")).toDF("num", "char") + } + + test("ClusteredDistribution generates Exchange with HashPartitioning") { + testEnsureStatefulOpPartitioning( + baseDf.queryExecution.sparkPlan, + requiredDistribution = keys => ClusteredDistribution(keys), + expectedPartitioning = + keys => HashPartitioning(keys, spark.sessionState.conf.numShufflePartitions), + expectShuffle = true) + } + + test("ClusteredDistribution with coalesce(1) generates Exchange with HashPartitioning") { + testEnsureStatefulOpPartitioning( + baseDf.coalesce(1).queryExecution.sparkPlan, + requiredDistribution = keys => ClusteredDistribution(keys), + expectedPartitioning = + keys => HashPartitioning(keys, spark.sessionState.conf.numShufflePartitions), + expectShuffle = true) + } + + test("AllTuples generates Exchange with SinglePartition") { + testEnsureStatefulOpPartitioning( + baseDf.queryExecution.sparkPlan, + requiredDistribution = _ => AllTuples, + expectedPartitioning = _ => SinglePartition, + expectShuffle = true) + } + + test("AllTuples with coalesce(1) doesn't need Exchange") { + testEnsureStatefulOpPartitioning( + baseDf.coalesce(1).queryExecution.sparkPlan, + requiredDistribution = _ => AllTuples, + expectedPartitioning = _ => SinglePartition, + expectShuffle = false) + } + + /** + * For `StatefulOperator` with the given `requiredChildDistribution`, and child SparkPlan + * `inputPlan`, ensures that the incremental planner adds exchanges, if required, in order to + * ensure the expected partitioning. + */ + private def testEnsureStatefulOpPartitioning( + inputPlan: SparkPlan, + requiredDistribution: Seq[Attribute] => Distribution, + expectedPartitioning: Seq[Attribute] => Partitioning, + expectShuffle: Boolean): Unit = { + val operator = TestStatefulOperator(inputPlan, requiredDistribution(inputPlan.output.take(1))) + val executed = executePlan(operator, OutputMode.Complete()) + if (expectShuffle) { + val exchange = executed.children.find(_.isInstanceOf[Exchange]) + if (exchange.isEmpty) { + fail(s"Was expecting an exchange but didn't get one in:\n$executed") + } + assert(exchange.get === + ShuffleExchange(expectedPartitioning(inputPlan.output.take(1)), inputPlan), + s"Exchange didn't have expected properties:\n${exchange.get}") + } else { + assert(!executed.children.exists(_.isInstanceOf[Exchange]), + s"Unexpected exchange found in:\n$executed") + } + } + + /** Executes a SparkPlan using the IncrementalPlanner used for Structured Streaming. */ + private def executePlan( + p: SparkPlan, + outputMode: OutputMode = OutputMode.Append()): SparkPlan = { + val execution = new IncrementalExecution( + spark, + null, + OutputMode.Complete(), + "chk", + UUID.randomUUID(), + 0L, + OffsetSeqMetadata()) { + override lazy val sparkPlan: SparkPlan = p transform { + case plan: SparkPlan => + val inputMap = plan.children.flatMap(_.output).map(a => (a.name, a)).toMap + plan transformExpressions { + case UnresolvedAttribute(Seq(u)) => + inputMap.getOrElse(u, + sys.error(s"Invalid Test: Cannot resolve $u given input $inputMap")) + } + } + } + execution.executedPlan + } +} + +/** Used to emulate a `StatefulOperator` with the given requiredDistribution. */ +case class TestStatefulOperator( + child: SparkPlan, + requiredDist: Distribution) extends UnaryExecNode with StatefulOperator { + override def output: Seq[Attribute] = child.output + override def doExecute(): RDD[InternalRow] = child.execute() + override def requiredChildDistribution: Seq[Distribution] = requiredDist :: Nil + override def stateInfo: Option[StatefulOperatorStateInfo] = None +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala new file mode 100644 index 000000000000..f3e8cf950a5a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -0,0 +1,519 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming + +import java.{util => ju} +import java.text.SimpleDateFormat +import java.util.Date + +import org.scalatest.{BeforeAndAfter, Matchers} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.functions.{count, window} +import org.apache.spark.sql.streaming.OutputMode._ + +class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matchers with Logging { + + import testImplicits._ + + after { + sqlContext.streams.active.foreach(_.stop()) + } + + test("EventTimeStats") { + val epsilon = 10E-6 + + val stats = EventTimeStats(max = 100, min = 10, avg = 20.0, count = 5) + stats.add(80L) + stats.max should be (100) + stats.min should be (10) + stats.avg should be (30.0 +- epsilon) + stats.count should be (6) + + val stats2 = EventTimeStats(80L, 5L, 15.0, 4) + stats.merge(stats2) + stats.max should be (100) + stats.min should be (5) + stats.avg should be (24.0 +- epsilon) + stats.count should be (10) + } + + test("EventTimeStats: avg on large values") { + val epsilon = 10E-6 + val largeValue = 10000000000L // 10B + // Make sure `largeValue` will cause overflow if we use a Long sum to calc avg. + assert(largeValue * largeValue != BigInt(largeValue) * BigInt(largeValue)) + val stats = + EventTimeStats(max = largeValue, min = largeValue, avg = largeValue, count = largeValue - 1) + stats.add(largeValue) + stats.avg should be (largeValue.toDouble +- epsilon) + + val stats2 = EventTimeStats( + max = largeValue + 1, + min = largeValue, + avg = largeValue + 1, + count = largeValue) + stats.merge(stats2) + stats.avg should be ((largeValue + 0.5) +- epsilon) + } + + test("error on bad column") { + val inputData = MemoryStream[Int].toDF() + val e = intercept[AnalysisException] { + inputData.withWatermark("badColumn", "1 minute") + } + assert(e.getMessage contains "badColumn") + } + + test("error on wrong type") { + val inputData = MemoryStream[Int].toDF() + val e = intercept[AnalysisException] { + inputData.withWatermark("value", "1 minute") + } + assert(e.getMessage contains "value") + assert(e.getMessage contains "int") + } + + test("event time and watermark metrics") { + // No event time metrics when there is no watermarking + val inputData1 = MemoryStream[Int] + val aggWithoutWatermark = inputData1.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) + + testStream(aggWithoutWatermark, outputMode = Complete)( + AddData(inputData1, 15), + CheckAnswer((15, 1)), + assertEventStats { e => assert(e.isEmpty) }, + AddData(inputData1, 10, 12, 14), + CheckAnswer((10, 3), (15, 1)), + assertEventStats { e => assert(e.isEmpty) } + ) + + // All event time metrics where watermarking is set + val inputData2 = MemoryStream[Int] + val aggWithWatermark = inputData2.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) + + testStream(aggWithWatermark)( + AddData(inputData2, 15), + CheckAnswer(), + assertEventStats { e => + assert(e.get("max") === formatTimestamp(15)) + assert(e.get("min") === formatTimestamp(15)) + assert(e.get("avg") === formatTimestamp(15)) + assert(e.get("watermark") === formatTimestamp(0)) + }, + AddData(inputData2, 10, 12, 14), + CheckAnswer(), + assertEventStats { e => + assert(e.get("max") === formatTimestamp(14)) + assert(e.get("min") === formatTimestamp(10)) + assert(e.get("avg") === formatTimestamp(12)) + assert(e.get("watermark") === formatTimestamp(5)) + }, + AddData(inputData2, 25), + CheckAnswer(), + assertEventStats { e => + assert(e.get("max") === formatTimestamp(25)) + assert(e.get("min") === formatTimestamp(25)) + assert(e.get("avg") === formatTimestamp(25)) + assert(e.get("watermark") === formatTimestamp(5)) + }, + AddData(inputData2, 25), + CheckAnswer((10, 3)), + assertEventStats { e => + assert(e.get("max") === formatTimestamp(25)) + assert(e.get("min") === formatTimestamp(25)) + assert(e.get("avg") === formatTimestamp(25)) + assert(e.get("watermark") === formatTimestamp(15)) + } + ) + } + + test("append mode") { + val inputData = MemoryStream[Int] + + val windowedAggregation = inputData.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) + + testStream(windowedAggregation)( + AddData(inputData, 10, 11, 12, 13, 14, 15), + CheckLastBatch(), + AddData(inputData, 25), // Advance watermark to 15 seconds + CheckLastBatch(), + assertNumStateRows(3), + AddData(inputData, 25), // Emit items less than watermark and drop their state + CheckLastBatch((10, 5)), + assertNumStateRows(2), + AddData(inputData, 10), // Should not emit anything as data less than watermark + CheckLastBatch(), + assertNumStateRows(2) + ) + } + + test("update mode") { + val inputData = MemoryStream[Int] + spark.conf.set("spark.sql.shuffle.partitions", "10") + + val windowedAggregation = inputData.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) + + testStream(windowedAggregation, OutputMode.Update)( + AddData(inputData, 10, 11, 12, 13, 14, 15), + CheckLastBatch((10, 5), (15, 1)), + AddData(inputData, 25), // Advance watermark to 15 seconds + CheckLastBatch((25, 1)), + assertNumStateRows(3), + AddData(inputData, 10, 25), // Ignore 10 as its less than watermark + CheckLastBatch((25, 2)), + assertNumStateRows(2), + AddData(inputData, 10), // Should not emit anything as data less than watermark + CheckLastBatch(), + assertNumStateRows(2) + ) + } + + test("delay in months and years handled correctly") { + val currentTimeMs = System.currentTimeMillis + val currentTime = new Date(currentTimeMs) + + val input = MemoryStream[Long] + val aggWithWatermark = input.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "2 years 5 months") + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) + + def monthsSinceEpoch(date: Date): Int = { date.getYear * 12 + date.getMonth } + + testStream(aggWithWatermark)( + AddData(input, currentTimeMs / 1000), + CheckAnswer(), + AddData(input, currentTimeMs / 1000), + CheckAnswer(), + assertEventStats { e => + assert(timestampFormat.parse(e.get("max")).getTime === (currentTimeMs / 1000) * 1000) + val watermarkTime = timestampFormat.parse(e.get("watermark")) + val monthDiff = monthsSinceEpoch(currentTime) - monthsSinceEpoch(watermarkTime) + // monthsSinceEpoch is like `math.floor(num)`, so monthDiff has two possible values. + assert(monthDiff === 29 || monthDiff === 30, + s"currentTime: $currentTime, watermarkTime: $watermarkTime") + } + ) + } + + test("recovery") { + val inputData = MemoryStream[Int] + val df = inputData.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) + + testStream(df)( + AddData(inputData, 10, 11, 12, 13, 14, 15), + CheckLastBatch(), + AddData(inputData, 25), // Advance watermark to 15 seconds + StopStream, + StartStream(), + CheckLastBatch(), + AddData(inputData, 25), // Evict items less than previous watermark. + CheckLastBatch((10, 5)), + StopStream, + AssertOnQuery { q => // purge commit and clear the sink + val commit = q.batchCommitLog.getLatest().map(_._1).getOrElse(-1L) + 1L + q.batchCommitLog.purge(commit) + q.sink.asInstanceOf[MemorySink].clear() + true + }, + StartStream(), + CheckLastBatch((10, 5)), // Recompute last batch and re-evict timestamp 10 + AddData(inputData, 30), // Advance watermark to 20 seconds + CheckLastBatch(), + StopStream, + StartStream(), // Watermark should still be 15 seconds + AddData(inputData, 17), + CheckLastBatch(), // We still do not see next batch + AddData(inputData, 30), // Advance watermark to 20 seconds + CheckLastBatch(), + AddData(inputData, 30), // Evict items less than previous watermark. + CheckLastBatch((15, 2)) // Ensure we see next window + ) + } + + test("dropping old data") { + val inputData = MemoryStream[Int] + + val windowedAggregation = inputData.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) + + testStream(windowedAggregation)( + AddData(inputData, 10, 11, 12), + CheckAnswer(), + AddData(inputData, 25), // Advance watermark to 15 seconds + CheckAnswer(), + AddData(inputData, 25), // Evict items less than previous watermark. + CheckAnswer((10, 3)), + AddData(inputData, 10), // 10 is later than 15 second watermark + CheckAnswer((10, 3)), + AddData(inputData, 25), + CheckAnswer((10, 3)) // Should not emit an incorrect partial result. + ) + } + + test("watermark with 2 streams") { + import org.apache.spark.sql.functions.sum + val first = MemoryStream[Int] + + val firstDf = first.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .select('value) + + val second = MemoryStream[Int] + + val secondDf = second.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "5 seconds") + .select('value) + + withTempDir { checkpointDir => + val unionWriter = firstDf.union(secondDf).agg(sum('value)) + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .format("memory") + .outputMode("complete") + .queryName("test") + + val union = unionWriter.start() + + def getWatermarkAfterData( + firstData: Seq[Int] = Seq.empty, + secondData: Seq[Int] = Seq.empty, + query: StreamingQuery = union): Long = { + if (firstData.nonEmpty) first.addData(firstData) + if (secondData.nonEmpty) second.addData(secondData) + query.processAllAvailable() + // add a dummy batch so lastExecution has the new watermark + first.addData(0) + query.processAllAvailable() + // get last watermark + val lastExecution = query.asInstanceOf[StreamingQueryWrapper].streamingQuery.lastExecution + lastExecution.offsetSeqMetadata.batchWatermarkMs + } + + // Global watermark starts at 0 until we get data from both sides + assert(getWatermarkAfterData(firstData = Seq(11)) == 0) + assert(getWatermarkAfterData(secondData = Seq(6)) == 1000) + // Global watermark stays at left watermark 1 when right watermark moves to 2 + assert(getWatermarkAfterData(secondData = Seq(8)) == 1000) + // Global watermark switches to right side value 2 when left watermark goes higher + assert(getWatermarkAfterData(firstData = Seq(21)) == 3000) + // Global watermark goes back to left + assert(getWatermarkAfterData(secondData = Seq(17, 28, 39)) == 11000) + // Global watermark stays on left as long as it's below right + assert(getWatermarkAfterData(firstData = Seq(31)) == 21000) + assert(getWatermarkAfterData(firstData = Seq(41)) == 31000) + // Global watermark switches back to right again + assert(getWatermarkAfterData(firstData = Seq(51)) == 34000) + + // Global watermark is updated correctly with simultaneous data from both sides + assert(getWatermarkAfterData(firstData = Seq(100), secondData = Seq(100)) == 90000) + assert(getWatermarkAfterData(firstData = Seq(120), secondData = Seq(110)) == 105000) + assert(getWatermarkAfterData(firstData = Seq(130), secondData = Seq(125)) == 120000) + + // Global watermark doesn't decrement with simultaneous data + assert(getWatermarkAfterData(firstData = Seq(100), secondData = Seq(100)) == 120000) + assert(getWatermarkAfterData(firstData = Seq(140), secondData = Seq(100)) == 120000) + assert(getWatermarkAfterData(firstData = Seq(100), secondData = Seq(135)) == 130000) + + // Global watermark recovers after restart, but left side watermark ahead of it does not. + assert(getWatermarkAfterData(firstData = Seq(200), secondData = Seq(190)) == 185000) + union.stop() + val union2 = unionWriter.start() + assert(getWatermarkAfterData(query = union2) == 185000) + // Even though the left side was ahead of 185000 in the last execution, the watermark won't + // increment until it gets past it in this execution. + assert(getWatermarkAfterData(secondData = Seq(200), query = union2) == 185000) + assert(getWatermarkAfterData(firstData = Seq(200), query = union2) == 190000) + } + } + + test("complete mode") { + val inputData = MemoryStream[Int] + + val windowedAggregation = inputData.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .groupBy(window($"eventTime", "5 seconds") as 'window) + .agg(count("*") as 'count) + .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) + + // No eviction when asked to compute complete results. + testStream(windowedAggregation, OutputMode.Complete)( + AddData(inputData, 10, 11, 12), + CheckAnswer((10, 3)), + AddData(inputData, 25), + CheckAnswer((10, 3), (25, 1)), + AddData(inputData, 25), + CheckAnswer((10, 3), (25, 2)), + AddData(inputData, 10), + CheckAnswer((10, 4), (25, 2)), + AddData(inputData, 25), + CheckAnswer((10, 4), (25, 3)) + ) + } + + test("group by on raw timestamp") { + val inputData = MemoryStream[Int] + + val windowedAggregation = inputData.toDF() + .withColumn("eventTime", $"value".cast("timestamp")) + .withWatermark("eventTime", "10 seconds") + .groupBy($"eventTime") + .agg(count("*") as 'count) + .select($"eventTime".cast("long").as[Long], $"count".as[Long]) + + testStream(windowedAggregation)( + AddData(inputData, 10), + CheckAnswer(), + AddData(inputData, 25), // Advance watermark to 15 seconds + CheckAnswer(), + AddData(inputData, 25), // Evict items less than previous watermark. + CheckAnswer((10, 1)) + ) + } + + test("delay threshold should not be negative.") { + val inputData = MemoryStream[Int].toDF() + var e = intercept[IllegalArgumentException] { + inputData.withWatermark("value", "-1 year") + } + assert(e.getMessage contains "should not be negative.") + + e = intercept[IllegalArgumentException] { + inputData.withWatermark("value", "1 year -13 months") + } + assert(e.getMessage contains "should not be negative.") + + e = intercept[IllegalArgumentException] { + inputData.withWatermark("value", "1 month -40 days") + } + assert(e.getMessage contains "should not be negative.") + + e = intercept[IllegalArgumentException] { + inputData.withWatermark("value", "-10 seconds") + } + assert(e.getMessage contains "should not be negative.") + } + + test("the new watermark should override the old one") { + val df = MemoryStream[(Long, Long)].toDF() + .withColumn("first", $"_1".cast("timestamp")) + .withColumn("second", $"_2".cast("timestamp")) + .withWatermark("first", "1 minute") + .withWatermark("second", "2 minutes") + + val eventTimeColumns = df.logicalPlan.output + .filter(_.metadata.contains(EventTimeWatermark.delayKey)) + assert(eventTimeColumns.size === 1) + assert(eventTimeColumns(0).name === "second") + } + + test("EventTime watermark should be ignored in batch query.") { + val df = testData + .withColumn("eventTime", $"key".cast("timestamp")) + .withWatermark("eventTime", "1 minute") + .select("eventTime") + .as[Long] + + checkDataset[Long](df, 1L to 100L: _*) + } + + test("SPARK-21565: watermark operator accepts attributes from replacement") { + withTempDir { dir => + dir.delete() + + val df = Seq(("a", 100.0, new java.sql.Timestamp(100L))) + .toDF("symbol", "price", "eventTime") + df.write.json(dir.getCanonicalPath) + + val input = spark.readStream.schema(df.schema) + .json(dir.getCanonicalPath) + + val groupEvents = input + .withWatermark("eventTime", "2 seconds") + .groupBy("symbol", "eventTime") + .agg(count("price") as 'count) + .select("symbol", "eventTime", "count") + val q = groupEvents.writeStream + .outputMode("append") + .format("console") + .start() + try { + q.processAllAvailable() + } finally { + q.stop() + } + } + } + + private def assertNumStateRows(numTotalRows: Long): AssertOnQuery = AssertOnQuery { q => + val progressWithData = q.recentProgress.filter(_.numInputRows > 0).lastOption.get + assert(progressWithData.stateOperators(0).numRowsTotal === numTotalRows) + true + } + + private def assertEventStats(body: ju.Map[String, String] => Unit): AssertOnQuery = { + AssertOnQuery { q => + body(q.recentProgress.filter(_.numInputRows > 0).lastOption.get.eventTime) + true + } + } + + private val timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 + timestampFormat.setTimeZone(ju.TimeZone.getTimeZone("UTC")) + + private def formatTimestamp(sec: Long): String = { + timestampFormat.format(new ju.Date(sec * 1000)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala new file mode 100644 index 000000000000..08db06b94904 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -0,0 +1,391 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.util.Locale + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{AnalysisException, DataFrame} +import org.apache.spark.sql.execution.DataSourceScanExec +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.util.Utils + +class FileStreamSinkSuite extends StreamTest { + import testImplicits._ + + test("unpartitioned writing and batch reading") { + val inputData = MemoryStream[Int] + val df = inputData.toDF() + + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + + var query: StreamingQuery = null + + try { + query = + df.writeStream + .option("checkpointLocation", checkpointDir) + .format("parquet") + .start(outputDir) + + inputData.addData(1, 2, 3) + + failAfter(streamingTimeout) { + query.processAllAvailable() + } + + val outputDf = spark.read.parquet(outputDir).as[Int] + checkDatasetUnorderly(outputDf, 1, 2, 3) + + } finally { + if (query != null) { + query.stop() + } + } + } + + test("SPARK-21167: encode and decode path correctly") { + val inputData = MemoryStream[String] + val ds = inputData.toDS() + + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + + val query = ds.map(s => (s, s.length)) + .toDF("value", "len") + .writeStream + .partitionBy("value") + .option("checkpointLocation", checkpointDir) + .format("parquet") + .start(outputDir) + + try { + // The output is partitoned by "value", so the value will appear in the file path. + // This is to test if we handle spaces in the path correctly. + inputData.addData("hello world") + failAfter(streamingTimeout) { + query.processAllAvailable() + } + val outputDf = spark.read.parquet(outputDir) + checkDatasetUnorderly(outputDf.as[(Int, String)], ("hello world".length, "hello world")) + } finally { + query.stop() + } + } + + test("partitioned writing and batch reading") { + val inputData = MemoryStream[Int] + val ds = inputData.toDS() + + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + + var query: StreamingQuery = null + + try { + query = + ds.map(i => (i, i * 1000)) + .toDF("id", "value") + .writeStream + .partitionBy("id") + .option("checkpointLocation", checkpointDir) + .format("parquet") + .start(outputDir) + + inputData.addData(1, 2, 3) + failAfter(streamingTimeout) { + query.processAllAvailable() + } + + val outputDf = spark.read.parquet(outputDir) + val expectedSchema = new StructType() + .add(StructField("value", IntegerType, nullable = false)) + .add(StructField("id", IntegerType)) + assert(outputDf.schema === expectedSchema) + + // Verify that MetadataLogFileIndex is being used and the correct partitioning schema has + // been inferred + val hadoopdFsRelations = outputDf.queryExecution.analyzed.collect { + case LogicalRelation(baseRelation: HadoopFsRelation, _, _, _) => baseRelation + } + assert(hadoopdFsRelations.size === 1) + assert(hadoopdFsRelations.head.location.isInstanceOf[MetadataLogFileIndex]) + assert(hadoopdFsRelations.head.partitionSchema.exists(_.name == "id")) + assert(hadoopdFsRelations.head.dataSchema.exists(_.name == "value")) + + // Verify the data is correctly read + checkDatasetUnorderly( + outputDf.as[(Int, Int)], + (1000, 1), (2000, 2), (3000, 3)) + + /** Check some condition on the partitions of the FileScanRDD generated by a DF */ + def checkFileScanPartitions(df: DataFrame)(func: Seq[FilePartition] => Unit): Unit = { + val getFileScanRDD = df.queryExecution.executedPlan.collect { + case scan: DataSourceScanExec if scan.inputRDDs().head.isInstanceOf[FileScanRDD] => + scan.inputRDDs().head.asInstanceOf[FileScanRDD] + }.headOption.getOrElse { + fail(s"No FileScan in query\n${df.queryExecution}") + } + func(getFileScanRDD.filePartitions) + } + + // Read without pruning + checkFileScanPartitions(outputDf) { partitions => + // There should be as many distinct partition values as there are distinct ids + assert(partitions.flatMap(_.files.map(_.partitionValues)).distinct.size === 3) + } + + // Read with pruning, should read only files in partition dir id=1 + checkFileScanPartitions(outputDf.filter("id = 1")) { partitions => + val filesToBeRead = partitions.flatMap(_.files) + assert(filesToBeRead.map(_.filePath).forall(_.contains("/id=1/"))) + assert(filesToBeRead.map(_.partitionValues).distinct.size === 1) + } + + // Read with pruning, should read only files in partition dir id=1 and id=2 + checkFileScanPartitions(outputDf.filter("id in (1,2)")) { partitions => + val filesToBeRead = partitions.flatMap(_.files) + assert(!filesToBeRead.map(_.filePath).exists(_.contains("/id=3/"))) + assert(filesToBeRead.map(_.partitionValues).distinct.size === 2) + } + } finally { + if (query != null) { + query.stop() + } + } + } + + test("partitioned writing and batch reading with 'basePath'") { + withTempDir { outputDir => + withTempDir { checkpointDir => + val outputPath = outputDir.getAbsolutePath + val inputData = MemoryStream[Int] + val ds = inputData.toDS() + + var query: StreamingQuery = null + + try { + query = + ds.map(i => (i, -i, i * 1000)) + .toDF("id1", "id2", "value") + .writeStream + .partitionBy("id1", "id2") + .option("checkpointLocation", checkpointDir.getAbsolutePath) + .format("parquet") + .start(outputPath) + + inputData.addData(1, 2, 3) + failAfter(streamingTimeout) { + query.processAllAvailable() + } + + val readIn = spark.read.option("basePath", outputPath).parquet(s"$outputDir/*/*") + checkDatasetUnorderly( + readIn.as[(Int, Int, Int)], + (1000, 1, -1), (2000, 2, -2), (3000, 3, -3)) + } finally { + if (query != null) { + query.stop() + } + } + } + } + } + + // This tests whether FileStreamSink works with aggregations. Specifically, it tests + // whether the correct streaming QueryExecution (i.e. IncrementalExecution) is used to + // to execute the trigger for writing data to file sink. See SPARK-18440 for more details. + test("writing with aggregation") { + + // Since FileStreamSink currently only supports append mode, we will test FileStreamSink + // with aggregations using event time windows and watermark, which allows + // aggregation + append mode. + val inputData = MemoryStream[Long] + val inputDF = inputData.toDF.toDF("time") + val outputDf = inputDF + .selectExpr("CAST(time AS timestamp) AS timestamp") + .withWatermark("timestamp", "10 seconds") + .groupBy(window($"timestamp", "5 seconds")) + .count() + .select("window.start", "window.end", "count") + + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + + var query: StreamingQuery = null + + try { + query = + outputDf.writeStream + .option("checkpointLocation", checkpointDir) + .format("parquet") + .start(outputDir) + + + def addTimestamp(timestampInSecs: Int*): Unit = { + inputData.addData(timestampInSecs.map(_ * 1L): _*) + failAfter(streamingTimeout) { + query.processAllAvailable() + } + } + + def check(expectedResult: ((Long, Long), Long)*): Unit = { + val outputDf = spark.read.parquet(outputDir) + .selectExpr( + "CAST(start as BIGINT) AS start", + "CAST(end as BIGINT) AS end", + "count") + checkDataset( + outputDf.as[(Long, Long, Long)], + expectedResult.map(x => (x._1._1, x._1._2, x._2)): _*) + } + + addTimestamp(100) // watermark = None before this, watermark = 100 - 10 = 90 after this + check() // nothing emitted yet + + addTimestamp(104, 123) // watermark = 90 before this, watermark = 123 - 10 = 113 after this + check() // nothing emitted yet + + addTimestamp(140) // wm = 113 before this, emit results on 100-105, wm = 130 after this + check((100L, 105L) -> 2L) + + addTimestamp(150) // wm = 130s before this, emit results on 120-125, wm = 150 after this + check((100L, 105L) -> 2L, (120L, 125L) -> 1L) + + } finally { + if (query != null) { + query.stop() + } + } + } + + test("Update and Complete output mode not supported") { + val df = MemoryStream[Int].toDF().groupBy().count() + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + + withTempDir { dir => + + def testOutputMode(mode: String): Unit = { + val e = intercept[AnalysisException] { + df.writeStream.format("parquet").outputMode(mode).start(dir.getCanonicalPath) + } + Seq(mode, "not support").foreach { w => + assert(e.getMessage.toLowerCase(Locale.ROOT).contains(w)) + } + } + + testOutputMode("update") + testOutputMode("complete") + } + } + + test("parquet") { + testFormat(None) // should not throw error as default format parquet when not specified + testFormat(Some("parquet")) + } + + test("text") { + testFormat(Some("text")) + } + + test("json") { + testFormat(Some("json")) + } + + def testFormat(format: Option[String]): Unit = { + val inputData = MemoryStream[Int] + val ds = inputData.toDS() + + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + + var query: StreamingQuery = null + + try { + val writer = ds.map(i => (i, i * 1000)).toDF("id", "value").writeStream + if (format.nonEmpty) { + writer.format(format.get) + } + query = writer.option("checkpointLocation", checkpointDir).start(outputDir) + } finally { + if (query != null) { + query.stop() + } + } + } + + test("FileStreamSink.ancestorIsMetadataDirectory()") { + val hadoopConf = spark.sparkContext.hadoopConfiguration + def assertAncestorIsMetadataDirectory(path: String): Unit = + assert(FileStreamSink.ancestorIsMetadataDirectory(new Path(path), hadoopConf)) + def assertAncestorIsNotMetadataDirectory(path: String): Unit = + assert(!FileStreamSink.ancestorIsMetadataDirectory(new Path(path), hadoopConf)) + + assertAncestorIsMetadataDirectory(s"/${FileStreamSink.metadataDir}") + assertAncestorIsMetadataDirectory(s"/${FileStreamSink.metadataDir}/") + assertAncestorIsMetadataDirectory(s"/a/${FileStreamSink.metadataDir}") + assertAncestorIsMetadataDirectory(s"/a/${FileStreamSink.metadataDir}/") + assertAncestorIsMetadataDirectory(s"/a/b/${FileStreamSink.metadataDir}/c") + assertAncestorIsMetadataDirectory(s"/a/b/${FileStreamSink.metadataDir}/c/") + + assertAncestorIsNotMetadataDirectory(s"/a/b/c") + assertAncestorIsNotMetadataDirectory(s"/a/b/c/${FileStreamSink.metadataDir}extra") + } + + test("SPARK-20460 Check name duplication in schema") { + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + val inputData = MemoryStream[(Int, Int)] + val df = inputData.toDF() + + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + + var query: StreamingQuery = null + try { + query = + df.writeStream + .option("checkpointLocation", checkpointDir) + .format("json") + .start(outputDir) + + inputData.addData((1, 1)) + + failAfter(streamingTimeout) { + query.processAllAvailable() + } + } finally { + if (query != null) { + query.stop() + } + } + + val errorMsg = intercept[AnalysisException] { + spark.read.schema(s"$c0 INT, $c1 INT").json(outputDir).as[(Int, Int)] + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) in the data schema: ")) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala new file mode 100644 index 000000000000..b6baaed1927e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -0,0 +1,1372 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.io.File +import java.net.URI + +import scala.util.Random + +import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem} +import org.scalatest.PrivateMethodTester +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.FileStreamSource.{FileEntry, SeenFilesMap} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.ExistsThrowsExceptionFileSystem._ +import org.apache.spark.sql.streaming.util.StreamManualClock +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + +abstract class FileStreamSourceTest + extends StreamTest with SharedSQLContext with PrivateMethodTester { + + import testImplicits._ + + /** + * A subclass `AddData` for adding data to files. This is meant to use the + * `FileStreamSource` actually being used in the execution. + */ + abstract class AddFileData extends AddData { + override def addData(query: Option[StreamExecution]): (Source, Offset) = { + require( + query.nonEmpty, + "Cannot add data when there is no query for finding the active file stream source") + + val sources = getSourcesFromStreamingQuery(query.get) + if (sources.isEmpty) { + throw new Exception( + "Could not find file source in the StreamExecution logical plan to add data to") + } else if (sources.size > 1) { + throw new Exception( + "Could not select the file source in the StreamExecution logical plan as there" + + "are multiple file sources:\n\t" + sources.mkString("\n\t")) + } + val source = sources.head + val newOffset = source.withBatchingLocked { + addData(source) + new FileStreamSourceOffset(source.currentLogOffset + 1) + } + logInfo(s"Added file to $source at offset $newOffset") + (source, newOffset) + } + + protected def addData(source: FileStreamSource): Unit + } + + case class AddTextFileData(content: String, src: File, tmp: File) + extends AddFileData { + + override def addData(source: FileStreamSource): Unit = { + val tempFile = Utils.tempFileWith(new File(tmp, "text")) + val finalFile = new File(src, tempFile.getName) + src.mkdirs() + require(stringToFile(tempFile, content).renameTo(finalFile)) + logInfo(s"Written text '$content' to file $finalFile") + } + } + + case class AddParquetFileData(data: DataFrame, src: File, tmp: File) extends AddFileData { + override def addData(source: FileStreamSource): Unit = { + AddParquetFileData.writeToFile(data, src, tmp) + } + } + + object AddParquetFileData { + def apply(seq: Seq[String], src: File, tmp: File): AddParquetFileData = { + AddParquetFileData(seq.toDS().toDF(), src, tmp) + } + + /** Write parquet files in a temp dir, and move the individual files to the 'src' dir */ + def writeToFile(df: DataFrame, src: File, tmp: File): Unit = { + val tmpDir = Utils.tempFileWith(new File(tmp, "parquet")) + df.write.parquet(tmpDir.getCanonicalPath) + src.mkdirs() + tmpDir.listFiles().foreach { f => + f.renameTo(new File(src, s"${f.getName}")) + } + } + } + + /** Use `format` and `path` to create FileStreamSource via DataFrameReader */ + def createFileStream( + format: String, + path: String, + schema: Option[StructType] = None, + options: Map[String, String] = Map.empty): DataFrame = { + val reader = + if (schema.isDefined) { + spark.readStream.format(format).schema(schema.get).options(options) + } else { + spark.readStream.format(format).options(options) + } + reader.load(path) + } + + protected def getSourceFromFileStream(df: DataFrame): FileStreamSource = { + val checkpointLocation = Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath + df.queryExecution.analyzed + .collect { case StreamingRelation(dataSource, _, _) => + // There is only one source in our tests so just set sourceId to 0 + dataSource.createSource(s"$checkpointLocation/sources/0").asInstanceOf[FileStreamSource] + }.head + } + + protected def getSourcesFromStreamingQuery(query: StreamExecution): Seq[FileStreamSource] = { + query.logicalPlan.collect { + case StreamingExecutionRelation(source, _) if source.isInstanceOf[FileStreamSource] => + source.asInstanceOf[FileStreamSource] + } + } + + + protected def withTempDirs(body: (File, File) => Unit) { + val src = Utils.createTempDir(namePrefix = "streaming.src") + val tmp = Utils.createTempDir(namePrefix = "streaming.tmp") + try { + body(src, tmp) + } finally { + Utils.deleteRecursively(src) + Utils.deleteRecursively(tmp) + } + } + + val valueSchema = new StructType().add("value", StringType) +} + +class FileStreamSourceSuite extends FileStreamSourceTest { + + import testImplicits._ + + override val streamingTimeout = 20.seconds + + /** Use `format` and `path` to create FileStreamSource via DataFrameReader */ + private def createFileStreamSource( + format: String, + path: String, + schema: Option[StructType] = None): FileStreamSource = { + getSourceFromFileStream(createFileStream(format, path, schema)) + } + + private def createFileStreamSourceAndGetSchema( + format: Option[String], + path: Option[String], + schema: Option[StructType] = None): StructType = { + val reader = spark.readStream + format.foreach(reader.format) + schema.foreach(reader.schema) + val df = + if (path.isDefined) { + reader.load(path.get) + } else { + reader.load() + } + df.queryExecution.analyzed + .collect { case s @ StreamingRelation(dataSource, _, _) => s.schema }.head + } + + // ============= Basic parameter exists tests ================ + + test("FileStreamSource schema: no path") { + def testError(): Unit = { + val e = intercept[IllegalArgumentException] { + createFileStreamSourceAndGetSchema(format = None, path = None, schema = None) + } + assert(e.getMessage.contains("path")) // reason is path, not schema + } + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "false") { testError() } + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { testError() } + } + + test("FileStreamSource schema: path doesn't exist (without schema) should throw exception") { + withTempDir { dir => + intercept[AnalysisException] { + val userSchema = new StructType().add(new StructField("value", IntegerType)) + val schema = createFileStreamSourceAndGetSchema( + format = None, path = Some(new File(dir, "1").getAbsolutePath), schema = None) + } + } + } + + test("FileStreamSource schema: path doesn't exist (with schema) should throw exception") { + withTempDir { dir => + intercept[AnalysisException] { + val userSchema = new StructType().add(new StructField("value", IntegerType)) + val schema = createFileStreamSourceAndGetSchema( + format = None, path = Some(new File(dir, "1").getAbsolutePath), schema = Some(userSchema)) + } + } + } + + + // =============== Text file stream schema tests ================ + + test("FileStreamSource schema: text, no existing files, no schema") { + withTempDir { src => + val schema = createFileStreamSourceAndGetSchema( + format = Some("text"), path = Some(src.getCanonicalPath), schema = None) + assert(schema === new StructType().add("value", StringType)) + } + } + + test("FileStreamSource schema: text, existing files, no schema") { + withTempDir { src => + stringToFile(new File(src, "1"), "a\nb\nc") + val schema = createFileStreamSourceAndGetSchema( + format = Some("text"), path = Some(src.getCanonicalPath), schema = None) + assert(schema === new StructType().add("value", StringType)) + } + } + + test("FileStreamSource schema: text, existing files, schema") { + withTempDir { src => + stringToFile(new File(src, "1"), "a\nb\nc") + val userSchema = new StructType().add("userColumn", StringType) + val schema = createFileStreamSourceAndGetSchema( + format = Some("text"), path = Some(src.getCanonicalPath), schema = Some(userSchema)) + assert(schema === userSchema) + } + } + + // =============== Parquet file stream schema tests ================ + + test("FileStreamSource schema: parquet, existing files, no schema") { + withTempDir { src => + Seq("a", "b", "c").toDS().as("userColumn").toDF().write + .mode(org.apache.spark.sql.SaveMode.Overwrite) + .parquet(src.getCanonicalPath) + + // Without schema inference, should throw error + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "false") { + intercept[IllegalArgumentException] { + createFileStreamSourceAndGetSchema( + format = Some("parquet"), path = Some(src.getCanonicalPath), schema = None) + } + } + + // With schema inference, should infer correct schema + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + val schema = createFileStreamSourceAndGetSchema( + format = Some("parquet"), path = Some(src.getCanonicalPath), schema = None) + assert(schema === new StructType().add("value", StringType)) + } + } + } + + test("FileStreamSource schema: parquet, existing files, schema") { + withTempPath { src => + Seq("a", "b", "c").toDS().as("oldUserColumn").toDF() + .write.parquet(new File(src, "1").getCanonicalPath) + val userSchema = new StructType().add("userColumn", StringType) + val schema = createFileStreamSourceAndGetSchema( + format = Some("parquet"), path = Some(src.getCanonicalPath), schema = Some(userSchema)) + assert(schema === userSchema) + } + } + + // =============== JSON file stream schema tests ================ + + test("FileStreamSource schema: json, no existing files, no schema") { + withTempDir { src => + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + + val e = intercept[AnalysisException] { + createFileStreamSourceAndGetSchema( + format = Some("json"), path = Some(src.getCanonicalPath), schema = None) + } + assert("Unable to infer schema for JSON. It must be specified manually.;" === e.getMessage) + } + } + } + + test("FileStreamSource schema: json, existing files, no schema") { + withTempDir { src => + + // Without schema inference, should throw error + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "false") { + intercept[IllegalArgumentException] { + createFileStreamSourceAndGetSchema( + format = Some("json"), path = Some(src.getCanonicalPath), schema = None) + } + } + + // With schema inference, should infer correct schema + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + stringToFile(new File(src, "1"), "{'c': '1'}\n{'c': '2'}\n{'c': '3'}") + val schema = createFileStreamSourceAndGetSchema( + format = Some("json"), path = Some(src.getCanonicalPath), schema = None) + assert(schema === new StructType().add("c", StringType)) + } + } + } + + test("FileStreamSource schema: json, existing files, schema") { + withTempDir { src => + stringToFile(new File(src, "1"), "{'c': '1'}\n{'c': '2'}\n{'c', '3'}") + val userSchema = new StructType().add("userColumn", StringType) + val schema = createFileStreamSourceAndGetSchema( + format = Some("json"), path = Some(src.getCanonicalPath), schema = Some(userSchema)) + assert(schema === userSchema) + } + } + + // =============== Text file stream tests ================ + + test("read from text files") { + withTempDirs { case (src, tmp) => + val textStream = createFileStream("text", src.getCanonicalPath) + val filtered = textStream.filter($"value" contains "keep") + + testStream(filtered)( + AddTextFileData("drop1\nkeep2\nkeep3", src, tmp), + CheckAnswer("keep2", "keep3"), + StopStream, + AddTextFileData("drop4\nkeep5\nkeep6", src, tmp), + StartStream(), + CheckAnswer("keep2", "keep3", "keep5", "keep6"), + AddTextFileData("drop7\nkeep8\nkeep9", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") + ) + } + } + + test("read from textfile") { + withTempDirs { case (src, tmp) => + val textStream = spark.readStream.textFile(src.getCanonicalPath) + val filtered = textStream.filter(_.contains("keep")) + + testStream(filtered)( + AddTextFileData("drop1\nkeep2\nkeep3", src, tmp), + CheckAnswer("keep2", "keep3"), + StopStream, + AddTextFileData("drop4\nkeep5\nkeep6", src, tmp), + StartStream(), + CheckAnswer("keep2", "keep3", "keep5", "keep6"), + AddTextFileData("drop7\nkeep8\nkeep9", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") + ) + } + } + + test("SPARK-17165 should not track the list of seen files indefinitely") { + // This test works by: + // 1. Create a file + // 2. Get it processed + // 3. Sleeps for a very short amount of time (larger than maxFileAge + // 4. Add another file (at this point the original file should have been purged + // 5. Test the size of the seenFiles internal data structure + + // Note that if we change maxFileAge to a very large number, the last step should fail. + withTempDirs { case (src, tmp) => + val textStream: DataFrame = + createFileStream("text", src.getCanonicalPath, options = Map("maxFileAge" -> "5ms")) + + testStream(textStream)( + AddTextFileData("a\nb", src, tmp), + CheckAnswer("a", "b"), + + // SLeeps longer than 5ms (maxFileAge) + // Unfortunately since a lot of file system does not have modification time granularity + // finer grained than 1 sec, we need to use 1 sec here. + AssertOnQuery { _ => Thread.sleep(1000); true }, + + AddTextFileData("c\nd", src, tmp), + CheckAnswer("a", "b", "c", "d"), + + AssertOnQuery("seen files should contain only one entry") { streamExecution => + val source = getSourcesFromStreamingQuery(streamExecution).head + assert(source.seenFiles.size == 1) + true + } + ) + } + } + + // =============== JSON file stream tests ================ + + test("read from json files") { + withTempDirs { case (src, tmp) => + val fileStream = createFileStream("json", src.getCanonicalPath, Some(valueSchema)) + val filtered = fileStream.filter($"value" contains "keep") + + testStream(filtered)( + AddTextFileData( + "{'value': 'drop1'}\n{'value': 'keep2'}\n{'value': 'keep3'}", + src, + tmp), + CheckAnswer("keep2", "keep3"), + StopStream, + AddTextFileData( + "{'value': 'drop4'}\n{'value': 'keep5'}\n{'value': 'keep6'}", + src, + tmp), + StartStream(), + CheckAnswer("keep2", "keep3", "keep5", "keep6"), + AddTextFileData( + "{'value': 'drop7'}\n{'value': 'keep8'}\n{'value': 'keep9'}", + src, + tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") + ) + } + } + + test("read from json files with inferring schema") { + withTempDirs { case (src, tmp) => + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + + // Add a file so that we can infer its schema + stringToFile(new File(src, "existing"), "{'c': 'drop1'}\n{'c': 'keep2'}\n{'c': 'keep3'}") + + val fileStream = createFileStream("json", src.getCanonicalPath) + assert(fileStream.schema === StructType(Seq(StructField("c", StringType)))) + + // FileStreamSource should infer the column "c" + val filtered = fileStream.filter($"c" contains "keep") + + testStream(filtered)( + AddTextFileData("{'c': 'drop4'}\n{'c': 'keep5'}\n{'c': 'keep6'}", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6") + ) + } + } + } + + test("reading from json files inside partitioned directory") { + withTempDirs { case (baseSrc, tmp) => + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + val src = new File(baseSrc, "type=X") + src.mkdirs() + + // Add a file so that we can infer its schema + stringToFile(new File(src, "existing"), "{'c': 'drop1'}\n{'c': 'keep2'}\n{'c': 'keep3'}") + + val fileStream = createFileStream("json", src.getCanonicalPath) + + // FileStreamSource should infer the column "c" + val filtered = fileStream.filter($"c" contains "keep") + + testStream(filtered)( + AddTextFileData("{'c': 'drop4'}\n{'c': 'keep5'}\n{'c': 'keep6'}", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6") + ) + } + } + } + + test("reading from json files with changing schema") { + withTempDirs { case (src, tmp) => + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + + // Add a file so that we can infer its schema + stringToFile(new File(src, "existing"), "{'k': 'value0'}") + + val fileStream = createFileStream("json", src.getCanonicalPath) + + // FileStreamSource should infer the column "k" + assert(fileStream.schema === StructType(Seq(StructField("k", StringType)))) + + // After creating DF and before starting stream, add data with different schema + // Should not affect the inferred schema any more + stringToFile(new File(src, "existing2"), "{'k': 'value1', 'v': 'new'}") + + testStream(fileStream)( + + // Should not pick up column v in the file added before start + AddTextFileData("{'k': 'value2'}", src, tmp), + CheckAnswer("value0", "value1", "value2"), + + // Should read data in column k, and ignore v + AddTextFileData("{'k': 'value3', 'v': 'new'}", src, tmp), + CheckAnswer("value0", "value1", "value2", "value3"), + + // Should ignore rows that do not have the necessary k column + AddTextFileData("{'v': 'value4'}", src, tmp), + CheckAnswer("value0", "value1", "value2", "value3", null)) + } + } + } + + // =============== Parquet file stream tests ================ + + test("read from parquet files") { + withTempDirs { case (src, tmp) => + val fileStream = createFileStream("parquet", src.getCanonicalPath, Some(valueSchema)) + val filtered = fileStream.filter($"value" contains "keep") + + testStream(filtered)( + AddParquetFileData(Seq("drop1", "keep2", "keep3"), src, tmp), + CheckAnswer("keep2", "keep3"), + StopStream, + AddParquetFileData(Seq("drop4", "keep5", "keep6"), src, tmp), + StartStream(), + CheckAnswer("keep2", "keep3", "keep5", "keep6"), + AddParquetFileData(Seq("drop7", "keep8", "keep9"), src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") + ) + } + } + + test("read from parquet files with changing schema") { + + withTempDirs { case (src, tmp) => + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + + // Add a file so that we can infer its schema + AddParquetFileData.writeToFile(Seq("value0").toDF("k"), src, tmp) + + val fileStream = createFileStream("parquet", src.getCanonicalPath) + + // FileStreamSource should infer the column "k" + assert(fileStream.schema === StructType(Seq(StructField("k", StringType)))) + + // After creating DF and before starting stream, add data with different schema + // Should not affect the inferred schema any more + AddParquetFileData.writeToFile(Seq(("value1", 0)).toDF("k", "v"), src, tmp) + + testStream(fileStream)( + // Should not pick up column v in the file added before start + AddParquetFileData(Seq("value2").toDF("k"), src, tmp), + CheckAnswer("value0", "value1", "value2"), + + // Should read data in column k, and ignore v + AddParquetFileData(Seq(("value3", 1)).toDF("k", "v"), src, tmp), + CheckAnswer("value0", "value1", "value2", "value3"), + + // Should ignore rows that do not have the necessary k column + AddParquetFileData(Seq("value5").toDF("v"), src, tmp), + CheckAnswer("value0", "value1", "value2", "value3", null) + ) + } + } + } + + // =============== file stream globbing tests ================ + + test("read new files in nested directories with globbing") { + withTempDirs { case (dir, tmp) => + + // src/*/* should consider all the files and directories that matches that glob. + // So any files that matches the glob as well as any files in directories that matches + // this glob should be read. + val fileStream = createFileStream("text", s"${dir.getCanonicalPath}/*/*") + val filtered = fileStream.filter($"value" contains "keep") + val subDir = new File(dir, "subdir") + val subSubDir = new File(subDir, "subsubdir") + val subSubSubDir = new File(subSubDir, "subsubsubdir") + + require(!subDir.exists()) + require(!subSubDir.exists()) + + testStream(filtered)( + // Create new dir/subdir and write to it, should read + AddTextFileData("drop1\nkeep2", subDir, tmp), + CheckAnswer("keep2"), + + // Add files to dir/subdir, should read + AddTextFileData("keep3", subDir, tmp), + CheckAnswer("keep2", "keep3"), + + // Create new dir/subdir/subsubdir and write to it, should read + AddTextFileData("keep4", subSubDir, tmp), + CheckAnswer("keep2", "keep3", "keep4"), + + // Add files to dir/subdir/subsubdir, should read + AddTextFileData("keep5", subSubDir, tmp), + CheckAnswer("keep2", "keep3", "keep4", "keep5"), + + // 1. Add file to src dir, should not read as globbing src/*/* does not capture files in + // dir, only captures files in dir/subdir/ + // 2. Add files to dir/subDir/subsubdir/subsubsubdir, should not read as src/*/* should + // not capture those files + AddTextFileData("keep6", dir, tmp), + AddTextFileData("keep7", subSubSubDir, tmp), + AddTextFileData("keep8", subDir, tmp), // needed to make query detect new data + CheckAnswer("keep2", "keep3", "keep4", "keep5", "keep8") + ) + } + } + + test("read new files in partitioned table with globbing, should not read partition data") { + withTempDirs { case (dir, tmp) => + val partitionFooSubDir = new File(dir, "partition=foo") + val partitionBarSubDir = new File(dir, "partition=bar") + + val schema = new StructType().add("value", StringType).add("partition", StringType) + val fileStream = createFileStream("json", s"${dir.getCanonicalPath}/*/*", Some(schema)) + val filtered = fileStream.filter($"value" contains "keep") + val nullStr = null.asInstanceOf[String] + testStream(filtered)( + // Create new partition=foo sub dir and write to it, should read only value, not partition + AddTextFileData("{'value': 'drop1'}\n{'value': 'keep2'}", partitionFooSubDir, tmp), + CheckAnswer(("keep2", nullStr)), + + // Append to same partition=1 sub dir, should read only value, not partition + AddTextFileData("{'value': 'keep3'}", partitionFooSubDir, tmp), + CheckAnswer(("keep2", nullStr), ("keep3", nullStr)), + + // Create new partition sub dir and write to it, should read only value, not partition + AddTextFileData("{'value': 'keep4'}", partitionBarSubDir, tmp), + CheckAnswer(("keep2", nullStr), ("keep3", nullStr), ("keep4", nullStr)), + + // Append to same partition=2 sub dir, should read only value, not partition + AddTextFileData("{'value': 'keep5'}", partitionBarSubDir, tmp), + CheckAnswer(("keep2", nullStr), ("keep3", nullStr), ("keep4", nullStr), ("keep5", nullStr)) + ) + } + } + + // =============== other tests ================ + + test("read new files in partitioned table without globbing, should read partition data") { + withTempDirs { case (dir, tmp) => + val partitionFooSubDir = new File(dir, "partition=foo") + val partitionBarSubDir = new File(dir, "partition=bar") + + val schema = new StructType().add("value", StringType).add("partition", StringType) + val fileStream = createFileStream("json", s"${dir.getCanonicalPath}", Some(schema)) + val filtered = fileStream.filter($"value" contains "keep") + testStream(filtered)( + // Create new partition=foo sub dir and write to it + AddTextFileData("{'value': 'drop1'}\n{'value': 'keep2'}", partitionFooSubDir, tmp), + CheckAnswer(("keep2", "foo")), + + // Append to same partition=foo sub dir + AddTextFileData("{'value': 'keep3'}", partitionFooSubDir, tmp), + CheckAnswer(("keep2", "foo"), ("keep3", "foo")), + + // Create new partition sub dir and write to it + AddTextFileData("{'value': 'keep4'}", partitionBarSubDir, tmp), + CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar")), + + // Append to same partition=bar sub dir + AddTextFileData("{'value': 'keep5'}", partitionBarSubDir, tmp), + CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar"), ("keep5", "bar")) + ) + } + } + + test("read data from outputs of another streaming query") { + withSQLConf(SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL.key -> "3") { + withTempDirs { case (outputDir, checkpointDir) => + // q1 is a streaming query that reads from memory and writes to text files + val q1Source = MemoryStream[String] + val q1 = + q1Source + .toDF() + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .format("text") + .start(outputDir.getCanonicalPath) + + // q2 is a streaming query that reads q1's text outputs + val q2 = + createFileStream("text", outputDir.getCanonicalPath).filter($"value" contains "keep") + + def q1AddData(data: String*): StreamAction = + Execute { _ => + q1Source.addData(data) + q1.processAllAvailable() + } + def q2ProcessAllAvailable(): StreamAction = Execute { q2 => q2.processAllAvailable() } + + testStream(q2)( + // batch 0 + q1AddData("drop1", "keep2"), + q2ProcessAllAvailable(), + CheckAnswer("keep2"), + + // batch 1 + Assert { + // create a text file that won't be on q1's sink log + // thus even if its content contains "keep", it should NOT appear in q2's answer + val shouldNotKeep = new File(outputDir, "should_not_keep.txt") + stringToFile(shouldNotKeep, "should_not_keep!!!") + shouldNotKeep.exists() + }, + q1AddData("keep3"), + q2ProcessAllAvailable(), + CheckAnswer("keep2", "keep3"), + + // batch 2: check that things work well when the sink log gets compacted + q1AddData("keep4"), + Assert { + // compact interval is 3, so file "2.compact" should exist + new File(outputDir, s"${FileStreamSink.metadataDir}/2.compact").exists() + }, + q2ProcessAllAvailable(), + CheckAnswer("keep2", "keep3", "keep4"), + + Execute { _ => q1.stop() } + ) + } + } + } + + test("start before another streaming query, and read its output") { + withTempDirs { case (outputDir, checkpointDir) => + // q1 is a streaming query that reads from memory and writes to text files + val q1Source = MemoryStream[String] + // define q1, but don't start it for now + val q1Write = + q1Source + .toDF() + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .format("text") + var q1: StreamingQuery = null + + val q2 = createFileStream("text", outputDir.getCanonicalPath).filter($"value" contains "keep") + + testStream(q2)( + AssertOnQuery { q2 => + val fileSource = getSourcesFromStreamingQuery(q2).head + // q1 has not started yet, verify that q2 doesn't know whether q1 has metadata + fileSource.sourceHasMetadata === None + }, + Execute { _ => + q1 = q1Write.start(outputDir.getCanonicalPath) + q1Source.addData("drop1", "keep2") + q1.processAllAvailable() + }, + AssertOnQuery { q2 => + q2.processAllAvailable() + val fileSource = getSourcesFromStreamingQuery(q2).head + // q1 has started, verify that q2 knows q1 has metadata by now + fileSource.sourceHasMetadata === Some(true) + }, + CheckAnswer("keep2"), + Execute { _ => q1.stop() } + ) + } + } + + test("when schema inference is turned on, should read partition data") { + def createFile(content: String, src: File, tmp: File): Unit = { + val tempFile = Utils.tempFileWith(new File(tmp, "text")) + val finalFile = new File(src, tempFile.getName) + require(!src.exists(), s"$src exists, dir: ${src.isDirectory}, file: ${src.isFile}") + require(src.mkdirs(), s"Cannot create $src") + require(src.isDirectory(), s"$src is not a directory") + require(stringToFile(tempFile, content).renameTo(finalFile)) + } + + withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { + withTempDirs { case (dir, tmp) => + val partitionFooSubDir = new File(dir, "partition=foo") + val partitionBarSubDir = new File(dir, "partition=bar") + + // Create file in partition, so we can infer the schema. + createFile("{'value': 'drop0'}", partitionFooSubDir, tmp) + + val fileStream = createFileStream("json", s"${dir.getCanonicalPath}") + val filtered = fileStream.filter($"value" contains "keep") + testStream(filtered)( + // Append to same partition=foo sub dir + AddTextFileData("{'value': 'drop1'}\n{'value': 'keep2'}", partitionFooSubDir, tmp), + CheckAnswer(("keep2", "foo")), + + // Append to same partition=foo sub dir + AddTextFileData("{'value': 'keep3'}", partitionFooSubDir, tmp), + CheckAnswer(("keep2", "foo"), ("keep3", "foo")), + + // Create new partition sub dir and write to it + AddTextFileData("{'value': 'keep4'}", partitionBarSubDir, tmp), + CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar")), + + // Append to same partition=bar sub dir + AddTextFileData("{'value': 'keep5'}", partitionBarSubDir, tmp), + CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar"), ("keep5", "bar")), + + AddTextFileData("{'value': 'keep6'}", partitionBarSubDir, tmp), + CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar"), ("keep5", "bar"), + ("keep6", "bar")) + ) + } + } + } + + test("fault tolerance") { + withTempDirs { case (src, tmp) => + val fileStream = createFileStream("text", src.getCanonicalPath) + val filtered = fileStream.filter($"value" contains "keep") + + testStream(filtered)( + AddTextFileData("drop1\nkeep2\nkeep3", src, tmp), + CheckAnswer("keep2", "keep3"), + StopStream, + AddTextFileData("drop4\nkeep5\nkeep6", src, tmp), + StartStream(), + CheckAnswer("keep2", "keep3", "keep5", "keep6"), + AddTextFileData("drop7\nkeep8\nkeep9", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9") + ) + } + } + + test("max files per trigger") { + withTempDir { case src => + var lastFileModTime: Option[Long] = None + + /** Create a text file with a single data item */ + def createFile(data: Int): File = { + val file = stringToFile(new File(src, s"$data.txt"), data.toString) + if (lastFileModTime.nonEmpty) file.setLastModified(lastFileModTime.get + 1000) + lastFileModTime = Some(file.lastModified) + file + } + + createFile(1) + createFile(2) + createFile(3) + + // Set up a query to read text files 2 at a time + val df = spark + .readStream + .option("maxFilesPerTrigger", 2) + .text(src.getCanonicalPath) + val q = df + .writeStream + .format("memory") + .queryName("file_data") + .start() + .asInstanceOf[StreamingQueryWrapper] + .streamingQuery + q.processAllAvailable() + val memorySink = q.sink.asInstanceOf[MemorySink] + val fileSource = getSourcesFromStreamingQuery(q).head + + /** Check the data read in the last batch */ + def checkLastBatchData(data: Int*): Unit = { + val schema = StructType(Seq(StructField("value", StringType))) + val df = spark.createDataFrame( + spark.sparkContext.makeRDD(memorySink.latestBatchData), schema) + checkAnswer(df, data.map(_.toString).toDF("value")) + } + + def checkAllData(data: Seq[Int]): Unit = { + val schema = StructType(Seq(StructField("value", StringType))) + val df = spark.createDataFrame( + spark.sparkContext.makeRDD(memorySink.allData), schema) + checkAnswer(df, data.map(_.toString).toDF("value")) + } + + /** Check how many batches have executed since the last time this check was made */ + var lastBatchId = -1L + def checkNumBatchesSinceLastCheck(numBatches: Int): Unit = { + require(lastBatchId >= 0) + assert(memorySink.latestBatchId.get === lastBatchId + numBatches) + lastBatchId = memorySink.latestBatchId.get + } + + checkLastBatchData(3) // (1 and 2) should be in batch 1, (3) should be in batch 2 (last) + checkAllData(1 to 3) + lastBatchId = memorySink.latestBatchId.get + + fileSource.withBatchingLocked { + createFile(4) + createFile(5) // 4 and 5 should be in a batch + createFile(6) + createFile(7) // 6 and 7 should be in the last batch + } + q.processAllAvailable() + checkNumBatchesSinceLastCheck(2) + checkLastBatchData(6, 7) + checkAllData(1 to 7) + + fileSource.withBatchingLocked { + createFile(8) + createFile(9) // 8 and 9 should be in a batch + createFile(10) + createFile(11) // 10 and 11 should be in a batch + createFile(12) // 12 should be in the last batch + } + q.processAllAvailable() + checkNumBatchesSinceLastCheck(3) + checkLastBatchData(12) + checkAllData(1 to 12) + + q.stop() + } + } + + testQuietly("max files per trigger - incorrect values") { + val testTable = "maxFilesPerTrigger_test" + withTable(testTable) { + withTempDir { case src => + def testMaxFilePerTriggerValue(value: String): Unit = { + val df = spark.readStream.option("maxFilesPerTrigger", value).text(src.getCanonicalPath) + val e = intercept[StreamingQueryException] { + // Note: `maxFilesPerTrigger` is checked in the stream thread when creating the source + val q = df.writeStream.format("memory").queryName(testTable).start() + try { + q.processAllAvailable() + } finally { + q.stop() + } + } + assert(e.getCause.isInstanceOf[IllegalArgumentException]) + Seq("maxFilesPerTrigger", value, "positive integer").foreach { s => + assert(e.getMessage.contains(s)) + } + } + + testMaxFilePerTriggerValue("not-a-integer") + testMaxFilePerTriggerValue("-1") + testMaxFilePerTriggerValue("0") + testMaxFilePerTriggerValue("10.1") + } + } + } + + test("explain") { + withTempDirs { case (src, tmp) => + src.mkdirs() + + val df = spark.readStream.format("text").load(src.getCanonicalPath).map(_ + "-x") + // Test `explain` not throwing errors + df.explain() + + val q = df.writeStream.queryName("file_explain").format("memory").start() + .asInstanceOf[StreamingQueryWrapper] + .streamingQuery + try { + assert("No physical plan. Waiting for data." === q.explainInternal(false)) + assert("No physical plan. Waiting for data." === q.explainInternal(true)) + + val tempFile = Utils.tempFileWith(new File(tmp, "text")) + val finalFile = new File(src, tempFile.getName) + require(stringToFile(tempFile, "foo").renameTo(finalFile)) + + q.processAllAvailable() + + val explainWithoutExtended = q.explainInternal(false) + // `extended = false` only displays the physical plan. + assert("Relation.*text".r.findAllMatchIn(explainWithoutExtended).size === 0) + assert(": Text".r.findAllMatchIn(explainWithoutExtended).size === 1) + + val explainWithExtended = q.explainInternal(true) + // `extended = true` displays 3 logical plans (Parsed/Optimized/Optimized) and 1 physical + // plan. + assert("Relation.*text".r.findAllMatchIn(explainWithExtended).size === 3) + assert(": Text".r.findAllMatchIn(explainWithExtended).size === 1) + } finally { + q.stop() + } + } + } + + test("SPARK-17372 - write file names to WAL as Array[String]") { + // Note: If this test takes longer than the timeout, then its likely that this is actually + // running a Spark job with 10000 tasks. This test tries to avoid that by + // 1. Setting the threshold for parallel file listing to very high + // 2. Using a query that should use constant folding to eliminate reading of the files + + val numFiles = 10000 + + // This is to avoid running a spark job to list of files in parallel + // by the InMemoryFileIndex. + spark.sessionState.conf.setConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD, numFiles * 2) + + withTempDirs { case (root, tmp) => + val src = new File(root, "a=1") + src.mkdirs() + + (1 to numFiles).map { _.toString }.foreach { i => + val tempFile = Utils.tempFileWith(new File(tmp, "text")) + val finalFile = new File(src, tempFile.getName) + stringToFile(finalFile, i) + } + assert(src.listFiles().size === numFiles) + + val files = spark.readStream.text(root.getCanonicalPath).as[(String, Int)] + + // Note this query will use constant folding to eliminate the file scan. + // This is to avoid actually running a Spark job with 10000 tasks + val df = files.filter("1 == 0").groupBy().count() + + testStream(df, OutputMode.Complete)( + AddTextFileData("0", src, tmp), + CheckAnswer(0) + ) + } + } + + test("compact interval metadata log") { + val _sources = PrivateMethod[Seq[Source]]('sources) + val _metadataLog = PrivateMethod[FileStreamSourceLog]('metadataLog) + + def verify( + execution: StreamExecution, + batchId: Long, + expectedBatches: Int, + expectedCompactInterval: Int): Boolean = { + import CompactibleFileStreamLog._ + + val fileSource = (execution invokePrivate _sources()).head.asInstanceOf[FileStreamSource] + val metadataLog = fileSource invokePrivate _metadataLog() + + if (isCompactionBatch(batchId, expectedCompactInterval)) { + val path = metadataLog.batchIdToPath(batchId) + + // Assert path name should be ended with compact suffix. + assert(path.getName.endsWith(COMPACT_FILE_SUFFIX), + "path does not end with compact file suffix") + + // Compacted batch should include all entries from start. + val entries = metadataLog.get(batchId) + assert(entries.isDefined, "Entries not defined") + assert(entries.get.length === metadataLog.allFiles().length, "clean up check") + assert(metadataLog.get(None, Some(batchId)).flatMap(_._2).length === + entries.get.length, "Length check") + } + + assert(metadataLog.allFiles().sortBy(_.batchId) === + metadataLog.get(None, Some(batchId)).flatMap(_._2).sortBy(_.batchId), + "Batch id mismatch") + + metadataLog.get(None, Some(batchId)).flatMap(_._2).length === expectedBatches + } + + withTempDirs { case (src, tmp) => + withSQLConf( + SQLConf.FILE_SOURCE_LOG_COMPACT_INTERVAL.key -> "2" + ) { + val fileStream = createFileStream("text", src.getCanonicalPath) + val filtered = fileStream.filter($"value" contains "keep") + val updateConf = Map(SQLConf.FILE_SOURCE_LOG_COMPACT_INTERVAL.key -> "5") + + testStream(filtered)( + AddTextFileData("drop1\nkeep2\nkeep3", src, tmp), + CheckAnswer("keep2", "keep3"), + AssertOnQuery(verify(_, 0L, 1, 2)), + AddTextFileData("drop4\nkeep5\nkeep6", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6"), + AssertOnQuery(verify(_, 1L, 2, 2)), + AddTextFileData("drop7\nkeep8\nkeep9", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9"), + AssertOnQuery(verify(_, 2L, 3, 2)), + StopStream, + StartStream(additionalConfs = updateConf), + AssertOnQuery(verify(_, 2L, 3, 2)), + AddTextFileData("drop10\nkeep11", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9", "keep11"), + AssertOnQuery(verify(_, 3L, 4, 2)), + AddTextFileData("drop12\nkeep13", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9", "keep11", "keep13"), + AssertOnQuery(verify(_, 4L, 5, 2)) + ) + } + } + } + + test("get arbitrary batch from FileStreamSource") { + withTempDirs { case (src, tmp) => + withSQLConf( + SQLConf.FILE_SOURCE_LOG_COMPACT_INTERVAL.key -> "2", + // Force deleting the old logs + SQLConf.FILE_SOURCE_LOG_CLEANUP_DELAY.key -> "1" + ) { + val fileStream = createFileStream("text", src.getCanonicalPath) + val filtered = fileStream.filter($"value" contains "keep") + + testStream(filtered)( + AddTextFileData("keep1", src, tmp), + CheckAnswer("keep1"), + AddTextFileData("keep2", src, tmp), + CheckAnswer("keep1", "keep2"), + AddTextFileData("keep3", src, tmp), + CheckAnswer("keep1", "keep2", "keep3"), + AssertOnQuery("check getBatch") { execution: StreamExecution => + val _sources = PrivateMethod[Seq[Source]]('sources) + val fileSource = + (execution invokePrivate _sources()).head.asInstanceOf[FileStreamSource] + + def verify(startId: Option[Int], endId: Int, expected: String*): Unit = { + val start = startId.map(new FileStreamSourceOffset(_)) + val end = FileStreamSourceOffset(endId) + + withSQLConf("spark.sql.streaming.unsupportedOperationCheck" -> "false") { + assert(fileSource.getBatch(start, end).as[String].collect().toSeq === expected) + } + } + + verify(startId = None, endId = 2, "keep1", "keep2", "keep3") + verify(startId = Some(0), endId = 1, "keep2") + verify(startId = Some(0), endId = 2, "keep2", "keep3") + verify(startId = Some(1), endId = 2, "keep3") + true + } + ) + } + } + } + + test("input row metrics") { + withTempDirs { case (src, tmp) => + val input = spark.readStream.format("text").load(src.getCanonicalPath) + testStream(input)( + AddTextFileData("100", src, tmp), + CheckAnswer("100"), + AssertOnQuery { query => + val actualProgress = query.recentProgress + .find(_.numInputRows > 0) + .getOrElse(sys.error("Could not find records with data.")) + assert(actualProgress.numInputRows === 1) + assert(actualProgress.sources(0).processedRowsPerSecond > 0.0) + true + } + ) + } + } + + test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") { + val options = new FileStreamOptions(Map("maxfilespertrigger" -> "1")) + assert(options.maxFilesPerTrigger == Some(1)) + } + + test("FileStreamSource offset - read Spark 2.1.0 offset json format") { + val offset = readOffsetFromResource("file-source-offset-version-2.1.0-json.txt") + assert(FileStreamSourceOffset(offset) === FileStreamSourceOffset(345)) + } + + test("FileStreamSource offset - read Spark 2.1.0 offset long format") { + val offset = readOffsetFromResource("file-source-offset-version-2.1.0-long.txt") + assert(FileStreamSourceOffset(offset) === FileStreamSourceOffset(345)) + } + + test("FileStreamSourceLog - read Spark 2.1.0 log format") { + assert(readLogFromResource("file-source-log-version-2.1.0") === Seq( + FileEntry("/a/b/0", 1480730949000L, 0L), + FileEntry("/a/b/1", 1480730950000L, 1L), + FileEntry("/a/b/2", 1480730950000L, 2L), + FileEntry("/a/b/3", 1480730950000L, 3L), + FileEntry("/a/b/4", 1480730951000L, 4L) + )) + } + + private def readLogFromResource(dir: String): Seq[FileEntry] = { + val input = getClass.getResource(s"/structured-streaming/$dir") + val log = new FileStreamSourceLog(FileStreamSourceLog.VERSION, spark, input.toString) + log.allFiles() + } + + private def readOffsetFromResource(file: String): SerializedOffset = { + import scala.io.Source + val str = Source.fromFile(getClass.getResource(s"/structured-streaming/$file").toURI).mkString + SerializedOffset(str.trim) + } + + private def runTwoBatchesAndVerifyResults( + src: File, + latestFirst: Boolean, + firstBatch: String, + secondBatch: String, + maxFileAge: Option[String] = None): Unit = { + val srcOptions = Map("latestFirst" -> latestFirst.toString, "maxFilesPerTrigger" -> "1") ++ + maxFileAge.map("maxFileAge" -> _) + val fileStream = createFileStream( + "text", + src.getCanonicalPath, + options = srcOptions) + val clock = new StreamManualClock() + testStream(fileStream)( + StartStream(trigger = ProcessingTime(10), triggerClock = clock), + AssertOnQuery { _ => + // Block until the first batch finishes. + eventually(timeout(streamingTimeout)) { + assert(clock.isStreamWaitingAt(0)) + } + true + }, + CheckLastBatch(firstBatch), + AdvanceManualClock(10), + AssertOnQuery { _ => + // Block until the second batch finishes. + eventually(timeout(streamingTimeout)) { + assert(clock.isStreamWaitingAt(10)) + } + true + }, + CheckLastBatch(secondBatch) + ) + } + + test("FileStreamSource - latestFirst") { + withTempDir { src => + // Prepare two files: 1.txt, 2.txt, and make sure they have different modified time. + val f1 = stringToFile(new File(src, "1.txt"), "1") + val f2 = stringToFile(new File(src, "2.txt"), "2") + f2.setLastModified(f1.lastModified + 1000) + + // Read oldest files first, so the first batch is "1", and the second batch is "2". + runTwoBatchesAndVerifyResults(src, latestFirst = false, firstBatch = "1", secondBatch = "2") + + // Read latest files first, so the first batch is "2", and the second batch is "1". + runTwoBatchesAndVerifyResults(src, latestFirst = true, firstBatch = "2", secondBatch = "1") + } + } + + test("SPARK-19813: Ignore maxFileAge when maxFilesPerTrigger and latestFirst is used") { + withTempDir { src => + // Prepare two files: 1.txt, 2.txt, and make sure they have different modified time. + val f1 = stringToFile(new File(src, "1.txt"), "1") + val f2 = stringToFile(new File(src, "2.txt"), "2") + f2.setLastModified(f1.lastModified + 3600 * 1000 /* 1 hour later */) + + runTwoBatchesAndVerifyResults(src, latestFirst = true, firstBatch = "2", secondBatch = "1", + maxFileAge = Some("1m") /* 1 minute */) + } + } + + test("SeenFilesMap") { + val map = new SeenFilesMap(maxAgeMs = 10, fileNameOnly = false) + + map.add("a", 5) + assert(map.size == 1) + map.purge() + assert(map.size == 1) + + // Add a new entry and purge should be no-op, since the gap is exactly 10 ms. + map.add("b", 15) + assert(map.size == 2) + map.purge() + assert(map.size == 2) + + // Add a new entry that's more than 10 ms than the first entry. We should be able to purge now. + map.add("c", 16) + assert(map.size == 3) + map.purge() + assert(map.size == 2) + + // Override existing entry shouldn't change the size + map.add("c", 25) + assert(map.size == 2) + + // Not a new file because we have seen c before + assert(!map.isNewFile("c", 20)) + + // Not a new file because timestamp is too old + assert(!map.isNewFile("d", 5)) + + // Finally a new file: never seen and not too old + assert(map.isNewFile("e", 20)) + } + + test("SeenFilesMap with fileNameOnly = true") { + val map = new SeenFilesMap(maxAgeMs = 10, fileNameOnly = true) + + map.add("file:///a/b/c/d", 5) + map.add("file:///a/b/c/e", 5) + assert(map.size === 2) + + assert(!map.isNewFile("d", 5)) + assert(!map.isNewFile("file:///d", 5)) + assert(!map.isNewFile("file:///x/d", 5)) + assert(!map.isNewFile("file:///x/y/d", 5)) + + map.add("s3:///bucket/d", 5) + map.add("s3n:///bucket/d", 5) + map.add("s3a:///bucket/d", 5) + assert(map.size === 2) + } + + test("SeenFilesMap should only consider a file old if it is earlier than last purge time") { + val map = new SeenFilesMap(maxAgeMs = 10, fileNameOnly = false) + + map.add("a", 20) + assert(map.size == 1) + + // Timestamp 5 should still considered a new file because purge time should be 0 + assert(map.isNewFile("b", 9)) + assert(map.isNewFile("b", 10)) + + // Once purge, purge time should be 10 and then b would be a old file if it is less than 10. + map.purge() + assert(!map.isNewFile("b", 9)) + assert(map.isNewFile("b", 10)) + } + + test("do not recheck that files exist during getBatch") { + withTempDir { temp => + spark.conf.set( + s"fs.$scheme.impl", + classOf[ExistsThrowsExceptionFileSystem].getName) + // add the metadata entries as a pre-req + val dir = new File(temp, "dir") // use non-existent directory to test whether log make the dir + val metadataLog = + new FileStreamSourceLog(FileStreamSourceLog.VERSION, spark, dir.getAbsolutePath) + assert(metadataLog.add(0, Array(FileEntry(s"$scheme:///file1", 100L, 0)))) + assert(metadataLog.add(1, Array(FileEntry(s"$scheme:///file2", 200L, 0)))) + + val newSource = new FileStreamSource(spark, s"$scheme:///", "parquet", StructType(Nil), Nil, + dir.getAbsolutePath, Map.empty) + // this method should throw an exception if `fs.exists` is called during resolveRelation + newSource.getBatch(None, FileStreamSourceOffset(1)) + } + } +} + +class FileStreamSourceStressTestSuite extends FileStreamSourceTest { + + import testImplicits._ + + testQuietly("file source stress test") { + val src = Utils.createTempDir(namePrefix = "streaming.src") + val tmp = Utils.createTempDir(namePrefix = "streaming.tmp") + + val fileStream = createFileStream("text", src.getCanonicalPath) + val ds = fileStream.as[String].map(_.toInt + 1) + runStressTest(ds, data => { + AddTextFileData(data.mkString("\n"), src, tmp) + }) + + Utils.deleteRecursively(src) + Utils.deleteRecursively(tmp) + } +} + +/** + * Fake FileSystem to test whether the method `fs.exists` is called during + * `DataSource.resolveRelation`. + */ +class ExistsThrowsExceptionFileSystem extends RawLocalFileSystem { + override def getUri: URI = { + URI.create(s"$scheme:///") + } + + override def exists(f: Path): Boolean = { + throw new IllegalArgumentException("Exists shouldn't have been called!") + } + + /** Simply return an empty file for now. */ + override def listStatus(file: Path): Array[FileStatus] = { + val emptyFile = new FileStatus() + emptyFile.setPath(file) + Array(emptyFile) + } +} + +object ExistsThrowsExceptionFileSystem { + val scheme = s"FileStreamSourceSuite${math.abs(Random.nextInt)}fs" +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamStressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamStressSuite.scala new file mode 100644 index 000000000000..28412ea07a75 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamStressSuite.scala @@ -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.sql.streaming + +import java.io.File +import java.util.UUID + +import scala.util.Random +import scala.util.control.NonFatal + +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.util.Utils + +/** + * A stress test for streaming queries that read and write files. This test consists of + * two threads: + * - one that writes out `numRecords` distinct integers to files of random sizes (the total + * number of records is fixed but each files size / creation time is random). + * - another that continually restarts a buggy streaming query (i.e. fails with 5% probability on + * any partition). + * + * At the end, the resulting files are loaded and the answer is checked. + */ +class FileStreamStressSuite extends StreamTest { + import testImplicits._ + + // Error message thrown in the streaming job for testing recovery. + private val injectedErrorMsg = "test suite injected failure!" + + testQuietly("fault tolerance stress test - unpartitioned output") { + stressTest(partitionWrites = false) + } + + testQuietly("fault tolerance stress test - partitioned output") { + stressTest(partitionWrites = true) + } + + def stressTest(partitionWrites: Boolean): Unit = { + val numRecords = 10000 + val inputDir = Utils.createTempDir(namePrefix = "stream.input").getCanonicalPath + val stagingDir = Utils.createTempDir(namePrefix = "stream.staging").getCanonicalPath + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + val checkpoint = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + + @volatile + var continue = true + @volatile + var stream: StreamingQuery = null + + val writer = new Thread("stream writer") { + override def run(): Unit = { + var i = numRecords + while (i > 0) { + val count = Random.nextInt(100) + var j = 0 + var string = "" + while (j < count && i > 0) { + if (i % 10000 == 0) { logError(s"Wrote record $i") } + string = string + i + "\n" + j += 1 + i -= 1 + } + + val uuid = UUID.randomUUID().toString + val fileName = new File(stagingDir, uuid) + stringToFile(fileName, string) + fileName.renameTo(new File(inputDir, uuid)) + val sleep = Random.nextInt(100) + Thread.sleep(sleep) + } + + logError("== DONE WRITING ==") + var done = false + while (!done) { + try { + stream.processAllAvailable() + done = true + } catch { + case NonFatal(_) => + } + } + + continue = false + stream.stop() + } + } + writer.start() + + val input = spark.readStream.format("text").load(inputDir) + + def startStream(): StreamingQuery = { + val errorMsg = injectedErrorMsg // work around serialization issue + val output = input + .repartition(5) + .as[String] + .mapPartitions { iter => + val rand = Random.nextInt(100) + if (rand < 10) { + sys.error(errorMsg) + } + iter.map(_.toLong) + } + .map(x => (x % 400, x.toString)) + .toDF("id", "data") + + if (partitionWrites) { + output + .writeStream + .partitionBy("id") + .format("parquet") + .option("checkpointLocation", checkpoint) + .start(outputDir) + } else { + output + .writeStream + .format("parquet") + .option("checkpointLocation", checkpoint) + .start(outputDir) + } + } + + var failures = 0 + while (continue) { + if (failures % 10 == 0) { logError(s"Query restart #$failures") } + stream = startStream() + + try { + stream.awaitTermination() + } catch { + case e: StreamingQueryException + if e.getCause != null && e.getCause.getCause != null && + e.getCause.getCause.getMessage.contains(injectedErrorMsg) => + // Getting the expected error message + failures += 1 + } + } + + logError(s"Stream restarted $failures times.") + assert(spark.read.parquet(outputDir).distinct().count() == numRecords) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala new file mode 100644 index 000000000000..9d74a5c701ef --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala @@ -0,0 +1,1083 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.sql.Date +import java.util.concurrent.ConcurrentHashMap + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.SparkException +import org.apache.spark.api.java.function.FlatMapGroupsWithStateFunction +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.plans.logical.FlatMapGroupsWithState +import org.apache.spark.sql.catalyst.plans.physical.UnknownPartitioning +import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ +import org.apache.spark.sql.execution.RDDScanExec +import org.apache.spark.sql.execution.streaming.{FlatMapGroupsWithStateExec, GroupStateImpl, MemoryStream} +import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreId, StateStoreMetrics, UnsafeRowPair} +import org.apache.spark.sql.streaming.FlatMapGroupsWithStateSuite.MemoryStateStore +import org.apache.spark.sql.streaming.util.StreamManualClock +import org.apache.spark.sql.types.{DataType, IntegerType} + +/** Class to check custom state types */ +case class RunningCount(count: Long) + +case class Result(key: Long, count: Int) + +class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest with BeforeAndAfterAll { + + import testImplicits._ + import GroupStateImpl._ + import GroupStateTimeout._ + + override def afterAll(): Unit = { + super.afterAll() + StateStore.stop() + } + + test("GroupState - get, exists, update, remove") { + var state: GroupStateImpl[String] = null + + def testState( + expectedData: Option[String], + shouldBeUpdated: Boolean = false, + shouldBeRemoved: Boolean = false): Unit = { + if (expectedData.isDefined) { + assert(state.exists) + assert(state.get === expectedData.get) + } else { + assert(!state.exists) + intercept[NoSuchElementException] { + state.get + } + } + assert(state.getOption === expectedData) + assert(state.hasUpdated === shouldBeUpdated) + assert(state.hasRemoved === shouldBeRemoved) + } + + // === Tests for state in streaming queries === + // Updating empty state + state = GroupStateImpl.createForStreaming(None, 1, 1, NoTimeout, hasTimedOut = false) + testState(None) + state.update("") + testState(Some(""), shouldBeUpdated = true) + + // Updating exiting state + state = GroupStateImpl.createForStreaming(Some("2"), 1, 1, NoTimeout, hasTimedOut = false) + testState(Some("2")) + state.update("3") + testState(Some("3"), shouldBeUpdated = true) + + // Removing state + state.remove() + testState(None, shouldBeRemoved = true, shouldBeUpdated = false) + state.remove() // should be still callable + state.update("4") + testState(Some("4"), shouldBeRemoved = false, shouldBeUpdated = true) + + // Updating by null throw exception + intercept[IllegalArgumentException] { + state.update(null) + } + } + + test("GroupState - setTimeout - with NoTimeout") { + for (initValue <- Seq(None, Some(5))) { + val states = Seq( + GroupStateImpl.createForStreaming(initValue, 1000, 1000, NoTimeout, hasTimedOut = false), + GroupStateImpl.createForBatch(NoTimeout) + ) + for (state <- states) { + // for streaming queries + testTimeoutDurationNotAllowed[UnsupportedOperationException](state) + testTimeoutTimestampNotAllowed[UnsupportedOperationException](state) + + // for batch queries + testTimeoutDurationNotAllowed[UnsupportedOperationException](state) + testTimeoutTimestampNotAllowed[UnsupportedOperationException](state) + } + } + } + + test("GroupState - setTimeout - with ProcessingTimeTimeout") { + // for streaming queries + var state: GroupStateImpl[Int] = GroupStateImpl.createForStreaming( + None, 1000, 1000, ProcessingTimeTimeout, hasTimedOut = false) + assert(state.getTimeoutTimestamp === NO_TIMESTAMP) + state.setTimeoutDuration(500) + assert(state.getTimeoutTimestamp === 1500) // can be set without initializing state + testTimeoutTimestampNotAllowed[UnsupportedOperationException](state) + + state.update(5) + assert(state.getTimeoutTimestamp === 1500) // does not change + state.setTimeoutDuration(1000) + assert(state.getTimeoutTimestamp === 2000) + state.setTimeoutDuration("2 second") + assert(state.getTimeoutTimestamp === 3000) + testTimeoutTimestampNotAllowed[UnsupportedOperationException](state) + + state.remove() + assert(state.getTimeoutTimestamp === 3000) // does not change + state.setTimeoutDuration(500) // can still be set + assert(state.getTimeoutTimestamp === 1500) + testTimeoutTimestampNotAllowed[UnsupportedOperationException](state) + + // for batch queries + state = GroupStateImpl.createForBatch(ProcessingTimeTimeout).asInstanceOf[GroupStateImpl[Int]] + assert(state.getTimeoutTimestamp === NO_TIMESTAMP) + state.setTimeoutDuration(500) + testTimeoutTimestampNotAllowed[UnsupportedOperationException](state) + + state.update(5) + state.setTimeoutDuration(1000) + state.setTimeoutDuration("2 second") + testTimeoutTimestampNotAllowed[UnsupportedOperationException](state) + + state.remove() + state.setTimeoutDuration(500) + testTimeoutTimestampNotAllowed[UnsupportedOperationException](state) + } + + test("GroupState - setTimeout - with EventTimeTimeout") { + var state: GroupStateImpl[Int] = GroupStateImpl.createForStreaming( + None, 1000, 1000, EventTimeTimeout, false) + + assert(state.getTimeoutTimestamp === NO_TIMESTAMP) + testTimeoutDurationNotAllowed[UnsupportedOperationException](state) + state.setTimeoutTimestamp(5000) + assert(state.getTimeoutTimestamp === 5000) // can be set without initializing state + + state.update(5) + assert(state.getTimeoutTimestamp === 5000) // does not change + state.setTimeoutTimestamp(10000) + assert(state.getTimeoutTimestamp === 10000) + state.setTimeoutTimestamp(new Date(20000)) + assert(state.getTimeoutTimestamp === 20000) + testTimeoutDurationNotAllowed[UnsupportedOperationException](state) + + state.remove() + assert(state.getTimeoutTimestamp === 20000) + state.setTimeoutTimestamp(5000) + assert(state.getTimeoutTimestamp === 5000) // can be set after removing state + testTimeoutDurationNotAllowed[UnsupportedOperationException](state) + + // for batch queries + state = GroupStateImpl.createForBatch(EventTimeTimeout).asInstanceOf[GroupStateImpl[Int]] + assert(state.getTimeoutTimestamp === NO_TIMESTAMP) + testTimeoutDurationNotAllowed[UnsupportedOperationException](state) + state.setTimeoutTimestamp(5000) + + state.update(5) + state.setTimeoutTimestamp(10000) + state.setTimeoutTimestamp(new Date(20000)) + testTimeoutDurationNotAllowed[UnsupportedOperationException](state) + + state.remove() + state.setTimeoutTimestamp(5000) + testTimeoutDurationNotAllowed[UnsupportedOperationException](state) + } + + test("GroupState - illegal params to setTimeout") { + var state: GroupStateImpl[Int] = null + + // Test setTimeout****() with illegal values + def testIllegalTimeout(body: => Unit): Unit = { + intercept[IllegalArgumentException] { + body + } + assert(state.getTimeoutTimestamp === NO_TIMESTAMP) + } + + state = GroupStateImpl.createForStreaming( + Some(5), 1000, 1000, ProcessingTimeTimeout, hasTimedOut = false) + testIllegalTimeout { + state.setTimeoutDuration(-1000) + } + testIllegalTimeout { + state.setTimeoutDuration(0) + } + testIllegalTimeout { + state.setTimeoutDuration("-2 second") + } + testIllegalTimeout { + state.setTimeoutDuration("-1 month") + } + testIllegalTimeout { + state.setTimeoutDuration("1 month -1 day") + } + + state = GroupStateImpl.createForStreaming( + Some(5), 1000, 1000, EventTimeTimeout, hasTimedOut = false) + testIllegalTimeout { + state.setTimeoutTimestamp(-10000) + } + testIllegalTimeout { + state.setTimeoutTimestamp(10000, "-3 second") + } + testIllegalTimeout { + state.setTimeoutTimestamp(10000, "-1 month") + } + testIllegalTimeout { + state.setTimeoutTimestamp(10000, "1 month -1 day") + } + testIllegalTimeout { + state.setTimeoutTimestamp(new Date(-10000)) + } + testIllegalTimeout { + state.setTimeoutTimestamp(new Date(-10000), "-3 second") + } + testIllegalTimeout { + state.setTimeoutTimestamp(new Date(-10000), "-1 month") + } + testIllegalTimeout { + state.setTimeoutTimestamp(new Date(-10000), "1 month -1 day") + } + } + + test("GroupState - hasTimedOut") { + for (timeoutConf <- Seq(NoTimeout, ProcessingTimeTimeout, EventTimeTimeout)) { + // for streaming queries + for (initState <- Seq(None, Some(5))) { + val state1 = GroupStateImpl.createForStreaming( + initState, 1000, 1000, timeoutConf, hasTimedOut = false) + assert(state1.hasTimedOut === false) + + val state2 = GroupStateImpl.createForStreaming( + initState, 1000, 1000, timeoutConf, hasTimedOut = true) + assert(state2.hasTimedOut === true) + } + + // for batch queries + assert(GroupStateImpl.createForBatch(timeoutConf).hasTimedOut === false) + } + } + + test("GroupState - primitive type") { + var intState = GroupStateImpl.createForStreaming[Int]( + None, 1000, 1000, NoTimeout, hasTimedOut = false) + intercept[NoSuchElementException] { + intState.get + } + assert(intState.getOption === None) + + intState = GroupStateImpl.createForStreaming[Int]( + Some(10), 1000, 1000, NoTimeout, hasTimedOut = false) + assert(intState.get == 10) + intState.update(0) + assert(intState.get == 0) + intState.remove() + intercept[NoSuchElementException] { + intState.get + } + } + + // Values used for testing StateStoreUpdater + val currentBatchTimestamp = 1000 + val currentBatchWatermark = 1000 + val beforeTimeoutThreshold = 999 + val afterTimeoutThreshold = 1001 + + // Tests for StateStoreUpdater.updateStateForKeysWithData() when timeout = NoTimeout + for (priorState <- Seq(None, Some(0))) { + val priorStateStr = if (priorState.nonEmpty) "prior state set" else "no prior state" + val testName = s"NoTimeout - $priorStateStr - " + + testStateUpdateWithData( + testName + "no update", + stateUpdates = state => { /* do nothing */ }, + timeoutConf = GroupStateTimeout.NoTimeout, + priorState = priorState, + expectedState = priorState) // should not change + + testStateUpdateWithData( + testName + "state updated", + stateUpdates = state => { state.update(5) }, + timeoutConf = GroupStateTimeout.NoTimeout, + priorState = priorState, + expectedState = Some(5)) // should change + + testStateUpdateWithData( + testName + "state removed", + stateUpdates = state => { state.remove() }, + timeoutConf = GroupStateTimeout.NoTimeout, + priorState = priorState, + expectedState = None) // should be removed + } + + // Tests for StateStoreUpdater.updateStateForKeysWithData() when timeout != NoTimeout + for (priorState <- Seq(None, Some(0))) { + for (priorTimeoutTimestamp <- Seq(NO_TIMESTAMP, 1000)) { + var testName = "" + if (priorState.nonEmpty) { + testName += "prior state set, " + if (priorTimeoutTimestamp == 1000) { + testName += "prior timeout set" + } else { + testName += "no prior timeout" + } + } else { + testName += "no prior state" + } + for (timeoutConf <- Seq(ProcessingTimeTimeout, EventTimeTimeout)) { + + testStateUpdateWithData( + s"$timeoutConf - $testName - no update", + stateUpdates = state => { /* do nothing */ }, + timeoutConf = timeoutConf, + priorState = priorState, + priorTimeoutTimestamp = priorTimeoutTimestamp, + expectedState = priorState, // state should not change + expectedTimeoutTimestamp = NO_TIMESTAMP) // timestamp should be reset + + testStateUpdateWithData( + s"$timeoutConf - $testName - state updated", + stateUpdates = state => { state.update(5) }, + timeoutConf = timeoutConf, + priorState = priorState, + priorTimeoutTimestamp = priorTimeoutTimestamp, + expectedState = Some(5), // state should change + expectedTimeoutTimestamp = NO_TIMESTAMP) // timestamp should be reset + + testStateUpdateWithData( + s"$timeoutConf - $testName - state removed", + stateUpdates = state => { state.remove() }, + timeoutConf = timeoutConf, + priorState = priorState, + priorTimeoutTimestamp = priorTimeoutTimestamp, + expectedState = None) // state should be removed + } + + testStateUpdateWithData( + s"ProcessingTimeTimeout - $testName - state and timeout duration updated", + stateUpdates = + (state: GroupState[Int]) => { state.update(5); state.setTimeoutDuration(5000) }, + timeoutConf = ProcessingTimeTimeout, + priorState = priorState, + priorTimeoutTimestamp = priorTimeoutTimestamp, + expectedState = Some(5), // state should change + expectedTimeoutTimestamp = currentBatchTimestamp + 5000) // timestamp should change + + testStateUpdateWithData( + s"EventTimeTimeout - $testName - state and timeout timestamp updated", + stateUpdates = + (state: GroupState[Int]) => { state.update(5); state.setTimeoutTimestamp(5000) }, + timeoutConf = EventTimeTimeout, + priorState = priorState, + priorTimeoutTimestamp = priorTimeoutTimestamp, + expectedState = Some(5), // state should change + expectedTimeoutTimestamp = 5000) // timestamp should change + + testStateUpdateWithData( + s"EventTimeTimeout - $testName - timeout timestamp updated to before watermark", + stateUpdates = + (state: GroupState[Int]) => { + state.update(5) + intercept[IllegalArgumentException] { + state.setTimeoutTimestamp(currentBatchWatermark - 1) // try to set to < watermark + } + }, + timeoutConf = EventTimeTimeout, + priorState = priorState, + priorTimeoutTimestamp = priorTimeoutTimestamp, + expectedState = Some(5), // state should change + expectedTimeoutTimestamp = NO_TIMESTAMP) // timestamp should not update + } + } + + // Currently disallowed cases for StateStoreUpdater.updateStateForKeysWithData(), + // Try to remove these cases in the future + for (priorTimeoutTimestamp <- Seq(NO_TIMESTAMP, 1000)) { + val testName = + if (priorTimeoutTimestamp != NO_TIMESTAMP) "prior timeout set" else "no prior timeout" + testStateUpdateWithData( + s"ProcessingTimeTimeout - $testName - setting timeout without init state not allowed", + stateUpdates = state => { state.setTimeoutDuration(5000) }, + timeoutConf = ProcessingTimeTimeout, + priorState = None, + priorTimeoutTimestamp = priorTimeoutTimestamp, + expectedException = classOf[IllegalStateException]) + + testStateUpdateWithData( + s"ProcessingTimeTimeout - $testName - setting timeout with state removal not allowed", + stateUpdates = state => { state.remove(); state.setTimeoutDuration(5000) }, + timeoutConf = ProcessingTimeTimeout, + priorState = Some(5), + priorTimeoutTimestamp = priorTimeoutTimestamp, + expectedException = classOf[IllegalStateException]) + + testStateUpdateWithData( + s"EventTimeTimeout - $testName - setting timeout without init state not allowed", + stateUpdates = state => { state.setTimeoutTimestamp(10000) }, + timeoutConf = EventTimeTimeout, + priorState = None, + priorTimeoutTimestamp = priorTimeoutTimestamp, + expectedException = classOf[IllegalStateException]) + + testStateUpdateWithData( + s"EventTimeTimeout - $testName - setting timeout with state removal not allowed", + stateUpdates = state => { state.remove(); state.setTimeoutTimestamp(10000) }, + timeoutConf = EventTimeTimeout, + priorState = Some(5), + priorTimeoutTimestamp = priorTimeoutTimestamp, + expectedException = classOf[IllegalStateException]) + } + + // Tests for StateStoreUpdater.updateStateForTimedOutKeys() + val preTimeoutState = Some(5) + for (timeoutConf <- Seq(ProcessingTimeTimeout, EventTimeTimeout)) { + testStateUpdateWithTimeout( + s"$timeoutConf - should not timeout", + stateUpdates = state => { assert(false, "function called without timeout") }, + timeoutConf = timeoutConf, + priorTimeoutTimestamp = afterTimeoutThreshold, + expectedState = preTimeoutState, // state should not change + expectedTimeoutTimestamp = afterTimeoutThreshold) // timestamp should not change + + testStateUpdateWithTimeout( + s"$timeoutConf - should timeout - no update/remove", + stateUpdates = state => { /* do nothing */ }, + timeoutConf = timeoutConf, + priorTimeoutTimestamp = beforeTimeoutThreshold, + expectedState = preTimeoutState, // state should not change + expectedTimeoutTimestamp = NO_TIMESTAMP) // timestamp should be reset + + testStateUpdateWithTimeout( + s"$timeoutConf - should timeout - update state", + stateUpdates = state => { state.update(5) }, + timeoutConf = timeoutConf, + priorTimeoutTimestamp = beforeTimeoutThreshold, + expectedState = Some(5), // state should change + expectedTimeoutTimestamp = NO_TIMESTAMP) // timestamp should be reset + + testStateUpdateWithTimeout( + s"$timeoutConf - should timeout - remove state", + stateUpdates = state => { state.remove() }, + timeoutConf = timeoutConf, + priorTimeoutTimestamp = beforeTimeoutThreshold, + expectedState = None, // state should be removed + expectedTimeoutTimestamp = NO_TIMESTAMP) + } + + testStateUpdateWithTimeout( + "ProcessingTimeTimeout - should timeout - timeout duration updated", + stateUpdates = state => { state.setTimeoutDuration(2000) }, + timeoutConf = ProcessingTimeTimeout, + priorTimeoutTimestamp = beforeTimeoutThreshold, + expectedState = preTimeoutState, // state should not change + expectedTimeoutTimestamp = currentBatchTimestamp + 2000) // timestamp should change + + testStateUpdateWithTimeout( + "ProcessingTimeTimeout - should timeout - timeout duration and state updated", + stateUpdates = state => { state.update(5); state.setTimeoutDuration(2000) }, + timeoutConf = ProcessingTimeTimeout, + priorTimeoutTimestamp = beforeTimeoutThreshold, + expectedState = Some(5), // state should change + expectedTimeoutTimestamp = currentBatchTimestamp + 2000) // timestamp should change + + testStateUpdateWithTimeout( + "EventTimeTimeout - should timeout - timeout timestamp updated", + stateUpdates = state => { state.setTimeoutTimestamp(5000) }, + timeoutConf = EventTimeTimeout, + priorTimeoutTimestamp = beforeTimeoutThreshold, + expectedState = preTimeoutState, // state should not change + expectedTimeoutTimestamp = 5000) // timestamp should change + + testStateUpdateWithTimeout( + "EventTimeTimeout - should timeout - timeout and state updated", + stateUpdates = state => { state.update(5); state.setTimeoutTimestamp(5000) }, + timeoutConf = EventTimeTimeout, + priorTimeoutTimestamp = beforeTimeoutThreshold, + expectedState = Some(5), // state should change + expectedTimeoutTimestamp = 5000) // timestamp should change + + test("flatMapGroupsWithState - streaming") { + // Function to maintain running count up to 2, and then remove the count + // Returns the data and the count if state is defined, otherwise does not return anything + val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => { + + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + if (count == 3) { + state.remove() + Iterator.empty + } else { + state.update(RunningCount(count)) + Iterator((key, count.toString)) + } + } + + val inputData = MemoryStream[String] + val result = + inputData.toDS() + .groupByKey(x => x) + .flatMapGroupsWithState(Update, GroupStateTimeout.NoTimeout)(stateFunc) + + testStream(result, Update)( + AddData(inputData, "a"), + CheckLastBatch(("a", "1")), + assertNumStateRows(total = 1, updated = 1), + AddData(inputData, "a", "b"), + CheckLastBatch(("a", "2"), ("b", "1")), + assertNumStateRows(total = 2, updated = 2), + StopStream, + StartStream(), + AddData(inputData, "a", "b"), // should remove state for "a" and not return anything for a + CheckLastBatch(("b", "2")), + assertNumStateRows(total = 1, updated = 2), + StopStream, + StartStream(), + AddData(inputData, "a", "c"), // should recreate state for "a" and return count as 1 and + CheckLastBatch(("a", "1"), ("c", "1")), + assertNumStateRows(total = 3, updated = 2) + ) + } + + test("flatMapGroupsWithState - streaming + func returns iterator that updates state lazily") { + // Function to maintain running count up to 2, and then remove the count + // Returns the data and the count if state is defined, otherwise does not return anything + // Additionally, it updates state lazily as the returned iterator get consumed + val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => { + values.flatMap { _ => + val count = state.getOption.map(_.count).getOrElse(0L) + 1 + if (count == 3) { + state.remove() + None + } else { + state.update(RunningCount(count)) + Some((key, count.toString)) + } + } + } + + val inputData = MemoryStream[String] + val result = + inputData.toDS() + .groupByKey(x => x) + .flatMapGroupsWithState(Update, GroupStateTimeout.NoTimeout)(stateFunc) + testStream(result, Update)( + AddData(inputData, "a", "a", "b"), + CheckLastBatch(("a", "1"), ("a", "2"), ("b", "1")), + StopStream, + StartStream(), + AddData(inputData, "a", "b"), // should remove state for "a" and not return anything for a + CheckLastBatch(("b", "2")), + StopStream, + StartStream(), + AddData(inputData, "a", "c"), // should recreate state for "a" and return count as 1 and + CheckLastBatch(("a", "1"), ("c", "1")) + ) + } + + test("flatMapGroupsWithState - streaming + aggregation") { + // Function to maintain running count up to 2, and then remove the count + // Returns the data and the count (-1 if count reached beyond 2 and state was just removed) + val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => { + + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + if (count == 3) { + state.remove() + Iterator(key -> "-1") + } else { + state.update(RunningCount(count)) + Iterator(key -> count.toString) + } + } + + val inputData = MemoryStream[String] + val result = + inputData.toDS() + .groupByKey(x => x) + .flatMapGroupsWithState(Append, GroupStateTimeout.NoTimeout)(stateFunc) + .groupByKey(_._1) + .count() + + testStream(result, Complete)( + AddData(inputData, "a"), + CheckLastBatch(("a", 1)), + AddData(inputData, "a", "b"), + // mapGroups generates ("a", "2"), ("b", "1"); so increases counts of a and b by 1 + CheckLastBatch(("a", 2), ("b", 1)), + StopStream, + StartStream(), + AddData(inputData, "a", "b"), + // mapGroups should remove state for "a" and generate ("a", "-1"), ("b", "2") ; + // so increment a and b by 1 + CheckLastBatch(("a", 3), ("b", 2)), + StopStream, + StartStream(), + AddData(inputData, "a", "c"), + // mapGroups should recreate state for "a" and generate ("a", "1"), ("c", "1") ; + // so increment a and c by 1 + CheckLastBatch(("a", 4), ("b", 2), ("c", 1)) + ) + } + + test("flatMapGroupsWithState - batch") { + // Function that returns running count only if its even, otherwise does not return + val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => { + if (state.exists) throw new IllegalArgumentException("state.exists should be false") + Iterator((key, values.size)) + } + val df = Seq("a", "a", "b").toDS + .groupByKey(x => x) + .flatMapGroupsWithState(Update, GroupStateTimeout.NoTimeout)(stateFunc).toDF + checkAnswer(df, Seq(("a", 2), ("b", 1)).toDF) + } + + test("flatMapGroupsWithState - streaming with processing time timeout") { + // Function to maintain running count up to 2, and then remove the count + // Returns the data and the count (-1 if count reached beyond 2 and state was just removed) + val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => { + if (state.hasTimedOut) { + state.remove() + Iterator((key, "-1")) + } else { + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + state.update(RunningCount(count)) + state.setTimeoutDuration("10 seconds") + Iterator((key, count.toString)) + } + } + + val clock = new StreamManualClock + val inputData = MemoryStream[String] + val result = + inputData.toDS() + .groupByKey(x => x) + .flatMapGroupsWithState(Update, ProcessingTimeTimeout)(stateFunc) + + testStream(result, Update)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock), + AddData(inputData, "a"), + AdvanceManualClock(1 * 1000), + CheckLastBatch(("a", "1")), + assertNumStateRows(total = 1, updated = 1), + + AddData(inputData, "b"), + AdvanceManualClock(1 * 1000), + CheckLastBatch(("b", "1")), + assertNumStateRows(total = 2, updated = 1), + + AddData(inputData, "b"), + AdvanceManualClock(10 * 1000), + CheckLastBatch(("a", "-1"), ("b", "2")), + assertNumStateRows(total = 1, updated = 2), + + StopStream, + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock), + + AddData(inputData, "c"), + AdvanceManualClock(11 * 1000), + CheckLastBatch(("b", "-1"), ("c", "1")), + assertNumStateRows(total = 1, updated = 2), + + AddData(inputData, "c"), + AdvanceManualClock(20 * 1000), + CheckLastBatch(("c", "2")), + assertNumStateRows(total = 1, updated = 1) + ) + } + + test("flatMapGroupsWithState - streaming with event time timeout + watermark") { + // Function to maintain the max event time + // Returns the max event time in the state, or -1 if the state was removed by timeout + val stateFunc = ( + key: String, + values: Iterator[(String, Long)], + state: GroupState[Long]) => { + val timeoutDelay = 5 + if (key != "a") { + Iterator.empty + } else { + if (state.hasTimedOut) { + state.remove() + Iterator((key, -1)) + } else { + val valuesSeq = values.toSeq + val maxEventTime = math.max(valuesSeq.map(_._2).max, state.getOption.getOrElse(0L)) + val timeoutTimestampMs = maxEventTime + timeoutDelay + state.update(maxEventTime) + state.setTimeoutTimestamp(timeoutTimestampMs * 1000) + Iterator((key, maxEventTime.toInt)) + } + } + } + val inputData = MemoryStream[(String, Int)] + val result = + inputData.toDS + .select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime")) + .withWatermark("eventTime", "10 seconds") + .as[(String, Long)] + .groupByKey(_._1) + .flatMapGroupsWithState(Update, EventTimeTimeout)(stateFunc) + + testStream(result, Update)( + StartStream(Trigger.ProcessingTime("1 second")), + AddData(inputData, ("a", 11), ("a", 13), ("a", 15)), // Set timeout timestamp of ... + CheckLastBatch(("a", 15)), // "a" to 15 + 5 = 20s, watermark to 5s + AddData(inputData, ("a", 4)), // Add data older than watermark for "a" + CheckLastBatch(), // No output as data should get filtered by watermark + AddData(inputData, ("dummy", 35)), // Set watermark = 35 - 10 = 25s + CheckLastBatch(), // No output as no data for "a" + AddData(inputData, ("a", 24)), // Add data older than watermark, should be ignored + CheckLastBatch(("a", -1)) // State for "a" should timeout and emit -1 + ) + } + + test("mapGroupsWithState - streaming") { + // Function to maintain running count up to 2, and then remove the count + // Returns the data and the count (-1 if count reached beyond 2 and state was just removed) + val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => { + + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + if (count == 3) { + state.remove() + (key, "-1") + } else { + state.update(RunningCount(count)) + (key, count.toString) + } + } + + val inputData = MemoryStream[String] + val result = + inputData.toDS() + .groupByKey(x => x) + .mapGroupsWithState(stateFunc) // Types = State: MyState, Out: (Str, Str) + + testStream(result, Update)( + AddData(inputData, "a"), + CheckLastBatch(("a", "1")), + assertNumStateRows(total = 1, updated = 1), + AddData(inputData, "a", "b"), + CheckLastBatch(("a", "2"), ("b", "1")), + assertNumStateRows(total = 2, updated = 2), + StopStream, + StartStream(), + AddData(inputData, "a", "b"), // should remove state for "a" and return count as -1 + CheckLastBatch(("a", "-1"), ("b", "2")), + assertNumStateRows(total = 1, updated = 2), + StopStream, + StartStream(), + AddData(inputData, "a", "c"), // should recreate state for "a" and return count as 1 + CheckLastBatch(("a", "1"), ("c", "1")), + assertNumStateRows(total = 3, updated = 2) + ) + } + + test("mapGroupsWithState - batch") { + // Test the following + // - no initial state + // - timeouts operations work, does not throw any error [SPARK-20792] + // - works with primitive state type + val stateFunc = (key: String, values: Iterator[String], state: GroupState[Int]) => { + if (state.exists) throw new IllegalArgumentException("state.exists should be false") + state.setTimeoutTimestamp(0, "1 hour") + state.update(10) + (key, values.size) + } + + checkAnswer( + spark.createDataset(Seq("a", "a", "b")) + .groupByKey(x => x) + .mapGroupsWithState(EventTimeTimeout)(stateFunc) + .toDF, + spark.createDataset(Seq(("a", 2), ("b", 1))).toDF) + } + + testQuietly("StateStore.abort on task failure handling") { + val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => { + if (FlatMapGroupsWithStateSuite.failInTask) throw new Exception("expected failure") + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + state.update(RunningCount(count)) + (key, count) + } + + val inputData = MemoryStream[String] + val result = + inputData.toDS() + .groupByKey(x => x) + .mapGroupsWithState(stateFunc) // Types = State: MyState, Out: (Str, Str) + + def setFailInTask(value: Boolean): AssertOnQuery = AssertOnQuery { q => + FlatMapGroupsWithStateSuite.failInTask = value + true + } + + testStream(result, Update)( + setFailInTask(false), + AddData(inputData, "a"), + CheckLastBatch(("a", 1L)), + AddData(inputData, "a"), + CheckLastBatch(("a", 2L)), + setFailInTask(true), + AddData(inputData, "a"), + ExpectFailure[SparkException](), // task should fail but should not increment count + setFailInTask(false), + StartStream(), + CheckLastBatch(("a", 3L)) // task should not fail, and should show correct count + ) + } + + test("output partitioning is unknown") { + val stateFunc = (key: String, values: Iterator[String], state: GroupState[RunningCount]) => key + val inputData = MemoryStream[String] + val result = inputData.toDS.groupByKey(x => x).mapGroupsWithState(stateFunc) + testStream(result, Update)( + AddData(inputData, "a"), + CheckLastBatch("a"), + AssertOnQuery(_.lastExecution.executedPlan.outputPartitioning === UnknownPartitioning(0)) + ) + } + + test("disallow complete mode") { + val stateFunc = (key: String, values: Iterator[String], state: GroupState[Int]) => { + Iterator[String]() + } + + var e = intercept[IllegalArgumentException] { + MemoryStream[String].toDS().groupByKey(x => x).flatMapGroupsWithState( + OutputMode.Complete, GroupStateTimeout.NoTimeout)(stateFunc) + } + assert(e.getMessage === "The output mode of function should be append or update") + + val javaStateFunc = new FlatMapGroupsWithStateFunction[String, String, Int, String] { + import java.util.{Iterator => JIterator} + override def call( + key: String, + values: JIterator[String], + state: GroupState[Int]): JIterator[String] = { null } + } + e = intercept[IllegalArgumentException] { + MemoryStream[String].toDS().groupByKey(x => x).flatMapGroupsWithState( + javaStateFunc, OutputMode.Complete, + implicitly[Encoder[Int]], implicitly[Encoder[String]], GroupStateTimeout.NoTimeout) + } + assert(e.getMessage === "The output mode of function should be append or update") + } + + def testWithTimeout(timeoutConf: GroupStateTimeout): Unit = { + test("SPARK-20714: watermark does not fail query when timeout = " + timeoutConf) { + // Function to maintain running count up to 2, and then remove the count + // Returns the data and the count (-1 if count reached beyond 2 and state was just removed) + val stateFunc = + (key: String, values: Iterator[(String, Long)], state: GroupState[RunningCount]) => { + if (state.hasTimedOut) { + state.remove() + Iterator((key, "-1")) + } else { + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + state.update(RunningCount(count)) + state.setTimeoutDuration("10 seconds") + Iterator((key, count.toString)) + } + } + + val clock = new StreamManualClock + val inputData = MemoryStream[(String, Long)] + val result = + inputData.toDF().toDF("key", "time") + .selectExpr("key", "cast(time as timestamp) as timestamp") + .withWatermark("timestamp", "10 second") + .as[(String, Long)] + .groupByKey(x => x._1) + .flatMapGroupsWithState(Update, ProcessingTimeTimeout)(stateFunc) + + testStream(result, Update)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock), + AddData(inputData, ("a", 1L)), + AdvanceManualClock(1 * 1000), + CheckLastBatch(("a", "1")) + ) + } + } + testWithTimeout(NoTimeout) + testWithTimeout(ProcessingTimeTimeout) + + def testStateUpdateWithData( + testName: String, + stateUpdates: GroupState[Int] => Unit, + timeoutConf: GroupStateTimeout, + priorState: Option[Int], + priorTimeoutTimestamp: Long = NO_TIMESTAMP, + expectedState: Option[Int] = None, + expectedTimeoutTimestamp: Long = NO_TIMESTAMP, + expectedException: Class[_ <: Exception] = null): Unit = { + + if (priorState.isEmpty && priorTimeoutTimestamp != NO_TIMESTAMP) { + return // there can be no prior timestamp, when there is no prior state + } + test(s"StateStoreUpdater - updates with data - $testName") { + val mapGroupsFunc = (key: Int, values: Iterator[Int], state: GroupState[Int]) => { + assert(state.hasTimedOut === false, "hasTimedOut not false") + assert(values.nonEmpty, "Some value is expected") + stateUpdates(state) + Iterator.empty + } + testStateUpdate( + testTimeoutUpdates = false, mapGroupsFunc, timeoutConf, + priorState, priorTimeoutTimestamp, + expectedState, expectedTimeoutTimestamp, expectedException) + } + } + + def testStateUpdateWithTimeout( + testName: String, + stateUpdates: GroupState[Int] => Unit, + timeoutConf: GroupStateTimeout, + priorTimeoutTimestamp: Long, + expectedState: Option[Int], + expectedTimeoutTimestamp: Long = NO_TIMESTAMP): Unit = { + + test(s"StateStoreUpdater - updates for timeout - $testName") { + val mapGroupsFunc = (key: Int, values: Iterator[Int], state: GroupState[Int]) => { + assert(state.hasTimedOut === true, "hasTimedOut not true") + assert(values.isEmpty, "values not empty") + stateUpdates(state) + Iterator.empty + } + + testStateUpdate( + testTimeoutUpdates = true, mapGroupsFunc, timeoutConf = timeoutConf, + preTimeoutState, priorTimeoutTimestamp, expectedState, expectedTimeoutTimestamp, null) + } + } + + def testStateUpdate( + testTimeoutUpdates: Boolean, + mapGroupsFunc: (Int, Iterator[Int], GroupState[Int]) => Iterator[Int], + timeoutConf: GroupStateTimeout, + priorState: Option[Int], + priorTimeoutTimestamp: Long, + expectedState: Option[Int], + expectedTimeoutTimestamp: Long, + expectedException: Class[_ <: Exception]): Unit = { + + val store = newStateStore() + val mapGroupsSparkPlan = newFlatMapGroupsWithStateExec( + mapGroupsFunc, timeoutConf, currentBatchTimestamp) + val updater = new mapGroupsSparkPlan.StateStoreUpdater(store) + val key = intToRow(0) + // Prepare store with prior state configs + if (priorState.nonEmpty) { + val row = updater.getStateRow(priorState.get) + updater.setTimeoutTimestamp(row, priorTimeoutTimestamp) + store.put(key.copy(), row.copy()) + } + + // Call updating function to update state store + def callFunction() = { + val returnedIter = if (testTimeoutUpdates) { + updater.updateStateForTimedOutKeys() + } else { + updater.updateStateForKeysWithData(Iterator(key)) + } + returnedIter.size // consume the iterator to force state updates + } + if (expectedException != null) { + // Call function and verify the exception type + val e = intercept[Exception] { callFunction() } + assert(e.getClass === expectedException, "Exception thrown but of the wrong type") + } else { + // Call function to update and verify updated state in store + callFunction() + val updatedStateRow = store.get(key) + assert( + Option(updater.getStateObj(updatedStateRow)).map(_.toString.toInt) === expectedState, + "final state not as expected") + if (updatedStateRow != null) { + assert( + updater.getTimeoutTimestamp(updatedStateRow) === expectedTimeoutTimestamp, + "final timeout timestamp not as expected") + } + } + } + + def newFlatMapGroupsWithStateExec( + func: (Int, Iterator[Int], GroupState[Int]) => Iterator[Int], + timeoutType: GroupStateTimeout = GroupStateTimeout.NoTimeout, + batchTimestampMs: Long = NO_TIMESTAMP): FlatMapGroupsWithStateExec = { + MemoryStream[Int] + .toDS + .groupByKey(x => x) + .flatMapGroupsWithState[Int, Int](Append, timeoutConf = timeoutType)(func) + .logicalPlan.collectFirst { + case FlatMapGroupsWithState(f, k, v, g, d, o, s, m, _, t, _) => + FlatMapGroupsWithStateExec( + f, k, v, g, d, o, None, s, m, t, + Some(currentBatchTimestamp), Some(currentBatchWatermark), RDDScanExec(g, null, "rdd")) + }.get + } + + def testTimeoutDurationNotAllowed[T <: Exception: Manifest](state: GroupStateImpl[_]): Unit = { + val prevTimestamp = state.getTimeoutTimestamp + intercept[T] { state.setTimeoutDuration(1000) } + assert(state.getTimeoutTimestamp === prevTimestamp) + intercept[T] { state.setTimeoutDuration("2 second") } + assert(state.getTimeoutTimestamp === prevTimestamp) + } + + def testTimeoutTimestampNotAllowed[T <: Exception: Manifest](state: GroupStateImpl[_]): Unit = { + val prevTimestamp = state.getTimeoutTimestamp + intercept[T] { state.setTimeoutTimestamp(2000) } + assert(state.getTimeoutTimestamp === prevTimestamp) + intercept[T] { state.setTimeoutTimestamp(2000, "1 second") } + assert(state.getTimeoutTimestamp === prevTimestamp) + intercept[T] { state.setTimeoutTimestamp(new Date(2000)) } + assert(state.getTimeoutTimestamp === prevTimestamp) + intercept[T] { state.setTimeoutTimestamp(new Date(2000), "1 second") } + assert(state.getTimeoutTimestamp === prevTimestamp) + } + + def newStateStore(): StateStore = new MemoryStateStore() + + val intProj = UnsafeProjection.create(Array[DataType](IntegerType)) + def intToRow(i: Int): UnsafeRow = { + intProj.apply(new GenericInternalRow(Array[Any](i))).copy() + } + + def rowToInt(row: UnsafeRow): Int = row.getInt(0) +} + +object FlatMapGroupsWithStateSuite { + + var failInTask = true + + class MemoryStateStore extends StateStore() { + import scala.collection.JavaConverters._ + private val map = new ConcurrentHashMap[UnsafeRow, UnsafeRow] + + override def iterator(): Iterator[UnsafeRowPair] = { + map.entrySet.iterator.asScala.map { case e => new UnsafeRowPair(e.getKey, e.getValue) } + } + + override def get(key: UnsafeRow): UnsafeRow = map.get(key) + override def put(key: UnsafeRow, newValue: UnsafeRow): Unit = { + map.put(key.copy(), newValue.copy()) + } + override def remove(key: UnsafeRow): Unit = { map.remove(key) } + override def commit(): Long = version + 1 + override def abort(): Unit = { } + override def id: StateStoreId = null + override def version: Long = 0 + override def metrics: StateStoreMetrics = new StateStoreMetrics(map.size, 0, Map.empty) + override def hasCommitted: Boolean = true + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala new file mode 100644 index 000000000000..7f2972edea72 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming + +import org.apache.spark.sql.execution.streaming._ + +class MemorySourceStressSuite extends StreamTest { + import testImplicits._ + + test("memory stress test") { + val input = MemoryStream[Int] + val mapped = input.toDS().map(_ + 1) + + runStressTest(mapped, AddData(input, _: _*)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala new file mode 100644 index 000000000000..f208f9bd9b6e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala @@ -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.streaming + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.execution.streaming.{LongOffset, Offset, SerializedOffset} + +trait OffsetSuite extends SparkFunSuite { + /** Creates test to check all the comparisons of offsets given a `one` that is less than `two`. */ + def compare(one: Offset, two: Offset): Unit = { + test(s"comparison $one <=> $two") { + assert(one == one) + assert(two == two) + assert(one != two) + assert(two != one) + } + } +} + +class LongOffsetSuite extends OffsetSuite { + val one = LongOffset(1) + val two = LongOffset(2) + val three = LongOffset(3) + compare(one, two) + + compare(LongOffset(SerializedOffset(one.json)), + LongOffset(SerializedOffset(three.json))) +} + + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala new file mode 100644 index 000000000000..368c4604dfca --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming + +trait StateStoreMetricsTest extends StreamTest { + + def assertNumStateRows(total: Seq[Long], updated: Seq[Long]): AssertOnQuery = + AssertOnQuery(s"Check total state rows = $total, updated state rows = $updated") { q => + val progressWithData = q.recentProgress.filter(_.numInputRows > 0).lastOption.get + assert( + progressWithData.stateOperators.map(_.numRowsTotal) === total, + "incorrect total rows") + assert( + progressWithData.stateOperators.map(_.numRowsUpdated) === updated, + "incorrect updates rows") + true + } + + def assertNumStateRows(total: Long, updated: Long): AssertOnQuery = + assertNumStateRows(Seq(total), Seq(updated)) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala new file mode 100644 index 000000000000..9c901062d570 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -0,0 +1,900 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.io.{File, InterruptedIOException, IOException, UncheckedIOException} +import java.nio.channels.ClosedByInterruptException +import java.util.concurrent.{CountDownLatch, ExecutionException, TimeoutException, TimeUnit} + +import scala.reflect.ClassTag +import scala.util.control.ControlThrowable + +import com.google.common.util.concurrent.UncheckedExecutionException +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.SparkContext +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.plans.logical.Range +import org.apache.spark.sql.catalyst.streaming.InternalOutputModes +import org.apache.spark.sql.execution.command.ExplainCommand +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProvider} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.StreamSourceProvider +import org.apache.spark.sql.streaming.util.StreamManualClock +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.util.Utils + +class StreamSuite extends StreamTest { + + import testImplicits._ + + test("map with recovery") { + val inputData = MemoryStream[Int] + val mapped = inputData.toDS().map(_ + 1) + + testStream(mapped)( + AddData(inputData, 1, 2, 3), + StartStream(), + CheckAnswer(2, 3, 4), + StopStream, + AddData(inputData, 4, 5, 6), + StartStream(), + CheckAnswer(2, 3, 4, 5, 6, 7)) + } + + test("join") { + // Make a table and ensure it will be broadcast. + val smallTable = Seq((1, "one"), (2, "two"), (4, "four")).toDF("number", "word") + + // Join the input stream with a table. + val inputData = MemoryStream[Int] + val joined = inputData.toDS().toDF().join(smallTable, $"value" === $"number") + + testStream(joined)( + AddData(inputData, 1, 2, 3), + CheckAnswer(Row(1, 1, "one"), Row(2, 2, "two")), + AddData(inputData, 4), + CheckAnswer(Row(1, 1, "one"), Row(2, 2, "two"), Row(4, 4, "four"))) + } + + + test("explain join") { + // Make a table and ensure it will be broadcast. + val smallTable = Seq((1, "one"), (2, "two"), (4, "four")).toDF("number", "word") + + // Join the input stream with a table. + val inputData = MemoryStream[Int] + val joined = inputData.toDF().join(smallTable, smallTable("number") === $"value") + + val outputStream = new java.io.ByteArrayOutputStream() + Console.withOut(outputStream) { + joined.explain() + } + assert(outputStream.toString.contains("StreamingRelation")) + } + + test("SPARK-20432: union one stream with itself") { + val df = spark.readStream.format(classOf[FakeDefaultSource].getName).load().select("a") + val unioned = df.union(df) + withTempDir { outputDir => + withTempDir { checkpointDir => + val query = + unioned + .writeStream.format("parquet") + .option("checkpointLocation", checkpointDir.getAbsolutePath) + .start(outputDir.getAbsolutePath) + try { + query.processAllAvailable() + val outputDf = spark.read.parquet(outputDir.getAbsolutePath).as[Long] + checkDatasetUnorderly[Long](outputDf, (0L to 10L).union((0L to 10L)).toArray: _*) + } finally { + query.stop() + } + } + } + } + + test("union two streams") { + val inputData1 = MemoryStream[Int] + val inputData2 = MemoryStream[Int] + + val unioned = inputData1.toDS().union(inputData2.toDS()) + + testStream(unioned)( + AddData(inputData1, 1, 3, 5), + CheckAnswer(1, 3, 5), + AddData(inputData2, 2, 4, 6), + CheckAnswer(1, 2, 3, 4, 5, 6), + StopStream, + AddData(inputData1, 7), + StartStream(), + AddData(inputData2, 8), + CheckAnswer(1, 2, 3, 4, 5, 6, 7, 8)) + } + + test("sql queries") { + val inputData = MemoryStream[Int] + inputData.toDF().createOrReplaceTempView("stream") + val evens = sql("SELECT * FROM stream WHERE value % 2 = 0") + + testStream(evens)( + AddData(inputData, 1, 2, 3, 4), + CheckAnswer(2, 4)) + } + + test("DataFrame reuse") { + def assertDF(df: DataFrame) { + withTempDir { outputDir => + withTempDir { checkpointDir => + val query = df.writeStream.format("parquet") + .option("checkpointLocation", checkpointDir.getAbsolutePath) + .start(outputDir.getAbsolutePath) + try { + query.processAllAvailable() + val outputDf = spark.read.parquet(outputDir.getAbsolutePath).as[Long] + checkDataset[Long](outputDf, (0L to 10L).toArray: _*) + } finally { + query.stop() + } + } + } + } + + val df = spark.readStream.format(classOf[FakeDefaultSource].getName).load() + assertDF(df) + assertDF(df) + } + + test("Within the same streaming query, one StreamingRelation should only be transformed to one " + + "StreamingExecutionRelation") { + val df = spark.readStream.format(classOf[FakeDefaultSource].getName).load() + var query: StreamExecution = null + try { + query = + df.union(df) + .writeStream + .format("memory") + .queryName("memory") + .start() + .asInstanceOf[StreamingQueryWrapper] + .streamingQuery + query.awaitInitialization(streamingTimeout.toMillis) + val executionRelations = + query + .logicalPlan + .collect { case ser: StreamingExecutionRelation => ser } + assert(executionRelations.size === 2) + assert(executionRelations.distinct.size === 1) + } finally { + if (query != null) { + query.stop() + } + } + } + + test("unsupported queries") { + val streamInput = MemoryStream[Int] + val batchInput = Seq(1, 2, 3).toDS() + + def assertError(expectedMsgs: Seq[String])(body: => Unit): Unit = { + val e = intercept[AnalysisException] { + body + } + expectedMsgs.foreach { s => assert(e.getMessage.contains(s)) } + } + + // Running streaming plan as a batch query + assertError("start" :: Nil) { + streamInput.toDS.map { i => i }.count() + } + + // Running non-streaming plan with as a streaming query + assertError("without streaming sources" :: "start" :: Nil) { + val ds = batchInput.map { i => i } + testStream(ds)() + } + + // Running streaming plan that cannot be incrementalized + assertError("not supported" :: "streaming" :: Nil) { + val ds = streamInput.toDS.map { i => i }.sort() + testStream(ds)() + } + } + + test("minimize delay between batch construction and execution") { + + // For each batch, we would retrieve new data's offsets and log them before we run the execution + // This checks whether the key of the offset log is the expected batch id + def CheckOffsetLogLatestBatchId(expectedId: Int): AssertOnQuery = + AssertOnQuery(_.offsetLog.getLatest().get._1 == expectedId, + s"offsetLog's latest should be $expectedId") + + // Check the latest batchid in the commit log + def CheckCommitLogLatestBatchId(expectedId: Int): AssertOnQuery = + AssertOnQuery(_.batchCommitLog.getLatest().get._1 == expectedId, + s"commitLog's latest should be $expectedId") + + // Ensure that there has not been an incremental execution after restart + def CheckNoIncrementalExecutionCurrentBatchId(): AssertOnQuery = + AssertOnQuery(_.lastExecution == null, s"lastExecution not expected to run") + + // For each batch, we would log the state change during the execution + // This checks whether the key of the state change log is the expected batch id + def CheckIncrementalExecutionCurrentBatchId(expectedId: Int): AssertOnQuery = + AssertOnQuery(_.lastExecution.asInstanceOf[IncrementalExecution].currentBatchId == expectedId, + s"lastExecution's currentBatchId should be $expectedId") + + // For each batch, we would log the sink change after the execution + // This checks whether the key of the sink change log is the expected batch id + def CheckSinkLatestBatchId(expectedId: Int): AssertOnQuery = + AssertOnQuery(_.sink.asInstanceOf[MemorySink].latestBatchId.get == expectedId, + s"sink's lastBatchId should be $expectedId") + + val inputData = MemoryStream[Int] + testStream(inputData.toDS())( + StartStream(ProcessingTime("10 seconds"), new StreamManualClock), + + /* -- batch 0 ----------------------- */ + // Add some data in batch 0 + AddData(inputData, 1, 2, 3), + AdvanceManualClock(10 * 1000), // 10 seconds + + /* -- batch 1 ----------------------- */ + // Check the results of batch 0 + CheckAnswer(1, 2, 3), + CheckIncrementalExecutionCurrentBatchId(0), + CheckCommitLogLatestBatchId(0), + CheckOffsetLogLatestBatchId(0), + CheckSinkLatestBatchId(0), + // Add some data in batch 1 + AddData(inputData, 4, 5, 6), + AdvanceManualClock(10 * 1000), + + /* -- batch _ ----------------------- */ + // Check the results of batch 1 + CheckAnswer(1, 2, 3, 4, 5, 6), + CheckIncrementalExecutionCurrentBatchId(1), + CheckCommitLogLatestBatchId(1), + CheckOffsetLogLatestBatchId(1), + CheckSinkLatestBatchId(1), + + AdvanceManualClock(10 * 1000), + AdvanceManualClock(10 * 1000), + AdvanceManualClock(10 * 1000), + + /* -- batch __ ---------------------- */ + // Check the results of batch 1 again; this is to make sure that, when there's no new data, + // the currentId does not get logged (e.g. as 2) even if the clock has advanced many times + CheckAnswer(1, 2, 3, 4, 5, 6), + CheckIncrementalExecutionCurrentBatchId(1), + CheckCommitLogLatestBatchId(1), + CheckOffsetLogLatestBatchId(1), + CheckSinkLatestBatchId(1), + + /* Stop then restart the Stream */ + StopStream, + StartStream(ProcessingTime("10 seconds"), new StreamManualClock(60 * 1000)), + + /* -- batch 1 no rerun ----------------- */ + // batch 1 would not re-run because the latest batch id logged in commit log is 1 + AdvanceManualClock(10 * 1000), + CheckNoIncrementalExecutionCurrentBatchId(), + + /* -- batch 2 ----------------------- */ + // Check the results of batch 1 + CheckAnswer(1, 2, 3, 4, 5, 6), + CheckCommitLogLatestBatchId(1), + CheckOffsetLogLatestBatchId(1), + CheckSinkLatestBatchId(1), + // Add some data in batch 2 + AddData(inputData, 7, 8, 9), + AdvanceManualClock(10 * 1000), + + /* -- batch 3 ----------------------- */ + // Check the results of batch 2 + CheckAnswer(1, 2, 3, 4, 5, 6, 7, 8, 9), + CheckIncrementalExecutionCurrentBatchId(2), + CheckCommitLogLatestBatchId(2), + CheckOffsetLogLatestBatchId(2), + CheckSinkLatestBatchId(2)) + } + + test("insert an extraStrategy") { + try { + spark.experimental.extraStrategies = TestStrategy :: Nil + + val inputData = MemoryStream[(String, Int)] + val df = inputData.toDS().map(_._1).toDF("a") + + testStream(df)( + AddData(inputData, ("so slow", 1)), + CheckAnswer("so fast")) + } finally { + spark.experimental.extraStrategies = Nil + } + } + + testQuietly("handle fatal errors thrown from the stream thread") { + for (e <- Seq( + new VirtualMachineError {}, + new ThreadDeath, + new LinkageError, + new ControlThrowable {} + )) { + val source = new Source { + override def getOffset: Option[Offset] = { + throw e + } + + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + throw e + } + + override def schema: StructType = StructType(Array(StructField("value", IntegerType))) + + override def stop(): Unit = {} + } + val df = Dataset[Int]( + sqlContext.sparkSession, + StreamingExecutionRelation(source, sqlContext.sparkSession)) + testStream(df)( + // `ExpectFailure(isFatalError = true)` verifies two things: + // - Fatal errors can be propagated to `StreamingQuery.exception` and + // `StreamingQuery.awaitTermination` like non fatal errors. + // - Fatal errors can be caught by UncaughtExceptionHandler. + ExpectFailure(isFatalError = true)(ClassTag(e.getClass)) + ) + } + } + + test("output mode API in Scala") { + assert(OutputMode.Append === InternalOutputModes.Append) + assert(OutputMode.Complete === InternalOutputModes.Complete) + assert(OutputMode.Update === InternalOutputModes.Update) + } + + test("explain") { + val inputData = MemoryStream[String] + val df = inputData.toDS().map(_ + "foo").groupBy("value").agg(count("*")) + + // Test `df.explain` + val explain = ExplainCommand(df.queryExecution.logical, extended = false) + val explainString = + spark.sessionState + .executePlan(explain) + .executedPlan + .executeCollect() + .map(_.getString(0)) + .mkString("\n") + assert(explainString.contains("StateStoreRestore")) + assert(explainString.contains("StreamingRelation")) + assert(!explainString.contains("LocalTableScan")) + + // Test StreamingQuery.display + val q = df.writeStream.queryName("memory_explain").outputMode("complete").format("memory") + .start() + .asInstanceOf[StreamingQueryWrapper] + .streamingQuery + try { + assert("No physical plan. Waiting for data." === q.explainInternal(false)) + assert("No physical plan. Waiting for data." === q.explainInternal(true)) + + inputData.addData("abc") + q.processAllAvailable() + + val explainWithoutExtended = q.explainInternal(false) + // `extended = false` only displays the physical plan. + assert("LocalRelation".r.findAllMatchIn(explainWithoutExtended).size === 0) + assert("LocalTableScan".r.findAllMatchIn(explainWithoutExtended).size === 1) + // Use "StateStoreRestore" to verify that it does output a streaming physical plan + assert(explainWithoutExtended.contains("StateStoreRestore")) + + val explainWithExtended = q.explainInternal(true) + // `extended = true` displays 3 logical plans (Parsed/Optimized/Optimized) and 1 physical + // plan. + assert("LocalRelation".r.findAllMatchIn(explainWithExtended).size === 3) + assert("LocalTableScan".r.findAllMatchIn(explainWithExtended).size === 1) + // Use "StateStoreRestore" to verify that it does output a streaming physical plan + assert(explainWithExtended.contains("StateStoreRestore")) + } finally { + q.stop() + } + } + + test("SPARK-19065: dropDuplicates should not create expressions using the same id") { + withTempPath { testPath => + val data = Seq((1, 2), (2, 3), (3, 4)) + data.toDS.write.mode("overwrite").json(testPath.getCanonicalPath) + val schema = spark.read.json(testPath.getCanonicalPath).schema + val query = spark + .readStream + .schema(schema) + .json(testPath.getCanonicalPath) + .dropDuplicates("_1") + .writeStream + .format("memory") + .queryName("testquery") + .outputMode("append") + .start() + try { + query.processAllAvailable() + if (query.exception.isDefined) { + throw query.exception.get + } + } finally { + query.stop() + } + } + } + + test("handle IOException when the streaming thread is interrupted (pre Hadoop 2.8)") { + // This test uses a fake source to throw the same IOException as pre Hadoop 2.8 when the + // streaming thread is interrupted. We should handle it properly by not failing the query. + ThrowingIOExceptionLikeHadoop12074.createSourceLatch = new CountDownLatch(1) + val query = spark + .readStream + .format(classOf[ThrowingIOExceptionLikeHadoop12074].getName) + .load() + .writeStream + .format("console") + .start() + assert(ThrowingIOExceptionLikeHadoop12074.createSourceLatch + .await(streamingTimeout.toMillis, TimeUnit.MILLISECONDS), + "ThrowingIOExceptionLikeHadoop12074.createSource wasn't called before timeout") + query.stop() + assert(query.exception.isEmpty) + } + + test("handle InterruptedIOException when the streaming thread is interrupted (Hadoop 2.8+)") { + // This test uses a fake source to throw the same InterruptedIOException as Hadoop 2.8+ when the + // streaming thread is interrupted. We should handle it properly by not failing the query. + ThrowingInterruptedIOException.createSourceLatch = new CountDownLatch(1) + val query = spark + .readStream + .format(classOf[ThrowingInterruptedIOException].getName) + .load() + .writeStream + .format("console") + .start() + assert(ThrowingInterruptedIOException.createSourceLatch + .await(streamingTimeout.toMillis, TimeUnit.MILLISECONDS), + "ThrowingInterruptedIOException.createSource wasn't called before timeout") + query.stop() + assert(query.exception.isEmpty) + } + + test("SPARK-19873: streaming aggregation with change in number of partitions") { + val inputData = MemoryStream[(Int, Int)] + val agg = inputData.toDS().groupBy("_1").count() + + testStream(agg, OutputMode.Complete())( + AddData(inputData, (1, 0), (2, 0)), + StartStream(additionalConfs = Map(SQLConf.SHUFFLE_PARTITIONS.key -> "2")), + CheckAnswer((1, 1), (2, 1)), + StopStream, + AddData(inputData, (3, 0), (2, 0)), + StartStream(additionalConfs = Map(SQLConf.SHUFFLE_PARTITIONS.key -> "5")), + CheckAnswer((1, 1), (2, 2), (3, 1)), + StopStream, + AddData(inputData, (3, 0), (1, 0)), + StartStream(additionalConfs = Map(SQLConf.SHUFFLE_PARTITIONS.key -> "1")), + CheckAnswer((1, 2), (2, 2), (3, 2))) + } + + testQuietly("recover from a Spark v2.1 checkpoint") { + var inputData: MemoryStream[Int] = null + var query: DataStreamWriter[Row] = null + + def prepareMemoryStream(): Unit = { + inputData = MemoryStream[Int] + inputData.addData(1, 2, 3, 4) + inputData.addData(3, 4, 5, 6) + inputData.addData(5, 6, 7, 8) + + query = inputData + .toDF() + .groupBy($"value") + .agg(count("*")) + .writeStream + .outputMode("complete") + .format("memory") + } + + // Get an existing checkpoint generated by Spark v2.1. + // v2.1 does not record # shuffle partitions in the offset metadata. + val resourceUri = + this.getClass.getResource("/structured-streaming/checkpoint-version-2.1.0").toURI + val checkpointDir = new File(resourceUri) + + // 1 - Test if recovery from the checkpoint is successful. + prepareMemoryStream() + val dir1 = Utils.createTempDir().getCanonicalFile // not using withTempDir {}, makes test flaky + // Copy the checkpoint to a temp dir to prevent changes to the original. + // Not doing this will lead to the test passing on the first run, but fail subsequent runs. + FileUtils.copyDirectory(checkpointDir, dir1) + // Checkpoint data was generated by a query with 10 shuffle partitions. + // In order to test reading from the checkpoint, the checkpoint must have two or more batches, + // since the last batch may be rerun. + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "10") { + var streamingQuery: StreamingQuery = null + try { + streamingQuery = + query.queryName("counts").option("checkpointLocation", dir1.getCanonicalPath).start() + streamingQuery.processAllAvailable() + inputData.addData(9) + streamingQuery.processAllAvailable() + + QueryTest.checkAnswer(spark.table("counts").toDF(), + Row("1", 1) :: Row("2", 1) :: Row("3", 2) :: Row("4", 2) :: + Row("5", 2) :: Row("6", 2) :: Row("7", 1) :: Row("8", 1) :: Row("9", 1) :: Nil) + } finally { + if (streamingQuery ne null) { + streamingQuery.stop() + } + } + } + + // 2 - Check recovery with wrong num shuffle partitions + prepareMemoryStream() + val dir2 = Utils.createTempDir().getCanonicalFile + FileUtils.copyDirectory(checkpointDir, dir2) + // Since the number of partitions is greater than 10, should throw exception. + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "15") { + var streamingQuery: StreamingQuery = null + try { + intercept[StreamingQueryException] { + streamingQuery = + query.queryName("badQuery").option("checkpointLocation", dir2.getCanonicalPath).start() + streamingQuery.processAllAvailable() + } + } finally { + if (streamingQuery ne null) { + streamingQuery.stop() + } + } + } + } + + test("calling stop() on a query cancels related jobs") { + val input = MemoryStream[Int] + val query = input + .toDS() + .map { i => + while (!org.apache.spark.TaskContext.get().isInterrupted()) { + // keep looping till interrupted by query.stop() + Thread.sleep(100) + } + i + } + .writeStream + .format("console") + .start() + + input.addData(1) + // wait for jobs to start + eventually(timeout(streamingTimeout)) { + assert(sparkContext.statusTracker.getActiveJobIds().nonEmpty) + } + + query.stop() + // make sure jobs are stopped + eventually(timeout(streamingTimeout)) { + assert(sparkContext.statusTracker.getActiveJobIds().isEmpty) + } + } + + test("batch id is updated correctly in the job description") { + val queryName = "memStream" + @volatile var jobDescription: String = null + def assertDescContainsQueryNameAnd(batch: Integer): Unit = { + // wait for listener event to be processed + spark.sparkContext.listenerBus.waitUntilEmpty(streamingTimeout.toMillis) + assert(jobDescription.contains(queryName) && jobDescription.contains(s"batch = $batch")) + } + + spark.sparkContext.addSparkListener(new SparkListener { + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + jobDescription = jobStart.properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION) + } + }) + + val input = MemoryStream[Int] + val query = input + .toDS() + .map(_ + 1) + .writeStream + .format("memory") + .queryName(queryName) + .start() + + input.addData(1) + query.processAllAvailable() + assertDescContainsQueryNameAnd(batch = 0) + input.addData(2, 3) + query.processAllAvailable() + assertDescContainsQueryNameAnd(batch = 1) + input.addData(4) + query.processAllAvailable() + assertDescContainsQueryNameAnd(batch = 2) + query.stop() + } + + test("should resolve the checkpoint path") { + withTempDir { dir => + val checkpointLocation = dir.getCanonicalPath + assert(!checkpointLocation.startsWith("file:/")) + val query = MemoryStream[Int].toDF + .writeStream + .option("checkpointLocation", checkpointLocation) + .format("console") + .start() + try { + val resolvedCheckpointDir = + query.asInstanceOf[StreamingQueryWrapper].streamingQuery.resolvedCheckpointRoot + assert(resolvedCheckpointDir.startsWith("file:/")) + } finally { + query.stop() + } + } + } + + testQuietly("specify custom state store provider") { + val providerClassName = classOf[TestStateStoreProvider].getCanonicalName + withSQLConf("spark.sql.streaming.stateStore.providerClass" -> providerClassName) { + val input = MemoryStream[Int] + val df = input.toDS().groupBy().count() + val query = df.writeStream.outputMode("complete").format("memory").queryName("name").start() + input.addData(1, 2, 3) + val e = intercept[Exception] { + query.awaitTermination() + } + + assert(e.getMessage.contains(providerClassName)) + assert(e.getMessage.contains("instantiated")) + } + } + + testQuietly("custom state store provider read from offset log") { + val input = MemoryStream[Int] + val df = input.toDS().groupBy().count() + val providerConf1 = "spark.sql.streaming.stateStore.providerClass" -> + "org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider" + val providerConf2 = "spark.sql.streaming.stateStore.providerClass" -> + classOf[TestStateStoreProvider].getCanonicalName + + def runQuery(queryName: String, checkpointLoc: String): Unit = { + val query = df.writeStream + .outputMode("complete") + .format("memory") + .queryName(queryName) + .option("checkpointLocation", checkpointLoc) + .start() + input.addData(1, 2, 3) + query.processAllAvailable() + query.stop() + } + + withTempDir { dir => + val checkpointLoc1 = new File(dir, "1").getCanonicalPath + withSQLConf(providerConf1) { + runQuery("query1", checkpointLoc1) // generate checkpoints + } + + val checkpointLoc2 = new File(dir, "2").getCanonicalPath + withSQLConf(providerConf2) { + // Verify new query will use new provider that throw error on loading + intercept[Exception] { + runQuery("query2", checkpointLoc2) + } + + // Verify old query from checkpoint will still use old provider + runQuery("query1", checkpointLoc1) + } + } + } + + for (e <- Seq( + new InterruptedException, + new InterruptedIOException, + new ClosedByInterruptException, + new UncheckedIOException("test", new ClosedByInterruptException), + new ExecutionException("test", new InterruptedException), + new UncheckedExecutionException("test", new InterruptedException))) { + test(s"view ${e.getClass.getSimpleName} as a normal query stop") { + ThrowingExceptionInCreateSource.createSourceLatch = new CountDownLatch(1) + ThrowingExceptionInCreateSource.exception = e + val query = spark + .readStream + .format(classOf[ThrowingExceptionInCreateSource].getName) + .load() + .writeStream + .format("console") + .start() + assert(ThrowingExceptionInCreateSource.createSourceLatch + .await(streamingTimeout.toMillis, TimeUnit.MILLISECONDS), + "ThrowingExceptionInCreateSource.createSource wasn't called before timeout") + query.stop() + assert(query.exception.isEmpty) + } + } +} + +abstract class FakeSource extends StreamSourceProvider { + private val fakeSchema = StructType(StructField("a", IntegerType) :: Nil) + + override def sourceSchema( + spark: SQLContext, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): (String, StructType) = ("fakeSource", fakeSchema) +} + +/** A fake StreamSourceProvider that creates a fake Source that cannot be reused. */ +class FakeDefaultSource extends FakeSource { + + override def createSource( + spark: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source = { + // Create a fake Source that emits 0 to 10. + new Source { + private var offset = -1L + + override def schema: StructType = StructType(StructField("a", IntegerType) :: Nil) + + override def getOffset: Option[Offset] = { + if (offset >= 10) { + None + } else { + offset += 1 + Some(LongOffset(offset)) + } + } + + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + val startOffset = start.map(_.asInstanceOf[LongOffset].offset).getOrElse(-1L) + 1 + val ds = new Dataset[java.lang.Long]( + spark.sparkSession, + Range( + startOffset, + end.asInstanceOf[LongOffset].offset + 1, + 1, + Some(spark.sparkSession.sparkContext.defaultParallelism), + isStreaming = true), + Encoders.LONG) + ds.toDF("a") + } + + override def stop() {} + } + } +} + +/** A fake source that throws the same IOException like pre Hadoop 2.8 when it's interrupted. */ +class ThrowingIOExceptionLikeHadoop12074 extends FakeSource { + import ThrowingIOExceptionLikeHadoop12074._ + + override def createSource( + spark: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source = { + createSourceLatch.countDown() + try { + Thread.sleep(30000) + throw new TimeoutException("sleep was not interrupted in 30 seconds") + } catch { + case ie: InterruptedException => + throw new IOException(ie.toString) + } + } +} + +object ThrowingIOExceptionLikeHadoop12074 { + /** + * A latch to allow the user to wait until `ThrowingIOExceptionLikeHadoop12074.createSource` is + * called. + */ + @volatile var createSourceLatch: CountDownLatch = null +} + +/** A fake source that throws InterruptedIOException like Hadoop 2.8+ when it's interrupted. */ +class ThrowingInterruptedIOException extends FakeSource { + import ThrowingInterruptedIOException._ + + override def createSource( + spark: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source = { + createSourceLatch.countDown() + try { + Thread.sleep(30000) + throw new TimeoutException("sleep was not interrupted in 30 seconds") + } catch { + case ie: InterruptedException => + val iie = new InterruptedIOException(ie.toString) + iie.initCause(ie) + throw iie + } + } +} + +object ThrowingInterruptedIOException { + /** + * A latch to allow the user to wait until `ThrowingInterruptedIOException.createSource` is + * called. + */ + @volatile var createSourceLatch: CountDownLatch = null +} + +class TestStateStoreProvider extends StateStoreProvider { + + override def init( + stateStoreId: StateStoreId, + keySchema: StructType, + valueSchema: StructType, + indexOrdinal: Option[Int], + storeConfs: StateStoreConf, + hadoopConf: Configuration): Unit = { + throw new Exception("Successfully instantiated") + } + + override def stateStoreId: StateStoreId = null + + override def close(): Unit = { } + + override def getStore(version: Long): StateStore = null +} + +/** A fake source that throws `ThrowingExceptionInCreateSource.exception` in `createSource` */ +class ThrowingExceptionInCreateSource extends FakeSource { + + override def createSource( + spark: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source = { + ThrowingExceptionInCreateSource.createSourceLatch.countDown() + try { + Thread.sleep(30000) + throw new TimeoutException("sleep was not interrupted in 30 seconds") + } catch { + case _: InterruptedException => + throw ThrowingExceptionInCreateSource.exception + } + } +} + +object ThrowingExceptionInCreateSource { + /** + * A latch to allow the user to wait until `ThrowingExceptionInCreateSource.createSource` is + * called. + */ + @volatile var createSourceLatch: CountDownLatch = null + @volatile var exception: Exception = null +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala new file mode 100644 index 000000000000..70b39b934071 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -0,0 +1,731 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.lang.Thread.UncaughtExceptionHandler + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.language.experimental.macros +import scala.reflect.ClassTag +import scala.util.Random +import scala.util.control.NonFatal + +import org.scalatest.{Assertions, BeforeAndAfterAll} +import org.scalatest.concurrent.{Eventually, Signaler, ThreadSignaler, TimeLimits} +import org.scalatest.concurrent.PatienceConfiguration.Timeout +import org.scalatest.exceptions.TestFailedDueToTimeoutException +import org.scalatest.time.Span +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.sql.{Dataset, Encoder, QueryTest, Row} +import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, RowEncoder} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.state.StateStore +import org.apache.spark.sql.streaming.StreamingQueryListener._ +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.util.{Clock, SystemClock, Utils} + +/** + * A framework for implementing tests for streaming queries and sources. + * + * A test consists of a set of steps (expressed as a `StreamAction`) that are executed in order, + * blocking as necessary to let the stream catch up. For example, the following adds some data to + * a stream, blocking until it can verify that the correct values are eventually produced. + * + * {{{ + * val inputData = MemoryStream[Int] + * val mapped = inputData.toDS().map(_ + 1) + * + * testStream(mapped)( + * AddData(inputData, 1, 2, 3), + * CheckAnswer(2, 3, 4)) + * }}} + * + * Note that while we do sleep to allow the other thread to progress without spinning, + * `StreamAction` checks should not depend on the amount of time spent sleeping. Instead they + * should check the actual progress of the stream before verifying the required test condition. + * + * Currently it is assumed that all streaming queries will eventually complete in 10 seconds to + * avoid hanging forever in the case of failures. However, individual suites can change this + * by overriding `streamingTimeout`. + */ +trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with BeforeAndAfterAll { + + implicit val defaultSignaler: Signaler = ThreadSignaler + override def afterAll(): Unit = { + super.afterAll() + StateStore.stop() // stop the state store maintenance thread and unload store providers + } + + /** How long to wait for an active stream to catch up when checking a result. */ + val streamingTimeout = 10.seconds + + /** A trait for actions that can be performed while testing a streaming DataFrame. */ + trait StreamAction + + /** A trait to mark actions that require the stream to be actively running. */ + trait StreamMustBeRunning + + /** + * Adds the given data to the stream. Subsequent check answers will block until this data has + * been processed. + */ + object AddData { + def apply[A](source: MemoryStream[A], data: A*): AddDataMemory[A] = + AddDataMemory(source, data) + } + + /** A trait that can be extended when testing a source. */ + trait AddData extends StreamAction { + /** + * Called to adding the data to a source. It should find the source to add data to from + * the active query, and then return the source object the data was added, as well as the + * offset of added data. + */ + def addData(query: Option[StreamExecution]): (Source, Offset) + } + + /** A trait that can be extended when testing a source. */ + trait ExternalAction extends StreamAction { + def runAction(): Unit + } + + case class AddDataMemory[A](source: MemoryStream[A], data: Seq[A]) extends AddData { + override def toString: String = s"AddData to $source: ${data.mkString(",")}" + + override def addData(query: Option[StreamExecution]): (Source, Offset) = { + (source, source.addData(data)) + } + } + + /** + * Checks to make sure that the current data stored in the sink matches the `expectedAnswer`. + * This operation automatically blocks until all added data has been processed. + */ + object CheckAnswer { + def apply[A : Encoder](data: A*): CheckAnswerRows = { + val encoder = encoderFor[A] + val toExternalRow = RowEncoder(encoder.schema).resolveAndBind() + CheckAnswerRows( + data.map(d => toExternalRow.fromRow(encoder.toRow(d))), + lastOnly = false, + isSorted = false) + } + + def apply(rows: Row*): CheckAnswerRows = CheckAnswerRows(rows, false, false) + } + + /** + * Checks to make sure that the current data stored in the sink matches the `expectedAnswer`. + * This operation automatically blocks until all added data has been processed. + */ + object CheckLastBatch { + def apply[A : Encoder](data: A*): CheckAnswerRows = { + apply(isSorted = false, data: _*) + } + + def apply[A: Encoder](isSorted: Boolean, data: A*): CheckAnswerRows = { + val encoder = encoderFor[A] + val toExternalRow = RowEncoder(encoder.schema).resolveAndBind() + CheckAnswerRows( + data.map(d => toExternalRow.fromRow(encoder.toRow(d))), + lastOnly = true, + isSorted = isSorted) + } + + def apply(rows: Row*): CheckAnswerRows = CheckAnswerRows(rows, true, false) + } + + case class CheckAnswerRows(expectedAnswer: Seq[Row], lastOnly: Boolean, isSorted: Boolean) + extends StreamAction with StreamMustBeRunning { + override def toString: String = s"$operatorName: ${expectedAnswer.mkString(",")}" + private def operatorName = if (lastOnly) "CheckLastBatch" else "CheckAnswer" + } + + /** Stops the stream. It must currently be running. */ + case object StopStream extends StreamAction with StreamMustBeRunning + + /** Starts the stream, resuming if data has already been processed. It must not be running. */ + case class StartStream( + trigger: Trigger = Trigger.ProcessingTime(0), + triggerClock: Clock = new SystemClock, + additionalConfs: Map[String, String] = Map.empty, + checkpointLocation: String = null) + extends StreamAction + + /** Advance the trigger clock's time manually. */ + case class AdvanceManualClock(timeToAdd: Long) extends StreamAction + + /** + * Signals that a failure is expected and should not kill the test. + * + * @param isFatalError if this is a fatal error. If so, the error should also be caught by + * UncaughtExceptionHandler. + * @param assertFailure a function to verify the error. + */ + case class ExpectFailure[T <: Throwable : ClassTag]( + assertFailure: Throwable => Unit = _ => {}, + isFatalError: Boolean = false) extends StreamAction { + val causeClass: Class[T] = implicitly[ClassTag[T]].runtimeClass.asInstanceOf[Class[T]] + override def toString(): String = + s"ExpectFailure[${causeClass.getName}, isFatalError: $isFatalError]" + } + + /** Assert that a body is true */ + class Assert(condition: => Boolean, val message: String = "") extends StreamAction { + def run(): Unit = { Assertions.assert(condition) } + override def toString: String = s"Assert(, $message)" + } + + object Assert { + def apply(condition: => Boolean, message: String = ""): Assert = new Assert(condition, message) + def apply(message: String)(body: => Unit): Assert = new Assert( { body; true }, message) + def apply(body: => Unit): Assert = new Assert( { body; true }, "") + } + + /** Assert that a condition on the active query is true */ + class AssertOnQuery(val condition: StreamExecution => Boolean, val message: String) + extends StreamAction { + override def toString: String = s"AssertOnQuery(, $message)" + } + + object AssertOnQuery { + def apply(condition: StreamExecution => Boolean, message: String = ""): AssertOnQuery = { + new AssertOnQuery(condition, message) + } + + def apply(message: String)(condition: StreamExecution => Boolean): AssertOnQuery = { + new AssertOnQuery(condition, message) + } + } + + /** Execute arbitrary code */ + object Execute { + def apply(func: StreamExecution => Any): AssertOnQuery = + AssertOnQuery(query => { func(query); true }) + } + + /** + * Executes the specified actions on the given streaming DataFrame and provides helpful + * error messages in the case of failures or incorrect answers. + * + * Note that if the stream is not explicitly started before an action that requires it to be + * running then it will be automatically started before performing any other actions. + */ + def testStream( + _stream: Dataset[_], + outputMode: OutputMode = OutputMode.Append)(actions: StreamAction*): Unit = synchronized { + import org.apache.spark.sql.streaming.util.StreamManualClock + + // `synchronized` is added to prevent the user from calling multiple `testStream`s concurrently + // because this method assumes there is only one active query in its `StreamingQueryListener` + // and it may not work correctly when multiple `testStream`s run concurrently. + + val stream = _stream.toDF() + val sparkSession = stream.sparkSession // use the session in DF, not the default session + var pos = 0 + var currentStream: StreamExecution = null + var lastStream: StreamExecution = null + val awaiting = new mutable.HashMap[Int, Offset]() // source index -> offset to wait for + val sink = new MemorySink(stream.schema, outputMode) + val resetConfValues = mutable.Map[String, Option[String]]() + + @volatile + var streamThreadDeathCause: Throwable = null + // Set UncaughtExceptionHandler in `onQueryStarted` so that we can ensure catching fatal errors + // during query initialization. + val listener = new StreamingQueryListener { + override def onQueryStarted(event: QueryStartedEvent): Unit = { + // Note: this assumes there is only one query active in the `testStream` method. + Thread.currentThread.setUncaughtExceptionHandler(new UncaughtExceptionHandler { + override def uncaughtException(t: Thread, e: Throwable): Unit = { + streamThreadDeathCause = e + } + }) + } + + override def onQueryProgress(event: QueryProgressEvent): Unit = {} + override def onQueryTerminated(event: QueryTerminatedEvent): Unit = {} + } + sparkSession.streams.addListener(listener) + + // If the test doesn't manually start the stream, we do it automatically at the beginning. + val startedManually = + actions.takeWhile(!_.isInstanceOf[StreamMustBeRunning]).exists(_.isInstanceOf[StartStream]) + val startedTest = if (startedManually) actions else StartStream() +: actions + + def testActions = actions.zipWithIndex.map { + case (a, i) => + if ((pos == i && startedManually) || (pos == (i + 1) && !startedManually)) { + "=> " + a.toString + } else { + " " + a.toString + } + }.mkString("\n") + + def currentOffsets = + if (currentStream != null) currentStream.committedOffsets.toString else "not started" + + def threadState = + if (currentStream != null && currentStream.microBatchThread.isAlive) "alive" else "dead" + def threadStackTrace = if (currentStream != null && currentStream.microBatchThread.isAlive) { + s"Thread stack trace: ${currentStream.microBatchThread.getStackTrace.mkString("\n")}" + } else { + "" + } + + def testState = + s""" + |== Progress == + |$testActions + | + |== Stream == + |Output Mode: $outputMode + |Stream state: $currentOffsets + |Thread state: $threadState + |$threadStackTrace + |${if (streamThreadDeathCause != null) stackTraceToString(streamThreadDeathCause) else ""} + | + |== Sink == + |${sink.toDebugString} + | + | + |== Plan == + |${if (currentStream != null) currentStream.lastExecution else ""} + """.stripMargin + + def verify(condition: => Boolean, message: String): Unit = { + if (!condition) { + failTest(message) + } + } + + def eventually[T](message: String)(func: => T): T = { + try { + Eventually.eventually(Timeout(streamingTimeout)) { + func + } + } catch { + case NonFatal(e) => + failTest(message, e) + } + } + + def failTest(message: String, cause: Throwable = null) = { + + // Recursively pretty print a exception with truncated stacktrace and internal cause + def exceptionToString(e: Throwable, prefix: String = ""): String = { + val base = s"$prefix${e.getMessage}" + + e.getStackTrace.take(10).mkString(s"\n$prefix", s"\n$prefix\t", "\n") + if (e.getCause != null) { + base + s"\n$prefix\tCaused by: " + exceptionToString(e.getCause, s"$prefix\t") + } else { + base + } + } + val c = Option(cause).map(exceptionToString(_)) + val m = if (message != null && message.size > 0) Some(message) else None + fail( + s""" + |${(m ++ c).mkString(": ")} + |$testState + """.stripMargin) + } + + var manualClockExpectedTime = -1L + val defaultCheckpointLocation = + Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath + try { + startedTest.foreach { action => + logInfo(s"Processing test stream action: $action") + action match { + case StartStream(trigger, triggerClock, additionalConfs, checkpointLocation) => + verify(currentStream == null, "stream already running") + verify(triggerClock.isInstanceOf[SystemClock] + || triggerClock.isInstanceOf[StreamManualClock], + "Use either SystemClock or StreamManualClock to start the stream") + if (triggerClock.isInstanceOf[StreamManualClock]) { + manualClockExpectedTime = triggerClock.asInstanceOf[StreamManualClock].getTimeMillis() + } + val metadataRoot = Option(checkpointLocation).getOrElse(defaultCheckpointLocation) + + additionalConfs.foreach(pair => { + val value = + if (sparkSession.conf.contains(pair._1)) { + Some(sparkSession.conf.get(pair._1)) + } else None + resetConfValues(pair._1) = value + sparkSession.conf.set(pair._1, pair._2) + }) + + lastStream = currentStream + currentStream = + sparkSession + .streams + .startQuery( + None, + Some(metadataRoot), + stream, + sink, + outputMode, + trigger = trigger, + triggerClock = triggerClock) + .asInstanceOf[StreamingQueryWrapper] + .streamingQuery + // Wait until the initialization finishes, because some tests need to use `logicalPlan` + // after starting the query. + try { + currentStream.awaitInitialization(streamingTimeout.toMillis) + } catch { + case _: StreamingQueryException => + // Ignore the exception. `StopStream` or `ExpectFailure` will catch it as well. + } + + case AdvanceManualClock(timeToAdd) => + verify(currentStream != null, + "can not advance manual clock when a stream is not running") + verify(currentStream.triggerClock.isInstanceOf[StreamManualClock], + s"can not advance clock of type ${currentStream.triggerClock.getClass}") + val clock = currentStream.triggerClock.asInstanceOf[StreamManualClock] + assert(manualClockExpectedTime >= 0) + + // Make sure we don't advance ManualClock too early. See SPARK-16002. + eventually("StreamManualClock has not yet entered the waiting state") { + assert(clock.isStreamWaitingAt(manualClockExpectedTime)) + } + + clock.advance(timeToAdd) + manualClockExpectedTime += timeToAdd + verify(clock.getTimeMillis() === manualClockExpectedTime, + s"Unexpected clock time after updating: " + + s"expecting $manualClockExpectedTime, current ${clock.getTimeMillis()}") + + case StopStream => + verify(currentStream != null, "can not stop a stream that is not running") + try failAfter(streamingTimeout) { + currentStream.stop() + verify(!currentStream.microBatchThread.isAlive, + s"microbatch thread not stopped") + verify(!currentStream.isActive, + "query.isActive() is false even after stopping") + verify(currentStream.exception.isEmpty, + s"query.exception() is not empty after clean stop: " + + currentStream.exception.map(_.toString()).getOrElse("")) + } catch { + case _: InterruptedException => + case e: org.scalatest.exceptions.TestFailedDueToTimeoutException => + failTest( + "Timed out while stopping and waiting for microbatchthread to terminate.", e) + case t: Throwable => + failTest("Error while stopping stream", t) + } finally { + lastStream = currentStream + currentStream = null + } + + case ef: ExpectFailure[_] => + verify(currentStream != null, "can not expect failure when stream is not running") + try failAfter(streamingTimeout) { + val thrownException = intercept[StreamingQueryException] { + currentStream.awaitTermination() + } + eventually("microbatch thread not stopped after termination with failure") { + assert(!currentStream.microBatchThread.isAlive) + } + verify(currentStream.exception === Some(thrownException), + s"incorrect exception returned by query.exception()") + + val exception = currentStream.exception.get + verify(exception.cause.getClass === ef.causeClass, + "incorrect cause in exception returned by query.exception()\n" + + s"\tExpected: ${ef.causeClass}\n\tReturned: ${exception.cause.getClass}") + if (ef.isFatalError) { + // This is a fatal error, `streamThreadDeathCause` should be set to this error in + // UncaughtExceptionHandler. + verify(streamThreadDeathCause != null && + streamThreadDeathCause.getClass === ef.causeClass, + "UncaughtExceptionHandler didn't receive the correct error\n" + + s"\tExpected: ${ef.causeClass}\n\tReturned: $streamThreadDeathCause") + streamThreadDeathCause = null + } + ef.assertFailure(exception.getCause) + } catch { + case _: InterruptedException => + case e: org.scalatest.exceptions.TestFailedDueToTimeoutException => + failTest("Timed out while waiting for failure", e) + case t: Throwable => + failTest("Error while checking stream failure", t) + } finally { + lastStream = currentStream + currentStream = null + } + + case a: AssertOnQuery => + verify(currentStream != null || lastStream != null, + "cannot assert when no stream has been started") + val streamToAssert = Option(currentStream).getOrElse(lastStream) + try { + verify(a.condition(streamToAssert), s"Assert on query failed: ${a.message}") + } catch { + case NonFatal(e) => + failTest(s"Assert on query failed: ${a.message}", e) + } + + case a: Assert => + val streamToAssert = Option(currentStream).getOrElse(lastStream) + verify({ a.run(); true }, s"Assert failed: ${a.message}") + + case a: AddData => + try { + + // If the query is running with manual clock, then wait for the stream execution + // thread to start waiting for the clock to increment. This is needed so that we + // are adding data when there is no trigger that is active. This would ensure that + // the data gets deterministically added to the next batch triggered after the manual + // clock is incremented in following AdvanceManualClock. This avoid race conditions + // between the test thread and the stream execution thread in tests using manual + // clock. + if (currentStream != null && + currentStream.triggerClock.isInstanceOf[StreamManualClock]) { + val clock = currentStream.triggerClock.asInstanceOf[StreamManualClock] + eventually("Error while synchronizing with manual clock before adding data") { + if (currentStream.isActive) { + assert(clock.isStreamWaitingAt(clock.getTimeMillis())) + } + } + if (!currentStream.isActive) { + failTest("Query terminated while synchronizing with manual clock") + } + } + // Add data + val queryToUse = Option(currentStream).orElse(Option(lastStream)) + val (source, offset) = a.addData(queryToUse) + + def findSourceIndex(plan: LogicalPlan): Option[Int] = { + plan + .collect { case StreamingExecutionRelation(s, _) => s } + .zipWithIndex + .find(_._1 == source) + .map(_._2) + } + + // Try to find the index of the source to which data was added. Either get the index + // from the current active query or the original input logical plan. + val sourceIndex = + queryToUse.flatMap { query => + findSourceIndex(query.logicalPlan) + }.orElse { + findSourceIndex(stream.logicalPlan) + }.getOrElse { + throw new IllegalArgumentException( + "Could find index of the source to which data was added") + } + + // Store the expected offset of added data to wait for it later + awaiting.put(sourceIndex, offset) + } catch { + case NonFatal(e) => + failTest("Error adding data", e) + } + + case e: ExternalAction => + e.runAction() + + case CheckAnswerRows(expectedAnswer, lastOnly, isSorted) => + verify(currentStream != null, "stream not running") + // Get the map of source index to the current source objects + val indexToSource = currentStream + .logicalPlan + .collect { case StreamingExecutionRelation(s, _) => s } + .zipWithIndex + .map(_.swap) + .toMap + + // Block until all data added has been processed for all the source + awaiting.foreach { case (sourceIndex, offset) => + failAfter(streamingTimeout) { + currentStream.awaitOffset(indexToSource(sourceIndex), offset) + } + } + + val sparkAnswer = try if (lastOnly) sink.latestBatchData else sink.allData catch { + case e: Exception => + failTest("Exception while getting data from sink", e) + } + + QueryTest.sameRows(expectedAnswer, sparkAnswer, isSorted).foreach { + error => failTest(error) + } + } + pos += 1 + } + if (streamThreadDeathCause != null) { + failTest("Stream Thread Died", streamThreadDeathCause) + } + } catch { + case _: InterruptedException if streamThreadDeathCause != null => + failTest("Stream Thread Died", streamThreadDeathCause) + case e: org.scalatest.exceptions.TestFailedDueToTimeoutException => + failTest("Timed out waiting for stream", e) + } finally { + if (currentStream != null && currentStream.microBatchThread.isAlive) { + currentStream.stop() + } + + // Rollback prev configuration values + resetConfValues.foreach { + case (key, Some(value)) => sparkSession.conf.set(key, value) + case (key, None) => sparkSession.conf.unset(key) + } + sparkSession.streams.removeListener(listener) + } + } + + + /** + * Creates a stress test that randomly starts/stops/adds data/checks the result. + * + * @param ds a dataframe that executes + 1 on a stream of integers, returning the result + * @param addData an add data action that adds the given numbers to the stream, encoding them + * as needed + * @param iterations the iteration number + */ + def runStressTest( + ds: Dataset[Int], + addData: Seq[Int] => StreamAction, + iterations: Int = 100): Unit = { + runStressTest(ds, Seq.empty, (data, running) => addData(data), iterations) + } + + /** + * Creates a stress test that randomly starts/stops/adds data/checks the result. + * + * @param ds a dataframe that executes + 1 on a stream of integers, returning the result + * @param prepareActions actions need to run before starting the stress test. + * @param addData an add data action that adds the given numbers to the stream, encoding them + * as needed + * @param iterations the iteration number + */ + def runStressTest( + ds: Dataset[Int], + prepareActions: Seq[StreamAction], + addData: (Seq[Int], Boolean) => StreamAction, + iterations: Int): Unit = { + implicit val intEncoder = ExpressionEncoder[Int]() + var dataPos = 0 + var running = true + val actions = new ArrayBuffer[StreamAction]() + actions ++= prepareActions + + def addCheck() = { actions += CheckAnswer(1 to dataPos: _*) } + + def addRandomData() = { + val numItems = Random.nextInt(10) + val data = dataPos until (dataPos + numItems) + dataPos += numItems + actions += addData(data, running) + } + + (1 to iterations).foreach { i => + val rand = Random.nextDouble() + if(!running) { + rand match { + case r if r < 0.7 => // AddData + addRandomData() + + case _ => // StartStream + actions += StartStream() + running = true + } + } else { + rand match { + case r if r < 0.1 => + addCheck() + + case r if r < 0.7 => // AddData + addRandomData() + + case _ => // StopStream + addCheck() + actions += StopStream + running = false + } + } + } + if(!running) { actions += StartStream() } + addCheck() + testStream(ds)(actions: _*) + } + + object AwaitTerminationTester { + + trait ExpectedBehavior + + /** Expect awaitTermination to not be blocked */ + case object ExpectNotBlocked extends ExpectedBehavior + + /** Expect awaitTermination to get blocked */ + case object ExpectBlocked extends ExpectedBehavior + + /** Expect awaitTermination to throw an exception */ + case class ExpectException[E <: Exception]()(implicit val t: ClassTag[E]) + extends ExpectedBehavior + + private val DEFAULT_TEST_TIMEOUT = 1.second + + def test( + expectedBehavior: ExpectedBehavior, + awaitTermFunc: () => Unit, + testTimeout: Span = DEFAULT_TEST_TIMEOUT + ): Unit = { + + expectedBehavior match { + case ExpectNotBlocked => + withClue("Got blocked when expected non-blocking.") { + failAfter(testTimeout) { + awaitTermFunc() + } + } + + case ExpectBlocked => + withClue("Was not blocked when expected.") { + intercept[TestFailedDueToTimeoutException] { + failAfter(testTimeout) { + awaitTermFunc() + } + } + } + + case e: ExpectException[_] => + val thrownException = + withClue(s"Did not throw ${e.t.runtimeClass.getSimpleName} when expected.") { + intercept[StreamingQueryException] { + failAfter(testTimeout) { + awaitTermFunc() + } + } + } + assert(thrownException.cause.getClass === e.t.runtimeClass, + "exception of incorrect type was throw") + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala new file mode 100644 index 000000000000..995cea3b37d4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -0,0 +1,564 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.util.{Locale, TimeZone} + +import org.scalatest.Assertions +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.{SparkEnv, SparkException} +import org.apache.spark.rdd.BlockRDD +import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.plans.logical.Aggregate +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.exchange.Exchange +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.state.StateStore +import org.apache.spark.sql.expressions.scalalang.typed +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.streaming.OutputMode._ +import org.apache.spark.sql.streaming.util.{MockSourceProvider, StreamManualClock} +import org.apache.spark.sql.types.StructType +import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId} + +object FailureSingleton { + var firstTime = true +} + +class StreamingAggregationSuite extends StateStoreMetricsTest + with BeforeAndAfterAll with Assertions { + + override def afterAll(): Unit = { + super.afterAll() + StateStore.stop() + } + + import testImplicits._ + + test("simple count, update mode") { + val inputData = MemoryStream[Int] + + val aggregated = + inputData.toDF() + .groupBy($"value") + .agg(count("*")) + .as[(Int, Long)] + + testStream(aggregated, Update)( + AddData(inputData, 3), + CheckLastBatch((3, 1)), + AddData(inputData, 3, 2), + CheckLastBatch((3, 2), (2, 1)), + StopStream, + StartStream(), + AddData(inputData, 3, 2, 1), + CheckLastBatch((3, 3), (2, 2), (1, 1)), + // By default we run in new tuple mode. + AddData(inputData, 4, 4, 4, 4), + CheckLastBatch((4, 4)) + ) + } + + test("count distinct") { + val inputData = MemoryStream[(Int, Seq[Int])] + + val aggregated = + inputData.toDF() + .select($"*", explode($"_2") as 'value) + .groupBy($"_1") + .agg(size(collect_set($"value"))) + .as[(Int, Int)] + + testStream(aggregated, Update)( + AddData(inputData, (1, Seq(1, 2))), + CheckLastBatch((1, 2)) + ) + } + + test("simple count, complete mode") { + val inputData = MemoryStream[Int] + + val aggregated = + inputData.toDF() + .groupBy($"value") + .agg(count("*")) + .as[(Int, Long)] + + testStream(aggregated, Complete)( + AddData(inputData, 3), + CheckLastBatch((3, 1)), + AddData(inputData, 2), + CheckLastBatch((3, 1), (2, 1)), + StopStream, + StartStream(), + AddData(inputData, 3, 2, 1), + CheckLastBatch((3, 2), (2, 2), (1, 1)), + AddData(inputData, 4, 4, 4, 4), + CheckLastBatch((4, 4), (3, 2), (2, 2), (1, 1)) + ) + } + + test("simple count, append mode") { + val inputData = MemoryStream[Int] + + val aggregated = + inputData.toDF() + .groupBy($"value") + .agg(count("*")) + .as[(Int, Long)] + + val e = intercept[AnalysisException] { + testStream(aggregated, Append)() + } + Seq("append", "not supported").foreach { m => + assert(e.getMessage.toLowerCase(Locale.ROOT).contains(m.toLowerCase(Locale.ROOT))) + } + } + + test("sort after aggregate in complete mode") { + val inputData = MemoryStream[Int] + + val aggregated = + inputData.toDF() + .groupBy($"value") + .agg(count("*")) + .toDF("value", "count") + .orderBy($"count".desc) + .as[(Int, Long)] + + testStream(aggregated, Complete)( + AddData(inputData, 3), + CheckLastBatch(isSorted = true, (3, 1)), + AddData(inputData, 2, 3), + CheckLastBatch(isSorted = true, (3, 2), (2, 1)), + StopStream, + StartStream(), + AddData(inputData, 3, 2, 1), + CheckLastBatch(isSorted = true, (3, 3), (2, 2), (1, 1)), + AddData(inputData, 4, 4, 4, 4), + CheckLastBatch(isSorted = true, (4, 4), (3, 3), (2, 2), (1, 1)) + ) + } + + test("state metrics") { + val inputData = MemoryStream[Int] + + val aggregated = + inputData.toDS() + .flatMap(x => Seq(x, x + 1)) + .toDF("value") + .groupBy($"value") + .agg(count("*")) + .as[(Int, Long)] + + implicit class RichStreamExecution(query: StreamExecution) { + def stateNodes: Seq[SparkPlan] = { + query.lastExecution.executedPlan.collect { + case p if p.isInstanceOf[StateStoreSaveExec] => p + } + } + } + + // Test with Update mode + testStream(aggregated, Update)( + AddData(inputData, 1), + CheckLastBatch((1, 1), (2, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 2 }, + AddData(inputData, 2, 3), + CheckLastBatch((2, 2), (3, 2), (4, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 4 } + ) + + // Test with Complete mode + inputData.reset() + testStream(aggregated, Complete)( + AddData(inputData, 1), + CheckLastBatch((1, 1), (2, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 2 }, + AddData(inputData, 2, 3), + CheckLastBatch((1, 1), (2, 2), (3, 2), (4, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 4 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 4 } + ) + } + + test("multiple keys") { + val inputData = MemoryStream[Int] + + val aggregated = + inputData.toDF() + .groupBy($"value", $"value" + 1) + .agg(count("*")) + .as[(Int, Int, Long)] + + testStream(aggregated, Update)( + AddData(inputData, 1, 2), + CheckLastBatch((1, 2, 1), (2, 3, 1)), + AddData(inputData, 1, 2), + CheckLastBatch((1, 2, 2), (2, 3, 2)) + ) + } + + testQuietly("midbatch failure") { + val inputData = MemoryStream[Int] + FailureSingleton.firstTime = true + val aggregated = + inputData.toDS() + .map { i => + if (i == 4 && FailureSingleton.firstTime) { + FailureSingleton.firstTime = false + sys.error("injected failure") + } + + i + } + .groupBy($"value") + .agg(count("*")) + .as[(Int, Long)] + + testStream(aggregated, Update)( + StartStream(), + AddData(inputData, 1, 2, 3, 4), + ExpectFailure[SparkException](), + StartStream(), + CheckLastBatch((1, 1), (2, 1), (3, 1), (4, 1)) + ) + } + + test("typed aggregators") { + val inputData = MemoryStream[(String, Int)] + val aggregated = inputData.toDS().groupByKey(_._1).agg(typed.sumLong(_._2)) + + testStream(aggregated, Update)( + AddData(inputData, ("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)), + CheckLastBatch(("a", 30), ("b", 3), ("c", 1)) + ) + } + + test("prune results by current_time, complete mode") { + import testImplicits._ + val clock = new StreamManualClock + val inputData = MemoryStream[Long] + val aggregated = + inputData.toDF() + .groupBy($"value") + .agg(count("*")) + .where('value >= current_timestamp().cast("long") - 10L) + + testStream(aggregated, Complete)( + StartStream(Trigger.ProcessingTime("10 seconds"), triggerClock = clock), + + // advance clock to 10 seconds, all keys retained + AddData(inputData, 0L, 5L, 5L, 10L), + AdvanceManualClock(10 * 1000), + CheckLastBatch((0L, 1), (5L, 2), (10L, 1)), + + // advance clock to 20 seconds, should retain keys >= 10 + AddData(inputData, 15L, 15L, 20L), + AdvanceManualClock(10 * 1000), + CheckLastBatch((10L, 1), (15L, 2), (20L, 1)), + + // advance clock to 30 seconds, should retain keys >= 20 + AddData(inputData, 0L, 85L), + AdvanceManualClock(10 * 1000), + CheckLastBatch((20L, 1), (85L, 1)), + + // bounce stream and ensure correct batch timestamp is used + // i.e., we don't take it from the clock, which is at 90 seconds. + StopStream, + AssertOnQuery { q => // clear the sink + q.sink.asInstanceOf[MemorySink].clear() + q.batchCommitLog.purge(3) + // advance by a minute i.e., 90 seconds total + clock.advance(60 * 1000L) + true + }, + StartStream(Trigger.ProcessingTime("10 seconds"), triggerClock = clock), + // The commit log blown, causing the last batch to re-run + CheckLastBatch((20L, 1), (85L, 1)), + AssertOnQuery { q => + clock.getTimeMillis() == 90000L + }, + + // advance clock to 100 seconds, should retain keys >= 90 + AddData(inputData, 85L, 90L, 100L, 105L), + AdvanceManualClock(10 * 1000), + CheckLastBatch((90L, 1), (100L, 1), (105L, 1)) + ) + } + + test("prune results by current_date, complete mode") { + import testImplicits._ + val clock = new StreamManualClock + val tz = TimeZone.getDefault.getID + val inputData = MemoryStream[Long] + val aggregated = + inputData.toDF() + .select(to_utc_timestamp(from_unixtime('value * DateTimeUtils.SECONDS_PER_DAY), tz)) + .toDF("value") + .groupBy($"value") + .agg(count("*")) + .where($"value".cast("date") >= date_sub(current_date(), 10)) + .select(($"value".cast("long") / DateTimeUtils.SECONDS_PER_DAY).cast("long"), $"count(1)") + testStream(aggregated, Complete)( + StartStream(Trigger.ProcessingTime("10 day"), triggerClock = clock), + // advance clock to 10 days, should retain all keys + AddData(inputData, 0L, 5L, 5L, 10L), + AdvanceManualClock(DateTimeUtils.MILLIS_PER_DAY * 10), + CheckLastBatch((0L, 1), (5L, 2), (10L, 1)), + // advance clock to 20 days, should retain keys >= 10 + AddData(inputData, 15L, 15L, 20L), + AdvanceManualClock(DateTimeUtils.MILLIS_PER_DAY * 10), + CheckLastBatch((10L, 1), (15L, 2), (20L, 1)), + // advance clock to 30 days, should retain keys >= 20 + AddData(inputData, 85L), + AdvanceManualClock(DateTimeUtils.MILLIS_PER_DAY * 10), + CheckLastBatch((20L, 1), (85L, 1)), + + // bounce stream and ensure correct batch timestamp is used + // i.e., we don't take it from the clock, which is at 90 days. + StopStream, + AssertOnQuery { q => // clear the sink + q.sink.asInstanceOf[MemorySink].clear() + q.batchCommitLog.purge(3) + // advance by 60 days i.e., 90 days total + clock.advance(DateTimeUtils.MILLIS_PER_DAY * 60) + true + }, + StartStream(Trigger.ProcessingTime("10 day"), triggerClock = clock), + // Commit log blown, causing a re-run of the last batch + CheckLastBatch((20L, 1), (85L, 1)), + + // advance clock to 100 days, should retain keys >= 90 + AddData(inputData, 85L, 90L, 100L, 105L), + AdvanceManualClock(DateTimeUtils.MILLIS_PER_DAY * 10), + CheckLastBatch((90L, 1), (100L, 1), (105L, 1)) + ) + } + + test("SPARK-19690: do not convert batch aggregation in streaming query to streaming") { + val streamInput = MemoryStream[Int] + val batchDF = Seq(1, 2, 3, 4, 5) + .toDF("value") + .withColumn("parity", 'value % 2) + .groupBy('parity) + .agg(count("*") as 'joinValue) + val joinDF = streamInput + .toDF() + .join(batchDF, 'value === 'parity) + + // make sure we're planning an aggregate in the first place + assert(batchDF.queryExecution.optimizedPlan match { case _: Aggregate => true }) + + testStream(joinDF, Append)( + AddData(streamInput, 0, 1, 2, 3), + CheckLastBatch((0, 0, 2), (1, 1, 3)), + AddData(streamInput, 0, 1, 2, 3), + CheckLastBatch((0, 0, 2), (1, 1, 3))) + } + + /** + * This method verifies certain properties in the SparkPlan of a streaming aggregation. + * First of all, it checks that the child of a `StateStoreRestoreExec` creates the desired + * data distribution, where the child could be an Exchange, or a `HashAggregateExec` which already + * provides the expected data distribution. + * + * The second thing it checks that the child provides the expected number of partitions. + * + * The third thing it checks that we don't add an unnecessary shuffle in-between + * `StateStoreRestoreExec` and `StateStoreSaveExec`. + */ + private def checkAggregationChain( + se: StreamExecution, + expectShuffling: Boolean, + expectedPartition: Int): Boolean = { + val executedPlan = se.lastExecution.executedPlan + val restore = executedPlan + .collect { case ss: StateStoreRestoreExec => ss } + .head + restore.child match { + case node: UnaryExecNode => + assert(node.outputPartitioning.numPartitions === expectedPartition, + "Didn't get the expected number of partitions.") + if (expectShuffling) { + assert(node.isInstanceOf[Exchange], s"Expected a shuffle, got: ${node.child}") + } else { + assert(!node.isInstanceOf[Exchange], "Didn't expect a shuffle") + } + + case _ => fail("Expected no shuffling") + } + var reachedRestore = false + // Check that there should be no exchanges after `StateStoreRestoreExec` + executedPlan.foreachUp { p => + if (reachedRestore) { + assert(!p.isInstanceOf[Exchange], "There should be no further exchanges") + } else { + reachedRestore = p.isInstanceOf[StateStoreRestoreExec] + } + } + true + } + + test("SPARK-21977: coalesce(1) with 0 partition RDD should be repartitioned to 1") { + val inputSource = new BlockRDDBackedSource(spark) + MockSourceProvider.withMockSources(inputSource) { + // `coalesce(1)` changes the partitioning of data to `SinglePartition` which by default + // satisfies the required distributions of all aggregations. Therefore in our SparkPlan, we + // don't have any shuffling. However, `coalesce(1)` only guarantees that the RDD has at most 1 + // partition. Which means that if we have an input RDD with 0 partitions, nothing gets + // executed. Therefore the StateStore's don't save any delta files for a given trigger. This + // then leads to `FileNotFoundException`s in the subsequent batch. + // This isn't the only problem though. Once we introduce a shuffle before + // `StateStoreRestoreExec`, the input to the operator is an empty iterator. When performing + // `groupBy().agg(...)`, `HashAggregateExec` returns a `0` value for all aggregations. If + // we fail to restore the previous state in `StateStoreRestoreExec`, we save the 0 value in + // `StateStoreSaveExec` losing all previous state. + val aggregated: Dataset[Long] = + spark.readStream.format((new MockSourceProvider).getClass.getCanonicalName) + .load().coalesce(1).groupBy().count().as[Long] + + testStream(aggregated, Complete())( + AddBlockData(inputSource, Seq(1)), + CheckLastBatch(1), + AssertOnQuery("Verify no shuffling") { se => + checkAggregationChain(se, expectShuffling = false, 1) + }, + AddBlockData(inputSource), // create an empty trigger + CheckLastBatch(1), + AssertOnQuery("Verify addition of exchange operator") { se => + checkAggregationChain(se, expectShuffling = true, 1) + }, + AddBlockData(inputSource, Seq(2, 3)), + CheckLastBatch(3), + AddBlockData(inputSource), + CheckLastBatch(3), + StopStream + ) + } + } + + test("SPARK-21977: coalesce(1) with aggregation should still be repartitioned when it " + + "has non-empty grouping keys") { + val inputSource = new BlockRDDBackedSource(spark) + MockSourceProvider.withMockSources(inputSource) { + withTempDir { tempDir => + + // `coalesce(1)` changes the partitioning of data to `SinglePartition` which by default + // satisfies the required distributions of all aggregations. However, when we have + // non-empty grouping keys, in streaming, we must repartition to + // `spark.sql.shuffle.partitions`, otherwise only a single StateStore is used to process + // all keys. This may be fine, however, if the user removes the coalesce(1) or changes to + // a `coalesce(2)` for example, then the default behavior is to shuffle to + // `spark.sql.shuffle.partitions` many StateStores. When this happens, all StateStore's + // except 1 will be missing their previous delta files, which causes the stream to fail + // with FileNotFoundException. + def createDf(partitions: Int): Dataset[(Long, Long)] = { + spark.readStream + .format((new MockSourceProvider).getClass.getCanonicalName) + .load().coalesce(partitions).groupBy('a % 1).count().as[(Long, Long)] + } + + testStream(createDf(1), Complete())( + StartStream(checkpointLocation = tempDir.getAbsolutePath), + AddBlockData(inputSource, Seq(1)), + CheckLastBatch((0L, 1L)), + AssertOnQuery("Verify addition of exchange operator") { se => + checkAggregationChain( + se, + expectShuffling = true, + spark.sessionState.conf.numShufflePartitions) + }, + StopStream + ) + + testStream(createDf(2), Complete())( + StartStream(checkpointLocation = tempDir.getAbsolutePath), + Execute(se => se.processAllAvailable()), + AddBlockData(inputSource, Seq(2), Seq(3), Seq(4)), + CheckLastBatch((0L, 4L)), + AssertOnQuery("Verify no exchange added") { se => + checkAggregationChain( + se, + expectShuffling = false, + spark.sessionState.conf.numShufflePartitions) + }, + AddBlockData(inputSource), + CheckLastBatch((0L, 4L)), + StopStream + ) + } + } + } + + /** Add blocks of data to the `BlockRDDBackedSource`. */ + case class AddBlockData(source: BlockRDDBackedSource, data: Seq[Int]*) extends AddData { + override def addData(query: Option[StreamExecution]): (Source, Offset) = { + source.addBlocks(data: _*) + (source, LongOffset(source.counter)) + } + } + + /** + * A Streaming Source that is backed by a BlockRDD and that can create RDDs with 0 blocks at will. + */ + class BlockRDDBackedSource(spark: SparkSession) extends Source { + var counter = 0L + private val blockMgr = SparkEnv.get.blockManager + private var blocks: Seq[BlockId] = Seq.empty + + def addBlocks(dataBlocks: Seq[Int]*): Unit = synchronized { + dataBlocks.foreach { data => + val id = TestBlockId(counter.toString) + blockMgr.putIterator(id, data.iterator, StorageLevel.MEMORY_ONLY) + blocks ++= id :: Nil + counter += 1 + } + counter += 1 + } + + override def getOffset: Option[Offset] = synchronized { + if (counter == 0) None else Some(LongOffset(counter)) + } + + override def getBatch(start: Option[Offset], end: Offset): DataFrame = synchronized { + val rdd = new BlockRDD[Int](spark.sparkContext, blocks.toArray) + .map(i => InternalRow(i)) // we don't really care about the values in this test + blocks = Seq.empty + spark.internalCreateDataFrame(rdd, schema, isStreaming = true).toDF() + } + override def schema: StructType = MockSourceProvider.fakeSchema + override def stop(): Unit = { + blockMgr.getMatchingBlockIds(_.isInstanceOf[TestBlockId]).foreach(blockMgr.removeBlock(_)) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala new file mode 100644 index 000000000000..533e1165fd59 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -0,0 +1,472 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.util.UUID + +import scala.util.Random + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.scheduler.ExecutorCacheTaskLocation +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet} +import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, Filter} +import org.apache.spark.sql.execution.LogicalRDD +import org.apache.spark.sql.execution.streaming.{MemoryStream, StatefulOperatorStateInfo, StreamingSymmetricHashJoinHelper} +import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreProviderId} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + + +class StreamingJoinSuite extends StreamTest with StateStoreMetricsTest with BeforeAndAfter { + + before { + SparkSession.setActiveSession(spark) // set this before force initializing 'joinExec' + spark.streams.stateStoreCoordinator // initialize the lazy coordinator + } + + after { + StateStore.stop() + } + + import testImplicits._ + test("stream stream inner join on non-time column") { + val input1 = MemoryStream[Int] + val input2 = MemoryStream[Int] + + val df1 = input1.toDF.select('value as "key", ('value * 2) as "leftValue") + val df2 = input2.toDF.select('value as "key", ('value * 3) as "rightValue") + val joined = df1.join(df2, "key") + + testStream(joined)( + AddData(input1, 1), + CheckAnswer(), + AddData(input2, 1, 10), // 1 arrived on input1 first, then input2, should join + CheckLastBatch((1, 2, 3)), + AddData(input1, 10), // 10 arrived on input2 first, then input1, should join + CheckLastBatch((10, 20, 30)), + AddData(input2, 1), // another 1 in input2 should join with 1 input1 + CheckLastBatch((1, 2, 3)), + StopStream, + StartStream(), + AddData(input1, 1), // multiple 1s should be kept in state causing multiple (1, 2, 3) + CheckLastBatch((1, 2, 3), (1, 2, 3)), + StopStream, + StartStream(), + AddData(input1, 100), + AddData(input2, 100), + CheckLastBatch((100, 200, 300)) + ) + } + + test("stream stream inner join on windows - without watermark") { + val input1 = MemoryStream[Int] + val input2 = MemoryStream[Int] + + val df1 = input1.toDF + .select('value as "key", 'value.cast("timestamp") as "timestamp", ('value * 2) as "leftValue") + .select('key, window('timestamp, "10 second"), 'leftValue) + + val df2 = input2.toDF + .select('value as "key", 'value.cast("timestamp") as "timestamp", + ('value * 3) as "rightValue") + .select('key, window('timestamp, "10 second"), 'rightValue) + + val joined = df1.join(df2, Seq("key", "window")) + .select('key, $"window.end".cast("long"), 'leftValue, 'rightValue) + + testStream(joined)( + AddData(input1, 1), + CheckLastBatch(), + AddData(input2, 1), + CheckLastBatch((1, 10, 2, 3)), + StopStream, + StartStream(), + AddData(input1, 25), + CheckLastBatch(), + StopStream, + StartStream(), + AddData(input2, 25), + CheckLastBatch((25, 30, 50, 75)), + AddData(input1, 1), + CheckLastBatch((1, 10, 2, 3)), // State for 1 still around as there is no watermark + StopStream, + StartStream(), + AddData(input1, 5), + CheckLastBatch(), + AddData(input2, 5), + CheckLastBatch((5, 10, 10, 15)) // No filter by any watermark + ) + } + + test("stream stream inner join on windows - with watermark") { + val input1 = MemoryStream[Int] + val input2 = MemoryStream[Int] + + val df1 = input1.toDF + .select('value as "key", 'value.cast("timestamp") as "timestamp", ('value * 2) as "leftValue") + .withWatermark("timestamp", "10 seconds") + .select('key, window('timestamp, "10 second"), 'leftValue) + + val df2 = input2.toDF + .select('value as "key", 'value.cast("timestamp") as "timestamp", + ('value * 3) as "rightValue") + .select('key, window('timestamp, "10 second"), 'rightValue) + + val joined = df1.join(df2, Seq("key", "window")) + .select('key, $"window.end".cast("long"), 'leftValue, 'rightValue) + + testStream(joined)( + AddData(input1, 1), + CheckAnswer(), + assertNumStateRows(total = 1, updated = 1), + + AddData(input2, 1), + CheckLastBatch((1, 10, 2, 3)), + assertNumStateRows(total = 2, updated = 1), + StopStream, + StartStream(), + + AddData(input1, 25), + CheckLastBatch(), // since there is only 1 watermark operator, the watermark should be 15 + assertNumStateRows(total = 3, updated = 1), + + AddData(input2, 25), + CheckLastBatch((25, 30, 50, 75)), // watermark = 15 should remove 2 rows having window=[0,10] + assertNumStateRows(total = 2, updated = 1), + StopStream, + StartStream(), + + AddData(input2, 1), + CheckLastBatch(), // Should not join as < 15 removed + assertNumStateRows(total = 2, updated = 0), // row not add as 1 < state key watermark = 15 + + AddData(input1, 5), + CheckLastBatch(), // Should not join or add to state as < 15 got filtered by watermark + assertNumStateRows(total = 2, updated = 0) + ) + } + + test("stream stream inner join with time range - with watermark - one side condition") { + import org.apache.spark.sql.functions._ + + val leftInput = MemoryStream[(Int, Int)] + val rightInput = MemoryStream[(Int, Int)] + + val df1 = leftInput.toDF.toDF("leftKey", "time") + .select('leftKey, 'time.cast("timestamp") as "leftTime", ('leftKey * 2) as "leftValue") + .withWatermark("leftTime", "10 seconds") + + val df2 = rightInput.toDF.toDF("rightKey", "time") + .select('rightKey, 'time.cast("timestamp") as "rightTime", ('rightKey * 3) as "rightValue") + .withWatermark("rightTime", "10 seconds") + + val joined = + df1.join(df2, expr("leftKey = rightKey AND leftTime < rightTime - interval 5 seconds")) + .select('leftKey, 'leftTime.cast("int"), 'rightTime.cast("int")) + + testStream(joined)( + AddData(leftInput, (1, 5)), + CheckAnswer(), + AddData(rightInput, (1, 11)), + CheckLastBatch((1, 5, 11)), + AddData(rightInput, (1, 10)), + CheckLastBatch(), // no match as neither 5, nor 10 from leftTime is less than rightTime 10 - 5 + assertNumStateRows(total = 3, updated = 1), + + // Increase event time watermark to 20s by adding data with time = 30s on both inputs + AddData(leftInput, (1, 3), (1, 30)), + CheckLastBatch((1, 3, 10), (1, 3, 11)), + assertNumStateRows(total = 5, updated = 2), + AddData(rightInput, (0, 30)), + CheckLastBatch(), + assertNumStateRows(total = 6, updated = 1), + + // event time watermark: max event time - 10 ==> 30 - 10 = 20 + // right side state constraint: 20 < leftTime < rightTime - 5 ==> rightTime > 25 + + // Run another batch with event time = 25 to clear right state where rightTime <= 25 + AddData(rightInput, (0, 30)), + CheckLastBatch(), + assertNumStateRows(total = 5, updated = 1), // removed (1, 11) and (1, 10), added (0, 30) + + // New data to right input should match with left side (1, 3) and (1, 5), as left state should + // not be cleared. But rows rightTime <= 20 should be filtered due to event time watermark and + // state rows with rightTime <= 25 should be removed from state. + // (1, 20) ==> filtered by event time watermark = 20 + // (1, 21) ==> passed filter, matched with left (1, 3) and (1, 5), not added to state + // as state watermark = 25 + // (1, 28) ==> passed filter, matched with left (1, 3) and (1, 5), added to state + AddData(rightInput, (1, 20), (1, 21), (1, 28)), + CheckLastBatch((1, 3, 21), (1, 5, 21), (1, 3, 28), (1, 5, 28)), + assertNumStateRows(total = 6, updated = 1), + + // New data to left input with leftTime <= 20 should be filtered due to event time watermark + AddData(leftInput, (1, 20), (1, 21)), + CheckLastBatch((1, 21, 28)), + assertNumStateRows(total = 7, updated = 1) + ) + } + + test("stream stream inner join with time range - with watermark - two side conditions") { + import org.apache.spark.sql.functions._ + + val leftInput = MemoryStream[(Int, Int)] + val rightInput = MemoryStream[(Int, Int)] + + val df1 = leftInput.toDF.toDF("leftKey", "time") + .select('leftKey, 'time.cast("timestamp") as "leftTime", ('leftKey * 2) as "leftValue") + .withWatermark("leftTime", "20 seconds") + + val df2 = rightInput.toDF.toDF("rightKey", "time") + .select('rightKey, 'time.cast("timestamp") as "rightTime", ('rightKey * 3) as "rightValue") + .withWatermark("rightTime", "30 seconds") + + val condition = expr( + "leftKey = rightKey AND " + + "leftTime BETWEEN rightTime - interval 10 seconds AND rightTime + interval 5 seconds") + + // This translates to leftTime <= rightTime + 5 seconds AND leftTime >= rightTime - 10 seconds + // So given leftTime, rightTime has to be BETWEEN leftTime - 5 seconds AND leftTime + 10 seconds + // + // =============== * ======================== * ============================== * ==> leftTime + // | | | + // |<---- 5s -->|<------ 10s ------>| |<------ 10s ------>|<---- 5s -->| + // | | | + // == * ============================== * =========>============== * ===============> rightTime + // + // E.g. + // if rightTime = 60, then it matches only leftTime = [50, 65] + // if leftTime = 20, then it match only with rightTime = [15, 30] + // + // State value predicates + // left side: + // values allowed: leftTime >= rightTime - 10s ==> leftTime > eventTimeWatermark - 10 + // drop state where leftTime < eventTime - 10 + // right side: + // values allowed: rightTime >= leftTime - 5s ==> rightTime > eventTimeWatermark - 5 + // drop state where rightTime < eventTime - 5 + + val joined = + df1.join(df2, condition).select('leftKey, 'leftTime.cast("int"), 'rightTime.cast("int")) + + testStream(joined)( + // If leftTime = 20, then it match only with rightTime = [15, 30] + AddData(leftInput, (1, 20)), + CheckAnswer(), + AddData(rightInput, (1, 14), (1, 15), (1, 25), (1, 26), (1, 30), (1, 31)), + CheckLastBatch((1, 20, 15), (1, 20, 25), (1, 20, 26), (1, 20, 30)), + assertNumStateRows(total = 7, updated = 6), + + // If rightTime = 60, then it matches only leftTime = [50, 65] + AddData(rightInput, (1, 60)), + CheckLastBatch(), // matches with nothing on the left + AddData(leftInput, (1, 49), (1, 50), (1, 65), (1, 66)), + CheckLastBatch((1, 50, 60), (1, 65, 60)), + assertNumStateRows(total = 12, updated = 4), + + // Event time watermark = min(left: 66 - delay 20 = 46, right: 60 - delay 30 = 30) = 30 + // Left state value watermark = 30 - 10 = slightly less than 20 (since condition has <=) + // Should drop < 20 from left, i.e., none + // Right state value watermark = 30 - 5 = slightly less than 25 (since condition has <=) + // Should drop < 25 from the right, i.e., 14 and 15 + AddData(leftInput, (1, 30), (1, 31)), // 30 should not be processed or added to stat + CheckLastBatch((1, 31, 26), (1, 31, 30), (1, 31, 31)), + assertNumStateRows(total = 11, updated = 1), // 12 - 2 removed + 1 added + + // Advance the watermark + AddData(rightInput, (1, 80)), + CheckLastBatch(), + assertNumStateRows(total = 12, updated = 1), + + // Event time watermark = min(left: 66 - delay 20 = 46, right: 80 - delay 30 = 50) = 46 + // Left state value watermark = 46 - 10 = slightly less than 36 (since condition has <=) + // Should drop < 36 from left, i.e., 20, 31 (30 was not added) + // Right state value watermark = 46 - 5 = slightly less than 41 (since condition has <=) + // Should drop < 41 from the right, i.e., 25, 26, 30, 31 + AddData(rightInput, (1, 50)), + CheckLastBatch((1, 49, 50), (1, 50, 50)), + assertNumStateRows(total = 7, updated = 1) // 12 - 6 removed + 1 added + ) + } + + testQuietly("stream stream inner join without equality predicate") { + val input1 = MemoryStream[Int] + val input2 = MemoryStream[Int] + + val df1 = input1.toDF.select('value as "leftKey", ('value * 2) as "leftValue") + val df2 = input2.toDF.select('value as "rightKey", ('value * 3) as "rightValue") + val joined = df1.join(df2, expr("leftKey < rightKey")) + val e = intercept[Exception] { + val q = joined.writeStream.format("memory").queryName("test").start() + input1.addData(1) + q.awaitTermination(10000) + } + assert(e.toString.contains("Stream stream joins without equality predicate is not supported")) + } + + testQuietly("extract watermark from time condition") { + val attributesToFindConstraintFor = Seq( + AttributeReference("leftTime", TimestampType)(), + AttributeReference("leftOther", IntegerType)()) + val metadataWithWatermark = new MetadataBuilder() + .putLong(EventTimeWatermark.delayKey, 1000) + .build() + val attributesWithWatermark = Seq( + AttributeReference("rightTime", TimestampType, metadata = metadataWithWatermark)(), + AttributeReference("rightOther", IntegerType)()) + + def watermarkFrom( + conditionStr: String, + rightWatermark: Option[Long] = Some(10000)): Option[Long] = { + val conditionExpr = Some(conditionStr).map { str => + val plan = + Filter( + spark.sessionState.sqlParser.parseExpression(str), + LogicalRDD( + attributesToFindConstraintFor ++ attributesWithWatermark, + spark.sparkContext.emptyRDD)(spark)) + plan.queryExecution.optimizedPlan.asInstanceOf[Filter].condition + } + StreamingSymmetricHashJoinHelper.getStateValueWatermark( + AttributeSet(attributesToFindConstraintFor), AttributeSet(attributesWithWatermark), + conditionExpr, rightWatermark) + } + + // Test comparison directionality. E.g. if leftTime < rightTime and rightTime > watermark, + // then cannot define constraint on leftTime. + assert(watermarkFrom("leftTime > rightTime") === Some(10000)) + assert(watermarkFrom("leftTime >= rightTime") === Some(9999)) + assert(watermarkFrom("leftTime < rightTime") === None) + assert(watermarkFrom("leftTime <= rightTime") === None) + assert(watermarkFrom("rightTime > leftTime") === None) + assert(watermarkFrom("rightTime >= leftTime") === None) + assert(watermarkFrom("rightTime < leftTime") === Some(10000)) + assert(watermarkFrom("rightTime <= leftTime") === Some(9999)) + + // Test type conversions + assert(watermarkFrom("CAST(leftTime AS LONG) > CAST(rightTime AS LONG)") === Some(10000)) + assert(watermarkFrom("CAST(leftTime AS LONG) < CAST(rightTime AS LONG)") === None) + assert(watermarkFrom("CAST(leftTime AS DOUBLE) > CAST(rightTime AS DOUBLE)") === Some(10000)) + assert(watermarkFrom("CAST(leftTime AS LONG) > CAST(rightTime AS DOUBLE)") === Some(10000)) + assert(watermarkFrom("CAST(leftTime AS LONG) > CAST(rightTime AS FLOAT)") === Some(10000)) + assert(watermarkFrom("CAST(leftTime AS DOUBLE) > CAST(rightTime AS FLOAT)") === Some(10000)) + assert(watermarkFrom("CAST(leftTime AS STRING) > CAST(rightTime AS STRING)") === None) + + // Test with timestamp type + calendar interval on either side of equation + // Note: timestamptype and calendar interval don't commute, so less valid combinations to test. + assert(watermarkFrom("leftTime > rightTime + interval 1 second") === Some(11000)) + assert(watermarkFrom("leftTime + interval 2 seconds > rightTime ") === Some(8000)) + assert(watermarkFrom("leftTime > rightTime - interval 3 second") === Some(7000)) + assert(watermarkFrom("rightTime < leftTime - interval 3 second") === Some(13000)) + assert(watermarkFrom("rightTime - interval 1 second < leftTime - interval 3 second") + === Some(12000)) + + // Test with casted long type + constants on either side of equation + // Note: long type and constants commute, so more combinations to test. + // -- Constants on the right + assert(watermarkFrom("CAST(leftTime AS LONG) > CAST(rightTime AS LONG) + 1") === Some(11000)) + assert(watermarkFrom("CAST(leftTime AS LONG) > CAST(rightTime AS LONG) - 1") === Some(9000)) + assert(watermarkFrom("CAST(leftTime AS LONG) > CAST((rightTime + interval 1 second) AS LONG)") + === Some(11000)) + assert(watermarkFrom("CAST(leftTime AS LONG) > 2 + CAST(rightTime AS LONG)") === Some(12000)) + assert(watermarkFrom("CAST(leftTime AS LONG) > -0.5 + CAST(rightTime AS LONG)") === Some(9500)) + assert(watermarkFrom("CAST(leftTime AS LONG) - CAST(rightTime AS LONG) > 2") === Some(12000)) + assert(watermarkFrom("-CAST(rightTime AS DOUBLE) + CAST(leftTime AS LONG) > 0.1") + === Some(10100)) + assert(watermarkFrom("0 > CAST(rightTime AS LONG) - CAST(leftTime AS LONG) + 0.2") + === Some(10200)) + // -- Constants on the left + assert(watermarkFrom("CAST(leftTime AS LONG) + 2 > CAST(rightTime AS LONG)") === Some(8000)) + assert(watermarkFrom("1 + CAST(leftTime AS LONG) > CAST(rightTime AS LONG)") === Some(9000)) + assert(watermarkFrom("CAST((leftTime + interval 3 second) AS LONG) > CAST(rightTime AS LONG)") + === Some(7000)) + assert(watermarkFrom("CAST(leftTime AS LONG) - 2 > CAST(rightTime AS LONG)") === Some(12000)) + assert(watermarkFrom("CAST(leftTime AS LONG) + 0.5 > CAST(rightTime AS LONG)") === Some(9500)) + assert(watermarkFrom("CAST(leftTime AS LONG) - CAST(rightTime AS LONG) - 2 > 0") + === Some(12000)) + assert(watermarkFrom("-CAST(rightTime AS LONG) + CAST(leftTime AS LONG) - 0.1 > 0") + === Some(10100)) + // -- Constants on both sides, mixed types + assert(watermarkFrom("CAST(leftTime AS LONG) - 2.0 > CAST(rightTime AS LONG) + 1") + === Some(13000)) + + // Test multiple conditions, should return minimum watermark + assert(watermarkFrom( + "leftTime > rightTime - interval 3 second AND rightTime < leftTime + interval 2 seconds") === + Some(7000)) // first condition wins + assert(watermarkFrom( + "leftTime > rightTime - interval 3 second AND rightTime < leftTime + interval 4 seconds") === + Some(6000)) // second condition wins + + // Test invalid comparisons + assert(watermarkFrom("cast(leftTime AS LONG) > leftOther") === None) // non-time attributes + assert(watermarkFrom("leftOther > rightOther") === None) // non-time attributes + assert(watermarkFrom("leftOther > rightOther AND leftTime > rightTime") === Some(10000)) + assert(watermarkFrom("cast(rightTime AS DOUBLE) < rightOther") === None) // non-time attributes + assert(watermarkFrom("leftTime > rightTime + interval 1 month") === None) // month not allowed + + // Test static comparisons + assert(watermarkFrom("cast(leftTime AS LONG) > 10") === Some(10000)) + } + + test("locality preferences of StateStoreAwareZippedRDD") { + import StreamingSymmetricHashJoinHelper._ + + withTempDir { tempDir => + val queryId = UUID.randomUUID + val opId = 0 + val path = Utils.createDirectory(tempDir.getAbsolutePath, Random.nextString(10)).toString + val stateInfo = StatefulOperatorStateInfo(path, queryId, opId, 0L) + + implicit val sqlContext = spark.sqlContext + val coordinatorRef = sqlContext.streams.stateStoreCoordinator + val numPartitions = 5 + val storeNames = Seq("name1", "name2") + + val partitionAndStoreNameToLocation = { + for (partIndex <- 0 until numPartitions; storeName <- storeNames) yield { + (partIndex, storeName) -> s"host-$partIndex-$storeName" + } + }.toMap + partitionAndStoreNameToLocation.foreach { case ((partIndex, storeName), hostName) => + val providerId = StateStoreProviderId(stateInfo, partIndex, storeName) + coordinatorRef.reportActiveInstance(providerId, hostName, s"exec-$hostName") + require( + coordinatorRef.getLocation(providerId) === + Some(ExecutorCacheTaskLocation(hostName, s"exec-$hostName").toString)) + } + + val rdd1 = spark.sparkContext.makeRDD(1 to 10, numPartitions) + val rdd2 = spark.sparkContext.makeRDD((1 to 10).map(_.toString), numPartitions) + val rdd = rdd1.stateStoreAwareZipPartitions(rdd2, stateInfo, storeNames, coordinatorRef) { + (left, right) => left.zip(right) + } + require(rdd.partitions.length === numPartitions) + for (partIndex <- 0 until numPartitions) { + val expectedLocations = storeNames.map { storeName => + val hostName = partitionAndStoreNameToLocation((partIndex, storeName)) + ExecutorCacheTaskLocation(hostName, s"exec-$hostName").toString + }.toSet + assert(rdd.preferredLocations(rdd.partitions(partIndex)).toSet === expectedLocations) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala new file mode 100644 index 000000000000..1fe639fcf284 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -0,0 +1,471 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.util.UUID + +import scala.collection.mutable +import scala.language.reflectiveCalls + +import org.scalactic.TolerantNumerics +import org.scalatest.BeforeAndAfter +import org.scalatest.PrivateMethodTester._ +import org.scalatest.concurrent.AsyncAssertions.Waiter +import org.scalatest.concurrent.PatienceConfiguration.Timeout + +import org.apache.spark.SparkException +import org.apache.spark.scheduler._ +import org.apache.spark.sql.{Encoder, SparkSession} +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.StreamingQueryListener._ +import org.apache.spark.sql.streaming.util.StreamManualClock +import org.apache.spark.util.JsonProtocol + +class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { + + import testImplicits._ + + // To make === between double tolerate inexact values + implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) + + after { + spark.streams.active.foreach(_.stop()) + assert(spark.streams.active.isEmpty) + assert(addedListeners().isEmpty) + // Make sure we don't leak any events to the next test + spark.sparkContext.listenerBus.waitUntilEmpty(10000) + } + + testQuietly("single listener, check trigger events are generated correctly") { + val clock = new StreamManualClock + val inputData = new MemoryStream[Int](0, sqlContext) + val df = inputData.toDS().as[Long].map { 10 / _ } + val listener = new EventCollector + + case class AssertStreamExecThreadToWaitForClock() + extends AssertOnQuery(q => { + eventually(Timeout(streamingTimeout)) { + if (q.exception.isEmpty) { + assert(clock.asInstanceOf[StreamManualClock].isStreamWaitingAt(clock.getTimeMillis)) + } + } + if (q.exception.isDefined) { + throw q.exception.get + } + true + }, "") + + try { + // No events until started + spark.streams.addListener(listener) + assert(listener.startEvent === null) + assert(listener.progressEvents.isEmpty) + assert(listener.terminationEvent === null) + + testStream(df, OutputMode.Append)( + + // Start event generated when query started + StartStream(ProcessingTime(100), triggerClock = clock), + AssertOnQuery { query => + assert(listener.startEvent !== null) + assert(listener.startEvent.id === query.id) + assert(listener.startEvent.runId === query.runId) + assert(listener.startEvent.name === query.name) + assert(listener.progressEvents.isEmpty) + assert(listener.terminationEvent === null) + true + }, + + // Progress event generated when data processed + AddData(inputData, 1, 2), + AdvanceManualClock(100), + AssertStreamExecThreadToWaitForClock(), + CheckAnswer(10, 5), + AssertOnQuery { query => + assert(listener.progressEvents.nonEmpty) + // SPARK-18868: We can't use query.lastProgress, because in progressEvents, we filter + // out non-zero input rows, but the lastProgress may be a zero input row trigger + val lastNonZeroProgress = query.recentProgress.filter(_.numInputRows > 0).lastOption + .getOrElse(fail("No progress updates received in StreamingQuery!")) + assert(listener.progressEvents.last.json === lastNonZeroProgress.json) + assert(listener.terminationEvent === null) + true + }, + + // Termination event generated when stopped cleanly + StopStream, + AssertOnQuery { query => + eventually(Timeout(streamingTimeout)) { + assert(listener.terminationEvent !== null) + assert(listener.terminationEvent.id === query.id) + assert(listener.terminationEvent.runId === query.runId) + assert(listener.terminationEvent.exception === None) + } + listener.checkAsyncErrors() + listener.reset() + true + }, + + // Termination event generated with exception message when stopped with error + StartStream(ProcessingTime(100), triggerClock = clock), + AssertStreamExecThreadToWaitForClock(), + AddData(inputData, 0), + AdvanceManualClock(100), // process bad data + ExpectFailure[SparkException](), + AssertOnQuery { query => + eventually(Timeout(streamingTimeout)) { + assert(listener.terminationEvent !== null) + assert(listener.terminationEvent.id === query.id) + assert(listener.terminationEvent.exception.nonEmpty) + // Make sure that the exception message reported through listener + // contains the actual exception and relevant stack trace + assert(!listener.terminationEvent.exception.get.contains("StreamingQueryException")) + assert( + listener.terminationEvent.exception.get.contains("java.lang.ArithmeticException")) + assert(listener.terminationEvent.exception.get.contains("StreamingQueryListenerSuite")) + } + listener.checkAsyncErrors() + true + } + ) + } finally { + spark.streams.removeListener(listener) + } + } + + test("SPARK-19594: all of listeners should receive QueryTerminatedEvent") { + val df = MemoryStream[Int].toDS().as[Long] + val listeners = (1 to 5).map(_ => new EventCollector) + try { + listeners.foreach(listener => spark.streams.addListener(listener)) + testStream(df, OutputMode.Append)( + StartStream(), + StopStream, + AssertOnQuery { query => + eventually(Timeout(streamingTimeout)) { + listeners.foreach(listener => assert(listener.terminationEvent !== null)) + listeners.foreach(listener => assert(listener.terminationEvent.id === query.id)) + listeners.foreach(listener => assert(listener.terminationEvent.runId === query.runId)) + listeners.foreach(listener => assert(listener.terminationEvent.exception === None)) + } + listeners.foreach(listener => listener.checkAsyncErrors()) + listeners.foreach(listener => listener.reset()) + true + } + ) + } finally { + listeners.foreach(spark.streams.removeListener) + } + } + + test("adding and removing listener") { + def isListenerActive(listener: EventCollector): Boolean = { + listener.reset() + testStream(MemoryStream[Int].toDS)( + StartStream(), + StopStream + ) + listener.startEvent != null + } + + try { + val listener1 = new EventCollector + val listener2 = new EventCollector + + spark.streams.addListener(listener1) + assert(isListenerActive(listener1) === true) + assert(isListenerActive(listener2) === false) + spark.streams.addListener(listener2) + assert(isListenerActive(listener1) === true) + assert(isListenerActive(listener2) === true) + spark.streams.removeListener(listener1) + assert(isListenerActive(listener1) === false) + assert(isListenerActive(listener2) === true) + } finally { + addedListeners().foreach(spark.streams.removeListener) + } + } + + test("event ordering") { + val listener = new EventCollector + withListenerAdded(listener) { + for (i <- 1 to 100) { + listener.reset() + require(listener.startEvent === null) + testStream(MemoryStream[Int].toDS)( + StartStream(), + Assert(listener.startEvent !== null, "onQueryStarted not called before query returned"), + StopStream, + Assert { listener.checkAsyncErrors() } + ) + } + } + } + + test("QueryStartedEvent serialization") { + def testSerialization(event: QueryStartedEvent): Unit = { + val json = JsonProtocol.sparkEventToJson(event) + val newEvent = JsonProtocol.sparkEventFromJson(json).asInstanceOf[QueryStartedEvent] + assert(newEvent.id === event.id) + assert(newEvent.runId === event.runId) + assert(newEvent.name === event.name) + } + + testSerialization(new QueryStartedEvent(UUID.randomUUID, UUID.randomUUID, "name")) + testSerialization(new QueryStartedEvent(UUID.randomUUID, UUID.randomUUID, null)) + } + + test("QueryProgressEvent serialization") { + def testSerialization(event: QueryProgressEvent): Unit = { + import scala.collection.JavaConverters._ + val json = JsonProtocol.sparkEventToJson(event) + val newEvent = JsonProtocol.sparkEventFromJson(json).asInstanceOf[QueryProgressEvent] + assert(newEvent.progress.json === event.progress.json) // json as a proxy for equality + assert(newEvent.progress.durationMs.asScala === event.progress.durationMs.asScala) + assert(newEvent.progress.eventTime.asScala === event.progress.eventTime.asScala) + } + testSerialization(new QueryProgressEvent(StreamingQueryStatusAndProgressSuite.testProgress1)) + testSerialization(new QueryProgressEvent(StreamingQueryStatusAndProgressSuite.testProgress2)) + } + + test("QueryTerminatedEvent serialization") { + def testSerialization(event: QueryTerminatedEvent): Unit = { + val json = JsonProtocol.sparkEventToJson(event) + val newEvent = JsonProtocol.sparkEventFromJson(json).asInstanceOf[QueryTerminatedEvent] + assert(newEvent.id === event.id) + assert(newEvent.runId === event.runId) + assert(newEvent.exception === event.exception) + } + + val exception = new RuntimeException("exception") + testSerialization( + new QueryTerminatedEvent(UUID.randomUUID, UUID.randomUUID, Some(exception.getMessage))) + } + + test("only one progress event per interval when no data") { + // This test will start a query but not push any data, and then check if we push too many events + withSQLConf(SQLConf.STREAMING_NO_DATA_PROGRESS_EVENT_INTERVAL.key -> "100ms") { + @volatile var numProgressEvent = 0 + val listener = new StreamingQueryListener { + override def onQueryStarted(event: QueryStartedEvent): Unit = {} + override def onQueryProgress(event: QueryProgressEvent): Unit = { + numProgressEvent += 1 + } + override def onQueryTerminated(event: QueryTerminatedEvent): Unit = {} + } + spark.streams.addListener(listener) + try { + val input = new MemoryStream[Int](0, sqlContext) { + @volatile var numTriggers = 0 + override def getOffset: Option[Offset] = { + numTriggers += 1 + super.getOffset + } + } + val clock = new StreamManualClock() + val actions = mutable.ArrayBuffer[StreamAction]() + actions += StartStream(trigger = ProcessingTime(10), triggerClock = clock) + for (_ <- 1 to 100) { + actions += AdvanceManualClock(10) + } + actions += AssertOnQuery { _ => + eventually(timeout(streamingTimeout)) { + assert(input.numTriggers > 100) // at least 100 triggers have occurred + } + true + } + // `recentProgress` should not receive too many no data events + actions += AssertOnQuery { q => + q.recentProgress.size > 1 && q.recentProgress.size <= 11 + } + testStream(input.toDS)(actions: _*) + spark.sparkContext.listenerBus.waitUntilEmpty(10000) + // 11 is the max value of the possible numbers of events. + assert(numProgressEvent > 1 && numProgressEvent <= 11) + } finally { + spark.streams.removeListener(listener) + } + } + } + + test("listener only posts events from queries started in the related sessions") { + val session1 = spark.newSession() + val session2 = spark.newSession() + val collector1 = new EventCollector + val collector2 = new EventCollector + + def runQuery(session: SparkSession): Unit = { + collector1.reset() + collector2.reset() + val mem = MemoryStream[Int](implicitly[Encoder[Int]], session.sqlContext) + testStream(mem.toDS)( + AddData(mem, 1, 2, 3), + CheckAnswer(1, 2, 3) + ) + session.sparkContext.listenerBus.waitUntilEmpty(5000) + } + + def assertEventsCollected(collector: EventCollector): Unit = { + assert(collector.startEvent !== null) + assert(collector.progressEvents.nonEmpty) + assert(collector.terminationEvent !== null) + } + + def assertEventsNotCollected(collector: EventCollector): Unit = { + assert(collector.startEvent === null) + assert(collector.progressEvents.isEmpty) + assert(collector.terminationEvent === null) + } + + assert(session1.ne(session2)) + assert(session1.streams.ne(session2.streams)) + + withListenerAdded(collector1, session1) { + assert(addedListeners(session1).nonEmpty) + + withListenerAdded(collector2, session2) { + assert(addedListeners(session2).nonEmpty) + + // query on session1 should send events only to collector1 + runQuery(session1) + assertEventsCollected(collector1) + assertEventsNotCollected(collector2) + + // query on session2 should send events only to collector2 + runQuery(session2) + assertEventsCollected(collector2) + assertEventsNotCollected(collector1) + } + } + } + + testQuietly("ReplayListenerBus should ignore broken event jsons generated in 2.0.0") { + // query-event-logs-version-2.0.0.txt has all types of events generated by + // Structured Streaming in Spark 2.0.0. + // SparkListenerApplicationEnd is the only valid event and it's the last event. We use it + // to verify that we can skip broken jsons generated by Structured Streaming. + testReplayListenerBusWithBorkenEventJsons("query-event-logs-version-2.0.0.txt") + } + + testQuietly("ReplayListenerBus should ignore broken event jsons generated in 2.0.1") { + // query-event-logs-version-2.0.1.txt has all types of events generated by + // Structured Streaming in Spark 2.0.1. + // SparkListenerApplicationEnd is the only valid event and it's the last event. We use it + // to verify that we can skip broken jsons generated by Structured Streaming. + testReplayListenerBusWithBorkenEventJsons("query-event-logs-version-2.0.1.txt") + } + + testQuietly("ReplayListenerBus should ignore broken event jsons generated in 2.0.2") { + // query-event-logs-version-2.0.2.txt has all types of events generated by + // Structured Streaming in Spark 2.0.2. + // SparkListenerApplicationEnd is the only valid event and it's the last event. We use it + // to verify that we can skip broken jsons generated by Structured Streaming. + testReplayListenerBusWithBorkenEventJsons("query-event-logs-version-2.0.2.txt") + } + + private def testReplayListenerBusWithBorkenEventJsons(fileName: String): Unit = { + val input = getClass.getResourceAsStream(s"/structured-streaming/$fileName") + val events = mutable.ArrayBuffer[SparkListenerEvent]() + try { + val replayer = new ReplayListenerBus() { + // Redirect all parsed events to `events` + override def doPostEvent( + listener: SparkListenerInterface, + event: SparkListenerEvent): Unit = { + events += event + } + } + // Add a dummy listener so that "doPostEvent" will be called. + replayer.addListener(new SparkListener {}) + replayer.replay(input, fileName) + // SparkListenerApplicationEnd is the only valid event + assert(events.size === 1) + assert(events(0).isInstanceOf[SparkListenerApplicationEnd]) + } finally { + input.close() + } + } + + private def withListenerAdded( + listener: StreamingQueryListener, + session: SparkSession = spark)(body: => Unit): Unit = { + try { + failAfter(streamingTimeout) { + session.streams.addListener(listener) + body + } + } finally { + session.streams.removeListener(listener) + } + } + + private def addedListeners(session: SparkSession = spark): Array[StreamingQueryListener] = { + val listenerBusMethod = + PrivateMethod[StreamingQueryListenerBus]('listenerBus) + val listenerBus = session.streams invokePrivate listenerBusMethod() + listenerBus.listeners.toArray.map(_.asInstanceOf[StreamingQueryListener]) + } + + /** Collects events from the StreamingQueryListener for testing */ + class EventCollector extends StreamingQueryListener { + // to catch errors in the async listener events + @volatile private var asyncTestWaiter = new Waiter + + @volatile var startEvent: QueryStartedEvent = null + @volatile var terminationEvent: QueryTerminatedEvent = null + + private val _progressEvents = new mutable.Queue[StreamingQueryProgress] + + def progressEvents: Seq[StreamingQueryProgress] = _progressEvents.synchronized { + _progressEvents.filter(_.numInputRows > 0) + } + + def reset(): Unit = { + startEvent = null + terminationEvent = null + _progressEvents.clear() + asyncTestWaiter = new Waiter + } + + def checkAsyncErrors(): Unit = { + asyncTestWaiter.await(timeout(streamingTimeout)) + } + + override def onQueryStarted(queryStarted: QueryStartedEvent): Unit = { + asyncTestWaiter { + startEvent = queryStarted + } + } + + override def onQueryProgress(queryProgress: QueryProgressEvent): Unit = { + asyncTestWaiter { + assert(startEvent != null, "onQueryProgress called before onQueryStarted") + _progressEvents.synchronized { _progressEvents += queryProgress.progress } + } + } + + override def onQueryTerminated(queryTerminated: QueryTerminatedEvent): Unit = { + asyncTestWaiter { + assert(startEvent != null, "onQueryTerminated called before onQueryStarted") + terminationEvent = queryTerminated + } + asyncTestWaiter.dismiss() + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala new file mode 100644 index 000000000000..46eec736d402 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala @@ -0,0 +1,323 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.util.concurrent.CountDownLatch + +import scala.concurrent.Future +import scala.util.Random +import scala.util.control.NonFatal + +import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.Eventually._ +import org.scalatest.concurrent.PatienceConfiguration.Timeout +import org.scalatest.time.Span +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.SparkException +import org.apache.spark.sql.{AnalysisException, Dataset} +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.util.BlockingSource +import org.apache.spark.util.Utils + +class StreamingQueryManagerSuite extends StreamTest with BeforeAndAfter { + + import AwaitTerminationTester._ + import testImplicits._ + + override val streamingTimeout = 20.seconds + + before { + assert(spark.streams.active.isEmpty) + spark.streams.resetTerminated() + } + + after { + assert(spark.streams.active.isEmpty) + spark.streams.resetTerminated() + } + + testQuietly("listing") { + val (m1, ds1) = makeDataset + val (m2, ds2) = makeDataset + val (m3, ds3) = makeDataset + + withQueriesOn(ds1, ds2, ds3) { queries => + require(queries.size === 3) + assert(spark.streams.active.toSet === queries.toSet) + val (q1, q2, q3) = (queries(0), queries(1), queries(2)) + + assert(spark.streams.get(q1.id).eq(q1)) + assert(spark.streams.get(q2.id).eq(q2)) + assert(spark.streams.get(q3.id).eq(q3)) + assert(spark.streams.get(java.util.UUID.randomUUID()) === null) // non-existent id + q1.stop() + + assert(spark.streams.active.toSet === Set(q2, q3)) + assert(spark.streams.get(q1.id) === null) + assert(spark.streams.get(q2.id).eq(q2)) + + m2.addData(0) // q2 should terminate with error + + eventually(Timeout(streamingTimeout)) { + require(!q2.isActive) + require(q2.exception.isDefined) + assert(spark.streams.get(q2.id) === null) + assert(spark.streams.active.toSet === Set(q3)) + } + } + } + + testQuietly("awaitAnyTermination without timeout and resetTerminated") { + val datasets = Seq.fill(5)(makeDataset._2) + withQueriesOn(datasets: _*) { queries => + require(queries.size === datasets.size) + assert(spark.streams.active.toSet === queries.toSet) + + // awaitAnyTermination should be blocking + testAwaitAnyTermination(ExpectBlocked) + + // Stop a query asynchronously and see if it is reported through awaitAnyTermination + val q1 = stopRandomQueryAsync(stopAfter = 100 milliseconds, withError = false) + testAwaitAnyTermination(ExpectNotBlocked) + require(!q1.isActive) // should be inactive by the time the prev awaitAnyTerm returned + + // All subsequent calls to awaitAnyTermination should be non-blocking + testAwaitAnyTermination(ExpectNotBlocked) + + // Resetting termination should make awaitAnyTermination() blocking again + spark.streams.resetTerminated() + testAwaitAnyTermination(ExpectBlocked) + + // Terminate a query asynchronously with exception and see awaitAnyTermination throws + // the exception + val q2 = stopRandomQueryAsync(100 milliseconds, withError = true) + testAwaitAnyTermination(ExpectException[SparkException]) + require(!q2.isActive) // should be inactive by the time the prev awaitAnyTerm returned + + // All subsequent calls to awaitAnyTermination should throw the exception + testAwaitAnyTermination(ExpectException[SparkException]) + + // Resetting termination should make awaitAnyTermination() blocking again + spark.streams.resetTerminated() + testAwaitAnyTermination(ExpectBlocked) + + // Terminate multiple queries, one with failure and see whether awaitAnyTermination throws + // the exception + val q3 = stopRandomQueryAsync(10 milliseconds, withError = false) + testAwaitAnyTermination(ExpectNotBlocked) + require(!q3.isActive) + val q4 = stopRandomQueryAsync(10 milliseconds, withError = true) + eventually(Timeout(streamingTimeout)) { require(!q4.isActive) } + // After q4 terminates with exception, awaitAnyTerm should start throwing exception + testAwaitAnyTermination(ExpectException[SparkException]) + } + } + + testQuietly("awaitAnyTermination with timeout and resetTerminated") { + val datasets = Seq.fill(6)(makeDataset._2) + withQueriesOn(datasets: _*) { queries => + require(queries.size === datasets.size) + assert(spark.streams.active.toSet === queries.toSet) + + // awaitAnyTermination should be blocking or non-blocking depending on timeout values + testAwaitAnyTermination( + ExpectBlocked, + awaitTimeout = 4 seconds, + expectedReturnedValue = false, + testBehaviorFor = 2 seconds) + + testAwaitAnyTermination( + ExpectNotBlocked, + awaitTimeout = 50 milliseconds, + expectedReturnedValue = false, + testBehaviorFor = 1 second) + + // Stop a query asynchronously within timeout and awaitAnyTerm should be unblocked + val q1 = stopRandomQueryAsync(stopAfter = 100 milliseconds, withError = false) + testAwaitAnyTermination( + ExpectNotBlocked, + awaitTimeout = 2 seconds, + expectedReturnedValue = true, + testBehaviorFor = 4 seconds) + require(!q1.isActive) // should be inactive by the time the prev awaitAnyTerm returned + + // All subsequent calls to awaitAnyTermination should be non-blocking even if timeout is high + testAwaitAnyTermination( + ExpectNotBlocked, awaitTimeout = 4 seconds, expectedReturnedValue = true) + + // Resetting termination should make awaitAnyTermination() blocking again + spark.streams.resetTerminated() + testAwaitAnyTermination( + ExpectBlocked, + awaitTimeout = 4 seconds, + expectedReturnedValue = false, + testBehaviorFor = 1 second) + + // Terminate a query asynchronously with exception within timeout, awaitAnyTermination should + // throws the exception + val q2 = stopRandomQueryAsync(100 milliseconds, withError = true) + testAwaitAnyTermination( + ExpectException[SparkException], + awaitTimeout = 4 seconds, + testBehaviorFor = 6 seconds) + require(!q2.isActive) // should be inactive by the time the prev awaitAnyTerm returned + + // All subsequent calls to awaitAnyTermination should throw the exception + testAwaitAnyTermination( + ExpectException[SparkException], + awaitTimeout = 2 seconds, + testBehaviorFor = 4 seconds) + + // Terminate a query asynchronously outside the timeout, awaitAnyTerm should be blocked + spark.streams.resetTerminated() + val q3 = stopRandomQueryAsync(2 seconds, withError = true) + testAwaitAnyTermination( + ExpectNotBlocked, + awaitTimeout = 100 milliseconds, + expectedReturnedValue = false, + testBehaviorFor = 4 seconds) + + // After that query is stopped, awaitAnyTerm should throw exception + eventually(Timeout(streamingTimeout)) { require(!q3.isActive) } // wait for query to stop + testAwaitAnyTermination( + ExpectException[SparkException], + awaitTimeout = 100 milliseconds, + testBehaviorFor = 4 seconds) + + + // Terminate multiple queries, one with failure and see whether awaitAnyTermination throws + // the exception + spark.streams.resetTerminated() + + val q4 = stopRandomQueryAsync(10 milliseconds, withError = false) + testAwaitAnyTermination( + ExpectNotBlocked, awaitTimeout = 2 seconds, expectedReturnedValue = true) + require(!q4.isActive) + val q5 = stopRandomQueryAsync(10 milliseconds, withError = true) + eventually(Timeout(streamingTimeout)) { require(!q5.isActive) } + // After q5 terminates with exception, awaitAnyTerm should start throwing exception + testAwaitAnyTermination(ExpectException[SparkException], awaitTimeout = 2 seconds) + } + } + + test("SPARK-18811: Source resolution should not block main thread") { + failAfter(streamingTimeout) { + BlockingSource.latch = new CountDownLatch(1) + withTempDir { tempDir => + // if source resolution was happening on the main thread, it would block the start call, + // now it should only be blocking the stream execution thread + val sq = spark.readStream + .format("org.apache.spark.sql.streaming.util.BlockingSource") + .load() + .writeStream + .format("org.apache.spark.sql.streaming.util.BlockingSource") + .option("checkpointLocation", tempDir.toString) + .start() + eventually(Timeout(streamingTimeout)) { + assert(sq.status.message.contains("Initializing sources")) + } + BlockingSource.latch.countDown() + sq.stop() + } + } + } + + /** Run a body of code by defining a query on each dataset */ + private def withQueriesOn(datasets: Dataset[_]*)(body: Seq[StreamingQuery] => Unit): Unit = { + failAfter(streamingTimeout) { + val queries = withClue("Error starting queries") { + datasets.zipWithIndex.map { case (ds, i) => + var query: StreamingQuery = null + try { + val df = ds.toDF + val metadataRoot = + Utils.createTempDir(namePrefix = "streaming.checkpoint").getCanonicalPath + query = + df.writeStream + .format("memory") + .queryName(s"query$i") + .option("checkpointLocation", metadataRoot) + .outputMode("append") + .start() + } catch { + case NonFatal(e) => + if (query != null) query.stop() + throw e + } + query + } + } + try { + body(queries) + } finally { + queries.foreach(_.stop()) + } + } + } + + /** Test the behavior of awaitAnyTermination */ + private def testAwaitAnyTermination( + expectedBehavior: ExpectedBehavior, + expectedReturnedValue: Boolean = false, + awaitTimeout: Span = null, + testBehaviorFor: Span = 4 seconds + ): Unit = { + + def awaitTermFunc(): Unit = { + if (awaitTimeout != null && awaitTimeout.toMillis > 0) { + val returnedValue = spark.streams.awaitAnyTermination(awaitTimeout.toMillis) + assert(returnedValue === expectedReturnedValue, "Returned value does not match expected") + } else { + spark.streams.awaitAnyTermination() + } + } + + AwaitTerminationTester.test(expectedBehavior, () => awaitTermFunc(), testBehaviorFor) + } + + /** Stop a random active query either with `stop()` or with an error */ + private def stopRandomQueryAsync(stopAfter: Span, withError: Boolean): StreamingQuery = { + + import scala.concurrent.ExecutionContext.Implicits.global + + val activeQueries = spark.streams.active + val queryToStop = activeQueries(Random.nextInt(activeQueries.length)) + Future { + Thread.sleep(stopAfter.toMillis) + if (withError) { + logDebug(s"Terminating query ${queryToStop.name} with error") + queryToStop.asInstanceOf[StreamingQueryWrapper].streamingQuery.logicalPlan.collect { + case StreamingExecutionRelation(source, _) => + source.asInstanceOf[MemoryStream[Int]].addData(0) + } + } else { + logDebug(s"Stopping query ${queryToStop.name}") + queryToStop.stop() + } + } + queryToStop + } + + private def makeDataset: (MemoryStream[Int], Dataset[Int]) = { + val inputData = MemoryStream[Int] + val mapped = inputData.toDS.map(6 / _) + (inputData, mapped) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala new file mode 100644 index 000000000000..79bb827e0de9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala @@ -0,0 +1,273 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.util.UUID + +import scala.collection.JavaConverters._ +import scala.language.postfixOps + +import org.json4s._ +import org.json4s.jackson.JsonMethods._ +import org.scalatest.concurrent.Eventually +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.StreamingQueryStatusAndProgressSuite._ + +class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { + test("StreamingQueryProgress - prettyJson") { + val json1 = testProgress1.prettyJson + assertJson( + json1, + s""" + |{ + | "id" : "${testProgress1.id.toString}", + | "runId" : "${testProgress1.runId.toString}", + | "name" : "myName", + | "timestamp" : "2016-12-05T20:54:20.827Z", + | "batchId" : 2, + | "numInputRows" : 678, + | "inputRowsPerSecond" : 10.0, + | "durationMs" : { + | "total" : 0 + | }, + | "eventTime" : { + | "avg" : "2016-12-05T20:54:20.827Z", + | "max" : "2016-12-05T20:54:20.827Z", + | "min" : "2016-12-05T20:54:20.827Z", + | "watermark" : "2016-12-05T20:54:20.827Z" + | }, + | "stateOperators" : [ { + | "numRowsTotal" : 0, + | "numRowsUpdated" : 1, + | "memoryUsedBytes" : 2 + | } ], + | "sources" : [ { + | "description" : "source", + | "startOffset" : 123, + | "endOffset" : 456, + | "numInputRows" : 678, + | "inputRowsPerSecond" : 10.0 + | } ], + | "sink" : { + | "description" : "sink" + | } + |} + """.stripMargin.trim) + assert(compact(parse(json1)) === testProgress1.json) + + val json2 = testProgress2.prettyJson + assertJson( + json2, + s""" + |{ + | "id" : "${testProgress2.id.toString}", + | "runId" : "${testProgress2.runId.toString}", + | "name" : null, + | "timestamp" : "2016-12-05T20:54:20.827Z", + | "batchId" : 2, + | "numInputRows" : 678, + | "durationMs" : { + | "total" : 0 + | }, + | "stateOperators" : [ { + | "numRowsTotal" : 0, + | "numRowsUpdated" : 1, + | "memoryUsedBytes" : 2 + | } ], + | "sources" : [ { + | "description" : "source", + | "startOffset" : 123, + | "endOffset" : 456, + | "numInputRows" : 678 + | } ], + | "sink" : { + | "description" : "sink" + | } + |} + """.stripMargin.trim) + assert(compact(parse(json2)) === testProgress2.json) + } + + test("StreamingQueryProgress - json") { + assert(compact(parse(testProgress1.json)) === testProgress1.json) + assert(compact(parse(testProgress2.json)) === testProgress2.json) + } + + test("StreamingQueryProgress - toString") { + assert(testProgress1.toString === testProgress1.prettyJson) + assert(testProgress2.toString === testProgress2.prettyJson) + } + + test("StreamingQueryStatus - prettyJson") { + val json = testStatus.prettyJson + assertJson( + json, + """ + |{ + | "message" : "active", + | "isDataAvailable" : true, + | "isTriggerActive" : false + |} + """.stripMargin.trim) + } + + test("StreamingQueryStatus - json") { + assert(compact(parse(testStatus.json)) === testStatus.json) + } + + test("StreamingQueryStatus - toString") { + assert(testStatus.toString === testStatus.prettyJson) + } + + test("progress classes should be Serializable") { + import testImplicits._ + + val inputData = MemoryStream[Int] + + val query = inputData.toDS() + .groupBy($"value") + .agg(count("*")) + .writeStream + .queryName("progress_serializable_test") + .format("memory") + .outputMode("complete") + .start() + try { + inputData.addData(1, 2, 3) + query.processAllAvailable() + + val progress = query.recentProgress + + // Make sure it generates the progress objects we want to test + assert(progress.exists { p => + p.sources.size >= 1 && p.stateOperators.size >= 1 && p.sink != null + }) + + val array = spark.sparkContext.parallelize(progress).collect() + assert(array.length === progress.length) + array.zip(progress).foreach { case (p1, p2) => + // Make sure we did serialize and deserialize the object + assert(p1 ne p2) + assert(p1.json === p2.json) + } + } finally { + query.stop() + } + } + + test("SPARK-19378: Continue reporting stateOp metrics even if there is no active trigger") { + import testImplicits._ + + withSQLConf(SQLConf.STREAMING_NO_DATA_PROGRESS_EVENT_INTERVAL.key -> "10") { + val inputData = MemoryStream[Int] + + val query = inputData.toDS().toDF("value") + .select('value) + .groupBy($"value") + .agg(count("*")) + .writeStream + .queryName("metric_continuity") + .format("memory") + .outputMode("complete") + .start() + try { + inputData.addData(1, 2) + query.processAllAvailable() + + val progress = query.lastProgress + assert(progress.stateOperators.length > 0) + // Should emit new progresses every 10 ms, but we could be facing a slow Jenkins + eventually(timeout(1 minute)) { + val nextProgress = query.lastProgress + assert(nextProgress.timestamp !== progress.timestamp) + assert(nextProgress.numInputRows === 0) + assert(nextProgress.stateOperators.head.numRowsTotal === 2) + assert(nextProgress.stateOperators.head.numRowsUpdated === 0) + } + } finally { + query.stop() + } + } + } + + def assertJson(source: String, expected: String): Unit = { + assert( + source.replaceAll("\r\n|\r|\n", System.lineSeparator) === + expected.replaceAll("\r\n|\r|\n", System.lineSeparator)) + } +} + +object StreamingQueryStatusAndProgressSuite { + val testProgress1 = new StreamingQueryProgress( + id = UUID.randomUUID, + runId = UUID.randomUUID, + name = "myName", + timestamp = "2016-12-05T20:54:20.827Z", + batchId = 2L, + durationMs = new java.util.HashMap(Map("total" -> 0L).mapValues(long2Long).asJava), + eventTime = new java.util.HashMap(Map( + "max" -> "2016-12-05T20:54:20.827Z", + "min" -> "2016-12-05T20:54:20.827Z", + "avg" -> "2016-12-05T20:54:20.827Z", + "watermark" -> "2016-12-05T20:54:20.827Z").asJava), + stateOperators = Array(new StateOperatorProgress( + numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 2)), + sources = Array( + new SourceProgress( + description = "source", + startOffset = "123", + endOffset = "456", + numInputRows = 678, + inputRowsPerSecond = 10.0, + processedRowsPerSecond = Double.PositiveInfinity // should not be present in the json + ) + ), + sink = new SinkProgress("sink") + ) + + val testProgress2 = new StreamingQueryProgress( + id = UUID.randomUUID, + runId = UUID.randomUUID, + name = null, // should not be present in the json + timestamp = "2016-12-05T20:54:20.827Z", + batchId = 2L, + durationMs = new java.util.HashMap(Map("total" -> 0L).mapValues(long2Long).asJava), + // empty maps should be handled correctly + eventTime = new java.util.HashMap(Map.empty[String, String].asJava), + stateOperators = Array(new StateOperatorProgress( + numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 2)), + sources = Array( + new SourceProgress( + description = "source", + startOffset = "123", + endOffset = "456", + numInputRows = 678, + inputRowsPerSecond = Double.NaN, // should not be present in the json + processedRowsPerSecond = Double.NegativeInfinity // should not be present in the json + ) + ), + sink = new SinkProgress("sink") + ) + + val testStatus = new StreamingQueryStatus("active", true, false) +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala new file mode 100644 index 000000000000..ab35079dca23 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -0,0 +1,743 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming + +import java.util.concurrent.CountDownLatch + +import org.apache.commons.lang3.RandomStringUtils +import org.mockito.Mockito._ +import org.scalactic.TolerantNumerics +import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.Eventually._ +import org.scalatest.concurrent.PatienceConfiguration.Timeout +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.util.{BlockingSource, MockSourceProvider, StreamManualClock} +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.ManualClock + + +class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging with MockitoSugar { + + import AwaitTerminationTester._ + import testImplicits._ + + // To make === between double tolerate inexact values + implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) + + after { + sqlContext.streams.active.foreach(_.stop()) + } + + test("name unique in active queries") { + withTempDir { dir => + def startQuery(name: Option[String]): StreamingQuery = { + val writer = MemoryStream[Int].toDS.writeStream + name.foreach(writer.queryName) + writer + .foreach(new TestForeachWriter) + .start() + } + + // No name by default, multiple active queries can have no name + val q1 = startQuery(name = None) + assert(q1.name === null) + val q2 = startQuery(name = None) + assert(q2.name === null) + + // Can be set by user + val q3 = startQuery(name = Some("q3")) + assert(q3.name === "q3") + + // Multiple active queries cannot have same name + val e = intercept[IllegalArgumentException] { + startQuery(name = Some("q3")) + } + + q1.stop() + q2.stop() + q3.stop() + } + } + + test( + "id unique in active queries + persists across restarts, runId unique across start/restarts") { + val inputData = MemoryStream[Int] + withTempDir { dir => + var cpDir: String = null + + def startQuery(restart: Boolean): StreamingQuery = { + if (cpDir == null || !restart) cpDir = s"$dir/${RandomStringUtils.randomAlphabetic(10)}" + MemoryStream[Int].toDS().groupBy().count() + .writeStream + .format("memory") + .outputMode("complete") + .queryName(s"name${RandomStringUtils.randomAlphabetic(10)}") + .option("checkpointLocation", cpDir) + .start() + } + + // id and runId unique for new queries + val q1 = startQuery(restart = false) + val q2 = startQuery(restart = false) + assert(q1.id !== q2.id) + assert(q1.runId !== q2.runId) + q1.stop() + q2.stop() + + // id persists across restarts, runId unique across restarts + val q3 = startQuery(restart = false) + q3.stop() + + val q4 = startQuery(restart = true) + q4.stop() + assert(q3.id === q3.id) + assert(q3.runId !== q4.runId) + + // Only one query with same id can be active + val q5 = startQuery(restart = false) + val e = intercept[IllegalStateException] { + startQuery(restart = true) + } + } + } + + testQuietly("isActive, exception, and awaitTermination") { + val inputData = MemoryStream[Int] + val mapped = inputData.toDS().map { 6 / _} + + testStream(mapped)( + AssertOnQuery(_.isActive === true), + AssertOnQuery(_.exception.isEmpty), + AddData(inputData, 1, 2), + CheckAnswer(6, 3), + TestAwaitTermination(ExpectBlocked), + TestAwaitTermination(ExpectBlocked, timeoutMs = 2000), + TestAwaitTermination(ExpectNotBlocked, timeoutMs = 10, expectedReturnValue = false), + StopStream, + AssertOnQuery(_.isActive === false), + AssertOnQuery(_.exception.isEmpty), + TestAwaitTermination(ExpectNotBlocked), + TestAwaitTermination(ExpectNotBlocked, timeoutMs = 2000, expectedReturnValue = true), + TestAwaitTermination(ExpectNotBlocked, timeoutMs = 10, expectedReturnValue = true), + StartStream(), + AssertOnQuery(_.isActive === true), + AddData(inputData, 0), + ExpectFailure[SparkException](), + AssertOnQuery(_.isActive === false), + TestAwaitTermination(ExpectException[SparkException]), + TestAwaitTermination(ExpectException[SparkException], timeoutMs = 2000), + TestAwaitTermination(ExpectException[SparkException], timeoutMs = 10), + AssertOnQuery(q => { + q.exception.get.startOffset === + q.committedOffsets.toOffsetSeq(Seq(inputData), OffsetSeqMetadata()).toString && + q.exception.get.endOffset === + q.availableOffsets.toOffsetSeq(Seq(inputData), OffsetSeqMetadata()).toString + }, "incorrect start offset or end offset on exception") + ) + } + + testQuietly("OneTime trigger, commit log, and exception") { + import Trigger.Once + val inputData = MemoryStream[Int] + val mapped = inputData.toDS().map { 6 / _} + + testStream(mapped)( + AssertOnQuery(_.isActive === true), + StopStream, + AddData(inputData, 1, 2), + StartStream(trigger = Once), + CheckAnswer(6, 3), + StopStream, // clears out StreamTest state + AssertOnQuery { q => + // both commit log and offset log contain the same (latest) batch id + q.batchCommitLog.getLatest().map(_._1).getOrElse(-1L) == + q.offsetLog.getLatest().map(_._1).getOrElse(-2L) + }, + AssertOnQuery { q => + // blow away commit log and sink result + q.batchCommitLog.purge(1) + q.sink.asInstanceOf[MemorySink].clear() + true + }, + StartStream(trigger = Once), + CheckAnswer(6, 3), // ensure we fall back to offset log and reprocess batch + StopStream, + AddData(inputData, 3), + StartStream(trigger = Once), + CheckLastBatch(2), // commit log should be back in place + StopStream, + AddData(inputData, 0), + StartStream(trigger = Once), + ExpectFailure[SparkException](), + AssertOnQuery(_.isActive === false), + AssertOnQuery(q => { + q.exception.get.startOffset === + q.committedOffsets.toOffsetSeq(Seq(inputData), OffsetSeqMetadata()).toString && + q.exception.get.endOffset === + q.availableOffsets.toOffsetSeq(Seq(inputData), OffsetSeqMetadata()).toString + }, "incorrect start offset or end offset on exception") + ) + } + + testQuietly("status, lastProgress, and recentProgress") { + import StreamingQuerySuite._ + clock = new StreamManualClock + + /** Custom MemoryStream that waits for manual clock to reach a time */ + val inputData = new MemoryStream[Int](0, sqlContext) { + // getOffset should take 50 ms the first time it is called + override def getOffset: Option[Offset] = { + val offset = super.getOffset + if (offset.nonEmpty) { + clock.waitTillTime(1050) + } + offset + } + + // getBatch should take 100 ms the first time it is called + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + if (start.isEmpty) clock.waitTillTime(1150) + super.getBatch(start, end) + } + } + + // query execution should take 350 ms the first time it is called + val mapped = inputData.toDS.coalesce(1).as[Long].map { x => + clock.waitTillTime(1500) // this will only wait the first time when clock < 1500 + 10 / x + }.agg(count("*")).as[Long] + + case class AssertStreamExecThreadIsWaitingForTime(targetTime: Long) + extends AssertOnQuery(q => { + eventually(Timeout(streamingTimeout)) { + if (q.exception.isEmpty) { + assert(clock.isStreamWaitingFor(targetTime)) + } + } + if (q.exception.isDefined) { + throw q.exception.get + } + true + }, "") { + override def toString: String = s"AssertStreamExecThreadIsWaitingForTime($targetTime)" + } + + case class AssertClockTime(time: Long) + extends AssertOnQuery(q => clock.getTimeMillis() === time, "") { + override def toString: String = s"AssertClockTime($time)" + } + + var lastProgressBeforeStop: StreamingQueryProgress = null + + testStream(mapped, OutputMode.Complete)( + StartStream(ProcessingTime(1000), triggerClock = clock), + AssertStreamExecThreadIsWaitingForTime(1000), + AssertOnQuery(_.status.isDataAvailable === false), + AssertOnQuery(_.status.isTriggerActive === false), + AssertOnQuery(_.status.message === "Waiting for next trigger"), + AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0), + + // Test status and progress while offset is being fetched + AddData(inputData, 1, 2), + AdvanceManualClock(1000), // time = 1000 to start new trigger, will block on getOffset + AssertStreamExecThreadIsWaitingForTime(1050), + AssertOnQuery(_.status.isDataAvailable === false), + AssertOnQuery(_.status.isTriggerActive === true), + AssertOnQuery(_.status.message.startsWith("Getting offsets from")), + AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0), + + // Test status and progress while batch is being fetched + AdvanceManualClock(50), // time = 1050 to unblock getOffset + AssertClockTime(1050), + AssertStreamExecThreadIsWaitingForTime(1150), // will block on getBatch that needs 1150 + AssertOnQuery(_.status.isDataAvailable === true), + AssertOnQuery(_.status.isTriggerActive === true), + AssertOnQuery(_.status.message === "Processing new data"), + AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0), + + // Test status and progress while batch is being processed + AdvanceManualClock(100), // time = 1150 to unblock getBatch + AssertClockTime(1150), + AssertStreamExecThreadIsWaitingForTime(1500), // will block in Spark job that needs 1500 + AssertOnQuery(_.status.isDataAvailable === true), + AssertOnQuery(_.status.isTriggerActive === true), + AssertOnQuery(_.status.message === "Processing new data"), + AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0), + + // Test status and progress while batch processing has completed + AssertOnQuery { _ => clock.getTimeMillis() === 1150 }, + AdvanceManualClock(350), // time = 1500 to unblock job + AssertClockTime(1500), + CheckAnswer(2), + AssertStreamExecThreadIsWaitingForTime(2000), + AssertOnQuery(_.status.isDataAvailable === true), + AssertOnQuery(_.status.isTriggerActive === false), + AssertOnQuery(_.status.message === "Waiting for next trigger"), + AssertOnQuery { query => + assert(query.lastProgress != null) + assert(query.recentProgress.exists(_.numInputRows > 0)) + assert(query.recentProgress.last.eq(query.lastProgress)) + + val progress = query.lastProgress + assert(progress.id === query.id) + assert(progress.name === query.name) + assert(progress.batchId === 0) + assert(progress.timestamp === "1970-01-01T00:00:01.000Z") // 100 ms in UTC + assert(progress.numInputRows === 2) + assert(progress.processedRowsPerSecond === 4.0) + + assert(progress.durationMs.get("getOffset") === 50) + assert(progress.durationMs.get("getBatch") === 100) + assert(progress.durationMs.get("queryPlanning") === 0) + assert(progress.durationMs.get("walCommit") === 0) + assert(progress.durationMs.get("triggerExecution") === 500) + + assert(progress.sources.length === 1) + assert(progress.sources(0).description contains "MemoryStream") + assert(progress.sources(0).startOffset === null) + assert(progress.sources(0).endOffset !== null) + assert(progress.sources(0).processedRowsPerSecond === 4.0) // 2 rows processed in 500 ms + + assert(progress.stateOperators.length === 1) + assert(progress.stateOperators(0).numRowsUpdated === 1) + assert(progress.stateOperators(0).numRowsTotal === 1) + + assert(progress.sink.description contains "MemorySink") + true + }, + + // Test whether input rate is updated after two batches + AssertStreamExecThreadIsWaitingForTime(2000), // blocked waiting for next trigger time + AddData(inputData, 1, 2), + AdvanceManualClock(500), // allow another trigger + AssertClockTime(2000), + AssertStreamExecThreadIsWaitingForTime(3000), // will block waiting for next trigger time + CheckAnswer(4), + AssertOnQuery(_.status.isDataAvailable === true), + AssertOnQuery(_.status.isTriggerActive === false), + AssertOnQuery(_.status.message === "Waiting for next trigger"), + AssertOnQuery { query => + assert(query.recentProgress.last.eq(query.lastProgress)) + assert(query.lastProgress.batchId === 1) + assert(query.lastProgress.inputRowsPerSecond === 2.0) + assert(query.lastProgress.sources(0).inputRowsPerSecond === 2.0) + true + }, + + // Test status and progress after data is not available for a trigger + AdvanceManualClock(1000), // allow another trigger + AssertStreamExecThreadIsWaitingForTime(4000), + AssertOnQuery(_.status.isDataAvailable === false), + AssertOnQuery(_.status.isTriggerActive === false), + AssertOnQuery(_.status.message === "Waiting for next trigger"), + + // Test status and progress after query stopped + AssertOnQuery { query => + lastProgressBeforeStop = query.lastProgress + true + }, + StopStream, + AssertOnQuery(_.lastProgress.json === lastProgressBeforeStop.json), + AssertOnQuery(_.status.isDataAvailable === false), + AssertOnQuery(_.status.isTriggerActive === false), + AssertOnQuery(_.status.message === "Stopped"), + + // Test status and progress after query terminated with error + StartStream(ProcessingTime(1000), triggerClock = clock), + AdvanceManualClock(1000), // ensure initial trigger completes before AddData + AddData(inputData, 0), + AdvanceManualClock(1000), // allow another trigger + ExpectFailure[SparkException](), + AssertOnQuery(_.status.isDataAvailable === false), + AssertOnQuery(_.status.isTriggerActive === false), + AssertOnQuery(_.status.message.startsWith("Terminated with exception")) + ) + } + + test("lastProgress should be null when recentProgress is empty") { + BlockingSource.latch = new CountDownLatch(1) + withTempDir { tempDir => + val sq = spark.readStream + .format("org.apache.spark.sql.streaming.util.BlockingSource") + .load() + .writeStream + .format("org.apache.spark.sql.streaming.util.BlockingSource") + .option("checkpointLocation", tempDir.toString) + .start() + // Creating source is blocked so recentProgress is empty and lastProgress should be null + assert(sq.lastProgress === null) + // Release the latch and stop the query + BlockingSource.latch.countDown() + sq.stop() + } + } + + test("codahale metrics") { + val inputData = MemoryStream[Int] + + /** Whether metrics of a query is registered for reporting */ + def isMetricsRegistered(query: StreamingQuery): Boolean = { + val sourceName = s"spark.streaming.${query.id}" + val sources = spark.sparkContext.env.metricsSystem.getSourcesByName(sourceName) + require(sources.size <= 1) + sources.nonEmpty + } + // Disabled by default + assert(spark.conf.get("spark.sql.streaming.metricsEnabled").toBoolean === false) + + withSQLConf("spark.sql.streaming.metricsEnabled" -> "false") { + testStream(inputData.toDF)( + AssertOnQuery { q => !isMetricsRegistered(q) }, + StopStream, + AssertOnQuery { q => !isMetricsRegistered(q) } + ) + } + + // Registered when enabled + withSQLConf("spark.sql.streaming.metricsEnabled" -> "true") { + testStream(inputData.toDF)( + AssertOnQuery { q => isMetricsRegistered(q) }, + StopStream, + AssertOnQuery { q => !isMetricsRegistered(q) } + ) + } + } + + test("input row calculation with mixed batch and streaming sources") { + val streamingTriggerDF = spark.createDataset(1 to 10).toDF + val streamingInputDF = createSingleTriggerStreamingDF(streamingTriggerDF).toDF("value") + val staticInputDF = spark.createDataFrame(Seq(1 -> "1", 2 -> "2")).toDF("value", "anotherValue") + + // Trigger input has 10 rows, static input has 2 rows, + // therefore after the first trigger, the calculated input rows should be 10 + val progress = getFirstProgress(streamingInputDF.join(staticInputDF, "value")) + assert(progress.numInputRows === 10) + assert(progress.sources.size === 1) + assert(progress.sources(0).numInputRows === 10) + } + + test("input row calculation with trigger input DF having multiple leaves") { + val streamingTriggerDF = + spark.createDataset(1 to 5).toDF.union(spark.createDataset(6 to 10).toDF) + require(streamingTriggerDF.logicalPlan.collectLeaves().size > 1) + val streamingInputDF = createSingleTriggerStreamingDF(streamingTriggerDF) + + // After the first trigger, the calculated input rows should be 10 + val progress = getFirstProgress(streamingInputDF) + assert(progress.numInputRows === 10) + assert(progress.sources.size === 1) + assert(progress.sources(0).numInputRows === 10) + } + + testQuietly("StreamExecution metadata garbage collection") { + val inputData = MemoryStream[Int] + val mapped = inputData.toDS().map(6 / _) + withSQLConf(SQLConf.MIN_BATCHES_TO_RETAIN.key -> "1") { + // Run 3 batches, and then assert that only 2 metadata files is are at the end + // since the first should have been purged. + testStream(mapped)( + AddData(inputData, 1, 2), + CheckAnswer(6, 3), + AddData(inputData, 1, 2), + CheckAnswer(6, 3, 6, 3), + AddData(inputData, 4, 6), + CheckAnswer(6, 3, 6, 3, 1, 1), + + AssertOnQuery("metadata log should contain only two files") { q => + val metadataLogDir = new java.io.File(q.offsetLog.metadataPath.toUri) + val logFileNames = metadataLogDir.listFiles().toSeq.map(_.getName()) + val toTest = logFileNames.filter(!_.endsWith(".crc")).sorted // Workaround for SPARK-17475 + assert(toTest.size == 2 && toTest.head == "1") + true + } + ) + } + + val inputData2 = MemoryStream[Int] + withSQLConf(SQLConf.MIN_BATCHES_TO_RETAIN.key -> "2") { + // Run 5 batches, and then assert that 3 metadata files is are at the end + // since the two should have been purged. + testStream(inputData2.toDS())( + AddData(inputData2, 1, 2), + CheckAnswer(1, 2), + AddData(inputData2, 1, 2), + CheckAnswer(1, 2, 1, 2), + AddData(inputData2, 3, 4), + CheckAnswer(1, 2, 1, 2, 3, 4), + AddData(inputData2, 5, 6), + CheckAnswer(1, 2, 1, 2, 3, 4, 5, 6), + AddData(inputData2, 7, 8), + CheckAnswer(1, 2, 1, 2, 3, 4, 5, 6, 7, 8), + + AssertOnQuery("metadata log should contain three files") { q => + val metadataLogDir = new java.io.File(q.offsetLog.metadataPath.toUri) + val logFileNames = metadataLogDir.listFiles().toSeq.map(_.getName()) + val toTest = logFileNames.filter(!_.endsWith(".crc")).sorted // Workaround for SPARK-17475 + assert(toTest.size == 3 && toTest.head == "2") + true + } + ) + } + } + + testQuietly("StreamingQuery should be Serializable but cannot be used in executors") { + def startQuery(ds: Dataset[Int], queryName: String): StreamingQuery = { + ds.writeStream + .queryName(queryName) + .format("memory") + .start() + } + + val input = MemoryStream[Int] + val q1 = startQuery(input.toDS, "stream_serializable_test_1") + val q2 = startQuery(input.toDS.map { i => + // Emulate that `StreamingQuery` get captured with normal usage unintentionally. + // It should not fail the query. + q1 + i + }, "stream_serializable_test_2") + val q3 = startQuery(input.toDS.map { i => + // Emulate that `StreamingQuery` is used in executors. We should fail the query with a clear + // error message. + q1.explain() + i + }, "stream_serializable_test_3") + try { + input.addData(1) + + // q2 should not fail since it doesn't use `q1` in the closure + q2.processAllAvailable() + + // The user calls `StreamingQuery` in the closure and it should fail + val e = intercept[StreamingQueryException] { + q3.processAllAvailable() + } + assert(e.getCause.isInstanceOf[SparkException]) + assert(e.getCause.getCause.isInstanceOf[IllegalStateException]) + assert(e.getMessage.contains("StreamingQuery cannot be used in executors")) + } finally { + q1.stop() + q2.stop() + q3.stop() + } + } + + test("StreamExecution should call stop() on sources when a stream is stopped") { + var calledStop = false + val source = new Source { + override def stop(): Unit = { + calledStop = true + } + override def getOffset: Option[Offset] = None + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + spark.emptyDataFrame + } + override def schema: StructType = MockSourceProvider.fakeSchema + } + + MockSourceProvider.withMockSources(source) { + val df = spark.readStream + .format("org.apache.spark.sql.streaming.util.MockSourceProvider") + .load() + + testStream(df)(StopStream) + + assert(calledStop, "Did not call stop on source for stopped stream") + } + } + + testQuietly("SPARK-19774: StreamExecution should call stop() on sources when a stream fails") { + var calledStop = false + val source1 = new Source { + override def stop(): Unit = { + throw new RuntimeException("Oh no!") + } + override def getOffset: Option[Offset] = Some(LongOffset(1)) + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + spark.range(2).toDF(MockSourceProvider.fakeSchema.fieldNames: _*) + } + override def schema: StructType = MockSourceProvider.fakeSchema + } + val source2 = new Source { + override def stop(): Unit = { + calledStop = true + } + override def getOffset: Option[Offset] = None + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + spark.emptyDataFrame + } + override def schema: StructType = MockSourceProvider.fakeSchema + } + + MockSourceProvider.withMockSources(source1, source2) { + val df1 = spark.readStream + .format("org.apache.spark.sql.streaming.util.MockSourceProvider") + .load() + .as[Int] + + val df2 = spark.readStream + .format("org.apache.spark.sql.streaming.util.MockSourceProvider") + .load() + .as[Int] + + testStream(df1.union(df2).map(i => i / 0))( + AssertOnQuery { sq => + intercept[StreamingQueryException](sq.processAllAvailable()) + sq.exception.isDefined && !sq.isActive + } + ) + + assert(calledStop, "Did not call stop on source for stopped stream") + } + } + + test("get the query id in source") { + @volatile var queryId: String = null + val source = new Source { + override def stop(): Unit = {} + override def getOffset: Option[Offset] = { + queryId = spark.sparkContext.getLocalProperty(StreamExecution.QUERY_ID_KEY) + None + } + override def getBatch(start: Option[Offset], end: Offset): DataFrame = spark.emptyDataFrame + override def schema: StructType = MockSourceProvider.fakeSchema + } + + MockSourceProvider.withMockSources(source) { + val df = spark.readStream + .format("org.apache.spark.sql.streaming.util.MockSourceProvider") + .load() + testStream(df)( + AssertOnQuery { sq => + sq.processAllAvailable() + assert(sq.id.toString === queryId) + assert(sq.runId.toString !== queryId) + true + } + ) + } + } + + test("processAllAvailable should not block forever when a query is stopped") { + val input = MemoryStream[Int] + input.addData(1) + val query = input.toDF().writeStream + .trigger(Trigger.Once()) + .format("console") + .start() + failAfter(streamingTimeout) { + query.processAllAvailable() + } + } + + /** Create a streaming DF that only execute one batch in which it returns the given static DF */ + private def createSingleTriggerStreamingDF(triggerDF: DataFrame): DataFrame = { + require(!triggerDF.isStreaming) + // A streaming Source that generate only on trigger and returns the given Dataframe as batch + val source = new Source() { + override def schema: StructType = triggerDF.schema + override def getOffset: Option[Offset] = Some(LongOffset(0)) + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + sqlContext.internalCreateDataFrame( + triggerDF.queryExecution.toRdd, triggerDF.schema, isStreaming = true) + } + override def stop(): Unit = {} + } + StreamingExecutionRelation(source, spark) + } + + /** Returns the query progress at the end of the first trigger of streaming DF */ + private def getFirstProgress(streamingDF: DataFrame): StreamingQueryProgress = { + try { + val q = streamingDF.writeStream.format("memory").queryName("test").start() + q.processAllAvailable() + q.recentProgress.head + } finally { + spark.streams.active.map(_.stop()) + } + } + + /** + * A [[StreamAction]] to test the behavior of `StreamingQuery.awaitTermination()`. + * + * @param expectedBehavior Expected behavior (not blocked, blocked, or exception thrown) + * @param timeoutMs Timeout in milliseconds + * When timeoutMs is less than or equal to 0, awaitTermination() is + * tested (i.e. w/o timeout) + * When timeoutMs is greater than 0, awaitTermination(timeoutMs) is + * tested + * @param expectedReturnValue Expected return value when awaitTermination(timeoutMs) is used + */ + case class TestAwaitTermination( + expectedBehavior: ExpectedBehavior, + timeoutMs: Int = -1, + expectedReturnValue: Boolean = false + ) extends AssertOnQuery( + TestAwaitTermination.assertOnQueryCondition(expectedBehavior, timeoutMs, expectedReturnValue), + "Error testing awaitTermination behavior" + ) { + override def toString(): String = { + s"TestAwaitTermination($expectedBehavior, timeoutMs = $timeoutMs, " + + s"expectedReturnValue = $expectedReturnValue)" + } + } + + object TestAwaitTermination { + + /** + * Tests the behavior of `StreamingQuery.awaitTermination`. + * + * @param expectedBehavior Expected behavior (not blocked, blocked, or exception thrown) + * @param timeoutMs Timeout in milliseconds + * When timeoutMs is less than or equal to 0, awaitTermination() is + * tested (i.e. w/o timeout) + * When timeoutMs is greater than 0, awaitTermination(timeoutMs) is + * tested + * @param expectedReturnValue Expected return value when awaitTermination(timeoutMs) is used + */ + def assertOnQueryCondition( + expectedBehavior: ExpectedBehavior, + timeoutMs: Int, + expectedReturnValue: Boolean + )(q: StreamExecution): Boolean = { + + def awaitTermFunc(): Unit = { + if (timeoutMs <= 0) { + q.awaitTermination() + } else { + val returnedValue = q.awaitTermination(timeoutMs) + assert(returnedValue === expectedReturnValue, "Returned value does not match expected") + } + } + AwaitTerminationTester.test(expectedBehavior, awaitTermFunc) + true // If the control reached here, then everything worked as expected + } + } +} + +object StreamingQuerySuite { + // Singleton reference to clock that does not get serialized in task closures + var clock: StreamManualClock = null +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala new file mode 100644 index 000000000000..aa163d2211c3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -0,0 +1,679 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming.test + +import java.io.File +import java.util.Locale +import java.util.concurrent.TimeUnit + +import scala.concurrent.duration._ + +import org.apache.hadoop.fs.Path +import org.mockito.Matchers.{any, eq => meq} +import org.mockito.Mockito._ +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql._ +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.{StreamSinkProvider, StreamSourceProvider} +import org.apache.spark.sql.streaming.{ProcessingTime => DeprecatedProcessingTime, _} +import org.apache.spark.sql.streaming.Trigger._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + +object LastOptions { + + var mockStreamSourceProvider = mock(classOf[StreamSourceProvider]) + var mockStreamSinkProvider = mock(classOf[StreamSinkProvider]) + var parameters: Map[String, String] = null + var schema: Option[StructType] = null + var partitionColumns: Seq[String] = Nil + + def clear(): Unit = { + parameters = null + schema = null + partitionColumns = null + reset(mockStreamSourceProvider) + reset(mockStreamSinkProvider) + } +} + +/** Dummy provider: returns no-op source/sink and records options in [[LastOptions]]. */ +class DefaultSource extends StreamSourceProvider with StreamSinkProvider { + + private val fakeSchema = StructType(StructField("a", IntegerType) :: Nil) + + override def sourceSchema( + spark: SQLContext, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): (String, StructType) = { + LastOptions.parameters = parameters + LastOptions.schema = schema + LastOptions.mockStreamSourceProvider.sourceSchema(spark, schema, providerName, parameters) + ("dummySource", fakeSchema) + } + + override def createSource( + spark: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source = { + LastOptions.parameters = parameters + LastOptions.schema = schema + LastOptions.mockStreamSourceProvider.createSource( + spark, metadataPath, schema, providerName, parameters) + new Source { + override def schema: StructType = fakeSchema + + override def getOffset: Option[Offset] = Some(new LongOffset(0)) + + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + import spark.implicits._ + + spark.internalCreateDataFrame(spark.sparkContext.emptyRDD, schema, isStreaming = true) + } + + override def stop() {} + } + } + + override def createSink( + spark: SQLContext, + parameters: Map[String, String], + partitionColumns: Seq[String], + outputMode: OutputMode): Sink = { + LastOptions.parameters = parameters + LastOptions.partitionColumns = partitionColumns + LastOptions.mockStreamSinkProvider.createSink(spark, parameters, partitionColumns, outputMode) + new Sink { + override def addBatch(batchId: Long, data: DataFrame): Unit = {} + } + } +} + +class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { + + private def newMetadataDir = + Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath + + after { + spark.streams.active.foreach(_.stop()) + } + + test("write cannot be called on streaming datasets") { + val e = intercept[AnalysisException] { + spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load() + .write + .save() + } + Seq("'write'", "not", "streaming Dataset/DataFrame").foreach { s => + assert(e.getMessage.toLowerCase(Locale.ROOT).contains(s.toLowerCase(Locale.ROOT))) + } + } + + test("resolve default source") { + spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load() + .writeStream + .format("org.apache.spark.sql.streaming.test") + .option("checkpointLocation", newMetadataDir) + .start() + .stop() + } + + test("resolve full class") { + spark.readStream + .format("org.apache.spark.sql.streaming.test.DefaultSource") + .load() + .writeStream + .format("org.apache.spark.sql.streaming.test") + .option("checkpointLocation", newMetadataDir) + .start() + .stop() + } + + test("options") { + val map = new java.util.HashMap[String, String] + map.put("opt3", "3") + + val df = spark.readStream + .format("org.apache.spark.sql.streaming.test") + .option("opt1", "1") + .options(Map("opt2" -> "2")) + .options(map) + .load() + + assert(LastOptions.parameters("opt1") == "1") + assert(LastOptions.parameters("opt2") == "2") + assert(LastOptions.parameters("opt3") == "3") + + LastOptions.clear() + + df.writeStream + .format("org.apache.spark.sql.streaming.test") + .option("opt1", "1") + .options(Map("opt2" -> "2")) + .options(map) + .option("checkpointLocation", newMetadataDir) + .start() + .stop() + + assert(LastOptions.parameters("opt1") == "1") + assert(LastOptions.parameters("opt2") == "2") + assert(LastOptions.parameters("opt3") == "3") + } + + test("partitioning") { + val df = spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load() + + df.writeStream + .format("org.apache.spark.sql.streaming.test") + .option("checkpointLocation", newMetadataDir) + .start() + .stop() + assert(LastOptions.partitionColumns == Nil) + + df.writeStream + .format("org.apache.spark.sql.streaming.test") + .option("checkpointLocation", newMetadataDir) + .partitionBy("a") + .start() + .stop() + assert(LastOptions.partitionColumns == Seq("a")) + + withSQLConf("spark.sql.caseSensitive" -> "false") { + df.writeStream + .format("org.apache.spark.sql.streaming.test") + .option("checkpointLocation", newMetadataDir) + .partitionBy("A") + .start() + .stop() + assert(LastOptions.partitionColumns == Seq("a")) + } + + intercept[AnalysisException] { + df.writeStream + .format("org.apache.spark.sql.streaming.test") + .option("checkpointLocation", newMetadataDir) + .partitionBy("b") + .start() + .stop() + } + } + + test("stream paths") { + val df = spark.readStream + .format("org.apache.spark.sql.streaming.test") + .option("checkpointLocation", newMetadataDir) + .load("/test") + + assert(LastOptions.parameters("path") == "/test") + + LastOptions.clear() + + df.writeStream + .format("org.apache.spark.sql.streaming.test") + .option("checkpointLocation", newMetadataDir) + .start("/test") + .stop() + + assert(LastOptions.parameters("path") == "/test") + } + + test("test different data types for options") { + val df = spark.readStream + .format("org.apache.spark.sql.streaming.test") + .option("intOpt", 56) + .option("boolOpt", false) + .option("doubleOpt", 6.7) + .load("/test") + + assert(LastOptions.parameters("intOpt") == "56") + assert(LastOptions.parameters("boolOpt") == "false") + assert(LastOptions.parameters("doubleOpt") == "6.7") + + LastOptions.clear() + df.writeStream + .format("org.apache.spark.sql.streaming.test") + .option("intOpt", 56) + .option("boolOpt", false) + .option("doubleOpt", 6.7) + .option("checkpointLocation", newMetadataDir) + .start("/test") + .stop() + + assert(LastOptions.parameters("intOpt") == "56") + assert(LastOptions.parameters("boolOpt") == "false") + assert(LastOptions.parameters("doubleOpt") == "6.7") + } + + test("unique query names") { + + /** Start a query with a specific name */ + def startQueryWithName(name: String = ""): StreamingQuery = { + spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load("/test") + .writeStream + .format("org.apache.spark.sql.streaming.test") + .option("checkpointLocation", newMetadataDir) + .queryName(name) + .start() + } + + /** Start a query without specifying a name */ + def startQueryWithoutName(): StreamingQuery = { + spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load("/test") + .writeStream + .format("org.apache.spark.sql.streaming.test") + .option("checkpointLocation", newMetadataDir) + .start() + } + + /** Get the names of active streams */ + def activeStreamNames: Set[String] = { + val streams = spark.streams.active + val names = streams.map(_.name).toSet + assert(streams.length === names.size, s"names of active queries are not unique: $names") + names + } + + val q1 = startQueryWithName("name") + + // Should not be able to start another query with the same name + intercept[IllegalArgumentException] { + startQueryWithName("name") + } + assert(activeStreamNames === Set("name")) + + // Should be able to start queries with other names + val q3 = startQueryWithName("another-name") + assert(activeStreamNames === Set("name", "another-name")) + + // Should be able to start queries with auto-generated names + val q4 = startQueryWithoutName() + assert(activeStreamNames.contains(q4.name)) + + // Should not be able to start a query with same auto-generated name + intercept[IllegalArgumentException] { + startQueryWithName(q4.name) + } + + // Should be able to start query with that name after stopping the previous query + q1.stop() + val q5 = startQueryWithName("name") + assert(activeStreamNames.contains("name")) + spark.streams.active.foreach(_.stop()) + } + + test("trigger") { + val df = spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load("/test") + + var q = df.writeStream + .format("org.apache.spark.sql.streaming.test") + .option("checkpointLocation", newMetadataDir) + .trigger(ProcessingTime(10.seconds)) + .start() + q.stop() + + assert(q.asInstanceOf[StreamingQueryWrapper].streamingQuery.trigger == ProcessingTime(10000)) + + q = df.writeStream + .format("org.apache.spark.sql.streaming.test") + .option("checkpointLocation", newMetadataDir) + .trigger(ProcessingTime(100, TimeUnit.SECONDS)) + .start() + q.stop() + + assert(q.asInstanceOf[StreamingQueryWrapper].streamingQuery.trigger == ProcessingTime(100000)) + } + + test("source metadataPath") { + LastOptions.clear() + + val checkpointLocationURI = new Path(newMetadataDir).toUri + + val df1 = spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load() + + val df2 = spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load() + + val q = df1.union(df2).writeStream + .format("org.apache.spark.sql.streaming.test") + .option("checkpointLocation", checkpointLocationURI.toString) + .trigger(ProcessingTime(10.seconds)) + .start() + q.processAllAvailable() + q.stop() + + verify(LastOptions.mockStreamSourceProvider).createSource( + any(), + meq(s"${makeQualifiedPath(checkpointLocationURI.toString)}/sources/0"), + meq(None), + meq("org.apache.spark.sql.streaming.test"), + meq(Map.empty)) + + verify(LastOptions.mockStreamSourceProvider).createSource( + any(), + meq(s"${makeQualifiedPath(checkpointLocationURI.toString)}/sources/1"), + meq(None), + meq("org.apache.spark.sql.streaming.test"), + meq(Map.empty)) + } + + private def newTextInput = Utils.createTempDir(namePrefix = "text").getCanonicalPath + + test("check foreach() catches null writers") { + val df = spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load() + + var w = df.writeStream + var e = intercept[IllegalArgumentException](w.foreach(null)) + Seq("foreach", "null").foreach { s => + assert(e.getMessage.toLowerCase(Locale.ROOT).contains(s.toLowerCase(Locale.ROOT))) + } + } + + + test("check foreach() does not support partitioning") { + val df = spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load() + val foreachWriter = new ForeachWriter[Row] { + override def open(partitionId: Long, version: Long): Boolean = false + override def process(value: Row): Unit = {} + override def close(errorOrNull: Throwable): Unit = {} + } + var w = df.writeStream.partitionBy("value") + var e = intercept[AnalysisException](w.foreach(foreachWriter).start()) + Seq("foreach", "partitioning").foreach { s => + assert(e.getMessage.toLowerCase(Locale.ROOT).contains(s.toLowerCase(Locale.ROOT))) + } + } + + test("ConsoleSink can be correctly loaded") { + LastOptions.clear() + val df = spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load() + + val sq = df.writeStream + .format("console") + .option("checkpointLocation", newMetadataDir) + .trigger(ProcessingTime(2.seconds)) + .start() + + sq.awaitTermination(2000L) + } + + test("prevent all column partitioning") { + withTempDir { dir => + val path = dir.getCanonicalPath + intercept[AnalysisException] { + spark.range(10).writeStream + .outputMode("append") + .partitionBy("id") + .format("parquet") + .start(path) + } + } + } + + test("ConsoleSink should not require checkpointLocation") { + LastOptions.clear() + val df = spark.readStream + .format("org.apache.spark.sql.streaming.test") + .load() + + val sq = df.writeStream.format("console").start() + sq.stop() + } + + private def testMemorySinkCheckpointRecovery(chkLoc: String, provideInWriter: Boolean): Unit = { + import testImplicits._ + val ms = new MemoryStream[Int](0, sqlContext) + val df = ms.toDF().toDF("a") + val tableName = "test" + def startQuery: StreamingQuery = { + val writer = df.groupBy("a") + .count() + .writeStream + .format("memory") + .queryName(tableName) + .outputMode("complete") + if (provideInWriter) { + writer.option("checkpointLocation", chkLoc) + } + writer.start() + } + // no exception here + val q = startQuery + ms.addData(0, 1) + q.processAllAvailable() + q.stop() + + checkAnswer( + spark.table(tableName), + Seq(Row(0, 1), Row(1, 1)) + ) + spark.sql(s"drop table $tableName") + // verify table is dropped + intercept[AnalysisException](spark.table(tableName).collect()) + val q2 = startQuery + ms.addData(0) + q2.processAllAvailable() + checkAnswer( + spark.table(tableName), + Seq(Row(0, 2), Row(1, 1)) + ) + + q2.stop() + } + + test("MemorySink can recover from a checkpoint in Complete Mode") { + val checkpointLoc = newMetadataDir + val checkpointDir = new File(checkpointLoc, "offsets") + checkpointDir.mkdirs() + assert(checkpointDir.exists()) + testMemorySinkCheckpointRecovery(checkpointLoc, provideInWriter = true) + } + + test("SPARK-18927: MemorySink can recover from a checkpoint provided in conf in Complete Mode") { + val checkpointLoc = newMetadataDir + val checkpointDir = new File(checkpointLoc, "offsets") + checkpointDir.mkdirs() + assert(checkpointDir.exists()) + withSQLConf(SQLConf.CHECKPOINT_LOCATION.key -> checkpointLoc) { + testMemorySinkCheckpointRecovery(checkpointLoc, provideInWriter = false) + } + } + + test("append mode memory sink's do not support checkpoint recovery") { + import testImplicits._ + val ms = new MemoryStream[Int](0, sqlContext) + val df = ms.toDF().toDF("a") + val checkpointLoc = newMetadataDir + val checkpointDir = new File(checkpointLoc, "offsets") + checkpointDir.mkdirs() + assert(checkpointDir.exists()) + + val e = intercept[AnalysisException] { + df.writeStream + .format("memory") + .queryName("test") + .option("checkpointLocation", checkpointLoc) + .outputMode("append") + .start() + } + assert(e.getMessage.contains("does not support recovering")) + assert(e.getMessage.contains("checkpoint location")) + } + + test("SPARK-18510: use user specified types for partition columns in file sources") { + import org.apache.spark.sql.functions.udf + import testImplicits._ + withTempDir { src => + val createArray = udf { (length: Long) => + for (i <- 1 to length.toInt) yield i.toString + } + spark.range(4).select(createArray('id + 1) as 'ex, 'id, 'id % 4 as 'part).coalesce(1).write + .partitionBy("part", "id") + .mode("overwrite") + .parquet(src.toString) + // Specify a random ordering of the schema, partition column in the middle, etc. + // Also let's say that the partition columns are Strings instead of Longs. + // partition columns should go to the end + val schema = new StructType() + .add("id", StringType) + .add("ex", ArrayType(StringType)) + + val sdf = spark.readStream + .schema(schema) + .format("parquet") + .load(src.toString) + + assert(sdf.schema.toList === List( + StructField("ex", ArrayType(StringType)), + StructField("part", IntegerType), // inferred partitionColumn dataType + StructField("id", StringType))) // used user provided partitionColumn dataType + + val sq = sdf.writeStream + .queryName("corruption_test") + .format("memory") + .start() + sq.processAllAvailable() + checkAnswer( + spark.table("corruption_test"), + // notice how `part` is ordered before `id` + Row(Array("1"), 0, "0") :: Row(Array("1", "2"), 1, "1") :: + Row(Array("1", "2", "3"), 2, "2") :: Row(Array("1", "2", "3", "4"), 3, "3") :: Nil + ) + sq.stop() + } + } + + test("user specified checkpointLocation precedes SQLConf") { + import testImplicits._ + withTempDir { checkpointPath => + withTempPath { userCheckpointPath => + assert(!userCheckpointPath.exists(), s"$userCheckpointPath should not exist") + withSQLConf(SQLConf.CHECKPOINT_LOCATION.key -> checkpointPath.getAbsolutePath) { + val queryName = "test_query" + val ds = MemoryStream[Int].toDS + ds.writeStream + .format("memory") + .queryName(queryName) + .option("checkpointLocation", userCheckpointPath.getAbsolutePath) + .start() + .stop() + assert(checkpointPath.listFiles().isEmpty, + "SQLConf path is used even if user specified checkpointLoc: " + + s"${checkpointPath.listFiles()} is not empty") + assert(userCheckpointPath.exists(), + s"The user specified checkpointLoc (userCheckpointPath) is not created") + } + } + } + } + + test("use SQLConf checkpoint dir when checkpointLocation is not specified") { + import testImplicits._ + withTempDir { checkpointPath => + withSQLConf(SQLConf.CHECKPOINT_LOCATION.key -> checkpointPath.getAbsolutePath) { + val queryName = "test_query" + val ds = MemoryStream[Int].toDS + ds.writeStream.format("memory").queryName(queryName).start().stop() + // Should use query name to create a folder in `checkpointPath` + val queryCheckpointDir = new File(checkpointPath, queryName) + assert(queryCheckpointDir.exists(), s"$queryCheckpointDir doesn't exist") + assert( + checkpointPath.listFiles().size === 1, + s"${checkpointPath.listFiles().toList} has 0 or more than 1 files ") + } + } + } + + test("use SQLConf checkpoint dir when checkpointLocation is not specified without query name") { + import testImplicits._ + withTempDir { checkpointPath => + withSQLConf(SQLConf.CHECKPOINT_LOCATION.key -> checkpointPath.getAbsolutePath) { + val ds = MemoryStream[Int].toDS + ds.writeStream.format("console").start().stop() + // Should create a random folder in `checkpointPath` + assert( + checkpointPath.listFiles().size === 1, + s"${checkpointPath.listFiles().toList} has 0 or more than 1 files ") + } + } + } + + test("temp checkpoint dir should be deleted if a query is stopped without errors") { + import testImplicits._ + val query = MemoryStream[Int].toDS.writeStream.format("console").start() + query.processAllAvailable() + val checkpointDir = new Path( + query.asInstanceOf[StreamingQueryWrapper].streamingQuery.resolvedCheckpointRoot) + val fs = checkpointDir.getFileSystem(spark.sessionState.newHadoopConf()) + assert(fs.exists(checkpointDir)) + query.stop() + assert(!fs.exists(checkpointDir)) + } + + testQuietly("temp checkpoint dir should not be deleted if a query is stopped with an error") { + import testImplicits._ + val input = MemoryStream[Int] + val query = input.toDS.map(_ / 0).writeStream.format("console").start() + val checkpointDir = new Path( + query.asInstanceOf[StreamingQueryWrapper].streamingQuery.resolvedCheckpointRoot) + val fs = checkpointDir.getFileSystem(spark.sessionState.newHadoopConf()) + assert(fs.exists(checkpointDir)) + input.addData(1) + intercept[StreamingQueryException] { + query.awaitTermination() + } + assert(fs.exists(checkpointDir)) + } + + test("SPARK-20431: Specify a schema by using a DDL-formatted string") { + spark.readStream + .format("org.apache.spark.sql.streaming.test") + .schema("aa INT") + .load() + + assert(LastOptions.schema.isDefined) + assert(LastOptions.schema.get === StructType(StructField("aa", IntegerType) :: Nil)) + + LastOptions.clear() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala new file mode 100644 index 000000000000..19ab2ff13e14 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming.util + +import java.util.concurrent.CountDownLatch + +import org.apache.spark.sql.{SQLContext, _} +import org.apache.spark.sql.execution.streaming.{LongOffset, Offset, Sink, Source} +import org.apache.spark.sql.sources.{StreamSinkProvider, StreamSourceProvider} +import org.apache.spark.sql.streaming.OutputMode +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} + +/** Dummy provider: returns a SourceProvider with a blocking `createSource` call. */ +class BlockingSource extends StreamSourceProvider with StreamSinkProvider { + + private val fakeSchema = StructType(StructField("a", IntegerType) :: Nil) + + override def sourceSchema( + spark: SQLContext, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): (String, StructType) = { + ("dummySource", fakeSchema) + } + + override def createSource( + spark: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source = { + BlockingSource.latch.await() + new Source { + override def schema: StructType = fakeSchema + override def getOffset: Option[Offset] = Some(new LongOffset(0)) + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + import spark.implicits._ + Seq[Int]().toDS().toDF() + } + override def stop() {} + } + } + + override def createSink( + spark: SQLContext, + parameters: Map[String, String], + partitionColumns: Seq[String], + outputMode: OutputMode): Sink = { + new Sink { + override def addBatch(batchId: Long, data: DataFrame): Unit = {} + } + } +} + +object BlockingSource { + var latch: CountDownLatch = null +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/MockSourceProvider.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/MockSourceProvider.scala new file mode 100644 index 000000000000..0bf05381a7f3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/MockSourceProvider.scala @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming.util + +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.execution.streaming.Source +import org.apache.spark.sql.sources.StreamSourceProvider +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} + +/** + * A StreamSourceProvider that provides mocked Sources for unit testing. Example usage: + * + * {{{ + * MockSourceProvider.withMockSources(source1, source2) { + * val df1 = spark.readStream + * .format("org.apache.spark.sql.streaming.util.MockSourceProvider") + * .load() + * + * val df2 = spark.readStream + * .format("org.apache.spark.sql.streaming.util.MockSourceProvider") + * .load() + * + * df1.union(df2) + * ... + * } + * }}} + */ +class MockSourceProvider extends StreamSourceProvider { + override def sourceSchema( + spark: SQLContext, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): (String, StructType) = { + ("dummySource", MockSourceProvider.fakeSchema) + } + + override def createSource( + spark: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source = { + MockSourceProvider.sourceProviderFunction() + } +} + +object MockSourceProvider { + // Function to generate sources. May provide multiple sources if the user implements such a + // function. + private var sourceProviderFunction: () => Source = _ + + final val fakeSchema = StructType(StructField("a", IntegerType) :: Nil) + + def withMockSources(source: Source, otherSources: Source*)(f: => Unit): Unit = { + var i = 0 + val sources = source +: otherSources + sourceProviderFunction = () => { + val source = sources(i % sources.length) + i += 1 + source + } + try { + f + } finally { + sourceProviderFunction = null + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/StreamManualClock.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/StreamManualClock.scala new file mode 100644 index 000000000000..c769a790a416 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/StreamManualClock.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming.util + +import org.apache.spark.util.ManualClock + +/** + * ManualClock used for streaming tests that allows checking whether the stream is waiting + * on the clock at expected times. + */ +class StreamManualClock(time: Long = 0L) extends ManualClock(time) with Serializable { + private var waitStartTime: Option[Long] = None + private var waitTargetTime: Option[Long] = None + + override def waitTillTime(targetTime: Long): Long = synchronized { + try { + waitStartTime = Some(getTimeMillis()) + waitTargetTime = Some(targetTime) + super.waitTillTime(targetTime) + } finally { + waitStartTime = None + waitTargetTime = None + } + } + + /** Is the streaming thread waiting for the clock to advance when it is at the given time */ + def isStreamWaitingAt(time: Long): Boolean = synchronized { + waitStartTime == Some(time) + } + + /** Is the streaming thread waiting for clock to advance to the given time */ + def isStreamWaitingFor(target: Long): Boolean = synchronized { + waitTargetTime == Some(target) + } +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala new file mode 100644 index 000000000000..569bac156b53 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -0,0 +1,778 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.test + +import java.io.File +import java.util.Locale +import java.util.concurrent.ConcurrentLinkedQueue + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage +import org.apache.spark.internal.io.HadoopMapReduceCommitProtocol +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + + +object LastOptions { + + var parameters: Map[String, String] = null + var schema: Option[StructType] = null + var saveMode: SaveMode = null + + def clear(): Unit = { + parameters = null + schema = null + saveMode = null + } +} + +/** Dummy provider. */ +class DefaultSource + extends RelationProvider + with SchemaRelationProvider + with CreatableRelationProvider { + + case class FakeRelation(sqlContext: SQLContext) extends BaseRelation { + override def schema: StructType = StructType(Seq(StructField("a", StringType))) + } + + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String], + schema: StructType + ): BaseRelation = { + LastOptions.parameters = parameters + LastOptions.schema = Some(schema) + FakeRelation(sqlContext) + } + + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String] + ): BaseRelation = { + LastOptions.parameters = parameters + LastOptions.schema = None + FakeRelation(sqlContext) + } + + override def createRelation( + sqlContext: SQLContext, + mode: SaveMode, + parameters: Map[String, String], + data: DataFrame): BaseRelation = { + LastOptions.parameters = parameters + LastOptions.schema = None + LastOptions.saveMode = mode + FakeRelation(sqlContext) + } +} + +/** Dummy provider with only RelationProvider and CreatableRelationProvider. */ +class DefaultSourceWithoutUserSpecifiedSchema + extends RelationProvider + with CreatableRelationProvider { + + case class FakeRelation(sqlContext: SQLContext) extends BaseRelation { + override def schema: StructType = StructType(Seq(StructField("a", StringType))) + } + + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + FakeRelation(sqlContext) + } + + override def createRelation( + sqlContext: SQLContext, + mode: SaveMode, + parameters: Map[String, String], + data: DataFrame): BaseRelation = { + FakeRelation(sqlContext) + } +} + +object MessageCapturingCommitProtocol { + val commitMessages = new ConcurrentLinkedQueue[TaskCommitMessage]() +} + +class MessageCapturingCommitProtocol(jobId: String, path: String) + extends HadoopMapReduceCommitProtocol(jobId, path) { + + // captures commit messages for testing + override def onTaskCommit(msg: TaskCommitMessage): Unit = { + MessageCapturingCommitProtocol.commitMessages.offer(msg) + } +} + + +class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with BeforeAndAfter { + import testImplicits._ + + private val userSchema = new StructType().add("s", StringType) + private val userSchemaString = "s STRING" + private val textSchema = new StructType().add("value", StringType) + private val data = Seq("1", "2", "3") + private val dir = Utils.createTempDir(namePrefix = "input").getCanonicalPath + + before { + Utils.deleteRecursively(new File(dir)) + } + + test("writeStream cannot be called on non-streaming datasets") { + val e = intercept[AnalysisException] { + spark.read + .format("org.apache.spark.sql.test") + .load() + .writeStream + .start() + } + Seq("'writeStream'", "only", "streaming Dataset/DataFrame").foreach { s => + assert(e.getMessage.toLowerCase(Locale.ROOT).contains(s.toLowerCase(Locale.ROOT))) + } + } + + + test("resolve default source") { + spark.read + .format("org.apache.spark.sql.test") + .load() + .write + .format("org.apache.spark.sql.test") + .save() + } + + test("resolve default source without extending SchemaRelationProvider") { + spark.read + .format("org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema") + .load() + .write + .format("org.apache.spark.sql.test.DefaultSourceWithoutUserSpecifiedSchema") + .save() + } + + test("resolve full class") { + spark.read + .format("org.apache.spark.sql.test.DefaultSource") + .load() + .write + .format("org.apache.spark.sql.test") + .save() + } + + test("options") { + val map = new java.util.HashMap[String, String] + map.put("opt3", "3") + + val df = spark.read + .format("org.apache.spark.sql.test") + .option("opt1", "1") + .options(Map("opt2" -> "2")) + .options(map) + .load() + + assert(LastOptions.parameters("opt1") == "1") + assert(LastOptions.parameters("opt2") == "2") + assert(LastOptions.parameters("opt3") == "3") + + LastOptions.clear() + + df.write + .format("org.apache.spark.sql.test") + .option("opt1", "1") + .options(Map("opt2" -> "2")) + .options(map) + .save() + + assert(LastOptions.parameters("opt1") == "1") + assert(LastOptions.parameters("opt2") == "2") + assert(LastOptions.parameters("opt3") == "3") + } + + test("save mode") { + val df = spark.read + .format("org.apache.spark.sql.test") + .load() + + df.write + .format("org.apache.spark.sql.test") + .mode(SaveMode.ErrorIfExists) + .save() + assert(LastOptions.saveMode === SaveMode.ErrorIfExists) + } + + test("test path option in load") { + spark.read + .format("org.apache.spark.sql.test") + .option("intOpt", 56) + .load("/test") + + assert(LastOptions.parameters("intOpt") == "56") + assert(LastOptions.parameters("path") == "/test") + + LastOptions.clear() + spark.read + .format("org.apache.spark.sql.test") + .option("intOpt", 55) + .load() + + assert(LastOptions.parameters("intOpt") == "55") + assert(!LastOptions.parameters.contains("path")) + + LastOptions.clear() + spark.read + .format("org.apache.spark.sql.test") + .option("intOpt", 54) + .load("/test", "/test1", "/test2") + + assert(LastOptions.parameters("intOpt") == "54") + assert(!LastOptions.parameters.contains("path")) + } + + test("test different data types for options") { + val df = spark.read + .format("org.apache.spark.sql.test") + .option("intOpt", 56) + .option("boolOpt", false) + .option("doubleOpt", 6.7) + .load("/test") + + assert(LastOptions.parameters("intOpt") == "56") + assert(LastOptions.parameters("boolOpt") == "false") + assert(LastOptions.parameters("doubleOpt") == "6.7") + + LastOptions.clear() + df.write + .format("org.apache.spark.sql.test") + .option("intOpt", 56) + .option("boolOpt", false) + .option("doubleOpt", 6.7) + .save("/test") + + assert(LastOptions.parameters("intOpt") == "56") + assert(LastOptions.parameters("boolOpt") == "false") + assert(LastOptions.parameters("doubleOpt") == "6.7") + } + + test("check jdbc() does not support partitioning or bucketing") { + val df = spark.read.text(Utils.createTempDir(namePrefix = "text").getCanonicalPath) + + var w = df.write.partitionBy("value") + var e = intercept[AnalysisException](w.jdbc(null, null, null)) + Seq("jdbc", "partitioning").foreach { s => + assert(e.getMessage.toLowerCase(Locale.ROOT).contains(s.toLowerCase(Locale.ROOT))) + } + + w = df.write.bucketBy(2, "value") + e = intercept[AnalysisException](w.jdbc(null, null, null)) + Seq("jdbc", "bucketing").foreach { s => + assert(e.getMessage.toLowerCase(Locale.ROOT).contains(s.toLowerCase(Locale.ROOT))) + } + } + + test("prevent all column partitioning") { + withTempDir { dir => + val path = dir.getCanonicalPath + intercept[AnalysisException] { + spark.range(10).write.format("parquet").mode("overwrite").partitionBy("id").save(path) + } + intercept[AnalysisException] { + spark.range(10).write.format("csv").mode("overwrite").partitionBy("id").save(path) + } + spark.emptyDataFrame.write.format("parquet").mode("overwrite").save(path) + } + } + + test("load API") { + spark.read.format("org.apache.spark.sql.test").load() + spark.read.format("org.apache.spark.sql.test").load(dir) + spark.read.format("org.apache.spark.sql.test").load(dir, dir, dir) + spark.read.format("org.apache.spark.sql.test").load(Seq(dir, dir): _*) + Option(dir).map(spark.read.format("org.apache.spark.sql.test").load) + } + + test("write path implements onTaskCommit API correctly") { + withSQLConf( + "spark.sql.sources.commitProtocolClass" -> + classOf[MessageCapturingCommitProtocol].getCanonicalName) { + withTempDir { dir => + val path = dir.getCanonicalPath + MessageCapturingCommitProtocol.commitMessages.clear() + spark.range(10).repartition(10).write.mode("overwrite").parquet(path) + assert(MessageCapturingCommitProtocol.commitMessages.size() == 10) + } + } + } + + test("read a data source that does not extend SchemaRelationProvider") { + val dfReader = spark.read + .option("from", "1") + .option("TO", "10") + .format("org.apache.spark.sql.sources.SimpleScanSource") + + // when users do not specify the schema + checkAnswer(dfReader.load(), spark.range(1, 11).toDF()) + + // when users specify the schema + val inputSchema = new StructType().add("s", IntegerType, nullable = false) + val e = intercept[AnalysisException] { dfReader.schema(inputSchema).load() } + assert(e.getMessage.contains( + "org.apache.spark.sql.sources.SimpleScanSource does not allow user-specified schemas")) + } + + test("read a data source that does not extend RelationProvider") { + val dfReader = spark.read + .option("from", "1") + .option("TO", "10") + .option("option_with_underscores", "someval") + .option("option.with.dots", "someval") + .format("org.apache.spark.sql.sources.AllDataTypesScanSource") + + // when users do not specify the schema + val e = intercept[AnalysisException] { dfReader.load() } + assert(e.getMessage.contains("A schema needs to be specified when using")) + + // when users specify the schema + val inputSchema = new StructType().add("s", StringType, nullable = false) + assert(dfReader.schema(inputSchema).load().count() == 10) + } + + test("text - API and behavior regarding schema") { + // Writer + spark.createDataset(data).write.mode(SaveMode.Overwrite).text(dir) + testRead(spark.read.text(dir), data, textSchema) + + // Reader, without user specified schema + testRead(spark.read.text(), Seq.empty, textSchema) + testRead(spark.read.text(dir, dir, dir), data ++ data ++ data, textSchema) + testRead(spark.read.text(Seq(dir, dir): _*), data ++ data, textSchema) + // Test explicit calls to single arg method - SPARK-16009 + testRead(Option(dir).map(spark.read.text).get, data, textSchema) + + // Reader, with user specified schema, should just apply user schema on the file data + testRead(spark.read.schema(userSchema).text(), Seq.empty, userSchema) + testRead(spark.read.schema(userSchema).text(dir), data, userSchema) + testRead(spark.read.schema(userSchema).text(dir, dir), data ++ data, userSchema) + testRead(spark.read.schema(userSchema).text(Seq(dir, dir): _*), data ++ data, userSchema) + } + + test("textFile - API and behavior regarding schema") { + spark.createDataset(data).write.mode(SaveMode.Overwrite).text(dir) + + // Reader, without user specified schema + testRead(spark.read.textFile().toDF(), Seq.empty, textSchema) + testRead(spark.read.textFile(dir).toDF(), data, textSchema) + testRead(spark.read.textFile(dir, dir).toDF(), data ++ data, textSchema) + testRead(spark.read.textFile(Seq(dir, dir): _*).toDF(), data ++ data, textSchema) + // Test explicit calls to single arg method - SPARK-16009 + testRead(Option(dir).map(spark.read.text).get, data, textSchema) + + // Reader, with user specified schema, should just apply user schema on the file data + val e = intercept[AnalysisException] { spark.read.schema(userSchema).textFile() } + assert(e.getMessage.toLowerCase(Locale.ROOT).contains( + "user specified schema not supported")) + intercept[AnalysisException] { spark.read.schema(userSchema).textFile(dir) } + intercept[AnalysisException] { spark.read.schema(userSchema).textFile(dir, dir) } + intercept[AnalysisException] { spark.read.schema(userSchema).textFile(Seq(dir, dir): _*) } + } + + test("csv - API and behavior regarding schema") { + // Writer + spark.createDataset(data).toDF("str").write.mode(SaveMode.Overwrite).csv(dir) + val df = spark.read.csv(dir) + checkAnswer(df, spark.createDataset(data).toDF()) + val schema = df.schema + + // Reader, without user specified schema + val message = intercept[AnalysisException] { + testRead(spark.read.csv(), Seq.empty, schema) + }.getMessage + assert(message.contains("Unable to infer schema for CSV. It must be specified manually.")) + + testRead(spark.read.csv(dir), data, schema) + testRead(spark.read.csv(dir, dir), data ++ data, schema) + testRead(spark.read.csv(Seq(dir, dir): _*), data ++ data, schema) + // Test explicit calls to single arg method - SPARK-16009 + testRead(Option(dir).map(spark.read.csv).get, data, schema) + + // Reader, with user specified schema, should just apply user schema on the file data + testRead(spark.read.schema(userSchema).csv(), Seq.empty, userSchema) + testRead(spark.read.schema(userSchema).csv(dir), data, userSchema) + testRead(spark.read.schema(userSchema).csv(dir, dir), data ++ data, userSchema) + testRead(spark.read.schema(userSchema).csv(Seq(dir, dir): _*), data ++ data, userSchema) + } + + test("json - API and behavior regarding schema") { + // Writer + spark.createDataset(data).toDF("str").write.mode(SaveMode.Overwrite).json(dir) + val df = spark.read.json(dir) + checkAnswer(df, spark.createDataset(data).toDF()) + val schema = df.schema + + // Reader, without user specified schema + intercept[AnalysisException] { + testRead(spark.read.json(), Seq.empty, schema) + } + testRead(spark.read.json(dir), data, schema) + testRead(spark.read.json(dir, dir), data ++ data, schema) + testRead(spark.read.json(Seq(dir, dir): _*), data ++ data, schema) + // Test explicit calls to single arg method - SPARK-16009 + testRead(Option(dir).map(spark.read.json).get, data, schema) + + // Reader, with user specified schema, data should be nulls as schema in file different + // from user schema + val expData = Seq[String](null, null, null) + testRead(spark.read.schema(userSchema).json(), Seq.empty, userSchema) + testRead(spark.read.schema(userSchema).json(dir), expData, userSchema) + testRead(spark.read.schema(userSchema).json(dir, dir), expData ++ expData, userSchema) + testRead(spark.read.schema(userSchema).json(Seq(dir, dir): _*), expData ++ expData, userSchema) + } + + test("parquet - API and behavior regarding schema") { + // Writer + spark.createDataset(data).toDF("str").write.mode(SaveMode.Overwrite).parquet(dir) + val df = spark.read.parquet(dir) + checkAnswer(df, spark.createDataset(data).toDF()) + val schema = df.schema + + // Reader, without user specified schema + intercept[AnalysisException] { + testRead(spark.read.parquet(), Seq.empty, schema) + } + testRead(spark.read.parquet(dir), data, schema) + testRead(spark.read.parquet(dir, dir), data ++ data, schema) + testRead(spark.read.parquet(Seq(dir, dir): _*), data ++ data, schema) + // Test explicit calls to single arg method - SPARK-16009 + testRead(Option(dir).map(spark.read.parquet).get, data, schema) + + // Reader, with user specified schema, data should be nulls as schema in file different + // from user schema + val expData = Seq[String](null, null, null) + testRead(spark.read.schema(userSchema).parquet(), Seq.empty, userSchema) + testRead(spark.read.schema(userSchema).parquet(dir), expData, userSchema) + testRead(spark.read.schema(userSchema).parquet(dir, dir), expData ++ expData, userSchema) + testRead( + spark.read.schema(userSchema).parquet(Seq(dir, dir): _*), expData ++ expData, userSchema) + } + + /** + * This only tests whether API compiles, but does not run it as orc() + * cannot be run without Hive classes. + */ + ignore("orc - API") { + // Reader, with user specified schema + // Refer to csv-specific test suites for behavior without user specified schema + spark.read.schema(userSchema).orc() + spark.read.schema(userSchema).orc(dir) + spark.read.schema(userSchema).orc(dir, dir, dir) + spark.read.schema(userSchema).orc(Seq(dir, dir): _*) + Option(dir).map(spark.read.schema(userSchema).orc) + + // Writer + spark.range(10).write.orc(dir) + } + + test("column nullability and comment - write and then read") { + Seq("json", "parquet", "csv").foreach { format => + val schema = StructType( + StructField("cl1", IntegerType, nullable = false).withComment("test") :: + StructField("cl2", IntegerType, nullable = true) :: + StructField("cl3", IntegerType, nullable = true) :: Nil) + val row = Row(3, null, 4) + val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema) + + val tableName = "tab" + withTable(tableName) { + df.write.format(format).mode("overwrite").saveAsTable(tableName) + // Verify the DDL command result: DESCRIBE TABLE + checkAnswer( + sql(s"desc $tableName").select("col_name", "comment").where($"comment" === "test"), + Row("cl1", "test") :: Nil) + // Verify the schema + val expectedFields = schema.fields.map(f => f.copy(nullable = true)) + assert(spark.table(tableName).schema == schema.copy(fields = expectedFields)) + } + } + } + + test("SPARK-17230: write out results of decimal calculation") { + val df = spark.range(99, 101) + .selectExpr("id", "cast(id as long) * cast('1.0' as decimal(38, 18)) as num") + df.write.mode(SaveMode.Overwrite).parquet(dir) + val df2 = spark.read.parquet(dir) + checkAnswer(df2, df) + } + + private def testRead( + df: => DataFrame, + expectedResult: Seq[String], + expectedSchema: StructType): Unit = { + checkAnswer(df, spark.createDataset(expectedResult).toDF()) + assert(df.schema === expectedSchema) + } + + test("saveAsTable with mode Append should not fail if the table not exists " + + "but a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Append).saveAsTable("same_name") + assert( + spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + } + } + } + + test("saveAsTable with mode Append should not fail if the table already exists " + + "and a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + sql("CREATE TABLE same_name(id LONG) USING parquet") + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Append).saveAsTable("same_name") + checkAnswer(spark.table("same_name"), spark.range(10).toDF()) + checkAnswer(spark.table("default.same_name"), spark.range(20).toDF()) + } + } + } + + test("saveAsTable with mode ErrorIfExists should not fail if the table not exists " + + "but a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.ErrorIfExists).saveAsTable("same_name") + assert( + spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + } + } + } + + test("saveAsTable with mode Overwrite should not drop the temp view if the table not exists " + + "but a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Overwrite).saveAsTable("same_name") + assert(spark.sessionState.catalog.getTempView("same_name").isDefined) + assert( + spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + } + } + } + + test("saveAsTable with mode Overwrite should not fail if the table already exists " + + "and a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + sql("CREATE TABLE same_name(id LONG) USING parquet") + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Overwrite).saveAsTable("same_name") + checkAnswer(spark.table("same_name"), spark.range(10).toDF()) + checkAnswer(spark.table("default.same_name"), spark.range(20).toDF()) + } + } + } + + test("saveAsTable with mode Ignore should create the table if the table not exists " + + "but a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Ignore).saveAsTable("same_name") + assert( + spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + } + } + } + + test("SPARK-18510: use user specified types for partition columns in file sources") { + import org.apache.spark.sql.functions.udf + withTempDir { src => + val createArray = udf { (length: Long) => + for (i <- 1 to length.toInt) yield i.toString + } + spark.range(4).select(createArray('id + 1) as 'ex, 'id, 'id % 4 as 'part).coalesce(1).write + .partitionBy("part", "id") + .mode("overwrite") + .parquet(src.toString) + // Specify a random ordering of the schema, partition column in the middle, etc. + // Also let's say that the partition columns are Strings instead of Longs. + // partition columns should go to the end + val schema = new StructType() + .add("id", StringType) + .add("ex", ArrayType(StringType)) + val df = spark.read + .schema(schema) + .format("parquet") + .load(src.toString) + + assert(df.schema.toList === List( + StructField("ex", ArrayType(StringType)), + StructField("part", IntegerType), // inferred partitionColumn dataType + StructField("id", StringType))) // used user provided partitionColumn dataType + + checkAnswer( + df, + // notice how `part` is ordered before `id` + Row(Array("1"), 0, "0") :: Row(Array("1", "2"), 1, "1") :: + Row(Array("1", "2", "3"), 2, "2") :: Row(Array("1", "2", "3", "4"), 3, "3") :: Nil + ) + } + } + + test("SPARK-18899: append to a bucketed table using DataFrameWriter with mismatched bucketing") { + withTable("t") { + Seq(1 -> "a", 2 -> "b").toDF("i", "j").write.bucketBy(2, "i").saveAsTable("t") + val e = intercept[AnalysisException] { + Seq(3 -> "c").toDF("i", "j").write.bucketBy(3, "i").mode("append").saveAsTable("t") + } + assert(e.message.contains("Specified bucketing does not match that of the existing table")) + } + } + + test("SPARK-18912: number of columns mismatch for non-file-based data source table") { + withTable("t") { + sql("CREATE TABLE t USING org.apache.spark.sql.test.DefaultSource") + + val e = intercept[AnalysisException] { + Seq(1 -> "a").toDF("a", "b").write + .format("org.apache.spark.sql.test.DefaultSource") + .mode("append").saveAsTable("t") + } + assert(e.message.contains("The column number of the existing table")) + } + } + + test("SPARK-18913: append to a table with special column names") { + withTable("t") { + Seq(1 -> "a").toDF("x.x", "y.y").write.saveAsTable("t") + Seq(2 -> "b").toDF("x.x", "y.y").write.mode("append").saveAsTable("t") + checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Nil) + } + } + + test("SPARK-16848: table API throws an exception for user specified schema") { + withTable("t") { + val schema = StructType(StructField("a", StringType) :: Nil) + val e = intercept[AnalysisException] { + spark.read.schema(schema).table("t") + }.getMessage + assert(e.contains("User specified schema not supported with `table`")) + } + } + + test("SPARK-20431: Specify a schema by using a DDL-formatted string") { + spark.createDataset(data).write.mode(SaveMode.Overwrite).text(dir) + testRead(spark.read.schema(userSchemaString).text(), Seq.empty, userSchema) + testRead(spark.read.schema(userSchemaString).text(dir), data, userSchema) + testRead(spark.read.schema(userSchemaString).text(dir, dir), data ++ data, userSchema) + testRead(spark.read.schema(userSchemaString).text(Seq(dir, dir): _*), data ++ data, userSchema) + } + + test("SPARK-20460 Check name duplication in buckets") { + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + var errorMsg = intercept[AnalysisException] { + Seq((1, 1)).toDF("col", c0).write.bucketBy(2, c0, c1).saveAsTable("t") + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) in the bucket definition")) + + errorMsg = intercept[AnalysisException] { + Seq((1, 1)).toDF("col", c0).write.bucketBy(2, "col").sortBy(c0, c1).saveAsTable("t") + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) in the sort definition")) + } + } + } + + test("SPARK-20460 Check name duplication in schema") { + def checkWriteDataColumnDuplication( + format: String, colName0: String, colName1: String, tempDir: File): Unit = { + val errorMsg = intercept[AnalysisException] { + Seq((1, 1)).toDF(colName0, colName1).write.format(format).mode("overwrite") + .save(tempDir.getAbsolutePath) + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) when inserting into")) + } + + def checkReadUserSpecifiedDataColumnDuplication( + df: DataFrame, format: String, colName0: String, colName1: String, tempDir: File): Unit = { + val testDir = Utils.createTempDir(tempDir.getAbsolutePath) + df.write.format(format).mode("overwrite").save(testDir.getAbsolutePath) + val errorMsg = intercept[AnalysisException] { + spark.read.format(format).schema(s"$colName0 INT, $colName1 INT") + .load(testDir.getAbsolutePath) + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) in the data schema:")) + } + + def checkReadPartitionColumnDuplication( + format: String, colName0: String, colName1: String, tempDir: File): Unit = { + val testDir = Utils.createTempDir(tempDir.getAbsolutePath) + Seq(1).toDF("col").write.format(format).mode("overwrite") + .save(s"${testDir.getAbsolutePath}/$colName0=1/$colName1=1") + val errorMsg = intercept[AnalysisException] { + spark.read.format(format).load(testDir.getAbsolutePath) + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) in the partition schema:")) + } + + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + withTempDir { src => + // Check CSV format + checkWriteDataColumnDuplication("csv", c0, c1, src) + checkReadUserSpecifiedDataColumnDuplication( + Seq((1, 1)).toDF("c0", "c1"), "csv", c0, c1, src) + // If `inferSchema` is true, a CSV format is duplicate-safe (See SPARK-16896) + var testDir = Utils.createTempDir(src.getAbsolutePath) + Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text(testDir.getAbsolutePath) + val df = spark.read.format("csv").option("inferSchema", true).option("header", true) + .load(testDir.getAbsolutePath) + checkAnswer(df, Row(1, 1)) + checkReadPartitionColumnDuplication("csv", c0, c1, src) + + // Check JSON format + checkWriteDataColumnDuplication("json", c0, c1, src) + checkReadUserSpecifiedDataColumnDuplication( + Seq((1, 1)).toDF("c0", "c1"), "json", c0, c1, src) + // Inferred schema cases + testDir = Utils.createTempDir(src.getAbsolutePath) + Seq(s"""{"$c0":3, "$c1":5}""").toDF().write.mode("overwrite") + .text(testDir.getAbsolutePath) + val errorMsg = intercept[AnalysisException] { + spark.read.format("json").option("inferSchema", true).load(testDir.getAbsolutePath) + }.getMessage + assert(errorMsg.contains("Found duplicate column(s) in the data schema:")) + checkReadPartitionColumnDuplication("json", c0, c1, src) + + // Check Parquet format + checkWriteDataColumnDuplication("parquet", c0, c1, src) + checkReadUserSpecifiedDataColumnDuplication( + Seq((1, 1)).toDF("c0", "c1"), "parquet", c0, c1, src) + checkReadPartitionColumnDuplication("parquet", c0, c1, src) + } + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/ProcessTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/ProcessTestUtils.scala index 152c9c8459de..df530d8587ef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/ProcessTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/ProcessTestUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.test -import java.io.{IOException, InputStream} +import java.io.{InputStream, IOException} import scala.sys.process.BasicIO diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala index 520dea7f7dd9..0cfe260e5215 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala @@ -17,18 +17,20 @@ package org.apache.spark.sql.test +import java.nio.charset.StandardCharsets + import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, SQLContext, SQLImplicits} +import org.apache.spark.sql.{DataFrame, SparkSession, SQLContext, SQLImplicits} /** * A collection of sample data used in SQL tests. */ private[sql] trait SQLTestData { self => - protected def sqlContext: SQLContext + protected def spark: SparkSession // Helper object to import SQL implicits without a concrete SQLContext private object internalImplicits extends SQLImplicits { - protected override def _sqlContext: SQLContext = self.sqlContext + protected override def _sqlContext: SQLContext = self.spark.sqlContext } import internalImplicits._ @@ -37,173 +39,173 @@ private[sql] trait SQLTestData { self => // Note: all test data should be lazy because the SQLContext is not set up yet. protected lazy val emptyTestData: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( Seq.empty[Int].map(i => TestData(i, i.toString))).toDF() - df.registerTempTable("emptyTestData") + df.createOrReplaceTempView("emptyTestData") df } protected lazy val testData: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))).toDF() - df.registerTempTable("testData") + df.createOrReplaceTempView("testData") df } protected lazy val testData2: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( TestData2(1, 1) :: TestData2(1, 2) :: TestData2(2, 1) :: TestData2(2, 2) :: TestData2(3, 1) :: TestData2(3, 2) :: Nil, 2).toDF() - df.registerTempTable("testData2") + df.createOrReplaceTempView("testData2") df } protected lazy val testData3: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( TestData3(1, None) :: TestData3(2, Some(2)) :: Nil).toDF() - df.registerTempTable("testData3") + df.createOrReplaceTempView("testData3") df } protected lazy val negativeData: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( (1 to 100).map(i => TestData(-i, (-i).toString))).toDF() - df.registerTempTable("negativeData") + df.createOrReplaceTempView("negativeData") df } protected lazy val largeAndSmallInts: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( LargeAndSmallInts(2147483644, 1) :: LargeAndSmallInts(1, 2) :: LargeAndSmallInts(2147483645, 1) :: LargeAndSmallInts(2, 2) :: LargeAndSmallInts(2147483646, 1) :: LargeAndSmallInts(3, 2) :: Nil).toDF() - df.registerTempTable("largeAndSmallInts") + df.createOrReplaceTempView("largeAndSmallInts") df } protected lazy val decimalData: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( DecimalData(1, 1) :: DecimalData(1, 2) :: DecimalData(2, 1) :: DecimalData(2, 2) :: DecimalData(3, 1) :: DecimalData(3, 2) :: Nil).toDF() - df.registerTempTable("decimalData") + df.createOrReplaceTempView("decimalData") df } protected lazy val binaryData: DataFrame = { - val df = sqlContext.sparkContext.parallelize( - BinaryData("12".getBytes, 1) :: - BinaryData("22".getBytes, 5) :: - BinaryData("122".getBytes, 3) :: - BinaryData("121".getBytes, 2) :: - BinaryData("123".getBytes, 4) :: Nil).toDF() - df.registerTempTable("binaryData") + val df = spark.sparkContext.parallelize( + BinaryData("12".getBytes(StandardCharsets.UTF_8), 1) :: + BinaryData("22".getBytes(StandardCharsets.UTF_8), 5) :: + BinaryData("122".getBytes(StandardCharsets.UTF_8), 3) :: + BinaryData("121".getBytes(StandardCharsets.UTF_8), 2) :: + BinaryData("123".getBytes(StandardCharsets.UTF_8), 4) :: Nil).toDF() + df.createOrReplaceTempView("binaryData") df } protected lazy val upperCaseData: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( UpperCaseData(1, "A") :: UpperCaseData(2, "B") :: UpperCaseData(3, "C") :: UpperCaseData(4, "D") :: UpperCaseData(5, "E") :: UpperCaseData(6, "F") :: Nil).toDF() - df.registerTempTable("upperCaseData") + df.createOrReplaceTempView("upperCaseData") df } protected lazy val lowerCaseData: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( LowerCaseData(1, "a") :: LowerCaseData(2, "b") :: LowerCaseData(3, "c") :: LowerCaseData(4, "d") :: Nil).toDF() - df.registerTempTable("lowerCaseData") + df.createOrReplaceTempView("lowerCaseData") df } protected lazy val arrayData: RDD[ArrayData] = { - val rdd = sqlContext.sparkContext.parallelize( + val rdd = spark.sparkContext.parallelize( ArrayData(Seq(1, 2, 3), Seq(Seq(1, 2, 3))) :: ArrayData(Seq(2, 3, 4), Seq(Seq(2, 3, 4))) :: Nil) - rdd.toDF().registerTempTable("arrayData") + rdd.toDF().createOrReplaceTempView("arrayData") rdd } protected lazy val mapData: RDD[MapData] = { - val rdd = sqlContext.sparkContext.parallelize( + val rdd = spark.sparkContext.parallelize( MapData(Map(1 -> "a1", 2 -> "b1", 3 -> "c1", 4 -> "d1", 5 -> "e1")) :: MapData(Map(1 -> "a2", 2 -> "b2", 3 -> "c2", 4 -> "d2")) :: MapData(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: MapData(Map(1 -> "a4", 2 -> "b4")) :: MapData(Map(1 -> "a5")) :: Nil) - rdd.toDF().registerTempTable("mapData") + rdd.toDF().createOrReplaceTempView("mapData") rdd } protected lazy val repeatedData: RDD[StringData] = { - val rdd = sqlContext.sparkContext.parallelize(List.fill(2)(StringData("test"))) - rdd.toDF().registerTempTable("repeatedData") + val rdd = spark.sparkContext.parallelize(List.fill(2)(StringData("test"))) + rdd.toDF().createOrReplaceTempView("repeatedData") rdd } protected lazy val nullableRepeatedData: RDD[StringData] = { - val rdd = sqlContext.sparkContext.parallelize( + val rdd = spark.sparkContext.parallelize( List.fill(2)(StringData(null)) ++ List.fill(2)(StringData("test"))) - rdd.toDF().registerTempTable("nullableRepeatedData") + rdd.toDF().createOrReplaceTempView("nullableRepeatedData") rdd } protected lazy val nullInts: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( NullInts(1) :: NullInts(2) :: NullInts(3) :: NullInts(null) :: Nil).toDF() - df.registerTempTable("nullInts") + df.createOrReplaceTempView("nullInts") df } protected lazy val allNulls: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( NullInts(null) :: NullInts(null) :: NullInts(null) :: NullInts(null) :: Nil).toDF() - df.registerTempTable("allNulls") + df.createOrReplaceTempView("allNulls") df } protected lazy val nullStrings: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( NullStrings(1, "abc") :: NullStrings(2, "ABC") :: NullStrings(3, null) :: Nil).toDF() - df.registerTempTable("nullStrings") + df.createOrReplaceTempView("nullStrings") df } protected lazy val tableName: DataFrame = { - val df = sqlContext.sparkContext.parallelize(TableName("test") :: Nil).toDF() - df.registerTempTable("tableName") + val df = spark.sparkContext.parallelize(TableName("test") :: Nil).toDF() + df.createOrReplaceTempView("tableName") df } protected lazy val unparsedStrings: RDD[String] = { - sqlContext.sparkContext.parallelize( + spark.sparkContext.parallelize( "1, A1, true, null" :: "2, B2, false, null" :: "3, C3, true, null" :: @@ -212,33 +214,44 @@ private[sql] trait SQLTestData { self => // An RDD with 4 elements and 8 partitions protected lazy val withEmptyParts: RDD[IntField] = { - val rdd = sqlContext.sparkContext.parallelize((1 to 4).map(IntField), 8) - rdd.toDF().registerTempTable("withEmptyParts") + val rdd = spark.sparkContext.parallelize((1 to 4).map(IntField), 8) + rdd.toDF().createOrReplaceTempView("withEmptyParts") rdd } protected lazy val person: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( Person(0, "mike", 30) :: Person(1, "jim", 20) :: Nil).toDF() - df.registerTempTable("person") + df.createOrReplaceTempView("person") df } protected lazy val salary: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( Salary(0, 2000.0) :: Salary(1, 1000.0) :: Nil).toDF() - df.registerTempTable("salary") + df.createOrReplaceTempView("salary") df } protected lazy val complexData: DataFrame = { - val df = sqlContext.sparkContext.parallelize( + val df = spark.sparkContext.parallelize( ComplexData(Map("1" -> 1), TestData(1, "1"), Seq(1, 1, 1), true) :: ComplexData(Map("2" -> 2), TestData(2, "2"), Seq(2, 2, 2), false) :: Nil).toDF() - df.registerTempTable("complexData") + df.createOrReplaceTempView("complexData") + df + } + + protected lazy val courseSales: DataFrame = { + val df = spark.sparkContext.parallelize( + CourseSales("dotNET", 2012, 10000) :: + CourseSales("Java", 2012, 20000) :: + CourseSales("dotNET", 2012, 5000) :: + CourseSales("dotNET", 2013, 48000) :: + CourseSales("Java", 2013, 30000) :: Nil).toDF() + df.createOrReplaceTempView("courseSales") df } @@ -246,7 +259,7 @@ private[sql] trait SQLTestData { self => * Initialize all test data such that all temp tables are properly registered. */ def loadTestData(): Unit = { - assert(sqlContext != null, "attempted to initialize test data before SQLContext.") + assert(spark != null, "attempted to initialize test data before SparkSession.") emptyTestData testData testData2 @@ -270,6 +283,7 @@ private[sql] trait SQLTestData { self => person salary complexData + courseSales } } @@ -295,4 +309,5 @@ private[sql] object SQLTestData { case class Person(id: Int, name: String, age: Int) case class Salary(personId: Int, salary: Double) case class ComplexData(m: Map[String, Int], s: TestData, a: Seq[Int], b: Boolean) + case class CourseSales(course: String, year: Int, earnings: Double) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 9214569f18e9..a14a1441a431 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -18,63 +18,67 @@ package org.apache.spark.sql.test import java.io.File -import java.util.UUID +import java.net.URI +import java.nio.file.Files +import java.util.{Locale, UUID} -import scala.util.Try +import scala.concurrent.duration._ import scala.language.implicitConversions +import scala.util.control.NonFatal -import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterAll +import org.scalatest.concurrent.Eventually import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.catalog.SessionCatalog.DEFAULT_DATABASE +import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.util.Utils +import org.apache.spark.sql.execution.FilterExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.{UninterruptibleThread, Utils} /** * Helper trait that should be extended by all SQL test suites. * - * This allows subclasses to plugin a custom [[SQLContext]]. It comes with test data + * This allows subclasses to plugin a custom `SQLContext`. It comes with test data * prepared in advance as well as all implicit conversions used extensively by dataframes. - * To use implicit methods, import `testImplicits._` instead of through the [[SQLContext]]. + * To use implicit methods, import `testImplicits._` instead of through the `SQLContext`. * - * Subclasses should *not* create [[SQLContext]]s in the test suite constructor, which is + * Subclasses should *not* create `SQLContext`s in the test suite constructor, which is * prone to leaving multiple overlapping [[org.apache.spark.SparkContext]]s in the same JVM. */ private[sql] trait SQLTestUtils - extends SparkFunSuite + extends SparkFunSuite with Eventually with BeforeAndAfterAll - with SQLTestData { self => + with SQLTestData + with PlanTest { self => - protected def sparkContext = sqlContext.sparkContext + protected def sparkContext = spark.sparkContext // Whether to materialize all test data before the first test is run private var loadTestDataBeforeTests = false // Shorthand for running a query using our SQLContext - protected lazy val sql = sqlContext.sql _ + protected lazy val sql = spark.sql _ /** * A helper object for importing SQL implicits. * - * Note that the alternative of importing `sqlContext.implicits._` is not possible here. - * This is because we create the [[SQLContext]] immediately before the first test is run, + * Note that the alternative of importing `spark.implicits._` is not possible here. + * This is because we create the `SQLContext` immediately before the first test is run, * but the implicits import is needed in the constructor. */ protected object testImplicits extends SQLImplicits { - protected override def _sqlContext: SQLContext = self.sqlContext - - // This must live here to preserve binary compatibility with Spark < 1.5. - implicit class StringToColumn(val sc: StringContext) { - def $(args: Any*): ColumnName = { - new ColumnName(sc.s(args: _*)) - } - } + protected override def _sqlContext: SQLContext = self.spark.sqlContext } /** - * Materialize the test data immediately after the [[SQLContext]] is set up. + * Materialize the test data immediately after the `SQLContext` is set up. * This is necessary if the data is accessed by name but not through direct reference. */ protected def setupTestData(): Unit = { @@ -88,29 +92,9 @@ private[sql] trait SQLTestUtils } } - /** - * The Hadoop configuration used by the active [[SQLContext]]. - */ - protected def hadoopConfiguration: Configuration = { - sparkContext.hadoopConfiguration - } - - /** - * Sets all SQL configurations specified in `pairs`, calls `f`, and then restore all SQL - * configurations. - * - * @todo Probably this method should be moved to a more general place - */ - protected def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { - val (keys, values) = pairs.unzip - val currentValues = keys.map(key => Try(sqlContext.conf.getConfString(key)).toOption) - (keys, values).zipped.foreach(sqlContext.conf.setConfString) - try f finally { - keys.zip(currentValues).foreach { - case (key, Some(value)) => sqlContext.conf.setConfString(key, value) - case (key, None) => sqlContext.conf.unsetConf(key) - } - } + protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + SparkSession.setActiveSession(spark) + super.withSQLConf(pairs: _*)(f) } /** @@ -125,6 +109,31 @@ private[sql] trait SQLTestUtils try f(path) finally Utils.deleteRecursively(path) } + /** + * Copy file in jar's resource to a temp file, then pass it to `f`. + * This function is used to make `f` can use the path of temp file(e.g. file:/), instead of + * path of jar's resource which starts with 'jar:file:/' + */ + protected def withResourceTempPath(resourcePath: String)(f: File => Unit): Unit = { + val inputStream = + Thread.currentThread().getContextClassLoader.getResourceAsStream(resourcePath) + withTempDir { dir => + val tmpFile = new File(dir, "tmp") + Files.copy(inputStream, tmpFile.toPath) + f(tmpFile) + } + } + + /** + * Waits for all tasks on all executors to be finished. + */ + protected def waitForTasksToFinish(): Unit = { + eventually(timeout(10.seconds)) { + assert(spark.sparkContext.statusTracker + .getExecutorInfos.map(_.numRunningTasks()).sum == 0) + } + } + /** * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` * returns. @@ -133,14 +142,58 @@ private[sql] trait SQLTestUtils */ protected def withTempDir(f: File => Unit): Unit = { val dir = Utils.createTempDir().getCanonicalFile - try f(dir) finally Utils.deleteRecursively(dir) + try f(dir) finally { + // wait for all tasks to finish before deleting files + waitForTasksToFinish() + Utils.deleteRecursively(dir) + } + } + + /** + * Creates the specified number of temporary directories, which is then passed to `f` and will be + * deleted after `f` returns. + */ + protected def withTempPaths(numPaths: Int)(f: Seq[File] => Unit): Unit = { + val files = Array.fill[File](numPaths)(Utils.createTempDir().getCanonicalFile) + try f(files) finally { + // wait for all tasks to finish before deleting files + waitForTasksToFinish() + files.foreach(Utils.deleteRecursively) + } + } + + /** + * Drops functions after calling `f`. A function is represented by (functionName, isTemporary). + */ + protected def withUserDefinedFunction(functions: (String, Boolean)*)(f: => Unit): Unit = { + try { + f + } catch { + case cause: Throwable => throw cause + } finally { + // If the test failed part way, we don't want to mask the failure by failing to remove + // temp tables that never got created. + functions.foreach { case (functionName, isTemporary) => + val withTemporary = if (isTemporary) "TEMPORARY" else "" + spark.sql(s"DROP $withTemporary FUNCTION IF EXISTS $functionName") + assert( + !spark.sessionState.catalog.functionExists(FunctionIdentifier(functionName)), + s"Function $functionName should have been dropped. But, it still exists.") + } + } } /** * Drops temporary table `tableName` after calling `f`. */ - protected def withTempTable(tableNames: String*)(f: => Unit): Unit = { - try f finally tableNames.foreach(sqlContext.dropTempTable) + protected def withTempView(tableNames: String*)(f: => Unit): Unit = { + try f finally { + // If the test failed part way, we don't want to mask the failure by failing to remove + // temp tables that never got created. + try tableNames.foreach(spark.catalog.dropTempView) catch { + case _: NoSuchTableException => + } + } } /** @@ -149,7 +202,18 @@ private[sql] trait SQLTestUtils protected def withTable(tableNames: String*)(f: => Unit): Unit = { try f finally { tableNames.foreach { name => - sqlContext.sql(s"DROP TABLE IF EXISTS $name") + spark.sql(s"DROP TABLE IF EXISTS $name") + } + } + } + + /** + * Drops view `viewName` after calling `f`. + */ + protected def withView(viewNames: String*)(f: => Unit): Unit = { + try f finally { + viewNames.foreach { name => + spark.sql(s"DROP VIEW IF EXISTS $name") } } } @@ -157,17 +221,51 @@ private[sql] trait SQLTestUtils /** * Creates a temporary database and switches current database to it before executing `f`. This * database is dropped after `f` returns. + * + * Note that this method doesn't switch current database before executing `f`. */ protected def withTempDatabase(f: String => Unit): Unit = { val dbName = s"db_${UUID.randomUUID().toString.replace('-', '_')}" try { - sqlContext.sql(s"CREATE DATABASE $dbName") + spark.sql(s"CREATE DATABASE $dbName") } catch { case cause: Throwable => fail("Failed to create temporary database", cause) } - try f(dbName) finally sqlContext.sql(s"DROP DATABASE $dbName CASCADE") + try f(dbName) finally { + if (spark.catalog.currentDatabase == dbName) { + spark.sql(s"USE $DEFAULT_DATABASE") + } + spark.sql(s"DROP DATABASE $dbName CASCADE") + } + } + + /** + * Drops database `dbName` after calling `f`. + */ + protected def withDatabase(dbNames: String*)(f: => Unit): Unit = { + try f finally { + dbNames.foreach { name => + spark.sql(s"DROP DATABASE IF EXISTS $name CASCADE") + } + spark.sql(s"USE $DEFAULT_DATABASE") + } + } + + /** + * Enables Locale `language` before executing `f`, then switches back to the default locale of JVM + * after `f` returns. + */ + protected def withLocale(language: String)(f: => Unit): Unit = { + val originalLocale = Locale.getDefault + try { + // Add Locale setting + Locale.setDefault(new Locale(language)) + f + } finally { + Locale.setDefault(originalLocale) + } } /** @@ -175,16 +273,95 @@ private[sql] trait SQLTestUtils * `f` returns. */ protected def activateDatabase(db: String)(f: => Unit): Unit = { - sqlContext.sql(s"USE $db") - try f finally sqlContext.sql(s"USE default") + spark.sessionState.catalog.setCurrentDatabase(db) + try f finally spark.sessionState.catalog.setCurrentDatabase("default") } /** - * Turn a logical plan into a [[DataFrame]]. This should be removed once we have an easier - * way to construct [[DataFrame]] directly out of local data without relying on implicits. + * Strip Spark-side filtering in order to check if a datasource filters rows correctly. + */ + protected def stripSparkFilter(df: DataFrame): DataFrame = { + val schema = df.schema + val withoutFilters = df.queryExecution.sparkPlan.transform { + case FilterExec(_, child) => child + } + + spark.internalCreateDataFrame(withoutFilters.execute(), schema) + } + + /** + * Turn a logical plan into a `DataFrame`. This should be removed once we have an easier + * way to construct `DataFrame` directly out of local data without relying on implicits. */ protected implicit def logicalPlanToSparkQuery(plan: LogicalPlan): DataFrame = { - DataFrame(sqlContext, plan) + Dataset.ofRows(spark, plan) + } + + /** + * Disable stdout and stderr when running the test. To not output the logs to the console, + * ConsoleAppender's `follow` should be set to `true` so that it will honors reassignments of + * System.out or System.err. Otherwise, ConsoleAppender will still output to the console even if + * we change System.out and System.err. + */ + protected def testQuietly(name: String)(f: => Unit): Unit = { + test(name) { + quietly { + f + } + } + } + + /** + * Run a test on a separate `UninterruptibleThread`. + */ + protected def testWithUninterruptibleThread(name: String, quietly: Boolean = false) + (body: => Unit): Unit = { + val timeoutMillis = 10000 + @transient var ex: Throwable = null + + def runOnThread(): Unit = { + val thread = new UninterruptibleThread(s"Testing thread for test $name") { + override def run(): Unit = { + try { + body + } catch { + case NonFatal(e) => + ex = e + } + } + } + thread.setDaemon(true) + thread.start() + thread.join(timeoutMillis) + if (thread.isAlive) { + thread.interrupt() + // If this interrupt does not work, then this thread is most likely running something that + // is not interruptible. There is not much point to wait for the thread to termniate, and + // we rather let the JVM terminate the thread on exit. + fail( + s"Test '$name' running on o.a.s.util.UninterruptibleThread timed out after" + + s" $timeoutMillis ms") + } else if (ex != null) { + throw ex + } + } + + if (quietly) { + testQuietly(name) { runOnThread() } + } else { + test(name) { runOnThread() } + } + } + + /** + * This method is used to make the given path qualified, when a path + * does not contain a scheme, this path will not be changed after the default + * FileSystem is changed. + */ + def makeQualifiedPath(path: String): URI = { + val hadoopPath = new Path(path) + val fs = hadoopPath.getFileSystem(spark.sessionState.newHadoopConf()) + fs.makeQualified(hadoopPath).toUri } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala index 963d10eed62e..cd8d0708d8a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala @@ -17,33 +17,56 @@ package org.apache.spark.sql.test -import org.apache.spark.sql.SQLContext +import scala.concurrent.duration._ +import org.scalatest.BeforeAndAfterEach +import org.scalatest.concurrent.Eventually + +import org.apache.spark.{DebugFilesystem, SparkConf} +import org.apache.spark.sql.{SparkSession, SQLContext} +import org.apache.spark.sql.internal.SQLConf /** - * Helper trait for SQL test suites where all tests share a single [[TestSQLContext]]. + * Helper trait for SQL test suites where all tests share a single [[TestSparkSession]]. */ -trait SharedSQLContext extends SQLTestUtils { +trait SharedSQLContext extends SQLTestUtils with BeforeAndAfterEach with Eventually { + + protected def sparkConf = { + new SparkConf() + .set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) + .set("spark.unsafe.exceptionOnMemoryLeak", "true") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") + } /** - * The [[TestSQLContext]] to use for all tests in this suite. + * The [[TestSparkSession]] to use for all tests in this suite. * * By default, the underlying [[org.apache.spark.SparkContext]] will be run in local * mode with the default test configurations. */ - private var _ctx: TestSQLContext = null + private var _spark: TestSparkSession = null + + /** + * The [[TestSparkSession]] to use for all tests in this suite. + */ + protected implicit def spark: SparkSession = _spark /** * The [[TestSQLContext]] to use for all tests in this suite. */ - protected def sqlContext: SQLContext = _ctx + protected implicit def sqlContext: SQLContext = _spark.sqlContext + + protected def createSparkSession: TestSparkSession = { + new TestSparkSession(sparkConf) + } /** - * Initialize the [[TestSQLContext]]. + * Initialize the [[TestSparkSession]]. */ protected override def beforeAll(): Unit = { - if (_ctx == null) { - _ctx = new TestSQLContext + SparkSession.sqlListener.set(null) + if (_spark == null) { + _spark = createSparkSession } // Ensure we have initialized the context before calling parent code super.beforeAll() @@ -53,13 +76,27 @@ trait SharedSQLContext extends SQLTestUtils { * Stop the underlying [[org.apache.spark.SparkContext]], if any. */ protected override def afterAll(): Unit = { - try { - if (_ctx != null) { - _ctx.sparkContext.stop() - _ctx = null - } - } finally { - super.afterAll() + super.afterAll() + if (_spark != null) { + _spark.sessionState.catalog.reset() + _spark.stop() + _spark = null + } + } + + protected override def beforeEach(): Unit = { + super.beforeEach() + DebugFilesystem.clearOpenStreams() + } + + protected override def afterEach(): Unit = { + super.afterEach() + // Clear all persistent datasets after each test + spark.sharedState.cacheManager.clearCache() + // files can be closed from other threads, so wait a bit + // normally this doesn't take more than 1s + eventually(timeout(10.seconds)) { + DebugFilesystem.assertNoOpenStreams() } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala index c89a1516503e..959edf9a4937 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -18,31 +18,25 @@ package org.apache.spark.sql.test import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.{SQLConf, SQLContext} - +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.internal.{SessionState, SessionStateBuilder, SQLConf, WithTestConf} /** - * A special [[SQLContext]] prepared for testing. + * A special `SparkSession` prepared for testing. */ -private[sql] class TestSQLContext(sc: SparkContext) extends SQLContext(sc) { self => - - def this() { +private[sql] class TestSparkSession(sc: SparkContext) extends SparkSession(sc) { self => + def this(sparkConf: SparkConf) { this(new SparkContext("local[2]", "test-sql-context", - new SparkConf().set("spark.sql.testkey", "true"))) + sparkConf.set("spark.sql.testkey", "true"))) } - protected[sql] override lazy val conf: SQLConf = new SQLConf { - - clear() - - override def clear(): Unit = { - super.clear() + def this() { + this(new SparkConf) + } - // Make sure we start with the default test configs even after clear - TestSQLContext.overrideConfs.map { - case (key, value) => setConfString(key, value) - } - } + @transient + override lazy val sessionState: SessionState = { + new TestSQLSessionStateBuilder(this, None).build() } // Needed for Java tests @@ -51,10 +45,11 @@ private[sql] class TestSQLContext(sc: SparkContext) extends SQLContext(sc) { sel } private object testData extends SQLTestData { - protected override def sqlContext: SQLContext = self + protected override def spark: SparkSession = self } } + private[sql] object TestSQLContext { /** @@ -65,3 +60,11 @@ private[sql] object TestSQLContext { // Fewer shuffle partitions to speed up testing. SQLConf.SHUFFLE_PARTITIONS.key -> "5") } + +private[sql] class TestSQLSessionStateBuilder( + session: SparkSession, + state: Option[SessionState]) + extends SessionStateBuilder(session, state) with WithTestConf { + override def overrideConfs: Map[String, String] = TestSQLContext.overrideConfs + override def newBuilder: NewBuilder = new TestSQLSessionStateBuilder(_, _) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index b46b0d2f6040..a239e39d9c5a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -20,9 +20,12 @@ package org.apache.spark.sql.util import scala.collection.mutable.ArrayBuffer import org.apache.spark._ -import org.apache.spark.sql.{functions, QueryTest} -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Project} -import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.{functions, AnalysisException, QueryTest} +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, InsertIntoTable, LogicalPlan, Project} +import org.apache.spark.sql.execution.{QueryExecution, WholeStageCodegenExec} +import org.apache.spark.sql.execution.datasources.{CreateTable, InsertIntoHadoopFsRelationCommand} +import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.test.SharedSQLContext class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { @@ -39,7 +42,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { metrics += ((funcName, qe, duration)) } } - sqlContext.listenerManager.register(listener) + spark.listenerManager.register(listener) val df = Seq(1 -> "a").toDF("i", "j") df.select("i").collect() @@ -55,10 +58,10 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { assert(metrics(1)._2.analyzed.isInstanceOf[Aggregate]) assert(metrics(1)._3 > 0) - sqlContext.listenerManager.unregister(listener) + spark.listenerManager.unregister(listener) } - test("execute callback functions when a DataFrame action failed") { + testQuietly("execute callback functions when a DataFrame action failed") { val metrics = ArrayBuffer.empty[(String, QueryExecution, Exception)] val listener = new QueryExecutionListener { override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = { @@ -68,13 +71,11 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { // Only test failed case here, so no need to implement `onSuccess` override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = {} } - sqlContext.listenerManager.register(listener) + spark.listenerManager.register(listener) val errorUdf = udf[Int, Int] { _ => throw new RuntimeException("udf error") } val df = sparkContext.makeRDD(Seq(1 -> "a")).toDF("i", "j") - // Ignore the log when we are expecting an exception. - sparkContext.setLogLevel("FATAL") val e = intercept[SparkException](df.select(errorUdf($"i")).collect()) assert(metrics.length == 1) @@ -82,7 +83,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { assert(metrics(0)._2.analyzed.isInstanceOf[Project]) assert(metrics(0)._3.getMessage == e.getMessage) - sqlContext.listenerManager.unregister(listener) + spark.listenerManager.unregister(listener) } test("get numRows metrics by callback") { @@ -92,10 +93,14 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = {} override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { - metrics += qe.executedPlan.longMetric("numInputRows").value.value + val metric = qe.executedPlan match { + case w: WholeStageCodegenExec => w.child.longMetric("numOutputRows") + case other => other.longMetric("numOutputRows") + } + metrics += metric.value } } - sqlContext.listenerManager.register(listener) + spark.listenerManager.register(listener) val df = Seq(1 -> "a").toDF("i", "j").groupBy("i").count() df.collect() @@ -103,11 +108,11 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { Seq(1 -> "a", 2 -> "a").toDF("i", "j").groupBy("i").count().collect() assert(metrics.length == 3) - assert(metrics(0) == 1) - assert(metrics(1) == 1) - assert(metrics(2) == 2) + assert(metrics(0) === 1) + assert(metrics(1) === 1) + assert(metrics(2) === 2) - sqlContext.listenerManager.unregister(listener) + spark.listenerManager.unregister(listener) } // TODO: Currently some LongSQLMetric use -1 as initial value, so if the accumulator is never @@ -122,15 +127,15 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = {} override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { - metrics += qe.executedPlan.longMetric("dataSize").value.value + metrics += qe.executedPlan.longMetric("dataSize").value val bottomAgg = qe.executedPlan.children(0).children(0) - metrics += bottomAgg.longMetric("dataSize").value.value + metrics += bottomAgg.longMetric("dataSize").value } } - sqlContext.listenerManager.register(listener) + spark.listenerManager.register(listener) val sparkListener = new SaveInfoListener - sqlContext.sparkContext.addSparkListener(sparkListener) + spark.sparkContext.addSparkListener(sparkListener) val df = (1 to 100).map(i => i -> i.toString).toDF("i", "j") df.groupBy("i").count().collect() @@ -140,7 +145,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { .filter(_._2.name == InternalAccumulator.PEAK_EXECUTION_MEMORY) assert(peakMemoryAccumulator.size == 1) - peakMemoryAccumulator.head._2.value.toLong + peakMemoryAccumulator.head._2.value.get.asInstanceOf[Long] } assert(sparkListener.getCompletedStageInfos.length == 2) @@ -153,6 +158,59 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { assert(metrics(0) == topAggDataSize) assert(metrics(1) == bottomAggDataSize) - sqlContext.listenerManager.unregister(listener) + spark.listenerManager.unregister(listener) + } + + test("execute callback functions for DataFrameWriter") { + val commands = ArrayBuffer.empty[(String, LogicalPlan)] + val exceptions = ArrayBuffer.empty[(String, Exception)] + val listener = new QueryExecutionListener { + override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = { + exceptions += funcName -> exception + } + + override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { + commands += funcName -> qe.logical + } + } + spark.listenerManager.register(listener) + + withTempPath { path => + spark.range(10).write.format("json").save(path.getCanonicalPath) + assert(commands.length == 1) + assert(commands.head._1 == "save") + assert(commands.head._2.isInstanceOf[InsertIntoHadoopFsRelationCommand]) + assert(commands.head._2.asInstanceOf[InsertIntoHadoopFsRelationCommand] + .fileFormat.isInstanceOf[JsonFileFormat]) + } + + withTable("tab") { + sql("CREATE TABLE tab(i long) using parquet") // adds commands(1) via onSuccess + spark.range(10).write.insertInto("tab") + assert(commands.length == 3) + assert(commands(2)._1 == "insertInto") + assert(commands(2)._2.isInstanceOf[InsertIntoTable]) + assert(commands(2)._2.asInstanceOf[InsertIntoTable].table + .asInstanceOf[UnresolvedRelation].tableIdentifier.table == "tab") + } + // exiting withTable adds commands(3) via onSuccess (drops tab) + + withTable("tab") { + spark.range(10).select($"id", $"id" % 5 as "p").write.partitionBy("p").saveAsTable("tab") + assert(commands.length == 5) + assert(commands(4)._1 == "saveAsTable") + assert(commands(4)._2.isInstanceOf[CreateTable]) + assert(commands(4)._2.asInstanceOf[CreateTable].tableDesc.partitionColumnNames == Seq("p")) + } + + withTable("tab") { + sql("CREATE TABLE tab(i long) using parquet") + val e = intercept[AnalysisException] { + spark.range(10).select($"id", $"id").write.insertInto("tab") + } + assert(exceptions.length == 1) + assert(exceptions.head._1 == "insertInto") + assert(exceptions.head._2 == e) + } } } diff --git a/sql/create-docs.sh b/sql/create-docs.sh new file mode 100755 index 000000000000..4353708d22f7 --- /dev/null +++ b/sql/create-docs.sh @@ -0,0 +1,53 @@ +#!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Script to create SQL API docs. This requires `mkdocs` and to build +# Spark first. After running this script the html docs can be found in +# $SPARK_HOME/sql/site + +set -o pipefail +set -e + +FWDIR="$(cd "`dirname "${BASH_SOURCE[0]}"`"; pwd)" +SPARK_HOME="$(cd "`dirname "${BASH_SOURCE[0]}"`"/..; pwd)" + +if ! hash python 2>/dev/null; then + echo "Missing python in your path, skipping SQL documentation generation." + exit 0 +fi + +if ! hash mkdocs 2>/dev/null; then + echo "Missing mkdocs in your path, trying to install mkdocs for SQL documentation generation." + pip install mkdocs +fi + +pushd "$FWDIR" > /dev/null + +# Now create the markdown file +rm -fr docs +mkdir docs +echo "Generating markdown files for SQL documentation." +"$SPARK_HOME/bin/spark-submit" gen-sql-markdown.py + +# Now create the HTML files +echo "Generating HTML files for SQL documentation." +mkdocs build --clean +rm -fr docs + +popd diff --git a/sql/gen-sql-markdown.py b/sql/gen-sql-markdown.py new file mode 100644 index 000000000000..fa8124b4513a --- /dev/null +++ b/sql/gen-sql-markdown.py @@ -0,0 +1,197 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +import os +from collections import namedtuple + +ExpressionInfo = namedtuple( + "ExpressionInfo", "className name usage arguments examples note since") + + +def _list_function_infos(jvm): + """ + Returns a list of function information via JVM. Sorts wrapped expression infos by name + and returns them. + """ + + jinfos = jvm.org.apache.spark.sql.api.python.PythonSQLUtils.listBuiltinFunctionInfos() + infos = [] + for jinfo in jinfos: + name = jinfo.getName() + usage = jinfo.getUsage() + usage = usage.replace("_FUNC_", name) if usage is not None else usage + infos.append(ExpressionInfo( + className=jinfo.getClassName(), + name=name, + usage=usage, + arguments=jinfo.getArguments().replace("_FUNC_", name), + examples=jinfo.getExamples().replace("_FUNC_", name), + note=jinfo.getNote(), + since=jinfo.getSince())) + return sorted(infos, key=lambda i: i.name) + + +def _make_pretty_usage(usage): + """ + Makes the usage description pretty and returns a formatted string if `usage` + is not an empty string. Otherwise, returns None. + """ + + if usage is not None and usage.strip() != "": + usage = "\n".join(map(lambda u: u.strip(), usage.split("\n"))) + return "%s\n\n" % usage + + +def _make_pretty_arguments(arguments): + """ + Makes the arguments description pretty and returns a formatted string if `arguments` + starts with the argument prefix. Otherwise, returns None. + + Expected input: + + Arguments: + * arg0 - ... + ... + * arg0 - ... + ... + + Expected output: + **Arguments:** + + * arg0 - ... + ... + * arg0 - ... + ... + + """ + + if arguments.startswith("\n Arguments:"): + arguments = "\n".join(map(lambda u: u[6:], arguments.strip().split("\n")[1:])) + return "**Arguments:**\n\n%s\n\n" % arguments + + +def _make_pretty_examples(examples): + """ + Makes the examples description pretty and returns a formatted string if `examples` + starts with the example prefix. Otherwise, returns None. + + Expected input: + + Examples: + > SELECT ...; + ... + > SELECT ...; + ... + + Expected output: + **Examples:** + + ``` + > SELECT ...; + ... + > SELECT ...; + ... + ``` + + """ + + if examples.startswith("\n Examples:"): + examples = "\n".join(map(lambda u: u[6:], examples.strip().split("\n")[1:])) + return "**Examples:**\n\n```\n%s\n```\n\n" % examples + + +def _make_pretty_note(note): + """ + Makes the note description pretty and returns a formatted string if `note` is not + an empty string. Otherwise, returns None. + + Expected input: + + ... + + Expected output: + **Note:** + + ... + + """ + + if note != "": + note = "\n".join(map(lambda n: n[4:], note.split("\n"))) + return "**Note:**\n%s\n" % note + + +def generate_sql_markdown(jvm, path): + """ + Generates a markdown file after listing the function information. The output file + is created in `path`. + + Expected output: + ### NAME + + USAGE + + **Arguments:** + + ARGUMENTS + + **Examples:** + + ``` + EXAMPLES + ``` + + **Note:** + + NOTE + + **Since:** SINCE + +
    + + """ + + with open(path, 'w') as mdfile: + for info in _list_function_infos(jvm): + name = info.name + usage = _make_pretty_usage(info.usage) + arguments = _make_pretty_arguments(info.arguments) + examples = _make_pretty_examples(info.examples) + note = _make_pretty_note(info.note) + since = info.since + + mdfile.write("### %s\n\n" % name) + if usage is not None: + mdfile.write("%s\n\n" % usage.strip()) + if arguments is not None: + mdfile.write(arguments) + if examples is not None: + mdfile.write(examples) + if note is not None: + mdfile.write(note) + if since is not None and since != "": + mdfile.write("**Since:** %s\n\n" % since.strip()) + mdfile.write("
    \n\n") + + +if __name__ == "__main__": + from pyspark.java_gateway import launch_gateway + + jvm = launch_gateway().jvm + markdown_file_path = "%s/docs/index.md" % os.path.dirname(sys.argv[0]) + generate_sql_markdown(jvm, markdown_file_path) diff --git a/sql/hive-thriftserver/if/TCLIService.thrift b/sql/hive-thriftserver/if/TCLIService.thrift new file mode 100644 index 000000000000..7cd6fa37cec3 --- /dev/null +++ b/sql/hive-thriftserver/if/TCLIService.thrift @@ -0,0 +1,1174 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Coding Conventions for this file: +// +// Structs/Enums/Unions +// * Struct, Enum, and Union names begin with a "T", +// and use a capital letter for each new word, with no underscores. +// * All fields should be declared as either optional or required. +// +// Functions +// * Function names start with a capital letter and have a capital letter for +// each new word, with no underscores. +// * Each function should take exactly one parameter, named TFunctionNameReq, +// and should return either void or TFunctionNameResp. This convention allows +// incremental updates. +// +// Services +// * Service names begin with the letter "T", use a capital letter for each +// new word (with no underscores), and end with the word "Service". + +namespace java org.apache.hive.service.cli.thrift +namespace cpp apache.hive.service.cli.thrift + +// List of protocol versions. A new token should be +// added to the end of this list every time a change is made. +enum TProtocolVersion { + HIVE_CLI_SERVICE_PROTOCOL_V1, + + // V2 adds support for asynchronous execution + HIVE_CLI_SERVICE_PROTOCOL_V2 + + // V3 add varchar type, primitive type qualifiers + HIVE_CLI_SERVICE_PROTOCOL_V3 + + // V4 add decimal precision/scale, char type + HIVE_CLI_SERVICE_PROTOCOL_V4 + + // V5 adds error details when GetOperationStatus returns in error state + HIVE_CLI_SERVICE_PROTOCOL_V5 + + // V6 uses binary type for binary payload (was string) and uses columnar result set + HIVE_CLI_SERVICE_PROTOCOL_V6 + + // V7 adds support for delegation token based connection + HIVE_CLI_SERVICE_PROTOCOL_V7 + + // V8 adds support for interval types + HIVE_CLI_SERVICE_PROTOCOL_V8 +} + +enum TTypeId { + BOOLEAN_TYPE, + TINYINT_TYPE, + SMALLINT_TYPE, + INT_TYPE, + BIGINT_TYPE, + FLOAT_TYPE, + DOUBLE_TYPE, + STRING_TYPE, + TIMESTAMP_TYPE, + BINARY_TYPE, + ARRAY_TYPE, + MAP_TYPE, + STRUCT_TYPE, + UNION_TYPE, + USER_DEFINED_TYPE, + DECIMAL_TYPE, + NULL_TYPE, + DATE_TYPE, + VARCHAR_TYPE, + CHAR_TYPE, + INTERVAL_YEAR_MONTH_TYPE, + INTERVAL_DAY_TIME_TYPE +} + +const set PRIMITIVE_TYPES = [ + TTypeId.BOOLEAN_TYPE, + TTypeId.TINYINT_TYPE, + TTypeId.SMALLINT_TYPE, + TTypeId.INT_TYPE, + TTypeId.BIGINT_TYPE, + TTypeId.FLOAT_TYPE, + TTypeId.DOUBLE_TYPE, + TTypeId.STRING_TYPE, + TTypeId.TIMESTAMP_TYPE, + TTypeId.BINARY_TYPE, + TTypeId.DECIMAL_TYPE, + TTypeId.NULL_TYPE, + TTypeId.DATE_TYPE, + TTypeId.VARCHAR_TYPE, + TTypeId.CHAR_TYPE, + TTypeId.INTERVAL_YEAR_MONTH_TYPE, + TTypeId.INTERVAL_DAY_TIME_TYPE +] + +const set COMPLEX_TYPES = [ + TTypeId.ARRAY_TYPE + TTypeId.MAP_TYPE + TTypeId.STRUCT_TYPE + TTypeId.UNION_TYPE + TTypeId.USER_DEFINED_TYPE +] + +const set COLLECTION_TYPES = [ + TTypeId.ARRAY_TYPE + TTypeId.MAP_TYPE +] + +const map TYPE_NAMES = { + TTypeId.BOOLEAN_TYPE: "BOOLEAN", + TTypeId.TINYINT_TYPE: "TINYINT", + TTypeId.SMALLINT_TYPE: "SMALLINT", + TTypeId.INT_TYPE: "INT", + TTypeId.BIGINT_TYPE: "BIGINT", + TTypeId.FLOAT_TYPE: "FLOAT", + TTypeId.DOUBLE_TYPE: "DOUBLE", + TTypeId.STRING_TYPE: "STRING", + TTypeId.TIMESTAMP_TYPE: "TIMESTAMP", + TTypeId.BINARY_TYPE: "BINARY", + TTypeId.ARRAY_TYPE: "ARRAY", + TTypeId.MAP_TYPE: "MAP", + TTypeId.STRUCT_TYPE: "STRUCT", + TTypeId.UNION_TYPE: "UNIONTYPE", + TTypeId.DECIMAL_TYPE: "DECIMAL", + TTypeId.NULL_TYPE: "NULL" + TTypeId.DATE_TYPE: "DATE" + TTypeId.VARCHAR_TYPE: "VARCHAR" + TTypeId.CHAR_TYPE: "CHAR" + TTypeId.INTERVAL_YEAR_MONTH_TYPE: "INTERVAL_YEAR_MONTH" + TTypeId.INTERVAL_DAY_TIME_TYPE: "INTERVAL_DAY_TIME" +} + +// Thrift does not support recursively defined types or forward declarations, +// which makes it difficult to represent Hive's nested types. +// To get around these limitations TTypeDesc employs a type list that maps +// integer "pointers" to TTypeEntry objects. The following examples show +// how different types are represented using this scheme: +// +// "INT": +// TTypeDesc { +// types = [ +// TTypeEntry.primitive_entry { +// type = INT_TYPE +// } +// ] +// } +// +// "ARRAY": +// TTypeDesc { +// types = [ +// TTypeEntry.array_entry { +// object_type_ptr = 1 +// }, +// TTypeEntry.primitive_entry { +// type = INT_TYPE +// } +// ] +// } +// +// "MAP": +// TTypeDesc { +// types = [ +// TTypeEntry.map_entry { +// key_type_ptr = 1 +// value_type_ptr = 2 +// }, +// TTypeEntry.primitive_entry { +// type = INT_TYPE +// }, +// TTypeEntry.primitive_entry { +// type = STRING_TYPE +// } +// ] +// } + +typedef i32 TTypeEntryPtr + +// Valid TTypeQualifiers key names +const string CHARACTER_MAXIMUM_LENGTH = "characterMaximumLength" + +// Type qualifier key name for decimal +const string PRECISION = "precision" +const string SCALE = "scale" + +union TTypeQualifierValue { + 1: optional i32 i32Value + 2: optional string stringValue +} + +// Type qualifiers for primitive type. +struct TTypeQualifiers { + 1: required map qualifiers +} + +// Type entry for a primitive type. +struct TPrimitiveTypeEntry { + // The primitive type token. This must satisfy the condition + // that type is in the PRIMITIVE_TYPES set. + 1: required TTypeId type + 2: optional TTypeQualifiers typeQualifiers +} + +// Type entry for an ARRAY type. +struct TArrayTypeEntry { + 1: required TTypeEntryPtr objectTypePtr +} + +// Type entry for a MAP type. +struct TMapTypeEntry { + 1: required TTypeEntryPtr keyTypePtr + 2: required TTypeEntryPtr valueTypePtr +} + +// Type entry for a STRUCT type. +struct TStructTypeEntry { + 1: required map nameToTypePtr +} + +// Type entry for a UNIONTYPE type. +struct TUnionTypeEntry { + 1: required map nameToTypePtr +} + +struct TUserDefinedTypeEntry { + // The fully qualified name of the class implementing this type. + 1: required string typeClassName +} + +// We use a union here since Thrift does not support inheritance. +union TTypeEntry { + 1: TPrimitiveTypeEntry primitiveEntry + 2: TArrayTypeEntry arrayEntry + 3: TMapTypeEntry mapEntry + 4: TStructTypeEntry structEntry + 5: TUnionTypeEntry unionEntry + 6: TUserDefinedTypeEntry userDefinedTypeEntry +} + +// Type descriptor for columns. +struct TTypeDesc { + // The "top" type is always the first element of the list. + // If the top type is an ARRAY, MAP, STRUCT, or UNIONTYPE + // type, then subsequent elements represent nested types. + 1: required list types +} + +// A result set column descriptor. +struct TColumnDesc { + // The name of the column + 1: required string columnName + + // The type descriptor for this column + 2: required TTypeDesc typeDesc + + // The ordinal position of this column in the schema + 3: required i32 position + + 4: optional string comment +} + +// Metadata used to describe the schema (column names, types, comments) +// of result sets. +struct TTableSchema { + 1: required list columns +} + +// A Boolean column value. +struct TBoolValue { + // NULL if value is unset. + 1: optional bool value +} + +// A Byte column value. +struct TByteValue { + // NULL if value is unset. + 1: optional byte value +} + +// A signed, 16 bit column value. +struct TI16Value { + // NULL if value is unset + 1: optional i16 value +} + +// A signed, 32 bit column value +struct TI32Value { + // NULL if value is unset + 1: optional i32 value +} + +// A signed 64 bit column value +struct TI64Value { + // NULL if value is unset + 1: optional i64 value +} + +// A floating point 64 bit column value +struct TDoubleValue { + // NULL if value is unset + 1: optional double value +} + +struct TStringValue { + // NULL if value is unset + 1: optional string value +} + +// A single column value in a result set. +// Note that Hive's type system is richer than Thrift's, +// so in some cases we have to map multiple Hive types +// to the same Thrift type. On the client-side this is +// disambiguated by looking at the Schema of the +// result set. +union TColumnValue { + 1: TBoolValue boolVal // BOOLEAN + 2: TByteValue byteVal // TINYINT + 3: TI16Value i16Val // SMALLINT + 4: TI32Value i32Val // INT + 5: TI64Value i64Val // BIGINT, TIMESTAMP + 6: TDoubleValue doubleVal // FLOAT, DOUBLE + 7: TStringValue stringVal // STRING, LIST, MAP, STRUCT, UNIONTYPE, BINARY, DECIMAL, NULL, INTERVAL_YEAR_MONTH, INTERVAL_DAY_TIME +} + +// Represents a row in a rowset. +struct TRow { + 1: required list colVals +} + +struct TBoolColumn { + 1: required list values + 2: required binary nulls +} + +struct TByteColumn { + 1: required list values + 2: required binary nulls +} + +struct TI16Column { + 1: required list values + 2: required binary nulls +} + +struct TI32Column { + 1: required list values + 2: required binary nulls +} + +struct TI64Column { + 1: required list values + 2: required binary nulls +} + +struct TDoubleColumn { + 1: required list values + 2: required binary nulls +} + +struct TStringColumn { + 1: required list values + 2: required binary nulls +} + +struct TBinaryColumn { + 1: required list values + 2: required binary nulls +} + +// Note that Hive's type system is richer than Thrift's, +// so in some cases we have to map multiple Hive types +// to the same Thrift type. On the client-side this is +// disambiguated by looking at the Schema of the +// result set. +union TColumn { + 1: TBoolColumn boolVal // BOOLEAN + 2: TByteColumn byteVal // TINYINT + 3: TI16Column i16Val // SMALLINT + 4: TI32Column i32Val // INT + 5: TI64Column i64Val // BIGINT, TIMESTAMP + 6: TDoubleColumn doubleVal // FLOAT, DOUBLE + 7: TStringColumn stringVal // STRING, LIST, MAP, STRUCT, UNIONTYPE, DECIMAL, NULL + 8: TBinaryColumn binaryVal // BINARY +} + +// Represents a rowset +struct TRowSet { + // The starting row offset of this rowset. + 1: required i64 startRowOffset + 2: required list rows + 3: optional list columns +} + +// The return status code contained in each response. +enum TStatusCode { + SUCCESS_STATUS, + SUCCESS_WITH_INFO_STATUS, + STILL_EXECUTING_STATUS, + ERROR_STATUS, + INVALID_HANDLE_STATUS +} + +// The return status of a remote request +struct TStatus { + 1: required TStatusCode statusCode + + // If status is SUCCESS_WITH_INFO, info_msgs may be populated with + // additional diagnostic information. + 2: optional list infoMessages + + // If status is ERROR, then the following fields may be set + 3: optional string sqlState // as defined in the ISO/IEF CLI specification + 4: optional i32 errorCode // internal error code + 5: optional string errorMessage +} + +// The state of an operation (i.e. a query or other +// asynchronous operation that generates a result set) +// on the server. +enum TOperationState { + // The operation has been initialized + INITIALIZED_STATE, + + // The operation is running. In this state the result + // set is not available. + RUNNING_STATE, + + // The operation has completed. When an operation is in + // this state its result set may be fetched. + FINISHED_STATE, + + // The operation was canceled by a client + CANCELED_STATE, + + // The operation was closed by a client + CLOSED_STATE, + + // The operation failed due to an error + ERROR_STATE, + + // The operation is in an unrecognized state + UKNOWN_STATE, + + // The operation is in an pending state + PENDING_STATE, +} + +// A string identifier. This is interpreted literally. +typedef string TIdentifier + +// A search pattern. +// +// Valid search pattern characters: +// '_': Any single character. +// '%': Any sequence of zero or more characters. +// '\': Escape character used to include special characters, +// e.g. '_', '%', '\'. If a '\' precedes a non-special +// character it has no special meaning and is interpreted +// literally. +typedef string TPattern + + +// A search pattern or identifier. Used as input +// parameter for many of the catalog functions. +typedef string TPatternOrIdentifier + +struct THandleIdentifier { + // 16 byte globally unique identifier + // This is the public ID of the handle and + // can be used for reporting. + 1: required binary guid, + + // 16 byte secret generated by the server + // and used to verify that the handle is not + // being hijacked by another user. + 2: required binary secret, +} + +// Client-side handle to persistent +// session information on the server-side. +struct TSessionHandle { + 1: required THandleIdentifier sessionId +} + +// The subtype of an OperationHandle. +enum TOperationType { + EXECUTE_STATEMENT, + GET_TYPE_INFO, + GET_CATALOGS, + GET_SCHEMAS, + GET_TABLES, + GET_TABLE_TYPES, + GET_COLUMNS, + GET_FUNCTIONS, + UNKNOWN, +} + +// Client-side reference to a task running +// asynchronously on the server. +struct TOperationHandle { + 1: required THandleIdentifier operationId + 2: required TOperationType operationType + + // If hasResultSet = TRUE, then this operation + // generates a result set that can be fetched. + // Note that the result set may be empty. + // + // If hasResultSet = FALSE, then this operation + // does not generate a result set, and calling + // GetResultSetMetadata or FetchResults against + // this OperationHandle will generate an error. + 3: required bool hasResultSet + + // For operations that don't generate result sets, + // modifiedRowCount is either: + // + // 1) The number of rows that were modified by + // the DML operation (e.g. number of rows inserted, + // number of rows deleted, etc). + // + // 2) 0 for operations that don't modify or add rows. + // + // 3) < 0 if the operation is capable of modifiying rows, + // but Hive is unable to determine how many rows were + // modified. For example, Hive's LOAD DATA command + // doesn't generate row count information because + // Hive doesn't inspect the data as it is loaded. + // + // modifiedRowCount is unset if the operation generates + // a result set. + 4: optional double modifiedRowCount +} + + +// OpenSession() +// +// Open a session (connection) on the server against +// which operations may be executed. +struct TOpenSessionReq { + // The version of the HiveServer2 protocol that the client is using. + 1: required TProtocolVersion client_protocol = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8 + + // Username and password for authentication. + // Depending on the authentication scheme being used, + // this information may instead be provided by a lower + // protocol layer, in which case these fields may be + // left unset. + 2: optional string username + 3: optional string password + + // Configuration overlay which is applied when the session is + // first created. + 4: optional map configuration +} + +struct TOpenSessionResp { + 1: required TStatus status + + // The protocol version that the server is using. + 2: required TProtocolVersion serverProtocolVersion = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8 + + // Session Handle + 3: optional TSessionHandle sessionHandle + + // The configuration settings for this session. + 4: optional map configuration +} + + +// CloseSession() +// +// Closes the specified session and frees any resources +// currently allocated to that session. Any open +// operations in that session will be canceled. +struct TCloseSessionReq { + 1: required TSessionHandle sessionHandle +} + +struct TCloseSessionResp { + 1: required TStatus status +} + + + +enum TGetInfoType { + CLI_MAX_DRIVER_CONNECTIONS = 0, + CLI_MAX_CONCURRENT_ACTIVITIES = 1, + CLI_DATA_SOURCE_NAME = 2, + CLI_FETCH_DIRECTION = 8, + CLI_SERVER_NAME = 13, + CLI_SEARCH_PATTERN_ESCAPE = 14, + CLI_DBMS_NAME = 17, + CLI_DBMS_VER = 18, + CLI_ACCESSIBLE_TABLES = 19, + CLI_ACCESSIBLE_PROCEDURES = 20, + CLI_CURSOR_COMMIT_BEHAVIOR = 23, + CLI_DATA_SOURCE_READ_ONLY = 25, + CLI_DEFAULT_TXN_ISOLATION = 26, + CLI_IDENTIFIER_CASE = 28, + CLI_IDENTIFIER_QUOTE_CHAR = 29, + CLI_MAX_COLUMN_NAME_LEN = 30, + CLI_MAX_CURSOR_NAME_LEN = 31, + CLI_MAX_SCHEMA_NAME_LEN = 32, + CLI_MAX_CATALOG_NAME_LEN = 34, + CLI_MAX_TABLE_NAME_LEN = 35, + CLI_SCROLL_CONCURRENCY = 43, + CLI_TXN_CAPABLE = 46, + CLI_USER_NAME = 47, + CLI_TXN_ISOLATION_OPTION = 72, + CLI_INTEGRITY = 73, + CLI_GETDATA_EXTENSIONS = 81, + CLI_NULL_COLLATION = 85, + CLI_ALTER_TABLE = 86, + CLI_ORDER_BY_COLUMNS_IN_SELECT = 90, + CLI_SPECIAL_CHARACTERS = 94, + CLI_MAX_COLUMNS_IN_GROUP_BY = 97, + CLI_MAX_COLUMNS_IN_INDEX = 98, + CLI_MAX_COLUMNS_IN_ORDER_BY = 99, + CLI_MAX_COLUMNS_IN_SELECT = 100, + CLI_MAX_COLUMNS_IN_TABLE = 101, + CLI_MAX_INDEX_SIZE = 102, + CLI_MAX_ROW_SIZE = 104, + CLI_MAX_STATEMENT_LEN = 105, + CLI_MAX_TABLES_IN_SELECT = 106, + CLI_MAX_USER_NAME_LEN = 107, + CLI_OJ_CAPABILITIES = 115, + + CLI_XOPEN_CLI_YEAR = 10000, + CLI_CURSOR_SENSITIVITY = 10001, + CLI_DESCRIBE_PARAMETER = 10002, + CLI_CATALOG_NAME = 10003, + CLI_COLLATION_SEQ = 10004, + CLI_MAX_IDENTIFIER_LEN = 10005, +} + +union TGetInfoValue { + 1: string stringValue + 2: i16 smallIntValue + 3: i32 integerBitmask + 4: i32 integerFlag + 5: i32 binaryValue + 6: i64 lenValue +} + +// GetInfo() +// +// This function is based on ODBC's CLIGetInfo() function. +// The function returns general information about the data source +// using the same keys as ODBC. +struct TGetInfoReq { + // The session to run this request against + 1: required TSessionHandle sessionHandle + + 2: required TGetInfoType infoType +} + +struct TGetInfoResp { + 1: required TStatus status + + 2: required TGetInfoValue infoValue +} + + +// ExecuteStatement() +// +// Execute a statement. +// The returned OperationHandle can be used to check on the +// status of the statement, and to fetch results once the +// statement has finished executing. +struct TExecuteStatementReq { + // The session to execute the statement against + 1: required TSessionHandle sessionHandle + + // The statement to be executed (DML, DDL, SET, etc) + 2: required string statement + + // Configuration properties that are overlayed on top of the + // the existing session configuration before this statement + // is executed. These properties apply to this statement + // only and will not affect the subsequent state of the Session. + 3: optional map confOverlay + + // Execute asynchronously when runAsync is true + 4: optional bool runAsync = false +} + +struct TExecuteStatementResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + +// GetTypeInfo() +// +// Get information about types supported by the HiveServer instance. +// The information is returned as a result set which can be fetched +// using the OperationHandle provided in the response. +// +// Refer to the documentation for ODBC's CLIGetTypeInfo function for +// the format of the result set. +struct TGetTypeInfoReq { + // The session to run this request against. + 1: required TSessionHandle sessionHandle +} + +struct TGetTypeInfoResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + + +// GetCatalogs() +// +// Returns the list of catalogs (databases) +// Results are ordered by TABLE_CATALOG +// +// Resultset columns : +// col1 +// name: TABLE_CAT +// type: STRING +// desc: Catalog name. NULL if not applicable. +// +struct TGetCatalogsReq { + // Session to run this request against + 1: required TSessionHandle sessionHandle +} + +struct TGetCatalogsResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + + +// GetSchemas() +// +// Retrieves the schema names available in this database. +// The results are ordered by TABLE_CATALOG and TABLE_SCHEM. +// col1 +// name: TABLE_SCHEM +// type: STRING +// desc: schema name +// col2 +// name: TABLE_CATALOG +// type: STRING +// desc: catalog name +struct TGetSchemasReq { + // Session to run this request against + 1: required TSessionHandle sessionHandle + + // Name of the catalog. Must not contain a search pattern. + 2: optional TIdentifier catalogName + + // schema name or pattern + 3: optional TPatternOrIdentifier schemaName +} + +struct TGetSchemasResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + + +// GetTables() +// +// Returns a list of tables with catalog, schema, and table +// type information. The information is returned as a result +// set which can be fetched using the OperationHandle +// provided in the response. +// Results are ordered by TABLE_TYPE, TABLE_CAT, TABLE_SCHEM, and TABLE_NAME +// +// Result Set Columns: +// +// col1 +// name: TABLE_CAT +// type: STRING +// desc: Catalog name. NULL if not applicable. +// +// col2 +// name: TABLE_SCHEM +// type: STRING +// desc: Schema name. +// +// col3 +// name: TABLE_NAME +// type: STRING +// desc: Table name. +// +// col4 +// name: TABLE_TYPE +// type: STRING +// desc: The table type, e.g. "TABLE", "VIEW", etc. +// +// col5 +// name: REMARKS +// type: STRING +// desc: Comments about the table +// +struct TGetTablesReq { + // Session to run this request against + 1: required TSessionHandle sessionHandle + + // Name of the catalog or a search pattern. + 2: optional TPatternOrIdentifier catalogName + + // Name of the schema or a search pattern. + 3: optional TPatternOrIdentifier schemaName + + // Name of the table or a search pattern. + 4: optional TPatternOrIdentifier tableName + + // List of table types to match + // e.g. "TABLE", "VIEW", "SYSTEM TABLE", "GLOBAL TEMPORARY", + // "LOCAL TEMPORARY", "ALIAS", "SYNONYM", etc. + 5: optional list tableTypes +} + +struct TGetTablesResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + + +// GetTableTypes() +// +// Returns the table types available in this database. +// The results are ordered by table type. +// +// col1 +// name: TABLE_TYPE +// type: STRING +// desc: Table type name. +struct TGetTableTypesReq { + // Session to run this request against + 1: required TSessionHandle sessionHandle +} + +struct TGetTableTypesResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + + +// GetColumns() +// +// Returns a list of columns in the specified tables. +// The information is returned as a result set which can be fetched +// using the OperationHandle provided in the response. +// Results are ordered by TABLE_CAT, TABLE_SCHEM, TABLE_NAME, +// and ORDINAL_POSITION. +// +// Result Set Columns are the same as those for the ODBC CLIColumns +// function. +// +struct TGetColumnsReq { + // Session to run this request against + 1: required TSessionHandle sessionHandle + + // Name of the catalog. Must not contain a search pattern. + 2: optional TIdentifier catalogName + + // Schema name or search pattern + 3: optional TPatternOrIdentifier schemaName + + // Table name or search pattern + 4: optional TPatternOrIdentifier tableName + + // Column name or search pattern + 5: optional TPatternOrIdentifier columnName +} + +struct TGetColumnsResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + + +// GetFunctions() +// +// Returns a list of functions supported by the data source. The +// behavior of this function matches +// java.sql.DatabaseMetaData.getFunctions() both in terms of +// inputs and outputs. +// +// Result Set Columns: +// +// col1 +// name: FUNCTION_CAT +// type: STRING +// desc: Function catalog (may be null) +// +// col2 +// name: FUNCTION_SCHEM +// type: STRING +// desc: Function schema (may be null) +// +// col3 +// name: FUNCTION_NAME +// type: STRING +// desc: Function name. This is the name used to invoke the function. +// +// col4 +// name: REMARKS +// type: STRING +// desc: Explanatory comment on the function. +// +// col5 +// name: FUNCTION_TYPE +// type: SMALLINT +// desc: Kind of function. One of: +// * functionResultUnknown - Cannot determine if a return value or a table +// will be returned. +// * functionNoTable - Does not a return a table. +// * functionReturnsTable - Returns a table. +// +// col6 +// name: SPECIFIC_NAME +// type: STRING +// desc: The name which uniquely identifies this function within its schema. +// In this case this is the fully qualified class name of the class +// that implements this function. +// +struct TGetFunctionsReq { + // Session to run this request against + 1: required TSessionHandle sessionHandle + + // A catalog name; must match the catalog name as it is stored in the + // database; "" retrieves those without a catalog; null means + // that the catalog name should not be used to narrow the search. + 2: optional TIdentifier catalogName + + // A schema name pattern; must match the schema name as it is stored + // in the database; "" retrieves those without a schema; null means + // that the schema name should not be used to narrow the search. + 3: optional TPatternOrIdentifier schemaName + + // A function name pattern; must match the function name as it is stored + // in the database. + 4: required TPatternOrIdentifier functionName +} + +struct TGetFunctionsResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + + +// GetOperationStatus() +// +// Get the status of an operation running on the server. +struct TGetOperationStatusReq { + // Session to run this request against + 1: required TOperationHandle operationHandle +} + +struct TGetOperationStatusResp { + 1: required TStatus status + 2: optional TOperationState operationState + + // If operationState is ERROR_STATE, then the following fields may be set + // sqlState as defined in the ISO/IEF CLI specification + 3: optional string sqlState + + // Internal error code + 4: optional i32 errorCode + + // Error message + 5: optional string errorMessage +} + + +// CancelOperation() +// +// Cancels processing on the specified operation handle and +// frees any resources which were allocated. +struct TCancelOperationReq { + // Operation to cancel + 1: required TOperationHandle operationHandle +} + +struct TCancelOperationResp { + 1: required TStatus status +} + + +// CloseOperation() +// +// Given an operation in the FINISHED, CANCELED, +// or ERROR states, CloseOperation() will free +// all of the resources which were allocated on +// the server to service the operation. +struct TCloseOperationReq { + 1: required TOperationHandle operationHandle +} + +struct TCloseOperationResp { + 1: required TStatus status +} + + +// GetResultSetMetadata() +// +// Retrieves schema information for the specified operation +struct TGetResultSetMetadataReq { + // Operation for which to fetch result set schema information + 1: required TOperationHandle operationHandle +} + +struct TGetResultSetMetadataResp { + 1: required TStatus status + 2: optional TTableSchema schema +} + + +enum TFetchOrientation { + // Get the next rowset. The fetch offset is ignored. + FETCH_NEXT, + + // Get the previous rowset. The fetch offset is ignored. + // NOT SUPPORTED + FETCH_PRIOR, + + // Return the rowset at the given fetch offset relative + // to the current rowset. + // NOT SUPPORTED + FETCH_RELATIVE, + + // Return the rowset at the specified fetch offset. + // NOT SUPPORTED + FETCH_ABSOLUTE, + + // Get the first rowset in the result set. + FETCH_FIRST, + + // Get the last rowset in the result set. + // NOT SUPPORTED + FETCH_LAST +} + +// FetchResults() +// +// Fetch rows from the server corresponding to +// a particular OperationHandle. +struct TFetchResultsReq { + // Operation from which to fetch results. + 1: required TOperationHandle operationHandle + + // The fetch orientation. For V1 this must be either + // FETCH_NEXT or FETCH_FIRST. Defaults to FETCH_NEXT. + 2: required TFetchOrientation orientation = TFetchOrientation.FETCH_NEXT + + // Max number of rows that should be returned in + // the rowset. + 3: required i64 maxRows + + // The type of a fetch results request. 0 represents Query output. 1 represents Log + 4: optional i16 fetchType = 0 +} + +struct TFetchResultsResp { + 1: required TStatus status + + // TRUE if there are more rows left to fetch from the server. + 2: optional bool hasMoreRows + + // The rowset. This is optional so that we have the + // option in the future of adding alternate formats for + // representing result set data, e.g. delimited strings, + // binary encoded, etc. + 3: optional TRowSet results +} + +// GetDelegationToken() +// Retrieve delegation token for the current user +struct TGetDelegationTokenReq { + // session handle + 1: required TSessionHandle sessionHandle + + // userid for the proxy user + 2: required string owner + + // designated renewer userid + 3: required string renewer +} + +struct TGetDelegationTokenResp { + // status of the request + 1: required TStatus status + + // delegation token string + 2: optional string delegationToken +} + +// CancelDelegationToken() +// Cancel the given delegation token +struct TCancelDelegationTokenReq { + // session handle + 1: required TSessionHandle sessionHandle + + // delegation token to cancel + 2: required string delegationToken +} + +struct TCancelDelegationTokenResp { + // status of the request + 1: required TStatus status +} + +// RenewDelegationToken() +// Renew the given delegation token +struct TRenewDelegationTokenReq { + // session handle + 1: required TSessionHandle sessionHandle + + // delegation token to renew + 2: required string delegationToken +} + +struct TRenewDelegationTokenResp { + // status of the request + 1: required TStatus status +} + +service TCLIService { + + TOpenSessionResp OpenSession(1:TOpenSessionReq req); + + TCloseSessionResp CloseSession(1:TCloseSessionReq req); + + TGetInfoResp GetInfo(1:TGetInfoReq req); + + TExecuteStatementResp ExecuteStatement(1:TExecuteStatementReq req); + + TGetTypeInfoResp GetTypeInfo(1:TGetTypeInfoReq req); + + TGetCatalogsResp GetCatalogs(1:TGetCatalogsReq req); + + TGetSchemasResp GetSchemas(1:TGetSchemasReq req); + + TGetTablesResp GetTables(1:TGetTablesReq req); + + TGetTableTypesResp GetTableTypes(1:TGetTableTypesReq req); + + TGetColumnsResp GetColumns(1:TGetColumnsReq req); + + TGetFunctionsResp GetFunctions(1:TGetFunctionsReq req); + + TGetOperationStatusResp GetOperationStatus(1:TGetOperationStatusReq req); + + TCancelOperationResp CancelOperation(1:TCancelOperationReq req); + + TCloseOperationResp CloseOperation(1:TCloseOperationReq req); + + TGetResultSetMetadataResp GetResultSetMetadata(1:TGetResultSetMetadataReq req); + + TFetchResultsResp FetchResults(1:TFetchResultsReq req); + + TGetDelegationTokenResp GetDelegationToken(1:TGetDelegationTokenReq req); + + TCancelDelegationTokenResp CancelDelegationToken(1:TCancelDelegationTokenReq req); + + TRenewDelegationTokenResp RenewDelegationToken(1:TRenewDelegationTokenReq req); +} diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index b5b2143292a6..3135a8a275da 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,13 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT + spark-parent_2.11 + 2.3.0-SNAPSHOT ../../pom.xml - org.apache.spark - spark-hive-thriftserver_2.10 + spark-hive-thriftserver_2.11 jar Spark Project Hive Thrift Server http://spark.apache.org/ @@ -60,25 +59,19 @@ ${hive.group} hive-jdbc - - ${hive.group} - hive-service - ${hive.group} hive-beeline - com.sun.jersey - jersey-core + org.eclipse.jetty + jetty-server + provided - com.sun.jersey - jersey-json - - - com.sun.jersey - jersey-server + org.eclipse.jetty + jetty-servlet + provided @@ -86,6 +79,11 @@ selenium-java test + + org.seleniumhq.selenium + selenium-htmlunit-driver + test + org.apache.spark spark-sql_${scala.binary.version} @@ -95,7 +93,23 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + net.sf.jpam + jpam @@ -118,6 +132,18 @@ + + add-source + generate-sources + + add-source + + + + src/gen/ + + + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java new file mode 100644 index 000000000000..6323d34eac73 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java @@ -0,0 +1,383 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TArrayTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TArrayTypeEntry"); + + private static final org.apache.thrift.protocol.TField OBJECT_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("objectTypePtr", org.apache.thrift.protocol.TType.I32, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TArrayTypeEntryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TArrayTypeEntryTupleSchemeFactory()); + } + + private int objectTypePtr; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OBJECT_TYPE_PTR((short)1, "objectTypePtr"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OBJECT_TYPE_PTR + return OBJECT_TYPE_PTR; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __OBJECTTYPEPTR_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OBJECT_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("objectTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr"))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TArrayTypeEntry.class, metaDataMap); + } + + public TArrayTypeEntry() { + } + + public TArrayTypeEntry( + int objectTypePtr) + { + this(); + this.objectTypePtr = objectTypePtr; + setObjectTypePtrIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TArrayTypeEntry(TArrayTypeEntry other) { + __isset_bitfield = other.__isset_bitfield; + this.objectTypePtr = other.objectTypePtr; + } + + public TArrayTypeEntry deepCopy() { + return new TArrayTypeEntry(this); + } + + @Override + public void clear() { + setObjectTypePtrIsSet(false); + this.objectTypePtr = 0; + } + + public int getObjectTypePtr() { + return this.objectTypePtr; + } + + public void setObjectTypePtr(int objectTypePtr) { + this.objectTypePtr = objectTypePtr; + setObjectTypePtrIsSet(true); + } + + public void unsetObjectTypePtr() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID); + } + + /** Returns true if field objectTypePtr is set (has been assigned a value) and false otherwise */ + public boolean isSetObjectTypePtr() { + return EncodingUtils.testBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID); + } + + public void setObjectTypePtrIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OBJECT_TYPE_PTR: + if (value == null) { + unsetObjectTypePtr(); + } else { + setObjectTypePtr((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OBJECT_TYPE_PTR: + return Integer.valueOf(getObjectTypePtr()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OBJECT_TYPE_PTR: + return isSetObjectTypePtr(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TArrayTypeEntry) + return this.equals((TArrayTypeEntry)that); + return false; + } + + public boolean equals(TArrayTypeEntry that) { + if (that == null) + return false; + + boolean this_present_objectTypePtr = true; + boolean that_present_objectTypePtr = true; + if (this_present_objectTypePtr || that_present_objectTypePtr) { + if (!(this_present_objectTypePtr && that_present_objectTypePtr)) + return false; + if (this.objectTypePtr != that.objectTypePtr) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_objectTypePtr = true; + builder.append(present_objectTypePtr); + if (present_objectTypePtr) + builder.append(objectTypePtr); + + return builder.toHashCode(); + } + + public int compareTo(TArrayTypeEntry other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TArrayTypeEntry typedOther = (TArrayTypeEntry)other; + + lastComparison = Boolean.valueOf(isSetObjectTypePtr()).compareTo(typedOther.isSetObjectTypePtr()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetObjectTypePtr()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.objectTypePtr, typedOther.objectTypePtr); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TArrayTypeEntry("); + boolean first = true; + + sb.append("objectTypePtr:"); + sb.append(this.objectTypePtr); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetObjectTypePtr()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'objectTypePtr' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TArrayTypeEntryStandardSchemeFactory implements SchemeFactory { + public TArrayTypeEntryStandardScheme getScheme() { + return new TArrayTypeEntryStandardScheme(); + } + } + + private static class TArrayTypeEntryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TArrayTypeEntry struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OBJECT_TYPE_PTR + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.objectTypePtr = iprot.readI32(); + struct.setObjectTypePtrIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TArrayTypeEntry struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(OBJECT_TYPE_PTR_FIELD_DESC); + oprot.writeI32(struct.objectTypePtr); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TArrayTypeEntryTupleSchemeFactory implements SchemeFactory { + public TArrayTypeEntryTupleScheme getScheme() { + return new TArrayTypeEntryTupleScheme(); + } + } + + private static class TArrayTypeEntryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TArrayTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.objectTypePtr); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TArrayTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.objectTypePtr = iprot.readI32(); + struct.setObjectTypePtrIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java new file mode 100644 index 000000000000..6b1b054d1aca --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java @@ -0,0 +1,550 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TBinaryColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBinaryColumn"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TBinaryColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TBinaryColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBinaryColumn.class, metaDataMap); + } + + public TBinaryColumn() { + } + + public TBinaryColumn( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = nulls; + } + + /** + * Performs a deep copy on other. + */ + public TBinaryColumn(TBinaryColumn other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(); + for (ByteBuffer other_element : other.values) { + ByteBuffer temp_binary_element = org.apache.thrift.TBaseHelper.copyBinary(other_element); +; + __this__values.add(temp_binary_element); + } + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); +; + } + } + + public TBinaryColumn deepCopy() { + return new TBinaryColumn(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(ByteBuffer elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return nulls; + } + + public void setNulls(byte[] nulls) { + setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = nulls; + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TBinaryColumn) + return this.equals((TBinaryColumn)that); + return false; + } + + public boolean equals(TBinaryColumn that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_values = true && (isSetValues()); + builder.append(present_values); + if (present_values) + builder.append(values); + + boolean present_nulls = true && (isSetNulls()); + builder.append(present_nulls); + if (present_nulls) + builder.append(nulls); + + return builder.toHashCode(); + } + + public int compareTo(TBinaryColumn other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TBinaryColumn typedOther = (TBinaryColumn)other; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TBinaryColumn("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TBinaryColumnStandardSchemeFactory implements SchemeFactory { + public TBinaryColumnStandardScheme getScheme() { + return new TBinaryColumnStandardScheme(); + } + } + + private static class TBinaryColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TBinaryColumn struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list110 = iprot.readListBegin(); + struct.values = new ArrayList(_list110.size); + for (int _i111 = 0; _i111 < _list110.size; ++_i111) + { + ByteBuffer _elem112; // optional + _elem112 = iprot.readBinary(); + struct.values.add(_elem112); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TBinaryColumn struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size())); + for (ByteBuffer _iter113 : struct.values) + { + oprot.writeBinary(_iter113); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TBinaryColumnTupleSchemeFactory implements SchemeFactory { + public TBinaryColumnTupleScheme getScheme() { + return new TBinaryColumnTupleScheme(); + } + } + + private static class TBinaryColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TBinaryColumn struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (ByteBuffer _iter114 : struct.values) + { + oprot.writeBinary(_iter114); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TBinaryColumn struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list115 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.values = new ArrayList(_list115.size); + for (int _i116 = 0; _i116 < _list115.size; ++_i116) + { + ByteBuffer _elem117; // optional + _elem117 = iprot.readBinary(); + struct.values.add(_elem117); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java new file mode 100644 index 000000000000..efd571cfdfbb --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TBoolColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBoolColumn"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TBoolColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TBoolColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBoolColumn.class, metaDataMap); + } + + public TBoolColumn() { + } + + public TBoolColumn( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = nulls; + } + + /** + * Performs a deep copy on other. + */ + public TBoolColumn(TBoolColumn other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(); + for (Boolean other_element : other.values) { + __this__values.add(other_element); + } + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); +; + } + } + + public TBoolColumn deepCopy() { + return new TBoolColumn(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(boolean elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return nulls; + } + + public void setNulls(byte[] nulls) { + setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = nulls; + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TBoolColumn) + return this.equals((TBoolColumn)that); + return false; + } + + public boolean equals(TBoolColumn that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_values = true && (isSetValues()); + builder.append(present_values); + if (present_values) + builder.append(values); + + boolean present_nulls = true && (isSetNulls()); + builder.append(present_nulls); + if (present_nulls) + builder.append(nulls); + + return builder.toHashCode(); + } + + public int compareTo(TBoolColumn other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TBoolColumn typedOther = (TBoolColumn)other; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TBoolColumn("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TBoolColumnStandardSchemeFactory implements SchemeFactory { + public TBoolColumnStandardScheme getScheme() { + return new TBoolColumnStandardScheme(); + } + } + + private static class TBoolColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TBoolColumn struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list54 = iprot.readListBegin(); + struct.values = new ArrayList(_list54.size); + for (int _i55 = 0; _i55 < _list54.size; ++_i55) + { + boolean _elem56; // optional + _elem56 = iprot.readBool(); + struct.values.add(_elem56); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TBoolColumn struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BOOL, struct.values.size())); + for (boolean _iter57 : struct.values) + { + oprot.writeBool(_iter57); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TBoolColumnTupleSchemeFactory implements SchemeFactory { + public TBoolColumnTupleScheme getScheme() { + return new TBoolColumnTupleScheme(); + } + } + + private static class TBoolColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TBoolColumn struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (boolean _iter58 : struct.values) + { + oprot.writeBool(_iter58); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TBoolColumn struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list59 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BOOL, iprot.readI32()); + struct.values = new ArrayList(_list59.size); + for (int _i60 = 0; _i60 < _list59.size; ++_i60) + { + boolean _elem61; // optional + _elem61 = iprot.readBool(); + struct.values.add(_elem61); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java new file mode 100644 index 000000000000..c7495ee79e4b --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java @@ -0,0 +1,386 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TBoolValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBoolValue"); + + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.BOOL, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TBoolValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TBoolValueTupleSchemeFactory()); + } + + private boolean value; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUE((short)1, "value"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUE + return VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __VALUE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.VALUE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBoolValue.class, metaDataMap); + } + + public TBoolValue() { + } + + /** + * Performs a deep copy on other. + */ + public TBoolValue(TBoolValue other) { + __isset_bitfield = other.__isset_bitfield; + this.value = other.value; + } + + public TBoolValue deepCopy() { + return new TBoolValue(this); + } + + @Override + public void clear() { + setValueIsSet(false); + this.value = false; + } + + public boolean isValue() { + return this.value; + } + + public void setValue(boolean value) { + this.value = value; + setValueIsSet(true); + } + + public void unsetValue() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + public void setValueIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((Boolean)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUE: + return Boolean.valueOf(isValue()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUE: + return isSetValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TBoolValue) + return this.equals((TBoolValue)that); + return false; + } + + public boolean equals(TBoolValue that) { + if (that == null) + return false; + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (this.value != that.value) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_value = true && (isSetValue()); + builder.append(present_value); + if (present_value) + builder.append(value); + + return builder.toHashCode(); + } + + public int compareTo(TBoolValue other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TBoolValue typedOther = (TBoolValue)other; + + lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TBoolValue("); + boolean first = true; + + if (isSetValue()) { + sb.append("value:"); + sb.append(this.value); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TBoolValueStandardSchemeFactory implements SchemeFactory { + public TBoolValueStandardScheme getScheme() { + return new TBoolValueStandardScheme(); + } + } + + private static class TBoolValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TBoolValue struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.value = iprot.readBool(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TBoolValue struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetValue()) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeBool(struct.value); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TBoolValueTupleSchemeFactory implements SchemeFactory { + public TBoolValueTupleScheme getScheme() { + return new TBoolValueTupleScheme(); + } + } + + private static class TBoolValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TBoolValue struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetValue()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetValue()) { + oprot.writeBool(struct.value); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TBoolValue struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.value = iprot.readBool(); + struct.setValueIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java new file mode 100644 index 000000000000..169bfdeab3ee --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TByteColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TByteColumn"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TByteColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TByteColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TByteColumn.class, metaDataMap); + } + + public TByteColumn() { + } + + public TByteColumn( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = nulls; + } + + /** + * Performs a deep copy on other. + */ + public TByteColumn(TByteColumn other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(); + for (Byte other_element : other.values) { + __this__values.add(other_element); + } + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); +; + } + } + + public TByteColumn deepCopy() { + return new TByteColumn(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(byte elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return nulls; + } + + public void setNulls(byte[] nulls) { + setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = nulls; + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TByteColumn) + return this.equals((TByteColumn)that); + return false; + } + + public boolean equals(TByteColumn that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_values = true && (isSetValues()); + builder.append(present_values); + if (present_values) + builder.append(values); + + boolean present_nulls = true && (isSetNulls()); + builder.append(present_nulls); + if (present_nulls) + builder.append(nulls); + + return builder.toHashCode(); + } + + public int compareTo(TByteColumn other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TByteColumn typedOther = (TByteColumn)other; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TByteColumn("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TByteColumnStandardSchemeFactory implements SchemeFactory { + public TByteColumnStandardScheme getScheme() { + return new TByteColumnStandardScheme(); + } + } + + private static class TByteColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TByteColumn struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list62 = iprot.readListBegin(); + struct.values = new ArrayList(_list62.size); + for (int _i63 = 0; _i63 < _list62.size; ++_i63) + { + byte _elem64; // optional + _elem64 = iprot.readByte(); + struct.values.add(_elem64); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TByteColumn struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BYTE, struct.values.size())); + for (byte _iter65 : struct.values) + { + oprot.writeByte(_iter65); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TByteColumnTupleSchemeFactory implements SchemeFactory { + public TByteColumnTupleScheme getScheme() { + return new TByteColumnTupleScheme(); + } + } + + private static class TByteColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TByteColumn struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (byte _iter66 : struct.values) + { + oprot.writeByte(_iter66); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TByteColumn struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list67 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BYTE, iprot.readI32()); + struct.values = new ArrayList(_list67.size); + for (int _i68 = 0; _i68 < _list67.size; ++_i68) + { + byte _elem69; // optional + _elem69 = iprot.readByte(); + struct.values.add(_elem69); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java new file mode 100644 index 000000000000..23d969375996 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java @@ -0,0 +1,386 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TByteValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TByteValue"); + + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.BYTE, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TByteValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TByteValueTupleSchemeFactory()); + } + + private byte value; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUE((short)1, "value"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUE + return VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __VALUE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.VALUE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TByteValue.class, metaDataMap); + } + + public TByteValue() { + } + + /** + * Performs a deep copy on other. + */ + public TByteValue(TByteValue other) { + __isset_bitfield = other.__isset_bitfield; + this.value = other.value; + } + + public TByteValue deepCopy() { + return new TByteValue(this); + } + + @Override + public void clear() { + setValueIsSet(false); + this.value = 0; + } + + public byte getValue() { + return this.value; + } + + public void setValue(byte value) { + this.value = value; + setValueIsSet(true); + } + + public void unsetValue() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + public void setValueIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((Byte)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUE: + return Byte.valueOf(getValue()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUE: + return isSetValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TByteValue) + return this.equals((TByteValue)that); + return false; + } + + public boolean equals(TByteValue that) { + if (that == null) + return false; + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (this.value != that.value) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_value = true && (isSetValue()); + builder.append(present_value); + if (present_value) + builder.append(value); + + return builder.toHashCode(); + } + + public int compareTo(TByteValue other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TByteValue typedOther = (TByteValue)other; + + lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TByteValue("); + boolean first = true; + + if (isSetValue()) { + sb.append("value:"); + sb.append(this.value); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TByteValueStandardSchemeFactory implements SchemeFactory { + public TByteValueStandardScheme getScheme() { + return new TByteValueStandardScheme(); + } + } + + private static class TByteValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TByteValue struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.BYTE) { + struct.value = iprot.readByte(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TByteValue struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetValue()) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeByte(struct.value); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TByteValueTupleSchemeFactory implements SchemeFactory { + public TByteValueTupleScheme getScheme() { + return new TByteValueTupleScheme(); + } + } + + private static class TByteValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TByteValue struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetValue()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetValue()) { + oprot.writeByte(struct.value); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TByteValue struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.value = iprot.readByte(); + struct.setValueIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java new file mode 100644 index 000000000000..54851b8d5131 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java @@ -0,0 +1,15414 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TCLIService { + + public interface Iface { + + public TOpenSessionResp OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException; + + public TCloseSessionResp CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException; + + public TGetInfoResp GetInfo(TGetInfoReq req) throws org.apache.thrift.TException; + + public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException; + + public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException; + + public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException; + + public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException; + + public TGetTablesResp GetTables(TGetTablesReq req) throws org.apache.thrift.TException; + + public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException; + + public TGetColumnsResp GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException; + + public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException; + + public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException; + + public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException; + + public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException; + + public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException; + + public TFetchResultsResp FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException; + + public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException; + + public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException; + + public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException; + + } + + public interface AsyncIface { + + public void OpenSession(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void CloseSession(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetInfo(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void ExecuteStatement(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetTypeInfo(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetCatalogs(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetSchemas(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetTables(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetTableTypes(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetColumns(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetFunctions(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetOperationStatus(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void CancelOperation(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void CloseOperation(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetResultSetMetadata(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void FetchResults(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetDelegationToken(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void CancelDelegationToken(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void RenewDelegationToken(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + } + + public static class Client extends org.apache.thrift.TServiceClient implements Iface { + public static class Factory implements org.apache.thrift.TServiceClientFactory { + public Factory() {} + public Client getClient(org.apache.thrift.protocol.TProtocol prot) { + return new Client(prot); + } + public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + return new Client(iprot, oprot); + } + } + + public Client(org.apache.thrift.protocol.TProtocol prot) + { + super(prot, prot); + } + + public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + super(iprot, oprot); + } + + public TOpenSessionResp OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException + { + send_OpenSession(req); + return recv_OpenSession(); + } + + public void send_OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException + { + OpenSession_args args = new OpenSession_args(); + args.setReq(req); + sendBase("OpenSession", args); + } + + public TOpenSessionResp recv_OpenSession() throws org.apache.thrift.TException + { + OpenSession_result result = new OpenSession_result(); + receiveBase(result, "OpenSession"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "OpenSession failed: unknown result"); + } + + public TCloseSessionResp CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException + { + send_CloseSession(req); + return recv_CloseSession(); + } + + public void send_CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException + { + CloseSession_args args = new CloseSession_args(); + args.setReq(req); + sendBase("CloseSession", args); + } + + public TCloseSessionResp recv_CloseSession() throws org.apache.thrift.TException + { + CloseSession_result result = new CloseSession_result(); + receiveBase(result, "CloseSession"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CloseSession failed: unknown result"); + } + + public TGetInfoResp GetInfo(TGetInfoReq req) throws org.apache.thrift.TException + { + send_GetInfo(req); + return recv_GetInfo(); + } + + public void send_GetInfo(TGetInfoReq req) throws org.apache.thrift.TException + { + GetInfo_args args = new GetInfo_args(); + args.setReq(req); + sendBase("GetInfo", args); + } + + public TGetInfoResp recv_GetInfo() throws org.apache.thrift.TException + { + GetInfo_result result = new GetInfo_result(); + receiveBase(result, "GetInfo"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetInfo failed: unknown result"); + } + + public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException + { + send_ExecuteStatement(req); + return recv_ExecuteStatement(); + } + + public void send_ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException + { + ExecuteStatement_args args = new ExecuteStatement_args(); + args.setReq(req); + sendBase("ExecuteStatement", args); + } + + public TExecuteStatementResp recv_ExecuteStatement() throws org.apache.thrift.TException + { + ExecuteStatement_result result = new ExecuteStatement_result(); + receiveBase(result, "ExecuteStatement"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "ExecuteStatement failed: unknown result"); + } + + public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException + { + send_GetTypeInfo(req); + return recv_GetTypeInfo(); + } + + public void send_GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException + { + GetTypeInfo_args args = new GetTypeInfo_args(); + args.setReq(req); + sendBase("GetTypeInfo", args); + } + + public TGetTypeInfoResp recv_GetTypeInfo() throws org.apache.thrift.TException + { + GetTypeInfo_result result = new GetTypeInfo_result(); + receiveBase(result, "GetTypeInfo"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTypeInfo failed: unknown result"); + } + + public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException + { + send_GetCatalogs(req); + return recv_GetCatalogs(); + } + + public void send_GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException + { + GetCatalogs_args args = new GetCatalogs_args(); + args.setReq(req); + sendBase("GetCatalogs", args); + } + + public TGetCatalogsResp recv_GetCatalogs() throws org.apache.thrift.TException + { + GetCatalogs_result result = new GetCatalogs_result(); + receiveBase(result, "GetCatalogs"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetCatalogs failed: unknown result"); + } + + public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException + { + send_GetSchemas(req); + return recv_GetSchemas(); + } + + public void send_GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException + { + GetSchemas_args args = new GetSchemas_args(); + args.setReq(req); + sendBase("GetSchemas", args); + } + + public TGetSchemasResp recv_GetSchemas() throws org.apache.thrift.TException + { + GetSchemas_result result = new GetSchemas_result(); + receiveBase(result, "GetSchemas"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetSchemas failed: unknown result"); + } + + public TGetTablesResp GetTables(TGetTablesReq req) throws org.apache.thrift.TException + { + send_GetTables(req); + return recv_GetTables(); + } + + public void send_GetTables(TGetTablesReq req) throws org.apache.thrift.TException + { + GetTables_args args = new GetTables_args(); + args.setReq(req); + sendBase("GetTables", args); + } + + public TGetTablesResp recv_GetTables() throws org.apache.thrift.TException + { + GetTables_result result = new GetTables_result(); + receiveBase(result, "GetTables"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTables failed: unknown result"); + } + + public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException + { + send_GetTableTypes(req); + return recv_GetTableTypes(); + } + + public void send_GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException + { + GetTableTypes_args args = new GetTableTypes_args(); + args.setReq(req); + sendBase("GetTableTypes", args); + } + + public TGetTableTypesResp recv_GetTableTypes() throws org.apache.thrift.TException + { + GetTableTypes_result result = new GetTableTypes_result(); + receiveBase(result, "GetTableTypes"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTableTypes failed: unknown result"); + } + + public TGetColumnsResp GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException + { + send_GetColumns(req); + return recv_GetColumns(); + } + + public void send_GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException + { + GetColumns_args args = new GetColumns_args(); + args.setReq(req); + sendBase("GetColumns", args); + } + + public TGetColumnsResp recv_GetColumns() throws org.apache.thrift.TException + { + GetColumns_result result = new GetColumns_result(); + receiveBase(result, "GetColumns"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetColumns failed: unknown result"); + } + + public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException + { + send_GetFunctions(req); + return recv_GetFunctions(); + } + + public void send_GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException + { + GetFunctions_args args = new GetFunctions_args(); + args.setReq(req); + sendBase("GetFunctions", args); + } + + public TGetFunctionsResp recv_GetFunctions() throws org.apache.thrift.TException + { + GetFunctions_result result = new GetFunctions_result(); + receiveBase(result, "GetFunctions"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetFunctions failed: unknown result"); + } + + public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException + { + send_GetOperationStatus(req); + return recv_GetOperationStatus(); + } + + public void send_GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException + { + GetOperationStatus_args args = new GetOperationStatus_args(); + args.setReq(req); + sendBase("GetOperationStatus", args); + } + + public TGetOperationStatusResp recv_GetOperationStatus() throws org.apache.thrift.TException + { + GetOperationStatus_result result = new GetOperationStatus_result(); + receiveBase(result, "GetOperationStatus"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetOperationStatus failed: unknown result"); + } + + public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException + { + send_CancelOperation(req); + return recv_CancelOperation(); + } + + public void send_CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException + { + CancelOperation_args args = new CancelOperation_args(); + args.setReq(req); + sendBase("CancelOperation", args); + } + + public TCancelOperationResp recv_CancelOperation() throws org.apache.thrift.TException + { + CancelOperation_result result = new CancelOperation_result(); + receiveBase(result, "CancelOperation"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CancelOperation failed: unknown result"); + } + + public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException + { + send_CloseOperation(req); + return recv_CloseOperation(); + } + + public void send_CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException + { + CloseOperation_args args = new CloseOperation_args(); + args.setReq(req); + sendBase("CloseOperation", args); + } + + public TCloseOperationResp recv_CloseOperation() throws org.apache.thrift.TException + { + CloseOperation_result result = new CloseOperation_result(); + receiveBase(result, "CloseOperation"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CloseOperation failed: unknown result"); + } + + public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException + { + send_GetResultSetMetadata(req); + return recv_GetResultSetMetadata(); + } + + public void send_GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException + { + GetResultSetMetadata_args args = new GetResultSetMetadata_args(); + args.setReq(req); + sendBase("GetResultSetMetadata", args); + } + + public TGetResultSetMetadataResp recv_GetResultSetMetadata() throws org.apache.thrift.TException + { + GetResultSetMetadata_result result = new GetResultSetMetadata_result(); + receiveBase(result, "GetResultSetMetadata"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetResultSetMetadata failed: unknown result"); + } + + public TFetchResultsResp FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException + { + send_FetchResults(req); + return recv_FetchResults(); + } + + public void send_FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException + { + FetchResults_args args = new FetchResults_args(); + args.setReq(req); + sendBase("FetchResults", args); + } + + public TFetchResultsResp recv_FetchResults() throws org.apache.thrift.TException + { + FetchResults_result result = new FetchResults_result(); + receiveBase(result, "FetchResults"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "FetchResults failed: unknown result"); + } + + public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException + { + send_GetDelegationToken(req); + return recv_GetDelegationToken(); + } + + public void send_GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException + { + GetDelegationToken_args args = new GetDelegationToken_args(); + args.setReq(req); + sendBase("GetDelegationToken", args); + } + + public TGetDelegationTokenResp recv_GetDelegationToken() throws org.apache.thrift.TException + { + GetDelegationToken_result result = new GetDelegationToken_result(); + receiveBase(result, "GetDelegationToken"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetDelegationToken failed: unknown result"); + } + + public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException + { + send_CancelDelegationToken(req); + return recv_CancelDelegationToken(); + } + + public void send_CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException + { + CancelDelegationToken_args args = new CancelDelegationToken_args(); + args.setReq(req); + sendBase("CancelDelegationToken", args); + } + + public TCancelDelegationTokenResp recv_CancelDelegationToken() throws org.apache.thrift.TException + { + CancelDelegationToken_result result = new CancelDelegationToken_result(); + receiveBase(result, "CancelDelegationToken"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CancelDelegationToken failed: unknown result"); + } + + public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException + { + send_RenewDelegationToken(req); + return recv_RenewDelegationToken(); + } + + public void send_RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException + { + RenewDelegationToken_args args = new RenewDelegationToken_args(); + args.setReq(req); + sendBase("RenewDelegationToken", args); + } + + public TRenewDelegationTokenResp recv_RenewDelegationToken() throws org.apache.thrift.TException + { + RenewDelegationToken_result result = new RenewDelegationToken_result(); + receiveBase(result, "RenewDelegationToken"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "RenewDelegationToken failed: unknown result"); + } + + } + public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { + public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { + private org.apache.thrift.async.TAsyncClientManager clientManager; + private org.apache.thrift.protocol.TProtocolFactory protocolFactory; + public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) { + this.clientManager = clientManager; + this.protocolFactory = protocolFactory; + } + public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) { + return new AsyncClient(protocolFactory, clientManager, transport); + } + } + + public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) { + super(protocolFactory, clientManager, transport); + } + + public void OpenSession(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + OpenSession_call method_call = new OpenSession_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class OpenSession_call extends org.apache.thrift.async.TAsyncMethodCall { + private TOpenSessionReq req; + public OpenSession_call(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("OpenSession", org.apache.thrift.protocol.TMessageType.CALL, 0)); + OpenSession_args args = new OpenSession_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TOpenSessionResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_OpenSession(); + } + } + + public void CloseSession(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + CloseSession_call method_call = new CloseSession_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class CloseSession_call extends org.apache.thrift.async.TAsyncMethodCall { + private TCloseSessionReq req; + public CloseSession_call(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CloseSession", org.apache.thrift.protocol.TMessageType.CALL, 0)); + CloseSession_args args = new CloseSession_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TCloseSessionResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_CloseSession(); + } + } + + public void GetInfo(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetInfo_call method_call = new GetInfo_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetInfo_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetInfoReq req; + public GetInfo_call(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetInfo", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetInfo_args args = new GetInfo_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetInfoResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetInfo(); + } + } + + public void ExecuteStatement(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + ExecuteStatement_call method_call = new ExecuteStatement_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class ExecuteStatement_call extends org.apache.thrift.async.TAsyncMethodCall { + private TExecuteStatementReq req; + public ExecuteStatement_call(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("ExecuteStatement", org.apache.thrift.protocol.TMessageType.CALL, 0)); + ExecuteStatement_args args = new ExecuteStatement_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TExecuteStatementResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_ExecuteStatement(); + } + } + + public void GetTypeInfo(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetTypeInfo_call method_call = new GetTypeInfo_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetTypeInfo_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetTypeInfoReq req; + public GetTypeInfo_call(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTypeInfo", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetTypeInfo_args args = new GetTypeInfo_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetTypeInfoResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetTypeInfo(); + } + } + + public void GetCatalogs(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetCatalogs_call method_call = new GetCatalogs_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetCatalogs_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetCatalogsReq req; + public GetCatalogs_call(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetCatalogs", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetCatalogs_args args = new GetCatalogs_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetCatalogsResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetCatalogs(); + } + } + + public void GetSchemas(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetSchemas_call method_call = new GetSchemas_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetSchemas_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetSchemasReq req; + public GetSchemas_call(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetSchemas", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetSchemas_args args = new GetSchemas_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetSchemasResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetSchemas(); + } + } + + public void GetTables(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetTables_call method_call = new GetTables_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetTables_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetTablesReq req; + public GetTables_call(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTables", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetTables_args args = new GetTables_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetTablesResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetTables(); + } + } + + public void GetTableTypes(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetTableTypes_call method_call = new GetTableTypes_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetTableTypes_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetTableTypesReq req; + public GetTableTypes_call(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTableTypes", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetTableTypes_args args = new GetTableTypes_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetTableTypesResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetTableTypes(); + } + } + + public void GetColumns(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetColumns_call method_call = new GetColumns_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetColumns_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetColumnsReq req; + public GetColumns_call(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetColumns", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetColumns_args args = new GetColumns_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetColumnsResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetColumns(); + } + } + + public void GetFunctions(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetFunctions_call method_call = new GetFunctions_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetFunctions_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetFunctionsReq req; + public GetFunctions_call(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetFunctions", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetFunctions_args args = new GetFunctions_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetFunctionsResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetFunctions(); + } + } + + public void GetOperationStatus(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetOperationStatus_call method_call = new GetOperationStatus_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetOperationStatus_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetOperationStatusReq req; + public GetOperationStatus_call(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetOperationStatus", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetOperationStatus_args args = new GetOperationStatus_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetOperationStatusResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetOperationStatus(); + } + } + + public void CancelOperation(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + CancelOperation_call method_call = new CancelOperation_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class CancelOperation_call extends org.apache.thrift.async.TAsyncMethodCall { + private TCancelOperationReq req; + public CancelOperation_call(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CancelOperation", org.apache.thrift.protocol.TMessageType.CALL, 0)); + CancelOperation_args args = new CancelOperation_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TCancelOperationResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_CancelOperation(); + } + } + + public void CloseOperation(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + CloseOperation_call method_call = new CloseOperation_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class CloseOperation_call extends org.apache.thrift.async.TAsyncMethodCall { + private TCloseOperationReq req; + public CloseOperation_call(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CloseOperation", org.apache.thrift.protocol.TMessageType.CALL, 0)); + CloseOperation_args args = new CloseOperation_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TCloseOperationResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_CloseOperation(); + } + } + + public void GetResultSetMetadata(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetResultSetMetadata_call method_call = new GetResultSetMetadata_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetResultSetMetadata_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetResultSetMetadataReq req; + public GetResultSetMetadata_call(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetResultSetMetadata", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetResultSetMetadata_args args = new GetResultSetMetadata_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetResultSetMetadataResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetResultSetMetadata(); + } + } + + public void FetchResults(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + FetchResults_call method_call = new FetchResults_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class FetchResults_call extends org.apache.thrift.async.TAsyncMethodCall { + private TFetchResultsReq req; + public FetchResults_call(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("FetchResults", org.apache.thrift.protocol.TMessageType.CALL, 0)); + FetchResults_args args = new FetchResults_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TFetchResultsResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_FetchResults(); + } + } + + public void GetDelegationToken(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetDelegationToken_call method_call = new GetDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetDelegationTokenReq req; + public GetDelegationToken_call(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetDelegationToken_args args = new GetDelegationToken_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetDelegationTokenResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetDelegationToken(); + } + } + + public void CancelDelegationToken(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + CancelDelegationToken_call method_call = new CancelDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class CancelDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall { + private TCancelDelegationTokenReq req; + public CancelDelegationToken_call(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CancelDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0)); + CancelDelegationToken_args args = new CancelDelegationToken_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TCancelDelegationTokenResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_CancelDelegationToken(); + } + } + + public void RenewDelegationToken(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + RenewDelegationToken_call method_call = new RenewDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class RenewDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall { + private TRenewDelegationTokenReq req; + public RenewDelegationToken_call(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("RenewDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0)); + RenewDelegationToken_args args = new RenewDelegationToken_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TRenewDelegationTokenResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_RenewDelegationToken(); + } + } + + } + + public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); + public Processor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected Processor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("OpenSession", new OpenSession()); + processMap.put("CloseSession", new CloseSession()); + processMap.put("GetInfo", new GetInfo()); + processMap.put("ExecuteStatement", new ExecuteStatement()); + processMap.put("GetTypeInfo", new GetTypeInfo()); + processMap.put("GetCatalogs", new GetCatalogs()); + processMap.put("GetSchemas", new GetSchemas()); + processMap.put("GetTables", new GetTables()); + processMap.put("GetTableTypes", new GetTableTypes()); + processMap.put("GetColumns", new GetColumns()); + processMap.put("GetFunctions", new GetFunctions()); + processMap.put("GetOperationStatus", new GetOperationStatus()); + processMap.put("CancelOperation", new CancelOperation()); + processMap.put("CloseOperation", new CloseOperation()); + processMap.put("GetResultSetMetadata", new GetResultSetMetadata()); + processMap.put("FetchResults", new FetchResults()); + processMap.put("GetDelegationToken", new GetDelegationToken()); + processMap.put("CancelDelegationToken", new CancelDelegationToken()); + processMap.put("RenewDelegationToken", new RenewDelegationToken()); + return processMap; + } + + public static class OpenSession extends org.apache.thrift.ProcessFunction { + public OpenSession() { + super("OpenSession"); + } + + public OpenSession_args getEmptyArgsInstance() { + return new OpenSession_args(); + } + + protected boolean isOneway() { + return false; + } + + public OpenSession_result getResult(I iface, OpenSession_args args) throws org.apache.thrift.TException { + OpenSession_result result = new OpenSession_result(); + result.success = iface.OpenSession(args.req); + return result; + } + } + + public static class CloseSession extends org.apache.thrift.ProcessFunction { + public CloseSession() { + super("CloseSession"); + } + + public CloseSession_args getEmptyArgsInstance() { + return new CloseSession_args(); + } + + protected boolean isOneway() { + return false; + } + + public CloseSession_result getResult(I iface, CloseSession_args args) throws org.apache.thrift.TException { + CloseSession_result result = new CloseSession_result(); + result.success = iface.CloseSession(args.req); + return result; + } + } + + public static class GetInfo extends org.apache.thrift.ProcessFunction { + public GetInfo() { + super("GetInfo"); + } + + public GetInfo_args getEmptyArgsInstance() { + return new GetInfo_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetInfo_result getResult(I iface, GetInfo_args args) throws org.apache.thrift.TException { + GetInfo_result result = new GetInfo_result(); + result.success = iface.GetInfo(args.req); + return result; + } + } + + public static class ExecuteStatement extends org.apache.thrift.ProcessFunction { + public ExecuteStatement() { + super("ExecuteStatement"); + } + + public ExecuteStatement_args getEmptyArgsInstance() { + return new ExecuteStatement_args(); + } + + protected boolean isOneway() { + return false; + } + + public ExecuteStatement_result getResult(I iface, ExecuteStatement_args args) throws org.apache.thrift.TException { + ExecuteStatement_result result = new ExecuteStatement_result(); + result.success = iface.ExecuteStatement(args.req); + return result; + } + } + + public static class GetTypeInfo extends org.apache.thrift.ProcessFunction { + public GetTypeInfo() { + super("GetTypeInfo"); + } + + public GetTypeInfo_args getEmptyArgsInstance() { + return new GetTypeInfo_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetTypeInfo_result getResult(I iface, GetTypeInfo_args args) throws org.apache.thrift.TException { + GetTypeInfo_result result = new GetTypeInfo_result(); + result.success = iface.GetTypeInfo(args.req); + return result; + } + } + + public static class GetCatalogs extends org.apache.thrift.ProcessFunction { + public GetCatalogs() { + super("GetCatalogs"); + } + + public GetCatalogs_args getEmptyArgsInstance() { + return new GetCatalogs_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetCatalogs_result getResult(I iface, GetCatalogs_args args) throws org.apache.thrift.TException { + GetCatalogs_result result = new GetCatalogs_result(); + result.success = iface.GetCatalogs(args.req); + return result; + } + } + + public static class GetSchemas extends org.apache.thrift.ProcessFunction { + public GetSchemas() { + super("GetSchemas"); + } + + public GetSchemas_args getEmptyArgsInstance() { + return new GetSchemas_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetSchemas_result getResult(I iface, GetSchemas_args args) throws org.apache.thrift.TException { + GetSchemas_result result = new GetSchemas_result(); + result.success = iface.GetSchemas(args.req); + return result; + } + } + + public static class GetTables extends org.apache.thrift.ProcessFunction { + public GetTables() { + super("GetTables"); + } + + public GetTables_args getEmptyArgsInstance() { + return new GetTables_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetTables_result getResult(I iface, GetTables_args args) throws org.apache.thrift.TException { + GetTables_result result = new GetTables_result(); + result.success = iface.GetTables(args.req); + return result; + } + } + + public static class GetTableTypes extends org.apache.thrift.ProcessFunction { + public GetTableTypes() { + super("GetTableTypes"); + } + + public GetTableTypes_args getEmptyArgsInstance() { + return new GetTableTypes_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetTableTypes_result getResult(I iface, GetTableTypes_args args) throws org.apache.thrift.TException { + GetTableTypes_result result = new GetTableTypes_result(); + result.success = iface.GetTableTypes(args.req); + return result; + } + } + + public static class GetColumns extends org.apache.thrift.ProcessFunction { + public GetColumns() { + super("GetColumns"); + } + + public GetColumns_args getEmptyArgsInstance() { + return new GetColumns_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetColumns_result getResult(I iface, GetColumns_args args) throws org.apache.thrift.TException { + GetColumns_result result = new GetColumns_result(); + result.success = iface.GetColumns(args.req); + return result; + } + } + + public static class GetFunctions extends org.apache.thrift.ProcessFunction { + public GetFunctions() { + super("GetFunctions"); + } + + public GetFunctions_args getEmptyArgsInstance() { + return new GetFunctions_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetFunctions_result getResult(I iface, GetFunctions_args args) throws org.apache.thrift.TException { + GetFunctions_result result = new GetFunctions_result(); + result.success = iface.GetFunctions(args.req); + return result; + } + } + + public static class GetOperationStatus extends org.apache.thrift.ProcessFunction { + public GetOperationStatus() { + super("GetOperationStatus"); + } + + public GetOperationStatus_args getEmptyArgsInstance() { + return new GetOperationStatus_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetOperationStatus_result getResult(I iface, GetOperationStatus_args args) throws org.apache.thrift.TException { + GetOperationStatus_result result = new GetOperationStatus_result(); + result.success = iface.GetOperationStatus(args.req); + return result; + } + } + + public static class CancelOperation extends org.apache.thrift.ProcessFunction { + public CancelOperation() { + super("CancelOperation"); + } + + public CancelOperation_args getEmptyArgsInstance() { + return new CancelOperation_args(); + } + + protected boolean isOneway() { + return false; + } + + public CancelOperation_result getResult(I iface, CancelOperation_args args) throws org.apache.thrift.TException { + CancelOperation_result result = new CancelOperation_result(); + result.success = iface.CancelOperation(args.req); + return result; + } + } + + public static class CloseOperation extends org.apache.thrift.ProcessFunction { + public CloseOperation() { + super("CloseOperation"); + } + + public CloseOperation_args getEmptyArgsInstance() { + return new CloseOperation_args(); + } + + protected boolean isOneway() { + return false; + } + + public CloseOperation_result getResult(I iface, CloseOperation_args args) throws org.apache.thrift.TException { + CloseOperation_result result = new CloseOperation_result(); + result.success = iface.CloseOperation(args.req); + return result; + } + } + + public static class GetResultSetMetadata extends org.apache.thrift.ProcessFunction { + public GetResultSetMetadata() { + super("GetResultSetMetadata"); + } + + public GetResultSetMetadata_args getEmptyArgsInstance() { + return new GetResultSetMetadata_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetResultSetMetadata_result getResult(I iface, GetResultSetMetadata_args args) throws org.apache.thrift.TException { + GetResultSetMetadata_result result = new GetResultSetMetadata_result(); + result.success = iface.GetResultSetMetadata(args.req); + return result; + } + } + + public static class FetchResults extends org.apache.thrift.ProcessFunction { + public FetchResults() { + super("FetchResults"); + } + + public FetchResults_args getEmptyArgsInstance() { + return new FetchResults_args(); + } + + protected boolean isOneway() { + return false; + } + + public FetchResults_result getResult(I iface, FetchResults_args args) throws org.apache.thrift.TException { + FetchResults_result result = new FetchResults_result(); + result.success = iface.FetchResults(args.req); + return result; + } + } + + public static class GetDelegationToken extends org.apache.thrift.ProcessFunction { + public GetDelegationToken() { + super("GetDelegationToken"); + } + + public GetDelegationToken_args getEmptyArgsInstance() { + return new GetDelegationToken_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetDelegationToken_result getResult(I iface, GetDelegationToken_args args) throws org.apache.thrift.TException { + GetDelegationToken_result result = new GetDelegationToken_result(); + result.success = iface.GetDelegationToken(args.req); + return result; + } + } + + public static class CancelDelegationToken extends org.apache.thrift.ProcessFunction { + public CancelDelegationToken() { + super("CancelDelegationToken"); + } + + public CancelDelegationToken_args getEmptyArgsInstance() { + return new CancelDelegationToken_args(); + } + + protected boolean isOneway() { + return false; + } + + public CancelDelegationToken_result getResult(I iface, CancelDelegationToken_args args) throws org.apache.thrift.TException { + CancelDelegationToken_result result = new CancelDelegationToken_result(); + result.success = iface.CancelDelegationToken(args.req); + return result; + } + } + + public static class RenewDelegationToken extends org.apache.thrift.ProcessFunction { + public RenewDelegationToken() { + super("RenewDelegationToken"); + } + + public RenewDelegationToken_args getEmptyArgsInstance() { + return new RenewDelegationToken_args(); + } + + protected boolean isOneway() { + return false; + } + + public RenewDelegationToken_result getResult(I iface, RenewDelegationToken_args args) throws org.apache.thrift.TException { + RenewDelegationToken_result result = new RenewDelegationToken_result(); + result.success = iface.RenewDelegationToken(args.req); + return result; + } + } + + } + + public static class OpenSession_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OpenSession_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new OpenSession_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new OpenSession_argsTupleSchemeFactory()); + } + + private TOpenSessionReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOpenSessionReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(OpenSession_args.class, metaDataMap); + } + + public OpenSession_args() { + } + + public OpenSession_args( + TOpenSessionReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public OpenSession_args(OpenSession_args other) { + if (other.isSetReq()) { + this.req = new TOpenSessionReq(other.req); + } + } + + public OpenSession_args deepCopy() { + return new OpenSession_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TOpenSessionReq getReq() { + return this.req; + } + + public void setReq(TOpenSessionReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TOpenSessionReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof OpenSession_args) + return this.equals((OpenSession_args)that); + return false; + } + + public boolean equals(OpenSession_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(OpenSession_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + OpenSession_args typedOther = (OpenSession_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("OpenSession_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class OpenSession_argsStandardSchemeFactory implements SchemeFactory { + public OpenSession_argsStandardScheme getScheme() { + return new OpenSession_argsStandardScheme(); + } + } + + private static class OpenSession_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, OpenSession_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TOpenSessionReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, OpenSession_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class OpenSession_argsTupleSchemeFactory implements SchemeFactory { + public OpenSession_argsTupleScheme getScheme() { + return new OpenSession_argsTupleScheme(); + } + } + + private static class OpenSession_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, OpenSession_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, OpenSession_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TOpenSessionReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class OpenSession_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OpenSession_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new OpenSession_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new OpenSession_resultTupleSchemeFactory()); + } + + private TOpenSessionResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOpenSessionResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(OpenSession_result.class, metaDataMap); + } + + public OpenSession_result() { + } + + public OpenSession_result( + TOpenSessionResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public OpenSession_result(OpenSession_result other) { + if (other.isSetSuccess()) { + this.success = new TOpenSessionResp(other.success); + } + } + + public OpenSession_result deepCopy() { + return new OpenSession_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TOpenSessionResp getSuccess() { + return this.success; + } + + public void setSuccess(TOpenSessionResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TOpenSessionResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof OpenSession_result) + return this.equals((OpenSession_result)that); + return false; + } + + public boolean equals(OpenSession_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(OpenSession_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + OpenSession_result typedOther = (OpenSession_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("OpenSession_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class OpenSession_resultStandardSchemeFactory implements SchemeFactory { + public OpenSession_resultStandardScheme getScheme() { + return new OpenSession_resultStandardScheme(); + } + } + + private static class OpenSession_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, OpenSession_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TOpenSessionResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, OpenSession_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class OpenSession_resultTupleSchemeFactory implements SchemeFactory { + public OpenSession_resultTupleScheme getScheme() { + return new OpenSession_resultTupleScheme(); + } + } + + private static class OpenSession_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, OpenSession_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, OpenSession_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TOpenSessionResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class CloseSession_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseSession_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CloseSession_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CloseSession_argsTupleSchemeFactory()); + } + + private TCloseSessionReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseSessionReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseSession_args.class, metaDataMap); + } + + public CloseSession_args() { + } + + public CloseSession_args( + TCloseSessionReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public CloseSession_args(CloseSession_args other) { + if (other.isSetReq()) { + this.req = new TCloseSessionReq(other.req); + } + } + + public CloseSession_args deepCopy() { + return new CloseSession_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TCloseSessionReq getReq() { + return this.req; + } + + public void setReq(TCloseSessionReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TCloseSessionReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CloseSession_args) + return this.equals((CloseSession_args)that); + return false; + } + + public boolean equals(CloseSession_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(CloseSession_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + CloseSession_args typedOther = (CloseSession_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CloseSession_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CloseSession_argsStandardSchemeFactory implements SchemeFactory { + public CloseSession_argsStandardScheme getScheme() { + return new CloseSession_argsStandardScheme(); + } + } + + private static class CloseSession_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CloseSession_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TCloseSessionReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CloseSession_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CloseSession_argsTupleSchemeFactory implements SchemeFactory { + public CloseSession_argsTupleScheme getScheme() { + return new CloseSession_argsTupleScheme(); + } + } + + private static class CloseSession_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CloseSession_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CloseSession_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TCloseSessionReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class CloseSession_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseSession_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CloseSession_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CloseSession_resultTupleSchemeFactory()); + } + + private TCloseSessionResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseSessionResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseSession_result.class, metaDataMap); + } + + public CloseSession_result() { + } + + public CloseSession_result( + TCloseSessionResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public CloseSession_result(CloseSession_result other) { + if (other.isSetSuccess()) { + this.success = new TCloseSessionResp(other.success); + } + } + + public CloseSession_result deepCopy() { + return new CloseSession_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TCloseSessionResp getSuccess() { + return this.success; + } + + public void setSuccess(TCloseSessionResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TCloseSessionResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CloseSession_result) + return this.equals((CloseSession_result)that); + return false; + } + + public boolean equals(CloseSession_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(CloseSession_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + CloseSession_result typedOther = (CloseSession_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CloseSession_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CloseSession_resultStandardSchemeFactory implements SchemeFactory { + public CloseSession_resultStandardScheme getScheme() { + return new CloseSession_resultStandardScheme(); + } + } + + private static class CloseSession_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CloseSession_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TCloseSessionResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CloseSession_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CloseSession_resultTupleSchemeFactory implements SchemeFactory { + public CloseSession_resultTupleScheme getScheme() { + return new CloseSession_resultTupleScheme(); + } + } + + private static class CloseSession_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CloseSession_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CloseSession_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TCloseSessionResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetInfo_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetInfo_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetInfo_argsTupleSchemeFactory()); + } + + private TGetInfoReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetInfoReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetInfo_args.class, metaDataMap); + } + + public GetInfo_args() { + } + + public GetInfo_args( + TGetInfoReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetInfo_args(GetInfo_args other) { + if (other.isSetReq()) { + this.req = new TGetInfoReq(other.req); + } + } + + public GetInfo_args deepCopy() { + return new GetInfo_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetInfoReq getReq() { + return this.req; + } + + public void setReq(TGetInfoReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetInfoReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetInfo_args) + return this.equals((GetInfo_args)that); + return false; + } + + public boolean equals(GetInfo_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(GetInfo_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetInfo_args typedOther = (GetInfo_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetInfo_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetInfo_argsStandardSchemeFactory implements SchemeFactory { + public GetInfo_argsStandardScheme getScheme() { + return new GetInfo_argsStandardScheme(); + } + } + + private static class GetInfo_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetInfo_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetInfoReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetInfo_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetInfo_argsTupleSchemeFactory implements SchemeFactory { + public GetInfo_argsTupleScheme getScheme() { + return new GetInfo_argsTupleScheme(); + } + } + + private static class GetInfo_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetInfoReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetInfo_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetInfo_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetInfo_resultTupleSchemeFactory()); + } + + private TGetInfoResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetInfoResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetInfo_result.class, metaDataMap); + } + + public GetInfo_result() { + } + + public GetInfo_result( + TGetInfoResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetInfo_result(GetInfo_result other) { + if (other.isSetSuccess()) { + this.success = new TGetInfoResp(other.success); + } + } + + public GetInfo_result deepCopy() { + return new GetInfo_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetInfoResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetInfoResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetInfoResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetInfo_result) + return this.equals((GetInfo_result)that); + return false; + } + + public boolean equals(GetInfo_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(GetInfo_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetInfo_result typedOther = (GetInfo_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetInfo_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetInfo_resultStandardSchemeFactory implements SchemeFactory { + public GetInfo_resultStandardScheme getScheme() { + return new GetInfo_resultStandardScheme(); + } + } + + private static class GetInfo_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetInfo_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetInfoResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetInfo_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetInfo_resultTupleSchemeFactory implements SchemeFactory { + public GetInfo_resultTupleScheme getScheme() { + return new GetInfo_resultTupleScheme(); + } + } + + private static class GetInfo_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetInfoResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class ExecuteStatement_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecuteStatement_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ExecuteStatement_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ExecuteStatement_argsTupleSchemeFactory()); + } + + private TExecuteStatementReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TExecuteStatementReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecuteStatement_args.class, metaDataMap); + } + + public ExecuteStatement_args() { + } + + public ExecuteStatement_args( + TExecuteStatementReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public ExecuteStatement_args(ExecuteStatement_args other) { + if (other.isSetReq()) { + this.req = new TExecuteStatementReq(other.req); + } + } + + public ExecuteStatement_args deepCopy() { + return new ExecuteStatement_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TExecuteStatementReq getReq() { + return this.req; + } + + public void setReq(TExecuteStatementReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TExecuteStatementReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ExecuteStatement_args) + return this.equals((ExecuteStatement_args)that); + return false; + } + + public boolean equals(ExecuteStatement_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(ExecuteStatement_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + ExecuteStatement_args typedOther = (ExecuteStatement_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ExecuteStatement_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ExecuteStatement_argsStandardSchemeFactory implements SchemeFactory { + public ExecuteStatement_argsStandardScheme getScheme() { + return new ExecuteStatement_argsStandardScheme(); + } + } + + private static class ExecuteStatement_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ExecuteStatement_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TExecuteStatementReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ExecuteStatement_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ExecuteStatement_argsTupleSchemeFactory implements SchemeFactory { + public ExecuteStatement_argsTupleScheme getScheme() { + return new ExecuteStatement_argsTupleScheme(); + } + } + + private static class ExecuteStatement_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TExecuteStatementReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class ExecuteStatement_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecuteStatement_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ExecuteStatement_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ExecuteStatement_resultTupleSchemeFactory()); + } + + private TExecuteStatementResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TExecuteStatementResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecuteStatement_result.class, metaDataMap); + } + + public ExecuteStatement_result() { + } + + public ExecuteStatement_result( + TExecuteStatementResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public ExecuteStatement_result(ExecuteStatement_result other) { + if (other.isSetSuccess()) { + this.success = new TExecuteStatementResp(other.success); + } + } + + public ExecuteStatement_result deepCopy() { + return new ExecuteStatement_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TExecuteStatementResp getSuccess() { + return this.success; + } + + public void setSuccess(TExecuteStatementResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TExecuteStatementResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ExecuteStatement_result) + return this.equals((ExecuteStatement_result)that); + return false; + } + + public boolean equals(ExecuteStatement_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(ExecuteStatement_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + ExecuteStatement_result typedOther = (ExecuteStatement_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ExecuteStatement_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ExecuteStatement_resultStandardSchemeFactory implements SchemeFactory { + public ExecuteStatement_resultStandardScheme getScheme() { + return new ExecuteStatement_resultStandardScheme(); + } + } + + private static class ExecuteStatement_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ExecuteStatement_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TExecuteStatementResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ExecuteStatement_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ExecuteStatement_resultTupleSchemeFactory implements SchemeFactory { + public ExecuteStatement_resultTupleScheme getScheme() { + return new ExecuteStatement_resultTupleScheme(); + } + } + + private static class ExecuteStatement_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TExecuteStatementResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetTypeInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTypeInfo_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetTypeInfo_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetTypeInfo_argsTupleSchemeFactory()); + } + + private TGetTypeInfoReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTypeInfoReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTypeInfo_args.class, metaDataMap); + } + + public GetTypeInfo_args() { + } + + public GetTypeInfo_args( + TGetTypeInfoReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetTypeInfo_args(GetTypeInfo_args other) { + if (other.isSetReq()) { + this.req = new TGetTypeInfoReq(other.req); + } + } + + public GetTypeInfo_args deepCopy() { + return new GetTypeInfo_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetTypeInfoReq getReq() { + return this.req; + } + + public void setReq(TGetTypeInfoReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetTypeInfoReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetTypeInfo_args) + return this.equals((GetTypeInfo_args)that); + return false; + } + + public boolean equals(GetTypeInfo_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(GetTypeInfo_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetTypeInfo_args typedOther = (GetTypeInfo_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetTypeInfo_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetTypeInfo_argsStandardSchemeFactory implements SchemeFactory { + public GetTypeInfo_argsStandardScheme getScheme() { + return new GetTypeInfo_argsStandardScheme(); + } + } + + private static class GetTypeInfo_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetTypeInfo_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetTypeInfoReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetTypeInfo_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetTypeInfo_argsTupleSchemeFactory implements SchemeFactory { + public GetTypeInfo_argsTupleScheme getScheme() { + return new GetTypeInfo_argsTupleScheme(); + } + } + + private static class GetTypeInfo_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetTypeInfoReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetTypeInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTypeInfo_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetTypeInfo_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetTypeInfo_resultTupleSchemeFactory()); + } + + private TGetTypeInfoResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTypeInfoResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTypeInfo_result.class, metaDataMap); + } + + public GetTypeInfo_result() { + } + + public GetTypeInfo_result( + TGetTypeInfoResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetTypeInfo_result(GetTypeInfo_result other) { + if (other.isSetSuccess()) { + this.success = new TGetTypeInfoResp(other.success); + } + } + + public GetTypeInfo_result deepCopy() { + return new GetTypeInfo_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetTypeInfoResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetTypeInfoResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetTypeInfoResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetTypeInfo_result) + return this.equals((GetTypeInfo_result)that); + return false; + } + + public boolean equals(GetTypeInfo_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(GetTypeInfo_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetTypeInfo_result typedOther = (GetTypeInfo_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetTypeInfo_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetTypeInfo_resultStandardSchemeFactory implements SchemeFactory { + public GetTypeInfo_resultStandardScheme getScheme() { + return new GetTypeInfo_resultStandardScheme(); + } + } + + private static class GetTypeInfo_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetTypeInfo_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetTypeInfoResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetTypeInfo_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetTypeInfo_resultTupleSchemeFactory implements SchemeFactory { + public GetTypeInfo_resultTupleScheme getScheme() { + return new GetTypeInfo_resultTupleScheme(); + } + } + + private static class GetTypeInfo_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetTypeInfoResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetCatalogs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetCatalogs_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetCatalogs_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetCatalogs_argsTupleSchemeFactory()); + } + + private TGetCatalogsReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetCatalogsReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetCatalogs_args.class, metaDataMap); + } + + public GetCatalogs_args() { + } + + public GetCatalogs_args( + TGetCatalogsReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetCatalogs_args(GetCatalogs_args other) { + if (other.isSetReq()) { + this.req = new TGetCatalogsReq(other.req); + } + } + + public GetCatalogs_args deepCopy() { + return new GetCatalogs_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetCatalogsReq getReq() { + return this.req; + } + + public void setReq(TGetCatalogsReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetCatalogsReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetCatalogs_args) + return this.equals((GetCatalogs_args)that); + return false; + } + + public boolean equals(GetCatalogs_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(GetCatalogs_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetCatalogs_args typedOther = (GetCatalogs_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetCatalogs_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetCatalogs_argsStandardSchemeFactory implements SchemeFactory { + public GetCatalogs_argsStandardScheme getScheme() { + return new GetCatalogs_argsStandardScheme(); + } + } + + private static class GetCatalogs_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetCatalogs_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetCatalogsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetCatalogs_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetCatalogs_argsTupleSchemeFactory implements SchemeFactory { + public GetCatalogs_argsTupleScheme getScheme() { + return new GetCatalogs_argsTupleScheme(); + } + } + + private static class GetCatalogs_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetCatalogsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetCatalogs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetCatalogs_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetCatalogs_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetCatalogs_resultTupleSchemeFactory()); + } + + private TGetCatalogsResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetCatalogsResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetCatalogs_result.class, metaDataMap); + } + + public GetCatalogs_result() { + } + + public GetCatalogs_result( + TGetCatalogsResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetCatalogs_result(GetCatalogs_result other) { + if (other.isSetSuccess()) { + this.success = new TGetCatalogsResp(other.success); + } + } + + public GetCatalogs_result deepCopy() { + return new GetCatalogs_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetCatalogsResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetCatalogsResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetCatalogsResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetCatalogs_result) + return this.equals((GetCatalogs_result)that); + return false; + } + + public boolean equals(GetCatalogs_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(GetCatalogs_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetCatalogs_result typedOther = (GetCatalogs_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetCatalogs_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetCatalogs_resultStandardSchemeFactory implements SchemeFactory { + public GetCatalogs_resultStandardScheme getScheme() { + return new GetCatalogs_resultStandardScheme(); + } + } + + private static class GetCatalogs_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetCatalogs_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetCatalogsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetCatalogs_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetCatalogs_resultTupleSchemeFactory implements SchemeFactory { + public GetCatalogs_resultTupleScheme getScheme() { + return new GetCatalogs_resultTupleScheme(); + } + } + + private static class GetCatalogs_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetCatalogsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetSchemas_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetSchemas_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetSchemas_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetSchemas_argsTupleSchemeFactory()); + } + + private TGetSchemasReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetSchemasReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetSchemas_args.class, metaDataMap); + } + + public GetSchemas_args() { + } + + public GetSchemas_args( + TGetSchemasReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetSchemas_args(GetSchemas_args other) { + if (other.isSetReq()) { + this.req = new TGetSchemasReq(other.req); + } + } + + public GetSchemas_args deepCopy() { + return new GetSchemas_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetSchemasReq getReq() { + return this.req; + } + + public void setReq(TGetSchemasReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetSchemasReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetSchemas_args) + return this.equals((GetSchemas_args)that); + return false; + } + + public boolean equals(GetSchemas_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(GetSchemas_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetSchemas_args typedOther = (GetSchemas_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetSchemas_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetSchemas_argsStandardSchemeFactory implements SchemeFactory { + public GetSchemas_argsStandardScheme getScheme() { + return new GetSchemas_argsStandardScheme(); + } + } + + private static class GetSchemas_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetSchemas_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetSchemasReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetSchemas_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetSchemas_argsTupleSchemeFactory implements SchemeFactory { + public GetSchemas_argsTupleScheme getScheme() { + return new GetSchemas_argsTupleScheme(); + } + } + + private static class GetSchemas_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetSchemas_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetSchemas_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetSchemasReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetSchemas_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetSchemas_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetSchemas_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetSchemas_resultTupleSchemeFactory()); + } + + private TGetSchemasResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetSchemasResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetSchemas_result.class, metaDataMap); + } + + public GetSchemas_result() { + } + + public GetSchemas_result( + TGetSchemasResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetSchemas_result(GetSchemas_result other) { + if (other.isSetSuccess()) { + this.success = new TGetSchemasResp(other.success); + } + } + + public GetSchemas_result deepCopy() { + return new GetSchemas_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetSchemasResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetSchemasResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetSchemasResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetSchemas_result) + return this.equals((GetSchemas_result)that); + return false; + } + + public boolean equals(GetSchemas_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(GetSchemas_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetSchemas_result typedOther = (GetSchemas_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetSchemas_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetSchemas_resultStandardSchemeFactory implements SchemeFactory { + public GetSchemas_resultStandardScheme getScheme() { + return new GetSchemas_resultStandardScheme(); + } + } + + private static class GetSchemas_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetSchemas_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetSchemasResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetSchemas_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetSchemas_resultTupleSchemeFactory implements SchemeFactory { + public GetSchemas_resultTupleScheme getScheme() { + return new GetSchemas_resultTupleScheme(); + } + } + + private static class GetSchemas_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetSchemas_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetSchemas_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetSchemasResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetTables_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTables_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetTables_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetTables_argsTupleSchemeFactory()); + } + + private TGetTablesReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTablesReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTables_args.class, metaDataMap); + } + + public GetTables_args() { + } + + public GetTables_args( + TGetTablesReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetTables_args(GetTables_args other) { + if (other.isSetReq()) { + this.req = new TGetTablesReq(other.req); + } + } + + public GetTables_args deepCopy() { + return new GetTables_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetTablesReq getReq() { + return this.req; + } + + public void setReq(TGetTablesReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetTablesReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetTables_args) + return this.equals((GetTables_args)that); + return false; + } + + public boolean equals(GetTables_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(GetTables_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetTables_args typedOther = (GetTables_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetTables_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetTables_argsStandardSchemeFactory implements SchemeFactory { + public GetTables_argsStandardScheme getScheme() { + return new GetTables_argsStandardScheme(); + } + } + + private static class GetTables_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetTables_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetTablesReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetTables_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetTables_argsTupleSchemeFactory implements SchemeFactory { + public GetTables_argsTupleScheme getScheme() { + return new GetTables_argsTupleScheme(); + } + } + + private static class GetTables_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetTables_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetTables_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetTablesReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetTables_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTables_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetTables_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetTables_resultTupleSchemeFactory()); + } + + private TGetTablesResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTablesResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTables_result.class, metaDataMap); + } + + public GetTables_result() { + } + + public GetTables_result( + TGetTablesResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetTables_result(GetTables_result other) { + if (other.isSetSuccess()) { + this.success = new TGetTablesResp(other.success); + } + } + + public GetTables_result deepCopy() { + return new GetTables_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetTablesResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetTablesResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetTablesResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetTables_result) + return this.equals((GetTables_result)that); + return false; + } + + public boolean equals(GetTables_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(GetTables_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetTables_result typedOther = (GetTables_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetTables_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetTables_resultStandardSchemeFactory implements SchemeFactory { + public GetTables_resultStandardScheme getScheme() { + return new GetTables_resultStandardScheme(); + } + } + + private static class GetTables_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetTables_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetTablesResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetTables_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetTables_resultTupleSchemeFactory implements SchemeFactory { + public GetTables_resultTupleScheme getScheme() { + return new GetTables_resultTupleScheme(); + } + } + + private static class GetTables_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetTables_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetTables_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetTablesResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetTableTypes_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTableTypes_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetTableTypes_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetTableTypes_argsTupleSchemeFactory()); + } + + private TGetTableTypesReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTableTypesReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTableTypes_args.class, metaDataMap); + } + + public GetTableTypes_args() { + } + + public GetTableTypes_args( + TGetTableTypesReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetTableTypes_args(GetTableTypes_args other) { + if (other.isSetReq()) { + this.req = new TGetTableTypesReq(other.req); + } + } + + public GetTableTypes_args deepCopy() { + return new GetTableTypes_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetTableTypesReq getReq() { + return this.req; + } + + public void setReq(TGetTableTypesReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetTableTypesReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetTableTypes_args) + return this.equals((GetTableTypes_args)that); + return false; + } + + public boolean equals(GetTableTypes_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(GetTableTypes_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetTableTypes_args typedOther = (GetTableTypes_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetTableTypes_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetTableTypes_argsStandardSchemeFactory implements SchemeFactory { + public GetTableTypes_argsStandardScheme getScheme() { + return new GetTableTypes_argsStandardScheme(); + } + } + + private static class GetTableTypes_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetTableTypes_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetTableTypesReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetTableTypes_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetTableTypes_argsTupleSchemeFactory implements SchemeFactory { + public GetTableTypes_argsTupleScheme getScheme() { + return new GetTableTypes_argsTupleScheme(); + } + } + + private static class GetTableTypes_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetTableTypesReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetTableTypes_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTableTypes_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetTableTypes_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetTableTypes_resultTupleSchemeFactory()); + } + + private TGetTableTypesResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTableTypesResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTableTypes_result.class, metaDataMap); + } + + public GetTableTypes_result() { + } + + public GetTableTypes_result( + TGetTableTypesResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetTableTypes_result(GetTableTypes_result other) { + if (other.isSetSuccess()) { + this.success = new TGetTableTypesResp(other.success); + } + } + + public GetTableTypes_result deepCopy() { + return new GetTableTypes_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetTableTypesResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetTableTypesResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetTableTypesResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetTableTypes_result) + return this.equals((GetTableTypes_result)that); + return false; + } + + public boolean equals(GetTableTypes_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(GetTableTypes_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetTableTypes_result typedOther = (GetTableTypes_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetTableTypes_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetTableTypes_resultStandardSchemeFactory implements SchemeFactory { + public GetTableTypes_resultStandardScheme getScheme() { + return new GetTableTypes_resultStandardScheme(); + } + } + + private static class GetTableTypes_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetTableTypes_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetTableTypesResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetTableTypes_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetTableTypes_resultTupleSchemeFactory implements SchemeFactory { + public GetTableTypes_resultTupleScheme getScheme() { + return new GetTableTypes_resultTupleScheme(); + } + } + + private static class GetTableTypes_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetTableTypesResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetColumns_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetColumns_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetColumns_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetColumns_argsTupleSchemeFactory()); + } + + private TGetColumnsReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetColumnsReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetColumns_args.class, metaDataMap); + } + + public GetColumns_args() { + } + + public GetColumns_args( + TGetColumnsReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetColumns_args(GetColumns_args other) { + if (other.isSetReq()) { + this.req = new TGetColumnsReq(other.req); + } + } + + public GetColumns_args deepCopy() { + return new GetColumns_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetColumnsReq getReq() { + return this.req; + } + + public void setReq(TGetColumnsReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetColumnsReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetColumns_args) + return this.equals((GetColumns_args)that); + return false; + } + + public boolean equals(GetColumns_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(GetColumns_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetColumns_args typedOther = (GetColumns_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetColumns_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetColumns_argsStandardSchemeFactory implements SchemeFactory { + public GetColumns_argsStandardScheme getScheme() { + return new GetColumns_argsStandardScheme(); + } + } + + private static class GetColumns_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetColumns_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetColumnsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetColumns_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetColumns_argsTupleSchemeFactory implements SchemeFactory { + public GetColumns_argsTupleScheme getScheme() { + return new GetColumns_argsTupleScheme(); + } + } + + private static class GetColumns_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetColumns_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetColumns_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetColumnsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetColumns_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetColumns_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetColumns_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetColumns_resultTupleSchemeFactory()); + } + + private TGetColumnsResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetColumnsResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetColumns_result.class, metaDataMap); + } + + public GetColumns_result() { + } + + public GetColumns_result( + TGetColumnsResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetColumns_result(GetColumns_result other) { + if (other.isSetSuccess()) { + this.success = new TGetColumnsResp(other.success); + } + } + + public GetColumns_result deepCopy() { + return new GetColumns_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetColumnsResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetColumnsResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetColumnsResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetColumns_result) + return this.equals((GetColumns_result)that); + return false; + } + + public boolean equals(GetColumns_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(GetColumns_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetColumns_result typedOther = (GetColumns_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetColumns_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetColumns_resultStandardSchemeFactory implements SchemeFactory { + public GetColumns_resultStandardScheme getScheme() { + return new GetColumns_resultStandardScheme(); + } + } + + private static class GetColumns_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetColumns_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetColumnsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetColumns_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetColumns_resultTupleSchemeFactory implements SchemeFactory { + public GetColumns_resultTupleScheme getScheme() { + return new GetColumns_resultTupleScheme(); + } + } + + private static class GetColumns_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetColumns_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetColumns_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetColumnsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetFunctions_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetFunctions_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetFunctions_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetFunctions_argsTupleSchemeFactory()); + } + + private TGetFunctionsReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetFunctionsReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetFunctions_args.class, metaDataMap); + } + + public GetFunctions_args() { + } + + public GetFunctions_args( + TGetFunctionsReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetFunctions_args(GetFunctions_args other) { + if (other.isSetReq()) { + this.req = new TGetFunctionsReq(other.req); + } + } + + public GetFunctions_args deepCopy() { + return new GetFunctions_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetFunctionsReq getReq() { + return this.req; + } + + public void setReq(TGetFunctionsReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetFunctionsReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetFunctions_args) + return this.equals((GetFunctions_args)that); + return false; + } + + public boolean equals(GetFunctions_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(GetFunctions_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetFunctions_args typedOther = (GetFunctions_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetFunctions_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetFunctions_argsStandardSchemeFactory implements SchemeFactory { + public GetFunctions_argsStandardScheme getScheme() { + return new GetFunctions_argsStandardScheme(); + } + } + + private static class GetFunctions_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetFunctions_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetFunctionsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetFunctions_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetFunctions_argsTupleSchemeFactory implements SchemeFactory { + public GetFunctions_argsTupleScheme getScheme() { + return new GetFunctions_argsTupleScheme(); + } + } + + private static class GetFunctions_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetFunctions_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetFunctions_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetFunctionsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetFunctions_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetFunctions_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetFunctions_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetFunctions_resultTupleSchemeFactory()); + } + + private TGetFunctionsResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetFunctionsResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetFunctions_result.class, metaDataMap); + } + + public GetFunctions_result() { + } + + public GetFunctions_result( + TGetFunctionsResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetFunctions_result(GetFunctions_result other) { + if (other.isSetSuccess()) { + this.success = new TGetFunctionsResp(other.success); + } + } + + public GetFunctions_result deepCopy() { + return new GetFunctions_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetFunctionsResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetFunctionsResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetFunctionsResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetFunctions_result) + return this.equals((GetFunctions_result)that); + return false; + } + + public boolean equals(GetFunctions_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(GetFunctions_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetFunctions_result typedOther = (GetFunctions_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetFunctions_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetFunctions_resultStandardSchemeFactory implements SchemeFactory { + public GetFunctions_resultStandardScheme getScheme() { + return new GetFunctions_resultStandardScheme(); + } + } + + private static class GetFunctions_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetFunctions_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetFunctionsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetFunctions_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetFunctions_resultTupleSchemeFactory implements SchemeFactory { + public GetFunctions_resultTupleScheme getScheme() { + return new GetFunctions_resultTupleScheme(); + } + } + + private static class GetFunctions_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetFunctions_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetFunctions_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetFunctionsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetOperationStatus_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOperationStatus_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetOperationStatus_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetOperationStatus_argsTupleSchemeFactory()); + } + + private TGetOperationStatusReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetOperationStatusReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetOperationStatus_args.class, metaDataMap); + } + + public GetOperationStatus_args() { + } + + public GetOperationStatus_args( + TGetOperationStatusReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetOperationStatus_args(GetOperationStatus_args other) { + if (other.isSetReq()) { + this.req = new TGetOperationStatusReq(other.req); + } + } + + public GetOperationStatus_args deepCopy() { + return new GetOperationStatus_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetOperationStatusReq getReq() { + return this.req; + } + + public void setReq(TGetOperationStatusReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetOperationStatusReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetOperationStatus_args) + return this.equals((GetOperationStatus_args)that); + return false; + } + + public boolean equals(GetOperationStatus_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(GetOperationStatus_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetOperationStatus_args typedOther = (GetOperationStatus_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetOperationStatus_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetOperationStatus_argsStandardSchemeFactory implements SchemeFactory { + public GetOperationStatus_argsStandardScheme getScheme() { + return new GetOperationStatus_argsStandardScheme(); + } + } + + private static class GetOperationStatus_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetOperationStatus_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetOperationStatusReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetOperationStatus_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetOperationStatus_argsTupleSchemeFactory implements SchemeFactory { + public GetOperationStatus_argsTupleScheme getScheme() { + return new GetOperationStatus_argsTupleScheme(); + } + } + + private static class GetOperationStatus_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetOperationStatusReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetOperationStatus_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOperationStatus_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetOperationStatus_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetOperationStatus_resultTupleSchemeFactory()); + } + + private TGetOperationStatusResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetOperationStatusResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetOperationStatus_result.class, metaDataMap); + } + + public GetOperationStatus_result() { + } + + public GetOperationStatus_result( + TGetOperationStatusResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetOperationStatus_result(GetOperationStatus_result other) { + if (other.isSetSuccess()) { + this.success = new TGetOperationStatusResp(other.success); + } + } + + public GetOperationStatus_result deepCopy() { + return new GetOperationStatus_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetOperationStatusResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetOperationStatusResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetOperationStatusResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetOperationStatus_result) + return this.equals((GetOperationStatus_result)that); + return false; + } + + public boolean equals(GetOperationStatus_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(GetOperationStatus_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetOperationStatus_result typedOther = (GetOperationStatus_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetOperationStatus_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetOperationStatus_resultStandardSchemeFactory implements SchemeFactory { + public GetOperationStatus_resultStandardScheme getScheme() { + return new GetOperationStatus_resultStandardScheme(); + } + } + + private static class GetOperationStatus_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetOperationStatus_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetOperationStatusResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetOperationStatus_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetOperationStatus_resultTupleSchemeFactory implements SchemeFactory { + public GetOperationStatus_resultTupleScheme getScheme() { + return new GetOperationStatus_resultTupleScheme(); + } + } + + private static class GetOperationStatus_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetOperationStatusResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class CancelOperation_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelOperation_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CancelOperation_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CancelOperation_argsTupleSchemeFactory()); + } + + private TCancelOperationReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelOperationReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelOperation_args.class, metaDataMap); + } + + public CancelOperation_args() { + } + + public CancelOperation_args( + TCancelOperationReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public CancelOperation_args(CancelOperation_args other) { + if (other.isSetReq()) { + this.req = new TCancelOperationReq(other.req); + } + } + + public CancelOperation_args deepCopy() { + return new CancelOperation_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TCancelOperationReq getReq() { + return this.req; + } + + public void setReq(TCancelOperationReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TCancelOperationReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CancelOperation_args) + return this.equals((CancelOperation_args)that); + return false; + } + + public boolean equals(CancelOperation_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(CancelOperation_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + CancelOperation_args typedOther = (CancelOperation_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CancelOperation_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CancelOperation_argsStandardSchemeFactory implements SchemeFactory { + public CancelOperation_argsStandardScheme getScheme() { + return new CancelOperation_argsStandardScheme(); + } + } + + private static class CancelOperation_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CancelOperation_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TCancelOperationReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CancelOperation_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CancelOperation_argsTupleSchemeFactory implements SchemeFactory { + public CancelOperation_argsTupleScheme getScheme() { + return new CancelOperation_argsTupleScheme(); + } + } + + private static class CancelOperation_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CancelOperation_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CancelOperation_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TCancelOperationReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class CancelOperation_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelOperation_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CancelOperation_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CancelOperation_resultTupleSchemeFactory()); + } + + private TCancelOperationResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelOperationResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelOperation_result.class, metaDataMap); + } + + public CancelOperation_result() { + } + + public CancelOperation_result( + TCancelOperationResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public CancelOperation_result(CancelOperation_result other) { + if (other.isSetSuccess()) { + this.success = new TCancelOperationResp(other.success); + } + } + + public CancelOperation_result deepCopy() { + return new CancelOperation_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TCancelOperationResp getSuccess() { + return this.success; + } + + public void setSuccess(TCancelOperationResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TCancelOperationResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CancelOperation_result) + return this.equals((CancelOperation_result)that); + return false; + } + + public boolean equals(CancelOperation_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(CancelOperation_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + CancelOperation_result typedOther = (CancelOperation_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CancelOperation_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CancelOperation_resultStandardSchemeFactory implements SchemeFactory { + public CancelOperation_resultStandardScheme getScheme() { + return new CancelOperation_resultStandardScheme(); + } + } + + private static class CancelOperation_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CancelOperation_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TCancelOperationResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CancelOperation_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CancelOperation_resultTupleSchemeFactory implements SchemeFactory { + public CancelOperation_resultTupleScheme getScheme() { + return new CancelOperation_resultTupleScheme(); + } + } + + private static class CancelOperation_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CancelOperation_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CancelOperation_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TCancelOperationResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class CloseOperation_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseOperation_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CloseOperation_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CloseOperation_argsTupleSchemeFactory()); + } + + private TCloseOperationReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseOperationReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseOperation_args.class, metaDataMap); + } + + public CloseOperation_args() { + } + + public CloseOperation_args( + TCloseOperationReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public CloseOperation_args(CloseOperation_args other) { + if (other.isSetReq()) { + this.req = new TCloseOperationReq(other.req); + } + } + + public CloseOperation_args deepCopy() { + return new CloseOperation_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TCloseOperationReq getReq() { + return this.req; + } + + public void setReq(TCloseOperationReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TCloseOperationReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CloseOperation_args) + return this.equals((CloseOperation_args)that); + return false; + } + + public boolean equals(CloseOperation_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(CloseOperation_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + CloseOperation_args typedOther = (CloseOperation_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CloseOperation_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CloseOperation_argsStandardSchemeFactory implements SchemeFactory { + public CloseOperation_argsStandardScheme getScheme() { + return new CloseOperation_argsStandardScheme(); + } + } + + private static class CloseOperation_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CloseOperation_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TCloseOperationReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CloseOperation_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CloseOperation_argsTupleSchemeFactory implements SchemeFactory { + public CloseOperation_argsTupleScheme getScheme() { + return new CloseOperation_argsTupleScheme(); + } + } + + private static class CloseOperation_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CloseOperation_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CloseOperation_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TCloseOperationReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class CloseOperation_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseOperation_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CloseOperation_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CloseOperation_resultTupleSchemeFactory()); + } + + private TCloseOperationResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseOperationResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseOperation_result.class, metaDataMap); + } + + public CloseOperation_result() { + } + + public CloseOperation_result( + TCloseOperationResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public CloseOperation_result(CloseOperation_result other) { + if (other.isSetSuccess()) { + this.success = new TCloseOperationResp(other.success); + } + } + + public CloseOperation_result deepCopy() { + return new CloseOperation_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TCloseOperationResp getSuccess() { + return this.success; + } + + public void setSuccess(TCloseOperationResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TCloseOperationResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CloseOperation_result) + return this.equals((CloseOperation_result)that); + return false; + } + + public boolean equals(CloseOperation_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(CloseOperation_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + CloseOperation_result typedOther = (CloseOperation_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CloseOperation_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CloseOperation_resultStandardSchemeFactory implements SchemeFactory { + public CloseOperation_resultStandardScheme getScheme() { + return new CloseOperation_resultStandardScheme(); + } + } + + private static class CloseOperation_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CloseOperation_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TCloseOperationResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CloseOperation_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CloseOperation_resultTupleSchemeFactory implements SchemeFactory { + public CloseOperation_resultTupleScheme getScheme() { + return new CloseOperation_resultTupleScheme(); + } + } + + private static class CloseOperation_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CloseOperation_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CloseOperation_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TCloseOperationResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetResultSetMetadata_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetResultSetMetadata_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetResultSetMetadata_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetResultSetMetadata_argsTupleSchemeFactory()); + } + + private TGetResultSetMetadataReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetResultSetMetadataReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetResultSetMetadata_args.class, metaDataMap); + } + + public GetResultSetMetadata_args() { + } + + public GetResultSetMetadata_args( + TGetResultSetMetadataReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetResultSetMetadata_args(GetResultSetMetadata_args other) { + if (other.isSetReq()) { + this.req = new TGetResultSetMetadataReq(other.req); + } + } + + public GetResultSetMetadata_args deepCopy() { + return new GetResultSetMetadata_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetResultSetMetadataReq getReq() { + return this.req; + } + + public void setReq(TGetResultSetMetadataReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetResultSetMetadataReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetResultSetMetadata_args) + return this.equals((GetResultSetMetadata_args)that); + return false; + } + + public boolean equals(GetResultSetMetadata_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(GetResultSetMetadata_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetResultSetMetadata_args typedOther = (GetResultSetMetadata_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetResultSetMetadata_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetResultSetMetadata_argsStandardSchemeFactory implements SchemeFactory { + public GetResultSetMetadata_argsStandardScheme getScheme() { + return new GetResultSetMetadata_argsStandardScheme(); + } + } + + private static class GetResultSetMetadata_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetResultSetMetadataReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetResultSetMetadata_argsTupleSchemeFactory implements SchemeFactory { + public GetResultSetMetadata_argsTupleScheme getScheme() { + return new GetResultSetMetadata_argsTupleScheme(); + } + } + + private static class GetResultSetMetadata_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetResultSetMetadataReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetResultSetMetadata_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetResultSetMetadata_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetResultSetMetadata_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetResultSetMetadata_resultTupleSchemeFactory()); + } + + private TGetResultSetMetadataResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetResultSetMetadataResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetResultSetMetadata_result.class, metaDataMap); + } + + public GetResultSetMetadata_result() { + } + + public GetResultSetMetadata_result( + TGetResultSetMetadataResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetResultSetMetadata_result(GetResultSetMetadata_result other) { + if (other.isSetSuccess()) { + this.success = new TGetResultSetMetadataResp(other.success); + } + } + + public GetResultSetMetadata_result deepCopy() { + return new GetResultSetMetadata_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetResultSetMetadataResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetResultSetMetadataResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetResultSetMetadataResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetResultSetMetadata_result) + return this.equals((GetResultSetMetadata_result)that); + return false; + } + + public boolean equals(GetResultSetMetadata_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(GetResultSetMetadata_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetResultSetMetadata_result typedOther = (GetResultSetMetadata_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetResultSetMetadata_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetResultSetMetadata_resultStandardSchemeFactory implements SchemeFactory { + public GetResultSetMetadata_resultStandardScheme getScheme() { + return new GetResultSetMetadata_resultStandardScheme(); + } + } + + private static class GetResultSetMetadata_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetResultSetMetadataResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetResultSetMetadata_resultTupleSchemeFactory implements SchemeFactory { + public GetResultSetMetadata_resultTupleScheme getScheme() { + return new GetResultSetMetadata_resultTupleScheme(); + } + } + + private static class GetResultSetMetadata_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetResultSetMetadataResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class FetchResults_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FetchResults_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new FetchResults_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new FetchResults_argsTupleSchemeFactory()); + } + + private TFetchResultsReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFetchResultsReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(FetchResults_args.class, metaDataMap); + } + + public FetchResults_args() { + } + + public FetchResults_args( + TFetchResultsReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public FetchResults_args(FetchResults_args other) { + if (other.isSetReq()) { + this.req = new TFetchResultsReq(other.req); + } + } + + public FetchResults_args deepCopy() { + return new FetchResults_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TFetchResultsReq getReq() { + return this.req; + } + + public void setReq(TFetchResultsReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TFetchResultsReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof FetchResults_args) + return this.equals((FetchResults_args)that); + return false; + } + + public boolean equals(FetchResults_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(FetchResults_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + FetchResults_args typedOther = (FetchResults_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("FetchResults_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class FetchResults_argsStandardSchemeFactory implements SchemeFactory { + public FetchResults_argsStandardScheme getScheme() { + return new FetchResults_argsStandardScheme(); + } + } + + private static class FetchResults_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, FetchResults_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TFetchResultsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, FetchResults_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class FetchResults_argsTupleSchemeFactory implements SchemeFactory { + public FetchResults_argsTupleScheme getScheme() { + return new FetchResults_argsTupleScheme(); + } + } + + private static class FetchResults_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, FetchResults_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, FetchResults_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TFetchResultsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class FetchResults_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FetchResults_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new FetchResults_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new FetchResults_resultTupleSchemeFactory()); + } + + private TFetchResultsResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFetchResultsResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(FetchResults_result.class, metaDataMap); + } + + public FetchResults_result() { + } + + public FetchResults_result( + TFetchResultsResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public FetchResults_result(FetchResults_result other) { + if (other.isSetSuccess()) { + this.success = new TFetchResultsResp(other.success); + } + } + + public FetchResults_result deepCopy() { + return new FetchResults_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TFetchResultsResp getSuccess() { + return this.success; + } + + public void setSuccess(TFetchResultsResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TFetchResultsResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof FetchResults_result) + return this.equals((FetchResults_result)that); + return false; + } + + public boolean equals(FetchResults_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(FetchResults_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + FetchResults_result typedOther = (FetchResults_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("FetchResults_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class FetchResults_resultStandardSchemeFactory implements SchemeFactory { + public FetchResults_resultStandardScheme getScheme() { + return new FetchResults_resultStandardScheme(); + } + } + + private static class FetchResults_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, FetchResults_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TFetchResultsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, FetchResults_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class FetchResults_resultTupleSchemeFactory implements SchemeFactory { + public FetchResults_resultTupleScheme getScheme() { + return new FetchResults_resultTupleScheme(); + } + } + + private static class FetchResults_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, FetchResults_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, FetchResults_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TFetchResultsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetDelegationToken_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetDelegationToken_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetDelegationToken_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetDelegationToken_argsTupleSchemeFactory()); + } + + private TGetDelegationTokenReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetDelegationTokenReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetDelegationToken_args.class, metaDataMap); + } + + public GetDelegationToken_args() { + } + + public GetDelegationToken_args( + TGetDelegationTokenReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetDelegationToken_args(GetDelegationToken_args other) { + if (other.isSetReq()) { + this.req = new TGetDelegationTokenReq(other.req); + } + } + + public GetDelegationToken_args deepCopy() { + return new GetDelegationToken_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetDelegationTokenReq getReq() { + return this.req; + } + + public void setReq(TGetDelegationTokenReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetDelegationTokenReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetDelegationToken_args) + return this.equals((GetDelegationToken_args)that); + return false; + } + + public boolean equals(GetDelegationToken_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(GetDelegationToken_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetDelegationToken_args typedOther = (GetDelegationToken_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetDelegationToken_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetDelegationToken_argsStandardSchemeFactory implements SchemeFactory { + public GetDelegationToken_argsStandardScheme getScheme() { + return new GetDelegationToken_argsStandardScheme(); + } + } + + private static class GetDelegationToken_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetDelegationToken_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetDelegationTokenReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetDelegationToken_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetDelegationToken_argsTupleSchemeFactory implements SchemeFactory { + public GetDelegationToken_argsTupleScheme getScheme() { + return new GetDelegationToken_argsTupleScheme(); + } + } + + private static class GetDelegationToken_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetDelegationTokenReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetDelegationToken_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetDelegationToken_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetDelegationToken_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetDelegationToken_resultTupleSchemeFactory()); + } + + private TGetDelegationTokenResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetDelegationTokenResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetDelegationToken_result.class, metaDataMap); + } + + public GetDelegationToken_result() { + } + + public GetDelegationToken_result( + TGetDelegationTokenResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetDelegationToken_result(GetDelegationToken_result other) { + if (other.isSetSuccess()) { + this.success = new TGetDelegationTokenResp(other.success); + } + } + + public GetDelegationToken_result deepCopy() { + return new GetDelegationToken_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetDelegationTokenResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetDelegationTokenResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetDelegationTokenResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetDelegationToken_result) + return this.equals((GetDelegationToken_result)that); + return false; + } + + public boolean equals(GetDelegationToken_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(GetDelegationToken_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + GetDelegationToken_result typedOther = (GetDelegationToken_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetDelegationToken_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetDelegationToken_resultStandardSchemeFactory implements SchemeFactory { + public GetDelegationToken_resultStandardScheme getScheme() { + return new GetDelegationToken_resultStandardScheme(); + } + } + + private static class GetDelegationToken_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetDelegationToken_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetDelegationTokenResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetDelegationToken_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetDelegationToken_resultTupleSchemeFactory implements SchemeFactory { + public GetDelegationToken_resultTupleScheme getScheme() { + return new GetDelegationToken_resultTupleScheme(); + } + } + + private static class GetDelegationToken_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetDelegationTokenResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class CancelDelegationToken_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelDelegationToken_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CancelDelegationToken_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CancelDelegationToken_argsTupleSchemeFactory()); + } + + private TCancelDelegationTokenReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelDelegationTokenReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelDelegationToken_args.class, metaDataMap); + } + + public CancelDelegationToken_args() { + } + + public CancelDelegationToken_args( + TCancelDelegationTokenReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public CancelDelegationToken_args(CancelDelegationToken_args other) { + if (other.isSetReq()) { + this.req = new TCancelDelegationTokenReq(other.req); + } + } + + public CancelDelegationToken_args deepCopy() { + return new CancelDelegationToken_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TCancelDelegationTokenReq getReq() { + return this.req; + } + + public void setReq(TCancelDelegationTokenReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TCancelDelegationTokenReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CancelDelegationToken_args) + return this.equals((CancelDelegationToken_args)that); + return false; + } + + public boolean equals(CancelDelegationToken_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(CancelDelegationToken_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + CancelDelegationToken_args typedOther = (CancelDelegationToken_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CancelDelegationToken_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CancelDelegationToken_argsStandardSchemeFactory implements SchemeFactory { + public CancelDelegationToken_argsStandardScheme getScheme() { + return new CancelDelegationToken_argsStandardScheme(); + } + } + + private static class CancelDelegationToken_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TCancelDelegationTokenReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CancelDelegationToken_argsTupleSchemeFactory implements SchemeFactory { + public CancelDelegationToken_argsTupleScheme getScheme() { + return new CancelDelegationToken_argsTupleScheme(); + } + } + + private static class CancelDelegationToken_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TCancelDelegationTokenReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class CancelDelegationToken_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelDelegationToken_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CancelDelegationToken_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CancelDelegationToken_resultTupleSchemeFactory()); + } + + private TCancelDelegationTokenResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelDelegationTokenResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelDelegationToken_result.class, metaDataMap); + } + + public CancelDelegationToken_result() { + } + + public CancelDelegationToken_result( + TCancelDelegationTokenResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public CancelDelegationToken_result(CancelDelegationToken_result other) { + if (other.isSetSuccess()) { + this.success = new TCancelDelegationTokenResp(other.success); + } + } + + public CancelDelegationToken_result deepCopy() { + return new CancelDelegationToken_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TCancelDelegationTokenResp getSuccess() { + return this.success; + } + + public void setSuccess(TCancelDelegationTokenResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TCancelDelegationTokenResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CancelDelegationToken_result) + return this.equals((CancelDelegationToken_result)that); + return false; + } + + public boolean equals(CancelDelegationToken_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(CancelDelegationToken_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + CancelDelegationToken_result typedOther = (CancelDelegationToken_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CancelDelegationToken_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CancelDelegationToken_resultStandardSchemeFactory implements SchemeFactory { + public CancelDelegationToken_resultStandardScheme getScheme() { + return new CancelDelegationToken_resultStandardScheme(); + } + } + + private static class CancelDelegationToken_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TCancelDelegationTokenResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CancelDelegationToken_resultTupleSchemeFactory implements SchemeFactory { + public CancelDelegationToken_resultTupleScheme getScheme() { + return new CancelDelegationToken_resultTupleScheme(); + } + } + + private static class CancelDelegationToken_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TCancelDelegationTokenResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class RenewDelegationToken_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RenewDelegationToken_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new RenewDelegationToken_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new RenewDelegationToken_argsTupleSchemeFactory()); + } + + private TRenewDelegationTokenReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRenewDelegationTokenReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RenewDelegationToken_args.class, metaDataMap); + } + + public RenewDelegationToken_args() { + } + + public RenewDelegationToken_args( + TRenewDelegationTokenReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public RenewDelegationToken_args(RenewDelegationToken_args other) { + if (other.isSetReq()) { + this.req = new TRenewDelegationTokenReq(other.req); + } + } + + public RenewDelegationToken_args deepCopy() { + return new RenewDelegationToken_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TRenewDelegationTokenReq getReq() { + return this.req; + } + + public void setReq(TRenewDelegationTokenReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TRenewDelegationTokenReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof RenewDelegationToken_args) + return this.equals((RenewDelegationToken_args)that); + return false; + } + + public boolean equals(RenewDelegationToken_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_req = true && (isSetReq()); + builder.append(present_req); + if (present_req) + builder.append(req); + + return builder.toHashCode(); + } + + public int compareTo(RenewDelegationToken_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + RenewDelegationToken_args typedOther = (RenewDelegationToken_args)other; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(typedOther.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, typedOther.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("RenewDelegationToken_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class RenewDelegationToken_argsStandardSchemeFactory implements SchemeFactory { + public RenewDelegationToken_argsStandardScheme getScheme() { + return new RenewDelegationToken_argsStandardScheme(); + } + } + + private static class RenewDelegationToken_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TRenewDelegationTokenReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class RenewDelegationToken_argsTupleSchemeFactory implements SchemeFactory { + public RenewDelegationToken_argsTupleScheme getScheme() { + return new RenewDelegationToken_argsTupleScheme(); + } + } + + private static class RenewDelegationToken_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TRenewDelegationTokenReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class RenewDelegationToken_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RenewDelegationToken_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new RenewDelegationToken_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new RenewDelegationToken_resultTupleSchemeFactory()); + } + + private TRenewDelegationTokenResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRenewDelegationTokenResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RenewDelegationToken_result.class, metaDataMap); + } + + public RenewDelegationToken_result() { + } + + public RenewDelegationToken_result( + TRenewDelegationTokenResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public RenewDelegationToken_result(RenewDelegationToken_result other) { + if (other.isSetSuccess()) { + this.success = new TRenewDelegationTokenResp(other.success); + } + } + + public RenewDelegationToken_result deepCopy() { + return new RenewDelegationToken_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TRenewDelegationTokenResp getSuccess() { + return this.success; + } + + public void setSuccess(TRenewDelegationTokenResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TRenewDelegationTokenResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof RenewDelegationToken_result) + return this.equals((RenewDelegationToken_result)that); + return false; + } + + public boolean equals(RenewDelegationToken_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_success = true && (isSetSuccess()); + builder.append(present_success); + if (present_success) + builder.append(success); + + return builder.toHashCode(); + } + + public int compareTo(RenewDelegationToken_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + RenewDelegationToken_result typedOther = (RenewDelegationToken_result)other; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("RenewDelegationToken_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class RenewDelegationToken_resultStandardSchemeFactory implements SchemeFactory { + public RenewDelegationToken_resultStandardScheme getScheme() { + return new RenewDelegationToken_resultStandardScheme(); + } + } + + private static class RenewDelegationToken_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TRenewDelegationTokenResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class RenewDelegationToken_resultTupleSchemeFactory implements SchemeFactory { + public RenewDelegationToken_resultTupleScheme getScheme() { + return new RenewDelegationToken_resultTupleScheme(); + } + } + + private static class RenewDelegationToken_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TRenewDelegationTokenResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + +} diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java new file mode 100644 index 000000000000..25a38b178428 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java @@ -0,0 +1,103 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TCLIServiceConstants { + + public static final Set PRIMITIVE_TYPES = new HashSet(); + static { + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.BOOLEAN_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.TINYINT_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.SMALLINT_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.INT_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.BIGINT_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.FLOAT_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.DOUBLE_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.STRING_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.TIMESTAMP_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.BINARY_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.DECIMAL_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.NULL_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.DATE_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.VARCHAR_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.CHAR_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.INTERVAL_YEAR_MONTH_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.INTERVAL_DAY_TIME_TYPE); + } + + public static final Set COMPLEX_TYPES = new HashSet(); + static { + COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.ARRAY_TYPE); + COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.MAP_TYPE); + COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.STRUCT_TYPE); + COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.UNION_TYPE); + COMPLEX_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.USER_DEFINED_TYPE); + } + + public static final Set COLLECTION_TYPES = new HashSet(); + static { + COLLECTION_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.ARRAY_TYPE); + COLLECTION_TYPES.add(org.apache.hive.service.cli.thrift.TTypeId.MAP_TYPE); + } + + public static final Map TYPE_NAMES = new HashMap(); + static { + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.BOOLEAN_TYPE, "BOOLEAN"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.TINYINT_TYPE, "TINYINT"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.SMALLINT_TYPE, "SMALLINT"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.INT_TYPE, "INT"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.BIGINT_TYPE, "BIGINT"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.FLOAT_TYPE, "FLOAT"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.DOUBLE_TYPE, "DOUBLE"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.STRING_TYPE, "STRING"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.TIMESTAMP_TYPE, "TIMESTAMP"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.BINARY_TYPE, "BINARY"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.ARRAY_TYPE, "ARRAY"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.MAP_TYPE, "MAP"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.STRUCT_TYPE, "STRUCT"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.UNION_TYPE, "UNIONTYPE"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.DECIMAL_TYPE, "DECIMAL"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.NULL_TYPE, "NULL"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.DATE_TYPE, "DATE"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.VARCHAR_TYPE, "VARCHAR"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.CHAR_TYPE, "CHAR"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.INTERVAL_YEAR_MONTH_TYPE, "INTERVAL_YEAR_MONTH"); + TYPE_NAMES.put(org.apache.hive.service.cli.thrift.TTypeId.INTERVAL_DAY_TIME_TYPE, "INTERVAL_DAY_TIME"); + } + + public static final String CHARACTER_MAXIMUM_LENGTH = "characterMaximumLength"; + + public static final String PRECISION = "precision"; + + public static final String SCALE = "scale"; + +} diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java new file mode 100644 index 000000000000..e23fcdd77a1a --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java @@ -0,0 +1,491 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TCancelDelegationTokenReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelDelegationTokenReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField DELEGATION_TOKEN_FIELD_DESC = new org.apache.thrift.protocol.TField("delegationToken", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCancelDelegationTokenReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCancelDelegationTokenReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String delegationToken; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + DELEGATION_TOKEN((short)2, "delegationToken"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // DELEGATION_TOKEN + return DELEGATION_TOKEN; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.DELEGATION_TOKEN, new org.apache.thrift.meta_data.FieldMetaData("delegationToken", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelDelegationTokenReq.class, metaDataMap); + } + + public TCancelDelegationTokenReq() { + } + + public TCancelDelegationTokenReq( + TSessionHandle sessionHandle, + String delegationToken) + { + this(); + this.sessionHandle = sessionHandle; + this.delegationToken = delegationToken; + } + + /** + * Performs a deep copy on other. + */ + public TCancelDelegationTokenReq(TCancelDelegationTokenReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetDelegationToken()) { + this.delegationToken = other.delegationToken; + } + } + + public TCancelDelegationTokenReq deepCopy() { + return new TCancelDelegationTokenReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.delegationToken = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getDelegationToken() { + return this.delegationToken; + } + + public void setDelegationToken(String delegationToken) { + this.delegationToken = delegationToken; + } + + public void unsetDelegationToken() { + this.delegationToken = null; + } + + /** Returns true if field delegationToken is set (has been assigned a value) and false otherwise */ + public boolean isSetDelegationToken() { + return this.delegationToken != null; + } + + public void setDelegationTokenIsSet(boolean value) { + if (!value) { + this.delegationToken = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case DELEGATION_TOKEN: + if (value == null) { + unsetDelegationToken(); + } else { + setDelegationToken((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case DELEGATION_TOKEN: + return getDelegationToken(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case DELEGATION_TOKEN: + return isSetDelegationToken(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCancelDelegationTokenReq) + return this.equals((TCancelDelegationTokenReq)that); + return false; + } + + public boolean equals(TCancelDelegationTokenReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_delegationToken = true && this.isSetDelegationToken(); + boolean that_present_delegationToken = true && that.isSetDelegationToken(); + if (this_present_delegationToken || that_present_delegationToken) { + if (!(this_present_delegationToken && that_present_delegationToken)) + return false; + if (!this.delegationToken.equals(that.delegationToken)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + builder.append(present_sessionHandle); + if (present_sessionHandle) + builder.append(sessionHandle); + + boolean present_delegationToken = true && (isSetDelegationToken()); + builder.append(present_delegationToken); + if (present_delegationToken) + builder.append(delegationToken); + + return builder.toHashCode(); + } + + public int compareTo(TCancelDelegationTokenReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TCancelDelegationTokenReq typedOther = (TCancelDelegationTokenReq)other; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetDelegationToken()).compareTo(typedOther.isSetDelegationToken()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDelegationToken()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.delegationToken, typedOther.delegationToken); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCancelDelegationTokenReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (!first) sb.append(", "); + sb.append("delegationToken:"); + if (this.delegationToken == null) { + sb.append("null"); + } else { + sb.append(this.delegationToken); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + if (!isSetDelegationToken()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'delegationToken' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCancelDelegationTokenReqStandardSchemeFactory implements SchemeFactory { + public TCancelDelegationTokenReqStandardScheme getScheme() { + return new TCancelDelegationTokenReqStandardScheme(); + } + } + + private static class TCancelDelegationTokenReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // DELEGATION_TOKEN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.delegationToken = iprot.readString(); + struct.setDelegationTokenIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.delegationToken != null) { + oprot.writeFieldBegin(DELEGATION_TOKEN_FIELD_DESC); + oprot.writeString(struct.delegationToken); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCancelDelegationTokenReqTupleSchemeFactory implements SchemeFactory { + public TCancelDelegationTokenReqTupleScheme getScheme() { + return new TCancelDelegationTokenReqTupleScheme(); + } + } + + private static class TCancelDelegationTokenReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + oprot.writeString(struct.delegationToken); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + struct.delegationToken = iprot.readString(); + struct.setDelegationTokenIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java new file mode 100644 index 000000000000..77c9ee77ec59 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TCancelDelegationTokenResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelDelegationTokenResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCancelDelegationTokenRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCancelDelegationTokenRespTupleSchemeFactory()); + } + + private TStatus status; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelDelegationTokenResp.class, metaDataMap); + } + + public TCancelDelegationTokenResp() { + } + + public TCancelDelegationTokenResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TCancelDelegationTokenResp(TCancelDelegationTokenResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + } + + public TCancelDelegationTokenResp deepCopy() { + return new TCancelDelegationTokenResp(this); + } + + @Override + public void clear() { + this.status = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCancelDelegationTokenResp) + return this.equals((TCancelDelegationTokenResp)that); + return false; + } + + public boolean equals(TCancelDelegationTokenResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + return builder.toHashCode(); + } + + public int compareTo(TCancelDelegationTokenResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TCancelDelegationTokenResp typedOther = (TCancelDelegationTokenResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCancelDelegationTokenResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCancelDelegationTokenRespStandardSchemeFactory implements SchemeFactory { + public TCancelDelegationTokenRespStandardScheme getScheme() { + return new TCancelDelegationTokenRespStandardScheme(); + } + } + + private static class TCancelDelegationTokenRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCancelDelegationTokenRespTupleSchemeFactory implements SchemeFactory { + public TCancelDelegationTokenRespTupleScheme getScheme() { + return new TCancelDelegationTokenRespTupleScheme(); + } + } + + private static class TCancelDelegationTokenRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java new file mode 100644 index 000000000000..45eac48ab12d --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TCancelOperationReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelOperationReq"); + + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCancelOperationReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCancelOperationReqTupleSchemeFactory()); + } + + private TOperationHandle operationHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OPERATION_HANDLE((short)1, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelOperationReq.class, metaDataMap); + } + + public TCancelOperationReq() { + } + + public TCancelOperationReq( + TOperationHandle operationHandle) + { + this(); + this.operationHandle = operationHandle; + } + + /** + * Performs a deep copy on other. + */ + public TCancelOperationReq(TCancelOperationReq other) { + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TCancelOperationReq deepCopy() { + return new TCancelOperationReq(this); + } + + @Override + public void clear() { + this.operationHandle = null; + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCancelOperationReq) + return this.equals((TCancelOperationReq)that); + return false; + } + + public boolean equals(TCancelOperationReq that) { + if (that == null) + return false; + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_operationHandle = true && (isSetOperationHandle()); + builder.append(present_operationHandle); + if (present_operationHandle) + builder.append(operationHandle); + + return builder.toHashCode(); + } + + public int compareTo(TCancelOperationReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TCancelOperationReq typedOther = (TCancelOperationReq)other; + + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCancelOperationReq("); + boolean first = true; + + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetOperationHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCancelOperationReqStandardSchemeFactory implements SchemeFactory { + public TCancelOperationReqStandardScheme getScheme() { + return new TCancelOperationReqStandardScheme(); + } + } + + private static class TCancelOperationReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelOperationReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelOperationReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.operationHandle != null) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCancelOperationReqTupleSchemeFactory implements SchemeFactory { + public TCancelOperationReqTupleScheme getScheme() { + return new TCancelOperationReqTupleScheme(); + } + } + + private static class TCancelOperationReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCancelOperationReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.operationHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCancelOperationReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java new file mode 100644 index 000000000000..2a39414d601a --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TCancelOperationResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelOperationResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCancelOperationRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCancelOperationRespTupleSchemeFactory()); + } + + private TStatus status; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelOperationResp.class, metaDataMap); + } + + public TCancelOperationResp() { + } + + public TCancelOperationResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TCancelOperationResp(TCancelOperationResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + } + + public TCancelOperationResp deepCopy() { + return new TCancelOperationResp(this); + } + + @Override + public void clear() { + this.status = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCancelOperationResp) + return this.equals((TCancelOperationResp)that); + return false; + } + + public boolean equals(TCancelOperationResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + return builder.toHashCode(); + } + + public int compareTo(TCancelOperationResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TCancelOperationResp typedOther = (TCancelOperationResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCancelOperationResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCancelOperationRespStandardSchemeFactory implements SchemeFactory { + public TCancelOperationRespStandardScheme getScheme() { + return new TCancelOperationRespStandardScheme(); + } + } + + private static class TCancelOperationRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelOperationResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelOperationResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCancelOperationRespTupleSchemeFactory implements SchemeFactory { + public TCancelOperationRespTupleScheme getScheme() { + return new TCancelOperationRespTupleScheme(); + } + } + + private static class TCancelOperationRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCancelOperationResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCancelOperationResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java new file mode 100644 index 000000000000..0cbb7ccced07 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TCloseOperationReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseOperationReq"); + + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCloseOperationReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCloseOperationReqTupleSchemeFactory()); + } + + private TOperationHandle operationHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OPERATION_HANDLE((short)1, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseOperationReq.class, metaDataMap); + } + + public TCloseOperationReq() { + } + + public TCloseOperationReq( + TOperationHandle operationHandle) + { + this(); + this.operationHandle = operationHandle; + } + + /** + * Performs a deep copy on other. + */ + public TCloseOperationReq(TCloseOperationReq other) { + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TCloseOperationReq deepCopy() { + return new TCloseOperationReq(this); + } + + @Override + public void clear() { + this.operationHandle = null; + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCloseOperationReq) + return this.equals((TCloseOperationReq)that); + return false; + } + + public boolean equals(TCloseOperationReq that) { + if (that == null) + return false; + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_operationHandle = true && (isSetOperationHandle()); + builder.append(present_operationHandle); + if (present_operationHandle) + builder.append(operationHandle); + + return builder.toHashCode(); + } + + public int compareTo(TCloseOperationReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TCloseOperationReq typedOther = (TCloseOperationReq)other; + + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCloseOperationReq("); + boolean first = true; + + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetOperationHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCloseOperationReqStandardSchemeFactory implements SchemeFactory { + public TCloseOperationReqStandardScheme getScheme() { + return new TCloseOperationReqStandardScheme(); + } + } + + private static class TCloseOperationReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseOperationReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseOperationReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.operationHandle != null) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCloseOperationReqTupleSchemeFactory implements SchemeFactory { + public TCloseOperationReqTupleScheme getScheme() { + return new TCloseOperationReqTupleScheme(); + } + } + + private static class TCloseOperationReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCloseOperationReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.operationHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCloseOperationReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java new file mode 100644 index 000000000000..7334d67173d7 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TCloseOperationResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseOperationResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCloseOperationRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCloseOperationRespTupleSchemeFactory()); + } + + private TStatus status; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseOperationResp.class, metaDataMap); + } + + public TCloseOperationResp() { + } + + public TCloseOperationResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TCloseOperationResp(TCloseOperationResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + } + + public TCloseOperationResp deepCopy() { + return new TCloseOperationResp(this); + } + + @Override + public void clear() { + this.status = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCloseOperationResp) + return this.equals((TCloseOperationResp)that); + return false; + } + + public boolean equals(TCloseOperationResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + return builder.toHashCode(); + } + + public int compareTo(TCloseOperationResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TCloseOperationResp typedOther = (TCloseOperationResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCloseOperationResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCloseOperationRespStandardSchemeFactory implements SchemeFactory { + public TCloseOperationRespStandardScheme getScheme() { + return new TCloseOperationRespStandardScheme(); + } + } + + private static class TCloseOperationRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseOperationResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseOperationResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCloseOperationRespTupleSchemeFactory implements SchemeFactory { + public TCloseOperationRespTupleScheme getScheme() { + return new TCloseOperationRespTupleScheme(); + } + } + + private static class TCloseOperationRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCloseOperationResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCloseOperationResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java new file mode 100644 index 000000000000..027e8295436b --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TCloseSessionReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseSessionReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCloseSessionReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCloseSessionReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseSessionReq.class, metaDataMap); + } + + public TCloseSessionReq() { + } + + public TCloseSessionReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TCloseSessionReq(TCloseSessionReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + } + + public TCloseSessionReq deepCopy() { + return new TCloseSessionReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCloseSessionReq) + return this.equals((TCloseSessionReq)that); + return false; + } + + public boolean equals(TCloseSessionReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + builder.append(present_sessionHandle); + if (present_sessionHandle) + builder.append(sessionHandle); + + return builder.toHashCode(); + } + + public int compareTo(TCloseSessionReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TCloseSessionReq typedOther = (TCloseSessionReq)other; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCloseSessionReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCloseSessionReqStandardSchemeFactory implements SchemeFactory { + public TCloseSessionReqStandardScheme getScheme() { + return new TCloseSessionReqStandardScheme(); + } + } + + private static class TCloseSessionReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseSessionReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseSessionReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCloseSessionReqTupleSchemeFactory implements SchemeFactory { + public TCloseSessionReqTupleScheme getScheme() { + return new TCloseSessionReqTupleScheme(); + } + } + + private static class TCloseSessionReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCloseSessionReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCloseSessionReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java new file mode 100644 index 000000000000..168c8fc775e3 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TCloseSessionResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseSessionResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCloseSessionRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCloseSessionRespTupleSchemeFactory()); + } + + private TStatus status; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseSessionResp.class, metaDataMap); + } + + public TCloseSessionResp() { + } + + public TCloseSessionResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TCloseSessionResp(TCloseSessionResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + } + + public TCloseSessionResp deepCopy() { + return new TCloseSessionResp(this); + } + + @Override + public void clear() { + this.status = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCloseSessionResp) + return this.equals((TCloseSessionResp)that); + return false; + } + + public boolean equals(TCloseSessionResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + return builder.toHashCode(); + } + + public int compareTo(TCloseSessionResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TCloseSessionResp typedOther = (TCloseSessionResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCloseSessionResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCloseSessionRespStandardSchemeFactory implements SchemeFactory { + public TCloseSessionRespStandardScheme getScheme() { + return new TCloseSessionRespStandardScheme(); + } + } + + private static class TCloseSessionRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseSessionResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseSessionResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCloseSessionRespTupleSchemeFactory implements SchemeFactory { + public TCloseSessionRespTupleScheme getScheme() { + return new TCloseSessionRespTupleScheme(); + } + } + + private static class TCloseSessionRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCloseSessionResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCloseSessionResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java new file mode 100644 index 000000000000..bfe50c7810f7 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java @@ -0,0 +1,732 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TColumn extends org.apache.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn"); + private static final org.apache.thrift.protocol.TField BOOL_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("boolVal", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField BYTE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("byteVal", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField I16_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i16Val", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField I32_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i32Val", org.apache.thrift.protocol.TType.STRUCT, (short)4); + private static final org.apache.thrift.protocol.TField I64_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i64Val", org.apache.thrift.protocol.TType.STRUCT, (short)5); + private static final org.apache.thrift.protocol.TField DOUBLE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("doubleVal", org.apache.thrift.protocol.TType.STRUCT, (short)6); + private static final org.apache.thrift.protocol.TField STRING_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("stringVal", org.apache.thrift.protocol.TType.STRUCT, (short)7); + private static final org.apache.thrift.protocol.TField BINARY_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("binaryVal", org.apache.thrift.protocol.TType.STRUCT, (short)8); + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + BOOL_VAL((short)1, "boolVal"), + BYTE_VAL((short)2, "byteVal"), + I16_VAL((short)3, "i16Val"), + I32_VAL((short)4, "i32Val"), + I64_VAL((short)5, "i64Val"), + DOUBLE_VAL((short)6, "doubleVal"), + STRING_VAL((short)7, "stringVal"), + BINARY_VAL((short)8, "binaryVal"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // BOOL_VAL + return BOOL_VAL; + case 2: // BYTE_VAL + return BYTE_VAL; + case 3: // I16_VAL + return I16_VAL; + case 4: // I32_VAL + return I32_VAL; + case 5: // I64_VAL + return I64_VAL; + case 6: // DOUBLE_VAL + return DOUBLE_VAL; + case 7: // STRING_VAL + return STRING_VAL; + case 8: // BINARY_VAL + return BINARY_VAL; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.BOOL_VAL, new org.apache.thrift.meta_data.FieldMetaData("boolVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TBoolColumn.class))); + tmpMap.put(_Fields.BYTE_VAL, new org.apache.thrift.meta_data.FieldMetaData("byteVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TByteColumn.class))); + tmpMap.put(_Fields.I16_VAL, new org.apache.thrift.meta_data.FieldMetaData("i16Val", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI16Column.class))); + tmpMap.put(_Fields.I32_VAL, new org.apache.thrift.meta_data.FieldMetaData("i32Val", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI32Column.class))); + tmpMap.put(_Fields.I64_VAL, new org.apache.thrift.meta_data.FieldMetaData("i64Val", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI64Column.class))); + tmpMap.put(_Fields.DOUBLE_VAL, new org.apache.thrift.meta_data.FieldMetaData("doubleVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TDoubleColumn.class))); + tmpMap.put(_Fields.STRING_VAL, new org.apache.thrift.meta_data.FieldMetaData("stringVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStringColumn.class))); + tmpMap.put(_Fields.BINARY_VAL, new org.apache.thrift.meta_data.FieldMetaData("binaryVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TBinaryColumn.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumn.class, metaDataMap); + } + + public TColumn() { + super(); + } + + public TColumn(_Fields setField, Object value) { + super(setField, value); + } + + public TColumn(TColumn other) { + super(other); + } + public TColumn deepCopy() { + return new TColumn(this); + } + + public static TColumn boolVal(TBoolColumn value) { + TColumn x = new TColumn(); + x.setBoolVal(value); + return x; + } + + public static TColumn byteVal(TByteColumn value) { + TColumn x = new TColumn(); + x.setByteVal(value); + return x; + } + + public static TColumn i16Val(TI16Column value) { + TColumn x = new TColumn(); + x.setI16Val(value); + return x; + } + + public static TColumn i32Val(TI32Column value) { + TColumn x = new TColumn(); + x.setI32Val(value); + return x; + } + + public static TColumn i64Val(TI64Column value) { + TColumn x = new TColumn(); + x.setI64Val(value); + return x; + } + + public static TColumn doubleVal(TDoubleColumn value) { + TColumn x = new TColumn(); + x.setDoubleVal(value); + return x; + } + + public static TColumn stringVal(TStringColumn value) { + TColumn x = new TColumn(); + x.setStringVal(value); + return x; + } + + public static TColumn binaryVal(TBinaryColumn value) { + TColumn x = new TColumn(); + x.setBinaryVal(value); + return x; + } + + + @Override + protected void checkType(_Fields setField, Object value) throws ClassCastException { + switch (setField) { + case BOOL_VAL: + if (value instanceof TBoolColumn) { + break; + } + throw new ClassCastException("Was expecting value of type TBoolColumn for field 'boolVal', but got " + value.getClass().getSimpleName()); + case BYTE_VAL: + if (value instanceof TByteColumn) { + break; + } + throw new ClassCastException("Was expecting value of type TByteColumn for field 'byteVal', but got " + value.getClass().getSimpleName()); + case I16_VAL: + if (value instanceof TI16Column) { + break; + } + throw new ClassCastException("Was expecting value of type TI16Column for field 'i16Val', but got " + value.getClass().getSimpleName()); + case I32_VAL: + if (value instanceof TI32Column) { + break; + } + throw new ClassCastException("Was expecting value of type TI32Column for field 'i32Val', but got " + value.getClass().getSimpleName()); + case I64_VAL: + if (value instanceof TI64Column) { + break; + } + throw new ClassCastException("Was expecting value of type TI64Column for field 'i64Val', but got " + value.getClass().getSimpleName()); + case DOUBLE_VAL: + if (value instanceof TDoubleColumn) { + break; + } + throw new ClassCastException("Was expecting value of type TDoubleColumn for field 'doubleVal', but got " + value.getClass().getSimpleName()); + case STRING_VAL: + if (value instanceof TStringColumn) { + break; + } + throw new ClassCastException("Was expecting value of type TStringColumn for field 'stringVal', but got " + value.getClass().getSimpleName()); + case BINARY_VAL: + if (value instanceof TBinaryColumn) { + break; + } + throw new ClassCastException("Was expecting value of type TBinaryColumn for field 'binaryVal', but got " + value.getClass().getSimpleName()); + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(field.id); + if (setField != null) { + switch (setField) { + case BOOL_VAL: + if (field.type == BOOL_VAL_FIELD_DESC.type) { + TBoolColumn boolVal; + boolVal = new TBoolColumn(); + boolVal.read(iprot); + return boolVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case BYTE_VAL: + if (field.type == BYTE_VAL_FIELD_DESC.type) { + TByteColumn byteVal; + byteVal = new TByteColumn(); + byteVal.read(iprot); + return byteVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case I16_VAL: + if (field.type == I16_VAL_FIELD_DESC.type) { + TI16Column i16Val; + i16Val = new TI16Column(); + i16Val.read(iprot); + return i16Val; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case I32_VAL: + if (field.type == I32_VAL_FIELD_DESC.type) { + TI32Column i32Val; + i32Val = new TI32Column(); + i32Val.read(iprot); + return i32Val; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case I64_VAL: + if (field.type == I64_VAL_FIELD_DESC.type) { + TI64Column i64Val; + i64Val = new TI64Column(); + i64Val.read(iprot); + return i64Val; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case DOUBLE_VAL: + if (field.type == DOUBLE_VAL_FIELD_DESC.type) { + TDoubleColumn doubleVal; + doubleVal = new TDoubleColumn(); + doubleVal.read(iprot); + return doubleVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case STRING_VAL: + if (field.type == STRING_VAL_FIELD_DESC.type) { + TStringColumn stringVal; + stringVal = new TStringColumn(); + stringVal.read(iprot); + return stringVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case BINARY_VAL: + if (field.type == BINARY_VAL_FIELD_DESC.type) { + TBinaryColumn binaryVal; + binaryVal = new TBinaryColumn(); + binaryVal.read(iprot); + return binaryVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case BOOL_VAL: + TBoolColumn boolVal = (TBoolColumn)value_; + boolVal.write(oprot); + return; + case BYTE_VAL: + TByteColumn byteVal = (TByteColumn)value_; + byteVal.write(oprot); + return; + case I16_VAL: + TI16Column i16Val = (TI16Column)value_; + i16Val.write(oprot); + return; + case I32_VAL: + TI32Column i32Val = (TI32Column)value_; + i32Val.write(oprot); + return; + case I64_VAL: + TI64Column i64Val = (TI64Column)value_; + i64Val.write(oprot); + return; + case DOUBLE_VAL: + TDoubleColumn doubleVal = (TDoubleColumn)value_; + doubleVal.write(oprot); + return; + case STRING_VAL: + TStringColumn stringVal = (TStringColumn)value_; + stringVal.write(oprot); + return; + case BINARY_VAL: + TBinaryColumn binaryVal = (TBinaryColumn)value_; + binaryVal.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(fieldID); + if (setField != null) { + switch (setField) { + case BOOL_VAL: + TBoolColumn boolVal; + boolVal = new TBoolColumn(); + boolVal.read(iprot); + return boolVal; + case BYTE_VAL: + TByteColumn byteVal; + byteVal = new TByteColumn(); + byteVal.read(iprot); + return byteVal; + case I16_VAL: + TI16Column i16Val; + i16Val = new TI16Column(); + i16Val.read(iprot); + return i16Val; + case I32_VAL: + TI32Column i32Val; + i32Val = new TI32Column(); + i32Val.read(iprot); + return i32Val; + case I64_VAL: + TI64Column i64Val; + i64Val = new TI64Column(); + i64Val.read(iprot); + return i64Val; + case DOUBLE_VAL: + TDoubleColumn doubleVal; + doubleVal = new TDoubleColumn(); + doubleVal.read(iprot); + return doubleVal; + case STRING_VAL: + TStringColumn stringVal; + stringVal = new TStringColumn(); + stringVal.read(iprot); + return stringVal; + case BINARY_VAL: + TBinaryColumn binaryVal; + binaryVal = new TBinaryColumn(); + binaryVal.read(iprot); + return binaryVal; + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + throw new TProtocolException("Couldn't find a field with field id " + fieldID); + } + } + + @Override + protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case BOOL_VAL: + TBoolColumn boolVal = (TBoolColumn)value_; + boolVal.write(oprot); + return; + case BYTE_VAL: + TByteColumn byteVal = (TByteColumn)value_; + byteVal.write(oprot); + return; + case I16_VAL: + TI16Column i16Val = (TI16Column)value_; + i16Val.write(oprot); + return; + case I32_VAL: + TI32Column i32Val = (TI32Column)value_; + i32Val.write(oprot); + return; + case I64_VAL: + TI64Column i64Val = (TI64Column)value_; + i64Val.write(oprot); + return; + case DOUBLE_VAL: + TDoubleColumn doubleVal = (TDoubleColumn)value_; + doubleVal.write(oprot); + return; + case STRING_VAL: + TStringColumn stringVal = (TStringColumn)value_; + stringVal.write(oprot); + return; + case BINARY_VAL: + TBinaryColumn binaryVal = (TBinaryColumn)value_; + binaryVal.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { + switch (setField) { + case BOOL_VAL: + return BOOL_VAL_FIELD_DESC; + case BYTE_VAL: + return BYTE_VAL_FIELD_DESC; + case I16_VAL: + return I16_VAL_FIELD_DESC; + case I32_VAL: + return I32_VAL_FIELD_DESC; + case I64_VAL: + return I64_VAL_FIELD_DESC; + case DOUBLE_VAL: + return DOUBLE_VAL_FIELD_DESC; + case STRING_VAL: + return STRING_VAL_FIELD_DESC; + case BINARY_VAL: + return BINARY_VAL_FIELD_DESC; + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected org.apache.thrift.protocol.TStruct getStructDesc() { + return STRUCT_DESC; + } + + @Override + protected _Fields enumForId(short id) { + return _Fields.findByThriftIdOrThrow(id); + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + + public TBoolColumn getBoolVal() { + if (getSetField() == _Fields.BOOL_VAL) { + return (TBoolColumn)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'boolVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setBoolVal(TBoolColumn value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.BOOL_VAL; + value_ = value; + } + + public TByteColumn getByteVal() { + if (getSetField() == _Fields.BYTE_VAL) { + return (TByteColumn)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'byteVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setByteVal(TByteColumn value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.BYTE_VAL; + value_ = value; + } + + public TI16Column getI16Val() { + if (getSetField() == _Fields.I16_VAL) { + return (TI16Column)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'i16Val' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setI16Val(TI16Column value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.I16_VAL; + value_ = value; + } + + public TI32Column getI32Val() { + if (getSetField() == _Fields.I32_VAL) { + return (TI32Column)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'i32Val' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setI32Val(TI32Column value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.I32_VAL; + value_ = value; + } + + public TI64Column getI64Val() { + if (getSetField() == _Fields.I64_VAL) { + return (TI64Column)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'i64Val' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setI64Val(TI64Column value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.I64_VAL; + value_ = value; + } + + public TDoubleColumn getDoubleVal() { + if (getSetField() == _Fields.DOUBLE_VAL) { + return (TDoubleColumn)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'doubleVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setDoubleVal(TDoubleColumn value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.DOUBLE_VAL; + value_ = value; + } + + public TStringColumn getStringVal() { + if (getSetField() == _Fields.STRING_VAL) { + return (TStringColumn)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'stringVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setStringVal(TStringColumn value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.STRING_VAL; + value_ = value; + } + + public TBinaryColumn getBinaryVal() { + if (getSetField() == _Fields.BINARY_VAL) { + return (TBinaryColumn)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'binaryVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setBinaryVal(TBinaryColumn value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.BINARY_VAL; + value_ = value; + } + + public boolean isSetBoolVal() { + return setField_ == _Fields.BOOL_VAL; + } + + + public boolean isSetByteVal() { + return setField_ == _Fields.BYTE_VAL; + } + + + public boolean isSetI16Val() { + return setField_ == _Fields.I16_VAL; + } + + + public boolean isSetI32Val() { + return setField_ == _Fields.I32_VAL; + } + + + public boolean isSetI64Val() { + return setField_ == _Fields.I64_VAL; + } + + + public boolean isSetDoubleVal() { + return setField_ == _Fields.DOUBLE_VAL; + } + + + public boolean isSetStringVal() { + return setField_ == _Fields.STRING_VAL; + } + + + public boolean isSetBinaryVal() { + return setField_ == _Fields.BINARY_VAL; + } + + + public boolean equals(Object other) { + if (other instanceof TColumn) { + return equals((TColumn)other); + } else { + return false; + } + } + + public boolean equals(TColumn other) { + return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); + } + + @Override + public int compareTo(TColumn other) { + int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + HashCodeBuilder hcb = new HashCodeBuilder(); + hcb.append(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + hcb.append(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); + } else { + hcb.append(value); + } + } + return hcb.toHashCode(); + } + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + +} diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java new file mode 100644 index 000000000000..247db6489457 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java @@ -0,0 +1,700 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TColumnDesc implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnDesc"); + + private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("columnName", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField TYPE_DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("typeDesc", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField POSITION_FIELD_DESC = new org.apache.thrift.protocol.TField("position", org.apache.thrift.protocol.TType.I32, (short)3); + private static final org.apache.thrift.protocol.TField COMMENT_FIELD_DESC = new org.apache.thrift.protocol.TField("comment", org.apache.thrift.protocol.TType.STRING, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TColumnDescStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TColumnDescTupleSchemeFactory()); + } + + private String columnName; // required + private TTypeDesc typeDesc; // required + private int position; // required + private String comment; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + COLUMN_NAME((short)1, "columnName"), + TYPE_DESC((short)2, "typeDesc"), + POSITION((short)3, "position"), + COMMENT((short)4, "comment"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // COLUMN_NAME + return COLUMN_NAME; + case 2: // TYPE_DESC + return TYPE_DESC; + case 3: // POSITION + return POSITION; + case 4: // COMMENT + return COMMENT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __POSITION_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.COMMENT}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.COLUMN_NAME, new org.apache.thrift.meta_data.FieldMetaData("columnName", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TYPE_DESC, new org.apache.thrift.meta_data.FieldMetaData("typeDesc", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeDesc.class))); + tmpMap.put(_Fields.POSITION, new org.apache.thrift.meta_data.FieldMetaData("position", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.COMMENT, new org.apache.thrift.meta_data.FieldMetaData("comment", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumnDesc.class, metaDataMap); + } + + public TColumnDesc() { + } + + public TColumnDesc( + String columnName, + TTypeDesc typeDesc, + int position) + { + this(); + this.columnName = columnName; + this.typeDesc = typeDesc; + this.position = position; + setPositionIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TColumnDesc(TColumnDesc other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetColumnName()) { + this.columnName = other.columnName; + } + if (other.isSetTypeDesc()) { + this.typeDesc = new TTypeDesc(other.typeDesc); + } + this.position = other.position; + if (other.isSetComment()) { + this.comment = other.comment; + } + } + + public TColumnDesc deepCopy() { + return new TColumnDesc(this); + } + + @Override + public void clear() { + this.columnName = null; + this.typeDesc = null; + setPositionIsSet(false); + this.position = 0; + this.comment = null; + } + + public String getColumnName() { + return this.columnName; + } + + public void setColumnName(String columnName) { + this.columnName = columnName; + } + + public void unsetColumnName() { + this.columnName = null; + } + + /** Returns true if field columnName is set (has been assigned a value) and false otherwise */ + public boolean isSetColumnName() { + return this.columnName != null; + } + + public void setColumnNameIsSet(boolean value) { + if (!value) { + this.columnName = null; + } + } + + public TTypeDesc getTypeDesc() { + return this.typeDesc; + } + + public void setTypeDesc(TTypeDesc typeDesc) { + this.typeDesc = typeDesc; + } + + public void unsetTypeDesc() { + this.typeDesc = null; + } + + /** Returns true if field typeDesc is set (has been assigned a value) and false otherwise */ + public boolean isSetTypeDesc() { + return this.typeDesc != null; + } + + public void setTypeDescIsSet(boolean value) { + if (!value) { + this.typeDesc = null; + } + } + + public int getPosition() { + return this.position; + } + + public void setPosition(int position) { + this.position = position; + setPositionIsSet(true); + } + + public void unsetPosition() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __POSITION_ISSET_ID); + } + + /** Returns true if field position is set (has been assigned a value) and false otherwise */ + public boolean isSetPosition() { + return EncodingUtils.testBit(__isset_bitfield, __POSITION_ISSET_ID); + } + + public void setPositionIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __POSITION_ISSET_ID, value); + } + + public String getComment() { + return this.comment; + } + + public void setComment(String comment) { + this.comment = comment; + } + + public void unsetComment() { + this.comment = null; + } + + /** Returns true if field comment is set (has been assigned a value) and false otherwise */ + public boolean isSetComment() { + return this.comment != null; + } + + public void setCommentIsSet(boolean value) { + if (!value) { + this.comment = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case COLUMN_NAME: + if (value == null) { + unsetColumnName(); + } else { + setColumnName((String)value); + } + break; + + case TYPE_DESC: + if (value == null) { + unsetTypeDesc(); + } else { + setTypeDesc((TTypeDesc)value); + } + break; + + case POSITION: + if (value == null) { + unsetPosition(); + } else { + setPosition((Integer)value); + } + break; + + case COMMENT: + if (value == null) { + unsetComment(); + } else { + setComment((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case COLUMN_NAME: + return getColumnName(); + + case TYPE_DESC: + return getTypeDesc(); + + case POSITION: + return Integer.valueOf(getPosition()); + + case COMMENT: + return getComment(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case COLUMN_NAME: + return isSetColumnName(); + case TYPE_DESC: + return isSetTypeDesc(); + case POSITION: + return isSetPosition(); + case COMMENT: + return isSetComment(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TColumnDesc) + return this.equals((TColumnDesc)that); + return false; + } + + public boolean equals(TColumnDesc that) { + if (that == null) + return false; + + boolean this_present_columnName = true && this.isSetColumnName(); + boolean that_present_columnName = true && that.isSetColumnName(); + if (this_present_columnName || that_present_columnName) { + if (!(this_present_columnName && that_present_columnName)) + return false; + if (!this.columnName.equals(that.columnName)) + return false; + } + + boolean this_present_typeDesc = true && this.isSetTypeDesc(); + boolean that_present_typeDesc = true && that.isSetTypeDesc(); + if (this_present_typeDesc || that_present_typeDesc) { + if (!(this_present_typeDesc && that_present_typeDesc)) + return false; + if (!this.typeDesc.equals(that.typeDesc)) + return false; + } + + boolean this_present_position = true; + boolean that_present_position = true; + if (this_present_position || that_present_position) { + if (!(this_present_position && that_present_position)) + return false; + if (this.position != that.position) + return false; + } + + boolean this_present_comment = true && this.isSetComment(); + boolean that_present_comment = true && that.isSetComment(); + if (this_present_comment || that_present_comment) { + if (!(this_present_comment && that_present_comment)) + return false; + if (!this.comment.equals(that.comment)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_columnName = true && (isSetColumnName()); + builder.append(present_columnName); + if (present_columnName) + builder.append(columnName); + + boolean present_typeDesc = true && (isSetTypeDesc()); + builder.append(present_typeDesc); + if (present_typeDesc) + builder.append(typeDesc); + + boolean present_position = true; + builder.append(present_position); + if (present_position) + builder.append(position); + + boolean present_comment = true && (isSetComment()); + builder.append(present_comment); + if (present_comment) + builder.append(comment); + + return builder.toHashCode(); + } + + public int compareTo(TColumnDesc other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TColumnDesc typedOther = (TColumnDesc)other; + + lastComparison = Boolean.valueOf(isSetColumnName()).compareTo(typedOther.isSetColumnName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetColumnName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnName, typedOther.columnName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTypeDesc()).compareTo(typedOther.isSetTypeDesc()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTypeDesc()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.typeDesc, typedOther.typeDesc); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetPosition()).compareTo(typedOther.isSetPosition()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPosition()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.position, typedOther.position); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetComment()).compareTo(typedOther.isSetComment()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetComment()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.comment, typedOther.comment); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TColumnDesc("); + boolean first = true; + + sb.append("columnName:"); + if (this.columnName == null) { + sb.append("null"); + } else { + sb.append(this.columnName); + } + first = false; + if (!first) sb.append(", "); + sb.append("typeDesc:"); + if (this.typeDesc == null) { + sb.append("null"); + } else { + sb.append(this.typeDesc); + } + first = false; + if (!first) sb.append(", "); + sb.append("position:"); + sb.append(this.position); + first = false; + if (isSetComment()) { + if (!first) sb.append(", "); + sb.append("comment:"); + if (this.comment == null) { + sb.append("null"); + } else { + sb.append(this.comment); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetColumnName()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'columnName' is unset! Struct:" + toString()); + } + + if (!isSetTypeDesc()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'typeDesc' is unset! Struct:" + toString()); + } + + if (!isSetPosition()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'position' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (typeDesc != null) { + typeDesc.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TColumnDescStandardSchemeFactory implements SchemeFactory { + public TColumnDescStandardScheme getScheme() { + return new TColumnDescStandardScheme(); + } + } + + private static class TColumnDescStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TColumnDesc struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // COLUMN_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.columnName = iprot.readString(); + struct.setColumnNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TYPE_DESC + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.typeDesc = new TTypeDesc(); + struct.typeDesc.read(iprot); + struct.setTypeDescIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // POSITION + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.position = iprot.readI32(); + struct.setPositionIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // COMMENT + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.comment = iprot.readString(); + struct.setCommentIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TColumnDesc struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.columnName != null) { + oprot.writeFieldBegin(COLUMN_NAME_FIELD_DESC); + oprot.writeString(struct.columnName); + oprot.writeFieldEnd(); + } + if (struct.typeDesc != null) { + oprot.writeFieldBegin(TYPE_DESC_FIELD_DESC); + struct.typeDesc.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(POSITION_FIELD_DESC); + oprot.writeI32(struct.position); + oprot.writeFieldEnd(); + if (struct.comment != null) { + if (struct.isSetComment()) { + oprot.writeFieldBegin(COMMENT_FIELD_DESC); + oprot.writeString(struct.comment); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TColumnDescTupleSchemeFactory implements SchemeFactory { + public TColumnDescTupleScheme getScheme() { + return new TColumnDescTupleScheme(); + } + } + + private static class TColumnDescTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TColumnDesc struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.columnName); + struct.typeDesc.write(oprot); + oprot.writeI32(struct.position); + BitSet optionals = new BitSet(); + if (struct.isSetComment()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetComment()) { + oprot.writeString(struct.comment); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TColumnDesc struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.columnName = iprot.readString(); + struct.setColumnNameIsSet(true); + struct.typeDesc = new TTypeDesc(); + struct.typeDesc.read(iprot); + struct.setTypeDescIsSet(true); + struct.position = iprot.readI32(); + struct.setPositionIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.comment = iprot.readString(); + struct.setCommentIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java new file mode 100644 index 000000000000..44da2cdd089d --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java @@ -0,0 +1,671 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TColumnValue extends org.apache.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnValue"); + private static final org.apache.thrift.protocol.TField BOOL_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("boolVal", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField BYTE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("byteVal", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField I16_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i16Val", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField I32_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i32Val", org.apache.thrift.protocol.TType.STRUCT, (short)4); + private static final org.apache.thrift.protocol.TField I64_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i64Val", org.apache.thrift.protocol.TType.STRUCT, (short)5); + private static final org.apache.thrift.protocol.TField DOUBLE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("doubleVal", org.apache.thrift.protocol.TType.STRUCT, (short)6); + private static final org.apache.thrift.protocol.TField STRING_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("stringVal", org.apache.thrift.protocol.TType.STRUCT, (short)7); + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + BOOL_VAL((short)1, "boolVal"), + BYTE_VAL((short)2, "byteVal"), + I16_VAL((short)3, "i16Val"), + I32_VAL((short)4, "i32Val"), + I64_VAL((short)5, "i64Val"), + DOUBLE_VAL((short)6, "doubleVal"), + STRING_VAL((short)7, "stringVal"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // BOOL_VAL + return BOOL_VAL; + case 2: // BYTE_VAL + return BYTE_VAL; + case 3: // I16_VAL + return I16_VAL; + case 4: // I32_VAL + return I32_VAL; + case 5: // I64_VAL + return I64_VAL; + case 6: // DOUBLE_VAL + return DOUBLE_VAL; + case 7: // STRING_VAL + return STRING_VAL; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.BOOL_VAL, new org.apache.thrift.meta_data.FieldMetaData("boolVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TBoolValue.class))); + tmpMap.put(_Fields.BYTE_VAL, new org.apache.thrift.meta_data.FieldMetaData("byteVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TByteValue.class))); + tmpMap.put(_Fields.I16_VAL, new org.apache.thrift.meta_data.FieldMetaData("i16Val", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI16Value.class))); + tmpMap.put(_Fields.I32_VAL, new org.apache.thrift.meta_data.FieldMetaData("i32Val", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI32Value.class))); + tmpMap.put(_Fields.I64_VAL, new org.apache.thrift.meta_data.FieldMetaData("i64Val", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI64Value.class))); + tmpMap.put(_Fields.DOUBLE_VAL, new org.apache.thrift.meta_data.FieldMetaData("doubleVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TDoubleValue.class))); + tmpMap.put(_Fields.STRING_VAL, new org.apache.thrift.meta_data.FieldMetaData("stringVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStringValue.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumnValue.class, metaDataMap); + } + + public TColumnValue() { + super(); + } + + public TColumnValue(_Fields setField, Object value) { + super(setField, value); + } + + public TColumnValue(TColumnValue other) { + super(other); + } + public TColumnValue deepCopy() { + return new TColumnValue(this); + } + + public static TColumnValue boolVal(TBoolValue value) { + TColumnValue x = new TColumnValue(); + x.setBoolVal(value); + return x; + } + + public static TColumnValue byteVal(TByteValue value) { + TColumnValue x = new TColumnValue(); + x.setByteVal(value); + return x; + } + + public static TColumnValue i16Val(TI16Value value) { + TColumnValue x = new TColumnValue(); + x.setI16Val(value); + return x; + } + + public static TColumnValue i32Val(TI32Value value) { + TColumnValue x = new TColumnValue(); + x.setI32Val(value); + return x; + } + + public static TColumnValue i64Val(TI64Value value) { + TColumnValue x = new TColumnValue(); + x.setI64Val(value); + return x; + } + + public static TColumnValue doubleVal(TDoubleValue value) { + TColumnValue x = new TColumnValue(); + x.setDoubleVal(value); + return x; + } + + public static TColumnValue stringVal(TStringValue value) { + TColumnValue x = new TColumnValue(); + x.setStringVal(value); + return x; + } + + + @Override + protected void checkType(_Fields setField, Object value) throws ClassCastException { + switch (setField) { + case BOOL_VAL: + if (value instanceof TBoolValue) { + break; + } + throw new ClassCastException("Was expecting value of type TBoolValue for field 'boolVal', but got " + value.getClass().getSimpleName()); + case BYTE_VAL: + if (value instanceof TByteValue) { + break; + } + throw new ClassCastException("Was expecting value of type TByteValue for field 'byteVal', but got " + value.getClass().getSimpleName()); + case I16_VAL: + if (value instanceof TI16Value) { + break; + } + throw new ClassCastException("Was expecting value of type TI16Value for field 'i16Val', but got " + value.getClass().getSimpleName()); + case I32_VAL: + if (value instanceof TI32Value) { + break; + } + throw new ClassCastException("Was expecting value of type TI32Value for field 'i32Val', but got " + value.getClass().getSimpleName()); + case I64_VAL: + if (value instanceof TI64Value) { + break; + } + throw new ClassCastException("Was expecting value of type TI64Value for field 'i64Val', but got " + value.getClass().getSimpleName()); + case DOUBLE_VAL: + if (value instanceof TDoubleValue) { + break; + } + throw new ClassCastException("Was expecting value of type TDoubleValue for field 'doubleVal', but got " + value.getClass().getSimpleName()); + case STRING_VAL: + if (value instanceof TStringValue) { + break; + } + throw new ClassCastException("Was expecting value of type TStringValue for field 'stringVal', but got " + value.getClass().getSimpleName()); + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(field.id); + if (setField != null) { + switch (setField) { + case BOOL_VAL: + if (field.type == BOOL_VAL_FIELD_DESC.type) { + TBoolValue boolVal; + boolVal = new TBoolValue(); + boolVal.read(iprot); + return boolVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case BYTE_VAL: + if (field.type == BYTE_VAL_FIELD_DESC.type) { + TByteValue byteVal; + byteVal = new TByteValue(); + byteVal.read(iprot); + return byteVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case I16_VAL: + if (field.type == I16_VAL_FIELD_DESC.type) { + TI16Value i16Val; + i16Val = new TI16Value(); + i16Val.read(iprot); + return i16Val; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case I32_VAL: + if (field.type == I32_VAL_FIELD_DESC.type) { + TI32Value i32Val; + i32Val = new TI32Value(); + i32Val.read(iprot); + return i32Val; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case I64_VAL: + if (field.type == I64_VAL_FIELD_DESC.type) { + TI64Value i64Val; + i64Val = new TI64Value(); + i64Val.read(iprot); + return i64Val; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case DOUBLE_VAL: + if (field.type == DOUBLE_VAL_FIELD_DESC.type) { + TDoubleValue doubleVal; + doubleVal = new TDoubleValue(); + doubleVal.read(iprot); + return doubleVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case STRING_VAL: + if (field.type == STRING_VAL_FIELD_DESC.type) { + TStringValue stringVal; + stringVal = new TStringValue(); + stringVal.read(iprot); + return stringVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case BOOL_VAL: + TBoolValue boolVal = (TBoolValue)value_; + boolVal.write(oprot); + return; + case BYTE_VAL: + TByteValue byteVal = (TByteValue)value_; + byteVal.write(oprot); + return; + case I16_VAL: + TI16Value i16Val = (TI16Value)value_; + i16Val.write(oprot); + return; + case I32_VAL: + TI32Value i32Val = (TI32Value)value_; + i32Val.write(oprot); + return; + case I64_VAL: + TI64Value i64Val = (TI64Value)value_; + i64Val.write(oprot); + return; + case DOUBLE_VAL: + TDoubleValue doubleVal = (TDoubleValue)value_; + doubleVal.write(oprot); + return; + case STRING_VAL: + TStringValue stringVal = (TStringValue)value_; + stringVal.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(fieldID); + if (setField != null) { + switch (setField) { + case BOOL_VAL: + TBoolValue boolVal; + boolVal = new TBoolValue(); + boolVal.read(iprot); + return boolVal; + case BYTE_VAL: + TByteValue byteVal; + byteVal = new TByteValue(); + byteVal.read(iprot); + return byteVal; + case I16_VAL: + TI16Value i16Val; + i16Val = new TI16Value(); + i16Val.read(iprot); + return i16Val; + case I32_VAL: + TI32Value i32Val; + i32Val = new TI32Value(); + i32Val.read(iprot); + return i32Val; + case I64_VAL: + TI64Value i64Val; + i64Val = new TI64Value(); + i64Val.read(iprot); + return i64Val; + case DOUBLE_VAL: + TDoubleValue doubleVal; + doubleVal = new TDoubleValue(); + doubleVal.read(iprot); + return doubleVal; + case STRING_VAL: + TStringValue stringVal; + stringVal = new TStringValue(); + stringVal.read(iprot); + return stringVal; + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + throw new TProtocolException("Couldn't find a field with field id " + fieldID); + } + } + + @Override + protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case BOOL_VAL: + TBoolValue boolVal = (TBoolValue)value_; + boolVal.write(oprot); + return; + case BYTE_VAL: + TByteValue byteVal = (TByteValue)value_; + byteVal.write(oprot); + return; + case I16_VAL: + TI16Value i16Val = (TI16Value)value_; + i16Val.write(oprot); + return; + case I32_VAL: + TI32Value i32Val = (TI32Value)value_; + i32Val.write(oprot); + return; + case I64_VAL: + TI64Value i64Val = (TI64Value)value_; + i64Val.write(oprot); + return; + case DOUBLE_VAL: + TDoubleValue doubleVal = (TDoubleValue)value_; + doubleVal.write(oprot); + return; + case STRING_VAL: + TStringValue stringVal = (TStringValue)value_; + stringVal.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { + switch (setField) { + case BOOL_VAL: + return BOOL_VAL_FIELD_DESC; + case BYTE_VAL: + return BYTE_VAL_FIELD_DESC; + case I16_VAL: + return I16_VAL_FIELD_DESC; + case I32_VAL: + return I32_VAL_FIELD_DESC; + case I64_VAL: + return I64_VAL_FIELD_DESC; + case DOUBLE_VAL: + return DOUBLE_VAL_FIELD_DESC; + case STRING_VAL: + return STRING_VAL_FIELD_DESC; + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected org.apache.thrift.protocol.TStruct getStructDesc() { + return STRUCT_DESC; + } + + @Override + protected _Fields enumForId(short id) { + return _Fields.findByThriftIdOrThrow(id); + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + + public TBoolValue getBoolVal() { + if (getSetField() == _Fields.BOOL_VAL) { + return (TBoolValue)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'boolVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setBoolVal(TBoolValue value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.BOOL_VAL; + value_ = value; + } + + public TByteValue getByteVal() { + if (getSetField() == _Fields.BYTE_VAL) { + return (TByteValue)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'byteVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setByteVal(TByteValue value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.BYTE_VAL; + value_ = value; + } + + public TI16Value getI16Val() { + if (getSetField() == _Fields.I16_VAL) { + return (TI16Value)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'i16Val' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setI16Val(TI16Value value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.I16_VAL; + value_ = value; + } + + public TI32Value getI32Val() { + if (getSetField() == _Fields.I32_VAL) { + return (TI32Value)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'i32Val' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setI32Val(TI32Value value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.I32_VAL; + value_ = value; + } + + public TI64Value getI64Val() { + if (getSetField() == _Fields.I64_VAL) { + return (TI64Value)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'i64Val' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setI64Val(TI64Value value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.I64_VAL; + value_ = value; + } + + public TDoubleValue getDoubleVal() { + if (getSetField() == _Fields.DOUBLE_VAL) { + return (TDoubleValue)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'doubleVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setDoubleVal(TDoubleValue value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.DOUBLE_VAL; + value_ = value; + } + + public TStringValue getStringVal() { + if (getSetField() == _Fields.STRING_VAL) { + return (TStringValue)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'stringVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setStringVal(TStringValue value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.STRING_VAL; + value_ = value; + } + + public boolean isSetBoolVal() { + return setField_ == _Fields.BOOL_VAL; + } + + + public boolean isSetByteVal() { + return setField_ == _Fields.BYTE_VAL; + } + + + public boolean isSetI16Val() { + return setField_ == _Fields.I16_VAL; + } + + + public boolean isSetI32Val() { + return setField_ == _Fields.I32_VAL; + } + + + public boolean isSetI64Val() { + return setField_ == _Fields.I64_VAL; + } + + + public boolean isSetDoubleVal() { + return setField_ == _Fields.DOUBLE_VAL; + } + + + public boolean isSetStringVal() { + return setField_ == _Fields.STRING_VAL; + } + + + public boolean equals(Object other) { + if (other instanceof TColumnValue) { + return equals((TColumnValue)other); + } else { + return false; + } + } + + public boolean equals(TColumnValue other) { + return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); + } + + @Override + public int compareTo(TColumnValue other) { + int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + HashCodeBuilder hcb = new HashCodeBuilder(); + hcb.append(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + hcb.append(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); + } else { + hcb.append(value); + } + } + return hcb.toHashCode(); + } + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + +} diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java new file mode 100644 index 000000000000..4fc54544c1be --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TDoubleColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDoubleColumn"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TDoubleColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TDoubleColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TDoubleColumn.class, metaDataMap); + } + + public TDoubleColumn() { + } + + public TDoubleColumn( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = nulls; + } + + /** + * Performs a deep copy on other. + */ + public TDoubleColumn(TDoubleColumn other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(); + for (Double other_element : other.values) { + __this__values.add(other_element); + } + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); +; + } + } + + public TDoubleColumn deepCopy() { + return new TDoubleColumn(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(double elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return nulls; + } + + public void setNulls(byte[] nulls) { + setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = nulls; + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TDoubleColumn) + return this.equals((TDoubleColumn)that); + return false; + } + + public boolean equals(TDoubleColumn that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_values = true && (isSetValues()); + builder.append(present_values); + if (present_values) + builder.append(values); + + boolean present_nulls = true && (isSetNulls()); + builder.append(present_nulls); + if (present_nulls) + builder.append(nulls); + + return builder.toHashCode(); + } + + public int compareTo(TDoubleColumn other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TDoubleColumn typedOther = (TDoubleColumn)other; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TDoubleColumn("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TDoubleColumnStandardSchemeFactory implements SchemeFactory { + public TDoubleColumnStandardScheme getScheme() { + return new TDoubleColumnStandardScheme(); + } + } + + private static class TDoubleColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TDoubleColumn struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list94 = iprot.readListBegin(); + struct.values = new ArrayList(_list94.size); + for (int _i95 = 0; _i95 < _list94.size; ++_i95) + { + double _elem96; // optional + _elem96 = iprot.readDouble(); + struct.values.add(_elem96); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TDoubleColumn struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.DOUBLE, struct.values.size())); + for (double _iter97 : struct.values) + { + oprot.writeDouble(_iter97); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TDoubleColumnTupleSchemeFactory implements SchemeFactory { + public TDoubleColumnTupleScheme getScheme() { + return new TDoubleColumnTupleScheme(); + } + } + + private static class TDoubleColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TDoubleColumn struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (double _iter98 : struct.values) + { + oprot.writeDouble(_iter98); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TDoubleColumn struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list99 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32()); + struct.values = new ArrayList(_list99.size); + for (int _i100 = 0; _i100 < _list99.size; ++_i100) + { + double _elem101; // optional + _elem101 = iprot.readDouble(); + struct.values.add(_elem101); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java new file mode 100644 index 000000000000..d21573633ef5 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java @@ -0,0 +1,386 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TDoubleValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDoubleValue"); + + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.DOUBLE, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TDoubleValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TDoubleValueTupleSchemeFactory()); + } + + private double value; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUE((short)1, "value"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUE + return VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __VALUE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.VALUE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TDoubleValue.class, metaDataMap); + } + + public TDoubleValue() { + } + + /** + * Performs a deep copy on other. + */ + public TDoubleValue(TDoubleValue other) { + __isset_bitfield = other.__isset_bitfield; + this.value = other.value; + } + + public TDoubleValue deepCopy() { + return new TDoubleValue(this); + } + + @Override + public void clear() { + setValueIsSet(false); + this.value = 0.0; + } + + public double getValue() { + return this.value; + } + + public void setValue(double value) { + this.value = value; + setValueIsSet(true); + } + + public void unsetValue() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + public void setValueIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((Double)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUE: + return Double.valueOf(getValue()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUE: + return isSetValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TDoubleValue) + return this.equals((TDoubleValue)that); + return false; + } + + public boolean equals(TDoubleValue that) { + if (that == null) + return false; + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (this.value != that.value) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_value = true && (isSetValue()); + builder.append(present_value); + if (present_value) + builder.append(value); + + return builder.toHashCode(); + } + + public int compareTo(TDoubleValue other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TDoubleValue typedOther = (TDoubleValue)other; + + lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TDoubleValue("); + boolean first = true; + + if (isSetValue()) { + sb.append("value:"); + sb.append(this.value); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TDoubleValueStandardSchemeFactory implements SchemeFactory { + public TDoubleValueStandardScheme getScheme() { + return new TDoubleValueStandardScheme(); + } + } + + private static class TDoubleValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TDoubleValue struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { + struct.value = iprot.readDouble(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TDoubleValue struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetValue()) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeDouble(struct.value); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TDoubleValueTupleSchemeFactory implements SchemeFactory { + public TDoubleValueTupleScheme getScheme() { + return new TDoubleValueTupleScheme(); + } + } + + private static class TDoubleValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TDoubleValue struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetValue()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetValue()) { + oprot.writeDouble(struct.value); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TDoubleValue struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.value = iprot.readDouble(); + struct.setValueIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java new file mode 100644 index 000000000000..4f157ad5a645 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java @@ -0,0 +1,769 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TExecuteStatementReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TExecuteStatementReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField STATEMENT_FIELD_DESC = new org.apache.thrift.protocol.TField("statement", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField CONF_OVERLAY_FIELD_DESC = new org.apache.thrift.protocol.TField("confOverlay", org.apache.thrift.protocol.TType.MAP, (short)3); + private static final org.apache.thrift.protocol.TField RUN_ASYNC_FIELD_DESC = new org.apache.thrift.protocol.TField("runAsync", org.apache.thrift.protocol.TType.BOOL, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TExecuteStatementReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TExecuteStatementReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String statement; // required + private Map confOverlay; // optional + private boolean runAsync; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + STATEMENT((short)2, "statement"), + CONF_OVERLAY((short)3, "confOverlay"), + RUN_ASYNC((short)4, "runAsync"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // STATEMENT + return STATEMENT; + case 3: // CONF_OVERLAY + return CONF_OVERLAY; + case 4: // RUN_ASYNC + return RUN_ASYNC; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __RUNASYNC_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.CONF_OVERLAY,_Fields.RUN_ASYNC}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.STATEMENT, new org.apache.thrift.meta_data.FieldMetaData("statement", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.CONF_OVERLAY, new org.apache.thrift.meta_data.FieldMetaData("confOverlay", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.RUN_ASYNC, new org.apache.thrift.meta_data.FieldMetaData("runAsync", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TExecuteStatementReq.class, metaDataMap); + } + + public TExecuteStatementReq() { + this.runAsync = false; + + } + + public TExecuteStatementReq( + TSessionHandle sessionHandle, + String statement) + { + this(); + this.sessionHandle = sessionHandle; + this.statement = statement; + } + + /** + * Performs a deep copy on other. + */ + public TExecuteStatementReq(TExecuteStatementReq other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetStatement()) { + this.statement = other.statement; + } + if (other.isSetConfOverlay()) { + Map __this__confOverlay = new HashMap(); + for (Map.Entry other_element : other.confOverlay.entrySet()) { + + String other_element_key = other_element.getKey(); + String other_element_value = other_element.getValue(); + + String __this__confOverlay_copy_key = other_element_key; + + String __this__confOverlay_copy_value = other_element_value; + + __this__confOverlay.put(__this__confOverlay_copy_key, __this__confOverlay_copy_value); + } + this.confOverlay = __this__confOverlay; + } + this.runAsync = other.runAsync; + } + + public TExecuteStatementReq deepCopy() { + return new TExecuteStatementReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.statement = null; + this.confOverlay = null; + this.runAsync = false; + + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getStatement() { + return this.statement; + } + + public void setStatement(String statement) { + this.statement = statement; + } + + public void unsetStatement() { + this.statement = null; + } + + /** Returns true if field statement is set (has been assigned a value) and false otherwise */ + public boolean isSetStatement() { + return this.statement != null; + } + + public void setStatementIsSet(boolean value) { + if (!value) { + this.statement = null; + } + } + + public int getConfOverlaySize() { + return (this.confOverlay == null) ? 0 : this.confOverlay.size(); + } + + public void putToConfOverlay(String key, String val) { + if (this.confOverlay == null) { + this.confOverlay = new HashMap(); + } + this.confOverlay.put(key, val); + } + + public Map getConfOverlay() { + return this.confOverlay; + } + + public void setConfOverlay(Map confOverlay) { + this.confOverlay = confOverlay; + } + + public void unsetConfOverlay() { + this.confOverlay = null; + } + + /** Returns true if field confOverlay is set (has been assigned a value) and false otherwise */ + public boolean isSetConfOverlay() { + return this.confOverlay != null; + } + + public void setConfOverlayIsSet(boolean value) { + if (!value) { + this.confOverlay = null; + } + } + + public boolean isRunAsync() { + return this.runAsync; + } + + public void setRunAsync(boolean runAsync) { + this.runAsync = runAsync; + setRunAsyncIsSet(true); + } + + public void unsetRunAsync() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RUNASYNC_ISSET_ID); + } + + /** Returns true if field runAsync is set (has been assigned a value) and false otherwise */ + public boolean isSetRunAsync() { + return EncodingUtils.testBit(__isset_bitfield, __RUNASYNC_ISSET_ID); + } + + public void setRunAsyncIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RUNASYNC_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case STATEMENT: + if (value == null) { + unsetStatement(); + } else { + setStatement((String)value); + } + break; + + case CONF_OVERLAY: + if (value == null) { + unsetConfOverlay(); + } else { + setConfOverlay((Map)value); + } + break; + + case RUN_ASYNC: + if (value == null) { + unsetRunAsync(); + } else { + setRunAsync((Boolean)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case STATEMENT: + return getStatement(); + + case CONF_OVERLAY: + return getConfOverlay(); + + case RUN_ASYNC: + return Boolean.valueOf(isRunAsync()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case STATEMENT: + return isSetStatement(); + case CONF_OVERLAY: + return isSetConfOverlay(); + case RUN_ASYNC: + return isSetRunAsync(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TExecuteStatementReq) + return this.equals((TExecuteStatementReq)that); + return false; + } + + public boolean equals(TExecuteStatementReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_statement = true && this.isSetStatement(); + boolean that_present_statement = true && that.isSetStatement(); + if (this_present_statement || that_present_statement) { + if (!(this_present_statement && that_present_statement)) + return false; + if (!this.statement.equals(that.statement)) + return false; + } + + boolean this_present_confOverlay = true && this.isSetConfOverlay(); + boolean that_present_confOverlay = true && that.isSetConfOverlay(); + if (this_present_confOverlay || that_present_confOverlay) { + if (!(this_present_confOverlay && that_present_confOverlay)) + return false; + if (!this.confOverlay.equals(that.confOverlay)) + return false; + } + + boolean this_present_runAsync = true && this.isSetRunAsync(); + boolean that_present_runAsync = true && that.isSetRunAsync(); + if (this_present_runAsync || that_present_runAsync) { + if (!(this_present_runAsync && that_present_runAsync)) + return false; + if (this.runAsync != that.runAsync) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + builder.append(present_sessionHandle); + if (present_sessionHandle) + builder.append(sessionHandle); + + boolean present_statement = true && (isSetStatement()); + builder.append(present_statement); + if (present_statement) + builder.append(statement); + + boolean present_confOverlay = true && (isSetConfOverlay()); + builder.append(present_confOverlay); + if (present_confOverlay) + builder.append(confOverlay); + + boolean present_runAsync = true && (isSetRunAsync()); + builder.append(present_runAsync); + if (present_runAsync) + builder.append(runAsync); + + return builder.toHashCode(); + } + + public int compareTo(TExecuteStatementReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TExecuteStatementReq typedOther = (TExecuteStatementReq)other; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStatement()).compareTo(typedOther.isSetStatement()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatement()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.statement, typedOther.statement); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetConfOverlay()).compareTo(typedOther.isSetConfOverlay()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetConfOverlay()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.confOverlay, typedOther.confOverlay); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetRunAsync()).compareTo(typedOther.isSetRunAsync()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRunAsync()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.runAsync, typedOther.runAsync); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TExecuteStatementReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (!first) sb.append(", "); + sb.append("statement:"); + if (this.statement == null) { + sb.append("null"); + } else { + sb.append(this.statement); + } + first = false; + if (isSetConfOverlay()) { + if (!first) sb.append(", "); + sb.append("confOverlay:"); + if (this.confOverlay == null) { + sb.append("null"); + } else { + sb.append(this.confOverlay); + } + first = false; + } + if (isSetRunAsync()) { + if (!first) sb.append(", "); + sb.append("runAsync:"); + sb.append(this.runAsync); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + if (!isSetStatement()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'statement' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TExecuteStatementReqStandardSchemeFactory implements SchemeFactory { + public TExecuteStatementReqStandardScheme getScheme() { + return new TExecuteStatementReqStandardScheme(); + } + } + + private static class TExecuteStatementReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TExecuteStatementReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // STATEMENT + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.statement = iprot.readString(); + struct.setStatementIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // CONF_OVERLAY + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map162 = iprot.readMapBegin(); + struct.confOverlay = new HashMap(2*_map162.size); + for (int _i163 = 0; _i163 < _map162.size; ++_i163) + { + String _key164; // required + String _val165; // required + _key164 = iprot.readString(); + _val165 = iprot.readString(); + struct.confOverlay.put(_key164, _val165); + } + iprot.readMapEnd(); + } + struct.setConfOverlayIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // RUN_ASYNC + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.runAsync = iprot.readBool(); + struct.setRunAsyncIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TExecuteStatementReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.statement != null) { + oprot.writeFieldBegin(STATEMENT_FIELD_DESC); + oprot.writeString(struct.statement); + oprot.writeFieldEnd(); + } + if (struct.confOverlay != null) { + if (struct.isSetConfOverlay()) { + oprot.writeFieldBegin(CONF_OVERLAY_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.confOverlay.size())); + for (Map.Entry _iter166 : struct.confOverlay.entrySet()) + { + oprot.writeString(_iter166.getKey()); + oprot.writeString(_iter166.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.isSetRunAsync()) { + oprot.writeFieldBegin(RUN_ASYNC_FIELD_DESC); + oprot.writeBool(struct.runAsync); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TExecuteStatementReqTupleSchemeFactory implements SchemeFactory { + public TExecuteStatementReqTupleScheme getScheme() { + return new TExecuteStatementReqTupleScheme(); + } + } + + private static class TExecuteStatementReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + oprot.writeString(struct.statement); + BitSet optionals = new BitSet(); + if (struct.isSetConfOverlay()) { + optionals.set(0); + } + if (struct.isSetRunAsync()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetConfOverlay()) { + { + oprot.writeI32(struct.confOverlay.size()); + for (Map.Entry _iter167 : struct.confOverlay.entrySet()) + { + oprot.writeString(_iter167.getKey()); + oprot.writeString(_iter167.getValue()); + } + } + } + if (struct.isSetRunAsync()) { + oprot.writeBool(struct.runAsync); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + struct.statement = iprot.readString(); + struct.setStatementIsSet(true); + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TMap _map168 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.confOverlay = new HashMap(2*_map168.size); + for (int _i169 = 0; _i169 < _map168.size; ++_i169) + { + String _key170; // required + String _val171; // required + _key170 = iprot.readString(); + _val171 = iprot.readString(); + struct.confOverlay.put(_key170, _val171); + } + } + struct.setConfOverlayIsSet(true); + } + if (incoming.get(1)) { + struct.runAsync = iprot.readBool(); + struct.setRunAsyncIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java new file mode 100644 index 000000000000..fdde51e70f78 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java @@ -0,0 +1,505 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TExecuteStatementResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TExecuteStatementResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TExecuteStatementRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TExecuteStatementRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TExecuteStatementResp.class, metaDataMap); + } + + public TExecuteStatementResp() { + } + + public TExecuteStatementResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TExecuteStatementResp(TExecuteStatementResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TExecuteStatementResp deepCopy() { + return new TExecuteStatementResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TExecuteStatementResp) + return this.equals((TExecuteStatementResp)that); + return false; + } + + public boolean equals(TExecuteStatementResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + builder.append(present_operationHandle); + if (present_operationHandle) + builder.append(operationHandle); + + return builder.toHashCode(); + } + + public int compareTo(TExecuteStatementResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TExecuteStatementResp typedOther = (TExecuteStatementResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TExecuteStatementResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TExecuteStatementRespStandardSchemeFactory implements SchemeFactory { + public TExecuteStatementRespStandardScheme getScheme() { + return new TExecuteStatementRespStandardScheme(); + } + } + + private static class TExecuteStatementRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TExecuteStatementResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TExecuteStatementResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TExecuteStatementRespTupleSchemeFactory implements SchemeFactory { + public TExecuteStatementRespTupleScheme getScheme() { + return new TExecuteStatementRespTupleScheme(); + } + } + + private static class TExecuteStatementRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java new file mode 100644 index 000000000000..b2a22effd91a --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java @@ -0,0 +1,57 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TFetchOrientation implements org.apache.thrift.TEnum { + FETCH_NEXT(0), + FETCH_PRIOR(1), + FETCH_RELATIVE(2), + FETCH_ABSOLUTE(3), + FETCH_FIRST(4), + FETCH_LAST(5); + + private final int value; + + private TFetchOrientation(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TFetchOrientation findByValue(int value) { + switch (value) { + case 0: + return FETCH_NEXT; + case 1: + return FETCH_PRIOR; + case 2: + return FETCH_RELATIVE; + case 3: + return FETCH_ABSOLUTE; + case 4: + return FETCH_FIRST; + case 5: + return FETCH_LAST; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java new file mode 100644 index 000000000000..068711fc4444 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java @@ -0,0 +1,710 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TFetchResultsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TFetchResultsReq"); + + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField ORIENTATION_FIELD_DESC = new org.apache.thrift.protocol.TField("orientation", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField MAX_ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("maxRows", org.apache.thrift.protocol.TType.I64, (short)3); + private static final org.apache.thrift.protocol.TField FETCH_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("fetchType", org.apache.thrift.protocol.TType.I16, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TFetchResultsReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TFetchResultsReqTupleSchemeFactory()); + } + + private TOperationHandle operationHandle; // required + private TFetchOrientation orientation; // required + private long maxRows; // required + private short fetchType; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OPERATION_HANDLE((short)1, "operationHandle"), + /** + * + * @see TFetchOrientation + */ + ORIENTATION((short)2, "orientation"), + MAX_ROWS((short)3, "maxRows"), + FETCH_TYPE((short)4, "fetchType"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OPERATION_HANDLE + return OPERATION_HANDLE; + case 2: // ORIENTATION + return ORIENTATION; + case 3: // MAX_ROWS + return MAX_ROWS; + case 4: // FETCH_TYPE + return FETCH_TYPE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __MAXROWS_ISSET_ID = 0; + private static final int __FETCHTYPE_ISSET_ID = 1; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.FETCH_TYPE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + tmpMap.put(_Fields.ORIENTATION, new org.apache.thrift.meta_data.FieldMetaData("orientation", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TFetchOrientation.class))); + tmpMap.put(_Fields.MAX_ROWS, new org.apache.thrift.meta_data.FieldMetaData("maxRows", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.FETCH_TYPE, new org.apache.thrift.meta_data.FieldMetaData("fetchType", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TFetchResultsReq.class, metaDataMap); + } + + public TFetchResultsReq() { + this.orientation = org.apache.hive.service.cli.thrift.TFetchOrientation.FETCH_NEXT; + + this.fetchType = (short)0; + + } + + public TFetchResultsReq( + TOperationHandle operationHandle, + TFetchOrientation orientation, + long maxRows) + { + this(); + this.operationHandle = operationHandle; + this.orientation = orientation; + this.maxRows = maxRows; + setMaxRowsIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TFetchResultsReq(TFetchResultsReq other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + if (other.isSetOrientation()) { + this.orientation = other.orientation; + } + this.maxRows = other.maxRows; + this.fetchType = other.fetchType; + } + + public TFetchResultsReq deepCopy() { + return new TFetchResultsReq(this); + } + + @Override + public void clear() { + this.operationHandle = null; + this.orientation = org.apache.hive.service.cli.thrift.TFetchOrientation.FETCH_NEXT; + + setMaxRowsIsSet(false); + this.maxRows = 0; + this.fetchType = (short)0; + + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + /** + * + * @see TFetchOrientation + */ + public TFetchOrientation getOrientation() { + return this.orientation; + } + + /** + * + * @see TFetchOrientation + */ + public void setOrientation(TFetchOrientation orientation) { + this.orientation = orientation; + } + + public void unsetOrientation() { + this.orientation = null; + } + + /** Returns true if field orientation is set (has been assigned a value) and false otherwise */ + public boolean isSetOrientation() { + return this.orientation != null; + } + + public void setOrientationIsSet(boolean value) { + if (!value) { + this.orientation = null; + } + } + + public long getMaxRows() { + return this.maxRows; + } + + public void setMaxRows(long maxRows) { + this.maxRows = maxRows; + setMaxRowsIsSet(true); + } + + public void unsetMaxRows() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAXROWS_ISSET_ID); + } + + /** Returns true if field maxRows is set (has been assigned a value) and false otherwise */ + public boolean isSetMaxRows() { + return EncodingUtils.testBit(__isset_bitfield, __MAXROWS_ISSET_ID); + } + + public void setMaxRowsIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAXROWS_ISSET_ID, value); + } + + public short getFetchType() { + return this.fetchType; + } + + public void setFetchType(short fetchType) { + this.fetchType = fetchType; + setFetchTypeIsSet(true); + } + + public void unsetFetchType() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __FETCHTYPE_ISSET_ID); + } + + /** Returns true if field fetchType is set (has been assigned a value) and false otherwise */ + public boolean isSetFetchType() { + return EncodingUtils.testBit(__isset_bitfield, __FETCHTYPE_ISSET_ID); + } + + public void setFetchTypeIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __FETCHTYPE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + case ORIENTATION: + if (value == null) { + unsetOrientation(); + } else { + setOrientation((TFetchOrientation)value); + } + break; + + case MAX_ROWS: + if (value == null) { + unsetMaxRows(); + } else { + setMaxRows((Long)value); + } + break; + + case FETCH_TYPE: + if (value == null) { + unsetFetchType(); + } else { + setFetchType((Short)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OPERATION_HANDLE: + return getOperationHandle(); + + case ORIENTATION: + return getOrientation(); + + case MAX_ROWS: + return Long.valueOf(getMaxRows()); + + case FETCH_TYPE: + return Short.valueOf(getFetchType()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OPERATION_HANDLE: + return isSetOperationHandle(); + case ORIENTATION: + return isSetOrientation(); + case MAX_ROWS: + return isSetMaxRows(); + case FETCH_TYPE: + return isSetFetchType(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TFetchResultsReq) + return this.equals((TFetchResultsReq)that); + return false; + } + + public boolean equals(TFetchResultsReq that) { + if (that == null) + return false; + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + boolean this_present_orientation = true && this.isSetOrientation(); + boolean that_present_orientation = true && that.isSetOrientation(); + if (this_present_orientation || that_present_orientation) { + if (!(this_present_orientation && that_present_orientation)) + return false; + if (!this.orientation.equals(that.orientation)) + return false; + } + + boolean this_present_maxRows = true; + boolean that_present_maxRows = true; + if (this_present_maxRows || that_present_maxRows) { + if (!(this_present_maxRows && that_present_maxRows)) + return false; + if (this.maxRows != that.maxRows) + return false; + } + + boolean this_present_fetchType = true && this.isSetFetchType(); + boolean that_present_fetchType = true && that.isSetFetchType(); + if (this_present_fetchType || that_present_fetchType) { + if (!(this_present_fetchType && that_present_fetchType)) + return false; + if (this.fetchType != that.fetchType) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_operationHandle = true && (isSetOperationHandle()); + builder.append(present_operationHandle); + if (present_operationHandle) + builder.append(operationHandle); + + boolean present_orientation = true && (isSetOrientation()); + builder.append(present_orientation); + if (present_orientation) + builder.append(orientation.getValue()); + + boolean present_maxRows = true; + builder.append(present_maxRows); + if (present_maxRows) + builder.append(maxRows); + + boolean present_fetchType = true && (isSetFetchType()); + builder.append(present_fetchType); + if (present_fetchType) + builder.append(fetchType); + + return builder.toHashCode(); + } + + public int compareTo(TFetchResultsReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TFetchResultsReq typedOther = (TFetchResultsReq)other; + + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOrientation()).compareTo(typedOther.isSetOrientation()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOrientation()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.orientation, typedOther.orientation); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaxRows()).compareTo(typedOther.isSetMaxRows()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaxRows()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maxRows, typedOther.maxRows); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetFetchType()).compareTo(typedOther.isSetFetchType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFetchType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fetchType, typedOther.fetchType); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TFetchResultsReq("); + boolean first = true; + + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + if (!first) sb.append(", "); + sb.append("orientation:"); + if (this.orientation == null) { + sb.append("null"); + } else { + sb.append(this.orientation); + } + first = false; + if (!first) sb.append(", "); + sb.append("maxRows:"); + sb.append(this.maxRows); + first = false; + if (isSetFetchType()) { + if (!first) sb.append(", "); + sb.append("fetchType:"); + sb.append(this.fetchType); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetOperationHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); + } + + if (!isSetOrientation()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'orientation' is unset! Struct:" + toString()); + } + + if (!isSetMaxRows()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'maxRows' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TFetchResultsReqStandardSchemeFactory implements SchemeFactory { + public TFetchResultsReqStandardScheme getScheme() { + return new TFetchResultsReqStandardScheme(); + } + } + + private static class TFetchResultsReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TFetchResultsReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // ORIENTATION + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.orientation = TFetchOrientation.findByValue(iprot.readI32()); + struct.setOrientationIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // MAX_ROWS + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.maxRows = iprot.readI64(); + struct.setMaxRowsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // FETCH_TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I16) { + struct.fetchType = iprot.readI16(); + struct.setFetchTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TFetchResultsReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.operationHandle != null) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.orientation != null) { + oprot.writeFieldBegin(ORIENTATION_FIELD_DESC); + oprot.writeI32(struct.orientation.getValue()); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(MAX_ROWS_FIELD_DESC); + oprot.writeI64(struct.maxRows); + oprot.writeFieldEnd(); + if (struct.isSetFetchType()) { + oprot.writeFieldBegin(FETCH_TYPE_FIELD_DESC); + oprot.writeI16(struct.fetchType); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TFetchResultsReqTupleSchemeFactory implements SchemeFactory { + public TFetchResultsReqTupleScheme getScheme() { + return new TFetchResultsReqTupleScheme(); + } + } + + private static class TFetchResultsReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TFetchResultsReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.operationHandle.write(oprot); + oprot.writeI32(struct.orientation.getValue()); + oprot.writeI64(struct.maxRows); + BitSet optionals = new BitSet(); + if (struct.isSetFetchType()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetFetchType()) { + oprot.writeI16(struct.fetchType); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TFetchResultsReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + struct.orientation = TFetchOrientation.findByValue(iprot.readI32()); + struct.setOrientationIsSet(true); + struct.maxRows = iprot.readI64(); + struct.setMaxRowsIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.fetchType = iprot.readI16(); + struct.setFetchTypeIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java new file mode 100644 index 000000000000..19991f1da3eb --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java @@ -0,0 +1,608 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TFetchResultsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TFetchResultsResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField HAS_MORE_ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("hasMoreRows", org.apache.thrift.protocol.TType.BOOL, (short)2); + private static final org.apache.thrift.protocol.TField RESULTS_FIELD_DESC = new org.apache.thrift.protocol.TField("results", org.apache.thrift.protocol.TType.STRUCT, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TFetchResultsRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TFetchResultsRespTupleSchemeFactory()); + } + + private TStatus status; // required + private boolean hasMoreRows; // optional + private TRowSet results; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + HAS_MORE_ROWS((short)2, "hasMoreRows"), + RESULTS((short)3, "results"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // HAS_MORE_ROWS + return HAS_MORE_ROWS; + case 3: // RESULTS + return RESULTS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __HASMOREROWS_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.HAS_MORE_ROWS,_Fields.RESULTS}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.HAS_MORE_ROWS, new org.apache.thrift.meta_data.FieldMetaData("hasMoreRows", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.RESULTS, new org.apache.thrift.meta_data.FieldMetaData("results", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRowSet.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TFetchResultsResp.class, metaDataMap); + } + + public TFetchResultsResp() { + } + + public TFetchResultsResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TFetchResultsResp(TFetchResultsResp other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + this.hasMoreRows = other.hasMoreRows; + if (other.isSetResults()) { + this.results = new TRowSet(other.results); + } + } + + public TFetchResultsResp deepCopy() { + return new TFetchResultsResp(this); + } + + @Override + public void clear() { + this.status = null; + setHasMoreRowsIsSet(false); + this.hasMoreRows = false; + this.results = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public boolean isHasMoreRows() { + return this.hasMoreRows; + } + + public void setHasMoreRows(boolean hasMoreRows) { + this.hasMoreRows = hasMoreRows; + setHasMoreRowsIsSet(true); + } + + public void unsetHasMoreRows() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __HASMOREROWS_ISSET_ID); + } + + /** Returns true if field hasMoreRows is set (has been assigned a value) and false otherwise */ + public boolean isSetHasMoreRows() { + return EncodingUtils.testBit(__isset_bitfield, __HASMOREROWS_ISSET_ID); + } + + public void setHasMoreRowsIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __HASMOREROWS_ISSET_ID, value); + } + + public TRowSet getResults() { + return this.results; + } + + public void setResults(TRowSet results) { + this.results = results; + } + + public void unsetResults() { + this.results = null; + } + + /** Returns true if field results is set (has been assigned a value) and false otherwise */ + public boolean isSetResults() { + return this.results != null; + } + + public void setResultsIsSet(boolean value) { + if (!value) { + this.results = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case HAS_MORE_ROWS: + if (value == null) { + unsetHasMoreRows(); + } else { + setHasMoreRows((Boolean)value); + } + break; + + case RESULTS: + if (value == null) { + unsetResults(); + } else { + setResults((TRowSet)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case HAS_MORE_ROWS: + return Boolean.valueOf(isHasMoreRows()); + + case RESULTS: + return getResults(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case HAS_MORE_ROWS: + return isSetHasMoreRows(); + case RESULTS: + return isSetResults(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TFetchResultsResp) + return this.equals((TFetchResultsResp)that); + return false; + } + + public boolean equals(TFetchResultsResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_hasMoreRows = true && this.isSetHasMoreRows(); + boolean that_present_hasMoreRows = true && that.isSetHasMoreRows(); + if (this_present_hasMoreRows || that_present_hasMoreRows) { + if (!(this_present_hasMoreRows && that_present_hasMoreRows)) + return false; + if (this.hasMoreRows != that.hasMoreRows) + return false; + } + + boolean this_present_results = true && this.isSetResults(); + boolean that_present_results = true && that.isSetResults(); + if (this_present_results || that_present_results) { + if (!(this_present_results && that_present_results)) + return false; + if (!this.results.equals(that.results)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_hasMoreRows = true && (isSetHasMoreRows()); + builder.append(present_hasMoreRows); + if (present_hasMoreRows) + builder.append(hasMoreRows); + + boolean present_results = true && (isSetResults()); + builder.append(present_results); + if (present_results) + builder.append(results); + + return builder.toHashCode(); + } + + public int compareTo(TFetchResultsResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TFetchResultsResp typedOther = (TFetchResultsResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetHasMoreRows()).compareTo(typedOther.isSetHasMoreRows()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHasMoreRows()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hasMoreRows, typedOther.hasMoreRows); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetResults()).compareTo(typedOther.isSetResults()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetResults()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.results, typedOther.results); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TFetchResultsResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetHasMoreRows()) { + if (!first) sb.append(", "); + sb.append("hasMoreRows:"); + sb.append(this.hasMoreRows); + first = false; + } + if (isSetResults()) { + if (!first) sb.append(", "); + sb.append("results:"); + if (this.results == null) { + sb.append("null"); + } else { + sb.append(this.results); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (results != null) { + results.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TFetchResultsRespStandardSchemeFactory implements SchemeFactory { + public TFetchResultsRespStandardScheme getScheme() { + return new TFetchResultsRespStandardScheme(); + } + } + + private static class TFetchResultsRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TFetchResultsResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // HAS_MORE_ROWS + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.hasMoreRows = iprot.readBool(); + struct.setHasMoreRowsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // RESULTS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.results = new TRowSet(); + struct.results.read(iprot); + struct.setResultsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TFetchResultsResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.isSetHasMoreRows()) { + oprot.writeFieldBegin(HAS_MORE_ROWS_FIELD_DESC); + oprot.writeBool(struct.hasMoreRows); + oprot.writeFieldEnd(); + } + if (struct.results != null) { + if (struct.isSetResults()) { + oprot.writeFieldBegin(RESULTS_FIELD_DESC); + struct.results.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TFetchResultsRespTupleSchemeFactory implements SchemeFactory { + public TFetchResultsRespTupleScheme getScheme() { + return new TFetchResultsRespTupleScheme(); + } + } + + private static class TFetchResultsRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TFetchResultsResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetHasMoreRows()) { + optionals.set(0); + } + if (struct.isSetResults()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetHasMoreRows()) { + oprot.writeBool(struct.hasMoreRows); + } + if (struct.isSetResults()) { + struct.results.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TFetchResultsResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.hasMoreRows = iprot.readBool(); + struct.setHasMoreRowsIsSet(true); + } + if (incoming.get(1)) { + struct.results = new TRowSet(); + struct.results.read(iprot); + struct.setResultsIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java new file mode 100644 index 000000000000..cfd157f701b2 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetCatalogsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetCatalogsReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetCatalogsReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetCatalogsReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetCatalogsReq.class, metaDataMap); + } + + public TGetCatalogsReq() { + } + + public TGetCatalogsReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetCatalogsReq(TGetCatalogsReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + } + + public TGetCatalogsReq deepCopy() { + return new TGetCatalogsReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetCatalogsReq) + return this.equals((TGetCatalogsReq)that); + return false; + } + + public boolean equals(TGetCatalogsReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + builder.append(present_sessionHandle); + if (present_sessionHandle) + builder.append(sessionHandle); + + return builder.toHashCode(); + } + + public int compareTo(TGetCatalogsReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetCatalogsReq typedOther = (TGetCatalogsReq)other; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetCatalogsReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetCatalogsReqStandardSchemeFactory implements SchemeFactory { + public TGetCatalogsReqStandardScheme getScheme() { + return new TGetCatalogsReqStandardScheme(); + } + } + + private static class TGetCatalogsReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetCatalogsReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetCatalogsReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetCatalogsReqTupleSchemeFactory implements SchemeFactory { + public TGetCatalogsReqTupleScheme getScheme() { + return new TGetCatalogsReqTupleScheme(); + } + } + + private static class TGetCatalogsReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java new file mode 100644 index 000000000000..1c5a35437d41 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java @@ -0,0 +1,505 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetCatalogsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetCatalogsResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetCatalogsRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetCatalogsRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetCatalogsResp.class, metaDataMap); + } + + public TGetCatalogsResp() { + } + + public TGetCatalogsResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetCatalogsResp(TGetCatalogsResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetCatalogsResp deepCopy() { + return new TGetCatalogsResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetCatalogsResp) + return this.equals((TGetCatalogsResp)that); + return false; + } + + public boolean equals(TGetCatalogsResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + builder.append(present_operationHandle); + if (present_operationHandle) + builder.append(operationHandle); + + return builder.toHashCode(); + } + + public int compareTo(TGetCatalogsResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetCatalogsResp typedOther = (TGetCatalogsResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetCatalogsResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetCatalogsRespStandardSchemeFactory implements SchemeFactory { + public TGetCatalogsRespStandardScheme getScheme() { + return new TGetCatalogsRespStandardScheme(); + } + } + + private static class TGetCatalogsRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetCatalogsResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetCatalogsResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetCatalogsRespTupleSchemeFactory implements SchemeFactory { + public TGetCatalogsRespTupleScheme getScheme() { + return new TGetCatalogsRespTupleScheme(); + } + } + + private static class TGetCatalogsRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java new file mode 100644 index 000000000000..a2c793bd9592 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java @@ -0,0 +1,818 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetColumnsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetColumnsReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)4); + private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("columnName", org.apache.thrift.protocol.TType.STRING, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetColumnsReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetColumnsReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String catalogName; // optional + private String schemaName; // optional + private String tableName; // optional + private String columnName; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + CATALOG_NAME((short)2, "catalogName"), + SCHEMA_NAME((short)3, "schemaName"), + TABLE_NAME((short)4, "tableName"), + COLUMN_NAME((short)5, "columnName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // CATALOG_NAME + return CATALOG_NAME; + case 3: // SCHEMA_NAME + return SCHEMA_NAME; + case 4: // TABLE_NAME + return TABLE_NAME; + case 5: // COLUMN_NAME + return COLUMN_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME,_Fields.TABLE_NAME,_Fields.COLUMN_NAME}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + tmpMap.put(_Fields.COLUMN_NAME, new org.apache.thrift.meta_data.FieldMetaData("columnName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetColumnsReq.class, metaDataMap); + } + + public TGetColumnsReq() { + } + + public TGetColumnsReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetColumnsReq(TGetColumnsReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetCatalogName()) { + this.catalogName = other.catalogName; + } + if (other.isSetSchemaName()) { + this.schemaName = other.schemaName; + } + if (other.isSetTableName()) { + this.tableName = other.tableName; + } + if (other.isSetColumnName()) { + this.columnName = other.columnName; + } + } + + public TGetColumnsReq deepCopy() { + return new TGetColumnsReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.catalogName = null; + this.schemaName = null; + this.tableName = null; + this.columnName = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getCatalogName() { + return this.catalogName; + } + + public void setCatalogName(String catalogName) { + this.catalogName = catalogName; + } + + public void unsetCatalogName() { + this.catalogName = null; + } + + /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ + public boolean isSetCatalogName() { + return this.catalogName != null; + } + + public void setCatalogNameIsSet(boolean value) { + if (!value) { + this.catalogName = null; + } + } + + public String getSchemaName() { + return this.schemaName; + } + + public void setSchemaName(String schemaName) { + this.schemaName = schemaName; + } + + public void unsetSchemaName() { + this.schemaName = null; + } + + /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ + public boolean isSetSchemaName() { + return this.schemaName != null; + } + + public void setSchemaNameIsSet(boolean value) { + if (!value) { + this.schemaName = null; + } + } + + public String getTableName() { + return this.tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public void unsetTableName() { + this.tableName = null; + } + + /** Returns true if field tableName is set (has been assigned a value) and false otherwise */ + public boolean isSetTableName() { + return this.tableName != null; + } + + public void setTableNameIsSet(boolean value) { + if (!value) { + this.tableName = null; + } + } + + public String getColumnName() { + return this.columnName; + } + + public void setColumnName(String columnName) { + this.columnName = columnName; + } + + public void unsetColumnName() { + this.columnName = null; + } + + /** Returns true if field columnName is set (has been assigned a value) and false otherwise */ + public boolean isSetColumnName() { + return this.columnName != null; + } + + public void setColumnNameIsSet(boolean value) { + if (!value) { + this.columnName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case CATALOG_NAME: + if (value == null) { + unsetCatalogName(); + } else { + setCatalogName((String)value); + } + break; + + case SCHEMA_NAME: + if (value == null) { + unsetSchemaName(); + } else { + setSchemaName((String)value); + } + break; + + case TABLE_NAME: + if (value == null) { + unsetTableName(); + } else { + setTableName((String)value); + } + break; + + case COLUMN_NAME: + if (value == null) { + unsetColumnName(); + } else { + setColumnName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case CATALOG_NAME: + return getCatalogName(); + + case SCHEMA_NAME: + return getSchemaName(); + + case TABLE_NAME: + return getTableName(); + + case COLUMN_NAME: + return getColumnName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case CATALOG_NAME: + return isSetCatalogName(); + case SCHEMA_NAME: + return isSetSchemaName(); + case TABLE_NAME: + return isSetTableName(); + case COLUMN_NAME: + return isSetColumnName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetColumnsReq) + return this.equals((TGetColumnsReq)that); + return false; + } + + public boolean equals(TGetColumnsReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_catalogName = true && this.isSetCatalogName(); + boolean that_present_catalogName = true && that.isSetCatalogName(); + if (this_present_catalogName || that_present_catalogName) { + if (!(this_present_catalogName && that_present_catalogName)) + return false; + if (!this.catalogName.equals(that.catalogName)) + return false; + } + + boolean this_present_schemaName = true && this.isSetSchemaName(); + boolean that_present_schemaName = true && that.isSetSchemaName(); + if (this_present_schemaName || that_present_schemaName) { + if (!(this_present_schemaName && that_present_schemaName)) + return false; + if (!this.schemaName.equals(that.schemaName)) + return false; + } + + boolean this_present_tableName = true && this.isSetTableName(); + boolean that_present_tableName = true && that.isSetTableName(); + if (this_present_tableName || that_present_tableName) { + if (!(this_present_tableName && that_present_tableName)) + return false; + if (!this.tableName.equals(that.tableName)) + return false; + } + + boolean this_present_columnName = true && this.isSetColumnName(); + boolean that_present_columnName = true && that.isSetColumnName(); + if (this_present_columnName || that_present_columnName) { + if (!(this_present_columnName && that_present_columnName)) + return false; + if (!this.columnName.equals(that.columnName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + builder.append(present_sessionHandle); + if (present_sessionHandle) + builder.append(sessionHandle); + + boolean present_catalogName = true && (isSetCatalogName()); + builder.append(present_catalogName); + if (present_catalogName) + builder.append(catalogName); + + boolean present_schemaName = true && (isSetSchemaName()); + builder.append(present_schemaName); + if (present_schemaName) + builder.append(schemaName); + + boolean present_tableName = true && (isSetTableName()); + builder.append(present_tableName); + if (present_tableName) + builder.append(tableName); + + boolean present_columnName = true && (isSetColumnName()); + builder.append(present_columnName); + if (present_columnName) + builder.append(columnName); + + return builder.toHashCode(); + } + + public int compareTo(TGetColumnsReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetColumnsReq typedOther = (TGetColumnsReq)other; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(typedOther.isSetCatalogName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCatalogName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, typedOther.catalogName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(typedOther.isSetSchemaName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSchemaName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, typedOther.schemaName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetColumnName()).compareTo(typedOther.isSetColumnName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetColumnName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnName, typedOther.columnName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetColumnsReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (isSetCatalogName()) { + if (!first) sb.append(", "); + sb.append("catalogName:"); + if (this.catalogName == null) { + sb.append("null"); + } else { + sb.append(this.catalogName); + } + first = false; + } + if (isSetSchemaName()) { + if (!first) sb.append(", "); + sb.append("schemaName:"); + if (this.schemaName == null) { + sb.append("null"); + } else { + sb.append(this.schemaName); + } + first = false; + } + if (isSetTableName()) { + if (!first) sb.append(", "); + sb.append("tableName:"); + if (this.tableName == null) { + sb.append("null"); + } else { + sb.append(this.tableName); + } + first = false; + } + if (isSetColumnName()) { + if (!first) sb.append(", "); + sb.append("columnName:"); + if (this.columnName == null) { + sb.append("null"); + } else { + sb.append(this.columnName); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetColumnsReqStandardSchemeFactory implements SchemeFactory { + public TGetColumnsReqStandardScheme getScheme() { + return new TGetColumnsReqStandardScheme(); + } + } + + private static class TGetColumnsReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetColumnsReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // CATALOG_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SCHEMA_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // COLUMN_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.columnName = iprot.readString(); + struct.setColumnNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetColumnsReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.catalogName != null) { + if (struct.isSetCatalogName()) { + oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); + oprot.writeString(struct.catalogName); + oprot.writeFieldEnd(); + } + } + if (struct.schemaName != null) { + if (struct.isSetSchemaName()) { + oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); + oprot.writeString(struct.schemaName); + oprot.writeFieldEnd(); + } + } + if (struct.tableName != null) { + if (struct.isSetTableName()) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.tableName); + oprot.writeFieldEnd(); + } + } + if (struct.columnName != null) { + if (struct.isSetColumnName()) { + oprot.writeFieldBegin(COLUMN_NAME_FIELD_DESC); + oprot.writeString(struct.columnName); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetColumnsReqTupleSchemeFactory implements SchemeFactory { + public TGetColumnsReqTupleScheme getScheme() { + return new TGetColumnsReqTupleScheme(); + } + } + + private static class TGetColumnsReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetColumnsReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetCatalogName()) { + optionals.set(0); + } + if (struct.isSetSchemaName()) { + optionals.set(1); + } + if (struct.isSetTableName()) { + optionals.set(2); + } + if (struct.isSetColumnName()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetCatalogName()) { + oprot.writeString(struct.catalogName); + } + if (struct.isSetSchemaName()) { + oprot.writeString(struct.schemaName); + } + if (struct.isSetTableName()) { + oprot.writeString(struct.tableName); + } + if (struct.isSetColumnName()) { + oprot.writeString(struct.columnName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetColumnsReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } + if (incoming.get(1)) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } + if (incoming.get(2)) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } + if (incoming.get(3)) { + struct.columnName = iprot.readString(); + struct.setColumnNameIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java new file mode 100644 index 000000000000..d6cf1be6d304 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java @@ -0,0 +1,505 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetColumnsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetColumnsResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetColumnsRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetColumnsRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetColumnsResp.class, metaDataMap); + } + + public TGetColumnsResp() { + } + + public TGetColumnsResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetColumnsResp(TGetColumnsResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetColumnsResp deepCopy() { + return new TGetColumnsResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetColumnsResp) + return this.equals((TGetColumnsResp)that); + return false; + } + + public boolean equals(TGetColumnsResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + builder.append(present_operationHandle); + if (present_operationHandle) + builder.append(operationHandle); + + return builder.toHashCode(); + } + + public int compareTo(TGetColumnsResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetColumnsResp typedOther = (TGetColumnsResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetColumnsResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetColumnsRespStandardSchemeFactory implements SchemeFactory { + public TGetColumnsRespStandardScheme getScheme() { + return new TGetColumnsRespStandardScheme(); + } + } + + private static class TGetColumnsRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetColumnsResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetColumnsResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetColumnsRespTupleSchemeFactory implements SchemeFactory { + public TGetColumnsRespTupleScheme getScheme() { + return new TGetColumnsRespTupleScheme(); + } + } + + private static class TGetColumnsRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetColumnsResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetColumnsResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java new file mode 100644 index 000000000000..6c6bb00e43e4 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java @@ -0,0 +1,592 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetDelegationTokenReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetDelegationTokenReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField RENEWER_FIELD_DESC = new org.apache.thrift.protocol.TField("renewer", org.apache.thrift.protocol.TType.STRING, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetDelegationTokenReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetDelegationTokenReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String owner; // required + private String renewer; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + OWNER((short)2, "owner"), + RENEWER((short)3, "renewer"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // OWNER + return OWNER; + case 3: // RENEWER + return RENEWER; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.RENEWER, new org.apache.thrift.meta_data.FieldMetaData("renewer", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetDelegationTokenReq.class, metaDataMap); + } + + public TGetDelegationTokenReq() { + } + + public TGetDelegationTokenReq( + TSessionHandle sessionHandle, + String owner, + String renewer) + { + this(); + this.sessionHandle = sessionHandle; + this.owner = owner; + this.renewer = renewer; + } + + /** + * Performs a deep copy on other. + */ + public TGetDelegationTokenReq(TGetDelegationTokenReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetOwner()) { + this.owner = other.owner; + } + if (other.isSetRenewer()) { + this.renewer = other.renewer; + } + } + + public TGetDelegationTokenReq deepCopy() { + return new TGetDelegationTokenReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.owner = null; + this.renewer = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getOwner() { + return this.owner; + } + + public void setOwner(String owner) { + this.owner = owner; + } + + public void unsetOwner() { + this.owner = null; + } + + /** Returns true if field owner is set (has been assigned a value) and false otherwise */ + public boolean isSetOwner() { + return this.owner != null; + } + + public void setOwnerIsSet(boolean value) { + if (!value) { + this.owner = null; + } + } + + public String getRenewer() { + return this.renewer; + } + + public void setRenewer(String renewer) { + this.renewer = renewer; + } + + public void unsetRenewer() { + this.renewer = null; + } + + /** Returns true if field renewer is set (has been assigned a value) and false otherwise */ + public boolean isSetRenewer() { + return this.renewer != null; + } + + public void setRenewerIsSet(boolean value) { + if (!value) { + this.renewer = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case OWNER: + if (value == null) { + unsetOwner(); + } else { + setOwner((String)value); + } + break; + + case RENEWER: + if (value == null) { + unsetRenewer(); + } else { + setRenewer((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case OWNER: + return getOwner(); + + case RENEWER: + return getRenewer(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case OWNER: + return isSetOwner(); + case RENEWER: + return isSetRenewer(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetDelegationTokenReq) + return this.equals((TGetDelegationTokenReq)that); + return false; + } + + public boolean equals(TGetDelegationTokenReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_owner = true && this.isSetOwner(); + boolean that_present_owner = true && that.isSetOwner(); + if (this_present_owner || that_present_owner) { + if (!(this_present_owner && that_present_owner)) + return false; + if (!this.owner.equals(that.owner)) + return false; + } + + boolean this_present_renewer = true && this.isSetRenewer(); + boolean that_present_renewer = true && that.isSetRenewer(); + if (this_present_renewer || that_present_renewer) { + if (!(this_present_renewer && that_present_renewer)) + return false; + if (!this.renewer.equals(that.renewer)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + builder.append(present_sessionHandle); + if (present_sessionHandle) + builder.append(sessionHandle); + + boolean present_owner = true && (isSetOwner()); + builder.append(present_owner); + if (present_owner) + builder.append(owner); + + boolean present_renewer = true && (isSetRenewer()); + builder.append(present_renewer); + if (present_renewer) + builder.append(renewer); + + return builder.toHashCode(); + } + + public int compareTo(TGetDelegationTokenReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetDelegationTokenReq typedOther = (TGetDelegationTokenReq)other; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOwner()).compareTo(typedOther.isSetOwner()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOwner()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, typedOther.owner); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetRenewer()).compareTo(typedOther.isSetRenewer()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRenewer()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.renewer, typedOther.renewer); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetDelegationTokenReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (!first) sb.append(", "); + sb.append("owner:"); + if (this.owner == null) { + sb.append("null"); + } else { + sb.append(this.owner); + } + first = false; + if (!first) sb.append(", "); + sb.append("renewer:"); + if (this.renewer == null) { + sb.append("null"); + } else { + sb.append(this.renewer); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + if (!isSetOwner()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'owner' is unset! Struct:" + toString()); + } + + if (!isSetRenewer()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'renewer' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetDelegationTokenReqStandardSchemeFactory implements SchemeFactory { + public TGetDelegationTokenReqStandardScheme getScheme() { + return new TGetDelegationTokenReqStandardScheme(); + } + } + + private static class TGetDelegationTokenReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OWNER + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.owner = iprot.readString(); + struct.setOwnerIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // RENEWER + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.renewer = iprot.readString(); + struct.setRenewerIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.owner != null) { + oprot.writeFieldBegin(OWNER_FIELD_DESC); + oprot.writeString(struct.owner); + oprot.writeFieldEnd(); + } + if (struct.renewer != null) { + oprot.writeFieldBegin(RENEWER_FIELD_DESC); + oprot.writeString(struct.renewer); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetDelegationTokenReqTupleSchemeFactory implements SchemeFactory { + public TGetDelegationTokenReqTupleScheme getScheme() { + return new TGetDelegationTokenReqTupleScheme(); + } + } + + private static class TGetDelegationTokenReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + oprot.writeString(struct.owner); + oprot.writeString(struct.renewer); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + struct.owner = iprot.readString(); + struct.setOwnerIsSet(true); + struct.renewer = iprot.readString(); + struct.setRenewerIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java new file mode 100644 index 000000000000..d14c5e029a35 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java @@ -0,0 +1,500 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetDelegationTokenResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetDelegationTokenResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField DELEGATION_TOKEN_FIELD_DESC = new org.apache.thrift.protocol.TField("delegationToken", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetDelegationTokenRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetDelegationTokenRespTupleSchemeFactory()); + } + + private TStatus status; // required + private String delegationToken; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + DELEGATION_TOKEN((short)2, "delegationToken"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // DELEGATION_TOKEN + return DELEGATION_TOKEN; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.DELEGATION_TOKEN}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.DELEGATION_TOKEN, new org.apache.thrift.meta_data.FieldMetaData("delegationToken", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetDelegationTokenResp.class, metaDataMap); + } + + public TGetDelegationTokenResp() { + } + + public TGetDelegationTokenResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetDelegationTokenResp(TGetDelegationTokenResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetDelegationToken()) { + this.delegationToken = other.delegationToken; + } + } + + public TGetDelegationTokenResp deepCopy() { + return new TGetDelegationTokenResp(this); + } + + @Override + public void clear() { + this.status = null; + this.delegationToken = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public String getDelegationToken() { + return this.delegationToken; + } + + public void setDelegationToken(String delegationToken) { + this.delegationToken = delegationToken; + } + + public void unsetDelegationToken() { + this.delegationToken = null; + } + + /** Returns true if field delegationToken is set (has been assigned a value) and false otherwise */ + public boolean isSetDelegationToken() { + return this.delegationToken != null; + } + + public void setDelegationTokenIsSet(boolean value) { + if (!value) { + this.delegationToken = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case DELEGATION_TOKEN: + if (value == null) { + unsetDelegationToken(); + } else { + setDelegationToken((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case DELEGATION_TOKEN: + return getDelegationToken(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case DELEGATION_TOKEN: + return isSetDelegationToken(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetDelegationTokenResp) + return this.equals((TGetDelegationTokenResp)that); + return false; + } + + public boolean equals(TGetDelegationTokenResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_delegationToken = true && this.isSetDelegationToken(); + boolean that_present_delegationToken = true && that.isSetDelegationToken(); + if (this_present_delegationToken || that_present_delegationToken) { + if (!(this_present_delegationToken && that_present_delegationToken)) + return false; + if (!this.delegationToken.equals(that.delegationToken)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_delegationToken = true && (isSetDelegationToken()); + builder.append(present_delegationToken); + if (present_delegationToken) + builder.append(delegationToken); + + return builder.toHashCode(); + } + + public int compareTo(TGetDelegationTokenResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetDelegationTokenResp typedOther = (TGetDelegationTokenResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetDelegationToken()).compareTo(typedOther.isSetDelegationToken()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDelegationToken()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.delegationToken, typedOther.delegationToken); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetDelegationTokenResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetDelegationToken()) { + if (!first) sb.append(", "); + sb.append("delegationToken:"); + if (this.delegationToken == null) { + sb.append("null"); + } else { + sb.append(this.delegationToken); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetDelegationTokenRespStandardSchemeFactory implements SchemeFactory { + public TGetDelegationTokenRespStandardScheme getScheme() { + return new TGetDelegationTokenRespStandardScheme(); + } + } + + private static class TGetDelegationTokenRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // DELEGATION_TOKEN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.delegationToken = iprot.readString(); + struct.setDelegationTokenIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.delegationToken != null) { + if (struct.isSetDelegationToken()) { + oprot.writeFieldBegin(DELEGATION_TOKEN_FIELD_DESC); + oprot.writeString(struct.delegationToken); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetDelegationTokenRespTupleSchemeFactory implements SchemeFactory { + public TGetDelegationTokenRespTupleScheme getScheme() { + return new TGetDelegationTokenRespTupleScheme(); + } + } + + private static class TGetDelegationTokenRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetDelegationToken()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetDelegationToken()) { + oprot.writeString(struct.delegationToken); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.delegationToken = iprot.readString(); + struct.setDelegationTokenIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java new file mode 100644 index 000000000000..ff45ee0386cb --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java @@ -0,0 +1,707 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetFunctionsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetFunctionsReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField FUNCTION_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("functionName", org.apache.thrift.protocol.TType.STRING, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetFunctionsReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetFunctionsReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String catalogName; // optional + private String schemaName; // optional + private String functionName; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + CATALOG_NAME((short)2, "catalogName"), + SCHEMA_NAME((short)3, "schemaName"), + FUNCTION_NAME((short)4, "functionName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // CATALOG_NAME + return CATALOG_NAME; + case 3: // SCHEMA_NAME + return SCHEMA_NAME; + case 4: // FUNCTION_NAME + return FUNCTION_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + tmpMap.put(_Fields.FUNCTION_NAME, new org.apache.thrift.meta_data.FieldMetaData("functionName", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetFunctionsReq.class, metaDataMap); + } + + public TGetFunctionsReq() { + } + + public TGetFunctionsReq( + TSessionHandle sessionHandle, + String functionName) + { + this(); + this.sessionHandle = sessionHandle; + this.functionName = functionName; + } + + /** + * Performs a deep copy on other. + */ + public TGetFunctionsReq(TGetFunctionsReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetCatalogName()) { + this.catalogName = other.catalogName; + } + if (other.isSetSchemaName()) { + this.schemaName = other.schemaName; + } + if (other.isSetFunctionName()) { + this.functionName = other.functionName; + } + } + + public TGetFunctionsReq deepCopy() { + return new TGetFunctionsReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.catalogName = null; + this.schemaName = null; + this.functionName = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getCatalogName() { + return this.catalogName; + } + + public void setCatalogName(String catalogName) { + this.catalogName = catalogName; + } + + public void unsetCatalogName() { + this.catalogName = null; + } + + /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ + public boolean isSetCatalogName() { + return this.catalogName != null; + } + + public void setCatalogNameIsSet(boolean value) { + if (!value) { + this.catalogName = null; + } + } + + public String getSchemaName() { + return this.schemaName; + } + + public void setSchemaName(String schemaName) { + this.schemaName = schemaName; + } + + public void unsetSchemaName() { + this.schemaName = null; + } + + /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ + public boolean isSetSchemaName() { + return this.schemaName != null; + } + + public void setSchemaNameIsSet(boolean value) { + if (!value) { + this.schemaName = null; + } + } + + public String getFunctionName() { + return this.functionName; + } + + public void setFunctionName(String functionName) { + this.functionName = functionName; + } + + public void unsetFunctionName() { + this.functionName = null; + } + + /** Returns true if field functionName is set (has been assigned a value) and false otherwise */ + public boolean isSetFunctionName() { + return this.functionName != null; + } + + public void setFunctionNameIsSet(boolean value) { + if (!value) { + this.functionName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case CATALOG_NAME: + if (value == null) { + unsetCatalogName(); + } else { + setCatalogName((String)value); + } + break; + + case SCHEMA_NAME: + if (value == null) { + unsetSchemaName(); + } else { + setSchemaName((String)value); + } + break; + + case FUNCTION_NAME: + if (value == null) { + unsetFunctionName(); + } else { + setFunctionName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case CATALOG_NAME: + return getCatalogName(); + + case SCHEMA_NAME: + return getSchemaName(); + + case FUNCTION_NAME: + return getFunctionName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case CATALOG_NAME: + return isSetCatalogName(); + case SCHEMA_NAME: + return isSetSchemaName(); + case FUNCTION_NAME: + return isSetFunctionName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetFunctionsReq) + return this.equals((TGetFunctionsReq)that); + return false; + } + + public boolean equals(TGetFunctionsReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_catalogName = true && this.isSetCatalogName(); + boolean that_present_catalogName = true && that.isSetCatalogName(); + if (this_present_catalogName || that_present_catalogName) { + if (!(this_present_catalogName && that_present_catalogName)) + return false; + if (!this.catalogName.equals(that.catalogName)) + return false; + } + + boolean this_present_schemaName = true && this.isSetSchemaName(); + boolean that_present_schemaName = true && that.isSetSchemaName(); + if (this_present_schemaName || that_present_schemaName) { + if (!(this_present_schemaName && that_present_schemaName)) + return false; + if (!this.schemaName.equals(that.schemaName)) + return false; + } + + boolean this_present_functionName = true && this.isSetFunctionName(); + boolean that_present_functionName = true && that.isSetFunctionName(); + if (this_present_functionName || that_present_functionName) { + if (!(this_present_functionName && that_present_functionName)) + return false; + if (!this.functionName.equals(that.functionName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + builder.append(present_sessionHandle); + if (present_sessionHandle) + builder.append(sessionHandle); + + boolean present_catalogName = true && (isSetCatalogName()); + builder.append(present_catalogName); + if (present_catalogName) + builder.append(catalogName); + + boolean present_schemaName = true && (isSetSchemaName()); + builder.append(present_schemaName); + if (present_schemaName) + builder.append(schemaName); + + boolean present_functionName = true && (isSetFunctionName()); + builder.append(present_functionName); + if (present_functionName) + builder.append(functionName); + + return builder.toHashCode(); + } + + public int compareTo(TGetFunctionsReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetFunctionsReq typedOther = (TGetFunctionsReq)other; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(typedOther.isSetCatalogName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCatalogName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, typedOther.catalogName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(typedOther.isSetSchemaName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSchemaName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, typedOther.schemaName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetFunctionName()).compareTo(typedOther.isSetFunctionName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFunctionName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.functionName, typedOther.functionName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetFunctionsReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (isSetCatalogName()) { + if (!first) sb.append(", "); + sb.append("catalogName:"); + if (this.catalogName == null) { + sb.append("null"); + } else { + sb.append(this.catalogName); + } + first = false; + } + if (isSetSchemaName()) { + if (!first) sb.append(", "); + sb.append("schemaName:"); + if (this.schemaName == null) { + sb.append("null"); + } else { + sb.append(this.schemaName); + } + first = false; + } + if (!first) sb.append(", "); + sb.append("functionName:"); + if (this.functionName == null) { + sb.append("null"); + } else { + sb.append(this.functionName); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + if (!isSetFunctionName()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'functionName' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetFunctionsReqStandardSchemeFactory implements SchemeFactory { + public TGetFunctionsReqStandardScheme getScheme() { + return new TGetFunctionsReqStandardScheme(); + } + } + + private static class TGetFunctionsReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetFunctionsReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // CATALOG_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SCHEMA_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // FUNCTION_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.functionName = iprot.readString(); + struct.setFunctionNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetFunctionsReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.catalogName != null) { + if (struct.isSetCatalogName()) { + oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); + oprot.writeString(struct.catalogName); + oprot.writeFieldEnd(); + } + } + if (struct.schemaName != null) { + if (struct.isSetSchemaName()) { + oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); + oprot.writeString(struct.schemaName); + oprot.writeFieldEnd(); + } + } + if (struct.functionName != null) { + oprot.writeFieldBegin(FUNCTION_NAME_FIELD_DESC); + oprot.writeString(struct.functionName); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetFunctionsReqTupleSchemeFactory implements SchemeFactory { + public TGetFunctionsReqTupleScheme getScheme() { + return new TGetFunctionsReqTupleScheme(); + } + } + + private static class TGetFunctionsReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + oprot.writeString(struct.functionName); + BitSet optionals = new BitSet(); + if (struct.isSetCatalogName()) { + optionals.set(0); + } + if (struct.isSetSchemaName()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetCatalogName()) { + oprot.writeString(struct.catalogName); + } + if (struct.isSetSchemaName()) { + oprot.writeString(struct.schemaName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + struct.functionName = iprot.readString(); + struct.setFunctionNameIsSet(true); + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } + if (incoming.get(1)) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java new file mode 100644 index 000000000000..3adafdacb54e --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java @@ -0,0 +1,505 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetFunctionsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetFunctionsResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetFunctionsRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetFunctionsRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetFunctionsResp.class, metaDataMap); + } + + public TGetFunctionsResp() { + } + + public TGetFunctionsResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetFunctionsResp(TGetFunctionsResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetFunctionsResp deepCopy() { + return new TGetFunctionsResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetFunctionsResp) + return this.equals((TGetFunctionsResp)that); + return false; + } + + public boolean equals(TGetFunctionsResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + builder.append(present_operationHandle); + if (present_operationHandle) + builder.append(operationHandle); + + return builder.toHashCode(); + } + + public int compareTo(TGetFunctionsResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetFunctionsResp typedOther = (TGetFunctionsResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetFunctionsResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetFunctionsRespStandardSchemeFactory implements SchemeFactory { + public TGetFunctionsRespStandardScheme getScheme() { + return new TGetFunctionsRespStandardScheme(); + } + } + + private static class TGetFunctionsRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetFunctionsResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetFunctionsResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetFunctionsRespTupleSchemeFactory implements SchemeFactory { + public TGetFunctionsRespTupleScheme getScheme() { + return new TGetFunctionsRespTupleScheme(); + } + } + + private static class TGetFunctionsRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java new file mode 100644 index 000000000000..0139bf04ec7d --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java @@ -0,0 +1,503 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetInfoReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetInfoReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField INFO_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("infoType", org.apache.thrift.protocol.TType.I32, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetInfoReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetInfoReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private TGetInfoType infoType; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + /** + * + * @see TGetInfoType + */ + INFO_TYPE((short)2, "infoType"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // INFO_TYPE + return INFO_TYPE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.INFO_TYPE, new org.apache.thrift.meta_data.FieldMetaData("infoType", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TGetInfoType.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetInfoReq.class, metaDataMap); + } + + public TGetInfoReq() { + } + + public TGetInfoReq( + TSessionHandle sessionHandle, + TGetInfoType infoType) + { + this(); + this.sessionHandle = sessionHandle; + this.infoType = infoType; + } + + /** + * Performs a deep copy on other. + */ + public TGetInfoReq(TGetInfoReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetInfoType()) { + this.infoType = other.infoType; + } + } + + public TGetInfoReq deepCopy() { + return new TGetInfoReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.infoType = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + /** + * + * @see TGetInfoType + */ + public TGetInfoType getInfoType() { + return this.infoType; + } + + /** + * + * @see TGetInfoType + */ + public void setInfoType(TGetInfoType infoType) { + this.infoType = infoType; + } + + public void unsetInfoType() { + this.infoType = null; + } + + /** Returns true if field infoType is set (has been assigned a value) and false otherwise */ + public boolean isSetInfoType() { + return this.infoType != null; + } + + public void setInfoTypeIsSet(boolean value) { + if (!value) { + this.infoType = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case INFO_TYPE: + if (value == null) { + unsetInfoType(); + } else { + setInfoType((TGetInfoType)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case INFO_TYPE: + return getInfoType(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case INFO_TYPE: + return isSetInfoType(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetInfoReq) + return this.equals((TGetInfoReq)that); + return false; + } + + public boolean equals(TGetInfoReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_infoType = true && this.isSetInfoType(); + boolean that_present_infoType = true && that.isSetInfoType(); + if (this_present_infoType || that_present_infoType) { + if (!(this_present_infoType && that_present_infoType)) + return false; + if (!this.infoType.equals(that.infoType)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + builder.append(present_sessionHandle); + if (present_sessionHandle) + builder.append(sessionHandle); + + boolean present_infoType = true && (isSetInfoType()); + builder.append(present_infoType); + if (present_infoType) + builder.append(infoType.getValue()); + + return builder.toHashCode(); + } + + public int compareTo(TGetInfoReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetInfoReq typedOther = (TGetInfoReq)other; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetInfoType()).compareTo(typedOther.isSetInfoType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetInfoType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.infoType, typedOther.infoType); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetInfoReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (!first) sb.append(", "); + sb.append("infoType:"); + if (this.infoType == null) { + sb.append("null"); + } else { + sb.append(this.infoType); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + if (!isSetInfoType()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'infoType' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetInfoReqStandardSchemeFactory implements SchemeFactory { + public TGetInfoReqStandardScheme getScheme() { + return new TGetInfoReqStandardScheme(); + } + } + + private static class TGetInfoReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetInfoReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // INFO_TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.infoType = TGetInfoType.findByValue(iprot.readI32()); + struct.setInfoTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetInfoReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.infoType != null) { + oprot.writeFieldBegin(INFO_TYPE_FIELD_DESC); + oprot.writeI32(struct.infoType.getValue()); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetInfoReqTupleSchemeFactory implements SchemeFactory { + public TGetInfoReqTupleScheme getScheme() { + return new TGetInfoReqTupleScheme(); + } + } + + private static class TGetInfoReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetInfoReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + oprot.writeI32(struct.infoType.getValue()); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetInfoReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + struct.infoType = TGetInfoType.findByValue(iprot.readI32()); + struct.setInfoTypeIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java new file mode 100644 index 000000000000..2faaa9211b3b --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java @@ -0,0 +1,493 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetInfoResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetInfoResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField INFO_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("infoValue", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetInfoRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetInfoRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TGetInfoValue infoValue; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + INFO_VALUE((short)2, "infoValue"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // INFO_VALUE + return INFO_VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.INFO_VALUE, new org.apache.thrift.meta_data.FieldMetaData("infoValue", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetInfoValue.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetInfoResp.class, metaDataMap); + } + + public TGetInfoResp() { + } + + public TGetInfoResp( + TStatus status, + TGetInfoValue infoValue) + { + this(); + this.status = status; + this.infoValue = infoValue; + } + + /** + * Performs a deep copy on other. + */ + public TGetInfoResp(TGetInfoResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetInfoValue()) { + this.infoValue = new TGetInfoValue(other.infoValue); + } + } + + public TGetInfoResp deepCopy() { + return new TGetInfoResp(this); + } + + @Override + public void clear() { + this.status = null; + this.infoValue = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TGetInfoValue getInfoValue() { + return this.infoValue; + } + + public void setInfoValue(TGetInfoValue infoValue) { + this.infoValue = infoValue; + } + + public void unsetInfoValue() { + this.infoValue = null; + } + + /** Returns true if field infoValue is set (has been assigned a value) and false otherwise */ + public boolean isSetInfoValue() { + return this.infoValue != null; + } + + public void setInfoValueIsSet(boolean value) { + if (!value) { + this.infoValue = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case INFO_VALUE: + if (value == null) { + unsetInfoValue(); + } else { + setInfoValue((TGetInfoValue)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case INFO_VALUE: + return getInfoValue(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case INFO_VALUE: + return isSetInfoValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetInfoResp) + return this.equals((TGetInfoResp)that); + return false; + } + + public boolean equals(TGetInfoResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_infoValue = true && this.isSetInfoValue(); + boolean that_present_infoValue = true && that.isSetInfoValue(); + if (this_present_infoValue || that_present_infoValue) { + if (!(this_present_infoValue && that_present_infoValue)) + return false; + if (!this.infoValue.equals(that.infoValue)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_infoValue = true && (isSetInfoValue()); + builder.append(present_infoValue); + if (present_infoValue) + builder.append(infoValue); + + return builder.toHashCode(); + } + + public int compareTo(TGetInfoResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetInfoResp typedOther = (TGetInfoResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetInfoValue()).compareTo(typedOther.isSetInfoValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetInfoValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.infoValue, typedOther.infoValue); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetInfoResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (!first) sb.append(", "); + sb.append("infoValue:"); + if (this.infoValue == null) { + sb.append("null"); + } else { + sb.append(this.infoValue); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + if (!isSetInfoValue()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'infoValue' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetInfoRespStandardSchemeFactory implements SchemeFactory { + public TGetInfoRespStandardScheme getScheme() { + return new TGetInfoRespStandardScheme(); + } + } + + private static class TGetInfoRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetInfoResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // INFO_VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.infoValue = new TGetInfoValue(); + struct.infoValue.read(iprot); + struct.setInfoValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetInfoResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.infoValue != null) { + oprot.writeFieldBegin(INFO_VALUE_FIELD_DESC); + struct.infoValue.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetInfoRespTupleSchemeFactory implements SchemeFactory { + public TGetInfoRespTupleScheme getScheme() { + return new TGetInfoRespTupleScheme(); + } + } + + private static class TGetInfoRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetInfoResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + struct.infoValue.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetInfoResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + struct.infoValue = new TGetInfoValue(); + struct.infoValue.read(iprot); + struct.setInfoValueIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java new file mode 100644 index 000000000000..d9dd62414f00 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java @@ -0,0 +1,180 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TGetInfoType implements org.apache.thrift.TEnum { + CLI_MAX_DRIVER_CONNECTIONS(0), + CLI_MAX_CONCURRENT_ACTIVITIES(1), + CLI_DATA_SOURCE_NAME(2), + CLI_FETCH_DIRECTION(8), + CLI_SERVER_NAME(13), + CLI_SEARCH_PATTERN_ESCAPE(14), + CLI_DBMS_NAME(17), + CLI_DBMS_VER(18), + CLI_ACCESSIBLE_TABLES(19), + CLI_ACCESSIBLE_PROCEDURES(20), + CLI_CURSOR_COMMIT_BEHAVIOR(23), + CLI_DATA_SOURCE_READ_ONLY(25), + CLI_DEFAULT_TXN_ISOLATION(26), + CLI_IDENTIFIER_CASE(28), + CLI_IDENTIFIER_QUOTE_CHAR(29), + CLI_MAX_COLUMN_NAME_LEN(30), + CLI_MAX_CURSOR_NAME_LEN(31), + CLI_MAX_SCHEMA_NAME_LEN(32), + CLI_MAX_CATALOG_NAME_LEN(34), + CLI_MAX_TABLE_NAME_LEN(35), + CLI_SCROLL_CONCURRENCY(43), + CLI_TXN_CAPABLE(46), + CLI_USER_NAME(47), + CLI_TXN_ISOLATION_OPTION(72), + CLI_INTEGRITY(73), + CLI_GETDATA_EXTENSIONS(81), + CLI_NULL_COLLATION(85), + CLI_ALTER_TABLE(86), + CLI_ORDER_BY_COLUMNS_IN_SELECT(90), + CLI_SPECIAL_CHARACTERS(94), + CLI_MAX_COLUMNS_IN_GROUP_BY(97), + CLI_MAX_COLUMNS_IN_INDEX(98), + CLI_MAX_COLUMNS_IN_ORDER_BY(99), + CLI_MAX_COLUMNS_IN_SELECT(100), + CLI_MAX_COLUMNS_IN_TABLE(101), + CLI_MAX_INDEX_SIZE(102), + CLI_MAX_ROW_SIZE(104), + CLI_MAX_STATEMENT_LEN(105), + CLI_MAX_TABLES_IN_SELECT(106), + CLI_MAX_USER_NAME_LEN(107), + CLI_OJ_CAPABILITIES(115), + CLI_XOPEN_CLI_YEAR(10000), + CLI_CURSOR_SENSITIVITY(10001), + CLI_DESCRIBE_PARAMETER(10002), + CLI_CATALOG_NAME(10003), + CLI_COLLATION_SEQ(10004), + CLI_MAX_IDENTIFIER_LEN(10005); + + private final int value; + + private TGetInfoType(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TGetInfoType findByValue(int value) { + switch (value) { + case 0: + return CLI_MAX_DRIVER_CONNECTIONS; + case 1: + return CLI_MAX_CONCURRENT_ACTIVITIES; + case 2: + return CLI_DATA_SOURCE_NAME; + case 8: + return CLI_FETCH_DIRECTION; + case 13: + return CLI_SERVER_NAME; + case 14: + return CLI_SEARCH_PATTERN_ESCAPE; + case 17: + return CLI_DBMS_NAME; + case 18: + return CLI_DBMS_VER; + case 19: + return CLI_ACCESSIBLE_TABLES; + case 20: + return CLI_ACCESSIBLE_PROCEDURES; + case 23: + return CLI_CURSOR_COMMIT_BEHAVIOR; + case 25: + return CLI_DATA_SOURCE_READ_ONLY; + case 26: + return CLI_DEFAULT_TXN_ISOLATION; + case 28: + return CLI_IDENTIFIER_CASE; + case 29: + return CLI_IDENTIFIER_QUOTE_CHAR; + case 30: + return CLI_MAX_COLUMN_NAME_LEN; + case 31: + return CLI_MAX_CURSOR_NAME_LEN; + case 32: + return CLI_MAX_SCHEMA_NAME_LEN; + case 34: + return CLI_MAX_CATALOG_NAME_LEN; + case 35: + return CLI_MAX_TABLE_NAME_LEN; + case 43: + return CLI_SCROLL_CONCURRENCY; + case 46: + return CLI_TXN_CAPABLE; + case 47: + return CLI_USER_NAME; + case 72: + return CLI_TXN_ISOLATION_OPTION; + case 73: + return CLI_INTEGRITY; + case 81: + return CLI_GETDATA_EXTENSIONS; + case 85: + return CLI_NULL_COLLATION; + case 86: + return CLI_ALTER_TABLE; + case 90: + return CLI_ORDER_BY_COLUMNS_IN_SELECT; + case 94: + return CLI_SPECIAL_CHARACTERS; + case 97: + return CLI_MAX_COLUMNS_IN_GROUP_BY; + case 98: + return CLI_MAX_COLUMNS_IN_INDEX; + case 99: + return CLI_MAX_COLUMNS_IN_ORDER_BY; + case 100: + return CLI_MAX_COLUMNS_IN_SELECT; + case 101: + return CLI_MAX_COLUMNS_IN_TABLE; + case 102: + return CLI_MAX_INDEX_SIZE; + case 104: + return CLI_MAX_ROW_SIZE; + case 105: + return CLI_MAX_STATEMENT_LEN; + case 106: + return CLI_MAX_TABLES_IN_SELECT; + case 107: + return CLI_MAX_USER_NAME_LEN; + case 115: + return CLI_OJ_CAPABILITIES; + case 10000: + return CLI_XOPEN_CLI_YEAR; + case 10001: + return CLI_CURSOR_SENSITIVITY; + case 10002: + return CLI_DESCRIBE_PARAMETER; + case 10003: + return CLI_CATALOG_NAME; + case 10004: + return CLI_COLLATION_SEQ; + case 10005: + return CLI_MAX_IDENTIFIER_LEN; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java new file mode 100644 index 000000000000..4fe59b1c5146 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java @@ -0,0 +1,593 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetInfoValue extends org.apache.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetInfoValue"); + private static final org.apache.thrift.protocol.TField STRING_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("stringValue", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField SMALL_INT_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("smallIntValue", org.apache.thrift.protocol.TType.I16, (short)2); + private static final org.apache.thrift.protocol.TField INTEGER_BITMASK_FIELD_DESC = new org.apache.thrift.protocol.TField("integerBitmask", org.apache.thrift.protocol.TType.I32, (short)3); + private static final org.apache.thrift.protocol.TField INTEGER_FLAG_FIELD_DESC = new org.apache.thrift.protocol.TField("integerFlag", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField BINARY_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("binaryValue", org.apache.thrift.protocol.TType.I32, (short)5); + private static final org.apache.thrift.protocol.TField LEN_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("lenValue", org.apache.thrift.protocol.TType.I64, (short)6); + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STRING_VALUE((short)1, "stringValue"), + SMALL_INT_VALUE((short)2, "smallIntValue"), + INTEGER_BITMASK((short)3, "integerBitmask"), + INTEGER_FLAG((short)4, "integerFlag"), + BINARY_VALUE((short)5, "binaryValue"), + LEN_VALUE((short)6, "lenValue"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STRING_VALUE + return STRING_VALUE; + case 2: // SMALL_INT_VALUE + return SMALL_INT_VALUE; + case 3: // INTEGER_BITMASK + return INTEGER_BITMASK; + case 4: // INTEGER_FLAG + return INTEGER_FLAG; + case 5: // BINARY_VALUE + return BINARY_VALUE; + case 6: // LEN_VALUE + return LEN_VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STRING_VALUE, new org.apache.thrift.meta_data.FieldMetaData("stringValue", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.SMALL_INT_VALUE, new org.apache.thrift.meta_data.FieldMetaData("smallIntValue", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); + tmpMap.put(_Fields.INTEGER_BITMASK, new org.apache.thrift.meta_data.FieldMetaData("integerBitmask", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.INTEGER_FLAG, new org.apache.thrift.meta_data.FieldMetaData("integerFlag", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.BINARY_VALUE, new org.apache.thrift.meta_data.FieldMetaData("binaryValue", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.LEN_VALUE, new org.apache.thrift.meta_data.FieldMetaData("lenValue", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetInfoValue.class, metaDataMap); + } + + public TGetInfoValue() { + super(); + } + + public TGetInfoValue(_Fields setField, Object value) { + super(setField, value); + } + + public TGetInfoValue(TGetInfoValue other) { + super(other); + } + public TGetInfoValue deepCopy() { + return new TGetInfoValue(this); + } + + public static TGetInfoValue stringValue(String value) { + TGetInfoValue x = new TGetInfoValue(); + x.setStringValue(value); + return x; + } + + public static TGetInfoValue smallIntValue(short value) { + TGetInfoValue x = new TGetInfoValue(); + x.setSmallIntValue(value); + return x; + } + + public static TGetInfoValue integerBitmask(int value) { + TGetInfoValue x = new TGetInfoValue(); + x.setIntegerBitmask(value); + return x; + } + + public static TGetInfoValue integerFlag(int value) { + TGetInfoValue x = new TGetInfoValue(); + x.setIntegerFlag(value); + return x; + } + + public static TGetInfoValue binaryValue(int value) { + TGetInfoValue x = new TGetInfoValue(); + x.setBinaryValue(value); + return x; + } + + public static TGetInfoValue lenValue(long value) { + TGetInfoValue x = new TGetInfoValue(); + x.setLenValue(value); + return x; + } + + + @Override + protected void checkType(_Fields setField, Object value) throws ClassCastException { + switch (setField) { + case STRING_VALUE: + if (value instanceof String) { + break; + } + throw new ClassCastException("Was expecting value of type String for field 'stringValue', but got " + value.getClass().getSimpleName()); + case SMALL_INT_VALUE: + if (value instanceof Short) { + break; + } + throw new ClassCastException("Was expecting value of type Short for field 'smallIntValue', but got " + value.getClass().getSimpleName()); + case INTEGER_BITMASK: + if (value instanceof Integer) { + break; + } + throw new ClassCastException("Was expecting value of type Integer for field 'integerBitmask', but got " + value.getClass().getSimpleName()); + case INTEGER_FLAG: + if (value instanceof Integer) { + break; + } + throw new ClassCastException("Was expecting value of type Integer for field 'integerFlag', but got " + value.getClass().getSimpleName()); + case BINARY_VALUE: + if (value instanceof Integer) { + break; + } + throw new ClassCastException("Was expecting value of type Integer for field 'binaryValue', but got " + value.getClass().getSimpleName()); + case LEN_VALUE: + if (value instanceof Long) { + break; + } + throw new ClassCastException("Was expecting value of type Long for field 'lenValue', but got " + value.getClass().getSimpleName()); + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(field.id); + if (setField != null) { + switch (setField) { + case STRING_VALUE: + if (field.type == STRING_VALUE_FIELD_DESC.type) { + String stringValue; + stringValue = iprot.readString(); + return stringValue; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case SMALL_INT_VALUE: + if (field.type == SMALL_INT_VALUE_FIELD_DESC.type) { + Short smallIntValue; + smallIntValue = iprot.readI16(); + return smallIntValue; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case INTEGER_BITMASK: + if (field.type == INTEGER_BITMASK_FIELD_DESC.type) { + Integer integerBitmask; + integerBitmask = iprot.readI32(); + return integerBitmask; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case INTEGER_FLAG: + if (field.type == INTEGER_FLAG_FIELD_DESC.type) { + Integer integerFlag; + integerFlag = iprot.readI32(); + return integerFlag; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case BINARY_VALUE: + if (field.type == BINARY_VALUE_FIELD_DESC.type) { + Integer binaryValue; + binaryValue = iprot.readI32(); + return binaryValue; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case LEN_VALUE: + if (field.type == LEN_VALUE_FIELD_DESC.type) { + Long lenValue; + lenValue = iprot.readI64(); + return lenValue; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case STRING_VALUE: + String stringValue = (String)value_; + oprot.writeString(stringValue); + return; + case SMALL_INT_VALUE: + Short smallIntValue = (Short)value_; + oprot.writeI16(smallIntValue); + return; + case INTEGER_BITMASK: + Integer integerBitmask = (Integer)value_; + oprot.writeI32(integerBitmask); + return; + case INTEGER_FLAG: + Integer integerFlag = (Integer)value_; + oprot.writeI32(integerFlag); + return; + case BINARY_VALUE: + Integer binaryValue = (Integer)value_; + oprot.writeI32(binaryValue); + return; + case LEN_VALUE: + Long lenValue = (Long)value_; + oprot.writeI64(lenValue); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(fieldID); + if (setField != null) { + switch (setField) { + case STRING_VALUE: + String stringValue; + stringValue = iprot.readString(); + return stringValue; + case SMALL_INT_VALUE: + Short smallIntValue; + smallIntValue = iprot.readI16(); + return smallIntValue; + case INTEGER_BITMASK: + Integer integerBitmask; + integerBitmask = iprot.readI32(); + return integerBitmask; + case INTEGER_FLAG: + Integer integerFlag; + integerFlag = iprot.readI32(); + return integerFlag; + case BINARY_VALUE: + Integer binaryValue; + binaryValue = iprot.readI32(); + return binaryValue; + case LEN_VALUE: + Long lenValue; + lenValue = iprot.readI64(); + return lenValue; + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + throw new TProtocolException("Couldn't find a field with field id " + fieldID); + } + } + + @Override + protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case STRING_VALUE: + String stringValue = (String)value_; + oprot.writeString(stringValue); + return; + case SMALL_INT_VALUE: + Short smallIntValue = (Short)value_; + oprot.writeI16(smallIntValue); + return; + case INTEGER_BITMASK: + Integer integerBitmask = (Integer)value_; + oprot.writeI32(integerBitmask); + return; + case INTEGER_FLAG: + Integer integerFlag = (Integer)value_; + oprot.writeI32(integerFlag); + return; + case BINARY_VALUE: + Integer binaryValue = (Integer)value_; + oprot.writeI32(binaryValue); + return; + case LEN_VALUE: + Long lenValue = (Long)value_; + oprot.writeI64(lenValue); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { + switch (setField) { + case STRING_VALUE: + return STRING_VALUE_FIELD_DESC; + case SMALL_INT_VALUE: + return SMALL_INT_VALUE_FIELD_DESC; + case INTEGER_BITMASK: + return INTEGER_BITMASK_FIELD_DESC; + case INTEGER_FLAG: + return INTEGER_FLAG_FIELD_DESC; + case BINARY_VALUE: + return BINARY_VALUE_FIELD_DESC; + case LEN_VALUE: + return LEN_VALUE_FIELD_DESC; + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected org.apache.thrift.protocol.TStruct getStructDesc() { + return STRUCT_DESC; + } + + @Override + protected _Fields enumForId(short id) { + return _Fields.findByThriftIdOrThrow(id); + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + + public String getStringValue() { + if (getSetField() == _Fields.STRING_VALUE) { + return (String)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'stringValue' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setStringValue(String value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.STRING_VALUE; + value_ = value; + } + + public short getSmallIntValue() { + if (getSetField() == _Fields.SMALL_INT_VALUE) { + return (Short)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'smallIntValue' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setSmallIntValue(short value) { + setField_ = _Fields.SMALL_INT_VALUE; + value_ = value; + } + + public int getIntegerBitmask() { + if (getSetField() == _Fields.INTEGER_BITMASK) { + return (Integer)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'integerBitmask' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setIntegerBitmask(int value) { + setField_ = _Fields.INTEGER_BITMASK; + value_ = value; + } + + public int getIntegerFlag() { + if (getSetField() == _Fields.INTEGER_FLAG) { + return (Integer)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'integerFlag' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setIntegerFlag(int value) { + setField_ = _Fields.INTEGER_FLAG; + value_ = value; + } + + public int getBinaryValue() { + if (getSetField() == _Fields.BINARY_VALUE) { + return (Integer)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'binaryValue' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setBinaryValue(int value) { + setField_ = _Fields.BINARY_VALUE; + value_ = value; + } + + public long getLenValue() { + if (getSetField() == _Fields.LEN_VALUE) { + return (Long)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'lenValue' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setLenValue(long value) { + setField_ = _Fields.LEN_VALUE; + value_ = value; + } + + public boolean isSetStringValue() { + return setField_ == _Fields.STRING_VALUE; + } + + + public boolean isSetSmallIntValue() { + return setField_ == _Fields.SMALL_INT_VALUE; + } + + + public boolean isSetIntegerBitmask() { + return setField_ == _Fields.INTEGER_BITMASK; + } + + + public boolean isSetIntegerFlag() { + return setField_ == _Fields.INTEGER_FLAG; + } + + + public boolean isSetBinaryValue() { + return setField_ == _Fields.BINARY_VALUE; + } + + + public boolean isSetLenValue() { + return setField_ == _Fields.LEN_VALUE; + } + + + public boolean equals(Object other) { + if (other instanceof TGetInfoValue) { + return equals((TGetInfoValue)other); + } else { + return false; + } + } + + public boolean equals(TGetInfoValue other) { + return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); + } + + @Override + public int compareTo(TGetInfoValue other) { + int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + HashCodeBuilder hcb = new HashCodeBuilder(); + hcb.append(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + hcb.append(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); + } else { + hcb.append(value); + } + } + return hcb.toHashCode(); + } + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + +} diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java new file mode 100644 index 000000000000..b88591ea1945 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetOperationStatusReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetOperationStatusReq"); + + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetOperationStatusReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetOperationStatusReqTupleSchemeFactory()); + } + + private TOperationHandle operationHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OPERATION_HANDLE((short)1, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetOperationStatusReq.class, metaDataMap); + } + + public TGetOperationStatusReq() { + } + + public TGetOperationStatusReq( + TOperationHandle operationHandle) + { + this(); + this.operationHandle = operationHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetOperationStatusReq(TGetOperationStatusReq other) { + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetOperationStatusReq deepCopy() { + return new TGetOperationStatusReq(this); + } + + @Override + public void clear() { + this.operationHandle = null; + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetOperationStatusReq) + return this.equals((TGetOperationStatusReq)that); + return false; + } + + public boolean equals(TGetOperationStatusReq that) { + if (that == null) + return false; + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_operationHandle = true && (isSetOperationHandle()); + builder.append(present_operationHandle); + if (present_operationHandle) + builder.append(operationHandle); + + return builder.toHashCode(); + } + + public int compareTo(TGetOperationStatusReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetOperationStatusReq typedOther = (TGetOperationStatusReq)other; + + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetOperationStatusReq("); + boolean first = true; + + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetOperationHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetOperationStatusReqStandardSchemeFactory implements SchemeFactory { + public TGetOperationStatusReqStandardScheme getScheme() { + return new TGetOperationStatusReqStandardScheme(); + } + } + + private static class TGetOperationStatusReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.operationHandle != null) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetOperationStatusReqTupleSchemeFactory implements SchemeFactory { + public TGetOperationStatusReqTupleScheme getScheme() { + return new TGetOperationStatusReqTupleScheme(); + } + } + + private static class TGetOperationStatusReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.operationHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java new file mode 100644 index 000000000000..94ba6bb1146d --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java @@ -0,0 +1,827 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetOperationStatusResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetOperationStatusResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationState", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField SQL_STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("sqlState", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField ERROR_CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorCode", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField ERROR_MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorMessage", org.apache.thrift.protocol.TType.STRING, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetOperationStatusRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetOperationStatusRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationState operationState; // optional + private String sqlState; // optional + private int errorCode; // optional + private String errorMessage; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + /** + * + * @see TOperationState + */ + OPERATION_STATE((short)2, "operationState"), + SQL_STATE((short)3, "sqlState"), + ERROR_CODE((short)4, "errorCode"), + ERROR_MESSAGE((short)5, "errorMessage"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_STATE + return OPERATION_STATE; + case 3: // SQL_STATE + return SQL_STATE; + case 4: // ERROR_CODE + return ERROR_CODE; + case 5: // ERROR_MESSAGE + return ERROR_MESSAGE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __ERRORCODE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.OPERATION_STATE,_Fields.SQL_STATE,_Fields.ERROR_CODE,_Fields.ERROR_MESSAGE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_STATE, new org.apache.thrift.meta_data.FieldMetaData("operationState", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TOperationState.class))); + tmpMap.put(_Fields.SQL_STATE, new org.apache.thrift.meta_data.FieldMetaData("sqlState", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.ERROR_CODE, new org.apache.thrift.meta_data.FieldMetaData("errorCode", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.ERROR_MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("errorMessage", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetOperationStatusResp.class, metaDataMap); + } + + public TGetOperationStatusResp() { + } + + public TGetOperationStatusResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetOperationStatusResp(TGetOperationStatusResp other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationState()) { + this.operationState = other.operationState; + } + if (other.isSetSqlState()) { + this.sqlState = other.sqlState; + } + this.errorCode = other.errorCode; + if (other.isSetErrorMessage()) { + this.errorMessage = other.errorMessage; + } + } + + public TGetOperationStatusResp deepCopy() { + return new TGetOperationStatusResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationState = null; + this.sqlState = null; + setErrorCodeIsSet(false); + this.errorCode = 0; + this.errorMessage = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + /** + * + * @see TOperationState + */ + public TOperationState getOperationState() { + return this.operationState; + } + + /** + * + * @see TOperationState + */ + public void setOperationState(TOperationState operationState) { + this.operationState = operationState; + } + + public void unsetOperationState() { + this.operationState = null; + } + + /** Returns true if field operationState is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationState() { + return this.operationState != null; + } + + public void setOperationStateIsSet(boolean value) { + if (!value) { + this.operationState = null; + } + } + + public String getSqlState() { + return this.sqlState; + } + + public void setSqlState(String sqlState) { + this.sqlState = sqlState; + } + + public void unsetSqlState() { + this.sqlState = null; + } + + /** Returns true if field sqlState is set (has been assigned a value) and false otherwise */ + public boolean isSetSqlState() { + return this.sqlState != null; + } + + public void setSqlStateIsSet(boolean value) { + if (!value) { + this.sqlState = null; + } + } + + public int getErrorCode() { + return this.errorCode; + } + + public void setErrorCode(int errorCode) { + this.errorCode = errorCode; + setErrorCodeIsSet(true); + } + + public void unsetErrorCode() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ERRORCODE_ISSET_ID); + } + + /** Returns true if field errorCode is set (has been assigned a value) and false otherwise */ + public boolean isSetErrorCode() { + return EncodingUtils.testBit(__isset_bitfield, __ERRORCODE_ISSET_ID); + } + + public void setErrorCodeIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ERRORCODE_ISSET_ID, value); + } + + public String getErrorMessage() { + return this.errorMessage; + } + + public void setErrorMessage(String errorMessage) { + this.errorMessage = errorMessage; + } + + public void unsetErrorMessage() { + this.errorMessage = null; + } + + /** Returns true if field errorMessage is set (has been assigned a value) and false otherwise */ + public boolean isSetErrorMessage() { + return this.errorMessage != null; + } + + public void setErrorMessageIsSet(boolean value) { + if (!value) { + this.errorMessage = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_STATE: + if (value == null) { + unsetOperationState(); + } else { + setOperationState((TOperationState)value); + } + break; + + case SQL_STATE: + if (value == null) { + unsetSqlState(); + } else { + setSqlState((String)value); + } + break; + + case ERROR_CODE: + if (value == null) { + unsetErrorCode(); + } else { + setErrorCode((Integer)value); + } + break; + + case ERROR_MESSAGE: + if (value == null) { + unsetErrorMessage(); + } else { + setErrorMessage((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_STATE: + return getOperationState(); + + case SQL_STATE: + return getSqlState(); + + case ERROR_CODE: + return Integer.valueOf(getErrorCode()); + + case ERROR_MESSAGE: + return getErrorMessage(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_STATE: + return isSetOperationState(); + case SQL_STATE: + return isSetSqlState(); + case ERROR_CODE: + return isSetErrorCode(); + case ERROR_MESSAGE: + return isSetErrorMessage(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetOperationStatusResp) + return this.equals((TGetOperationStatusResp)that); + return false; + } + + public boolean equals(TGetOperationStatusResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationState = true && this.isSetOperationState(); + boolean that_present_operationState = true && that.isSetOperationState(); + if (this_present_operationState || that_present_operationState) { + if (!(this_present_operationState && that_present_operationState)) + return false; + if (!this.operationState.equals(that.operationState)) + return false; + } + + boolean this_present_sqlState = true && this.isSetSqlState(); + boolean that_present_sqlState = true && that.isSetSqlState(); + if (this_present_sqlState || that_present_sqlState) { + if (!(this_present_sqlState && that_present_sqlState)) + return false; + if (!this.sqlState.equals(that.sqlState)) + return false; + } + + boolean this_present_errorCode = true && this.isSetErrorCode(); + boolean that_present_errorCode = true && that.isSetErrorCode(); + if (this_present_errorCode || that_present_errorCode) { + if (!(this_present_errorCode && that_present_errorCode)) + return false; + if (this.errorCode != that.errorCode) + return false; + } + + boolean this_present_errorMessage = true && this.isSetErrorMessage(); + boolean that_present_errorMessage = true && that.isSetErrorMessage(); + if (this_present_errorMessage || that_present_errorMessage) { + if (!(this_present_errorMessage && that_present_errorMessage)) + return false; + if (!this.errorMessage.equals(that.errorMessage)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_operationState = true && (isSetOperationState()); + builder.append(present_operationState); + if (present_operationState) + builder.append(operationState.getValue()); + + boolean present_sqlState = true && (isSetSqlState()); + builder.append(present_sqlState); + if (present_sqlState) + builder.append(sqlState); + + boolean present_errorCode = true && (isSetErrorCode()); + builder.append(present_errorCode); + if (present_errorCode) + builder.append(errorCode); + + boolean present_errorMessage = true && (isSetErrorMessage()); + builder.append(present_errorMessage); + if (present_errorMessage) + builder.append(errorMessage); + + return builder.toHashCode(); + } + + public int compareTo(TGetOperationStatusResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetOperationStatusResp typedOther = (TGetOperationStatusResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationState()).compareTo(typedOther.isSetOperationState()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationState()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationState, typedOther.operationState); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSqlState()).compareTo(typedOther.isSetSqlState()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSqlState()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sqlState, typedOther.sqlState); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetErrorCode()).compareTo(typedOther.isSetErrorCode()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetErrorCode()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorCode, typedOther.errorCode); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetErrorMessage()).compareTo(typedOther.isSetErrorMessage()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetErrorMessage()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorMessage, typedOther.errorMessage); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetOperationStatusResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationState()) { + if (!first) sb.append(", "); + sb.append("operationState:"); + if (this.operationState == null) { + sb.append("null"); + } else { + sb.append(this.operationState); + } + first = false; + } + if (isSetSqlState()) { + if (!first) sb.append(", "); + sb.append("sqlState:"); + if (this.sqlState == null) { + sb.append("null"); + } else { + sb.append(this.sqlState); + } + first = false; + } + if (isSetErrorCode()) { + if (!first) sb.append(", "); + sb.append("errorCode:"); + sb.append(this.errorCode); + first = false; + } + if (isSetErrorMessage()) { + if (!first) sb.append(", "); + sb.append("errorMessage:"); + if (this.errorMessage == null) { + sb.append("null"); + } else { + sb.append(this.errorMessage); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetOperationStatusRespStandardSchemeFactory implements SchemeFactory { + public TGetOperationStatusRespStandardScheme getScheme() { + return new TGetOperationStatusRespStandardScheme(); + } + } + + private static class TGetOperationStatusRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_STATE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.operationState = TOperationState.findByValue(iprot.readI32()); + struct.setOperationStateIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SQL_STATE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.sqlState = iprot.readString(); + struct.setSqlStateIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // ERROR_CODE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.errorCode = iprot.readI32(); + struct.setErrorCodeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // ERROR_MESSAGE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.errorMessage = iprot.readString(); + struct.setErrorMessageIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationState != null) { + if (struct.isSetOperationState()) { + oprot.writeFieldBegin(OPERATION_STATE_FIELD_DESC); + oprot.writeI32(struct.operationState.getValue()); + oprot.writeFieldEnd(); + } + } + if (struct.sqlState != null) { + if (struct.isSetSqlState()) { + oprot.writeFieldBegin(SQL_STATE_FIELD_DESC); + oprot.writeString(struct.sqlState); + oprot.writeFieldEnd(); + } + } + if (struct.isSetErrorCode()) { + oprot.writeFieldBegin(ERROR_CODE_FIELD_DESC); + oprot.writeI32(struct.errorCode); + oprot.writeFieldEnd(); + } + if (struct.errorMessage != null) { + if (struct.isSetErrorMessage()) { + oprot.writeFieldBegin(ERROR_MESSAGE_FIELD_DESC); + oprot.writeString(struct.errorMessage); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetOperationStatusRespTupleSchemeFactory implements SchemeFactory { + public TGetOperationStatusRespTupleScheme getScheme() { + return new TGetOperationStatusRespTupleScheme(); + } + } + + private static class TGetOperationStatusRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationState()) { + optionals.set(0); + } + if (struct.isSetSqlState()) { + optionals.set(1); + } + if (struct.isSetErrorCode()) { + optionals.set(2); + } + if (struct.isSetErrorMessage()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetOperationState()) { + oprot.writeI32(struct.operationState.getValue()); + } + if (struct.isSetSqlState()) { + oprot.writeString(struct.sqlState); + } + if (struct.isSetErrorCode()) { + oprot.writeI32(struct.errorCode); + } + if (struct.isSetErrorMessage()) { + oprot.writeString(struct.errorMessage); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.operationState = TOperationState.findByValue(iprot.readI32()); + struct.setOperationStateIsSet(true); + } + if (incoming.get(1)) { + struct.sqlState = iprot.readString(); + struct.setSqlStateIsSet(true); + } + if (incoming.get(2)) { + struct.errorCode = iprot.readI32(); + struct.setErrorCodeIsSet(true); + } + if (incoming.get(3)) { + struct.errorMessage = iprot.readString(); + struct.setErrorMessageIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java new file mode 100644 index 000000000000..3bf363c95846 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetResultSetMetadataReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetResultSetMetadataReq"); + + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetResultSetMetadataReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetResultSetMetadataReqTupleSchemeFactory()); + } + + private TOperationHandle operationHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OPERATION_HANDLE((short)1, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetResultSetMetadataReq.class, metaDataMap); + } + + public TGetResultSetMetadataReq() { + } + + public TGetResultSetMetadataReq( + TOperationHandle operationHandle) + { + this(); + this.operationHandle = operationHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetResultSetMetadataReq(TGetResultSetMetadataReq other) { + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetResultSetMetadataReq deepCopy() { + return new TGetResultSetMetadataReq(this); + } + + @Override + public void clear() { + this.operationHandle = null; + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetResultSetMetadataReq) + return this.equals((TGetResultSetMetadataReq)that); + return false; + } + + public boolean equals(TGetResultSetMetadataReq that) { + if (that == null) + return false; + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_operationHandle = true && (isSetOperationHandle()); + builder.append(present_operationHandle); + if (present_operationHandle) + builder.append(operationHandle); + + return builder.toHashCode(); + } + + public int compareTo(TGetResultSetMetadataReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetResultSetMetadataReq typedOther = (TGetResultSetMetadataReq)other; + + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetResultSetMetadataReq("); + boolean first = true; + + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetOperationHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetResultSetMetadataReqStandardSchemeFactory implements SchemeFactory { + public TGetResultSetMetadataReqStandardScheme getScheme() { + return new TGetResultSetMetadataReqStandardScheme(); + } + } + + private static class TGetResultSetMetadataReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.operationHandle != null) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetResultSetMetadataReqTupleSchemeFactory implements SchemeFactory { + public TGetResultSetMetadataReqTupleScheme getScheme() { + return new TGetResultSetMetadataReqTupleScheme(); + } + } + + private static class TGetResultSetMetadataReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.operationHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java new file mode 100644 index 000000000000..a9bef9f722c1 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java @@ -0,0 +1,505 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetResultSetMetadataResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetResultSetMetadataResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField SCHEMA_FIELD_DESC = new org.apache.thrift.protocol.TField("schema", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetResultSetMetadataRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetResultSetMetadataRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TTableSchema schema; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + SCHEMA((short)2, "schema"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // SCHEMA + return SCHEMA; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.SCHEMA}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.SCHEMA, new org.apache.thrift.meta_data.FieldMetaData("schema", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableSchema.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetResultSetMetadataResp.class, metaDataMap); + } + + public TGetResultSetMetadataResp() { + } + + public TGetResultSetMetadataResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetResultSetMetadataResp(TGetResultSetMetadataResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetSchema()) { + this.schema = new TTableSchema(other.schema); + } + } + + public TGetResultSetMetadataResp deepCopy() { + return new TGetResultSetMetadataResp(this); + } + + @Override + public void clear() { + this.status = null; + this.schema = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TTableSchema getSchema() { + return this.schema; + } + + public void setSchema(TTableSchema schema) { + this.schema = schema; + } + + public void unsetSchema() { + this.schema = null; + } + + /** Returns true if field schema is set (has been assigned a value) and false otherwise */ + public boolean isSetSchema() { + return this.schema != null; + } + + public void setSchemaIsSet(boolean value) { + if (!value) { + this.schema = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case SCHEMA: + if (value == null) { + unsetSchema(); + } else { + setSchema((TTableSchema)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case SCHEMA: + return getSchema(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case SCHEMA: + return isSetSchema(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetResultSetMetadataResp) + return this.equals((TGetResultSetMetadataResp)that); + return false; + } + + public boolean equals(TGetResultSetMetadataResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_schema = true && this.isSetSchema(); + boolean that_present_schema = true && that.isSetSchema(); + if (this_present_schema || that_present_schema) { + if (!(this_present_schema && that_present_schema)) + return false; + if (!this.schema.equals(that.schema)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_schema = true && (isSetSchema()); + builder.append(present_schema); + if (present_schema) + builder.append(schema); + + return builder.toHashCode(); + } + + public int compareTo(TGetResultSetMetadataResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetResultSetMetadataResp typedOther = (TGetResultSetMetadataResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSchema()).compareTo(typedOther.isSetSchema()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSchema()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schema, typedOther.schema); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetResultSetMetadataResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetSchema()) { + if (!first) sb.append(", "); + sb.append("schema:"); + if (this.schema == null) { + sb.append("null"); + } else { + sb.append(this.schema); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (schema != null) { + schema.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetResultSetMetadataRespStandardSchemeFactory implements SchemeFactory { + public TGetResultSetMetadataRespStandardScheme getScheme() { + return new TGetResultSetMetadataRespStandardScheme(); + } + } + + private static class TGetResultSetMetadataRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // SCHEMA + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.schema = new TTableSchema(); + struct.schema.read(iprot); + struct.setSchemaIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.schema != null) { + if (struct.isSetSchema()) { + oprot.writeFieldBegin(SCHEMA_FIELD_DESC); + struct.schema.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetResultSetMetadataRespTupleSchemeFactory implements SchemeFactory { + public TGetResultSetMetadataRespTupleScheme getScheme() { + return new TGetResultSetMetadataRespTupleScheme(); + } + } + + private static class TGetResultSetMetadataRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetSchema()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSchema()) { + struct.schema.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.schema = new TTableSchema(); + struct.schema.read(iprot); + struct.setSchemaIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java new file mode 100644 index 000000000000..c2aadaa49a1e --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java @@ -0,0 +1,606 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetSchemasReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetSchemasReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetSchemasReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetSchemasReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String catalogName; // optional + private String schemaName; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + CATALOG_NAME((short)2, "catalogName"), + SCHEMA_NAME((short)3, "schemaName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // CATALOG_NAME + return CATALOG_NAME; + case 3: // SCHEMA_NAME + return SCHEMA_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetSchemasReq.class, metaDataMap); + } + + public TGetSchemasReq() { + } + + public TGetSchemasReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetSchemasReq(TGetSchemasReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetCatalogName()) { + this.catalogName = other.catalogName; + } + if (other.isSetSchemaName()) { + this.schemaName = other.schemaName; + } + } + + public TGetSchemasReq deepCopy() { + return new TGetSchemasReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.catalogName = null; + this.schemaName = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getCatalogName() { + return this.catalogName; + } + + public void setCatalogName(String catalogName) { + this.catalogName = catalogName; + } + + public void unsetCatalogName() { + this.catalogName = null; + } + + /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ + public boolean isSetCatalogName() { + return this.catalogName != null; + } + + public void setCatalogNameIsSet(boolean value) { + if (!value) { + this.catalogName = null; + } + } + + public String getSchemaName() { + return this.schemaName; + } + + public void setSchemaName(String schemaName) { + this.schemaName = schemaName; + } + + public void unsetSchemaName() { + this.schemaName = null; + } + + /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ + public boolean isSetSchemaName() { + return this.schemaName != null; + } + + public void setSchemaNameIsSet(boolean value) { + if (!value) { + this.schemaName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case CATALOG_NAME: + if (value == null) { + unsetCatalogName(); + } else { + setCatalogName((String)value); + } + break; + + case SCHEMA_NAME: + if (value == null) { + unsetSchemaName(); + } else { + setSchemaName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case CATALOG_NAME: + return getCatalogName(); + + case SCHEMA_NAME: + return getSchemaName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case CATALOG_NAME: + return isSetCatalogName(); + case SCHEMA_NAME: + return isSetSchemaName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetSchemasReq) + return this.equals((TGetSchemasReq)that); + return false; + } + + public boolean equals(TGetSchemasReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_catalogName = true && this.isSetCatalogName(); + boolean that_present_catalogName = true && that.isSetCatalogName(); + if (this_present_catalogName || that_present_catalogName) { + if (!(this_present_catalogName && that_present_catalogName)) + return false; + if (!this.catalogName.equals(that.catalogName)) + return false; + } + + boolean this_present_schemaName = true && this.isSetSchemaName(); + boolean that_present_schemaName = true && that.isSetSchemaName(); + if (this_present_schemaName || that_present_schemaName) { + if (!(this_present_schemaName && that_present_schemaName)) + return false; + if (!this.schemaName.equals(that.schemaName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + builder.append(present_sessionHandle); + if (present_sessionHandle) + builder.append(sessionHandle); + + boolean present_catalogName = true && (isSetCatalogName()); + builder.append(present_catalogName); + if (present_catalogName) + builder.append(catalogName); + + boolean present_schemaName = true && (isSetSchemaName()); + builder.append(present_schemaName); + if (present_schemaName) + builder.append(schemaName); + + return builder.toHashCode(); + } + + public int compareTo(TGetSchemasReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetSchemasReq typedOther = (TGetSchemasReq)other; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(typedOther.isSetCatalogName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCatalogName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, typedOther.catalogName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(typedOther.isSetSchemaName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSchemaName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, typedOther.schemaName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetSchemasReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (isSetCatalogName()) { + if (!first) sb.append(", "); + sb.append("catalogName:"); + if (this.catalogName == null) { + sb.append("null"); + } else { + sb.append(this.catalogName); + } + first = false; + } + if (isSetSchemaName()) { + if (!first) sb.append(", "); + sb.append("schemaName:"); + if (this.schemaName == null) { + sb.append("null"); + } else { + sb.append(this.schemaName); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetSchemasReqStandardSchemeFactory implements SchemeFactory { + public TGetSchemasReqStandardScheme getScheme() { + return new TGetSchemasReqStandardScheme(); + } + } + + private static class TGetSchemasReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetSchemasReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // CATALOG_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SCHEMA_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetSchemasReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.catalogName != null) { + if (struct.isSetCatalogName()) { + oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); + oprot.writeString(struct.catalogName); + oprot.writeFieldEnd(); + } + } + if (struct.schemaName != null) { + if (struct.isSetSchemaName()) { + oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); + oprot.writeString(struct.schemaName); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetSchemasReqTupleSchemeFactory implements SchemeFactory { + public TGetSchemasReqTupleScheme getScheme() { + return new TGetSchemasReqTupleScheme(); + } + } + + private static class TGetSchemasReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetSchemasReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetCatalogName()) { + optionals.set(0); + } + if (struct.isSetSchemaName()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetCatalogName()) { + oprot.writeString(struct.catalogName); + } + if (struct.isSetSchemaName()) { + oprot.writeString(struct.schemaName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetSchemasReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } + if (incoming.get(1)) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java new file mode 100644 index 000000000000..ac1ea3e7cc7a --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java @@ -0,0 +1,505 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetSchemasResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetSchemasResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetSchemasRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetSchemasRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetSchemasResp.class, metaDataMap); + } + + public TGetSchemasResp() { + } + + public TGetSchemasResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetSchemasResp(TGetSchemasResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetSchemasResp deepCopy() { + return new TGetSchemasResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetSchemasResp) + return this.equals((TGetSchemasResp)that); + return false; + } + + public boolean equals(TGetSchemasResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + builder.append(present_operationHandle); + if (present_operationHandle) + builder.append(operationHandle); + + return builder.toHashCode(); + } + + public int compareTo(TGetSchemasResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetSchemasResp typedOther = (TGetSchemasResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetSchemasResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetSchemasRespStandardSchemeFactory implements SchemeFactory { + public TGetSchemasRespStandardScheme getScheme() { + return new TGetSchemasRespStandardScheme(); + } + } + + private static class TGetSchemasRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetSchemasResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetSchemasResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetSchemasRespTupleSchemeFactory implements SchemeFactory { + public TGetSchemasRespTupleScheme getScheme() { + return new TGetSchemasRespTupleScheme(); + } + } + + private static class TGetSchemasRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetSchemasResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetSchemasResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java new file mode 100644 index 000000000000..6f2c713e0be6 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetTableTypesReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTableTypesReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetTableTypesReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetTableTypesReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTableTypesReq.class, metaDataMap); + } + + public TGetTableTypesReq() { + } + + public TGetTableTypesReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetTableTypesReq(TGetTableTypesReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + } + + public TGetTableTypesReq deepCopy() { + return new TGetTableTypesReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetTableTypesReq) + return this.equals((TGetTableTypesReq)that); + return false; + } + + public boolean equals(TGetTableTypesReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + builder.append(present_sessionHandle); + if (present_sessionHandle) + builder.append(sessionHandle); + + return builder.toHashCode(); + } + + public int compareTo(TGetTableTypesReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetTableTypesReq typedOther = (TGetTableTypesReq)other; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetTableTypesReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetTableTypesReqStandardSchemeFactory implements SchemeFactory { + public TGetTableTypesReqStandardScheme getScheme() { + return new TGetTableTypesReqStandardScheme(); + } + } + + private static class TGetTableTypesReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTableTypesReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTableTypesReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetTableTypesReqTupleSchemeFactory implements SchemeFactory { + public TGetTableTypesReqTupleScheme getScheme() { + return new TGetTableTypesReqTupleScheme(); + } + } + + private static class TGetTableTypesReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java new file mode 100644 index 000000000000..6f33fbcf5dad --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java @@ -0,0 +1,505 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetTableTypesResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTableTypesResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetTableTypesRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetTableTypesRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTableTypesResp.class, metaDataMap); + } + + public TGetTableTypesResp() { + } + + public TGetTableTypesResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetTableTypesResp(TGetTableTypesResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetTableTypesResp deepCopy() { + return new TGetTableTypesResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetTableTypesResp) + return this.equals((TGetTableTypesResp)that); + return false; + } + + public boolean equals(TGetTableTypesResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + builder.append(present_operationHandle); + if (present_operationHandle) + builder.append(operationHandle); + + return builder.toHashCode(); + } + + public int compareTo(TGetTableTypesResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetTableTypesResp typedOther = (TGetTableTypesResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetTableTypesResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetTableTypesRespStandardSchemeFactory implements SchemeFactory { + public TGetTableTypesRespStandardScheme getScheme() { + return new TGetTableTypesRespStandardScheme(); + } + } + + private static class TGetTableTypesRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTableTypesResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTableTypesResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetTableTypesRespTupleSchemeFactory implements SchemeFactory { + public TGetTableTypesRespTupleScheme getScheme() { + return new TGetTableTypesRespTupleScheme(); + } + } + + private static class TGetTableTypesRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java new file mode 100644 index 000000000000..c973fcc24cb1 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java @@ -0,0 +1,870 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetTablesReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTablesReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)4); + private static final org.apache.thrift.protocol.TField TABLE_TYPES_FIELD_DESC = new org.apache.thrift.protocol.TField("tableTypes", org.apache.thrift.protocol.TType.LIST, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetTablesReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetTablesReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String catalogName; // optional + private String schemaName; // optional + private String tableName; // optional + private List tableTypes; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + CATALOG_NAME((short)2, "catalogName"), + SCHEMA_NAME((short)3, "schemaName"), + TABLE_NAME((short)4, "tableName"), + TABLE_TYPES((short)5, "tableTypes"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // CATALOG_NAME + return CATALOG_NAME; + case 3: // SCHEMA_NAME + return SCHEMA_NAME; + case 4: // TABLE_NAME + return TABLE_NAME; + case 5: // TABLE_TYPES + return TABLE_TYPES; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME,_Fields.TABLE_NAME,_Fields.TABLE_TYPES}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + tmpMap.put(_Fields.TABLE_TYPES, new org.apache.thrift.meta_data.FieldMetaData("tableTypes", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTablesReq.class, metaDataMap); + } + + public TGetTablesReq() { + } + + public TGetTablesReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetTablesReq(TGetTablesReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetCatalogName()) { + this.catalogName = other.catalogName; + } + if (other.isSetSchemaName()) { + this.schemaName = other.schemaName; + } + if (other.isSetTableName()) { + this.tableName = other.tableName; + } + if (other.isSetTableTypes()) { + List __this__tableTypes = new ArrayList(); + for (String other_element : other.tableTypes) { + __this__tableTypes.add(other_element); + } + this.tableTypes = __this__tableTypes; + } + } + + public TGetTablesReq deepCopy() { + return new TGetTablesReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.catalogName = null; + this.schemaName = null; + this.tableName = null; + this.tableTypes = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getCatalogName() { + return this.catalogName; + } + + public void setCatalogName(String catalogName) { + this.catalogName = catalogName; + } + + public void unsetCatalogName() { + this.catalogName = null; + } + + /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ + public boolean isSetCatalogName() { + return this.catalogName != null; + } + + public void setCatalogNameIsSet(boolean value) { + if (!value) { + this.catalogName = null; + } + } + + public String getSchemaName() { + return this.schemaName; + } + + public void setSchemaName(String schemaName) { + this.schemaName = schemaName; + } + + public void unsetSchemaName() { + this.schemaName = null; + } + + /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ + public boolean isSetSchemaName() { + return this.schemaName != null; + } + + public void setSchemaNameIsSet(boolean value) { + if (!value) { + this.schemaName = null; + } + } + + public String getTableName() { + return this.tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public void unsetTableName() { + this.tableName = null; + } + + /** Returns true if field tableName is set (has been assigned a value) and false otherwise */ + public boolean isSetTableName() { + return this.tableName != null; + } + + public void setTableNameIsSet(boolean value) { + if (!value) { + this.tableName = null; + } + } + + public int getTableTypesSize() { + return (this.tableTypes == null) ? 0 : this.tableTypes.size(); + } + + public java.util.Iterator getTableTypesIterator() { + return (this.tableTypes == null) ? null : this.tableTypes.iterator(); + } + + public void addToTableTypes(String elem) { + if (this.tableTypes == null) { + this.tableTypes = new ArrayList(); + } + this.tableTypes.add(elem); + } + + public List getTableTypes() { + return this.tableTypes; + } + + public void setTableTypes(List tableTypes) { + this.tableTypes = tableTypes; + } + + public void unsetTableTypes() { + this.tableTypes = null; + } + + /** Returns true if field tableTypes is set (has been assigned a value) and false otherwise */ + public boolean isSetTableTypes() { + return this.tableTypes != null; + } + + public void setTableTypesIsSet(boolean value) { + if (!value) { + this.tableTypes = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case CATALOG_NAME: + if (value == null) { + unsetCatalogName(); + } else { + setCatalogName((String)value); + } + break; + + case SCHEMA_NAME: + if (value == null) { + unsetSchemaName(); + } else { + setSchemaName((String)value); + } + break; + + case TABLE_NAME: + if (value == null) { + unsetTableName(); + } else { + setTableName((String)value); + } + break; + + case TABLE_TYPES: + if (value == null) { + unsetTableTypes(); + } else { + setTableTypes((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case CATALOG_NAME: + return getCatalogName(); + + case SCHEMA_NAME: + return getSchemaName(); + + case TABLE_NAME: + return getTableName(); + + case TABLE_TYPES: + return getTableTypes(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case CATALOG_NAME: + return isSetCatalogName(); + case SCHEMA_NAME: + return isSetSchemaName(); + case TABLE_NAME: + return isSetTableName(); + case TABLE_TYPES: + return isSetTableTypes(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetTablesReq) + return this.equals((TGetTablesReq)that); + return false; + } + + public boolean equals(TGetTablesReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_catalogName = true && this.isSetCatalogName(); + boolean that_present_catalogName = true && that.isSetCatalogName(); + if (this_present_catalogName || that_present_catalogName) { + if (!(this_present_catalogName && that_present_catalogName)) + return false; + if (!this.catalogName.equals(that.catalogName)) + return false; + } + + boolean this_present_schemaName = true && this.isSetSchemaName(); + boolean that_present_schemaName = true && that.isSetSchemaName(); + if (this_present_schemaName || that_present_schemaName) { + if (!(this_present_schemaName && that_present_schemaName)) + return false; + if (!this.schemaName.equals(that.schemaName)) + return false; + } + + boolean this_present_tableName = true && this.isSetTableName(); + boolean that_present_tableName = true && that.isSetTableName(); + if (this_present_tableName || that_present_tableName) { + if (!(this_present_tableName && that_present_tableName)) + return false; + if (!this.tableName.equals(that.tableName)) + return false; + } + + boolean this_present_tableTypes = true && this.isSetTableTypes(); + boolean that_present_tableTypes = true && that.isSetTableTypes(); + if (this_present_tableTypes || that_present_tableTypes) { + if (!(this_present_tableTypes && that_present_tableTypes)) + return false; + if (!this.tableTypes.equals(that.tableTypes)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + builder.append(present_sessionHandle); + if (present_sessionHandle) + builder.append(sessionHandle); + + boolean present_catalogName = true && (isSetCatalogName()); + builder.append(present_catalogName); + if (present_catalogName) + builder.append(catalogName); + + boolean present_schemaName = true && (isSetSchemaName()); + builder.append(present_schemaName); + if (present_schemaName) + builder.append(schemaName); + + boolean present_tableName = true && (isSetTableName()); + builder.append(present_tableName); + if (present_tableName) + builder.append(tableName); + + boolean present_tableTypes = true && (isSetTableTypes()); + builder.append(present_tableTypes); + if (present_tableTypes) + builder.append(tableTypes); + + return builder.toHashCode(); + } + + public int compareTo(TGetTablesReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetTablesReq typedOther = (TGetTablesReq)other; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(typedOther.isSetCatalogName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCatalogName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, typedOther.catalogName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(typedOther.isSetSchemaName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSchemaName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, typedOther.schemaName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(typedOther.isSetTableName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, typedOther.tableName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTableTypes()).compareTo(typedOther.isSetTableTypes()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableTypes()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableTypes, typedOther.tableTypes); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetTablesReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (isSetCatalogName()) { + if (!first) sb.append(", "); + sb.append("catalogName:"); + if (this.catalogName == null) { + sb.append("null"); + } else { + sb.append(this.catalogName); + } + first = false; + } + if (isSetSchemaName()) { + if (!first) sb.append(", "); + sb.append("schemaName:"); + if (this.schemaName == null) { + sb.append("null"); + } else { + sb.append(this.schemaName); + } + first = false; + } + if (isSetTableName()) { + if (!first) sb.append(", "); + sb.append("tableName:"); + if (this.tableName == null) { + sb.append("null"); + } else { + sb.append(this.tableName); + } + first = false; + } + if (isSetTableTypes()) { + if (!first) sb.append(", "); + sb.append("tableTypes:"); + if (this.tableTypes == null) { + sb.append("null"); + } else { + sb.append(this.tableTypes); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetTablesReqStandardSchemeFactory implements SchemeFactory { + public TGetTablesReqStandardScheme getScheme() { + return new TGetTablesReqStandardScheme(); + } + } + + private static class TGetTablesReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTablesReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // CATALOG_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SCHEMA_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // TABLE_TYPES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list172 = iprot.readListBegin(); + struct.tableTypes = new ArrayList(_list172.size); + for (int _i173 = 0; _i173 < _list172.size; ++_i173) + { + String _elem174; // optional + _elem174 = iprot.readString(); + struct.tableTypes.add(_elem174); + } + iprot.readListEnd(); + } + struct.setTableTypesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTablesReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.catalogName != null) { + if (struct.isSetCatalogName()) { + oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); + oprot.writeString(struct.catalogName); + oprot.writeFieldEnd(); + } + } + if (struct.schemaName != null) { + if (struct.isSetSchemaName()) { + oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); + oprot.writeString(struct.schemaName); + oprot.writeFieldEnd(); + } + } + if (struct.tableName != null) { + if (struct.isSetTableName()) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.tableName); + oprot.writeFieldEnd(); + } + } + if (struct.tableTypes != null) { + if (struct.isSetTableTypes()) { + oprot.writeFieldBegin(TABLE_TYPES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tableTypes.size())); + for (String _iter175 : struct.tableTypes) + { + oprot.writeString(_iter175); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetTablesReqTupleSchemeFactory implements SchemeFactory { + public TGetTablesReqTupleScheme getScheme() { + return new TGetTablesReqTupleScheme(); + } + } + + private static class TGetTablesReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetTablesReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetCatalogName()) { + optionals.set(0); + } + if (struct.isSetSchemaName()) { + optionals.set(1); + } + if (struct.isSetTableName()) { + optionals.set(2); + } + if (struct.isSetTableTypes()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetCatalogName()) { + oprot.writeString(struct.catalogName); + } + if (struct.isSetSchemaName()) { + oprot.writeString(struct.schemaName); + } + if (struct.isSetTableName()) { + oprot.writeString(struct.tableName); + } + if (struct.isSetTableTypes()) { + { + oprot.writeI32(struct.tableTypes.size()); + for (String _iter176 : struct.tableTypes) + { + oprot.writeString(_iter176); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetTablesReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } + if (incoming.get(1)) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } + if (incoming.get(2)) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } + if (incoming.get(3)) { + { + org.apache.thrift.protocol.TList _list177 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.tableTypes = new ArrayList(_list177.size); + for (int _i178 = 0; _i178 < _list177.size; ++_i178) + { + String _elem179; // optional + _elem179 = iprot.readString(); + struct.tableTypes.add(_elem179); + } + } + struct.setTableTypesIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java new file mode 100644 index 000000000000..d526f4478a24 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java @@ -0,0 +1,505 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetTablesResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTablesResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetTablesRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetTablesRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTablesResp.class, metaDataMap); + } + + public TGetTablesResp() { + } + + public TGetTablesResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetTablesResp(TGetTablesResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetTablesResp deepCopy() { + return new TGetTablesResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetTablesResp) + return this.equals((TGetTablesResp)that); + return false; + } + + public boolean equals(TGetTablesResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + builder.append(present_operationHandle); + if (present_operationHandle) + builder.append(operationHandle); + + return builder.toHashCode(); + } + + public int compareTo(TGetTablesResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetTablesResp typedOther = (TGetTablesResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetTablesResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetTablesRespStandardSchemeFactory implements SchemeFactory { + public TGetTablesRespStandardScheme getScheme() { + return new TGetTablesRespStandardScheme(); + } + } + + private static class TGetTablesRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTablesResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTablesResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetTablesRespTupleSchemeFactory implements SchemeFactory { + public TGetTablesRespTupleScheme getScheme() { + return new TGetTablesRespTupleScheme(); + } + } + + private static class TGetTablesRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetTablesResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetTablesResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java new file mode 100644 index 000000000000..d40115e83ec4 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetTypeInfoReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTypeInfoReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetTypeInfoReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetTypeInfoReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTypeInfoReq.class, metaDataMap); + } + + public TGetTypeInfoReq() { + } + + public TGetTypeInfoReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetTypeInfoReq(TGetTypeInfoReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + } + + public TGetTypeInfoReq deepCopy() { + return new TGetTypeInfoReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetTypeInfoReq) + return this.equals((TGetTypeInfoReq)that); + return false; + } + + public boolean equals(TGetTypeInfoReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + builder.append(present_sessionHandle); + if (present_sessionHandle) + builder.append(sessionHandle); + + return builder.toHashCode(); + } + + public int compareTo(TGetTypeInfoReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetTypeInfoReq typedOther = (TGetTypeInfoReq)other; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetTypeInfoReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetTypeInfoReqStandardSchemeFactory implements SchemeFactory { + public TGetTypeInfoReqStandardScheme getScheme() { + return new TGetTypeInfoReqStandardScheme(); + } + } + + private static class TGetTypeInfoReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetTypeInfoReqTupleSchemeFactory implements SchemeFactory { + public TGetTypeInfoReqTupleScheme getScheme() { + return new TGetTypeInfoReqTupleScheme(); + } + } + + private static class TGetTypeInfoReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java new file mode 100644 index 000000000000..59be1a33b55e --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java @@ -0,0 +1,505 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TGetTypeInfoResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTypeInfoResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetTypeInfoRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetTypeInfoRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTypeInfoResp.class, metaDataMap); + } + + public TGetTypeInfoResp() { + } + + public TGetTypeInfoResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetTypeInfoResp(TGetTypeInfoResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetTypeInfoResp deepCopy() { + return new TGetTypeInfoResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetTypeInfoResp) + return this.equals((TGetTypeInfoResp)that); + return false; + } + + public boolean equals(TGetTypeInfoResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + builder.append(present_operationHandle); + if (present_operationHandle) + builder.append(operationHandle); + + return builder.toHashCode(); + } + + public int compareTo(TGetTypeInfoResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TGetTypeInfoResp typedOther = (TGetTypeInfoResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(typedOther.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, typedOther.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetTypeInfoResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetTypeInfoRespStandardSchemeFactory implements SchemeFactory { + public TGetTypeInfoRespStandardScheme getScheme() { + return new TGetTypeInfoRespStandardScheme(); + } + } + + private static class TGetTypeInfoRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetTypeInfoRespTupleSchemeFactory implements SchemeFactory { + public TGetTypeInfoRespTupleScheme getScheme() { + return new TGetTypeInfoRespTupleScheme(); + } + } + + private static class TGetTypeInfoRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java new file mode 100644 index 000000000000..368273c341c7 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java @@ -0,0 +1,506 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class THandleIdentifier implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THandleIdentifier"); + + private static final org.apache.thrift.protocol.TField GUID_FIELD_DESC = new org.apache.thrift.protocol.TField("guid", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField SECRET_FIELD_DESC = new org.apache.thrift.protocol.TField("secret", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new THandleIdentifierStandardSchemeFactory()); + schemes.put(TupleScheme.class, new THandleIdentifierTupleSchemeFactory()); + } + + private ByteBuffer guid; // required + private ByteBuffer secret; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + GUID((short)1, "guid"), + SECRET((short)2, "secret"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // GUID + return GUID; + case 2: // SECRET + return SECRET; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.GUID, new org.apache.thrift.meta_data.FieldMetaData("guid", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.SECRET, new org.apache.thrift.meta_data.FieldMetaData("secret", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(THandleIdentifier.class, metaDataMap); + } + + public THandleIdentifier() { + } + + public THandleIdentifier( + ByteBuffer guid, + ByteBuffer secret) + { + this(); + this.guid = guid; + this.secret = secret; + } + + /** + * Performs a deep copy on other. + */ + public THandleIdentifier(THandleIdentifier other) { + if (other.isSetGuid()) { + this.guid = org.apache.thrift.TBaseHelper.copyBinary(other.guid); +; + } + if (other.isSetSecret()) { + this.secret = org.apache.thrift.TBaseHelper.copyBinary(other.secret); +; + } + } + + public THandleIdentifier deepCopy() { + return new THandleIdentifier(this); + } + + @Override + public void clear() { + this.guid = null; + this.secret = null; + } + + public byte[] getGuid() { + setGuid(org.apache.thrift.TBaseHelper.rightSize(guid)); + return guid == null ? null : guid.array(); + } + + public ByteBuffer bufferForGuid() { + return guid; + } + + public void setGuid(byte[] guid) { + setGuid(guid == null ? (ByteBuffer)null : ByteBuffer.wrap(guid)); + } + + public void setGuid(ByteBuffer guid) { + this.guid = guid; + } + + public void unsetGuid() { + this.guid = null; + } + + /** Returns true if field guid is set (has been assigned a value) and false otherwise */ + public boolean isSetGuid() { + return this.guid != null; + } + + public void setGuidIsSet(boolean value) { + if (!value) { + this.guid = null; + } + } + + public byte[] getSecret() { + setSecret(org.apache.thrift.TBaseHelper.rightSize(secret)); + return secret == null ? null : secret.array(); + } + + public ByteBuffer bufferForSecret() { + return secret; + } + + public void setSecret(byte[] secret) { + setSecret(secret == null ? (ByteBuffer)null : ByteBuffer.wrap(secret)); + } + + public void setSecret(ByteBuffer secret) { + this.secret = secret; + } + + public void unsetSecret() { + this.secret = null; + } + + /** Returns true if field secret is set (has been assigned a value) and false otherwise */ + public boolean isSetSecret() { + return this.secret != null; + } + + public void setSecretIsSet(boolean value) { + if (!value) { + this.secret = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case GUID: + if (value == null) { + unsetGuid(); + } else { + setGuid((ByteBuffer)value); + } + break; + + case SECRET: + if (value == null) { + unsetSecret(); + } else { + setSecret((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case GUID: + return getGuid(); + + case SECRET: + return getSecret(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case GUID: + return isSetGuid(); + case SECRET: + return isSetSecret(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof THandleIdentifier) + return this.equals((THandleIdentifier)that); + return false; + } + + public boolean equals(THandleIdentifier that) { + if (that == null) + return false; + + boolean this_present_guid = true && this.isSetGuid(); + boolean that_present_guid = true && that.isSetGuid(); + if (this_present_guid || that_present_guid) { + if (!(this_present_guid && that_present_guid)) + return false; + if (!this.guid.equals(that.guid)) + return false; + } + + boolean this_present_secret = true && this.isSetSecret(); + boolean that_present_secret = true && that.isSetSecret(); + if (this_present_secret || that_present_secret) { + if (!(this_present_secret && that_present_secret)) + return false; + if (!this.secret.equals(that.secret)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_guid = true && (isSetGuid()); + builder.append(present_guid); + if (present_guid) + builder.append(guid); + + boolean present_secret = true && (isSetSecret()); + builder.append(present_secret); + if (present_secret) + builder.append(secret); + + return builder.toHashCode(); + } + + public int compareTo(THandleIdentifier other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + THandleIdentifier typedOther = (THandleIdentifier)other; + + lastComparison = Boolean.valueOf(isSetGuid()).compareTo(typedOther.isSetGuid()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetGuid()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.guid, typedOther.guid); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSecret()).compareTo(typedOther.isSetSecret()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSecret()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.secret, typedOther.secret); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("THandleIdentifier("); + boolean first = true; + + sb.append("guid:"); + if (this.guid == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.guid, sb); + } + first = false; + if (!first) sb.append(", "); + sb.append("secret:"); + if (this.secret == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.secret, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetGuid()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'guid' is unset! Struct:" + toString()); + } + + if (!isSetSecret()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'secret' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class THandleIdentifierStandardSchemeFactory implements SchemeFactory { + public THandleIdentifierStandardScheme getScheme() { + return new THandleIdentifierStandardScheme(); + } + } + + private static class THandleIdentifierStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, THandleIdentifier struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // GUID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.guid = iprot.readBinary(); + struct.setGuidIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // SECRET + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.secret = iprot.readBinary(); + struct.setSecretIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, THandleIdentifier struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.guid != null) { + oprot.writeFieldBegin(GUID_FIELD_DESC); + oprot.writeBinary(struct.guid); + oprot.writeFieldEnd(); + } + if (struct.secret != null) { + oprot.writeFieldBegin(SECRET_FIELD_DESC); + oprot.writeBinary(struct.secret); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class THandleIdentifierTupleSchemeFactory implements SchemeFactory { + public THandleIdentifierTupleScheme getScheme() { + return new THandleIdentifierTupleScheme(); + } + } + + private static class THandleIdentifierTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, THandleIdentifier struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeBinary(struct.guid); + oprot.writeBinary(struct.secret); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, THandleIdentifier struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.guid = iprot.readBinary(); + struct.setGuidIsSet(true); + struct.secret = iprot.readBinary(); + struct.setSecretIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java new file mode 100644 index 000000000000..c83663072f87 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TI16Column implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI16Column"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TI16ColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TI16ColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI16Column.class, metaDataMap); + } + + public TI16Column() { + } + + public TI16Column( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = nulls; + } + + /** + * Performs a deep copy on other. + */ + public TI16Column(TI16Column other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(); + for (Short other_element : other.values) { + __this__values.add(other_element); + } + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); +; + } + } + + public TI16Column deepCopy() { + return new TI16Column(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(short elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return nulls; + } + + public void setNulls(byte[] nulls) { + setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = nulls; + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TI16Column) + return this.equals((TI16Column)that); + return false; + } + + public boolean equals(TI16Column that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_values = true && (isSetValues()); + builder.append(present_values); + if (present_values) + builder.append(values); + + boolean present_nulls = true && (isSetNulls()); + builder.append(present_nulls); + if (present_nulls) + builder.append(nulls); + + return builder.toHashCode(); + } + + public int compareTo(TI16Column other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TI16Column typedOther = (TI16Column)other; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TI16Column("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TI16ColumnStandardSchemeFactory implements SchemeFactory { + public TI16ColumnStandardScheme getScheme() { + return new TI16ColumnStandardScheme(); + } + } + + private static class TI16ColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TI16Column struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list70 = iprot.readListBegin(); + struct.values = new ArrayList(_list70.size); + for (int _i71 = 0; _i71 < _list70.size; ++_i71) + { + short _elem72; // optional + _elem72 = iprot.readI16(); + struct.values.add(_elem72); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TI16Column struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I16, struct.values.size())); + for (short _iter73 : struct.values) + { + oprot.writeI16(_iter73); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TI16ColumnTupleSchemeFactory implements SchemeFactory { + public TI16ColumnTupleScheme getScheme() { + return new TI16ColumnTupleScheme(); + } + } + + private static class TI16ColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TI16Column struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (short _iter74 : struct.values) + { + oprot.writeI16(_iter74); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TI16Column struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list75 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I16, iprot.readI32()); + struct.values = new ArrayList(_list75.size); + for (int _i76 = 0; _i76 < _list75.size; ++_i76) + { + short _elem77; // optional + _elem77 = iprot.readI16(); + struct.values.add(_elem77); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java new file mode 100644 index 000000000000..bb5ae9609de8 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java @@ -0,0 +1,386 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TI16Value implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI16Value"); + + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.I16, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TI16ValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TI16ValueTupleSchemeFactory()); + } + + private short value; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUE((short)1, "value"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUE + return VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __VALUE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.VALUE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI16Value.class, metaDataMap); + } + + public TI16Value() { + } + + /** + * Performs a deep copy on other. + */ + public TI16Value(TI16Value other) { + __isset_bitfield = other.__isset_bitfield; + this.value = other.value; + } + + public TI16Value deepCopy() { + return new TI16Value(this); + } + + @Override + public void clear() { + setValueIsSet(false); + this.value = 0; + } + + public short getValue() { + return this.value; + } + + public void setValue(short value) { + this.value = value; + setValueIsSet(true); + } + + public void unsetValue() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + public void setValueIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((Short)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUE: + return Short.valueOf(getValue()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUE: + return isSetValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TI16Value) + return this.equals((TI16Value)that); + return false; + } + + public boolean equals(TI16Value that) { + if (that == null) + return false; + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (this.value != that.value) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_value = true && (isSetValue()); + builder.append(present_value); + if (present_value) + builder.append(value); + + return builder.toHashCode(); + } + + public int compareTo(TI16Value other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TI16Value typedOther = (TI16Value)other; + + lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TI16Value("); + boolean first = true; + + if (isSetValue()) { + sb.append("value:"); + sb.append(this.value); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TI16ValueStandardSchemeFactory implements SchemeFactory { + public TI16ValueStandardScheme getScheme() { + return new TI16ValueStandardScheme(); + } + } + + private static class TI16ValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TI16Value struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.I16) { + struct.value = iprot.readI16(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TI16Value struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetValue()) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeI16(struct.value); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TI16ValueTupleSchemeFactory implements SchemeFactory { + public TI16ValueTupleScheme getScheme() { + return new TI16ValueTupleScheme(); + } + } + + private static class TI16ValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TI16Value struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetValue()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetValue()) { + oprot.writeI16(struct.value); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TI16Value struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.value = iprot.readI16(); + struct.setValueIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java new file mode 100644 index 000000000000..6c6c5f35b7c8 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TI32Column implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI32Column"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TI32ColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TI32ColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI32Column.class, metaDataMap); + } + + public TI32Column() { + } + + public TI32Column( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = nulls; + } + + /** + * Performs a deep copy on other. + */ + public TI32Column(TI32Column other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(); + for (Integer other_element : other.values) { + __this__values.add(other_element); + } + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); +; + } + } + + public TI32Column deepCopy() { + return new TI32Column(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(int elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return nulls; + } + + public void setNulls(byte[] nulls) { + setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = nulls; + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TI32Column) + return this.equals((TI32Column)that); + return false; + } + + public boolean equals(TI32Column that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_values = true && (isSetValues()); + builder.append(present_values); + if (present_values) + builder.append(values); + + boolean present_nulls = true && (isSetNulls()); + builder.append(present_nulls); + if (present_nulls) + builder.append(nulls); + + return builder.toHashCode(); + } + + public int compareTo(TI32Column other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TI32Column typedOther = (TI32Column)other; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TI32Column("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TI32ColumnStandardSchemeFactory implements SchemeFactory { + public TI32ColumnStandardScheme getScheme() { + return new TI32ColumnStandardScheme(); + } + } + + private static class TI32ColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TI32Column struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list78 = iprot.readListBegin(); + struct.values = new ArrayList(_list78.size); + for (int _i79 = 0; _i79 < _list78.size; ++_i79) + { + int _elem80; // optional + _elem80 = iprot.readI32(); + struct.values.add(_elem80); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TI32Column struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.values.size())); + for (int _iter81 : struct.values) + { + oprot.writeI32(_iter81); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TI32ColumnTupleSchemeFactory implements SchemeFactory { + public TI32ColumnTupleScheme getScheme() { + return new TI32ColumnTupleScheme(); + } + } + + private static class TI32ColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TI32Column struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (int _iter82 : struct.values) + { + oprot.writeI32(_iter82); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TI32Column struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list83 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.values = new ArrayList(_list83.size); + for (int _i84 = 0; _i84 < _list83.size; ++_i84) + { + int _elem85; // optional + _elem85 = iprot.readI32(); + struct.values.add(_elem85); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java new file mode 100644 index 000000000000..059408b96c8c --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java @@ -0,0 +1,386 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TI32Value implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI32Value"); + + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.I32, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TI32ValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TI32ValueTupleSchemeFactory()); + } + + private int value; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUE((short)1, "value"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUE + return VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __VALUE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.VALUE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI32Value.class, metaDataMap); + } + + public TI32Value() { + } + + /** + * Performs a deep copy on other. + */ + public TI32Value(TI32Value other) { + __isset_bitfield = other.__isset_bitfield; + this.value = other.value; + } + + public TI32Value deepCopy() { + return new TI32Value(this); + } + + @Override + public void clear() { + setValueIsSet(false); + this.value = 0; + } + + public int getValue() { + return this.value; + } + + public void setValue(int value) { + this.value = value; + setValueIsSet(true); + } + + public void unsetValue() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + public void setValueIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUE: + return Integer.valueOf(getValue()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUE: + return isSetValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TI32Value) + return this.equals((TI32Value)that); + return false; + } + + public boolean equals(TI32Value that) { + if (that == null) + return false; + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (this.value != that.value) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_value = true && (isSetValue()); + builder.append(present_value); + if (present_value) + builder.append(value); + + return builder.toHashCode(); + } + + public int compareTo(TI32Value other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TI32Value typedOther = (TI32Value)other; + + lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TI32Value("); + boolean first = true; + + if (isSetValue()) { + sb.append("value:"); + sb.append(this.value); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TI32ValueStandardSchemeFactory implements SchemeFactory { + public TI32ValueStandardScheme getScheme() { + return new TI32ValueStandardScheme(); + } + } + + private static class TI32ValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TI32Value struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.value = iprot.readI32(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TI32Value struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetValue()) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeI32(struct.value); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TI32ValueTupleSchemeFactory implements SchemeFactory { + public TI32ValueTupleScheme getScheme() { + return new TI32ValueTupleScheme(); + } + } + + private static class TI32ValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TI32Value struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetValue()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetValue()) { + oprot.writeI32(struct.value); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TI32Value struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.value = iprot.readI32(); + struct.setValueIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java new file mode 100644 index 000000000000..cc383ed089fa --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TI64Column implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI64Column"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TI64ColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TI64ColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI64Column.class, metaDataMap); + } + + public TI64Column() { + } + + public TI64Column( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = nulls; + } + + /** + * Performs a deep copy on other. + */ + public TI64Column(TI64Column other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(); + for (Long other_element : other.values) { + __this__values.add(other_element); + } + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); +; + } + } + + public TI64Column deepCopy() { + return new TI64Column(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(long elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return nulls; + } + + public void setNulls(byte[] nulls) { + setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = nulls; + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TI64Column) + return this.equals((TI64Column)that); + return false; + } + + public boolean equals(TI64Column that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_values = true && (isSetValues()); + builder.append(present_values); + if (present_values) + builder.append(values); + + boolean present_nulls = true && (isSetNulls()); + builder.append(present_nulls); + if (present_nulls) + builder.append(nulls); + + return builder.toHashCode(); + } + + public int compareTo(TI64Column other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TI64Column typedOther = (TI64Column)other; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TI64Column("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TI64ColumnStandardSchemeFactory implements SchemeFactory { + public TI64ColumnStandardScheme getScheme() { + return new TI64ColumnStandardScheme(); + } + } + + private static class TI64ColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TI64Column struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list86 = iprot.readListBegin(); + struct.values = new ArrayList(_list86.size); + for (int _i87 = 0; _i87 < _list86.size; ++_i87) + { + long _elem88; // optional + _elem88 = iprot.readI64(); + struct.values.add(_elem88); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TI64Column struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.values.size())); + for (long _iter89 : struct.values) + { + oprot.writeI64(_iter89); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TI64ColumnTupleSchemeFactory implements SchemeFactory { + public TI64ColumnTupleScheme getScheme() { + return new TI64ColumnTupleScheme(); + } + } + + private static class TI64ColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TI64Column struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (long _iter90 : struct.values) + { + oprot.writeI64(_iter90); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TI64Column struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list91 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.values = new ArrayList(_list91.size); + for (int _i92 = 0; _i92 < _list91.size; ++_i92) + { + long _elem93; // optional + _elem93 = iprot.readI64(); + struct.values.add(_elem93); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java new file mode 100644 index 000000000000..9a941cce0c07 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java @@ -0,0 +1,386 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TI64Value implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI64Value"); + + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.I64, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TI64ValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TI64ValueTupleSchemeFactory()); + } + + private long value; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUE((short)1, "value"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUE + return VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __VALUE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.VALUE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI64Value.class, metaDataMap); + } + + public TI64Value() { + } + + /** + * Performs a deep copy on other. + */ + public TI64Value(TI64Value other) { + __isset_bitfield = other.__isset_bitfield; + this.value = other.value; + } + + public TI64Value deepCopy() { + return new TI64Value(this); + } + + @Override + public void clear() { + setValueIsSet(false); + this.value = 0; + } + + public long getValue() { + return this.value; + } + + public void setValue(long value) { + this.value = value; + setValueIsSet(true); + } + + public void unsetValue() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + public void setValueIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((Long)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUE: + return Long.valueOf(getValue()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUE: + return isSetValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TI64Value) + return this.equals((TI64Value)that); + return false; + } + + public boolean equals(TI64Value that) { + if (that == null) + return false; + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (this.value != that.value) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_value = true && (isSetValue()); + builder.append(present_value); + if (present_value) + builder.append(value); + + return builder.toHashCode(); + } + + public int compareTo(TI64Value other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TI64Value typedOther = (TI64Value)other; + + lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TI64Value("); + boolean first = true; + + if (isSetValue()) { + sb.append("value:"); + sb.append(this.value); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TI64ValueStandardSchemeFactory implements SchemeFactory { + public TI64ValueStandardScheme getScheme() { + return new TI64ValueStandardScheme(); + } + } + + private static class TI64ValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TI64Value struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.value = iprot.readI64(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TI64Value struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetValue()) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeI64(struct.value); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TI64ValueTupleSchemeFactory implements SchemeFactory { + public TI64ValueTupleScheme getScheme() { + return new TI64ValueTupleScheme(); + } + } + + private static class TI64ValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TI64Value struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetValue()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetValue()) { + oprot.writeI64(struct.value); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TI64Value struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.value = iprot.readI64(); + struct.setValueIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java new file mode 100644 index 000000000000..425603cbdecb --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java @@ -0,0 +1,478 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TMapTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TMapTypeEntry"); + + private static final org.apache.thrift.protocol.TField KEY_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("keyTypePtr", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField VALUE_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("valueTypePtr", org.apache.thrift.protocol.TType.I32, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TMapTypeEntryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TMapTypeEntryTupleSchemeFactory()); + } + + private int keyTypePtr; // required + private int valueTypePtr; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + KEY_TYPE_PTR((short)1, "keyTypePtr"), + VALUE_TYPE_PTR((short)2, "valueTypePtr"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // KEY_TYPE_PTR + return KEY_TYPE_PTR; + case 2: // VALUE_TYPE_PTR + return VALUE_TYPE_PTR; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __KEYTYPEPTR_ISSET_ID = 0; + private static final int __VALUETYPEPTR_ISSET_ID = 1; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.KEY_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("keyTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr"))); + tmpMap.put(_Fields.VALUE_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("valueTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr"))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TMapTypeEntry.class, metaDataMap); + } + + public TMapTypeEntry() { + } + + public TMapTypeEntry( + int keyTypePtr, + int valueTypePtr) + { + this(); + this.keyTypePtr = keyTypePtr; + setKeyTypePtrIsSet(true); + this.valueTypePtr = valueTypePtr; + setValueTypePtrIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TMapTypeEntry(TMapTypeEntry other) { + __isset_bitfield = other.__isset_bitfield; + this.keyTypePtr = other.keyTypePtr; + this.valueTypePtr = other.valueTypePtr; + } + + public TMapTypeEntry deepCopy() { + return new TMapTypeEntry(this); + } + + @Override + public void clear() { + setKeyTypePtrIsSet(false); + this.keyTypePtr = 0; + setValueTypePtrIsSet(false); + this.valueTypePtr = 0; + } + + public int getKeyTypePtr() { + return this.keyTypePtr; + } + + public void setKeyTypePtr(int keyTypePtr) { + this.keyTypePtr = keyTypePtr; + setKeyTypePtrIsSet(true); + } + + public void unsetKeyTypePtr() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __KEYTYPEPTR_ISSET_ID); + } + + /** Returns true if field keyTypePtr is set (has been assigned a value) and false otherwise */ + public boolean isSetKeyTypePtr() { + return EncodingUtils.testBit(__isset_bitfield, __KEYTYPEPTR_ISSET_ID); + } + + public void setKeyTypePtrIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __KEYTYPEPTR_ISSET_ID, value); + } + + public int getValueTypePtr() { + return this.valueTypePtr; + } + + public void setValueTypePtr(int valueTypePtr) { + this.valueTypePtr = valueTypePtr; + setValueTypePtrIsSet(true); + } + + public void unsetValueTypePtr() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUETYPEPTR_ISSET_ID); + } + + /** Returns true if field valueTypePtr is set (has been assigned a value) and false otherwise */ + public boolean isSetValueTypePtr() { + return EncodingUtils.testBit(__isset_bitfield, __VALUETYPEPTR_ISSET_ID); + } + + public void setValueTypePtrIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUETYPEPTR_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case KEY_TYPE_PTR: + if (value == null) { + unsetKeyTypePtr(); + } else { + setKeyTypePtr((Integer)value); + } + break; + + case VALUE_TYPE_PTR: + if (value == null) { + unsetValueTypePtr(); + } else { + setValueTypePtr((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case KEY_TYPE_PTR: + return Integer.valueOf(getKeyTypePtr()); + + case VALUE_TYPE_PTR: + return Integer.valueOf(getValueTypePtr()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case KEY_TYPE_PTR: + return isSetKeyTypePtr(); + case VALUE_TYPE_PTR: + return isSetValueTypePtr(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TMapTypeEntry) + return this.equals((TMapTypeEntry)that); + return false; + } + + public boolean equals(TMapTypeEntry that) { + if (that == null) + return false; + + boolean this_present_keyTypePtr = true; + boolean that_present_keyTypePtr = true; + if (this_present_keyTypePtr || that_present_keyTypePtr) { + if (!(this_present_keyTypePtr && that_present_keyTypePtr)) + return false; + if (this.keyTypePtr != that.keyTypePtr) + return false; + } + + boolean this_present_valueTypePtr = true; + boolean that_present_valueTypePtr = true; + if (this_present_valueTypePtr || that_present_valueTypePtr) { + if (!(this_present_valueTypePtr && that_present_valueTypePtr)) + return false; + if (this.valueTypePtr != that.valueTypePtr) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_keyTypePtr = true; + builder.append(present_keyTypePtr); + if (present_keyTypePtr) + builder.append(keyTypePtr); + + boolean present_valueTypePtr = true; + builder.append(present_valueTypePtr); + if (present_valueTypePtr) + builder.append(valueTypePtr); + + return builder.toHashCode(); + } + + public int compareTo(TMapTypeEntry other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TMapTypeEntry typedOther = (TMapTypeEntry)other; + + lastComparison = Boolean.valueOf(isSetKeyTypePtr()).compareTo(typedOther.isSetKeyTypePtr()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetKeyTypePtr()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.keyTypePtr, typedOther.keyTypePtr); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetValueTypePtr()).compareTo(typedOther.isSetValueTypePtr()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValueTypePtr()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.valueTypePtr, typedOther.valueTypePtr); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TMapTypeEntry("); + boolean first = true; + + sb.append("keyTypePtr:"); + sb.append(this.keyTypePtr); + first = false; + if (!first) sb.append(", "); + sb.append("valueTypePtr:"); + sb.append(this.valueTypePtr); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetKeyTypePtr()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'keyTypePtr' is unset! Struct:" + toString()); + } + + if (!isSetValueTypePtr()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'valueTypePtr' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TMapTypeEntryStandardSchemeFactory implements SchemeFactory { + public TMapTypeEntryStandardScheme getScheme() { + return new TMapTypeEntryStandardScheme(); + } + } + + private static class TMapTypeEntryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TMapTypeEntry struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // KEY_TYPE_PTR + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.keyTypePtr = iprot.readI32(); + struct.setKeyTypePtrIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // VALUE_TYPE_PTR + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.valueTypePtr = iprot.readI32(); + struct.setValueTypePtrIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TMapTypeEntry struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(KEY_TYPE_PTR_FIELD_DESC); + oprot.writeI32(struct.keyTypePtr); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(VALUE_TYPE_PTR_FIELD_DESC); + oprot.writeI32(struct.valueTypePtr); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TMapTypeEntryTupleSchemeFactory implements SchemeFactory { + public TMapTypeEntryTupleScheme getScheme() { + return new TMapTypeEntryTupleScheme(); + } + } + + private static class TMapTypeEntryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TMapTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.keyTypePtr); + oprot.writeI32(struct.valueTypePtr); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TMapTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.keyTypePtr = iprot.readI32(); + struct.setKeyTypePtrIsSet(true); + struct.valueTypePtr = iprot.readI32(); + struct.setValueTypePtrIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java new file mode 100644 index 000000000000..c0481615b06d --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java @@ -0,0 +1,785 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TOpenSessionReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOpenSessionReq"); + + private static final org.apache.thrift.protocol.TField CLIENT_PROTOCOL_FIELD_DESC = new org.apache.thrift.protocol.TField("client_protocol", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField USERNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("username", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField PASSWORD_FIELD_DESC = new org.apache.thrift.protocol.TField("password", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField CONFIGURATION_FIELD_DESC = new org.apache.thrift.protocol.TField("configuration", org.apache.thrift.protocol.TType.MAP, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TOpenSessionReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TOpenSessionReqTupleSchemeFactory()); + } + + private TProtocolVersion client_protocol; // required + private String username; // optional + private String password; // optional + private Map configuration; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + /** + * + * @see TProtocolVersion + */ + CLIENT_PROTOCOL((short)1, "client_protocol"), + USERNAME((short)2, "username"), + PASSWORD((short)3, "password"), + CONFIGURATION((short)4, "configuration"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // CLIENT_PROTOCOL + return CLIENT_PROTOCOL; + case 2: // USERNAME + return USERNAME; + case 3: // PASSWORD + return PASSWORD; + case 4: // CONFIGURATION + return CONFIGURATION; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.USERNAME,_Fields.PASSWORD,_Fields.CONFIGURATION}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.CLIENT_PROTOCOL, new org.apache.thrift.meta_data.FieldMetaData("client_protocol", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TProtocolVersion.class))); + tmpMap.put(_Fields.USERNAME, new org.apache.thrift.meta_data.FieldMetaData("username", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.PASSWORD, new org.apache.thrift.meta_data.FieldMetaData("password", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.CONFIGURATION, new org.apache.thrift.meta_data.FieldMetaData("configuration", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOpenSessionReq.class, metaDataMap); + } + + public TOpenSessionReq() { + this.client_protocol = org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8; + + } + + public TOpenSessionReq( + TProtocolVersion client_protocol) + { + this(); + this.client_protocol = client_protocol; + } + + /** + * Performs a deep copy on other. + */ + public TOpenSessionReq(TOpenSessionReq other) { + if (other.isSetClient_protocol()) { + this.client_protocol = other.client_protocol; + } + if (other.isSetUsername()) { + this.username = other.username; + } + if (other.isSetPassword()) { + this.password = other.password; + } + if (other.isSetConfiguration()) { + Map __this__configuration = new HashMap(); + for (Map.Entry other_element : other.configuration.entrySet()) { + + String other_element_key = other_element.getKey(); + String other_element_value = other_element.getValue(); + + String __this__configuration_copy_key = other_element_key; + + String __this__configuration_copy_value = other_element_value; + + __this__configuration.put(__this__configuration_copy_key, __this__configuration_copy_value); + } + this.configuration = __this__configuration; + } + } + + public TOpenSessionReq deepCopy() { + return new TOpenSessionReq(this); + } + + @Override + public void clear() { + this.client_protocol = org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8; + + this.username = null; + this.password = null; + this.configuration = null; + } + + /** + * + * @see TProtocolVersion + */ + public TProtocolVersion getClient_protocol() { + return this.client_protocol; + } + + /** + * + * @see TProtocolVersion + */ + public void setClient_protocol(TProtocolVersion client_protocol) { + this.client_protocol = client_protocol; + } + + public void unsetClient_protocol() { + this.client_protocol = null; + } + + /** Returns true if field client_protocol is set (has been assigned a value) and false otherwise */ + public boolean isSetClient_protocol() { + return this.client_protocol != null; + } + + public void setClient_protocolIsSet(boolean value) { + if (!value) { + this.client_protocol = null; + } + } + + public String getUsername() { + return this.username; + } + + public void setUsername(String username) { + this.username = username; + } + + public void unsetUsername() { + this.username = null; + } + + /** Returns true if field username is set (has been assigned a value) and false otherwise */ + public boolean isSetUsername() { + return this.username != null; + } + + public void setUsernameIsSet(boolean value) { + if (!value) { + this.username = null; + } + } + + public String getPassword() { + return this.password; + } + + public void setPassword(String password) { + this.password = password; + } + + public void unsetPassword() { + this.password = null; + } + + /** Returns true if field password is set (has been assigned a value) and false otherwise */ + public boolean isSetPassword() { + return this.password != null; + } + + public void setPasswordIsSet(boolean value) { + if (!value) { + this.password = null; + } + } + + public int getConfigurationSize() { + return (this.configuration == null) ? 0 : this.configuration.size(); + } + + public void putToConfiguration(String key, String val) { + if (this.configuration == null) { + this.configuration = new HashMap(); + } + this.configuration.put(key, val); + } + + public Map getConfiguration() { + return this.configuration; + } + + public void setConfiguration(Map configuration) { + this.configuration = configuration; + } + + public void unsetConfiguration() { + this.configuration = null; + } + + /** Returns true if field configuration is set (has been assigned a value) and false otherwise */ + public boolean isSetConfiguration() { + return this.configuration != null; + } + + public void setConfigurationIsSet(boolean value) { + if (!value) { + this.configuration = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case CLIENT_PROTOCOL: + if (value == null) { + unsetClient_protocol(); + } else { + setClient_protocol((TProtocolVersion)value); + } + break; + + case USERNAME: + if (value == null) { + unsetUsername(); + } else { + setUsername((String)value); + } + break; + + case PASSWORD: + if (value == null) { + unsetPassword(); + } else { + setPassword((String)value); + } + break; + + case CONFIGURATION: + if (value == null) { + unsetConfiguration(); + } else { + setConfiguration((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case CLIENT_PROTOCOL: + return getClient_protocol(); + + case USERNAME: + return getUsername(); + + case PASSWORD: + return getPassword(); + + case CONFIGURATION: + return getConfiguration(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case CLIENT_PROTOCOL: + return isSetClient_protocol(); + case USERNAME: + return isSetUsername(); + case PASSWORD: + return isSetPassword(); + case CONFIGURATION: + return isSetConfiguration(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TOpenSessionReq) + return this.equals((TOpenSessionReq)that); + return false; + } + + public boolean equals(TOpenSessionReq that) { + if (that == null) + return false; + + boolean this_present_client_protocol = true && this.isSetClient_protocol(); + boolean that_present_client_protocol = true && that.isSetClient_protocol(); + if (this_present_client_protocol || that_present_client_protocol) { + if (!(this_present_client_protocol && that_present_client_protocol)) + return false; + if (!this.client_protocol.equals(that.client_protocol)) + return false; + } + + boolean this_present_username = true && this.isSetUsername(); + boolean that_present_username = true && that.isSetUsername(); + if (this_present_username || that_present_username) { + if (!(this_present_username && that_present_username)) + return false; + if (!this.username.equals(that.username)) + return false; + } + + boolean this_present_password = true && this.isSetPassword(); + boolean that_present_password = true && that.isSetPassword(); + if (this_present_password || that_present_password) { + if (!(this_present_password && that_present_password)) + return false; + if (!this.password.equals(that.password)) + return false; + } + + boolean this_present_configuration = true && this.isSetConfiguration(); + boolean that_present_configuration = true && that.isSetConfiguration(); + if (this_present_configuration || that_present_configuration) { + if (!(this_present_configuration && that_present_configuration)) + return false; + if (!this.configuration.equals(that.configuration)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_client_protocol = true && (isSetClient_protocol()); + builder.append(present_client_protocol); + if (present_client_protocol) + builder.append(client_protocol.getValue()); + + boolean present_username = true && (isSetUsername()); + builder.append(present_username); + if (present_username) + builder.append(username); + + boolean present_password = true && (isSetPassword()); + builder.append(present_password); + if (present_password) + builder.append(password); + + boolean present_configuration = true && (isSetConfiguration()); + builder.append(present_configuration); + if (present_configuration) + builder.append(configuration); + + return builder.toHashCode(); + } + + public int compareTo(TOpenSessionReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TOpenSessionReq typedOther = (TOpenSessionReq)other; + + lastComparison = Boolean.valueOf(isSetClient_protocol()).compareTo(typedOther.isSetClient_protocol()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetClient_protocol()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.client_protocol, typedOther.client_protocol); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetUsername()).compareTo(typedOther.isSetUsername()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUsername()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.username, typedOther.username); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetPassword()).compareTo(typedOther.isSetPassword()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPassword()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.password, typedOther.password); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetConfiguration()).compareTo(typedOther.isSetConfiguration()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetConfiguration()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.configuration, typedOther.configuration); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TOpenSessionReq("); + boolean first = true; + + sb.append("client_protocol:"); + if (this.client_protocol == null) { + sb.append("null"); + } else { + sb.append(this.client_protocol); + } + first = false; + if (isSetUsername()) { + if (!first) sb.append(", "); + sb.append("username:"); + if (this.username == null) { + sb.append("null"); + } else { + sb.append(this.username); + } + first = false; + } + if (isSetPassword()) { + if (!first) sb.append(", "); + sb.append("password:"); + if (this.password == null) { + sb.append("null"); + } else { + sb.append(this.password); + } + first = false; + } + if (isSetConfiguration()) { + if (!first) sb.append(", "); + sb.append("configuration:"); + if (this.configuration == null) { + sb.append("null"); + } else { + sb.append(this.configuration); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetClient_protocol()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'client_protocol' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TOpenSessionReqStandardSchemeFactory implements SchemeFactory { + public TOpenSessionReqStandardScheme getScheme() { + return new TOpenSessionReqStandardScheme(); + } + } + + private static class TOpenSessionReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TOpenSessionReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // CLIENT_PROTOCOL + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.client_protocol = TProtocolVersion.findByValue(iprot.readI32()); + struct.setClient_protocolIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // USERNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.username = iprot.readString(); + struct.setUsernameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // PASSWORD + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.password = iprot.readString(); + struct.setPasswordIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // CONFIGURATION + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map142 = iprot.readMapBegin(); + struct.configuration = new HashMap(2*_map142.size); + for (int _i143 = 0; _i143 < _map142.size; ++_i143) + { + String _key144; // required + String _val145; // required + _key144 = iprot.readString(); + _val145 = iprot.readString(); + struct.configuration.put(_key144, _val145); + } + iprot.readMapEnd(); + } + struct.setConfigurationIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TOpenSessionReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.client_protocol != null) { + oprot.writeFieldBegin(CLIENT_PROTOCOL_FIELD_DESC); + oprot.writeI32(struct.client_protocol.getValue()); + oprot.writeFieldEnd(); + } + if (struct.username != null) { + if (struct.isSetUsername()) { + oprot.writeFieldBegin(USERNAME_FIELD_DESC); + oprot.writeString(struct.username); + oprot.writeFieldEnd(); + } + } + if (struct.password != null) { + if (struct.isSetPassword()) { + oprot.writeFieldBegin(PASSWORD_FIELD_DESC); + oprot.writeString(struct.password); + oprot.writeFieldEnd(); + } + } + if (struct.configuration != null) { + if (struct.isSetConfiguration()) { + oprot.writeFieldBegin(CONFIGURATION_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.configuration.size())); + for (Map.Entry _iter146 : struct.configuration.entrySet()) + { + oprot.writeString(_iter146.getKey()); + oprot.writeString(_iter146.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TOpenSessionReqTupleSchemeFactory implements SchemeFactory { + public TOpenSessionReqTupleScheme getScheme() { + return new TOpenSessionReqTupleScheme(); + } + } + + private static class TOpenSessionReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TOpenSessionReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.client_protocol.getValue()); + BitSet optionals = new BitSet(); + if (struct.isSetUsername()) { + optionals.set(0); + } + if (struct.isSetPassword()) { + optionals.set(1); + } + if (struct.isSetConfiguration()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetUsername()) { + oprot.writeString(struct.username); + } + if (struct.isSetPassword()) { + oprot.writeString(struct.password); + } + if (struct.isSetConfiguration()) { + { + oprot.writeI32(struct.configuration.size()); + for (Map.Entry _iter147 : struct.configuration.entrySet()) + { + oprot.writeString(_iter147.getKey()); + oprot.writeString(_iter147.getValue()); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TOpenSessionReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.client_protocol = TProtocolVersion.findByValue(iprot.readI32()); + struct.setClient_protocolIsSet(true); + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.username = iprot.readString(); + struct.setUsernameIsSet(true); + } + if (incoming.get(1)) { + struct.password = iprot.readString(); + struct.setPasswordIsSet(true); + } + if (incoming.get(2)) { + { + org.apache.thrift.protocol.TMap _map148 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.configuration = new HashMap(2*_map148.size); + for (int _i149 = 0; _i149 < _map148.size; ++_i149) + { + String _key150; // required + String _val151; // required + _key150 = iprot.readString(); + _val151 = iprot.readString(); + struct.configuration.put(_key150, _val151); + } + } + struct.setConfigurationIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java new file mode 100644 index 000000000000..351f78b2de20 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java @@ -0,0 +1,790 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TOpenSessionResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOpenSessionResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField SERVER_PROTOCOL_VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("serverProtocolVersion", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField CONFIGURATION_FIELD_DESC = new org.apache.thrift.protocol.TField("configuration", org.apache.thrift.protocol.TType.MAP, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TOpenSessionRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TOpenSessionRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TProtocolVersion serverProtocolVersion; // required + private TSessionHandle sessionHandle; // optional + private Map configuration; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + /** + * + * @see TProtocolVersion + */ + SERVER_PROTOCOL_VERSION((short)2, "serverProtocolVersion"), + SESSION_HANDLE((short)3, "sessionHandle"), + CONFIGURATION((short)4, "configuration"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // SERVER_PROTOCOL_VERSION + return SERVER_PROTOCOL_VERSION; + case 3: // SESSION_HANDLE + return SESSION_HANDLE; + case 4: // CONFIGURATION + return CONFIGURATION; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.SESSION_HANDLE,_Fields.CONFIGURATION}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.SERVER_PROTOCOL_VERSION, new org.apache.thrift.meta_data.FieldMetaData("serverProtocolVersion", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TProtocolVersion.class))); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.CONFIGURATION, new org.apache.thrift.meta_data.FieldMetaData("configuration", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOpenSessionResp.class, metaDataMap); + } + + public TOpenSessionResp() { + this.serverProtocolVersion = org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8; + + } + + public TOpenSessionResp( + TStatus status, + TProtocolVersion serverProtocolVersion) + { + this(); + this.status = status; + this.serverProtocolVersion = serverProtocolVersion; + } + + /** + * Performs a deep copy on other. + */ + public TOpenSessionResp(TOpenSessionResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetServerProtocolVersion()) { + this.serverProtocolVersion = other.serverProtocolVersion; + } + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetConfiguration()) { + Map __this__configuration = new HashMap(); + for (Map.Entry other_element : other.configuration.entrySet()) { + + String other_element_key = other_element.getKey(); + String other_element_value = other_element.getValue(); + + String __this__configuration_copy_key = other_element_key; + + String __this__configuration_copy_value = other_element_value; + + __this__configuration.put(__this__configuration_copy_key, __this__configuration_copy_value); + } + this.configuration = __this__configuration; + } + } + + public TOpenSessionResp deepCopy() { + return new TOpenSessionResp(this); + } + + @Override + public void clear() { + this.status = null; + this.serverProtocolVersion = org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V8; + + this.sessionHandle = null; + this.configuration = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + /** + * + * @see TProtocolVersion + */ + public TProtocolVersion getServerProtocolVersion() { + return this.serverProtocolVersion; + } + + /** + * + * @see TProtocolVersion + */ + public void setServerProtocolVersion(TProtocolVersion serverProtocolVersion) { + this.serverProtocolVersion = serverProtocolVersion; + } + + public void unsetServerProtocolVersion() { + this.serverProtocolVersion = null; + } + + /** Returns true if field serverProtocolVersion is set (has been assigned a value) and false otherwise */ + public boolean isSetServerProtocolVersion() { + return this.serverProtocolVersion != null; + } + + public void setServerProtocolVersionIsSet(boolean value) { + if (!value) { + this.serverProtocolVersion = null; + } + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public int getConfigurationSize() { + return (this.configuration == null) ? 0 : this.configuration.size(); + } + + public void putToConfiguration(String key, String val) { + if (this.configuration == null) { + this.configuration = new HashMap(); + } + this.configuration.put(key, val); + } + + public Map getConfiguration() { + return this.configuration; + } + + public void setConfiguration(Map configuration) { + this.configuration = configuration; + } + + public void unsetConfiguration() { + this.configuration = null; + } + + /** Returns true if field configuration is set (has been assigned a value) and false otherwise */ + public boolean isSetConfiguration() { + return this.configuration != null; + } + + public void setConfigurationIsSet(boolean value) { + if (!value) { + this.configuration = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case SERVER_PROTOCOL_VERSION: + if (value == null) { + unsetServerProtocolVersion(); + } else { + setServerProtocolVersion((TProtocolVersion)value); + } + break; + + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case CONFIGURATION: + if (value == null) { + unsetConfiguration(); + } else { + setConfiguration((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case SERVER_PROTOCOL_VERSION: + return getServerProtocolVersion(); + + case SESSION_HANDLE: + return getSessionHandle(); + + case CONFIGURATION: + return getConfiguration(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case SERVER_PROTOCOL_VERSION: + return isSetServerProtocolVersion(); + case SESSION_HANDLE: + return isSetSessionHandle(); + case CONFIGURATION: + return isSetConfiguration(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TOpenSessionResp) + return this.equals((TOpenSessionResp)that); + return false; + } + + public boolean equals(TOpenSessionResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_serverProtocolVersion = true && this.isSetServerProtocolVersion(); + boolean that_present_serverProtocolVersion = true && that.isSetServerProtocolVersion(); + if (this_present_serverProtocolVersion || that_present_serverProtocolVersion) { + if (!(this_present_serverProtocolVersion && that_present_serverProtocolVersion)) + return false; + if (!this.serverProtocolVersion.equals(that.serverProtocolVersion)) + return false; + } + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_configuration = true && this.isSetConfiguration(); + boolean that_present_configuration = true && that.isSetConfiguration(); + if (this_present_configuration || that_present_configuration) { + if (!(this_present_configuration && that_present_configuration)) + return false; + if (!this.configuration.equals(that.configuration)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + boolean present_serverProtocolVersion = true && (isSetServerProtocolVersion()); + builder.append(present_serverProtocolVersion); + if (present_serverProtocolVersion) + builder.append(serverProtocolVersion.getValue()); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + builder.append(present_sessionHandle); + if (present_sessionHandle) + builder.append(sessionHandle); + + boolean present_configuration = true && (isSetConfiguration()); + builder.append(present_configuration); + if (present_configuration) + builder.append(configuration); + + return builder.toHashCode(); + } + + public int compareTo(TOpenSessionResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TOpenSessionResp typedOther = (TOpenSessionResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetServerProtocolVersion()).compareTo(typedOther.isSetServerProtocolVersion()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetServerProtocolVersion()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.serverProtocolVersion, typedOther.serverProtocolVersion); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetConfiguration()).compareTo(typedOther.isSetConfiguration()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetConfiguration()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.configuration, typedOther.configuration); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TOpenSessionResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (!first) sb.append(", "); + sb.append("serverProtocolVersion:"); + if (this.serverProtocolVersion == null) { + sb.append("null"); + } else { + sb.append(this.serverProtocolVersion); + } + first = false; + if (isSetSessionHandle()) { + if (!first) sb.append(", "); + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + } + if (isSetConfiguration()) { + if (!first) sb.append(", "); + sb.append("configuration:"); + if (this.configuration == null) { + sb.append("null"); + } else { + sb.append(this.configuration); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + if (!isSetServerProtocolVersion()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'serverProtocolVersion' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TOpenSessionRespStandardSchemeFactory implements SchemeFactory { + public TOpenSessionRespStandardScheme getScheme() { + return new TOpenSessionRespStandardScheme(); + } + } + + private static class TOpenSessionRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TOpenSessionResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // SERVER_PROTOCOL_VERSION + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.serverProtocolVersion = TProtocolVersion.findByValue(iprot.readI32()); + struct.setServerProtocolVersionIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // CONFIGURATION + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map152 = iprot.readMapBegin(); + struct.configuration = new HashMap(2*_map152.size); + for (int _i153 = 0; _i153 < _map152.size; ++_i153) + { + String _key154; // required + String _val155; // required + _key154 = iprot.readString(); + _val155 = iprot.readString(); + struct.configuration.put(_key154, _val155); + } + iprot.readMapEnd(); + } + struct.setConfigurationIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TOpenSessionResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.serverProtocolVersion != null) { + oprot.writeFieldBegin(SERVER_PROTOCOL_VERSION_FIELD_DESC); + oprot.writeI32(struct.serverProtocolVersion.getValue()); + oprot.writeFieldEnd(); + } + if (struct.sessionHandle != null) { + if (struct.isSetSessionHandle()) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + if (struct.configuration != null) { + if (struct.isSetConfiguration()) { + oprot.writeFieldBegin(CONFIGURATION_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.configuration.size())); + for (Map.Entry _iter156 : struct.configuration.entrySet()) + { + oprot.writeString(_iter156.getKey()); + oprot.writeString(_iter156.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TOpenSessionRespTupleSchemeFactory implements SchemeFactory { + public TOpenSessionRespTupleScheme getScheme() { + return new TOpenSessionRespTupleScheme(); + } + } + + private static class TOpenSessionRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TOpenSessionResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + oprot.writeI32(struct.serverProtocolVersion.getValue()); + BitSet optionals = new BitSet(); + if (struct.isSetSessionHandle()) { + optionals.set(0); + } + if (struct.isSetConfiguration()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetSessionHandle()) { + struct.sessionHandle.write(oprot); + } + if (struct.isSetConfiguration()) { + { + oprot.writeI32(struct.configuration.size()); + for (Map.Entry _iter157 : struct.configuration.entrySet()) + { + oprot.writeString(_iter157.getKey()); + oprot.writeString(_iter157.getValue()); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TOpenSessionResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + struct.serverProtocolVersion = TProtocolVersion.findByValue(iprot.readI32()); + struct.setServerProtocolVersionIsSet(true); + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } + if (incoming.get(1)) { + { + org.apache.thrift.protocol.TMap _map158 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.configuration = new HashMap(2*_map158.size); + for (int _i159 = 0; _i159 < _map158.size; ++_i159) + { + String _key160; // required + String _val161; // required + _key160 = iprot.readString(); + _val161 = iprot.readString(); + struct.configuration.put(_key160, _val161); + } + } + struct.setConfigurationIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java new file mode 100644 index 000000000000..8fbd8752eaca --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java @@ -0,0 +1,705 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TOperationHandle implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOperationHandle"); + + private static final org.apache.thrift.protocol.TField OPERATION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("operationId", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationType", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField HAS_RESULT_SET_FIELD_DESC = new org.apache.thrift.protocol.TField("hasResultSet", org.apache.thrift.protocol.TType.BOOL, (short)3); + private static final org.apache.thrift.protocol.TField MODIFIED_ROW_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("modifiedRowCount", org.apache.thrift.protocol.TType.DOUBLE, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TOperationHandleStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TOperationHandleTupleSchemeFactory()); + } + + private THandleIdentifier operationId; // required + private TOperationType operationType; // required + private boolean hasResultSet; // required + private double modifiedRowCount; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OPERATION_ID((short)1, "operationId"), + /** + * + * @see TOperationType + */ + OPERATION_TYPE((short)2, "operationType"), + HAS_RESULT_SET((short)3, "hasResultSet"), + MODIFIED_ROW_COUNT((short)4, "modifiedRowCount"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OPERATION_ID + return OPERATION_ID; + case 2: // OPERATION_TYPE + return OPERATION_TYPE; + case 3: // HAS_RESULT_SET + return HAS_RESULT_SET; + case 4: // MODIFIED_ROW_COUNT + return MODIFIED_ROW_COUNT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __HASRESULTSET_ISSET_ID = 0; + private static final int __MODIFIEDROWCOUNT_ISSET_ID = 1; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.MODIFIED_ROW_COUNT}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OPERATION_ID, new org.apache.thrift.meta_data.FieldMetaData("operationId", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, THandleIdentifier.class))); + tmpMap.put(_Fields.OPERATION_TYPE, new org.apache.thrift.meta_data.FieldMetaData("operationType", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TOperationType.class))); + tmpMap.put(_Fields.HAS_RESULT_SET, new org.apache.thrift.meta_data.FieldMetaData("hasResultSet", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.MODIFIED_ROW_COUNT, new org.apache.thrift.meta_data.FieldMetaData("modifiedRowCount", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOperationHandle.class, metaDataMap); + } + + public TOperationHandle() { + } + + public TOperationHandle( + THandleIdentifier operationId, + TOperationType operationType, + boolean hasResultSet) + { + this(); + this.operationId = operationId; + this.operationType = operationType; + this.hasResultSet = hasResultSet; + setHasResultSetIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TOperationHandle(TOperationHandle other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetOperationId()) { + this.operationId = new THandleIdentifier(other.operationId); + } + if (other.isSetOperationType()) { + this.operationType = other.operationType; + } + this.hasResultSet = other.hasResultSet; + this.modifiedRowCount = other.modifiedRowCount; + } + + public TOperationHandle deepCopy() { + return new TOperationHandle(this); + } + + @Override + public void clear() { + this.operationId = null; + this.operationType = null; + setHasResultSetIsSet(false); + this.hasResultSet = false; + setModifiedRowCountIsSet(false); + this.modifiedRowCount = 0.0; + } + + public THandleIdentifier getOperationId() { + return this.operationId; + } + + public void setOperationId(THandleIdentifier operationId) { + this.operationId = operationId; + } + + public void unsetOperationId() { + this.operationId = null; + } + + /** Returns true if field operationId is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationId() { + return this.operationId != null; + } + + public void setOperationIdIsSet(boolean value) { + if (!value) { + this.operationId = null; + } + } + + /** + * + * @see TOperationType + */ + public TOperationType getOperationType() { + return this.operationType; + } + + /** + * + * @see TOperationType + */ + public void setOperationType(TOperationType operationType) { + this.operationType = operationType; + } + + public void unsetOperationType() { + this.operationType = null; + } + + /** Returns true if field operationType is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationType() { + return this.operationType != null; + } + + public void setOperationTypeIsSet(boolean value) { + if (!value) { + this.operationType = null; + } + } + + public boolean isHasResultSet() { + return this.hasResultSet; + } + + public void setHasResultSet(boolean hasResultSet) { + this.hasResultSet = hasResultSet; + setHasResultSetIsSet(true); + } + + public void unsetHasResultSet() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __HASRESULTSET_ISSET_ID); + } + + /** Returns true if field hasResultSet is set (has been assigned a value) and false otherwise */ + public boolean isSetHasResultSet() { + return EncodingUtils.testBit(__isset_bitfield, __HASRESULTSET_ISSET_ID); + } + + public void setHasResultSetIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __HASRESULTSET_ISSET_ID, value); + } + + public double getModifiedRowCount() { + return this.modifiedRowCount; + } + + public void setModifiedRowCount(double modifiedRowCount) { + this.modifiedRowCount = modifiedRowCount; + setModifiedRowCountIsSet(true); + } + + public void unsetModifiedRowCount() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MODIFIEDROWCOUNT_ISSET_ID); + } + + /** Returns true if field modifiedRowCount is set (has been assigned a value) and false otherwise */ + public boolean isSetModifiedRowCount() { + return EncodingUtils.testBit(__isset_bitfield, __MODIFIEDROWCOUNT_ISSET_ID); + } + + public void setModifiedRowCountIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MODIFIEDROWCOUNT_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OPERATION_ID: + if (value == null) { + unsetOperationId(); + } else { + setOperationId((THandleIdentifier)value); + } + break; + + case OPERATION_TYPE: + if (value == null) { + unsetOperationType(); + } else { + setOperationType((TOperationType)value); + } + break; + + case HAS_RESULT_SET: + if (value == null) { + unsetHasResultSet(); + } else { + setHasResultSet((Boolean)value); + } + break; + + case MODIFIED_ROW_COUNT: + if (value == null) { + unsetModifiedRowCount(); + } else { + setModifiedRowCount((Double)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OPERATION_ID: + return getOperationId(); + + case OPERATION_TYPE: + return getOperationType(); + + case HAS_RESULT_SET: + return Boolean.valueOf(isHasResultSet()); + + case MODIFIED_ROW_COUNT: + return Double.valueOf(getModifiedRowCount()); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OPERATION_ID: + return isSetOperationId(); + case OPERATION_TYPE: + return isSetOperationType(); + case HAS_RESULT_SET: + return isSetHasResultSet(); + case MODIFIED_ROW_COUNT: + return isSetModifiedRowCount(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TOperationHandle) + return this.equals((TOperationHandle)that); + return false; + } + + public boolean equals(TOperationHandle that) { + if (that == null) + return false; + + boolean this_present_operationId = true && this.isSetOperationId(); + boolean that_present_operationId = true && that.isSetOperationId(); + if (this_present_operationId || that_present_operationId) { + if (!(this_present_operationId && that_present_operationId)) + return false; + if (!this.operationId.equals(that.operationId)) + return false; + } + + boolean this_present_operationType = true && this.isSetOperationType(); + boolean that_present_operationType = true && that.isSetOperationType(); + if (this_present_operationType || that_present_operationType) { + if (!(this_present_operationType && that_present_operationType)) + return false; + if (!this.operationType.equals(that.operationType)) + return false; + } + + boolean this_present_hasResultSet = true; + boolean that_present_hasResultSet = true; + if (this_present_hasResultSet || that_present_hasResultSet) { + if (!(this_present_hasResultSet && that_present_hasResultSet)) + return false; + if (this.hasResultSet != that.hasResultSet) + return false; + } + + boolean this_present_modifiedRowCount = true && this.isSetModifiedRowCount(); + boolean that_present_modifiedRowCount = true && that.isSetModifiedRowCount(); + if (this_present_modifiedRowCount || that_present_modifiedRowCount) { + if (!(this_present_modifiedRowCount && that_present_modifiedRowCount)) + return false; + if (this.modifiedRowCount != that.modifiedRowCount) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_operationId = true && (isSetOperationId()); + builder.append(present_operationId); + if (present_operationId) + builder.append(operationId); + + boolean present_operationType = true && (isSetOperationType()); + builder.append(present_operationType); + if (present_operationType) + builder.append(operationType.getValue()); + + boolean present_hasResultSet = true; + builder.append(present_hasResultSet); + if (present_hasResultSet) + builder.append(hasResultSet); + + boolean present_modifiedRowCount = true && (isSetModifiedRowCount()); + builder.append(present_modifiedRowCount); + if (present_modifiedRowCount) + builder.append(modifiedRowCount); + + return builder.toHashCode(); + } + + public int compareTo(TOperationHandle other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TOperationHandle typedOther = (TOperationHandle)other; + + lastComparison = Boolean.valueOf(isSetOperationId()).compareTo(typedOther.isSetOperationId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationId, typedOther.operationId); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationType()).compareTo(typedOther.isSetOperationType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationType, typedOther.operationType); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetHasResultSet()).compareTo(typedOther.isSetHasResultSet()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHasResultSet()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hasResultSet, typedOther.hasResultSet); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetModifiedRowCount()).compareTo(typedOther.isSetModifiedRowCount()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetModifiedRowCount()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.modifiedRowCount, typedOther.modifiedRowCount); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TOperationHandle("); + boolean first = true; + + sb.append("operationId:"); + if (this.operationId == null) { + sb.append("null"); + } else { + sb.append(this.operationId); + } + first = false; + if (!first) sb.append(", "); + sb.append("operationType:"); + if (this.operationType == null) { + sb.append("null"); + } else { + sb.append(this.operationType); + } + first = false; + if (!first) sb.append(", "); + sb.append("hasResultSet:"); + sb.append(this.hasResultSet); + first = false; + if (isSetModifiedRowCount()) { + if (!first) sb.append(", "); + sb.append("modifiedRowCount:"); + sb.append(this.modifiedRowCount); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetOperationId()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationId' is unset! Struct:" + toString()); + } + + if (!isSetOperationType()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationType' is unset! Struct:" + toString()); + } + + if (!isSetHasResultSet()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'hasResultSet' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (operationId != null) { + operationId.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TOperationHandleStandardSchemeFactory implements SchemeFactory { + public TOperationHandleStandardScheme getScheme() { + return new TOperationHandleStandardScheme(); + } + } + + private static class TOperationHandleStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TOperationHandle struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OPERATION_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationId = new THandleIdentifier(); + struct.operationId.read(iprot); + struct.setOperationIdIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.operationType = TOperationType.findByValue(iprot.readI32()); + struct.setOperationTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // HAS_RESULT_SET + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.hasResultSet = iprot.readBool(); + struct.setHasResultSetIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // MODIFIED_ROW_COUNT + if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { + struct.modifiedRowCount = iprot.readDouble(); + struct.setModifiedRowCountIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TOperationHandle struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.operationId != null) { + oprot.writeFieldBegin(OPERATION_ID_FIELD_DESC); + struct.operationId.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationType != null) { + oprot.writeFieldBegin(OPERATION_TYPE_FIELD_DESC); + oprot.writeI32(struct.operationType.getValue()); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(HAS_RESULT_SET_FIELD_DESC); + oprot.writeBool(struct.hasResultSet); + oprot.writeFieldEnd(); + if (struct.isSetModifiedRowCount()) { + oprot.writeFieldBegin(MODIFIED_ROW_COUNT_FIELD_DESC); + oprot.writeDouble(struct.modifiedRowCount); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TOperationHandleTupleSchemeFactory implements SchemeFactory { + public TOperationHandleTupleScheme getScheme() { + return new TOperationHandleTupleScheme(); + } + } + + private static class TOperationHandleTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TOperationHandle struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.operationId.write(oprot); + oprot.writeI32(struct.operationType.getValue()); + oprot.writeBool(struct.hasResultSet); + BitSet optionals = new BitSet(); + if (struct.isSetModifiedRowCount()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetModifiedRowCount()) { + oprot.writeDouble(struct.modifiedRowCount); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TOperationHandle struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.operationId = new THandleIdentifier(); + struct.operationId.read(iprot); + struct.setOperationIdIsSet(true); + struct.operationType = TOperationType.findByValue(iprot.readI32()); + struct.setOperationTypeIsSet(true); + struct.hasResultSet = iprot.readBool(); + struct.setHasResultSetIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.modifiedRowCount = iprot.readDouble(); + struct.setModifiedRowCountIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java new file mode 100644 index 000000000000..219866223a6b --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java @@ -0,0 +1,63 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TOperationState implements org.apache.thrift.TEnum { + INITIALIZED_STATE(0), + RUNNING_STATE(1), + FINISHED_STATE(2), + CANCELED_STATE(3), + CLOSED_STATE(4), + ERROR_STATE(5), + UKNOWN_STATE(6), + PENDING_STATE(7); + + private final int value; + + private TOperationState(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TOperationState findByValue(int value) { + switch (value) { + case 0: + return INITIALIZED_STATE; + case 1: + return RUNNING_STATE; + case 2: + return FINISHED_STATE; + case 3: + return CANCELED_STATE; + case 4: + return CLOSED_STATE; + case 5: + return ERROR_STATE; + case 6: + return UKNOWN_STATE; + case 7: + return PENDING_STATE; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java new file mode 100644 index 000000000000..b6d4b2fab9f9 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java @@ -0,0 +1,66 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TOperationType implements org.apache.thrift.TEnum { + EXECUTE_STATEMENT(0), + GET_TYPE_INFO(1), + GET_CATALOGS(2), + GET_SCHEMAS(3), + GET_TABLES(4), + GET_TABLE_TYPES(5), + GET_COLUMNS(6), + GET_FUNCTIONS(7), + UNKNOWN(8); + + private final int value; + + private TOperationType(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TOperationType findByValue(int value) { + switch (value) { + case 0: + return EXECUTE_STATEMENT; + case 1: + return GET_TYPE_INFO; + case 2: + return GET_CATALOGS; + case 3: + return GET_SCHEMAS; + case 4: + return GET_TABLES; + case 5: + return GET_TABLE_TYPES; + case 6: + return GET_COLUMNS; + case 7: + return GET_FUNCTIONS; + case 8: + return UNKNOWN; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java new file mode 100644 index 000000000000..9d2abf2b3b08 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java @@ -0,0 +1,512 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TPrimitiveTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TPrimitiveTypeEntry"); + + private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField TYPE_QUALIFIERS_FIELD_DESC = new org.apache.thrift.protocol.TField("typeQualifiers", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TPrimitiveTypeEntryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TPrimitiveTypeEntryTupleSchemeFactory()); + } + + private TTypeId type; // required + private TTypeQualifiers typeQualifiers; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + /** + * + * @see TTypeId + */ + TYPE((short)1, "type"), + TYPE_QUALIFIERS((short)2, "typeQualifiers"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TYPE + return TYPE; + case 2: // TYPE_QUALIFIERS + return TYPE_QUALIFIERS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.TYPE_QUALIFIERS}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TTypeId.class))); + tmpMap.put(_Fields.TYPE_QUALIFIERS, new org.apache.thrift.meta_data.FieldMetaData("typeQualifiers", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeQualifiers.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TPrimitiveTypeEntry.class, metaDataMap); + } + + public TPrimitiveTypeEntry() { + } + + public TPrimitiveTypeEntry( + TTypeId type) + { + this(); + this.type = type; + } + + /** + * Performs a deep copy on other. + */ + public TPrimitiveTypeEntry(TPrimitiveTypeEntry other) { + if (other.isSetType()) { + this.type = other.type; + } + if (other.isSetTypeQualifiers()) { + this.typeQualifiers = new TTypeQualifiers(other.typeQualifiers); + } + } + + public TPrimitiveTypeEntry deepCopy() { + return new TPrimitiveTypeEntry(this); + } + + @Override + public void clear() { + this.type = null; + this.typeQualifiers = null; + } + + /** + * + * @see TTypeId + */ + public TTypeId getType() { + return this.type; + } + + /** + * + * @see TTypeId + */ + public void setType(TTypeId type) { + this.type = type; + } + + public void unsetType() { + this.type = null; + } + + /** Returns true if field type is set (has been assigned a value) and false otherwise */ + public boolean isSetType() { + return this.type != null; + } + + public void setTypeIsSet(boolean value) { + if (!value) { + this.type = null; + } + } + + public TTypeQualifiers getTypeQualifiers() { + return this.typeQualifiers; + } + + public void setTypeQualifiers(TTypeQualifiers typeQualifiers) { + this.typeQualifiers = typeQualifiers; + } + + public void unsetTypeQualifiers() { + this.typeQualifiers = null; + } + + /** Returns true if field typeQualifiers is set (has been assigned a value) and false otherwise */ + public boolean isSetTypeQualifiers() { + return this.typeQualifiers != null; + } + + public void setTypeQualifiersIsSet(boolean value) { + if (!value) { + this.typeQualifiers = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TYPE: + if (value == null) { + unsetType(); + } else { + setType((TTypeId)value); + } + break; + + case TYPE_QUALIFIERS: + if (value == null) { + unsetTypeQualifiers(); + } else { + setTypeQualifiers((TTypeQualifiers)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TYPE: + return getType(); + + case TYPE_QUALIFIERS: + return getTypeQualifiers(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TYPE: + return isSetType(); + case TYPE_QUALIFIERS: + return isSetTypeQualifiers(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TPrimitiveTypeEntry) + return this.equals((TPrimitiveTypeEntry)that); + return false; + } + + public boolean equals(TPrimitiveTypeEntry that) { + if (that == null) + return false; + + boolean this_present_type = true && this.isSetType(); + boolean that_present_type = true && that.isSetType(); + if (this_present_type || that_present_type) { + if (!(this_present_type && that_present_type)) + return false; + if (!this.type.equals(that.type)) + return false; + } + + boolean this_present_typeQualifiers = true && this.isSetTypeQualifiers(); + boolean that_present_typeQualifiers = true && that.isSetTypeQualifiers(); + if (this_present_typeQualifiers || that_present_typeQualifiers) { + if (!(this_present_typeQualifiers && that_present_typeQualifiers)) + return false; + if (!this.typeQualifiers.equals(that.typeQualifiers)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_type = true && (isSetType()); + builder.append(present_type); + if (present_type) + builder.append(type.getValue()); + + boolean present_typeQualifiers = true && (isSetTypeQualifiers()); + builder.append(present_typeQualifiers); + if (present_typeQualifiers) + builder.append(typeQualifiers); + + return builder.toHashCode(); + } + + public int compareTo(TPrimitiveTypeEntry other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TPrimitiveTypeEntry typedOther = (TPrimitiveTypeEntry)other; + + lastComparison = Boolean.valueOf(isSetType()).compareTo(typedOther.isSetType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, typedOther.type); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTypeQualifiers()).compareTo(typedOther.isSetTypeQualifiers()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTypeQualifiers()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.typeQualifiers, typedOther.typeQualifiers); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TPrimitiveTypeEntry("); + boolean first = true; + + sb.append("type:"); + if (this.type == null) { + sb.append("null"); + } else { + sb.append(this.type); + } + first = false; + if (isSetTypeQualifiers()) { + if (!first) sb.append(", "); + sb.append("typeQualifiers:"); + if (this.typeQualifiers == null) { + sb.append("null"); + } else { + sb.append(this.typeQualifiers); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetType()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'type' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (typeQualifiers != null) { + typeQualifiers.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TPrimitiveTypeEntryStandardSchemeFactory implements SchemeFactory { + public TPrimitiveTypeEntryStandardScheme getScheme() { + return new TPrimitiveTypeEntryStandardScheme(); + } + } + + private static class TPrimitiveTypeEntryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.type = TTypeId.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TYPE_QUALIFIERS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.typeQualifiers = new TTypeQualifiers(); + struct.typeQualifiers.read(iprot); + struct.setTypeQualifiersIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.type != null) { + oprot.writeFieldBegin(TYPE_FIELD_DESC); + oprot.writeI32(struct.type.getValue()); + oprot.writeFieldEnd(); + } + if (struct.typeQualifiers != null) { + if (struct.isSetTypeQualifiers()) { + oprot.writeFieldBegin(TYPE_QUALIFIERS_FIELD_DESC); + struct.typeQualifiers.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TPrimitiveTypeEntryTupleSchemeFactory implements SchemeFactory { + public TPrimitiveTypeEntryTupleScheme getScheme() { + return new TPrimitiveTypeEntryTupleScheme(); + } + } + + private static class TPrimitiveTypeEntryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.type.getValue()); + BitSet optionals = new BitSet(); + if (struct.isSetTypeQualifiers()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetTypeQualifiers()) { + struct.typeQualifiers.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.type = TTypeId.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.typeQualifiers = new TTypeQualifiers(); + struct.typeQualifiers.read(iprot); + struct.setTypeQualifiersIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java new file mode 100644 index 000000000000..a4279d29f662 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java @@ -0,0 +1,63 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TProtocolVersion implements org.apache.thrift.TEnum { + HIVE_CLI_SERVICE_PROTOCOL_V1(0), + HIVE_CLI_SERVICE_PROTOCOL_V2(1), + HIVE_CLI_SERVICE_PROTOCOL_V3(2), + HIVE_CLI_SERVICE_PROTOCOL_V4(3), + HIVE_CLI_SERVICE_PROTOCOL_V5(4), + HIVE_CLI_SERVICE_PROTOCOL_V6(5), + HIVE_CLI_SERVICE_PROTOCOL_V7(6), + HIVE_CLI_SERVICE_PROTOCOL_V8(7); + + private final int value; + + private TProtocolVersion(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TProtocolVersion findByValue(int value) { + switch (value) { + case 0: + return HIVE_CLI_SERVICE_PROTOCOL_V1; + case 1: + return HIVE_CLI_SERVICE_PROTOCOL_V2; + case 2: + return HIVE_CLI_SERVICE_PROTOCOL_V3; + case 3: + return HIVE_CLI_SERVICE_PROTOCOL_V4; + case 4: + return HIVE_CLI_SERVICE_PROTOCOL_V5; + case 5: + return HIVE_CLI_SERVICE_PROTOCOL_V6; + case 6: + return HIVE_CLI_SERVICE_PROTOCOL_V7; + case 7: + return HIVE_CLI_SERVICE_PROTOCOL_V8; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java new file mode 100644 index 000000000000..a3e39c8cdf32 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java @@ -0,0 +1,491 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TRenewDelegationTokenReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRenewDelegationTokenReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField DELEGATION_TOKEN_FIELD_DESC = new org.apache.thrift.protocol.TField("delegationToken", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TRenewDelegationTokenReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TRenewDelegationTokenReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String delegationToken; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + DELEGATION_TOKEN((short)2, "delegationToken"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // DELEGATION_TOKEN + return DELEGATION_TOKEN; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.DELEGATION_TOKEN, new org.apache.thrift.meta_data.FieldMetaData("delegationToken", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRenewDelegationTokenReq.class, metaDataMap); + } + + public TRenewDelegationTokenReq() { + } + + public TRenewDelegationTokenReq( + TSessionHandle sessionHandle, + String delegationToken) + { + this(); + this.sessionHandle = sessionHandle; + this.delegationToken = delegationToken; + } + + /** + * Performs a deep copy on other. + */ + public TRenewDelegationTokenReq(TRenewDelegationTokenReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetDelegationToken()) { + this.delegationToken = other.delegationToken; + } + } + + public TRenewDelegationTokenReq deepCopy() { + return new TRenewDelegationTokenReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.delegationToken = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getDelegationToken() { + return this.delegationToken; + } + + public void setDelegationToken(String delegationToken) { + this.delegationToken = delegationToken; + } + + public void unsetDelegationToken() { + this.delegationToken = null; + } + + /** Returns true if field delegationToken is set (has been assigned a value) and false otherwise */ + public boolean isSetDelegationToken() { + return this.delegationToken != null; + } + + public void setDelegationTokenIsSet(boolean value) { + if (!value) { + this.delegationToken = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case DELEGATION_TOKEN: + if (value == null) { + unsetDelegationToken(); + } else { + setDelegationToken((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case DELEGATION_TOKEN: + return getDelegationToken(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case DELEGATION_TOKEN: + return isSetDelegationToken(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TRenewDelegationTokenReq) + return this.equals((TRenewDelegationTokenReq)that); + return false; + } + + public boolean equals(TRenewDelegationTokenReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_delegationToken = true && this.isSetDelegationToken(); + boolean that_present_delegationToken = true && that.isSetDelegationToken(); + if (this_present_delegationToken || that_present_delegationToken) { + if (!(this_present_delegationToken && that_present_delegationToken)) + return false; + if (!this.delegationToken.equals(that.delegationToken)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + builder.append(present_sessionHandle); + if (present_sessionHandle) + builder.append(sessionHandle); + + boolean present_delegationToken = true && (isSetDelegationToken()); + builder.append(present_delegationToken); + if (present_delegationToken) + builder.append(delegationToken); + + return builder.toHashCode(); + } + + public int compareTo(TRenewDelegationTokenReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TRenewDelegationTokenReq typedOther = (TRenewDelegationTokenReq)other; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(typedOther.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, typedOther.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetDelegationToken()).compareTo(typedOther.isSetDelegationToken()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDelegationToken()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.delegationToken, typedOther.delegationToken); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TRenewDelegationTokenReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (!first) sb.append(", "); + sb.append("delegationToken:"); + if (this.delegationToken == null) { + sb.append("null"); + } else { + sb.append(this.delegationToken); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + if (!isSetDelegationToken()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'delegationToken' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TRenewDelegationTokenReqStandardSchemeFactory implements SchemeFactory { + public TRenewDelegationTokenReqStandardScheme getScheme() { + return new TRenewDelegationTokenReqStandardScheme(); + } + } + + private static class TRenewDelegationTokenReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // DELEGATION_TOKEN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.delegationToken = iprot.readString(); + struct.setDelegationTokenIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.delegationToken != null) { + oprot.writeFieldBegin(DELEGATION_TOKEN_FIELD_DESC); + oprot.writeString(struct.delegationToken); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TRenewDelegationTokenReqTupleSchemeFactory implements SchemeFactory { + public TRenewDelegationTokenReqTupleScheme getScheme() { + return new TRenewDelegationTokenReqTupleScheme(); + } + } + + private static class TRenewDelegationTokenReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + oprot.writeString(struct.delegationToken); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + struct.delegationToken = iprot.readString(); + struct.setDelegationTokenIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java new file mode 100644 index 000000000000..5f3eb6c4d4b9 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TRenewDelegationTokenResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRenewDelegationTokenResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TRenewDelegationTokenRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TRenewDelegationTokenRespTupleSchemeFactory()); + } + + private TStatus status; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRenewDelegationTokenResp.class, metaDataMap); + } + + public TRenewDelegationTokenResp() { + } + + public TRenewDelegationTokenResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TRenewDelegationTokenResp(TRenewDelegationTokenResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + } + + public TRenewDelegationTokenResp deepCopy() { + return new TRenewDelegationTokenResp(this); + } + + @Override + public void clear() { + this.status = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TRenewDelegationTokenResp) + return this.equals((TRenewDelegationTokenResp)that); + return false; + } + + public boolean equals(TRenewDelegationTokenResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_status = true && (isSetStatus()); + builder.append(present_status); + if (present_status) + builder.append(status); + + return builder.toHashCode(); + } + + public int compareTo(TRenewDelegationTokenResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TRenewDelegationTokenResp typedOther = (TRenewDelegationTokenResp)other; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(typedOther.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, typedOther.status); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TRenewDelegationTokenResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TRenewDelegationTokenRespStandardSchemeFactory implements SchemeFactory { + public TRenewDelegationTokenRespStandardScheme getScheme() { + return new TRenewDelegationTokenRespStandardScheme(); + } + } + + private static class TRenewDelegationTokenRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TRenewDelegationTokenRespTupleSchemeFactory implements SchemeFactory { + public TRenewDelegationTokenRespTupleScheme getScheme() { + return new TRenewDelegationTokenRespTupleScheme(); + } + } + + private static class TRenewDelegationTokenRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java new file mode 100644 index 000000000000..a44cfb08ff01 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java @@ -0,0 +1,439 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TRow implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRow"); + + private static final org.apache.thrift.protocol.TField COL_VALS_FIELD_DESC = new org.apache.thrift.protocol.TField("colVals", org.apache.thrift.protocol.TType.LIST, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TRowStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TRowTupleSchemeFactory()); + } + + private List colVals; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + COL_VALS((short)1, "colVals"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // COL_VALS + return COL_VALS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.COL_VALS, new org.apache.thrift.meta_data.FieldMetaData("colVals", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnValue.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRow.class, metaDataMap); + } + + public TRow() { + } + + public TRow( + List colVals) + { + this(); + this.colVals = colVals; + } + + /** + * Performs a deep copy on other. + */ + public TRow(TRow other) { + if (other.isSetColVals()) { + List __this__colVals = new ArrayList(); + for (TColumnValue other_element : other.colVals) { + __this__colVals.add(new TColumnValue(other_element)); + } + this.colVals = __this__colVals; + } + } + + public TRow deepCopy() { + return new TRow(this); + } + + @Override + public void clear() { + this.colVals = null; + } + + public int getColValsSize() { + return (this.colVals == null) ? 0 : this.colVals.size(); + } + + public java.util.Iterator getColValsIterator() { + return (this.colVals == null) ? null : this.colVals.iterator(); + } + + public void addToColVals(TColumnValue elem) { + if (this.colVals == null) { + this.colVals = new ArrayList(); + } + this.colVals.add(elem); + } + + public List getColVals() { + return this.colVals; + } + + public void setColVals(List colVals) { + this.colVals = colVals; + } + + public void unsetColVals() { + this.colVals = null; + } + + /** Returns true if field colVals is set (has been assigned a value) and false otherwise */ + public boolean isSetColVals() { + return this.colVals != null; + } + + public void setColValsIsSet(boolean value) { + if (!value) { + this.colVals = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case COL_VALS: + if (value == null) { + unsetColVals(); + } else { + setColVals((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case COL_VALS: + return getColVals(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case COL_VALS: + return isSetColVals(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TRow) + return this.equals((TRow)that); + return false; + } + + public boolean equals(TRow that) { + if (that == null) + return false; + + boolean this_present_colVals = true && this.isSetColVals(); + boolean that_present_colVals = true && that.isSetColVals(); + if (this_present_colVals || that_present_colVals) { + if (!(this_present_colVals && that_present_colVals)) + return false; + if (!this.colVals.equals(that.colVals)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_colVals = true && (isSetColVals()); + builder.append(present_colVals); + if (present_colVals) + builder.append(colVals); + + return builder.toHashCode(); + } + + public int compareTo(TRow other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TRow typedOther = (TRow)other; + + lastComparison = Boolean.valueOf(isSetColVals()).compareTo(typedOther.isSetColVals()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetColVals()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.colVals, typedOther.colVals); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TRow("); + boolean first = true; + + sb.append("colVals:"); + if (this.colVals == null) { + sb.append("null"); + } else { + sb.append(this.colVals); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetColVals()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'colVals' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TRowStandardSchemeFactory implements SchemeFactory { + public TRowStandardScheme getScheme() { + return new TRowStandardScheme(); + } + } + + private static class TRowStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TRow struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // COL_VALS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list46 = iprot.readListBegin(); + struct.colVals = new ArrayList(_list46.size); + for (int _i47 = 0; _i47 < _list46.size; ++_i47) + { + TColumnValue _elem48; // optional + _elem48 = new TColumnValue(); + _elem48.read(iprot); + struct.colVals.add(_elem48); + } + iprot.readListEnd(); + } + struct.setColValsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TRow struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.colVals != null) { + oprot.writeFieldBegin(COL_VALS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.colVals.size())); + for (TColumnValue _iter49 : struct.colVals) + { + _iter49.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TRowTupleSchemeFactory implements SchemeFactory { + public TRowTupleScheme getScheme() { + return new TRowTupleScheme(); + } + } + + private static class TRowTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TRow struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.colVals.size()); + for (TColumnValue _iter50 : struct.colVals) + { + _iter50.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TRow struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list51 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.colVals = new ArrayList(_list51.size); + for (int _i52 = 0; _i52 < _list51.size; ++_i52) + { + TColumnValue _elem53; // optional + _elem53 = new TColumnValue(); + _elem53.read(iprot); + struct.colVals.add(_elem53); + } + } + struct.setColValsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java new file mode 100644 index 000000000000..d16c8a4bb32d --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java @@ -0,0 +1,702 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TRowSet implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowSet"); + + private static final org.apache.thrift.protocol.TField START_ROW_OFFSET_FIELD_DESC = new org.apache.thrift.protocol.TField("startRowOffset", org.apache.thrift.protocol.TType.I64, (short)1); + private static final org.apache.thrift.protocol.TField ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("rows", org.apache.thrift.protocol.TType.LIST, (short)2); + private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TRowSetStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TRowSetTupleSchemeFactory()); + } + + private long startRowOffset; // required + private List rows; // required + private List columns; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + START_ROW_OFFSET((short)1, "startRowOffset"), + ROWS((short)2, "rows"), + COLUMNS((short)3, "columns"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // START_ROW_OFFSET + return START_ROW_OFFSET; + case 2: // ROWS + return ROWS; + case 3: // COLUMNS + return COLUMNS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __STARTROWOFFSET_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.COLUMNS}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.START_ROW_OFFSET, new org.apache.thrift.meta_data.FieldMetaData("startRowOffset", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.ROWS, new org.apache.thrift.meta_data.FieldMetaData("rows", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRow.class)))); + tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumn.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRowSet.class, metaDataMap); + } + + public TRowSet() { + } + + public TRowSet( + long startRowOffset, + List rows) + { + this(); + this.startRowOffset = startRowOffset; + setStartRowOffsetIsSet(true); + this.rows = rows; + } + + /** + * Performs a deep copy on other. + */ + public TRowSet(TRowSet other) { + __isset_bitfield = other.__isset_bitfield; + this.startRowOffset = other.startRowOffset; + if (other.isSetRows()) { + List __this__rows = new ArrayList(); + for (TRow other_element : other.rows) { + __this__rows.add(new TRow(other_element)); + } + this.rows = __this__rows; + } + if (other.isSetColumns()) { + List __this__columns = new ArrayList(); + for (TColumn other_element : other.columns) { + __this__columns.add(new TColumn(other_element)); + } + this.columns = __this__columns; + } + } + + public TRowSet deepCopy() { + return new TRowSet(this); + } + + @Override + public void clear() { + setStartRowOffsetIsSet(false); + this.startRowOffset = 0; + this.rows = null; + this.columns = null; + } + + public long getStartRowOffset() { + return this.startRowOffset; + } + + public void setStartRowOffset(long startRowOffset) { + this.startRowOffset = startRowOffset; + setStartRowOffsetIsSet(true); + } + + public void unsetStartRowOffset() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STARTROWOFFSET_ISSET_ID); + } + + /** Returns true if field startRowOffset is set (has been assigned a value) and false otherwise */ + public boolean isSetStartRowOffset() { + return EncodingUtils.testBit(__isset_bitfield, __STARTROWOFFSET_ISSET_ID); + } + + public void setStartRowOffsetIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STARTROWOFFSET_ISSET_ID, value); + } + + public int getRowsSize() { + return (this.rows == null) ? 0 : this.rows.size(); + } + + public java.util.Iterator getRowsIterator() { + return (this.rows == null) ? null : this.rows.iterator(); + } + + public void addToRows(TRow elem) { + if (this.rows == null) { + this.rows = new ArrayList(); + } + this.rows.add(elem); + } + + public List getRows() { + return this.rows; + } + + public void setRows(List rows) { + this.rows = rows; + } + + public void unsetRows() { + this.rows = null; + } + + /** Returns true if field rows is set (has been assigned a value) and false otherwise */ + public boolean isSetRows() { + return this.rows != null; + } + + public void setRowsIsSet(boolean value) { + if (!value) { + this.rows = null; + } + } + + public int getColumnsSize() { + return (this.columns == null) ? 0 : this.columns.size(); + } + + public java.util.Iterator getColumnsIterator() { + return (this.columns == null) ? null : this.columns.iterator(); + } + + public void addToColumns(TColumn elem) { + if (this.columns == null) { + this.columns = new ArrayList(); + } + this.columns.add(elem); + } + + public List getColumns() { + return this.columns; + } + + public void setColumns(List columns) { + this.columns = columns; + } + + public void unsetColumns() { + this.columns = null; + } + + /** Returns true if field columns is set (has been assigned a value) and false otherwise */ + public boolean isSetColumns() { + return this.columns != null; + } + + public void setColumnsIsSet(boolean value) { + if (!value) { + this.columns = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case START_ROW_OFFSET: + if (value == null) { + unsetStartRowOffset(); + } else { + setStartRowOffset((Long)value); + } + break; + + case ROWS: + if (value == null) { + unsetRows(); + } else { + setRows((List)value); + } + break; + + case COLUMNS: + if (value == null) { + unsetColumns(); + } else { + setColumns((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case START_ROW_OFFSET: + return Long.valueOf(getStartRowOffset()); + + case ROWS: + return getRows(); + + case COLUMNS: + return getColumns(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case START_ROW_OFFSET: + return isSetStartRowOffset(); + case ROWS: + return isSetRows(); + case COLUMNS: + return isSetColumns(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TRowSet) + return this.equals((TRowSet)that); + return false; + } + + public boolean equals(TRowSet that) { + if (that == null) + return false; + + boolean this_present_startRowOffset = true; + boolean that_present_startRowOffset = true; + if (this_present_startRowOffset || that_present_startRowOffset) { + if (!(this_present_startRowOffset && that_present_startRowOffset)) + return false; + if (this.startRowOffset != that.startRowOffset) + return false; + } + + boolean this_present_rows = true && this.isSetRows(); + boolean that_present_rows = true && that.isSetRows(); + if (this_present_rows || that_present_rows) { + if (!(this_present_rows && that_present_rows)) + return false; + if (!this.rows.equals(that.rows)) + return false; + } + + boolean this_present_columns = true && this.isSetColumns(); + boolean that_present_columns = true && that.isSetColumns(); + if (this_present_columns || that_present_columns) { + if (!(this_present_columns && that_present_columns)) + return false; + if (!this.columns.equals(that.columns)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_startRowOffset = true; + builder.append(present_startRowOffset); + if (present_startRowOffset) + builder.append(startRowOffset); + + boolean present_rows = true && (isSetRows()); + builder.append(present_rows); + if (present_rows) + builder.append(rows); + + boolean present_columns = true && (isSetColumns()); + builder.append(present_columns); + if (present_columns) + builder.append(columns); + + return builder.toHashCode(); + } + + public int compareTo(TRowSet other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TRowSet typedOther = (TRowSet)other; + + lastComparison = Boolean.valueOf(isSetStartRowOffset()).compareTo(typedOther.isSetStartRowOffset()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStartRowOffset()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRowOffset, typedOther.startRowOffset); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetRows()).compareTo(typedOther.isSetRows()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRows()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rows, typedOther.rows); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetColumns()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TRowSet("); + boolean first = true; + + sb.append("startRowOffset:"); + sb.append(this.startRowOffset); + first = false; + if (!first) sb.append(", "); + sb.append("rows:"); + if (this.rows == null) { + sb.append("null"); + } else { + sb.append(this.rows); + } + first = false; + if (isSetColumns()) { + if (!first) sb.append(", "); + sb.append("columns:"); + if (this.columns == null) { + sb.append("null"); + } else { + sb.append(this.columns); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStartRowOffset()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'startRowOffset' is unset! Struct:" + toString()); + } + + if (!isSetRows()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'rows' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TRowSetStandardSchemeFactory implements SchemeFactory { + public TRowSetStandardScheme getScheme() { + return new TRowSetStandardScheme(); + } + } + + private static class TRowSetStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TRowSet struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // START_ROW_OFFSET + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.startRowOffset = iprot.readI64(); + struct.setStartRowOffsetIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // ROWS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list118 = iprot.readListBegin(); + struct.rows = new ArrayList(_list118.size); + for (int _i119 = 0; _i119 < _list118.size; ++_i119) + { + TRow _elem120; // optional + _elem120 = new TRow(); + _elem120.read(iprot); + struct.rows.add(_elem120); + } + iprot.readListEnd(); + } + struct.setRowsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // COLUMNS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list121 = iprot.readListBegin(); + struct.columns = new ArrayList(_list121.size); + for (int _i122 = 0; _i122 < _list121.size; ++_i122) + { + TColumn _elem123; // optional + _elem123 = new TColumn(); + _elem123.read(iprot); + struct.columns.add(_elem123); + } + iprot.readListEnd(); + } + struct.setColumnsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TRowSet struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(START_ROW_OFFSET_FIELD_DESC); + oprot.writeI64(struct.startRowOffset); + oprot.writeFieldEnd(); + if (struct.rows != null) { + oprot.writeFieldBegin(ROWS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.rows.size())); + for (TRow _iter124 : struct.rows) + { + _iter124.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.columns != null) { + if (struct.isSetColumns()) { + oprot.writeFieldBegin(COLUMNS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size())); + for (TColumn _iter125 : struct.columns) + { + _iter125.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TRowSetTupleSchemeFactory implements SchemeFactory { + public TRowSetTupleScheme getScheme() { + return new TRowSetTupleScheme(); + } + } + + private static class TRowSetTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TRowSet struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI64(struct.startRowOffset); + { + oprot.writeI32(struct.rows.size()); + for (TRow _iter126 : struct.rows) + { + _iter126.write(oprot); + } + } + BitSet optionals = new BitSet(); + if (struct.isSetColumns()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetColumns()) { + { + oprot.writeI32(struct.columns.size()); + for (TColumn _iter127 : struct.columns) + { + _iter127.write(oprot); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TRowSet struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.startRowOffset = iprot.readI64(); + struct.setStartRowOffsetIsSet(true); + { + org.apache.thrift.protocol.TList _list128 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.rows = new ArrayList(_list128.size); + for (int _i129 = 0; _i129 < _list128.size; ++_i129) + { + TRow _elem130; // optional + _elem130 = new TRow(); + _elem130.read(iprot); + struct.rows.add(_elem130); + } + } + struct.setRowsIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TList _list131 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.columns = new ArrayList(_list131.size); + for (int _i132 = 0; _i132 < _list131.size; ++_i132) + { + TColumn _elem133; // optional + _elem133 = new TColumn(); + _elem133.read(iprot); + struct.columns.add(_elem133); + } + } + struct.setColumnsIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java new file mode 100644 index 000000000000..82c00dd68a98 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TSessionHandle implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TSessionHandle"); + + private static final org.apache.thrift.protocol.TField SESSION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionId", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TSessionHandleStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TSessionHandleTupleSchemeFactory()); + } + + private THandleIdentifier sessionId; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_ID((short)1, "sessionId"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_ID + return SESSION_ID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_ID, new org.apache.thrift.meta_data.FieldMetaData("sessionId", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, THandleIdentifier.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TSessionHandle.class, metaDataMap); + } + + public TSessionHandle() { + } + + public TSessionHandle( + THandleIdentifier sessionId) + { + this(); + this.sessionId = sessionId; + } + + /** + * Performs a deep copy on other. + */ + public TSessionHandle(TSessionHandle other) { + if (other.isSetSessionId()) { + this.sessionId = new THandleIdentifier(other.sessionId); + } + } + + public TSessionHandle deepCopy() { + return new TSessionHandle(this); + } + + @Override + public void clear() { + this.sessionId = null; + } + + public THandleIdentifier getSessionId() { + return this.sessionId; + } + + public void setSessionId(THandleIdentifier sessionId) { + this.sessionId = sessionId; + } + + public void unsetSessionId() { + this.sessionId = null; + } + + /** Returns true if field sessionId is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionId() { + return this.sessionId != null; + } + + public void setSessionIdIsSet(boolean value) { + if (!value) { + this.sessionId = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_ID: + if (value == null) { + unsetSessionId(); + } else { + setSessionId((THandleIdentifier)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_ID: + return getSessionId(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_ID: + return isSetSessionId(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TSessionHandle) + return this.equals((TSessionHandle)that); + return false; + } + + public boolean equals(TSessionHandle that) { + if (that == null) + return false; + + boolean this_present_sessionId = true && this.isSetSessionId(); + boolean that_present_sessionId = true && that.isSetSessionId(); + if (this_present_sessionId || that_present_sessionId) { + if (!(this_present_sessionId && that_present_sessionId)) + return false; + if (!this.sessionId.equals(that.sessionId)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_sessionId = true && (isSetSessionId()); + builder.append(present_sessionId); + if (present_sessionId) + builder.append(sessionId); + + return builder.toHashCode(); + } + + public int compareTo(TSessionHandle other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TSessionHandle typedOther = (TSessionHandle)other; + + lastComparison = Boolean.valueOf(isSetSessionId()).compareTo(typedOther.isSetSessionId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionId, typedOther.sessionId); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TSessionHandle("); + boolean first = true; + + sb.append("sessionId:"); + if (this.sessionId == null) { + sb.append("null"); + } else { + sb.append(this.sessionId); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionId()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionId' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionId != null) { + sessionId.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TSessionHandleStandardSchemeFactory implements SchemeFactory { + public TSessionHandleStandardScheme getScheme() { + return new TSessionHandleStandardScheme(); + } + } + + private static class TSessionHandleStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TSessionHandle struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionId = new THandleIdentifier(); + struct.sessionId.read(iprot); + struct.setSessionIdIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TSessionHandle struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionId != null) { + oprot.writeFieldBegin(SESSION_ID_FIELD_DESC); + struct.sessionId.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TSessionHandleTupleSchemeFactory implements SchemeFactory { + public TSessionHandleTupleScheme getScheme() { + return new TSessionHandleTupleScheme(); + } + } + + private static class TSessionHandleTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TSessionHandle struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionId.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TSessionHandle struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionId = new THandleIdentifier(); + struct.sessionId.read(iprot); + struct.setSessionIdIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java new file mode 100644 index 000000000000..24a746e94965 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java @@ -0,0 +1,874 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TStatus implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStatus"); + + private static final org.apache.thrift.protocol.TField STATUS_CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("statusCode", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField INFO_MESSAGES_FIELD_DESC = new org.apache.thrift.protocol.TField("infoMessages", org.apache.thrift.protocol.TType.LIST, (short)2); + private static final org.apache.thrift.protocol.TField SQL_STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("sqlState", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField ERROR_CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorCode", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField ERROR_MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorMessage", org.apache.thrift.protocol.TType.STRING, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TStatusStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TStatusTupleSchemeFactory()); + } + + private TStatusCode statusCode; // required + private List infoMessages; // optional + private String sqlState; // optional + private int errorCode; // optional + private String errorMessage; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + /** + * + * @see TStatusCode + */ + STATUS_CODE((short)1, "statusCode"), + INFO_MESSAGES((short)2, "infoMessages"), + SQL_STATE((short)3, "sqlState"), + ERROR_CODE((short)4, "errorCode"), + ERROR_MESSAGE((short)5, "errorMessage"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS_CODE + return STATUS_CODE; + case 2: // INFO_MESSAGES + return INFO_MESSAGES; + case 3: // SQL_STATE + return SQL_STATE; + case 4: // ERROR_CODE + return ERROR_CODE; + case 5: // ERROR_MESSAGE + return ERROR_MESSAGE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __ERRORCODE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private _Fields optionals[] = {_Fields.INFO_MESSAGES,_Fields.SQL_STATE,_Fields.ERROR_CODE,_Fields.ERROR_MESSAGE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS_CODE, new org.apache.thrift.meta_data.FieldMetaData("statusCode", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TStatusCode.class))); + tmpMap.put(_Fields.INFO_MESSAGES, new org.apache.thrift.meta_data.FieldMetaData("infoMessages", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.SQL_STATE, new org.apache.thrift.meta_data.FieldMetaData("sqlState", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.ERROR_CODE, new org.apache.thrift.meta_data.FieldMetaData("errorCode", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.ERROR_MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("errorMessage", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStatus.class, metaDataMap); + } + + public TStatus() { + } + + public TStatus( + TStatusCode statusCode) + { + this(); + this.statusCode = statusCode; + } + + /** + * Performs a deep copy on other. + */ + public TStatus(TStatus other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetStatusCode()) { + this.statusCode = other.statusCode; + } + if (other.isSetInfoMessages()) { + List __this__infoMessages = new ArrayList(); + for (String other_element : other.infoMessages) { + __this__infoMessages.add(other_element); + } + this.infoMessages = __this__infoMessages; + } + if (other.isSetSqlState()) { + this.sqlState = other.sqlState; + } + this.errorCode = other.errorCode; + if (other.isSetErrorMessage()) { + this.errorMessage = other.errorMessage; + } + } + + public TStatus deepCopy() { + return new TStatus(this); + } + + @Override + public void clear() { + this.statusCode = null; + this.infoMessages = null; + this.sqlState = null; + setErrorCodeIsSet(false); + this.errorCode = 0; + this.errorMessage = null; + } + + /** + * + * @see TStatusCode + */ + public TStatusCode getStatusCode() { + return this.statusCode; + } + + /** + * + * @see TStatusCode + */ + public void setStatusCode(TStatusCode statusCode) { + this.statusCode = statusCode; + } + + public void unsetStatusCode() { + this.statusCode = null; + } + + /** Returns true if field statusCode is set (has been assigned a value) and false otherwise */ + public boolean isSetStatusCode() { + return this.statusCode != null; + } + + public void setStatusCodeIsSet(boolean value) { + if (!value) { + this.statusCode = null; + } + } + + public int getInfoMessagesSize() { + return (this.infoMessages == null) ? 0 : this.infoMessages.size(); + } + + public java.util.Iterator getInfoMessagesIterator() { + return (this.infoMessages == null) ? null : this.infoMessages.iterator(); + } + + public void addToInfoMessages(String elem) { + if (this.infoMessages == null) { + this.infoMessages = new ArrayList(); + } + this.infoMessages.add(elem); + } + + public List getInfoMessages() { + return this.infoMessages; + } + + public void setInfoMessages(List infoMessages) { + this.infoMessages = infoMessages; + } + + public void unsetInfoMessages() { + this.infoMessages = null; + } + + /** Returns true if field infoMessages is set (has been assigned a value) and false otherwise */ + public boolean isSetInfoMessages() { + return this.infoMessages != null; + } + + public void setInfoMessagesIsSet(boolean value) { + if (!value) { + this.infoMessages = null; + } + } + + public String getSqlState() { + return this.sqlState; + } + + public void setSqlState(String sqlState) { + this.sqlState = sqlState; + } + + public void unsetSqlState() { + this.sqlState = null; + } + + /** Returns true if field sqlState is set (has been assigned a value) and false otherwise */ + public boolean isSetSqlState() { + return this.sqlState != null; + } + + public void setSqlStateIsSet(boolean value) { + if (!value) { + this.sqlState = null; + } + } + + public int getErrorCode() { + return this.errorCode; + } + + public void setErrorCode(int errorCode) { + this.errorCode = errorCode; + setErrorCodeIsSet(true); + } + + public void unsetErrorCode() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ERRORCODE_ISSET_ID); + } + + /** Returns true if field errorCode is set (has been assigned a value) and false otherwise */ + public boolean isSetErrorCode() { + return EncodingUtils.testBit(__isset_bitfield, __ERRORCODE_ISSET_ID); + } + + public void setErrorCodeIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ERRORCODE_ISSET_ID, value); + } + + public String getErrorMessage() { + return this.errorMessage; + } + + public void setErrorMessage(String errorMessage) { + this.errorMessage = errorMessage; + } + + public void unsetErrorMessage() { + this.errorMessage = null; + } + + /** Returns true if field errorMessage is set (has been assigned a value) and false otherwise */ + public boolean isSetErrorMessage() { + return this.errorMessage != null; + } + + public void setErrorMessageIsSet(boolean value) { + if (!value) { + this.errorMessage = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS_CODE: + if (value == null) { + unsetStatusCode(); + } else { + setStatusCode((TStatusCode)value); + } + break; + + case INFO_MESSAGES: + if (value == null) { + unsetInfoMessages(); + } else { + setInfoMessages((List)value); + } + break; + + case SQL_STATE: + if (value == null) { + unsetSqlState(); + } else { + setSqlState((String)value); + } + break; + + case ERROR_CODE: + if (value == null) { + unsetErrorCode(); + } else { + setErrorCode((Integer)value); + } + break; + + case ERROR_MESSAGE: + if (value == null) { + unsetErrorMessage(); + } else { + setErrorMessage((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS_CODE: + return getStatusCode(); + + case INFO_MESSAGES: + return getInfoMessages(); + + case SQL_STATE: + return getSqlState(); + + case ERROR_CODE: + return Integer.valueOf(getErrorCode()); + + case ERROR_MESSAGE: + return getErrorMessage(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS_CODE: + return isSetStatusCode(); + case INFO_MESSAGES: + return isSetInfoMessages(); + case SQL_STATE: + return isSetSqlState(); + case ERROR_CODE: + return isSetErrorCode(); + case ERROR_MESSAGE: + return isSetErrorMessage(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TStatus) + return this.equals((TStatus)that); + return false; + } + + public boolean equals(TStatus that) { + if (that == null) + return false; + + boolean this_present_statusCode = true && this.isSetStatusCode(); + boolean that_present_statusCode = true && that.isSetStatusCode(); + if (this_present_statusCode || that_present_statusCode) { + if (!(this_present_statusCode && that_present_statusCode)) + return false; + if (!this.statusCode.equals(that.statusCode)) + return false; + } + + boolean this_present_infoMessages = true && this.isSetInfoMessages(); + boolean that_present_infoMessages = true && that.isSetInfoMessages(); + if (this_present_infoMessages || that_present_infoMessages) { + if (!(this_present_infoMessages && that_present_infoMessages)) + return false; + if (!this.infoMessages.equals(that.infoMessages)) + return false; + } + + boolean this_present_sqlState = true && this.isSetSqlState(); + boolean that_present_sqlState = true && that.isSetSqlState(); + if (this_present_sqlState || that_present_sqlState) { + if (!(this_present_sqlState && that_present_sqlState)) + return false; + if (!this.sqlState.equals(that.sqlState)) + return false; + } + + boolean this_present_errorCode = true && this.isSetErrorCode(); + boolean that_present_errorCode = true && that.isSetErrorCode(); + if (this_present_errorCode || that_present_errorCode) { + if (!(this_present_errorCode && that_present_errorCode)) + return false; + if (this.errorCode != that.errorCode) + return false; + } + + boolean this_present_errorMessage = true && this.isSetErrorMessage(); + boolean that_present_errorMessage = true && that.isSetErrorMessage(); + if (this_present_errorMessage || that_present_errorMessage) { + if (!(this_present_errorMessage && that_present_errorMessage)) + return false; + if (!this.errorMessage.equals(that.errorMessage)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_statusCode = true && (isSetStatusCode()); + builder.append(present_statusCode); + if (present_statusCode) + builder.append(statusCode.getValue()); + + boolean present_infoMessages = true && (isSetInfoMessages()); + builder.append(present_infoMessages); + if (present_infoMessages) + builder.append(infoMessages); + + boolean present_sqlState = true && (isSetSqlState()); + builder.append(present_sqlState); + if (present_sqlState) + builder.append(sqlState); + + boolean present_errorCode = true && (isSetErrorCode()); + builder.append(present_errorCode); + if (present_errorCode) + builder.append(errorCode); + + boolean present_errorMessage = true && (isSetErrorMessage()); + builder.append(present_errorMessage); + if (present_errorMessage) + builder.append(errorMessage); + + return builder.toHashCode(); + } + + public int compareTo(TStatus other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TStatus typedOther = (TStatus)other; + + lastComparison = Boolean.valueOf(isSetStatusCode()).compareTo(typedOther.isSetStatusCode()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatusCode()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.statusCode, typedOther.statusCode); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetInfoMessages()).compareTo(typedOther.isSetInfoMessages()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetInfoMessages()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.infoMessages, typedOther.infoMessages); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSqlState()).compareTo(typedOther.isSetSqlState()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSqlState()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sqlState, typedOther.sqlState); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetErrorCode()).compareTo(typedOther.isSetErrorCode()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetErrorCode()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorCode, typedOther.errorCode); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetErrorMessage()).compareTo(typedOther.isSetErrorMessage()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetErrorMessage()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorMessage, typedOther.errorMessage); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TStatus("); + boolean first = true; + + sb.append("statusCode:"); + if (this.statusCode == null) { + sb.append("null"); + } else { + sb.append(this.statusCode); + } + first = false; + if (isSetInfoMessages()) { + if (!first) sb.append(", "); + sb.append("infoMessages:"); + if (this.infoMessages == null) { + sb.append("null"); + } else { + sb.append(this.infoMessages); + } + first = false; + } + if (isSetSqlState()) { + if (!first) sb.append(", "); + sb.append("sqlState:"); + if (this.sqlState == null) { + sb.append("null"); + } else { + sb.append(this.sqlState); + } + first = false; + } + if (isSetErrorCode()) { + if (!first) sb.append(", "); + sb.append("errorCode:"); + sb.append(this.errorCode); + first = false; + } + if (isSetErrorMessage()) { + if (!first) sb.append(", "); + sb.append("errorMessage:"); + if (this.errorMessage == null) { + sb.append("null"); + } else { + sb.append(this.errorMessage); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatusCode()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'statusCode' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TStatusStandardSchemeFactory implements SchemeFactory { + public TStatusStandardScheme getScheme() { + return new TStatusStandardScheme(); + } + } + + private static class TStatusStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TStatus struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS_CODE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.statusCode = TStatusCode.findByValue(iprot.readI32()); + struct.setStatusCodeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // INFO_MESSAGES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list134 = iprot.readListBegin(); + struct.infoMessages = new ArrayList(_list134.size); + for (int _i135 = 0; _i135 < _list134.size; ++_i135) + { + String _elem136; // optional + _elem136 = iprot.readString(); + struct.infoMessages.add(_elem136); + } + iprot.readListEnd(); + } + struct.setInfoMessagesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SQL_STATE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.sqlState = iprot.readString(); + struct.setSqlStateIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // ERROR_CODE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.errorCode = iprot.readI32(); + struct.setErrorCodeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // ERROR_MESSAGE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.errorMessage = iprot.readString(); + struct.setErrorMessageIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TStatus struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.statusCode != null) { + oprot.writeFieldBegin(STATUS_CODE_FIELD_DESC); + oprot.writeI32(struct.statusCode.getValue()); + oprot.writeFieldEnd(); + } + if (struct.infoMessages != null) { + if (struct.isSetInfoMessages()) { + oprot.writeFieldBegin(INFO_MESSAGES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.infoMessages.size())); + for (String _iter137 : struct.infoMessages) + { + oprot.writeString(_iter137); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.sqlState != null) { + if (struct.isSetSqlState()) { + oprot.writeFieldBegin(SQL_STATE_FIELD_DESC); + oprot.writeString(struct.sqlState); + oprot.writeFieldEnd(); + } + } + if (struct.isSetErrorCode()) { + oprot.writeFieldBegin(ERROR_CODE_FIELD_DESC); + oprot.writeI32(struct.errorCode); + oprot.writeFieldEnd(); + } + if (struct.errorMessage != null) { + if (struct.isSetErrorMessage()) { + oprot.writeFieldBegin(ERROR_MESSAGE_FIELD_DESC); + oprot.writeString(struct.errorMessage); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TStatusTupleSchemeFactory implements SchemeFactory { + public TStatusTupleScheme getScheme() { + return new TStatusTupleScheme(); + } + } + + private static class TStatusTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TStatus struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.statusCode.getValue()); + BitSet optionals = new BitSet(); + if (struct.isSetInfoMessages()) { + optionals.set(0); + } + if (struct.isSetSqlState()) { + optionals.set(1); + } + if (struct.isSetErrorCode()) { + optionals.set(2); + } + if (struct.isSetErrorMessage()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetInfoMessages()) { + { + oprot.writeI32(struct.infoMessages.size()); + for (String _iter138 : struct.infoMessages) + { + oprot.writeString(_iter138); + } + } + } + if (struct.isSetSqlState()) { + oprot.writeString(struct.sqlState); + } + if (struct.isSetErrorCode()) { + oprot.writeI32(struct.errorCode); + } + if (struct.isSetErrorMessage()) { + oprot.writeString(struct.errorMessage); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TStatus struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.statusCode = TStatusCode.findByValue(iprot.readI32()); + struct.setStatusCodeIsSet(true); + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TList _list139 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.infoMessages = new ArrayList(_list139.size); + for (int _i140 = 0; _i140 < _list139.size; ++_i140) + { + String _elem141; // optional + _elem141 = iprot.readString(); + struct.infoMessages.add(_elem141); + } + } + struct.setInfoMessagesIsSet(true); + } + if (incoming.get(1)) { + struct.sqlState = iprot.readString(); + struct.setSqlStateIsSet(true); + } + if (incoming.get(2)) { + struct.errorCode = iprot.readI32(); + struct.setErrorCodeIsSet(true); + } + if (incoming.get(3)) { + struct.errorMessage = iprot.readString(); + struct.setErrorMessageIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java new file mode 100644 index 000000000000..e7fde45fd131 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java @@ -0,0 +1,54 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TStatusCode implements org.apache.thrift.TEnum { + SUCCESS_STATUS(0), + SUCCESS_WITH_INFO_STATUS(1), + STILL_EXECUTING_STATUS(2), + ERROR_STATUS(3), + INVALID_HANDLE_STATUS(4); + + private final int value; + + private TStatusCode(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TStatusCode findByValue(int value) { + switch (value) { + case 0: + return SUCCESS_STATUS; + case 1: + return SUCCESS_WITH_INFO_STATUS; + case 2: + return STILL_EXECUTING_STATUS; + case 3: + return ERROR_STATUS; + case 4: + return INVALID_HANDLE_STATUS; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java new file mode 100644 index 000000000000..3dae460c8621 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TStringColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStringColumn"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TStringColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TStringColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStringColumn.class, metaDataMap); + } + + public TStringColumn() { + } + + public TStringColumn( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = nulls; + } + + /** + * Performs a deep copy on other. + */ + public TStringColumn(TStringColumn other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(); + for (String other_element : other.values) { + __this__values.add(other_element); + } + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); +; + } + } + + public TStringColumn deepCopy() { + return new TStringColumn(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(String elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return nulls; + } + + public void setNulls(byte[] nulls) { + setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = nulls; + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TStringColumn) + return this.equals((TStringColumn)that); + return false; + } + + public boolean equals(TStringColumn that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_values = true && (isSetValues()); + builder.append(present_values); + if (present_values) + builder.append(values); + + boolean present_nulls = true && (isSetNulls()); + builder.append(present_nulls); + if (present_nulls) + builder.append(nulls); + + return builder.toHashCode(); + } + + public int compareTo(TStringColumn other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TStringColumn typedOther = (TStringColumn)other; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(typedOther.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, typedOther.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TStringColumn("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TStringColumnStandardSchemeFactory implements SchemeFactory { + public TStringColumnStandardScheme getScheme() { + return new TStringColumnStandardScheme(); + } + } + + private static class TStringColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TStringColumn struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list102 = iprot.readListBegin(); + struct.values = new ArrayList(_list102.size); + for (int _i103 = 0; _i103 < _list102.size; ++_i103) + { + String _elem104; // optional + _elem104 = iprot.readString(); + struct.values.add(_elem104); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TStringColumn struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size())); + for (String _iter105 : struct.values) + { + oprot.writeString(_iter105); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TStringColumnTupleSchemeFactory implements SchemeFactory { + public TStringColumnTupleScheme getScheme() { + return new TStringColumnTupleScheme(); + } + } + + private static class TStringColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TStringColumn struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (String _iter106 : struct.values) + { + oprot.writeString(_iter106); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TStringColumn struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list107 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.values = new ArrayList(_list107.size); + for (int _i108 = 0; _i108 < _list107.size; ++_i108) + { + String _elem109; // optional + _elem109 = iprot.readString(); + struct.values.add(_elem109); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java new file mode 100644 index 000000000000..af7a109775a8 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java @@ -0,0 +1,389 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TStringValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStringValue"); + + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TStringValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TStringValueTupleSchemeFactory()); + } + + private String value; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUE((short)1, "value"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUE + return VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private _Fields optionals[] = {_Fields.VALUE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStringValue.class, metaDataMap); + } + + public TStringValue() { + } + + /** + * Performs a deep copy on other. + */ + public TStringValue(TStringValue other) { + if (other.isSetValue()) { + this.value = other.value; + } + } + + public TStringValue deepCopy() { + return new TStringValue(this); + } + + @Override + public void clear() { + this.value = null; + } + + public String getValue() { + return this.value; + } + + public void setValue(String value) { + this.value = value; + } + + public void unsetValue() { + this.value = null; + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return this.value != null; + } + + public void setValueIsSet(boolean value) { + if (!value) { + this.value = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUE: + return getValue(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUE: + return isSetValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TStringValue) + return this.equals((TStringValue)that); + return false; + } + + public boolean equals(TStringValue that) { + if (that == null) + return false; + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (!this.value.equals(that.value)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_value = true && (isSetValue()); + builder.append(present_value); + if (present_value) + builder.append(value); + + return builder.toHashCode(); + } + + public int compareTo(TStringValue other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TStringValue typedOther = (TStringValue)other; + + lastComparison = Boolean.valueOf(isSetValue()).compareTo(typedOther.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, typedOther.value); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TStringValue("); + boolean first = true; + + if (isSetValue()) { + sb.append("value:"); + if (this.value == null) { + sb.append("null"); + } else { + sb.append(this.value); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TStringValueStandardSchemeFactory implements SchemeFactory { + public TStringValueStandardScheme getScheme() { + return new TStringValueStandardScheme(); + } + } + + private static class TStringValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TStringValue struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.value = iprot.readString(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TStringValue struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.value != null) { + if (struct.isSetValue()) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeString(struct.value); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TStringValueTupleSchemeFactory implements SchemeFactory { + public TStringValueTupleScheme getScheme() { + return new TStringValueTupleScheme(); + } + } + + private static class TStringValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TStringValue struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetValue()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetValue()) { + oprot.writeString(struct.value); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TStringValue struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.value = iprot.readString(); + struct.setValueIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java new file mode 100644 index 000000000000..20f5fb6c2907 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java @@ -0,0 +1,448 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TStructTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStructTypeEntry"); + + private static final org.apache.thrift.protocol.TField NAME_TO_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("nameToTypePtr", org.apache.thrift.protocol.TType.MAP, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TStructTypeEntryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TStructTypeEntryTupleSchemeFactory()); + } + + private Map nameToTypePtr; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME_TO_TYPE_PTR((short)1, "nameToTypePtr"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME_TO_TYPE_PTR + return NAME_TO_TYPE_PTR; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME_TO_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("nameToTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr")))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStructTypeEntry.class, metaDataMap); + } + + public TStructTypeEntry() { + } + + public TStructTypeEntry( + Map nameToTypePtr) + { + this(); + this.nameToTypePtr = nameToTypePtr; + } + + /** + * Performs a deep copy on other. + */ + public TStructTypeEntry(TStructTypeEntry other) { + if (other.isSetNameToTypePtr()) { + Map __this__nameToTypePtr = new HashMap(); + for (Map.Entry other_element : other.nameToTypePtr.entrySet()) { + + String other_element_key = other_element.getKey(); + Integer other_element_value = other_element.getValue(); + + String __this__nameToTypePtr_copy_key = other_element_key; + + Integer __this__nameToTypePtr_copy_value = other_element_value; + + __this__nameToTypePtr.put(__this__nameToTypePtr_copy_key, __this__nameToTypePtr_copy_value); + } + this.nameToTypePtr = __this__nameToTypePtr; + } + } + + public TStructTypeEntry deepCopy() { + return new TStructTypeEntry(this); + } + + @Override + public void clear() { + this.nameToTypePtr = null; + } + + public int getNameToTypePtrSize() { + return (this.nameToTypePtr == null) ? 0 : this.nameToTypePtr.size(); + } + + public void putToNameToTypePtr(String key, int val) { + if (this.nameToTypePtr == null) { + this.nameToTypePtr = new HashMap(); + } + this.nameToTypePtr.put(key, val); + } + + public Map getNameToTypePtr() { + return this.nameToTypePtr; + } + + public void setNameToTypePtr(Map nameToTypePtr) { + this.nameToTypePtr = nameToTypePtr; + } + + public void unsetNameToTypePtr() { + this.nameToTypePtr = null; + } + + /** Returns true if field nameToTypePtr is set (has been assigned a value) and false otherwise */ + public boolean isSetNameToTypePtr() { + return this.nameToTypePtr != null; + } + + public void setNameToTypePtrIsSet(boolean value) { + if (!value) { + this.nameToTypePtr = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME_TO_TYPE_PTR: + if (value == null) { + unsetNameToTypePtr(); + } else { + setNameToTypePtr((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME_TO_TYPE_PTR: + return getNameToTypePtr(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME_TO_TYPE_PTR: + return isSetNameToTypePtr(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TStructTypeEntry) + return this.equals((TStructTypeEntry)that); + return false; + } + + public boolean equals(TStructTypeEntry that) { + if (that == null) + return false; + + boolean this_present_nameToTypePtr = true && this.isSetNameToTypePtr(); + boolean that_present_nameToTypePtr = true && that.isSetNameToTypePtr(); + if (this_present_nameToTypePtr || that_present_nameToTypePtr) { + if (!(this_present_nameToTypePtr && that_present_nameToTypePtr)) + return false; + if (!this.nameToTypePtr.equals(that.nameToTypePtr)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_nameToTypePtr = true && (isSetNameToTypePtr()); + builder.append(present_nameToTypePtr); + if (present_nameToTypePtr) + builder.append(nameToTypePtr); + + return builder.toHashCode(); + } + + public int compareTo(TStructTypeEntry other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TStructTypeEntry typedOther = (TStructTypeEntry)other; + + lastComparison = Boolean.valueOf(isSetNameToTypePtr()).compareTo(typedOther.isSetNameToTypePtr()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNameToTypePtr()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nameToTypePtr, typedOther.nameToTypePtr); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TStructTypeEntry("); + boolean first = true; + + sb.append("nameToTypePtr:"); + if (this.nameToTypePtr == null) { + sb.append("null"); + } else { + sb.append(this.nameToTypePtr); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetNameToTypePtr()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nameToTypePtr' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TStructTypeEntryStandardSchemeFactory implements SchemeFactory { + public TStructTypeEntryStandardScheme getScheme() { + return new TStructTypeEntryStandardScheme(); + } + } + + private static class TStructTypeEntryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TStructTypeEntry struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME_TO_TYPE_PTR + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map10 = iprot.readMapBegin(); + struct.nameToTypePtr = new HashMap(2*_map10.size); + for (int _i11 = 0; _i11 < _map10.size; ++_i11) + { + String _key12; // required + int _val13; // required + _key12 = iprot.readString(); + _val13 = iprot.readI32(); + struct.nameToTypePtr.put(_key12, _val13); + } + iprot.readMapEnd(); + } + struct.setNameToTypePtrIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TStructTypeEntry struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.nameToTypePtr != null) { + oprot.writeFieldBegin(NAME_TO_TYPE_PTR_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.nameToTypePtr.size())); + for (Map.Entry _iter14 : struct.nameToTypePtr.entrySet()) + { + oprot.writeString(_iter14.getKey()); + oprot.writeI32(_iter14.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TStructTypeEntryTupleSchemeFactory implements SchemeFactory { + public TStructTypeEntryTupleScheme getScheme() { + return new TStructTypeEntryTupleScheme(); + } + } + + private static class TStructTypeEntryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TStructTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.nameToTypePtr.size()); + for (Map.Entry _iter15 : struct.nameToTypePtr.entrySet()) + { + oprot.writeString(_iter15.getKey()); + oprot.writeI32(_iter15.getValue()); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TStructTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map16 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.nameToTypePtr = new HashMap(2*_map16.size); + for (int _i17 = 0; _i17 < _map16.size; ++_i17) + { + String _key18; // required + int _val19; // required + _key18 = iprot.readString(); + _val19 = iprot.readI32(); + struct.nameToTypePtr.put(_key18, _val19); + } + } + struct.setNameToTypePtrIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java new file mode 100644 index 000000000000..ff5e54db7c16 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java @@ -0,0 +1,439 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TTableSchema implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableSchema"); + + private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TTableSchemaStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TTableSchemaTupleSchemeFactory()); + } + + private List columns; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + COLUMNS((short)1, "columns"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // COLUMNS + return COLUMNS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnDesc.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTableSchema.class, metaDataMap); + } + + public TTableSchema() { + } + + public TTableSchema( + List columns) + { + this(); + this.columns = columns; + } + + /** + * Performs a deep copy on other. + */ + public TTableSchema(TTableSchema other) { + if (other.isSetColumns()) { + List __this__columns = new ArrayList(); + for (TColumnDesc other_element : other.columns) { + __this__columns.add(new TColumnDesc(other_element)); + } + this.columns = __this__columns; + } + } + + public TTableSchema deepCopy() { + return new TTableSchema(this); + } + + @Override + public void clear() { + this.columns = null; + } + + public int getColumnsSize() { + return (this.columns == null) ? 0 : this.columns.size(); + } + + public java.util.Iterator getColumnsIterator() { + return (this.columns == null) ? null : this.columns.iterator(); + } + + public void addToColumns(TColumnDesc elem) { + if (this.columns == null) { + this.columns = new ArrayList(); + } + this.columns.add(elem); + } + + public List getColumns() { + return this.columns; + } + + public void setColumns(List columns) { + this.columns = columns; + } + + public void unsetColumns() { + this.columns = null; + } + + /** Returns true if field columns is set (has been assigned a value) and false otherwise */ + public boolean isSetColumns() { + return this.columns != null; + } + + public void setColumnsIsSet(boolean value) { + if (!value) { + this.columns = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case COLUMNS: + if (value == null) { + unsetColumns(); + } else { + setColumns((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case COLUMNS: + return getColumns(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case COLUMNS: + return isSetColumns(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TTableSchema) + return this.equals((TTableSchema)that); + return false; + } + + public boolean equals(TTableSchema that) { + if (that == null) + return false; + + boolean this_present_columns = true && this.isSetColumns(); + boolean that_present_columns = true && that.isSetColumns(); + if (this_present_columns || that_present_columns) { + if (!(this_present_columns && that_present_columns)) + return false; + if (!this.columns.equals(that.columns)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_columns = true && (isSetColumns()); + builder.append(present_columns); + if (present_columns) + builder.append(columns); + + return builder.toHashCode(); + } + + public int compareTo(TTableSchema other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TTableSchema typedOther = (TTableSchema)other; + + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(typedOther.isSetColumns()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetColumns()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TTableSchema("); + boolean first = true; + + sb.append("columns:"); + if (this.columns == null) { + sb.append("null"); + } else { + sb.append(this.columns); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetColumns()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'columns' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TTableSchemaStandardSchemeFactory implements SchemeFactory { + public TTableSchemaStandardScheme getScheme() { + return new TTableSchemaStandardScheme(); + } + } + + private static class TTableSchemaStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TTableSchema struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // COLUMNS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list38 = iprot.readListBegin(); + struct.columns = new ArrayList(_list38.size); + for (int _i39 = 0; _i39 < _list38.size; ++_i39) + { + TColumnDesc _elem40; // optional + _elem40 = new TColumnDesc(); + _elem40.read(iprot); + struct.columns.add(_elem40); + } + iprot.readListEnd(); + } + struct.setColumnsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TTableSchema struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.columns != null) { + oprot.writeFieldBegin(COLUMNS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size())); + for (TColumnDesc _iter41 : struct.columns) + { + _iter41.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TTableSchemaTupleSchemeFactory implements SchemeFactory { + public TTableSchemaTupleScheme getScheme() { + return new TTableSchemaTupleScheme(); + } + } + + private static class TTableSchemaTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TTableSchema struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.columns.size()); + for (TColumnDesc _iter42 : struct.columns) + { + _iter42.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TTableSchema struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list43 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.columns = new ArrayList(_list43.size); + for (int _i44 = 0; _i44 < _list43.size; ++_i44) + { + TColumnDesc _elem45; // optional + _elem45 = new TColumnDesc(); + _elem45.read(iprot); + struct.columns.add(_elem45); + } + } + struct.setColumnsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java new file mode 100644 index 000000000000..251f86a91471 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java @@ -0,0 +1,439 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TTypeDesc implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeDesc"); + + private static final org.apache.thrift.protocol.TField TYPES_FIELD_DESC = new org.apache.thrift.protocol.TField("types", org.apache.thrift.protocol.TType.LIST, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TTypeDescStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TTypeDescTupleSchemeFactory()); + } + + private List types; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TYPES((short)1, "types"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TYPES + return TYPES; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TYPES, new org.apache.thrift.meta_data.FieldMetaData("types", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeEntry.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeDesc.class, metaDataMap); + } + + public TTypeDesc() { + } + + public TTypeDesc( + List types) + { + this(); + this.types = types; + } + + /** + * Performs a deep copy on other. + */ + public TTypeDesc(TTypeDesc other) { + if (other.isSetTypes()) { + List __this__types = new ArrayList(); + for (TTypeEntry other_element : other.types) { + __this__types.add(new TTypeEntry(other_element)); + } + this.types = __this__types; + } + } + + public TTypeDesc deepCopy() { + return new TTypeDesc(this); + } + + @Override + public void clear() { + this.types = null; + } + + public int getTypesSize() { + return (this.types == null) ? 0 : this.types.size(); + } + + public java.util.Iterator getTypesIterator() { + return (this.types == null) ? null : this.types.iterator(); + } + + public void addToTypes(TTypeEntry elem) { + if (this.types == null) { + this.types = new ArrayList(); + } + this.types.add(elem); + } + + public List getTypes() { + return this.types; + } + + public void setTypes(List types) { + this.types = types; + } + + public void unsetTypes() { + this.types = null; + } + + /** Returns true if field types is set (has been assigned a value) and false otherwise */ + public boolean isSetTypes() { + return this.types != null; + } + + public void setTypesIsSet(boolean value) { + if (!value) { + this.types = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TYPES: + if (value == null) { + unsetTypes(); + } else { + setTypes((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TYPES: + return getTypes(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TYPES: + return isSetTypes(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TTypeDesc) + return this.equals((TTypeDesc)that); + return false; + } + + public boolean equals(TTypeDesc that) { + if (that == null) + return false; + + boolean this_present_types = true && this.isSetTypes(); + boolean that_present_types = true && that.isSetTypes(); + if (this_present_types || that_present_types) { + if (!(this_present_types && that_present_types)) + return false; + if (!this.types.equals(that.types)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_types = true && (isSetTypes()); + builder.append(present_types); + if (present_types) + builder.append(types); + + return builder.toHashCode(); + } + + public int compareTo(TTypeDesc other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TTypeDesc typedOther = (TTypeDesc)other; + + lastComparison = Boolean.valueOf(isSetTypes()).compareTo(typedOther.isSetTypes()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTypes()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.types, typedOther.types); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TTypeDesc("); + boolean first = true; + + sb.append("types:"); + if (this.types == null) { + sb.append("null"); + } else { + sb.append(this.types); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetTypes()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'types' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TTypeDescStandardSchemeFactory implements SchemeFactory { + public TTypeDescStandardScheme getScheme() { + return new TTypeDescStandardScheme(); + } + } + + private static class TTypeDescStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TTypeDesc struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TYPES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list30 = iprot.readListBegin(); + struct.types = new ArrayList(_list30.size); + for (int _i31 = 0; _i31 < _list30.size; ++_i31) + { + TTypeEntry _elem32; // optional + _elem32 = new TTypeEntry(); + _elem32.read(iprot); + struct.types.add(_elem32); + } + iprot.readListEnd(); + } + struct.setTypesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TTypeDesc struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.types != null) { + oprot.writeFieldBegin(TYPES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.types.size())); + for (TTypeEntry _iter33 : struct.types) + { + _iter33.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TTypeDescTupleSchemeFactory implements SchemeFactory { + public TTypeDescTupleScheme getScheme() { + return new TTypeDescTupleScheme(); + } + } + + private static class TTypeDescTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TTypeDesc struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.types.size()); + for (TTypeEntry _iter34 : struct.types) + { + _iter34.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TTypeDesc struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list35 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.types = new ArrayList(_list35.size); + for (int _i36 = 0; _i36 < _list35.size; ++_i36) + { + TTypeEntry _elem37; // optional + _elem37 = new TTypeEntry(); + _elem37.read(iprot); + struct.types.add(_elem37); + } + } + struct.setTypesIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java new file mode 100644 index 000000000000..af7c0b4f15d9 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java @@ -0,0 +1,610 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TTypeEntry extends org.apache.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeEntry"); + private static final org.apache.thrift.protocol.TField PRIMITIVE_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("primitiveEntry", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField ARRAY_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("arrayEntry", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField MAP_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("mapEntry", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField STRUCT_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("structEntry", org.apache.thrift.protocol.TType.STRUCT, (short)4); + private static final org.apache.thrift.protocol.TField UNION_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("unionEntry", org.apache.thrift.protocol.TType.STRUCT, (short)5); + private static final org.apache.thrift.protocol.TField USER_DEFINED_TYPE_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("userDefinedTypeEntry", org.apache.thrift.protocol.TType.STRUCT, (short)6); + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + PRIMITIVE_ENTRY((short)1, "primitiveEntry"), + ARRAY_ENTRY((short)2, "arrayEntry"), + MAP_ENTRY((short)3, "mapEntry"), + STRUCT_ENTRY((short)4, "structEntry"), + UNION_ENTRY((short)5, "unionEntry"), + USER_DEFINED_TYPE_ENTRY((short)6, "userDefinedTypeEntry"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // PRIMITIVE_ENTRY + return PRIMITIVE_ENTRY; + case 2: // ARRAY_ENTRY + return ARRAY_ENTRY; + case 3: // MAP_ENTRY + return MAP_ENTRY; + case 4: // STRUCT_ENTRY + return STRUCT_ENTRY; + case 5: // UNION_ENTRY + return UNION_ENTRY; + case 6: // USER_DEFINED_TYPE_ENTRY + return USER_DEFINED_TYPE_ENTRY; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.PRIMITIVE_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("primitiveEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TPrimitiveTypeEntry.class))); + tmpMap.put(_Fields.ARRAY_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("arrayEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TArrayTypeEntry.class))); + tmpMap.put(_Fields.MAP_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("mapEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TMapTypeEntry.class))); + tmpMap.put(_Fields.STRUCT_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("structEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStructTypeEntry.class))); + tmpMap.put(_Fields.UNION_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("unionEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TUnionTypeEntry.class))); + tmpMap.put(_Fields.USER_DEFINED_TYPE_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("userDefinedTypeEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TUserDefinedTypeEntry.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeEntry.class, metaDataMap); + } + + public TTypeEntry() { + super(); + } + + public TTypeEntry(_Fields setField, Object value) { + super(setField, value); + } + + public TTypeEntry(TTypeEntry other) { + super(other); + } + public TTypeEntry deepCopy() { + return new TTypeEntry(this); + } + + public static TTypeEntry primitiveEntry(TPrimitiveTypeEntry value) { + TTypeEntry x = new TTypeEntry(); + x.setPrimitiveEntry(value); + return x; + } + + public static TTypeEntry arrayEntry(TArrayTypeEntry value) { + TTypeEntry x = new TTypeEntry(); + x.setArrayEntry(value); + return x; + } + + public static TTypeEntry mapEntry(TMapTypeEntry value) { + TTypeEntry x = new TTypeEntry(); + x.setMapEntry(value); + return x; + } + + public static TTypeEntry structEntry(TStructTypeEntry value) { + TTypeEntry x = new TTypeEntry(); + x.setStructEntry(value); + return x; + } + + public static TTypeEntry unionEntry(TUnionTypeEntry value) { + TTypeEntry x = new TTypeEntry(); + x.setUnionEntry(value); + return x; + } + + public static TTypeEntry userDefinedTypeEntry(TUserDefinedTypeEntry value) { + TTypeEntry x = new TTypeEntry(); + x.setUserDefinedTypeEntry(value); + return x; + } + + + @Override + protected void checkType(_Fields setField, Object value) throws ClassCastException { + switch (setField) { + case PRIMITIVE_ENTRY: + if (value instanceof TPrimitiveTypeEntry) { + break; + } + throw new ClassCastException("Was expecting value of type TPrimitiveTypeEntry for field 'primitiveEntry', but got " + value.getClass().getSimpleName()); + case ARRAY_ENTRY: + if (value instanceof TArrayTypeEntry) { + break; + } + throw new ClassCastException("Was expecting value of type TArrayTypeEntry for field 'arrayEntry', but got " + value.getClass().getSimpleName()); + case MAP_ENTRY: + if (value instanceof TMapTypeEntry) { + break; + } + throw new ClassCastException("Was expecting value of type TMapTypeEntry for field 'mapEntry', but got " + value.getClass().getSimpleName()); + case STRUCT_ENTRY: + if (value instanceof TStructTypeEntry) { + break; + } + throw new ClassCastException("Was expecting value of type TStructTypeEntry for field 'structEntry', but got " + value.getClass().getSimpleName()); + case UNION_ENTRY: + if (value instanceof TUnionTypeEntry) { + break; + } + throw new ClassCastException("Was expecting value of type TUnionTypeEntry for field 'unionEntry', but got " + value.getClass().getSimpleName()); + case USER_DEFINED_TYPE_ENTRY: + if (value instanceof TUserDefinedTypeEntry) { + break; + } + throw new ClassCastException("Was expecting value of type TUserDefinedTypeEntry for field 'userDefinedTypeEntry', but got " + value.getClass().getSimpleName()); + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(field.id); + if (setField != null) { + switch (setField) { + case PRIMITIVE_ENTRY: + if (field.type == PRIMITIVE_ENTRY_FIELD_DESC.type) { + TPrimitiveTypeEntry primitiveEntry; + primitiveEntry = new TPrimitiveTypeEntry(); + primitiveEntry.read(iprot); + return primitiveEntry; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case ARRAY_ENTRY: + if (field.type == ARRAY_ENTRY_FIELD_DESC.type) { + TArrayTypeEntry arrayEntry; + arrayEntry = new TArrayTypeEntry(); + arrayEntry.read(iprot); + return arrayEntry; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case MAP_ENTRY: + if (field.type == MAP_ENTRY_FIELD_DESC.type) { + TMapTypeEntry mapEntry; + mapEntry = new TMapTypeEntry(); + mapEntry.read(iprot); + return mapEntry; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case STRUCT_ENTRY: + if (field.type == STRUCT_ENTRY_FIELD_DESC.type) { + TStructTypeEntry structEntry; + structEntry = new TStructTypeEntry(); + structEntry.read(iprot); + return structEntry; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case UNION_ENTRY: + if (field.type == UNION_ENTRY_FIELD_DESC.type) { + TUnionTypeEntry unionEntry; + unionEntry = new TUnionTypeEntry(); + unionEntry.read(iprot); + return unionEntry; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case USER_DEFINED_TYPE_ENTRY: + if (field.type == USER_DEFINED_TYPE_ENTRY_FIELD_DESC.type) { + TUserDefinedTypeEntry userDefinedTypeEntry; + userDefinedTypeEntry = new TUserDefinedTypeEntry(); + userDefinedTypeEntry.read(iprot); + return userDefinedTypeEntry; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case PRIMITIVE_ENTRY: + TPrimitiveTypeEntry primitiveEntry = (TPrimitiveTypeEntry)value_; + primitiveEntry.write(oprot); + return; + case ARRAY_ENTRY: + TArrayTypeEntry arrayEntry = (TArrayTypeEntry)value_; + arrayEntry.write(oprot); + return; + case MAP_ENTRY: + TMapTypeEntry mapEntry = (TMapTypeEntry)value_; + mapEntry.write(oprot); + return; + case STRUCT_ENTRY: + TStructTypeEntry structEntry = (TStructTypeEntry)value_; + structEntry.write(oprot); + return; + case UNION_ENTRY: + TUnionTypeEntry unionEntry = (TUnionTypeEntry)value_; + unionEntry.write(oprot); + return; + case USER_DEFINED_TYPE_ENTRY: + TUserDefinedTypeEntry userDefinedTypeEntry = (TUserDefinedTypeEntry)value_; + userDefinedTypeEntry.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(fieldID); + if (setField != null) { + switch (setField) { + case PRIMITIVE_ENTRY: + TPrimitiveTypeEntry primitiveEntry; + primitiveEntry = new TPrimitiveTypeEntry(); + primitiveEntry.read(iprot); + return primitiveEntry; + case ARRAY_ENTRY: + TArrayTypeEntry arrayEntry; + arrayEntry = new TArrayTypeEntry(); + arrayEntry.read(iprot); + return arrayEntry; + case MAP_ENTRY: + TMapTypeEntry mapEntry; + mapEntry = new TMapTypeEntry(); + mapEntry.read(iprot); + return mapEntry; + case STRUCT_ENTRY: + TStructTypeEntry structEntry; + structEntry = new TStructTypeEntry(); + structEntry.read(iprot); + return structEntry; + case UNION_ENTRY: + TUnionTypeEntry unionEntry; + unionEntry = new TUnionTypeEntry(); + unionEntry.read(iprot); + return unionEntry; + case USER_DEFINED_TYPE_ENTRY: + TUserDefinedTypeEntry userDefinedTypeEntry; + userDefinedTypeEntry = new TUserDefinedTypeEntry(); + userDefinedTypeEntry.read(iprot); + return userDefinedTypeEntry; + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + throw new TProtocolException("Couldn't find a field with field id " + fieldID); + } + } + + @Override + protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case PRIMITIVE_ENTRY: + TPrimitiveTypeEntry primitiveEntry = (TPrimitiveTypeEntry)value_; + primitiveEntry.write(oprot); + return; + case ARRAY_ENTRY: + TArrayTypeEntry arrayEntry = (TArrayTypeEntry)value_; + arrayEntry.write(oprot); + return; + case MAP_ENTRY: + TMapTypeEntry mapEntry = (TMapTypeEntry)value_; + mapEntry.write(oprot); + return; + case STRUCT_ENTRY: + TStructTypeEntry structEntry = (TStructTypeEntry)value_; + structEntry.write(oprot); + return; + case UNION_ENTRY: + TUnionTypeEntry unionEntry = (TUnionTypeEntry)value_; + unionEntry.write(oprot); + return; + case USER_DEFINED_TYPE_ENTRY: + TUserDefinedTypeEntry userDefinedTypeEntry = (TUserDefinedTypeEntry)value_; + userDefinedTypeEntry.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { + switch (setField) { + case PRIMITIVE_ENTRY: + return PRIMITIVE_ENTRY_FIELD_DESC; + case ARRAY_ENTRY: + return ARRAY_ENTRY_FIELD_DESC; + case MAP_ENTRY: + return MAP_ENTRY_FIELD_DESC; + case STRUCT_ENTRY: + return STRUCT_ENTRY_FIELD_DESC; + case UNION_ENTRY: + return UNION_ENTRY_FIELD_DESC; + case USER_DEFINED_TYPE_ENTRY: + return USER_DEFINED_TYPE_ENTRY_FIELD_DESC; + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected org.apache.thrift.protocol.TStruct getStructDesc() { + return STRUCT_DESC; + } + + @Override + protected _Fields enumForId(short id) { + return _Fields.findByThriftIdOrThrow(id); + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + + public TPrimitiveTypeEntry getPrimitiveEntry() { + if (getSetField() == _Fields.PRIMITIVE_ENTRY) { + return (TPrimitiveTypeEntry)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'primitiveEntry' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setPrimitiveEntry(TPrimitiveTypeEntry value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.PRIMITIVE_ENTRY; + value_ = value; + } + + public TArrayTypeEntry getArrayEntry() { + if (getSetField() == _Fields.ARRAY_ENTRY) { + return (TArrayTypeEntry)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'arrayEntry' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setArrayEntry(TArrayTypeEntry value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.ARRAY_ENTRY; + value_ = value; + } + + public TMapTypeEntry getMapEntry() { + if (getSetField() == _Fields.MAP_ENTRY) { + return (TMapTypeEntry)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'mapEntry' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setMapEntry(TMapTypeEntry value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.MAP_ENTRY; + value_ = value; + } + + public TStructTypeEntry getStructEntry() { + if (getSetField() == _Fields.STRUCT_ENTRY) { + return (TStructTypeEntry)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'structEntry' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setStructEntry(TStructTypeEntry value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.STRUCT_ENTRY; + value_ = value; + } + + public TUnionTypeEntry getUnionEntry() { + if (getSetField() == _Fields.UNION_ENTRY) { + return (TUnionTypeEntry)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'unionEntry' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setUnionEntry(TUnionTypeEntry value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.UNION_ENTRY; + value_ = value; + } + + public TUserDefinedTypeEntry getUserDefinedTypeEntry() { + if (getSetField() == _Fields.USER_DEFINED_TYPE_ENTRY) { + return (TUserDefinedTypeEntry)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'userDefinedTypeEntry' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setUserDefinedTypeEntry(TUserDefinedTypeEntry value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.USER_DEFINED_TYPE_ENTRY; + value_ = value; + } + + public boolean isSetPrimitiveEntry() { + return setField_ == _Fields.PRIMITIVE_ENTRY; + } + + + public boolean isSetArrayEntry() { + return setField_ == _Fields.ARRAY_ENTRY; + } + + + public boolean isSetMapEntry() { + return setField_ == _Fields.MAP_ENTRY; + } + + + public boolean isSetStructEntry() { + return setField_ == _Fields.STRUCT_ENTRY; + } + + + public boolean isSetUnionEntry() { + return setField_ == _Fields.UNION_ENTRY; + } + + + public boolean isSetUserDefinedTypeEntry() { + return setField_ == _Fields.USER_DEFINED_TYPE_ENTRY; + } + + + public boolean equals(Object other) { + if (other instanceof TTypeEntry) { + return equals((TTypeEntry)other); + } else { + return false; + } + } + + public boolean equals(TTypeEntry other) { + return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); + } + + @Override + public int compareTo(TTypeEntry other) { + int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + HashCodeBuilder hcb = new HashCodeBuilder(); + hcb.append(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + hcb.append(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); + } else { + hcb.append(value); + } + } + return hcb.toHashCode(); + } + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + +} diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java new file mode 100644 index 000000000000..40f05894623c --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java @@ -0,0 +1,105 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TTypeId implements org.apache.thrift.TEnum { + BOOLEAN_TYPE(0), + TINYINT_TYPE(1), + SMALLINT_TYPE(2), + INT_TYPE(3), + BIGINT_TYPE(4), + FLOAT_TYPE(5), + DOUBLE_TYPE(6), + STRING_TYPE(7), + TIMESTAMP_TYPE(8), + BINARY_TYPE(9), + ARRAY_TYPE(10), + MAP_TYPE(11), + STRUCT_TYPE(12), + UNION_TYPE(13), + USER_DEFINED_TYPE(14), + DECIMAL_TYPE(15), + NULL_TYPE(16), + DATE_TYPE(17), + VARCHAR_TYPE(18), + CHAR_TYPE(19), + INTERVAL_YEAR_MONTH_TYPE(20), + INTERVAL_DAY_TIME_TYPE(21); + + private final int value; + + private TTypeId(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TTypeId findByValue(int value) { + switch (value) { + case 0: + return BOOLEAN_TYPE; + case 1: + return TINYINT_TYPE; + case 2: + return SMALLINT_TYPE; + case 3: + return INT_TYPE; + case 4: + return BIGINT_TYPE; + case 5: + return FLOAT_TYPE; + case 6: + return DOUBLE_TYPE; + case 7: + return STRING_TYPE; + case 8: + return TIMESTAMP_TYPE; + case 9: + return BINARY_TYPE; + case 10: + return ARRAY_TYPE; + case 11: + return MAP_TYPE; + case 12: + return STRUCT_TYPE; + case 13: + return UNION_TYPE; + case 14: + return USER_DEFINED_TYPE; + case 15: + return DECIMAL_TYPE; + case 16: + return NULL_TYPE; + case 17: + return DATE_TYPE; + case 18: + return VARCHAR_TYPE; + case 19: + return CHAR_TYPE; + case 20: + return INTERVAL_YEAR_MONTH_TYPE; + case 21: + return INTERVAL_DAY_TIME_TYPE; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java new file mode 100644 index 000000000000..8c40687a0aab --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java @@ -0,0 +1,361 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TTypeQualifierValue extends org.apache.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeQualifierValue"); + private static final org.apache.thrift.protocol.TField I32_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("i32Value", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField STRING_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("stringValue", org.apache.thrift.protocol.TType.STRING, (short)2); + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + I32_VALUE((short)1, "i32Value"), + STRING_VALUE((short)2, "stringValue"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // I32_VALUE + return I32_VALUE; + case 2: // STRING_VALUE + return STRING_VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.I32_VALUE, new org.apache.thrift.meta_data.FieldMetaData("i32Value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.STRING_VALUE, new org.apache.thrift.meta_data.FieldMetaData("stringValue", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeQualifierValue.class, metaDataMap); + } + + public TTypeQualifierValue() { + super(); + } + + public TTypeQualifierValue(_Fields setField, Object value) { + super(setField, value); + } + + public TTypeQualifierValue(TTypeQualifierValue other) { + super(other); + } + public TTypeQualifierValue deepCopy() { + return new TTypeQualifierValue(this); + } + + public static TTypeQualifierValue i32Value(int value) { + TTypeQualifierValue x = new TTypeQualifierValue(); + x.setI32Value(value); + return x; + } + + public static TTypeQualifierValue stringValue(String value) { + TTypeQualifierValue x = new TTypeQualifierValue(); + x.setStringValue(value); + return x; + } + + + @Override + protected void checkType(_Fields setField, Object value) throws ClassCastException { + switch (setField) { + case I32_VALUE: + if (value instanceof Integer) { + break; + } + throw new ClassCastException("Was expecting value of type Integer for field 'i32Value', but got " + value.getClass().getSimpleName()); + case STRING_VALUE: + if (value instanceof String) { + break; + } + throw new ClassCastException("Was expecting value of type String for field 'stringValue', but got " + value.getClass().getSimpleName()); + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(field.id); + if (setField != null) { + switch (setField) { + case I32_VALUE: + if (field.type == I32_VALUE_FIELD_DESC.type) { + Integer i32Value; + i32Value = iprot.readI32(); + return i32Value; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case STRING_VALUE: + if (field.type == STRING_VALUE_FIELD_DESC.type) { + String stringValue; + stringValue = iprot.readString(); + return stringValue; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case I32_VALUE: + Integer i32Value = (Integer)value_; + oprot.writeI32(i32Value); + return; + case STRING_VALUE: + String stringValue = (String)value_; + oprot.writeString(stringValue); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(fieldID); + if (setField != null) { + switch (setField) { + case I32_VALUE: + Integer i32Value; + i32Value = iprot.readI32(); + return i32Value; + case STRING_VALUE: + String stringValue; + stringValue = iprot.readString(); + return stringValue; + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + throw new TProtocolException("Couldn't find a field with field id " + fieldID); + } + } + + @Override + protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case I32_VALUE: + Integer i32Value = (Integer)value_; + oprot.writeI32(i32Value); + return; + case STRING_VALUE: + String stringValue = (String)value_; + oprot.writeString(stringValue); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { + switch (setField) { + case I32_VALUE: + return I32_VALUE_FIELD_DESC; + case STRING_VALUE: + return STRING_VALUE_FIELD_DESC; + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected org.apache.thrift.protocol.TStruct getStructDesc() { + return STRUCT_DESC; + } + + @Override + protected _Fields enumForId(short id) { + return _Fields.findByThriftIdOrThrow(id); + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + + public int getI32Value() { + if (getSetField() == _Fields.I32_VALUE) { + return (Integer)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'i32Value' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setI32Value(int value) { + setField_ = _Fields.I32_VALUE; + value_ = value; + } + + public String getStringValue() { + if (getSetField() == _Fields.STRING_VALUE) { + return (String)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'stringValue' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setStringValue(String value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.STRING_VALUE; + value_ = value; + } + + public boolean isSetI32Value() { + return setField_ == _Fields.I32_VALUE; + } + + + public boolean isSetStringValue() { + return setField_ == _Fields.STRING_VALUE; + } + + + public boolean equals(Object other) { + if (other instanceof TTypeQualifierValue) { + return equals((TTypeQualifierValue)other); + } else { + return false; + } + } + + public boolean equals(TTypeQualifierValue other) { + return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); + } + + @Override + public int compareTo(TTypeQualifierValue other) { + int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + HashCodeBuilder hcb = new HashCodeBuilder(); + hcb.append(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + hcb.append(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + hcb.append(((org.apache.thrift.TEnum)getFieldValue()).getValue()); + } else { + hcb.append(value); + } + } + return hcb.toHashCode(); + } + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + +} diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java new file mode 100644 index 000000000000..39355551d372 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java @@ -0,0 +1,450 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TTypeQualifiers implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeQualifiers"); + + private static final org.apache.thrift.protocol.TField QUALIFIERS_FIELD_DESC = new org.apache.thrift.protocol.TField("qualifiers", org.apache.thrift.protocol.TType.MAP, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TTypeQualifiersStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TTypeQualifiersTupleSchemeFactory()); + } + + private Map qualifiers; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + QUALIFIERS((short)1, "qualifiers"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // QUALIFIERS + return QUALIFIERS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.QUALIFIERS, new org.apache.thrift.meta_data.FieldMetaData("qualifiers", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeQualifierValue.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeQualifiers.class, metaDataMap); + } + + public TTypeQualifiers() { + } + + public TTypeQualifiers( + Map qualifiers) + { + this(); + this.qualifiers = qualifiers; + } + + /** + * Performs a deep copy on other. + */ + public TTypeQualifiers(TTypeQualifiers other) { + if (other.isSetQualifiers()) { + Map __this__qualifiers = new HashMap(); + for (Map.Entry other_element : other.qualifiers.entrySet()) { + + String other_element_key = other_element.getKey(); + TTypeQualifierValue other_element_value = other_element.getValue(); + + String __this__qualifiers_copy_key = other_element_key; + + TTypeQualifierValue __this__qualifiers_copy_value = new TTypeQualifierValue(other_element_value); + + __this__qualifiers.put(__this__qualifiers_copy_key, __this__qualifiers_copy_value); + } + this.qualifiers = __this__qualifiers; + } + } + + public TTypeQualifiers deepCopy() { + return new TTypeQualifiers(this); + } + + @Override + public void clear() { + this.qualifiers = null; + } + + public int getQualifiersSize() { + return (this.qualifiers == null) ? 0 : this.qualifiers.size(); + } + + public void putToQualifiers(String key, TTypeQualifierValue val) { + if (this.qualifiers == null) { + this.qualifiers = new HashMap(); + } + this.qualifiers.put(key, val); + } + + public Map getQualifiers() { + return this.qualifiers; + } + + public void setQualifiers(Map qualifiers) { + this.qualifiers = qualifiers; + } + + public void unsetQualifiers() { + this.qualifiers = null; + } + + /** Returns true if field qualifiers is set (has been assigned a value) and false otherwise */ + public boolean isSetQualifiers() { + return this.qualifiers != null; + } + + public void setQualifiersIsSet(boolean value) { + if (!value) { + this.qualifiers = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case QUALIFIERS: + if (value == null) { + unsetQualifiers(); + } else { + setQualifiers((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case QUALIFIERS: + return getQualifiers(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case QUALIFIERS: + return isSetQualifiers(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TTypeQualifiers) + return this.equals((TTypeQualifiers)that); + return false; + } + + public boolean equals(TTypeQualifiers that) { + if (that == null) + return false; + + boolean this_present_qualifiers = true && this.isSetQualifiers(); + boolean that_present_qualifiers = true && that.isSetQualifiers(); + if (this_present_qualifiers || that_present_qualifiers) { + if (!(this_present_qualifiers && that_present_qualifiers)) + return false; + if (!this.qualifiers.equals(that.qualifiers)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_qualifiers = true && (isSetQualifiers()); + builder.append(present_qualifiers); + if (present_qualifiers) + builder.append(qualifiers); + + return builder.toHashCode(); + } + + public int compareTo(TTypeQualifiers other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TTypeQualifiers typedOther = (TTypeQualifiers)other; + + lastComparison = Boolean.valueOf(isSetQualifiers()).compareTo(typedOther.isSetQualifiers()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetQualifiers()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.qualifiers, typedOther.qualifiers); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TTypeQualifiers("); + boolean first = true; + + sb.append("qualifiers:"); + if (this.qualifiers == null) { + sb.append("null"); + } else { + sb.append(this.qualifiers); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetQualifiers()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'qualifiers' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TTypeQualifiersStandardSchemeFactory implements SchemeFactory { + public TTypeQualifiersStandardScheme getScheme() { + return new TTypeQualifiersStandardScheme(); + } + } + + private static class TTypeQualifiersStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TTypeQualifiers struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // QUALIFIERS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map0 = iprot.readMapBegin(); + struct.qualifiers = new HashMap(2*_map0.size); + for (int _i1 = 0; _i1 < _map0.size; ++_i1) + { + String _key2; // required + TTypeQualifierValue _val3; // required + _key2 = iprot.readString(); + _val3 = new TTypeQualifierValue(); + _val3.read(iprot); + struct.qualifiers.put(_key2, _val3); + } + iprot.readMapEnd(); + } + struct.setQualifiersIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TTypeQualifiers struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.qualifiers != null) { + oprot.writeFieldBegin(QUALIFIERS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.qualifiers.size())); + for (Map.Entry _iter4 : struct.qualifiers.entrySet()) + { + oprot.writeString(_iter4.getKey()); + _iter4.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TTypeQualifiersTupleSchemeFactory implements SchemeFactory { + public TTypeQualifiersTupleScheme getScheme() { + return new TTypeQualifiersTupleScheme(); + } + } + + private static class TTypeQualifiersTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TTypeQualifiers struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.qualifiers.size()); + for (Map.Entry _iter5 : struct.qualifiers.entrySet()) + { + oprot.writeString(_iter5.getKey()); + _iter5.getValue().write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TTypeQualifiers struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map6 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.qualifiers = new HashMap(2*_map6.size); + for (int _i7 = 0; _i7 < _map6.size; ++_i7) + { + String _key8; // required + TTypeQualifierValue _val9; // required + _key8 = iprot.readString(); + _val9 = new TTypeQualifierValue(); + _val9.read(iprot); + struct.qualifiers.put(_key8, _val9); + } + } + struct.setQualifiersIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java new file mode 100644 index 000000000000..73dd45d3dd01 --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java @@ -0,0 +1,448 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TUnionTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TUnionTypeEntry"); + + private static final org.apache.thrift.protocol.TField NAME_TO_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("nameToTypePtr", org.apache.thrift.protocol.TType.MAP, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TUnionTypeEntryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TUnionTypeEntryTupleSchemeFactory()); + } + + private Map nameToTypePtr; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME_TO_TYPE_PTR((short)1, "nameToTypePtr"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME_TO_TYPE_PTR + return NAME_TO_TYPE_PTR; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME_TO_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("nameToTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr")))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TUnionTypeEntry.class, metaDataMap); + } + + public TUnionTypeEntry() { + } + + public TUnionTypeEntry( + Map nameToTypePtr) + { + this(); + this.nameToTypePtr = nameToTypePtr; + } + + /** + * Performs a deep copy on other. + */ + public TUnionTypeEntry(TUnionTypeEntry other) { + if (other.isSetNameToTypePtr()) { + Map __this__nameToTypePtr = new HashMap(); + for (Map.Entry other_element : other.nameToTypePtr.entrySet()) { + + String other_element_key = other_element.getKey(); + Integer other_element_value = other_element.getValue(); + + String __this__nameToTypePtr_copy_key = other_element_key; + + Integer __this__nameToTypePtr_copy_value = other_element_value; + + __this__nameToTypePtr.put(__this__nameToTypePtr_copy_key, __this__nameToTypePtr_copy_value); + } + this.nameToTypePtr = __this__nameToTypePtr; + } + } + + public TUnionTypeEntry deepCopy() { + return new TUnionTypeEntry(this); + } + + @Override + public void clear() { + this.nameToTypePtr = null; + } + + public int getNameToTypePtrSize() { + return (this.nameToTypePtr == null) ? 0 : this.nameToTypePtr.size(); + } + + public void putToNameToTypePtr(String key, int val) { + if (this.nameToTypePtr == null) { + this.nameToTypePtr = new HashMap(); + } + this.nameToTypePtr.put(key, val); + } + + public Map getNameToTypePtr() { + return this.nameToTypePtr; + } + + public void setNameToTypePtr(Map nameToTypePtr) { + this.nameToTypePtr = nameToTypePtr; + } + + public void unsetNameToTypePtr() { + this.nameToTypePtr = null; + } + + /** Returns true if field nameToTypePtr is set (has been assigned a value) and false otherwise */ + public boolean isSetNameToTypePtr() { + return this.nameToTypePtr != null; + } + + public void setNameToTypePtrIsSet(boolean value) { + if (!value) { + this.nameToTypePtr = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME_TO_TYPE_PTR: + if (value == null) { + unsetNameToTypePtr(); + } else { + setNameToTypePtr((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME_TO_TYPE_PTR: + return getNameToTypePtr(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME_TO_TYPE_PTR: + return isSetNameToTypePtr(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TUnionTypeEntry) + return this.equals((TUnionTypeEntry)that); + return false; + } + + public boolean equals(TUnionTypeEntry that) { + if (that == null) + return false; + + boolean this_present_nameToTypePtr = true && this.isSetNameToTypePtr(); + boolean that_present_nameToTypePtr = true && that.isSetNameToTypePtr(); + if (this_present_nameToTypePtr || that_present_nameToTypePtr) { + if (!(this_present_nameToTypePtr && that_present_nameToTypePtr)) + return false; + if (!this.nameToTypePtr.equals(that.nameToTypePtr)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_nameToTypePtr = true && (isSetNameToTypePtr()); + builder.append(present_nameToTypePtr); + if (present_nameToTypePtr) + builder.append(nameToTypePtr); + + return builder.toHashCode(); + } + + public int compareTo(TUnionTypeEntry other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TUnionTypeEntry typedOther = (TUnionTypeEntry)other; + + lastComparison = Boolean.valueOf(isSetNameToTypePtr()).compareTo(typedOther.isSetNameToTypePtr()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNameToTypePtr()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nameToTypePtr, typedOther.nameToTypePtr); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TUnionTypeEntry("); + boolean first = true; + + sb.append("nameToTypePtr:"); + if (this.nameToTypePtr == null) { + sb.append("null"); + } else { + sb.append(this.nameToTypePtr); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetNameToTypePtr()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nameToTypePtr' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TUnionTypeEntryStandardSchemeFactory implements SchemeFactory { + public TUnionTypeEntryStandardScheme getScheme() { + return new TUnionTypeEntryStandardScheme(); + } + } + + private static class TUnionTypeEntryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TUnionTypeEntry struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME_TO_TYPE_PTR + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map20 = iprot.readMapBegin(); + struct.nameToTypePtr = new HashMap(2*_map20.size); + for (int _i21 = 0; _i21 < _map20.size; ++_i21) + { + String _key22; // required + int _val23; // required + _key22 = iprot.readString(); + _val23 = iprot.readI32(); + struct.nameToTypePtr.put(_key22, _val23); + } + iprot.readMapEnd(); + } + struct.setNameToTypePtrIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TUnionTypeEntry struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.nameToTypePtr != null) { + oprot.writeFieldBegin(NAME_TO_TYPE_PTR_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.nameToTypePtr.size())); + for (Map.Entry _iter24 : struct.nameToTypePtr.entrySet()) + { + oprot.writeString(_iter24.getKey()); + oprot.writeI32(_iter24.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TUnionTypeEntryTupleSchemeFactory implements SchemeFactory { + public TUnionTypeEntryTupleScheme getScheme() { + return new TUnionTypeEntryTupleScheme(); + } + } + + private static class TUnionTypeEntryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TUnionTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.nameToTypePtr.size()); + for (Map.Entry _iter25 : struct.nameToTypePtr.entrySet()) + { + oprot.writeString(_iter25.getKey()); + oprot.writeI32(_iter25.getValue()); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TUnionTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map26 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.nameToTypePtr = new HashMap(2*_map26.size); + for (int _i27 = 0; _i27 < _map26.size; ++_i27) + { + String _key28; // required + int _val29; // required + _key28 = iprot.readString(); + _val29 = iprot.readI32(); + struct.nameToTypePtr.put(_key28, _val29); + } + } + struct.setNameToTypePtrIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java new file mode 100644 index 000000000000..3a111a2c8c2c --- /dev/null +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java @@ -0,0 +1,385 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TUserDefinedTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TUserDefinedTypeEntry"); + + private static final org.apache.thrift.protocol.TField TYPE_CLASS_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("typeClassName", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TUserDefinedTypeEntryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TUserDefinedTypeEntryTupleSchemeFactory()); + } + + private String typeClassName; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TYPE_CLASS_NAME((short)1, "typeClassName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TYPE_CLASS_NAME + return TYPE_CLASS_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TYPE_CLASS_NAME, new org.apache.thrift.meta_data.FieldMetaData("typeClassName", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TUserDefinedTypeEntry.class, metaDataMap); + } + + public TUserDefinedTypeEntry() { + } + + public TUserDefinedTypeEntry( + String typeClassName) + { + this(); + this.typeClassName = typeClassName; + } + + /** + * Performs a deep copy on other. + */ + public TUserDefinedTypeEntry(TUserDefinedTypeEntry other) { + if (other.isSetTypeClassName()) { + this.typeClassName = other.typeClassName; + } + } + + public TUserDefinedTypeEntry deepCopy() { + return new TUserDefinedTypeEntry(this); + } + + @Override + public void clear() { + this.typeClassName = null; + } + + public String getTypeClassName() { + return this.typeClassName; + } + + public void setTypeClassName(String typeClassName) { + this.typeClassName = typeClassName; + } + + public void unsetTypeClassName() { + this.typeClassName = null; + } + + /** Returns true if field typeClassName is set (has been assigned a value) and false otherwise */ + public boolean isSetTypeClassName() { + return this.typeClassName != null; + } + + public void setTypeClassNameIsSet(boolean value) { + if (!value) { + this.typeClassName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TYPE_CLASS_NAME: + if (value == null) { + unsetTypeClassName(); + } else { + setTypeClassName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TYPE_CLASS_NAME: + return getTypeClassName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TYPE_CLASS_NAME: + return isSetTypeClassName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TUserDefinedTypeEntry) + return this.equals((TUserDefinedTypeEntry)that); + return false; + } + + public boolean equals(TUserDefinedTypeEntry that) { + if (that == null) + return false; + + boolean this_present_typeClassName = true && this.isSetTypeClassName(); + boolean that_present_typeClassName = true && that.isSetTypeClassName(); + if (this_present_typeClassName || that_present_typeClassName) { + if (!(this_present_typeClassName && that_present_typeClassName)) + return false; + if (!this.typeClassName.equals(that.typeClassName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_typeClassName = true && (isSetTypeClassName()); + builder.append(present_typeClassName); + if (present_typeClassName) + builder.append(typeClassName); + + return builder.toHashCode(); + } + + public int compareTo(TUserDefinedTypeEntry other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TUserDefinedTypeEntry typedOther = (TUserDefinedTypeEntry)other; + + lastComparison = Boolean.valueOf(isSetTypeClassName()).compareTo(typedOther.isSetTypeClassName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTypeClassName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.typeClassName, typedOther.typeClassName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TUserDefinedTypeEntry("); + boolean first = true; + + sb.append("typeClassName:"); + if (this.typeClassName == null) { + sb.append("null"); + } else { + sb.append(this.typeClassName); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetTypeClassName()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'typeClassName' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TUserDefinedTypeEntryStandardSchemeFactory implements SchemeFactory { + public TUserDefinedTypeEntryStandardScheme getScheme() { + return new TUserDefinedTypeEntryStandardScheme(); + } + } + + private static class TUserDefinedTypeEntryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TYPE_CLASS_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.typeClassName = iprot.readString(); + struct.setTypeClassNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.typeClassName != null) { + oprot.writeFieldBegin(TYPE_CLASS_NAME_FIELD_DESC); + oprot.writeString(struct.typeClassName); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TUserDefinedTypeEntryTupleSchemeFactory implements SchemeFactory { + public TUserDefinedTypeEntryTupleScheme getScheme() { + return new TUserDefinedTypeEntryTupleScheme(); + } + } + + private static class TUserDefinedTypeEntryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.typeClassName); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.typeClassName = iprot.readString(); + struct.setTypeClassNameIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java new file mode 100644 index 000000000000..9dd0efc03968 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java @@ -0,0 +1,184 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; + +/** + * AbstractService. + * + */ +public abstract class AbstractService implements Service { + + private static final Log LOG = LogFactory.getLog(AbstractService.class); + + /** + * Service state: initially {@link STATE#NOTINITED}. + */ + private STATE state = STATE.NOTINITED; + + /** + * Service name. + */ + private final String name; + /** + * Service start time. Will be zero until the service is started. + */ + private long startTime; + + /** + * The configuration. Will be null until the service is initialized. + */ + private HiveConf hiveConf; + + /** + * List of state change listeners; it is final to ensure + * that it will never be null. + */ + private final List listeners = + new ArrayList(); + + /** + * Construct the service. + * + * @param name + * service name + */ + public AbstractService(String name) { + this.name = name; + } + + @Override + public synchronized STATE getServiceState() { + return state; + } + + /** + * {@inheritDoc} + * + * @throws IllegalStateException + * if the current service state does not permit + * this action + */ + @Override + public synchronized void init(HiveConf hiveConf) { + ensureCurrentState(STATE.NOTINITED); + this.hiveConf = hiveConf; + changeState(STATE.INITED); + LOG.info("Service:" + getName() + " is inited."); + } + + /** + * {@inheritDoc} + * + * @throws IllegalStateException + * if the current service state does not permit + * this action + */ + @Override + public synchronized void start() { + startTime = System.currentTimeMillis(); + ensureCurrentState(STATE.INITED); + changeState(STATE.STARTED); + LOG.info("Service:" + getName() + " is started."); + } + + /** + * {@inheritDoc} + * + * @throws IllegalStateException + * if the current service state does not permit + * this action + */ + @Override + public synchronized void stop() { + if (state == STATE.STOPPED || + state == STATE.INITED || + state == STATE.NOTINITED) { + // already stopped, or else it was never + // started (eg another service failing canceled startup) + return; + } + ensureCurrentState(STATE.STARTED); + changeState(STATE.STOPPED); + LOG.info("Service:" + getName() + " is stopped."); + } + + @Override + public synchronized void register(ServiceStateChangeListener l) { + listeners.add(l); + } + + @Override + public synchronized void unregister(ServiceStateChangeListener l) { + listeners.remove(l); + } + + @Override + public String getName() { + return name; + } + + @Override + public synchronized HiveConf getHiveConf() { + return hiveConf; + } + + @Override + public long getStartTime() { + return startTime; + } + + /** + * Verify that a service is in a given state. + * + * @param currentState + * the desired state + * @throws IllegalStateException + * if the service state is different from + * the desired state + */ + private void ensureCurrentState(STATE currentState) { + ServiceOperations.ensureCurrentState(state, currentState); + } + + /** + * Change to a new state and notify all listeners. + * This is a private method that is only invoked from synchronized methods, + * which avoid having to clone the listener list. It does imply that + * the state change listener methods should be short lived, as they + * will delay the state transition. + * + * @param newState + * new service state + */ + private void changeState(STATE newState) { + state = newState; + // notify listeners + for (ServiceStateChangeListener l : listeners) { + l.stateChanged(this); + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/BreakableService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/BreakableService.java new file mode 100644 index 000000000000..9c44beb2fb42 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/BreakableService.java @@ -0,0 +1,121 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hive.service.Service.STATE; + +/** + * This is a service that can be configured to break on any of the lifecycle + * events, so test the failure handling of other parts of the service + * infrastructure. + * + * It retains a counter to the number of times each entry point is called - + * these counters are incremented before the exceptions are raised and + * before the superclass state methods are invoked. + * + */ +public class BreakableService extends AbstractService { + private boolean failOnInit; + private boolean failOnStart; + private boolean failOnStop; + private final int[] counts = new int[4]; + + public BreakableService() { + this(false, false, false); + } + + public BreakableService(boolean failOnInit, + boolean failOnStart, + boolean failOnStop) { + super("BreakableService"); + this.failOnInit = failOnInit; + this.failOnStart = failOnStart; + this.failOnStop = failOnStop; + inc(STATE.NOTINITED); + } + + private int convert(STATE state) { + switch (state) { + case NOTINITED: return 0; + case INITED: return 1; + case STARTED: return 2; + case STOPPED: return 3; + default: return 0; + } + } + + private void inc(STATE state) { + int index = convert(state); + counts[index] ++; + } + + public int getCount(STATE state) { + return counts[convert(state)]; + } + + private void maybeFail(boolean fail, String action) { + if (fail) { + throw new BrokenLifecycleEvent(action); + } + } + + @Override + public void init(HiveConf conf) { + inc(STATE.INITED); + maybeFail(failOnInit, "init"); + super.init(conf); + } + + @Override + public void start() { + inc(STATE.STARTED); + maybeFail(failOnStart, "start"); + super.start(); + } + + @Override + public void stop() { + inc(STATE.STOPPED); + maybeFail(failOnStop, "stop"); + super.stop(); + } + + public void setFailOnInit(boolean failOnInit) { + this.failOnInit = failOnInit; + } + + public void setFailOnStart(boolean failOnStart) { + this.failOnStart = failOnStart; + } + + public void setFailOnStop(boolean failOnStop) { + this.failOnStop = failOnStop; + } + + /** + * The exception explicitly raised on a failure + */ + public static class BrokenLifecycleEvent extends RuntimeException { + BrokenLifecycleEvent(String action) { + super("Lifecycle Failure during " + action); + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java new file mode 100644 index 000000000000..897911872b80 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java @@ -0,0 +1,133 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; + +/** + * CompositeService. + * + */ +public class CompositeService extends AbstractService { + + private static final Log LOG = LogFactory.getLog(CompositeService.class); + + private final List serviceList = new ArrayList(); + + public CompositeService(String name) { + super(name); + } + + public Collection getServices() { + return Collections.unmodifiableList(serviceList); + } + + protected synchronized void addService(Service service) { + serviceList.add(service); + } + + protected synchronized boolean removeService(Service service) { + return serviceList.remove(service); + } + + @Override + public synchronized void init(HiveConf hiveConf) { + for (Service service : serviceList) { + service.init(hiveConf); + } + super.init(hiveConf); + } + + @Override + public synchronized void start() { + int i = 0; + try { + for (int n = serviceList.size(); i < n; i++) { + Service service = serviceList.get(i); + service.start(); + } + super.start(); + } catch (Throwable e) { + LOG.error("Error starting services " + getName(), e); + // Note that the state of the failed service is still INITED and not + // STARTED. Even though the last service is not started completely, still + // call stop() on all services including failed service to make sure cleanup + // happens. + stop(i); + throw new ServiceException("Failed to Start " + getName(), e); + } + + } + + @Override + public synchronized void stop() { + if (this.getServiceState() == STATE.STOPPED) { + // The base composite-service is already stopped, don't do anything again. + return; + } + if (serviceList.size() > 0) { + stop(serviceList.size() - 1); + } + super.stop(); + } + + private synchronized void stop(int numOfServicesStarted) { + // stop in reserve order of start + for (int i = numOfServicesStarted; i >= 0; i--) { + Service service = serviceList.get(i); + try { + service.stop(); + } catch (Throwable t) { + LOG.info("Error stopping " + service.getName(), t); + } + } + } + + /** + * JVM Shutdown hook for CompositeService which will stop the given + * CompositeService gracefully in case of JVM shutdown. + */ + public static class CompositeServiceShutdownHook implements Runnable { + + private final CompositeService compositeService; + + public CompositeServiceShutdownHook(CompositeService compositeService) { + this.compositeService = compositeService; + } + + @Override + public void run() { + try { + // Stop the Composite Service + compositeService.stop(); + } catch (Throwable t) { + LOG.info("Error stopping " + compositeService.getName(), t); + } + } + } + + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java new file mode 100644 index 000000000000..ee51c24351c3 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.logging.LogFactory; +import org.apache.commons.logging.Log; + +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; + +/** + * The cookie signer generates a signature based on SHA digest + * and appends it to the cookie value generated at the + * server side. It uses SHA digest algorithm to sign and verify signatures. + */ +public class CookieSigner { + private static final String SIGNATURE = "&s="; + private static final String SHA_STRING = "SHA"; + private byte[] secretBytes; + private static final Log LOG = LogFactory.getLog(CookieSigner.class); + + /** + * Constructor + * @param secret Secret Bytes + */ + public CookieSigner(byte[] secret) { + if (secret == null) { + throw new IllegalArgumentException(" NULL Secret Bytes"); + } + this.secretBytes = secret.clone(); + } + + /** + * Sign the cookie given the string token as input. + * @param str Input token + * @return Signed token that can be used to create a cookie + */ + public String signCookie(String str) { + if (str == null || str.isEmpty()) { + throw new IllegalArgumentException("NULL or empty string to sign"); + } + String signature = getSignature(str); + + if (LOG.isDebugEnabled()) { + LOG.debug("Signature generated for " + str + " is " + signature); + } + return str + SIGNATURE + signature; + } + + /** + * Verify a signed string and extracts the original string. + * @param signedStr The already signed string + * @return Raw Value of the string without the signature + */ + public String verifyAndExtract(String signedStr) { + int index = signedStr.lastIndexOf(SIGNATURE); + if (index == -1) { + throw new IllegalArgumentException("Invalid input sign: " + signedStr); + } + String originalSignature = signedStr.substring(index + SIGNATURE.length()); + String rawValue = signedStr.substring(0, index); + String currentSignature = getSignature(rawValue); + + if (LOG.isDebugEnabled()) { + LOG.debug("Signature generated for " + rawValue + " inside verify is " + currentSignature); + } + if (!originalSignature.equals(currentSignature)) { + throw new IllegalArgumentException("Invalid sign, original = " + originalSignature + + " current = " + currentSignature); + } + return rawValue; + } + + /** + * Get the signature of the input string based on SHA digest algorithm. + * @param str Input token + * @return Signed String + */ + private String getSignature(String str) { + try { + MessageDigest md = MessageDigest.getInstance(SHA_STRING); + md.update(str.getBytes()); + md.update(secretBytes); + byte[] digest = md.digest(); + return new Base64(0).encodeToString(digest); + } catch (NoSuchAlgorithmException ex) { + throw new RuntimeException("Invalid SHA digest String: " + SHA_STRING + + " " + ex.getMessage(), ex); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java new file mode 100644 index 000000000000..5a508745414a --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java @@ -0,0 +1,83 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import org.apache.hadoop.hive.conf.HiveConf; + +/** + * FilterService. + * + */ +public class FilterService implements Service { + + + private final Service service; + private final long startTime = System.currentTimeMillis(); + + public FilterService(Service service) { + this.service = service; + } + + @Override + public void init(HiveConf config) { + service.init(config); + } + + @Override + public void start() { + service.start(); + } + + @Override + public void stop() { + service.stop(); + } + + + @Override + public void register(ServiceStateChangeListener listener) { + service.register(listener); + } + + @Override + public void unregister(ServiceStateChangeListener listener) { + service.unregister(listener); + } + + @Override + public String getName() { + return service.getName(); + } + + @Override + public HiveConf getHiveConf() { + return service.getHiveConf(); + } + + @Override + public STATE getServiceState() { + return service.getServiceState(); + } + + @Override + public long getStartTime() { + return startTime; + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java new file mode 100644 index 000000000000..0d0e3e4011b5 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/Service.java @@ -0,0 +1,122 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import org.apache.hadoop.hive.conf.HiveConf; + +/** + * Service. + * + */ +public interface Service { + + /** + * Service states + */ + enum STATE { + /** Constructed but not initialized */ + NOTINITED, + + /** Initialized but not started or stopped */ + INITED, + + /** started and not stopped */ + STARTED, + + /** stopped. No further state transitions are permitted */ + STOPPED + } + + /** + * Initialize the service. + * + * The transition must be from {@link STATE#NOTINITED} to {@link STATE#INITED} unless the + * operation failed and an exception was raised. + * + * @param conf + * the configuration of the service + */ + void init(HiveConf conf); + + + /** + * Start the service. + * + * The transition should be from {@link STATE#INITED} to {@link STATE#STARTED} unless the + * operation failed and an exception was raised. + */ + void start(); + + /** + * Stop the service. + * + * This operation must be designed to complete regardless of the initial state + * of the service, including the state of all its internal fields. + */ + void stop(); + + /** + * Register an instance of the service state change events. + * + * @param listener + * a new listener + */ + void register(ServiceStateChangeListener listener); + + /** + * Unregister a previously instance of the service state change events. + * + * @param listener + * the listener to unregister. + */ + void unregister(ServiceStateChangeListener listener); + + /** + * Get the name of this service. + * + * @return the service name + */ + String getName(); + + /** + * Get the configuration of this service. + * This is normally not a clone and may be manipulated, though there are no + * guarantees as to what the consequences of such actions may be + * + * @return the current configuration, unless a specific implementation chooses + * otherwise. + */ + HiveConf getHiveConf(); + + /** + * Get the current service state + * + * @return the state of the service + */ + STATE getServiceState(); + + /** + * Get the service start time + * + * @return the start time of the service. This will be zero if the service + * has not yet been started. + */ + long getStartTime(); + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceException.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceException.java new file mode 100644 index 000000000000..3622cf8920a8 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceException.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +/** + * ServiceException. + * + */ +public class ServiceException extends RuntimeException { + + public ServiceException(Throwable cause) { + super(cause); + } + + public ServiceException(String message) { + super(message); + } + + public ServiceException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java new file mode 100644 index 000000000000..f16863c1b41a --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java @@ -0,0 +1,141 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; + +/** + * ServiceOperations. + * + */ +public final class ServiceOperations { + private static final Log LOG = LogFactory.getLog(ServiceOperations.class); + + private ServiceOperations() { + } + + /** + * Verify that a service is in a given state. + * @param state the actual state a service is in + * @param expectedState the desired state + * @throws IllegalStateException if the service state is different from + * the desired state + */ + public static void ensureCurrentState(Service.STATE state, + Service.STATE expectedState) { + if (state != expectedState) { + throw new IllegalStateException("For this operation, the " + + "current service state must be " + + expectedState + + " instead of " + state); + } + } + + /** + * Initialize a service. + * + * The service state is checked before the operation begins. + * This process is not thread safe. + * @param service a service that must be in the state + * {@link Service.STATE#NOTINITED} + * @param configuration the configuration to initialize the service with + * @throws RuntimeException on a state change failure + * @throws IllegalStateException if the service is in the wrong state + */ + + public static void init(Service service, HiveConf configuration) { + Service.STATE state = service.getServiceState(); + ensureCurrentState(state, Service.STATE.NOTINITED); + service.init(configuration); + } + + /** + * Start a service. + * + * The service state is checked before the operation begins. + * This process is not thread safe. + * @param service a service that must be in the state + * {@link Service.STATE#INITED} + * @throws RuntimeException on a state change failure + * @throws IllegalStateException if the service is in the wrong state + */ + + public static void start(Service service) { + Service.STATE state = service.getServiceState(); + ensureCurrentState(state, Service.STATE.INITED); + service.start(); + } + + /** + * Initialize then start a service. + * + * The service state is checked before the operation begins. + * This process is not thread safe. + * @param service a service that must be in the state + * {@link Service.STATE#NOTINITED} + * @param configuration the configuration to initialize the service with + * @throws RuntimeException on a state change failure + * @throws IllegalStateException if the service is in the wrong state + */ + public static void deploy(Service service, HiveConf configuration) { + init(service, configuration); + start(service); + } + + /** + * Stop a service. + * + * Do nothing if the service is null or not in a state in which it can be/needs to be stopped. + * + * The service state is checked before the operation begins. + * This process is not thread safe. + * @param service a service or null + */ + public static void stop(Service service) { + if (service != null) { + Service.STATE state = service.getServiceState(); + if (state == Service.STATE.STARTED) { + service.stop(); + } + } + } + + /** + * Stop a service; if it is null do nothing. Exceptions are caught and + * logged at warn level. (but not Throwables). This operation is intended to + * be used in cleanup operations + * + * @param service a service; may be null + * @return any exception that was caught; null if none was. + */ + public static Exception stopQuietly(Service service) { + try { + stop(service); + } catch (Exception e) { + LOG.warn("When stopping the service " + service.getName() + + " : " + e, + e); + return e; + } + return null; + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java new file mode 100644 index 000000000000..a1ff10dc2bc9 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceStateChangeListener.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +/** + * ServiceStateChangeListener. + * + */ +public interface ServiceStateChangeListener { + + /** + * Callback to notify of a state change. The service will already + * have changed state before this callback is invoked. + * + * This operation is invoked on the thread that initiated the state change, + * while the service itself in a synchronized section. + *
      + *
    1. Any long-lived operation here will prevent the service state + * change from completing in a timely manner.
    2. + *
    3. If another thread is somehow invoked from the listener, and + * that thread invokes the methods of the service (including + * subclass-specific methods), there is a risk of a deadlock.
    4. + *
    + * + * + * @param service the service that has changed. + */ + void stateChanged(Service service); + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java new file mode 100644 index 000000000000..edb5eff9615b --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service; + +public class ServiceUtils { + + /* + * Get the index separating the user name from domain name (the user's name up + * to the first '/' or '@'). + * + * @param userName full user name. + * @return index of domain match or -1 if not found + */ + public static int indexOfDomainMatch(String userName) { + if (userName == null) { + return -1; + } + + int idx = userName.indexOf('/'); + int idx2 = userName.indexOf('@'); + int endIdx = Math.min(idx, idx2); // Use the earlier match. + // Unless at least one of '/' or '@' was not found, in + // which case, user the latter match. + if (endIdx == -1) { + endIdx = Math.max(idx, idx2); + } + return endIdx; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java new file mode 100644 index 000000000000..c8f93ff6a511 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java @@ -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.hive.service.auth; + +import javax.security.sasl.AuthenticationException; + +/** + * This authentication provider allows any combination of username and password. + */ +public class AnonymousAuthenticationProviderImpl implements PasswdAuthenticationProvider { + + @Override + public void Authenticate(String user, String password) throws AuthenticationException { + // no-op authentication + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java new file mode 100644 index 000000000000..4b95503eb19c --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java @@ -0,0 +1,71 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import javax.security.sasl.AuthenticationException; + +/** + * This class helps select a {@link PasswdAuthenticationProvider} for a given {@code AuthMethod}. + */ +public final class AuthenticationProviderFactory { + + public enum AuthMethods { + LDAP("LDAP"), + PAM("PAM"), + CUSTOM("CUSTOM"), + NONE("NONE"); + + private final String authMethod; + + AuthMethods(String authMethod) { + this.authMethod = authMethod; + } + + public String getAuthMethod() { + return authMethod; + } + + public static AuthMethods getValidAuthMethod(String authMethodStr) + throws AuthenticationException { + for (AuthMethods auth : AuthMethods.values()) { + if (authMethodStr.equals(auth.getAuthMethod())) { + return auth; + } + } + throw new AuthenticationException("Not a valid authentication method"); + } + } + + private AuthenticationProviderFactory() { + } + + public static PasswdAuthenticationProvider getAuthenticationProvider(AuthMethods authMethod) + throws AuthenticationException { + if (authMethod == AuthMethods.LDAP) { + return new LdapAuthenticationProviderImpl(); + } else if (authMethod == AuthMethods.PAM) { + return new PamAuthenticationProviderImpl(); + } else if (authMethod == AuthMethods.CUSTOM) { + return new CustomAuthenticationProviderImpl(); + } else if (authMethod == AuthMethods.NONE) { + return new AnonymousAuthenticationProviderImpl(); + } else { + throw new AuthenticationException("Unsupported authentication method"); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java new file mode 100644 index 000000000000..3dc0aa86e2d4 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java @@ -0,0 +1,50 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import javax.security.sasl.AuthenticationException; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.util.ReflectionUtils; + +/** + * This authentication provider implements the {@code CUSTOM} authentication. It allows a {@link + * PasswdAuthenticationProvider} to be specified at configuration time which may additionally + * implement {@link org.apache.hadoop.conf.Configurable Configurable} to grab Hive's {@link + * org.apache.hadoop.conf.Configuration Configuration}. + */ +public class CustomAuthenticationProviderImpl implements PasswdAuthenticationProvider { + + private final PasswdAuthenticationProvider customProvider; + + @SuppressWarnings("unchecked") + CustomAuthenticationProviderImpl() { + HiveConf conf = new HiveConf(); + Class customHandlerClass = + (Class) conf.getClass( + HiveConf.ConfVars.HIVE_SERVER2_CUSTOM_AUTHENTICATION_CLASS.varname, + PasswdAuthenticationProvider.class); + customProvider = ReflectionUtils.newInstance(customHandlerClass, conf); + } + + @Override + public void Authenticate(String user, String password) throws AuthenticationException { + customProvider.Authenticate(user, password); + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java new file mode 100644 index 000000000000..c5ade6528304 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java @@ -0,0 +1,365 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import javax.net.ssl.SSLServerSocket; +import javax.security.auth.login.LoginException; +import javax.security.sasl.Sasl; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.HiveMetaStore; +import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.shims.HadoopShims.KerberosNameShim; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.hive.thrift.DBTokenStore; +import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge; +import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server.ServerMode; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authorize.ProxyUsers; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.thrift.ThriftCLIService; +import org.apache.thrift.TProcessorFactory; +import org.apache.thrift.transport.TSSLTransportFactory; +import org.apache.thrift.transport.TServerSocket; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; +import org.apache.thrift.transport.TTransportFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class helps in some aspects of authentication. It creates the proper Thrift classes for the + * given configuration as well as helps with authenticating requests. + */ +public class HiveAuthFactory { + private static final Logger LOG = LoggerFactory.getLogger(HiveAuthFactory.class); + + + public enum AuthTypes { + NOSASL("NOSASL"), + NONE("NONE"), + LDAP("LDAP"), + KERBEROS("KERBEROS"), + CUSTOM("CUSTOM"), + PAM("PAM"); + + private final String authType; + + AuthTypes(String authType) { + this.authType = authType; + } + + public String getAuthName() { + return authType; + } + + } + + private HadoopThriftAuthBridge.Server saslServer; + private String authTypeStr; + private final String transportMode; + private final HiveConf conf; + + public static final String HS2_PROXY_USER = "hive.server2.proxy.user"; + public static final String HS2_CLIENT_TOKEN = "hiveserver2ClientToken"; + + public HiveAuthFactory(HiveConf conf) throws TTransportException { + this.conf = conf; + transportMode = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE); + authTypeStr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION); + + // In http mode we use NOSASL as the default auth type + if ("http".equalsIgnoreCase(transportMode)) { + if (authTypeStr == null) { + authTypeStr = AuthTypes.NOSASL.getAuthName(); + } + } else { + if (authTypeStr == null) { + authTypeStr = AuthTypes.NONE.getAuthName(); + } + if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { + saslServer = ShimLoader.getHadoopThriftAuthBridge() + .createServer(conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB), + conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL)); + // start delegation token manager + try { + // rawStore is only necessary for DBTokenStore + Object rawStore = null; + String tokenStoreClass = conf.getVar(HiveConf.ConfVars.METASTORE_CLUSTER_DELEGATION_TOKEN_STORE_CLS); + + if (tokenStoreClass.equals(DBTokenStore.class.getName())) { + HMSHandler baseHandler = new HiveMetaStore.HMSHandler( + "new db based metaserver", conf, true); + rawStore = baseHandler.getMS(); + } + + saslServer.startDelegationTokenSecretManager(conf, rawStore, ServerMode.HIVESERVER2); + } + catch (MetaException|IOException e) { + throw new TTransportException("Failed to start token manager", e); + } + } + } + } + + public Map getSaslProperties() { + Map saslProps = new HashMap(); + SaslQOP saslQOP = SaslQOP.fromString(conf.getVar(ConfVars.HIVE_SERVER2_THRIFT_SASL_QOP)); + saslProps.put(Sasl.QOP, saslQOP.toString()); + saslProps.put(Sasl.SERVER_AUTH, "true"); + return saslProps; + } + + public TTransportFactory getAuthTransFactory() throws LoginException { + TTransportFactory transportFactory; + if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { + try { + transportFactory = saslServer.createTransportFactory(getSaslProperties()); + } catch (TTransportException e) { + throw new LoginException(e.getMessage()); + } + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NONE.getAuthName())) { + transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.LDAP.getAuthName())) { + transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.PAM.getAuthName())) { + transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NOSASL.getAuthName())) { + transportFactory = new TTransportFactory(); + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.CUSTOM.getAuthName())) { + transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); + } else { + throw new LoginException("Unsupported authentication type " + authTypeStr); + } + return transportFactory; + } + + /** + * Returns the thrift processor factory for HiveServer2 running in binary mode + * @param service + * @return + * @throws LoginException + */ + public TProcessorFactory getAuthProcFactory(ThriftCLIService service) throws LoginException { + if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { + return KerberosSaslHelper.getKerberosProcessorFactory(saslServer, service); + } else { + return PlainSaslHelper.getPlainProcessorFactory(service); + } + } + + public String getRemoteUser() { + return saslServer == null ? null : saslServer.getRemoteUser(); + } + + public String getIpAddress() { + if (saslServer == null || saslServer.getRemoteAddress() == null) { + return null; + } else { + return saslServer.getRemoteAddress().getHostAddress(); + } + } + + // Perform kerberos login using the hadoop shim API if the configuration is available + public static void loginFromKeytab(HiveConf hiveConf) throws IOException { + String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL); + String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB); + if (principal.isEmpty() || keyTabFile.isEmpty()) { + throw new IOException("HiveServer2 Kerberos principal or keytab is not correctly configured"); + } else { + UserGroupInformation.loginUserFromKeytab(SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keyTabFile); + } + } + + // Perform SPNEGO login using the hadoop shim API if the configuration is available + public static UserGroupInformation loginFromSpnegoKeytabAndReturnUGI(HiveConf hiveConf) + throws IOException { + String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_PRINCIPAL); + String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_KEYTAB); + if (principal.isEmpty() || keyTabFile.isEmpty()) { + throw new IOException("HiveServer2 SPNEGO principal or keytab is not correctly configured"); + } else { + return UserGroupInformation.loginUserFromKeytabAndReturnUGI(SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keyTabFile); + } + } + + public static TTransport getSocketTransport(String host, int port, int loginTimeout) { + return new TSocket(host, port, loginTimeout); + } + + public static TTransport getSSLSocket(String host, int port, int loginTimeout) + throws TTransportException { + return TSSLTransportFactory.getClientSocket(host, port, loginTimeout); + } + + public static TTransport getSSLSocket(String host, int port, int loginTimeout, + String trustStorePath, String trustStorePassWord) throws TTransportException { + TSSLTransportFactory.TSSLTransportParameters params = + new TSSLTransportFactory.TSSLTransportParameters(); + params.setTrustStore(trustStorePath, trustStorePassWord); + params.requireClientAuth(true); + return TSSLTransportFactory.getClientSocket(host, port, loginTimeout, params); + } + + public static TServerSocket getServerSocket(String hiveHost, int portNum) + throws TTransportException { + InetSocketAddress serverAddress; + if (hiveHost == null || hiveHost.isEmpty()) { + // Wildcard bind + serverAddress = new InetSocketAddress(portNum); + } else { + serverAddress = new InetSocketAddress(hiveHost, portNum); + } + return new TServerSocket(serverAddress); + } + + public static TServerSocket getServerSSLSocket(String hiveHost, int portNum, String keyStorePath, + String keyStorePassWord, List sslVersionBlacklist) throws TTransportException, + UnknownHostException { + TSSLTransportFactory.TSSLTransportParameters params = + new TSSLTransportFactory.TSSLTransportParameters(); + params.setKeyStore(keyStorePath, keyStorePassWord); + InetSocketAddress serverAddress; + if (hiveHost == null || hiveHost.isEmpty()) { + // Wildcard bind + serverAddress = new InetSocketAddress(portNum); + } else { + serverAddress = new InetSocketAddress(hiveHost, portNum); + } + TServerSocket thriftServerSocket = + TSSLTransportFactory.getServerSocket(portNum, 0, serverAddress.getAddress(), params); + if (thriftServerSocket.getServerSocket() instanceof SSLServerSocket) { + List sslVersionBlacklistLocal = new ArrayList(); + for (String sslVersion : sslVersionBlacklist) { + sslVersionBlacklistLocal.add(sslVersion.trim().toLowerCase(Locale.ROOT)); + } + SSLServerSocket sslServerSocket = (SSLServerSocket) thriftServerSocket.getServerSocket(); + List enabledProtocols = new ArrayList(); + for (String protocol : sslServerSocket.getEnabledProtocols()) { + if (sslVersionBlacklistLocal.contains(protocol.toLowerCase(Locale.ROOT))) { + LOG.debug("Disabling SSL Protocol: " + protocol); + } else { + enabledProtocols.add(protocol); + } + } + sslServerSocket.setEnabledProtocols(enabledProtocols.toArray(new String[0])); + LOG.info("SSL Server Socket Enabled Protocols: " + + Arrays.toString(sslServerSocket.getEnabledProtocols())); + } + return thriftServerSocket; + } + + // retrieve delegation token for the given user + public String getDelegationToken(String owner, String renewer) throws HiveSQLException { + if (saslServer == null) { + throw new HiveSQLException( + "Delegation token only supported over kerberos authentication", "08S01"); + } + + try { + String tokenStr = saslServer.getDelegationTokenWithService(owner, renewer, HS2_CLIENT_TOKEN); + if (tokenStr == null || tokenStr.isEmpty()) { + throw new HiveSQLException( + "Received empty retrieving delegation token for user " + owner, "08S01"); + } + return tokenStr; + } catch (IOException e) { + throw new HiveSQLException( + "Error retrieving delegation token for user " + owner, "08S01", e); + } catch (InterruptedException e) { + throw new HiveSQLException("delegation token retrieval interrupted", "08S01", e); + } + } + + // cancel given delegation token + public void cancelDelegationToken(String delegationToken) throws HiveSQLException { + if (saslServer == null) { + throw new HiveSQLException( + "Delegation token only supported over kerberos authentication", "08S01"); + } + try { + saslServer.cancelDelegationToken(delegationToken); + } catch (IOException e) { + throw new HiveSQLException( + "Error canceling delegation token " + delegationToken, "08S01", e); + } + } + + public void renewDelegationToken(String delegationToken) throws HiveSQLException { + if (saslServer == null) { + throw new HiveSQLException( + "Delegation token only supported over kerberos authentication", "08S01"); + } + try { + saslServer.renewDelegationToken(delegationToken); + } catch (IOException e) { + throw new HiveSQLException( + "Error renewing delegation token " + delegationToken, "08S01", e); + } + } + + public String getUserFromToken(String delegationToken) throws HiveSQLException { + if (saslServer == null) { + throw new HiveSQLException( + "Delegation token only supported over kerberos authentication", "08S01"); + } + try { + return saslServer.getUserFromToken(delegationToken); + } catch (IOException e) { + throw new HiveSQLException( + "Error extracting user from delegation token " + delegationToken, "08S01", e); + } + } + + public static void verifyProxyAccess(String realUser, String proxyUser, String ipAddress, + HiveConf hiveConf) throws HiveSQLException { + try { + UserGroupInformation sessionUgi; + if (UserGroupInformation.isSecurityEnabled()) { + KerberosNameShim kerbName = ShimLoader.getHadoopShims().getKerberosNameShim(realUser); + sessionUgi = UserGroupInformation.createProxyUser( + kerbName.getServiceName(), UserGroupInformation.getLoginUser()); + } else { + sessionUgi = UserGroupInformation.createRemoteUser(realUser); + } + if (!proxyUser.equalsIgnoreCase(realUser)) { + ProxyUsers.refreshSuperUserGroupsConfiguration(hiveConf); + ProxyUsers.authorize(UserGroupInformation.createProxyUser(proxyUser, sessionUgi), + ipAddress, hiveConf); + } + } catch (IOException e) { + throw new HiveSQLException( + "Failed to validate proxy privilege of " + realUser + " for " + proxyUser, "08S01", e); + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java new file mode 100644 index 000000000000..f7375ee70783 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java @@ -0,0 +1,189 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.auth; + +import java.security.AccessControlContext; +import java.security.AccessController; +import java.security.PrivilegedExceptionAction; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.StringTokenizer; + +import javax.security.auth.Subject; + +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.http.protocol.BasicHttpContext; +import org.apache.http.protocol.HttpContext; +import org.ietf.jgss.GSSContext; +import org.ietf.jgss.GSSManager; +import org.ietf.jgss.GSSName; +import org.ietf.jgss.Oid; + +/** + * Utility functions for HTTP mode authentication. + */ +public final class HttpAuthUtils { + public static final String WWW_AUTHENTICATE = "WWW-Authenticate"; + public static final String AUTHORIZATION = "Authorization"; + public static final String BASIC = "Basic"; + public static final String NEGOTIATE = "Negotiate"; + private static final Log LOG = LogFactory.getLog(HttpAuthUtils.class); + private static final String COOKIE_ATTR_SEPARATOR = "&"; + private static final String COOKIE_CLIENT_USER_NAME = "cu"; + private static final String COOKIE_CLIENT_RAND_NUMBER = "rn"; + private static final String COOKIE_KEY_VALUE_SEPARATOR = "="; + private static final Set COOKIE_ATTRIBUTES = + new HashSet(Arrays.asList(COOKIE_CLIENT_USER_NAME, COOKIE_CLIENT_RAND_NUMBER)); + + /** + * @return Stringified Base64 encoded kerberosAuthHeader on success + * @throws Exception + */ + public static String getKerberosServiceTicket(String principal, String host, + String serverHttpUrl, boolean assumeSubject) throws Exception { + String serverPrincipal = + ShimLoader.getHadoopThriftAuthBridge().getServerPrincipal(principal, host); + if (assumeSubject) { + // With this option, we're assuming that the external application, + // using the JDBC driver has done a JAAS kerberos login already + AccessControlContext context = AccessController.getContext(); + Subject subject = Subject.getSubject(context); + if (subject == null) { + throw new Exception("The Subject is not set"); + } + return Subject.doAs(subject, new HttpKerberosClientAction(serverPrincipal, serverHttpUrl)); + } else { + // JAAS login from ticket cache to setup the client UserGroupInformation + UserGroupInformation clientUGI = + ShimLoader.getHadoopThriftAuthBridge().getCurrentUGIWithConf("kerberos"); + return clientUGI.doAs(new HttpKerberosClientAction(serverPrincipal, serverHttpUrl)); + } + } + + /** + * Creates and returns a HS2 cookie token. + * @param clientUserName Client User name. + * @return An unsigned cookie token generated from input parameters. + * The final cookie generated is of the following format : + * {@code cu=&rn=&s=} + */ + public static String createCookieToken(String clientUserName) { + StringBuffer sb = new StringBuffer(); + sb.append(COOKIE_CLIENT_USER_NAME).append(COOKIE_KEY_VALUE_SEPARATOR).append(clientUserName) + .append(COOKIE_ATTR_SEPARATOR); + sb.append(COOKIE_CLIENT_RAND_NUMBER).append(COOKIE_KEY_VALUE_SEPARATOR) + .append((new Random(System.currentTimeMillis())).nextLong()); + return sb.toString(); + } + + /** + * Parses a cookie token to retrieve client user name. + * @param tokenStr Token String. + * @return A valid user name if input is of valid format, else returns null. + */ + public static String getUserNameFromCookieToken(String tokenStr) { + Map map = splitCookieToken(tokenStr); + + if (!map.keySet().equals(COOKIE_ATTRIBUTES)) { + LOG.error("Invalid token with missing attributes " + tokenStr); + return null; + } + return map.get(COOKIE_CLIENT_USER_NAME); + } + + /** + * Splits the cookie token into attributes pairs. + * @param str input token. + * @return a map with the attribute pairs of the token if the input is valid. + * Else, returns null. + */ + private static Map splitCookieToken(String tokenStr) { + Map map = new HashMap(); + StringTokenizer st = new StringTokenizer(tokenStr, COOKIE_ATTR_SEPARATOR); + + while (st.hasMoreTokens()) { + String part = st.nextToken(); + int separator = part.indexOf(COOKIE_KEY_VALUE_SEPARATOR); + if (separator == -1) { + LOG.error("Invalid token string " + tokenStr); + return null; + } + String key = part.substring(0, separator); + String value = part.substring(separator + 1); + map.put(key, value); + } + return map; + } + + + private HttpAuthUtils() { + throw new UnsupportedOperationException("Can't initialize class"); + } + + /** + * We'll create an instance of this class within a doAs block so that the client's TGT credentials + * can be read from the Subject + */ + public static class HttpKerberosClientAction implements PrivilegedExceptionAction { + public static final String HTTP_RESPONSE = "HTTP_RESPONSE"; + public static final String SERVER_HTTP_URL = "SERVER_HTTP_URL"; + private final String serverPrincipal; + private final String serverHttpUrl; + private final Base64 base64codec; + private final HttpContext httpContext; + + public HttpKerberosClientAction(String serverPrincipal, String serverHttpUrl) { + this.serverPrincipal = serverPrincipal; + this.serverHttpUrl = serverHttpUrl; + base64codec = new Base64(0); + httpContext = new BasicHttpContext(); + httpContext.setAttribute(SERVER_HTTP_URL, serverHttpUrl); + } + + @Override + public String run() throws Exception { + // This Oid for Kerberos GSS-API mechanism. + Oid mechOid = new Oid("1.2.840.113554.1.2.2"); + // Oid for kerberos principal name + Oid krb5PrincipalOid = new Oid("1.2.840.113554.1.2.2.1"); + GSSManager manager = GSSManager.getInstance(); + // GSS name for server + GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid); + // Create a GSSContext for authentication with the service. + // We're passing client credentials as null since we want them to be read from the Subject. + GSSContext gssContext = + manager.createContext(serverName, mechOid, null, GSSContext.DEFAULT_LIFETIME); + gssContext.requestMutualAuth(false); + // Establish context + byte[] inToken = new byte[0]; + byte[] outToken = gssContext.initSecContext(inToken, 0, inToken.length); + gssContext.dispose(); + // Base64 encoded and stringified token for server + return new String(base64codec.encode(outToken)); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthenticationException.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthenticationException.java new file mode 100644 index 000000000000..57643256022e --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthenticationException.java @@ -0,0 +1,43 @@ +/** + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. See accompanying LICENSE file. + */ + +package org.apache.hive.service.auth; + +public class HttpAuthenticationException extends Exception { + + private static final long serialVersionUID = 0; + + /** + * @param cause original exception + */ + public HttpAuthenticationException(Throwable cause) { + super(cause); + } + + /** + * @param msg exception message + */ + public HttpAuthenticationException(String msg) { + super(msg); + } + + /** + * @param msg exception message + * @param cause original exception + */ + public HttpAuthenticationException(String msg, Throwable cause) { + super(msg, cause); + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java new file mode 100644 index 000000000000..52eb752f1e02 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java @@ -0,0 +1,111 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import java.io.IOException; +import java.util.Map; +import javax.security.sasl.SaslException; + +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge; +import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server; +import org.apache.hive.service.cli.thrift.TCLIService; +import org.apache.hive.service.cli.thrift.TCLIService.Iface; +import org.apache.hive.service.cli.thrift.ThriftCLIService; +import org.apache.thrift.TProcessor; +import org.apache.thrift.TProcessorFactory; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TTransport; + +public final class KerberosSaslHelper { + + public static TProcessorFactory getKerberosProcessorFactory(Server saslServer, + ThriftCLIService service) { + return new CLIServiceProcessorFactory(saslServer, service); + } + + public static TTransport getKerberosTransport(String principal, String host, + TTransport underlyingTransport, Map saslProps, boolean assumeSubject) + throws SaslException { + try { + String[] names = principal.split("[/@]"); + if (names.length != 3) { + throw new IllegalArgumentException("Kerberos principal should have 3 parts: " + principal); + } + + if (assumeSubject) { + return createSubjectAssumedTransport(principal, underlyingTransport, saslProps); + } else { + HadoopThriftAuthBridge.Client authBridge = + ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos"); + return authBridge.createClientTransport(principal, host, "KERBEROS", null, + underlyingTransport, saslProps); + } + } catch (IOException e) { + throw new SaslException("Failed to open client transport", e); + } + } + + public static TTransport createSubjectAssumedTransport(String principal, + TTransport underlyingTransport, Map saslProps) throws IOException { + String[] names = principal.split("[/@]"); + try { + TTransport saslTransport = + new TSaslClientTransport("GSSAPI", null, names[0], names[1], saslProps, null, + underlyingTransport); + return new TSubjectAssumingTransport(saslTransport); + } catch (SaslException se) { + throw new IOException("Could not instantiate SASL transport", se); + } + } + + public static TTransport getTokenTransport(String tokenStr, String host, + TTransport underlyingTransport, Map saslProps) throws SaslException { + HadoopThriftAuthBridge.Client authBridge = + ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos"); + + try { + return authBridge.createClientTransport(null, host, "DIGEST", tokenStr, underlyingTransport, + saslProps); + } catch (IOException e) { + throw new SaslException("Failed to open client transport", e); + } + } + + private KerberosSaslHelper() { + throw new UnsupportedOperationException("Can't initialize class"); + } + + private static class CLIServiceProcessorFactory extends TProcessorFactory { + + private final ThriftCLIService service; + private final Server saslServer; + + CLIServiceProcessorFactory(Server saslServer, ThriftCLIService service) { + super(null); + this.service = service; + this.saslServer = saslServer; + } + + @Override + public TProcessor getProcessor(TTransport trans) { + TProcessor sqlProcessor = new TCLIService.Processor(service); + return saslServer.wrapNonAssumingProcessor(sqlProcessor); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java new file mode 100644 index 000000000000..4e2ef90a1e90 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java @@ -0,0 +1,84 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import java.util.Hashtable; +import javax.naming.Context; +import javax.naming.NamingException; +import javax.naming.directory.InitialDirContext; +import javax.security.sasl.AuthenticationException; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hive.service.ServiceUtils; + +public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvider { + + private final String ldapURL; + private final String baseDN; + private final String ldapDomain; + + LdapAuthenticationProviderImpl() { + HiveConf conf = new HiveConf(); + ldapURL = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_URL); + baseDN = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN); + ldapDomain = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_DOMAIN); + } + + @Override + public void Authenticate(String user, String password) throws AuthenticationException { + + Hashtable env = new Hashtable(); + env.put(Context.INITIAL_CONTEXT_FACTORY, "com.sun.jndi.ldap.LdapCtxFactory"); + env.put(Context.PROVIDER_URL, ldapURL); + + // If the domain is available in the config, then append it unless domain is + // already part of the username. LDAP providers like Active Directory use a + // fully qualified user name like foo@bar.com. + if (!hasDomain(user) && ldapDomain != null) { + user = user + "@" + ldapDomain; + } + + if (password == null || password.isEmpty() || password.getBytes()[0] == 0) { + throw new AuthenticationException("Error validating LDAP user:" + + " a null or blank password has been provided"); + } + + // setup the security principal + String bindDN; + if (baseDN == null) { + bindDN = user; + } else { + bindDN = "uid=" + user + "," + baseDN; + } + env.put(Context.SECURITY_AUTHENTICATION, "simple"); + env.put(Context.SECURITY_PRINCIPAL, bindDN); + env.put(Context.SECURITY_CREDENTIALS, password); + + try { + // Create initial context + Context ctx = new InitialDirContext(env); + ctx.close(); + } catch (NamingException e) { + throw new AuthenticationException("Error validating LDAP user", e); + } + } + + private boolean hasDomain(String userName) { + return (ServiceUtils.indexOfDomainMatch(userName) > 0); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PamAuthenticationProviderImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PamAuthenticationProviderImpl.java new file mode 100644 index 000000000000..68f62c461790 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PamAuthenticationProviderImpl.java @@ -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.hive.service.auth; + +import javax.security.sasl.AuthenticationException; + +import net.sf.jpam.Pam; +import org.apache.hadoop.hive.conf.HiveConf; + +public class PamAuthenticationProviderImpl implements PasswdAuthenticationProvider { + + private final String pamServiceNames; + + PamAuthenticationProviderImpl() { + HiveConf conf = new HiveConf(); + pamServiceNames = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PAM_SERVICES); + } + + @Override + public void Authenticate(String user, String password) throws AuthenticationException { + + if (pamServiceNames == null || pamServiceNames.trim().isEmpty()) { + throw new AuthenticationException("No PAM services are set."); + } + + String[] pamServices = pamServiceNames.split(","); + for (String pamService : pamServices) { + Pam pam = new Pam(pamService); + boolean isAuthenticated = pam.authenticateSuccessful(user, password); + if (!isAuthenticated) { + throw new AuthenticationException( + "Error authenticating with the PAM service: " + pamService); + } + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java new file mode 100644 index 000000000000..1af1c1d06e7f --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import javax.security.sasl.AuthenticationException; + +public interface PasswdAuthenticationProvider { + + /** + * The Authenticate method is called by the HiveServer2 authentication layer + * to authenticate users for their requests. + * If a user is to be granted, return nothing/throw nothing. + * When a user is to be disallowed, throw an appropriate {@link AuthenticationException}. + * + * For an example implementation, see {@link LdapAuthenticationProviderImpl}. + * + * @param user The username received over the connection request + * @param password The password received over the connection request + * + * @throws AuthenticationException When a user is found to be + * invalid by the implementation + */ + void Authenticate(String user, String password) throws AuthenticationException; +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java new file mode 100644 index 000000000000..afc144199f1e --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java @@ -0,0 +1,154 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import java.io.IOException; +import java.security.Security; +import java.util.HashMap; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.auth.login.LoginException; +import javax.security.sasl.AuthenticationException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.SaslException; + +import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods; +import org.apache.hive.service.auth.PlainSaslServer.SaslPlainProvider; +import org.apache.hive.service.cli.thrift.TCLIService.Iface; +import org.apache.hive.service.cli.thrift.ThriftCLIService; +import org.apache.thrift.TProcessor; +import org.apache.thrift.TProcessorFactory; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportFactory; + +public final class PlainSaslHelper { + + public static TProcessorFactory getPlainProcessorFactory(ThriftCLIService service) { + return new SQLPlainProcessorFactory(service); + } + + // Register Plain SASL server provider + static { + Security.addProvider(new SaslPlainProvider()); + } + + public static TTransportFactory getPlainTransportFactory(String authTypeStr) + throws LoginException { + TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory(); + try { + saslFactory.addServerDefinition("PLAIN", authTypeStr, null, new HashMap(), + new PlainServerCallbackHandler(authTypeStr)); + } catch (AuthenticationException e) { + throw new LoginException("Error setting callback handler" + e); + } + return saslFactory; + } + + public static TTransport getPlainTransport(String username, String password, + TTransport underlyingTransport) throws SaslException { + return new TSaslClientTransport("PLAIN", null, null, null, new HashMap(), + new PlainCallbackHandler(username, password), underlyingTransport); + } + + private PlainSaslHelper() { + throw new UnsupportedOperationException("Can't initialize class"); + } + + private static final class PlainServerCallbackHandler implements CallbackHandler { + + private final AuthMethods authMethod; + + PlainServerCallbackHandler(String authMethodStr) throws AuthenticationException { + authMethod = AuthMethods.getValidAuthMethod(authMethodStr); + } + + @Override + public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + String username = null; + String password = null; + AuthorizeCallback ac = null; + + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + NameCallback nc = (NameCallback) callback; + username = nc.getName(); + } else if (callback instanceof PasswordCallback) { + PasswordCallback pc = (PasswordCallback) callback; + password = new String(pc.getPassword()); + } else if (callback instanceof AuthorizeCallback) { + ac = (AuthorizeCallback) callback; + } else { + throw new UnsupportedCallbackException(callback); + } + } + PasswdAuthenticationProvider provider = + AuthenticationProviderFactory.getAuthenticationProvider(authMethod); + provider.Authenticate(username, password); + if (ac != null) { + ac.setAuthorized(true); + } + } + } + + public static class PlainCallbackHandler implements CallbackHandler { + + private final String username; + private final String password; + + public PlainCallbackHandler(String username, String password) { + this.username = username; + this.password = password; + } + + @Override + public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + NameCallback nameCallback = (NameCallback) callback; + nameCallback.setName(username); + } else if (callback instanceof PasswordCallback) { + PasswordCallback passCallback = (PasswordCallback) callback; + passCallback.setPassword(password.toCharArray()); + } else { + throw new UnsupportedCallbackException(callback); + } + } + } + } + + private static final class SQLPlainProcessorFactory extends TProcessorFactory { + + private final ThriftCLIService service; + + SQLPlainProcessorFactory(ThriftCLIService service) { + super(null); + this.service = service; + } + + @Override + public TProcessor getProcessor(TTransport trans) { + return new TSetIpAddressProcessor(service); + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslServer.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslServer.java new file mode 100644 index 000000000000..cd675da29af1 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslServer.java @@ -0,0 +1,177 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.auth; + +import java.io.IOException; +import java.security.Provider; +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.Map; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.SaslException; +import javax.security.sasl.SaslServer; +import javax.security.sasl.SaslServerFactory; + +import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods; + +/** + * Sun JDK only provides a PLAIN client and no server. This class implements the Plain SASL server + * conforming to RFC #4616 (http://www.ietf.org/rfc/rfc4616.txt). + */ +public class PlainSaslServer implements SaslServer { + + public static final String PLAIN_METHOD = "PLAIN"; + private String user; + private final CallbackHandler handler; + + PlainSaslServer(CallbackHandler handler, String authMethodStr) throws SaslException { + this.handler = handler; + AuthMethods.getValidAuthMethod(authMethodStr); + } + + @Override + public String getMechanismName() { + return PLAIN_METHOD; + } + + @Override + public byte[] evaluateResponse(byte[] response) throws SaslException { + try { + // parse the response + // message = [authzid] UTF8NUL authcid UTF8NUL passwd' + + Deque tokenList = new ArrayDeque(); + StringBuilder messageToken = new StringBuilder(); + for (byte b : response) { + if (b == 0) { + tokenList.addLast(messageToken.toString()); + messageToken = new StringBuilder(); + } else { + messageToken.append((char) b); + } + } + tokenList.addLast(messageToken.toString()); + + // validate response + if (tokenList.size() < 2 || tokenList.size() > 3) { + throw new SaslException("Invalid message format"); + } + String passwd = tokenList.removeLast(); + user = tokenList.removeLast(); + // optional authzid + String authzId; + if (tokenList.isEmpty()) { + authzId = user; + } else { + authzId = tokenList.removeLast(); + } + if (user == null || user.isEmpty()) { + throw new SaslException("No user name provided"); + } + if (passwd == null || passwd.isEmpty()) { + throw new SaslException("No password name provided"); + } + + NameCallback nameCallback = new NameCallback("User"); + nameCallback.setName(user); + PasswordCallback pcCallback = new PasswordCallback("Password", false); + pcCallback.setPassword(passwd.toCharArray()); + AuthorizeCallback acCallback = new AuthorizeCallback(user, authzId); + + Callback[] cbList = {nameCallback, pcCallback, acCallback}; + handler.handle(cbList); + if (!acCallback.isAuthorized()) { + throw new SaslException("Authentication failed"); + } + } catch (IllegalStateException eL) { + throw new SaslException("Invalid message format", eL); + } catch (IOException eI) { + throw new SaslException("Error validating the login", eI); + } catch (UnsupportedCallbackException eU) { + throw new SaslException("Error validating the login", eU); + } + return null; + } + + @Override + public boolean isComplete() { + return user != null; + } + + @Override + public String getAuthorizationID() { + return user; + } + + @Override + public byte[] unwrap(byte[] incoming, int offset, int len) { + throw new UnsupportedOperationException(); + } + + @Override + public byte[] wrap(byte[] outgoing, int offset, int len) { + throw new UnsupportedOperationException(); + } + + @Override + public Object getNegotiatedProperty(String propName) { + return null; + } + + @Override + public void dispose() {} + + public static class SaslPlainServerFactory implements SaslServerFactory { + + @Override + public SaslServer createSaslServer(String mechanism, String protocol, String serverName, + Map props, CallbackHandler cbh) { + if (PLAIN_METHOD.equals(mechanism)) { + try { + return new PlainSaslServer(cbh, protocol); + } catch (SaslException e) { + /* This is to fulfill the contract of the interface which states that an exception shall + be thrown when a SaslServer cannot be created due to an error but null should be + returned when a Server can't be created due to the parameters supplied. And the only + thing PlainSaslServer can fail on is a non-supported authentication mechanism. + That's why we return null instead of throwing the Exception */ + return null; + } + } + return null; + } + + @Override + public String[] getMechanismNames(Map props) { + return new String[] {PLAIN_METHOD}; + } + } + + public static class SaslPlainProvider extends Provider { + + public SaslPlainProvider() { + super("HiveSaslPlain", 1.0, "Hive Plain SASL provider"); + put("SaslServerFactory.PLAIN", SaslPlainServerFactory.class.getName()); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java new file mode 100644 index 000000000000..ad4dfd75f470 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/SaslQOP.java @@ -0,0 +1,65 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.auth; + +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; + +/** + * Possible values of SASL quality-of-protection value. + */ +public enum SaslQOP { + // Authentication only. + AUTH("auth"), + // Authentication and integrity checking by using signatures. + AUTH_INT("auth-int"), + // Authentication, integrity and confidentiality checking by using signatures and encryption. + AUTH_CONF("auth-conf"); + + public final String saslQop; + + private static final Map STR_TO_ENUM = new HashMap(); + + static { + for (SaslQOP saslQop : values()) { + STR_TO_ENUM.put(saslQop.toString(), saslQop); + } + } + + SaslQOP(String saslQop) { + this.saslQop = saslQop; + } + + public String toString() { + return saslQop; + } + + public static SaslQOP fromString(String str) { + if (str != null) { + str = str.toLowerCase(Locale.ROOT); + } + SaslQOP saslQOP = STR_TO_ENUM.get(str); + if (saslQOP == null) { + throw new IllegalArgumentException( + "Unknown auth type: " + str + " Allowed values are: " + STR_TO_ENUM.keySet()); + } + return saslQOP; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java new file mode 100644 index 000000000000..9a61ad49942c --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java @@ -0,0 +1,114 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.auth; + +import org.apache.hive.service.cli.thrift.TCLIService; +import org.apache.hive.service.cli.thrift.TCLIService.Iface; +import org.apache.thrift.TException; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class is responsible for setting the ipAddress for operations executed via HiveServer2. + * + * - IP address is only set for operations that calls listeners with hookContext + * - IP address is only set if the underlying transport mechanism is socket + * + * @see org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext + */ +public class TSetIpAddressProcessor extends TCLIService.Processor { + + private static final Logger LOGGER = LoggerFactory.getLogger(TSetIpAddressProcessor.class.getName()); + + public TSetIpAddressProcessor(Iface iface) { + super(iface); + } + + @Override + public boolean process(final TProtocol in, final TProtocol out) throws TException { + setIpAddress(in); + setUserName(in); + try { + return super.process(in, out); + } finally { + THREAD_LOCAL_USER_NAME.remove(); + THREAD_LOCAL_IP_ADDRESS.remove(); + } + } + + private void setUserName(final TProtocol in) { + TTransport transport = in.getTransport(); + if (transport instanceof TSaslServerTransport) { + String userName = ((TSaslServerTransport) transport).getSaslServer().getAuthorizationID(); + THREAD_LOCAL_USER_NAME.set(userName); + } + } + + protected void setIpAddress(final TProtocol in) { + TTransport transport = in.getTransport(); + TSocket tSocket = getUnderlyingSocketFromTransport(transport); + if (tSocket == null) { + LOGGER.warn("Unknown Transport, cannot determine ipAddress"); + } else { + THREAD_LOCAL_IP_ADDRESS.set(tSocket.getSocket().getInetAddress().getHostAddress()); + } + } + + private TSocket getUnderlyingSocketFromTransport(TTransport transport) { + while (transport != null) { + if (transport instanceof TSaslServerTransport) { + transport = ((TSaslServerTransport) transport).getUnderlyingTransport(); + } + if (transport instanceof TSaslClientTransport) { + transport = ((TSaslClientTransport) transport).getUnderlyingTransport(); + } + if (transport instanceof TSocket) { + return (TSocket) transport; + } + } + return null; + } + + private static final ThreadLocal THREAD_LOCAL_IP_ADDRESS = new ThreadLocal() { + @Override + protected synchronized String initialValue() { + return null; + } + }; + + private static final ThreadLocal THREAD_LOCAL_USER_NAME = new ThreadLocal() { + @Override + protected synchronized String initialValue() { + return null; + } + }; + + public static String getUserIpAddress() { + return THREAD_LOCAL_IP_ADDRESS.get(); + } + + public static String getUserName() { + return THREAD_LOCAL_USER_NAME.get(); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSubjectAssumingTransport.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSubjectAssumingTransport.java new file mode 100644 index 000000000000..2422e86c6b46 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSubjectAssumingTransport.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.auth; + +import java.security.AccessControlContext; +import java.security.AccessController; +import java.security.PrivilegedActionException; +import java.security.PrivilegedExceptionAction; +import javax.security.auth.Subject; + +import org.apache.hadoop.hive.thrift.TFilterTransport; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; + +/** + * This is used on the client side, where the API explicitly opens a transport to + * the server using the Subject.doAs(). + */ +public class TSubjectAssumingTransport extends TFilterTransport { + + public TSubjectAssumingTransport(TTransport wrapped) { + super(wrapped); + } + + @Override + public void open() throws TTransportException { + try { + AccessControlContext context = AccessController.getContext(); + Subject subject = Subject.getSubject(context); + Subject.doAs(subject, new PrivilegedExceptionAction() { + public Void run() { + try { + wrapped.open(); + } catch (TTransportException tte) { + // Wrap the transport exception in an RTE, since Subject.doAs() then goes + // and unwraps this for us out of the doAs block. We then unwrap one + // more time in our catch clause to get back the TTE. (ugh) + throw new RuntimeException(tte); + } + return null; + } + }); + } catch (PrivilegedActionException ioe) { + throw new RuntimeException("Received an ioe we never threw!", ioe); + } catch (RuntimeException rte) { + if (rte.getCause() instanceof TTransportException) { + throw (TTransportException) rte.getCause(); + } else { + throw rte; + } + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java new file mode 100644 index 000000000000..791ddcbd2c5b --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java @@ -0,0 +1,507 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import javax.security.auth.login.LoginException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.shims.Utils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.service.CompositeService; +import org.apache.hive.service.ServiceException; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.operation.Operation; +import org.apache.hive.service.cli.session.SessionManager; +import org.apache.hive.service.cli.thrift.TProtocolVersion; +import org.apache.hive.service.server.HiveServer2; + +/** + * CLIService. + * + */ +public class CLIService extends CompositeService implements ICLIService { + + public static final TProtocolVersion SERVER_VERSION; + + static { + TProtocolVersion[] protocols = TProtocolVersion.values(); + SERVER_VERSION = protocols[protocols.length - 1]; + } + + private final Log LOG = LogFactory.getLog(CLIService.class.getName()); + + private HiveConf hiveConf; + private SessionManager sessionManager; + private UserGroupInformation serviceUGI; + private UserGroupInformation httpUGI; + // The HiveServer2 instance running this service + private final HiveServer2 hiveServer2; + + public CLIService(HiveServer2 hiveServer2) { + super(CLIService.class.getSimpleName()); + this.hiveServer2 = hiveServer2; + } + + @Override + public synchronized void init(HiveConf hiveConf) { + this.hiveConf = hiveConf; + sessionManager = new SessionManager(hiveServer2); + addService(sessionManager); + // If the hadoop cluster is secure, do a kerberos login for the service from the keytab + if (UserGroupInformation.isSecurityEnabled()) { + try { + HiveAuthFactory.loginFromKeytab(hiveConf); + this.serviceUGI = Utils.getUGI(); + } catch (IOException e) { + throw new ServiceException("Unable to login to kerberos with given principal/keytab", e); + } catch (LoginException e) { + throw new ServiceException("Unable to login to kerberos with given principal/keytab", e); + } + + // Also try creating a UGI object for the SPNego principal + String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_PRINCIPAL); + String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_KEYTAB); + if (principal.isEmpty() || keyTabFile.isEmpty()) { + LOG.info("SPNego httpUGI not created, spNegoPrincipal: " + principal + + ", ketabFile: " + keyTabFile); + } else { + try { + this.httpUGI = HiveAuthFactory.loginFromSpnegoKeytabAndReturnUGI(hiveConf); + LOG.info("SPNego httpUGI successfully created."); + } catch (IOException e) { + LOG.warn("SPNego httpUGI creation failed: ", e); + } + } + } + // creates connection to HMS and thus *must* occur after kerberos login above + try { + applyAuthorizationConfigPolicy(hiveConf); + } catch (Exception e) { + throw new RuntimeException("Error applying authorization policy on hive configuration: " + + e.getMessage(), e); + } + setupBlockedUdfs(); + super.init(hiveConf); + } + + private void applyAuthorizationConfigPolicy(HiveConf newHiveConf) throws HiveException, + MetaException { + // authorization setup using SessionState should be revisited eventually, as + // authorization and authentication are not session specific settings + SessionState ss = new SessionState(newHiveConf); + ss.setIsHiveServerQuery(true); + SessionState.start(ss); + ss.applyAuthorizationPolicy(); + } + + private void setupBlockedUdfs() { + FunctionRegistry.setupPermissionsForBuiltinUDFs( + hiveConf.getVar(ConfVars.HIVE_SERVER2_BUILTIN_UDF_WHITELIST), + hiveConf.getVar(ConfVars.HIVE_SERVER2_BUILTIN_UDF_BLACKLIST)); + } + + public UserGroupInformation getServiceUGI() { + return this.serviceUGI; + } + + public UserGroupInformation getHttpUGI() { + return this.httpUGI; + } + + @Override + public synchronized void start() { + super.start(); + // Initialize and test a connection to the metastore + IMetaStoreClient metastoreClient = null; + try { + metastoreClient = new HiveMetaStoreClient(hiveConf); + metastoreClient.getDatabases("default"); + } catch (Exception e) { + throw new ServiceException("Unable to connect to MetaStore!", e); + } + finally { + if (metastoreClient != null) { + metastoreClient.close(); + } + } + } + + @Override + public synchronized void stop() { + super.stop(); + } + + /** + * @deprecated Use {@link #openSession(TProtocolVersion, String, String, String, Map)} + */ + @Deprecated + public SessionHandle openSession(TProtocolVersion protocol, String username, String password, + Map configuration) throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, null, configuration, false, null); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + /** + * @deprecated Use {@link #openSessionWithImpersonation(TProtocolVersion, String, String, String, Map, String)} + */ + @Deprecated + public SessionHandle openSessionWithImpersonation(TProtocolVersion protocol, String username, + String password, Map configuration, String delegationToken) + throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, null, configuration, + true, delegationToken); + LOG.debug(sessionHandle + ": openSessionWithImpersonation()"); + return sessionHandle; + } + + public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, + Map configuration) throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, ipAddress, configuration, false, null); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + public SessionHandle openSessionWithImpersonation(TProtocolVersion protocol, String username, + String password, String ipAddress, Map configuration, String delegationToken) + throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, ipAddress, configuration, + true, delegationToken); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) + */ + @Override + public SessionHandle openSession(String username, String password, Map configuration) + throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(SERVER_VERSION, username, password, null, configuration, false, null); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) + */ + @Override + public SessionHandle openSessionWithImpersonation(String username, String password, Map configuration, + String delegationToken) throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(SERVER_VERSION, username, password, null, configuration, + true, delegationToken); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public void closeSession(SessionHandle sessionHandle) + throws HiveSQLException { + sessionManager.closeSession(sessionHandle); + LOG.debug(sessionHandle + ": closeSession()"); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List) + */ + @Override + public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType getInfoType) + throws HiveSQLException { + GetInfoValue infoValue = sessionManager.getSession(sessionHandle) + .getInfo(getInfoType); + LOG.debug(sessionHandle + ": getInfo()"); + return infoValue; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#executeStatement(org.apache.hive.service.cli.SessionHandle, + * java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, + Map confOverlay) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .executeStatement(statement, confOverlay); + LOG.debug(sessionHandle + ": executeStatement()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#executeStatementAsync(org.apache.hive.service.cli.SessionHandle, + * java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, + Map confOverlay) throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .executeStatementAsync(statement, confOverlay); + LOG.debug(sessionHandle + ": executeStatementAsync()"); + return opHandle; + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTypeInfo(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTypeInfo(SessionHandle sessionHandle) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getTypeInfo(); + LOG.debug(sessionHandle + ": getTypeInfo()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getCatalogs(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getCatalogs(SessionHandle sessionHandle) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getCatalogs(); + LOG.debug(sessionHandle + ": getCatalogs()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String) + */ + @Override + public OperationHandle getSchemas(SessionHandle sessionHandle, + String catalogName, String schemaName) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getSchemas(catalogName, schemaName); + LOG.debug(sessionHandle + ": getSchemas()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List) + */ + @Override + public OperationHandle getTables(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, List tableTypes) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getTables(catalogName, schemaName, tableName, tableTypes); + LOG.debug(sessionHandle + ": getTables()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTableTypes(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTableTypes(SessionHandle sessionHandle) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getTableTypes(); + LOG.debug(sessionHandle + ": getTableTypes()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getColumns(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getColumns(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, String columnName) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getColumns(catalogName, schemaName, tableName, columnName); + LOG.debug(sessionHandle + ": getColumns()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getFunctions(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getFunctions(SessionHandle sessionHandle, + String catalogName, String schemaName, String functionName) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getFunctions(catalogName, schemaName, functionName); + LOG.debug(sessionHandle + ": getFunctions()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getOperationStatus(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public OperationStatus getOperationStatus(OperationHandle opHandle) + throws HiveSQLException { + Operation operation = sessionManager.getOperationManager().getOperation(opHandle); + /** + * If this is a background operation run asynchronously, + * we block for a configured duration, before we return + * (duration: HIVE_SERVER2_LONG_POLLING_TIMEOUT). + * However, if the background operation is complete, we return immediately. + */ + if (operation.shouldRunAsync()) { + HiveConf conf = operation.getParentSession().getHiveConf(); + long timeout = HiveConf.getTimeVar(conf, + HiveConf.ConfVars.HIVE_SERVER2_LONG_POLLING_TIMEOUT, TimeUnit.MILLISECONDS); + try { + operation.getBackgroundHandle().get(timeout, TimeUnit.MILLISECONDS); + } catch (TimeoutException e) { + // No Op, return to the caller since long polling timeout has expired + LOG.trace(opHandle + ": Long polling timed out"); + } catch (CancellationException e) { + // The background operation thread was cancelled + LOG.trace(opHandle + ": The background operation was cancelled", e); + } catch (ExecutionException e) { + // The background operation thread was aborted + LOG.warn(opHandle + ": The background operation was aborted", e); + } catch (InterruptedException e) { + // No op, this thread was interrupted + // In this case, the call might return sooner than long polling timeout + } + } + OperationStatus opStatus = operation.getStatus(); + LOG.debug(opHandle + ": getOperationStatus()"); + return opStatus; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#cancelOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void cancelOperation(OperationHandle opHandle) + throws HiveSQLException { + sessionManager.getOperationManager().getOperation(opHandle) + .getParentSession().cancelOperation(opHandle); + LOG.debug(opHandle + ": cancelOperation()"); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void closeOperation(OperationHandle opHandle) + throws HiveSQLException { + sessionManager.getOperationManager().getOperation(opHandle) + .getParentSession().closeOperation(opHandle); + LOG.debug(opHandle + ": closeOperation"); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public TableSchema getResultSetMetadata(OperationHandle opHandle) + throws HiveSQLException { + TableSchema tableSchema = sessionManager.getOperationManager() + .getOperation(opHandle).getParentSession().getResultSetMetadata(opHandle); + LOG.debug(opHandle + ": getResultSetMetadata()"); + return tableSchema; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#fetchResults(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public RowSet fetchResults(OperationHandle opHandle) + throws HiveSQLException { + return fetchResults(opHandle, Operation.DEFAULT_FETCH_ORIENTATION, + Operation.DEFAULT_FETCH_MAX_ROWS, FetchType.QUERY_OUTPUT); + } + + @Override + public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, + long maxRows, FetchType fetchType) throws HiveSQLException { + RowSet rowSet = sessionManager.getOperationManager().getOperation(opHandle) + .getParentSession().fetchResults(opHandle, orientation, maxRows, fetchType); + LOG.debug(opHandle + ": fetchResults()"); + return rowSet; + } + + // obtain delegation token for the give user from metastore + public synchronized String getDelegationTokenFromMetaStore(String owner) + throws HiveSQLException, UnsupportedOperationException, LoginException, IOException { + if (!hiveConf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL) || + !hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS)) { + throw new UnsupportedOperationException( + "delegation token is can only be obtained for a secure remote metastore"); + } + + try { + Hive.closeCurrent(); + return Hive.get(hiveConf).getDelegationToken(owner, owner); + } catch (HiveException e) { + if (e.getCause() instanceof UnsupportedOperationException) { + throw (UnsupportedOperationException)e.getCause(); + } else { + throw new HiveSQLException("Error connect metastore to setup impersonation", e); + } + } + } + + @Override + public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String owner, String renewer) throws HiveSQLException { + String delegationToken = sessionManager.getSession(sessionHandle) + .getDelegationToken(authFactory, owner, renewer); + LOG.info(sessionHandle + ": getDelegationToken()"); + return delegationToken; + } + + @Override + public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + sessionManager.getSession(sessionHandle).cancelDelegationToken(authFactory, tokenStr); + LOG.info(sessionHandle + ": cancelDelegationToken()"); + } + + @Override + public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + sessionManager.getSession(sessionHandle).renewDelegationToken(authFactory, tokenStr); + LOG.info(sessionHandle + ": renewDelegationToken()"); + } + + public SessionManager getSessionManager() { + return sessionManager; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceClient.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceClient.java new file mode 100644 index 000000000000..3155c238ff68 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceClient.java @@ -0,0 +1,56 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.Collections; + +import org.apache.hive.service.auth.HiveAuthFactory; + + +/** + * CLIServiceClient. + * + */ +public abstract class CLIServiceClient implements ICLIService { + private static final long DEFAULT_MAX_ROWS = 1000; + + public SessionHandle openSession(String username, String password) + throws HiveSQLException { + return openSession(username, password, Collections.emptyMap()); + } + + @Override + public RowSet fetchResults(OperationHandle opHandle) throws HiveSQLException { + // TODO: provide STATIC default value + return fetchResults(opHandle, FetchOrientation.FETCH_NEXT, DEFAULT_MAX_ROWS, FetchType.QUERY_OUTPUT); + } + + @Override + public abstract String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String owner, String renewer) throws HiveSQLException; + + @Override + public abstract void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException; + + @Override + public abstract void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException; + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceUtils.java new file mode 100644 index 000000000000..bf2380632fa6 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIServiceUtils.java @@ -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.hive.service.cli; + +import org.apache.log4j.Layout; +import org.apache.log4j.PatternLayout; + +/** + * CLIServiceUtils. + * + */ +public class CLIServiceUtils { + + + private static final char SEARCH_STRING_ESCAPE = '\\'; + public static final Layout verboseLayout = new PatternLayout( + "%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n"); + public static final Layout nonVerboseLayout = new PatternLayout( + "%-5p : %m%n"); + + /** + * Convert a SQL search pattern into an equivalent Java Regex. + * + * @param pattern input which may contain '%' or '_' wildcard characters, or + * these characters escaped using {@code getSearchStringEscape()}. + * @return replace %/_ with regex search characters, also handle escaped + * characters. + */ + public static String patternToRegex(String pattern) { + if (pattern == null) { + return ".*"; + } else { + StringBuilder result = new StringBuilder(pattern.length()); + + boolean escaped = false; + for (int i = 0, len = pattern.length(); i < len; i++) { + char c = pattern.charAt(i); + if (escaped) { + if (c != SEARCH_STRING_ESCAPE) { + escaped = false; + } + result.append(c); + } else { + if (c == SEARCH_STRING_ESCAPE) { + escaped = true; + continue; + } else if (c == '%') { + result.append(".*"); + } else if (c == '_') { + result.append('.'); + } else { + result.append(Character.toLowerCase(c)); + } + } + } + return result.toString(); + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Column.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Column.java new file mode 100644 index 000000000000..2e21f18d6126 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Column.java @@ -0,0 +1,423 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.nio.ByteBuffer; +import java.util.AbstractList; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.BitSet; +import java.util.List; + +import com.google.common.primitives.Booleans; +import com.google.common.primitives.Bytes; +import com.google.common.primitives.Doubles; +import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; +import com.google.common.primitives.Shorts; +import org.apache.hive.service.cli.thrift.TBinaryColumn; +import org.apache.hive.service.cli.thrift.TBoolColumn; +import org.apache.hive.service.cli.thrift.TByteColumn; +import org.apache.hive.service.cli.thrift.TColumn; +import org.apache.hive.service.cli.thrift.TDoubleColumn; +import org.apache.hive.service.cli.thrift.TI16Column; +import org.apache.hive.service.cli.thrift.TI32Column; +import org.apache.hive.service.cli.thrift.TI64Column; +import org.apache.hive.service.cli.thrift.TStringColumn; + +/** + * Column. + */ +public class Column extends AbstractList { + + private static final int DEFAULT_SIZE = 100; + + private final Type type; + + private BitSet nulls; + + private int size; + private boolean[] boolVars; + private byte[] byteVars; + private short[] shortVars; + private int[] intVars; + private long[] longVars; + private double[] doubleVars; + private List stringVars; + private List binaryVars; + + public Column(Type type, BitSet nulls, Object values) { + this.type = type; + this.nulls = nulls; + if (type == Type.BOOLEAN_TYPE) { + boolVars = (boolean[]) values; + size = boolVars.length; + } else if (type == Type.TINYINT_TYPE) { + byteVars = (byte[]) values; + size = byteVars.length; + } else if (type == Type.SMALLINT_TYPE) { + shortVars = (short[]) values; + size = shortVars.length; + } else if (type == Type.INT_TYPE) { + intVars = (int[]) values; + size = intVars.length; + } else if (type == Type.BIGINT_TYPE) { + longVars = (long[]) values; + size = longVars.length; + } else if (type == Type.DOUBLE_TYPE) { + doubleVars = (double[]) values; + size = doubleVars.length; + } else if (type == Type.BINARY_TYPE) { + binaryVars = (List) values; + size = binaryVars.size(); + } else if (type == Type.STRING_TYPE) { + stringVars = (List) values; + size = stringVars.size(); + } else { + throw new IllegalStateException("invalid union object"); + } + } + + public Column(Type type) { + nulls = new BitSet(); + switch (type) { + case BOOLEAN_TYPE: + boolVars = new boolean[DEFAULT_SIZE]; + break; + case TINYINT_TYPE: + byteVars = new byte[DEFAULT_SIZE]; + break; + case SMALLINT_TYPE: + shortVars = new short[DEFAULT_SIZE]; + break; + case INT_TYPE: + intVars = new int[DEFAULT_SIZE]; + break; + case BIGINT_TYPE: + longVars = new long[DEFAULT_SIZE]; + break; + case FLOAT_TYPE: + case DOUBLE_TYPE: + type = Type.DOUBLE_TYPE; + doubleVars = new double[DEFAULT_SIZE]; + break; + case BINARY_TYPE: + binaryVars = new ArrayList(); + break; + default: + type = Type.STRING_TYPE; + stringVars = new ArrayList(); + } + this.type = type; + } + + public Column(TColumn colValues) { + if (colValues.isSetBoolVal()) { + type = Type.BOOLEAN_TYPE; + nulls = toBitset(colValues.getBoolVal().getNulls()); + boolVars = Booleans.toArray(colValues.getBoolVal().getValues()); + size = boolVars.length; + } else if (colValues.isSetByteVal()) { + type = Type.TINYINT_TYPE; + nulls = toBitset(colValues.getByteVal().getNulls()); + byteVars = Bytes.toArray(colValues.getByteVal().getValues()); + size = byteVars.length; + } else if (colValues.isSetI16Val()) { + type = Type.SMALLINT_TYPE; + nulls = toBitset(colValues.getI16Val().getNulls()); + shortVars = Shorts.toArray(colValues.getI16Val().getValues()); + size = shortVars.length; + } else if (colValues.isSetI32Val()) { + type = Type.INT_TYPE; + nulls = toBitset(colValues.getI32Val().getNulls()); + intVars = Ints.toArray(colValues.getI32Val().getValues()); + size = intVars.length; + } else if (colValues.isSetI64Val()) { + type = Type.BIGINT_TYPE; + nulls = toBitset(colValues.getI64Val().getNulls()); + longVars = Longs.toArray(colValues.getI64Val().getValues()); + size = longVars.length; + } else if (colValues.isSetDoubleVal()) { + type = Type.DOUBLE_TYPE; + nulls = toBitset(colValues.getDoubleVal().getNulls()); + doubleVars = Doubles.toArray(colValues.getDoubleVal().getValues()); + size = doubleVars.length; + } else if (colValues.isSetBinaryVal()) { + type = Type.BINARY_TYPE; + nulls = toBitset(colValues.getBinaryVal().getNulls()); + binaryVars = colValues.getBinaryVal().getValues(); + size = binaryVars.size(); + } else if (colValues.isSetStringVal()) { + type = Type.STRING_TYPE; + nulls = toBitset(colValues.getStringVal().getNulls()); + stringVars = colValues.getStringVal().getValues(); + size = stringVars.size(); + } else { + throw new IllegalStateException("invalid union object"); + } + } + + public Column extractSubset(int start, int end) { + BitSet subNulls = nulls.get(start, end); + if (type == Type.BOOLEAN_TYPE) { + Column subset = new Column(type, subNulls, Arrays.copyOfRange(boolVars, start, end)); + boolVars = Arrays.copyOfRange(boolVars, end, size); + nulls = nulls.get(start, size); + size = boolVars.length; + return subset; + } + if (type == Type.TINYINT_TYPE) { + Column subset = new Column(type, subNulls, Arrays.copyOfRange(byteVars, start, end)); + byteVars = Arrays.copyOfRange(byteVars, end, size); + nulls = nulls.get(start, size); + size = byteVars.length; + return subset; + } + if (type == Type.SMALLINT_TYPE) { + Column subset = new Column(type, subNulls, Arrays.copyOfRange(shortVars, start, end)); + shortVars = Arrays.copyOfRange(shortVars, end, size); + nulls = nulls.get(start, size); + size = shortVars.length; + return subset; + } + if (type == Type.INT_TYPE) { + Column subset = new Column(type, subNulls, Arrays.copyOfRange(intVars, start, end)); + intVars = Arrays.copyOfRange(intVars, end, size); + nulls = nulls.get(start, size); + size = intVars.length; + return subset; + } + if (type == Type.BIGINT_TYPE) { + Column subset = new Column(type, subNulls, Arrays.copyOfRange(longVars, start, end)); + longVars = Arrays.copyOfRange(longVars, end, size); + nulls = nulls.get(start, size); + size = longVars.length; + return subset; + } + if (type == Type.DOUBLE_TYPE) { + Column subset = new Column(type, subNulls, Arrays.copyOfRange(doubleVars, start, end)); + doubleVars = Arrays.copyOfRange(doubleVars, end, size); + nulls = nulls.get(start, size); + size = doubleVars.length; + return subset; + } + if (type == Type.BINARY_TYPE) { + Column subset = new Column(type, subNulls, binaryVars.subList(start, end)); + binaryVars = binaryVars.subList(end, binaryVars.size()); + nulls = nulls.get(start, size); + size = binaryVars.size(); + return subset; + } + if (type == Type.STRING_TYPE) { + Column subset = new Column(type, subNulls, stringVars.subList(start, end)); + stringVars = stringVars.subList(end, stringVars.size()); + nulls = nulls.get(start, size); + size = stringVars.size(); + return subset; + } + throw new IllegalStateException("invalid union object"); + } + + private static final byte[] MASKS = new byte[] { + 0x01, 0x02, 0x04, 0x08, 0x10, 0x20, 0x40, (byte)0x80 + }; + + private static BitSet toBitset(byte[] nulls) { + BitSet bitset = new BitSet(); + int bits = nulls.length * 8; + for (int i = 0; i < bits; i++) { + bitset.set(i, (nulls[i / 8] & MASKS[i % 8]) != 0); + } + return bitset; + } + + private static byte[] toBinary(BitSet bitset) { + byte[] nulls = new byte[1 + (bitset.length() / 8)]; + for (int i = 0; i < bitset.length(); i++) { + nulls[i / 8] |= bitset.get(i) ? MASKS[i % 8] : 0; + } + return nulls; + } + + public Type getType() { + return type; + } + + @Override + public Object get(int index) { + if (nulls.get(index)) { + return null; + } + switch (type) { + case BOOLEAN_TYPE: + return boolVars[index]; + case TINYINT_TYPE: + return byteVars[index]; + case SMALLINT_TYPE: + return shortVars[index]; + case INT_TYPE: + return intVars[index]; + case BIGINT_TYPE: + return longVars[index]; + case DOUBLE_TYPE: + return doubleVars[index]; + case STRING_TYPE: + return stringVars.get(index); + case BINARY_TYPE: + return binaryVars.get(index).array(); + } + return null; + } + + @Override + public int size() { + return size; + } + + public TColumn toTColumn() { + TColumn value = new TColumn(); + ByteBuffer nullMasks = ByteBuffer.wrap(toBinary(nulls)); + switch (type) { + case BOOLEAN_TYPE: + value.setBoolVal(new TBoolColumn(Booleans.asList(Arrays.copyOfRange(boolVars, 0, size)), nullMasks)); + break; + case TINYINT_TYPE: + value.setByteVal(new TByteColumn(Bytes.asList(Arrays.copyOfRange(byteVars, 0, size)), nullMasks)); + break; + case SMALLINT_TYPE: + value.setI16Val(new TI16Column(Shorts.asList(Arrays.copyOfRange(shortVars, 0, size)), nullMasks)); + break; + case INT_TYPE: + value.setI32Val(new TI32Column(Ints.asList(Arrays.copyOfRange(intVars, 0, size)), nullMasks)); + break; + case BIGINT_TYPE: + value.setI64Val(new TI64Column(Longs.asList(Arrays.copyOfRange(longVars, 0, size)), nullMasks)); + break; + case DOUBLE_TYPE: + value.setDoubleVal(new TDoubleColumn(Doubles.asList(Arrays.copyOfRange(doubleVars, 0, size)), nullMasks)); + break; + case STRING_TYPE: + value.setStringVal(new TStringColumn(stringVars, nullMasks)); + break; + case BINARY_TYPE: + value.setBinaryVal(new TBinaryColumn(binaryVars, nullMasks)); + break; + } + return value; + } + + private static final ByteBuffer EMPTY_BINARY = ByteBuffer.allocate(0); + private static final String EMPTY_STRING = ""; + + public void addValue(Type type, Object field) { + switch (type) { + case BOOLEAN_TYPE: + nulls.set(size, field == null); + boolVars()[size] = field == null ? true : (Boolean)field; + break; + case TINYINT_TYPE: + nulls.set(size, field == null); + byteVars()[size] = field == null ? 0 : (Byte) field; + break; + case SMALLINT_TYPE: + nulls.set(size, field == null); + shortVars()[size] = field == null ? 0 : (Short)field; + break; + case INT_TYPE: + nulls.set(size, field == null); + intVars()[size] = field == null ? 0 : (Integer)field; + break; + case BIGINT_TYPE: + nulls.set(size, field == null); + longVars()[size] = field == null ? 0 : (Long)field; + break; + case FLOAT_TYPE: + nulls.set(size, field == null); + doubleVars()[size] = field == null ? 0 : ((Float)field).doubleValue(); + break; + case DOUBLE_TYPE: + nulls.set(size, field == null); + doubleVars()[size] = field == null ? 0 : (Double)field; + break; + case BINARY_TYPE: + nulls.set(binaryVars.size(), field == null); + binaryVars.add(field == null ? EMPTY_BINARY : ByteBuffer.wrap((byte[])field)); + break; + default: + nulls.set(stringVars.size(), field == null); + stringVars.add(field == null ? EMPTY_STRING : String.valueOf(field)); + break; + } + size++; + } + + private boolean[] boolVars() { + if (boolVars.length == size) { + boolean[] newVars = new boolean[size << 1]; + System.arraycopy(boolVars, 0, newVars, 0, size); + return boolVars = newVars; + } + return boolVars; + } + + private byte[] byteVars() { + if (byteVars.length == size) { + byte[] newVars = new byte[size << 1]; + System.arraycopy(byteVars, 0, newVars, 0, size); + return byteVars = newVars; + } + return byteVars; + } + + private short[] shortVars() { + if (shortVars.length == size) { + short[] newVars = new short[size << 1]; + System.arraycopy(shortVars, 0, newVars, 0, size); + return shortVars = newVars; + } + return shortVars; + } + + private int[] intVars() { + if (intVars.length == size) { + int[] newVars = new int[size << 1]; + System.arraycopy(intVars, 0, newVars, 0, size); + return intVars = newVars; + } + return intVars; + } + + private long[] longVars() { + if (longVars.length == size) { + long[] newVars = new long[size << 1]; + System.arraycopy(longVars, 0, newVars, 0, size); + return longVars = newVars; + } + return longVars; + } + + private double[] doubleVars() { + if (doubleVars.length == size) { + double[] newVars = new double[size << 1]; + System.arraycopy(doubleVars, 0, newVars, 0, size); + return doubleVars = newVars; + } + return doubleVars; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java new file mode 100644 index 000000000000..47a582e2223e --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java @@ -0,0 +1,149 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.hive.service.cli.thrift.TColumn; +import org.apache.hive.service.cli.thrift.TRow; +import org.apache.hive.service.cli.thrift.TRowSet; + +/** + * ColumnBasedSet. + */ +public class ColumnBasedSet implements RowSet { + + private long startOffset; + + private final Type[] types; // non-null only for writing (server-side) + private final List columns; + + public ColumnBasedSet(TableSchema schema) { + types = schema.toTypes(); + columns = new ArrayList(); + for (ColumnDescriptor colDesc : schema.getColumnDescriptors()) { + columns.add(new Column(colDesc.getType())); + } + } + + public ColumnBasedSet(TRowSet tRowSet) { + types = null; + columns = new ArrayList(); + for (TColumn tvalue : tRowSet.getColumns()) { + columns.add(new Column(tvalue)); + } + startOffset = tRowSet.getStartRowOffset(); + } + + private ColumnBasedSet(Type[] types, List columns, long startOffset) { + this.types = types; + this.columns = columns; + this.startOffset = startOffset; + } + + @Override + public ColumnBasedSet addRow(Object[] fields) { + for (int i = 0; i < fields.length; i++) { + columns.get(i).addValue(types[i], fields[i]); + } + return this; + } + + public List getColumns() { + return columns; + } + + @Override + public int numColumns() { + return columns.size(); + } + + @Override + public int numRows() { + return columns.isEmpty() ? 0 : columns.get(0).size(); + } + + @Override + public ColumnBasedSet extractSubset(int maxRows) { + int numRows = Math.min(numRows(), maxRows); + + List subset = new ArrayList(); + for (int i = 0; i < columns.size(); i++) { + subset.add(columns.get(i).extractSubset(0, numRows)); + } + ColumnBasedSet result = new ColumnBasedSet(types, subset, startOffset); + startOffset += numRows; + return result; + } + + @Override + public long getStartOffset() { + return startOffset; + } + + @Override + public void setStartOffset(long startOffset) { + this.startOffset = startOffset; + } + + public TRowSet toTRowSet() { + TRowSet tRowSet = new TRowSet(startOffset, new ArrayList()); + for (int i = 0; i < columns.size(); i++) { + tRowSet.addToColumns(columns.get(i).toTColumn()); + } + return tRowSet; + } + + @Override + public Iterator iterator() { + return new Iterator() { + + private int index; + private final Object[] convey = new Object[numColumns()]; + + @Override + public boolean hasNext() { + return index < numRows(); + } + + @Override + public Object[] next() { + for (int i = 0; i < columns.size(); i++) { + convey[i] = columns.get(i).get(index); + } + index++; + return convey; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("remove"); + } + }; + } + + public Object[] fill(int index, Object[] convey) { + for (int i = 0; i < columns.size(); i++) { + convey[i] = columns.get(i).get(index); + } + return convey; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java new file mode 100644 index 000000000000..f0bbf1469316 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java @@ -0,0 +1,99 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hive.service.cli.thrift.TColumnDesc; + + +/** + * ColumnDescriptor. + * + */ +public class ColumnDescriptor { + private final String name; + private final String comment; + private final TypeDescriptor type; + // ordinal position of this column in the schema + private final int position; + + public ColumnDescriptor(String name, String comment, TypeDescriptor type, int position) { + this.name = name; + this.comment = comment; + this.type = type; + this.position = position; + } + + public ColumnDescriptor(TColumnDesc tColumnDesc) { + name = tColumnDesc.getColumnName(); + comment = tColumnDesc.getComment(); + type = new TypeDescriptor(tColumnDesc.getTypeDesc()); + position = tColumnDesc.getPosition(); + } + + public ColumnDescriptor(FieldSchema column, int position) { + name = column.getName(); + comment = column.getComment(); + type = new TypeDescriptor(column.getType()); + this.position = position; + } + + public static ColumnDescriptor newPrimitiveColumnDescriptor(String name, String comment, Type type, int position) { + // Current usage looks like it's only for metadata columns, but if that changes then + // this method may need to require a type qualifiers aruments. + return new ColumnDescriptor(name, comment, new TypeDescriptor(type), position); + } + + public String getName() { + return name; + } + + public String getComment() { + return comment; + } + + public TypeDescriptor getTypeDescriptor() { + return type; + } + + public int getOrdinalPosition() { + return position; + } + + public TColumnDesc toTColumnDesc() { + TColumnDesc tColumnDesc = new TColumnDesc(); + tColumnDesc.setColumnName(name); + tColumnDesc.setComment(comment); + tColumnDesc.setTypeDesc(type.toTTypeDesc()); + tColumnDesc.setPosition(position); + return tColumnDesc; + } + + public Type getType() { + return type.getType(); + } + + public boolean isPrimitive() { + return type.getType().isPrimitiveType(); + } + + public String getTypeName() { + return type.getTypeName(); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnValue.java new file mode 100644 index 000000000000..40144cfe33fa --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnValue.java @@ -0,0 +1,307 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; + +import org.apache.hadoop.hive.common.type.HiveChar; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; +import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; +import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hive.service.cli.thrift.TBoolValue; +import org.apache.hive.service.cli.thrift.TByteValue; +import org.apache.hive.service.cli.thrift.TColumnValue; +import org.apache.hive.service.cli.thrift.TDoubleValue; +import org.apache.hive.service.cli.thrift.TI16Value; +import org.apache.hive.service.cli.thrift.TI32Value; +import org.apache.hive.service.cli.thrift.TI64Value; +import org.apache.hive.service.cli.thrift.TStringValue; + +/** + * Protocols before HIVE_CLI_SERVICE_PROTOCOL_V6 (used by RowBasedSet) + * + */ +public class ColumnValue { + + private static TColumnValue booleanValue(Boolean value) { + TBoolValue tBoolValue = new TBoolValue(); + if (value != null) { + tBoolValue.setValue(value); + } + return TColumnValue.boolVal(tBoolValue); + } + + private static TColumnValue byteValue(Byte value) { + TByteValue tByteValue = new TByteValue(); + if (value != null) { + tByteValue.setValue(value); + } + return TColumnValue.byteVal(tByteValue); + } + + private static TColumnValue shortValue(Short value) { + TI16Value tI16Value = new TI16Value(); + if (value != null) { + tI16Value.setValue(value); + } + return TColumnValue.i16Val(tI16Value); + } + + private static TColumnValue intValue(Integer value) { + TI32Value tI32Value = new TI32Value(); + if (value != null) { + tI32Value.setValue(value); + } + return TColumnValue.i32Val(tI32Value); + } + + private static TColumnValue longValue(Long value) { + TI64Value tI64Value = new TI64Value(); + if (value != null) { + tI64Value.setValue(value); + } + return TColumnValue.i64Val(tI64Value); + } + + private static TColumnValue floatValue(Float value) { + TDoubleValue tDoubleValue = new TDoubleValue(); + if (value != null) { + tDoubleValue.setValue(value); + } + return TColumnValue.doubleVal(tDoubleValue); + } + + private static TColumnValue doubleValue(Double value) { + TDoubleValue tDoubleValue = new TDoubleValue(); + if (value != null) { + tDoubleValue.setValue(value); + } + return TColumnValue.doubleVal(tDoubleValue); + } + + private static TColumnValue stringValue(String value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value); + } + return TColumnValue.stringVal(tStringValue); + } + + private static TColumnValue stringValue(HiveChar value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStringValue); + } + + private static TColumnValue stringValue(HiveVarchar value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStringValue); + } + + private static TColumnValue dateValue(Date value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value.toString()); + } + return new TColumnValue(TColumnValue.stringVal(tStringValue)); + } + + private static TColumnValue timestampValue(Timestamp value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStringValue); + } + + private static TColumnValue stringValue(HiveDecimal value) { + TStringValue tStrValue = new TStringValue(); + if (value != null) { + tStrValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStrValue); + } + + private static TColumnValue stringValue(HiveIntervalYearMonth value) { + TStringValue tStrValue = new TStringValue(); + if (value != null) { + tStrValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStrValue); + } + + private static TColumnValue stringValue(HiveIntervalDayTime value) { + TStringValue tStrValue = new TStringValue(); + if (value != null) { + tStrValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStrValue); + } + + public static TColumnValue toTColumnValue(Type type, Object value) { + switch (type) { + case BOOLEAN_TYPE: + return booleanValue((Boolean)value); + case TINYINT_TYPE: + return byteValue((Byte)value); + case SMALLINT_TYPE: + return shortValue((Short)value); + case INT_TYPE: + return intValue((Integer)value); + case BIGINT_TYPE: + return longValue((Long)value); + case FLOAT_TYPE: + return floatValue((Float)value); + case DOUBLE_TYPE: + return doubleValue((Double)value); + case STRING_TYPE: + return stringValue((String)value); + case CHAR_TYPE: + return stringValue((HiveChar)value); + case VARCHAR_TYPE: + return stringValue((HiveVarchar)value); + case DATE_TYPE: + return dateValue((Date)value); + case TIMESTAMP_TYPE: + return timestampValue((Timestamp)value); + case INTERVAL_YEAR_MONTH_TYPE: + return stringValue((HiveIntervalYearMonth) value); + case INTERVAL_DAY_TIME_TYPE: + return stringValue((HiveIntervalDayTime) value); + case DECIMAL_TYPE: + return stringValue(((HiveDecimal)value)); + case BINARY_TYPE: + return stringValue((String)value); + case ARRAY_TYPE: + case MAP_TYPE: + case STRUCT_TYPE: + case UNION_TYPE: + case USER_DEFINED_TYPE: + return stringValue((String)value); + default: + return null; + } + } + + private static Boolean getBooleanValue(TBoolValue tBoolValue) { + if (tBoolValue.isSetValue()) { + return tBoolValue.isValue(); + } + return null; + } + + private static Byte getByteValue(TByteValue tByteValue) { + if (tByteValue.isSetValue()) { + return tByteValue.getValue(); + } + return null; + } + + private static Short getShortValue(TI16Value tI16Value) { + if (tI16Value.isSetValue()) { + return tI16Value.getValue(); + } + return null; + } + + private static Integer getIntegerValue(TI32Value tI32Value) { + if (tI32Value.isSetValue()) { + return tI32Value.getValue(); + } + return null; + } + + private static Long getLongValue(TI64Value tI64Value) { + if (tI64Value.isSetValue()) { + return tI64Value.getValue(); + } + return null; + } + + private static Double getDoubleValue(TDoubleValue tDoubleValue) { + if (tDoubleValue.isSetValue()) { + return tDoubleValue.getValue(); + } + return null; + } + + private static String getStringValue(TStringValue tStringValue) { + if (tStringValue.isSetValue()) { + return tStringValue.getValue(); + } + return null; + } + + private static Date getDateValue(TStringValue tStringValue) { + if (tStringValue.isSetValue()) { + return Date.valueOf(tStringValue.getValue()); + } + return null; + } + + private static Timestamp getTimestampValue(TStringValue tStringValue) { + if (tStringValue.isSetValue()) { + return Timestamp.valueOf(tStringValue.getValue()); + } + return null; + } + + private static byte[] getBinaryValue(TStringValue tString) { + if (tString.isSetValue()) { + return tString.getValue().getBytes(); + } + return null; + } + + private static BigDecimal getBigDecimalValue(TStringValue tStringValue) { + if (tStringValue.isSetValue()) { + return new BigDecimal(tStringValue.getValue()); + } + return null; + } + + public static Object toColumnValue(TColumnValue value) { + TColumnValue._Fields field = value.getSetField(); + switch (field) { + case BOOL_VAL: + return getBooleanValue(value.getBoolVal()); + case BYTE_VAL: + return getByteValue(value.getByteVal()); + case I16_VAL: + return getShortValue(value.getI16Val()); + case I32_VAL: + return getIntegerValue(value.getI32Val()); + case I64_VAL: + return getLongValue(value.getI64Val()); + case DOUBLE_VAL: + return getDoubleValue(value.getDoubleVal()); + case STRING_VAL: + return getStringValue(value.getStringVal()); + } + throw new IllegalArgumentException("never"); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java new file mode 100644 index 000000000000..9cad5be198c0 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java @@ -0,0 +1,208 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.List; +import java.util.Map; + +import org.apache.hive.service.auth.HiveAuthFactory; + + +/** + * EmbeddedCLIServiceClient. + * + */ +public class EmbeddedCLIServiceClient extends CLIServiceClient { + private final ICLIService cliService; + + public EmbeddedCLIServiceClient(ICLIService cliService) { + this.cliService = cliService; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#openSession(java.lang.String, java.lang.String, java.util.Map) + */ + @Override + public SessionHandle openSession(String username, String password, + Map configuration) throws HiveSQLException { + return cliService.openSession(username, password, configuration); + } + + @Override + public SessionHandle openSessionWithImpersonation(String username, String password, + Map configuration, String delegationToken) throws HiveSQLException { + throw new HiveSQLException("Impersonated session is not supported in the embedded mode"); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#closeSession(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public void closeSession(SessionHandle sessionHandle) throws HiveSQLException { + cliService.closeSession(sessionHandle); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List) + */ + @Override + public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType getInfoType) + throws HiveSQLException { + return cliService.getInfo(sessionHandle, getInfoType); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#executeStatement(org.apache.hive.service.cli.SessionHandle, + * java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, + Map confOverlay) throws HiveSQLException { + return cliService.executeStatement(sessionHandle, statement, confOverlay); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#executeStatementAsync(org.apache.hive.service.cli.SessionHandle, + * java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, + Map confOverlay) throws HiveSQLException { + return cliService.executeStatementAsync(sessionHandle, statement, confOverlay); + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getTypeInfo(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTypeInfo(SessionHandle sessionHandle) throws HiveSQLException { + return cliService.getTypeInfo(sessionHandle); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getCatalogs(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getCatalogs(SessionHandle sessionHandle) throws HiveSQLException { + return cliService.getCatalogs(sessionHandle); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String) + */ + @Override + public OperationHandle getSchemas(SessionHandle sessionHandle, String catalogName, + String schemaName) throws HiveSQLException { + return cliService.getSchemas(sessionHandle, catalogName, schemaName); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List) + */ + @Override + public OperationHandle getTables(SessionHandle sessionHandle, String catalogName, + String schemaName, String tableName, List tableTypes) throws HiveSQLException { + return cliService.getTables(sessionHandle, catalogName, schemaName, tableName, tableTypes); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getTableTypes(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTableTypes(SessionHandle sessionHandle) throws HiveSQLException { + return cliService.getTableTypes(sessionHandle); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getColumns(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.lang.String) + */ + @Override + public OperationHandle getColumns(SessionHandle sessionHandle, String catalogName, + String schemaName, String tableName, String columnName) throws HiveSQLException { + return cliService.getColumns(sessionHandle, catalogName, schemaName, tableName, columnName); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getFunctions(org.apache.hive.service.cli.SessionHandle, java.lang.String) + */ + @Override + public OperationHandle getFunctions(SessionHandle sessionHandle, + String catalogName, String schemaName, String functionName) + throws HiveSQLException { + return cliService.getFunctions(sessionHandle, catalogName, schemaName, functionName); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getOperationStatus(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public OperationStatus getOperationStatus(OperationHandle opHandle) throws HiveSQLException { + return cliService.getOperationStatus(opHandle); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#cancelOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { + cliService.cancelOperation(opHandle); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#closeOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void closeOperation(OperationHandle opHandle) throws HiveSQLException { + cliService.closeOperation(opHandle); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.CLIServiceClient#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public TableSchema getResultSetMetadata(OperationHandle opHandle) throws HiveSQLException { + return cliService.getResultSetMetadata(opHandle); + } + + @Override + public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, + long maxRows, FetchType fetchType) throws HiveSQLException { + return cliService.fetchResults(opHandle, orientation, maxRows, fetchType); + } + + + @Override + public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String owner, String renewer) throws HiveSQLException { + return cliService.getDelegationToken(sessionHandle, authFactory, owner, renewer); + } + + @Override + public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + cliService.cancelDelegationToken(sessionHandle, authFactory, tokenStr); + } + + @Override + public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + cliService.renewDelegationToken(sessionHandle, authFactory, tokenStr); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchOrientation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchOrientation.java new file mode 100644 index 000000000000..ffa6f2e1f374 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchOrientation.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TFetchOrientation; + +/** + * FetchOrientation. + * + */ +public enum FetchOrientation { + FETCH_NEXT(TFetchOrientation.FETCH_NEXT), + FETCH_PRIOR(TFetchOrientation.FETCH_PRIOR), + FETCH_RELATIVE(TFetchOrientation.FETCH_RELATIVE), + FETCH_ABSOLUTE(TFetchOrientation.FETCH_ABSOLUTE), + FETCH_FIRST(TFetchOrientation.FETCH_FIRST), + FETCH_LAST(TFetchOrientation.FETCH_LAST); + + private TFetchOrientation tFetchOrientation; + + FetchOrientation(TFetchOrientation tFetchOrientation) { + this.tFetchOrientation = tFetchOrientation; + } + + public static FetchOrientation getFetchOrientation(TFetchOrientation tFetchOrientation) { + for (FetchOrientation fetchOrientation : values()) { + if (tFetchOrientation.equals(fetchOrientation.toTFetchOrientation())) { + return fetchOrientation; + } + } + // TODO: Should this really default to FETCH_NEXT? + return FETCH_NEXT; + } + + public TFetchOrientation toTFetchOrientation() { + return tFetchOrientation; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchType.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchType.java new file mode 100644 index 000000000000..a8e7fe19b0bc --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchType.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +/** + * FetchType indicates the type of fetchResults request. + * It maps the TFetchType, which is generated from Thrift interface. + */ +public enum FetchType { + QUERY_OUTPUT((short)0), + LOG((short)1); + + private final short tFetchType; + + FetchType(short tFetchType) { + this.tFetchType = tFetchType; + } + + public static FetchType getFetchType(short tFetchType) { + for (FetchType fetchType : values()) { + if (tFetchType == fetchType.toTFetchType()) { + return fetchType; + } + } + return QUERY_OUTPUT; + } + + public short toTFetchType() { + return tFetchType; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoType.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoType.java new file mode 100644 index 000000000000..8dd33a88fdeb --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoType.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TGetInfoType; + +/** + * GetInfoType. + * + */ +public enum GetInfoType { + CLI_MAX_DRIVER_CONNECTIONS(TGetInfoType.CLI_MAX_DRIVER_CONNECTIONS), + CLI_MAX_CONCURRENT_ACTIVITIES(TGetInfoType.CLI_MAX_CONCURRENT_ACTIVITIES), + CLI_DATA_SOURCE_NAME(TGetInfoType.CLI_DATA_SOURCE_NAME), + CLI_FETCH_DIRECTION(TGetInfoType.CLI_FETCH_DIRECTION), + CLI_SERVER_NAME(TGetInfoType.CLI_SERVER_NAME), + CLI_SEARCH_PATTERN_ESCAPE(TGetInfoType.CLI_SEARCH_PATTERN_ESCAPE), + CLI_DBMS_NAME(TGetInfoType.CLI_DBMS_NAME), + CLI_DBMS_VER(TGetInfoType.CLI_DBMS_VER), + CLI_ACCESSIBLE_TABLES(TGetInfoType.CLI_ACCESSIBLE_TABLES), + CLI_ACCESSIBLE_PROCEDURES(TGetInfoType.CLI_ACCESSIBLE_PROCEDURES), + CLI_CURSOR_COMMIT_BEHAVIOR(TGetInfoType.CLI_CURSOR_COMMIT_BEHAVIOR), + CLI_DATA_SOURCE_READ_ONLY(TGetInfoType.CLI_DATA_SOURCE_READ_ONLY), + CLI_DEFAULT_TXN_ISOLATION(TGetInfoType.CLI_DEFAULT_TXN_ISOLATION), + CLI_IDENTIFIER_CASE(TGetInfoType.CLI_IDENTIFIER_CASE), + CLI_IDENTIFIER_QUOTE_CHAR(TGetInfoType.CLI_IDENTIFIER_QUOTE_CHAR), + CLI_MAX_COLUMN_NAME_LEN(TGetInfoType.CLI_MAX_COLUMN_NAME_LEN), + CLI_MAX_CURSOR_NAME_LEN(TGetInfoType.CLI_MAX_CURSOR_NAME_LEN), + CLI_MAX_SCHEMA_NAME_LEN(TGetInfoType.CLI_MAX_SCHEMA_NAME_LEN), + CLI_MAX_CATALOG_NAME_LEN(TGetInfoType.CLI_MAX_CATALOG_NAME_LEN), + CLI_MAX_TABLE_NAME_LEN(TGetInfoType.CLI_MAX_TABLE_NAME_LEN), + CLI_SCROLL_CONCURRENCY(TGetInfoType.CLI_SCROLL_CONCURRENCY), + CLI_TXN_CAPABLE(TGetInfoType.CLI_TXN_CAPABLE), + CLI_USER_NAME(TGetInfoType.CLI_USER_NAME), + CLI_TXN_ISOLATION_OPTION(TGetInfoType.CLI_TXN_ISOLATION_OPTION), + CLI_INTEGRITY(TGetInfoType.CLI_INTEGRITY), + CLI_GETDATA_EXTENSIONS(TGetInfoType.CLI_GETDATA_EXTENSIONS), + CLI_NULL_COLLATION(TGetInfoType.CLI_NULL_COLLATION), + CLI_ALTER_TABLE(TGetInfoType.CLI_ALTER_TABLE), + CLI_ORDER_BY_COLUMNS_IN_SELECT(TGetInfoType.CLI_ORDER_BY_COLUMNS_IN_SELECT), + CLI_SPECIAL_CHARACTERS(TGetInfoType.CLI_SPECIAL_CHARACTERS), + CLI_MAX_COLUMNS_IN_GROUP_BY(TGetInfoType.CLI_MAX_COLUMNS_IN_GROUP_BY), + CLI_MAX_COLUMNS_IN_INDEX(TGetInfoType.CLI_MAX_COLUMNS_IN_INDEX), + CLI_MAX_COLUMNS_IN_ORDER_BY(TGetInfoType.CLI_MAX_COLUMNS_IN_ORDER_BY), + CLI_MAX_COLUMNS_IN_SELECT(TGetInfoType.CLI_MAX_COLUMNS_IN_SELECT), + CLI_MAX_COLUMNS_IN_TABLE(TGetInfoType.CLI_MAX_COLUMNS_IN_TABLE), + CLI_MAX_INDEX_SIZE(TGetInfoType.CLI_MAX_INDEX_SIZE), + CLI_MAX_ROW_SIZE(TGetInfoType.CLI_MAX_ROW_SIZE), + CLI_MAX_STATEMENT_LEN(TGetInfoType.CLI_MAX_STATEMENT_LEN), + CLI_MAX_TABLES_IN_SELECT(TGetInfoType.CLI_MAX_TABLES_IN_SELECT), + CLI_MAX_USER_NAME_LEN(TGetInfoType.CLI_MAX_USER_NAME_LEN), + CLI_OJ_CAPABILITIES(TGetInfoType.CLI_OJ_CAPABILITIES), + + CLI_XOPEN_CLI_YEAR(TGetInfoType.CLI_XOPEN_CLI_YEAR), + CLI_CURSOR_SENSITIVITY(TGetInfoType.CLI_CURSOR_SENSITIVITY), + CLI_DESCRIBE_PARAMETER(TGetInfoType.CLI_DESCRIBE_PARAMETER), + CLI_CATALOG_NAME(TGetInfoType.CLI_CATALOG_NAME), + CLI_COLLATION_SEQ(TGetInfoType.CLI_COLLATION_SEQ), + CLI_MAX_IDENTIFIER_LEN(TGetInfoType.CLI_MAX_IDENTIFIER_LEN); + + private final TGetInfoType tInfoType; + + GetInfoType(TGetInfoType tInfoType) { + this.tInfoType = tInfoType; + } + + public static GetInfoType getGetInfoType(TGetInfoType tGetInfoType) { + for (GetInfoType infoType : values()) { + if (tGetInfoType.equals(infoType.tInfoType)) { + return infoType; + } + } + throw new IllegalArgumentException("Unrecognized Thrift TGetInfoType value: " + tGetInfoType); + } + + public TGetInfoType toTGetInfoType() { + return tInfoType; + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoValue.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoValue.java new file mode 100644 index 000000000000..ba92ff4ab5c1 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoValue.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TGetInfoValue; + +/** + * GetInfoValue. + * + */ +public class GetInfoValue { + private String stringValue = null; + private short shortValue; + private int intValue; + private long longValue; + + public GetInfoValue(String stringValue) { + this.stringValue = stringValue; + } + + public GetInfoValue(short shortValue) { + this.shortValue = shortValue; + } + + public GetInfoValue(int intValue) { + this.intValue = intValue; + } + + public GetInfoValue(long longValue) { + this.longValue = longValue; + } + + public GetInfoValue(TGetInfoValue tGetInfoValue) { + switch (tGetInfoValue.getSetField()) { + case STRING_VALUE: + stringValue = tGetInfoValue.getStringValue(); + break; + default: + throw new IllegalArgumentException("Unreconigzed TGetInfoValue"); + } + } + + public TGetInfoValue toTGetInfoValue() { + TGetInfoValue tInfoValue = new TGetInfoValue(); + if (stringValue != null) { + tInfoValue.setStringValue(stringValue); + } + return tInfoValue; + } + + public String getStringValue() { + return stringValue; + } + + public short getShortValue() { + return shortValue; + } + + public int getIntValue() { + return intValue; + } + + public long getLongValue() { + return longValue; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Handle.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Handle.java new file mode 100644 index 000000000000..cf3427ae20f3 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Handle.java @@ -0,0 +1,78 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.THandleIdentifier; + + + + +public abstract class Handle { + + private final HandleIdentifier handleId; + + public Handle() { + handleId = new HandleIdentifier(); + } + + public Handle(HandleIdentifier handleId) { + this.handleId = handleId; + } + + public Handle(THandleIdentifier tHandleIdentifier) { + this.handleId = new HandleIdentifier(tHandleIdentifier); + } + + public HandleIdentifier getHandleIdentifier() { + return handleId; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((handleId == null) ? 0 : handleId.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof Handle)) { + return false; + } + Handle other = (Handle) obj; + if (handleId == null) { + if (other.handleId != null) { + return false; + } + } else if (!handleId.equals(other.handleId)) { + return false; + } + return true; + } + + @Override + public abstract String toString(); + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java new file mode 100644 index 000000000000..4dc80da8dc50 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java @@ -0,0 +1,113 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.nio.ByteBuffer; +import java.util.UUID; + +import org.apache.hive.service.cli.thrift.THandleIdentifier; + +/** + * HandleIdentifier. + * + */ +public class HandleIdentifier { + private final UUID publicId; + private final UUID secretId; + + public HandleIdentifier() { + publicId = UUID.randomUUID(); + secretId = UUID.randomUUID(); + } + + public HandleIdentifier(UUID publicId, UUID secretId) { + this.publicId = publicId; + this.secretId = secretId; + } + + public HandleIdentifier(THandleIdentifier tHandleId) { + ByteBuffer bb = ByteBuffer.wrap(tHandleId.getGuid()); + this.publicId = new UUID(bb.getLong(), bb.getLong()); + bb = ByteBuffer.wrap(tHandleId.getSecret()); + this.secretId = new UUID(bb.getLong(), bb.getLong()); + } + + public UUID getPublicId() { + return publicId; + } + + public UUID getSecretId() { + return secretId; + } + + public THandleIdentifier toTHandleIdentifier() { + byte[] guid = new byte[16]; + byte[] secret = new byte[16]; + ByteBuffer guidBB = ByteBuffer.wrap(guid); + ByteBuffer secretBB = ByteBuffer.wrap(secret); + guidBB.putLong(publicId.getMostSignificantBits()); + guidBB.putLong(publicId.getLeastSignificantBits()); + secretBB.putLong(secretId.getMostSignificantBits()); + secretBB.putLong(secretId.getLeastSignificantBits()); + return new THandleIdentifier(ByteBuffer.wrap(guid), ByteBuffer.wrap(secret)); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((publicId == null) ? 0 : publicId.hashCode()); + result = prime * result + ((secretId == null) ? 0 : secretId.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof HandleIdentifier)) { + return false; + } + HandleIdentifier other = (HandleIdentifier) obj; + if (publicId == null) { + if (other.publicId != null) { + return false; + } + } else if (!publicId.equals(other.publicId)) { + return false; + } + if (secretId == null) { + if (other.secretId != null) { + return false; + } + } else if (!secretId.equals(other.secretId)) { + return false; + } + return true; + } + + @Override + public String toString() { + return publicId.toString(); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java new file mode 100644 index 000000000000..86e57fbf31fe --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java @@ -0,0 +1,249 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hive.service.cli.thrift.TStatus; +import org.apache.hive.service.cli.thrift.TStatusCode; + +/** + * HiveSQLException. + * + */ +public class HiveSQLException extends SQLException { + + /** + * + */ + private static final long serialVersionUID = -6095254671958748094L; + + /** + * + */ + public HiveSQLException() { + super(); + } + + /** + * @param reason + */ + public HiveSQLException(String reason) { + super(reason); + } + + /** + * @param cause + */ + public HiveSQLException(Throwable cause) { + super(cause); + } + + /** + * @param reason + * @param sqlState + */ + public HiveSQLException(String reason, String sqlState) { + super(reason, sqlState); + } + + /** + * @param reason + * @param cause + */ + public HiveSQLException(String reason, Throwable cause) { + super(reason, cause); + } + + /** + * @param reason + * @param sqlState + * @param vendorCode + */ + public HiveSQLException(String reason, String sqlState, int vendorCode) { + super(reason, sqlState, vendorCode); + } + + /** + * @param reason + * @param sqlState + * @param cause + */ + public HiveSQLException(String reason, String sqlState, Throwable cause) { + super(reason, sqlState, cause); + } + + /** + * @param reason + * @param sqlState + * @param vendorCode + * @param cause + */ + public HiveSQLException(String reason, String sqlState, int vendorCode, Throwable cause) { + super(reason, sqlState, vendorCode, cause); + } + + public HiveSQLException(TStatus status) { + // TODO: set correct vendorCode field + super(status.getErrorMessage(), status.getSqlState(), status.getErrorCode()); + if (status.getInfoMessages() != null) { + initCause(toCause(status.getInfoMessages())); + } + } + + /** + * Converts current object to a {@link TStatus} object + * @return a {@link TStatus} object + */ + public TStatus toTStatus() { + // TODO: convert sqlState, etc. + TStatus tStatus = new TStatus(TStatusCode.ERROR_STATUS); + tStatus.setSqlState(getSQLState()); + tStatus.setErrorCode(getErrorCode()); + tStatus.setErrorMessage(getMessage()); + tStatus.setInfoMessages(toString(this)); + return tStatus; + } + + /** + * Converts the specified {@link Exception} object into a {@link TStatus} object + * @param e a {@link Exception} object + * @return a {@link TStatus} object + */ + public static TStatus toTStatus(Exception e) { + if (e instanceof HiveSQLException) { + return ((HiveSQLException)e).toTStatus(); + } + TStatus tStatus = new TStatus(TStatusCode.ERROR_STATUS); + tStatus.setErrorMessage(e.getMessage()); + tStatus.setInfoMessages(toString(e)); + return tStatus; + } + + /** + * Converts a {@link Throwable} object into a flattened list of texts including its stack trace + * and the stack traces of the nested causes. + * @param ex a {@link Throwable} object + * @return a flattened list of texts including the {@link Throwable} object's stack trace + * and the stack traces of the nested causes. + */ + public static List toString(Throwable ex) { + return toString(ex, null); + } + + private static List toString(Throwable cause, StackTraceElement[] parent) { + StackTraceElement[] trace = cause.getStackTrace(); + int m = trace.length - 1; + if (parent != null) { + int n = parent.length - 1; + while (m >= 0 && n >= 0 && trace[m].equals(parent[n])) { + m--; + n--; + } + } + List detail = enroll(cause, trace, m); + cause = cause.getCause(); + if (cause != null) { + detail.addAll(toString(cause, trace)); + } + return detail; + } + + private static List enroll(Throwable ex, StackTraceElement[] trace, int max) { + List details = new ArrayList(); + StringBuilder builder = new StringBuilder(); + builder.append('*').append(ex.getClass().getName()).append(':'); + builder.append(ex.getMessage()).append(':'); + builder.append(trace.length).append(':').append(max); + details.add(builder.toString()); + for (int i = 0; i <= max; i++) { + builder.setLength(0); + builder.append(trace[i].getClassName()).append(':'); + builder.append(trace[i].getMethodName()).append(':'); + String fileName = trace[i].getFileName(); + builder.append(fileName == null ? "" : fileName).append(':'); + builder.append(trace[i].getLineNumber()); + details.add(builder.toString()); + } + return details; + } + + /** + * Converts a flattened list of texts including the stack trace and the stack + * traces of the nested causes into a {@link Throwable} object. + * @param details a flattened list of texts including the stack trace and the stack + * traces of the nested causes + * @return a {@link Throwable} object + */ + public static Throwable toCause(List details) { + return toStackTrace(details, null, 0); + } + + private static Throwable toStackTrace(List details, StackTraceElement[] parent, int index) { + String detail = details.get(index++); + if (!detail.startsWith("*")) { + return null; // should not be happened. ignore remaining + } + int i1 = detail.indexOf(':'); + int i3 = detail.lastIndexOf(':'); + int i2 = detail.substring(0, i3).lastIndexOf(':'); + String exceptionClass = detail.substring(1, i1); + String exceptionMessage = detail.substring(i1 + 1, i2); + Throwable ex = newInstance(exceptionClass, exceptionMessage); + + Integer length = Integer.valueOf(detail.substring(i2 + 1, i3)); + Integer unique = Integer.valueOf(detail.substring(i3 + 1)); + + int i = 0; + StackTraceElement[] trace = new StackTraceElement[length]; + for (; i <= unique; i++) { + detail = details.get(index++); + int j1 = detail.indexOf(':'); + int j3 = detail.lastIndexOf(':'); + int j2 = detail.substring(0, j3).lastIndexOf(':'); + String className = detail.substring(0, j1); + String methodName = detail.substring(j1 + 1, j2); + String fileName = detail.substring(j2 + 1, j3); + if (fileName.isEmpty()) { + fileName = null; + } + int lineNumber = Integer.valueOf(detail.substring(j3 + 1)); + trace[i] = new StackTraceElement(className, methodName, fileName, lineNumber); + } + int common = trace.length - i; + if (common > 0) { + System.arraycopy(parent, parent.length - common, trace, trace.length - common, common); + } + if (details.size() > index) { + ex.initCause(toStackTrace(details, trace, index)); + } + ex.setStackTrace(trace); + return ex; + } + + private static Throwable newInstance(String className, String message) { + try { + return (Throwable)Class.forName(className).getConstructor(String.class).newInstance(message); + } catch (Exception e) { + return new RuntimeException(className + ":" + message); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ICLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ICLIService.java new file mode 100644 index 000000000000..c9cc1f4da56f --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ICLIService.java @@ -0,0 +1,105 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.cli; + +import java.util.List; +import java.util.Map; + + + + +import org.apache.hive.service.auth.HiveAuthFactory; + +public interface ICLIService { + + SessionHandle openSession(String username, String password, + Map configuration) + throws HiveSQLException; + + SessionHandle openSessionWithImpersonation(String username, String password, + Map configuration, String delegationToken) + throws HiveSQLException; + + void closeSession(SessionHandle sessionHandle) + throws HiveSQLException; + + GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType infoType) + throws HiveSQLException; + + OperationHandle executeStatement(SessionHandle sessionHandle, String statement, + Map confOverlay) + throws HiveSQLException; + + OperationHandle executeStatementAsync(SessionHandle sessionHandle, + String statement, Map confOverlay) + throws HiveSQLException; + + OperationHandle getTypeInfo(SessionHandle sessionHandle) + throws HiveSQLException; + + OperationHandle getCatalogs(SessionHandle sessionHandle) + throws HiveSQLException; + + OperationHandle getSchemas(SessionHandle sessionHandle, + String catalogName, String schemaName) + throws HiveSQLException; + + OperationHandle getTables(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, List tableTypes) + throws HiveSQLException; + + OperationHandle getTableTypes(SessionHandle sessionHandle) + throws HiveSQLException; + + OperationHandle getColumns(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, String columnName) + throws HiveSQLException; + + OperationHandle getFunctions(SessionHandle sessionHandle, + String catalogName, String schemaName, String functionName) + throws HiveSQLException; + + OperationStatus getOperationStatus(OperationHandle opHandle) + throws HiveSQLException; + + void cancelOperation(OperationHandle opHandle) + throws HiveSQLException; + + void closeOperation(OperationHandle opHandle) + throws HiveSQLException; + + TableSchema getResultSetMetadata(OperationHandle opHandle) + throws HiveSQLException; + + RowSet fetchResults(OperationHandle opHandle) + throws HiveSQLException; + + RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, + long maxRows, FetchType fetchType) throws HiveSQLException; + + String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String owner, String renewer) throws HiveSQLException; + + void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException; + + void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException; + + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationHandle.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationHandle.java new file mode 100644 index 000000000000..5426e2847123 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationHandle.java @@ -0,0 +1,102 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TOperationHandle; +import org.apache.hive.service.cli.thrift.TProtocolVersion; + +public class OperationHandle extends Handle { + + private final OperationType opType; + private final TProtocolVersion protocol; + private boolean hasResultSet = false; + + public OperationHandle(OperationType opType, TProtocolVersion protocol) { + super(); + this.opType = opType; + this.protocol = protocol; + } + + // dummy handle for ThriftCLIService + public OperationHandle(TOperationHandle tOperationHandle) { + this(tOperationHandle, TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V1); + } + + public OperationHandle(TOperationHandle tOperationHandle, TProtocolVersion protocol) { + super(tOperationHandle.getOperationId()); + this.opType = OperationType.getOperationType(tOperationHandle.getOperationType()); + this.hasResultSet = tOperationHandle.isHasResultSet(); + this.protocol = protocol; + } + + public OperationType getOperationType() { + return opType; + } + + public void setHasResultSet(boolean hasResultSet) { + this.hasResultSet = hasResultSet; + } + + public boolean hasResultSet() { + return hasResultSet; + } + + public TOperationHandle toTOperationHandle() { + TOperationHandle tOperationHandle = new TOperationHandle(); + tOperationHandle.setOperationId(getHandleIdentifier().toTHandleIdentifier()); + tOperationHandle.setOperationType(opType.toTOperationType()); + tOperationHandle.setHasResultSet(hasResultSet); + return tOperationHandle; + } + + public TProtocolVersion getProtocolVersion() { + return protocol; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = super.hashCode(); + result = prime * result + ((opType == null) ? 0 : opType.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!super.equals(obj)) { + return false; + } + if (!(obj instanceof OperationHandle)) { + return false; + } + OperationHandle other = (OperationHandle) obj; + if (opType != other.opType) { + return false; + } + return true; + } + + @Override + public String toString() { + return "OperationHandle [opType=" + opType + ", getHandleIdentifier()=" + getHandleIdentifier() + + "]"; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java new file mode 100644 index 000000000000..116518011841 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TOperationState; + +/** + * OperationState. + * + */ +public enum OperationState { + INITIALIZED(TOperationState.INITIALIZED_STATE, false), + RUNNING(TOperationState.RUNNING_STATE, false), + FINISHED(TOperationState.FINISHED_STATE, true), + CANCELED(TOperationState.CANCELED_STATE, true), + CLOSED(TOperationState.CLOSED_STATE, true), + ERROR(TOperationState.ERROR_STATE, true), + UNKNOWN(TOperationState.UKNOWN_STATE, false), + PENDING(TOperationState.PENDING_STATE, false); + + private final TOperationState tOperationState; + private final boolean terminal; + + OperationState(TOperationState tOperationState, boolean terminal) { + this.tOperationState = tOperationState; + this.terminal = terminal; + } + + // must be sync with TOperationState in order + public static OperationState getOperationState(TOperationState tOperationState) { + return OperationState.values()[tOperationState.getValue()]; + } + + public static void validateTransition(OperationState oldState, + OperationState newState) + throws HiveSQLException { + switch (oldState) { + case INITIALIZED: + switch (newState) { + case PENDING: + case RUNNING: + case CANCELED: + case CLOSED: + return; + } + break; + case PENDING: + switch (newState) { + case RUNNING: + case FINISHED: + case CANCELED: + case ERROR: + case CLOSED: + return; + } + break; + case RUNNING: + switch (newState) { + case FINISHED: + case CANCELED: + case ERROR: + case CLOSED: + return; + } + break; + case FINISHED: + case CANCELED: + case ERROR: + if (OperationState.CLOSED.equals(newState)) { + return; + } + break; + default: + // fall-through + } + throw new HiveSQLException("Illegal Operation state transition " + + "from " + oldState + " to " + newState); + } + + public void validateTransition(OperationState newState) + throws HiveSQLException { + validateTransition(this, newState); + } + + public TOperationState toTOperationState() { + return tOperationState; + } + + public boolean isTerminal() { + return terminal; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationStatus.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationStatus.java new file mode 100644 index 000000000000..e45b828193da --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationStatus.java @@ -0,0 +1,43 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +/** + * OperationStatus + * + */ +public class OperationStatus { + + private final OperationState state; + private final HiveSQLException operationException; + + public OperationStatus(OperationState state, HiveSQLException operationException) { + this.state = state; + this.operationException = operationException; + } + + public OperationState getState() { + return state; + } + + public HiveSQLException getOperationException() { + return operationException; + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationType.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationType.java new file mode 100644 index 000000000000..429d9a4c2568 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationType.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.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TOperationType; + +/** + * OperationType. + * + */ +public enum OperationType { + + UNKNOWN_OPERATION(TOperationType.UNKNOWN), + EXECUTE_STATEMENT(TOperationType.EXECUTE_STATEMENT), + GET_TYPE_INFO(TOperationType.GET_TYPE_INFO), + GET_CATALOGS(TOperationType.GET_CATALOGS), + GET_SCHEMAS(TOperationType.GET_SCHEMAS), + GET_TABLES(TOperationType.GET_TABLES), + GET_TABLE_TYPES(TOperationType.GET_TABLE_TYPES), + GET_COLUMNS(TOperationType.GET_COLUMNS), + GET_FUNCTIONS(TOperationType.GET_FUNCTIONS); + + private TOperationType tOperationType; + + OperationType(TOperationType tOpType) { + this.tOperationType = tOpType; + } + + public static OperationType getOperationType(TOperationType tOperationType) { + // TODO: replace this with a Map? + for (OperationType opType : values()) { + if (tOperationType.equals(opType.tOperationType)) { + return opType; + } + } + return OperationType.UNKNOWN_OPERATION; + } + + public TOperationType toTOperationType() { + return tOperationType; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java new file mode 100644 index 000000000000..6e4d43fd5df6 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +/** + * PatternOrIdentifier. + * + */ +public class PatternOrIdentifier { + + boolean isPattern = false; + String text; + + public PatternOrIdentifier(String tpoi) { + text = tpoi; + isPattern = false; + } + + public boolean isPattern() { + return isPattern; + } + + public boolean isIdentifier() { + return !isPattern; + } + + @Override + public String toString() { + return text; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java new file mode 100644 index 000000000000..7452137f077d --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java @@ -0,0 +1,140 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.hive.service.cli.thrift.TColumnValue; +import org.apache.hive.service.cli.thrift.TRow; +import org.apache.hive.service.cli.thrift.TRowSet; + +/** + * RowBasedSet + */ +public class RowBasedSet implements RowSet { + + private long startOffset; + + private final Type[] types; // non-null only for writing (server-side) + private final RemovableList rows; + + public RowBasedSet(TableSchema schema) { + types = schema.toTypes(); + rows = new RemovableList(); + } + + public RowBasedSet(TRowSet tRowSet) { + types = null; + rows = new RemovableList(tRowSet.getRows()); + startOffset = tRowSet.getStartRowOffset(); + } + + private RowBasedSet(Type[] types, List rows, long startOffset) { + this.types = types; + this.rows = new RemovableList(rows); + this.startOffset = startOffset; + } + + @Override + public RowBasedSet addRow(Object[] fields) { + TRow tRow = new TRow(); + for (int i = 0; i < fields.length; i++) { + tRow.addToColVals(ColumnValue.toTColumnValue(types[i], fields[i])); + } + rows.add(tRow); + return this; + } + + @Override + public int numColumns() { + return rows.isEmpty() ? 0 : rows.get(0).getColVals().size(); + } + + @Override + public int numRows() { + return rows.size(); + } + + public RowBasedSet extractSubset(int maxRows) { + int numRows = Math.min(numRows(), maxRows); + RowBasedSet result = new RowBasedSet(types, rows.subList(0, numRows), startOffset); + rows.removeRange(0, numRows); + startOffset += numRows; + return result; + } + + public long getStartOffset() { + return startOffset; + } + + public void setStartOffset(long startOffset) { + this.startOffset = startOffset; + } + + public int getSize() { + return rows.size(); + } + + public TRowSet toTRowSet() { + TRowSet tRowSet = new TRowSet(); + tRowSet.setStartRowOffset(startOffset); + tRowSet.setRows(new ArrayList(rows)); + return tRowSet; + } + + @Override + public Iterator iterator() { + return new Iterator() { + + final Iterator iterator = rows.iterator(); + final Object[] convey = new Object[numColumns()]; + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public Object[] next() { + TRow row = iterator.next(); + List values = row.getColVals(); + for (int i = 0; i < values.size(); i++) { + convey[i] = ColumnValue.toColumnValue(values.get(i)); + } + return convey; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("remove"); + } + }; + } + + private static class RemovableList extends ArrayList { + RemovableList() { super(); } + RemovableList(List rows) { super(rows); } + @Override + public void removeRange(int fromIndex, int toIndex) { + super.removeRange(fromIndex, toIndex); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSet.java new file mode 100644 index 000000000000..ab0787e1d389 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSet.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TRowSet; + +public interface RowSet extends Iterable { + + RowSet addRow(Object[] fields); + + RowSet extractSubset(int maxRows); + + int numColumns(); + + int numRows(); + + long getStartOffset(); + + void setStartOffset(long startOffset); + + TRowSet toTRowSet(); +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSetFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSetFactory.java new file mode 100644 index 000000000000..e8f68eaaf906 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSetFactory.java @@ -0,0 +1,41 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.cli.thrift.TProtocolVersion; +import org.apache.hive.service.cli.thrift.TRowSet; + +import static org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6; + +public class RowSetFactory { + + public static RowSet create(TableSchema schema, TProtocolVersion version) { + if (version.getValue() >= HIVE_CLI_SERVICE_PROTOCOL_V6.getValue()) { + return new ColumnBasedSet(schema); + } + return new RowBasedSet(schema); + } + + public static RowSet create(TRowSet results, TProtocolVersion version) { + if (version.getValue() >= HIVE_CLI_SERVICE_PROTOCOL_V6.getValue()) { + return new ColumnBasedSet(results); + } + return new RowBasedSet(results); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/SessionHandle.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/SessionHandle.java new file mode 100644 index 000000000000..52e0ad4834d8 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/SessionHandle.java @@ -0,0 +1,67 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.UUID; + +import org.apache.hive.service.cli.thrift.TProtocolVersion; +import org.apache.hive.service.cli.thrift.TSessionHandle; + + +/** + * SessionHandle. + * + */ +public class SessionHandle extends Handle { + + private final TProtocolVersion protocol; + + public SessionHandle(TProtocolVersion protocol) { + this.protocol = protocol; + } + + // dummy handle for ThriftCLIService + public SessionHandle(TSessionHandle tSessionHandle) { + this(tSessionHandle, TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V1); + } + + public SessionHandle(TSessionHandle tSessionHandle, TProtocolVersion protocol) { + super(tSessionHandle.getSessionId()); + this.protocol = protocol; + } + + public UUID getSessionId() { + return getHandleIdentifier().getPublicId(); + } + + public TSessionHandle toTSessionHandle() { + TSessionHandle tSessionHandle = new TSessionHandle(); + tSessionHandle.setSessionId(getHandleIdentifier().toTHandleIdentifier()); + return tSessionHandle; + } + + public TProtocolVersion getProtocolVersion() { + return protocol; + } + + @Override + public String toString() { + return "SessionHandle [" + getHandleIdentifier() + "]"; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TableSchema.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TableSchema.java new file mode 100644 index 000000000000..ee019bc73710 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TableSchema.java @@ -0,0 +1,102 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Schema; +import org.apache.hive.service.cli.thrift.TColumnDesc; +import org.apache.hive.service.cli.thrift.TTableSchema; + +/** + * TableSchema. + * + */ +public class TableSchema { + private final List columns = new ArrayList(); + + public TableSchema() { + } + + public TableSchema(int numColumns) { + // TODO: remove this constructor + } + + public TableSchema(TTableSchema tTableSchema) { + for (TColumnDesc tColumnDesc : tTableSchema.getColumns()) { + columns.add(new ColumnDescriptor(tColumnDesc)); + } + } + + public TableSchema(List fieldSchemas) { + int pos = 1; + for (FieldSchema field : fieldSchemas) { + columns.add(new ColumnDescriptor(field, pos++)); + } + } + + public TableSchema(Schema schema) { + this(schema.getFieldSchemas()); + } + + public List getColumnDescriptors() { + return new ArrayList(columns); + } + + public ColumnDescriptor getColumnDescriptorAt(int pos) { + return columns.get(pos); + } + + public int getSize() { + return columns.size(); + } + + public void clear() { + columns.clear(); + } + + + public TTableSchema toTTableSchema() { + TTableSchema tTableSchema = new TTableSchema(); + for (ColumnDescriptor col : columns) { + tTableSchema.addToColumns(col.toTColumnDesc()); + } + return tTableSchema; + } + + public Type[] toTypes() { + Type[] types = new Type[columns.size()]; + for (int i = 0; i < types.length; i++) { + types[i] = columns.get(i).getType(); + } + return types; + } + + public TableSchema addPrimitiveColumn(String columnName, Type columnType, String columnComment) { + columns.add(ColumnDescriptor.newPrimitiveColumnDescriptor(columnName, columnComment, columnType, columns.size() + 1)); + return this; + } + + public TableSchema addStringColumn(String columnName, String columnComment) { + columns.add(ColumnDescriptor.newPrimitiveColumnDescriptor(columnName, columnComment, Type.STRING_TYPE, columns.size() + 1)); + return this; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Type.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Type.java new file mode 100644 index 000000000000..7752ec03a29b --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Type.java @@ -0,0 +1,349 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.sql.DatabaseMetaData; +import java.util.Locale; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hive.service.cli.thrift.TTypeId; + +/** + * Type. + * + */ +public enum Type { + NULL_TYPE("VOID", + java.sql.Types.NULL, + TTypeId.NULL_TYPE), + BOOLEAN_TYPE("BOOLEAN", + java.sql.Types.BOOLEAN, + TTypeId.BOOLEAN_TYPE), + TINYINT_TYPE("TINYINT", + java.sql.Types.TINYINT, + TTypeId.TINYINT_TYPE), + SMALLINT_TYPE("SMALLINT", + java.sql.Types.SMALLINT, + TTypeId.SMALLINT_TYPE), + INT_TYPE("INT", + java.sql.Types.INTEGER, + TTypeId.INT_TYPE), + BIGINT_TYPE("BIGINT", + java.sql.Types.BIGINT, + TTypeId.BIGINT_TYPE), + FLOAT_TYPE("FLOAT", + java.sql.Types.FLOAT, + TTypeId.FLOAT_TYPE), + DOUBLE_TYPE("DOUBLE", + java.sql.Types.DOUBLE, + TTypeId.DOUBLE_TYPE), + STRING_TYPE("STRING", + java.sql.Types.VARCHAR, + TTypeId.STRING_TYPE), + CHAR_TYPE("CHAR", + java.sql.Types.CHAR, + TTypeId.CHAR_TYPE, + true, false, false), + VARCHAR_TYPE("VARCHAR", + java.sql.Types.VARCHAR, + TTypeId.VARCHAR_TYPE, + true, false, false), + DATE_TYPE("DATE", + java.sql.Types.DATE, + TTypeId.DATE_TYPE), + TIMESTAMP_TYPE("TIMESTAMP", + java.sql.Types.TIMESTAMP, + TTypeId.TIMESTAMP_TYPE), + INTERVAL_YEAR_MONTH_TYPE("INTERVAL_YEAR_MONTH", + java.sql.Types.OTHER, + TTypeId.INTERVAL_YEAR_MONTH_TYPE), + INTERVAL_DAY_TIME_TYPE("INTERVAL_DAY_TIME", + java.sql.Types.OTHER, + TTypeId.INTERVAL_DAY_TIME_TYPE), + BINARY_TYPE("BINARY", + java.sql.Types.BINARY, + TTypeId.BINARY_TYPE), + DECIMAL_TYPE("DECIMAL", + java.sql.Types.DECIMAL, + TTypeId.DECIMAL_TYPE, + true, false, false), + ARRAY_TYPE("ARRAY", + java.sql.Types.ARRAY, + TTypeId.ARRAY_TYPE, + true, true), + MAP_TYPE("MAP", + java.sql.Types.JAVA_OBJECT, + TTypeId.MAP_TYPE, + true, true), + STRUCT_TYPE("STRUCT", + java.sql.Types.STRUCT, + TTypeId.STRUCT_TYPE, + true, false), + UNION_TYPE("UNIONTYPE", + java.sql.Types.OTHER, + TTypeId.UNION_TYPE, + true, false), + USER_DEFINED_TYPE("USER_DEFINED", + java.sql.Types.OTHER, + TTypeId.USER_DEFINED_TYPE, + true, false); + + private final String name; + private final TTypeId tType; + private final int javaSQLType; + private final boolean isQualified; + private final boolean isComplex; + private final boolean isCollection; + + Type(String name, int javaSQLType, TTypeId tType, boolean isQualified, boolean isComplex, boolean isCollection) { + this.name = name; + this.javaSQLType = javaSQLType; + this.tType = tType; + this.isQualified = isQualified; + this.isComplex = isComplex; + this.isCollection = isCollection; + } + + Type(String name, int javaSQLType, TTypeId tType, boolean isComplex, boolean isCollection) { + this(name, javaSQLType, tType, false, isComplex, isCollection); + } + + Type(String name, int javaSqlType, TTypeId tType) { + this(name, javaSqlType, tType, false, false, false); + } + + public boolean isPrimitiveType() { + return !isComplex; + } + + public boolean isQualifiedType() { + return isQualified; + } + + public boolean isComplexType() { + return isComplex; + } + + public boolean isCollectionType() { + return isCollection; + } + + public static Type getType(TTypeId tType) { + for (Type type : values()) { + if (tType.equals(type.tType)) { + return type; + } + } + throw new IllegalArgumentException("Unregonized Thrift TTypeId value: " + tType); + } + + public static Type getType(String name) { + if (name == null) { + throw new IllegalArgumentException("Invalid type name: null"); + } + for (Type type : values()) { + if (name.equalsIgnoreCase(type.name)) { + return type; + } else if (type.isQualifiedType() || type.isComplexType()) { + if (name.toUpperCase(Locale.ROOT).startsWith(type.name)) { + return type; + } + } + } + throw new IllegalArgumentException("Unrecognized type name: " + name); + } + + /** + * Radix for this type (typically either 2 or 10) + * Null is returned for data types where this is not applicable. + */ + public Integer getNumPrecRadix() { + if (this.isNumericType()) { + return 10; + } + return null; + } + + /** + * Maximum precision for numeric types. + * Returns null for non-numeric types. + * @return + */ + public Integer getMaxPrecision() { + switch (this) { + case TINYINT_TYPE: + return 3; + case SMALLINT_TYPE: + return 5; + case INT_TYPE: + return 10; + case BIGINT_TYPE: + return 19; + case FLOAT_TYPE: + return 7; + case DOUBLE_TYPE: + return 15; + case DECIMAL_TYPE: + return HiveDecimal.MAX_PRECISION; + default: + return null; + } + } + + public boolean isNumericType() { + switch (this) { + case TINYINT_TYPE: + case SMALLINT_TYPE: + case INT_TYPE: + case BIGINT_TYPE: + case FLOAT_TYPE: + case DOUBLE_TYPE: + case DECIMAL_TYPE: + return true; + default: + return false; + } + } + + /** + * Prefix used to quote a literal of this type (may be null) + */ + public String getLiteralPrefix() { + return null; + } + + /** + * Suffix used to quote a literal of this type (may be null) + * @return + */ + public String getLiteralSuffix() { + return null; + } + + /** + * Can you use NULL for this type? + * @return + * DatabaseMetaData.typeNoNulls - does not allow NULL values + * DatabaseMetaData.typeNullable - allows NULL values + * DatabaseMetaData.typeNullableUnknown - nullability unknown + */ + public Short getNullable() { + // All Hive types are nullable + return DatabaseMetaData.typeNullable; + } + + /** + * Is the type case sensitive? + * @return + */ + public Boolean isCaseSensitive() { + switch (this) { + case STRING_TYPE: + return true; + default: + return false; + } + } + + /** + * Parameters used in creating the type (may be null) + * @return + */ + public String getCreateParams() { + return null; + } + + /** + * Can you use WHERE based on this type? + * @return + * DatabaseMetaData.typePredNone - No support + * DatabaseMetaData.typePredChar - Only support with WHERE .. LIKE + * DatabaseMetaData.typePredBasic - Supported except for WHERE .. LIKE + * DatabaseMetaData.typeSearchable - Supported for all WHERE .. + */ + public Short getSearchable() { + if (isPrimitiveType()) { + return DatabaseMetaData.typeSearchable; + } + return DatabaseMetaData.typePredNone; + } + + /** + * Is this type unsigned? + * @return + */ + public Boolean isUnsignedAttribute() { + if (isNumericType()) { + return false; + } + return true; + } + + /** + * Can this type represent money? + * @return + */ + public Boolean isFixedPrecScale() { + return false; + } + + /** + * Can this type be used for an auto-increment value? + * @return + */ + public Boolean isAutoIncrement() { + return false; + } + + /** + * Localized version of type name (may be null). + * @return + */ + public String getLocalizedName() { + return null; + } + + /** + * Minimum scale supported for this type + * @return + */ + public Short getMinimumScale() { + return 0; + } + + /** + * Maximum scale supported for this type + * @return + */ + public Short getMaximumScale() { + return 0; + } + + public TTypeId toTType() { + return tType; + } + + public int toJavaSQLType() { + return javaSQLType; + } + + public String getName() { + return name; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java new file mode 100644 index 000000000000..b80fd67884ad --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java @@ -0,0 +1,159 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.List; + +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hive.service.cli.thrift.TPrimitiveTypeEntry; +import org.apache.hive.service.cli.thrift.TTypeDesc; +import org.apache.hive.service.cli.thrift.TTypeEntry; + +/** + * TypeDescriptor. + * + */ +public class TypeDescriptor { + + private final Type type; + private String typeName = null; + private TypeQualifiers typeQualifiers = null; + + public TypeDescriptor(Type type) { + this.type = type; + } + + public TypeDescriptor(TTypeDesc tTypeDesc) { + List tTypeEntries = tTypeDesc.getTypes(); + TPrimitiveTypeEntry top = tTypeEntries.get(0).getPrimitiveEntry(); + this.type = Type.getType(top.getType()); + if (top.isSetTypeQualifiers()) { + setTypeQualifiers(TypeQualifiers.fromTTypeQualifiers(top.getTypeQualifiers())); + } + } + + public TypeDescriptor(String typeName) { + this.type = Type.getType(typeName); + if (this.type.isComplexType()) { + this.typeName = typeName; + } else if (this.type.isQualifiedType()) { + PrimitiveTypeInfo pti = TypeInfoFactory.getPrimitiveTypeInfo(typeName); + setTypeQualifiers(TypeQualifiers.fromTypeInfo(pti)); + } + } + + public Type getType() { + return type; + } + + public TTypeDesc toTTypeDesc() { + TPrimitiveTypeEntry primitiveEntry = new TPrimitiveTypeEntry(type.toTType()); + if (getTypeQualifiers() != null) { + primitiveEntry.setTypeQualifiers(getTypeQualifiers().toTTypeQualifiers()); + } + TTypeEntry entry = TTypeEntry.primitiveEntry(primitiveEntry); + + TTypeDesc desc = new TTypeDesc(); + desc.addToTypes(entry); + return desc; + } + + public String getTypeName() { + if (typeName != null) { + return typeName; + } else { + return type.getName(); + } + } + + public TypeQualifiers getTypeQualifiers() { + return typeQualifiers; + } + + public void setTypeQualifiers(TypeQualifiers typeQualifiers) { + this.typeQualifiers = typeQualifiers; + } + + /** + * The column size for this type. + * For numeric data this is the maximum precision. + * For character data this is the length in characters. + * For datetime types this is the length in characters of the String representation + * (assuming the maximum allowed precision of the fractional seconds component). + * For binary data this is the length in bytes. + * Null is returned for data types where the column size is not applicable. + */ + public Integer getColumnSize() { + if (type.isNumericType()) { + return getPrecision(); + } + switch (type) { + case STRING_TYPE: + case BINARY_TYPE: + return Integer.MAX_VALUE; + case CHAR_TYPE: + case VARCHAR_TYPE: + return typeQualifiers.getCharacterMaximumLength(); + case DATE_TYPE: + return 10; + case TIMESTAMP_TYPE: + return 29; + default: + return null; + } + } + + /** + * Maximum precision for numeric types. + * Returns null for non-numeric types. + * @return + */ + public Integer getPrecision() { + if (this.type == Type.DECIMAL_TYPE) { + return typeQualifiers.getPrecision(); + } + return this.type.getMaxPrecision(); + } + + /** + * The number of fractional digits for this type. + * Null is returned for data types where this is not applicable. + */ + public Integer getDecimalDigits() { + switch (this.type) { + case BOOLEAN_TYPE: + case TINYINT_TYPE: + case SMALLINT_TYPE: + case INT_TYPE: + case BIGINT_TYPE: + return 0; + case FLOAT_TYPE: + return 7; + case DOUBLE_TYPE: + return 15; + case DECIMAL_TYPE: + return typeQualifiers.getScale(); + case TIMESTAMP_TYPE: + return 9; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java new file mode 100644 index 000000000000..c6da52c15a2b --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java @@ -0,0 +1,133 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; +import org.apache.hive.service.cli.thrift.TCLIServiceConstants; +import org.apache.hive.service.cli.thrift.TTypeQualifierValue; +import org.apache.hive.service.cli.thrift.TTypeQualifiers; + +/** + * This class holds type qualifier information for a primitive type, + * such as char/varchar length or decimal precision/scale. + */ +public class TypeQualifiers { + private Integer characterMaximumLength; + private Integer precision; + private Integer scale; + + public TypeQualifiers() {} + + public Integer getCharacterMaximumLength() { + return characterMaximumLength; + } + public void setCharacterMaximumLength(int characterMaximumLength) { + this.characterMaximumLength = characterMaximumLength; + } + + public TTypeQualifiers toTTypeQualifiers() { + TTypeQualifiers ret = null; + + Map qMap = new HashMap(); + if (getCharacterMaximumLength() != null) { + TTypeQualifierValue val = new TTypeQualifierValue(); + val.setI32Value(getCharacterMaximumLength().intValue()); + qMap.put(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH, val); + } + + if (precision != null) { + TTypeQualifierValue val = new TTypeQualifierValue(); + val.setI32Value(precision.intValue()); + qMap.put(TCLIServiceConstants.PRECISION, val); + } + + if (scale != null) { + TTypeQualifierValue val = new TTypeQualifierValue(); + val.setI32Value(scale.intValue()); + qMap.put(TCLIServiceConstants.SCALE, val); + } + + if (qMap.size() > 0) { + ret = new TTypeQualifiers(qMap); + } + + return ret; + } + + public static TypeQualifiers fromTTypeQualifiers(TTypeQualifiers ttq) { + TypeQualifiers ret = null; + if (ttq != null) { + ret = new TypeQualifiers(); + Map tqMap = ttq.getQualifiers(); + + if (tqMap.containsKey(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH)) { + ret.setCharacterMaximumLength( + tqMap.get(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH).getI32Value()); + } + + if (tqMap.containsKey(TCLIServiceConstants.PRECISION)) { + ret.setPrecision(tqMap.get(TCLIServiceConstants.PRECISION).getI32Value()); + } + + if (tqMap.containsKey(TCLIServiceConstants.SCALE)) { + ret.setScale(tqMap.get(TCLIServiceConstants.SCALE).getI32Value()); + } + } + return ret; + } + + public static TypeQualifiers fromTypeInfo(PrimitiveTypeInfo pti) { + TypeQualifiers result = null; + if (pti instanceof VarcharTypeInfo) { + result = new TypeQualifiers(); + result.setCharacterMaximumLength(((VarcharTypeInfo)pti).getLength()); + } else if (pti instanceof CharTypeInfo) { + result = new TypeQualifiers(); + result.setCharacterMaximumLength(((CharTypeInfo)pti).getLength()); + } else if (pti instanceof DecimalTypeInfo) { + result = new TypeQualifiers(); + result.setPrecision(((DecimalTypeInfo)pti).precision()); + result.setScale(((DecimalTypeInfo)pti).scale()); + } + return result; + } + + public Integer getPrecision() { + return precision; + } + + public void setPrecision(Integer precision) { + this.precision = precision; + } + + public Integer getScale() { + return scale; + } + + public void setScale(Integer scale) { + this.scale = scale; + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java new file mode 100644 index 000000000000..af36057bdaec --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java @@ -0,0 +1,86 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.hive.metastore.TableType; + +/** + * ClassicTableTypeMapping. + * Classic table type mapping : + * Managed Table to Table + * External Table to Table + * Virtual View to View + */ +public class ClassicTableTypeMapping implements TableTypeMapping { + + public enum ClassicTableTypes { + TABLE, + VIEW, + } + + private final Map hiveToClientMap = new HashMap(); + private final Map clientToHiveMap = new HashMap(); + + public ClassicTableTypeMapping() { + hiveToClientMap.put(TableType.MANAGED_TABLE.toString(), + ClassicTableTypes.TABLE.toString()); + hiveToClientMap.put(TableType.EXTERNAL_TABLE.toString(), + ClassicTableTypes.TABLE.toString()); + hiveToClientMap.put(TableType.VIRTUAL_VIEW.toString(), + ClassicTableTypes.VIEW.toString()); + + clientToHiveMap.put(ClassicTableTypes.TABLE.toString(), + TableType.MANAGED_TABLE.toString()); + clientToHiveMap.put(ClassicTableTypes.VIEW.toString(), + TableType.VIRTUAL_VIEW.toString()); + } + + @Override + public String mapToHiveType(String clientTypeName) { + if (clientToHiveMap.containsKey(clientTypeName)) { + return clientToHiveMap.get(clientTypeName); + } else { + return clientTypeName; + } + } + + @Override + public String mapToClientType(String hiveTypeName) { + if (hiveToClientMap.containsKey(hiveTypeName)) { + return hiveToClientMap.get(hiveTypeName); + } else { + return hiveTypeName; + } + } + + @Override + public Set getTableTypeNames() { + Set typeNameSet = new HashSet(); + for (ClassicTableTypes typeNames : ClassicTableTypes.values()) { + typeNameSet.add(typeNames.toString()); + } + return typeNameSet; + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java new file mode 100644 index 000000000000..3f2de108f069 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.cli.operation; + +import java.sql.SQLException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.hadoop.hive.ql.processors.CommandProcessor; +import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.session.HiveSession; + +public abstract class ExecuteStatementOperation extends Operation { + protected String statement = null; + protected Map confOverlay = new HashMap(); + + public ExecuteStatementOperation(HiveSession parentSession, String statement, + Map confOverlay, boolean runInBackground) { + super(parentSession, OperationType.EXECUTE_STATEMENT, runInBackground); + this.statement = statement; + setConfOverlay(confOverlay); + } + + public String getStatement() { + return statement; + } + + public static ExecuteStatementOperation newExecuteStatementOperation( + HiveSession parentSession, String statement, Map confOverlay, boolean runAsync) + throws HiveSQLException { + String[] tokens = statement.trim().split("\\s+"); + CommandProcessor processor = null; + try { + processor = CommandProcessorFactory.getForHiveCommand(tokens, parentSession.getHiveConf()); + } catch (SQLException e) { + throw new HiveSQLException(e.getMessage(), e.getSQLState(), e); + } + if (processor == null) { + return new SQLOperation(parentSession, statement, confOverlay, runAsync); + } + return new HiveCommandOperation(parentSession, statement, processor, confOverlay); + } + + protected Map getConfOverlay() { + return confOverlay; + } + + protected void setConfOverlay(Map confOverlay) { + if (confOverlay != null) { + this.confOverlay = confOverlay; + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java new file mode 100644 index 000000000000..8868ec18e0f5 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java @@ -0,0 +1,81 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetCatalogsOperation. + * + */ +public class GetCatalogsOperation extends MetadataOperation { + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addStringColumn("TABLE_CAT", "Catalog name. NULL if not applicable."); + + private final RowSet rowSet; + + protected GetCatalogsOperation(HiveSession parentSession) { + super(parentSession, OperationType.GET_CATALOGS); + rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + try { + if (isAuthV2Enabled()) { + authorizeMetaGets(HiveOperationType.GET_CATALOGS, null); + } + setState(OperationState.FINISHED); + } catch (HiveSQLException e) { + setState(OperationState.ERROR); + throw e; + } + + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java new file mode 100644 index 000000000000..5efb0759383a --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java @@ -0,0 +1,234 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.sql.DatabaseMetaData; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.regex.Pattern; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType; +import org.apache.hive.service.cli.ColumnDescriptor; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.Type; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetColumnsOperation. + * + */ +public class GetColumnsOperation extends MetadataOperation { + + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addPrimitiveColumn("TABLE_CAT", Type.STRING_TYPE, + "Catalog name. NULL if not applicable") + .addPrimitiveColumn("TABLE_SCHEM", Type.STRING_TYPE, + "Schema name") + .addPrimitiveColumn("TABLE_NAME", Type.STRING_TYPE, + "Table name") + .addPrimitiveColumn("COLUMN_NAME", Type.STRING_TYPE, + "Column name") + .addPrimitiveColumn("DATA_TYPE", Type.INT_TYPE, + "SQL type from java.sql.Types") + .addPrimitiveColumn("TYPE_NAME", Type.STRING_TYPE, + "Data source dependent type name, for a UDT the type name is fully qualified") + .addPrimitiveColumn("COLUMN_SIZE", Type.INT_TYPE, + "Column size. For char or date types this is the maximum number of characters," + + " for numeric or decimal types this is precision.") + .addPrimitiveColumn("BUFFER_LENGTH", Type.TINYINT_TYPE, + "Unused") + .addPrimitiveColumn("DECIMAL_DIGITS", Type.INT_TYPE, + "The number of fractional digits") + .addPrimitiveColumn("NUM_PREC_RADIX", Type.INT_TYPE, + "Radix (typically either 10 or 2)") + .addPrimitiveColumn("NULLABLE", Type.INT_TYPE, + "Is NULL allowed") + .addPrimitiveColumn("REMARKS", Type.STRING_TYPE, + "Comment describing column (may be null)") + .addPrimitiveColumn("COLUMN_DEF", Type.STRING_TYPE, + "Default value (may be null)") + .addPrimitiveColumn("SQL_DATA_TYPE", Type.INT_TYPE, + "Unused") + .addPrimitiveColumn("SQL_DATETIME_SUB", Type.INT_TYPE, + "Unused") + .addPrimitiveColumn("CHAR_OCTET_LENGTH", Type.INT_TYPE, + "For char types the maximum number of bytes in the column") + .addPrimitiveColumn("ORDINAL_POSITION", Type.INT_TYPE, + "Index of column in table (starting at 1)") + .addPrimitiveColumn("IS_NULLABLE", Type.STRING_TYPE, + "\"NO\" means column definitely does not allow NULL values; " + + "\"YES\" means the column might allow NULL values. An empty " + + "string means nobody knows.") + .addPrimitiveColumn("SCOPE_CATALOG", Type.STRING_TYPE, + "Catalog of table that is the scope of a reference attribute " + + "(null if DATA_TYPE isn't REF)") + .addPrimitiveColumn("SCOPE_SCHEMA", Type.STRING_TYPE, + "Schema of table that is the scope of a reference attribute " + + "(null if the DATA_TYPE isn't REF)") + .addPrimitiveColumn("SCOPE_TABLE", Type.STRING_TYPE, + "Table name that this the scope of a reference attribure " + + "(null if the DATA_TYPE isn't REF)") + .addPrimitiveColumn("SOURCE_DATA_TYPE", Type.SMALLINT_TYPE, + "Source type of a distinct type or user-generated Ref type, " + + "SQL type from java.sql.Types (null if DATA_TYPE isn't DISTINCT or user-generated REF)") + .addPrimitiveColumn("IS_AUTO_INCREMENT", Type.STRING_TYPE, + "Indicates whether this column is auto incremented."); + + private final String catalogName; + private final String schemaName; + private final String tableName; + private final String columnName; + + private final RowSet rowSet; + + protected GetColumnsOperation(HiveSession parentSession, String catalogName, String schemaName, + String tableName, String columnName) { + super(parentSession, OperationType.GET_COLUMNS); + this.catalogName = catalogName; + this.schemaName = schemaName; + this.tableName = tableName; + this.columnName = columnName; + this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + try { + IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); + String schemaPattern = convertSchemaPattern(schemaName); + String tablePattern = convertIdentifierPattern(tableName, true); + + Pattern columnPattern = null; + if (columnName != null) { + columnPattern = Pattern.compile(convertIdentifierPattern(columnName, false)); + } + + List dbNames = metastoreClient.getDatabases(schemaPattern); + Collections.sort(dbNames); + Map> db2Tabs = new HashMap<>(); + + for (String dbName : dbNames) { + List tableNames = metastoreClient.getTables(dbName, tablePattern); + Collections.sort(tableNames); + db2Tabs.put(dbName, tableNames); + } + + if (isAuthV2Enabled()) { + List privObjs = getPrivObjs(db2Tabs); + String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName + + ", tablePattern : " + tableName; + authorizeMetaGets(HiveOperationType.GET_COLUMNS, privObjs, cmdStr); + } + + for (Entry> dbTabs : db2Tabs.entrySet()) { + String dbName = dbTabs.getKey(); + List tableNames = dbTabs.getValue(); + for (Table table : metastoreClient.getTableObjectsByName(dbName, tableNames)) { + TableSchema schema = new TableSchema(metastoreClient.getSchema(dbName, table.getTableName())); + for (ColumnDescriptor column : schema.getColumnDescriptors()) { + if (columnPattern != null && !columnPattern.matcher(column.getName()).matches()) { + continue; + } + Object[] rowData = new Object[] { + null, // TABLE_CAT + table.getDbName(), // TABLE_SCHEM + table.getTableName(), // TABLE_NAME + column.getName(), // COLUMN_NAME + column.getType().toJavaSQLType(), // DATA_TYPE + column.getTypeName(), // TYPE_NAME + column.getTypeDescriptor().getColumnSize(), // COLUMN_SIZE + null, // BUFFER_LENGTH, unused + column.getTypeDescriptor().getDecimalDigits(), // DECIMAL_DIGITS + column.getType().getNumPrecRadix(), // NUM_PREC_RADIX + DatabaseMetaData.columnNullable, // NULLABLE + column.getComment(), // REMARKS + null, // COLUMN_DEF + null, // SQL_DATA_TYPE + null, // SQL_DATETIME_SUB + null, // CHAR_OCTET_LENGTH + column.getOrdinalPosition(), // ORDINAL_POSITION + "YES", // IS_NULLABLE + null, // SCOPE_CATALOG + null, // SCOPE_SCHEMA + null, // SCOPE_TABLE + null, // SOURCE_DATA_TYPE + "NO", // IS_AUTO_INCREMENT + }; + rowSet.addRow(rowData); + } + } + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + + } + + + private List getPrivObjs(Map> db2Tabs) { + List privObjs = new ArrayList<>(); + for (Entry> dbTabs : db2Tabs.entrySet()) { + for (String tabName : dbTabs.getValue()) { + privObjs.add(new HivePrivilegeObject(HivePrivilegeObjectType.TABLE_OR_VIEW, dbTabs.getKey(), + tabName)); + } + } + return privObjs; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java new file mode 100644 index 000000000000..5273c386b83d --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.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.hive.service.cli.operation; + +import java.sql.DatabaseMetaData; +import java.util.List; +import java.util.Set; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.ql.exec.FunctionInfo; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils; +import org.apache.hive.service.cli.CLIServiceUtils; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.Type; +import org.apache.hive.service.cli.session.HiveSession; +import org.apache.thrift.TException; + +/** + * GetFunctionsOperation. + * + */ +public class GetFunctionsOperation extends MetadataOperation { + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addPrimitiveColumn("FUNCTION_CAT", Type.STRING_TYPE, + "Function catalog (may be null)") + .addPrimitiveColumn("FUNCTION_SCHEM", Type.STRING_TYPE, + "Function schema (may be null)") + .addPrimitiveColumn("FUNCTION_NAME", Type.STRING_TYPE, + "Function name. This is the name used to invoke the function") + .addPrimitiveColumn("REMARKS", Type.STRING_TYPE, + "Explanatory comment on the function") + .addPrimitiveColumn("FUNCTION_TYPE", Type.INT_TYPE, + "Kind of function.") + .addPrimitiveColumn("SPECIFIC_NAME", Type.STRING_TYPE, + "The name which uniquely identifies this function within its schema"); + + private final String catalogName; + private final String schemaName; + private final String functionName; + + private final RowSet rowSet; + + public GetFunctionsOperation(HiveSession parentSession, + String catalogName, String schemaName, String functionName) { + super(parentSession, OperationType.GET_FUNCTIONS); + this.catalogName = catalogName; + this.schemaName = schemaName; + this.functionName = functionName; + this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + if (isAuthV2Enabled()) { + // get databases for schema pattern + IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); + String schemaPattern = convertSchemaPattern(schemaName); + List matchingDbs; + try { + matchingDbs = metastoreClient.getDatabases(schemaPattern); + } catch (TException e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + // authorize this call on the schema objects + List privObjs = HivePrivilegeObjectUtils + .getHivePrivDbObjects(matchingDbs); + String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; + authorizeMetaGets(HiveOperationType.GET_FUNCTIONS, privObjs, cmdStr); + } + + try { + if ((null == catalogName || "".equals(catalogName)) + && (null == schemaName || "".equals(schemaName))) { + Set functionNames = FunctionRegistry + .getFunctionNames(CLIServiceUtils.patternToRegex(functionName)); + for (String functionName : functionNames) { + FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo(functionName); + Object[] rowData = new Object[] { + null, // FUNCTION_CAT + null, // FUNCTION_SCHEM + functionInfo.getDisplayName(), // FUNCTION_NAME + "", // REMARKS + (functionInfo.isGenericUDTF() ? + DatabaseMetaData.functionReturnsTable + : DatabaseMetaData.functionNoTable), // FUNCTION_TYPE + functionInfo.getClass().getCanonicalName() + }; + rowSet.addRow(rowData); + } + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java new file mode 100644 index 000000000000..d6f6280f1c39 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java @@ -0,0 +1,96 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetSchemasOperation. + * + */ +public class GetSchemasOperation extends MetadataOperation { + private final String catalogName; + private final String schemaName; + + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addStringColumn("TABLE_SCHEM", "Schema name.") + .addStringColumn("TABLE_CATALOG", "Catalog name."); + + private RowSet rowSet; + + protected GetSchemasOperation(HiveSession parentSession, + String catalogName, String schemaName) { + super(parentSession, OperationType.GET_SCHEMAS); + this.catalogName = catalogName; + this.schemaName = schemaName; + this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + if (isAuthV2Enabled()) { + String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; + authorizeMetaGets(HiveOperationType.GET_SCHEMAS, null, cmdStr); + } + try { + IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); + String schemaPattern = convertSchemaPattern(schemaName); + for (String dbName : metastoreClient.getDatabases(schemaPattern)) { + rowSet.addRow(new Object[] {dbName, DEFAULT_HIVE_CATALOG}); + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java new file mode 100644 index 000000000000..3ae012a72764 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java @@ -0,0 +1,93 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetTableTypesOperation. + * + */ +public class GetTableTypesOperation extends MetadataOperation { + + protected static TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addStringColumn("TABLE_TYPE", "Table type name."); + + private final RowSet rowSet; + private final TableTypeMapping tableTypeMapping; + + protected GetTableTypesOperation(HiveSession parentSession) { + super(parentSession, OperationType.GET_TABLE_TYPES); + String tableMappingStr = getParentSession().getHiveConf() + .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); + tableTypeMapping = + TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); + rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + if (isAuthV2Enabled()) { + authorizeMetaGets(HiveOperationType.GET_TABLETYPES, null); + } + try { + for (TableType type : TableType.values()) { + rowSet.addRow(new String[] {tableTypeMapping.mapToClientType(type.toString())}); + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java new file mode 100644 index 000000000000..1a7ca79163d7 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java @@ -0,0 +1,135 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetTablesOperation. + * + */ +public class GetTablesOperation extends MetadataOperation { + + private final String catalogName; + private final String schemaName; + private final String tableName; + private final List tableTypes = new ArrayList(); + private final RowSet rowSet; + private final TableTypeMapping tableTypeMapping; + + + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addStringColumn("TABLE_CAT", "Catalog name. NULL if not applicable.") + .addStringColumn("TABLE_SCHEM", "Schema name.") + .addStringColumn("TABLE_NAME", "Table name.") + .addStringColumn("TABLE_TYPE", "The table type, e.g. \"TABLE\", \"VIEW\", etc.") + .addStringColumn("REMARKS", "Comments about the table."); + + protected GetTablesOperation(HiveSession parentSession, + String catalogName, String schemaName, String tableName, + List tableTypes) { + super(parentSession, OperationType.GET_TABLES); + this.catalogName = catalogName; + this.schemaName = schemaName; + this.tableName = tableName; + String tableMappingStr = getParentSession().getHiveConf() + .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); + tableTypeMapping = + TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); + if (tableTypes != null) { + this.tableTypes.addAll(tableTypes); + } + this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + try { + IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); + String schemaPattern = convertSchemaPattern(schemaName); + List matchingDbs = metastoreClient.getDatabases(schemaPattern); + if(isAuthV2Enabled()){ + List privObjs = HivePrivilegeObjectUtils.getHivePrivDbObjects(matchingDbs); + String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; + authorizeMetaGets(HiveOperationType.GET_TABLES, privObjs, cmdStr); + } + + String tablePattern = convertIdentifierPattern(tableName, true); + for (String dbName : metastoreClient.getDatabases(schemaPattern)) { + List tableNames = metastoreClient.getTables(dbName, tablePattern); + for (Table table : metastoreClient.getTableObjectsByName(dbName, tableNames)) { + Object[] rowData = new Object[] { + DEFAULT_HIVE_CATALOG, + table.getDbName(), + table.getTableName(), + tableTypeMapping.mapToClientType(table.getTableType()), + table.getParameters().get("comment") + }; + if (tableTypes.isEmpty() || tableTypes.contains( + tableTypeMapping.mapToClientType(table.getTableType()))) { + rowSet.addRow(rowData); + } + } + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java new file mode 100644 index 000000000000..0f72071d7e7d --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java @@ -0,0 +1,142 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.Type; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetTypeInfoOperation. + * + */ +public class GetTypeInfoOperation extends MetadataOperation { + + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addPrimitiveColumn("TYPE_NAME", Type.STRING_TYPE, + "Type name") + .addPrimitiveColumn("DATA_TYPE", Type.INT_TYPE, + "SQL data type from java.sql.Types") + .addPrimitiveColumn("PRECISION", Type.INT_TYPE, + "Maximum precision") + .addPrimitiveColumn("LITERAL_PREFIX", Type.STRING_TYPE, + "Prefix used to quote a literal (may be null)") + .addPrimitiveColumn("LITERAL_SUFFIX", Type.STRING_TYPE, + "Suffix used to quote a literal (may be null)") + .addPrimitiveColumn("CREATE_PARAMS", Type.STRING_TYPE, + "Parameters used in creating the type (may be null)") + .addPrimitiveColumn("NULLABLE", Type.SMALLINT_TYPE, + "Can you use NULL for this type") + .addPrimitiveColumn("CASE_SENSITIVE", Type.BOOLEAN_TYPE, + "Is it case sensitive") + .addPrimitiveColumn("SEARCHABLE", Type.SMALLINT_TYPE, + "Can you use \"WHERE\" based on this type") + .addPrimitiveColumn("UNSIGNED_ATTRIBUTE", Type.BOOLEAN_TYPE, + "Is it unsigned") + .addPrimitiveColumn("FIXED_PREC_SCALE", Type.BOOLEAN_TYPE, + "Can it be a money value") + .addPrimitiveColumn("AUTO_INCREMENT", Type.BOOLEAN_TYPE, + "Can it be used for an auto-increment value") + .addPrimitiveColumn("LOCAL_TYPE_NAME", Type.STRING_TYPE, + "Localized version of type name (may be null)") + .addPrimitiveColumn("MINIMUM_SCALE", Type.SMALLINT_TYPE, + "Minimum scale supported") + .addPrimitiveColumn("MAXIMUM_SCALE", Type.SMALLINT_TYPE, + "Maximum scale supported") + .addPrimitiveColumn("SQL_DATA_TYPE", Type.INT_TYPE, + "Unused") + .addPrimitiveColumn("SQL_DATETIME_SUB", Type.INT_TYPE, + "Unused") + .addPrimitiveColumn("NUM_PREC_RADIX", Type.INT_TYPE, + "Usually 2 or 10"); + + private final RowSet rowSet; + + protected GetTypeInfoOperation(HiveSession parentSession) { + super(parentSession, OperationType.GET_TYPE_INFO); + rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion()); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + if (isAuthV2Enabled()) { + authorizeMetaGets(HiveOperationType.GET_TYPEINFO, null); + } + try { + for (Type type : Type.values()) { + Object[] rowData = new Object[] { + type.getName(), // TYPE_NAME + type.toJavaSQLType(), // DATA_TYPE + type.getMaxPrecision(), // PRECISION + type.getLiteralPrefix(), // LITERAL_PREFIX + type.getLiteralSuffix(), // LITERAL_SUFFIX + type.getCreateParams(), // CREATE_PARAMS + type.getNullable(), // NULLABLE + type.isCaseSensitive(), // CASE_SENSITIVE + type.getSearchable(), // SEARCHABLE + type.isUnsignedAttribute(), // UNSIGNED_ATTRIBUTE + type.isFixedPrecScale(), // FIXED_PREC_SCALE + type.isAutoIncrement(), // AUTO_INCREMENT + type.getLocalizedName(), // LOCAL_TYPE_NAME + type.getMinimumScale(), // MINIMUM_SCALE + type.getMaximumScale(), // MAXIMUM_SCALE + null, // SQL_DATA_TYPE, unused + null, // SQL_DATETIME_SUB, unused + type.getNumPrecRadix() //NUM_PREC_RADIX + }; + rowSet.addRow(rowData); + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java new file mode 100644 index 000000000000..bcc66cf811b2 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java @@ -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.hive.service.cli.operation; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FileReader; +import java.io.IOException; +import java.io.PrintStream; +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hive.metastore.api.Schema; +import org.apache.hadoop.hive.ql.processors.CommandProcessor; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.io.IOUtils; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * Executes a HiveCommand + */ +public class HiveCommandOperation extends ExecuteStatementOperation { + private CommandProcessor commandProcessor; + private TableSchema resultSchema = null; + + /** + * For processors other than Hive queries (Driver), they output to session.out (a temp file) + * first and the fetchOne/fetchN/fetchAll functions get the output from pipeIn. + */ + private BufferedReader resultReader; + + + protected HiveCommandOperation(HiveSession parentSession, String statement, + CommandProcessor commandProcessor, Map confOverlay) { + super(parentSession, statement, confOverlay, false); + this.commandProcessor = commandProcessor; + setupSessionIO(parentSession.getSessionState()); + } + + private void setupSessionIO(SessionState sessionState) { + try { + LOG.info("Putting temp output to file " + sessionState.getTmpOutputFile().toString()); + sessionState.in = null; // hive server's session input stream is not used + // open a per-session file in auto-flush mode for writing temp results + sessionState.out = new PrintStream(new FileOutputStream(sessionState.getTmpOutputFile()), true, "UTF-8"); + // TODO: for hadoop jobs, progress is printed out to session.err, + // we should find a way to feed back job progress to client + sessionState.err = new PrintStream(System.err, true, "UTF-8"); + } catch (IOException e) { + LOG.error("Error in creating temp output file ", e); + try { + sessionState.in = null; + sessionState.out = new PrintStream(System.out, true, "UTF-8"); + sessionState.err = new PrintStream(System.err, true, "UTF-8"); + } catch (UnsupportedEncodingException ee) { + LOG.error("Error creating PrintStream", e); + ee.printStackTrace(); + sessionState.out = null; + sessionState.err = null; + } + } + } + + + private void tearDownSessionIO() { + IOUtils.cleanup(LOG, parentSession.getSessionState().out); + IOUtils.cleanup(LOG, parentSession.getSessionState().err); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + try { + String command = getStatement().trim(); + String[] tokens = statement.split("\\s"); + String commandArgs = command.substring(tokens[0].length()).trim(); + + CommandProcessorResponse response = commandProcessor.run(commandArgs); + int returnCode = response.getResponseCode(); + if (returnCode != 0) { + throw toSQLException("Error while processing statement", response); + } + Schema schema = response.getSchema(); + if (schema != null) { + setHasResultSet(true); + resultSchema = new TableSchema(schema); + } else { + setHasResultSet(false); + resultSchema = new TableSchema(); + } + } catch (HiveSQLException e) { + setState(OperationState.ERROR); + throw e; + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException("Error running query: " + e.toString(), e); + } + setState(OperationState.FINISHED); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.operation.Operation#close() + */ + @Override + public void close() throws HiveSQLException { + setState(OperationState.CLOSED); + tearDownSessionIO(); + cleanTmpFile(); + cleanupOperationLog(); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.operation.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + return resultSchema; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.operation.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + resetResultReader(); + } + List rows = readResults((int) maxRows); + RowSet rowSet = RowSetFactory.create(resultSchema, getProtocolVersion()); + + for (String row : rows) { + rowSet.addRow(new String[] {row}); + } + return rowSet; + } + + /** + * Reads the temporary results for non-Hive (non-Driver) commands to the + * resulting List of strings. + * @param nLines number of lines read at once. If it is <= 0, then read all lines. + */ + private List readResults(int nLines) throws HiveSQLException { + if (resultReader == null) { + SessionState sessionState = getParentSession().getSessionState(); + File tmp = sessionState.getTmpOutputFile(); + try { + resultReader = new BufferedReader(new FileReader(tmp)); + } catch (FileNotFoundException e) { + LOG.error("File " + tmp + " not found. ", e); + throw new HiveSQLException(e); + } + } + List results = new ArrayList(); + + for (int i = 0; i < nLines || nLines <= 0; ++i) { + try { + String line = resultReader.readLine(); + if (line == null) { + // reached the end of the result file + break; + } else { + results.add(line); + } + } catch (IOException e) { + LOG.error("Reading temp results encountered an exception: ", e); + throw new HiveSQLException(e); + } + } + return results; + } + + private void cleanTmpFile() { + resetResultReader(); + SessionState sessionState = getParentSession().getSessionState(); + File tmp = sessionState.getTmpOutputFile(); + tmp.delete(); + } + + private void resetResultReader() { + if (resultReader != null) { + IOUtils.cleanup(LOG, resultReader); + resultReader = null; + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java new file mode 100644 index 000000000000..b530f217125b --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java @@ -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.hive.service.cli.operation; + +import java.util.HashSet; +import java.util.Set; + +import org.apache.hadoop.hive.metastore.TableType; + +/** + * HiveTableTypeMapping. + * Default table type mapping + * + */ +public class HiveTableTypeMapping implements TableTypeMapping { + + @Override + public String mapToHiveType(String clientTypeName) { + return clientTypeName; + } + + @Override + public String mapToClientType(String hiveTypeName) { + return hiveTypeName; + } + + @Override + public Set getTableTypeNames() { + Set typeNameSet = new HashSet(); + for (TableType typeNames : TableType.values()) { + typeNameSet.add(typeNames.toString()); + } + return typeNameSet; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java new file mode 100644 index 000000000000..cb804318ace9 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java @@ -0,0 +1,209 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; +import java.io.CharArrayWriter; +import java.util.Enumeration; +import java.util.regex.Pattern; + +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.log.PerfLogger; +import org.apache.hadoop.hive.ql.session.OperationLog; +import org.apache.hadoop.hive.ql.session.OperationLog.LoggingLevel; +import org.apache.hive.service.cli.CLIServiceUtils; +import org.apache.log4j.Appender; +import org.apache.log4j.ConsoleAppender; +import org.apache.log4j.Layout; +import org.apache.log4j.Logger; +import org.apache.log4j.WriterAppender; +import org.apache.log4j.spi.Filter; +import org.apache.log4j.spi.LoggingEvent; + +import com.google.common.base.Joiner; + +/** + * An Appender to divert logs from individual threads to the LogObject they belong to. + */ +public class LogDivertAppender extends WriterAppender { + private static final Logger LOG = Logger.getLogger(LogDivertAppender.class.getName()); + private final OperationManager operationManager; + private boolean isVerbose; + private Layout verboseLayout; + + /** + * A log filter that filters messages coming from the logger with the given names. + * It be used as a white list filter or a black list filter. + * We apply black list filter on the Loggers used by the log diversion stuff, so that + * they don't generate more logs for themselves when they process logs. + * White list filter is used for less verbose log collection + */ + private static class NameFilter extends Filter { + private Pattern namePattern; + private LoggingLevel loggingMode; + private OperationManager operationManager; + + /* Patterns that are excluded in verbose logging level. + * Filter out messages coming from log processing classes, or we'll run an infinite loop. + */ + private static final Pattern verboseExcludeNamePattern = Pattern.compile(Joiner.on("|") + .join(new String[] {LOG.getName(), OperationLog.class.getName(), + OperationManager.class.getName()})); + + /* Patterns that are included in execution logging level. + * In execution mode, show only select logger messages. + */ + private static final Pattern executionIncludeNamePattern = Pattern.compile(Joiner.on("|") + .join(new String[] {"org.apache.hadoop.mapreduce.JobSubmitter", + "org.apache.hadoop.mapreduce.Job", "SessionState", Task.class.getName(), + "org.apache.hadoop.hive.ql.exec.spark.status.SparkJobMonitor"})); + + /* Patterns that are included in performance logging level. + * In performance mode, show execution and performance logger messages. + */ + private static final Pattern performanceIncludeNamePattern = Pattern.compile( + executionIncludeNamePattern.pattern() + "|" + PerfLogger.class.getName()); + + private void setCurrentNamePattern(OperationLog.LoggingLevel mode) { + if (mode == OperationLog.LoggingLevel.VERBOSE) { + this.namePattern = verboseExcludeNamePattern; + } else if (mode == OperationLog.LoggingLevel.EXECUTION) { + this.namePattern = executionIncludeNamePattern; + } else if (mode == OperationLog.LoggingLevel.PERFORMANCE) { + this.namePattern = performanceIncludeNamePattern; + } + } + + NameFilter( + OperationLog.LoggingLevel loggingMode, OperationManager op) { + this.operationManager = op; + this.loggingMode = loggingMode; + setCurrentNamePattern(loggingMode); + } + + @Override + public int decide(LoggingEvent ev) { + OperationLog log = operationManager.getOperationLogByThread(); + boolean excludeMatches = (loggingMode == OperationLog.LoggingLevel.VERBOSE); + + if (log == null) { + return Filter.DENY; + } + + OperationLog.LoggingLevel currentLoggingMode = log.getOpLoggingLevel(); + // If logging is disabled, deny everything. + if (currentLoggingMode == OperationLog.LoggingLevel.NONE) { + return Filter.DENY; + } + // Look at the current session's setting + // and set the pattern and excludeMatches accordingly. + if (currentLoggingMode != loggingMode) { + loggingMode = currentLoggingMode; + setCurrentNamePattern(loggingMode); + } + + boolean isMatch = namePattern.matcher(ev.getLoggerName()).matches(); + + if (excludeMatches == isMatch) { + // Deny if this is black-list filter (excludeMatches = true) and it + // matched + // or if this is whitelist filter and it didn't match + return Filter.DENY; + } + return Filter.NEUTRAL; + } + } + + /** This is where the log message will go to */ + private final CharArrayWriter writer = new CharArrayWriter(); + + private void setLayout(boolean isVerbose, Layout lo) { + if (isVerbose) { + if (lo == null) { + lo = CLIServiceUtils.verboseLayout; + LOG.info("Cannot find a Layout from a ConsoleAppender. Using default Layout pattern."); + } + } else { + lo = CLIServiceUtils.nonVerboseLayout; + } + setLayout(lo); + } + + private void initLayout(boolean isVerbose) { + // There should be a ConsoleAppender. Copy its Layout. + Logger root = Logger.getRootLogger(); + Layout layout = null; + + Enumeration appenders = root.getAllAppenders(); + while (appenders.hasMoreElements()) { + Appender ap = (Appender) appenders.nextElement(); + if (ap.getClass().equals(ConsoleAppender.class)) { + layout = ap.getLayout(); + break; + } + } + setLayout(isVerbose, layout); + } + + public LogDivertAppender(OperationManager operationManager, + OperationLog.LoggingLevel loggingMode) { + isVerbose = (loggingMode == OperationLog.LoggingLevel.VERBOSE); + initLayout(isVerbose); + setWriter(writer); + setName("LogDivertAppender"); + this.operationManager = operationManager; + this.verboseLayout = isVerbose ? layout : CLIServiceUtils.verboseLayout; + addFilter(new NameFilter(loggingMode, operationManager)); + } + + @Override + public void doAppend(LoggingEvent event) { + OperationLog log = operationManager.getOperationLogByThread(); + + // Set current layout depending on the verbose/non-verbose mode. + if (log != null) { + boolean isCurrModeVerbose = (log.getOpLoggingLevel() == OperationLog.LoggingLevel.VERBOSE); + + // If there is a logging level change from verbose->non-verbose or vice-versa since + // the last subAppend call, change the layout to preserve consistency. + if (isCurrModeVerbose != isVerbose) { + isVerbose = isCurrModeVerbose; + setLayout(isVerbose, verboseLayout); + } + } + super.doAppend(event); + } + + /** + * Overrides WriterAppender.subAppend(), which does the real logging. No need + * to worry about concurrency since log4j calls this synchronously. + */ + @Override + protected void subAppend(LoggingEvent event) { + super.subAppend(event); + // That should've gone into our writer. Notify the LogContext. + String logOutput = writer.toString(); + writer.reset(); + + OperationLog log = operationManager.getOperationLogByThread(); + if (log == null) { + LOG.debug(" ---+++=== Dropped log event from thread " + event.getThreadName()); + return; + } + log.writeOperationLog(logOutput); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java new file mode 100644 index 000000000000..6c819876a556 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java @@ -0,0 +1,134 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.util.List; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * MetadataOperation. + * + */ +public abstract class MetadataOperation extends Operation { + + protected static final String DEFAULT_HIVE_CATALOG = ""; + protected static TableSchema RESULT_SET_SCHEMA; + private static final char SEARCH_STRING_ESCAPE = '\\'; + + protected MetadataOperation(HiveSession parentSession, OperationType opType) { + super(parentSession, opType, false); + setHasResultSet(true); + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#close() + */ + @Override + public void close() throws HiveSQLException { + setState(OperationState.CLOSED); + cleanupOperationLog(); + } + + /** + * Convert wildchars and escape sequence from JDBC format to datanucleous/regex + */ + protected String convertIdentifierPattern(final String pattern, boolean datanucleusFormat) { + if (pattern == null) { + return convertPattern("%", true); + } else { + return convertPattern(pattern, datanucleusFormat); + } + } + + /** + * Convert wildchars and escape sequence of schema pattern from JDBC format to datanucleous/regex + * The schema pattern treats empty string also as wildchar + */ + protected String convertSchemaPattern(final String pattern) { + if ((pattern == null) || pattern.isEmpty()) { + return convertPattern("%", true); + } else { + return convertPattern(pattern, true); + } + } + + /** + * Convert a pattern containing JDBC catalog search wildcards into + * Java regex patterns. + * + * @param pattern input which may contain '%' or '_' wildcard characters, or + * these characters escaped using {@link #getSearchStringEscape()}. + * @return replace %/_ with regex search characters, also handle escaped + * characters. + * + * The datanucleus module expects the wildchar as '*'. The columns search on the + * other hand is done locally inside the hive code and that requires the regex wildchar + * format '.*' This is driven by the datanucleusFormat flag. + */ + private String convertPattern(final String pattern, boolean datanucleusFormat) { + String wStr; + if (datanucleusFormat) { + wStr = "*"; + } else { + wStr = ".*"; + } + return pattern + .replaceAll("([^\\\\])%", "$1" + wStr).replaceAll("\\\\%", "%").replaceAll("^%", wStr) + .replaceAll("([^\\\\])_", "$1.").replaceAll("\\\\_", "_").replaceAll("^_", "."); + } + + protected boolean isAuthV2Enabled(){ + SessionState ss = SessionState.get(); + return (ss.isAuthorizationModeV2() && + HiveConf.getBoolVar(ss.getConf(), HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)); + } + + protected void authorizeMetaGets(HiveOperationType opType, List inpObjs) + throws HiveSQLException { + authorizeMetaGets(opType, inpObjs, null); + } + + protected void authorizeMetaGets(HiveOperationType opType, List inpObjs, + String cmdString) throws HiveSQLException { + SessionState ss = SessionState.get(); + HiveAuthzContext.Builder ctxBuilder = new HiveAuthzContext.Builder(); + ctxBuilder.setUserIpAddress(ss.getUserIpAddress()); + ctxBuilder.setCommandString(cmdString); + try { + ss.getAuthorizerV2().checkPrivileges(opType, inpObjs, null, + ctxBuilder.build()); + } catch (HiveAuthzPluginException | HiveAccessControlException e) { + throw new HiveSQLException(e.getMessage(), e); + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java new file mode 100644 index 000000000000..19153b654b08 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java @@ -0,0 +1,322 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service.cli.operation; + +import java.io.File; +import java.io.FileNotFoundException; +import java.util.EnumSet; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.OperationLog; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationHandle; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationStatus; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; +import org.apache.hive.service.cli.thrift.TProtocolVersion; + +public abstract class Operation { + protected final HiveSession parentSession; + private OperationState state = OperationState.INITIALIZED; + private final OperationHandle opHandle; + private HiveConf configuration; + public static final Log LOG = LogFactory.getLog(Operation.class.getName()); + public static final FetchOrientation DEFAULT_FETCH_ORIENTATION = FetchOrientation.FETCH_NEXT; + public static final long DEFAULT_FETCH_MAX_ROWS = 100; + protected boolean hasResultSet; + protected volatile HiveSQLException operationException; + protected final boolean runAsync; + protected volatile Future backgroundHandle; + protected OperationLog operationLog; + protected boolean isOperationLogEnabled; + + private long operationTimeout; + private long lastAccessTime; + + protected static final EnumSet DEFAULT_FETCH_ORIENTATION_SET = + EnumSet.of(FetchOrientation.FETCH_NEXT,FetchOrientation.FETCH_FIRST); + + protected Operation(HiveSession parentSession, OperationType opType, boolean runInBackground) { + this.parentSession = parentSession; + this.runAsync = runInBackground; + this.opHandle = new OperationHandle(opType, parentSession.getProtocolVersion()); + lastAccessTime = System.currentTimeMillis(); + operationTimeout = HiveConf.getTimeVar(parentSession.getHiveConf(), + HiveConf.ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT, TimeUnit.MILLISECONDS); + } + + public Future getBackgroundHandle() { + return backgroundHandle; + } + + protected void setBackgroundHandle(Future backgroundHandle) { + this.backgroundHandle = backgroundHandle; + } + + public boolean shouldRunAsync() { + return runAsync; + } + + public void setConfiguration(HiveConf configuration) { + this.configuration = new HiveConf(configuration); + } + + public HiveConf getConfiguration() { + return new HiveConf(configuration); + } + + public HiveSession getParentSession() { + return parentSession; + } + + public OperationHandle getHandle() { + return opHandle; + } + + public TProtocolVersion getProtocolVersion() { + return opHandle.getProtocolVersion(); + } + + public OperationType getType() { + return opHandle.getOperationType(); + } + + public OperationStatus getStatus() { + return new OperationStatus(state, operationException); + } + + public boolean hasResultSet() { + return hasResultSet; + } + + protected void setHasResultSet(boolean hasResultSet) { + this.hasResultSet = hasResultSet; + opHandle.setHasResultSet(hasResultSet); + } + + public OperationLog getOperationLog() { + return operationLog; + } + + protected final OperationState setState(OperationState newState) throws HiveSQLException { + state.validateTransition(newState); + this.state = newState; + this.lastAccessTime = System.currentTimeMillis(); + return this.state; + } + + public boolean isTimedOut(long current) { + if (operationTimeout == 0) { + return false; + } + if (operationTimeout > 0) { + // check only when it's in terminal state + return state.isTerminal() && lastAccessTime + operationTimeout <= current; + } + return lastAccessTime + -operationTimeout <= current; + } + + public long getLastAccessTime() { + return lastAccessTime; + } + + public long getOperationTimeout() { + return operationTimeout; + } + + public void setOperationTimeout(long operationTimeout) { + this.operationTimeout = operationTimeout; + } + + protected void setOperationException(HiveSQLException operationException) { + this.operationException = operationException; + } + + protected final void assertState(OperationState state) throws HiveSQLException { + if (this.state != state) { + throw new HiveSQLException("Expected state " + state + ", but found " + this.state); + } + this.lastAccessTime = System.currentTimeMillis(); + } + + public boolean isRunning() { + return OperationState.RUNNING.equals(state); + } + + public boolean isFinished() { + return OperationState.FINISHED.equals(state); + } + + public boolean isCanceled() { + return OperationState.CANCELED.equals(state); + } + + public boolean isFailed() { + return OperationState.ERROR.equals(state); + } + + protected void createOperationLog() { + if (parentSession.isOperationLogEnabled()) { + File operationLogFile = new File(parentSession.getOperationLogSessionDir(), + opHandle.getHandleIdentifier().toString()); + isOperationLogEnabled = true; + + // create log file + try { + if (operationLogFile.exists()) { + LOG.warn("The operation log file should not exist, but it is already there: " + + operationLogFile.getAbsolutePath()); + operationLogFile.delete(); + } + if (!operationLogFile.createNewFile()) { + // the log file already exists and cannot be deleted. + // If it can be read/written, keep its contents and use it. + if (!operationLogFile.canRead() || !operationLogFile.canWrite()) { + LOG.warn("The already existed operation log file cannot be recreated, " + + "and it cannot be read or written: " + operationLogFile.getAbsolutePath()); + isOperationLogEnabled = false; + return; + } + } + } catch (Exception e) { + LOG.warn("Unable to create operation log file: " + operationLogFile.getAbsolutePath(), e); + isOperationLogEnabled = false; + return; + } + + // create OperationLog object with above log file + try { + operationLog = new OperationLog(opHandle.toString(), operationLogFile, parentSession.getHiveConf()); + } catch (FileNotFoundException e) { + LOG.warn("Unable to instantiate OperationLog object for operation: " + + opHandle, e); + isOperationLogEnabled = false; + return; + } + + // register this operationLog to current thread + OperationLog.setCurrentOperationLog(operationLog); + } + } + + protected void unregisterOperationLog() { + if (isOperationLogEnabled) { + OperationLog.removeCurrentOperationLog(); + } + } + + /** + * Invoked before runInternal(). + * Set up some preconditions, or configurations. + */ + protected void beforeRun() { + createOperationLog(); + } + + /** + * Invoked after runInternal(), even if an exception is thrown in runInternal(). + * Clean up resources, which was set up in beforeRun(). + */ + protected void afterRun() { + unregisterOperationLog(); + } + + /** + * Implemented by subclass of Operation class to execute specific behaviors. + * @throws HiveSQLException + */ + protected abstract void runInternal() throws HiveSQLException; + + public void run() throws HiveSQLException { + beforeRun(); + try { + runInternal(); + } finally { + afterRun(); + } + } + + protected void cleanupOperationLog() { + if (isOperationLogEnabled) { + if (operationLog == null) { + LOG.error("Operation [ " + opHandle.getHandleIdentifier() + " ] " + + "logging is enabled, but its OperationLog object cannot be found."); + } else { + operationLog.close(); + } + } + } + + // TODO: make this abstract and implement in subclasses. + public void cancel() throws HiveSQLException { + setState(OperationState.CANCELED); + throw new UnsupportedOperationException("SQLOperation.cancel()"); + } + + public abstract void close() throws HiveSQLException; + + public abstract TableSchema getResultSetSchema() throws HiveSQLException; + + public abstract RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException; + + public RowSet getNextRowSet() throws HiveSQLException { + return getNextRowSet(FetchOrientation.FETCH_NEXT, DEFAULT_FETCH_MAX_ROWS); + } + + /** + * Verify if the given fetch orientation is part of the default orientation types. + * @param orientation + * @throws HiveSQLException + */ + protected void validateDefaultFetchOrientation(FetchOrientation orientation) + throws HiveSQLException { + validateFetchOrientation(orientation, DEFAULT_FETCH_ORIENTATION_SET); + } + + /** + * Verify if the given fetch orientation is part of the supported orientation types. + * @param orientation + * @param supportedOrientations + * @throws HiveSQLException + */ + protected void validateFetchOrientation(FetchOrientation orientation, + EnumSet supportedOrientations) throws HiveSQLException { + if (!supportedOrientations.contains(orientation)) { + throw new HiveSQLException("The fetch type " + orientation.toString() + + " is not supported for this resultset", "HY106"); + } + } + + protected HiveSQLException toSQLException(String prefix, CommandProcessorResponse response) { + HiveSQLException ex = new HiveSQLException(prefix + ": " + response.getErrorMessage(), + response.getSQLState(), response.getResponseCode()); + if (response.getException() != null) { + ex.initCause(response.getException()); + } + return ex; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java new file mode 100644 index 000000000000..92c340a29c10 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java @@ -0,0 +1,284 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Schema; +import org.apache.hadoop.hive.ql.session.OperationLog; +import org.apache.hive.service.AbstractService; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationHandle; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationStatus; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; +import org.apache.log4j.Appender; +import org.apache.log4j.Logger; + +/** + * OperationManager. + * + */ +public class OperationManager extends AbstractService { + private final Log LOG = LogFactory.getLog(OperationManager.class.getName()); + + private final Map handleToOperation = + new HashMap(); + + public OperationManager() { + super(OperationManager.class.getSimpleName()); + } + + @Override + public synchronized void init(HiveConf hiveConf) { + if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) { + initOperationLogCapture(hiveConf.getVar( + HiveConf.ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL)); + } else { + LOG.debug("Operation level logging is turned off"); + } + super.init(hiveConf); + } + + @Override + public synchronized void start() { + super.start(); + // TODO + } + + @Override + public synchronized void stop() { + // TODO + super.stop(); + } + + private void initOperationLogCapture(String loggingMode) { + // Register another Appender (with the same layout) that talks to us. + Appender ap = new LogDivertAppender(this, OperationLog.getLoggingLevel(loggingMode)); + Logger.getRootLogger().addAppender(ap); + } + + public ExecuteStatementOperation newExecuteStatementOperation(HiveSession parentSession, + String statement, Map confOverlay, boolean runAsync) + throws HiveSQLException { + ExecuteStatementOperation executeStatementOperation = ExecuteStatementOperation + .newExecuteStatementOperation(parentSession, statement, confOverlay, runAsync); + addOperation(executeStatementOperation); + return executeStatementOperation; + } + + public GetTypeInfoOperation newGetTypeInfoOperation(HiveSession parentSession) { + GetTypeInfoOperation operation = new GetTypeInfoOperation(parentSession); + addOperation(operation); + return operation; + } + + public GetCatalogsOperation newGetCatalogsOperation(HiveSession parentSession) { + GetCatalogsOperation operation = new GetCatalogsOperation(parentSession); + addOperation(operation); + return operation; + } + + public GetSchemasOperation newGetSchemasOperation(HiveSession parentSession, + String catalogName, String schemaName) { + GetSchemasOperation operation = new GetSchemasOperation(parentSession, catalogName, schemaName); + addOperation(operation); + return operation; + } + + public MetadataOperation newGetTablesOperation(HiveSession parentSession, + String catalogName, String schemaName, String tableName, + List tableTypes) { + MetadataOperation operation = + new GetTablesOperation(parentSession, catalogName, schemaName, tableName, tableTypes); + addOperation(operation); + return operation; + } + + public GetTableTypesOperation newGetTableTypesOperation(HiveSession parentSession) { + GetTableTypesOperation operation = new GetTableTypesOperation(parentSession); + addOperation(operation); + return operation; + } + + public GetColumnsOperation newGetColumnsOperation(HiveSession parentSession, + String catalogName, String schemaName, String tableName, String columnName) { + GetColumnsOperation operation = new GetColumnsOperation(parentSession, + catalogName, schemaName, tableName, columnName); + addOperation(operation); + return operation; + } + + public GetFunctionsOperation newGetFunctionsOperation(HiveSession parentSession, + String catalogName, String schemaName, String functionName) { + GetFunctionsOperation operation = new GetFunctionsOperation(parentSession, + catalogName, schemaName, functionName); + addOperation(operation); + return operation; + } + + public Operation getOperation(OperationHandle operationHandle) throws HiveSQLException { + Operation operation = getOperationInternal(operationHandle); + if (operation == null) { + throw new HiveSQLException("Invalid OperationHandle: " + operationHandle); + } + return operation; + } + + private synchronized Operation getOperationInternal(OperationHandle operationHandle) { + return handleToOperation.get(operationHandle); + } + + private synchronized Operation removeTimedOutOperation(OperationHandle operationHandle) { + Operation operation = handleToOperation.get(operationHandle); + if (operation != null && operation.isTimedOut(System.currentTimeMillis())) { + handleToOperation.remove(operationHandle); + return operation; + } + return null; + } + + private synchronized void addOperation(Operation operation) { + handleToOperation.put(operation.getHandle(), operation); + } + + private synchronized Operation removeOperation(OperationHandle opHandle) { + return handleToOperation.remove(opHandle); + } + + public OperationStatus getOperationStatus(OperationHandle opHandle) + throws HiveSQLException { + return getOperation(opHandle).getStatus(); + } + + public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { + Operation operation = getOperation(opHandle); + OperationState opState = operation.getStatus().getState(); + if (opState == OperationState.CANCELED || + opState == OperationState.CLOSED || + opState == OperationState.FINISHED || + opState == OperationState.ERROR || + opState == OperationState.UNKNOWN) { + // Cancel should be a no-op in either cases + LOG.debug(opHandle + ": Operation is already aborted in state - " + opState); + } + else { + LOG.debug(opHandle + ": Attempting to cancel from state - " + opState); + operation.cancel(); + } + } + + public void closeOperation(OperationHandle opHandle) throws HiveSQLException { + Operation operation = removeOperation(opHandle); + if (operation == null) { + throw new HiveSQLException("Operation does not exist!"); + } + operation.close(); + } + + public TableSchema getOperationResultSetSchema(OperationHandle opHandle) + throws HiveSQLException { + return getOperation(opHandle).getResultSetSchema(); + } + + public RowSet getOperationNextRowSet(OperationHandle opHandle) + throws HiveSQLException { + return getOperation(opHandle).getNextRowSet(); + } + + public RowSet getOperationNextRowSet(OperationHandle opHandle, + FetchOrientation orientation, long maxRows) + throws HiveSQLException { + return getOperation(opHandle).getNextRowSet(orientation, maxRows); + } + + public RowSet getOperationLogRowSet(OperationHandle opHandle, + FetchOrientation orientation, long maxRows) + throws HiveSQLException { + // get the OperationLog object from the operation + OperationLog operationLog = getOperation(opHandle).getOperationLog(); + if (operationLog == null) { + throw new HiveSQLException("Couldn't find log associated with operation handle: " + opHandle); + } + + // read logs + List logs; + try { + logs = operationLog.readOperationLog(isFetchFirst(orientation), maxRows); + } catch (SQLException e) { + throw new HiveSQLException(e.getMessage(), e.getCause()); + } + + + // convert logs to RowSet + TableSchema tableSchema = new TableSchema(getLogSchema()); + RowSet rowSet = RowSetFactory.create(tableSchema, getOperation(opHandle).getProtocolVersion()); + for (String log : logs) { + rowSet.addRow(new String[] {log}); + } + + return rowSet; + } + + private boolean isFetchFirst(FetchOrientation fetchOrientation) { + //TODO: Since OperationLog is moved to package o.a.h.h.ql.session, + // we may add a Enum there and map FetchOrientation to it. + if (fetchOrientation.equals(FetchOrientation.FETCH_FIRST)) { + return true; + } + return false; + } + + private Schema getLogSchema() { + Schema schema = new Schema(); + FieldSchema fieldSchema = new FieldSchema(); + fieldSchema.setName("operation_log"); + fieldSchema.setType("string"); + schema.addToFieldSchemas(fieldSchema); + return schema; + } + + public OperationLog getOperationLogByThread() { + return OperationLog.getCurrentOperationLog(); + } + + public List removeExpiredOperations(OperationHandle[] handles) { + List removed = new ArrayList(); + for (OperationHandle handle : handles) { + Operation operation = removeTimedOutOperation(handle); + if (operation != null) { + LOG.warn("Operation " + handle + " is timed-out and will be closed"); + removed.add(operation); + } + } + return removed; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java new file mode 100644 index 000000000000..5014cedd870b --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java @@ -0,0 +1,473 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UnsupportedEncodingException; +import java.security.PrivilegedExceptionAction; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.Future; +import java.util.concurrent.RejectedExecutionException; + +import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Schema; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.exec.ExplainTask; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.VariableSubstitution; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.OperationLog; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.SerDeUtils; +import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.shims.Utils; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; +import org.apache.hive.service.server.ThreadWithGarbageCleanup; + +/** + * SQLOperation. + * + */ +public class SQLOperation extends ExecuteStatementOperation { + + private Driver driver = null; + private CommandProcessorResponse response; + private TableSchema resultSchema = null; + private Schema mResultSchema = null; + private SerDe serde = null; + private boolean fetchStarted = false; + + public SQLOperation(HiveSession parentSession, String statement, Map confOverlay, boolean runInBackground) { + // TODO: call setRemoteUser in ExecuteStatementOperation or higher. + super(parentSession, statement, confOverlay, runInBackground); + } + + /*** + * Compile the query and extract metadata + * @param sqlOperationConf + * @throws HiveSQLException + */ + public void prepare(HiveConf sqlOperationConf) throws HiveSQLException { + setState(OperationState.RUNNING); + + try { + driver = new Driver(sqlOperationConf, getParentSession().getUserName()); + + // set the operation handle information in Driver, so that thrift API users + // can use the operation handle they receive, to lookup query information in + // Yarn ATS + String guid64 = Base64.encodeBase64URLSafeString(getHandle().getHandleIdentifier() + .toTHandleIdentifier().getGuid()).trim(); + driver.setOperationId(guid64); + + // In Hive server mode, we are not able to retry in the FetchTask + // case, when calling fetch queries since execute() has returned. + // For now, we disable the test attempts. + driver.setTryCount(Integer.MAX_VALUE); + + String subStatement = new VariableSubstitution().substitute(sqlOperationConf, statement); + response = driver.compileAndRespond(subStatement); + if (0 != response.getResponseCode()) { + throw toSQLException("Error while compiling statement", response); + } + + mResultSchema = driver.getSchema(); + + // hasResultSet should be true only if the query has a FetchTask + // "explain" is an exception for now + if(driver.getPlan().getFetchTask() != null) { + //Schema has to be set + if (mResultSchema == null || !mResultSchema.isSetFieldSchemas()) { + throw new HiveSQLException("Error compiling query: Schema and FieldSchema " + + "should be set when query plan has a FetchTask"); + } + resultSchema = new TableSchema(mResultSchema); + setHasResultSet(true); + } else { + setHasResultSet(false); + } + // Set hasResultSet true if the plan has ExplainTask + // TODO explain should use a FetchTask for reading + for (Task task: driver.getPlan().getRootTasks()) { + if (task.getClass() == ExplainTask.class) { + resultSchema = new TableSchema(mResultSchema); + setHasResultSet(true); + break; + } + } + } catch (HiveSQLException e) { + setState(OperationState.ERROR); + throw e; + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException("Error running query: " + e.toString(), e); + } + } + + private void runQuery(HiveConf sqlOperationConf) throws HiveSQLException { + try { + // In Hive server mode, we are not able to retry in the FetchTask + // case, when calling fetch queries since execute() has returned. + // For now, we disable the test attempts. + driver.setTryCount(Integer.MAX_VALUE); + response = driver.run(); + if (0 != response.getResponseCode()) { + throw toSQLException("Error while processing statement", response); + } + } catch (HiveSQLException e) { + // If the operation was cancelled by another thread, + // Driver#run will return a non-zero response code. + // We will simply return if the operation state is CANCELED, + // otherwise throw an exception + if (getStatus().getState() == OperationState.CANCELED) { + return; + } + else { + setState(OperationState.ERROR); + throw e; + } + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException("Error running query: " + e.toString(), e); + } + setState(OperationState.FINISHED); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.PENDING); + final HiveConf opConfig = getConfigForOperation(); + prepare(opConfig); + if (!shouldRunAsync()) { + runQuery(opConfig); + } else { + // We'll pass ThreadLocals in the background thread from the foreground (handler) thread + final SessionState parentSessionState = SessionState.get(); + // ThreadLocal Hive object needs to be set in background thread. + // The metastore client in Hive is associated with right user. + final Hive parentHive = getSessionHive(); + // Current UGI will get used by metastore when metsatore is in embedded mode + // So this needs to get passed to the new background thread + final UserGroupInformation currentUGI = getCurrentUGI(opConfig); + // Runnable impl to call runInternal asynchronously, + // from a different thread + Runnable backgroundOperation = new Runnable() { + @Override + public void run() { + PrivilegedExceptionAction doAsAction = new PrivilegedExceptionAction() { + @Override + public Object run() throws HiveSQLException { + Hive.set(parentHive); + SessionState.setCurrentSessionState(parentSessionState); + // Set current OperationLog in this async thread for keeping on saving query log. + registerCurrentOperationLog(); + try { + runQuery(opConfig); + } catch (HiveSQLException e) { + setOperationException(e); + LOG.error("Error running hive query: ", e); + } finally { + unregisterOperationLog(); + } + return null; + } + }; + + try { + currentUGI.doAs(doAsAction); + } catch (Exception e) { + setOperationException(new HiveSQLException(e)); + LOG.error("Error running hive query as user : " + currentUGI.getShortUserName(), e); + } + finally { + /** + * We'll cache the ThreadLocal RawStore object for this background thread for an orderly cleanup + * when this thread is garbage collected later. + * @see org.apache.hive.service.server.ThreadWithGarbageCleanup#finalize() + */ + if (ThreadWithGarbageCleanup.currentThread() instanceof ThreadWithGarbageCleanup) { + ThreadWithGarbageCleanup currentThread = + (ThreadWithGarbageCleanup) ThreadWithGarbageCleanup.currentThread(); + currentThread.cacheThreadLocalRawStore(); + } + } + } + }; + try { + // This submit blocks if no background threads are available to run this operation + Future backgroundHandle = + getParentSession().getSessionManager().submitBackgroundOperation(backgroundOperation); + setBackgroundHandle(backgroundHandle); + } catch (RejectedExecutionException rejected) { + setState(OperationState.ERROR); + throw new HiveSQLException("The background threadpool cannot accept" + + " new task for execution, please retry the operation", rejected); + } + } + } + + /** + * Returns the current UGI on the stack + * @param opConfig + * @return UserGroupInformation + * @throws HiveSQLException + */ + private UserGroupInformation getCurrentUGI(HiveConf opConfig) throws HiveSQLException { + try { + return Utils.getUGI(); + } catch (Exception e) { + throw new HiveSQLException("Unable to get current user", e); + } + } + + /** + * Returns the ThreadLocal Hive for the current thread + * @return Hive + * @throws HiveSQLException + */ + private Hive getSessionHive() throws HiveSQLException { + try { + return Hive.get(); + } catch (HiveException e) { + throw new HiveSQLException("Failed to get ThreadLocal Hive object", e); + } + } + + private void registerCurrentOperationLog() { + if (isOperationLogEnabled) { + if (operationLog == null) { + LOG.warn("Failed to get current OperationLog object of Operation: " + + getHandle().getHandleIdentifier()); + isOperationLogEnabled = false; + return; + } + OperationLog.setCurrentOperationLog(operationLog); + } + } + + private void cleanup(OperationState state) throws HiveSQLException { + setState(state); + if (shouldRunAsync()) { + Future backgroundHandle = getBackgroundHandle(); + if (backgroundHandle != null) { + backgroundHandle.cancel(true); + } + } + if (driver != null) { + driver.close(); + driver.destroy(); + } + driver = null; + + SessionState ss = SessionState.get(); + if (ss.getTmpOutputFile() != null) { + ss.getTmpOutputFile().delete(); + } + } + + @Override + public void cancel() throws HiveSQLException { + cleanup(OperationState.CANCELED); + } + + @Override + public void close() throws HiveSQLException { + cleanup(OperationState.CLOSED); + cleanupOperationLog(); + } + + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + if (resultSchema == null) { + resultSchema = new TableSchema(driver.getSchema()); + } + return resultSchema; + } + + private final transient List convey = new ArrayList(); + + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + validateDefaultFetchOrientation(orientation); + assertState(OperationState.FINISHED); + + RowSet rowSet = RowSetFactory.create(resultSchema, getProtocolVersion()); + + try { + /* if client is requesting fetch-from-start and its not the first time reading from this operation + * then reset the fetch position to beginning + */ + if (orientation.equals(FetchOrientation.FETCH_FIRST) && fetchStarted) { + driver.resetFetch(); + } + fetchStarted = true; + driver.setMaxRows((int) maxRows); + if (driver.getResults(convey)) { + return decode(convey, rowSet); + } + return rowSet; + } catch (IOException e) { + throw new HiveSQLException(e); + } catch (CommandNeedRetryException e) { + throw new HiveSQLException(e); + } catch (Exception e) { + throw new HiveSQLException(e); + } finally { + convey.clear(); + } + } + + private RowSet decode(List rows, RowSet rowSet) throws Exception { + if (driver.isFetchingTable()) { + return prepareFromRow(rows, rowSet); + } + return decodeFromString(rows, rowSet); + } + + // already encoded to thrift-able object in ThriftFormatter + private RowSet prepareFromRow(List rows, RowSet rowSet) throws Exception { + for (Object row : rows) { + rowSet.addRow((Object[]) row); + } + return rowSet; + } + + private RowSet decodeFromString(List rows, RowSet rowSet) + throws SQLException, SerDeException { + getSerDe(); + StructObjectInspector soi = (StructObjectInspector) serde.getObjectInspector(); + List fieldRefs = soi.getAllStructFieldRefs(); + + Object[] deserializedFields = new Object[fieldRefs.size()]; + Object rowObj; + ObjectInspector fieldOI; + + int protocol = getProtocolVersion().getValue(); + for (Object rowString : rows) { + try { + rowObj = serde.deserialize(new BytesWritable(((String)rowString).getBytes("UTF-8"))); + } catch (UnsupportedEncodingException e) { + throw new SerDeException(e); + } + for (int i = 0; i < fieldRefs.size(); i++) { + StructField fieldRef = fieldRefs.get(i); + fieldOI = fieldRef.getFieldObjectInspector(); + Object fieldData = soi.getStructFieldData(rowObj, fieldRef); + deserializedFields[i] = SerDeUtils.toThriftPayload(fieldData, fieldOI, protocol); + } + rowSet.addRow(deserializedFields); + } + return rowSet; + } + + private SerDe getSerDe() throws SQLException { + if (serde != null) { + return serde; + } + try { + List fieldSchemas = mResultSchema.getFieldSchemas(); + StringBuilder namesSb = new StringBuilder(); + StringBuilder typesSb = new StringBuilder(); + + if (fieldSchemas != null && !fieldSchemas.isEmpty()) { + for (int pos = 0; pos < fieldSchemas.size(); pos++) { + if (pos != 0) { + namesSb.append(","); + typesSb.append(","); + } + namesSb.append(fieldSchemas.get(pos).getName()); + typesSb.append(fieldSchemas.get(pos).getType()); + } + } + String names = namesSb.toString(); + String types = typesSb.toString(); + + serde = new LazySimpleSerDe(); + Properties props = new Properties(); + if (names.length() > 0) { + LOG.debug("Column names: " + names); + props.setProperty(serdeConstants.LIST_COLUMNS, names); + } + if (types.length() > 0) { + LOG.debug("Column types: " + types); + props.setProperty(serdeConstants.LIST_COLUMN_TYPES, types); + } + SerDeUtils.initializeSerDe(serde, new HiveConf(), props, null); + + } catch (Exception ex) { + ex.printStackTrace(); + throw new SQLException("Could not create ResultSet: " + ex.getMessage(), ex); + } + return serde; + } + + /** + * If there are query specific settings to overlay, then create a copy of config + * There are two cases we need to clone the session config that's being passed to hive driver + * 1. Async query - + * If the client changes a config setting, that shouldn't reflect in the execution already underway + * 2. confOverlay - + * The query specific settings should only be applied to the query config and not session + * @return new configuration + * @throws HiveSQLException + */ + private HiveConf getConfigForOperation() throws HiveSQLException { + HiveConf sqlOperationConf = getParentSession().getHiveConf(); + if (!getConfOverlay().isEmpty() || shouldRunAsync()) { + // clone the parent session config for this query + sqlOperationConf = new HiveConf(sqlOperationConf); + + // apply overlay query specific settings, if any + for (Map.Entry confEntry : getConfOverlay().entrySet()) { + try { + sqlOperationConf.verifyAndSet(confEntry.getKey(), confEntry.getValue()); + } catch (IllegalArgumentException e) { + throw new HiveSQLException("Error applying statement specific settings", e); + } + } + } + return sqlOperationConf; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java new file mode 100644 index 000000000000..e59d19ea6be4 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.util.Set; + + +public interface TableTypeMapping { + /** + * Map client's table type name to hive's table type + * @param clientTypeName + * @return + */ + String mapToHiveType(String clientTypeName); + + /** + * Map hive's table type name to client's table type + * @param hiveTypeName + * @return + */ + String mapToClientType(String hiveTypeName); + + /** + * Get all the table types of this mapping + * @return + */ + Set getTableTypeNames(); +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMappingFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMappingFactory.java new file mode 100644 index 000000000000..d8ac2696b3d5 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMappingFactory.java @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +public class TableTypeMappingFactory { + + public enum TableTypeMappings { + HIVE, + CLASSIC + } + private static TableTypeMapping hiveTableTypeMapping = new HiveTableTypeMapping(); + private static TableTypeMapping classicTableTypeMapping = new ClassicTableTypeMapping(); + + public static TableTypeMapping getTableTypeMapping(String mappingType) { + if (TableTypeMappings.CLASSIC.toString().equalsIgnoreCase(mappingType)) { + return classicTableTypeMapping; + } else { + return hiveTableTypeMapping; + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSession.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSession.java new file mode 100644 index 000000000000..65f9b298bf4f --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSession.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.hive.service.cli.session; + +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.*; + +public interface HiveSession extends HiveSessionBase { + + void open(Map sessionConfMap) throws Exception; + + IMetaStoreClient getMetaStoreClient() throws HiveSQLException; + + /** + * getInfo operation handler + * @param getInfoType + * @return + * @throws HiveSQLException + */ + GetInfoValue getInfo(GetInfoType getInfoType) throws HiveSQLException; + + /** + * execute operation handler + * @param statement + * @param confOverlay + * @return + * @throws HiveSQLException + */ + OperationHandle executeStatement(String statement, + Map confOverlay) throws HiveSQLException; + + /** + * execute operation handler + * @param statement + * @param confOverlay + * @return + * @throws HiveSQLException + */ + OperationHandle executeStatementAsync(String statement, + Map confOverlay) throws HiveSQLException; + + /** + * getTypeInfo operation handler + * @return + * @throws HiveSQLException + */ + OperationHandle getTypeInfo() throws HiveSQLException; + + /** + * getCatalogs operation handler + * @return + * @throws HiveSQLException + */ + OperationHandle getCatalogs() throws HiveSQLException; + + /** + * getSchemas operation handler + * @param catalogName + * @param schemaName + * @return + * @throws HiveSQLException + */ + OperationHandle getSchemas(String catalogName, String schemaName) + throws HiveSQLException; + + /** + * getTables operation handler + * @param catalogName + * @param schemaName + * @param tableName + * @param tableTypes + * @return + * @throws HiveSQLException + */ + OperationHandle getTables(String catalogName, String schemaName, + String tableName, List tableTypes) throws HiveSQLException; + + /** + * getTableTypes operation handler + * @return + * @throws HiveSQLException + */ + OperationHandle getTableTypes() throws HiveSQLException ; + + /** + * getColumns operation handler + * @param catalogName + * @param schemaName + * @param tableName + * @param columnName + * @return + * @throws HiveSQLException + */ + OperationHandle getColumns(String catalogName, String schemaName, + String tableName, String columnName) throws HiveSQLException; + + /** + * getFunctions operation handler + * @param catalogName + * @param schemaName + * @param functionName + * @return + * @throws HiveSQLException + */ + OperationHandle getFunctions(String catalogName, String schemaName, + String functionName) throws HiveSQLException; + + /** + * close the session + * @throws HiveSQLException + */ + void close() throws HiveSQLException; + + void cancelOperation(OperationHandle opHandle) throws HiveSQLException; + + void closeOperation(OperationHandle opHandle) throws HiveSQLException; + + TableSchema getResultSetMetadata(OperationHandle opHandle) + throws HiveSQLException; + + RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, + long maxRows, FetchType fetchType) throws HiveSQLException; + + String getDelegationToken(HiveAuthFactory authFactory, String owner, + String renewer) throws HiveSQLException; + + void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException; + + void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException; + + void closeExpiredOperations(); + + long getNoOperationTime(); +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java new file mode 100644 index 000000000000..b72c18b2b213 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java @@ -0,0 +1,90 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hive.service.cli.SessionHandle; +import org.apache.hive.service.cli.operation.OperationManager; +import org.apache.hive.service.cli.thrift.TProtocolVersion; + +import java.io.File; + +/** + * Methods that don't need to be executed under a doAs + * context are here. Rest of them in HiveSession interface + */ +public interface HiveSessionBase { + + TProtocolVersion getProtocolVersion(); + + /** + * Set the session manager for the session + * @param sessionManager + */ + void setSessionManager(SessionManager sessionManager); + + /** + * Get the session manager for the session + */ + SessionManager getSessionManager(); + + /** + * Set operation manager for the session + * @param operationManager + */ + void setOperationManager(OperationManager operationManager); + + /** + * Check whether operation logging is enabled and session dir is created successfully + */ + boolean isOperationLogEnabled(); + + /** + * Get the session dir, which is the parent dir of operation logs + * @return a file representing the parent directory of operation logs + */ + File getOperationLogSessionDir(); + + /** + * Set the session dir, which is the parent dir of operation logs + * @param operationLogRootDir the parent dir of the session dir + */ + void setOperationLogSessionDir(File operationLogRootDir); + + SessionHandle getSessionHandle(); + + String getUsername(); + + String getPassword(); + + HiveConf getHiveConf(); + + SessionState getSessionState(); + + String getUserName(); + + void setUserName(String userName); + + String getIpAddress(); + + void setIpAddress(String ipAddress); + + long getLastAccessTime(); +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java new file mode 100644 index 000000000000..c56a107d4246 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +import org.apache.hadoop.hive.conf.HiveConf; +/** + * HiveSessionHookContext. + * Interface passed to the HiveServer2 session hook execution. This enables + * the hook implementation to access session config, user and session handle + */ +public interface HiveSessionHookContext { + + /** + * Retrieve session conf + * @return + */ + HiveConf getSessionConf(); + + /** + * The get the username starting the session + * @return + */ + String getSessionUser(); + + /** + * Retrieve handle for the session + * @return + */ + String getSessionHandle(); +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java new file mode 100644 index 000000000000..1ee4ac8a1d39 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +import org.apache.hadoop.hive.conf.HiveConf; + +/** + * + * HiveSessionHookContextImpl. + * Session hook context implementation which is created by session manager + * and passed to hook invocation. + */ +public class HiveSessionHookContextImpl implements HiveSessionHookContext { + + private final HiveSession hiveSession; + + HiveSessionHookContextImpl(HiveSession hiveSession) { + this.hiveSession = hiveSession; + } + + @Override + public HiveConf getSessionConf() { + return hiveSession.getHiveConf(); + } + + + @Override + public String getSessionUser() { + return hiveSession.getUserName(); + } + + @Override + public String getSessionHandle() { + return hiveSession.getSessionHandle().toString(); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java new file mode 100644 index 000000000000..47bfaa86021d --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java @@ -0,0 +1,734 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.common.cli.HiveFileProcessor; +import org.apache.hadoop.hive.common.cli.IHiveFileProcessor; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.exec.FetchFormatter; +import org.apache.hadoop.hive.ql.exec.ListSinkOperator; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.history.HiveHistory; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.processors.SetProcessor; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hive.common.util.HiveVersionInfo; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.FetchType; +import org.apache.hive.service.cli.GetInfoType; +import org.apache.hive.service.cli.GetInfoValue; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationHandle; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.SessionHandle; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.operation.ExecuteStatementOperation; +import org.apache.hive.service.cli.operation.GetCatalogsOperation; +import org.apache.hive.service.cli.operation.GetColumnsOperation; +import org.apache.hive.service.cli.operation.GetFunctionsOperation; +import org.apache.hive.service.cli.operation.GetSchemasOperation; +import org.apache.hive.service.cli.operation.GetTableTypesOperation; +import org.apache.hive.service.cli.operation.GetTypeInfoOperation; +import org.apache.hive.service.cli.operation.MetadataOperation; +import org.apache.hive.service.cli.operation.Operation; +import org.apache.hive.service.cli.operation.OperationManager; +import org.apache.hive.service.cli.thrift.TProtocolVersion; +import org.apache.hive.service.server.ThreadWithGarbageCleanup; + +/** + * HiveSession + * + */ +public class HiveSessionImpl implements HiveSession { + private final SessionHandle sessionHandle; + private String username; + private final String password; + private HiveConf hiveConf; + private SessionState sessionState; + private String ipAddress; + private static final String FETCH_WORK_SERDE_CLASS = + "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"; + private static final Log LOG = LogFactory.getLog(HiveSessionImpl.class); + private SessionManager sessionManager; + private OperationManager operationManager; + private final Set opHandleSet = new HashSet(); + private boolean isOperationLogEnabled; + private File sessionLogDir; + private volatile long lastAccessTime; + private volatile long lastIdleTime; + + public HiveSessionImpl(TProtocolVersion protocol, String username, String password, + HiveConf serverhiveConf, String ipAddress) { + this.username = username; + this.password = password; + this.sessionHandle = new SessionHandle(protocol); + this.hiveConf = new HiveConf(serverhiveConf); + this.ipAddress = ipAddress; + + try { + // In non-impersonation mode, map scheduler queue to current user + // if fair scheduler is configured. + if (! hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && + hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_MAP_FAIR_SCHEDULER_QUEUE)) { + ShimLoader.getHadoopShims().refreshDefaultQueue(hiveConf, username); + } + } catch (IOException e) { + LOG.warn("Error setting scheduler queue: " + e, e); + } + // Set an explicit session name to control the download directory name + hiveConf.set(ConfVars.HIVESESSIONID.varname, + sessionHandle.getHandleIdentifier().toString()); + // Use thrift transportable formatter + hiveConf.set(ListSinkOperator.OUTPUT_FORMATTER, + FetchFormatter.ThriftFormatter.class.getName()); + hiveConf.setInt(ListSinkOperator.OUTPUT_PROTOCOL, protocol.getValue()); + } + + @Override + /** + * Opens a new HiveServer2 session for the client connection. + * Creates a new SessionState object that will be associated with this HiveServer2 session. + * When the server executes multiple queries in the same session, + * this SessionState object is reused across multiple queries. + * Note that if doAs is true, this call goes through a proxy object, + * which wraps the method logic in a UserGroupInformation#doAs. + * That's why it is important to create SessionState here rather than in the constructor. + */ + public void open(Map sessionConfMap) throws HiveSQLException { + sessionState = new SessionState(hiveConf, username); + sessionState.setUserIpAddress(ipAddress); + sessionState.setIsHiveServerQuery(true); + SessionState.start(sessionState); + try { + sessionState.reloadAuxJars(); + } catch (IOException e) { + String msg = "Failed to load reloadable jar file path: " + e; + LOG.error(msg, e); + throw new HiveSQLException(msg, e); + } + // Process global init file: .hiverc + processGlobalInitFile(); + if (sessionConfMap != null) { + configureSession(sessionConfMap); + } + lastAccessTime = System.currentTimeMillis(); + lastIdleTime = lastAccessTime; + } + + /** + * It is used for processing hiverc file from HiveServer2 side. + */ + private class GlobalHivercFileProcessor extends HiveFileProcessor { + @Override + protected BufferedReader loadFile(String fileName) throws IOException { + FileInputStream initStream = null; + BufferedReader bufferedReader = null; + initStream = new FileInputStream(fileName); + bufferedReader = new BufferedReader(new InputStreamReader(initStream)); + return bufferedReader; + } + + @Override + protected int processCmd(String cmd) { + int rc = 0; + String cmd_trimed = cmd.trim(); + try { + executeStatementInternal(cmd_trimed, null, false); + } catch (HiveSQLException e) { + rc = -1; + LOG.warn("Failed to execute HQL command in global .hiverc file.", e); + } + return rc; + } + } + + private void processGlobalInitFile() { + IHiveFileProcessor processor = new GlobalHivercFileProcessor(); + + try { + String hiverc = hiveConf.getVar(ConfVars.HIVE_SERVER2_GLOBAL_INIT_FILE_LOCATION); + if (hiverc != null) { + File hivercFile = new File(hiverc); + if (hivercFile.isDirectory()) { + hivercFile = new File(hivercFile, SessionManager.HIVERCFILE); + } + if (hivercFile.isFile()) { + LOG.info("Running global init file: " + hivercFile); + int rc = processor.processFile(hivercFile.getAbsolutePath()); + if (rc != 0) { + LOG.error("Failed on initializing global .hiverc file"); + } + } else { + LOG.debug("Global init file " + hivercFile + " does not exist"); + } + } + } catch (IOException e) { + LOG.warn("Failed on initializing global .hiverc file", e); + } + } + + private void configureSession(Map sessionConfMap) throws HiveSQLException { + SessionState.setCurrentSessionState(sessionState); + for (Map.Entry entry : sessionConfMap.entrySet()) { + String key = entry.getKey(); + if (key.startsWith("set:")) { + try { + SetProcessor.setVariable(key.substring(4), entry.getValue()); + } catch (Exception e) { + throw new HiveSQLException(e); + } + } else if (key.startsWith("use:")) { + SessionState.get().setCurrentDatabase(entry.getValue()); + } else { + hiveConf.verifyAndSet(key, entry.getValue()); + } + } + } + + @Override + public void setOperationLogSessionDir(File operationLogRootDir) { + sessionLogDir = new File(operationLogRootDir, sessionHandle.getHandleIdentifier().toString()); + isOperationLogEnabled = true; + if (!sessionLogDir.exists()) { + if (!sessionLogDir.mkdir()) { + LOG.warn("Unable to create operation log session directory: " + + sessionLogDir.getAbsolutePath()); + isOperationLogEnabled = false; + } + } + if (isOperationLogEnabled) { + LOG.info("Operation log session directory is created: " + sessionLogDir.getAbsolutePath()); + } + } + + @Override + public boolean isOperationLogEnabled() { + return isOperationLogEnabled; + } + + @Override + public File getOperationLogSessionDir() { + return sessionLogDir; + } + + @Override + public TProtocolVersion getProtocolVersion() { + return sessionHandle.getProtocolVersion(); + } + + @Override + public SessionManager getSessionManager() { + return sessionManager; + } + + @Override + public void setSessionManager(SessionManager sessionManager) { + this.sessionManager = sessionManager; + } + + private OperationManager getOperationManager() { + return operationManager; + } + + @Override + public void setOperationManager(OperationManager operationManager) { + this.operationManager = operationManager; + } + + protected synchronized void acquire(boolean userAccess) { + // Need to make sure that the this HiveServer2's session's SessionState is + // stored in the thread local for the handler thread. + SessionState.setCurrentSessionState(sessionState); + if (userAccess) { + lastAccessTime = System.currentTimeMillis(); + } + } + + /** + * 1. We'll remove the ThreadLocal SessionState as this thread might now serve + * other requests. + * 2. We'll cache the ThreadLocal RawStore object for this background thread for an orderly cleanup + * when this thread is garbage collected later. + * @see org.apache.hive.service.server.ThreadWithGarbageCleanup#finalize() + */ + protected synchronized void release(boolean userAccess) { + SessionState.detachSession(); + if (ThreadWithGarbageCleanup.currentThread() instanceof ThreadWithGarbageCleanup) { + ThreadWithGarbageCleanup currentThread = + (ThreadWithGarbageCleanup) ThreadWithGarbageCleanup.currentThread(); + currentThread.cacheThreadLocalRawStore(); + } + if (userAccess) { + lastAccessTime = System.currentTimeMillis(); + } + if (opHandleSet.isEmpty()) { + lastIdleTime = System.currentTimeMillis(); + } else { + lastIdleTime = 0; + } + } + + @Override + public SessionHandle getSessionHandle() { + return sessionHandle; + } + + @Override + public String getUsername() { + return username; + } + + @Override + public String getPassword() { + return password; + } + + @Override + public HiveConf getHiveConf() { + hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHOUTPUTSERDE, FETCH_WORK_SERDE_CLASS); + return hiveConf; + } + + @Override + public IMetaStoreClient getMetaStoreClient() throws HiveSQLException { + try { + return Hive.get(getHiveConf()).getMSC(); + } catch (HiveException e) { + throw new HiveSQLException("Failed to get metastore connection", e); + } catch (MetaException e) { + throw new HiveSQLException("Failed to get metastore connection", e); + } + } + + @Override + public GetInfoValue getInfo(GetInfoType getInfoType) + throws HiveSQLException { + acquire(true); + try { + switch (getInfoType) { + case CLI_SERVER_NAME: + return new GetInfoValue("Hive"); + case CLI_DBMS_NAME: + return new GetInfoValue("Apache Hive"); + case CLI_DBMS_VER: + return new GetInfoValue(HiveVersionInfo.getVersion()); + case CLI_MAX_COLUMN_NAME_LEN: + return new GetInfoValue(128); + case CLI_MAX_SCHEMA_NAME_LEN: + return new GetInfoValue(128); + case CLI_MAX_TABLE_NAME_LEN: + return new GetInfoValue(128); + case CLI_TXN_CAPABLE: + default: + throw new HiveSQLException("Unrecognized GetInfoType value: " + getInfoType.toString()); + } + } finally { + release(true); + } + } + + @Override + public OperationHandle executeStatement(String statement, Map confOverlay) + throws HiveSQLException { + return executeStatementInternal(statement, confOverlay, false); + } + + @Override + public OperationHandle executeStatementAsync(String statement, Map confOverlay) + throws HiveSQLException { + return executeStatementInternal(statement, confOverlay, true); + } + + private OperationHandle executeStatementInternal(String statement, Map confOverlay, + boolean runAsync) + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + ExecuteStatementOperation operation = operationManager + .newExecuteStatementOperation(getSession(), statement, confOverlay, runAsync); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + // Refering to SQLOperation.java,there is no chance that a HiveSQLException throws and the asyn + // background operation submits to thread pool successfully at the same time. So, Cleanup + // opHandle directly when got HiveSQLException + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getTypeInfo() + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetTypeInfoOperation operation = operationManager.newGetTypeInfoOperation(getSession()); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getCatalogs() + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetCatalogsOperation operation = operationManager.newGetCatalogsOperation(getSession()); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getSchemas(String catalogName, String schemaName) + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetSchemasOperation operation = + operationManager.newGetSchemasOperation(getSession(), catalogName, schemaName); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getTables(String catalogName, String schemaName, String tableName, + List tableTypes) + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + MetadataOperation operation = + operationManager.newGetTablesOperation(getSession(), catalogName, schemaName, tableName, tableTypes); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getTableTypes() + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetTableTypesOperation operation = operationManager.newGetTableTypesOperation(getSession()); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getColumns(String catalogName, String schemaName, + String tableName, String columnName) throws HiveSQLException { + acquire(true); + String addedJars = Utilities.getResourceFiles(hiveConf, SessionState.ResourceType.JAR); + if (StringUtils.isNotBlank(addedJars)) { + IMetaStoreClient metastoreClient = getSession().getMetaStoreClient(); + metastoreClient.setHiveAddedJars(addedJars); + } + OperationManager operationManager = getOperationManager(); + GetColumnsOperation operation = operationManager.newGetColumnsOperation(getSession(), + catalogName, schemaName, tableName, columnName); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getFunctions(String catalogName, String schemaName, String functionName) + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetFunctionsOperation operation = operationManager + .newGetFunctionsOperation(getSession(), catalogName, schemaName, functionName); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public void close() throws HiveSQLException { + try { + acquire(true); + // Iterate through the opHandles and close their operations + for (OperationHandle opHandle : opHandleSet) { + operationManager.closeOperation(opHandle); + } + opHandleSet.clear(); + // Cleanup session log directory. + cleanupSessionLogDir(); + HiveHistory hiveHist = sessionState.getHiveHistory(); + if (null != hiveHist) { + hiveHist.closeStream(); + } + try { + sessionState.close(); + } finally { + sessionState = null; + } + } catch (IOException ioe) { + throw new HiveSQLException("Failure to close", ioe); + } finally { + if (sessionState != null) { + try { + sessionState.close(); + } catch (Throwable t) { + LOG.warn("Error closing session", t); + } + sessionState = null; + } + release(true); + } + } + + private void cleanupSessionLogDir() { + if (isOperationLogEnabled) { + try { + FileUtils.forceDelete(sessionLogDir); + } catch (Exception e) { + LOG.error("Failed to cleanup session log dir: " + sessionHandle, e); + } + } + } + + @Override + public SessionState getSessionState() { + return sessionState; + } + + @Override + public String getUserName() { + return username; + } + + @Override + public void setUserName(String userName) { + this.username = userName; + } + + @Override + public long getLastAccessTime() { + return lastAccessTime; + } + + @Override + public void closeExpiredOperations() { + OperationHandle[] handles = opHandleSet.toArray(new OperationHandle[opHandleSet.size()]); + if (handles.length > 0) { + List operations = operationManager.removeExpiredOperations(handles); + if (!operations.isEmpty()) { + closeTimedOutOperations(operations); + } + } + } + + @Override + public long getNoOperationTime() { + return lastIdleTime > 0 ? System.currentTimeMillis() - lastIdleTime : 0; + } + + private void closeTimedOutOperations(List operations) { + acquire(false); + try { + for (Operation operation : operations) { + opHandleSet.remove(operation.getHandle()); + try { + operation.close(); + } catch (Exception e) { + LOG.warn("Exception is thrown closing timed-out operation " + operation.getHandle(), e); + } + } + } finally { + release(false); + } + } + + @Override + public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { + acquire(true); + try { + sessionManager.getOperationManager().cancelOperation(opHandle); + } finally { + release(true); + } + } + + @Override + public void closeOperation(OperationHandle opHandle) throws HiveSQLException { + acquire(true); + try { + operationManager.closeOperation(opHandle); + opHandleSet.remove(opHandle); + } finally { + release(true); + } + } + + @Override + public TableSchema getResultSetMetadata(OperationHandle opHandle) throws HiveSQLException { + acquire(true); + try { + return sessionManager.getOperationManager().getOperationResultSetSchema(opHandle); + } finally { + release(true); + } + } + + @Override + public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, + long maxRows, FetchType fetchType) throws HiveSQLException { + acquire(true); + try { + if (fetchType == FetchType.QUERY_OUTPUT) { + return operationManager.getOperationNextRowSet(opHandle, orientation, maxRows); + } + return operationManager.getOperationLogRowSet(opHandle, orientation, maxRows); + } finally { + release(true); + } + } + + protected HiveSession getSession() { + return this; + } + + @Override + public String getIpAddress() { + return ipAddress; + } + + @Override + public void setIpAddress(String ipAddress) { + this.ipAddress = ipAddress; + } + + @Override + public String getDelegationToken(HiveAuthFactory authFactory, String owner, String renewer) + throws HiveSQLException { + HiveAuthFactory.verifyProxyAccess(getUsername(), owner, getIpAddress(), getHiveConf()); + return authFactory.getDelegationToken(owner, renewer); + } + + @Override + public void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException { + HiveAuthFactory.verifyProxyAccess(getUsername(), getUserFromToken(authFactory, tokenStr), + getIpAddress(), getHiveConf()); + authFactory.cancelDelegationToken(tokenStr); + } + + @Override + public void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException { + HiveAuthFactory.verifyProxyAccess(getUsername(), getUserFromToken(authFactory, tokenStr), + getIpAddress(), getHiveConf()); + authFactory.renewDelegationToken(tokenStr); + } + + // extract the real user from the given token string + private String getUserFromToken(HiveAuthFactory authFactory, String tokenStr) throws HiveSQLException { + return authFactory.getUserFromToken(tokenStr); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java new file mode 100644 index 000000000000..762dbb2faade --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java @@ -0,0 +1,182 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +import java.io.IOException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.shims.Utils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.thrift.TProtocolVersion; + +/** + * + * HiveSessionImplwithUGI. + * HiveSession with connecting user's UGI and delegation token if required + */ +public class HiveSessionImplwithUGI extends HiveSessionImpl { + public static final String HS2TOKEN = "HiveServer2ImpersonationToken"; + + private UserGroupInformation sessionUgi = null; + private String delegationTokenStr = null; + private Hive sessionHive = null; + private HiveSession proxySession = null; + static final Log LOG = LogFactory.getLog(HiveSessionImplwithUGI.class); + + public HiveSessionImplwithUGI(TProtocolVersion protocol, String username, String password, + HiveConf hiveConf, String ipAddress, String delegationToken) throws HiveSQLException { + super(protocol, username, password, hiveConf, ipAddress); + setSessionUGI(username); + setDelegationToken(delegationToken); + + // create a new metastore connection for this particular user session + Hive.set(null); + try { + sessionHive = Hive.get(getHiveConf()); + } catch (HiveException e) { + throw new HiveSQLException("Failed to setup metastore connection", e); + } + } + + // setup appropriate UGI for the session + public void setSessionUGI(String owner) throws HiveSQLException { + if (owner == null) { + throw new HiveSQLException("No username provided for impersonation"); + } + if (UserGroupInformation.isSecurityEnabled()) { + try { + sessionUgi = UserGroupInformation.createProxyUser( + owner, UserGroupInformation.getLoginUser()); + } catch (IOException e) { + throw new HiveSQLException("Couldn't setup proxy user", e); + } + } else { + sessionUgi = UserGroupInformation.createRemoteUser(owner); + } + } + + public UserGroupInformation getSessionUgi() { + return this.sessionUgi; + } + + public String getDelegationToken() { + return this.delegationTokenStr; + } + + @Override + protected synchronized void acquire(boolean userAccess) { + super.acquire(userAccess); + // if we have a metastore connection with impersonation, then set it first + if (sessionHive != null) { + Hive.set(sessionHive); + } + } + + /** + * Close the file systems for the session and remove it from the FileSystem cache. + * Cancel the session's delegation token and close the metastore connection + */ + @Override + public void close() throws HiveSQLException { + try { + acquire(true); + cancelDelegationToken(); + } finally { + try { + super.close(); + } finally { + try { + FileSystem.closeAllForUGI(sessionUgi); + } catch (IOException ioe) { + throw new HiveSQLException("Could not clean up file-system handles for UGI: " + + sessionUgi, ioe); + } + } + } + } + + /** + * Enable delegation token for the session + * save the token string and set the token.signature in hive conf. The metastore client uses + * this token.signature to determine where to use kerberos or delegation token + * @throws HiveException + * @throws IOException + */ + private void setDelegationToken(String delegationTokenStr) throws HiveSQLException { + this.delegationTokenStr = delegationTokenStr; + if (delegationTokenStr != null) { + getHiveConf().set("hive.metastore.token.signature", HS2TOKEN); + try { + Utils.setTokenStr(sessionUgi, delegationTokenStr, HS2TOKEN); + } catch (IOException e) { + throw new HiveSQLException("Couldn't setup delegation token in the ugi", e); + } + } + } + + // If the session has a delegation token obtained from the metastore, then cancel it + private void cancelDelegationToken() throws HiveSQLException { + if (delegationTokenStr != null) { + try { + Hive.get(getHiveConf()).cancelDelegationToken(delegationTokenStr); + } catch (HiveException e) { + throw new HiveSQLException("Couldn't cancel delegation token", e); + } + // close the metastore connection created with this delegation token + Hive.closeCurrent(); + } + } + + @Override + protected HiveSession getSession() { + assert proxySession != null; + + return proxySession; + } + + public void setProxySession(HiveSession proxySession) { + this.proxySession = proxySession; + } + + @Override + public String getDelegationToken(HiveAuthFactory authFactory, String owner, + String renewer) throws HiveSQLException { + return authFactory.getDelegationToken(owner, renewer); + } + + @Override + public void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException { + authFactory.cancelDelegationToken(tokenStr); + } + + @Override + public void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException { + authFactory.renewDelegationToken(tokenStr); + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java new file mode 100644 index 000000000000..8e539512f741 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionProxy.java @@ -0,0 +1,91 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +/** + * Proxy wrapper on HiveSession to execute operations + * by impersonating given user + */ +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; +import java.lang.reflect.UndeclaredThrowableException; +import java.security.PrivilegedActionException; +import java.security.PrivilegedExceptionAction; + +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.service.cli.HiveSQLException; + +public class HiveSessionProxy implements InvocationHandler { + private final HiveSession base; + private final UserGroupInformation ugi; + + public HiveSessionProxy(HiveSession hiveSession, UserGroupInformation ugi) { + this.base = hiveSession; + this.ugi = ugi; + } + + public static HiveSession getProxy(HiveSession hiveSession, UserGroupInformation ugi) + throws IllegalArgumentException, HiveSQLException { + return (HiveSession)Proxy.newProxyInstance(HiveSession.class.getClassLoader(), + new Class[] {HiveSession.class}, + new HiveSessionProxy(hiveSession, ugi)); + } + + @Override + public Object invoke(Object arg0, final Method method, final Object[] args) + throws Throwable { + try { + if (method.getDeclaringClass() == HiveSessionBase.class) { + return invoke(method, args); + } + return ugi.doAs( + new PrivilegedExceptionAction() { + @Override + public Object run() throws HiveSQLException { + return invoke(method, args); + } + }); + } catch (UndeclaredThrowableException e) { + Throwable innerException = e.getCause(); + if (innerException instanceof PrivilegedActionException) { + throw innerException.getCause(); + } else { + throw e.getCause(); + } + } + } + + private Object invoke(final Method method, final Object[] args) throws HiveSQLException { + try { + return method.invoke(base, args); + } catch (InvocationTargetException e) { + if (e.getCause() instanceof HiveSQLException) { + throw (HiveSQLException)e.getCause(); + } + throw new RuntimeException(e.getCause()); + } catch (IllegalArgumentException e) { + throw new RuntimeException(e); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + } +} + diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java new file mode 100644 index 000000000000..859f9c8b449e --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java @@ -0,0 +1,361 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.session; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Date; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hive.service.CompositeService; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.SessionHandle; +import org.apache.hive.service.cli.operation.OperationManager; +import org.apache.hive.service.cli.thrift.TProtocolVersion; +import org.apache.hive.service.server.HiveServer2; +import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; + +/** + * SessionManager. + * + */ +public class SessionManager extends CompositeService { + + private static final Log LOG = LogFactory.getLog(SessionManager.class); + public static final String HIVERCFILE = ".hiverc"; + private HiveConf hiveConf; + private final Map handleToSession = + new ConcurrentHashMap(); + private final OperationManager operationManager = new OperationManager(); + private ThreadPoolExecutor backgroundOperationPool; + private boolean isOperationLogEnabled; + private File operationLogRootDir; + + private long checkInterval; + private long sessionTimeout; + private boolean checkOperation; + + private volatile boolean shutdown; + // The HiveServer2 instance running this service + private final HiveServer2 hiveServer2; + + public SessionManager(HiveServer2 hiveServer2) { + super(SessionManager.class.getSimpleName()); + this.hiveServer2 = hiveServer2; + } + + @Override + public synchronized void init(HiveConf hiveConf) { + this.hiveConf = hiveConf; + //Create operation log root directory, if operation logging is enabled + if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) { + initOperationLogRootDir(); + } + createBackgroundOperationPool(); + addService(operationManager); + super.init(hiveConf); + } + + private void createBackgroundOperationPool() { + int poolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS); + LOG.info("HiveServer2: Background operation thread pool size: " + poolSize); + int poolQueueSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_WAIT_QUEUE_SIZE); + LOG.info("HiveServer2: Background operation thread wait queue size: " + poolQueueSize); + long keepAliveTime = HiveConf.getTimeVar( + hiveConf, ConfVars.HIVE_SERVER2_ASYNC_EXEC_KEEPALIVE_TIME, TimeUnit.SECONDS); + LOG.info( + "HiveServer2: Background operation thread keepalive time: " + keepAliveTime + " seconds"); + + // Create a thread pool with #poolSize threads + // Threads terminate when they are idle for more than the keepAliveTime + // A bounded blocking queue is used to queue incoming operations, if #operations > poolSize + String threadPoolName = "HiveServer2-Background-Pool"; + backgroundOperationPool = new ThreadPoolExecutor(poolSize, poolSize, + keepAliveTime, TimeUnit.SECONDS, new LinkedBlockingQueue(poolQueueSize), + new ThreadFactoryWithGarbageCleanup(threadPoolName)); + backgroundOperationPool.allowCoreThreadTimeOut(true); + + checkInterval = HiveConf.getTimeVar( + hiveConf, ConfVars.HIVE_SERVER2_SESSION_CHECK_INTERVAL, TimeUnit.MILLISECONDS); + sessionTimeout = HiveConf.getTimeVar( + hiveConf, ConfVars.HIVE_SERVER2_IDLE_SESSION_TIMEOUT, TimeUnit.MILLISECONDS); + checkOperation = HiveConf.getBoolVar(hiveConf, + ConfVars.HIVE_SERVER2_IDLE_SESSION_CHECK_OPERATION); + } + + private void initOperationLogRootDir() { + operationLogRootDir = new File( + hiveConf.getVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LOG_LOCATION)); + isOperationLogEnabled = true; + + if (operationLogRootDir.exists() && !operationLogRootDir.isDirectory()) { + LOG.warn("The operation log root directory exists, but it is not a directory: " + + operationLogRootDir.getAbsolutePath()); + isOperationLogEnabled = false; + } + + if (!operationLogRootDir.exists()) { + if (!operationLogRootDir.mkdirs()) { + LOG.warn("Unable to create operation log root directory: " + + operationLogRootDir.getAbsolutePath()); + isOperationLogEnabled = false; + } + } + + if (isOperationLogEnabled) { + LOG.info("Operation log root directory is created: " + operationLogRootDir.getAbsolutePath()); + try { + FileUtils.forceDeleteOnExit(operationLogRootDir); + } catch (IOException e) { + LOG.warn("Failed to schedule cleanup HS2 operation logging root dir: " + + operationLogRootDir.getAbsolutePath(), e); + } + } + } + + @Override + public synchronized void start() { + super.start(); + if (checkInterval > 0) { + startTimeoutChecker(); + } + } + + private void startTimeoutChecker() { + final long interval = Math.max(checkInterval, 3000L); // minimum 3 seconds + Runnable timeoutChecker = new Runnable() { + @Override + public void run() { + for (sleepInterval(interval); !shutdown; sleepInterval(interval)) { + long current = System.currentTimeMillis(); + for (HiveSession session : new ArrayList(handleToSession.values())) { + if (sessionTimeout > 0 && session.getLastAccessTime() + sessionTimeout <= current + && (!checkOperation || session.getNoOperationTime() > sessionTimeout)) { + SessionHandle handle = session.getSessionHandle(); + LOG.warn("Session " + handle + " is Timed-out (last access : " + + new Date(session.getLastAccessTime()) + ") and will be closed"); + try { + closeSession(handle); + } catch (HiveSQLException e) { + LOG.warn("Exception is thrown closing session " + handle, e); + } + } else { + session.closeExpiredOperations(); + } + } + } + } + + private void sleepInterval(long interval) { + try { + Thread.sleep(interval); + } catch (InterruptedException e) { + // ignore + } + } + }; + backgroundOperationPool.execute(timeoutChecker); + } + + @Override + public synchronized void stop() { + super.stop(); + shutdown = true; + if (backgroundOperationPool != null) { + backgroundOperationPool.shutdown(); + long timeout = hiveConf.getTimeVar( + ConfVars.HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT, TimeUnit.SECONDS); + try { + backgroundOperationPool.awaitTermination(timeout, TimeUnit.SECONDS); + } catch (InterruptedException e) { + LOG.warn("HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT = " + timeout + + " seconds has been exceeded. RUNNING background operations will be shut down", e); + } + backgroundOperationPool = null; + } + cleanupLoggingRootDir(); + } + + private void cleanupLoggingRootDir() { + if (isOperationLogEnabled) { + try { + FileUtils.forceDelete(operationLogRootDir); + } catch (Exception e) { + LOG.warn("Failed to cleanup root dir of HS2 logging: " + operationLogRootDir + .getAbsolutePath(), e); + } + } + } + + public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, + Map sessionConf) throws HiveSQLException { + return openSession(protocol, username, password, ipAddress, sessionConf, false, null); + } + + /** + * Opens a new session and creates a session handle. + * The username passed to this method is the effective username. + * If withImpersonation is true (==doAs true) we wrap all the calls in HiveSession + * within a UGI.doAs, where UGI corresponds to the effective user. + * + * Please see {@code org.apache.hive.service.cli.thrift.ThriftCLIService.getUserName()} for + * more details. + * + * @param protocol + * @param username + * @param password + * @param ipAddress + * @param sessionConf + * @param withImpersonation + * @param delegationToken + * @return + * @throws HiveSQLException + */ + public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, + Map sessionConf, boolean withImpersonation, String delegationToken) + throws HiveSQLException { + HiveSession session; + // If doAs is set to true for HiveServer2, we will create a proxy object for the session impl. + // Within the proxy object, we wrap the method call in a UserGroupInformation#doAs + if (withImpersonation) { + HiveSessionImplwithUGI sessionWithUGI = new HiveSessionImplwithUGI(protocol, username, password, + hiveConf, ipAddress, delegationToken); + session = HiveSessionProxy.getProxy(sessionWithUGI, sessionWithUGI.getSessionUgi()); + sessionWithUGI.setProxySession(session); + } else { + session = new HiveSessionImpl(protocol, username, password, hiveConf, ipAddress); + } + session.setSessionManager(this); + session.setOperationManager(operationManager); + try { + session.open(sessionConf); + } catch (Exception e) { + try { + session.close(); + } catch (Throwable t) { + LOG.warn("Error closing session", t); + } + session = null; + throw new HiveSQLException("Failed to open new session: " + e, e); + } + if (isOperationLogEnabled) { + session.setOperationLogSessionDir(operationLogRootDir); + } + handleToSession.put(session.getSessionHandle(), session); + return session.getSessionHandle(); + } + + public void closeSession(SessionHandle sessionHandle) throws HiveSQLException { + HiveSession session = handleToSession.remove(sessionHandle); + if (session == null) { + throw new HiveSQLException("Session does not exist!"); + } + session.close(); + } + + public HiveSession getSession(SessionHandle sessionHandle) throws HiveSQLException { + HiveSession session = handleToSession.get(sessionHandle); + if (session == null) { + throw new HiveSQLException("Invalid SessionHandle: " + sessionHandle); + } + return session; + } + + public OperationManager getOperationManager() { + return operationManager; + } + + private static ThreadLocal threadLocalIpAddress = new ThreadLocal() { + @Override + protected synchronized String initialValue() { + return null; + } + }; + + public static void setIpAddress(String ipAddress) { + threadLocalIpAddress.set(ipAddress); + } + + public static void clearIpAddress() { + threadLocalIpAddress.remove(); + } + + public static String getIpAddress() { + return threadLocalIpAddress.get(); + } + + private static ThreadLocal threadLocalUserName = new ThreadLocal(){ + @Override + protected synchronized String initialValue() { + return null; + } + }; + + public static void setUserName(String userName) { + threadLocalUserName.set(userName); + } + + public static void clearUserName() { + threadLocalUserName.remove(); + } + + public static String getUserName() { + return threadLocalUserName.get(); + } + + private static ThreadLocal threadLocalProxyUserName = new ThreadLocal(){ + @Override + protected synchronized String initialValue() { + return null; + } + }; + + public static void setProxyUserName(String userName) { + LOG.debug("setting proxy user name based on query param to: " + userName); + threadLocalProxyUserName.set(userName); + } + + public static String getProxyUserName() { + return threadLocalProxyUserName.get(); + } + + public static void clearProxyUserName() { + threadLocalProxyUserName.remove(); + } + + public Future submitBackgroundOperation(Runnable r) { + return backgroundOperationPool.submit(r); + } + + public int getOpenSessionCount() { + return handleToSession.size(); + } +} + diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java new file mode 100644 index 000000000000..6c9efba9e59a --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.thrift; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.CLIService; +import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; +import org.apache.thrift.TProcessorFactory; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.server.TThreadPoolServer; +import org.apache.thrift.transport.TServerSocket; +import org.apache.thrift.transport.TTransportFactory; + + +public class ThriftBinaryCLIService extends ThriftCLIService { + + public ThriftBinaryCLIService(CLIService cliService) { + super(cliService, ThriftBinaryCLIService.class.getSimpleName()); + } + + @Override + public void run() { + try { + // Server thread pool + String threadPoolName = "HiveServer2-Handler-Pool"; + ExecutorService executorService = new ThreadPoolExecutor(minWorkerThreads, maxWorkerThreads, + workerKeepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), + new ThreadFactoryWithGarbageCleanup(threadPoolName)); + + // Thrift configs + hiveAuthFactory = new HiveAuthFactory(hiveConf); + TTransportFactory transportFactory = hiveAuthFactory.getAuthTransFactory(); + TProcessorFactory processorFactory = hiveAuthFactory.getAuthProcFactory(this); + TServerSocket serverSocket = null; + List sslVersionBlacklist = new ArrayList(); + for (String sslVersion : hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(",")) { + sslVersionBlacklist.add(sslVersion); + } + if (!hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL)) { + serverSocket = HiveAuthFactory.getServerSocket(hiveHost, portNum); + } else { + String keyStorePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH).trim(); + if (keyStorePath.isEmpty()) { + throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname + + " Not configured for SSL connection"); + } + String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf, + HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname); + serverSocket = HiveAuthFactory.getServerSSLSocket(hiveHost, portNum, keyStorePath, + keyStorePassword, sslVersionBlacklist); + } + + // Server args + int maxMessageSize = hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_MAX_MESSAGE_SIZE); + int requestTimeout = (int) hiveConf.getTimeVar( + HiveConf.ConfVars.HIVE_SERVER2_THRIFT_LOGIN_TIMEOUT, TimeUnit.SECONDS); + int beBackoffSlotLength = (int) hiveConf.getTimeVar( + HiveConf.ConfVars.HIVE_SERVER2_THRIFT_LOGIN_BEBACKOFF_SLOT_LENGTH, TimeUnit.MILLISECONDS); + TThreadPoolServer.Args sargs = new TThreadPoolServer.Args(serverSocket) + .processorFactory(processorFactory).transportFactory(transportFactory) + .protocolFactory(new TBinaryProtocol.Factory()) + .inputProtocolFactory(new TBinaryProtocol.Factory(true, true, maxMessageSize, maxMessageSize)) + .requestTimeout(requestTimeout).requestTimeoutUnit(TimeUnit.SECONDS) + .beBackoffSlotLength(beBackoffSlotLength).beBackoffSlotLengthUnit(TimeUnit.MILLISECONDS) + .executorService(executorService); + + // TCP Server + server = new TThreadPoolServer(sargs); + server.setServerEventHandler(serverEventHandler); + String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + + portNum + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; + LOG.info(msg); + server.serve(); + } catch (Throwable t) { + LOG.fatal( + "Error starting HiveServer2: could not start " + + ThriftBinaryCLIService.class.getSimpleName(), t); + System.exit(-1); + } + } + +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java new file mode 100644 index 000000000000..ad7a9a238f8a --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -0,0 +1,689 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.thrift; + +import javax.security.auth.login.LoginException; +import java.io.IOException; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hive.service.AbstractService; +import org.apache.hive.service.ServiceException; +import org.apache.hive.service.ServiceUtils; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.auth.TSetIpAddressProcessor; +import org.apache.hive.service.cli.*; +import org.apache.hive.service.cli.session.SessionManager; +import org.apache.hive.service.server.HiveServer2; +import org.apache.thrift.TException; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.server.ServerContext; +import org.apache.thrift.server.TServer; +import org.apache.thrift.server.TServerEventHandler; +import org.apache.thrift.transport.TTransport; + +/** + * ThriftCLIService. + * + */ +public abstract class ThriftCLIService extends AbstractService implements TCLIService.Iface, Runnable { + + public static final Log LOG = LogFactory.getLog(ThriftCLIService.class.getName()); + + protected CLIService cliService; + private static final TStatus OK_STATUS = new TStatus(TStatusCode.SUCCESS_STATUS); + protected static HiveAuthFactory hiveAuthFactory; + + protected int portNum; + protected InetAddress serverIPAddress; + protected String hiveHost; + protected TServer server; + protected org.eclipse.jetty.server.Server httpServer; + + private boolean isStarted = false; + protected boolean isEmbedded = false; + + protected HiveConf hiveConf; + + protected int minWorkerThreads; + protected int maxWorkerThreads; + protected long workerKeepAliveTime; + + protected TServerEventHandler serverEventHandler; + protected ThreadLocal currentServerContext; + + static class ThriftCLIServerContext implements ServerContext { + private SessionHandle sessionHandle = null; + + public void setSessionHandle(SessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public SessionHandle getSessionHandle() { + return sessionHandle; + } + } + + public ThriftCLIService(CLIService service, String serviceName) { + super(serviceName); + this.cliService = service; + currentServerContext = new ThreadLocal(); + serverEventHandler = new TServerEventHandler() { + @Override + public ServerContext createContext( + TProtocol input, TProtocol output) { + return new ThriftCLIServerContext(); + } + + @Override + public void deleteContext(ServerContext serverContext, + TProtocol input, TProtocol output) { + ThriftCLIServerContext context = (ThriftCLIServerContext)serverContext; + SessionHandle sessionHandle = context.getSessionHandle(); + if (sessionHandle != null) { + LOG.info("Session disconnected without closing properly, close it now"); + try { + cliService.closeSession(sessionHandle); + } catch (HiveSQLException e) { + LOG.warn("Failed to close session: " + e, e); + } + } + } + + @Override + public void preServe() { + } + + @Override + public void processContext(ServerContext serverContext, + TTransport input, TTransport output) { + currentServerContext.set(serverContext); + } + }; + } + + @Override + public synchronized void init(HiveConf hiveConf) { + this.hiveConf = hiveConf; + // Initialize common server configs needed in both binary & http modes + String portString; + hiveHost = System.getenv("HIVE_SERVER2_THRIFT_BIND_HOST"); + if (hiveHost == null) { + hiveHost = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST); + } + try { + if (hiveHost != null && !hiveHost.isEmpty()) { + serverIPAddress = InetAddress.getByName(hiveHost); + } else { + serverIPAddress = InetAddress.getLocalHost(); + } + } catch (UnknownHostException e) { + throw new ServiceException(e); + } + // HTTP mode + if (HiveServer2.isHTTPTransportMode(hiveConf)) { + workerKeepAliveTime = + hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_WORKER_KEEPALIVE_TIME, + TimeUnit.SECONDS); + portString = System.getenv("HIVE_SERVER2_THRIFT_HTTP_PORT"); + if (portString != null) { + portNum = Integer.valueOf(portString); + } else { + portNum = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT); + } + } + // Binary mode + else { + workerKeepAliveTime = + hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_WORKER_KEEPALIVE_TIME, TimeUnit.SECONDS); + portString = System.getenv("HIVE_SERVER2_THRIFT_PORT"); + if (portString != null) { + portNum = Integer.valueOf(portString); + } else { + portNum = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT); + } + } + minWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MIN_WORKER_THREADS); + maxWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MAX_WORKER_THREADS); + super.init(hiveConf); + } + + @Override + public synchronized void start() { + super.start(); + if (!isStarted && !isEmbedded) { + new Thread(this).start(); + isStarted = true; + } + } + + @Override + public synchronized void stop() { + if (isStarted && !isEmbedded) { + if(server != null) { + server.stop(); + LOG.info("Thrift server has stopped"); + } + if((httpServer != null) && httpServer.isStarted()) { + try { + httpServer.stop(); + LOG.info("Http server has stopped"); + } catch (Exception e) { + LOG.error("Error stopping Http server: ", e); + } + } + isStarted = false; + } + super.stop(); + } + + public int getPortNumber() { + return portNum; + } + + public InetAddress getServerIPAddress() { + return serverIPAddress; + } + + @Override + public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) + throws TException { + TGetDelegationTokenResp resp = new TGetDelegationTokenResp(); + resp.setStatus(notSupportTokenErrorStatus()); + return resp; + } + + @Override + public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) + throws TException { + TCancelDelegationTokenResp resp = new TCancelDelegationTokenResp(); + resp.setStatus(notSupportTokenErrorStatus()); + return resp; + } + + @Override + public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) + throws TException { + TRenewDelegationTokenResp resp = new TRenewDelegationTokenResp(); + resp.setStatus(notSupportTokenErrorStatus()); + return resp; + } + + private TStatus notSupportTokenErrorStatus() { + TStatus errorStatus = new TStatus(TStatusCode.ERROR_STATUS); + errorStatus.setErrorMessage("Delegation token is not supported"); + return errorStatus; + } + + @Override + public TOpenSessionResp OpenSession(TOpenSessionReq req) throws TException { + LOG.info("Client protocol version: " + req.getClient_protocol()); + TOpenSessionResp resp = new TOpenSessionResp(); + try { + SessionHandle sessionHandle = getSessionHandle(req, resp); + resp.setSessionHandle(sessionHandle.toTSessionHandle()); + // TODO: set real configuration map + resp.setConfiguration(new HashMap()); + resp.setStatus(OK_STATUS); + ThriftCLIServerContext context = + (ThriftCLIServerContext)currentServerContext.get(); + if (context != null) { + context.setSessionHandle(sessionHandle); + } + } catch (Exception e) { + LOG.warn("Error opening session: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + private String getIpAddress() { + String clientIpAddress; + // Http transport mode. + // We set the thread local ip address, in ThriftHttpServlet. + if (cliService.getHiveConf().getVar( + ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { + clientIpAddress = SessionManager.getIpAddress(); + } + else { + // Kerberos + if (isKerberosAuthMode()) { + clientIpAddress = hiveAuthFactory.getIpAddress(); + } + // Except kerberos, NOSASL + else { + clientIpAddress = TSetIpAddressProcessor.getUserIpAddress(); + } + } + LOG.debug("Client's IP Address: " + clientIpAddress); + return clientIpAddress; + } + + /** + * Returns the effective username. + * 1. If hive.server2.allow.user.substitution = false: the username of the connecting user + * 2. If hive.server2.allow.user.substitution = true: the username of the end user, + * that the connecting user is trying to proxy for. + * This includes a check whether the connecting user is allowed to proxy for the end user. + * @param req + * @return + * @throws HiveSQLException + */ + private String getUserName(TOpenSessionReq req) throws HiveSQLException { + String userName = null; + // Kerberos + if (isKerberosAuthMode()) { + userName = hiveAuthFactory.getRemoteUser(); + } + // Except kerberos, NOSASL + if (userName == null) { + userName = TSetIpAddressProcessor.getUserName(); + } + // Http transport mode. + // We set the thread local username, in ThriftHttpServlet. + if (cliService.getHiveConf().getVar( + ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { + userName = SessionManager.getUserName(); + } + if (userName == null) { + userName = req.getUsername(); + } + + userName = getShortName(userName); + String effectiveClientUser = getProxyUser(userName, req.getConfiguration(), getIpAddress()); + LOG.debug("Client's username: " + effectiveClientUser); + return effectiveClientUser; + } + + private String getShortName(String userName) { + String ret = null; + if (userName != null) { + int indexOfDomainMatch = ServiceUtils.indexOfDomainMatch(userName); + ret = (indexOfDomainMatch <= 0) ? userName : + userName.substring(0, indexOfDomainMatch); + } + + return ret; + } + + /** + * Create a session handle + * @param req + * @param res + * @return + * @throws HiveSQLException + * @throws LoginException + * @throws IOException + */ + SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) + throws HiveSQLException, LoginException, IOException { + String userName = getUserName(req); + String ipAddress = getIpAddress(); + TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION, + req.getClient_protocol()); + SessionHandle sessionHandle; + if (cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && + (userName != null)) { + String delegationTokenStr = getDelegationToken(userName); + sessionHandle = cliService.openSessionWithImpersonation(protocol, userName, + req.getPassword(), ipAddress, req.getConfiguration(), delegationTokenStr); + } else { + sessionHandle = cliService.openSession(protocol, userName, req.getPassword(), + ipAddress, req.getConfiguration()); + } + res.setServerProtocolVersion(protocol); + return sessionHandle; + } + + + private String getDelegationToken(String userName) + throws HiveSQLException, LoginException, IOException { + if (userName == null || !cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION) + .equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString())) { + return null; + } + try { + return cliService.getDelegationTokenFromMetaStore(userName); + } catch (UnsupportedOperationException e) { + // The delegation token is not applicable in the given deployment mode + } + return null; + } + + private TProtocolVersion getMinVersion(TProtocolVersion... versions) { + TProtocolVersion[] values = TProtocolVersion.values(); + int current = values[values.length - 1].getValue(); + for (TProtocolVersion version : versions) { + if (current > version.getValue()) { + current = version.getValue(); + } + } + for (TProtocolVersion version : values) { + if (version.getValue() == current) { + return version; + } + } + throw new IllegalArgumentException("never"); + } + + @Override + public TCloseSessionResp CloseSession(TCloseSessionReq req) throws TException { + TCloseSessionResp resp = new TCloseSessionResp(); + try { + SessionHandle sessionHandle = new SessionHandle(req.getSessionHandle()); + cliService.closeSession(sessionHandle); + resp.setStatus(OK_STATUS); + ThriftCLIServerContext context = + (ThriftCLIServerContext)currentServerContext.get(); + if (context != null) { + context.setSessionHandle(null); + } + } catch (Exception e) { + LOG.warn("Error closing session: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetInfoResp GetInfo(TGetInfoReq req) throws TException { + TGetInfoResp resp = new TGetInfoResp(); + try { + GetInfoValue getInfoValue = + cliService.getInfo(new SessionHandle(req.getSessionHandle()), + GetInfoType.getGetInfoType(req.getInfoType())); + resp.setInfoValue(getInfoValue.toTGetInfoValue()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting info: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws TException { + TExecuteStatementResp resp = new TExecuteStatementResp(); + try { + SessionHandle sessionHandle = new SessionHandle(req.getSessionHandle()); + String statement = req.getStatement(); + Map confOverlay = req.getConfOverlay(); + Boolean runAsync = req.isRunAsync(); + OperationHandle operationHandle = runAsync ? + cliService.executeStatementAsync(sessionHandle, statement, confOverlay) + : cliService.executeStatement(sessionHandle, statement, confOverlay); + resp.setOperationHandle(operationHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error executing statement: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws TException { + TGetTypeInfoResp resp = new TGetTypeInfoResp(); + try { + OperationHandle operationHandle = cliService.getTypeInfo(new SessionHandle(req.getSessionHandle())); + resp.setOperationHandle(operationHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting type info: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws TException { + TGetCatalogsResp resp = new TGetCatalogsResp(); + try { + OperationHandle opHandle = cliService.getCatalogs(new SessionHandle(req.getSessionHandle())); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting catalogs: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws TException { + TGetSchemasResp resp = new TGetSchemasResp(); + try { + OperationHandle opHandle = cliService.getSchemas( + new SessionHandle(req.getSessionHandle()), req.getCatalogName(), req.getSchemaName()); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting schemas: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetTablesResp GetTables(TGetTablesReq req) throws TException { + TGetTablesResp resp = new TGetTablesResp(); + try { + OperationHandle opHandle = cliService + .getTables(new SessionHandle(req.getSessionHandle()), req.getCatalogName(), + req.getSchemaName(), req.getTableName(), req.getTableTypes()); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting tables: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws TException { + TGetTableTypesResp resp = new TGetTableTypesResp(); + try { + OperationHandle opHandle = cliService.getTableTypes(new SessionHandle(req.getSessionHandle())); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting table types: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetColumnsResp GetColumns(TGetColumnsReq req) throws TException { + TGetColumnsResp resp = new TGetColumnsResp(); + try { + OperationHandle opHandle = cliService.getColumns( + new SessionHandle(req.getSessionHandle()), + req.getCatalogName(), + req.getSchemaName(), + req.getTableName(), + req.getColumnName()); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting columns: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws TException { + TGetFunctionsResp resp = new TGetFunctionsResp(); + try { + OperationHandle opHandle = cliService.getFunctions( + new SessionHandle(req.getSessionHandle()), req.getCatalogName(), + req.getSchemaName(), req.getFunctionName()); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting functions: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws TException { + TGetOperationStatusResp resp = new TGetOperationStatusResp(); + try { + OperationStatus operationStatus = cliService.getOperationStatus( + new OperationHandle(req.getOperationHandle())); + resp.setOperationState(operationStatus.getState().toTOperationState()); + HiveSQLException opException = operationStatus.getOperationException(); + if (opException != null) { + resp.setSqlState(opException.getSQLState()); + resp.setErrorCode(opException.getErrorCode()); + resp.setErrorMessage(opException.getMessage()); + } + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting operation status: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws TException { + TCancelOperationResp resp = new TCancelOperationResp(); + try { + cliService.cancelOperation(new OperationHandle(req.getOperationHandle())); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error cancelling operation: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws TException { + TCloseOperationResp resp = new TCloseOperationResp(); + try { + cliService.closeOperation(new OperationHandle(req.getOperationHandle())); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error closing operation: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) + throws TException { + TGetResultSetMetadataResp resp = new TGetResultSetMetadataResp(); + try { + TableSchema schema = cliService.getResultSetMetadata(new OperationHandle(req.getOperationHandle())); + resp.setSchema(schema.toTTableSchema()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting result set metadata: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TFetchResultsResp FetchResults(TFetchResultsReq req) throws TException { + TFetchResultsResp resp = new TFetchResultsResp(); + try { + RowSet rowSet = cliService.fetchResults( + new OperationHandle(req.getOperationHandle()), + FetchOrientation.getFetchOrientation(req.getOrientation()), + req.getMaxRows(), + FetchType.getFetchType(req.getFetchType())); + resp.setResults(rowSet.toTRowSet()); + resp.setHasMoreRows(false); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error fetching results: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public abstract void run(); + + /** + * If the proxy user name is provided then check privileges to substitute the user. + * @param realUser + * @param sessionConf + * @param ipAddress + * @return + * @throws HiveSQLException + */ + private String getProxyUser(String realUser, Map sessionConf, + String ipAddress) throws HiveSQLException { + String proxyUser = null; + // Http transport mode. + // We set the thread local proxy username, in ThriftHttpServlet. + if (cliService.getHiveConf().getVar( + ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { + proxyUser = SessionManager.getProxyUserName(); + LOG.debug("Proxy user from query string: " + proxyUser); + } + + if (proxyUser == null && sessionConf != null && sessionConf.containsKey(HiveAuthFactory.HS2_PROXY_USER)) { + String proxyUserFromThriftBody = sessionConf.get(HiveAuthFactory.HS2_PROXY_USER); + LOG.debug("Proxy user from thrift body: " + proxyUserFromThriftBody); + proxyUser = proxyUserFromThriftBody; + } + + if (proxyUser == null) { + return realUser; + } + + // check whether substitution is allowed + if (!hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ALLOW_USER_SUBSTITUTION)) { + throw new HiveSQLException("Proxy user substitution is not allowed"); + } + + // If there's no authentication, then directly substitute the user + if (HiveAuthFactory.AuthTypes.NONE.toString() + .equalsIgnoreCase(hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION))) { + return proxyUser; + } + + // Verify proxy user privilege of the realUser for the proxyUser + HiveAuthFactory.verifyProxyAccess(realUser, proxyUser, ipAddress, hiveConf); + LOG.debug("Verified proxy user: " + proxyUser); + return proxyUser; + } + + private boolean isKerberosAuthMode() { + return cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION) + .equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString()); + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java new file mode 100644 index 000000000000..1af45398b895 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java @@ -0,0 +1,440 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.thrift; + +import java.util.List; +import java.util.Map; + +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.*; +import org.apache.thrift.TException; + +/** + * ThriftCLIServiceClient. + * + */ +public class ThriftCLIServiceClient extends CLIServiceClient { + private final TCLIService.Iface cliService; + + public ThriftCLIServiceClient(TCLIService.Iface cliService) { + this.cliService = cliService; + } + + public void checkStatus(TStatus status) throws HiveSQLException { + if (TStatusCode.ERROR_STATUS.equals(status.getStatusCode())) { + throw new HiveSQLException(status); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) + */ + @Override + public SessionHandle openSession(String username, String password, + Map configuration) + throws HiveSQLException { + try { + TOpenSessionReq req = new TOpenSessionReq(); + req.setUsername(username); + req.setPassword(password); + req.setConfiguration(configuration); + TOpenSessionResp resp = cliService.OpenSession(req); + checkStatus(resp.getStatus()); + return new SessionHandle(resp.getSessionHandle(), resp.getServerProtocolVersion()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public SessionHandle openSessionWithImpersonation(String username, String password, + Map configuration, String delegationToken) throws HiveSQLException { + throw new HiveSQLException("open with impersonation operation is not supported in the client"); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public void closeSession(SessionHandle sessionHandle) throws HiveSQLException { + try { + TCloseSessionReq req = new TCloseSessionReq(sessionHandle.toTSessionHandle()); + TCloseSessionResp resp = cliService.CloseSession(req); + checkStatus(resp.getStatus()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List) + */ + @Override + public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType infoType) + throws HiveSQLException { + try { + // FIXME extract the right info type + TGetInfoReq req = new TGetInfoReq(sessionHandle.toTSessionHandle(), infoType.toTGetInfoType()); + TGetInfoResp resp = cliService.GetInfo(req); + checkStatus(resp.getStatus()); + return new GetInfoValue(resp.getInfoValue()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#executeStatement(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, + Map confOverlay) + throws HiveSQLException { + return executeStatementInternal(sessionHandle, statement, confOverlay, false); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#executeStatementAsync(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, + Map confOverlay) + throws HiveSQLException { + return executeStatementInternal(sessionHandle, statement, confOverlay, true); + } + + private OperationHandle executeStatementInternal(SessionHandle sessionHandle, String statement, + Map confOverlay, boolean isAsync) + throws HiveSQLException { + try { + TExecuteStatementReq req = + new TExecuteStatementReq(sessionHandle.toTSessionHandle(), statement); + req.setConfOverlay(confOverlay); + req.setRunAsync(isAsync); + TExecuteStatementResp resp = cliService.ExecuteStatement(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTypeInfo(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTypeInfo(SessionHandle sessionHandle) throws HiveSQLException { + try { + TGetTypeInfoReq req = new TGetTypeInfoReq(sessionHandle.toTSessionHandle()); + TGetTypeInfoResp resp = cliService.GetTypeInfo(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getCatalogs(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getCatalogs(SessionHandle sessionHandle) throws HiveSQLException { + try { + TGetCatalogsReq req = new TGetCatalogsReq(sessionHandle.toTSessionHandle()); + TGetCatalogsResp resp = cliService.GetCatalogs(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String) + */ + @Override + public OperationHandle getSchemas(SessionHandle sessionHandle, String catalogName, + String schemaName) + throws HiveSQLException { + try { + TGetSchemasReq req = new TGetSchemasReq(sessionHandle.toTSessionHandle()); + req.setCatalogName(catalogName); + req.setSchemaName(schemaName); + TGetSchemasResp resp = cliService.GetSchemas(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List) + */ + @Override + public OperationHandle getTables(SessionHandle sessionHandle, String catalogName, + String schemaName, String tableName, List tableTypes) + throws HiveSQLException { + try { + TGetTablesReq req = new TGetTablesReq(sessionHandle.toTSessionHandle()); + req.setTableName(tableName); + req.setTableTypes(tableTypes); + req.setSchemaName(schemaName); + TGetTablesResp resp = cliService.GetTables(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTableTypes(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTableTypes(SessionHandle sessionHandle) throws HiveSQLException { + try { + TGetTableTypesReq req = new TGetTableTypesReq(sessionHandle.toTSessionHandle()); + TGetTableTypesResp resp = cliService.GetTableTypes(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getColumns(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getColumns(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, String columnName) + throws HiveSQLException { + try { + TGetColumnsReq req = new TGetColumnsReq(); + req.setSessionHandle(sessionHandle.toTSessionHandle()); + req.setCatalogName(catalogName); + req.setSchemaName(schemaName); + req.setTableName(tableName); + req.setColumnName(columnName); + TGetColumnsResp resp = cliService.GetColumns(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getFunctions(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getFunctions(SessionHandle sessionHandle, + String catalogName, String schemaName, String functionName) throws HiveSQLException { + try { + TGetFunctionsReq req = new TGetFunctionsReq(sessionHandle.toTSessionHandle(), functionName); + req.setCatalogName(catalogName); + req.setSchemaName(schemaName); + TGetFunctionsResp resp = cliService.GetFunctions(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getOperationStatus(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public OperationStatus getOperationStatus(OperationHandle opHandle) throws HiveSQLException { + try { + TGetOperationStatusReq req = new TGetOperationStatusReq(opHandle.toTOperationHandle()); + TGetOperationStatusResp resp = cliService.GetOperationStatus(req); + // Checks the status of the RPC call, throws an exception in case of error + checkStatus(resp.getStatus()); + OperationState opState = OperationState.getOperationState(resp.getOperationState()); + HiveSQLException opException = null; + if (opState == OperationState.ERROR) { + opException = new HiveSQLException(resp.getErrorMessage(), resp.getSqlState(), resp.getErrorCode()); + } + return new OperationStatus(opState, opException); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#cancelOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { + try { + TCancelOperationReq req = new TCancelOperationReq(opHandle.toTOperationHandle()); + TCancelOperationResp resp = cliService.CancelOperation(req); + checkStatus(resp.getStatus()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void closeOperation(OperationHandle opHandle) + throws HiveSQLException { + try { + TCloseOperationReq req = new TCloseOperationReq(opHandle.toTOperationHandle()); + TCloseOperationResp resp = cliService.CloseOperation(req); + checkStatus(resp.getStatus()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public TableSchema getResultSetMetadata(OperationHandle opHandle) + throws HiveSQLException { + try { + TGetResultSetMetadataReq req = new TGetResultSetMetadataReq(opHandle.toTOperationHandle()); + TGetResultSetMetadataResp resp = cliService.GetResultSetMetadata(req); + checkStatus(resp.getStatus()); + return new TableSchema(resp.getSchema()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + @Override + public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, long maxRows, + FetchType fetchType) throws HiveSQLException { + try { + TFetchResultsReq req = new TFetchResultsReq(); + req.setOperationHandle(opHandle.toTOperationHandle()); + req.setOrientation(orientation.toTFetchOrientation()); + req.setMaxRows(maxRows); + req.setFetchType(fetchType.toTFetchType()); + TFetchResultsResp resp = cliService.FetchResults(req); + checkStatus(resp.getStatus()); + return RowSetFactory.create(resp.getResults(), opHandle.getProtocolVersion()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#fetchResults(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public RowSet fetchResults(OperationHandle opHandle) throws HiveSQLException { + // TODO: set the correct default fetch size + return fetchResults(opHandle, FetchOrientation.FETCH_NEXT, 10000, FetchType.QUERY_OUTPUT); + } + + @Override + public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String owner, String renewer) throws HiveSQLException { + TGetDelegationTokenReq req = new TGetDelegationTokenReq( + sessionHandle.toTSessionHandle(), owner, renewer); + try { + TGetDelegationTokenResp tokenResp = cliService.GetDelegationToken(req); + checkStatus(tokenResp.getStatus()); + return tokenResp.getDelegationToken(); + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + @Override + public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + TCancelDelegationTokenReq cancelReq = new TCancelDelegationTokenReq( + sessionHandle.toTSessionHandle(), tokenStr); + try { + TCancelDelegationTokenResp cancelResp = + cliService.CancelDelegationToken(cancelReq); + checkStatus(cancelResp.getStatus()); + return; + } catch (TException e) { + throw new HiveSQLException(e); + } + } + + @Override + public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + TRenewDelegationTokenReq cancelReq = new TRenewDelegationTokenReq( + sessionHandle.toTSessionHandle(), tokenStr); + try { + TRenewDelegationTokenResp renewResp = + cliService.RenewDelegationToken(cancelReq); + checkStatus(renewResp.getStatus()); + return; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java new file mode 100644 index 000000000000..341a7fdbb59b --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -0,0 +1,183 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.thrift; + +import java.util.Arrays; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.Shell; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.CLIService; +import org.apache.hive.service.cli.thrift.TCLIService.Iface; +import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; +import org.apache.thrift.TProcessor; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TProtocolFactory; +import org.apache.thrift.server.TServlet; +import org.eclipse.jetty.server.AbstractConnectionFactory; +import org.eclipse.jetty.server.ConnectionFactory; +import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.util.ssl.SslContextFactory; +import org.eclipse.jetty.util.thread.ExecutorThreadPool; +import org.eclipse.jetty.util.thread.ScheduledExecutorScheduler; + + +public class ThriftHttpCLIService extends ThriftCLIService { + + public ThriftHttpCLIService(CLIService cliService) { + super(cliService, ThriftHttpCLIService.class.getSimpleName()); + } + + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ + @Override + public void run() { + try { + // Server thread pool + // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests + String threadPoolName = "HiveServer2-HttpHandler-Pool"; + ExecutorService executorService = new ThreadPoolExecutor(minWorkerThreads, maxWorkerThreads, + workerKeepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), + new ThreadFactoryWithGarbageCleanup(threadPoolName)); + ExecutorThreadPool threadPool = new ExecutorThreadPool(executorService); + + // HTTP Server + httpServer = new org.eclipse.jetty.server.Server(threadPool); + + // Connector configs + + ConnectionFactory[] connectionFactories; + boolean useSsl = hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL); + String schemeName = useSsl ? "https" : "http"; + // Change connector if SSL is used + if (useSsl) { + String keyStorePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH).trim(); + String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf, + HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname); + if (keyStorePath.isEmpty()) { + throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname + + " Not configured for SSL connection"); + } + SslContextFactory sslContextFactory = new SslContextFactory(); + String[] excludedProtocols = hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(","); + LOG.info("HTTP Server SSL: adding excluded protocols: " + Arrays.toString(excludedProtocols)); + sslContextFactory.addExcludeProtocols(excludedProtocols); + LOG.info("HTTP Server SSL: SslContextFactory.getExcludeProtocols = " + + Arrays.toString(sslContextFactory.getExcludeProtocols())); + sslContextFactory.setKeyStorePath(keyStorePath); + sslContextFactory.setKeyStorePassword(keyStorePassword); + connectionFactories = AbstractConnectionFactory.getFactories( + sslContextFactory, new HttpConnectionFactory()); + } else { + connectionFactories = new ConnectionFactory[] { new HttpConnectionFactory() }; + } + ServerConnector connector = new ServerConnector( + httpServer, + null, + // Call this full constructor to set this, which forces daemon threads: + new ScheduledExecutorScheduler("HiveServer2-HttpHandler-JettyScheduler", true), + null, + -1, + -1, + connectionFactories); + + connector.setPort(portNum); + // Linux:yes, Windows:no + connector.setReuseAddress(!Shell.WINDOWS); + int maxIdleTime = (int) hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME, + TimeUnit.MILLISECONDS); + connector.setIdleTimeout(maxIdleTime); + + httpServer.addConnector(connector); + + // Thrift configs + hiveAuthFactory = new HiveAuthFactory(hiveConf); + TProcessor processor = new TCLIService.Processor(this); + TProtocolFactory protocolFactory = new TBinaryProtocol.Factory(); + // Set during the init phase of HiveServer2 if auth mode is kerberos + // UGI for the hive/_HOST (kerberos) principal + UserGroupInformation serviceUGI = cliService.getServiceUGI(); + // UGI for the http/_HOST (SPNego) principal + UserGroupInformation httpUGI = cliService.getHttpUGI(); + String authType = hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION); + TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory, authType, + serviceUGI, httpUGI); + + // Context handler + final ServletContextHandler context = new ServletContextHandler( + ServletContextHandler.SESSIONS); + context.setContextPath("/"); + String httpPath = getHttpPath(hiveConf + .getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH)); + httpServer.setHandler(context); + context.addServlet(new ServletHolder(thriftHttpServlet), httpPath); + + // TODO: check defaults: maxTimeout, keepalive, maxBodySize, bodyRecieveDuration, etc. + // Finally, start the server + httpServer.start(); + String msg = "Started " + ThriftHttpCLIService.class.getSimpleName() + " in " + schemeName + + " mode on port " + portNum + " path=" + httpPath + " with " + minWorkerThreads + "..." + + maxWorkerThreads + " worker threads"; + LOG.info(msg); + httpServer.join(); + } catch (Throwable t) { + LOG.fatal( + "Error starting HiveServer2: could not start " + + ThriftHttpCLIService.class.getSimpleName(), t); + System.exit(-1); + } + } + + /** + * The config parameter can be like "path", "/path", "/path/", "path/*", "/path1/path2/*" and so on. + * httpPath should end up as "/*", "/path/*" or "/path1/../pathN/*" + * @param httpPath + * @return + */ + private String getHttpPath(String httpPath) { + if(httpPath == null || httpPath.equals("")) { + httpPath = "/*"; + } + else { + if(!httpPath.startsWith("/")) { + httpPath = "/" + httpPath; + } + if(httpPath.endsWith("/")) { + httpPath = httpPath + "*"; + } + if(!httpPath.endsWith("/*")) { + httpPath = httpPath + "/*"; + } + } + return httpPath; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java new file mode 100644 index 000000000000..e15d2d0566d2 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java @@ -0,0 +1,545 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.thrift; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.security.PrivilegedExceptionAction; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import javax.servlet.ServletException; +import javax.servlet.http.Cookie; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.core.NewCookie; + +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.codec.binary.StringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.shims.HadoopShims.KerberosNameShim; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.service.auth.AuthenticationProviderFactory; +import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.auth.HttpAuthUtils; +import org.apache.hive.service.auth.HttpAuthenticationException; +import org.apache.hive.service.auth.PasswdAuthenticationProvider; +import org.apache.hive.service.cli.session.SessionManager; +import org.apache.hive.service.CookieSigner; +import org.apache.thrift.TProcessor; +import org.apache.thrift.protocol.TProtocolFactory; +import org.apache.thrift.server.TServlet; +import org.ietf.jgss.GSSContext; +import org.ietf.jgss.GSSCredential; +import org.ietf.jgss.GSSException; +import org.ietf.jgss.GSSManager; +import org.ietf.jgss.GSSName; +import org.ietf.jgss.Oid; + +/** + * + * ThriftHttpServlet + * + */ +public class ThriftHttpServlet extends TServlet { + + private static final long serialVersionUID = 1L; + public static final Log LOG = LogFactory.getLog(ThriftHttpServlet.class.getName()); + private final String authType; + private final UserGroupInformation serviceUGI; + private final UserGroupInformation httpUGI; + private HiveConf hiveConf = new HiveConf(); + + // Class members for cookie based authentication. + private CookieSigner signer; + public static final String AUTH_COOKIE = "hive.server2.auth"; + private static final Random RAN = new Random(); + private boolean isCookieAuthEnabled; + private String cookieDomain; + private String cookiePath; + private int cookieMaxAge; + private boolean isCookieSecure; + private boolean isHttpOnlyCookie; + + public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory, + String authType, UserGroupInformation serviceUGI, UserGroupInformation httpUGI) { + super(processor, protocolFactory); + this.authType = authType; + this.serviceUGI = serviceUGI; + this.httpUGI = httpUGI; + this.isCookieAuthEnabled = hiveConf.getBoolVar( + ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_AUTH_ENABLED); + // Initialize the cookie based authentication related variables. + if (isCookieAuthEnabled) { + // Generate the signer with secret. + String secret = Long.toString(RAN.nextLong()); + LOG.debug("Using the random number as the secret for cookie generation " + secret); + this.signer = new CookieSigner(secret.getBytes()); + this.cookieMaxAge = (int) hiveConf.getTimeVar( + ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_MAX_AGE, TimeUnit.SECONDS); + this.cookieDomain = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_DOMAIN); + this.cookiePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_PATH); + this.isCookieSecure = hiveConf.getBoolVar( + ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_IS_SECURE); + this.isHttpOnlyCookie = hiveConf.getBoolVar( + ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_IS_HTTPONLY); + } + } + + @Override + protected void doPost(HttpServletRequest request, HttpServletResponse response) + throws ServletException, IOException { + String clientUserName = null; + String clientIpAddress; + boolean requireNewCookie = false; + + try { + // If the cookie based authentication is already enabled, parse the + // request and validate the request cookies. + if (isCookieAuthEnabled) { + clientUserName = validateCookie(request); + requireNewCookie = (clientUserName == null); + if (requireNewCookie) { + LOG.info("Could not validate cookie sent, will try to generate a new cookie"); + } + } + // If the cookie based authentication is not enabled or the request does + // not have a valid cookie, use the kerberos or password based authentication + // depending on the server setup. + if (clientUserName == null) { + // For a kerberos setup + if (isKerberosAuthMode(authType)) { + clientUserName = doKerberosAuth(request); + } + // For password based authentication + else { + clientUserName = doPasswdAuth(request, authType); + } + } + LOG.debug("Client username: " + clientUserName); + + // Set the thread local username to be used for doAs if true + SessionManager.setUserName(clientUserName); + + // find proxy user if any from query param + String doAsQueryParam = getDoAsQueryParam(request.getQueryString()); + if (doAsQueryParam != null) { + SessionManager.setProxyUserName(doAsQueryParam); + } + + clientIpAddress = request.getRemoteAddr(); + LOG.debug("Client IP Address: " + clientIpAddress); + // Set the thread local ip address + SessionManager.setIpAddress(clientIpAddress); + // Generate new cookie and add it to the response + if (requireNewCookie && + !authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) { + String cookieToken = HttpAuthUtils.createCookieToken(clientUserName); + Cookie hs2Cookie = createCookie(signer.signCookie(cookieToken)); + + if (isHttpOnlyCookie) { + response.setHeader("SET-COOKIE", getHttpOnlyCookieHeader(hs2Cookie)); + } else { + response.addCookie(hs2Cookie); + } + LOG.info("Cookie added for clientUserName " + clientUserName); + } + super.doPost(request, response); + } + catch (HttpAuthenticationException e) { + LOG.error("Error: ", e); + // Send a 401 to the client + response.setStatus(HttpServletResponse.SC_UNAUTHORIZED); + if(isKerberosAuthMode(authType)) { + response.addHeader(HttpAuthUtils.WWW_AUTHENTICATE, HttpAuthUtils.NEGOTIATE); + } + response.getWriter().println("Authentication Error: " + e.getMessage()); + } + finally { + // Clear the thread locals + SessionManager.clearUserName(); + SessionManager.clearIpAddress(); + SessionManager.clearProxyUserName(); + } + } + + /** + * Retrieves the client name from cookieString. If the cookie does not + * correspond to a valid client, the function returns null. + * @param cookies HTTP Request cookies. + * @return Client Username if cookieString has a HS2 Generated cookie that is currently valid. + * Else, returns null. + */ + private String getClientNameFromCookie(Cookie[] cookies) { + // Current Cookie Name, Current Cookie Value + String currName, currValue; + + // Following is the main loop which iterates through all the cookies send by the client. + // The HS2 generated cookies are of the format hive.server2.auth= + // A cookie which is identified as a hiveserver2 generated cookie is validated + // by calling signer.verifyAndExtract(). If the validation passes, send the + // username for which the cookie is validated to the caller. If no client side + // cookie passes the validation, return null to the caller. + for (Cookie currCookie : cookies) { + // Get the cookie name + currName = currCookie.getName(); + if (!currName.equals(AUTH_COOKIE)) { + // Not a HS2 generated cookie, continue. + continue; + } + // If we reached here, we have match for HS2 generated cookie + currValue = currCookie.getValue(); + // Validate the value. + currValue = signer.verifyAndExtract(currValue); + // Retrieve the user name, do the final validation step. + if (currValue != null) { + String userName = HttpAuthUtils.getUserNameFromCookieToken(currValue); + + if (userName == null) { + LOG.warn("Invalid cookie token " + currValue); + continue; + } + //We have found a valid cookie in the client request. + if (LOG.isDebugEnabled()) { + LOG.debug("Validated the cookie for user " + userName); + } + return userName; + } + } + // No valid HS2 generated cookies found, return null + return null; + } + + /** + * Convert cookie array to human readable cookie string + * @param cookies Cookie Array + * @return String containing all the cookies separated by a newline character. + * Each cookie is of the format [key]=[value] + */ + private String toCookieStr(Cookie[] cookies) { + String cookieStr = ""; + + for (Cookie c : cookies) { + cookieStr += c.getName() + "=" + c.getValue() + " ;\n"; + } + return cookieStr; + } + + /** + * Validate the request cookie. This function iterates over the request cookie headers + * and finds a cookie that represents a valid client/server session. If it finds one, it + * returns the client name associated with the session. Else, it returns null. + * @param request The HTTP Servlet Request send by the client + * @return Client Username if the request has valid HS2 cookie, else returns null + * @throws UnsupportedEncodingException + */ + private String validateCookie(HttpServletRequest request) throws UnsupportedEncodingException { + // Find all the valid cookies associated with the request. + Cookie[] cookies = request.getCookies(); + + if (cookies == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("No valid cookies associated with the request " + request); + } + return null; + } + if (LOG.isDebugEnabled()) { + LOG.debug("Received cookies: " + toCookieStr(cookies)); + } + return getClientNameFromCookie(cookies); + } + + /** + * Generate a server side cookie given the cookie value as the input. + * @param str Input string token. + * @return The generated cookie. + * @throws UnsupportedEncodingException + */ + private Cookie createCookie(String str) throws UnsupportedEncodingException { + if (LOG.isDebugEnabled()) { + LOG.debug("Cookie name = " + AUTH_COOKIE + " value = " + str); + } + Cookie cookie = new Cookie(AUTH_COOKIE, str); + + cookie.setMaxAge(cookieMaxAge); + if (cookieDomain != null) { + cookie.setDomain(cookieDomain); + } + if (cookiePath != null) { + cookie.setPath(cookiePath); + } + cookie.setSecure(isCookieSecure); + return cookie; + } + + /** + * Generate httponly cookie from HS2 cookie + * @param cookie HS2 generated cookie + * @return The httponly cookie + */ + private static String getHttpOnlyCookieHeader(Cookie cookie) { + NewCookie newCookie = new NewCookie(cookie.getName(), cookie.getValue(), + cookie.getPath(), cookie.getDomain(), cookie.getVersion(), + cookie.getComment(), cookie.getMaxAge(), cookie.getSecure()); + return newCookie + "; HttpOnly"; + } + + /** + * Do the LDAP/PAM authentication + * @param request + * @param authType + * @throws HttpAuthenticationException + */ + private String doPasswdAuth(HttpServletRequest request, String authType) + throws HttpAuthenticationException { + String userName = getUsername(request, authType); + // No-op when authType is NOSASL + if (!authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) { + try { + AuthMethods authMethod = AuthMethods.getValidAuthMethod(authType); + PasswdAuthenticationProvider provider = + AuthenticationProviderFactory.getAuthenticationProvider(authMethod); + provider.Authenticate(userName, getPassword(request, authType)); + + } catch (Exception e) { + throw new HttpAuthenticationException(e); + } + } + return userName; + } + + /** + * Do the GSS-API kerberos authentication. + * We already have a logged in subject in the form of serviceUGI, + * which GSS-API will extract information from. + * In case of a SPNego request we use the httpUGI, + * for the authenticating service tickets. + * @param request + * @return + * @throws HttpAuthenticationException + */ + private String doKerberosAuth(HttpServletRequest request) + throws HttpAuthenticationException { + // Try authenticating with the http/_HOST principal + if (httpUGI != null) { + try { + return httpUGI.doAs(new HttpKerberosServerAction(request, httpUGI)); + } catch (Exception e) { + LOG.info("Failed to authenticate with http/_HOST kerberos principal, " + + "trying with hive/_HOST kerberos principal"); + } + } + // Now try with hive/_HOST principal + try { + return serviceUGI.doAs(new HttpKerberosServerAction(request, serviceUGI)); + } catch (Exception e) { + LOG.error("Failed to authenticate with hive/_HOST kerberos principal"); + throw new HttpAuthenticationException(e); + } + + } + + class HttpKerberosServerAction implements PrivilegedExceptionAction { + HttpServletRequest request; + UserGroupInformation serviceUGI; + + HttpKerberosServerAction(HttpServletRequest request, + UserGroupInformation serviceUGI) { + this.request = request; + this.serviceUGI = serviceUGI; + } + + @Override + public String run() throws HttpAuthenticationException { + // Get own Kerberos credentials for accepting connection + GSSManager manager = GSSManager.getInstance(); + GSSContext gssContext = null; + String serverPrincipal = getPrincipalWithoutRealm( + serviceUGI.getUserName()); + try { + // This Oid for Kerberos GSS-API mechanism. + Oid kerberosMechOid = new Oid("1.2.840.113554.1.2.2"); + // Oid for SPNego GSS-API mechanism. + Oid spnegoMechOid = new Oid("1.3.6.1.5.5.2"); + // Oid for kerberos principal name + Oid krb5PrincipalOid = new Oid("1.2.840.113554.1.2.2.1"); + + // GSS name for server + GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid); + + // GSS credentials for server + GSSCredential serverCreds = manager.createCredential(serverName, + GSSCredential.DEFAULT_LIFETIME, + new Oid[]{kerberosMechOid, spnegoMechOid}, + GSSCredential.ACCEPT_ONLY); + + // Create a GSS context + gssContext = manager.createContext(serverCreds); + // Get service ticket from the authorization header + String serviceTicketBase64 = getAuthHeader(request, authType); + byte[] inToken = Base64.decodeBase64(serviceTicketBase64.getBytes()); + gssContext.acceptSecContext(inToken, 0, inToken.length); + // Authenticate or deny based on its context completion + if (!gssContext.isEstablished()) { + throw new HttpAuthenticationException("Kerberos authentication failed: " + + "unable to establish context with the service ticket " + + "provided by the client."); + } + else { + return getPrincipalWithoutRealmAndHost(gssContext.getSrcName().toString()); + } + } + catch (GSSException e) { + throw new HttpAuthenticationException("Kerberos authentication failed: ", e); + } + finally { + if (gssContext != null) { + try { + gssContext.dispose(); + } catch (GSSException e) { + // No-op + } + } + } + } + + private String getPrincipalWithoutRealm(String fullPrincipal) + throws HttpAuthenticationException { + KerberosNameShim fullKerberosName; + try { + fullKerberosName = ShimLoader.getHadoopShims().getKerberosNameShim(fullPrincipal); + } catch (IOException e) { + throw new HttpAuthenticationException(e); + } + String serviceName = fullKerberosName.getServiceName(); + String hostName = fullKerberosName.getHostName(); + String principalWithoutRealm = serviceName; + if (hostName != null) { + principalWithoutRealm = serviceName + "/" + hostName; + } + return principalWithoutRealm; + } + + private String getPrincipalWithoutRealmAndHost(String fullPrincipal) + throws HttpAuthenticationException { + KerberosNameShim fullKerberosName; + try { + fullKerberosName = ShimLoader.getHadoopShims().getKerberosNameShim(fullPrincipal); + return fullKerberosName.getShortName(); + } catch (IOException e) { + throw new HttpAuthenticationException(e); + } + } + } + + private String getUsername(HttpServletRequest request, String authType) + throws HttpAuthenticationException { + String[] creds = getAuthHeaderTokens(request, authType); + // Username must be present + if (creds[0] == null || creds[0].isEmpty()) { + throw new HttpAuthenticationException("Authorization header received " + + "from the client does not contain username."); + } + return creds[0]; + } + + private String getPassword(HttpServletRequest request, String authType) + throws HttpAuthenticationException { + String[] creds = getAuthHeaderTokens(request, authType); + // Password must be present + if (creds[1] == null || creds[1].isEmpty()) { + throw new HttpAuthenticationException("Authorization header received " + + "from the client does not contain username."); + } + return creds[1]; + } + + private String[] getAuthHeaderTokens(HttpServletRequest request, + String authType) throws HttpAuthenticationException { + String authHeaderBase64 = getAuthHeader(request, authType); + String authHeaderString = StringUtils.newStringUtf8( + Base64.decodeBase64(authHeaderBase64.getBytes())); + String[] creds = authHeaderString.split(":"); + return creds; + } + + /** + * Returns the base64 encoded auth header payload + * @param request + * @param authType + * @return + * @throws HttpAuthenticationException + */ + private String getAuthHeader(HttpServletRequest request, String authType) + throws HttpAuthenticationException { + String authHeader = request.getHeader(HttpAuthUtils.AUTHORIZATION); + // Each http request must have an Authorization header + if (authHeader == null || authHeader.isEmpty()) { + throw new HttpAuthenticationException("Authorization header received " + + "from the client is empty."); + } + + String authHeaderBase64String; + int beginIndex; + if (isKerberosAuthMode(authType)) { + beginIndex = (HttpAuthUtils.NEGOTIATE + " ").length(); + } + else { + beginIndex = (HttpAuthUtils.BASIC + " ").length(); + } + authHeaderBase64String = authHeader.substring(beginIndex); + // Authorization header must have a payload + if (authHeaderBase64String == null || authHeaderBase64String.isEmpty()) { + throw new HttpAuthenticationException("Authorization header received " + + "from the client does not contain any data."); + } + return authHeaderBase64String; + } + + private boolean isKerberosAuthMode(String authType) { + return authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString()); + } + + private static String getDoAsQueryParam(String queryString) { + if (LOG.isDebugEnabled()) { + LOG.debug("URL query string:" + queryString); + } + if (queryString == null) { + return null; + } + Map params = javax.servlet.http.HttpUtils.parseQueryString( queryString ); + Set keySet = params.keySet(); + for (String key: keySet) { + if (key.equalsIgnoreCase("doAs")) { + return params.get(key)[0]; + } + } + return null; + } + +} + + diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java new file mode 100644 index 000000000000..9bf96cff572e --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java @@ -0,0 +1,277 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.server; + +import java.util.Properties; + +import org.apache.commons.cli.GnuParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.OptionBuilder; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.common.LogUtils; +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hive.common.util.HiveStringUtils; +import org.apache.hive.service.CompositeService; +import org.apache.hive.service.cli.CLIService; +import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService; +import org.apache.hive.service.cli.thrift.ThriftCLIService; +import org.apache.hive.service.cli.thrift.ThriftHttpCLIService; + +/** + * HiveServer2. + * + */ +public class HiveServer2 extends CompositeService { + private static final Log LOG = LogFactory.getLog(HiveServer2.class); + + private CLIService cliService; + private ThriftCLIService thriftCLIService; + + public HiveServer2() { + super(HiveServer2.class.getSimpleName()); + HiveConf.setLoadHiveServer2Config(true); + } + + @Override + public synchronized void init(HiveConf hiveConf) { + cliService = new CLIService(this); + addService(cliService); + if (isHTTPTransportMode(hiveConf)) { + thriftCLIService = new ThriftHttpCLIService(cliService); + } else { + thriftCLIService = new ThriftBinaryCLIService(cliService); + } + addService(thriftCLIService); + super.init(hiveConf); + + // Add a shutdown hook for catching SIGTERM & SIGINT + final HiveServer2 hiveServer2 = this; + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + hiveServer2.stop(); + } + }); + } + + public static boolean isHTTPTransportMode(HiveConf hiveConf) { + String transportMode = System.getenv("HIVE_SERVER2_TRANSPORT_MODE"); + if (transportMode == null) { + transportMode = hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE); + } + if (transportMode != null && (transportMode.equalsIgnoreCase("http"))) { + return true; + } + return false; + } + + @Override + public synchronized void start() { + super.start(); + } + + @Override + public synchronized void stop() { + LOG.info("Shutting down HiveServer2"); + HiveConf hiveConf = this.getHiveConf(); + super.stop(); + } + + private static void startHiveServer2() throws Throwable { + long attempts = 0, maxAttempts = 1; + while (true) { + LOG.info("Starting HiveServer2"); + HiveConf hiveConf = new HiveConf(); + maxAttempts = hiveConf.getLongVar(HiveConf.ConfVars.HIVE_SERVER2_MAX_START_ATTEMPTS); + HiveServer2 server = null; + try { + server = new HiveServer2(); + server.init(hiveConf); + server.start(); + ShimLoader.getHadoopShims().startPauseMonitor(hiveConf); + break; + } catch (Throwable throwable) { + if (server != null) { + try { + server.stop(); + } catch (Throwable t) { + LOG.info("Exception caught when calling stop of HiveServer2 before retrying start", t); + } finally { + server = null; + } + } + if (++attempts >= maxAttempts) { + throw new Error("Max start attempts " + maxAttempts + " exhausted", throwable); + } else { + LOG.warn("Error starting HiveServer2 on attempt " + attempts + + ", will retry in 60 seconds", throwable); + try { + Thread.sleep(60L * 1000L); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + } + } + + public static void main(String[] args) { + HiveConf.setLoadHiveServer2Config(true); + try { + ServerOptionsProcessor oproc = new ServerOptionsProcessor("hiveserver2"); + ServerOptionsProcessorResponse oprocResponse = oproc.parse(args); + + // NOTE: It is critical to do this here so that log4j is reinitialized + // before any of the other core hive classes are loaded + String initLog4jMessage = LogUtils.initHiveLog4j(); + LOG.debug(initLog4jMessage); + HiveStringUtils.startupShutdownMessage(HiveServer2.class, args, LOG); + + // Log debug message from "oproc" after log4j initialize properly + LOG.debug(oproc.getDebugMessage().toString()); + + // Call the executor which will execute the appropriate command based on the parsed options + oprocResponse.getServerOptionsExecutor().execute(); + } catch (LogInitializationException e) { + LOG.error("Error initializing log: " + e.getMessage(), e); + System.exit(-1); + } + } + + /** + * ServerOptionsProcessor. + * Process arguments given to HiveServer2 (-hiveconf property=value) + * Set properties in System properties + * Create an appropriate response object, + * which has executor to execute the appropriate command based on the parsed options. + */ + public static class ServerOptionsProcessor { + private final Options options = new Options(); + private org.apache.commons.cli.CommandLine commandLine; + private final String serverName; + private final StringBuilder debugMessage = new StringBuilder(); + + @SuppressWarnings("static-access") + public ServerOptionsProcessor(String serverName) { + this.serverName = serverName; + // -hiveconf x=y + options.addOption(OptionBuilder + .withValueSeparator() + .hasArgs(2) + .withArgName("property=value") + .withLongOpt("hiveconf") + .withDescription("Use value for given property") + .create()); + options.addOption(new Option("H", "help", false, "Print help information")); + } + + public ServerOptionsProcessorResponse parse(String[] argv) { + try { + commandLine = new GnuParser().parse(options, argv); + // Process --hiveconf + // Get hiveconf param values and set the System property values + Properties confProps = commandLine.getOptionProperties("hiveconf"); + for (String propKey : confProps.stringPropertyNames()) { + // save logging message for log4j output latter after log4j initialize properly + debugMessage.append("Setting " + propKey + "=" + confProps.getProperty(propKey) + ";\n"); + System.setProperty(propKey, confProps.getProperty(propKey)); + } + + // Process --help + if (commandLine.hasOption('H')) { + return new ServerOptionsProcessorResponse(new HelpOptionExecutor(serverName, options)); + } + } catch (ParseException e) { + // Error out & exit - we were not able to parse the args successfully + System.err.println("Error starting HiveServer2 with given arguments: "); + System.err.println(e.getMessage()); + System.exit(-1); + } + // Default executor, when no option is specified + return new ServerOptionsProcessorResponse(new StartOptionExecutor()); + } + + StringBuilder getDebugMessage() { + return debugMessage; + } + } + + /** + * The response sent back from {@link ServerOptionsProcessor#parse(String[])} + */ + static class ServerOptionsProcessorResponse { + private final ServerOptionsExecutor serverOptionsExecutor; + + ServerOptionsProcessorResponse(ServerOptionsExecutor serverOptionsExecutor) { + this.serverOptionsExecutor = serverOptionsExecutor; + } + + ServerOptionsExecutor getServerOptionsExecutor() { + return serverOptionsExecutor; + } + } + + /** + * The executor interface for running the appropriate HiveServer2 command based on parsed options + */ + interface ServerOptionsExecutor { + void execute(); + } + + /** + * HelpOptionExecutor: executes the --help option by printing out the usage + */ + static class HelpOptionExecutor implements ServerOptionsExecutor { + private final Options options; + private final String serverName; + + HelpOptionExecutor(String serverName, Options options) { + this.options = options; + this.serverName = serverName; + } + + @Override + public void execute() { + new HelpFormatter().printHelp(serverName, options); + System.exit(0); + } + } + + /** + * StartOptionExecutor: starts HiveServer2. + * This is the default executor, when no option is specified. + */ + static class StartOptionExecutor implements ServerOptionsExecutor { + @Override + public void execute() { + try { + startHiveServer2(); + } catch (Throwable t) { + LOG.fatal("Error starting HiveServer2", t); + System.exit(-1); + } + } + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadFactoryWithGarbageCleanup.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadFactoryWithGarbageCleanup.java new file mode 100644 index 000000000000..94f8126552e9 --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadFactoryWithGarbageCleanup.java @@ -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.hive.service.server; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ThreadFactory; + +import org.apache.hadoop.hive.metastore.RawStore; + +/** + * A ThreadFactory for constructing new HiveServer2 threads that lets you plug + * in custom cleanup code to be called before this thread is GC-ed. + * Currently cleans up the following: + * 1. ThreadLocal RawStore object: + * In case of an embedded metastore, HiveServer2 threads (foreground and background) + * end up caching a ThreadLocal RawStore object. The ThreadLocal RawStore object has + * an instance of PersistenceManagerFactory and PersistenceManager. + * The PersistenceManagerFactory keeps a cache of PersistenceManager objects, + * which are only removed when PersistenceManager#close method is called. + * HiveServer2 uses ExecutorService for managing thread pools for foreground and background threads. + * ExecutorService unfortunately does not provide any hooks to be called, + * when a thread from the pool is terminated. + * As a solution, we're using this ThreadFactory to keep a cache of RawStore objects per thread. + * And we are doing clean shutdown in the finalizer for each thread. + */ +public class ThreadFactoryWithGarbageCleanup implements ThreadFactory { + + private static Map threadRawStoreMap = new ConcurrentHashMap(); + + private final String namePrefix; + + public ThreadFactoryWithGarbageCleanup(String threadPoolName) { + namePrefix = threadPoolName; + } + + @Override + public Thread newThread(Runnable runnable) { + Thread newThread = new ThreadWithGarbageCleanup(runnable); + newThread.setName(namePrefix + ": Thread-" + newThread.getId()); + return newThread; + } + + public static Map getThreadRawStoreMap() { + return threadRawStoreMap; + } +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java new file mode 100644 index 000000000000..8ee98103f7ef --- /dev/null +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java @@ -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.hive.service.server; + +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.metastore.HiveMetaStore; +import org.apache.hadoop.hive.metastore.RawStore; + +/** + * A HiveServer2 thread used to construct new server threads. + * In particular, this thread ensures an orderly cleanup, + * when killed by its corresponding ExecutorService. + */ +public class ThreadWithGarbageCleanup extends Thread { + private static final Log LOG = LogFactory.getLog(ThreadWithGarbageCleanup.class); + + Map threadRawStoreMap = + ThreadFactoryWithGarbageCleanup.getThreadRawStoreMap(); + + public ThreadWithGarbageCleanup(Runnable runnable) { + super(runnable); + } + + /** + * Add any Thread specific garbage cleanup code here. + * Currently, it shuts down the RawStore object for this thread if it is not null. + */ + @Override + public void finalize() throws Throwable { + cleanRawStore(); + super.finalize(); + } + + private void cleanRawStore() { + Long threadId = this.getId(); + RawStore threadLocalRawStore = threadRawStoreMap.get(threadId); + if (threadLocalRawStore != null) { + LOG.debug("RawStore: " + threadLocalRawStore + ", for the thread: " + + this.getName() + " will be closed now."); + threadLocalRawStore.shutdown(); + threadRawStoreMap.remove(threadId); + } + } + + /** + * Cache the ThreadLocal RawStore object. Called from the corresponding thread. + */ + public void cacheThreadLocalRawStore() { + Long threadId = this.getId(); + RawStore threadLocalRawStore = HiveMetaStore.HMSHandler.getRawStore(); + if (threadLocalRawStore != null && !threadRawStoreMap.containsKey(threadId)) { + LOG.debug("Adding RawStore: " + threadLocalRawStore + ", for the thread: " + + this.getName() + " to threadRawStoreMap for future cleanup."); + threadRawStoreMap.put(threadId, threadLocalRawStore); + } + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/hive/service/server/HiveServerServerOptionsProcessor.scala b/sql/hive-thriftserver/src/main/scala/org/apache/hive/service/server/HiveServerServerOptionsProcessor.scala deleted file mode 100644 index 2228f651e238..000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/hive/service/server/HiveServerServerOptionsProcessor.scala +++ /dev/null @@ -1,37 +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.hive.service.server - -import org.apache.hive.service.server.HiveServer2.{StartOptionExecutor, ServerOptionsProcessor} - -/** - * Class to upgrade a package-private class to public, and - * implement a `process()` operation consistent with - * the behavior of older Hive versions - * @param serverName name of the hive server - */ -private[apache] class HiveServerServerOptionsProcessor(serverName: String) - extends ServerOptionsProcessor(serverName) { - - def process(args: Array[String]): Boolean = { - // A parse failure automatically triggers a system exit - val response = super.parse(args) - val executor = response.getServerOptionsExecutor() - // return true if the parsed option was to start the service - executor.isInstanceOf[StartOptionExecutor] - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index a4fd0c3ce970..7442c987efc7 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -23,29 +23,28 @@ import java.util.concurrent.atomic.AtomicBoolean import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} -import org.apache.hive.service.server.{HiveServerServerOptionsProcessor, HiveServer2} +import org.apache.hive.service.server.HiveServer2 +import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart} -import org.apache.spark.sql.SQLConf -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.{ShutdownHookManager, Utils} -import org.apache.spark.{Logging, SparkContext} - /** * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a * `HiveThriftServer2` thrift server. */ object HiveThriftServer2 extends Logging { - var LOG = LogFactory.getLog(classOf[HiveServer2]) - var uiTab: Option[ThriftServerTab] = _ + var uiTab: Option[ThriftServerTab] = None var listener: HiveThriftServer2Listener = _ /** @@ -53,9 +52,14 @@ object HiveThriftServer2 extends Logging { * Starts a new thrift server with the given context. */ @DeveloperApi - def startWithContext(sqlContext: HiveContext): Unit = { + def startWithContext(sqlContext: SQLContext): Unit = { val server = new HiveThriftServer2(sqlContext) - server.init(sqlContext.hiveconf) + + val executionHive = HiveUtils.newClientForExecution( + sqlContext.sparkContext.conf, + sqlContext.sessionState.newHadoopConf()) + + server.init(executionHive.conf) server.start() listener = new HiveThriftServer2Listener(server, sqlContext.conf) sqlContext.sparkContext.addSparkListener(listener) @@ -67,10 +71,9 @@ object HiveThriftServer2 extends Logging { } def main(args: Array[String]) { - val optionsProcessor = new HiveServerServerOptionsProcessor("HiveThriftServer2") - if (!optionsProcessor.process(args)) { - System.exit(-1) - } + Utils.initDaemon(log) + val optionsProcessor = new HiveServer2.ServerOptionsProcessor("HiveThriftServer2") + optionsProcessor.parse(args) logInfo("Starting SparkContext") SparkSQLEnv.init() @@ -80,12 +83,16 @@ object HiveThriftServer2 extends Logging { uiTab.foreach(_.detach()) } + val executionHive = HiveUtils.newClientForExecution( + SparkSQLEnv.sqlContext.sparkContext.conf, + SparkSQLEnv.sqlContext.sessionState.newHadoopConf()) + try { - val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) - server.init(SparkSQLEnv.hiveContext.hiveconf) + val server = new HiveThriftServer2(SparkSQLEnv.sqlContext) + server.init(executionHive.conf) server.start() logInfo("HiveThriftServer2 started") - listener = new HiveThriftServer2Listener(server, SparkSQLEnv.hiveContext.conf) + listener = new HiveThriftServer2Listener(server, SparkSQLEnv.sqlContext.conf) SparkSQLEnv.sparkContext.addSparkListener(listener) uiTab = if (SparkSQLEnv.sparkContext.getConf.getBoolean("spark.ui.enabled", true)) { Some(new ThriftServerTab(SparkSQLEnv.sparkContext)) @@ -148,7 +155,7 @@ object HiveThriftServer2 extends Logging { /** - * A inner sparkListener called in sc.stop to clean up the HiveThriftServer2 + * An inner sparkListener called in sc.stop to clean up the HiveThriftServer2 */ private[thriftserver] class HiveThriftServer2Listener( val server: HiveServer2, @@ -260,7 +267,7 @@ object HiveThriftServer2 extends Logging { } } -private[hive] class HiveThriftServer2(hiveContext: HiveContext) +private[hive] class HiveThriftServer2(sqlContext: SQLContext) extends HiveServer2 with ReflectedCompositeService { // state is tracked internally so that the server only attempts to shut down if it successfully @@ -268,7 +275,7 @@ private[hive] class HiveThriftServer2(hiveContext: HiveContext) private val started = new AtomicBoolean(false) override def init(hiveConf: HiveConf) { - val sparkSqlCliService = new SparkSQLCLIService(this, hiveContext) + val sparkSqlCliService = new SparkSQLCLIService(this, sqlContext) setSuperField(this, "cliService", sparkSqlCliService) addService(sparkSqlCliService) @@ -285,7 +292,7 @@ private[hive] class HiveThriftServer2(hiveContext: HiveContext) private def isHTTPTransportMode(hiveConf: HiveConf): Boolean = { val transportMode = hiveConf.getVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE) - transportMode.toLowerCase(Locale.ENGLISH).equals("http") + transportMode.toLowerCase(Locale.ROOT).equals("http") } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 719b03e1c7c7..f5191fa9132b 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.hive.thriftserver import java.security.PrivilegedExceptionAction import java.sql.{Date, Timestamp} +import java.util.{Arrays, Map => JMap, UUID} import java.util.concurrent.RejectedExecutionException -import java.util.{Arrays, UUID, Map => JMap} import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, Map => SMap} +import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal import org.apache.hadoop.hive.metastore.api.FieldSchema @@ -32,32 +32,48 @@ import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.ExecuteStatementOperation import org.apache.hive.service.cli.session.HiveSession -import org.apache.spark.Logging -import org.apache.spark.sql.execution.SetCommand -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLContext} +import org.apache.spark.sql.execution.command.SetCommand +import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.sql.{DataFrame, SQLConf, Row => SparkRow} - +import org.apache.spark.util.{Utils => SparkUtils} private[hive] class SparkExecuteStatementOperation( parentSession: HiveSession, statement: String, confOverlay: JMap[String, String], runInBackground: Boolean = true) - (hiveContext: HiveContext, sessionToActivePool: SMap[SessionHandle, String]) + (sqlContext: SQLContext, sessionToActivePool: JMap[SessionHandle, String]) extends ExecuteStatementOperation(parentSession, statement, confOverlay, runInBackground) with Logging { private var result: DataFrame = _ + + // We cache the returned rows to get iterators again in case the user wants to use FETCH_FIRST. + // This is only used when `spark.sql.thriftServer.incrementalCollect` is set to `false`. + // In case of `true`, this will be `None` and FETCH_FIRST will trigger re-execution. + private var resultList: Option[Array[SparkRow]] = _ + private var iter: Iterator[SparkRow] = _ private var dataTypes: Array[DataType] = _ private var statementId: String = _ + private lazy val resultSchema: TableSchema = { + if (result == null || result.schema.isEmpty) { + new TableSchema(Arrays.asList(new FieldSchema("Result", "string", ""))) + } else { + logInfo(s"Result Schema: ${result.schema}") + SparkExecuteStatementOperation.getTableSchema(result.schema) + } + } + def close(): Unit = { // RDDs will be cleaned automatically upon garbage collection. - hiveContext.sparkContext.clearJobGroup() logDebug(s"CLOSING $statementId") cleanup(OperationState.CLOSED) + sqlContext.sparkContext.clearJobGroup() } def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int) { @@ -83,9 +99,11 @@ private[hive] class SparkExecuteStatementOperation( case DateType => to += from.getAs[Date](ordinal) case TimestampType => - to += from.getAs[Timestamp](ordinal) - case BinaryType | _: ArrayType | _: StructType | _: MapType => - val hiveString = HiveContext.toHiveString((from.get(ordinal), dataTypes(ordinal))) + to += from.getAs[Timestamp](ordinal) + case BinaryType => + to += from.getAs[Array[Byte]](ordinal) + case _: ArrayType | _: StructType | _: MapType => + val hiveString = HiveUtils.toHiveString((from.get(ordinal), dataTypes(ordinal))) to += hiveString } } @@ -95,6 +113,20 @@ private[hive] class SparkExecuteStatementOperation( assertState(OperationState.FINISHED) setHasResultSet(true) val resultRowSet: RowSet = RowSetFactory.create(getResultSetSchema, getProtocolVersion) + + // Reset iter to header when fetching start from first row + if (order.equals(FetchOrientation.FETCH_FIRST)) { + iter = if (sqlContext.getConf(SQLConf.THRIFTSERVER_INCREMENTAL_COLLECT.key).toBoolean) { + resultList = None + result.toLocalIterator.asScala + } else { + if (resultList.isEmpty) { + resultList = Some(result.collect()) + } + resultList.get.iterator + } + } + if (!iter.hasNext) { resultRowSet } else { @@ -120,24 +152,14 @@ private[hive] class SparkExecuteStatementOperation( } } - def getResultSetSchema: TableSchema = { - if (result == null || result.queryExecution.analyzed.output.size == 0) { - new TableSchema(Arrays.asList(new FieldSchema("Result", "string", ""))) - } else { - logInfo(s"Result Schema: ${result.queryExecution.analyzed.output}") - val schema = result.queryExecution.analyzed.output.map { attr => - new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") - } - new TableSchema(schema.asJava) - } - } + def getResultSetSchema: TableSchema = resultSchema - override def run(): Unit = { + override def runInternal(): Unit = { setState(OperationState.PENDING) setHasResultSet(true) // avoid no resultset for async run if (!runInBackground) { - runInternal() + execute() } else { val sparkServiceUGI = Utils.getUGI() @@ -149,7 +171,7 @@ private[hive] class SparkExecuteStatementOperation( val doAsAction = new PrivilegedExceptionAction[Unit]() { override def run(): Unit = { try { - runInternal() + execute() } catch { case e: HiveSQLException => setOperationException(e) @@ -186,13 +208,12 @@ private[hive] class SparkExecuteStatementOperation( } } - override def runInternal(): Unit = { + private def execute(): Unit = { statementId = UUID.randomUUID().toString logInfo(s"Running query '$statement' with $statementId") setState(OperationState.RUNNING) // Always use the latest class loader provided by executionHive's state. - val executionHiveClassLoader = - hiveContext.executionHive.state.getConf.getClassLoader + val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) HiveThriftServer2.listener.onStatementStart( @@ -201,27 +222,28 @@ private[hive] class SparkExecuteStatementOperation( statement, statementId, parentSession.getUsername) - hiveContext.sparkContext.setJobGroup(statementId, statement) - sessionToActivePool.get(parentSession.getSessionHandle).foreach { pool => - hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) + sqlContext.sparkContext.setJobGroup(statementId, statement) + val pool = sessionToActivePool.get(parentSession.getSessionHandle) + if (pool != null) { + sqlContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) } try { - result = hiveContext.sql(statement) + result = sqlContext.sql(statement) logDebug(result.queryExecution.toString()) result.queryExecution.logical match { case SetCommand(Some((SQLConf.THRIFTSERVER_POOL.key, Some(value)))) => - sessionToActivePool(parentSession.getSessionHandle) = value + sessionToActivePool.put(parentSession.getSessionHandle, value) logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") case _ => } HiveThriftServer2.listener.onStatementParsed(statementId, result.queryExecution.toString()) iter = { - val useIncrementalCollect = - hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean - if (useIncrementalCollect) { - result.rdd.toLocalIterator + if (sqlContext.getConf(SQLConf.THRIFTSERVER_INCREMENTAL_COLLECT.key).toBoolean) { + resultList = None + result.toLocalIterator.asScala } else { - result.collect().iterator + resultList = Some(result.collect()) + resultList.get.iterator } } dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray @@ -230,7 +252,9 @@ private[hive] class SparkExecuteStatementOperation( if (getStatus().getState() == OperationState.CANCELED) { return } else { - setState(OperationState.ERROR); + setState(OperationState.ERROR) + HiveThriftServer2.listener.onStatementError( + statementId, e.getMessage, SparkUtils.exceptionString(e)) throw e } // Actually do need to catch Throwable as some failures don't inherit from Exception and @@ -240,7 +264,7 @@ private[hive] class SparkExecuteStatementOperation( logError(s"Error executing query, currentState $currentState, ", e) setState(OperationState.ERROR) HiveThriftServer2.listener.onStatementError( - statementId, e.getMessage, e.getStackTraceString) + statementId, e.getMessage, SparkUtils.exceptionString(e)) throw new HiveSQLException(e.toString) } setState(OperationState.FINISHED) @@ -249,9 +273,6 @@ private[hive] class SparkExecuteStatementOperation( override def cancel(): Unit = { logInfo(s"Cancel '$statement' with $statementId") - if (statementId != null) { - hiveContext.sparkContext.cancelJobGroup(statementId) - } cleanup(OperationState.CANCELED) } @@ -263,5 +284,18 @@ private[hive] class SparkExecuteStatementOperation( backgroundHandle.cancel(true) } } + if (statementId != null) { + sqlContext.sparkContext.cancelJobGroup(statementId) + } + } +} + +object SparkExecuteStatementOperation { + def getTableSchema(structType: StructType): TableSchema = { + val schema = structType.map { field => + val attrTypeString = if (field.dataType == NullType) "void" else field.dataType.catalogString + new FieldSchema(field.name, attrTypeString, field.getComment.getOrElse("")) + } + new TableSchema(schema.asJava) } } 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 6419002a2aa8..832a15d09599 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -20,13 +20,10 @@ package org.apache.spark.sql.hive.thriftserver import java.io._ import java.util.{ArrayList => JArrayList, Locale} -import org.apache.spark.sql.AnalysisException - import scala.collection.JavaConverters._ import jline.console.ConsoleReader import jline.console.history.FileHistory - import org.apache.commons.lang3.StringUtils import org.apache.commons.logging.LogFactory import org.apache.hadoop.conf.Configuration @@ -35,22 +32,27 @@ import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, SetProcessor, CommandProcessor, CommandProcessorFactory} +import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.log4j.{Level, Logger} import org.apache.thrift.transport.TSocket -import org.apache.spark.Logging -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.util.{ShutdownHookManager, Utils} +import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.util.ShutdownHookManager /** * This code doesn't support remote connections in Hive 1.2+, as the underlying CliDriver * has dropped its support. */ private[hive] object SparkSQLCLIDriver extends Logging { - private var prompt = "spark-sql" - private var continuedPrompt = "".padTo(prompt.length, ' ') + private val prompt = "spark-sql" + private val continuedPrompt = "".padTo(prompt.length, ' ') private var transport: TSocket = _ + private final val SPARK_HADOOP_PROP_PREFIX = "spark.hadoop." installSignalHandler() @@ -81,11 +83,17 @@ private[hive] object SparkSQLCLIDriver extends Logging { System.exit(1) } + val sparkConf = new SparkConf(loadDefaults = true) + val hadoopConf = SparkHadoopUtil.get.newConfiguration(sparkConf) + val extraConfigs = HiveUtils.formatTimeVarsForHiveClient(hadoopConf) + val cliConf = new HiveConf(classOf[SessionState]) - // Override the location of the metastore since this is only used for local execution. - HiveContext.newTemporaryConfiguration().foreach { - case (key, value) => cliConf.set(key, value) + (hadoopConf.iterator().asScala.map(kv => kv.getKey -> kv.getValue) + ++ sparkConf.getAll.toMap ++ extraConfigs).foreach { + case (k, v) => + cliConf.set(k, v) } + val sessionState = new CliSessionState(cliConf) sessionState.in = System.in @@ -135,6 +143,16 @@ private[hive] object SparkSQLCLIDriver extends Logging { // Hive 1.2 + not supported in CLI throw new RuntimeException("Remote operations not supported") } + // Respect the configurations set by --hiveconf from the command line + // (based on Hive's CliDriver). + val hiveConfFromCmd = sessionState.getOverriddenConfigurations.entrySet().asScala + val newHiveConf = hiveConfFromCmd.map { kv => + // If the same property is configured by spark.hadoop.xxx, we ignore it and + // obey settings from spark properties + val k = kv.getKey + val v = sys.props.getOrElseUpdate(SPARK_HADOOP_PROP_PREFIX + k, kv.getValue) + (k, v) + } val cli = new SparkSQLCLIDriver cli.setHiveVariables(oproc.getHiveVariables) @@ -151,12 +169,17 @@ private[hive] object SparkSQLCLIDriver extends Logging { } if (sessionState.database != null) { - SparkSQLEnv.hiveContext.runSqlHive(s"USE ${sessionState.database}") + SparkSQLEnv.sqlContext.sessionState.catalog.setCurrentDatabase( + s"${sessionState.database}") } // Execute -i init files (always in silent mode) cli.processInitFiles(sessionState) + newHiveConf.foreach { kv => + SparkSQLEnv.sqlContext.setConf(kv._1, kv._2) + } + if (sessionState.execString != null) { System.exit(cli.processLine(sessionState.execString)) } @@ -194,6 +217,20 @@ private[hive] object SparkSQLCLIDriver extends Logging { logWarning(e.getMessage) } + // add shutdown hook to flush the history to history file + ShutdownHookManager.addShutdownHook { () => + reader.getHistory match { + case h: FileHistory => + try { + h.flush() + } catch { + case e: IOException => + logWarning("WARNING: Failed to write command history file: " + e.getMessage) + } + case _ => + } + } + // TODO: missing /* val clientTransportTSocketField = classOf[CliSessionState].getDeclaredField("transport") @@ -250,10 +287,14 @@ private[hive] object SparkSQLCLIDriver extends Logging { private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { private val sessionState = SessionState.get().asInstanceOf[CliSessionState] - private val LOG = LogFactory.getLog("CliDriver") + private val LOG = LogFactory.getLog(classOf[SparkSQLCLIDriver]) private val console = new SessionState.LogHelper(LOG) + if (sessionState.getIsSilent) { + Logger.getRootLogger.setLevel(Level.WARN) + } + private val isRemoteMode = { SparkSQLCLIDriver.isRemoteMode(sessionState) } @@ -270,17 +311,22 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { throw new RuntimeException("Remote operations not supported") } + override def setHiveVariables(hiveVariables: java.util.Map[String, String]): Unit = { + hiveVariables.asScala.foreach(kv => SparkSQLEnv.sqlContext.conf.setConfString(kv._1, kv._2)) + } + override def processCmd(cmd: String): Int = { val cmd_trimmed: String = cmd.trim() - val cmd_lower = cmd_trimmed.toLowerCase(Locale.ENGLISH) + val cmd_lower = cmd_trimmed.toLowerCase(Locale.ROOT) val tokens: Array[String] = cmd_trimmed.split("\\s+") val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() if (cmd_lower.equals("quit") || - cmd_lower.equals("exit") || - tokens(0).toLowerCase(Locale.ENGLISH).equals("source") || - cmd_trimmed.startsWith("!") || - tokens(0).toLowerCase.equals("list") || - isRemoteMode) { + cmd_lower.equals("exit")) { + sessionState.close() + System.exit(0) + } + if (tokens(0).toLowerCase(Locale.ROOT).equals("source") || + cmd_trimmed.startsWith("!") || isRemoteMode) { val start = System.currentTimeMillis() super.processCmd(cmd) val end = System.currentTimeMillis() @@ -295,7 +341,8 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (proc != null) { // scalastyle:off println if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor] || - proc.isInstanceOf[AddResourceProcessor]) { + proc.isInstanceOf[AddResourceProcessor] || proc.isInstanceOf[ListResourceProcessor] || + proc.isInstanceOf[ResetProcessor] ) { val driver = new SparkSQLDriver driver.init() @@ -357,7 +404,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (counter != 0) { responseMsg += s", Fetched $counter row(s)" } - console.printInfo(responseMsg , null) + console.printInfo(responseMsg, null) // Destroy the driver to release all the locks. driver.destroy() } else { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala index 5ad8c54f296d..ad1f5eb9ca3a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -25,28 +25,30 @@ import scala.collection.JavaConverters._ import org.apache.commons.logging.Log import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.shims.Utils import org.apache.hadoop.security.UserGroupInformation +import org.apache.hive.service.{AbstractService, Service, ServiceException} import org.apache.hive.service.Service.STATE import org.apache.hive.service.auth.HiveAuthFactory import org.apache.hive.service.cli._ import org.apache.hive.service.server.HiveServer2 -import org.apache.hive.service.{AbstractService, Service, ServiceException} -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ -private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, hiveContext: HiveContext) +private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, sqlContext: SQLContext) extends CLIService(hiveServer) with ReflectedCompositeService { override def init(hiveConf: HiveConf) { setSuperField(this, "hiveConf", hiveConf) - val sparkSqlSessionManager = new SparkSQLSessionManager(hiveServer, hiveContext) + val sparkSqlSessionManager = new SparkSQLSessionManager(hiveServer, sqlContext) setSuperField(this, "sessionManager", sparkSqlSessionManager) addService(sparkSqlSessionManager) var sparkServiceUGI: UserGroupInformation = null + var httpUGI: UserGroupInformation = null if (UserGroupInformation.isSecurityEnabled) { try { @@ -57,6 +59,20 @@ private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, hiveContext: Hiv case e @ (_: IOException | _: LoginException) => throw new ServiceException("Unable to login to kerberos with given principal/keytab", e) } + + // Try creating spnego UGI if it is configured. + val principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_PRINCIPAL).trim + val keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_KEYTAB).trim + if (principal.nonEmpty && keyTabFile.nonEmpty) { + try { + httpUGI = HiveAuthFactory.loginFromSpnegoKeytabAndReturnUGI(hiveConf) + setSuperField(this, "httpUGI", httpUGI) + } catch { + case e: IOException => + throw new ServiceException("Unable to login to spnego with given principal " + + s"$principal and keytab $keyTabFile: $e", e) + } + } } initCompositeService(hiveConf) @@ -66,7 +82,7 @@ private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, hiveContext: Hiv getInfoType match { case GetInfoType.CLI_SERVER_NAME => new GetInfoValue("Spark SQL") case GetInfoType.CLI_DBMS_NAME => new GetInfoValue("Spark SQL") - case GetInfoType.CLI_DBMS_VER => new GetInfoValue(hiveContext.sparkContext.version) + case GetInfoType.CLI_DBMS_VER => new GetInfoValue(sqlContext.sparkContext.version) case _ => super.getInfo(sessionHandle, getInfoType) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index f1ec7238520a..677590217344 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -17,9 +17,7 @@ package org.apache.spark.sql.hive.thriftserver -import java.util.{Arrays, ArrayList => JArrayList, List => JList} -import org.apache.log4j.LogManager -import org.apache.spark.sql.AnalysisException +import java.util.{ArrayList => JArrayList, Arrays, List => JList} import scala.collection.JavaConverters._ @@ -28,11 +26,12 @@ import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse -import org.apache.spark.Logging -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{AnalysisException, SQLContext} +import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} -private[hive] class SparkSQLDriver( - val context: HiveContext = SparkSQLEnv.hiveContext) + +private[hive] class SparkSQLDriver(val context: SQLContext = SparkSQLEnv.sqlContext) extends Driver with Logging { @@ -42,14 +41,14 @@ private[hive] class SparkSQLDriver( override def init(): Unit = { } - private def getResultSetSchema(query: context.QueryExecution): Schema = { + private def getResultSetSchema(query: QueryExecution): Schema = { val analyzed = query.analyzed logDebug(s"Result Schema: ${analyzed.output}") if (analyzed.output.isEmpty) { new Schema(Arrays.asList(new FieldSchema("Response code", "string", "")), null) } else { val fieldSchemas = analyzed.output.map { attr => - new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + new FieldSchema(attr.name, attr.dataType.catalogString, "") } new Schema(fieldSchemas.asJava, null) @@ -60,8 +59,10 @@ private[hive] class SparkSQLDriver( // TODO unify the error code try { context.sparkContext.setJobDescription(command) - val execution = context.executePlan(context.sql(command).logicalPlan) - hiveResponse = execution.stringResult() + val execution = context.sessionState.executePlan(context.sql(command).logicalPlan) + hiveResponse = SQLExecution.withNewExecutionId(context.sparkSession, execution) { + execution.hiveResultString() + } tableSchema = getResultSetSchema(execution) new CommandProcessorResponse(0) } catch { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index bacf6cc458fd..01c4eb131a56 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -19,55 +19,43 @@ package org.apache.spark.sql.hive.thriftserver import java.io.PrintStream -import scala.collection.JavaConverters._ - -import org.apache.spark.scheduler.StatsReportListener -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{SparkSession, SQLContext} +import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} import org.apache.spark.util.Utils /** A singleton object for the master program. The slaves should not access this. */ private[hive] object SparkSQLEnv extends Logging { logDebug("Initializing SparkSQLEnv") - var hiveContext: HiveContext = _ + var sqlContext: SQLContext = _ var sparkContext: SparkContext = _ def init() { - if (hiveContext == null) { + if (sqlContext == null) { val sparkConf = new SparkConf(loadDefaults = true) - val maybeSerializer = sparkConf.getOption("spark.serializer") - val maybeKryoReferenceTracking = sparkConf.getOption("spark.kryo.referenceTracking") // If user doesn't specify the appName, we want to get [SparkSQL::localHostName] instead of // the default appName [SparkSQLCLIDriver] in cli or beeline. val maybeAppName = sparkConf .getOption("spark.app.name") .filterNot(_ == classOf[SparkSQLCLIDriver].getName) + .filterNot(_ == classOf[HiveThriftServer2].getName) sparkConf .setAppName(maybeAppName.getOrElse(s"SparkSQL::${Utils.localHostName()}")) - .set( - "spark.serializer", - maybeSerializer.getOrElse("org.apache.spark.serializer.KryoSerializer")) - .set( - "spark.kryo.referenceTracking", - maybeKryoReferenceTracking.getOrElse("false")) - - sparkContext = new SparkContext(sparkConf) - sparkContext.addSparkListener(new StatsReportListener()) - hiveContext = new HiveContext(sparkContext) - - hiveContext.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8")) - hiveContext.metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8")) - hiveContext.metadataHive.setError(new PrintStream(System.err, true, "UTF-8")) - hiveContext.setConf("spark.sql.hive.version", HiveContext.hiveExecutionVersion) + val sparkSession = SparkSession.builder.config(sparkConf).enableHiveSupport().getOrCreate() + sparkContext = sparkSession.sparkContext + sqlContext = sparkSession.sqlContext - if (log.isDebugEnabled) { - hiveContext.hiveconf.getAllProperties.asScala.toSeq.sorted.foreach { case (k, v) => - logDebug(s"HiveConf var: $k=$v") - } - } + val metadataHive = sparkSession + .sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog] + .client.newSession() + metadataHive.setOut(new PrintStream(System.out, true, "UTF-8")) + metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8")) + metadataHive.setError(new PrintStream(System.err, true, "UTF-8")) + sparkSession.conf.set("spark.sql.hive.version", HiveUtils.hiveExecutionVersion) } } @@ -78,7 +66,7 @@ private[hive] object SparkSQLEnv extends Logging { if (SparkSQLEnv.sparkContext != null) { sparkContext.stop() sparkContext = null - hiveContext = null + sqlContext = null } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index 33aaead3fbf9..7adaafe5ad5c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -27,12 +27,13 @@ import org.apache.hive.service.cli.session.SessionManager import org.apache.hive.service.cli.thrift.TProtocolVersion import org.apache.hive.service.server.HiveServer2 -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager -private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, hiveContext: HiveContext) +private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: SQLContext) extends SessionManager(hiveServer) with ReflectedCompositeService { @@ -41,6 +42,11 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, hiveContext: override def init(hiveConf: HiveConf) { setSuperField(this, "hiveConf", hiveConf) + // Create operation log root directory, if operation logging is enabled + if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) { + invoke(classOf[SessionManager], this, "initOperationLogRootDir") + } + val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS) setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) getAncestorField[Log](this, 3, "LOG").info( @@ -66,16 +72,23 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, hiveContext: val session = super.getSession(sessionHandle) HiveThriftServer2.listener.onSessionCreated( session.getIpAddress, sessionHandle.getSessionId.toString, session.getUsername) - val ctx = hiveContext.newSession() - ctx.setConf("spark.sql.hive.version", HiveContext.hiveExecutionVersion) - sparkSqlOperationManager.sessionToContexts += sessionHandle -> ctx + val ctx = if (sqlContext.conf.hiveThriftServerSingleSession) { + sqlContext + } else { + sqlContext.newSession() + } + ctx.setConf("spark.sql.hive.version", HiveUtils.hiveExecutionVersion) + if (sessionConf != null && sessionConf.containsKey("use:database")) { + ctx.sql(s"use ${sessionConf.get("use:database")}") + } + sparkSqlOperationManager.sessionToContexts.put(sessionHandle, ctx) sessionHandle } override def closeSession(sessionHandle: SessionHandle) { HiveThriftServer2.listener.onSessionClosed(sessionHandle.getSessionId.toString) super.closeSession(sessionHandle) - sparkSqlOperationManager.sessionToActivePool -= sessionHandle + sparkSqlOperationManager.sessionToActivePool.remove(sessionHandle) sparkSqlOperationManager.sessionToContexts.remove(sessionHandle) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 476651a559d2..a0e5012633f5 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -18,14 +18,16 @@ package org.apache.spark.sql.hive.thriftserver.server import java.util.{Map => JMap} -import scala.collection.mutable.Map +import java.util.concurrent.ConcurrentHashMap import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} import org.apache.hive.service.cli.session.HiveSession -import org.apache.spark.Logging -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.thriftserver.{SparkExecuteStatementOperation, ReflectionUtils} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.hive.thriftserver.{ReflectionUtils, SparkExecuteStatementOperation} /** * Executes queries using Spark SQL, and maintains a list of handles to active queries. @@ -36,18 +38,21 @@ private[thriftserver] class SparkSQLOperationManager() val handleToOperation = ReflectionUtils .getSuperField[JMap[OperationHandle, Operation]](this, "handleToOperation") - val sessionToActivePool = Map[SessionHandle, String]() - val sessionToContexts = Map[SessionHandle, HiveContext]() + val sessionToActivePool = new ConcurrentHashMap[SessionHandle, String]() + val sessionToContexts = new ConcurrentHashMap[SessionHandle, SQLContext]() override def newExecuteStatementOperation( parentSession: HiveSession, statement: String, confOverlay: JMap[String, String], async: Boolean): ExecuteStatementOperation = synchronized { - val hiveContext = sessionToContexts(parentSession.getSessionHandle) - val runInBackground = async && hiveContext.hiveThriftServerAsync + val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) + require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + + s" initialized or had already closed.") + val conf = sqlContext.sessionState.conf + val runInBackground = async && conf.getConf(HiveUtils.HIVE_THRIFT_SERVER_ASYNC) val operation = new SparkExecuteStatementOperation(parentSession, statement, confOverlay, - runInBackground)(hiveContext, sessionToActivePool) + runInBackground)(sqlContext, sessionToActivePool) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created Operation for $statement with session=$parentSession, " + s"runInBackground=$runInBackground") diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index e990bd06011f..f517bffccdf3 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -23,13 +23,14 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.commons.lang3.StringEscapeUtils -import org.apache.spark.Logging -import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{SessionInfo, ExecutionState, ExecutionInfo} -import org.apache.spark.ui.UIUtils._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo, ExecutionState, SessionInfo} import org.apache.spark.ui._ +import org.apache.spark.ui.UIUtils._ -/** Page for Spark Web UI that shows statistics of a thrift server */ +/** Page for Spark Web UI that shows statistics of the thrift server */ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("") with Logging { private val listener = parent.listener @@ -71,7 +72,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" val table = if (numStatement > 0) { val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Duration", "Statement", "State", "Detail") - val dataRows = listener.getExecutionList + val dataRows = listener.getExecutionList.sortBy(_.startTimestamp).reverse def generateDataRow(info: ExecutionInfo): Seq[Node] = { val jobLink = info.jobId.map { id: String => @@ -102,7 +103,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" } val content = -
    SQL Statistics
    ++ +
    SQL Statistics ({numStatement})
    ++
      {table.getOrElse("No statistics have been generated yet.")} @@ -141,7 +142,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" val sessionList = listener.getSessionList val numBatches = sessionList.size val table = if (numBatches > 0) { - val dataRows = sessionList + val dataRows = sessionList.sortBy(_.startTimestamp).reverse val headerRow = Seq("User", "IP", "Session ID", "Start Time", "Finish Time", "Duration", "Total Execute") def generateDataRow(session: SessionInfo): Seq[Node] = { @@ -163,7 +164,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" } val content = -
      Session Statistics
      ++ +
      Session Statistics ({numBatches})
      ++
        {table.getOrElse("No statistics have been generated yet.")} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index af16cb31df18..5cd2fdf6437c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -23,12 +23,13 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.commons.lang3.StringEscapeUtils -import org.apache.spark.Logging + +import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo, ExecutionState} -import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui._ +import org.apache.spark.ui.UIUtils._ -/** Page for Spark Web UI that shows statistics of a streaming job */ +/** Page for Spark Web UI that shows statistics of jobs running in the thrift server */ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) extends WebUIPage("session") with Logging { @@ -38,7 +39,8 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { - val parameterId = request.getParameter("id") + // stripXSS is called first to remove suspicious characters used in XSS attacks + val parameterId = UIUtils.stripXSS(request.getParameter("id")) require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") val content = @@ -59,12 +61,12 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) UIUtils.headerSparkPage("JDBC/ODBC Session", content, parent, Some(5000)) } - /** Generate basic stats of the streaming program */ + /** Generate basic stats of the thrift server program */ private def generateBasicStats(): Seq[Node] = { val timeSinceStart = System.currentTimeMillis() - startTime.getTime
        • - Started at: {startTime.toString} + Started at: {formatDate(startTime)}
        • Time since start: {formatDurationVerbose(timeSinceStart)} @@ -145,42 +147,6 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab)
    } - /** Generate stats of batch sessions of the thrift server program */ - private def generateSessionStatsTable(): Seq[Node] = { - val sessionList = listener.getSessionList - val numBatches = sessionList.size - val table = if (numBatches > 0) { - val dataRows = - sessionList.sortBy(_.startTimestamp).reverse.map ( session => - Seq( - session.userName, - session.ip, - session.sessionId, - formatDate(session.startTimestamp), - formatDate(session.finishTimestamp), - formatDurationOption(Some(session.totalTime)), - session.totalExecution.toString - ) - ).toSeq - val headerRow = Seq("User", "IP", "Session ID", "Start Time", "Finish Time", "Duration", - "Total Execute") - Some(listingTable(headerRow, dataRows)) - } else { - None - } - - val content = -
    Session Statistics
    ++ -
    -
      - {table.getOrElse("No statistics have been generated yet.")} -
    -
    - - content - } - - /** * Returns a human-readable string representing a duration such as "5 second 35 ms" */ @@ -196,4 +162,3 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) UIUtils.listingTable(headers, generateDataRow, data, fixedWidth = true) } } - diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala index 4eabeaa6735e..db2066009b35 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala @@ -17,13 +17,14 @@ package org.apache.spark.sql.hive.thriftserver.ui -import org.apache.spark.sql.hive.thriftserver.{HiveThriftServer2, SparkSQLEnv} +import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.hive.thriftserver.HiveThriftServer2 import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab._ import org.apache.spark.ui.{SparkUI, SparkUITab} -import org.apache.spark.{SparkContext, Logging, SparkException} /** - * Spark Web UI tab that shows statistics of a streaming job. + * Spark Web UI tab that shows statistics of jobs running in the thrift server. * This assumes the given SparkContext has enabled its SparkUI. */ private[thriftserver] class ThriftServerTab(sparkContext: SparkContext) diff --git a/sql/hive-thriftserver/src/test/resources/TestUDTF.jar b/sql/hive-thriftserver/src/test/resources/TestUDTF.jar new file mode 100644 index 000000000000..514f2d5d26fd Binary files /dev/null and b/sql/hive-thriftserver/src/test/resources/TestUDTF.jar differ diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala new file mode 100644 index 000000000000..3f135cc86498 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreLazyInitializationSuite.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.util.Utils + +class HiveMetastoreLazyInitializationSuite extends SparkFunSuite { + + test("lazily initialize Hive client") { + val spark = SparkSession.builder() + .appName("HiveMetastoreLazyInitializationSuite") + .master("local[2]") + .enableHiveSupport() + .config("spark.hadoop.hive.metastore.uris", "thrift://127.0.0.1:11111") + .getOrCreate() + val originalLevel = org.apache.log4j.Logger.getRootLogger().getLevel + try { + // Avoid outputting a lot of expected warning logs + spark.sparkContext.setLogLevel("error") + + // We should be able to run Spark jobs without Hive client. + assert(spark.sparkContext.range(0, 1).count() === 1) + + // Make sure that we are not using the local derby metastore. + val exceptionString = Utils.exceptionString(intercept[AnalysisException] { + spark.sql("show tables") + }) + for (msg <- Seq( + "show tables", + "Could not connect to meta store", + "org.apache.thrift.transport.TTransportException", + "Connection refused")) { + exceptionString.contains(msg) + } + } finally { + spark.sparkContext.setLogLevel(originalLevel.toString) + spark.stop() + } + } +} 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 3fa5c8528b60..933fd7369380 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 @@ -18,51 +18,58 @@ package org.apache.spark.sql.hive.thriftserver import java.io._ +import java.nio.charset.StandardCharsets import java.sql.Timestamp import java.util.Date import scala.collection.mutable.ArrayBuffer +import scala.concurrent.Promise import scala.concurrent.duration._ -import scala.concurrent.{Await, Promise} -import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.scalatest.BeforeAndAfter +import org.scalatest.BeforeAndAfterAll -import org.apache.spark.util.Utils -import org.apache.spark.{Logging, SparkFunSuite} +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging +import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer +import org.apache.spark.util.{ThreadUtils, Utils} /** * A test suite for the `spark-sql` CLI tool. Note that all test cases share the same temporary * Hive metastore and warehouse. */ -class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging { +class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { val warehousePath = Utils.createTempDir() val metastorePath = Utils.createTempDir() val scratchDirPath = Utils.createTempDir() - before { + override def beforeAll(): Unit = { + super.beforeAll() warehousePath.delete() metastorePath.delete() scratchDirPath.delete() } - after { - warehousePath.delete() - metastorePath.delete() - scratchDirPath.delete() + override def afterAll(): Unit = { + try { + warehousePath.delete() + metastorePath.delete() + scratchDirPath.delete() + } finally { + super.afterAll() + } } /** * Run a CLI operation and expect all the queries and expected answers to be returned. + * * @param timeout maximum time for the commands to complete * @param extraArgs any extra arguments * @param errorResponses a sequence of strings whose presence in the stdout of the forked process * is taken as an immediate error condition. That is: if a line containing * with one of these strings is found, fail the test immediately. * The default value is `Seq("Error:")` - * - * @param queriesAndExpectedAnswers one or more tupes of query + answer + * @param queriesAndExpectedAnswers one or more tuples of query + answer */ def runCliWithin( timeout: FiniteDuration, @@ -79,9 +86,13 @@ class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging { val jdbcUrl = s"jdbc:derby:;databaseName=$metastorePath;create=true" s"""$cliScript | --master local + | --driver-java-options -Dderby.system.durability=test + | --conf spark.ui.enabled=false | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$jdbcUrl | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath | --hiveconf ${ConfVars.SCRATCHDIR}=$scratchDirPath + | --hiveconf conf1=conftest + | --hiveconf conf2=1 """.stripMargin.split("\\s+").toSeq ++ extraArgs } @@ -114,7 +125,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging { val process = new ProcessBuilder(command: _*).start() - val stdinWriter = new OutputStreamWriter(process.getOutputStream) + val stdinWriter = new OutputStreamWriter(process.getOutputStream, StandardCharsets.UTF_8) stdinWriter.write(queriesString) stdinWriter.flush() stdinWriter.close() @@ -123,7 +134,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging { new ProcessOutputCapturer(process.getErrorStream, captureOutput("stderr")).start() try { - Await.result(foundAllExpectedAnswers.future, timeout) + ThreadUtils.awaitResult(foundAllExpectedAnswers.future, timeout) } catch { case cause: Throwable => val message = s""" @@ -153,41 +164,38 @@ class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging { runCliWithin(3.minute)( "CREATE TABLE hive_test(key INT, val STRING);" - -> "OK", + -> "", "SHOW TABLES;" -> "hive_test", s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE hive_test;" - -> "OK", + -> "", "CACHE TABLE hive_test;" -> "", "SELECT COUNT(*) FROM hive_test;" -> "5", "DROP TABLE hive_test;" - -> "OK" + -> "" ) } test("Single command with -e") { - runCliWithin(2.minute, Seq("-e", "SHOW DATABASES;"))("" -> "OK") + runCliWithin(2.minute, Seq("-e", "SHOW DATABASES;"))("" -> "") } test("Single command with --database") { runCliWithin(2.minute)( "CREATE DATABASE hive_test_db;" - -> "OK", + -> "", "USE hive_test_db;" - -> "OK", + -> "", "CREATE TABLE hive_test(key INT, val STRING);" - -> "OK", + -> "", "SHOW TABLES;" -> "hive_test" ) runCliWithin(2.minute, Seq("--database", "hive_test_db", "-e", "SHOW TABLES;"))( - "" - -> "OK", - "" - -> "hive_test" + "" -> "hive_test" ) } @@ -204,19 +212,19 @@ class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging { """CREATE TABLE t1(key string, val string) |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'; """.stripMargin - -> "OK", + -> "", "CREATE TABLE sourceTable (key INT, val STRING);" - -> "OK", + -> "", s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE sourceTable;" - -> "OK", + -> "", "INSERT INTO TABLE t1 SELECT key, val FROM sourceTable;" -> "", "SELECT count(key) FROM t1;" -> "5", "DROP TABLE t1;" - -> "OK", + -> "", "DROP TABLE sourceTable;" - -> "OK" + -> "" ) } @@ -224,7 +232,68 @@ class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging { runCliWithin(timeout = 2.minute, errorResponses = Seq("AnalysisException"))( "select * from nonexistent_table;" - -> "Error in query: Table not found: nonexistent_table;" + -> "Error in query: Table or view not found: nonexistent_table;" + ) + } + + test("SPARK-11624 Spark SQL CLI should set sessionState only once") { + runCliWithin(2.minute, Seq("-e", "!echo \"This is a test for Spark-11624\";"))( + "" -> "This is a test for Spark-11624") + } + + test("list jars") { + val jarFile = Thread.currentThread().getContextClassLoader.getResource("TestUDTF.jar") + runCliWithin(2.minute)( + s"ADD JAR $jarFile;" -> "", + s"LIST JARS;" -> "TestUDTF.jar" + ) + } + + test("list jar ") { + val jarFile = Thread.currentThread().getContextClassLoader.getResource("TestUDTF.jar") + runCliWithin(2.minute)( + s"ADD JAR $jarFile;" -> "", + s"List JAR $jarFile;" -> "TestUDTF.jar" + ) + } + + test("list files") { + val dataFilePath = Thread.currentThread(). + getContextClassLoader.getResource("data/files/small_kv.txt") + runCliWithin(2.minute)( + s"ADD FILE $dataFilePath;" -> "", + s"LIST FILES;" -> "small_kv.txt" ) } + + test("list file ") { + val dataFilePath = Thread.currentThread(). + getContextClassLoader.getResource("data/files/small_kv.txt") + runCliWithin(2.minute)( + s"ADD FILE $dataFilePath;" -> "", + s"LIST FILE $dataFilePath;" -> "small_kv.txt" + ) + } + + test("apply hiveconf from cli command") { + runCliWithin(2.minute)( + "SET conf1;" -> "conftest", + "SET conf2;" -> "1", + "SET conf3=${hiveconf:conf1};" -> "conftest", + "SET conf3;" -> "conftest" + ) + } + + test("SPARK-21451: spark.sql.warehouse.dir should respect options in --hiveconf") { + runCliWithin(1.minute)("set spark.sql.warehouse.dir;" -> warehousePath.getAbsolutePath) + } + + test("SPARK-21451: Apply spark.hadoop.* configurations") { + val tmpDir = Utils.createTempDir(namePrefix = "SPARK-21451") + runCliWithin( + 1.minute, + Seq(s"--conf", s"spark.hadoop.${ConfVars.METASTOREWAREHOUSE}=$tmpDir"))( + "set spark.sql.warehouse.dir;" -> tmpDir.getAbsolutePath) + tmpDir.delete() + } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala new file mode 100644 index 000000000000..5f9ea4d26790 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import org.apache.hadoop.hive.cli.CliSessionState +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.session.SessionState + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.sql.hive.HiveUtils + +class HiveCliSessionStateSuite extends SparkFunSuite { + + def withSessionClear(f: () => Unit): Unit = { + try f finally SessionState.detachSession() + } + + test("CliSessionState will be reused") { + withSessionClear { () => + val hiveConf = new HiveConf(classOf[SessionState]) + HiveUtils.newTemporaryConfiguration(useInMemoryDerby = false).foreach { + case (key, value) => hiveConf.set(key, value) + } + val sessionState: SessionState = new CliSessionState(hiveConf) + SessionState.start(sessionState) + val s1 = SessionState.get + val sparkConf = new SparkConf() + val hadoopConf = SparkHadoopUtil.get.newConfiguration(sparkConf) + val s2 = HiveUtils.newClientForMetadata(sparkConf, hadoopConf).getState + assert(s1 === s2) + assert(s2.isInstanceOf[CliSessionState]) + } + } + + test("SessionState will not be reused") { + withSessionClear { () => + val sparkConf = new SparkConf() + val hadoopConf = SparkHadoopUtil.get.newConfiguration(sparkConf) + HiveUtils.newTemporaryConfiguration(useInMemoryDerby = false).foreach { + case (key, value) => hadoopConf.set(key, value) + } + val hiveClient = HiveUtils.newClientForMetadata(sparkConf, hadoopConf) + val s1 = hiveClient.getState + val s2 = hiveClient.newSession().getState + assert(s1 !== s2) + } + } +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index ff8ca0150649..4997d7f96afa 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -19,31 +19,32 @@ package org.apache.spark.sql.hive.thriftserver import java.io.File import java.net.URL +import java.nio.charset.StandardCharsets import java.sql.{Date, DriverManager, SQLException, Statement} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.{ExecutionContext, Future, Promise} import scala.concurrent.duration._ -import scala.concurrent.{Await, Promise, future} +import scala.io.Source import scala.util.{Random, Try} -import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.jdbc.HiveDriver import org.apache.hive.service.auth.PlainSaslHelper -import org.apache.hive.service.cli.GetInfoType +import org.apache.hive.service.cli.{FetchOrientation, FetchType, GetInfoType} import org.apache.hive.service.cli.thrift.TCLIService.Client import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient import org.apache.thrift.protocol.TBinaryProtocol import org.apache.thrift.transport.TSocket import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer -import org.apache.spark.util.Utils -import org.apache.spark.{Logging, SparkFunSuite} +import org.apache.spark.util.{ThreadUtils, Utils} object TestData { def getTestDataFilePath(name: String): URL = { @@ -90,11 +91,54 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } } + test("SPARK-16563 ThriftCLIService FetchResults repeat fetching result") { + withCLIServiceClient { client => + val user = System.getProperty("user.name") + val sessionHandle = client.openSession(user, "") + + withJdbcStatement("test_16563") { statement => + val queries = Seq( + "CREATE TABLE test_16563(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_16563") + + queries.foreach(statement.execute) + val confOverlay = new java.util.HashMap[java.lang.String, java.lang.String] + val operationHandle = client.executeStatement( + sessionHandle, + "SELECT * FROM test_16563", + confOverlay) + + // Fetch result first time + assertResult(5, "Fetching result first time from next row") { + + val rows_next = client.fetchResults( + operationHandle, + FetchOrientation.FETCH_NEXT, + 1000, + FetchType.QUERY_OUTPUT) + + rows_next.numRows() + } + + // Fetch result second time from first row + assertResult(5, "Repeat fetching result from first row") { + + val rows_first = client.fetchResults( + operationHandle, + FetchOrientation.FETCH_FIRST, + 1000, + FetchType.QUERY_OUTPUT) + + rows_first.numRows() + } + } + } + } + test("JDBC query execution") { - withJdbcStatement { statement => + withJdbcStatement("test") { statement => val queries = Seq( "SET spark.sql.shuffle.partitions=3", - "DROP TABLE IF EXISTS test", "CREATE TABLE test(key INT, val STRING)", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", "CACHE TABLE test") @@ -110,18 +154,17 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } test("Checks Hive version") { - withJdbcStatement { statement => + withJdbcStatement() { statement => val resultSet = statement.executeQuery("SET spark.sql.hive.version") resultSet.next() assert(resultSet.getString(1) === "spark.sql.hive.version") - assert(resultSet.getString(2) === HiveContext.hiveExecutionVersion) + assert(resultSet.getString(2) === HiveUtils.hiveExecutionVersion) } } test("SPARK-3004 regression: result set containing NULL") { - withJdbcStatement { statement => + withJdbcStatement("test_null") { statement => val queries = Seq( - "DROP TABLE IF EXISTS test_null", "CREATE TABLE test_null(key INT, val STRING)", s"LOAD DATA LOCAL INPATH '${TestData.smallKvWithNull}' OVERWRITE INTO TABLE test_null") @@ -140,9 +183,8 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } test("SPARK-4292 regression: result set iterator issue") { - withJdbcStatement { statement => + withJdbcStatement("test_4292") { statement => val queries = Seq( - "DROP TABLE IF EXISTS test_4292", "CREATE TABLE test_4292(key INT, val STRING)", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_4292") @@ -154,15 +196,12 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { resultSet.next() assert(resultSet.getInt(1) === key) } - - statement.executeQuery("DROP TABLE IF EXISTS test_4292") } } test("SPARK-4309 regression: Date type support") { - withJdbcStatement { statement => + withJdbcStatement("test_date") { statement => val queries = Seq( - "DROP TABLE IF EXISTS test_date", "CREATE TABLE test_date(key INT, value STRING)", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_date") @@ -178,9 +217,8 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } test("SPARK-4407 regression: Complex type support") { - withJdbcStatement { statement => + withJdbcStatement("test_map") { statement => val queries = Seq( - "DROP TABLE IF EXISTS test_map", "CREATE TABLE test_map(key INT, value STRING)", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_map") @@ -201,18 +239,35 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } } + test("SPARK-12143 regression: Binary type support") { + withJdbcStatement("test_binary") { statement => + val queries = Seq( + "CREATE TABLE test_binary(key INT, value STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_binary") + + queries.foreach(statement.execute) + + val expected: Array[Byte] = "val_238".getBytes + assertResult(expected) { + val resultSet = statement.executeQuery( + "SELECT CAST(value as BINARY) FROM test_binary LIMIT 1") + resultSet.next() + resultSet.getObject(1) + } + } + } + test("test multiple session") { - import org.apache.spark.sql.SQLConf + import org.apache.spark.sql.internal.SQLConf var defaultV1: String = null var defaultV2: String = null var data: ArrayBuffer[Int] = null - withMultipleConnectionJdbcStatement( + withMultipleConnectionJdbcStatement("test_map")( // create table { statement => val queries = Seq( - "DROP TABLE IF EXISTS test_map", "CREATE TABLE test_map(key INT, value STRING)", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_map", "CACHE TABLE test_table AS SELECT key FROM test_map ORDER BY key DESC", @@ -223,7 +278,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { val plan = statement.executeQuery("explain select * from test_table") plan.next() plan.next() - assert(plan.getString(1).contains("InMemoryColumnarTableScan")) + assert(plan.getString(1).contains("InMemoryTableScan")) val rs1 = statement.executeQuery("SELECT key FROM test_table ORDER BY KEY DESC") val buf1 = new collection.mutable.ArrayBuffer[Int]() @@ -309,7 +364,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { val plan = statement.executeQuery("explain select key from test_map ORDER BY key DESC") plan.next() plan.next() - assert(plan.getString(1).contains("InMemoryColumnarTableScan")) + assert(plan.getString(1).contains("InMemoryTableScan")) val rs = statement.executeQuery("SELECT key FROM test_map ORDER BY KEY DESC") val buf = new collection.mutable.ArrayBuffer[Int]() @@ -347,44 +402,69 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { ) } - test("test jdbc cancel") { - withJdbcStatement { statement => + // This test often hangs and then times out, leaving the hanging processes. + // Let's ignore it and improve the test. + ignore("test jdbc cancel") { + withJdbcStatement("test_map") { statement => val queries = Seq( - "DROP TABLE IF EXISTS test_map", "CREATE TABLE test_map(key INT, value STRING)", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_map") queries.foreach(statement.execute) - - val largeJoin = "SELECT COUNT(*) FROM test_map " + - List.fill(10)("join test_map").mkString(" ") - val f = future { Thread.sleep(100); statement.cancel(); } - val e = intercept[SQLException] { - statement.executeQuery(largeJoin) + implicit val ec = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonSingleThreadExecutor("test-jdbc-cancel")) + try { + // Start a very-long-running query that will take hours to finish, then cancel it in order + // to demonstrate that cancellation works. + val f = Future { + statement.executeQuery( + "SELECT COUNT(*) FROM test_map " + + List.fill(10)("join test_map").mkString(" ")) + } + // Note that this is slightly race-prone: if the cancel is issued before the statement + // begins executing then we'll fail with a timeout. As a result, this fixed delay is set + // slightly more conservatively than may be strictly necessary. + Thread.sleep(1000) + statement.cancel() + val e = intercept[SparkException] { + ThreadUtils.awaitResult(f, 3.minute) + }.getCause + assert(e.isInstanceOf[SQLException]) + assert(e.getMessage.contains("cancelled")) + + // Cancellation is a no-op if spark.sql.hive.thriftServer.async=false + statement.executeQuery("SET spark.sql.hive.thriftServer.async=false") + try { + val sf = Future { + statement.executeQuery( + "SELECT COUNT(*) FROM test_map " + + List.fill(4)("join test_map").mkString(" ") + ) + } + // Similarly, this is also slightly race-prone on fast machines where the query above + // might race and complete before we issue the cancel. + Thread.sleep(1000) + statement.cancel() + val rs1 = ThreadUtils.awaitResult(sf, 3.minute) + rs1.next() + assert(rs1.getInt(1) === math.pow(5, 5)) + rs1.close() + + val rs2 = statement.executeQuery("SELECT COUNT(*) FROM test_map") + rs2.next() + assert(rs2.getInt(1) === 5) + rs2.close() + } finally { + statement.executeQuery("SET spark.sql.hive.thriftServer.async=true") + } + } finally { + ec.shutdownNow() } - assert(e.getMessage contains "cancelled") - Await.result(f, 3.minute) - - // cancel is a noop - statement.executeQuery("SET spark.sql.hive.thriftServer.async=false") - val sf = future { Thread.sleep(100); statement.cancel(); } - val smallJoin = "SELECT COUNT(*) FROM test_map " + - List.fill(4)("join test_map").mkString(" ") - val rs1 = statement.executeQuery(smallJoin) - Await.result(sf, 3.minute) - rs1.next() - assert(rs1.getInt(1) === math.pow(5, 5)) - rs1.close() - - val rs2 = statement.executeQuery("SELECT COUNT(*) FROM test_map") - rs2.next() - assert(rs2.getInt(1) === 5) - rs2.close() } } test("test add jar") { - withMultipleConnectionJdbcStatement( + withMultipleConnectionJdbcStatement("smallKV", "addJar")( { statement => val jarFile = @@ -398,10 +478,8 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { { statement => val queries = Seq( - "DROP TABLE IF EXISTS smallKV", "CREATE TABLE smallKV(key INT, val STRING)", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE smallKV", - "DROP TABLE IF EXISTS addJar", """CREATE TABLE addJar(key string) |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' """.stripMargin) @@ -430,15 +508,12 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { expectedResult.close() assert(expectedResultBuffer === actualResultBuffer) - - statement.executeQuery("DROP TABLE IF EXISTS addJar") - statement.executeQuery("DROP TABLE IF EXISTS smallKV") } ) } test("Checks Hive version via SET -v") { - withJdbcStatement { statement => + withJdbcStatement() { statement => val resultSet = statement.executeQuery("SET -v") val conf = mutable.Map.empty[String, String] @@ -451,7 +526,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } test("Checks Hive version via SET") { - withJdbcStatement { statement => + withJdbcStatement() { statement => val resultSet = statement.executeQuery("SET") val conf = mutable.Map.empty[String, String] @@ -462,16 +537,174 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { assert(conf.get("spark.sql.hive.version") === Some("1.2.1")) } } + + test("SPARK-11595 ADD JAR with input path having URL scheme") { + withJdbcStatement("test_udtf") { statement => + try { + val jarPath = "../hive/src/test/resources/TestUDTF.jar" + val jarURL = s"file://${System.getProperty("user.dir")}/$jarPath" + + Seq( + s"ADD JAR $jarURL", + s"""CREATE TEMPORARY FUNCTION udtf_count2 + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + """.stripMargin + ).foreach(statement.execute) + + val rs1 = statement.executeQuery("DESCRIBE FUNCTION udtf_count2") + + assert(rs1.next()) + assert(rs1.getString(1) === "Function: udtf_count2") + + assert(rs1.next()) + assertResult("Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2") { + rs1.getString(1) + } + + assert(rs1.next()) + assert(rs1.getString(1) === "Usage: N/A.") + + val dataPath = "../hive/src/test/resources/data/files/kv1.txt" + + Seq( + "CREATE TABLE test_udtf(key INT, value STRING)", + s"LOAD DATA LOCAL INPATH '$dataPath' OVERWRITE INTO TABLE test_udtf" + ).foreach(statement.execute) + + val rs2 = statement.executeQuery( + "SELECT key, cc FROM test_udtf LATERAL VIEW udtf_count2(value) dd AS cc") + + assert(rs2.next()) + assert(rs2.getInt(1) === 97) + assert(rs2.getInt(2) === 500) + + assert(rs2.next()) + assert(rs2.getInt(1) === 97) + assert(rs2.getInt(2) === 500) + } finally { + statement.executeQuery("DROP TEMPORARY FUNCTION udtf_count2") + } + } + } + + test("SPARK-11043 check operation log root directory") { + val expectedLine = + "Operation log root directory is created: " + operationLogPath.getAbsoluteFile + val bufferSrc = Source.fromFile(logPath) + Utils.tryWithSafeFinally { + assert(bufferSrc.getLines().exists(_.contains(expectedLine))) + } { + bufferSrc.close() + } + } +} + +class SingleSessionSuite extends HiveThriftJdbcTest { + override def mode: ServerMode.Value = ServerMode.binary + + override protected def extraConf: Seq[String] = + "--conf spark.sql.hive.thriftServer.singleSession=true" :: Nil + + test("share the temporary functions across JDBC connections") { + withMultipleConnectionJdbcStatement()( + { statement => + val jarPath = "../hive/src/test/resources/TestUDTF.jar" + val jarURL = s"file://${System.getProperty("user.dir")}/$jarPath" + + // Configurations and temporary functions added in this session should be visible to all + // the other sessions. + Seq( + "SET foo=bar", + s"ADD JAR $jarURL", + s"""CREATE TEMPORARY FUNCTION udtf_count2 + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + """.stripMargin + ).foreach(statement.execute) + }, + + { statement => + try { + val rs1 = statement.executeQuery("SET foo") + + assert(rs1.next()) + assert(rs1.getString(1) === "foo") + assert(rs1.getString(2) === "bar") + + val rs2 = statement.executeQuery("DESCRIBE FUNCTION udtf_count2") + + assert(rs2.next()) + assert(rs2.getString(1) === "Function: udtf_count2") + + assert(rs2.next()) + assertResult("Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2") { + rs2.getString(1) + } + + assert(rs2.next()) + assert(rs2.getString(1) === "Usage: N/A.") + } finally { + statement.executeQuery("DROP TEMPORARY FUNCTION udtf_count2") + } + } + ) + } + + test("unable to changing spark.sql.hive.thriftServer.singleSession using JDBC connections") { + withJdbcStatement() { statement => + // JDBC connections are not able to set the conf spark.sql.hive.thriftServer.singleSession + val e = intercept[SQLException] { + statement.executeQuery("SET spark.sql.hive.thriftServer.singleSession=false") + }.getMessage + assert(e.contains( + "Cannot modify the value of a static config: spark.sql.hive.thriftServer.singleSession")) + } + } + + test("share the current database and temporary tables across JDBC connections") { + withMultipleConnectionJdbcStatement()( + { statement => + statement.execute("CREATE DATABASE IF NOT EXISTS db1") + }, + + { statement => + val rs1 = statement.executeQuery("SELECT current_database()") + assert(rs1.next()) + assert(rs1.getString(1) === "default") + + statement.execute("USE db1") + + val rs2 = statement.executeQuery("SELECT current_database()") + assert(rs2.next()) + assert(rs2.getString(1) === "db1") + + statement.execute("CREATE TEMP VIEW tempView AS SELECT 123") + }, + + { statement => + // the current database is set to db1 by another JDBC connection. + val rs1 = statement.executeQuery("SELECT current_database()") + assert(rs1.next()) + assert(rs1.getString(1) === "db1") + + val rs2 = statement.executeQuery("SELECT * from tempView") + assert(rs2.next()) + assert(rs2.getString(1) === "123") + + statement.execute("USE default") + statement.execute("DROP VIEW tempView") + statement.execute("DROP DATABASE db1 CASCADE") + } + ) + } } class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { override def mode: ServerMode.Value = ServerMode.http test("JDBC query execution") { - withJdbcStatement { statement => + withJdbcStatement("test") { statement => val queries = Seq( "SET spark.sql.shuffle.partitions=3", - "DROP TABLE IF EXISTS test", "CREATE TABLE test(key INT, val STRING)", s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", "CACHE TABLE test") @@ -487,11 +720,11 @@ class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { } test("Checks Hive version") { - withJdbcStatement { statement => + withJdbcStatement() { statement => val resultSet = statement.executeQuery("SET spark.sql.hive.version") resultSet.next() assert(resultSet.getString(1) === "spark.sql.hive.version") - assert(resultSet.getString(2) === HiveContext.hiveExecutionVersion) + assert(resultSet.getString(2) === HiveUtils.hiveExecutionVersion) } } } @@ -513,7 +746,7 @@ abstract class HiveThriftJdbcTest extends HiveThriftServer2Test { s"jdbc:hive2://localhost:$serverPort/" } - def withMultipleConnectionJdbcStatement(fs: (Statement => Unit)*) { + def withMultipleConnectionJdbcStatement(tableNames: String*)(fs: (Statement => Unit)*) { val user = System.getProperty("user.name") val connections = fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } val statements = connections.map(_.createStatement()) @@ -521,13 +754,16 @@ abstract class HiveThriftJdbcTest extends HiveThriftServer2Test { try { statements.zip(fs).foreach { case (s, f) => f(s) } } finally { + tableNames.foreach { name => + statements(0).execute(s"DROP TABLE IF EXISTS $name") + } statements.foreach(_.close()) connections.foreach(_.close()) } } - def withJdbcStatement(f: Statement => Unit) { - withMultipleConnectionJdbcStatement(f) + def withJdbcStatement(tableNames: String*)(f: Statement => Unit) { + withMultipleConnectionJdbcStatement(tableNames: _*)(f) } } @@ -550,10 +786,13 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl protected def metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" private val pidDir: File = Utils.createTempDir("thriftserver-pid") - private var logPath: File = _ + protected var logPath: File = _ + protected var operationLogPath: File = _ private var logTailingProcess: Process = _ private var diagnosisBuffer: ArrayBuffer[String] = ArrayBuffer.empty[String] + protected def extraConf: Seq[String] = Nil + protected def serverStartCommand(port: Int) = { val portConf = if (mode == ServerMode.binary) { ConfVars.HIVE_SERVER2_THRIFT_PORT @@ -574,7 +813,7 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n """.stripMargin, new File(s"$tempLog4jConf/log4j.properties"), - UTF_8) + StandardCharsets.UTF_8) tempLog4jConf } @@ -585,21 +824,23 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=localhost | --hiveconf ${ConfVars.HIVE_SERVER2_TRANSPORT_MODE}=$mode + | --hiveconf ${ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LOG_LOCATION}=$operationLogPath | --hiveconf $portConf=$port | --driver-class-path $driverClassPath | --driver-java-options -Dlog4j.debug | --conf spark.ui.enabled=false + | ${extraConf.mkString("\n")} """.stripMargin.split("\\s+").toSeq } /** - * String to scan for when looking for the the thrift binary endpoint running. + * String to scan for when looking for the thrift binary endpoint running. * This can change across Hive versions. */ val THRIFT_BINARY_SERVICE_LIVE = "Starting ThriftBinaryCLIService on port" /** - * String to scan for when looking for the the thrift HTTP endpoint running. + * String to scan for when looking for the thrift HTTP endpoint running. * This can change across Hive versions. */ val THRIFT_HTTP_SERVICE_LIVE = "Started ThriftHttpCLIService in http" @@ -611,6 +852,8 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl warehousePath.delete() metastorePath = Utils.createTempDir() metastorePath.delete() + operationLogPath = Utils.createTempDir() + operationLogPath.delete() logPath = null logTailingProcess = null @@ -632,11 +875,15 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl extraEnvironment = Map( // Disables SPARK_TESTING to exclude log4j.properties in test directories. "SPARK_TESTING" -> "0", + // But set SPARK_SQL_TESTING to make spark-class happy. + "SPARK_SQL_TESTING" -> "1", // Points SPARK_PID_DIR to SPARK_HOME, otherwise only 1 Thrift server instance can be // started at a time, which is not Jenkins friendly. "SPARK_PID_DIR" -> pidDir.getCanonicalPath), redirectStderr = true) + logInfo(s"COMMAND: $command") + logInfo(s"OUTPUT: $lines") lines.split("\n").collectFirst { case line if line.contains(LOG_FILE_MARK) => new File(line.drop(LOG_FILE_MARK.length)) }.getOrElse { @@ -671,7 +918,7 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl process } - Await.result(serverStarted.future, SERVER_STARTUP_TIMEOUT) + ThreadUtils.awaitResult(serverStarted.future, SERVER_STARTUP_TIMEOUT) } private def stopThriftServer(): Unit = { @@ -687,6 +934,9 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl metastorePath.delete() metastorePath = null + operationLogPath.delete() + operationLogPath = null + Option(logPath).foreach(_.delete()) logPath = null @@ -708,6 +958,7 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl } override protected def beforeAll(): Unit = { + super.beforeAll() // Chooses a random port between 10000 and 19999 listeningPort = 10000 + Random.nextInt(10000) diagnosisBuffer.clear() @@ -729,7 +980,11 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl } override protected def afterAll(): Unit = { - stopThriftServer() - logInfo("HiveThriftServer2 stopped") + try { + stopThriftServer() + logInfo("HiveThriftServer2 stopped") + } finally { + super.afterAll() + } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala new file mode 100644 index 000000000000..fb8a7e273ae4 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/JdbcConnectionUriSuite.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.sql.DriverManager + +import org.apache.hive.jdbc.HiveDriver + +import org.apache.spark.util.Utils + +class JdbcConnectionUriSuite extends HiveThriftServer2Test { + Utils.classForName(classOf[HiveDriver].getCanonicalName) + + override def mode: ServerMode.Value = ServerMode.binary + + val JDBC_TEST_DATABASE = "jdbc_test_database" + val USER = System.getProperty("user.name") + val PASSWORD = "" + + override protected def beforeAll(): Unit = { + super.beforeAll() + + val jdbcUri = s"jdbc:hive2://localhost:$serverPort/" + val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD) + val statement = connection.createStatement() + statement.execute(s"CREATE DATABASE $JDBC_TEST_DATABASE") + connection.close() + } + + override protected def afterAll(): Unit = { + try { + val jdbcUri = s"jdbc:hive2://localhost:$serverPort/" + val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD) + val statement = connection.createStatement() + statement.execute(s"DROP DATABASE $JDBC_TEST_DATABASE") + connection.close() + } finally { + super.afterAll() + } + } + + test("SPARK-17819 Support default database in connection URIs") { + val jdbcUri = s"jdbc:hive2://localhost:$serverPort/$JDBC_TEST_DATABASE" + val connection = DriverManager.getConnection(jdbcUri, USER, PASSWORD) + val statement = connection.createStatement() + try { + val resultSet = statement.executeQuery("select current_database()") + resultSet.next() + assert(resultSet.getString(1) === JDBC_TEST_DATABASE) + } finally { + statement.close() + connection.close() + } + } +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala new file mode 100644 index 000000000000..06e398066204 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala @@ -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.hive.thriftserver + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types.{IntegerType, NullType, StringType, StructField, StructType} + +class SparkExecuteStatementOperationSuite extends SparkFunSuite { + test("SPARK-17112 `select null` via JDBC triggers IllegalArgumentException in ThriftServer") { + val field1 = StructField("NULL", NullType) + val field2 = StructField("(IF(true, NULL, NULL))", NullType) + val tableSchema = StructType(Seq(field1, field2)) + val columns = SparkExecuteStatementOperation.getTableSchema(tableSchema).getColumnDescriptors() + assert(columns.size() == 2) + assert(columns.get(0).getType() == org.apache.hive.service.cli.Type.NULL_TYPE) + assert(columns.get(1).getType() == org.apache.hive.service.cli.Type.NULL_TYPE) + } + + test("SPARK-20146 Comment should be preserved") { + val field1 = StructField("column1", StringType).withComment("comment 1") + val field2 = StructField("column2", IntegerType) + val tableSchema = StructType(Seq(field1, field2)) + val columns = SparkExecuteStatementOperation.getTableSchema(tableSchema).getColumnDescriptors() + assert(columns.size() == 2) + assert(columns.get(0).getType() == org.apache.hive.service.cli.Type.STRING_TYPE) + assert(columns.get(0).getComment() == "comment 1") + assert(columns.get(1).getType() == org.apache.hive.service.cli.Type.INT_TYPE) + assert(columns.get(1).getComment() == "") + } +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala index bf431cd6b026..4c53dd8f4616 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala @@ -74,7 +74,7 @@ class UISeleniumSuite } ignore("thrift server ui test") { - withJdbcStatement { statement => + withJdbcStatement("test_map") { statement => val baseURL = s"http://localhost:$uiPort" val queries = Seq( 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 2d0d7b8af358..45791c69b4cb 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 @@ -20,17 +20,16 @@ package org.apache.spark.sql.hive.execution import java.io.File import java.util.{Locale, TimeZone} -import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHive -import org.apache.spark.tags.ExtendedHiveTest +import org.apache.spark.sql.internal.SQLConf /** * Runs the test cases that are included in the hive distribution. */ -@ExtendedHiveTest class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // TODO: bundle in jar files... get from classpath private lazy val hiveQueryDir = TestHive.getHiveFile( @@ -40,11 +39,16 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalLocale = Locale.getDefault private val originalColumnBatchSize = TestHive.conf.columnBatchSize private val originalInMemoryPartitionPruning = TestHive.conf.inMemoryPartitionPruning + private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled + private val originalSessionLocalTimeZone = TestHive.conf.sessionLocalTimeZone - def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) + def testCases: Seq[(String, File)] = { + hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) + } override def beforeAll() { - TestHive.cacheTables = true + super.beforeAll() + TestHive.setCacheTables(true) // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting @@ -53,22 +57,33 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, 5) // Enable in-memory partition pruning for testing purposes TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, true) + // Ensures that cross joins are enabled so that we can test them + TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, true) + // Fix session local timezone to America/Los_Angeles for those timezone sensitive tests + // (timestamp_*) + TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, "America/Los_Angeles") RuleExecutor.resetTime() } override def afterAll() { - TestHive.cacheTables = false - TimeZone.setDefault(originalTimeZone) - Locale.setDefault(originalLocale) - TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize) - TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning) - - // For debugging dump some statistics about how much time was spent in various optimizer rules. - logWarning(RuleExecutor.dumpTimeSpent()) + try { + TestHive.setCacheTables(false) + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize) + TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning) + TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) + TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, originalSessionLocalTimeZone) + + // For debugging dump some statistics about how much time was spent in various optimizer rules + logWarning(RuleExecutor.dumpTimeSpent()) + } finally { + super.afterAll() + } } /** A list of tests deemed out of scope currently and thus completely disregarded. */ - override def blackList = Seq( + override def blackList: Seq[String] = Seq( // These tests use hooks that are not on the classpath and thus break all subsequent execution. "hook_order", "hook_context_cs", @@ -103,7 +118,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "alter_merge", "alter_concatenate_indexed_table", "protectmode2", - //"describe_table", + // "describe_table", "describe_comment_nonascii", "create_merge_compressed", @@ -163,7 +178,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "skewjoin", "database", - // These tests fail and and exit the JVM. + // These tests fail and exit the JVM. "auto_join18_multi_distinct", "join18_multi_distinct", "input44", @@ -283,12 +298,14 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "compute_stats_empty_table", "compute_stats_long", "create_view_translate", - "show_create_table_serde", "show_tblproperties", // Odd changes to output "merge4", + // Unsupported underscore syntax. + "inputddl5", + // Thift is broken... "inputddl8", @@ -308,39 +325,237 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // The difference between the double numbers generated by Hive and Spark // can be ignored (e.g., 0.6633880657639323 and 0.6633880657639322) - "udaf_corr" - ) + "udaf_corr", - /** - * The set of tests that are believed to be working in catalyst. Tests not on whiteList or - * blacklist are implicitly marked as ignored. - */ - override def whiteList = Seq( - "add_part_exist", - "add_part_multiple", - "add_partition_no_whitelist", - "add_partition_with_whitelist", - "alias_casted_column", - "alter2", - "alter3", + // Feature removed in HIVE-11145 + "alter_partition_protect_mode", + "drop_partitions_ignore_protection", + "protectmode", + + // Hive returns null rather than NaN when n = 1 + "udaf_covar_samp", + + // The implementation of GROUPING__ID in Hive is wrong (not match with doc). + "groupby_grouping_id1", + "groupby_grouping_id2", + "groupby_grouping_sets1", + + // Spark parser treats numerical literals differently: it creates decimals instead of doubles. + "udf_abs", + "udf_format_number", + "udf_round", + "udf_round_3", + "view_cast", + + // These tests check the VIEW table definition, but Spark handles CREATE VIEW itself and + // generates different View Expanded Text. + "alter_view_as_select", + + // We don't support show create table commands in general + "show_create_table_alter", + "show_create_table_db_table", + "show_create_table_delimited", + "show_create_table_does_not_exist", + "show_create_table_index", + "show_create_table_partitioned", + "show_create_table_serde", + "show_create_table_view", + + // These tests try to change how a table is bucketed, which we don't support "alter4", - "alter5", + "sort_merge_join_desc_5", + "sort_merge_join_desc_6", + "sort_merge_join_desc_7", + + // These tests try to create a table with bucketed columns, which we don't support + "auto_join32", + "auto_join_filters", + "auto_smb_mapjoin_14", + "ct_case_insensitive", + "explain_rearrange", + "groupby_sort_10", + "groupby_sort_2", + "groupby_sort_3", + "groupby_sort_4", + "groupby_sort_5", + "groupby_sort_7", + "groupby_sort_8", + "groupby_sort_9", + "groupby_sort_test_1", + "inputddl4", + "join_filters", + "join_nulls", + "join_nullsafe", + "load_dyn_part2", + "orc_empty_files", + "reduce_deduplicate", + "smb_mapjoin9", + "smb_mapjoin_1", + "smb_mapjoin_10", + "smb_mapjoin_13", + "smb_mapjoin_14", + "smb_mapjoin_15", + "smb_mapjoin_16", + "smb_mapjoin_17", + "smb_mapjoin_2", + "smb_mapjoin_21", + "smb_mapjoin_25", + "smb_mapjoin_3", + "smb_mapjoin_4", + "smb_mapjoin_5", + "smb_mapjoin_6", + "smb_mapjoin_7", + "smb_mapjoin_8", + "sort_merge_join_desc_1", + "sort_merge_join_desc_2", + "sort_merge_join_desc_3", + "sort_merge_join_desc_4", + + // These tests try to create a table with skewed columns, which we don't support + "create_skewed_table1", + "skewjoinopt13", + "skewjoinopt18", + "skewjoinopt9", + + // This test tries to create a table like with TBLPROPERTIES clause, which we don't support. + "create_like_tbl_props", + + // Index commands are not supported + "drop_index", + "drop_index_removes_partition_dirs", "alter_index", + "auto_sortmerge_join_1", + "auto_sortmerge_join_10", + "auto_sortmerge_join_11", + "auto_sortmerge_join_12", + "auto_sortmerge_join_13", + "auto_sortmerge_join_14", + "auto_sortmerge_join_15", + "auto_sortmerge_join_16", + "auto_sortmerge_join_2", + "auto_sortmerge_join_3", + "auto_sortmerge_join_4", + "auto_sortmerge_join_5", + "auto_sortmerge_join_6", + "auto_sortmerge_join_7", + "auto_sortmerge_join_8", + "auto_sortmerge_join_9", + + // Macro commands are not supported + "macro", + + // Create partitioned view is not supported + "create_like_view", + "describe_formatted_view_partitioned", + + // This uses CONCATENATE, which we don't support "alter_merge_2", + + // TOUCH is not supported + "touch", + + // INPUTDRIVER and OUTPUTDRIVER are not supported + "inoutdriver", + + // We do not support ALTER TABLE ADD COLUMN, ALTER TABLE REPLACE COLUMN, + // ALTER TABLE CHANGE COLUMN, and ALTER TABLE SET FILEFORMAT. + // We have converted the useful parts of these tests to tests + // in org.apache.spark.sql.hive.execution.SQLQuerySuite. "alter_partition_format_loc", - "alter_partition_protect_mode", - "alter_partition_with_whitelist", - "alter_rename_partition", - "alter_table_serde", "alter_varchar1", "alter_varchar2", - "alter_view_as_select", - "ambiguous_col", - "annotate_stats_join", - "annotate_stats_limit", - "annotate_stats_part", - "annotate_stats_table", - "annotate_stats_union", + "date_3", + "diff_part_input_formats", + "disallow_incompatible_type_change_off", + "fileformat_mix", + "input3", + "partition_schema1", + "partition_wise_fileformat4", + "partition_wise_fileformat5", + "partition_wise_fileformat6", + "partition_wise_fileformat7", + "rename_column", + + // The following fails due to describe extended. + "alter3", + "alter5", + "alter_table_serde", + "input_part10", + "input_part10_win", + "inputddl6", + "inputddl7", + "part_inherit_tbl_props_empty", + "serde_reported_schema", + "stats0", + "stats_empty_partition", + "unicode_notation", + "union_remove_11", + "union_remove_3", + + // The following fails due to alter table partitions with predicate. + "drop_partitions_filter", + "drop_partitions_filter2", + "drop_partitions_filter3", + + // The following failes due to truncate table + "truncate_table", + + // We do not support DFS command. + // We have converted the useful parts of these tests to tests + // in org.apache.spark.sql.hive.execution.SQLQuerySuite. + "drop_database_removes_partition_dirs", + "drop_table_removes_partition_dirs", + + // These tests use EXPLAIN FORMATTED, which is not supported + "input4", + "join0", + "plan_json", + + // This test uses CREATE EXTERNAL TABLE without specifying LOCATION + "alter2", + + // [SPARK-16248][SQL] Whitelist the list of Hive fallback functions + "udf_field", + "udf_reflect2", + "udf_xpath", + "udf_xpath_boolean", + "udf_xpath_double", + "udf_xpath_float", + "udf_xpath_int", + "udf_xpath_long", + "udf_xpath_short", + "udf_xpath_string", + + // These tests DROP TABLE that don't exist (but do not specify IF EXISTS) + "alter_rename_partition1", + "date_1", + "date_4", + "date_join1", + "date_serde", + "insert_compressed", + "lateral_view_cp", + "leftsemijoin", + "mapjoin_subquery2", + "nomore_ambiguous_table_col", + "partition_date", + "partition_varchar1", + "ppd_repeated_alias", + "push_or", + "reducesink_dedup", + "subquery_in", + "subquery_notin_having", + "timestamp_3", + "timestamp_lazy", + "udaf_covar_pop", + "union31", + "union_date", + "varchar_2", + "varchar_join1", + + // This test assumes we parse scientific decimals as doubles (we parse them as decimals) + "literal_double", + + // These tests are duplicates of joinXYZ "auto_join0", "auto_join1", "auto_join10", @@ -352,47 +567,56 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "auto_join15", "auto_join17", "auto_join18", - "auto_join19", "auto_join2", "auto_join20", "auto_join21", - "auto_join22", "auto_join23", "auto_join24", - "auto_join25", - "auto_join26", - "auto_join27", - "auto_join28", "auto_join3", - "auto_join30", - "auto_join31", - "auto_join32", "auto_join4", "auto_join5", "auto_join6", "auto_join7", "auto_join8", "auto_join9", - "auto_join_filters", + + // These tests are based on the Hive's hash function, which is different from Spark + "auto_join19", + "auto_join22", + "auto_join25", + "auto_join26", + "auto_join27", + "auto_join28", + "auto_join30", + "auto_join31", "auto_join_nulls", "auto_join_reordering_values", - "auto_smb_mapjoin_14", - "auto_sortmerge_join_1", - "auto_sortmerge_join_10", - "auto_sortmerge_join_11", - "auto_sortmerge_join_12", - "auto_sortmerge_join_13", - "auto_sortmerge_join_14", - "auto_sortmerge_join_15", - "auto_sortmerge_join_16", - "auto_sortmerge_join_2", - "auto_sortmerge_join_3", - "auto_sortmerge_join_4", - "auto_sortmerge_join_5", - "auto_sortmerge_join_6", - "auto_sortmerge_join_7", - "auto_sortmerge_join_8", - "auto_sortmerge_join_9", + "correlationoptimizer1", + "correlationoptimizer2", + "correlationoptimizer3", + "correlationoptimizer4", + "multiMapJoin1", + "orc_dictionary_threshold", + "udf_hash" + ) + + /** + * The set of tests that are believed to be working in catalyst. Tests not on whiteList or + * blacklist are implicitly marked as ignored. + */ + override def whiteList: Seq[String] = Seq( + "add_part_exist", + "add_part_multiple", + "add_partition_no_whitelist", + "add_partition_with_whitelist", + "alias_casted_column", + "alter_partition_with_whitelist", + "ambiguous_col", + "annotate_stats_join", + "annotate_stats_limit", + "annotate_stats_part", + "annotate_stats_table", + "annotate_stats_union", "binary_constant", "binarysortable_1", "cast1", @@ -405,15 +629,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "compute_stats_long", "compute_stats_string", "convert_enum_to_string", - "correlationoptimizer1", "correlationoptimizer10", "correlationoptimizer11", "correlationoptimizer13", "correlationoptimizer14", "correlationoptimizer15", - "correlationoptimizer2", - "correlationoptimizer3", - "correlationoptimizer4", "correlationoptimizer6", "correlationoptimizer7", "correlationoptimizer8", @@ -421,57 +641,35 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "count", "cp_mj_rc", "create_insert_outputformat", - "create_like_tbl_props", - "create_like_view", "create_nested_type", - "create_skewed_table1", "create_struct_table", "create_view_translate", "cross_join", "cross_product_check_1", "cross_product_check_2", - "ct_case_insensitive", "database_drop", "database_location", "database_properties", - "date_1", "date_2", - "date_3", - "date_4", "date_comparison", - "date_join1", - "date_serde", "decimal_1", "decimal_4", "decimal_join", "default_partition_name", "delimiter", "desc_non_existent_tbl", - "describe_formatted_view_partitioned", - "diff_part_input_formats", "disable_file_format_check", - "disallow_incompatible_type_change_off", "distinct_stats", - "drop_database_removes_partition_dirs", "drop_function", - "drop_index", - "drop_index_removes_partition_dirs", "drop_multi_partitions", - "drop_partitions_filter", - "drop_partitions_filter2", - "drop_partitions_filter3", - "drop_partitions_ignore_protection", "drop_table", "drop_table2", - "drop_table_removes_partition_dirs", "drop_view", "dynamic_partition_skip_default", "escape_clusterby1", "escape_distributeby1", "escape_orderby1", "escape_sortby1", - "explain_rearrange", - "fileformat_mix", "fileformat_sequencefile", "fileformat_text", "filter_join_breaktask", @@ -480,9 +678,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby11", "groupby12", "groupby1_limit", - "groupby_grouping_id1", - "groupby_grouping_id2", - "groupby_grouping_sets1", "groupby_grouping_sets2", "groupby_grouping_sets3", "groupby_grouping_sets4", @@ -528,22 +723,12 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby_neg_float", "groupby_ppd", "groupby_ppr", - "groupby_sort_10", - "groupby_sort_2", - "groupby_sort_3", - "groupby_sort_4", - "groupby_sort_5", "groupby_sort_6", - "groupby_sort_7", - "groupby_sort_8", - "groupby_sort_9", - "groupby_sort_test_1", "having", "implicit_cast1", "index_serde", "infer_bucket_sort_dyn_part", "innerjoin", - "inoutdriver", "input", "input0", "input1", @@ -565,8 +750,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "input26", "input28", "input2_limit", - "input3", - "input4", "input40", "input41", "input49", @@ -578,8 +761,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "input_limit", "input_part0", "input_part1", - "input_part10", - "input_part10_win", "input_part2", "input_part3", "input_part4", @@ -592,16 +773,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "inputddl1", "inputddl2", "inputddl3", - "inputddl4", - "inputddl5", - "inputddl6", - "inputddl7", "inputddl8", "insert1", "insert1_overwrite_partitions", "insert2_overwrite_partitions", - "insert_compressed", - "join0", "join1", "join10", "join11", @@ -649,25 +824,19 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "join_array", "join_casesensitive", "join_empty", - "join_filters", "join_hive_626", "join_map_ppr", - "join_nulls", - "join_nullsafe", "join_rc", "join_reorder2", "join_reorder3", "join_reorder4", "join_star", "lateral_view", - "lateral_view_cp", "lateral_view_noalias", "lateral_view_ppd", - "leftsemijoin", "leftsemijoin_mr", "limit_pushdown_negative", "lineage1", - "literal_double", "literal_ints", "literal_string", "load_dyn_part1", @@ -677,7 +846,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "load_dyn_part13", "load_dyn_part14", "load_dyn_part14_win", - "load_dyn_part2", "load_dyn_part3", "load_dyn_part4", "load_dyn_part5", @@ -688,12 +856,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "load_file_with_space_in_the_name", "loadpart1", "louter_join_ppr", - "macro", "mapjoin_distinct", "mapjoin_filter_on_outerjoin", "mapjoin_mapjoin", "mapjoin_subquery", - "mapjoin_subquery2", "mapjoin_test_outer", "mapreduce1", "mapreduce2", @@ -707,7 +873,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "merge2", "merge4", "mergejoins", - "multiMapJoin1", "multiMapJoin2", "multi_insert_gby", "multi_insert_gby3", @@ -715,7 +880,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "multi_join_union", "multigroupby_singlemr", "noalias_subq1", - "nomore_ambiguous_table_col", "nonblock_op_deduplicate", "notable_alias1", "notable_alias2", @@ -730,28 +894,17 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "nullinput2", "nullscript", "optional_outer", - "orc_dictionary_threshold", - "orc_empty_files", "order", "order2", "outer_join_ppr", "parallel", "parenthesis_star_by", "part_inherit_tbl_props", - "part_inherit_tbl_props_empty", "part_inherit_tbl_props_with_star", "partcols1", - "partition_date", - "partition_schema1", "partition_serde_format", "partition_type_check", - "partition_varchar1", - "partition_wise_fileformat4", - "partition_wise_fileformat5", - "partition_wise_fileformat6", - "partition_wise_fileformat7", "partition_wise_fileformat9", - "plan_json", "ppd1", "ppd2", "ppd_clusterby", @@ -770,7 +923,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "ppd_outer_join4", "ppd_outer_join5", "ppd_random", - "ppd_repeated_alias", "ppd_udf_col", "ppd_union", "ppr_allchildsarenull", @@ -778,8 +930,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "ppr_pushdown2", "ppr_pushdown3", "progress_1", - "protectmode", - "push_or", "query_with_semi", "quote1", "quote2", @@ -788,12 +938,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "rcfile_null_value", "rcfile_toleratecorruptions", "rcfile_union", - "reduce_deduplicate", "reduce_deduplicate_exclude_gby", "reduce_deduplicate_exclude_join", "reduce_deduplicate_extended", - "reducesink_dedup", - "rename_column", "router_join_ppr", "select_as_omitted", "select_unquote_and", @@ -802,68 +949,29 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "semicolon", "semijoin", "serde_regex", - "serde_reported_schema", "set_variable_sub", "show_columns", - "show_create_table_alter", - "show_create_table_db_table", - "show_create_table_delimited", - "show_create_table_does_not_exist", - "show_create_table_index", - "show_create_table_partitioned", - "show_create_table_serde", - "show_create_table_view", "show_describe_func_quotes", "show_functions", "show_partitions", "show_tblproperties", - "skewjoinopt13", - "skewjoinopt18", - "skewjoinopt9", - "smb_mapjoin9", - "smb_mapjoin_1", - "smb_mapjoin_10", - "smb_mapjoin_13", - "smb_mapjoin_14", - "smb_mapjoin_15", - "smb_mapjoin_16", - "smb_mapjoin_17", - "smb_mapjoin_2", - "smb_mapjoin_21", - "smb_mapjoin_25", - "smb_mapjoin_3", - "smb_mapjoin_4", - "smb_mapjoin_5", - "smb_mapjoin_6", - "smb_mapjoin_7", - "smb_mapjoin_8", "sort", - "sort_merge_join_desc_1", - "sort_merge_join_desc_2", - "sort_merge_join_desc_3", - "sort_merge_join_desc_4", - "sort_merge_join_desc_5", - "sort_merge_join_desc_6", - "sort_merge_join_desc_7", - "stats0", "stats_aggregator_error_1", - "stats_empty_partition", "stats_publisher_error_1", "subq2", + "subquery_exists", + "subquery_exists_having", + "subquery_notexists", + "subquery_notexists_having", + "subquery_in_having", "tablename_with_select", - "timestamp_3", "timestamp_comparison", - "timestamp_lazy", "timestamp_null", - "touch", "transform_ppr1", "transform_ppr2", - "truncate_table", "type_cast_1", "type_widening", "udaf_collect_set", - "udaf_covar_pop", - "udaf_covar_samp", "udaf_histogram_numeric", "udf2", "udf5", @@ -873,11 +981,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_10_trims", "udf_E", "udf_PI", - "udf_abs", "udf_acos", "udf_add", - "udf_array", - "udf_array_contains", + // "udf_array", -- done in array.sql + // "udf_array_contains", -- done in array.sql "udf_ascii", "udf_asin", "udf_atan", @@ -913,15 +1020,12 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_elt", "udf_equal", "udf_exp", - "udf_field", "udf_find_in_set", "udf_float", "udf_floor", - "udf_format_number", "udf_from_unixtime", "udf_greaterthan", "udf_greaterthanorequal", - "udf_hash", "udf_hex", "udf_if", "udf_index", @@ -959,14 +1063,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_power", "udf_radians", "udf_rand", - "udf_reflect2", "udf_regexp", "udf_regexp_extract", "udf_regexp_replace", "udf_repeat", "udf_rlike", - "udf_round", - "udf_round_3", "udf_rpad", "udf_rtrim", "udf_sign", @@ -1002,15 +1103,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_variance", "udf_weekofyear", "udf_when", - "udf_xpath", - "udf_xpath_boolean", - "udf_xpath_double", - "udf_xpath_float", - "udf_xpath_int", - "udf_xpath_long", - "udf_xpath_short", - "udf_xpath_string", - "unicode_notation", "union10", "union11", "union13", @@ -1032,7 +1124,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "union29", "union3", "union30", - "union31", "union33", "union34", "union4", @@ -1041,15 +1132,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "union7", "union8", "union9", - "union_date", "union_lateralview", "union_ppr", - "union_remove_11", - "union_remove_3", "union_remove_6", "union_script", - "varchar_2", - "varchar_join1", "varchar_union1", "view", "view_cast", diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala index 92bb9e6d73af..c7d953a731b9 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -38,7 +38,8 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte private val testTempDir = Utils.createTempDir() override def beforeAll() { - TestHive.cacheTables = true + super.beforeAll() + TestHive.setCacheTables(true) // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting @@ -94,16 +95,20 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte // This is used to generate golden files. sql("set hive.plan.serialization.format=kryo") // Explicitly set fs to local fs. - sql(s"set fs.default.name=file://$testTempDir/") + sql(s"set fs.defaultFS=file://$testTempDir/") // Ask Hive to run jobs in-process as a single map and reduce task. - sql("set mapred.job.tracker=local") + sql("set mapreduce.jobtracker.address=local") } override def afterAll() { - TestHive.cacheTables = false - TimeZone.setDefault(originalTimeZone) - Locale.setDefault(originalLocale) - TestHive.reset() + try { + TestHive.setCacheTables(false) + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + TestHive.reset() + } finally { + super.afterAll() + } } ///////////////////////////////////////////////////////////////////////////// @@ -454,6 +459,9 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte |window w1 as (distribute by p_mfgr sort by p_name rows between 2 preceding and 2 following) """.stripMargin, reset = false) + /* Disabled because: + - Spark uses a different default stddev. + - Tiny numerical differences in stddev results. createQueryTest("windowing.q -- 15. testExpressions", s""" |select p_mfgr,p_name, p_size, @@ -472,7 +480,7 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name | rows between 2 preceding and 2 following) """.stripMargin, reset = false) - + */ createQueryTest("windowing.q -- 16. testMultipleWindows", s""" |select p_mfgr,p_name, p_size, @@ -526,33 +534,11 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte | rows between 2 preceding and 2 following); """.stripMargin, reset = false) - // collect_set() output array in an arbitrary order, hence causes different result - // when running this test suite under Java 7 and 8. - // We change the original sql query a little bit for making the test suite passed - // under different JDK - createQueryTest("windowing.q -- 20. testSTATs", - """ - |select p_mfgr,p_name, p_size, sdev, sdev_pop, uniq_data, var, cor, covarp - |from ( - |select p_mfgr,p_name, p_size, - |stddev(p_retailprice) over w1 as sdev, - |stddev_pop(p_retailprice) over w1 as sdev_pop, - |collect_set(p_size) over w1 as uniq_size, - |variance(p_retailprice) over w1 as var, - |corr(p_size, p_retailprice) over w1 as cor, - |covar_pop(p_size, p_retailprice) over w1 as covarp - |from part - |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name - | rows between 2 preceding and 2 following) - |) t lateral view explode(uniq_size) d as uniq_data - |order by p_mfgr,p_name, p_size, sdev, sdev_pop, uniq_data, var, cor, covarp - """.stripMargin, reset = false) - createQueryTest("windowing.q -- 21. testDISTs", """ |select p_mfgr,p_name, p_size, |histogram_numeric(p_retailprice, 5) over w1 as hist, - |percentile(p_partkey, 0.5) over w1 as per, + |percentile(p_partkey, cast(0.5 as double)) over w1 as per, |row_number() over(distribute by p_mfgr sort by p_name) as rn |from part |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name @@ -766,7 +752,8 @@ class HiveWindowFunctionQueryFileSuite private val testTempDir = Utils.createTempDir() override def beforeAll() { - TestHive.cacheTables = true + super.beforeAll() + TestHive.setCacheTables(true) // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting @@ -777,16 +764,20 @@ class HiveWindowFunctionQueryFileSuite // This is used to generate golden files. // sql("set hive.plan.serialization.format=kryo") // Explicitly set fs to local fs. - // sql(s"set fs.default.name=file://$testTempDir/") + // sql(s"set fs.defaultFS=file://$testTempDir/") // Ask Hive to run jobs in-process as a single map and reduce task. - // sql("set mapred.job.tracker=local") + // sql("set mapreduce.jobtracker.address=local") } override def afterAll() { - TestHive.cacheTables = false - TimeZone.setDefault(originalTimeZone) - Locale.setDefault(originalLocale) - TestHive.reset() + try { + TestHive.setCacheTables(false) + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + TestHive.reset() + } finally { + super.afterAll() + } } override def blackList: Seq[String] = Seq( @@ -810,15 +801,17 @@ class HiveWindowFunctionQueryFileSuite "windowing_ntile", "windowing_udaf", "windowing_windowspec", - "windowing_rank" - ) + "windowing_rank", - override def whiteList: Seq[String] = Seq( - "windowing_udaf2", + // These tests DROP TABLE that don't exist (but do not specify IF EXISTS) "windowing_columnPruning", "windowing_adjust_rowcontainer_sz" ) + override def whiteList: Seq[String] = Seq( + "windowing_udaf2" + ) + // Only run those query tests in the realWhileList (do not try other ignored query files). override def testCases: Seq[(String, File)] = super.testCases.filter { case (name, _) => realWhiteList.contains(name) diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index d96f3e2b9f62..66fad85ea026 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,13 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT + spark-parent_2.11 + 2.3.0-SNAPSHOT ../../pom.xml - org.apache.spark - spark-hive_2.10 + spark-hive_2.11 jar Spark Project Hive http://spark.apache.org/ @@ -60,7 +59,23 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-sql_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-catalyst_${scala.binary.version} + test-jar + ${project.version} + test + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + - -da -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m + -da -Xmx3g -XX:ReservedCodeCacheSize=${CodeCacheSize} - - org.codehaus.mojo - build-helper-maven-plugin - - - add-default-sources - generate-sources - - add-source - - - - v${hive.version.short}/src/main/scala - - - - - - - org.apache.maven.plugins - maven-dependency-plugin + maven-enforcer-plugin - copy-dependencies - package + enforce-versions - copy-dependencies + enforce - - ${basedir}/../../lib_managed/jars - false - false - true - org.datanucleus + + + + *:hive-cli + + + diff --git a/sql/hive/src/main/java/org/apache/hadoop/hive/ql/io/orc/SparkOrcNewRecordReader.java b/sql/hive/src/main/java/org/apache/hadoop/hive/ql/io/orc/SparkOrcNewRecordReader.java new file mode 100644 index 000000000000..f093637d412f --- /dev/null +++ b/sql/hive/src/main/java/org/apache/hadoop/hive/ql/io/orc/SparkOrcNewRecordReader.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.io.orc; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.TaskAttemptContext; + +import java.io.IOException; +import java.util.List; + +/** + * This is based on hive-exec-1.2.1 + * {@link org.apache.hadoop.hive.ql.io.orc.OrcNewInputFormat.OrcRecordReader}. + * This class exposes getObjectInspector which can be used for reducing + * NameNode calls in OrcRelation. + */ +public class SparkOrcNewRecordReader extends + org.apache.hadoop.mapreduce.RecordReader { + private final org.apache.hadoop.hive.ql.io.orc.RecordReader reader; + private final int numColumns; + OrcStruct value; + private float progress = 0.0f; + private ObjectInspector objectInspector; + + public SparkOrcNewRecordReader(Reader file, Configuration conf, + long offset, long length) throws IOException { + List types = file.getTypes(); + numColumns = (types.size() == 0) ? 0 : types.get(0).getSubtypesCount(); + value = new OrcStruct(numColumns); + this.reader = OrcInputFormat.createReaderFromFile(file, conf, offset, + length); + this.objectInspector = file.getObjectInspector(); + } + + @Override + public void close() throws IOException { + reader.close(); + } + + @Override + public NullWritable getCurrentKey() throws IOException, + InterruptedException { + return NullWritable.get(); + } + + @Override + public OrcStruct getCurrentValue() throws IOException, + InterruptedException { + return value; + } + + @Override + public float getProgress() throws IOException, InterruptedException { + return progress; + } + + @Override + public void initialize(InputSplit split, TaskAttemptContext context) + throws IOException, InterruptedException { + } + + @Override + public boolean nextKeyValue() throws IOException, InterruptedException { + if (reader.hasNext()) { + reader.next(value); + progress = reader.getProgress(); + return true; + } else { + return false; + } + } + + public ObjectInspector getObjectInspector() { + return objectInspector; + } +} diff --git a/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index 4a774fbf1fdf..e7d762fbebe7 100644 --- a/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/hive/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -1 +1,2 @@ -org.apache.spark.sql.hive.orc.DefaultSource +org.apache.spark.sql.hive.orc.OrcFileFormat +org.apache.spark.sql.hive.execution.HiveFileFormat \ No newline at end of file diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala deleted file mode 100644 index 7f8449cdc282..000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import scala.language.implicitConversions - -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.AbstractSparkSQLParser -import org.apache.spark.sql.hive.execution.{AddJar, AddFile, HiveNativeCommand} - -/** - * A parser that recognizes all HiveQL constructs together with Spark SQL specific extensions. - */ -private[hive] class ExtendedHiveQlParser extends AbstractSparkSQLParser { - // Keyword is a convention with AbstractSparkSQLParser, which will scan all of the `Keyword` - // properties via reflection the class in runtime for constructing the SqlLexical object - protected val ADD = Keyword("ADD") - protected val DFS = Keyword("DFS") - protected val FILE = Keyword("FILE") - protected val JAR = Keyword("JAR") - - protected lazy val start: Parser[LogicalPlan] = dfs | addJar | addFile | hiveQl - - protected lazy val hiveQl: Parser[LogicalPlan] = - restInput ^^ { - case statement => HiveQl.createPlan(statement.trim) - } - - protected lazy val dfs: Parser[LogicalPlan] = - DFS ~> wholeInput ^^ { - case command => HiveNativeCommand(command.trim) - } - - private lazy val addFile: Parser[LogicalPlan] = - ADD ~ FILE ~> restInput ^^ { - case input => AddFile(input.trim) - } - - private lazy val addJar: Parser[LogicalPlan] = - ADD ~ JAR ~> restInput ^^ { - case input => AddJar(input.trim) - } -} 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 2d72b959af13..02a5117f005e 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 @@ -17,93 +17,27 @@ package org.apache.spark.sql.hive -import java.io.File -import java.net.{URL, URLClassLoader} -import java.sql.Timestamp -import java.util.concurrent.TimeUnit -import java.util.regex.Pattern - -import scala.collection.JavaConverters._ -import scala.collection.mutable.HashMap -import scala.language.implicitConversions - -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.hive.common.StatsSetupConst -import org.apache.hadoop.hive.common.`type`.HiveDecimal -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.metadata.Table -import org.apache.hadoop.hive.ql.parse.VariableSubstitution -import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable} - +import org.apache.spark.SparkContext import org.apache.spark.api.java.JavaSparkContext -import org.apache.spark.sql.SQLConf.SQLConfEntry -import org.apache.spark.sql.SQLConf.SQLConfEntry._ -import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression} -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.{InternalRow, ParserDialect, SqlParser} -import org.apache.spark.sql.execution.datasources.{ResolveDataSource, DataSourceStrategy, PreInsertCastAndRename, PreWriteCheck} -import org.apache.spark.sql.execution.ui.SQLListener -import org.apache.spark.sql.execution.{CacheManager, ExecutedCommand, ExtractPythonUDFs, SetCommand} -import org.apache.spark.sql.hive.client._ -import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, HiveNativeCommand} -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String -import org.apache.spark.util.Utils -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{SparkSession, SQLContext} -/** - * This is the HiveQL Dialect, this dialect is strongly bind with HiveContext - */ -private[hive] class HiveQLDialect(sqlContext: HiveContext) extends ParserDialect { - override def parse(sqlText: String): LogicalPlan = { - sqlContext.executionHive.withHiveState { - HiveQl.parseSql(sqlText) - } - } -} - -/** - * Returns the current database of metadataHive. - */ -private[hive] case class CurrentDatabase(ctx: HiveContext) - extends LeafExpression with CodegenFallback { - override def dataType: DataType = StringType - override def foldable: Boolean = true - override def nullable: Boolean = false - override def eval(input: InternalRow): Any = { - UTF8String.fromString(ctx.metadataHive.currentDatabase) - } -} - /** * An instance of the Spark SQL execution engine that integrates with data stored in Hive. * Configuration for Hive is read from hive-site.xml on the classpath. - * - * @since 1.0.0 */ -class HiveContext private[hive]( - sc: SparkContext, - cacheManager: CacheManager, - listener: SQLListener, - @transient private val execHive: ClientWrapper, - @transient private val metaHive: ClientInterface, - isRootContext: Boolean) - extends SQLContext(sc, cacheManager, listener, isRootContext) with Logging { +@deprecated("Use SparkSession.builder.enableHiveSupport instead", "2.0.0") +class HiveContext private[hive](_sparkSession: SparkSession) + extends SQLContext(_sparkSession) with Logging { + self => def this(sc: SparkContext) = { - this(sc, new CacheManager, SQLContext.createListenerAndUI(sc), null, null, true) + this(SparkSession.builder().sparkContext(HiveUtils.withHiveExternalCatalog(sc)).getOrCreate()) } - def this(sc: JavaSparkContext) = this(sc.sc) - - import org.apache.spark.sql.hive.HiveContext._ - logDebug("create HiveContext") + def this(sc: JavaSparkContext) = this(sc.sc) /** * Returns a new HiveContext as new session, which will have separated SQLConf, UDF/UDAF, @@ -111,224 +45,9 @@ class HiveContext private[hive]( * and Hive client (both of execution and metadata) with existing HiveContext. */ override def newSession(): HiveContext = { - new HiveContext( - sc = sc, - cacheManager = cacheManager, - listener = listener, - execHive = executionHive.newSession(), - metaHive = metadataHive.newSession(), - isRootContext = false) + new HiveContext(sparkSession.newSession()) } - /** - * When true, enables an experimental feature where metastore tables that use the parquet SerDe - * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive - * SerDe. - */ - protected[sql] def convertMetastoreParquet: Boolean = getConf(CONVERT_METASTORE_PARQUET) - - /** - * When true, also tries to merge possibly different but compatible Parquet schemas in different - * Parquet data files. - * - * This configuration is only effective when "spark.sql.hive.convertMetastoreParquet" is true. - */ - protected[sql] def convertMetastoreParquetWithSchemaMerging: Boolean = - getConf(CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING) - - /** - * 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(CONVERT_CTAS) - - /** - * The version of the hive client that will be used to communicate with the metastore. Note that - * this does not necessarily need to be the same version of Hive that is used internally by - * Spark SQL for execution. - */ - protected[hive] def hiveMetastoreVersion: String = getConf(HIVE_METASTORE_VERSION) - - /** - * The location of the jars that should be used to instantiate the HiveMetastoreClient. This - * property can be one of three options: - * - a classpath in the standard format for both hive and hadoop. - * - builtin - attempt to discover the jars that were used to load Spark SQL and use those. This - * option is only valid when using the execution version of Hive. - * - maven - download the correct version of hive on demand from maven. - */ - protected[hive] def hiveMetastoreJars: String = getConf(HIVE_METASTORE_JARS) - - /** - * A comma separated list of class prefixes that should be loaded using the classloader that - * is shared between Spark SQL and a specific version of Hive. An example of classes that should - * be shared is JDBC drivers that are needed to talk to the metastore. Other classes that need - * to be shared are those that interact with classes that are already shared. For example, - * custom appenders that are used by log4j. - */ - protected[hive] def hiveMetastoreSharedPrefixes: Seq[String] = - getConf(HIVE_METASTORE_SHARED_PREFIXES).filterNot(_ == "") - - /** - * A comma separated list of class prefixes that should explicitly be reloaded for each version - * of Hive that Spark SQL is communicating with. For example, Hive UDFs that are declared in a - * prefix that typically would be shared (i.e. org.apache.spark.*) - */ - protected[hive] def hiveMetastoreBarrierPrefixes: Seq[String] = - getConf(HIVE_METASTORE_BARRIER_PREFIXES).filterNot(_ == "") - - /* - * hive thrift server use background spark sql thread pool to execute sql queries - */ - protected[hive] def hiveThriftServerAsync: Boolean = getConf(HIVE_THRIFT_SERVER_ASYNC) - - @transient - protected[sql] lazy val substitutor = new VariableSubstitution() - - /** - * The copy of the hive client that is used for execution. Currently this must always be - * Hive 13 as this is the version of Hive that is packaged with Spark SQL. This copy of the - * client is used for execution related tasks like registering temporary functions or ensuring - * that the ThreadLocal SessionState is correctly populated. This copy of Hive is *not* used - * for storing persistent metadata, and only point to a dummy metastore in a temporary directory. - */ - @transient - protected[hive] lazy val executionHive: ClientWrapper = if (execHive != null) { - execHive - } else { - logInfo(s"Initializing execution hive, version $hiveExecutionVersion") - val loader = new IsolatedClientLoader( - version = IsolatedClientLoader.hiveVersion(hiveExecutionVersion), - execJars = Seq(), - config = newTemporaryConfiguration(), - isolationOn = false, - baseClassLoader = Utils.getContextOrSparkClassLoader) - loader.createClient().asInstanceOf[ClientWrapper] - } - - /** - * Overrides default Hive configurations to avoid breaking changes to Spark SQL users. - * - allow SQL11 keywords to be used as identifiers - */ - private[sql] def defaultOverrides() = { - setConf(ConfVars.HIVE_SUPPORT_SQL11_RESERVED_KEYWORDS.varname, "false") - } - - defaultOverrides() - - /** - * The copy of the Hive client that is used to retrieve metadata from the Hive MetaStore. - * The version of the Hive client that is used here must match the metastore that is configured - * in the hive-site.xml file. - */ - @transient - protected[hive] lazy val metadataHive: ClientInterface = if (metaHive != null) { - metaHive - } else { - val metaVersion = IsolatedClientLoader.hiveVersion(hiveMetastoreVersion) - - // We instantiate a HiveConf here to read in the hive-site.xml file and then pass the options - // into the isolated client loader - val metadataConf = new HiveConf() - - val defaultWarehouseLocation = metadataConf.get("hive.metastore.warehouse.dir") - logInfo("default warehouse location is " + defaultWarehouseLocation) - - // `configure` goes second to override other settings. - val allConfig = metadataConf.asScala.map(e => e.getKey -> e.getValue).toMap ++ configure - - val isolatedLoader = if (hiveMetastoreJars == "builtin") { - if (hiveExecutionVersion != hiveMetastoreVersion) { - throw new IllegalArgumentException( - "Builtin jars can only be used when hive execution version == hive metastore version. " + - s"Execution: ${hiveExecutionVersion} != Metastore: ${hiveMetastoreVersion}. " + - "Specify a vaild path to the correct hive jars using $HIVE_METASTORE_JARS " + - s"or change ${HIVE_METASTORE_VERSION.key} to $hiveExecutionVersion.") - } - - // We recursively find all jars in the class loader chain, - // starting from the given classLoader. - def allJars(classLoader: ClassLoader): Array[URL] = classLoader match { - case null => Array.empty[URL] - case urlClassLoader: URLClassLoader => - urlClassLoader.getURLs ++ allJars(urlClassLoader.getParent) - case other => allJars(other.getParent) - } - - val classLoader = Utils.getContextOrSparkClassLoader - val jars = allJars(classLoader) - if (jars.length == 0) { - throw new IllegalArgumentException( - "Unable to locate hive jars to connect to metastore. " + - "Please set spark.sql.hive.metastore.jars.") - } - - logInfo( - s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using Spark classes.") - new IsolatedClientLoader( - version = metaVersion, - execJars = jars.toSeq, - config = allConfig, - isolationOn = true, - barrierPrefixes = hiveMetastoreBarrierPrefixes, - sharedPrefixes = hiveMetastoreSharedPrefixes) - } else if (hiveMetastoreJars == "maven") { - // TODO: Support for loading the jars from an already downloaded location. - logInfo( - s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using maven.") - IsolatedClientLoader.forVersion( - version = hiveMetastoreVersion, - config = allConfig, - barrierPrefixes = hiveMetastoreBarrierPrefixes, - sharedPrefixes = hiveMetastoreSharedPrefixes) - } else { - // Convert to files and expand any directories. - val jars = - hiveMetastoreJars - .split(File.pathSeparator) - .flatMap { - case path if new File(path).getName() == "*" => - val files = new File(path).getParentFile().listFiles() - if (files == null) { - logWarning(s"Hive jar path '$path' does not exist.") - Nil - } else { - files.filter(_.getName().toLowerCase().endsWith(".jar")) - } - case path => - new File(path) :: Nil - } - .map(_.toURI.toURL) - - logInfo( - s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion " + - s"using ${jars.mkString(":")}") - new IsolatedClientLoader( - version = metaVersion, - execJars = jars.toSeq, - config = allConfig, - isolationOn = true, - barrierPrefixes = hiveMetastoreBarrierPrefixes, - sharedPrefixes = hiveMetastoreSharedPrefixes) - } - isolatedLoader.createClient() - } - - protected[sql] override def parseSql(sql: String): LogicalPlan = { - super.parseSql(substitutor.substitute(hiveconf, sql)) - } - - override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = - new this.QueryExecution(plan) - /** * 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 @@ -338,447 +57,7 @@ class HiveContext private[hive]( * @since 1.3.0 */ def refreshTable(tableName: String): Unit = { - val tableIdent = SqlParser.parseTableIdentifier(tableName) - catalog.refreshTable(tableIdent) + sparkSession.catalog.refreshTable(tableName) } - protected[hive] def invalidateTable(tableName: String): Unit = { - val tableIdent = SqlParser.parseTableIdentifier(tableName) - catalog.invalidateTable(tableIdent) - } - - /** - * Analyzes the given table in the current database to generate statistics, which will be - * used in query optimizations. - * - * Right now, it only supports Hive tables and it only updates the size of a Hive table - * in the Hive metastore. - * - * @since 1.2.0 - */ - def analyze(tableName: String) { - val tableIdent = SqlParser.parseTableIdentifier(tableName) - val relation = EliminateSubQueries(catalog.lookupRelation(tableIdent)) - - relation match { - case relation: MetastoreRelation => - // This method is mainly based on - // org.apache.hadoop.hive.ql.stats.StatsUtils.getFileSizeForTable(HiveConf, Table) - // in Hive 0.13 (except that we do not use fs.getContentSummary). - // TODO: Generalize statistics collection. - // TODO: Why fs.getContentSummary returns wrong size on Jenkins? - // Can we use fs.getContentSummary in future? - // Seems fs.getContentSummary returns wrong table size on Jenkins. So we use - // countFileSize to count the table size. - val stagingDir = metadataHive.getConf(HiveConf.ConfVars.STAGINGDIR.varname, - HiveConf.ConfVars.STAGINGDIR.defaultStrVal) - - def calculateTableSize(fs: FileSystem, path: Path): Long = { - val fileStatus = fs.getFileStatus(path) - val size = if (fileStatus.isDir) { - fs.listStatus(path) - .map { status => - if (!status.getPath().getName().startsWith(stagingDir)) { - calculateTableSize(fs, status.getPath) - } else { - 0L - } - } - .sum - } else { - fileStatus.getLen - } - - size - } - - def getFileSizeForTable(conf: HiveConf, table: Table): Long = { - val path = table.getPath - var size: Long = 0L - try { - val fs = path.getFileSystem(conf) - size = calculateTableSize(fs, path) - } catch { - case e: Exception => - logWarning( - s"Failed to get the size of table ${table.getTableName} in the " + - s"database ${table.getDbName} because of ${e.toString}", e) - size = 0L - } - - size - } - - val tableParameters = relation.hiveQlTable.getParameters - val oldTotalSize = - Option(tableParameters.get(StatsSetupConst.TOTAL_SIZE)) - .map(_.toLong) - .getOrElse(0L) - val newTotalSize = getFileSizeForTable(hiveconf, relation.hiveQlTable) - // Update the Hive metastore if the total size of the table is different than the size - // recorded in the Hive metastore. - // This logic is based on org.apache.hadoop.hive.ql.exec.StatsTask.aggregateStats(). - if (newTotalSize > 0 && newTotalSize != oldTotalSize) { - catalog.client.alterTable( - relation.table.copy( - properties = relation.table.properties + - (StatsSetupConst.TOTAL_SIZE -> newTotalSize.toString))) - } - case otherRelation => - throw new UnsupportedOperationException( - s"Analyze only works for Hive tables, but $tableName is a ${otherRelation.nodeName}") - } - } - - override def setConf(key: String, value: String): Unit = { - super.setConf(key, value) - executionHive.runSqlHive(s"SET $key=$value") - metadataHive.runSqlHive(s"SET $key=$value") - // If users put any Spark SQL setting in the spark conf (e.g. spark-defaults.conf), - // this setConf will be called in the constructor of the SQLContext. - // Also, calling hiveconf will create a default session containing a HiveConf, which - // will interfer with the creation of executionHive (which is a lazy val). So, - // we put hiveconf.set at the end of this method. - hiveconf.set(key, value) - } - - override private[sql] def setConf[T](entry: SQLConfEntry[T], value: T): Unit = { - setConf(entry.key, entry.stringConverter(value)) - } - - /* A catalyst metadata catalog that points to the Hive Metastore. */ - @transient - override protected[sql] lazy val catalog = - new HiveMetastoreCatalog(metadataHive, this) with OverrideCatalog - - // Note that HiveUDFs will be overridden by functions registered in this context. - @transient - override protected[sql] lazy val functionRegistry: FunctionRegistry = - new HiveFunctionRegistry(FunctionRegistry.builtin.copy()) - - // The Hive UDF current_database() is foldable, will be evaluated by optimizer, but the optimizer - // can't access the SessionState of metadataHive. - functionRegistry.registerFunction( - "current_database", - (expressions: Seq[Expression]) => new CurrentDatabase(this)) - - /* An analyzer that uses the Hive metastore. */ - @transient - override protected[sql] lazy val analyzer: Analyzer = - new Analyzer(catalog, functionRegistry, conf) { - override val extendedResolutionRules = - catalog.ParquetConversions :: - catalog.CreateTables :: - catalog.PreInsertionCasts :: - ExtractPythonUDFs :: - ResolveHiveWindowFunction :: - PreInsertCastAndRename :: - (if (conf.runSQLOnFile) new ResolveDataSource(self) :: Nil else Nil) - - override val extendedCheckRules = Seq( - PreWriteCheck(catalog) - ) - } - - /** Overridden by child classes that need to set configuration before the client init. */ - protected def configure(): Map[String, String] = { - // Hive 0.14.0 introduces timeout operations in HiveConf, and changes default values of a bunch - // of time `ConfVar`s by adding time suffixes (`s`, `ms`, and `d` etc.). This breaks backwards- - // compatibility when users are trying to connecting to a Hive metastore of lower version, - // because these options are expected to be integral values in lower versions of Hive. - // - // Here we enumerate all time `ConfVar`s and convert their values to numeric strings according - // to their output time units. - Seq( - ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY -> TimeUnit.SECONDS, - ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.METASTORE_CLIENT_SOCKET_LIFETIME -> TimeUnit.SECONDS, - ConfVars.HMSHANDLERINTERVAL -> TimeUnit.MILLISECONDS, - ConfVars.METASTORE_EVENT_DB_LISTENER_TTL -> TimeUnit.SECONDS, - ConfVars.METASTORE_EVENT_CLEAN_FREQ -> TimeUnit.SECONDS, - ConfVars.METASTORE_EVENT_EXPIRY_DURATION -> TimeUnit.SECONDS, - ConfVars.METASTORE_AGGREGATE_STATS_CACHE_TTL -> TimeUnit.SECONDS, - ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_WRITER_WAIT -> TimeUnit.MILLISECONDS, - ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_READER_WAIT -> TimeUnit.MILLISECONDS, - ConfVars.HIVES_AUTO_PROGRESS_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_LOG_INCREMENTAL_PLAN_PROGRESS_INTERVAL -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_STATS_JDBC_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_STATS_RETRIES_WAIT -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES -> TimeUnit.SECONDS, - ConfVars.HIVE_ZOOKEEPER_SESSION_TIMEOUT -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_ZOOKEEPER_CONNECTION_BASESLEEPTIME -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_TXN_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_COMPACTOR_WORKER_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_COMPACTOR_CHECK_INTERVAL -> TimeUnit.SECONDS, - ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_SERVER2_THRIFT_HTTP_WORKER_KEEPALIVE_TIME -> TimeUnit.SECONDS, - ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_MAX_AGE -> TimeUnit.SECONDS, - ConfVars.HIVE_SERVER2_THRIFT_LOGIN_BEBACKOFF_SLOT_LENGTH -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_SERVER2_THRIFT_LOGIN_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_SERVER2_THRIFT_WORKER_KEEPALIVE_TIME -> TimeUnit.SECONDS, - ConfVars.HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_SERVER2_ASYNC_EXEC_KEEPALIVE_TIME -> TimeUnit.SECONDS, - ConfVars.HIVE_SERVER2_LONG_POLLING_TIMEOUT -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_SERVER2_SESSION_CHECK_INTERVAL -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_SERVER2_IDLE_SESSION_TIMEOUT -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT -> TimeUnit.MILLISECONDS, - ConfVars.SERVER_READ_SOCKET_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_LOCALIZE_RESOURCE_WAIT_INTERVAL -> TimeUnit.MILLISECONDS, - ConfVars.SPARK_CLIENT_FUTURE_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.SPARK_JOB_MONITOR_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.SPARK_RPC_CLIENT_CONNECT_TIMEOUT -> TimeUnit.MILLISECONDS, - ConfVars.SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT -> TimeUnit.MILLISECONDS - ).map { case (confVar, unit) => - confVar.varname -> hiveconf.getTimeVar(confVar, unit).toString - }.toMap - } - - /** - * SQLConf and HiveConf contracts: - * - * 1. create a new SessionState for each HiveContext - * 2. when the Hive session is first initialized, params in HiveConf will get picked up by the - * SQLConf. Additionally, any properties set by set() or a SET command inside sql() will be - * set in the SQLConf *as well as* in the HiveConf. - */ - @transient - protected[hive] lazy val hiveconf: HiveConf = { - val c = executionHive.conf - setConf(c.getAllProperties) - c - } - - protected[sql] override lazy val conf: SQLConf = new SQLConf { - override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") - override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) - } - - protected[sql] override def getSQLDialect(): ParserDialect = { - if (conf.dialect == "hiveql") { - new HiveQLDialect(this) - } else { - super.getSQLDialect() - } - } - - @transient - private val hivePlanner = new SparkPlanner with HiveStrategies { - val hiveContext = self - - override def strategies: Seq[Strategy] = experimental.extraStrategies ++ Seq( - DataSourceStrategy, - HiveCommandStrategy(self), - HiveDDLStrategy, - DDLStrategy, - TakeOrderedAndProject, - InMemoryScans, - HiveTableScans, - DataSinks, - Scripts, - HashAggregation, - Aggregation, - LeftSemiJoin, - EquiJoinSelection, - BasicOperators, - BroadcastNestedLoop, - CartesianProduct, - DefaultJoin - ) - } - - private def functionOrMacroDDLPattern(command: String) = Pattern.compile( - ".*(create|drop)\\s+(temporary\\s+)?(function|macro).+", Pattern.DOTALL).matcher(command) - - protected[hive] def runSqlHive(sql: String): Seq[String] = { - val command = sql.trim.toLowerCase - if (functionOrMacroDDLPattern(command).matches()) { - executionHive.runSqlHive(sql) - } else if (command.startsWith("set")) { - metadataHive.runSqlHive(sql) - executionHive.runSqlHive(sql) - } else { - metadataHive.runSqlHive(sql) - } - } - - @transient - override protected[sql] val planner = hivePlanner - - /** Extends QueryExecution with hive specific features. */ - protected[sql] class QueryExecution(logicalPlan: LogicalPlan) - extends org.apache.spark.sql.execution.QueryExecution(this, logicalPlan) { - - /** - * Returns the result as a hive compatible sequence of strings. For native commands, the - * execution is simply passed back to Hive. - */ - def stringResult(): Seq[String] = executedPlan match { - case ExecutedCommand(desc: DescribeHiveTableCommand) => - // If it is a describe command for a Hive table, we want to have the output format - // be similar with Hive. - desc.run(self).map { - case Row(name: String, dataType: String, comment) => - Seq(name, dataType, - Option(comment.asInstanceOf[String]).getOrElse("")) - .map(s => String.format(s"%-20s", s)) - .mkString("\t") - } - case command: ExecutedCommand => - command.executeCollect().map(_.getString(0)) - - case other => - val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq - // We need the types so we can output struct field names - val types = analyzed.output.map(_.dataType) - // Reformat to match hive tab delimited output. - result.map(_.zip(types).map(HiveContext.toHiveString)).map(_.mkString("\t")).toSeq - } - - override def simpleString: String = - logical match { - case _: HiveNativeCommand => "" - case _: SetCommand => "" - case _ => super.simpleString - } - } - - protected[sql] override def addJar(path: String): Unit = { - // Add jar to Hive and classloader - executionHive.addJar(path) - metadataHive.addJar(path) - Thread.currentThread().setContextClassLoader(executionHive.clientLoader.classLoader) - super.addJar(path) - } -} - - -private[hive] object HiveContext { - /** The version of hive used internally by Spark SQL. */ - val hiveExecutionVersion: String = "1.2.1" - - val HIVE_METASTORE_VERSION = stringConf("spark.sql.hive.metastore.version", - defaultValue = Some(hiveExecutionVersion), - doc = "Version of the Hive metastore. Available options are " + - s"0.12.0 through $hiveExecutionVersion.") - - val HIVE_EXECUTION_VERSION = stringConf( - key = "spark.sql.hive.version", - defaultValue = Some(hiveExecutionVersion), - doc = "Version of Hive used internally by Spark SQL.") - - val HIVE_METASTORE_JARS = stringConf("spark.sql.hive.metastore.jars", - defaultValue = Some("builtin"), - doc = s""" - | Location of the jars that should be used to instantiate the HiveMetastoreClient. - | This property can be one of three options: " - | 1. "builtin" - | Use Hive ${hiveExecutionVersion}, which is bundled with the Spark assembly jar when - | -Phive is enabled. When this option is chosen, - | spark.sql.hive.metastore.version must be either - | ${hiveExecutionVersion} or not defined. - | 2. "maven" - | Use Hive jars of specified version downloaded from Maven repositories. - | 3. A classpath in the standard format for both Hive and Hadoop. - """.stripMargin) - val CONVERT_METASTORE_PARQUET = booleanConf("spark.sql.hive.convertMetastoreParquet", - defaultValue = Some(true), - doc = "When set to false, Spark SQL will use the Hive SerDe for parquet tables instead of " + - "the built in support.") - - val CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING = booleanConf( - "spark.sql.hive.convertMetastoreParquet.mergeSchema", - defaultValue = Some(false), - doc = "TODO") - - val CONVERT_CTAS = booleanConf("spark.sql.hive.convertCTAS", - defaultValue = Some(false), - doc = "TODO") - - val HIVE_METASTORE_SHARED_PREFIXES = stringSeqConf("spark.sql.hive.metastore.sharedPrefixes", - defaultValue = Some(jdbcPrefixes), - doc = "A comma separated list of class prefixes that should be loaded using the classloader " + - "that is shared between Spark SQL and a specific version of Hive. An example of classes " + - "that should be shared is JDBC drivers that are needed to talk to the metastore. Other " + - "classes that need to be shared are those that interact with classes that are already " + - "shared. For example, custom appenders that are used by log4j.") - - private def jdbcPrefixes = Seq( - "com.mysql.jdbc", "org.postgresql", "com.microsoft.sqlserver", "oracle.jdbc") - - val HIVE_METASTORE_BARRIER_PREFIXES = stringSeqConf("spark.sql.hive.metastore.barrierPrefixes", - defaultValue = Some(Seq()), - doc = "A comma separated list of class prefixes that should explicitly be reloaded for each " + - "version of Hive that Spark SQL is communicating with. For example, Hive UDFs that are " + - "declared in a prefix that typically would be shared (i.e. org.apache.spark.*).") - - val HIVE_THRIFT_SERVER_ASYNC = booleanConf("spark.sql.hive.thriftServer.async", - defaultValue = Some(true), - doc = "TODO") - - /** Constructs a configuration for hive, where the metastore is located in a temp directory. */ - def newTemporaryConfiguration(): Map[String, String] = { - val tempDir = Utils.createTempDir() - val localMetastore = new File(tempDir, "metastore") - val propMap: HashMap[String, String] = HashMap() - // We have to mask all properties in hive-site.xml that relates to metastore data source - // as we used a local metastore here. - HiveConf.ConfVars.values().foreach { confvar => - if (confvar.varname.contains("datanucleus") || confvar.varname.contains("jdo") - || confvar.varname.contains("hive.metastore.rawstore.impl")) { - propMap.put(confvar.varname, confvar.getDefaultExpr()) - } - } - propMap.put(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, localMetastore.toURI.toString) - propMap.put(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, - s"jdbc:derby:;databaseName=${localMetastore.getAbsolutePath};create=true") - propMap.put("datanucleus.rdbms.datastoreAdapterClassName", - "org.datanucleus.store.rdbms.adapter.DerbyAdapter") - propMap.toMap - } - - protected val primitiveTypes = - Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, - ShortType, DateType, TimestampType, BinaryType) - - protected[sql] def toHiveString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.toSeq.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "NULL" - case (d: Int, DateType) => new DateWritable(d).toString - case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString - case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") - case (decimal: java.math.BigDecimal, DecimalType()) => - // Hive strips trailing zeros so use its toString - HiveDecimal.create(decimal).toString - case (other, tpe) if primitiveTypes contains tpe => other.toString - } - - /** Hive outputs fields of structs slightly differently than top level attributes. */ - protected def toHiveStructString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.toSeq.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "null" - case (s: String, StringType) => "\"" + s + "\"" - case (decimal, DecimalType()) => decimal.toString - case (other, tpe) if primitiveTypes contains tpe => other.toString - } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala new file mode 100644 index 000000000000..96dc983b0bfc --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -0,0 +1,1380 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.IOException +import java.lang.reflect.InvocationTargetException +import java.util +import java.util.Locale + +import scala.collection.mutable +import scala.util.control.NonFatal + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.hive.ql.metadata.HiveException +import org.apache.thrift.TException + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.ColumnStat +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.execution.command.DDLUtils +import org.apache.spark.sql.execution.datasources.{PartitioningUtils, SourceOptions} +import org.apache.spark.sql.hive.client.HiveClient +import org.apache.spark.sql.internal.HiveSerDe +import org.apache.spark.sql.internal.StaticSQLConf._ +import org.apache.spark.sql.types.{DataType, StructType} + + +/** + * A persistent implementation of the system catalog using Hive. + * All public methods must be synchronized for thread-safety. + */ +private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configuration) + extends ExternalCatalog with Logging { + + import CatalogTypes.TablePartitionSpec + import HiveExternalCatalog._ + import CatalogTableType._ + + /** + * A Hive client used to interact with the metastore. + */ + lazy val client: HiveClient = { + HiveUtils.newClientForMetadata(conf, hadoopConf) + } + + // Exceptions thrown by the hive client that we would like to wrap + private val clientExceptions = Set( + classOf[HiveException].getCanonicalName, + classOf[TException].getCanonicalName, + classOf[InvocationTargetException].getCanonicalName) + + /** + * Whether this is an exception thrown by the hive client that should be wrapped. + * + * Due to classloader isolation issues, pattern matching won't work here so we need + * to compare the canonical names of the exceptions, which we assume to be stable. + */ + private def isClientException(e: Throwable): Boolean = { + var temp: Class[_] = e.getClass + var found = false + while (temp != null && !found) { + found = clientExceptions.contains(temp.getCanonicalName) + temp = temp.getSuperclass + } + found + } + + /** + * Run some code involving `client` in a [[synchronized]] block and wrap certain + * exceptions thrown in the process in [[AnalysisException]]. + */ + private def withClient[T](body: => T): T = synchronized { + try { + body + } catch { + case NonFatal(exception) if isClientException(exception) => + val e = exception match { + // Since we are using shim, the exceptions thrown by the underlying method of + // Method.invoke() are wrapped by InvocationTargetException + case i: InvocationTargetException => i.getCause + case o => o + } + throw new AnalysisException( + e.getClass.getCanonicalName + ": " + e.getMessage, cause = Some(e)) + } + } + + /** + * Get the raw table metadata from hive metastore directly. The raw table metadata may contains + * special data source properties and should not be exposed outside of `HiveExternalCatalog`. We + * should interpret these special data source properties and restore the original table metadata + * before returning it. + */ + private[hive] def getRawTable(db: String, table: String): CatalogTable = withClient { + client.getTable(db, table) + } + + /** + * If the given table properties contains datasource properties, throw an exception. We will do + * this check when create or alter a table, i.e. when we try to write table metadata to Hive + * metastore. + */ + private def verifyTableProperties(table: CatalogTable): Unit = { + val invalidKeys = table.properties.keys.filter(_.startsWith(SPARK_SQL_PREFIX)) + if (invalidKeys.nonEmpty) { + throw new AnalysisException(s"Cannot persistent ${table.qualifiedName} into hive metastore " + + s"as table property keys may not start with '$SPARK_SQL_PREFIX': " + + invalidKeys.mkString("[", ", ", "]")) + } + // External users are not allowed to set/switch the table type. In Hive metastore, the table + // type can be switched by changing the value of a case-sensitive table property `EXTERNAL`. + if (table.properties.contains("EXTERNAL")) { + throw new AnalysisException("Cannot set or change the preserved property key: 'EXTERNAL'") + } + } + + /** + * Checks the validity of column names. Hive metastore disallows the table to use comma in + * data column names. Partition columns do not have such a restriction. Views do not have such + * a restriction. + */ + private def verifyColumnNames(table: CatalogTable): Unit = { + if (table.tableType != VIEW) { + table.dataSchema.map(_.name).foreach { colName => + if (colName.contains(",")) { + throw new AnalysisException("Cannot create a table having a column whose name contains " + + s"commas in Hive metastore. Table: ${table.identifier}; Column: $colName") + } + } + } + } + + // -------------------------------------------------------------------------- + // Databases + // -------------------------------------------------------------------------- + + override protected def doCreateDatabase( + dbDefinition: CatalogDatabase, + ignoreIfExists: Boolean): Unit = withClient { + client.createDatabase(dbDefinition, ignoreIfExists) + } + + override protected def doDropDatabase( + db: String, + ignoreIfNotExists: Boolean, + cascade: Boolean): Unit = withClient { + client.dropDatabase(db, ignoreIfNotExists, cascade) + } + + /** + * Alter a database whose name matches the one specified in `dbDefinition`, + * assuming the database exists. + * + * Note: As of now, this only supports altering database properties! + */ + override def alterDatabase(dbDefinition: CatalogDatabase): Unit = withClient { + val existingDb = getDatabase(dbDefinition.name) + if (existingDb.properties == dbDefinition.properties) { + logWarning(s"Request to alter database ${dbDefinition.name} is a no-op because " + + s"the provided database properties are the same as the old ones. Hive does not " + + s"currently support altering other database fields.") + } + client.alterDatabase(dbDefinition) + } + + override def getDatabase(db: String): CatalogDatabase = withClient { + client.getDatabase(db) + } + + override def databaseExists(db: String): Boolean = withClient { + client.databaseExists(db) + } + + override def listDatabases(): Seq[String] = withClient { + client.listDatabases("*") + } + + override def listDatabases(pattern: String): Seq[String] = withClient { + client.listDatabases(pattern) + } + + override def setCurrentDatabase(db: String): Unit = withClient { + client.setCurrentDatabase(db) + } + + // -------------------------------------------------------------------------- + // Tables + // -------------------------------------------------------------------------- + + override protected def doCreateTable( + tableDefinition: CatalogTable, + ignoreIfExists: Boolean): Unit = withClient { + assert(tableDefinition.identifier.database.isDefined) + val db = tableDefinition.identifier.database.get + val table = tableDefinition.identifier.table + requireDbExists(db) + verifyTableProperties(tableDefinition) + verifyColumnNames(tableDefinition) + + if (tableExists(db, table) && !ignoreIfExists) { + throw new TableAlreadyExistsException(db = db, table = table) + } + + // Ideally we should not create a managed table with location, but Hive serde table can + // specify location for managed table. And in [[CreateDataSourceTableAsSelectCommand]] we have + // to create the table directory and write out data before we create this table, to avoid + // exposing a partial written table. + val needDefaultTableLocation = tableDefinition.tableType == MANAGED && + tableDefinition.storage.locationUri.isEmpty + + val tableLocation = if (needDefaultTableLocation) { + Some(CatalogUtils.stringToURI(defaultTablePath(tableDefinition.identifier))) + } else { + tableDefinition.storage.locationUri + } + + if (DDLUtils.isDatasourceTable(tableDefinition)) { + createDataSourceTable( + tableDefinition.withNewStorage(locationUri = tableLocation), + ignoreIfExists) + } else { + val tableWithDataSourceProps = tableDefinition.copy( + // We can't leave `locationUri` empty and count on Hive metastore to set a default table + // location, because Hive metastore uses hive.metastore.warehouse.dir to generate default + // table location for tables in default database, while we expect to use the location of + // default database. + storage = tableDefinition.storage.copy(locationUri = tableLocation), + // Here we follow data source tables and put table metadata like table schema, partition + // columns etc. in table properties, so that we can work around the Hive metastore issue + // about not case preserving and make Hive serde table and view support mixed-case column + // names. + properties = tableDefinition.properties ++ tableMetaToTableProps(tableDefinition)) + client.createTable(tableWithDataSourceProps, ignoreIfExists) + } + } + + private def createDataSourceTable(table: CatalogTable, ignoreIfExists: Boolean): Unit = { + // data source table always have a provider, it's guaranteed by `DDLUtils.isDatasourceTable`. + val provider = table.provider.get + val options = new SourceOptions(table.storage.properties) + + // To work around some hive metastore issues, e.g. not case-preserving, bad decimal type + // support, no column nullability, etc., we should do some extra works before saving table + // metadata into Hive metastore: + // 1. Put table metadata like table schema, partition columns, etc. in table properties. + // 2. Check if this table is hive compatible. + // 2.1 If it's not hive compatible, set location URI, schema, partition columns and bucket + // spec to empty and save table metadata to Hive. + // 2.2 If it's hive compatible, set serde information in table metadata and try to save + // it to Hive. If it fails, treat it as not hive compatible and go back to 2.1 + val tableProperties = tableMetaToTableProps(table) + + // put table provider and partition provider in table properties. + tableProperties.put(DATASOURCE_PROVIDER, provider) + if (table.tracksPartitionsInCatalog) { + tableProperties.put(TABLE_PARTITION_PROVIDER, TABLE_PARTITION_PROVIDER_CATALOG) + } + + // Ideally we should also put `locationUri` in table properties like provider, schema, etc. + // However, in older version of Spark we already store table location in storage properties + // with key "path". Here we keep this behaviour for backward compatibility. + val storagePropsWithLocation = table.storage.properties ++ + table.storage.locationUri.map("path" -> CatalogUtils.URIToString(_)) + + // converts the table metadata to Spark SQL specific format, i.e. set data schema, names and + // bucket specification to empty. Note that partition columns are retained, so that we can + // call partition-related Hive API later. + def newSparkSQLSpecificMetastoreTable(): CatalogTable = { + table.copy( + // Hive only allows directory paths as location URIs while Spark SQL data source tables + // also allow file paths. For non-hive-compatible format, we should not set location URI + // to avoid hive metastore to throw exception. + storage = table.storage.copy( + locationUri = None, + properties = storagePropsWithLocation), + schema = table.partitionSchema, + bucketSpec = None, + properties = table.properties ++ tableProperties) + } + + // converts the table metadata to Hive compatible format, i.e. set the serde information. + def newHiveCompatibleMetastoreTable(serde: HiveSerDe): CatalogTable = { + val location = if (table.tableType == EXTERNAL) { + // When we hit this branch, we are saving an external data source table with hive + // compatible format, which means the data source is file-based and must have a `path`. + require(table.storage.locationUri.isDefined, + "External file-based data source table must have a `path` entry in storage properties.") + Some(table.location) + } else { + None + } + + table.copy( + storage = table.storage.copy( + locationUri = location, + inputFormat = serde.inputFormat, + outputFormat = serde.outputFormat, + serde = serde.serde, + properties = storagePropsWithLocation + ), + properties = table.properties ++ tableProperties) + } + + val qualifiedTableName = table.identifier.quotedString + val maybeSerde = HiveSerDe.sourceToSerDe(provider) + + val (hiveCompatibleTable, logMessage) = maybeSerde match { + case _ if options.skipHiveMetadata => + val message = + s"Persisting data source table $qualifiedTableName into Hive metastore in" + + "Spark SQL specific format, which is NOT compatible with Hive." + (None, message) + + // our bucketing is un-compatible with hive(different hash function) + case _ if table.bucketSpec.nonEmpty => + val message = + s"Persisting bucketed data source table $qualifiedTableName into " + + "Hive metastore in Spark SQL specific format, which is NOT compatible with Hive. " + (None, message) + + case Some(serde) => + val message = + s"Persisting file based data source table $qualifiedTableName into " + + s"Hive metastore in Hive compatible format." + (Some(newHiveCompatibleMetastoreTable(serde)), message) + + case _ => + val message = + s"Couldn't find corresponding Hive SerDe for data source provider $provider. " + + s"Persisting data source table $qualifiedTableName into Hive metastore in " + + s"Spark SQL specific format, which is NOT compatible with Hive." + (None, message) + } + + (hiveCompatibleTable, logMessage) match { + case (Some(table), message) => + // We first try to save the metadata of the table in a Hive compatible way. + // If Hive throws an error, we fall back to save its metadata in the Spark SQL + // specific way. + try { + logInfo(message) + saveTableIntoHive(table, ignoreIfExists) + } catch { + case NonFatal(e) => + val warningMessage = + s"Could not persist ${table.identifier.quotedString} in a Hive " + + "compatible way. Persisting it into Hive metastore in Spark SQL specific format." + logWarning(warningMessage, e) + saveTableIntoHive(newSparkSQLSpecificMetastoreTable(), ignoreIfExists) + } + + case (None, message) => + logWarning(message) + saveTableIntoHive(newSparkSQLSpecificMetastoreTable(), ignoreIfExists) + } + } + + /** + * Data source tables may be non Hive compatible and we need to store table metadata in table + * properties to workaround some Hive metastore limitations. + * This method puts table schema, partition column names, bucket specification into a map, which + * can be used as table properties later. + */ + private def tableMetaToTableProps(table: CatalogTable): mutable.Map[String, String] = { + tableMetaToTableProps(table, table.schema) + } + + private def tableMetaToTableProps( + table: CatalogTable, + schema: StructType): mutable.Map[String, String] = { + val partitionColumns = table.partitionColumnNames + val bucketSpec = table.bucketSpec + + val properties = new mutable.HashMap[String, String] + + properties.put(CREATED_SPARK_VERSION, table.createVersion) + + // Serialized JSON schema string may be too long to be stored into a single metastore table + // property. In this case, we split the JSON string and store each part as a separate table + // property. + val threshold = conf.get(SCHEMA_STRING_LENGTH_THRESHOLD) + val schemaJsonString = schema.json + // Split the JSON string. + val parts = schemaJsonString.grouped(threshold).toSeq + properties.put(DATASOURCE_SCHEMA_NUMPARTS, parts.size.toString) + parts.zipWithIndex.foreach { case (part, index) => + properties.put(s"$DATASOURCE_SCHEMA_PART_PREFIX$index", part) + } + + if (partitionColumns.nonEmpty) { + properties.put(DATASOURCE_SCHEMA_NUMPARTCOLS, partitionColumns.length.toString) + partitionColumns.zipWithIndex.foreach { case (partCol, index) => + properties.put(s"$DATASOURCE_SCHEMA_PARTCOL_PREFIX$index", partCol) + } + } + + if (bucketSpec.isDefined) { + val BucketSpec(numBuckets, bucketColumnNames, sortColumnNames) = bucketSpec.get + + properties.put(DATASOURCE_SCHEMA_NUMBUCKETS, numBuckets.toString) + properties.put(DATASOURCE_SCHEMA_NUMBUCKETCOLS, bucketColumnNames.length.toString) + bucketColumnNames.zipWithIndex.foreach { case (bucketCol, index) => + properties.put(s"$DATASOURCE_SCHEMA_BUCKETCOL_PREFIX$index", bucketCol) + } + + if (sortColumnNames.nonEmpty) { + properties.put(DATASOURCE_SCHEMA_NUMSORTCOLS, sortColumnNames.length.toString) + sortColumnNames.zipWithIndex.foreach { case (sortCol, index) => + properties.put(s"$DATASOURCE_SCHEMA_SORTCOL_PREFIX$index", sortCol) + } + } + } + + properties + } + + private def defaultTablePath(tableIdent: TableIdentifier): String = { + val dbLocation = getDatabase(tableIdent.database.get).locationUri + new Path(new Path(dbLocation), tableIdent.table).toString + } + + private def saveTableIntoHive(tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit = { + assert(DDLUtils.isDatasourceTable(tableDefinition), + "saveTableIntoHive only takes data source table.") + // If this is an external data source table... + if (tableDefinition.tableType == EXTERNAL && + // ... that is not persisted as Hive compatible format (external tables in Hive compatible + // format always set `locationUri` to the actual data location and should NOT be hacked as + // following.) + tableDefinition.storage.locationUri.isEmpty) { + // !! HACK ALERT !! + // + // Due to a restriction of Hive metastore, here we have to set `locationUri` to a temporary + // directory that doesn't exist yet but can definitely be successfully created, and then + // delete it right after creating the external data source table. This location will be + // persisted to Hive metastore as standard Hive table location URI, but Spark SQL doesn't + // really use it. Also, since we only do this workaround for external tables, deleting the + // directory after the fact doesn't do any harm. + // + // Please refer to https://issues.apache.org/jira/browse/SPARK-15269 for more details. + val tempPath = { + val dbLocation = new Path(getDatabase(tableDefinition.database).locationUri) + new Path(dbLocation, tableDefinition.identifier.table + "-__PLACEHOLDER__") + } + + try { + client.createTable( + tableDefinition.withNewStorage(locationUri = Some(tempPath.toUri)), + ignoreIfExists) + } finally { + FileSystem.get(tempPath.toUri, hadoopConf).delete(tempPath, true) + } + } else { + client.createTable(tableDefinition, ignoreIfExists) + } + } + + override protected def doDropTable( + db: String, + table: String, + ignoreIfNotExists: Boolean, + purge: Boolean): Unit = withClient { + requireDbExists(db) + client.dropTable(db, table, ignoreIfNotExists, purge) + } + + override protected def doRenameTable( + db: String, + oldName: String, + newName: String): Unit = withClient { + val rawTable = getRawTable(db, oldName) + + // Note that Hive serde tables don't use path option in storage properties to store the value + // of table location, but use `locationUri` field to store it directly. And `locationUri` field + // will be updated automatically in Hive metastore by the `alterTable` call at the end of this + // method. Here we only update the path option if the path option already exists in storage + // properties, to avoid adding a unnecessary path option for Hive serde tables. + val hasPathOption = CaseInsensitiveMap(rawTable.storage.properties).contains("path") + val storageWithNewPath = if (rawTable.tableType == MANAGED && hasPathOption) { + // If it's a managed table with path option and we are renaming it, then the path option + // becomes inaccurate and we need to update it according to the new table name. + val newTablePath = defaultTablePath(TableIdentifier(newName, Some(db))) + updateLocationInStorageProps(rawTable, Some(newTablePath)) + } else { + rawTable.storage + } + + val newTable = rawTable.copy( + identifier = TableIdentifier(newName, Some(db)), + storage = storageWithNewPath) + + client.alterTable(db, oldName, newTable) + } + + private def getLocationFromStorageProps(table: CatalogTable): Option[String] = { + CaseInsensitiveMap(table.storage.properties).get("path") + } + + private def updateLocationInStorageProps( + table: CatalogTable, + newPath: Option[String]): CatalogStorageFormat = { + // We can't use `filterKeys` here, as the map returned by `filterKeys` is not serializable, + // while `CatalogTable` should be serializable. + val propsWithoutPath = table.storage.properties.filter { + case (k, v) => k.toLowerCase(Locale.ROOT) != "path" + } + table.storage.copy(properties = propsWithoutPath ++ newPath.map("path" -> _)) + } + + /** + * Alter a table whose name that matches the one specified in `tableDefinition`, + * assuming the table exists. This method does not change the properties for data source and + * statistics. + * + * Note: As of now, this doesn't support altering table schema, partition column names and bucket + * specification. We will ignore them even if users do specify different values for these fields. + */ + override def alterTable(tableDefinition: CatalogTable): Unit = withClient { + assert(tableDefinition.identifier.database.isDefined) + val db = tableDefinition.identifier.database.get + requireTableExists(db, tableDefinition.identifier.table) + verifyTableProperties(tableDefinition) + + if (tableDefinition.tableType == VIEW) { + client.alterTable(tableDefinition) + } else { + val oldTableDef = getRawTable(db, tableDefinition.identifier.table) + + val newStorage = if (DDLUtils.isHiveTable(tableDefinition)) { + tableDefinition.storage + } else { + // We can't alter the table storage of data source table directly for 2 reasons: + // 1. internally we use path option in storage properties to store the value of table + // location, but the given `tableDefinition` is from outside and doesn't have the path + // option, we need to add it manually. + // 2. this data source table may be created on a file, not a directory, then we can't set + // the `locationUri` field and save it to Hive metastore, because Hive only allows + // directory as table location. + // + // For example, an external data source table is created with a single file '/path/to/file'. + // Internally, we will add a path option with value '/path/to/file' to storage properties, + // and set the `locationUri` to a special value due to SPARK-15269(please see + // `saveTableIntoHive` for more details). When users try to get the table metadata back, we + // will restore the `locationUri` field from the path option and remove the path option from + // storage properties. When users try to alter the table storage, the given + // `tableDefinition` will have `locationUri` field with value `/path/to/file` and the path + // option is not set. + // + // Here we need 2 extra steps: + // 1. add path option to storage properties, to match the internal format, i.e. using path + // option to store the value of table location. + // 2. set the `locationUri` field back to the old one from the existing table metadata, + // if users don't want to alter the table location. This step is necessary as the + // `locationUri` is not always same with the path option, e.g. in the above example + // `locationUri` is a special value and we should respect it. Note that, if users + // want to alter the table location to a file path, we will fail. This should be fixed + // in the future. + + val newLocation = tableDefinition.storage.locationUri.map(CatalogUtils.URIToString(_)) + val storageWithPathOption = tableDefinition.storage.copy( + properties = tableDefinition.storage.properties ++ newLocation.map("path" -> _)) + + val oldLocation = getLocationFromStorageProps(oldTableDef) + if (oldLocation == newLocation) { + storageWithPathOption.copy(locationUri = oldTableDef.storage.locationUri) + } else { + storageWithPathOption + } + } + + val partitionProviderProp = if (tableDefinition.tracksPartitionsInCatalog) { + TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_CATALOG + } else { + TABLE_PARTITION_PROVIDER -> TABLE_PARTITION_PROVIDER_FILESYSTEM + } + + // Add old data source properties to table properties, to retain the data source table format. + // Add old stats properties to table properties, to retain spark's stats. + // Set the `schema`, `partitionColumnNames` and `bucketSpec` from the old table definition, + // to retain the spark specific format if it is. + val propsFromOldTable = oldTableDef.properties.filter { case (k, v) => + k.startsWith(DATASOURCE_PREFIX) || k.startsWith(STATISTICS_PREFIX) || + k.startsWith(CREATED_SPARK_VERSION) + } + val newTableProps = propsFromOldTable ++ tableDefinition.properties + partitionProviderProp + val newDef = tableDefinition.copy( + storage = newStorage, + schema = oldTableDef.schema, + partitionColumnNames = oldTableDef.partitionColumnNames, + bucketSpec = oldTableDef.bucketSpec, + properties = newTableProps) + + client.alterTable(newDef) + } + } + + override def alterTableSchema(db: String, table: String, schema: StructType): Unit = withClient { + requireTableExists(db, table) + val rawTable = getRawTable(db, table) + // Add table metadata such as table schema, partition columns, etc. to table properties. + val updatedProperties = rawTable.properties ++ tableMetaToTableProps(rawTable, schema) + val withNewSchema = rawTable.copy(properties = updatedProperties, schema = schema) + verifyColumnNames(withNewSchema) + + if (isDatasourceTable(rawTable)) { + // For data source tables, first try to write it with the schema set; if that does not work, + // try again with updated properties and the partition schema. This is a simplified version of + // what createDataSourceTable() does, and may leave the table in a state unreadable by Hive + // (for example, the schema does not match the data source schema, or does not match the + // storage descriptor). + try { + client.alterTable(withNewSchema) + } catch { + case NonFatal(e) => + val warningMessage = + s"Could not alter schema of table ${rawTable.identifier.quotedString} in a Hive " + + "compatible way. Updating Hive metastore in Spark SQL specific format." + logWarning(warningMessage, e) + client.alterTable(withNewSchema.copy(schema = rawTable.partitionSchema)) + } + } else { + client.alterTable(withNewSchema) + } + } + + override def alterTableStats( + db: String, + table: String, + stats: Option[CatalogStatistics]): Unit = withClient { + requireTableExists(db, table) + val rawTable = getRawTable(db, table) + + // For datasource tables and hive serde tables created by spark 2.1 or higher, + // the data schema is stored in the table properties. + val schema = restoreTableMetadata(rawTable).schema + + // convert table statistics to properties so that we can persist them through hive client + var statsProperties = + if (stats.isDefined) { + statsToProperties(stats.get, schema) + } else { + new mutable.HashMap[String, String]() + } + + val oldTableNonStatsProps = rawTable.properties.filterNot(_._1.startsWith(STATISTICS_PREFIX)) + val updatedTable = rawTable.copy(properties = oldTableNonStatsProps ++ statsProperties) + client.alterTable(updatedTable) + } + + override def getTable(db: String, table: String): CatalogTable = withClient { + restoreTableMetadata(getRawTable(db, table)) + } + + /** + * Restores table metadata from the table properties. This method is kind of a opposite version + * of [[createTable]]. + * + * It reads table schema, provider, partition column names and bucket specification from table + * properties, and filter out these special entries from table properties. + */ + private def restoreTableMetadata(inputTable: CatalogTable): CatalogTable = { + if (conf.get(DEBUG_MODE)) { + return inputTable + } + + var table = inputTable + + table.properties.get(DATASOURCE_PROVIDER) match { + case None if table.tableType == VIEW => + // If this is a view created by Spark 2.2 or higher versions, we should restore its schema + // from table properties. + if (table.properties.contains(DATASOURCE_SCHEMA_NUMPARTS)) { + table = table.copy(schema = getSchemaFromTableProperties(table)) + } + + // No provider in table properties, which means this is a Hive serde table. + case None => + table = restoreHiveSerdeTable(table) + + // This is a regular data source table. + case Some(provider) => + table = restoreDataSourceTable(table, provider) + } + + // Restore version info + val version: String = table.properties.getOrElse(CREATED_SPARK_VERSION, "2.2 or prior") + + // Restore Spark's statistics from information in Metastore. + val restoredStats = + statsFromProperties(table.properties, table.identifier.table, table.schema) + if (restoredStats.isDefined) { + table = table.copy(stats = restoredStats) + } + + // Get the original table properties as defined by the user. + table.copy( + createVersion = version, + properties = table.properties.filterNot { case (key, _) => key.startsWith(SPARK_SQL_PREFIX) }) + } + + // Reorder table schema to put partition columns at the end. Before Spark 2.2, the partition + // columns are not put at the end of schema. We need to reorder it when reading the schema + // from the table properties. + private def reorderSchema(schema: StructType, partColumnNames: Seq[String]): StructType = { + val partitionFields = partColumnNames.map { partCol => + schema.find(_.name == partCol).getOrElse { + throw new AnalysisException("The metadata is corrupted. Unable to find the " + + s"partition column names from the schema. schema: ${schema.catalogString}. " + + s"Partition columns: ${partColumnNames.mkString("[", ", ", "]")}") + } + } + StructType(schema.filterNot(partitionFields.contains) ++ partitionFields) + } + + private def restoreHiveSerdeTable(table: CatalogTable): CatalogTable = { + val options = new SourceOptions(table.storage.properties) + val hiveTable = table.copy( + provider = Some(DDLUtils.HIVE_PROVIDER), + tracksPartitionsInCatalog = true) + + // If this is a Hive serde table created by Spark 2.1 or higher versions, we should restore its + // schema from table properties. + if (table.properties.contains(DATASOURCE_SCHEMA_NUMPARTS)) { + val schemaFromTableProps = getSchemaFromTableProperties(table) + val partColumnNames = getPartitionColumnsFromTableProperties(table) + val reorderedSchema = reorderSchema(schema = schemaFromTableProps, partColumnNames) + + if (DataType.equalsIgnoreCaseAndNullability(reorderedSchema, table.schema) || + options.respectSparkSchema) { + hiveTable.copy( + schema = reorderedSchema, + partitionColumnNames = partColumnNames, + bucketSpec = getBucketSpecFromTableProperties(table)) + } else { + // Hive metastore may change the table schema, e.g. schema inference. If the table + // schema we read back is different(ignore case and nullability) from the one in table + // properties which was written when creating table, we should respect the table schema + // from hive. + logWarning(s"The table schema given by Hive metastore(${table.schema.simpleString}) is " + + "different from the schema when this table was created by Spark SQL" + + s"(${schemaFromTableProps.simpleString}). We have to fall back to the table schema " + + "from Hive metastore which is not case preserving.") + hiveTable.copy(schemaPreservesCase = false) + } + } else { + hiveTable.copy(schemaPreservesCase = false) + } + } + + private def restoreDataSourceTable(table: CatalogTable, provider: String): CatalogTable = { + // Internally we store the table location in storage properties with key "path" for data + // source tables. Here we set the table location to `locationUri` field and filter out the + // path option in storage properties, to avoid exposing this concept externally. + val storageWithLocation = { + val tableLocation = getLocationFromStorageProps(table) + // We pass None as `newPath` here, to remove the path option in storage properties. + updateLocationInStorageProps(table, newPath = None).copy( + locationUri = tableLocation.map(CatalogUtils.stringToURI(_))) + } + val partitionProvider = table.properties.get(TABLE_PARTITION_PROVIDER) + + val schemaFromTableProps = getSchemaFromTableProperties(table) + val partColumnNames = getPartitionColumnsFromTableProperties(table) + val reorderedSchema = reorderSchema(schema = schemaFromTableProps, partColumnNames) + + table.copy( + provider = Some(provider), + storage = storageWithLocation, + schema = reorderedSchema, + partitionColumnNames = partColumnNames, + bucketSpec = getBucketSpecFromTableProperties(table), + tracksPartitionsInCatalog = partitionProvider == Some(TABLE_PARTITION_PROVIDER_CATALOG)) + } + + override def tableExists(db: String, table: String): Boolean = withClient { + client.tableExists(db, table) + } + + override def listTables(db: String): Seq[String] = withClient { + requireDbExists(db) + client.listTables(db) + } + + override def listTables(db: String, pattern: String): Seq[String] = withClient { + requireDbExists(db) + client.listTables(db, pattern) + } + + override def loadTable( + db: String, + table: String, + loadPath: String, + isOverwrite: Boolean, + isSrcLocal: Boolean): Unit = withClient { + requireTableExists(db, table) + client.loadTable( + loadPath, + s"$db.$table", + isOverwrite, + isSrcLocal) + } + + override def loadPartition( + db: String, + table: String, + loadPath: String, + partition: TablePartitionSpec, + isOverwrite: Boolean, + inheritTableSpecs: Boolean, + isSrcLocal: Boolean): Unit = withClient { + requireTableExists(db, table) + + val orderedPartitionSpec = new util.LinkedHashMap[String, String]() + getTable(db, table).partitionColumnNames.foreach { colName => + // Hive metastore is not case preserving and keeps partition columns with lower cased names, + // and Hive will validate the column names in partition spec to make sure they are partition + // columns. Here we Lowercase the column names before passing the partition spec to Hive + // client, to satisfy Hive. + orderedPartitionSpec.put(colName.toLowerCase, partition(colName)) + } + + client.loadPartition( + loadPath, + db, + table, + orderedPartitionSpec, + isOverwrite, + inheritTableSpecs, + isSrcLocal) + } + + override def loadDynamicPartitions( + db: String, + table: String, + loadPath: String, + partition: TablePartitionSpec, + replace: Boolean, + numDP: Int): Unit = withClient { + requireTableExists(db, table) + + val orderedPartitionSpec = new util.LinkedHashMap[String, String]() + getTable(db, table).partitionColumnNames.foreach { colName => + // Hive metastore is not case preserving and keeps partition columns with lower cased names, + // and Hive will validate the column names in partition spec to make sure they are partition + // columns. Here we Lowercase the column names before passing the partition spec to Hive + // client, to satisfy Hive. + orderedPartitionSpec.put(colName.toLowerCase, partition(colName)) + } + + client.loadDynamicPartitions( + loadPath, + db, + table, + orderedPartitionSpec, + replace, + numDP) + } + + // -------------------------------------------------------------------------- + // Partitions + // -------------------------------------------------------------------------- + + // Hive metastore is not case preserving and the partition columns are always lower cased. We need + // to lower case the column names in partition specification before calling partition related Hive + // APIs, to match this behaviour. + private def lowerCasePartitionSpec(spec: TablePartitionSpec): TablePartitionSpec = { + spec.map { case (k, v) => k.toLowerCase -> v } + } + + // Build a map from lower-cased partition column names to exact column names for a given table + private def buildLowerCasePartColNameMap(table: CatalogTable): Map[String, String] = { + val actualPartColNames = table.partitionColumnNames + actualPartColNames.map(colName => (colName.toLowerCase, colName)).toMap + } + + // Hive metastore is not case preserving and the column names of the partition specification we + // get from the metastore are always lower cased. We should restore them w.r.t. the actual table + // partition columns. + private def restorePartitionSpec( + spec: TablePartitionSpec, + partColMap: Map[String, String]): TablePartitionSpec = { + spec.map { case (k, v) => partColMap(k.toLowerCase) -> v } + } + + private def restorePartitionSpec( + spec: TablePartitionSpec, + partCols: Seq[String]): TablePartitionSpec = { + spec.map { case (k, v) => partCols.find(_.equalsIgnoreCase(k)).get -> v } + } + + override def createPartitions( + db: String, + table: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit = withClient { + requireTableExists(db, table) + + val tableMeta = getTable(db, table) + val partitionColumnNames = tableMeta.partitionColumnNames + val tablePath = new Path(tableMeta.location) + val partsWithLocation = parts.map { p => + // Ideally we can leave the partition location empty and let Hive metastore to set it. + // However, Hive metastore is not case preserving and will generate wrong partition location + // with lower cased partition column names. Here we set the default partition location + // manually to avoid this problem. + val partitionPath = p.storage.locationUri.map(uri => new Path(uri)).getOrElse { + ExternalCatalogUtils.generatePartitionPath(p.spec, partitionColumnNames, tablePath) + } + p.copy(storage = p.storage.copy(locationUri = Some(partitionPath.toUri))) + } + val lowerCasedParts = partsWithLocation.map(p => p.copy(spec = lowerCasePartitionSpec(p.spec))) + client.createPartitions(db, table, lowerCasedParts, ignoreIfExists) + } + + override def dropPartitions( + db: String, + table: String, + parts: Seq[TablePartitionSpec], + ignoreIfNotExists: Boolean, + purge: Boolean, + retainData: Boolean): Unit = withClient { + requireTableExists(db, table) + client.dropPartitions( + db, table, parts.map(lowerCasePartitionSpec), ignoreIfNotExists, purge, retainData) + } + + override def renamePartitions( + db: String, + table: String, + specs: Seq[TablePartitionSpec], + newSpecs: Seq[TablePartitionSpec]): Unit = withClient { + client.renamePartitions( + db, table, specs.map(lowerCasePartitionSpec), newSpecs.map(lowerCasePartitionSpec)) + + val tableMeta = getTable(db, table) + val partitionColumnNames = tableMeta.partitionColumnNames + // Hive metastore is not case preserving and keeps partition columns with lower cased names. + // When Hive rename partition for managed tables, it will create the partition location with + // a default path generate by the new spec with lower cased partition column names. This is + // unexpected and we need to rename them manually and alter the partition location. + val hasUpperCasePartitionColumn = partitionColumnNames.exists(col => col.toLowerCase != col) + if (tableMeta.tableType == MANAGED && hasUpperCasePartitionColumn) { + val tablePath = new Path(tableMeta.location) + val fs = tablePath.getFileSystem(hadoopConf) + val newParts = newSpecs.map { spec => + val rightPath = renamePartitionDirectory(fs, tablePath, partitionColumnNames, spec) + val partition = client.getPartition(db, table, lowerCasePartitionSpec(spec)) + partition.copy(storage = partition.storage.copy(locationUri = Some(rightPath.toUri))) + } + alterPartitions(db, table, newParts) + } + } + + /** + * Rename the partition directory w.r.t. the actual partition columns. + * + * It will recursively rename the partition directory from the first partition column, to be most + * compatible with different file systems. e.g. in some file systems, renaming `a=1/b=2` to + * `A=1/B=2` will result to `a=1/B=2`, while in some other file systems, the renaming works, but + * will leave an empty directory `a=1`. + */ + private def renamePartitionDirectory( + fs: FileSystem, + tablePath: Path, + partCols: Seq[String], + newSpec: TablePartitionSpec): Path = { + import ExternalCatalogUtils.getPartitionPathString + + var currentFullPath = tablePath + partCols.foreach { col => + val partValue = newSpec(col) + val expectedPartitionString = getPartitionPathString(col, partValue) + val expectedPartitionPath = new Path(currentFullPath, expectedPartitionString) + + if (fs.exists(expectedPartitionPath)) { + // It is possible that some parental partition directories already exist or doesn't need to + // be renamed. e.g. the partition columns are `a` and `B`, then we don't need to rename + // `/table_path/a=1`. Or we already have a partition directory `A=1/B=2`, and we rename + // another partition to `A=1/B=3`, then we will have `A=1/B=2` and `a=1/b=3`, and we should + // just move `a=1/b=3` into `A=1` with new name `B=3`. + } else { + val actualPartitionString = getPartitionPathString(col.toLowerCase, partValue) + val actualPartitionPath = new Path(currentFullPath, actualPartitionString) + try { + fs.rename(actualPartitionPath, expectedPartitionPath) + } catch { + case e: IOException => + throw new SparkException("Unable to rename partition path from " + + s"$actualPartitionPath to $expectedPartitionPath", e) + } + } + currentFullPath = expectedPartitionPath + } + + currentFullPath + } + + private def statsToProperties( + stats: CatalogStatistics, + schema: StructType): Map[String, String] = { + + var statsProperties: Map[String, String] = + Map(STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) + if (stats.rowCount.isDefined) { + statsProperties += STATISTICS_NUM_ROWS -> stats.rowCount.get.toString() + } + + val colNameTypeMap: Map[String, DataType] = + schema.fields.map(f => (f.name, f.dataType)).toMap + stats.colStats.foreach { case (colName, colStat) => + colStat.toMap(colName, colNameTypeMap(colName)).foreach { case (k, v) => + statsProperties += (columnStatKeyPropName(colName, k) -> v) + } + } + + statsProperties + } + + private def statsFromProperties( + properties: Map[String, String], + table: String, + schema: StructType): Option[CatalogStatistics] = { + + val statsProps = properties.filterKeys(_.startsWith(STATISTICS_PREFIX)) + if (statsProps.isEmpty) { + None + } else { + + val colStats = new mutable.HashMap[String, ColumnStat] + + // For each column, recover its column stats. Note that this is currently a O(n^2) operation, + // but given the number of columns it usually not enormous, this is probably OK as a start. + // If we want to map this a linear operation, we'd need a stronger contract between the + // naming convention used for serialization. + schema.foreach { field => + if (statsProps.contains(columnStatKeyPropName(field.name, ColumnStat.KEY_VERSION))) { + // If "version" field is defined, then the column stat is defined. + val keyPrefix = columnStatKeyPropName(field.name, "") + val colStatMap = statsProps.filterKeys(_.startsWith(keyPrefix)).map { case (k, v) => + (k.drop(keyPrefix.length), v) + } + + ColumnStat.fromMap(table, field, colStatMap).foreach { + colStat => colStats += field.name -> colStat + } + } + } + + Some(CatalogStatistics( + sizeInBytes = BigInt(statsProps(STATISTICS_TOTAL_SIZE)), + rowCount = statsProps.get(STATISTICS_NUM_ROWS).map(BigInt(_)), + colStats = colStats.toMap)) + } + } + + override def alterPartitions( + db: String, + table: String, + newParts: Seq[CatalogTablePartition]): Unit = withClient { + val lowerCasedParts = newParts.map(p => p.copy(spec = lowerCasePartitionSpec(p.spec))) + + val rawTable = getRawTable(db, table) + + // For datasource tables and hive serde tables created by spark 2.1 or higher, + // the data schema is stored in the table properties. + val schema = restoreTableMetadata(rawTable).schema + + // convert partition statistics to properties so that we can persist them through hive api + val withStatsProps = lowerCasedParts.map(p => { + if (p.stats.isDefined) { + val statsProperties = statsToProperties(p.stats.get, schema) + p.copy(parameters = p.parameters ++ statsProperties) + } else { + p + } + }) + + // Note: Before altering table partitions in Hive, you *must* set the current database + // to the one that contains the table of interest. Otherwise you will end up with the + // most helpful error message ever: "Unable to alter partition. alter is not possible." + // See HIVE-2742 for more detail. + client.setCurrentDatabase(db) + client.alterPartitions(db, table, withStatsProps) + } + + override def getPartition( + db: String, + table: String, + spec: TablePartitionSpec): CatalogTablePartition = withClient { + val part = client.getPartition(db, table, lowerCasePartitionSpec(spec)) + restorePartitionMetadata(part, getTable(db, table)) + } + + /** + * Restores partition metadata from the partition properties. + * + * Reads partition-level statistics from partition properties, puts these + * into [[CatalogTablePartition#stats]] and removes these special entries + * from the partition properties. + */ + private def restorePartitionMetadata( + partition: CatalogTablePartition, + table: CatalogTable): CatalogTablePartition = { + val restoredSpec = restorePartitionSpec(partition.spec, table.partitionColumnNames) + + // Restore Spark's statistics from information in Metastore. + // Note: partition-level statistics were introduced in 2.3. + val restoredStats = + statsFromProperties(partition.parameters, table.identifier.table, table.schema) + if (restoredStats.isDefined) { + partition.copy( + spec = restoredSpec, + stats = restoredStats, + parameters = partition.parameters.filterNot { + case (key, _) => key.startsWith(SPARK_SQL_PREFIX) }) + } else { + partition.copy(spec = restoredSpec) + } + } + + /** + * Returns the specified partition or None if it does not exist. + */ + override def getPartitionOption( + db: String, + table: String, + spec: TablePartitionSpec): Option[CatalogTablePartition] = withClient { + client.getPartitionOption(db, table, lowerCasePartitionSpec(spec)).map { part => + restorePartitionMetadata(part, getTable(db, table)) + } + } + + /** + * Returns the partition names from hive metastore for a given table in a database. + */ + override def listPartitionNames( + db: String, + table: String, + partialSpec: Option[TablePartitionSpec] = None): Seq[String] = withClient { + val catalogTable = getTable(db, table) + val partColNameMap = buildLowerCasePartColNameMap(catalogTable).mapValues(escapePathName) + val clientPartitionNames = + client.getPartitionNames(catalogTable, partialSpec.map(lowerCasePartitionSpec)) + clientPartitionNames.map { partitionPath => + val partSpec = PartitioningUtils.parsePathFragmentAsSeq(partitionPath) + partSpec.map { case (partName, partValue) => + partColNameMap(partName.toLowerCase) + "=" + escapePathName(partValue) + }.mkString("/") + } + } + + /** + * Returns the partitions from hive metastore for a given table in a database. + */ + override def listPartitions( + db: String, + table: String, + partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] = withClient { + val partColNameMap = buildLowerCasePartColNameMap(getTable(db, table)) + val res = client.getPartitions(db, table, partialSpec.map(lowerCasePartitionSpec)).map { part => + part.copy(spec = restorePartitionSpec(part.spec, partColNameMap)) + } + + partialSpec match { + // This might be a bug of Hive: When the partition value inside the partial partition spec + // contains dot, and we ask Hive to list partitions w.r.t. the partial partition spec, Hive + // treats dot as matching any single character and may return more partitions than we + // expected. Here we do an extra filter to drop unexpected partitions. + case Some(spec) if spec.exists(_._2.contains(".")) => + res.filter(p => isPartialPartitionSpec(spec, p.spec)) + case _ => res + } + } + + override def listPartitionsByFilter( + db: String, + table: String, + predicates: Seq[Expression], + defaultTimeZoneId: String): Seq[CatalogTablePartition] = withClient { + val rawTable = getRawTable(db, table) + val catalogTable = restoreTableMetadata(rawTable) + + val partColNameMap = buildLowerCasePartColNameMap(catalogTable) + + val clientPrunedPartitions = + client.getPartitionsByFilter(rawTable, predicates).map { part => + part.copy(spec = restorePartitionSpec(part.spec, partColNameMap)) + } + prunePartitionsByFilter(catalogTable, clientPrunedPartitions, predicates, defaultTimeZoneId) + } + + // -------------------------------------------------------------------------- + // Functions + // -------------------------------------------------------------------------- + + override protected def doCreateFunction( + db: String, + funcDefinition: CatalogFunction): Unit = withClient { + requireDbExists(db) + // Hive's metastore is case insensitive. However, Hive's createFunction does + // not normalize the function name (unlike the getFunction part). So, + // we are normalizing the function name. + val functionName = funcDefinition.identifier.funcName.toLowerCase(Locale.ROOT) + requireFunctionNotExists(db, functionName) + val functionIdentifier = funcDefinition.identifier.copy(funcName = functionName) + client.createFunction(db, funcDefinition.copy(identifier = functionIdentifier)) + } + + override protected def doDropFunction(db: String, name: String): Unit = withClient { + requireFunctionExists(db, name) + client.dropFunction(db, name) + } + + override protected def doAlterFunction( + db: String, funcDefinition: CatalogFunction): Unit = withClient { + requireDbExists(db) + val functionName = funcDefinition.identifier.funcName.toLowerCase(Locale.ROOT) + requireFunctionExists(db, functionName) + val functionIdentifier = funcDefinition.identifier.copy(funcName = functionName) + client.alterFunction(db, funcDefinition.copy(identifier = functionIdentifier)) + } + + override protected def doRenameFunction( + db: String, + oldName: String, + newName: String): Unit = withClient { + requireFunctionExists(db, oldName) + requireFunctionNotExists(db, newName) + client.renameFunction(db, oldName, newName) + } + + override def getFunction(db: String, funcName: String): CatalogFunction = withClient { + requireFunctionExists(db, funcName) + client.getFunction(db, funcName) + } + + override def functionExists(db: String, funcName: String): Boolean = withClient { + requireDbExists(db) + client.functionExists(db, funcName) + } + + override def listFunctions(db: String, pattern: String): Seq[String] = withClient { + requireDbExists(db) + client.listFunctions(db, pattern) + } + +} + +object HiveExternalCatalog { + val SPARK_SQL_PREFIX = "spark.sql." + + val DATASOURCE_PREFIX = SPARK_SQL_PREFIX + "sources." + val DATASOURCE_PROVIDER = DATASOURCE_PREFIX + "provider" + val DATASOURCE_SCHEMA = DATASOURCE_PREFIX + "schema" + val DATASOURCE_SCHEMA_PREFIX = DATASOURCE_SCHEMA + "." + val DATASOURCE_SCHEMA_NUMPARTS = DATASOURCE_SCHEMA_PREFIX + "numParts" + val DATASOURCE_SCHEMA_NUMPARTCOLS = DATASOURCE_SCHEMA_PREFIX + "numPartCols" + val DATASOURCE_SCHEMA_NUMSORTCOLS = DATASOURCE_SCHEMA_PREFIX + "numSortCols" + val DATASOURCE_SCHEMA_NUMBUCKETS = DATASOURCE_SCHEMA_PREFIX + "numBuckets" + val DATASOURCE_SCHEMA_NUMBUCKETCOLS = DATASOURCE_SCHEMA_PREFIX + "numBucketCols" + val DATASOURCE_SCHEMA_PART_PREFIX = DATASOURCE_SCHEMA_PREFIX + "part." + val DATASOURCE_SCHEMA_PARTCOL_PREFIX = DATASOURCE_SCHEMA_PREFIX + "partCol." + val DATASOURCE_SCHEMA_BUCKETCOL_PREFIX = DATASOURCE_SCHEMA_PREFIX + "bucketCol." + val DATASOURCE_SCHEMA_SORTCOL_PREFIX = DATASOURCE_SCHEMA_PREFIX + "sortCol." + + val STATISTICS_PREFIX = SPARK_SQL_PREFIX + "statistics." + val STATISTICS_TOTAL_SIZE = STATISTICS_PREFIX + "totalSize" + val STATISTICS_NUM_ROWS = STATISTICS_PREFIX + "numRows" + val STATISTICS_COL_STATS_PREFIX = STATISTICS_PREFIX + "colStats." + + val TABLE_PARTITION_PROVIDER = SPARK_SQL_PREFIX + "partitionProvider" + val TABLE_PARTITION_PROVIDER_CATALOG = "catalog" + val TABLE_PARTITION_PROVIDER_FILESYSTEM = "filesystem" + + val CREATED_SPARK_VERSION = SPARK_SQL_PREFIX + "create.version" + + /** + * Returns the fully qualified name used in table properties for a particular column stat. + * For example, for column "mycol", and "min" stat, this should return + * "spark.sql.statistics.colStats.mycol.min". + */ + private def columnStatKeyPropName(columnName: String, statKey: String): String = { + STATISTICS_COL_STATS_PREFIX + columnName + "." + statKey + } + + // A persisted data source table always store its schema in the catalog. + private def getSchemaFromTableProperties(metadata: CatalogTable): StructType = { + val errorMessage = "Could not read schema from the hive metastore because it is corrupted." + val props = metadata.properties + val schema = props.get(DATASOURCE_SCHEMA) + if (schema.isDefined) { + // Originally, we used `spark.sql.sources.schema` to store the schema of a data source table. + // After SPARK-6024, we removed this flag. + // Although we are not using `spark.sql.sources.schema` any more, we need to still support. + DataType.fromJson(schema.get).asInstanceOf[StructType] + } else if (props.filterKeys(_.startsWith(DATASOURCE_SCHEMA_PREFIX)).isEmpty) { + // If there is no schema information in table properties, it means the schema of this table + // was empty when saving into metastore, which is possible in older version(prior to 2.1) of + // Spark. We should respect it. + new StructType() + } else { + val numSchemaParts = props.get(DATASOURCE_SCHEMA_NUMPARTS) + if (numSchemaParts.isDefined) { + val parts = (0 until numSchemaParts.get.toInt).map { index => + val part = metadata.properties.get(s"$DATASOURCE_SCHEMA_PART_PREFIX$index").orNull + if (part == null) { + throw new AnalysisException(errorMessage + + s" (missing part $index of the schema, ${numSchemaParts.get} parts are expected).") + } + part + } + // Stick all parts back to a single schema string. + DataType.fromJson(parts.mkString).asInstanceOf[StructType] + } else { + throw new AnalysisException(errorMessage) + } + } + } + + private def getColumnNamesByType( + props: Map[String, String], + colType: String, + typeName: String): Seq[String] = { + for { + numCols <- props.get(s"spark.sql.sources.schema.num${colType.capitalize}Cols").toSeq + index <- 0 until numCols.toInt + } yield props.getOrElse( + s"$DATASOURCE_SCHEMA_PREFIX${colType}Col.$index", + throw new AnalysisException( + s"Corrupted $typeName in catalog: $numCols parts expected, but part $index is missing." + ) + ) + } + + private def getPartitionColumnsFromTableProperties(metadata: CatalogTable): Seq[String] = { + getColumnNamesByType(metadata.properties, "part", "partitioning columns") + } + + private def getBucketSpecFromTableProperties(metadata: CatalogTable): Option[BucketSpec] = { + metadata.properties.get(DATASOURCE_SCHEMA_NUMBUCKETS).map { numBuckets => + BucketSpec( + numBuckets.toInt, + getColumnNamesByType(metadata.properties, "bucket", "bucketing columns"), + getColumnNamesByType(metadata.properties, "sort", "sorting columns")) + } + } + + /** + * Detects a data source table. This checks both the table provider and the table properties, + * unlike DDLUtils which just checks the former. + */ + private[spark] def isDatasourceTable(table: CatalogTable): Boolean = { + val provider = table.provider.orElse(table.properties.get(DATASOURCE_PROVIDER)) + provider.isDefined && provider != Some(DDLUtils.HIVE_PROVIDER) + } + +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 36f0708f9da3..4dec2f71b8a5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -17,20 +17,23 @@ package org.apache.spark.sql.hive +import java.lang.reflect.{ParameterizedType, Type, WildcardType} + import scala.collection.JavaConverters._ -import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} -import org.apache.hadoop.hive.serde2.objectinspector.primitive._ +import org.apache.hadoop.{io => hadoopIo} +import org.apache.hadoop.hive.common.`type`.{HiveChar, HiveDecimal, HiveVarchar} +import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.hive.serde2.objectinspector.{StructField => HiveStructField, _} +import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.hive.serde2.typeinfo.{DecimalTypeInfo, TypeInfoFactory} -import org.apache.hadoop.hive.serde2.{io => hiveIo} -import org.apache.hadoop.{io => hadoopIo} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.types import org.apache.spark.sql.types._ -import org.apache.spark.sql.{AnalysisException, types} import org.apache.spark.unsafe.types.UTF8String /** @@ -50,8 +53,8 @@ import org.apache.spark.unsafe.types.UTF8String * java.sql.Date * java.sql.Timestamp * Complex Types => - * Map: [[MapData]] - * List: [[ArrayData]] + * Map: `MapData` + * List: `ArrayData` * Struct: [[org.apache.spark.sql.catalyst.InternalRow]] * Union: NOT SUPPORTED YET * The Complex types plays as a container, which can hold arbitrary data types. @@ -61,6 +64,7 @@ import org.apache.spark.unsafe.types.UTF8String * Primitive Type * Java Boxed Primitives: * org.apache.hadoop.hive.common.type.HiveVarchar + * org.apache.hadoop.hive.common.type.HiveChar * java.lang.String * java.lang.Integer * java.lang.Boolean @@ -75,6 +79,7 @@ import org.apache.spark.unsafe.types.UTF8String * java.sql.Timestamp * Writables: * org.apache.hadoop.hive.serde2.io.HiveVarcharWritable + * org.apache.hadoop.hive.serde2.io.HiveCharWritable * org.apache.hadoop.io.Text * org.apache.hadoop.io.IntWritable * org.apache.hadoop.hive.serde2.io.DoubleWritable @@ -93,7 +98,8 @@ import org.apache.spark.unsafe.types.UTF8String * Struct: Object[] / java.util.List / java POJO * Union: class StandardUnion { byte tag; Object object } * - * NOTICE: HiveVarchar is not supported by catalyst, it will be simply considered as String type. + * NOTICE: HiveVarchar/HiveChar is not supported by catalyst, it will be simply considered as + * String type. * * * 2. Hive ObjectInspector is a group of flexible APIs to inspect value in different data @@ -137,6 +143,7 @@ import org.apache.spark.unsafe.types.UTF8String * Primitive Object Inspectors: * WritableConstantStringObjectInspector * WritableConstantHiveVarcharObjectInspector + * WritableConstantHiveCharObjectInspector * WritableConstantHiveDecimalObjectInspector * WritableConstantTimestampObjectInspector * WritableConstantIntObjectInspector @@ -173,7 +180,7 @@ import org.apache.spark.unsafe.types.UTF8String */ private[hive] trait HiveInspectors { - def javaClassToDataType(clz: Class[_]): DataType = clz match { + def javaTypeToDataType(clz: Type): DataType = clz match { // writable case c: Class[_] if c == classOf[hadoopIo.DoubleWritable] => DoubleType case c: Class[_] if c == classOf[hiveIo.DoubleWritable] => DoubleType @@ -213,385 +220,524 @@ private[hive] trait HiveInspectors { case c: Class[_] if c == java.lang.Float.TYPE => FloatType case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType - case c: Class[_] if c.isArray => ArrayType(javaClassToDataType(c.getComponentType)) + case c: Class[_] if c.isArray => ArrayType(javaTypeToDataType(c.getComponentType)) // Hive seems to return this for struct types? case c: Class[_] if c == classOf[java.lang.Object] => NullType - // java list type unsupported - case c: Class[_] if c == classOf[java.util.List[_]] => + case p: ParameterizedType if isSubClassOf(p.getRawType, classOf[java.util.List[_]]) => + val Array(elementType) = p.getActualTypeArguments + ArrayType(javaTypeToDataType(elementType)) + + case p: ParameterizedType if isSubClassOf(p.getRawType, classOf[java.util.Map[_, _]]) => + val Array(keyType, valueType) = p.getActualTypeArguments + MapType(javaTypeToDataType(keyType), javaTypeToDataType(valueType)) + + // raw java list type unsupported + case c: Class[_] if isSubClassOf(c, classOf[java.util.List[_]]) => + throw new AnalysisException( + "Raw list type in java is unsupported because Spark cannot infer the element type.") + + // raw java map type unsupported + case c: Class[_] if isSubClassOf(c, classOf[java.util.Map[_, _]]) => throw new AnalysisException( - "List type in java is unsupported because " + - "JVM type erasure makes spark fail to catch a component type in List<>") + "Raw map type in java is unsupported because Spark cannot infer key and value types.") - // java map type unsupported - case c: Class[_] if c == classOf[java.util.Map[_, _]] => + case _: WildcardType => throw new AnalysisException( - "Map type in java is unsupported because " + - "JVM type erasure makes spark fail to catch key and value types in Map<>") + "Collection types with wildcards (e.g. List or Map) are unsupported because " + + "Spark cannot infer the data type for these type parameters.") case c => throw new AnalysisException(s"Unsupported java type $c") } - /** - * Converts hive types to native catalyst types. - * @param data the data in Hive type - * @param oi the ObjectInspector associated with the Hive Type - * @return convert the data into catalyst type - * TODO return the function of (data => Any) instead for performance consideration - * - * Strictly follows the following order in unwrapping (constant OI has the higher priority): - * Constant Null object inspector => - * return null - * Constant object inspector => - * extract the value from constant object inspector - * Check whether the `data` is null => - * return null if true - * If object inspector prefers writable => - * extract writable from `data` and then get the catalyst type from the writable - * Extract the java object directly from the object inspector - * - * NOTICE: the complex data type requires recursive unwrapping. - */ - def unwrap(data: Any, oi: ObjectInspector): Any = oi match { - case coi: ConstantObjectInspector if coi.getWritableConstantValue == null => null - case poi: WritableConstantStringObjectInspector => - UTF8String.fromString(poi.getWritableConstantValue.toString) - case poi: WritableConstantHiveVarcharObjectInspector => - UTF8String.fromString(poi.getWritableConstantValue.getHiveVarchar.getValue) - case poi: WritableConstantHiveDecimalObjectInspector => - HiveShim.toCatalystDecimal( - PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector, - poi.getWritableConstantValue.getHiveDecimal) - case poi: WritableConstantTimestampObjectInspector => - val t = poi.getWritableConstantValue - t.getSeconds * 1000000L + t.getNanos / 1000L - case poi: WritableConstantIntObjectInspector => - poi.getWritableConstantValue.get() - case poi: WritableConstantDoubleObjectInspector => - poi.getWritableConstantValue.get() - case poi: WritableConstantBooleanObjectInspector => - poi.getWritableConstantValue.get() - case poi: WritableConstantLongObjectInspector => - poi.getWritableConstantValue.get() - case poi: WritableConstantFloatObjectInspector => - poi.getWritableConstantValue.get() - case poi: WritableConstantShortObjectInspector => - poi.getWritableConstantValue.get() - case poi: WritableConstantByteObjectInspector => - poi.getWritableConstantValue.get() - case poi: WritableConstantBinaryObjectInspector => - val writable = poi.getWritableConstantValue - val temp = new Array[Byte](writable.getLength) - System.arraycopy(writable.getBytes, 0, temp, 0, temp.length) - temp - case poi: WritableConstantDateObjectInspector => - DateTimeUtils.fromJavaDate(poi.getWritableConstantValue.get()) - case mi: StandardConstantMapObjectInspector => - // take the value from the map inspector object, rather than the input data - val keyValues = mi.getWritableConstantValue.asScala.toSeq - val keys = keyValues.map(kv => unwrap(kv._1, mi.getMapKeyObjectInspector)).toArray - val values = keyValues.map(kv => unwrap(kv._2, mi.getMapValueObjectInspector)).toArray - ArrayBasedMapData(keys, values) - case li: StandardConstantListObjectInspector => - // take the value from the list inspector object, rather than the input data - val values = li.getWritableConstantValue.asScala - .map(unwrap(_, li.getListElementObjectInspector)) - .toArray - new GenericArrayData(values) - // if the value is null, we don't care about the object inspector type - case _ if data == null => null - case poi: VoidObjectInspector => null // always be null for void object inspector - case pi: PrimitiveObjectInspector => pi match { - // We think HiveVarchar is also a String - case hvoi: HiveVarcharObjectInspector if hvoi.preferWritable() => - UTF8String.fromString(hvoi.getPrimitiveWritableObject(data).getHiveVarchar.getValue) - case hvoi: HiveVarcharObjectInspector => - UTF8String.fromString(hvoi.getPrimitiveJavaObject(data).getValue) - case x: StringObjectInspector if x.preferWritable() => - UTF8String.fromString(x.getPrimitiveWritableObject(data).toString) - case x: StringObjectInspector => - UTF8String.fromString(x.getPrimitiveJavaObject(data)) - case x: IntObjectInspector if x.preferWritable() => x.get(data) - case x: BooleanObjectInspector if x.preferWritable() => x.get(data) - case x: FloatObjectInspector if x.preferWritable() => x.get(data) - case x: DoubleObjectInspector if x.preferWritable() => x.get(data) - case x: LongObjectInspector if x.preferWritable() => x.get(data) - case x: ShortObjectInspector if x.preferWritable() => x.get(data) - case x: ByteObjectInspector if x.preferWritable() => x.get(data) - case x: HiveDecimalObjectInspector => HiveShim.toCatalystDecimal(x, data) - case x: BinaryObjectInspector if x.preferWritable() => - // BytesWritable.copyBytes() only available since Hadoop2 - // In order to keep backward-compatible, we have to copy the - // bytes with old apis - val bw = x.getPrimitiveWritableObject(data) - val result = new Array[Byte](bw.getLength()) - System.arraycopy(bw.getBytes(), 0, result, 0, bw.getLength()) - result - case x: DateObjectInspector if x.preferWritable() => - DateTimeUtils.fromJavaDate(x.getPrimitiveWritableObject(data).get()) - case x: DateObjectInspector => DateTimeUtils.fromJavaDate(x.getPrimitiveJavaObject(data)) - case x: TimestampObjectInspector if x.preferWritable() => - val t = x.getPrimitiveWritableObject(data) - t.getSeconds * 1000000L + t.getNanos / 1000L - case ti: TimestampObjectInspector => - DateTimeUtils.fromJavaTimestamp(ti.getPrimitiveJavaObject(data)) - case _ => pi.getPrimitiveJavaObject(data) - } - case li: ListObjectInspector => - Option(li.getList(data)) - .map { l => - val values = l.asScala.map(unwrap(_, li.getListElementObjectInspector)).toArray - new GenericArrayData(values) - } - .orNull - case mi: MapObjectInspector => - val map = mi.getMap(data) - if (map == null) { - null - } else { - val keyValues = map.asScala.toSeq - val keys = keyValues.map(kv => unwrap(kv._1, mi.getMapKeyObjectInspector)).toArray - val values = keyValues.map(kv => unwrap(kv._2, mi.getMapValueObjectInspector)).toArray - ArrayBasedMapData(keys, values) - } - // currently, hive doesn't provide the ConstantStructObjectInspector - case si: StructObjectInspector => - val allRefs = si.getAllStructFieldRefs - InternalRow.fromSeq(allRefs.asScala.map( - r => unwrap(si.getStructFieldData(data, r), r.getFieldObjectInspector))) + private def isSubClassOf(t: Type, parent: Class[_]): Boolean = t match { + case cls: Class[_] => parent.isAssignableFrom(cls) + case _ => false } + private def withNullSafe(f: Any => Any): Any => Any = { + input => if (input == null) null else f(input) + } /** * Wraps with Hive types based on object inspector. - * TODO: Consolidate all hive OI/data interface code. */ protected def wrapperFor(oi: ObjectInspector, dataType: DataType): Any => Any = oi match { - case _: JavaHiveVarcharObjectInspector => - (o: Any) => - if (o != null) { - val s = o.asInstanceOf[UTF8String].toString - new HiveVarchar(s, s.size) - } else { - null - } - - case _: JavaHiveDecimalObjectInspector => + case _ if dataType.isInstanceOf[UserDefinedType[_]] => + val sqlType = dataType.asInstanceOf[UserDefinedType[_]].sqlType + wrapperFor(oi, sqlType) + case x: ConstantObjectInspector => (o: Any) => - if (o != null) { - HiveDecimal.create(o.asInstanceOf[Decimal].toJavaBigDecimal) - } else { - null - } - - case _: JavaDateObjectInspector => - (o: Any) => - if (o != null) { - DateTimeUtils.toJavaDate(o.asInstanceOf[Int]) - } else { - null - } - - case _: JavaTimestampObjectInspector => - (o: Any) => - if (o != null) { - DateTimeUtils.toJavaTimestamp(o.asInstanceOf[Long]) - } else { - null + x.getWritableConstantValue + case x: PrimitiveObjectInspector => x match { + // TODO we don't support the HiveVarcharObjectInspector yet. + case _: StringObjectInspector if x.preferWritable() => + withNullSafe(o => getStringWritable(o)) + case _: StringObjectInspector => + withNullSafe(o => o.asInstanceOf[UTF8String].toString()) + case _: IntObjectInspector if x.preferWritable() => + withNullSafe(o => getIntWritable(o)) + case _: IntObjectInspector => + withNullSafe(o => o.asInstanceOf[java.lang.Integer]) + case _: BooleanObjectInspector if x.preferWritable() => + withNullSafe(o => getBooleanWritable(o)) + case _: BooleanObjectInspector => + withNullSafe(o => o.asInstanceOf[java.lang.Boolean]) + case _: FloatObjectInspector if x.preferWritable() => + withNullSafe(o => getFloatWritable(o)) + case _: FloatObjectInspector => + withNullSafe(o => o.asInstanceOf[java.lang.Float]) + case _: DoubleObjectInspector if x.preferWritable() => + withNullSafe(o => getDoubleWritable(o)) + case _: DoubleObjectInspector => + withNullSafe(o => o.asInstanceOf[java.lang.Double]) + case _: LongObjectInspector if x.preferWritable() => + withNullSafe(o => getLongWritable(o)) + case _: LongObjectInspector => + withNullSafe(o => o.asInstanceOf[java.lang.Long]) + case _: ShortObjectInspector if x.preferWritable() => + withNullSafe(o => getShortWritable(o)) + case _: ShortObjectInspector => + withNullSafe(o => o.asInstanceOf[java.lang.Short]) + case _: ByteObjectInspector if x.preferWritable() => + withNullSafe(o => getByteWritable(o)) + case _: ByteObjectInspector => + withNullSafe(o => o.asInstanceOf[java.lang.Byte]) + case _: JavaHiveVarcharObjectInspector => + withNullSafe { o => + val s = o.asInstanceOf[UTF8String].toString + new HiveVarchar(s, s.length) } + case _: JavaHiveCharObjectInspector => + withNullSafe { o => + val s = o.asInstanceOf[UTF8String].toString + new HiveChar(s, s.length) + } + case _: JavaHiveDecimalObjectInspector => + withNullSafe(o => + HiveDecimal.create(o.asInstanceOf[Decimal].toJavaBigDecimal)) + case _: JavaDateObjectInspector => + withNullSafe(o => + DateTimeUtils.toJavaDate(o.asInstanceOf[Int])) + case _: JavaTimestampObjectInspector => + withNullSafe(o => + DateTimeUtils.toJavaTimestamp(o.asInstanceOf[Long])) + case _: HiveDecimalObjectInspector if x.preferWritable() => + withNullSafe(o => getDecimalWritable(o.asInstanceOf[Decimal])) + case _: HiveDecimalObjectInspector => + withNullSafe(o => + HiveDecimal.create(o.asInstanceOf[Decimal].toJavaBigDecimal)) + case _: BinaryObjectInspector if x.preferWritable() => + withNullSafe(o => getBinaryWritable(o)) + case _: BinaryObjectInspector => + withNullSafe(o => o.asInstanceOf[Array[Byte]]) + case _: DateObjectInspector if x.preferWritable() => + withNullSafe(o => getDateWritable(o)) + case _: DateObjectInspector => + withNullSafe(o => DateTimeUtils.toJavaDate(o.asInstanceOf[Int])) + case _: TimestampObjectInspector if x.preferWritable() => + withNullSafe(o => getTimestampWritable(o)) + case _: TimestampObjectInspector => + withNullSafe(o => DateTimeUtils.toJavaTimestamp(o.asInstanceOf[Long])) + case _: VoidObjectInspector => + (_: Any) => null // always be null for void object inspector + } case soi: StandardStructObjectInspector => val schema = dataType.asInstanceOf[StructType] val wrappers = soi.getAllStructFieldRefs.asScala.zip(schema.fields).map { case (ref, field) => wrapperFor(ref.getFieldObjectInspector, field.dataType) } - (o: Any) => { - if (o != null) { - val struct = soi.create() - val row = o.asInstanceOf[InternalRow] - soi.getAllStructFieldRefs.asScala.zip(wrappers).zipWithIndex.foreach { - case ((field, wrapper), i) => - soi.setStructFieldData(struct, field, wrapper(row.get(i, schema(i).dataType))) - } - struct - } else { - null + withNullSafe { o => + val struct = soi.create() + val row = o.asInstanceOf[InternalRow] + soi.getAllStructFieldRefs.asScala.zip(wrappers).zipWithIndex.foreach { + case ((field, wrapper), i) => + soi.setStructFieldData(struct, field, wrapper(row.get(i, schema(i).dataType))) + } + struct + } + + case ssoi: SettableStructObjectInspector => + val structType = dataType.asInstanceOf[StructType] + val wrappers = ssoi.getAllStructFieldRefs.asScala.zip(structType).map { + case (ref, tpe) => wrapperFor(ref.getFieldObjectInspector, tpe.dataType) + } + withNullSafe { o => + val row = o.asInstanceOf[InternalRow] + // 1. create the pojo (most likely) object + val result = ssoi.create() + ssoi.getAllStructFieldRefs.asScala.zip(wrappers).zipWithIndex.foreach { + case ((field, wrapper), i) => + val tpe = structType(i).dataType + ssoi.setStructFieldData( + result, + field, + wrapper(row.get(i, tpe)).asInstanceOf[AnyRef]) } + result + } + + case soi: StructObjectInspector => + val structType = dataType.asInstanceOf[StructType] + val wrappers = soi.getAllStructFieldRefs.asScala.zip(structType).map { + case (ref, tpe) => wrapperFor(ref.getFieldObjectInspector, tpe.dataType) + } + withNullSafe { o => + val row = o.asInstanceOf[InternalRow] + val result = new java.util.ArrayList[AnyRef](wrappers.size) + soi.getAllStructFieldRefs.asScala.zip(wrappers).zipWithIndex.foreach { + case ((field, wrapper), i) => + val tpe = structType(i).dataType + result.add(wrapper(row.get(i, tpe)).asInstanceOf[AnyRef]) + } + result } case loi: ListObjectInspector => val elementType = dataType.asInstanceOf[ArrayType].elementType val wrapper = wrapperFor(loi.getListElementObjectInspector, elementType) - (o: Any) => { - if (o != null) { - val array = o.asInstanceOf[ArrayData] - val values = new java.util.ArrayList[Any](array.numElements()) - array.foreach(elementType, (_, e) => { - values.add(wrapper(e)) - }) - values - } else { - null - } + withNullSafe { o => + val array = o.asInstanceOf[ArrayData] + val values = new java.util.ArrayList[Any](array.numElements()) + array.foreach(elementType, (_, e) => values.add(wrapper(e))) + values } case moi: MapObjectInspector => val mt = dataType.asInstanceOf[MapType] val keyWrapper = wrapperFor(moi.getMapKeyObjectInspector, mt.keyType) val valueWrapper = wrapperFor(moi.getMapValueObjectInspector, mt.valueType) - - (o: Any) => { - if (o != null) { + withNullSafe { o => val map = o.asInstanceOf[MapData] val jmap = new java.util.HashMap[Any, Any](map.numElements()) - map.foreach(mt.keyType, mt.valueType, (k, v) => { - jmap.put(keyWrapper(k), valueWrapper(v)) - }) + map.foreach(mt.keyType, mt.valueType, (k, v) => + jmap.put(keyWrapper(k), valueWrapper(v))) jmap - } else { - null } - } case _ => identity[Any] } /** - * Builds specific unwrappers ahead of time according to object inspector + * Builds unwrappers ahead of time according to object inspector * types to avoid pattern matching and branching costs per row. + * + * Strictly follows the following order in unwrapping (constant OI has the higher priority): + * Constant Null object inspector => + * return null + * Constant object inspector => + * extract the value from constant object inspector + * If object inspector prefers writable => + * extract writable from `data` and then get the catalyst type from the writable + * Extract the java object directly from the object inspector + * + * NOTICE: the complex data type requires recursive unwrapping. + * + * @param objectInspector the ObjectInspector used to create an unwrapper. + * @return A function that unwraps data objects. + * Use the overloaded HiveStructField version for in-place updating of a MutableRow. */ - def unwrapperFor(field: HiveStructField): (Any, MutableRow, Int) => Unit = + def unwrapperFor(objectInspector: ObjectInspector): Any => Any = + objectInspector match { + case coi: ConstantObjectInspector if coi.getWritableConstantValue == null => + _ => null + case poi: WritableConstantStringObjectInspector => + val constant = UTF8String.fromString(poi.getWritableConstantValue.toString) + _ => constant + case poi: WritableConstantHiveVarcharObjectInspector => + val constant = UTF8String.fromString(poi.getWritableConstantValue.getHiveVarchar.getValue) + _ => constant + case poi: WritableConstantHiveCharObjectInspector => + val constant = UTF8String.fromString(poi.getWritableConstantValue.getHiveChar.getValue) + _ => constant + case poi: WritableConstantHiveDecimalObjectInspector => + val constant = HiveShim.toCatalystDecimal( + PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector, + poi.getWritableConstantValue.getHiveDecimal) + _ => constant + case poi: WritableConstantTimestampObjectInspector => + val t = poi.getWritableConstantValue + val constant = t.getSeconds * 1000000L + t.getNanos / 1000L + _ => constant + case poi: WritableConstantIntObjectInspector => + val constant = poi.getWritableConstantValue.get() + _ => constant + case poi: WritableConstantDoubleObjectInspector => + val constant = poi.getWritableConstantValue.get() + _ => constant + case poi: WritableConstantBooleanObjectInspector => + val constant = poi.getWritableConstantValue.get() + _ => constant + case poi: WritableConstantLongObjectInspector => + val constant = poi.getWritableConstantValue.get() + _ => constant + case poi: WritableConstantFloatObjectInspector => + val constant = poi.getWritableConstantValue.get() + _ => constant + case poi: WritableConstantShortObjectInspector => + val constant = poi.getWritableConstantValue.get() + _ => constant + case poi: WritableConstantByteObjectInspector => + val constant = poi.getWritableConstantValue.get() + _ => constant + case poi: WritableConstantBinaryObjectInspector => + val writable = poi.getWritableConstantValue + val constant = new Array[Byte](writable.getLength) + System.arraycopy(writable.getBytes, 0, constant, 0, constant.length) + _ => constant + case poi: WritableConstantDateObjectInspector => + val constant = DateTimeUtils.fromJavaDate(poi.getWritableConstantValue.get()) + _ => constant + case mi: StandardConstantMapObjectInspector => + val keyUnwrapper = unwrapperFor(mi.getMapKeyObjectInspector) + val valueUnwrapper = unwrapperFor(mi.getMapValueObjectInspector) + val keyValues = mi.getWritableConstantValue + val constant = ArrayBasedMapData(keyValues, keyUnwrapper, valueUnwrapper) + _ => constant + case li: StandardConstantListObjectInspector => + val unwrapper = unwrapperFor(li.getListElementObjectInspector) + val values = li.getWritableConstantValue.asScala + .map(unwrapper) + .toArray + val constant = new GenericArrayData(values) + _ => constant + case poi: VoidObjectInspector => + _ => null // always be null for void object inspector + case pi: PrimitiveObjectInspector => pi match { + // We think HiveVarchar/HiveChar is also a String + case hvoi: HiveVarcharObjectInspector if hvoi.preferWritable() => + data: Any => { + if (data != null) { + UTF8String.fromString(hvoi.getPrimitiveWritableObject(data).getHiveVarchar.getValue) + } else { + null + } + } + case hvoi: HiveVarcharObjectInspector => + data: Any => { + if (data != null) { + UTF8String.fromString(hvoi.getPrimitiveJavaObject(data).getValue) + } else { + null + } + } + case hvoi: HiveCharObjectInspector if hvoi.preferWritable() => + data: Any => { + if (data != null) { + UTF8String.fromString(hvoi.getPrimitiveWritableObject(data).getHiveChar.getValue) + } else { + null + } + } + case hvoi: HiveCharObjectInspector => + data: Any => { + if (data != null) { + UTF8String.fromString(hvoi.getPrimitiveJavaObject(data).getValue) + } else { + null + } + } + case x: StringObjectInspector if x.preferWritable() => + data: Any => { + if (data != null) { + // Text is in UTF-8 already. No need to convert again via fromString. Copy bytes + val wObj = x.getPrimitiveWritableObject(data) + val result = wObj.copyBytes() + UTF8String.fromBytes(result, 0, result.length) + } else { + null + } + } + case x: StringObjectInspector => + data: Any => { + if (data != null) { + UTF8String.fromString(x.getPrimitiveJavaObject(data)) + } else { + null + } + } + case x: IntObjectInspector if x.preferWritable() => + data: Any => { + if (data != null) x.get(data) else null + } + case x: BooleanObjectInspector if x.preferWritable() => + data: Any => { + if (data != null) x.get(data) else null + } + case x: FloatObjectInspector if x.preferWritable() => + data: Any => { + if (data != null) x.get(data) else null + } + case x: DoubleObjectInspector if x.preferWritable() => + data: Any => { + if (data != null) x.get(data) else null + } + case x: LongObjectInspector if x.preferWritable() => + data: Any => { + if (data != null) x.get(data) else null + } + case x: ShortObjectInspector if x.preferWritable() => + data: Any => { + if (data != null) x.get(data) else null + } + case x: ByteObjectInspector if x.preferWritable() => + data: Any => { + if (data != null) x.get(data) else null + } + case x: HiveDecimalObjectInspector => + data: Any => { + if (data != null) { + HiveShim.toCatalystDecimal(x, data) + } else { + null + } + } + case x: BinaryObjectInspector if x.preferWritable() => + data: Any => { + if (data != null) { + // BytesWritable.copyBytes() only available since Hadoop2 + // In order to keep backward-compatible, we have to copy the + // bytes with old apis + val bw = x.getPrimitiveWritableObject(data) + val result = new Array[Byte](bw.getLength()) + System.arraycopy(bw.getBytes(), 0, result, 0, bw.getLength()) + result + } else { + null + } + } + case x: DateObjectInspector if x.preferWritable() => + data: Any => { + if (data != null) { + DateTimeUtils.fromJavaDate(x.getPrimitiveWritableObject(data).get()) + } else { + null + } + } + case x: DateObjectInspector => + data: Any => { + if (data != null) { + DateTimeUtils.fromJavaDate(x.getPrimitiveJavaObject(data)) + } else { + null + } + } + case x: TimestampObjectInspector if x.preferWritable() => + data: Any => { + if (data != null) { + val t = x.getPrimitiveWritableObject(data) + t.getSeconds * 1000000L + t.getNanos / 1000L + } else { + null + } + } + case ti: TimestampObjectInspector => + data: Any => { + if (data != null) { + DateTimeUtils.fromJavaTimestamp(ti.getPrimitiveJavaObject(data)) + } else { + null + } + } + case _ => + data: Any => { + if (data != null) { + pi.getPrimitiveJavaObject(data) + } else { + null + } + } + } + case li: ListObjectInspector => + val unwrapper = unwrapperFor(li.getListElementObjectInspector) + data: Any => { + if (data != null) { + Option(li.getList(data)) + .map { l => + val values = l.asScala.map(unwrapper).toArray + new GenericArrayData(values) + } + .orNull + } else { + null + } + } + case mi: MapObjectInspector => + val keyUnwrapper = unwrapperFor(mi.getMapKeyObjectInspector) + val valueUnwrapper = unwrapperFor(mi.getMapValueObjectInspector) + data: Any => { + if (data != null) { + val map = mi.getMap(data) + if (map == null) { + null + } else { + ArrayBasedMapData(map, keyUnwrapper, valueUnwrapper) + } + } else { + null + } + } + // currently, hive doesn't provide the ConstantStructObjectInspector + case si: StructObjectInspector => + val fields = si.getAllStructFieldRefs.asScala + val unwrappers = fields.map { field => + val unwrapper = unwrapperFor(field.getFieldObjectInspector) + data: Any => unwrapper(si.getStructFieldData(data, field)) + } + data: Any => { + if (data != null) { + InternalRow.fromSeq(unwrappers.map(_(data))) + } else { + null + } + } + } + + /** + * Builds unwrappers ahead of time according to object inspector + * types to avoid pattern matching and branching costs per row. + * + * @param field The HiveStructField to create an unwrapper for. + * @return A function that performs in-place updating of a MutableRow. + * Use the overloaded ObjectInspector version for assignments. + */ + def unwrapperFor(field: HiveStructField): (Any, InternalRow, Int) => Unit = field.getFieldObjectInspector match { case oi: BooleanObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setBoolean(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setBoolean(ordinal, oi.get(value)) case oi: ByteObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setByte(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setByte(ordinal, oi.get(value)) case oi: ShortObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setShort(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setShort(ordinal, oi.get(value)) case oi: IntObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setInt(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setInt(ordinal, oi.get(value)) case oi: LongObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setLong(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setLong(ordinal, oi.get(value)) case oi: FloatObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setFloat(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setFloat(ordinal, oi.get(value)) case oi: DoubleObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value)) case oi => - (value: Any, row: MutableRow, ordinal: Int) => row(ordinal) = unwrap(value, oi) - } - - /** - * Converts native catalyst types to the types expected by Hive - * @param a the value to be wrapped - * @param oi This ObjectInspector associated with the value returned by this function, and - * the ObjectInspector should also be consistent with those returned from - * toInspector: DataType => ObjectInspector and - * toInspector: Expression => ObjectInspector - * - * Strictly follows the following order in wrapping (constant OI has the higher priority): - * Constant object inspector => return the bundled value of Constant object inspector - * Check whether the `a` is null => return null if true - * If object inspector prefers writable object => return a Writable for the given data `a` - * Map the catalyst data to the boxed java primitive - * - * NOTICE: the complex data type requires recursive wrapping. - */ - def wrap(a: Any, oi: ObjectInspector, dataType: DataType): AnyRef = oi match { - case x: ConstantObjectInspector => x.getWritableConstantValue - case _ if a == null => null - case x: PrimitiveObjectInspector => x match { - // TODO we don't support the HiveVarcharObjectInspector yet. - case _: StringObjectInspector if x.preferWritable() => getStringWritable(a) - case _: StringObjectInspector => a.asInstanceOf[UTF8String].toString() - case _: IntObjectInspector if x.preferWritable() => getIntWritable(a) - case _: IntObjectInspector => a.asInstanceOf[java.lang.Integer] - case _: BooleanObjectInspector if x.preferWritable() => getBooleanWritable(a) - case _: BooleanObjectInspector => a.asInstanceOf[java.lang.Boolean] - case _: FloatObjectInspector if x.preferWritable() => getFloatWritable(a) - case _: FloatObjectInspector => a.asInstanceOf[java.lang.Float] - case _: DoubleObjectInspector if x.preferWritable() => getDoubleWritable(a) - case _: DoubleObjectInspector => a.asInstanceOf[java.lang.Double] - case _: LongObjectInspector if x.preferWritable() => getLongWritable(a) - case _: LongObjectInspector => a.asInstanceOf[java.lang.Long] - case _: ShortObjectInspector if x.preferWritable() => getShortWritable(a) - case _: ShortObjectInspector => a.asInstanceOf[java.lang.Short] - case _: ByteObjectInspector if x.preferWritable() => getByteWritable(a) - case _: ByteObjectInspector => a.asInstanceOf[java.lang.Byte] - case _: HiveDecimalObjectInspector if x.preferWritable() => - getDecimalWritable(a.asInstanceOf[Decimal]) - case _: HiveDecimalObjectInspector => - HiveDecimal.create(a.asInstanceOf[Decimal].toJavaBigDecimal) - case _: BinaryObjectInspector if x.preferWritable() => getBinaryWritable(a) - case _: BinaryObjectInspector => a.asInstanceOf[Array[Byte]] - case _: DateObjectInspector if x.preferWritable() => getDateWritable(a) - case _: DateObjectInspector => DateTimeUtils.toJavaDate(a.asInstanceOf[Int]) - case _: TimestampObjectInspector if x.preferWritable() => getTimestampWritable(a) - case _: TimestampObjectInspector => DateTimeUtils.toJavaTimestamp(a.asInstanceOf[Long]) + val unwrapper = unwrapperFor(oi) + (value: Any, row: InternalRow, ordinal: Int) => row(ordinal) = unwrapper(value) } - case x: SettableStructObjectInspector => - val fieldRefs = x.getAllStructFieldRefs - val structType = dataType.asInstanceOf[StructType] - val row = a.asInstanceOf[InternalRow] - // 1. create the pojo (most likely) object - val result = x.create() - var i = 0 - while (i < fieldRefs.size) { - // 2. set the property for the pojo - val tpe = structType(i).dataType - x.setStructFieldData( - result, - fieldRefs.get(i), - wrap(row.get(i, tpe), fieldRefs.get(i).getFieldObjectInspector, tpe)) - i += 1 - } - - result - case x: StructObjectInspector => - val fieldRefs = x.getAllStructFieldRefs - val structType = dataType.asInstanceOf[StructType] - val row = a.asInstanceOf[InternalRow] - val result = new java.util.ArrayList[AnyRef](fieldRefs.size) - var i = 0 - while (i < fieldRefs.size) { - val tpe = structType(i).dataType - result.add(wrap(row.get(i, tpe), fieldRefs.get(i).getFieldObjectInspector, tpe)) - i += 1 - } - result - case x: ListObjectInspector => - val list = new java.util.ArrayList[Object] - val tpe = dataType.asInstanceOf[ArrayType].elementType - a.asInstanceOf[ArrayData].foreach(tpe, (_, e) => { - list.add(wrap(e, x.getListElementObjectInspector, tpe)) - }) - list - case x: MapObjectInspector => - val keyType = dataType.asInstanceOf[MapType].keyType - val valueType = dataType.asInstanceOf[MapType].valueType - val map = a.asInstanceOf[MapData] - - // Some UDFs seem to assume we pass in a HashMap. - val hashMap = new java.util.HashMap[Any, Any](map.numElements()) - - map.foreach(keyType, valueType, (k, v) => { - hashMap.put(wrap(k, x.getMapKeyObjectInspector, keyType), - wrap(v, x.getMapValueObjectInspector, valueType)) - }) - - hashMap + def wrap(a: Any, oi: ObjectInspector, dataType: DataType): AnyRef = { + wrapperFor(oi, dataType)(a).asInstanceOf[AnyRef] } def wrap( row: InternalRow, - inspectors: Seq[ObjectInspector], + wrappers: Array[(Any) => Any], cache: Array[AnyRef], dataTypes: Array[DataType]): Array[AnyRef] = { var i = 0 - while (i < inspectors.length) { - cache(i) = wrap(row.get(i, dataTypes(i)), inspectors(i), dataTypes(i)) + val length = wrappers.length + while (i < length) { + cache(i) = wrappers(i)(row.get(i, dataTypes(i))).asInstanceOf[AnyRef] i += 1 } cache @@ -599,12 +745,13 @@ private[hive] trait HiveInspectors { def wrap( row: Seq[Any], - inspectors: Seq[ObjectInspector], + wrappers: Array[(Any) => Any], cache: Array[AnyRef], dataTypes: Array[DataType]): Array[AnyRef] = { var i = 0 - while (i < inspectors.length) { - cache(i) = wrap(row(i), inspectors(i), dataTypes(i)) + val length = wrappers.length + while (i < length) { + cache(i) = wrappers(i)(row(i)).asInstanceOf[AnyRef] i += 1 } cache @@ -643,7 +790,7 @@ private[hive] trait HiveInspectors { /** * Map the catalyst expression to ObjectInspector, however, - * if the expression is [[Literal]] or foldable, a constant writable object inspector returns; + * if the expression is `Literal` or foldable, a constant writable object inspector returns; * Otherwise, we always get the object inspector according to its data type(in catalyst) * @param expr Catalyst expression to be mapped * @return Hive java objectinspector (recursively). @@ -681,9 +828,8 @@ private[hive] trait HiveInspectors { ObjectInspectorFactory.getStandardConstantListObjectInspector(listObjectInspector, null) } else { val list = new java.util.ArrayList[Object]() - value.asInstanceOf[ArrayData].foreach(dt, (_, e) => { - list.add(wrap(e, listObjectInspector, dt)) - }) + value.asInstanceOf[ArrayData].foreach(dt, (_, e) => + list.add(wrap(e, listObjectInspector, dt))) ObjectInspectorFactory.getStandardConstantListObjectInspector(listObjectInspector, list) } case Literal(value, MapType(keyType, valueType, _)) => @@ -695,9 +841,8 @@ private[hive] trait HiveInspectors { val map = value.asInstanceOf[MapData] val jmap = new java.util.HashMap[Any, Any](map.numElements()) - map.foreach(keyType, valueType, (k, v) => { - jmap.put(wrap(k, keyOI, keyType), wrap(v, valueOI, valueType)) - }) + map.foreach(keyType, valueType, (k, v) => + jmap.put(wrap(k, keyOI, keyType), wrap(v, valueOI, valueType))) ObjectInspectorFactory.getStandardConstantMapObjectInspector(keyOI, valueOI, jmap) } 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 f4d45714fae4..f0f2c493498b 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 @@ -17,891 +17,282 @@ package org.apache.spark.sql.hive -import scala.collection.JavaConverters._ -import scala.collection.mutable +import scala.util.control.NonFatal -import com.google.common.base.Objects -import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} +import com.google.common.util.concurrent.Striped import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.common.StatsSetupConst -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.Warehouse -import org.apache.hadoop.hive.metastore.api.FieldSchema -import org.apache.hadoop.hive.ql.metadata._ -import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.analysis.{Catalog, MultiInstanceRelation, OverrideCatalog} -import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} +import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.catalyst.util.DataTypeParser -import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation -import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation, Partition => ParquetPartition, PartitionSpec, ResolvedDataSource} -import org.apache.spark.sql.execution.{FileRelation, datasources} -import org.apache.spark.sql.hive.client._ -import org.apache.spark.sql.hive.execution.HiveNativeCommand -import org.apache.spark.sql.sources._ +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.{AnalysisException, SQLContext, SaveMode} -private[hive] case class HiveSerDe( - inputFormat: Option[String] = None, - outputFormat: Option[String] = None, - serde: Option[String] = None) - -private[hive] object HiveSerDe { - /** - * Get the Hive SerDe information from the data source abbreviation string or classname. - * - * @param source Currently the source abbreviation can be one of the following: - * SequenceFile, RCFile, ORC, PARQUET, and case insensitive. - * @param hiveConf Hive Conf - * @return HiveSerDe associated with the specified source - */ - def sourceToSerDe(source: String, hiveConf: HiveConf): Option[HiveSerDe] = { - val serdeMap = Map( - "sequencefile" -> - HiveSerDe( - inputFormat = Option("org.apache.hadoop.mapred.SequenceFileInputFormat"), - outputFormat = Option("org.apache.hadoop.mapred.SequenceFileOutputFormat")), - - "rcfile" -> - HiveSerDe( - inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat"), - serde = Option(hiveConf.getVar(HiveConf.ConfVars.HIVEDEFAULTRCFILESERDE))), - - "orc" -> - HiveSerDe( - inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"), - serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde")), - - "parquet" -> - HiveSerDe( - inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"), - serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"))) - - val key = source.toLowerCase match { - case s if s.startsWith("org.apache.spark.sql.parquet") => "parquet" - case s if s.startsWith("org.apache.spark.sql.orc") => "orc" - case s => s +/** + * Legacy catalog for interacting with the Hive metastore. + * + * This is still used for things like creating data source tables, but in the future will be + * cleaned up to integrate more nicely with [[HiveExternalCatalog]]. + */ +private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Logging { + // these are def_s and not val/lazy val since the latter would introduce circular references + private def sessionState = sparkSession.sessionState + private def catalogProxy = sparkSession.sessionState.catalog + import HiveMetastoreCatalog._ + + /** These locks guard against multiple attempts to instantiate a table, which wastes memory. */ + private val tableCreationLocks = Striped.lazyWeakLock(100) + + /** Acquires a lock on the table cache for the duration of `f`. */ + private def withTableCreationLock[A](tableName: QualifiedTableName, f: => A): A = { + val lock = tableCreationLocks.get(tableName) + lock.lock() + try f finally { + lock.unlock() } - - serdeMap.get(key) } -} - -private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: HiveContext) - extends Catalog with Logging { - - val conf = hive.conf - - /** Usages should lock on `this`. */ - protected[hive] lazy val hiveWarehouse = new Warehouse(hive.hiveconf) - /** A fully qualified identifier for a table (i.e., database.tableName) */ - case class QualifiedTableName(database: String, name: String) - - private def getQualifiedTableName(tableIdent: TableIdentifier) = { - QualifiedTableName( - tableIdent.database.getOrElse(client.currentDatabase).toLowerCase, - tableIdent.table.toLowerCase) - } - - private def getQualifiedTableName(hiveTable: HiveTable) = { - QualifiedTableName( - hiveTable.specifiedDatabase.getOrElse(client.currentDatabase).toLowerCase, - hiveTable.name.toLowerCase) + // For testing only + private[hive] def getCachedDataSourceTable(table: TableIdentifier): LogicalPlan = { + val key = QualifiedTableName( + table.database.getOrElse(sessionState.catalog.getCurrentDatabase).toLowerCase, + table.table.toLowerCase) + catalogProxy.getCachedTable(key) } - /** A cache of Spark SQL data source tables that have been accessed. */ - protected[hive] val cachedDataSourceTables: LoadingCache[QualifiedTableName, LogicalPlan] = { - val cacheLoader = new CacheLoader[QualifiedTableName, LogicalPlan]() { - override def load(in: QualifiedTableName): LogicalPlan = { - logDebug(s"Creating new cached data source for $in") - val table = client.getTable(in.database, in.name) - - def schemaStringFromParts: Option[String] = { - table.properties.get("spark.sql.sources.schema.numParts").map { numParts => - val parts = (0 until numParts.toInt).map { index => - val part = table.properties.get(s"spark.sql.sources.schema.part.$index").orNull - if (part == null) { - throw new AnalysisException( - "Could not read schema from the metastore because it is corrupted " + - s"(missing part $index of the schema, $numParts parts are expected).") - } - - part - } - // Stick all parts back to a single schema string. - parts.mkString - } - } - - def partColsFromParts: Option[Seq[String]] = { - table.properties.get("spark.sql.sources.schema.numPartCols").map { numPartCols => - (0 until numPartCols.toInt).map { index => - val partCol = table.properties.get(s"spark.sql.sources.schema.partCol.$index").orNull - if (partCol == null) { - throw new AnalysisException( - "Could not read partitioned columns from the metastore because it is corrupted " + - s"(missing part $index of the it, $numPartCols parts are expected).") - } - - partCol + private def getCached( + tableIdentifier: QualifiedTableName, + pathsInMetastore: Seq[Path], + schemaInMetastore: StructType, + expectedFileFormat: Class[_ <: FileFormat], + partitionSchema: Option[StructType]): Option[LogicalRelation] = { + + catalogProxy.getCachedTable(tableIdentifier) match { + case null => None // Cache miss + case logical @ LogicalRelation(relation: HadoopFsRelation, _, _, _) => + val cachedRelationFileFormatClass = relation.fileFormat.getClass + + expectedFileFormat match { + case `cachedRelationFileFormatClass` => + // If we have the same paths, same schema, and same partition spec, + // we will use the cached relation. + val useCached = + relation.location.rootPaths.toSet == pathsInMetastore.toSet && + logical.schema.sameType(schemaInMetastore) && + // We don't support hive bucketed tables. This function `getCached` is only used for + // converting supported Hive tables to data source tables. + relation.bucketSpec.isEmpty && + relation.partitionSchema == partitionSchema.getOrElse(StructType(Nil)) + + if (useCached) { + Some(logical) + } else { + // If the cached relation is not updated, we invalidate it right away. + catalogProxy.invalidateCachedTable(tableIdentifier) + None } - } + case _ => + logWarning(s"Table $tableIdentifier should be stored as $expectedFileFormat. " + + s"However, we are getting a ${relation.fileFormat} from the metastore cache. " + + "This cached entry will be invalidated.") + catalogProxy.invalidateCachedTable(tableIdentifier) + None } - - // Originally, we used spark.sql.sources.schema to store the schema of a data source table. - // After SPARK-6024, we removed this flag. - // Although we are not using spark.sql.sources.schema any more, we need to still support. - val schemaString = - table.properties.get("spark.sql.sources.schema").orElse(schemaStringFromParts) - - val userSpecifiedSchema = - schemaString.map(s => DataType.fromJson(s).asInstanceOf[StructType]) - - // We only need names at here since userSpecifiedSchema we loaded from the metastore - // contains partition columns. We can always get datatypes of partitioning columns - // from userSpecifiedSchema. - val partitionColumns = partColsFromParts.getOrElse(Nil) - - // It does not appear that the ql client for the metastore has a way to enumerate all the - // SerDe properties directly... - val options = table.serdeProperties - - val resolvedRelation = - ResolvedDataSource( - hive, - userSpecifiedSchema, - partitionColumns.toArray, - table.properties("spark.sql.sources.provider"), - options) - - LogicalRelation(resolvedRelation.relation) - } + case other => + logWarning(s"Table $tableIdentifier should be stored as $expectedFileFormat. " + + s"However, we are getting a $other from the metastore cache. " + + "This cached entry will be invalidated.") + catalogProxy.invalidateCachedTable(tableIdentifier) + None } - - CacheBuilder.newBuilder().maximumSize(1000).build(cacheLoader) - } - - override def refreshTable(tableIdent: TableIdentifier): Unit = { - // refreshTable does not eagerly reload the cache. It just invalidate the cache. - // Next time when we use the table, it will be populated in the cache. - // Since we also cache ParquetRelations converted from Hive Parquet tables and - // adding converted ParquetRelations into the cache is not defined in the load function - // of the cache (instead, we add the cache entry in convertToParquetRelation), - // it is better at here to invalidate the cache to avoid confusing waring logs from the - // cache loader (e.g. cannot find data source provider, which is only defined for - // data source table.). - invalidateTable(tableIdent) - } - - def invalidateTable(tableIdent: TableIdentifier): Unit = { - cachedDataSourceTables.invalidate(getQualifiedTableName(tableIdent)) } - def createDataSourceTable( - tableIdent: TableIdentifier, - userSpecifiedSchema: Option[StructType], - partitionColumns: Array[String], - provider: String, + def convertToLogicalRelation( + relation: HiveTableRelation, options: Map[String, String], - isExternal: Boolean): Unit = { - val QualifiedTableName(dbName, tblName) = getQualifiedTableName(tableIdent) - - val tableProperties = new mutable.HashMap[String, String] - tableProperties.put("spark.sql.sources.provider", provider) - - // Saves optional user specified schema. Serialized JSON schema string may be too long to be - // stored into a single metastore SerDe property. In this case, we split the JSON string and - // store each part as a separate SerDe property. - userSpecifiedSchema.foreach { schema => - val threshold = conf.schemaStringLengthThreshold - val schemaJsonString = schema.json - // Split the JSON string. - val parts = schemaJsonString.grouped(threshold).toSeq - tableProperties.put("spark.sql.sources.schema.numParts", parts.size.toString) - parts.zipWithIndex.foreach { case (part, index) => - tableProperties.put(s"spark.sql.sources.schema.part.$index", part) - } - } - - if (userSpecifiedSchema.isDefined && partitionColumns.length > 0) { - tableProperties.put("spark.sql.sources.schema.numPartCols", partitionColumns.length.toString) - partitionColumns.zipWithIndex.foreach { case (partCol, index) => - tableProperties.put(s"spark.sql.sources.schema.partCol.$index", partCol) - } - } - - if (userSpecifiedSchema.isEmpty && partitionColumns.length > 0) { - // The table does not have a specified schema, which means that the schema will be inferred - // when we load the table. So, we are not expecting partition columns and we will discover - // partitions when we load the table. However, if there are specified partition columns, - // we simply ignore them and provide a warning message. - logWarning( - s"The schema and partitions of table $tableIdent will be inferred when it is loaded. " + - s"Specified partition columns (${partitionColumns.mkString(",")}) will be ignored.") - } - - val tableType = if (isExternal) { - tableProperties.put("EXTERNAL", "TRUE") - ExternalTable - } else { - tableProperties.put("EXTERNAL", "FALSE") - ManagedTable - } - - val maybeSerDe = HiveSerDe.sourceToSerDe(provider, hive.hiveconf) - val dataSource = ResolvedDataSource( - hive, userSpecifiedSchema, partitionColumns, provider, options) - - def newSparkSQLSpecificMetastoreTable(): HiveTable = { - HiveTable( - specifiedDatabase = Option(dbName), - name = tblName, - schema = Nil, - partitionColumns = Nil, - tableType = tableType, - properties = tableProperties.toMap, - serdeProperties = options) - } - - def newHiveCompatibleMetastoreTable(relation: HadoopFsRelation, serde: HiveSerDe): HiveTable = { - def schemaToHiveColumn(schema: StructType): Seq[HiveColumn] = { - schema.map { field => - HiveColumn( - name = field.name, - hiveType = HiveMetastoreTypes.toMetastoreType(field.dataType), - comment = "") - } - } - - assert(partitionColumns.isEmpty) - assert(relation.partitionColumns.isEmpty) - - HiveTable( - specifiedDatabase = Option(dbName), - name = tblName, - schema = schemaToHiveColumn(relation.schema), - partitionColumns = Nil, - tableType = tableType, - properties = tableProperties.toMap, - serdeProperties = options, - location = Some(relation.paths.head), - viewText = None, // TODO We need to place the SQL string here. - inputFormat = serde.inputFormat, - outputFormat = serde.outputFormat, - serde = serde.serde) - } - - // TODO: Support persisting partitioned data source relations in Hive compatible format - val qualifiedTableName = tableIdent.quotedString - val (hiveCompatibleTable, logMessage) = (maybeSerDe, dataSource.relation) match { - case (Some(serde), relation: HadoopFsRelation) - if relation.paths.length == 1 && relation.partitionColumns.isEmpty => - val hiveTable = newHiveCompatibleMetastoreTable(relation, serde) - val message = - s"Persisting data source relation $qualifiedTableName with a single input path " + - s"into Hive metastore in Hive compatible format. Input path: ${relation.paths.head}." - (Some(hiveTable), message) - - case (Some(serde), relation: HadoopFsRelation) if relation.partitionColumns.nonEmpty => - val message = - s"Persisting partitioned data source relation $qualifiedTableName into " + - "Hive metastore in Spark SQL specific format, which is NOT compatible with Hive. " + - "Input path(s): " + relation.paths.mkString("\n", "\n", "") - (None, message) - - case (Some(serde), relation: HadoopFsRelation) => - val message = - s"Persisting data source relation $qualifiedTableName with multiple input paths into " + - "Hive metastore in Spark SQL specific format, which is NOT compatible with Hive. " + - s"Input paths: " + relation.paths.mkString("\n", "\n", "") - (None, message) - - case (Some(serde), _) => - val message = - s"Data source relation $qualifiedTableName is not a " + - s"${classOf[HadoopFsRelation].getSimpleName}. Persisting it into Hive metastore " + - "in Spark SQL specific format, which is NOT compatible with Hive." - (None, message) + fileFormatClass: Class[_ <: FileFormat], + fileType: String): LogicalRelation = { + val metastoreSchema = relation.tableMeta.schema + val tableIdentifier = + QualifiedTableName(relation.tableMeta.database, relation.tableMeta.identifier.table) - case _ => - val message = - s"Couldn't find corresponding Hive SerDe for data source provider $provider. " + - s"Persisting data source relation $qualifiedTableName into Hive metastore in " + - s"Spark SQL specific format, which is NOT compatible with Hive." - (None, message) - } + val lazyPruningEnabled = sparkSession.sqlContext.conf.manageFilesourcePartitions + val tablePath = new Path(relation.tableMeta.location) + val fileFormat = fileFormatClass.newInstance() - (hiveCompatibleTable, logMessage) match { - case (Some(table), message) => - // We first try to save the metadata of the table in a Hive compatible way. - // If Hive throws an error, we fall back to save its metadata in the Spark SQL - // specific way. - try { - logInfo(message) - client.createTable(table) - } catch { - case throwable: Throwable => - val warningMessage = - s"Could not persist $qualifiedTableName in a Hive compatible way. Persisting " + - s"it into Hive metastore in Spark SQL specific format." - logWarning(warningMessage, throwable) - val sparkSqlSpecificTable = newSparkSQLSpecificMetastoreTable() - client.createTable(sparkSqlSpecificTable) + val result = if (relation.isPartitioned) { + val partitionSchema = relation.tableMeta.partitionSchema + val rootPaths: Seq[Path] = if (lazyPruningEnabled) { + Seq(tablePath) + } else { + // By convention (for example, see CatalogFileIndex), the definition of a + // partitioned table's paths depends on whether that table has any actual partitions. + // Partitioned tables without partitions use the location of the table's base path. + // Partitioned tables with partitions use the locations of those partitions' data + // locations,_omitting_ the table's base path. + val paths = sparkSession.sharedState.externalCatalog + .listPartitions(tableIdentifier.database, tableIdentifier.name) + .map(p => new Path(p.storage.locationUri.get)) + + if (paths.isEmpty) { + Seq(tablePath) + } else { + paths } - - case (None, message) => - logWarning(message) - val hiveTable = newSparkSQLSpecificMetastoreTable() - client.createTable(hiveTable) - } - } - - def hiveDefaultTableFilePath(tableIdent: TableIdentifier): String = { - // Code based on: hiveWarehouse.getTablePath(currentDatabase, tableName) - val QualifiedTableName(dbName, tblName) = getQualifiedTableName(tableIdent) - new Path(new Path(client.getDatabase(dbName).location), tblName).toString - } - - override def tableExists(tableIdent: TableIdentifier): Boolean = { - val QualifiedTableName(dbName, tblName) = getQualifiedTableName(tableIdent) - client.getTableOption(dbName, tblName).isDefined - } - - override def lookupRelation( - tableIdent: TableIdentifier, - alias: Option[String]): LogicalPlan = { - val qualifiedTableName = getQualifiedTableName(tableIdent) - val table = client.getTable(qualifiedTableName.database, qualifiedTableName.name) - - if (table.properties.get("spark.sql.sources.provider").isDefined) { - val dataSourceTable = cachedDataSourceTables(qualifiedTableName) - val tableWithQualifiers = Subquery(qualifiedTableName.name, dataSourceTable) - // Then, if alias is specified, wrap the table with a Subquery using the alias. - // Otherwise, wrap the table with a Subquery using the table name. - alias.map(a => Subquery(a, tableWithQualifiers)).getOrElse(tableWithQualifiers) - } else if (table.tableType == VirtualView) { - val viewText = table.viewText.getOrElse(sys.error("Invalid view without text.")) - alias match { - // because hive use things like `_c0` to build the expanded text - // currently we cannot support view from "create view v1(c1) as ..." - case None => Subquery(table.name, HiveQl.createPlan(viewText)) - case Some(aliasText) => Subquery(aliasText, HiveQl.createPlan(viewText)) } - } else { - MetastoreRelation(qualifiedTableName.database, qualifiedTableName.name, alias)(table)(hive) - } - } - - private def convertToParquetRelation(metastoreRelation: MetastoreRelation): LogicalRelation = { - val metastoreSchema = StructType.fromAttributes(metastoreRelation.output) - val mergeSchema = hive.convertMetastoreParquetWithSchemaMerging - // NOTE: Instead of passing Metastore schema directly to `ParquetRelation`, 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 `ParquetRelation`. - val parquetOptions = Map( - ParquetRelation.METASTORE_SCHEMA -> metastoreSchema.json, - ParquetRelation.MERGE_SCHEMA -> mergeSchema.toString) - val tableIdentifier = - QualifiedTableName(metastoreRelation.databaseName, metastoreRelation.tableName) - - def getCached( - tableIdentifier: QualifiedTableName, - pathsInMetastore: Seq[String], - schemaInMetastore: StructType, - partitionSpecInMetastore: Option[PartitionSpec]): Option[LogicalRelation] = { - cachedDataSourceTables.getIfPresent(tableIdentifier) match { - case null => None // Cache miss - case logical @ LogicalRelation(parquetRelation: ParquetRelation, _) => - // If we have the same paths, same schema, and same partition spec, - // we will use the cached Parquet Relation. - val useCached = - parquetRelation.paths.toSet == pathsInMetastore.toSet && - logical.schema.sameType(metastoreSchema) && - parquetRelation.partitionSpec == partitionSpecInMetastore.getOrElse { - PartitionSpec(StructType(Nil), Array.empty[datasources.Partition]) + withTableCreationLock(tableIdentifier, { + val cached = getCached( + tableIdentifier, + rootPaths, + metastoreSchema, + fileFormatClass, + Some(partitionSchema)) + + val logicalRelation = cached.getOrElse { + val sizeInBytes = relation.stats.sizeInBytes.toLong + val fileIndex = { + val index = new CatalogFileIndex(sparkSession, relation.tableMeta, sizeInBytes) + if (lazyPruningEnabled) { + index + } else { + index.filterPartitions(Nil) // materialize all the partitions in memory } - - if (useCached) { - Some(logical) - } else { - // If the cached relation is not updated, we invalidate it right away. - cachedDataSourceTables.invalidate(tableIdentifier) - None } - case other => - logWarning( - s"${metastoreRelation.databaseName}.${metastoreRelation.tableName} should be stored " + - s"as Parquet. However, we are getting a $other from the metastore cache. " + - s"This cached entry will be invalidated.") - cachedDataSourceTables.invalidate(tableIdentifier) - None - } - } - val result = if (metastoreRelation.hiveQlTable.isPartitioned) { - val partitionSchema = StructType.fromAttributes(metastoreRelation.partitionKeys) - val partitionColumnDataTypes = partitionSchema.map(_.dataType) - // We're converting the entire table into ParquetRelation, so predicates to Hive metastore - // are empty. - val partitions = metastoreRelation.getHiveQlPartitions().map { p => - val location = p.getLocation - val values = InternalRow.fromSeq(p.getValues.asScala.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) - - val cached = getCached(tableIdentifier, paths, metastoreSchema, Some(partitionSpec)) - val parquetRelation = cached.getOrElse { - val created = LogicalRelation( - new ParquetRelation( - paths.toArray, None, Some(partitionSpec), parquetOptions)(hive)) - cachedDataSourceTables.put(tableIdentifier, created) - created - } + val (dataSchema, updatedTable) = + inferIfNeeded(relation, options, fileFormat, Option(fileIndex)) + + val fsRelation = HadoopFsRelation( + location = fileIndex, + partitionSchema = partitionSchema, + dataSchema = dataSchema, + bucketSpec = None, + fileFormat = fileFormat, + options = options)(sparkSession = sparkSession) + val created = LogicalRelation(fsRelation, updatedTable) + catalogProxy.cacheTable(tableIdentifier, created) + created + } - parquetRelation + logicalRelation + }) } else { - val paths = Seq(metastoreRelation.hiveQlTable.getDataLocation.toString) - - val cached = getCached(tableIdentifier, paths, metastoreSchema, None) - val parquetRelation = cached.getOrElse { - val created = LogicalRelation( - new ParquetRelation(paths.toArray, None, None, parquetOptions)(hive)) - cachedDataSourceTables.put(tableIdentifier, created) - created - } + val rootPath = tablePath + withTableCreationLock(tableIdentifier, { + val cached = getCached( + tableIdentifier, + Seq(rootPath), + metastoreSchema, + fileFormatClass, + None) + val logicalRelation = cached.getOrElse { + val (dataSchema, updatedTable) = inferIfNeeded(relation, options, fileFormat) + val created = + LogicalRelation( + DataSource( + sparkSession = sparkSession, + paths = rootPath.toString :: Nil, + userSpecifiedSchema = Option(dataSchema), + bucketSpec = None, + options = options, + className = fileType).resolveRelation(), + table = updatedTable) + + catalogProxy.cacheTable(tableIdentifier, created) + created + } - parquetRelation + logicalRelation + }) } - - result.copy(expectedOutputAttributes = Some(metastoreRelation.output)) - } - - override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { - val db = databaseName.getOrElse(client.currentDatabase) - - client.listTables(db).map(tableName => (tableName, false)) - } - - /** - * When scanning or writing to non-partitioned Metastore Parquet tables, convert them to Parquet - * data source relations for better performance. - */ - object ParquetConversions extends Rule[LogicalPlan] { - override def apply(plan: LogicalPlan): LogicalPlan = { - if (!plan.resolved || plan.analyzed) { - return plan - } - - plan transformUp { - // Write path - case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists) - // Inserting into partitioned table is not supported in Parquet data source (yet). - if !r.hiveQlTable.isPartitioned && hive.convertMetastoreParquet && - r.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => - val parquetRelation = convertToParquetRelation(r) - InsertIntoTable(parquetRelation, partition, child, overwrite, ifNotExists) - - // Write path - case InsertIntoHiveTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists) - // Inserting into partitioned table is not supported in Parquet data source (yet). - if !r.hiveQlTable.isPartitioned && hive.convertMetastoreParquet && - r.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => - val parquetRelation = convertToParquetRelation(r) - InsertIntoTable(parquetRelation, partition, child, overwrite, ifNotExists) - - // Read path - case relation: MetastoreRelation if hive.convertMetastoreParquet && - relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => - val parquetRelation = convertToParquetRelation(relation) - Subquery(relation.alias.getOrElse(relation.tableName), parquetRelation) - } + // The inferred schema may have different field names as the table schema, we should respect + // it, but also respect the exprId in table relation output. + assert(result.output.length == relation.output.length && + result.output.zip(relation.output).forall { case (a1, a2) => a1.dataType == a2.dataType }) + val newOutput = result.output.zip(relation.output).map { + case (a1, a2) => a1.withExprId(a2.exprId) } + result.copy(output = newOutput) } - /** - * Creates any tables required for query execution. - * For example, because of a CREATE TABLE X AS statement. - */ - object CreateTables extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - // Wait until children are resolved. - case p: LogicalPlan if !p.childrenResolved => p - case p: LogicalPlan if p.resolved => p - - case CreateViewAsSelect(table, child, allowExisting, replace, sql) => - if (conf.nativeView) { - if (allowExisting && replace) { - throw new AnalysisException( - "It is not allowed to define a view with both IF NOT EXISTS and OR REPLACE.") - } - - val QualifiedTableName(dbName, tblName) = getQualifiedTableName(table) - - execution.CreateViewAsSelect( - table.copy( - specifiedDatabase = Some(dbName), - name = tblName), - child.output, - allowExisting, - replace) - } else { - HiveNativeCommand(sql) - } - - case p @ CreateTableAsSelect(table, child, allowExisting) => - val schema = if (table.schema.nonEmpty) { - table.schema - } else { - child.output.map { - attr => new HiveColumn( - attr.name, - HiveMetastoreTypes.toMetastoreType(attr.dataType), null) - } - } - - val desc = table.copy(schema = schema) - - if (hive.convertCTAS && table.serde.isEmpty) { - // 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 (table.specifiedDatabase.isDefined) { - throw new AnalysisException( - "Cannot specify database name in a CTAS statement " + - "when spark.sql.hive.convertCTAS is set to true.") - } + private def inferIfNeeded( + relation: HiveTableRelation, + options: Map[String, String], + fileFormat: FileFormat, + fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = { + val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode + val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase + val tableName = relation.tableMeta.identifier.unquotedString + if (shouldInfer) { + logInfo(s"Inferring case-sensitive schema for table $tableName (inference mode: " + + s"$inferenceMode)") + val fileIndex = fileIndexOpt.getOrElse { + val rootPath = new Path(relation.tableMeta.location) + new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None) + } - val mode = if (allowExisting) SaveMode.Ignore else SaveMode.ErrorIfExists - CreateTableUsingAsSelect( - TableIdentifier(desc.name), - conf.defaultDataSourceName, - temporary = false, - Array.empty[String], - mode, - options = Map.empty[String, String], - child - ) - } else { - val desc = if (table.serde.isEmpty) { - // add default serde - table.copy( - serde = Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) - } else { - table + val inferredSchema = fileFormat + .inferSchema( + sparkSession, + options, + fileIndex.listFiles(Nil, Nil).flatMap(_.files)) + .map(mergeWithMetastoreSchema(relation.tableMeta.schema, _)) + + inferredSchema match { + case Some(schema) => + if (inferenceMode == INFER_AND_SAVE) { + updateCatalogSchema(relation.tableMeta.identifier, schema) } - - val QualifiedTableName(dbName, tblName) = getQualifiedTableName(table) - - execution.CreateTableAsSelect( - desc.copy( - specifiedDatabase = Some(dbName), - name = tblName), - child, - allowExisting) - } - } - } - - /** - * Casts input data to correct data types according to table definition before inserting into - * that table. - */ - object PreInsertionCasts extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan.transform { - // Wait until children are resolved. - case p: LogicalPlan if !p.childrenResolved => p - - case p @ InsertIntoTable(table: MetastoreRelation, _, child, _, _) => - castChildOutput(p, table, child) - } - - def castChildOutput(p: InsertIntoTable, table: MetastoreRelation, child: LogicalPlan) - : LogicalPlan = { - val childOutputDataTypes = child.output.map(_.dataType) - val numDynamicPartitions = p.partition.values.count(_.isEmpty) - val tableOutputDataTypes = - (table.attributes ++ table.partitionKeys.takeRight(numDynamicPartitions)) - .take(child.output.length).map(_.dataType) - - if (childOutputDataTypes == tableOutputDataTypes) { - InsertIntoHiveTable(table, p.partition, p.child, p.overwrite, p.ifNotExists) - } else if (childOutputDataTypes.size == tableOutputDataTypes.size && - childOutputDataTypes.zip(tableOutputDataTypes) - .forall { case (left, right) => left.sameType(right) }) { - // If both types ignoring nullability of ArrayType, MapType, StructType are the same, - // use InsertIntoHiveTable instead of InsertIntoTable. - InsertIntoHiveTable(table, p.partition, p.child, p.overwrite, p.ifNotExists) - } else { - // Only do the casting when child output data types differ from table output data types. - val castedChildOutput = child.output.zip(table.output).map { - case (input, output) if input.dataType != output.dataType => - Alias(Cast(input, output.dataType), input.name)() - case (input, _) => input - } - - p.copy(child = logical.Project(castedChildOutput, child)) + (schema, relation.tableMeta.copy(schema = schema)) + case None => + logWarning(s"Unable to infer schema for table $tableName from file format " + + s"$fileFormat (inference mode: $inferenceMode). Using metastore schema.") + (relation.tableMeta.schema, relation.tableMeta) } - } - } - - /** - * UNIMPLEMENTED: It needs to be decided how we will persist in-memory tables to the metastore. - * For now, if this functionality is desired mix in the in-memory [[OverrideCatalog]]. - */ - override def registerTable(tableIdent: TableIdentifier, plan: LogicalPlan): Unit = { - throw new UnsupportedOperationException - } - - /** - * UNIMPLEMENTED: It needs to be decided how we will persist in-memory tables to the metastore. - * For now, if this functionality is desired mix in the in-memory [[OverrideCatalog]]. - */ - override def unregisterTable(tableIdent: TableIdentifier): Unit = { - throw new UnsupportedOperationException - } - - override def unregisterAllTables(): Unit = {} -} - -/** - * A logical plan representing insertion into Hive table. - * This plan ignores nullability of ArrayType, MapType, StructType unlike InsertIntoTable - * because Hive table doesn't have nullability for ARRAY, MAP, STRUCT types. - */ -private[hive] case class InsertIntoHiveTable( - table: MetastoreRelation, - partition: Map[String, Option[String]], - child: LogicalPlan, - overwrite: Boolean, - ifNotExists: Boolean) - extends LogicalPlan { - - override def children: Seq[LogicalPlan] = child :: Nil - override def output: Seq[Attribute] = Seq.empty - - val numDynamicPartitions = partition.values.count(_.isEmpty) - - // This is the expected schema of the table prepared to be inserted into, - // including dynamic partition columns. - val tableOutput = table.attributes ++ table.partitionKeys.takeRight(numDynamicPartitions) - - override lazy val resolved: Boolean = childrenResolved && child.output.zip(tableOutput).forall { - case (childAttr, tableAttr) => childAttr.dataType.sameType(tableAttr.dataType) - } -} - -private[hive] case class MetastoreRelation - (databaseName: String, tableName: String, alias: Option[String]) - (val table: HiveTable) - (@transient private val sqlContext: SQLContext) - extends LeafNode with MultiInstanceRelation with FileRelation { - - override def equals(other: Any): Boolean = other match { - case relation: MetastoreRelation => - databaseName == relation.databaseName && - tableName == relation.tableName && - alias == relation.alias && - output == relation.output - case _ => false - } - - override def hashCode(): Int = { - Objects.hashCode(databaseName, tableName, alias, output) - } - - @transient val hiveQlTable: Table = { - // We start by constructing an API table as Hive performs several important transformations - // internally when converting an API table to a QL table. - val tTable = new org.apache.hadoop.hive.metastore.api.Table() - tTable.setTableName(table.name) - tTable.setDbName(table.database) - - val tableParameters = new java.util.HashMap[String, String]() - tTable.setParameters(tableParameters) - table.properties.foreach { case (k, v) => tableParameters.put(k, v) } - - tTable.setTableType(table.tableType.name) - - val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() - tTable.setSd(sd) - sd.setCols(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) - tTable.setPartitionKeys( - table.partitionColumns.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) - - table.location.foreach(sd.setLocation) - table.inputFormat.foreach(sd.setInputFormat) - table.outputFormat.foreach(sd.setOutputFormat) - - val serdeInfo = new org.apache.hadoop.hive.metastore.api.SerDeInfo - table.serde.foreach(serdeInfo.setSerializationLib) - sd.setSerdeInfo(serdeInfo) - - val serdeParameters = new java.util.HashMap[String, String]() - table.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } - serdeInfo.setParameters(serdeParameters) - - new Table(tTable) - } - - @transient override lazy val statistics: Statistics = Statistics( - sizeInBytes = { - val totalSize = hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE) - val rawDataSize = hiveQlTable.getParameters.get(StatsSetupConst.RAW_DATA_SIZE) - // TODO: check if this estimate is valid for tables after partition pruning. - // NOTE: getting `totalSize` directly from params is kind of hacky, but this should be - // relatively cheap if parameters for the table are populated into the metastore. An - // alternative would be going through Hadoop's FileSystem API, which can be expensive if a lot - // of RPCs are involved. Besides `totalSize`, there are also `numFiles`, `numRows`, - // `rawDataSize` keys (see StatsSetupConst in Hive) that we can look at in the future. - BigInt( - // When table is external,`totalSize` is always zero, which will influence join strategy - // so when `totalSize` is zero, use `rawDataSize` instead - // if the size is still less than zero, we use default size - Option(totalSize).map(_.toLong).filter(_ > 0) - .getOrElse(Option(rawDataSize).map(_.toLong).filter(_ > 0) - .getOrElse(sqlContext.conf.defaultSizeInBytes))) - } - ) - - // When metastore partition pruning is turned off, we cache the list of all partitions to - // mimic the behavior of Spark < 1.5 - lazy val allPartitions = table.getAllPartitions - - def getHiveQlPartitions(predicates: Seq[Expression] = Nil): Seq[Partition] = { - val rawPartitions = if (sqlContext.conf.metastorePartitionPruning) { - table.getPartitions(predicates) - } else { - allPartitions - } - - rawPartitions.map { p => - val tPartition = new org.apache.hadoop.hive.metastore.api.Partition - tPartition.setDbName(databaseName) - tPartition.setTableName(tableName) - tPartition.setValues(p.values.asJava) - - val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() - tPartition.setSd(sd) - sd.setCols(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) - - sd.setLocation(p.storage.location) - sd.setInputFormat(p.storage.inputFormat) - sd.setOutputFormat(p.storage.outputFormat) - - val serdeInfo = new org.apache.hadoop.hive.metastore.api.SerDeInfo - sd.setSerdeInfo(serdeInfo) - serdeInfo.setSerializationLib(p.storage.serde) - - val serdeParameters = new java.util.HashMap[String, String]() - serdeInfo.setParameters(serdeParameters) - table.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } - p.storage.serdeProperties.foreach { case (k, v) => serdeParameters.put(k, v) } - - new Partition(hiveQlTable, tPartition) - } - } - - /** Only compare database and tablename, not alias. */ - override def sameResult(plan: LogicalPlan): Boolean = { - plan match { - case mr: MetastoreRelation => - mr.databaseName == databaseName && mr.tableName == tableName - case _ => false - } - } - - val tableDesc = new TableDesc( - hiveQlTable.getInputFormatClass, - // The class of table should be org.apache.hadoop.hive.ql.metadata.Table because - // getOutputFormatClass will use HiveFileFormatUtils.getOutputFormatSubstitute to - // substitute some output formats, e.g. substituting SequenceFileOutputFormat to - // HiveSequenceFileOutputFormat. - hiveQlTable.getOutputFormatClass, - hiveQlTable.getMetadata - ) - - implicit class SchemaAttribute(f: HiveColumn) { - def toAttribute: AttributeReference = AttributeReference( - f.name, - HiveMetastoreTypes.toDataType(f.hiveType), - // Since data can be dumped in randomly with no validation, everything is nullable. - nullable = true - )(qualifiers = Seq(alias.getOrElse(tableName))) - } - - /** PartitionKey attributes */ - val partitionKeys = table.partitionColumns.map(_.toAttribute) - - /** Non-partitionKey attributes */ - val attributes = table.schema.map(_.toAttribute) - - val output = attributes ++ partitionKeys - - /** An attribute map that can be used to lookup original attributes based on expression id. */ - val attributeMap = AttributeMap(output.map(o => (o, o))) - - /** An attribute map for determining the ordinal for non-partition columns. */ - val columnOrdinals = AttributeMap(attributes.zipWithIndex) - - override def inputFiles: Array[String] = { - val partLocations = table.getPartitions(Nil).map(_.storage.location).toArray - if (partLocations.nonEmpty) { - partLocations } else { - Array( - table.location.getOrElse( - sys.error(s"Could not get the location of ${table.qualifiedName}."))) + (relation.tableMeta.schema, relation.tableMeta) } } - - override def newInstance(): MetastoreRelation = { - MetastoreRelation(databaseName, tableName, alias)(table)(sqlContext) + private def updateCatalogSchema(identifier: TableIdentifier, schema: StructType): Unit = try { + val db = identifier.database.get + logInfo(s"Saving case-sensitive schema for table ${identifier.unquotedString}") + sparkSession.sharedState.externalCatalog.alterTableSchema(db, identifier.table, schema) + } catch { + case NonFatal(ex) => + logWarning(s"Unable to save case-sensitive schema for table ${identifier.unquotedString}", ex) } } -private[hive] object HiveMetastoreTypes { - def toDataType(metastoreType: String): DataType = DataTypeParser.parse(metastoreType) - - def decimalMetastoreString(decimalType: DecimalType): String = decimalType match { - case DecimalType.Fixed(precision, scale) => s"decimal($precision,$scale)" - case _ => s"decimal($HiveShim.UNLIMITED_DECIMAL_PRECISION,$HiveShim.UNLIMITED_DECIMAL_SCALE)" - } - - def toMetastoreType(dt: DataType): String = dt match { - case ArrayType(elementType, _) => s"array<${toMetastoreType(elementType)}>" - case StructType(fields) => - s"struct<${fields.map(f => s"${f.name}:${toMetastoreType(f.dataType)}").mkString(",")}>" - case MapType(keyType, valueType, _) => - s"map<${toMetastoreType(keyType)},${toMetastoreType(valueType)}>" - case StringType => "string" - case FloatType => "float" - case IntegerType => "int" - case ByteType => "tinyint" - case ShortType => "smallint" - case DoubleType => "double" - case LongType => "bigint" - case BinaryType => "binary" - case BooleanType => "boolean" - case DateType => "date" - case d: DecimalType => decimalMetastoreString(d) - case TimestampType => "timestamp" - case NullType => "void" - case udt: UserDefinedType[_] => toMetastoreType(udt.sqlType) +private[hive] object HiveMetastoreCatalog { + def mergeWithMetastoreSchema( + metastoreSchema: StructType, + inferredSchema: StructType): StructType = try { + // Find any nullable fields in mestastore schema that are missing from the inferred schema. + val metastoreFields = metastoreSchema.map(f => f.name.toLowerCase -> f).toMap + val missingNullables = metastoreFields + .filterKeys(!inferredSchema.map(_.name.toLowerCase).contains(_)) + .values + .filter(_.nullable) + // Merge missing nullable fields to inferred schema and build a case-insensitive field map. + val inferredFields = StructType(inferredSchema ++ missingNullables) + .map(f => f.name.toLowerCase -> f).toMap + StructType(metastoreSchema.map(f => f.copy(name = inferredFields(f.name).name))) + } catch { + case NonFatal(_) => + val msg = s"""Detected conflicting schemas when merging the schema obtained from the Hive + | Metastore with the one inferred from the file format. Metastore schema: + |${metastoreSchema.prettyJson} + | + |Inferred schema: + |${inferredSchema.prettyJson} + """.stripMargin + throw new SparkException(msg) } } 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 deleted file mode 100644 index ab88c1e68fd7..000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ /dev/null @@ -1,1867 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import java.sql.Date -import java.util.Locale - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer - -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} -import org.apache.hadoop.hive.ql.lib.Node -import org.apache.hadoop.hive.ql.parse._ -import org.apache.hadoop.hive.ql.plan.PlanUtils -import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.ql.{Context, ErrorMsg} -import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe - -import org.apache.spark.Logging -import org.apache.spark.sql.{AnalysisException, catalyst} -import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.{logical, _} -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.trees.CurrentOrigin -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.execution.ExplainCommand -import org.apache.spark.sql.execution.datasources.DescribeCommand -import org.apache.spark.sql.hive.HiveShim._ -import org.apache.spark.sql.hive.client._ -import org.apache.spark.sql.hive.execution.{AnalyzeTable, DropTable, HiveNativeCommand, HiveScriptIOSchema} -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval -import org.apache.spark.util.random.RandomSampler - -/** - * Used when we need to start parsing the AST before deciding that we are going to pass the command - * back for Hive to execute natively. Will be replaced with a native command that contains the - * cmd string. - */ -private[hive] case object NativePlaceholder extends LogicalPlan { - override def children: Seq[LogicalPlan] = Seq.empty - override def output: Seq[Attribute] = Seq.empty -} - -private[hive] case class CreateTableAsSelect( - tableDesc: HiveTable, - child: LogicalPlan, - allowExisting: Boolean) extends UnaryNode with Command { - - override def output: Seq[Attribute] = Seq.empty[Attribute] - override lazy val resolved: Boolean = - tableDesc.specifiedDatabase.isDefined && - tableDesc.schema.size > 0 && - tableDesc.serde.isDefined && - tableDesc.inputFormat.isDefined && - tableDesc.outputFormat.isDefined && - childrenResolved -} - -private[hive] case class CreateViewAsSelect( - tableDesc: HiveTable, - child: LogicalPlan, - allowExisting: Boolean, - replace: Boolean, - sql: String) extends UnaryNode with Command { - override def output: Seq[Attribute] = Seq.empty[Attribute] - override lazy val resolved: Boolean = false -} - -/** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ -private[hive] object HiveQl extends Logging { - protected val nativeCommands = Seq( - "TOK_ALTERDATABASE_OWNER", - "TOK_ALTERDATABASE_PROPERTIES", - "TOK_ALTERINDEX_PROPERTIES", - "TOK_ALTERINDEX_REBUILD", - "TOK_ALTERTABLE", - "TOK_ALTERTABLE_ADDCOLS", - "TOK_ALTERTABLE_ADDPARTS", - "TOK_ALTERTABLE_ALTERPARTS", - "TOK_ALTERTABLE_ARCHIVE", - "TOK_ALTERTABLE_CLUSTER_SORT", - "TOK_ALTERTABLE_DROPPARTS", - "TOK_ALTERTABLE_PARTITION", - "TOK_ALTERTABLE_PROPERTIES", - "TOK_ALTERTABLE_RENAME", - "TOK_ALTERTABLE_RENAMECOL", - "TOK_ALTERTABLE_REPLACECOLS", - "TOK_ALTERTABLE_SKEWED", - "TOK_ALTERTABLE_TOUCH", - "TOK_ALTERTABLE_UNARCHIVE", - "TOK_ALTERVIEW_ADDPARTS", - "TOK_ALTERVIEW_AS", - "TOK_ALTERVIEW_DROPPARTS", - "TOK_ALTERVIEW_PROPERTIES", - "TOK_ALTERVIEW_RENAME", - - "TOK_CREATEDATABASE", - "TOK_CREATEFUNCTION", - "TOK_CREATEINDEX", - "TOK_CREATEMACRO", - "TOK_CREATEROLE", - - "TOK_DESCDATABASE", - "TOK_DESCFUNCTION", - - "TOK_DROPDATABASE", - "TOK_DROPFUNCTION", - "TOK_DROPINDEX", - "TOK_DROPMACRO", - "TOK_DROPROLE", - "TOK_DROPTABLE_PROPERTIES", - "TOK_DROPVIEW", - "TOK_DROPVIEW_PROPERTIES", - - "TOK_EXPORT", - - "TOK_GRANT", - "TOK_GRANT_ROLE", - - "TOK_IMPORT", - - "TOK_LOAD", - - "TOK_LOCKTABLE", - - "TOK_MSCK", - - "TOK_REVOKE", - - "TOK_SHOW_COMPACTIONS", - "TOK_SHOW_CREATETABLE", - "TOK_SHOW_GRANT", - "TOK_SHOW_ROLE_GRANT", - "TOK_SHOW_ROLE_PRINCIPALS", - "TOK_SHOW_ROLES", - "TOK_SHOW_SET_ROLE", - "TOK_SHOW_TABLESTATUS", - "TOK_SHOW_TBLPROPERTIES", - "TOK_SHOW_TRANSACTIONS", - "TOK_SHOWCOLUMNS", - "TOK_SHOWDATABASES", - "TOK_SHOWFUNCTIONS", - "TOK_SHOWINDEXES", - "TOK_SHOWLOCKS", - "TOK_SHOWPARTITIONS", - - "TOK_SWITCHDATABASE", - - "TOK_UNLOCKTABLE" - ) - - // 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 - - protected val hqlParser = new ExtendedHiveQlParser - - /** - * A set of implicit transformations that allow Hive ASTNodes to be rewritten by transformations - * similar to [[catalyst.trees.TreeNode]]. - * - * Note that this should be considered very experimental and is not indented as a replacement - * for TreeNode. Primarily it should be noted ASTNodes are not immutable and do not appear to - * have clean copy semantics. Therefore, users of this class should take care when - * copying/modifying trees that might be used elsewhere. - */ - implicit class TransformableNode(n: ASTNode) { - /** - * Returns a copy of this node where `rule` has been recursively applied to it and all of its - * children. When `rule` does not apply to a given node it is left unchanged. - * @param rule the function use to transform this nodes children - */ - def transform(rule: PartialFunction[ASTNode, ASTNode]): ASTNode = { - try { - val afterRule = rule.applyOrElse(n, identity[ASTNode]) - afterRule.withChildren( - nilIfEmpty(afterRule.getChildren) - .asInstanceOf[Seq[ASTNode]] - .map(ast => Option(ast).map(_.transform(rule)).orNull)) - } catch { - case e: Exception => - logError(dumpTree(n).toString) - throw e - } - } - - /** - * Returns a scala.Seq equivalent to [s] or Nil if [s] is null. - */ - private def nilIfEmpty[A](s: java.util.List[A]): Seq[A] = - Option(s).map(_.asScala).getOrElse(Nil) - - /** - * Returns this ASTNode with the text changed to `newText`. - */ - def withText(newText: String): ASTNode = { - n.token.asInstanceOf[org.antlr.runtime.CommonToken].setText(newText) - n - } - - /** - * Returns this ASTNode with the children changed to `newChildren`. - */ - def withChildren(newChildren: Seq[ASTNode]): ASTNode = { - (1 to n.getChildCount).foreach(_ => n.deleteChild(0)) - n.addChildren(newChildren.asJava) - n - } - - /** - * Throws an error if this is not equal to other. - * - * Right now this function only checks the name, type, text and children of the node - * for equality. - */ - def checkEquals(other: ASTNode): Unit = { - def check(field: String, f: ASTNode => Any): Unit = if (f(n) != f(other)) { - sys.error(s"$field does not match for trees. " + - s"'${f(n)}' != '${f(other)}' left: ${dumpTree(n)}, right: ${dumpTree(other)}") - } - check("name", _.getName) - check("type", _.getType) - check("text", _.getText) - check("numChildren", n => nilIfEmpty(n.getChildren).size) - - val leftChildren = nilIfEmpty(n.getChildren).asInstanceOf[Seq[ASTNode]] - val rightChildren = nilIfEmpty(other.getChildren).asInstanceOf[Seq[ASTNode]] - leftChildren zip rightChildren foreach { - case (l, r) => l checkEquals r - } - } - } - - /** - * Returns the AST for the given SQL string. - */ - def getAst(sql: String): ASTNode = { - /* - * Context has to be passed in hive0.13.1. - * Otherwise, there will be Null pointer exception, - * when retrieving properties form HiveConf. - */ - val hContext = createContext() - val node = getAst(sql, hContext) - hContext.clear() - node - } - - private def createContext(): Context = new Context(hiveConf) - - private def getAst(sql: String, context: Context) = - ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql, context)) - - /** - * Returns the HiveConf - */ - private[this] def hiveConf: HiveConf = { - var ss = SessionState.get() - // SessionState is lazy initialization, it can be null here - if (ss == null) { - val original = Thread.currentThread().getContextClassLoader - val conf = new HiveConf(classOf[SessionState]) - conf.setClassLoader(original) - ss = new SessionState(conf) - SessionState.start(ss) - } - ss.getConf - } - - /** Returns a LogicalPlan for a given HiveQL string. */ - def parseSql(sql: String): LogicalPlan = hqlParser.parse(sql) - - val errorRegEx = "line (\\d+):(\\d+) (.*)".r - - /** Creates LogicalPlan for a given HiveQL string. */ - def createPlan(sql: String): LogicalPlan = { - try { - val context = createContext() - val tree = getAst(sql, context) - val plan = if (nativeCommands contains tree.getText) { - HiveNativeCommand(sql) - } else { - nodeToPlan(tree, context) match { - case NativePlaceholder => HiveNativeCommand(sql) - case other => other - } - } - context.clear() - plan - } catch { - 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: MatchError => throw e - 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) - } - } - - def parseDdl(ddl: String): Seq[Attribute] = { - val tree = - try { - ParseUtils.findRootNonNullToken( - (new ParseDriver).parse(ddl, null /* no context required for parsing alone */)) - } catch { - case pe: org.apache.hadoop.hive.ql.parse.ParseException => - throw new RuntimeException(s"Failed to parse ddl: '$ddl'", pe) - } - assert(tree.asInstanceOf[ASTNode].getText == "TOK_CREATETABLE", "Only CREATE TABLE supported.") - val tableOps = tree.getChildren - val colList = - tableOps.asScala - .find(_.asInstanceOf[ASTNode].getText == "TOK_TABCOLLIST") - .getOrElse(sys.error("No columnList!")).getChildren - - colList.asScala.map(nodeToAttribute) - } - - /** Extractor for matching Hive's AST Tokens. */ - object Token { - /** @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(_.asScala.toList).getOrElse(Nil).asInstanceOf[Seq[ASTNode]])) - case _ => None - } - } - - protected def getClauses( - clauseNames: Seq[String], - nodeList: Seq[ASTNode]): Seq[Option[ASTNode]] = { - var remainingNodes = nodeList - val clauses = clauseNames.map { clauseName => - val (matches, nonMatches) = remainingNodes.partition(_.getText.toUpperCase == clauseName) - remainingNodes = nonMatches ++ (if (matches.nonEmpty) matches.tail else Nil) - matches.headOption - } - - if (remainingNodes.nonEmpty) { - sys.error( - s"""Unhandled clauses: ${remainingNodes.map(dumpTree(_)).mkString("\n")}. - |You are likely trying to use an unsupported Hive feature."""".stripMargin) - } - clauses - } - - def getClause(clauseName: String, nodeList: Seq[Node]): Node = - getClauseOption(clauseName, nodeList).getOrElse(sys.error( - s"Expected clause $clauseName missing from ${nodeList.map(dumpTree(_)).mkString("\n")}")) - - def getClauseOption(clauseName: String, nodeList: Seq[Node]): Option[Node] = { - nodeList.filter { case ast: ASTNode => ast.getText == clauseName } match { - case Seq(oneMatch) => Some(oneMatch) - case Seq() => None - case _ => sys.error(s"Found multiple instances of clause $clauseName") - } - } - - protected def nodeToAttribute(node: Node): Attribute = node match { - case Token("TOK_TABCOL", Token(colName, Nil) :: dataType :: Nil) => - AttributeReference(colName, nodeToDataType(dataType), true)() - - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") - } - - protected def nodeToDataType(node: Node): DataType = node match { - case Token("TOK_DECIMAL", precision :: scale :: Nil) => - DecimalType(precision.getText.toInt, scale.getText.toInt) - case Token("TOK_DECIMAL", precision :: Nil) => - DecimalType(precision.getText.toInt, 0) - case Token("TOK_DECIMAL", Nil) => DecimalType.USER_DEFAULT - case Token("TOK_BIGINT", Nil) => LongType - case Token("TOK_INT", Nil) => IntegerType - case Token("TOK_TINYINT", Nil) => ByteType - case Token("TOK_SMALLINT", Nil) => ShortType - case Token("TOK_BOOLEAN", Nil) => BooleanType - case Token("TOK_STRING", Nil) => StringType - case Token("TOK_VARCHAR", Token(_, Nil) :: Nil) => StringType - case Token("TOK_FLOAT", Nil) => FloatType - case Token("TOK_DOUBLE", Nil) => DoubleType - case Token("TOK_DATE", Nil) => DateType - case Token("TOK_TIMESTAMP", Nil) => TimestampType - case Token("TOK_BINARY", Nil) => BinaryType - case Token("TOK_LIST", elementType :: Nil) => ArrayType(nodeToDataType(elementType)) - case Token("TOK_STRUCT", - Token("TOK_TABCOLLIST", fields) :: Nil) => - StructType(fields.map(nodeToStructField)) - case Token("TOK_MAP", - keyType :: - valueType :: Nil) => - MapType(nodeToDataType(keyType), nodeToDataType(valueType)) - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for DataType:\n ${dumpTree(a).toString} ") - } - - protected def nodeToStructField(node: Node): StructField = node match { - case Token("TOK_TABCOL", - Token(fieldName, Nil) :: - dataType :: Nil) => - StructField(fieldName, nodeToDataType(dataType), nullable = true) - case Token("TOK_TABCOL", - Token(fieldName, Nil) :: - dataType :: - _ /* comment */:: Nil) => - StructField(fieldName, nodeToDataType(dataType), nullable = true) - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for StructField:\n ${dumpTree(a).toString} ") - } - - protected def extractTableIdent(tableNameParts: Node): TableIdentifier = { - tableNameParts.getChildren.asScala.map { - case Token(part, Nil) => cleanIdentifier(part) - } match { - case Seq(tableOnly) => TableIdentifier(tableOnly) - case Seq(databaseName, table) => TableIdentifier(table, Some(databaseName)) - case other => sys.error("Hive only supports tables names like 'tableName' " + - s"or 'databaseName.tableName', found '$other'") - } - } - - /** - * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) - * is equivalent to - * SELECT MAX(value) FROM src GROUP BY k1, k2 UNION SELECT MAX(value) FROM src GROUP BY k2 - * Check the following link for details. - * -https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C+Grouping+and+Rollup - * - * The bitmask denotes the grouping expressions validity for a grouping set, - * the bitmask also be called as grouping id (`GROUPING__ID`, the virtual column in Hive) - * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of - * GROUPING SETS (k1, k2) and (k2) should be 3 and 2 respectively. - */ - protected def extractGroupingSet(children: Seq[ASTNode]): (Seq[Expression], Seq[Int]) = { - val (keyASTs, setASTs) = children.partition( n => n match { - case Token("TOK_GROUPING_SETS_EXPRESSION", children) => false // grouping sets - case _ => true // grouping keys - }) - - val keys = keyASTs.map(nodeToExpr).toSeq - val keyMap = keyASTs.map(_.toStringTree).zipWithIndex.toMap - - val bitmasks: Seq[Int] = setASTs.map(set => set match { - case Token("TOK_GROUPING_SETS_EXPRESSION", null) => 0 - case Token("TOK_GROUPING_SETS_EXPRESSION", children) => - children.foldLeft(0)((bitmap, col) => { - val colString = col.asInstanceOf[ASTNode].toStringTree() - require(keyMap.contains(colString), s"$colString doens't show up in the GROUP BY list") - bitmap | 1 << keyMap(colString) - }) - case _ => sys.error("Expect GROUPING SETS clause") - }) - - (keys, bitmasks) - } - - protected def getProperties(node: Node): Seq[(String, String)] = node match { - case Token("TOK_TABLEPROPLIST", list) => - list.map { - case Token("TOK_TABLEPROPERTY", Token(key, Nil) :: Token(value, Nil) :: Nil) => - (unquoteString(key) -> unquoteString(value)) - } - } - - private def createView( - view: ASTNode, - context: Context, - viewNameParts: ASTNode, - query: ASTNode, - schema: Seq[HiveColumn], - properties: Map[String, String], - allowExist: Boolean, - replace: Boolean): CreateViewAsSelect = { - val TableIdentifier(viewName, dbName) = extractTableIdent(viewNameParts) - - val originalText = context.getTokenRewriteStream - .toString(query.getTokenStartIndex, query.getTokenStopIndex) - - val tableDesc = HiveTable( - specifiedDatabase = dbName, - name = viewName, - schema = schema, - partitionColumns = Seq.empty[HiveColumn], - properties = properties, - serdeProperties = Map[String, String](), - tableType = VirtualView, - location = None, - inputFormat = None, - outputFormat = None, - serde = None, - viewText = Some(originalText)) - - // We need to keep the original SQL string so that if `spark.sql.nativeView` is - // false, we can fall back to use hive native command later. - // We can remove this when parser is configurable(can access SQLConf) in the future. - val sql = context.getTokenRewriteStream - .toString(view.getTokenStartIndex, view.getTokenStopIndex) - CreateViewAsSelect(tableDesc, nodeToPlan(query, context), allowExist, replace, sql) - } - - protected def nodeToPlan(node: ASTNode, context: Context): LogicalPlan = node match { - // Special drop table that also uncaches. - case Token("TOK_DROPTABLE", - Token("TOK_TABNAME", tableNameParts) :: - ifExists) => - val tableName = tableNameParts.map { case Token(p, Nil) => p }.mkString(".") - DropTable(tableName, ifExists.nonEmpty) - // Support "ANALYZE TABLE tableNmae COMPUTE STATISTICS noscan" - case Token("TOK_ANALYZE", - Token("TOK_TAB", Token("TOK_TABNAME", tableNameParts) :: partitionSpec) :: - isNoscan) => - // Reference: - // https://cwiki.apache.org/confluence/display/Hive/StatsDev#StatsDev-ExistingTables - if (partitionSpec.nonEmpty) { - // Analyze partitions will be treated as a Hive native command. - NativePlaceholder - } else if (isNoscan.isEmpty) { - // If users do not specify "noscan", it will be treated as a Hive native command. - NativePlaceholder - } else { - val tableName = tableNameParts.map { case Token(p, Nil) => p }.mkString(".") - AnalyzeTable(tableName) - } - // Just fake explain for any of the native commands. - case Token("TOK_EXPLAIN", explainArgs) - if noExplainCommands.contains(explainArgs.head.getText) => - ExplainCommand(OneRowRelation) - 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, context), - 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, context), - extended = extended.isDefined) - - case Token("TOK_DESCTABLE", describeArgs) => - // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL - val Some(tableType) :: formatted :: extended :: pretty :: Nil = - getClauses(Seq("TOK_TABTYPE", "FORMATTED", "EXTENDED", "PRETTY"), describeArgs) - if (formatted.isDefined || pretty.isDefined) { - // FORMATTED and PRETTY are not supported and this statement will be treated as - // a Hive native command. - NativePlaceholder - } else { - tableType match { - case Token("TOK_TABTYPE", nameParts) if nameParts.size == 1 => { - nameParts.head match { - case Token(".", dbName :: tableName :: Nil) => - // It is describing a table with the format like "describe db.table". - // TODO: Actually, a user may mean tableName.columnName. Need to resolve this issue. - val tableIdent = extractTableIdent(nameParts.head) - DescribeCommand( - UnresolvedRelation(tableIdent, None), isExtended = extended.isDefined) - case Token(".", dbName :: tableName :: colName :: Nil) => - // It is describing a column with the format like "describe db.table column". - NativePlaceholder - case tableName => - // It is describing a table with the format like "describe table". - DescribeCommand( - UnresolvedRelation(TableIdentifier(tableName.getText), None), - isExtended = extended.isDefined) - } - } - // All other cases. - case _ => NativePlaceholder - } - } - - case view @ Token("TOK_ALTERVIEW", children) => - val Some(viewNameParts) :: maybeQuery :: ignores = - getClauses(Seq( - "TOK_TABNAME", - "TOK_QUERY", - "TOK_ALTERVIEW_ADDPARTS", - "TOK_ALTERVIEW_DROPPARTS", - "TOK_ALTERVIEW_PROPERTIES", - "TOK_ALTERVIEW_RENAME"), children) - - // if ALTER VIEW doesn't have query part, let hive to handle it. - maybeQuery.map { query => - createView(view, context, viewNameParts, query, Nil, Map(), false, true) - }.getOrElse(NativePlaceholder) - - case view @ Token("TOK_CREATEVIEW", children) - if children.collect { case t @ Token("TOK_QUERY", _) => t }.nonEmpty => - val Seq( - Some(viewNameParts), - Some(query), - maybeComment, - replace, - allowExisting, - maybeProperties, - maybeColumns, - maybePartCols - ) = getClauses(Seq( - "TOK_TABNAME", - "TOK_QUERY", - "TOK_TABLECOMMENT", - "TOK_ORREPLACE", - "TOK_IFNOTEXISTS", - "TOK_TABLEPROPERTIES", - "TOK_TABCOLNAME", - "TOK_VIEWPARTCOLS"), children) - - // If the view is partitioned, we let hive handle it. - if (maybePartCols.isDefined) { - NativePlaceholder - } else { - val schema = maybeColumns.map { cols => - BaseSemanticAnalyzer.getColumns(cols, true).asScala.map { field => - // We can't specify column types when create view, so fill it with null first, and - // update it after the schema has been resolved later. - HiveColumn(field.getName, null, field.getComment) - } - }.getOrElse(Seq.empty[HiveColumn]) - - val properties = scala.collection.mutable.Map.empty[String, String] - - maybeProperties.foreach { - case Token("TOK_TABLEPROPERTIES", list :: Nil) => - properties ++= getProperties(list) - } - - maybeComment.foreach { - case Token("TOK_TABLECOMMENT", child :: Nil) => - val comment = BaseSemanticAnalyzer.unescapeSQLString(child.getText) - if (comment ne null) { - properties += ("comment" -> comment) - } - } - - createView(view, context, viewNameParts, query, schema, properties.toMap, - allowExisting.isDefined, replace.isDefined) - } - - case Token("TOK_CREATETABLE", children) - if children.collect { case t @ Token("TOK_QUERY", _) => t }.nonEmpty => - // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL - val ( - Some(tableNameParts) :: - _ /* likeTable */ :: - externalTable :: - Some(query) :: - allowExisting +: - ignores) = - getClauses( - Seq( - "TOK_TABNAME", - "TOK_LIKETABLE", - "EXTERNAL", - "TOK_QUERY", - "TOK_IFNOTEXISTS", - "TOK_TABLECOMMENT", - "TOK_TABCOLLIST", - "TOK_TABLEPARTCOLS", // Partitioned by - "TOK_TABLEBUCKETS", // Clustered by - "TOK_TABLESKEWED", // Skewed by - "TOK_TABLEROWFORMAT", - "TOK_TABLESERIALIZER", - "TOK_FILEFORMAT_GENERIC", - "TOK_TABLEFILEFORMAT", // User-provided InputFormat and OutputFormat - "TOK_STORAGEHANDLER", // Storage handler - "TOK_TABLELOCATION", - "TOK_TABLEPROPERTIES"), - children) - val TableIdentifier(tblName, dbName) = extractTableIdent(tableNameParts) - - // TODO add bucket support - var tableDesc: HiveTable = HiveTable( - specifiedDatabase = dbName, - name = tblName, - schema = Seq.empty[HiveColumn], - partitionColumns = Seq.empty[HiveColumn], - properties = Map[String, String](), - serdeProperties = Map[String, String](), - tableType = if (externalTable.isDefined) ExternalTable else ManagedTable, - location = None, - inputFormat = None, - outputFormat = None, - serde = None, - viewText = None) - - // default storage type abbreviation (e.g. RCFile, ORC, PARQUET etc.) - val defaultStorageType = hiveConf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT) - // handle the default format for the storage type abbreviation - val hiveSerDe = HiveSerDe.sourceToSerDe(defaultStorageType, hiveConf).getOrElse { - HiveSerDe( - inputFormat = Option("org.apache.hadoop.mapred.TextInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) - } - - hiveSerDe.inputFormat.foreach(f => tableDesc = tableDesc.copy(inputFormat = Some(f))) - hiveSerDe.outputFormat.foreach(f => tableDesc = tableDesc.copy(outputFormat = Some(f))) - hiveSerDe.serde.foreach(f => tableDesc = tableDesc.copy(serde = Some(f))) - - children.collect { - case list @ Token("TOK_TABCOLLIST", _) => - val cols = BaseSemanticAnalyzer.getColumns(list, true) - if (cols != null) { - tableDesc = tableDesc.copy( - schema = cols.asScala.map { field => - HiveColumn(field.getName, field.getType, field.getComment) - }) - } - case Token("TOK_TABLECOMMENT", child :: Nil) => - val comment = BaseSemanticAnalyzer.unescapeSQLString(child.getText) - // TODO support the sql text - tableDesc = tableDesc.copy(viewText = Option(comment)) - case Token("TOK_TABLEPARTCOLS", list @ Token("TOK_TABCOLLIST", _) :: Nil) => - val cols = BaseSemanticAnalyzer.getColumns(list(0), false) - if (cols != null) { - tableDesc = tableDesc.copy( - partitionColumns = cols.asScala.map { field => - HiveColumn(field.getName, field.getType, field.getComment) - }) - } - case Token("TOK_TABLEROWFORMAT", Token("TOK_SERDEPROPS", child :: Nil) :: Nil) => - val serdeParams = new java.util.HashMap[String, String]() - child match { - case Token("TOK_TABLEROWFORMATFIELD", rowChild1 :: rowChild2) => - val fieldDelim = BaseSemanticAnalyzer.unescapeSQLString (rowChild1.getText()) - serdeParams.put(serdeConstants.FIELD_DELIM, fieldDelim) - serdeParams.put(serdeConstants.SERIALIZATION_FORMAT, fieldDelim) - if (rowChild2.length > 1) { - val fieldEscape = BaseSemanticAnalyzer.unescapeSQLString (rowChild2(0).getText) - serdeParams.put(serdeConstants.ESCAPE_CHAR, fieldEscape) - } - case Token("TOK_TABLEROWFORMATCOLLITEMS", rowChild :: Nil) => - val collItemDelim = BaseSemanticAnalyzer.unescapeSQLString(rowChild.getText) - serdeParams.put(serdeConstants.COLLECTION_DELIM, collItemDelim) - case Token("TOK_TABLEROWFORMATMAPKEYS", rowChild :: Nil) => - val mapKeyDelim = BaseSemanticAnalyzer.unescapeSQLString(rowChild.getText) - serdeParams.put(serdeConstants.MAPKEY_DELIM, mapKeyDelim) - case Token("TOK_TABLEROWFORMATLINES", rowChild :: Nil) => - val lineDelim = BaseSemanticAnalyzer.unescapeSQLString(rowChild.getText) - if (!(lineDelim == "\n") && !(lineDelim == "10")) { - throw new AnalysisException( - SemanticAnalyzer.generateErrorMessage( - rowChild, - ErrorMsg.LINES_TERMINATED_BY_NON_NEWLINE.getMsg)) - } - serdeParams.put(serdeConstants.LINE_DELIM, lineDelim) - case Token("TOK_TABLEROWFORMATNULL", rowChild :: Nil) => - val nullFormat = BaseSemanticAnalyzer.unescapeSQLString(rowChild.getText) - // TODO support the nullFormat - case _ => assert(false) - } - tableDesc = tableDesc.copy( - serdeProperties = tableDesc.serdeProperties ++ serdeParams.asScala) - case Token("TOK_TABLELOCATION", child :: Nil) => - var location = BaseSemanticAnalyzer.unescapeSQLString(child.getText) - location = EximUtil.relativeToAbsolutePath(hiveConf, location) - tableDesc = tableDesc.copy(location = Option(location)) - case Token("TOK_TABLESERIALIZER", child :: Nil) => - tableDesc = tableDesc.copy( - serde = Option(BaseSemanticAnalyzer.unescapeSQLString(child.getChild(0).getText))) - if (child.getChildCount == 2) { - val serdeParams = new java.util.HashMap[String, String]() - BaseSemanticAnalyzer.readProps( - (child.getChild(1).getChild(0)).asInstanceOf[ASTNode], serdeParams) - tableDesc = tableDesc.copy( - serdeProperties = tableDesc.serdeProperties ++ serdeParams.asScala) - } - case Token("TOK_FILEFORMAT_GENERIC", child :: Nil) => - child.getText().toLowerCase(Locale.ENGLISH) match { - case "orc" => - tableDesc = tableDesc.copy( - inputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) - if (tableDesc.serde.isEmpty) { - tableDesc = tableDesc.copy( - serde = Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) - } - - case "parquet" => - tableDesc = tableDesc.copy( - inputFormat = - Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"), - outputFormat = - Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) - if (tableDesc.serde.isEmpty) { - tableDesc = tableDesc.copy( - serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) - } - - case "rcfile" => - tableDesc = tableDesc.copy( - inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"), - outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) - if (tableDesc.serde.isEmpty) { - tableDesc = tableDesc.copy( - serde = Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) - } - - case "textfile" => - tableDesc = tableDesc.copy( - inputFormat = - Option("org.apache.hadoop.mapred.TextInputFormat"), - outputFormat = - Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) - - case "sequencefile" => - tableDesc = tableDesc.copy( - inputFormat = Option("org.apache.hadoop.mapred.SequenceFileInputFormat"), - outputFormat = Option("org.apache.hadoop.mapred.SequenceFileOutputFormat")) - - case "avro" => - tableDesc = tableDesc.copy( - inputFormat = - Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat"), - outputFormat = - Option("org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat")) - if (tableDesc.serde.isEmpty) { - tableDesc = tableDesc.copy( - serde = Option("org.apache.hadoop.hive.serde2.avro.AvroSerDe")) - } - - case _ => - throw new SemanticException( - s"Unrecognized file format in STORED AS clause: ${child.getText}") - } - - case Token("TOK_TABLESERIALIZER", - Token("TOK_SERDENAME", Token(serdeName, Nil) :: otherProps) :: Nil) => - tableDesc = tableDesc.copy(serde = Option(unquoteString(serdeName))) - - otherProps match { - case Token("TOK_TABLEPROPERTIES", list :: Nil) :: Nil => - tableDesc = tableDesc.copy( - serdeProperties = tableDesc.serdeProperties ++ getProperties(list)) - case Nil => - } - - case Token("TOK_TABLEPROPERTIES", list :: Nil) => - tableDesc = tableDesc.copy(properties = tableDesc.properties ++ getProperties(list)) - case list @ Token("TOK_TABLEFILEFORMAT", children) => - tableDesc = tableDesc.copy( - inputFormat = - Option(BaseSemanticAnalyzer.unescapeSQLString(list.getChild(0).getText)), - outputFormat = - Option(BaseSemanticAnalyzer.unescapeSQLString(list.getChild(1).getText))) - case Token("TOK_STORAGEHANDLER", _) => - throw new AnalysisException(ErrorMsg.CREATE_NON_NATIVE_AS.getMsg()) - case _ => // Unsupport features - } - - CreateTableAsSelect(tableDesc, nodeToPlan(query, context), allowExisting != None) - - // If its not a "CTAS" like above then take it as a native command - case Token("TOK_CREATETABLE", _) => NativePlaceholder - - // Support "TRUNCATE TABLE table_name [PARTITION partition_spec]" - case Token("TOK_TRUNCATETABLE", - Token("TOK_TABLE_PARTITION", table) :: Nil) => NativePlaceholder - - case Token("TOK_QUERY", queryArgs) - if Seq("TOK_FROM", "TOK_INSERT").contains(queryArgs.head.getText) => - - val (fromClause: Option[ASTNode], insertClauses, cteRelations) = - queryArgs match { - case Token("TOK_FROM", args: Seq[ASTNode]) :: insertClauses => - // check if has CTE - insertClauses.last match { - case Token("TOK_CTE", cteClauses) => - val cteRelations = cteClauses.map(node => { - val relation = nodeToRelation(node, context).asInstanceOf[Subquery] - (relation.alias, relation) - }).toMap - (Some(args.head), insertClauses.init, Some(cteRelations)) - - case _ => (Some(args.head), insertClauses, None) - } - - case Token("TOK_INSERT", _) :: Nil => (None, queryArgs, None) - } - - // Return one query for each insert clause. - val queries = insertClauses.map { case Token("TOK_INSERT", singleInsert) => - val ( - intoClause :: - destClause :: - selectClause :: - selectDistinctClause :: - whereClause :: - groupByClause :: - rollupGroupByClause :: - cubeGroupByClause :: - groupingSetsClause :: - orderByClause :: - havingClause :: - sortByClause :: - clusterByClause :: - distributeByClause :: - limitClause :: - lateralViewClause :: - windowClause :: Nil) = { - getClauses( - Seq( - "TOK_INSERT_INTO", - "TOK_DESTINATION", - "TOK_SELECT", - "TOK_SELECTDI", - "TOK_WHERE", - "TOK_GROUPBY", - "TOK_ROLLUP_GROUPBY", - "TOK_CUBE_GROUPBY", - "TOK_GROUPING_SETS", - "TOK_ORDERBY", - "TOK_HAVING", - "TOK_SORTBY", - "TOK_CLUSTERBY", - "TOK_DISTRIBUTEBY", - "TOK_LIMIT", - "TOK_LATERAL_VIEW", - "WINDOW"), - singleInsert) - } - - val relations = fromClause match { - case Some(f) => nodeToRelation(f, context) - case None => OneRowRelation - } - - val withWhere = whereClause.map { whereNode => - val Seq(whereExpr) = whereNode.getChildren.asScala - Filter(nodeToExpr(whereExpr), relations) - }.getOrElse(relations) - - val select = - (selectClause orElse selectDistinctClause).getOrElse(sys.error("No select clause.")) - - // Script transformations are expressed as a select clause with a single expression of type - // TOK_TRANSFORM - val transformation = select.getChildren.iterator().next() match { - case Token("TOK_SELEXPR", - Token("TOK_TRANSFORM", - Token("TOK_EXPLIST", inputExprs) :: - Token("TOK_SERDE", inputSerdeClause) :: - Token("TOK_RECORDWRITER", writerClause) :: - // TODO: Need to support other types of (in/out)put - Token(script, Nil) :: - Token("TOK_SERDE", outputSerdeClause) :: - Token("TOK_RECORDREADER", readerClause) :: - outputClause) :: Nil) => - - val (output, schemaLess) = outputClause match { - case Token("TOK_ALIASLIST", aliases) :: Nil => - (aliases.map { case Token(name, Nil) => AttributeReference(name, StringType)() }, - false) - case Token("TOK_TABCOLLIST", attributes) :: Nil => - (attributes.map { case Token("TOK_TABCOL", Token(name, Nil) :: dataType :: Nil) => - AttributeReference(name, nodeToDataType(dataType))() }, false) - case Nil => - (List(AttributeReference("key", StringType)(), - AttributeReference("value", StringType)()), true) - } - - type SerDeInfo = ( - Seq[(String, String)], // Input row format information - Option[String], // Optional input SerDe class - Seq[(String, String)], // Input SerDe properties - Boolean // Whether to use default record reader/writer - ) - - def matchSerDe(clause: Seq[ASTNode]): SerDeInfo = clause match { - case Token("TOK_SERDEPROPS", propsClause) :: Nil => - val rowFormat = propsClause.map { - case Token(name, Token(value, Nil) :: Nil) => (name, value) - } - (rowFormat, None, Nil, false) - - case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: Nil) :: Nil => - (Nil, Some(BaseSemanticAnalyzer.unescapeSQLString(serdeClass)), Nil, false) - - case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: - Token("TOK_TABLEPROPERTIES", - Token("TOK_TABLEPROPLIST", propsClause) :: Nil) :: Nil) :: Nil => - val serdeProps = propsClause.map { - case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) => - (BaseSemanticAnalyzer.unescapeSQLString(name), - BaseSemanticAnalyzer.unescapeSQLString(value)) - } - - // SPARK-10310: Special cases LazySimpleSerDe - // TODO Fully supports user-defined record reader/writer classes - val unescapedSerDeClass = BaseSemanticAnalyzer.unescapeSQLString(serdeClass) - val useDefaultRecordReaderWriter = - unescapedSerDeClass == classOf[LazySimpleSerDe].getCanonicalName - (Nil, Some(unescapedSerDeClass), serdeProps, useDefaultRecordReaderWriter) - - case Nil => - // Uses default TextRecordReader/TextRecordWriter, sets field delimiter here - val serdeProps = Seq(serdeConstants.FIELD_DELIM -> "\t") - (Nil, Option(hiveConf.getVar(ConfVars.HIVESCRIPTSERDE)), serdeProps, true) - } - - val (inRowFormat, inSerdeClass, inSerdeProps, useDefaultRecordReader) = - matchSerDe(inputSerdeClause) - - val (outRowFormat, outSerdeClass, outSerdeProps, useDefaultRecordWriter) = - matchSerDe(outputSerdeClause) - - val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) - - // TODO Adds support for user-defined record reader/writer classes - val recordReaderClass = if (useDefaultRecordReader) { - Option(hiveConf.getVar(ConfVars.HIVESCRIPTRECORDREADER)) - } else { - None - } - - val recordWriterClass = if (useDefaultRecordWriter) { - Option(hiveConf.getVar(ConfVars.HIVESCRIPTRECORDWRITER)) - } else { - None - } - - val schema = HiveScriptIOSchema( - inRowFormat, outRowFormat, - inSerdeClass, outSerdeClass, - inSerdeProps, outSerdeProps, - recordReaderClass, recordWriterClass, - schemaLess) - - Some( - logical.ScriptTransformation( - inputExprs.map(nodeToExpr), - unescapedScript, - output, - withWhere, schema)) - case _ => None - } - - val withLateralView = lateralViewClause.map { lv => - val Token("TOK_SELECT", - Token("TOK_SELEXPR", clauses) :: Nil) = lv.getChildren.iterator().next() - - val alias = getClause("TOK_TABALIAS", clauses).getChildren.iterator().next() - .asInstanceOf[ASTNode].getText - - val (generator, attributes) = nodesToGenerator(clauses) - Generate( - generator, - join = true, - outer = false, - Some(alias.toLowerCase), - attributes.map(UnresolvedAttribute(_)), - withWhere) - }.getOrElse(withWhere) - - // The projection of the query can either be a normal projection, an aggregation - // (if there is a group by) or a script transformation. - val withProject: LogicalPlan = transformation.getOrElse { - val selectExpressions = - select.getChildren.asScala.flatMap(selExprNodeToExpr).map(UnresolvedAlias) - Seq( - groupByClause.map(e => e match { - case Token("TOK_GROUPBY", children) => - // Not a transformation so must be either project or aggregation. - Aggregate(children.map(nodeToExpr), selectExpressions, withLateralView) - case _ => sys.error("Expect GROUP BY") - }), - groupingSetsClause.map(e => e match { - case Token("TOK_GROUPING_SETS", children) => - val(groupByExprs, masks) = extractGroupingSet(children) - GroupingSets(masks, groupByExprs, withLateralView, selectExpressions) - case _ => sys.error("Expect GROUPING SETS") - }), - rollupGroupByClause.map(e => e match { - case Token("TOK_ROLLUP_GROUPBY", children) => - Rollup(children.map(nodeToExpr), withLateralView, selectExpressions) - case _ => sys.error("Expect WITH ROLLUP") - }), - cubeGroupByClause.map(e => e match { - case Token("TOK_CUBE_GROUPBY", children) => - Cube(children.map(nodeToExpr), withLateralView, selectExpressions) - case _ => sys.error("Expect WITH CUBE") - }), - Some(Project(selectExpressions, withLateralView))).flatten.head - } - - // Handle HAVING clause. - val withHaving = havingClause.map { h => - val havingExpr = h.getChildren.asScala match { case Seq(hexpr) => nodeToExpr(hexpr) } - // Note that we added a cast to boolean. If the expression itself is already boolean, - // the optimizer will get rid of the unnecessary cast. - Filter(Cast(havingExpr, BooleanType), withProject) - }.getOrElse(withProject) - - // Handle SELECT DISTINCT - val withDistinct = - if (selectDistinctClause.isDefined) Distinct(withHaving) else withHaving - - // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause. - val withSort = - (orderByClause, sortByClause, distributeByClause, clusterByClause) match { - case (Some(totalOrdering), None, None, None) => - Sort(totalOrdering.getChildren.asScala.map(nodeToSortOrder), true, withDistinct) - case (None, Some(perPartitionOrdering), None, None) => - Sort( - perPartitionOrdering.getChildren.asScala.map(nodeToSortOrder), - false, withDistinct) - case (None, None, Some(partitionExprs), None) => - RepartitionByExpression( - partitionExprs.getChildren.asScala.map(nodeToExpr), withDistinct) - case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => - Sort( - perPartitionOrdering.getChildren.asScala.map(nodeToSortOrder), false, - RepartitionByExpression( - partitionExprs.getChildren.asScala.map(nodeToExpr), - withDistinct)) - case (None, None, None, Some(clusterExprs)) => - Sort( - clusterExprs.getChildren.asScala.map(nodeToExpr).map(SortOrder(_, Ascending)), - false, - RepartitionByExpression( - clusterExprs.getChildren.asScala.map(nodeToExpr), - withDistinct)) - case (None, None, None, None) => withDistinct - case _ => sys.error("Unsupported set of ordering / distribution clauses.") - } - - val withLimit = - limitClause.map(l => nodeToExpr(l.getChildren.iterator().next())) - .map(Limit(_, withSort)) - .getOrElse(withSort) - - // Collect all window specifications defined in the WINDOW clause. - val windowDefinitions = windowClause.map(_.getChildren.asScala.collect { - case Token("TOK_WINDOWDEF", - Token(windowName, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => - windowName -> nodesToWindowSpecification(spec) - }.toMap) - // Handle cases like - // window w1 as (partition by p_mfgr order by p_name - // range between 2 preceding and 2 following), - // w2 as w1 - val resolvedCrossReference = windowDefinitions.map { - windowDefMap => windowDefMap.map { - case (windowName, WindowSpecReference(other)) => - (windowName, windowDefMap(other).asInstanceOf[WindowSpecDefinition]) - case o => o.asInstanceOf[(String, WindowSpecDefinition)] - } - } - - val withWindowDefinitions = - resolvedCrossReference.map(WithWindowDefinition(_, withLimit)).getOrElse(withLimit) - - // TOK_INSERT_INTO means to add files to the table. - // TOK_DESTINATION means to overwrite the table. - val resultDestination = - (intoClause orElse destClause).getOrElse(sys.error("No destination found.")) - val overwrite = intoClause.isEmpty - nodeToDest( - resultDestination, - withWindowDefinitions, - overwrite) - } - - // If there are multiple INSERTS just UNION them together into on query. - val query = queries.reduceLeft(Union) - - // return With plan if there is CTE - cteRelations.map(With(query, _)).getOrElse(query) - - // HIVE-9039 renamed TOK_UNION => TOK_UNIONALL while adding TOK_UNIONDISTINCT - case Token("TOK_UNIONALL", left :: right :: Nil) => - Union(nodeToPlan(left, context), nodeToPlan(right, context)) - - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for $node:\n ${dumpTree(a).toString} ") - } - - val allJoinTokens = "(TOK_.*JOIN)".r - val laterViewToken = "TOK_LATERAL_VIEW(.*)".r - def nodeToRelation(node: Node, context: Context): LogicalPlan = node match { - case Token("TOK_SUBQUERY", - query :: Token(alias, Nil) :: Nil) => - Subquery(cleanIdentifier(alias), nodeToPlan(query, context)) - - case Token(laterViewToken(isOuter), selectClause :: relationClause :: Nil) => - val Token("TOK_SELECT", - Token("TOK_SELEXPR", clauses) :: Nil) = selectClause - - val alias = getClause("TOK_TABALIAS", clauses).getChildren.iterator().next() - .asInstanceOf[ASTNode].getText - - val (generator, attributes) = nodesToGenerator(clauses) - Generate( - generator, - join = true, - outer = isOuter.nonEmpty, - Some(alias.toLowerCase), - attributes.map(UnresolvedAttribute(_)), - nodeToRelation(relationClause, context)) - - /* All relations, possibly with aliases or sampling clauses. */ - case Token("TOK_TABREF", clauses) => - // If the last clause is not a token then it's the alias of the table. - val (nonAliasClauses, aliasClause) = - if (clauses.last.getText.startsWith("TOK")) { - (clauses, None) - } else { - (clauses.dropRight(1), Some(clauses.last)) - } - - val (Some(tableNameParts) :: - splitSampleClause :: - bucketSampleClause :: Nil) = { - getClauses(Seq("TOK_TABNAME", "TOK_TABLESPLITSAMPLE", "TOK_TABLEBUCKETSAMPLE"), - nonAliasClauses) - } - - val tableIdent = extractTableIdent(tableNameParts) - val alias = aliasClause.map { case Token(a, Nil) => cleanIdentifier(a) } - val relation = UnresolvedRelation(tableIdent, alias) - - // Apply sampling if requested. - (bucketSampleClause orElse splitSampleClause).map { - case Token("TOK_TABLESPLITSAMPLE", - Token("TOK_ROWCOUNT", Nil) :: - Token(count, Nil) :: Nil) => - Limit(Literal(count.toInt), relation) - case Token("TOK_TABLESPLITSAMPLE", - Token("TOK_PERCENT", Nil) :: - Token(fraction, Nil) :: Nil) => - // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling - // function takes X PERCENT as the input and the range of X is [0, 100], we need to - // adjust the fraction. - require( - fraction.toDouble >= (0.0 - RandomSampler.roundingEpsilon) - && fraction.toDouble <= (100.0 + RandomSampler.roundingEpsilon), - s"Sampling fraction ($fraction) must be on interval [0, 100]") - Sample(0.0, fraction.toDouble / 100, withReplacement = false, (math.random * 1000).toInt, - relation) - case Token("TOK_TABLEBUCKETSAMPLE", - Token(numerator, Nil) :: - Token(denominator, Nil) :: Nil) => - val fraction = numerator.toDouble / denominator.toDouble - Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, relation) - case a: ASTNode => - throw new NotImplementedError( - s"""No parse rules for sampling clause: ${a.getType}, text: ${a.getText} : - |${dumpTree(a).toString}" + - """.stripMargin) - }.getOrElse(relation) - - case Token("TOK_UNIQUEJOIN", joinArgs) => - val tableOrdinals = - joinArgs.zipWithIndex.filter { - case (arg, i) => arg.getText == "TOK_TABREF" - }.map(_._2) - - val isPreserved = tableOrdinals.map(i => (i - 1 < 0) || joinArgs(i - 1).getText == "PRESERVE") - val tables = tableOrdinals.map(i => nodeToRelation(joinArgs(i), context)) - val joinExpressions = - tableOrdinals.map(i => joinArgs(i + 1).getChildren.asScala.map(nodeToExpr)) - - val joinConditions = joinExpressions.sliding(2).map { - case Seq(c1, c2) => - val predicates = (c1, c2).zipped.map { case (e1, e2) => EqualTo(e1, e2): Expression } - predicates.reduceLeft(And) - }.toBuffer - - val joinType = isPreserved.sliding(2).map { - case Seq(true, true) => FullOuter - case Seq(true, false) => LeftOuter - case Seq(false, true) => RightOuter - case Seq(false, false) => Inner - }.toBuffer - - val joinedTables = tables.reduceLeft(Join(_, _, Inner, None)) - - // Must be transform down. - val joinedResult = joinedTables transform { - case j: Join => - j.copy( - condition = Some(joinConditions.remove(joinConditions.length - 1)), - joinType = joinType.remove(joinType.length - 1)) - } - - val groups = joinExpressions.head.indices.map(i => Coalesce(joinExpressions.map(_(i)))) - - // Unique join is not really the same as an outer join so we must group together results where - // the joinExpressions are the same, taking the First of each value is only okay because the - // user of a unique join is implicitly promising that there is only one result. - // TODO: This doesn't actually work since [[Star]] is not a valid aggregate expression. - // instead we should figure out how important supporting this feature is and whether it is - // worth the number of hacks that will be required to implement it. Namely, we need to add - // some sort of mapped star expansion that would expand all child output row to be similarly - // named output expressions where some aggregate expression has been applied (i.e. First). - // Aggregate(groups, Star(None, First(_)) :: Nil, joinedResult) - throw new UnsupportedOperationException - - case Token(allJoinTokens(joinToken), - relation1 :: - relation2 :: other) => - if (!(other.size <= 1)) { - sys.error(s"Unsupported join operation: $other") - } - - val joinType = joinToken match { - case "TOK_JOIN" => Inner - case "TOK_CROSSJOIN" => Inner - case "TOK_RIGHTOUTERJOIN" => RightOuter - case "TOK_LEFTOUTERJOIN" => LeftOuter - case "TOK_FULLOUTERJOIN" => FullOuter - case "TOK_LEFTSEMIJOIN" => LeftSemi - } - Join(nodeToRelation(relation1, context), - nodeToRelation(relation2, context), - joinType, - other.headOption.map(nodeToExpr)) - - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") - } - - def nodeToSortOrder(node: Node): SortOrder = node match { - case Token("TOK_TABSORTCOLNAMEASC", sortExpr :: Nil) => - SortOrder(nodeToExpr(sortExpr), Ascending) - case Token("TOK_TABSORTCOLNAMEDESC", sortExpr :: Nil) => - SortOrder(nodeToExpr(sortExpr), Descending) - - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") - } - - val destinationToken = "TOK_DESTINATION|TOK_INSERT_INTO".r - protected def nodeToDest( - node: Node, - query: LogicalPlan, - overwrite: Boolean): LogicalPlan = node match { - case Token(destinationToken(), - Token("TOK_DIR", - Token("TOK_TMP_FILE", Nil) :: Nil) :: Nil) => - query - - case Token(destinationToken(), - Token("TOK_TAB", - tableArgs) :: Nil) => - val Some(tableNameParts) :: partitionClause :: Nil = - getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) - - val tableIdent = extractTableIdent(tableNameParts) - - val partitionKeys = partitionClause.map(_.getChildren.asScala.map { - // Parse partitions. We also make keys case insensitive. - case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => - cleanIdentifier(key.toLowerCase) -> Some(PlanUtils.stripQuotes(value)) - case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => - cleanIdentifier(key.toLowerCase) -> None - }.toMap).getOrElse(Map.empty) - - InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, false) - - case Token(destinationToken(), - Token("TOK_TAB", - tableArgs) :: - Token("TOK_IFNOTEXISTS", - ifNotExists) :: Nil) => - val Some(tableNameParts) :: partitionClause :: Nil = - getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) - - val tableIdent = extractTableIdent(tableNameParts) - - val partitionKeys = partitionClause.map(_.getChildren.asScala.map { - // Parse partitions. We also make keys case insensitive. - case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => - cleanIdentifier(key.toLowerCase) -> Some(PlanUtils.stripQuotes(value)) - case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => - cleanIdentifier(key.toLowerCase) -> None - }.toMap).getOrElse(Map.empty) - - InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, true) - - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for ${a.getName}:" + - s"\n ${dumpTree(a).toString} ") - } - - protected def selExprNodeToExpr(node: Node): Option[Expression] = node match { - case Token("TOK_SELEXPR", e :: Nil) => - Some(nodeToExpr(e)) - - case Token("TOK_SELEXPR", e :: Token(alias, Nil) :: Nil) => - Some(Alias(nodeToExpr(e), cleanIdentifier(alias))()) - - case Token("TOK_SELEXPR", e :: aliasChildren) => - var aliasNames = ArrayBuffer[String]() - aliasChildren.foreach { _ match { - case Token(name, Nil) => aliasNames += cleanIdentifier(name) - case _ => - } - } - Some(MultiAlias(nodeToExpr(e), aliasNames)) - - /* Hints are ignored */ - case Token("TOK_HINTLIST", _) => None - - case a: ASTNode => - throw new NotImplementedError(s"No parse rules for ${a.getName }:" + - s"\n ${dumpTree(a).toString } ") - } - - protected val escapedIdentifier = "`([^`]+)`".r - protected val doubleQuotedString = "\"([^\"]+)\"".r - protected val singleQuotedString = "'([^']+)'".r - - protected def unquoteString(str: String) = str match { - case singleQuotedString(s) => s - case doubleQuotedString(s) => s - case other => other - } - - /** Strips backticks from ident if present */ - protected def cleanIdentifier(ident: String): String = ident match { - case escapedIdentifier(i) => i - case plainIdent => plainIdent - } - - val numericAstTypes = Seq( - HiveParser.Number, - HiveParser.TinyintLiteral, - HiveParser.SmallintLiteral, - HiveParser.BigintLiteral, - HiveParser.DecimalLiteral) - - /* Case insensitive matches */ - val COUNT = "(?i)COUNT".r - val SUM = "(?i)SUM".r - val AND = "(?i)AND".r - val OR = "(?i)OR".r - val NOT = "(?i)NOT".r - val TRUE = "(?i)TRUE".r - val FALSE = "(?i)FALSE".r - val LIKE = "(?i)LIKE".r - val RLIKE = "(?i)RLIKE".r - val REGEXP = "(?i)REGEXP".r - val IN = "(?i)IN".r - val DIV = "(?i)DIV".r - val BETWEEN = "(?i)BETWEEN".r - val WHEN = "(?i)WHEN".r - val CASE = "(?i)CASE".r - - protected def nodeToExpr(node: Node): Expression = node match { - /* Attribute References */ - case Token("TOK_TABLE_OR_COL", - Token(name, Nil) :: Nil) => - UnresolvedAttribute.quoted(cleanIdentifier(name)) - case Token(".", qualifier :: Token(attr, Nil) :: Nil) => - nodeToExpr(qualifier) match { - case UnresolvedAttribute(nameParts) => - UnresolvedAttribute(nameParts :+ cleanIdentifier(attr)) - case other => UnresolvedExtractValue(other, Literal(attr)) - } - - /* Stars (*) */ - case Token("TOK_ALLCOLREF", Nil) => UnresolvedStar(None) - // The format of dbName.tableName.* cannot be parsed by HiveParser. TOK_TABNAME will only - // has a single child which is tableName. - case Token("TOK_ALLCOLREF", Token("TOK_TABNAME", Token(name, Nil) :: Nil) :: Nil) => - UnresolvedStar(Some(UnresolvedAttribute.parseAttributeName(name))) - - /* Aggregate Functions */ - case Token("TOK_FUNCTIONSTAR", Token(COUNT(), Nil) :: Nil) => Count(Literal(1)) - case Token("TOK_FUNCTIONDI", Token(COUNT(), Nil) :: args) => CountDistinct(args.map(nodeToExpr)) - case Token("TOK_FUNCTIONDI", Token(SUM(), Nil) :: arg :: Nil) => SumDistinct(nodeToExpr(arg)) - - /* Casts */ - case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), StringType) - case Token("TOK_FUNCTION", Token("TOK_VARCHAR", _) :: arg :: Nil) => - Cast(nodeToExpr(arg), StringType) - case Token("TOK_FUNCTION", Token("TOK_CHAR", _) :: arg :: Nil) => - Cast(nodeToExpr(arg), StringType) - case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), IntegerType) - case Token("TOK_FUNCTION", Token("TOK_BIGINT", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), LongType) - case Token("TOK_FUNCTION", Token("TOK_FLOAT", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), FloatType) - case Token("TOK_FUNCTION", Token("TOK_DOUBLE", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), DoubleType) - case Token("TOK_FUNCTION", Token("TOK_SMALLINT", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), ShortType) - case Token("TOK_FUNCTION", Token("TOK_TINYINT", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), ByteType) - case Token("TOK_FUNCTION", Token("TOK_BINARY", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), BinaryType) - case Token("TOK_FUNCTION", Token("TOK_BOOLEAN", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), BooleanType) - case Token("TOK_FUNCTION", Token("TOK_DECIMAL", precision :: scale :: nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), DecimalType(precision.getText.toInt, scale.getText.toInt)) - case Token("TOK_FUNCTION", Token("TOK_DECIMAL", precision :: Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), DecimalType(precision.getText.toInt, 0)) - case Token("TOK_FUNCTION", Token("TOK_DECIMAL", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), DecimalType.USER_DEFAULT) - case Token("TOK_FUNCTION", Token("TOK_TIMESTAMP", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), TimestampType) - case Token("TOK_FUNCTION", Token("TOK_DATE", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), DateType) - - /* Arithmetic */ - 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)) - case Token("-", left :: right:: Nil) => Subtract(nodeToExpr(left), nodeToExpr(right)) - case Token("*", left :: right:: Nil) => Multiply(nodeToExpr(left), nodeToExpr(right)) - case Token("/", left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) - case Token(DIV(), left :: right:: Nil) => - Cast(Divide(nodeToExpr(left), nodeToExpr(right)), LongType) - case Token("%", left :: right:: Nil) => Remainder(nodeToExpr(left), nodeToExpr(right)) - case Token("&", left :: right:: Nil) => BitwiseAnd(nodeToExpr(left), nodeToExpr(right)) - case Token("|", left :: right:: Nil) => BitwiseOr(nodeToExpr(left), nodeToExpr(right)) - case Token("^", left :: right:: Nil) => BitwiseXor(nodeToExpr(left), nodeToExpr(right)) - - /* Comparisons */ - case Token("=", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right)) - case Token("==", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right)) - case Token("<=>", left :: right:: Nil) => EqualNullSafe(nodeToExpr(left), nodeToExpr(right)) - case Token("!=", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right))) - case Token("<>", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right))) - case Token(">", left :: right:: Nil) => GreaterThan(nodeToExpr(left), nodeToExpr(right)) - case Token(">=", left :: right:: Nil) => GreaterThanOrEqual(nodeToExpr(left), nodeToExpr(right)) - case Token("<", left :: right:: Nil) => LessThan(nodeToExpr(left), nodeToExpr(right)) - case Token("<=", left :: right:: Nil) => LessThanOrEqual(nodeToExpr(left), nodeToExpr(right)) - case Token(LIKE(), left :: right:: Nil) => Like(nodeToExpr(left), nodeToExpr(right)) - case Token(RLIKE(), left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) - case Token(REGEXP(), left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) - case Token("TOK_FUNCTION", Token("TOK_ISNOTNULL", Nil) :: child :: Nil) => - IsNotNull(nodeToExpr(child)) - case Token("TOK_FUNCTION", Token("TOK_ISNULL", Nil) :: child :: Nil) => - IsNull(nodeToExpr(child)) - case Token("TOK_FUNCTION", Token(IN(), Nil) :: value :: list) => - In(nodeToExpr(value), list.map(nodeToExpr)) - case Token("TOK_FUNCTION", - Token(BETWEEN(), Nil) :: - kw :: - target :: - minValue :: - maxValue :: Nil) => - - val targetExpression = nodeToExpr(target) - val betweenExpr = - And( - GreaterThanOrEqual(targetExpression, nodeToExpr(minValue)), - LessThanOrEqual(targetExpression, nodeToExpr(maxValue))) - kw match { - case Token("KW_FALSE", Nil) => betweenExpr - case Token("KW_TRUE", Nil) => Not(betweenExpr) - } - - /* Boolean Logic */ - case Token(AND(), left :: right:: Nil) => And(nodeToExpr(left), nodeToExpr(right)) - case Token(OR(), left :: right:: Nil) => Or(nodeToExpr(left), nodeToExpr(right)) - case Token(NOT(), child :: Nil) => Not(nodeToExpr(child)) - case Token("!", child :: Nil) => Not(nodeToExpr(child)) - - /* Case statements */ - case Token("TOK_FUNCTION", Token(WHEN(), Nil) :: branches) => - CaseWhen(branches.map(nodeToExpr)) - case Token("TOK_FUNCTION", Token(CASE(), Nil) :: branches) => - val keyExpr = nodeToExpr(branches.head) - CaseKeyWhen(keyExpr, branches.drop(1).map(nodeToExpr)) - - /* Complex datatype manipulation */ - case Token("[", child :: ordinal :: Nil) => - UnresolvedExtractValue(nodeToExpr(child), nodeToExpr(ordinal)) - - /* Window Functions */ - case Token("TOK_FUNCTION", Token(name, Nil) +: args :+ Token("TOK_WINDOWSPEC", spec)) => - val function = UnresolvedWindowFunction(name, args.map(nodeToExpr)) - nodesToWindowSpecification(spec) match { - case reference: WindowSpecReference => - UnresolvedWindowExpression(function, reference) - case definition: WindowSpecDefinition => - WindowExpression(function, definition) - } - case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => - // Safe to use Literal(1)? - val function = UnresolvedWindowFunction(name, Literal(1) :: Nil) - nodesToWindowSpecification(spec) match { - case reference: WindowSpecReference => - UnresolvedWindowExpression(function, reference) - case definition: WindowSpecDefinition => - WindowExpression(function, definition) - } - - /* UDFs - Must be last otherwise will preempt built in functions */ - case Token("TOK_FUNCTION", Token(name, Nil) :: args) => - UnresolvedFunction(name, args.map(nodeToExpr), isDistinct = false) - // Aggregate function with DISTINCT keyword. - case Token("TOK_FUNCTIONDI", Token(name, Nil) :: args) => - UnresolvedFunction(name, args.map(nodeToExpr), isDistinct = true) - case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: args) => - UnresolvedFunction(name, UnresolvedStar(None) :: Nil, isDistinct = false) - - /* Literals */ - case Token("TOK_NULL", Nil) => Literal.create(null, NullType) - case Token(TRUE(), Nil) => Literal.create(true, BooleanType) - case Token(FALSE(), Nil) => Literal.create(false, BooleanType) - case Token("TOK_STRINGLITERALSEQUENCE", strings) => - Literal(strings.map(s => BaseSemanticAnalyzer.unescapeSQLString(s.getText)).mkString) - - // This code is adapted from - // /ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java#L223 - case ast: ASTNode if numericAstTypes contains ast.getType => - var v: Literal = null - try { - if (ast.getText.endsWith("L")) { - // Literal bigint. - v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toLong, LongType) - } else if (ast.getText.endsWith("S")) { - // Literal smallint. - v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toShort, ShortType) - } else if (ast.getText.endsWith("Y")) { - // Literal tinyint. - v = Literal.create(ast.getText.substring(0, ast.getText.length() - 1).toByte, ByteType) - } else if (ast.getText.endsWith("BD") || ast.getText.endsWith("D")) { - // Literal decimal - val strVal = ast.getText.stripSuffix("D").stripSuffix("B") - v = Literal(Decimal(strVal)) - } else { - v = Literal.create(ast.getText.toDouble, DoubleType) - v = Literal.create(ast.getText.toLong, LongType) - v = Literal.create(ast.getText.toInt, IntegerType) - } - } catch { - case nfe: NumberFormatException => // Do nothing - } - - if (v == null) { - sys.error(s"Failed to parse number '${ast.getText}'.") - } else { - v - } - - case ast: ASTNode if ast.getType == HiveParser.StringLiteral => - Literal(BaseSemanticAnalyzer.unescapeSQLString(ast.getText)) - - case ast: ASTNode if ast.getType == HiveParser.TOK_DATELITERAL => - Literal(Date.valueOf(ast.getText.substring(1, ast.getText.length - 1))) - - case ast: ASTNode if ast.getType == HiveParser.TOK_CHARSETLITERAL => - Literal(BaseSemanticAnalyzer.charSetString(ast.getChild(0).getText, ast.getChild(1).getText)) - - case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_YEAR_MONTH_LITERAL => - Literal(CalendarInterval.fromYearMonthString(ast.getText)) - - case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL => - Literal(CalendarInterval.fromDayTimeString(ast.getText)) - - case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_YEAR_LITERAL => - Literal(CalendarInterval.fromSingleUnitString("year", ast.getText)) - - case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_MONTH_LITERAL => - Literal(CalendarInterval.fromSingleUnitString("month", ast.getText)) - - case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_DAY_LITERAL => - Literal(CalendarInterval.fromSingleUnitString("day", ast.getText)) - - case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_HOUR_LITERAL => - Literal(CalendarInterval.fromSingleUnitString("hour", ast.getText)) - - case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_MINUTE_LITERAL => - Literal(CalendarInterval.fromSingleUnitString("minute", ast.getText)) - - case ast: ASTNode if ast.getType == HiveParser.TOK_INTERVAL_SECOND_LITERAL => - Literal(CalendarInterval.fromSingleUnitString("second", ast.getText)) - - case a: ASTNode => - throw new NotImplementedError( - s"""No parse rules for ASTNode type: ${a.getType}, text: ${a.getText} : - |${dumpTree(a).toString}" + - """.stripMargin) - } - - /* Case insensitive matches for Window Specification */ - val PRECEDING = "(?i)preceding".r - val FOLLOWING = "(?i)following".r - val CURRENT = "(?i)current".r - def nodesToWindowSpecification(nodes: Seq[ASTNode]): WindowSpec = nodes match { - case Token(windowName, Nil) :: Nil => - // Refer to a window spec defined in the window clause. - WindowSpecReference(windowName) - case Nil => - // OVER() - WindowSpecDefinition( - partitionSpec = Nil, - orderSpec = Nil, - frameSpecification = UnspecifiedFrame) - case spec => - val (partitionClause :: rowFrame :: rangeFrame :: Nil) = - getClauses( - Seq( - "TOK_PARTITIONINGSPEC", - "TOK_WINDOWRANGE", - "TOK_WINDOWVALUES"), - spec) - - // Handle Partition By and Order By. - val (partitionSpec, orderSpec) = partitionClause.map { partitionAndOrdering => - val (partitionByClause :: orderByClause :: sortByClause :: clusterByClause :: Nil) = - getClauses( - Seq("TOK_DISTRIBUTEBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_CLUSTERBY"), - partitionAndOrdering.getChildren.asScala.asInstanceOf[Seq[ASTNode]]) - - (partitionByClause, orderByClause.orElse(sortByClause), clusterByClause) match { - case (Some(partitionByExpr), Some(orderByExpr), None) => - (partitionByExpr.getChildren.asScala.map(nodeToExpr), - orderByExpr.getChildren.asScala.map(nodeToSortOrder)) - case (Some(partitionByExpr), None, None) => - (partitionByExpr.getChildren.asScala.map(nodeToExpr), Nil) - case (None, Some(orderByExpr), None) => - (Nil, orderByExpr.getChildren.asScala.map(nodeToSortOrder)) - case (None, None, Some(clusterByExpr)) => - val expressions = clusterByExpr.getChildren.asScala.map(nodeToExpr) - (expressions, expressions.map(SortOrder(_, Ascending))) - case _ => - throw new NotImplementedError( - s"""No parse rules for Node ${partitionAndOrdering.getName} - """.stripMargin) - } - }.getOrElse { - (Nil, Nil) - } - - // Handle Window Frame - val windowFrame = - if (rowFrame.isEmpty && rangeFrame.isEmpty) { - UnspecifiedFrame - } else { - val frameType = rowFrame.map(_ => RowFrame).getOrElse(RangeFrame) - def nodeToBoundary(node: Node): FrameBoundary = node match { - case Token(PRECEDING(), Token(count, Nil) :: Nil) => - if (count.toLowerCase() == "unbounded") { - UnboundedPreceding - } else { - ValuePreceding(count.toInt) - } - case Token(FOLLOWING(), Token(count, Nil) :: Nil) => - if (count.toLowerCase() == "unbounded") { - UnboundedFollowing - } else { - ValueFollowing(count.toInt) - } - case Token(CURRENT(), Nil) => CurrentRow - case _ => - throw new NotImplementedError( - s"""No parse rules for the Window Frame Boundary based on Node ${node.getName} - """.stripMargin) - } - - rowFrame.orElse(rangeFrame).map { frame => - frame.getChildren.asScala.toList match { - case precedingNode :: followingNode :: Nil => - SpecifiedWindowFrame( - frameType, - nodeToBoundary(precedingNode), - nodeToBoundary(followingNode)) - case precedingNode :: Nil => - SpecifiedWindowFrame(frameType, nodeToBoundary(precedingNode), CurrentRow) - case _ => - throw new NotImplementedError( - s"""No parse rules for the Window Frame based on Node ${frame.getName} - """.stripMargin) - } - }.getOrElse(sys.error(s"If you see this, please file a bug report with your query.")) - } - - WindowSpecDefinition(partitionSpec, orderSpec, windowFrame) - } - - val explode = "(?i)explode".r - def nodesToGenerator(nodes: Seq[Node]): (Generator, Seq[String]) = { - val function = nodes.head - - val attributes = nodes.flatMap { - case Token(a, Nil) => a.toLowerCase :: Nil - case _ => Nil - } - - function match { - case Token("TOK_FUNCTION", Token(explode(), Nil) :: child :: Nil) => - (Explode(nodeToExpr(child)), attributes) - - case Token("TOK_FUNCTION", Token(functionName, Nil) :: children) => - val functionInfo: FunctionInfo = - Option(FunctionRegistry.getFunctionInfo(functionName.toLowerCase)).getOrElse( - sys.error(s"Couldn't find function $functionName")) - val functionClassName = functionInfo.getFunctionClass.getName - - (HiveGenericUDTF( - new HiveFunctionWrapper(functionClassName), - children.map(nodeToExpr)), attributes) - - case a: ASTNode => - throw new NotImplementedError( - s"""No parse rules for ASTNode type: ${a.getType}, text: ${a.getText}, tree: - |${dumpTree(a).toString} - """.stripMargin) - } - } - - def dumpTree(node: Node, builder: StringBuilder = new StringBuilder, indent: Int = 0) - : StringBuilder = { - node match { - 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") - } - - Option(node.getChildren).map(_.asScala).getOrElse(Nil).foreach(dumpTree(_, builder, indent + 1)) - builder - } -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala new file mode 100644 index 000000000000..b256ffc27b19 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.Locale + +import scala.util.{Failure, Success, Try} +import scala.util.control.NonFatal + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.ql.exec.{UDAF, UDF} +import org.apache.hadoop.hive.ql.exec.{FunctionRegistry => HiveFunctionRegistry} +import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, GenericUDF, GenericUDTF} + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry +import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, FunctionResourceLoader, GlobalTempViewManager, SessionCatalog} +import org.apache.spark.sql.catalyst.expressions.{Cast, Expression} +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DecimalType, DoubleType} + + +private[sql] class HiveSessionCatalog( + externalCatalog: HiveExternalCatalog, + globalTempViewManager: GlobalTempViewManager, + val metastoreCatalog: HiveMetastoreCatalog, + functionRegistry: FunctionRegistry, + conf: SQLConf, + hadoopConf: Configuration, + parser: ParserInterface, + functionResourceLoader: FunctionResourceLoader) + extends SessionCatalog( + externalCatalog, + globalTempViewManager, + functionRegistry, + conf, + hadoopConf, + parser, + functionResourceLoader) { + + /** + * Constructs a [[Expression]] based on the provided class that represents a function. + * + * This performs reflection to decide what type of [[Expression]] to return in the builder. + */ + override def makeFunctionExpression( + name: String, + clazz: Class[_], + input: Seq[Expression]): Expression = { + + Try(super.makeFunctionExpression(name, clazz, input)).getOrElse { + var udfExpr: Option[Expression] = None + try { + // When we instantiate hive UDF wrapper class, we may throw exception if the input + // expressions don't satisfy the hive UDF, such as type mismatch, input number + // mismatch, etc. Here we catch the exception and throw AnalysisException instead. + if (classOf[UDF].isAssignableFrom(clazz)) { + udfExpr = Some(HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), input)) + udfExpr.get.dataType // Force it to check input data types. + } else if (classOf[GenericUDF].isAssignableFrom(clazz)) { + udfExpr = Some(HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), input)) + udfExpr.get.dataType // Force it to check input data types. + } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) { + udfExpr = Some(HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), input)) + udfExpr.get.dataType // Force it to check input data types. + } else if (classOf[UDAF].isAssignableFrom(clazz)) { + udfExpr = Some(HiveUDAFFunction( + name, + new HiveFunctionWrapper(clazz.getName), + input, + isUDAFBridgeRequired = true)) + udfExpr.get.dataType // Force it to check input data types. + } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) { + udfExpr = Some(HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), input)) + udfExpr.get.asInstanceOf[HiveGenericUDTF].elementSchema // Force it to check data types. + } + } catch { + case NonFatal(e) => + val analysisException = + new AnalysisException(s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}': $e") + analysisException.setStackTrace(e.getStackTrace) + throw analysisException + } + udfExpr.getOrElse { + throw new AnalysisException(s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}'") + } + } + } + + override def lookupFunction(name: FunctionIdentifier, children: Seq[Expression]): Expression = { + try { + lookupFunction0(name, children) + } catch { + case NonFatal(_) => + // SPARK-16228 ExternalCatalog may recognize `double`-type only. + val newChildren = children.map { child => + if (child.dataType.isInstanceOf[DecimalType]) Cast(child, DoubleType) else child + } + lookupFunction0(name, newChildren) + } + } + + private def lookupFunction0(name: FunctionIdentifier, children: Seq[Expression]): Expression = { + val database = name.database.map(formatDatabaseName) + val funcName = name.copy(database = database) + Try(super.lookupFunction(funcName, children)) match { + case Success(expr) => expr + case Failure(error) => + if (functionRegistry.functionExists(funcName)) { + // If the function actually exists in functionRegistry, it means that there is an + // error when we create the Expression using the given children. + // We need to throw the original exception. + throw error + } else { + // This function is not in functionRegistry, let's try to load it as a Hive's + // built-in function. + // Hive is case insensitive. + val functionName = funcName.unquotedString.toLowerCase(Locale.ROOT) + if (!hiveFunctions.contains(functionName)) { + failFunctionLookup(funcName) + } + + // TODO: Remove this fallback path once we implement the list of fallback functions + // defined below in hiveFunctions. + val functionInfo = { + try { + Option(HiveFunctionRegistry.getFunctionInfo(functionName)).getOrElse( + failFunctionLookup(funcName)) + } catch { + // If HiveFunctionRegistry.getFunctionInfo throws an exception, + // we are failing to load a Hive builtin function, which means that + // the given function is not a Hive builtin function. + case NonFatal(e) => failFunctionLookup(funcName) + } + } + val className = functionInfo.getFunctionClass.getName + val functionIdentifier = + FunctionIdentifier(functionName.toLowerCase(Locale.ROOT), database) + val func = CatalogFunction(functionIdentifier, className, Nil) + // Put this Hive built-in function to our function registry. + registerFunction(func, overrideIfExists = false) + // Now, we need to create the Expression. + functionRegistry.lookupFunction(functionIdentifier, children) + } + } + } + + // TODO Removes this method after implementing Spark native "histogram_numeric". + override def functionExists(name: FunctionIdentifier): Boolean = { + super.functionExists(name) || hiveFunctions.contains(name.funcName) + } + + /** List of functions we pass over to Hive. Note that over time this list should go to 0. */ + // We have a list of Hive built-in functions that we do not support. So, we will check + // Hive's function registry and lazily load needed functions into our own function registry. + // List of functions we are explicitly not supporting are: + // compute_stats, context_ngrams, create_union, + // current_user, ewah_bitmap, ewah_bitmap_and, ewah_bitmap_empty, ewah_bitmap_or, field, + // in_file, index, matchpath, ngrams, noop, noopstreaming, noopwithmap, + // noopwithmapstreaming, parse_url_tuple, reflect2, windowingtablefunction. + // Note: don't forget to update SessionCatalog.isTemporaryFunction + private val hiveFunctions = Seq( + "histogram_numeric" + ) +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala new file mode 100644 index 000000000000..92cb4ef11c9e --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.Analyzer +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlanner +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.hive.client.HiveClient +import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SessionResourceLoader, SessionState} + +/** + * Builder that produces a Hive-aware `SessionState`. + */ +@Experimental +@InterfaceStability.Unstable +class HiveSessionStateBuilder(session: SparkSession, parentState: Option[SessionState] = None) + extends BaseSessionStateBuilder(session, parentState) { + + private def externalCatalog: HiveExternalCatalog = + session.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog] + + /** + * Create a Hive aware resource loader. + */ + override protected lazy val resourceLoader: HiveSessionResourceLoader = { + val client: HiveClient = externalCatalog.client.newSession() + new HiveSessionResourceLoader(session, client) + } + + /** + * Create a [[HiveSessionCatalog]]. + */ + override protected lazy val catalog: HiveSessionCatalog = { + val catalog = new HiveSessionCatalog( + externalCatalog, + session.sharedState.globalTempViewManager, + new HiveMetastoreCatalog(session), + functionRegistry, + conf, + SessionState.newHadoopConf(session.sparkContext.hadoopConfiguration, conf), + sqlParser, + resourceLoader) + parentState.foreach(_.catalog.copyStateTo(catalog)) + catalog + } + + /** + * A logical query plan `Analyzer` with rules specific to Hive. + */ + override protected def analyzer: Analyzer = new Analyzer(catalog, conf) { + override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = + new ResolveHiveSerdeTable(session) +: + new FindDataSourceTable(session) +: + new ResolveSQLOnFile(session) +: + customResolutionRules + + override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = + new DetermineTableStats(session) +: + RelationConversions(conf, catalog) +: + PreprocessTableCreation(session) +: + PreprocessTableInsertion(conf) +: + DataSourceAnalysis(conf) +: + HiveAnalysis +: + customPostHocResolutionRules + + override val extendedCheckRules: Seq[LogicalPlan => Unit] = + PreWriteCheck +: + PreReadCheck +: + customCheckRules + } + + /** + * Planner that takes into account Hive-specific strategies. + */ + override protected def planner: SparkPlanner = { + new SparkPlanner(session.sparkContext, conf, experimentalMethods) with HiveStrategies { + override val sparkSession: SparkSession = session + + override def extraPlanningStrategies: Seq[Strategy] = + super.extraPlanningStrategies ++ customPlanningStrategies + + override def strategies: Seq[Strategy] = { + experimentalMethods.extraStrategies ++ + extraPlanningStrategies ++ Seq( + FileSourceStrategy, + DataSourceStrategy(conf), + SpecialLimits, + InMemoryScans, + HiveTableScans, + Scripts, + Aggregation, + JoinSelection, + BasicOperators + ) + } + } + } + + override protected def newBuilder: NewBuilder = new HiveSessionStateBuilder(_, _) +} + +class HiveSessionResourceLoader( + session: SparkSession, + client: HiveClient) + extends SessionResourceLoader(session) { + override def addJar(path: String): Unit = { + client.addJar(path) + super.addJar(path) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala index f0697613cff3..9e9894803ce2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala @@ -20,25 +20,25 @@ package org.apache.spark.sql.hive import java.io.{InputStream, OutputStream} import java.rmi.server.UID -import org.apache.avro.Schema - import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.ClassTag -import com.esotericsoftware.kryo.Kryo -import com.esotericsoftware.kryo.io.{Input, Output} - +import com.google.common.base.Objects +import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.exec.{UDF, Utilities} import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro import org.apache.hadoop.hive.serde2.ColumnProjectionUtils import org.apache.hadoop.hive.serde2.avro.{AvroGenericRecordWritable, AvroSerdeUtils} import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector import org.apache.hadoop.io.Writable +import org.apache.hive.com.esotericsoftware.kryo.Kryo +import org.apache.hive.com.esotericsoftware.kryo.io.{Input, Output} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.types.Decimal import org.apache.spark.util.Utils @@ -47,6 +47,7 @@ private[hive] object HiveShim { // scale Hive 0.13 infers for BigDecimals from sources that don't specify them (e.g. UDFs) val UNLIMITED_DECIMAL_PRECISION = 38 val UNLIMITED_DECIMAL_SCALE = 18 + val HIVE_GENERIC_UDF_MACRO_CLS = "org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro" /* * This function in hive-0.13 become private, but we have to do this to walkaround hive bug @@ -68,13 +69,13 @@ private[hive] object HiveShim { } /* - * Cannot use ColumnProjectionUtils.appendReadColumns directly, if ids is null or empty + * Cannot use ColumnProjectionUtils.appendReadColumns directly, if ids is null */ def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { - if (ids != null && ids.nonEmpty) { + if (ids != null) { ColumnProjectionUtils.appendReadColumns(conf, ids.asJava) } - if (names != null && names.nonEmpty) { + if (names != null) { appendReadColumnNames(conf, names) } } @@ -125,6 +126,26 @@ private[hive] object HiveShim { // for Serialization def this() = this(null) + override def hashCode(): Int = { + if (functionClassName == HIVE_GENERIC_UDF_MACRO_CLS) { + Objects.hashCode(functionClassName, instance.asInstanceOf[GenericUDFMacro].getBody()) + } else { + functionClassName.hashCode() + } + } + + override def equals(other: Any): Boolean = other match { + case a: HiveFunctionWrapper if functionClassName == a.functionClassName => + // In case of udf macro, check to make sure they point to the same underlying UDF + if (functionClassName == HIVE_GENERIC_UDF_MACRO_CLS) { + a.instance.asInstanceOf[GenericUDFMacro].getBody() == + instance.asInstanceOf[GenericUDFMacro].getBody() + } else { + true + } + case _ => false + } + @transient def deserializeObjectByKryo[T: ClassTag]( kryo: Kryo, 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 d38ad9127327..805b3171cdaa 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -17,40 +17,217 @@ package org.apache.spark.sql.hive +import java.io.IOException +import java.util.Locale + +import org.apache.hadoop.fs.{FileSystem, Path} + import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ -import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.datasources.{CreateTableUsing, CreateTableUsingAsSelect, DescribeCommand} -import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand, _} +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTable, LogicalPlan, + ScriptTransformation} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.command.{CreateTableCommand, DDLUtils} +import org.apache.spark.sql.execution.datasources.{CreateTable, LogicalRelation} +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} import org.apache.spark.sql.hive.execution._ +import org.apache.spark.sql.hive.orc.OrcFileFormat +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} -private[hive] trait HiveStrategies { - // Possibly being too clever with types here... or not clever enough. - self: SQLContext#SparkPlanner => +/** + * Determine the database, serde/format and schema of the Hive serde table, according to the storage + * properties. + */ +class ResolveHiveSerdeTable(session: SparkSession) extends Rule[LogicalPlan] { + private def determineHiveSerde(table: CatalogTable): CatalogTable = { + if (table.storage.serde.nonEmpty) { + table + } else { + if (table.bucketSpec.isDefined) { + throw new AnalysisException("Creating bucketed Hive serde table is not supported yet.") + } - val hiveContext: HiveContext + val defaultStorage = HiveSerDe.getDefaultStorage(session.sessionState.conf) + val options = new HiveOptions(table.storage.properties) - object Scripts extends Strategy { - def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.ScriptTransformation(input, script, output, child, schema: HiveScriptIOSchema) => - ScriptTransformation(input, script, output, planLater(child), schema)(hiveContext) :: Nil - case _ => Nil + val fileStorage = if (options.fileFormat.isDefined) { + HiveSerDe.sourceToSerDe(options.fileFormat.get) match { + case Some(s) => + CatalogStorageFormat.empty.copy( + inputFormat = s.inputFormat, + outputFormat = s.outputFormat, + serde = s.serde) + case None => + throw new IllegalArgumentException(s"invalid fileFormat: '${options.fileFormat.get}'") + } + } else if (options.hasInputOutputFormat) { + CatalogStorageFormat.empty.copy( + inputFormat = options.inputFormat, + outputFormat = options.outputFormat) + } else { + CatalogStorageFormat.empty + } + + val rowStorage = if (options.serde.isDefined) { + CatalogStorageFormat.empty.copy(serde = options.serde) + } else { + CatalogStorageFormat.empty + } + + val storage = table.storage.copy( + inputFormat = fileStorage.inputFormat.orElse(defaultStorage.inputFormat), + outputFormat = fileStorage.outputFormat.orElse(defaultStorage.outputFormat), + serde = rowStorage.serde.orElse(fileStorage.serde).orElse(defaultStorage.serde), + properties = options.serdeProperties) + + table.copy(storage = storage) + } + } + + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case c @ CreateTable(t, _, query) if DDLUtils.isHiveTable(t) => + // Finds the database name if the name does not exist. + val dbName = t.identifier.database.getOrElse(session.catalog.currentDatabase) + val table = t.copy(identifier = t.identifier.copy(database = Some(dbName))) + + // Determines the serde/format of Hive tables + val withStorage = determineHiveSerde(table) + + // Infers the schema, if empty, because the schema could be determined by Hive + // serde. + val withSchema = if (query.isEmpty) { + val inferred = HiveUtils.inferSchema(withStorage) + if (inferred.schema.length <= 0) { + throw new AnalysisException("Unable to infer the schema. " + + s"The schema specification is required to create the table ${inferred.identifier}.") + } + inferred + } else { + withStorage + } + + c.copy(tableDesc = withSchema) + } +} + +class DetermineTableStats(session: SparkSession) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case relation: HiveTableRelation + if DDLUtils.isHiveTable(relation.tableMeta) && relation.tableMeta.stats.isEmpty => + val table = relation.tableMeta + val sizeInBytes = if (session.sessionState.conf.fallBackToHdfsForStatsEnabled) { + try { + val hadoopConf = session.sessionState.newHadoopConf() + val tablePath = new Path(table.location) + val fs: FileSystem = tablePath.getFileSystem(hadoopConf) + fs.getContentSummary(tablePath).getLength + } catch { + case e: IOException => + logWarning("Failed to get table size from hdfs.", e) + session.sessionState.conf.defaultSizeInBytes + } + } else { + session.sessionState.conf.defaultSizeInBytes + } + + val withStats = table.copy(stats = Some(CatalogStatistics(sizeInBytes = BigInt(sizeInBytes)))) + relation.copy(tableMeta = withStats) + } +} + +/** + * Replaces generic operations with specific variants that are designed to work with Hive. + * + * Note that, this rule must be run after `PreprocessTableCreation` and + * `PreprocessTableInsertion`. + */ +object HiveAnalysis extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case InsertIntoTable(r: HiveTableRelation, partSpec, query, overwrite, ifPartitionNotExists) + if DDLUtils.isHiveTable(r.tableMeta) => + InsertIntoHiveTable(r.tableMeta, partSpec, query, overwrite, ifPartitionNotExists) + + case CreateTable(tableDesc, mode, None) if DDLUtils.isHiveTable(tableDesc) => + DDLUtils.checkDataSchemaFieldNames(tableDesc) + CreateTableCommand(tableDesc, ignoreIfExists = mode == SaveMode.Ignore) + + case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) => + DDLUtils.checkDataSchemaFieldNames(tableDesc) + CreateHiveTableAsSelectCommand(tableDesc, query, mode) + + case InsertIntoDir(isLocal, storage, provider, child, overwrite) + if DDLUtils.isHiveTable(provider) => + val outputPath = new Path(storage.locationUri.get) + if (overwrite) DDLUtils.verifyNotReadPath(child, outputPath) + + InsertIntoHiveDirCommand(isLocal, storage, child, overwrite) + } +} + +/** + * Relation conversion from metastore relations to data source relations for better performance + * + * - When writing to non-partitioned Hive-serde Parquet/Orc tables + * - When scanning Hive-serde Parquet/ORC tables + * + * This rule must be run before all other DDL post-hoc resolution rules, i.e. + * `PreprocessTableCreation`, `PreprocessTableInsertion`, `DataSourceAnalysis` and `HiveAnalysis`. + */ +case class RelationConversions( + conf: SQLConf, + sessionCatalog: HiveSessionCatalog) extends Rule[LogicalPlan] { + private def isConvertible(relation: HiveTableRelation): Boolean = { + val serde = relation.tableMeta.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) + serde.contains("parquet") && conf.getConf(HiveUtils.CONVERT_METASTORE_PARQUET) || + serde.contains("orc") && conf.getConf(HiveUtils.CONVERT_METASTORE_ORC) + } + + private def convert(relation: HiveTableRelation): LogicalRelation = { + val serde = relation.tableMeta.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) + if (serde.contains("parquet")) { + val options = Map(ParquetOptions.MERGE_SCHEMA -> + conf.getConf(HiveUtils.CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING).toString) + sessionCatalog.metastoreCatalog + .convertToLogicalRelation(relation, options, classOf[ParquetFileFormat], "parquet") + } else { + val options = Map[String, String]() + sessionCatalog.metastoreCatalog + .convertToLogicalRelation(relation, options, classOf[OrcFileFormat], "orc") + } + } + + override def apply(plan: LogicalPlan): LogicalPlan = { + plan transformUp { + // Write path + case InsertIntoTable(r: HiveTableRelation, partition, query, overwrite, ifPartitionNotExists) + // Inserting into partitioned table is not supported in Parquet/Orc data source (yet). + if query.resolved && DDLUtils.isHiveTable(r.tableMeta) && + !r.isPartitioned && isConvertible(r) => + InsertIntoTable(convert(r), partition, query, overwrite, ifPartitionNotExists) + + // Read path + case relation: HiveTableRelation + if DDLUtils.isHiveTable(relation.tableMeta) && isConvertible(relation) => + convert(relation) } } +} + +private[hive] trait HiveStrategies { + // Possibly being too clever with types here... or not clever enough. + self: SparkPlanner => + + val sparkSession: SparkSession - object DataSinks extends Strategy { + object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.InsertIntoTable( - table: MetastoreRelation, partition, child, overwrite, ifNotExists) => - execution.InsertIntoHiveTable( - table, partition, planLater(child), overwrite, ifNotExists) :: Nil - case hive.InsertIntoHiveTable( - table: MetastoreRelation, partition, child, overwrite, ifNotExists) => - execution.InsertIntoHiveTable( - table, partition, planLater(child), overwrite, ifNotExists) :: Nil + case ScriptTransformation(input, script, output, child, ioschema) => + val hiveIoSchema = HiveScriptIOSchema(ioschema) + ScriptTransformationExec(input, script, output, planLater(child), hiveIoSchema) :: Nil case _ => Nil } } @@ -61,10 +238,10 @@ private[hive] trait HiveStrategies { */ object HiveTableScans extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case PhysicalOperation(projectList, predicates, relation: MetastoreRelation) => + case PhysicalOperation(projectList, predicates, relation: HiveTableRelation) => // Filter out all predicates that only deal with partition keys, these are given to the // hive table scan operator to be used for partition pruning. - val partitionKeyIds = AttributeSet(relation.partitionKeys) + val partitionKeyIds = AttributeSet(relation.partitionCols) val (pruningPredicates, otherPredicates) = predicates.partition { predicate => !predicate.references.isEmpty && predicate.references.subsetOf(partitionKeyIds) @@ -74,47 +251,9 @@ private[hive] trait HiveStrategies { projectList, otherPredicates, identity[Seq[Expression]], - HiveTableScan(_, relation, pruningPredicates)(hiveContext)) :: Nil + HiveTableScanExec(_, relation, pruningPredicates)(sparkSession)) :: Nil case _ => Nil } } - - object HiveDDLStrategy extends Strategy { - def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case CreateTableUsing( - tableIdent, userSpecifiedSchema, provider, false, opts, allowExisting, managedIfNoPath) => - val cmd = - CreateMetastoreDataSource( - tableIdent, userSpecifiedSchema, provider, opts, allowExisting, managedIfNoPath) - ExecutedCommand(cmd) :: Nil - - case CreateTableUsingAsSelect( - tableIdent, provider, false, partitionCols, mode, opts, query) => - val cmd = - CreateMetastoreDataSourceAsSelect(tableIdent, provider, partitionCols, mode, opts, query) - ExecutedCommand(cmd) :: Nil - - case _ => Nil - } - } - - case class HiveCommandStrategy(context: HiveContext) extends Strategy { - def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case describe: DescribeCommand => - val resolvedTable = context.executePlan(describe.table).analyzed - resolvedTable match { - case t: MetastoreRelation => - ExecutedCommand( - DescribeHiveTableCommand(t, describe.output, describe.isExtended)) :: Nil - - case o: LogicalPlan => - val resultPlan = context.executePlan(o).executedPlan - ExecutedCommand(RunnableDescribeCommand( - resultPlan, describe.output, describe.isExtended)) :: Nil - } - - case _ => Nil - } - } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala new file mode 100644 index 000000000000..80b9a3dc9605 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -0,0 +1,497 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.net.{URL, URLClassLoader} +import java.nio.charset.StandardCharsets +import java.sql.Timestamp +import java.util.Locale +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.collection.mutable.HashMap +import scala.language.implicitConversions + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable} +import org.apache.hadoop.util.VersionInfo + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.execution.command.DDLUtils +import org.apache.spark.sql.hive.client._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf._ +import org.apache.spark.sql.internal.StaticSQLConf.{CATALOG_IMPLEMENTATION, WAREHOUSE_PATH} +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + + +private[spark] object HiveUtils extends Logging { + + def withHiveExternalCatalog(sc: SparkContext): SparkContext = { + sc.conf.set(CATALOG_IMPLEMENTATION.key, "hive") + sc + } + + /** The version of hive used internally by Spark SQL. */ + val hiveExecutionVersion: String = "1.2.1" + + val HIVE_METASTORE_VERSION = buildConf("spark.sql.hive.metastore.version") + .doc("Version of the Hive metastore. Available options are " + + s"0.12.0 through $hiveExecutionVersion.") + .stringConf + .createWithDefault(hiveExecutionVersion) + + val HIVE_EXECUTION_VERSION = buildConf("spark.sql.hive.version") + .doc("Version of Hive used internally by Spark SQL.") + .stringConf + .createWithDefault(hiveExecutionVersion) + + val HIVE_METASTORE_JARS = buildConf("spark.sql.hive.metastore.jars") + .doc(s""" + | Location of the jars that should be used to instantiate the HiveMetastoreClient. + | This property can be one of three options: " + | 1. "builtin" + | Use Hive ${hiveExecutionVersion}, which is bundled with the Spark assembly when + | -Phive is enabled. When this option is chosen, + | spark.sql.hive.metastore.version must be either + | ${hiveExecutionVersion} or not defined. + | 2. "maven" + | Use Hive jars of specified version downloaded from Maven repositories. + | 3. A classpath in the standard format for both Hive and Hadoop. + """.stripMargin) + .stringConf + .createWithDefault("builtin") + + val CONVERT_METASTORE_PARQUET = buildConf("spark.sql.hive.convertMetastoreParquet") + .doc("When set to true, the built-in Parquet reader and writer are used to process " + + "parquet tables created by using the HiveQL syntax, instead of Hive serde.") + .booleanConf + .createWithDefault(true) + + val CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING = + buildConf("spark.sql.hive.convertMetastoreParquet.mergeSchema") + .doc("When true, also tries to merge possibly different but compatible Parquet schemas in " + + "different Parquet data files. This configuration is only effective " + + "when \"spark.sql.hive.convertMetastoreParquet\" is true.") + .booleanConf + .createWithDefault(false) + + val CONVERT_METASTORE_ORC = buildConf("spark.sql.hive.convertMetastoreOrc") + .internal() + .doc("When set to true, the built-in ORC reader and writer are used to process " + + "ORC tables created by using the HiveQL syntax, instead of Hive serde.") + .booleanConf + .createWithDefault(false) + + val HIVE_METASTORE_SHARED_PREFIXES = buildConf("spark.sql.hive.metastore.sharedPrefixes") + .doc("A comma separated list of class prefixes that should be loaded using the classloader " + + "that is shared between Spark SQL and a specific version of Hive. An example of classes " + + "that should be shared is JDBC drivers that are needed to talk to the metastore. Other " + + "classes that need to be shared are those that interact with classes that are already " + + "shared. For example, custom appenders that are used by log4j.") + .stringConf + .toSequence + .createWithDefault(jdbcPrefixes) + + private def jdbcPrefixes = Seq( + "com.mysql.jdbc", "org.postgresql", "com.microsoft.sqlserver", "oracle.jdbc") + + val HIVE_METASTORE_BARRIER_PREFIXES = buildConf("spark.sql.hive.metastore.barrierPrefixes") + .doc("A comma separated list of class prefixes that should explicitly be reloaded for each " + + "version of Hive that Spark SQL is communicating with. For example, Hive UDFs that are " + + "declared in a prefix that typically would be shared (i.e. org.apache.spark.*).") + .stringConf + .toSequence + .createWithDefault(Nil) + + val HIVE_THRIFT_SERVER_ASYNC = buildConf("spark.sql.hive.thriftServer.async") + .doc("When set to true, Hive Thrift server executes SQL queries in an asynchronous way.") + .booleanConf + .createWithDefault(true) + + /** + * The version of the hive client that will be used to communicate with the metastore. Note that + * this does not necessarily need to be the same version of Hive that is used internally by + * Spark SQL for execution. + */ + private def hiveMetastoreVersion(conf: SQLConf): String = { + conf.getConf(HIVE_METASTORE_VERSION) + } + + /** + * The location of the jars that should be used to instantiate the HiveMetastoreClient. This + * property can be one of three options: + * - a classpath in the standard format for both hive and hadoop. + * - builtin - attempt to discover the jars that were used to load Spark SQL and use those. This + * option is only valid when using the execution version of Hive. + * - maven - download the correct version of hive on demand from maven. + */ + private def hiveMetastoreJars(conf: SQLConf): String = { + conf.getConf(HIVE_METASTORE_JARS) + } + + /** + * A comma separated list of class prefixes that should be loaded using the classloader that + * is shared between Spark SQL and a specific version of Hive. An example of classes that should + * be shared is JDBC drivers that are needed to talk to the metastore. Other classes that need + * to be shared are those that interact with classes that are already shared. For example, + * custom appenders that are used by log4j. + */ + private def hiveMetastoreSharedPrefixes(conf: SQLConf): Seq[String] = { + conf.getConf(HIVE_METASTORE_SHARED_PREFIXES).filterNot(_ == "") + } + + /** + * A comma separated list of class prefixes that should explicitly be reloaded for each version + * of Hive that Spark SQL is communicating with. For example, Hive UDFs that are declared in a + * prefix that typically would be shared (i.e. org.apache.spark.*) + */ + private def hiveMetastoreBarrierPrefixes(conf: SQLConf): Seq[String] = { + conf.getConf(HIVE_METASTORE_BARRIER_PREFIXES).filterNot(_ == "") + } + + /** + * Change time configurations needed to create a [[HiveClient]] into unified [[Long]] format. + */ + private[hive] def formatTimeVarsForHiveClient(hadoopConf: Configuration): Map[String, String] = { + // Hive 0.14.0 introduces timeout operations in HiveConf, and changes default values of a bunch + // of time `ConfVar`s by adding time suffixes (`s`, `ms`, and `d` etc.). This breaks backwards- + // compatibility when users are trying to connecting to a Hive metastore of lower version, + // because these options are expected to be integral values in lower versions of Hive. + // + // Here we enumerate all time `ConfVar`s and convert their values to numeric strings according + // to their output time units. + Seq( + ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY -> TimeUnit.SECONDS, + ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.METASTORE_CLIENT_SOCKET_LIFETIME -> TimeUnit.SECONDS, + ConfVars.HMSHANDLERINTERVAL -> TimeUnit.MILLISECONDS, + ConfVars.METASTORE_EVENT_DB_LISTENER_TTL -> TimeUnit.SECONDS, + ConfVars.METASTORE_EVENT_CLEAN_FREQ -> TimeUnit.SECONDS, + ConfVars.METASTORE_EVENT_EXPIRY_DURATION -> TimeUnit.SECONDS, + ConfVars.METASTORE_AGGREGATE_STATS_CACHE_TTL -> TimeUnit.SECONDS, + ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_WRITER_WAIT -> TimeUnit.MILLISECONDS, + ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_READER_WAIT -> TimeUnit.MILLISECONDS, + ConfVars.HIVES_AUTO_PROGRESS_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.HIVE_LOG_INCREMENTAL_PLAN_PROGRESS_INTERVAL -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_STATS_JDBC_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.HIVE_STATS_RETRIES_WAIT -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES -> TimeUnit.SECONDS, + ConfVars.HIVE_ZOOKEEPER_SESSION_TIMEOUT -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_ZOOKEEPER_CONNECTION_BASESLEEPTIME -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_TXN_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.HIVE_COMPACTOR_WORKER_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.HIVE_COMPACTOR_CHECK_INTERVAL -> TimeUnit.SECONDS, + ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_SERVER2_THRIFT_HTTP_WORKER_KEEPALIVE_TIME -> TimeUnit.SECONDS, + ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_MAX_AGE -> TimeUnit.SECONDS, + ConfVars.HIVE_SERVER2_THRIFT_LOGIN_BEBACKOFF_SLOT_LENGTH -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_SERVER2_THRIFT_LOGIN_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.HIVE_SERVER2_THRIFT_WORKER_KEEPALIVE_TIME -> TimeUnit.SECONDS, + ConfVars.HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.HIVE_SERVER2_ASYNC_EXEC_KEEPALIVE_TIME -> TimeUnit.SECONDS, + ConfVars.HIVE_SERVER2_LONG_POLLING_TIMEOUT -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_SERVER2_SESSION_CHECK_INTERVAL -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_SERVER2_IDLE_SESSION_TIMEOUT -> TimeUnit.MILLISECONDS, + ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT -> TimeUnit.MILLISECONDS, + ConfVars.SERVER_READ_SOCKET_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.HIVE_LOCALIZE_RESOURCE_WAIT_INTERVAL -> TimeUnit.MILLISECONDS, + ConfVars.SPARK_CLIENT_FUTURE_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.SPARK_JOB_MONITOR_TIMEOUT -> TimeUnit.SECONDS, + ConfVars.SPARK_RPC_CLIENT_CONNECT_TIMEOUT -> TimeUnit.MILLISECONDS, + ConfVars.SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT -> TimeUnit.MILLISECONDS + ).map { case (confVar, unit) => + confVar.varname -> HiveConf.getTimeVar(hadoopConf, confVar, unit).toString + }.toMap + } + + /** + * Check current Thread's SessionState type + * @return true when SessionState.get returns an instance of CliSessionState, + * false when it gets non-CliSessionState instance or null + */ + def isCliSessionState(): Boolean = { + val state = SessionState.get + var temp: Class[_] = if (state != null) state.getClass else null + var found = false + while (temp != null && !found) { + found = temp.getName == "org.apache.hadoop.hive.cli.CliSessionState" + temp = temp.getSuperclass + } + found + } + + /** + * Create a [[HiveClient]] used for execution. + * + * Currently this must always be Hive 13 as this is the version of Hive that is packaged + * with Spark SQL. This copy of the client is used for execution related tasks like + * registering temporary functions or ensuring that the ThreadLocal SessionState is + * correctly populated. This copy of Hive is *not* used for storing persistent metadata, + * and only point to a dummy metastore in a temporary directory. + */ + protected[hive] def newClientForExecution( + conf: SparkConf, + hadoopConf: Configuration): HiveClientImpl = { + logInfo(s"Initializing execution hive, version $hiveExecutionVersion") + val loader = new IsolatedClientLoader( + version = IsolatedClientLoader.hiveVersion(hiveExecutionVersion), + sparkConf = conf, + execJars = Seq.empty, + hadoopConf = hadoopConf, + config = newTemporaryConfiguration(useInMemoryDerby = true), + isolationOn = false, + baseClassLoader = Utils.getContextOrSparkClassLoader) + loader.createClient().asInstanceOf[HiveClientImpl] + } + + /** + * Create a [[HiveClient]] used to retrieve metadata from the Hive MetaStore. + * + * The version of the Hive client that is used here must match the metastore that is configured + * in the hive-site.xml file. + */ + protected[hive] def newClientForMetadata( + conf: SparkConf, + hadoopConf: Configuration): HiveClient = { + val configurations = formatTimeVarsForHiveClient(hadoopConf) + newClientForMetadata(conf, hadoopConf, configurations) + } + + protected[hive] def newClientForMetadata( + conf: SparkConf, + hadoopConf: Configuration, + configurations: Map[String, String]): HiveClient = { + val sqlConf = new SQLConf + sqlConf.setConf(SQLContext.getSQLProperties(conf)) + val hiveMetastoreVersion = HiveUtils.hiveMetastoreVersion(sqlConf) + val hiveMetastoreJars = HiveUtils.hiveMetastoreJars(sqlConf) + val hiveMetastoreSharedPrefixes = HiveUtils.hiveMetastoreSharedPrefixes(sqlConf) + val hiveMetastoreBarrierPrefixes = HiveUtils.hiveMetastoreBarrierPrefixes(sqlConf) + val metaVersion = IsolatedClientLoader.hiveVersion(hiveMetastoreVersion) + + val isolatedLoader = if (hiveMetastoreJars == "builtin") { + if (hiveExecutionVersion != hiveMetastoreVersion) { + throw new IllegalArgumentException( + "Builtin jars can only be used when hive execution version == hive metastore version. " + + s"Execution: $hiveExecutionVersion != Metastore: $hiveMetastoreVersion. " + + "Specify a vaild path to the correct hive jars using $HIVE_METASTORE_JARS " + + s"or change ${HIVE_METASTORE_VERSION.key} to $hiveExecutionVersion.") + } + + // We recursively find all jars in the class loader chain, + // starting from the given classLoader. + def allJars(classLoader: ClassLoader): Array[URL] = classLoader match { + case null => Array.empty[URL] + case urlClassLoader: URLClassLoader => + urlClassLoader.getURLs ++ allJars(urlClassLoader.getParent) + case other => allJars(other.getParent) + } + + val classLoader = Utils.getContextOrSparkClassLoader + val jars = allJars(classLoader) + if (jars.length == 0) { + throw new IllegalArgumentException( + "Unable to locate hive jars to connect to metastore. " + + "Please set spark.sql.hive.metastore.jars.") + } + + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using Spark classes.") + new IsolatedClientLoader( + version = metaVersion, + sparkConf = conf, + hadoopConf = hadoopConf, + execJars = jars.toSeq, + config = configurations, + isolationOn = !isCliSessionState(), + barrierPrefixes = hiveMetastoreBarrierPrefixes, + sharedPrefixes = hiveMetastoreSharedPrefixes) + } else if (hiveMetastoreJars == "maven") { + // TODO: Support for loading the jars from an already downloaded location. + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion using maven.") + IsolatedClientLoader.forVersion( + hiveMetastoreVersion = hiveMetastoreVersion, + hadoopVersion = VersionInfo.getVersion, + sparkConf = conf, + hadoopConf = hadoopConf, + config = configurations, + barrierPrefixes = hiveMetastoreBarrierPrefixes, + sharedPrefixes = hiveMetastoreSharedPrefixes) + } else { + // Convert to files and expand any directories. + val jars = + hiveMetastoreJars + .split(File.pathSeparator) + .flatMap { + case path if new File(path).getName == "*" => + val files = new File(path).getParentFile.listFiles() + if (files == null) { + logWarning(s"Hive jar path '$path' does not exist.") + Nil + } else { + files.filter(_.getName.toLowerCase(Locale.ROOT).endsWith(".jar")) + } + case path => + new File(path) :: Nil + } + .map(_.toURI.toURL) + + logInfo( + s"Initializing HiveMetastoreConnection version $hiveMetastoreVersion " + + s"using ${jars.mkString(":")}") + new IsolatedClientLoader( + version = metaVersion, + sparkConf = conf, + hadoopConf = hadoopConf, + execJars = jars.toSeq, + config = configurations, + isolationOn = true, + barrierPrefixes = hiveMetastoreBarrierPrefixes, + sharedPrefixes = hiveMetastoreSharedPrefixes) + } + isolatedLoader.createClient() + } + + /** Constructs a configuration for hive, where the metastore is located in a temp directory. */ + def newTemporaryConfiguration(useInMemoryDerby: Boolean): Map[String, String] = { + val withInMemoryMode = if (useInMemoryDerby) "memory:" else "" + + val tempDir = Utils.createTempDir() + val localMetastore = new File(tempDir, "metastore") + val propMap: HashMap[String, String] = HashMap() + // We have to mask all properties in hive-site.xml that relates to metastore data source + // as we used a local metastore here. + HiveConf.ConfVars.values().foreach { confvar => + if (confvar.varname.contains("datanucleus") || confvar.varname.contains("jdo") + || confvar.varname.contains("hive.metastore.rawstore.impl")) { + propMap.put(confvar.varname, confvar.getDefaultExpr()) + } + } + propMap.put(WAREHOUSE_PATH.key, localMetastore.toURI.toString) + propMap.put(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname, + s"jdbc:derby:${withInMemoryMode};databaseName=${localMetastore.getAbsolutePath};create=true") + propMap.put("datanucleus.rdbms.datastoreAdapterClassName", + "org.datanucleus.store.rdbms.adapter.DerbyAdapter") + + // SPARK-11783: When "hive.metastore.uris" is set, the metastore connection mode will be + // remote (https://cwiki.apache.org/confluence/display/Hive/AdminManual+MetastoreAdmin + // mentions that "If hive.metastore.uris is empty local mode is assumed, remote otherwise"). + // Remote means that the metastore server is running in its own process. + // When the mode is remote, configurations like "javax.jdo.option.ConnectionURL" will not be + // used (because they are used by remote metastore server that talks to the database). + // Because execution Hive should always connects to an embedded derby metastore. + // We have to remove the value of hive.metastore.uris. So, the execution Hive client connects + // to the actual embedded derby metastore instead of the remote metastore. + // You can search HiveConf.ConfVars.METASTOREURIS in the code of HiveConf (in Hive's repo). + // Then, you will find that the local metastore mode is only set to true when + // hive.metastore.uris is not set. + propMap.put(ConfVars.METASTOREURIS.varname, "") + + // The execution client will generate garbage events, therefore the listeners that are generated + // for the execution clients are useless. In order to not output garbage, we don't generate + // these listeners. + propMap.put(ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname, "") + propMap.put(ConfVars.METASTORE_EVENT_LISTENERS.varname, "") + propMap.put(ConfVars.METASTORE_END_FUNCTION_LISTENERS.varname, "") + + // SPARK-21451: Spark will gather all `spark.hadoop.*` properties from a `SparkConf` to a + // Hadoop Configuration internally, as long as it happens after SparkContext initialized. + // Some instances such as `CliSessionState` used in `SparkSQLCliDriver` may also rely on these + // Configuration. But it happens before SparkContext initialized, we need to take them from + // system properties in the form of regular hadoop configurations. + SparkHadoopUtil.get.appendSparkHadoopConfigs(sys.props.toMap, propMap) + + propMap.toMap + } + + protected val primitiveTypes = + Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, + ShortType, DateType, TimestampType, BinaryType) + + protected[sql] def toHiveString(a: (Any, DataType)): String = a match { + case (struct: Row, StructType(fields)) => + struct.toSeq.zip(fields).map { + case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" + }.mkString("{", ",", "}") + case (seq: Seq[_], ArrayType(typ, _)) => + seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") + case (map: Map[_, _], MapType(kType, vType, _)) => + map.map { + case (key, value) => + toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) + }.toSeq.sorted.mkString("{", ",", "}") + case (null, _) => "NULL" + case (d: Int, DateType) => new DateWritable(d).toString + case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString + case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) + case (decimal: java.math.BigDecimal, DecimalType()) => + // Hive strips trailing zeros so use its toString + HiveDecimal.create(decimal).toString + case (other, tpe) if primitiveTypes contains tpe => other.toString + } + + /** Hive outputs fields of structs slightly differently than top level attributes. */ + protected def toHiveStructString(a: (Any, DataType)): String = a match { + case (struct: Row, StructType(fields)) => + struct.toSeq.zip(fields).map { + case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" + }.mkString("{", ",", "}") + case (seq: Seq[_], ArrayType(typ, _)) => + seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") + case (map: Map[_, _], MapType(kType, vType, _)) => + map.map { + case (key, value) => + toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) + }.toSeq.sorted.mkString("{", ",", "}") + case (null, _) => "null" + case (s: String, StringType) => "\"" + s + "\"" + case (decimal, DecimalType()) => decimal.toString + case (other, tpe) if primitiveTypes contains tpe => other.toString + } + + /** + * Infers the schema for Hive serde tables and returns the CatalogTable with the inferred schema. + * When the tables are data source tables or the schema already exists, returns the original + * CatalogTable. + */ + def inferSchema(table: CatalogTable): CatalogTable = { + if (DDLUtils.isDatasourceTable(table) || table.dataSchema.nonEmpty) { + table + } else { + val hiveTable = HiveClientImpl.toHiveTable(table) + // Note: Hive separates partition columns and the schema, but for us the + // partition columns are part of the schema + val partCols = hiveTable.getPartCols.asScala.map(HiveClientImpl.fromHiveColumn) + val dataCols = hiveTable.getCols.asScala.map(HiveClientImpl.fromHiveColumn) + table.copy(schema = StructType(dataCols ++ partCols)) + } + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 69f481c49a65..cc8907a0bbc9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -17,26 +17,32 @@ package org.apache.spark.sql.hive -import java.util +import java.util.Properties +import scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, PathFilter} -import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable, Hive, HiveUtils, HiveStorageHandler} +import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde2.Deserializer -import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, StructObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} -import org.apache.spark.Logging import org.apache.spark.broadcast.Broadcast +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.{SerializableConfiguration, Utils} @@ -57,30 +63,37 @@ private[hive] sealed trait TableReader { private[hive] class HadoopTableReader( @transient private val attributes: Seq[Attribute], - @transient private val relation: MetastoreRelation, - @transient private val sc: HiveContext, - hiveExtraConf: HiveConf) - extends TableReader with Logging { - - // Hadoop honors "mapred.map.tasks" as hint, but will ignore when mapred.job.tracker is "local". - // https://hadoop.apache.org/docs/r1.0.4/mapred-default.html + @transient private val partitionKeys: Seq[Attribute], + @transient private val tableDesc: TableDesc, + @transient private val sparkSession: SparkSession, + hadoopConf: Configuration) + extends TableReader with CastSupport with Logging { + + // Hadoop honors "mapreduce.job.maps" as hint, + // but will ignore when mapreduce.jobtracker.address is "local". + // https://hadoop.apache.org/docs/r2.6.5/hadoop-mapreduce-client/hadoop-mapreduce-client-core/ + // mapred-default.xml // // In order keep consistency with Hive, we will let it be 0 in local mode also. - private val _minSplitsPerRDD = if (sc.sparkContext.isLocal) { + private val _minSplitsPerRDD = if (sparkSession.sparkContext.isLocal) { 0 // will splitted based on block by default. } else { - math.max(sc.hiveconf.getInt("mapred.map.tasks", 1), sc.sparkContext.defaultMinPartitions) + math.max(hadoopConf.getInt("mapreduce.job.maps", 1), + sparkSession.sparkContext.defaultMinPartitions) } - // TODO: set aws s3 credentials. + SparkHadoopUtil.get.appendS3AndSparkHadoopConfigurations( + sparkSession.sparkContext.conf, hadoopConf) + + private val _broadcastedHadoopConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) - private val _broadcastedHiveConf = - sc.sparkContext.broadcast(new SerializableConfiguration(hiveExtraConf)) + override def conf: SQLConf = sparkSession.sessionState.conf override def makeRDDForTable(hiveTable: HiveTable): RDD[InternalRow] = makeRDDForTable( hiveTable, - Utils.classForName(relation.tableDesc.getSerdeClassName).asInstanceOf[Class[Deserializer]], + Utils.classForName(tableDesc.getSerdeClassName).asInstanceOf[Class[Deserializer]], filterOpt = None) /** @@ -102,8 +115,8 @@ class HadoopTableReader( // Create local references to member variables, so that the entire `this` object won't be // serialized in the closure below. - val tableDesc = relation.tableDesc - val broadcastedHiveConf = _broadcastedHiveConf + val localTableDesc = tableDesc + val broadcastedHadoopConf = _broadcastedHadoopConf val tablePath = hiveTable.getPath val inputPathStr = applyFilterIfNeeded(tablePath, filterOpt) @@ -111,15 +124,15 @@ class HadoopTableReader( // logDebug("Table input: %s".format(tablePath)) val ifc = hiveTable.getInputFormatClass .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] - val hadoopRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) + val hadoopRDD = createHadoopRdd(localTableDesc, inputPathStr, ifc) val attrsWithIndex = attributes.zipWithIndex - val mutableRow = new SpecificMutableRow(attributes.map(_.dataType)) + val mutableRow = new SpecificInternalRow(attributes.map(_.dataType)) val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => - val hconf = broadcastedHiveConf.value.value + val hconf = broadcastedHadoopConf.value.value val deserializer = deserializerClass.newInstance() - deserializer.initialize(hconf, tableDesc.getProperties) + deserializer.initialize(hconf, localTableDesc.getProperties) HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow, deserializer) } @@ -143,24 +156,23 @@ class HadoopTableReader( * subdirectory of each partition being read. If None, then all files are accepted. */ def makeRDDForPartitionedTable( - partitionToDeserializer: Map[HivePartition, - Class[_ <: Deserializer]], + partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], filterOpt: Option[PathFilter]): RDD[InternalRow] = { // SPARK-5068:get FileStatus and do the filtering locally when the path is not exists def verifyPartitionPath( partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]]): Map[HivePartition, Class[_ <: Deserializer]] = { - if (!sc.conf.verifyPartitionPath) { + if (!sparkSession.sessionState.conf.verifyPartitionPath) { partitionToDeserializer } else { - var existPathSet = collection.mutable.Set[String]() - var pathPatternSet = collection.mutable.Set[String]() + val existPathSet = collection.mutable.Set[String]() + val pathPatternSet = collection.mutable.Set[String]() partitionToDeserializer.filter { case (partition, partDeserializer) => def updateExistPathSetByPathPattern(pathPatternStr: String) { val pathPattern = new Path(pathPatternStr) - val fs = pathPattern.getFileSystem(sc.hiveconf) + val fs = pathPattern.getFileSystem(hadoopConf) val matches = fs.globStatus(pathPattern) matches.foreach(fileStatus => existPathSet += fileStatus.getPath.toString) } @@ -173,8 +185,8 @@ class HadoopTableReader( } val partPath = partition.getDataLocation - val partNum = Utilities.getPartitionDesc(partition).getPartSpec.size(); - var pathPatternStr = getPathPatternByPath(partNum, partPath) + val partNum = Utilities.getPartitionDesc(partition).getPartSpec.size() + val pathPatternStr = getPathPatternByPath(partNum, partPath) if (!pathPatternSet.contains(pathPatternStr)) { pathPatternSet += pathPatternStr updateExistPathSetByPathPattern(pathPatternStr) @@ -205,36 +217,47 @@ class HadoopTableReader( partCols.map(col => new String(partSpec.get(col))).toArray } - // Create local references so that the outer object isn't serialized. - val tableDesc = relation.tableDesc - val broadcastedHiveConf = _broadcastedHiveConf + val broadcastedHiveConf = _broadcastedHadoopConf val localDeserializer = partDeserializer - val mutableRow = new SpecificMutableRow(attributes.map(_.dataType)) + val mutableRow = new SpecificInternalRow(attributes.map(_.dataType)) // Splits all attributes into two groups, partition key attributes and those that are not. // Attached indices indicate the position of each attribute in the output schema. val (partitionKeyAttrs, nonPartitionKeyAttrs) = attributes.zipWithIndex.partition { case (attr, _) => - relation.partitionKeys.contains(attr) + partitionKeys.contains(attr) } - def fillPartitionKeys(rawPartValues: Array[String], row: MutableRow): Unit = { + def fillPartitionKeys(rawPartValues: Array[String], row: InternalRow): Unit = { partitionKeyAttrs.foreach { case (attr, ordinal) => - val partOrdinal = relation.partitionKeys.indexOf(attr) - row(ordinal) = Cast(Literal(rawPartValues(partOrdinal)), attr.dataType).eval(null) + val partOrdinal = partitionKeys.indexOf(attr) + row(ordinal) = cast(Literal(rawPartValues(partOrdinal)), attr.dataType).eval(null) } } // Fill all partition keys to the given MutableRow object fillPartitionKeys(partValues, mutableRow) - createHadoopRdd(tableDesc, inputPathStr, ifc).mapPartitions { iter => + val tableProperties = tableDesc.getProperties + + // Create local references so that the outer object isn't serialized. + val localTableDesc = tableDesc + createHadoopRdd(localTableDesc, inputPathStr, ifc).mapPartitions { iter => val hconf = broadcastedHiveConf.value.value val deserializer = localDeserializer.newInstance() - deserializer.initialize(hconf, partProps) + // SPARK-13709: For SerDes like AvroSerDe, some essential information (e.g. Avro schema + // information) may be defined in table properties. Here we should merge table properties + // and partition properties before initializing the deserializer. Note that partition + // properties take a higher priority here. For example, a partition may have a different + // SerDe as the one defined in table properties. + val props = new Properties(tableProperties) + partProps.asScala.foreach { + case (key, value) => props.setProperty(key, value) + } + deserializer.initialize(hconf, props) // get the table deserializer - val tableSerDe = tableDesc.getDeserializerClass.newInstance() - tableSerDe.initialize(hconf, tableDesc.getProperties) + val tableSerDe = localTableDesc.getDeserializerClass.newInstance() + tableSerDe.initialize(hconf, localTableDesc.getProperties) // fill the non partition key attributes HadoopTableReader.fillObject(iter, deserializer, nonPartitionKeyAttrs, @@ -244,7 +267,7 @@ class HadoopTableReader( // Even if we don't use any partitions, we still need an empty RDD if (hivePartitionRDDs.size == 0) { - new EmptyRDD[InternalRow](sc.sparkContext) + new EmptyRDD[InternalRow](sparkSession.sparkContext) } else { new UnionRDD(hivePartitionRDDs(0).context, hivePartitionRDDs) } @@ -257,7 +280,7 @@ class HadoopTableReader( private def applyFilterIfNeeded(path: Path, filterOpt: Option[PathFilter]): String = { filterOpt match { case Some(filter) => - val fs = path.getFileSystem(sc.hiveconf) + val fs = path.getFileSystem(hadoopConf) val filteredFiles = fs.listStatus(path, filter).map(_.getPath.toString) filteredFiles.mkString(",") case None => path.toString @@ -276,8 +299,8 @@ class HadoopTableReader( val initializeJobConfFunc = HadoopTableReader.initializeLocalJobConfFunc(path, tableDesc) _ val rdd = new HadoopRDD( - sc.sparkContext, - _broadcastedHiveConf.asInstanceOf[Broadcast[SerializableConfiguration]], + sparkSession.sparkContext, + _broadcastedHadoopConf.asInstanceOf[Broadcast[SerializableConfiguration]], Some(initializeJobConfFunc), inputFormatClass, classOf[Writable], @@ -295,11 +318,12 @@ private[hive] object HiveTableUtil { // that calls Hive.get() which tries to access metastore, but it's not valid in runtime // it would be fixed in next version of hive but till then, we should use this instead def configureJobPropertiesForStorageHandler( - tableDesc: TableDesc, jobConf: JobConf, input: Boolean) { + tableDesc: TableDesc, conf: Configuration, input: Boolean) { val property = tableDesc.getProperties.getProperty(META_TABLE_STORAGE) - val storageHandler = HiveUtils.getStorageHandler(jobConf, property) + val storageHandler = + org.apache.hadoop.hive.ql.metadata.HiveUtils.getStorageHandler(conf, property) if (storageHandler != null) { - val jobProperties = new util.LinkedHashMap[String, String] + val jobProperties = new java.util.LinkedHashMap[String, String] if (input) { storageHandler.configureInputJobProperties(tableDesc, jobProperties) } else { @@ -342,7 +366,7 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging { iterator: Iterator[Writable], rawDeser: Deserializer, nonPartitionKeyAttrs: Seq[(Attribute, Int)], - mutableRow: MutableRow, + mutableRow: InternalRow, tableDeser: Deserializer): Iterator[InternalRow] = { val soi = if (rawDeser.getObjectInspector.equals(tableDeser.getObjectInspector)) { @@ -363,39 +387,43 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging { * Builds specific unwrappers ahead of time according to object inspector * types to avoid pattern matching and branching costs per row. */ - val unwrappers: Seq[(Any, MutableRow, Int) => Unit] = fieldRefs.map { + val unwrappers: Seq[(Any, InternalRow, Int) => Unit] = fieldRefs.map { _.getFieldObjectInspector match { case oi: BooleanObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setBoolean(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setBoolean(ordinal, oi.get(value)) case oi: ByteObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setByte(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setByte(ordinal, oi.get(value)) case oi: ShortObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setShort(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setShort(ordinal, oi.get(value)) case oi: IntObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setInt(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setInt(ordinal, oi.get(value)) case oi: LongObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setLong(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setLong(ordinal, oi.get(value)) case oi: FloatObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setFloat(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setFloat(ordinal, oi.get(value)) case oi: DoubleObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value)) + (value: Any, row: InternalRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value)) case oi: HiveVarcharObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => + (value: Any, row: InternalRow, ordinal: Int) => + row.update(ordinal, UTF8String.fromString(oi.getPrimitiveJavaObject(value).getValue)) + case oi: HiveCharObjectInspector => + (value: Any, row: InternalRow, ordinal: Int) => row.update(ordinal, UTF8String.fromString(oi.getPrimitiveJavaObject(value).getValue)) case oi: HiveDecimalObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => + (value: Any, row: InternalRow, ordinal: Int) => row.update(ordinal, HiveShim.toCatalystDecimal(oi, value)) case oi: TimestampObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => + (value: Any, row: InternalRow, ordinal: Int) => row.setLong(ordinal, DateTimeUtils.fromJavaTimestamp(oi.getPrimitiveJavaObject(value))) case oi: DateObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => + (value: Any, row: InternalRow, ordinal: Int) => row.setInt(ordinal, DateTimeUtils.fromJavaDate(oi.getPrimitiveJavaObject(value))) case oi: BinaryObjectInspector => - (value: Any, row: MutableRow, ordinal: Int) => + (value: Any, row: InternalRow, ordinal: Int) => row.update(ordinal, oi.getPrimitiveJavaObject(value)) case oi => - (value: Any, row: MutableRow, ordinal: Int) => row(ordinal) = unwrap(value, oi) + val unwrapper = unwrapperFor(oi) + (value: Any, row: InternalRow, ordinal: Int) => row(ordinal) = unwrapper(value) } } @@ -405,7 +433,8 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging { iterator.map { value => val raw = converter.convert(rawDeser.deserialize(value)) var i = 0 - while (i < fieldRefs.length) { + val length = fieldRefs.length + while (i < length) { val fieldValue = soi.getStructFieldData(raw, fieldRefs(i)) if (fieldValue == null) { mutableRow.setNullAt(fieldOrdinals(i)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala deleted file mode 100644 index 9d9a55edd731..000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientInterface.scala +++ /dev/null @@ -1,192 +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.client - -import java.io.PrintStream -import java.util.{Map => JMap} -import javax.annotation.Nullable - -import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchTableException} -import org.apache.spark.sql.catalyst.expressions.Expression - -private[hive] case class HiveDatabase(name: String, location: String) - -private[hive] abstract class TableType { val name: String } -private[hive] case object ExternalTable extends TableType { override val name = "EXTERNAL_TABLE" } -private[hive] case object IndexTable extends TableType { override val name = "INDEX_TABLE" } -private[hive] case object ManagedTable extends TableType { override val name = "MANAGED_TABLE" } -private[hive] case object VirtualView extends TableType { override val name = "VIRTUAL_VIEW" } - -// TODO: Use this for Tables and Partitions -private[hive] case class HiveStorageDescriptor( - location: String, - inputFormat: String, - outputFormat: String, - serde: String, - serdeProperties: Map[String, String]) - -private[hive] case class HivePartition( - values: Seq[String], - storage: HiveStorageDescriptor) - -private[hive] case class HiveColumn(name: String, @Nullable hiveType: String, comment: String) -private[hive] case class HiveTable( - specifiedDatabase: Option[String], - name: String, - schema: Seq[HiveColumn], - partitionColumns: Seq[HiveColumn], - properties: Map[String, String], - serdeProperties: Map[String, String], - tableType: TableType, - location: Option[String] = None, - inputFormat: Option[String] = None, - outputFormat: Option[String] = None, - serde: Option[String] = None, - viewText: Option[String] = None) { - - @transient - private[client] var client: ClientInterface = _ - - private[client] def withClient(ci: ClientInterface): this.type = { - client = ci - this - } - - def database: String = specifiedDatabase.getOrElse(sys.error("database not resolved")) - - def isPartitioned: Boolean = partitionColumns.nonEmpty - - def getAllPartitions: Seq[HivePartition] = client.getAllPartitions(this) - - def getPartitions(predicates: Seq[Expression]): Seq[HivePartition] = - client.getPartitionsByFilter(this, predicates) - - // Hive does not support backticks when passing names to the client. - def qualifiedName: String = s"$database.$name" -} - -/** - * An externally visible interface to the Hive client. This interface is shared across both the - * internal and external classloaders for a given version of Hive and thus must expose only - * shared classes. - */ -private[hive] trait ClientInterface { - - /** Returns the Hive Version of this client. */ - def version: HiveVersion - - /** Returns the configuration for the given key in the current session. */ - def getConf(key: String, defaultValue: String): String - - /** - * Runs a HiveQL command using Hive, returning the results as a list of strings. Each row will - * result in one string. - */ - def runSqlHive(sql: String): Seq[String] - - def setOut(stream: PrintStream): Unit - def setInfo(stream: PrintStream): Unit - def setError(stream: PrintStream): Unit - - /** Returns the names of all tables in the given database. */ - def listTables(dbName: String): Seq[String] - - /** Returns the name of the active database. */ - def currentDatabase: String - - /** Returns the metadata for specified database, throwing an exception if it doesn't exist */ - def getDatabase(name: String): HiveDatabase = { - getDatabaseOption(name).getOrElse(throw new NoSuchDatabaseException) - } - - /** Returns the metadata for a given database, or None if it doesn't exist. */ - def getDatabaseOption(name: String): Option[HiveDatabase] - - /** Returns the specified table, or throws [[NoSuchTableException]]. */ - def getTable(dbName: String, tableName: String): HiveTable = { - getTableOption(dbName, tableName).getOrElse(throw new NoSuchTableException) - } - - /** Returns the metadata for the specified table or None if it doens't exist. */ - def getTableOption(dbName: String, tableName: String): Option[HiveTable] - - /** Creates a view with the given metadata. */ - def createView(view: HiveTable): Unit - - /** Updates the given view with new metadata. */ - def alertView(view: HiveTable): Unit - - /** Creates a table with the given metadata. */ - def createTable(table: HiveTable): Unit - - /** Updates the given table with new metadata. */ - def alterTable(table: HiveTable): Unit - - /** Creates a new database with the given name. */ - def createDatabase(database: HiveDatabase): Unit - - /** Returns the specified paritition or None if it does not exist. */ - def getPartitionOption( - hTable: HiveTable, - partitionSpec: JMap[String, String]): Option[HivePartition] - - /** Returns all partitions for the given table. */ - def getAllPartitions(hTable: HiveTable): Seq[HivePartition] - - /** Returns partitions filtered by predicates for the given table. */ - def getPartitionsByFilter(hTable: HiveTable, predicates: Seq[Expression]): Seq[HivePartition] - - /** Loads a static partition into an existing table. */ - def loadPartition( - loadPath: String, - tableName: String, - partSpec: java.util.LinkedHashMap[String, String], // Hive relies on LinkedHashMap ordering - replace: Boolean, - holdDDLTime: Boolean, - inheritTableSpecs: Boolean, - isSkewedStoreAsSubdir: Boolean): Unit - - /** Loads data into an existing table. */ - def loadTable( - loadPath: String, // TODO URI - tableName: String, - replace: Boolean, - holdDDLTime: Boolean): Unit - - /** Loads new dynamic partitions into an existing table. */ - def loadDynamicPartitions( - loadPath: String, - tableName: String, - partSpec: java.util.LinkedHashMap[String, String], // Hive relies on LinkedHashMap ordering - replace: Boolean, - numDP: Int, - holdDDLTime: Boolean, - listBucketingEnabled: Boolean): Unit - - /** Add a jar into class loader */ - def addJar(path: String): Unit - - /** Return a ClientInterface as new session, that will share the class loader and Hive client */ - def newSession(): ClientInterface - - /** Run a function within Hive state (SessionState, HiveConf, Hive client and class loader) */ - def withHiveState[A](f: => A): A - - /** Used for testing only. Removes all metadata from this instance of Hive. */ - def reset(): Unit -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala deleted file mode 100644 index 3dce86c48074..000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala +++ /dev/null @@ -1,575 +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.client - -import java.io.{File, PrintStream} -import java.util.{Map => JMap} -import javax.annotation.concurrent.GuardedBy - -import scala.collection.JavaConverters._ -import scala.language.reflectiveCalls - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.api.{Database, FieldSchema} -import org.apache.hadoop.hive.metastore.{TableType => HTableType} -import org.apache.hadoop.hive.ql.metadata.Hive -import org.apache.hadoop.hive.ql.processors._ -import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.ql.{Driver, metadata} -import org.apache.hadoop.hive.shims.{HadoopShims, ShimLoader} -import org.apache.hadoop.util.VersionInfo - -import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.util.{CircularBuffer, Utils} - -/** - * A class that wraps the HiveClient and converts its responses to externally visible classes. - * Note that this class is typically loaded with an internal classloader for each instantiation, - * allowing it to interact directly with a specific isolated version of Hive. Loading this class - * with the isolated classloader however will result in it only being visible as a ClientInterface, - * not a ClientWrapper. - * - * This class needs to interact with multiple versions of Hive, but will always be compiled with - * the 'native', execution version of Hive. Therefore, any places where hive breaks compatibility - * must use reflection after matching on `version`. - * - * @param version the version of hive used when pick function calls that are not compatible. - * @param config a collection of configuration options that will be added to the hive conf before - * opening the hive client. - * @param initClassLoader the classloader used when creating the `state` field of - * this ClientWrapper. - */ -private[hive] class ClientWrapper( - override val version: HiveVersion, - config: Map[String, String], - initClassLoader: ClassLoader, - val clientLoader: IsolatedClientLoader) - extends ClientInterface - with Logging { - - overrideHadoopShims() - - // !! HACK ALERT !! - // - // Internally, Hive `ShimLoader` tries to load different versions of Hadoop shims by checking - // major version number gathered from Hadoop jar files: - // - // - For major version number 1, load `Hadoop20SShims`, where "20S" stands for Hadoop 0.20 with - // security. - // - For major version number 2, load `Hadoop23Shims`, where "23" stands for Hadoop 0.23. - // - // However, APIs in Hadoop 2.0.x and 2.1.x versions were in flux due to historical reasons. It - // turns out that Hadoop 2.0.x versions should also be used together with `Hadoop20SShims`, but - // `Hadoop23Shims` is chosen because the major version number here is 2. - // - // To fix this issue, we try to inspect Hadoop version via `org.apache.hadoop.utils.VersionInfo` - // and load `Hadoop20SShims` for Hadoop 1.x and 2.0.x versions. If Hadoop version information is - // not available, we decide whether to override the shims or not by checking for existence of a - // probe method which doesn't exist in Hadoop 1.x or 2.0.x versions. - private def overrideHadoopShims(): Unit = { - val hadoopVersion = VersionInfo.getVersion - val VersionPattern = """(\d+)\.(\d+).*""".r - - hadoopVersion match { - case null => - logError("Failed to inspect Hadoop version") - - // Using "Path.getPathWithoutSchemeAndAuthority" as the probe method. - val probeMethod = "getPathWithoutSchemeAndAuthority" - if (!classOf[Path].getDeclaredMethods.exists(_.getName == probeMethod)) { - logInfo( - s"Method ${classOf[Path].getCanonicalName}.$probeMethod not found, " + - s"we are probably using Hadoop 1.x or 2.0.x") - loadHadoop20SShims() - } - - case VersionPattern(majorVersion, minorVersion) => - logInfo(s"Inspected Hadoop version: $hadoopVersion") - - // Loads Hadoop20SShims for 1.x and 2.0.x versions - val (major, minor) = (majorVersion.toInt, minorVersion.toInt) - if (major < 2 || (major == 2 && minor == 0)) { - loadHadoop20SShims() - } - } - - // Logs the actual loaded Hadoop shims class - val loadedShimsClassName = ShimLoader.getHadoopShims.getClass.getCanonicalName - logInfo(s"Loaded $loadedShimsClassName for Hadoop version $hadoopVersion") - } - - private def loadHadoop20SShims(): Unit = { - val hadoop20SShimsClassName = "org.apache.hadoop.hive.shims.Hadoop20SShims" - logInfo(s"Loading Hadoop shims $hadoop20SShimsClassName") - - try { - val shimsField = classOf[ShimLoader].getDeclaredField("hadoopShims") - // scalastyle:off classforname - val shimsClass = Class.forName(hadoop20SShimsClassName) - // scalastyle:on classforname - val shims = classOf[HadoopShims].cast(shimsClass.newInstance()) - shimsField.setAccessible(true) - shimsField.set(null, shims) - } catch { case cause: Throwable => - throw new RuntimeException(s"Failed to load $hadoop20SShimsClassName", cause) - } - } - - // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. - private val outputBuffer = new CircularBuffer() - - private val shim = version match { - case hive.v12 => new Shim_v0_12() - case hive.v13 => new Shim_v0_13() - case hive.v14 => new Shim_v0_14() - case hive.v1_0 => new Shim_v1_0() - case hive.v1_1 => new Shim_v1_1() - case hive.v1_2 => new Shim_v1_2() - } - - // Create an internal session state for this ClientWrapper. - val state = { - val original = Thread.currentThread().getContextClassLoader - // Switch to the initClassLoader. - Thread.currentThread().setContextClassLoader(initClassLoader) - val ret = try { - val initialConf = new HiveConf(classOf[SessionState]) - // HiveConf is a Hadoop Configuration, which has a field of classLoader and - // the initial value will be the current thread's context class loader - // (i.e. initClassLoader at here). - // We call initialConf.setClassLoader(initClassLoader) at here to make - // this action explicit. - initialConf.setClassLoader(initClassLoader) - config.foreach { case (k, v) => - if (k.toLowerCase.contains("password")) { - logDebug(s"Hive Config: $k=xxx") - } else { - logDebug(s"Hive Config: $k=$v") - } - initialConf.set(k, v) - } - val state = new SessionState(initialConf) - if (clientLoader.cachedHive != null) { - Hive.set(clientLoader.cachedHive.asInstanceOf[Hive]) - } - SessionState.start(state) - state.out = new PrintStream(outputBuffer, true, "UTF-8") - state.err = new PrintStream(outputBuffer, true, "UTF-8") - state - } finally { - Thread.currentThread().setContextClassLoader(original) - } - ret - } - - /** Returns the configuration for the current session. */ - def conf: HiveConf = SessionState.get().getConf - - override def getConf(key: String, defaultValue: String): String = { - conf.get(key, defaultValue) - } - - // We use hive's conf for compatibility. - private val retryLimit = conf.getIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES) - private val retryDelayMillis = shim.getMetastoreClientConnectRetryDelayMillis(conf) - - /** - * Runs `f` with multiple retries in case the hive metastore is temporarily unreachable. - */ - private def retryLocked[A](f: => A): A = clientLoader.synchronized { - // Hive sometimes retries internally, so set a deadline to avoid compounding delays. - val deadline = System.nanoTime + (retryLimit * retryDelayMillis * 1e6).toLong - var numTries = 0 - var caughtException: Exception = null - do { - numTries += 1 - try { - return f - } catch { - case e: Exception if causedByThrift(e) => - caughtException = e - logWarning( - "HiveClientWrapper got thrift exception, destroying client and retrying " + - s"(${retryLimit - numTries} tries remaining)", e) - clientLoader.cachedHive = null - Thread.sleep(retryDelayMillis) - } - } while (numTries <= retryLimit && System.nanoTime < deadline) - if (System.nanoTime > deadline) { - logWarning("Deadline exceeded") - } - throw caughtException - } - - private def causedByThrift(e: Throwable): Boolean = { - var target = e - while (target != null) { - val msg = target.getMessage() - if (msg != null && msg.matches("(?s).*(TApplication|TProtocol|TTransport)Exception.*")) { - return true - } - target = target.getCause() - } - false - } - - def client: Hive = { - if (clientLoader.cachedHive != null) { - clientLoader.cachedHive.asInstanceOf[Hive] - } else { - val c = Hive.get(conf) - clientLoader.cachedHive = c - c - } - } - - /** - * Runs `f` with ThreadLocal session state and classloaders configured for this version of hive. - */ - def withHiveState[A](f: => A): A = retryLocked { - val original = Thread.currentThread().getContextClassLoader - // Set the thread local metastore client to the client associated with this ClientWrapper. - Hive.set(client) - // The classloader in clientLoader could be changed after addJar, always use the latest - // classloader - state.getConf.setClassLoader(clientLoader.classLoader) - // setCurrentSessionState will use the classLoader associated - // with the HiveConf in `state` to override the context class loader of the current - // thread. - shim.setCurrentSessionState(state) - val ret = try f finally { - Thread.currentThread().setContextClassLoader(original) - } - ret - } - - def setOut(stream: PrintStream): Unit = withHiveState { - state.out = stream - } - - def setInfo(stream: PrintStream): Unit = withHiveState { - state.info = stream - } - - def setError(stream: PrintStream): Unit = withHiveState { - state.err = stream - } - - override def currentDatabase: String = withHiveState { - state.getCurrentDatabase - } - - override def createDatabase(database: HiveDatabase): Unit = withHiveState { - client.createDatabase( - new Database( - database.name, - "", - new File(database.location).toURI.toString, - new java.util.HashMap), - true) - } - - override def getDatabaseOption(name: String): Option[HiveDatabase] = withHiveState { - Option(client.getDatabase(name)).map { d => - HiveDatabase( - name = d.getName, - location = d.getLocationUri) - } - } - - override def getTableOption( - dbName: String, - tableName: String): Option[HiveTable] = withHiveState { - - logDebug(s"Looking up $dbName.$tableName") - - val hiveTable = Option(client.getTable(dbName, tableName, false)) - val converted = hiveTable.map { h => - - HiveTable( - name = h.getTableName, - specifiedDatabase = Option(h.getDbName), - schema = h.getCols.asScala.map(f => HiveColumn(f.getName, f.getType, f.getComment)), - partitionColumns = h.getPartCols.asScala.map(f => - HiveColumn(f.getName, f.getType, f.getComment)), - properties = h.getParameters.asScala.toMap, - serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.asScala.toMap, - tableType = h.getTableType match { - case HTableType.MANAGED_TABLE => ManagedTable - case HTableType.EXTERNAL_TABLE => ExternalTable - case HTableType.VIRTUAL_VIEW => VirtualView - case HTableType.INDEX_TABLE => IndexTable - }, - location = shim.getDataLocation(h), - inputFormat = Option(h.getInputFormatClass).map(_.getName), - outputFormat = Option(h.getOutputFormatClass).map(_.getName), - serde = Option(h.getSerializationLib), - viewText = Option(h.getViewExpandedText)).withClient(this) - } - converted - } - - private def toInputFormat(name: String) = - Utils.classForName(name).asInstanceOf[Class[_ <: org.apache.hadoop.mapred.InputFormat[_, _]]] - - private def toOutputFormat(name: String) = - Utils.classForName(name) - .asInstanceOf[Class[_ <: org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]]] - - private def toQlTable(table: HiveTable): metadata.Table = { - val qlTable = new metadata.Table(table.database, table.name) - - qlTable.setFields(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) - qlTable.setPartCols( - table.partitionColumns.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) - table.properties.foreach { case (k, v) => qlTable.setProperty(k, v) } - table.serdeProperties.foreach { case (k, v) => qlTable.setSerdeParam(k, v) } - - // set owner - qlTable.setOwner(conf.getUser) - // set create time - qlTable.setCreateTime((System.currentTimeMillis() / 1000).asInstanceOf[Int]) - - table.location.foreach { loc => shim.setDataLocation(qlTable, loc) } - table.inputFormat.map(toInputFormat).foreach(qlTable.setInputFormatClass) - table.outputFormat.map(toOutputFormat).foreach(qlTable.setOutputFormatClass) - table.serde.foreach(qlTable.setSerializationLib) - - qlTable - } - - private def toViewTable(view: HiveTable): metadata.Table = { - // TODO: this is duplicated with `toQlTable` except the table type stuff. - val tbl = new metadata.Table(view.database, view.name) - tbl.setTableType(HTableType.VIRTUAL_VIEW) - tbl.setSerializationLib(null) - tbl.clearSerDeInfo() - - // TODO: we will save the same SQL string to original and expanded text, which is different - // from Hive. - tbl.setViewOriginalText(view.viewText.get) - tbl.setViewExpandedText(view.viewText.get) - - tbl.setFields(view.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) - view.properties.foreach { case (k, v) => tbl.setProperty(k, v) } - - // set owner - tbl.setOwner(conf.getUser) - // set create time - tbl.setCreateTime((System.currentTimeMillis() / 1000).asInstanceOf[Int]) - - tbl - } - - override def createView(view: HiveTable): Unit = withHiveState { - client.createTable(toViewTable(view)) - } - - override def alertView(view: HiveTable): Unit = withHiveState { - client.alterTable(view.qualifiedName, toViewTable(view)) - } - - override def createTable(table: HiveTable): Unit = withHiveState { - val qlTable = toQlTable(table) - client.createTable(qlTable) - } - - override def alterTable(table: HiveTable): Unit = withHiveState { - val qlTable = toQlTable(table) - client.alterTable(table.qualifiedName, qlTable) - } - - private def toHivePartition(partition: metadata.Partition): HivePartition = { - val apiPartition = partition.getTPartition - HivePartition( - values = Option(apiPartition.getValues).map(_.asScala).getOrElse(Seq.empty), - storage = HiveStorageDescriptor( - location = apiPartition.getSd.getLocation, - inputFormat = apiPartition.getSd.getInputFormat, - outputFormat = apiPartition.getSd.getOutputFormat, - serde = apiPartition.getSd.getSerdeInfo.getSerializationLib, - serdeProperties = apiPartition.getSd.getSerdeInfo.getParameters.asScala.toMap)) - } - - override def getPartitionOption( - table: HiveTable, - partitionSpec: JMap[String, String]): Option[HivePartition] = withHiveState { - - val qlTable = toQlTable(table) - val qlPartition = client.getPartition(qlTable, partitionSpec, false) - Option(qlPartition).map(toHivePartition) - } - - override def getAllPartitions(hTable: HiveTable): Seq[HivePartition] = withHiveState { - val qlTable = toQlTable(hTable) - shim.getAllPartitions(client, qlTable).map(toHivePartition) - } - - override def getPartitionsByFilter( - hTable: HiveTable, - predicates: Seq[Expression]): Seq[HivePartition] = withHiveState { - val qlTable = toQlTable(hTable) - shim.getPartitionsByFilter(client, qlTable, predicates).map(toHivePartition) - } - - override def listTables(dbName: String): Seq[String] = withHiveState { - client.getAllTables(dbName).asScala - } - - /** - * Runs the specified SQL query using Hive. - */ - override def runSqlHive(sql: String): Seq[String] = { - val maxResults = 100000 - val results = runHive(sql, maxResults) - // It is very confusing when you only get back some of the results... - if (results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") - results - } - - /** - * Execute the command using Hive and return the results as a sequence. Each element - * in the sequence is one row. - */ - protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = withHiveState { - logDebug(s"Running hiveql '$cmd'") - if (cmd.toLowerCase.startsWith("set")) { logDebug(s"Changing config: $cmd") } - try { - val cmd_trimmed: String = cmd.trim() - val tokens: Array[String] = cmd_trimmed.split("\\s+") - // The remainder of the command. - val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() - val proc = shim.getCommandProcessor(tokens(0), conf) - proc match { - case driver: Driver => - val response: CommandProcessorResponse = driver.run(cmd) - // Throw an exception if there is an error in query processing. - if (response.getResponseCode != 0) { - driver.close() - throw new QueryExecutionException(response.getErrorMessage) - } - driver.setMaxRows(maxRows) - - val results = shim.getDriverResults(driver) - driver.close() - results - - case _ => - if (state.out != null) { - // scalastyle:off println - state.out.println(tokens(0) + " " + cmd_1) - // scalastyle:on println - } - Seq(proc.run(cmd_1).getResponseCode.toString) - } - } catch { - case e: Exception => - logError( - s""" - |====================== - |HIVE FAILURE OUTPUT - |====================== - |${outputBuffer.toString} - |====================== - |END HIVE FAILURE OUTPUT - |====================== - """.stripMargin) - throw e - } - } - - def loadPartition( - loadPath: String, - tableName: String, - partSpec: java.util.LinkedHashMap[String, String], - replace: Boolean, - holdDDLTime: Boolean, - inheritTableSpecs: Boolean, - isSkewedStoreAsSubdir: Boolean): Unit = withHiveState { - shim.loadPartition( - client, - new Path(loadPath), // TODO: Use URI - tableName, - partSpec, - replace, - holdDDLTime, - inheritTableSpecs, - isSkewedStoreAsSubdir) - } - - def loadTable( - loadPath: String, // TODO URI - tableName: String, - replace: Boolean, - holdDDLTime: Boolean): Unit = withHiveState { - shim.loadTable( - client, - new Path(loadPath), - tableName, - replace, - holdDDLTime) - } - - def loadDynamicPartitions( - loadPath: String, - tableName: String, - partSpec: java.util.LinkedHashMap[String, String], - replace: Boolean, - numDP: Int, - holdDDLTime: Boolean, - listBucketingEnabled: Boolean): Unit = withHiveState { - shim.loadDynamicPartitions( - client, - new Path(loadPath), - tableName, - partSpec, - replace, - numDP, - holdDDLTime, - listBucketingEnabled) - } - - def addJar(path: String): Unit = { - clientLoader.addJar(path) - runSqlHive(s"ADD JAR $path") - } - - def newSession(): ClientWrapper = { - clientLoader.createClient().asInstanceOf[ClientWrapper] - } - - def reset(): Unit = withHiveState { - client.getAllTables("default").asScala.foreach { t => - logDebug(s"Deleting table $t") - val table = client.getTable("default", t) - client.getIndexes("default", t, 255).asScala.foreach { index => - shim.dropIndex(client, "default", t, index.getIndexName) - } - if (!table.isIndexTable) { - client.dropTable("default", t) - } - } - client.getAllDatabases.asScala.filterNot(_ == "default").foreach { db => - logDebug(s"Dropping Database: $db") - client.dropDatabase(db, true, false, true) - } - } -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala new file mode 100644 index 000000000000..ee3eb2ee8abe --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala @@ -0,0 +1,281 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.client + +import java.io.PrintStream + +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.Expression + + +/** + * An externally visible interface to the Hive client. This interface is shared across both the + * internal and external classloaders for a given version of Hive and thus must expose only + * shared classes. + */ +private[hive] trait HiveClient { + + /** Returns the Hive Version of this client. */ + def version: HiveVersion + + /** Returns the configuration for the given key in the current session. */ + def getConf(key: String, defaultValue: String): String + + /** + * Return the associated Hive SessionState of this [[HiveClientImpl]] + * @return [[Any]] not SessionState to avoid linkage error + */ + def getState: Any + + /** + * Runs a HiveQL command using Hive, returning the results as a list of strings. Each row will + * result in one string. + */ + def runSqlHive(sql: String): Seq[String] + + def setOut(stream: PrintStream): Unit + def setInfo(stream: PrintStream): Unit + def setError(stream: PrintStream): Unit + + /** Returns the names of all tables in the given database. */ + def listTables(dbName: String): Seq[String] + + /** Returns the names of tables in the given database that matches the given pattern. */ + def listTables(dbName: String, pattern: String): Seq[String] + + /** Sets the name of current database. */ + def setCurrentDatabase(databaseName: String): Unit + + /** Returns the metadata for specified database, throwing an exception if it doesn't exist */ + def getDatabase(name: String): CatalogDatabase + + /** Return whether a table/view with the specified name exists. */ + def databaseExists(dbName: String): Boolean + + /** List the names of all the databases that match the specified pattern. */ + def listDatabases(pattern: String): Seq[String] + + /** Return whether a table/view with the specified name exists. */ + def tableExists(dbName: String, tableName: String): Boolean + + /** Returns the specified table, or throws [[NoSuchTableException]]. */ + final def getTable(dbName: String, tableName: String): CatalogTable = { + getTableOption(dbName, tableName).getOrElse(throw new NoSuchTableException(dbName, tableName)) + } + + /** Returns the metadata for the specified table or None if it doesn't exist. */ + def getTableOption(dbName: String, tableName: String): Option[CatalogTable] + + /** Creates a table with the given metadata. */ + def createTable(table: CatalogTable, ignoreIfExists: Boolean): Unit + + /** Drop the specified table. */ + def dropTable(dbName: String, tableName: String, ignoreIfNotExists: Boolean, purge: Boolean): Unit + + /** Alter a table whose name matches the one specified in `table`, assuming it exists. */ + final def alterTable(table: CatalogTable): Unit = { + alterTable(table.database, table.identifier.table, table) + } + + /** + * Updates the given table with new metadata, optionally renaming the table or + * moving across different database. + */ + def alterTable(dbName: String, tableName: String, table: CatalogTable): Unit + + /** Creates a new database with the given name. */ + def createDatabase(database: CatalogDatabase, ignoreIfExists: Boolean): Unit + + /** + * Drop the specified database, if it exists. + * + * @param name database to drop + * @param ignoreIfNotExists if true, do not throw error if the database does not exist + * @param cascade whether to remove all associated objects such as tables and functions + */ + def dropDatabase(name: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit + + /** + * Alter a database whose name matches the one specified in `database`, assuming it exists. + */ + def alterDatabase(database: CatalogDatabase): Unit + + /** + * Create one or many partitions in the given table. + */ + def createPartitions( + db: String, + table: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit + + /** + * Drop one or many partitions in the given table, assuming they exist. + */ + def dropPartitions( + db: String, + table: String, + specs: Seq[TablePartitionSpec], + ignoreIfNotExists: Boolean, + purge: Boolean, + retainData: Boolean): Unit + + /** + * Rename one or many existing table partitions, assuming they exist. + */ + def renamePartitions( + db: String, + table: String, + specs: Seq[TablePartitionSpec], + newSpecs: Seq[TablePartitionSpec]): Unit + + /** + * Alter one or more table partitions whose specs match the ones specified in `newParts`, + * assuming the partitions exist. + */ + def alterPartitions( + db: String, + table: String, + newParts: Seq[CatalogTablePartition]): Unit + + /** Returns the specified partition, or throws [[NoSuchPartitionException]]. */ + final def getPartition( + dbName: String, + tableName: String, + spec: TablePartitionSpec): CatalogTablePartition = { + getPartitionOption(dbName, tableName, spec).getOrElse { + throw new NoSuchPartitionException(dbName, tableName, spec) + } + } + + /** + * Returns the partition names for the given table that match the supplied partition spec. + * If no partition spec is specified, all partitions are returned. + * + * The returned sequence is sorted as strings. + */ + def getPartitionNames( + table: CatalogTable, + partialSpec: Option[TablePartitionSpec] = None): Seq[String] + + /** Returns the specified partition or None if it does not exist. */ + final def getPartitionOption( + db: String, + table: String, + spec: TablePartitionSpec): Option[CatalogTablePartition] = { + getPartitionOption(getTable(db, table), spec) + } + + /** Returns the specified partition or None if it does not exist. */ + def getPartitionOption( + table: CatalogTable, + spec: TablePartitionSpec): Option[CatalogTablePartition] + + /** + * Returns the partitions for the given table that match the supplied partition spec. + * If no partition spec is specified, all partitions are returned. + */ + final def getPartitions( + db: String, + table: String, + partialSpec: Option[TablePartitionSpec]): Seq[CatalogTablePartition] = { + getPartitions(getTable(db, table), partialSpec) + } + + /** + * Returns the partitions for the given table that match the supplied partition spec. + * If no partition spec is specified, all partitions are returned. + */ + def getPartitions( + catalogTable: CatalogTable, + partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] + + /** Returns partitions filtered by predicates for the given table. */ + def getPartitionsByFilter( + catalogTable: CatalogTable, + predicates: Seq[Expression]): Seq[CatalogTablePartition] + + /** Loads a static partition into an existing table. */ + def loadPartition( + loadPath: String, + dbName: String, + tableName: String, + partSpec: java.util.LinkedHashMap[String, String], // Hive relies on LinkedHashMap ordering + replace: Boolean, + inheritTableSpecs: Boolean, + isSrcLocal: Boolean): Unit + + /** Loads data into an existing table. */ + def loadTable( + loadPath: String, // TODO URI + tableName: String, + replace: Boolean, + isSrcLocal: Boolean): Unit + + /** Loads new dynamic partitions into an existing table. */ + def loadDynamicPartitions( + loadPath: String, + dbName: String, + tableName: String, + partSpec: java.util.LinkedHashMap[String, String], // Hive relies on LinkedHashMap ordering + replace: Boolean, + numDP: Int): Unit + + /** Create a function in an existing database. */ + def createFunction(db: String, func: CatalogFunction): Unit + + /** Drop an existing function in the database. */ + def dropFunction(db: String, name: String): Unit + + /** Rename an existing function in the database. */ + def renameFunction(db: String, oldName: String, newName: String): Unit + + /** Alter a function whose name matches the one specified in `func`, assuming it exists. */ + def alterFunction(db: String, func: CatalogFunction): Unit + + /** Return an existing function in the database, assuming it exists. */ + final def getFunction(db: String, name: String): CatalogFunction = { + getFunctionOption(db, name).getOrElse(throw new NoSuchPermanentFunctionException(db, name)) + } + + /** Return an existing function in the database, or None if it doesn't exist. */ + def getFunctionOption(db: String, name: String): Option[CatalogFunction] + + /** Return whether a function exists in the specified database. */ + final def functionExists(db: String, name: String): Boolean = { + getFunctionOption(db, name).isDefined + } + + /** Return the names of all functions that match the given pattern in the database. */ + def listFunctions(db: String, pattern: String): Seq[String] + + /** Add a jar into class loader */ + def addJar(path: String): Unit + + /** Return a [[HiveClient]] as new session, that will share the class loader and Hive client */ + def newSession(): HiveClient + + /** Run a function within Hive state (SessionState, HiveConf, Hive client and class loader) */ + def withHiveState[A](f: => A): A + + /** Used for testing only. Removes all metadata from this instance of Hive. */ + def reset(): Unit + +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala new file mode 100644 index 000000000000..c4e48c9360db --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -0,0 +1,1016 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.client + +import java.io.{File, PrintStream} +import java.util.Locale + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.common.StatsSetupConst +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.metastore.{TableType => HiveTableType} +import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema, Order} +import org.apache.hadoop.hive.metastore.api.{SerDeInfo, StorageDescriptor} +import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition => HivePartition, Table => HiveTable} +import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_ASC +import org.apache.hadoop.hive.ql.processors._ +import org.apache.hadoop.hive.ql.session.SessionState + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchPartitionException} +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} +import org.apache.spark.sql.execution.QueryExecutionException +import org.apache.spark.sql.execution.command.DDLUtils +import org.apache.spark.sql.hive.HiveExternalCatalog +import org.apache.spark.sql.hive.client.HiveClientImpl._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.{CircularBuffer, Utils} + +/** + * A class that wraps the HiveClient and converts its responses to externally visible classes. + * Note that this class is typically loaded with an internal classloader for each instantiation, + * allowing it to interact directly with a specific isolated version of Hive. Loading this class + * with the isolated classloader however will result in it only being visible as a [[HiveClient]], + * not a [[HiveClientImpl]]. + * + * This class needs to interact with multiple versions of Hive, but will always be compiled with + * the 'native', execution version of Hive. Therefore, any places where hive breaks compatibility + * must use reflection after matching on `version`. + * + * Every HiveClientImpl creates an internal HiveConf object. This object is using the given + * `hadoopConf` as the base. All options set in the `sparkConf` will be applied to the HiveConf + * object and overrides any exiting options. Then, options in extraConfig will be applied + * to the HiveConf object and overrides any existing options. + * + * @param version the version of hive used when pick function calls that are not compatible. + * @param sparkConf all configuration options set in SparkConf. + * @param hadoopConf the base Configuration object used by the HiveConf created inside + * this HiveClientImpl. + * @param extraConfig a collection of configuration options that will be added to the + * hive conf before opening the hive client. + * @param initClassLoader the classloader used when creating the `state` field of + * this [[HiveClientImpl]]. + */ +private[hive] class HiveClientImpl( + override val version: HiveVersion, + sparkConf: SparkConf, + hadoopConf: Configuration, + extraConfig: Map[String, String], + initClassLoader: ClassLoader, + val clientLoader: IsolatedClientLoader) + extends HiveClient + with Logging { + + // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. + private val outputBuffer = new CircularBuffer() + + private val shim = version match { + case hive.v12 => new Shim_v0_12() + case hive.v13 => new Shim_v0_13() + case hive.v14 => new Shim_v0_14() + case hive.v1_0 => new Shim_v1_0() + case hive.v1_1 => new Shim_v1_1() + case hive.v1_2 => new Shim_v1_2() + case hive.v2_0 => new Shim_v2_0() + case hive.v2_1 => new Shim_v2_1() + } + + // Create an internal session state for this HiveClientImpl. + val state: SessionState = { + val original = Thread.currentThread().getContextClassLoader + if (clientLoader.isolationOn) { + // Switch to the initClassLoader. + Thread.currentThread().setContextClassLoader(initClassLoader) + // Set up kerberos credentials for UserGroupInformation.loginUser within current class loader + if (sparkConf.contains("spark.yarn.principal") && sparkConf.contains("spark.yarn.keytab")) { + val principal = sparkConf.get("spark.yarn.principal") + val keytab = sparkConf.get("spark.yarn.keytab") + SparkHadoopUtil.get.loginUserFromKeytab(principal, keytab) + } + try { + newState() + } finally { + Thread.currentThread().setContextClassLoader(original) + } + } else { + // Isolation off means we detect a CliSessionState instance in current thread. + // 1: Inside the spark project, we have already started a CliSessionState in + // `SparkSQLCLIDriver`, which contains configurations from command lines. Later, we call + // `SparkSQLEnv.init()` there, which would new a hive client again. so we should keep those + // configurations and reuse the existing instance of `CliSessionState`. In this case, + // SessionState.get will always return a CliSessionState. + // 2: In another case, a user app may start a CliSessionState outside spark project with built + // in hive jars, which will turn off isolation, if SessionSate.detachSession is + // called to remove the current state after that, hive client created later will initialize + // its own state by newState() + val ret = SessionState.get + if (ret != null) { + // hive.metastore.warehouse.dir is determined in SharedState after the CliSessionState + // instance constructed, we need to follow that change here. + Option(hadoopConf.get(ConfVars.METASTOREWAREHOUSE.varname)).foreach { dir => + ret.getConf.setVar(ConfVars.METASTOREWAREHOUSE, dir) + } + ret + } else { + newState() + } + } + } + + // Log the default warehouse location. + logInfo( + s"Warehouse location for Hive client " + + s"(version ${version.fullVersion}) is ${conf.getVar(ConfVars.METASTOREWAREHOUSE)}") + + private def newState(): SessionState = { + val hiveConf = new HiveConf(classOf[SessionState]) + // HiveConf is a Hadoop Configuration, which has a field of classLoader and + // the initial value will be the current thread's context class loader + // (i.e. initClassLoader at here). + // We call initialConf.setClassLoader(initClassLoader) at here to make + // this action explicit. + hiveConf.setClassLoader(initClassLoader) + + // 1: Take all from the hadoopConf to this hiveConf. + // This hadoopConf contains user settings in Hadoop's core-site.xml file + // and Hive's hive-site.xml file. Note, we load hive-site.xml file manually in + // SharedState and put settings in this hadoopConf instead of relying on HiveConf + // to load user settings. Otherwise, HiveConf's initialize method will override + // settings in the hadoopConf. This issue only shows up when spark.sql.hive.metastore.jars + // is not set to builtin. When spark.sql.hive.metastore.jars is builtin, the classpath + // has hive-site.xml. So, HiveConf will use that to override its default values. + // 2: we set all spark confs to this hiveConf. + // 3: we set all entries in config to this hiveConf. + (hadoopConf.iterator().asScala.map(kv => kv.getKey -> kv.getValue) + ++ sparkConf.getAll.toMap ++ extraConfig).foreach { case (k, v) => + logDebug( + s""" + |Applying Hadoop/Hive/Spark and extra properties to Hive Conf: + |$k=${if (k.toLowerCase(Locale.ROOT).contains("password")) "xxx" else v} + """.stripMargin) + hiveConf.set(k, v) + } + val state = new SessionState(hiveConf) + if (clientLoader.cachedHive != null) { + Hive.set(clientLoader.cachedHive.asInstanceOf[Hive]) + } + SessionState.start(state) + state.out = new PrintStream(outputBuffer, true, "UTF-8") + state.err = new PrintStream(outputBuffer, true, "UTF-8") + state + } + + /** Returns the configuration for the current session. */ + def conf: HiveConf = state.getConf + + private val userName = state.getAuthenticator.getUserName + + override def getConf(key: String, defaultValue: String): String = { + conf.get(key, defaultValue) + } + + // We use hive's conf for compatibility. + private val retryLimit = conf.getIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES) + private val retryDelayMillis = shim.getMetastoreClientConnectRetryDelayMillis(conf) + + /** + * Runs `f` with multiple retries in case the hive metastore is temporarily unreachable. + */ + private def retryLocked[A](f: => A): A = clientLoader.synchronized { + // Hive sometimes retries internally, so set a deadline to avoid compounding delays. + val deadline = System.nanoTime + (retryLimit * retryDelayMillis * 1e6).toLong + var numTries = 0 + var caughtException: Exception = null + do { + numTries += 1 + try { + return f + } catch { + case e: Exception if causedByThrift(e) => + caughtException = e + logWarning( + "HiveClient got thrift exception, destroying client and retrying " + + s"(${retryLimit - numTries} tries remaining)", e) + clientLoader.cachedHive = null + Thread.sleep(retryDelayMillis) + } + } while (numTries <= retryLimit && System.nanoTime < deadline) + if (System.nanoTime > deadline) { + logWarning("Deadline exceeded") + } + throw caughtException + } + + private def causedByThrift(e: Throwable): Boolean = { + var target = e + while (target != null) { + val msg = target.getMessage() + if (msg != null && msg.matches("(?s).*(TApplication|TProtocol|TTransport)Exception.*")) { + return true + } + target = target.getCause() + } + false + } + + private def client: Hive = { + if (clientLoader.cachedHive != null) { + clientLoader.cachedHive.asInstanceOf[Hive] + } else { + val c = Hive.get(conf) + clientLoader.cachedHive = c + c + } + } + + /** Return the associated Hive [[SessionState]] of this [[HiveClientImpl]] */ + override def getState: SessionState = withHiveState(state) + + /** + * Runs `f` with ThreadLocal session state and classloaders configured for this version of hive. + */ + def withHiveState[A](f: => A): A = retryLocked { + val original = Thread.currentThread().getContextClassLoader + val originalConfLoader = state.getConf.getClassLoader + // The classloader in clientLoader could be changed after addJar, always use the latest + // classloader. We explicitly set the context class loader since "conf.setClassLoader" does + // not do that, and the Hive client libraries may need to load classes defined by the client's + // class loader. + Thread.currentThread().setContextClassLoader(clientLoader.classLoader) + state.getConf.setClassLoader(clientLoader.classLoader) + // Set the thread local metastore client to the client associated with this HiveClientImpl. + Hive.set(client) + // Replace conf in the thread local Hive with current conf + Hive.get(conf) + // setCurrentSessionState will use the classLoader associated + // with the HiveConf in `state` to override the context class loader of the current + // thread. + shim.setCurrentSessionState(state) + val ret = try f finally { + state.getConf.setClassLoader(originalConfLoader) + Thread.currentThread().setContextClassLoader(original) + HiveCatalogMetrics.incrementHiveClientCalls(1) + } + ret + } + + def setOut(stream: PrintStream): Unit = withHiveState { + state.out = stream + } + + def setInfo(stream: PrintStream): Unit = withHiveState { + state.info = stream + } + + def setError(stream: PrintStream): Unit = withHiveState { + state.err = stream + } + + override def setCurrentDatabase(databaseName: String): Unit = withHiveState { + if (databaseExists(databaseName)) { + state.setCurrentDatabase(databaseName) + } else { + throw new NoSuchDatabaseException(databaseName) + } + } + + override def createDatabase( + database: CatalogDatabase, + ignoreIfExists: Boolean): Unit = withHiveState { + client.createDatabase( + new HiveDatabase( + database.name, + database.description, + CatalogUtils.URIToString(database.locationUri), + Option(database.properties).map(_.asJava).orNull), + ignoreIfExists) + } + + override def dropDatabase( + name: String, + ignoreIfNotExists: Boolean, + cascade: Boolean): Unit = withHiveState { + client.dropDatabase(name, true, ignoreIfNotExists, cascade) + } + + override def alterDatabase(database: CatalogDatabase): Unit = withHiveState { + client.alterDatabase( + database.name, + new HiveDatabase( + database.name, + database.description, + CatalogUtils.URIToString(database.locationUri), + Option(database.properties).map(_.asJava).orNull)) + } + + override def getDatabase(dbName: String): CatalogDatabase = withHiveState { + Option(client.getDatabase(dbName)).map { d => + CatalogDatabase( + name = d.getName, + description = d.getDescription, + locationUri = CatalogUtils.stringToURI(d.getLocationUri), + properties = Option(d.getParameters).map(_.asScala.toMap).orNull) + }.getOrElse(throw new NoSuchDatabaseException(dbName)) + } + + override def databaseExists(dbName: String): Boolean = withHiveState { + client.databaseExists(dbName) + } + + override def listDatabases(pattern: String): Seq[String] = withHiveState { + client.getDatabasesByPattern(pattern).asScala + } + + override def tableExists(dbName: String, tableName: String): Boolean = withHiveState { + Option(client.getTable(dbName, tableName, false /* do not throw exception */)).nonEmpty + } + + override def getTableOption( + dbName: String, + tableName: String): Option[CatalogTable] = withHiveState { + logDebug(s"Looking up $dbName.$tableName") + Option(client.getTable(dbName, tableName, false)).map { h => + // Note: Hive separates partition columns and the schema, but for us the + // partition columns are part of the schema + val cols = h.getCols.asScala.map(fromHiveColumn) + val partCols = h.getPartCols.asScala.map(fromHiveColumn) + val schema = StructType(cols ++ partCols) + + val bucketSpec = if (h.getNumBuckets > 0) { + val sortColumnOrders = h.getSortCols.asScala + // Currently Spark only supports columns to be sorted in ascending order + // but Hive can support both ascending and descending order. If all the columns + // are sorted in ascending order, only then propagate the sortedness information + // to downstream processing / optimizations in Spark + // TODO: In future we can have Spark support columns sorted in descending order + val allAscendingSorted = sortColumnOrders.forall(_.getOrder == HIVE_COLUMN_ORDER_ASC) + + val sortColumnNames = if (allAscendingSorted) { + sortColumnOrders.map(_.getCol) + } else { + Seq.empty + } + Option(BucketSpec(h.getNumBuckets, h.getBucketCols.asScala, sortColumnNames)) + } else { + None + } + + // Skew spec and storage handler can't be mapped to CatalogTable (yet) + val unsupportedFeatures = ArrayBuffer.empty[String] + + if (!h.getSkewedColNames.isEmpty) { + unsupportedFeatures += "skewed columns" + } + + if (h.getStorageHandler != null) { + unsupportedFeatures += "storage handler" + } + + if (h.getTableType == HiveTableType.VIRTUAL_VIEW && partCols.nonEmpty) { + unsupportedFeatures += "partitioned view" + } + + val properties = Option(h.getParameters).map(_.asScala.toMap).orNull + + // Hive-generated Statistics are also recorded in ignoredProperties + val ignoredProperties = scala.collection.mutable.Map.empty[String, String] + for (key <- HiveStatisticsProperties; value <- properties.get(key)) { + ignoredProperties += key -> value + } + + val excludedTableProperties = HiveStatisticsProperties ++ Set( + // The property value of "comment" is moved to the dedicated field "comment" + "comment", + // For EXTERNAL_TABLE, the table properties has a particular field "EXTERNAL". This is added + // in the function toHiveTable. + "EXTERNAL" + ) + + val filteredProperties = properties.filterNot { + case (key, _) => excludedTableProperties.contains(key) + } + val comment = properties.get("comment") + + // Here we are reading statistics from Hive. + // Note that this statistics could be overridden by Spark's statistics if that's available. + val totalSize = properties.get(StatsSetupConst.TOTAL_SIZE).map(BigInt(_)) + val rawDataSize = properties.get(StatsSetupConst.RAW_DATA_SIZE).map(BigInt(_)) + val rowCount = properties.get(StatsSetupConst.ROW_COUNT).map(BigInt(_)).filter(_ >= 0) + // TODO: check if this estimate is valid for tables after partition pruning. + // NOTE: getting `totalSize` directly from params is kind of hacky, but this should be + // relatively cheap if parameters for the table are populated into the metastore. + // Currently, only totalSize, rawDataSize, and rowCount are used to build the field `stats` + // TODO: stats should include all the other two fields (`numFiles` and `numPartitions`). + // (see StatsSetupConst in Hive) + val stats = + // When table is external, `totalSize` is always zero, which will influence join strategy + // so when `totalSize` is zero, use `rawDataSize` instead. When `rawDataSize` is also zero, + // return None. Later, we will use the other ways to estimate the statistics. + if (totalSize.isDefined && totalSize.get > 0L) { + Some(CatalogStatistics(sizeInBytes = totalSize.get, rowCount = rowCount)) + } else if (rawDataSize.isDefined && rawDataSize.get > 0) { + Some(CatalogStatistics(sizeInBytes = rawDataSize.get, rowCount = rowCount)) + } else { + // TODO: still fill the rowCount even if sizeInBytes is empty. Might break anything? + None + } + + CatalogTable( + identifier = TableIdentifier(h.getTableName, Option(h.getDbName)), + tableType = h.getTableType match { + case HiveTableType.EXTERNAL_TABLE => CatalogTableType.EXTERNAL + case HiveTableType.MANAGED_TABLE => CatalogTableType.MANAGED + case HiveTableType.VIRTUAL_VIEW => CatalogTableType.VIEW + case HiveTableType.INDEX_TABLE => + throw new AnalysisException("Hive index table is not supported.") + }, + schema = schema, + partitionColumnNames = partCols.map(_.name), + // If the table is written by Spark, we will put bucketing information in table properties, + // and will always overwrite the bucket spec in hive metastore by the bucketing information + // in table properties. This means, if we have bucket spec in both hive metastore and + // table properties, we will trust the one in table properties. + bucketSpec = bucketSpec, + owner = h.getOwner, + createTime = h.getTTable.getCreateTime.toLong * 1000, + lastAccessTime = h.getLastAccessTime.toLong * 1000, + storage = CatalogStorageFormat( + locationUri = shim.getDataLocation(h).map(CatalogUtils.stringToURI), + // To avoid ClassNotFound exception, we try our best to not get the format class, but get + // the class name directly. However, for non-native tables, there is no interface to get + // the format class name, so we may still throw ClassNotFound in this case. + inputFormat = Option(h.getTTable.getSd.getInputFormat).orElse { + Option(h.getStorageHandler).map(_.getInputFormatClass.getName) + }, + outputFormat = Option(h.getTTable.getSd.getOutputFormat).orElse { + Option(h.getStorageHandler).map(_.getOutputFormatClass.getName) + }, + serde = Option(h.getSerializationLib), + compressed = h.getTTable.getSd.isCompressed, + properties = Option(h.getTTable.getSd.getSerdeInfo.getParameters) + .map(_.asScala.toMap).orNull + ), + // For EXTERNAL_TABLE, the table properties has a particular field "EXTERNAL". This is added + // in the function toHiveTable. + properties = filteredProperties, + stats = stats, + comment = comment, + // In older versions of Spark(before 2.2.0), we expand the view original text and store + // that into `viewExpandedText`, and that should be used in view resolution. So we get + // `viewExpandedText` instead of `viewOriginalText` for viewText here. + viewText = Option(h.getViewExpandedText), + unsupportedFeatures = unsupportedFeatures, + ignoredProperties = ignoredProperties.toMap) + } + } + + override def createTable(table: CatalogTable, ignoreIfExists: Boolean): Unit = withHiveState { + client.createTable(toHiveTable(table, Some(userName)), ignoreIfExists) + } + + override def dropTable( + dbName: String, + tableName: String, + ignoreIfNotExists: Boolean, + purge: Boolean): Unit = withHiveState { + shim.dropTable(client, dbName, tableName, true, ignoreIfNotExists, purge) + } + + override def alterTable( + dbName: String, + tableName: String, + table: CatalogTable): Unit = withHiveState { + // getTableOption removes all the Hive-specific properties. Here, we fill them back to ensure + // these properties are still available to the others that share the same Hive metastore. + // If users explicitly alter these Hive-specific properties through ALTER TABLE DDL, we respect + // these user-specified values. + val hiveTable = toHiveTable( + table.copy(properties = table.ignoredProperties ++ table.properties), Some(userName)) + // Do not use `table.qualifiedName` here because this may be a rename + val qualifiedTableName = s"$dbName.$tableName" + shim.alterTable(client, qualifiedTableName, hiveTable) + } + + override def createPartitions( + db: String, + table: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit = withHiveState { + shim.createPartitions(client, db, table, parts, ignoreIfExists) + } + + override def dropPartitions( + db: String, + table: String, + specs: Seq[TablePartitionSpec], + ignoreIfNotExists: Boolean, + purge: Boolean, + retainData: Boolean): Unit = withHiveState { + // TODO: figure out how to drop multiple partitions in one call + val hiveTable = client.getTable(db, table, true /* throw exception */) + // do the check at first and collect all the matching partitions + val matchingParts = + specs.flatMap { s => + assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid") + // The provided spec here can be a partial spec, i.e. it will match all partitions + // whose specs are supersets of this partial spec. E.g. If a table has partitions + // (b='1', c='1') and (b='1', c='2'), a partial spec of (b='1') will match both. + val parts = client.getPartitions(hiveTable, s.asJava).asScala + if (parts.isEmpty && !ignoreIfNotExists) { + throw new AnalysisException( + s"No partition is dropped. One partition spec '$s' does not exist in table '$table' " + + s"database '$db'") + } + parts.map(_.getValues) + }.distinct + var droppedParts = ArrayBuffer.empty[java.util.List[String]] + matchingParts.foreach { partition => + try { + shim.dropPartition(client, db, table, partition, !retainData, purge) + } catch { + case e: Exception => + val remainingParts = matchingParts.toBuffer -- droppedParts + logError( + s""" + |====================== + |Attempt to drop the partition specs in table '$table' database '$db': + |${specs.mkString("\n")} + |In this attempt, the following partitions have been dropped successfully: + |${droppedParts.mkString("\n")} + |The remaining partitions have not been dropped: + |${remainingParts.mkString("\n")} + |====================== + """.stripMargin) + throw e + } + droppedParts += partition + } + } + + override def renamePartitions( + db: String, + table: String, + specs: Seq[TablePartitionSpec], + newSpecs: Seq[TablePartitionSpec]): Unit = withHiveState { + require(specs.size == newSpecs.size, "number of old and new partition specs differ") + val catalogTable = getTable(db, table) + val hiveTable = toHiveTable(catalogTable, Some(userName)) + specs.zip(newSpecs).foreach { case (oldSpec, newSpec) => + val hivePart = getPartitionOption(catalogTable, oldSpec) + .map { p => toHivePartition(p.copy(spec = newSpec), hiveTable) } + .getOrElse { throw new NoSuchPartitionException(db, table, oldSpec) } + client.renamePartition(hiveTable, oldSpec.asJava, hivePart) + } + } + + override def alterPartitions( + db: String, + table: String, + newParts: Seq[CatalogTablePartition]): Unit = withHiveState { + val hiveTable = toHiveTable(getTable(db, table), Some(userName)) + shim.alterPartitions(client, table, newParts.map { p => toHivePartition(p, hiveTable) }.asJava) + } + + /** + * Returns the partition names for the given table that match the supplied partition spec. + * If no partition spec is specified, all partitions are returned. + * + * The returned sequence is sorted as strings. + */ + override def getPartitionNames( + table: CatalogTable, + partialSpec: Option[TablePartitionSpec] = None): Seq[String] = withHiveState { + val hivePartitionNames = + partialSpec match { + case None => + // -1 for result limit means "no limit/return all" + client.getPartitionNames(table.database, table.identifier.table, -1) + case Some(s) => + assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid") + client.getPartitionNames(table.database, table.identifier.table, s.asJava, -1) + } + hivePartitionNames.asScala.sorted + } + + override def getPartitionOption( + table: CatalogTable, + spec: TablePartitionSpec): Option[CatalogTablePartition] = withHiveState { + val hiveTable = toHiveTable(table, Some(userName)) + val hivePartition = client.getPartition(hiveTable, spec.asJava, false) + Option(hivePartition).map(fromHivePartition) + } + + /** + * Returns the partitions for the given table that match the supplied partition spec. + * If no partition spec is specified, all partitions are returned. + */ + override def getPartitions( + table: CatalogTable, + spec: Option[TablePartitionSpec]): Seq[CatalogTablePartition] = withHiveState { + val hiveTable = toHiveTable(table, Some(userName)) + val parts = spec match { + case None => shim.getAllPartitions(client, hiveTable).map(fromHivePartition) + case Some(s) => + assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid") + client.getPartitions(hiveTable, s.asJava).asScala.map(fromHivePartition) + } + HiveCatalogMetrics.incrementFetchedPartitions(parts.length) + parts + } + + override def getPartitionsByFilter( + table: CatalogTable, + predicates: Seq[Expression]): Seq[CatalogTablePartition] = withHiveState { + val hiveTable = toHiveTable(table, Some(userName)) + val parts = shim.getPartitionsByFilter(client, hiveTable, predicates).map(fromHivePartition) + HiveCatalogMetrics.incrementFetchedPartitions(parts.length) + parts + } + + override def listTables(dbName: String): Seq[String] = withHiveState { + client.getAllTables(dbName).asScala + } + + override def listTables(dbName: String, pattern: String): Seq[String] = withHiveState { + client.getTablesByPattern(dbName, pattern).asScala + } + + /** + * Runs the specified SQL query using Hive. + */ + override def runSqlHive(sql: String): Seq[String] = { + val maxResults = 100000 + val results = runHive(sql, maxResults) + // It is very confusing when you only get back some of the results... + if (results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") + results + } + + /** + * Execute the command using Hive and return the results as a sequence. Each element + * in the sequence is one row. + */ + protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = withHiveState { + logDebug(s"Running hiveql '$cmd'") + if (cmd.toLowerCase(Locale.ROOT).startsWith("set")) { logDebug(s"Changing config: $cmd") } + try { + val cmd_trimmed: String = cmd.trim() + val tokens: Array[String] = cmd_trimmed.split("\\s+") + // The remainder of the command. + val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() + val proc = shim.getCommandProcessor(tokens(0), conf) + proc match { + case driver: Driver => + val response: CommandProcessorResponse = driver.run(cmd) + // Throw an exception if there is an error in query processing. + if (response.getResponseCode != 0) { + driver.close() + CommandProcessorFactory.clean(conf) + throw new QueryExecutionException(response.getErrorMessage) + } + driver.setMaxRows(maxRows) + + val results = shim.getDriverResults(driver) + driver.close() + CommandProcessorFactory.clean(conf) + results + + case _ => + if (state.out != null) { + // scalastyle:off println + state.out.println(tokens(0) + " " + cmd_1) + // scalastyle:on println + } + Seq(proc.run(cmd_1).getResponseCode.toString) + } + } catch { + case e: Exception => + logError( + s""" + |====================== + |HIVE FAILURE OUTPUT + |====================== + |${outputBuffer.toString} + |====================== + |END HIVE FAILURE OUTPUT + |====================== + """.stripMargin) + throw e + } + } + + def loadPartition( + loadPath: String, + dbName: String, + tableName: String, + partSpec: java.util.LinkedHashMap[String, String], + replace: Boolean, + inheritTableSpecs: Boolean, + isSrcLocal: Boolean): Unit = withHiveState { + val hiveTable = client.getTable(dbName, tableName, true /* throw exception */) + shim.loadPartition( + client, + new Path(loadPath), // TODO: Use URI + s"$dbName.$tableName", + partSpec, + replace, + inheritTableSpecs, + isSkewedStoreAsSubdir = hiveTable.isStoredAsSubDirectories, + isSrcLocal = isSrcLocal) + } + + def loadTable( + loadPath: String, // TODO URI + tableName: String, + replace: Boolean, + isSrcLocal: Boolean): Unit = withHiveState { + shim.loadTable( + client, + new Path(loadPath), + tableName, + replace, + isSrcLocal) + } + + def loadDynamicPartitions( + loadPath: String, + dbName: String, + tableName: String, + partSpec: java.util.LinkedHashMap[String, String], + replace: Boolean, + numDP: Int): Unit = withHiveState { + val hiveTable = client.getTable(dbName, tableName, true /* throw exception */) + shim.loadDynamicPartitions( + client, + new Path(loadPath), + s"$dbName.$tableName", + partSpec, + replace, + numDP, + listBucketingEnabled = hiveTable.isStoredAsSubDirectories) + } + + override def createFunction(db: String, func: CatalogFunction): Unit = withHiveState { + shim.createFunction(client, db, func) + } + + override def dropFunction(db: String, name: String): Unit = withHiveState { + shim.dropFunction(client, db, name) + } + + override def renameFunction(db: String, oldName: String, newName: String): Unit = withHiveState { + shim.renameFunction(client, db, oldName, newName) + } + + override def alterFunction(db: String, func: CatalogFunction): Unit = withHiveState { + shim.alterFunction(client, db, func) + } + + override def getFunctionOption( + db: String, name: String): Option[CatalogFunction] = withHiveState { + shim.getFunctionOption(client, db, name) + } + + override def listFunctions(db: String, pattern: String): Seq[String] = withHiveState { + shim.listFunctions(client, db, pattern) + } + + def addJar(path: String): Unit = { + val uri = new Path(path).toUri + val jarURL = if (uri.getScheme == null) { + // `path` is a local file path without a URL scheme + new File(path).toURI.toURL + } else { + // `path` is a URL with a scheme + uri.toURL + } + clientLoader.addJar(jarURL) + runSqlHive(s"ADD JAR $path") + } + + def newSession(): HiveClientImpl = { + clientLoader.createClient().asInstanceOf[HiveClientImpl] + } + + def reset(): Unit = withHiveState { + client.getAllTables("default").asScala.foreach { t => + logDebug(s"Deleting table $t") + val table = client.getTable("default", t) + client.getIndexes("default", t, 255).asScala.foreach { index => + shim.dropIndex(client, "default", t, index.getIndexName) + } + if (!table.isIndexTable) { + client.dropTable("default", t) + } + } + client.getAllDatabases.asScala.filterNot(_ == "default").foreach { db => + logDebug(s"Dropping Database: $db") + client.dropDatabase(db, true, false, true) + } + } +} + +private[hive] object HiveClientImpl { + /** Converts the native StructField to Hive's FieldSchema. */ + def toHiveColumn(c: StructField): FieldSchema = { + val typeString = if (c.metadata.contains(HIVE_TYPE_STRING)) { + c.metadata.getString(HIVE_TYPE_STRING) + } else { + c.dataType.catalogString + } + new FieldSchema(c.name, typeString, c.getComment().orNull) + } + + /** Builds the native StructField from Hive's FieldSchema. */ + def fromHiveColumn(hc: FieldSchema): StructField = { + val columnType = try { + CatalystSqlParser.parseDataType(hc.getType) + } catch { + case e: ParseException => + throw new SparkException("Cannot recognize hive type string: " + hc.getType, e) + } + + val metadata = if (hc.getType != columnType.catalogString) { + new MetadataBuilder().putString(HIVE_TYPE_STRING, hc.getType).build() + } else { + Metadata.empty + } + + val field = StructField( + name = hc.getName, + dataType = columnType, + nullable = true, + metadata = metadata) + Option(hc.getComment).map(field.withComment).getOrElse(field) + } + + private def toInputFormat(name: String) = + Utils.classForName(name).asInstanceOf[Class[_ <: org.apache.hadoop.mapred.InputFormat[_, _]]] + + private def toOutputFormat(name: String) = + Utils.classForName(name) + .asInstanceOf[Class[_ <: org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]]] + + /** + * Converts the native table metadata representation format CatalogTable to Hive's Table. + */ + def toHiveTable(table: CatalogTable, userName: Option[String] = None): HiveTable = { + val hiveTable = new HiveTable(table.database, table.identifier.table) + // For EXTERNAL_TABLE, we also need to set EXTERNAL field in the table properties. + // Otherwise, Hive metastore will change the table to a MANAGED_TABLE. + // (metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L1095-L1105) + hiveTable.setTableType(table.tableType match { + case CatalogTableType.EXTERNAL => + hiveTable.setProperty("EXTERNAL", "TRUE") + HiveTableType.EXTERNAL_TABLE + case CatalogTableType.MANAGED => + HiveTableType.MANAGED_TABLE + case CatalogTableType.VIEW => HiveTableType.VIRTUAL_VIEW + }) + // Note: In Hive the schema and partition columns must be disjoint sets + val (partCols, schema) = table.schema.map(toHiveColumn).partition { c => + table.partitionColumnNames.contains(c.getName) + } + // after SPARK-19279, it is not allowed to create a hive table with an empty schema, + // so here we should not add a default col schema + if (schema.isEmpty && HiveExternalCatalog.isDatasourceTable(table)) { + // This is a hack to preserve existing behavior. Before Spark 2.0, we do not + // set a default serde here (this was done in Hive), and so if the user provides + // an empty schema Hive would automatically populate the schema with a single + // field "col". However, after SPARK-14388, we set the default serde to + // LazySimpleSerde so this implicit behavior no longer happens. Therefore, + // we need to do it in Spark ourselves. + hiveTable.setFields( + Seq(new FieldSchema("col", "array", "from deserializer")).asJava) + } else { + hiveTable.setFields(schema.asJava) + } + hiveTable.setPartCols(partCols.asJava) + userName.foreach(hiveTable.setOwner) + hiveTable.setCreateTime((table.createTime / 1000).toInt) + hiveTable.setLastAccessTime((table.lastAccessTime / 1000).toInt) + table.storage.locationUri.map(CatalogUtils.URIToString).foreach { loc => + hiveTable.getTTable.getSd.setLocation(loc)} + table.storage.inputFormat.map(toInputFormat).foreach(hiveTable.setInputFormatClass) + table.storage.outputFormat.map(toOutputFormat).foreach(hiveTable.setOutputFormatClass) + hiveTable.setSerializationLib( + table.storage.serde.getOrElse("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) + table.storage.properties.foreach { case (k, v) => hiveTable.setSerdeParam(k, v) } + table.properties.foreach { case (k, v) => hiveTable.setProperty(k, v) } + table.comment.foreach { c => hiveTable.setProperty("comment", c) } + // Hive will expand the view text, so it needs 2 fields: viewOriginalText and viewExpandedText. + // Since we don't expand the view text, but only add table properties, we map the `viewText` to + // the both fields in hive table. + table.viewText.foreach { t => + hiveTable.setViewOriginalText(t) + hiveTable.setViewExpandedText(t) + } + + table.bucketSpec match { + case Some(bucketSpec) if DDLUtils.isHiveTable(table) => + hiveTable.setNumBuckets(bucketSpec.numBuckets) + hiveTable.setBucketCols(bucketSpec.bucketColumnNames.toList.asJava) + + if (bucketSpec.sortColumnNames.nonEmpty) { + hiveTable.setSortCols( + bucketSpec.sortColumnNames + .map(col => new Order(col, HIVE_COLUMN_ORDER_ASC)) + .toList + .asJava + ) + } + case _ => + } + + hiveTable + } + + /** + * Converts the native partition metadata representation format CatalogTablePartition to + * Hive's Partition. + */ + def toHivePartition( + p: CatalogTablePartition, + ht: HiveTable): HivePartition = { + val tpart = new org.apache.hadoop.hive.metastore.api.Partition + val partValues = ht.getPartCols.asScala.map { hc => + p.spec.get(hc.getName).getOrElse { + throw new IllegalArgumentException( + s"Partition spec is missing a value for column '${hc.getName}': ${p.spec}") + } + } + val storageDesc = new StorageDescriptor + val serdeInfo = new SerDeInfo + p.storage.locationUri.map(CatalogUtils.URIToString(_)).foreach(storageDesc.setLocation) + p.storage.inputFormat.foreach(storageDesc.setInputFormat) + p.storage.outputFormat.foreach(storageDesc.setOutputFormat) + p.storage.serde.foreach(serdeInfo.setSerializationLib) + serdeInfo.setParameters(p.storage.properties.asJava) + storageDesc.setSerdeInfo(serdeInfo) + tpart.setDbName(ht.getDbName) + tpart.setTableName(ht.getTableName) + tpart.setValues(partValues.asJava) + tpart.setSd(storageDesc) + tpart.setParameters(mutable.Map(p.parameters.toSeq: _*).asJava) + new HivePartition(ht, tpart) + } + + /** + * Build the native partition metadata from Hive's Partition. + */ + def fromHivePartition(hp: HivePartition): CatalogTablePartition = { + val apiPartition = hp.getTPartition + CatalogTablePartition( + spec = Option(hp.getSpec).map(_.asScala.toMap).getOrElse(Map.empty), + storage = CatalogStorageFormat( + locationUri = Option(CatalogUtils.stringToURI(apiPartition.getSd.getLocation)), + inputFormat = Option(apiPartition.getSd.getInputFormat), + outputFormat = Option(apiPartition.getSd.getOutputFormat), + serde = Option(apiPartition.getSd.getSerdeInfo.getSerializationLib), + compressed = apiPartition.getSd.isCompressed, + properties = Option(apiPartition.getSd.getSerdeInfo.getParameters) + .map(_.asScala.toMap).orNull), + parameters = + if (hp.getParameters() != null) hp.getParameters().asScala.toMap else Map.empty) + } + + // Below is the key of table properties for storing Hive-generated statistics + private val HiveStatisticsProperties = Set( + StatsSetupConst.COLUMN_STATS_ACCURATE, + StatsSetupConst.NUM_FILES, + StatsSetupConst.NUM_PARTITIONS, + StatsSetupConst.ROW_COUNT, + StatsSetupConst.RAW_DATA_SIZE, + StatsSetupConst.TOTAL_SIZE + ) +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 48bbb21e6c1d..cde20da186ac 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -18,28 +18,41 @@ package org.apache.spark.sql.hive.client import java.lang.{Boolean => JBoolean, Integer => JInteger, Long => JLong} -import java.lang.reflect.{Method, Modifier} +import java.lang.reflect.{InvocationTargetException, Method, Modifier} import java.net.URI -import java.util.{ArrayList => JArrayList, List => JList, Map => JMap, Set => JSet} +import java.util.{ArrayList => JArrayList, List => JList, Locale, Map => JMap, Set => JSet} import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ +import scala.util.Try +import scala.util.control.NonFatal -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.api.{EnvironmentContext, Function => HiveFunction, FunctionType} +import org.apache.hadoop.hive.metastore.api.{MetaException, PrincipalType, ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver -import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} +import org.apache.hadoop.hive.ql.io.AcidUtils +import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition, Table} +import org.apache.hadoop.hive.ql.plan.AddPartitionDesc import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.spark.Logging +import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException +import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTablePartition, CatalogUtils, FunctionResource, FunctionResourceType} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{StringType, IntegralType} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegralType, StringType} +import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.util.Utils /** - * A shim that defines the interface between ClientWrapper and the underlying Hive library used to - * talk to the metastore. Each Hive version has its own implementation of this class, defining + * A shim that defines the interface between [[HiveClientImpl]] and the underlying Hive library used + * to talk to the metastore. Each Hive version has its own implementation of this class, defining * version-specific version of needed functions. * * The guideline for writing shims is: @@ -52,7 +65,6 @@ private[client] sealed abstract class Shim { /** * Set the current SessionState to the given SessionState. Also, set the context classloader of * the current thread to the one set in the HiveConf of this given `state`. - * @param state */ def setCurrentSessionState(state: SessionState): Unit @@ -74,22 +86,33 @@ private[client] sealed abstract class Shim { def getMetastoreClientConnectRetryDelayMillis(conf: HiveConf): Long + def alterTable(hive: Hive, tableName: String, table: Table): Unit + + def alterPartitions(hive: Hive, tableName: String, newParts: JList[Partition]): Unit + + def createPartitions( + hive: Hive, + db: String, + table: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit + def loadPartition( hive: Hive, loadPath: Path, tableName: String, partSpec: JMap[String, String], replace: Boolean, - holdDDLTime: Boolean, inheritTableSpecs: Boolean, - isSkewedStoreAsSubdir: Boolean): Unit + isSkewedStoreAsSubdir: Boolean, + isSrcLocal: Boolean): Unit def loadTable( hive: Hive, loadPath: Path, tableName: String, replace: Boolean, - holdDDLTime: Boolean): Unit + isSrcLocal: Boolean): Unit def loadDynamicPartitions( hive: Hive, @@ -98,11 +121,38 @@ private[client] sealed abstract class Shim { partSpec: JMap[String, String], replace: Boolean, numDP: Int, - holdDDLTime: Boolean, listBucketingEnabled: Boolean): Unit + def createFunction(hive: Hive, db: String, func: CatalogFunction): Unit + + def dropFunction(hive: Hive, db: String, name: String): Unit + + def renameFunction(hive: Hive, db: String, oldName: String, newName: String): Unit + + def alterFunction(hive: Hive, db: String, func: CatalogFunction): Unit + + def getFunctionOption(hive: Hive, db: String, name: String): Option[CatalogFunction] + + def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] + def dropIndex(hive: Hive, dbName: String, tableName: String, indexName: String): Unit + def dropTable( + hive: Hive, + dbName: String, + tableName: String, + deleteData: Boolean, + ignoreIfNotExists: Boolean, + purge: Boolean): Unit + + def dropPartition( + hive: Hive, + dbName: String, + tableName: String, + part: JList[String], + deleteData: Boolean, + purge: Boolean): Unit + protected def findStaticMethod(klass: Class[_], name: String, args: Class[_]*): Method = { val method = findMethod(klass, name, args: _*) require(Modifier.isStatic(method.getModifiers()), @@ -113,10 +163,13 @@ private[client] sealed abstract class Shim { protected def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = { klass.getMethod(name, args: _*) } - } private[client] class Shim_v0_12 extends Shim with Logging { + // See HIVE-12224, HOLD_DDLTIME was broken as soon as it landed + protected lazy val holdDDLTime = JBoolean.FALSE + // deletes the underlying data along with metadata + protected lazy val deleteDataInDropIndex = JBoolean.TRUE private lazy val startMethod = findStaticMethod( @@ -145,6 +198,22 @@ private[client] class Shim_v0_12 extends Shim with Logging { classOf[Driver], "getResults", classOf[JArrayList[String]]) + private lazy val createPartitionMethod = + findMethod( + classOf[Hive], + "createPartition", + classOf[Table], + classOf[JMap[String, String]], + classOf[Path], + classOf[JMap[String, String]], + classOf[String], + classOf[String], + JInteger.TYPE, + classOf[JList[Object]], + classOf[String], + classOf[JMap[String, String]], + classOf[JList[Object]], + classOf[JList[Object]]) private lazy val loadPartitionMethod = findMethod( classOf[Hive], @@ -183,6 +252,18 @@ private[client] class Shim_v0_12 extends Shim with Logging { classOf[String], classOf[String], JBoolean.TYPE) + private lazy val alterTableMethod = + findMethod( + classOf[Hive], + "alterTable", + classOf[String], + classOf[Table]) + private lazy val alterPartitionsMethod = + findMethod( + classOf[Hive], + "alterPartitions", + classOf[String], + classOf[JList[Partition]]) override def setCurrentSessionState(state: SessionState): Unit = { // Starting from Hive 0.13, setCurrentSessionState will internally override @@ -200,6 +281,44 @@ private[client] class Shim_v0_12 extends Shim with Logging { override def setDataLocation(table: Table, loc: String): Unit = setDataLocationMethod.invoke(table, new URI(loc)) + // Follows exactly the same logic of DDLTask.createPartitions in Hive 0.12 + override def createPartitions( + hive: Hive, + database: String, + tableName: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit = { + val table = hive.getTable(database, tableName) + parts.foreach { s => + val location = s.storage.locationUri.map( + uri => new Path(table.getPath, new Path(uri))).orNull + val params = if (s.parameters.nonEmpty) s.parameters.asJava else null + val spec = s.spec.asJava + if (hive.getPartition(table, spec, false) != null && ignoreIfExists) { + // Ignore this partition since it already exists and ignoreIfExists == true + } else { + if (location == null && table.isView()) { + throw new HiveException("LOCATION clause illegal for view partition"); + } + + createPartitionMethod.invoke( + hive, + table, + spec, + location, + params, // partParams + null, // inputFormat + null, // outputFormat + -1: JInteger, // numBuckets + null, // cols + null, // serializationLib + null, // serdeParams + null, // bucketCols + null) // sortCols + } + } + } + override def getAllPartitions(hive: Hive, table: Table): Seq[Partition] = getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]].asScala.toSeq @@ -233,11 +352,11 @@ private[client] class Shim_v0_12 extends Shim with Logging { tableName: String, partSpec: JMap[String, String], replace: Boolean, - holdDDLTime: Boolean, inheritTableSpecs: Boolean, - isSkewedStoreAsSubdir: Boolean): Unit = { + isSkewedStoreAsSubdir: Boolean, + isSrcLocal: Boolean): Unit = { loadPartitionMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean, - holdDDLTime: JBoolean, inheritTableSpecs: JBoolean, isSkewedStoreAsSubdir: JBoolean) + JBoolean.FALSE, inheritTableSpecs: JBoolean, isSkewedStoreAsSubdir: JBoolean) } override def loadTable( @@ -245,8 +364,8 @@ private[client] class Shim_v0_12 extends Shim with Logging { loadPath: Path, tableName: String, replace: Boolean, - holdDDLTime: Boolean): Unit = { - loadTableMethod.invoke(hive, loadPath, tableName, replace: JBoolean, holdDDLTime: JBoolean) + isSrcLocal: Boolean): Unit = { + loadTableMethod.invoke(hive, loadPath, tableName, replace: JBoolean, holdDDLTime) } override def loadDynamicPartitions( @@ -256,16 +375,73 @@ private[client] class Shim_v0_12 extends Shim with Logging { partSpec: JMap[String, String], replace: Boolean, numDP: Int, - holdDDLTime: Boolean, listBucketingEnabled: Boolean): Unit = { loadDynamicPartitionsMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean, - numDP: JInteger, holdDDLTime: JBoolean, listBucketingEnabled: JBoolean) + numDP: JInteger, holdDDLTime, listBucketingEnabled: JBoolean) } override def dropIndex(hive: Hive, dbName: String, tableName: String, indexName: String): Unit = { - dropIndexMethod.invoke(hive, dbName, tableName, indexName, true: JBoolean) + dropIndexMethod.invoke(hive, dbName, tableName, indexName, deleteDataInDropIndex) + } + + override def dropTable( + hive: Hive, + dbName: String, + tableName: String, + deleteData: Boolean, + ignoreIfNotExists: Boolean, + purge: Boolean): Unit = { + if (purge) { + throw new UnsupportedOperationException("DROP TABLE ... PURGE") + } + hive.dropTable(dbName, tableName, deleteData, ignoreIfNotExists) + } + + override def alterTable(hive: Hive, tableName: String, table: Table): Unit = { + alterTableMethod.invoke(hive, tableName, table) + } + + override def alterPartitions(hive: Hive, tableName: String, newParts: JList[Partition]): Unit = { + alterPartitionsMethod.invoke(hive, tableName, newParts) + } + + override def dropPartition( + hive: Hive, + dbName: String, + tableName: String, + part: JList[String], + deleteData: Boolean, + purge: Boolean): Unit = { + if (purge) { + throw new UnsupportedOperationException("ALTER TABLE ... DROP PARTITION ... PURGE") + } + hive.dropPartition(dbName, tableName, part, deleteData) + } + + override def createFunction(hive: Hive, db: String, func: CatalogFunction): Unit = { + throw new AnalysisException("Hive 0.12 doesn't support creating permanent functions. " + + "Please use Hive 0.13 or higher.") + } + + def dropFunction(hive: Hive, db: String, name: String): Unit = { + throw new NoSuchPermanentFunctionException(db, name) + } + + def renameFunction(hive: Hive, db: String, oldName: String, newName: String): Unit = { + throw new NoSuchPermanentFunctionException(db, oldName) + } + + def alterFunction(hive: Hive, db: String, func: CatalogFunction): Unit = { + throw new NoSuchPermanentFunctionException(db, func.identifier.funcName) + } + + def getFunctionOption(hive: Hive, db: String, name: String): Option[CatalogFunction] = { + None } + def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] = { + Seq.empty[String] + } } private[client] class Shim_v0_13 extends Shim_v0_12 { @@ -309,9 +485,99 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { override def setDataLocation(table: Table, loc: String): Unit = setDataLocationMethod.invoke(table, new Path(loc)) + override def createPartitions( + hive: Hive, + db: String, + table: String, + parts: Seq[CatalogTablePartition], + ignoreIfExists: Boolean): Unit = { + val addPartitionDesc = new AddPartitionDesc(db, table, ignoreIfExists) + parts.zipWithIndex.foreach { case (s, i) => + addPartitionDesc.addPartition( + s.spec.asJava, s.storage.locationUri.map(CatalogUtils.URIToString(_)).orNull) + if (s.parameters.nonEmpty) { + addPartitionDesc.getPartition(i).setPartParams(s.parameters.asJava) + } + } + hive.createPartitions(addPartitionDesc) + } + override def getAllPartitions(hive: Hive, table: Table): Seq[Partition] = getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]].asScala.toSeq + private def toHiveFunction(f: CatalogFunction, db: String): HiveFunction = { + val resourceUris = f.resources.map { resource => + new ResourceUri(ResourceType.valueOf( + resource.resourceType.resourceType.toUpperCase(Locale.ROOT)), resource.uri) + } + new HiveFunction( + f.identifier.funcName, + db, + f.className, + null, + PrincipalType.USER, + (System.currentTimeMillis / 1000).toInt, + FunctionType.JAVA, + resourceUris.asJava) + } + + override def createFunction(hive: Hive, db: String, func: CatalogFunction): Unit = { + hive.createFunction(toHiveFunction(func, db)) + } + + override def dropFunction(hive: Hive, db: String, name: String): Unit = { + hive.dropFunction(db, name) + } + + override def renameFunction(hive: Hive, db: String, oldName: String, newName: String): Unit = { + val catalogFunc = getFunctionOption(hive, db, oldName) + .getOrElse(throw new NoSuchPermanentFunctionException(db, oldName)) + .copy(identifier = FunctionIdentifier(newName, Some(db))) + val hiveFunc = toHiveFunction(catalogFunc, db) + hive.alterFunction(db, oldName, hiveFunc) + } + + override def alterFunction(hive: Hive, db: String, func: CatalogFunction): Unit = { + hive.alterFunction(db, func.identifier.funcName, toHiveFunction(func, db)) + } + + private def fromHiveFunction(hf: HiveFunction): CatalogFunction = { + val name = FunctionIdentifier(hf.getFunctionName, Option(hf.getDbName)) + val resources = hf.getResourceUris.asScala.map { uri => + val resourceType = uri.getResourceType() match { + case ResourceType.ARCHIVE => "archive" + case ResourceType.FILE => "file" + case ResourceType.JAR => "jar" + case r => throw new AnalysisException(s"Unknown resource type: $r") + } + FunctionResource(FunctionResourceType.fromString(resourceType), uri.getUri()) + } + CatalogFunction(name, hf.getClassName, resources) + } + + override def getFunctionOption(hive: Hive, db: String, name: String): Option[CatalogFunction] = { + try { + Option(hive.getFunction(db, name)).map(fromHiveFunction) + } catch { + case NonFatal(e) if isCausedBy(e, s"$name does not exist") => + None + } + } + + private def isCausedBy(e: Throwable, matchMassage: String): Boolean = { + if (e.getMessage.contains(matchMassage)) { + true + } else if (e.getCause != null) { + isCausedBy(e.getCause, matchMassage) + } else { + false + } + } + + override def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] = { + hive.getFunctions(db, pattern).asScala + } + /** * Converts catalyst expression to the format that Hive's getPartitionsByFilter() expects, i.e. * a string that represents partition predicates like "str_key=\"value\" and int_key=1 ...". @@ -320,22 +586,83 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { */ def convertFilters(table: Table, filters: Seq[Expression]): String = { // hive varchar is treated as catalyst string, but hive varchar can't be pushed down. - val varcharKeys = table.getPartitionKeys.asScala - .filter(col => col.getType.startsWith(serdeConstants.VARCHAR_TYPE_NAME)) + lazy val varcharKeys = table.getPartitionKeys.asScala + .filter(col => col.getType.startsWith(serdeConstants.VARCHAR_TYPE_NAME) || + col.getType.startsWith(serdeConstants.CHAR_TYPE_NAME)) .map(col => col.getName).toSet - filters.collect { - case op @ BinaryComparison(a: Attribute, Literal(v, _: IntegralType)) => - s"${a.name} ${op.symbol} $v" - case op @ BinaryComparison(Literal(v, _: IntegralType), a: Attribute) => - s"$v ${op.symbol} ${a.name}" - case op @ BinaryComparison(a: Attribute, Literal(v, _: StringType)) + object ExtractableLiteral { + def unapply(expr: Expression): Option[String] = expr match { + case Literal(value, _: IntegralType) => Some(value.toString) + case Literal(value, _: StringType) => Some(quoteStringLiteral(value.toString)) + case _ => None + } + } + + object ExtractableLiterals { + def unapply(exprs: Seq[Expression]): Option[Seq[String]] = { + exprs.map(ExtractableLiteral.unapply).foldLeft(Option(Seq.empty[String])) { + case (Some(accum), Some(value)) => Some(accum :+ value) + case _ => None + } + } + } + + object ExtractableValues { + private lazy val valueToLiteralString: PartialFunction[Any, String] = { + case value: Byte => value.toString + case value: Short => value.toString + case value: Int => value.toString + case value: Long => value.toString + case value: UTF8String => quoteStringLiteral(value.toString) + } + + def unapply(values: Set[Any]): Option[Seq[String]] = { + values.toSeq.foldLeft(Option(Seq.empty[String])) { + case (Some(accum), value) if valueToLiteralString.isDefinedAt(value) => + Some(accum :+ valueToLiteralString(value)) + case _ => None + } + } + } + + def convertInToOr(a: Attribute, values: Seq[String]): String = { + values.map(value => s"${a.name} = $value").mkString("(", " or ", ")") + } + + lazy val convert: PartialFunction[Expression, String] = { + case In(a: Attribute, ExtractableLiterals(values)) + if !varcharKeys.contains(a.name) && values.nonEmpty => + convertInToOr(a, values) + case InSet(a: Attribute, ExtractableValues(values)) + if !varcharKeys.contains(a.name) && values.nonEmpty => + convertInToOr(a, values) + case op @ BinaryComparison(a: Attribute, ExtractableLiteral(value)) if !varcharKeys.contains(a.name) => - s"""${a.name} ${op.symbol} "$v"""" - case op @ BinaryComparison(Literal(v, _: StringType), a: Attribute) + s"${a.name} ${op.symbol} $value" + case op @ BinaryComparison(ExtractableLiteral(value), a: Attribute) if !varcharKeys.contains(a.name) => - s""""$v" ${op.symbol} ${a.name}""" - }.mkString(" and ") + s"$value ${op.symbol} ${a.name}" + case op @ And(expr1, expr2) + if convert.isDefinedAt(expr1) || convert.isDefinedAt(expr2) => + (convert.lift(expr1) ++ convert.lift(expr2)).mkString("(", " and ", ")") + case op @ Or(expr1, expr2) + if convert.isDefinedAt(expr1) && convert.isDefinedAt(expr2) => + s"(${convert(expr1)} or ${convert(expr2)})" + } + + filters.map(convert.lift).collect { case Some(filterString) => filterString }.mkString(" and ") + } + + private def quoteStringLiteral(str: String): String = { + if (!str.contains("\"")) { + s""""$str"""" + } else if (!str.contains("'")) { + s"""'$str'""" + } else { + throw new UnsupportedOperationException( + """Partition filter cannot have both `"` and `'` characters""") + } } override def getPartitionsByFilter( @@ -346,12 +673,41 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { // Hive getPartitionsByFilter() takes a string that represents partition // predicates like "str_key=\"value\" and int_key=1 ..." val filter = convertFilters(table, predicates) + val partitions = if (filter.isEmpty) { getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]] } else { logDebug(s"Hive metastore filter is '$filter'.") - getPartitionsByFilterMethod.invoke(hive, table, filter).asInstanceOf[JArrayList[Partition]] + val tryDirectSqlConfVar = HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL + // We should get this config value from the metaStore. otherwise hit SPARK-18681. + // To be compatible with hive-0.12 and hive-0.13, In the future we can achieve this by: + // val tryDirectSql = hive.getMetaConf(tryDirectSqlConfVar.varname).toBoolean + val tryDirectSql = hive.getMSC.getConfigValue(tryDirectSqlConfVar.varname, + tryDirectSqlConfVar.defaultBoolVal.toString).toBoolean + try { + // Hive may throw an exception when calling this method in some circumstances, such as + // when filtering on a non-string partition column when the hive config key + // hive.metastore.try.direct.sql is false + getPartitionsByFilterMethod.invoke(hive, table, filter) + .asInstanceOf[JArrayList[Partition]] + } catch { + case ex: InvocationTargetException if ex.getCause.isInstanceOf[MetaException] && + !tryDirectSql => + logWarning("Caught Hive MetaException attempting to get partition metadata by " + + "filter from Hive. Falling back to fetching all partition metadata, which will " + + "degrade performance. Modifying your Hive metastore configuration to set " + + s"${tryDirectSqlConfVar.varname} to true may resolve this problem.", ex) + // HiveShim clients are expected to handle a superset of the requested partitions + getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]] + case ex: InvocationTargetException if ex.getCause.isInstanceOf[MetaException] && + tryDirectSql => + throw new RuntimeException("Caught Hive MetaException attempting to get partition " + + "metadata by filter from Hive. You can set the Spark configuration setting " + + s"${SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key} to false to work around this " + + "problem, however this will result in degraded performance. Please report a bug: " + + "https://issues.apache.org/jira/browse/SPARK", ex) + } } partitions.asScala.toSeq @@ -375,6 +731,11 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { private[client] class Shim_v0_14 extends Shim_v0_13 { + // true if this is an ACID operation + protected lazy val isAcid = JBoolean.FALSE + // true if list bucketing enabled + protected lazy val isSkewedStoreAsSubdir = JBoolean.FALSE + private lazy val loadPartitionMethod = findMethod( classOf[Hive], @@ -411,6 +772,15 @@ private[client] class Shim_v0_14 extends Shim_v0_13 { JBoolean.TYPE, JBoolean.TYPE, JBoolean.TYPE) + private lazy val dropTableMethod = + findMethod( + classOf[Hive], + "dropTable", + classOf[String], + classOf[String], + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE) private lazy val getTimeVarMethod = findMethod( classOf[HiveConf], @@ -424,12 +794,12 @@ private[client] class Shim_v0_14 extends Shim_v0_13 { tableName: String, partSpec: JMap[String, String], replace: Boolean, - holdDDLTime: Boolean, inheritTableSpecs: Boolean, - isSkewedStoreAsSubdir: Boolean): Unit = { + isSkewedStoreAsSubdir: Boolean, + isSrcLocal: Boolean): Unit = { loadPartitionMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean, - holdDDLTime: JBoolean, inheritTableSpecs: JBoolean, isSkewedStoreAsSubdir: JBoolean, - isSrcLocal(loadPath, hive.getConf()): JBoolean, JBoolean.FALSE) + holdDDLTime, inheritTableSpecs: JBoolean, isSkewedStoreAsSubdir: JBoolean, + isSrcLocal: JBoolean, isAcid) } override def loadTable( @@ -437,9 +807,9 @@ private[client] class Shim_v0_14 extends Shim_v0_13 { loadPath: Path, tableName: String, replace: Boolean, - holdDDLTime: Boolean): Unit = { - loadTableMethod.invoke(hive, loadPath, tableName, replace: JBoolean, holdDDLTime: JBoolean, - isSrcLocal(loadPath, hive.getConf()): JBoolean, JBoolean.FALSE, JBoolean.FALSE) + isSrcLocal: Boolean): Unit = { + loadTableMethod.invoke(hive, loadPath, tableName, replace: JBoolean, holdDDLTime, + isSrcLocal: JBoolean, isSkewedStoreAsSubdir, isAcid) } override def loadDynamicPartitions( @@ -449,10 +819,20 @@ private[client] class Shim_v0_14 extends Shim_v0_13 { partSpec: JMap[String, String], replace: Boolean, numDP: Int, - holdDDLTime: Boolean, listBucketingEnabled: Boolean): Unit = { loadDynamicPartitionsMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean, - numDP: JInteger, holdDDLTime: JBoolean, listBucketingEnabled: JBoolean, JBoolean.FALSE) + numDP: JInteger, holdDDLTime, listBucketingEnabled: JBoolean, isAcid) + } + + override def dropTable( + hive: Hive, + dbName: String, + tableName: String, + deleteData: Boolean, + ignoreIfNotExists: Boolean, + purge: Boolean): Unit = { + dropTableMethod.invoke(hive, dbName, tableName, deleteData: JBoolean, + ignoreIfNotExists: JBoolean, purge: JBoolean) } override def getMetastoreClientConnectRetryDelayMillis(conf: HiveConf): Long = { @@ -462,12 +842,6 @@ private[client] class Shim_v0_14 extends Shim_v0_13 { TimeUnit.MILLISECONDS).asInstanceOf[Long] } - protected def isSrcLocal(path: Path, conf: HiveConf): Boolean = { - val localFs = FileSystem.getLocal(conf) - val pathFs = FileSystem.get(path.toUri(), conf) - localFs.getUri() == pathFs.getUri() - } - } private[client] class Shim_v1_0 extends Shim_v0_14 { @@ -476,6 +850,9 @@ private[client] class Shim_v1_0 extends Shim_v0_14 { private[client] class Shim_v1_1 extends Shim_v1_0 { + // throws an exception if the index does not exist + protected lazy val throwExceptionInDropIndex = JBoolean.TRUE + private lazy val dropIndexMethod = findMethod( classOf[Hive], @@ -487,13 +864,17 @@ private[client] class Shim_v1_1 extends Shim_v1_0 { JBoolean.TYPE) override def dropIndex(hive: Hive, dbName: String, tableName: String, indexName: String): Unit = { - dropIndexMethod.invoke(hive, dbName, tableName, indexName, true: JBoolean, true: JBoolean) + dropIndexMethod.invoke(hive, dbName, tableName, indexName, throwExceptionInDropIndex, + deleteDataInDropIndex) } } private[client] class Shim_v1_2 extends Shim_v1_1 { + // txnId can be 0 unless isAcid == true + protected lazy val txnIdInLoadDynamicPartitions: JLong = 0L + private lazy val loadDynamicPartitionsMethod = findMethod( classOf[Hive], @@ -508,6 +889,19 @@ private[client] class Shim_v1_2 extends Shim_v1_1 { JBoolean.TYPE, JLong.TYPE) + private lazy val dropOptionsClass = + Utils.classForName("org.apache.hadoop.hive.metastore.PartitionDropOptions") + private lazy val dropOptionsDeleteData = dropOptionsClass.getField("deleteData") + private lazy val dropOptionsPurge = dropOptionsClass.getField("purgeData") + private lazy val dropPartitionMethod = + findMethod( + classOf[Hive], + "dropPartition", + classOf[String], + classOf[String], + classOf[JList[String]], + dropOptionsClass) + override def loadDynamicPartitions( hive: Hive, loadPath: Path, @@ -515,11 +909,204 @@ private[client] class Shim_v1_2 extends Shim_v1_1 { partSpec: JMap[String, String], replace: Boolean, numDP: Int, - holdDDLTime: Boolean, listBucketingEnabled: Boolean): Unit = { loadDynamicPartitionsMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean, - numDP: JInteger, holdDDLTime: JBoolean, listBucketingEnabled: JBoolean, JBoolean.FALSE, - 0L: JLong) + numDP: JInteger, holdDDLTime, listBucketingEnabled: JBoolean, isAcid, + txnIdInLoadDynamicPartitions) + } + + override def dropPartition( + hive: Hive, + dbName: String, + tableName: String, + part: JList[String], + deleteData: Boolean, + purge: Boolean): Unit = { + val dropOptions = dropOptionsClass.newInstance().asInstanceOf[Object] + dropOptionsDeleteData.setBoolean(dropOptions, deleteData) + dropOptionsPurge.setBoolean(dropOptions, purge) + dropPartitionMethod.invoke(hive, dbName, tableName, part, dropOptions) } } + +private[client] class Shim_v2_0 extends Shim_v1_2 { + private lazy val loadPartitionMethod = + findMethod( + classOf[Hive], + "loadPartition", + classOf[Path], + classOf[String], + classOf[JMap[String, String]], + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE) + private lazy val loadTableMethod = + findMethod( + classOf[Hive], + "loadTable", + classOf[Path], + classOf[String], + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE) + private lazy val loadDynamicPartitionsMethod = + findMethod( + classOf[Hive], + "loadDynamicPartitions", + classOf[Path], + classOf[String], + classOf[JMap[String, String]], + JBoolean.TYPE, + JInteger.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JLong.TYPE) + + override def loadPartition( + hive: Hive, + loadPath: Path, + tableName: String, + partSpec: JMap[String, String], + replace: Boolean, + inheritTableSpecs: Boolean, + isSkewedStoreAsSubdir: Boolean, + isSrcLocal: Boolean): Unit = { + loadPartitionMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean, + inheritTableSpecs: JBoolean, isSkewedStoreAsSubdir: JBoolean, + isSrcLocal: JBoolean, isAcid) + } + + override def loadTable( + hive: Hive, + loadPath: Path, + tableName: String, + replace: Boolean, + isSrcLocal: Boolean): Unit = { + loadTableMethod.invoke(hive, loadPath, tableName, replace: JBoolean, isSrcLocal: JBoolean, + isSkewedStoreAsSubdir, isAcid) + } + + override def loadDynamicPartitions( + hive: Hive, + loadPath: Path, + tableName: String, + partSpec: JMap[String, String], + replace: Boolean, + numDP: Int, + listBucketingEnabled: Boolean): Unit = { + loadDynamicPartitionsMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean, + numDP: JInteger, listBucketingEnabled: JBoolean, isAcid, txnIdInLoadDynamicPartitions) + } + +} + +private[client] class Shim_v2_1 extends Shim_v2_0 { + + // true if there is any following stats task + protected lazy val hasFollowingStatsTask = JBoolean.FALSE + // TODO: Now, always set environmentContext to null. In the future, we should avoid setting + // hive-generated stats to -1 when altering tables by using environmentContext. See Hive-12730 + protected lazy val environmentContextInAlterTable = null + + private lazy val loadPartitionMethod = + findMethod( + classOf[Hive], + "loadPartition", + classOf[Path], + classOf[String], + classOf[JMap[String, String]], + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE) + private lazy val loadTableMethod = + findMethod( + classOf[Hive], + "loadTable", + classOf[Path], + classOf[String], + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE) + private lazy val loadDynamicPartitionsMethod = + findMethod( + classOf[Hive], + "loadDynamicPartitions", + classOf[Path], + classOf[String], + classOf[JMap[String, String]], + JBoolean.TYPE, + JInteger.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JLong.TYPE, + JBoolean.TYPE, + classOf[AcidUtils.Operation]) + private lazy val alterTableMethod = + findMethod( + classOf[Hive], + "alterTable", + classOf[String], + classOf[Table], + classOf[EnvironmentContext]) + private lazy val alterPartitionsMethod = + findMethod( + classOf[Hive], + "alterPartitions", + classOf[String], + classOf[JList[Partition]], + classOf[EnvironmentContext]) + + override def loadPartition( + hive: Hive, + loadPath: Path, + tableName: String, + partSpec: JMap[String, String], + replace: Boolean, + inheritTableSpecs: Boolean, + isSkewedStoreAsSubdir: Boolean, + isSrcLocal: Boolean): Unit = { + loadPartitionMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean, + inheritTableSpecs: JBoolean, isSkewedStoreAsSubdir: JBoolean, + isSrcLocal: JBoolean, isAcid, hasFollowingStatsTask) + } + + override def loadTable( + hive: Hive, + loadPath: Path, + tableName: String, + replace: Boolean, + isSrcLocal: Boolean): Unit = { + loadTableMethod.invoke(hive, loadPath, tableName, replace: JBoolean, isSrcLocal: JBoolean, + isSkewedStoreAsSubdir, isAcid, hasFollowingStatsTask) + } + + override def loadDynamicPartitions( + hive: Hive, + loadPath: Path, + tableName: String, + partSpec: JMap[String, String], + replace: Boolean, + numDP: Int, + listBucketingEnabled: Boolean): Unit = { + loadDynamicPartitionsMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean, + numDP: JInteger, listBucketingEnabled: JBoolean, isAcid, txnIdInLoadDynamicPartitions, + hasFollowingStatsTask, AcidUtils.Operation.NOT_ACID) + } + + override def alterTable(hive: Hive, tableName: String, table: Table): Unit = { + alterTableMethod.invoke(hive, tableName, table, environmentContextInAlterTable) + } + + override def alterPartitions(hive: Hive, tableName: String, newParts: JList[Partition]): Unit = { + alterPartitionsMethod.invoke(hive, tableName, newParts, environmentContextInAlterTable) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index f99c3ed2ae98..930f0dd4b32b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -22,37 +22,66 @@ import java.lang.reflect.InvocationTargetException import java.net.{URL, URLClassLoader} import java.util -import scala.collection.mutable -import scala.language.reflectiveCalls import scala.util.Try import org.apache.commons.io.{FileUtils, IOUtils} +import org.apache.hadoop.conf.Configuration -import org.apache.spark.Logging +import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkSubmitUtils -import org.apache.spark.util.{MutableURLClassLoader, Utils} - +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util.quietly -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.internal.NonClosableMutableURLClassLoader +import org.apache.spark.util.{MutableURLClassLoader, Utils} /** Factory for `IsolatedClientLoader` with specific versions of hive. */ -private[hive] object IsolatedClientLoader { +private[hive] object IsolatedClientLoader extends Logging { /** * Creates isolated Hive client loaders by downloading the requested version from maven. */ def forVersion( - version: String, + hiveMetastoreVersion: String, + hadoopVersion: String, + sparkConf: SparkConf, + hadoopConf: Configuration, config: Map[String, String] = Map.empty, ivyPath: Option[String] = None, sharedPrefixes: Seq[String] = Seq.empty, barrierPrefixes: Seq[String] = Seq.empty): IsolatedClientLoader = synchronized { - val resolvedVersion = hiveVersion(version) - val files = resolvedVersions.getOrElseUpdate(resolvedVersion, - downloadVersion(resolvedVersion, ivyPath)) + val resolvedVersion = hiveVersion(hiveMetastoreVersion) + // We will first try to share Hadoop classes. If we cannot resolve the Hadoop artifact + // with the given version, we will use Hadoop 2.6 and then will not share Hadoop classes. + var sharesHadoopClasses = true + val files = if (resolvedVersions.contains((resolvedVersion, hadoopVersion))) { + resolvedVersions((resolvedVersion, hadoopVersion)) + } else { + val (downloadedFiles, actualHadoopVersion) = + try { + (downloadVersion(resolvedVersion, hadoopVersion, ivyPath), hadoopVersion) + } catch { + case e: RuntimeException if e.getMessage.contains("hadoop") => + // If the error message contains hadoop, it is probably because the hadoop + // version cannot be resolved. + logWarning(s"Failed to resolve Hadoop artifacts for the version $hadoopVersion. " + + s"We will change the hadoop version from $hadoopVersion to 2.6.0 and try again. " + + "Hadoop classes will not be shared between Spark and Hive metastore client. " + + "It is recommended to set jars used by Hive metastore client through " + + "spark.sql.hive.metastore.jars in the production environment.") + sharesHadoopClasses = false + (downloadVersion(resolvedVersion, "2.6.5", ivyPath), "2.6.5") + } + resolvedVersions.put((resolvedVersion, actualHadoopVersion), downloadedFiles) + resolvedVersions((resolvedVersion, actualHadoopVersion)) + } + new IsolatedClientLoader( - version = hiveVersion(version), + hiveVersion(hiveMetastoreVersion), + sparkConf, execJars = files, + hadoopConf = hadoopConf, config = config, + sharesHadoopClasses = sharesHadoopClasses, sharedPrefixes = sharedPrefixes, barrierPrefixes = barrierPrefixes) } @@ -63,21 +92,27 @@ private[hive] object IsolatedClientLoader { case "14" | "0.14" | "0.14.0" => hive.v14 case "1.0" | "1.0.0" => hive.v1_0 case "1.1" | "1.1.0" => hive.v1_1 - case "1.2" | "1.2.0" | "1.2.1" => hive.v1_2 + case "1.2" | "1.2.0" | "1.2.1" | "1.2.2" => hive.v1_2 + case "2.0" | "2.0.0" | "2.0.1" => hive.v2_0 + case "2.1" | "2.1.0" | "2.1.1" => hive.v2_1 } - private def downloadVersion(version: HiveVersion, ivyPath: Option[String]): Seq[URL] = { + private def downloadVersion( + version: HiveVersion, + hadoopVersion: String, + ivyPath: Option[String]): Seq[URL] = { val hiveArtifacts = version.extraDeps ++ Seq("hive-metastore", "hive-exec", "hive-common", "hive-serde") .map(a => s"org.apache.hive:$a:${version.fullVersion}") ++ Seq("com.google.guava:guava:14.0.1", - "org.apache.hadoop:hadoop-client:2.4.0") + s"org.apache.hadoop:hadoop-client:$hadoopVersion") val classpath = quietly { SparkSubmitUtils.resolveMavenCoordinates( hiveArtifacts.mkString(","), - Some("http://www.datanucleus.org/downloads/maven2"), - ivyPath, + SparkSubmitUtils.buildIvySettings( + Some("http://www.datanucleus.org/downloads/maven2"), + ivyPath), exclusions = version.exclusions) } val allFiles = classpath.split(",").map(new File(_)).toSet @@ -85,22 +120,26 @@ private[hive] object IsolatedClientLoader { // TODO: Remove copy logic. val tempDir = Utils.createTempDir(namePrefix = s"hive-${version}") allFiles.foreach(f => FileUtils.copyFileToDirectory(f, tempDir)) + logInfo(s"Downloaded metastore jars to ${tempDir.getCanonicalPath}") tempDir.listFiles().map(_.toURI.toURL) } - private def resolvedVersions = new scala.collection.mutable.HashMap[HiveVersion, Seq[URL]] + // A map from a given pair of HiveVersion and Hadoop version to jar files. + // It is only used by forVersion. + private val resolvedVersions = + new scala.collection.mutable.HashMap[(HiveVersion, String), Seq[URL]] } /** - * Creates a Hive `ClientInterface` using a classloader that works according to the following rules: + * Creates a [[HiveClient]] using a classloader that works according to the following rules: * - Shared classes: Java, Scala, logging, and Spark classes are delegated to `baseClassLoader` - * allowing the results of calls to the `ClientInterface` to be visible externally. + * allowing the results of calls to the [[HiveClient]] to be visible externally. * - Hive classes: new instances are loaded from `execJars`. These classes are not * accessible externally due to their custom loading. - * - ClientWrapper: a new copy is created for each instance of `IsolatedClassLoader`. + * - [[HiveClientImpl]]: a new copy is created for each instance of `IsolatedClassLoader`. * This new instance is able to see a specific version of hive without using reflection. Since * this is a unique instance, it is not visible externally other than as a generic - * `ClientInterface`, unless `isolationOn` is set to `false`. + * [[HiveClient]], unless `isolationOn` is set to `false`. * * @param version The version of hive on the classpath. used to pick specific function signatures * that are not compatible across versions. @@ -108,14 +147,18 @@ private[hive] object IsolatedClientLoader { * @param config A set of options that will be added to the HiveConf of the constructed client. * @param isolationOn When true, custom versions of barrier classes will be constructed. Must be * true unless loading the version of hive that is on Sparks classloader. + * @param sharesHadoopClasses When true, we will share Hadoop classes between Spark and * @param rootClassLoader The system root classloader. Must not know about Hive classes. * @param baseClassLoader The spark classloader that is used to load shared classes. */ private[hive] class IsolatedClientLoader( val version: HiveVersion, + val sparkConf: SparkConf, + val hadoopConf: Configuration, val execJars: Seq[URL] = Seq.empty, val config: Map[String, String] = Map.empty, val isolationOn: Boolean = true, + val sharesHadoopClasses: Boolean = true, val rootClassLoader: ClassLoader = ClassLoader.getSystemClassLoader.getParent.getParent, val baseClassLoader: ClassLoader = Thread.currentThread().getContextClassLoader, val sharedPrefixes: Seq[String] = Seq.empty, @@ -128,20 +171,24 @@ private[hive] class IsolatedClientLoader( /** All jars used by the hive specific classloader. */ protected def allJars = execJars.toArray - protected def isSharedClass(name: String): Boolean = + protected def isSharedClass(name: String): Boolean = { + val isHadoopClass = + name.startsWith("org.apache.hadoop.") && !name.startsWith("org.apache.hadoop.hive.") + name.contains("slf4j") || name.contains("log4j") || name.startsWith("org.apache.spark.") || - (name.startsWith("org.apache.hadoop.") && !name.startsWith("org.apache.hadoop.hive.")) || + (sharesHadoopClasses && isHadoopClass) || name.startsWith("scala.") || (name.startsWith("com.google") && !name.startsWith("com.google.cloud")) || name.startsWith("java.lang.") || name.startsWith("java.net") || sharedPrefixes.exists(name.startsWith) + } /** True if `name` refers to a spark class that must see specific version of Hive. */ protected def isBarrierClass(name: String): Boolean = - name.startsWith(classOf[ClientWrapper].getName) || + name.startsWith(classOf[HiveClientImpl].getName) || name.startsWith(classOf[Shim].getName) || barrierPrefixes.exists(name.startsWith) @@ -173,9 +220,15 @@ private[hive] class IsolatedClientLoader( logDebug(s"hive class: $name - ${getResource(classToPath(name))}") super.loadClass(name, resolve) } else { - // For shared classes, we delegate to baseClassLoader. + // For shared classes, we delegate to baseClassLoader, but fall back in case the + // class is not found. logDebug(s"shared class: $name") - baseClassLoader.loadClass(name) + try { + baseClassLoader.loadClass(name) + } catch { + case _: ClassNotFoundException => + super.loadClass(name, resolve) + } } } } @@ -190,15 +243,14 @@ private[hive] class IsolatedClientLoader( new NonClosableMutableURLClassLoader(isolatedClassLoader) } - private[hive] def addJar(path: String): Unit = synchronized { - val jarURL = new java.io.File(path).toURI.toURL - classLoader.addURL(jarURL) + private[hive] def addJar(path: URL): Unit = synchronized { + classLoader.addURL(path) } /** The isolated client interface to Hive. */ - private[hive] def createClient(): ClientInterface = { + private[hive] def createClient(): HiveClient = { if (!isolationOn) { - return new ClientWrapper(version, config, baseClassLoader, this) + return new HiveClientImpl(version, sparkConf, hadoopConf, config, baseClassLoader, this) } // Pre-reflective instantiation setup. logDebug("Initializing the logger to avoid disaster...") @@ -207,10 +259,10 @@ private[hive] class IsolatedClientLoader( try { classLoader - .loadClass(classOf[ClientWrapper].getName) + .loadClass(classOf[HiveClientImpl].getName) .getConstructors.head - .newInstance(version, config, classLoader, this) - .asInstanceOf[ClientInterface] + .newInstance(version, sparkConf, hadoopConf, config, classLoader, this) + .asInstanceOf[HiveClient] } catch { case e: InvocationTargetException => if (e.getCause().isInstanceOf[NoClassDefFoundError]) { @@ -218,7 +270,7 @@ private[hive] class IsolatedClientLoader( throw new ClassNotFoundException( s"$cnf when creating Hive client using classpath: ${execJars.mkString(", ")}\n" + "Please make sure that jars for your version of hive and hadoop are included in the " + - s"paths passed to ${HiveContext.HIVE_METASTORE_JARS}.") + s"paths passed to ${HiveUtils.HIVE_METASTORE_JARS.key}.", e) } else { throw e } @@ -233,14 +285,3 @@ private[hive] class IsolatedClientLoader( */ private[hive] var cachedHive: Any = null } - -/** - * URL class loader that exposes the `addURL` and `getURLs` methods in URLClassLoader. - * This class loader cannot be closed (its `close` method is a no-op). - */ -private[sql] class NonClosableMutableURLClassLoader( - parent: ClassLoader) - extends MutableURLClassLoader(Array.empty, parent) { - - override def close(): Unit = {} -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala index b1b8439efa01..c14154a3b3c2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive /** Support for interacting with different versions of the HiveMetastoreClient */ package object client { - private[client] abstract class HiveVersion( + private[hive] sealed abstract class HiveVersion( val fullVersion: String, val extraDeps: Seq[String] = Nil, val exclusions: Seq[String] = Nil) @@ -56,12 +56,22 @@ package object client { "net.hydromatic:linq4j", "net.hydromatic:quidem")) - case object v1_2 extends HiveVersion("1.2.1", + case object v1_2 extends HiveVersion("1.2.2", exclusions = Seq("eigenbase:eigenbase-properties", "org.apache.curator:*", "org.pentaho:pentaho-aggdesigner-algorithm", "net.hydromatic:linq4j", "net.hydromatic:quidem")) + + case object v2_0 extends HiveVersion("2.0.1", + exclusions = Seq("org.apache.curator:*", + "org.pentaho:pentaho-aggdesigner-algorithm")) + + case object v2_1 extends HiveVersion("2.1.1", + exclusions = Seq("org.apache.curator:*", + "org.pentaho:pentaho-aggdesigner-algorithm")) + + val allSupportedHiveVersions = Set(v12, v13, v14, v1_0, v1_1, v1_2, v2_0, v2_1) } // scalastyle:on diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala new file mode 100644 index 000000000000..65e8b4e3c725 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import scala.util.control.NonFatal + +import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession} +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} +import org.apache.spark.sql.execution.command.RunnableCommand + + +/** + * Create table and insert the query result into it. + * + * @param tableDesc the Table Describe, which may contains serde, storage handler etc. + * @param query the query whose result will be insert into the new relation + * @param mode SaveMode + */ +case class CreateHiveTableAsSelectCommand( + tableDesc: CatalogTable, + query: LogicalPlan, + mode: SaveMode) + extends RunnableCommand { + + private val tableIdentifier = tableDesc.identifier + + override def innerChildren: Seq[LogicalPlan] = Seq(query) + + override def run(sparkSession: SparkSession): Seq[Row] = { + if (sparkSession.sessionState.catalog.tableExists(tableIdentifier)) { + assert(mode != SaveMode.Overwrite, + s"Expect the table $tableIdentifier has been dropped when the save mode is Overwrite") + + if (mode == SaveMode.ErrorIfExists) { + throw new AnalysisException(s"$tableIdentifier already exists.") + } + if (mode == SaveMode.Ignore) { + // Since the table already exists and the save mode is Ignore, we will just return. + return Seq.empty + } + + sparkSession.sessionState.executePlan( + InsertIntoTable( + UnresolvedRelation(tableIdentifier), + Map(), + query, + overwrite = false, + ifPartitionNotExists = false)).toRdd + } else { + // TODO ideally, we should get the output data ready first and then + // add the relation into catalog, just in case of failure occurs while data + // processing. + assert(tableDesc.schema.isEmpty) + sparkSession.sessionState.catalog.createTable( + tableDesc.copy(schema = query.schema), ignoreIfExists = false) + + try { + sparkSession.sessionState.executePlan( + InsertIntoTable( + UnresolvedRelation(tableIdentifier), + Map(), + query, + overwrite = true, + ifPartitionNotExists = false)).toRdd + } catch { + case NonFatal(e) => + // drop the created table. + sparkSession.sessionState.catalog.dropTable(tableIdentifier, ignoreIfNotExists = true, + purge = false) + throw e + } + } + + Seq.empty[Row] + } + + override def argString: String = { + s"[Database:${tableDesc.database}}, " + + s"TableName: ${tableDesc.identifier.table}, " + + s"InsertIntoHiveTable]" + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala deleted file mode 100644 index e72a60b42e65..000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ /dev/null @@ -1,98 +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.execution - -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} -import org.apache.spark.sql.execution.RunnableCommand -import org.apache.spark.sql.hive.client.{HiveColumn, HiveTable} -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes, MetastoreRelation} -import org.apache.spark.sql.{AnalysisException, Row, SQLContext} - -/** - * Create table and insert the query result into it. - * @param tableDesc the Table Describe, which may contains serde, storage handler etc. - * @param query the query whose result will be insert into the new relation - * @param allowExisting allow continue working if it's already exists, otherwise - * raise exception - */ -private[hive] -case class CreateTableAsSelect( - tableDesc: HiveTable, - query: LogicalPlan, - allowExisting: Boolean) - extends RunnableCommand { - - val tableIdentifier = TableIdentifier(tableDesc.name, Some(tableDesc.database)) - - override def children: Seq[LogicalPlan] = Seq(query) - - override def run(sqlContext: SQLContext): Seq[Row] = { - val hiveContext = sqlContext.asInstanceOf[HiveContext] - lazy val metastoreRelation: MetastoreRelation = { - import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe - import org.apache.hadoop.io.Text - import org.apache.hadoop.mapred.TextInputFormat - - val withFormat = - tableDesc.copy( - inputFormat = - tableDesc.inputFormat.orElse(Some(classOf[TextInputFormat].getName)), - outputFormat = - tableDesc.outputFormat - .orElse(Some(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]].getName)), - serde = tableDesc.serde.orElse(Some(classOf[LazySimpleSerDe].getName()))) - - val withSchema = if (withFormat.schema.isEmpty) { - // Hive doesn't support specifying the column list for target table in CTAS - // However we don't think SparkSQL should follow that. - tableDesc.copy(schema = - query.output.map(c => - HiveColumn(c.name, HiveMetastoreTypes.toMetastoreType(c.dataType), null))) - } else { - withFormat - } - - hiveContext.catalog.client.createTable(withSchema) - - // Get the Metastore Relation - hiveContext.catalog.lookupRelation(tableIdentifier, None) match { - case r: MetastoreRelation => r - } - } - // TODO ideally, we should get the output data ready first and then - // add the relation into catalog, just in case of failure occurs while data - // processing. - if (hiveContext.catalog.tableExists(tableIdentifier)) { - if (allowExisting) { - // table already exists, will do nothing, to keep consistent with Hive - } else { - throw new AnalysisException(s"$tableIdentifier already exists.") - } - } else { - hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true, false)).toRdd - } - - Seq.empty[Row] - } - - override def argString: String = { - s"[Database:${tableDesc.database}}, TableName: ${tableDesc.name}, InsertIntoHiveTable]" - } -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateViewAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateViewAsSelect.scala deleted file mode 100644 index 2c81115ee4fe..000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateViewAsSelect.scala +++ /dev/null @@ -1,98 +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.execution - -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.hive.{HiveMetastoreTypes, HiveContext} -import org.apache.spark.sql.{AnalysisException, Row, SQLContext} -import org.apache.spark.sql.execution.RunnableCommand -import org.apache.spark.sql.hive.client.{HiveColumn, HiveTable} - -/** - * Create Hive view on non-hive-compatible tables by specifying schema ourselves instead of - * depending on Hive meta-store. - */ -// TODO: Note that this class can NOT canonicalize the view SQL string entirely, which is different -// from Hive and may not work for some cases like create view on self join. -private[hive] case class CreateViewAsSelect( - tableDesc: HiveTable, - childSchema: Seq[Attribute], - allowExisting: Boolean, - orReplace: Boolean) extends RunnableCommand { - - assert(tableDesc.schema == Nil || tableDesc.schema.length == childSchema.length) - assert(tableDesc.viewText.isDefined) - - val tableIdentifier = TableIdentifier(tableDesc.name, Some(tableDesc.database)) - - override def run(sqlContext: SQLContext): Seq[Row] = { - val hiveContext = sqlContext.asInstanceOf[HiveContext] - - if (hiveContext.catalog.tableExists(tableIdentifier)) { - if (allowExisting) { - // view already exists, will do nothing, to keep consistent with Hive - } else if (orReplace) { - hiveContext.catalog.client.alertView(prepareTable()) - } else { - throw new AnalysisException(s"View $tableIdentifier already exists. " + - "If you want to update the view definition, please use ALTER VIEW AS or " + - "CREATE OR REPLACE VIEW AS") - } - } else { - hiveContext.catalog.client.createView(prepareTable()) - } - - Seq.empty[Row] - } - - private def prepareTable(): HiveTable = { - // setup column types according to the schema of child. - val schema = if (tableDesc.schema == Nil) { - childSchema.map { attr => - HiveColumn(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), null) - } - } else { - childSchema.zip(tableDesc.schema).map { case (attr, col) => - HiveColumn(col.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), col.comment) - } - } - - val columnNames = childSchema.map(f => verbose(f.name)) - - // When user specified column names for view, we should create a project to do the renaming. - // When no column name specified, we still need to create a project to declare the columns - // we need, to make us more robust to top level `*`s. - val projectList = if (tableDesc.schema == Nil) { - columnNames.mkString(", ") - } else { - columnNames.zip(tableDesc.schema.map(f => verbose(f.name))).map { - case (name, alias) => s"$name AS $alias" - }.mkString(", ") - } - - val viewName = verbose(tableDesc.name) - - val expandedText = s"SELECT $projectList FROM (${tableDesc.viewText.get}) $viewName" - - tableDesc.copy(schema = schema, viewText = Some(expandedText)) - } - - // escape backtick with double-backtick in column name and wrap it with backtick. - private def verbose(name: String) = s"`${name.replaceAll("`", "``")}`" -} 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 deleted file mode 100644 index 441b6b6033e1..000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ /dev/null @@ -1,63 +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.execution - -import scala.collection.JavaConverters._ - -import org.apache.hadoop.hive.metastore.api.FieldSchema - -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.execution.RunnableCommand -import org.apache.spark.sql.hive.MetastoreRelation -import org.apache.spark.sql.{Row, SQLContext} - -/** - * Implementation for "describe [extended] table". - */ -private[hive] -case class DescribeHiveTableCommand( - table: MetastoreRelation, - override val output: Seq[Attribute], - isExtended: Boolean) extends RunnableCommand { - - override def run(sqlContext: SQLContext): Seq[Row] = { - // Trying to mimic the format of Hive's output. But not exactly the same. - var results: Seq[(String, String, String)] = Nil - - val columns: Seq[FieldSchema] = table.hiveQlTable.getCols.asScala - val partitionColumns: Seq[FieldSchema] = table.hiveQlTable.getPartCols.asScala - results ++= columns.map(field => (field.getName, field.getType, field.getComment)) - if (partitionColumns.nonEmpty) { - val partColumnInfo = - partitionColumns.map(field => (field.getName, field.getType, field.getComment)) - results ++= - partColumnInfo ++ - Seq(("# Partition Information", "", "")) ++ - Seq((s"# ${output(0).name}", output(1).name, output(2).name)) ++ - partColumnInfo - } - - if (isExtended) { - results ++= Seq(("Detailed Table Information", table.hiveQlTable.getTTable.toString, "")) - } - - results.map { case (name, dataType, comment) => - Row(name, dataType, comment) - } - } -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala new file mode 100644 index 000000000000..ac735e8b383f --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -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.sql.hive.execution + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.hive.ql.exec.Utilities +import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} +import org.apache.hadoop.hive.serde2.Serializer +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorUtils, StructObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred.{JobConf, Reporter} +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, OutputWriterFactory} +import org.apache.spark.sql.hive.{HiveInspectors, HiveTableUtil} +import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} +import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableJobConf + +/** + * `FileFormat` for writing Hive tables. + * + * TODO: implement the read logic. + */ +class HiveFileFormat(fileSinkConf: FileSinkDesc) + extends FileFormat with DataSourceRegister with Logging { + + def this() = this(null) + + override def shortName(): String = "hive" + + override def inferSchema( + sparkSession: SparkSession, + options: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = { + throw new UnsupportedOperationException(s"inferSchema is not supported for hive data source.") + } + + override def prepareWrite( + sparkSession: SparkSession, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + val conf = job.getConfiguration + val tableDesc = fileSinkConf.getTableInfo + conf.set("mapred.output.format.class", tableDesc.getOutputFileFormatClassName) + + // When speculation is on and output committer class name contains "Direct", we should warn + // users that they may loss data if they are using a direct output committer. + val speculationEnabled = sparkSession.sparkContext.conf.getBoolean("spark.speculation", false) + val outputCommitterClass = conf.get("mapred.output.committer.class", "") + if (speculationEnabled && outputCommitterClass.contains("Direct")) { + val warningMessage = + s"$outputCommitterClass may be an output committer that writes data directly to " + + "the final location. Because speculation is enabled, this output committer may " + + "cause data loss (see the case in SPARK-10063). If possible, please use an output " + + "committer that does not have this behavior (e.g. FileOutputCommitter)." + logWarning(warningMessage) + } + + // Add table properties from storage handler to hadoopConf, so any custom storage + // handler settings can be set to hadoopConf + HiveTableUtil.configureJobPropertiesForStorageHandler(tableDesc, conf, false) + Utilities.copyTableJobPropertiesToConf(tableDesc, conf) + + // Avoid referencing the outer object. + val fileSinkConfSer = fileSinkConf + new OutputWriterFactory { + private val jobConf = new SerializableJobConf(new JobConf(conf)) + @transient private lazy val outputFormat = + jobConf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef, Writable]] + + override def getFileExtension(context: TaskAttemptContext): String = { + Utilities.getFileExtension(jobConf.value, fileSinkConfSer.getCompressed, outputFormat) + } + + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new HiveOutputWriter(path, fileSinkConfSer, jobConf.value, dataSchema) + } + } + } +} + +class HiveOutputWriter( + path: String, + fileSinkConf: FileSinkDesc, + jobConf: JobConf, + dataSchema: StructType) extends OutputWriter with HiveInspectors { + + private def tableDesc = fileSinkConf.getTableInfo + + private val serializer = { + val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] + serializer.initialize(null, tableDesc.getProperties) + serializer + } + + private val hiveWriter = HiveFileFormatUtils.getHiveRecordWriter( + jobConf, + tableDesc, + serializer.getSerializedClass, + fileSinkConf, + new Path(path), + Reporter.NULL) + + private val standardOI = ObjectInspectorUtils + .getStandardObjectInspector( + tableDesc.getDeserializer.getObjectInspector, + ObjectInspectorCopyOption.JAVA) + .asInstanceOf[StructObjectInspector] + + private val fieldOIs = + standardOI.getAllStructFieldRefs.asScala.map(_.getFieldObjectInspector).toArray + private val dataTypes = dataSchema.map(_.dataType).toArray + private val wrappers = fieldOIs.zip(dataTypes).map { case (f, dt) => wrapperFor(f, dt) } + private val outputData = new Array[Any](fieldOIs.length) + + override def write(row: InternalRow): Unit = { + var i = 0 + while (i < fieldOIs.length) { + outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, dataTypes(i))) + i += 1 + } + hiveWriter.write(serializer.serialize(outputData, standardOI)) + } + + override def close(): Unit = { + // Seems the boolean value passed into close does not matter. + hiveWriter.close(false) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala deleted file mode 100644 index 41b645b2c9c9..000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala +++ /dev/null @@ -1,34 +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.execution - -import org.apache.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.execution.RunnableCommand -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.types.StringType -import org.apache.spark.sql.{Row, SQLContext} - -private[hive] -case class HiveNativeCommand(sql: String) extends RunnableCommand { - - override def output: Seq[AttributeReference] = - Seq(AttributeReference("result", StringType, nullable = false)()) - - override def run(sqlContext: SQLContext): Seq[Row] = - sqlContext.asInstanceOf[HiveContext].runSqlHive(sql).map(Row(_)) -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala new file mode 100644 index 000000000000..5c515515b9b9 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import java.util.Locale + +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap + +/** + * Options for the Hive data source. Note that rule `DetermineHiveSerde` will extract Hive + * serde/format information from these options. + */ +class HiveOptions(@transient private val parameters: CaseInsensitiveMap[String]) + extends Serializable { + import HiveOptions._ + + def this(parameters: Map[String, String]) = this(CaseInsensitiveMap(parameters)) + + val fileFormat = parameters.get(FILE_FORMAT).map(_.toLowerCase(Locale.ROOT)) + val inputFormat = parameters.get(INPUT_FORMAT) + val outputFormat = parameters.get(OUTPUT_FORMAT) + + if (inputFormat.isDefined != outputFormat.isDefined) { + throw new IllegalArgumentException("Cannot specify only inputFormat or outputFormat, you " + + "have to specify both of them.") + } + + def hasInputOutputFormat: Boolean = inputFormat.isDefined + + if (fileFormat.isDefined && inputFormat.isDefined) { + throw new IllegalArgumentException("Cannot specify fileFormat and inputFormat/outputFormat " + + "together for Hive data source.") + } + + val serde = parameters.get(SERDE) + + if (fileFormat.isDefined && serde.isDefined) { + if (!Set("sequencefile", "textfile", "rcfile").contains(fileFormat.get)) { + throw new IllegalArgumentException( + s"fileFormat '${fileFormat.get}' already specifies a serde.") + } + } + + val containsDelimiters = delimiterOptions.keys.exists(parameters.contains) + + if (containsDelimiters) { + if (serde.isDefined) { + throw new IllegalArgumentException("Cannot specify delimiters with a custom serde.") + } + if (fileFormat.isEmpty) { + throw new IllegalArgumentException("Cannot specify delimiters without fileFormat.") + } + if (fileFormat.get != "textfile") { + throw new IllegalArgumentException("Cannot specify delimiters as they are only compatible " + + s"with fileFormat 'textfile', not ${fileFormat.get}.") + } + } + + for (lineDelim <- parameters.get("lineDelim") if lineDelim != "\n") { + throw new IllegalArgumentException("Hive data source only support newline '\\n' as " + + s"line delimiter, but given: $lineDelim.") + } + + def serdeProperties: Map[String, String] = parameters.filterKeys { + k => !lowerCasedOptionNames.contains(k.toLowerCase(Locale.ROOT)) + }.map { case (k, v) => delimiterOptions.getOrElse(k, k) -> v } +} + +object HiveOptions { + private val lowerCasedOptionNames = collection.mutable.Set[String]() + + private def newOption(name: String): String = { + lowerCasedOptionNames += name.toLowerCase(Locale.ROOT) + name + } + + val FILE_FORMAT = newOption("fileFormat") + val INPUT_FORMAT = newOption("inputFormat") + val OUTPUT_FORMAT = newOption("outputFormat") + val SERDE = newOption("serde") + + // A map from the public delimiter option keys to the underlying Hive serde property keys. + val delimiterOptions = Map( + "fieldDelim" -> "field.delim", + "escapeDelim" -> "escape.delim", + // The following typo is inherited from Hive... + "collectionDelim" -> "colelction.delim", + "mapkeyDelim" -> "mapkey.delim", + "lineDelim" -> "line.delim").map { case (k, v) => k.toLowerCase(Locale.ROOT) -> v } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala deleted file mode 100644 index 806d2b9b0b7d..000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ /dev/null @@ -1,140 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.execution - -import scala.collection.JavaConverters._ - -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition} -import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.hadoop.hive.serde2.objectinspector._ -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.hive._ -import org.apache.spark.sql.types.{BooleanType, DataType} - -/** - * The Hive table scan operator. Column and partition pruning are both handled. - * - * @param requestedAttributes Attributes to be fetched from the Hive table. - * @param relation The Hive table be be scanned. - * @param partitionPruningPred An optional partition pruning predicate for partitioned table. - */ -private[hive] -case class HiveTableScan( - requestedAttributes: Seq[Attribute], - relation: MetastoreRelation, - partitionPruningPred: Seq[Expression])( - @transient val context: HiveContext) - extends LeafNode { - - require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned, - "Partition pruning predicates only supported for partitioned tables.") - - // Retrieve the original attributes based on expression ID so that capitalization matches. - val attributes = requestedAttributes.map(relation.attributeMap) - - // Bind all partition key attribute references in the partition pruning predicate for later - // evaluation. - private[this] val boundPruningPred = partitionPruningPred.reduceLeftOption(And).map { pred => - require( - pred.dataType == BooleanType, - s"Data type of predicate $pred must be BooleanType rather than ${pred.dataType}.") - - BindReferences.bindReference(pred, relation.partitionKeys) - } - - // Create a local copy of hiveconf,so that scan specific modifications should not impact - // other queries - @transient - private[this] val hiveExtraConf = new HiveConf(context.hiveconf) - - // append columns ids and names before broadcast - addColumnMetadataToConf(hiveExtraConf) - - @transient - private[this] val hadoopReader = - new HadoopTableReader(attributes, relation, context, hiveExtraConf) - - private[this] def castFromString(value: String, dataType: DataType) = { - Cast(Literal(value), dataType).eval(null) - } - - private def addColumnMetadataToConf(hiveConf: HiveConf) { - // Specifies needed column IDs for those non-partitioning columns. - val neededColumnIDs = attributes.flatMap(relation.columnOrdinals.get).map(o => o: Integer) - - HiveShim.appendReadColumns(hiveConf, neededColumnIDs, attributes.map(_.name)) - - val tableDesc = relation.tableDesc - val deserializer = tableDesc.getDeserializerClass.newInstance - deserializer.initialize(hiveConf, tableDesc.getProperties) - - // Specifies types and object inspectors of columns to be scanned. - val structOI = ObjectInspectorUtils - .getStandardObjectInspector( - deserializer.getObjectInspector, - ObjectInspectorCopyOption.JAVA) - .asInstanceOf[StructObjectInspector] - - val columnTypeNames = structOI - .getAllStructFieldRefs.asScala - .map(_.getFieldObjectInspector) - .map(TypeInfoUtils.getTypeInfoFromObjectInspector(_).getTypeName) - .mkString(",") - - hiveConf.set(serdeConstants.LIST_COLUMN_TYPES, columnTypeNames) - hiveConf.set(serdeConstants.LIST_COLUMNS, relation.attributes.map(_.name).mkString(",")) - } - - /** - * Prunes partitions not involve the query plan. - * - * @param partitions All partitions of the relation. - * @return Partitions that are involved in the query plan. - */ - private[hive] def prunePartitions(partitions: Seq[HivePartition]) = { - boundPruningPred match { - case None => partitions - case Some(shouldKeep) => partitions.filter { part => - val dataTypes = relation.partitionKeys.map(_.dataType) - val castedValues = part.getValues.asScala.zip(dataTypes) - .map { case (value, dataType) => castFromString(value, dataType) } - - // Only partitioned values are needed here, since the predicate has already been bound to - // partition key attribute references. - val row = InternalRow.fromSeq(castedValues) - shouldKeep.eval(row).asInstanceOf[Boolean] - } - } - } - - protected override def doExecute(): RDD[InternalRow] = if (!relation.hiveQlTable.isPartitioned) { - hadoopReader.makeRDDForTable(relation.hiveQlTable) - } else { - hadoopReader.makeRDDForPartitionedTable( - prunePartitions(relation.getHiveQlPartitions(partitionPruningPred))) - } - - override def output: Seq[Attribute] = attributes -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala new file mode 100644 index 000000000000..48d0b4a63e54 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition} +import org.apache.hadoop.hive.ql.plan.TableDesc +import org.apache.hadoop.hive.serde.serdeConstants +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.CastSupport +import org.apache.spark.sql.catalyst.catalog.HiveTableRelation +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.hive._ +import org.apache.spark.sql.hive.client.HiveClientImpl +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{BooleanType, DataType} +import org.apache.spark.util.Utils + +/** + * The Hive table scan operator. Column and partition pruning are both handled. + * + * @param requestedAttributes Attributes to be fetched from the Hive table. + * @param relation The Hive table be scanned. + * @param partitionPruningPred An optional partition pruning predicate for partitioned table. + */ +private[hive] +case class HiveTableScanExec( + requestedAttributes: Seq[Attribute], + relation: HiveTableRelation, + partitionPruningPred: Seq[Expression])( + @transient private val sparkSession: SparkSession) + extends LeafExecNode with CastSupport { + + require(partitionPruningPred.isEmpty || relation.isPartitioned, + "Partition pruning predicates only supported for partitioned tables.") + + override def conf: SQLConf = sparkSession.sessionState.conf + + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + + override def producedAttributes: AttributeSet = outputSet ++ + AttributeSet(partitionPruningPred.flatMap(_.references)) + + private val originalAttributes = AttributeMap(relation.output.map(a => a -> a)) + + override val output: Seq[Attribute] = { + // Retrieve the original attributes based on expression ID so that capitalization matches. + requestedAttributes.map(originalAttributes) + } + + // Bind all partition key attribute references in the partition pruning predicate for later + // evaluation. + private lazy val boundPruningPred = partitionPruningPred.reduceLeftOption(And).map { pred => + require( + pred.dataType == BooleanType, + s"Data type of predicate $pred must be BooleanType rather than ${pred.dataType}.") + + BindReferences.bindReference(pred, relation.partitionCols) + } + + @transient private lazy val hiveQlTable = HiveClientImpl.toHiveTable(relation.tableMeta) + @transient private lazy val tableDesc = new TableDesc( + hiveQlTable.getInputFormatClass, + hiveQlTable.getOutputFormatClass, + hiveQlTable.getMetadata) + + // Create a local copy of hadoopConf,so that scan specific modifications should not impact + // other queries + @transient private lazy val hadoopConf = { + val c = sparkSession.sessionState.newHadoopConf() + // append columns ids and names before broadcast + addColumnMetadataToConf(c) + c + } + + @transient private lazy val hadoopReader = new HadoopTableReader( + output, + relation.partitionCols, + tableDesc, + sparkSession, + hadoopConf) + + private def castFromString(value: String, dataType: DataType) = { + cast(Literal(value), dataType).eval(null) + } + + private def addColumnMetadataToConf(hiveConf: Configuration): Unit = { + // Specifies needed column IDs for those non-partitioning columns. + val columnOrdinals = AttributeMap(relation.dataCols.zipWithIndex) + val neededColumnIDs = output.flatMap(columnOrdinals.get).map(o => o: Integer) + + HiveShim.appendReadColumns(hiveConf, neededColumnIDs, output.map(_.name)) + + val deserializer = tableDesc.getDeserializerClass.newInstance + deserializer.initialize(hiveConf, tableDesc.getProperties) + + // Specifies types and object inspectors of columns to be scanned. + val structOI = ObjectInspectorUtils + .getStandardObjectInspector( + deserializer.getObjectInspector, + ObjectInspectorCopyOption.JAVA) + .asInstanceOf[StructObjectInspector] + + val columnTypeNames = structOI + .getAllStructFieldRefs.asScala + .map(_.getFieldObjectInspector) + .map(TypeInfoUtils.getTypeInfoFromObjectInspector(_).getTypeName) + .mkString(",") + + hiveConf.set(serdeConstants.LIST_COLUMN_TYPES, columnTypeNames) + hiveConf.set(serdeConstants.LIST_COLUMNS, relation.dataCols.map(_.name).mkString(",")) + } + + /** + * Prunes partitions not involve the query plan. + * + * @param partitions All partitions of the relation. + * @return Partitions that are involved in the query plan. + */ + private[hive] def prunePartitions(partitions: Seq[HivePartition]) = { + boundPruningPred match { + case None => partitions + case Some(shouldKeep) => partitions.filter { part => + val dataTypes = relation.partitionCols.map(_.dataType) + val castedValues = part.getValues.asScala.zip(dataTypes) + .map { case (value, dataType) => castFromString(value, dataType) } + + // Only partitioned values are needed here, since the predicate has already been bound to + // partition key attribute references. + val row = InternalRow.fromSeq(castedValues) + shouldKeep.eval(row).asInstanceOf[Boolean] + } + } + } + + // exposed for tests + @transient lazy val rawPartitions = { + val prunedPartitions = if (sparkSession.sessionState.conf.metastorePartitionPruning) { + // Retrieve the original attributes based on expression ID so that capitalization matches. + val normalizedFilters = partitionPruningPred.map(_.transform { + case a: AttributeReference => originalAttributes(a) + }) + sparkSession.sharedState.externalCatalog.listPartitionsByFilter( + relation.tableMeta.database, + relation.tableMeta.identifier.table, + normalizedFilters, + sparkSession.sessionState.conf.sessionLocalTimeZone) + } else { + sparkSession.sharedState.externalCatalog.listPartitions( + relation.tableMeta.database, + relation.tableMeta.identifier.table) + } + prunedPartitions.map(HiveClientImpl.toHivePartition(_, hiveQlTable)) + } + + protected override def doExecute(): RDD[InternalRow] = { + // Using dummyCallSite, as getCallSite can turn out to be expensive with + // with multiple partitions. + val rdd = if (!relation.isPartitioned) { + Utils.withDummyCallSite(sqlContext.sparkContext) { + hadoopReader.makeRDDForTable(hiveQlTable) + } + } else { + Utils.withDummyCallSite(sqlContext.sparkContext) { + hadoopReader.makeRDDForPartitionedTable(prunePartitions(rawPartitions)) + } + } + val numOutputRows = longMetric("numOutputRows") + // Avoid to serialize MetastoreRelation because schema is lazy. (see SPARK-15649) + val outputSchema = schema + rdd.mapPartitionsWithIndexInternal { (index, iter) => + val proj = UnsafeProjection.create(outputSchema) + proj.initialize(index) + iter.map { r => + numOutputRows += 1 + proj(r) + } + } + } + + override lazy val canonicalized: HiveTableScanExec = { + val input: AttributeSeq = relation.output + HiveTableScanExec( + requestedAttributes.map(QueryPlan.normalizeExprId(_, input)), + relation.canonicalized, + QueryPlan.normalizePredicates(partitionPruningPred, input))(sparkSession) + } + + override def otherCopyArgs: Seq[AnyRef] = Seq(sparkSession) +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala new file mode 100644 index 000000000000..918c8be00d69 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import scala.language.existentials + +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.hive.common.FileUtils +import org.apache.hadoop.hive.ql.plan.TableDesc +import org.apache.hadoop.hive.serde.serdeConstants +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.apache.hadoop.mapred._ + +import org.apache.spark.SparkException +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.hive.client.HiveClientImpl + +/** + * Command for writing the results of `query` to file system. + * + * The syntax of using this command in SQL is: + * {{{ + * INSERT OVERWRITE [LOCAL] DIRECTORY + * path + * [ROW FORMAT row_format] + * [STORED AS file_format] + * SELECT ... + * }}} + * + * @param isLocal whether the path specified in `storage` is a local directory + * @param storage storage format used to describe how the query result is stored. + * @param query the logical plan representing data to write to + * @param overwrite whether overwrites existing directory + */ +case class InsertIntoHiveDirCommand( + isLocal: Boolean, + storage: CatalogStorageFormat, + query: LogicalPlan, + overwrite: Boolean) extends SaveAsHiveFile { + + override def children: Seq[LogicalPlan] = query :: Nil + + override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { + assert(children.length == 1) + assert(storage.locationUri.nonEmpty) + + val hiveTable = HiveClientImpl.toHiveTable(CatalogTable( + identifier = TableIdentifier(storage.locationUri.get.toString, Some("default")), + tableType = org.apache.spark.sql.catalyst.catalog.CatalogTableType.VIEW, + storage = storage, + schema = query.schema + )) + hiveTable.getMetadata.put(serdeConstants.SERIALIZATION_LIB, + storage.serde.getOrElse(classOf[LazySimpleSerDe].getName)) + + val tableDesc = new TableDesc( + hiveTable.getInputFormatClass, + hiveTable.getOutputFormatClass, + hiveTable.getMetadata + ) + + val hadoopConf = sparkSession.sessionState.newHadoopConf() + val jobConf = new JobConf(hadoopConf) + + val targetPath = new Path(storage.locationUri.get) + val writeToPath = + if (isLocal) { + val localFileSystem = FileSystem.getLocal(jobConf) + localFileSystem.makeQualified(targetPath) + } else { + val qualifiedPath = FileUtils.makeQualified(targetPath, hadoopConf) + val dfs = qualifiedPath.getFileSystem(jobConf) + if (!dfs.exists(qualifiedPath)) { + dfs.mkdirs(qualifiedPath.getParent) + } + qualifiedPath + } + + val tmpPath = getExternalTmpPath(sparkSession, hadoopConf, writeToPath) + val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc( + tmpPath.toString, tableDesc, false) + + try { + saveAsHiveFile( + sparkSession = sparkSession, + plan = children.head, + hadoopConf = hadoopConf, + fileSinkConf = fileSinkConf, + outputLocation = tmpPath.toString) + + val fs = writeToPath.getFileSystem(hadoopConf) + if (overwrite && fs.exists(writeToPath)) { + fs.listStatus(writeToPath).foreach { existFile => + if (Option(existFile.getPath) != createdTempDir) fs.delete(existFile.getPath, true) + } + } + + fs.listStatus(tmpPath).foreach { + tmpFile => fs.rename(tmpFile.getPath, writeToPath) + } + } catch { + case e: Throwable => + throw new SparkException( + "Failed inserting overwrite directory " + storage.locationUri.get, e) + } finally { + deleteExternalTmpPath(hadoopConf) + } + + Seq.empty[Row] + } +} + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index f936cf565b2b..e5b59ed7a1a6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -17,132 +17,89 @@ package org.apache.spark.sql.hive.execution -import java.util +import scala.util.control.NonFatal -import scala.collection.JavaConverters._ - -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.ql.ErrorMsg import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.hadoop.hive.ql.{Context, ErrorMsg} -import org.apache.hadoop.hive.serde2.Serializer -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption -import org.apache.hadoop.hive.serde2.objectinspector._ -import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.SparkException +import org.apache.spark.sql.{AnalysisException, Row, SparkSession} +import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.execution.{UnaryNode, SparkPlan} -import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.command.CommandUtils import org.apache.spark.sql.hive._ -import org.apache.spark.sql.types.DataType -import org.apache.spark.{SparkException, TaskContext} -import org.apache.spark.util.SerializableJobConf +import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} +import org.apache.spark.sql.hive.client.HiveClientImpl -private[hive] + +/** + * Command for writing data out to a Hive table. + * + * This class is mostly a mess, for legacy reasons (since it evolved in organic ways and had to + * follow Hive's internal implementations closely, which itself was a mess too). Please don't + * blame Reynold for this! He was just moving code around! + * + * In the future we should converge the write path for Hive with the normal data source write path, + * as defined in `org.apache.spark.sql.execution.datasources.FileFormatWriter`. + * + * @param table the metadata of the table. + * @param partition a map from the partition key to the partition value (optional). If the partition + * value is optional, dynamic partition insert will be performed. + * As an example, `INSERT INTO tbl PARTITION (a=1, b=2) AS ...` would have + * + * {{{ + * Map('a' -> Some('1'), 'b' -> Some('2')) + * }}} + * + * and `INSERT INTO tbl PARTITION (a=1, b) AS ...` + * would have + * + * {{{ + * Map('a' -> Some('1'), 'b' -> None) + * }}}. + * @param query the logical plan representing data to write to. + * @param overwrite overwrite existing table or partitions. + * @param ifPartitionNotExists If true, only write if the partition does not exist. + * Only valid for static partitions. + */ case class InsertIntoHiveTable( - table: MetastoreRelation, + table: CatalogTable, partition: Map[String, Option[String]], - child: SparkPlan, + query: LogicalPlan, overwrite: Boolean, - ifNotExists: Boolean) extends UnaryNode with HiveInspectors { - - @transient val sc: HiveContext = sqlContext.asInstanceOf[HiveContext] - @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass - @transient private lazy val hiveContext = new Context(sc.hiveconf) - @transient private lazy val catalog = sc.catalog - - private def newSerializer(tableDesc: TableDesc): Serializer = { - val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] - serializer.initialize(null, tableDesc.getProperties) - serializer - } - - def output: Seq[Attribute] = Seq.empty - - private def saveAsHiveFile( - rdd: RDD[InternalRow], - valueClass: Class[_], - fileSinkConf: FileSinkDesc, - conf: SerializableJobConf, - writerContainer: SparkHiveWriterContainer): Unit = { - assert(valueClass != null, "Output value class not set") - conf.value.setOutputValueClass(valueClass) - - val outputFileFormatClassName = fileSinkConf.getTableInfo.getOutputFileFormatClassName - assert(outputFileFormatClassName != null, "Output format class not set") - conf.value.set("mapred.output.format.class", outputFileFormatClassName) - - FileOutputFormat.setOutputPath( - conf.value, - SparkHiveWriterContainer.createPathFromString(fileSinkConf.getDirName, conf.value)) - log.debug("Saving as hadoop file of type " + valueClass.getSimpleName) - - writerContainer.driverSideSetup() - sc.sparkContext.runJob(rdd, writeToFile _) - writerContainer.commitJob() - - // Note that this function is executed on executor side - def writeToFile(context: TaskContext, iterator: Iterator[InternalRow]): Unit = { - val serializer = newSerializer(fileSinkConf.getTableInfo) - val standardOI = ObjectInspectorUtils - .getStandardObjectInspector( - fileSinkConf.getTableInfo.getDeserializer.getObjectInspector, - ObjectInspectorCopyOption.JAVA) - .asInstanceOf[StructObjectInspector] - - val fieldOIs = standardOI.getAllStructFieldRefs.asScala - .map(_.getFieldObjectInspector).toArray - val dataTypes: Array[DataType] = child.output.map(_.dataType).toArray - val wrappers = fieldOIs.zip(dataTypes).map { case (f, dt) => wrapperFor(f, dt)} - val outputData = new Array[Any](fieldOIs.length) - - writerContainer.executorSideSetup(context.stageId, context.partitionId, context.attemptNumber) - - iterator.foreach { row => - var i = 0 - while (i < fieldOIs.length) { - outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, dataTypes(i))) - i += 1 - } - - writerContainer - .getLocalFileWriter(row, table.schema) - .write(serializer.serialize(outputData, standardOI)) - } + ifPartitionNotExists: Boolean) extends SaveAsHiveFile { - writerContainer.close() - } - } + override def children: Seq[LogicalPlan] = query :: Nil /** * Inserts all the rows in the table into Hive. Row objects are properly serialized with the * `org.apache.hadoop.hive.serde2.SerDe` and the * `org.apache.hadoop.mapred.OutputFormat` provided by the table definition. - * - * Note: this is run once and then kept to avoid double insertions. */ - protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { + override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { + assert(children.length == 1) + + val externalCatalog = sparkSession.sharedState.externalCatalog + val hadoopConf = sparkSession.sessionState.newHadoopConf() + + val hiveQlTable = HiveClientImpl.toHiveTable(table) // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer // instances within the closure, since Serializer is not serializable while TableDesc is. - val tableDesc = table.tableDesc - val tableLocation = table.hiveQlTable.getDataLocation - val tmpLocation = hiveContext.getExternalTmpPath(tableLocation) + val tableDesc = new TableDesc( + hiveQlTable.getInputFormatClass, + // The class of table should be org.apache.hadoop.hive.ql.metadata.Table because + // getOutputFormatClass will use HiveFileFormatUtils.getOutputFormatSubstitute to + // substitute some output formats, e.g. substituting SequenceFileOutputFormat to + // HiveSequenceFileOutputFormat. + hiveQlTable.getOutputFormatClass, + hiveQlTable.getMetadata + ) + val tableLocation = hiveQlTable.getDataLocation + val tmpLocation = getExternalTmpPath(sparkSession, hadoopConf, tableLocation) val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) - val isCompressed = sc.hiveconf.getBoolean( - ConfVars.COMPRESSRESULT.varname, ConfVars.COMPRESSRESULT.defaultBoolVal) - - if (isCompressed) { - // Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec", - // and "mapred.output.compression.type" have no impact on ORC because it uses table properties - // to store compression information. - sc.hiveconf.set("mapred.output.compress", "true") - fileSinkConf.setCompressed(true) - fileSinkConf.setCompressCodec(sc.hiveconf.get("mapred.output.compression.codec")) - fileSinkConf.setCompressType(sc.hiveconf.get("mapred.output.compression.type")) - } val numDynamicPartitions = partition.values.count(_.isEmpty) val numStaticPartitions = partition.values.count(_.nonEmpty) @@ -153,126 +110,150 @@ case class InsertIntoHiveTable( // All partition column names in the format of "//..." val partitionColumns = fileSinkConf.getTableInfo.getProperties.getProperty("partition_columns") - val partitionColumnNames = Option(partitionColumns).map(_.split("/")).orNull + val partitionColumnNames = Option(partitionColumns).map(_.split("/")).getOrElse(Array.empty) + + // By this time, the partition map must match the table's partition columns + if (partitionColumnNames.toSet != partition.keySet) { + throw new SparkException( + s"""Requested partitioning does not match the ${table.identifier.table} table: + |Requested partitions: ${partition.keys.mkString(",")} + |Table partitions: ${table.partitionColumnNames.mkString(",")}""".stripMargin) + } // Validate partition spec if there exist any dynamic partitions if (numDynamicPartitions > 0) { // Report error if dynamic partitioning is not enabled - if (!sc.hiveconf.getBoolVar(HiveConf.ConfVars.DYNAMICPARTITIONING)) { + if (!hadoopConf.get("hive.exec.dynamic.partition", "true").toBoolean) { throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_DISABLED.getMsg) } // Report error if dynamic partition strict mode is on but no static partition is found if (numStaticPartitions == 0 && - sc.hiveconf.getVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE).equalsIgnoreCase("strict")) { + hadoopConf.get("hive.exec.dynamic.partition.mode", "strict").equalsIgnoreCase("strict")) { throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_STRICT_MODE.getMsg) } // Report error if any static partition appears after a dynamic partition val isDynamic = partitionColumnNames.map(partitionSpec(_).isEmpty) if (isDynamic.init.zip(isDynamic.tail).contains((true, false))) { - throw new SparkException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) + throw new AnalysisException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) } } - val jobConf = new JobConf(sc.hiveconf) - val jobConfSer = new SerializableJobConf(jobConf) - - // When speculation is on and output committer class name contains "Direct", we should warn - // users that they may loss data if they are using a direct output committer. - val speculationEnabled = sqlContext.sparkContext.conf.getBoolean("spark.speculation", false) - val outputCommitterClass = jobConf.get("mapred.output.committer.class", "") - if (speculationEnabled && outputCommitterClass.contains("Direct")) { - val warningMessage = - s"$outputCommitterClass may be an output committer that writes data directly to " + - "the final location. Because speculation is enabled, this output committer may " + - "cause data loss (see the case in SPARK-10063). If possible, please use a output " + - "committer that does not have this behavior (e.g. FileOutputCommitter)." - logWarning(warningMessage) + table.bucketSpec match { + case Some(bucketSpec) => + // Writes to bucketed hive tables are allowed only if user does not care about maintaining + // table's bucketing ie. both "hive.enforce.bucketing" and "hive.enforce.sorting" are + // set to false + val enforceBucketingConfig = "hive.enforce.bucketing" + val enforceSortingConfig = "hive.enforce.sorting" + + val message = s"Output Hive table ${table.identifier} is bucketed but Spark" + + "currently does NOT populate bucketed output which is compatible with Hive." + + if (hadoopConf.get(enforceBucketingConfig, "true").toBoolean || + hadoopConf.get(enforceSortingConfig, "true").toBoolean) { + throw new AnalysisException(message) + } else { + logWarning(message + s" Inserting data anyways since both $enforceBucketingConfig and " + + s"$enforceSortingConfig are set to false.") + } + case _ => // do nothing since table has no bucketing } - val writerContainer = if (numDynamicPartitions > 0) { - val dynamicPartColNames = partitionColumnNames.takeRight(numDynamicPartitions) - new SparkHiveDynamicPartitionWriterContainer(jobConf, fileSinkConf, dynamicPartColNames) - } else { - new SparkHiveWriterContainer(jobConf, fileSinkConf) + val partitionAttributes = partitionColumnNames.takeRight(numDynamicPartitions).map { name => + query.resolve(name :: Nil, sparkSession.sessionState.analyzer.resolver).getOrElse { + throw new AnalysisException( + s"Unable to resolve $name given [${query.output.map(_.name).mkString(", ")}]") + }.asInstanceOf[Attribute] } - saveAsHiveFile(child.execute(), outputClass, fileSinkConf, jobConfSer, writerContainer) + saveAsHiveFile( + sparkSession = sparkSession, + plan = children.head, + hadoopConf = hadoopConf, + fileSinkConf = fileSinkConf, + outputLocation = tmpLocation.toString, + partitionAttributes = partitionAttributes) - val outputPath = FileOutputFormat.getOutputPath(jobConf) - // Have to construct the format of dbname.tablename. - val qualifiedTableName = s"${table.databaseName}.${table.tableName}" - // TODO: Correctly set holdDDLTime. - // In most of the time, we should have holdDDLTime = false. - // holdDDLTime will be true when TOK_HOLD_DDLTIME presents in the query as a hint. - val holdDDLTime = false if (partition.nonEmpty) { - - // loadPartition call orders directories created on the iteration order of the this map - val orderedPartitionSpec = new util.LinkedHashMap[String, String]() - table.hiveQlTable.getPartCols.asScala.foreach { entry => - orderedPartitionSpec.put(entry.getName, partitionSpec.get(entry.getName).getOrElse("")) - } - - // inheritTableSpecs is set to true. It should be set to false for a IMPORT query - // which is currently considered as a Hive native command. - val inheritTableSpecs = true - // TODO: Correctly set isSkewedStoreAsSubdir. - val isSkewedStoreAsSubdir = false if (numDynamicPartitions > 0) { - catalog.synchronized { - catalog.client.loadDynamicPartitions( - outputPath.toString, - qualifiedTableName, - orderedPartitionSpec, - overwrite, - numDynamicPartitions, - holdDDLTime, - isSkewedStoreAsSubdir) - } + externalCatalog.loadDynamicPartitions( + db = table.database, + table = table.identifier.table, + tmpLocation.toString, + partitionSpec, + overwrite, + numDynamicPartitions) } else { // scalastyle:off // ifNotExists is only valid with static partition, refer to // https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DML#LanguageManualDML-InsertingdataintoHiveTablesfromqueries // scalastyle:on val oldPart = - catalog.client.getPartitionOption( - catalog.client.getTable(table.databaseName, table.tableName), - partitionSpec.asJava) - - if (oldPart.isEmpty || !ifNotExists) { - catalog.client.loadPartition( - outputPath.toString, - qualifiedTableName, - orderedPartitionSpec, - overwrite, - holdDDLTime, - inheritTableSpecs, - isSkewedStoreAsSubdir) + externalCatalog.getPartitionOption( + table.database, + table.identifier.table, + partitionSpec) + + var doHiveOverwrite = overwrite + + if (oldPart.isEmpty || !ifPartitionNotExists) { + // SPARK-18107: Insert overwrite runs much slower than hive-client. + // Newer Hive largely improves insert overwrite performance. As Spark uses older Hive + // version and we may not want to catch up new Hive version every time. We delete the + // Hive partition first and then load data file into the Hive partition. + if (oldPart.nonEmpty && overwrite) { + oldPart.get.storage.locationUri.foreach { uri => + val partitionPath = new Path(uri) + val fs = partitionPath.getFileSystem(hadoopConf) + if (fs.exists(partitionPath)) { + if (!fs.delete(partitionPath, true)) { + throw new RuntimeException( + "Cannot remove partition directory '" + partitionPath.toString) + } + // Don't let Hive do overwrite operation since it is slower. + doHiveOverwrite = false + } + } + } + + // inheritTableSpecs is set to true. It should be set to false for an IMPORT query + // which is currently considered as a Hive native command. + val inheritTableSpecs = true + externalCatalog.loadPartition( + table.database, + table.identifier.table, + tmpLocation.toString, + partitionSpec, + isOverwrite = doHiveOverwrite, + inheritTableSpecs = inheritTableSpecs, + isSrcLocal = false) } } } else { - catalog.client.loadTable( - outputPath.toString, // TODO: URI - qualifiedTableName, + externalCatalog.loadTable( + table.database, + table.identifier.table, + tmpLocation.toString, // TODO: URI overwrite, - holdDDLTime) + isSrcLocal = false) } - // Invalidate the cache. - sqlContext.cacheManager.invalidateCache(table) + // Attempt to delete the staging directory and the inclusive files. If failed, the files are + // expected to be dropped at the normal termination of VM since deleteOnExit is used. + deleteExternalTmpPath(hadoopConf) + + // un-cache this table. + sparkSession.catalog.uncacheTable(table.identifier.quotedString) + sparkSession.sessionState.catalog.refreshTable(table.identifier) + + CommandUtils.updateTableStats(sparkSession, table) // It would be nice to just return the childRdd unchanged so insert operations could be chained, // however for now we return an empty list to simplify compatibility checks with hive, which // does not return anything for insert operations. // TODO: implement hive compatibility as rules. - Seq.empty[InternalRow] - } - - override def executeCollect(): Array[InternalRow] = sideEffectResult.toArray - - protected override def doExecute(): RDD[InternalRow] = { - sqlContext.sparkContext.parallelize(sideEffectResult.asInstanceOf[Seq[InternalRow]], 1) + Seq.empty[Row] } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala new file mode 100644 index 000000000000..2d74ef040ef5 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import java.io.{File, IOException} +import java.net.URI +import java.text.SimpleDateFormat +import java.util.{Date, Locale, Random} + +import scala.util.control.NonFatal + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.hive.common.FileUtils +import org.apache.hadoop.hive.ql.exec.TaskRunner + +import org.apache.spark.internal.io.FileCommitProtocol +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.command.DataWritingCommand +import org.apache.spark.sql.execution.datasources.FileFormatWriter +import org.apache.spark.sql.hive.HiveExternalCatalog +import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} +import org.apache.spark.sql.hive.client.HiveVersion + +// Base trait from which all hive insert statement physical execution extends. +private[hive] trait SaveAsHiveFile extends DataWritingCommand { + + var createdTempDir: Option[Path] = None + + protected def saveAsHiveFile( + sparkSession: SparkSession, + plan: SparkPlan, + hadoopConf: Configuration, + fileSinkConf: FileSinkDesc, + outputLocation: String, + customPartitionLocations: Map[TablePartitionSpec, String] = Map.empty, + partitionAttributes: Seq[Attribute] = Nil): Set[String] = { + + val isCompressed = hadoopConf.get("hive.exec.compress.output", "false").toBoolean + if (isCompressed) { + // Please note that isCompressed, "mapreduce.output.fileoutputformat.compress", + // "mapreduce.output.fileoutputformat.compress.codec", and + // "mapreduce.output.fileoutputformat.compress.type" + // have no impact on ORC because it uses table properties to store compression information. + hadoopConf.set("mapreduce.output.fileoutputformat.compress", "true") + fileSinkConf.setCompressed(true) + fileSinkConf.setCompressCodec(hadoopConf + .get("mapreduce.output.fileoutputformat.compress.codec")) + fileSinkConf.setCompressType(hadoopConf + .get("mapreduce.output.fileoutputformat.compress.type")) + } + + val committer = FileCommitProtocol.instantiate( + sparkSession.sessionState.conf.fileCommitProtocolClass, + jobId = java.util.UUID.randomUUID().toString, + outputPath = outputLocation) + + FileFormatWriter.write( + sparkSession = sparkSession, + plan = plan, + fileFormat = new HiveFileFormat(fileSinkConf), + committer = committer, + outputSpec = FileFormatWriter.OutputSpec(outputLocation, customPartitionLocations), + hadoopConf = hadoopConf, + partitionColumns = partitionAttributes, + bucketSpec = None, + statsTrackers = Seq(basicWriteJobStatsTracker(hadoopConf)), + options = Map.empty) + } + + protected def getExternalTmpPath( + sparkSession: SparkSession, + hadoopConf: Configuration, + path: Path): Path = { + import org.apache.spark.sql.hive.client.hive._ + + // Before Hive 1.1, when inserting into a table, Hive will create the staging directory under + // a common scratch directory. After the writing is finished, Hive will simply empty the table + // directory and move the staging directory to it. + // After Hive 1.1, Hive will create the staging directory under the table directory, and when + // moving staging directory to table directory, Hive will still empty the table directory, but + // will exclude the staging directory there. + // We have to follow the Hive behavior here, to avoid troubles. For example, if we create + // staging directory under the table director for Hive prior to 1.1, the staging directory will + // be removed by Hive when Hive is trying to empty the table directory. + val hiveVersionsUsingOldExternalTempPath: Set[HiveVersion] = Set(v12, v13, v14, v1_0) + val hiveVersionsUsingNewExternalTempPath: Set[HiveVersion] = Set(v1_1, v1_2, v2_0, v2_1) + + // Ensure all the supported versions are considered here. + assert(hiveVersionsUsingNewExternalTempPath ++ hiveVersionsUsingOldExternalTempPath == + allSupportedHiveVersions) + + val externalCatalog = sparkSession.sharedState.externalCatalog + val hiveVersion = externalCatalog.asInstanceOf[HiveExternalCatalog].client.version + val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging") + val scratchDir = hadoopConf.get("hive.exec.scratchdir", "/tmp/hive") + + if (hiveVersionsUsingOldExternalTempPath.contains(hiveVersion)) { + oldVersionExternalTempPath(path, hadoopConf, scratchDir) + } else if (hiveVersionsUsingNewExternalTempPath.contains(hiveVersion)) { + newVersionExternalTempPath(path, hadoopConf, stagingDir) + } else { + throw new IllegalStateException("Unsupported hive version: " + hiveVersion.fullVersion) + } + } + + protected def deleteExternalTmpPath(hadoopConf: Configuration) : Unit = { + // Attempt to delete the staging directory and the inclusive files. If failed, the files are + // expected to be dropped at the normal termination of VM since deleteOnExit is used. + try { + createdTempDir.foreach { path => + val fs = path.getFileSystem(hadoopConf) + if (fs.delete(path, true)) { + // If we successfully delete the staging directory, remove it from FileSystem's cache. + fs.cancelDeleteOnExit(path) + } + } + } catch { + case NonFatal(e) => + val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging") + logWarning(s"Unable to delete staging directory: $stagingDir.\n" + e) + } + } + + // Mostly copied from Context.java#getExternalTmpPath of Hive 0.13 + private def oldVersionExternalTempPath( + path: Path, + hadoopConf: Configuration, + scratchDir: String): Path = { + val extURI: URI = path.toUri + val scratchPath = new Path(scratchDir, executionId) + var dirPath = new Path( + extURI.getScheme, + extURI.getAuthority, + scratchPath.toUri.getPath + "-" + TaskRunner.getTaskRunnerID()) + + try { + val fs: FileSystem = dirPath.getFileSystem(hadoopConf) + dirPath = new Path(fs.makeQualified(dirPath).toString()) + + if (!FileUtils.mkdir(fs, dirPath, true, hadoopConf)) { + throw new IllegalStateException("Cannot create staging directory: " + dirPath.toString) + } + createdTempDir = Some(dirPath) + fs.deleteOnExit(dirPath) + } catch { + case e: IOException => + throw new RuntimeException("Cannot create staging directory: " + dirPath.toString, e) + } + dirPath + } + + // Mostly copied from Context.java#getExternalTmpPath of Hive 1.2 + private def newVersionExternalTempPath( + path: Path, + hadoopConf: Configuration, + stagingDir: String): Path = { + val extURI: URI = path.toUri + if (extURI.getScheme == "viewfs") { + getExtTmpPathRelTo(path.getParent, hadoopConf, stagingDir) + } else { + new Path(getExternalScratchDir(extURI, hadoopConf, stagingDir), "-ext-10000") + } + } + + private def getExtTmpPathRelTo( + path: Path, + hadoopConf: Configuration, + stagingDir: String): Path = { + new Path(getStagingDir(path, hadoopConf, stagingDir), "-ext-10000") // Hive uses 10000 + } + + private def getExternalScratchDir( + extURI: URI, + hadoopConf: Configuration, + stagingDir: String): Path = { + getStagingDir( + new Path(extURI.getScheme, extURI.getAuthority, extURI.getPath), + hadoopConf, + stagingDir) + } + + private def getStagingDir( + inputPath: Path, + hadoopConf: Configuration, + stagingDir: String): Path = { + val inputPathUri: URI = inputPath.toUri + val inputPathName: String = inputPathUri.getPath + val fs: FileSystem = inputPath.getFileSystem(hadoopConf) + var stagingPathName: String = + if (inputPathName.indexOf(stagingDir) == -1) { + new Path(inputPathName, stagingDir).toString + } else { + inputPathName.substring(0, inputPathName.indexOf(stagingDir) + stagingDir.length) + } + + // SPARK-20594: This is a walk-around fix to resolve a Hive bug. Hive requires that the + // staging directory needs to avoid being deleted when users set hive.exec.stagingdir + // under the table directory. + if (FileUtils.isSubDir(new Path(stagingPathName), inputPath, fs) && + !stagingPathName.stripPrefix(inputPathName).stripPrefix(File.separator).startsWith(".")) { + logDebug(s"The staging dir '$stagingPathName' should be a child directory starts " + + "with '.' to avoid being deleted if we set hive.exec.stagingdir under the table " + + "directory.") + stagingPathName = new Path(inputPathName, ".hive-staging").toString + } + + val dir: Path = + fs.makeQualified( + new Path(stagingPathName + "_" + executionId + "-" + TaskRunner.getTaskRunnerID)) + logDebug("Created staging dir = " + dir + " for path = " + inputPath) + try { + if (!FileUtils.mkdir(fs, dir, true, hadoopConf)) { + throw new IllegalStateException("Cannot create staging directory '" + dir.toString + "'") + } + createdTempDir = Some(dir) + fs.deleteOnExit(dir) + } catch { + case e: IOException => + throw new RuntimeException( + "Cannot create staging directory '" + dir.toString + "': " + e.getMessage, e) + } + dir + } + + private def executionId: String = { + val rand: Random = new Random + val format = new SimpleDateFormat("yyyy-MM-dd_HH-mm-ss_SSS", Locale.US) + "hive_" + format.format(new Date) + "_" + Math.abs(rand.nextLong) + } +} + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala deleted file mode 100644 index b30117f0de99..000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ /dev/null @@ -1,399 +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.execution - -import java.io._ -import java.util.Properties -import javax.annotation.Nullable - -import scala.collection.JavaConverters._ -import scala.util.control.NonFatal - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hive.ql.exec.{RecordReader, RecordWriter} -import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.hadoop.hive.serde2.AbstractSerDe -import org.apache.hadoop.hive.serde2.objectinspector._ -import org.apache.hadoop.io.Writable - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.hive.HiveShim._ -import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} -import org.apache.spark.sql.types.DataType -import org.apache.spark.util.{CircularBuffer, RedirectThread, SerializableConfiguration, Utils} -import org.apache.spark.{Logging, TaskContext} - -/** - * Transforms the input by forking and running the specified script. - * - * @param input the set of expression that should be passed to the script. - * @param script the command that should be executed. - * @param output the attributes that are produced by the script. - */ -private[hive] -case class ScriptTransformation( - input: Seq[Expression], - script: String, - output: Seq[Attribute], - child: SparkPlan, - ioschema: HiveScriptIOSchema)(@transient private val sc: HiveContext) - extends UnaryNode { - - override def otherCopyArgs: Seq[HiveContext] = sc :: Nil - - private val serializedHiveConf = new SerializableConfiguration(sc.hiveconf) - - protected override def doExecute(): RDD[InternalRow] = { - def processIterator(inputIterator: Iterator[InternalRow]): Iterator[InternalRow] = { - val cmd = List("/bin/bash", "-c", script) - val builder = new ProcessBuilder(cmd.asJava) - - val proc = builder.start() - val inputStream = proc.getInputStream - val outputStream = proc.getOutputStream - val errorStream = proc.getErrorStream - val localHiveConf = serializedHiveConf.value - - // In order to avoid deadlocks, we need to consume the error output of the child process. - // To avoid issues caused by large error output, we use a circular buffer to limit the amount - // of error output that we retain. See SPARK-7862 for more discussion of the deadlock / hang - // that motivates this. - val stderrBuffer = new CircularBuffer(2048) - new RedirectThread( - errorStream, - stderrBuffer, - "Thread-ScriptTransformation-STDERR-Consumer").start() - - val outputProjection = new InterpretedProjection(input, child.output) - - // This nullability is a performance optimization in order to avoid an Option.foreach() call - // inside of a loop - @Nullable val (inputSerde, inputSoi) = ioschema.initInputSerDe(input).getOrElse((null, null)) - - // This new thread will consume the ScriptTransformation's input rows and write them to the - // external process. That process's output will be read by this current thread. - val writerThread = new ScriptTransformationWriterThread( - inputIterator, - input.map(_.dataType), - outputProjection, - inputSerde, - inputSoi, - ioschema, - outputStream, - proc, - stderrBuffer, - TaskContext.get(), - localHiveConf - ) - - // This nullability is a performance optimization in order to avoid an Option.foreach() call - // inside of a loop - @Nullable val (outputSerde, outputSoi) = { - ioschema.initOutputSerDe(output).getOrElse((null, null)) - } - - val reader = new BufferedReader(new InputStreamReader(inputStream)) - val outputIterator: Iterator[InternalRow] = new Iterator[InternalRow] with HiveInspectors { - var curLine: String = null - val scriptOutputStream = new DataInputStream(inputStream) - - @Nullable val scriptOutputReader = - ioschema.recordReader(scriptOutputStream, localHiveConf).orNull - - var scriptOutputWritable: Writable = null - val reusedWritableObject: Writable = if (null != outputSerde) { - outputSerde.getSerializedClass().newInstance - } else { - null - } - val mutableRow = new SpecificMutableRow(output.map(_.dataType)) - - override def hasNext: Boolean = { - if (outputSerde == null) { - if (curLine == null) { - curLine = reader.readLine() - if (curLine == null) { - if (writerThread.exception.isDefined) { - throw writerThread.exception.get - } - false - } else { - true - } - } else { - true - } - } else if (scriptOutputWritable == null) { - scriptOutputWritable = reusedWritableObject - - if (scriptOutputReader != null) { - if (scriptOutputReader.next(scriptOutputWritable) <= 0) { - writerThread.exception.foreach(throw _) - false - } else { - true - } - } else { - try { - scriptOutputWritable.readFields(scriptOutputStream) - true - } catch { - case _: EOFException => - if (writerThread.exception.isDefined) { - throw writerThread.exception.get - } - false - } - } - } else { - true - } - } - - override def next(): InternalRow = { - if (!hasNext) { - throw new NoSuchElementException - } - if (outputSerde == null) { - val prevLine = curLine - curLine = reader.readLine() - if (!ioschema.schemaLess) { - new GenericInternalRow( - prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD")) - .map(CatalystTypeConverters.convertToCatalyst)) - } else { - new GenericInternalRow( - prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2) - .map(CatalystTypeConverters.convertToCatalyst)) - } - } else { - val raw = outputSerde.deserialize(scriptOutputWritable) - scriptOutputWritable = null - val dataList = outputSoi.getStructFieldsDataAsList(raw) - val fieldList = outputSoi.getAllStructFieldRefs() - var i = 0 - while (i < dataList.size()) { - if (dataList.get(i) == null) { - mutableRow.setNullAt(i) - } else { - mutableRow(i) = unwrap(dataList.get(i), fieldList.get(i).getFieldObjectInspector) - } - i += 1 - } - mutableRow - } - } - } - - writerThread.start() - - outputIterator - } - - child.execute().mapPartitions { iter => - if (iter.hasNext) { - processIterator(iter) - } else { - // If the input iterator has no rows then do not launch the external script. - Iterator.empty - } - } - } -} - -private class ScriptTransformationWriterThread( - iter: Iterator[InternalRow], - inputSchema: Seq[DataType], - outputProjection: Projection, - @Nullable inputSerde: AbstractSerDe, - @Nullable inputSoi: ObjectInspector, - ioschema: HiveScriptIOSchema, - outputStream: OutputStream, - proc: Process, - stderrBuffer: CircularBuffer, - taskContext: TaskContext, - conf: Configuration - ) extends Thread("Thread-ScriptTransformation-Feed") with Logging { - - setDaemon(true) - - @volatile private var _exception: Throwable = null - - /** Contains the exception thrown while writing the parent iterator to the external process. */ - def exception: Option[Throwable] = Option(_exception) - - override def run(): Unit = Utils.logUncaughtExceptions { - TaskContext.setTaskContext(taskContext) - - val dataOutputStream = new DataOutputStream(outputStream) - @Nullable val scriptInputWriter = ioschema.recordWriter(dataOutputStream, conf).orNull - - // We can't use Utils.tryWithSafeFinally here because we also need a `catch` block, so - // let's use a variable to record whether the `finally` block was hit due to an exception - var threwException: Boolean = true - val len = inputSchema.length - try { - iter.map(outputProjection).foreach { row => - if (inputSerde == null) { - val data = if (len == 0) { - ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES") - } else { - val sb = new StringBuilder - sb.append(row.get(0, inputSchema(0))) - var i = 1 - while (i < len) { - sb.append(ioschema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD")) - sb.append(row.get(i, inputSchema(i))) - i += 1 - } - sb.append(ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")) - sb.toString() - } - outputStream.write(data.getBytes("utf-8")) - } else { - val writable = inputSerde.serialize( - row.asInstanceOf[GenericInternalRow].values, inputSoi) - - if (scriptInputWriter != null) { - scriptInputWriter.write(writable) - } else { - prepareWritable(writable, ioschema.outputSerdeProps).write(dataOutputStream) - } - } - } - outputStream.close() - threwException = false - } catch { - case NonFatal(e) => - // An error occurred while writing input, so kill the child process. According to the - // Javadoc this call will not throw an exception: - _exception = e - proc.destroy() - throw e - } finally { - try { - if (proc.waitFor() != 0) { - logError(stderrBuffer.toString) // log the stderr circular buffer - } - } catch { - case NonFatal(exceptionFromFinallyBlock) => - if (!threwException) { - throw exceptionFromFinallyBlock - } else { - log.error("Exception in finally block", exceptionFromFinallyBlock) - } - } - } - } -} - -/** - * The wrapper class of Hive input and output schema properties - */ -private[hive] -case class HiveScriptIOSchema ( - inputRowFormat: Seq[(String, String)], - outputRowFormat: Seq[(String, String)], - inputSerdeClass: Option[String], - outputSerdeClass: Option[String], - inputSerdeProps: Seq[(String, String)], - outputSerdeProps: Seq[(String, String)], - recordReaderClass: Option[String], - recordWriterClass: Option[String], - schemaLess: Boolean) extends ScriptInputOutputSchema with HiveInspectors { - - private val defaultFormat = Map( - ("TOK_TABLEROWFORMATFIELD", "\t"), - ("TOK_TABLEROWFORMATLINES", "\n") - ) - - val inputRowFormatMap = inputRowFormat.toMap.withDefault((k) => defaultFormat(k)) - val outputRowFormatMap = outputRowFormat.toMap.withDefault((k) => defaultFormat(k)) - - - def initInputSerDe(input: Seq[Expression]): Option[(AbstractSerDe, ObjectInspector)] = { - inputSerdeClass.map { serdeClass => - val (columns, columnTypes) = parseAttrs(input) - val serde = initSerDe(serdeClass, columns, columnTypes, inputSerdeProps) - val fieldObjectInspectors = columnTypes.map(toInspector) - val objectInspector = ObjectInspectorFactory - .getStandardStructObjectInspector(columns.asJava, fieldObjectInspectors.asJava) - .asInstanceOf[ObjectInspector] - (serde, objectInspector) - } - } - - def initOutputSerDe(output: Seq[Attribute]): Option[(AbstractSerDe, StructObjectInspector)] = { - outputSerdeClass.map { serdeClass => - val (columns, columnTypes) = parseAttrs(output) - val serde = initSerDe(serdeClass, columns, columnTypes, outputSerdeProps) - val structObjectInspector = serde.getObjectInspector().asInstanceOf[StructObjectInspector] - (serde, structObjectInspector) - } - } - - private def parseAttrs(attrs: Seq[Expression]): (Seq[String], Seq[DataType]) = { - val columns = attrs.zipWithIndex.map(e => s"${e._1.prettyName}_${e._2}") - val columnTypes = attrs.map(_.dataType) - (columns, columnTypes) - } - - private def initSerDe( - serdeClassName: String, - columns: Seq[String], - columnTypes: Seq[DataType], - serdeProps: Seq[(String, String)]): AbstractSerDe = { - - val serde = Utils.classForName(serdeClassName).newInstance.asInstanceOf[AbstractSerDe] - - val columnTypesNames = columnTypes.map(_.toTypeInfo.getTypeName()).mkString(",") - - var propsMap = serdeProps.toMap + (serdeConstants.LIST_COLUMNS -> columns.mkString(",")) - propsMap = propsMap + (serdeConstants.LIST_COLUMN_TYPES -> columnTypesNames) - - val properties = new Properties() - properties.putAll(propsMap.asJava) - serde.initialize(null, properties) - - serde - } - - def recordReader( - inputStream: InputStream, - conf: Configuration): Option[RecordReader] = { - recordReaderClass.map { klass => - val instance = Utils.classForName(klass).newInstance().asInstanceOf[RecordReader] - val props = new Properties() - props.putAll(outputSerdeProps.toMap.asJava) - instance.initialize(inputStream, conf, props) - instance - } - } - - def recordWriter(outputStream: OutputStream, conf: Configuration): Option[RecordWriter] = { - recordWriterClass.map { klass => - val instance = Utils.classForName(klass).newInstance().asInstanceOf[RecordWriter] - instance.initialize(outputStream, conf) - instance - } - } -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala new file mode 100644 index 000000000000..d786a610f153 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala @@ -0,0 +1,440 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import java.io._ +import java.nio.charset.StandardCharsets +import java.util.Properties +import javax.annotation.Nullable + +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.ql.exec.{RecordReader, RecordWriter} +import org.apache.hadoop.hive.serde.serdeConstants +import org.apache.hadoop.hive.serde2.AbstractSerDe +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.io.Writable + +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.hive.HiveInspectors +import org.apache.spark.sql.hive.HiveShim._ +import org.apache.spark.sql.types.DataType +import org.apache.spark.util.{CircularBuffer, RedirectThread, SerializableConfiguration, Utils} + +/** + * Transforms the input by forking and running the specified script. + * + * @param input the set of expression that should be passed to the script. + * @param script the command that should be executed. + * @param output the attributes that are produced by the script. + */ +case class ScriptTransformationExec( + input: Seq[Expression], + script: String, + output: Seq[Attribute], + child: SparkPlan, + ioschema: HiveScriptIOSchema) + extends UnaryExecNode { + + override def producedAttributes: AttributeSet = outputSet -- inputSet + + override def outputPartitioning: Partitioning = child.outputPartitioning + + protected override def doExecute(): RDD[InternalRow] = { + def processIterator(inputIterator: Iterator[InternalRow], hadoopConf: Configuration) + : Iterator[InternalRow] = { + val cmd = List("/bin/bash", "-c", script) + val builder = new ProcessBuilder(cmd.asJava) + + val proc = builder.start() + val inputStream = proc.getInputStream + val outputStream = proc.getOutputStream + val errorStream = proc.getErrorStream + + // In order to avoid deadlocks, we need to consume the error output of the child process. + // To avoid issues caused by large error output, we use a circular buffer to limit the amount + // of error output that we retain. See SPARK-7862 for more discussion of the deadlock / hang + // that motivates this. + val stderrBuffer = new CircularBuffer(2048) + new RedirectThread( + errorStream, + stderrBuffer, + "Thread-ScriptTransformation-STDERR-Consumer").start() + + val outputProjection = new InterpretedProjection(input, child.output) + + // This nullability is a performance optimization in order to avoid an Option.foreach() call + // inside of a loop + @Nullable val (inputSerde, inputSoi) = ioschema.initInputSerDe(input).getOrElse((null, null)) + + // This new thread will consume the ScriptTransformation's input rows and write them to the + // external process. That process's output will be read by this current thread. + val writerThread = new ScriptTransformationWriterThread( + inputIterator, + input.map(_.dataType), + outputProjection, + inputSerde, + inputSoi, + ioschema, + outputStream, + proc, + stderrBuffer, + TaskContext.get(), + hadoopConf + ) + + // This nullability is a performance optimization in order to avoid an Option.foreach() call + // inside of a loop + @Nullable val (outputSerde, outputSoi) = { + ioschema.initOutputSerDe(output).getOrElse((null, null)) + } + + val reader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8)) + val outputIterator: Iterator[InternalRow] = new Iterator[InternalRow] with HiveInspectors { + var curLine: String = null + val scriptOutputStream = new DataInputStream(inputStream) + + @Nullable val scriptOutputReader = + ioschema.recordReader(scriptOutputStream, hadoopConf).orNull + + var scriptOutputWritable: Writable = null + val reusedWritableObject: Writable = if (null != outputSerde) { + outputSerde.getSerializedClass().newInstance + } else { + null + } + val mutableRow = new SpecificInternalRow(output.map(_.dataType)) + + @transient + lazy val unwrappers = outputSoi.getAllStructFieldRefs.asScala.map(unwrapperFor) + + private def checkFailureAndPropagate(cause: Throwable = null): Unit = { + if (writerThread.exception.isDefined) { + throw writerThread.exception.get + } + + if (!proc.isAlive) { + val exitCode = proc.exitValue() + if (exitCode != 0) { + logError(stderrBuffer.toString) // log the stderr circular buffer + throw new SparkException(s"Subprocess exited with status $exitCode. " + + s"Error: ${stderrBuffer.toString}", cause) + } + } + } + + override def hasNext: Boolean = { + try { + if (outputSerde == null) { + if (curLine == null) { + curLine = reader.readLine() + if (curLine == null) { + checkFailureAndPropagate() + return false + } + } + } else if (scriptOutputWritable == null) { + scriptOutputWritable = reusedWritableObject + + if (scriptOutputReader != null) { + if (scriptOutputReader.next(scriptOutputWritable) <= 0) { + checkFailureAndPropagate() + return false + } + } else { + try { + scriptOutputWritable.readFields(scriptOutputStream) + } catch { + case _: EOFException => + // This means that the stdout of `proc` (ie. TRANSFORM process) has exhausted. + // Ideally the proc should *not* be alive at this point but + // there can be a lag between EOF being written out and the process + // being terminated. So explicitly waiting for the process to be done. + proc.waitFor() + checkFailureAndPropagate() + return false + } + } + } + + true + } catch { + case NonFatal(e) => + // If this exception is due to abrupt / unclean termination of `proc`, + // then detect it and propagate a better exception message for end users + checkFailureAndPropagate(e) + + throw e + } + } + + override def next(): InternalRow = { + if (!hasNext) { + throw new NoSuchElementException + } + if (outputSerde == null) { + val prevLine = curLine + curLine = reader.readLine() + if (!ioschema.schemaLess) { + new GenericInternalRow( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD")) + .map(CatalystTypeConverters.convertToCatalyst)) + } else { + new GenericInternalRow( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2) + .map(CatalystTypeConverters.convertToCatalyst)) + } + } else { + val raw = outputSerde.deserialize(scriptOutputWritable) + scriptOutputWritable = null + val dataList = outputSoi.getStructFieldsDataAsList(raw) + var i = 0 + while (i < dataList.size()) { + if (dataList.get(i) == null) { + mutableRow.setNullAt(i) + } else { + unwrappers(i)(dataList.get(i), mutableRow, i) + } + i += 1 + } + mutableRow + } + } + } + + writerThread.start() + + outputIterator + } + + val broadcastedHadoopConf = + new SerializableConfiguration(sqlContext.sessionState.newHadoopConf()) + + child.execute().mapPartitions { iter => + if (iter.hasNext) { + val proj = UnsafeProjection.create(schema) + processIterator(iter, broadcastedHadoopConf.value).map(proj) + } else { + // If the input iterator has no rows then do not launch the external script. + Iterator.empty + } + } + } +} + +private class ScriptTransformationWriterThread( + iter: Iterator[InternalRow], + inputSchema: Seq[DataType], + outputProjection: Projection, + @Nullable inputSerde: AbstractSerDe, + @Nullable inputSoi: ObjectInspector, + ioschema: HiveScriptIOSchema, + outputStream: OutputStream, + proc: Process, + stderrBuffer: CircularBuffer, + taskContext: TaskContext, + conf: Configuration + ) extends Thread("Thread-ScriptTransformation-Feed") with Logging { + + setDaemon(true) + + @volatile private var _exception: Throwable = null + + /** Contains the exception thrown while writing the parent iterator to the external process. */ + def exception: Option[Throwable] = Option(_exception) + + override def run(): Unit = Utils.logUncaughtExceptions { + TaskContext.setTaskContext(taskContext) + + val dataOutputStream = new DataOutputStream(outputStream) + @Nullable val scriptInputWriter = ioschema.recordWriter(dataOutputStream, conf).orNull + + // We can't use Utils.tryWithSafeFinally here because we also need a `catch` block, so + // let's use a variable to record whether the `finally` block was hit due to an exception + var threwException: Boolean = true + val len = inputSchema.length + try { + iter.map(outputProjection).foreach { row => + if (inputSerde == null) { + val data = if (len == 0) { + ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES") + } else { + val sb = new StringBuilder + sb.append(row.get(0, inputSchema(0))) + var i = 1 + while (i < len) { + sb.append(ioschema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD")) + sb.append(row.get(i, inputSchema(i))) + i += 1 + } + sb.append(ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")) + sb.toString() + } + outputStream.write(data.getBytes(StandardCharsets.UTF_8)) + } else { + val writable = inputSerde.serialize( + row.asInstanceOf[GenericInternalRow].values, inputSoi) + + if (scriptInputWriter != null) { + scriptInputWriter.write(writable) + } else { + prepareWritable(writable, ioschema.outputSerdeProps).write(dataOutputStream) + } + } + } + threwException = false + } catch { + case t: Throwable => + // An error occurred while writing input, so kill the child process. According to the + // Javadoc this call will not throw an exception: + _exception = t + proc.destroy() + throw t + } finally { + try { + Utils.tryLogNonFatalError(outputStream.close()) + if (proc.waitFor() != 0) { + logError(stderrBuffer.toString) // log the stderr circular buffer + } + } catch { + case NonFatal(exceptionFromFinallyBlock) => + if (!threwException) { + throw exceptionFromFinallyBlock + } else { + log.error("Exception in finally block", exceptionFromFinallyBlock) + } + } + } + } +} + +object HiveScriptIOSchema { + def apply(input: ScriptInputOutputSchema): HiveScriptIOSchema = { + HiveScriptIOSchema( + input.inputRowFormat, + input.outputRowFormat, + input.inputSerdeClass, + input.outputSerdeClass, + input.inputSerdeProps, + input.outputSerdeProps, + input.recordReaderClass, + input.recordWriterClass, + input.schemaLess) + } +} + +/** + * The wrapper class of Hive input and output schema properties + */ +case class HiveScriptIOSchema ( + inputRowFormat: Seq[(String, String)], + outputRowFormat: Seq[(String, String)], + inputSerdeClass: Option[String], + outputSerdeClass: Option[String], + inputSerdeProps: Seq[(String, String)], + outputSerdeProps: Seq[(String, String)], + recordReaderClass: Option[String], + recordWriterClass: Option[String], + schemaLess: Boolean) + extends HiveInspectors { + + private val defaultFormat = Map( + ("TOK_TABLEROWFORMATFIELD", "\t"), + ("TOK_TABLEROWFORMATLINES", "\n") + ) + + val inputRowFormatMap = inputRowFormat.toMap.withDefault((k) => defaultFormat(k)) + val outputRowFormatMap = outputRowFormat.toMap.withDefault((k) => defaultFormat(k)) + + + def initInputSerDe(input: Seq[Expression]): Option[(AbstractSerDe, ObjectInspector)] = { + inputSerdeClass.map { serdeClass => + val (columns, columnTypes) = parseAttrs(input) + val serde = initSerDe(serdeClass, columns, columnTypes, inputSerdeProps) + val fieldObjectInspectors = columnTypes.map(toInspector) + val objectInspector = ObjectInspectorFactory + .getStandardStructObjectInspector(columns.asJava, fieldObjectInspectors.asJava) + .asInstanceOf[ObjectInspector] + (serde, objectInspector) + } + } + + def initOutputSerDe(output: Seq[Attribute]): Option[(AbstractSerDe, StructObjectInspector)] = { + outputSerdeClass.map { serdeClass => + val (columns, columnTypes) = parseAttrs(output) + val serde = initSerDe(serdeClass, columns, columnTypes, outputSerdeProps) + val structObjectInspector = serde.getObjectInspector().asInstanceOf[StructObjectInspector] + (serde, structObjectInspector) + } + } + + private def parseAttrs(attrs: Seq[Expression]): (Seq[String], Seq[DataType]) = { + val columns = attrs.zipWithIndex.map(e => s"${e._1.prettyName}_${e._2}") + val columnTypes = attrs.map(_.dataType) + (columns, columnTypes) + } + + private def initSerDe( + serdeClassName: String, + columns: Seq[String], + columnTypes: Seq[DataType], + serdeProps: Seq[(String, String)]): AbstractSerDe = { + + val serde = Utils.classForName(serdeClassName).newInstance.asInstanceOf[AbstractSerDe] + + val columnTypesNames = columnTypes.map(_.toTypeInfo.getTypeName()).mkString(",") + + var propsMap = serdeProps.toMap + (serdeConstants.LIST_COLUMNS -> columns.mkString(",")) + propsMap = propsMap + (serdeConstants.LIST_COLUMN_TYPES -> columnTypesNames) + + val properties = new Properties() + properties.putAll(propsMap.asJava) + serde.initialize(null, properties) + + serde + } + + def recordReader( + inputStream: InputStream, + conf: Configuration): Option[RecordReader] = { + recordReaderClass.map { klass => + val instance = Utils.classForName(klass).newInstance().asInstanceOf[RecordReader] + val props = new Properties() + props.putAll(outputSerdeProps.toMap.asJava) + instance.initialize(inputStream, conf, props) + instance + } + } + + def recordWriter(outputStream: OutputStream, conf: Configuration): Option[RecordWriter] = { + recordWriterClass.map { klass => + val instance = Utils.classForName(klass).newInstance().asInstanceOf[RecordWriter] + instance.initialize(outputStream, conf) + instance + } + } +} 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 deleted file mode 100644 index 94210a5394f9..000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ /dev/null @@ -1,277 +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.execution - -import org.apache.hadoop.hive.metastore.MetaStoreUtils - -import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.RunnableCommand -import org.apache.spark.sql.execution.datasources.{LogicalRelation, ResolvedDataSource} -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types._ - -/** - * Analyzes the given table in the current database to generate statistics, which will be - * used in query optimizations. - * - * Right now, it only supports Hive tables and it only updates the size of a Hive table - * in the Hive metastore. - */ -private[hive] -case class AnalyzeTable(tableName: String) extends RunnableCommand { - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.asInstanceOf[HiveContext].analyze(tableName) - Seq.empty[Row] - } -} - -/** - * Drops a table from the metastore and removes it if it is cached. - */ -private[hive] -case class DropTable( - tableName: String, - ifExists: Boolean) extends RunnableCommand { - - override def run(sqlContext: SQLContext): Seq[Row] = { - val hiveContext = sqlContext.asInstanceOf[HiveContext] - val ifExistsClause = if (ifExists) "IF EXISTS " else "" - try { - hiveContext.cacheManager.tryUncacheQuery(hiveContext.table(tableName)) - } catch { - // This table's metadata is not in Hive metastore (e.g. the table does not exist). - case _: org.apache.hadoop.hive.ql.metadata.InvalidTableException => - case _: org.apache.spark.sql.catalyst.analysis.NoSuchTableException => - // Other Throwables can be caused by users providing wrong parameters in OPTIONS - // (e.g. invalid paths). We catch it and log a warning message. - // Users should be able to drop such kinds of tables regardless if there is an error. - case e: Throwable => log.warn(s"${e.getMessage}", e) - } - hiveContext.invalidateTable(tableName) - hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName") - hiveContext.catalog.unregisterTable(TableIdentifier(tableName)) - Seq.empty[Row] - } -} - -private[hive] -case class AddJar(path: String) extends RunnableCommand { - - override val output: Seq[Attribute] = { - val schema = StructType( - StructField("result", IntegerType, false) :: Nil) - schema.toAttributes - } - - override def run(sqlContext: SQLContext): Seq[Row] = { - sqlContext.addJar(path) - - Seq(Row(0)) - } -} - -private[hive] -case class AddFile(path: String) extends RunnableCommand { - - override def run(sqlContext: SQLContext): Seq[Row] = { - val hiveContext = sqlContext.asInstanceOf[HiveContext] - hiveContext.runSqlHive(s"ADD FILE $path") - hiveContext.sparkContext.addFile(path) - Seq.empty[Row] - } -} - -private[hive] -case class CreateMetastoreDataSource( - tableIdent: TableIdentifier, - userSpecifiedSchema: Option[StructType], - provider: String, - options: Map[String, String], - allowExisting: Boolean, - managedIfNoPath: Boolean) extends RunnableCommand { - - override def run(sqlContext: SQLContext): Seq[Row] = { - // Since we are saving metadata to metastore, we need to check if metastore supports - // the table name and database name we have for this query. MetaStoreUtils.validateName - // is the method used by Hive to check if a table name or a database name is valid for - // the metastore. - if (!MetaStoreUtils.validateName(tableIdent.table)) { - throw new AnalysisException(s"Table name ${tableIdent.table} is not a valid name for " + - s"metastore. Metastore only accepts table name containing characters, numbers and _.") - } - if (tableIdent.database.isDefined && !MetaStoreUtils.validateName(tableIdent.database.get)) { - throw new AnalysisException(s"Database name ${tableIdent.database.get} is not a valid name " + - s"for metastore. Metastore only accepts database name containing " + - s"characters, numbers and _.") - } - - val tableName = tableIdent.unquotedString - val hiveContext = sqlContext.asInstanceOf[HiveContext] - - if (hiveContext.catalog.tableExists(tableIdent)) { - if (allowExisting) { - return Seq.empty[Row] - } else { - throw new AnalysisException(s"Table $tableName already exists.") - } - } - - var isExternal = true - val optionsWithPath = - if (!options.contains("path") && managedIfNoPath) { - isExternal = false - options + ("path" -> hiveContext.catalog.hiveDefaultTableFilePath(tableIdent)) - } else { - options - } - - hiveContext.catalog.createDataSourceTable( - tableIdent, - userSpecifiedSchema, - Array.empty[String], - provider, - optionsWithPath, - isExternal) - - Seq.empty[Row] - } -} - -private[hive] -case class CreateMetastoreDataSourceAsSelect( - tableIdent: TableIdentifier, - provider: String, - partitionColumns: Array[String], - mode: SaveMode, - options: Map[String, String], - query: LogicalPlan) extends RunnableCommand { - - override def run(sqlContext: SQLContext): Seq[Row] = { - // Since we are saving metadata to metastore, we need to check if metastore supports - // the table name and database name we have for this query. MetaStoreUtils.validateName - // is the method used by Hive to check if a table name or a database name is valid for - // the metastore. - if (!MetaStoreUtils.validateName(tableIdent.table)) { - throw new AnalysisException(s"Table name ${tableIdent.table} is not a valid name for " + - s"metastore. Metastore only accepts table name containing characters, numbers and _.") - } - if (tableIdent.database.isDefined && !MetaStoreUtils.validateName(tableIdent.database.get)) { - throw new AnalysisException(s"Database name ${tableIdent.database.get} is not a valid name " + - s"for metastore. Metastore only accepts database name containing " + - s"characters, numbers and _.") - } - - val tableName = tableIdent.unquotedString - val hiveContext = sqlContext.asInstanceOf[HiveContext] - var createMetastoreTable = false - var isExternal = true - val optionsWithPath = - if (!options.contains("path")) { - isExternal = false - options + ("path" -> hiveContext.catalog.hiveDefaultTableFilePath(tableIdent)) - } else { - options - } - - var existingSchema = None: Option[StructType] - if (sqlContext.catalog.tableExists(tableIdent)) { - // Check if we need to throw an exception or just return. - mode match { - case SaveMode.ErrorIfExists => - 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] - case SaveMode.Append => - // Check if the specified data source match the data source of the existing table. - val resolved = ResolvedDataSource( - sqlContext, Some(query.schema.asNullable), partitionColumns, provider, optionsWithPath) - val createdRelation = LogicalRelation(resolved.relation) - EliminateSubQueries(sqlContext.catalog.lookupRelation(tableIdent)) match { - case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _) => - if (l.relation != 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 - throw new AnalysisException(errorMessage) - } - existingSchema = Some(l.schema) - case o => - throw new AnalysisException(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 - } - - val data = DataFrame(hiveContext, query) - val df = existingSchema match { - // If we are inserting into an existing table, just use the existing schema. - case Some(schema) => sqlContext.internalCreateDataFrame(data.queryExecution.toRdd, schema) - case None => data - } - - // Create the relation based on the data of df. - val resolved = - ResolvedDataSource(sqlContext, provider, partitionColumns, mode, optionsWithPath, df) - - if (createMetastoreTable) { - // We will use the schema of resolved.relation as the schema of the table (instead of - // the schema of df). It is important since the nullability may be changed by the relation - // provider (for example, see org.apache.spark.sql.parquet.DefaultSource). - hiveContext.catalog.createDataSourceTable( - tableIdent, - Some(resolved.relation.schema), - partitionColumns, - provider, - optionsWithPath, - isExternal) - } - - // Refresh the cache of the table in the catalog. - hiveContext.catalog.refreshTable(tableIdent) - Seq.empty[Row] - } -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index a9db70119d01..e9bdcf00b934 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -17,122 +17,38 @@ package org.apache.spark.sql.hive -import scala.collection.mutable.ArrayBuffer +import java.nio.ByteBuffer + import scala.collection.JavaConverters._ -import scala.util.Try +import scala.collection.mutable.ArrayBuffer -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ConstantObjectInspector} -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory import org.apache.hadoop.hive.ql.exec._ import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ -import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationBuffer +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper +import org.apache.hadoop.hive.serde2.objectinspector.{ConstantObjectInspector, ObjectInspector, ObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions -import org.apache.spark.Logging -import org.apache.spark.sql.AnalysisException +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.ArrayData import org.apache.spark.sql.hive.HiveShim._ import org.apache.spark.sql.types._ -private[hive] class HiveFunctionRegistry(underlying: analysis.FunctionRegistry) - extends analysis.FunctionRegistry with HiveInspectors { - - def getFunctionInfo(name: String): FunctionInfo = FunctionRegistry.getFunctionInfo(name) - - override def lookupFunction(name: String, children: Seq[Expression]): Expression = { - Try(underlying.lookupFunction(name, children)).getOrElse { - // We only look it up to see if it exists, but do not include it in the HiveUDF since it is - // not always serializable. - val functionInfo: FunctionInfo = - Option(FunctionRegistry.getFunctionInfo(name.toLowerCase)).getOrElse( - throw new AnalysisException(s"undefined function $name")) - - val functionClassName = functionInfo.getFunctionClass.getName - - // When we instantiate hive UDF wrapper class, we may throw exception if the input expressions - // don't satisfy the hive UDF, such as type mismatch, input number mismatch, etc. Here we - // catch the exception and throw AnalysisException instead. - try { - if (classOf[GenericUDFMacro].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUDF( - new HiveFunctionWrapper(functionClassName, functionInfo.getGenericUDF), children) - } else if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveSimpleUDF(new HiveFunctionWrapper(functionClassName), children) - } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUDF(new HiveFunctionWrapper(functionClassName), children) - } else if ( - classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveUDAFFunction(new HiveFunctionWrapper(functionClassName), children) - } else if (classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveUDAFFunction( - new HiveFunctionWrapper(functionClassName), children, isUDAFBridgeRequired = true) - } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) { - val udtf = HiveGenericUDTF(new HiveFunctionWrapper(functionClassName), children) - udtf.elementTypes // Force it to check input data types. - udtf - } else { - throw new AnalysisException(s"No handler for udf ${functionInfo.getFunctionClass}") - } - } catch { - case analysisException: AnalysisException => - // If the exception is an AnalysisException, just throw it. - throw analysisException - case throwable: Throwable => - // If there is any other error, we throw an AnalysisException. - val errorMessage = s"No handler for Hive udf ${functionInfo.getFunctionClass} " + - s"because: ${throwable.getMessage}." - throw new AnalysisException(errorMessage) - } - } - } +private[hive] case class HiveSimpleUDF( + name: String, funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) + extends Expression + with HiveInspectors + with CodegenFallback + with Logging + with UserDefinedExpression { - override def registerFunction(name: String, info: ExpressionInfo, builder: FunctionBuilder) - : Unit = underlying.registerFunction(name, info, builder) - - /* List all of the registered function names. */ - override def listFunction(): Seq[String] = { - (FunctionRegistry.getFunctionNames.asScala ++ underlying.listFunction()).toList.sorted - } - - /* Get the class of the registered function by specified name. */ - override def lookupFunction(name: String): Option[ExpressionInfo] = { - underlying.lookupFunction(name).orElse( - Try { - val info = FunctionRegistry.getFunctionInfo(name) - val annotation = info.getFunctionClass.getAnnotation(classOf[Description]) - if (annotation != null) { - Some(new ExpressionInfo( - info.getFunctionClass.getCanonicalName, - annotation.name(), - annotation.value(), - annotation.extended())) - } else { - Some(new ExpressionInfo( - info.getFunctionClass.getCanonicalName, - name, - null, - null)) - } - }.getOrElse(None)) - } -} - -private[hive] case class HiveSimpleUDF(funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) - extends Expression with HiveInspectors with CodegenFallback with Logging { - - override def deterministic: Boolean = isUDFDeterministic + override def deterministic: Boolean = isUDFDeterministic && children.forall(_.deterministic) override def nullable: Boolean = true @@ -148,8 +64,8 @@ private[hive] case class HiveSimpleUDF(funcWrapper: HiveFunctionWrapper, childre @transient private lazy val isUDFDeterministic = { - val udfType = function.getClass().getAnnotation(classOf[HiveUDFType]) - udfType != null && udfType.deterministic() + val udfType = function.getClass.getAnnotation(classOf[HiveUDFType]) + udfType != null && udfType.deterministic() && !udfType.stateful() } override def foldable: Boolean = isUDFDeterministic && children.forall(_.foldable) @@ -158,11 +74,14 @@ private[hive] case class HiveSimpleUDF(funcWrapper: HiveFunctionWrapper, childre @transient private lazy val conversionHelper = new ConversionHelper(method, arguments) - override val dataType = javaClassToDataType(method.getReturnType) + override lazy val dataType = javaTypeToDataType(method.getGenericReturnType) @transient - lazy val returnInspector = ObjectInspectorFactory.getReflectionObjectInspector( - method.getGenericReturnType(), ObjectInspectorOptions.JAVA) + private lazy val wrappers = children.map(x => wrapperFor(toInspector(x), x.dataType)).toArray + + @transient + lazy val unwrapper = unwrapperFor(ObjectInspectorFactory.getReflectionObjectInspector( + method.getGenericReturnType, ObjectInspectorOptions.JAVA)) @transient private lazy val cached: Array[AnyRef] = new Array[AnyRef](children.length) @@ -172,37 +91,47 @@ private[hive] case class HiveSimpleUDF(funcWrapper: HiveFunctionWrapper, childre // TODO: Finish input output types. override def eval(input: InternalRow): Any = { - val inputs = wrap(children.map(c => c.eval(input)), arguments, cached, inputDataTypes) + val inputs = wrap(children.map(_.eval(input)), wrappers, cached, inputDataTypes) val ret = FunctionRegistry.invoke( method, function, conversionHelper.convertIfNecessary(inputs : _*): _*) - unwrap(ret, returnInspector) + unwrapper(ret) } override def toString: String = { s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" } + + override def prettyName: String = name + + override def sql: String = s"$name(${children.map(_.sql).mkString(", ")})" } // Adapter from Catalyst ExpressionResult to Hive DeferredObject private[hive] class DeferredObjectAdapter(oi: ObjectInspector, dataType: DataType) extends DeferredObject with HiveInspectors { + private val wrapper = wrapperFor(oi, dataType) private var func: () => Any = _ def set(func: () => Any): Unit = { this.func = func } override def prepare(i: Int): Unit = {} - override def get(): AnyRef = wrap(func(), oi, dataType) + override def get(): AnyRef = wrapper(func()).asInstanceOf[AnyRef] } -private[hive] case class HiveGenericUDF(funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) - extends Expression with HiveInspectors with CodegenFallback with Logging { +private[hive] case class HiveGenericUDF( + name: String, funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) + extends Expression + with HiveInspectors + with CodegenFallback + with Logging + with UserDefinedExpression { override def nullable: Boolean = true - override def deterministic: Boolean = isUDFDeterministic + override def deterministic: Boolean = isUDFDeterministic && children.forall(_.deterministic) override def foldable: Boolean = isUDFDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector] @@ -218,266 +147,46 @@ private[hive] case class HiveGenericUDF(funcWrapper: HiveFunctionWrapper, childr function.initializeAndFoldConstants(argumentInspectors.toArray) } + @transient + private lazy val unwrapper = unwrapperFor(returnInspector) + @transient private lazy val isUDFDeterministic = { val udfType = function.getClass.getAnnotation(classOf[HiveUDFType]) - udfType != null && udfType.deterministic() + udfType != null && udfType.deterministic() && !udfType.stateful() } @transient - private lazy val deferedObjects = argumentInspectors.zip(children).map { case (inspect, child) => + private lazy val deferredObjects = argumentInspectors.zip(children).map { case (inspect, child) => new DeferredObjectAdapter(inspect, child.dataType) }.toArray[DeferredObject] - override val dataType: DataType = inspectorToDataType(returnInspector) + override lazy val dataType: DataType = inspectorToDataType(returnInspector) override def eval(input: InternalRow): Any = { returnInspector // Make sure initialized. var i = 0 - while (i < children.length) { + val length = children.length + while (i < length) { val idx = i - deferedObjects(i).asInstanceOf[DeferredObjectAdapter].set( - () => { - children(idx).eval(input) - }) - i += 1 - } - unwrap(function.evaluate(deferedObjects), returnInspector) - } - - override def toString: String = { - s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" - } -} - -/** - * Resolves [[UnresolvedWindowFunction]] to [[HiveWindowFunction]]. - */ -private[spark] object ResolveHiveWindowFunction extends Rule[LogicalPlan] { - private def shouldResolveFunction( - unresolvedWindowFunction: UnresolvedWindowFunction, - windowSpec: WindowSpecDefinition): Boolean = { - unresolvedWindowFunction.childrenResolved && windowSpec.childrenResolved - } - - def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case p: LogicalPlan if !p.childrenResolved => p - - // We are resolving WindowExpressions at here. When we get here, we have already - // replaced those WindowSpecReferences. - case p: LogicalPlan => - p transformExpressions { - // We will not start to resolve the function unless all arguments are resolved - // and all expressions in window spec are fixed. - case WindowExpression( - u @ UnresolvedWindowFunction(name, children), - windowSpec: WindowSpecDefinition) if shouldResolveFunction(u, windowSpec) => - // First, let's find the window function info. - val windowFunctionInfo: WindowFunctionInfo = - Option(FunctionRegistry.getWindowFunctionInfo(name.toLowerCase)).getOrElse( - throw new AnalysisException(s"Couldn't find window function $name")) - - // Get the class of this function. - // In Hive 0.12, there is no windowFunctionInfo.getFunctionClass. So, we use - // windowFunctionInfo.getfInfo().getFunctionClass for both Hive 0.13 and Hive 0.13.1. - val functionClass = windowFunctionInfo.getFunctionClass() - val newChildren = - // Rank(), DENSE_RANK(), CUME_DIST(), and PERCENT_RANK() do not take explicit - // input parameters and requires implicit parameters, which - // are expressions in Order By clause. - if (classOf[GenericUDAFRank].isAssignableFrom(functionClass)) { - if (children.nonEmpty) { - throw new AnalysisException(s"$name does not take input parameters.") - } - windowSpec.orderSpec.map(_.child) - } else { - children - } - - // If the class is UDAF, we need to use UDAFBridge. - val isUDAFBridgeRequired = - if (classOf[UDAF].isAssignableFrom(functionClass)) { - true - } else { - false - } - - // Create the HiveWindowFunction. For the meaning of isPivotResult, see the doc of - // HiveWindowFunction. - val windowFunction = - HiveWindowFunction( - new HiveFunctionWrapper(functionClass.getName), - windowFunctionInfo.isPivotResult, - isUDAFBridgeRequired, - newChildren) - - // Second, check if the specified window function can accept window definition. - windowSpec.frameSpecification match { - case frame: SpecifiedWindowFrame if !windowFunctionInfo.isSupportsWindow => - // This Hive window function does not support user-speficied window frame. - throw new AnalysisException( - s"Window function $name does not take a frame specification.") - case frame: SpecifiedWindowFrame if windowFunctionInfo.isSupportsWindow && - windowFunctionInfo.isPivotResult => - // These two should not be true at the same time when a window frame is defined. - // If so, throw an exception. - throw new AnalysisException(s"Could not handle Hive window function $name because " + - s"it supports both a user specified window frame and pivot result.") - case _ => // OK - } - // Resolve those UnspecifiedWindowFrame because the physical Window operator still needs - // a window frame specification to work. - val newWindowSpec = windowSpec.frameSpecification match { - case UnspecifiedFrame => - val newWindowFrame = - SpecifiedWindowFrame.defaultWindowFrame( - windowSpec.orderSpec.nonEmpty, - windowFunctionInfo.isSupportsWindow) - WindowSpecDefinition(windowSpec.partitionSpec, windowSpec.orderSpec, newWindowFrame) - case _ => windowSpec - } - - // Finally, we create a WindowExpression with the resolved window function and - // specified window spec. - WindowExpression(windowFunction, newWindowSpec) - } - } -} - -/** - * A [[WindowFunction]] implementation wrapping Hive's window function. - * @param funcWrapper The wrapper for the Hive Window Function. - * @param pivotResult If it is true, the Hive function will return a list of values representing - * the values of the added columns. Otherwise, a single value is returned for - * current row. - * @param isUDAFBridgeRequired If it is true, the function returned by functionWrapper's - * createFunction is UDAF, we need to use GenericUDAFBridge to wrap - * it as a GenericUDAFResolver2. - * @param children Input parameters. - */ -private[hive] case class HiveWindowFunction( - funcWrapper: HiveFunctionWrapper, - pivotResult: Boolean, - isUDAFBridgeRequired: Boolean, - children: Seq[Expression]) extends WindowFunction - with HiveInspectors with Unevaluable { - - // Hive window functions are based on GenericUDAFResolver2. - type UDFType = GenericUDAFResolver2 - - @transient - protected lazy val resolver: GenericUDAFResolver2 = - if (isUDAFBridgeRequired) { - new GenericUDAFBridge(funcWrapper.createFunction[UDAF]()) - } else { - funcWrapper.createFunction[GenericUDAFResolver2]() - } - - @transient - protected lazy val inputInspectors = children.map(toInspector).toArray - - // The GenericUDAFEvaluator used to evaluate the window function. - @transient - protected lazy val evaluator: GenericUDAFEvaluator = { - val parameterInfo = new SimpleGenericUDAFParameterInfo(inputInspectors, false, false) - resolver.getEvaluator(parameterInfo) - } - - // The object inspector of values returned from the Hive window function. - @transient - protected lazy val returnInspector = { - evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inputInspectors) - } - - override val dataType: DataType = - if (!pivotResult) { - inspectorToDataType(returnInspector) - } else { - // If pivotResult is true, we should take the element type out as the data type of this - // function. - inspectorToDataType(returnInspector) match { - case ArrayType(dt, _) => dt - case _ => - sys.error( - s"error resolve the data type of window function ${funcWrapper.functionClassName}") - } - } - - override def nullable: Boolean = true - - @transient - lazy val inputProjection = new InterpretedProjection(children) - - @transient - private var hiveEvaluatorBuffer: AggregationBuffer = _ - // Output buffer. - private var outputBuffer: Any = _ - - @transient - private lazy val inputDataTypes: Array[DataType] = children.map(_.dataType).toArray - - override def init(): Unit = { - evaluator.init(GenericUDAFEvaluator.Mode.COMPLETE, inputInspectors) - } - - // Reset the hiveEvaluatorBuffer and outputPosition - override def reset(): Unit = { - // We create a new aggregation buffer to workaround the bug in GenericUDAFRowNumber. - // Basically, GenericUDAFRowNumberEvaluator.reset calls RowNumberBuffer.init. - // However, RowNumberBuffer.init does not really reset this buffer. - hiveEvaluatorBuffer = evaluator.getNewAggregationBuffer - evaluator.reset(hiveEvaluatorBuffer) - } - - override def prepareInputParameters(input: InternalRow): AnyRef = { - wrap( - inputProjection(input), - inputInspectors, - new Array[AnyRef](children.length), - inputDataTypes) - } - - // Add input parameters for a single row. - override def update(input: AnyRef): Unit = { - evaluator.iterate(hiveEvaluatorBuffer, input.asInstanceOf[Array[AnyRef]]) - } - - override def batchUpdate(inputs: Array[AnyRef]): Unit = { - var i = 0 - while (i < inputs.length) { - evaluator.iterate(hiveEvaluatorBuffer, inputs(i).asInstanceOf[Array[AnyRef]]) + deferredObjects(i).asInstanceOf[DeferredObjectAdapter] + .set(() => children(idx).eval(input)) i += 1 } + unwrapper(function.evaluate(deferredObjects)) } - override def evaluate(): Unit = { - outputBuffer = unwrap(evaluator.evaluate(hiveEvaluatorBuffer), returnInspector) - } - - override def get(index: Int): Any = { - if (!pivotResult) { - // if pivotResult is false, we will get a single value for all rows in the frame. - outputBuffer - } else { - // if pivotResult is true, we will get a ArrayData having the same size with the size - // of the window frame. At here, we will return the result at the position of - // index in the output buffer. - outputBuffer.asInstanceOf[ArrayData].get(index, dataType) - } - } + override def prettyName: String = name override def toString: String = { s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" } - - override def newInstance(): WindowFunction = - new HiveWindowFunction(funcWrapper, pivotResult, isUDAFBridgeRequired, children) } /** * Converts a Hive Generic User Defined Table Generating Function (UDTF) to a - * [[Generator]]. Note that the semantics of Generators do not allow + * `Generator`. Note that the semantics of Generators do not allow * Generators to maintain state in between input rows. Thus UDTFs that rely on partitioning * dependent operations like calls to `close()` before producing output will not operate the same as * in Hive. However, in practice this should not affect compatibility for most sane UDTFs @@ -487,9 +196,10 @@ private[hive] case class HiveWindowFunction( * user defined aggregations, which have clean semantics even in a partitioned execution. */ private[hive] case class HiveGenericUDTF( + name: String, funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) - extends Generator with HiveInspectors with CodegenFallback { + extends Generator with HiveInspectors with CodegenFallback with UserDefinedExpression { @transient protected lazy val function: GenericUDTF = { @@ -510,19 +220,26 @@ private[hive] case class HiveGenericUDTF( @transient protected lazy val collector = new UDTFCollector - override lazy val elementTypes = outputInspector.getAllStructFieldRefs.asScala.map { - field => (inspectorToDataType(field.getFieldObjectInspector), true, field.getFieldName) - } + override lazy val elementSchema = StructType(outputInspector.getAllStructFieldRefs.asScala.map { + field => StructField(field.getFieldName, inspectorToDataType(field.getFieldObjectInspector), + nullable = true) + }) @transient private lazy val inputDataTypes: Array[DataType] = children.map(_.dataType).toArray + @transient + private lazy val wrappers = children.map(x => wrapperFor(toInspector(x), x.dataType)).toArray + + @transient + private lazy val unwrapper = unwrapperFor(outputInspector) + override def eval(input: InternalRow): TraversableOnce[InternalRow] = { outputInspector // Make sure initialized. val inputProjection = new InterpretedProjection(children) - function.process(wrap(inputProjection(input), inputInspectors, udtInput, inputDataTypes)) + function.process(wrap(inputProjection(input), wrappers, udtInput, inputDataTypes)) collector.collectRows() } @@ -533,7 +250,7 @@ private[hive] case class HiveGenericUDTF( // We need to clone the input here because implementations of // GenericUDTF reuse the same object. Luckily they are always an array, so // it is easy to clone. - collected += unwrap(input, outputInspector).asInstanceOf[InternalRow] + collected += unwrapper(input).asInstanceOf[InternalRow] } def collectRows(): Seq[InternalRow] = { @@ -552,19 +269,51 @@ private[hive] case class HiveGenericUDTF( override def toString: String = { s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" } + + override def prettyName: String = name } /** - * Currently we don't support partial aggregation for queries using Hive UDAF, which may hurt - * performance a lot. + * While being evaluated by Spark SQL, the aggregation state of a Hive UDAF may be in the following + * three formats: + * + * 1. An instance of some concrete `GenericUDAFEvaluator.AggregationBuffer` class + * + * This is the native Hive representation of an aggregation state. Hive `GenericUDAFEvaluator` + * methods like `iterate()`, `merge()`, `terminatePartial()`, and `terminate()` use this format. + * We call these methods to evaluate Hive UDAFs. + * + * 2. A Java object that can be inspected using the `ObjectInspector` returned by the + * `GenericUDAFEvaluator.init()` method. + * + * Hive uses this format to produce a serializable aggregation state so that it can shuffle + * partial aggregation results. Whenever we need to convert a Hive `AggregationBuffer` instance + * into a Spark SQL value, we have to convert it to this format first and then do the conversion + * with the help of `ObjectInspector`s. + * + * 3. A Spark SQL value + * + * We use this format for serializing Hive UDAF aggregation states on Spark side. To be more + * specific, we convert `AggregationBuffer`s into equivalent Spark SQL values, write them into + * `UnsafeRow`s, and then retrieve the byte array behind those `UnsafeRow`s as serialization + * results. + * + * We may use the following methods to convert the aggregation state back and forth: + * + * - `wrap()`/`wrapperFor()`: from 3 to 1 + * - `unwrap()`/`unwrapperFor()`: from 1 to 3 + * - `GenericUDAFEvaluator.terminatePartial()`: from 2 to 3 */ private[hive] case class HiveUDAFFunction( + name: String, funcWrapper: HiveFunctionWrapper, children: Seq[Expression], isUDAFBridgeRequired: Boolean = false, mutableAggBufferOffset: Int = 0, inputAggBufferOffset: Int = 0) - extends ImperativeAggregate with HiveInspectors { + extends TypedImperativeAggregate[GenericUDAFEvaluator.AggregationBuffer] + with HiveInspectors + with UserDefinedExpression { override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = copy(mutableAggBufferOffset = newMutableAggBufferOffset) @@ -572,76 +321,154 @@ private[hive] case class HiveUDAFFunction( override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = copy(inputAggBufferOffset = newInputAggBufferOffset) + // Hive `ObjectInspector`s for all child expressions (input parameters of the function). + @transient + private lazy val inputInspectors = children.map(toInspector).toArray + + // Spark SQL data types of input parameters. @transient - private lazy val resolver = - if (isUDAFBridgeRequired) { + private lazy val inputDataTypes: Array[DataType] = children.map(_.dataType).toArray + + private def newEvaluator(): GenericUDAFEvaluator = { + val resolver = if (isUDAFBridgeRequired) { new GenericUDAFBridge(funcWrapper.createFunction[UDAF]()) } else { funcWrapper.createFunction[AbstractGenericUDAFResolver]() } + val parameterInfo = new SimpleGenericUDAFParameterInfo(inputInspectors, false, false) + resolver.getEvaluator(parameterInfo) + } + + // The UDAF evaluator used to consume raw input rows and produce partial aggregation results. + @transient + private lazy val partial1ModeEvaluator = newEvaluator() + + // Hive `ObjectInspector` used to inspect partial aggregation results. @transient - private lazy val inspectors = children.map(toInspector).toArray + private val partialResultInspector = partial1ModeEvaluator.init( + GenericUDAFEvaluator.Mode.PARTIAL1, + inputInspectors + ) + // The UDAF evaluator used to merge partial aggregation results. @transient - private lazy val functionAndInspector = { - val parameterInfo = new SimpleGenericUDAFParameterInfo(inspectors, false, false) - val f = resolver.getEvaluator(parameterInfo) - f -> f.init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors) + private lazy val partial2ModeEvaluator = { + val evaluator = newEvaluator() + evaluator.init(GenericUDAFEvaluator.Mode.PARTIAL2, Array(partialResultInspector)) + evaluator } + // Spark SQL data type of partial aggregation results @transient - private lazy val function = functionAndInspector._1 + private lazy val partialResultDataType = inspectorToDataType(partialResultInspector) + // The UDAF evaluator used to compute the final result from a partial aggregation result objects. @transient - private lazy val returnInspector = functionAndInspector._2 + private lazy val finalModeEvaluator = newEvaluator() + // Hive `ObjectInspector` used to inspect the final aggregation result object. @transient - private[this] var buffer: GenericUDAFEvaluator.AggregationBuffer = _ + private val returnInspector = finalModeEvaluator.init( + GenericUDAFEvaluator.Mode.FINAL, + Array(partialResultInspector) + ) - override def eval(input: InternalRow): Any = unwrap(function.evaluate(buffer), returnInspector) + // Wrapper functions used to wrap Spark SQL input arguments into Hive specific format. + @transient + private lazy val inputWrappers = children.map(x => wrapperFor(toInspector(x), x.dataType)).toArray + // Unwrapper function used to unwrap final aggregation result objects returned by Hive UDAFs into + // Spark SQL specific format. @transient - private lazy val inputProjection = new InterpretedProjection(children) + private lazy val resultUnwrapper = unwrapperFor(returnInspector) @transient - private lazy val cached = new Array[AnyRef](children.length) + private lazy val cached: Array[AnyRef] = new Array[AnyRef](children.length) @transient - private lazy val inputDataTypes: Array[DataType] = children.map(_.dataType).toArray + private lazy val aggBufferSerDe: AggregationBufferSerDe = new AggregationBufferSerDe + + override def nullable: Boolean = true + + override lazy val dataType: DataType = inspectorToDataType(returnInspector) + + override def prettyName: String = name + + override def sql(isDistinct: Boolean): String = { + val distinct = if (isDistinct) "DISTINCT " else " " + s"$name($distinct${children.map(_.sql).mkString(", ")})" + } + + override def createAggregationBuffer(): AggregationBuffer = + partial1ModeEvaluator.getNewAggregationBuffer + + @transient + private lazy val inputProjection = UnsafeProjection.create(children) - // Hive UDAF has its own buffer, so we don't need to occupy a slot in the aggregation - // buffer for it. - override def aggBufferSchema: StructType = StructType(Nil) + override def update(buffer: AggregationBuffer, input: InternalRow): AggregationBuffer = { + partial1ModeEvaluator.iterate( + buffer, wrap(inputProjection(input), inputWrappers, cached, inputDataTypes)) + buffer + } - override def update(_buffer: MutableRow, input: InternalRow): Unit = { - val inputs = inputProjection(input) - function.iterate(buffer, wrap(inputs, inspectors, cached, inputDataTypes)) + override def merge(buffer: AggregationBuffer, input: AggregationBuffer): AggregationBuffer = { + // The 2nd argument of the Hive `GenericUDAFEvaluator.merge()` method is an input aggregation + // buffer in the 3rd format mentioned in the ScalaDoc of this class. Originally, Hive converts + // this `AggregationBuffer`s into this format before shuffling partial aggregation results, and + // calls `GenericUDAFEvaluator.terminatePartial()` to do the conversion. + partial2ModeEvaluator.merge(buffer, partial1ModeEvaluator.terminatePartial(input)) + buffer } - override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = { - throw new UnsupportedOperationException( - "Hive UDAF doesn't support partial aggregate") + override def eval(buffer: AggregationBuffer): Any = { + resultUnwrapper(finalModeEvaluator.terminate(buffer)) } - override def initialize(_buffer: MutableRow): Unit = { - buffer = function.getNewAggregationBuffer + override def serialize(buffer: AggregationBuffer): Array[Byte] = { + // Serializes an `AggregationBuffer` that holds partial aggregation results so that we can + // shuffle it for global aggregation later. + aggBufferSerDe.serialize(buffer) } - override val aggBufferAttributes: Seq[AttributeReference] = Nil + override def deserialize(bytes: Array[Byte]): AggregationBuffer = { + // Deserializes an `AggregationBuffer` from the shuffled partial aggregation phase to prepare + // for global aggregation by merging multiple partial aggregation results within a single group. + aggBufferSerDe.deserialize(bytes) + } - // Note: although this simply copies aggBufferAttributes, this common code can not be placed - // in the superclass because that will lead to initialization ordering issues. - override val inputAggBufferAttributes: Seq[AttributeReference] = Nil + // Helper class used to de/serialize Hive UDAF `AggregationBuffer` objects + private class AggregationBufferSerDe { + private val partialResultUnwrapper = unwrapperFor(partialResultInspector) - // We rely on Hive to check the input data types, so use `AnyDataType` here to bypass our - // catalyst type checking framework. - override def inputTypes: Seq[AbstractDataType] = children.map(_ => AnyDataType) + private val partialResultWrapper = wrapperFor(partialResultInspector, partialResultDataType) - override def nullable: Boolean = true + private val projection = UnsafeProjection.create(Array(partialResultDataType)) - override def supportsPartial: Boolean = false + private val mutableRow = new GenericInternalRow(1) - override val dataType: DataType = inspectorToDataType(returnInspector) -} + def serialize(buffer: AggregationBuffer): Array[Byte] = { + // `GenericUDAFEvaluator.terminatePartial()` converts an `AggregationBuffer` into an object + // that can be inspected by the `ObjectInspector` returned by `GenericUDAFEvaluator.init()`. + // Then we can unwrap it to a Spark SQL value. + mutableRow.update(0, partialResultUnwrapper(partial1ModeEvaluator.terminatePartial(buffer))) + val unsafeRow = projection(mutableRow) + val bytes = ByteBuffer.allocate(unsafeRow.getSizeInBytes) + unsafeRow.writeTo(bytes) + bytes.array() + } + def deserialize(bytes: Array[Byte]): AggregationBuffer = { + // `GenericUDAFEvaluator` doesn't provide any method that is capable to convert an object + // returned by `GenericUDAFEvaluator.terminatePartial()` back to an `AggregationBuffer`. The + // workaround here is creating an initial `AggregationBuffer` first and then merge the + // deserialized object into the buffer. + val buffer = partial2ModeEvaluator.getNewAggregationBuffer + val unsafeRow = new UnsafeRow(1) + unsafeRow.pointTo(bytes, bytes.length) + val partialResult = unsafeRow.get(0, partialResultDataType) + partial2ModeEvaluator.merge(buffer, partialResultWrapper(partialResult)) + buffer + } + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala deleted file mode 100644 index 93c016b6c6c7..000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ /dev/null @@ -1,251 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import java.text.NumberFormat -import java.util.Date - -import scala.collection.mutable - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} -import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} -import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.hadoop.io.Writable -import org.apache.hadoop.mapred._ -import org.apache.hadoop.hive.common.FileUtils - -import org.apache.spark.mapred.SparkHadoopMapRedUtil -import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} -import org.apache.spark.sql.types._ -import org.apache.spark.util.SerializableJobConf - -/** - * Internal helper class that saves an RDD using a Hive OutputFormat. - * It is based on [[SparkHadoopWriter]]. - */ -private[hive] class SparkHiveWriterContainer( - jobConf: JobConf, - fileSinkConf: FileSinkDesc) - extends Logging - with SparkHadoopMapRedUtil - with Serializable { - - private val now = new Date() - private val tableDesc: TableDesc = fileSinkConf.getTableInfo - // Add table properties from storage handler to jobConf, so any custom storage - // handler settings can be set to jobConf - if (tableDesc != null) { - HiveTableUtil.configureJobPropertiesForStorageHandler(tableDesc, jobConf, false) - Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf) - } - protected val conf = new SerializableJobConf(jobConf) - - private var jobID = 0 - private var splitID = 0 - private var attemptID = 0 - private var jID: SerializableWritable[JobID] = null - private var taID: SerializableWritable[TaskAttemptID] = null - - @transient private var writer: FileSinkOperator.RecordWriter = null - @transient protected lazy val committer = conf.value.getOutputCommitter - @transient protected lazy val jobContext = newJobContext(conf.value, jID.value) - @transient private lazy val taskContext = newTaskAttemptContext(conf.value, taID.value) - @transient private lazy val outputFormat = - conf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef, Writable]] - - def driverSideSetup() { - setIDs(0, 0, 0) - setConfParams() - committer.setupJob(jobContext) - } - - def executorSideSetup(jobId: Int, splitId: Int, attemptId: Int) { - setIDs(jobId, splitId, attemptId) - setConfParams() - committer.setupTask(taskContext) - initWriters() - } - - protected def getOutputName: String = { - val numberFormat = NumberFormat.getInstance() - numberFormat.setMinimumIntegerDigits(5) - numberFormat.setGroupingUsed(false) - val extension = Utilities.getFileExtension(conf.value, fileSinkConf.getCompressed, outputFormat) - "part-" + numberFormat.format(splitID) + extension - } - - def getLocalFileWriter(row: InternalRow, schema: StructType): FileSinkOperator.RecordWriter = { - writer - } - - def close() { - // Seems the boolean value passed into close does not matter. - writer.close(false) - commit() - } - - def commitJob() { - committer.commitJob(jobContext) - } - - protected def initWriters() { - // NOTE this method is executed at the executor side. - // For Hive tables without partitions or with only static partitions, only 1 writer is needed. - writer = HiveFileFormatUtils.getHiveRecordWriter( - conf.value, - fileSinkConf.getTableInfo, - conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], - fileSinkConf, - FileOutputFormat.getTaskOutputPath(conf.value, getOutputName), - Reporter.NULL) - } - - protected def commit() { - SparkHadoopMapRedUtil.commitTask(committer, taskContext, jobID, splitID) - } - - private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { - jobID = jobId - splitID = splitId - attemptID = attemptId - - jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobId)) - taID = new SerializableWritable[TaskAttemptID]( - new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID)) - } - - private def setConfParams() { - conf.value.set("mapred.job.id", jID.value.toString) - conf.value.set("mapred.tip.id", taID.value.getTaskID.toString) - conf.value.set("mapred.task.id", taID.value.toString) - conf.value.setBoolean("mapred.task.is.map", true) - conf.value.setInt("mapred.task.partition", splitID) - } -} - -private[hive] object SparkHiveWriterContainer { - def createPathFromString(path: String, conf: JobConf): Path = { - if (path == null) { - throw new IllegalArgumentException("Output path is null") - } - val outputPath = new Path(path) - val fs = outputPath.getFileSystem(conf) - if (outputPath == null || fs == null) { - throw new IllegalArgumentException("Incorrectly formatted output path") - } - outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) - } -} - -private[spark] object SparkHiveDynamicPartitionWriterContainer { - val SUCCESSFUL_JOB_OUTPUT_DIR_MARKER = "mapreduce.fileoutputcommitter.marksuccessfuljobs" -} - -private[spark] class SparkHiveDynamicPartitionWriterContainer( - jobConf: JobConf, - fileSinkConf: FileSinkDesc, - dynamicPartColNames: Array[String]) - extends SparkHiveWriterContainer(jobConf, fileSinkConf) { - - import SparkHiveDynamicPartitionWriterContainer._ - - private val defaultPartName = jobConf.get( - ConfVars.DEFAULTPARTITIONNAME.varname, ConfVars.DEFAULTPARTITIONNAME.defaultStrVal) - - @transient private var writers: mutable.HashMap[String, FileSinkOperator.RecordWriter] = _ - - override protected def initWriters(): Unit = { - // NOTE: This method is executed at the executor side. - // Actual writers are created for each dynamic partition on the fly. - writers = mutable.HashMap.empty[String, FileSinkOperator.RecordWriter] - } - - override def close(): Unit = { - writers.values.foreach(_.close(false)) - commit() - } - - override def commitJob(): Unit = { - // This is a hack to avoid writing _SUCCESS mark file. In lower versions of Hadoop (e.g. 1.0.4), - // semantics of FileSystem.globStatus() is different from higher versions (e.g. 2.4.1) and will - // include _SUCCESS file when glob'ing for dynamic partition data files. - // - // Better solution is to add a step similar to what Hive FileSinkOperator.jobCloseOp does: - // calling something like Utilities.mvFileToFinalPath to cleanup the output directory and then - // load it with loadDynamicPartitions/loadPartition/loadTable. - val oldMarker = conf.value.getBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, true) - conf.value.setBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, false) - super.commitJob() - conf.value.setBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, oldMarker) - } - - override def getLocalFileWriter(row: InternalRow, schema: StructType) - : FileSinkOperator.RecordWriter = { - def convertToHiveRawString(col: String, value: Any): String = { - val raw = String.valueOf(value) - schema(col).dataType match { - case DateType => DateTimeUtils.dateToString(raw.toInt) - case _: DecimalType => BigDecimal(raw).toString() - case _ => raw - } - } - - val nonDynamicPartLen = row.numFields - dynamicPartColNames.length - val dynamicPartPath = dynamicPartColNames.zipWithIndex.map { case (colName, i) => - val rawVal = row.get(nonDynamicPartLen + i, schema(colName).dataType) - val string = if (rawVal == null) null else convertToHiveRawString(colName, rawVal) - val colString = - if (string == null || string.isEmpty) { - defaultPartName - } else { - FileUtils.escapePathName(string, defaultPartName) - } - s"/$colName=$colString" - }.mkString - - def newWriter(): FileSinkOperator.RecordWriter = { - val newFileSinkDesc = new FileSinkDesc( - fileSinkConf.getDirName + dynamicPartPath, - fileSinkConf.getTableInfo, - fileSinkConf.getCompressed) - newFileSinkDesc.setCompressCodec(fileSinkConf.getCompressCodec) - newFileSinkDesc.setCompressType(fileSinkConf.getCompressType) - - // use the path like ${hive_tmp}/_temporary/${attemptId}/ - // to avoid write to the same file when `spark.speculation=true` - val path = FileOutputFormat.getTaskOutputPath( - conf.value, - dynamicPartPath.stripPrefix("/") + "/" + getOutputName) - - HiveFileFormatUtils.getHiveRecordWriter( - conf.value, - fileSinkConf.getTableInfo, - conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], - newFileSinkDesc, - path, - Reporter.NULL) - } - - writers.getOrElseUpdate(dynamicPartPath, newWriter()) - } -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala new file mode 100644 index 000000000000..4d92a6704437 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.orc + +import java.net.URI +import java.util.Properties + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.ql.io.orc._ +import org.apache.hadoop.hive.serde2.objectinspector.{SettableStructObjectInspector, StructObjectInspector} +import org.apache.hadoop.hive.serde2.typeinfo.{StructTypeInfo, TypeInfoUtils} +import org.apache.hadoop.io.{NullWritable, Writable} +import org.apache.hadoop.mapred.{JobConf, OutputFormat => MapRedOutputFormat, RecordWriter, Reporter} +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit} + +import org.apache.spark.TaskContext +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.hive.{HiveInspectors, HiveShim} +import org.apache.spark.sql.sources.{Filter, _} +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +/** + * `FileFormat` for reading ORC files. If this is moved or renamed, please update + * `DataSource`'s backwardCompatibilityMap. + */ +class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable { + + override def shortName(): String = "orc" + + override def toString: String = "ORC" + + override def inferSchema( + sparkSession: SparkSession, + options: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = { + OrcFileOperator.readSchema( + files.map(_.getPath.toUri.toString), + Some(sparkSession.sessionState.newHadoopConf()) + ) + } + + override def prepareWrite( + sparkSession: SparkSession, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf) + + val configuration = job.getConfiguration + + configuration.set(OrcRelation.ORC_COMPRESSION, orcOptions.compressionCodec) + configuration match { + case conf: JobConf => + conf.setOutputFormat(classOf[OrcOutputFormat]) + case conf => + conf.setClass( + "mapred.output.format.class", + classOf[OrcOutputFormat], + classOf[MapRedOutputFormat[_, _]]) + } + + new OutputWriterFactory { + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new OrcOutputWriter(path, dataSchema, context) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + val compressionExtension: String = { + val name = context.getConfiguration.get(OrcRelation.ORC_COMPRESSION) + OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "") + } + + compressionExtension + ".orc" + } + } + } + + override def isSplitable( + sparkSession: SparkSession, + options: Map[String, String], + path: Path): Boolean = { + true + } + + override def buildReader( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { + if (sparkSession.sessionState.conf.orcFilterPushDown) { + // Sets pushed predicates + OrcFilters.createFilter(requiredSchema, filters.toArray).foreach { f => + hadoopConf.set(OrcRelation.SARG_PUSHDOWN, f.toKryo) + hadoopConf.setBoolean(ConfVars.HIVEOPTINDEXFILTER.varname, true) + } + } + + val broadcastedHadoopConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + + (file: PartitionedFile) => { + val conf = broadcastedHadoopConf.value.value + + // SPARK-8501: Empty ORC files always have an empty schema stored in their footer. In this + // case, `OrcFileOperator.readSchema` returns `None`, and we can't read the underlying file + // using the given physical schema. Instead, we simply return an empty iterator. + val maybePhysicalSchema = OrcFileOperator.readSchema(Seq(file.filePath), Some(conf)) + if (maybePhysicalSchema.isEmpty) { + Iterator.empty + } else { + val physicalSchema = maybePhysicalSchema.get + OrcRelation.setRequiredColumns(conf, physicalSchema, requiredSchema) + + val orcRecordReader = { + val job = Job.getInstance(conf) + FileInputFormat.setInputPaths(job, file.filePath) + + val fileSplit = new FileSplit( + new Path(new URI(file.filePath)), file.start, file.length, Array.empty + ) + // Custom OrcRecordReader is used to get + // ObjectInspector during recordReader creation itself and can + // avoid NameNode call in unwrapOrcStructs per file. + // Specifically would be helpful for partitioned datasets. + val orcReader = OrcFile.createReader( + new Path(new URI(file.filePath)), OrcFile.readerOptions(conf)) + new SparkOrcNewRecordReader(orcReader, conf, fileSplit.getStart, fileSplit.getLength) + } + + val recordsIterator = new RecordReaderIterator[OrcStruct](orcRecordReader) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => recordsIterator.close())) + + // Unwraps `OrcStruct`s to `UnsafeRow`s + OrcRelation.unwrapOrcStructs( + conf, + requiredSchema, + Some(orcRecordReader.getObjectInspector.asInstanceOf[StructObjectInspector]), + recordsIterator) + } + } + } +} + +private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration) + extends HiveInspectors { + + def serialize(row: InternalRow): Writable = { + wrapOrcStruct(cachedOrcStruct, structOI, row) + serializer.serialize(cachedOrcStruct, structOI) + } + + private[this] val serializer = { + val table = new Properties() + table.setProperty("columns", dataSchema.fieldNames.mkString(",")) + table.setProperty("columns.types", dataSchema.map(_.dataType.catalogString).mkString(":")) + + val serde = new OrcSerde + serde.initialize(conf, table) + serde + } + + // Object inspector converted from the schema of the relation to be serialized. + private[this] val structOI = { + val typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(dataSchema.catalogString) + OrcStruct.createObjectInspector(typeInfo.asInstanceOf[StructTypeInfo]) + .asInstanceOf[SettableStructObjectInspector] + } + + private[this] val cachedOrcStruct = structOI.create().asInstanceOf[OrcStruct] + + // Wrapper functions used to wrap Spark SQL input arguments into Hive specific format + private[this] val wrappers = dataSchema.zip(structOI.getAllStructFieldRefs().asScala.toSeq).map { + case (f, i) => wrapperFor(i.getFieldObjectInspector, f.dataType) + } + + private[this] def wrapOrcStruct( + struct: OrcStruct, + oi: SettableStructObjectInspector, + row: InternalRow): Unit = { + val fieldRefs = oi.getAllStructFieldRefs + var i = 0 + val size = fieldRefs.size + while (i < size) { + + oi.setStructFieldData( + struct, + fieldRefs.get(i), + wrappers(i)(row.get(i, dataSchema(i).dataType)) + ) + i += 1 + } + } +} + +private[orc] class OrcOutputWriter( + path: String, + dataSchema: StructType, + context: TaskAttemptContext) + extends OutputWriter { + + private[this] val serializer = new OrcSerializer(dataSchema, context.getConfiguration) + + // `OrcRecordWriter.close()` creates an empty file if no rows are written at all. We use this + // flag to decide whether `OrcRecordWriter.close()` needs to be called. + private var recordWriterInstantiated = false + + private lazy val recordWriter: RecordWriter[NullWritable, Writable] = { + recordWriterInstantiated = true + new OrcOutputFormat().getRecordWriter( + new Path(path).getFileSystem(context.getConfiguration), + context.getConfiguration.asInstanceOf[JobConf], + path, + Reporter.NULL + ).asInstanceOf[RecordWriter[NullWritable, Writable]] + } + + override def write(row: InternalRow): Unit = { + recordWriter.write(NullWritable.get(), serializer.serialize(row)) + } + + override def close(): Unit = { + if (recordWriterInstantiated) { + recordWriter.close(Reporter.NULL) + } + } +} + +private[orc] object OrcRelation extends HiveInspectors { + // The references of Hive's classes will be minimized. + val ORC_COMPRESSION = "orc.compress" + + // This constant duplicates `OrcInputFormat.SARG_PUSHDOWN`, which is unfortunately not public. + private[orc] val SARG_PUSHDOWN = "sarg.pushdown" + + // The extensions for ORC compression codecs + val extensionsForCompressionCodecNames = Map( + "NONE" -> "", + "SNAPPY" -> ".snappy", + "ZLIB" -> ".zlib", + "LZO" -> ".lzo") + + def unwrapOrcStructs( + conf: Configuration, + dataSchema: StructType, + maybeStructOI: Option[StructObjectInspector], + iterator: Iterator[Writable]): Iterator[InternalRow] = { + val deserializer = new OrcSerde + val mutableRow = new SpecificInternalRow(dataSchema.map(_.dataType)) + val unsafeProjection = UnsafeProjection.create(dataSchema) + + def unwrap(oi: StructObjectInspector): Iterator[InternalRow] = { + val (fieldRefs, fieldOrdinals) = dataSchema.zipWithIndex.map { + case (field, ordinal) => oi.getStructFieldRef(field.name) -> ordinal + }.unzip + + val unwrappers = fieldRefs.map(unwrapperFor) + + iterator.map { value => + val raw = deserializer.deserialize(value) + var i = 0 + val length = fieldRefs.length + while (i < length) { + val fieldValue = oi.getStructFieldData(raw, fieldRefs(i)) + if (fieldValue == null) { + mutableRow.setNullAt(fieldOrdinals(i)) + } else { + unwrappers(i)(fieldValue, mutableRow, fieldOrdinals(i)) + } + i += 1 + } + unsafeProjection(mutableRow) + } + } + + maybeStructOI.map(unwrap).getOrElse(Iterator.empty) + } + + def setRequiredColumns( + conf: Configuration, physicalSchema: StructType, requestedSchema: StructType): Unit = { + val ids = requestedSchema.map(a => physicalSchema.fieldIndex(a.name): Integer) + val (sortedIDs, sortedNames) = ids.zip(requestedSchema.fieldNames).sorted.unzip + HiveShim.appendReadColumns(conf, sortedIDs, sortedNames) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala index 0f9a1a6ef3b2..5a3fcd7a759c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala @@ -22,16 +22,15 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.io.orc.{OrcFile, Reader} import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector -import org.apache.spark.Logging import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.hive.HiveMetastoreTypes +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.types.StructType -private[orc] object OrcFileOperator extends Logging { +private[hive] object OrcFileOperator extends Logging { /** - * Retrieves a ORC file reader from a given path. The path can point to either a directory or a - * single ORC file. If it points to an directory, it picks any non-empty ORC file within that + * Retrieves an ORC file reader from a given path. The path can point to either a directory or a + * single ORC file. If it points to a directory, it picks any non-empty ORC file within that * directory. * * The reader returned by this method is mainly used for two purposes: @@ -39,13 +38,12 @@ private[orc] object OrcFileOperator extends Logging { * 1. Retrieving file metadata (schema and compression codecs, etc.) * 2. Read the actual file content (in this case, the given path should point to the target file) * - * @note As recorded by SPARK-8501, ORC writes an empty schema (struct<>struct<>) to an * ORC file if the file contains zero rows. This is OK for Hive since the schema of the * table is managed by metastore. But this becomes a problem when reading ORC files * directly from HDFS via Spark SQL, because we have to discover the schema from raw ORC - * files. So this method always tries to find a ORC file whose schema is non-empty, and + * files. So this method always tries to find an ORC file whose schema is non-empty, and * create the result reader from that file. If no such file is found, it returns `None`. - * * @todo Needs to consider all files when schema evolution is taken into account. */ def getFileReader(basePath: String, config: Option[Configuration] = None): Option[Reader] = { @@ -73,16 +71,15 @@ private[orc] object OrcFileOperator extends Logging { } } - def readSchema(path: String, conf: Option[Configuration]): StructType = { - val reader = getFileReader(path, conf).getOrElse { - throw new AnalysisException( - s"Failed to discover schema from ORC files stored in $path. " + - "Probably there are either no ORC files or only empty ORC files.") + def readSchema(paths: Seq[String], conf: Option[Configuration]): Option[StructType] = { + // Take the first file where we can open a valid reader if we can find one. Otherwise just + // return None to indicate we can't infer the schema. + paths.flatMap(getFileReader(_, conf)).headOption.map { reader => + val readerInspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector] + val schema = readerInspector.getTypeName + logDebug(s"Reading schema from file $paths, got Hive schema string: $schema") + CatalystSqlParser.parseDataType(schema).asInstanceOf[StructType] } - val readerInspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector] - val schema = readerInspector.getTypeName - logDebug(s"Reading schema from file $path, got Hive schema string: $schema") - HiveMetastoreTypes.toDataType(schema).asInstanceOf[StructType] } def getObjectInspector( @@ -91,20 +88,14 @@ private[orc] object OrcFileOperator extends Logging { } def listOrcFiles(pathStr: String, conf: Configuration): Seq[Path] = { + // TODO: Check if the paths coming in are already qualified and simplify. val origPath = new Path(pathStr) val fs = origPath.getFileSystem(conf) - val path = origPath.makeQualified(fs.getUri, fs.getWorkingDirectory) val paths = SparkHadoopUtil.get.listLeafStatuses(fs, origPath) - .filterNot(_.isDir) + .filterNot(_.isDirectory) .map(_.getPath) .filterNot(_.getName.startsWith("_")) .filterNot(_.getName.startsWith(".")) - - if (paths == null || paths.isEmpty) { - throw new IllegalArgumentException( - s"orcFileOperator: path $path does not have valid orc files matching the pattern") - } - paths } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala index 27193f54d3a9..d9efd0cb457c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala @@ -17,118 +17,136 @@ package org.apache.spark.sql.hive.orc -import org.apache.hadoop.hive.common.`type`.{HiveChar, HiveDecimal, HiveVarchar} -import org.apache.hadoop.hive.ql.io.sarg.{SearchArgumentFactory, SearchArgument} +import org.apache.hadoop.hive.ql.io.sarg.{SearchArgument, SearchArgumentFactory} import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.Builder -import org.apache.hadoop.hive.serde2.io.DateWritable -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types._ /** - * It may be optimized by push down partial filters. But we are conservative here. - * Because if some filters fail to be parsed, the tree may be corrupted, - * and cannot be used anymore. + * Helper object for building ORC `SearchArgument`s, which are used for ORC predicate push-down. + * + * Due to limitation of ORC `SearchArgument` builder, we had to end up with a pretty weird double- + * checking pattern when converting `And`/`Or`/`Not` filters. + * + * An ORC `SearchArgument` must be built in one pass using a single builder. For example, you can't + * build `a = 1` and `b = 2` first, and then combine them into `a = 1 AND b = 2`. This is quite + * different from the cases in Spark SQL or Parquet, where complex filters can be easily built using + * existing simpler ones. + * + * The annoying part is that, `SearchArgument` builder methods like `startAnd()`, `startOr()`, and + * `startNot()` mutate internal state of the builder instance. This forces us to translate all + * convertible filters with a single builder instance. However, before actually converting a filter, + * we've no idea whether it can be recognized by ORC or not. Thus, when an inconvertible filter is + * found, we may already end up with a builder whose internal state is inconsistent. + * + * For example, to convert an `And` filter with builder `b`, we call `b.startAnd()` first, and then + * try to convert its children. Say we convert `left` child successfully, but find that `right` + * child is inconvertible. Alas, `b.startAnd()` call can't be rolled back, and `b` is inconsistent + * now. + * + * The workaround employed here is that, for `And`/`Or`/`Not`, we first try to convert their + * children with brand new builders, and only do the actual conversion with the right builder + * instance when the children are proven to be convertible. + * + * P.S.: Hive seems to use `SearchArgument` together with `ExprNodeGenericFuncDesc` only. Usage of + * builder methods mentioned above can only be found in test code, where all tested filters are + * known to be convertible. */ private[orc] object OrcFilters extends Logging { - def createFilter(filters: Array[Filter]): Option[SearchArgument] = { + def createFilter(schema: StructType, filters: Array[Filter]): Option[SearchArgument] = { + val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap + + // First, tries to convert each filter individually to see whether it's convertible, and then + // collect all convertible ones to build the final `SearchArgument`. + val convertibleFilters = for { + filter <- filters + _ <- buildSearchArgument(dataTypeMap, filter, SearchArgumentFactory.newBuilder()) + } yield filter + for { - // Combines all filters with `And`s to produce a single conjunction predicate - conjunction <- filters.reduceOption(And) + // Combines all convertible filters using `And` to produce a single conjunction + conjunction <- convertibleFilters.reduceOption(And) // Then tries to build a single ORC `SearchArgument` for the conjunction predicate - builder <- buildSearchArgument(conjunction, SearchArgumentFactory.newBuilder()) + builder <- buildSearchArgument(dataTypeMap, conjunction, SearchArgumentFactory.newBuilder()) } yield builder.build() } - private def buildSearchArgument(expression: Filter, builder: Builder): Option[Builder] = { + private def buildSearchArgument( + dataTypeMap: Map[String, DataType], + expression: Filter, + builder: Builder): Option[Builder] = { def newBuilder = SearchArgumentFactory.newBuilder() - def isSearchableLiteral(value: Any): Boolean = value match { - // These are types recognized by the `SearchArgumentImpl.BuilderImpl.boxLiteral()` method. - case _: String | _: Long | _: Double | _: Byte | _: Short | _: Integer | _: Float => true - case _: DateWritable | _: HiveDecimal | _: HiveChar | _: HiveVarchar => true + def isSearchableType(dataType: DataType): Boolean = dataType match { + // Only the values in the Spark types below can be recognized by + // the `SearchArgumentImpl.BuilderImpl.boxLiteral()` method. + case ByteType | ShortType | FloatType | DoubleType => true + case IntegerType | LongType | StringType | BooleanType => true + case TimestampType | _: DecimalType => true case _ => false } - // lian: I probably missed something here, and had to end up with a pretty weird double-checking - // pattern when converting `And`/`Or`/`Not` filters. - // - // The annoying part is that, `SearchArgument` builder methods like `startAnd()` `startOr()`, - // and `startNot()` mutate internal state of the builder instance. This forces us to translate - // all convertible filters with a single builder instance. However, before actually converting a - // filter, we've no idea whether it can be recognized by ORC or not. Thus, when an inconvertible - // filter is found, we may already end up with a builder whose internal state is inconsistent. - // - // For example, to convert an `And` filter with builder `b`, we call `b.startAnd()` first, and - // then try to convert its children. Say we convert `left` child successfully, but find that - // `right` child is inconvertible. Alas, `b.startAnd()` call can't be rolled back, and `b` is - // inconsistent now. - // - // The workaround employed here is that, for `And`/`Or`/`Not`, we first try to convert their - // children with brand new builders, and only do the actual conversion with the right builder - // instance when the children are proven to be convertible. - // - // P.S.: Hive seems to use `SearchArgument` together with `ExprNodeGenericFuncDesc` only. - // Usage of builder methods mentioned above can only be found in test code, where all tested - // filters are known to be convertible. - expression match { case And(left, right) => - val tryLeft = buildSearchArgument(left, newBuilder) - val tryRight = buildSearchArgument(right, newBuilder) - - val conjunction = for { - _ <- tryLeft - _ <- tryRight - lhs <- buildSearchArgument(left, builder.startAnd()) - rhs <- buildSearchArgument(right, lhs) + // At here, it is not safe to just convert one side if we do not understand the + // other side. Here is an example used to explain the reason. + // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to + // convert b in ('1'). If we only convert a = 2, we will end up with a filter + // NOT(a = 2), which will generate wrong results. + // Pushing one side of AND down is only safe to do at the top level. + // You can see ParquetRelation's initializeLocalJobFunc method as an example. + for { + _ <- buildSearchArgument(dataTypeMap, left, newBuilder) + _ <- buildSearchArgument(dataTypeMap, right, newBuilder) + lhs <- buildSearchArgument(dataTypeMap, left, builder.startAnd()) + rhs <- buildSearchArgument(dataTypeMap, right, lhs) } yield rhs.end() - // For filter `left AND right`, we can still push down `left` even if `right` is not - // convertible, and vice versa. - conjunction - .orElse(tryLeft.flatMap(_ => buildSearchArgument(left, builder))) - .orElse(tryRight.flatMap(_ => buildSearchArgument(right, builder))) - case Or(left, right) => for { - _ <- buildSearchArgument(left, newBuilder) - _ <- buildSearchArgument(right, newBuilder) - lhs <- buildSearchArgument(left, builder.startOr()) - rhs <- buildSearchArgument(right, lhs) + _ <- buildSearchArgument(dataTypeMap, left, newBuilder) + _ <- buildSearchArgument(dataTypeMap, right, newBuilder) + lhs <- buildSearchArgument(dataTypeMap, left, builder.startOr()) + rhs <- buildSearchArgument(dataTypeMap, right, lhs) } yield rhs.end() case Not(child) => for { - _ <- buildSearchArgument(child, newBuilder) - negate <- buildSearchArgument(child, builder.startNot()) + _ <- buildSearchArgument(dataTypeMap, child, newBuilder) + negate <- buildSearchArgument(dataTypeMap, child, builder.startNot()) } yield negate.end() - case EqualTo(attribute, value) if isSearchableLiteral(value) => + // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` + // call is mandatory. ORC `SearchArgument` builder requires that all leaf predicates must be + // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). + + case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(builder.startAnd().equals(attribute, value).end()) - case EqualNullSafe(attribute, value) if isSearchableLiteral(value) => + case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(builder.startAnd().nullSafeEquals(attribute, value).end()) - case LessThan(attribute, value) if isSearchableLiteral(value) => + case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(builder.startAnd().lessThan(attribute, value).end()) - case LessThanOrEqual(attribute, value) if isSearchableLiteral(value) => + case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(builder.startAnd().lessThanEquals(attribute, value).end()) - case GreaterThan(attribute, value) if isSearchableLiteral(value) => + case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(builder.startNot().lessThanEquals(attribute, value).end()) - case GreaterThanOrEqual(attribute, value) if isSearchableLiteral(value) => + case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(builder.startNot().lessThan(attribute, value).end()) - case IsNull(attribute) => + case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => Some(builder.startAnd().isNull(attribute).end()) - case IsNotNull(attribute) => + case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => Some(builder.startNot().isNull(attribute).end()) - case In(attribute, values) if values.forall(isSearchableLiteral) => + case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => Some(builder.startAnd().in(attribute, values.map(_.asInstanceOf[AnyRef]): _*).end()) case _ => None diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala new file mode 100644 index 000000000000..7f94c8c57902 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.orc + +import java.util.Locale + +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.internal.SQLConf + +/** + * Options for the ORC data source. + */ +private[orc] class OrcOptions( + @transient private val parameters: CaseInsensitiveMap[String], + @transient private val sqlConf: SQLConf) + extends Serializable { + + import OrcOptions._ + + def this(parameters: Map[String, String], sqlConf: SQLConf) = + this(CaseInsensitiveMap(parameters), sqlConf) + + /** + * Compression codec to use. + * Acceptable values are defined in [[shortOrcCompressionCodecNames]]. + */ + val compressionCodec: String = { + // `compression`, `orc.compress`, and `spark.sql.orc.compression.codec` are + // in order of precedence from highest to lowest. + val orcCompressionConf = parameters.get(OrcRelation.ORC_COMPRESSION) + val codecName = parameters + .get("compression") + .orElse(orcCompressionConf) + .getOrElse(sqlConf.orcCompressionCodec) + .toLowerCase(Locale.ROOT) + if (!shortOrcCompressionCodecNames.contains(codecName)) { + val availableCodecs = shortOrcCompressionCodecNames.keys.map(_.toLowerCase(Locale.ROOT)) + throw new IllegalArgumentException(s"Codec [$codecName] " + + s"is not available. Available codecs are ${availableCodecs.mkString(", ")}.") + } + shortOrcCompressionCodecNames(codecName) + } +} + +private[orc] object OrcOptions { + // The ORC compression short names + private val shortOrcCompressionCodecNames = Map( + "none" -> "NONE", + "uncompressed" -> "NONE", + "snappy" -> "SNAPPY", + "zlib" -> "ZLIB", + "lzo" -> "LZO") +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala deleted file mode 100644 index 45de56703976..000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ /dev/null @@ -1,335 +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.orc - -import java.util.Properties - -import com.google.common.base.Objects -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.io.orc.{OrcInputFormat, OrcOutputFormat, OrcSerde, OrcSplit, OrcStruct} -import org.apache.hadoop.hive.serde2.objectinspector.SettableStructObjectInspector -import org.apache.hadoop.hive.serde2.typeinfo.{StructTypeInfo, TypeInfoUtils} -import org.apache.hadoop.io.{NullWritable, Writable} -import org.apache.hadoop.mapred.{InputFormat => MapRedInputFormat, JobConf, OutputFormat => MapRedOutputFormat, RecordWriter, Reporter} -import org.apache.hadoop.mapreduce.lib.input.FileInputFormat -import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} - -import org.apache.spark.Logging -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.mapred.SparkHadoopMapRedUtil -import org.apache.spark.rdd.{HadoopRDD, RDD} -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.datasources.PartitionSpec -import org.apache.spark.sql.hive.{HiveContext, HiveInspectors, HiveMetastoreTypes, HiveShim} -import org.apache.spark.sql.sources.{Filter, _} -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{Row, SQLContext} -import org.apache.spark.util.SerializableConfiguration - -private[sql] class DefaultSource extends HadoopFsRelationProvider with DataSourceRegister { - - override def shortName(): String = "orc" - - override def createRelation( - sqlContext: SQLContext, - paths: Array[String], - dataSchema: Option[StructType], - partitionColumns: Option[StructType], - parameters: Map[String, String]): HadoopFsRelation = { - assert( - sqlContext.isInstanceOf[HiveContext], - "The ORC data source can only be used with HiveContext.") - - new OrcRelation(paths, dataSchema, None, partitionColumns, parameters)(sqlContext) - } -} - -private[orc] class OrcOutputWriter( - path: String, - dataSchema: StructType, - context: TaskAttemptContext) - extends OutputWriter with SparkHadoopMapRedUtil with HiveInspectors { - - private val serializer = { - val table = new Properties() - table.setProperty("columns", dataSchema.fieldNames.mkString(",")) - table.setProperty("columns.types", dataSchema.map { f => - HiveMetastoreTypes.toMetastoreType(f.dataType) - }.mkString(":")) - - val serde = new OrcSerde - val configuration = SparkHadoopUtil.get.getConfigurationFromJobContext(context) - serde.initialize(configuration, table) - serde - } - - // Object inspector converted from the schema of the relation to be written. - private val structOI = { - val typeInfo = - TypeInfoUtils.getTypeInfoFromTypeString( - HiveMetastoreTypes.toMetastoreType(dataSchema)) - - OrcStruct.createObjectInspector(typeInfo.asInstanceOf[StructTypeInfo]) - .asInstanceOf[SettableStructObjectInspector] - } - - // `OrcRecordWriter.close()` creates an empty file if no rows are written at all. We use this - // flag to decide whether `OrcRecordWriter.close()` needs to be called. - private var recordWriterInstantiated = false - - private lazy val recordWriter: RecordWriter[NullWritable, Writable] = { - recordWriterInstantiated = true - - val conf = SparkHadoopUtil.get.getConfigurationFromJobContext(context) - val uniqueWriteJobId = conf.get("spark.sql.sources.writeJobUUID") - val taskAttemptId = SparkHadoopUtil.get.getTaskAttemptIDFromTaskAttemptContext(context) - val partition = taskAttemptId.getTaskID.getId - val filename = f"part-r-$partition%05d-$uniqueWriteJobId.orc" - - new OrcOutputFormat().getRecordWriter( - new Path(path, filename).getFileSystem(conf), - conf.asInstanceOf[JobConf], - new Path(path, filename).toString, - Reporter.NULL - ).asInstanceOf[RecordWriter[NullWritable, Writable]] - } - - override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") - - private def wrapOrcStruct( - struct: OrcStruct, - oi: SettableStructObjectInspector, - row: InternalRow): Unit = { - val fieldRefs = oi.getAllStructFieldRefs - var i = 0 - while (i < fieldRefs.size) { - oi.setStructFieldData( - struct, - fieldRefs.get(i), - wrap( - row.get(i, dataSchema(i).dataType), - fieldRefs.get(i).getFieldObjectInspector, - dataSchema(i).dataType)) - i += 1 - } - } - - val cachedOrcStruct = structOI.create().asInstanceOf[OrcStruct] - - override protected[sql] def writeInternal(row: InternalRow): Unit = { - wrapOrcStruct(cachedOrcStruct, structOI, row) - - recordWriter.write( - NullWritable.get(), - serializer.serialize(cachedOrcStruct, structOI)) - } - - override def close(): Unit = { - if (recordWriterInstantiated) { - recordWriter.close(Reporter.NULL) - } - } -} - -private[sql] class OrcRelation( - override val paths: Array[String], - maybeDataSchema: Option[StructType], - maybePartitionSpec: Option[PartitionSpec], - override val userDefinedPartitionColumns: Option[StructType], - parameters: Map[String, String])( - @transient val sqlContext: SQLContext) - extends HadoopFsRelation(maybePartitionSpec) - with Logging { - - private[sql] def this( - paths: Array[String], - maybeDataSchema: Option[StructType], - maybePartitionSpec: Option[PartitionSpec], - parameters: Map[String, String])( - sqlContext: SQLContext) = { - this( - paths, - maybeDataSchema, - maybePartitionSpec, - maybePartitionSpec.map(_.partitionColumns), - parameters)(sqlContext) - } - - override val dataSchema: StructType = maybeDataSchema.getOrElse { - OrcFileOperator.readSchema( - paths.head, Some(sqlContext.sparkContext.hadoopConfiguration)) - } - - override def needConversion: Boolean = false - - override def equals(other: Any): Boolean = other match { - case that: OrcRelation => - paths.toSet == that.paths.toSet && - dataSchema == that.dataSchema && - schema == that.schema && - partitionColumns == that.partitionColumns - case _ => false - } - - override def hashCode(): Int = { - Objects.hashCode( - paths.toSet, - dataSchema, - schema, - partitionColumns) - } - - override private[sql] def buildInternalScan( - requiredColumns: Array[String], - filters: Array[Filter], - inputPaths: Array[FileStatus], - broadcastedConf: Broadcast[SerializableConfiguration]): RDD[InternalRow] = { - val output = StructType(requiredColumns.map(dataSchema(_))).toAttributes - OrcTableScan(output, this, filters, inputPaths).execute() - } - - override def prepareJobForWrite(job: Job): OutputWriterFactory = { - SparkHadoopUtil.get.getConfigurationFromJobContext(job) match { - case conf: JobConf => - conf.setOutputFormat(classOf[OrcOutputFormat]) - case conf => - conf.setClass( - "mapred.output.format.class", - classOf[OrcOutputFormat], - classOf[MapRedOutputFormat[_, _]]) - } - - new OutputWriterFactory { - override def newInstance( - path: String, - dataSchema: StructType, - context: TaskAttemptContext): OutputWriter = { - new OrcOutputWriter(path, dataSchema, context) - } - } - } -} - -private[orc] case class OrcTableScan( - attributes: Seq[Attribute], - @transient relation: OrcRelation, - filters: Array[Filter], - @transient inputPaths: Array[FileStatus]) - extends Logging - with HiveInspectors { - - @transient private val sqlContext = relation.sqlContext - - private def addColumnIds( - output: Seq[Attribute], - relation: OrcRelation, - conf: Configuration): Unit = { - val ids = output.map(a => relation.dataSchema.fieldIndex(a.name): Integer) - val (sortedIds, sortedNames) = ids.zip(attributes.map(_.name)).sorted.unzip - HiveShim.appendReadColumns(conf, sortedIds, sortedNames) - } - - // Transform all given raw `Writable`s into `InternalRow`s. - private def fillObject( - path: String, - conf: Configuration, - iterator: Iterator[Writable], - nonPartitionKeyAttrs: Seq[Attribute]): Iterator[InternalRow] = { - val deserializer = new OrcSerde - val maybeStructOI = OrcFileOperator.getObjectInspector(path, Some(conf)) - val mutableRow = new SpecificMutableRow(attributes.map(_.dataType)) - val unsafeProjection = UnsafeProjection.create(StructType.fromAttributes(nonPartitionKeyAttrs)) - - // SPARK-8501: ORC writes an empty schema ("struct<>") to an ORC file if the file contains zero - // rows, and thus couldn't give a proper ObjectInspector. In this case we just return an empty - // partition since we know that this file is empty. - maybeStructOI.map { soi => - val (fieldRefs, fieldOrdinals) = nonPartitionKeyAttrs.zipWithIndex.map { - case (attr, ordinal) => - soi.getStructFieldRef(attr.name) -> ordinal - }.unzip - val unwrappers = fieldRefs.map(unwrapperFor) - // Map each tuple to a row object - iterator.map { value => - val raw = deserializer.deserialize(value) - var i = 0 - while (i < fieldRefs.length) { - val fieldValue = soi.getStructFieldData(raw, fieldRefs(i)) - if (fieldValue == null) { - mutableRow.setNullAt(fieldOrdinals(i)) - } else { - unwrappers(i)(fieldValue, mutableRow, fieldOrdinals(i)) - } - i += 1 - } - unsafeProjection(mutableRow) - } - }.getOrElse { - Iterator.empty - } - } - - def execute(): RDD[InternalRow] = { - val job = new Job(sqlContext.sparkContext.hadoopConfiguration) - val conf = SparkHadoopUtil.get.getConfigurationFromJobContext(job) - - // Tries to push down filters if ORC filter push-down is enabled - if (sqlContext.conf.orcFilterPushDown) { - OrcFilters.createFilter(filters).foreach { f => - conf.set(OrcTableScan.SARG_PUSHDOWN, f.toKryo) - conf.setBoolean(ConfVars.HIVEOPTINDEXFILTER.varname, true) - } - } - - // Sets requested columns - addColumnIds(attributes, relation, conf) - - if (inputPaths.isEmpty) { - // the input path probably be pruned, return an empty RDD. - return sqlContext.sparkContext.emptyRDD[InternalRow] - } - FileInputFormat.setInputPaths(job, inputPaths.map(_.getPath): _*) - - val inputFormatClass = - classOf[OrcInputFormat] - .asInstanceOf[Class[_ <: MapRedInputFormat[NullWritable, Writable]]] - - val rdd = sqlContext.sparkContext.hadoopRDD( - conf.asInstanceOf[JobConf], - inputFormatClass, - classOf[NullWritable], - classOf[Writable] - ).asInstanceOf[HadoopRDD[NullWritable, Writable]] - - val wrappedConf = new SerializableConfiguration(conf) - - rdd.mapPartitionsWithInputSplit { case (split: OrcSplit, iterator) => - val writableIterator = iterator.map(_._2) - fillObject(split.getPath.toString, wrappedConf.value, writableIterator, attributes) - } - } -} - -private[orc] object OrcTableScan { - // This constant duplicates `OrcInputFormat.SARG_PUSHDOWN`, which is unfortunately not public. - private[orc] val SARG_PUSHDOWN = "sarg.pushdown" -} 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 6883d305cbea..b6be00dbb3a7 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 @@ -18,44 +18,80 @@ package org.apache.spark.sql.hive.test import java.io.File +import java.net.URI import java.util.{Set => JavaSet} import scala.collection.JavaConverters._ import scala.collection.mutable import scala.language.implicitConversions +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.exec.FunctionRegistry -import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe -import org.apache.spark.sql.{SQLContext, SQLConf} -import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.CacheTableCommand +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{SparkSession, SQLContext} +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} +import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} +import org.apache.spark.sql.execution.command.CacheTableCommand import org.apache.spark.sql.hive._ -import org.apache.spark.sql.hive.execution.HiveNativeCommand +import org.apache.spark.sql.hive.client.HiveClient +import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf, WithTestConf} +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.util.{ShutdownHookManager, Utils} -import org.apache.spark.{SparkConf, SparkContext} // SPARK-3729: Test key required to check for initialization errors with config. object TestHive extends TestHiveContext( new SparkContext( - System.getProperty("spark.sql.test.master", "local[32]"), + System.getProperty("spark.sql.test.master", "local[1]"), "TestSQLContext", new SparkConf() .set("spark.sql.test", "") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") .set("spark.sql.hive.metastore.barrierPrefixes", "org.apache.spark.sql.hive.execution.PairSerDe") + .set("spark.sql.warehouse.dir", TestHiveContext.makeWarehouseDir().toURI.getPath) // SPARK-8910 - .set("spark.ui.enabled", "false"))) + .set("spark.ui.enabled", "false") + .set("spark.unsafe.exceptionOnMemoryLeak", "true"))) + + +case class TestHiveVersion(hiveClient: HiveClient) + extends TestHiveContext(TestHive.sparkContext, hiveClient) + + +private[hive] class TestHiveExternalCatalog( + conf: SparkConf, + hadoopConf: Configuration, + hiveClient: Option[HiveClient] = None) + extends HiveExternalCatalog(conf, hadoopConf) with Logging { + + override lazy val client: HiveClient = + hiveClient.getOrElse { + HiveUtils.newClientForMetadata(conf, hadoopConf) + } +} + -trait TestHiveSingleton { - protected val sqlContext: SQLContext = TestHive - protected val hiveContext: TestHiveContext = TestHive +private[hive] class TestHiveSharedState( + sc: SparkContext, + hiveClient: Option[HiveClient] = None) + extends SharedState(sc) { + + override lazy val externalCatalog: TestHiveExternalCatalog = { + new TestHiveExternalCatalog( + sc.conf, + sc.hadoopConfiguration, + hiveClient) + } } + /** * A locally running test instance of Spark's Hive execution engine. * @@ -67,71 +103,137 @@ trait TestHiveSingleton { * hive metastore seems to lead to weird non-deterministic failures. Therefore, the execution of * test cases that rely on TestHive must be serialized. */ -class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { - self => +class TestHiveContext( + @transient override val sparkSession: TestHiveSparkSession) + extends SQLContext(sparkSession) { - import HiveContext._ + /** + * If loadTestTables is false, no test tables are loaded. Note that this flag can only be true + * when running in the JVM, i.e. it needs to be false when calling from Python. + */ + def this(sc: SparkContext, loadTestTables: Boolean = true) { + this(new TestHiveSparkSession(HiveUtils.withHiveExternalCatalog(sc), loadTestTables)) + } - // By clearing the port we force Spark to pick a new one. This allows us to rerun tests - // without restarting the JVM. - System.clearProperty("spark.hostPort") - CommandProcessorFactory.clean(hiveconf) + def this(sc: SparkContext, hiveClient: HiveClient) { + this(new TestHiveSparkSession(HiveUtils.withHiveExternalCatalog(sc), + hiveClient, + loadTestTables = false)) + } - hiveconf.set("hive.plan.serialization.format", "javaXML") + override def newSession(): TestHiveContext = { + new TestHiveContext(sparkSession.newSession()) + } - lazy val warehousePath = Utils.createTempDir(namePrefix = "warehouse-") + def setCacheTables(c: Boolean): Unit = { + sparkSession.setCacheTables(c) + } - lazy val scratchDirPath = { - val dir = Utils.createTempDir(namePrefix = "scratch-") - dir.delete() - dir + def getHiveFile(path: String): File = { + sparkSession.getHiveFile(path) } - private lazy val temporaryConfig = newTemporaryConfiguration() + def loadTestTable(name: String): Unit = { + sparkSession.loadTestTable(name) + } - /** Sets up the system initially or after a RESET command */ - protected override def configure(): Map[String, String] = { - super.configure() ++ temporaryConfig ++ Map( - ConfVars.METASTOREWAREHOUSE.varname -> warehousePath.toURI.toString, - ConfVars.METASTORE_INTEGER_JDO_PUSHDOWN.varname -> "true", - ConfVars.SCRATCHDIR.varname -> scratchDirPath.toURI.toString, - ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY.varname -> "1" - ) + def reset(): Unit = { + sparkSession.reset() } - val testTempDir = Utils.createTempDir() +} - // For some hive test case which contain ${system:test.tmp.dir} - System.setProperty("test.tmp.dir", testTempDir.getCanonicalPath) +/** + * A [[SparkSession]] used in [[TestHiveContext]]. + * + * @param sc SparkContext + * @param existingSharedState optional [[SharedState]] + * @param parentSessionState optional parent [[SessionState]] + * @param loadTestTables if true, load the test tables. They can only be loaded when running + * in the JVM, i.e when calling from Python this flag has to be false. + */ +private[hive] class TestHiveSparkSession( + @transient private val sc: SparkContext, + @transient private val existingSharedState: Option[TestHiveSharedState], + @transient private val parentSessionState: Option[SessionState], + private val loadTestTables: Boolean) + extends SparkSession(sc) with Logging { self => + + def this(sc: SparkContext, loadTestTables: Boolean) { + this( + sc, + existingSharedState = None, + parentSessionState = None, + loadTestTables) + } - /** The location of the compiled hive distribution */ - lazy val hiveHome = envVarToFile("HIVE_HOME") - /** The location of the hive source code. */ - lazy val hiveDevHome = envVarToFile("HIVE_DEV_HOME") + def this(sc: SparkContext, hiveClient: HiveClient, loadTestTables: Boolean) { + this( + sc, + existingSharedState = Some(new TestHiveSharedState(sc, Some(hiveClient))), + parentSessionState = None, + loadTestTables) + } - // Override so we can intercept relative paths and rewrite them to point at hive. - override def runSqlHive(sql: String): Seq[String] = - super.runSqlHive(rewritePaths(substitutor.substitute(this.hiveconf, sql))) + { // set the metastore temporary configuration + val metastoreTempConf = HiveUtils.newTemporaryConfiguration(useInMemoryDerby = false) ++ Map( + ConfVars.METASTORE_INTEGER_JDO_PUSHDOWN.varname -> "true", + // scratch directory used by Hive's metastore client + ConfVars.SCRATCHDIR.varname -> TestHiveContext.makeScratchDir().toURI.toString, + ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY.varname -> "1") - override def executePlan(plan: LogicalPlan): this.QueryExecution = - new this.QueryExecution(plan) + metastoreTempConf.foreach { case (k, v) => + sc.hadoopConfiguration.set(k, v) + } + } - protected[sql] override lazy val conf: SQLConf = new SQLConf { - // The super.getConf(SQLConf.DIALECT) is "sql" by default, we need to set it as "hiveql" - override def dialect: String = super.getConf(SQLConf.DIALECT, "hiveql") - override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) + assume(sc.conf.get(CATALOG_IMPLEMENTATION) == "hive") - clear() + @transient + override lazy val sharedState: TestHiveSharedState = { + existingSharedState.getOrElse(new TestHiveSharedState(sc)) + } - override def clear(): Unit = { - super.clear() + @transient + override lazy val sessionState: SessionState = { + new TestHiveSessionStateBuilder(this, parentSessionState).build() + } - TestHiveContext.overrideConfs.map { - case (key, value) => setConfString(key, value) - } - } + lazy val metadataHive: HiveClient = sharedState.externalCatalog.client.newSession() + + override def newSession(): TestHiveSparkSession = { + new TestHiveSparkSession(sc, Some(sharedState), None, loadTestTables) + } + + override def cloneSession(): SparkSession = { + val result = new TestHiveSparkSession( + sparkContext, + Some(sharedState), + Some(sessionState), + loadTestTables) + result.sessionState // force copy of SessionState + result + } + + private var cacheTables: Boolean = false + + def setCacheTables(c: Boolean): Unit = { + cacheTables = c } + // By clearing the port we force Spark to pick a new one. This allows us to rerun tests + // without restarting the JVM. + System.clearProperty("spark.hostPort") + + // For some hive test case which contain ${system:test.tmp.dir} + System.setProperty("test.tmp.dir", Utils.createTempDir().toURI.getPath) + + /** The location of the compiled hive distribution */ + lazy val hiveHome = envVarToFile("HIVE_HOME") + + /** The location of the hive source code. */ + lazy val hiveDevHome = envVarToFile("HIVE_DEV_HOME") + /** * Returns the value of specified environmental variable as a [[java.io.File]] after checking * to ensure it exists @@ -140,71 +242,34 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { Option(System.getenv(envVar)).map(new File(_)) } - /** - * Replaces relative paths to the parent directory "../" with hiveDevHome since this is how the - * hive test cases assume the system is set up. - */ - private def rewritePaths(cmd: String): String = - if (cmd.toUpperCase contains "LOAD DATA") { - val testDataLocation = - hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath) - cmd.replaceAll("\\.\\./\\.\\./", testDataLocation + "/") - } else { - cmd - } - val hiveFilesTemp = File.createTempFile("catalystHiveFiles", "") hiveFilesTemp.delete() hiveFilesTemp.mkdir() ShutdownHookManager.registerShutdownDeleteDir(hiveFilesTemp) - val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { - new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) + def getHiveFile(path: String): File = { + new File(Thread.currentThread().getContextClassLoader.getResource(path).getFile) + } + + private def quoteHiveFile(path : String) = if (Utils.isWindows) { + getHiveFile(path).getPath.replace('\\', '/') } else { - new File("sql" + File.separator + "hive" + File.separator + "src" + File.separator + "test" + - File.separator + "resources") + getHiveFile(path).getPath } - def getHiveFile(path: String): File = { - val stripped = path.replaceAll("""\.\.\/""", "").replace('/', File.separatorChar) - hiveDevHome - .map(new File(_, stripped)) - .filter(_.exists) - .getOrElse(new File(inRepoTests, stripped)) + def getWarehousePath(): String = { + val tempConf = new SQLConf + sc.conf.getAll.foreach { case (k, v) => tempConf.setConfString(k, v) } + tempConf.warehousePath } val describedTable = "DESCRIBE (\\w+)".r - /** - * Override QueryExecution with special debug workflow. - */ - class QueryExecution(logicalPlan: LogicalPlan) - extends super.QueryExecution(logicalPlan) { - def this(sql: String) = this(parseSql(sql)) - override lazy val analyzed = { - val describedTables = logical match { - case HiveNativeCommand(describedTable(tbl)) => tbl :: Nil - case CacheTableCommand(tbl, _, _) => tbl :: Nil - case _ => Nil - } - - // Make sure any test tables referenced are loaded. - val referencedTables = - describedTables ++ - logical.collect { case UnresolvedRelation(tableIdent, _) => tableIdent.table } - val referencedTestTables = referencedTables.filter(testTables.contains) - logDebug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") - referencedTestTables.foreach(loadTestTable) - // Proceed with analysis. - analyzer.execute(logical) - } - } - case class TestTable(name: String, commands: (() => Unit)*) protected[hive] implicit class SqlCmd(sql: String) { def cmd: () => Unit = { - () => new QueryExecution(sql).stringResult(): Unit + () => new TestHiveQueryExecution(sql).hiveResultString(): Unit } } @@ -219,168 +284,176 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { testTables += (testTable.name -> testTable) } - // The test tables that are defined in the Hive QTestUtil. - // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java - // https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql - @transient - val hiveQTestUtilTables = Seq( - TestTable("src", - "CREATE TABLE src (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' INTO TABLE src".cmd), - TestTable("src1", - "CREATE TABLE src1 (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), - TestTable("srcpart", () => { - runSqlHive( - "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") - for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { - runSqlHive( - s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' + if (loadTestTables) { + // 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 + @transient + val hiveQTestUtilTables: Seq[TestTable] = Seq( + TestTable("src", + "CREATE TABLE src (key INT, value STRING)".cmd, + s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' INTO TABLE src".cmd), + TestTable("src1", + "CREATE TABLE src1 (key INT, value STRING)".cmd, + s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), + TestTable("srcpart", () => { + "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)" + .cmd.apply() + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { + s""" + |LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') - """.stripMargin) - } - }), - TestTable("srcpart1", () => { - runSqlHive("CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") - for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { - runSqlHive( - s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' + """.stripMargin.cmd.apply() + } + }), + TestTable("srcpart1", () => { + "CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)" + .cmd.apply() + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { + s""" + |LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') - """.stripMargin) - } - }), - TestTable("src_thrift", () => { - import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer - import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} - import org.apache.thrift.protocol.TBinaryProtocol + """.stripMargin.cmd.apply() + } + }), + TestTable("src_thrift", () => { + import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer + import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} + import org.apache.thrift.protocol.TBinaryProtocol + + s""" + |CREATE TABLE src_thrift(fake INT) + |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}' + |WITH SERDEPROPERTIES( + | 'serialization.class'='org.apache.spark.sql.hive.test.Complex', + | 'serialization.format'='${classOf[TBinaryProtocol].getName}' + |) + |STORED AS + |INPUTFORMAT '${classOf[SequenceFileInputFormat[_, _]].getName}' + |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_, _]].getName}' + """.stripMargin.cmd.apply() - runSqlHive( s""" - |CREATE TABLE src_thrift(fake INT) - |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}' - |WITH SERDEPROPERTIES( - | 'serialization.class'='org.apache.spark.sql.hive.test.Complex', - | 'serialization.format'='${classOf[TBinaryProtocol].getName}' - |) - |STORED AS - |INPUTFORMAT '${classOf[SequenceFileInputFormat[_, _]].getName}' - |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_, _]].getName}' - """.stripMargin) - - runSqlHive( - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/complex.seq")}' INTO TABLE src_thrift") - }), - TestTable("serdeins", - s"""CREATE TABLE serdeins (key INT, value STRING) - |ROW FORMAT SERDE '${classOf[LazySimpleSerDe].getCanonicalName}' - |WITH SERDEPROPERTIES ('field.delim'='\\t') - """.stripMargin.cmd, - "INSERT OVERWRITE TABLE serdeins SELECT * FROM src".cmd), - TestTable("episodes", - s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) - |STORED AS avro - |TBLPROPERTIES ( - | 'avro.schema.literal'='{ - | "type": "record", - | "name": "episodes", - | "namespace": "testing.hive.avro.serde", - | "fields": [ - | { - | "name": "title", - | "type": "string", - | "doc": "episode title" - | }, - | { - | "name": "air_date", - | "type": "string", - | "doc": "initial date" - | }, - | { - | "name": "doctor", - | "type": "int", - | "doc": "main actor playing the Doctor in episode" - | } - | ] - | }' - |) - """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/episodes.avro")}' INTO TABLE episodes".cmd - ), - // THIS TABLE IS NOT THE SAME AS THE HIVE TEST TABLE episodes_partitioned AS DYNAMIC PARITIONING - // IS NOT YET SUPPORTED - TestTable("episodes_part", - s"""CREATE TABLE episodes_part (title STRING, air_date STRING, doctor INT) - |PARTITIONED BY (doctor_pt INT) - |STORED AS avro - |TBLPROPERTIES ( - | 'avro.schema.literal'='{ - | "type": "record", - | "name": "episodes", - | "namespace": "testing.hive.avro.serde", - | "fields": [ - | { - | "name": "title", - | "type": "string", - | "doc": "episode title" - | }, - | { - | "name": "air_date", - | "type": "string", - | "doc": "initial date" - | }, - | { - | "name": "doctor", - | "type": "int", - | "doc": "main actor playing the Doctor in episode" - | } - | ] - | }' - |) - """.stripMargin.cmd, - // WORKAROUND: Required to pass schema to SerDe for partitioned tables. - // TODO: Pass this automatically from the table to partitions. - s""" - |ALTER TABLE episodes_part SET SERDEPROPERTIES ( - | 'avro.schema.literal'='{ - | "type": "record", - | "name": "episodes", - | "namespace": "testing.hive.avro.serde", - | "fields": [ - | { - | "name": "title", - | "type": "string", - | "doc": "episode title" - | }, - | { - | "name": "air_date", - | "type": "string", - | "doc": "initial date" - | }, - | { - | "name": "doctor", - | "type": "int", - | "doc": "main actor playing the Doctor in episode" - | } - | ] - | }' - |) - """.stripMargin.cmd, - s""" - INSERT OVERWRITE TABLE episodes_part PARTITION (doctor_pt=1) - SELECT title, air_date, doctor FROM episodes - """.cmd + |LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/complex.seq")}' + |INTO TABLE src_thrift + """.stripMargin.cmd.apply() + }), + TestTable("serdeins", + s"""CREATE TABLE serdeins (key INT, value STRING) + |ROW FORMAT SERDE '${classOf[LazySimpleSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ('field.delim'='\\t') + """.stripMargin.cmd, + "INSERT OVERWRITE TABLE serdeins SELECT * FROM src".cmd), + TestTable("episodes", + s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) + |STORED AS avro + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + |LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/episodes.avro")}' + |INTO TABLE episodes + """.stripMargin.cmd ), - TestTable("src_json", - s"""CREATE TABLE src_json (json STRING) STORED AS TEXTFILE - """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/json.txt")}' INTO TABLE src_json".cmd) - ) + // THIS TABLE IS NOT THE SAME AS THE HIVE TEST TABLE episodes_partitioned AS DYNAMIC + // PARTITIONING IS NOT YET SUPPORTED + TestTable("episodes_part", + s"""CREATE TABLE episodes_part (title STRING, air_date STRING, doctor INT) + |PARTITIONED BY (doctor_pt INT) + |STORED AS avro + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + // WORKAROUND: Required to pass schema to SerDe for partitioned tables. + // TODO: Pass this automatically from the table to partitions. + s""" + |ALTER TABLE episodes_part SET SERDEPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + INSERT OVERWRITE TABLE episodes_part PARTITION (doctor_pt=1) + SELECT title, air_date, doctor FROM episodes + """.cmd + ), + TestTable("src_json", + s"""CREATE TABLE src_json (json STRING) STORED AS TEXTFILE + """.stripMargin.cmd, + s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/json.txt")}' INTO TABLE src_json".cmd) + ) - hiveQTestUtilTables.foreach(registerTestTable) + hiveQTestUtilTables.foreach(registerTestTable) + } private val loadedTables = new collection.mutable.HashSet[String] - var cacheTables: Boolean = false + def getLoadedTables: collection.mutable.HashSet[String] = loadedTables + def loadTestTable(name: String) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infinite mutually recursive table loading. @@ -388,10 +461,20 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { logDebug(s"Loading test table $name") val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) - createCmds.foreach(_()) + + // test tables are loaded lazily, so they may be loaded in the middle a query execution which + // has already set the execution id. + if (sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) == null) { + // We don't actually have a `QueryExecution` here, use a fake one instead. + SQLExecution.withNewExecutionId(this, new QueryExecution(this, OneRowRelation())) { + createCmds.foreach(_()) + } + } else { + createCmds.foreach(_()) + } if (cacheTables) { - cacheTable(name) + new SQLContext(self).cacheTable(name) } } } @@ -410,53 +493,88 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { try { // HACK: Hive is too noisy by default. org.apache.log4j.LogManager.getCurrentLoggers.asScala.foreach { log => - log.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN) + val logger = log.asInstanceOf[org.apache.log4j.Logger] + if (!logger.getName.contains("org.apache.spark")) { + logger.setLevel(org.apache.log4j.Level.WARN) + } } - cacheManager.clearCache() + // Clean out the Hive warehouse between each suite + val warehouseDir = new File(new URI(sparkContext.conf.get("spark.sql.warehouse.dir")).getPath) + Utils.deleteRecursively(warehouseDir) + warehouseDir.mkdir() + + sharedState.cacheManager.clearCache() loadedTables.clear() - catalog.cachedDataSourceTables.invalidateAll() - catalog.client.reset() - catalog.unregisterAllTables() + sessionState.catalog.reset() + metadataHive.reset() - FunctionRegistry.getFunctionNames.asScala.filterNot(originalUDFs.contains(_)). - foreach { udfName => FunctionRegistry.unregisterTemporaryUDF(udfName) } + // HDFS root scratch dir requires the write all (733) permission. For each connecting user, + // an HDFS scratch dir: ${hive.exec.scratchdir}/ is created, with + // ${hive.scratch.dir.permission}. To resolve the permission issue, the simplest way is to + // delete it. Later, it will be re-created with the right permission. + val location = new Path(sc.hadoopConfiguration.get(ConfVars.SCRATCHDIR.varname)) + val fs = location.getFileSystem(sc.hadoopConfiguration) + fs.delete(location, true) // Some tests corrupt this value on purpose, which breaks the RESET call below. - hiveconf.set("fs.default.name", new File(".").toURI.toString) + sessionState.conf.setConfString("fs.defaultFS", new File(".").toURI.toString) // It is important that we RESET first as broken hooks that might have been set could break // other sql exec here. - executionHive.runSqlHive("RESET") metadataHive.runSqlHive("RESET") // For some reason, RESET does not reset the following variables... // https://issues.apache.org/jira/browse/HIVE-9004 - runSqlHive("set hive.table.parameters.default=") - runSqlHive("set datanucleus.cache.collections=true") - runSqlHive("set datanucleus.cache.collections.lazy=true") + metadataHive.runSqlHive("set hive.table.parameters.default=") + metadataHive.runSqlHive("set datanucleus.cache.collections=true") + metadataHive.runSqlHive("set datanucleus.cache.collections.lazy=true") // Lots of tests fail if we do not change the partition whitelist from the default. - runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*") - - configure().foreach { - case (k, v) => - metadataHive.runSqlHive(s"SET $k=$v") - } - defaultOverrides() + metadataHive.runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*") - runSqlHive("USE default") - - // Just loading src makes a lot of tests pass. This is because some tests do something like - // drop an index on src at the beginning. Since we just pass DDL to hive this bypasses our - // Analyzer and thus the test table auto-loading mechanism. - // Remove after we handle more DDL operations natively. - loadTestTable("src") - loadTestTable("srcpart") + sessionState.catalog.setCurrentDatabase("default") } catch { case e: Exception => logError("FATAL ERROR: Failed to reset TestDB state.", e) } } + +} + + +private[hive] class TestHiveQueryExecution( + sparkSession: TestHiveSparkSession, + logicalPlan: LogicalPlan) + extends QueryExecution(sparkSession, logicalPlan) with Logging { + + def this(sparkSession: TestHiveSparkSession, sql: String) { + this(sparkSession, sparkSession.sessionState.sqlParser.parsePlan(sql)) + } + + def this(sql: String) { + this(TestHive.sparkSession, sql) + } + + override lazy val analyzed: LogicalPlan = { + val describedTables = logical match { + case CacheTableCommand(tbl, _, _) => tbl.table :: Nil + case _ => Nil + } + + // Make sure any test tables referenced are loaded. + val referencedTables = + describedTables ++ + logical.collect { case UnresolvedRelation(tableIdent) => tableIdent.table } + val resolver = sparkSession.sessionState.conf.resolver + val referencedTestTables = sparkSession.testTables.keys.filter { testTable => + referencedTables.exists(resolver(_, testTable)) + } + logDebug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") + referencedTestTables.foreach(sparkSession.loadTestTable) + // Proceed with analysis. + sparkSession.sessionState.analyzer.execute(logical) + } } + private[hive] object TestHiveContext { /** @@ -467,4 +585,32 @@ private[hive] object TestHiveContext { // Fewer shuffle partitions to speed up testing. SQLConf.SHUFFLE_PARTITIONS.key -> "5" ) + + def makeWarehouseDir(): File = { + val warehouseDir = Utils.createTempDir(namePrefix = "warehouse") + warehouseDir.delete() + warehouseDir + } + + def makeScratchDir(): File = { + val scratchDir = Utils.createTempDir(namePrefix = "scratch") + scratchDir.delete() + scratchDir + } + +} + +private[sql] class TestHiveSessionStateBuilder( + session: SparkSession, + state: Option[SessionState]) + extends HiveSessionStateBuilder(session, state) + with WithTestConf { + + override def overrideConfs: Map[String, String] = TestHiveContext.overrideConfs + + override def createQueryExecution: (LogicalPlan) => QueryExecution = { plan => + new TestHiveQueryExecution(session.asInstanceOf[TestHiveSparkSession], plan) + } + + override protected def newBuilder: NewBuilder = new TestHiveSessionStateBuilder(_, _) } diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java index b4bf9eef8fca..636ce10da373 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java @@ -26,21 +26,19 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.*; import org.apache.spark.sql.expressions.Window; import org.apache.spark.sql.expressions.UserDefinedAggregateFunction; import static org.apache.spark.sql.functions.*; import org.apache.spark.sql.hive.test.TestHive$; -import org.apache.spark.sql.hive.aggregate.MyDoubleSum; +import test.org.apache.spark.sql.MyDoubleSum; public class JavaDataFrameSuite { - private transient JavaSparkContext sc; - private transient HiveContext hc; + private transient SQLContext hc; - DataFrame df; + Dataset df; - private static void checkAnswer(DataFrame actual, List expected) { + private static void checkAnswer(Dataset actual, List expected) { String errorMessage = QueryTest$.MODULE$.checkAnswer(actual, expected); if (errorMessage != null) { Assert.fail(errorMessage); @@ -50,14 +48,12 @@ private static void checkAnswer(DataFrame actual, List expected) { @Before public void setUp() throws IOException { hc = TestHive$.MODULE$; - sc = new JavaSparkContext(hc.sparkContext()); - List jsonObjects = new ArrayList<>(10); for (int i = 0; i < 10; i++) { jsonObjects.add("{\"key\":" + i + ", \"value\":\"str" + i + "\"}"); } - df = hc.read().json(sc.parallelize(jsonObjects)); - df.registerTempTable("window_table"); + df = hc.read().json(hc.createDataset(jsonObjects, Encoders.STRING())); + df.createOrReplaceTempView("window_table"); } @After @@ -82,12 +78,12 @@ public void saveTableAndQueryIt() { @Test public void testUDAF() { - DataFrame df = hc.range(0, 100).unionAll(hc.range(0, 100)).select(col("id").as("value")); + Dataset df = hc.range(0, 100).union(hc.range(0, 100)).select(col("id").as("value")); UserDefinedAggregateFunction udaf = new MyDoubleSum(); UserDefinedAggregateFunction registeredUDAF = hc.udf().register("mydoublesum", udaf); // Create Columns for the UDAF. For now, callUDF does not take an argument to specific if // we want to use distinct aggregation. - DataFrame aggregatedDF = + Dataset aggregatedDF = df.groupBy() .agg( udaf.distinct(col("value")), 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 8c4af1b8eaf4..25bd4d0017bd 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 @@ -31,11 +31,12 @@ 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.Dataset; +import org.apache.spark.sql.Encoders; import org.apache.spark.sql.QueryTest$; import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.hive.test.TestHive$; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructField; @@ -46,15 +47,14 @@ public class JavaMetastoreDataSourcesSuite { private transient JavaSparkContext sc; - private transient HiveContext sqlContext; + private transient SQLContext sqlContext; - String originalDefaultSource; File path; Path hiveManagedPath; FileSystem fs; - DataFrame df; + Dataset df; - private static void checkAnswer(DataFrame actual, List expected) { + private static void checkAnswer(Dataset actual, List expected) { String errorMessage = QueryTest$.MODULE$.checkAnswer(actual, expected); if (errorMessage != null) { Assert.fail(errorMessage); @@ -66,26 +66,23 @@ 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( - new TableIdentifier("javaSavedTable"))); + HiveSessionCatalog catalog = (HiveSessionCatalog) sqlContext.sessionState().catalog(); + hiveManagedPath = new Path(catalog.defaultTablePath(new TableIdentifier("javaSavedTable"))); fs = hiveManagedPath.getFileSystem(sc.hadoopConfiguration()); - if (fs.exists(hiveManagedPath)){ - fs.delete(hiveManagedPath, true); - } + 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.read().json(rdd); - df.registerTempTable("jsonTable"); + Dataset ds = sqlContext.createDataset(jsonObjects, Encoders.STRING()); + df = sqlContext.read().json(ds); + df.createOrReplaceTempView("jsonTable"); } @After @@ -111,7 +108,7 @@ public void saveExternalTableAndQueryIt() { sqlContext.sql("SELECT * FROM javaSavedTable"), df.collectAsList()); - DataFrame loadedDF = + Dataset loadedDF = sqlContext.createExternalTable("externalTable", "org.apache.spark.sql.json", options); checkAnswer(loadedDF, df.collectAsList()); @@ -137,7 +134,7 @@ public void saveExternalTableWithSchemaAndQueryIt() { List fields = new ArrayList<>(); fields.add(DataTypes.createStructField("b", DataTypes.StringType, true)); StructType schema = DataTypes.createStructType(fields); - DataFrame loadedDF = + Dataset loadedDF = sqlContext.createExternalTable("externalTable", "org.apache.spark.sql.json", schema, options); checkAnswer( diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFRawList.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFRawList.java new file mode 100644 index 000000000000..8211cbf16f7b --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFRawList.java @@ -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.hive.execution; + +import org.apache.hadoop.hive.ql.exec.UDF; + +import java.util.Collections; +import java.util.List; + +/** + * UDF that returns a raw (non-parameterized) java List. + */ +public class UDFRawList extends UDF { + @SuppressWarnings("rawtypes") + public List evaluate(Object o) { + return Collections.singletonList("data1"); + } +} diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFRawMap.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFRawMap.java new file mode 100644 index 000000000000..58c81f9945d7 --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFRawMap.java @@ -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.hive.execution; + +import org.apache.hadoop.hive.ql.exec.UDF; + +import java.util.Collections; +import java.util.Map; + +/** + * UDF that returns a raw (non-parameterized) java Map. + */ +public class UDFRawMap extends UDF { + @SuppressWarnings("rawtypes") + public Map evaluate(Object o) { + return Collections.singletonMap("a", "1"); + } +} diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToIntIntMap.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToIntIntMap.java index b3e8bcbbd822..91b9673a0920 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToIntIntMap.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToIntIntMap.java @@ -23,13 +23,13 @@ import java.util.Map; public class UDFToIntIntMap extends UDF { - public Map evaluate(Object o) { - return new HashMap() { - { - put(1, 1); - put(2, 1); - put(3, 1); - } - }; - } + public Map evaluate(Object o) { + return new HashMap() { + { + put(1, 1); + put(2, 1); + put(3, 1); + } + }; + } } diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToListInt.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToListInt.java index 67576a72f198..66fc8c09fd17 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToListInt.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToListInt.java @@ -19,11 +19,11 @@ import org.apache.hadoop.hive.ql.exec.UDF; +import java.util.ArrayList; import java.util.Arrays; -import java.util.List; public class UDFToListInt extends UDF { - public List evaluate(Object o) { - return Arrays.asList(1, 2, 3); - } + public ArrayList evaluate(Object o) { + return new ArrayList<>(Arrays.asList(1, 2, 3)); + } } diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToListMapStringListInt.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToListMapStringListInt.java new file mode 100644 index 000000000000..d16f27221d17 --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToListMapStringListInt.java @@ -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.hive.execution; + +import org.apache.hadoop.hive.ql.exec.UDF; + +import java.util.*; + +/** + * UDF that returns a nested list of maps that uses a string as its key and a list of ints as its + * values. + */ +public class UDFToListMapStringListInt extends UDF { + public List>> evaluate(Object o) { + final Map> map = new HashMap<>(); + map.put("a", Arrays.asList(1, 2)); + map.put("b", Arrays.asList(3, 4)); + return Collections.singletonList(map); + } +} diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToListString.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToListString.java index f02395cbba88..5185b47a5615 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToListString.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToListString.java @@ -23,7 +23,7 @@ import java.util.List; public class UDFToListString extends UDF { - public List evaluate(Object o) { - return Arrays.asList("data1", "data2", "data3"); - } + public List evaluate(Object o) { + return Arrays.asList("data1", "data2", "data3"); + } } diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToStringIntMap.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToStringIntMap.java index 9eea5c9a881f..b7ca60e036f7 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToStringIntMap.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFToStringIntMap.java @@ -20,16 +20,15 @@ import org.apache.hadoop.hive.ql.exec.UDF; import java.util.HashMap; -import java.util.Map; public class UDFToStringIntMap extends UDF { - public Map evaluate(Object o) { - return new HashMap() { - { - put("key1", 1); - put("key2", 2); - put("key3", 3); - } - }; - } + public HashMap evaluate(Object o) { + return new HashMap() { + { + put("key1", 1); + put("key2", 2); + put("key3", 3); + } + }; + } } diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFWildcardList.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFWildcardList.java new file mode 100644 index 000000000000..717e1117b99a --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFWildcardList.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive.execution; + +import org.apache.hadoop.hive.ql.exec.UDF; + +import java.util.Collections; +import java.util.List; + +/** + * UDF that returns a raw (non-parameterized) java List. + */ +public class UDFWildcardList extends UDF { + public List evaluate(Object o) { + return Collections.singletonList("data1"); + } +} diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java index e010112bb932..a8cbd4fab15b 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java @@ -39,7 +39,7 @@ * does not contain union fields that are not supported by Spark SQL. */ -@SuppressWarnings({"ALL", "unchecked"}) +@SuppressWarnings("all") public class Complex implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Complex"); @@ -50,7 +50,7 @@ public class Complex implements org.apache.thrift.TBase, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + private static final Map, SchemeFactory> schemes = new HashMap<>(); static { schemes.put(StandardScheme.class, new ComplexStandardSchemeFactory()); schemes.put(TupleScheme.class, new ComplexTupleSchemeFactory()); @@ -72,7 +72,7 @@ public enum _Fields implements org.apache.thrift.TFieldIdEnum { LINT_STRING((short)5, "lintString"), M_STRING_STRING((short)6, "mStringString"); - private static final Map byName = new HashMap(); + private static final Map byName = new HashMap<>(); static { for (_Fields field : EnumSet.allOf(_Fields.class)) { @@ -141,7 +141,7 @@ public String getFieldName() { private byte __isset_bitfield = 0; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { - Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<>(_Fields.class); tmpMap.put(_Fields.AINT, new org.apache.thrift.meta_data.FieldMetaData("aint", org.apache.thrift.TFieldRequirementType.DEFAULT, new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); tmpMap.put(_Fields.A_STRING, new org.apache.thrift.meta_data.FieldMetaData("aString", org.apache.thrift.TFieldRequirementType.DEFAULT, @@ -194,28 +194,28 @@ public Complex(Complex other) { this.aString = other.aString; } if (other.isSetLint()) { - List __this__lint = new ArrayList(); + List __this__lint = new ArrayList<>(); for (Integer other_element : other.lint) { __this__lint.add(other_element); } this.lint = __this__lint; } if (other.isSetLString()) { - List __this__lString = new ArrayList(); + List __this__lString = new ArrayList<>(); for (String other_element : other.lString) { __this__lString.add(other_element); } this.lString = __this__lString; } if (other.isSetLintString()) { - List __this__lintString = new ArrayList(); + List __this__lintString = new ArrayList<>(); for (IntString other_element : other.lintString) { __this__lintString.add(new IntString(other_element)); } this.lintString = __this__lintString; } if (other.isSetMStringString()) { - Map __this__mStringString = new HashMap(); + Map __this__mStringString = new HashMap<>(); for (Map.Entry other_element : other.mStringString.entrySet()) { String other_element_key = other_element.getKey(); @@ -339,7 +339,7 @@ public java.util.Iterator getLStringIterator() { public void addToLString(String elem) { if (this.lString == null) { - this.lString = new ArrayList(); + this.lString = new ArrayList<>(); } this.lString.add(elem); } @@ -411,7 +411,7 @@ public int getMStringStringSize() { public void putToMStringString(String key, String val) { if (this.mStringString == null) { - this.mStringString = new HashMap(); + this.mStringString = new HashMap<>(); } this.mStringString.put(key, val); } @@ -489,6 +489,7 @@ public void setFieldValue(_Fields field, Object value) { } break; + default: } } @@ -512,6 +513,7 @@ public Object getFieldValue(_Fields field) { case M_STRING_STRING: return getMStringString(); + default: } throw new IllegalStateException(); } @@ -535,75 +537,91 @@ public boolean isSet(_Fields field) { return isSetLintString(); case M_STRING_STRING: return isSetMStringString(); + default: } throw new IllegalStateException(); } @Override public boolean equals(Object that) { - if (that == null) + if (that == null) { return false; - if (that instanceof Complex) + } + if (that instanceof Complex) { return this.equals((Complex)that); + } return false; } public boolean equals(Complex that) { - if (that == null) + if (that == null) { return false; + } boolean this_present_aint = true; boolean that_present_aint = true; if (this_present_aint || that_present_aint) { - if (!(this_present_aint && that_present_aint)) + if (!(this_present_aint && that_present_aint)) { return false; - if (this.aint != that.aint) + } + if (this.aint != that.aint) { return false; + } } boolean this_present_aString = true && this.isSetAString(); boolean that_present_aString = true && that.isSetAString(); if (this_present_aString || that_present_aString) { - if (!(this_present_aString && that_present_aString)) + if (!(this_present_aString && that_present_aString)) { return false; - if (!this.aString.equals(that.aString)) + } + if (!this.aString.equals(that.aString)) { return false; + } } boolean this_present_lint = true && this.isSetLint(); boolean that_present_lint = true && that.isSetLint(); if (this_present_lint || that_present_lint) { - if (!(this_present_lint && that_present_lint)) + if (!(this_present_lint && that_present_lint)) { return false; - if (!this.lint.equals(that.lint)) + } + if (!this.lint.equals(that.lint)) { return false; + } } boolean this_present_lString = true && this.isSetLString(); boolean that_present_lString = true && that.isSetLString(); if (this_present_lString || that_present_lString) { - if (!(this_present_lString && that_present_lString)) + if (!(this_present_lString && that_present_lString)) { return false; - if (!this.lString.equals(that.lString)) + } + if (!this.lString.equals(that.lString)) { return false; + } } boolean this_present_lintString = true && this.isSetLintString(); boolean that_present_lintString = true && that.isSetLintString(); if (this_present_lintString || that_present_lintString) { - if (!(this_present_lintString && that_present_lintString)) + if (!(this_present_lintString && that_present_lintString)) { return false; - if (!this.lintString.equals(that.lintString)) + } + if (!this.lintString.equals(that.lintString)) { return false; + } } boolean this_present_mStringString = true && this.isSetMStringString(); boolean that_present_mStringString = true && that.isSetMStringString(); if (this_present_mStringString || that_present_mStringString) { - if (!(this_present_mStringString && that_present_mStringString)) + if (!(this_present_mStringString && that_present_mStringString)) { return false; - if (!this.mStringString.equals(that.mStringString)) + } + if (!this.mStringString.equals(that.mStringString)) { return false; + } } return true; @@ -615,33 +633,39 @@ public int hashCode() { boolean present_aint = true; builder.append(present_aint); - if (present_aint) + if (present_aint) { builder.append(aint); + } boolean present_aString = true && (isSetAString()); builder.append(present_aString); - if (present_aString) + if (present_aString) { builder.append(aString); + } boolean present_lint = true && (isSetLint()); builder.append(present_lint); - if (present_lint) + if (present_lint) { builder.append(lint); + } boolean present_lString = true && (isSetLString()); builder.append(present_lString); - if (present_lString) + if (present_lString) { builder.append(lString); + } boolean present_lintString = true && (isSetLintString()); builder.append(present_lintString); - if (present_lintString) + if (present_lintString) { builder.append(lintString); + } boolean present_mStringString = true && (isSetMStringString()); builder.append(present_mStringString); - if (present_mStringString) + if (present_mStringString) { builder.append(mStringString); + } return builder.toHashCode(); } @@ -737,7 +761,9 @@ public String toString() { sb.append("aint:"); sb.append(this.aint); first = false; - if (!first) sb.append(", "); + if (!first) { + sb.append(", "); + } sb.append("aString:"); if (this.aString == null) { sb.append("null"); @@ -745,7 +771,9 @@ public String toString() { sb.append(this.aString); } first = false; - if (!first) sb.append(", "); + if (!first) { + sb.append(", "); + } sb.append("lint:"); if (this.lint == null) { sb.append("null"); @@ -753,7 +781,9 @@ public String toString() { sb.append(this.lint); } first = false; - if (!first) sb.append(", "); + if (!first) { + sb.append(", "); + } sb.append("lString:"); if (this.lString == null) { sb.append("null"); @@ -761,7 +791,9 @@ public String toString() { sb.append(this.lString); } first = false; - if (!first) sb.append(", "); + if (!first) { + sb.append(", "); + } sb.append("lintString:"); if (this.lintString == null) { sb.append("null"); @@ -769,7 +801,9 @@ public String toString() { sb.append(this.lintString); } first = false; - if (!first) sb.append(", "); + if (!first) { + sb.append(", "); + } sb.append("mStringString:"); if (this.mStringString == null) { sb.append("null"); @@ -842,7 +876,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Complex struct) thr if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { org.apache.thrift.protocol.TList _list0 = iprot.readListBegin(); - struct.lint = new ArrayList(_list0.size); + struct.lint = new ArrayList<>(_list0.size); for (int _i1 = 0; _i1 < _list0.size; ++_i1) { int _elem2; // required @@ -860,7 +894,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Complex struct) thr if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { org.apache.thrift.protocol.TList _list3 = iprot.readListBegin(); - struct.lString = new ArrayList(_list3.size); + struct.lString = new ArrayList<>(_list3.size); for (int _i4 = 0; _i4 < _list3.size; ++_i4) { String _elem5; // required @@ -878,7 +912,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, Complex struct) thr if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { org.apache.thrift.protocol.TList _list6 = iprot.readListBegin(); - struct.lintString = new ArrayList(_list6.size); + struct.lintString = new ArrayList<>(_list6.size); for (int _i7 = 0; _i7 < _list6.size; ++_i7) { IntString _elem8; // required @@ -1080,7 +1114,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Complex struct) thro if (incoming.get(2)) { { org.apache.thrift.protocol.TList _list21 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32()); - struct.lint = new ArrayList(_list21.size); + struct.lint = new ArrayList<>(_list21.size); for (int _i22 = 0; _i22 < _list21.size; ++_i22) { int _elem23; // required @@ -1093,7 +1127,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Complex struct) thro if (incoming.get(3)) { { org.apache.thrift.protocol.TList _list24 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.lString = new ArrayList(_list24.size); + struct.lString = new ArrayList<>(_list24.size); for (int _i25 = 0; _i25 < _list24.size; ++_i25) { String _elem26; // required @@ -1106,7 +1140,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Complex struct) thro if (incoming.get(4)) { { org.apache.thrift.protocol.TList _list27 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.lintString = new ArrayList(_list27.size); + struct.lintString = new ArrayList<>(_list27.size); for (int _i28 = 0; _i28 < _list27.size; ++_i28) { IntString _elem29; // required @@ -1120,7 +1154,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, Complex struct) thro if (incoming.get(5)) { { org.apache.thrift.protocol.TMap _map30 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.mStringString = new HashMap(2*_map30.size); + struct.mStringString = new HashMap<>(2*_map30.size); for (int _i31 = 0; _i31 < _map30.size; ++_i31) { String _key32; // required diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/test/TestHiveSingleton.scala b/sql/hive/src/test/java/org/apache/spark/sql/hive/test/TestHiveSingleton.scala new file mode 100644 index 000000000000..df7988f542b7 --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/test/TestHiveSingleton.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.test + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.hive.HiveExternalCatalog +import org.apache.spark.sql.hive.client.HiveClient + + +trait TestHiveSingleton extends SparkFunSuite with BeforeAndAfterAll { + protected val spark: SparkSession = TestHive.sparkSession + protected val hiveContext: TestHiveContext = TestHive + protected val hiveClient: HiveClient = + spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client + + protected override def afterAll(): Unit = { + try { + hiveContext.reset() + } finally { + super.afterAll() + } + } + +} diff --git a/sql/hive/src/test/resources/avroDecimal/decimal.avro b/sql/hive/src/test/resources/avroDecimal/decimal.avro new file mode 100755 index 000000000000..6da423f78661 Binary files /dev/null and b/sql/hive/src/test/resources/avroDecimal/decimal.avro differ diff --git a/sql/hive/src/test/resources/data/conf/hive-log4j.properties b/sql/hive/src/test/resources/data/conf/hive-log4j.properties index 885c86f2b94f..6a042472adb9 100644 --- a/sql/hive/src/test/resources/data/conf/hive-log4j.properties +++ b/sql/hive/src/test/resources/data/conf/hive-log4j.properties @@ -47,7 +47,7 @@ log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) # # console -# Add "console" to rootlogger above if you want to use this +# Add "console" to rootlogger above if you want to use this # log4j.appender.console=org.apache.log4j.ConsoleAppender diff --git a/sql/hive/src/test/resources/data/scripts/cat.py b/sql/hive/src/test/resources/data/scripts/cat.py index 2395b2cdeb39..aea0362f899f 100644 --- a/sql/hive/src/test/resources/data/scripts/cat.py +++ b/sql/hive/src/test/resources/data/scripts/cat.py @@ -16,14 +16,14 @@ # specific language governing permissions and limitations # under the License. # -import sys, re -import datetime +from __future__ import print_function +import sys import os -table_name=None -if os.environ.has_key('hive_streaming_tablename'): - table_name=os.environ['hive_streaming_tablename'] +table_name = None +if os.environ in 'hive_streaming_tablename': + table_name = os.environ['hive_streaming_tablename'] for line in sys.stdin: - print line - print >> sys.stderr, "dummy" + print(line) + print("dummy", file=sys.stderr) diff --git a/sql/hive/src/test/resources/data/scripts/cat_error.py b/sql/hive/src/test/resources/data/scripts/cat_error.py index 9642efec8ecb..dc1bccece947 100644 --- a/sql/hive/src/test/resources/data/scripts/cat_error.py +++ b/sql/hive/src/test/resources/data/scripts/cat_error.py @@ -19,6 +19,6 @@ import sys for line in sys.stdin: - print line + print(line) sys.exit(1) diff --git a/sql/hive/src/test/resources/data/scripts/doubleescapedtab.py b/sql/hive/src/test/resources/data/scripts/doubleescapedtab.py index d373067baed2..ff5a8b82f429 100644 --- a/sql/hive/src/test/resources/data/scripts/doubleescapedtab.py +++ b/sql/hive/src/test/resources/data/scripts/doubleescapedtab.py @@ -19,6 +19,5 @@ import sys for line in sys.stdin: - print "1\\\\\\t2" - print "1\\\\\\\\t2" - + print("1\\\\\\t2") + print("1\\\\\\\\t2") diff --git a/sql/hive/src/test/resources/data/scripts/dumpdata_script.py b/sql/hive/src/test/resources/data/scripts/dumpdata_script.py index c96c9e529bbb..341a1b40e07a 100644 --- a/sql/hive/src/test/resources/data/scripts/dumpdata_script.py +++ b/sql/hive/src/test/resources/data/scripts/dumpdata_script.py @@ -19,9 +19,9 @@ import sys for i in xrange(50): - for j in xrange(5): - for k in xrange(20022): - print 20000 * i + k + for j in xrange(5): + for k in xrange(20022): + print(20000 * i + k) for line in sys.stdin: - pass + pass diff --git a/sql/hive/src/test/resources/data/scripts/escapedcarriagereturn.py b/sql/hive/src/test/resources/data/scripts/escapedcarriagereturn.py index 475928a2430f..894cbdd13951 100644 --- a/sql/hive/src/test/resources/data/scripts/escapedcarriagereturn.py +++ b/sql/hive/src/test/resources/data/scripts/escapedcarriagereturn.py @@ -19,5 +19,4 @@ import sys for line in sys.stdin: - print "1\\\\r2" - + print("1\\\\r2") diff --git a/sql/hive/src/test/resources/data/scripts/escapednewline.py b/sql/hive/src/test/resources/data/scripts/escapednewline.py index 0d5751454bed..ff47fe573470 100644 --- a/sql/hive/src/test/resources/data/scripts/escapednewline.py +++ b/sql/hive/src/test/resources/data/scripts/escapednewline.py @@ -19,5 +19,4 @@ import sys for line in sys.stdin: - print "1\\\\n2" - + print("1\\\\n2") diff --git a/sql/hive/src/test/resources/data/scripts/escapedtab.py b/sql/hive/src/test/resources/data/scripts/escapedtab.py index 549c91e44463..d9743eec5642 100644 --- a/sql/hive/src/test/resources/data/scripts/escapedtab.py +++ b/sql/hive/src/test/resources/data/scripts/escapedtab.py @@ -19,5 +19,4 @@ import sys for line in sys.stdin: - print "1\\\\t2" - + print("1\\\\t2") diff --git a/sql/hive/src/test/resources/data/scripts/input20_script.py b/sql/hive/src/test/resources/data/scripts/input20_script.py index 40e3683dc3d3..08669cbf0a1a 100644 --- a/sql/hive/src/test/resources/data/scripts/input20_script.py +++ b/sql/hive/src/test/resources/data/scripts/input20_script.py @@ -21,10 +21,10 @@ line = sys.stdin.readline() x = 1 while line: - tem = sys.stdin.readline() - if line == tem: - x = x + 1 - else: - print str(x).strip()+'\t'+re.sub('\t','_',line.strip()) - line = tem - x = 1 \ No newline at end of file + tem = sys.stdin.readline() + if line == tem: + x += 1 + else: + print(str(x).strip()+'\t'+re.sub('\t', '_', line.strip())) + line = tem + x = 1 diff --git a/sql/hive/src/test/resources/data/scripts/newline.py b/sql/hive/src/test/resources/data/scripts/newline.py index 6500d900dd8a..59c313fcc29f 100644 --- a/sql/hive/src/test/resources/data/scripts/newline.py +++ b/sql/hive/src/test/resources/data/scripts/newline.py @@ -19,6 +19,6 @@ import sys for line in sys.stdin: - print "1\\n2" - print "1\\r2" - print "1\\t2" + print("1\\n2") + print("1\\r2") + print("1\\t2") diff --git a/sql/hive/src/test/resources/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e b/sql/hive/src/test/resources/golden/'1' + 1.0-0-404b0ea20c125c9648b7919a8f41add3 similarity index 100% rename from sql/hive/src/test/resources/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e rename to sql/hive/src/test/resources/golden/'1' + 1.0-0-404b0ea20c125c9648b7919a8f41add3 diff --git a/sql/hive/src/test/resources/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 b/sql/hive/src/test/resources/golden/1 + 1.0-0-77ca48f121bd2ef41efb9ee3bc28418 similarity index 100% rename from sql/hive/src/test/resources/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 rename to sql/hive/src/test/resources/golden/1 + 1.0-0-77ca48f121bd2ef41efb9ee3bc28418 diff --git a/sql/hive/src/test/resources/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 b/sql/hive/src/test/resources/golden/1.0 + '1'-0-6beb1ef5178117a9fd641008ed5ebb80 similarity index 100% rename from sql/hive/src/test/resources/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 rename to sql/hive/src/test/resources/golden/1.0 + '1'-0-6beb1ef5178117a9fd641008ed5ebb80 diff --git a/sql/hive/src/test/resources/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 b/sql/hive/src/test/resources/golden/1.0 + 1-0-bec2842d2b009973b4d4b8f10b5554f8 similarity index 100% rename from sql/hive/src/test/resources/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 rename to sql/hive/src/test/resources/golden/1.0 + 1-0-bec2842d2b009973b4d4b8f10b5554f8 diff --git a/sql/hive/src/test/resources/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d b/sql/hive/src/test/resources/golden/1.0 + 1.0-0-eafdfdbb14980ee517c388dc117d91a8 similarity index 100% rename from sql/hive/src/test/resources/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d rename to sql/hive/src/test/resources/golden/1.0 + 1.0-0-eafdfdbb14980ee517c388dc117d91a8 diff --git a/sql/hive/src/test/resources/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 b/sql/hive/src/test/resources/golden/1.0 + 1L-0-ef273f05968cd0e91af8c76949c73798 similarity index 100% rename from sql/hive/src/test/resources/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 rename to sql/hive/src/test/resources/golden/1.0 + 1L-0-ef273f05968cd0e91af8c76949c73798 diff --git a/sql/hive/src/test/resources/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 b/sql/hive/src/test/resources/golden/1.0 + 1S-0-9f93538c38920d52b322bfc40cc2f31a similarity index 100% rename from sql/hive/src/test/resources/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 rename to sql/hive/src/test/resources/golden/1.0 + 1S-0-9f93538c38920d52b322bfc40cc2f31a diff --git a/sql/hive/src/test/resources/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 b/sql/hive/src/test/resources/golden/1.0 + 1Y-0-9e354e022b1b423f366bf79ed7522f2a similarity index 100% rename from sql/hive/src/test/resources/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 rename to sql/hive/src/test/resources/golden/1.0 + 1Y-0-9e354e022b1b423f366bf79ed7522f2a diff --git a/sql/hive/src/test/resources/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 b/sql/hive/src/test/resources/golden/1L + 1.0-0-9b0510d0bb3e9ee6a7698369b008a280 similarity index 100% rename from sql/hive/src/test/resources/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 rename to sql/hive/src/test/resources/golden/1L + 1.0-0-9b0510d0bb3e9ee6a7698369b008a280 diff --git a/sql/hive/src/test/resources/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e b/sql/hive/src/test/resources/golden/1S + 1.0-0-c3d54e5b6034b7796ed16896a434d1ba similarity index 100% rename from sql/hive/src/test/resources/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e rename to sql/hive/src/test/resources/golden/1S + 1.0-0-c3d54e5b6034b7796ed16896a434d1ba diff --git a/sql/hive/src/test/resources/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d b/sql/hive/src/test/resources/golden/1Y + 1.0-0-7b54e1d367c2ed1f5c181298ee5470d0 similarity index 100% rename from sql/hive/src/test/resources/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d rename to sql/hive/src/test/resources/golden/1Y + 1.0-0-7b54e1d367c2ed1f5c181298ee5470d0 diff --git a/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for CUBE #1-0-63b61fb3f0e74226001ad279be440864 b/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for CUBE #1-0-63b61fb3f0e74226001ad279be440864 deleted file mode 100644 index dac1b84b916d..000000000000 --- a/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for CUBE #1-0-63b61fb3f0e74226001ad279be440864 +++ /dev/null @@ -1,6 +0,0 @@ -500 NULL 0 -91 0 1 -84 1 1 -105 2 1 -113 3 1 -107 4 1 diff --git a/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for CUBE #2-0-7a511f02a16f0af4f810b1666cfcd896 b/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for CUBE #2-0-7a511f02a16f0af4f810b1666cfcd896 deleted file mode 100644 index c7cb747c0a65..000000000000 --- a/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for CUBE #2-0-7a511f02a16f0af4f810b1666cfcd896 +++ /dev/null @@ -1,10 +0,0 @@ -1 NULL -3 2 -1 NULL -1 2 -1 NULL 3 2 -1 NULL 4 2 -1 NULL 5 2 -1 NULL 6 2 -1 NULL 12 2 -1 NULL 14 2 -1 NULL 15 2 -1 NULL 22 2 diff --git a/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for GroupingSet-0-8c14c24670a4b06c440346277ce9cf1c b/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for GroupingSet-0-8c14c24670a4b06c440346277ce9cf1c deleted file mode 100644 index c7cb747c0a65..000000000000 --- a/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for GroupingSet-0-8c14c24670a4b06c440346277ce9cf1c +++ /dev/null @@ -1,10 +0,0 @@ -1 NULL -3 2 -1 NULL -1 2 -1 NULL 3 2 -1 NULL 4 2 -1 NULL 5 2 -1 NULL 6 2 -1 NULL 12 2 -1 NULL 14 2 -1 NULL 15 2 -1 NULL 22 2 diff --git a/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for Rollup #1-0-a78e3dbf242f240249e36b3d3fd0926a b/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for Rollup #1-0-a78e3dbf242f240249e36b3d3fd0926a deleted file mode 100644 index dac1b84b916d..000000000000 --- a/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for Rollup #1-0-a78e3dbf242f240249e36b3d3fd0926a +++ /dev/null @@ -1,6 +0,0 @@ -500 NULL 0 -91 0 1 -84 1 1 -105 2 1 -113 3 1 -107 4 1 diff --git a/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for Rollup #2-0-bf180c9d1a18f61b9d9f31bb0115cf89 b/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for Rollup #2-0-bf180c9d1a18f61b9d9f31bb0115cf89 deleted file mode 100644 index 1eea4a9b2368..000000000000 --- a/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for Rollup #2-0-bf180c9d1a18f61b9d9f31bb0115cf89 +++ /dev/null @@ -1,10 +0,0 @@ -1 0 5 3 -1 0 15 3 -1 0 25 3 -1 0 60 3 -1 0 75 3 -1 0 80 3 -1 0 100 3 -1 0 140 3 -1 0 145 3 -1 0 150 3 diff --git a/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for Rollup #3-0-9257085d123728730be96b6d9fbb84ce b/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for Rollup #3-0-9257085d123728730be96b6d9fbb84ce deleted file mode 100644 index 1eea4a9b2368..000000000000 --- a/sql/hive/src/test/resources/golden/SPARK-8976 Wrong Result for Rollup #3-0-9257085d123728730be96b6d9fbb84ce +++ /dev/null @@ -1,10 +0,0 @@ -1 0 5 3 -1 0 15 3 -1 0 25 3 -1 0 60 3 -1 0 75 3 -1 0 80 3 -1 0 100 3 -1 0 140 3 -1 0 145 3 -1 0 150 3 diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 deleted file mode 100644 index 11487abed2b6..000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 +++ /dev/null @@ -1,4 +0,0 @@ -key int -value string - -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871688, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, 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:{transient_lastDdlTime=1413871688}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 deleted file mode 100644 index 979969dcbfd3..000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 +++ /dev/null @@ -1,10 +0,0 @@ -key int -value string -ds string - -# Partition Information -# col_name data_type comment - -ds string - -Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1413871689, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 deleted file mode 100644 index 7e14edcdead2..000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 +++ /dev/null @@ -1,10 +0,0 @@ -key int -value string -ds string - -# Partition Information -# col_name data_type comment - -ds string - -Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1413871689, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/test/test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 deleted file mode 100644 index 77a764a814eb..000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 +++ /dev/null @@ -1,10 +0,0 @@ -key int -value string -ds string - -# Partition Information -# col_name data_type comment - -ds string - -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871689, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, 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:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1413871689}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 deleted file mode 100644 index c8606b1acad0..000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 +++ /dev/null @@ -1,10 +0,0 @@ -key int -value string -ds string - -# Partition Information -# col_name data_type comment - -ds string - -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871689, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 deleted file mode 100644 index 59922d3b7a08..000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 +++ /dev/null @@ -1,10 +0,0 @@ -key int -value string -ds string - -# Partition Information -# col_name data_type comment - -ds string - -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871689, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 deleted file mode 100644 index 45ef75553947..000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 +++ /dev/null @@ -1,4 +0,0 @@ -key int -value string - -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871688, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871688, transient_lastDdlTime=1413871688, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 deleted file mode 100644 index d6804307f3dc..000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 +++ /dev/null @@ -1,4 +0,0 @@ -key int -value string - -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871688, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871688, transient_lastDdlTime=1413871688, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 deleted file mode 100644 index 77ba51afd246..000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 +++ /dev/null @@ -1,10 +0,0 @@ -key int -value string -ds string - -# Partition Information -# col_name data_type comment - -ds string - -Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1413871689, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test/ds=2010, 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), parameters:{transient_lastDdlTime=1413871689}) diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 b/sql/hive/src/test/resources/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 deleted file mode 100644 index dd347f3e8f58..000000000000 --- a/sql/hive/src/test/resources/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 +++ /dev/null @@ -1,5 +0,0 @@ -0 val_0 NULL NULL -0 val_0 NULL NULL -0 val_0 NULL NULL -2 val_2 NULL NULL -4 val_4 NULL NULL diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 b/sql/hive/src/test/resources/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 deleted file mode 100644 index 12087837cebf..000000000000 --- a/sql/hive/src/test/resources/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 +++ /dev/null @@ -1,5 +0,0 @@ -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -2 val_2 2 val_2 -4 val_4 4 val_4 diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 b/sql/hive/src/test/resources/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 deleted file mode 100644 index 6839c16243bc..000000000000 --- a/sql/hive/src/test/resources/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 +++ /dev/null @@ -1,5 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -2 val_2 -4 val_4 diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 b/sql/hive/src/test/resources/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 deleted file mode 100644 index 6839c16243bc..000000000000 --- a/sql/hive/src/test/resources/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 +++ /dev/null @@ -1,5 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -2 val_2 -4 val_4 diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f b/sql/hive/src/test/resources/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f deleted file mode 100644 index 879a6e7bcbd1..000000000000 --- a/sql/hive/src/test/resources/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f +++ /dev/null @@ -1,5 +0,0 @@ -0 val -0 val -0 val -2 val -4 val diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 b/sql/hive/src/test/resources/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 deleted file mode 100644 index 6839c16243bc..000000000000 --- a/sql/hive/src/test/resources/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 +++ /dev/null @@ -1,5 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -2 val_2 -4 val_4 diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 b/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 deleted file mode 100644 index 600b37771689..000000000000 --- a/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 +++ /dev/null @@ -1 +0,0 @@ -val_238 7 diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c b/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c deleted file mode 100644 index ad69f390bc8d..000000000000 --- a/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c +++ /dev/null @@ -1 +0,0 @@ -1 val_238 7 diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-8-84e700f9dc6033c1f237fcdb95e31a0c b/sql/hive/src/test/resources/golden/alter_varchar2-8-84e700f9dc6033c1f237fcdb95e31a0c deleted file mode 100644 index ad69f390bc8d..000000000000 --- a/sql/hive/src/test/resources/golden/alter_varchar2-8-84e700f9dc6033c1f237fcdb95e31a0c +++ /dev/null @@ -1 +0,0 @@ -1 val_238 7 diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-9-4c12c4c53d99338796be34e603dc612c b/sql/hive/src/test/resources/golden/alter_varchar2-9-4c12c4c53d99338796be34e603dc612c deleted file mode 100644 index 1f8ddaec9003..000000000000 --- a/sql/hive/src/test/resources/golden/alter_varchar2-9-4c12c4c53d99338796be34e603dc612c +++ /dev/null @@ -1 +0,0 @@ -2 238 3 diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-7-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/auto_join14_hadoop20-2-2b9ccaa793eae0e73bf76335d3d6880 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar2-7-50131c0ba7b7a6b65c789a5a8497bada rename to sql/hive/src/test/resources/golden/auto_join14_hadoop20-2-2b9ccaa793eae0e73bf76335d3d6880 diff --git a/sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 b/sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-cf71a4c4cce08635cc80a64a1ae6bc83 similarity index 100% rename from sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 rename to sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-cf71a4c4cce08635cc80a64a1ae6bc83 diff --git a/sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 b/sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-dfc876530eeaa7c42978d1bc0b1fd58 similarity index 100% rename from sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 rename to sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-dfc876530eeaa7c42978d1bc0b1fd58 diff --git a/sql/hive/src/test/resources/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/combine1-2-6142f47d3fcdd4323162014d5eb35e07 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f rename to sql/hive/src/test/resources/golden/combine1-2-6142f47d3fcdd4323162014d5eb35e07 diff --git a/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine1-3-10266e3d5dd4c841c0d65030b1edba7c similarity index 100% rename from sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf rename to sql/hive/src/test/resources/golden/combine1-3-10266e3d5dd4c841c0d65030b1edba7c diff --git a/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine1-4-9cbd6d400fb6c3cd09010e3dbd76601 similarity index 100% rename from sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 rename to sql/hive/src/test/resources/golden/combine1-4-9cbd6d400fb6c3cd09010e3dbd76601 diff --git a/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine1-5-1ba2d6f3bb3348da3fee7fab4f283f34 similarity index 100% rename from sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 rename to sql/hive/src/test/resources/golden/combine1-5-1ba2d6f3bb3348da3fee7fab4f283f34 diff --git a/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine2-2-6142f47d3fcdd4323162014d5eb35e07 similarity index 100% rename from sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea rename to sql/hive/src/test/resources/golden/combine2-2-6142f47d3fcdd4323162014d5eb35e07 diff --git a/sql/hive/src/test/resources/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine2-3-10266e3d5dd4c841c0d65030b1edba7c similarity index 100% rename from sql/hive/src/test/resources/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf rename to sql/hive/src/test/resources/golden/combine2-3-10266e3d5dd4c841c0d65030b1edba7c diff --git a/sql/hive/src/test/resources/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine2-4-9cbd6d400fb6c3cd09010e3dbd76601 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 rename to sql/hive/src/test/resources/golden/combine2-4-9cbd6d400fb6c3cd09010e3dbd76601 diff --git a/sql/hive/src/test/resources/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine2-5-1ba2d6f3bb3348da3fee7fab4f283f34 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 rename to sql/hive/src/test/resources/golden/combine2-5-1ba2d6f3bb3348da3fee7fab4f283f34 diff --git a/sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 b/sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 new file mode 100644 index 000000000000..a01c2622c68e --- /dev/null +++ b/sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 @@ -0,0 +1 @@ +1 NULL 1 NULL 1.0 NULL true NULL 1 NULL 1.0 NULL 1 NULL 1 NULL 1 NULL 1970-01-01 NULL NULL 1 NULL diff --git a/sql/hive/src/test/resources/golden/constant null testing-0-9a02bc7de09bcabcbd4c91f54a814c20 b/sql/hive/src/test/resources/golden/constant null testing-0-9a02bc7de09bcabcbd4c91f54a814c20 deleted file mode 100644 index 7c41615f8c18..000000000000 --- a/sql/hive/src/test/resources/golden/constant null testing-0-9a02bc7de09bcabcbd4c91f54a814c20 +++ /dev/null @@ -1 +0,0 @@ -1 NULL 1 NULL 1.0 NULL true NULL 1 NULL 1.0 NULL 1 NULL 1 NULL 1 NULL 1970-01-01 NULL 1969-12-31 16:00:00.001 NULL 1 NULL diff --git a/sql/hive/src/test/resources/golden/date_3-0-c26de4559926ddb0127d2dc5ea154774 b/sql/hive/src/test/resources/golden/date_3-0-c26de4559926ddb0127d2dc5ea154774 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/date_3-1-d9a07d08f5204ae8208fd88c9255d447 b/sql/hive/src/test/resources/golden/date_3-1-d9a07d08f5204ae8208fd88c9255d447 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/date_3-2-a937c6e5a2c655930e0d3f80883ecc16 b/sql/hive/src/test/resources/golden/date_3-2-a937c6e5a2c655930e0d3f80883ecc16 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/date_3-3-4cf49e71b636df754871a675f9e4e24 b/sql/hive/src/test/resources/golden/date_3-3-4cf49e71b636df754871a675f9e4e24 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/date_3-3-c26f0641e7cec1093273b258e6bf7120 b/sql/hive/src/test/resources/golden/date_3-3-c26f0641e7cec1093273b258e6bf7120 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/date_3-4-e009f358964f6d1236cfc03283e2b06f b/sql/hive/src/test/resources/golden/date_3-4-e009f358964f6d1236cfc03283e2b06f deleted file mode 100644 index 66d2220d06de..000000000000 --- a/sql/hive/src/test/resources/golden/date_3-4-e009f358964f6d1236cfc03283e2b06f +++ /dev/null @@ -1 +0,0 @@ -1 2011-01-01 diff --git a/sql/hive/src/test/resources/golden/date_3-5-c26de4559926ddb0127d2dc5ea154774 b/sql/hive/src/test/resources/golden/date_3-5-c26de4559926ddb0127d2dc5ea154774 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/diff_part_input_formats-0-12652a5a33548c245772e8d0894af5ad b/sql/hive/src/test/resources/golden/diff_part_input_formats-0-12652a5a33548c245772e8d0894af5ad deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/diff_part_input_formats-1-961f7cb386a6eacd391dcb189cbeddaa b/sql/hive/src/test/resources/golden/diff_part_input_formats-1-961f7cb386a6eacd391dcb189cbeddaa deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/diff_part_input_formats-2-28cd0f9b01baa8627a013339dc9508ce b/sql/hive/src/test/resources/golden/diff_part_input_formats-2-28cd0f9b01baa8627a013339dc9508ce deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/diff_part_input_formats-3-c6eef43568e8ed96299720d30a6235e1 b/sql/hive/src/test/resources/golden/diff_part_input_formats-3-c6eef43568e8ed96299720d30a6235e1 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-ce3797dc14a603cba2a5e58c8612de5b deleted file mode 100644 index 60878ffb7706..000000000000 --- a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-ce3797dc14a603cba2a5e58c8612de5b +++ /dev/null @@ -1 +0,0 @@ -238 val_238 diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-f5340880d2be7b0643eb995673e89d11 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-f5340880d2be7b0643eb995673e89d11 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-714ab8c97f4d8993680b91e1ed8f3782 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-714ab8c97f4d8993680b91e1ed8f3782 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-34064fd15c28dba55865cb8f3c5ba68c b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-34064fd15c28dba55865cb8f3c5ba68c deleted file mode 100644 index 573c4b56de59..000000000000 --- a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-34064fd15c28dba55865cb8f3c5ba68c +++ /dev/null @@ -1 +0,0 @@ -1 {"a1":"b1"} foo1 diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-6-f40a07d7654573e1a8517770eb8529e7 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-6-f40a07d7654573e1a8517770eb8529e7 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-0-b454ca2d55b61fd597540dbe38eb51ab b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-0-b454ca2d55b61fd597540dbe38eb51ab deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-1-ece80e0bd1236c547da7eceac114e602 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-1-ece80e0bd1236c547da7eceac114e602 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-2-fb7b53f61989f4f645dac4a8f017d6ee b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-2-fb7b53f61989f4f645dac4a8f017d6ee deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-3-46fe5bb027667f528d7179b239e3427f b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-3-46fe5bb027667f528d7179b239e3427f deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-4-26dcd2b2f263b5b417430efcf354663a b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-4-26dcd2b2f263b5b417430efcf354663a deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-6-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-6-7a9e67189d3d4151f23b12c22bde06b5 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-7-16c31455a193e1cb06a2ede4e9f5d5dd b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-7-16c31455a193e1cb06a2ede4e9f5d5dd deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-0-97b52abf021c81b8364041c1a0bbccf3 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-0-97b52abf021c81b8364041c1a0bbccf3 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-1-f11a45c42752d06821ccd26d948d51ff b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-1-f11a45c42752d06821ccd26d948d51ff deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-2-c0b85445b616f93c5e6d090fa35072e7 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-2-c0b85445b616f93c5e6d090fa35072e7 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-4-b2ca31dd6cc5c32e33df700786f5b208 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-4-b2ca31dd6cc5c32e33df700786f5b208 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-0-c6dff7eb0a793f9cd555164d23eda699 b/sql/hive/src/test/resources/golden/fileformat_mix-0-c6dff7eb0a793f9cd555164d23eda699 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b b/sql/hive/src/test/resources/golden/fileformat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-2-701660c0ea117b11d12de54dc661bc3e b/sql/hive/src/test/resources/golden/fileformat_mix-2-701660c0ea117b11d12de54dc661bc3e deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-3-2b2316f235737a3f9a30fb05a082e132 b/sql/hive/src/test/resources/golden/fileformat_mix-3-2b2316f235737a3f9a30fb05a082e132 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-4-fcda187f1366ff93a113cbe670335198 b/sql/hive/src/test/resources/golden/fileformat_mix-4-fcda187f1366ff93a113cbe670335198 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 b/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 deleted file mode 100644 index 1b79f38e25b2..000000000000 --- a/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 +++ /dev/null @@ -1 +0,0 @@ -500 diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 b/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 deleted file mode 100644 index e34118512c1d..000000000000 --- a/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 +++ /dev/null @@ -1,500 +0,0 @@ -238 -86 -311 -27 -165 -409 -255 -278 -98 -484 -265 -193 -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -429 -374 -152 -469 -145 -495 -37 -327 -281 -277 -209 -15 -82 -403 -166 -417 -430 -252 -292 -219 -287 -153 -193 -338 -446 -459 -394 -237 -482 -174 -413 -494 -207 -199 -466 -208 -174 -399 -396 -247 -417 -489 -162 -377 -397 -309 -365 -266 -439 -342 -367 -325 -167 -195 -475 -17 -113 -155 -203 -339 -0 -455 -128 -311 -316 -57 -302 -205 -149 -438 -345 -129 -170 -20 -489 -157 -378 -221 -92 -111 -47 -72 -4 -280 -35 -427 -277 -208 -356 -399 -169 -382 -498 -125 -386 -437 -469 -192 -286 -187 -176 -54 -459 -51 -138 -103 -239 -213 -216 -430 -278 -176 -289 -221 -65 -318 -332 -311 -275 -137 -241 -83 -333 -180 -284 -12 -230 -181 -67 -260 -404 -384 -489 -353 -373 -272 -138 -217 -84 -348 -466 -58 -8 -411 -230 -208 -348 -24 -463 -431 -179 -172 -42 -129 -158 -119 -496 -0 -322 -197 -468 -393 -454 -100 -298 -199 -191 -418 -96 -26 -165 -327 -230 -205 -120 -131 -51 -404 -43 -436 -156 -469 -468 -308 -95 -196 -288 -481 -457 -98 -282 -197 -187 -318 -318 -409 -470 -137 -369 -316 -169 -413 -85 -77 -0 -490 -87 -364 -179 -118 -134 -395 -282 -138 -238 -419 -15 -118 -72 -90 -307 -19 -435 -10 -277 -273 -306 -224 -309 -389 -327 -242 -369 -392 -272 -331 -401 -242 -452 -177 -226 -5 -497 -402 -396 -317 -395 -58 -35 -336 -95 -11 -168 -34 -229 -233 -143 -472 -322 -498 -160 -195 -42 -321 -430 -119 -489 -458 -78 -76 -41 -223 -492 -149 -449 -218 -228 -138 -453 -30 -209 -64 -468 -76 -74 -342 -69 -230 -33 -368 -103 -296 -113 -216 -367 -344 -167 -274 -219 -239 -485 -116 -223 -256 -263 -70 -487 -480 -401 -288 -191 -5 -244 -438 -128 -467 -432 -202 -316 -229 -469 -463 -280 -2 -35 -283 -331 -235 -80 -44 -193 -321 -335 -104 -466 -366 -175 -403 -483 -53 -105 -257 -406 -409 -190 -406 -401 -114 -258 -90 -203 -262 -348 -424 -12 -396 -201 -217 -164 -431 -454 -478 -298 -125 -431 -164 -424 -187 -382 -5 -70 -397 -480 -291 -24 -351 -255 -104 -70 -163 -438 -119 -414 -200 -491 -237 -439 -360 -248 -479 -305 -417 -199 -444 -120 -429 -169 -443 -323 -325 -277 -230 -478 -178 -468 -310 -317 -333 -493 -460 -207 -249 -265 -480 -83 -136 -353 -172 -214 -462 -233 -406 -133 -175 -189 -454 -375 -401 -421 -407 -384 -256 -26 -134 -67 -384 -379 -18 -462 -492 -100 -298 -9 -341 -498 -146 -458 -362 -186 -285 -348 -167 -18 -273 -183 -281 -344 -97 -469 -315 -84 -28 -37 -448 -152 -348 -307 -194 -414 -477 -222 -126 -90 -169 -403 -400 -200 -97 diff --git a/sql/hive/src/test/resources/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/groupby1-3-c8478dac3497697b4375ee35118a5c3e similarity index 100% rename from sql/hive/src/test/resources/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea rename to sql/hive/src/test/resources/golden/groupby1-3-c8478dac3497697b4375ee35118a5c3e diff --git a/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b b/sql/hive/src/test/resources/golden/groupby1-5-c9cee6382b64bd3d71177527961b8be2 similarity index 100% rename from sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b rename to sql/hive/src/test/resources/golden/groupby1-5-c9cee6382b64bd3d71177527961b8be2 diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/groupby1_limit-0-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-50131c0ba7b7a6b65c789a5a8497bada rename to sql/hive/src/test/resources/golden/groupby1_limit-0-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-a071dedef216e84d1cb2f0de6d34fd1a b/sql/hive/src/test/resources/golden/groupby1_map-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-a071dedef216e84d1cb2f0de6d34fd1a rename to sql/hive/src/test/resources/golden/groupby1_map-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-5-2a1bd5ed3955825a9dbb76769f7fe4ea b/sql/hive/src/test/resources/golden/groupby1_map_skew-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-5-2a1bd5ed3955825a9dbb76769f7fe4ea rename to sql/hive/src/test/resources/golden/groupby1_map_skew-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-8-2a1bd5ed3955825a9dbb76769f7fe4ea b/sql/hive/src/test/resources/golden/groupby1_noskew-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-8-2a1bd5ed3955825a9dbb76769f7fe4ea rename to sql/hive/src/test/resources/golden/groupby1_noskew-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-9-40110efef10f6f7b873dcd1d53463101 b/sql/hive/src/test/resources/golden/groupby2_limit-0-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-9-40110efef10f6f7b873dcd1d53463101 rename to sql/hive/src/test/resources/golden/groupby2_limit-0-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-3-10a71bca930d911cc4c2022575b17299 b/sql/hive/src/test/resources/golden/groupby2_map-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-3-10a71bca930d911cc4c2022575b17299 rename to sql/hive/src/test/resources/golden/groupby2_map-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-5-10a71bca930d911cc4c2022575b17299 b/sql/hive/src/test/resources/golden/groupby2_map_skew-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-5-10a71bca930d911cc4c2022575b17299 rename to sql/hive/src/test/resources/golden/groupby2_map_skew-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-6-d1c175a9d042ecd389f2f93fc867591d b/sql/hive/src/test/resources/golden/groupby2_noskew-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-6-d1c175a9d042ecd389f2f93fc867591d rename to sql/hive/src/test/resources/golden/groupby2_noskew-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d b/sql/hive/src/test/resources/golden/groupby4_map-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d rename to sql/hive/src/test/resources/golden/groupby4_map-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 b/sql/hive/src/test/resources/golden/groupby4_map_skew-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 rename to sql/hive/src/test/resources/golden/groupby4_map_skew-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_noskew-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby4_noskew-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_map-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby5_map-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_map_skew-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby5_map_skew-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_noskew-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby5_noskew-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_map-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby6_map-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_map_skew-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby6_map_skew-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_noskew-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby6_noskew-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map-3-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby7_map-3-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map_skew-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby7_map_skew-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_noskew-3-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby7_noskew-3-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_map-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby8_map-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_map_skew-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby8_map_skew-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_noskew-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby8_noskew-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_map_ppr-2-be2c0b32a02a1154bfdee1a52530f387 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/groupby_map_ppr-2-be2c0b32a02a1154bfdee1a52530f387 diff --git a/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 b/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 index d3ffb995aff4..93ba96ec8c15 100644 --- a/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 +++ b/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 @@ -1,2 +1,2 @@ -a int -b double +A int +B double diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/input12_hadoop20-0-2b9ccaa793eae0e73bf76335d3d6880 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/input12_hadoop20-0-2b9ccaa793eae0e73bf76335d3d6880 diff --git a/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 b/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 index d3ffb995aff4..93ba96ec8c15 100644 --- a/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 +++ b/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 @@ -1,2 +1,2 @@ -a int -b double +A int +B double diff --git a/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd b/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd index d3ffb995aff4..93ba96ec8c15 100644 --- a/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd +++ b/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd @@ -1,2 +1,2 @@ -a int -b double +A int +B double diff --git a/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b b/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b index 77eaef91c9c3..d52fcf0ebbdb 100644 --- a/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b +++ b/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b @@ -1,3 +1,3 @@ -a array -b double -c map +A array +B double +C map diff --git a/sql/hive/src/test/resources/golden/input3-0-2c80ec90d4d2c9c7446c05651bb76bff b/sql/hive/src/test/resources/golden/input3-0-2c80ec90d4d2c9c7446c05651bb76bff deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d b/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d deleted file mode 100644 index d3ffb995aff4..000000000000 --- a/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d +++ /dev/null @@ -1,2 +0,0 @@ -a int -b double diff --git a/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 b/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 deleted file mode 100644 index bd673a6c1f1d..000000000000 --- a/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 +++ /dev/null @@ -1,4 +0,0 @@ -a array -b double -c map -x double diff --git a/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c deleted file mode 100644 index f5b9883df09c..000000000000 --- a/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c +++ /dev/null @@ -1,4 +0,0 @@ -src -srcpart -test3a -test3c diff --git a/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 b/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/input3-13-23bbec31affef0d758bc4a40490e0b9a b/sql/hive/src/test/resources/golden/input3-13-23bbec31affef0d758bc4a40490e0b9a deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 b/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 deleted file mode 100644 index ea55abd79231..000000000000 --- a/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 +++ /dev/null @@ -1,4 +0,0 @@ -r1 int -r2 double - -Detailed Table Information Table(tableName:test3c, dbName:default, owner:marmbrus, createTime:1413882084, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:r1, type:int, comment:null), FieldSchema(name:r2, type:double, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test3c, 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=0, last_modified_by=marmbrus, last_modified_time=1413882084, transient_lastDdlTime=1413882084, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/input3-2-fa2aceba8cdcb869262e8ad6d431f491 b/sql/hive/src/test/resources/golden/input3-2-fa2aceba8cdcb869262e8ad6d431f491 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 b/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 deleted file mode 100644 index 77eaef91c9c3..000000000000 --- a/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 +++ /dev/null @@ -1,3 +0,0 @@ -a array -b double -c map diff --git a/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c deleted file mode 100644 index b584fd7c6fd3..000000000000 --- a/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c +++ /dev/null @@ -1,4 +0,0 @@ -src -srcpart -test3a -test3b diff --git a/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 b/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/input3-6-ba8c440158c2519353d02471bfb05694 b/sql/hive/src/test/resources/golden/input3-6-ba8c440158c2519353d02471bfb05694 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 b/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 deleted file mode 100644 index bd673a6c1f1d..000000000000 --- a/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 +++ /dev/null @@ -1,4 +0,0 @@ -a array -b double -c map -x double diff --git a/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be b/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/input3-9-5076c1c35053b09173f6acdf1b5e9d6e b/sql/hive/src/test/resources/golden/input3-9-5076c1c35053b09173f6acdf1b5e9d6e deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/input_testsequencefile-0-dd959af1968381d0ed90178d349b01a7 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/input_testsequencefile-0-dd959af1968381d0ed90178d349b01a7 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/input_testsequencefile-1-ddbb8d5e5dc0988bda96ac2b4aec8f94 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/input_testsequencefile-1-ddbb8d5e5dc0988bda96ac2b4aec8f94 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/input_testsequencefile-5-25715870c569b0f8c3d483e3a38b3199 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/input_testsequencefile-5-25715870c569b0f8c3d483e3a38b3199 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/join14_hadoop20-1-2b9ccaa793eae0e73bf76335d3d6880 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/join14_hadoop20-1-2b9ccaa793eae0e73bf76335d3d6880 diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f deleted file mode 100644 index 573541ac9702..000000000000 --- a/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/leftsemijoin_mr-7-6b9861b999092f1ea4fa1fd27a666af6 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/leftsemijoin_mr-7-6b9861b999092f1ea4fa1fd27a666af6 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin_mr-7-8e9c2969b999557363e40f9ebb3f6d7c b/sql/hive/src/test/resources/golden/leftsemijoin_mr-7-8e9c2969b999557363e40f9ebb3f6d7c deleted file mode 100644 index 573541ac9702..000000000000 --- a/sql/hive/src/test/resources/golden/leftsemijoin_mr-7-8e9c2969b999557363e40f9ebb3f6d7c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/merge2-2-6142f47d3fcdd4323162014d5eb35e07 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/merge2-2-6142f47d3fcdd4323162014d5eb35e07 diff --git a/sql/hive/src/test/resources/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf deleted file mode 100644 index 573541ac9702..000000000000 --- a/sql/hive/src/test/resources/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/merge2-3-10266e3d5dd4c841c0d65030b1edba7c similarity index 100% rename from sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/merge2-3-10266e3d5dd4c841c0d65030b1edba7c diff --git a/sql/hive/src/test/resources/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 deleted file mode 100644 index 573541ac9702..000000000000 --- a/sql/hive/src/test/resources/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 deleted file mode 100644 index 573541ac9702..000000000000 --- a/sql/hive/src/test/resources/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/merge2-4-9cbd6d400fb6c3cd09010e3dbd76601 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/merge2-4-9cbd6d400fb6c3cd09010e3dbd76601 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/merge2-5-1ba2d6f3bb3348da3fee7fab4f283f34 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/merge2-5-1ba2d6f3bb3348da3fee7fab4f283f34 diff --git a/sql/hive/src/test/resources/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea deleted file mode 100644 index 573541ac9702..000000000000 --- a/sql/hive/src/test/resources/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 b/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 deleted file mode 100644 index 573541ac9702..000000000000 --- a/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/parallel-0-6dc30e2de057022e63bd2a645fbec4c2 similarity index 100% rename from sql/hive/src/test/resources/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f rename to sql/hive/src/test/resources/golden/parallel-0-6dc30e2de057022e63bd2a645fbec4c2 diff --git a/sql/hive/src/test/resources/golden/partition_schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c b/sql/hive/src/test/resources/golden/partition_schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 b/sql/hive/src/test/resources/golden/partition_schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 b/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 deleted file mode 100644 index c97e50a8a58c..000000000000 --- a/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 +++ /dev/null @@ -1,8 +0,0 @@ -key string -value string -dt string - -# Partition Information -# col_name data_type comment - -dt string diff --git a/sql/hive/src/test/resources/golden/partition_schema1-3-fdef2e7e9e40868305d21c1b0df019bb b/sql/hive/src/test/resources/golden/partition_schema1-3-fdef2e7e9e40868305d21c1b0df019bb deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 b/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 deleted file mode 100644 index 39db984c884a..000000000000 --- a/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 +++ /dev/null @@ -1,9 +0,0 @@ -key string -value string -x string -dt string - -# Partition Information -# col_name data_type comment - -dt string diff --git a/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 b/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 deleted file mode 100644 index c97e50a8a58c..000000000000 --- a/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 +++ /dev/null @@ -1,8 +0,0 @@ -key string -value string -dt string - -# Partition Information -# col_name data_type comment - -dt string diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat4-0-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat4-0-c854b607353e810be297d3159be30da4 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat4-1-c561806d8f9ad419dc9b17ae995aab68 b/sql/hive/src/test/resources/golden/partition_wise_fileformat4-1-c561806d8f9ad419dc9b17ae995aab68 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat4-2-b9f8c3b822051854770f61e5ae5b48b0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat4-2-b9f8c3b822051854770f61e5ae5b48b0 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat4-3-9837451512e92e982f1bd9a12b132e84 b/sql/hive/src/test/resources/golden/partition_wise_fileformat4-3-9837451512e92e982f1bd9a12b132e84 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat4-4-58cfa555b061057f559fc6b9c2f6c631 b/sql/hive/src/test/resources/golden/partition_wise_fileformat4-4-58cfa555b061057f559fc6b9c2f6c631 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat4-5-ac79def5434bb8a926237d0db8db2e84 b/sql/hive/src/test/resources/golden/partition_wise_fileformat4-5-ac79def5434bb8a926237d0db8db2e84 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf deleted file mode 100644 index 573541ac9702..000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-1-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-1-c854b607353e810be297d3159be30da4 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-2-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-2-6c4f7b115f18953dcc7710fa97287459 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-3-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-3-f5f427b174dca478c14eddc371c0025a deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-4-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-4-da1b1887eb530c7e9d37667b99c9793f deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-5-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-5-517aaa22478287fa80eef4a19f2cb9ff deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 deleted file mode 100644 index eb4c6a843cb5..000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 +++ /dev/null @@ -1,2 +0,0 @@ -101 25 -102 25 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-7-a0eeded14b3d337a74189a5d02c7a5ad b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-7-a0eeded14b3d337a74189a5d02c7a5ad deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 deleted file mode 100644 index 95846abf28b2..000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 +++ /dev/null @@ -1,3 +0,0 @@ -101 25 -102 25 -103 25 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf deleted file mode 100644 index 573541ac9702..000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-1-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-1-c854b607353e810be297d3159be30da4 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-2-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-2-6c4f7b115f18953dcc7710fa97287459 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-3-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-3-f5f427b174dca478c14eddc371c0025a deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-4-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-4-da1b1887eb530c7e9d37667b99c9793f deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-5-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-5-517aaa22478287fa80eef4a19f2cb9ff deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f deleted file mode 100644 index 0cfbf08886fc..000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f +++ /dev/null @@ -1 +0,0 @@ -2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 deleted file mode 100644 index 0cfbf08886fc..000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 +++ /dev/null @@ -1 +0,0 @@ -2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf deleted file mode 100644 index 573541ac9702..000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-1-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-1-c854b607353e810be297d3159be30da4 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-2-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-2-6c4f7b115f18953dcc7710fa97287459 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-3-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-3-f5f427b174dca478c14eddc371c0025a deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 deleted file mode 100644 index 60d3b2f4a4cd..000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 +++ /dev/null @@ -1 +0,0 @@ -15 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 deleted file mode 100644 index 0cfbf08886fc..000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 +++ /dev/null @@ -1 +0,0 @@ -2 diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-11-25715870c569b0f8c3d483e3a38b3199 similarity index 100% rename from sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 rename to sql/hive/src/test/resources/golden/rcfile_lazydecompress-11-25715870c569b0f8c3d483e3a38b3199 diff --git a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c deleted file mode 100644 index 573541ac9702..000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 deleted file mode 100644 index 573541ac9702..000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-5-dd959af1968381d0ed90178d349b01a7 similarity index 100% rename from sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e rename to sql/hive/src/test/resources/golden/rcfile_lazydecompress-5-dd959af1968381d0ed90178d349b01a7 diff --git a/sql/hive/src/test/resources/golden/rename_column-0-f7eb4bd6f226be0c13117294be250271 b/sql/hive/src/test/resources/golden/rename_column-0-f7eb4bd6f226be0c13117294be250271 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index 017e14d2ebed..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -a int -b int -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-10-7ef160935cece55338bd4d52277b0203 b/sql/hive/src/test/resources/golden/rename_column-10-7ef160935cece55338bd4d52277b0203 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index a92663b0674b..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -b int -a1 int test comment1 -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-12-379d54e3aa66daacff23c75007dfa008 b/sql/hive/src/test/resources/golden/rename_column-12-379d54e3aa66daacff23c75007dfa008 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index 899341a88185..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -a2 int test comment2 -b int -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-14-25bfcf66698b12f82903f72f13fea4e6 b/sql/hive/src/test/resources/golden/rename_column-14-25bfcf66698b12f82903f72f13fea4e6 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index 26b38dcc6d85..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -b int -a int test comment2 -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-16-d032f4795c1186255acea241387adf93 b/sql/hive/src/test/resources/golden/rename_column-16-d032f4795c1186255acea241387adf93 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c deleted file mode 100644 index 85c1918f4656..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c +++ /dev/null @@ -1,2 +0,0 @@ -src -srcpart diff --git a/sql/hive/src/test/resources/golden/rename_column-18-fe4463a19f61099983f50bb51cfcd335 b/sql/hive/src/test/resources/golden/rename_column-18-fe4463a19f61099983f50bb51cfcd335 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-19-70b42434913b9d2eb17cd216c4f8039f b/sql/hive/src/test/resources/golden/rename_column-19-70b42434913b9d2eb17cd216c4f8039f deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-2-b2b2dfa681d01296fdacb4f56fb6db3a b/sql/hive/src/test/resources/golden/rename_column-2-b2b2dfa681d01296fdacb4f56fb6db3a deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-20-f7eb4bd6f226be0c13117294be250271 b/sql/hive/src/test/resources/golden/rename_column-20-f7eb4bd6f226be0c13117294be250271 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index 017e14d2ebed..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -a int -b int -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-22-b2b2dfa681d01296fdacb4f56fb6db3a b/sql/hive/src/test/resources/golden/rename_column-22-b2b2dfa681d01296fdacb4f56fb6db3a deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index 2fbb615dd599..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -a string -b int -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-24-e4bf0dd372b886b2afcca5b2dc089409 b/sql/hive/src/test/resources/golden/rename_column-24-e4bf0dd372b886b2afcca5b2dc089409 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index 173fbad7b1eb..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -a1 int -b int -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-26-89761e1c7afe3a5b9858f287cb808ccd b/sql/hive/src/test/resources/golden/rename_column-26-89761e1c7afe3a5b9858f287cb808ccd deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index bad9feb96a88..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -a2 int -b int -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 b/sql/hive/src/test/resources/golden/rename_column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index 4f23db53afff..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -b int -a int -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index 2fbb615dd599..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -a string -b int -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-30-7ef160935cece55338bd4d52277b0203 b/sql/hive/src/test/resources/golden/rename_column-30-7ef160935cece55338bd4d52277b0203 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index a92663b0674b..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -b int -a1 int test comment1 -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-32-379d54e3aa66daacff23c75007dfa008 b/sql/hive/src/test/resources/golden/rename_column-32-379d54e3aa66daacff23c75007dfa008 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index 899341a88185..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -a2 int test comment2 -b int -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-34-25bfcf66698b12f82903f72f13fea4e6 b/sql/hive/src/test/resources/golden/rename_column-34-25bfcf66698b12f82903f72f13fea4e6 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index 26b38dcc6d85..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -b int -a int test comment2 -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-36-d032f4795c1186255acea241387adf93 b/sql/hive/src/test/resources/golden/rename_column-36-d032f4795c1186255acea241387adf93 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-37-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/rename_column-37-9c36cac1372650b703400c60dd29042c deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-4-e4bf0dd372b886b2afcca5b2dc089409 b/sql/hive/src/test/resources/golden/rename_column-4-e4bf0dd372b886b2afcca5b2dc089409 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index 173fbad7b1eb..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -a1 int -b int -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-6-89761e1c7afe3a5b9858f287cb808ccd b/sql/hive/src/test/resources/golden/rename_column-6-89761e1c7afe3a5b9858f287cb808ccd deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index bad9feb96a88..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -a2 int -b int -c int diff --git a/sql/hive/src/test/resources/golden/rename_column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 b/sql/hive/src/test/resources/golden/rename_column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b deleted file mode 100644 index 4f23db53afff..000000000000 --- a/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b +++ /dev/null @@ -1,3 +0,0 @@ -b int -a int -c int diff --git a/sql/hive/src/test/resources/golden/show_columns-2-b74990316ec4245fd8a7011e684b39da b/sql/hive/src/test/resources/golden/show_columns-2-b74990316ec4245fd8a7011e684b39da index 70c14c3ef34a..2f7168cba930 100644 --- a/sql/hive/src/test/resources/golden/show_columns-2-b74990316ec4245fd8a7011e684b39da +++ b/sql/hive/src/test/resources/golden/show_columns-2-b74990316ec4245fd8a7011e684b39da @@ -1,3 +1,3 @@ -key -value -ds +KEY +VALUE +ds diff --git a/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 b/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 index 175795534fff..f400819b67c2 100644 --- a/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 +++ b/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 @@ -1,4 +1,5 @@ case +cbrt ceil ceiling coalesce @@ -17,3 +18,6 @@ covar_samp create_union cume_dist current_database +current_date +current_timestamp +current_user diff --git a/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c b/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c index 3c25d656bda1..19458fc86e43 100644 --- a/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c +++ b/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c @@ -2,6 +2,7 @@ assert_true case coalesce current_database +current_date decode e encode diff --git a/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 b/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 index cd2e58d04a4e..1d05f843a7e0 100644 --- a/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 +++ b/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 @@ -1,4 +1,6 @@ +current_date date_add +date_format date_sub datediff to_date diff --git a/core/src/test/resources/spark-events/local-1422981759269/SPARK_VERSION_1.2.0 b/sql/hive/src/test/resources/golden/subquery_exists-0-71049df380c600f02fb6c00d19999e8d old mode 100755 new mode 100644 similarity index 100% rename from core/src/test/resources/spark-events/local-1422981759269/SPARK_VERSION_1.2.0 rename to sql/hive/src/test/resources/golden/subquery_exists-0-71049df380c600f02fb6c00d19999e8d diff --git a/sql/hive/src/test/resources/golden/subquery_exists-1-57688cd1babd6a79bc3b2d2ec434b39 b/sql/hive/src/test/resources/golden/subquery_exists-1-57688cd1babd6a79bc3b2d2ec434b39 new file mode 100644 index 000000000000..7babf117d853 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_exists-1-57688cd1babd6a79bc3b2d2ec434b39 @@ -0,0 +1,11 @@ +98 val_98 +92 val_92 +96 val_96 +95 val_95 +98 val_98 +90 val_90 +95 val_95 +90 val_90 +97 val_97 +90 val_90 +97 val_97 diff --git a/core/src/test/resources/spark-events/local-1422981780767/APPLICATION_COMPLETE b/sql/hive/src/test/resources/golden/subquery_exists-2-4c686f9b9cf51ae1b369acfa43d6c73f old mode 100755 new mode 100644 similarity index 100% rename from core/src/test/resources/spark-events/local-1422981780767/APPLICATION_COMPLETE rename to sql/hive/src/test/resources/golden/subquery_exists-2-4c686f9b9cf51ae1b369acfa43d6c73f diff --git a/sql/hive/src/test/resources/golden/subquery_exists-3-da5828589960a60826f5a08948850d78 b/sql/hive/src/test/resources/golden/subquery_exists-3-da5828589960a60826f5a08948850d78 new file mode 100644 index 000000000000..7babf117d853 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_exists-3-da5828589960a60826f5a08948850d78 @@ -0,0 +1,11 @@ +98 val_98 +92 val_92 +96 val_96 +95 val_95 +98 val_98 +90 val_90 +95 val_95 +90 val_90 +97 val_97 +90 val_90 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/subquery_exists-4-2058d464561ef7b24d896ec8ecb21a00 b/sql/hive/src/test/resources/golden/subquery_exists-4-2058d464561ef7b24d896ec8ecb21a00 new file mode 100644 index 000000000000..7babf117d853 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_exists-4-2058d464561ef7b24d896ec8ecb21a00 @@ -0,0 +1,11 @@ +98 val_98 +92 val_92 +96 val_96 +95 val_95 +98 val_98 +90 val_90 +95 val_95 +90 val_90 +97 val_97 +90 val_90 +97 val_97 diff --git a/core/src/test/resources/spark-events/local-1422981780767/SPARK_VERSION_1.2.0 b/sql/hive/src/test/resources/golden/subquery_exists_having-0-927435f429722c2de003e376b9f0bbd2 old mode 100755 new mode 100644 similarity index 100% rename from core/src/test/resources/spark-events/local-1422981780767/SPARK_VERSION_1.2.0 rename to sql/hive/src/test/resources/golden/subquery_exists_having-0-927435f429722c2de003e376b9f0bbd2 diff --git a/sql/hive/src/test/resources/golden/subquery_exists_having-1-b7ac11dbf892c229e180a2bc761117fe b/sql/hive/src/test/resources/golden/subquery_exists_having-1-b7ac11dbf892c229e180a2bc761117fe new file mode 100644 index 000000000000..3347981aef54 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_exists_having-1-b7ac11dbf892c229e180a2bc761117fe @@ -0,0 +1,6 @@ +90 3 +92 1 +95 2 +96 1 +97 2 +98 2 diff --git a/core/src/test/resources/spark-events/local-1425081759269/APPLICATION_COMPLETE b/sql/hive/src/test/resources/golden/subquery_exists_having-2-4f0b2dbae1324cdc5f3ead83b632e503 old mode 100755 new mode 100644 similarity index 100% rename from core/src/test/resources/spark-events/local-1425081759269/APPLICATION_COMPLETE rename to sql/hive/src/test/resources/golden/subquery_exists_having-2-4f0b2dbae1324cdc5f3ead83b632e503 diff --git a/sql/hive/src/test/resources/golden/subquery_exists_having-3-da5828589960a60826f5a08948850d78 b/sql/hive/src/test/resources/golden/subquery_exists_having-3-da5828589960a60826f5a08948850d78 new file mode 100644 index 000000000000..3347981aef54 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_exists_having-3-da5828589960a60826f5a08948850d78 @@ -0,0 +1,6 @@ +90 3 +92 1 +95 2 +96 1 +97 2 +98 2 diff --git a/sql/hive/src/test/resources/golden/subquery_exists_having-4-fd5457ec549cc2265848f3c95a60693d b/sql/hive/src/test/resources/golden/subquery_exists_having-4-fd5457ec549cc2265848f3c95a60693d new file mode 100644 index 000000000000..3347981aef54 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_exists_having-4-fd5457ec549cc2265848f3c95a60693d @@ -0,0 +1,6 @@ +90 3 +92 1 +95 2 +96 1 +97 2 +98 2 diff --git a/sql/hive/src/test/resources/golden/subquery_exists_having-5-aafe13388d5795b26035167edd90a69b b/sql/hive/src/test/resources/golden/subquery_exists_having-5-aafe13388d5795b26035167edd90a69b new file mode 100644 index 000000000000..6278d429b33b --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_exists_having-5-aafe13388d5795b26035167edd90a69b @@ -0,0 +1,6 @@ +90 val_90 +92 val_92 +95 val_95 +96 val_96 +97 val_97 +98 val_98 diff --git a/core/src/test/resources/spark-events/local-1425081759269/SPARK_VERSION_1.2.0 b/sql/hive/src/test/resources/golden/subquery_in-0-d3f50875bd5dff172cf813fdb7d738eb old mode 100755 new mode 100644 similarity index 100% rename from core/src/test/resources/spark-events/local-1425081759269/SPARK_VERSION_1.2.0 rename to sql/hive/src/test/resources/golden/subquery_in-0-d3f50875bd5dff172cf813fdb7d738eb diff --git a/core/src/test/resources/spark-events/local-1426533911241/APPLICATION_COMPLETE b/sql/hive/src/test/resources/golden/subquery_in-1-dda16565b98926fc3587de937b9401c7 old mode 100755 new mode 100644 similarity index 100% rename from core/src/test/resources/spark-events/local-1426533911241/APPLICATION_COMPLETE rename to sql/hive/src/test/resources/golden/subquery_in-1-dda16565b98926fc3587de937b9401c7 diff --git a/core/src/test/resources/spark-events/local-1426533911241/SPARK_VERSION_1.2.0 b/sql/hive/src/test/resources/golden/subquery_in-10-3cd5ddc0f57e69745cbca1d5a8dd87c4 old mode 100755 new mode 100644 similarity index 100% rename from core/src/test/resources/spark-events/local-1426533911241/SPARK_VERSION_1.2.0 rename to sql/hive/src/test/resources/golden/subquery_in-10-3cd5ddc0f57e69745cbca1d5a8dd87c4 diff --git a/sql/hive/src/test/resources/golden/subquery_in-11-21659892bff071ffb0dec9134dd465a8 b/sql/hive/src/test/resources/golden/subquery_in-11-21659892bff071ffb0dec9134dd465a8 new file mode 100644 index 000000000000..ebc1f9f49aae --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in-11-21659892bff071ffb0dec9134dd465a8 @@ -0,0 +1,2 @@ +almond antique medium spring khaki 6 +almond antique salmon chartreuse burlywood 6 diff --git a/core/src/test/resources/spark-events/local-1426633911242/APPLICATION_COMPLETE b/sql/hive/src/test/resources/golden/subquery_in-12-79fc971b8a399c25e1e2a1a30e08f336 old mode 100755 new mode 100644 similarity index 100% rename from core/src/test/resources/spark-events/local-1426633911242/APPLICATION_COMPLETE rename to sql/hive/src/test/resources/golden/subquery_in-12-79fc971b8a399c25e1e2a1a30e08f336 diff --git a/sql/hive/src/test/resources/golden/subquery_in-13-f17e8105a6efd193ef1065110d1145a6 b/sql/hive/src/test/resources/golden/subquery_in-13-f17e8105a6efd193ef1065110d1145a6 new file mode 100644 index 000000000000..b97a52c4c3bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in-13-f17e8105a6efd193ef1065110d1145a6 @@ -0,0 +1,6 @@ +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 +Manufacturer#3 almond antique misty red olive 1 +Manufacturer#4 almond aquamarine yellow dodger mint 7 +Manufacturer#5 almond antique sky peru orange 2 diff --git a/core/src/test/resources/spark-events/local-1426633911242/SPARK_VERSION_1.2.0 b/sql/hive/src/test/resources/golden/subquery_in-14-df6d4aad4f4c5d0675b1fbceac367fe2 old mode 100755 new mode 100644 similarity index 100% rename from core/src/test/resources/spark-events/local-1426633911242/SPARK_VERSION_1.2.0 rename to sql/hive/src/test/resources/golden/subquery_in-14-df6d4aad4f4c5d0675b1fbceac367fe2 diff --git a/sql/hive/src/test/resources/golden/subquery_in-15-bacf85b0769b4030514a6f96c64d1ff7 b/sql/hive/src/test/resources/golden/subquery_in-15-bacf85b0769b4030514a6f96c64d1ff7 new file mode 100644 index 000000000000..a2b502fa1097 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in-15-bacf85b0769b4030514a6f96c64d1ff7 @@ -0,0 +1,490 @@ +10 val_10 +11 val_11 +12 val_12 +12 val_12 +15 val_15 +15 val_15 +17 val_17 +18 val_18 +18 val_18 +19 val_19 +20 val_20 +24 val_24 +24 val_24 +26 val_26 +26 val_26 +27 val_27 +28 val_28 +30 val_30 +33 val_33 +34 val_34 +35 val_35 +35 val_35 +35 val_35 +37 val_37 +37 val_37 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +51 val_51 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +67 val_67 +69 val_69 +70 val_70 +70 val_70 +70 val_70 +72 val_72 +72 val_72 +74 val_74 +76 val_76 +76 val_76 +77 val_77 +78 val_78 +80 val_80 +82 val_82 +83 val_83 +83 val_83 +84 val_84 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +90 val_90 +90 val_90 +90 val_90 +92 val_92 +95 val_95 +95 val_95 +96 val_96 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 diff --git a/sql/hive/src/test/resources/golden/subquery_in-16-d51e0128520c31dbe041ffa4ae22dd4b b/sql/hive/src/test/resources/golden/subquery_in-16-d51e0128520c31dbe041ffa4ae22dd4b new file mode 100644 index 000000000000..b97a52c4c3bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in-16-d51e0128520c31dbe041ffa4ae22dd4b @@ -0,0 +1,6 @@ +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#1 almond antique burnished rose metallic 2 +Manufacturer#2 almond aquamarine midnight light salmon 2 +Manufacturer#3 almond antique misty red olive 1 +Manufacturer#4 almond aquamarine yellow dodger mint 7 +Manufacturer#5 almond antique sky peru orange 2 diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-0-72ba9397f487a914380dc15afaef1058 b/sql/hive/src/test/resources/golden/subquery_in-17-5f132cdb7fc12e6389d620472df5ba7f similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_format_loc-0-72ba9397f487a914380dc15afaef1058 rename to sql/hive/src/test/resources/golden/subquery_in-17-5f132cdb7fc12e6389d620472df5ba7f diff --git a/sql/hive/src/test/resources/golden/subquery_in-18-f80281d529559f7f35ee5b42d53dd2ca b/sql/hive/src/test/resources/golden/subquery_in-18-f80281d529559f7f35ee5b42d53dd2ca new file mode 100644 index 000000000000..352142bbd1e1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in-18-f80281d529559f7f35ee5b42d53dd2ca @@ -0,0 +1,10 @@ +2320 9821 +4297 1798 +40216 217 +61336 8855 +64128 9141 +82704 7721 +108570 8571 +115118 7630 +115209 7721 +155190 7706 diff --git a/sql/hive/src/test/resources/golden/subquery_in-19-466013b596cc4160456daab670684af6 b/sql/hive/src/test/resources/golden/subquery_in-19-466013b596cc4160456daab670684af6 new file mode 100644 index 000000000000..b849cf75f218 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in-19-466013b596cc4160456daab670684af6 @@ -0,0 +1,2 @@ +4297 1798 +108570 8571 diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-10-71631c1e516c81ffdceac80f2d57ce09 b/sql/hive/src/test/resources/golden/subquery_in-2-374e39786feb745cd70f25be58bfa24 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_format_loc-10-71631c1e516c81ffdceac80f2d57ce09 rename to sql/hive/src/test/resources/golden/subquery_in-2-374e39786feb745cd70f25be58bfa24 diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-12-1553ad79b098b737ea8def91134eb0e9 b/sql/hive/src/test/resources/golden/subquery_in-3-42f922e862f882b9927abf566fe43050 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_format_loc-12-1553ad79b098b737ea8def91134eb0e9 rename to sql/hive/src/test/resources/golden/subquery_in-3-42f922e862f882b9927abf566fe43050 diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-15-bc83e8a2f8edf84f603109d14440dc83 b/sql/hive/src/test/resources/golden/subquery_in-4-c76f8bd9221a571ffdbbaa248570d31d similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_format_loc-15-bc83e8a2f8edf84f603109d14440dc83 rename to sql/hive/src/test/resources/golden/subquery_in-4-c76f8bd9221a571ffdbbaa248570d31d diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-17-7e411fcfdd8f169c503ed89dc56ee335 b/sql/hive/src/test/resources/golden/subquery_in-5-3cec6e623c64903b3c6204d0548f543b similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_format_loc-17-7e411fcfdd8f169c503ed89dc56ee335 rename to sql/hive/src/test/resources/golden/subquery_in-5-3cec6e623c64903b3c6204d0548f543b diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-19-56cadf0f555e355726dfed1929ad0508 b/sql/hive/src/test/resources/golden/subquery_in-6-8b37b644ebdb9007c609043c6c855cb0 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_format_loc-19-56cadf0f555e355726dfed1929ad0508 rename to sql/hive/src/test/resources/golden/subquery_in-6-8b37b644ebdb9007c609043c6c855cb0 diff --git a/sql/hive/src/test/resources/golden/subquery_in-7-208c9201161f60c2c7e521b0b33f0b19 b/sql/hive/src/test/resources/golden/subquery_in-7-208c9201161f60c2c7e521b0b33f0b19 new file mode 100644 index 000000000000..a2b502fa1097 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in-7-208c9201161f60c2c7e521b0b33f0b19 @@ -0,0 +1,490 @@ +10 val_10 +11 val_11 +12 val_12 +12 val_12 +15 val_15 +15 val_15 +17 val_17 +18 val_18 +18 val_18 +19 val_19 +20 val_20 +24 val_24 +24 val_24 +26 val_26 +26 val_26 +27 val_27 +28 val_28 +30 val_30 +33 val_33 +34 val_34 +35 val_35 +35 val_35 +35 val_35 +37 val_37 +37 val_37 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +51 val_51 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +67 val_67 +69 val_69 +70 val_70 +70 val_70 +70 val_70 +72 val_72 +72 val_72 +74 val_74 +76 val_76 +76 val_76 +77 val_77 +78 val_78 +80 val_80 +82 val_82 +83 val_83 +83 val_83 +84 val_84 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +90 val_90 +90 val_90 +90 val_90 +92 val_92 +95 val_95 +95 val_95 +96 val_96 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-2-bc83e8a2f8edf84f603109d14440dc83 b/sql/hive/src/test/resources/golden/subquery_in-8-d7212bf1f2c9e019b7142314b823a979 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_format_loc-2-bc83e8a2f8edf84f603109d14440dc83 rename to sql/hive/src/test/resources/golden/subquery_in-8-d7212bf1f2c9e019b7142314b823a979 diff --git a/sql/hive/src/test/resources/golden/subquery_in-9-3d9f3ef5aa4fbb982a28109af8db9805 b/sql/hive/src/test/resources/golden/subquery_in-9-3d9f3ef5aa4fbb982a28109af8db9805 new file mode 100644 index 000000000000..a2b502fa1097 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in-9-3d9f3ef5aa4fbb982a28109af8db9805 @@ -0,0 +1,490 @@ +10 val_10 +11 val_11 +12 val_12 +12 val_12 +15 val_15 +15 val_15 +17 val_17 +18 val_18 +18 val_18 +19 val_19 +20 val_20 +24 val_24 +24 val_24 +26 val_26 +26 val_26 +27 val_27 +28 val_28 +30 val_30 +33 val_33 +34 val_34 +35 val_35 +35 val_35 +35 val_35 +37 val_37 +37 val_37 +41 val_41 +42 val_42 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +51 val_51 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +67 val_67 +69 val_69 +70 val_70 +70 val_70 +70 val_70 +72 val_72 +72 val_72 +74 val_74 +76 val_76 +76 val_76 +77 val_77 +78 val_78 +80 val_80 +82 val_82 +83 val_83 +83 val_83 +84 val_84 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +90 val_90 +90 val_90 +90 val_90 +92 val_92 +95 val_95 +95 val_95 +96 val_96 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +134 val_134 +136 val_136 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +150 val_150 +152 val_152 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +214 val_214 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +218 val_218 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +222 val_222 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +235 val_235 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +241 val_241 +242 val_242 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +265 val_265 +266 val_266 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +307 val_307 +308 val_308 +309 val_309 +309 val_309 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +323 val_323 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +332 val_332 +333 val_333 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +353 val_353 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +367 val_367 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +424 val_424 +427 val_427 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +460 val_460 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +478 val_478 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 +498 val_498 +498 val_498 diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-4-7e411fcfdd8f169c503ed89dc56ee335 b/sql/hive/src/test/resources/golden/subquery_in_having-0-dda16565b98926fc3587de937b9401c7 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_format_loc-4-7e411fcfdd8f169c503ed89dc56ee335 rename to sql/hive/src/test/resources/golden/subquery_in_having-0-dda16565b98926fc3587de937b9401c7 diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-6-56cadf0f555e355726dfed1929ad0508 b/sql/hive/src/test/resources/golden/subquery_in_having-1-374e39786feb745cd70f25be58bfa24 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_format_loc-6-56cadf0f555e355726dfed1929ad0508 rename to sql/hive/src/test/resources/golden/subquery_in_having-1-374e39786feb745cd70f25be58bfa24 diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-7-cee355b012efdc3bc7d584268a7025c2 b/sql/hive/src/test/resources/golden/subquery_in_having-10-b8ded52f10f8103684cda7bba20d2201 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_format_loc-7-cee355b012efdc3bc7d584268a7025c2 rename to sql/hive/src/test/resources/golden/subquery_in_having-10-b8ded52f10f8103684cda7bba20d2201 diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 b/sql/hive/src/test/resources/golden/subquery_in_having-11-ddeeedb49ded9eb733a4792fff83abe4 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_format_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 rename to sql/hive/src/test/resources/golden/subquery_in_having-11-ddeeedb49ded9eb733a4792fff83abe4 diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-0-5fa6071842a0443346cf6db677a33412 b/sql/hive/src/test/resources/golden/subquery_in_having-2-877cbfc817ff3718f65073378a0c0829 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar1-0-5fa6071842a0443346cf6db677a33412 rename to sql/hive/src/test/resources/golden/subquery_in_having-2-877cbfc817ff3718f65073378a0c0829 diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-3-63a96439d273b9ad3304d3036bd79e35 b/sql/hive/src/test/resources/golden/subquery_in_having-3-63a96439d273b9ad3304d3036bd79e35 new file mode 100644 index 000000000000..0f66cd6930d8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in_having-3-63a96439d273b9ad3304d3036bd79e35 @@ -0,0 +1,303 @@ +10 1 +11 1 +12 2 +15 2 +17 1 +18 2 +19 1 +20 1 +24 2 +26 2 +27 1 +28 1 +30 1 +33 1 +34 1 +35 3 +37 2 +41 1 +42 2 +43 1 +44 1 +47 1 +51 2 +53 1 +54 1 +57 1 +58 2 +64 1 +65 1 +66 1 +67 2 +69 1 +70 3 +72 2 +74 1 +76 2 +77 1 +78 1 +80 1 +82 1 +83 2 +84 2 +85 1 +86 1 +87 1 +90 3 +92 1 +95 2 +96 1 +97 2 +98 2 +100 2 +103 2 +104 2 +105 1 +111 1 +113 2 +114 1 +116 1 +118 2 +119 3 +120 2 +125 2 +126 1 +128 3 +129 2 +131 1 +133 1 +134 2 +136 1 +137 2 +138 4 +143 1 +145 1 +146 2 +149 2 +150 1 +152 2 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 2 +165 2 +166 1 +167 3 +168 1 +169 4 +170 1 +172 2 +174 2 +175 2 +176 2 +177 1 +178 1 +179 2 +180 1 +181 1 +183 1 +186 1 +187 3 +189 1 +190 1 +191 2 +192 1 +193 3 +194 1 +195 2 +196 1 +197 2 +199 3 +200 2 +201 1 +202 1 +203 2 +205 2 +207 2 +208 3 +209 2 +213 2 +214 1 +216 2 +217 2 +218 1 +219 2 +221 2 +222 1 +223 2 +224 2 +226 1 +228 1 +229 2 +230 5 +233 2 +235 1 +237 2 +238 2 +239 2 +241 1 +242 2 +244 1 +247 1 +248 1 +249 1 +252 1 +255 2 +256 2 +257 1 +258 1 +260 1 +262 1 +263 1 +265 2 +266 1 +272 2 +273 3 +274 1 +275 1 +277 4 +278 2 +280 2 +281 2 +282 2 +283 1 +284 1 +285 1 +286 1 +287 1 +288 2 +289 1 +291 1 +292 1 +296 1 +298 3 +302 1 +305 1 +306 1 +307 2 +308 1 +309 2 +310 1 +311 3 +315 1 +316 3 +317 2 +318 3 +321 2 +322 2 +323 1 +325 2 +327 3 +331 2 +332 1 +333 2 +335 1 +336 1 +338 1 +339 1 +341 1 +342 2 +344 2 +345 1 +348 5 +351 1 +353 2 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 2 +368 1 +369 3 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 2 +384 3 +386 1 +389 1 +392 1 +393 1 +394 1 +395 2 +396 3 +397 2 +399 2 +400 1 +401 5 +402 1 +403 3 +404 2 +406 4 +407 1 +409 3 +411 1 +413 2 +414 2 +417 3 +418 1 +419 1 +421 1 +424 2 +427 1 +429 2 +430 3 +431 3 +432 1 +435 1 +436 1 +437 1 +438 3 +439 2 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 3 +455 1 +457 1 +458 2 +459 2 +460 1 +462 2 +463 2 +466 3 +467 1 +468 4 +469 5 +470 1 +472 1 +475 1 +477 1 +478 2 +479 1 +480 3 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 4 +490 1 +491 1 +492 2 +493 1 +494 1 +495 1 +496 1 +497 1 +498 3 diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-4-5d1259d48aa4b26931f1dbe686a0d2d7 b/sql/hive/src/test/resources/golden/subquery_in_having-4-5d1259d48aa4b26931f1dbe686a0d2d7 new file mode 100644 index 000000000000..52337d4d9809 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in_having-4-5d1259d48aa4b26931f1dbe686a0d2d7 @@ -0,0 +1,31 @@ +0 3 +5 3 +35 3 +70 3 +90 3 +119 3 +128 3 +167 3 +187 3 +193 3 +199 3 +208 3 +273 3 +298 3 +311 3 +316 3 +318 3 +327 3 +369 3 +384 3 +396 3 +403 3 +409 3 +417 3 +430 3 +431 3 +438 3 +454 3 +466 3 +480 3 +498 3 diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-1-be11cb1f18ab19550011417126264fea b/sql/hive/src/test/resources/golden/subquery_in_having-5-1beb605f3b9b0825c69dc5f52d085225 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar1-1-be11cb1f18ab19550011417126264fea rename to sql/hive/src/test/resources/golden/subquery_in_having-5-1beb605f3b9b0825c69dc5f52d085225 diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-10-c1a57b45952193d04b5411c5b6a31139 b/sql/hive/src/test/resources/golden/subquery_in_having-6-9543704852a4d71a85b90b85a0c5c0a5 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar1-10-c1a57b45952193d04b5411c5b6a31139 rename to sql/hive/src/test/resources/golden/subquery_in_having-6-9543704852a4d71a85b90b85a0c5c0a5 diff --git a/sql/hive/src/test/resources/golden/subquery_in_having-7-6bba00f0273f13733fadbe10b43876f5 b/sql/hive/src/test/resources/golden/subquery_in_having-7-6bba00f0273f13733fadbe10b43876f5 new file mode 100644 index 000000000000..6278d429b33b --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_in_having-7-6bba00f0273f13733fadbe10b43876f5 @@ -0,0 +1,6 @@ +90 val_90 +92 val_92 +95 val_95 +96 val_96 +97 val_97 +98 val_98 diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd b/sql/hive/src/test/resources/golden/subquery_in_having-8-662f1f7435da5d66fd4b09244387c06b similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd rename to sql/hive/src/test/resources/golden/subquery_in_having-8-662f1f7435da5d66fd4b09244387c06b diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c b/sql/hive/src/test/resources/golden/subquery_in_having-9-24ca942f094b14b92086305cc125e833 similarity index 100% rename from sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c rename to sql/hive/src/test/resources/golden/subquery_in_having-9-24ca942f094b14b92086305cc125e833 diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-14-5fa6071842a0443346cf6db677a33412 b/sql/hive/src/test/resources/golden/subquery_notexists-0-75cd3855b33f05667ae76896f4b25d3d similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar1-14-5fa6071842a0443346cf6db677a33412 rename to sql/hive/src/test/resources/golden/subquery_notexists-0-75cd3855b33f05667ae76896f4b25d3d diff --git a/sql/hive/src/test/resources/golden/subquery_notexists-1-4ae5bcc868eb27add076db2cb3ca9678 b/sql/hive/src/test/resources/golden/subquery_notexists-1-4ae5bcc868eb27add076db2cb3ca9678 new file mode 100644 index 000000000000..ce5158c00263 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_notexists-1-4ae5bcc868eb27add076db2cb3ca9678 @@ -0,0 +1,119 @@ +165 val_165 +193 val_193 +150 val_150 +128 val_128 +146 val_146 +152 val_152 +145 val_145 +15 val_15 +166 val_166 +153 val_153 +193 val_193 +174 val_174 +199 val_199 +174 val_174 +162 val_162 +167 val_167 +195 val_195 +17 val_17 +113 val_113 +155 val_155 +0 val_0 +128 val_128 +149 val_149 +129 val_129 +170 val_170 +157 val_157 +111 val_111 +169 val_169 +125 val_125 +192 val_192 +187 val_187 +176 val_176 +138 val_138 +103 val_103 +176 val_176 +137 val_137 +180 val_180 +12 val_12 +181 val_181 +138 val_138 +179 val_179 +172 val_172 +129 val_129 +158 val_158 +119 val_119 +0 val_0 +197 val_197 +100 val_100 +199 val_199 +191 val_191 +165 val_165 +120 val_120 +131 val_131 +156 val_156 +196 val_196 +197 val_197 +187 val_187 +137 val_137 +169 val_169 +0 val_0 +179 val_179 +118 val_118 +134 val_134 +138 val_138 +15 val_15 +118 val_118 +19 val_19 +10 val_10 +177 val_177 +11 val_11 +168 val_168 +143 val_143 +160 val_160 +195 val_195 +119 val_119 +149 val_149 +138 val_138 +103 val_103 +113 val_113 +167 val_167 +116 val_116 +191 val_191 +128 val_128 +2 val_2 +193 val_193 +104 val_104 +175 val_175 +105 val_105 +190 val_190 +114 val_114 +12 val_12 +164 val_164 +125 val_125 +164 val_164 +187 val_187 +104 val_104 +163 val_163 +119 val_119 +199 val_199 +120 val_120 +169 val_169 +178 val_178 +136 val_136 +172 val_172 +133 val_133 +175 val_175 +189 val_189 +134 val_134 +18 val_18 +100 val_100 +146 val_146 +186 val_186 +167 val_167 +18 val_18 +183 val_183 +152 val_152 +194 val_194 +126 val_126 +169 val_169 diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-2-ba9453c6b6a627286691f3930c2b26d0 b/sql/hive/src/test/resources/golden/subquery_notexists-2-73a67f6cae6d8e68efebdab4fbade162 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar1-2-ba9453c6b6a627286691f3930c2b26d0 rename to sql/hive/src/test/resources/golden/subquery_notexists-2-73a67f6cae6d8e68efebdab4fbade162 diff --git a/sql/hive/src/test/resources/golden/subquery_notexists-3-a8b49a691e12360c7c3fa5df113ba8cf b/sql/hive/src/test/resources/golden/subquery_notexists-3-a8b49a691e12360c7c3fa5df113ba8cf new file mode 100644 index 000000000000..ce5158c00263 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_notexists-3-a8b49a691e12360c7c3fa5df113ba8cf @@ -0,0 +1,119 @@ +165 val_165 +193 val_193 +150 val_150 +128 val_128 +146 val_146 +152 val_152 +145 val_145 +15 val_15 +166 val_166 +153 val_153 +193 val_193 +174 val_174 +199 val_199 +174 val_174 +162 val_162 +167 val_167 +195 val_195 +17 val_17 +113 val_113 +155 val_155 +0 val_0 +128 val_128 +149 val_149 +129 val_129 +170 val_170 +157 val_157 +111 val_111 +169 val_169 +125 val_125 +192 val_192 +187 val_187 +176 val_176 +138 val_138 +103 val_103 +176 val_176 +137 val_137 +180 val_180 +12 val_12 +181 val_181 +138 val_138 +179 val_179 +172 val_172 +129 val_129 +158 val_158 +119 val_119 +0 val_0 +197 val_197 +100 val_100 +199 val_199 +191 val_191 +165 val_165 +120 val_120 +131 val_131 +156 val_156 +196 val_196 +197 val_197 +187 val_187 +137 val_137 +169 val_169 +0 val_0 +179 val_179 +118 val_118 +134 val_134 +138 val_138 +15 val_15 +118 val_118 +19 val_19 +10 val_10 +177 val_177 +11 val_11 +168 val_168 +143 val_143 +160 val_160 +195 val_195 +119 val_119 +149 val_149 +138 val_138 +103 val_103 +113 val_113 +167 val_167 +116 val_116 +191 val_191 +128 val_128 +2 val_2 +193 val_193 +104 val_104 +175 val_175 +105 val_105 +190 val_190 +114 val_114 +12 val_12 +164 val_164 +125 val_125 +164 val_164 +187 val_187 +104 val_104 +163 val_163 +119 val_119 +199 val_199 +120 val_120 +169 val_169 +178 val_178 +136 val_136 +172 val_172 +133 val_133 +175 val_175 +189 val_189 +134 val_134 +18 val_18 +100 val_100 +146 val_146 +186 val_186 +167 val_167 +18 val_18 +183 val_183 +152 val_152 +194 val_194 +126 val_126 +169 val_169 diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d b/sql/hive/src/test/resources/golden/subquery_notexists_having-0-872612e3ae6ef1445982517a94200075 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d rename to sql/hive/src/test/resources/golden/subquery_notexists_having-0-872612e3ae6ef1445982517a94200075 diff --git a/sql/hive/src/test/resources/golden/subquery_notexists_having-1-8f6c09c8a89cc5939c1c309d660e7b3e b/sql/hive/src/test/resources/golden/subquery_notexists_having-1-8f6c09c8a89cc5939c1c309d660e7b3e new file mode 100644 index 000000000000..f722855aa13a --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_notexists_having-1-8f6c09c8a89cc5939c1c309d660e7b3e @@ -0,0 +1,14 @@ +0 val_0 +10 val_10 +11 val_11 +12 val_12 +100 val_100 +103 val_103 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +119 val_119 diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-6-f7d529dc66c022b64e0b287c82f92778 b/sql/hive/src/test/resources/golden/subquery_notexists_having-2-fb172ff54d6814f42360cb9f30f4882e similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar1-6-f7d529dc66c022b64e0b287c82f92778 rename to sql/hive/src/test/resources/golden/subquery_notexists_having-2-fb172ff54d6814f42360cb9f30f4882e diff --git a/sql/hive/src/test/resources/golden/subquery_notexists_having-3-edd8e7bbc4bfde58cf744fc0901e2ac b/sql/hive/src/test/resources/golden/subquery_notexists_having-3-edd8e7bbc4bfde58cf744fc0901e2ac new file mode 100644 index 000000000000..f722855aa13a --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_notexists_having-3-edd8e7bbc4bfde58cf744fc0901e2ac @@ -0,0 +1,14 @@ +0 val_0 +10 val_10 +11 val_11 +12 val_12 +100 val_100 +103 val_103 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +119 val_119 diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-8-bdde28ebc875c39f9630d95379eee68 b/sql/hive/src/test/resources/golden/subquery_notin_having-0-d3f50875bd5dff172cf813fdb7d738eb similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar1-8-bdde28ebc875c39f9630d95379eee68 rename to sql/hive/src/test/resources/golden/subquery_notin_having-0-d3f50875bd5dff172cf813fdb7d738eb diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-0-22c4186110b5770deaf7f03cf08326b7 b/sql/hive/src/test/resources/golden/subquery_notin_having-1-dda16565b98926fc3587de937b9401c7 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar2-0-22c4186110b5770deaf7f03cf08326b7 rename to sql/hive/src/test/resources/golden/subquery_notin_having-1-dda16565b98926fc3587de937b9401c7 diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc b/sql/hive/src/test/resources/golden/subquery_notin_having-2-374e39786feb745cd70f25be58bfa24 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc rename to sql/hive/src/test/resources/golden/subquery_notin_having-2-374e39786feb745cd70f25be58bfa24 diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-2-3a20c238eab602ad3d593b1eb3fa6dbb b/sql/hive/src/test/resources/golden/subquery_notin_having-3-21a44539fd357dc260687003554fe02a similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar2-2-3a20c238eab602ad3d593b1eb3fa6dbb rename to sql/hive/src/test/resources/golden/subquery_notin_having-3-21a44539fd357dc260687003554fe02a diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb b/sql/hive/src/test/resources/golden/subquery_notin_having-4-dea2fabba75cc13e7fa8df072f6b557b similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb rename to sql/hive/src/test/resources/golden/subquery_notin_having-4-dea2fabba75cc13e7fa8df072f6b557b diff --git a/sql/hive/src/test/resources/golden/subquery_notin_having-5-341feddde788c15197d08d7969dafe19 b/sql/hive/src/test/resources/golden/subquery_notin_having-5-341feddde788c15197d08d7969dafe19 new file mode 100644 index 000000000000..90cc9444dd13 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_notin_having-5-341feddde788c15197d08d7969dafe19 @@ -0,0 +1,2 @@ +Manufacturer#1 1173.15 +Manufacturer#2 1690.68 diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-6-3250407f20f3766c18f44b8bfae1829d b/sql/hive/src/test/resources/golden/subquery_notin_having-6-7ed33e3bcdc0728a69995ef0b2fa54a5 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar2-6-3250407f20f3766c18f44b8bfae1829d rename to sql/hive/src/test/resources/golden/subquery_notin_having-6-7ed33e3bcdc0728a69995ef0b2fa54a5 diff --git a/sql/hive/src/test/resources/golden/subquery_notin_having-7-44bdb73da0c1f4089b6edb43614e3e04 b/sql/hive/src/test/resources/golden/subquery_notin_having-7-44bdb73da0c1f4089b6edb43614e3e04 new file mode 100644 index 000000000000..90cc9444dd13 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery_notin_having-7-44bdb73da0c1f4089b6edb43614e3e04 @@ -0,0 +1,2 @@ +Manufacturer#1 1173.15 +Manufacturer#2 1690.68 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #3-0-76ee270337f664b36cacfc6528ac109 b/sql/hive/src/test/resources/golden/timestamp cast #3-0-76ee270337f664b36cacfc6528ac109 deleted file mode 100644 index d00491fd7e5b..000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #3-0-76ee270337f664b36cacfc6528ac109 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #5-0-dbd7bcd167d322d6617b884c02c7f247 b/sql/hive/src/test/resources/golden/timestamp cast #5-0-dbd7bcd167d322d6617b884c02c7f247 deleted file mode 100644 index 84a31a5a6970..000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #5-0-dbd7bcd167d322d6617b884c02c7f247 +++ /dev/null @@ -1 +0,0 @@ --0.001 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #7-0-1d70654217035f8ce5f64344f4c5a80f b/sql/hive/src/test/resources/golden/timestamp cast #7-0-1d70654217035f8ce5f64344f4c5a80f deleted file mode 100644 index 3fbedf693b51..000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #7-0-1d70654217035f8ce5f64344f4c5a80f +++ /dev/null @@ -1 +0,0 @@ --2 diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad b/sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-d9065e533430691d70b3370174fbbd50 similarity index 100% rename from sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-672d4cb385b7ced2e446f132474293ad rename to sql/hive/src/test/resources/golden/windowing.q -- 21. testDISTs-0-d9065e533430691d70b3370174fbbd50 diff --git a/sql/hive/src/test/resources/hive-site.xml b/sql/hive/src/test/resources/hive-site.xml new file mode 100644 index 000000000000..17297b3e22a7 --- /dev/null +++ b/sql/hive/src/test/resources/hive-site.xml @@ -0,0 +1,26 @@ + + + + + + + hive.in.test + true + Internal marker for test. + + diff --git a/sql/hive/src/test/resources/hive-test-path-helper.txt b/sql/hive/src/test/resources/hive-test-path-helper.txt new file mode 100644 index 000000000000..356b131ea114 --- /dev/null +++ b/sql/hive/src/test/resources/hive-test-path-helper.txt @@ -0,0 +1 @@ +This file is here so we can match on it and find the path to the current folder. diff --git a/sql/hive/src/test/resources/log4j.properties b/sql/hive/src/test/resources/log4j.properties index fea3404769d9..a48ae9fc5edd 100644 --- a/sql/hive/src/test/resources/log4j.properties +++ b/sql/hive/src/test/resources/log4j.properties @@ -59,3 +59,7 @@ log4j.logger.hive.ql.metadata.Hive=OFF log4j.additivity.org.apache.hadoop.hive.ql.io.RCFile=false log4j.logger.org.apache.hadoop.hive.ql.io.RCFile=ERROR + +# Parquet related logging +log4j.logger.org.apache.parquet.CorruptStatistics=ERROR +log4j.logger.parquet.CorruptStatistics=ERROR diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q deleted file mode 100644 index c640ca148b70..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q +++ /dev/null @@ -1,43 +0,0 @@ - -select - cdouble - ,Round(cdouble, 2) - ,Floor(cdouble) - ,Ceil(cdouble) - ,Rand(98007) as rnd - ,Exp(ln(cdouble)) - ,Ln(cdouble) - ,Ln(cfloat) - ,Log10(cdouble) - -- Use log2 as a representative function to test all input types. - ,Log2(cdouble) - ,Log2(cfloat) - ,Log2(cbigint) - ,Log2(cint) - ,Log2(csmallint) - ,Log2(ctinyint) - ,Log(2.0, cdouble) - ,Pow(log2(cdouble), 2.0) - ,Power(log2(cdouble), 2.0) - ,Sqrt(cdouble) - ,Sqrt(cbigint) - ,Bin(cbigint) - ,Hex(cdouble) - ,Conv(cbigint, 10, 16) - ,Abs(cdouble) - ,Abs(ctinyint) - ,Pmod(cint, 3) - ,Sin(cdouble) - ,Asin(cdouble) - ,Cos(cdouble) - ,ACos(cdouble) - ,Atan(cdouble) - ,Degrees(cdouble) - ,Radians(cdouble) - ,Positive(cdouble) - ,Positive(cbigint) - ,Negative(cdouble) - ,Sign(cdouble) - ,Sign(cbigint) -from alltypesorc order by rnd limit 400; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv deleted file mode 100644 index 51f231008f6d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv +++ /dev/null @@ -1 +0,0 @@ -SET hive.vectorized.execution.enabled = false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv deleted file mode 100644 index 18e02dc854ba..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv +++ /dev/null @@ -1 +0,0 @@ -SET hive.vectorized.execution.enabled = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q deleted file mode 100644 index 8f0a60b713ab..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q +++ /dev/null @@ -1,8 +0,0 @@ -SET hive.metastore.partition.name.whitelist.pattern=[\\x20-\\x7E&&[^,]]* ; --- This pattern matches all printable ASCII characters (disallow unicode) and disallows commas - -CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); -SHOW PARTITIONS part_whitelist_test; - -ALTER TABLE part_whitelist_test ADD PARTITION (ds='1,2,3,4'); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/addpart1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/addpart1.q deleted file mode 100644 index a7c9fe91f6cd..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/addpart1.q +++ /dev/null @@ -1,11 +0,0 @@ - -create table addpart1 (a int) partitioned by (b string, c string); - -alter table addpart1 add partition (b='f', c='s'); - -show partitions addpart1; - -alter table addpart1 add partition (b='f', c=''); - -show prtitions addpart1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q deleted file mode 100644 index 4193315d3004..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q +++ /dev/null @@ -1,16 +0,0 @@ -set hive.exec.concatenate.check.index=true; -create table src_rc_concatenate_test(key int, value string) stored as rcfile; - -load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_concatenate_test; -load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_concatenate_test; -load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_concatenate_test; - -show table extended like `src_rc_concatenate_test`; - -select count(1) from src_rc_concatenate_test; -select sum(hash(key)), sum(hash(value)) from src_rc_concatenate_test; - -create index src_rc_concatenate_test_index on table src_rc_concatenate_test(key) as 'compact' WITH DEFERRED REBUILD IDXPROPERTIES ("prop1"="val1", "prop2"="val2"); -show indexes on src_rc_concatenate_test; - -alter table src_rc_concatenate_test concatenate; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_non_native.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_non_native.q deleted file mode 100644 index 73ae85377883..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_non_native.q +++ /dev/null @@ -1,6 +0,0 @@ - -CREATE TABLE non_native1(key int, value string) -STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; - --- we do not support ALTER TABLE on non-native tables yet -ALTER TABLE non_native1 RENAME TO new_non_native; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q deleted file mode 100644 index e10f77cf3f16..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q +++ /dev/null @@ -1,11 +0,0 @@ --- create testing table -create table alter_coltype(key string, value string) partitioned by (dt string, ts string); - --- insert and create a partition -insert overwrite table alter_coltype partition(dt='100x', ts='6:30pm') select * from src1; - -desc alter_coltype; - --- alter partition change multiple keys at same time -alter table alter_coltype partition column (dt int, ts int); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q deleted file mode 100644 index 66eba75d4084..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q +++ /dev/null @@ -1,12 +0,0 @@ --- create testing table -create table alter_coltype(key string, value string) partitioned by (dt string, ts string); - --- insert and create a partition -insert overwrite table alter_coltype partition(dt='100x', ts='6:30pm') select * from src1; - -desc alter_coltype; - --- alter partition key column with invalid column name -alter table alter_coltype partition column (dd int); - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q deleted file mode 100644 index ad016c5f3a76..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q +++ /dev/null @@ -1,11 +0,0 @@ --- create testing table -create table alter_coltype(key string, value string) partitioned by (dt string, ts string); - --- insert and create a partition -insert overwrite table alter_coltype partition(dt='100x', ts='6:30pm') select * from src1; - -desc alter_coltype; - --- alter partition key column data type for ts column to a wrong type -alter table alter_coltype partition column (ts time); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q deleted file mode 100644 index 8cbb25cfa972..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q +++ /dev/null @@ -1,8 +0,0 @@ --- Create table -create table if not exists alter_part_invalidspec(key string, value string ) partitioned by (year string, month string) stored as textfile ; - --- Load data -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_invalidspec partition (year='1996', month='10'); -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_invalidspec partition (year='1996', month='12'); - -alter table alter_part_invalidspec partition (year='1997') enable no_drop; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q deleted file mode 100644 index 3c0ff02b1ac1..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q +++ /dev/null @@ -1,9 +0,0 @@ --- Create table -create table if not exists alter_part_nodrop_part(key string, value string ) partitioned by (year string, month string) stored as textfile ; - --- Load data -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_part partition (year='1996', month='10'); -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_part partition (year='1996', month='12'); - -alter table alter_part_nodrop_part partition (year='1996') enable no_drop; -alter table alter_part_nodrop_part drop partition (year='1996'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q deleted file mode 100644 index f2135b1aa02e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q +++ /dev/null @@ -1,9 +0,0 @@ --- Create table -create table if not exists alter_part_nodrop_table(key string, value string ) partitioned by (year string, month string) stored as textfile ; - --- Load data -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_table partition (year='1996', month='10'); -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_table partition (year='1996', month='12'); - -alter table alter_part_nodrop_table partition (year='1996') enable no_drop; -drop table alter_part_nodrop_table; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q deleted file mode 100644 index 7376d8bfe4a7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q +++ /dev/null @@ -1,11 +0,0 @@ --- create table -create table if not exists alter_part_offline (key string, value string ) partitioned by (year string, month string) stored as textfile ; - --- Load data -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_offline partition (year='1996', month='10'); -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_offline partition (year='1996', month='12'); - -alter table alter_part_offline partition (year='1996') disable offline; -select * from alter_part_offline where year = '1996'; -alter table alter_part_offline partition (year='1996') enable offline; -select * from alter_part_offline where year = '1996'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q deleted file mode 100644 index 6e33bc0782d2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q +++ /dev/null @@ -1,9 +0,0 @@ -SET hive.metastore.partition.name.whitelist.pattern=[\\x20-\\x7E&&[^,]]* ; --- This pattern matches all printable ASCII characters (disallow unicode) and disallows commas - -CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); -SHOW PARTITIONS part_whitelist_test; - -ALTER TABLE part_whitelist_test ADD PARTITION (ds='1'); - -ALTER TABLE part_whitelist_test PARTITION (ds='1') rename to partition (ds='1,2,3'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q deleted file mode 100644 index be971f184986..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q +++ /dev/null @@ -1,6 +0,0 @@ -create table alter_rename_partition_src ( col1 string ) stored as textfile ; -load data local inpath '../../data/files/test.dat' overwrite into table alter_rename_partition_src ; -create table alter_rename_partition ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile; -insert overwrite table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') select col1 from alter_rename_partition_src ; - -alter table alter_rename_partition partition (pCol1='nonexist_part1:', pcol2='nonexist_part2:') rename to partition (pCol1='new_part1:', pcol2='new_part2:'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q deleted file mode 100644 index 4babdda2dbe2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q +++ /dev/null @@ -1,6 +0,0 @@ -create table alter_rename_partition_src ( col1 string ) stored as textfile ; -load data local inpath '../../data/files/test.dat' overwrite into table alter_rename_partition_src ; -create table alter_rename_partition ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile; -insert overwrite table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') select col1 from alter_rename_partition_src ; - -alter table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') rename to partition (pCol1='old_part1:', pcol2='old_part2:'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q deleted file mode 100644 index 3af807ef6121..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q +++ /dev/null @@ -1,6 +0,0 @@ -create table alter_rename_partition_src ( col1 string ) stored as textfile ; -load data local inpath '../../data/files/test.dat' overwrite into table alter_rename_partition_src ; -create table alter_rename_partition ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile; -insert overwrite table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') select col1 from alter_rename_partition_src ; - -alter table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') rename to partition (pCol1='old_part1:', pcol2='old_part2:', pcol3='old_part3:'); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_add_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_add_partition.q deleted file mode 100644 index 2427c3b2a45f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_add_partition.q +++ /dev/null @@ -1,5 +0,0 @@ -create table mp (a int) partitioned by (b int); - --- should fail -alter table mp add partition (b='1', c='1'); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q deleted file mode 100644 index fad194d016ec..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q +++ /dev/null @@ -1,7 +0,0 @@ -drop table aa; -create table aa ( test STRING ) - ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.RegexSerDe' - WITH SERDEPROPERTIES ("input.regex" = "(.*)", "output.format.string" = "$1s"); - -alter table aa set serdeproperties ("input.regex" = "[^\\](.*)", "output.format.string" = "$1s"); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q deleted file mode 100644 index 30fe4d9916ab..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q +++ /dev/null @@ -1,4 +0,0 @@ -DROP VIEW testView; - --- Cannot ALTER VIEW AS SELECT if view currently does not exist -ALTER VIEW testView AS SELECT * FROM srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q deleted file mode 100644 index dca6770b1b17..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q +++ /dev/null @@ -1,12 +0,0 @@ -CREATE VIEW testViewPart PARTITIONED ON (value) -AS -SELECT key, value -FROM src -WHERE key=86; - -ALTER VIEW testViewPart -ADD PARTITION (value='val_86') PARTITION (value='val_xyz'); -DESCRIBE FORMATTED testViewPart; - --- If a view has partition, could not replace it with ALTER VIEW AS SELECT -ALTER VIEW testViewPart as SELECT * FROM srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure.q deleted file mode 100644 index 705b985095fa..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure.q +++ /dev/null @@ -1,3 +0,0 @@ -DROP VIEW xxx3; -CREATE VIEW xxx3 AS SELECT * FROM src; -ALTER TABLE xxx3 REPLACE COLUMNS (xyz int); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure2.q deleted file mode 100644 index 26d2c4f3ad2f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure2.q +++ /dev/null @@ -1,8 +0,0 @@ -DROP VIEW xxx4; -CREATE VIEW xxx4 -PARTITIONED ON (value) -AS -SELECT * FROM src; - --- should fail: need to use ALTER VIEW, not ALTER TABLE -ALTER TABLE xxx4 ADD PARTITION (value='val_86'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure3.q deleted file mode 100644 index 49c17a8b573c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure3.q +++ /dev/null @@ -1,2 +0,0 @@ --- should fail: can't use ALTER VIEW on a table -ALTER VIEW srcpart ADD PARTITION (ds='2012-12-31', hr='23'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure4.q deleted file mode 100644 index e2fad270b1d8..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure4.q +++ /dev/null @@ -1,8 +0,0 @@ -DROP VIEW xxx5; -CREATE VIEW xxx5 -PARTITIONED ON (value) -AS -SELECT * FROM src; - --- should fail: LOCATION clause is illegal -ALTER VIEW xxx5 ADD PARTITION (value='val_86') LOCATION '/foo/bar/baz'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure5.q deleted file mode 100644 index e44766e11306..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure5.q +++ /dev/null @@ -1,8 +0,0 @@ -DROP VIEW xxx6; -CREATE VIEW xxx6 -PARTITIONED ON (value) -AS -SELECT * FROM src; - --- should fail: partition column name does not match -ALTER VIEW xxx6 ADD PARTITION (v='val_86'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure6.q deleted file mode 100644 index dab7b145f7c4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure6.q +++ /dev/null @@ -1,11 +0,0 @@ -DROP VIEW xxx7; -CREATE VIEW xxx7 -PARTITIONED ON (key) -AS -SELECT hr,key FROM srcpart; - -SET hive.mapred.mode=strict; - --- strict mode should cause this to fail since view partition --- predicate does not correspond to an underlying table partition predicate -ALTER VIEW xxx7 ADD PARTITION (key=10); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure7.q deleted file mode 100644 index eff04c5b47de..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure7.q +++ /dev/null @@ -1,8 +0,0 @@ -DROP VIEW xxx8; -CREATE VIEW xxx8 -PARTITIONED ON (ds,hr) -AS -SELECT key,ds,hr FROM srcpart; - --- should fail: need to fill in all partition columns -ALTER VIEW xxx8 ADD PARTITION (ds='2011-01-01'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure8.q deleted file mode 100644 index 9dff78425061..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure8.q +++ /dev/null @@ -1,3 +0,0 @@ --- should fail: can't use ALTER VIEW on a table -CREATE TABLE invites (foo INT, bar STRING) PARTITIONED BY (ds STRING); -ALTER VIEW invites RENAME TO invites2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure9.q deleted file mode 100644 index 0f40fad90d97..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure9.q +++ /dev/null @@ -1,7 +0,0 @@ -DROP VIEW xxx4; -CREATE VIEW xxx4 -AS -SELECT * FROM src; - --- should fail: need to use ALTER VIEW, not ALTER TABLE -ALTER TABLE xxx4 RENAME TO xxx4a; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/altern1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/altern1.q deleted file mode 100644 index 60414c1f3a7a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/altern1.q +++ /dev/null @@ -1,4 +0,0 @@ - -create table altern1(a int, b int) partitioned by (ds string); -alter table altern1 replace columns(a int, b int, ds string); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col.q deleted file mode 100644 index 866cec126f78..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col.q +++ /dev/null @@ -1 +0,0 @@ -FROM (SELECT key, concat(value) AS key FROM src) a SELECT a.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col0.q deleted file mode 100644 index 46349c60bc79..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col0.q +++ /dev/null @@ -1,2 +0,0 @@ --- TOK_ALLCOLREF -explain select * from (select * from (select * from src) a join (select * from src1) b on (a.key = b.key)) t; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q deleted file mode 100644 index 9e8bcbd1bbf7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.support.quoted.identifiers=none; --- TOK_TABLE_OR_COL -explain select * from (select `.*` from (select * from src) a join (select * from src1) b on (a.key = b.key)) t; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q deleted file mode 100644 index 33d4aed3cd9a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.support.quoted.identifiers=none; --- DOT -explain select * from (select a.`[kv].*`, b.`[kv].*` from (select * from src) a join (select * from src1) b on (a.key = b.key)) t; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze.q deleted file mode 100644 index 874f5bfc1412..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze.q +++ /dev/null @@ -1 +0,0 @@ -analyze table srcpart compute statistics; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze1.q deleted file mode 100644 index 057a1a0b482e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze1.q +++ /dev/null @@ -1 +0,0 @@ -analyze table srcpart partition (key) compute statistics; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q deleted file mode 100644 index 78a97019f192..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q +++ /dev/null @@ -1 +0,0 @@ -analyze table nonexistent compute statistics; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_view.q deleted file mode 100644 index af4970f52e8b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_view.q +++ /dev/null @@ -1,6 +0,0 @@ -DROP VIEW av; - -CREATE VIEW av AS SELECT * FROM src; - --- should fail: can't analyze a view...yet -ANALYZE TABLE av COMPUTE STATISTICS; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive1.q deleted file mode 100644 index a4b50f5e1410..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive1.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition twice. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE srcpart_archived LIKE srcpart; - -INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE srcpart_archived ARCHIVE PARTITION (ds='2008-04-08', hr='12'); -ALTER TABLE srcpart_archived ARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive2.q deleted file mode 100644 index ff8dcb248568..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive2.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to unarchive a non-archived partition --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -drop table tstsrcpart; -create table tstsrcpart like srcpart; -insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='12') -select key, value from srcpart where ds='2008-04-08' and hr='12'; - -ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive3.q deleted file mode 100644 index 53057daa1b62..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive3.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.archive.enabled = true; --- Tests archiving a table --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -ALTER TABLE srcpart ARCHIVE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive4.q deleted file mode 100644 index 56d6f1798deb..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive4.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.archive.enabled = true; --- Tests archiving multiple partitions --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -ALTER TABLE srcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12') PARTITION (ds='2008-04-08', hr='11'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive5.q deleted file mode 100644 index 4f6dc8d72cee..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive5.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.archive.enabled = true; --- Tests creating a partition where the partition value will collide with the --- a intermediate directory - -ALTER TABLE srcpart ADD PARTITION (ds='2008-04-08', hr='14_INTERMEDIATE_ORIGINAL') diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q deleted file mode 100644 index 130b37b5c9d5..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q +++ /dev/null @@ -1,18 +0,0 @@ -USE default; - -set hive.archive.enabled = true; -set hive.enforce.bucketing = true; - -drop table tstsrcpart; - -create table tstsrcpart like srcpart; - --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20) --- The version of GzipCodec that is provided in Hadoop 0.20 silently ignores --- file format errors. However, versions of Hadoop that include --- HADOOP-6835 (e.g. 0.23 and 1.x) cause a Wrong File Format exception --- to be thrown during the LOAD step. This former behavior is tested --- in clientpositive/archive_corrupt.q - -load data local inpath '../../data/files/archive_corrupt.rc' overwrite into table tstsrcpart partition (ds='2008-04-08', hr='11'); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert1.q deleted file mode 100644 index deaff63d673a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert1.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to insert into archived partition. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert2.q deleted file mode 100644 index d744f2487694..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert2.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to insert into archived partition. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert3.q deleted file mode 100644 index c6cb142824c8..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert3.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to create partition inside of archived directory. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert4.q deleted file mode 100644 index c36f3ef9e877..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert4.q +++ /dev/null @@ -1,15 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to (possible) dynamic insert into archived partition. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); - -SET hive.exec.dynamic.partition=true; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr) -SELECT key, value, hr FROM srcpart WHERE ds='2008-04-08' AND hr='12'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi1.q deleted file mode 100644 index 8c702ed008bf..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi1.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition twice. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi2.q deleted file mode 100644 index d3cfb89c9874..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi2.q +++ /dev/null @@ -1,12 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to unarchive a non-archived partition group --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -drop table tstsrcpart; -create table tstsrcpart like srcpart; -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; -insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='12') -select key, value from srcpart where ds='2008-04-08' and hr='12'; - -ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi3.q deleted file mode 100644 index 75f5dfad47b3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi3.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive outer partition group containing other partition inside. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi4.q deleted file mode 100644 index abe0647ae6ee..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi4.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive inner partition contained in archived partition group. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi5.q deleted file mode 100644 index 71635e054a1e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi5.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to unarchive outer partition group containing other partition inside. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); -ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi6.q deleted file mode 100644 index 5bb1474fdc38..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi6.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to unarchive inner partition contained in archived partition group. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); -ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi7.q deleted file mode 100644 index db7f392737e9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi7.q +++ /dev/null @@ -1,12 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition group with custom locations. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; -ALTER TABLE tstsrcpart ADD PARTITION (ds='2008-04-08', hr='12') -LOCATION "${system:test.tmp.dir}/tstsrc"; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec1.q deleted file mode 100644 index d83b19d9fe31..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec1.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition twice. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE srcpart_archived LIKE srcpart; - -INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE srcpart_archived ARCHIVE PARTITION (ds='2008-04-08', nonexistingpart='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec2.q deleted file mode 100644 index ed14bbf688d5..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec2.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition twice. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE srcpart_archived LIKE srcpart; - -INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE srcpart_archived ARCHIVE PARTITION (hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec3.q deleted file mode 100644 index f27ad6d63b08..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec3.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition twice. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE srcpart_archived LIKE srcpart; - -INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE srcpart_archived ARCHIVE PARTITION (); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec4.q deleted file mode 100644 index 491c2ac4596f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec4.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition twice. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE srcpart_archived LIKE srcpart; - -INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE srcpart_archived ARCHIVE PARTITION (hr='12', ds='2008-04-08'); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec5.q deleted file mode 100644 index bb25ef2c7e0f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec5.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition twice. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE srcpart_archived (key string, value string) partitioned by (ds string, hr int, min int); - -INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12', min='00') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE srcpart_archived ARCHIVE PARTITION (ds='2008-04-08', min='00'); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q deleted file mode 100644 index a1709dae5f5b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q +++ /dev/null @@ -1,7 +0,0 @@ -set hive.security.authorization.enabled=true; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory; - --- running a sql query to initialize the authorization - not needed in real HS2 mode -show tables; - -add jar ${system:maven.local.repository}/org/apache/hive/hcatalog/hive-hcatalog-core/${system:hive.version}/hive-hcatalog-core-${system:hive.version}.jar; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q deleted file mode 100644 index 8abdd2b3cde8..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -set user.name=user1; --- check add partition without insert privilege -create table tpart(i int, j int) partitioned by (k string); - -set user.name=user2; -alter table tpart add partition (k = 'abc'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q deleted file mode 100644 index f716262e23bb..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if alter table owner fails --- for now, alter db owner is allowed only for admin - -create database dbao; -alter database dbao set owner user user2; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q deleted file mode 100644 index f9049350180e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q +++ /dev/null @@ -1,8 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if alter table owner fails -alter database default set owner user user1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q deleted file mode 100644 index de91e9192330..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; -create role all; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q deleted file mode 100644 index 42a42f65b28a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; -create role default; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q deleted file mode 100644 index 0d14cde6d546..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; -create role None; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q deleted file mode 100644 index d5ea284f1474..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q +++ /dev/null @@ -1,17 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; - -create role testrole; -show roles; -drop role TESTROLE; -show roles; -create role TESTROLE; -show roles; -grant role testROLE to user hive_admin_user; -set role testrolE; -set role adMin; -show roles; -create role TESTRoLE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q deleted file mode 100644 index 02bbe090cfba..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q +++ /dev/null @@ -1,7 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=hive_test_user; - --- permanent function creation should fail for non-admin roles -create function perm_fn as 'org.apache.hadoop.hive.ql.udf.UDFAscii'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q deleted file mode 100644 index 8760fa8d8225..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q +++ /dev/null @@ -1,8 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=hive_test_user; - --- temp function creation should fail for non-admin roles -create temporary function temp_fn as 'org.apache.hadoop.hive.ql.udf.UDFAscii'; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q deleted file mode 100644 index c904a100c515..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q +++ /dev/null @@ -1,8 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=hive_test_user; - --- temp macro creation should fail for non-admin roles -create temporary macro mymacro1(x double) x * x; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q deleted file mode 100644 index a84fe64bd618..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; --- this test will fail because hive_test_user is not in admin role. -create role r1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q deleted file mode 100644 index 9b1f2ea6c6ac..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - --- check create view without select privileges -create table t1(i int); -set user.name=user1; -create view v1 as select * from t1; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q deleted file mode 100644 index 1cf74a365d79..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - --- check query without select privilege fails -create table t1(i int); - -set user.name=user1; -create table t2 as select * from t1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q deleted file mode 100644 index 47663c9bb93e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if alter table fails as different user -create table t1(i int); -desc t1; - -grant all on table t1 to user user2; -revoke select on table t1 from user user2; - -set user.name=user2; -desc t1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q deleted file mode 100644 index 7d47a7b64967..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q +++ /dev/null @@ -1,7 +0,0 @@ -set hive.security.authorization.enabled=true; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory; - --- running a sql query to initialize the authorization - not needed in real HS2 mode -show tables; -dfs -ls ${system:test.tmp.dir}/ - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q deleted file mode 100644 index 64b300c8d9b2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set role ALL; -SELECT TRANSFORM (*) USING 'cat' AS (key, value) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q deleted file mode 100644 index edeae9b71d7a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q +++ /dev/null @@ -1,22 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- ensure that drop database cascade works -create database dba1; -create table dba1.tab1(i int); -drop database dba1 cascade; - --- check if drop database fails if the db has a table for which user does not have permission -create database dba2; -create table dba2.tab2(i int); - -set user.name=hive_admin_user; -set role ADMIN; -alter database dba2 set owner user user2; - -set user.name=user2; -show current roles; -drop database dba2 cascade ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q deleted file mode 100644 index 46d4d0f92c8e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q +++ /dev/null @@ -1,27 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if changing owner and dropping as other user works -create database dba1; - -set user.name=hive_admin_user; -set role ADMIN; -alter database dba1 set owner user user2; - -set user.name=user2; -show current roles; -drop database dba1; - - -set user.name=user1; --- check if dropping db as another user fails -show current roles; -create database dba2; - -set user.name=user2; -show current roles; - -drop database dba2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q deleted file mode 100644 index a7aa17f5abfc..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; -show current roles; -create role r1; -set role ALL; -show current roles; -drop role r1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q deleted file mode 100644 index f05e9458fa80..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/authz_drop_part_1; - --- check drop partition without delete privilege -create table tpart(i int, j int) partitioned by (k string); -alter table tpart add partition (k = 'abc') location 'file:${system:test.tmp.dir}/authz_drop_part_1' ; -set user.name=user1; -alter table tpart drop partition (k = 'abc'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_1.q deleted file mode 100644 index c38dab5eb702..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_1.q +++ /dev/null @@ -1,7 +0,0 @@ -create table authorization_fail_1 (key int, value string); -set hive.security.authorization.enabled=true; - -grant Create on table authorization_fail_1 to user hive_test_user; -grant Create on table authorization_fail_1 to user hive_test_user; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_2.q deleted file mode 100644 index 341e44774d9c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_2.q +++ /dev/null @@ -1,7 +0,0 @@ -create table authorization_fail_2 (key int, value string) partitioned by (ds string); - -set hive.security.authorization.enabled=true; - -alter table authorization_fail_2 add partition (ds='2010'); - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_3.q deleted file mode 100644 index 6a56daa05fee..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_3.q +++ /dev/null @@ -1,12 +0,0 @@ --- SORT_BEFORE_DIFF - -create table authorization_fail_3 (key int, value string) partitioned by (ds string); -set hive.security.authorization.enabled=true; - -grant Create on table authorization_fail_3 to user hive_test_user; -alter table authorization_fail_3 add partition (ds='2010'); - -show grant user hive_test_user on table authorization_fail_3; -show grant user hive_test_user on table authorization_fail_3 partition (ds='2010'); - -select key from authorization_fail_3 where ds='2010'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_4.q deleted file mode 100644 index f0cb6459a255..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_4.q +++ /dev/null @@ -1,15 +0,0 @@ --- SORT_BEFORE_DIFF - -create table authorization_fail_4 (key int, value string) partitioned by (ds string); - -set hive.security.authorization.enabled=true; -grant Alter on table authorization_fail_4 to user hive_test_user; -ALTER TABLE authorization_fail_4 SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); - -grant Create on table authorization_fail_4 to user hive_test_user; -alter table authorization_fail_4 add partition (ds='2010'); - -show grant user hive_test_user on table authorization_fail_4; -show grant user hive_test_user on table authorization_fail_4 partition (ds='2010'); - -select key from authorization_fail_4 where ds='2010'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_5.q deleted file mode 100644 index b4efab5667f6..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_5.q +++ /dev/null @@ -1,20 +0,0 @@ --- SORT_BEFORE_DIFF - -create table authorization_fail (key int, value string) partitioned by (ds string); -set hive.security.authorization.enabled=true; - -grant Alter on table authorization_fail to user hive_test_user; -ALTER TABLE authorization_fail SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); - -grant Create on table authorization_fail to user hive_test_user; -grant Select on table authorization_fail to user hive_test_user; -alter table authorization_fail add partition (ds='2010'); - -show grant user hive_test_user on table authorization_fail; -show grant user hive_test_user on table authorization_fail partition (ds='2010'); - -revoke Select on table authorization_fail partition (ds='2010') from user hive_test_user; - -show grant user hive_test_user on table authorization_fail partition (ds='2010'); - -select key from authorization_fail where ds='2010'; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_6.q deleted file mode 100644 index 977246948cad..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_6.q +++ /dev/null @@ -1,6 +0,0 @@ --- SORT_BEFORE_DIFF - -create table authorization_part_fail (key int, value string) partitioned by (ds string); -set hive.security.authorization.enabled=true; - -ALTER TABLE authorization_part_fail SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_7.q deleted file mode 100644 index 492deed10bfe..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_7.q +++ /dev/null @@ -1,17 +0,0 @@ --- SORT_BEFORE_DIFF - -create table authorization_fail (key int, value string); - -set hive.security.authorization.enabled=true; - -create role hive_test_role_fail; - -grant role hive_test_role_fail to user hive_test_user; -grant select on table authorization_fail to role hive_test_role_fail; -show role grant user hive_test_user; - -show grant role hive_test_role_fail on table authorization_fail; - -drop role hive_test_role_fail; - -select key from authorization_fail; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q deleted file mode 100644 index d969e39027e9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.security.authorization.enabled=true; - -create database db_to_fail; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q deleted file mode 100644 index 87719b0043e2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.security.authorization.enabled=false; -create database db_fail_to_drop; -set hive.security.authorization.enabled=true; - -drop database db_fail_to_drop; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q deleted file mode 100644 index f3c86b97ce76..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set user.name=user1; --- current user has been set (comment line before the set cmd is resulting in parse error!!) - -CREATE TABLE table_priv_allf(i int); - --- grant insert to user2 WITH grant option -GRANT INSERT ON table_priv_allf TO USER user2 with grant option; - -set user.name=user2; --- try grant all to user3, without having all privileges -GRANT ALL ON table_priv_allf TO USER user3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q deleted file mode 100644 index 7808cb3ec7b3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q +++ /dev/null @@ -1,16 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set user.name=user1; --- current user has been set (comment line before the set cmd is resulting in parse error!!) - -CREATE TABLE tauth_gdup(i int); - --- It should be possible to revert owners privileges -revoke SELECT ON tauth_gdup from user user1; - -show grant user user1 on table tauth_gdup; - --- Owner already has all privileges granted, another grant would become duplicate --- and result in error -GRANT INSERT ON tauth_gdup TO USER user1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q deleted file mode 100644 index 8dc8e45a7907..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set user.name=user1; --- current user has been set (comment line before the set cmd is resulting in parse error!!) - -CREATE TABLE table_priv_gfail1(i int); - -set user.name=user2; --- try grant insert to user3 as user2 -GRANT INSERT ON table_priv_gfail1 TO USER user3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q deleted file mode 100644 index d51c1c3507ee..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set user.name=user1; --- current user has been set (comment line before the set cmd is resulting in parse error!!) - -CREATE TABLE table_priv_gfail1(i int); - --- grant insert to user2 WITHOUT grant option -GRANT INSERT ON table_priv_gfail1 TO USER user2; - -set user.name=user2; --- try grant insert to user3 -GRANT INSERT ON table_priv_gfail1 TO USER user3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q deleted file mode 100644 index 2fa3cb260b07..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - --- check insert without select priv -create table t1(i int); - -set user.name=user1; -create table user2tab(i int); -insert into table t1 select * from user2tab; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q deleted file mode 100644 index b9bee4ea40d4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - --- check insert without select priv -create table t1(i int); - -set user.name=user1; -create table t2(i int); -insert into table t2 select * from t1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q deleted file mode 100644 index 2a1da23daeb1..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q +++ /dev/null @@ -1,6 +0,0 @@ -create table if not exists authorization_invalid_v1 (key int, value string); -grant delete on table authorization_invalid_v1 to user hive_test_user; -drop table authorization_invalid_v1; - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q deleted file mode 100644 index 9c724085d990..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; - -create table if not exists authorization_invalid_v2 (key int, value string); -grant index on table authorization_invalid_v2 to user hive_test_user; -drop table authorization_invalid_v2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q deleted file mode 100644 index 8a3300cb2e37..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if alter table fails as different user -create table t1(i int); - -set user.name=user2; -alter table t1 rename to tnew1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q deleted file mode 100644 index 0172c4c74c82..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if alter table fails as different user -create table t1(i int); - -set user.name=user2; -ALTER TABLE t1 SET SERDEPROPERTIES ('field.delim' = ','); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q deleted file mode 100644 index 2d0e52da008d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if create table fails as different user -create table t1(i int); - -set user.name=user2; -drop table t1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q deleted file mode 100644 index 76bbab42b375..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if create table fails as different user -create table t1(i int); -create view vt1 as select * from t1; - -set user.name=user2; -drop view vt1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_part.q deleted file mode 100644 index a654a2380c75..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_part.q +++ /dev/null @@ -1,37 +0,0 @@ --- SORT_BEFORE_DIFF - -create table authorization_part_fail (key int, value string) partitioned by (ds string); -ALTER TABLE authorization_part_fail SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); -create table src_auth as select * from src; -set hive.security.authorization.enabled=true; - -grant Create on table authorization_part_fail to user hive_test_user; -grant Update on table authorization_part_fail to user hive_test_user; -grant Drop on table authorization_part_fail to user hive_test_user; -grant select on table src_auth to user hive_test_user; - --- column grant to group - -grant select(key) on table authorization_part_fail to group hive_test_group1; -grant select on table authorization_part_fail to group hive_test_group1; - -show grant group hive_test_group1 on table authorization_part_fail; - -insert overwrite table authorization_part_fail partition (ds='2010') select key, value from src_auth; -show grant group hive_test_group1 on table authorization_part_fail(key) partition (ds='2010'); -show grant group hive_test_group1 on table authorization_part_fail partition (ds='2010'); -select key, value from authorization_part_fail where ds='2010' order by key limit 20; - -insert overwrite table authorization_part_fail partition (ds='2011') select key, value from src_auth; -show grant group hive_test_group1 on table authorization_part_fail(key) partition (ds='2011'); -show grant group hive_test_group1 on table authorization_part_fail partition (ds='2011'); -select key, value from authorization_part_fail where ds='2011' order by key limit 20; - -select key,value, ds from authorization_part_fail where ds>='2010' order by key, ds limit 20; - -revoke select on table authorization_part_fail partition (ds='2010') from group hive_test_group1; - -select key,value, ds from authorization_part_fail where ds>='2010' order by key, ds limit 20; - -drop table authorization_part_fail; -drop table src_auth; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q deleted file mode 100644 index bbf3b66970b6..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q +++ /dev/null @@ -1,29 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; - --- the test verifies that authorization is happening with privileges of the current roles - --- grant privileges with grant option for table to role2 -create role role2; -grant role role2 to user user2; -create table tpriv_current_role(i int); -grant all on table tpriv_current_role to role role2 with grant option; - -set user.name=user2; --- switch to user2 - --- by default all roles should be in current roles, and grant to new user should work -show current roles; -grant all on table tpriv_current_role to user user3; - -set role role2; --- switch to role2, grant should work -grant all on table tpriv_current_role to user user4; -show grant user user4 on table tpriv_current_role; - -set role PUBLIC; --- set role to public, should fail as role2 is not one of the current roles -grant all on table tpriv_current_role to user user5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q deleted file mode 100644 index 002389f203e2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q +++ /dev/null @@ -1 +0,0 @@ -create role PUBLIC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q deleted file mode 100644 index 69c5a8de8b05..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q +++ /dev/null @@ -1 +0,0 @@ -drop role PUBLIC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q deleted file mode 100644 index e19bf370fa07..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set user.name=user1; --- current user has been set (comment line before the set cmd is resulting in parse error!!) - -CREATE TABLE table_priv_rfail1(i int); - --- grant insert to user2 -GRANT INSERT ON table_priv_rfail1 TO USER user2; - -set user.name=user3; --- try dropping the privilege as user3 -REVOKE INSERT ON TABLE table_priv_rfail1 FROM USER user2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q deleted file mode 100644 index 4b0cf3286ae7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q +++ /dev/null @@ -1,18 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set user.name=user1; --- current user has been set (comment line before the set cmd is resulting in parse error!!) - -CREATE TABLE table_priv_rfai2(i int); - --- grant insert to user2 -GRANT INSERT ON table_priv_rfai2 TO USER user2; -GRANT SELECT ON table_priv_rfai2 TO USER user3 WITH GRANT OPTION; - -set user.name=user3; --- grant select as user3 to user 2 -GRANT SELECT ON table_priv_rfai2 TO USER user2; - --- try dropping the privilege as user3 -REVOKE INSERT ON TABLE table_priv_rfai2 FROM USER user2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q deleted file mode 100644 index a819d204f56b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q +++ /dev/null @@ -1,12 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; --- this is applicable to any security mode as check is in metastore -create role role1; -create role role2; -grant role role1 to role role2; - --- this will create a cycle -grant role role2 to role role1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q deleted file mode 100644 index 423f030630b6..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q +++ /dev/null @@ -1,24 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set user.name=hive_admin_user; -set role ADMIN; --- this is applicable to any security mode as check is in metastore - -create role role1; - -create role role2; -grant role role2 to role role1; - -create role role3; -grant role role3 to role role2; - -create role role4; -grant role role4 to role role3; - -create role role5; -grant role role5 to role role4; - --- this will create a cycle in middle of the hierarchy -grant role role2 to role role4; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q deleted file mode 100644 index c5c500a71251..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q +++ /dev/null @@ -1,22 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; - -set role ADMIN; - ----------------------------------------- --- role granting with admin option --- since user2 doesn't have admin option for role_noadmin, last grant should fail ----------------------------------------- - -create role role_noadmin; -create role src_role_wadmin; -grant src_role_wadmin to user user2 with admin option; -grant role_noadmin to user user2; -show role grant user user2; - - -set user.name=user2; -set role role_noadmin; -grant src_role_wadmin to user user3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q deleted file mode 100644 index d9f4c7cdb850..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q +++ /dev/null @@ -1,74 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -set user.name=hive_admin_user; -show current roles; -set role ADMIN; - ----------- --- create the following user, role mapping --- user1 -> role1 -> role2 -> role3 ----------- - -create role role1; -grant role1 to user user1; - -create role role2; -grant role2 to role role1; - -create role role3; -grant role3 to role role2; - - -create table t1(i int); -grant select on t1 to role role3; - -set user.name=user1; -show current roles; -select * from t1; - -set user.name=hive_admin_user; -show current roles; -grant select on t1 to role role2; - - -set user.name=user1; -show current roles; -select * from t1; - -set user.name=hive_admin_user; -set role ADMIN; -show current roles; -revoke select on table t1 from role role2; - - -create role role4; -grant role4 to user user1; -grant role3 to role role4;; - -set user.name=user1; -show current roles; -select * from t1; - -set user.name=hive_admin_user; -show current roles; -set role ADMIN; - --- Revoke role3 from hierarchy one at a time and check permissions --- after revoking from both, select should fail -revoke role3 from role role2; - -set user.name=user1; -show current roles; -select * from t1; - -set user.name=hive_admin_user; -show current roles; -set role ADMIN; -revoke role3 from role role4; - -set user.name=user1; -show current roles; -select * from t1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q deleted file mode 100644 index 39871793af39..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q +++ /dev/null @@ -1,9 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - --- check query without select privilege fails -create table t1(i int); - -set user.name=user1; -select * from t1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q deleted file mode 100644 index a4071cd0d4d8..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - --- check create view without select privileges -create table t1(i int); -create view v1 as select * from t1; -set user.name=user1; -select * from v1; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q deleted file mode 100644 index 9ba3a82a5608..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; - --- an error should be thrown if 'set role ' is done for role that does not exist - -set role nosuchroleexists; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q deleted file mode 100644 index 03f748fcc9b7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q +++ /dev/null @@ -1,16 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; - --- an error should be thrown if 'set role ' is done for role that does not exist - -create role rset_role_neg; -grant role rset_role_neg to user user2; - -set user.name=user2; -set role rset_role_neg; -set role public; -set role nosuchroleexists;; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q deleted file mode 100644 index d8190de950de..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if alter table fails as different user -create table t_show_parts(i int) partitioned by (j string); - -set user.name=user2; -show partitions t_show_parts; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q deleted file mode 100644 index 2afe87fc30c9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; --- This test will fail because hive_test_user is not in admin role -show principals role1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q deleted file mode 100644 index 69cea2f2673f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q +++ /dev/null @@ -1,2 +0,0 @@ --- This test will fail because the command is not currently supported in auth mode v1 -show principals role1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q deleted file mode 100644 index 0fc9fca940c3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; --- This test will fail because hive_test_user is not in admin role -show roles; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q deleted file mode 100644 index 285600b23a14..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q +++ /dev/null @@ -1,9 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - --- check add partition without insert privilege -create table t1(i int, j int); -set user.name=user1; -truncate table t1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q deleted file mode 100644 index d82ac710cc3b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_add_part; -dfs -touchz ${system:test.tmp.dir}/a_uri_add_part/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/a_uri_add_part/1.txt; - -create table tpart(i int, j int) partitioned by (k string); -alter table tpart add partition (k = 'abc') location '${system:test.tmp.dir}/a_uri_add_part/'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q deleted file mode 100644 index d38ba74d9006..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q +++ /dev/null @@ -1,16 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_alterpart_loc_perm; -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_alterpart_loc; -dfs -touchz ${system:test.tmp.dir}/az_uri_alterpart_loc/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/az_uri_alterpart_loc/1.txt; - -create table tpart(i int, j int) partitioned by (k string); -alter table tpart add partition (k = 'abc') location '${system:test.tmp.dir}/az_uri_alterpart_loc_perm/'; - -alter table tpart partition (k = 'abc') set location '${system:test.tmp.dir}/az_uri_alterpart_loc/'; - - --- Attempt to set partition to location without permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q deleted file mode 100644 index c446b8636fb3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_altertab_setloc; -dfs -touchz ${system:test.tmp.dir}/az_uri_altertab_setloc/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/az_uri_altertab_setloc/1.txt; - -create table t1(i int); - -alter table t1 set location '${system:test.tmp.dir}/az_uri_altertab_setloc/1.txt' - --- Attempt to set location of table to a location without permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q deleted file mode 100644 index c8e1fb43ee31..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_crtab1; -dfs -touchz ${system:test.tmp.dir}/a_uri_crtab1/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/a_uri_crtab1/1.txt; - -create table t1(i int) location '${system:test.tmp.dir}/a_uri_crtab_ext'; - --- Attempt to create table with dir that does not have write permission should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q deleted file mode 100644 index c8549b4563b2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_crtab_ext; -dfs -touchz ${system:test.tmp.dir}/a_uri_crtab_ext/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/a_uri_crtab_ext/1.txt; - -create external table t1(i int) location '${system:test.tmp.dir}/a_uri_crtab_ext'; - --- Attempt to create table with dir that does not have write permission should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q deleted file mode 100644 index edfdf5a8fc40..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q +++ /dev/null @@ -1,12 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_createdb; -dfs -touchz ${system:test.tmp.dir}/az_uri_createdb/1.txt; -dfs -chmod 300 ${system:test.tmp.dir}/az_uri_createdb/1.txt; - -create database az_test_db location '${system:test.tmp.dir}/az_uri_createdb/'; - --- Attempt to create db for dir without sufficient permissions should fail - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q deleted file mode 100644 index 81763916a0b8..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q +++ /dev/null @@ -1,22 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -set hive.test.mode=true; -set hive.test.mode.prefix=; -set hive.test.mode.nosamplelist=export_auth_uri; - - -create table export_auth_uri ( dep_id int comment "department id") - stored as textfile; - -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/export_auth_uri/temp; -dfs -rmr target/tmp/ql/test/data/exports/export_auth_uri; - - -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/export_auth_uri/; -dfs -chmod 555 target/tmp/ql/test/data/exports/export_auth_uri; - -export table export_auth_uri to 'ql/test/data/exports/export_auth_uri'; - --- Attempt to export to location without sufficient permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q deleted file mode 100644 index 4ea4dc0a4747..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q +++ /dev/null @@ -1,25 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -set hive.test.mode=true; -set hive.test.mode.prefix=; -set hive.test.mode.nosamplelist=import_auth_uri; - - -create table import_auth_uri ( dep_id int comment "department id") - stored as textfile; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/import_auth_uri/temp; -dfs -rmr target/tmp/ql/test/data/exports/import_auth_uri; -export table import_auth_uri to 'ql/test/data/exports/import_auth_uri'; -drop table import_auth_uri; - -dfs -touchz target/tmp/ql/test/data/exports/import_auth_uri/1.txt; -dfs -chmod 555 target/tmp/ql/test/data/exports/import_auth_uri/1.txt; - -create database importer; -use importer; - -import from 'ql/test/data/exports/import_auth_uri'; - --- Attempt to import from location without sufficient permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q deleted file mode 100644 index 1a8f9cb2ad19..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_index; -dfs -touchz ${system:test.tmp.dir}/az_uri_index/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/az_uri_index/1.txt; - - -create table t1(i int); -create index idt1 on table t1 (i) as 'COMPACT' WITH DEFERRED REBUILD LOCATION '${system:test.tmp.dir}/az_uri_index/'; - --- Attempt to use location for index that does not have permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q deleted file mode 100644 index 81b6e522c1ab..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_insert; -dfs -touchz ${system:test.tmp.dir}/az_uri_insert/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/az_uri_insert/1.txt; - -create table t1(i int, j int); - -insert overwrite directory '${system:test.tmp.dir}/az_uri_insert/' select * from t1; - --- Attempt to insert into uri without permissions should fail - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q deleted file mode 100644 index 0a2fd8919f45..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_insert_local; -dfs -touchz ${system:test.tmp.dir}/az_uri_insert_local/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/az_uri_insert_local/1.txt; - -create table t1(i int, j int); - -insert overwrite local directory '${system:test.tmp.dir}/az_uri_insert_local/' select * from t1; - --- Attempt to insert into uri without permissions should fail - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q deleted file mode 100644 index 6af41f0cdaa2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/authz_uri_load_data; -dfs -touchz ${system:test.tmp.dir}/authz_uri_load_data/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/authz_uri_load_data/1.txt; - -create table t1(i int); -load data inpath 'pfile:${system:test.tmp.dir}/authz_uri_load_data/' overwrite into table t1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q deleted file mode 100644 index d8beac370d4b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set hive.security.authorization.enabled=true; -set user.name=user33; -create database db23221; -use db23221; - -set user.name=user44; -create table twew221(a string); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q deleted file mode 100644 index bfd316523777..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q +++ /dev/null @@ -1 +0,0 @@ -grant role PUBLIC to user hive_test_user; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q deleted file mode 100644 index 2b29822371b1..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q +++ /dev/null @@ -1 +0,0 @@ -revoke role PUBLIC from user hive_test_user; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/autolocal1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/autolocal1.q deleted file mode 100644 index bd1c9d6e15a7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/autolocal1.q +++ /dev/null @@ -1,15 +0,0 @@ -set mapred.job.tracker=abracadabra; -set hive.exec.mode.local.auto.inputbytes.max=1; -set hive.exec.mode.local.auto=true; - --- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20) --- hadoop0.23 changes the behavior of JobClient initialization --- in hadoop0.20, JobClient initialization tries to get JobTracker's address --- this throws the expected IllegalArgumentException --- in hadoop0.23, JobClient initialization only initializes cluster --- and get user group information --- not attempts to get JobTracker's address --- no IllegalArgumentException thrown in JobClient Initialization --- an exception is thrown when JobClient submitJob - -SELECT key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_exec_hooks.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_exec_hooks.q deleted file mode 100644 index 709d8d9c8544..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_exec_hooks.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.exec.pre.hooks="org.this.is.a.bad.class"; - -EXPLAIN -SELECT x.* FROM SRC x LIMIT 20; - -SELECT x.* FROM SRC x LIMIT 20; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_indextype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_indextype.q deleted file mode 100644 index 8f5bf42664b9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_indextype.q +++ /dev/null @@ -1 +0,0 @@ -CREATE INDEX srcpart_index_proj ON TABLE srcpart(key) AS 'UNKNOWN' WITH DEFERRED REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_sample_clause.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_sample_clause.q deleted file mode 100644 index fd6769827b82..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_sample_clause.q +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TABLE dest1(key INT, value STRING, dt STRING, hr STRING) STORED AS TEXTFILE; - -INSERT OVERWRITE TABLE dest1 SELECT s.* -FROM srcpart TABLESAMPLE (BUCKET 1 OUT OF 2) s -WHERE s.ds='2008-04-08' and s.hr='11'; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q deleted file mode 100644 index 6bebb8942d61..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q +++ /dev/null @@ -1,42 +0,0 @@ -CREATE TABLE srcbucket_mapjoin_part (key int, value string) - partitioned by (ds string) CLUSTERED BY (key) INTO 3 BUCKETS - STORED AS TEXTFILE; -load data local inpath '../../data/files/srcbucket20.txt' - INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../../data/files/srcbucket21.txt' - INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../../data/files/srcbucket22.txt' - INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); - -CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) - partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS - STORED AS TEXTFILE; -load data local inpath '../../data/files/srcbucket22.txt' - INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); -load data local inpath '../../data/files/srcbucket23.txt' - INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); - --- The number of buckets in the 2 tables above (being joined later) dont match. --- Throw an error if the user requested a bucketed mapjoin to be enforced. --- In the default case (hive.enforce.bucketmapjoin=false), the query succeeds --- even though mapjoin is not being performed - -explain -select a.key, a.value, b.value -from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b -on a.key=b.key and a.ds="2008-04-08" and b.ds="2008-04-08"; - -set hive.optimize.bucketmapjoin = true; - -explain -select /*+mapjoin(b)*/ a.key, a.value, b.value -from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b -on a.key=b.key and a.ds="2008-04-08" and b.ds="2008-04-08"; - -set hive.enforce.bucketmapjoin=true; - -explain -select /*+mapjoin(b)*/ a.key, a.value, b.value -from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b -on a.key=b.key and a.ds="2008-04-08" and b.ds="2008-04-08"; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q deleted file mode 100644 index 802fcd903c0a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q +++ /dev/null @@ -1,20 +0,0 @@ --- Although the user has specified a bucketed map-join, the number of buckets in the table --- do not match the number of files -drop table table1; -drop table table2; - -create table table1(key string, value string) clustered by (key, value) -into 2 BUCKETS stored as textfile; -create table table2(key string, value string) clustered by (value, key) -into 2 BUCKETS stored as textfile; - -load data local inpath '../../data/files/T1.txt' overwrite into table table1; - -load data local inpath '../../data/files/T1.txt' overwrite into table table2; -load data local inpath '../../data/files/T2.txt' overwrite into table table2; - -set hive.optimize.bucketmapjoin = true; -set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; - -select /*+ mapjoin(b) */ count(*) from table1 a join table2 b on a.key=b.key and a.value=b.value; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q deleted file mode 100644 index ac5abebb0b4b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q +++ /dev/null @@ -1,24 +0,0 @@ --- Although the user has specified a bucketed map-join, the number of buckets in the table --- do not match the number of files -drop table table1; -drop table table2; - -create table table1(key string, value string) partitioned by (ds string) clustered by (key, value) -into 2 BUCKETS stored as textfile; -create table table2(key string, value string) clustered by (value, key) -into 2 BUCKETS stored as textfile; - -load data local inpath '../../data/files/T1.txt' overwrite into table table1 partition (ds='1'); -load data local inpath '../../data/files/T2.txt' overwrite into table table1 partition (ds='1'); - -load data local inpath '../../data/files/T1.txt' overwrite into table table1 partition (ds='2'); - -load data local inpath '../../data/files/T1.txt' overwrite into table table2; -load data local inpath '../../data/files/T2.txt' overwrite into table table2; - -set hive.optimize.bucketmapjoin = true; -set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; - -select /*+ mapjoin(b) */ count(*) from table1 a join table2 b -on a.key=b.key and a.value=b.value and a.ds is not null; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cachingprintstream.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cachingprintstream.q deleted file mode 100644 index d57a4517f00f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cachingprintstream.q +++ /dev/null @@ -1,8 +0,0 @@ -set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifyCachingPrintStreamHook; -set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyCachingPrintStreamHook; - -SELECT count(*) FROM src; -FROM src SELECT TRANSFORM (key, value) USING 'FAKE_SCRIPT_SHOULD_NOT_EXIST' AS key, value; - -set hive.exec.failure.hooks=; -set hive.exec.post.hooks=; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q deleted file mode 100644 index bc980448a9e2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q +++ /dev/null @@ -1,6 +0,0 @@ --- TaskLog retrieval upon Null Pointer Exception in Cluster - -CREATE TEMPORARY FUNCTION evaluate_npe AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFEvaluateNPE'; - -FROM src -SELECT evaluate_npe(src.key) LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbydistributeby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbydistributeby.q deleted file mode 100644 index 4c6a9b38d785..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbydistributeby.q +++ /dev/null @@ -1,8 +0,0 @@ -CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; - -FROM src -INSERT OVERWRITE TABLE dest1 -MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value -USING 'cat' AS (tkey, ten, one, tvalue) -CLUSTER BY tvalue, tkey -DISTRIBUTE BY tvalue, tkey; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbyorderby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbyorderby.q deleted file mode 100644 index d9ee9b9d262d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbyorderby.q +++ /dev/null @@ -1,5 +0,0 @@ -FROM src -MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value -USING 'cat' AS (tkey, ten, one, tvalue) -CLUSTER BY tvalue, tkey -ORDER BY ten, one; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbysortby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbysortby.q deleted file mode 100644 index 7b4e744ba66d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbysortby.q +++ /dev/null @@ -1,8 +0,0 @@ -CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; - -FROM src -INSERT OVERWRITE TABLE dest1 -MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value -USING 'cat' AS (tkey, ten, one, tvalue) -CLUSTER BY tvalue, tkey -SORT BY ten, one; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern2.q deleted file mode 100644 index 9ed8944d2bb6..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern2.q +++ /dev/null @@ -1,3 +0,0 @@ -EXPLAIN -SELECT x.key, x.value as v1, y.* FROM SRC x JOIN SRC y ON (x.key = y.key) CLUSTER BY key; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern3.q deleted file mode 100644 index 23f73667edf5..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern3.q +++ /dev/null @@ -1,2 +0,0 @@ -EXPLAIN -SELECT x.key as k1, x.value FROM SRC x CLUSTER BY x.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern4.q deleted file mode 100644 index 3a9b45ca6057..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern4.q +++ /dev/null @@ -1,2 +0,0 @@ -EXPLAIN -SELECT x.key as k1, x.value FROM SRC x CLUSTER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q deleted file mode 100644 index 9a3e0b2efe69..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.mapred.supports.subdirectories=true; - -CREATE TABLE skewedtable (key STRING, value STRING) SKEWED BY (key) ON (1,5,6); - -ALTER TABLE skewedtable CHANGE key key INT; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename1.q deleted file mode 100644 index d99b821802df..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename1.q +++ /dev/null @@ -1,6 +0,0 @@ -drop table tstsrc; -create table tstsrc like src; -insert overwrite table tstsrc -select key, value from src; - -alter table tstsrc change src_not_exist key_value string; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename2.q deleted file mode 100644 index cccc8ad54e30..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename2.q +++ /dev/null @@ -1,6 +0,0 @@ -drop table tstsrc; -create table tstsrc like src; -insert overwrite table tstsrc -select key, value from src; - -alter table tstsrc change key value string; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename3.q deleted file mode 100644 index 91c9537a99ad..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename3.q +++ /dev/null @@ -1 +0,0 @@ -alter table src change key key; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename4.q deleted file mode 100644 index dd89a5a10b22..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename4.q +++ /dev/null @@ -1,6 +0,0 @@ -drop table tstsrc; -create table tstsrc like src; -insert overwrite table tstsrc -select key, value from src; - -alter table tstsrc change key key2 string after key_value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename5.q deleted file mode 100644 index 3827b83361fb..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename5.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.mapred.supports.subdirectories=true; - -CREATE TABLE skewedtable (key STRING, value STRING) SKEWED BY (key) ON (1,5,6); - -ALTER TABLE skewedtable CHANGE key key_new STRING; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q deleted file mode 100644 index b4887c411585..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q +++ /dev/null @@ -1,16 +0,0 @@ -DROP TABLE Employee_Part; - -CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string) -row format delimited fields terminated by '|' stored as textfile; - -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); - --- dynamic partitioning syntax -explain -analyze table Employee_Part partition (employeeSalary='4000.0', country) compute statistics for columns employeeName, employeeID; -analyze table Employee_Part partition (employeeSalary='4000.0', country) compute statistics for columns employeeName, employeeID; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q deleted file mode 100644 index 2f8e9271ddd3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q +++ /dev/null @@ -1,16 +0,0 @@ -DROP TABLE Employee_Part; - -CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string) -row format delimited fields terminated by '|' stored as textfile; - -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); - --- don't specify all partitioning keys -explain -analyze table Employee_Part partition (employeeSalary='2000.0') compute statistics for columns employeeID; -analyze table Employee_Part partition (employeeSalary='2000.0') compute statistics for columns employeeID; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q deleted file mode 100644 index 34f91fc8d1de..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q +++ /dev/null @@ -1,16 +0,0 @@ -DROP TABLE Employee_Part; - -CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string) -row format delimited fields terminated by '|' stored as textfile; - -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); - --- specify invalid values for the partitioning keys -explain -analyze table Employee_Part partition (employeeSalary='4000.0', country='Canada') compute statistics for columns employeeName, employeeID; -analyze table Employee_Part partition (employeeSalary='4000.0', country='Canada') compute statistics for columns employeeName, employeeID; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q deleted file mode 100644 index 49d89dd12132..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q +++ /dev/null @@ -1,16 +0,0 @@ -DROP TABLE Employee_Part; - -CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string) -row format delimited fields terminated by '|' stored as textfile; - -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); - --- specify partitioning clause multiple times -explain -analyze table Employee_Part partition (employeeSalary='4000.0', country='USA') partition(employeeSalary='2000.0', country='USA') compute statistics for columns employeeName, employeeID; -analyze table Employee_Part partition (employeeSalary='4000.0', country='USA') partition(employeeSalary='2000.0', country='USA') compute statistics for columns employeeName, employeeID; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q deleted file mode 100644 index a4e0056bff37..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q +++ /dev/null @@ -1,22 +0,0 @@ - -DROP TABLE IF EXISTS UserVisits_web_text_none; - -CREATE TABLE UserVisits_web_text_none ( - sourceIP string, - destURL string, - visitDate string, - adRevenue float, - userAgent string, - cCode string, - lCode string, - sKeyword string, - avgTimeOnSite int) -row format delimited fields terminated by '|' stored as textfile; - -LOAD DATA LOCAL INPATH "../../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none; - -explain -analyze table UserVisits_web_text_none compute statistics for columns destIP; - -analyze table UserVisits_web_text_none compute statistics for columns destIP; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q deleted file mode 100644 index 85a5f0a02194..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q +++ /dev/null @@ -1,17 +0,0 @@ - -DROP TABLE IF EXISTS table_complex_type; - -CREATE TABLE table_complex_type ( - a STRING, - b ARRAY, - c ARRAY>, - d MAP> - ) STORED AS TEXTFILE; - -LOAD DATA LOCAL INPATH '../../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table_complex_type; - - -explain -analyze table table_complex_type compute statistics for columns d; - -analyze table table_complex_type compute statistics for columns d; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q deleted file mode 100644 index a4e0056bff37..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q +++ /dev/null @@ -1,22 +0,0 @@ - -DROP TABLE IF EXISTS UserVisits_web_text_none; - -CREATE TABLE UserVisits_web_text_none ( - sourceIP string, - destURL string, - visitDate string, - adRevenue float, - userAgent string, - cCode string, - lCode string, - sKeyword string, - avgTimeOnSite int) -row format delimited fields terminated by '|' stored as textfile; - -LOAD DATA LOCAL INPATH "../../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none; - -explain -analyze table UserVisits_web_text_none compute statistics for columns destIP; - -analyze table UserVisits_web_text_none compute statistics for columns destIP; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_double_bigint.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_double_bigint.q deleted file mode 100644 index 8ee4b277cbf7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_double_bigint.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.mapred.mode=strict; - --- This should fail until we fix the issue with precision when casting a bigint to a double - -select * from src where cast(1 as bigint) = 1.0 limit 10; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_string_bigint.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_string_bigint.q deleted file mode 100644 index 810f65d4d2b4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_string_bigint.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.mapred.mode=strict; - ---This should fail until we fix the issue with precision when casting a bigint to a double - -select * from src where cast(1 as bigint) = '1' limit 10; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q deleted file mode 100644 index c314a940f95c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q +++ /dev/null @@ -1,8 +0,0 @@ - -compile `import org.apache.hadoop.hive.ql.exec.UDF \; -public class Pyth extsfgsfgfsends UDF { - public double evaluate(double a, double b){ - return Math.sqrt((a*a) + (b*b)) \; - } -} ` AS GROOVY NAMED Pyth.groovy; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q deleted file mode 100644 index 597481128035..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q +++ /dev/null @@ -1,7 +0,0 @@ -create table tab_int(a int); - --- insert some data -LOAD DATA LOCAL INPATH "../../data/files/int.txt" INTO TABLE tab_int; - --- compute stats should raise an error since the number of bit vectors > 1024 -select compute_stats(a, 10000) from tab_int; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q deleted file mode 100644 index 3b71e00b2eaa..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q +++ /dev/null @@ -1 +0,0 @@ -create function default.badfunc as 'my.nonexistent.class'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q deleted file mode 100644 index ae95391edd3e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q +++ /dev/null @@ -1 +0,0 @@ -create function nonexistentdb.badfunc as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q deleted file mode 100644 index 208306459329..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q +++ /dev/null @@ -1 +0,0 @@ -create function default.badfunc as 'java.lang.String'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_insert_outputformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_insert_outputformat.q deleted file mode 100644 index a052663055ef..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_insert_outputformat.q +++ /dev/null @@ -1,11 +0,0 @@ - - -CREATE TABLE table_test_output_format(key INT, value STRING) STORED AS - INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' - OUTPUTFORMAT 'org.apache.hadoop.mapred.MapFileOutputFormat'; - -FROM src -INSERT OVERWRITE TABLE table_test_output_format SELECT src.key, src.value LIMIT 10; - -describe table_test_output_format; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view1.q deleted file mode 100644 index c332278b84f6..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view1.q +++ /dev/null @@ -1,6 +0,0 @@ --- Cannot add or drop partition columns with CREATE OR REPLACE VIEW if partitions currently exist (must specify partition columns) - -drop view v; -create view v partitioned on (ds, hr) as select * from srcpart; -alter view v add partition (ds='1',hr='2'); -create or replace view v as select * from srcpart; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view2.q deleted file mode 100644 index b53dd07ce8ae..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view2.q +++ /dev/null @@ -1,6 +0,0 @@ --- Cannot add or drop partition columns with CREATE OR REPLACE VIEW if partitions currently exist - -drop view v; -create view v partitioned on (ds, hr) as select * from srcpart; -alter view v add partition (ds='1',hr='2'); -create or replace view v partitioned on (hr) as select * from srcpart; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view3.q deleted file mode 100644 index d6fa7785dfa9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view3.q +++ /dev/null @@ -1,3 +0,0 @@ --- Existing table is not a view - -create or replace view src as select ds, hr from srcpart; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view4.q deleted file mode 100644 index 12b6059b9e3e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view4.q +++ /dev/null @@ -1,5 +0,0 @@ --- View must have at least one non-partition column. - -drop view v; -create view v partitioned on (ds, hr) as select * from srcpart; -create or replace view v partitioned on (ds, hr) as select ds, hr from srcpart; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view5.q deleted file mode 100644 index 4eb9c94896d8..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view5.q +++ /dev/null @@ -1,5 +0,0 @@ --- Can't combine IF NOT EXISTS and OR REPLACE. - -drop view v; -create view v partitioned on (ds, hr) as select * from srcpart; -create or replace view if not exists v as select * from srcpart; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view6.q deleted file mode 100644 index a2f916fb2652..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view6.q +++ /dev/null @@ -1,5 +0,0 @@ --- Can't update view to have an invalid definition - -drop view v; -create view v partitioned on (ds, hr) as select * from srcpart; -create or replace view v partitioned on (ds, hr) as blah; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view7.q deleted file mode 100644 index 765a96572a04..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view7.q +++ /dev/null @@ -1,7 +0,0 @@ --- Can't update view to have a view cycle (1) - -drop view v; -create view v1 partitioned on (ds, hr) as select * from srcpart; -create view v2 partitioned on (ds, hr) as select * from v1; -create view v3 partitioned on (ds, hr) as select * from v2; -create or replace view v1 partitioned on (ds, hr) as select * from v3; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view8.q deleted file mode 100644 index f3a59b1d07be..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view8.q +++ /dev/null @@ -1,5 +0,0 @@ --- Can't update view to have a view cycle (2) - -drop view v; -create view v1 partitioned on (ds, hr) as select * from srcpart; -create or replace view v1 partitioned on (ds, hr) as select * from v1; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q deleted file mode 100644 index 1d6574e73960..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.mapred.supports.subdirectories=true; - -CREATE TABLE skewed_table (key STRING, value STRING) SKEWED BY (key) ON ((1),(5,8),(6)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q deleted file mode 100644 index 726f6dd1dfcf..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.mapred.supports.subdirectories=true; - -CREATE TABLE skewed_table (key STRING, value STRING) SKEWED BY (key,key) ON ((1),(5),(6)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q deleted file mode 100644 index 30dd4181653d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.mapred.supports.subdirectories=true; - -CREATE TABLE skewed_table (key STRING, value STRING) SKEWED BY (key_non) ON ((1),(5),(6)); - \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure1.q deleted file mode 100644 index e87c12b8a1fe..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure1.q +++ /dev/null @@ -1 +0,0 @@ -create table table_in_database_creation_not_exist.test as select * from src limit 1; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure2.q deleted file mode 100644 index 0bddae066450..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure2.q +++ /dev/null @@ -1 +0,0 @@ -create table `table_in_database_creation_not_exist.test` as select * from src limit 1; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure3.q deleted file mode 100644 index 9f9f5f64dfd9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure3.q +++ /dev/null @@ -1 +0,0 @@ -create table table_in_database_creation_not_exist.test (a string); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure4.q deleted file mode 100644 index 67745e011141..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure4.q +++ /dev/null @@ -1 +0,0 @@ -create table `table_in_database_creation_not_exist.test` (a string); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_wrong_regex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_wrong_regex.q deleted file mode 100644 index dc91c9c9ef05..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_wrong_regex.q +++ /dev/null @@ -1,4 +0,0 @@ -drop table aa; -create table aa ( test STRING ) - ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.RegexSerDe' - WITH SERDEPROPERTIES ("input.regex" = "[^\\](.*)", "output.format.string" = "$1s"); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_udaf_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_udaf_failure.q deleted file mode 100644 index e0bb408a64f2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_udaf_failure.q +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TEMPORARY FUNCTION test_udaf AS 'org.apache.hadoop.hive.ql.udf.UDAFWrongArgLengthForTestCase'; - -EXPLAIN -SELECT test_udaf(length(src.value)) FROM src; - -SELECT test_udaf(length(src.value)) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_genericudf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_genericudf.q deleted file mode 100644 index 07010c11c7d4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_genericudf.q +++ /dev/null @@ -1 +0,0 @@ -CREATE TEMPORARY FUNCTION dummy_genericudf AS 'org.apache.hadoop.hive.ql.udf.generic.DummyGenericUDF'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q deleted file mode 100644 index a243fff033c4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q +++ /dev/null @@ -1 +0,0 @@ -CREATE TEMPORARY FUNCTION dummy_function AS 'org.apache.hadoop.hive.ql.udf.DummyFunction'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure1.q deleted file mode 100644 index c9060c676649..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure1.q +++ /dev/null @@ -1,6 +0,0 @@ - -DROP VIEW xxx12; - --- views and tables share the same namespace -CREATE TABLE xxx12(key int); -CREATE VIEW xxx12 AS SELECT key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure2.q deleted file mode 100644 index 6fdcd4a9d377..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure2.q +++ /dev/null @@ -1,6 +0,0 @@ - -DROP VIEW xxx4; - --- views and tables share the same namespace -CREATE VIEW xxx4 AS SELECT key FROM src; -CREATE TABLE xxx4(key int); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure3.q deleted file mode 100644 index ad5fc499edf9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure3.q +++ /dev/null @@ -1,5 +0,0 @@ -DROP VIEW xxx13; - --- number of explicit view column defs must match underlying SELECT -CREATE VIEW xxx13(x,y,z) AS -SELECT key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure4.q deleted file mode 100644 index eecde65e1137..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure4.q +++ /dev/null @@ -1,5 +0,0 @@ -DROP VIEW xxx5; - --- duplicate column names are illegal -CREATE VIEW xxx5(x,x) AS -SELECT key,value FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure5.q deleted file mode 100644 index f72089916873..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure5.q +++ /dev/null @@ -1,9 +0,0 @@ -DROP VIEW xxx14; - --- Ideally (and according to SQL:200n), this should actually be legal, --- but since internally we impose the new column descriptors by --- reference to underlying name rather than position, we have to make --- it illegal. There's an easy workaround (provide the unique names --- via direct column aliases, e.g. SELECT key AS x, key AS y) -CREATE VIEW xxx14(x,y) AS -SELECT key,key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure6.q deleted file mode 100644 index 57f52a8af149..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure6.q +++ /dev/null @@ -1,6 +0,0 @@ -DROP VIEW xxx15; - --- should fail: baz is not a column -CREATE VIEW xxx15 -PARTITIONED ON (baz) -AS SELECT key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure7.q deleted file mode 100644 index 00d7f9fbf4ed..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure7.q +++ /dev/null @@ -1,6 +0,0 @@ -DROP VIEW xxx16; - --- should fail: must have at least one non-partitioning column -CREATE VIEW xxx16 -PARTITIONED ON (key) -AS SELECT key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure8.q deleted file mode 100644 index 08291826d978..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure8.q +++ /dev/null @@ -1,6 +0,0 @@ -DROP VIEW xxx17; - --- should fail: partitioning key must be at end -CREATE VIEW xxx17 -PARTITIONED ON (key) -AS SELECT key,value FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure9.q deleted file mode 100644 index d7d44a49c393..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure9.q +++ /dev/null @@ -1,6 +0,0 @@ -DROP VIEW xxx18; - --- should fail: partitioning columns out of order -CREATE VIEW xxx18 -PARTITIONED ON (value,key) -AS SELECT key+1 as k2,key,value FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ctas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ctas.q deleted file mode 100644 index 507a7a76b1ee..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ctas.q +++ /dev/null @@ -1,5 +0,0 @@ - - -create external table nzhang_ctas4 as select key, value from src; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q deleted file mode 100644 index 2160b4719662..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q +++ /dev/null @@ -1,4 +0,0 @@ -explain -with q1 as ( select key from q2 where key = '5'), -q2 as ( select key from q1 where key = '5') -select * from (select key from q1) a; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q deleted file mode 100644 index e52a1d97db80..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q +++ /dev/null @@ -1 +0,0 @@ -select * from (with q1 as ( select key from q2 where key = '5') select * from q1) a; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_already_exists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_already_exists.q deleted file mode 100644 index 3af7607739a5..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_already_exists.q +++ /dev/null @@ -1,5 +0,0 @@ -SHOW DATABASES; - --- Try to create a database that already exists -CREATE DATABASE test_db; -CREATE DATABASE test_db; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_invalid_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_invalid_name.q deleted file mode 100644 index 5d6749542b47..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_invalid_name.q +++ /dev/null @@ -1,4 +0,0 @@ -SHOW DATABASES; - --- Try to create a database with an invalid name -CREATE DATABASE `test.db`; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q deleted file mode 100644 index 66a940e63dea..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q +++ /dev/null @@ -1,4 +0,0 @@ -SHOW DATABASES; - --- Try to drop a database that does not exist -DROP DATABASE does_not_exist; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty.q deleted file mode 100644 index ae5a443f1062..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty.q +++ /dev/null @@ -1,8 +0,0 @@ -SHOW DATABASES; - --- Try to drop a non-empty database -CREATE DATABASE test_db; -USE test_db; -CREATE TABLE t(a INT); -USE default; -DROP DATABASE test_db; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q deleted file mode 100644 index e1cb81c93f27..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q +++ /dev/null @@ -1,8 +0,0 @@ -SHOW DATABASES; - --- Try to drop a non-empty database in restrict mode -CREATE DATABASE db_drop_non_empty_restrict; -USE db_drop_non_empty_restrict; -CREATE TABLE t(a INT); -USE default; -DROP DATABASE db_drop_non_empty_restrict; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q deleted file mode 100644 index 5cd469769e0a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q +++ /dev/null @@ -1,4 +0,0 @@ -SHOW DATABASES; - --- Try to switch to a database that does not exist -USE does_not_exist; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal2.q deleted file mode 100644 index 711dc9e0fd35..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal2.q +++ /dev/null @@ -1,2 +0,0 @@ --- Not in YYYY-MM-DD format -SELECT DATE '2001/01/01' FROM src LIMIT 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal3.q deleted file mode 100644 index 9483509b6bb7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal3.q +++ /dev/null @@ -1,2 +0,0 @@ --- Invalid date value -SELECT DATE '2001-01-32' FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q deleted file mode 100644 index 1c658c79b99e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.support.concurrency=true; -set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; - -drop database if exists drop_nodblock; -create database drop_nodblock; -lock database drop_nodblock shared; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q deleted file mode 100644 index ef4b323f063b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.support.concurrency=true; -set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; - -drop database if exists drop_nodbunlock; -create database drop_nodbunlock; -unlock database drop_nodbunlock; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q deleted file mode 100644 index 4a0c6c25c67c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.support.concurrency=true; -set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; - -drop table if exists drop_notablelock; -create table drop_notablelock (c int); -lock table drop_notablelock shared; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q deleted file mode 100644 index 0b00046579f4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.support.concurrency=true; -set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; - -drop table if exists drop_notableunlock; -create table drop_notableunlock (c int); -unlock table drop_notableunlock; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ddltime.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ddltime.q deleted file mode 100644 index 3517a6046de1..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ddltime.q +++ /dev/null @@ -1,6 +0,0 @@ - -create table T2 like srcpart; - -insert overwrite table T2 partition (ds = '2010-06-21', hr='1') select /*+ HOLD_DDLTIME */ key, value from src where key > 10; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision.q deleted file mode 100644 index f49649837e21..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision.q +++ /dev/null @@ -1,10 +0,0 @@ -DROP TABLE IF EXISTS DECIMAL_PRECISION; - -CREATE TABLE DECIMAL_PRECISION(dec decimal) -ROW FORMAT DELIMITED - FIELDS TERMINATED BY ' ' -STORED AS TEXTFILE; - -SELECT dec * 123456789012345678901234567890.123456789bd FROM DECIMAL_PRECISION; - -DROP TABLE DECIMAL_PRECISION; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision_1.q deleted file mode 100644 index 036ff1facc0a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision_1.q +++ /dev/null @@ -1,10 +0,0 @@ -DROP TABLE IF EXISTS DECIMAL_PRECISION; - -CREATE TABLE DECIMAL_PRECISION(dec decimal) -ROW FORMAT DELIMITED - FIELDS TERMINATED BY ' ' -STORED AS TEXTFILE; - -SELECT * from DECIMAL_PRECISION WHERE dec > 1234567890123456789.0123456789bd; - -DROP TABLE DECIMAL_PRECISION; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/default_partition_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/default_partition_name.q deleted file mode 100644 index 816b6cb80a96..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/default_partition_name.q +++ /dev/null @@ -1,3 +0,0 @@ -create table default_partition_name (key int, value string) partitioned by (ds string); - -alter table default_partition_name add partition(ds='__HIVE_DEFAULT_PARTITION__'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q deleted file mode 100644 index 0bd6985e031b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q +++ /dev/null @@ -1,4 +0,0 @@ - -ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar; -DELETE JAR ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar; -CREATE TABLE DELETEJAR(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure1.q deleted file mode 100644 index f7304b12e65f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure1.q +++ /dev/null @@ -1 +0,0 @@ -DESC NonExistentTable; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure2.q deleted file mode 100644 index f28b61046649..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure2.q +++ /dev/null @@ -1,2 +0,0 @@ -DESC srcpart; -DESC srcpart PARTITION(ds='2012-04-08', hr='15'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure3.q deleted file mode 100644 index bee0ea5788b4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure3.q +++ /dev/null @@ -1,5 +0,0 @@ -CREATE DATABASE db1; -CREATE TABLE db1.t1(key1 INT, value1 STRING) PARTITIONED BY (ds STRING, part STRING); - --- describe database.table.column -DESCRIBE db1.t1.key1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath1.q deleted file mode 100644 index ea72f83e1d58..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath1.q +++ /dev/null @@ -1 +0,0 @@ -describe src_thrift.$elem$; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath2.q deleted file mode 100644 index f1fee1ac444d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath2.q +++ /dev/null @@ -1 +0,0 @@ -describe src_thrift.$key$; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath3.q deleted file mode 100644 index 4a11f6845f39..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath3.q +++ /dev/null @@ -1 +0,0 @@ -describe src_thrift.lint.abc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath4.q deleted file mode 100644 index 0912bf1cd9dd..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath4.q +++ /dev/null @@ -1 +0,0 @@ -describe src_thrift.mStringString.abc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q deleted file mode 100644 index d0d748cf4ffd..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q +++ /dev/null @@ -1,17 +0,0 @@ -SET hive.metastore.disallow.incompatible.col.type.changes=true; -SELECT * FROM src LIMIT 1; -CREATE TABLE test_table123 (a INT, b MAP) PARTITIONED BY (ds STRING) STORED AS SEQUENCEFILE; -INSERT OVERWRITE TABLE test_table123 PARTITION(ds="foo1") SELECT 1, MAP("a1", "b1") FROM src LIMIT 1; -SELECT * from test_table123 WHERE ds="foo1"; -ALTER TABLE test_table123 REPLACE COLUMNS (a INT, b MAP); -ALTER TABLE test_table123 REPLACE COLUMNS (a BIGINT, b MAP); -ALTER TABLE test_table123 REPLACE COLUMNS (a INT, b MAP); -ALTER TABLE test_table123 REPLACE COLUMNS (a DOUBLE, b MAP); -ALTER TABLE test_table123 REPLACE COLUMNS (a TINYINT, b MAP); -ALTER TABLE test_table123 REPLACE COLUMNS (a BOOLEAN, b MAP); -ALTER TABLE test_table123 REPLACE COLUMNS (a TINYINT, b MAP); -ALTER TABLE test_table123 CHANGE COLUMN a a_new BOOLEAN; --- All the above ALTERs will succeed since they are between compatible types. --- The following ALTER will fail as MAP and STRING are not --- compatible. -ALTER TABLE test_table123 REPLACE COLUMNS (a INT, b STRING); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q deleted file mode 100644 index 4460c3edd7e4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q +++ /dev/null @@ -1,6 +0,0 @@ -SET hive.metastore.disallow.incompatible.col.type.changes=true; -SELECT * FROM src LIMIT 1; -CREATE TABLE test_table123 (a INT, b STRING) PARTITIONED BY (ds STRING) STORED AS SEQUENCEFILE; -INSERT OVERWRITE TABLE test_table123 PARTITION(ds="foo1") SELECT 1, "one" FROM src LIMIT 1; -SELECT * from test_table123 WHERE ds="foo1"; -ALTER TABLE test_table123 CHANGE COLUMN b b MAP; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q deleted file mode 100644 index 892ef00e3f86..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.exec.drop.ignorenonexistent=false; --- Can't use DROP FUNCTION if the function doesn't exist and IF EXISTS isn't specified -drop function nonexistent_function; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_function_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_function_failure.q deleted file mode 100644 index 51dc5e9d8e32..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_function_failure.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.exec.drop.ignorenonexistent=false; --- Can't use DROP TEMPORARY FUNCTION if the function doesn't exist and IF EXISTS isn't specified -DROP TEMPORARY FUNCTION UnknownFunction; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_index_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_index_failure.q deleted file mode 100644 index 6e907dfa99b2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_index_failure.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.exec.drop.ignorenonexistent=false; --- Can't use DROP INDEX if the index doesn't exist and IF EXISTS isn't specified -DROP INDEX UnknownIndex ON src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_native_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_native_udf.q deleted file mode 100644 index ae047bbc1780..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_native_udf.q +++ /dev/null @@ -1 +0,0 @@ -DROP TEMPORARY FUNCTION max; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_failure.q deleted file mode 100644 index c2074f69cbf3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_failure.q +++ /dev/null @@ -1,11 +0,0 @@ -create table mp (a string) partitioned by (b string, c string); - -alter table mp add partition (b='1', c='1'); -alter table mp add partition (b='1', c='2'); -alter table mp add partition (b='2', c='2'); - -show partitions mp; - -set hive.exec.drop.ignorenonexistent=false; --- Can't use DROP PARTITION if the partition doesn't exist and IF EXISTS isn't specified -alter table mp drop partition (b='3'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q deleted file mode 100644 index df476ed7c463..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q +++ /dev/null @@ -1,8 +0,0 @@ -create table ptestfilter1 (a string, b int) partitioned by (c string, d string); - -alter table ptestfilter1 add partition (c='US', d=1); -show partitions ptestfilter1; - -set hive.exec.drop.ignorenonexistent=false; -alter table ptestfilter1 drop partition (c='US', d<1); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure1.q deleted file mode 100644 index d47c08b876fc..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure1.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.exec.drop.ignorenonexistent=false; --- Can't use DROP TABLE if the table doesn't exist and IF EXISTS isn't specified -DROP TABLE UnknownTable; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure2.q deleted file mode 100644 index 631e4ffba7a4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure2.q +++ /dev/null @@ -1,3 +0,0 @@ -CREATE VIEW xxx6 AS SELECT key FROM src; --- Can't use DROP TABLE on a view -DROP TABLE xxx6; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure3.q deleted file mode 100644 index 534ce0b0324a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure3.q +++ /dev/null @@ -1,12 +0,0 @@ -create database dtf3; -use dtf3; - -create table drop_table_failure_temp(col STRING) partitioned by (p STRING); - -alter table drop_table_failure_temp add partition (p ='p1'); -alter table drop_table_failure_temp add partition (p ='p2'); -alter table drop_table_failure_temp add partition (p ='p3'); - -alter table drop_table_failure_temp partition (p ='p3') ENABLE NO_DROP; - -drop table drop_table_failure_temp; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure1.q deleted file mode 100644 index 79cb4e445b05..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure1.q +++ /dev/null @@ -1,6 +0,0 @@ - - -CREATE TABLE xxx1(key int); - --- Can't use DROP VIEW on a base table -DROP VIEW xxx1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure2.q deleted file mode 100644 index 93bb16232d57..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure2.q +++ /dev/null @@ -1,3 +0,0 @@ -SET hive.exec.drop.ignorenonexistent=false; --- Can't use DROP VIEW if the view doesn't exist and IF EXISTS isn't specified -DROP VIEW UnknownView; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q deleted file mode 100644 index b2e8567f09e1..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q +++ /dev/null @@ -1 +0,0 @@ -FROM src SELECT TRANSFORM (key, value) USING "awk -F'\001' '{print $0}'" AS (foo, foo); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q deleted file mode 100644 index dabbc351bc38..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q +++ /dev/null @@ -1 +0,0 @@ -FROM src SELECT TRANSFORM (key, value) USING "awk -F'\001' '{print $0}'" AS (foo STRING, foo STRING); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert1.q deleted file mode 100644 index fcbc7d5444a4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert1.q +++ /dev/null @@ -1,7 +0,0 @@ - -create table dest1_din1(key int, value string); - -from src -insert overwrite table dest1_din1 select key, value -insert overwrite table dest1_din1 select key, value; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert2.q deleted file mode 100644 index 4f79a0352f21..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert2.q +++ /dev/null @@ -1,6 +0,0 @@ - -create table dest1_din2(key int, value string) partitioned by (ds string); - -from src -insert overwrite table dest1_din2 partition (ds='1') select key, value -insert overwrite table dest1_din2 partition (ds='1') select key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert3.q deleted file mode 100644 index 7b271a56d184..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert3.q +++ /dev/null @@ -1,4 +0,0 @@ - -from src -insert overwrite directory '${system:test.tmp.dir}/dest1' select key, value -insert overwrite directory '${system:test.tmp.dir}/dest1' select key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part1.q deleted file mode 100644 index 9f0b6c7a0cc8..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part1.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nostrict; -set hive.exec.max.dynamic.partitions=2; - - -create table dynamic_partition (key string) partitioned by (value string); - -insert overwrite table dynamic_partition partition(hr) select key, value from src; - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part2.q deleted file mode 100644 index 00a92783c054..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part2.q +++ /dev/null @@ -1,11 +0,0 @@ - -create table nzhang_part1 (key string, value string) partitioned by (ds string, hr string); - -set hive.exec.dynamic.partition=true; - -insert overwrite table nzhang_part1 partition(ds='11', hr) select key, value from srcpart where ds is not null; - -show partitions nzhang_part1; - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part3.q deleted file mode 100644 index 7a8c58a6b255..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part3.q +++ /dev/null @@ -1,9 +0,0 @@ -set hive.exec.max.dynamic.partitions=600; -set hive.exec.max.dynamic.partitions.pernode=600; -set hive.exec.dynamic.partition.mode=nonstrict; -set hive.exec.dynamic.partition=true; -set hive.exec.max.created.files=100; - -create table nzhang_part( key string) partitioned by (value string); - -insert overwrite table nzhang_part partition(value) select key, value from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part4.q deleted file mode 100644 index 9aff7aa6310d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part4.q +++ /dev/null @@ -1,7 +0,0 @@ -create table nzhang_part4 (key string) partitioned by (ds string, hr string, value string); - -set hive.exec.dynamic.partition=true; - -insert overwrite table nzhang_part4 partition(value = 'aaa', ds='11', hr) select key, hr from srcpart where ds is not null; - -drop table nzhang_part4; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled deleted file mode 100644 index a8fce595005d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled +++ /dev/null @@ -1,24 +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. - -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nonstrict; -set hive.stats.autogether=false; -set hive.error.on.empty.partition=true; - -create table dyn_err(key string, value string) partitioned by (ds string); - -insert overwrite table dyn_err partition(ds) select key, value, ds from srcpart where ds is not null and key = 'no exists'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max.q deleted file mode 100644 index 6a7a6255b959..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max.q +++ /dev/null @@ -1,16 +0,0 @@ -USE default; - --- Test of hive.exec.max.dynamic.partitions --- Set hive.exec.max.dynamic.partitions.pernode to a large value so it will be ignored - -CREATE TABLE max_parts(key STRING) PARTITIONED BY (value STRING); - -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nonstrict; -set hive.exec.max.dynamic.partitions=10; -set hive.exec.max.dynamic.partitions.pernode=1000; - -INSERT OVERWRITE TABLE max_parts PARTITION(value) -SELECT key, value -FROM src -LIMIT 50; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q deleted file mode 100644 index a411ec520b6d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q +++ /dev/null @@ -1,15 +0,0 @@ -USE default; - --- Test of hive.exec.max.dynamic.partitions.pernode - -CREATE TABLE max_parts(key STRING) PARTITIONED BY (value STRING); - -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nonstrict; -set hive.exec.max.dynamic.partitions=1000; -set hive.exec.max.dynamic.partitions.pernode=10; - -INSERT OVERWRITE TABLE max_parts PARTITION(value) -SELECT key, value -FROM src -LIMIT 50; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q deleted file mode 100644 index 0ad99d100dc0..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q +++ /dev/null @@ -1,17 +0,0 @@ -SET hive.metastore.partition.name.whitelist.pattern=[^9]*; -set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifyTableDirectoryIsEmptyHook; - -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nonstrict; - -create table source_table like srcpart; - -create table dest_table like srcpart; - -load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE source_table partition(ds='2008-04-08', hr=11); - --- Tests creating dynamic partitions with characters not in the whitelist (i.e. 9) --- If the directory is not empty the hook will throw an error, instead the error should come from the metastore --- This shows that no dynamic partitions were created and left behind or had directories created - -insert overwrite table dest_table partition (ds, hr) select key, hr, ds, value from source_table where ds='2008-04-08' order by value asc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q deleted file mode 100644 index ca60d047efdd..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q +++ /dev/null @@ -1,12 +0,0 @@ -CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING, hr STRING); -CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRING); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - -ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='h1'); -ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='h2'); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - --- for exchange_part_test1 the value of ds is not given and the value of hr is given, thus this query will fail -alter table exchange_part_test1 exchange partition (hr='h1') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q deleted file mode 100644 index 7083edc32b98..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q +++ /dev/null @@ -1,12 +0,0 @@ -CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING); -CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05'); -ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05'); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - --- exchange_part_test1 table partition (ds='2013-04-05') already exists thus this query will fail -alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q deleted file mode 100644 index 6dfe81a8b056..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q +++ /dev/null @@ -1,13 +0,0 @@ -CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING, hr STRING); -CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRING); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='1'); -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='2'); -ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='3'); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - --- exchange_part_test1 table partition (ds='2013-04-05') already exists thus this query will fail -alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q deleted file mode 100644 index 60671e52e05d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q +++ /dev/null @@ -1,13 +0,0 @@ -CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING, hr STRING); -CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRING); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='1'); -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='2'); -ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='1'); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - --- exchange_part_test2 table partition (ds='2013-04-05') already exists thus this query will fail -alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q deleted file mode 100644 index 38c0eda2368b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING); -CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING); -SHOW PARTITIONS exchange_part_test1; - --- exchange_part_test2 partition (ds='2013-04-05') does not exist thus this query will fail -alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q deleted file mode 100644 index 7b926a3a8a51..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q +++ /dev/null @@ -1,2 +0,0 @@ --- t1 does not exist and the query fails -alter table t1 exchange partition (ds='2013-04-05') with table t2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q deleted file mode 100644 index 48fcd74a6f22..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q +++ /dev/null @@ -1,8 +0,0 @@ -CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING); -SHOW PARTITIONS exchange_part_test1; - -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05'); -SHOW PARTITIONS exchange_part_test1; - --- exchange_part_test2 table does not exist thus this query will fail -alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q deleted file mode 100644 index 23e86e96ca4b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q +++ /dev/null @@ -1,11 +0,0 @@ -CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING); -CREATE TABLE exchange_part_test2 (f1 string, f2 string) PARTITIONED BY (ds STRING); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05'); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - --- exchange_part_test1 and exchange_part_test2 do not have the same scheme and thus they fail -ALTER TABLE exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05') WITH TABLE exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q deleted file mode 100644 index 6ffc33acb92e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q +++ /dev/null @@ -1,12 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'nosuchschema://nosuchauthority/ql/test/data/exports/exim_department'; -drop table exim_department; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q deleted file mode 100644 index 970e6463e24a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q +++ /dev/null @@ -1,24 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department identifier") - stored as textfile - tblproperties("maker"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q deleted file mode 100644 index 358918363d83..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q +++ /dev/null @@ -1,38 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_employee ( emp_id int comment "employee id") - comment "employee table" - partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; -export table exim_employee to 'ql/test/data/exports/exim_employee'; -drop table exim_employee; - -create database importer; -use importer; - -create table exim_employee ( emp_id int comment "employee id") - comment "table of employees" - partitioned by (emp_country string comment "iso code", emp_state string comment "free-form text") - stored as textfile - tblproperties("maker"="krishna"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="ka"); -import from 'ql/test/data/exports/exim_employee'; -describe extended exim_employee; -select * from exim_employee; -drop table exim_employee; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q deleted file mode 100644 index 45268c21c00e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q +++ /dev/null @@ -1,23 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_key int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q deleted file mode 100644 index cad6c90fd316..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q +++ /dev/null @@ -1,23 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id", dep_name string) - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q deleted file mode 100644 index f5f904f42af5..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q +++ /dev/null @@ -1,23 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id bigint comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q deleted file mode 100644 index c56329c03f89..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q +++ /dev/null @@ -1,23 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - stored as rcfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q deleted file mode 100644 index afaedcd37bf7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q +++ /dev/null @@ -1,26 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - stored as inputformat "org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat" - outputformat "org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat" - inputdriver "org.apache.hadoop.hive.howl.rcfile.RCFileInputDriver" - outputdriver "org.apache.hadoop.hive.howl.rcfile.RCFileOutputDriver" - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q deleted file mode 100644 index 230b28c402cc..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q +++ /dev/null @@ -1,24 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q deleted file mode 100644 index c2e00a966346..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q +++ /dev/null @@ -1,28 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - row format serde "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe" - with serdeproperties ("serialization.format"="0") - stored as inputformat "org.apache.hadoop.mapred.TextInputFormat" - outputformat "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat" - inputdriver "org.apache.hadoop.hive.howl.rcfile.RCFileInputDriver" - outputdriver "org.apache.hadoop.hive.howl.rcfile.RCFileOutputDriver" - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q deleted file mode 100644 index a6586ead0c23..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q +++ /dev/null @@ -1,24 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - clustered by (dep_id) into 10 buckets - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q deleted file mode 100644 index 990a686ebeea..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q +++ /dev/null @@ -1,25 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - clustered by (dep_id) sorted by (dep_id desc) into 10 buckets - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - clustered by (dep_id) sorted by (dep_id asc) into 10 buckets - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q deleted file mode 100644 index 289bcf001fde..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q +++ /dev/null @@ -1,9 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - clustered by (dep_id) sorted by (dep_id desc) into 10 buckets - stored by "org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler" - tblproperties("creator"="krishna"); -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q deleted file mode 100644 index 02537ef022d8..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q +++ /dev/null @@ -1,24 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - stored by "org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler" - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; - \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q deleted file mode 100644 index 897c6747354b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q +++ /dev/null @@ -1,25 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - partitioned by (dep_org string) - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; - \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q deleted file mode 100644 index 12013e5ccfc4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q +++ /dev/null @@ -1,25 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - partitioned by (dep_org string) - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department partition (dep_org="hr"); -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; - \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q deleted file mode 100644 index d8d2b8008c9e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q +++ /dev/null @@ -1,26 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - partitioned by (dep_org string) - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department partition (dep_org="hr"); -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - partitioned by (dep_mgr string) - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; - \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q deleted file mode 100644 index 82dcce945595..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q +++ /dev/null @@ -1,30 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_employee ( emp_id int comment "employee id") - comment "employee table" - partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; -export table exim_employee to 'ql/test/data/exports/exim_employee'; -drop table exim_employee; - -create database importer; -use importer; -import table exim_employee partition (emp_country="us") from 'ql/test/data/exports/exim_employee'; -describe extended exim_employee; -select * from exim_employee; -drop table exim_employee; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q deleted file mode 100644 index d92efeb9a70e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q +++ /dev/null @@ -1,30 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_employee ( emp_id int comment "employee id") - comment "employee table" - partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; -export table exim_employee to 'ql/test/data/exports/exim_employee'; -drop table exim_employee; - -create database importer; -use importer; -import table exim_employee partition (emp_country="us", emp_state="kl") from 'ql/test/data/exports/exim_employee'; -describe extended exim_employee; -select * from exim_employee; -drop table exim_employee; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q deleted file mode 100644 index 12d827b9c838..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q +++ /dev/null @@ -1,23 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -import external table exim_department from 'ql/test/data/exports/exim_department'; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -drop table exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q deleted file mode 100644 index 726dee53955a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q +++ /dev/null @@ -1,30 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - location 'ql/test/data/tablestore/exim_department' - tblproperties("creator"="krishna"); -import table exim_department from 'ql/test/data/exports/exim_department' - location 'ql/test/data/tablestore2/exim_department'; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; - - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q deleted file mode 100644 index d187c7820203..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q +++ /dev/null @@ -1,35 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_employee ( emp_id int comment "employee id") - comment "employee table" - partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; -export table exim_employee to 'ql/test/data/exports/exim_employee'; -drop table exim_employee; - -create database importer; -use importer; - -create table exim_employee ( emp_id int comment "employee id") - comment "employee table" - partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") - stored as textfile - tblproperties("creator"="krishna"); -import external table exim_employee partition (emp_country="us", emp_state="tn") - from 'ql/test/data/exports/exim_employee'; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; -drop table exim_employee; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q deleted file mode 100644 index b818686f773d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int) stored as textfile; - -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; - -set hive.security.authorization.enabled=false; -drop table exim_department; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q deleted file mode 100644 index 4acefb9f0ae1..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q +++ /dev/null @@ -1,22 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int) stored as textfile; -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int) stored as textfile; -set hive.security.authorization.enabled=true; -import from 'ql/test/data/exports/exim_department'; - -set hive.security.authorization.enabled=false; -drop table exim_department; -drop database importer; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q deleted file mode 100644 index 467014e4679f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q +++ /dev/null @@ -1,31 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; -set hive.test.mode.nosamplelist=exim_department,exim_employee; - -create table exim_employee ( emp_id int comment "employee id") - comment "employee table" - partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="tn"); -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; -export table exim_employee to 'ql/test/data/exports/exim_employee'; -drop table exim_employee; - -create database importer; -use importer; -create table exim_employee ( emp_id int comment "employee id") - comment "employee table" - partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") - stored as textfile - tblproperties("creator"="krishna"); - -set hive.security.authorization.enabled=true; -import from 'ql/test/data/exports/exim_employee'; -set hive.security.authorization.enabled=false; - -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; -drop table exim_employee; -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q deleted file mode 100644 index 595fa7e76495..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q +++ /dev/null @@ -1,23 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; -set hive.test.mode.nosamplelist=exim_department,exim_employee; - -create table exim_department ( dep_id int) stored as textfile; -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -set hive.security.authorization.enabled=true; -import from 'ql/test/data/exports/exim_department'; - -set hive.security.authorization.enabled=false; -select * from exim_department; -drop table exim_department; -drop database importer; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external1.q deleted file mode 100644 index d56c955050bc..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external1.q +++ /dev/null @@ -1,3 +0,0 @@ - -create external table external1(a int, b int) location 'invalidscheme://data.s3ndemo.hive/kv'; -describe external1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external2.q deleted file mode 100644 index 0df85a09afdd..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external2.q +++ /dev/null @@ -1,4 +0,0 @@ - -create external table external2(a int, b int) partitioned by (ds string); -alter table external2 add partition (ds='2008-01-01') location 'invalidscheme://data.s3ndemo.hive/pkv/2008-01-01'; -describe external2 partition (ds='2008-01-01'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q deleted file mode 100644 index 82230f782eac..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q +++ /dev/null @@ -1,7 +0,0 @@ -CREATE TABLE fetchtask_ioexception ( - KEY STRING, - VALUE STRING) STORED AS SEQUENCEFILE; - -LOAD DATA LOCAL INPATH '../../data/files/kv1_broken.seq' OVERWRITE INTO TABLE fetchtask_ioexception; - -SELECT * FROM fetchtask_ioexception; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q deleted file mode 100644 index 286cf1afb491..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q +++ /dev/null @@ -1,13 +0,0 @@ -dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_file_with_header_footer_negative/; - -dfs -copyFromLocal ../data/files/header_footer_table_1 hdfs:///tmp/test_file_with_header_footer_negative/header_footer_table_1; - -dfs -copyFromLocal ../data/files/header_footer_table_2 hdfs:///tmp/test_file_with_header_footer_negative/header_footer_table_2; - -CREATE EXTERNAL TABLE header_footer_table_1 (name string, message string, id int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LOCATION 'hdfs:///tmp/test_file_with_header_footer_negative/header_footer_table_1' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="200"); - -SELECT * FROM header_footer_table_1; - -DROP TABLE header_footer_table_1; - -dfs -rmr hdfs:///tmp/test_file_with_header_footer_negative; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_bad_class.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_bad_class.q deleted file mode 100644 index 33dd4fa614f0..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_bad_class.q +++ /dev/null @@ -1,3 +0,0 @@ -CREATE TABLE dest1(key INT, value STRING) STORED AS - INPUTFORMAT 'ClassDoesNotExist' - OUTPUTFORMAT 'java.lang.Void'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_input.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_input.q deleted file mode 100644 index c514562b2416..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_input.q +++ /dev/null @@ -1,8 +0,0 @@ -CREATE TABLE dest1(key INT, value STRING) STORED AS - INPUTFORMAT 'java.lang.Void' - OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'; - -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 10; - -SELECT dest1.* FROM dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_output.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_output.q deleted file mode 100644 index a9cef1eada16..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_output.q +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TABLE dest1(key INT, value STRING) STORED AS - INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' - OUTPUTFORMAT 'java.lang.Void'; - -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name1.q deleted file mode 100644 index f50369b13857..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name1.q +++ /dev/null @@ -1,2 +0,0 @@ -set fs.default.name='http://www.example.com; -show tables; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name2.q deleted file mode 100644 index 485c3db06823..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name2.q +++ /dev/null @@ -1,2 +0,0 @@ -set fs.default.name='http://www.example.com; -SELECT * FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/genericFileFormat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/genericFileFormat.q deleted file mode 100644 index bd633b9760ab..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/genericFileFormat.q +++ /dev/null @@ -1 +0,0 @@ -create table testFail (a int) stored as foo; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q deleted file mode 100644 index cecd9c6bd807..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.map.aggr=true; -set hive.groupby.skewindata=true; -set mapred.reduce.tasks=31; - -CREATE TABLE dest1(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; - -EXPLAIN -FROM src -INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1); - -FROM src -INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1); - -SELECT dest1.* FROM dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q deleted file mode 100644 index e3b0066112c5..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.map.aggr=false; -set hive.groupby.skewindata=true; - -CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; - -EXPLAIN -FROM src -INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1); - -FROM src -INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1); - -SELECT dest_g2.* FROM dest_g2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q deleted file mode 100644 index 168aeb1261b3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q +++ /dev/null @@ -1,36 +0,0 @@ -set hive.map.aggr=true; -set hive.groupby.skewindata=true; -set mapred.reduce.tasks=31; - -CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE) STORED AS TEXTFILE; - -EXPLAIN -FROM src -INSERT OVERWRITE TABLE dest1 SELECT - sum(substr(src.value,5)), - avg(substr(src.value,5)), - avg(DISTINCT substr(src.value,5)), - max(substr(src.value,5)), - min(substr(src.value,5)), - std(substr(src.value,5)), - stddev_samp(substr(src.value,5)), - variance(substr(src.value,5)), - var_samp(substr(src.value,5)), - sum(DISTINCT substr(src.value, 5)), - count(DISTINCT substr(src.value, 5)); - -FROM src -INSERT OVERWRITE TABLE dest1 SELECT - sum(substr(src.value,5)), - avg(substr(src.value,5)), - avg(DISTINCT substr(src.value,5)), - max(substr(src.value,5)), - min(substr(src.value,5)), - std(substr(src.value,5)), - stddev_samp(substr(src.value,5)), - variance(substr(src.value,5)), - var_samp(substr(src.value,5)), - sum(DISTINCT substr(src.value, 5)), - count(DISTINCT substr(src.value, 5)); - -SELECT dest1.* FROM dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q deleted file mode 100644 index 1a28477918c8..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q +++ /dev/null @@ -1,36 +0,0 @@ -set hive.map.aggr=false; -set hive.groupby.skewindata=true; - -CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE) STORED AS TEXTFILE; - -EXPLAIN -FROM src -INSERT OVERWRITE TABLE dest1 SELECT - sum(substr(src.value,5)), - avg(substr(src.value,5)), - avg(DISTINCT substr(src.value,5)), - max(substr(src.value,5)), - min(substr(src.value,5)), - std(substr(src.value,5)), - stddev_samp(substr(src.value,5)), - variance(substr(src.value,5)), - var_samp(substr(src.value,5)), - sum(DISTINCT substr(src.value, 5)), - count(DISTINCT substr(src.value, 5)); - - -FROM src -INSERT OVERWRITE TABLE dest1 SELECT - sum(substr(src.value,5)), - avg(substr(src.value,5)), - avg(DISTINCT substr(src.value,5)), - max(substr(src.value,5)), - min(substr(src.value,5)), - std(substr(src.value,5)), - stddev_samp(substr(src.value,5)), - variance(substr(src.value,5)), - var_samp(substr(src.value,5)), - sum(DISTINCT substr(src.value, 5)), - count(DISTINCT substr(src.value, 5)); - -SELECT dest1.* FROM dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube1.q deleted file mode 100644 index a0bc177ad635..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube1.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.map.aggr=false; - -SELECT key, count(distinct value) FROM src GROUP BY key with cube; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube2.q deleted file mode 100644 index f8ecb6a2d434..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube2.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.map.aggr=true; - -SELECT key, value, count(distinct value) FROM src GROUP BY key, value with cube; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_id1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_id1.q deleted file mode 100644 index ac5b6f7b0305..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_id1.q +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; - -SELECT GROUPING__ID FROM T1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q deleted file mode 100644 index ec6b16bfb28c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q +++ /dev/null @@ -1,5 +0,0 @@ -CREATE TABLE T1(a STRING, b STRING, c STRING); - --- Check for empty grouping set -SELECT * FROM T1 GROUP BY a GROUPING SETS (()); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q deleted file mode 100644 index c988e04e74fa..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE T1(a STRING, b STRING, c STRING); - --- Check for mupltiple empty grouping sets -SELECT * FROM T1 GROUP BY b GROUPING SETS ((), (), ()); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q deleted file mode 100644 index 3e7355242295..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE T1(a STRING, b STRING, c STRING); - --- Grouping sets expression is not in GROUP BY clause -SELECT a FROM T1 GROUP BY a GROUPING SETS (a, b); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q deleted file mode 100644 index cf6352c47d7e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE T1(a STRING, b STRING, c STRING); - --- Expression 'a' is not in GROUP BY clause -SELECT a FROM T1 GROUP BY b GROUPING SETS (b); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q deleted file mode 100644 index 7df3318a644c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q +++ /dev/null @@ -1,5 +0,0 @@ -CREATE TABLE T1(a STRING, b STRING, c STRING); - --- Alias in GROUPING SETS -SELECT a as c, count(*) FROM T1 GROUP BY c GROUPING SETS (c); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q deleted file mode 100644 index 2783047698e7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q +++ /dev/null @@ -1,8 +0,0 @@ -set hive.new.job.grouping.set.cardinality=2; - -CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; - --- Since 4 grouping sets would be generated for the query below, an additional MR job should be created --- This is not allowed with distincts. -SELECT a, b, count(distinct c) from T1 group by a, b with cube; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q deleted file mode 100644 index 6c9d5133ad7e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.new.job.grouping.set.cardinality=2; -set hive.map.aggr=true; -set hive.groupby.skewindata=true; - -CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; - --- Since 4 grouping sets would be generated for the query below, an additional MR job should be created --- This is not allowed with map-side aggregation and skew -SELECT a, b, count(1) from T1 group by a, b with cube; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_invalid_position.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_invalid_position.q deleted file mode 100644 index 173a752e351a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_invalid_position.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.groupby.orderby.position.alias=true; - --- invalid position alias in group by -SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY 3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_key.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_key.q deleted file mode 100644 index 20970152c33c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_key.q +++ /dev/null @@ -1 +0,0 @@ -SELECT concat(value, concat(value)) FROM src GROUP BY concat(value); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup1.q deleted file mode 100644 index 636674427607..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup1.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.map.aggr=false; - -SELECT key, value, count(1) FROM src GROUP BY key, value with rollup; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup2.q deleted file mode 100644 index aa19b523e9d9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup2.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.map.aggr=true; - -SELECT key, value, count(key) FROM src GROUP BY key, value with rollup; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/having1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/having1.q deleted file mode 100644 index 71f4fd13a0a0..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/having1.q +++ /dev/null @@ -1,2 +0,0 @@ -EXPLAIN SELECT * FROM src HAVING key > 300; -SELECT * FROM src HAVING key > 300; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q deleted file mode 100644 index 1ab828c8beae..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q +++ /dev/null @@ -1,7 +0,0 @@ --- begin part(string, int) pass(string, string) -CREATE TABLE tab1 (id1 int,id2 string) PARTITIONED BY(month string,day int) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' ; -LOAD DATA LOCAL INPATH '../../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='second'); - -select * from tab1; -drop table tab1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type2.q deleted file mode 100644 index 243828820989..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type2.q +++ /dev/null @@ -1,3 +0,0 @@ -create table tab1 (id1 int, id2 string) PARTITIONED BY(month string,day int) row format delimited fields terminated by ','; -alter table tab1 add partition (month='June', day='second'); -drop table tab1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q deleted file mode 100644 index 49e6a092fc12..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q +++ /dev/null @@ -1,4 +0,0 @@ -create table tab1(c int) partitioned by (i int); -alter table tab1 add partition(i = "some name"); - -drop table tab1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q deleted file mode 100644 index 50f486e6245c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q +++ /dev/null @@ -1,3 +0,0 @@ -create table tab1(s string) PARTITIONED BY(dt date, st string); -alter table tab1 add partition (dt=date 'foo', st='foo'); -drop table tab1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q deleted file mode 100644 index a17cd1fec536..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q +++ /dev/null @@ -1,7 +0,0 @@ -EXPLAIN -CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; - -SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -SET hive.map.aggr=false; -CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; -ALTER INDEX src1_index ON src REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q deleted file mode 100644 index 5bb889c02774..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q +++ /dev/null @@ -1,12 +0,0 @@ -set hive.stats.dbclass=fs; -drop index src_index on src; - -CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; -ALTER INDEX src_index ON src REBUILD; - -SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" SELECT `_bucketname` , `_offsets` FROM default__src_src_index__ WHERE key<1000; -SET hive.index.compact.file=${system:test.tmp.dir}/index_result; -SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; -SET hive.index.compact.query.max.entries=5; -SELECT key, value FROM src WHERE key=100 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q deleted file mode 100644 index c6600e69b6a7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.stats.dbclass=fs; -drop index src_index on src; - -CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; -ALTER INDEX src_index ON src REBUILD; - -SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" SELECT `_bucketname` , `_offsets` FROM default__src_src_index__ WHERE key<1000; -SET hive.index.compact.file=${system:test.tmp.dir}/index_result; -SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; -SET hive.index.compact.query.max.size=1024; -SELECT key, value FROM src WHERE key=100 ORDER BY key; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input1.q deleted file mode 100644 index 92a6791acb65..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input1.q +++ /dev/null @@ -1 +0,0 @@ -SELECT a.* FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input2.q deleted file mode 100644 index 0fe907d9d8ae..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input2.q +++ /dev/null @@ -1 +0,0 @@ -SELECT a.key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input4.q deleted file mode 100644 index 60aea3208c4e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input4.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.mapred.mode=strict; - -select * from srcpart a join - (select b.key, count(1) as count from srcpart b where b.ds = '2008-04-08' and b.hr = '14' group by b.key) subq - where a.ds = '2008-04-08' and a.hr = '11' limit 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input41.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input41.q deleted file mode 100644 index 872ab1014874..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input41.q +++ /dev/null @@ -1,5 +0,0 @@ -select * from - (select * from src - union all - select * from srcpart where ds = '2009-08-09' - )x; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input_part0_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input_part0_neg.q deleted file mode 100644 index 4656693d4838..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input_part0_neg.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.mapred.mode=strict; - -SELECT x.* FROM SRCPART x WHERE key = '2008-04-08'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into1.q deleted file mode 100644 index 8c197670211b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into1.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.lock.numretries=5; -set hive.lock.sleep.between.retries=5; - -DROP TABLE insert_into1_neg; - -CREATE TABLE insert_into1_neg (key int, value string); - -LOCK TABLE insert_into1_neg SHARED; -INSERT INTO TABLE insert_into1_neg SELECT * FROM src LIMIT 100; - -DROP TABLE insert_into1_neg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into2.q deleted file mode 100644 index 73a3b6ff1370..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into2.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.lock.numretries=5; -set hive.lock.sleep.between.retries=5; - -DROP TABLE insert_into1_neg; -CREATE TABLE insert_into1_neg (key int, value string); - -LOCK TABLE insert_into1_neg EXCLUSIVE; -INSERT INTO TABLE insert_into1_neg SELECT * FROM src LIMIT 100; - -DROP TABLE insert_into1_neg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into3.q deleted file mode 100644 index 4d048b337ec4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into3.q +++ /dev/null @@ -1,16 +0,0 @@ -set hive.lock.numretries=5; -set hive.lock.sleep.between.retries=5; - -DROP TABLE insert_into3_neg; - -CREATE TABLE insert_into3_neg (key int, value string) - PARTITIONED BY (ds string); - -INSERT INTO TABLE insert_into3_neg PARTITION (ds='1') - SELECT * FROM src LIMIT 100; - -LOCK TABLE insert_into3_neg PARTITION (ds='1') SHARED; -INSERT INTO TABLE insert_into3_neg PARTITION (ds='1') - SELECT * FROM src LIMIT 100; - -DROP TABLE insert_into3_neg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into4.q deleted file mode 100644 index b8944e742b4d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into4.q +++ /dev/null @@ -1,16 +0,0 @@ -set hive.lock.numretries=5; -set hive.lock.sleep.between.retries=5; - -DROP TABLE insert_into3_neg; - -CREATE TABLE insert_into3_neg (key int, value string) - PARTITIONED BY (ds string); - -INSERT INTO TABLE insert_into3_neg PARTITION (ds='1') - SELECT * FROM src LIMIT 100; - -LOCK TABLE insert_into3_neg PARTITION (ds='1') EXCLUSIVE; -INSERT INTO TABLE insert_into3_neg PARTITION (ds='1') - SELECT * FROM src LIMIT 100; - -DROP TABLE insert_into3_neg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q deleted file mode 100644 index c20c168a887c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q +++ /dev/null @@ -1,9 +0,0 @@ -DROP TABLE if exists insert_into5_neg; - -CREATE TABLE insert_into5_neg (key int, value string) TBLPROPERTIES ("immutable"="true"); - -INSERT INTO TABLE insert_into5_neg SELECT * FROM src LIMIT 100; - -INSERT INTO TABLE insert_into5_neg SELECT * FROM src LIMIT 100; - -DROP TABLE insert_into5_neg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q deleted file mode 100644 index a92ee5ca94a3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q +++ /dev/null @@ -1,17 +0,0 @@ -DROP TABLE IF EXISTS insert_into6_neg; - -CREATE TABLE insert_into6_neg (key int, value string) - PARTITIONED BY (ds string) TBLPROPERTIES("immutable"="true") ; - -INSERT INTO TABLE insert_into6_neg PARTITION (ds='1') - SELECT * FROM src LIMIT 100; - -INSERT INTO TABLE insert_into6_neg PARTITION (ds='2') - SELECT * FROM src LIMIT 100; - -SELECT COUNT(*) from insert_into6_neg; - -INSERT INTO TABLE insert_into6_neg PARTITION (ds='1') - SELECT * FROM src LIMIT 100; - -DROP TABLE insert_into6_neg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_view_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_view_failure.q deleted file mode 100644 index 1f5e13906259..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_view_failure.q +++ /dev/null @@ -1,5 +0,0 @@ -DROP VIEW xxx2; -CREATE VIEW xxx2 AS SELECT * FROM src; -INSERT OVERWRITE TABLE xxx2 -SELECT key, value -FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertexternal1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertexternal1.q deleted file mode 100644 index 01ebae102232..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertexternal1.q +++ /dev/null @@ -1,8 +0,0 @@ -set hive.insert.into.external.tables=false; - - -create external table texternal(key string, val string) partitioned by (insertdate string); - -alter table texternal add partition (insertdate='2008-01-01') location 'pfile://${system:test.tmp.dir}/texternal/2008-01-01'; -from src insert overwrite table texternal partition (insertdate='2008-01-01') select *; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q deleted file mode 100644 index a8f77c28a825..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q +++ /dev/null @@ -1,9 +0,0 @@ -set hive.exec.dynamic.partition=true; - -create table srcpart_dp like srcpart; - -create table destpart_dp like srcpart; - -load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_dp partition(ds='2008-04-08', hr=11); - -insert overwrite table destpart_dp partition (ds='2008-04-08', hr) if not exists select key, value, hr from srcpart_dp where ds='2008-04-08'; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q deleted file mode 100644 index ad37cff79b58..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q +++ /dev/null @@ -1,3 +0,0 @@ - -select timestamp('2001-01-01 00:00:01') - timestamp('2000-01-01 00:00:01') from src; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_avg_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_avg_syntax.q deleted file mode 100644 index d5b58e076553..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_avg_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT avg(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q deleted file mode 100644 index 73e4729aa0fc..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q +++ /dev/null @@ -1,2 +0,0 @@ -create table tbl (a binary); -select cast (a as int) from tbl limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q deleted file mode 100644 index 50ec48152548..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q +++ /dev/null @@ -1,2 +0,0 @@ -create table tbl (a binary); -select cast (a as tinyint) from tbl limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q deleted file mode 100644 index 16f56ec5d340..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q +++ /dev/null @@ -1,2 +0,0 @@ -create table tbl (a binary); -select cast (a as smallint) from tbl limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q deleted file mode 100644 index bd222f14b469..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q +++ /dev/null @@ -1,2 +0,0 @@ -create table tbl (a binary); -select cast (a as bigint) from tbl limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q deleted file mode 100644 index 594fd2bb6f62..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q +++ /dev/null @@ -1,2 +0,0 @@ -create table tbl (a binary); -select cast (a as float) from tbl limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q deleted file mode 100644 index 40ff801460ef..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q +++ /dev/null @@ -1,2 +0,0 @@ -create table tbl (a binary); -select cast (a as double) from tbl limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q deleted file mode 100644 index 00cd98ed13b7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q +++ /dev/null @@ -1 +0,0 @@ -select cast (2 as binary) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q deleted file mode 100644 index f31344f835bb..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q +++ /dev/null @@ -1 +0,0 @@ -select cast(cast (2 as smallint) as binary) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q deleted file mode 100644 index af23d29f4e98..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q +++ /dev/null @@ -1 +0,0 @@ -select cast(cast (2 as tinyint) as binary) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q deleted file mode 100644 index 91abe1e6b8a2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q +++ /dev/null @@ -1 +0,0 @@ -select cast(cast (2 as bigint) as binary) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q deleted file mode 100644 index afd99be9765a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q +++ /dev/null @@ -1 +0,0 @@ -select cast(cast (2 as float) as binary) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q deleted file mode 100644 index c2143c5c9e95..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q +++ /dev/null @@ -1 +0,0 @@ -select cast(cast (2 as double) as binary) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q deleted file mode 100644 index ba7d164c7715..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q +++ /dev/null @@ -1,2 +0,0 @@ -drop table invalid_char_length_1; -create table invalid_char_length_1 (c1 char(1000000)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q deleted file mode 100644 index 866b43d31273..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q +++ /dev/null @@ -1 +0,0 @@ -select cast(value as char(100000)) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q deleted file mode 100644 index 481b630d2048..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q +++ /dev/null @@ -1,3 +0,0 @@ -drop table invalid_char_length_3; -create table invalid_char_length_3 (c1 char(0)); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config1.q deleted file mode 100644 index c49ac8a69086..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config1.q +++ /dev/null @@ -1,3 +0,0 @@ -set mapred.input.dir.recursive=true; - -CREATE TABLE skewedtable (key STRING, value STRING) SKEWED BY (key) ON (1,5,6); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config2.q deleted file mode 100644 index fa023c8c4b5f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config2.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.mapred.supports.subdirectories=false; -set hive.optimize.union.remove=true; - -select count(1) from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl1.q deleted file mode 100644 index 2e1ea6b00561..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl1.q +++ /dev/null @@ -1,9 +0,0 @@ - -CREATE TABLE inv_valid_tbl1 COMMENT 'This is a thrift based table' - PARTITIONED BY(aint DATETIME, country STRING) - CLUSTERED BY(aint) SORTED BY(lint) INTO 32 BUCKETS - ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer' - WITH SERDEPROPERTIES ('serialization.class' = 'org.apache.hadoop.hive.serde2.thrift.test.Complex', - 'serialization.format' = 'org.apache.thrift.protocol.TBinaryProtocol') - STORED AS SEQUENCEFILE; -DESCRIBE EXTENDED inv_valid_tbl1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl2.q deleted file mode 100644 index 408919ee2d63..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl2.q +++ /dev/null @@ -1 +0,0 @@ -create tabl tmp_zshao_22 (id int, name strin; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_mapjoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_mapjoin1.q deleted file mode 100644 index 56d9211d28eb..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_mapjoin1.q +++ /dev/null @@ -1 +0,0 @@ -select /*+ MAPJOIN(a) ,MAPJOIN(b)*/ * from src a join src b on (a.key=b.key and a.value=b.value); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_max_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_max_syntax.q deleted file mode 100644 index 20033734090f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_max_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT max(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_min_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_min_syntax.q deleted file mode 100644 index 584283a08a9e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_min_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT min(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column.q deleted file mode 100644 index 106ba4221319..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column.q +++ /dev/null @@ -1,4 +0,0 @@ --- Create table -create table if not exists test_invalid_column(key string, value string ) partitioned by (year string, month string) stored as textfile ; - -select * from test_invalid_column where column1=123; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q deleted file mode 100644 index bc70dbca2077..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q +++ /dev/null @@ -1,4 +0,0 @@ --- Create table -create table if not exists test_invalid_column(key string, value string ) partitioned by (year string, month string) stored as textfile ; - -select * from (select * from test_invalid_column) subq where subq = 123; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q deleted file mode 100644 index b821e6129a7b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q +++ /dev/null @@ -1,4 +0,0 @@ --- Create table -create table if not exists test_invalid_column(key string, value string ) partitioned by (year string, month string) stored as textfile ; - -select * from test_invalid_column where test_invalid_column=123; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_expression.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_expression.q deleted file mode 100644 index 01617f9363b5..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_expression.q +++ /dev/null @@ -1 +0,0 @@ -select foo from a a where foo > .foo; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_std_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_std_syntax.q deleted file mode 100644 index 13104198a6db..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_std_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT std(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q deleted file mode 100644 index c6a12526559e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT stddev_samp(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_sum_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_sum_syntax.q deleted file mode 100644 index 2d591baa24eb..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_sum_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT sum(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter1.q deleted file mode 100644 index bb19cff8a93e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter1.q +++ /dev/null @@ -1,2 +0,0 @@ -CREATE TABLE alter_test (d STRING); -ALTER TABLE alter_test CHANGE d d DATETIME; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter2.q deleted file mode 100644 index aa01b358727b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter2.q +++ /dev/null @@ -1,2 +0,0 @@ -CREATE TABLE alter_test (d STRING); -ALTER TABLE alter_test ADD COLUMNS (ds DATETIME); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_create2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_create2.q deleted file mode 100644 index 978f4244a6ba..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_create2.q +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE datetime_test (d DATETIME); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_transform.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_transform.q deleted file mode 100644 index dfc4864acf43..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_transform.q +++ /dev/null @@ -1 +0,0 @@ -SELECT TRANSFORM(*) USING 'cat' AS (key DATETIME) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_tbl_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_tbl_name.q deleted file mode 100644 index 09394e71ada9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_tbl_name.q +++ /dev/null @@ -1 +0,0 @@ -create table invalid-name(a int, b string); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q deleted file mode 100644 index ce2a8c476911..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT var_samp(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q deleted file mode 100644 index 43de018c9f14..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q +++ /dev/null @@ -1,2 +0,0 @@ -drop table if exists invalid_varchar_length_1; -create table invalid_varchar_length_1 (c1 varchar(1000000)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q deleted file mode 100644 index 3c199d31e7ff..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q +++ /dev/null @@ -1 +0,0 @@ -select cast(value as varchar(100000)) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q deleted file mode 100644 index fed04764a944..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q +++ /dev/null @@ -1,3 +0,0 @@ -drop table if exists invalid_varchar_length_3; -create table invalid_varchar_length_3 (c1 varchar(0)); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_variance_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_variance_syntax.q deleted file mode 100644 index 5b478299317a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_variance_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT variance(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalidate_view1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalidate_view1.q deleted file mode 100644 index dd39c5eb4a4f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalidate_view1.q +++ /dev/null @@ -1,11 +0,0 @@ -DROP VIEW xxx8; -DROP VIEW xxx9; - --- create two levels of view reference, then invalidate intermediate view --- by dropping a column from underlying table, and verify that --- querying outermost view results in full error context -CREATE TABLE xxx10 (key int, value int); -CREATE VIEW xxx9 AS SELECT * FROM xxx10; -CREATE VIEW xxx8 AS SELECT * FROM xxx9 xxx; -ALTER TABLE xxx10 REPLACE COLUMNS (key int); -SELECT * FROM xxx8 yyy; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join2.q deleted file mode 100644 index 98a5f1e6629c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join2.q +++ /dev/null @@ -1,5 +0,0 @@ -SELECT /*+ MAPJOIN(x) */ x.key, x.value, y.value -FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key); - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join28.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join28.q deleted file mode 100644 index 32ff105c2e45..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join28.q +++ /dev/null @@ -1,15 +0,0 @@ -CREATE TABLE dest_j1(key STRING, value STRING) STORED AS TEXTFILE; - --- Mapjoin followed by mapjoin is not supported. --- The same query would work fine without the hint. --- Note that there is a positive test with the same name in clientpositive -EXPLAIN -INSERT OVERWRITE TABLE dest_j1 -SELECT /*+ MAPJOIN(z) */ subq.key1, z.value -FROM -(SELECT /*+ MAPJOIN(x) */ x.key as key1, x.value as value1, y.key as key2, y.value as value2 - FROM src1 x JOIN src y ON (x.key = y.key)) subq - JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11); - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join29.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join29.q deleted file mode 100644 index 53a1652d25b2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join29.q +++ /dev/null @@ -1,10 +0,0 @@ -CREATE TABLE dest_j1(key STRING, cnt1 INT, cnt2 INT); - --- Mapjoin followed by group by is not supported. --- The same query would work without the hint --- Note that there is a positive test with the same name in clientpositive -EXPLAIN -INSERT OVERWRITE TABLE dest_j1 -SELECT /*+ MAPJOIN(subq1) */ subq1.key, subq1.cnt, subq2.cnt -FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN - (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join32.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join32.q deleted file mode 100644 index 54a4dcd9afe2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join32.q +++ /dev/null @@ -1,14 +0,0 @@ -CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE; - --- Mapjoin followed by Mapjoin is not supported. --- The same query would work without the hint --- Note that there is a positive test with the same name in clientpositive -EXPLAIN EXTENDED -INSERT OVERWRITE TABLE dest_j1 -SELECT /*+ MAPJOIN(x,z) */ x.key, z.value, y.value -FROM src1 x JOIN src y ON (x.key = y.key) -JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11); - - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join35.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join35.q deleted file mode 100644 index fc8f77ca1232..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join35.q +++ /dev/null @@ -1,18 +0,0 @@ -CREATE TABLE dest_j1(key STRING, value STRING, val2 INT) STORED AS TEXTFILE; - --- Mapjoin followed by union is not supported. --- The same query would work without the hint --- Note that there is a positive test with the same name in clientpositive -EXPLAIN EXTENDED -INSERT OVERWRITE TABLE dest_j1 -SELECT /*+ MAPJOIN(x) */ x.key, x.value, subq1.cnt -FROM -( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key - UNION ALL - SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key -) subq1 -JOIN src1 x ON (x.key = subq1.key); - - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q deleted file mode 100644 index e39a38e2fcd4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q +++ /dev/null @@ -1,3 +0,0 @@ -explain select * -from src s1 , -src s2 on s1.key = s2.key; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q deleted file mode 100644 index c0da913c2881..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q +++ /dev/null @@ -1,6 +0,0 @@ - - -explain select s1.key, s2.key -from src s1, src s2 -where key = s2.key -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q deleted file mode 100644 index 8e219637eb0c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q +++ /dev/null @@ -1,5 +0,0 @@ - -explain select s1.key, s2.key -from src s1, src s2 -where INPUT__FILE__NAME = s2.INPUT__FILE__NAME -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_nonexistent_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_nonexistent_part.q deleted file mode 100644 index b4a4757d2214..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_nonexistent_part.q +++ /dev/null @@ -1,4 +0,0 @@ -SET hive.security.authorization.enabled = true; -SELECT * -FROM srcpart s1 join src s2 on s1.key == s2.key -WHERE s1.ds='non-existent'; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/joinneg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/joinneg.q deleted file mode 100644 index a4967fd5dfb4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/joinneg.q +++ /dev/null @@ -1,6 +0,0 @@ -EXPLAIN FROM -(SELECT src.* FROM src) x -JOIN -(SELECT src.* FROM src) Y -ON (x.key = b.key) -SELECT Y.*; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_alias.q deleted file mode 100644 index 50d535e6e1ec..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_alias.q +++ /dev/null @@ -1,3 +0,0 @@ --- Check alias count for LATERAL VIEW syntax: --- explode returns a table with only 1 col - should be an error if query specifies >1 col aliases -SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol1, myCol2 LIMIT 3; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_join.q deleted file mode 100644 index 818754ecbf05..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_join.q +++ /dev/null @@ -1 +0,0 @@ -SELECT src.key FROM src LATERAL VIEW explode(array(1,2,3)) AS myTable JOIN src b ON src.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q deleted file mode 100644 index d59394544ccf..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q +++ /dev/null @@ -1,7 +0,0 @@ -set hive.limit.query.max.table.partition=1; - -explain select * from srcpart limit 1; -select * from srcpart limit 1; - -explain select * from srcpart; -select * from srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q deleted file mode 100644 index 0afd4a965ab9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q +++ /dev/null @@ -1,18 +0,0 @@ -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nonstrict; -set hive.stats.autogather=true; -set hive.compute.query.using.stats=true; - -create table part (c int) partitioned by (d string); -insert into table part partition (d) -select hr,ds from srcpart; - -set hive.limit.query.max.table.partition=1; - -explain select count(*) from part; -select count(*) from part; - -set hive.compute.query.using.stats=false; - -explain select count(*) from part; -select count(*) from part; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/line_terminator.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/line_terminator.q deleted file mode 100644 index ad3542c40ace..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/line_terminator.q +++ /dev/null @@ -1,3 +0,0 @@ -CREATE TABLE mytable (col1 STRING, col2 INT) -ROW FORMAT DELIMITED -LINES TERMINATED BY ','; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q deleted file mode 100644 index eb72d940a539..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q +++ /dev/null @@ -1,4 +0,0 @@ -create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile; -alter table hive_test_src add partition (pcol1 = 'test_part'); -set hive.security.authorization.enabled=true; -load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q deleted file mode 100644 index 75a5216e00d8..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q +++ /dev/null @@ -1,5 +0,0 @@ - -CREATE TABLE non_native2(key int, value string) -STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; - -LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE non_native2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q deleted file mode 100644 index 32653631ad6a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q +++ /dev/null @@ -1,3 +0,0 @@ -create table hive_test_src ( col1 string ) stored as textfile; -set hive.security.authorization.enabled=true; -load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q deleted file mode 100644 index 315988dc0a95..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q +++ /dev/null @@ -1,3 +0,0 @@ -create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile; -set hive.security.authorization.enabled=true; -load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q deleted file mode 100644 index 81517991b26f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q +++ /dev/null @@ -1,2 +0,0 @@ -create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile; -load data local inpath '../../data/files/test.dat' into table hive_test_src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q deleted file mode 100644 index c56f0d408d4a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q +++ /dev/null @@ -1,7 +0,0 @@ -set hive.mapred.supports.subdirectories=true; - --- Load data can't work with table with stored as directories -CREATE TABLE if not exists stored_as_dirs_multiple (col1 STRING, col2 int, col3 STRING) -SKEWED BY (col1, col2) ON (('s1',1), ('s3',3), ('s13',13), ('s78',78)) stored as DIRECTORIES; - -LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE stored_as_dirs_multiple; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q deleted file mode 100644 index 64182eac8362..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q +++ /dev/null @@ -1,3 +0,0 @@ -DROP VIEW xxx11; -CREATE VIEW xxx11 AS SELECT * FROM src; -LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE xxx11; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q deleted file mode 100644 index f0c3b59d30dd..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q +++ /dev/null @@ -1,6 +0,0 @@ --- test for loading into tables with the correct file format --- test for loading into partitions with the correct file format - - -CREATE TABLE load_wrong_fileformat_T1(name STRING) STORED AS SEQUENCEFILE; -LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE load_wrong_fileformat_T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q deleted file mode 100644 index 4d79bbeb102c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q +++ /dev/null @@ -1,6 +0,0 @@ --- test for loading into tables with the correct file format --- test for loading into partitions with the correct file format - - -CREATE TABLE T1(name STRING) STORED AS RCFILE; -LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T1; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q deleted file mode 100644 index 050c819a2f04..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q +++ /dev/null @@ -1,6 +0,0 @@ --- test for loading into tables with the correct file format --- test for loading into partitions with the correct file format - - -CREATE TABLE T1(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T1; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q deleted file mode 100644 index 7f5ad754142a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q +++ /dev/null @@ -1,3 +0,0 @@ - -CREATE TABLE loadpart1(a STRING, b STRING) PARTITIONED BY (ds STRING,ds1 STRING); -LOAD DATA LOCAL INPATH '../../data1/files/kv1.txt' INTO TABLE loadpart1 PARTITION(ds='2009-05-05'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q deleted file mode 100644 index ed9e21dd8a1f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.exec.mode.local.auto=true; -set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateLocalErrorsHook; - -FROM src SELECT TRANSFORM(key, value) USING 'python ../../data/scripts/cat_error.py' AS (key, value); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg1.q deleted file mode 100644 index e1b58fca80af..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg1.q +++ /dev/null @@ -1,10 +0,0 @@ -drop table tstsrc; -create table tstsrc like src; -insert overwrite table tstsrc select key, value from src; - -set hive.lock.numretries=0; -set hive.unlock.numretries=0; - -LOCK TABLE tstsrc SHARED; -LOCK TABLE tstsrc SHARED; -LOCK TABLE tstsrc EXCLUSIVE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg2.q deleted file mode 100644 index a4604cd47065..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg2.q +++ /dev/null @@ -1,6 +0,0 @@ -drop table tstsrc; -create table tstsrc like src; -insert overwrite table tstsrc select key, value from src; - -set hive.unlock.numretries=0; -UNLOCK TABLE tstsrc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg3.q deleted file mode 100644 index f2252f7bdf4d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg3.q +++ /dev/null @@ -1,9 +0,0 @@ -drop table tstsrcpart; -create table tstsrcpart like srcpart; - -insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') -select key, value from srcpart where ds='2008-04-08' and hr='11'; - -set hive.lock.numretries=0; -set hive.unlock.numretries=0; -UNLOCK TABLE tstsrcpart PARTITION(ds='2008-04-08', hr='11'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg4.q deleted file mode 100644 index b47644cca362..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg4.q +++ /dev/null @@ -1,12 +0,0 @@ -drop table tstsrcpart; -create table tstsrcpart like srcpart; - -insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') -select key, value from srcpart where ds='2008-04-08' and hr='11'; - -set hive.lock.numretries=0; -set hive.unlock.numretries=0; - -LOCK TABLE tstsrcpart PARTITION(ds='2008-04-08', hr='11') EXCLUSIVE; -SHOW LOCKS tstsrcpart PARTITION(ds='2008-04-08', hr='12'); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg5.q deleted file mode 100644 index 19c1ce28c242..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg5.q +++ /dev/null @@ -1,2 +0,0 @@ -drop table tstsrcpart; -show locks tstsrcpart extended; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q deleted file mode 100644 index 4966f2b9b282..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q +++ /dev/null @@ -1,17 +0,0 @@ -create database lockneg1; -use lockneg1; - -create table tstsrcpart like default.srcpart; - -insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') -select key, value from default.srcpart where ds='2008-04-08' and hr='11'; - -lock database lockneg1 shared; -show locks database lockneg1; -select count(1) from tstsrcpart where ds='2008-04-08' and hr='11'; - -unlock database lockneg1; -show locks database lockneg1; -lock database lockneg1 exclusive; -show locks database lockneg1; -select count(1) from tstsrcpart where ds='2008-04-08' and hr='11'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q deleted file mode 100644 index 1f9ad90898dc..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.lock.numretries=0; - -create database lockneg4; - -lock database lockneg4 exclusive; -lock database lockneg4 shared; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q deleted file mode 100644 index 8cbe31083b40..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q +++ /dev/null @@ -1,8 +0,0 @@ -set hive.lock.numretries=0; - -create database lockneg9; - -lock database lockneg9 shared; -show locks database lockneg9; - -drop database lockneg9; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q deleted file mode 100644 index 4127a6f150a1..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q +++ /dev/null @@ -1,15 +0,0 @@ -set hive.lock.numretries=0; - -create database lockneg2; -use lockneg2; - -create table tstsrcpart like default.srcpart; - -insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') -select key, value from default.srcpart where ds='2008-04-08' and hr='11'; - -lock table tstsrcpart shared; -show locks; - -lock database lockneg2 exclusive; -show locks; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/macro_unused_parameter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/macro_unused_parameter.q deleted file mode 100644 index 523710ddf3a5..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/macro_unused_parameter.q +++ /dev/null @@ -1 +0,0 @@ -CREATE TEMPORARY MACRO BAD_MACRO (x INT, y INT) x; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q deleted file mode 100644 index 76c7ae94d4b6..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.exec.mode.local.auto=false; -set hive.exec.job.debug.capture.stacktraces=true; -set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateStackTracesHook; - -FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value); - --- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) --- Hadoop 0.23 changes the getTaskDiagnostics behavior --- The Error Code of hive failure MapReduce job changes --- In Hadoop 0.20 --- Hive failure MapReduce job gets 20000 as Error Code --- In Hadoop 0.23 --- Hive failure MapReduce job gets 2 as Error Code diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q deleted file mode 100644 index 9d0548cc10f5..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.exec.mode.local.auto=false; -set hive.exec.job.debug.capture.stacktraces=true; -set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateStackTracesHook; - -FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value); - --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.23) --- Hadoop 0.23 changes the getTaskDiagnostics behavior --- The Error Code of hive failure MapReduce job changes --- In Hadoop 0.20 --- Hive failure MapReduce job gets 20000 as Error Code --- In Hadoop 0.23 --- Hive failure MapReduce job gets 2 as Error Code diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q deleted file mode 100644 index c93aedb3137b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.exec.mode.local.auto=false; -set hive.exec.job.debug.capture.stacktraces=false; -set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateStackTracesHook; - -FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value); - --- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) --- Hadoop 0.23 changes the getTaskDiagnostics behavior --- The Error Code of hive failure MapReduce job changes --- In Hadoop 0.20 --- Hive failure MapReduce job gets 20000 as Error Code --- In Hadoop 0.23 --- Hive failure MapReduce job gets 2 as Error Code diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q deleted file mode 100644 index e319944958c2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.exec.mode.local.auto=false; -set hive.exec.job.debug.capture.stacktraces=false; -set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateStackTracesHook; - -FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value); - --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.23) --- Hadoop 0.23 changes the getTaskDiagnostics behavior --- The Error Code of hive failure MapReduce job changes --- In Hadoop 0.20 --- Hive failure MapReduce job gets 20000 as Error Code --- In Hadoop 0.23 --- Hive failure MapReduce job gets 2 as Error Code diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_1.q deleted file mode 100644 index 0a48c01546ec..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_1.q +++ /dev/null @@ -1,3 +0,0 @@ -create table src2 like src; -CREATE INDEX src_index_merge_test ON TABLE src2(key) as 'COMPACT' WITH DEFERRED REBUILD; -alter table src2 concatenate; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_2.q deleted file mode 100644 index a4fab1c8b804..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_2.q +++ /dev/null @@ -1,3 +0,0 @@ -create table srcpart2 (key int, value string) partitioned by (ds string); -insert overwrite table srcpart2 partition (ds='2011') select * from src; -alter table srcpart2 concatenate; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_3.q deleted file mode 100644 index 6bc645e4c237..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_3.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.enforce.bucketing=true; -set hive.enforce.sorting=true; - -create table srcpart2 (key int, value string) partitioned by (ds string) clustered by (key) sorted by (key) into 2 buckets stored as RCFILE; -insert overwrite table srcpart2 partition (ds='2011') select * from src; -alter table srcpart2 partition (ds = '2011') concatenate; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/minimr_broken_pipe.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/minimr_broken_pipe.q deleted file mode 100644 index 8dda9cdf4a37..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/minimr_broken_pipe.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -set hive.exec.script.allow.partial.consumption = false; --- Tests exception in ScriptOperator.close() by passing to the operator a small amount of data -SELECT TRANSFORM(*) USING 'true' AS a, b FROM (SELECT TRANSFORM(*) USING 'echo' AS a, b FROM src LIMIT 1) tmp; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q deleted file mode 100644 index 09f13f52aead..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q +++ /dev/null @@ -1,15 +0,0 @@ - -create table nestedcomplex ( -simple_int int, -max_nested_array array>>>>>>>>>>>>>>>>>>>>>>, -max_nested_map array>>>>>>>>>>>>>>>>>>>>>, -max_nested_struct array>>>>>>>>>>>>>>>>>>>>>>, -simple_string string) - -; - - --- This should fail in as extended nesting levels are not enabled using the serdeproperty hive.serialization.extend.nesting.levels -load data local inpath '../../data/files/nested_complex.txt' overwrite into table nestedcomplex; - -select * from nestedcomplex sort by simple_int; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/no_matching_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/no_matching_udf.q deleted file mode 100644 index 0c24b1626a53..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/no_matching_udf.q +++ /dev/null @@ -1 +0,0 @@ -SELECT percentile(3.5, 0.99) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nonkey_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nonkey_groupby.q deleted file mode 100644 index 431e04efd934..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nonkey_groupby.q +++ /dev/null @@ -1 +0,0 @@ -EXPLAIN SELECT key, count(1) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q deleted file mode 100644 index 6669bf62d882..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q +++ /dev/null @@ -1,7 +0,0 @@ - -CREATE TABLE nopart_insert(a STRING, b STRING) PARTITIONED BY (ds STRING); - -INSERT OVERWRITE TABLE nopart_insert -SELECT TRANSFORM(src.key, src.value) USING '../../data/scripts/error_script' AS (tkey, tvalue) -FROM src; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q deleted file mode 100644 index 966982fd5ce5..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q +++ /dev/null @@ -1,5 +0,0 @@ - -CREATE TABLE nopart_load(a STRING, b STRING) PARTITIONED BY (ds STRING); - -load data local inpath '../../data/files/kv1.txt' overwrite into table nopart_load ; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias4.q deleted file mode 100644 index e7ad6b79d3ed..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias4.q +++ /dev/null @@ -1,4 +0,0 @@ -EXPLAIN -SELECT key from src JOIN src1 on src1.key=src.key; - -SELECT key from src JOIN src1 on src1.key=src.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_invalid_position.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_invalid_position.q deleted file mode 100644 index 4dbf2a6d56a2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_invalid_position.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.groupby.orderby.position.alias=true; - --- invalid position alias in order by -SELECT src.key, src.value FROM src ORDER BY 0; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_position_unsupported.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_position_unsupported.q deleted file mode 100644 index a490c2306ec4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_position_unsupported.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.groupby.orderby.position.alias=true; - --- position alias is not supported when SELECT * -SELECT src.* FROM src ORDER BY 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderbysortby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderbysortby.q deleted file mode 100644 index 5dff69fdbb78..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderbysortby.q +++ /dev/null @@ -1,8 +0,0 @@ -CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; - -FROM src -INSERT OVERWRITE TABLE dest1 -MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value -USING 'cat' AS (tkey, ten, one, tvalue) -ORDER BY tvalue, tkey -SORT BY ten, one; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q deleted file mode 100644 index 745a7867264e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q +++ /dev/null @@ -1,3 +0,0 @@ -drop table if exists parquet_char; - -create table parquet_char (t char(10)) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q deleted file mode 100644 index 89d3602fd3e9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q +++ /dev/null @@ -1,3 +0,0 @@ -drop table if exists parquet_date; - -create table parquet_date (t date) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q deleted file mode 100644 index 8a4973110a51..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q +++ /dev/null @@ -1,3 +0,0 @@ -drop table if exists parquet_decimal; - -create table parquet_decimal (t decimal(4,2)) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q deleted file mode 100644 index 4ef36fa0efc4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q +++ /dev/null @@ -1,3 +0,0 @@ -drop table if exists parquet_timestamp; - -create table parquet_timestamp (t timestamp) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q deleted file mode 100644 index 55825f76dc24..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q +++ /dev/null @@ -1,3 +0,0 @@ -drop table if exists parquet_varchar; - -create table parquet_varchar (t varchar(10)) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/part_col_complex_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/part_col_complex_type.q deleted file mode 100644 index 4b9eb847db54..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/part_col_complex_type.q +++ /dev/null @@ -1 +0,0 @@ -create table t (a string) partitioned by (b map); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part.q deleted file mode 100644 index 541599915afc..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part.q +++ /dev/null @@ -1,15 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode3; - -create table tbl_protectmode3 (col string) partitioned by (p string); -alter table tbl_protectmode3 add partition (p='p1'); -alter table tbl_protectmode3 add partition (p='p2'); - -select * from tbl_protectmode3 where p='p1'; -select * from tbl_protectmode3 where p='p2'; - -alter table tbl_protectmode3 partition (p='p1') enable offline; - -select * from tbl_protectmode3 where p='p2'; -select * from tbl_protectmode3 where p='p1'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part1.q deleted file mode 100644 index 99256da285c1..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part1.q +++ /dev/null @@ -1,21 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode5; - -create table tbl_protectmode5_1 (col string); - -create table tbl_protectmode5 (col string) partitioned by (p string); -alter table tbl_protectmode5 add partition (p='p1'); -alter table tbl_protectmode5 add partition (p='p2'); - -insert overwrite table tbl_protectmode5_1 -select col from tbl_protectmode5 where p='p1'; -insert overwrite table tbl_protectmode5_1 -select col from tbl_protectmode5 where p='p2'; - -alter table tbl_protectmode5 partition (p='p1') enable offline; - -insert overwrite table tbl_protectmode5_1 -select col from tbl_protectmode5 where p='p2'; -insert overwrite table tbl_protectmode5_1 -select col from tbl_protectmode5 where p='p1'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q deleted file mode 100644 index 3fdc03699656..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q +++ /dev/null @@ -1,9 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode6; - -create table tbl_protectmode6 (c1 string,c2 string) partitioned by (p string); -alter table tbl_protectmode6 add partition (p='p1'); -LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1'); -alter table tbl_protectmode6 partition (p='p1') enable offline; -LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q deleted file mode 100644 index b4e508ff9818..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q +++ /dev/null @@ -1,10 +0,0 @@ --- protect mode: syntax to change protect mode works and queries to drop partitions are blocked if it is marked no drop - -drop table tbl_protectmode_no_drop; - -create table tbl_protectmode_no_drop (c1 string,c2 string) partitioned by (p string); -alter table tbl_protectmode_no_drop add partition (p='p1'); -alter table tbl_protectmode_no_drop partition (p='p1') enable no_drop; -desc extended tbl_protectmode_no_drop partition (p='p1'); - -alter table tbl_protectmode_no_drop drop partition (p='p1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl1.q deleted file mode 100644 index 236129902c07..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl1.q +++ /dev/null @@ -1,8 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode_1; - -create table tbl_protectmode_1 (col string); -select * from tbl_protectmode_1; -alter table tbl_protectmode_1 enable offline; -select * from tbl_protectmode_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl2.q deleted file mode 100644 index 05964c35e9e0..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl2.q +++ /dev/null @@ -1,12 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode2; - -create table tbl_protectmode2 (col string) partitioned by (p string); -alter table tbl_protectmode2 add partition (p='p1'); -alter table tbl_protectmode2 enable no_drop; -alter table tbl_protectmode2 enable offline; -alter table tbl_protectmode2 disable no_drop; -desc extended tbl_protectmode2; - -select * from tbl_protectmode2 where p='p1'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl3.q deleted file mode 100644 index bbaa2670875b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl3.q +++ /dev/null @@ -1,10 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode_4; - -create table tbl_protectmode_4 (col string); -select col from tbl_protectmode_4; -alter table tbl_protectmode_4 enable offline; -desc extended tbl_protectmode_4; - -select col from tbl_protectmode_4; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl4.q deleted file mode 100644 index c7880de6d8ae..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl4.q +++ /dev/null @@ -1,15 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode_tbl4; -drop table tbl_protectmode_tbl4_src; - -create table tbl_protectmode_tbl4_src (col string); - -create table tbl_protectmode_tbl4 (col string) partitioned by (p string); -alter table tbl_protectmode_tbl4 add partition (p='p1'); -alter table tbl_protectmode_tbl4 enable no_drop; -alter table tbl_protectmode_tbl4 enable offline; -alter table tbl_protectmode_tbl4 disable no_drop; -desc extended tbl_protectmode_tbl4; - -select col from tbl_protectmode_tbl4 where p='not_exist'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl5.q deleted file mode 100644 index cd848fd4a1b9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl5.q +++ /dev/null @@ -1,15 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode_tbl5; -drop table tbl_protectmode_tbl5_src; - -create table tbl_protectmode_tbl5_src (col string); - -create table tbl_protectmode_tbl5 (col string) partitioned by (p string); -alter table tbl_protectmode_tbl5 add partition (p='p1'); -alter table tbl_protectmode_tbl5 enable no_drop; -alter table tbl_protectmode_tbl5 enable offline; -alter table tbl_protectmode_tbl5 disable no_drop; -desc extended tbl_protectmode_tbl5; - -insert overwrite table tbl_protectmode_tbl5 partition (p='not_exist') select col from tbl_protectmode_tbl5_src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl6.q deleted file mode 100644 index 26248cc6b487..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl6.q +++ /dev/null @@ -1,8 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode_tbl6; - -create table tbl_protectmode_tbl6 (col string); -alter table tbl_protectmode_tbl6 enable no_drop cascade; - -drop table tbl_protectmode_tbl6; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl7.q deleted file mode 100644 index afff8404edc0..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl7.q +++ /dev/null @@ -1,13 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode_tbl7; -create table tbl_protectmode_tbl7 (col string) partitioned by (p string); -alter table tbl_protectmode_tbl7 add partition (p='p1'); -alter table tbl_protectmode_tbl7 enable no_drop; - -alter table tbl_protectmode_tbl7 drop partition (p='p1'); - -alter table tbl_protectmode_tbl7 add partition (p='p1'); -alter table tbl_protectmode_tbl7 enable no_drop cascade; - -alter table tbl_protectmode_tbl7 drop partition (p='p1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl8.q deleted file mode 100644 index 809c287fc502..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl8.q +++ /dev/null @@ -1,13 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode_tbl8; -create table tbl_protectmode_tbl8 (col string) partitioned by (p string); -alter table tbl_protectmode_tbl8 add partition (p='p1'); -alter table tbl_protectmode_tbl8 enable no_drop; - -alter table tbl_protectmode_tbl8 drop partition (p='p1'); - -alter table tbl_protectmode_tbl8 enable no_drop cascade; - -alter table tbl_protectmode_tbl8 add partition (p='p1'); -alter table tbl_protectmode_tbl8 drop partition (p='p1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q deleted file mode 100644 index a4ef2acbfd40..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q +++ /dev/null @@ -1,9 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode__no_drop; - -create table tbl_protectmode__no_drop (col string); -select * from tbl_protectmode__no_drop; -alter table tbl_protectmode__no_drop enable no_drop; -desc extended tbl_protectmode__no_drop; -drop table tbl_protectmode__no_drop; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q deleted file mode 100644 index ef372259ed3e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q +++ /dev/null @@ -1,20 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; - --- testAggrFuncsWithNoGBYNoPartDef -select p_mfgr, -sum(p_retailprice) as s1 -from part; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q deleted file mode 100644 index 58430423436b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q +++ /dev/null @@ -1,28 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; - --- testAmbiguousWindowDefn -select p_mfgr, p_name, p_size, -sum(p_size) over (w1) as s1, -sum(p_size) over (w2) as s2, -sum(p_size) over (w3) as s3 -from part -distribute by p_mfgr -sort by p_mfgr -window w1 as (rows between 2 preceding and 2 following), - w2 as (rows between unbounded preceding and current row), - w3 as w3; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q deleted file mode 100644 index caebebf8eaa4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q +++ /dev/null @@ -1,19 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - --- testPartitonBySortBy -select p_mfgr, p_name, p_size, -sum(p_retailprice) over (distribute by p_mfgr order by p_mfgr) as s1 -from part -; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q deleted file mode 100644 index 3a0304188d2a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q +++ /dev/null @@ -1,22 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - --- testDuplicateWindowAlias -select p_mfgr, p_name, p_size, -sum(p_size) over (w1) as s1, -sum(p_size) over (w2) as s2 -from part -window w1 as (partition by p_mfgr order by p_mfgr rows between 2 preceding and 2 following), - w2 as w1, - w2 as (rows between unbounded preceding and current row); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q deleted file mode 100644 index f351a1448b15..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q +++ /dev/null @@ -1,20 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - --- testHavingLeadWithNoGBYNoWindowing -select p_mfgr,p_name, p_size -from part -having lead(p_size, 1) over() <= p_size -distribute by p_mfgr -sort by p_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q deleted file mode 100644 index d0d3d3fae23f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q +++ /dev/null @@ -1,22 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - --- testHavingLeadWithPTF -select p_mfgr,p_name, p_size -from noop(on part -partition by p_mfgr -order by p_name) -having lead(p_size, 1) over() <= p_size -distribute by p_mfgr -sort by p_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q deleted file mode 100644 index 40a39cb68b5e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q +++ /dev/null @@ -1,21 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING, - p_complex array -); - --- testInvalidValueBoundary -select p_mfgr,p_name, p_size, -sum(p_size) over (w1) as s , -dense_rank() over(w1) as dr -from part -window w1 as (partition by p_mfgr order by p_complex range between 2 preceding and current row); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q deleted file mode 100644 index 80441e4f571f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q +++ /dev/null @@ -1,20 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - --- testJoinWithAmbigousAlias -select abc.* -from noop(on part -partition by p_mfgr -order by p_name -) abc join part on abc.p_partkey = p1.p_partkey; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q deleted file mode 100644 index 1c98b8743cd7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q +++ /dev/null @@ -1,19 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - --- testPartitonBySortBy -select p_mfgr, p_name, p_size, -sum(p_retailprice) over (partition by p_mfgr sort by p_mfgr) as s1 -from part -; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q deleted file mode 100644 index 8f4a21bd6c96..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q +++ /dev/null @@ -1,21 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - --- testWhereWithRankCond -select p_mfgr,p_name, p_size, -rank() over() as r -from part -where r < 4 -distribute by p_mfgr -sort by p_mfgr; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries.q deleted file mode 100644 index ddab4367bb66..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries.q +++ /dev/null @@ -1,17 +0,0 @@ --- data setup -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -select p_mfgr, p_name, p_size, - sum(p_retailprice) over (rows unbounded following) as s1 - from part distribute by p_mfgr sort by p_name; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q deleted file mode 100644 index 16cb52ca8414..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q +++ /dev/null @@ -1,17 +0,0 @@ --- data setup -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -select p_mfgr, p_name, p_size, - sum(p_retailprice) over (range unbounded following) as s1 - from part distribute by p_mfgr sort by p_name; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/recursive_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/recursive_view.q deleted file mode 100644 index 590523e9b625..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/recursive_view.q +++ /dev/null @@ -1,15 +0,0 @@ --- Can't have recursive views - -drop table t; -drop view r0; -drop view r1; -drop view r2; -drop view r3; -create table t (id int); -create view r0 as select * from t; -create view r1 as select * from r0; -create view r2 as select * from r1; -create view r3 as select * from r2; -drop view r0; -alter view r3 rename to r0; -select * from r0; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q deleted file mode 100644 index a171961a683e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.support.quoted.identifiers=none; -EXPLAIN -SELECT `+++` FROM srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q deleted file mode 100644 index 7bac1c775522..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.support.quoted.identifiers=none; -EXPLAIN -SELECT `.a.` FROM srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q deleted file mode 100644 index 300d14550888..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.support.quoted.identifiers=none; -EXPLAIN -SELECT `..`, count(1) FROM srcpart GROUP BY `..`; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sa_fail_hook3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sa_fail_hook3.q deleted file mode 100644 index e54201c09e6f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sa_fail_hook3.q +++ /dev/null @@ -1,4 +0,0 @@ -create table mp2 (a string) partitioned by (b string); -alter table mp2 add partition (b='1'); -alter table mp2 partition (b='1') enable NO_DROP; -alter table mp2 drop partition (b='1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sample.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sample.q deleted file mode 100644 index 0086352f8c47..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sample.q +++ /dev/null @@ -1 +0,0 @@ -explain extended SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 5 OUT OF 4 on key) s \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe2.q deleted file mode 100644 index 1c3093c0e702..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe2.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.exec.script.allow.partial.consumption = false; --- Tests exception in ScriptOperator.processOp() by passing extra data needed to fill pipe buffer -SELECT TRANSFORM(key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value) USING 'true' as a,b,c,d FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe3.q deleted file mode 100644 index 60f93d209802..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe3.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.exec.script.allow.partial.consumption = true; --- Test to ensure that a script with a bad error code still fails even with partial consumption -SELECT TRANSFORM(*) USING 'false' AS a, b FROM (SELECT TRANSFORM(*) USING 'echo' AS a, b FROM src LIMIT 1) tmp; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q deleted file mode 100644 index 8ca849b82d8a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q +++ /dev/null @@ -1,7 +0,0 @@ -EXPLAIN -SELECT TRANSFORM(src.key, src.value) USING '../../data/scripts/error_script' AS (tkey, tvalue) -FROM src; - -SELECT TRANSFORM(src.key, src.value) USING '../../data/scripts/error_script' AS (tkey, tvalue) -FROM src; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_charliteral.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_charliteral.q deleted file mode 100644 index 1e4c70e663f0..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_charliteral.q +++ /dev/null @@ -1,3 +0,0 @@ --- Check that charSetLiteral syntax conformance --- Check that a sane error message with correct line/column numbers is emitted with helpful context tokens. -select _c17, count(1) from tmp_tl_foo group by _c17 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_udtf_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_udtf_alias.q deleted file mode 100644 index 8ace4414fc14..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_udtf_alias.q +++ /dev/null @@ -1,3 +0,0 @@ --- Check alias count for SELECT UDTF() syntax: --- explode returns a table with only 1 col - should be an error if query specifies >1 col aliases -SELECT explode(array(1,2,3)) AS (myCol1, myCol2) LIMIT 3; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin1.q deleted file mode 100644 index 06e6cad34b4d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin1.q +++ /dev/null @@ -1,2 +0,0 @@ --- reference rhs of semijoin in select-clause -select b.value from src a left semi join src b on (b.key = a.key and b.key = '100'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin2.q deleted file mode 100644 index 46faae641640..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin2.q +++ /dev/null @@ -1,2 +0,0 @@ --- rhs table reference in the where clause -select a.value from src a left semi join src b on a.key = b.key where b.value = 'val_18'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin3.q deleted file mode 100644 index 35b455a7292d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin3.q +++ /dev/null @@ -1,2 +0,0 @@ --- rhs table reference in group by -select * from src a left semi join src b on a.key = b.key group by b.value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin4.q deleted file mode 100644 index 4e52ebfb3cde..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin4.q +++ /dev/null @@ -1,3 +0,0 @@ --- rhs table is a view and reference the view in where clause -select a.value from src a left semi join (select key , value from src where key > 100) b on a.key = b.key where b.value = 'val_108' ; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex.q deleted file mode 100644 index 13b3f165b968..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex.q +++ /dev/null @@ -1,17 +0,0 @@ -USE default; --- This should fail because Regex SerDe doesn't support STRUCT -CREATE TABLE serde_regex( - host STRING, - identity STRING, - user STRING, - time TIMESTAMP, - request STRING, - status INT, - size INT, - referer STRING, - agent STRING, - strct STRUCT) -ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' -WITH SERDEPROPERTIES ( - "input.regex" = "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) ([^ \"]*|\"[^\"]*\") (-|[0-9]*) (-|[0-9]*)(?: ([^ \"]*|\"[^\"]*\") ([^ \"]*|\"[^\"]*\"))?") -STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q deleted file mode 100644 index d523d03e906c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q +++ /dev/null @@ -1,23 +0,0 @@ -USE default; --- Mismatch between the number of matching groups and columns, throw run time exception. Ideally this should throw a compile time exception. See JIRA-3023 for more details. - CREATE TABLE serde_regex( - host STRING, - identity STRING, - user STRING, - time STRING, - request STRING, - status STRING, - size STRING, - referer STRING, - agent STRING) -ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' -WITH SERDEPROPERTIES ( - "input.regex" = "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) ([^ \"]*|\"[^\"]*\") (-|[0-9]*) (-|[0-9]*)" -) -STORED AS TEXTFILE; - -LOAD DATA LOCAL INPATH "../../data/files/apache.access.log" INTO TABLE serde_regex; -LOAD DATA LOCAL INPATH "../../data/files/apache.access.2.log" INTO TABLE serde_regex; - --- raise an exception -SELECT * FROM serde_regex ORDER BY time; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex3.q deleted file mode 100644 index 5a0295c971c2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex3.q +++ /dev/null @@ -1,14 +0,0 @@ -USE default; --- null input.regex, raise an exception - CREATE TABLE serde_regex( - host STRING, - identity STRING, - user STRING, - time STRING, - request STRING, - status STRING, - size STRING, - referer STRING, - agent STRING) -ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' -STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q deleted file mode 100644 index 4cb48664b602..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q +++ /dev/null @@ -1,5 +0,0 @@ --- should fail: hive.join.cache.size accepts int type -desc src; - -set hive.conf.validation=true; -set hive.join.cache.size=test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q deleted file mode 100644 index 330aafd19858..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q +++ /dev/null @@ -1,5 +0,0 @@ --- should fail: hive.map.aggr.hash.min.reduction accepts float type -desc src; - -set hive.conf.validation=true; -set hive.map.aggr.hash.min.reduction=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q deleted file mode 100644 index 579e9408b6c3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q +++ /dev/null @@ -1,5 +0,0 @@ --- should fail: hive.fetch.task.conversion accepts minimal or more -desc src; - -set hive.conf.validation=true; -set hive.fetch.task.conversion=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_table_property.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_table_property.q deleted file mode 100644 index d582aaeb386c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_table_property.q +++ /dev/null @@ -1,4 +0,0 @@ -create table testTable(col1 int, col2 int); - --- set a table property = null, it should be caught by the grammar -alter table testTable set tblproperties ('a'=); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns1.q deleted file mode 100644 index 25705dc3d527..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns1.q +++ /dev/null @@ -1,2 +0,0 @@ -SHOW COLUMNS from shcol_test; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns2.q deleted file mode 100644 index c55b449a0b5f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns2.q +++ /dev/null @@ -1,2 +0,0 @@ -SHOW COLUMNS from shcol_test foo; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns3.q deleted file mode 100644 index 508a786609d8..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns3.q +++ /dev/null @@ -1,7 +0,0 @@ -CREATE DATABASE test_db; -USE test_db; -CREATE TABLE foo(a INT); - -use default; -SHOW COLUMNS from test_db.foo from test_db; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q deleted file mode 100644 index 83e5093aa1f2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q +++ /dev/null @@ -1,2 +0,0 @@ -SHOW CREATE TABLE tmp_nonexist; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_index.q deleted file mode 100644 index 0dd0ef9a255b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_index.q +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TABLE tmp_showcrt (key int, value string); -CREATE INDEX tmp_index on table tmp_showcrt(key) as 'compact' WITH DEFERRED REBUILD; -SHOW CREATE TABLE default__tmp_showcrt_tmp_index__; -DROP INDEX tmp_index on tmp_showcrt; -DROP TABLE tmp_showcrt; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_partitions1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_partitions1.q deleted file mode 100644 index 71f68c894f2a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_partitions1.q +++ /dev/null @@ -1 +0,0 @@ -SHOW PARTITIONS NonExistentTable; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tableproperties1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tableproperties1.q deleted file mode 100644 index 254a1d3a5ac3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tableproperties1.q +++ /dev/null @@ -1 +0,0 @@ -SHOW TBLPROPERTIES NonExistentTable; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad1.q deleted file mode 100644 index 1bc94d6392c6..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad1.q +++ /dev/null @@ -1 +0,0 @@ -SHOW TABLES JOIN; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad2.q deleted file mode 100644 index 5e828b647ac3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad2.q +++ /dev/null @@ -1 +0,0 @@ -SHOW TABLES FROM default LIKE a b; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db1.q deleted file mode 100644 index d0141f6c291c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db1.q +++ /dev/null @@ -1 +0,0 @@ -SHOW TABLES FROM nonexistent; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db2.q deleted file mode 100644 index ee0deba87a94..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db2.q +++ /dev/null @@ -1 +0,0 @@ -SHOW TABLES FROM nonexistent LIKE 'test'; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus.q deleted file mode 100644 index 283b5836e27f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus.q +++ /dev/null @@ -1 +0,0 @@ -SHOW TABLE EXTENDED LIKE `srcpar*` PARTITION(ds='2008-04-08', hr=11); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q deleted file mode 100644 index 242e16528554..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q +++ /dev/null @@ -1 +0,0 @@ -SHOW TABLE EXTENDED LIKE `srcpart` PARTITION(ds='2008-14-08', hr=11); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q deleted file mode 100644 index 880323c604b6..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q +++ /dev/null @@ -1,23 +0,0 @@ -set hive.enforce.bucketing = true; -set hive.enforce.sorting = true; -set hive.exec.reducers.max = 1; - - -CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS; - - -CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS; - -insert overwrite table smb_bucket4_1 -select * from src; - -insert overwrite table smb_bucket4_2 -select * from src; - -set hive.optimize.bucketmapjoin = true; -set hive.optimize.bucketmapjoin.sortedmerge = true; - -select /*+mapjoin(a)*/ * from smb_bucket4_1 a left outer join smb_bucket4_2 b on a.key = b.key; - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_mapjoin_14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_mapjoin_14.q deleted file mode 100644 index 54bfba03d82d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_mapjoin_14.q +++ /dev/null @@ -1,38 +0,0 @@ -set hive.enforce.bucketing = true; -set hive.enforce.sorting = true; -set hive.exec.reducers.max = 1; - -CREATE TABLE tbl1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; -CREATE TABLE tbl2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; - -insert overwrite table tbl1 -select * from src where key < 10; - -insert overwrite table tbl2 -select * from src where key < 10; - -set hive.optimize.bucketmapjoin = true; -set hive.optimize.bucketmapjoin.sortedmerge = true; -set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; - --- A join is being performed across different sub-queries, where a mapjoin is being performed in each of them. --- Each sub-query should be converted to a sort-merge join. --- A join followed by mapjoin is not allowed, so this query should fail. --- Once HIVE-3403 is in, this should be automatically converted to a sort-merge join without the hint -explain -select src1.key, src1.cnt1, src2.cnt1 from -( - select key, count(*) as cnt1 from - ( - select /*+mapjoin(a)*/ a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key - ) subq1 group by key -) src1 -join -( - select key, count(*) as cnt1 from - ( - select /*+mapjoin(a)*/ a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key - ) subq2 group by key -) src2 -on src1.key = src2.key -order by src1.key, src1.cnt1, src2.cnt1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q deleted file mode 100644 index 7d11f450edfd..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q +++ /dev/null @@ -1,28 +0,0 @@ -create table table_asc(key int, value string) CLUSTERED BY (key) SORTED BY (key asc) -INTO 1 BUCKETS STORED AS RCFILE; -create table table_desc(key int, value string) CLUSTERED BY (key) SORTED BY (key desc) -INTO 1 BUCKETS STORED AS RCFILE; - -set hive.enforce.bucketing = true; -set hive.enforce.sorting = true; - -insert overwrite table table_asc select key, value from src; -insert overwrite table table_desc select key, value from src; -set hive.optimize.bucketmapjoin = true; -set hive.optimize.bucketmapjoin.sortedmerge = true; -set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; - --- If the user asked for sort merge join to be enforced (by setting --- hive.enforce.sortmergebucketmapjoin to true), an error should be thrown, since --- one of the tables is in ascending order and the other is in descending order, --- and sort merge bucket mapjoin cannot be performed. In the default mode, the --- query would succeed, although a regular map-join would be performed instead of --- what the user asked. - -explain -select /*+mapjoin(a)*/ * from table_asc a join table_desc b on a.key = b.key; - -set hive.enforce.sortmergebucketmapjoin=true; - -explain -select /*+mapjoin(a)*/ * from table_asc a join table_desc b on a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_out_of_range.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_out_of_range.q deleted file mode 100644 index 66af1fd7da68..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_out_of_range.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; - -select key from src tablesample(105 percent); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format.q deleted file mode 100644 index f71cc4487910..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; - -select key from src tablesample(1 percent); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q deleted file mode 100644 index 1a13c0ff4cb2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; - -select key from src tablesample(1K); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q deleted file mode 100644 index 1b2872d3d7ed..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q +++ /dev/null @@ -1,18 +0,0 @@ --- In this test, there is a dummy stats aggregator which throws an error when the --- method connect is called (as indicated by the parameter hive.test.dummystats.aggregator) --- If stats need not be reliable, the statement succeeds. However, if stats are supposed --- to be reliable (by setting hive.stats.reliable to true), the insert statement fails --- because stats cannot be collected for this statement - -create table tmptable(key string, value string); - -set hive.stats.dbclass=custom; -set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; -set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator; -set hive.test.dummystats.aggregator=connect; - -set hive.stats.reliable=false; -INSERT OVERWRITE TABLE tmptable select * from src; - -set hive.stats.reliable=true; -INSERT OVERWRITE TABLE tmptable select * from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q deleted file mode 100644 index 0fa9ff682037..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q +++ /dev/null @@ -1,16 +0,0 @@ --- In this test, the stats aggregator does not exists. --- If stats need not be reliable, the statement succeeds. However, if stats are supposed --- to be reliable (by setting hive.stats.reliable to true), the insert statement fails --- because stats cannot be collected for this statement - -create table tmptable(key string, value string); - -set hive.stats.dbclass=custom; -set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; -set hive.stats.default.aggregator=""; - -set hive.stats.reliable=false; -INSERT OVERWRITE TABLE tmptable select * from src; - -set hive.stats.reliable=true; -INSERT OVERWRITE TABLE tmptable select * from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_noscan_non_native.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_noscan_non_native.q deleted file mode 100644 index bde66278360c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_noscan_non_native.q +++ /dev/null @@ -1,6 +0,0 @@ - -CREATE TABLE non_native1(key int, value string) -STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; - --- we do not support analyze table ... noscan on non-native tables yet -analyze table non_native1 compute statistics noscan; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q deleted file mode 100644 index 47a8148e0869..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q +++ /dev/null @@ -1,31 +0,0 @@ -set datanucleus.cache.collections=false; -set hive.stats.autogather=false; -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nonstrict; -set mapred.min.split.size=256; -set mapred.min.split.size.per.node=256; -set mapred.min.split.size.per.rack=256; -set mapred.max.split.size=256; - --- test analyze table ... compute statistics partialscan - --- 1. prepare data -CREATE table analyze_srcpart_partial_scan (key STRING, value STRING) -partitioned by (ds string, hr string) -stored as rcfile; -insert overwrite table analyze_srcpart_partial_scan partition (ds, hr) select * from srcpart where ds is not null; -describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11); - - --- 2. partialscan -explain -analyze table analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11) compute statistics partialscan; -analyze table analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11) compute statistics partialscan; - --- 3. confirm result -describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11); -describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-09',hr=11); -drop table analyze_srcpart_partial_scan; - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q deleted file mode 100644 index c206b8b5d765..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q +++ /dev/null @@ -1,5 +0,0 @@ - -CREATE EXTERNAL TABLE external_table (key int, value string); - --- we do not support analyze table ... partialscan on EXTERNAL tables yet -analyze table external_table compute statistics partialscan; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q deleted file mode 100644 index 8e02ced85e70..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q +++ /dev/null @@ -1,6 +0,0 @@ - -CREATE TABLE non_native1(key int, value string) -STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; - --- we do not support analyze table ... partialscan on non-native tables yet -analyze table non_native1 compute statistics partialscan; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q deleted file mode 100644 index 56d93d08aa69..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q +++ /dev/null @@ -1,12 +0,0 @@ -set datanucleus.cache.collections=false; -set hive.stats.autogather=true; -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nonstrict; - --- test analyze table ... compute statistics partialscan - -create table analyze_srcpart_partial_scan like srcpart; -insert overwrite table analyze_srcpart_partial_scan partition (ds, hr) select * from srcpart where ds is not null; -analyze table analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11) compute statistics partialscan; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q deleted file mode 100644 index be7c4f72feb9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q +++ /dev/null @@ -1,18 +0,0 @@ --- In this test, there is a dummy stats publisher which throws an error when the --- method connect is called (as indicated by the parameter hive.test.dummystats.publisher) --- If stats need not be reliable, the statement succeeds. However, if stats are supposed --- to be reliable (by setting hive.stats.reliable to true), the insert statement fails --- because stats cannot be collected for this statement - -create table tmptable(key string, value string); - -set hive.stats.dbclass=custom; -set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; -set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator; -set hive.test.dummystats.publisher=connect; - -set hive.stats.reliable=false; -INSERT OVERWRITE TABLE tmptable select * from src; - -set hive.stats.reliable=true; -INSERT OVERWRITE TABLE tmptable select * from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q deleted file mode 100644 index 652afe7c5bfb..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q +++ /dev/null @@ -1,16 +0,0 @@ --- In this test, the stats publisher does not exists. --- If stats need not be reliable, the statement succeeds. However, if stats are supposed --- to be reliable (by setting hive.stats.reliable to true), the insert statement fails --- because stats cannot be collected for this statement - -create table tmptable(key string, value string); - -set hive.stats.dbclass=custom; -set hive.stats.default.publisher=""; -set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator; - -set hive.stats.reliable=false; -INSERT OVERWRITE TABLE tmptable select * from src; - -set hive.stats.reliable=true; -INSERT OVERWRITE TABLE tmptable select * from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_join.q deleted file mode 100644 index d618ee28fdb2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_join.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.mapred.mode=strict; - -SELECT * FROM src src1 JOIN src src2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_orderby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_orderby.q deleted file mode 100644 index 781cdbb05088..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_orderby.q +++ /dev/null @@ -1,7 +0,0 @@ -set hive.mapred.mode=strict; - -EXPLAIN -SELECT src.key, src.value from src order by src.key; - -SELECT src.key, src.value from src order by src.key; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_pruning.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_pruning.q deleted file mode 100644 index 270ab2f593ac..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_pruning.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.mapred.mode=strict; - -EXPLAIN -SELECT count(1) FROM srcPART; - -SELECT count(1) FROM srcPART; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subq_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subq_insert.q deleted file mode 100644 index 0bc9e24e4828..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subq_insert.q +++ /dev/null @@ -1,2 +0,0 @@ -EXPLAIN -SELECT * FROM (INSERT OVERWRITE TABLE src1 SELECT * FROM src ) y; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q deleted file mode 100644 index 9013df6f938d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q +++ /dev/null @@ -1,10 +0,0 @@ - - -select * -from src b -where exists - (select count(*) - from src a - where b.value = a.value and a.key = b.key and a.value > 'val_9' - ) -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q deleted file mode 100644 index a9bc6ee6a38c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q +++ /dev/null @@ -1,5 +0,0 @@ - - -select count(*) -from src -group by src.key in (select key from src s1 where s1.key > '9') \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q deleted file mode 100644 index 1365389cb269..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q +++ /dev/null @@ -1,6 +0,0 @@ - - - -select src.key in (select key from src s1 where s1.key > '9') -from src -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q deleted file mode 100644 index 6805c5b16b0f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q +++ /dev/null @@ -1,7 +0,0 @@ - - -explain - select * -from src -where src.key in (select * from src s1 where s1.key > '9') -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q deleted file mode 100644 index e8c41e6b17ae..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q +++ /dev/null @@ -1,18 +0,0 @@ - - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -select * -from part x -where x.p_name in (select y.p_name from part y where exists (select z.p_name from part z where y.p_name = z.p_name)) -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q deleted file mode 100644 index 852b2953ff46..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q +++ /dev/null @@ -1,10 +0,0 @@ - - -select * -from src b -where not exists - (select sum(1) - from src a - where b.value = a.value and a.key = b.key and a.value > 'val_9' - ) -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q deleted file mode 100644 index d442f077c070..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q +++ /dev/null @@ -1,6 +0,0 @@ - - -select * -from src -where src.key in (select key from src where key > '9') -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q deleted file mode 100644 index 8ea94c5fc6d7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q +++ /dev/null @@ -1,6 +0,0 @@ - -explain -select * -from src -where src.key in (select key from src) in (select key from src) -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q deleted file mode 100644 index 99ff9ca70383..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q +++ /dev/null @@ -1,6 +0,0 @@ - - -select * -from src -where key in (select key from src) -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q deleted file mode 100644 index 105d3d22d9d2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q +++ /dev/null @@ -1,26 +0,0 @@ -DROP TABLE part; - --- data setup -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; - - --- corr and windowing -select p_mfgr, p_name, p_size -from part a -where a.p_size in - (select first_value(p_size) over(partition by p_mfgr order by p_size) - from part b - where a.p_brand = b.p_brand) -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q deleted file mode 100644 index c2c322178f38..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q +++ /dev/null @@ -1,5 +0,0 @@ - -select count(*) -from src -where src.key in (select key from src s1 where s1.key > '9') or src.value is not null -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch1.q deleted file mode 100644 index 9efbba0082b6..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch1.q +++ /dev/null @@ -1 +0,0 @@ -ALTER TABLE srcpart TOUCH PARTITION (ds='2008-04-08', hr='13'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch2.q deleted file mode 100644 index 923a171e0482..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch2.q +++ /dev/null @@ -1 +0,0 @@ -ALTER TABLE src TOUCH PARTITION (ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_bucketed_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_bucketed_column.q deleted file mode 100644 index e53665695a39..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_bucketed_column.q +++ /dev/null @@ -1,7 +0,0 @@ --- Tests truncating a bucketed column - -CREATE TABLE test_tab (key STRING, value STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS RCFILE; - -INSERT OVERWRITE TABLE test_tab SELECT * FROM src; - -TRUNCATE TABLE test_tab COLUMNS (key); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q deleted file mode 100644 index 13f32c8968a1..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q +++ /dev/null @@ -1,9 +0,0 @@ --- Tests truncating a column from an indexed table - -CREATE TABLE test_tab (key STRING, value STRING) STORED AS RCFILE; - -INSERT OVERWRITE TABLE test_tab SELECT * FROM src; - -CREATE INDEX test_tab_index ON TABLE test_tab (key) as 'COMPACT' WITH DEFERRED REBUILD; - -TRUNCATE TABLE test_tab COLUMNS (value); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q deleted file mode 100644 index 0ece6007f7b6..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.mapred.supports.subdirectories=true; -set mapred.input.dir.recursive=true; - --- Tests truncating a column on which a table is list bucketed - -CREATE TABLE test_tab (key STRING, value STRING) STORED AS RCFILE; - -ALTER TABLE test_tab -SKEWED BY (key) ON ("484") -STORED AS DIRECTORIES; - -INSERT OVERWRITE TABLE test_tab SELECT * FROM src; - -TRUNCATE TABLE test_tab COLUMNS (key); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_seqfile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_seqfile.q deleted file mode 100644 index 903540dae898..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_seqfile.q +++ /dev/null @@ -1,7 +0,0 @@ --- Tests truncating a column from a table stored as a sequence file - -CREATE TABLE test_tab (key STRING, value STRING) STORED AS SEQUENCEFILE; - -INSERT OVERWRITE TABLE test_tab SELECT * FROM src; - -TRUNCATE TABLE test_tab COLUMNS (key); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q deleted file mode 100644 index 5509552811b0..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q +++ /dev/null @@ -1,7 +0,0 @@ --- Tests attempting to truncate a column in a table that doesn't exist - -CREATE TABLE test_tab (key STRING, value STRING) STORED AS RCFILE; - -INSERT OVERWRITE TABLE test_tab SELECT * FROM src; - -TRUNCATE TABLE test_tab COLUMNS (doesnt_exist); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column.q deleted file mode 100644 index 134743ac13a5..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column.q +++ /dev/null @@ -1,7 +0,0 @@ --- Tests truncating a partition column - -CREATE TABLE test_tab (key STRING, value STRING) PARTITIONED BY (part STRING) STORED AS RCFILE; - -INSERT OVERWRITE TABLE test_tab PARTITION (part = '1') SELECT * FROM src; - -TRUNCATE TABLE test_tab COLUMNS (part); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column2.q deleted file mode 100644 index 47635208a781..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column2.q +++ /dev/null @@ -1,7 +0,0 @@ --- Tests truncating a partition column - -CREATE TABLE test_tab (key STRING, value STRING) PARTITIONED BY (part STRING) STORED AS RCFILE; - -INSERT OVERWRITE TABLE test_tab PARTITION (part = '1') SELECT * FROM src; - -TRUNCATE TABLE test_tab PARTITION (part = '1') COLUMNS (part); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure1.q deleted file mode 100644 index f6cfa44bbb12..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure1.q +++ /dev/null @@ -1,2 +0,0 @@ --- partition spec for non-partitioned table -TRUNCATE TABLE src partition (ds='2008-04-08', hr='11'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure2.q deleted file mode 100644 index 1137d893eb0e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure2.q +++ /dev/null @@ -1,2 +0,0 @@ --- full partition spec for not existing partition -TRUNCATE TABLE srcpart partition (ds='2012-12-17', hr='15'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure3.q deleted file mode 100644 index c5cf58775b30..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure3.q +++ /dev/null @@ -1,4 +0,0 @@ -create external table external1 (a int, b int) partitioned by (ds string); - --- trucate for non-managed table -TRUNCATE TABLE external1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure4.q deleted file mode 100644 index a7f1e92d5598..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure4.q +++ /dev/null @@ -1,5 +0,0 @@ -CREATE TABLE non_native(key int, value string) -STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; - --- trucate for non-native table -TRUNCATE TABLE non_native; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udaf_invalid_place.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udaf_invalid_place.q deleted file mode 100644 index f37ce72ae419..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udaf_invalid_place.q +++ /dev/null @@ -1 +0,0 @@ -select distinct key, sum(key) from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q deleted file mode 100644 index c2a132d4db05..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid first argument -SELECT array_contains(1, 2) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q deleted file mode 100644 index 36f85d34a6e0..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid second argument -SELECT array_contains(array(1, 2, 3), '2') FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true.q deleted file mode 100644 index 73b3f9654f1c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true.q +++ /dev/null @@ -1,7 +0,0 @@ -DESCRIBE FUNCTION ASSERT_TRUE; - -EXPLAIN SELECT ASSERT_TRUE(x > 0) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; -SELECT ASSERT_TRUE(x > 0) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; - -EXPLAIN SELECT ASSERT_TRUE(x < 2) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; -SELECT ASSERT_TRUE(x < 2) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true2.q deleted file mode 100644 index 4b62220764bb..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true2.q +++ /dev/null @@ -1,2 +0,0 @@ -EXPLAIN SELECT 1 + ASSERT_TRUE(x < 2) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; -SELECT 1 + ASSERT_TRUE(x < 2) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_coalesce.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_coalesce.q deleted file mode 100644 index 7405e387caf7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_coalesce.q +++ /dev/null @@ -1 +0,0 @@ -SELECT COALESCE(array('a', 'b'), '2.0') FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q deleted file mode 100644 index 8c2017bc636c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument number -SELECT concat_ws('-') FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q deleted file mode 100644 index c49e7868bbb5..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT concat_ws('[]', array(100, 200, 50)) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q deleted file mode 100644 index 72b86271f5ea..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT concat_ws(1234, array('www', 'facebook', 'com')) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q deleted file mode 100644 index fbe4902d644c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q +++ /dev/null @@ -1 +0,0 @@ -SELECT elt(3) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q deleted file mode 100644 index bb1fdbf789e3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM src_thrift -SELECT elt(1, src_thrift.lintstring) -WHERE src_thrift.lintstring IS NOT NULL; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q deleted file mode 100644 index 9703c82d8a4d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q +++ /dev/null @@ -1 +0,0 @@ -SELECT field(3) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_type.q deleted file mode 100644 index 61b2cd06496e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_type.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM src_thrift -SELECT field(1, src_thrift.lintstring) -WHERE src_thrift.lintstring IS NOT NULL; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q deleted file mode 100644 index 18c985c60684..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument length -SELECT format_number(12332.123456) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q deleted file mode 100644 index 7959c20b28e5..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument length -SELECT format_number(12332.123456, 2, 3) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q deleted file mode 100644 index 7d90ef86da7b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument(second argument should be >= 0) -SELECT format_number(12332.123456, -4) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q deleted file mode 100644 index e545f4aa1420..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT format_number(12332.123456, 4.01) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q deleted file mode 100644 index a6f71778f143..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT format_number(array(12332.123456, 321.23), 5) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q deleted file mode 100644 index e5b11b9b71ee..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT format_number(12332.123456, "4") FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q deleted file mode 100644 index aa4a3a44751c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type(format_number returns the result as a string) -SELECT format_number(format_number(12332.123456, 4), 2) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q deleted file mode 100644 index 21ca6e7d3625..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q +++ /dev/null @@ -1 +0,0 @@ -CREATE TEMPORARY FUNCTION moo AS 'org.apache.hadoop.hive.ql.Driver'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_not_bool.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_not_bool.q deleted file mode 100644 index 74458d0c3db2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_not_bool.q +++ /dev/null @@ -1 +0,0 @@ -SELECT IF('STRING', 1, 1) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q deleted file mode 100644 index ad19364c3307..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q +++ /dev/null @@ -1 +0,0 @@ -SELECT IF(TRUE, 1) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_in.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_in.q deleted file mode 100644 index ce9ce54fac68..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_in.q +++ /dev/null @@ -1 +0,0 @@ -SELECT 3 IN (array(1,2,3)) FROM src; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q deleted file mode 100644 index ac8253fb1e94..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q +++ /dev/null @@ -1 +0,0 @@ -SELECT instr('abcd') FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q deleted file mode 100644 index 9ac3ed661489..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM src_thrift -SELECT instr('abcd', src_thrift.lintstring) -WHERE src_thrift.lintstring IS NOT NULL; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q deleted file mode 100644 index 68050fd95cd2..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q +++ /dev/null @@ -1 +0,0 @@ -select default.nonexistfunc() from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q deleted file mode 100644 index bcfa217737e3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q +++ /dev/null @@ -1 +0,0 @@ -create function lookup as 'org.apache.hadoop.hive.ql.udf.UDFFileLookup' using file '../../data/files/sales.txt'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q deleted file mode 100644 index ca7caad54d64..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q +++ /dev/null @@ -1 +0,0 @@ -SELECT locate('a', 'b', 1, 2) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q deleted file mode 100644 index 4bbf79a310b0..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM src_thrift -SELECT locate('abcd', src_thrift.lintstring) -WHERE src_thrift.lintstring IS NOT NULL; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q deleted file mode 100644 index ebb6c2ab418e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q +++ /dev/null @@ -1 +0,0 @@ -SELECT map_keys(map("a", "1"), map("b", "2")) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q deleted file mode 100644 index 0757d1494f3c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q +++ /dev/null @@ -1 +0,0 @@ -SELECT map_keys(array(1, 2, 3)) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q deleted file mode 100644 index c97476a1263e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q +++ /dev/null @@ -1 +0,0 @@ -SELECT map_values(map("a", "1"), map("b", "2")) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q deleted file mode 100644 index cc060ea0f0ec..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q +++ /dev/null @@ -1 +0,0 @@ -SELECT map_values(array(1, 2, 3, 4)) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_max.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_max.q deleted file mode 100644 index 7282e0759603..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_max.q +++ /dev/null @@ -1,2 +0,0 @@ -SELECT max(map("key", key, "value", value)) -FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_min.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_min.q deleted file mode 100644 index b9528fa6dafe..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_min.q +++ /dev/null @@ -1,2 +0,0 @@ -SELECT min(map("key", key, "value", value)) -FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q deleted file mode 100644 index d37665dde69b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q +++ /dev/null @@ -1 +0,0 @@ -create function lookup as 'org.apache.hadoop.hive.ql.udf.UDFFileLookup' using file 'nonexistent_file.txt'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong1.q deleted file mode 100644 index 88ca4fefc305..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong1.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument length -SELECT printf() FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong2.q deleted file mode 100644 index 01ed2ffcf017..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong2.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT printf(100) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong3.q deleted file mode 100644 index 71f118b8dc0d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong3.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT printf("Hello World %s", array("invalid", "argument")) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong4.q deleted file mode 100644 index 71f118b8dc0d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong4.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT printf("Hello World %s", array("invalid", "argument")) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q deleted file mode 100644 index 476dfa21a237..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q +++ /dev/null @@ -1 +0,0 @@ -create temporary function default.myfunc as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_reflect_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_reflect_neg.q deleted file mode 100644 index 67efb64505d9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_reflect_neg.q +++ /dev/null @@ -1,9 +0,0 @@ -SELECT reflect("java.lang.StringClassThatDoesNotExist", "valueOf", 1), - reflect("java.lang.String", "methodThatDoesNotExist"), - reflect("java.lang.Math", "max", "overloadthatdoesnotexist", 3), - reflect("java.lang.Math", "min", 2, 3), - reflect("java.lang.Math", "round", 2.5), - reflect("java.lang.Math", "exp", 1.0), - reflect("java.lang.Math", "floor", 1.9) -FROM src LIMIT 1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q deleted file mode 100644 index c628ff8aa197..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q +++ /dev/null @@ -1,5 +0,0 @@ -FROM src_thrift -SELECT size(src_thrift.lint, src_thrift.lintstring), - size() -WHERE src_thrift.lint IS NOT NULL - AND NOT (src_thrift.mstringstring IS NULL) LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_type.q deleted file mode 100644 index 16695f6adc3f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_type.q +++ /dev/null @@ -1 +0,0 @@ -SELECT SIZE('wrong type: string') FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q deleted file mode 100644 index 9954f4ab4d3c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument number -SELECT sort_array(array(2, 5, 4), 3) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q deleted file mode 100644 index 32c264551949..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT sort_array("Invalid") FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q deleted file mode 100644 index 034de06b8e39..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT sort_array(array(array(10, 20), array(5, 15), array(3, 13))) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error.q deleted file mode 100644 index 846f87c2e51b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error.q +++ /dev/null @@ -1,3 +0,0 @@ -CREATE TEMPORARY FUNCTION test_error AS 'org.apache.hadoop.hive.ql.udf.UDFTestErrorOnFalse'; - -SELECT test_error(key < 125 OR key > 130) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error_reduce.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error_reduce.q deleted file mode 100644 index b1a06f2a07af..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error_reduce.q +++ /dev/null @@ -1,11 +0,0 @@ -CREATE TEMPORARY FUNCTION test_error AS 'org.apache.hadoop.hive.ql.udf.UDFTestErrorOnFalse'; - - -SELECT test_error(key < 125 OR key > 130) -FROM ( - SELECT * - FROM src - DISTRIBUTE BY rand() -) map_output; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong.q deleted file mode 100644 index d4d2d2e48517..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong.q +++ /dev/null @@ -1,6 +0,0 @@ -SELECT CASE - WHEN TRUE THEN 2 - WHEN '1' THEN 4 - ELSE 5 - END -FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q deleted file mode 100644 index 942ae5d8315f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q +++ /dev/null @@ -1 +0,0 @@ -SELECT explode(map(1,'one',2,'two',3,'three')) as (myKey,myVal) FROM src GROUP BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q deleted file mode 100644 index 00d359a75ce0..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q +++ /dev/null @@ -1 +0,0 @@ -SELECT explode(map(1,'one',2,'two',3,'three')) as (myKey,myVal,myVal2) FROM src; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q deleted file mode 100644 index 51df8fa862e1..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q +++ /dev/null @@ -1 +0,0 @@ -select explode(array(1),array(2)) as myCol from src; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q deleted file mode 100644 index ae8dff7bad8d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q +++ /dev/null @@ -1 +0,0 @@ -SELECT explode(null) as myNull FROM src GROUP BY key; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_invalid_place.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_invalid_place.q deleted file mode 100644 index ab84a801e9ed..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_invalid_place.q +++ /dev/null @@ -1 +0,0 @@ -select distinct key, explode(key) from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported1.q deleted file mode 100644 index 04e98d52c548..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported1.q +++ /dev/null @@ -1 +0,0 @@ -SELECT explode(array(1,2,3)) as myCol, key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported3.q deleted file mode 100644 index f4fe0dde3e62..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported3.q +++ /dev/null @@ -1 +0,0 @@ -SELECT explode(array(1,2,3)) as myCol FROM src GROUP BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union2.q deleted file mode 100644 index 38db488eaf68..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union2.q +++ /dev/null @@ -1,13 +0,0 @@ - - -create table if not exists union2_t1(r string, c string, v array); -create table if not exists union2_t2(s string, c string, v string); - -explain -SELECT s.r, s.c, sum(s.v) -FROM ( - SELECT a.r AS r, a.c AS c, a.v AS v FROM union2_t1 a - UNION ALL - SELECT b.s AS r, b.c AS c, 0 + b.v AS v FROM union2_t2 b -) s -GROUP BY s.r, s.c; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union22.q deleted file mode 100644 index 72f3314bdac9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union22.q +++ /dev/null @@ -1,26 +0,0 @@ -create table dst_union22(k1 string, k2 string, k3 string, k4 string) partitioned by (ds string); -create table dst_union22_delta(k0 string, k1 string, k2 string, k3 string, k4 string, k5 string) partitioned by (ds string); - -insert overwrite table dst_union22 partition (ds='1') -select key, value, key , value from src; - -insert overwrite table dst_union22_delta partition (ds='1') -select key, key, value, key, value, value from src; - -set hive.merge.mapfiles=false; - --- Union followed by Mapjoin is not supported. --- The same query would work without the hint --- Note that there is a positive test with the same name in clientpositive -explain extended -insert overwrite table dst_union22 partition (ds='2') -select * from -( -select k1 as k1, k2 as k2, k3 as k3, k4 as k4 from dst_union22_delta where ds = '1' and k0 <= 50 -union all -select /*+ MAPJOIN(b) */ a.k1 as k1, a.k2 as k2, b.k3 as k3, b.k4 as k4 -from dst_union22 a left outer join (select * from dst_union22_delta where ds = '1' and k0 > 50) b on -a.k1 = b.k1 and a.ds='1' -where a.k1 > 20 -) -subq; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union3.q deleted file mode 100644 index ce657478c150..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union3.q +++ /dev/null @@ -1,5 +0,0 @@ --- Ensure that UNION ALL columns are in the correct order on both sides --- Ensure that the appropriate error message is propagated -CREATE TABLE IF NOT EXISTS union3 (bar int, baz int); -SELECT * FROM ( SELECT f.bar, f.baz FROM union3 f UNION ALL SELECT b.baz, b.bar FROM union3 b ) c; -DROP TABLE union3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin.q deleted file mode 100644 index d6a19c397d80..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM UNIQUEJOIN (SELECT src.key from src WHERE src.key<4) a (a.key), PRESERVE src b(b.key) -SELECT a.key, b.key; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin2.q deleted file mode 100644 index 6e9a08251407..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin2.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM UNIQUEJOIN src a (a.key), PRESERVE src b (b.key, b.val) -SELECT a.key, b.key; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin3.q deleted file mode 100644 index 89a8f1b2aaa8..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin3.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM UNIQUEJOIN src a (a.key), PRESERVE src b (b.key) JOIN src c ON c.key -SELECT a.key; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_table_property.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_table_property.q deleted file mode 100644 index 7a24e652b46f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_table_property.q +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TABLE testTable(col1 INT, col2 INT); -ALTER TABLE testTable SET TBLPROPERTIES ('a'='1', 'c'='3'); -SHOW TBLPROPERTIES testTable; - --- unset a subset of the properties and some non-existed properties without if exists -ALTER TABLE testTable UNSET TBLPROPERTIES ('c', 'x', 'y', 'z'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_view_property.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_view_property.q deleted file mode 100644 index 11131006e998..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_view_property.q +++ /dev/null @@ -1,6 +0,0 @@ -CREATE VIEW testView AS SELECT value FROM src WHERE key=86; -ALTER VIEW testView SET TBLPROPERTIES ('propA'='100', 'propB'='200'); -SHOW TBLPROPERTIES testView; - --- unset a subset of the properties and some non-existed properties without if exists -ALTER VIEW testView UNSET TBLPROPERTIES ('propB', 'propX', 'propY', 'propZ'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q deleted file mode 100644 index c5b593e4bb55..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q +++ /dev/null @@ -1 +0,0 @@ -select nonexistfunc(key) over () from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q deleted file mode 100644 index b54b7a532176..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q +++ /dev/null @@ -1,15 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -select sum(lead(p_retailprice,1)) as s1 from part; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q deleted file mode 100644 index 15f8fae292bb..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q +++ /dev/null @@ -1,26 +0,0 @@ -DROP TABLE IF EXISTS part; - --- data setup -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; - - -select p_mfgr, p_name, p_size, -min(p_retailprice), -rank() over(distribute by p_mfgr sort by p_name)as r, -dense_rank() over(distribute by p_mfgr sort by p_name) as dr, -p_size, p_size - lag(p_size,-1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz -from part -group by p_mfgr, p_name, p_size -; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_over.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_over.q deleted file mode 100644 index 3ca1104b0158..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_over.q +++ /dev/null @@ -1,17 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -select p_mfgr, -lead(p_retailprice,1) as s1 -from part; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/wrong_column_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/wrong_column_type.q deleted file mode 100644 index 490f0c3b4d11..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/wrong_column_type.q +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE dest1(a float); - -INSERT OVERWRITE TABLE dest1 -SELECT array(1.0,2.0) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q index 235b7c1b3fcd..6a9a20f3207b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q @@ -5,7 +5,7 @@ set hive.auto.convert.join = true; CREATE TABLE dest1(c1 INT, c2 STRING) STORED AS TEXTFILE; -set mapred.job.tracker=localhost:58; +set mapreduce.jobtracker.address=localhost:58; set hive.exec.mode.local.auto=true; explain diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket5.q index 877f8a50a0e3..87f6eca4dd4e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket5.q @@ -4,7 +4,7 @@ set hive.enforce.sorting = true; set hive.exec.reducers.max = 1; set hive.merge.mapfiles = true; set hive.merge.mapredfiles = true; -set mapred.reduce.tasks = 2; +set mapreduce.job.reduces = 2; -- Tests that when a multi insert inserts into a bucketed table and a table which is not bucketed -- the bucketed table is not merged and the table which is not bucketed is diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_num_reducers.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_num_reducers.q index 37ae6cc7adea..84fe3919d7a6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_num_reducers.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_num_reducers.q @@ -1,6 +1,6 @@ set hive.enforce.bucketing = true; set hive.exec.mode.local.auto=false; -set mapred.reduce.tasks = 10; +set mapreduce.job.reduces = 10; -- This test sets number of mapred tasks to 10 for a database with 50 buckets, -- and uses a post-hook to confirm that 10 tasks were created diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q index d2e12e82d4a2..ae72f98fa424 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q @@ -1,5 +1,5 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; -set mapred.min.split.size = 64; +set mapreduce.input.fileinputformat.split.minsize = 64; CREATE TABLE T1(name STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine1.q index 86abf0996057..5ecfc2172478 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine1.q @@ -1,11 +1,11 @@ set hive.exec.compress.output = true; set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; -set mapred.min.split.size=256; -set mapred.min.split.size.per.node=256; -set mapred.min.split.size.per.rack=256; -set mapred.max.split.size=256; +set mapreduce.input.fileinputformat.split.minsize=256; +set mapreduce.input.fileinputformat.split.minsize.per.node=256; +set mapreduce.input.fileinputformat.split.minsize.per.rack=256; +set mapreduce.input.fileinputformat.split.maxsize=256; -set mapred.output.compression.codec=org.apache.hadoop.io.compress.GzipCodec; +set mapreduce.output.fileoutputformat.compress.codec=org.apache.hadoop.io.compress.GzipCodec; create table combine1_1(key string, value string) stored as textfile; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2.q index cfd9856f0868..acd0dd5e5bc9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2.q @@ -1,10 +1,10 @@ USE default; set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; -set mapred.min.split.size=256; -set mapred.min.split.size.per.node=256; -set mapred.min.split.size.per.rack=256; -set mapred.max.split.size=256; +set mapreduce.input.fileinputformat.split.minsize=256; +set mapreduce.input.fileinputformat.split.minsize.per.node=256; +set mapreduce.input.fileinputformat.split.minsize.per.rack=256; +set mapreduce.input.fileinputformat.split.maxsize=256; set hive.exec.dynamic.partition=true; set hive.exec.dynamic.partition.mode=nonstrict; set mapred.cache.shared.enabled=false; @@ -18,7 +18,7 @@ set hive.merge.smallfiles.avgsize=0; create table combine2(key string) partitioned by (value string); -- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) --- This test sets mapred.max.split.size=256 and hive.merge.smallfiles.avgsize=0 +-- This test sets mapreduce.input.fileinputformat.split.maxsize=256 and hive.merge.smallfiles.avgsize=0 -- in an attempt to force the generation of multiple splits and multiple output files. -- However, Hadoop 0.20 is incapable of generating splits smaller than the block size -- when using CombineFileInputFormat, so only one split is generated. This has a diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_hadoop20.q index 8f9a59d49753..597d3ae479b9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_hadoop20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_hadoop20.q @@ -1,10 +1,10 @@ USE default; set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; -set mapred.min.split.size=256; -set mapred.min.split.size.per.node=256; -set mapred.min.split.size.per.rack=256; -set mapred.max.split.size=256; +set mapreduce.input.fileinputformat.split.minsize=256; +set mapreduce.input.fileinputformat.split.minsize.per.node=256; +set mapreduce.input.fileinputformat.split.minsize.per.rack=256; +set mapreduce.input.fileinputformat.split.maxsize=256; set hive.exec.dynamic.partition=true; set hive.exec.dynamic.partition.mode=nonstrict; set mapred.cache.shared.enabled=false; @@ -17,7 +17,7 @@ set hive.merge.smallfiles.avgsize=0; create table combine2(key string) partitioned by (value string); -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) --- This test sets mapred.max.split.size=256 and hive.merge.smallfiles.avgsize=0 +-- This test sets mapreduce.input.fileinputformat.split.maxsize=256 and hive.merge.smallfiles.avgsize=0 -- in an attempt to force the generation of multiple splits and multiple output files. -- However, Hadoop 0.20 is incapable of generating splits smaller than the block size -- when using CombineFileInputFormat, so only one split is generated. This has a diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_win.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_win.q index f6090bb99b29..4f7174a1b636 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_win.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_win.q @@ -1,8 +1,8 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; -set mapred.min.split.size=256; -set mapred.min.split.size.per.node=256; -set mapred.min.split.size.per.rack=256; -set mapred.max.split.size=256; +set mapreduce.input.fileinputformat.split.minsize=256; +set mapreduce.input.fileinputformat.split.minsize.per.node=256; +set mapreduce.input.fileinputformat.split.minsize.per.rack=256; +set mapreduce.input.fileinputformat.split.maxsize=256; set hive.exec.dynamic.partition=true; set hive.exec.dynamic.partition.mode=nonstrict; set mapred.cache.shared.enabled=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine3.q index c9afc91bb456..35dd442027b4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine3.q @@ -1,9 +1,9 @@ set hive.exec.compress.output = true; set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; -set mapred.min.split.size=256; -set mapred.min.split.size.per.node=256; -set mapred.min.split.size.per.rack=256; -set mapred.max.split.size=256; +set mapreduce.input.fileinputformat.split.minsize=256; +set mapreduce.input.fileinputformat.split.minsize.per.node=256; +set mapreduce.input.fileinputformat.split.minsize.per.rack=256; +set mapreduce.input.fileinputformat.split.maxsize=256; drop table combine_3_srcpart_seq_rc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_1.q index f348e5902263..5e51d11864dd 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_1.q @@ -1,4 +1,4 @@ -set fs.default.name=invalidscheme:///; +set fs.defaultFS=invalidscheme:///; CREATE TABLE table1 (a STRING, b STRING) STORED AS TEXTFILE; DESCRIBE table1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q index f39689de03a5..979c9072303c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q @@ -49,7 +49,7 @@ describe formatted nzhang_CTAS4; explain extended create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10; -set mapred.job.tracker=localhost:58; +set mapreduce.jobtracker.address=localhost:58; set hive.exec.mode.local.auto=true; create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1.q index 1275eab281f4..0d75857e54e5 100755 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1.q @@ -3,12 +3,12 @@ set hive.groupby.skewindata=true; CREATE TABLE dest_g1(key INT, value DOUBLE) STORED AS TEXTFILE; -set fs.default.name=invalidscheme:///; +set fs.defaultFS=invalidscheme:///; EXPLAIN FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key; -set fs.default.name=file:///; +set fs.defaultFS=file:///; FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q index 55133332a866..bbb2859a9d45 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q @@ -1,4 +1,4 @@ -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q index dde37dfd4714..7883d948d067 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q index f346cb7e9014..a5ac3762ce79 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=true; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q index c587b5f658f6..6341eefb5043 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q @@ -1,6 +1,6 @@ set hive.map.aggr=false; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest_g1(key INT, value DOUBLE) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q index 30499248cac1..df4693446d6c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q @@ -1,4 +1,4 @@ -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; EXPLAIN SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key ORDER BY src.key LIMIT 5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q index 794ec758e9ed..7b6e175c2df0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q index 55d1a34b3c92..3aeae0d5c33d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_skew.q index 39a2a178e3a5..998156d05f99 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_skew.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=true; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q index 6d7cb61e2d44..fab4f5d097f1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q @@ -1,6 +1,6 @@ set hive.map.aggr=false; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q index b2450c9ea04e..9ef556cdc583 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q @@ -1,6 +1,6 @@ set hive.map.aggr=false; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map.q index 7ecc71dfab64..36ba5d89c0f7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q index 50243beca9ef..6f0a9635a284 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_skew.q index 07d10c2d741d..64a49e2525ed 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_skew.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=true; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew.q index d33f12c5744e..4fd98efd6ef4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew.q @@ -1,7 +1,7 @@ set hive.map.aggr=false; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q index 86d8986f1df7..85ee8ac43e52 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q @@ -1,7 +1,7 @@ set hive.map.aggr=false; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map.q index 8ecce23eb832..d71721875bbf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(key INT) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map_skew.q index eb2001c6b21b..d1ecba143d62 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map_skew.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=true; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(key INT) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q index a1ebf90aadfe..63530c262c14 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q @@ -1,7 +1,7 @@ set hive.map.aggr=false; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map.q index 4fd6445d7927..4418bbffec7a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(key INT) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map_skew.q index eccd45dd5b42..ef20dacf0599 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map_skew.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=true; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(key INT) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q index e96568b398d8..17b322b890ff 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q @@ -1,7 +1,7 @@ set hive.map.aggr=false; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q index ced122fae3f5..bef0eeee0e89 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q index 0d3727b05285..ee93b218ac78 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=true; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q index 466c13222f29..72fff08decf0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q @@ -1,7 +1,7 @@ set hive.map.aggr=false; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q index 2b8c5db41ea9..75149b140415 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q @@ -1,7 +1,7 @@ set hive.map.aggr=true; set hive.multigroupby.singlereducer=false; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q index 5895ed459984..7c7829aac2d6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q index ee6d7bf83084..905986d417df 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=true; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q index 8c2308e5d75c..1f63453672a4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q @@ -1,7 +1,7 @@ set hive.map.aggr=false; set hive.multigroupby.singlereducer=false; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q index e673cc61622c..2ce57e98072f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q @@ -1,6 +1,6 @@ set hive.map.aggr=false; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q index 0252e993363a..9def7d64721e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q index b5e1f63a4525..788bc683697d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=true; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q index da85504ca18c..17885c56b3f1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q @@ -1,7 +1,7 @@ set hive.map.aggr=false; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE DEST1(key INT, value STRING) STORED AS TEXTFILE; CREATE TABLE DEST2(key INT, value STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q index 4a199365cf96..9cb98aa909e1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(key STRING, c1 INT, c2 STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q index cb3ee8291861..841df75af18b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q @@ -1,6 +1,6 @@ set hive.map.aggr=true; set hive.groupby.skewindata=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE dest1(key STRING, c1 INT, c2 STRING, C3 INT, c4 INT) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q index 7401a9ca1d9b..cdf4bb1cac9d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q @@ -248,7 +248,7 @@ SELECT * FROM outputTbl4 ORDER BY key1, key2, key3; set hive.map.aggr=true; set hive.multigroupby.singlereducer=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE DEST1(key INT, cnt INT); CREATE TABLE DEST2(key INT, val STRING, cnt INT); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q index db0faa04da0e..1c23fad76eff 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q @@ -249,7 +249,7 @@ SELECT * FROM outputTbl4 ORDER BY key1, key2, key3; set hive.map.aggr=true; set hive.multigroupby.singlereducer=false; -set mapred.reduce.tasks=31; +set mapreduce.job.reduces=31; CREATE TABLE DEST1(key INT, cnt INT); CREATE TABLE DEST2(key INT, val STRING, cnt INT); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/hook_context_cs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/hook_context_cs.q index 94ba14802f01..996c9d99f0b9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/hook_context_cs.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/hook_context_cs.q @@ -5,7 +5,7 @@ ALTER TABLE vcsc ADD partition (ds='dummy') location '${system:test.tmp.dir}/Ver set hive.exec.pre.hooks=org.apache.hadoop.hive.ql.hooks.VerifyContentSummaryCacheHook; SELECT a.c, b.c FROM vcsc a JOIN vcsc b ON a.ds = 'dummy' AND b.ds = 'dummy' AND a.c = b.c; -set mapred.job.tracker=local; +set mapreduce.jobtracker.address=local; set hive.exec.pre.hooks = ; set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyContentSummaryCacheHook; SELECT a.c, b.c FROM vcsc a JOIN vcsc b ON a.ds = 'dummy' AND b.ds = 'dummy' AND a.c = b.c; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q index 728b8cc4a949..5d3c6c43c640 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q @@ -63,7 +63,7 @@ set hive.merge.mapredfiles=true; set hive.merge.smallfiles.avgsize=200; set hive.exec.compress.output=false; set hive.exec.dynamic.partition=true; -set mapred.reduce.tasks=2; +set mapreduce.job.reduces=2; -- Tests dynamic partitions where bucketing/sorting can be inferred, but some partitions are -- merged and some are moved. Currently neither should be bucketed or sorted, in the future, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q index 41c1a13980cf..aa49b0dc64c4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q @@ -1,7 +1,7 @@ set hive.exec.infer.bucket.sort=true; set hive.exec.infer.bucket.sort.num.buckets.power.two=true; set hive.merge.mapredfiles=true; -set mapred.reduce.tasks=2; +set mapreduce.job.reduces=2; -- This tests inferring how data is bucketed/sorted from the operators in the reducer -- and populating that information in partitions' metadata. In particular, those cases diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q index 2255bdb34913..3a454f77bc4d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q @@ -1,7 +1,7 @@ set hive.exec.infer.bucket.sort=true; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; -set mapred.reduce.tasks=2; +set mapreduce.job.reduces=2; CREATE TABLE test_table (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input12_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input12_hadoop20.q index 318cd378db13..31e99e8d9464 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input12_hadoop20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input12_hadoop20.q @@ -1,4 +1,4 @@ -set mapred.job.tracker=localhost:58; +set mapreduce.jobtracker.address=localhost:58; set hive.exec.mode.local.auto=true; -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input39_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input39_hadoop20.q index 29e9fae1da9e..362c164176a9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input39_hadoop20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input39_hadoop20.q @@ -15,7 +15,7 @@ select key, value from src; set hive.test.mode=true; set hive.mapred.mode=strict; -set mapred.job.tracker=localhost:58; +set mapreduce.jobtracker.address=localhost:58; set hive.exec.mode.local.auto=true; explain @@ -24,7 +24,7 @@ select count(1) from t1 join t2 on t1.key=t2.key where t1.ds='1' and t2.ds='1'; select count(1) from t1 join t2 on t1.key=t2.key where t1.ds='1' and t2.ds='1'; set hive.test.mode=false; -set mapred.job.tracker; +set mapreduce.jobtracker.address; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testsequencefile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testsequencefile.q index d9926888cef9..2b16c5cd0864 100755 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testsequencefile.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testsequencefile.q @@ -1,5 +1,5 @@ -set mapred.output.compress=true; -set mapred.output.compression.type=BLOCK; +set mapreduce.output.fileoutputformat.compress=true; +set mapreduce.output.fileoutputformat.compress.type=BLOCK; CREATE TABLE dest4_sequencefile(key INT, value STRING) STORED AS SEQUENCEFILE; @@ -10,5 +10,5 @@ INSERT OVERWRITE TABLE dest4_sequencefile SELECT src.key, src.value; FROM src INSERT OVERWRITE TABLE dest4_sequencefile SELECT src.key, src.value; -set mapred.output.compress=false; +set mapreduce.output.fileoutputformat.compress=false; SELECT dest4_sequencefile.* FROM dest4_sequencefile; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join14_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join14_hadoop20.q index a12ef1afb055..b3d75b63bd40 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join14_hadoop20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join14_hadoop20.q @@ -2,7 +2,7 @@ CREATE TABLE dest1(c1 INT, c2 STRING) STORED AS TEXTFILE; -set mapred.job.tracker=localhost:58; +set mapreduce.jobtracker.address=localhost:58; set hive.exec.mode.local.auto=true; EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q index c9ebe0e8fad1..d98247b63d34 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q @@ -9,7 +9,7 @@ SELECT * FROM T1; SELECT * FROM T2; set hive.auto.convert.join=false; -set mapred.reduce.tasks=2; +set mapreduce.job.reduces=2; set hive.join.emit.interval=100; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge2.q index 8b77bd2fe19b..9189e7c0d1af 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge2.q @@ -1,9 +1,9 @@ set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; -set mapred.min.split.size=256; -set mapred.min.split.size.per.node=256; -set mapred.min.split.size.per.rack=256; -set mapred.max.split.size=256; +set mapreduce.input.fileinputformat.split.minsize=256; +set mapreduce.input.fileinputformat.split.minsize.per.node=256; +set mapreduce.input.fileinputformat.split.minsize.per.rack=256; +set mapreduce.input.fileinputformat.split.maxsize=256; create table test1(key int, val int); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_createas1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_createas1.q index 872692567b37..dcb2a853bae5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_createas1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_createas1.q @@ -1,5 +1,5 @@ -set mapred.max.split.size=100; -set mapred.min.split.size=1; +set mapreduce.input.fileinputformat.split.maxsize=100; +set mapreduce.input.fileinputformat.split.minsize=1; DROP TABLE orc_createas1a; DROP TABLE orc_createas1b; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q index 1f5f54ae19ee..93f8f519cf21 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q @@ -1,6 +1,6 @@ SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -SET mapred.min.split.size=1000; -SET mapred.max.split.size=5000; +SET mapreduce.input.fileinputformat.split.minsize=1000; +SET mapreduce.input.fileinputformat.split.maxsize=5000; create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q index c34be867e484..3a74de82a472 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q @@ -1,6 +1,6 @@ SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -SET mapred.min.split.size=1000; -SET mapred.max.split.size=5000; +SET mapreduce.input.fileinputformat.split.minsize=1000; +SET mapreduce.input.fileinputformat.split.maxsize=5000; create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q index a93590eacca0..82f68a9ae56b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q @@ -1,6 +1,6 @@ SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -SET mapred.min.split.size=1000; -SET mapred.max.split.size=5000; +SET mapreduce.input.fileinputformat.split.minsize=1000; +SET mapreduce.input.fileinputformat.split.maxsize=5000; create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_varchar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_varchar.q index 0fecc664e46d..99f58cd73f79 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_varchar.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_varchar.q @@ -1,6 +1,6 @@ SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -SET mapred.min.split.size=1000; -SET mapred.max.split.size=5000; +SET mapreduce.input.fileinputformat.split.minsize=1000; +SET mapreduce.input.fileinputformat.split.maxsize=5000; create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_split_elimination.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_split_elimination.q index 54eb23e776b8..9aa868f9d2f0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_split_elimination.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_split_elimination.q @@ -3,8 +3,8 @@ create table orc_split_elim (userid bigint, string1 string, subtype double, deci load data local inpath '../../data/files/orc_split_elim.orc' into table orc_split_elim; SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -SET mapred.min.split.size=1000; -SET mapred.max.split.size=5000; +SET mapreduce.input.fileinputformat.split.minsize=1000; +SET mapreduce.input.fileinputformat.split.maxsize=5000; SET hive.optimize.index.filter=false; -- The above table will have 5 splits with the followings stats diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel.q index 03edeaadeef5..3ac60306551e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel.q @@ -1,4 +1,4 @@ -set mapred.job.name='test_parallel'; +set mapreduce.job.name='test_parallel'; set hive.exec.parallel=true; set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q index 73c394064484..777771f22763 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q @@ -2,7 +2,7 @@ create table src5 (key string, value string); load data local inpath '../../data/files/kv5.txt' into table src5; load data local inpath '../../data/files/kv5.txt' into table src5; -set mapred.reduce.tasks = 4; +set mapreduce.job.reduces = 4; set hive.optimize.sampling.orderby=true; set hive.optimize.sampling.orderby.percent=0.66f; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_createas1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_createas1.q index f36203724c15..14e13c56b1db 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_createas1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_createas1.q @@ -1,6 +1,6 @@ set hive.merge.rcfile.block.level=true; -set mapred.max.split.size=100; -set mapred.min.split.size=1; +set mapreduce.input.fileinputformat.split.maxsize=100; +set mapreduce.input.fileinputformat.split.minsize=1; DROP TABLE rcfile_createas1a; DROP TABLE rcfile_createas1b; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q index 7f55d10bd645..43a15a06f870 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q @@ -10,7 +10,7 @@ SELECT key, value FROM rcfileTableLazyDecompress where key > 238 and key < 400 O SELECT key, count(1) FROM rcfileTableLazyDecompress where key > 238 group by key ORDER BY key ASC; -set mapred.output.compress=true; +set mapreduce.output.fileoutputformat.compress=true; set hive.exec.compress.output=true; FROM src @@ -22,6 +22,6 @@ SELECT key, value FROM rcfileTableLazyDecompress where key > 238 and key < 400 O SELECT key, count(1) FROM rcfileTableLazyDecompress where key > 238 group by key ORDER BY key ASC; -set mapred.output.compress=false; +set mapreduce.output.fileoutputformat.compress=false; set hive.exec.compress.output=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge1.q index 1f6f1bd251c2..25071579cb04 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge1.q @@ -1,6 +1,6 @@ set hive.merge.rcfile.block.level=false; set hive.exec.dynamic.partition=true; -set mapred.max.split.size=100; +set mapreduce.input.fileinputformat.split.maxsize=100; set mapref.min.split.size=1; DROP TABLE rcfile_merge1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge2.q index 215d5ebc4a25..15ffb90bf627 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge2.q @@ -1,7 +1,7 @@ set hive.merge.rcfile.block.level=true; set hive.exec.dynamic.partition=true; -set mapred.max.split.size=100; -set mapred.min.split.size=1; +set mapreduce.input.fileinputformat.split.maxsize=100; +set mapreduce.input.fileinputformat.split.minsize=1; DROP TABLE rcfile_merge2a; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge3.q index 39fbd2564664..787ab4a8d7fa 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge3.q @@ -1,6 +1,6 @@ set hive.merge.rcfile.block.level=true; -set mapred.max.split.size=100; -set mapred.min.split.size=1; +set mapreduce.input.fileinputformat.split.maxsize=100; +set mapreduce.input.fileinputformat.split.minsize=1; DROP TABLE rcfile_merge3a; DROP TABLE rcfile_merge3b; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge4.q index fe6df28566cf..77ac381c65bb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge4.q @@ -1,6 +1,6 @@ set hive.merge.rcfile.block.level=true; -set mapred.max.split.size=100; -set mapred.min.split.size=1; +set mapreduce.input.fileinputformat.split.maxsize=100; +set mapreduce.input.fileinputformat.split.minsize=1; DROP TABLE rcfile_merge3a; DROP TABLE rcfile_merge3b; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q index 12f2bcd46ec8..bf12ba5ed8e6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q @@ -1,8 +1,8 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; -set mapred.max.split.size=300; -set mapred.min.split.size=300; -set mapred.min.split.size.per.node=300; -set mapred.min.split.size.per.rack=300; +set mapreduce.input.fileinputformat.split.maxsize=300; +set mapreduce.input.fileinputformat.split.minsize=300; +set mapreduce.input.fileinputformat.split.minsize.per.node=300; +set mapreduce.input.fileinputformat.split.minsize.per.rack=300; set hive.exec.mode.local.auto=true; set hive.merge.smallfiles.avgsize=1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q index 484e1fa617d8..5d1bd184d2ad 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q @@ -1,15 +1,15 @@ USE default; set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; -set mapred.max.split.size=300; -set mapred.min.split.size=300; -set mapred.min.split.size.per.node=300; -set mapred.min.split.size.per.rack=300; +set mapreduce.input.fileinputformat.split.maxsize=300; +set mapreduce.input.fileinputformat.split.minsize=300; +set mapreduce.input.fileinputformat.split.minsize.per.node=300; +set mapreduce.input.fileinputformat.split.minsize.per.rack=300; set hive.exec.mode.local.auto=true; set hive.merge.smallfiles.avgsize=1; -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) --- This test sets mapred.max.split.size=300 and hive.merge.smallfiles.avgsize=1 +-- This test sets mapreduce.input.fileinputformat.split.maxsize=300 and hive.merge.smallfiles.avgsize=1 -- in an attempt to force the generation of multiple splits and multiple output files. -- However, Hadoop 0.20 is incapable of generating splits smaller than the block size -- when using CombineFileInputFormat, so only one split is generated. This has a @@ -25,7 +25,7 @@ create table sih_src as select key, value from sih_i_part order by key, value; create table sih_src2 as select key, value from sih_src order by key, value; set hive.exec.post.hooks = org.apache.hadoop.hive.ql.hooks.VerifyIsLocalModeHook ; -set mapred.job.tracker=localhost:58; +set mapreduce.jobtracker.address=localhost:58; set hive.exec.mode.local.auto.input.files.max=1; -- Sample split, running locally limited by num tasks diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split_sample.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split_sample.q index 952eaf72f10c..eb774f15829b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split_sample.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split_sample.q @@ -1,14 +1,14 @@ USE default; set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; -set mapred.max.split.size=300; -set mapred.min.split.size=300; -set mapred.min.split.size.per.node=300; -set mapred.min.split.size.per.rack=300; +set mapreduce.input.fileinputformat.split.maxsize=300; +set mapreduce.input.fileinputformat.split.minsize=300; +set mapreduce.input.fileinputformat.split.minsize.per.node=300; +set mapreduce.input.fileinputformat.split.minsize.per.rack=300; set hive.merge.smallfiles.avgsize=1; -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20) --- This test sets mapred.max.split.size=300 and hive.merge.smallfiles.avgsize=1 +-- This test sets mapreduce.input.fileinputformat.split.maxsize=300 and hive.merge.smallfiles.avgsize=1 -- in an attempt to force the generation of multiple splits and multiple output files. -- However, Hadoop 0.20 is incapable of generating splits smaller than the block size -- when using CombineFileInputFormat, so only one split is generated. This has a @@ -72,10 +72,10 @@ select t1.key as k1, t2.key as k from ss_src1 tablesample(80 percent) t1 full ou -- shrink last split explain select count(1) from ss_src2 tablesample(1 percent); -set mapred.max.split.size=300000; -set mapred.min.split.size=300000; -set mapred.min.split.size.per.node=300000; -set mapred.min.split.size.per.rack=300000; +set mapreduce.input.fileinputformat.split.maxsize=300000; +set mapreduce.input.fileinputformat.split.minsize=300000; +set mapreduce.input.fileinputformat.split.minsize.per.node=300000; +set mapreduce.input.fileinputformat.split.minsize.per.rack=300000; select count(1) from ss_src2 tablesample(1 percent); select count(1) from ss_src2 tablesample(50 percent); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1.q index cdf92e44cf67..caf359c9e6b4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1.q @@ -2,13 +2,13 @@ set datanucleus.cache.collections=false; set hive.stats.autogather=false; set hive.exec.dynamic.partition=true; set hive.exec.dynamic.partition.mode=nonstrict; -set mapred.min.split.size=256; -set mapred.min.split.size.per.node=256; -set mapred.min.split.size.per.rack=256; -set mapred.max.split.size=256; +set mapreduce.input.fileinputformat.split.minsize=256; +set mapreduce.input.fileinputformat.split.minsize.per.node=256; +set mapreduce.input.fileinputformat.split.minsize.per.rack=256; +set mapreduce.input.fileinputformat.split.maxsize=256; -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S) --- This test uses mapred.max.split.size/mapred.max.split.size for controlling +-- This test uses mapreduce.input.fileinputformat.split.maxsize/mapred.max.split.size for controlling -- number of input splits, which is not effective in hive 0.20. -- stats_partscan_1_23.q is the same test with this but has different result. diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1_23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1_23.q index 1e5f360b20cb..07694891fd6f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1_23.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1_23.q @@ -2,13 +2,13 @@ set datanucleus.cache.collections=false; set hive.stats.autogather=false; set hive.exec.dynamic.partition=true; set hive.exec.dynamic.partition.mode=nonstrict; -set mapred.min.split.size=256; -set mapred.min.split.size.per.node=256; -set mapred.min.split.size.per.rack=256; -set mapred.max.split.size=256; +set mapreduce.input.fileinputformat.split.minsize=256; +set mapreduce.input.fileinputformat.split.minsize.per.node=256; +set mapreduce.input.fileinputformat.split.minsize.per.rack=256; +set mapreduce.input.fileinputformat.split.maxsize=256; -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) --- This test uses mapred.max.split.size/mapred.max.split.size for controlling +-- This test uses mapreduce.input.fileinputformat.split.maxsize/mapred.max.split.size for controlling -- number of input splits. -- stats_partscan_1.q is the same test with this but has different result. diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q index f065385688a1..5b5d669a7c12 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q @@ -1,6 +1,6 @@ CREATE TABLE kafka (contents STRING); LOAD DATA LOCAL INPATH '../../data/files/text-en.txt' INTO TABLE kafka; -set mapred.reduce.tasks=1; +set mapreduce.job.reduces=1; set hive.exec.reducers.max=1; SELECT context_ngrams(sentences(lower(contents)), array(null), 100, 1000).estfrequency FROM kafka; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q index 6a2fde52e42f..39e6e30ae694 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q @@ -1,6 +1,6 @@ CREATE TABLE kafka (contents STRING); LOAD DATA LOCAL INPATH '../../data/files/text-en.txt' INTO TABLE kafka; -set mapred.reduce.tasks=1; +set mapreduce.job.reduces=1; set hive.exec.reducers.max=1; SELECT ngrams(sentences(lower(contents)), 1, 100, 1000).estfrequency FROM kafka; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/ambiguous_join_col.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/ambiguous_join_col.q deleted file mode 100644 index e70aae46275b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/ambiguous_join_col.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src src1 JOIN src src2 ON src1.key = src2.key -INSERT OVERWRITE TABLE dest1 SELECT key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/duplicate_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/duplicate_alias.q deleted file mode 100644 index 5fd22460c037..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/duplicate_alias.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src a JOIN src a ON (a.key = a.key) -INSERT OVERWRITE TABLE dest1 SELECT a.key, a.value diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/garbage.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/garbage.q deleted file mode 100644 index 6c8c751f21c3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/garbage.q +++ /dev/null @@ -1 +0,0 @@ -this is totally garbage SELECT src.key WHERE a lot of garbage diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/insert_wrong_number_columns.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/insert_wrong_number_columns.q deleted file mode 100644 index aadfbde33836..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/insert_wrong_number_columns.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value, 1 WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_create_table.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_create_table.q deleted file mode 100644 index 899bbd368b18..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_create_table.q +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE mytable ( - a INT - b STRING -); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_dot.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_dot.q deleted file mode 100644 index 36b9bd2a3b98..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_dot.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.value.member WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_function_param2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_function_param2.q deleted file mode 100644 index 3543449b8870..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_function_param2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT substr('1234', 'abc'), src.value WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_index.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_index.q deleted file mode 100644 index 146bc5dc9f3b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_index.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.key[0], src.value diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index.q deleted file mode 100644 index c40f079f60aa..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src_thrift -INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[0], src_thrift.lstring['abc'] diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index2.q deleted file mode 100644 index 99d0b3d4162a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src_thrift -INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[0], src_thrift.lstring[1 + 2] diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index.q deleted file mode 100644 index c2b9eab61b80..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src_thrift -INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[0], src_thrift.mstringstring[0] diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index2.q deleted file mode 100644 index 5828f0709f53..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src_thrift -INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[0], src_thrift.mstringstring[concat('abc', 'abc')] diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_select.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_select.q deleted file mode 100644 index fd1298577be8..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_select.q +++ /dev/null @@ -1,4 +0,0 @@ -SELECT - trim(trim(a)) - trim(b) -FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/macro_reserved_word.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/macro_reserved_word.q deleted file mode 100644 index 359eb9de93ba..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/macro_reserved_word.q +++ /dev/null @@ -1 +0,0 @@ -CREATE TEMPORARY MACRO DOUBLE (x DOUBLE) 1.0 / (1.0 + EXP(-x)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/missing_overwrite.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/missing_overwrite.q deleted file mode 100644 index 1bfeee382ea3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/missing_overwrite.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/nonkey_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/nonkey_groupby.q deleted file mode 100644 index ad0f4415cbd8..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/nonkey_groupby.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 group by src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/quoted_string.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/quoted_string.q deleted file mode 100644 index 0252a9e11cdf..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/quoted_string.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234", src.value WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column1.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column1.q deleted file mode 100644 index 429cead63beb..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.dummycol WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column2.q deleted file mode 100644 index 3767dc4e6502..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.dummykey < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column3.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column3.q deleted file mode 100644 index 2fc5f490f118..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column3.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 group by src.dummycol diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column4.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column4.q deleted file mode 100644 index 8ad8dd12e46e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column4.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 group by dummysrc.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column5.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column5.q deleted file mode 100644 index 766b0e5255fe..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column5.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE dummysrc.key < 100 group by src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column6.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column6.q deleted file mode 100644 index bb76c2862348..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column6.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', dummysrc.value WHERE src.key < 100 group by src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function1.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function1.q deleted file mode 100644 index d8ff6325b95f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', dummyfn(src.value, 10) WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function2.q deleted file mode 100644 index f7d255934db5..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE anotherdummyfn('abc', src.key) + 10 < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function3.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function3.q deleted file mode 100644 index 87d4edc98786..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function3.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE anotherdummyfn('abc', src.key) + 10 < 100 group by src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function4.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function4.q deleted file mode 100644 index cfe70e4f2fdc..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function4.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', dummyfn(src.key) WHERE src.key < 100 group by src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table1.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table1.q deleted file mode 100644 index 585ef6d7f2db..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM dummySrc -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table2.q deleted file mode 100644 index 2c69c16be590..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dummyDest SELECT '1234', src.value WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct1.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct1.q deleted file mode 100755 index d92c3bb8df4b..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT DISTINCT src.key, substr(src.value,4,1) GROUP BY src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct2.q deleted file mode 100755 index 53fb550b3d11..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.key, DISTINCT substr(src.value,4,1) GROUP BY src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/case_sensitivity.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/case_sensitivity.q deleted file mode 100644 index d7f737150766..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/case_sensitivity.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM SRC_THRIFT -INSERT OVERWRITE TABLE dest1 SELECT src_Thrift.LINT[1], src_thrift.lintstring[0].MYSTRING where src_thrift.liNT[0] > 0 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/cast1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/cast1.q deleted file mode 100644 index 6269c6a4e76f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/cast1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -SELECT 3 + 2, 3.0 + 2, 3 + 2.0, 3.0 + 2.0, 3 + CAST(2.0 AS INT), CAST(1 AS BOOLEAN), CAST(TRUE AS INT) WHERE src.key = 86 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby1.q deleted file mode 100755 index 96b29b05cc7a..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby2.q deleted file mode 100755 index d741eb60b6bb..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) GROUP BY substr(src.key,1,1) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby3.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby3.q deleted file mode 100755 index 03b1248a11cb..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby3.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -SELECT sum(substr(src.value,5)), avg(substr(src.value,5)), avg(DISTINCT substr(src.value,5)), max(substr(src.value,5)), min(substr(src.value,5)) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby4.q deleted file mode 100755 index 85271a9caf6e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby4.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -SELECT substr(src.key,1,1) GROUP BY substr(src.key,1,1) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby5.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby5.q deleted file mode 100755 index ebd65b306972..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby5.q +++ /dev/null @@ -1,4 +0,0 @@ - -SELECT src.key, sum(substr(src.value,5)) -FROM src -GROUP BY src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby6.q deleted file mode 100755 index 80654f2a9ce6..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby6.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -SELECT DISTINCT substr(src.value,5,1) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input1.q deleted file mode 100644 index fdd290d6b136..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input2.q deleted file mode 100644 index 4e1612ea972e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input2.q +++ /dev/null @@ -1,4 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.* WHERE src.key < 100 -INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key >= 100 and src.key < 200 -INSERT OVERWRITE TABLE dest3 PARTITION(ds='2008-04-08', hr='12') SELECT src.key, 2 WHERE src.key >= 200 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input20.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input20.q deleted file mode 100644 index f30cf27017d9..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input20.q +++ /dev/null @@ -1,9 +0,0 @@ -FROM ( - FROM src - MAP src.key % 2, src.key % 5 - USING 'cat' - CLUSTER BY key -) tmap -REDUCE tmap.key, tmap.value -USING 'uniq -c | sed "s@^ *@@" | sed "s@\t@_@" | sed "s@ @\t@"' -AS key, value diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input3.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input3.q deleted file mode 100644 index fc53e94d39f0..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input3.q +++ /dev/null @@ -1,5 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.* WHERE src.key < 100 -INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key >= 100 and src.key < 200 -INSERT OVERWRITE TABLE dest3 PARTITION(ds='2008-04-08', hr='12') SELECT src.key, 2 WHERE src.key >= 200 and src.key < 300 -INSERT OVERWRITE DIRECTORY '../../../../build/contrib/hive/ql/test/data/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input4.q deleted file mode 100644 index 03e6de48faca..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input4.q +++ /dev/null @@ -1,7 +0,0 @@ -FROM ( - FROM src - SELECT TRANSFORM(src.key, src.value) - USING '/bin/cat' AS (tkey, tvalue) - CLUSTER BY tkey -) tmap -INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input5.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input5.q deleted file mode 100644 index a46abc75833f..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input5.q +++ /dev/null @@ -1,7 +0,0 @@ -FROM ( - FROM src_thrift - SELECT TRANSFORM(src_thrift.lint, src_thrift.lintstring) - USING '/bin/cat' AS (tkey, tvalue) - CLUSTER BY tkey -) tmap -INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input6.q deleted file mode 100644 index d6f25a935ae7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input6.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src1 -INSERT OVERWRITE TABLE dest1 SELECT src1.key, src1.value WHERE src1.key is null diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input7.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input7.q deleted file mode 100644 index 33a82953c26e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input7.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src1 -INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input8.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input8.q deleted file mode 100644 index 0843b9ba4e55..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input8.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src1 -SELECT 4 + NULL, src1.key - NULL, NULL + NULL diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input9.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input9.q deleted file mode 100644 index 2892f0b2dfc4..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input9.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src1 -INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key where NULL = NULL diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_part1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input_part1.q deleted file mode 100644 index d45d1cd0b47e..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_part1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM srcpart -SELECT srcpart.key, srcpart.value, srcpart.hr, srcpart.ds WHERE srcpart.key < 100 and srcpart.ds = '2008-04-08' and srcpart.hr = '12' diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testsequencefile.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testsequencefile.q deleted file mode 100755 index cf9a092417e1..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testsequencefile.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest4_sequencefile SELECT src.key, src.value diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath.q deleted file mode 100755 index 7699bff75552..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src_thrift -SELECT src_thrift.lint[1], src_thrift.lintstring[0].mystring, src_thrift.mstringstring['key_2'] diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath2.q deleted file mode 100644 index 08abaf4fad8d..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src_thrift -SELECT size(src_thrift.lint), size(src_thrift.lintstring), size(src_thrift.mstringstring) where src_thrift.lint IS NOT NULL AND NOT (src_thrift.mstringstring IS NULL) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join1.q deleted file mode 100644 index 739c39dd8f71..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join1.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM src src1 JOIN src src2 ON (src1.key = src2.key) -INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join2.q deleted file mode 100644 index a02d87f09f58..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join2.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) -INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join3.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join3.q deleted file mode 100644 index b57c9569d728..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join3.q +++ /dev/null @@ -1,4 +0,0 @@ -FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key = src3.key) -INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join4.q deleted file mode 100644 index 2e5967fb7d85..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join4.q +++ /dev/null @@ -1,14 +0,0 @@ -FROM ( - FROM - ( - FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 - ) a - LEFT OUTER JOIN - ( - FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 - ) b - ON (a.c1 = b.c3) - SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 -) c -SELECT c.c1, c.c2, c.c3, c.c4 - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join5.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join5.q deleted file mode 100644 index 63a38f554a24..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join5.q +++ /dev/null @@ -1,15 +0,0 @@ -FROM ( - FROM - ( - FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 - ) a - RIGHT OUTER JOIN - ( - FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 - ) b - ON (a.c1 = b.c3) - SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 -) c -SELECT c.c1, c.c2, c.c3, c.c4 - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join6.q deleted file mode 100644 index 110451cf3039..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join6.q +++ /dev/null @@ -1,16 +0,0 @@ -FROM ( - FROM - ( - FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 - ) a - FULL OUTER JOIN - ( - FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 - ) b - ON (a.c1 = b.c3) - SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 -) c -SELECT c.c1, c.c2, c.c3, c.c4 - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join7.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join7.q deleted file mode 100644 index 65797b44a2cb..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join7.q +++ /dev/null @@ -1,21 +0,0 @@ -FROM ( - FROM - ( - FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 - ) a - FULL OUTER JOIN - ( - FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 - ) b - ON (a.c1 = b.c3) - LEFT OUTER JOIN - ( - FROM src src3 SELECT src3.key AS c5, src3.value AS c6 WHERE src3.key > 20 and src3.key < 25 - ) c - ON (a.c1 = c.c5) - SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4, c.c5 AS c5, c.c6 AS c6 -) c -SELECT c.c1, c.c2, c.c3, c.c4, c.c5, c.c6 - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join8.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join8.q deleted file mode 100644 index d215b07a6720..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join8.q +++ /dev/null @@ -1,14 +0,0 @@ -FROM ( - FROM - ( - FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 - ) a - LEFT OUTER JOIN - ( - FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 - ) b - ON (a.c1 = b.c3) - SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 -) c -SELECT c.c1, c.c2, c.c3, c.c4 where c.c3 IS NULL AND c.c1 IS NOT NULL - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample1.q deleted file mode 100644 index 3a168b999d70..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample1.q +++ /dev/null @@ -1,5 +0,0 @@ --- no input pruning, no sample filter -SELECT s.* -FROM srcpart TABLESAMPLE (BUCKET 1 OUT OF 1 ON rand()) s -WHERE s.ds='2008-04-08' and s.hr='11' - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample2.q deleted file mode 100644 index b505b896fa2c..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample2.q +++ /dev/null @@ -1,4 +0,0 @@ --- input pruning, no sample filter --- default table sample columns -INSERT OVERWRITE TABLE dest1 SELECT s.* -FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2) s diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample3.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample3.q deleted file mode 100644 index 42d5a2bbec34..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample3.q +++ /dev/null @@ -1,4 +0,0 @@ --- sample columns not same as bucket columns --- no input pruning, sample filter -INSERT OVERWRITE TABLE dest1 SELECT s.* -- here's another test -FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key, value) s diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample4.q deleted file mode 100644 index 7b5ab03380ae..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample4.q +++ /dev/null @@ -1,4 +0,0 @@ --- bucket column is the same as table sample --- No need for sample filter -INSERT OVERWRITE TABLE dest1 SELECT s.* -FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key) s diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample5.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample5.q deleted file mode 100644 index b9b48fdc7188..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample5.q +++ /dev/null @@ -1,3 +0,0 @@ --- no input pruning, sample filter -INSERT OVERWRITE TABLE dest1 SELECT s.* -- here's another test -FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 on key) s diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample6.q deleted file mode 100644 index 0ee026f0f368..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample6.q +++ /dev/null @@ -1,3 +0,0 @@ --- both input pruning and sample filter -INSERT OVERWRITE TABLE dest1 SELECT s.* -FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 4 on key) s diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample7.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample7.q deleted file mode 100644 index f17ce105c357..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample7.q +++ /dev/null @@ -1,4 +0,0 @@ --- both input pruning and sample filter -INSERT OVERWRITE TABLE dest1 SELECT s.* -FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 4 on key) s -WHERE s.key > 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/subq.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/subq.q deleted file mode 100644 index 6392dbcc4380..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/subq.q +++ /dev/null @@ -1,4 +0,0 @@ -FROM ( - FROM src select src.* WHERE src.key < 100 -) unioninput -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.* diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf1.q deleted file mode 100644 index 2ecf46e742c3..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf1.q +++ /dev/null @@ -1,5 +0,0 @@ -FROM src SELECT 'a' LIKE '%a%', 'b' LIKE '%a%', 'ab' LIKE '%a%', 'ab' LIKE '%a_', - '%_' LIKE '\%\_', 'ab' LIKE '\%\_', 'ab' LIKE '_a%', 'ab' LIKE 'a', - '' RLIKE '.*', 'a' RLIKE '[ab]', '' RLIKE '[ab]', 'hadoop' RLIKE '[a-z]*', 'hadoop' RLIKE 'o*', - REGEXP_REPLACE('abc', 'b', 'c'), REGEXP_REPLACE('abc', 'z', 'a'), REGEXP_REPLACE('abbbb', 'bb', 'b'), REGEXP_REPLACE('hadoop', '(.)[a-z]*', '$1ive') - WHERE src.key = 86 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf4.q deleted file mode 100644 index f3a7598e1721..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf4.q +++ /dev/null @@ -1 +0,0 @@ -SELECT round(1.0), round(1.5), round(-1.5), floor(1.0), floor(1.5), floor(-1.5), sqrt(1.0), sqrt(-1.0), sqrt(0.0), ceil(1.0), ceil(1.5), ceil(-1.5), ceiling(1.0), rand(3), +3, -3, 1++2, 1+-2, ~1 FROM dest1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf6.q deleted file mode 100644 index 65791c41c1ff..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf6.q +++ /dev/null @@ -1 +0,0 @@ -FROM src SELECT CONCAT('a', 'b'), IF(TRUE, 1 ,2) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_case.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_case.q deleted file mode 100644 index 0c86da219869..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_case.q +++ /dev/null @@ -1,10 +0,0 @@ -SELECT CASE 1 - WHEN 1 THEN 2 - WHEN 3 THEN 4 - ELSE 5 - END, - CASE 11 - WHEN 12 THEN 13 - WHEN 14 THEN 15 - END -FROM src LIMIT 1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_when.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_when.q deleted file mode 100644 index 99ed09990b87..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_when.q +++ /dev/null @@ -1,10 +0,0 @@ -SELECT CASE - WHEN 1=1 THEN 2 - WHEN 3=5 THEN 4 - ELSE 5 - END, - CASE - WHEN 12=11 THEN 13 - WHEN 14=10 THEN 15 - END -FROM src LIMIT 1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/union.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/union.q deleted file mode 100644 index 6a6b9882aee7..000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/union.q +++ /dev/null @@ -1,6 +0,0 @@ -FROM ( - FROM src select src.key, src.value WHERE src.key < 100 - UNION ALL - FROM src SELECT src.* WHERE src.key > 100 -) unioninput -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.* diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala b/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala index 2590040f2ec1..4fbbbacb7608 100644 --- a/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala +++ b/sql/hive/src/test/resources/regression-test-SPARK-8489/Main.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.SparkSession /** * Entry point in test application for SPARK-8489. @@ -28,19 +27,23 @@ import org.apache.spark.sql.hive.HiveContext * * This is used in org.apache.spark.sql.hive.HiveSparkSubmitSuite. */ +// TODO: actually rebuild this jar with the new changes. object Main { def main(args: Array[String]) { // scalastyle:off println println("Running regression test for SPARK-8489.") - val sc = new SparkContext("local", "testing") - val hc = new HiveContext(sc) + val spark = SparkSession.builder + .master("local") + .appName("testing") + .enableHiveSupport() + .getOrCreate() // This line should not throw scala.reflect.internal.MissingRequirementError. // See SPARK-8470 for more detail. - val df = hc.createDataFrame(Seq(MyCoolClass("1", "2", "3"))) + val df = spark.createDataFrame(Seq(MyCoolClass("1", "2", "3"))) df.collect() println("Regression test for SPARK-8489 success!") // scalastyle:on println - sc.stop() + spark.stop() } } diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.10.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.10.jar new file mode 100644 index 000000000000..3f28d37b9315 Binary files /dev/null and b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.10.jar differ diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.11.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.11.jar new file mode 100644 index 000000000000..5e093697e219 Binary files /dev/null and b/sql/hive/src/test/resources/regression-test-SPARK-8489/test-2.11.jar differ diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8489/test.jar b/sql/hive/src/test/resources/regression-test-SPARK-8489/test.jar deleted file mode 100644 index 5944aa6076a5..000000000000 Binary files a/sql/hive/src/test/resources/regression-test-SPARK-8489/test.jar and /dev/null differ diff --git a/sql/hive/src/test/resources/test_script.sh b/sql/hive/src/test/resources/test_script.sh new file mode 100755 index 000000000000..eb0c50e98292 --- /dev/null +++ b/sql/hive/src/test/resources/test_script.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +while read line +do + echo "$line" | sed $'s/\t/_/' +done < /dev/stdin diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala new file mode 100644 index 000000000000..d9cf1f361c1d --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst + +import java.sql.Timestamp + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.unsafe.types.CalendarInterval + +class ExpressionSQLBuilderSuite extends QueryTest with TestHiveSingleton { + protected def checkSQL(e: Expression, expectedSQL: String): Unit = { + val actualSQL = e.sql + try { + assert(actualSQL == expectedSQL) + } catch { + case cause: Throwable => + fail( + s"""Wrong SQL generated for the following expression: + | + |${e.prettyName} + | + |$cause + """.stripMargin) + } + } + + test("literal") { + checkSQL(Literal("foo"), "'foo'") + checkSQL(Literal("\"foo\""), "'\"foo\"'") + checkSQL(Literal("'foo'"), "'\\'foo\\''") + checkSQL(Literal(1: Byte), "1Y") + checkSQL(Literal(2: Short), "2S") + checkSQL(Literal(4: Int), "4") + checkSQL(Literal(8: Long), "8L") + checkSQL(Literal(1.5F), "CAST(1.5 AS FLOAT)") + checkSQL(Literal(Float.PositiveInfinity), "CAST('Infinity' AS FLOAT)") + checkSQL(Literal(Float.NegativeInfinity), "CAST('-Infinity' AS FLOAT)") + checkSQL(Literal(Float.NaN), "CAST('NaN' AS FLOAT)") + checkSQL(Literal(2.5D), "2.5D") + checkSQL(Literal(Double.PositiveInfinity), "CAST('Infinity' AS DOUBLE)") + checkSQL(Literal(Double.NegativeInfinity), "CAST('-Infinity' AS DOUBLE)") + checkSQL(Literal(Double.NaN), "CAST('NaN' AS DOUBLE)") + checkSQL(Literal(BigDecimal("10.0000000").underlying), "10.0000000BD") + checkSQL(Literal(Array(0x01, 0xA3).map(_.toByte)), "X'01A3'") + checkSQL( + Literal(Timestamp.valueOf("2016-01-01 00:00:00")), "TIMESTAMP('2016-01-01 00:00:00.0')") + // TODO tests for decimals + } + + test("attributes") { + checkSQL('a.int, "`a`") + checkSQL(Symbol("foo bar").int, "`foo bar`") + // Keyword + checkSQL('int.int, "`int`") + } + + test("binary comparisons") { + checkSQL('a.int === 'b.int, "(`a` = `b`)") + checkSQL('a.int <=> 'b.int, "(`a` <=> `b`)") + checkSQL('a.int =!= 'b.int, "(NOT (`a` = `b`))") + + checkSQL('a.int < 'b.int, "(`a` < `b`)") + checkSQL('a.int <= 'b.int, "(`a` <= `b`)") + checkSQL('a.int > 'b.int, "(`a` > `b`)") + checkSQL('a.int >= 'b.int, "(`a` >= `b`)") + + checkSQL('a.int in ('b.int, 'c.int), "(`a` IN (`b`, `c`))") + checkSQL('a.int in (1, 2), "(`a` IN (1, 2))") + + checkSQL('a.int.isNull, "(`a` IS NULL)") + checkSQL('a.int.isNotNull, "(`a` IS NOT NULL)") + } + + test("logical operators") { + checkSQL('a.boolean && 'b.boolean, "(`a` AND `b`)") + checkSQL('a.boolean || 'b.boolean, "(`a` OR `b`)") + checkSQL(!'a.boolean, "(NOT `a`)") + checkSQL(If('a.boolean, 'b.int, 'c.int), "(IF(`a`, `b`, `c`))") + } + + test("arithmetic expressions") { + checkSQL('a.int + 'b.int, "(`a` + `b`)") + checkSQL('a.int - 'b.int, "(`a` - `b`)") + checkSQL('a.int * 'b.int, "(`a` * `b`)") + checkSQL('a.int / 'b.int, "(`a` / `b`)") + checkSQL('a.int % 'b.int, "(`a` % `b`)") + + checkSQL(-'a.int, "(- `a`)") + checkSQL(-('a.int + 'b.int), "(- (`a` + `b`))") + } + + test("window specification") { + val frame = SpecifiedWindowFrame.defaultWindowFrame( + hasOrderSpecification = true, + acceptWindowFrame = true + ) + + checkSQL( + WindowSpecDefinition('a.int :: Nil, Nil, frame), + s"(PARTITION BY `a` ${frame.sql})" + ) + + checkSQL( + WindowSpecDefinition('a.int :: 'b.string :: Nil, Nil, frame), + s"(PARTITION BY `a`, `b` ${frame.sql})" + ) + + checkSQL( + WindowSpecDefinition(Nil, 'a.int.asc :: Nil, frame), + s"(ORDER BY `a` ASC NULLS FIRST ${frame.sql})" + ) + + checkSQL( + WindowSpecDefinition(Nil, 'a.int.asc :: 'b.string.desc :: Nil, frame), + s"(ORDER BY `a` ASC NULLS FIRST, `b` DESC NULLS LAST ${frame.sql})" + ) + + checkSQL( + WindowSpecDefinition('a.int :: 'b.string :: Nil, 'c.int.asc :: 'd.string.desc :: Nil, frame), + s"(PARTITION BY `a`, `b` ORDER BY `c` ASC NULLS FIRST, `d` DESC NULLS LAST ${frame.sql})" + ) + } + + test("interval arithmetic") { + val interval = Literal(new CalendarInterval(0, CalendarInterval.MICROS_PER_DAY)) + + checkSQL( + TimeAdd('a, interval), + "`a` + interval 1 days" + ) + + checkSQL( + TimeSub('a, interval), + "`a` - interval 1 days" + ) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala new file mode 100644 index 000000000000..e599d1ab1d48 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala @@ -0,0 +1,232 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import scala.concurrent.duration._ + +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFPercentileApprox + +import org.apache.spark.sql.Column +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.catalog.CatalogFunction +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile +import org.apache.spark.sql.hive.HiveSessionCatalog +import org.apache.spark.sql.hive.execution.TestingTypedCount +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.LongType +import org.apache.spark.util.Benchmark + +class ObjectHashAggregateExecBenchmark extends BenchmarkBase with TestHiveSingleton { + ignore("Hive UDAF vs Spark AF") { + val N = 2 << 15 + + val benchmark = new Benchmark( + name = "hive udaf vs spark af", + valuesPerIteration = N, + minNumIters = 5, + warmupTime = 5.seconds, + minTime = 10.seconds, + outputPerIteration = true + ) + + registerHiveFunction("hive_percentile_approx", classOf[GenericUDAFPercentileApprox]) + + sparkSession.range(N).createOrReplaceTempView("t") + + benchmark.addCase("hive udaf w/o group by") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "false") + sparkSession.sql("SELECT hive_percentile_approx(id, 0.5) FROM t").collect() + } + + benchmark.addCase("spark af w/o group by") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") + sparkSession.sql("SELECT percentile_approx(id, 0.5) FROM t").collect() + } + + benchmark.addCase("hive udaf w/ group by") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "false") + sparkSession.sql( + s"SELECT hive_percentile_approx(id, 0.5) FROM t GROUP BY CAST(id / ${N / 4} AS BIGINT)" + ).collect() + } + + benchmark.addCase("spark af w/ group by w/o fallback") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") + sparkSession.sql( + s"SELECT percentile_approx(id, 0.5) FROM t GROUP BY CAST(id / ${N / 4} AS BIGINT)" + ).collect() + } + + benchmark.addCase("spark af w/ group by w/ fallback") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") + sparkSession.conf.set(SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key, "2") + sparkSession.sql( + s"SELECT percentile_approx(id, 0.5) FROM t GROUP BY CAST(id / ${N / 4} AS BIGINT)" + ).collect() + } + + benchmark.run() + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + hive udaf vs spark af: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + hive udaf w/o group by 5326 / 5408 0.0 81264.2 1.0X + spark af w/o group by 93 / 111 0.7 1415.6 57.4X + hive udaf w/ group by 3804 / 3946 0.0 58050.1 1.4X + spark af w/ group by w/o fallback 71 / 90 0.9 1085.7 74.8X + spark af w/ group by w/ fallback 98 / 111 0.7 1501.6 54.1X + */ + } + + ignore("ObjectHashAggregateExec vs SortAggregateExec - typed_count") { + val N: Long = 1024 * 1024 * 100 + + val benchmark = new Benchmark( + name = "object agg v.s. sort agg", + valuesPerIteration = N, + minNumIters = 1, + warmupTime = 10.seconds, + minTime = 45.seconds, + outputPerIteration = true + ) + + import sparkSession.implicits._ + + def typed_count(column: Column): Column = + Column(TestingTypedCount(column.expr).toAggregateExpression()) + + val df = sparkSession.range(N) + + benchmark.addCase("sort agg w/ group by") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "false") + df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).collect() + } + + benchmark.addCase("object agg w/ group by w/o fallback") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") + df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).collect() + } + + benchmark.addCase("object agg w/ group by w/ fallback") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") + sparkSession.conf.set(SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key, "2") + df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).collect() + } + + benchmark.addCase("sort agg w/o group by") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "false") + df.select(typed_count($"id")).collect() + } + + benchmark.addCase("object agg w/o group by w/o fallback") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") + df.select(typed_count($"id")).collect() + } + + benchmark.run() + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + object agg v.s. sort agg: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + sort agg w/ group by 31251 / 31908 3.4 298.0 1.0X + object agg w/ group by w/o fallback 6903 / 7141 15.2 65.8 4.5X + object agg w/ group by w/ fallback 20945 / 21613 5.0 199.7 1.5X + sort agg w/o group by 4734 / 5463 22.1 45.2 6.6X + object agg w/o group by w/o fallback 4310 / 4529 24.3 41.1 7.3X + */ + } + + ignore("ObjectHashAggregateExec vs SortAggregateExec - percentile_approx") { + val N = 2 << 20 + + val benchmark = new Benchmark( + name = "object agg v.s. sort agg", + valuesPerIteration = N, + minNumIters = 5, + warmupTime = 15.seconds, + minTime = 45.seconds, + outputPerIteration = true + ) + + import sparkSession.implicits._ + + val df = sparkSession.range(N).coalesce(1) + + benchmark.addCase("sort agg w/ group by") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "false") + df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).collect() + } + + benchmark.addCase("object agg w/ group by w/o fallback") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") + df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).collect() + } + + benchmark.addCase("object agg w/ group by w/ fallback") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") + sparkSession.conf.set(SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key, "2") + df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).collect() + } + + benchmark.addCase("sort agg w/o group by") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "false") + df.select(percentile_approx($"id", 0.5)).collect() + } + + benchmark.addCase("object agg w/o group by w/o fallback") { _ => + sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") + df.select(percentile_approx($"id", 0.5)).collect() + } + + benchmark.run() + + /* + Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5 + Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + + object agg v.s. sort agg: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + sort agg w/ group by 3418 / 3530 0.6 1630.0 1.0X + object agg w/ group by w/o fallback 3210 / 3314 0.7 1530.7 1.1X + object agg w/ group by w/ fallback 3419 / 3511 0.6 1630.1 1.0X + sort agg w/o group by 4336 / 4499 0.5 2067.3 0.8X + object agg w/o group by w/o fallback 4271 / 4372 0.5 2036.7 0.8X + */ + } + + private def registerHiveFunction(functionName: String, clazz: Class[_]): Unit = { + val sessionCatalog = sparkSession.sessionState.catalog.asInstanceOf[HiveSessionCatalog] + val functionIdentifier = FunctionIdentifier(functionName, database = None) + val func = CatalogFunction(functionIdentifier, clazz.getName, resources = Nil) + sessionCatalog.registerFunction(func, overrideIfExists = false) + } + + private def percentile_approx( + column: Column, percentage: Double, isDistinct: Boolean = false): Column = { + val approxPercentile = new ApproximatePercentile(column.expr, Literal(percentage)) + Column(approxPercentile.toAggregateExpression(isDistinct)) + } +} 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 5c2fc7d82ffb..d3cbf898e243 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 @@ -19,28 +19,35 @@ package org.apache.spark.sql.hive import java.io.File -import org.apache.spark.sql.columnar.InMemoryColumnarTableScan -import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation +import org.apache.spark.sql.{AnalysisException, Dataset, QueryTest, SaveMode} +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.{AnalysisException, QueryTest, SaveMode} +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.StructType import org.apache.spark.storage.RDDBlockId import org.apache.spark.util.Utils -class CachedTableSuite extends QueryTest with TestHiveSingleton { +class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import hiveContext._ def rddIdOf(tableName: String): Int = { - val executedPlan = table(tableName).queryExecution.executedPlan - executedPlan.collect { - case InMemoryColumnarTableScan(_, _, relation) => + val plan = table(tableName).queryExecution.sparkPlan + plan.collect { + case InMemoryTableScanExec(_, _, relation) => relation.cachedColumnBuffers.id case _ => - fail(s"Table $tableName is not cached\n" + executedPlan) + fail(s"Table $tableName is not cached\n" + plan) }.head } def isMaterialized(rddId: Int): Boolean = { - sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)).nonEmpty + val maybeBlock = sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)) + maybeBlock.foreach(_ => sparkContext.env.blockManager.releaseLock(RDDBlockId(rddId, 0))) + maybeBlock.nonEmpty } test("cache table") { @@ -94,46 +101,67 @@ class CachedTableSuite extends QueryTest with TestHiveSingleton { sql("DROP TABLE IF EXISTS nonexistantTable") } - test("correct error on uncache of non-cached table") { - intercept[IllegalArgumentException] { - hiveContext.uncacheTable("src") + test("uncache of nonexistant tables") { + // make sure table doesn't exist + intercept[NoSuchTableException](spark.table("nonexistantTable")) + intercept[NoSuchTableException] { + spark.catalog.uncacheTable("nonexistantTable") + } + intercept[NoSuchTableException] { + sql("UNCACHE TABLE nonexistantTable") + } + sql("UNCACHE TABLE IF EXISTS nonexistantTable") + } + + test("no error on uncache of non-cached table") { + val tableName = "newTable" + withTable(tableName) { + sql(s"CREATE TABLE $tableName(a INT)") + // no error will be reported in the following three ways to uncache a table. + spark.catalog.uncacheTable(tableName) + sql("UNCACHE TABLE newTable") + sparkSession.table(tableName).unpersist() } } test("'CACHE TABLE' and 'UNCACHE TABLE' HiveQL statement") { sql("CACHE TABLE src") assertCached(table("src")) - assert(hiveContext.isCached("src"), "Table 'src' should be cached") + assert(spark.catalog.isCached("src"), "Table 'src' should be cached") sql("UNCACHE TABLE src") assertCached(table("src"), 0) - assert(!hiveContext.isCached("src"), "Table 'src' should not be cached") + assert(!spark.catalog.isCached("src"), "Table 'src' should not be cached") } test("CACHE TABLE tableName AS SELECT * FROM anotherTable") { - sql("CACHE TABLE testCacheTable AS SELECT * FROM src") - assertCached(table("testCacheTable")) + withTempView("testCacheTable") { + sql("CACHE TABLE testCacheTable AS SELECT * FROM src") + assertCached(table("testCacheTable")) - val rddId = rddIdOf("testCacheTable") - assert( - isMaterialized(rddId), - "Eagerly cached in-memory table should have already been materialized") + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") - uncacheTable("testCacheTable") - assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + uncacheTable("testCacheTable") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } } test("CACHE TABLE tableName AS SELECT ...") { - sql("CACHE TABLE testCacheTable AS SELECT key FROM src LIMIT 10") - assertCached(table("testCacheTable")) + withTempView("testCacheTable") { + sql("CACHE TABLE testCacheTable AS SELECT key FROM src LIMIT 10") + assertCached(table("testCacheTable")) - val rddId = rddIdOf("testCacheTable") - assert( - isMaterialized(rddId), - "Eagerly cached in-memory table should have already been materialized") + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") - uncacheTable("testCacheTable") - assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + uncacheTable("testCacheTable") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } } test("CACHE LAZY TABLE tableName") { @@ -155,9 +183,11 @@ class CachedTableSuite extends QueryTest with TestHiveSingleton { } test("CACHE TABLE with Hive UDF") { - sql("CACHE TABLE udfTest AS SELECT * FROM src WHERE floor(key) = 1") - assertCached(table("udfTest")) - uncacheTable("udfTest") + withTempView("udfTest") { + sql("CACHE TABLE udfTest AS SELECT * FROM src WHERE floor(key) = 1") + assertCached(table("udfTest")) + uncacheTable("udfTest") + } } test("REFRESH TABLE also needs to recache the data (data source tables)") { @@ -165,39 +195,72 @@ class CachedTableSuite extends QueryTest with TestHiveSingleton { tempPath.delete() table("src").write.mode(SaveMode.Overwrite).parquet(tempPath.toString) sql("DROP TABLE IF EXISTS refreshTable") - createExternalTable("refreshTable", tempPath.toString, "parquet") - checkAnswer( - table("refreshTable"), - table("src").collect()) + sparkSession.catalog.createTable("refreshTable", tempPath.toString, "parquet") + checkAnswer(table("refreshTable"), table("src")) // Cache the table. sql("CACHE TABLE refreshTable") assertCached(table("refreshTable")) // Append new data. table("src").write.mode(SaveMode.Append).parquet(tempPath.toString) - // We are still using the old data. + assertCached(table("refreshTable")) + + // We are using the new data. assertCached(table("refreshTable")) checkAnswer( table("refreshTable"), - table("src").collect()) - // Refresh the table. + table("src").union(table("src")).collect()) + + // Drop the table and create it again. + sql("DROP TABLE refreshTable") + sparkSession.catalog.createExternalTable("refreshTable", tempPath.toString, "parquet") + // It is not cached. + assert(!isCached("refreshTable"), "refreshTable should not be cached.") + // Refresh the table. REFRESH TABLE command should not make a uncached + // table cached. sql("REFRESH TABLE refreshTable") + checkAnswer( + table("refreshTable"), + table("src").union(table("src")).collect()) + // It is not cached. + assert(!isCached("refreshTable"), "refreshTable should not be cached.") + + sql("DROP TABLE refreshTable") + Utils.deleteRecursively(tempPath) + } + + test("SPARK-15678: REFRESH PATH") { + val tempPath: File = Utils.createTempDir() + tempPath.delete() + table("src").write.mode(SaveMode.Overwrite).parquet(tempPath.toString) + sql("DROP TABLE IF EXISTS refreshTable") + sparkSession.catalog.createExternalTable("refreshTable", tempPath.toString, "parquet") + checkAnswer( + table("refreshTable"), + table("src").collect()) + // Cache the table. + sql("CACHE TABLE refreshTable") + assertCached(table("refreshTable")) + // Append new data. + table("src").write.mode(SaveMode.Append).parquet(tempPath.toString) + assertCached(table("refreshTable")) + // We are using the new data. assertCached(table("refreshTable")) checkAnswer( table("refreshTable"), - table("src").unionAll(table("src")).collect()) + table("src").union(table("src")).collect()) // Drop the table and create it again. sql("DROP TABLE refreshTable") - createExternalTable("refreshTable", tempPath.toString, "parquet") + sparkSession.catalog.createExternalTable("refreshTable", tempPath.toString, "parquet") // It is not cached. assert(!isCached("refreshTable"), "refreshTable should not be cached.") - // Refresh the table. REFRESH TABLE command should not make a uncached + // Refresh the table. REFRESH command should not make a uncached // table cached. - sql("REFRESH TABLE refreshTable") + sql(s"REFRESH ${tempPath.toString}") checkAnswer( table("refreshTable"), - table("src").unionAll(table("src")).collect()) + table("src").union(table("src")).collect()) // It is not cached. assert(!isCached("refreshTable"), "refreshTable should not be cached.") @@ -205,13 +268,83 @@ class CachedTableSuite extends QueryTest with TestHiveSingleton { Utils.deleteRecursively(tempPath) } + test("Cache/Uncache Qualified Tables") { + withTempDatabase { db => + withTempView("cachedTable") { + sql(s"CREATE TABLE $db.cachedTable STORED AS PARQUET AS SELECT 1") + sql(s"CACHE TABLE $db.cachedTable") + assertCached(spark.table(s"$db.cachedTable")) + + activateDatabase(db) { + assertCached(spark.table("cachedTable")) + sql("UNCACHE TABLE cachedTable") + assert(!spark.catalog.isCached("cachedTable"), "Table 'cachedTable' should not be cached") + sql(s"CACHE TABLE cachedTable") + assert(spark.catalog.isCached("cachedTable"), "Table 'cachedTable' should be cached") + } + + sql(s"UNCACHE TABLE $db.cachedTable") + assert(!spark.catalog.isCached(s"$db.cachedTable"), + "Table 'cachedTable' should not be cached") + } + } + } + + test("Cache Table As Select - having database name") { + withTempDatabase { db => + withTempView("cachedTable") { + val e = intercept[ParseException] { + sql(s"CACHE TABLE $db.cachedTable AS SELECT 1") + }.getMessage + assert(e.contains("It is not allowed to add database prefix ") && + e.contains("to the table name in CACHE TABLE AS SELECT")) + } + } + } + test("SPARK-11246 cache parquet table") { sql("CREATE TABLE cachedTable STORED AS PARQUET AS SELECT 1") cacheTable("cachedTable") val sparkPlan = sql("SELECT * FROM cachedTable").queryExecution.sparkPlan - assert(sparkPlan.collect { case e: InMemoryColumnarTableScan => e }.size === 1) + assert(sparkPlan.collect { case e: InMemoryTableScanExec => e }.size === 1) sql("DROP TABLE cachedTable") } + + test("cache a table using CatalogFileIndex") { + withTable("test") { + sql("CREATE TABLE test(i int) PARTITIONED BY (p int) STORED AS parquet") + val tableMeta = spark.sharedState.externalCatalog.getTable("default", "test") + val catalogFileIndex = new CatalogFileIndex(spark, tableMeta, 0) + + val dataSchema = StructType(tableMeta.schema.filterNot { f => + tableMeta.partitionColumnNames.contains(f.name) + }) + val relation = HadoopFsRelation( + location = catalogFileIndex, + partitionSchema = tableMeta.partitionSchema, + dataSchema = dataSchema, + bucketSpec = None, + fileFormat = new ParquetFileFormat(), + options = Map.empty)(sparkSession = spark) + + val plan = LogicalRelation(relation, tableMeta) + spark.sharedState.cacheManager.cacheQuery(Dataset.ofRows(spark, plan)) + + assert(spark.sharedState.cacheManager.lookupCachedData(plan).isDefined) + + val sameCatalog = new CatalogFileIndex(spark, tableMeta, 0) + val sameRelation = HadoopFsRelation( + location = sameCatalog, + partitionSchema = tableMeta.partitionSchema, + dataSchema = dataSchema, + bucketSpec = None, + fileFormat = new ParquetFileFormat(), + options = Map.empty)(sparkSession = spark) + val samePlan = LogicalRelation(sameRelation, tableMeta) + + assert(spark.sharedState.cacheManager.lookupCachedData(samePlan).isDefined) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala index 34b2edb44b03..f262ef62be03 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala @@ -24,9 +24,7 @@ import org.apache.spark.SparkFunSuite /** * Verify that some classes load and that others are not found on the classpath. * - * - * This is used to detect classpath and shading conflict, especially between - * Spark's required Kryo version and that which can be found in some Hive versions. + * This is used to detect classpath and shading conflicts. */ class ClasspathDependenciesSuite extends SparkFunSuite { private val classloader = this.getClass.getClassLoader @@ -40,10 +38,6 @@ class ClasspathDependenciesSuite extends SparkFunSuite { classloader.loadClass(classname) } - private def assertLoads(classes: String*): Unit = { - classes.foreach(assertLoads) - } - private def findResource(classname: String): URL = { val resource = resourceName(classname) classloader.getResource(resource) @@ -63,17 +57,12 @@ class ClasspathDependenciesSuite extends SparkFunSuite { } } - private def assertClassNotFound(classes: String*): Unit = { - classes.foreach(assertClassNotFound) + test("shaded Protobuf") { + assertLoads("org.apache.hive.com.google.protobuf.ServiceException") } - private val KRYO = "com.esotericsoftware.kryo.Kryo" - - private val SPARK_HIVE = "org.apache.hive." - private val SPARK_SHADED = "org.spark-project.hive.shaded." - - test("shaded Protobuf") { - assertLoads(SPARK_SHADED + "com.google.protobuf.ServiceException") + test("shaded Kryo") { + assertLoads("org.apache.hive.com.esotericsoftware.kryo.Kryo") } test("hive-common") { @@ -86,25 +75,13 @@ class ClasspathDependenciesSuite extends SparkFunSuite { private val STD_INSTANTIATOR = "org.objenesis.strategy.StdInstantiatorStrategy" - test("unshaded kryo") { - assertLoads(KRYO, STD_INSTANTIATOR) - } - test("Forbidden Dependencies") { - assertClassNotFound( - SPARK_HIVE + KRYO, - SPARK_SHADED + KRYO, - "org.apache.hive." + KRYO, - "com.esotericsoftware.shaded." + STD_INSTANTIATOR, - SPARK_HIVE + "com.esotericsoftware.shaded." + STD_INSTANTIATOR, - "org.apache.hive.com.esotericsoftware.shaded." + STD_INSTANTIATOR - ) + assertClassNotFound("com.esotericsoftware.shaded." + STD_INSTANTIATOR) + assertClassNotFound("org.apache.hive.com.esotericsoftware.shaded." + STD_INSTANTIATOR) } test("parquet-hadoop-bundle") { - assertLoads( - "parquet.hadoop.ParquetOutputFormat", - "parquet.hadoop.ParquetInputFormat" - ) + assertLoads("parquet.hadoop.ParquetOutputFormat") + assertLoads("parquet.hadoop.ParquetInputFormat") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index cf737836939f..aa1973de7f67 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -19,18 +19,31 @@ package org.apache.spark.sql.hive import scala.util.Try -import org.scalatest.BeforeAndAfter +import org.scalatest.BeforeAndAfterEach +import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.{AnalysisException, QueryTest} +class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAndAfterEach { + import spark.implicits._ -class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter { - import hiveContext.implicits._ + override protected def beforeEach(): Unit = { + super.beforeEach() + if (spark.catalog.listTables().collect().map(_.name).contains("src")) { + spark.catalog.dropTempView("src") + } + Seq((1, "")).toDF("key", "value").createOrReplaceTempView("src") + Seq((1, 1, 1)).toDF("a", "a", "b").createOrReplaceTempView("dupAttributes") + } - before { - Seq((1, 1, 1)).toDF("a", "a", "b").registerTempTable("dupAttributes") + override protected def afterEach(): Unit = { + try { + spark.catalog.dropTempView("src") + spark.catalog.dropTempView("dupAttributes") + } finally { + super.afterEach() + } } positionTest("ambiguous attribute reference 1", @@ -117,12 +130,12 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd * @param token a unique token in the string that should be indicated by the exception */ def positionTest(name: String, query: String, token: String): Unit = { - def parseTree = - Try(quietly(HiveQl.dumpTree(HiveQl.getAst(query)))).getOrElse("") + def ast = spark.sessionState.sqlParser.parsePlan(query) + def parseTree = Try(quietly(ast.treeString)).getOrElse("") test(name) { val error = intercept[AnalysisException] { - quietly(hiveContext.sql(query)) + quietly(spark.sql(query)) } assert(!error.getMessage.contains("Seq(")) @@ -140,10 +153,7 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd 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)) - ) + fail(s"start not returned for error on token $token\n${ast.treeString}") } assert(expectedStart === actualStart, s"""Incorrect start position. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala new file mode 100644 index 000000000000..8a7423663f28 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} + + +class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEach { + + private var sc: SparkContext = null + private var hc: HiveContext = null + + override def beforeAll(): Unit = { + super.beforeAll() + sc = SparkContext.getOrCreate(new SparkConf().setMaster("local").setAppName("test")) + HiveUtils.newTemporaryConfiguration(useInMemoryDerby = true).foreach { case (k, v) => + sc.hadoopConfiguration.set(k, v) + } + hc = new HiveContext(sc) + } + + override def afterEach(): Unit = { + try { + hc.sharedState.cacheManager.clearCache() + hc.sessionState.catalog.reset() + } finally { + super.afterEach() + } + } + + override def afterAll(): Unit = { + try { + sc = null + hc = null + } finally { + super.afterAll() + } + } + + test("basic operations") { + val _hc = hc + import _hc.implicits._ + val df1 = (1 to 20).map { i => (i, i) }.toDF("a", "x") + val df2 = (1 to 100).map { i => (i, i % 10, i % 2 == 0) }.toDF("a", "b", "c") + .select($"a", $"b") + .filter($"a" > 10 && $"b" > 6 && $"c") + val df3 = df1.join(df2, "a") + val res = df3.collect() + val expected = Seq((18, 18, 8)).toDF("a", "x", "b").collect() + assert(res.toSeq == expected.toSeq) + df3.createOrReplaceTempView("mai_table") + val df4 = hc.table("mai_table") + val res2 = df4.collect() + assert(res2.toSeq == expected.toSeq) + } + + test("basic DDLs") { + val _hc = hc + import _hc.implicits._ + val databases = hc.sql("SHOW DATABASES").collect().map(_.getString(0)) + assert(databases.toSeq == Seq("default")) + hc.sql("CREATE DATABASE mee_db") + hc.sql("USE mee_db") + val databases2 = hc.sql("SHOW DATABASES").collect().map(_.getString(0)) + assert(databases2.toSet == Set("default", "mee_db")) + val df = (1 to 10).map { i => ("bob" + i.toString, i) }.toDF("name", "age") + df.createOrReplaceTempView("mee_table") + hc.sql("CREATE TABLE moo_table (name string, age int)") + hc.sql("INSERT INTO moo_table SELECT * FROM mee_table") + assert( + hc.sql("SELECT * FROM moo_table order by name").collect().toSeq == + df.collect().toSeq.sortBy(_.getString(0))) + val tables = hc.sql("SHOW TABLES IN mee_db").select("tableName").collect().map(_.getString(0)) + assert(tables.toSet == Set("moo_table", "mee_table")) + hc.sql("DROP TABLE moo_table") + hc.sql("DROP TABLE mee_table") + val tables2 = hc.sql("SHOW TABLES IN mee_db").select("tableName").collect().map(_.getString(0)) + assert(tables2.isEmpty) + hc.sql("USE default") + hc.sql("DROP DATABASE mee_db CASCADE") + val databases3 = hc.sql("SHOW DATABASES").collect().map(_.getString(0)) + assert(databases3.toSeq == Seq("default")) + } + +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala deleted file mode 100644 index 2e5cae415e54..000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import org.apache.spark.sql.{DataFrame, QueryTest} -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.scalatest.BeforeAndAfterAll - -// TODO ideally we should put the test suite into the package `sql`, as -// `hive` package is optional in compiling, however, `SQLContext.sql` doesn't -// support the `cube` or `rollup` yet. -class HiveDataFrameAnalyticsSuite extends QueryTest with TestHiveSingleton with BeforeAndAfterAll { - import hiveContext.implicits._ - import hiveContext.sql - - private var testData: DataFrame = _ - - override def beforeAll() { - testData = Seq((1, 2), (2, 4)).toDF("a", "b") - hiveContext.registerDataFrameAsTable(testData, "mytable") - } - - override def afterAll(): Unit = { - hiveContext.dropTempTable("mytable") - } - - test("rollup") { - checkAnswer( - testData.rollup($"a" + $"b", $"b").agg(sum($"a" - $"b")), - sql("select a + b, b, sum(a - b) from mytable group by a + b, b with rollup").collect() - ) - - checkAnswer( - testData.rollup("a", "b").agg(sum("b")), - sql("select a, b, sum(b) from mytable group by a, b with rollup").collect() - ) - } - - test("cube") { - checkAnswer( - testData.cube($"a" + $"b", $"b").agg(sum($"a" - $"b")), - sql("select a + b, b, sum(a - b) from mytable group by a + b, b with cube").collect() - ) - - checkAnswer( - testData.cube("a", "b").agg(sum("b")), - sql("select a, b, sum(b) from mytable group by a, b with cube").collect() - ) - } -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala index f621367eb553..cdc259d75b13 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameJoinSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.{Row, QueryTest} +import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.hive.test.TestHiveSingleton class HiveDataFrameJoinSuite extends QueryTest with TestHiveSingleton { - import hiveContext.implicits._ + import spark.implicits._ // We should move this into SQL package if we make case sensitivity configurable in SQL. test("join - self join auto resolve ambiguity with case insensitivity") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameWindowSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameWindowSuite.scala deleted file mode 100644 index 2c98f1c3cc49..000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameWindowSuite.scala +++ /dev/null @@ -1,259 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import org.apache.spark.sql.{Row, QueryTest} -import org.apache.spark.sql.expressions.Window -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.hive.test.TestHiveSingleton - -class HiveDataFrameWindowSuite extends QueryTest with TestHiveSingleton { - import hiveContext.implicits._ - import hiveContext.sql - - test("reuse window partitionBy") { - val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - val w = Window.partitionBy("key").orderBy("value") - - checkAnswer( - df.select( - lead("key", 1).over(w), - lead("value", 1).over(w)), - Row(1, "1") :: Row(2, "2") :: Row(null, null) :: Row(null, null) :: Nil) - } - - test("reuse window orderBy") { - val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - val w = Window.orderBy("value").partitionBy("key") - - checkAnswer( - df.select( - lead("key", 1).over(w), - lead("value", 1).over(w)), - Row(1, "1") :: Row(2, "2") :: Row(null, null) :: Row(null, null) :: Nil) - } - - test("lead") { - val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") - - checkAnswer( - df.select( - lead("value", 1).over(Window.partitionBy($"key").orderBy($"value"))), - sql( - """SELECT - | lead(value) OVER (PARTITION BY key ORDER BY value) - | FROM window_table""".stripMargin).collect()) - } - - test("lag") { - val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") - - checkAnswer( - df.select( - lag("value", 1).over(Window.partitionBy($"key").orderBy($"value"))), - sql( - """SELECT - | lag(value) OVER (PARTITION BY key ORDER BY value) - | FROM window_table""".stripMargin).collect()) - } - - test("lead with default value") { - val df = Seq((1, "1"), (1, "1"), (2, "2"), (1, "1"), - (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") - checkAnswer( - df.select( - lead("value", 2, "n/a").over(Window.partitionBy("key").orderBy("value"))), - sql( - """SELECT - | lead(value, 2, "n/a") OVER (PARTITION BY key ORDER BY value) - | FROM window_table""".stripMargin).collect()) - } - - test("lag with default value") { - val df = Seq((1, "1"), (1, "1"), (2, "2"), (1, "1"), - (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") - checkAnswer( - df.select( - lag("value", 2, "n/a").over(Window.partitionBy($"key").orderBy($"value"))), - sql( - """SELECT - | lag(value, 2, "n/a") OVER (PARTITION BY key ORDER BY value) - | FROM window_table""".stripMargin).collect()) - } - - test("rank functions in unspecific window") { - val df = Seq((1, "1"), (2, "2"), (1, "2"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") - checkAnswer( - df.select( - $"key", - max("key").over(Window.partitionBy("value").orderBy("key")), - min("key").over(Window.partitionBy("value").orderBy("key")), - mean("key").over(Window.partitionBy("value").orderBy("key")), - count("key").over(Window.partitionBy("value").orderBy("key")), - sum("key").over(Window.partitionBy("value").orderBy("key")), - ntile(2).over(Window.partitionBy("value").orderBy("key")), - rowNumber().over(Window.partitionBy("value").orderBy("key")), - denseRank().over(Window.partitionBy("value").orderBy("key")), - rank().over(Window.partitionBy("value").orderBy("key")), - cumeDist().over(Window.partitionBy("value").orderBy("key")), - percentRank().over(Window.partitionBy("value").orderBy("key"))), - sql( - s"""SELECT - |key, - |max(key) over (partition by value order by key), - |min(key) over (partition by value order by key), - |avg(key) over (partition by value order by key), - |count(key) over (partition by value order by key), - |sum(key) over (partition by value order by key), - |ntile(2) over (partition by value order by key), - |row_number() over (partition by value order by key), - |dense_rank() over (partition by value order by key), - |rank() over (partition by value order by key), - |cume_dist() over (partition by value order by key), - |percent_rank() over (partition by value order by key) - |FROM window_table""".stripMargin).collect()) - } - - test("aggregation and rows between") { - val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") - checkAnswer( - df.select( - avg("key").over(Window.partitionBy($"value").orderBy($"key").rowsBetween(-1, 2))), - sql( - """SELECT - | avg(key) OVER - | (PARTITION BY value ORDER BY key ROWS BETWEEN 1 preceding and 2 following) - | FROM window_table""".stripMargin).collect()) - } - - test("aggregation and range betweens") { - val df = Seq((1, "1"), (2, "2"), (1, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") - checkAnswer( - df.select( - avg("key").over(Window.partitionBy($"value").orderBy($"key").rangeBetween(-1, 1))), - sql( - """SELECT - | avg(key) OVER - | (PARTITION BY value ORDER BY key RANGE BETWEEN 1 preceding and 1 following) - | FROM window_table""".stripMargin).collect()) - } - - test("aggregation and rows betweens with unbounded") { - val df = Seq((1, "1"), (2, "2"), (2, "3"), (1, "3"), (3, "2"), (4, "3")).toDF("key", "value") - df.registerTempTable("window_table") - checkAnswer( - df.select( - $"key", - last("value").over( - Window.partitionBy($"value").orderBy($"key").rowsBetween(0, Long.MaxValue)), - last("value").over( - Window.partitionBy($"value").orderBy($"key").rowsBetween(Long.MinValue, 0)), - last("value").over(Window.partitionBy($"value").orderBy($"key").rowsBetween(-1, 3))), - sql( - """SELECT - | key, - | last_value(value) OVER - | (PARTITION BY value ORDER BY key ROWS between current row and unbounded following), - | last_value(value) OVER - | (PARTITION BY value ORDER BY key ROWS between unbounded preceding and current row), - | last_value(value) OVER - | (PARTITION BY value ORDER BY key ROWS between 1 preceding and 3 following) - | FROM window_table""".stripMargin).collect()) - } - - test("aggregation and range betweens with unbounded") { - val df = Seq((5, "1"), (5, "2"), (4, "2"), (6, "2"), (3, "1"), (2, "2")).toDF("key", "value") - df.registerTempTable("window_table") - checkAnswer( - df.select( - $"key", - last("value").over( - Window.partitionBy($"value").orderBy($"key").rangeBetween(-2, -1)) - .equalTo("2") - .as("last_v"), - avg("key").over(Window.partitionBy("value").orderBy("key").rangeBetween(Long.MinValue, 1)) - .as("avg_key1"), - avg("key").over(Window.partitionBy("value").orderBy("key").rangeBetween(0, Long.MaxValue)) - .as("avg_key2"), - avg("key").over(Window.partitionBy("value").orderBy("key").rangeBetween(-1, 0)) - .as("avg_key3") - ), - sql( - """SELECT - | key, - | last_value(value) OVER - | (PARTITION BY value ORDER BY key RANGE BETWEEN 2 preceding and 1 preceding) == "2", - | avg(key) OVER - | (PARTITION BY value ORDER BY key RANGE BETWEEN unbounded preceding and 1 following), - | avg(key) OVER - | (PARTITION BY value ORDER BY key RANGE BETWEEN current row and unbounded following), - | avg(key) OVER - | (PARTITION BY value ORDER BY key RANGE BETWEEN 1 preceding and current row) - | FROM window_table""".stripMargin).collect()) - } - - test("reverse sliding range frame") { - val df = Seq( - (1, "Thin", "Cell Phone", 6000), - (2, "Normal", "Tablet", 1500), - (3, "Mini", "Tablet", 5500), - (4, "Ultra thin", "Cell Phone", 5500), - (5, "Very thin", "Cell Phone", 6000), - (6, "Big", "Tablet", 2500), - (7, "Bendable", "Cell Phone", 3000), - (8, "Foldable", "Cell Phone", 3000), - (9, "Pro", "Tablet", 4500), - (10, "Pro2", "Tablet", 6500)). - toDF("id", "product", "category", "revenue") - val window = Window. - partitionBy($"category"). - orderBy($"revenue".desc). - rangeBetween(-2000L, 1000L) - checkAnswer( - df.select( - $"id", - avg($"revenue").over(window).cast("int")), - Row(1, 5833) :: Row(2, 2000) :: Row(3, 5500) :: - Row(4, 5833) :: Row(5, 5833) :: Row(6, 2833) :: - Row(7, 3000) :: Row(8, 3000) :: Row(9, 5500) :: - Row(10, 6000) :: Nil) - } - - // This is here to illustrate the fact that reverse order also reverses offsets. - test("reverse unbounded range frame") { - val df = Seq(1, 2, 4, 3, 2, 1). - map(Tuple1.apply). - toDF("value") - val window = Window.orderBy($"value".desc) - checkAnswer( - df.select( - $"value", - sum($"value").over(window.rangeBetween(Long.MinValue, 1)), - sum($"value").over(window.rangeBetween(1, Long.MaxValue))), - Row(1, 13, null) :: Row(2, 13, 2) :: Row(4, 7, 9) :: - Row(3, 11, 6) :: Row(2, 13, 2) :: Row(1, 13, null) :: Nil) - - } -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala new file mode 100644 index 000000000000..d43534d5914d --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.SparkConf +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.execution.command.DDLUtils +import org.apache.spark.sql.types.StructType + +/** + * Test suite for the [[HiveExternalCatalog]]. + */ +class HiveExternalCatalogSuite extends ExternalCatalogSuite { + + private val externalCatalog: HiveExternalCatalog = { + val catalog = new HiveExternalCatalog(new SparkConf, new Configuration) + catalog.client.reset() + catalog + } + + protected override val utils: CatalogTestUtils = new CatalogTestUtils { + override val tableInputFormat: String = "org.apache.hadoop.mapred.SequenceFileInputFormat" + override val tableOutputFormat: String = "org.apache.hadoop.mapred.SequenceFileOutputFormat" + override def newEmptyCatalog(): ExternalCatalog = externalCatalog + override val defaultProvider: String = "hive" + } + + protected override def resetState(): Unit = { + externalCatalog.client.reset() + } + + import utils._ + + test("SPARK-18647: do not put provider in table properties for Hive serde table") { + val catalog = newBasicCatalog() + val hiveTable = CatalogTable( + identifier = TableIdentifier("hive_tbl", Some("db1")), + tableType = CatalogTableType.MANAGED, + storage = storageFormat, + schema = new StructType().add("col1", "int").add("col2", "string"), + provider = Some("hive")) + catalog.createTable(hiveTable, ignoreIfExists = false) + + val rawTable = externalCatalog.client.getTable("db1", "hive_tbl") + assert(!rawTable.properties.contains(HiveExternalCatalog.DATASOURCE_PROVIDER)) + assert(DDLUtils.isHiveTable(externalCatalog.getTable("db1", "hive_tbl"))) + } + + Seq("parquet", "hive").foreach { format => + test(s"Partition columns should be put at the end of table schema for the format $format") { + val catalog = newBasicCatalog() + val newSchema = new StructType() + .add("col1", "int") + .add("col2", "string") + .add("partCol1", "int") + .add("partCol2", "string") + val table = CatalogTable( + identifier = TableIdentifier("tbl", Some("db1")), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType() + .add("col1", "int") + .add("partCol1", "int") + .add("partCol2", "string") + .add("col2", "string"), + provider = Some(format), + partitionColumnNames = Seq("partCol1", "partCol2")) + catalog.createTable(table, ignoreIfExists = false) + + val restoredTable = externalCatalog.getTable("db1", "tbl") + assert(restoredTable.schema == newSchema) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala new file mode 100644 index 000000000000..305f5b533d59 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.nio.file.Files + +import org.apache.spark.TestUtils +import org.apache.spark.sql.{QueryTest, Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.util.Utils + +/** + * Test HiveExternalCatalog backward compatibility. + * + * Note that, this test suite will automatically download spark binary packages of different + * versions to a local directory `/tmp/spark-test`. If there is already a spark folder with + * expected version under this local directory, e.g. `/tmp/spark-test/spark-2.0.3`, we will skip the + * downloading for this spark version. + */ +class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { + private val wareHousePath = Utils.createTempDir(namePrefix = "warehouse") + private val tmpDataDir = Utils.createTempDir(namePrefix = "test-data") + // For local test, you can set `sparkTestingDir` to a static value like `/tmp/test-spark`, to + // avoid downloading Spark of different versions in each run. + private val sparkTestingDir = Utils.createTempDir(namePrefix = "test-spark") + private val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + + override def afterAll(): Unit = { + Utils.deleteRecursively(wareHousePath) + Utils.deleteRecursively(tmpDataDir) + Utils.deleteRecursively(sparkTestingDir) + super.afterAll() + } + + private def downloadSpark(version: String): Unit = { + import scala.sys.process._ + + val url = s"https://d3kbcqa49mib13.cloudfront.net/spark-$version-bin-hadoop2.7.tgz" + + Seq("wget", url, "-q", "-P", sparkTestingDir.getCanonicalPath).! + + val downloaded = new File(sparkTestingDir, s"spark-$version-bin-hadoop2.7.tgz").getCanonicalPath + val targetDir = new File(sparkTestingDir, s"spark-$version").getCanonicalPath + + Seq("mkdir", targetDir).! + + Seq("tar", "-xzf", downloaded, "-C", targetDir, "--strip-components=1").! + + Seq("rm", downloaded).! + } + + private def genDataDir(name: String): String = { + new File(tmpDataDir, name).getCanonicalPath + } + + override def beforeAll(): Unit = { + super.beforeAll() + + val tempPyFile = File.createTempFile("test", ".py") + Files.write(tempPyFile.toPath, + s""" + |from pyspark.sql import SparkSession + | + |spark = SparkSession.builder.enableHiveSupport().getOrCreate() + |version_index = spark.conf.get("spark.sql.test.version.index", None) + | + |spark.sql("create table data_source_tbl_{} using json as select 1 i".format(version_index)) + | + |spark.sql("create table hive_compatible_data_source_tbl_" + version_index + \\ + | " using parquet as select 1 i") + | + |json_file = "${genDataDir("json_")}" + str(version_index) + |spark.range(1, 2).selectExpr("cast(id as int) as i").write.json(json_file) + |spark.sql("create table external_data_source_tbl_" + version_index + \\ + | "(i int) using json options (path '{}')".format(json_file)) + | + |parquet_file = "${genDataDir("parquet_")}" + str(version_index) + |spark.range(1, 2).selectExpr("cast(id as int) as i").write.parquet(parquet_file) + |spark.sql("create table hive_compatible_external_data_source_tbl_" + version_index + \\ + | "(i int) using parquet options (path '{}')".format(parquet_file)) + | + |json_file2 = "${genDataDir("json2_")}" + str(version_index) + |spark.range(1, 2).selectExpr("cast(id as int) as i").write.json(json_file2) + |spark.sql("create table external_table_without_schema_" + version_index + \\ + | " using json options (path '{}')".format(json_file2)) + | + |spark.sql("create view v_{} as select 1 i".format(version_index)) + """.stripMargin.getBytes("utf8")) + + PROCESS_TABLES.testingVersions.zipWithIndex.foreach { case (version, index) => + val sparkHome = new File(sparkTestingDir, s"spark-$version") + if (!sparkHome.exists()) { + downloadSpark(version) + } + + val args = Seq( + "--name", "prepare testing tables", + "--master", "local[2]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--conf", s"spark.sql.warehouse.dir=${wareHousePath.getCanonicalPath}", + "--conf", s"spark.sql.test.version.index=$index", + "--driver-java-options", s"-Dderby.system.home=${wareHousePath.getCanonicalPath}", + tempPyFile.getCanonicalPath) + runSparkSubmit(args, Some(sparkHome.getCanonicalPath)) + } + + tempPyFile.delete() + } + + test("backward compatibility") { + val args = Seq( + "--class", PROCESS_TABLES.getClass.getName.stripSuffix("$"), + "--name", "HiveExternalCatalog backward compatibility test", + "--master", "local[2]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--conf", s"spark.sql.warehouse.dir=${wareHousePath.getCanonicalPath}", + "--driver-java-options", s"-Dderby.system.home=${wareHousePath.getCanonicalPath}", + unusedJar.toString) + runSparkSubmit(args) + } +} + +object PROCESS_TABLES extends QueryTest with SQLTestUtils { + // Tests the latest version of every release line. + val testingVersions = Seq("2.0.2", "2.1.1", "2.2.0") + + protected var spark: SparkSession = _ + + def main(args: Array[String]): Unit = { + val session = SparkSession.builder() + .enableHiveSupport() + .getOrCreate() + spark = session + + testingVersions.indices.foreach { index => + Seq( + s"data_source_tbl_$index", + s"hive_compatible_data_source_tbl_$index", + s"external_data_source_tbl_$index", + s"hive_compatible_external_data_source_tbl_$index", + s"external_table_without_schema_$index").foreach { tbl => + val tableMeta = spark.sharedState.externalCatalog.getTable("default", tbl) + + // make sure we can insert and query these tables. + session.sql(s"insert into $tbl select 2") + checkAnswer(session.sql(s"select * from $tbl"), Row(1) :: Row(2) :: Nil) + checkAnswer(session.sql(s"select i from $tbl where i > 1"), Row(2)) + + // make sure we can rename table. + val newName = tbl + "_renamed" + sql(s"ALTER TABLE $tbl RENAME TO $newName") + val readBack = spark.sharedState.externalCatalog.getTable("default", newName) + + val actualTableLocation = readBack.storage.locationUri.get.getPath + val expectedLocation = if (tableMeta.tableType == CatalogTableType.EXTERNAL) { + tableMeta.storage.locationUri.get.getPath + } else { + spark.sessionState.catalog.defaultTablePath(TableIdentifier(newName, None)).getPath + } + assert(actualTableLocation == expectedLocation) + + // make sure we can alter table location. + withTempDir { dir => + val path = dir.toURI.toString.stripSuffix("/") + sql(s"ALTER TABLE ${tbl}_renamed SET LOCATION '$path'") + val readBack = spark.sharedState.externalCatalog.getTable("default", tbl + "_renamed") + val actualTableLocation = readBack.storage.locationUri.get.getPath + val expected = dir.toURI.getPath.stripSuffix("/") + assert(actualTableLocation == expected) + } + } + + // test permanent view + checkAnswer(sql(s"select i from v_$index"), Row(1)) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalSessionCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalSessionCatalogSuite.scala new file mode 100644 index 000000000000..285f35b0b0ea --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalSessionCatalogSuite.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.spark.sql.catalyst.catalog.{CatalogTestUtils, ExternalCatalog, SessionCatalogSuite} +import org.apache.spark.sql.hive.test.TestHiveSingleton + +class HiveExternalSessionCatalogSuite extends SessionCatalogSuite with TestHiveSingleton { + + protected override val isHiveExternalCatalog = true + + private val externalCatalog = { + val catalog = spark.sharedState.externalCatalog + catalog.asInstanceOf[HiveExternalCatalog].client.reset() + catalog + } + + protected val utils = new CatalogTestUtils { + override val tableInputFormat: String = "org.apache.hadoop.mapred.SequenceFileInputFormat" + override val tableOutputFormat: String = + "org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat" + override val defaultProvider: String = "hive" + override def newEmptyCatalog(): ExternalCatalog = externalCatalog + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala index 8bb9058cd74e..c300660458fd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala @@ -28,13 +28,19 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn import org.apache.hadoop.io.LongWritable import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.catalyst.util.{MapData, GenericArrayData, ArrayBasedMapData} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData} import org.apache.spark.sql.types._ -import org.apache.spark.sql.Row class HiveInspectorSuite extends SparkFunSuite with HiveInspectors { + + def unwrap(data: Any, oi: ObjectInspector): Any = { + val unwrapper = unwrapperFor(oi) + unwrapper(data) + } + test("Test wrap SettableStructObjectInspector") { val udaf = new UDAFPercentile.PercentileLongEvaluator() udaf.init() @@ -75,6 +81,7 @@ class HiveInspectorSuite extends SparkFunSuite with HiveInspectors { val data = Literal(true) :: + Literal(null) :: Literal(0.asInstanceOf[Byte]) :: Literal(0.asInstanceOf[Short]) :: Literal(0) :: @@ -83,7 +90,7 @@ class HiveInspectorSuite extends SparkFunSuite with HiveInspectors { Literal(0.asInstanceOf[Double]) :: Literal("0") :: Literal(java.sql.Date.valueOf("2014-09-23")) :: - Literal(Decimal(BigDecimal(123.123))) :: + Literal(Decimal(BigDecimal("123.123"))) :: Literal(new java.sql.Timestamp(123123)) :: Literal(Array[Byte](1, 2, 3)) :: Literal.create(Seq[Int](1, 2, 3), ArrayType(IntegerType)) :: diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala new file mode 100644 index 000000000000..0c28a1b609bb --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetadataCacheSuite.scala @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils + +/** + * Test suite to handle metadata cache related. + */ +class HiveMetadataCacheSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + + test("SPARK-16337 temporary view refresh") { + withTempView("view_refresh") { + withTable("view_table") { + // Create a Parquet directory + spark.range(start = 0, end = 100, step = 1, numPartitions = 3) + .write.saveAsTable("view_table") + + // Read the table in + spark.table("view_table").filter("id > -1").createOrReplaceTempView("view_refresh") + assert(sql("select count(*) from view_refresh").first().getLong(0) == 100) + + // Delete a file using the Hadoop file system interface since the path returned by + // inputFiles is not recognizable by Java IO. + val p = new Path(spark.table("view_table").inputFiles.head) + assert(p.getFileSystem(hiveContext.sessionState.newHadoopConf()).delete(p, false)) + + // Read it again and now we should see a FileNotFoundException + val e = intercept[SparkException] { + sql("select count(*) from view_refresh").first() + } + assert(e.getMessage.contains("FileNotFoundException")) + assert(e.getMessage.contains("REFRESH")) + + // Refresh and we should be able to read it again. + spark.catalog.refreshTable("view_refresh") + val newCount = sql("select count(*) from view_refresh").first().getLong(0) + assert(newCount > 0 && newCount < 100) + } + } + } + + def testCaching(pruningEnabled: Boolean): Unit = { + test(s"partitioned table is cached when partition pruning is $pruningEnabled") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> pruningEnabled.toString) { + withTable("test") { + withTempDir { dir => + spark.range(5).selectExpr("id", "id as f1", "id as f2").write + .partitionBy("f1", "f2") + .mode("overwrite") + .parquet(dir.getAbsolutePath) + + spark.sql(s""" + |create external table test (id long) + |partitioned by (f1 int, f2 int) + |stored as parquet + |location "${dir.toURI}"""".stripMargin) + spark.sql("msck repair table test") + + val df = spark.sql("select * from test") + assert(sql("select * from test").count() == 5) + + def deleteRandomFile(): Unit = { + val p = new Path(spark.table("test").inputFiles.head) + assert(p.getFileSystem(hiveContext.sessionState.newHadoopConf()).delete(p, true)) + } + + // Delete a file, then assert that we tried to read it. This means the table was cached. + deleteRandomFile() + val e = intercept[SparkException] { + sql("select * from test").count() + } + assert(e.getMessage.contains("FileNotFoundException")) + + // Test refreshing the cache. + spark.catalog.refreshTable("test") + assert(sql("select * from test").count() == 4) + assert(spark.table("test").inputFiles.length == 4) + + // Test refresh by path separately since it goes through different code paths than + // refreshTable does. + deleteRandomFile() + spark.catalog.cacheTable("test") + spark.catalog.refreshByPath("/some-invalid-path") // no-op + val e2 = intercept[SparkException] { + sql("select * from test").count() + } + assert(e2.getMessage.contains("FileNotFoundException")) + spark.catalog.refreshByPath(dir.getAbsolutePath) + assert(sql("select * from test").count() == 3) + } + } + } + } + } + + for (pruningEnabled <- Seq(true, false)) { + testCaching(pruningEnabled) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index d63f3d399652..18137e7ea1d6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -17,36 +17,123 @@ package org.apache.spark.sql.hive -import java.io.File - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.hive.client.{ExternalTable, ManagedTable} +import org.apache.spark.sql.{QueryTest, Row, SaveMode} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{ExamplePointUDT, SQLTestUtils} -import org.apache.spark.sql.types.{DecimalType, StringType, StructType} -import org.apache.spark.sql.{SQLConf, QueryTest, Row, SaveMode} +import org.apache.spark.sql.types._ -class HiveMetastoreCatalogSuite extends SparkFunSuite with TestHiveSingleton { - import hiveContext.implicits._ +class HiveMetastoreCatalogSuite extends TestHiveSingleton with SQLTestUtils { + import spark.implicits._ test("struct field should accept underscore in sub-column name") { val hiveTypeStr = "struct" - val dateType = HiveMetastoreTypes.toDataType(hiveTypeStr) - assert(dateType.isInstanceOf[StructType]) + val dataType = CatalystSqlParser.parseDataType(hiveTypeStr) + assert(dataType.isInstanceOf[StructType]) } test("udt to metastore type conversion") { val udt = new ExamplePointUDT - assertResult(HiveMetastoreTypes.toMetastoreType(udt.sqlType)) { - HiveMetastoreTypes.toMetastoreType(udt) + assertResult(udt.sqlType.catalogString) { + udt.catalogString } } test("duplicated metastore relations") { - val df = hiveContext.sql("SELECT * FROM src") + val df = spark.sql("SELECT * FROM src") logInfo(df.queryExecution.toString) df.as('a).join(df.as('b), $"a.key" === $"b.key") } + + test("should not truncate struct type catalog string") { + def field(n: Int): StructField = { + StructField("col" + n, StringType) + } + val dataType = StructType((1 to 100).map(field)) + assert(CatalystSqlParser.parseDataType(dataType.catalogString) == dataType) + } + + test("view relation") { + withView("vw1") { + spark.sql("create view vw1 as select 1 as id") + val plan = spark.sql("select id from vw1").queryExecution.analyzed + val aliases = plan.collect { + case x @ SubqueryAlias("vw1", _) => x + } + assert(aliases.size == 1) + } + } + + test("Validate catalog metadata for supported data types") { + withTable("t") { + sql( + """ + |CREATE TABLE t ( + |c1 boolean, + |c2 tinyint, + |c3 smallint, + |c4 short, + |c5 bigint, + |c6 long, + |c7 float, + |c8 double, + |c9 date, + |c10 timestamp, + |c11 string, + |c12 char(10), + |c13 varchar(10), + |c14 binary, + |c15 decimal, + |c16 decimal(10), + |c17 decimal(10,2), + |c18 array, + |c19 array, + |c20 array, + |c21 map, + |c22 map, + |c23 struct, + |c24 struct + |) + """.stripMargin) + + val schema = hiveClient.getTable("default", "t").schema + val expectedSchema = new StructType() + .add("c1", "boolean") + .add("c2", "tinyint") + .add("c3", "smallint") + .add("c4", "short") + .add("c5", "bigint") + .add("c6", "long") + .add("c7", "float") + .add("c8", "double") + .add("c9", "date") + .add("c10", "timestamp") + .add("c11", "string") + .add("c12", "string", true, + new MetadataBuilder().putString(HIVE_TYPE_STRING, "char(10)").build()) + .add("c13", "string", true, + new MetadataBuilder().putString(HIVE_TYPE_STRING, "varchar(10)").build()) + .add("c14", "binary") + .add("c15", "decimal") + .add("c16", "decimal(10)") + .add("c17", "decimal(10,2)") + .add("c18", "array") + .add("c19", "array") + .add("c20", "array", true, + new MetadataBuilder().putString(HIVE_TYPE_STRING, "array").build()) + .add("c21", "map") + .add("c22", "map", true, + new MetadataBuilder().putString(HIVE_TYPE_STRING, "map").build()) + .add("c23", "struct") + .add("c24", "struct", true, + new MetadataBuilder().putString(HIVE_TYPE_STRING, "struct").build()) + assert(schema == expectedSchema) + } + } } class DataSourceWithHiveMetastoreCatalogSuite @@ -60,17 +147,17 @@ class DataSourceWithHiveMetastoreCatalogSuite ).coalesce(1) Seq( - "parquet" -> ( + "parquet" -> (( "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat", "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat", "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe" - ), + )), - "orc" -> ( + "orc" -> (( "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat", "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat", "org.apache.hadoop.hive.ql.io.orc.OrcSerde" - ) + )) ).foreach { case (provider, (inputFormat, outputFormat, serde)) => test(s"Persist non-partitioned $provider relation into metastore as managed table") { withTable("t") { @@ -82,20 +169,20 @@ class DataSourceWithHiveMetastoreCatalogSuite .saveAsTable("t") } - val hiveTable = catalog.client.getTable("default", "t") - assert(hiveTable.inputFormat === Some(inputFormat)) - assert(hiveTable.outputFormat === Some(outputFormat)) - assert(hiveTable.serde === Some(serde)) + val hiveTable = sessionState.catalog.getTableMetadata(TableIdentifier("t", Some("default"))) + assert(hiveTable.storage.inputFormat === Some(inputFormat)) + assert(hiveTable.storage.outputFormat === Some(outputFormat)) + assert(hiveTable.storage.serde === Some(serde)) - assert(!hiveTable.isPartitioned) - assert(hiveTable.tableType === ManagedTable) + assert(hiveTable.partitionColumnNames.isEmpty) + assert(hiveTable.tableType === CatalogTableType.MANAGED) val columns = hiveTable.schema assert(columns.map(_.name) === Seq("d1", "d2")) - assert(columns.map(_.hiveType) === Seq("decimal(10,3)", "string")) + assert(columns.map(_.dataType) === Seq(DecimalType(10, 3), StringType)) checkAnswer(table("t"), testDF) - assert(runSqlHive("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) + assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) } } @@ -113,20 +200,22 @@ class DataSourceWithHiveMetastoreCatalogSuite .saveAsTable("t") } - val hiveTable = catalog.client.getTable("default", "t") - assert(hiveTable.inputFormat === Some(inputFormat)) - assert(hiveTable.outputFormat === Some(outputFormat)) - assert(hiveTable.serde === Some(serde)) + val hiveTable = + sessionState.catalog.getTableMetadata(TableIdentifier("t", Some("default"))) + assert(hiveTable.storage.inputFormat === Some(inputFormat)) + assert(hiveTable.storage.outputFormat === Some(outputFormat)) + assert(hiveTable.storage.serde === Some(serde)) - assert(hiveTable.tableType === ExternalTable) - assert(hiveTable.location.get === path.toURI.toString.stripSuffix(File.separator)) + assert(hiveTable.tableType === CatalogTableType.EXTERNAL) + assert(hiveTable.storage.locationUri === Some(makeQualifiedPath(dir.getAbsolutePath))) val columns = hiveTable.schema assert(columns.map(_.name) === Seq("d1", "d2")) - assert(columns.map(_.hiveType) === Seq("decimal(10,3)", "string")) + assert(columns.map(_.dataType) === Seq(DecimalType(10, 3), StringType)) checkAnswer(table("t"), testDF) - assert(runSqlHive("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) + assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === + Seq("1.1\t1", "2.1\t2")) } } } @@ -134,31 +223,30 @@ class DataSourceWithHiveMetastoreCatalogSuite test(s"Persist non-partitioned $provider relation into metastore as managed table using CTAS") { withTempPath { dir => withTable("t") { - val path = dir.getCanonicalPath - sql( s"""CREATE TABLE t USING $provider - |OPTIONS (path '$path') + |OPTIONS (path '${dir.toURI}') |AS SELECT 1 AS d1, "val_1" AS d2 """.stripMargin) - val hiveTable = catalog.client.getTable("default", "t") - assert(hiveTable.inputFormat === Some(inputFormat)) - assert(hiveTable.outputFormat === Some(outputFormat)) - assert(hiveTable.serde === Some(serde)) + val hiveTable = + sessionState.catalog.getTableMetadata(TableIdentifier("t", Some("default"))) + assert(hiveTable.storage.inputFormat === Some(inputFormat)) + assert(hiveTable.storage.outputFormat === Some(outputFormat)) + assert(hiveTable.storage.serde === Some(serde)) - assert(hiveTable.isPartitioned === false) - assert(hiveTable.tableType === ExternalTable) - assert(hiveTable.partitionColumns.length === 0) + assert(hiveTable.partitionColumnNames.isEmpty) + assert(hiveTable.tableType === CatalogTableType.EXTERNAL) val columns = hiveTable.schema assert(columns.map(_.name) === Seq("d1", "d2")) - assert(columns.map(_.hiveType) === Seq("int", "string")) + assert(columns.map(_.dataType) === Seq(IntegerType, StringType)) checkAnswer(table("t"), Row(1, "val_1")) - assert(runSqlHive("SELECT * FROM t") === Seq("1\tval_1")) + assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === Seq("1\tval_1")) } } } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala index 5596ec6882ea..09c15473b21c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.hive +import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.execution.datasources.parquet.ParquetTest import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.{QueryTest, Row} case class Cases(lower: String, UPPER: String) @@ -51,8 +51,8 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton test("Converting Hive to Parquet Table via saveAsParquetFile") { withTempPath { dir => sql("SELECT * FROM src").write.parquet(dir.getCanonicalPath) - hiveContext.read.parquet(dir.getCanonicalPath).registerTempTable("p") - withTempTable("p") { + spark.read.parquet(dir.getCanonicalPath).createOrReplaceTempView("p") + withTempView("p") { checkAnswer( sql("SELECT * FROM src ORDER BY key"), sql("SELECT * from p ORDER BY key").collect().toSeq) @@ -61,11 +61,12 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton } test("INSERT OVERWRITE TABLE Parquet table") { - withParquetTable((1 to 10).map(i => (i, s"val_$i")), "t") { + // Don't run with vectorized: currently relies on UnsafeRow. + withParquetTable((1 to 10).map(i => (i, s"val_$i")), "t", false) { withTempPath { file => sql("SELECT * FROM t LIMIT 1").write.parquet(file.getCanonicalPath) - hiveContext.read.parquet(file.getCanonicalPath).registerTempTable("p") - withTempTable("p") { + spark.read.parquet(file.getCanonicalPath).createOrReplaceTempView("p") + withTempView("p") { // let's do three overwrites for good measure sql("INSERT OVERWRITE TABLE p SELECT * FROM t") sql("INSERT OVERWRITE TABLE p SELECT * FROM t") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala deleted file mode 100644 index 528a7398b10d..000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import org.apache.hadoop.hive.serde.serdeConstants -import org.scalatest.BeforeAndAfterAll - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.hive.client.{ExternalTable, HiveColumn, HiveTable, ManagedTable} - - -class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { - private def extractTableDesc(sql: String): (HiveTable, Boolean) = { - HiveQl.createPlan(sql).collect { - case CreateTableAsSelect(desc, child, allowExisting) => (desc, allowExisting) - }.head - } - - test("Test CTAS #1") { - val s1 = - """CREATE EXTERNAL TABLE IF NOT EXISTS mydb.page_view - |(viewTime INT, - |userid BIGINT, - |page_url STRING, - |referrer_url STRING, - |ip STRING COMMENT 'IP Address of the User', - |country STRING COMMENT 'country of origination') - |COMMENT 'This is the staging page view table' - |PARTITIONED BY (dt STRING COMMENT 'date type', hour STRING COMMENT 'hour of the day') - |ROW FORMAT DELIMITED FIELDS TERMINATED BY '\054' STORED AS RCFILE - |LOCATION '/user/external/page_view' - |TBLPROPERTIES ('p1'='v1', 'p2'='v2') - |AS SELECT * FROM src""".stripMargin - - val (desc, exists) = extractTableDesc(s1) - assert(exists == true) - assert(desc.specifiedDatabase == Some("mydb")) - assert(desc.name == "page_view") - assert(desc.tableType == ExternalTable) - assert(desc.location == Some("/user/external/page_view")) - assert(desc.schema == - HiveColumn("viewtime", "int", null) :: - HiveColumn("userid", "bigint", null) :: - HiveColumn("page_url", "string", null) :: - HiveColumn("referrer_url", "string", null) :: - HiveColumn("ip", "string", "IP Address of the User") :: - HiveColumn("country", "string", "country of origination") :: Nil) - // TODO will be SQLText - assert(desc.viewText == Option("This is the staging page view table")) - assert(desc.partitionColumns == - HiveColumn("dt", "string", "date type") :: - HiveColumn("hour", "string", "hour of the day") :: Nil) - assert(desc.serdeProperties == - Map((serdeConstants.SERIALIZATION_FORMAT, "\054"), (serdeConstants.FIELD_DELIM, "\054"))) - assert(desc.inputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) - assert(desc.outputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) - assert(desc.serde == Option("org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe")) - assert(desc.properties == Map(("p1", "v1"), ("p2", "v2"))) - } - - test("Test CTAS #2") { - val s2 = - """CREATE EXTERNAL TABLE IF NOT EXISTS mydb.page_view - |(viewTime INT, - |userid BIGINT, - |page_url STRING, - |referrer_url STRING, - |ip STRING COMMENT 'IP Address of the User', - |country STRING COMMENT 'country of origination') - |COMMENT 'This is the staging page view table' - |PARTITIONED BY (dt STRING COMMENT 'date type', hour STRING COMMENT 'hour of the day') - |ROW FORMAT SERDE 'parquet.hive.serde.ParquetHiveSerDe' - | STORED AS - | INPUTFORMAT 'parquet.hive.DeprecatedParquetInputFormat' - | OUTPUTFORMAT 'parquet.hive.DeprecatedParquetOutputFormat' - |LOCATION '/user/external/page_view' - |TBLPROPERTIES ('p1'='v1', 'p2'='v2') - |AS SELECT * FROM src""".stripMargin - - val (desc, exists) = extractTableDesc(s2) - assert(exists == true) - assert(desc.specifiedDatabase == Some("mydb")) - assert(desc.name == "page_view") - assert(desc.tableType == ExternalTable) - assert(desc.location == Some("/user/external/page_view")) - assert(desc.schema == - HiveColumn("viewtime", "int", null) :: - HiveColumn("userid", "bigint", null) :: - HiveColumn("page_url", "string", null) :: - HiveColumn("referrer_url", "string", null) :: - HiveColumn("ip", "string", "IP Address of the User") :: - HiveColumn("country", "string", "country of origination") :: Nil) - // TODO will be SQLText - assert(desc.viewText == Option("This is the staging page view table")) - assert(desc.partitionColumns == - HiveColumn("dt", "string", "date type") :: - HiveColumn("hour", "string", "hour of the day") :: Nil) - assert(desc.serdeProperties == Map()) - assert(desc.inputFormat == Option("parquet.hive.DeprecatedParquetInputFormat")) - assert(desc.outputFormat == Option("parquet.hive.DeprecatedParquetOutputFormat")) - assert(desc.serde == Option("parquet.hive.serde.ParquetHiveSerDe")) - assert(desc.properties == Map(("p1", "v1"), ("p2", "v2"))) - } - - test("Test CTAS #3") { - val s3 = """CREATE TABLE page_view AS SELECT * FROM src""" - val (desc, exists) = extractTableDesc(s3) - assert(exists == false) - assert(desc.specifiedDatabase == None) - assert(desc.name == "page_view") - assert(desc.tableType == ManagedTable) - assert(desc.location == None) - assert(desc.schema == Seq.empty[HiveColumn]) - assert(desc.viewText == None) // TODO will be SQLText - assert(desc.serdeProperties == Map()) - assert(desc.inputFormat == Option("org.apache.hadoop.mapred.TextInputFormat")) - assert(desc.outputFormat == Option("org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat")) - assert(desc.serde.isEmpty) - assert(desc.properties == Map()) - } - - test("Test CTAS #4") { - val s4 = - """CREATE TABLE page_view - |STORED BY 'storage.handler.class.name' AS SELECT * FROM src""".stripMargin - intercept[AnalysisException] { - extractTableDesc(s4) - } - } - - test("Test CTAS #5") { - val s5 = """CREATE TABLE ctas2 - | ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" - | WITH SERDEPROPERTIES("serde_p1"="p1","serde_p2"="p2") - | STORED AS RCFile - | TBLPROPERTIES("tbl_p1"="p11", "tbl_p2"="p22") - | AS - | SELECT key, value - | FROM src - | ORDER BY key, value""".stripMargin - val (desc, exists) = extractTableDesc(s5) - assert(exists == false) - assert(desc.specifiedDatabase == None) - assert(desc.name == "ctas2") - assert(desc.tableType == ManagedTable) - assert(desc.location == None) - assert(desc.schema == Seq.empty[HiveColumn]) - assert(desc.viewText == None) // TODO will be SQLText - assert(desc.serdeProperties == Map(("serde_p1" -> "p1"), ("serde_p2" -> "p2"))) - assert(desc.inputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) - assert(desc.outputFormat == Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) - assert(desc.serde == Option("org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe")) - assert(desc.properties == Map(("tbl_p1" -> "p11"), ("tbl_p2" -> "p22"))) - } - - test("Invalid interval term should throw AnalysisException") { - def assertError(sql: String, errorMessage: String): Unit = { - val e = intercept[AnalysisException] { - HiveQl.parseSql(sql) - } - assert(e.getMessage.contains(errorMessage)) - } - assertError("select interval '42-32' year to month", - "month 32 outside range [0, 11]") - assertError("select interval '5 49:12:15' day to second", - "hour 49 outside range [0, 23]") - assertError("select interval '.1111111111' second", - "nanosecond 1111111111 outside range") - } -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala new file mode 100644 index 000000000000..f2d27671094d --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala @@ -0,0 +1,323 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 scala.util.Random + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.execution.datasources.FileStatusCache +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode.{Value => InferenceMode, _} +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types._ + +class HiveSchemaInferenceSuite + extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach { + + import HiveSchemaInferenceSuite._ + import HiveExternalCatalog.DATASOURCE_SCHEMA_PREFIX + + override def beforeEach(): Unit = { + super.beforeEach() + FileStatusCache.resetForTesting() + } + + override def afterEach(): Unit = { + super.afterEach() + spark.sessionState.catalog.invalidateAllCachedTables() + FileStatusCache.resetForTesting() + } + + private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog] + private val client = externalCatalog.client + + // Return a copy of the given schema with all field names converted to lower case. + private def lowerCaseSchema(schema: StructType): StructType = { + StructType(schema.map(f => f.copy(name = f.name.toLowerCase))) + } + + // Create a Hive external test table containing the given field and partition column names. + // Returns a case-sensitive schema for the table. + private def setupExternalTable( + fileType: String, + fields: Seq[String], + partitionCols: Seq[String], + dir: File): StructType = { + // Treat all table fields as bigints... + val structFields = fields.map { field => + StructField( + name = field, + dataType = LongType, + nullable = true, + metadata = Metadata.empty) + } + // and all partition columns as ints + val partitionStructFields = partitionCols.map { field => + StructField( + // Partition column case isn't preserved + name = field.toLowerCase, + dataType = IntegerType, + nullable = true, + metadata = Metadata.empty) + } + val schema = StructType(structFields ++ partitionStructFields) + + // Write some test data (partitioned if specified) + val writer = spark.range(NUM_RECORDS) + .selectExpr((fields ++ partitionCols).map("id as " + _): _*) + .write + .partitionBy(partitionCols: _*) + .mode("overwrite") + fileType match { + case ORC_FILE_TYPE => + writer.orc(dir.getAbsolutePath) + case PARQUET_FILE_TYPE => + writer.parquet(dir.getAbsolutePath) + } + + // Create Hive external table with lowercased schema + val serde = HiveSerDe.serdeMap(fileType) + client.createTable( + CatalogTable( + identifier = TableIdentifier(table = TEST_TABLE_NAME, database = Option(DATABASE)), + tableType = CatalogTableType.EXTERNAL, + storage = CatalogStorageFormat( + locationUri = Option(dir.toURI), + inputFormat = serde.inputFormat, + outputFormat = serde.outputFormat, + serde = serde.serde, + compressed = false, + properties = Map("serialization.format" -> "1")), + schema = schema, + provider = Option("hive"), + partitionColumnNames = partitionCols.map(_.toLowerCase), + properties = Map.empty), + true) + + // Add partition records (if specified) + if (!partitionCols.isEmpty) { + spark.catalog.recoverPartitions(TEST_TABLE_NAME) + } + + // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false + // and that the raw table returned by the Hive client doesn't have any Spark SQL properties + // set (table needs to be obtained from client since HiveExternalCatalog filters these + // properties out). + assert(!externalCatalog.getTable(DATABASE, TEST_TABLE_NAME).schemaPreservesCase) + val rawTable = client.getTable(DATABASE, TEST_TABLE_NAME) + assert(rawTable.properties.filterKeys(_.startsWith(DATASOURCE_SCHEMA_PREFIX)) == Map.empty) + schema + } + + private def withTestTables( + fileType: String)(f: (Seq[String], Seq[String], StructType) => Unit): Unit = { + // Test both a partitioned and unpartitioned Hive table + val tableFields = Seq( + (Seq("fieldOne"), Seq("partCol1", "partCol2")), + (Seq("fieldOne", "fieldTwo"), Seq.empty[String])) + + tableFields.foreach { case (fields, partCols) => + withTempDir { dir => + val schema = setupExternalTable(fileType, fields, partCols, dir) + withTable(TEST_TABLE_NAME) { f(fields, partCols, schema) } + } + } + } + + private def withFileTypes(f: (String) => Unit): Unit + = Seq(ORC_FILE_TYPE, PARQUET_FILE_TYPE).foreach(f) + + private def withInferenceMode(mode: InferenceMode)(f: => Unit): Unit = { + withSQLConf( + HiveUtils.CONVERT_METASTORE_ORC.key -> "true", + SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key -> mode.toString)(f) + } + + private val inferenceKey = SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key + + private def testFieldQuery(fields: Seq[String]): Unit = { + if (!fields.isEmpty) { + val query = s"SELECT * FROM ${TEST_TABLE_NAME} WHERE ${Random.shuffle(fields).head} >= 0" + assert(spark.sql(query).count == NUM_RECORDS) + } + } + + private def testTableSchema(expectedSchema: StructType): Unit + = assert(spark.table(TEST_TABLE_NAME).schema == expectedSchema) + + withFileTypes { fileType => + test(s"$fileType: schema should be inferred and saved when INFER_AND_SAVE is specified") { + withInferenceMode(INFER_AND_SAVE) { + withTestTables(fileType) { (fields, partCols, schema) => + testFieldQuery(fields) + testFieldQuery(partCols) + testTableSchema(schema) + + // Verify the catalog table now contains the updated schema and properties + val catalogTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME) + assert(catalogTable.schemaPreservesCase) + assert(catalogTable.schema == schema) + assert(catalogTable.partitionColumnNames == partCols.map(_.toLowerCase)) + } + } + } + } + + withFileTypes { fileType => + test(s"$fileType: schema should be inferred but not stored when INFER_ONLY is specified") { + withInferenceMode(INFER_ONLY) { + withTestTables(fileType) { (fields, partCols, schema) => + val originalTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME) + testFieldQuery(fields) + testFieldQuery(partCols) + testTableSchema(schema) + // Catalog table shouldn't be altered + assert(externalCatalog.getTable(DATABASE, TEST_TABLE_NAME) == originalTable) + } + } + } + } + + withFileTypes { fileType => + test(s"$fileType: schema should not be inferred when NEVER_INFER is specified") { + withInferenceMode(NEVER_INFER) { + withTestTables(fileType) { (fields, partCols, schema) => + val originalTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME) + // Only check the table schema as the test queries will break + testTableSchema(lowerCaseSchema(schema)) + assert(externalCatalog.getTable(DATABASE, TEST_TABLE_NAME) == originalTable) + } + } + } + } + + test("mergeWithMetastoreSchema() should return expected results") { + // Field type conflict resolution + assertResult( + StructType(Seq( + StructField("lowerCase", StringType), + StructField("UPPERCase", DoubleType, nullable = false)))) { + + HiveMetastoreCatalog.mergeWithMetastoreSchema( + StructType(Seq( + StructField("lowercase", StringType), + StructField("uppercase", DoubleType, nullable = false))), + + StructType(Seq( + StructField("lowerCase", BinaryType), + StructField("UPPERCase", IntegerType, nullable = true)))) + } + + // MetaStore schema is subset of parquet schema + assertResult( + StructType(Seq( + StructField("UPPERCase", DoubleType, nullable = false)))) { + + HiveMetastoreCatalog.mergeWithMetastoreSchema( + StructType(Seq( + StructField("uppercase", DoubleType, nullable = false))), + + StructType(Seq( + StructField("lowerCase", BinaryType), + StructField("UPPERCase", IntegerType, nullable = true)))) + } + + // Metastore schema contains additional non-nullable fields. + assert(intercept[Throwable] { + HiveMetastoreCatalog.mergeWithMetastoreSchema( + StructType(Seq( + StructField("uppercase", DoubleType, nullable = false), + StructField("lowerCase", BinaryType, nullable = false))), + + StructType(Seq( + StructField("UPPERCase", IntegerType, nullable = true)))) + }.getMessage.contains("Detected conflicting schemas")) + + // Conflicting non-nullable field names + intercept[Throwable] { + HiveMetastoreCatalog.mergeWithMetastoreSchema( + StructType(Seq(StructField("lower", StringType, nullable = false))), + StructType(Seq(StructField("lowerCase", BinaryType)))) + } + + // Check that merging missing nullable fields works as expected. + assertResult( + StructType(Seq( + StructField("firstField", StringType, nullable = true), + StructField("secondField", StringType, nullable = true), + StructField("thirdfield", StringType, nullable = true)))) { + HiveMetastoreCatalog.mergeWithMetastoreSchema( + StructType(Seq( + StructField("firstfield", StringType, nullable = true), + StructField("secondfield", StringType, nullable = true), + StructField("thirdfield", StringType, nullable = true))), + StructType(Seq( + StructField("firstField", StringType, nullable = true), + StructField("secondField", StringType, nullable = true)))) + } + + // Merge should fail if the Metastore contains any additional fields that are not + // nullable. + assert(intercept[Throwable] { + HiveMetastoreCatalog.mergeWithMetastoreSchema( + StructType(Seq( + StructField("firstfield", StringType, nullable = true), + StructField("secondfield", StringType, nullable = true), + StructField("thirdfield", StringType, nullable = false))), + StructType(Seq( + StructField("firstField", StringType, nullable = true), + StructField("secondField", StringType, nullable = true)))) + }.getMessage.contains("Detected conflicting schemas")) + + // Schema merge should maintain metastore order. + assertResult( + StructType(Seq( + StructField("first_field", StringType, nullable = true), + StructField("second_field", StringType, nullable = true), + StructField("third_field", StringType, nullable = true), + StructField("fourth_field", StringType, nullable = true), + StructField("fifth_field", StringType, nullable = true)))) { + HiveMetastoreCatalog.mergeWithMetastoreSchema( + StructType(Seq( + StructField("first_field", StringType, nullable = true), + StructField("second_field", StringType, nullable = true), + StructField("third_field", StringType, nullable = true), + StructField("fourth_field", StringType, nullable = true), + StructField("fifth_field", StringType, nullable = true))), + StructType(Seq( + StructField("fifth_field", StringType, nullable = true), + StructField("third_field", StringType, nullable = true), + StructField("second_field", StringType, nullable = true)))) + } + } +} + +object HiveSchemaInferenceSuite { + private val NUM_RECORDS = 10 + private val DATABASE = "default" + private val TEST_TABLE_NAME = "test_table" + private val ORC_FILE_TYPE = "orc" + private val PARQUET_FILE_TYPE = "parquet" +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionStateSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionStateSuite.scala new file mode 100644 index 000000000000..958ad3e1c3ce --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionStateSuite.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.sql._ +import org.apache.spark.sql.hive.test.TestHiveSingleton + +/** + * Run all tests from `SessionStateSuite` with a Hive based `SessionState`. + */ +class HiveSessionStateSuite extends SessionStateSuite + with TestHiveSingleton with BeforeAndAfterEach { + + override def beforeAll(): Unit = { + // Reuse the singleton session + activeSession = spark + } + + override def afterAll(): Unit = { + // Set activeSession to null to avoid stopping the singleton session + activeSession = null + super.afterAll() + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 10e4ae2c5030..21b3e281490c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -17,42 +17,91 @@ package org.apache.spark.sql.hive -import java.io.File -import java.sql.Timestamp -import java.util.Date +import java.io.{BufferedWriter, File, FileWriter} -import scala.collection.mutable.ArrayBuffer +import scala.tools.nsc.Properties -import org.scalatest.Matchers -import org.scalatest.concurrent.Timeouts -import org.scalatest.exceptions.TestFailedDueToTimeoutException -import org.scalatest.time.SpanSugar._ +import org.apache.hadoop.fs.Path +import org.scalatest.{BeforeAndAfterEach, Matchers} import org.apache.spark._ -import org.apache.spark.sql.{SQLContext, QueryTest} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{QueryTest, Row, SparkSession} +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.hive.test.{TestHive, TestHiveContext} -import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer -import org.apache.spark.sql.types.DecimalType +import org.apache.spark.sql.types.{DecimalType, StructType} import org.apache.spark.util.{ResetSystemProperties, Utils} /** * This suite tests spark-submit with applications using HiveContext. */ class HiveSparkSubmitSuite - extends SparkFunSuite + extends SparkSubmitTestUtils with Matchers - // This test suite sometimes gets extremely slow out of unknown reason on Jenkins. Here we - // add a timestamp to provide more diagnosis information. - with ResetSystemProperties - with Timeouts { + with BeforeAndAfterEach + with ResetSystemProperties { // TODO: rewrite these or mark them as slow tests to be run sparingly - def beforeAll() { + override def beforeEach() { + super.beforeEach() System.setProperty("spark.testing", "true") } + test("temporary Hive UDF: define a UDF and use it") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) + val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) + val jarsString = Seq(jar1, jar2).map(j => j.toString).mkString(",") + val args = Seq( + "--class", TemporaryHiveUDFTest.getClass.getName.stripSuffix("$"), + "--name", "TemporaryHiveUDFTest", + "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--driver-java-options", "-Dderby.system.durability=test", + "--jars", jarsString, + unusedJar.toString, "SparkSubmitClassA", "SparkSubmitClassB") + runSparkSubmit(args) + } + + test("permanent Hive UDF: define a UDF and use it") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) + val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) + val jarsString = Seq(jar1, jar2).map(j => j.toString).mkString(",") + val args = Seq( + "--class", PermanentHiveUDFTest1.getClass.getName.stripSuffix("$"), + "--name", "PermanentHiveUDFTest1", + "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--driver-java-options", "-Dderby.system.durability=test", + "--jars", jarsString, + unusedJar.toString, "SparkSubmitClassA", "SparkSubmitClassB") + runSparkSubmit(args) + } + + test("permanent Hive UDF: use a already defined permanent function") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) + val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) + val jarsString = Seq(jar1, jar2).map(j => j.toString).mkString(",") + val args = Seq( + "--class", PermanentHiveUDFTest2.getClass.getName.stripSuffix("$"), + "--name", "PermanentHiveUDFTest2", + "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--driver-java-options", "-Dderby.system.durability=test", + "--jars", jarsString, + unusedJar.toString, "SparkSubmitClassA", "SparkSubmitClassB") + runSparkSubmit(args) + } + test("SPARK-8368: includes jars passed in through --jars") { val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) @@ -66,6 +115,7 @@ class HiveSparkSubmitSuite "--master", "local-cluster[2,1,1024]", "--conf", "spark.ui.enabled=false", "--conf", "spark.master.rest.enabled=false", + "--driver-java-options", "-Dderby.system.durability=test", "--jars", jarsString, unusedJar.toString, "SparkSubmitClassA", "SparkSubmitClassB") runSparkSubmit(args) @@ -79,6 +129,9 @@ class HiveSparkSubmitSuite "--master", "local-cluster[2,1,1024]", "--conf", "spark.ui.enabled=false", "--conf", "spark.master.rest.enabled=false", + "--conf", "spark.sql.hive.metastore.version=0.12", + "--conf", "spark.sql.hive.metastore.jars=maven", + "--driver-java-options", "-Dderby.system.durability=test", unusedJar.toString) runSparkSubmit(args) } @@ -89,10 +142,16 @@ class HiveSparkSubmitSuite // Before the fix in SPARK-8470, this results in a MissingRequirementError because // the HiveContext code mistakenly overrides the class loader that contains user classes. // For more detail, see sql/hive/src/test/resources/regression-test-SPARK-8489/*scala. - val testJar = "sql/hive/src/test/resources/regression-test-SPARK-8489/test.jar" + val version = Properties.versionNumberString match { + case v if v.startsWith("2.12") || v.startsWith("2.11") => v.substring(0, 4) + case x => throw new Exception(s"Unsupported Scala Version: $x") + } + val jarDir = getTestResourcePath("regression-test-SPARK-8489") + val testJar = s"$jarDir/test-$version.jar" val args = Seq( "--conf", "spark.ui.enabled=false", "--conf", "spark.master.rest.enabled=false", + "--driver-java-options", "-Dderby.system.durability=test", "--class", "Main", testJar) runSparkSubmit(args) @@ -104,6 +163,9 @@ class HiveSparkSubmitSuite "--class", SPARK_9757.getClass.getName.stripSuffix("$"), "--name", "SparkSQLConfTest", "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--driver-java-options", "-Dderby.system.durability=test", unusedJar.toString) runSparkSubmit(args) } @@ -114,77 +176,392 @@ class HiveSparkSubmitSuite "--class", SPARK_11009.getClass.getName.stripSuffix("$"), "--name", "SparkSQLConfTest", "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--driver-java-options", "-Dderby.system.durability=test", + unusedJar.toString) + runSparkSubmit(args) + } + + test("SPARK-14244 fix window partition size attribute binding failure") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SPARK_14244.getClass.getName.stripSuffix("$"), + "--name", "SparkSQLConfTest", + "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--driver-java-options", "-Dderby.system.durability=test", + unusedJar.toString) + runSparkSubmit(args) + } + + test("set spark.sql.warehouse.dir") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SetWarehouseLocationTest.getClass.getName.stripSuffix("$"), + "--name", "SetSparkWarehouseLocationTest", + "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--driver-java-options", "-Dderby.system.durability=test", unusedJar.toString) runSparkSubmit(args) } - // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. - // This is copied from org.apache.spark.deploy.SparkSubmitSuite - private def runSparkSubmit(args: Seq[String]): Unit = { - val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) - val history = ArrayBuffer.empty[String] - val commands = Seq("./bin/spark-submit") ++ args - val commandLine = commands.mkString("'", "' '", "'") - - val builder = new ProcessBuilder(commands: _*).directory(new File(sparkHome)) - val env = builder.environment() - env.put("SPARK_TESTING", "1") - env.put("SPARK_HOME", sparkHome) - - def captureOutput(source: String)(line: String): Unit = { - // This test suite has some weird behaviors when executed on Jenkins: - // - // 1. Sometimes it gets extremely slow out of unknown reason on Jenkins. Here we add a - // timestamp to provide more diagnosis information. - // 2. Log lines are not correctly redirected to unit-tests.log as expected, so here we print - // them out for debugging purposes. - val logLine = s"${new Timestamp(new Date().getTime)} - $source> $line" - // scalastyle:off println - println(logLine) - // scalastyle:on println - history += logLine + test("set hive.metastore.warehouse.dir") { + // In this test, we set hive.metastore.warehouse.dir in hive-site.xml but + // not set spark.sql.warehouse.dir. So, the warehouse dir should be + // the value of hive.metastore.warehouse.dir. Also, the value of + // spark.sql.warehouse.dir should be set to the value of hive.metastore.warehouse.dir. + + val hiveWarehouseLocation = Utils.createTempDir() + hiveWarehouseLocation.delete() + val hiveSiteXmlContent = + s""" + | + | + | hive.metastore.warehouse.dir + | $hiveWarehouseLocation + | + | + """.stripMargin + + // Write a hive-site.xml containing a setting of hive.metastore.warehouse.dir. + val hiveSiteDir = Utils.createTempDir() + val file = new File(hiveSiteDir.getCanonicalPath, "hive-site.xml") + val bw = new BufferedWriter(new FileWriter(file)) + bw.write(hiveSiteXmlContent) + bw.close() + + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SetWarehouseLocationTest.getClass.getName.stripSuffix("$"), + "--name", "SetHiveWarehouseLocationTest", + "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--conf", s"spark.sql.test.expectedWarehouseDir=$hiveWarehouseLocation", + "--conf", s"spark.driver.extraClassPath=${hiveSiteDir.getCanonicalPath}", + "--driver-java-options", "-Dderby.system.durability=test", + unusedJar.toString) + runSparkSubmit(args) + } + + test("SPARK-16901: set javax.jdo.option.ConnectionURL") { + // In this test, we set javax.jdo.option.ConnectionURL and set metastore version to + // 0.13. This test will make sure that javax.jdo.option.ConnectionURL will not be + // overridden by hive's default settings when we create a HiveConf object inside + // HiveClientImpl. Please see SPARK-16901 for more details. + + val metastoreLocation = Utils.createTempDir() + metastoreLocation.delete() + val metastoreURL = + s"jdbc:derby:memory:;databaseName=${metastoreLocation.getAbsolutePath};create=true" + val hiveSiteXmlContent = + s""" + | + | + | javax.jdo.option.ConnectionURL + | $metastoreURL + | + | + """.stripMargin + + // Write a hive-site.xml containing a setting of hive.metastore.warehouse.dir. + val hiveSiteDir = Utils.createTempDir() + val file = new File(hiveSiteDir.getCanonicalPath, "hive-site.xml") + val bw = new BufferedWriter(new FileWriter(file)) + bw.write(hiveSiteXmlContent) + bw.close() + + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SetMetastoreURLTest.getClass.getName.stripSuffix("$"), + "--name", "SetMetastoreURLTest", + "--master", "local[1]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--conf", s"spark.sql.test.expectedMetastoreURL=$metastoreURL", + "--conf", s"spark.driver.extraClassPath=${hiveSiteDir.getCanonicalPath}", + "--driver-java-options", "-Dderby.system.durability=test", + unusedJar.toString) + runSparkSubmit(args) + } + + test("SPARK-18360: default table path of tables in default database should depend on the " + + "location of default database") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SPARK_18360.getClass.getName.stripSuffix("$"), + "--name", "SPARK-18360", + "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--driver-java-options", "-Dderby.system.durability=test", + unusedJar.toString) + runSparkSubmit(args) + } + + test("SPARK-18989: DESC TABLE should not fail with format class not found") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + + val argsForCreateTable = Seq( + "--class", SPARK_18989_CREATE_TABLE.getClass.getName.stripSuffix("$"), + "--name", "SPARK-18947", + "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--jars", TestHive.getHiveFile("hive-contrib-0.13.1.jar").getCanonicalPath, + unusedJar.toString) + runSparkSubmit(argsForCreateTable) + + val argsForShowTables = Seq( + "--class", SPARK_18989_DESC_TABLE.getClass.getName.stripSuffix("$"), + "--name", "SPARK-18947", + "--master", "local-cluster[2,1,1024]", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + unusedJar.toString) + runSparkSubmit(argsForShowTables) + } +} + +object SetMetastoreURLTest extends Logging { + def main(args: Array[String]): Unit = { + Utils.configTestLog4j("INFO") + + val sparkConf = new SparkConf(loadDefaults = true) + val builder = SparkSession.builder() + .config(sparkConf) + .config("spark.ui.enabled", "false") + .config("spark.sql.hive.metastore.version", "0.13.1") + // The issue described in SPARK-16901 only appear when + // spark.sql.hive.metastore.jars is not set to builtin. + .config("spark.sql.hive.metastore.jars", "maven") + .enableHiveSupport() + + val spark = builder.getOrCreate() + val expectedMetastoreURL = + spark.conf.get("spark.sql.test.expectedMetastoreURL") + logInfo(s"spark.sql.test.expectedMetastoreURL is $expectedMetastoreURL") + + if (expectedMetastoreURL == null) { + throw new Exception( + s"spark.sql.test.expectedMetastoreURL should be set.") } - val process = builder.start() - new ProcessOutputCapturer(process.getInputStream, captureOutput("stdout")).start() - new ProcessOutputCapturer(process.getErrorStream, captureOutput("stderr")).start() + // HiveExternalCatalog is used when Hive support is enabled. + val actualMetastoreURL = + spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client + .getConf("javax.jdo.option.ConnectionURL", "this_is_a_wrong_URL") + logInfo(s"javax.jdo.option.ConnectionURL is $actualMetastoreURL") - try { - val exitCode = failAfter(300.seconds) { process.waitFor() } - if (exitCode != 0) { - // include logs in output. Note that logging is async and may not have completed - // at the time this exception is raised - Thread.sleep(1000) - val historyLog = history.mkString("\n") - fail { - s"""spark-submit returned with exit code $exitCode. - |Command line: $commandLine - | - |$historyLog - """.stripMargin - } + if (actualMetastoreURL != expectedMetastoreURL) { + throw new Exception( + s"Expected value of javax.jdo.option.ConnectionURL is $expectedMetastoreURL. But, " + + s"the actual value is $actualMetastoreURL") + } + } +} + +object SetWarehouseLocationTest extends Logging { + def main(args: Array[String]): Unit = { + Utils.configTestLog4j("INFO") + + val sparkConf = new SparkConf(loadDefaults = true).set("spark.ui.enabled", "false") + val providedExpectedWarehouseLocation = + sparkConf.getOption("spark.sql.test.expectedWarehouseDir") + + val (sparkSession, expectedWarehouseLocation) = providedExpectedWarehouseLocation match { + case Some(warehouseDir) => + // If spark.sql.test.expectedWarehouseDir is set, the warehouse dir is set + // through spark-summit. So, neither spark.sql.warehouse.dir nor + // hive.metastore.warehouse.dir is set at here. + (new TestHiveContext(new SparkContext(sparkConf)).sparkSession, warehouseDir) + case None => + val warehouseLocation = Utils.createTempDir() + warehouseLocation.delete() + val hiveWarehouseLocation = Utils.createTempDir() + hiveWarehouseLocation.delete() + // If spark.sql.test.expectedWarehouseDir is not set, we will set + // spark.sql.warehouse.dir and hive.metastore.warehouse.dir. + // We are expecting that the value of spark.sql.warehouse.dir will override the + // value of hive.metastore.warehouse.dir. + val session = new TestHiveContext(new SparkContext(sparkConf + .set("spark.sql.warehouse.dir", warehouseLocation.toString) + .set("hive.metastore.warehouse.dir", hiveWarehouseLocation.toString))) + .sparkSession + (session, warehouseLocation.toString) + + } + + if (sparkSession.conf.get("spark.sql.warehouse.dir") != expectedWarehouseLocation) { + throw new Exception( + "spark.sql.warehouse.dir is not set to the expected warehouse location " + + s"$expectedWarehouseLocation.") + } + + val catalog = sparkSession.sessionState.catalog + + sparkSession.sql("drop table if exists testLocation") + sparkSession.sql("drop database if exists testLocationDB cascade") + + { + sparkSession.sql("create table testLocation (a int)") + val tableMetadata = + catalog.getTableMetadata(TableIdentifier("testLocation", Some("default"))) + val expectedLocation = + CatalogUtils.stringToURI(s"file:${expectedWarehouseLocation.toString}/testlocation") + val actualLocation = tableMetadata.location + if (actualLocation != expectedLocation) { + throw new Exception( + s"Expected table location is $expectedLocation. But, it is actually $actualLocation") } - } catch { - case to: TestFailedDueToTimeoutException => - val historyLog = history.mkString("\n") - fail(s"Timeout of $commandLine" + - s" See the log4j logs for more detail." + - s"\n$historyLog", to) - case t: Throwable => throw t - } finally { - // Ensure we still kill the process in case it timed out - process.destroy() + sparkSession.sql("drop table testLocation") + } + + { + sparkSession.sql("create database testLocationDB") + sparkSession.sql("use testLocationDB") + sparkSession.sql("create table testLocation (a int)") + val tableMetadata = + catalog.getTableMetadata(TableIdentifier("testLocation", Some("testLocationDB"))) + val expectedLocation = CatalogUtils.stringToURI( + s"file:${expectedWarehouseLocation.toString}/testlocationdb.db/testlocation") + val actualLocation = tableMetadata.location + if (actualLocation != expectedLocation) { + throw new Exception( + s"Expected table location is $expectedLocation. But, it is actually $actualLocation") + } + sparkSession.sql("drop table testLocation") + sparkSession.sql("use default") + sparkSession.sql("drop database testLocationDB") } } } +// This application is used to test defining a new Hive UDF (with an associated jar) +// and use this UDF. We need to run this test in separate JVM to make sure we +// can load the jar defined with the function. +object TemporaryHiveUDFTest extends Logging { + def main(args: Array[String]) { + Utils.configTestLog4j("INFO") + val conf = new SparkConf() + conf.set("spark.ui.enabled", "false") + val sc = new SparkContext(conf) + val hiveContext = new TestHiveContext(sc) + + // Load a Hive UDF from the jar. + logInfo("Registering a temporary Hive UDF provided in a jar.") + val jar = hiveContext.getHiveFile("hive-contrib-0.13.1.jar").getCanonicalPath + hiveContext.sql( + s""" + |CREATE TEMPORARY FUNCTION example_max + |AS 'org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMax' + |USING JAR '$jar' + """.stripMargin) + val source = + hiveContext.createDataFrame((1 to 10).map(i => (i, s"str$i"))).toDF("key", "val") + source.createOrReplaceTempView("sourceTable") + // Actually use the loaded UDF. + logInfo("Using the UDF.") + val result = hiveContext.sql( + "SELECT example_max(key) as key, val FROM sourceTable GROUP BY val") + logInfo("Running a simple query on the table.") + val count = result.orderBy("key", "val").count() + if (count != 10) { + throw new Exception(s"Result table should have 10 rows instead of $count rows") + } + hiveContext.sql("DROP temporary FUNCTION example_max") + logInfo("Test finishes.") + sc.stop() + } +} + +// This application is used to test defining a new Hive UDF (with an associated jar) +// and use this UDF. We need to run this test in separate JVM to make sure we +// can load the jar defined with the function. +object PermanentHiveUDFTest1 extends Logging { + def main(args: Array[String]) { + Utils.configTestLog4j("INFO") + val conf = new SparkConf() + conf.set("spark.ui.enabled", "false") + val sc = new SparkContext(conf) + val hiveContext = new TestHiveContext(sc) + + // Load a Hive UDF from the jar. + logInfo("Registering a permanent Hive UDF provided in a jar.") + val jar = hiveContext.getHiveFile("hive-contrib-0.13.1.jar").getCanonicalPath + hiveContext.sql( + s""" + |CREATE FUNCTION example_max + |AS 'org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMax' + |USING JAR '$jar' + """.stripMargin) + val source = + hiveContext.createDataFrame((1 to 10).map(i => (i, s"str$i"))).toDF("key", "val") + source.createOrReplaceTempView("sourceTable") + // Actually use the loaded UDF. + logInfo("Using the UDF.") + val result = hiveContext.sql( + "SELECT example_max(key) as key, val FROM sourceTable GROUP BY val") + logInfo("Running a simple query on the table.") + val count = result.orderBy("key", "val").count() + if (count != 10) { + throw new Exception(s"Result table should have 10 rows instead of $count rows") + } + hiveContext.sql("DROP FUNCTION example_max") + logInfo("Test finishes.") + sc.stop() + } +} + +// This application is used to test that a pre-defined permanent function with a jar +// resources can be used. We need to run this test in separate JVM to make sure we +// can load the jar defined with the function. +object PermanentHiveUDFTest2 extends Logging { + def main(args: Array[String]) { + Utils.configTestLog4j("INFO") + val conf = new SparkConf() + conf.set("spark.ui.enabled", "false") + val sc = new SparkContext(conf) + val hiveContext = new TestHiveContext(sc) + // Load a Hive UDF from the jar. + logInfo("Write the metadata of a permanent Hive UDF into metastore.") + val jar = hiveContext.getHiveFile("hive-contrib-0.13.1.jar").getCanonicalPath + val function = CatalogFunction( + FunctionIdentifier("example_max"), + "org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMax", + FunctionResource(JarResource, jar) :: Nil) + hiveContext.sessionState.catalog.createFunction(function, ignoreIfExists = false) + val source = + hiveContext.createDataFrame((1 to 10).map(i => (i, s"str$i"))).toDF("key", "val") + source.createOrReplaceTempView("sourceTable") + // Actually use the loaded UDF. + logInfo("Using the UDF.") + val result = hiveContext.sql( + "SELECT example_max(key) as key, val FROM sourceTable GROUP BY val") + logInfo("Running a simple query on the table.") + val count = result.orderBy("key", "val").count() + if (count != 10) { + throw new Exception(s"Result table should have 10 rows instead of $count rows") + } + hiveContext.sql("DROP FUNCTION example_max") + logInfo("Test finishes.") + sc.stop() + } +} + // This object is used for testing SPARK-8368: https://issues.apache.org/jira/browse/SPARK-8368. // We test if we can load user jars in both driver and executors when HiveContext is used. object SparkSubmitClassLoaderTest extends Logging { def main(args: Array[String]) { Utils.configTestLog4j("INFO") val conf = new SparkConf() + val hiveWarehouseLocation = Utils.createTempDir() conf.set("spark.ui.enabled", "false") + conf.set("spark.sql.warehouse.dir", hiveWarehouseLocation.toString) val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) val df = hiveContext.createDataFrame((1 to 100).map(i => (i, i))).toDF("i", "j") @@ -199,14 +576,14 @@ object SparkSubmitClassLoaderTest extends Logging { } // Second, we load classes at the executor side. logInfo("Testing load classes at the executor side.") - val result = df.mapPartitions { x => + val result = df.rdd.mapPartitions { x => var exception: String = null try { Utils.classForName(args(0)) Utils.classForName(args(1)) } catch { case t: Throwable => - exception = t + "\n" + t.getStackTraceString + exception = t + "\n" + Utils.exceptionString(t) exception = exception.replaceAll("\n", "\n\t") } Option(exception).toSeq.iterator @@ -224,7 +601,7 @@ object SparkSubmitClassLoaderTest extends Logging { """.stripMargin) val source = hiveContext.createDataFrame((1 to 10).map(i => (i, s"str$i"))).toDF("key", "val") - source.registerTempTable("sourceTable") + source.createOrReplaceTempView("sourceTable") // Load a Hive SerDe from the jar. logInfo("Creating a Hive table with a SerDe provided in a jar.") hiveContext.sql( @@ -288,19 +665,21 @@ object SparkSQLConfTest extends Logging { object SPARK_9757 extends QueryTest { import org.apache.spark.sql.functions._ - protected var sqlContext: SQLContext = _ + protected var spark: SparkSession = _ def main(args: Array[String]): Unit = { Utils.configTestLog4j("INFO") + val hiveWarehouseLocation = Utils.createTempDir() val sparkContext = new SparkContext( new SparkConf() .set("spark.sql.hive.metastore.version", "0.13.1") .set("spark.sql.hive.metastore.jars", "maven") - .set("spark.ui.enabled", "false")) + .set("spark.ui.enabled", "false") + .set("spark.sql.warehouse.dir", hiveWarehouseLocation.toString)) val hiveContext = new TestHiveContext(sparkContext) - sqlContext = hiveContext + spark = hiveContext.sparkSession import hiveContext.implicits._ val dir = Utils.createTempDir() @@ -335,7 +714,7 @@ object SPARK_9757 extends QueryTest { object SPARK_11009 extends QueryTest { import org.apache.spark.sql.functions._ - protected var sqlContext: SQLContext = _ + protected var spark: SparkSession = _ def main(args: Array[String]): Unit = { Utils.configTestLog4j("INFO") @@ -346,13 +725,13 @@ object SPARK_11009 extends QueryTest { .set("spark.sql.shuffle.partitions", "100")) val hiveContext = new TestHiveContext(sparkContext) - sqlContext = hiveContext + spark = hiveContext.sparkSession try { - val df = sqlContext.range(1 << 20) + val df = spark.range(1 << 20) val df2 = df.select((df("id") % 1000).alias("A"), (df("id") / 1000).alias("B")) val ws = Window.partitionBy(df2("A")).orderBy(df2("B")) - val df3 = df2.select(df2("A"), df2("B"), rowNumber().over(ws).alias("rn")).filter("rn < 0") + val df3 = df2.select(df2("A"), df2("B"), row_number().over(ws).alias("rn")).filter("rn < 0") if (df3.rdd.count() != 0) { throw new Exception("df3 should have 0 output row.") } @@ -361,3 +740,97 @@ object SPARK_11009 extends QueryTest { } } } + +object SPARK_14244 extends QueryTest { + import org.apache.spark.sql.expressions.Window + import org.apache.spark.sql.functions._ + + protected var spark: SparkSession = _ + + def main(args: Array[String]): Unit = { + Utils.configTestLog4j("INFO") + + val sparkContext = new SparkContext( + new SparkConf() + .set("spark.ui.enabled", "false") + .set("spark.sql.shuffle.partitions", "100")) + + val hiveContext = new TestHiveContext(sparkContext) + spark = hiveContext.sparkSession + + import hiveContext.implicits._ + + try { + val window = Window.orderBy('id) + val df = spark.range(2).select(cume_dist().over(window).as('cdist)).orderBy('cdist) + checkAnswer(df, Seq(Row(0.5D), Row(1.0D))) + } finally { + sparkContext.stop() + } + } +} + +object SPARK_18360 { + def main(args: Array[String]): Unit = { + val spark = SparkSession.builder() + .config("spark.ui.enabled", "false") + .enableHiveSupport().getOrCreate() + + val defaultDbLocation = spark.catalog.getDatabase("default").locationUri + assert(new Path(defaultDbLocation) == new Path(spark.sharedState.warehousePath)) + + val hiveClient = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client + + try { + val tableMeta = CatalogTable( + identifier = TableIdentifier("test_tbl", Some("default")), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType().add("i", "int"), + provider = Some(DDLUtils.HIVE_PROVIDER)) + + val newWarehousePath = Utils.createTempDir().getAbsolutePath + hiveClient.runSqlHive(s"SET hive.metastore.warehouse.dir=$newWarehousePath") + hiveClient.createTable(tableMeta, ignoreIfExists = false) + val rawTable = hiveClient.getTable("default", "test_tbl") + // Hive will use the value of `hive.metastore.warehouse.dir` to generate default table + // location for tables in default database. + assert(rawTable.storage.locationUri.map( + CatalogUtils.URIToString(_)).get.contains(newWarehousePath)) + hiveClient.dropTable("default", "test_tbl", ignoreIfNotExists = false, purge = false) + + spark.sharedState.externalCatalog.createTable(tableMeta, ignoreIfExists = false) + val readBack = spark.sharedState.externalCatalog.getTable("default", "test_tbl") + // Spark SQL will use the location of default database to generate default table + // location for tables in default database. + assert(readBack.storage.locationUri.map(CatalogUtils.URIToString(_)) + .get.contains(defaultDbLocation)) + } finally { + hiveClient.dropTable("default", "test_tbl", ignoreIfNotExists = true, purge = false) + hiveClient.runSqlHive(s"SET hive.metastore.warehouse.dir=$defaultDbLocation") + } + } +} + +object SPARK_18989_CREATE_TABLE { + def main(args: Array[String]): Unit = { + val spark = SparkSession.builder().enableHiveSupport().getOrCreate() + spark.sql( + """ + |CREATE TABLE IF NOT EXISTS base64_tbl(val string) STORED AS + |INPUTFORMAT 'org.apache.hadoop.hive.contrib.fileformat.base64.Base64TextInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.contrib.fileformat.base64.Base64TextOutputFormat' + """.stripMargin) + } +} + +object SPARK_18989_DESC_TABLE { + def main(args: Array[String]): Unit = { + val spark = SparkSession.builder().enableHiveSupport().getOrCreate() + try { + spark.sql("DESC base64_tbl") + } finally { + spark.sql("DROP TABLE IF EXISTS base64_tbl") + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala new file mode 100644 index 000000000000..fdbfcf1a6844 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala @@ -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.hive + +import org.apache.hadoop.hive.conf.HiveConf.ConfVars + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils + +class HiveUtilsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + + test("newTemporaryConfiguration overwrites listener configurations") { + Seq(true, false).foreach { useInMemoryDerby => + val conf = HiveUtils.newTemporaryConfiguration(useInMemoryDerby) + assert(conf(ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname) === "") + assert(conf(ConfVars.METASTORE_EVENT_LISTENERS.varname) === "") + assert(conf(ConfVars.METASTORE_END_FUNCTION_LISTENERS.varname) === "") + } + } + + test("newTemporaryConfiguration respect spark.hadoop.foo=bar in SparkConf") { + sys.props.put("spark.hadoop.foo", "bar") + Seq(true, false) foreach { useInMemoryDerby => + val hiveConf = HiveUtils.newTemporaryConfiguration(useInMemoryDerby) + assert(!hiveConf.contains("spark.hadoop.foo")) + assert(hiveConf("foo") === "bar") + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveVariableSubstitutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveVariableSubstitutionSuite.scala new file mode 100644 index 000000000000..84d3946ca5c6 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveVariableSubstitutionSuite.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.hive.test.TestHiveSingleton + +class HiveVariableSubstitutionSuite extends QueryTest with TestHiveSingleton { + test("SET hivevar with prefix") { + spark.sql("SET hivevar:county=gram") + assert(spark.conf.getOption("county") === Some("gram")) + } + + test("SET hivevar with dotted name") { + spark.sql("SET hivevar:eloquent.mosquito.alphabet=zip") + assert(spark.conf.getOption("eloquent.mosquito.alphabet") === Some("zip")) + } + + test("hivevar substitution") { + spark.conf.set("pond", "bus") + checkAnswer(spark.sql("SELECT '${hivevar:pond}'"), Row("bus") :: Nil) + } + + test("variable substitution without a prefix") { + spark.sql("SET hivevar:flask=plaid") + checkAnswer(spark.sql("SELECT '${flask}'"), Row("plaid") :: Nil) + } + + test("variable substitution precedence") { + spark.conf.set("turn.aloof", "questionable") + spark.sql("SET hivevar:turn.aloof=dime") + // hivevar clobbers the conf setting + checkAnswer(spark.sql("SELECT '${turn.aloof}'"), Row("dime") :: Nil) + } +} 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 deleted file mode 100644 index 81ee9ba71beb..000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ /dev/null @@ -1,262 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import java.io.File - -import org.apache.hadoop.hive.conf.HiveConf -import org.scalatest.BeforeAndAfter - -import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.sql.{QueryTest, _} -import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils - -case class TestData(key: Int, value: String) - -case class ThreeCloumntable(key: Int, value: String, key1: String) - -class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter { - import hiveContext.implicits._ - import hiveContext.sql - - val testData = hiveContext.sparkContext.parallelize( - (1 to 100).map(i => TestData(i, i.toString))).toDF() - - before { - // Since every we are doing tests for DDL statements, - // it is better to reset before every test. - hiveContext.reset() - // Register the testData, which will be used in every test. - testData.registerTempTable("testData") - } - - test("insertInto() HiveTable") { - sql("CREATE TABLE createAndInsertTest (key int, value string)") - - // Add some data. - testData.write.mode(SaveMode.Append).insertInto("createAndInsertTest") - - // Make sure the table has also been updated. - checkAnswer( - sql("SELECT * FROM createAndInsertTest"), - testData.collect().toSeq - ) - - // Add more data. - testData.write.mode(SaveMode.Append).insertInto("createAndInsertTest") - - // Make sure the table has been updated. - checkAnswer( - sql("SELECT * FROM createAndInsertTest"), - testData.toDF().collect().toSeq ++ testData.toDF().collect().toSeq - ) - - // Now overwrite. - testData.write.mode(SaveMode.Overwrite).insertInto("createAndInsertTest") - - // Make sure the registered table has also been updated. - checkAnswer( - sql("SELECT * FROM createAndInsertTest"), - testData.collect().toSeq - ) - } - - test("Double create fails when allowExisting = false") { - sql("CREATE TABLE doubleCreateAndInsertTest (key int, value string)") - - intercept[QueryExecutionException] { - sql("CREATE TABLE doubleCreateAndInsertTest (key int, value string)") - } - } - - test("Double create does not fail when allowExisting = true") { - 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") { - val schema = StructType(StructField("m", MapType(StringType, StringType), true) :: Nil) - val rowRDD = hiveContext.sparkContext.parallelize( - (1 to 100).map(i => Row(scala.collection.mutable.HashMap(s"key$i" -> s"value$i")))) - val df = hiveContext.createDataFrame(rowRDD, schema) - df.registerTempTable("tableWithMapValue") - sql("CREATE TABLE hiveTableWithMapValue(m MAP )") - sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue") - - checkAnswer( - sql("SELECT * FROM hiveTableWithMapValue"), - rowRDD.collect().toSeq - ) - - sql("DROP TABLE hiveTableWithMapValue") - } - - test("SPARK-4203:random partition directory order") { - sql("CREATE TABLE tmp_table (key int, value string)") - val tmpDir = Utils.createTempDir() - val stagingDir = new HiveConf().getVar(HiveConf.ConfVars.STAGINGDIR) - - sql( - s""" - |CREATE TABLE table_with_partition(c1 string) - |PARTITIONED by (p1 string,p2 string,p3 string,p4 string,p5 string) - |location '${tmpDir.toURI.toString}' - """.stripMargin) - sql( - """ - |INSERT OVERWRITE TABLE table_with_partition - |partition (p1='a',p2='b',p3='c',p4='c',p5='1') - |SELECT 'blarr' FROM tmp_table - """.stripMargin) - sql( - """ - |INSERT OVERWRITE TABLE table_with_partition - |partition (p1='a',p2='b',p3='c',p4='c',p5='2') - |SELECT 'blarr' FROM tmp_table - """.stripMargin) - sql( - """ - |INSERT OVERWRITE TABLE table_with_partition - |partition (p1='a',p2='b',p3='c',p4='c',p5='3') - |SELECT 'blarr' FROM tmp_table - """.stripMargin) - sql( - """ - |INSERT OVERWRITE TABLE table_with_partition - |partition (p1='a',p2='b',p3='c',p4='c',p5='4') - |SELECT 'blarr' FROM tmp_table - """.stripMargin) - def listFolders(path: File, acc: List[String]): List[List[String]] = { - val dir = path.listFiles() - val folders = dir.filter { e => e.isDirectory && !e.getName().startsWith(stagingDir) }.toList - if (folders.isEmpty) { - List(acc.reverse) - } else { - folders.flatMap(x => listFolders(x, x.getName :: acc)) - } - } - val expected = List( - "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=2"::Nil, - "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=3"::Nil , - "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=1"::Nil , - "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=4"::Nil - ) - assert(listFolders(tmpDir, List()).sortBy(_.toString()) === expected.sortBy(_.toString)) - sql("DROP TABLE table_with_partition") - sql("DROP TABLE tmp_table") - } - - test("Insert ArrayType.containsNull == false") { - val schema = StructType(Seq( - StructField("a", ArrayType(StringType, containsNull = false)))) - val rowRDD = hiveContext.sparkContext.parallelize((1 to 100).map(i => Row(Seq(s"value$i")))) - val df = hiveContext.createDataFrame(rowRDD, schema) - df.registerTempTable("tableWithArrayValue") - sql("CREATE TABLE hiveTableWithArrayValue(a Array )") - sql("INSERT OVERWRITE TABLE hiveTableWithArrayValue SELECT a FROM tableWithArrayValue") - - checkAnswer( - sql("SELECT * FROM hiveTableWithArrayValue"), - rowRDD.collect().toSeq) - - sql("DROP TABLE hiveTableWithArrayValue") - } - - test("Insert MapType.valueContainsNull == false") { - val schema = StructType(Seq( - StructField("m", MapType(StringType, StringType, valueContainsNull = false)))) - val rowRDD = hiveContext.sparkContext.parallelize( - (1 to 100).map(i => Row(Map(s"key$i" -> s"value$i")))) - val df = hiveContext.createDataFrame(rowRDD, schema) - df.registerTempTable("tableWithMapValue") - sql("CREATE TABLE hiveTableWithMapValue(m Map )") - sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue") - - checkAnswer( - sql("SELECT * FROM hiveTableWithMapValue"), - rowRDD.collect().toSeq) - - sql("DROP TABLE hiveTableWithMapValue") - } - - test("Insert StructType.fields.exists(_.nullable == false)") { - val schema = StructType(Seq( - StructField("s", StructType(Seq(StructField("f", StringType, nullable = false)))))) - val rowRDD = hiveContext.sparkContext.parallelize( - (1 to 100).map(i => Row(Row(s"value$i")))) - val df = hiveContext.createDataFrame(rowRDD, schema) - df.registerTempTable("tableWithStructValue") - sql("CREATE TABLE hiveTableWithStructValue(s Struct )") - sql("INSERT OVERWRITE TABLE hiveTableWithStructValue SELECT s FROM tableWithStructValue") - - checkAnswer( - sql("SELECT * FROM hiveTableWithStructValue"), - rowRDD.collect().toSeq) - - sql("DROP TABLE hiveTableWithStructValue") - } - - test("SPARK-5498:partition schema does not match table schema") { - val testData = hiveContext.sparkContext.parallelize( - (1 to 10).map(i => TestData(i, i.toString))).toDF() - testData.registerTempTable("testData") - - val testDatawithNull = hiveContext.sparkContext.parallelize( - (1 to 10).map(i => ThreeCloumntable(i, i.toString, null))).toDF() - - val tmpDir = Utils.createTempDir() - sql( - s""" - |CREATE TABLE table_with_partition(key int,value string) - |PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' - """.stripMargin) - sql( - """ - |INSERT OVERWRITE TABLE table_with_partition - |partition (ds='1') SELECT key,value FROM testData - """.stripMargin) - - // test schema the same between partition and table - sql("ALTER TABLE table_with_partition CHANGE COLUMN key key BIGINT") - checkAnswer(sql("select key,value from table_with_partition where ds='1' "), - testData.collect().toSeq - ) - - // test difference type of field - sql("ALTER TABLE table_with_partition CHANGE COLUMN key key BIGINT") - checkAnswer(sql("select key,value from table_with_partition where ds='1' "), - testData.collect().toSeq - ) - - // add column to table - sql("ALTER TABLE table_with_partition ADD COLUMNS(key1 string)") - checkAnswer(sql("select key,value,key1 from table_with_partition where ds='1' "), - testDatawithNull.collect().toSeq - ) - - // change column name to table - sql("ALTER TABLE table_with_partition CHANGE COLUMN key keynew BIGINT") - checkAnswer(sql("select keynew,value from table_with_partition where ds='1' "), - testData.collect().toSeq - ) - - sql("DROP TABLE table_with_partition") - } -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala new file mode 100644 index 000000000000..aa5cae33f5cd --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -0,0 +1,731 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkException +import org.apache.spark.sql.{QueryTest, _} +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + +case class TestData(key: Int, value: String) + +case class ThreeCloumntable(key: Int, value: String, key1: String) + +class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter + with SQLTestUtils { + import spark.implicits._ + + override lazy val testData = spark.sparkContext.parallelize( + (1 to 100).map(i => TestData(i, i.toString))).toDF() + + before { + // Since every we are doing tests for DDL statements, + // it is better to reset before every test. + hiveContext.reset() + // Creates a temporary view with testData, which will be used in all tests. + testData.createOrReplaceTempView("testData") + } + + test("insertInto() HiveTable") { + withTable("createAndInsertTest") { + sql("CREATE TABLE createAndInsertTest (key int, value string)") + + // Add some data. + testData.write.mode(SaveMode.Append).insertInto("createAndInsertTest") + + // Make sure the table has also been updated. + checkAnswer( + sql("SELECT * FROM createAndInsertTest"), + testData.collect().toSeq + ) + + // Add more data. + testData.write.mode(SaveMode.Append).insertInto("createAndInsertTest") + + // Make sure the table has been updated. + checkAnswer( + sql("SELECT * FROM createAndInsertTest"), + testData.toDF().collect().toSeq ++ testData.toDF().collect().toSeq + ) + + // Now overwrite. + testData.write.mode(SaveMode.Overwrite).insertInto("createAndInsertTest") + + // Make sure the registered table has also been updated. + checkAnswer( + sql("SELECT * FROM createAndInsertTest"), + testData.collect().toSeq + ) + } + } + + test("Double create fails when allowExisting = false") { + withTable("doubleCreateAndInsertTest") { + sql("CREATE TABLE doubleCreateAndInsertTest (key int, value string)") + + intercept[AnalysisException] { + sql("CREATE TABLE doubleCreateAndInsertTest (key int, value string)") + } + } + } + + test("Double create does not fail when allowExisting = true") { + withTable("doubleCreateAndInsertTest") { + 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") { + val schema = StructType(StructField("m", MapType(StringType, StringType), true) :: Nil) + val rowRDD = spark.sparkContext.parallelize( + (1 to 100).map(i => Row(scala.collection.mutable.HashMap(s"key$i" -> s"value$i")))) + val df = spark.createDataFrame(rowRDD, schema) + df.createOrReplaceTempView("tableWithMapValue") + sql("CREATE TABLE hiveTableWithMapValue(m MAP )") + sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue") + + checkAnswer( + sql("SELECT * FROM hiveTableWithMapValue"), + rowRDD.collect().toSeq + ) + + sql("DROP TABLE hiveTableWithMapValue") + } + + test("SPARK-4203:random partition directory order") { + sql("CREATE TABLE tmp_table (key int, value string)") + val tmpDir = Utils.createTempDir() + // The default value of hive.exec.stagingdir. + val stagingDir = ".hive-staging" + + sql( + s""" + |CREATE TABLE table_with_partition(c1 string) + |PARTITIONED by (p1 string,p2 string,p3 string,p4 string,p5 string) + |location '${tmpDir.toURI.toString}' + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='1') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='2') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='3') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='4') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) + def listFolders(path: File, acc: List[String]): List[List[String]] = { + val dir = path.listFiles() + val folders = dir.filter { e => e.isDirectory && !e.getName().startsWith(stagingDir) }.toList + if (folders.isEmpty) { + List(acc.reverse) + } else { + folders.flatMap(x => listFolders(x, x.getName :: acc)) + } + } + val expected = List( + "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=2"::Nil, + "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=3"::Nil, + "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=1"::Nil, + "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=4"::Nil + ) + assert(listFolders(tmpDir, List()).sortBy(_.toString()) === expected.sortBy(_.toString)) + sql("DROP TABLE table_with_partition") + sql("DROP TABLE tmp_table") + } + + testPartitionedTable("INSERT OVERWRITE - partition IF NOT EXISTS") { tableName => + val selQuery = s"select a, b, c, d from $tableName" + sql( + s""" + |INSERT OVERWRITE TABLE $tableName + |partition (b=2, c=3) + |SELECT 1, 4 + """.stripMargin) + checkAnswer(sql(selQuery), Row(1, 2, 3, 4)) + + sql( + s""" + |INSERT OVERWRITE TABLE $tableName + |partition (b=2, c=3) + |SELECT 5, 6 + """.stripMargin) + checkAnswer(sql(selQuery), Row(5, 2, 3, 6)) + + val e = intercept[AnalysisException] { + sql( + s""" + |INSERT OVERWRITE TABLE $tableName + |partition (b=2, c) IF NOT EXISTS + |SELECT 7, 8, 3 + """.stripMargin) + } + assert(e.getMessage.contains( + "Dynamic partitions do not support IF NOT EXISTS. Specified partitions with value: [c]")) + + // If the partition already exists, the insert will overwrite the data + // unless users specify IF NOT EXISTS + sql( + s""" + |INSERT OVERWRITE TABLE $tableName + |partition (b=2, c=3) IF NOT EXISTS + |SELECT 9, 10 + """.stripMargin) + checkAnswer(sql(selQuery), Row(5, 2, 3, 6)) + + // ADD PARTITION has the same effect, even if no actual data is inserted. + sql(s"ALTER TABLE $tableName ADD PARTITION (b=21, c=31)") + sql( + s""" + |INSERT OVERWRITE TABLE $tableName + |partition (b=21, c=31) IF NOT EXISTS + |SELECT 20, 24 + """.stripMargin) + checkAnswer(sql(selQuery), Row(5, 2, 3, 6)) + } + + test("Insert ArrayType.containsNull == false") { + val schema = StructType(Seq( + StructField("a", ArrayType(StringType, containsNull = false)))) + val rowRDD = spark.sparkContext.parallelize((1 to 100).map(i => Row(Seq(s"value$i")))) + val df = spark.createDataFrame(rowRDD, schema) + df.createOrReplaceTempView("tableWithArrayValue") + sql("CREATE TABLE hiveTableWithArrayValue(a Array )") + sql("INSERT OVERWRITE TABLE hiveTableWithArrayValue SELECT a FROM tableWithArrayValue") + + checkAnswer( + sql("SELECT * FROM hiveTableWithArrayValue"), + rowRDD.collect().toSeq) + + sql("DROP TABLE hiveTableWithArrayValue") + } + + test("Insert MapType.valueContainsNull == false") { + val schema = StructType(Seq( + StructField("m", MapType(StringType, StringType, valueContainsNull = false)))) + val rowRDD = spark.sparkContext.parallelize( + (1 to 100).map(i => Row(Map(s"key$i" -> s"value$i")))) + val df = spark.createDataFrame(rowRDD, schema) + df.createOrReplaceTempView("tableWithMapValue") + sql("CREATE TABLE hiveTableWithMapValue(m Map )") + sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue") + + checkAnswer( + sql("SELECT * FROM hiveTableWithMapValue"), + rowRDD.collect().toSeq) + + sql("DROP TABLE hiveTableWithMapValue") + } + + test("Insert StructType.fields.exists(_.nullable == false)") { + val schema = StructType(Seq( + StructField("s", StructType(Seq(StructField("f", StringType, nullable = false)))))) + val rowRDD = spark.sparkContext.parallelize( + (1 to 100).map(i => Row(Row(s"value$i")))) + val df = spark.createDataFrame(rowRDD, schema) + df.createOrReplaceTempView("tableWithStructValue") + sql("CREATE TABLE hiveTableWithStructValue(s Struct )") + sql("INSERT OVERWRITE TABLE hiveTableWithStructValue SELECT s FROM tableWithStructValue") + + checkAnswer( + sql("SELECT * FROM hiveTableWithStructValue"), + rowRDD.collect().toSeq) + + sql("DROP TABLE hiveTableWithStructValue") + } + + test("Test partition mode = strict") { + withSQLConf(("hive.exec.dynamic.partition.mode", "strict")) { + withTable("partitioned") { + sql("CREATE TABLE partitioned (id bigint, data string) PARTITIONED BY (part string)") + val data = (1 to 10).map(i => (i, s"data-$i", if ((i % 2) == 0) "even" else "odd")) + .toDF("id", "data", "part") + + intercept[SparkException] { + data.write.insertInto("partitioned") + } + } + } + } + + test("Detect table partitioning") { + withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) { + withTable("source", "partitioned") { + sql("CREATE TABLE source (id bigint, data string, part string)") + val data = (1 to 10).map(i => (i, s"data-$i", if ((i % 2) == 0) "even" else "odd")).toDF() + + data.write.insertInto("source") + checkAnswer(sql("SELECT * FROM source"), data.collect().toSeq) + + sql("CREATE TABLE partitioned (id bigint, data string) PARTITIONED BY (part string)") + // this will pick up the output partitioning from the table definition + spark.table("source").write.insertInto("partitioned") + + checkAnswer(sql("SELECT * FROM partitioned"), data.collect().toSeq) + } + } + } + + private def testPartitionedHiveSerDeTable(testName: String)(f: String => Unit): Unit = { + test(s"Hive SerDe table - $testName") { + val hiveTable = "hive_table" + + withTable(hiveTable) { + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + sql( + s""" + |CREATE TABLE $hiveTable (a INT, d INT) + |PARTITIONED BY (b INT, c INT) STORED AS TEXTFILE + """.stripMargin) + f(hiveTable) + } + } + } + } + + private def testPartitionedDataSourceTable(testName: String)(f: String => Unit): Unit = { + test(s"Data source table - $testName") { + val dsTable = "ds_table" + + withTable(dsTable) { + sql( + s""" + |CREATE TABLE $dsTable (a INT, b INT, c INT, d INT) + |USING PARQUET PARTITIONED BY (b, c) + """.stripMargin) + f(dsTable) + } + } + } + + private def testPartitionedTable(testName: String)(f: String => Unit): Unit = { + testPartitionedHiveSerDeTable(testName)(f) + testPartitionedDataSourceTable(testName)(f) + } + + testPartitionedTable("partitionBy() can't be used together with insertInto()") { tableName => + val cause = intercept[AnalysisException] { + Seq((1, 2, 3, 4)).toDF("a", "b", "c", "d").write.partitionBy("b", "c").insertInto(tableName) + } + + assert(cause.getMessage.contains("insertInto() can't be used together with partitionBy().")) + } + + testPartitionedTable( + "SPARK-16036: better error message when insert into a table with mismatch schema") { + tableName => + val e = intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION(b=1, c=2) SELECT 1, 2, 3") + } + assert(e.message.contains( + "target table has 4 column(s) but the inserted data has 5 column(s)")) + } + + testPartitionedTable("SPARK-16037: INSERT statement should match columns by position") { + tableName => + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + sql(s"INSERT INTO TABLE $tableName SELECT 1, 4, 2 AS c, 3 AS b") + checkAnswer(sql(s"SELECT a, b, c, d FROM $tableName"), Row(1, 2, 3, 4)) + sql(s"INSERT OVERWRITE TABLE $tableName SELECT 1, 4, 2, 3") + checkAnswer(sql(s"SELECT a, b, c, 4 FROM $tableName"), Row(1, 2, 3, 4)) + } + } + + testPartitionedTable("INSERT INTO a partitioned table (semantic and error handling)") { + tableName => + withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) { + sql(s"INSERT INTO TABLE $tableName PARTITION (b=2, c=3) SELECT 1, 4") + + sql(s"INSERT INTO TABLE $tableName PARTITION (b=6, c=7) SELECT 5, 8") + + sql(s"INSERT INTO TABLE $tableName PARTITION (c=11, b=10) SELECT 9, 12") + + // c is defined twice. Analyzer will complain. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (b=14, c=15, c=16) SELECT 13") + } + + // d is not a partitioning column. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (b=14, c=15, d=16) SELECT 13, 14") + } + + // d is not a partitioning column. The total number of columns is correct. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (b=14, c=15, d=16) SELECT 13") + } + + // The data is missing a column. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (c=15, b=16) SELECT 13") + } + + // d is not a partitioning column. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (b=15, d=15) SELECT 13, 14") + } + + // The statement is missing a column. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (b=15) SELECT 13, 14") + } + + // The statement is missing a column. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (b=15) SELECT 13, 14, 16") + } + + sql(s"INSERT INTO TABLE $tableName PARTITION (b=14, c) SELECT 13, 16, 15") + + // Dynamic partitioning columns need to be after static partitioning columns. + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName PARTITION (b, c=19) SELECT 17, 20, 18") + } + + sql(s"INSERT INTO TABLE $tableName PARTITION (b, c) SELECT 17, 20, 18, 19") + + sql(s"INSERT INTO TABLE $tableName PARTITION (c, b) SELECT 21, 24, 22, 23") + + sql(s"INSERT INTO TABLE $tableName SELECT 25, 28, 26, 27") + + checkAnswer( + sql(s"SELECT a, b, c, d FROM $tableName"), + Row(1, 2, 3, 4) :: + Row(5, 6, 7, 8) :: + Row(9, 10, 11, 12) :: + Row(13, 14, 15, 16) :: + Row(17, 18, 19, 20) :: + Row(21, 22, 23, 24) :: + Row(25, 26, 27, 28) :: Nil + ) + } + } + + testPartitionedTable("insertInto() should match columns by position and ignore column names") { + tableName => + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + // Columns `df.c` and `df.d` are resolved by position, and thus mapped to partition columns + // `b` and `c` of the target table. + val df = Seq((1, 2, 3, 4)).toDF("a", "b", "c", "d") + df.write.insertInto(tableName) + + checkAnswer( + sql(s"SELECT a, b, c, d FROM $tableName"), + Row(1, 3, 4, 2) + ) + } + } + + testPartitionedTable("insertInto() should match unnamed columns by position") { + tableName => + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + // Columns `c + 1` and `d + 1` are resolved by position, and thus mapped to partition + // columns `b` and `c` of the target table. + val df = Seq((1, 2, 3, 4)).toDF("a", "b", "c", "d") + df.select('a + 1, 'b + 1, 'c + 1, 'd + 1).write.insertInto(tableName) + + checkAnswer( + sql(s"SELECT a, b, c, d FROM $tableName"), + Row(2, 4, 5, 3) + ) + } + } + + testPartitionedTable("insertInto() should reject missing columns") { + tableName => + withTable("t") { + sql("CREATE TABLE t (a INT, b INT)") + + intercept[AnalysisException] { + spark.table("t").write.insertInto(tableName) + } + } + } + + testPartitionedTable("insertInto() should reject extra columns") { + tableName => + withTable("t") { + sql("CREATE TABLE t (a INT, b INT, c INT, d INT, e INT)") + + intercept[AnalysisException] { + spark.table("t").write.insertInto(tableName) + } + } + } + + private def testBucketedTable(testName: String)(f: String => Unit): Unit = { + test(s"Hive SerDe table - $testName") { + val hiveTable = "hive_table" + + withTable(hiveTable) { + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + sql( + s""" + |CREATE TABLE $hiveTable (a INT, d INT) + |PARTITIONED BY (b INT, c INT) + |CLUSTERED BY(a) + |SORTED BY(a, d) INTO 256 BUCKETS + |STORED AS TEXTFILE + """.stripMargin) + f(hiveTable) + } + } + } + } + + testBucketedTable("INSERT should NOT fail if strict bucketing is NOT enforced") { + tableName => + withSQLConf("hive.enforce.bucketing" -> "false", "hive.enforce.sorting" -> "false") { + sql(s"INSERT INTO TABLE $tableName SELECT 1, 4, 2 AS c, 3 AS b") + checkAnswer(sql(s"SELECT a, b, c, d FROM $tableName"), Row(1, 2, 3, 4)) + } + } + + testBucketedTable("INSERT should fail if strict bucketing / sorting is enforced") { + tableName => + withSQLConf("hive.enforce.bucketing" -> "true", "hive.enforce.sorting" -> "false") { + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName SELECT 1, 2, 3, 4") + } + } + withSQLConf("hive.enforce.bucketing" -> "false", "hive.enforce.sorting" -> "true") { + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName SELECT 1, 2, 3, 4") + } + } + withSQLConf("hive.enforce.bucketing" -> "true", "hive.enforce.sorting" -> "true") { + intercept[AnalysisException] { + sql(s"INSERT INTO TABLE $tableName SELECT 1, 2, 3, 4") + } + } + } + + test("SPARK-20594: hive.exec.stagingdir was deleted by Hive") { + // Set hive.exec.stagingdir under the table directory without start with ".". + withSQLConf("hive.exec.stagingdir" -> "./test") { + withTable("test_table") { + sql("CREATE TABLE test_table (key int)") + sql("INSERT OVERWRITE TABLE test_table SELECT 1") + checkAnswer(sql("SELECT * FROM test_table"), Row(1)) + } + } + } + + test("insert overwrite to dir from hive metastore table") { + withTempDir { dir => + val path = dir.toURI.getPath + + sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10") + + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY '${path}' + |STORED AS orc + |SELECT * FROM src where key < 10 + """.stripMargin) + + // use orc data source to check the data of path is right. + withTempView("orc_source") { + sql( + s""" + |CREATE TEMPORARY VIEW orc_source + |USING org.apache.spark.sql.hive.orc + |OPTIONS ( + | PATH '${dir.getCanonicalPath}' + |) + """.stripMargin) + + checkAnswer( + sql("select * from orc_source"), + sql("select * from src where key < 10")) + } + } + } + + test("insert overwrite to local dir from temp table") { + withTempView("test_insert_table") { + spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") + + withTempDir { dir => + val path = dir.toURI.getPath + + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY '${path}' + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + """.stripMargin) + + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY '${path}' + |STORED AS orc + |SELECT * FROM test_insert_table + """.stripMargin) + + // use orc data source to check the data of path is right. + checkAnswer( + spark.read.orc(dir.getCanonicalPath), + sql("select * from test_insert_table")) + } + } + } + + test("insert overwrite to dir from temp table") { + withTempView("test_insert_table") { + spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") + + withTempDir { dir => + val pathUri = dir.toURI + + sql( + s""" + |INSERT OVERWRITE DIRECTORY '${pathUri}' + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + """.stripMargin) + + sql( + s""" + |INSERT OVERWRITE DIRECTORY '${pathUri}' + |STORED AS orc + |SELECT * FROM test_insert_table + """.stripMargin) + + // use orc data source to check the data of path is right. + checkAnswer( + spark.read.orc(dir.getCanonicalPath), + sql("select * from test_insert_table")) + } + } + } + + test("multi insert overwrite to dir") { + withTempView("test_insert_table") { + spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") + + withTempDir { dir => + val pathUri = dir.toURI + + withTempDir { dir2 => + val pathUri2 = dir2.toURI + + sql( + s""" + |FROM test_insert_table + |INSERT OVERWRITE DIRECTORY '${pathUri}' + |STORED AS orc + |SELECT id + |INSERT OVERWRITE DIRECTORY '${pathUri2}' + |STORED AS orc + |SELECT * + """.stripMargin) + + // use orc data source to check the data of path is right. + checkAnswer( + spark.read.orc(dir.getCanonicalPath), + sql("select id from test_insert_table")) + + checkAnswer( + spark.read.orc(dir2.getCanonicalPath), + sql("select * from test_insert_table")) + } + } + } + } + + test("insert overwrite to dir to illegal path") { + withTempView("test_insert_table") { + spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") + + val e = intercept[IllegalArgumentException] { + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY 'abc://a' + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + """.stripMargin) + }.getMessage + + assert(e.contains("Wrong FS: abc://a, expected: file:///")) + } + } + + test("insert overwrite to dir with mixed syntax") { + withTempView("test_insert_table") { + spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") + + val e = intercept[ParseException] { + sql( + s""" + |INSERT OVERWRITE DIRECTORY 'file://tmp' + |USING json + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + """.stripMargin) + }.getMessage + + assert(e.contains("mismatched input 'ROW'")) + } + } + + test("insert overwrite to dir with multi inserts") { + withTempView("test_insert_table") { + spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") + + val e = intercept[ParseException] { + sql( + s""" + |INSERT OVERWRITE DIRECTORY 'file://tmp2' + |USING json + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + |INSERT OVERWRITE DIRECTORY 'file://tmp2' + |USING json + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + """.stripMargin) + }.getMessage + + assert(e.contains("mismatched input 'ROW'")) + } + } +} 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 183aca29cf98..32db22e704b3 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 @@ -1,28 +1,28 @@ /* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.TestHiveSingleton -import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.QueryTest import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.hive.test.TestHiveSingleton class ListTablesSuite extends QueryTest with TestHiveSingleton with BeforeAndAfterAll { import hiveContext._ @@ -31,18 +31,25 @@ class ListTablesSuite extends QueryTest with TestHiveSingleton with BeforeAndAft val df = sparkContext.parallelize((1 to 10).map(i => (i, s"str$i"))).toDF("key", "value") override def beforeAll(): Unit = { + super.beforeAll() // The catalog in HiveContext is a case insensitive one. - catalog.registerTable(TableIdentifier("ListTablesSuiteTable"), df.logicalPlan) + sessionState.catalog.createTempView( + "ListTablesSuiteTable", df.logicalPlan, overrideIfExists = true) 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(TableIdentifier("ListTablesSuiteTable")) - sql("DROP TABLE IF EXISTS HiveListTablesSuiteTable") - sql("DROP TABLE IF EXISTS ListTablesSuiteDB.HiveInDBListTablesSuiteTable") - sql("DROP DATABASE IF EXISTS ListTablesSuiteDB") + try { + sessionState.catalog.dropTable( + TableIdentifier("ListTablesSuiteTable"), ignoreIfNotExists = true, purge = false) + sql("DROP TABLE IF EXISTS HiveListTablesSuiteTable") + sql("DROP TABLE IF EXISTS ListTablesSuiteDB.HiveInDBListTablesSuiteTable") + sql("DROP DATABASE IF EXISTS ListTablesSuiteDB") + } finally { + super.afterAll() + } } test("get all tables of current database") { @@ -51,10 +58,10 @@ class ListTablesSuite extends QueryTest with TestHiveSingleton with BeforeAndAft // We are using default DB. checkAnswer( allTables.filter("tableName = 'listtablessuitetable'"), - Row("listtablessuitetable", true)) + Row("", "listtablessuitetable", true)) checkAnswer( allTables.filter("tableName = 'hivelisttablessuitetable'"), - Row("hivelisttablessuitetable", false)) + Row("default", "hivelisttablessuitetable", false)) assert(allTables.filter("tableName = 'hiveindblisttablessuitetable'").count() === 0) } } @@ -64,11 +71,11 @@ class ListTablesSuite extends QueryTest with TestHiveSingleton with BeforeAndAft case allTables => checkAnswer( allTables.filter("tableName = 'listtablessuitetable'"), - Row("listtablessuitetable", true)) + Row("", "listtablessuitetable", true)) assert(allTables.filter("tableName = 'hivelisttablessuitetable'").count() === 0) checkAnswer( allTables.filter("tableName = 'hiveindblisttablessuitetable'"), - Row("hiveindblisttablessuitetable", false)) + Row("listtablessuitedb", "hiveindblisttablessuitetable", false)) } } } 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 f74eb1500b98..29b0e6c8533e 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 @@ -17,20 +17,25 @@ package org.apache.spark.sql.hive -import java.io.{IOException, File} +import java.io.File import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.fs.Path +import org.apache.spark.SparkContext import org.apache.spark.sql._ -import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.hive.client.{HiveTable, ManagedTable} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.execution.command.CreateTableCommand +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.hive.HiveExternalCatalog._ +import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ -import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.util.Utils /** @@ -38,11 +43,12 @@ import org.apache.spark.util.Utils */ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import hiveContext._ - import hiveContext.implicits._ + import spark.implicits._ var jsonFilePath: String = _ override def beforeAll(): Unit = { + super.beforeAll() jsonFilePath = Utils.getSparkClassLoader.getResource("sample.json").getFile } @@ -76,8 +82,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv |) """.stripMargin) - withTempTable("expectedJsonTable") { - read.json(jsonFilePath).registerTempTable("expectedJsonTable") + withTempView("expectedJsonTable") { + read.json(jsonFilePath).createOrReplaceTempView("expectedJsonTable") checkAnswer( sql("SELECT a, b, `c_!@(3)`, ``.`d!`, ``.`=` FROM jsonTable"), sql("SELECT a, b, `c_!@(3)`, ``.`d!`, ``.`=` FROM expectedJsonTable")) @@ -106,8 +112,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv assert(expectedSchema === table("jsonTable").schema) - withTempTable("expectedJsonTable") { - read.json(jsonFilePath).registerTempTable("expectedJsonTable") + withTempView("expectedJsonTable") { + read.json(jsonFilePath).createOrReplaceTempView("expectedJsonTable") checkAnswer( sql("SELECT b, ``.`=` FROM jsonTable"), sql("SELECT b, ``.`=` FROM expectedJsonTable")) @@ -163,13 +169,13 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv test("check change without refresh") { withTempPath { tempDir => withTable("jsonTable") { - (("a", "b") :: Nil).toDF().toJSON.saveAsTextFile(tempDir.getCanonicalPath) + (("a", "b") :: Nil).toDF().toJSON.rdd.saveAsTextFile(tempDir.getCanonicalPath) sql( s"""CREATE TABLE jsonTable |USING org.apache.spark.sql.json |OPTIONS ( - | path '${tempDir.getCanonicalPath}' + | path '${tempDir.toURI}' |) """.stripMargin) @@ -178,7 +184,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv Row("a", "b")) Utils.deleteRecursively(tempDir) - (("a1", "b1", "c1") :: Nil).toDF().toJSON.saveAsTextFile(tempDir.getCanonicalPath) + (("a1", "b1", "c1") :: Nil).toDF().toJSON.rdd.saveAsTextFile(tempDir.getCanonicalPath) // Schema is cached so the new column does not show. The updated values in existing columns // will show. @@ -188,24 +194,24 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sql("REFRESH TABLE jsonTable") - // Check that the refresh worked + // After refresh, schema is not changed. checkAnswer( sql("SELECT * FROM jsonTable"), - Row("a1", "b1", "c1")) + Row("a1", "b1")) } } } test("drop, change, recreate") { withTempPath { tempDir => - (("a", "b") :: Nil).toDF().toJSON.saveAsTextFile(tempDir.getCanonicalPath) + (("a", "b") :: Nil).toDF().toJSON.rdd.saveAsTextFile(tempDir.getCanonicalPath) withTable("jsonTable") { sql( s"""CREATE TABLE jsonTable |USING org.apache.spark.sql.json |OPTIONS ( - | path '${tempDir.getCanonicalPath}' + | path '${tempDir.toURI}' |) """.stripMargin) @@ -214,7 +220,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv Row("a", "b")) Utils.deleteRecursively(tempDir) - (("a", "b", "c") :: Nil).toDF().toJSON.saveAsTextFile(tempDir.getCanonicalPath) + (("a", "b", "c") :: Nil).toDF().toJSON.rdd.saveAsTextFile(tempDir.getCanonicalPath) sql("DROP TABLE jsonTable") @@ -222,7 +228,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv s"""CREATE TABLE jsonTable |USING org.apache.spark.sql.json |OPTIONS ( - | path '${tempDir.getCanonicalPath}' + | path '${tempDir.toURI}' |) """.stripMargin) @@ -244,21 +250,21 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv |) """.stripMargin) - withTempTable("expectedJsonTable") { - read.json(jsonFilePath).registerTempTable("expectedJsonTable") + withTempView("expectedJsonTable") { + read.json(jsonFilePath).createOrReplaceTempView("expectedJsonTable") checkAnswer( sql("SELECT * FROM jsonTable"), sql("SELECT `c_!@(3)` FROM expectedJsonTable").collect().toSeq) // Discard the cached relation. - invalidateTable("jsonTable") + sessionState.refreshTable("jsonTable") checkAnswer( sql("SELECT * FROM jsonTable"), sql("SELECT `c_!@(3)` FROM expectedJsonTable").collect().toSeq) - invalidateTable("jsonTable") + sessionState.refreshTable("jsonTable") val expectedSchema = StructType(StructField("c_!@(3)", IntegerType, true) :: Nil) assert(expectedSchema === table("jsonTable").schema) @@ -281,7 +287,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv s"""CREATE TABLE ctasJsonTable |USING org.apache.spark.sql.json.DefaultSource |OPTIONS ( - | path '$tempPath' + | path '${tempPath.toURI}' |) AS |SELECT * FROM jsonTable """.stripMargin) @@ -297,7 +303,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv test("CTAS with IF NOT EXISTS") { withTempPath { path => - val tempPath = path.getCanonicalPath + val tempPath = path.toURI withTable("jsonTable", "ctasJsonTable") { sql( @@ -330,7 +336,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv }.getMessage assert( - message.contains("Table ctasJsonTable already exists."), + message.contains("Table default.ctasJsonTable already exists."), "We should complain that ctasJsonTable already exists") // The following statement should be fine if it has IF NOT EXISTS. @@ -346,7 +352,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv """.stripMargin) // Discard the cached relation. - invalidateTable("ctasJsonTable") + sessionState.refreshTable("ctasJsonTable") // Schema should not be changed. assert(table("ctasJsonTable").schema === table("jsonTable").schema) @@ -368,10 +374,10 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv |) """.stripMargin) - val expectedPath = catalog.hiveDefaultTableFilePath(TableIdentifier("ctasJsonTable")) + val expectedPath = sessionState.catalog.defaultTablePath(TableIdentifier("ctasJsonTable")) val filesystemPath = new Path(expectedPath) - val fs = filesystemPath.getFileSystem(sparkContext.hadoopConfiguration) - if (fs.exists(filesystemPath)) fs.delete(filesystemPath, true) + val fs = filesystemPath.getFileSystem(spark.sessionState.newHadoopConf()) + fs.delete(filesystemPath, true) // It is a managed table when we do not specify the location. sql( @@ -402,17 +408,17 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv } } - test("SPARK-5286 Fail to drop an invalid table when using the data source API") { - withTable("jsonTable") { - sql( - s"""CREATE TABLE jsonTable - |USING org.apache.spark.sql.json.DefaultSource - |OPTIONS ( - | path 'it is not a path at all!' - |) - """.stripMargin) + test("saveAsTable(CTAS) using append and insertInto when the target table is Hive serde") { + val tableName = "tab1" + withTable(tableName) { + sql(s"CREATE TABLE $tableName STORED AS SEQUENCEFILE AS SELECT 1 AS key, 'abc' AS value") - sql("DROP TABLE jsonTable").collect().foreach(i => logInfo(i.toString)) + val df = sql(s"SELECT key, value FROM $tableName") + df.write.insertInto(tableName) + checkAnswer( + sql(s"SELECT * FROM $tableName"), + Row(1, "abc") :: Row(1, "abc") :: Nil + ) } } @@ -434,7 +440,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sql("SELECT * FROM savedJsonTable tmp where tmp.a > 5"), (6 to 10).map(i => Row(i, s"str$i"))) - invalidateTable("savedJsonTable") + sessionState.refreshTable("savedJsonTable") checkAnswer( sql("SELECT * FROM savedJsonTable where savedJsonTable.a < 5"), @@ -472,8 +478,9 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv // Drop table will also delete the data. sql("DROP TABLE savedJsonTable") - intercept[IOException] { - read.json(catalog.hiveDefaultTableFilePath(TableIdentifier("savedJsonTable"))) + intercept[AnalysisException] { + read.json( + sessionState.catalog.defaultTablePath(TableIdentifier("savedJsonTable")).toString) } } @@ -498,9 +505,9 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv test("create external table") { withTempPath { tempPath => withTable("savedJsonTable", "createdJsonTable") { - val df = read.json(sparkContext.parallelize((1 to 10).map { i => + val df = read.json((1 to 10).map { i => s"""{ "a": $i, "b": "str$i" }""" - })) + }.toDS()) withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "not a source name") { df.write @@ -511,13 +518,13 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv } withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "json") { - createExternalTable("createdJsonTable", tempPath.toString) + sparkSession.catalog.createExternalTable("createdJsonTable", tempPath.toString) assert(table("createdJsonTable").schema === df.schema) checkAnswer(sql("SELECT * FROM createdJsonTable"), df) assert( intercept[AnalysisException] { - createExternalTable("createdJsonTable", jsonFilePath.toString) + sparkSession.catalog.createExternalTable("createdJsonTable", jsonFilePath.toString) }.getMessage.contains("Table createdJsonTable already exists."), "We should complain that createdJsonTable already exists") } @@ -529,7 +536,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv // Try to specify the schema. withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "not a source name") { val schema = StructType(StructField("b", StringType, true) :: Nil) - createExternalTable( + sparkSession.catalog.createExternalTable( "createdJsonTable", "org.apache.spark.sql.json", schema, @@ -540,25 +547,30 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sql("SELECT b FROM savedJsonTable")) sql("DROP TABLE createdJsonTable") - - assert( - intercept[RuntimeException] { - createExternalTable( - "createdJsonTable", - "org.apache.spark.sql.json", - schema, - Map.empty[String, String]) - }.getMessage.contains("key not found: path"), - "We should complain that path is not specified.") } } } } + test("path required error") { + assert( + intercept[AnalysisException] { + sparkSession.catalog.createExternalTable( + "createdJsonTable", + "org.apache.spark.sql.json", + Map.empty[String, String]) + + table("createdJsonTable") + }.getMessage.contains("Unable to infer schema"), + "We should complain that path is not specified.") + + sql("DROP TABLE IF EXISTS createdJsonTable") + } + test("scan a parquet table created through a CTAS statement") { - withSQLConf(HiveContext.CONVERT_METASTORE_PARQUET.key -> "true") { - withTempTable("jt") { - (1 to 10).map(i => i -> s"str$i").toDF("a", "b").registerTempTable("jt") + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "true") { + withTempView("jt") { + (1 to 10).map(i => i -> s"str$i").toDF("a", "b").createOrReplaceTempView("jt") withTable("test_parquet_ctas") { sql( @@ -571,9 +583,9 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv Row(3) :: Row(4) :: Nil) table("test_parquet_ctas").queryExecution.optimizedPlan match { - case LogicalRelation(p: ParquetRelation, _) => // OK + case LogicalRelation(p: HadoopFsRelation, _, _, _) => // OK case _ => - fail(s"test_parquet_ctas should have be converted to ${classOf[ParquetRelation]}") + fail(s"test_parquet_ctas should have be converted to ${classOf[HadoopFsRelation]}") } } } @@ -626,7 +638,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .mode(SaveMode.Append) .saveAsTable("arrayInParquet") - refreshTable("arrayInParquet") + sparkSession.catalog.refreshTable("arrayInParquet") checkAnswer( sql("SELECT a FROM arrayInParquet"), @@ -685,7 +697,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .mode(SaveMode.Append) .saveAsTable("mapInParquet") - refreshTable("mapInParquet") + sparkSession.catalog.refreshTable("mapInParquet") checkAnswer( sql("SELECT a FROM mapInParquet"), @@ -697,21 +709,24 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv } test("SPARK-6024 wide schema support") { - withSQLConf(SQLConf.SCHEMA_STRING_LENGTH_THRESHOLD.key -> "4000") { - withTable("wide_schema") { + assert(spark.sparkContext.conf.get(SCHEMA_STRING_LENGTH_THRESHOLD) == 4000) + withTable("wide_schema") { + withTempDir { tempDir => // We will need 80 splits for this schema if the threshold is 4000. - val schema = StructType((1 to 5000).map(i => StructField(s"c_$i", StringType, true))) + val schema = StructType((1 to 5000).map(i => StructField(s"c_$i", StringType))) - // Manually create a metastore data source table. - catalog.createDataSourceTable( - tableIdent = TableIdentifier("wide_schema"), - userSpecifiedSchema = Some(schema), - partitionColumns = Array.empty[String], - provider = "json", - options = Map("path" -> "just a dummy path"), - isExternal = false) + val tableDesc = CatalogTable( + identifier = TableIdentifier("wide_schema"), + tableType = CatalogTableType.EXTERNAL, + storage = CatalogStorageFormat.empty.copy( + properties = Map("path" -> tempDir.getCanonicalPath) + ), + schema = schema, + provider = Some("json") + ) + spark.sessionState.catalog.createTable(tableDesc, ignoreIfExists = false) - invalidateTable("wide_schema") + sessionState.refreshTable("wide_schema") val actualSchema = table("wide_schema").schema assert(schema === actualSchema) @@ -723,44 +738,50 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv val tableName = "spark6655" withTable(tableName) { val schema = StructType(StructField("int", IntegerType, true) :: Nil) - val hiveTable = HiveTable( - specifiedDatabase = Some("default"), - name = tableName, - schema = Seq.empty, - partitionColumns = Seq.empty, + val hiveTable = CatalogTable( + identifier = TableIdentifier(tableName, Some("default")), + tableType = CatalogTableType.MANAGED, + schema = new StructType, + provider = Some("json"), + storage = CatalogStorageFormat( + locationUri = None, + inputFormat = None, + outputFormat = None, + serde = None, + compressed = false, + properties = Map( + "path" -> sessionState.catalog.defaultTablePath(TableIdentifier(tableName)).toString) + ), properties = Map( - "spark.sql.sources.provider" -> "json", - "spark.sql.sources.schema" -> schema.json, - "EXTERNAL" -> "FALSE"), - tableType = ManagedTable, - serdeProperties = Map( - "path" -> catalog.hiveDefaultTableFilePath(TableIdentifier(tableName)))) + DATASOURCE_PROVIDER -> "json", + DATASOURCE_SCHEMA -> schema.json, + "EXTERNAL" -> "FALSE")) - catalog.client.createTable(hiveTable) + hiveClient.createTable(hiveTable, ignoreIfExists = false) - invalidateTable(tableName) + sessionState.refreshTable(tableName) val actualSchema = table(tableName).schema assert(schema === actualSchema) } } - test("Saving partition columns information") { + test("Saving partitionBy columns information") { val df = (1 to 10).map(i => (i, i + 1, s"str$i", s"str${i + 1}")).toDF("a", "b", "c", "d") val tableName = s"partitionInfo_${System.currentTimeMillis()}" withTable(tableName) { df.write.format("parquet").partitionBy("d", "b").saveAsTable(tableName) - invalidateTable(tableName) - val metastoreTable = catalog.client.getTable("default", tableName) + sessionState.refreshTable(tableName) + val metastoreTable = hiveClient.getTable("default", tableName) val expectedPartitionColumns = StructType(df.schema("d") :: df.schema("b") :: Nil) - val numPartCols = metastoreTable.properties("spark.sql.sources.schema.numPartCols").toInt + val numPartCols = metastoreTable.properties(DATASOURCE_SCHEMA_NUMPARTCOLS).toInt assert(numPartCols == 2) val actualPartitionColumns = StructType( (0 until numPartCols).map { index => - df.schema(metastoreTable.properties(s"spark.sql.sources.schema.partCol.$index")) + df.schema(metastoreTable.properties(s"$DATASOURCE_SCHEMA_PARTCOL_PREFIX$index")) }) // Make sure partition columns are correctly stored in metastore. assert( @@ -775,6 +796,59 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv } } + test("Saving information for sortBy and bucketBy columns") { + val df = (1 to 10).map(i => (i, i + 1, s"str$i", s"str${i + 1}")).toDF("a", "b", "c", "d") + val tableName = s"bucketingInfo_${System.currentTimeMillis()}" + + withTable(tableName) { + df.write + .format("parquet") + .bucketBy(8, "d", "b") + .sortBy("c") + .saveAsTable(tableName) + sessionState.refreshTable(tableName) + val metastoreTable = hiveClient.getTable("default", tableName) + val expectedBucketByColumns = StructType(df.schema("d") :: df.schema("b") :: Nil) + val expectedSortByColumns = StructType(df.schema("c") :: Nil) + + val numBuckets = metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETS).toInt + assert(numBuckets == 8) + + val numBucketCols = metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETCOLS).toInt + assert(numBucketCols == 2) + + val numSortCols = metastoreTable.properties(DATASOURCE_SCHEMA_NUMSORTCOLS).toInt + assert(numSortCols == 1) + + val actualBucketByColumns = + StructType( + (0 until numBucketCols).map { index => + df.schema(metastoreTable.properties(s"$DATASOURCE_SCHEMA_BUCKETCOL_PREFIX$index")) + }) + // Make sure bucketBy columns are correctly stored in metastore. + assert( + expectedBucketByColumns.sameType(actualBucketByColumns), + s"Partitions columns stored in metastore $actualBucketByColumns is not the " + + s"partition columns defined by the saveAsTable operation $expectedBucketByColumns.") + + val actualSortByColumns = + StructType( + (0 until numSortCols).map { index => + df.schema(metastoreTable.properties(s"$DATASOURCE_SCHEMA_SORTCOL_PREFIX$index")) + }) + // Make sure sortBy columns are correctly stored in metastore. + assert( + expectedSortByColumns.sameType(actualSortByColumns), + s"Partitions columns stored in metastore $actualSortByColumns is not the " + + s"partition columns defined by the saveAsTable operation $expectedSortByColumns.") + + // Check the content of the saved table. + checkAnswer( + table(tableName).select("c", "b", "d", "a"), + df.select("c", "b", "d", "a")) + } + } + test("insert into a table") { def createDF(from: Int, to: Int): DataFrame = { (from to to).map(i => i -> s"str$i").toDF("c1", "c2") @@ -830,20 +904,454 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv } } + test("append table using different formats") { + def createDF(from: Int, to: Int): DataFrame = { + (from to to).map(i => i -> s"str$i").toDF("c1", "c2") + } + + withTable("appendOrcToParquet") { + createDF(0, 9).write.format("parquet").saveAsTable("appendOrcToParquet") + val e = intercept[AnalysisException] { + createDF(10, 19).write.mode(SaveMode.Append).format("orc").saveAsTable("appendOrcToParquet") + } + assert(e.getMessage.contains( + "The format of the existing table default.appendOrcToParquet is `ParquetFileFormat`. " + + "It doesn't match the specified format `OrcFileFormat`")) + } + + withTable("appendParquetToJson") { + createDF(0, 9).write.format("json").saveAsTable("appendParquetToJson") + val e = intercept[AnalysisException] { + createDF(10, 19).write.mode(SaveMode.Append).format("parquet") + .saveAsTable("appendParquetToJson") + } + assert(e.getMessage.contains( + "The format of the existing table default.appendParquetToJson is `JsonFileFormat`. " + + "It doesn't match the specified format `ParquetFileFormat`")) + } + + withTable("appendTextToJson") { + createDF(0, 9).write.format("json").saveAsTable("appendTextToJson") + val e = intercept[AnalysisException] { + createDF(10, 19).write.mode(SaveMode.Append).format("text") + .saveAsTable("appendTextToJson") + } + assert(e.getMessage.contains( + "The format of the existing table default.appendTextToJson is `JsonFileFormat`. " + + "It doesn't match the specified format `TextFileFormat`")) + } + } + + test("append a table using the same formats but different names") { + def createDF(from: Int, to: Int): DataFrame = { + (from to to).map(i => i -> s"str$i").toDF("c1", "c2") + } + + withTable("appendParquet") { + createDF(0, 9).write.format("parquet").saveAsTable("appendParquet") + createDF(10, 19).write.mode(SaveMode.Append).format("org.apache.spark.sql.parquet") + .saveAsTable("appendParquet") + checkAnswer( + sql("SELECT p.c1, p.c2 FROM appendParquet p WHERE p.c1 > 5"), + (6 to 19).map(i => Row(i, s"str$i"))) + } + + withTable("appendParquet") { + createDF(0, 9).write.format("org.apache.spark.sql.parquet").saveAsTable("appendParquet") + createDF(10, 19).write.mode(SaveMode.Append).format("parquet").saveAsTable("appendParquet") + checkAnswer( + sql("SELECT p.c1, p.c2 FROM appendParquet p WHERE p.c1 > 5"), + (6 to 19).map(i => Row(i, s"str$i"))) + } + + withTable("appendParquet") { + createDF(0, 9).write.format("org.apache.spark.sql.parquet.DefaultSource") + .saveAsTable("appendParquet") + createDF(10, 19).write.mode(SaveMode.Append) + .format("org.apache.spark.sql.execution.datasources.parquet.DefaultSource") + .saveAsTable("appendParquet") + checkAnswer( + sql("SELECT p.c1, p.c2 FROM appendParquet p WHERE p.c1 > 5"), + (6 to 19).map(i => Row(i, s"str$i"))) + } + } + test("SPARK-8156:create table to specific database by 'use dbname' ") { val df = (1 to 3).map(i => (i, s"val_$i", i * 2)).toDF("a", "b", "c") - sqlContext.sql("""create database if not exists testdb8156""") - sqlContext.sql("""use testdb8156""") + spark.sql("""create database if not exists testdb8156""") + spark.sql("""use testdb8156""") df.write .format("parquet") .mode(SaveMode.Overwrite) .saveAsTable("ttt3") checkAnswer( - sqlContext.sql("show TABLES in testdb8156").filter("tableName = 'ttt3'"), - Row("ttt3", false)) - sqlContext.sql("""use default""") - sqlContext.sql("""drop database if exists testdb8156 CASCADE""") + spark.sql("show TABLES in testdb8156").filter("tableName = 'ttt3'"), + Row("testdb8156", "ttt3", false)) + spark.sql("""use default""") + spark.sql("""drop database if exists testdb8156 CASCADE""") + } + + + test("skip hive metadata on table creation") { + withTempDir { tempPath => + val schema = StructType((1 to 5).map(i => StructField(s"c_$i", StringType))) + + val tableDesc1 = CatalogTable( + identifier = TableIdentifier("not_skip_hive_metadata"), + tableType = CatalogTableType.EXTERNAL, + storage = CatalogStorageFormat.empty.copy( + locationUri = Some(tempPath.toURI), + properties = Map("skipHiveMetadata" -> "false") + ), + schema = schema, + provider = Some("parquet") + ) + spark.sessionState.catalog.createTable(tableDesc1, ignoreIfExists = false) + + // As a proxy for verifying that the table was stored in Hive compatible format, + // we verify that each column of the table is of native type StringType. + assert(hiveClient.getTable("default", "not_skip_hive_metadata").schema + .forall(_.dataType == StringType)) + + val tableDesc2 = CatalogTable( + identifier = TableIdentifier("skip_hive_metadata", Some("default")), + tableType = CatalogTableType.EXTERNAL, + storage = CatalogStorageFormat.empty.copy( + properties = Map("path" -> tempPath.getCanonicalPath, "skipHiveMetadata" -> "true") + ), + schema = schema, + provider = Some("parquet") + ) + spark.sessionState.catalog.createTable(tableDesc2, ignoreIfExists = false) + + // As a proxy for verifying that the table was stored in SparkSQL format, + // we verify that the table has a column type as array of StringType. + assert(hiveClient.getTable("default", "skip_hive_metadata").schema + .forall(_.dataType == ArrayType(StringType))) + } + } + + test("CTAS: persisted partitioned data source table") { + withTempPath { dir => + withTable("t") { + sql( + s"""CREATE TABLE t USING PARQUET + |OPTIONS (PATH '${dir.toURI}') + |PARTITIONED BY (a) + |AS SELECT 1 AS a, 2 AS b + """.stripMargin + ) + + val metastoreTable = hiveClient.getTable("default", "t") + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMPARTCOLS).toInt === 1) + assert(!metastoreTable.properties.contains(DATASOURCE_SCHEMA_NUMBUCKETS)) + assert(!metastoreTable.properties.contains(DATASOURCE_SCHEMA_NUMBUCKETCOLS)) + assert(!metastoreTable.properties.contains(DATASOURCE_SCHEMA_NUMSORTCOLS)) + + checkAnswer(table("t"), Row(2, 1)) + } + } + } + + test("CTAS: persisted bucketed data source table") { + withTempPath { dir => + withTable("t") { + sql( + s"""CREATE TABLE t USING PARQUET + |OPTIONS (PATH '${dir.toURI}') + |CLUSTERED BY (a) SORTED BY (b) INTO 2 BUCKETS + |AS SELECT 1 AS a, 2 AS b + """.stripMargin + ) + + val metastoreTable = hiveClient.getTable("default", "t") + assert(!metastoreTable.properties.contains(DATASOURCE_SCHEMA_NUMPARTCOLS)) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETS).toInt === 2) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETCOLS).toInt === 1) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMSORTCOLS).toInt === 1) + + checkAnswer(table("t"), Row(1, 2)) + } + } + + withTempPath { dir => + withTable("t") { + sql( + s"""CREATE TABLE t USING PARQUET + |OPTIONS (PATH '${dir.toURI}') + |CLUSTERED BY (a) INTO 2 BUCKETS + |AS SELECT 1 AS a, 2 AS b + """.stripMargin + ) + + val metastoreTable = hiveClient.getTable("default", "t") + assert(!metastoreTable.properties.contains(DATASOURCE_SCHEMA_NUMPARTCOLS)) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETS).toInt === 2) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETCOLS).toInt === 1) + assert(!metastoreTable.properties.contains(DATASOURCE_SCHEMA_NUMSORTCOLS)) + + checkAnswer(table("t"), Row(1, 2)) + } + } + } + + test("CTAS: persisted partitioned bucketed data source table") { + withTempPath { dir => + withTable("t") { + sql( + s"""CREATE TABLE t USING PARQUET + |OPTIONS (PATH '${dir.toURI}') + |PARTITIONED BY (a) + |CLUSTERED BY (b) SORTED BY (c) INTO 2 BUCKETS + |AS SELECT 1 AS a, 2 AS b, 3 AS c + """.stripMargin + ) + + val metastoreTable = hiveClient.getTable("default", "t") + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMPARTCOLS).toInt === 1) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETS).toInt === 2) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMBUCKETCOLS).toInt === 1) + assert(metastoreTable.properties(DATASOURCE_SCHEMA_NUMSORTCOLS).toInt === 1) + + checkAnswer(table("t"), Row(2, 3, 1)) + } + } + } + + test("saveAsTable[append]: the column order doesn't matter") { + withTable("saveAsTable_column_order") { + Seq((1, 2)).toDF("i", "j").write.saveAsTable("saveAsTable_column_order") + Seq((3, 4)).toDF("j", "i").write.mode("append").saveAsTable("saveAsTable_column_order") + checkAnswer( + table("saveAsTable_column_order"), + Seq((1, 2), (4, 3)).toDF("i", "j")) + } + } + + test("saveAsTable[append]: mismatch column names") { + withTable("saveAsTable_mismatch_column_names") { + Seq((1, 2)).toDF("i", "j").write.saveAsTable("saveAsTable_mismatch_column_names") + val e = intercept[AnalysisException] { + Seq((3, 4)).toDF("i", "k") + .write.mode("append").saveAsTable("saveAsTable_mismatch_column_names") + } + assert(e.getMessage.contains("cannot resolve")) + } + } + + test("saveAsTable[append]: too many columns") { + withTable("saveAsTable_too_many_columns") { + Seq((1, 2)).toDF("i", "j").write.saveAsTable("saveAsTable_too_many_columns") + val e = intercept[AnalysisException] { + Seq((3, 4, 5)).toDF("i", "j", "k") + .write.mode("append").saveAsTable("saveAsTable_too_many_columns") + } + assert(e.getMessage.contains("doesn't match")) + } + } + + test("create a temp view using hive") { + val tableName = "tab1" + withTable(tableName) { + val e = intercept[AnalysisException] { + sql( + s""" + |CREATE TEMPORARY VIEW $tableName + |(col1 int) + |USING hive + """.stripMargin) + }.getMessage + assert(e.contains("Hive data source can only be used with tables, you can't use it with " + + "CREATE TEMP VIEW USING")) + } + } + + test("saveAsTable - source and target are the same table") { + val tableName = "tab1" + withTable(tableName) { + Seq((1, 2)).toDF("i", "j").write.saveAsTable(tableName) + + table(tableName).write.mode(SaveMode.Append).saveAsTable(tableName) + checkAnswer(table(tableName), + Seq(Row(1, 2), Row(1, 2))) + + table(tableName).write.mode(SaveMode.Ignore).saveAsTable(tableName) + checkAnswer(table(tableName), + Seq(Row(1, 2), Row(1, 2))) + + var e = intercept[AnalysisException] { + table(tableName).write.mode(SaveMode.Overwrite).saveAsTable(tableName) + }.getMessage + assert(e.contains(s"Cannot overwrite table default.$tableName that is also being read from")) + + e = intercept[AnalysisException] { + table(tableName).write.mode(SaveMode.ErrorIfExists).saveAsTable(tableName) + }.getMessage + assert(e.contains(s"Table `$tableName` already exists")) + } + } + + test("insertInto - source and target are the same table") { + val tableName = "tab1" + withTable(tableName) { + Seq((1, 2)).toDF("i", "j").write.saveAsTable(tableName) + + table(tableName).write.mode(SaveMode.Append).insertInto(tableName) + checkAnswer( + table(tableName), + Seq(Row(1, 2), Row(1, 2))) + + table(tableName).write.mode(SaveMode.Ignore).insertInto(tableName) + checkAnswer( + table(tableName), + Seq(Row(1, 2), Row(1, 2), Row(1, 2), Row(1, 2))) + + table(tableName).write.mode(SaveMode.ErrorIfExists).insertInto(tableName) + checkAnswer( + table(tableName), + Seq(Row(1, 2), Row(1, 2), Row(1, 2), Row(1, 2), Row(1, 2), Row(1, 2), Row(1, 2), Row(1, 2))) + + val e = intercept[AnalysisException] { + table(tableName).write.mode(SaveMode.Overwrite).insertInto(tableName) + }.getMessage + assert(e.contains(s"Cannot overwrite a path that is also being read from")) + } + } + + test("saveAsTable[append]: less columns") { + withTable("saveAsTable_less_columns") { + Seq((1, 2)).toDF("i", "j").write.saveAsTable("saveAsTable_less_columns") + val e = intercept[AnalysisException] { + Seq((4)).toDF("j") + .write.mode("append").saveAsTable("saveAsTable_less_columns") + } + assert(e.getMessage.contains("doesn't match")) + } + } + + test("SPARK-15025: create datasource table with path with select") { + withTempPath { dir => + withTable("t") { + sql( + s"""CREATE TABLE t USING PARQUET + |OPTIONS (PATH '${dir.toURI}') + |AS SELECT 1 AS a, 2 AS b, 3 AS c + """.stripMargin + ) + sql("insert into t values (2, 3, 4)") + checkAnswer(table("t"), Seq(Row(1, 2, 3), Row(2, 3, 4))) + val catalogTable = hiveClient.getTable("default", "t") + assert(catalogTable.storage.locationUri.isDefined) + } + } + } + + test("SPARK-15269 external data source table creation") { + withTempPath { dir => + val path = dir.toURI.toString + spark.range(1).write.json(path) + + withTable("t") { + sql(s"CREATE TABLE t USING json OPTIONS (PATH '$path')") + sql("DROP TABLE t") + sql(s"CREATE TABLE t USING json AS SELECT 1 AS c") + } + } + } + + test("read table with corrupted schema") { + try { + val schema = StructType(StructField("int", IntegerType, true) :: Nil) + val hiveTable = CatalogTable( + identifier = TableIdentifier("t", Some("default")), + tableType = CatalogTableType.MANAGED, + schema = new StructType, + provider = Some("json"), + storage = CatalogStorageFormat.empty, + properties = Map( + DATASOURCE_PROVIDER -> "json", + // no DATASOURCE_SCHEMA_NUMPARTS + DATASOURCE_SCHEMA_PART_PREFIX + 0 -> schema.json)) + + hiveClient.createTable(hiveTable, ignoreIfExists = false) + + val e = intercept[AnalysisException] { + sharedState.externalCatalog.getTable("default", "t") + }.getMessage + assert(e.contains(s"Could not read schema from the hive metastore because it is corrupted")) + + withDebugMode { + val tableMeta = sharedState.externalCatalog.getTable("default", "t") + assert(tableMeta.identifier == TableIdentifier("t", Some("default"))) + assert(tableMeta.properties(DATASOURCE_PROVIDER) == "json") + } + } finally { + hiveClient.dropTable("default", "t", ignoreIfNotExists = true, purge = true) + } + } + + test("should keep data source entries in table properties when debug mode is on") { + withDebugMode { + val newSession = sparkSession.newSession() + newSession.sql("CREATE TABLE abc(i int) USING json") + val tableMeta = newSession.sessionState.catalog.getTableMetadata(TableIdentifier("abc")) + assert(tableMeta.properties(DATASOURCE_SCHEMA_NUMPARTS).toInt == 1) + assert(tableMeta.properties(DATASOURCE_PROVIDER) == "json") + } + } + + test("Infer schema for Hive serde tables") { + val tableName = "tab1" + val avroSchema = + """{ + | "name": "test_record", + | "type": "record", + | "fields": [ { + | "name": "f0", + | "type": "int" + | }] + |} + """.stripMargin + + Seq(true, false).foreach { isPartitioned => + withTable(tableName) { + val partitionClause = if (isPartitioned) "PARTITIONED BY (ds STRING)" else "" + // Creates the (non-)partitioned Avro table + val plan = sql( + s""" + |CREATE TABLE $tableName + |$partitionClause + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |TBLPROPERTIES ('avro.schema.literal' = '$avroSchema') + """.stripMargin + ).queryExecution.analyzed + + assert(plan.isInstanceOf[CreateTableCommand] && + plan.asInstanceOf[CreateTableCommand].table.dataSchema.nonEmpty) + + if (isPartitioned) { + sql(s"INSERT OVERWRITE TABLE $tableName partition (ds='a') SELECT 1") + checkAnswer(spark.table(tableName), Row(1, "a")) + } else { + sql(s"INSERT OVERWRITE TABLE $tableName SELECT 1") + checkAnswer(spark.table(tableName), Row(1)) + } + } + } + } + + private def withDebugMode(f: => Unit): Unit = { + val previousValue = sparkSession.sparkContext.conf.get(DEBUG_MODE) + try { + sparkSession.sparkContext.conf.set(DEBUG_MODE, true) + f + } finally { + sparkSession.sparkContext.conf.set(DEBUG_MODE, previousValue) + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index f16c257ab5ab..9060ce2e0eb4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -17,30 +17,40 @@ package org.apache.spark.sql.hive +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{AnalysisException, QueryTest, SaveMode} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.sql.{AnalysisException, QueryTest, SaveMode} class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { - private lazy val df = sqlContext.range(10).coalesce(1) + private lazy val df = spark.range(10).coalesce(1).toDF() private def checkTablePath(dbName: String, tableName: String): Unit = { - val metastoreTable = hiveContext.catalog.client.getTable(dbName, tableName) - val expectedPath = hiveContext.catalog.client.getDatabase(dbName).location + "/" + tableName + val metastoreTable = spark.sharedState.externalCatalog.getTable(dbName, tableName) + val expectedPath = new Path(new Path( + spark.sharedState.externalCatalog.getDatabase(dbName).locationUri), tableName).toUri - assert(metastoreTable.serdeProperties("path") === expectedPath) + assert(metastoreTable.location === expectedPath) + } + + private def getTableNames(dbName: Option[String] = None): Array[String] = { + dbName match { + case Some(db) => spark.catalog.listTables(db).collect().map(_.name) + case None => spark.catalog.listTables().collect().map(_.name) + } } test(s"saveAsTable() to non-default database - with USE - Overwrite") { withTempDatabase { db => activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") - assert(sqlContext.tableNames().contains("t")) - checkAnswer(sqlContext.table("t"), df) + assert(getTableNames().contains("t")) + checkAnswer(spark.table("t"), df) } - assert(sqlContext.tableNames(db).contains("t")) - checkAnswer(sqlContext.table(s"$db.t"), df) + assert(getTableNames(Option(db)).contains("t")) + checkAnswer(spark.table(s"$db.t"), df) checkTablePath(db, "t") } @@ -49,8 +59,8 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle test(s"saveAsTable() to non-default database - without USE - Overwrite") { withTempDatabase { db => df.write.mode(SaveMode.Overwrite).saveAsTable(s"$db.t") - assert(sqlContext.tableNames(db).contains("t")) - checkAnswer(sqlContext.table(s"$db.t"), df) + assert(getTableNames(Option(db)).contains("t")) + checkAnswer(spark.table(s"$db.t"), df) checkTablePath(db, "t") } @@ -63,20 +73,20 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle val path = dir.getCanonicalPath df.write.format("parquet").mode(SaveMode.Overwrite).save(path) - sqlContext.createExternalTable("t", path, "parquet") - assert(sqlContext.tableNames(db).contains("t")) - checkAnswer(sqlContext.table("t"), df) + spark.catalog.createExternalTable("t", path, "parquet") + assert(getTableNames(Option(db)).contains("t")) + checkAnswer(spark.table("t"), df) sql( s""" |CREATE TABLE t1 |USING parquet |OPTIONS ( - | path '$path' + | path '${dir.toURI}' |) """.stripMargin) - assert(sqlContext.tableNames(db).contains("t1")) - checkAnswer(sqlContext.table("t1"), df) + assert(getTableNames(Option(db)).contains("t1")) + checkAnswer(spark.table("t1"), df) } } } @@ -87,21 +97,21 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempPath { dir => val path = dir.getCanonicalPath df.write.format("parquet").mode(SaveMode.Overwrite).save(path) - sqlContext.createExternalTable(s"$db.t", path, "parquet") + spark.catalog.createExternalTable(s"$db.t", path, "parquet") - assert(sqlContext.tableNames(db).contains("t")) - checkAnswer(sqlContext.table(s"$db.t"), df) + assert(getTableNames(Option(db)).contains("t")) + checkAnswer(spark.table(s"$db.t"), df) sql( s""" |CREATE TABLE $db.t1 |USING parquet |OPTIONS ( - | path '$path' + | path '${dir.toURI}' |) """.stripMargin) - assert(sqlContext.tableNames(db).contains("t1")) - checkAnswer(sqlContext.table(s"$db.t1"), df) + assert(getTableNames(Option(db)).contains("t1")) + checkAnswer(spark.table(s"$db.t1"), df) } } } @@ -111,12 +121,12 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") df.write.mode(SaveMode.Append).saveAsTable("t") - assert(sqlContext.tableNames().contains("t")) - checkAnswer(sqlContext.table("t"), df.unionAll(df)) + assert(getTableNames().contains("t")) + checkAnswer(spark.table("t"), df.union(df)) } - assert(sqlContext.tableNames(db).contains("t")) - checkAnswer(sqlContext.table(s"$db.t"), df.unionAll(df)) + assert(getTableNames(Option(db)).contains("t")) + checkAnswer(spark.table(s"$db.t"), df.union(df)) checkTablePath(db, "t") } @@ -126,8 +136,8 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => df.write.mode(SaveMode.Overwrite).saveAsTable(s"$db.t") df.write.mode(SaveMode.Append).saveAsTable(s"$db.t") - assert(sqlContext.tableNames(db).contains("t")) - checkAnswer(sqlContext.table(s"$db.t"), df.unionAll(df)) + assert(getTableNames(Option(db)).contains("t")) + checkAnswer(spark.table(s"$db.t"), df.union(df)) checkTablePath(db, "t") } @@ -137,10 +147,10 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") - assert(sqlContext.tableNames().contains("t")) + assert(getTableNames().contains("t")) df.write.insertInto(s"$db.t") - checkAnswer(sqlContext.table(s"$db.t"), df.unionAll(df)) + checkAnswer(spark.table(s"$db.t"), df.union(df)) } } } @@ -149,13 +159,13 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { df.write.mode(SaveMode.Overwrite).saveAsTable("t") - assert(sqlContext.tableNames().contains("t")) + assert(getTableNames().contains("t")) } - assert(sqlContext.tableNames(db).contains("t")) + assert(getTableNames(Option(db)).contains("t")) df.write.insertInto(s"$db.t") - checkAnswer(sqlContext.table(s"$db.t"), df.unionAll(df)) + checkAnswer(spark.table(s"$db.t"), df.union(df)) } } @@ -163,10 +173,10 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { sql("CREATE TABLE t (key INT)") - checkAnswer(sqlContext.table("t"), sqlContext.emptyDataFrame) + checkAnswer(spark.table("t"), spark.emptyDataFrame) } - checkAnswer(sqlContext.table(s"$db.t"), sqlContext.emptyDataFrame) + checkAnswer(spark.table(s"$db.t"), spark.emptyDataFrame) } } @@ -174,21 +184,21 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle withTempDatabase { db => activateDatabase(db) { sql(s"CREATE TABLE t (key INT)") - assert(sqlContext.tableNames().contains("t")) - assert(!sqlContext.tableNames("default").contains("t")) + assert(getTableNames().contains("t")) + assert(!getTableNames(Option("default")).contains("t")) } - assert(!sqlContext.tableNames().contains("t")) - assert(sqlContext.tableNames(db).contains("t")) + assert(!getTableNames().contains("t")) + assert(getTableNames(Option(db)).contains("t")) activateDatabase(db) { sql(s"DROP TABLE t") - assert(!sqlContext.tableNames().contains("t")) - assert(!sqlContext.tableNames("default").contains("t")) + assert(!getTableNames().contains("t")) + assert(!getTableNames(Option("default")).contains("t")) } - assert(!sqlContext.tableNames().contains("t")) - assert(!sqlContext.tableNames(db).contains("t")) + assert(!getTableNames().contains("t")) + assert(!getTableNames(Option(db)).contains("t")) } } @@ -204,22 +214,22 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle s"""CREATE EXTERNAL TABLE t (id BIGINT) |PARTITIONED BY (p INT) |STORED AS PARQUET - |LOCATION '$path' + |LOCATION '${dir.toURI}' """.stripMargin) - checkAnswer(sqlContext.table("t"), sqlContext.emptyDataFrame) + checkAnswer(spark.table("t"), spark.emptyDataFrame) df.write.parquet(s"$path/p=1") sql("ALTER TABLE t ADD PARTITION (p=1)") sql("REFRESH TABLE t") - checkAnswer(sqlContext.table("t"), df.withColumn("p", lit(1))) + checkAnswer(spark.table("t"), df.withColumn("p", lit(1))) df.write.parquet(s"$path/p=2") sql("ALTER TABLE t ADD PARTITION (p=2)") - hiveContext.refreshTable("t") + spark.catalog.refreshTable("t") checkAnswer( - sqlContext.table("t"), - df.withColumn("p", lit(1)).unionAll(df.withColumn("p", lit(2)))) + spark.table("t"), + df.withColumn("p", lit(1)).union(df.withColumn("p", lit(2)))) } } } @@ -236,22 +246,22 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle s"""CREATE EXTERNAL TABLE $db.t (id BIGINT) |PARTITIONED BY (p INT) |STORED AS PARQUET - |LOCATION '$path' + |LOCATION '${dir.toURI}' """.stripMargin) - checkAnswer(sqlContext.table(s"$db.t"), sqlContext.emptyDataFrame) + checkAnswer(spark.table(s"$db.t"), spark.emptyDataFrame) df.write.parquet(s"$path/p=1") sql(s"ALTER TABLE $db.t ADD PARTITION (p=1)") sql(s"REFRESH TABLE $db.t") - checkAnswer(sqlContext.table(s"$db.t"), df.withColumn("p", lit(1))) + checkAnswer(spark.table(s"$db.t"), df.withColumn("p", lit(1))) df.write.parquet(s"$path/p=2") sql(s"ALTER TABLE $db.t ADD PARTITION (p=2)") - hiveContext.refreshTable(s"$db.t") + spark.catalog.refreshTable(s"$db.t") checkAnswer( - sqlContext.table(s"$db.t"), - df.withColumn("p", lit(1)).unionAll(df.withColumn("p", lit(2)))) + spark.table(s"$db.t"), + df.withColumn("p", lit(1)).union(df.withColumn("p", lit(2)))) } } } @@ -261,19 +271,17 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle val message = intercept[AnalysisException] { df.write.format("parquet").saveAsTable("`d:b`.`t:a`") }.getMessage - assert(message.contains("is not a valid name for metastore")) + assert(message.contains("Database 'd:b' not found")) } { val message = intercept[AnalysisException] { df.write.format("parquet").saveAsTable("`d:b`.`table`") }.getMessage - assert(message.contains("is not a valid name for metastore")) + assert(message.contains("Database 'd:b' not found")) } - withTempPath { dir => - val path = dir.getCanonicalPath - + withTempDir { dir => { val message = intercept[AnalysisException] { sql( @@ -281,11 +289,12 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle |CREATE TABLE `d:b`.`t:a` (a int) |USING parquet |OPTIONS ( - | path '$path' + | path '${dir.toURI}' |) """.stripMargin) }.getMessage - assert(message.contains("is not a valid name for metastore")) + assert(message.contains("`t:a` is not a valid name for tables/databases. " + + "Valid names only contain alphabet characters, numbers and _.")) } { @@ -295,11 +304,11 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle |CREATE TABLE `d:b`.`table` (a int) |USING parquet |OPTIONS ( - | path '$path' + | path '${dir.toURI}' |) """.stripMargin) }.getMessage - assert(message.contains("is not a valid name for metastore")) + assert(message.contains("Database 'd:b' not found")) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala index 49aab85cf1aa..05b6059472f5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala @@ -19,18 +19,17 @@ package org.apache.spark.sql.hive import java.sql.Timestamp -import org.apache.hadoop.hive.conf.HiveConf - +import org.apache.spark.sql.Row import org.apache.spark.sql.execution.datasources.parquet.ParquetCompatibilityTest -import org.apache.spark.sql.{Row, SQLConf} import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHiveSingleton { /** * Set the staging directory (and hence path to ignore Parquet files under) - * to that set by [[HiveConf.ConfVars.STAGINGDIR]]. + * to the default value of hive.exec.stagingdir. */ - private val stagingDir = new HiveConf().getVar(HiveConf.ConfVars.STAGINGDIR) + private val stagingDir = ".hive-staging" override protected def logParquetSchema(path: String): Unit = { val schema = readParquetSchema(path, { path => @@ -46,14 +45,14 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi private def testParquetHiveCompatibility(row: Row, hiveTypes: String*): Unit = { withTable("parquet_compat") { withTempPath { dir => - val path = dir.getCanonicalPath + val path = dir.toURI.toString // Hive columns are always nullable, so here we append a all-null row. val rows = row :: Row(Seq.fill(row.length)(null): _*) :: Nil // Don't convert Hive metastore Parquet tables to let Hive write those Parquet files. - withSQLConf(HiveContext.CONVERT_METASTORE_PARQUET.key -> "false") { - withTempTable("data") { + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") { + withTempView("data") { val fields = hiveTypes.zipWithIndex.map { case (typ, index) => s" col_$index $typ" } val ddl = @@ -69,12 +68,12 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi |$ddl """.stripMargin) - sqlContext.sql(ddl) + spark.sql(ddl) - val schema = sqlContext.table("parquet_compat").schema - val rowRDD = sqlContext.sparkContext.parallelize(rows).coalesce(1) - sqlContext.createDataFrame(rowRDD, schema).registerTempTable("data") - sqlContext.sql("INSERT INTO TABLE parquet_compat SELECT * FROM data") + val schema = spark.table("parquet_compat").schema + val rowRDD = spark.sparkContext.parallelize(rows).coalesce(1) + spark.createDataFrame(rowRDD, schema).createOrReplaceTempView("data") + spark.sql("INSERT INTO TABLE parquet_compat SELECT * FROM data") } } @@ -83,7 +82,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi // Unfortunately parquet-hive doesn't add `UTF8` annotation to BINARY when writing strings. // Have to assume all BINARY values are strings here. withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING.key -> "true") { - checkAnswer(sqlContext.read.parquet(path), rows) + checkAnswer(spark.read.parquet(path), rows) } } } @@ -136,4 +135,10 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with TestHi Row(Row(1, Seq("foo", "bar", null))), "STRUCT>") } + + test("SPARK-16344: array of struct with a single field named 'array_element'") { + testParquetHiveCompatibility( + Row(Seq(Row(1))), + "ARRAY>") + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala new file mode 100644 index 000000000000..9440a17677eb --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala @@ -0,0 +1,535 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.apache.hadoop.fs.Path + +import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.util.Utils + +class PartitionProviderCompatibilitySuite + extends QueryTest with TestHiveSingleton with SQLTestUtils { + import testImplicits._ + + private def setupPartitionedDatasourceTable(tableName: String, dir: File): Unit = { + spark.range(5).selectExpr("id as fieldOne", "id as partCol").write + .partitionBy("partCol") + .mode("overwrite") + .parquet(dir.getAbsolutePath) + + spark.sql(s""" + |create table $tableName (fieldOne long, partCol int) + |using parquet + |options (path "${dir.toURI}") + |partitioned by (partCol)""".stripMargin) + } + + private def verifyIsLegacyTable(tableName: String): Unit = { + val unsupportedCommands = Seq( + s"ALTER TABLE $tableName ADD PARTITION (partCol=1) LOCATION '/foo'", + s"ALTER TABLE $tableName PARTITION (partCol=1) RENAME TO PARTITION (partCol=2)", + s"ALTER TABLE $tableName PARTITION (partCol=1) SET LOCATION '/foo'", + s"ALTER TABLE $tableName DROP PARTITION (partCol=1)", + s"DESCRIBE $tableName PARTITION (partCol=1)", + s"SHOW PARTITIONS $tableName") + + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { + for (cmd <- unsupportedCommands) { + val e = intercept[AnalysisException] { + spark.sql(cmd) + } + assert(e.getMessage.contains("partition metadata is not stored in the Hive metastore"), e) + } + } + } + + test("convert partition provider to hive with repair table") { + withTable("test") { + withTempDir { dir => + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "false") { + setupPartitionedDatasourceTable("test", dir) + assert(spark.sql("select * from test").count() == 5) + } + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { + verifyIsLegacyTable("test") + spark.catalog.recoverPartitions("test") + spark.sql("show partitions test").count() // check we are a new table + + // sanity check table performance + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol < 2").count() == 2) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 2) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 2) + } + } + } + } + + test("when partition management is enabled, new tables have partition provider hive") { + withTable("test") { + withTempDir { dir => + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { + setupPartitionedDatasourceTable("test", dir) + spark.sql("show partitions test").count() // check we are a new table + assert(spark.sql("select * from test").count() == 0) // needs repair + spark.catalog.recoverPartitions("test") + assert(spark.sql("select * from test").count() == 5) + } + } + } + } + + test("when partition management is disabled, new tables have no partition provider") { + withTable("test") { + withTempDir { dir => + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "false") { + setupPartitionedDatasourceTable("test", dir) + verifyIsLegacyTable("test") + assert(spark.sql("select * from test").count() == 5) + } + } + } + } + + test("when partition management is disabled, we preserve the old behavior even for new tables") { + withTable("test") { + withTempDir { dir => + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { + setupPartitionedDatasourceTable("test", dir) + spark.sql("show partitions test").count() // check we are a new table + spark.sql("refresh table test") + assert(spark.sql("select * from test").count() == 0) + } + // disabled + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "false") { + val e = intercept[AnalysisException] { + spark.sql(s"show partitions test") + } + assert(e.getMessage.contains("filesource partition management is disabled")) + spark.sql("refresh table test") + assert(spark.sql("select * from test").count() == 5) + } + // then enabled again + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { + spark.sql("refresh table test") + assert(spark.sql("select * from test").count() == 0) + } + } + } + } + + test("insert overwrite partition of legacy datasource table") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "false") { + withTable("test") { + withTempDir { dir => + setupPartitionedDatasourceTable("test", dir) + spark.sql( + """insert overwrite table test + |partition (partCol=1) + |select * from range(100)""".stripMargin) + assert(spark.sql("select * from test").count() == 104) + + // Overwriting entire table + spark.sql("insert overwrite table test select id, id from range(10)".stripMargin) + assert(spark.sql("select * from test").count() == 10) + } + } + } + } + + test("insert overwrite partition of new datasource table overwrites just partition") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { + withTable("test") { + withTempDir { dir => + setupPartitionedDatasourceTable("test", dir) + spark.catalog.recoverPartitions("test") + spark.sql( + """insert overwrite table test + |partition (partCol=1) + |select * from range(100)""".stripMargin) + assert(spark.sql("select * from test").count() == 104) + + // Test overwriting a partition that has a custom location + withTempDir { dir2 => + sql( + s"""alter table test partition (partCol=1) + |set location '${dir2.toURI}'""".stripMargin) + assert(sql("select * from test").count() == 4) + sql( + """insert overwrite table test + |partition (partCol=1) + |select * from range(30)""".stripMargin) + sql( + """insert overwrite table test + |partition (partCol=1) + |select * from range(20)""".stripMargin) + assert(sql("select * from test").count() == 24) + } + } + } + } + } + + for (enabled <- Seq(true, false)) { + test(s"SPARK-18544 append with saveAsTable - partition management $enabled") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> enabled.toString) { + withTable("test") { + withTempDir { dir => + setupPartitionedDatasourceTable("test", dir) + if (enabled) { + assert(spark.table("test").count() == 0) + } else { + assert(spark.table("test").count() == 5) + } + // Table `test` has 5 partitions, from `partCol=0` to `partCol=4`, which are invisible + // because we have not run `REPAIR TABLE` yet. Here we add 10 more partitions from + // `partCol=3` to `partCol=12`, to test the following behaviors: + // 1. invisible partitions are still invisible if they are not overwritten. + // 2. invisible partitions become visible if they are overwritten. + // 3. newly added partitions should be visible. + spark.range(3, 13).selectExpr("id as fieldOne", "id as partCol") + .write.partitionBy("partCol").mode("append").saveAsTable("test") + + if (enabled) { + // Only the newly written partitions are visible, which means the partitions + // `partCol=0`, `partCol=1` and `partCol=2` are still invisible, so we can only see + // 5 + 10 - 3 = 12 records. + assert(spark.table("test").count() == 12) + // Repair the table to make all partitions visible. + sql("msck repair table test") + assert(spark.table("test").count() == 15) + } else { + assert(spark.table("test").count() == 15) + } + } + } + } + } + + test(s"SPARK-18635 special chars in partition values - partition management $enabled") { + withTable("test") { + spark.range(10) + .selectExpr("id", "id as A", "'%' as B") + .write.partitionBy("A", "B").mode("overwrite") + .saveAsTable("test") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("select * from test where B = '%'").count() == 10) + assert(spark.sql("select * from test where B = '$'").count() == 0) + spark.range(10) + .selectExpr("id", "id as A", "'=' as B") + .write.mode("append").insertInto("test") + spark.sql("insert into test partition (A, B) select id, id, '%=' from range(10)") + assert(spark.sql("select * from test").count() == 30) + assert(spark.sql("select * from test where B = '%'").count() == 10) + assert(spark.sql("select * from test where B = '='").count() == 10) + assert(spark.sql("select * from test where B = '%='").count() == 10) + + // show partitions sanity check + val parts = spark.sql("show partitions test").collect().map(_.get(0)).toSeq + assert(parts.length == 30) + assert(parts.contains("A=0/B=%25")) + assert(parts.contains("A=0/B=%3D")) + assert(parts.contains("A=0/B=%25%3D")) + + // drop partition sanity check + spark.sql("alter table test drop partition (A=1, B='%')") + assert(spark.sql("select * from test").count() == 29) // 1 file in dropped partition + + withTempDir { dir => + // custom locations sanity check + spark.sql(s""" + |alter table test partition (A=0, B='%') + |set location '${dir.toURI}'""".stripMargin) + assert(spark.sql("select * from test").count() == 28) // moved to empty dir + + // rename partition sanity check + spark.sql(s""" + |alter table test partition (A=5, B='%') + |rename to partition (A=100, B='%')""".stripMargin) + assert(spark.sql("select * from test where a = 5 and b = '%'").count() == 0) + assert(spark.sql("select * from test where a = 100 and b = '%'").count() == 1) + + // try with A=0 which has a custom location + spark.sql("insert into test partition (A=0, B='%') select 1") + spark.sql(s""" + |alter table test partition (A=0, B='%') + |rename to partition (A=101, B='%')""".stripMargin) + assert(spark.sql("select * from test where a = 0 and b = '%'").count() == 0) + assert(spark.sql("select * from test where a = 101 and b = '%'").count() == 1) + } + } + } + + test(s"SPARK-18659 insert overwrite table files - partition management $enabled") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> enabled.toString) { + withTable("test") { + spark.range(10) + .selectExpr("id", "id as A", "'x' as B") + .write.partitionBy("A", "B").mode("overwrite") + .saveAsTable("test") + spark.sql("insert overwrite table test select id, id, 'x' from range(1)") + assert(spark.sql("select * from test").count() == 1) + + spark.range(10) + .selectExpr("id", "id as A", "'x' as B") + .write.partitionBy("A", "B").mode("overwrite") + .saveAsTable("test") + spark.sql( + "insert overwrite table test partition (A, B) select id, id, 'x' from range(1)") + assert(spark.sql("select * from test").count() == 1) + } + } + } + + test(s"SPARK-18659 insert overwrite table with lowercase - partition management $enabled") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> enabled.toString) { + withTable("test") { + spark.range(10) + .selectExpr("id", "id as A", "'x' as B") + .write.partitionBy("A", "B").mode("overwrite") + .saveAsTable("test") + // note that 'A', 'B' are lowercase instead of their original case here + spark.sql("insert overwrite table test partition (a=1, b) select id, 'x' from range(1)") + assert(spark.sql("select * from test").count() == 10) + } + } + } + + test(s"SPARK-19887 partition value is null - partition management $enabled") { + withTable("test") { + Seq((1, "p", 1), (2, null, 2)).toDF("a", "b", "c") + .write.partitionBy("b", "c").saveAsTable("test") + checkAnswer(spark.table("test"), + Row(1, "p", 1) :: Row(2, null, 2) :: Nil) + + Seq((3, null: String, 3)).toDF("a", "b", "c") + .write.mode("append").partitionBy("b", "c").saveAsTable("test") + checkAnswer(spark.table("test"), + Row(1, "p", 1) :: Row(2, null, 2) :: Row(3, null, 3) :: Nil) + // make sure partition pruning also works. + checkAnswer(spark.table("test").filter($"b".isNotNull), Row(1, "p", 1)) + + // empty string is an invalid partition value and we treat it as null when read back. + Seq((4, "", 4)).toDF("a", "b", "c") + .write.mode("append").partitionBy("b", "c").saveAsTable("test") + checkAnswer(spark.table("test"), + Row(1, "p", 1) :: Row(2, null, 2) :: Row(3, null, 3) :: Row(4, null, 4) :: Nil) + } + } + } + + /** + * Runs a test against a multi-level partitioned table, then validates that the custom locations + * were respected by the output writer. + * + * The initial partitioning structure is: + * /P1=0/P2=0 -- custom location a + * /P1=0/P2=1 -- custom location b + * /P1=1/P2=0 -- custom location c + * /P1=1/P2=1 -- default location + */ + private def testCustomLocations(testFn: => Unit): Unit = { + val base = Utils.createTempDir(namePrefix = "base") + val a = Utils.createTempDir(namePrefix = "a") + val b = Utils.createTempDir(namePrefix = "b") + val c = Utils.createTempDir(namePrefix = "c") + try { + spark.sql(s""" + |create table test (id long, P1 int, P2 int) + |using parquet + |options (path "${base.toURI}") + |partitioned by (P1, P2)""".stripMargin) + spark.sql(s"alter table test add partition (P1=0, P2=0) location '${a.toURI}'") + spark.sql(s"alter table test add partition (P1=0, P2=1) location '${b.toURI}'") + spark.sql(s"alter table test add partition (P1=1, P2=0) location '${c.toURI}'") + spark.sql(s"alter table test add partition (P1=1, P2=1)") + + testFn + + // Now validate the partition custom locations were respected + val initialCount = spark.sql("select * from test").count() + val numA = spark.sql("select * from test where P1=0 and P2=0").count() + val numB = spark.sql("select * from test where P1=0 and P2=1").count() + val numC = spark.sql("select * from test where P1=1 and P2=0").count() + Utils.deleteRecursively(a) + spark.sql("refresh table test") + assert(spark.sql("select * from test where P1=0 and P2=0").count() == 0) + assert(spark.sql("select * from test").count() == initialCount - numA) + Utils.deleteRecursively(b) + spark.sql("refresh table test") + assert(spark.sql("select * from test where P1=0 and P2=1").count() == 0) + assert(spark.sql("select * from test").count() == initialCount - numA - numB) + Utils.deleteRecursively(c) + spark.sql("refresh table test") + assert(spark.sql("select * from test where P1=1 and P2=0").count() == 0) + assert(spark.sql("select * from test").count() == initialCount - numA - numB - numC) + } finally { + Utils.deleteRecursively(base) + Utils.deleteRecursively(a) + Utils.deleteRecursively(b) + Utils.deleteRecursively(c) + spark.sql("drop table test") + } + } + + test("sanity check table setup") { + testCustomLocations { + assert(spark.sql("select * from test").count() == 0) + assert(spark.sql("show partitions test").count() == 4) + } + } + + test("insert into partial dynamic partitions") { + testCustomLocations { + spark.sql("insert into test partition (P1=0, P2) select id, id from range(10)") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 12) + spark.sql("insert into test partition (P1=0, P2) select id, id from range(10)") + assert(spark.sql("select * from test").count() == 20) + assert(spark.sql("show partitions test").count() == 12) + spark.sql("insert into test partition (P1=1, P2) select id, id from range(10)") + assert(spark.sql("select * from test").count() == 30) + assert(spark.sql("show partitions test").count() == 20) + spark.sql("insert into test partition (P1=2, P2) select id, id from range(10)") + assert(spark.sql("select * from test").count() == 40) + assert(spark.sql("show partitions test").count() == 30) + } + } + + test("insert into fully dynamic partitions") { + testCustomLocations { + spark.sql("insert into test partition (P1, P2) select id, id, id from range(10)") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 12) + spark.sql("insert into test partition (P1, P2) select id, id, id from range(10)") + assert(spark.sql("select * from test").count() == 20) + assert(spark.sql("show partitions test").count() == 12) + } + } + + test("insert into static partition") { + testCustomLocations { + spark.sql("insert into test partition (P1=0, P2=0) select id from range(10)") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 4) + spark.sql("insert into test partition (P1=0, P2=0) select id from range(10)") + assert(spark.sql("select * from test").count() == 20) + assert(spark.sql("show partitions test").count() == 4) + spark.sql("insert into test partition (P1=1, P2=1) select id from range(10)") + assert(spark.sql("select * from test").count() == 30) + assert(spark.sql("show partitions test").count() == 4) + } + } + + test("overwrite partial dynamic partitions") { + testCustomLocations { + spark.sql("insert overwrite table test partition (P1=0, P2) select id, id from range(10)") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 12) + spark.sql("insert overwrite table test partition (P1=0, P2) select id, id from range(5)") + assert(spark.sql("select * from test").count() == 5) + assert(spark.sql("show partitions test").count() == 7) + spark.sql("insert overwrite table test partition (P1=0, P2) select id, id from range(1)") + assert(spark.sql("select * from test").count() == 1) + assert(spark.sql("show partitions test").count() == 3) + spark.sql("insert overwrite table test partition (P1=1, P2) select id, id from range(10)") + assert(spark.sql("select * from test").count() == 11) + assert(spark.sql("show partitions test").count() == 11) + spark.sql("insert overwrite table test partition (P1=1, P2) select id, id from range(1)") + assert(spark.sql("select * from test").count() == 2) + assert(spark.sql("show partitions test").count() == 2) + spark.sql("insert overwrite table test partition (P1=3, P2) select id, id from range(100)") + assert(spark.sql("select * from test").count() == 102) + assert(spark.sql("show partitions test").count() == 102) + } + } + + test("overwrite fully dynamic partitions") { + testCustomLocations { + spark.sql("insert overwrite table test partition (P1, P2) select id, id, id from range(10)") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 10) + spark.sql("insert overwrite table test partition (P1, P2) select id, id, id from range(5)") + assert(spark.sql("select * from test").count() == 5) + assert(spark.sql("show partitions test").count() == 5) + } + } + + test("overwrite static partition") { + testCustomLocations { + spark.sql("insert overwrite table test partition (P1=0, P2=0) select id from range(10)") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 4) + spark.sql("insert overwrite table test partition (P1=0, P2=0) select id from range(5)") + assert(spark.sql("select * from test").count() == 5) + assert(spark.sql("show partitions test").count() == 4) + spark.sql("insert overwrite table test partition (P1=1, P2=1) select id from range(5)") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 4) + spark.sql("insert overwrite table test partition (P1=1, P2=2) select id from range(5)") + assert(spark.sql("select * from test").count() == 15) + assert(spark.sql("show partitions test").count() == 5) + } + } + + test("append data with DataFrameWriter") { + testCustomLocations { + val df = Seq((1L, 0, 0), (2L, 0, 0)).toDF("id", "P1", "P2") + df.write.partitionBy("P1", "P2").mode("append").saveAsTable("test") + assert(spark.sql("select * from test").count() == 2) + assert(spark.sql("show partitions test").count() == 4) + val df2 = Seq((3L, 2, 2)).toDF("id", "P1", "P2") + df2.write.partitionBy("P1", "P2").mode("append").saveAsTable("test") + assert(spark.sql("select * from test").count() == 3) + assert(spark.sql("show partitions test").count() == 5) + } + } + + test("SPARK-19359: renaming partition should not leave useless directories") { + withTable("t", "t1") { + Seq((1, 2, 3)).toDF("id", "A", "B").write.partitionBy("A", "B").saveAsTable("t") + spark.sql("alter table t partition(A=2, B=3) rename to partition(A=4, B=5)") + + var table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + var tablePath = new Path(table.location) + val fs = tablePath.getFileSystem(spark.sessionState.newHadoopConf()) + // the `A=2` directory is still there, we follow this behavior from hive. + assert(fs.listStatus(tablePath) + .filterNot(_.getPath.toString.contains("A=2")).count(_.isDirectory) == 1) + assert(fs.listStatus(new Path(tablePath, "A=4")).count(_.isDirectory) == 1) + + + Seq((1, 2, 3, 4)).toDF("id", "A", "b", "C").write.partitionBy("A", "b", "C").saveAsTable("t1") + spark.sql("alter table t1 partition(A=2, b=3, C=4) rename to partition(A=4, b=5, C=6)") + table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1")) + tablePath = new Path(table.location) + // the `A=2` directory is still there, we follow this behavior from hive. + assert(fs.listStatus(tablePath) + .filterNot(_.getPath.toString.contains("A=2")).count(_.isDirectory) == 1) + assert(fs.listStatus(new Path(tablePath, "A=4")).count(_.isDirectory) == 1) + assert(fs.listStatus(new Path(new Path(tablePath, "A=4"), "b=5")).count(_.isDirectory) == 1) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala new file mode 100644 index 000000000000..54d3962a46b4 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala @@ -0,0 +1,425 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.concurrent.{Executors, TimeUnit} + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.datasources.FileStatusCache +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils + +class PartitionedTablePerfStatsSuite + extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach { + + override def beforeEach(): Unit = { + super.beforeEach() + FileStatusCache.resetForTesting() + } + + override def afterEach(): Unit = { + super.afterEach() + FileStatusCache.resetForTesting() + } + + private case class TestSpec(setupTable: (String, File) => Unit, isDatasourceTable: Boolean) + + /** + * Runs a test against both converted hive and native datasource tables. The test can use the + * passed TestSpec object for setup and inspecting test parameters. + */ + private def genericTest(testName: String)(fn: TestSpec => Unit): Unit = { + test("hive table: " + testName) { + fn(TestSpec(setupPartitionedHiveTable, false)) + } + test("datasource table: " + testName) { + fn(TestSpec(setupPartitionedDatasourceTable, true)) + } + } + + private def setupPartitionedHiveTable(tableName: String, dir: File): Unit = { + setupPartitionedHiveTable(tableName, dir, 5) + } + + private def setupPartitionedHiveTable( + tableName: String, dir: File, scale: Int, repair: Boolean = true): Unit = { + spark.range(scale).selectExpr("id as fieldOne", "id as partCol1", "id as partCol2").write + .partitionBy("partCol1", "partCol2") + .mode("overwrite") + .parquet(dir.getAbsolutePath) + + spark.sql(s""" + |create external table $tableName (fieldOne long) + |partitioned by (partCol1 int, partCol2 int) + |stored as parquet + |location "${dir.toURI}"""".stripMargin) + if (repair) { + spark.sql(s"msck repair table $tableName") + } + } + + private def setupPartitionedDatasourceTable(tableName: String, dir: File): Unit = { + setupPartitionedDatasourceTable(tableName, dir, 5) + } + + private def setupPartitionedDatasourceTable( + tableName: String, dir: File, scale: Int, repair: Boolean = true): Unit = { + spark.range(scale).selectExpr("id as fieldOne", "id as partCol1", "id as partCol2").write + .partitionBy("partCol1", "partCol2") + .mode("overwrite") + .parquet(dir.getAbsolutePath) + + spark.sql(s""" + |create table $tableName (fieldOne long, partCol1 int, partCol2 int) + |using parquet + |options (path "${dir.toURI}") + |partitioned by (partCol1, partCol2)""".stripMargin) + if (repair) { + spark.sql(s"msck repair table $tableName") + } + } + + genericTest("partitioned pruned table reports only selected files") { spec => + assert(spark.sqlContext.getConf(HiveUtils.CONVERT_METASTORE_PARQUET.key) == "true") + withTable("test") { + withTempDir { dir => + spec.setupTable("test", dir) + val df = spark.sql("select * from test") + assert(df.count() == 5) + assert(df.inputFiles.length == 5) // unpruned + + val df2 = spark.sql("select * from test where partCol1 = 3 or partCol2 = 4") + assert(df2.count() == 2) + assert(df2.inputFiles.length == 2) // pruned, so we have less files + + val df3 = spark.sql("select * from test where PARTCOL1 = 3 or partcol2 = 4") + assert(df3.count() == 2) + assert(df3.inputFiles.length == 2) + + val df4 = spark.sql("select * from test where partCol1 = 999") + assert(df4.count() == 0) + assert(df4.inputFiles.length == 0) + + val df5 = spark.sql("select * from test where fieldOne = 4") + assert(df5.count() == 1) + assert(df5.inputFiles.length == 5) + } + } + } + + genericTest("lazy partition pruning reads only necessary partition data") { spec => + withSQLConf( + SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true", + SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE.key -> "0") { + withTable("test") { + withTempDir { dir => + spec.setupTable("test", dir) + HiveCatalogMetrics.reset() + spark.sql("select * from test where partCol1 = 999").count() + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 0) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + + HiveCatalogMetrics.reset() + spark.sql("select * from test where partCol1 < 2").count() + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 2) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 2) + + HiveCatalogMetrics.reset() + spark.sql("select * from test where partCol1 < 3").count() + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 3) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 3) + + // should read all + HiveCatalogMetrics.reset() + spark.sql("select * from test").count() + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + + // read all should not be cached + HiveCatalogMetrics.reset() + spark.sql("select * from test").count() + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + + // cache should be disabled + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + } + } + } + } + + genericTest("lazy partition pruning with file status caching enabled") { spec => + withSQLConf( + SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true", + SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE.key -> "9999999") { + withTable("test") { + withTempDir { dir => + spec.setupTable("test", dir) + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 = 999").count() == 0) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 0) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 < 2").count() == 2) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 2) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 2) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 < 3").count() == 3) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 3) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 1) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 2) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 2) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 3) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 5) + } + } + } + } + + genericTest("file status caching respects refresh table and refreshByPath") { spec => + withSQLConf( + SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true", + SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE.key -> "9999999") { + withTable("test") { + withTempDir { dir => + spec.setupTable("test", dir) + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + + HiveCatalogMetrics.reset() + spark.sql("refresh table test") + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + + spark.catalog.cacheTable("test") + HiveCatalogMetrics.reset() + spark.catalog.refreshByPath(dir.getAbsolutePath) + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + } + } + } + } + + genericTest("file status cache respects size limit") { spec => + withSQLConf( + SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true", + SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE.key -> "1" /* 1 byte */) { + withTable("test") { + withTempDir { dir => + spec.setupTable("test", dir) + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 10) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + } + } + } + } + + test("datasource table: table setup does not scan filesystem") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { + withTable("test") { + withTempDir { dir => + HiveCatalogMetrics.reset() + setupPartitionedDatasourceTable("test", dir, scale = 10, repair = false) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + } + } + } + } + + test("hive table: table setup does not scan filesystem") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { + withTable("test") { + withTempDir { dir => + HiveCatalogMetrics.reset() + setupPartitionedHiveTable("test", dir, scale = 10, repair = false) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + } + } + } + } + + test("hive table: num hive client calls does not scale with partition count") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { + withTable("test") { + withTempDir { dir => + setupPartitionedHiveTable("test", dir, scale = 100) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 = 1").count() == 1) + assert(HiveCatalogMetrics.METRIC_HIVE_CLIENT_CALLS.getCount() > 0) + assert(HiveCatalogMetrics.METRIC_HIVE_CLIENT_CALLS.getCount() < 10) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 100) + assert(HiveCatalogMetrics.METRIC_HIVE_CLIENT_CALLS.getCount() < 10) + + HiveCatalogMetrics.reset() + assert(spark.sql("show partitions test").count() == 100) + assert(HiveCatalogMetrics.METRIC_HIVE_CLIENT_CALLS.getCount() < 10) + } + } + } + } + + test("datasource table: num hive client calls does not scale with partition count") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { + withTable("test") { + withTempDir { dir => + setupPartitionedDatasourceTable("test", dir, scale = 100) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 = 1").count() == 1) + assert(HiveCatalogMetrics.METRIC_HIVE_CLIENT_CALLS.getCount() > 0) + assert(HiveCatalogMetrics.METRIC_HIVE_CLIENT_CALLS.getCount() < 10) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 100) + assert(HiveCatalogMetrics.METRIC_HIVE_CLIENT_CALLS.getCount() < 10) + + HiveCatalogMetrics.reset() + assert(spark.sql("show partitions test").count() == 100) + assert(HiveCatalogMetrics.METRIC_HIVE_CLIENT_CALLS.getCount() < 10) + } + } + } + } + + test("hive table: files read and cached when filesource partition management is off") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "false") { + withTable("test") { + withTempDir { dir => + setupPartitionedHiveTable("test", dir) + + // We actually query the partitions from hive each time the table is resolved in this + // mode. This is kind of terrible, but is needed to preserve the legacy behavior + // of doing plan cache validation based on the entire partition set. + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 = 999").count() == 0) + // 5 from table resolution, another 5 from InMemoryFileIndex + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 10) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 < 2").count() == 2) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + } + } + } + } + + test("datasource table: all partition data cached in memory when partition management is off") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "false") { + withTable("test") { + withTempDir { dir => + setupPartitionedDatasourceTable("test", dir) + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 = 999").count() == 0) + + // not using metastore + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 0) + + // reads and caches all the files initially + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 < 2").count() == 2) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 0) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 0) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + } + } + } + } + + test("SPARK-18700: table loaded only once even when resolved concurrently") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "false") { + withTable("test") { + withTempDir { dir => + HiveCatalogMetrics.reset() + setupPartitionedHiveTable("test", dir, 50) + // select the table in multi-threads + val executorPool = Executors.newFixedThreadPool(10) + (1 to 10).map(threadId => { + val runnable = new Runnable { + override def run(): Unit = { + spark.sql("select * from test where partCol1 = 999").count() + } + } + executorPool.execute(runnable) + None + }) + executorPool.shutdown() + executorPool.awaitTermination(30, TimeUnit.SECONDS) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 50) + assert(HiveCatalogMetrics.METRIC_PARALLEL_LISTING_JOB_COUNT.getCount() == 1) + } + } + } + } + + test("resolveRelation for a FileFormat DataSource without userSchema scan filesystem only once") { + withTempDir { dir => + import spark.implicits._ + Seq(1).toDF("a").write.mode("overwrite").save(dir.getAbsolutePath) + HiveCatalogMetrics.reset() + spark.read.parquet(dir.getAbsolutePath) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 1) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 1) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala index f542a5a02508..b2dc401ce1ef 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala @@ -17,26 +17,31 @@ package org.apache.spark.sql.hive +import java.io.File +import java.sql.Timestamp + import com.google.common.io.Files +import org.apache.hadoop.fs.FileSystem -import org.apache.spark.util.Utils -import org.apache.spark.sql.{QueryTest, _} +import org.apache.spark.sql._ import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.util.Utils class QueryPartitionSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { - import hiveContext.implicits._ + import spark.implicits._ test("SPARK-5068: query data when path doesn't exist") { withSQLConf((SQLConf.HIVE_VERIFY_PARTITION_PATH.key, "true")) { val testData = sparkContext.parallelize( (1 to 10).map(i => TestData(i, i.toString))).toDF() - testData.registerTempTable("testData") + testData.createOrReplaceTempView("testData") val tmpDir = Files.createTempDir() // create the table for test sql(s"CREATE TABLE table_with_partition(key int,value string) " + - s"PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ") + s"PARTITIONED by (ds string) location '${tmpDir.toURI}' ") sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') " + "SELECT key,value FROM testData") sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='2') " + @@ -60,8 +65,24 @@ class QueryPartitionSuite extends QueryTest with SQLTestUtils with TestHiveSingl checkAnswer(sql("select key,value from table_with_partition"), testData.toDF.collect ++ testData.toDF.collect ++ testData.toDF.collect) - sql("DROP TABLE table_with_partition") - sql("DROP TABLE createAndInsertTest") + sql("DROP TABLE IF EXISTS table_with_partition") + sql("DROP TABLE IF EXISTS createAndInsertTest") + } + } + + test("SPARK-21739: Cast expression should initialize timezoneId") { + withTable("table_with_timestamp_partition") { + sql("CREATE TABLE table_with_timestamp_partition(value int) PARTITIONED BY (ts TIMESTAMP)") + sql("INSERT OVERWRITE TABLE table_with_timestamp_partition " + + "PARTITION (ts = '2010-01-01 00:00:00.000') VALUES (1)") + + // test for Cast expression in TableReader + checkAnswer(sql("SELECT * FROM table_with_timestamp_partition"), + Seq(Row(1, Timestamp.valueOf("2010-01-01 00:00:00.000")))) + + // test for Cast expression in HiveTableScanExec + checkAnswer(sql("SELECT value FROM table_with_timestamp_partition " + + "WHERE ts = '2010-01-01 00:00:00.000'"), Row(1)) } } } 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 deleted file mode 100644 index 93dcb10f7a29..000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.serializer.JavaSerializer - -class SerializationSuite extends SparkFunSuite { - - test("[SPARK-5840] HiveContext should be serializable") { - val hiveContext = org.apache.spark.sql.hive.test.TestHive - hiveContext.hiveconf - val serializer = new JavaSerializer(new SparkConf()).newInstance() - val bytes = serializer.serialize(hiveContext) - val deSer = serializer.deserialize[AnyRef](bytes) - } -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala new file mode 100644 index 000000000000..fad81c7e9474 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.util.Utils + +class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + import testImplicits._ + + test("data source table with user specified schema") { + withTable("ddl_test") { + val jsonFilePath = Utils.getSparkClassLoader.getResource("sample.json").getFile + + sql( + s"""CREATE TABLE ddl_test ( + | a STRING, + | b STRING, + | `extra col` ARRAY, + | `` STRUCT> + |) + |USING json + |OPTIONS ( + | PATH '$jsonFilePath' + |) + """.stripMargin + ) + + checkCreateTable("ddl_test") + } + } + + test("data source table CTAS") { + withTable("ddl_test") { + sql( + s"""CREATE TABLE ddl_test + |USING json + |AS SELECT 1 AS a, "foo" AS b + """.stripMargin + ) + + checkCreateTable("ddl_test") + } + } + + test("partitioned data source table") { + withTable("ddl_test") { + sql( + s"""CREATE TABLE ddl_test + |USING json + |PARTITIONED BY (b) + |AS SELECT 1 AS a, "foo" AS b + """.stripMargin + ) + + checkCreateTable("ddl_test") + } + } + + test("bucketed data source table") { + withTable("ddl_test") { + sql( + s"""CREATE TABLE ddl_test + |USING json + |CLUSTERED BY (a) SORTED BY (b) INTO 2 BUCKETS + |AS SELECT 1 AS a, "foo" AS b + """.stripMargin + ) + + checkCreateTable("ddl_test") + } + } + + test("partitioned bucketed data source table") { + withTable("ddl_test") { + sql( + s"""CREATE TABLE ddl_test + |USING json + |PARTITIONED BY (c) + |CLUSTERED BY (a) SORTED BY (b) INTO 2 BUCKETS + |AS SELECT 1 AS a, "foo" AS b, 2.5 AS c + """.stripMargin + ) + + checkCreateTable("ddl_test") + } + } + + test("data source table using Dataset API") { + withTable("ddl_test") { + spark + .range(3) + .select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd, 'id as 'e) + .write + .mode("overwrite") + .partitionBy("a", "b") + .bucketBy(2, "c", "d") + .saveAsTable("ddl_test") + + checkCreateTable("ddl_test") + } + } + + test("simple hive table") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |TBLPROPERTIES ( + | 'prop1' = 'value1', + | 'prop2' = 'value2' + |) + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("simple external hive table") { + withTempDir { dir => + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |LOCATION '${dir.toURI}' + |TBLPROPERTIES ( + | 'prop1' = 'value1', + | 'prop2' = 'value2' + |) + """.stripMargin + ) + + checkCreateTable("t1") + } + } + } + + test("partitioned hive table") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |COMMENT 'bla' + |PARTITIONED BY ( + | p1 BIGINT COMMENT 'bla', + | p2 STRING + |) + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive table with explicit storage info") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |COLLECTION ITEMS TERMINATED BY '@' + |MAP KEYS TERMINATED BY '#' + |NULL DEFINED AS 'NaN' + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive table with STORED AS clause") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |STORED AS PARQUET + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive table with serde info") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |WITH SERDEPROPERTIES ( + | 'mapkey.delim' = ',', + | 'field.delim' = ',' + |) + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive view") { + withView("v1") { + sql("CREATE VIEW v1 AS SELECT 1 AS a") + checkCreateView("v1") + } + } + + test("hive view with output columns") { + withView("v1") { + sql("CREATE VIEW v1 (b) AS SELECT 1 AS a") + checkCreateView("v1") + } + } + + test("hive bucketing is supported") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 (a INT, b STRING) + |CLUSTERED BY (a) + |SORTED BY (b) + |INTO 2 BUCKETS + """.stripMargin + ) + checkCreateTable("t1") + } + } + + test("hive partitioned view is not supported") { + withTable("t1") { + withView("v1") { + sql( + s""" + |CREATE TABLE t1 (c1 INT, c2 STRING) + |PARTITIONED BY ( + | p1 BIGINT COMMENT 'bla', + | p2 STRING ) + """.stripMargin) + + createRawHiveTable( + s""" + |CREATE VIEW v1 + |PARTITIONED ON (p1, p2) + |AS SELECT * from t1 + """.stripMargin + ) + + val cause = intercept[AnalysisException] { + sql("SHOW CREATE TABLE v1") + } + + assert(cause.getMessage.contains(" - partitioned view")) + } + } + } + + private def createRawHiveTable(ddl: String): Unit = { + hiveContext.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client.runSqlHive(ddl) + } + + private def checkCreateTable(table: String): Unit = { + checkCreateTableOrView(TableIdentifier(table, Some("default")), "TABLE") + } + + private def checkCreateView(table: String): Unit = { + checkCreateTableOrView(TableIdentifier(table, Some("default")), "VIEW") + } + + private def checkCreateTableOrView(table: TableIdentifier, checkType: String): Unit = { + val db = table.database.getOrElse("default") + val expected = spark.sharedState.externalCatalog.getTable(db, table.table) + val shownDDL = sql(s"SHOW CREATE TABLE ${table.quotedString}").head().getString(0) + sql(s"DROP $checkType ${table.quotedString}") + + try { + sql(shownDDL) + val actual = spark.sharedState.externalCatalog.getTable(db, table.table) + checkCatalogTables(expected, actual) + } finally { + sql(s"DROP $checkType IF EXISTS ${table.table}") + } + } + + private def checkCatalogTables(expected: CatalogTable, actual: CatalogTable): Unit = { + def normalize(table: CatalogTable): CatalogTable = { + val nondeterministicProps = Set( + "CreateTime", + "transient_lastDdlTime", + "grantTime", + "lastUpdateTime", + "last_modified_by", + "last_modified_time", + "Owner:", + // The following are hive specific schema parameters which we do not need to match exactly. + "totalNumberFiles", + "maxFileSize", + "minFileSize" + ) + + table.copy( + createTime = 0L, + lastAccessTime = 0L, + properties = table.properties.filterKeys(!nondeterministicProps.contains(_)), + stats = None, + ignoredProperties = Map.empty + ) + } + assert(normalize(actual) == normalize(expected)) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SparkSubmitTestUtils.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SparkSubmitTestUtils.scala new file mode 100644 index 000000000000..ede44df4afe1 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SparkSubmitTestUtils.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.io.File +import java.sql.Timestamp +import java.util.Date + +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.concurrent.TimeLimits +import org.scalatest.exceptions.TestFailedDueToTimeoutException +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer +import org.apache.spark.util.Utils + +trait SparkSubmitTestUtils extends SparkFunSuite with TimeLimits { + + // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. + // This is copied from org.apache.spark.deploy.SparkSubmitSuite + protected def runSparkSubmit(args: Seq[String], sparkHomeOpt: Option[String] = None): Unit = { + val sparkHome = sparkHomeOpt.getOrElse( + sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))) + val history = ArrayBuffer.empty[String] + val sparkSubmit = if (Utils.isWindows) { + // On Windows, `ProcessBuilder.directory` does not change the current working directory. + new File("..\\..\\bin\\spark-submit.cmd").getAbsolutePath + } else { + "./bin/spark-submit" + } + val commands = Seq(sparkSubmit) ++ args + val commandLine = commands.mkString("'", "' '", "'") + + val builder = new ProcessBuilder(commands: _*).directory(new File(sparkHome)) + val env = builder.environment() + env.put("SPARK_TESTING", "1") + env.put("SPARK_HOME", sparkHome) + + def captureOutput(source: String)(line: String): Unit = { + // This test suite has some weird behaviors when executed on Jenkins: + // + // 1. Sometimes it gets extremely slow out of unknown reason on Jenkins. Here we add a + // timestamp to provide more diagnosis information. + // 2. Log lines are not correctly redirected to unit-tests.log as expected, so here we print + // them out for debugging purposes. + val logLine = s"${new Timestamp(new Date().getTime)} - $source> $line" + // scalastyle:off println + println(logLine) + // scalastyle:on println + history += logLine + } + + val process = builder.start() + new ProcessOutputCapturer(process.getInputStream, captureOutput("stdout")).start() + new ProcessOutputCapturer(process.getErrorStream, captureOutput("stderr")).start() + + try { + val exitCode = failAfter(300.seconds) { process.waitFor() } + if (exitCode != 0) { + // include logs in output. Note that logging is async and may not have completed + // at the time this exception is raised + Thread.sleep(1000) + val historyLog = history.mkString("\n") + fail { + s"""spark-submit returned with exit code $exitCode. + |Command line: $commandLine + | + |$historyLog + """.stripMargin + } + } + } catch { + case to: TestFailedDueToTimeoutException => + val historyLog = history.mkString("\n") + fail(s"Timeout of $commandLine" + + s" See the log4j logs for more detail." + + s"\n$historyLog", to) + case t: Throwable => throw t + } finally { + // Ensure we still kill the process in case it timed out + process.destroy() + } + } +} 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 9bb32f11b76b..9ff9ecf7f367 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 @@ -17,112 +17,1141 @@ package org.apache.spark.sql.hive +import java.io.{File, PrintWriter} + import scala.reflect.ClassTag +import scala.util.matching.Regex + +import org.apache.hadoop.hive.common.StatsSetupConst -import org.apache.spark.sql.{Row, SQLConf, QueryTest} +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException +import org.apache.spark.sql.catalyst.catalog.{CatalogStatistics, HiveTableRelation} +import org.apache.spark.sql.catalyst.plans.logical.ColumnStat +import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.execution.command.DDLUtils +import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.hive.execution._ +import org.apache.spark.sql.hive.HiveExternalCatalog._ import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + + +class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleton { + test("Hive serde tables should fallback to HDFS for size estimation") { + withSQLConf(SQLConf.ENABLE_FALL_BACK_TO_HDFS_FOR_STATS.key -> "true") { + withTable("csv_table") { + withTempDir { tempDir => + // EXTERNAL OpenCSVSerde table pointing to LOCATION + val file1 = new File(tempDir + "/data1") + val writer1 = new PrintWriter(file1) + writer1.write("1,2") + writer1.close() -class StatisticsSuite extends QueryTest with TestHiveSingleton { - import hiveContext.sql + val file2 = new File(tempDir + "/data2") + val writer2 = new PrintWriter(file2) + writer2.write("1,2") + writer2.close() - test("parse analyze commands") { - def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) { - val parsed = HiveQl.parseSql(analyzeCommand) - val operators = parsed.collect { - case a: AnalyzeTable => a - case o => o + sql( + s""" + |CREATE EXTERNAL TABLE csv_table(page_id INT, impressions INT) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.OpenCSVSerde' + |WITH SERDEPROPERTIES ( + |\"separatorChar\" = \",\", + |\"quoteChar\" = \"\\\"\", + |\"escapeChar\" = \"\\\\\") + |LOCATION '${tempDir.toURI}'""".stripMargin) + + val relation = spark.table("csv_table").queryExecution.analyzed.children.head + .asInstanceOf[HiveTableRelation] + + val properties = relation.tableMeta.ignoredProperties + assert(properties("totalSize").toLong <= 0, "external table totalSize must be <= 0") + assert(properties("rawDataSize").toLong <= 0, "external table rawDataSize must be <= 0") + + val sizeInBytes = relation.stats.sizeInBytes + assert(sizeInBytes === BigInt(file1.length() + file2.length())) + } } + } + } - assert(operators.size === 1) - if (operators(0).getClass() != c) { - fail( - s"""$analyzeCommand expected command: $c, but got ${operators(0)} - |parsed command: - |$parsed - """.stripMargin) + test("analyze Hive serde tables") { + def queryTotalSize(tableName: String): BigInt = + spark.table(tableName).queryExecution.analyzed.stats.sizeInBytes + + // Non-partitioned table + val nonPartTable = "non_part_table" + withTable(nonPartTable) { + sql(s"CREATE TABLE $nonPartTable (key STRING, value STRING)") + sql(s"INSERT INTO TABLE $nonPartTable SELECT * FROM src") + sql(s"INSERT INTO TABLE $nonPartTable SELECT * FROM src") + + sql(s"ANALYZE TABLE $nonPartTable COMPUTE STATISTICS noscan") + + assert(queryTotalSize(nonPartTable) === BigInt(11624)) + } + + // Partitioned table + val partTable = "part_table" + withTable(partTable) { + sql(s"CREATE TABLE $partTable (key STRING, value STRING) PARTITIONED BY (ds STRING)") + sql(s"INSERT INTO TABLE $partTable PARTITION (ds='2010-01-01') SELECT * FROM src") + sql(s"INSERT INTO TABLE $partTable PARTITION (ds='2010-01-02') SELECT * FROM src") + sql(s"INSERT INTO TABLE $partTable PARTITION (ds='2010-01-03') SELECT * FROM src") + + assert(queryTotalSize(partTable) === spark.sessionState.conf.defaultSizeInBytes) + + sql(s"ANALYZE TABLE $partTable COMPUTE STATISTICS noscan") + + assert(queryTotalSize(partTable) === BigInt(17436)) + } + + // Try to analyze a temp table + withView("tempTable") { + sql("""SELECT * FROM src""").createOrReplaceTempView("tempTable") + intercept[AnalysisException] { + sql("ANALYZE TABLE tempTable COMPUTE STATISTICS") } } + } + + test("analyze non hive compatible datasource tables") { + val table = "parquet_tab" + withTable(table) { + sql( + s""" + |CREATE TABLE $table (a int, b int) + |USING parquet + |OPTIONS (skipHiveMetadata true) + """.stripMargin) - assertAnalyzeCommand( - "ANALYZE TABLE Table1 COMPUTE STATISTICS", - classOf[HiveNativeCommand]) - assertAnalyzeCommand( - "ANALYZE TABLE Table1 PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS", - classOf[HiveNativeCommand]) - assertAnalyzeCommand( - "ANALYZE TABLE Table1 PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS noscan", - classOf[HiveNativeCommand]) - assertAnalyzeCommand( - "ANALYZE TABLE Table1 PARTITION(ds, hr) COMPUTE STATISTICS", - classOf[HiveNativeCommand]) - assertAnalyzeCommand( - "ANALYZE TABLE Table1 PARTITION(ds, hr) COMPUTE STATISTICS noscan", - classOf[HiveNativeCommand]) + // Verify that the schema stored in catalog is a dummy one used for + // data source tables. The actual schema is stored in table properties. + val rawSchema = hiveClient.getTable("default", table).schema + val metadata = new MetadataBuilder().putString("comment", "from deserializer").build() + val expectedRawSchema = new StructType().add("col", "array", true, metadata) + assert(rawSchema == expectedRawSchema) - assertAnalyzeCommand( - "ANALYZE TABLE Table1 COMPUTE STATISTICS nOscAn", - classOf[AnalyzeTable]) + val actualSchema = spark.sharedState.externalCatalog.getTable("default", table).schema + val expectedActualSchema = new StructType() + .add("a", "int") + .add("b", "int") + assert(actualSchema == expectedActualSchema) + + sql(s"INSERT INTO $table VALUES (1, 1)") + sql(s"INSERT INTO $table VALUES (2, 1)") + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS a, b") + val fetchedStats0 = + checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(2)) + assert(fetchedStats0.get.colStats == Map( + "a" -> ColumnStat(2, Some(1), Some(2), 0, 4, 4), + "b" -> ColumnStat(1, Some(1), Some(1), 0, 4, 4))) + } } - test("analyze MetastoreRelations") { - def queryTotalSize(tableName: String): BigInt = - hiveContext.catalog.lookupRelation(TableIdentifier(tableName)).statistics.sizeInBytes + test("Analyze hive serde tables when schema is not same as schema in table properties") { + val table = "hive_serde" + withTable(table) { + sql(s"CREATE TABLE $table (C1 INT, C2 STRING, C3 DOUBLE)") - // Non-partitioned table - sql("CREATE TABLE analyzeTable (key STRING, value STRING)").collect() - sql("INSERT INTO TABLE analyzeTable SELECT * FROM src").collect() - sql("INSERT INTO TABLE analyzeTable SELECT * FROM src").collect() + // Verify that the table schema stored in hive catalog is + // different than the schema stored in table properties. + val rawSchema = hiveClient.getTable("default", table).schema + val expectedRawSchema = new StructType() + .add("c1", "int") + .add("c2", "string") + .add("c3", "double") + assert(rawSchema == expectedRawSchema) - sql("ANALYZE TABLE analyzeTable COMPUTE STATISTICS noscan") + val actualSchema = spark.sharedState.externalCatalog.getTable("default", table).schema + val expectedActualSchema = new StructType() + .add("C1", "int") + .add("C2", "string") + .add("C3", "double") + assert(actualSchema == expectedActualSchema) - assert(queryTotalSize("analyzeTable") === BigInt(11624)) + sql(s"INSERT INTO TABLE $table SELECT 1, 'a', 10.0") + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS C1") + val fetchedStats1 = + checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(1)).get + assert(fetchedStats1.colStats == Map( + "C1" -> ColumnStat(distinctCount = 1, min = Some(1), max = Some(1), nullCount = 0, + avgLen = 4, maxLen = 4))) + } + } - sql("DROP TABLE analyzeTable").collect() + test("SPARK-21079 - analyze table with location different than that of individual partitions") { + val tableName = "analyzeTable_part" + withTable(tableName) { + withTempPath { path => + sql(s"CREATE TABLE $tableName (key STRING, value STRING) PARTITIONED BY (ds STRING)") - // Partitioned table - sql( - """ - |CREATE TABLE analyzeTable_part (key STRING, value STRING) PARTITIONED BY (ds STRING) - """.stripMargin).collect() - sql( - """ - |INSERT INTO TABLE analyzeTable_part PARTITION (ds='2010-01-01') - |SELECT * FROM src - """.stripMargin).collect() - sql( - """ - |INSERT INTO TABLE analyzeTable_part PARTITION (ds='2010-01-02') - |SELECT * FROM src - """.stripMargin).collect() + val partitionDates = List("2010-01-01", "2010-01-02", "2010-01-03") + partitionDates.foreach { ds => + sql(s"INSERT INTO TABLE $tableName PARTITION (ds='$ds') SELECT * FROM src") + } + + sql(s"ALTER TABLE $tableName SET LOCATION '${path.toURI}'") + + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS noscan") + + assert(getCatalogStatistics(tableName).sizeInBytes === BigInt(17436)) + } + } + } + + test("SPARK-21079 - analyze partitioned table with only a subset of partitions visible") { + val sourceTableName = "analyzeTable_part" + val tableName = "analyzeTable_part_vis" + withTable(sourceTableName, tableName) { + withTempPath { path => + // Create a table with 3 partitions all located under a single top-level directory 'path' + sql( + s""" + |CREATE TABLE $sourceTableName (key STRING, value STRING) + |PARTITIONED BY (ds STRING) + |LOCATION '${path.toURI}' + """.stripMargin) + + val partitionDates = List("2010-01-01", "2010-01-02", "2010-01-03") + partitionDates.foreach { ds => + sql( + s""" + |INSERT INTO TABLE $sourceTableName PARTITION (ds='$ds') + |SELECT * FROM src + """.stripMargin) + } + + // Create another table referring to the same location + sql( + s""" + |CREATE TABLE $tableName (key STRING, value STRING) + |PARTITIONED BY (ds STRING) + |LOCATION '${path.toURI}' + """.stripMargin) + + // Register only one of the partitions found on disk + val ds = partitionDates.head + sql(s"ALTER TABLE $tableName ADD PARTITION (ds='$ds')") + + // Analyze original table - expect 3 partitions + sql(s"ANALYZE TABLE $sourceTableName COMPUTE STATISTICS noscan") + assert(getCatalogStatistics(sourceTableName).sizeInBytes === BigInt(3 * 5812)) + + // Analyze partial-copy table - expect only 1 partition + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS noscan") + assert(getCatalogStatistics(tableName).sizeInBytes === BigInt(5812)) + } + } + } + + test("analyze single partition") { + val tableName = "analyzeTable_part" + + def queryStats(ds: String): CatalogStatistics = { + val partition = + spark.sessionState.catalog.getPartition(TableIdentifier(tableName), Map("ds" -> ds)) + partition.stats.get + } + + def createPartition(ds: String, query: String): Unit = { + sql(s"INSERT INTO TABLE $tableName PARTITION (ds='$ds') $query") + } + + withTable(tableName) { + sql(s"CREATE TABLE $tableName (key STRING, value STRING) PARTITIONED BY (ds STRING)") + + createPartition("2010-01-01", "SELECT '1', 'A' from src") + createPartition("2010-01-02", "SELECT '1', 'A' from src UNION ALL SELECT '1', 'A' from src") + createPartition("2010-01-03", "SELECT '1', 'A' from src") + + sql(s"ANALYZE TABLE $tableName PARTITION (ds='2010-01-01') COMPUTE STATISTICS NOSCAN") + + sql(s"ANALYZE TABLE $tableName PARTITION (ds='2010-01-02') COMPUTE STATISTICS NOSCAN") + + assert(queryStats("2010-01-01").rowCount === None) + assert(queryStats("2010-01-01").sizeInBytes === 2000) + + assert(queryStats("2010-01-02").rowCount === None) + assert(queryStats("2010-01-02").sizeInBytes === 2*2000) + + sql(s"ANALYZE TABLE $tableName PARTITION (ds='2010-01-01') COMPUTE STATISTICS") + + sql(s"ANALYZE TABLE $tableName PARTITION (ds='2010-01-02') COMPUTE STATISTICS") + + assert(queryStats("2010-01-01").rowCount.get === 500) + assert(queryStats("2010-01-01").sizeInBytes === 2000) + + assert(queryStats("2010-01-02").rowCount.get === 2*500) + assert(queryStats("2010-01-02").sizeInBytes === 2*2000) + } + } + + test("analyze a set of partitions") { + val tableName = "analyzeTable_part" + + def queryStats(ds: String, hr: String): Option[CatalogStatistics] = { + val tableId = TableIdentifier(tableName) + val partition = + spark.sessionState.catalog.getPartition(tableId, Map("ds" -> ds, "hr" -> hr)) + partition.stats + } + + def assertPartitionStats( + ds: String, + hr: String, + rowCount: Option[BigInt], + sizeInBytes: BigInt): Unit = { + val stats = queryStats(ds, hr).get + assert(stats.rowCount === rowCount) + assert(stats.sizeInBytes === sizeInBytes) + } + + def createPartition(ds: String, hr: Int, query: String): Unit = { + sql(s"INSERT INTO TABLE $tableName PARTITION (ds='$ds', hr=$hr) $query") + } + + withTable(tableName) { + sql(s"CREATE TABLE $tableName (key STRING, value STRING) PARTITIONED BY (ds STRING, hr INT)") + + createPartition("2010-01-01", 10, "SELECT '1', 'A' from src") + createPartition("2010-01-01", 11, "SELECT '1', 'A' from src") + createPartition("2010-01-02", 10, "SELECT '1', 'A' from src") + createPartition("2010-01-02", 11, + "SELECT '1', 'A' from src UNION ALL SELECT '1', 'A' from src") + + sql(s"ANALYZE TABLE $tableName PARTITION (ds='2010-01-01') COMPUTE STATISTICS NOSCAN") + + assertPartitionStats("2010-01-01", "10", rowCount = None, sizeInBytes = 2000) + assertPartitionStats("2010-01-01", "11", rowCount = None, sizeInBytes = 2000) + assert(queryStats("2010-01-02", "10") === None) + assert(queryStats("2010-01-02", "11") === None) + + sql(s"ANALYZE TABLE $tableName PARTITION (ds='2010-01-02') COMPUTE STATISTICS NOSCAN") + + assertPartitionStats("2010-01-01", "10", rowCount = None, sizeInBytes = 2000) + assertPartitionStats("2010-01-01", "11", rowCount = None, sizeInBytes = 2000) + assertPartitionStats("2010-01-02", "10", rowCount = None, sizeInBytes = 2000) + assertPartitionStats("2010-01-02", "11", rowCount = None, sizeInBytes = 2*2000) + + sql(s"ANALYZE TABLE $tableName PARTITION (ds='2010-01-01') COMPUTE STATISTICS") + + assertPartitionStats("2010-01-01", "10", rowCount = Some(500), sizeInBytes = 2000) + assertPartitionStats("2010-01-01", "11", rowCount = Some(500), sizeInBytes = 2000) + assertPartitionStats("2010-01-02", "10", rowCount = None, sizeInBytes = 2000) + assertPartitionStats("2010-01-02", "11", rowCount = None, sizeInBytes = 2*2000) + + sql(s"ANALYZE TABLE $tableName PARTITION (ds='2010-01-02') COMPUTE STATISTICS") + + assertPartitionStats("2010-01-01", "10", rowCount = Some(500), sizeInBytes = 2000) + assertPartitionStats("2010-01-01", "11", rowCount = Some(500), sizeInBytes = 2000) + assertPartitionStats("2010-01-02", "10", rowCount = Some(500), sizeInBytes = 2000) + assertPartitionStats("2010-01-02", "11", rowCount = Some(2*500), sizeInBytes = 2*2000) + } + } + + test("analyze all partitions") { + val tableName = "analyzeTable_part" + + def assertPartitionStats( + ds: String, + hr: String, + rowCount: Option[BigInt], + sizeInBytes: BigInt): Unit = { + val stats = spark.sessionState.catalog.getPartition(TableIdentifier(tableName), + Map("ds" -> ds, "hr" -> hr)).stats.get + assert(stats.rowCount === rowCount) + assert(stats.sizeInBytes === sizeInBytes) + } + + def createPartition(ds: String, hr: Int, query: String): Unit = { + sql(s"INSERT INTO TABLE $tableName PARTITION (ds='$ds', hr=$hr) $query") + } + + withTable(tableName) { + sql(s"CREATE TABLE $tableName (key STRING, value STRING) PARTITIONED BY (ds STRING, hr INT)") + + createPartition("2010-01-01", 10, "SELECT '1', 'A' from src") + createPartition("2010-01-01", 11, "SELECT '1', 'A' from src") + createPartition("2010-01-02", 10, "SELECT '1', 'A' from src") + createPartition("2010-01-02", 11, + "SELECT '1', 'A' from src UNION ALL SELECT '1', 'A' from src") + + sql(s"ANALYZE TABLE $tableName PARTITION (ds, hr) COMPUTE STATISTICS NOSCAN") + + assertPartitionStats("2010-01-01", "10", rowCount = None, sizeInBytes = 2000) + assertPartitionStats("2010-01-01", "11", rowCount = None, sizeInBytes = 2000) + assertPartitionStats("2010-01-01", "11", rowCount = None, sizeInBytes = 2000) + assertPartitionStats("2010-01-02", "11", rowCount = None, sizeInBytes = 2*2000) + + sql(s"ANALYZE TABLE $tableName PARTITION (ds, hr) COMPUTE STATISTICS") + + assertPartitionStats("2010-01-01", "10", rowCount = Some(500), sizeInBytes = 2000) + assertPartitionStats("2010-01-01", "11", rowCount = Some(500), sizeInBytes = 2000) + assertPartitionStats("2010-01-01", "11", rowCount = Some(500), sizeInBytes = 2000) + assertPartitionStats("2010-01-02", "11", rowCount = Some(2*500), sizeInBytes = 2*2000) + } + } + + test("analyze partitions for an empty table") { + val tableName = "analyzeTable_part" + + withTable(tableName) { + sql(s"CREATE TABLE $tableName (key STRING, value STRING) PARTITIONED BY (ds STRING)") + + // make sure there is no exception + sql(s"ANALYZE TABLE $tableName PARTITION (ds) COMPUTE STATISTICS NOSCAN") + + // make sure there is no exception + sql(s"ANALYZE TABLE $tableName PARTITION (ds) COMPUTE STATISTICS") + } + } + + test("analyze partitions case sensitivity") { + val tableName = "analyzeTable_part" + withTable(tableName) { + sql(s"CREATE TABLE $tableName (key STRING, value STRING) PARTITIONED BY (ds STRING)") + + sql(s"INSERT INTO TABLE $tableName PARTITION (ds='2010-01-01') SELECT * FROM src") + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + sql(s"ANALYZE TABLE $tableName PARTITION (DS='2010-01-01') COMPUTE STATISTICS") + } + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val message = intercept[AnalysisException] { + sql(s"ANALYZE TABLE $tableName PARTITION (DS='2010-01-01') COMPUTE STATISTICS") + }.getMessage + assert(message.contains( + s"DS is not a valid partition column in table `default`.`${tableName.toLowerCase}`")) + } + } + } + + test("analyze partial partition specifications") { + + val tableName = "analyzeTable_part" + + def assertAnalysisException(partitionSpec: String): Unit = { + val message = intercept[AnalysisException] { + sql(s"ANALYZE TABLE $tableName $partitionSpec COMPUTE STATISTICS") + }.getMessage + assert(message.contains("The list of partition columns with values " + + s"in partition specification for table '${tableName.toLowerCase}' in database 'default' " + + "is not a prefix of the list of partition columns defined in the table schema")) + } + + withTable(tableName) { + sql( + s""" + |CREATE TABLE $tableName (key STRING, value STRING) + |PARTITIONED BY (a STRING, b INT, c STRING) + """.stripMargin) + + sql(s"INSERT INTO TABLE $tableName PARTITION (a='a1', b=10, c='c1') SELECT * FROM src") + + sql(s"ANALYZE TABLE $tableName PARTITION (a='a1') COMPUTE STATISTICS") + sql(s"ANALYZE TABLE $tableName PARTITION (a='a1', b=10) COMPUTE STATISTICS") + sql(s"ANALYZE TABLE $tableName PARTITION (A='a1', b=10) COMPUTE STATISTICS") + sql(s"ANALYZE TABLE $tableName PARTITION (b=10, a='a1') COMPUTE STATISTICS") + sql(s"ANALYZE TABLE $tableName PARTITION (b=10, A='a1') COMPUTE STATISTICS") + + assertAnalysisException("PARTITION (b=10)") + assertAnalysisException("PARTITION (a, b=10)") + assertAnalysisException("PARTITION (b=10, c='c1')") + assertAnalysisException("PARTITION (a, b=10, c='c1')") + assertAnalysisException("PARTITION (c='c1')") + assertAnalysisException("PARTITION (a, b, c='c1')") + assertAnalysisException("PARTITION (a='a1', c='c1')") + assertAnalysisException("PARTITION (a='a1', b, c='c1')") + } + } + + test("analyze non-existent partition") { + + def assertAnalysisException(analyzeCommand: String, errorMessage: String): Unit = { + val message = intercept[AnalysisException] { + sql(analyzeCommand) + }.getMessage + assert(message.contains(errorMessage)) + } + + val tableName = "analyzeTable_part" + withTable(tableName) { + sql(s"CREATE TABLE $tableName (key STRING, value STRING) PARTITIONED BY (ds STRING)") + + sql(s"INSERT INTO TABLE $tableName PARTITION (ds='2010-01-01') SELECT * FROM src") + + assertAnalysisException( + s"ANALYZE TABLE $tableName PARTITION (hour=20) COMPUTE STATISTICS", + s"hour is not a valid partition column in table `default`.`${tableName.toLowerCase}`" + ) + + assertAnalysisException( + s"ANALYZE TABLE $tableName PARTITION (hour) COMPUTE STATISTICS", + s"hour is not a valid partition column in table `default`.`${tableName.toLowerCase}`" + ) + + intercept[NoSuchPartitionException] { + sql(s"ANALYZE TABLE $tableName PARTITION (ds='2011-02-30') COMPUTE STATISTICS") + } + } + } + + test("test table-level statistics for hive tables created in HiveExternalCatalog") { + val textTable = "textTable" + withTable(textTable) { + // Currently Spark's statistics are self-contained, we don't have statistics until we use + // the `ANALYZE TABLE` command. + sql(s"CREATE TABLE $textTable (key STRING, value STRING) STORED AS TEXTFILE") + checkTableStats( + textTable, + hasSizeInBytes = false, + expectedRowCounts = None) + sql(s"INSERT INTO TABLE $textTable SELECT * FROM src") + checkTableStats( + textTable, + hasSizeInBytes = true, + expectedRowCounts = None) + + // noscan won't count the number of rows + sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS noscan") + val fetchedStats1 = + checkTableStats(textTable, hasSizeInBytes = true, expectedRowCounts = None) + + // without noscan, we count the number of rows + sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS") + val fetchedStats2 = + checkTableStats(textTable, hasSizeInBytes = true, expectedRowCounts = Some(500)) + assert(fetchedStats1.get.sizeInBytes == fetchedStats2.get.sizeInBytes) + } + } + + test("keep existing row count in stats with noscan if table is not changed") { + val textTable = "textTable" + withTable(textTable) { + sql(s"CREATE TABLE $textTable (key STRING, value STRING)") + sql(s"INSERT INTO TABLE $textTable SELECT * FROM src") + sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS") + val fetchedStats1 = + checkTableStats(textTable, hasSizeInBytes = true, expectedRowCounts = Some(500)) + + sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS noscan") + // when the table is not changed, total size is the same, and the old row count is kept + val fetchedStats2 = + checkTableStats(textTable, hasSizeInBytes = true, expectedRowCounts = Some(500)) + assert(fetchedStats1 == fetchedStats2) + } + } + + test("keep existing column stats if table is not changed") { + val table = "update_col_stats_table" + withTable(table) { + sql(s"CREATE TABLE $table (c1 INT, c2 STRING, c3 DOUBLE)") + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS c1") + val fetchedStats0 = + checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) + assert(fetchedStats0.get.colStats == Map("c1" -> ColumnStat(0, None, None, 0, 4, 4))) + + // Insert new data and analyze: have the latest column stats. + sql(s"INSERT INTO TABLE $table SELECT 1, 'a', 10.0") + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS c1") + val fetchedStats1 = + checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(1)).get + assert(fetchedStats1.colStats == Map( + "c1" -> ColumnStat(distinctCount = 1, min = Some(1), max = Some(1), nullCount = 0, + avgLen = 4, maxLen = 4))) + + // Analyze another column: since the table is not changed, the precious column stats are kept. + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS c2") + val fetchedStats2 = + checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(1)).get + assert(fetchedStats2.colStats == Map( + "c1" -> ColumnStat(distinctCount = 1, min = Some(1), max = Some(1), nullCount = 0, + avgLen = 4, maxLen = 4), + "c2" -> ColumnStat(distinctCount = 1, min = None, max = None, nullCount = 0, + avgLen = 1, maxLen = 1))) + + // Insert new data and analyze: stale column stats are removed and newly collected column + // stats are added. + sql(s"INSERT INTO TABLE $table SELECT 2, 'b', 20.0") + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS c1, c3") + val fetchedStats3 = + checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(2)).get + assert(fetchedStats3.colStats == Map( + "c1" -> ColumnStat(distinctCount = 2, min = Some(1), max = Some(2), nullCount = 0, + avgLen = 4, maxLen = 4), + "c3" -> ColumnStat(distinctCount = 2, min = Some(10.0), max = Some(20.0), nullCount = 0, + avgLen = 8, maxLen = 8))) + } + } + + private def createNonPartitionedTable( + tabName: String, + analyzedBySpark: Boolean = true, + analyzedByHive: Boolean = true): Unit = { sql( - """ - |INSERT INTO TABLE analyzeTable_part PARTITION (ds='2010-01-03') - |SELECT * FROM src - """.stripMargin).collect() + s""" + |CREATE TABLE $tabName (key STRING, value STRING) + |STORED AS TEXTFILE + |TBLPROPERTIES ('prop1' = 'val1', 'prop2' = 'val2') + """.stripMargin) + sql(s"INSERT INTO TABLE $tabName SELECT * FROM src") + if (analyzedBySpark) sql(s"ANALYZE TABLE $tabName COMPUTE STATISTICS") + // This is to mimic the scenario in which Hive genrates statistics before we reading it + if (analyzedByHive) hiveClient.runSqlHive(s"ANALYZE TABLE $tabName COMPUTE STATISTICS") + val describeResult1 = hiveClient.runSqlHive(s"DESCRIBE FORMATTED $tabName") + + val tableMetadata = getCatalogTable(tabName).properties + // statistics info is not contained in the metadata of the original table + assert(Seq(StatsSetupConst.COLUMN_STATS_ACCURATE, + StatsSetupConst.NUM_FILES, + StatsSetupConst.NUM_PARTITIONS, + StatsSetupConst.ROW_COUNT, + StatsSetupConst.RAW_DATA_SIZE, + StatsSetupConst.TOTAL_SIZE).forall(!tableMetadata.contains(_))) - assert(queryTotalSize("analyzeTable_part") === hiveContext.conf.defaultSizeInBytes) + if (analyzedByHive) { + assert(StringUtils.filterPattern(describeResult1, "*numRows\\s+500*").nonEmpty) + } else { + assert(StringUtils.filterPattern(describeResult1, "*numRows\\s+500*").isEmpty) + } + } - sql("ANALYZE TABLE analyzeTable_part COMPUTE STATISTICS noscan") + private def extractStatsPropValues( + descOutput: Seq[String], + propKey: String): Option[BigInt] = { + val str = descOutput + .filterNot(_.contains(STATISTICS_PREFIX)) + .filter(_.contains(propKey)) + if (str.isEmpty) { + None + } else { + assert(str.length == 1, "found more than one matches") + val pattern = new Regex(s"""$propKey\\s+(-?\\d+)""") + val pattern(value) = str.head.trim + Option(BigInt(value)) + } + } - assert(queryTotalSize("analyzeTable_part") === BigInt(17436)) + test("get statistics when not analyzed in Hive or Spark") { + val tabName = "tab1" + withTable(tabName) { + createNonPartitionedTable(tabName, analyzedByHive = false, analyzedBySpark = false) + checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = None) - sql("DROP TABLE analyzeTable_part").collect() + // ALTER TABLE SET TBLPROPERTIES invalidates some contents of Hive specific statistics + // This is triggered by the Hive alterTable API + val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED $tabName") - // Try to analyze a temp table - sql("""SELECT * FROM src""").registerTempTable("tempTable") - intercept[UnsupportedOperationException] { - hiveContext.analyze("tempTable") + val rawDataSize = extractStatsPropValues(describeResult, "rawDataSize") + val numRows = extractStatsPropValues(describeResult, "numRows") + val totalSize = extractStatsPropValues(describeResult, "totalSize") + assert(rawDataSize.isEmpty, "rawDataSize should not be shown without table analysis") + assert(numRows.isEmpty, "numRows should not be shown without table analysis") + assert(totalSize.isDefined && totalSize.get > 0, "totalSize is lost") } - hiveContext.catalog.unregisterTable(TableIdentifier("tempTable")) } - test("estimates the size of a test MetastoreRelation") { + test("alter table rename after analyze table") { + Seq(true, false).foreach { analyzedBySpark => + val oldName = "tab1" + val newName = "tab2" + withTable(oldName, newName) { + createNonPartitionedTable(oldName, analyzedByHive = true, analyzedBySpark = analyzedBySpark) + val fetchedStats1 = checkTableStats( + oldName, hasSizeInBytes = true, expectedRowCounts = Some(500)) + sql(s"ALTER TABLE $oldName RENAME TO $newName") + val fetchedStats2 = checkTableStats( + newName, hasSizeInBytes = true, expectedRowCounts = Some(500)) + assert(fetchedStats1 == fetchedStats2) + + // ALTER TABLE RENAME does not affect the contents of Hive specific statistics + val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED $newName") + + val rawDataSize = extractStatsPropValues(describeResult, "rawDataSize") + val numRows = extractStatsPropValues(describeResult, "numRows") + val totalSize = extractStatsPropValues(describeResult, "totalSize") + assert(rawDataSize.isDefined && rawDataSize.get > 0, "rawDataSize is lost") + assert(numRows.isDefined && numRows.get == 500, "numRows is lost") + assert(totalSize.isDefined && totalSize.get > 0, "totalSize is lost") + } + } + } + + test("alter table should not have the side effect to store statistics in Spark side") { + val table = "alter_table_side_effect" + withTable(table) { + sql(s"CREATE TABLE $table (i string, j string)") + sql(s"INSERT INTO TABLE $table SELECT 'a', 'b'") + val catalogTable1 = getCatalogTable(table) + val hiveSize1 = BigInt(catalogTable1.ignoredProperties(StatsSetupConst.TOTAL_SIZE)) + + sql(s"ALTER TABLE $table SET TBLPROPERTIES ('prop1' = 'a')") + + sql(s"INSERT INTO TABLE $table SELECT 'c', 'd'") + val catalogTable2 = getCatalogTable(table) + val hiveSize2 = BigInt(catalogTable2.ignoredProperties(StatsSetupConst.TOTAL_SIZE)) + // After insertion, Hive's stats should be changed. + assert(hiveSize2 > hiveSize1) + // We haven't generate stats in Spark, so we should still use Hive's stats here. + assert(catalogTable2.stats.get.sizeInBytes == hiveSize2) + } + } + + private def testAlterTableProperties(tabName: String, alterTablePropCmd: String): Unit = { + Seq(true, false).foreach { analyzedBySpark => + withTable(tabName) { + createNonPartitionedTable(tabName, analyzedByHive = true, analyzedBySpark = analyzedBySpark) + checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = Some(500)) + + // Run ALTER TABLE command + sql(alterTablePropCmd) + + val describeResult = hiveClient.runSqlHive(s"DESCRIBE FORMATTED $tabName") + + val totalSize = extractStatsPropValues(describeResult, "totalSize") + assert(totalSize.isDefined && totalSize.get > 0, "totalSize is lost") + + // ALTER TABLE SET/UNSET TBLPROPERTIES invalidates some Hive specific statistics, but not + // Spark specific statistics. This is triggered by the Hive alterTable API. + val numRows = extractStatsPropValues(describeResult, "numRows") + assert(numRows.isDefined && numRows.get == -1, "numRows is lost") + val rawDataSize = extractStatsPropValues(describeResult, "rawDataSize") + assert(rawDataSize.isDefined && rawDataSize.get == -1, "rawDataSize is lost") + + if (analyzedBySpark) { + checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = Some(500)) + } else { + checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = None) + } + } + } + } + + test("alter table SET TBLPROPERTIES after analyze table") { + testAlterTableProperties("set_prop_table", + "ALTER TABLE set_prop_table SET TBLPROPERTIES ('foo' = 'a')") + } + + test("alter table UNSET TBLPROPERTIES after analyze table") { + testAlterTableProperties("unset_prop_table", + "ALTER TABLE unset_prop_table UNSET TBLPROPERTIES ('prop1')") + } + + /** + * To see if stats exist, we need to check spark's stats properties instead of catalog + * statistics, because hive would change stats in metastore and thus change catalog statistics. + */ + private def getStatsProperties(tableName: String): Map[String, String] = { + val hTable = hiveClient.getTable(spark.sessionState.catalog.getCurrentDatabase, tableName) + hTable.properties.filterKeys(_.startsWith(STATISTICS_PREFIX)) + } + + test("change stats after insert command for hive table") { + val table = s"change_stats_insert_hive_table" + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + sql(s"CREATE TABLE $table (i int, j string)") + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) + assert(fetched1.get.sizeInBytes == 0) + assert(fetched1.get.colStats.size == 2) + + // insert into command + sql(s"INSERT INTO TABLE $table SELECT 1, 'abc'") + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes > 0) + assert(fetched2.get.colStats.isEmpty) + val statsProp = getStatsProperties(table) + assert(statsProp(STATISTICS_TOTAL_SIZE).toLong == fetched2.get.sizeInBytes) + } else { + assert(getStatsProperties(table).isEmpty) + } + } + } + } + } + + test("change stats after load data command") { + val table = "change_stats_load_table" + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + sql(s"CREATE TABLE $table (i INT, j STRING) STORED AS PARQUET") + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(0)) + assert(fetched1.get.sizeInBytes == 0) + assert(fetched1.get.colStats.size == 2) + + withTempDir { loadPath => + // load data command + val file = new File(loadPath + "/data") + val writer = new PrintWriter(file) + writer.write("2,xyz") + writer.close() + sql(s"LOAD DATA INPATH '${loadPath.toURI.toString}' INTO TABLE $table") + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes > 0) + assert(fetched2.get.colStats.isEmpty) + val statsProp = getStatsProperties(table) + assert(statsProp(STATISTICS_TOTAL_SIZE).toLong == fetched2.get.sizeInBytes) + } else { + assert(getStatsProperties(table).isEmpty) + } + } + } + } + } + } + + test("change stats after add/drop partition command") { + val table = "change_stats_part_table" + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_UPDATE_SIZE.key -> autoUpdate.toString) { + withTable(table) { + sql(s"CREATE TABLE $table (i INT, j STRING) PARTITIONED BY (ds STRING, hr STRING)") + // table has two partitions initially + for (ds <- Seq("2008-04-08"); hr <- Seq("11", "12")) { + sql(s"INSERT OVERWRITE TABLE $table PARTITION (ds='$ds',hr='$hr') SELECT 1, 'a'") + } + // analyze to get initial stats + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched1 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = Some(2)) + assert(fetched1.get.sizeInBytes > 0) + assert(fetched1.get.colStats.size == 2) + + withTempPaths(numPaths = 2) { case Seq(dir1, dir2) => + val file1 = new File(dir1 + "/data") + val writer1 = new PrintWriter(file1) + writer1.write("1,a") + writer1.close() + + val file2 = new File(dir2 + "/data") + val writer2 = new PrintWriter(file2) + writer2.write("1,a") + writer2.close() + + // add partition command + sql( + s""" + |ALTER TABLE $table ADD + |PARTITION (ds='2008-04-09', hr='11') LOCATION '${dir1.toURI.toString}' + |PARTITION (ds='2008-04-09', hr='12') LOCATION '${dir2.toURI.toString}' + """.stripMargin) + if (autoUpdate) { + val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched2.get.sizeInBytes > fetched1.get.sizeInBytes) + assert(fetched2.get.colStats.isEmpty) + val statsProp = getStatsProperties(table) + assert(statsProp(STATISTICS_TOTAL_SIZE).toLong == fetched2.get.sizeInBytes) + } else { + assert(getStatsProperties(table).isEmpty) + } + + // now the table has four partitions, generate stats again + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS i, j") + val fetched3 = checkTableStats( + table, hasSizeInBytes = true, expectedRowCounts = Some(4)) + assert(fetched3.get.sizeInBytes > 0) + assert(fetched3.get.colStats.size == 2) + + // drop partition command + sql(s"ALTER TABLE $table DROP PARTITION (ds='2008-04-08'), PARTITION (hr='12')") + assert(spark.sessionState.catalog.listPartitions(TableIdentifier(table)) + .map(_.spec).toSet == Set(Map("ds" -> "2008-04-09", "hr" -> "11"))) + // only one partition left + if (autoUpdate) { + val fetched4 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetched4.get.sizeInBytes < fetched1.get.sizeInBytes) + assert(fetched4.get.colStats.isEmpty) + val statsProp = getStatsProperties(table) + assert(statsProp(STATISTICS_TOTAL_SIZE).toLong == fetched4.get.sizeInBytes) + } else { + assert(getStatsProperties(table).isEmpty) + } + } + } + } + } + } + + test("add/drop partitions - managed table") { + val catalog = spark.sessionState.catalog + val managedTable = "partitionedTable" + withTable(managedTable) { + sql( + s""" + |CREATE TABLE $managedTable (key INT, value STRING) + |PARTITIONED BY (ds STRING, hr STRING) + """.stripMargin) + + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { + sql( + s""" + |INSERT OVERWRITE TABLE $managedTable + |partition (ds='$ds',hr='$hr') + |SELECT 1, 'a' + """.stripMargin) + } + + checkTableStats( + managedTable, hasSizeInBytes = false, expectedRowCounts = None) + + sql(s"ANALYZE TABLE $managedTable COMPUTE STATISTICS") + + val stats1 = checkTableStats( + managedTable, hasSizeInBytes = true, expectedRowCounts = Some(4)) + + sql( + s""" + |ALTER TABLE $managedTable DROP PARTITION (ds='2008-04-08'), + |PARTITION (hr='12') + """.stripMargin) + assert(catalog.listPartitions(TableIdentifier(managedTable)).map(_.spec).toSet == + Set(Map("ds" -> "2008-04-09", "hr" -> "11"))) + + sql(s"ANALYZE TABLE $managedTable COMPUTE STATISTICS") + + val stats2 = checkTableStats( + managedTable, hasSizeInBytes = true, expectedRowCounts = Some(1)) + assert(stats1.get.sizeInBytes > stats2.get.sizeInBytes) + + sql(s"ALTER TABLE $managedTable ADD PARTITION (ds='2008-04-08', hr='12')") + sql(s"ANALYZE TABLE $managedTable COMPUTE STATISTICS") + val stats4 = checkTableStats( + managedTable, hasSizeInBytes = true, expectedRowCounts = Some(1)) + + assert(stats1.get.sizeInBytes > stats4.get.sizeInBytes) + assert(stats4.get.sizeInBytes == stats2.get.sizeInBytes) + } + } + + test("test statistics of LogicalRelation converted from Hive serde tables") { + val parquetTable = "parquetTable" + val orcTable = "orcTable" + withTable(parquetTable, orcTable) { + sql(s"CREATE TABLE $parquetTable (key STRING, value STRING) STORED AS PARQUET") + sql(s"CREATE TABLE $orcTable (key STRING, value STRING) STORED AS ORC") + sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src") + sql(s"INSERT INTO TABLE $orcTable SELECT * FROM src") + + // the default value for `spark.sql.hive.convertMetastoreParquet` is true, here we just set it + // for robustness + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "true") { + checkTableStats(parquetTable, hasSizeInBytes = false, expectedRowCounts = None) + sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") + checkTableStats(parquetTable, hasSizeInBytes = true, expectedRowCounts = Some(500)) + } + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "true") { + // We still can get tableSize from Hive before Analyze + checkTableStats(orcTable, hasSizeInBytes = true, expectedRowCounts = None) + sql(s"ANALYZE TABLE $orcTable COMPUTE STATISTICS") + checkTableStats(orcTable, hasSizeInBytes = true, expectedRowCounts = Some(500)) + } + } + } + + test("verify serialized column stats after analyzing columns") { + import testImplicits._ + + val tableName = "column_stats_test2" + // (data.head.productArity - 1) because the last column does not support stats collection. + assert(stats.size == data.head.productArity - 1) + val df = data.toDF(stats.keys.toSeq :+ "carray" : _*) + + withTable(tableName) { + df.write.saveAsTable(tableName) + + // Collect statistics + sql(s"analyze table $tableName compute STATISTICS FOR COLUMNS " + stats.keys.mkString(", ")) + + // Validate statistics + val table = hiveClient.getTable("default", tableName) + + val props = table.properties.filterKeys(_.startsWith("spark.sql.statistics.colStats")) + assert(props == Map( + "spark.sql.statistics.colStats.cbinary.avgLen" -> "3", + "spark.sql.statistics.colStats.cbinary.distinctCount" -> "2", + "spark.sql.statistics.colStats.cbinary.maxLen" -> "3", + "spark.sql.statistics.colStats.cbinary.nullCount" -> "1", + "spark.sql.statistics.colStats.cbinary.version" -> "1", + "spark.sql.statistics.colStats.cbool.avgLen" -> "1", + "spark.sql.statistics.colStats.cbool.distinctCount" -> "2", + "spark.sql.statistics.colStats.cbool.max" -> "true", + "spark.sql.statistics.colStats.cbool.maxLen" -> "1", + "spark.sql.statistics.colStats.cbool.min" -> "false", + "spark.sql.statistics.colStats.cbool.nullCount" -> "1", + "spark.sql.statistics.colStats.cbool.version" -> "1", + "spark.sql.statistics.colStats.cbyte.avgLen" -> "1", + "spark.sql.statistics.colStats.cbyte.distinctCount" -> "2", + "spark.sql.statistics.colStats.cbyte.max" -> "2", + "spark.sql.statistics.colStats.cbyte.maxLen" -> "1", + "spark.sql.statistics.colStats.cbyte.min" -> "1", + "spark.sql.statistics.colStats.cbyte.nullCount" -> "1", + "spark.sql.statistics.colStats.cbyte.version" -> "1", + "spark.sql.statistics.colStats.cdate.avgLen" -> "4", + "spark.sql.statistics.colStats.cdate.distinctCount" -> "2", + "spark.sql.statistics.colStats.cdate.max" -> "2016-05-09", + "spark.sql.statistics.colStats.cdate.maxLen" -> "4", + "spark.sql.statistics.colStats.cdate.min" -> "2016-05-08", + "spark.sql.statistics.colStats.cdate.nullCount" -> "1", + "spark.sql.statistics.colStats.cdate.version" -> "1", + "spark.sql.statistics.colStats.cdecimal.avgLen" -> "16", + "spark.sql.statistics.colStats.cdecimal.distinctCount" -> "2", + "spark.sql.statistics.colStats.cdecimal.max" -> "8.000000000000000000", + "spark.sql.statistics.colStats.cdecimal.maxLen" -> "16", + "spark.sql.statistics.colStats.cdecimal.min" -> "1.000000000000000000", + "spark.sql.statistics.colStats.cdecimal.nullCount" -> "1", + "spark.sql.statistics.colStats.cdecimal.version" -> "1", + "spark.sql.statistics.colStats.cdouble.avgLen" -> "8", + "spark.sql.statistics.colStats.cdouble.distinctCount" -> "2", + "spark.sql.statistics.colStats.cdouble.max" -> "6.0", + "spark.sql.statistics.colStats.cdouble.maxLen" -> "8", + "spark.sql.statistics.colStats.cdouble.min" -> "1.0", + "spark.sql.statistics.colStats.cdouble.nullCount" -> "1", + "spark.sql.statistics.colStats.cdouble.version" -> "1", + "spark.sql.statistics.colStats.cfloat.avgLen" -> "4", + "spark.sql.statistics.colStats.cfloat.distinctCount" -> "2", + "spark.sql.statistics.colStats.cfloat.max" -> "7.0", + "spark.sql.statistics.colStats.cfloat.maxLen" -> "4", + "spark.sql.statistics.colStats.cfloat.min" -> "1.0", + "spark.sql.statistics.colStats.cfloat.nullCount" -> "1", + "spark.sql.statistics.colStats.cfloat.version" -> "1", + "spark.sql.statistics.colStats.cint.avgLen" -> "4", + "spark.sql.statistics.colStats.cint.distinctCount" -> "2", + "spark.sql.statistics.colStats.cint.max" -> "4", + "spark.sql.statistics.colStats.cint.maxLen" -> "4", + "spark.sql.statistics.colStats.cint.min" -> "1", + "spark.sql.statistics.colStats.cint.nullCount" -> "1", + "spark.sql.statistics.colStats.cint.version" -> "1", + "spark.sql.statistics.colStats.clong.avgLen" -> "8", + "spark.sql.statistics.colStats.clong.distinctCount" -> "2", + "spark.sql.statistics.colStats.clong.max" -> "5", + "spark.sql.statistics.colStats.clong.maxLen" -> "8", + "spark.sql.statistics.colStats.clong.min" -> "1", + "spark.sql.statistics.colStats.clong.nullCount" -> "1", + "spark.sql.statistics.colStats.clong.version" -> "1", + "spark.sql.statistics.colStats.cshort.avgLen" -> "2", + "spark.sql.statistics.colStats.cshort.distinctCount" -> "2", + "spark.sql.statistics.colStats.cshort.max" -> "3", + "spark.sql.statistics.colStats.cshort.maxLen" -> "2", + "spark.sql.statistics.colStats.cshort.min" -> "1", + "spark.sql.statistics.colStats.cshort.nullCount" -> "1", + "spark.sql.statistics.colStats.cshort.version" -> "1", + "spark.sql.statistics.colStats.cstring.avgLen" -> "3", + "spark.sql.statistics.colStats.cstring.distinctCount" -> "2", + "spark.sql.statistics.colStats.cstring.maxLen" -> "3", + "spark.sql.statistics.colStats.cstring.nullCount" -> "1", + "spark.sql.statistics.colStats.cstring.version" -> "1", + "spark.sql.statistics.colStats.ctimestamp.avgLen" -> "8", + "spark.sql.statistics.colStats.ctimestamp.distinctCount" -> "2", + "spark.sql.statistics.colStats.ctimestamp.max" -> "2016-05-09 00:00:02.0", + "spark.sql.statistics.colStats.ctimestamp.maxLen" -> "8", + "spark.sql.statistics.colStats.ctimestamp.min" -> "2016-05-08 00:00:01.0", + "spark.sql.statistics.colStats.ctimestamp.nullCount" -> "1", + "spark.sql.statistics.colStats.ctimestamp.version" -> "1" + )) + } + } + + private def testUpdatingTableStats(tableDescription: String, createTableCmd: String): Unit = { + test("test table-level statistics for " + tableDescription) { + val parquetTable = "parquetTable" + withTable(parquetTable) { + sql(createTableCmd) + val catalogTable = getCatalogTable(parquetTable) + assert(DDLUtils.isDatasourceTable(catalogTable)) + + // Add a filter to avoid creating too many partitions + sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src WHERE key < 10") + checkTableStats(parquetTable, hasSizeInBytes = false, expectedRowCounts = None) + + // noscan won't count the number of rows + sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS noscan") + val fetchedStats1 = + checkTableStats(parquetTable, hasSizeInBytes = true, expectedRowCounts = None) + + sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src WHERE key < 10") + sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS noscan") + val fetchedStats2 = + checkTableStats(parquetTable, hasSizeInBytes = true, expectedRowCounts = None) + assert(fetchedStats2.get.sizeInBytes > fetchedStats1.get.sizeInBytes) + + // without noscan, we count the number of rows + sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") + val fetchedStats3 = + checkTableStats(parquetTable, hasSizeInBytes = true, expectedRowCounts = Some(20)) + assert(fetchedStats3.get.sizeInBytes == fetchedStats2.get.sizeInBytes) + } + } + } + + testUpdatingTableStats( + "data source table created in HiveExternalCatalog", + "CREATE TABLE parquetTable (key STRING, value STRING) USING PARQUET") + + testUpdatingTableStats( + "partitioned data source table", + "CREATE TABLE parquetTable (key STRING, value STRING) USING PARQUET PARTITIONED BY (key)") + + /** Used to test refreshing cached metadata once table stats are updated. */ + private def getStatsBeforeAfterUpdate(isAnalyzeColumns: Boolean) + : (CatalogStatistics, CatalogStatistics) = { + val tableName = "tbl" + var statsBeforeUpdate: CatalogStatistics = null + var statsAfterUpdate: CatalogStatistics = null + withTable(tableName) { + val tableIndent = TableIdentifier(tableName, Some("default")) + val catalog = spark.sessionState.catalog.asInstanceOf[HiveSessionCatalog] + sql(s"CREATE TABLE $tableName (key int) USING PARQUET") + sql(s"INSERT INTO $tableName SELECT 1") + if (isAnalyzeColumns) { + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS key") + } else { + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS") + } + // Table lookup will make the table cached. + spark.table(tableIndent) + statsBeforeUpdate = catalog.metastoreCatalog.getCachedDataSourceTable(tableIndent) + .asInstanceOf[LogicalRelation].catalogTable.get.stats.get + + sql(s"INSERT INTO $tableName SELECT 2") + if (isAnalyzeColumns) { + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS key") + } else { + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS") + } + spark.table(tableIndent) + statsAfterUpdate = catalog.metastoreCatalog.getCachedDataSourceTable(tableIndent) + .asInstanceOf[LogicalRelation].catalogTable.get.stats.get + } + (statsBeforeUpdate, statsAfterUpdate) + } + + test("test refreshing table stats of cached data source table by `ANALYZE TABLE` statement") { + val (statsBeforeUpdate, statsAfterUpdate) = getStatsBeforeAfterUpdate(isAnalyzeColumns = false) + + assert(statsBeforeUpdate.sizeInBytes > 0) + assert(statsBeforeUpdate.rowCount == Some(1)) + + assert(statsAfterUpdate.sizeInBytes > statsBeforeUpdate.sizeInBytes) + assert(statsAfterUpdate.rowCount == Some(2)) + } + + test("estimates the size of a test Hive serde tables") { val df = sql("""SELECT * FROM src""") - val sizes = df.queryExecution.analyzed.collect { case mr: MetastoreRelation => - mr.statistics.sizeInBytes + val sizes = df.queryExecution.analyzed.collect { + case relation: HiveTableRelation => relation.stats.sizeInBytes } assert(sizes.size === 1, s"Size wrong for:\n ${df.queryExecution}") assert(sizes(0).equals(BigInt(5812)), @@ -142,31 +1171,31 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { // Assert src has a size smaller than the threshold. val sizes = df.queryExecution.analyzed.collect { - case r if ct.runtimeClass.isAssignableFrom(r.getClass) => r.statistics.sizeInBytes + case r if ct.runtimeClass.isAssignableFrom(r.getClass) => r.stats.sizeInBytes } - assert(sizes.size === 2 && sizes(0) <= hiveContext.conf.autoBroadcastJoinThreshold - && sizes(1) <= hiveContext.conf.autoBroadcastJoinThreshold, + assert(sizes.size === 2 && sizes(0) <= spark.sessionState.conf.autoBroadcastJoinThreshold + && sizes(1) <= spark.sessionState.conf.autoBroadcastJoinThreshold, s"query should contain two relations, each of which has size smaller than autoConvertSize") // Using `sparkPlan` because for relevant patterns in HashJoin to be // matched, other strategies need to be applied. - var bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } + var bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastHashJoinExec => j } assert(bhj.size === 1, s"actual query plans do not contain broadcast join: ${df.queryExecution}") checkAnswer(df, expectedAnswer) // check correctness of output - hiveContext.conf.settings.synchronized { - val tmp = hiveContext.conf.autoBroadcastJoinThreshold + spark.sessionState.conf.settings.synchronized { + val tmp = spark.sessionState.conf.autoBroadcastJoinThreshold sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1""") df = sql(query) - bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } + bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastHashJoinExec => j } assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") - val shj = df.queryExecution.sparkPlan.collect { case j: SortMergeJoin => j } + val shj = df.queryExecution.sparkPlan.collect { case j: SortMergeJoinExec => j } assert(shj.size === 1, - "ShuffledHashJoin should be planned when BroadcastHashJoin is turned off") + "SortMergeJoin should be planned when BroadcastHashJoin is turned off") sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=$tmp""") } @@ -174,7 +1203,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { after() } - /** Tests for MetastoreRelation */ + /** Tests for Hive serde tables */ val metastoreQuery = """SELECT * FROM src a JOIN src b ON a.key = 238 AND a.key = b.key""" val metastoreAnswer = Seq.fill(4)(Row(238, "val_238", 238, "val_238")) mkTest( @@ -182,7 +1211,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { () => (), metastoreQuery, metastoreAnswer, - implicitly[ClassTag[MetastoreRelation]] + implicitly[ClassTag[HiveTableRelation]] ) } @@ -196,39 +1225,37 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { // Assert src has a size smaller than the threshold. val sizes = df.queryExecution.analyzed.collect { - case r if implicitly[ClassTag[MetastoreRelation]].runtimeClass - .isAssignableFrom(r.getClass) => - r.statistics.sizeInBytes + case relation: HiveTableRelation => relation.stats.sizeInBytes } - assert(sizes.size === 2 && sizes(1) <= hiveContext.conf.autoBroadcastJoinThreshold - && sizes(0) <= hiveContext.conf.autoBroadcastJoinThreshold, + assert(sizes.size === 2 && sizes(1) <= spark.sessionState.conf.autoBroadcastJoinThreshold + && sizes(0) <= spark.sessionState.conf.autoBroadcastJoinThreshold, s"query should contain two relations, each of which has size smaller than autoConvertSize") // Using `sparkPlan` because for relevant patterns in HashJoin to be // matched, other strategies need to be applied. var bhj = df.queryExecution.sparkPlan.collect { - case j: BroadcastLeftSemiJoinHash => j + case j: BroadcastHashJoinExec => j } assert(bhj.size === 1, s"actual query plans do not contain broadcast join: ${df.queryExecution}") checkAnswer(df, answer) // check correctness of output - hiveContext.conf.settings.synchronized { - val tmp = hiveContext.conf.autoBroadcastJoinThreshold + spark.sessionState.conf.settings.synchronized { + val tmp = spark.sessionState.conf.autoBroadcastJoinThreshold sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1") df = sql(leftSemiJoinQuery) bhj = df.queryExecution.sparkPlan.collect { - case j: BroadcastLeftSemiJoinHash => j + case j: BroadcastHashJoinExec => j } assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") val shj = df.queryExecution.sparkPlan.collect { - case j: LeftSemiJoinHash => j + case j: SortMergeJoinExec => j } assert(shj.size === 1, - "LeftSemiJoinHash should be planned when BroadcastHashJoin is turned off") + "SortMergeJoinExec should be planned when BroadcastHashJoin is turned off") sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=$tmp") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/TestHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/TestHiveSuite.scala new file mode 100644 index 000000000000..72f8e8ff7c68 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/TestHiveSuite.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.hive.test.{TestHiveSingleton, TestHiveSparkSession} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils + + +class TestHiveSuite extends TestHiveSingleton with SQLTestUtils { + test("load test table based on case sensitivity") { + val testHiveSparkSession = spark.asInstanceOf[TestHiveSparkSession] + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + sql("SELECT * FROM SRC").queryExecution.analyzed + assert(testHiveSparkSession.getLoadedTables.contains("src")) + assert(testHiveSparkSession.getLoadedTables.size == 1) + } + testHiveSparkSession.reset() + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val err = intercept[AnalysisException] { + sql("SELECT * FROM SRC").queryExecution.analyzed + } + assert(err.message.contains("Table or view not found")) + } + testHiveSparkSession.reset() + } + + test("SPARK-15887: hive-site.xml should be loaded") { + assert(hiveClient.getConf("hive.in.test", "") == "true") + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala index 3ab457681119..88cc42efd0fe 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala @@ -17,19 +17,182 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.QueryTest +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils case class FunctionResult(f1: String, f2: String) -class UDFSuite extends QueryTest with TestHiveSingleton { +/** + * A test suite for UDF related functionalities. Because Hive metastore is + * case insensitive, database names and function names have both upper case + * letters and lower case letters. + */ +class UDFSuite + extends QueryTest + with SQLTestUtils + with TestHiveSingleton + with BeforeAndAfterEach { + + import spark.implicits._ + + private[this] val functionName = "myUPper" + private[this] val functionNameUpper = "MYUPPER" + private[this] val functionNameLower = "myupper" + + private[this] val functionClass = + classOf[org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper].getCanonicalName + + private var testDF: DataFrame = null + private[this] val testTableName = "testDF_UDFSuite" + private var expectedDF: DataFrame = null + + override def beforeAll(): Unit = { + sql("USE default") + + testDF = (1 to 10).map(i => s"sTr$i").toDF("value") + testDF.createOrReplaceTempView(testTableName) + expectedDF = (1 to 10).map(i => s"STR$i").toDF("value") + super.beforeAll() + } + + override def afterEach(): Unit = { + sql("USE default") + super.afterEach() + } test("UDF case insensitive") { - hiveContext.udf.register("random0", () => { Math.random() }) - hiveContext.udf.register("RANDOM1", () => { Math.random() }) - hiveContext.udf.register("strlenScala", (_: String).length + (_: Int)) - assert(hiveContext.sql("SELECT RANDOM0() FROM src LIMIT 1").head().getDouble(0) >= 0.0) - assert(hiveContext.sql("SELECT RANDOm1() FROM src LIMIT 1").head().getDouble(0) >= 0.0) - assert(hiveContext.sql("SELECT strlenscala('test', 1) FROM src LIMIT 1").head().getInt(0) === 5) + spark.udf.register("random0", () => { Math.random() }) + spark.udf.register("RANDOM1", () => { Math.random() }) + spark.udf.register("strlenScala", (_: String).length + (_: Int)) + assert(sql("SELECT RANDOM0() FROM src LIMIT 1").head().getDouble(0) >= 0.0) + assert(sql("SELECT RANDOm1() FROM src LIMIT 1").head().getDouble(0) >= 0.0) + assert(sql("SELECT strlenscala('test', 1) FROM src LIMIT 1").head().getInt(0) === 5) + } + + test("temporary function: create and drop") { + withUserDefinedFunction(functionName -> true) { + intercept[AnalysisException] { + sql(s"CREATE TEMPORARY FUNCTION default.$functionName AS '$functionClass'") + } + sql(s"CREATE TEMPORARY FUNCTION $functionName AS '$functionClass'") + checkAnswer( + sql(s"SELECT $functionNameLower(value) from $testTableName"), + expectedDF + ) + intercept[AnalysisException] { + sql(s"DROP TEMPORARY FUNCTION default.$functionName") + } + } + } + + test("permanent function: create and drop without specifying db name") { + withUserDefinedFunction(functionName -> false) { + sql(s"CREATE FUNCTION $functionName AS '$functionClass'") + checkAnswer( + sql("SHOW functions like '.*upper'"), + Row(s"default.$functionNameLower") + ) + checkAnswer( + sql(s"SELECT $functionName(value) from $testTableName"), + expectedDF + ) + assert( + sql("SHOW functions").collect() + .map(_.getString(0)) + .contains(s"default.$functionNameLower")) + } + } + + test("permanent function: create and drop with a db name") { + // For this block, drop function command uses functionName as the function name. + withUserDefinedFunction(functionNameUpper -> false) { + sql(s"CREATE FUNCTION default.$functionName AS '$functionClass'") + // TODO: Re-enable it after can distinguish qualified and unqualified function name + // in SessionCatalog.lookupFunction. + // checkAnswer( + // sql(s"SELECT default.myuPPer(value) from $testTableName"), + // expectedDF + // ) + checkAnswer( + sql(s"SELECT $functionName(value) from $testTableName"), + expectedDF + ) + checkAnswer( + sql(s"SELECT default.$functionName(value) from $testTableName"), + expectedDF + ) + } + + // For this block, drop function command uses default.functionName as the function name. + withUserDefinedFunction(s"DEfault.$functionNameLower" -> false) { + sql(s"CREATE FUNCTION dEFault.$functionName AS '$functionClass'") + checkAnswer( + sql(s"SELECT $functionNameUpper(value) from $testTableName"), + expectedDF + ) + } + } + + test("permanent function: create and drop a function in another db") { + // For this block, drop function command uses functionName as the function name. + withTempDatabase { dbName => + withUserDefinedFunction(functionName -> false) { + sql(s"CREATE FUNCTION $dbName.$functionName AS '$functionClass'") + // TODO: Re-enable it after can distinguish qualified and unqualified function name + // checkAnswer( + // sql(s"SELECT $dbName.myuPPer(value) from $testTableName"), + // expectedDF + // ) + + checkAnswer( + sql(s"SHOW FUNCTIONS like $dbName.$functionNameUpper"), + Row(s"$dbName.$functionNameLower") + ) + + sql(s"USE $dbName") + + checkAnswer( + sql(s"SELECT $functionName(value) from $testTableName"), + expectedDF + ) + + sql(s"USE default") + + checkAnswer( + sql(s"SELECT $dbName.$functionName(value) from $testTableName"), + expectedDF + ) + + sql(s"USE $dbName") + } + + sql(s"USE default") + + // For this block, drop function command uses default.functionName as the function name. + withUserDefinedFunction(s"$dbName.$functionNameUpper" -> false) { + sql(s"CREATE FUNCTION $dbName.$functionName AS '$functionClass'") + // TODO: Re-enable it after can distinguish qualified and unqualified function name + // checkAnswer( + // sql(s"SELECT $dbName.myupper(value) from $testTableName"), + // expectedDF + // ) + + sql(s"USE $dbName") + + assert( + sql("SHOW functions").collect() + .map(_.getString(0)) + .contains(s"$dbName.$functionNameLower")) + checkAnswer( + sql(s"SELECT $functionNameLower(value) from $testTableName"), + expectedDF + ) + + sql(s"USE default") + } + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala index 5e7b93d45710..031c1a5ec0ec 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala @@ -22,7 +22,8 @@ import java.util.Collections import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.spark.{Logging, SparkFunSuite} +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -64,8 +65,13 @@ class FiltersSuite extends SparkFunSuite with Logging { (Literal("") === a("varchar", StringType)) :: Nil, "") + filterTest("SPARK-19912 String literals should be escaped for Hive metastore partition pruning", + (a("stringcol", StringType) === Literal("p1\" and q=\"q1")) :: + (Literal("p2\" and q=\"q2") === a("stringcol", StringType)) :: Nil, + """stringcol = 'p1" and q="q1' and 'p2" and q="q2' = stringcol""") + private def filterTest(name: String, filters: Seq[Expression], result: String) = { - test(name){ + test(name) { val converted = shim.convertFilters(testTable, filters) if (converted != result) { fail( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientBuilder.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientBuilder.scala new file mode 100644 index 000000000000..ae804ce7c7b0 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientBuilder.scala @@ -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.sql.hive.client + +import java.io.File + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.util.VersionInfo + +import org.apache.spark.SparkConf +import org.apache.spark.util.Utils + +private[client] object HiveClientBuilder { + // In order to speed up test execution during development or in Jenkins, you can specify the path + // of an existing Ivy cache: + private val ivyPath: Option[String] = { + sys.env.get("SPARK_VERSIONS_SUITE_IVY_PATH").orElse( + Some(new File(sys.props("java.io.tmpdir"), "hive-ivy-cache").getAbsolutePath)) + } + + private def buildConf(extraConf: Map[String, String]) = { + lazy val warehousePath = Utils.createTempDir() + lazy val metastorePath = Utils.createTempDir() + metastorePath.delete() + extraConf ++ Map( + "javax.jdo.option.ConnectionURL" -> s"jdbc:derby:;databaseName=$metastorePath;create=true", + "hive.metastore.warehouse.dir" -> warehousePath.toString) + } + + // for testing only + def buildClient( + version: String, + hadoopConf: Configuration, + extraConf: Map[String, String] = Map.empty): HiveClient = { + IsolatedClientLoader.forVersion( + hiveMetastoreVersion = version, + hadoopVersion = VersionInfo.getVersion, + sparkConf = new SparkConf(), + hadoopConf = hadoopConf, + config = buildConf(extraConf), + ivyPath = ivyPath).createClient() + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala new file mode 100644 index 000000000000..3eedcf7e0874 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.client + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.conf.HiveConf +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.expressions.{EmptyRow, Expression, In, InSet} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser + +// TODO: Refactor this to `HivePartitionFilteringSuite` +class HiveClientSuite(version: String) + extends HiveVersionSuite(version) with BeforeAndAfterAll { + import CatalystSqlParser._ + + private val tryDirectSqlKey = HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname + + private val testPartitionCount = 3 * 24 * 4 + + private def init(tryDirectSql: Boolean): HiveClient = { + val storageFormat = CatalogStorageFormat( + locationUri = None, + inputFormat = None, + outputFormat = None, + serde = None, + compressed = false, + properties = Map.empty) + + val hadoopConf = new Configuration() + hadoopConf.setBoolean(tryDirectSqlKey, tryDirectSql) + val client = buildClient(hadoopConf) + client + .runSqlHive("CREATE TABLE test (value INT) PARTITIONED BY (ds INT, h INT, chunk STRING)") + + val partitions = + for { + ds <- 20170101 to 20170103 + h <- 0 to 23 + chunk <- Seq("aa", "ab", "ba", "bb") + } yield CatalogTablePartition(Map( + "ds" -> ds.toString, + "h" -> h.toString, + "chunk" -> chunk + ), storageFormat) + assert(partitions.size == testPartitionCount) + + client.createPartitions( + "default", "test", partitions, ignoreIfExists = false) + client + } + + override def beforeAll() { + client = init(true) + } + + test(s"getPartitionsByFilter returns all partitions when $tryDirectSqlKey=false") { + val client = init(false) + val filteredPartitions = client.getPartitionsByFilter(client.getTable("default", "test"), + Seq(parseExpression("ds=20170101"))) + + assert(filteredPartitions.size == testPartitionCount) + } + + test("getPartitionsByFilter: ds=20170101") { + testMetastorePartitionFiltering( + "ds=20170101", + 20170101 to 20170101, + 0 to 23, + "aa" :: "ab" :: "ba" :: "bb" :: Nil) + } + + test("getPartitionsByFilter: ds=(20170101 + 1) and h=0") { + // Should return all partitions where h=0 because getPartitionsByFilter does not support + // comparisons to non-literal values + testMetastorePartitionFiltering( + "ds=(20170101 + 1) and h=0", + 20170101 to 20170103, + 0 to 0, + "aa" :: "ab" :: "ba" :: "bb" :: Nil) + } + + test("getPartitionsByFilter: chunk='aa'") { + testMetastorePartitionFiltering( + "chunk='aa'", + 20170101 to 20170103, + 0 to 23, + "aa" :: Nil) + } + + test("getPartitionsByFilter: 20170101=ds") { + testMetastorePartitionFiltering( + "20170101=ds", + 20170101 to 20170101, + 0 to 23, + "aa" :: "ab" :: "ba" :: "bb" :: Nil) + } + + test("getPartitionsByFilter: ds=20170101 and h=10") { + testMetastorePartitionFiltering( + "ds=20170101 and h=10", + 20170101 to 20170101, + 10 to 10, + "aa" :: "ab" :: "ba" :: "bb" :: Nil) + } + + test("getPartitionsByFilter: ds=20170101 or ds=20170102") { + testMetastorePartitionFiltering( + "ds=20170101 or ds=20170102", + 20170101 to 20170102, + 0 to 23, + "aa" :: "ab" :: "ba" :: "bb" :: Nil) + } + + test("getPartitionsByFilter: ds in (20170102, 20170103) (using IN expression)") { + testMetastorePartitionFiltering( + "ds in (20170102, 20170103)", + 20170102 to 20170103, + 0 to 23, + "aa" :: "ab" :: "ba" :: "bb" :: Nil) + } + + test("getPartitionsByFilter: ds in (20170102, 20170103) (using INSET expression)") { + testMetastorePartitionFiltering( + "ds in (20170102, 20170103)", + 20170102 to 20170103, + 0 to 23, + "aa" :: "ab" :: "ba" :: "bb" :: Nil, { + case expr @ In(v, list) if expr.inSetConvertible => + InSet(v, list.map(_.eval(EmptyRow)).toSet) + }) + } + + test("getPartitionsByFilter: chunk in ('ab', 'ba') (using IN expression)") { + testMetastorePartitionFiltering( + "chunk in ('ab', 'ba')", + 20170101 to 20170103, + 0 to 23, + "ab" :: "ba" :: Nil) + } + + test("getPartitionsByFilter: chunk in ('ab', 'ba') (using INSET expression)") { + testMetastorePartitionFiltering( + "chunk in ('ab', 'ba')", + 20170101 to 20170103, + 0 to 23, + "ab" :: "ba" :: Nil, { + case expr @ In(v, list) if expr.inSetConvertible => + InSet(v, list.map(_.eval(EmptyRow)).toSet) + }) + } + + test("getPartitionsByFilter: (ds=20170101 and h>=8) or (ds=20170102 and h<8)") { + val day1 = (20170101 to 20170101, 8 to 23, Seq("aa", "ab", "ba", "bb")) + val day2 = (20170102 to 20170102, 0 to 7, Seq("aa", "ab", "ba", "bb")) + testMetastorePartitionFiltering( + "(ds=20170101 and h>=8) or (ds=20170102 and h<8)", + day1 :: day2 :: Nil) + } + + test("getPartitionsByFilter: (ds=20170101 and h>=8) or (ds=20170102 and h<(7+1))") { + val day1 = (20170101 to 20170101, 8 to 23, Seq("aa", "ab", "ba", "bb")) + // Day 2 should include all hours because we can't build a filter for h<(7+1) + val day2 = (20170102 to 20170102, 0 to 23, Seq("aa", "ab", "ba", "bb")) + testMetastorePartitionFiltering( + "(ds=20170101 and h>=8) or (ds=20170102 and h<(7+1))", + day1 :: day2 :: Nil) + } + + test("getPartitionsByFilter: " + + "chunk in ('ab', 'ba') and ((ds=20170101 and h>=8) or (ds=20170102 and h<8))") { + val day1 = (20170101 to 20170101, 8 to 23, Seq("ab", "ba")) + val day2 = (20170102 to 20170102, 0 to 7, Seq("ab", "ba")) + testMetastorePartitionFiltering( + "chunk in ('ab', 'ba') and ((ds=20170101 and h>=8) or (ds=20170102 and h<8))", + day1 :: day2 :: Nil) + } + + private def testMetastorePartitionFiltering( + filterString: String, + expectedDs: Seq[Int], + expectedH: Seq[Int], + expectedChunks: Seq[String]): Unit = { + testMetastorePartitionFiltering( + filterString, + (expectedDs, expectedH, expectedChunks) :: Nil, + identity) + } + + private def testMetastorePartitionFiltering( + filterString: String, + expectedDs: Seq[Int], + expectedH: Seq[Int], + expectedChunks: Seq[String], + transform: Expression => Expression): Unit = { + testMetastorePartitionFiltering( + filterString, + (expectedDs, expectedH, expectedChunks) :: Nil, + identity) + } + + private def testMetastorePartitionFiltering( + filterString: String, + expectedPartitionCubes: Seq[(Seq[Int], Seq[Int], Seq[String])]): Unit = { + testMetastorePartitionFiltering(filterString, expectedPartitionCubes, identity) + } + + private def testMetastorePartitionFiltering( + filterString: String, + expectedPartitionCubes: Seq[(Seq[Int], Seq[Int], Seq[String])], + transform: Expression => Expression): Unit = { + val filteredPartitions = client.getPartitionsByFilter(client.getTable("default", "test"), + Seq( + transform(parseExpression(filterString)) + )) + + val expectedPartitionCount = expectedPartitionCubes.map { + case (expectedDs, expectedH, expectedChunks) => + expectedDs.size * expectedH.size * expectedChunks.size + }.sum + + val expectedPartitions = expectedPartitionCubes.map { + case (expectedDs, expectedH, expectedChunks) => + for { + ds <- expectedDs + h <- expectedH + chunk <- expectedChunks + } yield Set( + "ds" -> ds.toString, + "h" -> h.toString, + "chunk" -> chunk + ) + }.reduce(_ ++ _) + + val actualFilteredPartitionCount = filteredPartitions.size + + assert(actualFilteredPartitionCount == expectedPartitionCount, + s"Expected $expectedPartitionCount partitions but got $actualFilteredPartitionCount") + assert(filteredPartitions.map(_.spec.toSet).toSet == expectedPartitions.toSet) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuites.scala new file mode 100644 index 000000000000..de1be2115b2d --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuites.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.client + +import scala.collection.immutable.IndexedSeq + +import org.scalatest.Suite + +class HiveClientSuites extends Suite with HiveClientVersions { + override def nestedSuites: IndexedSeq[Suite] = { + // Hive 0.12 does not provide the partition filtering API we call + versions.filterNot(_ == "0.12").map(new HiveClientSuite(_)) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala new file mode 100644 index 000000000000..2e7dfde8b2fa --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.client + +import scala.collection.immutable.IndexedSeq + +import org.apache.spark.SparkFunSuite + +private[client] trait HiveClientVersions { + protected val versions = IndexedSeq("0.12", "0.13", "0.14", "1.0", "1.1", "1.2", "2.0", "2.1") +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala new file mode 100644 index 000000000000..951ebfad4590 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.client + +import org.apache.hadoop.conf.Configuration +import org.scalactic.source.Position +import org.scalatest.Tag + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.hive.HiveUtils + +private[client] abstract class HiveVersionSuite(version: String) extends SparkFunSuite { + protected var client: HiveClient = null + + protected def buildClient(hadoopConf: Configuration): HiveClient = { + // Hive changed the default of datanucleus.schema.autoCreateAll from true to false and + // hive.metastore.schema.verification from false to true since 2.0 + // For details, see the JIRA HIVE-6113 and HIVE-12463 + if (version == "2.0" || version == "2.1") { + hadoopConf.set("datanucleus.schema.autoCreateAll", "true") + hadoopConf.set("hive.metastore.schema.verification", "false") + } + HiveClientBuilder + .buildClient(version, hadoopConf, HiveUtils.formatTimeVarsForHiveClient(hadoopConf)) + } + + override def suiteName: String = s"${super.suiteName}($version)" + + override protected def test(testName: String, testTags: Tag*)(testFun: => Any) + (implicit pos: Position): Unit = { + super.test(s"$version: $testName", testTags: _*)(testFun) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index c6d034a23a1c..edb9a9ffbaaf 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -17,45 +17,72 @@ package org.apache.spark.sql.hive.client -import java.io.File +import java.io.{ByteArrayOutputStream, File, PrintStream} +import java.net.URI -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.{Logging, SparkFunSuite} -import org.apache.spark.sql.catalyst.expressions.{NamedExpression, Literal, AttributeReference, EqualTo} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.apache.hadoop.mapred.TextInputFormat + +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{AnalysisException, Row} +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchPermanentFunctionException} +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} import org.apache.spark.sql.catalyst.util.quietly +import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} +import org.apache.spark.sql.hive.test.TestHiveVersion import org.apache.spark.sql.types.IntegerType +import org.apache.spark.sql.types.StructType import org.apache.spark.tags.ExtendedHiveTest -import org.apache.spark.util.Utils +import org.apache.spark.util.{MutableURLClassLoader, Utils} /** - * A simple set of tests that call the methods of a hive ClientInterface, loading different version + * A simple set of tests that call the methods of a [[HiveClient]], loading different version * of hive from maven central. These tests are simple in that they are mostly just testing to make * sure that reflective calls are not throwing NoSuchMethod error, but the actually functionality * is not fully tested. */ +// TODO: Refactor this to `HiveClientSuite` and make it a subclass of `HiveVersionSuite` @ExtendedHiveTest class VersionsSuite extends SparkFunSuite with Logging { - // Do not use a temp path here to speed up subsequent executions of the unit test during - // development. - private val ivyPath = Some( - new File(sys.props("java.io.tmpdir"), "hive-ivy-cache").getAbsolutePath()) - - private def buildConf() = { - lazy val warehousePath = Utils.createTempDir() - lazy val metastorePath = Utils.createTempDir() - metastorePath.delete() - Map( - "javax.jdo.option.ConnectionURL" -> s"jdbc:derby:;databaseName=$metastorePath;create=true", - "hive.metastore.warehouse.dir" -> warehousePath.toString) + import HiveClientBuilder.buildClient + + /** + * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` + * returns. + */ + protected def withTempDir(f: File => Unit): Unit = { + val dir = Utils.createTempDir().getCanonicalFile + try f(dir) finally Utils.deleteRecursively(dir) + } + + /** + * Drops table `tableName` after calling `f`. + */ + protected def withTable(tableNames: String*)(f: => Unit): Unit = { + try f finally { + tableNames.foreach { name => + versionSpark.sql(s"DROP TABLE IF EXISTS $name") + } + } } test("success sanity check") { - val badClient = IsolatedClientLoader.forVersion(HiveContext.hiveExecutionVersion, - buildConf(), - ivyPath).createClient() - val db = new HiveDatabase("default", "") - badClient.createDatabase(db) + val badClient = buildClient(HiveUtils.hiveExecutionVersion, new Configuration()) + val db = new CatalogDatabase("default", "desc", new URI("loc"), Map()) + badClient.createDatabase(db, ignoreIfExists = true) + } + + test("hadoop configuration preserved") { + val hadoopConf = new Configuration() + hadoopConf.set("test", "success") + val client = buildClient(HiveUtils.hiveExecutionVersion, hadoopConf) + assert("success" === client.getConf("test", null)) } private def getNestedMessages(e: Throwable): String = { @@ -70,136 +97,750 @@ class VersionsSuite extends SparkFunSuite with Logging { private val emptyDir = Utils.createTempDir().getCanonicalPath - private def partSpec = { - val hashMap = new java.util.LinkedHashMap[String, String] - hashMap.put("key", "1") - hashMap - } - // Its actually pretty easy to mess things up and have all of your tests "pass" by accidentally // connecting to an auto-populated, in-process metastore. Let's make sure we are getting the // versions right by forcing a known compatibility failure. // TODO: currently only works on mysql where we manually create the schema... ignore("failure sanity check") { val e = intercept[Throwable] { - val badClient = quietly { - IsolatedClientLoader.forVersion("13", buildConf(), ivyPath).createClient() - } + val badClient = quietly { buildClient("13", new Configuration()) } } assert(getNestedMessages(e) contains "Unknown column 'A0.OWNER_NAME' in 'field list'") } - private val versions = Seq("12", "13", "14", "1.0.0", "1.1.0", "1.2.0") + private val versions = Seq("0.12", "0.13", "0.14", "1.0", "1.1", "1.2", "2.0", "2.1") - private var client: ClientInterface = null + private var client: HiveClient = null + + private var versionSpark: TestHiveVersion = null versions.foreach { version => test(s"$version: create client") { client = null System.gc() // Hack to avoid SEGV on some JVM versions. - client = IsolatedClientLoader.forVersion(version, buildConf(), ivyPath).createClient() + val hadoopConf = new Configuration() + hadoopConf.set("test", "success") + // Hive changed the default of datanucleus.schema.autoCreateAll from true to false and + // hive.metastore.schema.verification from false to true since 2.0 + // For details, see the JIRA HIVE-6113 and HIVE-12463 + if (version == "2.0" || version == "2.1") { + hadoopConf.set("datanucleus.schema.autoCreateAll", "true") + hadoopConf.set("hive.metastore.schema.verification", "false") + } + client = buildClient(version, hadoopConf, HiveUtils.formatTimeVarsForHiveClient(hadoopConf)) + if (versionSpark != null) versionSpark.reset() + versionSpark = TestHiveVersion(client) + assert(versionSpark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client + .version.fullVersion.startsWith(version)) + } + + def table(database: String, tableName: String): CatalogTable = { + CatalogTable( + identifier = TableIdentifier(tableName, Some(database)), + tableType = CatalogTableType.MANAGED, + schema = new StructType().add("key", "int"), + storage = CatalogStorageFormat( + locationUri = None, + inputFormat = Some(classOf[TextInputFormat].getName), + outputFormat = Some(classOf[HiveIgnoreKeyTextOutputFormat[_, _]].getName), + serde = Some(classOf[LazySimpleSerDe].getName()), + compressed = false, + properties = Map.empty + )) } + /////////////////////////////////////////////////////////////////////////// + // Database related API + /////////////////////////////////////////////////////////////////////////// + + val tempDatabasePath = Utils.createTempDir().toURI + test(s"$version: createDatabase") { - val db = HiveDatabase("default", "") - client.createDatabase(db) + val defaultDB = CatalogDatabase("default", "desc", new URI("loc"), Map()) + client.createDatabase(defaultDB, ignoreIfExists = true) + val tempDB = CatalogDatabase( + "temporary", description = "test create", tempDatabasePath, Map()) + client.createDatabase(tempDB, ignoreIfExists = true) + } + + test(s"$version: setCurrentDatabase") { + client.setCurrentDatabase("default") + } + + test(s"$version: getDatabase") { + // No exception should be thrown + client.getDatabase("default") + intercept[NoSuchDatabaseException](client.getDatabase("nonexist")) + } + + test(s"$version: databaseExists") { + assert(client.databaseExists("default") == true) + assert(client.databaseExists("nonexist") == false) } + test(s"$version: listDatabases") { + assert(client.listDatabases("defau.*") == Seq("default")) + } + + test(s"$version: alterDatabase") { + val database = client.getDatabase("temporary").copy(properties = Map("flag" -> "true")) + client.alterDatabase(database) + assert(client.getDatabase("temporary").properties.contains("flag")) + } + + test(s"$version: dropDatabase") { + assert(client.databaseExists("temporary") == true) + client.dropDatabase("temporary", ignoreIfNotExists = false, cascade = true) + assert(client.databaseExists("temporary") == false) + } + + /////////////////////////////////////////////////////////////////////////// + // Table related API + /////////////////////////////////////////////////////////////////////////// + test(s"$version: createTable") { - val table = - HiveTable( - specifiedDatabase = Option("default"), - name = "src", - schema = Seq(HiveColumn("key", "int", "")), - partitionColumns = Seq.empty, - properties = Map.empty, - serdeProperties = Map.empty, - tableType = ManagedTable, - location = None, - inputFormat = - Some(classOf[org.apache.hadoop.mapred.TextInputFormat].getName), - outputFormat = - Some(classOf[org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat[_, _]].getName), - serde = - Some(classOf[org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe].getName())) - - client.createTable(table) + client.createTable(table("default", tableName = "src"), ignoreIfExists = false) + client.createTable(table("default", "temporary"), ignoreIfExists = false) + } + + test(s"$version: loadTable") { + client.loadTable( + emptyDir, + tableName = "src", + replace = false, + isSrcLocal = false) + } + + test(s"$version: tableExists") { + // No exception should be thrown + assert(client.tableExists("default", "src")) + assert(!client.tableExists("default", "nonexistent")) } test(s"$version: getTable") { + // No exception should be thrown client.getTable("default", "src") } - test(s"$version: listTables") { - assert(client.listTables("default") === Seq("src")) + test(s"$version: getTableOption") { + assert(client.getTableOption("default", "src").isDefined) } - test(s"$version: currentDatabase") { - assert(client.currentDatabase === "default") + test(s"$version: alterTable(table: CatalogTable)") { + val newTable = client.getTable("default", "src").copy(properties = Map("changed" -> "")) + client.alterTable(newTable) + assert(client.getTable("default", "src").properties.contains("changed")) } - test(s"$version: getDatabase") { - client.getDatabase("default") + test(s"$version: alterTable(dbName: String, tableName: String, table: CatalogTable)") { + val newTable = client.getTable("default", "src").copy(properties = Map("changedAgain" -> "")) + client.alterTable("default", "src", newTable) + assert(client.getTable("default", "src").properties.contains("changedAgain")) + } + + test(s"$version: alterTable - rename") { + val newTable = client.getTable("default", "src") + .copy(identifier = TableIdentifier("tgt", database = Some("default"))) + assert(!client.tableExists("default", "tgt")) + + client.alterTable("default", "src", newTable) + + assert(client.tableExists("default", "tgt")) + assert(!client.tableExists("default", "src")) } - test(s"$version: alterTable") { - client.alterTable(client.getTable("default", "src")) + test(s"$version: alterTable - change database") { + val tempDB = CatalogDatabase( + "temporary", description = "test create", tempDatabasePath, Map()) + client.createDatabase(tempDB, ignoreIfExists = true) + + val newTable = client.getTable("default", "tgt") + .copy(identifier = TableIdentifier("tgt", database = Some("temporary"))) + assert(!client.tableExists("temporary", "tgt")) + + client.alterTable("default", "tgt", newTable) + + assert(client.tableExists("temporary", "tgt")) + assert(!client.tableExists("default", "tgt")) } - test(s"$version: set command") { - client.runSqlHive("SET spark.sql.test.key=1") + test(s"$version: alterTable - change database and table names") { + val newTable = client.getTable("temporary", "tgt") + .copy(identifier = TableIdentifier("src", database = Some("default"))) + assert(!client.tableExists("default", "src")) + + client.alterTable("temporary", "tgt", newTable) + + assert(client.tableExists("default", "src")) + assert(!client.tableExists("temporary", "tgt")) + } + + test(s"$version: listTables(database)") { + assert(client.listTables("default") === Seq("src", "temporary")) + } + + test(s"$version: listTables(database, pattern)") { + assert(client.listTables("default", pattern = "src") === Seq("src")) + assert(client.listTables("default", pattern = "nonexist").isEmpty) + } + + test(s"$version: dropTable") { + val versionsWithoutPurge = versions.takeWhile(_ != "0.14") + // First try with the purge option set. This should fail if the version is < 0.14, in which + // case we check the version and try without it. + try { + client.dropTable("default", tableName = "temporary", ignoreIfNotExists = false, + purge = true) + assert(!versionsWithoutPurge.contains(version)) + } catch { + case _: UnsupportedOperationException => + assert(versionsWithoutPurge.contains(version)) + client.dropTable("default", tableName = "temporary", ignoreIfNotExists = false, + purge = false) + } + assert(client.listTables("default") === Seq("src")) + } + + /////////////////////////////////////////////////////////////////////////// + // Partition related API + /////////////////////////////////////////////////////////////////////////// + + val storageFormat = CatalogStorageFormat( + locationUri = None, + inputFormat = None, + outputFormat = None, + serde = None, + compressed = false, + properties = Map.empty) + + test(s"$version: sql create partitioned table") { + client.runSqlHive("CREATE TABLE src_part (value INT) PARTITIONED BY (key1 INT, key2 INT)") + } + + val testPartitionCount = 2 + + test(s"$version: createPartitions") { + val partitions = (1 to testPartitionCount).map { key2 => + CatalogTablePartition(Map("key1" -> "1", "key2" -> key2.toString), storageFormat) + } + client.createPartitions( + "default", "src_part", partitions, ignoreIfExists = true) } - test(s"$version: create partitioned table DDL") { - client.runSqlHive("CREATE TABLE src_part (value INT) PARTITIONED BY (key INT)") - client.runSqlHive("ALTER TABLE src_part ADD PARTITION (key = '1')") + test(s"$version: getPartitionNames(catalogTable)") { + val partitionNames = (1 to testPartitionCount).map(key2 => s"key1=1/key2=$key2") + assert(partitionNames == client.getPartitionNames(client.getTable("default", "src_part"))) } - test(s"$version: getPartitions") { - client.getAllPartitions(client.getTable("default", "src_part")) + test(s"$version: getPartitions(catalogTable)") { + assert(testPartitionCount == + client.getPartitions(client.getTable("default", "src_part")).size) } test(s"$version: getPartitionsByFilter") { - client.getPartitionsByFilter(client.getTable("default", "src_part"), Seq(EqualTo( - AttributeReference("key", IntegerType, false)(NamedExpression.newExprId), - Literal(1)))) + // Only one partition [1, 1] for key2 == 1 + val result = client.getPartitionsByFilter(client.getTable("default", "src_part"), + Seq(EqualTo(AttributeReference("key2", IntegerType)(), Literal(1)))) + + // Hive 0.12 doesn't support getPartitionsByFilter, it ignores the filter condition. + if (version != "0.12") { + assert(result.size == 1) + } else { + assert(result.size == testPartitionCount) + } + } + + test(s"$version: getPartition") { + // No exception should be thrown + client.getPartition("default", "src_part", Map("key1" -> "1", "key2" -> "2")) + } + + test(s"$version: getPartitionOption(db: String, table: String, spec: TablePartitionSpec)") { + val partition = client.getPartitionOption( + "default", "src_part", Map("key1" -> "1", "key2" -> "2")) + assert(partition.isDefined) + } + + test(s"$version: getPartitionOption(table: CatalogTable, spec: TablePartitionSpec)") { + val partition = client.getPartitionOption( + client.getTable("default", "src_part"), Map("key1" -> "1", "key2" -> "2")) + assert(partition.isDefined) + } + + test(s"$version: getPartitions(db: String, table: String)") { + assert(testPartitionCount == client.getPartitions("default", "src_part", None).size) } test(s"$version: loadPartition") { + val partSpec = new java.util.LinkedHashMap[String, String] + partSpec.put("key1", "1") + partSpec.put("key2", "2") + client.loadPartition( emptyDir, - "default.src_part", + "default", + "src_part", partSpec, - false, - false, - false, - false) - } - - test(s"$version: loadTable") { - client.loadTable( - emptyDir, - "src", - false, - false) + replace = false, + inheritTableSpecs = false, + isSrcLocal = false) } test(s"$version: loadDynamicPartitions") { + val partSpec = new java.util.LinkedHashMap[String, String] + partSpec.put("key1", "1") + partSpec.put("key2", "") // Dynamic partition + client.loadDynamicPartitions( emptyDir, - "default.src_part", + "default", + "src_part", partSpec, - false, - 1, - false, - false) + replace = false, + numDP = 1) + } + + test(s"$version: renamePartitions") { + val oldSpec = Map("key1" -> "1", "key2" -> "1") + val newSpec = Map("key1" -> "1", "key2" -> "3") + client.renamePartitions("default", "src_part", Seq(oldSpec), Seq(newSpec)) + + // Checks the existence of the new partition (key1 = 1, key2 = 3) + assert(client.getPartitionOption("default", "src_part", newSpec).isDefined) } - test(s"$version: create index and reset") { + test(s"$version: alterPartitions") { + val spec = Map("key1" -> "1", "key2" -> "2") + val newLocation = new URI(Utils.createTempDir().toURI.toString.stripSuffix("/")) + val storage = storageFormat.copy( + locationUri = Some(newLocation), + // needed for 0.12 alter partitions + serde = Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) + val partition = CatalogTablePartition(spec, storage) + client.alterPartitions("default", "src_part", Seq(partition)) + assert(client.getPartition("default", "src_part", spec) + .storage.locationUri == Some(newLocation)) + } + + test(s"$version: dropPartitions") { + val spec = Map("key1" -> "1", "key2" -> "3") + val versionsWithoutPurge = versions.takeWhile(_ != "1.2") + // Similar to dropTable; try with purge set, and if it fails, make sure we're running + // with a version that is older than the minimum (1.2 in this case). + try { + client.dropPartitions("default", "src_part", Seq(spec), ignoreIfNotExists = true, + purge = true, retainData = false) + assert(!versionsWithoutPurge.contains(version)) + } catch { + case _: UnsupportedOperationException => + assert(versionsWithoutPurge.contains(version)) + client.dropPartitions("default", "src_part", Seq(spec), ignoreIfNotExists = true, + purge = false, retainData = false) + } + + assert(client.getPartitionOption("default", "src_part", spec).isEmpty) + } + + /////////////////////////////////////////////////////////////////////////// + // Function related API + /////////////////////////////////////////////////////////////////////////// + + def function(name: String, className: String): CatalogFunction = { + CatalogFunction( + FunctionIdentifier(name, Some("default")), className, Seq.empty[FunctionResource]) + } + + test(s"$version: createFunction") { + val functionClass = "org.apache.spark.MyFunc1" + if (version == "0.12") { + // Hive 0.12 doesn't support creating permanent functions + intercept[AnalysisException] { + client.createFunction("default", function("func1", functionClass)) + } + } else { + client.createFunction("default", function("func1", functionClass)) + } + } + + test(s"$version: functionExists") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + assert(client.functionExists("default", "func1") == false) + } else { + assert(client.functionExists("default", "func1") == true) + } + } + + test(s"$version: renameFunction") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + intercept[NoSuchPermanentFunctionException] { + client.renameFunction("default", "func1", "func2") + } + } else { + client.renameFunction("default", "func1", "func2") + assert(client.functionExists("default", "func2") == true) + } + } + + test(s"$version: alterFunction") { + val functionClass = "org.apache.spark.MyFunc2" + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + intercept[NoSuchPermanentFunctionException] { + client.alterFunction("default", function("func2", functionClass)) + } + } else { + client.alterFunction("default", function("func2", functionClass)) + } + } + + test(s"$version: getFunction") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + intercept[NoSuchPermanentFunctionException] { + client.getFunction("default", "func2") + } + } else { + // No exception should be thrown + val func = client.getFunction("default", "func2") + assert(func.className == "org.apache.spark.MyFunc2") + } + } + + test(s"$version: getFunctionOption") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + assert(client.getFunctionOption("default", "func2").isEmpty) + } else { + assert(client.getFunctionOption("default", "func2").isDefined) + assert(client.getFunctionOption("default", "the_func_not_exists").isEmpty) + } + } + + test(s"$version: listFunctions") { + if (version == "0.12") { + // Hive 0.12 doesn't allow customized permanent functions + assert(client.listFunctions("default", "fun.*").isEmpty) + } else { + assert(client.listFunctions("default", "fun.*").size == 1) + } + } + + test(s"$version: dropFunction") { + if (version == "0.12") { + // Hive 0.12 doesn't support creating permanent functions + intercept[NoSuchPermanentFunctionException] { + client.dropFunction("default", "func2") + } + } else { + // No exception should be thrown + client.dropFunction("default", "func2") + assert(client.listFunctions("default", "fun.*").size == 0) + } + } + + /////////////////////////////////////////////////////////////////////////// + // SQL related API + /////////////////////////////////////////////////////////////////////////// + + test(s"$version: sql set command") { + client.runSqlHive("SET spark.sql.test.key=1") + } + + test(s"$version: sql create index and reset") { client.runSqlHive("CREATE TABLE indexed_table (key INT)") client.runSqlHive("CREATE INDEX index_1 ON TABLE indexed_table(key) " + "as 'COMPACT' WITH DEFERRED REBUILD") + } + + /////////////////////////////////////////////////////////////////////////// + // Miscellaneous API + /////////////////////////////////////////////////////////////////////////// + + test(s"$version: version") { + assert(client.version.fullVersion.startsWith(version)) + } + + test(s"$version: getConf") { + assert("success" === client.getConf("test", null)) + } + + test(s"$version: setOut") { + client.setOut(new PrintStream(new ByteArrayOutputStream())) + } + + test(s"$version: setInfo") { + client.setInfo(new PrintStream(new ByteArrayOutputStream())) + } + + test(s"$version: setError") { + client.setError(new PrintStream(new ByteArrayOutputStream())) + } + + test(s"$version: newSession") { + val newClient = client.newSession() + assert(newClient != null) + } + + test(s"$version: withHiveState and addJar") { + val newClassPath = "." + client.addJar(newClassPath) + client.withHiveState { + // No exception should be thrown. + // withHiveState changes the classloader to MutableURLClassLoader + val classLoader = Thread.currentThread().getContextClassLoader + .asInstanceOf[MutableURLClassLoader] + + val urls = classLoader.getURLs() + urls.contains(new File(newClassPath).toURI.toURL) + } + } + + test(s"$version: reset") { + // Clears all database, tables, functions... client.reset() + assert(client.listTables("default").isEmpty) + } + + /////////////////////////////////////////////////////////////////////////// + // End-To-End tests + /////////////////////////////////////////////////////////////////////////// + + test(s"$version: CREATE TABLE AS SELECT") { + withTable("tbl") { + versionSpark.sql("CREATE TABLE tbl AS SELECT 1 AS a") + assert(versionSpark.table("tbl").collect().toSeq == Seq(Row(1))) + val tableMeta = versionSpark.sessionState.catalog.getTableMetadata(TableIdentifier("tbl")) + val totalSize = tableMeta.stats.map(_.sizeInBytes) + // Except 0.12, all the following versions will fill the Hive-generated statistics + if (version == "0.12") { + assert(totalSize.isEmpty) + } else { + assert(totalSize.nonEmpty && totalSize.get > 0) + } + } + } + + test(s"$version: Delete the temporary staging directory and files after each insert") { + withTempDir { tmpDir => + withTable("tab") { + versionSpark.sql( + s""" + |CREATE TABLE tab(c1 string) + |location '${tmpDir.toURI.toString}' + """.stripMargin) + + (1 to 3).map { i => + versionSpark.sql(s"INSERT OVERWRITE TABLE tab SELECT '$i'") + } + def listFiles(path: File): List[String] = { + val dir = path.listFiles() + val folders = dir.filter(_.isDirectory).toList + val filePaths = dir.map(_.getName).toList + folders.flatMap(listFiles) ++: filePaths + } + // expect 2 files left: `.part-00000-random-uuid.crc` and `part-00000-random-uuid` + // 0.12, 0.13, 1.0 and 1.1 also has another two more files ._SUCCESS.crc and _SUCCESS + val metadataFiles = Seq("._SUCCESS.crc", "_SUCCESS") + assert(listFiles(tmpDir).filterNot(metadataFiles.contains).length == 2) + } + } + } + + test(s"$version: SPARK-13709: reading partitioned Avro table with nested schema") { + withTempDir { dir => + val path = dir.toURI.toString + val tableName = "spark_13709" + val tempTableName = "spark_13709_temp" + + new File(dir.getAbsolutePath, tableName).mkdir() + new File(dir.getAbsolutePath, tempTableName).mkdir() + + val avroSchema = + """{ + | "name": "test_record", + | "type": "record", + | "fields": [ { + | "name": "f0", + | "type": "int" + | }, { + | "name": "f1", + | "type": { + | "type": "record", + | "name": "inner", + | "fields": [ { + | "name": "f10", + | "type": "int" + | }, { + | "name": "f11", + | "type": "double" + | } ] + | } + | } ] + |} + """.stripMargin + + withTable(tableName, tempTableName) { + // Creates the external partitioned Avro table to be tested. + versionSpark.sql( + s"""CREATE EXTERNAL TABLE $tableName + |PARTITIONED BY (ds STRING) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |LOCATION '$path/$tableName' + |TBLPROPERTIES ('avro.schema.literal' = '$avroSchema') + """.stripMargin + ) + + // Creates an temporary Avro table used to prepare testing Avro file. + versionSpark.sql( + s"""CREATE EXTERNAL TABLE $tempTableName + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |LOCATION '$path/$tempTableName' + |TBLPROPERTIES ('avro.schema.literal' = '$avroSchema') + """.stripMargin + ) + + // Generates Avro data. + versionSpark.sql(s"INSERT OVERWRITE TABLE $tempTableName SELECT 1, STRUCT(2, 2.5)") + + // Adds generated Avro data as a new partition to the testing table. + versionSpark.sql( + s"ALTER TABLE $tableName ADD PARTITION (ds = 'foo') LOCATION '$path/$tempTableName'") + + // The following query fails before SPARK-13709 is fixed. This is because when reading + // data from table partitions, Avro deserializer needs the Avro schema, which is defined + // in table property "avro.schema.literal". However, we only initializes the deserializer + // using partition properties, which doesn't include the wanted property entry. Merging + // two sets of properties solves the problem. + assert(versionSpark.sql(s"SELECT * FROM $tableName").collect() === + Array(Row(1, Row(2, 2.5D), "foo"))) + } + } + } + + test(s"$version: CTAS for managed data source tables") { + withTable("t", "t1") { + versionSpark.range(1).write.saveAsTable("t") + assert(versionSpark.table("t").collect() === Array(Row(0))) + versionSpark.sql("create table t1 using parquet as select 2 as a") + assert(versionSpark.table("t1").collect() === Array(Row(2))) + } } + + test(s"$version: Decimal support of Avro Hive serde") { + val tableName = "tab1" + // TODO: add the other logical types. For details, see the link: + // https://avro.apache.org/docs/1.8.1/spec.html#Logical+Types + val avroSchema = + """{ + | "name": "test_record", + | "type": "record", + | "fields": [ { + | "name": "f0", + | "type": [ + | "null", + | { + | "precision": 38, + | "scale": 2, + | "type": "bytes", + | "logicalType": "decimal" + | } + | ] + | } ] + |} + """.stripMargin + + Seq(true, false).foreach { isPartitioned => + withTable(tableName) { + val partitionClause = if (isPartitioned) "PARTITIONED BY (ds STRING)" else "" + // Creates the (non-)partitioned Avro table + versionSpark.sql( + s""" + |CREATE TABLE $tableName + |$partitionClause + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |TBLPROPERTIES ('avro.schema.literal' = '$avroSchema') + """.stripMargin + ) + + val errorMsg = "data type mismatch: cannot cast DecimalType(2,1) to BinaryType" + + if (isPartitioned) { + val insertStmt = s"INSERT OVERWRITE TABLE $tableName partition (ds='a') SELECT 1.3" + if (version == "0.12" || version == "0.13") { + val e = intercept[AnalysisException](versionSpark.sql(insertStmt)).getMessage + assert(e.contains(errorMsg)) + } else { + versionSpark.sql(insertStmt) + assert(versionSpark.table(tableName).collect() === + versionSpark.sql("SELECT 1.30, 'a'").collect()) + } + } else { + val insertStmt = s"INSERT OVERWRITE TABLE $tableName SELECT 1.3" + if (version == "0.12" || version == "0.13") { + val e = intercept[AnalysisException](versionSpark.sql(insertStmt)).getMessage + assert(e.contains(errorMsg)) + } else { + versionSpark.sql(insertStmt) + assert(versionSpark.table(tableName).collect() === + versionSpark.sql("SELECT 1.30").collect()) + } + } + } + } + } + + test(s"$version: read avro file containing decimal") { + val url = Thread.currentThread().getContextClassLoader.getResource("avroDecimal") + val location = new File(url.getFile) + + val tableName = "tab1" + val avroSchema = + """{ + | "name": "test_record", + | "type": "record", + | "fields": [ { + | "name": "f0", + | "type": [ + | "null", + | { + | "precision": 38, + | "scale": 2, + | "type": "bytes", + | "logicalType": "decimal" + | } + | ] + | } ] + |} + """.stripMargin + withTable(tableName) { + versionSpark.sql( + s""" + |CREATE TABLE $tableName + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |WITH SERDEPROPERTIES ('respectSparkSchema' = 'true') + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat' + |LOCATION '$location' + |TBLPROPERTIES ('avro.schema.literal' = '$avroSchema') + """.stripMargin + ) + assert(versionSpark.table(tableName).collect() === + versionSpark.sql("SELECT 1.30").collect()) + } + } + + // TODO: add more tests. } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index ea80060e370e..f245a79f805a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -18,17 +18,21 @@ package org.apache.spark.sql.hive.execution import scala.collection.JavaConverters._ +import scala.util.Random + +import test.org.apache.spark.sql.MyDoubleAvg +import test.org.apache.spark.sql.MyDoubleSum -import org.apache.spark.SparkException import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction} import org.apache.spark.sql.functions._ -import org.apache.spark.sql.hive.aggregate.{MyDoubleAvg, MyDoubleSum} import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ + class ScalaAggregateFunction(schema: StructType) extends UserDefinedAggregateFunction { def inputSchema: StructType = schema @@ -66,14 +70,68 @@ class ScalaAggregateFunction(schema: StructType) extends UserDefinedAggregateFun } } +class ScalaAggregateFunctionWithoutInputSchema extends UserDefinedAggregateFunction { + + def inputSchema: StructType = StructType(Nil) + + def bufferSchema: StructType = StructType(StructField("value", LongType) :: Nil) + + def dataType: DataType = LongType + + def deterministic: Boolean = true + + def initialize(buffer: MutableAggregationBuffer): Unit = { + buffer.update(0, 0L) + } + + def update(buffer: MutableAggregationBuffer, input: Row): Unit = { + buffer.update(0, input.getAs[Seq[Row]](0).map(_.getAs[Int]("v")).sum + buffer.getLong(0)) + } + + def merge(buffer1: MutableAggregationBuffer, buffer2: Row): Unit = { + buffer1.update(0, buffer1.getLong(0) + buffer2.getLong(0)) + } + + def evaluate(buffer: Row): Any = { + buffer.getLong(0) + } +} + +class LongProductSum extends UserDefinedAggregateFunction { + def inputSchema: StructType = new StructType() + .add("a", LongType) + .add("b", LongType) + + def bufferSchema: StructType = new StructType() + .add("product", LongType) + + def dataType: DataType = LongType + + def deterministic: Boolean = true + + def initialize(buffer: MutableAggregationBuffer): Unit = { + buffer(0) = 0L + } + + def update(buffer: MutableAggregationBuffer, input: Row): Unit = { + if (!(input.isNullAt(0) || input.isNullAt(1))) { + buffer(0) = buffer.getLong(0) + input.getLong(0) * input.getLong(1) + } + } + + def merge(buffer1: MutableAggregationBuffer, buffer2: Row): Unit = { + buffer1(0) = buffer1.getLong(0) + buffer2.getLong(0) + } + + def evaluate(buffer: Row): Any = + buffer.getLong(0) +} + abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import testImplicits._ - var originalUseAggregate2: Boolean = _ - override def beforeAll(): Unit = { - originalUseAggregate2 = sqlContext.conf.useSqlAggregate2 - sqlContext.setConf(SQLConf.USE_SQL_AGGREGATE2.key, "true") + super.beforeAll() val data1 = Seq[(Integer, Integer)]( (1, 10), (null, -60), @@ -106,27 +164,56 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te (3, null, null)).toDF("key", "value1", "value2") data2.write.saveAsTable("agg2") - val emptyDF = sqlContext.createDataFrame( + val data3 = Seq[(Seq[Integer], Integer, Integer)]( + (Seq[Integer](1, 1), 10, -10), + (Seq[Integer](null), -60, 60), + (Seq[Integer](1, 1), 30, -30), + (Seq[Integer](1), 30, 30), + (Seq[Integer](2), 1, 1), + (null, -10, 10), + (Seq[Integer](2, 3), -1, null), + (Seq[Integer](2, 3), 1, 1), + (Seq[Integer](2, 3, 4), null, 1), + (Seq[Integer](null), 100, -10), + (Seq[Integer](3), null, 3), + (null, null, null), + (Seq[Integer](3), null, null)).toDF("key", "value1", "value2") + data3.write.saveAsTable("agg3") + + val emptyDF = spark.createDataFrame( sparkContext.emptyRDD[Row], StructType(StructField("key", StringType) :: StructField("value", IntegerType) :: Nil)) - emptyDF.registerTempTable("emptyTable") + emptyDF.createOrReplaceTempView("emptyTable") // Register UDAFs - sqlContext.udf.register("mydoublesum", new MyDoubleSum) - sqlContext.udf.register("mydoubleavg", new MyDoubleAvg) + spark.udf.register("mydoublesum", new MyDoubleSum) + spark.udf.register("mydoubleavg", new MyDoubleAvg) + spark.udf.register("longProductSum", new LongProductSum) } override def afterAll(): Unit = { - sqlContext.sql("DROP TABLE IF EXISTS agg1") - sqlContext.sql("DROP TABLE IF EXISTS agg2") - sqlContext.dropTempTable("emptyTable") - sqlContext.setConf(SQLConf.USE_SQL_AGGREGATE2.key, originalUseAggregate2.toString) + try { + spark.sql("DROP TABLE IF EXISTS agg1") + spark.sql("DROP TABLE IF EXISTS agg2") + spark.sql("DROP TABLE IF EXISTS agg3") + spark.catalog.dropTempView("emptyTable") + } finally { + super.afterAll() + } + } + + test("group by function") { + Seq((1, 2)).toDF("a", "b").createOrReplaceTempView("data") + + checkAnswer( + sql("SELECT floor(a) AS a, collect_set(b) FROM data GROUP BY floor(a) ORDER BY a"), + Row(1, Array(2)) :: Nil) } test("empty table") { // If there is no GROUP BY clause and the table is empty, we will generate a single row. checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT | AVG(value), @@ -143,7 +230,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(null, 0, 0, 0, null, null, null, null, null) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT | AVG(value), @@ -162,7 +249,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te // If there is a GROUP BY clause and the table is empty, there is no output. checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT | AVG(value), @@ -182,7 +269,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te test("null literal") { checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT | AVG(null), @@ -198,7 +285,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te test("only do grouping") { checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT key |FROM agg1 @@ -207,7 +294,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(1) :: Row(2) :: Row(3) :: Row(null) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT DISTINCT value1, key |FROM agg2 @@ -224,7 +311,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(null, null) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT value1, key |FROM agg2 @@ -240,11 +327,46 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(100, null) :: Row(null, 3) :: Row(null, null) :: Nil) + + checkAnswer( + spark.sql( + """ + |SELECT DISTINCT key + |FROM agg3 + """.stripMargin), + Row(Seq[Integer](1, 1)) :: + Row(Seq[Integer](null)) :: + Row(Seq[Integer](1)) :: + Row(Seq[Integer](2)) :: + Row(null) :: + Row(Seq[Integer](2, 3)) :: + Row(Seq[Integer](2, 3, 4)) :: + Row(Seq[Integer](3)) :: Nil) + + checkAnswer( + spark.sql( + """ + |SELECT value1, key + |FROM agg3 + |GROUP BY value1, key + """.stripMargin), + Row(10, Seq[Integer](1, 1)) :: + Row(-60, Seq[Integer](null)) :: + Row(30, Seq[Integer](1, 1)) :: + Row(30, Seq[Integer](1)) :: + Row(1, Seq[Integer](2)) :: + Row(-10, null) :: + Row(-1, Seq[Integer](2, 3)) :: + Row(1, Seq[Integer](2, 3)) :: + Row(null, Seq[Integer](2, 3, 4)) :: + Row(100, Seq[Integer](null)) :: + Row(null, Seq[Integer](3)) :: + Row(null, null) :: Nil) } test("case in-sensitive resolution") { checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT avg(value), kEY - 100 |FROM agg1 @@ -253,7 +375,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(20.0, -99) :: Row(-0.5, -98) :: Row(null, -97) :: Row(10.0, null) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT sum(distinct value1), kEY - 100, count(distinct value1) |FROM agg2 @@ -262,7 +384,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(40, -99, 2) :: Row(0, -98, 2) :: Row(null, -97, 0) :: Row(30, null, 3) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT valUe * key - 100 |FROM agg1 @@ -278,7 +400,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te test("test average no key in output") { checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT avg(value) |FROM agg1 @@ -289,7 +411,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te test("test average") { checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT key, avg(value) |FROM agg1 @@ -298,7 +420,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(1, 20.0) :: Row(2, -0.5) :: Row(3, null) :: Row(null, 10.0) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT key, mean(value) |FROM agg1 @@ -307,7 +429,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(1, 20.0) :: Row(2, -0.5) :: Row(3, null) :: Row(null, 10.0) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT avg(value), key |FROM agg1 @@ -316,7 +438,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(20.0, 1) :: Row(-0.5, 2) :: Row(null, 3) :: Row(10.0, null) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT avg(value) + 1.5, key + 10 |FROM agg1 @@ -325,7 +447,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(21.5, 11) :: Row(1.0, 12) :: Row(null, 13) :: Row(11.5, null) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT avg(value) FROM agg1 """.stripMargin), @@ -337,7 +459,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te // deterministic. withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT | first_valUE(key), @@ -353,7 +475,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(null, 3, null, 3, 1, 3, 1, 3) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT | first_valUE(key), @@ -372,7 +494,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te test("udaf") { checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT | key, @@ -390,9 +512,22 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(null, null, 110.0, null, null, 10.0) :: Nil) } + test("non-deterministic children expressions of UDAF") { + val e = intercept[AnalysisException] { + spark.sql( + """ + |SELECT mydoublesum(value + 1.5 * key + rand()) + |FROM agg1 + |GROUP BY key + """.stripMargin) + }.getMessage + assert(Seq("nondeterministic expression", + "should not appear in the arguments of an aggregate function").forall(e.contains)) + } + test("interpreted aggregate function") { checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT mydoublesum(value), key |FROM agg1 @@ -401,14 +536,14 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(60.0, 1) :: Row(-1.0, 2) :: Row(null, 3) :: Row(30.0, null) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT mydoublesum(value) FROM agg1 """.stripMargin), Row(89.0) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT mydoublesum(null) """.stripMargin), @@ -417,7 +552,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te test("interpreted and expression-based aggregation functions") { checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT mydoublesum(value), key, avg(value) |FROM agg1 @@ -429,7 +564,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(30.0, null, 10.0) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT | mydoublesum(value + 1.5 * key), @@ -449,7 +584,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te test("single distinct column set") { // DISTINCT is not meaningful with Max and Min, so we just ignore the DISTINCT keyword. checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT | min(distinct value1), @@ -462,7 +597,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(-60, 70.0, 101.0/9.0, 5.6, 100)) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT | mydoubleavg(distinct value1), @@ -481,7 +616,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(110.0, 10.0, 20.0, null, 109.0, 11.0, 30.0) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT | key, @@ -499,7 +634,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(null, 110.0, 60.0, 30.0, 110.0, 110.0) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT | count(value1), @@ -516,9 +651,53 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(3, 4, 4, 3, null) :: Nil) } + test("single distinct multiple columns set") { + checkAnswer( + spark.sql( + """ + |SELECT + | key, + | count(distinct value1, value2) + |FROM agg2 + |GROUP BY key + """.stripMargin), + Row(null, 3) :: + Row(1, 3) :: + Row(2, 1) :: + Row(3, 0) :: Nil) + } + + test("multiple distinct multiple columns sets") { + checkAnswer( + spark.sql( + """ + |SELECT + | key, + | count(distinct value1), + | sum(distinct value1), + | count(distinct value2), + | sum(distinct value2), + | count(distinct value1, value2), + | longProductSum(distinct value1, value2), + | count(value1), + | sum(value1), + | count(value2), + | sum(value2), + | longProductSum(value1, value2), + | count(*), + | count(1) + |FROM agg2 + |GROUP BY key + """.stripMargin), + Row(null, 3, 30, 3, 60, 3, -4700, 3, 30, 3, 60, -4700, 4, 4) :: + Row(1, 2, 40, 3, -10, 3, -100, 3, 70, 3, -10, -100, 3, 3) :: + Row(2, 2, 0, 1, 1, 1, 1, 3, 1, 3, 3, 2, 4, 4) :: + Row(3, 0, null, 1, 3, 0, 0, 0, null, 1, 3, 0, 2, 2) :: Nil) + } + test("test count") { checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT | count(value2), @@ -541,7 +720,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te Row(0, null, 1, 1, null) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT | count(value2), @@ -620,89 +799,76 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te (5, 8, 17), (6, 2, 11)).toDF("a", "b", "c") - covar_tab.registerTempTable("covar_tab") + covar_tab.createOrReplaceTempView("covar_tab") checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT corr(b, c) FROM covar_tab WHERE a < 1 """.stripMargin), Row(null) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT corr(b, c) FROM covar_tab WHERE a < 3 """.stripMargin), Row(null) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT corr(b, c) FROM covar_tab WHERE a = 3 """.stripMargin), - Row(null) :: Nil) + Row(Double.NaN) :: Nil) checkAnswer( - sqlContext.sql( + spark.sql( """ |SELECT a, corr(b, c) FROM covar_tab GROUP BY a ORDER BY a """.stripMargin), Row(1, null) :: Row(2, null) :: - Row(3, null) :: - Row(4, null) :: - Row(5, null) :: - Row(6, null) :: Nil) + Row(3, Double.NaN) :: + Row(4, Double.NaN) :: + Row(5, Double.NaN) :: + Row(6, Double.NaN) :: Nil) - val corr7 = sqlContext.sql("SELECT corr(b, c) FROM covar_tab").collect()(0).getDouble(0) + val corr7 = spark.sql("SELECT corr(b, c) FROM covar_tab").collect()(0).getDouble(0) assert(math.abs(corr7 - 0.6633880657639323) < 1e-12) - - withSQLConf(SQLConf.USE_SQL_AGGREGATE2.key -> "false") { - val errorMessage = intercept[SparkException] { - val df = Seq.tabulate(10)(i => (1.0 * i, 2.0 * i, i * -1.0)).toDF("a", "b", "c") - val corr1 = df.repartition(2).groupBy().agg(corr("a", "b")).collect()(0).getDouble(0) - }.getMessage - assert(errorMessage.contains("java.lang.UnsupportedOperationException: " + - "Corr only supports the new AggregateExpression2")) - } - } - - test("test Last implemented based on AggregateExpression1") { - // TODO: Remove this test once we remove AggregateExpression1. - import org.apache.spark.sql.functions._ - val df = Seq((1, 1), (2, 2), (3, 3)).toDF("i", "j").repartition(1) - withSQLConf( - SQLConf.SHUFFLE_PARTITIONS.key -> "1", - SQLConf.USE_SQL_AGGREGATE2.key -> "false") { - - checkAnswer( - df.groupBy("i").agg(last("j")), - df - ) - } } - test("error handling") { - withSQLConf("spark.sql.useAggregate2" -> "false") { - val errorMessage = intercept[AnalysisException] { - sqlContext.sql( - """ - |SELECT - | key, - | sum(value + 1.5 * key), - | mydoublesum(value), - | mydoubleavg(value) - |FROM agg1 - |GROUP BY key - """.stripMargin).collect() - }.getMessage - assert(errorMessage.contains("implemented based on the new Aggregate Function interface")) - } + test("covariance: covar_pop and covar_samp") { + // non-trivial example. To reproduce in python, use: + // >>> import numpy as np + // >>> a = np.array(range(20)) + // >>> b = np.array([x * x - 2 * x + 3.5 for x in range(20)]) + // >>> np.cov(a, b, bias = 0)[0][1] + // 595.0 + // >>> np.cov(a, b, bias = 1)[0][1] + // 565.25 + val df = Seq.tabulate(20)(x => (1.0 * x, x * x - 2 * x + 3.5)).toDF("a", "b") + val cov_samp = df.groupBy().agg(covar_samp("a", "b")).collect()(0).getDouble(0) + assert(math.abs(cov_samp - 595.0) < 1e-12) + + val cov_pop = df.groupBy().agg(covar_pop("a", "b")).collect()(0).getDouble(0) + assert(math.abs(cov_pop - 565.25) < 1e-12) + + val df2 = Seq.tabulate(20)(x => (1 * x, x * x * x - 2)).toDF("a", "b") + val cov_samp2 = df2.groupBy().agg(covar_samp("a", "b")).collect()(0).getDouble(0) + assert(math.abs(cov_samp2 - 11564.0) < 1e-12) + + val cov_pop2 = df2.groupBy().agg(covar_pop("a", "b")).collect()(0).getDouble(0) + assert(math.abs(cov_pop2 - 10985.799999999999) < 1e-12) + + // one row test + val df3 = Seq.tabulate(1)(x => (1 * x, x * x * x - 2)).toDF("a", "b") + checkAnswer(df3.groupBy().agg(covar_samp("a", "b")), Row(Double.NaN)) + checkAnswer(df3.groupBy().agg(covar_pop("a", "b")), Row(0.0)) } test("no aggregation function (SPARK-11486)") { - val df = sqlContext.range(20).selectExpr("id", "repeat(id, 1) as s") + val df = spark.range(20).selectExpr("id", "repeat(id, 1) as s") .groupBy("s").count() .groupBy().count() checkAnswer(df, Row(20) :: Nil) @@ -718,11 +884,11 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), DateType, TimestampType, ArrayType(IntegerType), MapType(StringType, LongType), struct, - new MyDenseVectorUDT()) - // Right now, we will use SortBasedAggregate to handle UDAFs. - // UnsafeRow.mutableFieldTypes.asScala.toSeq will trigger SortBasedAggregate to use + new UDT.MyDenseVectorUDT()) + // Right now, we will use SortAggregate to handle UDAFs. + // UnsafeRow.mutableFieldTypes.asScala.toSeq will trigger SortAggregate to use // UnsafeRow as the aggregation buffer. While, dataTypes will trigger - // SortBasedAggregate to use a safe row as the aggregation buffer. + // SortAggregate to use a safe row as the aggregation buffer. Seq(dataTypes, UnsafeRow.mutableFieldTypes.asScala.toSeq).foreach { dataTypes => val fields = dataTypes.zipWithIndex.map { case (dataType, index) => StructField(s"col$index", dataType, nullable = true) @@ -741,7 +907,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te RandomDataGenerator.forType( dataType = schemaForGenerator, nullable = true, - seed = Some(System.nanoTime())) + new Random(System.nanoTime())) val dataGenerator = maybeDataGenerator .getOrElse(fail(s"Failed to create data generator for schema $schemaForGenerator")) @@ -756,96 +922,126 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te } // Create a DF for the schema with random data. - val rdd = sqlContext.sparkContext.parallelize(data, 1) - val df = sqlContext.createDataFrame(rdd, schema) + val rdd = spark.sparkContext.parallelize(data, 1) + val df = spark.createDataFrame(rdd, schema) val allColumns = df.schema.fields.map(f => col(f.name)) - val expectedAnaswer = + val expectedAnswer = data .find(r => r.getInt(0) == 50) .getOrElse(fail("A row with id 50 should be the expected answer.")) checkAnswer( df.groupBy().agg(udaf(allColumns: _*)), // udaf returns a Row as the output value. - Row(expectedAnaswer) + Row(expectedAnswer) ) } } -} - -class SortBasedAggregationQuerySuite extends AggregationQuerySuite { - var originalUnsafeEnabled: Boolean = _ + test("udaf without specifying inputSchema") { + withTempView("noInputSchemaUDAF") { + spark.udf.register("noInputSchema", new ScalaAggregateFunctionWithoutInputSchema) + + val data = + Row(1, Seq(Row(1), Row(2), Row(3))) :: + Row(1, Seq(Row(4), Row(5), Row(6))) :: + Row(2, Seq(Row(-10))) :: Nil + val schema = + StructType( + StructField("key", IntegerType) :: + StructField("myArray", + ArrayType(StructType(StructField("v", IntegerType) :: Nil))) :: Nil) + spark.createDataFrame( + sparkContext.parallelize(data, 2), + schema) + .createOrReplaceTempView("noInputSchemaUDAF") - override def beforeAll(): Unit = { - originalUnsafeEnabled = sqlContext.conf.unsafeEnabled - sqlContext.setConf(SQLConf.UNSAFE_ENABLED.key, "false") - super.beforeAll() - } + checkAnswer( + spark.sql( + """ + |SELECT key, noInputSchema(myArray) + |FROM noInputSchemaUDAF + |GROUP BY key + """.stripMargin), + Row(1, 21) :: Row(2, -10) :: Nil) - override def afterAll(): Unit = { - super.afterAll() - sqlContext.setConf(SQLConf.UNSAFE_ENABLED.key, originalUnsafeEnabled.toString) + checkAnswer( + spark.sql( + """ + |SELECT noInputSchema(myArray) + |FROM noInputSchemaUDAF + """.stripMargin), + Row(11) :: Nil) + } } -} - -class TungstenAggregationQuerySuite extends AggregationQuerySuite { - - var originalUnsafeEnabled: Boolean = _ - override def beforeAll(): Unit = { - originalUnsafeEnabled = sqlContext.conf.unsafeEnabled - sqlContext.setConf(SQLConf.UNSAFE_ENABLED.key, "true") - super.beforeAll() + test("SPARK-15206: single distinct aggregate function in having clause") { + checkAnswer( + sql( + """ + |select key, count(distinct value1) + |from agg2 group by key + |having count(distinct value1) > 0 + """.stripMargin), + Seq( + Row(null, 3), + Row(1, 2), + Row(2, 2) + ) + ) } - override def afterAll(): Unit = { - super.afterAll() - sqlContext.setConf(SQLConf.UNSAFE_ENABLED.key, originalUnsafeEnabled.toString) + test("SPARK-15206: multiple distinct aggregate function in having clause") { + checkAnswer( + sql( + """ + |select key, count(distinct value1), count(distinct value2) + |from agg2 group by key + |having count(distinct value1) > 0 and count(distinct value2) = 3 + """.stripMargin), + Seq( + Row(null, 3, 3), + Row(1, 2, 3) + ) + ) } } -class TungstenAggregationQueryWithControlledFallbackSuite extends AggregationQuerySuite { - var originalUnsafeEnabled: Boolean = _ +class HashAggregationQuerySuite extends AggregationQuerySuite - override def beforeAll(): Unit = { - originalUnsafeEnabled = sqlContext.conf.unsafeEnabled - sqlContext.setConf(SQLConf.UNSAFE_ENABLED.key, "true") - super.beforeAll() - } - override def afterAll(): Unit = { - super.afterAll() - sqlContext.setConf(SQLConf.UNSAFE_ENABLED.key, originalUnsafeEnabled.toString) - sqlContext.conf.unsetConf("spark.sql.TungstenAggregate.testFallbackStartsAt") - } +class HashAggregationQueryWithControlledFallbackSuite extends AggregationQuerySuite { override protected def checkAnswer(actual: => DataFrame, expectedAnswer: Seq[Row]): Unit = { - (0 to 2).foreach { fallbackStartsAt => - sqlContext.setConf( - "spark.sql.TungstenAggregate.testFallbackStartsAt", - fallbackStartsAt.toString) - - // Create a new df to make sure its physical operator picks up - // spark.sql.TungstenAggregate.testFallbackStartsAt. - // todo: remove it? - val newActual = DataFrame(sqlContext, actual.logicalPlan) - - QueryTest.checkAnswer(newActual, expectedAnswer) match { - case Some(errorMessage) => - val newErrorMessage = - s""" - |The following aggregation query failed when using TungstenAggregate with - |controlled fallback (it falls back to sort-based aggregation once it has processed - |$fallbackStartsAt input rows). The query is - |${actual.queryExecution} - | - |$errorMessage - """.stripMargin - - fail(newErrorMessage) - case None => + Seq("true", "false").foreach { enableTwoLevelMaps => + withSQLConf("spark.sql.codegen.aggregate.map.twolevel.enable" -> + enableTwoLevelMaps) { + (1 to 3).foreach { fallbackStartsAt => + withSQLConf("spark.sql.TungstenAggregate.testFallbackStartsAt" -> + s"${(fallbackStartsAt - 1).toString}, ${fallbackStartsAt.toString}") { + // Create a new df to make sure its physical operator picks up + // spark.sql.TungstenAggregate.testFallbackStartsAt. + // todo: remove it? + val newActual = Dataset.ofRows(spark, actual.logicalPlan) + + QueryTest.checkAnswer(newActual, expectedAnswer) match { + case Some(errorMessage) => + val newErrorMessage = + s""" + |The following aggregation query failed when using HashAggregate with + |controlled fallback (it falls back to bytes to bytes map once it has processed + |${fallbackStartsAt - 1} input rows and to sort-based aggregation once it has + |processed $fallbackStartsAt input rows). The query is ${actual.queryExecution} + | + |$errorMessage + """.stripMargin + + fail(newErrorMessage) + case None => // Success + } + } + } } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala index a3f5921a0cb2..c58a66418991 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -19,15 +19,15 @@ package org.apache.spark.sql.hive.execution import java.io.File -import org.apache.spark.sql.hive.test.TestHive._ /** * A set of test cases based on the big-data-benchmark. * https://amplab.cs.berkeley.edu/benchmark/ */ class BigDataBenchmarkSuite extends HiveComparisonTest { - val testDataDirectory = new File("target" + File.separator + "big-data-benchmark-testdata") + import org.apache.spark.sql.hive.test.TestHive.sparkSession._ + val testDataDirectory = new File("target" + File.separator + "big-data-benchmark-testdata") val userVisitPath = new File(testDataDirectory, "uservisits").getCanonicalPath val testTables = Seq( TestTable( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala index e38d1eb5779f..07d8c5bacb1a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.hive.execution +import org.scalatest.BeforeAndAfterAll + import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.sql.hive.test.TestHiveContext -import org.scalatest.BeforeAndAfterAll class ConcurrentHiveSuite extends SparkFunSuite with BeforeAndAfterAll { ignore("multiple instances not supported") { @@ -29,9 +30,9 @@ class ConcurrentHiveSuite extends SparkFunSuite with BeforeAndAfterAll { conf.set("spark.ui.enabled", "false") val ts = new TestHiveContext(new SparkContext("local", s"TestSQLContext$i", conf)) - ts.executeSql("SHOW TABLES").toRdd.collect() - ts.executeSql("SELECT * FROM src").toRdd.collect() - ts.executeSql("SHOW TABLES").toRdd.collect() + ts.sparkSession.sql("SHOW TABLES").collect() + ts.sparkSession.sql("SELECT * FROM src").collect() + ts.sparkSession.sql("SHOW TABLES").collect() } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala new file mode 100644 index 000000000000..6937e97a47dc --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -0,0 +1,442 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import java.io.File + +import com.google.common.io.Files + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.StructType + +class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + import testImplicits._ + + protected override def beforeAll(): Unit = { + super.beforeAll() + + // Use catalog to create table instead of SQL string here, because we don't support specifying + // table properties for data source table with SQL API now. + hiveContext.sessionState.catalog.createTable( + CatalogTable( + identifier = TableIdentifier("parquet_tab1"), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType().add("c1", "int").add("c2", "string"), + provider = Some("parquet"), + properties = Map("my_key1" -> "v1") + ), + ignoreIfExists = false + ) + + sql( + """ + |CREATE TABLE parquet_tab2 (c1 INT, c2 STRING) + |STORED AS PARQUET + |TBLPROPERTIES('prop1Key'="prop1Val", '`prop2Key`'="prop2Val") + """.stripMargin) + sql("CREATE TABLE parquet_tab3(col1 int, `col 2` int)") + sql("CREATE TABLE parquet_tab4 (price int, qty int) partitioned by (year int, month int)") + sql("INSERT INTO parquet_tab4 PARTITION(year = 2015, month = 1) SELECT 1, 1") + sql("INSERT INTO parquet_tab4 PARTITION(year = 2015, month = 2) SELECT 2, 2") + sql("INSERT INTO parquet_tab4 PARTITION(year = 2016, month = 2) SELECT 3, 3") + sql("INSERT INTO parquet_tab4 PARTITION(year = 2016, month = 3) SELECT 3, 3") + sql( + """ + |CREATE TABLE parquet_tab5 (price int, qty int) + |PARTITIONED BY (year int, month int, hour int, minute int, sec int, extra int) + """.stripMargin) + sql( + """ + |INSERT INTO parquet_tab5 + |PARTITION(year = 2016, month = 3, hour = 10, minute = 10, sec = 10, extra = 1) SELECT 3, 3 + """.stripMargin) + sql( + """ + |INSERT INTO parquet_tab5 + |PARTITION(year = 2016, month = 4, hour = 10, minute = 10, sec = 10, extra = 1) SELECT 3, 3 + """.stripMargin) + sql("CREATE VIEW parquet_view1 as select * from parquet_tab4") + } + + override protected def afterAll(): Unit = { + try { + sql("DROP TABLE IF EXISTS parquet_tab1") + sql("DROP TABLE IF EXISTS parquet_tab2") + sql("DROP TABLE IF EXISTS parquet_tab3") + sql("DROP VIEW IF EXISTS parquet_view1") + sql("DROP TABLE IF EXISTS parquet_tab4") + sql("DROP TABLE IF EXISTS parquet_tab5") + } finally { + super.afterAll() + } + } + + test("show tables") { + withTable("show1a", "show2b") { + sql("CREATE TABLE show1a(c1 int)") + sql("CREATE TABLE show2b(c2 int)") + checkAnswer( + sql("SHOW TABLES IN default 'show1*'"), + Row("default", "show1a", false) :: Nil) + checkAnswer( + sql("SHOW TABLES IN default 'show1*|show2*'"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: Nil) + checkAnswer( + sql("SHOW TABLES 'show1*|show2*'"), + Row("default", "show1a", false) :: + Row("default", "show2b", false) :: Nil) + assert( + sql("SHOW TABLES").count() >= 2) + assert( + sql("SHOW TABLES IN default").count() >= 2) + } + } + + test("show tblproperties of data source tables - basic") { + checkAnswer( + sql("SHOW TBLPROPERTIES parquet_tab1").filter(s"key = 'my_key1'"), + Row("my_key1", "v1") :: Nil + ) + + checkAnswer( + sql(s"SHOW TBLPROPERTIES parquet_tab1('my_key1')"), + Row("v1") :: Nil + ) + } + + test("show tblproperties for datasource table - errors") { + val message1 = intercept[NoSuchTableException] { + sql("SHOW TBLPROPERTIES badtable") + }.getMessage + assert(message1.contains("Table or view 'badtable' not found in database 'default'")) + + // When key is not found, a row containing the error is returned. + checkAnswer( + sql("SHOW TBLPROPERTIES parquet_tab1('invalid.prop.key')"), + Row("Table default.parquet_tab1 does not have property: invalid.prop.key") :: Nil + ) + } + + test("show tblproperties for hive table") { + checkAnswer(sql("SHOW TBLPROPERTIES parquet_tab2('prop1Key')"), Row("prop1Val")) + checkAnswer(sql("SHOW TBLPROPERTIES parquet_tab2('`prop2Key`')"), Row("prop2Val")) + } + + test("show tblproperties for spark temporary table - empty row") { + withTempView("parquet_temp") { + sql( + """ + |CREATE TEMPORARY VIEW parquet_temp (c1 INT, c2 STRING) + |USING org.apache.spark.sql.parquet.DefaultSource + """.stripMargin) + + // An empty sequence of row is returned for session temporary table. + checkAnswer(sql("SHOW TBLPROPERTIES parquet_temp"), Nil) + } + } + + Seq(true, false).foreach { local => + val loadQuery = if (local) "LOAD DATA LOCAL" else "LOAD DATA" + test(loadQuery) { + testLoadData(loadQuery, local) + } + } + + private def testLoadData(loadQuery: String, local: Boolean): Unit = { + // employee.dat has two columns separated by '|', the first is an int, the second is a string. + // Its content looks like: + // 16|john + // 17|robert + val testData = hiveContext.getHiveFile("data/files/employee.dat").getCanonicalFile() + + /** + * Run a function with a copy of the input data file when running with non-local input. The + * semantics in this mode are that the input file is moved to the destination, so we have + * to make a copy so that subsequent tests have access to the original file. + */ + def withInputFile(fn: File => Unit): Unit = { + if (local) { + fn(testData) + } else { + val tmp = File.createTempFile(testData.getName(), ".tmp") + Files.copy(testData, tmp) + try { + fn(tmp) + } finally { + tmp.delete() + } + } + } + + withTable("non_part_table", "part_table") { + sql( + """ + |CREATE TABLE non_part_table (employeeID INT, employeeName STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + // LOAD DATA INTO non-partitioned table can't specify partition + intercept[AnalysisException] { + sql( + s"""$loadQuery INPATH "${testData.toURI}" INTO TABLE non_part_table PARTITION(ds="1")""") + } + + withInputFile { path => + sql(s"""$loadQuery INPATH "${path.toURI}" INTO TABLE non_part_table""") + + // Non-local mode is expected to move the file, while local mode is expected to copy it. + // Check once here that the behavior is the expected. + assert(local === path.exists()) + } + + checkAnswer( + sql("SELECT * FROM non_part_table WHERE employeeID = 16"), + Row(16, "john") :: Nil) + + // Incorrect URI. + // file://path/to/data/files/employee.dat + // + // TODO: need a similar test for non-local mode. + if (local) { + val incorrectUri = "file://path/to/data/files/employee.dat" + intercept[AnalysisException] { + sql(s"""LOAD DATA LOCAL INPATH "$incorrectUri" INTO TABLE non_part_table""") + } + } + + // Use URI as inpath: + // file:/path/to/data/files/employee.dat + withInputFile { path => + sql(s"""$loadQuery INPATH "${path.toURI}" INTO TABLE non_part_table""") + } + + checkAnswer( + sql("SELECT * FROM non_part_table WHERE employeeID = 16"), + Row(16, "john") :: Row(16, "john") :: Nil) + + // Overwrite existing data. + withInputFile { path => + sql(s"""$loadQuery INPATH "${path.toURI}" OVERWRITE INTO TABLE non_part_table""") + } + + checkAnswer( + sql("SELECT * FROM non_part_table WHERE employeeID = 16"), + Row(16, "john") :: Nil) + + sql( + """ + |CREATE TABLE part_table (employeeID INT, employeeName STRING) + |PARTITIONED BY (c STRING, d STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + // LOAD DATA INTO partitioned table must specify partition + withInputFile { f => + val path = f.toURI + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table""") + } + + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(c="1")""") + } + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(d="1")""") + } + intercept[AnalysisException] { + sql(s"""$loadQuery INPATH "$path" INTO TABLE part_table PARTITION(c="1", k="2")""") + } + } + + withInputFile { f => + sql(s"""$loadQuery INPATH "${f.toURI}" INTO TABLE part_table PARTITION(c="1", d="2")""") + } + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '2'"), + sql("SELECT * FROM non_part_table").collect()) + + // Different order of partition columns. + withInputFile { f => + sql(s"""$loadQuery INPATH "${f.toURI}" INTO TABLE part_table PARTITION(d="1", c="2")""") + } + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '2' AND d = '1'"), + sql("SELECT * FROM non_part_table").collect()) + } + } + + test("Truncate Table") { + withTable("non_part_table", "part_table") { + sql( + """ + |CREATE TABLE non_part_table (employeeID INT, employeeName STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + val testData = hiveContext.getHiveFile("data/files/employee.dat").toURI + + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE non_part_table""") + checkAnswer( + sql("SELECT * FROM non_part_table WHERE employeeID = 16"), + Row(16, "john") :: Nil) + + val testResults = sql("SELECT * FROM non_part_table").collect() + + sql("TRUNCATE TABLE non_part_table") + checkAnswer(sql("SELECT * FROM non_part_table"), Seq.empty[Row]) + + sql( + """ + |CREATE TABLE part_table (employeeID INT, employeeName STRING) + |PARTITIONED BY (c STRING, d STRING) + |ROW FORMAT DELIMITED + |FIELDS TERMINATED BY '|' + |LINES TERMINATED BY '\n' + """.stripMargin) + + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE part_table PARTITION(c="1", d="1")""") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '1'"), + testResults) + + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE part_table PARTITION(c="1", d="2")""") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '2'"), + testResults) + + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE part_table PARTITION(c="2", d="2")""") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '2' AND d = '2'"), + testResults) + + sql("TRUNCATE TABLE part_table PARTITION(c='1', d='1')") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '1'"), + Seq.empty[Row]) + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1' AND d = '2'"), + testResults) + + sql("TRUNCATE TABLE part_table PARTITION(c='1')") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table WHERE c = '1'"), + Seq.empty[Row]) + + sql("TRUNCATE TABLE part_table") + checkAnswer( + sql("SELECT employeeID, employeeName FROM part_table"), + Seq.empty[Row]) + } + } + + + test("show partitions - show everything") { + checkAnswer( + sql("show partitions parquet_tab4"), + Row("year=2015/month=1") :: + Row("year=2015/month=2") :: + Row("year=2016/month=2") :: + Row("year=2016/month=3") :: Nil) + + checkAnswer( + sql("show partitions default.parquet_tab4"), + Row("year=2015/month=1") :: + Row("year=2015/month=2") :: + Row("year=2016/month=2") :: + Row("year=2016/month=3") :: Nil) + } + + test("show partitions - show everything more than 5 part keys") { + checkAnswer( + sql("show partitions parquet_tab5"), + Row("year=2016/month=3/hour=10/minute=10/sec=10/extra=1") :: + Row("year=2016/month=4/hour=10/minute=10/sec=10/extra=1") :: Nil) + } + + test("show partitions - filter") { + checkAnswer( + sql("show partitions default.parquet_tab4 PARTITION(year=2015)"), + Row("year=2015/month=1") :: + Row("year=2015/month=2") :: Nil) + + checkAnswer( + sql("show partitions default.parquet_tab4 PARTITION(year=2015, month=1)"), + Row("year=2015/month=1") :: Nil) + + checkAnswer( + sql("show partitions default.parquet_tab4 PARTITION(month=2)"), + Row("year=2015/month=2") :: + Row("year=2016/month=2") :: Nil) + } + + test("show partitions - empty row") { + withTempView("parquet_temp") { + sql( + """ + |CREATE TEMPORARY VIEW parquet_temp (c1 INT, c2 STRING) + |USING org.apache.spark.sql.parquet.DefaultSource + """.stripMargin) + // An empty sequence of row is returned for session temporary table. + intercept[NoSuchTableException] { + sql("SHOW PARTITIONS parquet_temp") + } + + val message1 = intercept[AnalysisException] { + sql("SHOW PARTITIONS parquet_tab3") + }.getMessage + assert(message1.contains("not allowed on a table that is not partitioned")) + + val message2 = intercept[AnalysisException] { + sql("SHOW PARTITIONS parquet_tab4 PARTITION(abcd=2015, xyz=1)") + }.getMessage + assert(message2.contains("Non-partitioning column(s) [abcd, xyz] are specified")) + + val message3 = intercept[AnalysisException] { + sql("SHOW PARTITIONS parquet_view1") + }.getMessage + assert(message3.contains("is not allowed on a view")) + } + } + + test("show partitions - datasource") { + withTable("part_datasrc") { + val df = (1 to 3).map(i => (i, s"val_$i", i * 2)).toDF("a", "b", "c") + df.write + .partitionBy("a") + .format("parquet") + .mode(SaveMode.Overwrite) + .saveAsTable("part_datasrc") + + assert(sql("SHOW PARTITIONS part_datasrc").count() == 3) + } + } + +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index aa95ba94fa87..cee82cda4628 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -18,24 +18,28 @@ package org.apache.spark.sql.hive.execution import java.io._ +import java.nio.charset.StandardCharsets +import java.util +import java.util.Locale import scala.util.control.NonFatal import org.scalatest.{BeforeAndAfterAll, GivenWhenThen} import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.Dataset import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.{SetCommand, ExplainCommand} -import org.apache.spark.sql.execution.datasources.DescribeCommand -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} /** * Allows the creations of tests that execute the same query against both hive * and catalyst, comparing the results. * - * The "golden" results from Hive are cached in an retrieved both from the classpath and + * The "golden" results from Hive are cached in and retrieved both from the classpath and * [[answerCache]] to speed up testing. * * See the documentation of public vals in this class for information on how test execution can be @@ -44,6 +48,17 @@ import org.apache.spark.sql.hive.test.TestHive abstract class HiveComparisonTest extends SparkFunSuite with BeforeAndAfterAll with GivenWhenThen { + /** + * Path to the test datasets. We find this by looking up "hive-test-path-helper.txt" file. + * + * Before we run the query in Spark, we replace "../../data" with this path. + */ + private val testDataPath: String = { + Thread.currentThread.getContextClassLoader + .getResource("hive-test-path-helper.txt") + .getPath.replace("/hive-test-path-helper.txt", "/data") + } + /** * When set, any cache files that result in test failures will be deleted. Used when the test * harness or hive have been updated thus requiring new golden answers to be computed for some @@ -126,12 +141,20 @@ abstract class HiveComparisonTest protected val cacheDigest = java.security.MessageDigest.getInstance("MD5") protected def getMd5(str: String): String = { val digest = java.security.MessageDigest.getInstance("MD5") - digest.update(str.replaceAll(System.lineSeparator(), "\n").getBytes("utf-8")) + digest.update(str.replaceAll(System.lineSeparator(), "\n").getBytes(StandardCharsets.UTF_8)) new java.math.BigInteger(1, digest.digest).toString(16) } + override protected def afterAll(): Unit = { + try { + TestHive.reset() + } finally { + super.afterAll() + } + } + protected def prepareAnswer( - hiveQuery: TestHive.type#QueryExecution, + hiveQuery: TestHiveQueryExecution, answer: Seq[String]): Seq[String] = { def isSorted(plan: LogicalPlan): Boolean = plan match { @@ -145,16 +168,8 @@ abstract class HiveComparisonTest // Hack: Hive simply prints the result of a SET command to screen, // and does not return it as a query answer. case _: SetCommand => Seq("0") - case HiveNativeCommand(c) if c.toLowerCase.contains("desc") => - answer - .filterNot(nonDeterministicLine) - .map(_.replaceAll("from deserializer", "")) - .map(_.replaceAll("None", "")) - .map(_.trim) - .filterNot(_ == "") - case _: HiveNativeCommand => answer.filterNot(nonDeterministicLine).filterNot(_ == "") case _: ExplainCommand => answer - case _: DescribeCommand => + case _: DescribeTableCommand | ShowColumnsCommand(_, _) => // Filter out non-deterministic lines and lines which do not have actual results but // can introduce problems because of the way Hive formats these lines. // Then, remove empty lines. Do not sort the results. @@ -178,12 +193,7 @@ abstract class HiveComparisonTest "last_modified_by", "last_modified_time", "Owner:", - "COLUMN_STATS_ACCURATE", // The following are hive specific schema parameters which we do not need to match exactly. - "numFiles", - "numRows", - "rawDataSize", - "totalSize", "totalNumberFiles", "maxFileSize", "minFileSize" @@ -194,6 +204,7 @@ abstract class HiveComparisonTest // This list contains indicators for those lines which do not have actual results and we // want to ignore. lazy val ignoredLineIndicators = Seq( + "# Detailed Table Information", "# Partition Information", "# col_name" ) @@ -209,7 +220,12 @@ abstract class HiveComparisonTest } val installHooksCommand = "(?i)SET.*hooks".r - def createQueryTest(testCaseName: String, sql: String, reset: Boolean = true) { + def createQueryTest( + testCaseName: String, + sql: String, + reset: Boolean = true, + tryWithoutResettingFirst: Boolean = false, + skip: Boolean = false) { // testCaseName must not contain ':', which is not allowed to appear in a filename of Windows assert(!testCaseName.contains(":")) @@ -238,11 +254,9 @@ abstract class HiveComparisonTest } test(testCaseName) { + assume(!skip) logDebug(s"=== HIVE TEST: $testCaseName ===") - // Clear old output for this testcase. - outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) - val sqlWithoutComment = sql.split("\n").filterNot(l => l.matches("--.*(?<=[^\\\\]);")).mkString("\n") val allQueries = @@ -269,11 +283,33 @@ abstract class HiveComparisonTest }.mkString("\n== Console version of this test ==\n", "\n", "\n") } - try { + def doTest(reset: Boolean, isSpeculative: Boolean = false): Unit = { + // Clear old output for this testcase. + outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) + if (reset) { TestHive.reset() } + // Many tests drop indexes on src and srcpart at the beginning, so we need to load those + // tables here. Since DROP INDEX DDL is just passed to Hive, it bypasses the analyzer and + // thus the tables referenced in those DDL commands cannot be extracted for use by our + // test table auto-loading mechanism. In addition, the tests which use the SHOW TABLES + // command expect these tables to exist. + val hasShowTableCommand = + queryList.exists(_.toLowerCase(Locale.ROOT).contains("show tables")) + for (table <- Seq("src", "srcpart")) { + val hasMatchingQuery = queryList.exists { query => + val normalizedQuery = query.toLowerCase(Locale.ROOT).stripSuffix(";") + normalizedQuery.endsWith(table) || + normalizedQuery.contains(s"from $table") || + normalizedQuery.contains(s"from default.$table") + } + if (hasShowTableCommand || hasMatchingQuery) { + TestHive.loadTestTable(table) + } + } + val hiveCacheFiles = queryList.zipWithIndex.map { case (queryString, i) => val cachedAnswerName = s"$testCaseName-$i-${getMd5(queryString)}" @@ -299,65 +335,24 @@ abstract class HiveComparisonTest logInfo(s"Using answer cache for test: $testCaseName") hiveCachedResults } else { - - val hiveQueries = queryList.map(new TestHive.QueryExecution(_)) - // Make sure we can at least parse everything before attempting hive execution. - // Note this must only look at the logical plan as we might not be able to analyze if - // other DDL has not been executed yet. - hiveQueries.foreach(_.logical) - val computedResults = (queryList.zipWithIndex, hiveQueries, hiveCacheFiles).zipped.map { - case ((queryString, i), hiveQuery, cachedAnswerFile) => - try { - // Hooks often break the harness and don't really affect our test anyway, don't - // even try running them. - if (installHooksCommand.findAllMatchIn(queryString).nonEmpty) { - sys.error("hive exec hooks not supported for tests.") - } - - logWarning(s"Running query ${i + 1}/${queryList.size} with hive.") - // Analyze the query with catalyst to ensure test tables are loaded. - val answer = hiveQuery.analyzed match { - case _: ExplainCommand => - // No need to execute EXPLAIN queries as we don't check the output. - Nil - case _ => TestHive.runSqlHive(queryString) - } - - // We need to add a new line to non-empty answers so we can differentiate Seq() - // from Seq(""). - stringToFile( - cachedAnswerFile, answer.mkString("\n") + (if (answer.nonEmpty) "\n" else "")) - answer - } catch { - case e: Exception => - val errorMessage = - s""" - |Failed to generate golden answer for query: - |Error: ${e.getMessage} - |${stackTraceToString(e)} - |$queryString - """.stripMargin - stringToFile( - new File(hiveFailedDirectory, testCaseName), - errorMessage + consoleTestCase) - fail(errorMessage) - } - }.toSeq - if (reset) { TestHive.reset() } - - computedResults + throw new UnsupportedOperationException( + "Cannot find result file for test case: " + testCaseName) } // Run w/ catalyst val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => - val query = new TestHive.QueryExecution(queryString) - try { (query, prepareAnswer(query, query.stringResult())) } catch { + val query = new TestHiveQueryExecution(queryString.replace("../../data", testDataPath)) + def getResult(): Seq[String] = { + SQLExecution.withNewExecutionId(query.sparkSession, query)(query.hiveResultString()) + } + try { (query, prepareAnswer(query, getResult())) } catch { case e: Throwable => val errorMessage = s""" |Failed to execute query using catalyst: |Error: ${e.getMessage} |${stackTraceToString(e)} + |$queryString |$query |== HIVE - ${hive.size} row(s) == |${hive.mkString("\n")} @@ -365,7 +360,7 @@ abstract class HiveComparisonTest stringToFile(new File(failedDirectory, testCaseName), errorMessage + consoleTestCase) fail(errorMessage) } - }.toSeq + } (queryList, hiveResults, catalystResults).zipped.foreach { case (query, hive, (hiveQuery, catalyst)) => @@ -374,8 +369,9 @@ abstract class HiveComparisonTest // We will ignore the ExplainCommand, ShowFunctions, DescribeFunction if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && - (!hiveQuery.logical.isInstanceOf[ShowFunctions]) && - (!hiveQuery.logical.isInstanceOf[DescribeFunction]) && + (!hiveQuery.logical.isInstanceOf[ShowFunctionsCommand]) && + (!hiveQuery.logical.isInstanceOf[DescribeFunctionCommand]) && + (!hiveQuery.logical.isInstanceOf[DescribeTableCommand]) && preparedHive != catalyst) { val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive @@ -391,27 +387,28 @@ abstract class HiveComparisonTest // If this query is reading other tables that were created during this test run // also print out the query plans and results for those. val computedTablesMessages: String = try { - val tablesRead = new TestHive.QueryExecution(query).executedPlan.collect { - case ts: HiveTableScan => ts.relation.tableName + val tablesRead = new TestHiveQueryExecution(query).executedPlan.collect { + case ts: HiveTableScanExec => ts.relation.tableMeta.identifier }.toSet TestHive.reset() - val executions = queryList.map(new TestHive.QueryExecution(_)) + val executions = queryList.map(new TestHiveQueryExecution(_)) executions.foreach(_.toRdd) val tablesGenerated = queryList.zip(executions).flatMap { - case (q, e) => e.executedPlan.collect { - case i: InsertIntoHiveTable if tablesRead contains i.table.tableName => + case (q, e) => e.analyzed.collect { + case i: InsertIntoHiveTable if tablesRead contains i.table.identifier => (q, e, i) } } tablesGenerated.map { case (hiveql, execution, insert) => + val rdd = Dataset.ofRows(TestHive.sparkSession, insert.query).queryExecution.toRdd s""" |=== Generated Table === |$hiveql |$execution |== Results == - |${insert.child.execute().collect().mkString("\n")} + |${rdd.collect().mkString("\n")} """.stripMargin }.mkString("\n") @@ -430,35 +427,52 @@ abstract class HiveComparisonTest """.stripMargin stringToFile(new File(wrongDirectory, testCaseName), errorMessage + consoleTestCase) - fail(errorMessage) + if (isSpeculative && !reset) { + fail("Failed on first run; retrying") + } else { + fail(errorMessage) + } } } // Touch passed file. new FileOutputStream(new File(passedDirectory, testCaseName)).close() + } + + val canSpeculativelyTryWithoutReset: Boolean = { + val excludedSubstrings = Seq( + "into table", + "create table", + "drop index" + ) + !queryList.map(_.toLowerCase(Locale.ROOT)).exists { query => + excludedSubstrings.exists(s => query.contains(s)) + } + } + + val savedSettings = new util.HashMap[String, String] + savedSettings.putAll(TestHive.conf.settings) + try { + try { + if (tryWithoutResettingFirst && canSpeculativelyTryWithoutReset) { + doTest(reset = false, isSpeculative = true) + } else { + doTest(reset) + } + } catch { + case tf: org.scalatest.exceptions.TestFailedException => + if (tryWithoutResettingFirst && canSpeculativelyTryWithoutReset) { + logWarning("Test failed without reset(); retrying with reset()") + doTest(reset = true) + } else { + throw tf + } + } } catch { case tf: org.scalatest.exceptions.TestFailedException => throw tf - case originalException: Exception => - if (System.getProperty("spark.hive.canarytest") != null) { - // When we encounter an error we check to see if the environment is still - // okay by running a simple query. If this fails then we halt testing since - // something must have gone seriously wrong. - try { - new TestHive.QueryExecution("SELECT key FROM src").stringResult() - TestHive.runSqlHive("SELECT key FROM src") - } catch { - case e: Exception => - logError(s"FATAL ERROR: Canary query threw $e This implies that the " + - "testing environment has likely been corrupted.") - // The testing setup traps exits so wait here for a long time so the developer - // can see when things started to go wrong. - Thread.sleep(1000000) - } - } - - // If the canary query didn't fail then the environment is still okay, - // so just throw the original exception. - throw originalException + } finally { + TestHive.conf.settings.clear() + TestHive.conf.settings.putAll(savedSettings) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala new file mode 100644 index 000000000000..668da5fb4732 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -0,0 +1,2012 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import java.io.File +import java.net.URI + +import scala.language.existentials + +import org.apache.hadoop.fs.Path +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.SparkException +import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.execution.command.{DDLSuite, DDLUtils} +import org.apache.spark.sql.hive.HiveExternalCatalog +import org.apache.spark.sql.hive.orc.OrcFileOperator +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + +// TODO(gatorsmile): combine HiveCatalogedDDLSuite and HiveDDLSuite +class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeAndAfterEach { + override def afterEach(): Unit = { + try { + // drop all databases, tables and functions after each test + spark.sessionState.catalog.reset() + } finally { + super.afterEach() + } + } + + protected override def generateTable( + catalog: SessionCatalog, + name: TableIdentifier, + isDataSource: Boolean): CatalogTable = { + val storage = + if (isDataSource) { + val serde = HiveSerDe.sourceToSerDe("parquet") + assert(serde.isDefined, "The default format is not Hive compatible") + CatalogStorageFormat( + locationUri = Some(catalog.defaultTablePath(name)), + inputFormat = serde.get.inputFormat, + outputFormat = serde.get.outputFormat, + serde = serde.get.serde, + compressed = false, + properties = Map("serialization.format" -> "1")) + } else { + CatalogStorageFormat( + locationUri = Some(catalog.defaultTablePath(name)), + inputFormat = Some("org.apache.hadoop.mapred.SequenceFileInputFormat"), + outputFormat = Some("org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat"), + serde = Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"), + compressed = false, + properties = Map("serialization.format" -> "1")) + } + val metadata = new MetadataBuilder() + .putString("key", "value") + .build() + CatalogTable( + identifier = name, + tableType = CatalogTableType.EXTERNAL, + storage = storage, + schema = new StructType() + .add("col1", "int", nullable = true, metadata = metadata) + .add("col2", "string") + .add("a", "int") + .add("b", "int"), + provider = if (isDataSource) Some("parquet") else Some("hive"), + partitionColumnNames = Seq("a", "b"), + createTime = 0L, + createVersion = org.apache.spark.SPARK_VERSION, + tracksPartitionsInCatalog = true) + } + + protected override def normalizeCatalogTable(table: CatalogTable): CatalogTable = { + val nondeterministicProps = Set( + "CreateTime", + "transient_lastDdlTime", + "grantTime", + "lastUpdateTime", + "last_modified_by", + "last_modified_time", + "Owner:", + "COLUMN_STATS_ACCURATE", + // The following are hive specific schema parameters which we do not need to match exactly. + "numFiles", + "numRows", + "rawDataSize", + "totalSize", + "totalNumberFiles", + "maxFileSize", + "minFileSize" + ) + + table.copy( + createTime = 0L, + lastAccessTime = 0L, + owner = "", + properties = table.properties.filterKeys(!nondeterministicProps.contains(_)), + // View texts are checked separately + viewText = None + ) + } + + test("alter table: set location") { + testSetLocation(isDatasourceTable = false) + } + + test("alter table: set properties") { + testSetProperties(isDatasourceTable = false) + } + + test("alter table: unset properties") { + testUnsetProperties(isDatasourceTable = false) + } + + test("alter table: set serde") { + testSetSerde(isDatasourceTable = false) + } + + test("alter table: set serde partition") { + testSetSerdePartition(isDatasourceTable = false) + } + + test("alter table: change column") { + testChangeColumn(isDatasourceTable = false) + } + + test("alter table: rename partition") { + testRenamePartitions(isDatasourceTable = false) + } + + test("alter table: drop partition") { + testDropPartitions(isDatasourceTable = false) + } + + test("alter table: add partition") { + testAddPartitions(isDatasourceTable = false) + } + + test("drop table") { + testDropTable(isDatasourceTable = false) + } +} + +class HiveDDLSuite + extends QueryTest with SQLTestUtils with TestHiveSingleton with BeforeAndAfterEach { + import testImplicits._ + val hiveFormats = Seq("PARQUET", "ORC", "TEXTFILE", "SEQUENCEFILE", "RCFILE", "AVRO") + + override def afterEach(): Unit = { + try { + // drop all databases, tables and functions after each test + spark.sessionState.catalog.reset() + } finally { + super.afterEach() + } + } + // check if the directory for recording the data of the table exists. + private def tableDirectoryExists( + tableIdentifier: TableIdentifier, + dbPath: Option[String] = None): Boolean = { + val expectedTablePath = + if (dbPath.isEmpty) { + hiveContext.sessionState.catalog.defaultTablePath(tableIdentifier) + } else { + new Path(new Path(dbPath.get), tableIdentifier.table).toUri + } + val filesystemPath = new Path(expectedTablePath.toString) + val fs = filesystemPath.getFileSystem(spark.sessionState.newHadoopConf()) + fs.exists(filesystemPath) + } + + test("drop tables") { + withTable("tab1") { + val tabName = "tab1" + + assert(!tableDirectoryExists(TableIdentifier(tabName))) + sql(s"CREATE TABLE $tabName(c1 int)") + + assert(tableDirectoryExists(TableIdentifier(tabName))) + sql(s"DROP TABLE $tabName") + + assert(!tableDirectoryExists(TableIdentifier(tabName))) + sql(s"DROP TABLE IF EXISTS $tabName") + sql(s"DROP VIEW IF EXISTS $tabName") + } + } + + test("create a hive table without schema") { + import testImplicits._ + withTempPath { tempDir => + withTable("tab1", "tab2") { + (("a", "b") :: Nil).toDF().write.json(tempDir.getCanonicalPath) + + var e = intercept[AnalysisException] { sql("CREATE TABLE tab1 USING hive") }.getMessage + assert(e.contains("Unable to infer the schema. The schema specification is required to " + + "create the table `default`.`tab1`")) + + e = intercept[AnalysisException] { + sql(s"CREATE TABLE tab2 location '${tempDir.getCanonicalPath}'") + }.getMessage + assert(e.contains("Unable to infer the schema. The schema specification is required to " + + "create the table `default`.`tab2`")) + } + } + } + + test("drop external tables in default database") { + withTempDir { tmpDir => + val tabName = "tab1" + withTable(tabName) { + assert(tmpDir.listFiles.isEmpty) + sql( + s""" + |create table $tabName + |stored as parquet + |location '${tmpDir.toURI}' + |as select 1, '3' + """.stripMargin) + + val hiveTable = + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName, Some("default"))) + assert(hiveTable.tableType == CatalogTableType.EXTERNAL) + + assert(tmpDir.listFiles.nonEmpty) + sql(s"DROP TABLE $tabName") + assert(tmpDir.listFiles.nonEmpty) + } + } + } + + test("drop external data source table in default database") { + withTempDir { tmpDir => + val tabName = "tab1" + withTable(tabName) { + assert(tmpDir.listFiles.isEmpty) + + withSQLConf(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> "true") { + Seq(1 -> "a").toDF("i", "j") + .write + .mode(SaveMode.Overwrite) + .format("parquet") + .option("path", tmpDir.toString) + .saveAsTable(tabName) + } + + val hiveTable = + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName, Some("default"))) + // This data source table is external table + assert(hiveTable.tableType == CatalogTableType.EXTERNAL) + + assert(tmpDir.listFiles.nonEmpty) + sql(s"DROP TABLE $tabName") + // The data are not deleted since the table type is EXTERNAL + assert(tmpDir.listFiles.nonEmpty) + } + } + } + + test("create table and view with comment") { + val catalog = spark.sessionState.catalog + val tabName = "tab1" + withTable(tabName) { + sql(s"CREATE TABLE $tabName(c1 int) COMMENT 'BLABLA'") + val viewName = "view1" + withView(viewName) { + sql(s"CREATE VIEW $viewName COMMENT 'no comment' AS SELECT * FROM $tabName") + val tableMetadata = catalog.getTableMetadata(TableIdentifier(tabName, Some("default"))) + val viewMetadata = catalog.getTableMetadata(TableIdentifier(viewName, Some("default"))) + assert(tableMetadata.comment == Option("BLABLA")) + assert(viewMetadata.comment == Option("no comment")) + // Ensure that `comment` is removed from the table property + assert(tableMetadata.properties.get("comment").isEmpty) + assert(viewMetadata.properties.get("comment").isEmpty) + } + } + } + + test("create Hive-serde table and view with unicode columns and comment") { + val catalog = spark.sessionState.catalog + val tabName = "tab1" + val viewName = "view1" + // scalastyle:off + // non ascii characters are not allowed in the source code, so we disable the scalastyle. + val colName1 = "和" + val colName2 = "尼" + val comment = "庙" + // scalastyle:on + withTable(tabName) { + sql(s""" + |CREATE TABLE $tabName(`$colName1` int COMMENT '$comment') + |COMMENT '$comment' + |PARTITIONED BY (`$colName2` int) + """.stripMargin) + sql(s"INSERT OVERWRITE TABLE $tabName partition (`$colName2`=2) SELECT 1") + withView(viewName) { + sql( + s""" + |CREATE VIEW $viewName(`$colName1` COMMENT '$comment', `$colName2`) + |COMMENT '$comment' + |AS SELECT `$colName1`, `$colName2` FROM $tabName + """.stripMargin) + val tableMetadata = catalog.getTableMetadata(TableIdentifier(tabName, Some("default"))) + val viewMetadata = catalog.getTableMetadata(TableIdentifier(viewName, Some("default"))) + assert(tableMetadata.comment == Option(comment)) + assert(viewMetadata.comment == Option(comment)) + + assert(tableMetadata.schema.fields.length == 2 && viewMetadata.schema.fields.length == 2) + val column1InTable = tableMetadata.schema.fields.head + val column1InView = viewMetadata.schema.fields.head + assert(column1InTable.name == colName1 && column1InView.name == colName1) + assert(column1InTable.getComment() == Option(comment)) + assert(column1InView.getComment() == Option(comment)) + + assert(tableMetadata.schema.fields(1).name == colName2 && + viewMetadata.schema.fields(1).name == colName2) + + checkAnswer(sql(s"SELECT `$colName1`, `$colName2` FROM $tabName"), Row(1, 2) :: Nil) + checkAnswer(sql(s"SELECT `$colName1`, `$colName2` FROM $viewName"), Row(1, 2) :: Nil) + } + } + } + + test("create table: partition column names exist in table definition") { + val e = intercept[AnalysisException] { + sql("CREATE TABLE tbl(a int) PARTITIONED BY (a string)") + } + assert(e.message == "Found duplicate column(s) in the table definition of `default`.`tbl`: `a`") + } + + test("add/drop partition with location - managed table") { + val tab = "tab_with_partitions" + withTempDir { tmpDir => + val basePath = new File(tmpDir.getCanonicalPath) + val part1Path = new File(basePath + "/part1") + val part2Path = new File(basePath + "/part2") + val dirSet = part1Path :: part2Path :: Nil + + // Before data insertion, all the directory are empty + assert(dirSet.forall(dir => dir.listFiles == null || dir.listFiles.isEmpty)) + + withTable(tab) { + sql( + s""" + |CREATE TABLE $tab (key INT, value STRING) + |PARTITIONED BY (ds STRING, hr STRING) + """.stripMargin) + sql( + s""" + |ALTER TABLE $tab ADD + |PARTITION (ds='2008-04-08', hr=11) LOCATION '${part1Path.toURI}' + |PARTITION (ds='2008-04-08', hr=12) LOCATION '${part2Path.toURI}' + """.stripMargin) + assert(dirSet.forall(dir => dir.listFiles == null || dir.listFiles.isEmpty)) + + sql(s"INSERT OVERWRITE TABLE $tab partition (ds='2008-04-08', hr=11) SELECT 1, 'a'") + sql(s"INSERT OVERWRITE TABLE $tab partition (ds='2008-04-08', hr=12) SELECT 2, 'b'") + // add partition will not delete the data + assert(dirSet.forall(dir => dir.listFiles.nonEmpty)) + checkAnswer( + spark.table(tab), + Row(1, "a", "2008-04-08", "11") :: Row(2, "b", "2008-04-08", "12") :: Nil + ) + + sql(s"ALTER TABLE $tab DROP PARTITION (ds='2008-04-08', hr=11)") + // drop partition will delete the data + assert(part1Path.listFiles == null || part1Path.listFiles.isEmpty) + assert(part2Path.listFiles.nonEmpty) + + sql(s"DROP TABLE $tab") + // drop table will delete the data of the managed table + assert(dirSet.forall(dir => dir.listFiles == null || dir.listFiles.isEmpty)) + } + } + } + + test("SPARK-19129: drop partition with a empty string will drop the whole table") { + val df = spark.createDataFrame(Seq((0, "a"), (1, "b"))).toDF("partCol1", "name") + df.write.mode("overwrite").partitionBy("partCol1").saveAsTable("partitionedTable") + val e = intercept[AnalysisException] { + spark.sql("alter table partitionedTable drop partition(partCol1='')") + }.getMessage + assert(e.contains("Partition spec is invalid. The spec ([partCol1=]) contains an empty " + + "partition column value")) + } + + test("add/drop partitions - external table") { + val catalog = spark.sessionState.catalog + withTempDir { tmpDir => + val basePath = tmpDir.getCanonicalPath + val partitionPath_1stCol_part1 = new File(basePath + "/ds=2008-04-08") + val partitionPath_1stCol_part2 = new File(basePath + "/ds=2008-04-09") + val partitionPath_part1 = new File(basePath + "/ds=2008-04-08/hr=11") + val partitionPath_part2 = new File(basePath + "/ds=2008-04-09/hr=11") + val partitionPath_part3 = new File(basePath + "/ds=2008-04-08/hr=12") + val partitionPath_part4 = new File(basePath + "/ds=2008-04-09/hr=12") + val dirSet = + tmpDir :: partitionPath_1stCol_part1 :: partitionPath_1stCol_part2 :: + partitionPath_part1 :: partitionPath_part2 :: partitionPath_part3 :: + partitionPath_part4 :: Nil + + val externalTab = "extTable_with_partitions" + withTable(externalTab) { + assert(tmpDir.listFiles.isEmpty) + sql( + s""" + |CREATE EXTERNAL TABLE $externalTab (key INT, value STRING) + |PARTITIONED BY (ds STRING, hr STRING) + |LOCATION '${tmpDir.toURI}' + """.stripMargin) + + // Before data insertion, all the directory are empty + assert(dirSet.forall(dir => dir.listFiles == null || dir.listFiles.isEmpty)) + + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { + sql( + s""" + |INSERT OVERWRITE TABLE $externalTab + |partition (ds='$ds',hr='$hr') + |SELECT 1, 'a' + """.stripMargin) + } + + val hiveTable = catalog.getTableMetadata(TableIdentifier(externalTab, Some("default"))) + assert(hiveTable.tableType == CatalogTableType.EXTERNAL) + // After data insertion, all the directory are not empty + assert(dirSet.forall(dir => dir.listFiles.nonEmpty)) + + val message = intercept[AnalysisException] { + sql(s"ALTER TABLE $externalTab DROP PARTITION (ds='2008-04-09', unknownCol='12')") + } + assert(message.getMessage.contains("unknownCol is not a valid partition column in table " + + "`default`.`exttable_with_partitions`")) + + sql( + s""" + |ALTER TABLE $externalTab DROP PARTITION (ds='2008-04-08'), + |PARTITION (hr='12') + """.stripMargin) + assert(catalog.listPartitions(TableIdentifier(externalTab)).map(_.spec).toSet == + Set(Map("ds" -> "2008-04-09", "hr" -> "11"))) + // drop partition will not delete the data of external table + assert(dirSet.forall(dir => dir.listFiles.nonEmpty)) + + sql( + s""" + |ALTER TABLE $externalTab ADD PARTITION (ds='2008-04-08', hr='12') + |PARTITION (ds='2008-04-08', hr=11) + """.stripMargin) + assert(catalog.listPartitions(TableIdentifier(externalTab)).map(_.spec).toSet == + Set(Map("ds" -> "2008-04-08", "hr" -> "11"), + Map("ds" -> "2008-04-08", "hr" -> "12"), + Map("ds" -> "2008-04-09", "hr" -> "11"))) + // add partition will not delete the data + assert(dirSet.forall(dir => dir.listFiles.nonEmpty)) + + sql(s"DROP TABLE $externalTab") + // drop table will not delete the data of external table + assert(dirSet.forall(dir => dir.listFiles.nonEmpty)) + } + } + } + + test("drop views") { + withTable("tab1") { + val tabName = "tab1" + spark.range(10).write.saveAsTable("tab1") + withView("view1") { + val viewName = "view1" + + assert(tableDirectoryExists(TableIdentifier(tabName))) + assert(!tableDirectoryExists(TableIdentifier(viewName))) + sql(s"CREATE VIEW $viewName AS SELECT * FROM tab1") + + assert(tableDirectoryExists(TableIdentifier(tabName))) + assert(!tableDirectoryExists(TableIdentifier(viewName))) + sql(s"DROP VIEW $viewName") + + assert(tableDirectoryExists(TableIdentifier(tabName))) + sql(s"DROP VIEW IF EXISTS $viewName") + } + } + } + + test("alter views - rename") { + val tabName = "tab1" + withTable(tabName) { + spark.range(10).write.saveAsTable(tabName) + val oldViewName = "view1" + val newViewName = "view2" + withView(oldViewName, newViewName) { + val catalog = spark.sessionState.catalog + sql(s"CREATE VIEW $oldViewName AS SELECT * FROM $tabName") + + assert(catalog.tableExists(TableIdentifier(oldViewName))) + assert(!catalog.tableExists(TableIdentifier(newViewName))) + sql(s"ALTER VIEW $oldViewName RENAME TO $newViewName") + assert(!catalog.tableExists(TableIdentifier(oldViewName))) + assert(catalog.tableExists(TableIdentifier(newViewName))) + } + } + } + + test("alter views - set/unset tblproperties") { + val tabName = "tab1" + withTable(tabName) { + spark.range(10).write.saveAsTable(tabName) + val viewName = "view1" + withView(viewName) { + def checkProperties(expected: Map[String, String]): Boolean = { + val properties = spark.sessionState.catalog.getTableMetadata(TableIdentifier(viewName)) + .properties + properties.filterNot { case (key, value) => + Seq("transient_lastDdlTime", CatalogTable.VIEW_DEFAULT_DATABASE).contains(key) || + key.startsWith(CatalogTable.VIEW_QUERY_OUTPUT_PREFIX) + } == expected + } + sql(s"CREATE VIEW $viewName AS SELECT * FROM $tabName") + + checkProperties(Map()) + sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')") + checkProperties(Map("p" -> "an")) + + // no exception or message will be issued if we set it again + sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')") + checkProperties(Map("p" -> "an")) + + // the value will be updated if we set the same key to a different value + sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'b')") + checkProperties(Map("p" -> "b")) + + sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") + checkProperties(Map()) + + val message = intercept[AnalysisException] { + sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") + }.getMessage + assert(message.contains( + "Attempted to unset non-existent property 'p' in table '`default`.`view1`'")) + } + } + } + + private def assertErrorForAlterTableOnView(sqlText: String): Unit = { + val message = intercept[AnalysisException](sql(sqlText)).getMessage + assert(message.contains("Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead")) + } + + private def assertErrorForAlterViewOnTable(sqlText: String): Unit = { + val message = intercept[AnalysisException](sql(sqlText)).getMessage + assert(message.contains("Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead")) + } + + test("create table - SET TBLPROPERTIES EXTERNAL to TRUE") { + val tabName = "tab1" + withTable(tabName) { + val message = intercept[AnalysisException] { + sql(s"CREATE TABLE $tabName (height INT, length INT) TBLPROPERTIES('EXTERNAL'='TRUE')") + }.getMessage + assert(message.contains("Cannot set or change the preserved property key: 'EXTERNAL'")) + } + } + + test("alter table - SET TBLPROPERTIES EXTERNAL to TRUE") { + val tabName = "tab1" + withTable(tabName) { + val catalog = spark.sessionState.catalog + sql(s"CREATE TABLE $tabName (height INT, length INT)") + assert( + catalog.getTableMetadata(TableIdentifier(tabName)).tableType == CatalogTableType.MANAGED) + val message = intercept[AnalysisException] { + sql(s"ALTER TABLE $tabName SET TBLPROPERTIES ('EXTERNAL' = 'TRUE')") + }.getMessage + assert(message.contains("Cannot set or change the preserved property key: 'EXTERNAL'")) + // The table type is not changed to external + assert( + catalog.getTableMetadata(TableIdentifier(tabName)).tableType == CatalogTableType.MANAGED) + // The table property is case sensitive. Thus, external is allowed + sql(s"ALTER TABLE $tabName SET TBLPROPERTIES ('external' = 'TRUE')") + // The table type is not changed to external + assert( + catalog.getTableMetadata(TableIdentifier(tabName)).tableType == CatalogTableType.MANAGED) + } + } + + test("alter views and alter table - misuse") { + val tabName = "tab1" + withTable(tabName) { + spark.range(10).write.saveAsTable(tabName) + val oldViewName = "view1" + val newViewName = "view2" + withView(oldViewName, newViewName) { + val catalog = spark.sessionState.catalog + sql(s"CREATE VIEW $oldViewName AS SELECT * FROM $tabName") + + assert(catalog.tableExists(TableIdentifier(tabName))) + assert(catalog.tableExists(TableIdentifier(oldViewName))) + assert(!catalog.tableExists(TableIdentifier(newViewName))) + + assertErrorForAlterViewOnTable(s"ALTER VIEW $tabName RENAME TO $newViewName") + + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName RENAME TO $newViewName") + + assertErrorForAlterViewOnTable(s"ALTER VIEW $tabName SET TBLPROPERTIES ('p' = 'an')") + + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName SET TBLPROPERTIES ('p' = 'an')") + + assertErrorForAlterViewOnTable(s"ALTER VIEW $tabName UNSET TBLPROPERTIES ('p')") + + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName UNSET TBLPROPERTIES ('p')") + + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName SET LOCATION '/path/to/home'") + + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName SET SERDE 'whatever'") + + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName SET SERDEPROPERTIES ('x' = 'y')") + + assertErrorForAlterTableOnView( + s"ALTER TABLE $oldViewName PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y')") + + assertErrorForAlterTableOnView( + s"ALTER TABLE $oldViewName ADD IF NOT EXISTS PARTITION (a='4', b='8')") + + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName DROP IF EXISTS PARTITION (a='2')") + + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName RECOVER PARTITIONS") + + assertErrorForAlterTableOnView( + s"ALTER TABLE $oldViewName PARTITION (a='1') RENAME TO PARTITION (a='100')") + + assert(catalog.tableExists(TableIdentifier(tabName))) + assert(catalog.tableExists(TableIdentifier(oldViewName))) + assert(!catalog.tableExists(TableIdentifier(newViewName))) + } + } + } + + test("alter table partition - storage information") { + sql("CREATE TABLE boxes (height INT, length INT) PARTITIONED BY (width INT)") + sql("INSERT OVERWRITE TABLE boxes PARTITION (width=4) SELECT 4, 4") + val catalog = spark.sessionState.catalog + val expectedSerde = "com.sparkbricks.serde.ColumnarSerDe" + val expectedSerdeProps = Map("compress" -> "true") + val expectedSerdePropsString = + expectedSerdeProps.map { case (k, v) => s"'$k'='$v'" }.mkString(", ") + val oldPart = catalog.getPartition(TableIdentifier("boxes"), Map("width" -> "4")) + assume(oldPart.storage.serde != Some(expectedSerde), "bad test: serde was already set") + assume(oldPart.storage.properties.filterKeys(expectedSerdeProps.contains) != + expectedSerdeProps, "bad test: serde properties were already set") + sql(s"""ALTER TABLE boxes PARTITION (width=4) + | SET SERDE '$expectedSerde' + | WITH SERDEPROPERTIES ($expectedSerdePropsString) + |""".stripMargin) + val newPart = catalog.getPartition(TableIdentifier("boxes"), Map("width" -> "4")) + assert(newPart.storage.serde == Some(expectedSerde)) + assert(newPart.storage.properties.filterKeys(expectedSerdeProps.contains) == + expectedSerdeProps) + } + + test("MSCK REPAIR RABLE") { + val catalog = spark.sessionState.catalog + val tableIdent = TableIdentifier("tab1") + sql("CREATE TABLE tab1 (height INT, length INT) PARTITIONED BY (a INT, b INT)") + val part1 = Map("a" -> "1", "b" -> "5") + val part2 = Map("a" -> "2", "b" -> "6") + val root = new Path(catalog.getTableMetadata(tableIdent).location) + val fs = root.getFileSystem(spark.sparkContext.hadoopConfiguration) + // valid + fs.mkdirs(new Path(new Path(root, "a=1"), "b=5")) + fs.createNewFile(new Path(new Path(root, "a=1/b=5"), "a.csv")) // file + fs.createNewFile(new Path(new Path(root, "a=1/b=5"), "_SUCCESS")) // file + fs.mkdirs(new Path(new Path(root, "A=2"), "B=6")) + fs.createNewFile(new Path(new Path(root, "A=2/B=6"), "b.csv")) // file + fs.createNewFile(new Path(new Path(root, "A=2/B=6"), "c.csv")) // file + fs.createNewFile(new Path(new Path(root, "A=2/B=6"), ".hiddenFile")) // file + fs.mkdirs(new Path(new Path(root, "A=2/B=6"), "_temporary")) + + // invalid + fs.mkdirs(new Path(new Path(root, "a"), "b")) // bad name + fs.mkdirs(new Path(new Path(root, "b=1"), "a=1")) // wrong order + fs.mkdirs(new Path(root, "a=4")) // not enough columns + fs.createNewFile(new Path(new Path(root, "a=1"), "b=4")) // file + fs.createNewFile(new Path(new Path(root, "a=1"), "_SUCCESS")) // _SUCCESS + fs.mkdirs(new Path(new Path(root, "a=1"), "_temporary")) // _temporary + fs.mkdirs(new Path(new Path(root, "a=1"), ".b=4")) // start with . + + try { + sql("MSCK REPAIR TABLE tab1") + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == + Set(part1, part2)) + assert(catalog.getPartition(tableIdent, part1).parameters("numFiles") == "1") + assert(catalog.getPartition(tableIdent, part2).parameters("numFiles") == "2") + } finally { + fs.delete(root, true) + } + } + + test("drop table using drop view") { + withTable("tab1") { + sql("CREATE TABLE tab1(c1 int)") + val message = intercept[AnalysisException] { + sql("DROP VIEW tab1") + }.getMessage + assert(message.contains("Cannot drop a table with DROP VIEW. Please use DROP TABLE instead")) + } + } + + test("drop view using drop table") { + withTable("tab1") { + spark.range(10).write.saveAsTable("tab1") + withView("view1") { + sql("CREATE VIEW view1 AS SELECT * FROM tab1") + val message = intercept[AnalysisException] { + sql("DROP TABLE view1") + }.getMessage + assert(message.contains("Cannot drop a view with DROP TABLE. Please use DROP VIEW instead")) + } + } + } + + test("create view with mismatched schema") { + withTable("tab1") { + spark.range(10).write.saveAsTable("tab1") + withView("view1") { + val e = intercept[AnalysisException] { + sql("CREATE VIEW view1 (col1, col3) AS SELECT * FROM tab1") + }.getMessage + assert(e.contains("the SELECT clause (num: `1`) does not match") + && e.contains("CREATE VIEW (num: `2`)")) + } + } + } + + test("create view with specified schema") { + withView("view1") { + sql("CREATE VIEW view1 (col1, col2) AS SELECT 1, 2") + checkAnswer( + sql("SELECT * FROM view1"), + Row(1, 2) :: Nil + ) + } + } + + test("desc table for Hive table - partitioned table") { + withTable("tbl") { + sql("CREATE TABLE tbl(a int) PARTITIONED BY (b int)") + + assert(sql("DESC tbl").collect().containsSlice( + Seq( + Row("a", "int", null), + Row("b", "int", null), + Row("# Partition Information", "", ""), + Row("# col_name", "data_type", "comment"), + Row("b", "int", null) + ) + )) + } + } + + test("desc table for Hive table - bucketed + sorted table") { + withTable("tbl") { + sql(s""" + CREATE TABLE tbl (id int, name string) + PARTITIONED BY (ds string) + CLUSTERED BY(id) + SORTED BY(id, name) INTO 1024 BUCKETS + """) + + val x = sql("DESC FORMATTED tbl").collect() + assert(x.containsSlice( + Seq( + Row("Num Buckets", "1024", ""), + Row("Bucket Columns", "[`id`]", ""), + Row("Sort Columns", "[`id`, `name`]", "") + ) + )) + } + } + + test("desc table for data source table using Hive Metastore") { + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive") + val tabName = "tab1" + withTable(tabName) { + sql(s"CREATE TABLE $tabName(a int comment 'test') USING parquet ") + + checkAnswer( + sql(s"DESC $tabName").select("col_name", "data_type", "comment"), + Row("a", "int", "test") :: Nil + ) + } + } + + private def createDatabaseWithLocation(tmpDir: File, dirExists: Boolean): Unit = { + val catalog = spark.sessionState.catalog + val dbName = "db1" + val tabName = "tab1" + val fs = new Path(tmpDir.toString).getFileSystem(spark.sessionState.newHadoopConf()) + withTable(tabName) { + if (dirExists) { + assert(tmpDir.listFiles.isEmpty) + } else { + assert(!fs.exists(new Path(tmpDir.toString))) + } + sql(s"CREATE DATABASE $dbName Location '${tmpDir.toURI.getPath.stripSuffix("/")}'") + val db1 = catalog.getDatabaseMetadata(dbName) + val dbPath = new URI(tmpDir.toURI.toString.stripSuffix("/")) + assert(db1 == CatalogDatabase(dbName, "", dbPath, Map.empty)) + sql("USE db1") + + sql(s"CREATE TABLE $tabName as SELECT 1") + assert(tableDirectoryExists(TableIdentifier(tabName), Option(tmpDir.toString))) + + assert(tmpDir.listFiles.nonEmpty) + sql(s"DROP TABLE $tabName") + + assert(tmpDir.listFiles.isEmpty) + sql("USE default") + sql(s"DROP DATABASE $dbName") + assert(!fs.exists(new Path(tmpDir.toString))) + } + } + + test("create/drop database - location without pre-created directory") { + withTempPath { tmpDir => + createDatabaseWithLocation(tmpDir, dirExists = false) + } + } + + test("create/drop database - location with pre-created directory") { + withTempDir { tmpDir => + createDatabaseWithLocation(tmpDir, dirExists = true) + } + } + + private def dropDatabase(cascade: Boolean, tableExists: Boolean): Unit = { + val dbName = "db1" + val dbPath = new Path(spark.sessionState.conf.warehousePath) + val fs = dbPath.getFileSystem(spark.sessionState.newHadoopConf()) + + sql(s"CREATE DATABASE $dbName") + val catalog = spark.sessionState.catalog + val expectedDBLocation = s"file:${dbPath.toUri.getPath.stripSuffix("/")}/$dbName.db" + val expectedDBUri = CatalogUtils.stringToURI(expectedDBLocation) + val db1 = catalog.getDatabaseMetadata(dbName) + assert(db1 == CatalogDatabase( + dbName, + "", + expectedDBUri, + Map.empty)) + // the database directory was created + assert(fs.exists(dbPath) && fs.isDirectory(dbPath)) + sql(s"USE $dbName") + + val tabName = "tab1" + assert(!tableDirectoryExists(TableIdentifier(tabName), Option(expectedDBLocation))) + sql(s"CREATE TABLE $tabName as SELECT 1") + assert(tableDirectoryExists(TableIdentifier(tabName), Option(expectedDBLocation))) + + if (!tableExists) { + sql(s"DROP TABLE $tabName") + assert(!tableDirectoryExists(TableIdentifier(tabName), Option(expectedDBLocation))) + } + + sql(s"USE default") + val sqlDropDatabase = s"DROP DATABASE $dbName ${if (cascade) "CASCADE" else "RESTRICT"}" + if (tableExists && !cascade) { + val message = intercept[AnalysisException] { + sql(sqlDropDatabase) + }.getMessage + assert(message.contains(s"Database $dbName is not empty. One or more tables exist.")) + // the database directory was not removed + assert(fs.exists(new Path(expectedDBLocation))) + } else { + sql(sqlDropDatabase) + // the database directory was removed and the inclusive table directories are also removed + assert(!fs.exists(new Path(expectedDBLocation))) + } + } + + test("drop database containing tables - CASCADE") { + dropDatabase(cascade = true, tableExists = true) + } + + test("drop an empty database - CASCADE") { + dropDatabase(cascade = true, tableExists = false) + } + + test("drop database containing tables - RESTRICT") { + dropDatabase(cascade = false, tableExists = true) + } + + test("drop an empty database - RESTRICT") { + dropDatabase(cascade = false, tableExists = false) + } + + test("drop default database") { + Seq("true", "false").foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { + var message = intercept[AnalysisException] { + sql("DROP DATABASE default") + }.getMessage + assert(message.contains("Can not drop default database")) + + // SQLConf.CASE_SENSITIVE does not affect the result + // because the Hive metastore is not case sensitive. + message = intercept[AnalysisException] { + sql("DROP DATABASE DeFault") + }.getMessage + assert(message.contains("Can not drop default database")) + } + } + } + + test("Create Cataloged Table As Select - Drop Table After Runtime Exception") { + withTable("tab") { + intercept[SparkException] { + sql( + """ + |CREATE TABLE tab + |STORED AS TEXTFILE + |SELECT 1 AS a, (SELECT a FROM (SELECT 1 AS a UNION ALL SELECT 2 AS a) t) AS b + """.stripMargin) + } + // After hitting runtime exception, we should drop the created table. + assert(!spark.sessionState.catalog.tableExists(TableIdentifier("tab"))) + } + } + + test("CREATE TABLE LIKE a temporary view") { + // CREATE TABLE LIKE a temporary view. + withCreateTableLikeTempView(location = None) + + // CREATE TABLE LIKE a temporary view location ... + withTempDir { tmpDir => + withCreateTableLikeTempView(Some(tmpDir.toURI.toString)) + } + } + + private def withCreateTableLikeTempView(location : Option[String]): Unit = { + val sourceViewName = "tab1" + val targetTabName = "tab2" + val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED + withTempView(sourceViewName) { + withTable(targetTabName) { + spark.range(10).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd) + .createTempView(sourceViewName) + + val locationClause = if (location.nonEmpty) s"LOCATION '${location.getOrElse("")}'" else "" + sql(s"CREATE TABLE $targetTabName LIKE $sourceViewName $locationClause") + + val sourceTable = spark.sessionState.catalog.getTempViewOrPermanentTableMetadata( + TableIdentifier(sourceViewName)) + val targetTable = spark.sessionState.catalog.getTableMetadata( + TableIdentifier(targetTabName, Some("default"))) + + checkCreateTableLike(sourceTable, targetTable, tableType) + } + } + } + + test("CREATE TABLE LIKE a data source table") { + // CREATE TABLE LIKE a data source table. + withCreateTableLikeDSTable(location = None) + + // CREATE TABLE LIKE a data source table location ... + withTempDir { tmpDir => + withCreateTableLikeDSTable(Some(tmpDir.toURI.toString)) + } + } + + private def withCreateTableLikeDSTable(location : Option[String]): Unit = { + val sourceTabName = "tab1" + val targetTabName = "tab2" + val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED + withTable(sourceTabName, targetTabName) { + spark.range(10).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd) + .write.format("json").saveAsTable(sourceTabName) + + val locationClause = if (location.nonEmpty) s"LOCATION '${location.getOrElse("")}'" else "" + sql(s"CREATE TABLE $targetTabName LIKE $sourceTabName $locationClause") + + val sourceTable = + spark.sessionState.catalog.getTableMetadata( + TableIdentifier(sourceTabName, Some("default"))) + val targetTable = + spark.sessionState.catalog.getTableMetadata( + TableIdentifier(targetTabName, Some("default"))) + // The table type of the source table should be a Hive-managed data source table + assert(DDLUtils.isDatasourceTable(sourceTable)) + assert(sourceTable.tableType == CatalogTableType.MANAGED) + + checkCreateTableLike(sourceTable, targetTable, tableType) + } + } + + test("CREATE TABLE LIKE an external data source table") { + // CREATE TABLE LIKE an external data source table. + withCreateTableLikeExtDSTable(location = None) + + // CREATE TABLE LIKE an external data source table location ... + withTempDir { tmpDir => + withCreateTableLikeExtDSTable(Some(tmpDir.toURI.toString)) + } + } + + private def withCreateTableLikeExtDSTable(location : Option[String]): Unit = { + val sourceTabName = "tab1" + val targetTabName = "tab2" + val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED + withTable(sourceTabName, targetTabName) { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.range(10).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd) + .write.format("parquet").save(path) + sql(s"CREATE TABLE $sourceTabName USING parquet OPTIONS (PATH '${dir.toURI}')") + + val locationClause = if (location.nonEmpty) s"LOCATION '${location.getOrElse("")}'" else "" + sql(s"CREATE TABLE $targetTabName LIKE $sourceTabName $locationClause") + + // The source table should be an external data source table + val sourceTable = spark.sessionState.catalog.getTableMetadata( + TableIdentifier(sourceTabName, Some("default"))) + val targetTable = spark.sessionState.catalog.getTableMetadata( + TableIdentifier(targetTabName, Some("default"))) + // The table type of the source table should be an external data source table + assert(DDLUtils.isDatasourceTable(sourceTable)) + assert(sourceTable.tableType == CatalogTableType.EXTERNAL) + + checkCreateTableLike(sourceTable, targetTable, tableType) + } + } + } + + test("CREATE TABLE LIKE a managed Hive serde table") { + // CREATE TABLE LIKE a managed Hive serde table. + withCreateTableLikeManagedHiveTable(location = None) + + // CREATE TABLE LIKE a managed Hive serde table location ... + withTempDir { tmpDir => + withCreateTableLikeManagedHiveTable(Some(tmpDir.toURI.toString)) + } + } + + private def withCreateTableLikeManagedHiveTable(location : Option[String]): Unit = { + val sourceTabName = "tab1" + val targetTabName = "tab2" + val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED + val catalog = spark.sessionState.catalog + withTable(sourceTabName, targetTabName) { + sql(s"CREATE TABLE $sourceTabName TBLPROPERTIES('prop1'='value1') AS SELECT 1 key, 'a'") + + val locationClause = if (location.nonEmpty) s"LOCATION '${location.getOrElse("")}'" else "" + sql(s"CREATE TABLE $targetTabName LIKE $sourceTabName $locationClause") + + val sourceTable = catalog.getTableMetadata( + TableIdentifier(sourceTabName, Some("default"))) + assert(sourceTable.tableType == CatalogTableType.MANAGED) + assert(sourceTable.properties.get("prop1").nonEmpty) + val targetTable = catalog.getTableMetadata( + TableIdentifier(targetTabName, Some("default"))) + + checkCreateTableLike(sourceTable, targetTable, tableType) + } + } + + test("CREATE TABLE LIKE an external Hive serde table") { + // CREATE TABLE LIKE an external Hive serde table. + withCreateTableLikeExtHiveTable(location = None) + + // CREATE TABLE LIKE an external Hive serde table location ... + withTempDir { tmpDir => + withCreateTableLikeExtHiveTable(Some(tmpDir.toURI.toString)) + } + } + + private def withCreateTableLikeExtHiveTable(location : Option[String]): Unit = { + val catalog = spark.sessionState.catalog + val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED + withTempDir { tmpDir => + val basePath = tmpDir.toURI + val sourceTabName = "tab1" + val targetTabName = "tab2" + withTable(sourceTabName, targetTabName) { + assert(tmpDir.listFiles.isEmpty) + sql( + s""" + |CREATE EXTERNAL TABLE $sourceTabName (key INT comment 'test', value STRING) + |COMMENT 'Apache Spark' + |PARTITIONED BY (ds STRING, hr STRING) + |LOCATION '$basePath' + """.stripMargin) + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { + sql( + s""" + |INSERT OVERWRITE TABLE $sourceTabName + |partition (ds='$ds',hr='$hr') + |SELECT 1, 'a' + """.stripMargin) + } + + val locationClause = if (location.nonEmpty) s"LOCATION '${location.getOrElse("")}'" else "" + sql(s"CREATE TABLE $targetTabName LIKE $sourceTabName $locationClause") + + val sourceTable = catalog.getTableMetadata( + TableIdentifier(sourceTabName, Some("default"))) + assert(sourceTable.tableType == CatalogTableType.EXTERNAL) + assert(sourceTable.comment == Option("Apache Spark")) + val targetTable = catalog.getTableMetadata( + TableIdentifier(targetTabName, Some("default"))) + + checkCreateTableLike(sourceTable, targetTable, tableType) + } + } + } + + test("CREATE TABLE LIKE a view") { + // CREATE TABLE LIKE a view. + withCreateTableLikeView(location = None) + + // CREATE TABLE LIKE a view location ... + withTempDir { tmpDir => + withCreateTableLikeView(Some(tmpDir.toURI.toString)) + } + } + + private def withCreateTableLikeView(location : Option[String]): Unit = { + val sourceTabName = "tab1" + val sourceViewName = "view" + val targetTabName = "tab2" + val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED + withTable(sourceTabName, targetTabName) { + withView(sourceViewName) { + spark.range(10).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd) + .write.format("json").saveAsTable(sourceTabName) + sql(s"CREATE VIEW $sourceViewName AS SELECT * FROM $sourceTabName") + + val locationClause = if (location.nonEmpty) s"LOCATION '${location.getOrElse("")}'" else "" + sql(s"CREATE TABLE $targetTabName LIKE $sourceViewName $locationClause") + + val sourceView = spark.sessionState.catalog.getTableMetadata( + TableIdentifier(sourceViewName, Some("default"))) + // The original source should be a VIEW with an empty path + assert(sourceView.tableType == CatalogTableType.VIEW) + assert(sourceView.viewText.nonEmpty) + assert(sourceView.viewDefaultDatabase == Some("default")) + assert(sourceView.viewQueryColumnNames == Seq("a", "b", "c", "d")) + val targetTable = spark.sessionState.catalog.getTableMetadata( + TableIdentifier(targetTabName, Some("default"))) + + checkCreateTableLike(sourceView, targetTable, tableType) + } + } + } + + private def checkCreateTableLike( + sourceTable: CatalogTable, + targetTable: CatalogTable, + tableType: CatalogTableType): Unit = { + // The created table should be a MANAGED table or EXTERNAL table with empty view text + // and original text. + assert(targetTable.tableType == tableType, + s"the created table must be a/an ${tableType.name} table") + assert(targetTable.viewText.isEmpty, + "the view text in the created table must be empty") + assert(targetTable.viewDefaultDatabase.isEmpty, + "the view default database in the created table must be empty") + assert(targetTable.viewQueryColumnNames.isEmpty, + "the view query output columns in the created table must be empty") + assert(targetTable.comment.isEmpty, + "the comment in the created table must be empty") + assert(targetTable.unsupportedFeatures.isEmpty, + "the unsupportedFeatures in the create table must be empty") + + val metastoreGeneratedProperties = Seq( + "CreateTime", + "transient_lastDdlTime", + "grantTime", + "lastUpdateTime", + "last_modified_by", + "last_modified_time", + "Owner:", + "totalNumberFiles", + "maxFileSize", + "minFileSize" + ) + assert(targetTable.properties.filterKeys(!metastoreGeneratedProperties.contains(_)).isEmpty, + "the table properties of source tables should not be copied in the created table") + + if (DDLUtils.isDatasourceTable(sourceTable) || + sourceTable.tableType == CatalogTableType.VIEW) { + assert(DDLUtils.isDatasourceTable(targetTable), + "the target table should be a data source table") + } else { + assert(!DDLUtils.isDatasourceTable(targetTable), + "the target table should be a Hive serde table") + } + + if (sourceTable.tableType == CatalogTableType.VIEW) { + // Source table is a temporary/permanent view, which does not have a provider. The created + // target table uses the default data source format + assert(targetTable.provider == Option(spark.sessionState.conf.defaultDataSourceName)) + } else { + assert(targetTable.provider == sourceTable.provider) + } + + assert(targetTable.storage.locationUri.nonEmpty, "target table path should not be empty") + + // User-specified location and sourceTable's location can be same or different, + // when we creating an external table. So we don't need to do this check + if (tableType != CatalogTableType.EXTERNAL) { + assert(sourceTable.storage.locationUri != targetTable.storage.locationUri, + "source table/view path should be different from target table path") + } + + // The source table contents should not been seen in the target table. + assert(spark.table(sourceTable.identifier).count() != 0, "the source table should be nonempty") + assert(spark.table(targetTable.identifier).count() == 0, "the target table should be empty") + + // Their schema should be identical + checkAnswer( + sql(s"DESC ${sourceTable.identifier}"), + sql(s"DESC ${targetTable.identifier}")) + + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + // Check whether the new table can be inserted using the data from the original table + sql(s"INSERT INTO TABLE ${targetTable.identifier} SELECT * FROM ${sourceTable.identifier}") + } + + // After insertion, the data should be identical + checkAnswer( + sql(s"SELECT * FROM ${sourceTable.identifier}"), + sql(s"SELECT * FROM ${targetTable.identifier}")) + } + + test("create table with the same name as an index table") { + val tabName = "tab1" + val indexName = tabName + "_index" + withTable(tabName) { + // Spark SQL does not support creating index. Thus, we have to use Hive client. + val client = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client + sql(s"CREATE TABLE $tabName(a int)") + + try { + client.runSqlHive( + s"CREATE INDEX $indexName ON TABLE $tabName (a) AS 'COMPACT' WITH DEFERRED REBUILD") + val indexTabName = + spark.sessionState.catalog.listTables("default", s"*$indexName*").head.table + + // Even if index tables exist, listTables and getTable APIs should still work + checkAnswer( + spark.catalog.listTables().toDF(), + Row(indexTabName, "default", null, null, false) :: + Row(tabName, "default", null, "MANAGED", false) :: Nil) + assert(spark.catalog.getTable("default", indexTabName).name === indexTabName) + + intercept[TableAlreadyExistsException] { + sql(s"CREATE TABLE $indexTabName(b int)") + } + intercept[TableAlreadyExistsException] { + sql(s"ALTER TABLE $tabName RENAME TO $indexTabName") + } + + // When tableExists is not invoked, we still can get an AnalysisException + val e = intercept[AnalysisException] { + sql(s"DESCRIBE $indexTabName") + }.getMessage + assert(e.contains("Hive index table is not supported.")) + } finally { + client.runSqlHive(s"DROP INDEX IF EXISTS $indexName ON $tabName") + } + } + } + + test("insert skewed table") { + val tabName = "tab1" + withTable(tabName) { + // Spark SQL does not support creating skewed table. Thus, we have to use Hive client. + val client = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client + client.runSqlHive( + s""" + |CREATE Table $tabName(col1 int, col2 int) + |PARTITIONED BY (part1 string, part2 string) + |SKEWED BY (col1) ON (3, 4) STORED AS DIRECTORIES + """.stripMargin) + val hiveTable = + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tabName, Some("default"))) + + assert(hiveTable.unsupportedFeatures.contains("skewed columns")) + + // Call loadDynamicPartitions against a skewed table with enabling list bucketing + sql( + s""" + |INSERT OVERWRITE TABLE $tabName + |PARTITION (part1='a', part2) + |SELECT 3, 4, 'b' + """.stripMargin) + + // Call loadPartitions against a skewed table with enabling list bucketing + sql( + s""" + |INSERT INTO TABLE $tabName + |PARTITION (part1='a', part2='b') + |SELECT 1, 2 + """.stripMargin) + + checkAnswer( + sql(s"SELECT * from $tabName"), + Row(3, 4, "a", "b") :: Row(1, 2, "a", "b") :: Nil) + } + } + + test("desc table for data source table - no user-defined schema") { + Seq("parquet", "json", "orc").foreach { fileFormat => + withTable("t1") { + withTempPath { dir => + val path = dir.toURI.toString + spark.range(1).write.format(fileFormat).save(path) + sql(s"CREATE TABLE t1 USING $fileFormat OPTIONS (PATH '$path')") + + val desc = sql("DESC FORMATTED t1").collect().toSeq + + assert(desc.contains(Row("id", "bigint", null))) + } + } + } + } + + test("datasource and statistics table property keys are not allowed") { + import org.apache.spark.sql.hive.HiveExternalCatalog.DATASOURCE_PREFIX + import org.apache.spark.sql.hive.HiveExternalCatalog.STATISTICS_PREFIX + + withTable("tbl") { + sql("CREATE TABLE tbl(a INT) STORED AS parquet") + + Seq(DATASOURCE_PREFIX, STATISTICS_PREFIX).foreach { forbiddenPrefix => + val e = intercept[AnalysisException] { + sql(s"ALTER TABLE tbl SET TBLPROPERTIES ('${forbiddenPrefix}foo' = 'loser')") + } + assert(e.getMessage.contains(forbiddenPrefix + "foo")) + + val e2 = intercept[AnalysisException] { + sql(s"ALTER TABLE tbl UNSET TBLPROPERTIES ('${forbiddenPrefix}foo')") + } + assert(e2.getMessage.contains(forbiddenPrefix + "foo")) + + val e3 = intercept[AnalysisException] { + sql(s"CREATE TABLE tbl (a INT) TBLPROPERTIES ('${forbiddenPrefix}foo'='anything')") + } + assert(e3.getMessage.contains(forbiddenPrefix + "foo")) + } + } + } + + test("truncate table - datasource table") { + import testImplicits._ + + val data = (1 to 10).map { i => (i, i) }.toDF("width", "length") + // Test both a Hive compatible and incompatible code path. + Seq("json", "parquet").foreach { format => + withTable("rectangles") { + data.write.format(format).saveAsTable("rectangles") + assume(spark.table("rectangles").collect().nonEmpty, + "bad test; table was empty to begin with") + + sql("TRUNCATE TABLE rectangles") + assert(spark.table("rectangles").collect().isEmpty) + + // not supported since the table is not partitioned + val e = intercept[AnalysisException] { + sql("TRUNCATE TABLE rectangles PARTITION (width=1)") + } + assert(e.message.contains("Operation not allowed")) + } + } + } + + test("truncate partitioned table - datasource table") { + import testImplicits._ + + val data = (1 to 10).map { i => (i % 3, i % 5, i) }.toDF("width", "length", "height") + + withTable("partTable") { + data.write.partitionBy("width", "length").saveAsTable("partTable") + // supported since partitions are stored in the metastore + sql("TRUNCATE TABLE partTable PARTITION (width=1, length=1)") + assert(spark.table("partTable").filter($"width" === 1).collect().nonEmpty) + assert(spark.table("partTable").filter($"width" === 1 && $"length" === 1).collect().isEmpty) + } + + withTable("partTable") { + data.write.partitionBy("width", "length").saveAsTable("partTable") + // support partial partition spec + sql("TRUNCATE TABLE partTable PARTITION (width=1)") + assert(spark.table("partTable").collect().nonEmpty) + assert(spark.table("partTable").filter($"width" === 1).collect().isEmpty) + } + + withTable("partTable") { + data.write.partitionBy("width", "length").saveAsTable("partTable") + // do nothing if no partition is matched for the given partial partition spec + sql("TRUNCATE TABLE partTable PARTITION (width=100)") + assert(spark.table("partTable").count() == data.count()) + + // throw exception if no partition is matched for the given non-partial partition spec. + intercept[NoSuchPartitionException] { + sql("TRUNCATE TABLE partTable PARTITION (width=100, length=100)") + } + + // throw exception if the column in partition spec is not a partition column. + val e = intercept[AnalysisException] { + sql("TRUNCATE TABLE partTable PARTITION (unknown=1)") + } + assert(e.message.contains("unknown is not a valid partition column")) + } + } + + test("create hive serde table with new syntax") { + withTable("t", "t2", "t3") { + withTempPath { path => + sql( + s""" + |CREATE TABLE t(id int) USING hive + |OPTIONS(fileFormat 'orc', compression 'Zlib') + |LOCATION '${path.toURI}' + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(DDLUtils.isHiveTable(table)) + assert(table.storage.serde == Some("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + assert(table.storage.properties.get("compression") == Some("Zlib")) + assert(spark.table("t").collect().isEmpty) + + sql("INSERT INTO t SELECT 1") + checkAnswer(spark.table("t"), Row(1)) + // Check if this is compressed as ZLIB. + val maybeOrcFile = path.listFiles().find(!_.getName.endsWith(".crc")) + assert(maybeOrcFile.isDefined) + val orcFilePath = maybeOrcFile.get.toPath.toString + val expectedCompressionKind = + OrcFileOperator.getFileReader(orcFilePath).get.getCompression + assert("ZLIB" === expectedCompressionKind.name()) + + sql("CREATE TABLE t2 USING HIVE AS SELECT 1 AS c1, 'a' AS c2") + val table2 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t2")) + assert(DDLUtils.isHiveTable(table2)) + assert(table2.storage.serde == Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) + checkAnswer(spark.table("t2"), Row(1, "a")) + + sql("CREATE TABLE t3(a int, p int) USING hive PARTITIONED BY (p)") + sql("INSERT INTO t3 PARTITION(p=1) SELECT 0") + checkAnswer(spark.table("t3"), Row(0, 1)) + } + } + } + + test("create hive serde table with Catalog") { + withTable("t") { + withTempDir { dir => + val df = spark.catalog.createExternalTable( + "t", + "hive", + new StructType().add("i", "int"), + Map("path" -> dir.getCanonicalPath, "fileFormat" -> "parquet")) + assert(df.collect().isEmpty) + + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(DDLUtils.isHiveTable(table)) + assert(table.storage.inputFormat == + Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat")) + assert(table.storage.outputFormat == + Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) + assert(table.storage.serde == + Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + + sql("INSERT INTO t SELECT 1") + checkAnswer(spark.table("t"), Row(1)) + } + } + } + + test("create hive serde table with DataFrameWriter.saveAsTable") { + withTable("t", "t1") { + Seq(1 -> "a").toDF("i", "j") + .write.format("hive").option("fileFormat", "avro").saveAsTable("t") + checkAnswer(spark.table("t"), Row(1, "a")) + + Seq("c" -> 1).toDF("i", "j").write.format("hive") + .mode(SaveMode.Overwrite).option("fileFormat", "parquet").saveAsTable("t") + checkAnswer(spark.table("t"), Row("c", 1)) + + var table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(DDLUtils.isHiveTable(table)) + assert(table.storage.inputFormat == + Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat")) + assert(table.storage.outputFormat == + Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) + assert(table.storage.serde == + Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + + Seq(9 -> "x").toDF("i", "j") + .write.format("hive").mode(SaveMode.Overwrite).option("fileFormat", "avro").saveAsTable("t") + checkAnswer(spark.table("t"), Row(9, "x")) + + table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(DDLUtils.isHiveTable(table)) + assert(table.storage.inputFormat == + Some("org.apache.hadoop.hive.ql.io.avro.AvroContainerInputFormat")) + assert(table.storage.outputFormat == + Some("org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat")) + assert(table.storage.serde == + Some("org.apache.hadoop.hive.serde2.avro.AvroSerDe")) + + val e2 = intercept[AnalysisException] { + Seq(1 -> "a").toDF("i", "j").write.format("hive").bucketBy(4, "i").saveAsTable("t1") + } + assert(e2.message.contains("Creating bucketed Hive serde table is not supported yet")) + + val e3 = intercept[AnalysisException] { + spark.table("t").write.format("hive").mode("overwrite").saveAsTable("t") + } + assert(e3.message.contains("Cannot overwrite table default.t that is also being read from")) + } + } + + test("append data to hive serde table") { + withTable("t", "t1") { + Seq(1 -> "a").toDF("i", "j") + .write.format("hive").option("fileFormat", "avro").saveAsTable("t") + checkAnswer(spark.table("t"), Row(1, "a")) + + sql("INSERT INTO t SELECT 2, 'b'") + checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Nil) + + Seq(3 -> "c").toDF("i", "j") + .write.format("hive").mode("append").saveAsTable("t") + checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Row(3, "c") :: Nil) + + Seq("c" -> 3).toDF("i", "j") + .write.format("hive").mode("append").saveAsTable("t") + checkAnswer(spark.table("t"), Row(1, "a") :: Row(2, "b") :: Row(3, "c") + :: Row(null, "3") :: Nil) + + Seq(4 -> "d").toDF("i", "j").write.saveAsTable("t1") + + val e = intercept[AnalysisException] { + Seq(5 -> "e").toDF("i", "j") + .write.format("hive").mode("append").saveAsTable("t1") + } + assert(e.message.contains("The format of the existing table default.t1 is " + + "`ParquetFileFormat`. It doesn't match the specified format `HiveFileFormat`.")) + } + } + + test("create partitioned hive serde table as select") { + withTable("t", "t1") { + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + Seq(10 -> "y").toDF("i", "j").write.format("hive").partitionBy("i").saveAsTable("t") + checkAnswer(spark.table("t"), Row("y", 10) :: Nil) + + Seq((1, 2, 3)).toDF("i", "j", "k").write.mode("overwrite").format("hive") + .partitionBy("j", "k").saveAsTable("t") + checkAnswer(spark.table("t"), Row(1, 2, 3) :: Nil) + + spark.sql("create table t1 using hive partitioned by (i) as select 1 as i, 'a' as j") + checkAnswer(spark.table("t1"), Row("a", 1) :: Nil) + } + } + } + + test("read/write files with hive data source is not allowed") { + withTempDir { dir => + val e = intercept[AnalysisException] { + spark.read.format("hive").load(dir.getAbsolutePath) + } + assert(e.message.contains("Hive data source can only be used with tables")) + + val e2 = intercept[AnalysisException] { + Seq(1 -> "a").toDF("i", "j").write.format("hive").save(dir.getAbsolutePath) + } + assert(e2.message.contains("Hive data source can only be used with tables")) + + val e3 = intercept[AnalysisException] { + spark.readStream.format("hive").load(dir.getAbsolutePath) + } + assert(e3.message.contains("Hive data source can only be used with tables")) + + val e4 = intercept[AnalysisException] { + spark.readStream.schema(new StructType()).parquet(dir.getAbsolutePath) + .writeStream.format("hive").start(dir.getAbsolutePath) + } + assert(e4.message.contains("Hive data source can only be used with tables")) + } + } + + test("partitioned table should always put partition columns at the end of table schema") { + def getTableColumns(tblName: String): Seq[String] = { + spark.sessionState.catalog.getTableMetadata(TableIdentifier(tblName)).schema.map(_.name) + } + + withTable("t", "t1", "t2", "t3", "t4", "t5", "t6") { + sql("CREATE TABLE t(a int, b int, c int, d int) USING parquet PARTITIONED BY (d, b)") + assert(getTableColumns("t") == Seq("a", "c", "d", "b")) + + sql("CREATE TABLE t1 USING parquet PARTITIONED BY (d, b) AS SELECT 1 a, 1 b, 1 c, 1 d") + assert(getTableColumns("t1") == Seq("a", "c", "d", "b")) + + Seq((1, 1, 1, 1)).toDF("a", "b", "c", "d").write.partitionBy("d", "b").saveAsTable("t2") + assert(getTableColumns("t2") == Seq("a", "c", "d", "b")) + + withTempPath { path => + val dataPath = new File(new File(path, "d=1"), "b=1").getCanonicalPath + Seq(1 -> 1).toDF("a", "c").write.save(dataPath) + + sql(s"CREATE TABLE t3 USING parquet LOCATION '${path.toURI}'") + assert(getTableColumns("t3") == Seq("a", "c", "d", "b")) + } + + sql("CREATE TABLE t4(a int, b int, c int, d int) USING hive PARTITIONED BY (d, b)") + assert(getTableColumns("t4") == Seq("a", "c", "d", "b")) + + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + sql("CREATE TABLE t5 USING hive PARTITIONED BY (d, b) AS SELECT 1 a, 1 b, 1 c, 1 d") + assert(getTableColumns("t5") == Seq("a", "c", "d", "b")) + + Seq((1, 1, 1, 1)).toDF("a", "b", "c", "d").write.format("hive") + .partitionBy("d", "b").saveAsTable("t6") + assert(getTableColumns("t6") == Seq("a", "c", "d", "b")) + } + } + } + + test("create hive table with a non-existing location") { + withTable("t", "t1") { + withTempPath { dir => + spark.sql(s"CREATE TABLE t(a int, b int) USING hive LOCATION '${dir.toURI}'") + + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + spark.sql("INSERT INTO TABLE t SELECT 1, 2") + assert(dir.exists()) + + checkAnswer(spark.table("t"), Row(1, 2)) + } + // partition table + withTempPath { dir => + spark.sql( + s""" + |CREATE TABLE t1(a int, b int) + |USING hive + |PARTITIONED BY(a) + |LOCATION '${dir.toURI}' + """.stripMargin) + + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1")) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + spark.sql("INSERT INTO TABLE t1 PARTITION(a=1) SELECT 2") + + val partDir = new File(dir, "a=1") + assert(partDir.exists()) + + checkAnswer(spark.table("t1"), Row(2, 1)) + } + } + } + + Seq(true, false).foreach { shouldDelete => + val tcName = if (shouldDelete) "non-existing" else "existed" + + test(s"CTAS for external hive table with a $tcName location") { + withTable("t", "t1") { + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + withTempDir { dir => + if (shouldDelete) dir.delete() + spark.sql( + s""" + |CREATE TABLE t + |USING hive + |LOCATION '${dir.toURI}' + |AS SELECT 3 as a, 4 as b, 1 as c, 2 as d + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + checkAnswer(spark.table("t"), Row(3, 4, 1, 2)) + } + // partition table + withTempDir { dir => + if (shouldDelete) dir.delete() + spark.sql( + s""" + |CREATE TABLE t1 + |USING hive + |PARTITIONED BY(a, b) + |LOCATION '${dir.toURI}' + |AS SELECT 3 as a, 4 as b, 1 as c, 2 as d + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1")) + assert(table.location == makeQualifiedPath(dir.getAbsolutePath)) + + val partDir = new File(dir, "a=3") + assert(partDir.exists()) + + checkAnswer(spark.table("t1"), Row(1, 2, 3, 4)) + } + } + } + } + } + + Seq("parquet", "hive").foreach { datasource => + Seq("a b", "a:b", "a%b", "a,b").foreach { specialChars => + test(s"partition column name of $datasource table containing $specialChars") { + withTable("t") { + withTempDir { dir => + spark.sql( + s""" + |CREATE TABLE t(a string, `$specialChars` string) + |USING $datasource + |PARTITIONED BY(`$specialChars`) + |LOCATION '${dir.toURI}' + """.stripMargin) + + assert(dir.listFiles().isEmpty) + spark.sql(s"INSERT INTO TABLE t PARTITION(`$specialChars`=2) SELECT 1") + val partEscaped = s"${ExternalCatalogUtils.escapePathName(specialChars)}=2" + val partFile = new File(dir, partEscaped) + assert(partFile.listFiles().nonEmpty) + checkAnswer(spark.table("t"), Row("1", "2") :: Nil) + + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + spark.sql(s"INSERT INTO TABLE t PARTITION(`$specialChars`) SELECT 3, 4") + val partEscaped1 = s"${ExternalCatalogUtils.escapePathName(specialChars)}=4" + val partFile1 = new File(dir, partEscaped1) + assert(partFile1.listFiles().nonEmpty) + checkAnswer(spark.table("t"), Row("1", "2") :: Row("3", "4") :: Nil) + } + } + } + } + } + } + + Seq("a b", "a:b", "a%b").foreach { specialChars => + test(s"hive table: location uri contains $specialChars") { + // On Windows, it looks colon in the file name is illegal by default. See + // https://support.microsoft.com/en-us/help/289627 + assume(!Utils.isWindows || specialChars != "a:b") + + withTable("t") { + withTempDir { dir => + val loc = new File(dir, specialChars) + loc.mkdir() + // The parser does not recognize the backslashes on Windows as they are. + // These currently should be escaped. + val escapedLoc = loc.getAbsolutePath.replace("\\", "\\\\") + spark.sql( + s""" + |CREATE TABLE t(a string) + |USING hive + |LOCATION '$escapedLoc' + """.stripMargin) + + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table.location == makeQualifiedPath(loc.getAbsolutePath)) + assert(new Path(table.location).toString.contains(specialChars)) + + assert(loc.listFiles().isEmpty) + if (specialChars != "a:b") { + spark.sql("INSERT INTO TABLE t SELECT 1") + assert(loc.listFiles().length >= 1) + checkAnswer(spark.table("t"), Row("1") :: Nil) + } else { + val e = intercept[AnalysisException] { + spark.sql("INSERT INTO TABLE t SELECT 1") + }.getMessage + assert(e.contains("java.net.URISyntaxException: Relative path in absolute URI: a:b")) + } + } + + withTempDir { dir => + val loc = new File(dir, specialChars) + loc.mkdir() + val escapedLoc = loc.getAbsolutePath.replace("\\", "\\\\") + spark.sql( + s""" + |CREATE TABLE t1(a string, b string) + |USING hive + |PARTITIONED BY(b) + |LOCATION '$escapedLoc' + """.stripMargin) + + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t1")) + assert(table.location == makeQualifiedPath(loc.getAbsolutePath)) + assert(new Path(table.location).toString.contains(specialChars)) + + assert(loc.listFiles().isEmpty) + if (specialChars != "a:b") { + spark.sql("INSERT INTO TABLE t1 PARTITION(b=2) SELECT 1") + val partFile = new File(loc, "b=2") + assert(partFile.listFiles().nonEmpty) + checkAnswer(spark.table("t1"), Row("1", "2") :: Nil) + + spark.sql("INSERT INTO TABLE t1 PARTITION(b='2017-03-03 12:13%3A14') SELECT 1") + val partFile1 = new File(loc, "b=2017-03-03 12:13%3A14") + assert(!partFile1.exists()) + + if (!Utils.isWindows) { + // Actual path becomes "b=2017-03-03%2012%3A13%253A14" on Windows. + val partFile2 = new File(loc, "b=2017-03-03 12%3A13%253A14") + assert(partFile2.listFiles().nonEmpty) + checkAnswer(spark.table("t1"), + Row("1", "2") :: Row("1", "2017-03-03 12:13%3A14") :: Nil) + } + } else { + val e = intercept[AnalysisException] { + spark.sql("INSERT INTO TABLE t1 PARTITION(b=2) SELECT 1") + }.getMessage + assert(e.contains("java.net.URISyntaxException: Relative path in absolute URI: a:b")) + + val e1 = intercept[AnalysisException] { + spark.sql("INSERT INTO TABLE t1 PARTITION(b='2017-03-03 12:13%3A14') SELECT 1") + }.getMessage + assert(e1.contains("java.net.URISyntaxException: Relative path in absolute URI: a:b")) + } + } + } + } + } + + test("SPARK-19905: Hive SerDe table input paths") { + withTable("spark_19905") { + withTempView("spark_19905_view") { + spark.range(10).createOrReplaceTempView("spark_19905_view") + sql("CREATE TABLE spark_19905 STORED AS RCFILE AS SELECT * FROM spark_19905_view") + assert(spark.table("spark_19905").inputFiles.nonEmpty) + assert(sql("SELECT input_file_name() FROM spark_19905").count() > 0) + } + } + } + + hiveFormats.foreach { tableType => + test(s"alter hive serde table add columns -- partitioned - $tableType") { + withTable("tab") { + sql( + s""" + |CREATE TABLE tab (c1 int, c2 int) + |PARTITIONED BY (c3 int) STORED AS $tableType + """.stripMargin) + + sql("INSERT INTO tab PARTITION (c3=1) VALUES (1, 2)") + sql("ALTER TABLE tab ADD COLUMNS (c4 int)") + + checkAnswer( + sql("SELECT * FROM tab WHERE c3 = 1"), + Seq(Row(1, 2, null, 1)) + ) + assert(spark.table("tab").schema + .contains(StructField("c4", IntegerType))) + sql("INSERT INTO tab PARTITION (c3=2) VALUES (2, 3, 4)") + checkAnswer( + spark.table("tab"), + Seq(Row(1, 2, null, 1), Row(2, 3, 4, 2)) + ) + checkAnswer( + sql("SELECT * FROM tab WHERE c3 = 2 AND c4 IS NOT NULL"), + Seq(Row(2, 3, 4, 2)) + ) + + sql("ALTER TABLE tab ADD COLUMNS (c5 char(10))") + assert(spark.table("tab").schema.find(_.name == "c5") + .get.metadata.getString("HIVE_TYPE_STRING") == "char(10)") + } + } + } + + hiveFormats.foreach { tableType => + test(s"alter hive serde table add columns -- with predicate - $tableType ") { + withTable("tab") { + sql(s"CREATE TABLE tab (c1 int, c2 int) STORED AS $tableType") + sql("INSERT INTO tab VALUES (1, 2)") + sql("ALTER TABLE tab ADD COLUMNS (c4 int)") + checkAnswer( + sql("SELECT * FROM tab WHERE c4 IS NULL"), + Seq(Row(1, 2, null)) + ) + assert(spark.table("tab").schema + .contains(StructField("c4", IntegerType))) + sql("INSERT INTO tab VALUES (2, 3, 4)") + checkAnswer( + sql("SELECT * FROM tab WHERE c4 = 4 "), + Seq(Row(2, 3, 4)) + ) + checkAnswer( + spark.table("tab"), + Seq(Row(1, 2, null), Row(2, 3, 4)) + ) + } + } + } + + Seq(true, false).foreach { caseSensitive => + test(s"alter add columns with existing column name - caseSensitive $caseSensitive") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> s"$caseSensitive") { + withTable("tab") { + sql("CREATE TABLE tab (c1 int) PARTITIONED BY (c2 int) STORED AS PARQUET") + if (!caseSensitive) { + // duplicating partitioning column name + val e1 = intercept[AnalysisException] { + sql("ALTER TABLE tab ADD COLUMNS (C2 string)") + }.getMessage + assert(e1.contains("Found duplicate column(s)")) + + // duplicating data column name + val e2 = intercept[AnalysisException] { + sql("ALTER TABLE tab ADD COLUMNS (C1 string)") + }.getMessage + assert(e2.contains("Found duplicate column(s)")) + } else { + // hive catalog will still complains that c1 is duplicate column name because hive + // identifiers are case insensitive. + val e1 = intercept[AnalysisException] { + sql("ALTER TABLE tab ADD COLUMNS (C2 string)") + }.getMessage + assert(e1.contains("HiveException")) + + // hive catalog will still complains that c1 is duplicate column name because hive + // identifiers are case insensitive. + val e2 = intercept[AnalysisException] { + sql("ALTER TABLE tab ADD COLUMNS (C1 string)") + }.getMessage + assert(e2.contains("HiveException")) + } + } + } + } + } + + test("SPARK-21216: join with a streaming DataFrame") { + import org.apache.spark.sql.execution.streaming.MemoryStream + import testImplicits._ + + implicit val _sqlContext = spark.sqlContext + + Seq((1, "one"), (2, "two"), (4, "four")).toDF("number", "word").createOrReplaceTempView("t1") + // Make a table and ensure it will be broadcast. + sql("""CREATE TABLE smallTable(word string, number int) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |STORED AS TEXTFILE + """.stripMargin) + + sql( + """INSERT INTO smallTable + |SELECT word, number from t1 + """.stripMargin) + + val inputData = MemoryStream[Int] + val joined = inputData.toDS().toDF() + .join(spark.table("smallTable"), $"value" === $"number") + + val sq = joined.writeStream + .format("memory") + .queryName("t2") + .start() + try { + inputData.addData(1, 2) + + sq.processAllAvailable() + + checkAnswer( + spark.table("t2"), + Seq(Row(1, "one", 1), Row(2, "two", 2)) + ) + } finally { + sq.stop() + } + } + + test("table name with schema") { + // regression test for SPARK-11778 + withDatabase("usrdb") { + spark.sql("create schema usrdb") + withTable("usrdb.test") { + spark.sql("create table usrdb.test(c int)") + spark.read.table("usrdb.test") + } + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala index 94162da4eae1..3066a4f305f0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -18,47 +18,85 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils /** * A set of tests that validates support for Hive Explain command. */ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + import testImplicits._ + + test("show cost in explain command") { + // For readability, we only show optimized plan and physical plan in explain cost command + checkKeywordsExist(sql("EXPLAIN COST SELECT * FROM src "), + "Optimized Logical Plan", "Physical Plan") + checkKeywordsNotExist(sql("EXPLAIN COST SELECT * FROM src "), + "Parsed Logical Plan", "Analyzed Logical Plan") + + // Only has sizeInBytes before ANALYZE command + checkKeywordsExist(sql("EXPLAIN COST SELECT * FROM src "), "sizeInBytes") + checkKeywordsNotExist(sql("EXPLAIN COST SELECT * FROM src "), "rowCount") + + // Has both sizeInBytes and rowCount after ANALYZE command + sql("ANALYZE TABLE src COMPUTE STATISTICS") + checkKeywordsExist(sql("EXPLAIN COST SELECT * FROM src "), "sizeInBytes", "rowCount") + + // No cost information + checkKeywordsNotExist(sql("EXPLAIN SELECT * FROM src "), "sizeInBytes", "rowCount") + } test("explain extended command") { - checkExistence(sql(" explain select * from src where key=123 "), true, - "== Physical Plan ==") - checkExistence(sql(" explain select * from src where key=123 "), false, + checkKeywordsExist(sql(" explain select * from src where key=123 "), + "== Physical Plan ==", + "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") + + checkKeywordsNotExist(sql(" explain select * from src where key=123 "), "== Parsed Logical Plan ==", "== Analyzed Logical Plan ==", - "== Optimized Logical Plan ==") - checkExistence(sql(" explain extended select * from src where key=123 "), true, + "== Optimized Logical Plan ==", + "Owner", + "Database", + "Created", + "Last Access", + "Type", + "Provider", + "Properties", + "Statistics", + "Location", + "Serde Library", + "InputFormat", + "OutputFormat", + "Partition Provider", + "Schema" + ) + + checkKeywordsExist(sql(" explain extended select * from src where key=123 "), "== Parsed Logical Plan ==", "== Analyzed Logical Plan ==", "== Optimized Logical Plan ==", - "== Physical Plan ==", - "Code Generation") + "== Physical Plan ==") } test("explain create table command") { - checkExistence(sql("explain create table temp__b as select * from src limit 2"), true, + checkKeywordsExist(sql("explain create table temp__b as select * from src limit 2"), "== Physical Plan ==", "InsertIntoHiveTable", "Limit", "src") - checkExistence(sql("explain extended create table temp__b as select * from src limit 2"), true, + checkKeywordsExist(sql("explain extended create table temp__b as select * from src limit 2"), "== Parsed Logical Plan ==", "== Analyzed Logical Plan ==", "== Optimized Logical Plan ==", "== Physical Plan ==", - "CreateTableAsSelect", + "CreateHiveTableAsSelect", "InsertIntoHiveTable", "Limit", "src") - checkExistence(sql( + checkKeywordsExist(sql( """ | EXPLAIN EXTENDED CREATE TABLE temp__b | ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" @@ -66,40 +104,58 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto | STORED AS RCFile | TBLPROPERTIES("tbl_p1"="p11", "tbl_p2"="p22") | AS SELECT * FROM src LIMIT 2 - """.stripMargin), true, + """.stripMargin), "== Parsed Logical Plan ==", "== Analyzed Logical Plan ==", "== Optimized Logical Plan ==", "== Physical Plan ==", - "CreateTableAsSelect", + "CreateHiveTableAsSelect", "InsertIntoHiveTable", "Limit", "src") } - test("SPARK-6212: The EXPLAIN output of CTAS only shows the analyzed plan") { - withTempTable("jt") { - val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""")) - hiveContext.read.json(rdd).registerTempTable("jt") + test("SPARK-17409: The EXPLAIN output of CTAS only shows the analyzed plan") { + withTempView("jt") { + val ds = (1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""").toDS() + spark.read.json(ds).createOrReplaceTempView("jt") val outputs = sql( s""" |EXPLAIN EXTENDED |CREATE TABLE t1 |AS |SELECT * FROM jt - """.stripMargin).collect().map(_.mkString).mkString + """.stripMargin).collect().map(_.mkString).mkString val shouldContain = "== Parsed Logical Plan ==" :: "== Analyzed Logical Plan ==" :: "Subquery" :: "== Optimized Logical Plan ==" :: "== Physical Plan ==" :: - "CreateTableAsSelect" :: "InsertIntoHiveTable" :: "jt" :: Nil + "CreateHiveTableAsSelect" :: "InsertIntoHiveTable" :: "jt" :: Nil for (key <- shouldContain) { assert(outputs.contains(key), s"$key doesn't exist in result") } val physicalIndex = outputs.indexOf("== Physical Plan ==") - assert(!outputs.substring(physicalIndex).contains("Subquery"), - "Physical Plan should not contain Subquery since it's eliminated by optimizer") + assert(outputs.substring(physicalIndex).contains("Subquery"), + "Physical Plan should contain SubqueryAlias since the query should not be optimized") + } + } + + test("EXPLAIN CODEGEN command") { + checkKeywordsExist(sql("EXPLAIN CODEGEN SELECT 1"), + "WholeStageCodegen", + "Generated code:", + "/* 001 */ public Object generate(Object[] references) {", + "/* 002 */ return new GeneratedIterator(references);", + "/* 003 */ }" + ) + + checkKeywordsNotExist(sql("EXPLAIN CODEGEN SELECT 1"), + "== Physical Plan ==" + ) + + intercept[ParseException] { + sql("EXPLAIN EXTENDED CODEGEN SELECT 1") } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala deleted file mode 100644 index 0d4c7f86b315..000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala +++ /dev/null @@ -1,53 +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.execution - -import org.apache.spark.sql.{Row, QueryTest} -import org.apache.spark.sql.hive.test.{TestHive, TestHiveSingleton} - -/** - * A set of tests that validates commands can also be queried by like a table - */ -class HiveOperatorQueryableSuite extends QueryTest with TestHiveSingleton { - import hiveContext._ - - test("SPARK-5324 query result of describe command") { - hiveContext.loadTestTable("src") - - // register a describe command to be a temp table - sql("desc src").registerTempTable("mydesc") - checkAnswer( - sql("desc mydesc"), - Seq( - Row("col_name", "string", "name of the column"), - Row("data_type", "string", "data type of the column"), - Row("comment", "string", "comment of the column"))) - - checkAnswer( - sql("select * from mydesc"), - Seq( - Row("key", "int", null), - Row("value", "string", null))) - - checkAnswer( - sql("select col_name, data_type, comment from mydesc"), - Seq( - Row("key", "int", null), - Row("value", "string", null))) - } -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala index cd055f9eca37..78cdc67800c1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala @@ -17,20 +17,20 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.functions._ import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.expressions.Window +import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.test.TestHiveSingleton class HivePlanTest extends QueryTest with TestHiveSingleton { - import hiveContext.sql - import hiveContext.implicits._ + import spark.sql + import spark.implicits._ test("udf constant folding") { - Seq.empty[Tuple1[Int]].toDF("a").registerTempTable("t") - val optimized = sql("SELECT cos(null) FROM t").queryExecution.optimizedPlan - val correctAnswer = sql("SELECT cast(null as double) FROM t").queryExecution.optimizedPlan + Seq.empty[Tuple1[Int]].toDF("a").createOrReplaceTempView("t") + val optimized = sql("SELECT cos(null) AS c FROM t").queryExecution.optimizedPlan + val correctAnswer = sql("SELECT cast(null as double) AS c FROM t").queryExecution.optimizedPlan comparePlans(optimized, correctAnswer) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala index f7b37dae0a5f..bb4ce6d3aa3f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.util._ /** * A framework for running the query tests that are listed as a set of text files. * - * TestSuites that derive from this class must provide a map of testCaseName -> testCaseFiles + * TestSuites that derive from this class must provide a map of testCaseName to testCaseFiles * that should be included. Additionally, there is support for whitelisting and blacklisting * tests as development progresses. */ @@ -40,14 +40,14 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { def testCases: Seq[(String, File)] - val runAll = + val runAll: Boolean = !(System.getProperty("spark.hive.alltests") == null) || runOnlyDirectories.nonEmpty || skipDirectories.nonEmpty - val whiteListProperty = "spark.hive.whitelist" + val whiteListProperty: String = "spark.hive.whitelist" // Allow the whiteList to be overridden by a system property - val realWhiteList = + val realWhiteList: Seq[String] = Option(System.getProperty(whiteListProperty)).map(_.split(",").toSeq).getOrElse(whiteList) // Go through all the test cases and add them to scala test. @@ -59,7 +59,7 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { runAll) { // Build a test case and submit it to scala test framework... val queriesString = fileToString(testCaseFile) - createQueryTest(testCaseName, queriesString) + createQueryTest(testCaseName, queriesString, reset = true, tryWithoutResettingFirst = true) } else { // Only output warnings for the built in whitelist as this clutters the output when the user // trying to execute a single test from the commandline. 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 fc72e3c7dc6a..2ea51791d0f7 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 @@ -18,24 +18,26 @@ package org.apache.spark.sql.hive.execution import java.io.File +import java.net.URI +import java.sql.Timestamp import java.util.{Locale, TimeZone} -import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoin - import scala.util.Try -import org.scalatest.BeforeAndAfter - import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkFiles, SparkException} -import org.apache.spark.sql.{AnalysisException, DataFrame, Row} +import org.apache.spark.{SparkFiles, TestUtils} +import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Cast +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec import org.apache.spark.sql.hive._ -import org.apache.spark.sql.hive.test.TestHiveContext import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils case class TestData(a: Int, b: String) @@ -43,32 +45,42 @@ case class TestData(a: Int, b: String) * A set of test cases expressed in Hive QL that are not covered by the tests * included in the hive distribution. */ -class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { +class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAndAfter { private val originalTimeZone = TimeZone.getDefault private val originalLocale = Locale.getDefault import org.apache.spark.sql.hive.test.TestHive.implicits._ + private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled + + def spark: SparkSession = sparkSession + override def beforeAll() { - TestHive.cacheTables = true + super.beforeAll() + TestHive.setCacheTables(true) // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting Locale.setDefault(Locale.US) + // Ensures that cross joins are enabled so that we can test them + TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, true) } override def afterAll() { - TestHive.cacheTables = false - TimeZone.setDefault(originalTimeZone) - Locale.setDefault(originalLocale) - sql("DROP TEMPORARY FUNCTION udtf_count2") + try { + TestHive.setCacheTables(false) + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + sql("DROP TEMPORARY FUNCTION IF EXISTS udtf_count2") + TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) + } finally { + super.afterAll() + } } - test("SPARK-4908: concurrent hive native commands") { - (1 to 100).par.map { _ => - sql("USE default") - sql("SHOW DATABASES") - } + private def assertUnsupportedFeature(body: => Unit): Unit = { + val e = intercept[ParseException] { body } + assert(e.getMessage.toLowerCase(Locale.ROOT).contains("operation not allowed")) } // Testing the Broadcast based join for cartesian join (cross join) @@ -113,7 +125,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { test("SPARK-10484 Optimize the Cartesian (Cross) Join with broadcast based JOIN") { def assertBroadcastNestedLoopJoin(sqlText: String): Unit = { assert(sql(sqlText).queryExecution.sparkPlan.collect { - case _: BroadcastNestedLoopJoin => 1 + case _: BroadcastNestedLoopJoinExec => 1 }.nonEmpty) } @@ -123,60 +135,6 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { assertBroadcastNestedLoopJoin(spark_10484_4) } - createQueryTest("SPARK-8976 Wrong Result for Rollup #1", - """ - SELECT count(*) AS cnt, key % 5,GROUPING__ID FROM src group by key%5 WITH ROLLUP - """.stripMargin) - - createQueryTest("SPARK-8976 Wrong Result for Rollup #2", - """ - SELECT - count(*) AS cnt, - key % 5 as k1, - key-5 as k2, - GROUPING__ID as k3 - FROM src group by key%5, key-5 - WITH ROLLUP ORDER BY cnt, k1, k2, k3 LIMIT 10 - """.stripMargin) - - createQueryTest("SPARK-8976 Wrong Result for Rollup #3", - """ - SELECT - count(*) AS cnt, - key % 5 as k1, - key-5 as k2, - GROUPING__ID as k3 - FROM (SELECT key, key%2, key - 5 FROM src) t group by key%5, key-5 - WITH ROLLUP ORDER BY cnt, k1, k2, k3 LIMIT 10 - """.stripMargin) - - createQueryTest("SPARK-8976 Wrong Result for CUBE #1", - """ - SELECT count(*) AS cnt, key % 5,GROUPING__ID FROM src group by key%5 WITH CUBE - """.stripMargin) - - createQueryTest("SPARK-8976 Wrong Result for CUBE #2", - """ - SELECT - count(*) AS cnt, - key % 5 as k1, - key-5 as k2, - GROUPING__ID as k3 - FROM (SELECT key, key%2, key - 5 FROM src) t group by key%5, key-5 - WITH CUBE ORDER BY cnt, k1, k2, k3 LIMIT 10 - """.stripMargin) - - createQueryTest("SPARK-8976 Wrong Result for GroupingSet", - """ - SELECT - count(*) AS cnt, - key % 5 as k1, - key-5 as k2, - GROUPING__ID as k3 - FROM (SELECT key, key%2, key - 5 FROM src) t group by key%5, key-5 - GROUPING SETS (key%5, key-5) ORDER BY cnt, k1, k2, k3 LIMIT 10 - """.stripMargin) - createQueryTest("insert table with generator with column name", """ | CREATE TABLE gen_tmp (key Int); @@ -251,20 +209,16 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |IF(TRUE, CAST(NULL AS BINARY), CAST("1" AS BINARY)) AS COL18, |IF(FALSE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL19, |IF(TRUE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL20, - |IF(FALSE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL21, - |IF(TRUE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL22, - |IF(FALSE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL23, - |IF(TRUE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL24 + |IF(TRUE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL21, + |IF(FALSE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL22, + |IF(TRUE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL23 |FROM src LIMIT 1""".stripMargin) - createQueryTest("constant array", - """ - |SELECT sort_array( - | sort_array( - | array("hadoop distributed file system", - | "enterprise databases", "hadoop map-reduce"))) - |FROM src LIMIT 1; - """.stripMargin) + test("constant null testing timestamp") { + val r1 = sql("SELECT IF(FALSE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL20") + .collect().head + assert(new Timestamp(1000) == r1.getTimestamp(0)) + } createQueryTest("null case", "SELECT case when(true) then 1 else null end FROM src LIMIT 1") @@ -319,12 +273,6 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { "SELECT 11 % 10, IF((101.1 % 100.0) BETWEEN 1.01 AND 1.11, \"true\", \"false\"), " + "(101 / 2) % 10 FROM src LIMIT 1") - test("Query expressed in SQL") { - setConf("spark.sql.dialect", "sql") - assert(sql("SELECT 1").collect() === Array(Row(1))) - setConf("spark.sql.dialect", "hiveql") - } - test("Query expressed in HiveQL") { sql("FROM src SELECT key").collect() } @@ -374,19 +322,12 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("trivial join ON clause", "SELECT * FROM src a JOIN src b ON a.key = b.key") - createQueryTest("small.cartesian", - "SELECT a.key, b.key FROM (SELECT key FROM src WHERE key < 1) a JOIN " + - "(SELECT key FROM src WHERE key = 2) b") - createQueryTest("length.udf", "SELECT length(\"test\") FROM src LIMIT 1") createQueryTest("partitioned table scan", "SELECT ds, hr, key, value FROM srcpart") - createQueryTest("hash", - "SELECT hash('test') FROM src LIMIT 1") - createQueryTest("create table as", """ |CREATE TABLE createdtable AS SELECT * FROM src; @@ -429,32 +370,38 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { """.stripMargin) test("SPARK-7270: consider dynamic partition when comparing table output") { - sql(s"CREATE TABLE test_partition (a STRING) PARTITIONED BY (b BIGINT, c STRING)") - sql(s"CREATE TABLE ptest (a STRING, b BIGINT, c STRING)") + withTable("test_partition", "ptest") { + sql(s"CREATE TABLE test_partition (a STRING) PARTITIONED BY (b BIGINT, c STRING)") + sql(s"CREATE TABLE ptest (a STRING, b BIGINT, c STRING)") - val analyzedPlan = sql( - """ + val analyzedPlan = sql( + """ |INSERT OVERWRITE table test_partition PARTITION (b=1, c) |SELECT 'a', 'c' from ptest """.stripMargin).queryExecution.analyzed - assertResult(false, "Incorrect cast detected\n" + analyzedPlan) { + assertResult(false, "Incorrect cast detected\n" + analyzedPlan) { var hasCast = false - analyzedPlan.collect { - case p: Project => p.transformExpressionsUp { case c: Cast => hasCast = true; c } + analyzedPlan.collect { + case p: Project => p.transformExpressionsUp { case c: Cast => hasCast = true; c } + } + hasCast } - hasCast } } + // Some tests suing script transformation are skipped as it requires `/bin/bash` which + // can be missing or differently located. createQueryTest("transform", - "SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src") + "SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src", + skip = !TestUtils.testCommandAvailable("/bin/bash")) createQueryTest("schema-less transform", """ |SELECT TRANSFORM (key, value) USING 'cat' FROM src; |SELECT TRANSFORM (*) USING 'cat' FROM src; - """.stripMargin) + """.stripMargin, + skip = !TestUtils.testCommandAvailable("/bin/bash")) val delimiter = "'\t'" @@ -462,19 +409,22 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { s""" |SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} |USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src; - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin.replaceAll("\n", " "), + skip = !TestUtils.testCommandAvailable("/bin/bash")) createQueryTest("transform with custom field delimiter2", s""" |SELECT TRANSFORM (key, value) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} |USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src; - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin.replaceAll("\n", " "), + skip = !TestUtils.testCommandAvailable("/bin/bash")) createQueryTest("transform with custom field delimiter3", s""" |SELECT TRANSFORM (*) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} |USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src; - """.stripMargin.replaceAll("\n", " ")) + """.stripMargin.replaceAll("\n", " "), + skip = !TestUtils.testCommandAvailable("/bin/bash")) createQueryTest("transform with SerDe", """ @@ -482,16 +432,18 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' |USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM src; - """.stripMargin.replaceAll(System.lineSeparator(), " ")) + """.stripMargin.replaceAll(System.lineSeparator(), " "), + skip = !TestUtils.testCommandAvailable("/bin/bash")) test("transform with SerDe2") { - - sql("CREATE TABLE small_src(key INT, value STRING)") - sql("INSERT OVERWRITE TABLE small_src SELECT key, value FROM src LIMIT 10") - - val expected = sql("SELECT key FROM small_src").collect().head - val res = sql( - """ + assume(TestUtils.testCommandAvailable("/bin/bash")) + withTable("small_src") { + sql("CREATE TABLE small_src(key INT, value STRING)") + sql("INSERT OVERWRITE TABLE small_src SELECT key, value FROM src LIMIT 10") + + val expected = sql("SELECT key FROM small_src").collect().head + val res = sql( + """ |SELECT TRANSFORM (key) ROW FORMAT SERDE |'org.apache.hadoop.hive.serde2.avro.AvroSerDe' |WITH SERDEPROPERTIES ('avro.schema.literal'='{"namespace": @@ -503,7 +455,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |FROM small_src """.stripMargin.replaceAll(System.lineSeparator(), " ")).collect().head - assert(expected(0) === res(0)) + assert(expected(0) === res(0)) + } } createQueryTest("transform with SerDe3", @@ -513,7 +466,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |('serialization.last.column.takes.rest'='true') USING 'cat' AS (tKey, tValue) |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' |WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src; - """.stripMargin.replaceAll(System.lineSeparator(), " ")) + """.stripMargin.replaceAll(System.lineSeparator(), " "), + skip = !TestUtils.testCommandAvailable("/bin/bash")) createQueryTest("transform with SerDe4", """ @@ -522,7 +476,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |('serialization.last.column.takes.rest'='true') USING 'cat' ROW FORMAT SERDE |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES |('serialization.last.column.takes.rest'='true') FROM src; - """.stripMargin.replaceAll(System.lineSeparator(), " ")) + """.stripMargin.replaceAll(System.lineSeparator(), " "), + skip = !TestUtils.testCommandAvailable("/bin/bash")) createQueryTest("LIKE", "SELECT * FROM src WHERE value LIKE '%1%'") @@ -606,26 +561,32 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { // Jdk version leads to different query output for double, so not use createQueryTest here test("timestamp cast #1") { val res = sql("SELECT CAST(CAST(1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head - assert(0.001 == res.getDouble(0)) + assert(1 == res.getDouble(0)) } createQueryTest("timestamp cast #2", "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") - createQueryTest("timestamp cast #3", - "SELECT CAST(CAST(1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1") + test("timestamp cast #3") { + val res = sql("SELECT CAST(CAST(1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head + assert(1200 == res.getInt(0)) + } createQueryTest("timestamp cast #4", "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") - createQueryTest("timestamp cast #5", - "SELECT CAST(CAST(-1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + test("timestamp cast #5") { + val res = sql("SELECT CAST(CAST(-1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head + assert(-1 == res.get(0)) + } createQueryTest("timestamp cast #6", "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") - createQueryTest("timestamp cast #7", - "SELECT CAST(CAST(-1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1") + test("timestamp cast #7") { + val res = sql("SELECT CAST(CAST(-1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head + assert(-1200 == res.getInt(0)) + } createQueryTest("timestamp cast #8", "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") @@ -648,7 +609,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |select * where key = 4 """.stripMargin) - // test get_json_object again Hive, because the HiveCompatabilitySuite cannot handle result + // test get_json_object again Hive, because the HiveCompatibilitySuite cannot handle result // with newline in it. createQueryTest("get_json_object #1", "SELECT get_json_object(src_json.json, '$') FROM src_json") @@ -710,11 +671,13 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { test("implement identity function using case statement") { val actual = sql("SELECT (CASE key WHEN key THEN key END) FROM src") + .rdd .map { case Row(i: Int) => i } .collect() .toSet val expected = sql("SELECT key FROM src") + .rdd .map { case Row(i: Int) => i } .collect() .toSet @@ -733,12 +696,12 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("case sensitivity when query Hive table", "SELECT srcalias.KEY, SRCALIAS.value FROM sRc SrCAlias WHERE SrCAlias.kEy < 15") - test("case sensitivity: registered table") { + test("case sensitivity: created temporary view") { val testData = TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(2, "str2") :: Nil) - testData.toDF().registerTempTable("REGisteredTABle") + testData.toDF().createOrReplaceTempView("REGisteredTABle") assertResult(Array(Row(2, "str2"))) { sql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " + @@ -748,7 +711,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { def isExplanation(result: DataFrame): Boolean = { val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } - explanation.contains("== Physical Plan ==") + explanation.head.startsWith("== Physical Plan ==") } test("SPARK-1704: Explain commands as a DataFrame") { @@ -762,14 +725,14 @@ 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") + .zipWithIndex.map {case ((value, attr), key) => HavingRow(key, value, attr)} + TestHive.sparkContext.parallelize(fixture).toDF().createOrReplaceTempView("having_test") val results = sql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value HAVING attr > 3") .collect() - .map(x => Pair(x.getString(0), x.getInt(1))) + .map(x => (x.getString(0), x.getInt(1))) - assert(results === Array(Pair("foo", 4))) + assert(results === Array(("foo", 4))) TestHive.reset() } @@ -781,6 +744,24 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { assert(sql("select key from src having key > 490").collect().size < 100) } + test("union/except/intersect") { + assertResult(Array(Row(1), Row(1))) { + sql("select 1 as a union all select 1 as a").collect() + } + assertResult(Array(Row(1))) { + sql("select 1 as a union distinct select 1 as a").collect() + } + assertResult(Array(Row(1))) { + sql("select 1 as a union select 1 as a").collect() + } + assertResult(Array()) { + sql("select 1 as a except select 1 as a").collect() + } + assertResult(Array(Row(1))) { + sql("select 1 as a intersect select 1 as a").collect() + } + } + test("SPARK-5383 alias for udfs with multi output columns") { assert( sql("select stack(2, key, value, key, value) as (a, b) from src limit 5") @@ -800,137 +781,28 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { assert(sql("select array(key, *) from src limit 5").collect().size == 5) } - test("Query Hive native command execution result") { - val databaseName = "test_native_commands" - - assertResult(0) { - sql(s"DROP DATABASE IF EXISTS $databaseName").count() - } - - assertResult(0) { - sql(s"CREATE DATABASE $databaseName").count() - } - - assert( - sql("SHOW DATABASES") - .select('result) - .collect() - .map(_.getString(0)) - .contains(databaseName)) - - assert(isExplanation(sql(s"EXPLAIN SELECT key, COUNT(*) FROM src GROUP BY key"))) - - TestHive.reset() - } - test("Exactly once semantics for DDL and command statements") { val tableName = "test_exactly_once" - val q0 = sql(s"CREATE TABLE $tableName(key INT, value STRING)") - - // If the table was not created, the following assertion would fail - assert(Try(table(tableName)).isSuccess) - - // If the CREATE TABLE command got executed again, the following assertion would fail - assert(Try(q0.count()).isSuccess) - } - - test("DESCRIBE commands") { - sql(s"CREATE TABLE test_describe_commands1 (key INT, value STRING) PARTITIONED BY (dt STRING)") - - sql( - """FROM src INSERT OVERWRITE TABLE test_describe_commands1 PARTITION (dt='2008-06-08') - |SELECT key, value - """.stripMargin) - - // Describe a table - assertResult( - Array( - Row("key", "int", null), - Row("value", "string", null), - Row("dt", "string", null), - Row("# Partition Information", "", ""), - Row("# col_name", "data_type", "comment"), - Row("dt", "string", null)) - ) { - sql("DESCRIBE test_describe_commands1") - .select('col_name, 'data_type, 'comment) - .collect() - } - - // Describe a table with a fully qualified table name - assertResult( - Array( - Row("key", "int", null), - Row("value", "string", null), - Row("dt", "string", null), - Row("# Partition Information", "", ""), - Row("# col_name", "data_type", "comment"), - Row("dt", "string", null)) - ) { - sql("DESCRIBE default.test_describe_commands1") - .select('col_name, 'data_type, 'comment) - .collect() - } - - // Describe a column is a native command - assertResult(Array(Array("value", "string", "from deserializer"))) { - sql("DESCRIBE test_describe_commands1 value") - .select('result) - .collect() - .map(_.getString(0).split("\t").map(_.trim)) - } - - // Describe a column is a native command - assertResult(Array(Array("value", "string", "from deserializer"))) { - sql("DESCRIBE default.test_describe_commands1 value") - .select('result) - .collect() - .map(_.getString(0).split("\t").map(_.trim)) - } + withTable(tableName) { + val q0 = sql(s"CREATE TABLE $tableName(key INT, value STRING)") - // Describe a partition is a native command - assertResult( - Array( - Array("key", "int"), - Array("value", "string"), - Array("dt", "string"), - Array(""), - Array("# Partition Information"), - Array("# col_name", "data_type", "comment"), - Array(""), - Array("dt", "string")) - ) { - sql("DESCRIBE test_describe_commands1 PARTITION (dt='2008-06-08')") - .select('result) - .collect() - .map(_.getString(0).replaceAll("None", "").trim.split("\t").map(_.trim)) - } + // If the table was not created, the following assertion would fail + assert(Try(table(tableName)).isSuccess) - // Describe a registered temporary table. - val testData = - TestHive.sparkContext.parallelize( - TestData(1, "str1") :: - TestData(1, "str2") :: Nil) - testData.toDF().registerTempTable("test_describe_commands2") - - assertResult( - Array( - Row("a", "int", ""), - Row("b", "string", "")) - ) { - sql("DESCRIBE test_describe_commands2") - .select('col_name, 'data_type, 'comment) - .collect() + // If the CREATE TABLE command got executed again, the following assertion would fail + assert(Try(q0.count()).isSuccess) } } test("SPARK-2263: Insert Map values") { - sql("CREATE TABLE m(value MAP)") - sql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10") - sql("SELECT * FROM m").collect().zip(sql("SELECT * FROM src LIMIT 10").collect()).map { - case (Row(map: Map[_, _]), Row(key: Int, value: String)) => - assert(map.size === 1) - assert(map.head === (key, value)) + withTable("m") { + sql("CREATE TABLE m(value MAP)") + sql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10") + sql("SELECT * FROM m").collect().zip(sql("SELECT * FROM src LIMIT 10").collect()).foreach { + case (Row(map: Map[_, _]), Row(key: Int, value: String)) => + assert(map.size === 1) + assert(map.head === ((key, value))) + } } } @@ -948,8 +820,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { test("ADD JAR command 2") { // this is a test case from mapjoin_addjar.q - val testJar = TestHive.getHiveFile("hive-hcatalog-core-0.13.1.jar").getCanonicalPath - val testData = TestHive.getHiveFile("data/files/sample.json").getCanonicalPath + val testJar = TestHive.getHiveFile("hive-hcatalog-core-0.13.1.jar").toURI + val testData = TestHive.getHiveFile("data/files/sample.json").toURI sql(s"ADD JAR $testJar") sql( """CREATE TABLE t1(a string, b string) @@ -957,20 +829,29 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""") sql("select * from src join t1 on src.key = t1.a") sql("DROP TABLE t1") + assert(sql("list jars"). + filter(_.getString(0).contains("hive-hcatalog-core-0.13.1.jar")).count() > 0) + assert(sql("list jar"). + filter(_.getString(0).contains("hive-hcatalog-core-0.13.1.jar")).count() > 0) + val testJar2 = TestHive.getHiveFile("TestUDTF.jar").getCanonicalPath + sql(s"ADD JAR $testJar2") + assert(sql(s"list jar $testJar").count() == 1) } test("CREATE TEMPORARY FUNCTION") { - val funcJar = TestHive.getHiveFile("TestUDTF.jar").getCanonicalPath - sql(s"ADD JAR $funcJar") + val funcJar = TestHive.getHiveFile("TestUDTF.jar") + val jarURL = funcJar.toURI.toURL + sql(s"ADD JAR $jarURL") sql( """CREATE TEMPORARY FUNCTION udtf_count2 AS - | 'org.apache.spark.sql.hive.execution.GenericUDTFCount2'""".stripMargin) + |'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + """.stripMargin) assert(sql("DESCRIBE FUNCTION udtf_count2").count > 1) sql("DROP TEMPORARY FUNCTION udtf_count2") } test("ADD FILE command") { - val testFile = TestHive.getHiveFile("data/files/v1.txt").getCanonicalFile + val testFile = TestHive.getHiveFile("data/files/v1.txt").toURI sql(s"ADD FILE $testFile") val checkAddFileRDD = sparkContext.parallelize(1 to 2, 1).mapPartitions { _ => @@ -978,11 +859,13 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } assert(checkAddFileRDD.first()) + assert(sql("list files"). + filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql("list file"). + filter(_.getString(0).contains("data/files/v1.txt")).count() > 0) + assert(sql(s"list file $testFile").count() == 1) } - case class LogEntry(filename: String, message: String) - case class LogFile(name: String) - createQueryTest("dynamic_partition", """ |DROP TABLE IF EXISTS dynamic_part_table; @@ -1034,7 +917,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { .mkString("/") // Loads partition data to a temporary table to verify contents - val path = s"$warehousePath/dynamic_part_table/$partFolder/part-00000" + val warehousePathFile = new URI(sparkSession.getWarehousePath()).getPath + val path = s"$warehousePathFile/dynamic_part_table/$partFolder/part-00000" sql("DROP TABLE IF EXISTS dp_verify") sql("CREATE TABLE dp_verify(intcol INT)") @@ -1066,7 +950,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql("SET hive.exec.dynamic.partition.mode=strict") // Should throw when using strict dynamic partition mode without any static partition - intercept[SparkException] { + intercept[AnalysisException] { sql( """INSERT INTO TABLE dp_test PARTITION(dp) |SELECT key, value, key % 5 FROM src @@ -1076,7 +960,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql("SET hive.exec.dynamic.partition.mode=nonstrict") // Should throw when a static partition appears after a dynamic partition - intercept[SparkException] { + intercept[AnalysisException] { sql( """INSERT INTO TABLE dp_test PARTITION(dp, sp = 1) |SELECT key, value, key % 5 FROM src @@ -1084,9 +968,9 @@ 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") + test("SPARK-3414 regression: should store analyzed logical plan when creating a temporary view") { + sparkContext.makeRDD(Seq.empty[LogEntry]).toDF().createOrReplaceTempView("rawLogs") + sparkContext.makeRDD(Seq.empty[LogFile]).toDF().createOrReplaceTempView("logFiles") sql( """ @@ -1097,29 +981,29 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { FROM logFiles ) files ON rawLogs.filename = files.name - """).registerTempTable("boom") + """).createOrReplaceTempView("boom") // This should be successfully analyzed sql("SELECT * FROM boom").queryExecution.analyzed } - test("SPARK-3810: PreInsertionCasts static partitioning support") { + test("SPARK-3810: PreprocessTableInsertion static partitioning support") { val analyzedPlan = { loadTestTable("srcpart") sql("DROP TABLE IF EXISTS withparts") sql("CREATE TABLE withparts LIKE srcpart") sql("INSERT INTO TABLE withparts PARTITION(ds='1', hr='2') SELECT key, value FROM src") .queryExecution.analyzed - } + } assertResult(1, "Duplicated project detected\n" + analyzedPlan) { analyzedPlan.collect { - case _: Project => () - }.size + case i: InsertIntoHiveTable => i.query.collect { case p: Project => () }.size + }.sum } } - test("SPARK-3810: PreInsertionCasts dynamic partitioning support") { + test("SPARK-3810: PreprocessTableInsertion dynamic partitioning support") { val analyzedPlan = { loadTestTable("srcpart") sql("DROP TABLE IF EXISTS withparts") @@ -1127,14 +1011,14 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql("SET hive.exec.dynamic.partition.mode=nonstrict") sql("CREATE TABLE IF NOT EXISTS withparts LIKE srcpart") - sql("INSERT INTO TABLE withparts PARTITION(ds, hr) SELECT key, value FROM src") + sql("INSERT INTO TABLE withparts PARTITION(ds, hr) SELECT key, value, '1', '2' FROM src") .queryExecution.analyzed } - assertResult(1, "Duplicated project detected\n" + analyzedPlan) { + assertResult(2, "Duplicated project detected\n" + analyzedPlan) { analyzedPlan.collect { - case _: Project => () - }.size + case i: InsertIntoHiveTable => i.query.collect { case p: Project => () }.size + }.sum } } @@ -1158,51 +1042,6 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { assert(getConf(testKey, "0") == "") } - test("SET commands semantics for a HiveContext") { - // Adapted from its SQL counterpart. - val testKey = "spark.sql.key.usedfortestonly" - val testVal = "test.val.0" - val nonexistentKey = "nonexistent" - def collectResults(df: DataFrame): Set[Any] = - df.collect().map { - case Row(key: String, value: String) => key -> value - case Row(key: String, defaultValue: String, doc: String) => (key, defaultValue, doc) - }.toSet - conf.clear() - - val expectedConfs = conf.getAllDefinedConfs.toSet - assertResult(expectedConfs)(collectResults(sql("SET -v"))) - - // "SET" itself returns all config variables currently specified in SQLConf. - // TODO: Should we be listing the default here always? probably... - assert(sql("SET").collect().size === TestHiveContext.overrideConfs.size) - - val defaults = collectResults(sql("SET")) - assertResult(Set(testKey -> testVal)) { - collectResults(sql(s"SET $testKey=$testVal")) - } - - assert(hiveconf.get(testKey, "") === testVal) - assertResult(defaults ++ Set(testKey -> testVal))(collectResults(sql("SET"))) - - sql(s"SET ${testKey + testKey}=${testVal + testVal}") - assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(defaults ++ Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - collectResults(sql("SET")) - } - - // "SET key" - assertResult(Set(testKey -> testVal)) { - collectResults(sql(s"SET $testKey")) - } - - assertResult(Set(nonexistentKey -> "")) { - collectResults(sql(s"SET $nonexistentKey")) - } - - conf.clear() - } - test("current_database with multiple sessions") { sql("create database a") sql("use a") @@ -1235,12 +1074,123 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } + test("use database") { + val currentDatabase = sql("select current_database()").first().getString(0) + + sql("CREATE DATABASE hive_test_db") + sql("USE hive_test_db") + assert("hive_test_db" == sql("select current_database()").first().getString(0)) + + intercept[AnalysisException] { + sql("USE not_existing_db") + } + + sql(s"USE $currentDatabase") + assert(currentDatabase == sql("select current_database()").first().getString(0)) + } + + test("lookup hive UDF in another thread") { + val e = intercept[AnalysisException] { + range(1).selectExpr("not_a_udf()") + } + assert(e.getMessage.contains("Undefined function")) + assert(e.getMessage.contains("not_a_udf")) + var success = false + val t = new Thread("test") { + override def run(): Unit = { + val e = intercept[AnalysisException] { + range(1).selectExpr("not_a_udf()") + } + assert(e.getMessage.contains("Undefined function")) + assert(e.getMessage.contains("not_a_udf")) + success = true + } + } + t.start() + t.join() + assert(success) + } + createQueryTest("select from thrift based table", "SELECT * from src_thrift") // Put tests that depend on specific Hive settings before these last two test, // since they modify /clear stuff. + + test("role management commands are not supported") { + assertUnsupportedFeature { sql("CREATE ROLE my_role") } + assertUnsupportedFeature { sql("DROP ROLE my_role") } + assertUnsupportedFeature { sql("SHOW CURRENT ROLES") } + assertUnsupportedFeature { sql("SHOW ROLES") } + assertUnsupportedFeature { sql("SHOW GRANT") } + assertUnsupportedFeature { sql("SHOW ROLE GRANT USER my_principal") } + assertUnsupportedFeature { sql("SHOW PRINCIPALS my_role") } + assertUnsupportedFeature { sql("SET ROLE my_role") } + assertUnsupportedFeature { sql("GRANT my_role TO USER my_user") } + assertUnsupportedFeature { sql("GRANT ALL ON my_table TO USER my_user") } + assertUnsupportedFeature { sql("REVOKE my_role FROM USER my_user") } + assertUnsupportedFeature { sql("REVOKE ALL ON my_table FROM USER my_user") } + } + + test("import/export commands are not supported") { + assertUnsupportedFeature { sql("IMPORT TABLE my_table FROM 'my_path'") } + assertUnsupportedFeature { sql("EXPORT TABLE my_table TO 'my_path'") } + } + + test("some show commands are not supported") { + assertUnsupportedFeature { sql("SHOW COMPACTIONS") } + assertUnsupportedFeature { sql("SHOW TRANSACTIONS") } + assertUnsupportedFeature { sql("SHOW INDEXES ON my_table") } + assertUnsupportedFeature { sql("SHOW LOCKS my_table") } + } + + test("lock/unlock table and database commands are not supported") { + assertUnsupportedFeature { sql("LOCK TABLE my_table SHARED") } + assertUnsupportedFeature { sql("UNLOCK TABLE my_table") } + assertUnsupportedFeature { sql("LOCK DATABASE my_db SHARED") } + assertUnsupportedFeature { sql("UNLOCK DATABASE my_db") } + } + + test("create/drop/alter index commands are not supported") { + assertUnsupportedFeature { + sql("CREATE INDEX my_index ON TABLE my_table(a) as 'COMPACT' WITH DEFERRED REBUILD")} + assertUnsupportedFeature { sql("DROP INDEX my_index ON my_table") } + assertUnsupportedFeature { sql("ALTER INDEX my_index ON my_table REBUILD")} + assertUnsupportedFeature { + sql("ALTER INDEX my_index ON my_table set IDXPROPERTIES (\"prop1\"=\"val1_new\")")} + } + + test("create/drop macro commands are not supported") { + assertUnsupportedFeature { + sql("CREATE TEMPORARY MACRO SIGMOID (x DOUBLE) 1.0 / (1.0 + EXP(-x))") + } + assertUnsupportedFeature { sql("DROP TEMPORARY MACRO SIGMOID") } + } + + test("dynamic partitioning is allowed when hive.exec.dynamic.partition.mode is nonstrict") { + val modeConfKey = "hive.exec.dynamic.partition.mode" + withTable("with_parts") { + sql("CREATE TABLE with_parts(key INT) PARTITIONED BY (p INT)") + + withSQLConf(modeConfKey -> "nonstrict") { + sql("INSERT OVERWRITE TABLE with_parts partition(p) select 1, 2") + assert(spark.table("with_parts").filter($"p" === 2).collect().head == Row(1, 2)) + } + + val originalValue = spark.sparkContext.hadoopConfiguration.get(modeConfKey, "nonstrict") + try { + spark.sparkContext.hadoopConfiguration.set(modeConfKey, "nonstrict") + sql("INSERT OVERWRITE TABLE with_parts partition(p) select 3, 4") + assert(spark.table("with_parts").filter($"p" === 4).collect().head == Row(3, 4)) + } finally { + spark.sparkContext.hadoopConfiguration.set(modeConfKey, originalValue) + } + } + } } // for SPARK-2180 test case class HavingRow(key: Int, value: String, attr: Int) + +case class LogEntry(filename: String, message: String) +case class LogFile(name: String) 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 b08db6de2d2f..ce92fbf34942 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.hive.test.TestHive.{read, sparkContext, jsonRDD, sql} +import org.apache.spark.sql.hive.test.TestHive.{read, sparkContext, sql} import org.apache.spark.sql.hive.test.TestHive.implicits._ case class Nested(a: Int, B: Int) @@ -31,15 +31,15 @@ case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) class HiveResolutionSuite extends HiveComparisonTest { test("SPARK-3698: case insensitive test for nested data") { - read.json(sparkContext.makeRDD( - """{"a": [{"a": {"a": 1}}]}""" :: Nil)).registerTempTable("nested") + read.json(Seq("""{"a": [{"a": {"a": 1}}]}""").toDS()) + .createOrReplaceTempView("nested") // This should be successfully analyzed sql("SELECT a[0].A.A from nested").queryExecution.analyzed } test("SPARK-5278: check ambiguous reference to fields") { - read.json(sparkContext.makeRDD( - """{"a": [{"b": 1, "B": 2}]}""" :: Nil)).registerTempTable("nested") + read.json(Seq("""{"a": [{"b": 1, "B": 2}]}""").toDS()) + .createOrReplaceTempView("nested") // there are 2 filed matching field name "b", we should report Ambiguous reference error val exception = intercept[AnalysisException] { @@ -78,7 +78,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().createOrReplaceTempView("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"), @@ -89,14 +89,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().createOrReplaceTempView("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().createOrReplaceTempView("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/HiveSQLViewSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSQLViewSuite.scala new file mode 100644 index 000000000000..97e4c2b6b2db --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSQLViewSuite.scala @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.execution.SQLViewSuite +import org.apache.spark.sql.hive.test.{TestHive, TestHiveSingleton} +import org.apache.spark.sql.types.StructType + +/** + * A test suite for Hive view related functionality. + */ +class HiveSQLViewSuite extends SQLViewSuite with TestHiveSingleton { + import testImplicits._ + + test("create a permanent/temp view using a hive, built-in, and permanent user function") { + val permanentFuncName = "myUpper" + val permanentFuncClass = + classOf[org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper].getCanonicalName + val builtInFuncNameInLowerCase = "abs" + val builtInFuncNameInMixedCase = "aBs" + val hiveFuncName = "histogram_numeric" + + withUserDefinedFunction(permanentFuncName -> false) { + sql(s"CREATE FUNCTION $permanentFuncName AS '$permanentFuncClass'") + withTable("tab1") { + (1 to 10).map(i => (s"$i", i)).toDF("str", "id").write.saveAsTable("tab1") + Seq("VIEW", "TEMPORARY VIEW").foreach { viewMode => + withView("view1") { + sql( + s""" + |CREATE $viewMode view1 + |AS SELECT + |$permanentFuncName(str), + |$builtInFuncNameInLowerCase(id), + |$builtInFuncNameInMixedCase(id) as aBs, + |$hiveFuncName(id, 5) over() + |FROM tab1 + """.stripMargin) + checkAnswer(sql("select count(*) FROM view1"), Row(10)) + } + } + } + } + } + + test("create a permanent/temp view using a temporary function") { + val tempFunctionName = "temp" + val functionClass = + classOf[org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper].getCanonicalName + withUserDefinedFunction(tempFunctionName -> true) { + sql(s"CREATE TEMPORARY FUNCTION $tempFunctionName AS '$functionClass'") + withView("view1", "tempView1") { + withTable("tab1") { + (1 to 10).map(i => s"$i").toDF("id").write.saveAsTable("tab1") + + // temporary view + sql(s"CREATE TEMPORARY VIEW tempView1 AS SELECT $tempFunctionName(id) from tab1") + checkAnswer(sql("select count(*) FROM tempView1"), Row(10)) + + // permanent view + val e = intercept[AnalysisException] { + sql(s"CREATE VIEW view1 AS SELECT $tempFunctionName(id) from tab1") + }.getMessage + assert(e.contains("Not allowed to create a permanent view `view1` by referencing " + + s"a temporary function `$tempFunctionName`")) + } + } + } + } + + test("SPARK-14933 - create view from hive parquet table") { + withTable("t_part") { + withView("v_part") { + spark.sql("create table t_part stored as parquet as select 1 as a, 2 as b") + spark.sql("create view v_part as select * from t_part") + checkAnswer( + sql("select * from t_part"), + sql("select * from v_part")) + } + } + } + + test("SPARK-14933 - create view from hive orc table") { + withTable("t_orc") { + withView("v_orc") { + spark.sql("create table t_orc stored as orc as select 1 as a, 2 as b") + spark.sql("create view v_orc as select * from t_orc") + checkAnswer( + sql("select * from t_orc"), + sql("select * from v_orc")) + } + } + } + + test("make sure we can resolve view created by old version of Spark") { + withTable("hive_table") { + withView("old_view") { + spark.sql("CREATE TABLE hive_table AS SELECT 1 AS a, 2 AS b") + // The views defined by older versions of Spark(before 2.2) will have empty view default + // database name, and all the relations referenced in the viewText will have database part + // defined. + val view = CatalogTable( + identifier = TableIdentifier("old_view"), + tableType = CatalogTableType.VIEW, + storage = CatalogStorageFormat.empty, + schema = new StructType().add("a", "int").add("b", "int"), + viewText = Some("SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b` FROM (SELECT " + + "`gen_attr_0`, `gen_attr_1` FROM (SELECT `a` AS `gen_attr_0`, `b` AS " + + "`gen_attr_1` FROM hive_table) AS gen_subquery_0) AS hive_table") + ) + hiveContext.sessionState.catalog.createTable(view, ignoreIfExists = false) + val df = sql("SELECT * FROM old_view") + // Check the output rows. + checkAnswer(df, Row(1, 2)) + // Check the output schema. + assert(df.schema.sameType(view.schema)) + } + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index 5586a793618b..1c9f00141ae1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -17,24 +17,33 @@ package org.apache.spark.sql.hive.execution +import java.net.URI + import org.scalatest.BeforeAndAfterAll +import org.apache.spark.sql.{AnalysisException, SaveMode} +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.execution.command.{CreateTableCommand, DDLUtils} +import org.apache.spark.sql.execution.datasources.CreateTable +import org.apache.spark.sql.execution.metric.InputOutputMetricsHelper import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.types.StructType /** * A set of tests that validates support for Hive SerDe. */ -class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { +class HiveSerDeSuite extends HiveComparisonTest with PlanTest with BeforeAndAfterAll { override def beforeAll(): Unit = { import TestHive._ import org.apache.hadoop.hive.serde2.RegexSerDe - super.beforeAll() - TestHive.cacheTables = false + super.beforeAll() + TestHive.setCacheTables(false) sql(s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") """.stripMargin) - sql(s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/sales.txt")}' INTO TABLE sales") + sql(s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/sales.txt").toURI}' INTO TABLE sales") } // table sales is not a cache table, and will be clear after reset @@ -47,4 +56,139 @@ class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { createQueryTest("Read with AvroSerDe", "SELECT * FROM episodes") createQueryTest("Read Partitioned with AvroSerDe", "SELECT * FROM episodes_part") + + test("Checking metrics correctness") { + import TestHive._ + + val episodesCnt = sql("select * from episodes").count() + val episodesRes = InputOutputMetricsHelper.run(sql("select * from episodes").toDF()) + assert(episodesRes === (episodesCnt, 0L, episodesCnt) :: Nil) + + val serdeinsCnt = sql("select * from serdeins").count() + val serdeinsRes = InputOutputMetricsHelper.run(sql("select * from serdeins").toDF()) + assert(serdeinsRes === (serdeinsCnt, 0L, serdeinsCnt) :: Nil) + } + + private def extractTableDesc(sql: String): (CatalogTable, Boolean) = { + TestHive.sessionState.sqlParser.parsePlan(sql).collect { + case CreateTable(tableDesc, mode, _) => (tableDesc, mode == SaveMode.Ignore) + }.head + } + + private def analyzeCreateTable(sql: String): CatalogTable = { + TestHive.sessionState.analyzer.execute(TestHive.sessionState.sqlParser.parsePlan(sql)).collect { + case CreateTableCommand(tableDesc, _) => tableDesc + }.head + } + + test("Test the default fileformat for Hive-serde tables") { + withSQLConf("hive.default.fileformat" -> "orc") { + val (desc, exists) = extractTableDesc("CREATE TABLE IF NOT EXISTS fileformat_test (id int)") + assert(exists) + assert(desc.storage.inputFormat == Some("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat")) + assert(desc.storage.outputFormat == Some("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) + assert(desc.storage.serde == Some("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + } + + withSQLConf("hive.default.fileformat" -> "parquet") { + val (desc, exists) = extractTableDesc("CREATE TABLE IF NOT EXISTS fileformat_test (id int)") + assert(exists) + val input = desc.storage.inputFormat + val output = desc.storage.outputFormat + val serde = desc.storage.serde + assert(input == Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat")) + assert(output == Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) + assert(serde == Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + } + } + + test("create hive serde table with new syntax - basic") { + val sql = + """ + |CREATE TABLE t + |(id int, name string COMMENT 'blabla') + |USING hive + |OPTIONS (fileFormat 'parquet', my_prop 1) + |LOCATION '/tmp/file' + |COMMENT 'BLABLA' + """.stripMargin + + val table = analyzeCreateTable(sql) + assert(table.schema == new StructType() + .add("id", "int") + .add("name", "string", nullable = true, comment = "blabla")) + assert(table.provider == Some(DDLUtils.HIVE_PROVIDER)) + assert(table.storage.locationUri == Some(new URI("/tmp/file"))) + assert(table.storage.properties == Map("my_prop" -> "1")) + assert(table.comment == Some("BLABLA")) + + assert(table.storage.inputFormat == + Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat")) + assert(table.storage.outputFormat == + Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) + assert(table.storage.serde == + Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + } + + test("create hive serde table with new syntax - with partition and bucketing") { + val v1 = "CREATE TABLE t (c1 int, c2 int) USING hive PARTITIONED BY (c2)" + val table = analyzeCreateTable(v1) + assert(table.schema == new StructType().add("c1", "int").add("c2", "int")) + assert(table.partitionColumnNames == Seq("c2")) + // check the default formats + assert(table.storage.serde == Some("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe")) + assert(table.storage.inputFormat == Some("org.apache.hadoop.mapred.TextInputFormat")) + assert(table.storage.outputFormat == + Some("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat")) + + val v2 = "CREATE TABLE t (c1 int, c2 int) USING hive CLUSTERED BY (c2) INTO 4 BUCKETS" + val e2 = intercept[AnalysisException](analyzeCreateTable(v2)) + assert(e2.message.contains("Creating bucketed Hive serde table is not supported yet")) + + val v3 = + """ + |CREATE TABLE t (c1 int, c2 int) USING hive + |PARTITIONED BY (c2) + |CLUSTERED BY (c2) INTO 4 BUCKETS""".stripMargin + val e3 = intercept[AnalysisException](analyzeCreateTable(v3)) + assert(e3.message.contains("Creating bucketed Hive serde table is not supported yet")) + } + + test("create hive serde table with new syntax - Hive options error checking") { + val v1 = "CREATE TABLE t (c1 int) USING hive OPTIONS (inputFormat 'abc')" + val e1 = intercept[IllegalArgumentException](analyzeCreateTable(v1)) + assert(e1.getMessage.contains("Cannot specify only inputFormat or outputFormat")) + + val v2 = "CREATE TABLE t (c1 int) USING hive OPTIONS " + + "(fileFormat 'x', inputFormat 'a', outputFormat 'b')" + val e2 = intercept[IllegalArgumentException](analyzeCreateTable(v2)) + assert(e2.getMessage.contains( + "Cannot specify fileFormat and inputFormat/outputFormat together")) + + val v3 = "CREATE TABLE t (c1 int) USING hive OPTIONS (fileFormat 'parquet', serde 'a')" + val e3 = intercept[IllegalArgumentException](analyzeCreateTable(v3)) + assert(e3.getMessage.contains("fileFormat 'parquet' already specifies a serde")) + + val v4 = "CREATE TABLE t (c1 int) USING hive OPTIONS (serde 'a', fieldDelim ' ')" + val e4 = intercept[IllegalArgumentException](analyzeCreateTable(v4)) + assert(e4.getMessage.contains("Cannot specify delimiters with a custom serde")) + + val v5 = "CREATE TABLE t (c1 int) USING hive OPTIONS (fieldDelim ' ')" + val e5 = intercept[IllegalArgumentException](analyzeCreateTable(v5)) + assert(e5.getMessage.contains("Cannot specify delimiters without fileFormat")) + + val v6 = "CREATE TABLE t (c1 int) USING hive OPTIONS (fileFormat 'parquet', fieldDelim ' ')" + val e6 = intercept[IllegalArgumentException](analyzeCreateTable(v6)) + assert(e6.getMessage.contains( + "Cannot specify delimiters as they are only compatible with fileFormat 'textfile'")) + + // The value of 'fileFormat' option is case-insensitive. + val v7 = "CREATE TABLE t (c1 int) USING hive OPTIONS (fileFormat 'TEXTFILE', lineDelim ',')" + val e7 = intercept[IllegalArgumentException](analyzeCreateTable(v7)) + assert(e7.getMessage.contains("Hive data source only support newline '\\n' as line delimiter")) + + val v8 = "CREATE TABLE t (c1 int) USING hive OPTIONS (fileFormat 'wrong')" + val e8 = intercept[IllegalArgumentException](analyzeCreateTable(v8)) + assert(e8.getMessage.contains("invalid fileFormat: 'wrong'")) + } } 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 2209fc2f30a3..3f9bb8de42e0 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,14 +18,14 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.Row -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.{TestHive, TestHiveSingleton} import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ - +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.util.Utils -class HiveTableScanSuite extends HiveComparisonTest { +class HiveTableScanSuite extends HiveComparisonTest with SQLTestUtils with TestHiveSingleton { createQueryTest("partition_based_table_scan_with_different_serde", """ @@ -66,7 +66,7 @@ class HiveTableScanSuite extends HiveComparisonTest { TestHive.sql("DROP TABLE IF EXISTS timestamp_query_null") TestHive.sql( """ - CREATE EXTERNAL TABLE timestamp_query_null (time TIMESTAMP,id INT) + CREATE TABLE timestamp_query_null (time TIMESTAMP,id INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' LINES TERMINATED BY '\n' @@ -81,13 +81,115 @@ class HiveTableScanSuite extends HiveComparisonTest { } test("Spark-4959 Attributes are case sensitive when using a select query from a projection") { - sql("create table spark_4959 (col1 string)") - sql("""insert into table spark_4959 select "hi" from src limit 1""") - table("spark_4959").select( - 'col1.as("CaseSensitiveColName"), - 'col1.as("CaseSensitiveColName2")).registerTempTable("spark_4959_2") - - assert(sql("select CaseSensitiveColName from spark_4959_2").head() === Row("hi")) - assert(sql("select casesensitivecolname from spark_4959_2").head() === Row("hi")) + withTable("spark_4959") { + sql("create table spark_4959 (col1 string)") + sql("""insert into table spark_4959 select "hi" from src limit 1""") + table("spark_4959").select( + 'col1.as("CaseSensitiveColName"), + 'col1.as("CaseSensitiveColName2")).createOrReplaceTempView("spark_4959_2") + + assert(sql("select CaseSensitiveColName from spark_4959_2").head() === Row("hi")) + assert(sql("select casesensitivecolname from spark_4959_2").head() === Row("hi")) + } + } + + private def checkNumScannedPartitions(stmt: String, expectedNumParts: Int): Unit = { + val plan = sql(stmt).queryExecution.sparkPlan + val numPartitions = plan.collectFirst { + case p: HiveTableScanExec => p.rawPartitions.length + }.getOrElse(0) + assert(numPartitions == expectedNumParts) + } + + test("Verify SQLConf HIVE_METASTORE_PARTITION_PRUNING") { + val view = "src" + withTempView(view) { + spark.range(1, 5).createOrReplaceTempView(view) + val table = "table_with_partition" + withTable(table) { + sql( + s""" + |CREATE TABLE $table(id string) + |PARTITIONED BY (p1 string,p2 string,p3 string,p4 string,p5 string) + """.stripMargin) + sql( + s""" + |FROM $view v + |INSERT INTO TABLE $table + |PARTITION (p1='a',p2='b',p3='c',p4='d',p5='e') + |SELECT v.id + |INSERT INTO TABLE $table + |PARTITION (p1='a',p2='c',p3='c',p4='d',p5='e') + |SELECT v.id + """.stripMargin) + + Seq("true", "false").foreach { hivePruning => + withSQLConf(SQLConf.HIVE_METASTORE_PARTITION_PRUNING.key -> hivePruning) { + // If the pruning predicate is used, getHiveQlPartitions should only return the + // qualified partition; Otherwise, it return all the partitions. + val expectedNumPartitions = if (hivePruning == "true") 1 else 2 + checkNumScannedPartitions( + stmt = s"SELECT id, p2 FROM $table WHERE p2 <= 'b'", expectedNumPartitions) + } + } + + Seq("true", "false").foreach { hivePruning => + withSQLConf(SQLConf.HIVE_METASTORE_PARTITION_PRUNING.key -> hivePruning) { + // If the pruning predicate does not exist, getHiveQlPartitions should always + // return all the partitions. + checkNumScannedPartitions( + stmt = s"SELECT id, p2 FROM $table WHERE id <= 3", expectedNumParts = 2) + } + } + } + } + } + + test("SPARK-16926: number of table and partition columns match for new partitioned table") { + val view = "src" + withTempView(view) { + spark.range(1, 5).createOrReplaceTempView(view) + val table = "table_with_partition" + withTable(table) { + sql( + s""" + |CREATE TABLE $table(id string) + |PARTITIONED BY (p1 string,p2 string,p3 string,p4 string,p5 string) + """.stripMargin) + sql( + s""" + |FROM $view v + |INSERT INTO TABLE $table + |PARTITION (p1='a',p2='b',p3='c',p4='d',p5='e') + |SELECT v.id + |INSERT INTO TABLE $table + |PARTITION (p1='a',p2='c',p3='c',p4='d',p5='e') + |SELECT v.id + """.stripMargin) + val scan = getHiveTableScanExec(s"SELECT * FROM $table") + val numDataCols = scan.relation.dataCols.length + scan.rawPartitions.foreach(p => assert(p.getCols.size == numDataCols)) + } + } + } + + test("HiveTableScanExec canonicalization for different orders of partition filters") { + val table = "hive_tbl_part" + withTable(table) { + sql( + s""" + |CREATE TABLE $table (id int) + |PARTITIONED BY (a int, b int) + """.stripMargin) + val scan1 = getHiveTableScanExec(s"SELECT * FROM $table WHERE a = 1 AND b = 2") + val scan2 = getHiveTableScanExec(s"SELECT * FROM $table WHERE b = 2 AND a = 1") + assert(scan1.sameResult(scan2)) + } + } + + private def getHiveTableScanExec(query: String): HiveTableScanExec = { + sql(query).queryExecution.sparkPlan.collectFirst { + case p: HiveTableScanExec => p + }.get } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala index 197e9bfb02c4..2de429bdabb7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala @@ -18,32 +18,40 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.catalyst.expressions.{Cast, EqualTo} -import org.apache.spark.sql.execution.Project +import org.apache.spark.sql.execution.ProjectExec import org.apache.spark.sql.hive.test.TestHive /** * A set of tests that validate type promotion and coercion rules. */ class HiveTypeCoercionSuite extends HiveComparisonTest { - val baseTypes = Seq("1", "1.0", "1L", "1S", "1Y", "'1'") + val baseTypes = Seq( + ("1", "1"), + ("1.0", "CAST(1.0 AS DOUBLE)"), + ("1L", "1L"), + ("1S", "1S"), + ("1Y", "1Y"), + ("'1'", "'1'")) - baseTypes.foreach { i => - baseTypes.foreach { j => - createQueryTest(s"$i + $j", s"SELECT $i + $j FROM src LIMIT 1") + baseTypes.foreach { case (ni, si) => + baseTypes.foreach { case (nj, sj) => + createQueryTest(s"$ni + $nj", s"SELECT $si + $sj FROM src LIMIT 1") } } val nullVal = "null" - baseTypes.init.foreach { i => + baseTypes.init.foreach { case (i, s) => createQueryTest(s"case when then $i else $nullVal end ", - s"SELECT case when true then $i else $nullVal end FROM src limit 1") + s"SELECT case when true then $s else $nullVal end FROM src limit 1") createQueryTest(s"case when then $nullVal else $i end ", - s"SELECT case when true then $nullVal else $i end FROM src limit 1") + s"SELECT case when true then $nullVal else $s end FROM src limit 1") } test("[SPARK-2210] boolean cast on boolean value should be removed") { val q = "select cast(cast(key=0 as boolean) as boolean) from src" - val project = TestHive.sql(q).queryExecution.executedPlan.collect { case e: Project => e }.head + val project = TestHive.sql(q).queryExecution.sparkPlan.collect { + case e: ProjectExec => e + }.head // No cast expression introduced project.transformAllExpressions { case c: Cast => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala new file mode 100644 index 000000000000..8986fb58c646 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.hive.ql.udf.UDAFPercentile +import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, GenericUDAFEvaluator, GenericUDAFMax} +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.{AggregationBuffer, Mode} +import org.apache.hadoop.hive.ql.util.JavaDataModel +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo +import test.org.apache.spark.sql.MyDoubleAvg + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils + +class HiveUDAFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { + import testImplicits._ + + protected override def beforeAll(): Unit = { + sql(s"CREATE TEMPORARY FUNCTION mock AS '${classOf[MockUDAF].getName}'") + sql(s"CREATE TEMPORARY FUNCTION hive_max AS '${classOf[GenericUDAFMax].getName}'") + + Seq( + (0: Integer) -> "val_0", + (1: Integer) -> "val_1", + (2: Integer) -> null, + (3: Integer) -> null + ).toDF("key", "value").repartition(2).createOrReplaceTempView("t") + } + + protected override def afterAll(): Unit = { + sql(s"DROP TEMPORARY FUNCTION IF EXISTS mock") + sql(s"DROP TEMPORARY FUNCTION IF EXISTS hive_max") + } + + test("built-in Hive UDAF") { + val df = sql("SELECT key % 2, hive_max(key) FROM t GROUP BY key % 2") + + val aggs = df.queryExecution.executedPlan.collect { + case agg: ObjectHashAggregateExec => agg + } + + // There should be two aggregate operators, one for partial aggregation, and the other for + // global aggregation. + assert(aggs.length == 2) + + checkAnswer(df, Seq( + Row(0, 2), + Row(1, 3) + )) + } + + test("customized Hive UDAF") { + val df = sql("SELECT key % 2, mock(value) FROM t GROUP BY key % 2") + + val aggs = df.queryExecution.executedPlan.collect { + case agg: ObjectHashAggregateExec => agg + } + + // There should be two aggregate operators, one for partial aggregation, and the other for + // global aggregation. + assert(aggs.length == 2) + + checkAnswer(df, Seq( + Row(0, Row(1, 1)), + Row(1, Row(1, 1)) + )) + } + + test("call JAVA UDAF") { + withTempView("temp") { + withUserDefinedFunction("myDoubleAvg" -> false) { + spark.range(1, 10).toDF("value").createOrReplaceTempView("temp") + sql(s"CREATE FUNCTION myDoubleAvg AS '${classOf[MyDoubleAvg].getName}'") + checkAnswer( + spark.sql("SELECT default.myDoubleAvg(value) as my_avg from temp"), + Row(105.0)) + } + } + } + + test("non-deterministic children expressions of UDAF") { + withTempView("view1") { + spark.range(1).selectExpr("id as x", "id as y").createTempView("view1") + withUserDefinedFunction("testUDAFPercentile" -> true) { + // non-deterministic children of Hive UDAF + sql(s"CREATE TEMPORARY FUNCTION testUDAFPercentile AS '${classOf[UDAFPercentile].getName}'") + val e1 = intercept[AnalysisException] { + sql("SELECT testUDAFPercentile(x, rand()) from view1 group by y") + }.getMessage + assert(Seq("nondeterministic expression", + "should not appear in the arguments of an aggregate function").forall(e1.contains)) + } + } + } +} + +/** + * A testing Hive UDAF that computes the counts of both non-null values and nulls of a given column. + */ +class MockUDAF extends AbstractGenericUDAFResolver { + override def getEvaluator(info: Array[TypeInfo]): GenericUDAFEvaluator = new MockUDAFEvaluator +} + +class MockUDAFBuffer(var nonNullCount: Long, var nullCount: Long) + extends GenericUDAFEvaluator.AbstractAggregationBuffer { + + override def estimate(): Int = JavaDataModel.PRIMITIVES2 * 2 +} + +class MockUDAFEvaluator extends GenericUDAFEvaluator { + private val nonNullCountOI = PrimitiveObjectInspectorFactory.javaLongObjectInspector + + private val nullCountOI = PrimitiveObjectInspectorFactory.javaLongObjectInspector + + private val bufferOI = { + val fieldNames = Seq("nonNullCount", "nullCount").asJava + val fieldOIs = Seq(nonNullCountOI: ObjectInspector, nullCountOI: ObjectInspector).asJava + ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames, fieldOIs) + } + + private val nonNullCountField = bufferOI.getStructFieldRef("nonNullCount") + + private val nullCountField = bufferOI.getStructFieldRef("nullCount") + + override def getNewAggregationBuffer: AggregationBuffer = new MockUDAFBuffer(0L, 0L) + + override def reset(agg: AggregationBuffer): Unit = { + val buffer = agg.asInstanceOf[MockUDAFBuffer] + buffer.nonNullCount = 0L + buffer.nullCount = 0L + } + + override def init(mode: Mode, parameters: Array[ObjectInspector]): ObjectInspector = bufferOI + + override def iterate(agg: AggregationBuffer, parameters: Array[AnyRef]): Unit = { + val buffer = agg.asInstanceOf[MockUDAFBuffer] + if (parameters.head eq null) { + buffer.nullCount += 1L + } else { + buffer.nonNullCount += 1L + } + } + + override def merge(agg: AggregationBuffer, partial: Object): Unit = { + if (partial ne null) { + val nonNullCount = nonNullCountOI.get(bufferOI.getStructFieldData(partial, nonNullCountField)) + val nullCount = nullCountOI.get(bufferOI.getStructFieldData(partial, nullCountField)) + val buffer = agg.asInstanceOf[MockUDAFBuffer] + buffer.nonNullCount += nonNullCount + buffer.nullCount += nullCount + } + } + + override def terminatePartial(agg: AggregationBuffer): AnyRef = { + val buffer = agg.asInstanceOf[MockUDAFBuffer] + Array[Object](buffer.nonNullCount: java.lang.Long, buffer.nullCount: java.lang.Long) + } + + override def terminate(agg: AggregationBuffer): AnyRef = terminatePartial(agg) +} 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 5f9a447759b4..6198d4963df3 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 @@ -17,20 +17,25 @@ package org.apache.spark.sql.hive.execution -import java.io.{DataInput, DataOutput} +import java.io.{DataInput, DataOutput, File, PrintWriter} import java.util.{ArrayList, Arrays, Properties} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hive.ql.udf.UDAFPercentile -import org.apache.hadoop.hive.ql.udf.generic.{GenericUDFOPAnd, GenericUDTFExplode, GenericUDAFAverage, GenericUDF} +import org.apache.hadoop.hive.ql.exec.UDF +import org.apache.hadoop.hive.ql.udf.{UDAFPercentile, UDFType} +import org.apache.hadoop.hive.ql.udf.generic._ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorFactory} import org.apache.hadoop.hive.serde2.{AbstractSerDe, SerDeStats} -import org.apache.hadoop.io.Writable -import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SQLConf} -import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.io.{LongWritable, Writable} +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.functions.max +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.util.Utils case class Fields(f1: Int, f2: Int, f3: Int, f4: Int, f5: Int) @@ -44,10 +49,10 @@ case class ListStringCaseClass(l: Seq[String]) /** * A test suite for Hive custom UDFs. */ -class HiveUDFSuite extends QueryTest with TestHiveSingleton { +class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { - import hiveContext.{udf, sql} - import hiveContext.implicits._ + import spark.udf + import spark.implicits._ test("spark sql udf test that returns a struct") { udf.register("getStruct", (_: Int) => Fields(1, 2, 3, 4, 5)) @@ -69,67 +74,61 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton { } test("hive struct udf") { - sql( - """ - |CREATE EXTERNAL TABLE hiveUDFTestTable ( - | pair STRUCT - |) - |PARTITIONED BY (partition STRING) - |ROW FORMAT SERDE '%s' - |STORED AS SEQUENCEFILE - """. - stripMargin.format(classOf[PairSerDe].getName)) - - val location = Utils.getSparkClassLoader.getResource("data/files/testUDF").getFile - sql(s""" - ALTER TABLE hiveUDFTestTable - ADD IF NOT EXISTS PARTITION(partition='testUDF') - LOCATION '$location'""") - - sql(s"CREATE TEMPORARY FUNCTION testUDF AS '${classOf[PairUDF].getName}'") - sql("SELECT testUDF(pair) FROM hiveUDFTestTable") - sql("DROP TEMPORARY FUNCTION IF EXISTS testUDF") + withTable("hiveUDFTestTable") { + sql( + """ + |CREATE TABLE hiveUDFTestTable ( + | pair STRUCT + |) + |PARTITIONED BY (partition STRING) + |ROW FORMAT SERDE '%s' + |STORED AS SEQUENCEFILE + """. + stripMargin.format(classOf[PairSerDe].getName)) + + val location = Utils.getSparkClassLoader.getResource("data/files/testUDF").getFile + sql(s""" + ALTER TABLE hiveUDFTestTable + ADD IF NOT EXISTS PARTITION(partition='testUDF') + LOCATION '$location'""") + + sql(s"CREATE TEMPORARY FUNCTION testUDF AS '${classOf[PairUDF].getName}'") + sql("SELECT testUDF(pair) FROM hiveUDFTestTable") + sql("DROP TEMPORARY FUNCTION IF EXISTS testUDF") + } } test("Max/Min on named_struct") { - def testOrderInStruct(): Unit = { - checkAnswer(sql( - """ - |SELECT max(named_struct( - | "key", key, - | "value", value)).value FROM src - """.stripMargin), Seq(Row("val_498"))) - checkAnswer(sql( - """ - |SELECT min(named_struct( - | "key", key, - | "value", value)).value FROM src - """.stripMargin), Seq(Row("val_0"))) + checkAnswer(sql( + """ + |SELECT max(named_struct( + | "key", key, + | "value", value)).value FROM src + """.stripMargin), Seq(Row("val_498"))) + checkAnswer(sql( + """ + |SELECT min(named_struct( + | "key", key, + | "value", value)).value FROM src + """.stripMargin), Seq(Row("val_0"))) - // nested struct cases - checkAnswer(sql( - """ - |SELECT max(named_struct( - | "key", named_struct( - "key", key, - "value", value), - | "value", value)).value FROM src - """.stripMargin), Seq(Row("val_498"))) - checkAnswer(sql( - """ - |SELECT min(named_struct( - | "key", named_struct( - "key", key, - "value", value), - | "value", value)).value FROM src - """.stripMargin), Seq(Row("val_0"))) - } - val codegenDefault = hiveContext.getConf(SQLConf.CODEGEN_ENABLED) - hiveContext.setConf(SQLConf.CODEGEN_ENABLED, true) - testOrderInStruct() - hiveContext.setConf(SQLConf.CODEGEN_ENABLED, false) - testOrderInStruct() - hiveContext.setConf(SQLConf.CODEGEN_ENABLED, codegenDefault) + // nested struct cases + checkAnswer(sql( + """ + |SELECT max(named_struct( + | "key", named_struct( + "key", key, + "value", value), + | "value", value)).value FROM src + """.stripMargin), Seq(Row("val_498"))) + checkAnswer(sql( + """ + |SELECT min(named_struct( + | "key", named_struct( + "key", key, + "value", value), + | "value", value)).value FROM src + """.stripMargin), Seq(Row("val_0"))) } test("SPARK-6409 UDAF Average test") { @@ -149,18 +148,60 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton { sql("SELECT array(max(key), max(key)) FROM src").collect().toSeq) } + test("SPARK-16228 Percentile needs explicit cast to double") { + sql("select percentile(value, cast(0.5 as double)) from values 1,2,3 T(value)") + sql("select percentile_approx(value, cast(0.5 as double)) from values 1.0,2.0,3.0 T(value)") + sql("select percentile(value, 0.5) from values 1,2,3 T(value)") + sql("select percentile_approx(value, 0.5) from values 1.0,2.0,3.0 T(value)") + } + test("Generic UDAF aggregates") { - checkAnswer(sql("SELECT ceiling(percentile_approx(key, 0.99999)) FROM src LIMIT 1"), + + checkAnswer(sql( + """ + |SELECT percentile_approx(2, 0.99999), + | sum(distinct 1), + | count(distinct 1,2,3,4) FROM src LIMIT 1 + """.stripMargin), sql("SELECT 2, 1, 1 FROM src LIMIT 1").collect().toSeq) + + checkAnswer(sql( + """ + |SELECT ceiling(percentile_approx(distinct key, 0.99999)), + | count(distinct key), + | sum(distinct key), + | count(distinct 1), + | sum(distinct 1), + | sum(1) FROM src LIMIT 1 + """.stripMargin), + sql( + """ + |SELECT max(key), + | count(distinct key), + | sum(distinct key), + | 1, 1, sum(1) FROM src LIMIT 1 + """.stripMargin).collect().toSeq) + + checkAnswer(sql( + """ + |SELECT ceiling(percentile_approx(distinct key, 0.9 + 0.09999)), + | count(distinct key), sum(distinct key), + | count(distinct 1), sum(distinct 1), + | sum(1) FROM src LIMIT 1 + """.stripMargin), + sql("SELECT max(key), count(distinct key), sum(distinct key), 1, 1, sum(1) FROM src LIMIT 1") + .collect().toSeq) + + checkAnswer(sql("SELECT ceiling(percentile_approx(key, 0.99999D)) 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"), + checkAnswer(sql("SELECT percentile_approx(100.0D, array(0.9D, 0.9D)) FROM src LIMIT 1"), sql("SELECT array(100, 100) FROM src LIMIT 1").collect().toSeq) - } + } test("UDFIntegerToString") { - val testData = hiveContext.sparkContext.parallelize( + val testData = spark.sparkContext.parallelize( IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil).toDF() - testData.registerTempTable("integerTable") + testData.createOrReplaceTempView("integerTable") val udfName = classOf[UDFIntegerToString].getName sql(s"CREATE TEMPORARY FUNCTION testUDFIntegerToString AS '$udfName'") @@ -173,73 +214,122 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton { } test("UDFToListString") { - val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() - testData.registerTempTable("inputTable") + val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() + testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToListString AS '${classOf[UDFToListString].getName}'") - val errMsg = intercept[AnalysisException] { - sql("SELECT testUDFToListString(s) FROM inputTable") - } - assert(errMsg.getMessage contains "List type in java is unsupported because " + - "JVM type erasure makes spark fail to catch a component type in List<>;") + checkAnswer( + sql("SELECT testUDFToListString(s) FROM inputTable"), + Seq(Row(Seq("data1", "data2", "data3")))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToListString") hiveContext.reset() } test("UDFToListInt") { - val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() - testData.registerTempTable("inputTable") + val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() + testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToListInt AS '${classOf[UDFToListInt].getName}'") - val errMsg = intercept[AnalysisException] { - sql("SELECT testUDFToListInt(s) FROM inputTable") - } - assert(errMsg.getMessage contains "List type in java is unsupported because " + - "JVM type erasure makes spark fail to catch a component type in List<>;") + checkAnswer( + sql("SELECT testUDFToListInt(s) FROM inputTable"), + Seq(Row(Seq(1, 2, 3)))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToListInt") hiveContext.reset() } test("UDFToStringIntMap") { - val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() - testData.registerTempTable("inputTable") + val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() + testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToStringIntMap " + s"AS '${classOf[UDFToStringIntMap].getName}'") - val errMsg = intercept[AnalysisException] { - sql("SELECT testUDFToStringIntMap(s) FROM inputTable") - } - assert(errMsg.getMessage contains "Map type in java is unsupported because " + - "JVM type erasure makes spark fail to catch key and value types in Map<>;") + checkAnswer( + sql("SELECT testUDFToStringIntMap(s) FROM inputTable"), + Seq(Row(Map("key1" -> 1, "key2" -> 2, "key3" -> 3)))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToStringIntMap") hiveContext.reset() } test("UDFToIntIntMap") { - val testData = hiveContext.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() - testData.registerTempTable("inputTable") + val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() + testData.createOrReplaceTempView("inputTable") sql(s"CREATE TEMPORARY FUNCTION testUDFToIntIntMap " + s"AS '${classOf[UDFToIntIntMap].getName}'") - val errMsg = intercept[AnalysisException] { - sql("SELECT testUDFToIntIntMap(s) FROM inputTable") - } - assert(errMsg.getMessage contains "Map type in java is unsupported because " + - "JVM type erasure makes spark fail to catch key and value types in Map<>;") + checkAnswer( + sql("SELECT testUDFToIntIntMap(s) FROM inputTable"), + Seq(Row(Map(1 -> 1, 2 -> 1, 3 -> 1)))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToIntIntMap") hiveContext.reset() } + test("UDFToListMapStringListInt") { + val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() + testData.createOrReplaceTempView("inputTable") + + sql(s"CREATE TEMPORARY FUNCTION testUDFToListMapStringListInt " + + s"AS '${classOf[UDFToListMapStringListInt].getName}'") + checkAnswer( + sql("SELECT testUDFToListMapStringListInt(s) FROM inputTable"), + Seq(Row(Seq(Map("a" -> Seq(1, 2), "b" -> Seq(3, 4)))))) + + sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToListMapStringListInt") + hiveContext.reset() + } + + test("UDFRawList") { + val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() + testData.createOrReplaceTempView("inputTable") + + sql(s"CREATE TEMPORARY FUNCTION testUDFRawList " + + s"AS '${classOf[UDFRawList].getName}'") + val err = intercept[AnalysisException](sql("SELECT testUDFRawList(s) FROM inputTable")) + assert(err.getMessage.contains( + "Raw list type in java is unsupported because Spark cannot infer the element type.")) + + sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFRawList") + hiveContext.reset() + } + + test("UDFRawMap") { + val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() + testData.createOrReplaceTempView("inputTable") + + sql(s"CREATE TEMPORARY FUNCTION testUDFRawMap " + + s"AS '${classOf[UDFRawMap].getName}'") + val err = intercept[AnalysisException](sql("SELECT testUDFRawMap(s) FROM inputTable")) + assert(err.getMessage.contains( + "Raw map type in java is unsupported because Spark cannot infer key and value types.")) + + sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFRawMap") + hiveContext.reset() + } + + test("UDFWildcardList") { + val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() + testData.createOrReplaceTempView("inputTable") + + sql(s"CREATE TEMPORARY FUNCTION testUDFWildcardList " + + s"AS '${classOf[UDFWildcardList].getName}'") + val err = intercept[AnalysisException](sql("SELECT testUDFWildcardList(s) FROM inputTable")) + assert(err.getMessage.contains( + "Collection types with wildcards (e.g. List or Map) are unsupported " + + "because Spark cannot infer the data type for these type parameters.")) + + sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFWildcardList") + hiveContext.reset() + } + test("UDFListListInt") { - val testData = hiveContext.sparkContext.parallelize( + val testData = spark.sparkContext.parallelize( ListListIntCaseClass(Nil) :: ListListIntCaseClass(Seq((1, 2, 3))) :: ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: Nil).toDF() - testData.registerTempTable("listListIntTable") + testData.createOrReplaceTempView("listListIntTable") sql(s"CREATE TEMPORARY FUNCTION testUDFListListInt AS '${classOf[UDFListListInt].getName}'") checkAnswer( @@ -251,10 +341,10 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton { } test("UDFListString") { - val testData = hiveContext.sparkContext.parallelize( + val testData = spark.sparkContext.parallelize( ListStringCaseClass(Seq("a", "b", "c")) :: ListStringCaseClass(Seq("d", "e")) :: Nil).toDF() - testData.registerTempTable("listStringTable") + testData.createOrReplaceTempView("listStringTable") sql(s"CREATE TEMPORARY FUNCTION testUDFListString AS '${classOf[UDFListString].getName}'") checkAnswer( @@ -266,9 +356,9 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton { } test("UDFStringString") { - val testData = hiveContext.sparkContext.parallelize( + val testData = spark.sparkContext.parallelize( StringCaseClass("world") :: StringCaseClass("goodbye") :: Nil).toDF() - testData.registerTempTable("stringTable") + testData.createOrReplaceTempView("stringTable") sql(s"CREATE TEMPORARY FUNCTION testStringStringUDF AS '${classOf[UDFStringString].getName}'") checkAnswer( @@ -285,12 +375,12 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton { } test("UDFTwoListList") { - val testData = hiveContext.sparkContext.parallelize( + val testData = spark.sparkContext.parallelize( ListListIntCaseClass(Nil) :: ListListIntCaseClass(Seq((1, 2, 3))) :: ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: Nil).toDF() - testData.registerTempTable("TwoListTable") + testData.createOrReplaceTempView("TwoListTable") sql(s"CREATE TEMPORARY FUNCTION testUDFTwoListList AS '${classOf[UDFTwoListList].getName}'") checkAnswer( @@ -301,60 +391,252 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton { hiveContext.reset() } + test("non-deterministic children of UDF") { + withUserDefinedFunction("testStringStringUDF" -> true, "testGenericUDFHash" -> true) { + // HiveSimpleUDF + sql(s"CREATE TEMPORARY FUNCTION testStringStringUDF AS '${classOf[UDFStringString].getName}'") + val df1 = sql("SELECT testStringStringUDF(rand(), \"hello\")") + assert(!df1.logicalPlan.asInstanceOf[Project].projectList.forall(_.deterministic)) + + // HiveGenericUDF + sql(s"CREATE TEMPORARY FUNCTION testGenericUDFHash AS '${classOf[GenericUDFHash].getName}'") + val df2 = sql("SELECT testGenericUDFHash(rand())") + assert(!df2.logicalPlan.asInstanceOf[Project].projectList.forall(_.deterministic)) + } + } + test("Hive UDFs with insufficient number of input arguments should trigger an analysis error") { - Seq((1, 2)).toDF("a", "b").registerTempTable("testUDF") + withTempView("testUDF") { + Seq((1, 2)).toDF("a", "b").createOrReplaceTempView("testUDF") + + def testErrorMsgForFunc(funcName: String, className: String): Unit = { + withUserDefinedFunction(funcName -> true) { + sql(s"CREATE TEMPORARY FUNCTION $funcName AS '$className'") + val message = intercept[AnalysisException] { + sql(s"SELECT $funcName() FROM testUDF") + }.getMessage + assert(message.contains(s"No handler for UDF/UDAF/UDTF '$className'")) + } + } - { // HiveSimpleUDF - sql(s"CREATE TEMPORARY FUNCTION testUDFTwoListList AS '${classOf[UDFTwoListList].getName}'") - val message = intercept[AnalysisException] { - sql("SELECT testUDFTwoListList() FROM testUDF") - }.getMessage - assert(message.contains("No handler for Hive udf")) - sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFTwoListList") - } + testErrorMsgForFunc("testUDFTwoListList", classOf[UDFTwoListList].getName) - { // HiveGenericUDF - sql(s"CREATE TEMPORARY FUNCTION testUDFAnd AS '${classOf[GenericUDFOPAnd].getName}'") - val message = intercept[AnalysisException] { - sql("SELECT testUDFAnd() FROM testUDF") - }.getMessage - assert(message.contains("No handler for Hive udf")) - sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFAnd") - } + testErrorMsgForFunc("testUDFAnd", classOf[GenericUDFOPAnd].getName) - { // Hive UDAF - sql(s"CREATE TEMPORARY FUNCTION testUDAFPercentile AS '${classOf[UDAFPercentile].getName}'") - val message = intercept[AnalysisException] { - sql("SELECT testUDAFPercentile(a) FROM testUDF GROUP BY b") - }.getMessage - assert(message.contains("No handler for Hive udf")) - sql("DROP TEMPORARY FUNCTION IF EXISTS testUDAFPercentile") - } + testErrorMsgForFunc("testUDAFPercentile", classOf[UDAFPercentile].getName) + + // AbstractGenericUDAFResolver + testErrorMsgForFunc("testUDAFAverage", classOf[GenericUDAFAverage].getName) - { // AbstractGenericUDAFResolver - sql(s"CREATE TEMPORARY FUNCTION testUDAFAverage AS '${classOf[GenericUDAFAverage].getName}'") - val message = intercept[AnalysisException] { - sql("SELECT testUDAFAverage() FROM testUDF GROUP BY b") - }.getMessage - assert(message.contains("No handler for Hive udf")) - sql("DROP TEMPORARY FUNCTION IF EXISTS testUDAFAverage") + testErrorMsgForFunc("testUDTFExplode", classOf[GenericUDTFExplode].getName) } + } - { - // Hive UDTF - sql(s"CREATE TEMPORARY FUNCTION testUDTFExplode AS '${classOf[GenericUDTFExplode].getName}'") - val message = intercept[AnalysisException] { - sql("SELECT testUDTFExplode() FROM testUDF") - }.getMessage - assert(message.contains("No handler for Hive udf")) - sql("DROP TEMPORARY FUNCTION IF EXISTS testUDTFExplode") + test("Hive UDF in group by") { + withTempView("tab1") { + Seq(Tuple1(1451400761)).toDF("test_date").createOrReplaceTempView("tab1") + sql(s"CREATE TEMPORARY FUNCTION testUDFToDate AS '${classOf[GenericUDFToDate].getName}'") + val count = sql("select testUDFToDate(cast(test_date as timestamp))" + + " from tab1 group by testUDFToDate(cast(test_date as timestamp))").count() + sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToDate") + assert(count == 1) } + } + + test("SPARK-11522 select input_file_name from non-parquet table") { + + withTempDir { tempDir => + + // EXTERNAL OpenCSVSerde table pointing to LOCATION + + val file1 = new File(tempDir + "/data1") + val writer1 = new PrintWriter(file1) + writer1.write("1,2") + writer1.close() + + val file2 = new File(tempDir + "/data2") + val writer2 = new PrintWriter(file2) + writer2.write("1,2") + writer2.close() + + sql( + s"""CREATE EXTERNAL TABLE csv_table(page_id INT, impressions INT) + ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.OpenCSVSerde' + WITH SERDEPROPERTIES ( + \"separatorChar\" = \",\", + \"quoteChar\" = \"\\\"\", + \"escapeChar\" = \"\\\\\") + LOCATION '${tempDir.toURI}' + """) + + val answer1 = + sql("SELECT input_file_name() FROM csv_table").head().getString(0) + assert(answer1.contains("data1") || answer1.contains("data2")) + + val count1 = sql("SELECT input_file_name() FROM csv_table").distinct().count() + assert(count1 == 2) + sql("DROP TABLE csv_table") + + // EXTERNAL pointing to LOCATION - sqlContext.dropTempTable("testUDF") + sql( + s"""CREATE EXTERNAL TABLE external_t5 (c1 int, c2 int) + ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + LOCATION '${tempDir.toURI}' + """) + + val answer2 = + sql("SELECT input_file_name() as file FROM external_t5").head().getString(0) + assert(answer1.contains("data1") || answer1.contains("data2")) + + val count2 = sql("SELECT input_file_name() as file FROM external_t5").distinct().count + assert(count2 == 2) + sql("DROP TABLE external_t5") + } + + withTempDir { tempDir => + + // External parquet pointing to LOCATION + + val parquetLocation = s"${tempDir.toURI}/external_parquet" + sql("SELECT 1, 2").write.parquet(parquetLocation) + + sql( + s"""CREATE EXTERNAL TABLE external_parquet(c1 int, c2 int) + STORED AS PARQUET + LOCATION '$parquetLocation' + """) + + val answer3 = + sql("SELECT input_file_name() as file FROM external_parquet").head().getString(0) + assert(answer3.contains("external_parquet")) + + val count3 = sql("SELECT input_file_name() as file FROM external_parquet").distinct().count + assert(count3 == 1) + sql("DROP TABLE external_parquet") + } + + // Non-External parquet pointing to /tmp/... + sql("CREATE TABLE parquet_tmp STORED AS parquet AS SELECT 1, 2") + + val answer4 = + sql("SELECT input_file_name() as file FROM parquet_tmp").head().getString(0) + assert(answer4.contains("parquet_tmp")) + + val count4 = sql("SELECT input_file_name() as file FROM parquet_tmp").distinct().count + assert(count4 == 1) + sql("DROP TABLE parquet_tmp") + } + + test("Hive Stateful UDF") { + withUserDefinedFunction("statefulUDF" -> true, "statelessUDF" -> true) { + sql(s"CREATE TEMPORARY FUNCTION statefulUDF AS '${classOf[StatefulUDF].getName}'") + sql(s"CREATE TEMPORARY FUNCTION statelessUDF AS '${classOf[StatelessUDF].getName}'") + val testData = spark.range(10).repartition(1) + + // Expected Max(s) is 10 as statefulUDF returns the sequence number starting from 1. + checkAnswer(testData.selectExpr("statefulUDF() as s").agg(max($"s")), Row(10)) + + // Expected Max(s) is 5 as statefulUDF returns the sequence number starting from 1, + // and the data is evenly distributed into 2 partitions. + checkAnswer(testData.repartition(2) + .selectExpr("statefulUDF() as s").agg(max($"s")), Row(5)) + + // Expected Max(s) is 1, as stateless UDF is deterministic and foldable and replaced + // by constant 1 by ConstantFolding optimizer. + checkAnswer(testData.selectExpr("statelessUDF() as s").agg(max($"s")), Row(1)) + } + } + + test("Show persistent functions") { + val testData = spark.sparkContext.parallelize(StringCaseClass("") :: Nil).toDF() + withTempView("inputTable") { + testData.createOrReplaceTempView("inputTable") + withUserDefinedFunction("testUDFToListInt" -> false) { + val numFunc = spark.catalog.listFunctions().count() + sql(s"CREATE FUNCTION testUDFToListInt AS '${classOf[UDFToListInt].getName}'") + assert(spark.catalog.listFunctions().count() == numFunc + 1) + checkAnswer( + sql("SELECT testUDFToListInt(s) FROM inputTable"), + Seq(Row(Seq(1, 2, 3)))) + assert(sql("show functions").count() == numFunc + 1) + assert(spark.catalog.listFunctions().count() == numFunc + 1) + } + } + } + + test("Temp function has dots in the names") { + withUserDefinedFunction("test_avg" -> false, "`default.test_avg`" -> true) { + sql(s"CREATE FUNCTION test_avg AS '${classOf[GenericUDAFAverage].getName}'") + checkAnswer(sql("SELECT test_avg(1)"), Row(1.0)) + // temp function containing dots in the name + spark.udf.register("default.test_avg", () => { Math.random() + 2}) + assert(sql("SELECT `default.test_avg`()").head().getDouble(0) >= 2.0) + checkAnswer(sql("SELECT test_avg(1)"), Row(1.0)) + } + } + + test("Call the function registered in the not-current database") { + Seq("true", "false").foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) { + withDatabase("dAtABaSe1") { + sql("CREATE DATABASE dAtABaSe1") + withUserDefinedFunction("dAtABaSe1.test_avg" -> false) { + sql(s"CREATE FUNCTION dAtABaSe1.test_avg AS '${classOf[GenericUDAFAverage].getName}'") + checkAnswer(sql("SELECT dAtABaSe1.test_avg(1)"), Row(1.0)) + } + val message = intercept[AnalysisException] { + sql("SELECT dAtABaSe1.unknownFunc(1)") + }.getMessage + assert(message.contains("Undefined function: 'unknownFunc'") && + message.contains("nor a permanent function registered in the database 'dAtABaSe1'")) + } + } + } + } + + test("UDTF") { + withUserDefinedFunction("udtf_count2" -> true) { + sql(s"ADD JAR ${hiveContext.getHiveFile("TestUDTF.jar").getCanonicalPath}") + // The function source code can be found at: + // https://cwiki.apache.org/confluence/display/Hive/DeveloperGuide+UDTF + sql( + """ + |CREATE TEMPORARY FUNCTION udtf_count2 + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + """.stripMargin) + + checkAnswer( + sql("SELECT key, cc FROM src LATERAL VIEW udtf_count2(value) dd AS cc"), + Row(97, 500) :: Row(97, 500) :: Nil) + + checkAnswer( + sql("SELECT udtf_count2(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Nil) + } + } + + test("permanent UDTF") { + withUserDefinedFunction("udtf_count_temp" -> false) { + sql( + s""" + |CREATE FUNCTION udtf_count_temp + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + |USING JAR '${hiveContext.getHiveFile("TestUDTF.jar").toURI}' + """.stripMargin) + + checkAnswer( + sql("SELECT key, cc FROM src LATERAL VIEW udtf_count_temp(value) dd AS cc"), + Row(97, 500) :: Row(97, 500) :: Nil) + + checkAnswer( + sql("SELECT udtf_count_temp(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Nil) + } } } @@ -420,3 +702,22 @@ class PairUDF extends GenericUDF { override def getDisplayString(p1: Array[String]): String = "" } + +@UDFType(stateful = true) +class StatefulUDF extends UDF { + private val result = new LongWritable(0) + + def evaluate(): LongWritable = { + result.set(result.get() + 1) + result + } +} + +class StatelessUDF extends UDF { + private val result = new LongWritable(0) + + def evaluate(): LongWritable = { + result.set(result.get() + 1) + result + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala new file mode 100644 index 000000000000..5c248b9acd04 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import scala.language.existentials + +import org.apache.hadoop.conf.Configuration +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.StaticSQLConf._ +import org.apache.spark.sql.types._ +import org.apache.spark.tags.ExtendedHiveTest +import org.apache.spark.util.Utils + +/** + * A separate set of DDL tests that uses Hive 2.1 libraries, which behave a little differently + * from the built-in ones. + */ +@ExtendedHiveTest +class Hive_2_1_DDLSuite extends SparkFunSuite with TestHiveSingleton with BeforeAndAfterEach + with BeforeAndAfterAll { + + // Create a custom HiveExternalCatalog instance with the desired configuration. We cannot + // use SparkSession here since there's already an active on managed by the TestHive object. + private var catalog = { + val warehouse = Utils.createTempDir() + val metastore = Utils.createTempDir() + metastore.delete() + val sparkConf = new SparkConf() + .set(SparkLauncher.SPARK_MASTER, "local") + .set(WAREHOUSE_PATH.key, warehouse.toURI().toString()) + .set(CATALOG_IMPLEMENTATION.key, "hive") + .set(HiveUtils.HIVE_METASTORE_VERSION.key, "2.1") + .set(HiveUtils.HIVE_METASTORE_JARS.key, "maven") + + val hadoopConf = new Configuration() + hadoopConf.set("hive.metastore.warehouse.dir", warehouse.toURI().toString()) + hadoopConf.set("javax.jdo.option.ConnectionURL", + s"jdbc:derby:;databaseName=${metastore.getAbsolutePath()};create=true") + // These options are needed since the defaults in Hive 2.1 cause exceptions with an + // empty metastore db. + hadoopConf.set("datanucleus.schema.autoCreateAll", "true") + hadoopConf.set("hive.metastore.schema.verification", "false") + + new HiveExternalCatalog(sparkConf, hadoopConf) + } + + override def afterEach: Unit = { + catalog.listTables("default").foreach { t => + catalog.dropTable("default", t, true, false) + } + spark.sessionState.catalog.reset() + } + + override def afterAll(): Unit = { + catalog = null + } + + test("SPARK-21617: ALTER TABLE for non-compatible DataSource tables") { + testAlterTable( + "t1", + "CREATE TABLE t1 (c1 int) USING json", + StructType(Array(StructField("c1", IntegerType), StructField("c2", IntegerType))), + hiveCompatible = false) + } + + test("SPARK-21617: ALTER TABLE for Hive-compatible DataSource tables") { + testAlterTable( + "t1", + "CREATE TABLE t1 (c1 int) USING parquet", + StructType(Array(StructField("c1", IntegerType), StructField("c2", IntegerType)))) + } + + test("SPARK-21617: ALTER TABLE for Hive tables") { + testAlterTable( + "t1", + "CREATE TABLE t1 (c1 int) STORED AS parquet", + StructType(Array(StructField("c1", IntegerType), StructField("c2", IntegerType)))) + } + + test("SPARK-21617: ALTER TABLE with incompatible schema on Hive-compatible table") { + val exception = intercept[AnalysisException] { + testAlterTable( + "t1", + "CREATE TABLE t1 (c1 string) USING parquet", + StructType(Array(StructField("c2", IntegerType)))) + } + assert(exception.getMessage().contains("types incompatible with the existing columns")) + } + + private def testAlterTable( + tableName: String, + createTableStmt: String, + updatedSchema: StructType, + hiveCompatible: Boolean = true): Unit = { + spark.sql(createTableStmt) + val oldTable = spark.sessionState.catalog.externalCatalog.getTable("default", tableName) + catalog.createTable(oldTable, true) + catalog.alterTableSchema("default", tableName, updatedSchema) + + val updatedTable = catalog.getTable("default", tableName) + assert(updatedTable.schema.fieldNames === updatedSchema.fieldNames) + } + +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala new file mode 100644 index 000000000000..9eaf44c043c7 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala @@ -0,0 +1,451 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import scala.util.Random + +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax +import org.scalatest.Matchers._ + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.analysis.UnresolvedFunction +import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} +import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile +import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types._ + +class ObjectHashAggregateSuite + extends QueryTest + with SQLTestUtils + with TestHiveSingleton + with ExpressionEvalHelper { + + import testImplicits._ + + protected override def beforeAll(): Unit = { + sql(s"CREATE TEMPORARY FUNCTION hive_max AS '${classOf[GenericUDAFMax].getName}'") + } + + protected override def afterAll(): Unit = { + sql(s"DROP TEMPORARY FUNCTION IF EXISTS hive_max") + } + + test("typed_count without grouping keys") { + val df = Seq((1: Integer, 2), (null, 2), (3: Integer, 4)).toDF("a", "b") + + checkAnswer( + df.coalesce(1).select(typed_count($"a")), + Seq(Row(2)) + ) + } + + test("typed_count without grouping keys and empty input") { + val df = Seq.empty[(Integer, Int)].toDF("a", "b") + + checkAnswer( + df.coalesce(1).select(typed_count($"a")), + Seq(Row(0)) + ) + } + + test("typed_count with grouping keys") { + val df = Seq((1: Integer, 1), (null, 1), (2: Integer, 2)).toDF("a", "b") + + checkAnswer( + df.coalesce(1).groupBy($"b").agg(typed_count($"a")), + Seq( + Row(1, 1), + Row(2, 1)) + ) + } + + test("typed_count fallback to sort-based aggregation") { + withSQLConf(SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "2") { + val df = Seq( + (null, 1), + (null, 1), + (1: Integer, 1), + (2: Integer, 2), + (2: Integer, 2), + (2: Integer, 2) + ).toDF("a", "b") + + checkAnswer( + df.coalesce(1).groupBy($"b").agg(typed_count($"a")), + Seq(Row(1, 1), Row(2, 3)) + ) + } + } + + test("random input data types") { + val dataTypes = Seq( + // Integral types + ByteType, ShortType, IntegerType, LongType, + + // Fractional types + FloatType, DoubleType, + + // Decimal types + DecimalType(25, 5), DecimalType(6, 5), + + // Datetime types + DateType, TimestampType, + + // Complex types + ArrayType(IntegerType), + MapType(DoubleType, LongType), + new StructType() + .add("f1", FloatType, nullable = true) + .add("f2", ArrayType(BooleanType), nullable = true), + + // UDT + new UDT.MyDenseVectorUDT(), + + // Others + StringType, + BinaryType, NullType, BooleanType + ) + + dataTypes.sliding(2, 1).map(_.toSeq).foreach { dataTypes => + // Schema used to generate random input data. + val schemaForGenerator = StructType(dataTypes.zipWithIndex.map { + case (fieldType, index) => + StructField(s"col_$index", fieldType, nullable = true) + }) + + // Schema of the DataFrame to be tested. + val schema = StructType( + StructField("id", IntegerType, nullable = false) +: schemaForGenerator.fields + ) + + logInfo(s"Testing schema:\n${schema.treeString}") + + // Creates a DataFrame for the schema with random data. + val data = generateRandomRows(schemaForGenerator) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data, 1), schema) + val aggFunctions = schema.fieldNames.map(f => typed_count(col(f))) + + checkAnswer( + df.agg(aggFunctions.head, aggFunctions.tail: _*), + Row.fromSeq(data.map(_.toSeq).transpose.map(_.count(_ != null): Long)) + ) + + checkAnswer( + df.groupBy($"id" % 4 as 'mod).agg(aggFunctions.head, aggFunctions.tail: _*), + data.groupBy(_.getInt(0) % 4).map { case (key, value) => + key -> Row.fromSeq(value.map(_.toSeq).transpose.map(_.count(_ != null): Long)) + }.toSeq.map { + case (key, value) => Row.fromSeq(key +: value.toSeq) + } + ) + + withSQLConf(SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "5") { + checkAnswer( + df.agg(aggFunctions.head, aggFunctions.tail: _*), + Row.fromSeq(data.map(_.toSeq).transpose.map(_.count(_ != null): Long)) + ) + } + } + } + + private def percentile_approx( + column: Column, percentage: Double, isDistinct: Boolean = false): Column = { + val approxPercentile = new ApproximatePercentile(column.expr, Literal(percentage)) + Column(approxPercentile.toAggregateExpression(isDistinct)) + } + + private def typed_count(column: Column): Column = + Column(TestingTypedCount(column.expr).toAggregateExpression()) + + // Generates 50 random rows for a given schema. + private def generateRandomRows(schemaForGenerator: StructType): Seq[Row] = { + val dataGenerator = RandomDataGenerator.forType( + dataType = schemaForGenerator, + nullable = true, + new Random(System.nanoTime()) + ).getOrElse { + fail(s"Failed to create data generator for schema $schemaForGenerator") + } + + (1 to 50).map { i => + dataGenerator() match { + case row: Row => Row.fromSeq(i +: row.toSeq) + case null => Row.fromSeq(i +: Seq.fill(schemaForGenerator.length)(null)) + case other => fail( + s"Row or null is expected to be generated, " + + s"but a ${other.getClass.getCanonicalName} is generated." + ) + } + } + } + + makeRandomizedTests() + + private def makeRandomizedTests(): Unit = { + // A TypedImperativeAggregate function + val typed = percentile_approx($"c0", 0.5) + + // A Spark SQL native aggregate function with partial aggregation support that can be executed + // by the Tungsten `HashAggregateExec` + val withPartialUnsafe = max($"c1") + + // A Spark SQL native aggregate function with partial aggregation support that can only be + // executed by the Tungsten `HashAggregateExec` + val withPartialSafe = max($"c2") + + // A Spark SQL native distinct aggregate function + val withDistinct = countDistinct($"c3") + + val allAggs = Seq( + "typed" -> typed, + "with partial + unsafe" -> withPartialUnsafe, + "with partial + safe" -> withPartialSafe, + "with distinct" -> withDistinct + ) + + val builtinNumericTypes = Seq( + // Integral types + ByteType, ShortType, IntegerType, LongType, + + // Fractional types + FloatType, DoubleType + ) + + val numericTypes = builtinNumericTypes ++ Seq( + // Decimal types + DecimalType(25, 5), DecimalType(6, 5) + ) + + val dateTimeTypes = Seq(DateType, TimestampType) + + val arrayType = ArrayType(IntegerType) + + val structType = new StructType() + .add("f1", FloatType, nullable = true) + .add("f2", ArrayType(BooleanType), nullable = true) + + val mapType = MapType(DoubleType, LongType) + + val complexTypes = Seq(arrayType, mapType, structType) + + val orderedComplexType = Seq(arrayType, structType) + + val orderedTypes = numericTypes ++ dateTimeTypes ++ orderedComplexType ++ Seq( + StringType, BinaryType, NullType, BooleanType + ) + + val udt = new UDT.MyDenseVectorUDT() + + val fixedLengthTypes = builtinNumericTypes ++ Seq(BooleanType, NullType) + + val varLenTypes = complexTypes ++ Seq(StringType, BinaryType, udt) + + val varLenOrderedTypes = varLenTypes.intersect(orderedTypes) + + val allTypes = orderedTypes :+ udt + + val seed = System.nanoTime() + val random = new Random(seed) + + logInfo(s"Using random seed $seed") + + // Generates a random schema for the randomized data generator + val schema = new StructType() + .add("c0", numericTypes(random.nextInt(numericTypes.length)), nullable = true) + .add("c1", fixedLengthTypes(random.nextInt(fixedLengthTypes.length)), nullable = true) + .add("c2", varLenOrderedTypes(random.nextInt(varLenOrderedTypes.length)), nullable = true) + .add("c3", allTypes(random.nextInt(allTypes.length)), nullable = true) + + logInfo( + s"""Using the following random schema to generate all the randomized aggregation tests: + | + |${schema.treeString} + """.stripMargin + ) + + // Builds a randomly generated DataFrame + val schemaWithId = StructType(StructField("id", IntegerType, nullable = false) +: schema.fields) + val data = generateRandomRows(schema) + val df = spark.createDataFrame(spark.sparkContext.parallelize(data, 1), schemaWithId) + + // Tests all combinations of length 1 to 5 types of aggregate functions + (1 to allAggs.length) foreach { i => + allAggs.combinations(i) foreach { targetAggs => + val (names, aggs) = targetAggs.unzip + + // Tests aggregation of w/ and w/o grouping keys + Seq(true, false).foreach { withGroupingKeys => + + // Tests aggregation with empty and non-empty input rows + Seq(true, false).foreach { emptyInput => + + // Builds the aggregation to be tested according to different configurations + def doAggregation(df: DataFrame): DataFrame = { + val baseDf = if (emptyInput) { + val emptyRows = spark.sparkContext.parallelize(Seq.empty[Row], 1) + spark.createDataFrame(emptyRows, schemaWithId) + } else { + df + } + + if (withGroupingKeys) { + baseDf + .groupBy($"id" % 10 as "group") + .agg(aggs.head, aggs.tail: _*) + .orderBy("group") + } else { + baseDf.agg(aggs.head, aggs.tail: _*) + } + } + + // Currently Spark SQL doesn't support evaluating distinct aggregate function together + // with aggregate functions without partial aggregation support. + test( + s"randomized aggregation test - " + + s"${names.mkString("[", ", ", "]")} - " + + s"${if (withGroupingKeys) "with" else "without"} grouping keys - " + + s"with ${if (emptyInput) "empty" else "non-empty"} input" + ) { + var expected: Seq[Row] = null + var actual1: Seq[Row] = null + var actual2: Seq[Row] = null + + // Disables `ObjectHashAggregateExec` to obtain a standard answer + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { + val aggDf = doAggregation(df) + + if (aggs.intersect(Seq(withPartialSafe, typed)).nonEmpty) { + assert(containsSortAggregateExec(aggDf)) + assert(!containsObjectHashAggregateExec(aggDf)) + assert(!containsHashAggregateExec(aggDf)) + } else { + assert(!containsSortAggregateExec(aggDf)) + assert(!containsObjectHashAggregateExec(aggDf)) + assert(containsHashAggregateExec(aggDf)) + } + + expected = aggDf.collect().toSeq + } + + // Enables `ObjectHashAggregateExec` + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "true") { + val aggDf = doAggregation(df) + + if (aggs.contains(typed)) { + assert(!containsSortAggregateExec(aggDf)) + assert(containsObjectHashAggregateExec(aggDf)) + assert(!containsHashAggregateExec(aggDf)) + } else if (aggs.contains(withPartialSafe)) { + assert(containsSortAggregateExec(aggDf)) + assert(!containsObjectHashAggregateExec(aggDf)) + assert(!containsHashAggregateExec(aggDf)) + } else { + assert(!containsSortAggregateExec(aggDf)) + assert(!containsObjectHashAggregateExec(aggDf)) + assert(containsHashAggregateExec(aggDf)) + } + + // Disables sort-based aggregation fallback (we only generate 50 rows, so 100 is + // big enough) to obtain a result to be checked. + withSQLConf(SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "100") { + actual1 = aggDf.collect().toSeq + } + + // Enables sort-based aggregation fallback to obtain another result to be checked. + withSQLConf(SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "3") { + // Here we are not reusing `aggDf` because the physical plan in `aggDf` is + // cached and won't be re-planned using the new fallback threshold. + actual2 = doAggregation(df).collect().toSeq + } + } + + doubleSafeCheckRows(actual1, expected, 1e-4) + doubleSafeCheckRows(actual2, expected, 1e-4) + } + } + } + } + } + } + + private def containsSortAggregateExec(df: DataFrame): Boolean = { + df.queryExecution.executedPlan.collectFirst { + case _: SortAggregateExec => () + }.nonEmpty + } + + private def containsObjectHashAggregateExec(df: DataFrame): Boolean = { + df.queryExecution.executedPlan.collectFirst { + case _: ObjectHashAggregateExec => () + }.nonEmpty + } + + private def containsHashAggregateExec(df: DataFrame): Boolean = { + df.queryExecution.executedPlan.collectFirst { + case _: HashAggregateExec => () + }.nonEmpty + } + + private def doubleSafeCheckRows(actual: Seq[Row], expected: Seq[Row], tolerance: Double): Unit = { + assert(actual.length == expected.length) + actual.zip(expected).foreach { case (lhs: Row, rhs: Row) => + assert(lhs.length == rhs.length) + lhs.toSeq.zip(rhs.toSeq).foreach { + case (a: Double, b: Double) => checkResult(a, b +- tolerance, DoubleType) + case (a, b) => a == b + } + } + } + + test("SPARK-18403 Fix unsafe data false sharing issue in ObjectHashAggregateExec") { + // SPARK-18403: An unsafe data false sharing issue may trigger OOM / SIGSEGV when evaluating + // certain aggregate functions. To reproduce this issue, the following conditions must be + // met: + // + // 1. The aggregation must be evaluated using `ObjectHashAggregateExec`; + // 2. There must be an input column whose data type involves `ArrayType` or `MapType`; + // 3. Sort-based aggregation fallback must be triggered during evaluation. + withSQLConf( + SQLConf.USE_OBJECT_HASH_AGG.key -> "true", + SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "1" + ) { + checkAnswer( + Seq + .fill(2)(Tuple1(Array.empty[Int])) + .toDF("c0") + .groupBy(lit(1)) + .agg(typed_count($"c0"), max($"c0")), + Row(1, 2, Array.empty[Int]) + ) + + checkAnswer( + Seq + .fill(2)(Tuple1(Map.empty[Int, Int])) + .toDF("c0") + .groupBy(lit(1)) + .agg(typed_count($"c0"), first($"c0")), + Row(1, 2, Map.empty[Int, Int]) + ) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala new file mode 100644 index 000000000000..94384185d190 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, LogicalRelation, PruneFileSourcePartitions} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.StructType + +class PruneFileSourcePartitionsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("PruneFileSourcePartitions", Once, PruneFileSourcePartitions) :: Nil + } + + test("PruneFileSourcePartitions should not change the output of LogicalRelation") { + withTable("test") { + withTempDir { dir => + sql( + s""" + |CREATE EXTERNAL TABLE test(i int) + |PARTITIONED BY (p int) + |STORED AS parquet + |LOCATION '${dir.toURI}'""".stripMargin) + + val tableMeta = spark.sharedState.externalCatalog.getTable("default", "test") + val catalogFileIndex = new CatalogFileIndex(spark, tableMeta, 0) + + val dataSchema = StructType(tableMeta.schema.filterNot { f => + tableMeta.partitionColumnNames.contains(f.name) + }) + val relation = HadoopFsRelation( + location = catalogFileIndex, + partitionSchema = tableMeta.partitionSchema, + dataSchema = dataSchema, + bucketSpec = None, + fileFormat = new ParquetFileFormat(), + options = Map.empty)(sparkSession = spark) + + val logicalRelation = LogicalRelation(relation, tableMeta) + val query = Project(Seq('i, 'p), Filter('p === 1, logicalRelation)).analyze + + val optimized = Optimize.execute(query) + assert(optimized.missingInput.isEmpty) + } + } + } + + test("SPARK-20986 Reset table's statistics after PruneFileSourcePartitions rule") { + withTable("tbl") { + spark.range(10).selectExpr("id", "id % 3 as p").write.partitionBy("p").saveAsTable("tbl") + sql(s"ANALYZE TABLE tbl COMPUTE STATISTICS") + val tableStats = spark.sessionState.catalog.getTableMetadata(TableIdentifier("tbl")).stats + assert(tableStats.isDefined && tableStats.get.sizeInBytes > 0, "tableStats is lost") + + val df = sql("SELECT * FROM tbl WHERE p = 1") + val sizes1 = df.queryExecution.analyzed.collect { + case relation: LogicalRelation => relation.catalogTable.get.stats.get.sizeInBytes + } + assert(sizes1.size === 1, s"Size wrong for:\n ${df.queryExecution}") + assert(sizes1(0) == tableStats.get.sizeInBytes) + + val relations = df.queryExecution.optimizedPlan.collect { + case relation: LogicalRelation => relation + } + assert(relations.size === 1, s"Size wrong for:\n ${df.queryExecution}") + val size2 = relations(0).stats.sizeInBytes + assert(size2 == relations(0).catalogTable.get.stats.get.sizeInBytes) + assert(size2 < tableStats.get.sizeInBytes) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 210d56674541..cc592cf6ca62 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -21,18 +21,22 @@ import scala.collection.JavaConverters._ import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} /** * A set of test cases that validate partition and column pruning. */ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { - TestHive.cacheTables = false - // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, need to reset - // the environment to ensure all referenced tables in this suites are not cached in-memory. - // Refer to https://issues.apache.org/jira/browse/SPARK-2283 for details. - TestHive.reset() + override def beforeAll(): Unit = { + super.beforeAll() + TestHive.setCacheTables(false) + // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, + // need to reset the environment to ensure all referenced tables in this suites are + // not cached in-memory. Refer to https://issues.apache.org/jira/browse/SPARK-2283 + // for details. + TestHive.reset() + } // Column pruning tests @@ -144,13 +148,13 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { expectedScannedColumns: Seq[String], expectedPartValues: Seq[Seq[String]]): Unit = { test(s"$testCaseName - pruning test") { - val plan = new TestHive.QueryExecution(sql).executedPlan + val plan = new TestHiveQueryExecution(sql).sparkPlan val actualOutputColumns = plan.output.map(_.name) val (actualScannedColumns, actualPartValues) = plan.collect { - case p @ HiveTableScan(columns, relation, _) => + case p @ HiveTableScanExec(columns, relation, _) => val columnNames = columns.map(_.name) - val partValues = if (relation.table.isPartitioned) { - p.prunePartitions(relation.getHiveQlPartitions()).map(_.getValues) + val partValues = if (relation.isPartitioned) { + p.prunePartitions(p.rawPartitions).map(_.getValues) } else { Seq.empty } @@ -158,7 +162,12 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { }.head assert(actualOutputColumns === expectedOutputColumns, "Output columns mismatch") - assert(actualScannedColumns === expectedScannedColumns, "Scanned columns mismatch") + + // Scanned columns in `HiveTableScanExec` are generated by the `pruneFilterProject` method + // in `SparkPlanner`. This method internally uses `AttributeSet.toSeq`, in which + // the returned output columns are sorted by the names and expression ids. + assert(actualScannedColumns.sorted === expectedScannedColumns.sorted, + "Scanned columns mismatch") val actualPartitions = actualPartValues.map(_.asScala.mkString(",")).sorted val expectedPartitions = expectedPartValues.map(_.mkString(",")).sorted diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala new file mode 100644 index 000000000000..022cb7177339 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.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.hive.execution + +import org.apache.spark.sql.execution.metric.SQLMetricsTestUtils +import org.apache.spark.sql.hive.test.TestHiveSingleton + +class SQLMetricsSuite extends SQLMetricsTestUtils with TestHiveSingleton { + + test("writing data out metrics: hive") { + testMetricsNonDynamicPartition("hive", "t1") + } + + test("writing data out metrics dynamic partition: hive") { + withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) { + testMetricsDynamicPartition("hive", "hive", "t1") + } + } +} 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 af48d478953b..09c59000b3e3 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,18 +17,26 @@ package org.apache.spark.sql.hive.execution +import java.io.File +import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} +import java.util.{Locale, Set} -import scala.collection.JavaConverters._ +import com.google.common.io.Files +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.spark.TestUtils import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.{TableIdentifier, DefaultParserDialect} -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, EliminateSubQueries} -import org.apache.spark.sql.catalyst.errors.DialectException -import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, FunctionRegistry} +import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, CatalogUtils, HiveTableRelation} +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.hive.{HiveContext, HiveQLDialect, MetastoreRelation} -import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -51,13 +59,6 @@ case class Order( state: String, month: Int) -case class WindowData( - month: Int, - area: String, - product: Int) -/** A SQL Dialect for testing purpose, and it can not be nested type */ -class MyDialect extends DefaultParserDialect - /** * A collection of hive query tests where we generate the answers ourselves instead of depending on * Hive to generate them (in contrast to HiveQuerySuite). Often this is because the query is @@ -65,34 +66,55 @@ class MyDialect extends DefaultParserDialect */ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import hiveContext._ - import hiveContext.implicits._ - - test("UDTF") { - sql(s"ADD JAR ${hiveContext.getHiveFile("TestUDTF.jar").getCanonicalPath()}") - // The function source code can be found at: - // https://cwiki.apache.org/confluence/display/Hive/DeveloperGuide+UDTF - sql( - """ - |CREATE TEMPORARY FUNCTION udtf_count2 - |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' - """.stripMargin) - - checkAnswer( - sql("SELECT key, cc FROM src LATERAL VIEW udtf_count2(value) dd AS cc"), - Row(97, 500) :: Row(97, 500) :: Nil) - - checkAnswer( - sql("SELECT udtf_count2(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), - Row(3) :: Row(3) :: Nil) + import spark.implicits._ + + test("query global temp view") { + val df = Seq(1).toDF("i1") + df.createGlobalTempView("tbl1") + val global_temp_db = spark.conf.get("spark.sql.globalTempDatabase") + checkAnswer(spark.sql(s"select * from ${global_temp_db}.tbl1"), Row(1)) + spark.sql(s"drop view ${global_temp_db}.tbl1") + } + + test("non-existent global temp view") { + val global_temp_db = spark.conf.get("spark.sql.globalTempDatabase") + val message = intercept[AnalysisException] { + spark.sql(s"select * from ${global_temp_db}.nonexistentview") + }.getMessage + assert(message.contains("Table or view not found")) + } + + test("script") { + assume(TestUtils.testCommandAvailable("/bin/bash")) + assume(TestUtils.testCommandAvailable("echo | sed")) + val scriptFilePath = getTestResourcePath("test_script.sh") + val df = Seq(("x1", "y1", "z1"), ("x2", "y2", "z2")).toDF("c1", "c2", "c3") + df.createOrReplaceTempView("script_table") + val query1 = sql( + s""" + |SELECT col1 FROM (from(SELECT c1, c2, c3 FROM script_table) tempt_table + |REDUCE c1, c2, c3 USING 'bash $scriptFilePath' AS + |(col1 STRING, col2 STRING)) script_test_table""".stripMargin) + checkAnswer(query1, Row("x1_y1") :: Row("x2_y2") :: Nil) } test("SPARK-6835: udtf in lateral view") { val df = Seq((1, 1)).toDF("c1", "c2") - df.registerTempTable("table1") + df.createOrReplaceTempView("table1") val query = sql("SELECT c1, v FROM table1 LATERAL VIEW stack(3, 1, c1 + 1, c1 + 2) d AS v") checkAnswer(query, Row(1, 1) :: Row(1, 2) :: Row(1, 3) :: Nil) } + test("SPARK-13651: generator outputs shouldn't be resolved from its child's output") { + withTempView("src") { + Seq(("id1", "value1")).toDF("key", "value").createOrReplaceTempView("src") + val query = + sql("SELECT genoutput.* FROM src " + + "LATERAL VIEW explode(map('key1', 100, 'key2', 200)) genoutput AS key, value") + checkAnswer(query, Row("key1", 100) :: Row("key2", 200) :: Nil) + } + } + test("SPARK-6851: Self-joined converted parquet tables") { val orders = Seq( Order(1, "Atlas", "MTB", 234, "2015-01-07", "John D", "Pacifica", "CA", 20151), @@ -111,106 +133,202 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { Order(1, "Atlas", "MTB", 434, "2015-01-07", "John D", "Pacifica", "CA", 20151), Order(11, "Swift", "YFlikr", 137, "2015-01-23", "John D", "Hayward", "CA", 20151)) - orders.toDF.registerTempTable("orders1") - orderUpdates.toDF.registerTempTable("orderupdates1") + orders.toDF.createOrReplaceTempView("orders1") + orderUpdates.toDF.createOrReplaceTempView("orderupdates1") - sql( - """CREATE TABLE orders( - | id INT, - | make String, - | type String, - | price INT, - | pdate String, - | customer String, - | city String) - |PARTITIONED BY (state STRING, month INT) - |STORED AS PARQUET - """.stripMargin) + withTable("orders", "orderupdates") { + sql( + """CREATE TABLE orders( + | id INT, + | make String, + | type String, + | price INT, + | pdate String, + | customer String, + | city String) + |PARTITIONED BY (state STRING, month INT) + |STORED AS PARQUET + """.stripMargin) - sql( - """CREATE TABLE orderupdates( - | id INT, - | make String, - | type String, - | price INT, - | pdate String, - | customer String, - | city String) - |PARTITIONED BY (state STRING, month INT) - |STORED AS PARQUET - """.stripMargin) + sql( + """CREATE TABLE orderupdates( + | id INT, + | make String, + | type String, + | price INT, + | pdate String, + | customer String, + | city String) + |PARTITIONED BY (state STRING, month INT) + |STORED AS PARQUET + """.stripMargin) - sql("set hive.exec.dynamic.partition.mode=nonstrict") - sql("INSERT INTO TABLE orders PARTITION(state, month) SELECT * FROM orders1") - sql("INSERT INTO TABLE orderupdates PARTITION(state, month) SELECT * FROM orderupdates1") + sql("set hive.exec.dynamic.partition.mode=nonstrict") + sql("INSERT INTO TABLE orders PARTITION(state, month) SELECT * FROM orders1") + sql("INSERT INTO TABLE orderupdates PARTITION(state, month) SELECT * FROM orderupdates1") - checkAnswer( - sql( - """ - |select orders.state, orders.month - |from orders - |join ( - | select distinct orders.state,orders.month - | from orders - | join orderupdates - | on orderupdates.id = orders.id) ao - | on ao.state = orders.state and ao.month = orders.month - """.stripMargin), - (1 to 6).map(_ => Row("CA", 20151))) + checkAnswer( + sql( + """ + |select orders.state, orders.month + |from orders + |join ( + | select distinct orders.state,orders.month + | from orders + | join orderupdates + | on orderupdates.id = orders.id) ao + | on ao.state = orders.state and ao.month = orders.month + """.stripMargin), + (1 to 6).map(_ => Row("CA", 20151))) + } } test("show functions") { - val allBuiltinFunctions = - (FunctionRegistry.builtin.listFunction().toSet[String] ++ - org.apache.hadoop.hive.ql.exec.FunctionRegistry.getFunctionNames.asScala).toList.sorted - // The TestContext is shared by all the test cases, some functions may be registered before - // this, so we check that all the builtin functions are returned. + val allBuiltinFunctions = FunctionRegistry.builtin.listFunction().map(_.unquotedString) val allFunctions = sql("SHOW functions").collect().map(r => r(0)) allBuiltinFunctions.foreach { f => assert(allFunctions.contains(f)) } - checkAnswer(sql("SHOW functions abs"), Row("abs")) - checkAnswer(sql("SHOW functions 'abs'"), Row("abs")) - checkAnswer(sql("SHOW functions abc.abs"), Row("abs")) - checkAnswer(sql("SHOW functions `abc`.`abs`"), Row("abs")) - checkAnswer(sql("SHOW functions `abc`.`abs`"), Row("abs")) - checkAnswer(sql("SHOW functions `~`"), Row("~")) - checkAnswer(sql("SHOW functions `a function doens't exist`"), Nil) - checkAnswer(sql("SHOW functions `weekofyea.*`"), Row("weekofyear")) - // this probably will failed if we add more function with `sha` prefixing. - checkAnswer(sql("SHOW functions `sha.*`"), Row("sha") :: Row("sha1") :: Row("sha2") :: Nil) - } - - test("describe functions") { - // The Spark SQL built-in functions - checkExistence(sql("describe function extended upper"), true, + withTempDatabase { db => + def createFunction(names: Seq[String]): Unit = { + names.foreach { name => + sql( + s""" + |CREATE TEMPORARY FUNCTION $name + |AS '${classOf[PairUDF].getName}' + """.stripMargin) + } + } + def dropFunction(names: Seq[String]): Unit = { + names.foreach { name => + sql(s"DROP TEMPORARY FUNCTION $name") + } + } + createFunction(Seq("temp_abs", "temp_weekofyear", "temp_sha", "temp_sha1", "temp_sha2")) + + checkAnswer(sql("SHOW functions temp_abs"), Row("temp_abs")) + checkAnswer(sql("SHOW functions 'temp_abs'"), Row("temp_abs")) + checkAnswer(sql(s"SHOW functions $db.temp_abs"), Row("temp_abs")) + checkAnswer(sql(s"SHOW functions `$db`.`temp_abs`"), Row("temp_abs")) + checkAnswer(sql(s"SHOW functions `$db`.`temp_abs`"), Row("temp_abs")) + checkAnswer(sql("SHOW functions `a function doens't exist`"), Nil) + checkAnswer(sql("SHOW functions `temp_weekofyea*`"), Row("temp_weekofyear")) + + // this probably will failed if we add more function with `sha` prefixing. + checkAnswer( + sql("SHOW functions `temp_sha*`"), + List(Row("temp_sha"), Row("temp_sha1"), Row("temp_sha2"))) + + // Test '|' for alternation. + checkAnswer( + sql("SHOW functions 'temp_sha*|temp_weekofyea*'"), + List(Row("temp_sha"), Row("temp_sha1"), Row("temp_sha2"), Row("temp_weekofyear"))) + + dropFunction(Seq("temp_abs", "temp_weekofyear", "temp_sha", "temp_sha1", "temp_sha2")) + } + } + + test("describe functions - built-in functions") { + checkKeywordsExist(sql("describe function extended upper"), "Function: upper", "Class: org.apache.spark.sql.catalyst.expressions.Upper", - "Usage: upper(str) - Returns str with all characters changed to uppercase", + "Usage: upper(str) - Returns `str` with all characters changed to uppercase", "Extended Usage:", - "> SELECT upper('SparkSql')", - "'SPARKSQL'") + "Examples:", + "> SELECT upper('SparkSql');", + "SPARKSQL") - checkExistence(sql("describe functioN Upper"), true, + checkKeywordsExist(sql("describe functioN Upper"), "Function: upper", "Class: org.apache.spark.sql.catalyst.expressions.Upper", - "Usage: upper(str) - Returns str with all characters changed to uppercase") + "Usage: upper(str) - Returns `str` with all characters changed to uppercase") - checkExistence(sql("describe functioN Upper"), false, + checkKeywordsNotExist(sql("describe functioN Upper"), "Extended Usage") - checkExistence(sql("describe functioN abcadf"), true, - "Function: abcadf is not found.") + checkKeywordsExist(sql("describe functioN abcadf"), + "Function: abcadf not found.") - checkExistence(sql("describe functioN `~`"), true, + checkKeywordsExist(sql("describe functioN `~`"), "Function: ~", - "Class: org.apache.hadoop.hive.ql.udf.UDFOPBitNot", - "Usage: ~ n - Bitwise not") + "Class: org.apache.spark.sql.catalyst.expressions.BitwiseNot", + "Usage: ~ expr - Returns the result of bitwise NOT of `expr`.") + + // Hard coded describe functions + checkKeywordsExist(sql("describe function `<>`"), + "Function: <>", + "Usage: expr1 <> expr2 - Returns true if `expr1` is not equal to `expr2`") + + checkKeywordsExist(sql("describe function `!=`"), + "Function: !=", + "Usage: expr1 != expr2 - Returns true if `expr1` is not equal to `expr2`") + + checkKeywordsExist(sql("describe function `between`"), + "Function: between", + "Usage: expr1 [NOT] BETWEEN expr2 AND expr3 - " + + "evaluate if `expr1` is [not] in between `expr2` and `expr3`") + + checkKeywordsExist(sql("describe function `case`"), + "Function: case", + "Usage: CASE expr1 WHEN expr2 THEN expr3 " + + "[WHEN expr4 THEN expr5]* [ELSE expr6] END - " + + "When `expr1` = `expr2`, returns `expr3`; " + + "when `expr1` = `expr4`, return `expr5`; else return `expr6`") + } + + test("describe functions - user defined functions") { + withUserDefinedFunction("udtf_count" -> false) { + sql( + s""" + |CREATE FUNCTION udtf_count + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + |USING JAR '${hiveContext.getHiveFile("TestUDTF.jar").toURI}' + """.stripMargin) + + checkKeywordsExist(sql("describe function udtf_count"), + "Function: default.udtf_count", + "Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A") + + checkAnswer( + sql("SELECT udtf_count(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Nil) + + checkKeywordsExist(sql("describe function udtf_count"), + "Function: default.udtf_count", + "Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A") + } + } + + test("describe functions - temporary user defined functions") { + withUserDefinedFunction("udtf_count_temp" -> true) { + sql( + s""" + |CREATE TEMPORARY FUNCTION udtf_count_temp + |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + |USING JAR '${hiveContext.getHiveFile("TestUDTF.jar").toURI}' + """.stripMargin) + + checkKeywordsExist(sql("describe function udtf_count_temp"), + "Function: udtf_count_temp", + "Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A") + + checkAnswer( + sql("SELECT udtf_count_temp(a) FROM (SELECT 1 AS a FROM src LIMIT 3) t"), + Row(3) :: Row(3) :: Nil) + + checkKeywordsExist(sql("describe function udtf_count_temp"), + "Function: udtf_count_temp", + "Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2", + "Usage: N/A") + } } test("SPARK-5371: union with null and sum") { val df = Seq((1, 1)).toDF("c1", "c2") - df.registerTempTable("table1") + df.createOrReplaceTempView("table1") val query = sql( """ @@ -233,28 +351,47 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("CTAS with WITH clause") { - val df = Seq((1, 1)).toDF("c1", "c2") - df.registerTempTable("table1") - sql( - """ - |CREATE TABLE with_table1 AS - |WITH T AS ( - | SELECT * - | FROM table1 - |) - |SELECT * - |FROM T - """.stripMargin) - val query = sql("SELECT * FROM with_table1") - checkAnswer(query, Row(1, 1) :: Nil) + val df = Seq((1, 1)).toDF("c1", "c2") + df.createOrReplaceTempView("table1") + withTable("with_table1") { + sql( + """ + |CREATE TABLE with_table1 AS + |WITH T AS ( + | SELECT * + | FROM table1 + |) + |SELECT * + |FROM T + """.stripMargin) + val query = sql("SELECT * FROM with_table1") + checkAnswer(query, Row(1, 1) :: Nil) + } } test("explode nested Field") { - Seq(NestedArray1(NestedArray2(Seq(1, 2, 3)))).toDF.registerTempTable("nestedArray") + Seq(NestedArray1(NestedArray2(Seq(1, 2, 3)))).toDF.createOrReplaceTempView("nestedArray") checkAnswer( sql("SELECT ints FROM nestedArray LATERAL VIEW explode(a.b) a AS ints"), Row(1) :: Row(2) :: Row(3) :: Nil) + + checkAnswer( + sql("SELECT `ints` FROM nestedArray LATERAL VIEW explode(a.b) `a` AS `ints`"), + Row(1) :: Row(2) :: Row(3) :: Nil) + + checkAnswer( + sql("SELECT `a`.`ints` FROM nestedArray LATERAL VIEW explode(a.b) `a` AS `ints`"), + Row(1) :: Row(2) :: Row(3) :: Nil) + + checkAnswer( + sql( + """ + |SELECT `weird``tab`.`weird``col` + |FROM nestedArray + |LATERAL VIEW explode(a.b) `weird``tab` AS `weird``col` + """.stripMargin), + Row(1) :: Row(2) :: Row(3) :: Nil) } test("SPARK-4512 Fix attribute reference resolution error when using SORT BY") { @@ -264,217 +401,290 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { ) } - test("CTAS without serde") { - def checkRelation(tableName: String, isDataSourceParquet: Boolean): Unit = { - val relation = EliminateSubQueries(catalog.lookupRelation(TableIdentifier(tableName))) - relation match { - case LogicalRelation(r: ParquetRelation, _) => - if (!isDataSourceParquet) { - fail( - s"${classOf[MetastoreRelation].getCanonicalName} is expected, but found " + - s"${ParquetRelation.getClass.getCanonicalName}.") - } + def checkRelation( + tableName: String, + isDataSourceTable: Boolean, + format: String, + userSpecifiedLocation: Option[String] = None): Unit = { + var relation: LogicalPlan = null + withSQLConf( + HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false", + HiveUtils.CONVERT_METASTORE_ORC.key -> "false") { + relation = EliminateSubqueryAliases(spark.table(tableName).queryExecution.analyzed) + } + val catalogTable = + sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) + relation match { + case LogicalRelation(r: HadoopFsRelation, _, _, _) => + if (!isDataSourceTable) { + fail( + s"${classOf[HiveTableRelation].getCanonicalName} is expected, but found " + + s"${HadoopFsRelation.getClass.getCanonicalName}.") + } + userSpecifiedLocation match { + case Some(location) => + assert(r.options("path") === location) + case None => // OK. + } + assert(catalogTable.provider.get === format) - case r: MetastoreRelation => - if (isDataSourceParquet) { - fail( - s"${ParquetRelation.getClass.getCanonicalName} is expected, but found " + - s"${classOf[MetastoreRelation].getCanonicalName}.") - } - } + case r: HiveTableRelation => + if (isDataSourceTable) { + fail( + s"${HadoopFsRelation.getClass.getCanonicalName} is expected, but found " + + s"${classOf[HiveTableRelation].getCanonicalName}.") + } + userSpecifiedLocation match { + case Some(location) => + assert(r.tableMeta.location === CatalogUtils.stringToURI(location)) + case None => // OK. + } + // Also make sure that the format and serde are as desired. + assert(catalogTable.storage.inputFormat.get.toLowerCase(Locale.ROOT).contains(format)) + assert(catalogTable.storage.outputFormat.get.toLowerCase(Locale.ROOT).contains(format)) + val serde = catalogTable.storage.serde.get + format match { + case "sequence" | "text" => assert(serde.contains("LazySimpleSerDe")) + case "rcfile" => assert(serde.contains("LazyBinaryColumnarSerDe")) + case _ => assert(serde.toLowerCase(Locale.ROOT).contains(format)) + } + } + + // When a user-specified location is defined, the table type needs to be EXTERNAL. + val actualTableType = catalogTable.tableType + userSpecifiedLocation match { + case Some(location) => + assert(actualTableType === CatalogTableType.EXTERNAL) + case None => + assert(actualTableType === CatalogTableType.MANAGED) } + } - val originalConf = convertCTAS + test("CTAS without serde without location") { + val originalConf = sessionState.conf.convertCTAS - setConf(HiveContext.CONVERT_CTAS, true) + setConf(SQLConf.CONVERT_CTAS, true) + val defaultDataSource = sessionState.conf.defaultDataSourceName try { 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] { + val message = intercept[AnalysisException] { sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value") }.getMessage - assert(message.contains("ctas1 already exists")) - checkRelation("ctas1", true) + assert(message.contains("already exists")) + checkRelation("ctas1", true, defaultDataSource) 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 default.ctas1 AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", true, defaultDataSource) + sql("DROP TABLE ctas1") sql("CREATE TABLE ctas1 stored as textfile" + " AS SELECT key k, value FROM src ORDER BY k, value") - checkRelation("ctas1", true) + checkRelation("ctas1", false, "text") 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) + checkRelation("ctas1", false, "sequence") 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) + checkRelation("ctas1", false, "rcfile") 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) + checkRelation("ctas1", false, "orc") 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) + checkRelation("ctas1", false, "parquet") sql("DROP TABLE ctas1") } finally { - setConf(HiveContext.CONVERT_CTAS, originalConf) + setConf(SQLConf.CONVERT_CTAS, originalConf) sql("DROP TABLE IF EXISTS ctas1") } } - test("SQL dialect at the start of HiveContext") { - val hiveContext = new HiveContext(sqlContext.sparkContext) - val dialectConf = "spark.sql.dialect" - checkAnswer(hiveContext.sql(s"set $dialectConf"), Row(dialectConf, "hiveql")) - assert(hiveContext.getSQLDialect().getClass === classOf[HiveQLDialect]) + test("CTAS with default fileformat") { + val table = "ctas1" + val ctas = s"CREATE TABLE IF NOT EXISTS $table SELECT key k, value FROM src" + withSQLConf(SQLConf.CONVERT_CTAS.key -> "true") { + withSQLConf("hive.default.fileformat" -> "textfile") { + withTable(table) { + sql(ctas) + // We should use parquet here as that is the default datasource fileformat. The default + // datasource file format is controlled by `spark.sql.sources.default` configuration. + // This testcase verifies that setting `hive.default.fileformat` has no impact on + // the target table's fileformat in case of CTAS. + assert(sessionState.conf.defaultDataSourceName === "parquet") + checkRelation(tableName = table, isDataSourceTable = true, format = "parquet") + } + } + withSQLConf("spark.sql.sources.default" -> "orc") { + withTable(table) { + sql(ctas) + checkRelation(tableName = table, isDataSourceTable = true, format = "orc") + } + } + } } - test("SQL Dialect Switching") { - assert(getSQLDialect().getClass === classOf[HiveQLDialect]) - setConf("spark.sql.dialect", classOf[MyDialect].getCanonicalName()) - assert(getSQLDialect().getClass === classOf[MyDialect]) - assert(sql("SELECT 1").collect() === Array(Row(1))) + test("CTAS without serde with location") { + withSQLConf(SQLConf.CONVERT_CTAS.key -> "true") { + withTempDir { dir => + val defaultDataSource = sessionState.conf.defaultDataSourceName - // set the dialect back to the DefaultSQLDialect - sql("SET spark.sql.dialect=sql") - assert(getSQLDialect().getClass === classOf[DefaultParserDialect]) - sql("SET spark.sql.dialect=hiveql") - assert(getSQLDialect().getClass === classOf[HiveQLDialect]) + val tempLocation = dir.toURI.getPath.stripSuffix("/") + sql(s"CREATE TABLE ctas1 LOCATION 'file:$tempLocation/c1'" + + " AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", true, defaultDataSource, Some(s"file:$tempLocation/c1")) + sql("DROP TABLE ctas1") - // set invalid dialect - sql("SET spark.sql.dialect.abc=MyTestClass") - sql("SET spark.sql.dialect=abc") - intercept[Exception] { - sql("SELECT 1") - } - // test if the dialect set back to HiveQLDialect - getSQLDialect().getClass === classOf[HiveQLDialect] + sql(s"CREATE TABLE ctas1 LOCATION 'file:$tempLocation/c2'" + + " AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", true, defaultDataSource, Some(s"file:$tempLocation/c2")) + sql("DROP TABLE ctas1") - sql("SET spark.sql.dialect=MyTestClass") - intercept[DialectException] { - sql("SELECT 1") + sql(s"CREATE TABLE ctas1 stored as textfile LOCATION 'file:$tempLocation/c3'" + + " AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", false, "text", Some(s"file:$tempLocation/c3")) + sql("DROP TABLE ctas1") + + sql(s"CREATE TABLE ctas1 stored as sequenceFile LOCATION 'file:$tempLocation/c4'" + + " AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", false, "sequence", Some(s"file:$tempLocation/c4")) + sql("DROP TABLE ctas1") + + sql(s"CREATE TABLE ctas1 stored as rcfile LOCATION 'file:$tempLocation/c5'" + + " AS SELECT key k, value FROM src ORDER BY k, value") + checkRelation("ctas1", false, "rcfile", Some(s"file:$tempLocation/c5")) + sql("DROP TABLE ctas1") + } } - // test if the dialect set back to HiveQLDialect - assert(getSQLDialect().getClass === classOf[HiveQLDialect]) } test("CTAS with serde") { - sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value").collect() - sql( - """CREATE TABLE ctas2 - | ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" - | WITH SERDEPROPERTIES("serde_p1"="p1","serde_p2"="p2") - | STORED AS RCFile - | TBLPROPERTIES("tbl_p1"="p11", "tbl_p2"="p22") - | AS - | SELECT key, value - | FROM src - | ORDER BY key, value""".stripMargin).collect() - sql( - """CREATE TABLE ctas3 - | ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' LINES TERMINATED BY '\012' - | STORED AS textfile AS - | SELECT key, value - | FROM src - | ORDER BY key, value""".stripMargin).collect() - - // the table schema may like (key: integer, value: string) - sql( - """CREATE TABLE IF NOT EXISTS ctas4 AS - | SELECT 1 AS key, value FROM src LIMIT 1""".stripMargin).collect() - // do nothing cause the table ctas4 already existed. - sql( - """CREATE TABLE IF NOT EXISTS ctas4 AS - | SELECT key, value FROM src ORDER BY key, value""".stripMargin).collect() + withTable("ctas1", "ctas2", "ctas3", "ctas4", "ctas5") { + sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value") + sql( + """CREATE TABLE ctas2 + | ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" + | WITH SERDEPROPERTIES("serde_p1"="p1","serde_p2"="p2") + | STORED AS RCFile + | TBLPROPERTIES("tbl_p1"="p11", "tbl_p2"="p22") + | AS + | SELECT key, value + | FROM src + | ORDER BY key, value""".stripMargin) + + val storageCtas2 = spark.sessionState.catalog. + getTableMetadata(TableIdentifier("ctas2")).storage + assert(storageCtas2.inputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileInputFormat")) + assert(storageCtas2.outputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileOutputFormat")) + assert(storageCtas2.serde == Some("org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe")) - checkAnswer( - sql("SELECT k, value FROM ctas1 ORDER BY k, value"), - sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq) - checkAnswer( - sql("SELECT key, value FROM ctas2 ORDER BY key, value"), sql( - """ + """CREATE TABLE ctas3 + | ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' LINES TERMINATED BY '\012' + | STORED AS textfile AS + | SELECT key, value + | FROM src + | ORDER BY key, value""".stripMargin) + + // the table schema may like (key: integer, value: string) + sql( + """CREATE TABLE IF NOT EXISTS ctas4 AS + | SELECT 1 AS key, value FROM src LIMIT 1""".stripMargin) + // do nothing cause the table ctas4 already existed. + sql( + """CREATE TABLE IF NOT EXISTS ctas4 AS + | SELECT key, value FROM src ORDER BY key, value""".stripMargin) + + checkAnswer( + sql("SELECT k, value FROM ctas1 ORDER BY k, value"), + sql("SELECT key, value FROM src ORDER BY key, value")) + checkAnswer( + sql("SELECT key, value FROM ctas2 ORDER BY key, value"), + sql( + """ SELECT key, value FROM src - ORDER BY key, value""").collect().toSeq) - checkAnswer( - sql("SELECT key, value FROM ctas3 ORDER BY key, value"), - sql( - """ + ORDER BY key, value""")) + checkAnswer( + sql("SELECT key, value FROM ctas3 ORDER BY key, value"), + sql( + """ SELECT key, value FROM src - ORDER BY key, value""").collect().toSeq) - intercept[AnalysisException] { - sql( - """CREATE TABLE ctas4 AS - | SELECT key, value FROM src ORDER BY key, value""".stripMargin).collect() - } - checkAnswer( - sql("SELECT key, value FROM ctas4 ORDER BY key, value"), - sql("SELECT key, value FROM ctas4 LIMIT 1").collect().toSeq) - - checkExistence(sql("DESC EXTENDED ctas2"), true, - "name:key", "type:string", "name:value", "ctas2", - "org.apache.hadoop.hive.ql.io.RCFileInputFormat", - "org.apache.hadoop.hive.ql.io.RCFileOutputFormat", - "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe", - "serde_p1=p1", "serde_p2=p2", "tbl_p1=p11", "tbl_p2=p22", "MANAGED_TABLE" - ) - - sql( - """CREATE TABLE ctas5 - | STORED AS parquet AS - | SELECT key, value - | FROM src - | ORDER BY key, value""".stripMargin).collect() - - withSQLConf(HiveContext.CONVERT_METASTORE_PARQUET.key -> "false") { - checkExistence(sql("DESC EXTENDED ctas5"), true, - "name:key", "type:string", "name:value", "ctas5", - "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat", - "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat", - "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe", - "MANAGED_TABLE" - ) - } - - // use the Hive SerDe for parquet tables - withSQLConf(HiveContext.CONVERT_METASTORE_PARQUET.key -> "false") { + ORDER BY key, value""")) + intercept[AnalysisException] { + sql( + """CREATE TABLE ctas4 AS + | SELECT key, value FROM src ORDER BY key, value""".stripMargin) + } checkAnswer( - sql("SELECT key, value FROM ctas5 ORDER BY key, value"), - sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq) + sql("SELECT key, value FROM ctas4 ORDER BY key, value"), + sql("SELECT key, value FROM ctas4 LIMIT 1").collect().toSeq) + + sql( + """CREATE TABLE ctas5 + | STORED AS parquet AS + | SELECT key, value + | FROM src + | ORDER BY key, value""".stripMargin) + val storageCtas5 = spark.sessionState.catalog. + getTableMetadata(TableIdentifier("ctas5")).storage + assert(storageCtas5.inputFormat == + Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat")) + assert(storageCtas5.outputFormat == + Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) + assert(storageCtas5.serde == + Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + + + // use the Hive SerDe for parquet tables + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") { + checkAnswer( + sql("SELECT key, value FROM ctas5 ORDER BY key, value"), + sql("SELECT key, value FROM src ORDER BY key, value")) + } } } test("specifying the column list for CTAS") { - Seq((1, "111111"), (2, "222222")).toDF("key", "value").registerTempTable("mytable1") - - sql("create table gen__tmp(a int, b string) as select key, value from mytable1") - checkAnswer( - sql("SELECT a, b from gen__tmp"), - sql("select key, value from mytable1").collect()) - sql("DROP TABLE gen__tmp") + withTempView("mytable1") { + Seq((1, "111111"), (2, "222222")).toDF("key", "value").createOrReplaceTempView("mytable1") + withTable("gen__tmp") { + sql("create table gen__tmp as select key as a, value as b from mytable1") + checkAnswer( + sql("SELECT a, b from gen__tmp"), + sql("select key, value from mytable1").collect()) + } - sql("create table gen__tmp(a double, b double) as select key, value from mytable1") - checkAnswer( - sql("SELECT a, b from gen__tmp"), - sql("select cast(key as double), cast(value as double) from mytable1").collect()) - sql("DROP TABLE gen__tmp") + withTable("gen__tmp") { + val e = intercept[AnalysisException] { + sql("create table gen__tmp(a int, b string) as select key, value from mytable1") + }.getMessage + assert(e.contains("Schema may not be specified in a Create Table As Select (CTAS)")) + } - sql("drop table mytable1") + withTable("gen__tmp") { + val e = intercept[AnalysisException] { + sql( + """ + |CREATE TABLE gen__tmp + |PARTITIONED BY (key string) + |AS SELECT key, value FROM mytable1 + """.stripMargin) + }.getMessage + assert(e.contains("A Create Table As Select (CTAS) statement is not allowed to " + + "create a partitioned table using Hive's file formats")) + } + } } test("command substitution") { @@ -483,13 +693,13 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sql("SELECT key FROM ${hiveconf:tbl} ORDER BY key, value limit 1"), sql("SELECT key FROM src ORDER BY key, value limit 1").collect().toSeq) - sql("set hive.variable.substitute=false") // disable the substitution + sql("set spark.sql.variable.substitute=false") // disable the substitution sql("set tbl2=src") intercept[Exception] { sql("SELECT key FROM ${hiveconf:tbl2} ORDER BY key, value limit 1").collect() } - sql("set hive.variable.substitute=true") // enable the substitution + sql("set spark.sql.variable.substitute=true") // enable the substitution checkAnswer( sql("SELECT key FROM ${hiveconf:tbl2} ORDER BY key, value limit 1"), sql("SELECT key FROM src ORDER BY key, value limit 1").collect().toSeq) @@ -514,40 +724,46 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("double nested data") { - sparkContext.parallelize(Nested1(Nested2(Nested3(1))) :: Nil) - .toDF().registerTempTable("nested") - checkAnswer( - sql("SELECT f1.f2.f3 FROM nested"), - Row(1)) + withTable("test_ctas_1234") { + sparkContext.parallelize(Nested1(Nested2(Nested3(1))) :: Nil) + .toDF().createOrReplaceTempView("nested") + checkAnswer( + sql("SELECT f1.f2.f3 FROM nested"), + Row(1)) - sql("CREATE TABLE test_ctas_1234 AS SELECT * from nested") - checkAnswer( - sql("SELECT * FROM test_ctas_1234"), - sql("SELECT * FROM nested").collect().toSeq) + sql("CREATE TABLE test_ctas_1234 AS SELECT * from nested") + checkAnswer( + sql("SELECT * FROM test_ctas_1234"), + sql("SELECT * FROM nested").collect().toSeq) - intercept[AnalysisException] { - sql("CREATE TABLE test_ctas_1234 AS SELECT * from notexists").collect() + intercept[AnalysisException] { + sql("CREATE TABLE test_ctas_1234 AS SELECT * from notexists").collect() + } } } test("test CTAS") { - sql("CREATE TABLE test_ctas_123 AS SELECT key, value FROM src") - checkAnswer( - sql("SELECT key, value FROM test_ctas_123 ORDER BY key"), - sql("SELECT key, value FROM src ORDER BY key").collect().toSeq) + withTable("test_ctas_1234") { + sql("CREATE TABLE test_ctas_123 AS SELECT key, value FROM src") + checkAnswer( + sql("SELECT key, value FROM test_ctas_123 ORDER BY key"), + sql("SELECT key, value FROM src ORDER BY key").collect().toSeq) + } } test("SPARK-4825 save join to table") { - val testData = sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)).toDF() - sql("CREATE TABLE test1 (key INT, value STRING)") - testData.write.mode(SaveMode.Append).insertInto("test1") - sql("CREATE TABLE test2 (key INT, value STRING)") - testData.write.mode(SaveMode.Append).insertInto("test2") - testData.write.mode(SaveMode.Append).insertInto("test2") - sql("CREATE TABLE test AS SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key") - checkAnswer( - table("test"), - sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").collect().toSeq) + withTable("test1", "test2", "test") { + val testData = sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)).toDF() + sql("CREATE TABLE test1 (key INT, value STRING)") + testData.write.mode(SaveMode.Append).insertInto("test1") + sql("CREATE TABLE test2 (key INT, value STRING)") + testData.write.mode(SaveMode.Append).insertInto("test2") + testData.write.mode(SaveMode.Append).insertInto("test2") + sql("CREATE TABLE test AS SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key") + checkAnswer( + table("test"), + sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").collect().toSeq) + } } test("SPARK-3708 Backticks aren't handled correctly is aliases") { @@ -599,7 +815,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-4963 DataFrame sample on mutable row return wrong result") { sql("SELECT * FROM src WHERE key % 2 = 0") .sample(withReplacement = false, fraction = 0.3) - .registerTempTable("sampled") + .createOrReplaceTempView("sampled") (1 to 10).foreach { i => checkAnswer( sql("SELECT * FROM sampled WHERE key % 2 = 1"), @@ -607,7 +823,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - test("SPARK-4699 HiveContext should be case insensitive by default") { + test("SPARK-4699 SparkSession with Hive Support should be case insensitive by default") { checkAnswer( sql("SELECT KEY FROM Src ORDER BY value"), sql("SELECT key FROM src ORDER BY value").collect().toSeq) @@ -624,7 +840,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { val rowRdd = sparkContext.parallelize(row :: Nil) - hiveContext.createDataFrame(rowRdd, schema).registerTempTable("testTable") + spark.createDataFrame(rowRdd, schema).createOrReplaceTempView("testTable") sql( """CREATE TABLE nullValuesInInnerComplexTypes @@ -649,30 +865,30 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("SPARK-4296 Grouping field with Hive UDF as sub expression") { - val rdd = sparkContext.makeRDD( """{"a": "str", "b":"1", "c":"1970-01-01 00:00:00"}""" :: Nil) - read.json(rdd).registerTempTable("data") + val ds = Seq("""{"a": "str", "b":"1", "c":"1970-01-01 00:00:00"}""").toDS() + read.json(ds).createOrReplaceTempView("data") checkAnswer( sql("SELECT concat(a, '-', b), year(c) FROM data GROUP BY concat(a, '-', b), year(c)"), Row("str-1", 1970)) dropTempTable("data") - read.json(rdd).registerTempTable("data") + read.json(ds).createOrReplaceTempView("data") checkAnswer(sql("SELECT year(c) + 1 FROM data GROUP BY year(c) + 1"), Row(1971)) dropTempTable("data") } test("resolve udtf in projection #1") { - val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) - read.json(rdd).registerTempTable("data") + val ds = (1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""").toDS() + read.json(ds).createOrReplaceTempView("data") val df = sql("SELECT explode(a) AS val FROM data") val col = df("val") } test("resolve udtf in projection #2") { - val rdd = sparkContext.makeRDD((1 to 2).map(i => s"""{"a":[$i, ${i + 1}]}""")) - read.json(rdd).registerTempTable("data") + val ds = (1 to 2).map(i => s"""{"a":[$i, ${i + 1}]}""").toDS() + read.json(ds).createOrReplaceTempView("data") checkAnswer(sql("SELECT explode(map(1, 1)) FROM data LIMIT 1"), Row(1, 1) :: Nil) checkAnswer(sql("SELECT explode(map(1, 1)) as (k1, k2) FROM data LIMIT 1"), Row(1, 1) :: Nil) intercept[AnalysisException] { @@ -686,8 +902,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { // TGF with non-TGF in project is allowed in Spark SQL, but not in Hive test("TGF with non-TGF in projection") { - val rdd = sparkContext.makeRDD( """{"a": "1", "b":"1"}""" :: Nil) - read.json(rdd).registerTempTable("data") + val ds = Seq("""{"a": "1", "b":"1"}""").toDS() + read.json(ds).createOrReplaceTempView("data") checkAnswer( sql("SELECT explode(map(a, b)) as (k1, k2), a, b FROM data"), Row("1", "1", "1", "1") :: Nil) @@ -700,15 +916,13 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { // 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}]}""")) - read.json(rdd).registerTempTable("data") - val originalConf = convertCTAS - setConf(HiveContext.CONVERT_CTAS, false) + val ds = (1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""").toDS() + read.json(ds).createOrReplaceTempView("data") - try { + withSQLConf(SQLConf.CONVERT_CTAS.key -> "false") { sql("CREATE TABLE explodeTest (key bigInt)") table("explodeTest").queryExecution.analyzed match { - case metastoreRelation: MetastoreRelation => // OK + case SubqueryAlias(_, r: HiveTableRelation) => // OK case _ => fail("To correctly test the fix of SPARK-5875, explodeTest should be a MetastoreRelation") } @@ -721,287 +935,108 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sql("DROP TABLE explodeTest") dropTempTable("data") - } finally { - setConf(HiveContext.CONVERT_CTAS, originalConf) } } test("sanity test for SPARK-6618") { - (1 to 100).par.map { i => - val tableName = s"SPARK_6618_table_$i" - sql(s"CREATE TABLE $tableName (col1 string)") - catalog.lookupRelation(TableIdentifier(tableName)) - table(tableName) - tables() - sql(s"DROP TABLE $tableName") + val threads: Seq[Thread] = (1 to 10).map { i => + new Thread("test-thread-" + i) { + override def run(): Unit = { + val tableName = s"SPARK_6618_table_$i" + sql(s"CREATE TABLE $tableName (col1 string)") + sessionState.catalog.lookupRelation(TableIdentifier(tableName)) + table(tableName) + tables() + sql(s"DROP TABLE $tableName") + } + } } + threads.foreach(_.start()) + threads.foreach(_.join(10000)) } test("SPARK-5203 union with different decimal precision") { - Seq.empty[(Decimal, Decimal)] + Seq.empty[(java.math.BigDecimal, java.math.BigDecimal)] .toDF("d1", "d2") .select($"d1".cast(DecimalType(10, 5)).as("d")) - .registerTempTable("dn") + .createOrReplaceTempView("dn") sql("select d from dn union all select d * 2 from dn") .queryExecution.analyzed } + test("Star Expansion - script transform") { + assume(TestUtils.testCommandAvailable("/bin/bash")) + val data = (1 to 100000).map { i => (i, i, i) } + data.toDF("d1", "d2", "d3").createOrReplaceTempView("script_trans") + assert(100000 === sql("SELECT TRANSFORM (*) USING 'cat' FROM script_trans").count()) + } + test("test script transform for stdout") { + assume(TestUtils.testCommandAvailable("/bin/bash")) val data = (1 to 100000).map { i => (i, i, i) } - data.toDF("d1", "d2", "d3").registerTempTable("script_trans") + data.toDF("d1", "d2", "d3").createOrReplaceTempView("script_trans") assert(100000 === - sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat' AS (a,b,c) FROM script_trans") - .queryExecution.toRdd.count()) + sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat' AS (a,b,c) FROM script_trans").count()) } test("test script transform for stderr") { + assume(TestUtils.testCommandAvailable("/bin/bash")) val data = (1 to 100000).map { i => (i, i, i) } - data.toDF("d1", "d2", "d3").registerTempTable("script_trans") + data.toDF("d1", "d2", "d3").createOrReplaceTempView("script_trans") assert(0 === - sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat 1>&2' AS (a,b,c) FROM script_trans") - .queryExecution.toRdd.count()) + sql("SELECT TRANSFORM (d1, d2, d3) USING 'cat 1>&2' AS (a,b,c) FROM script_trans").count()) } test("test script transform data type") { + assume(TestUtils.testCommandAvailable("/bin/bash")) val data = (1 to 5).map { i => (i, i) } - data.toDF("key", "value").registerTempTable("test") + data.toDF("key", "value").createOrReplaceTempView("test") checkAnswer( sql("""FROM - |(FROM test SELECT TRANSFORM(key, value) USING 'cat' AS (thing1 int, thing2 string)) t + |(FROM test SELECT TRANSFORM(key, value) USING 'cat' AS (`thing1` int, thing2 string)) t |SELECT thing1 + 1 """.stripMargin), (2 to 6).map(i => Row(i))) } - test("window function: udaf with aggregate expressin") { - val data = Seq( - WindowData(1, "a", 5), - WindowData(2, "a", 6), - WindowData(3, "b", 7), - WindowData(4, "b", 8), - WindowData(5, "c", 9), - WindowData(6, "c", 10) - ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") - - checkAnswer( - sql( - """ - |select area, sum(product), sum(sum(product)) over (partition by area) - |from windowData group by month, area - """.stripMargin), - Seq( - ("a", 5, 11), - ("a", 6, 11), - ("b", 7, 15), - ("b", 8, 15), - ("c", 9, 19), - ("c", 10, 19) - ).map(i => Row(i._1, i._2, i._3))) - - checkAnswer( - sql( - """ - |select area, sum(product) - 1, sum(sum(product)) over (partition by area) - |from windowData group by month, area - """.stripMargin), - Seq( - ("a", 4, 11), - ("a", 5, 11), - ("b", 6, 15), - ("b", 7, 15), - ("c", 8, 19), - ("c", 9, 19) - ).map(i => Row(i._1, i._2, i._3))) - - checkAnswer( - sql( - """ - |select area, sum(product), sum(product) / sum(sum(product)) over (partition by area) - |from windowData group by month, area - """.stripMargin), - Seq( - ("a", 5, 5d/11), - ("a", 6, 6d/11), - ("b", 7, 7d/15), - ("b", 8, 8d/15), - ("c", 10, 10d/19), - ("c", 9, 9d/19) - ).map(i => Row(i._1, i._2, i._3))) - - checkAnswer( - sql( - """ - |select area, sum(product), sum(product) / sum(sum(product) - 1) over (partition by area) - |from windowData group by month, area - """.stripMargin), - Seq( - ("a", 5, 5d/9), - ("a", 6, 6d/9), - ("b", 7, 7d/13), - ("b", 8, 8d/13), - ("c", 10, 10d/17), - ("c", 9, 9d/17) - ).map(i => Row(i._1, i._2, i._3))) - } - - test("window function: refer column in inner select block") { - val data = Seq( - WindowData(1, "a", 5), - WindowData(2, "a", 6), - WindowData(3, "b", 7), - WindowData(4, "b", 8), - WindowData(5, "c", 9), - WindowData(6, "c", 10) - ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") - - checkAnswer( - sql( - """ - |select area, rank() over (partition by area order by tmp.month) + tmp.tmp1 as c1 - |from (select month, area, product, 1 as tmp1 from windowData) tmp - """.stripMargin), - Seq( - ("a", 2), - ("a", 3), - ("b", 2), - ("b", 3), - ("c", 2), - ("c", 3) - ).map(i => Row(i._1, i._2))) - } - - test("window function: partition and order expressions") { - val data = Seq( - WindowData(1, "a", 5), - WindowData(2, "a", 6), - WindowData(3, "b", 7), - WindowData(4, "b", 8), - WindowData(5, "c", 9), - WindowData(6, "c", 10) - ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") - - checkAnswer( - sql( - """ - |select month, area, product, sum(product + 1) over (partition by 1 order by 2) - |from windowData - """.stripMargin), - Seq( - (1, "a", 5, 51), - (2, "a", 6, 51), - (3, "b", 7, 51), - (4, "b", 8, 51), - (5, "c", 9, 51), - (6, "c", 10, 51) - ).map(i => Row(i._1, i._2, i._3, i._4))) - - checkAnswer( - sql( - """ - |select month, area, product, sum(product) - |over (partition by month % 2 order by 10 - product) - |from windowData - """.stripMargin), - Seq( - (1, "a", 5, 21), - (2, "a", 6, 24), - (3, "b", 7, 16), - (4, "b", 8, 18), - (5, "c", 9, 9), - (6, "c", 10, 10) - ).map(i => Row(i._1, i._2, i._3, i._4))) - } - - test("window function: expressions in arguments of a window functions") { - val data = Seq( - WindowData(1, "a", 5), - WindowData(2, "a", 6), - WindowData(3, "b", 7), - WindowData(4, "b", 8), - WindowData(5, "c", 9), - WindowData(6, "c", 10) - ) - sparkContext.parallelize(data).toDF().registerTempTable("windowData") - - checkAnswer( - sql( - """ - |select month, area, month % 2, - |lag(product, 1 + 1, product) over (partition by month % 2 order by area) - |from windowData - """.stripMargin), - Seq( - (1, "a", 1, 5), - (2, "a", 0, 6), - (3, "b", 1, 7), - (4, "b", 0, 8), - (5, "c", 1, 5), - (6, "c", 0, 6) - ).map(i => Row(i._1, i._2, i._3, i._4))) - } - - test("window function: multiple window expressions in a single expression") { - val nums = sparkContext.parallelize(1 to 10).map(x => (x, x % 2)).toDF("x", "y") - nums.registerTempTable("nums") - - val expected = - Row(1, 1, 1, 55, 1, 57) :: - Row(0, 2, 3, 55, 2, 60) :: - Row(1, 3, 6, 55, 4, 65) :: - Row(0, 4, 10, 55, 6, 71) :: - Row(1, 5, 15, 55, 9, 79) :: - Row(0, 6, 21, 55, 12, 88) :: - Row(1, 7, 28, 55, 16, 99) :: - Row(0, 8, 36, 55, 20, 111) :: - Row(1, 9, 45, 55, 25, 125) :: - Row(0, 10, 55, 55, 30, 140) :: Nil + test("Sorting columns are not in Generate") { + withTempView("data") { + spark.range(1, 5) + .select(array($"id", $"id" + 1).as("a"), $"id".as("b"), (lit(10) - $"id").as("c")) + .createOrReplaceTempView("data") - val actual = sql( - """ - |SELECT - | y, - | x, - | sum(x) OVER w1 AS running_sum, - | sum(x) OVER w2 AS total_sum, - | sum(x) OVER w3 AS running_sum_per_y, - | ((sum(x) OVER w1) + (sum(x) OVER w2) + (sum(x) OVER w3)) as combined2 - |FROM nums - |WINDOW w1 AS (ORDER BY x ROWS BETWEEN UnBOUNDED PRECEDiNG AND CuRRENT RoW), - | w2 AS (ORDER BY x ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOuNDED FoLLOWING), - | w3 AS (PARTITION BY y ORDER BY x ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) - """.stripMargin) + // case 1: missing sort columns are resolvable if join is true + checkAnswer( + sql("SELECT explode(a) AS val, b FROM data WHERE b < 2 order by val, c"), + Row(1, 1) :: Row(2, 1) :: Nil) - checkAnswer(actual, expected) + // case 2: missing sort columns are resolvable if join is false + checkAnswer( + sql("SELECT explode(a) AS val FROM data order by val, c"), + Seq(1, 2, 2, 3, 3, 4, 4, 5).map(i => Row(i))) - dropTempTable("nums") + // case 3: missing sort columns are resolvable if join is true and outer is true + checkAnswer( + sql( + """ + |SELECT C.val, b FROM data LATERAL VIEW OUTER explode(a) C as val + |where b < 2 order by c, val, b + """.stripMargin), + Row(1, 1) :: Row(2, 1) :: Nil) + } } test("test case key when") { - (1 to 5).map(i => (i, i.toString)).toDF("k", "v").registerTempTable("t") + (1 to 5).map(i => (i, i.toString)).toDF("k", "v").createOrReplaceTempView("t") checkAnswer( sql("SELECT CASE k WHEN 2 THEN 22 WHEN 4 THEN 44 ELSE 0 END, v FROM t"), Row(0, "1") :: Row(22, "2") :: Row(0, "3") :: Row(44, "4") :: Row(0, "5") :: Nil) } - test("SPARK-7595: Window will cause resolve failed with self join") { - sql("SELECT * FROM src") // Force loading of src table. - - checkAnswer(sql( - """ - |with - | v1 as (select key, count(value) over (partition by key) cnt_val from src), - | v2 as (select v1.key, v1_lag.cnt_val from v1, v1 v1_lag where v1.key = v1_lag.key) - | select * from v2 order by key limit 1 - """.stripMargin), Row(0, 3)) - } - test("SPARK-7269 Check analysis failed in case in-sensitive") { Seq(1, 2, 3).map { i => (i.toString, i.toString) - }.toDF("key", "value").registerTempTable("df_analysis") + }.toDF("key", "value").createOrReplaceTempView("df_analysis") sql("SELECT kEy from df_analysis group by key").collect() sql("SELECT kEy+3 from df_analysis group by key+3").collect() sql("SELECT kEy+3, a.kEy, A.kEy from df_analysis A group by key").collect() @@ -1020,29 +1055,6 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { checkAnswer(sql("SELECT CAST('775983671874188101' as BIGINT)"), Row(775983671874188101L)) } - // `Math.exp(1.0)` has different result for different jdk version, so not use createQueryTest - test("udf_java_method") { - checkAnswer(sql( - """ - |SELECT java_method("java.lang.String", "valueOf", 1), - | java_method("java.lang.String", "isEmpty"), - | java_method("java.lang.Math", "max", 2, 3), - | java_method("java.lang.Math", "min", 2, 3), - | java_method("java.lang.Math", "round", 2.5), - | java_method("java.lang.Math", "exp", 1.0), - | java_method("java.lang.Math", "floor", 1.9) - |FROM src tablesample (1 rows) - """.stripMargin), - Row( - "1", - "true", - java.lang.Math.max(2, 3).toString, - java.lang.Math.min(2, 3).toString, - java.lang.Math.round(2.5).toString, - java.lang.Math.exp(1.0).toString, - java.lang.Math.floor(1.9).toString)) - } - test("dynamic partition value test") { try { sql("set hive.exec.dynamic.partition.mode=nonstrict") @@ -1135,7 +1147,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-8588 HiveTypeCoercion.inConversion fires too early") { val df = createDataFrame(Seq((1, "2014-01-01"), (2, "2015-01-01"), (3, "2016-01-01"))) - df.toDF("id", "datef").registerTempTable("test_SPARK8588") + df.toDF("id", "datef").createOrReplaceTempView("test_SPARK8588") checkAnswer( sql( """ @@ -1148,9 +1160,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("SPARK-9371: fix the support for special chars in column names for hive context") { - read.json(sparkContext.makeRDD( - """{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""" :: Nil)) - .registerTempTable("t") + val ds = Seq("""{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""").toDS() + read.json(ds).createOrReplaceTempView("t") checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) } @@ -1176,42 +1187,42 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { "interval 4 minutes 59 seconds 889 milliseconds 987 microseconds"))) } - test("specifying database name for a temporary table is not allowed") { + test("specifying database name for a temporary view is not allowed") { withTempPath { dir => - val path = dir.getCanonicalPath + val path = dir.toURI.toString val df = sparkContext.parallelize(1 to 10).map(i => (i, i.toString)).toDF("num", "str") df .write .format("parquet") .save(path) - val message = intercept[AnalysisException] { - sqlContext.sql( + // We don't support creating a temporary table while specifying a database + intercept[AnalysisException] { + spark.sql( s""" - |CREATE TEMPORARY TABLE db.t - |USING parquet - |OPTIONS ( - | path '$path' - |) - """.stripMargin) - }.getMessage - assert(message.contains("Specifying database name or other qualifiers are not allowed")) + |CREATE TEMPORARY VIEW db.t + |USING parquet + |OPTIONS ( + | path '$path' + |) + """.stripMargin) + } - // If you use backticks to quote the name of a temporary table having dot in it. - sqlContext.sql( + // If you use backticks to quote the name then it's OK. + spark.sql( s""" - |CREATE TEMPORARY TABLE `db.t` + |CREATE TEMPORARY VIEW `db.t` |USING parquet |OPTIONS ( | path '$path' |) - """.stripMargin) - checkAnswer(sqlContext.table("`db.t`"), df) + """.stripMargin) + checkAnswer(spark.table("`db.t`"), df) } } test("SPARK-10593 same column names in lateral view") { - val df = sqlContext.sql( + val df = spark.sql( """ |select |insideLayer2.json as a2 @@ -1224,18 +1235,19 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { checkAnswer(df, Row("text inside layer 2") :: Nil) } - test("SPARK-10310: " + + ignore("SPARK-10310: " + "script transformation using default input/output SerDe and record reader/writer") { - sqlContext + spark .range(5) .selectExpr("id AS a", "id AS b") - .registerTempTable("test") + .createOrReplaceTempView("test") + val scriptFilePath = getTestResourcePath("data") checkAnswer( sql( - """FROM( + s"""FROM( | FROM test SELECT TRANSFORM(a, b) - | USING 'python src/test/resources/data/scripts/test_transform.py "\t"' + | USING 'python $scriptFilePath/scripts/test_transform.py "\t"' | AS (c STRING, d STRING) |) t |SELECT c @@ -1243,18 +1255,19 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { (0 until 5).map(i => Row(i + "#"))) } - test("SPARK-10310: script transformation using LazySimpleSerDe") { - sqlContext + ignore("SPARK-10310: script transformation using LazySimpleSerDe") { + spark .range(5) .selectExpr("id AS a", "id AS b") - .registerTempTable("test") + .createOrReplaceTempView("test") + val scriptFilePath = getTestResourcePath("data") val df = sql( - """FROM test + s"""FROM test |SELECT TRANSFORM(a, b) |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' |WITH SERDEPROPERTIES('field.delim' = '|') - |USING 'python src/test/resources/data/scripts/test_transform.py "|"' + |USING 'python $scriptFilePath/scripts/test_transform.py "|"' |AS (c STRING, d STRING) |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' |WITH SERDEPROPERTIES('field.delim' = '|') @@ -1265,9 +1278,9 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("SPARK-10741: Sort on Aggregate using parquet") { withTable("test10741") { - withTempTable("src") { - Seq("a" -> 5, "a" -> 9, "b" -> 6).toDF().registerTempTable("src") - sql("CREATE TABLE test10741(c1 STRING, c2 INT) STORED AS PARQUET AS SELECT * FROM src") + withTempView("src") { + Seq("a" -> 5, "a" -> 9, "b" -> 6).toDF("c1", "c2").createOrReplaceTempView("src") + sql("CREATE TABLE test10741 STORED AS PARQUET AS SELECT * FROM src") } checkAnswer(sql( @@ -1288,11 +1301,12 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - test("run sql directly on files") { - val df = sqlContext.range(100) + test("run sql directly on files - parquet") { + val df = spark.range(100).toDF() withTempPath(f => { df.write.parquet(f.getCanonicalPath) - checkAnswer(sql(s"select id from parquet.`${f.getCanonicalPath}`"), + // data source type is case insensitive + checkAnswer(sql(s"select id from Parquet.`${f.getCanonicalPath}`"), df) checkAnswer(sql(s"select id from `org.apache.spark.sql.parquet`.`${f.getCanonicalPath}`"), df) @@ -1301,131 +1315,739 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { }) } - test("correctly parse CREATE VIEW statement") { - withSQLConf(SQLConf.NATIVE_VIEW.key -> "true") { - withTable("jt") { - val df = (1 until 10).map(i => i -> i).toDF("i", "j") - df.write.format("json").saveAsTable("jt") + test("run sql directly on files - orc") { + val df = spark.range(100).toDF() + withTempPath(f => { + df.write.orc(f.getCanonicalPath) + // data source type is case insensitive + checkAnswer(sql(s"select id from ORC.`${f.getCanonicalPath}`"), + df) + checkAnswer(sql(s"select id from `org.apache.spark.sql.hive.orc`.`${f.getCanonicalPath}`"), + df) + checkAnswer(sql(s"select a.id from orc.`${f.getCanonicalPath}` as a"), + df) + }) + } + + test("run sql directly on files - csv") { + val df = spark.range(100).toDF() + withTempPath(f => { + df.write.csv(f.getCanonicalPath) + // data source type is case insensitive + checkAnswer(sql(s"select cast(_c0 as int) id from CSV.`${f.getCanonicalPath}`"), + df) + checkAnswer( + sql(s"select cast(_c0 as int) id from `com.databricks.spark.csv`.`${f.getCanonicalPath}`"), + df) + checkAnswer(sql(s"select cast(a._c0 as int) id from csv.`${f.getCanonicalPath}` as a"), + df) + }) + } + + test("run sql directly on files - json") { + val df = spark.range(100).toDF() + withTempPath(f => { + df.write.json(f.getCanonicalPath) + // data source type is case insensitive + checkAnswer(sql(s"select id from jsoN.`${f.getCanonicalPath}`"), + df) + checkAnswer(sql(s"select id from `org.apache.spark.sql.json`.`${f.getCanonicalPath}`"), + df) + checkAnswer(sql(s"select a.id from json.`${f.getCanonicalPath}` as a"), + df) + }) + } + + test("run sql directly on files - hive") { + withTempPath(f => { + spark.range(100).toDF.write.parquet(f.getCanonicalPath) + + var e = intercept[AnalysisException] { + sql(s"select id from hive.`${f.getCanonicalPath}`") + } + assert(e.message.contains("Unsupported data source type for direct query on files: hive")) + + // data source type is case insensitive + e = intercept[AnalysisException] { + sql(s"select id from HIVE.`${f.getCanonicalPath}`") + } + assert(e.message.contains("Unsupported data source type for direct query on files: HIVE")) + }) + } + + test("SPARK-8976 Wrong Result for Rollup #1") { + checkAnswer(sql( + "SELECT count(*) AS cnt, key % 5, grouping_id() FROM src GROUP BY key%5 WITH ROLLUP"), + Seq( + (113, 3, 0), + (91, 0, 0), + (500, null, 1), + (84, 1, 0), + (105, 2, 0), + (107, 4, 0) + ).map(i => Row(i._1, i._2, i._3))) + } + + test("SPARK-8976 Wrong Result for Rollup #2") { + checkAnswer(sql( + """ + |SELECT count(*) AS cnt, key % 5 AS k1, key-5 AS k2, grouping_id() AS k3 + |FROM src GROUP BY key%5, key-5 + |WITH ROLLUP ORDER BY cnt, k1, k2, k3 LIMIT 10 + """.stripMargin), + Seq( + (1, 0, 5, 0), + (1, 0, 15, 0), + (1, 0, 25, 0), + (1, 0, 60, 0), + (1, 0, 75, 0), + (1, 0, 80, 0), + (1, 0, 100, 0), + (1, 0, 140, 0), + (1, 0, 145, 0), + (1, 0, 150, 0) + ).map(i => Row(i._1, i._2, i._3, i._4))) + } + + test("SPARK-8976 Wrong Result for Rollup #3") { + checkAnswer(sql( + """ + |SELECT count(*) AS cnt, key % 5 AS k1, key-5 AS k2, grouping_id() AS k3 + |FROM (SELECT key, key%2, key - 5 FROM src) t GROUP BY key%5, key-5 + |WITH ROLLUP ORDER BY cnt, k1, k2, k3 LIMIT 10 + """.stripMargin), + Seq( + (1, 0, 5, 0), + (1, 0, 15, 0), + (1, 0, 25, 0), + (1, 0, 60, 0), + (1, 0, 75, 0), + (1, 0, 80, 0), + (1, 0, 100, 0), + (1, 0, 140, 0), + (1, 0, 145, 0), + (1, 0, 150, 0) + ).map(i => Row(i._1, i._2, i._3, i._4))) + } + + test("SPARK-8976 Wrong Result for CUBE #1") { + checkAnswer(sql( + "SELECT count(*) AS cnt, key % 5, grouping_id() FROM src GROUP BY key%5 WITH CUBE"), + Seq( + (113, 3, 0), + (91, 0, 0), + (500, null, 1), + (84, 1, 0), + (105, 2, 0), + (107, 4, 0) + ).map(i => Row(i._1, i._2, i._3))) + } + + test("SPARK-8976 Wrong Result for CUBE #2") { + checkAnswer(sql( + """ + |SELECT count(*) AS cnt, key % 5 AS k1, key-5 AS k2, grouping_id() AS k3 + |FROM (SELECT key, key%2, key - 5 FROM src) t GROUP BY key%5, key-5 + |WITH CUBE ORDER BY cnt, k1, k2, k3 LIMIT 10 + """.stripMargin), + Seq( + (1, null, -3, 2), + (1, null, -1, 2), + (1, null, 3, 2), + (1, null, 4, 2), + (1, null, 5, 2), + (1, null, 6, 2), + (1, null, 12, 2), + (1, null, 14, 2), + (1, null, 15, 2), + (1, null, 22, 2) + ).map(i => Row(i._1, i._2, i._3, i._4))) + } + + test("SPARK-8976 Wrong Result for GroupingSet") { + checkAnswer(sql( + """ + |SELECT count(*) AS cnt, key % 5 AS k1, key-5 AS k2, grouping_id() AS k3 + |FROM (SELECT key, key%2, key - 5 FROM src) t GROUP BY key%5, key-5 + |GROUPING SETS (key%5, key-5) ORDER BY cnt, k1, k2, k3 LIMIT 10 + """.stripMargin), + Seq( + (1, null, -3, 2), + (1, null, -1, 2), + (1, null, 3, 2), + (1, null, 4, 2), + (1, null, 5, 2), + (1, null, 6, 2), + (1, null, 12, 2), + (1, null, 14, 2), + (1, null, 15, 2), + (1, null, 22, 2) + ).map(i => Row(i._1, i._2, i._3, i._4))) + } + + ignore("SPARK-10562: partition by column with mixed case name") { + withTable("tbl10562") { + val df = Seq(2012 -> "a").toDF("Year", "val") + df.write.partitionBy("Year").saveAsTable("tbl10562") + checkAnswer(sql("SELECT year FROM tbl10562"), Row(2012)) + checkAnswer(sql("SELECT Year FROM tbl10562"), Row(2012)) + checkAnswer(sql("SELECT yEAr FROM tbl10562"), Row(2012)) +// TODO(ekl) this is causing test flakes [SPARK-18167], but we think the issue is derby specific +// checkAnswer(sql("SELECT val FROM tbl10562 WHERE Year > 2015"), Nil) + checkAnswer(sql("SELECT val FROM tbl10562 WHERE Year == 2012"), Row("a")) + } + } + + test("SPARK-11453: append data to partitioned table") { + withTable("tbl11453") { + Seq("1" -> "10", "2" -> "20").toDF("i", "j") + .write.partitionBy("i").saveAsTable("tbl11453") + + Seq("3" -> "30").toDF("i", "j") + .write.mode(SaveMode.Append).partitionBy("i").saveAsTable("tbl11453") + checkAnswer( + spark.read.table("tbl11453").select("i", "j").orderBy("i"), + Row("1", "10") :: Row("2", "20") :: Row("3", "30") :: Nil) + + // make sure case sensitivity is correct. + Seq("4" -> "40").toDF("i", "j") + .write.mode(SaveMode.Append).partitionBy("I").saveAsTable("tbl11453") + checkAnswer( + spark.read.table("tbl11453").select("i", "j").orderBy("i"), + Row("1", "10") :: Row("2", "20") :: Row("3", "30") :: Row("4", "40") :: Nil) + } + } + + test("SPARK-11590: use native json_tuple in lateral view") { + checkAnswer(sql( + """ + |SELECT a, b + |FROM (SELECT '{"f1": "value1", "f2": 12}' json) test + |LATERAL VIEW json_tuple(json, 'f1', 'f2') jt AS a, b + """.stripMargin), Row("value1", "12")) + + // we should use `c0`, `c1`... as the name of fields if no alias is provided, to follow hive. + checkAnswer(sql( + """ + |SELECT c0, c1 + |FROM (SELECT '{"f1": "value1", "f2": 12}' json) test + |LATERAL VIEW json_tuple(json, 'f1', 'f2') jt + """.stripMargin), Row("value1", "12")) + + // we can also use `json_tuple` in project list. + checkAnswer(sql( + """ + |SELECT json_tuple(json, 'f1', 'f2') + |FROM (SELECT '{"f1": "value1", "f2": 12}' json) test + """.stripMargin), Row("value1", "12")) + + // we can also mix `json_tuple` with other project expressions. + checkAnswer(sql( + """ + |SELECT json_tuple(json, 'f1', 'f2'), 3.14, str + |FROM (SELECT '{"f1": "value1", "f2": 12}' json, 'hello' as str) test + """.stripMargin), Row("value1", "12", BigDecimal("3.14"), "hello")) + } + + test("multi-insert with lateral view") { + withTempView("t1") { + spark.range(10) + .select(array($"id", $"id" + 1).as("arr"), $"id") + .createOrReplaceTempView("source") + withTable("dest1", "dest2") { + sql("CREATE TABLE dest1 (i INT)") + sql("CREATE TABLE dest2 (i INT)") sql( - """CREATE VIEW IF NOT EXISTS - |default.testView (c1 COMMENT 'blabla', c2 COMMENT 'blabla') - |COMMENT 'blabla' - |TBLPROPERTIES ('a' = 'b') - |AS SELECT * FROM jt""".stripMargin) - checkAnswer(sql("SELECT c1, c2 FROM testView ORDER BY c1"), (1 to 9).map(i => Row(i, i))) - sql("DROP VIEW testView") + """ + |FROM source + |INSERT OVERWRITE TABLE dest1 + |SELECT id + |WHERE id > 3 + |INSERT OVERWRITE TABLE dest2 + |select col LATERAL VIEW EXPLODE(arr) exp AS col + |WHERE col > 3 + """.stripMargin) + + checkAnswer( + spark.table("dest1"), + sql("SELECT id FROM source WHERE id > 3")) + checkAnswer( + spark.table("dest2"), + sql("SELECT col FROM source LATERAL VIEW EXPLODE(arr) exp AS col WHERE col > 3")) } } } - test("correctly handle CREATE VIEW IF NOT EXISTS") { - withSQLConf(SQLConf.NATIVE_VIEW.key -> "true") { - withTable("jt", "jt2") { - sqlContext.range(1, 10).write.format("json").saveAsTable("jt") - sql("CREATE VIEW testView AS SELECT id FROM jt") + test("derived from Hive query file: drop_database_removes_partition_dirs.q") { + // This test verifies that if a partition exists outside a table's current location when the + // database is dropped the partition's location is dropped as well. + sql("DROP database if exists test_database CASCADE") + sql("CREATE DATABASE test_database") + val previousCurrentDB = sessionState.catalog.getCurrentDatabase + sql("USE test_database") + sql("drop table if exists test_table") - val df = (1 until 10).map(i => i -> i).toDF("i", "j") - df.write.format("json").saveAsTable("jt2") - sql("CREATE VIEW IF NOT EXISTS testView AS SELECT * FROM jt2") + val tempDir = System.getProperty("test.tmp.dir") + assert(tempDir != null, "TestHive should set test.tmp.dir.") + + sql( + """ + |CREATE TABLE test_table (key int, value STRING) + |PARTITIONED BY (part STRING) + |STORED AS RCFILE + |LOCATION 'file:${system:test.tmp.dir}/drop_database_removes_partition_dirs_table' + """.stripMargin) + sql( + """ + |ALTER TABLE test_table ADD PARTITION (part = '1') + |LOCATION 'file:${system:test.tmp.dir}/drop_database_removes_partition_dirs_table2/part=1' + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE test_table PARTITION (part = '1') + |SELECT * FROM default.src + """.stripMargin) + checkAnswer( + sql("select part, key, value from test_table"), + sql("select '1' as part, key, value from default.src") + ) + val path = new Path( + new Path(s"file:$tempDir"), + "drop_database_removes_partition_dirs_table2") + val fs = path.getFileSystem(sparkContext.hadoopConfiguration) + // The partition dir is not empty. + assert(fs.listStatus(new Path(path, "part=1")).nonEmpty) + + sql(s"USE $previousCurrentDB") + sql("DROP DATABASE test_database CASCADE") + + // This table dir should not exist after we drop the entire database with the mode + // of CASCADE. This probably indicates a Hive bug, which returns the wrong table + // root location. So, the table's directory still there. We should change the condition + // to fs.exists(path) after we handle fs operations. + assert( + fs.exists(path), + "Thank you for making the changes of letting Spark SQL handle filesystem operations " + + "for DDL commands. Originally, Hive metastore does not delete the table root directory " + + "for this case. Now, please change this condition to !fs.exists(path).") + } + + test("derived from Hive query file: drop_table_removes_partition_dirs.q") { + // This test verifies that if a partition exists outside the table's current location when the + // table is dropped the partition's location is dropped as well. + sql("drop table if exists test_table") + + val tempDir = System.getProperty("test.tmp.dir") + assert(tempDir != null, "TestHive should set test.tmp.dir.") - // make sure our view doesn't change. - checkAnswer(sql("SELECT * FROM testView ORDER BY id"), (1 to 9).map(i => Row(i))) - sql("DROP VIEW testView") + sql( + """ + |CREATE TABLE test_table (key int, value STRING) + |PARTITIONED BY (part STRING) + |STORED AS RCFILE + |LOCATION 'file:${system:test.tmp.dir}/drop_table_removes_partition_dirs_table2' + """.stripMargin) + sql( + """ + |ALTER TABLE test_table ADD PARTITION (part = '1') + |LOCATION 'file:${system:test.tmp.dir}/drop_table_removes_partition_dirs_table2/part=1' + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE test_table PARTITION (part = '1') + |SELECT * FROM default.src + """.stripMargin) + checkAnswer( + sql("select part, key, value from test_table"), + sql("select '1' as part, key, value from src") + ) + val path = new Path(new Path(s"file:$tempDir"), "drop_table_removes_partition_dirs_table2") + val fs = path.getFileSystem(sparkContext.hadoopConfiguration) + // The partition dir is not empty. + assert(fs.listStatus(new Path(path, "part=1")).nonEmpty) + + sql("drop table test_table") + assert(fs.exists(path), "This is an external table, so the data should not have been dropped") + } + + test("select partitioned table") { + val table = "table_with_partition" + withTable(table) { + sql( + s""" + |CREATE TABLE $table(c1 string) + |PARTITIONED BY (p1 string,p2 string,p3 string,p4 string,p5 string) + """.stripMargin) + sql( + s""" + |INSERT OVERWRITE TABLE $table + |PARTITION (p1='a',p2='b',p3='c',p4='d',p5='e') + |SELECT 'blarr' + """.stripMargin) + + // project list is the same order of paritioning columns in table definition + checkAnswer( + sql(s"SELECT p1, p2, p3, p4, p5, c1 FROM $table"), + Row("a", "b", "c", "d", "e", "blarr") :: Nil) + + // project list does not have the same order of paritioning columns in table definition + checkAnswer( + sql(s"SELECT p2, p3, p4, p1, p5, c1 FROM $table"), + Row("b", "c", "d", "a", "e", "blarr") :: Nil) + + // project list contains partial partition columns in table definition + checkAnswer( + sql(s"SELECT p2, p1, p5, c1 FROM $table"), + Row("b", "a", "e", "blarr") :: Nil) + } + } + + test("SPARK-14981: DESC not supported for sorting columns") { + withTable("t") { + val cause = intercept[ParseException] { + sql( + """CREATE TABLE t USING PARQUET + |OPTIONS (PATH '/path/to/file') + |CLUSTERED BY (a) SORTED BY (b DESC) INTO 2 BUCKETS + |AS SELECT 1 AS a, 2 AS b + """.stripMargin + ) } + + assert(cause.getMessage.contains("Column ordering must be ASC, was 'DESC'")) } } - test("correctly handle CREATE OR REPLACE VIEW") { - withSQLConf(SQLConf.NATIVE_VIEW.key -> "true") { - withTable("jt", "jt2") { - sqlContext.range(1, 10).write.format("json").saveAsTable("jt") - sql("CREATE OR REPLACE VIEW testView AS SELECT id FROM jt") - checkAnswer(sql("SELECT * FROM testView ORDER BY id"), (1 to 9).map(i => Row(i))) + test("insert into datasource table") { + withTable("tbl") { + sql("CREATE TABLE tbl(i INT, j STRING) USING parquet") + Seq(1 -> "a").toDF("i", "j").write.mode("overwrite").insertInto("tbl") + checkAnswer(sql("SELECT * FROM tbl"), Row(1, "a")) + } + } - val df = (1 until 10).map(i => i -> i).toDF("i", "j") - df.write.format("json").saveAsTable("jt2") - sql("CREATE OR REPLACE VIEW testView AS SELECT * FROM jt2") - // make sure the view has been changed. - checkAnswer(sql("SELECT * FROM testView ORDER BY i"), (1 to 9).map(i => Row(i, i))) + test("spark-15557 promote string test") { + withTable("tbl") { + sql("CREATE TABLE tbl(c1 string, c2 string)") + sql("insert into tbl values ('3', '2.3')") + checkAnswer( + sql("select (cast (99 as decimal(19,6)) + cast('3' as decimal)) * cast('2.3' as decimal)"), + Row(204.0) + ) + checkAnswer( + sql("select (cast(99 as decimal(19,6)) + '3') *'2.3' from tbl"), + Row(234.6) + ) + checkAnswer( + sql("select (cast(99 as decimal(19,6)) + c1) * c2 from tbl"), + Row(234.6) + ) + } + } - sql("DROP VIEW testView") + test("SPARK-15752 optimize metadata only query for hive table") { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + withTable("data_15752", "srcpart_15752", "srctext_15752") { + val df = Seq((1, "2"), (3, "4")).toDF("key", "value") + df.createOrReplaceTempView("data_15752") + sql( + """ + |CREATE TABLE srcpart_15752 (col1 INT, col2 STRING) + |PARTITIONED BY (partcol1 INT, partcol2 STRING) STORED AS parquet + """.stripMargin) + for (partcol1 <- Seq(0, 1); partcol2 <- Seq("a", "b")) { + sql( + s""" + |INSERT OVERWRITE TABLE srcpart_15752 + |PARTITION (partcol1='$partcol1', partcol2='$partcol2') + |select key, value from data_15752 + """.stripMargin) + } + checkAnswer( + sql("select partcol1 from srcpart_15752 group by partcol1"), + Row(0) :: Row(1) :: Nil) + checkAnswer( + sql("select partcol1 from srcpart_15752 where partcol1 = 1 group by partcol1"), + Row(1)) + checkAnswer( + sql("select partcol1, count(distinct partcol2) from srcpart_15752 group by partcol1"), + Row(0, 2) :: Row(1, 2) :: Nil) + checkAnswer( + sql("select partcol1, count(distinct partcol2) from srcpart_15752 where partcol1 = 1 " + + "group by partcol1"), + Row(1, 2) :: Nil) + checkAnswer(sql("select distinct partcol1 from srcpart_15752"), Row(0) :: Row(1) :: Nil) + checkAnswer(sql("select distinct partcol1 from srcpart_15752 where partcol1 = 1"), Row(1)) + checkAnswer( + sql("select distinct col from (select partcol1 + 1 as col from srcpart_15752 " + + "where partcol1 = 1) t"), + Row(2)) + checkAnswer(sql("select distinct partcol1 from srcpart_15752 where partcol1 = 1"), Row(1)) + checkAnswer(sql("select max(partcol1) from srcpart_15752"), Row(1)) + checkAnswer(sql("select max(partcol1) from srcpart_15752 where partcol1 = 1"), Row(1)) + checkAnswer(sql("select max(partcol1) from (select partcol1 from srcpart_15752) t"), Row(1)) + checkAnswer( + sql("select max(col) from (select partcol1 + 1 as col from srcpart_15752 " + + "where partcol1 = 1) t"), + Row(2)) - val e = intercept[AnalysisException] { - sql("CREATE OR REPLACE VIEW IF NOT EXISTS testView AS SELECT id FROM jt") + sql( + """ + |CREATE TABLE srctext_15752 (col1 INT, col2 STRING) + |PARTITIONED BY (partcol1 INT, partcol2 STRING) STORED AS textfile + """.stripMargin) + for (partcol1 <- Seq(0, 1); partcol2 <- Seq("a", "b")) { + sql( + s""" + |INSERT OVERWRITE TABLE srctext_15752 + |PARTITION (partcol1='$partcol1', partcol2='$partcol2') + |select key, value from data_15752 + """.stripMargin) } - assert(e.message.contains("not allowed to define a view")) + checkAnswer( + sql("select partcol1 from srctext_15752 group by partcol1"), + Row(0) :: Row(1) :: Nil) + checkAnswer( + sql("select partcol1 from srctext_15752 where partcol1 = 1 group by partcol1"), + Row(1)) + checkAnswer( + sql("select partcol1, count(distinct partcol2) from srctext_15752 group by partcol1"), + Row(0, 2) :: Row(1, 2) :: Nil) + checkAnswer( + sql("select partcol1, count(distinct partcol2) from srctext_15752 where partcol1 = 1 " + + "group by partcol1"), + Row(1, 2) :: Nil) + checkAnswer(sql("select distinct partcol1 from srctext_15752"), Row(0) :: Row(1) :: Nil) + checkAnswer(sql("select distinct partcol1 from srctext_15752 where partcol1 = 1"), Row(1)) + checkAnswer( + sql("select distinct col from (select partcol1 + 1 as col from srctext_15752 " + + "where partcol1 = 1) t"), + Row(2)) + checkAnswer(sql("select max(partcol1) from srctext_15752"), Row(1)) + checkAnswer(sql("select max(partcol1) from srctext_15752 where partcol1 = 1"), Row(1)) + checkAnswer(sql("select max(partcol1) from (select partcol1 from srctext_15752) t"), Row(1)) + checkAnswer( + sql("select max(col) from (select partcol1 + 1 as col from srctext_15752 " + + "where partcol1 = 1) t"), + Row(2)) } } } - test("correctly handle ALTER VIEW") { - withSQLConf(SQLConf.NATIVE_VIEW.key -> "true") { - withTable("jt", "jt2") { - sqlContext.range(1, 10).write.format("json").saveAsTable("jt") - sql("CREATE VIEW testView AS SELECT id FROM jt") + test("SPARK-17354: Partitioning by dates/timestamps works with Parquet vectorized reader") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { + sql( + """CREATE TABLE order(id INT) + |PARTITIONED BY (pd DATE, pt TIMESTAMP) + |STORED AS PARQUET + """.stripMargin) + + sql("set hive.exec.dynamic.partition.mode=nonstrict") + sql( + """INSERT INTO TABLE order PARTITION(pd, pt) + |SELECT 1 AS id, CAST('1990-02-24' AS DATE) AS pd, CAST('1990-02-24' AS TIMESTAMP) AS pt + """.stripMargin) + val actual = sql("SELECT * FROM order") + val expected = sql( + "SELECT 1 AS id, CAST('1990-02-24' AS DATE) AS pd, CAST('1990-02-24' AS TIMESTAMP) AS pt") + checkAnswer(actual, expected) + sql("DROP TABLE order") + } + } + + + test("SPARK-17108: Fix BIGINT and INT comparison failure in spark sql") { + withTable("t1", "t2", "t3") { + sql("create table t1(a map>)") + sql("select * from t1 where a[1] is not null") + + sql("create table t2(a map>)") + sql("select * from t2 where a[1] is not null") + + sql("create table t3(a map>)") + sql("select * from t3 where a[1L] is not null") + } + } + + test("SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH") { + withTempDir { dir => + val path = dir.toURI.toString.stripSuffix("/") + val dirPath = dir.getAbsoluteFile + for (i <- 1 to 3) { + Files.write(s"$i", new File(dirPath, s"part-r-0000$i"), StandardCharsets.UTF_8) + } + for (i <- 5 to 7) { + Files.write(s"$i", new File(dirPath, s"part-s-0000$i"), StandardCharsets.UTF_8) + } - val df = (1 until 10).map(i => i -> i).toDF("i", "j") - df.write.format("json").saveAsTable("jt2") - sql("ALTER VIEW testView AS SELECT * FROM jt2") - // make sure the view has been changed. - checkAnswer(sql("SELECT * FROM testView ORDER BY i"), (1 to 9).map(i => Row(i, i))) + withTable("load_t") { + sql("CREATE TABLE load_t (a STRING)") + sql(s"LOAD DATA LOCAL INPATH '$path/*part-r*' INTO TABLE load_t") + checkAnswer(sql("SELECT * FROM load_t"), Seq(Row("1"), Row("2"), Row("3"))) - sql("DROP VIEW testView") + val m = intercept[AnalysisException] { + sql("LOAD DATA LOCAL INPATH '/non-exist-folder/*part*' INTO TABLE load_t") + }.getMessage + assert(m.contains("LOAD DATA input path does not exist")) + + val m2 = intercept[AnalysisException] { + sql(s"LOAD DATA LOCAL INPATH '$path*/*part*' INTO TABLE load_t") + }.getMessage + assert(m2.contains("LOAD DATA input path allows only filename wildcard")) } } } - test("create hive view for json table") { - // json table is not hive-compatible, make sure the new flag fix it. - withSQLConf(SQLConf.NATIVE_VIEW.key -> "true") { - withTable("jt") { - sqlContext.range(1, 10).write.format("json").saveAsTable("jt") - sql("CREATE VIEW testView AS SELECT id FROM jt") - checkAnswer(sql("SELECT * FROM testView ORDER BY id"), (1 to 9).map(i => Row(i))) - sql("DROP VIEW testView") + test("Insert overwrite with partition") { + withTable("tableWithPartition") { + sql( + """ + |CREATE TABLE tableWithPartition (key int, value STRING) + |PARTITIONED BY (part STRING) + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE tableWithPartition PARTITION (part = '1') + |SELECT * FROM default.src + """.stripMargin) + checkAnswer( + sql("SELECT part, key, value FROM tableWithPartition"), + sql("SELECT '1' AS part, key, value FROM default.src") + ) + + sql( + """ + |INSERT OVERWRITE TABLE tableWithPartition PARTITION (part = '1') + |SELECT * FROM VALUES (1, "one"), (2, "two"), (3, null) AS data(key, value) + """.stripMargin) + checkAnswer( + sql("SELECT part, key, value FROM tableWithPartition"), + sql( + """ + |SELECT '1' AS part, key, value FROM VALUES + |(1, "one"), (2, "two"), (3, null) AS data(key, value) + """.stripMargin) + ) + } + } + + test("SPARK-19292: filter with partition columns should be case-insensitive on Hive tables") { + withTable("tbl") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + sql("CREATE TABLE tbl(i int, j int) USING hive PARTITIONED BY (j)") + sql("INSERT INTO tbl PARTITION(j=10) SELECT 1") + checkAnswer(spark.table("tbl"), Row(1, 10)) + + checkAnswer(sql("SELECT i, j FROM tbl WHERE J=10"), Row(1, 10)) + checkAnswer(spark.table("tbl").filter($"J" === 10), Row(1, 10)) } } } - test("create hive view for partitioned parquet table") { - // partitioned parquet table is not hive-compatible, make sure the new flag fix it. - withSQLConf(SQLConf.NATIVE_VIEW.key -> "true") { - withTable("parTable") { - val df = Seq(1 -> "a").toDF("i", "j") - df.write.format("parquet").partitionBy("i").saveAsTable("parTable") - sql("CREATE VIEW testView AS SELECT i, j FROM parTable") - checkAnswer(sql("SELECT * FROM testView"), Row(1, "a")) - sql("DROP VIEW testView") + test("SPARK-17409: Do Not Optimize Query in CTAS (Hive Serde Table) More Than Once") { + withTable("bar") { + withTempView("foo") { + sql("select 0 as id").createOrReplaceTempView("foo") + // If we optimize the query in CTAS more than once, the following saveAsTable will fail + // with the error: `GROUP BY position 0 is not in select list (valid range is [1, 1])` + sql("SELECT * FROM foo group by id").toDF().write.format("hive").saveAsTable("bar") + checkAnswer(spark.table("bar"), Row(0) :: Nil) + val tableMetadata = spark.sessionState.catalog.getTableMetadata(TableIdentifier("bar")) + assert(tableMetadata.provider == Some("hive"), "the expected table is a Hive serde table") } } } - test("create hive view for joined tables") { - // make sure the new flag can handle some complex cases like join and schema change. - withSQLConf(SQLConf.NATIVE_VIEW.key -> "true") { - withTable("jt1", "jt2") { - sqlContext.range(1, 10).toDF("id1").write.format("json").saveAsTable("jt1") - sqlContext.range(1, 10).toDF("id2").write.format("json").saveAsTable("jt2") - sql("CREATE VIEW testView AS SELECT * FROM jt1 JOIN jt2 ON id1 == id2") - checkAnswer(sql("SELECT * FROM testView ORDER BY id1"), (1 to 9).map(i => Row(i, i))) + test("Auto alias construction of get_json_object") { + val df = Seq(("1", """{"f1": "value1", "f5": 5.23}""")).toDF("key", "jstring") + val expectedMsg = "Cannot create a table having a column whose name contains commas " + + "in Hive metastore. Table: `default`.`t`; Column: get_json_object(jstring, $.f1)" - val df = (1 until 10).map(i => i -> i).toDF("id1", "newCol") - df.write.format("json").mode(SaveMode.Overwrite).saveAsTable("jt1") - checkAnswer(sql("SELECT * FROM testView ORDER BY id1"), (1 to 9).map(i => Row(i, i))) + withTable("t") { + val e = intercept[AnalysisException] { + df.select($"key", functions.get_json_object($"jstring", "$.f1")) + .write.format("hive").saveAsTable("t") + }.getMessage + assert(e.contains(expectedMsg)) + } - sql("DROP VIEW testView") + withTempView("tempView") { + withTable("t") { + df.createTempView("tempView") + val e = intercept[AnalysisException] { + sql("CREATE TABLE t AS SELECT key, get_json_object(jstring, '$.f1') FROM tempView") + }.getMessage + assert(e.contains(expectedMsg)) } } } - test("SPARK-10562: partition by column with mixed case name") { - withTable("tbl10562") { - val df = Seq(2012 -> "a").toDF("Year", "val") - df.write.partitionBy("Year").saveAsTable("tbl10562") - checkAnswer(sql("SELECT Year FROM tbl10562"), Row(2012)) - checkAnswer(sql("SELECT yEAr FROM tbl10562"), Row(2012)) - checkAnswer(sql("SELECT val FROM tbl10562 WHERE Year > 2015"), Nil) - checkAnswer(sql("SELECT val FROM tbl10562 WHERE Year == 2012"), Row("a")) + test("SPARK-19912 String literals should be escaped for Hive metastore partition pruning") { + withTable("spark_19912") { + Seq( + (1, "p1", "q1"), + (2, "'", "q2"), + (3, "\"", "q3"), + (4, "p1\" and q=\"q1", "q4") + ).toDF("a", "p", "q").write.partitionBy("p", "q").saveAsTable("spark_19912") + + val table = spark.table("spark_19912") + checkAnswer(table.filter($"p" === "'").select($"a"), Row(2)) + checkAnswer(table.filter($"p" === "\"").select($"a"), Row(3)) + checkAnswer(table.filter($"p" === "p1\" and q=\"q1").select($"a"), Row(4)) + } + } + + test("SPARK-21721: Clear FileSystem deleterOnExit cache if path is successfully removed") { + val table = "test21721" + withTable(table) { + val deleteOnExitField = classOf[FileSystem].getDeclaredField("deleteOnExit") + deleteOnExitField.setAccessible(true) + + val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration) + val setOfPath = deleteOnExitField.get(fs).asInstanceOf[Set[Path]] + + val testData = sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)).toDF() + sql(s"CREATE TABLE $table (key INT, value STRING)") + val pathSizeToDeleteOnExit = setOfPath.size() + + (0 to 10).foreach(_ => testData.write.mode(SaveMode.Append).insertInto(table)) + + assert(setOfPath.size() == pathSizeToDeleteOnExit) + } + } + + test("SPARK-21912 ORC/Parquet table should not create invalid column names") { + Seq(" ", ",", ";", "{", "}", "(", ")", "\n", "\t", "=").foreach { name => + withTable("t21912") { + Seq("ORC", "PARQUET").foreach { source => + val m = intercept[AnalysisException] { + sql(s"CREATE TABLE t21912(`col$name` INT) USING $source") + }.getMessage + assert(m.contains(s"contains invalid character(s)")) + + val m2 = intercept[AnalysisException] { + sql(s"CREATE TABLE t21912 USING $source AS SELECT 1 `col$name`") + }.getMessage + assert(m2.contains(s"contains invalid character(s)")) + + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") { + val m3 = intercept[AnalysisException] { + sql(s"CREATE TABLE t21912(`col$name` INT) USING hive OPTIONS (fileFormat '$source')") + }.getMessage + assert(m3.contains(s"contains invalid character(s)")) + } + } + + // TODO: After SPARK-21929, we need to check ORC, too. + Seq("PARQUET").foreach { source => + sql(s"CREATE TABLE t21912(`col` INT) USING $source") + val m = intercept[AnalysisException] { + sql(s"ALTER TABLE t21912 ADD COLUMNS(`col$name` INT)") + }.getMessage + assert(m.contains(s"contains invalid character(s)")) + } + } } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala index 7cfdb886b585..5318b4650b01 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala @@ -20,16 +20,17 @@ package org.apache.spark.sql.hive.execution import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.scalatest.exceptions.TestFailedException -import org.apache.spark.TaskContext +import org.apache.spark.{SparkException, TaskContext, TestUtils} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.execution.{UnaryNode, SparkPlan, SparkPlanTest} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest, UnaryExecNode} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.types.StringType class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { - import hiveContext.implicits._ + import spark.implicits._ private val noSerdeIOSchema = HiveScriptIOSchema( inputRowFormat = Seq.empty, @@ -49,69 +50,95 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { ) test("cat without SerDe") { + assume(TestUtils.testCommandAvailable("/bin/bash")) + val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") checkAnswer( rowsDf, - (child: SparkPlan) => new ScriptTransformation( + (child: SparkPlan) => new ScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "cat", output = Seq(AttributeReference("a", StringType)()), child = child, ioschema = noSerdeIOSchema - )(hiveContext), + ), rowsDf.collect()) } test("cat with LazySimpleSerDe") { + assume(TestUtils.testCommandAvailable("/bin/bash")) + val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") checkAnswer( rowsDf, - (child: SparkPlan) => new ScriptTransformation( + (child: SparkPlan) => new ScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "cat", output = Seq(AttributeReference("a", StringType)()), child = child, ioschema = serdeIOSchema - )(hiveContext), + ), rowsDf.collect()) } test("script transformation should not swallow errors from upstream operators (no serde)") { + assume(TestUtils.testCommandAvailable("/bin/bash")) + val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") val e = intercept[TestFailedException] { checkAnswer( rowsDf, - (child: SparkPlan) => new ScriptTransformation( + (child: SparkPlan) => new ScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "cat", output = Seq(AttributeReference("a", StringType)()), child = ExceptionInjectingOperator(child), ioschema = noSerdeIOSchema - )(hiveContext), + ), rowsDf.collect()) } assert(e.getMessage().contains("intentional exception")) } test("script transformation should not swallow errors from upstream operators (with serde)") { + assume(TestUtils.testCommandAvailable("/bin/bash")) + val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") val e = intercept[TestFailedException] { checkAnswer( rowsDf, - (child: SparkPlan) => new ScriptTransformation( + (child: SparkPlan) => new ScriptTransformationExec( input = Seq(rowsDf.col("a").expr), script = "cat", output = Seq(AttributeReference("a", StringType)()), child = ExceptionInjectingOperator(child), ioschema = serdeIOSchema - )(hiveContext), + ), rowsDf.collect()) } assert(e.getMessage().contains("intentional exception")) } + + test("SPARK-14400 script transformation should fail for bad script command") { + assume(TestUtils.testCommandAvailable("/bin/bash")) + + val rowsDf = Seq("a", "b", "c").map(Tuple1.apply).toDF("a") + + val e = intercept[SparkException] { + val plan = + new ScriptTransformationExec( + input = Seq(rowsDf.col("a").expr), + script = "some_non_existent_command", + output = Seq(AttributeReference("a", StringType)()), + child = rowsDf.queryExecution.sparkPlan, + ioschema = serdeIOSchema) + SparkPlanTest.executePlan(plan, hiveContext) + } + assert(e.getMessage.contains("Subprocess exited with status")) + } } -private case class ExceptionInjectingOperator(child: SparkPlan) extends UnaryNode { +private case class ExceptionInjectingOperator(child: SparkPlan) extends UnaryExecNode { override protected def doExecute(): RDD[InternalRow] = { child.execute().map { x => assert(TaskContext.get() != null) // Make sure that TaskContext is defined. @@ -119,5 +146,8 @@ private case class ExceptionInjectingOperator(child: SparkPlan) extends UnaryNod throw new IllegalArgumentException("intentional exception") } } + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = child.outputPartitioning } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/TestingTypedCount.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/TestingTypedCount.scala new file mode 100644 index 000000000000..31b24301767a --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/TestingTypedCount.scala @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.{ImperativeAggregate, TypedImperativeAggregate} +import org.apache.spark.sql.hive.execution.TestingTypedCount.State +import org.apache.spark.sql.types._ + +@ExpressionDescription( + usage = "_FUNC_(expr) - A testing aggregate function resembles COUNT " + + "but implements ObjectAggregateFunction.") +case class TestingTypedCount( + child: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) + extends TypedImperativeAggregate[TestingTypedCount.State] { + + def this(child: Expression) = this(child, 0, 0) + + override def children: Seq[Expression] = child :: Nil + + override def dataType: DataType = LongType + + override def nullable: Boolean = false + + override def createAggregationBuffer(): State = TestingTypedCount.State(0L) + + override def update(buffer: State, input: InternalRow): State = { + if (child.eval(input) != null) { + buffer.count += 1 + } + buffer + } + + override def merge(buffer: State, input: State): State = { + buffer.count += input.count + buffer + } + + override def eval(buffer: State): Any = buffer.count + + override def serialize(buffer: State): Array[Byte] = { + val byteStream = new ByteArrayOutputStream() + val dataStream = new DataOutputStream(byteStream) + dataStream.writeLong(buffer.count) + byteStream.toByteArray + } + + override def deserialize(storageFormat: Array[Byte]): State = { + val byteStream = new ByteArrayInputStream(storageFormat) + val dataStream = new DataInputStream(byteStream) + TestingTypedCount.State(dataStream.readLong()) + } + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = + copy(inputAggBufferOffset = newInputAggBufferOffset) + + override val prettyName: String = "typed_count" +} + +object TestingTypedCount { + case class State(var count: Long) +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala new file mode 100644 index 000000000000..3f9485dd018b --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/WindowQuerySuite.scala @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql._ +import org.apache.spark.sql.hive.test.{TestHive, TestHiveSingleton} +import org.apache.spark.sql.test.SQLTestUtils + +/** + * This suite contains a couple of Hive window tests which fail in the typical setup due to tiny + * numerical differences or due semantic differences between Hive and Spark. + */ +class WindowQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + + override def beforeAll(): Unit = { + super.beforeAll() + sql("DROP TABLE IF EXISTS part") + sql( + """ + |CREATE TABLE part( + | p_partkey INT, + | p_name STRING, + | p_mfgr STRING, + | p_brand STRING, + | p_type STRING, + | p_size INT, + | p_container STRING, + | p_retailprice DOUBLE, + | p_comment STRING) + """.stripMargin) + val testData1 = TestHive.getHiveFile("data/files/part_tiny.txt").toURI + sql( + s""" + |LOAD DATA LOCAL INPATH '$testData1' overwrite into table part + """.stripMargin) + } + + override def afterAll(): Unit = { + try { + sql("DROP TABLE IF EXISTS part") + } finally { + super.afterAll() + } + } + + test("windowing.q -- 15. testExpressions") { + // Moved because: + // - Spark uses a different default stddev (sample instead of pop) + // - Tiny numerical differences in stddev results. + // - Different StdDev behavior when n=1 (NaN instead of 0) + checkAnswer(sql(s""" + |select p_mfgr,p_name, p_size, + |rank() over(distribute by p_mfgr sort by p_name) as r, + |dense_rank() over(distribute by p_mfgr sort by p_name) as dr, + |cume_dist() over(distribute by p_mfgr sort by p_name) as cud, + |percent_rank() over(distribute by p_mfgr sort by p_name) as pr, + |ntile(3) over(distribute by p_mfgr sort by p_name) as nt, + |count(p_size) over(distribute by p_mfgr sort by p_name) as ca, + |avg(p_size) over(distribute by p_mfgr sort by p_name) as avg, + |stddev(p_size) over(distribute by p_mfgr sort by p_name) as st, + |first_value(p_size % 5) over(distribute by p_mfgr sort by p_name) as fv, + |last_value(p_size) over(distribute by p_mfgr sort by p_name) as lv, + |first_value(p_size) over w1 as fvW1 + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + """.stripMargin), + // scalastyle:off + Seq( + Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 1, 1, 0.3333333333333333, 0.0, 1, 2, 2.0, 0.0, 2, 2, 2), + Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 1, 1, 0.3333333333333333, 0.0, 1, 2, 2.0, 0.0, 2, 2, 2), + Row("Manufacturer#1", "almond antique chartreuse lavender yellow", 34, 3, 2, 0.5, 0.4, 2, 3, 12.666666666666666, 18.475208614068027, 2, 34, 2), + Row("Manufacturer#1", "almond antique salmon chartreuse burlywood", 6, 4, 3, 0.6666666666666666, 0.6, 2, 4, 11.0, 15.448840301675292, 2, 6, 2), + Row("Manufacturer#1", "almond aquamarine burnished black steel", 28, 5, 4, 0.8333333333333334, 0.8, 3, 5, 14.4, 15.388307249337076, 2, 28, 34), + Row("Manufacturer#1", "almond aquamarine pink moccasin thistle", 42, 6, 5, 1.0, 1.0, 3, 6, 19.0, 17.787636155487327, 2, 42, 6), + Row("Manufacturer#2", "almond antique violet chocolate turquoise", 14, 1, 1, 0.2, 0.0, 1, 1, 14.0, Double.NaN, 4, 14, 14), + Row("Manufacturer#2", "almond antique violet turquoise frosted", 40, 2, 2, 0.4, 0.25, 1, 2, 27.0, 18.384776310850235, 4, 40, 14), + Row("Manufacturer#2", "almond aquamarine midnight light salmon", 2, 3, 3, 0.6, 0.5, 2, 3, 18.666666666666668, 19.42506971244462, 4, 2, 14), + Row("Manufacturer#2", "almond aquamarine rose maroon antique", 25, 4, 4, 0.8, 0.75, 2, 4, 20.25, 16.17353805861084, 4, 25, 40), + Row("Manufacturer#2", "almond aquamarine sandy cyan gainsboro", 18, 5, 5, 1.0, 1.0, 3, 5, 19.8, 14.042791745233567, 4, 18, 2), + Row("Manufacturer#3", "almond antique chartreuse khaki white", 17, 1, 1, 0.2, 0.0, 1, 1, 17.0,Double.NaN, 2, 17, 17), + Row("Manufacturer#3", "almond antique forest lavender goldenrod", 14, 2, 2, 0.4, 0.25, 1, 2, 15.5, 2.1213203435596424, 2, 14, 17), + Row("Manufacturer#3", "almond antique metallic orange dim", 19, 3, 3, 0.6, 0.5, 2, 3, 16.666666666666668, 2.516611478423583, 2, 19, 17), + Row("Manufacturer#3", "almond antique misty red olive", 1, 4, 4, 0.8, 0.75, 2, 4, 12.75, 8.098353742170895, 2, 1, 14), + Row("Manufacturer#3", "almond antique olive coral navajo", 45, 5, 5, 1.0, 1.0, 3, 5, 19.2, 16.037456157383566, 2, 45, 19), + Row("Manufacturer#4", "almond antique gainsboro frosted violet", 10, 1, 1, 0.2, 0.0, 1, 1, 10.0, Double.NaN, 0, 10, 10), + Row("Manufacturer#4", "almond antique violet mint lemon", 39, 2, 2, 0.4, 0.25, 1, 2, 24.5, 20.506096654409877, 0, 39, 10), + Row("Manufacturer#4", "almond aquamarine floral ivory bisque", 27, 3, 3, 0.6, 0.5, 2, 3, 25.333333333333332, 14.571661996262929, 0, 27, 10), + Row("Manufacturer#4", "almond aquamarine yellow dodger mint", 7, 4, 4, 0.8, 0.75, 2, 4, 20.75, 15.01943185787443, 0, 7, 39), + Row("Manufacturer#4", "almond azure aquamarine papaya violet", 12, 5, 5, 1.0, 1.0, 3, 5, 19.0, 13.583077707206124, 0, 12, 27), + Row("Manufacturer#5", "almond antique blue firebrick mint", 31, 1, 1, 0.2, 0.0, 1, 1, 31.0, Double.NaN, 1, 31, 31), + Row("Manufacturer#5", "almond antique medium spring khaki", 6, 2, 2, 0.4, 0.25, 1, 2, 18.5, 17.67766952966369, 1, 6, 31), + Row("Manufacturer#5", "almond antique sky peru orange", 2, 3, 3, 0.6, 0.5, 2, 3, 13.0, 15.716233645501712, 1, 2, 31), + Row("Manufacturer#5", "almond aquamarine dodger light gainsboro", 46, 4, 4, 0.8, 0.75, 2, 4, 21.25, 20.902551678363736, 1, 46, 6), + Row("Manufacturer#5", "almond azure blanched chiffon midnight", 23, 5, 5, 1.0, 1.0, 3, 5, 21.6, 18.1190507477627, 1, 23, 2))) + // scalastyle:on + } + + test("windowing.q -- 20. testSTATs") { + // Moved because: + // - Spark uses a different default stddev/variance (sample instead of pop) + // - Tiny numerical differences in aggregation results. + checkAnswer(sql(""" + |select p_mfgr,p_name, p_size, sdev, sdev_pop, uniq_data, var, cor, covarp + |from ( + |select p_mfgr,p_name, p_size, + |stddev_pop(p_retailprice) over w1 as sdev, + |stddev_pop(p_retailprice) over w1 as sdev_pop, + |collect_set(p_size) over w1 as uniq_size, + |var_pop(p_retailprice) over w1 as var, + |corr(p_size, p_retailprice) over w1 as cor, + |covar_pop(p_size, p_retailprice) over w1 as covarp + |from part + |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name + | rows between 2 preceding and 2 following) + |) t lateral view explode(uniq_size) d as uniq_data + |order by p_mfgr,p_name, p_size, sdev, sdev_pop, uniq_data, var, cor, covarp + """.stripMargin), + // scalastyle:off + Seq( + Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 258.10677784349247, 258.10677784349247, 2, 66619.10876874997, 0.811328754177887, 2801.7074999999995), + Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 258.10677784349247, 258.10677784349247, 6, 66619.10876874997, 0.811328754177887, 2801.7074999999995), + Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 258.10677784349247, 258.10677784349247, 34, 66619.10876874997, 0.811328754177887, 2801.7074999999995), + Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 273.70217881648085, 273.70217881648085, 2, 74912.88268888886, 1.0, 4128.782222222221), + Row("Manufacturer#1", "almond antique burnished rose metallic", 2, 273.70217881648085, 273.70217881648085, 34, 74912.88268888886, 1.0, 4128.782222222221), + Row("Manufacturer#1", "almond antique chartreuse lavender yellow", 34, 230.9015158547037, 230.9015158547037, 2, 53315.510023999974, 0.6956393773976641, 2210.7864), + Row("Manufacturer#1", "almond antique chartreuse lavender yellow", 34, 230.9015158547037, 230.9015158547037, 6, 53315.510023999974, 0.6956393773976641, 2210.7864), + Row("Manufacturer#1", "almond antique chartreuse lavender yellow", 34, 230.9015158547037, 230.9015158547037, 28, 53315.510023999974, 0.6956393773976641, 2210.7864), + Row("Manufacturer#1", "almond antique chartreuse lavender yellow", 34, 230.9015158547037, 230.9015158547037, 34, 53315.510023999974, 0.6956393773976641, 2210.7864), + Row("Manufacturer#1", "almond antique salmon chartreuse burlywood", 6, 202.73109328368943, 202.73109328368943, 2, 41099.89618399999, 0.6307859771012139, 2009.9536000000007), + Row("Manufacturer#1", "almond antique salmon chartreuse burlywood", 6, 202.73109328368943, 202.73109328368943, 6, 41099.89618399999, 0.6307859771012139, 2009.9536000000007), + Row("Manufacturer#1", "almond antique salmon chartreuse burlywood", 6, 202.73109328368943, 202.73109328368943, 28, 41099.89618399999, 0.6307859771012139, 2009.9536000000007), + Row("Manufacturer#1", "almond antique salmon chartreuse burlywood", 6, 202.73109328368943, 202.73109328368943, 34, 41099.89618399999, 0.6307859771012139, 2009.9536000000007), + Row("Manufacturer#1", "almond antique salmon chartreuse burlywood", 6, 202.73109328368943, 202.73109328368943, 42, 41099.89618399999, 0.6307859771012139, 2009.9536000000007), + Row("Manufacturer#1", "almond aquamarine burnished black steel", 28, 121.60645179738611, 121.60645179738611, 6, 14788.129118749992, 0.2036684720435979, 331.1337500000004), + Row("Manufacturer#1", "almond aquamarine burnished black steel", 28, 121.60645179738611, 121.60645179738611, 28, 14788.129118749992, 0.2036684720435979, 331.1337500000004), + Row("Manufacturer#1", "almond aquamarine burnished black steel", 28, 121.60645179738611, 121.60645179738611, 34, 14788.129118749992, 0.2036684720435979, 331.1337500000004), + Row("Manufacturer#1", "almond aquamarine burnished black steel", 28, 121.60645179738611, 121.60645179738611, 42, 14788.129118749992, 0.2036684720435979, 331.1337500000004), + Row("Manufacturer#1", "almond aquamarine pink moccasin thistle", 42, 96.57515864168516, 96.57515864168516, 6, 9326.761266666656, -1.4442181184933883E-4, -0.20666666666708502), + Row("Manufacturer#1", "almond aquamarine pink moccasin thistle", 42, 96.57515864168516, 96.57515864168516, 28, 9326.761266666656, -1.4442181184933883E-4, -0.20666666666708502), + Row("Manufacturer#1", "almond aquamarine pink moccasin thistle", 42, 96.57515864168516, 96.57515864168516, 42, 9326.761266666656, -1.4442181184933883E-4, -0.20666666666708502), + Row("Manufacturer#2", "almond antique violet chocolate turquoise", 14, 142.23631697518977, 142.23631697518977, 2, 20231.16986666666, -0.4936952655452319, -1113.7466666666658), + Row("Manufacturer#2", "almond antique violet chocolate turquoise", 14, 142.23631697518977, 142.23631697518977, 14, 20231.16986666666, -0.4936952655452319, -1113.7466666666658), + Row("Manufacturer#2", "almond antique violet chocolate turquoise", 14, 142.23631697518977, 142.23631697518977, 40, 20231.16986666666, -0.4936952655452319, -1113.7466666666658), + Row("Manufacturer#2", "almond antique violet turquoise frosted", 40, 137.7630649884068, 137.7630649884068, 2, 18978.662074999997, -0.5205630897335946, -1004.4812499999995), + Row("Manufacturer#2", "almond antique violet turquoise frosted", 40, 137.7630649884068, 137.7630649884068, 14, 18978.662074999997, -0.5205630897335946, -1004.4812499999995), + Row("Manufacturer#2", "almond antique violet turquoise frosted", 40, 137.7630649884068, 137.7630649884068, 25, 18978.662074999997, -0.5205630897335946, -1004.4812499999995), + Row("Manufacturer#2", "almond antique violet turquoise frosted", 40, 137.7630649884068, 137.7630649884068, 40, 18978.662074999997, -0.5205630897335946, -1004.4812499999995), + Row("Manufacturer#2", "almond aquamarine midnight light salmon", 2, 130.03972279269132, 130.03972279269132, 2, 16910.329504000005, -0.46908967495720255, -766.1791999999995), + Row("Manufacturer#2", "almond aquamarine midnight light salmon", 2, 130.03972279269132, 130.03972279269132, 14, 16910.329504000005, -0.46908967495720255, -766.1791999999995), + Row("Manufacturer#2", "almond aquamarine midnight light salmon", 2, 130.03972279269132, 130.03972279269132, 18, 16910.329504000005, -0.46908967495720255, -766.1791999999995), + Row("Manufacturer#2", "almond aquamarine midnight light salmon", 2, 130.03972279269132, 130.03972279269132, 25, 16910.329504000005, -0.46908967495720255, -766.1791999999995), + Row("Manufacturer#2", "almond aquamarine midnight light salmon", 2, 130.03972279269132, 130.03972279269132, 40, 16910.329504000005, -0.46908967495720255, -766.1791999999995), + Row("Manufacturer#2", "almond aquamarine rose maroon antique", 25, 135.55100986344593, 135.55100986344593, 2, 18374.076275000018, -0.6091405874714462, -1128.1787499999987), + Row("Manufacturer#2", "almond aquamarine rose maroon antique", 25, 135.55100986344593, 135.55100986344593, 18, 18374.076275000018, -0.6091405874714462, -1128.1787499999987), + Row("Manufacturer#2", "almond aquamarine rose maroon antique", 25, 135.55100986344593, 135.55100986344593, 25, 18374.076275000018, -0.6091405874714462, -1128.1787499999987), + Row("Manufacturer#2", "almond aquamarine rose maroon antique", 25, 135.55100986344593, 135.55100986344593, 40, 18374.076275000018, -0.6091405874714462, -1128.1787499999987), + Row("Manufacturer#2", "almond aquamarine sandy cyan gainsboro", 18, 156.44019460768035, 156.44019460768035, 2, 24473.534488888898, -0.9571686373491605, -1441.4466666666676), + Row("Manufacturer#2", "almond aquamarine sandy cyan gainsboro", 18, 156.44019460768035, 156.44019460768035, 18, 24473.534488888898, -0.9571686373491605, -1441.4466666666676), + Row("Manufacturer#2", "almond aquamarine sandy cyan gainsboro", 18, 156.44019460768035, 156.44019460768035, 25, 24473.534488888898, -0.9571686373491605, -1441.4466666666676), + Row("Manufacturer#3", "almond antique chartreuse khaki white", 17, 196.77422668858057, 196.77422668858057, 14, 38720.0962888889, 0.5557168646224995, 224.6944444444446), + Row("Manufacturer#3", "almond antique chartreuse khaki white", 17, 196.77422668858057, 196.77422668858057, 17, 38720.0962888889, 0.5557168646224995, 224.6944444444446), + Row("Manufacturer#3", "almond antique chartreuse khaki white", 17, 196.77422668858057, 196.77422668858057, 19, 38720.0962888889, 0.5557168646224995, 224.6944444444446), + Row("Manufacturer#3", "almond antique forest lavender goldenrod", 14, 275.1414418985261, 275.1414418985261, 1, 75702.81305000003, -0.6720833036576083, -1296.9000000000003), + Row("Manufacturer#3", "almond antique forest lavender goldenrod", 14, 275.1414418985261, 275.1414418985261, 14, 75702.81305000003, -0.6720833036576083, -1296.9000000000003), + Row("Manufacturer#3", "almond antique forest lavender goldenrod", 14, 275.1414418985261, 275.1414418985261, 17, 75702.81305000003, -0.6720833036576083, -1296.9000000000003), + Row("Manufacturer#3", "almond antique forest lavender goldenrod", 14, 275.1414418985261, 275.1414418985261, 19, 75702.81305000003, -0.6720833036576083, -1296.9000000000003), + Row("Manufacturer#3", "almond antique metallic orange dim", 19, 260.23473614412046, 260.23473614412046, 1, 67722.11789600001, -0.5703526513979519, -2129.0664), + Row("Manufacturer#3", "almond antique metallic orange dim", 19, 260.23473614412046, 260.23473614412046, 14, 67722.11789600001, -0.5703526513979519, -2129.0664), + Row("Manufacturer#3", "almond antique metallic orange dim", 19, 260.23473614412046, 260.23473614412046, 17, 67722.11789600001, -0.5703526513979519, -2129.0664), + Row("Manufacturer#3", "almond antique metallic orange dim", 19, 260.23473614412046, 260.23473614412046, 19, 67722.11789600001, -0.5703526513979519, -2129.0664), + Row("Manufacturer#3", "almond antique metallic orange dim", 19, 260.23473614412046, 260.23473614412046, 45, 67722.11789600001, -0.5703526513979519, -2129.0664), + Row("Manufacturer#3", "almond antique misty red olive", 1, 275.913996235693, 275.913996235693, 1, 76128.53331875002, -0.5774768996448021, -2547.7868749999993), + Row("Manufacturer#3", "almond antique misty red olive", 1, 275.913996235693, 275.913996235693, 14, 76128.53331875002, -0.5774768996448021, -2547.7868749999993), + Row("Manufacturer#3", "almond antique misty red olive", 1, 275.913996235693, 275.913996235693, 19, 76128.53331875002, -0.5774768996448021, -2547.7868749999993), + Row("Manufacturer#3", "almond antique misty red olive", 1, 275.913996235693, 275.913996235693, 45, 76128.53331875002, -0.5774768996448021, -2547.7868749999993), + Row("Manufacturer#3", "almond antique olive coral navajo", 45, 260.58159187137954, 260.58159187137954, 1, 67902.7660222222, -0.8710736366736884, -4099.731111111111), + Row("Manufacturer#3", "almond antique olive coral navajo", 45, 260.58159187137954, 260.58159187137954, 19, 67902.7660222222, -0.8710736366736884, -4099.731111111111), + Row("Manufacturer#3", "almond antique olive coral navajo", 45, 260.58159187137954, 260.58159187137954, 45, 67902.7660222222, -0.8710736366736884, -4099.731111111111), + Row("Manufacturer#4", "almond antique gainsboro frosted violet", 10, 170.1301188959661, 170.1301188959661, 10, 28944.25735555556, -0.6656975320098423, -1347.4777777777779), + Row("Manufacturer#4", "almond antique gainsboro frosted violet", 10, 170.1301188959661, 170.1301188959661, 27, 28944.25735555556, -0.6656975320098423, -1347.4777777777779), + Row("Manufacturer#4", "almond antique gainsboro frosted violet", 10, 170.1301188959661, 170.1301188959661, 39, 28944.25735555556, -0.6656975320098423, -1347.4777777777779), + Row("Manufacturer#4", "almond antique violet mint lemon", 39, 242.26834609323197, 242.26834609323197, 7, 58693.95151875002, -0.8051852719193339, -2537.328125), + Row("Manufacturer#4", "almond antique violet mint lemon", 39, 242.26834609323197, 242.26834609323197, 10, 58693.95151875002, -0.8051852719193339, -2537.328125), + Row("Manufacturer#4", "almond antique violet mint lemon", 39, 242.26834609323197, 242.26834609323197, 27, 58693.95151875002, -0.8051852719193339, -2537.328125), + Row("Manufacturer#4", "almond antique violet mint lemon", 39, 242.26834609323197, 242.26834609323197, 39, 58693.95151875002, -0.8051852719193339, -2537.328125), + Row("Manufacturer#4", "almond aquamarine floral ivory bisque", 27, 234.10001662537323, 234.10001662537323, 7, 54802.81778400003, -0.6046935574240581, -1719.8079999999995), + Row("Manufacturer#4", "almond aquamarine floral ivory bisque", 27, 234.10001662537323, 234.10001662537323, 10, 54802.81778400003, -0.6046935574240581, -1719.8079999999995), + Row("Manufacturer#4", "almond aquamarine floral ivory bisque", 27, 234.10001662537323, 234.10001662537323, 12, 54802.81778400003, -0.6046935574240581, -1719.8079999999995), + Row("Manufacturer#4", "almond aquamarine floral ivory bisque", 27, 234.10001662537323, 234.10001662537323, 27, 54802.81778400003, -0.6046935574240581, -1719.8079999999995), + Row("Manufacturer#4", "almond aquamarine floral ivory bisque", 27, 234.10001662537323, 234.10001662537323, 39, 54802.81778400003, -0.6046935574240581, -1719.8079999999995), + Row("Manufacturer#4", "almond aquamarine yellow dodger mint", 7, 247.33427141977316, 247.33427141977316, 7, 61174.241818750015, -0.5508665654707869, -1719.0368749999975), + Row("Manufacturer#4", "almond aquamarine yellow dodger mint", 7, 247.33427141977316, 247.33427141977316, 12, 61174.241818750015, -0.5508665654707869, -1719.0368749999975), + Row("Manufacturer#4", "almond aquamarine yellow dodger mint", 7, 247.33427141977316, 247.33427141977316, 27, 61174.241818750015, -0.5508665654707869, -1719.0368749999975), + Row("Manufacturer#4", "almond aquamarine yellow dodger mint", 7, 247.33427141977316, 247.33427141977316, 39, 61174.241818750015, -0.5508665654707869, -1719.0368749999975), + Row("Manufacturer#4", "almond azure aquamarine papaya violet", 12, 283.33443305668936, 283.33443305668936, 7, 80278.4009555556, -0.7755740084632333, -1867.4888888888881), + Row("Manufacturer#4", "almond azure aquamarine papaya violet", 12, 283.33443305668936, 283.33443305668936, 12, 80278.4009555556, -0.7755740084632333, -1867.4888888888881), + Row("Manufacturer#4", "almond azure aquamarine papaya violet", 12, 283.33443305668936, 283.33443305668936, 27, 80278.4009555556, -0.7755740084632333, -1867.4888888888881), + Row("Manufacturer#5", "almond antique blue firebrick mint", 31, 83.69879024746344, 83.69879024746344, 2, 7005.487488888881, 0.3900430308728505, 418.9233333333353), + Row("Manufacturer#5", "almond antique blue firebrick mint", 31, 83.69879024746344, 83.69879024746344, 6, 7005.487488888881, 0.3900430308728505, 418.9233333333353), + Row("Manufacturer#5", "almond antique blue firebrick mint", 31, 83.69879024746344, 83.69879024746344, 31, 7005.487488888881, 0.3900430308728505, 418.9233333333353), + Row("Manufacturer#5", "almond antique medium spring khaki", 6, 316.68049612345885, 316.68049612345885, 2, 100286.53662500005, -0.7136129117761831, -4090.853749999999), + Row("Manufacturer#5", "almond antique medium spring khaki", 6, 316.68049612345885, 316.68049612345885, 6, 100286.53662500005, -0.7136129117761831, -4090.853749999999), + Row("Manufacturer#5", "almond antique medium spring khaki", 6, 316.68049612345885, 316.68049612345885, 31, 100286.53662500005, -0.7136129117761831, -4090.853749999999), + Row("Manufacturer#5", "almond antique medium spring khaki", 6, 316.68049612345885, 316.68049612345885, 46, 100286.53662500005, -0.7136129117761831, -4090.853749999999), + Row("Manufacturer#5", "almond antique sky peru orange", 2, 285.4050629824216, 285.4050629824216, 2, 81456.04997600004, -0.712858514567818, -3297.2011999999986), + Row("Manufacturer#5", "almond antique sky peru orange", 2, 285.4050629824216, 285.4050629824216, 6, 81456.04997600004, -0.712858514567818, -3297.2011999999986), + Row("Manufacturer#5", "almond antique sky peru orange", 2, 285.4050629824216, 285.4050629824216, 23, 81456.04997600004, -0.712858514567818, -3297.2011999999986), + Row("Manufacturer#5", "almond antique sky peru orange", 2, 285.4050629824216, 285.4050629824216, 31, 81456.04997600004, -0.712858514567818, -3297.2011999999986), + Row("Manufacturer#5", "almond antique sky peru orange", 2, 285.4050629824216, 285.4050629824216, 46, 81456.04997600004, -0.712858514567818, -3297.2011999999986), + Row("Manufacturer#5", "almond aquamarine dodger light gainsboro", 46, 285.43749038756283, 285.43749038756283, 2, 81474.56091875004, -0.9841287871533909, -4871.028125000002), + Row("Manufacturer#5", "almond aquamarine dodger light gainsboro", 46, 285.43749038756283, 285.43749038756283, 6, 81474.56091875004, -0.9841287871533909, -4871.028125000002), + Row("Manufacturer#5", "almond aquamarine dodger light gainsboro", 46, 285.43749038756283, 285.43749038756283, 23, 81474.56091875004, -0.9841287871533909, -4871.028125000002), + Row("Manufacturer#5", "almond aquamarine dodger light gainsboro", 46, 285.43749038756283, 285.43749038756283, 46, 81474.56091875004, -0.9841287871533909, -4871.028125000002), + Row("Manufacturer#5", "almond azure blanched chiffon midnight", 23, 315.9225931564038, 315.9225931564038, 2, 99807.08486666666, -0.9978877469246935, -5664.856666666666), + Row("Manufacturer#5", "almond azure blanched chiffon midnight", 23, 315.9225931564038, 315.9225931564038, 23, 99807.08486666666, -0.9978877469246935, -5664.856666666666), + Row("Manufacturer#5", "almond azure blanched chiffon midnight", 23, 315.9225931564038, 315.9225931564038, 46, 99807.08486666666, -0.9978877469246935, -5664.856666666666))) + // scalastyle:on + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala new file mode 100644 index 000000000000..de6f0d67f173 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala @@ -0,0 +1,347 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.orc + +import java.nio.charset.StandardCharsets +import java.sql.{Date, Timestamp} + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.hive.ql.io.sarg.{PredicateLeaf, SearchArgument} + +import org.apache.spark.sql.{Column, DataFrame, QueryTest} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} + +/** + * A test suite that tests ORC filter API based filter pushdown optimization. + */ +class OrcFilterSuite extends QueryTest with OrcTest { + private def checkFilterPredicate( + df: DataFrame, + predicate: Predicate, + checker: (SearchArgument) => Unit): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => + maybeRelation = Some(orcRelation) + filters + }.flatten.reduceLeftOption(_ && _) + assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") + + val (_, selectedFilters, _) = + DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) + assert(selectedFilters.nonEmpty, "No filter is pushed down") + + val maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters.toArray) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $selectedFilters") + checker(maybeFilter.get) + } + + private def checkFilterPredicate + (predicate: Predicate, filterOperator: PredicateLeaf.Operator) + (implicit df: DataFrame): Unit = { + def checkComparisonOperator(filter: SearchArgument) = { + val operator = filter.getLeaves.asScala + assert(operator.map(_.getOperator).contains(filterOperator)) + } + checkFilterPredicate(df, predicate, checkComparisonOperator) + } + + private def checkFilterPredicate + (predicate: Predicate, stringExpr: String) + (implicit df: DataFrame): Unit = { + def checkLogicalOperator(filter: SearchArgument) = { + assert(filter.toString == stringExpr) + } + checkFilterPredicate(df, predicate, checkLogicalOperator) + } + + private def checkNoFilterPredicate + (predicate: Predicate) + (implicit df: DataFrame): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => + maybeRelation = Some(orcRelation) + filters + }.flatten.reduceLeftOption(_ && _) + assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") + + val (_, selectedFilters, _) = + DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) + assert(selectedFilters.nonEmpty, "No filter is pushed down") + + val maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters.toArray) + assert(maybeFilter.isEmpty, s"Could generate filter predicate for $selectedFilters") + } + + test("filter pushdown - integer") { + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - long") { + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - float") { + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - double") { + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - string") { + withOrcDataFrame((1 to 4).map(i => Tuple1(i.toString))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === "1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < "2", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= "4", PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal("1") === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal("1") <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal("2") > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal("3") < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("1") >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("4") <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - boolean") { + withOrcDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === true, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < true, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= false, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(false) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(false) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(false) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(true) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - decimal") { + withOrcDataFrame((1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> BigDecimal.valueOf(1), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > BigDecimal.valueOf(3), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= BigDecimal.valueOf(1), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(2)) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate( + Literal(BigDecimal.valueOf(3)) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(4)) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - timestamp") { + val timeString = "2015-08-20 14:57:00" + val timestamps = (1 to 4).map { i => + val milliseconds = Timestamp.valueOf(timeString).getTime + i * 3600 + new Timestamp(milliseconds) + } + withOrcDataFrame(timestamps.map(Tuple1(_))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === timestamps(0), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < timestamps(1), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(timestamps(0)) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(timestamps(0)) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(timestamps(1)) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(timestamps(2)) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(0)) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(3)) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - combinations with logical operators") { + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => + // Because `ExpressionTree` is not accessible at Hive 1.2.x, this should be checked + // in string form in order to check filter creation including logical operators + // such as `and`, `or` or `not`. So, this function uses `SearchArgument.toString()` + // to produce string expression and then compare it to given string expression below. + // This might have to be changed after Hive version is upgraded. + checkFilterPredicate( + '_1.isNotNull, + """leaf-0 = (IS_NULL _1) + |expr = (not leaf-0)""".stripMargin.trim + ) + checkFilterPredicate( + '_1 =!= 1, + """leaf-0 = (IS_NULL _1) + |leaf-1 = (EQUALS _1 1) + |expr = (and (not leaf-0) (not leaf-1))""".stripMargin.trim + ) + checkFilterPredicate( + !('_1 < 4), + """leaf-0 = (IS_NULL _1) + |leaf-1 = (LESS_THAN _1 4) + |expr = (and (not leaf-0) (not leaf-1))""".stripMargin.trim + ) + checkFilterPredicate( + '_1 < 2 || '_1 > 3, + """leaf-0 = (LESS_THAN _1 2) + |leaf-1 = (LESS_THAN_EQUALS _1 3) + |expr = (or leaf-0 (not leaf-1))""".stripMargin.trim + ) + checkFilterPredicate( + '_1 < 2 && '_1 > 3, + """leaf-0 = (IS_NULL _1) + |leaf-1 = (LESS_THAN _1 2) + |leaf-2 = (LESS_THAN_EQUALS _1 3) + |expr = (and (not leaf-0) leaf-1 (not leaf-2))""".stripMargin.trim + ) + } + } + + test("no filter pushdown - non-supported types") { + implicit class IntToBinary(int: Int) { + def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8) + } + // ArrayType + withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => + checkNoFilterPredicate('_1.isNull) + } + // BinaryType + withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => + checkNoFilterPredicate('_1 <=> 1.b) + } + // DateType + val stringDate = "2015-01-01" + withOrcDataFrame(Seq(Tuple1(Date.valueOf(stringDate)))) { implicit df => + checkNoFilterPredicate('_1 === Date.valueOf(stringDate)) + } + // MapType + withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => + checkNoFilterPredicate('_1.isNotNull) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala index 92043d66c914..ba0a7605da71 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala @@ -17,16 +17,20 @@ package org.apache.spark.sql.hive.orc +import java.io.File + import org.apache.hadoop.fs.Path -import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.catalog.CatalogUtils +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.HadoopFsRelationTest import org.apache.spark.sql.types._ class OrcHadoopFsRelationSuite extends HadoopFsRelationTest { import testImplicits._ - override val dataSourceName: String = classOf[DefaultSource].getCanonicalName + override val dataSourceName: String = classOf[OrcFileFormat].getCanonicalName // ORC does not play well with NullType and UDT. override protected def supportsDataType(dataType: DataType): Boolean = dataType match { @@ -38,12 +42,9 @@ class OrcHadoopFsRelationSuite extends HadoopFsRelationTest { test("save()/load() - partitioned table - simple queries - partition columns in data") { withTempDir { file => - val basePath = new Path(file.getCanonicalPath) - val fs = basePath.getFileSystem(SparkHadoopUtil.get.conf) - val qualifiedBasePath = fs.makeQualified(basePath) - for (p1 <- 1 to 2; p2 <- Seq("foo", "bar")) { - val partitionDir = new Path(qualifiedBasePath, s"p1=$p1/p2=$p2") + val partitionDir = new Path( + CatalogUtils.URIToString(makeQualifiedPath(file.getCanonicalPath)), s"p1=$p1/p2=$p2") sparkContext .parallelize(for (i <- 1 to 3) yield (i, s"val_$i", p1)) .toDF("a", "b", "p1") @@ -55,9 +56,63 @@ class OrcHadoopFsRelationSuite extends HadoopFsRelationTest { StructType(dataSchema.fields :+ StructField("p1", IntegerType, nullable = true)) checkQueries( - hiveContext.read.options(Map( + spark.read.options(Map( "path" -> file.getCanonicalPath, "dataSchema" -> dataSchemaWithPartition.json)).format(dataSourceName).load()) } } + + test("SPARK-12218: 'Not' is included in ORC filter pushdown") { + import testImplicits._ + + withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/table1" + (1 to 5).map(i => (i, (i % 2).toString)).toDF("a", "b").write.orc(path) + + checkAnswer( + spark.read.orc(path).where("not (a = 2) or not(b in ('1'))"), + (1 to 5).map(i => Row(i, (i % 2).toString))) + + checkAnswer( + spark.read.orc(path).where("not (a = 2 and b in ('1'))"), + (1 to 5).map(i => Row(i, (i % 2).toString))) + } + } + } + + test("SPARK-13543: Support for specifying compression codec for ORC via option()") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/table1" + val df = (1 to 5).map(i => (i, (i % 2).toString)).toDF("a", "b") + df.write + .option("compression", "ZlIb") + .orc(path) + + // Check if this is compressed as ZLIB. + val maybeOrcFile = new File(path).listFiles().find { f => + !f.getName.startsWith("_") && f.getName.endsWith(".zlib.orc") + } + assert(maybeOrcFile.isDefined) + val orcFilePath = maybeOrcFile.get.toPath.toString + val expectedCompressionKind = + OrcFileOperator.getFileReader(orcFilePath).get.getCompression + assert("ZLIB" === expectedCompressionKind.name()) + + val copyDf = spark + .read + .orc(path) + checkAnswer(df, copyDf) + } + } + + test("Default compression codec is snappy for ORC compression") { + withTempPath { file => + spark.range(0, 10).write + .orc(file.getCanonicalPath) + val expectedCompressionKind = + OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression + assert("SNAPPY" === expectedCompressionKind.name()) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala index 52e09f9496f0..d1ce3f1e2f05 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcPartitionDiscoverySuite.scala @@ -22,8 +22,8 @@ import java.io.File import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag -import org.scalatest.BeforeAndAfterAll import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql._ import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -37,8 +37,8 @@ case class OrcParDataWithKey(intField: Int, pi: Int, stringField: String, ps: St // TODO This test suite duplicates ParquetPartitionDiscoverySuite a lot class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with BeforeAndAfterAll { - import hiveContext._ - import hiveContext.implicits._ + import spark._ + import spark.implicits._ val defaultPartitionName = ConfVars.DEFAULTPARTITIONNAME.defaultStrVal @@ -59,7 +59,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B } protected def withTempTable(tableName: String)(f: => Unit): Unit = { - try f finally hiveContext.dropTempTable(tableName) + try f finally spark.catalog.dropTempView(tableName) } protected def makePartitionDir( @@ -90,7 +90,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) } - read.orc(base.getCanonicalPath).registerTempTable("t") + read.orc(base.getCanonicalPath).createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -137,7 +137,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) } - read.orc(base.getCanonicalPath).registerTempTable("t") + read.orc(base.getCanonicalPath).createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -189,7 +189,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B read .option(ConfVars.DEFAULTPARTITIONNAME.varname, defaultPartitionName) .orc(base.getCanonicalPath) - .registerTempTable("t") + .createOrReplaceTempView("t") withTempTable("t") { checkAnswer( @@ -231,7 +231,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B read .option(ConfVars.DEFAULTPARTITIONNAME.varname, defaultPartitionName) .orc(base.getCanonicalPath) - .registerTempTable("t") + .createOrReplaceTempView("t") withTempTable("t") { checkAnswer( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala index 7efeab528c1d..60ccd996d6d5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala @@ -17,16 +17,23 @@ package org.apache.spark.sql.hive.orc -import java.io.File +import java.nio.charset.StandardCharsets +import java.sql.Timestamp -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.io.orc.CompressionKind +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.ql.io.orc.{OrcStruct, SparkOrcNewRecordReader} import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.HiveTableRelation +import org.apache.spark.sql.execution.datasources.{LogicalRelation, RecordReaderIterator} +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.util.Utils case class AllDataTypesWithNonPrimitiveType( stringField: String, @@ -51,12 +58,6 @@ case class Person(name: String, age: Int, contacts: Seq[Contact]) class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { - def getTempFilePath(prefix: String, suffix: String = ""): File = { - val tempFile = File.createTempFile(prefix, suffix) - tempFile.delete() - tempFile - } - test("Read/write All Types") { val data = (0 to 255).map { i => (s"$i", i, i.toLong, i.toFloat, i.toDouble, i.toShort, i.toByte, i % 2 == 0) @@ -64,20 +65,20 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { withOrcFile(data) { file => checkAnswer( - sqlContext.read.orc(file), + spark.read.orc(file), data.toDF().collect()) } } test("Read/write binary data") { - withOrcFile(BinaryData("test".getBytes("utf8")) :: Nil) { file => + withOrcFile(BinaryData("test".getBytes(StandardCharsets.UTF_8)) :: Nil) { file => val bytes = read.orc(file).head().getAs[Array[Byte]](0) - assert(new String(bytes, "utf8") === "test") + assert(new String(bytes, StandardCharsets.UTF_8) === "test") } } test("Read/write all types with non-primitive type") { - val data = (0 to 255).map { i => + val data: Seq[AllDataTypesWithNonPrimitiveType] = (0 to 255).map { i => AllDataTypesWithNonPrimitiveType( s"$i", i, i.toLong, i.toFloat, i.toDouble, i.toShort, i.toByte, i % 2 == 0, 0 until i, @@ -94,10 +95,20 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { } } + test("Read/write UserDefinedType") { + withTempPath { path => + val data = Seq((1, new UDT.MyDenseVector(Array(0.25, 2.25, 4.25)))) + val udtDF = data.toDF("id", "vectors") + udtDF.write.orc(path.getAbsolutePath) + val readBack = spark.read.schema(udtDF.schema).orc(path.getAbsolutePath) + checkAnswer(udtDF, readBack) + } + } + test("Creating case class RDD table") { val data = (1 to 100).map(i => (i, s"val_$i")) - sparkContext.parallelize(data).toDF().registerTempTable("t") - withTempTable("t") { + sparkContext.parallelize(data).toDF().createOrReplaceTempView("t") + withTempView("t") { checkAnswer(sql("SELECT * FROM t"), data.toDF().collect()) } } @@ -118,6 +129,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { // expr = (not leaf-0) assertResult(10) { sql("SELECT name, contacts FROM t where age > 5") + .rdd .flatMap(_.getAs[Seq[_]]("contacts")) .count() } @@ -130,7 +142,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { val df = sql("SELECT name, contacts FROM t WHERE age > 5 AND age < 8") assert(df.count() === 2) assertResult(4) { - df.flatMap(_.getAs[Seq[_]]("contacts")).count() + df.rdd.flatMap(_.getAs[Seq[_]]("contacts")).count() } } @@ -142,7 +154,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { val df = sql("SELECT name, contacts FROM t WHERE age < 2 OR age > 8") assert(df.count() === 3) assertResult(6) { - df.flatMap(_.getAs[Seq[_]]("contacts")).count() + df.rdd.flatMap(_.getAs[Seq[_]]("contacts")).count() } } } @@ -150,11 +162,11 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { test("save and load case class RDD with `None`s as orc") { val data = ( - None: Option[Int], - None: Option[Long], - None: Option[Float], - None: Option[Double], - None: Option[Boolean] + Option.empty[Int], + Option.empty[Long], + Option.empty[Float], + Option.empty[Double], + Option.empty[Boolean] ) :: Nil withOrcFile(data) { file => @@ -164,39 +176,68 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { } } - // We only support zlib in Hive 0.12.0 now - test("Default compression options for writing to an ORC file") { - withOrcFile((1 to 100).map(i => (i, s"val_$i"))) { file => - assertResult(CompressionKind.ZLIB) { - OrcFileOperator.getFileReader(file).get.getCompression - } + test("SPARK-16610: Respect orc.compress option when compression is unset") { + // Respect `orc.compress`. + withTempPath { file => + spark.range(0, 10).write + .option("orc.compress", "ZLIB") + .orc(file.getCanonicalPath) + val expectedCompressionKind = + OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression + assert("ZLIB" === expectedCompressionKind.name()) + } + + // `compression` overrides `orc.compress`. + withTempPath { file => + spark.range(0, 10).write + .option("compression", "ZLIB") + .option("orc.compress", "SNAPPY") + .orc(file.getCanonicalPath) + val expectedCompressionKind = + OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression + assert("ZLIB" === expectedCompressionKind.name()) } } - // Following codec is supported in hive-0.13.1, ignore it now - ignore("Other compression options for writing to an ORC file - 0.13.1 and above") { - val data = (1 to 100).map(i => (i, s"val_$i")) - val conf = sparkContext.hadoopConfiguration + // Hive supports zlib, snappy and none for Hive 1.2.1. + test("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") { + withTempPath { file => + spark.range(0, 10).write + .option("compression", "ZLIB") + .orc(file.getCanonicalPath) + val expectedCompressionKind = + OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression + assert("ZLIB" === expectedCompressionKind.name()) + } - conf.set(ConfVars.HIVE_ORC_DEFAULT_COMPRESS.varname, "SNAPPY") - withOrcFile(data) { file => - assertResult(CompressionKind.SNAPPY) { - OrcFileOperator.getFileReader(file).get.getCompression - } + withTempPath { file => + spark.range(0, 10).write + .option("compression", "SNAPPY") + .orc(file.getCanonicalPath) + val expectedCompressionKind = + OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression + assert("SNAPPY" === expectedCompressionKind.name()) } - conf.set(ConfVars.HIVE_ORC_DEFAULT_COMPRESS.varname, "NONE") - withOrcFile(data) { file => - assertResult(CompressionKind.NONE) { - OrcFileOperator.getFileReader(file).get.getCompression - } + withTempPath { file => + spark.range(0, 10).write + .option("compression", "NONE") + .orc(file.getCanonicalPath) + val expectedCompressionKind = + OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression + assert("NONE" === expectedCompressionKind.name()) } + } - conf.set(ConfVars.HIVE_ORC_DEFAULT_COMPRESS.varname, "LZO") - withOrcFile(data) { file => - assertResult(CompressionKind.LZO) { - OrcFileOperator.getFileReader(file).get.getCompression - } + // Following codec is not supported in Hive 1.2.1, ignore it now + ignore("LZO compression options for writing to an ORC file not supported in Hive 1.2.1") { + withTempPath { file => + spark.range(0, 10).write + .option("compression", "LZO") + .orc(file.getCanonicalPath) + val expectedCompressionKind = + OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression + assert("LZO" === expectedCompressionKind.name()) } } @@ -214,22 +255,22 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { test("appending") { val data = (0 until 10).map(i => (i, i.toString)) - createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") + createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") withOrcTable(data, "t") { sql("INSERT INTO TABLE t SELECT * FROM tmp") checkAnswer(table("t"), (data ++ data).map(Row.fromTuple)) } - catalog.unregisterTable(TableIdentifier("tmp")) + sessionState.catalog.dropTable(TableIdentifier("tmp"), ignoreIfNotExists = true, purge = false) } test("overwriting") { val data = (0 until 10).map(i => (i, i.toString)) - createDataFrame(data).toDF("c1", "c2").registerTempTable("tmp") + createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") withOrcTable(data, "t") { sql("INSERT OVERWRITE TABLE t SELECT * FROM tmp") checkAnswer(table("t"), data.map(Row.fromTuple)) } - catalog.unregisterTable(TableIdentifier("tmp")) + sessionState.catalog.dropTable(TableIdentifier("tmp"), ignoreIfNotExists = true, purge = false) } test("self-join") { @@ -244,7 +285,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { val queryOutput = selfJoin.queryExecution.analyzed.output assertResult(4, "Field count mismatches")(queryOutput.size) - assertResult(2, "Duplicated expression ID in query plan:\n $selfJoin") { + assertResult(2, s"Duplicated expression ID in query plan:\n $selfJoin") { queryOutput.filter(_.name == "_1").map(_.exprId).size } @@ -253,7 +294,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { } test("nested data - struct with array field") { - val data = (1 to 10).map(i => Tuple1((i, Seq("val_$i")))) + val data = (1 to 10).map(i => Tuple1((i, Seq(s"val_$i")))) withOrcTable(data, "t") { checkAnswer(sql("SELECT `_1`.`_2`[0] FROM t"), data.map { case Tuple1((_, Seq(string))) => Row(string) @@ -262,7 +303,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { } test("nested data - array of struct") { - val data = (1 to 10).map(i => Tuple1(Seq(i -> "val_$i"))) + val data = (1 to 10).map(i => Tuple1(Seq(i -> s"val_$i"))) withOrcTable(data, "t") { checkAnswer(sql("SELECT `_1`[0].`_2` FROM t"), data.map { case Tuple1(Seq((_, string))) => Row(string) @@ -292,12 +333,12 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { withTempPath { dir => val path = dir.getCanonicalPath - sqlContext.range(0, 10).select('id as "Acol").write.format("orc").save(path) - sqlContext.read.format("orc").load(path).schema("Acol") + spark.range(0, 10).select('id as "Acol").write.format("orc").save(path) + spark.read.format("orc").load(path).schema("Acol") intercept[IllegalArgumentException] { - sqlContext.read.format("orc").load(path).schema("acol") + spark.read.format("orc").load(path).schema("acol") } - checkAnswer(sqlContext.read.format("orc").load(path).select("acol").sort("acol"), + checkAnswer(spark.read.format("orc").load(path).select("acol").sort("acol"), (0 until 10).map(Row(_))) } } @@ -307,38 +348,38 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { val path = dir.getCanonicalPath withTable("empty_orc") { - withTempTable("empty", "single") { - sqlContext.sql( + withTempView("empty", "single") { + spark.sql( s"""CREATE TABLE empty_orc(key INT, value STRING) |STORED AS ORC - |LOCATION '$path' + |LOCATION '${dir.toURI}' """.stripMargin) val emptyDF = Seq.empty[(Int, String)].toDF("key", "value").coalesce(1) - emptyDF.registerTempTable("empty") + emptyDF.createOrReplaceTempView("empty") // This creates 1 empty ORC file with Hive ORC SerDe. We are using this trick because // Spark SQL ORC data source always avoids write empty ORC files. - sqlContext.sql( + spark.sql( s"""INSERT INTO TABLE empty_orc |SELECT key, value FROM empty """.stripMargin) val errorMessage = intercept[AnalysisException] { - sqlContext.read.orc(path) + spark.read.orc(path) }.getMessage - assert(errorMessage.contains("Failed to discover schema from ORC files")) + assert(errorMessage.contains("Unable to infer schema for ORC")) val singleRowDF = Seq((0, "foo")).toDF("key", "value").coalesce(1) - singleRowDF.registerTempTable("single") + singleRowDF.createOrReplaceTempView("single") - sqlContext.sql( + spark.sql( s"""INSERT INTO TABLE empty_orc |SELECT key, value FROM single """.stripMargin) - val df = sqlContext.read.orc(path) + val df = spark.read.orc(path) assert(df.schema === singleRowDF.schema.asNullable) checkAnswer(df, singleRowDF) } @@ -350,29 +391,233 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { withTempPath { dir => withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { import testImplicits._ - val path = dir.getCanonicalPath - sqlContext.range(10).coalesce(1).write.orc(path) - val df = sqlContext.read.orc(path) - def checkPredicate(pred: Column, answer: Seq[Long]): Unit = { - checkAnswer(df.where(pred), answer.map(Row(_))) + // For field "a", the first column has odds integers. This is to check the filtered count + // when `isNull` is performed. For Field "b", `isNotNull` of ORC file filters rows + // only when all the values are null (maybe this works differently when the data + // or query is complicated). So, simply here a column only having `null` is added. + val data = (0 until 10).map { i => + val maybeInt = if (i % 2 == 0) None else Some(i) + val nullValue: Option[String] = None + (maybeInt, nullValue) } + // It needs to repartition data so that we can have several ORC files + // in order to skip stripes in ORC. + createDataFrame(data).toDF("a", "b").repartition(10).write.orc(path) + val df = spark.read.orc(path) + + def checkPredicate(pred: Column, answer: Seq[Row]): Unit = { + val sourceDf = stripSparkFilter(df.where(pred)) + val data = sourceDf.collect().toSet + val expectedData = answer.toSet + + // When a filter is pushed to ORC, ORC can apply it to rows. So, we can check + // the number of rows returned from the ORC to make sure our filter pushdown work. + // A tricky part is, ORC does not process filter rows fully but return some possible + // results. So, this checks if the number of result is less than the original count + // of data, and then checks if it contains the expected data. + assert( + sourceDf.count < 10 && expectedData.subsetOf(data), + s"No data was filtered for predicate: $pred") + } + + checkPredicate('a === 5, List(5).map(Row(_, null))) + checkPredicate('a <=> 5, List(5).map(Row(_, null))) + checkPredicate('a < 5, List(1, 3).map(Row(_, null))) + checkPredicate('a <= 5, List(1, 3, 5).map(Row(_, null))) + checkPredicate('a > 5, List(7, 9).map(Row(_, null))) + checkPredicate('a >= 5, List(5, 7, 9).map(Row(_, null))) + checkPredicate('a.isNull, List(null).map(Row(_, null))) + checkPredicate('b.isNotNull, List()) + checkPredicate('a.isin(3, 5, 7), List(3, 5, 7).map(Row(_, null))) + checkPredicate('a > 0 && 'a < 3, List(1).map(Row(_, null))) + checkPredicate('a < 1 || 'a > 8, List(9).map(Row(_, null))) + checkPredicate(!('a > 3), List(1, 3).map(Row(_, null))) + checkPredicate(!('a > 0 && 'a < 3), List(3, 5, 7, 9).map(Row(_, null))) + } + } + } - checkPredicate('id === 5, Seq(5L)) - checkPredicate('id <=> 5, Seq(5L)) - checkPredicate('id < 5, 0L to 4L) - checkPredicate('id <= 5, 0L to 5L) - checkPredicate('id > 5, 6L to 9L) - checkPredicate('id >= 5, 5L to 9L) - checkPredicate('id.isNull, Seq.empty[Long]) - checkPredicate('id.isNotNull, 0L to 9L) - checkPredicate('id.isin(1L, 3L, 5L), Seq(1L, 3L, 5L)) - checkPredicate('id > 0 && 'id < 3, 1L to 2L) - checkPredicate('id < 1 || 'id > 8, Seq(0L, 9L)) - checkPredicate(!('id > 3), 0L to 3L) - checkPredicate(!('id > 0 && 'id < 3), Seq(0L) ++ (3L to 9L)) + test("Verify the ORC conversion parameter: CONVERT_METASTORE_ORC") { + withTempView("single") { + val singleRowDF = Seq((0, "foo")).toDF("key", "value") + singleRowDF.createOrReplaceTempView("single") + + Seq("true", "false").foreach { orcConversion => + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> orcConversion) { + withTable("dummy_orc") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.sql( + s""" + |CREATE TABLE dummy_orc(key INT, value STRING) + |STORED AS ORC + |LOCATION '${dir.toURI}' + """.stripMargin) + + spark.sql( + s""" + |INSERT INTO TABLE dummy_orc + |SELECT key, value FROM single + """.stripMargin) + + val df = spark.sql("SELECT * FROM dummy_orc WHERE key=0") + checkAnswer(df, singleRowDF) + + val queryExecution = df.queryExecution + if (orcConversion == "true") { + queryExecution.analyzed.collectFirst { + case _: LogicalRelation => () + }.getOrElse { + fail(s"Expecting the query plan to convert orc to data sources, " + + s"but got:\n$queryExecution") + } + } else { + queryExecution.analyzed.collectFirst { + case _: HiveTableRelation => () + }.getOrElse { + fail(s"Expecting no conversion from orc to data sources, " + + s"but got:\n$queryExecution") + } + } + } + } + } } } } + + test("converted ORC table supports resolving mixed case field") { + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "true") { + withTable("dummy_orc") { + withTempPath { dir => + val df = spark.range(5).selectExpr("id", "id as valueField", "id as partitionValue") + df.write + .partitionBy("partitionValue") + .mode("overwrite") + .orc(dir.getAbsolutePath) + + spark.sql(s""" + |create external table dummy_orc (id long, valueField long) + |partitioned by (partitionValue int) + |stored as orc + |location "${dir.toURI}"""".stripMargin) + spark.sql(s"msck repair table dummy_orc") + checkAnswer(spark.sql("select * from dummy_orc"), df) + } + } + } + } + + test("SPARK-14962 Produce correct results on array type with isnotnull") { + withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { + val data = (0 until 10).map(i => Tuple1(Array(i))) + withOrcFile(data) { file => + val actual = spark + .read + .orc(file) + .where("_1 is not null") + val expected = data.toDF() + checkAnswer(actual, expected) + } + } + } + + test("SPARK-15198 Support for pushing down filters for boolean types") { + withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { + val data = (0 until 10).map(_ => (true, false)) + withOrcFile(data) { file => + val df = spark.read.orc(file).where("_2 == true") + val actual = stripSparkFilter(df).count() + + // ORC filter should be applied and the total count should be 0. + assert(actual === 0) + } + } + } + + test("Support for pushing down filters for decimal types") { + withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { + val data = (0 until 10).map(i => Tuple1(BigDecimal.valueOf(i))) + withTempPath { file => + // It needs to repartition data so that we can have several ORC files + // in order to skip stripes in ORC. + createDataFrame(data).toDF("a").repartition(10).write.orc(file.getCanonicalPath) + val df = spark.read.orc(file.getCanonicalPath).where("a == 2") + val actual = stripSparkFilter(df).count() + + assert(actual < 10) + } + } + } + + test("Support for pushing down filters for timestamp types") { + withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { + val timeString = "2015-08-20 14:57:00" + val data = (0 until 10).map { i => + val milliseconds = Timestamp.valueOf(timeString).getTime + i * 3600 + Tuple1(new Timestamp(milliseconds)) + } + withTempPath { file => + // It needs to repartition data so that we can have several ORC files + // in order to skip stripes in ORC. + createDataFrame(data).toDF("a").repartition(10).write.orc(file.getCanonicalPath) + val df = spark.read.orc(file.getCanonicalPath).where(s"a == '$timeString'") + val actual = stripSparkFilter(df).count() + + assert(actual < 10) + } + } + } + + test("column nullability and comment - write and then read") { + val schema = (new StructType) + .add("cl1", IntegerType, nullable = false, comment = "test") + .add("cl2", IntegerType, nullable = true) + .add("cl3", IntegerType, nullable = true) + val row = Row(3, null, 4) + val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema) + + val tableName = "tab" + withTable(tableName) { + df.write.format("orc").mode("overwrite").saveAsTable(tableName) + // Verify the DDL command result: DESCRIBE TABLE + checkAnswer( + sql(s"desc $tableName").select("col_name", "comment").where($"comment" === "test"), + Row("cl1", "test") :: Nil) + // Verify the schema + val expectedFields = schema.fields.map(f => f.copy(nullable = true)) + assert(spark.table(tableName).schema == schema.copy(fields = expectedFields)) + } + } + + test("Empty schema does not read data from ORC file") { + val data = Seq((1, 1), (2, 2)) + withOrcFile(data) { path => + val requestedSchema = StructType(Nil) + val conf = new Configuration() + val physicalSchema = OrcFileOperator.readSchema(Seq(path), Some(conf)).get + OrcRelation.setRequiredColumns(conf, physicalSchema, requestedSchema) + val maybeOrcReader = OrcFileOperator.getFileReader(path, Some(conf)) + assert(maybeOrcReader.isDefined) + val orcRecordReader = new SparkOrcNewRecordReader( + maybeOrcReader.get, conf, 0, maybeOrcReader.get.getContentLength) + + val recordsIterator = new RecordReaderIterator[OrcStruct](orcRecordReader) + try { + assert(recordsIterator.next().toString == "{null, null}") + } finally { + recordsIterator.close() + } + } + } + + test("read from multiple orc input paths") { + val path1 = Utils.createTempDir() + val path2 = Utils.createTempDir() + makeOrcFile((1 to 10).map(Tuple1.apply), path1) + makeOrcFile((1 to 10).map(Tuple1.apply), path2) + assertResult(20)(read.orc(path1.getCanonicalPath, path2.getCanonicalPath).count()) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala index 7a34cf731b4c..781de6631f32 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala @@ -23,11 +23,15 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils case class OrcData(intField: Int, stringField: String) abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndAfterAll { - import hiveContext._ + import spark._ var orcTableDir: File = null var orcTableAsDir: File = null @@ -35,21 +39,17 @@ abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndA override def beforeAll(): Unit = { super.beforeAll() - orcTableAsDir = File.createTempFile("orctests", "sparksql") - orcTableAsDir.delete() - orcTableAsDir.mkdir() + orcTableAsDir = Utils.createTempDir("orctests", "sparksql") // Hack: to prepare orc data files using hive external tables - orcTableDir = File.createTempFile("orctests", "sparksql") - orcTableDir.delete() - orcTableDir.mkdir() + orcTableDir = Utils.createTempDir("orctests", "sparksql") import org.apache.spark.sql.hive.test.TestHive.implicits._ sparkContext .makeRDD(1 to 10) .map(i => OrcData(i, s"part-$i")) .toDF() - .registerTempTable(s"orc_temp_table") + .createOrReplaceTempView(s"orc_temp_table") sql( s"""CREATE EXTERNAL TABLE normal_orc( @@ -57,7 +57,7 @@ abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndA | stringField STRING |) |STORED AS ORC - |LOCATION '${orcTableAsDir.getCanonicalPath}' + |LOCATION '${orcTableAsDir.toURI}' """.stripMargin) sql( @@ -66,11 +66,6 @@ abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndA """.stripMargin) } - override def afterAll(): Unit = { - orcTableDir.delete() - orcTableAsDir.delete() - } - test("create temporary orc table") { checkAnswer(sql("SELECT COUNT(*) FROM normal_orc_source"), Row(10)) @@ -130,17 +125,17 @@ abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndA val df = sql( """ |SELECT - | CAST(null as TINYINT), - | CAST(null as SMALLINT), - | CAST(null as INT), - | CAST(null as BIGINT), - | CAST(null as FLOAT), - | CAST(null as DOUBLE), - | CAST(null as DECIMAL(7,2)), - | CAST(null as TIMESTAMP), - | CAST(null as DATE), - | CAST(null as STRING), - | CAST(null as VARCHAR(10)) + | CAST(null as TINYINT) as c0, + | CAST(null as SMALLINT) as c1, + | CAST(null as INT) as c2, + | CAST(null as BIGINT) as c3, + | CAST(null as FLOAT) as c4, + | CAST(null as DOUBLE) as c5, + | CAST(null as DECIMAL(7,2)) as c6, + | CAST(null as TIMESTAMP) as c7, + | CAST(null as DATE) as c8, + | CAST(null as STRING) as c9, + | CAST(null as VARCHAR(10)) as c10 |FROM orc_temp_table limit 1 """.stripMargin) @@ -152,26 +147,131 @@ abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndA sql("DROP TABLE IF EXISTS orcNullValues") } + + test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") { + val conf = sqlContext.sessionState.conf + assert(new OrcOptions(Map("Orc.Compress" -> "NONE"), conf).compressionCodec == "NONE") + } + + test("SPARK-19459/SPARK-18220: read char/varchar column written by Hive") { + val location = Utils.createTempDir() + val uri = location.toURI + try { + hiveClient.runSqlHive("USE default") + hiveClient.runSqlHive( + """ + |CREATE EXTERNAL TABLE hive_orc( + | a STRING, + | b CHAR(10), + | c VARCHAR(10), + | d ARRAY) + |STORED AS orc""".stripMargin) + // Hive throws an exception if I assign the location in the create table statement. + hiveClient.runSqlHive( + s"ALTER TABLE hive_orc SET LOCATION '$uri'") + hiveClient.runSqlHive( + """ + |INSERT INTO TABLE hive_orc + |SELECT 'a', 'b', 'c', ARRAY(CAST('d' AS CHAR(3))) + |FROM (SELECT 1) t""".stripMargin) + + // We create a different table in Spark using the same schema which points to + // the same location. + spark.sql( + s""" + |CREATE EXTERNAL TABLE spark_orc( + | a STRING, + | b CHAR(10), + | c VARCHAR(10), + | d ARRAY) + |STORED AS orc + |LOCATION '$uri'""".stripMargin) + val result = Row("a", "b ", "c", Seq("d ")) + checkAnswer(spark.table("hive_orc"), result) + checkAnswer(spark.table("spark_orc"), result) + } finally { + hiveClient.runSqlHive("DROP TABLE IF EXISTS hive_orc") + hiveClient.runSqlHive("DROP TABLE IF EXISTS spark_orc") + Utils.deleteRecursively(location) + } + } + + test("SPARK-21839: Add SQL config for ORC compression") { + val conf = sqlContext.sessionState.conf + // Test if the default of spark.sql.orc.compression.codec is snappy + assert(new OrcOptions(Map.empty[String, String], conf).compressionCodec == "SNAPPY") + + // OrcOptions's parameters have a higher priority than SQL configuration. + // `compression` -> `orc.compression` -> `spark.sql.orc.compression.codec` + withSQLConf(SQLConf.ORC_COMPRESSION.key -> "uncompressed") { + assert(new OrcOptions(Map.empty[String, String], conf).compressionCodec == "NONE") + val map1 = Map("orc.compress" -> "zlib") + val map2 = Map("orc.compress" -> "zlib", "compression" -> "lzo") + assert(new OrcOptions(map1, conf).compressionCodec == "ZLIB") + assert(new OrcOptions(map2, conf).compressionCodec == "LZO") + } + + // Test all the valid options of spark.sql.orc.compression.codec + Seq("NONE", "UNCOMPRESSED", "SNAPPY", "ZLIB", "LZO").foreach { c => + withSQLConf(SQLConf.ORC_COMPRESSION.key -> c) { + val expected = if (c == "UNCOMPRESSED") "NONE" else c + assert(new OrcOptions(Map.empty[String, String], conf).compressionCodec == expected) + } + } + } } class OrcSourceSuite extends OrcSuite { override def beforeAll(): Unit = { super.beforeAll() - hiveContext.sql( - s"""CREATE TEMPORARY TABLE normal_orc_source + spark.sql( + s"""CREATE TEMPORARY VIEW normal_orc_source |USING org.apache.spark.sql.hive.orc |OPTIONS ( - | PATH '${new File(orcTableAsDir.getAbsolutePath).getCanonicalPath}' + | PATH '${new File(orcTableAsDir.getAbsolutePath).toURI}' |) """.stripMargin) - hiveContext.sql( - s"""CREATE TEMPORARY TABLE normal_orc_as_source + spark.sql( + s"""CREATE TEMPORARY VIEW normal_orc_as_source |USING org.apache.spark.sql.hive.orc |OPTIONS ( - | PATH '${new File(orcTableAsDir.getAbsolutePath).getCanonicalPath}' + | PATH '${new File(orcTableAsDir.getAbsolutePath).toURI}' |) """.stripMargin) } + + test("SPARK-12218 Converting conjunctions into ORC SearchArguments") { + // The `LessThan` should be converted while the `StringContains` shouldn't + val schema = new StructType( + Array( + StructField("a", IntegerType, nullable = true), + StructField("b", StringType, nullable = true))) + assertResult( + """leaf-0 = (LESS_THAN a 10) + |expr = leaf-0 + """.stripMargin.trim + ) { + OrcFilters.createFilter(schema, Array( + LessThan("a", 10), + StringContains("b", "prefix") + )).get.toString + } + + // The `LessThan` should be converted while the whole inner `And` shouldn't + assertResult( + """leaf-0 = (LESS_THAN a 10) + |expr = leaf-0 + """.stripMargin.trim + ) { + OrcFilters.createFilter(schema, Array( + LessThan("a", 10), + Not(And( + GreaterThan("a", 1), + StringContains("b", "prefix") + )) + )).get.toString + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala index 88a0ed511749..a2f08c5ba72c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcTest.scala @@ -23,8 +23,8 @@ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag import org.apache.spark.sql._ -import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils private[sql] trait OrcTest extends SQLTestUtils with TestHiveSingleton { import testImplicits._ @@ -43,17 +43,17 @@ private[sql] trait OrcTest extends SQLTestUtils with TestHiveSingleton { } /** - * Writes `data` to a Orc file and reads it back as a [[DataFrame]], + * Writes `data` to a Orc file and reads it back as a `DataFrame`, * which is then passed to `f`. The Orc file will be deleted after `f` returns. */ protected def withOrcDataFrame[T <: Product: ClassTag: TypeTag] (data: Seq[T]) (f: DataFrame => Unit): Unit = { - withOrcFile(data)(path => f(sqlContext.read.orc(path))) + withOrcFile(data)(path => f(spark.read.orc(path))) } /** - * Writes `data` to a Orc file, reads it back as a [[DataFrame]] and registers it as a + * Writes `data` to a Orc file, reads it back as a `DataFrame` and registers it as a * temporary table named `tableName`, then call `f`. The temporary table together with the * Orc file will be dropped/deleted after `f` returns. */ @@ -61,8 +61,8 @@ private[sql] trait OrcTest extends SQLTestUtils with TestHiveSingleton { (data: Seq[T], tableName: String) (f: => Unit): Unit = { withOrcDataFrame(data) { df => - sqlContext.registerDataFrameAsTable(df, tableName) - withTempTable(tableName)(f) + df.createOrReplaceTempView(tableName) + withTempView(tableName)(f) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 905eb7a3925b..740e0837350c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -20,11 +20,14 @@ package org.apache.spark.sql.hive import java.io.File import org.apache.spark.sql._ -import org.apache.spark.sql.execution.datasources.{InsertIntoDataSource, InsertIntoHadoopFsRelation, LogicalRelation} -import org.apache.spark.sql.execution.{ExecutedCommand, PhysicalRDD} -import org.apache.spark.sql.hive.execution.HiveTableScan +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.HiveTableRelation +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.DataSourceScanExec +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.hive.execution.HiveTableScanExec import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -55,6 +58,7 @@ case class ParquetDataWithKeyAndComplexTypes( */ class ParquetMetastoreSuite extends ParquetPartitioningTest { import hiveContext._ + import spark.implicits._ override def beforeAll(): Unit = { super.beforeAll() @@ -77,7 +81,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' - location '${partitionedTableDir.getCanonicalPath}' + location '${partitionedTableDir.toURI}' """) sql(s""" @@ -91,7 +95,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' - location '${partitionedTableDirWithKey.getCanonicalPath}' + location '${partitionedTableDirWithKey.toURI}' """) sql(s""" @@ -104,7 +108,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' - location '${new File(normalTableDir, "normal").getCanonicalPath}' + location '${new File(normalTableDir, "normal").toURI}' """) sql(s""" @@ -120,7 +124,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' - LOCATION '${partitionedTableDirWithComplexTypes.getCanonicalPath}' + LOCATION '${partitionedTableDirWithComplexTypes.toURI}' """) sql(s""" @@ -136,7 +140,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' - LOCATION '${partitionedTableDirWithKeyAndComplexTypes.getCanonicalPath}' + LOCATION '${partitionedTableDirWithKeyAndComplexTypes.toURI}' """) sql( @@ -168,12 +172,11 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { sql(s"ALTER TABLE partitioned_parquet_with_complextypes ADD PARTITION (p=$p)") } - val rdd1 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""")) - read.json(rdd1).registerTempTable("jt") - val rdd2 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":[$i, null]}""")) - read.json(rdd2).registerTempTable("jt_array") + (1 to 10).map(i => (i, s"str$i")).toDF("a", "b").createOrReplaceTempView("jt") + (1 to 10).map(i => Tuple1(Seq(new Integer(i), null))).toDF("a") + .createOrReplaceTempView("jt_array") - setConf(HiveContext.CONVERT_METASTORE_PARQUET, true) + assert(spark.sqlContext.getConf(HiveUtils.CONVERT_METASTORE_PARQUET.key) == "true") } override def afterAll(): Unit = { @@ -184,18 +187,18 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { "normal_parquet", "jt", "jt_array", - "test_parquet") - setConf(HiveContext.CONVERT_METASTORE_PARQUET, false) + "test_parquet") + super.afterAll() } test(s"conversion is working") { assert( - sql("SELECT * FROM normal_parquet").queryExecution.executedPlan.collect { - case _: HiveTableScan => true + sql("SELECT * FROM normal_parquet").queryExecution.sparkPlan.collect { + case _: HiveTableScanExec => true }.isEmpty) assert( - sql("SELECT * FROM normal_parquet").queryExecution.executedPlan.collect { - case _: PhysicalRDD => true + sql("SELECT * FROM normal_parquet").queryExecution.sparkPlan.collect { + case _: DataSourceScanExec => true }.nonEmpty) } @@ -282,10 +285,10 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { ) table("test_parquet_ctas").queryExecution.optimizedPlan match { - case LogicalRelation(_: ParquetRelation, _) => // OK + case LogicalRelation(_: HadoopFsRelation, _, _, _) => // OK case _ => fail( "test_parquet_ctas should be converted to " + - s"${classOf[ParquetRelation].getCanonicalName }") + s"${classOf[HadoopFsRelation ].getCanonicalName }") } } } @@ -305,12 +308,11 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { """.stripMargin) val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt") - df.queryExecution.executedPlan match { - case ExecutedCommand(InsertIntoHadoopFsRelation(_: ParquetRelation, _, _)) => // OK + df.queryExecution.analyzed match { + case cmd: InsertIntoHadoopFsRelationCommand => + assert(cmd.catalogTable.map(_.identifier.table) === Some("test_insert_parquet")) case o => fail("test_insert_parquet should be converted to a " + - s"${classOf[ParquetRelation].getCanonicalName} and " + - s"${classOf[InsertIntoDataSource].getCanonicalName} is expcted as the SparkPlan. " + - s"However, found a ${o.toString} ") + s"${classOf[HadoopFsRelation ].getCanonicalName}. However, found a ${o.toString}") } checkAnswer( @@ -335,12 +337,11 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { """.stripMargin) val df = sql("INSERT INTO TABLE test_insert_parquet SELECT a FROM jt_array") - df.queryExecution.executedPlan match { - case ExecutedCommand(InsertIntoHadoopFsRelation(r: ParquetRelation, _, _)) => // OK + df.queryExecution.analyzed match { + case cmd: InsertIntoHadoopFsRelationCommand => + assert(cmd.catalogTable.map(_.identifier.table) === Some("test_insert_parquet")) case o => fail("test_insert_parquet should be converted to a " + - s"${classOf[ParquetRelation].getCanonicalName} and " + - s"${classOf[InsertIntoDataSource].getCanonicalName} is expcted as the SparkPlan." + - s"However, found a ${o.toString} ") + s"${classOf[HadoopFsRelation ].getCanonicalName}. However, found a ${o.toString}") } checkAnswer( @@ -369,35 +370,36 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { assertResult(2) { analyzed.collect { - case r @ LogicalRelation(_: ParquetRelation, _) => r + case r @ LogicalRelation(_: HadoopFsRelation, _, _, _) => r }.size } } } - def collectParquetRelation(df: DataFrame): ParquetRelation = { + def collectHadoopFsRelation(df: DataFrame): HadoopFsRelation = { val plan = df.queryExecution.analyzed plan.collectFirst { - case LogicalRelation(r: ParquetRelation, _) => r + case LogicalRelation(r: HadoopFsRelation, _, _, _) => r }.getOrElse { - fail(s"Expecting a ParquetRelation2, but got:\n$plan") + fail(s"Expecting a HadoopFsRelation 2, but got:\n$plan") } } test("SPARK-7749: non-partitioned metastore Parquet table lookup should use cached relation") { withTable("nonPartitioned") { sql( - s"""CREATE TABLE nonPartitioned ( - | key INT, - | value STRING - |) - |STORED AS PARQUET - """.stripMargin) + """ + |CREATE TABLE nonPartitioned ( + | key INT, + | value STRING + |) + |STORED AS PARQUET + """.stripMargin) // First lookup fills the cache - val r1 = collectParquetRelation(table("nonPartitioned")) + val r1 = collectHadoopFsRelation(table("nonPartitioned")) // Second lookup should reuse the cache - val r2 = collectParquetRelation(table("nonPartitioned")) + val r2 = collectHadoopFsRelation(table("nonPartitioned")) // They should be the same instance assert(r1 eq r2) } @@ -406,29 +408,58 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { test("SPARK-7749: partitioned metastore Parquet table lookup should use cached relation") { withTable("partitioned") { sql( - s"""CREATE TABLE partitioned ( - | key INT, - | value STRING - |) - |PARTITIONED BY (part INT) - |STORED AS PARQUET - """.stripMargin) + """ + |CREATE TABLE partitioned ( + | key INT, + | value STRING + |) + |PARTITIONED BY (part INT) + |STORED AS PARQUET + """.stripMargin) + + // First lookup fills the cache + val r1 = collectHadoopFsRelation(table("partitioned")) + // Second lookup should reuse the cache + val r2 = collectHadoopFsRelation(table("partitioned")) + // They should be the same instance + assert(r1 eq r2) + } + } + + test("SPARK-15968: nonempty partitioned metastore Parquet table lookup should use cached " + + "relation") { + withTable("partitioned") { + sql( + """ + |CREATE TABLE partitioned ( + | key INT, + | value STRING + |) + |PARTITIONED BY (part INT) + |STORED AS PARQUET + """.stripMargin) + sql("INSERT INTO TABLE partitioned PARTITION(part=0) SELECT 1 as key, 'one' as value") // First lookup fills the cache - val r1 = collectParquetRelation(table("partitioned")) + val r1 = collectHadoopFsRelation(table("partitioned")) // Second lookup should reuse the cache - val r2 = collectParquetRelation(table("partitioned")) + val r2 = collectHadoopFsRelation(table("partitioned")) // They should be the same instance assert(r1 eq r2) } } + private def getCachedDataSourceTable(table: TableIdentifier): LogicalPlan = { + sessionState.catalog.asInstanceOf[HiveSessionCatalog].metastoreCatalog + .getCachedDataSourceTable(table) + } + test("Caching converted data source Parquet Relations") { - def checkCached(tableIdentifier: catalog.QualifiedTableName): Unit = { + def checkCached(tableIdentifier: TableIdentifier): Unit = { // Converted test_parquet should be cached. - catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) match { + getCachedDataSourceTable(tableIdentifier) match { case null => fail("Converted test_parquet should be cached in the cache.") - case logical @ LogicalRelation(parquetRelation: ParquetRelation, _) => // OK + case LogicalRelation(_: HadoopFsRelation, _, _, _) => // OK case other => fail( "The cached test_parquet should be a Parquet Relation. " + @@ -451,17 +482,17 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' """.stripMargin) - var tableIdentifier = catalog.QualifiedTableName("default", "test_insert_parquet") + var tableIdentifier = TableIdentifier("test_insert_parquet", Some("default")) // First, make sure the converted test_parquet is not cached. - assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null) + assert(getCachedDataSourceTable(tableIdentifier) === null) // Table lookup will make the table cached. table("test_insert_parquet") checkCached(tableIdentifier) // For insert into non-partitioned table, we will do the conversion, // so the converted test_insert_parquet should be cached. - invalidateTable("test_insert_parquet") - assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null) + sessionState.refreshTable("test_insert_parquet") + assert(getCachedDataSourceTable(tableIdentifier) === null) sql( """ |INSERT INTO TABLE test_insert_parquet @@ -473,8 +504,8 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { sql("select * from test_insert_parquet"), sql("select a, b from jt").collect()) // Invalidate the cache. - invalidateTable("test_insert_parquet") - assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null) + sessionState.refreshTable("test_insert_parquet") + assert(getCachedDataSourceTable(tableIdentifier) === null) // Create a partitioned table. sql( @@ -491,8 +522,8 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' """.stripMargin) - tableIdentifier = catalog.QualifiedTableName("default", "test_parquet_partitioned_cache_test") - assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null) + tableIdentifier = TableIdentifier("test_parquet_partitioned_cache_test", Some("default")) + assert(getCachedDataSourceTable(tableIdentifier) === null) sql( """ |INSERT INTO TABLE test_parquet_partitioned_cache_test @@ -501,14 +532,14 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { """.stripMargin) // Right now, insert into a partitioned Parquet is not supported in data source Parquet. // So, we expect it is not cached. - assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null) + assert(getCachedDataSourceTable(tableIdentifier) === null) sql( """ |INSERT INTO TABLE test_parquet_partitioned_cache_test |PARTITION (`date`='2015-04-02') |select a, b from jt """.stripMargin) - assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null) + assert(getCachedDataSourceTable(tableIdentifier) === null) // Make sure we can cache the partitioned table. table("test_parquet_partitioned_cache_test") @@ -523,11 +554,126 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { |select b, '2015-04-02', a FROM jt """.stripMargin).collect()) - invalidateTable("test_parquet_partitioned_cache_test") - assert(catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) === null) + sessionState.refreshTable("test_parquet_partitioned_cache_test") + assert(getCachedDataSourceTable(tableIdentifier) === null) dropTables("test_insert_parquet", "test_parquet_partitioned_cache_test") } + + test("SPARK-15248: explicitly added partitions should be readable") { + withTable("test_added_partitions", "test_temp") { + withTempDir { src => + val partitionDir = new File(src, "partition").toURI + sql( + """ + |CREATE TABLE test_added_partitions (a STRING) + |PARTITIONED BY (b INT) + |STORED AS PARQUET + """.stripMargin) + + // Temp view that is used to insert data into partitioned table + Seq("foo", "bar").toDF("a").createOrReplaceTempView("test_temp") + sql("INSERT INTO test_added_partitions PARTITION(b='0') SELECT a FROM test_temp") + + checkAnswer( + sql("SELECT * FROM test_added_partitions"), + Seq(Row("foo", 0), Row("bar", 0))) + + // Create partition without data files and check whether it can be read + sql(s"ALTER TABLE test_added_partitions ADD PARTITION (b='1') LOCATION '$partitionDir'") + checkAnswer( + sql("SELECT * FROM test_added_partitions"), + Seq(Row("foo", 0), Row("bar", 0))) + + // Add data files to partition directory and check whether they can be read + sql("INSERT INTO TABLE test_added_partitions PARTITION (b=1) select 'baz' as a") + checkAnswer( + sql("SELECT * FROM test_added_partitions"), + Seq(Row("foo", 0), Row("bar", 0), Row("baz", 1))) + + // Check it with pruning predicates + checkAnswer( + sql("SELECT * FROM test_added_partitions where b = 0"), + Seq(Row("foo", 0), Row("bar", 0))) + checkAnswer( + sql("SELECT * FROM test_added_partitions where b = 1"), + Seq(Row("baz", 1))) + checkAnswer( + sql("SELECT * FROM test_added_partitions where b = 2"), + Seq.empty) + + // Also verify the inputFiles implementation + assert(sql("select * from test_added_partitions").inputFiles.length == 2) + assert(sql("select * from test_added_partitions where b = 0").inputFiles.length == 1) + assert(sql("select * from test_added_partitions where b = 1").inputFiles.length == 1) + assert(sql("select * from test_added_partitions where b = 2").inputFiles.length == 0) + } + } + } + + test("Explicitly added partitions should be readable after load") { + withTable("test_added_partitions") { + withTempDir { src => + val newPartitionDir = src.toURI.toString + spark.range(2).selectExpr("cast(id as string)").toDF("a").write + .mode("overwrite") + .parquet(newPartitionDir) + + sql( + """ + |CREATE TABLE test_added_partitions (a STRING) + |PARTITIONED BY (b INT) + |STORED AS PARQUET + """.stripMargin) + + // Create partition without data files and check whether it can be read + sql(s"ALTER TABLE test_added_partitions ADD PARTITION (b='1')") + // This table fetch is to fill the cache with zero leaf files + checkAnswer(spark.table("test_added_partitions"), Seq.empty) + + sql( + s""" + |LOAD DATA LOCAL INPATH '$newPartitionDir' OVERWRITE + |INTO TABLE test_added_partitions PARTITION(b='1') + """.stripMargin) + + checkAnswer( + spark.table("test_added_partitions"), + Seq(Row("0", 1), Row("1", 1))) + } + } + } + + test("Non-partitioned table readable after load") { + withTable("tab") { + withTempDir { src => + val newPartitionDir = src.toURI.toString + spark.range(2).selectExpr("cast(id as string)").toDF("a").write + .mode("overwrite") + .parquet(newPartitionDir) + + sql("CREATE TABLE tab (a STRING) STORED AS PARQUET") + + // This table fetch is to fill the cache with zero leaf files + checkAnswer(spark.table("tab"), Seq.empty) + + sql( + s""" + |LOAD DATA LOCAL INPATH '$newPartitionDir' OVERWRITE + |INTO TABLE tab + """.stripMargin) + + checkAnswer(spark.table("tab"), Seq(Row("0"), Row("1"))) + } + } + } + + test("self-join") { + val table = spark.table("normal_parquet") + val selfJoin = table.as("t1").crossJoin(table.as("t2")) + checkAnswer(selfJoin, + sql("SELECT * FROM normal_parquet x CROSS JOIN normal_parquet y")) + } } /** @@ -535,7 +681,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { */ class ParquetSourceSuite extends ParquetPartitioningTest { import testImplicits._ - import hiveContext._ + import spark._ override def beforeAll(): Unit = { super.beforeAll() @@ -546,42 +692,42 @@ class ParquetSourceSuite extends ParquetPartitioningTest { "normal_parquet") sql( s""" - create temporary table partitioned_parquet + CREATE TEMPORARY VIEW partitioned_parquet USING org.apache.spark.sql.parquet OPTIONS ( - path '${partitionedTableDir.getCanonicalPath}' + path '${partitionedTableDir.toURI}' ) """) sql( s""" - create temporary table partitioned_parquet_with_key + CREATE TEMPORARY VIEW partitioned_parquet_with_key USING org.apache.spark.sql.parquet OPTIONS ( - path '${partitionedTableDirWithKey.getCanonicalPath}' + path '${partitionedTableDirWithKey.toURI}' ) """) sql( s""" - create temporary table normal_parquet + CREATE TEMPORARY VIEW normal_parquet USING org.apache.spark.sql.parquet OPTIONS ( - path '${new File(partitionedTableDir, "p=1").getCanonicalPath}' + path '${new File(partitionedTableDir, "p=1").toURI}' ) """) sql( s""" - CREATE TEMPORARY TABLE partitioned_parquet_with_key_and_complextypes + CREATE TEMPORARY VIEW partitioned_parquet_with_key_and_complextypes USING org.apache.spark.sql.parquet OPTIONS ( - path '${partitionedTableDirWithKeyAndComplexTypes.getCanonicalPath}' + path '${partitionedTableDirWithKeyAndComplexTypes.toURI}' ) """) sql( s""" - CREATE TEMPORARY TABLE partitioned_parquet_with_complextypes + CREATE TEMPORARY VIEW partitioned_parquet_with_complextypes USING org.apache.spark.sql.parquet OPTIONS ( - path '${partitionedTableDirWithComplexTypes.getCanonicalPath}' + path '${partitionedTableDirWithComplexTypes.toURI}' ) """) } @@ -590,7 +736,7 @@ class ParquetSourceSuite extends ParquetPartitioningTest { sql("drop table if exists spark_6016_fix") // Create a DataFrame with two partitions. So, the created table will have two parquet files. - val df1 = read.json(sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i}"""), 2)) + val df1 = (1 to 10).map(Tuple1(_)).toDF("a").coalesce(2) df1.write.mode(SaveMode.Overwrite).format("parquet").saveAsTable("spark_6016_fix") checkAnswer( sql("select * from spark_6016_fix"), @@ -598,7 +744,7 @@ class ParquetSourceSuite extends ParquetPartitioningTest { ) // Create a DataFrame with four partitions. So, the created table will have four parquet files. - val df2 = read.json(sparkContext.parallelize((1 to 10).map(i => s"""{"b":$i}"""), 4)) + val df2 = (1 to 10).map(Tuple1(_)).toDF("b").coalesce(4) df2.write.mode(SaveMode.Overwrite).format("parquet").saveAsTable("spark_6016_fix") // For the bug of SPARK-6016, we are caching two outdated footers for df1. Then, // since the new table has four parquet files, we are trying to read new footers from two files @@ -614,29 +760,70 @@ class ParquetSourceSuite extends ParquetPartitioningTest { test("SPARK-8811: compatibility with array of struct in Hive") { withTempPath { dir => - val path = dir.getCanonicalPath - withTable("array_of_struct") { val conf = Seq( - HiveContext.CONVERT_METASTORE_PARQUET.key -> "false", + HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false", SQLConf.PARQUET_BINARY_AS_STRING.key -> "true", SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> "false") withSQLConf(conf: _*) { sql( s"""CREATE TABLE array_of_struct - |STORED AS PARQUET LOCATION '$path' - |AS SELECT '1st', '2nd', ARRAY(NAMED_STRUCT('a', 'val_a', 'b', 'val_b')) + |STORED AS PARQUET LOCATION '${dir.toURI}' + |AS SELECT + | '1st' AS a, + | '2nd' AS b, + | ARRAY(NAMED_STRUCT('a', 'val_a', 'b', 'val_b')) AS c """.stripMargin) checkAnswer( - sqlContext.read.parquet(path), + spark.read.parquet(dir.getCanonicalPath), Row("1st", "2nd", Seq(Row("val_a", "val_b")))) } } } } + test("Verify the PARQUET conversion parameter: CONVERT_METASTORE_PARQUET") { + withTempView("single") { + val singleRowDF = Seq((0, "foo")).toDF("key", "value") + singleRowDF.createOrReplaceTempView("single") + + Seq("true", "false").foreach { parquetConversion => + withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> parquetConversion) { + val tableName = "test_parquet_ctas" + withTable(tableName) { + sql( + s""" + |CREATE TABLE $tableName STORED AS PARQUET + |AS SELECT tmp.key, tmp.value FROM single tmp + """.stripMargin) + + val df = spark.sql(s"SELECT * FROM $tableName WHERE key=0") + checkAnswer(df, singleRowDF) + + val queryExecution = df.queryExecution + if (parquetConversion == "true") { + queryExecution.analyzed.collectFirst { + case _: LogicalRelation => + }.getOrElse { + fail(s"Expecting the query plan to convert parquet to data sources, " + + s"but got:\n$queryExecution") + } + } else { + queryExecution.analyzed.collectFirst { + case _: HiveTableRelation => + }.getOrElse { + fail(s"Expecting no conversion from parquet to data sources, " + + s"but got:\n$queryExecution") + } + } + } + } + } + } + } + test("values in arrays and maps stored in parquet are always nullable") { val df = createDataFrame(Tuple2(Map(2 -> 3), Seq(4, 5, 6)) :: Nil).toDF("m", "a") val mapType1 = MapType(IntegerType, IntegerType, valueContainsNull = false) @@ -695,6 +882,7 @@ abstract class ParquetPartitioningTest extends QueryTest with SQLTestUtils with var partitionedTableDirWithKeyAndComplexTypes: File = null override def beforeAll(): Unit = { + super.beforeAll() partitionedTableDir = Utils.createTempDir() normalTableDir = Utils.createTempDir() @@ -752,6 +940,7 @@ abstract class ParquetPartitioningTest extends QueryTest with SQLTestUtils with /** * Drop named tables if they exist + * * @param tableNames tables to drop */ def dropTables(tableNames: String*): Unit = { @@ -849,7 +1038,7 @@ abstract class ParquetPartitioningTest extends QueryTest with SQLTestUtils with test(s"hive udfs $table") { checkAnswer( sql(s"SELECT concat(stringField, stringField) FROM $table"), - sql(s"SELECT stringField FROM $table").map { + sql(s"SELECT stringField FROM $table").rdd.map { case Row(s: String) => Row(s + s) }.collect().toSeq) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadWithHiveSupportSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadWithHiveSupportSuite.scala new file mode 100644 index 000000000000..f277f99805a4 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadWithHiveSupportSuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION + +class BucketedReadWithHiveSupportSuite extends BucketedReadSuite with TestHiveSingleton { + protected override def beforeAll(): Unit = { + super.beforeAll() + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive") + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteWithHiveSupportSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteWithHiveSupportSuite.scala new file mode 100644 index 000000000000..454e2f65d5d8 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteWithHiveSupportSuite.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.sources + +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION + +class BucketedWriteWithHiveSupportSuite extends BucketedWriteSuite with TestHiveSingleton { + protected override def beforeAll(): Unit = { + super.beforeAll() + assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive") + } + + override protected def fileFormatsToTest: Seq[String] = Seq("parquet", "orc") +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestRelationSuite.scala index dc0531a6d4bc..f9387fae4a4c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestRelationSuite.scala @@ -18,14 +18,14 @@ package org.apache.spark.sql.sources import org.apache.hadoop.fs.Path + import org.apache.spark.SparkException import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils - -class CommitFailureTestRelationSuite extends SQLTestUtils with TestHiveSingleton { - +class CommitFailureTestRelationSuite extends SQLTestUtils with TestHiveSingleton { // When committing a task, `CommitFailureTestSource` throws an exception for testing purpose. val dataSourceName: String = classOf[CommitFailureTestSource].getCanonicalName @@ -34,10 +34,46 @@ class CommitFailureTestRelationSuite extends SQLTestUtils with TestHiveSingleton // Here we coalesce partition number to 1 to ensure that only a single task is issued. This // prevents race condition happened when FileOutputCommitter tries to remove the `_temporary` // directory while committing/aborting the job. See SPARK-8513 for more details. - val df = sqlContext.range(0, 10).coalesce(1) + val df = spark.range(0, 10).coalesce(1) + intercept[SparkException] { + df.write.format(dataSourceName).save(file.getCanonicalPath) + } + + val fs = new Path(file.getCanonicalPath).getFileSystem(SparkHadoopUtil.get.conf) + assert(!fs.exists(new Path(file.getCanonicalPath, "_temporary"))) + } + } + + test("call failure callbacks before close writer - default") { + SimpleTextRelation.failCommitter = false + withTempPath { file => + // fail the job in the middle of writing + val divideByZero = udf((x: Int) => { x / (x - 1)}) + val df = spark.range(0, 10).coalesce(1).select(divideByZero(col("id"))) + + SimpleTextRelation.callbackCalled = false intercept[SparkException] { df.write.format(dataSourceName).save(file.getCanonicalPath) } + assert(SimpleTextRelation.callbackCalled, "failure callback should be called") + + val fs = new Path(file.getCanonicalPath).getFileSystem(SparkHadoopUtil.get.conf) + assert(!fs.exists(new Path(file.getCanonicalPath, "_temporary"))) + } + } + + test("call failure callbacks before close writer - partitioned") { + SimpleTextRelation.failCommitter = false + withTempPath { file => + // fail the job in the middle of writing + val df = spark.range(0, 10).coalesce(1).select(col("id").mod(2).as("key"), col("id")) + + SimpleTextRelation.callbackCalled = false + SimpleTextRelation.failWriter = true + intercept[SparkException] { + df.write.format(dataSourceName).partitionBy("key").save(file.getCanonicalPath) + } + assert(SimpleTextRelation.callbackCalled, "failure callback should be called") val fs = new Path(file.getCanonicalPath).getFileSystem(SparkHadoopUtil.get.conf) assert(!fs.exists(new Path(file.getCanonicalPath, "_temporary"))) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala new file mode 100644 index 000000000000..7501334f94dd --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} + +import org.apache.spark.TaskContext +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory} +import org.apache.spark.sql.types.StructType + +class CommitFailureTestSource extends SimpleTextSource { + /** + * Prepares a write job and returns an + * [[org.apache.spark.sql.execution.datasources.OutputWriterFactory]]. + * Client side job preparation can + * be put here. For example, user defined output committer can be configured here + * by setting the output committer class in the conf of spark.sql.sources.outputCommitterClass. + */ + override def prepareWrite( + sparkSession: SparkSession, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = + new OutputWriterFactory { + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new SimpleTextOutputWriter(path, dataSchema, context) { + var failed = false + TaskContext.get().addTaskFailureListener { (t: TaskContext, e: Throwable) => + failed = true + SimpleTextRelation.callbackCalled = true + } + + override def write(row: InternalRow): Unit = { + if (SimpleTextRelation.failWriter) { + sys.error("Intentional task writer failure for testing purpose.") + + } + super.write(row) + } + + override def close(): Unit = { + super.close() + sys.error("Intentional task commitment failure for testing purpose.") + } + } + } + + override def getFileExtension(context: TaskAttemptContext): String = "" + } + + override def shortName(): String = "commit-failure-test" +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala new file mode 100644 index 000000000000..80aff446bc24 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -0,0 +1,851 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 scala.util.Random + +import org.apache.hadoop.fs.Path + +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.sql._ +import org.apache.spark.sql.execution.DataSourceScanExec +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types._ + + +abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with TestHiveSingleton { + import spark.implicits._ + + val dataSourceName: String + + protected def supportsDataType(dataType: DataType): Boolean = true + + val dataSchema = + StructType( + Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", StringType, nullable = false))) + + lazy val testDF = (1 to 3).map(i => (i, s"val_$i")).toDF("a", "b") + + lazy val partitionedTestDF1 = (for { + i <- 1 to 3 + p2 <- Seq("foo", "bar") + } yield (i, s"val_$i", 1, p2)).toDF("a", "b", "p1", "p2") + + lazy val partitionedTestDF2 = (for { + i <- 1 to 3 + p2 <- Seq("foo", "bar") + } yield (i, s"val_$i", 2, p2)).toDF("a", "b", "p1", "p2") + + lazy val partitionedTestDF = partitionedTestDF1.union(partitionedTestDF2) + + def checkQueries(df: DataFrame): Unit = { + // Selects everything + checkAnswer( + df, + for (i <- 1 to 3; p1 <- 1 to 2; p2 <- Seq("foo", "bar")) yield Row(i, s"val_$i", p1, p2)) + + // Simple filtering and partition pruning + checkAnswer( + df.filter('a > 1 && 'p1 === 2), + for (i <- 2 to 3; p2 <- Seq("foo", "bar")) yield Row(i, s"val_$i", 2, p2)) + + // Simple projection and filtering + checkAnswer( + df.filter('a > 1).select('b, 'a + 1), + for (i <- 2 to 3; _ <- 1 to 2; _ <- Seq("foo", "bar")) yield Row(s"val_$i", i + 1)) + + // Simple projection and partition pruning + checkAnswer( + df.filter('a > 1 && 'p1 < 2).select('b, 'p1), + for (i <- 2 to 3; _ <- Seq("foo", "bar")) yield Row(s"val_$i", 1)) + + // Project many copies of columns with different types (reproduction for SPARK-7858) + checkAnswer( + df.filter('a > 1 && 'p1 < 2).select('b, 'b, 'b, 'b, 'p1, 'p1, 'p1, 'p1), + for (i <- 2 to 3; _ <- Seq("foo", "bar")) + yield Row(s"val_$i", s"val_$i", s"val_$i", s"val_$i", 1, 1, 1, 1)) + + // Self-join + df.createOrReplaceTempView("t") + withTempView("t") { + checkAnswer( + sql( + """SELECT l.a, r.b, l.p1, r.p2 + |FROM t l JOIN t r + |ON l.a = r.a AND l.p1 = r.p1 AND l.p2 = r.p2 + """.stripMargin), + for (i <- 1 to 3; p1 <- 1 to 2; p2 <- Seq("foo", "bar")) yield Row(i, s"val_$i", p1, p2)) + } + } + + private val supportedDataTypes = Seq( + StringType, BinaryType, + NullType, BooleanType, + ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), + DateType, TimestampType, + ArrayType(IntegerType), + MapType(StringType, LongType), + new StructType() + .add("f1", FloatType, nullable = true) + .add("f2", ArrayType(BooleanType, containsNull = true), nullable = true), + new UDT.MyDenseVectorUDT() + ).filter(supportsDataType) + + for (dataType <- supportedDataTypes) { + for (parquetDictionaryEncodingEnabled <- Seq(true, false)) { + test(s"test all data types - $dataType with parquet.enable.dictionary = " + + s"$parquetDictionaryEncodingEnabled") { + + val extraOptions = Map[String, String]( + "parquet.enable.dictionary" -> parquetDictionaryEncodingEnabled.toString + ) + + withTempPath { file => + val path = file.getCanonicalPath + + val dataGenerator = RandomDataGenerator.forType( + dataType = dataType, + nullable = true, + new Random(System.nanoTime()) + ).getOrElse { + fail(s"Failed to create data generator for schema $dataType") + } + + // Create a DF for the schema with random data. The index field is used to sort the + // DataFrame. This is a workaround for SPARK-10591. + val schema = new StructType() + .add("index", IntegerType, nullable = false) + .add("col", dataType, nullable = true) + val rdd = + spark.sparkContext.parallelize((1 to 10).map(i => Row(i, dataGenerator()))) + val df = spark.createDataFrame(rdd, schema).orderBy("index").coalesce(1) + + df.write + .mode("overwrite") + .format(dataSourceName) + .option("dataSchema", df.schema.json) + .options(extraOptions) + .save(path) + + val loadedDF = spark + .read + .format(dataSourceName) + .option("dataSchema", df.schema.json) + .schema(df.schema) + .options(extraOptions) + .load(path) + .orderBy("index") + + checkAnswer(loadedDF, df) + } + } + } + } + + test("save()/load() - non-partitioned table - Overwrite") { + withTempPath { file => + testDF.write.mode(SaveMode.Overwrite).format(dataSourceName).save(file.getCanonicalPath) + testDF.write.mode(SaveMode.Overwrite).format(dataSourceName).save(file.getCanonicalPath) + + checkAnswer( + spark.read.format(dataSourceName) + .option("path", file.getCanonicalPath) + .option("dataSchema", dataSchema.json) + .load(), + testDF.collect()) + } + } + + test("save()/load() - non-partitioned table - Append") { + withTempPath { file => + testDF.write.mode(SaveMode.Overwrite).format(dataSourceName).save(file.getCanonicalPath) + testDF.write.mode(SaveMode.Append).format(dataSourceName).save(file.getCanonicalPath) + + checkAnswer( + spark.read.format(dataSourceName) + .option("dataSchema", dataSchema.json) + .load(file.getCanonicalPath).orderBy("a"), + testDF.union(testDF).orderBy("a").collect()) + } + } + + test("save()/load() - non-partitioned table - ErrorIfExists") { + withTempDir { file => + intercept[AnalysisException] { + testDF.write.format(dataSourceName).mode(SaveMode.ErrorIfExists).save(file.getCanonicalPath) + } + } + } + + test("save()/load() - non-partitioned table - Ignore") { + withTempDir { file => + testDF.write.mode(SaveMode.Ignore).format(dataSourceName).save(file.getCanonicalPath) + + val path = new Path(file.getCanonicalPath) + val fs = path.getFileSystem(spark.sessionState.newHadoopConf()) + assert(fs.listStatus(path).isEmpty) + } + } + + test("save()/load() - partitioned table - simple queries") { + withTempPath { file => + partitionedTestDF.write + .format(dataSourceName) + .mode(SaveMode.ErrorIfExists) + .partitionBy("p1", "p2") + .save(file.getCanonicalPath) + + checkQueries( + spark.read.format(dataSourceName) + .option("dataSchema", dataSchema.json) + .load(file.getCanonicalPath)) + } + } + + test("save()/load() - partitioned table - Overwrite") { + withTempPath { file => + partitionedTestDF.write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .partitionBy("p1", "p2") + .save(file.getCanonicalPath) + + partitionedTestDF.write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .partitionBy("p1", "p2") + .save(file.getCanonicalPath) + + checkAnswer( + spark.read.format(dataSourceName) + .option("dataSchema", dataSchema.json) + .load(file.getCanonicalPath), + partitionedTestDF.collect()) + } + } + + test("save()/load() - partitioned table - Append") { + withTempPath { file => + partitionedTestDF.write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .partitionBy("p1", "p2") + .save(file.getCanonicalPath) + + partitionedTestDF.write + .format(dataSourceName) + .mode(SaveMode.Append) + .partitionBy("p1", "p2") + .save(file.getCanonicalPath) + + checkAnswer( + spark.read.format(dataSourceName) + .option("dataSchema", dataSchema.json) + .load(file.getCanonicalPath), + partitionedTestDF.union(partitionedTestDF).collect()) + } + } + + test("save()/load() - partitioned table - Append - new partition values") { + withTempPath { file => + partitionedTestDF1.write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .partitionBy("p1", "p2") + .save(file.getCanonicalPath) + + partitionedTestDF2.write + .format(dataSourceName) + .mode(SaveMode.Append) + .partitionBy("p1", "p2") + .save(file.getCanonicalPath) + + checkAnswer( + spark.read.format(dataSourceName) + .option("dataSchema", dataSchema.json) + .load(file.getCanonicalPath), + partitionedTestDF.collect()) + } + } + + test("save()/load() - partitioned table - ErrorIfExists") { + withTempDir { file => + intercept[AnalysisException] { + partitionedTestDF.write + .format(dataSourceName) + .mode(SaveMode.ErrorIfExists) + .partitionBy("p1", "p2") + .save(file.getCanonicalPath) + } + } + } + + test("save()/load() - partitioned table - Ignore") { + withTempDir { file => + partitionedTestDF.write + .format(dataSourceName).mode(SaveMode.Ignore).save(file.getCanonicalPath) + + val path = new Path(file.getCanonicalPath) + val fs = path.getFileSystem(SparkHadoopUtil.get.conf) + assert(fs.listStatus(path).isEmpty) + } + } + + test("saveAsTable()/load() - non-partitioned table - Overwrite") { + testDF.write.format(dataSourceName).mode(SaveMode.Overwrite) + .option("dataSchema", dataSchema.json) + .saveAsTable("t") + + withTable("t") { + checkAnswer(spark.table("t"), testDF.collect()) + } + } + + test("saveAsTable()/load() - non-partitioned table - Append") { + testDF.write.format(dataSourceName).mode(SaveMode.Overwrite).saveAsTable("t") + testDF.write.format(dataSourceName).mode(SaveMode.Append).saveAsTable("t") + + withTable("t") { + checkAnswer(spark.table("t"), testDF.union(testDF).orderBy("a").collect()) + } + } + + test("saveAsTable()/load() - non-partitioned table - ErrorIfExists") { + withTable("t") { + sql("CREATE TABLE t(i INT) USING parquet") + intercept[AnalysisException] { + testDF.write.format(dataSourceName).mode(SaveMode.ErrorIfExists).saveAsTable("t") + } + } + } + + test("saveAsTable()/load() - non-partitioned table - Ignore") { + withTable("t") { + sql("CREATE TABLE t(i INT) USING parquet") + testDF.write.format(dataSourceName).mode(SaveMode.Ignore).saveAsTable("t") + assert(spark.table("t").collect().isEmpty) + } + } + + test("saveAsTable()/load() - partitioned table - simple queries") { + partitionedTestDF.write.format(dataSourceName) + .mode(SaveMode.Overwrite) + .option("dataSchema", dataSchema.json) + .saveAsTable("t") + + withTable("t") { + checkQueries(spark.table("t")) + } + } + + test("saveAsTable()/load() - partitioned table - boolean type") { + spark.range(2) + .select('id, ('id % 2 === 0).as("b")) + .write.partitionBy("b").saveAsTable("t") + + withTable("t") { + checkAnswer( + spark.table("t").sort('id), + Row(0, true) :: Row(1, false) :: Nil + ) + } + } + + test("saveAsTable()/load() - partitioned table - Overwrite") { + partitionedTestDF.write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .option("dataSchema", dataSchema.json) + .partitionBy("p1", "p2") + .saveAsTable("t") + + partitionedTestDF.write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .option("dataSchema", dataSchema.json) + .partitionBy("p1", "p2") + .saveAsTable("t") + + withTable("t") { + checkAnswer(spark.table("t"), partitionedTestDF.collect()) + } + } + + test("saveAsTable()/load() - partitioned table - Append") { + partitionedTestDF.write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .option("dataSchema", dataSchema.json) + .partitionBy("p1", "p2") + .saveAsTable("t") + + partitionedTestDF.write + .format(dataSourceName) + .mode(SaveMode.Append) + .option("dataSchema", dataSchema.json) + .partitionBy("p1", "p2") + .saveAsTable("t") + + withTable("t") { + checkAnswer(spark.table("t"), partitionedTestDF.union(partitionedTestDF).collect()) + } + } + + test("saveAsTable()/load() - partitioned table - Append - new partition values") { + partitionedTestDF1.write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .option("dataSchema", dataSchema.json) + .partitionBy("p1", "p2") + .saveAsTable("t") + + partitionedTestDF2.write + .format(dataSourceName) + .mode(SaveMode.Append) + .option("dataSchema", dataSchema.json) + .partitionBy("p1", "p2") + .saveAsTable("t") + + withTable("t") { + checkAnswer(spark.table("t"), partitionedTestDF.collect()) + } + } + + test("saveAsTable()/load() - partitioned table - Append - mismatched partition columns") { + partitionedTestDF1.write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .option("dataSchema", dataSchema.json) + .partitionBy("p1", "p2") + .saveAsTable("t") + + // Using only a subset of all partition columns + intercept[AnalysisException] { + partitionedTestDF2.write + .format(dataSourceName) + .mode(SaveMode.Append) + .option("dataSchema", dataSchema.json) + .partitionBy("p1") + .saveAsTable("t") + } + } + + test("saveAsTable()/load() - partitioned table - ErrorIfExists") { + Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") + + withTempView("t") { + intercept[AnalysisException] { + partitionedTestDF.write + .format(dataSourceName) + .mode(SaveMode.ErrorIfExists) + .option("dataSchema", dataSchema.json) + .partitionBy("p1", "p2") + .saveAsTable("t") + } + } + } + + test("saveAsTable()/load() - partitioned table - Ignore") { + Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") + + withTempView("t") { + partitionedTestDF.write + .format(dataSourceName) + .mode(SaveMode.Ignore) + .option("dataSchema", dataSchema.json) + .partitionBy("p1", "p2") + .saveAsTable("t") + + assert(spark.table("t").collect().isEmpty) + } + } + + test("load() - with directory of unpartitioned data in nested subdirs") { + withTempPath { dir => + val subdir = new File(dir, "subdir") + + val dataInDir = Seq(1, 2, 3).toDF("value") + val dataInSubdir = Seq(4, 5, 6).toDF("value") + + /* + + Directory structure to be generated + + dir + | + |___ [ files of dataInDir ] + | + |___ subsubdir + | + |___ [ files of dataInSubdir ] + */ + + // Generated dataInSubdir, not data in dir + dataInSubdir.write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .save(subdir.getCanonicalPath) + + // Inferring schema should throw error as it should not find any file to infer + val e = intercept[Exception] { + spark.read.format(dataSourceName).load(dir.getCanonicalPath) + } + + e match { + case _: AnalysisException => + assert(e.getMessage.contains("infer")) + + case _: java.util.NoSuchElementException if e.getMessage.contains("dataSchema") => + // Ignore error, the source format requires schema to be provided by user + // This is needed for SimpleTextHadoopFsRelationSuite as SimpleTextSource needs schema + + case _ => + fail("Unexpected error trying to infer schema from empty dir", e) + } + + /** Test whether data is read with the given path matches the expected answer */ + def testWithPath(path: File, expectedAnswer: Seq[Row]): Unit = { + val df = spark.read + .format(dataSourceName) + .schema(dataInDir.schema) // avoid schema inference for any format + .load(path.getCanonicalPath) + checkAnswer(df, expectedAnswer) + } + + // Verify that reading by path 'dir/' gives empty results as there are no files in 'file' + // and it should not pick up files in 'dir/subdir' + require(subdir.exists) + require(subdir.listFiles().exists(!_.isDirectory)) + testWithPath(dir, Seq.empty) + + // Verify that if there is data in dir, then reading by path 'dir/' reads only dataInDir + dataInDir.write + .format(dataSourceName) + .mode(SaveMode.Append) // append to prevent subdir from being deleted + .save(dir.getCanonicalPath) + require(dir.listFiles().exists(!_.isDirectory)) + require(subdir.exists()) + require(subdir.listFiles().exists(!_.isDirectory)) + testWithPath(dir, dataInDir.collect()) + } + } + + test("Hadoop style globbing - unpartitioned data") { + withTempPath { file => + + val dir = file.getCanonicalPath + val subdir = new File(dir, "subdir") + val subsubdir = new File(subdir, "subsubdir") + val anotherSubsubdir = + new File(new File(dir, "another-subdir"), "another-subsubdir") + + val dataInSubdir = Seq(1, 2, 3).toDF("value") + val dataInSubsubdir = Seq(4, 5, 6).toDF("value") + val dataInAnotherSubsubdir = Seq(7, 8, 9).toDF("value") + + dataInSubdir.write + .format (dataSourceName) + .mode (SaveMode.Overwrite) + .save (subdir.getCanonicalPath) + + dataInSubsubdir.write + .format (dataSourceName) + .mode (SaveMode.Overwrite) + .save (subsubdir.getCanonicalPath) + + dataInAnotherSubsubdir.write + .format (dataSourceName) + .mode (SaveMode.Overwrite) + .save (anotherSubsubdir.getCanonicalPath) + + require(subdir.exists) + require(subdir.listFiles().exists(!_.isDirectory)) + require(subsubdir.exists) + require(subsubdir.listFiles().exists(!_.isDirectory)) + require(anotherSubsubdir.exists) + require(anotherSubsubdir.listFiles().exists(!_.isDirectory)) + + /* + Directory structure generated + + dir + | + |___ subdir + | | + | |___ [ files of dataInSubdir ] + | | + | |___ subsubdir + | | + | |___ [ files of dataInSubsubdir ] + | + | + |___ anotherSubdir + | + |___ anotherSubsubdir + | + |___ [ files of dataInAnotherSubsubdir ] + */ + + val schema = dataInSubdir.schema + + /** Check whether data is read with the given path matches the expected answer */ + def check(path: String, expectedDf: DataFrame): Unit = { + val df = spark.read + .format(dataSourceName) + .schema(schema) // avoid schema inference for any format, expected to be same format + .load(path) + checkAnswer(df, expectedDf) + } + + check(s"$dir/*/", dataInSubdir) + check(s"$dir/sub*/*", dataInSubdir.union(dataInSubsubdir)) + check(s"$dir/another*/*", dataInAnotherSubsubdir) + check(s"$dir/*/another*", dataInAnotherSubsubdir) + check(s"$dir/*/*", dataInSubdir.union(dataInSubsubdir).union(dataInAnotherSubsubdir)) + } + } + + test("Hadoop style globbing - partitioned data with schema inference") { + + // Tests the following on partition data + // - partitions are not discovered with globbing and without base path set. + // - partitions are discovered with globbing and base path set, though more detailed + // tests for this is in ParquetPartitionDiscoverySuite + + withTempPath { path => + val dir = path.getCanonicalPath + partitionedTestDF.write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .partitionBy("p1", "p2") + .save(dir) + + def check( + path: String, + expectedResult: Either[DataFrame, String], + basePath: Option[String] = None + ): Unit = { + try { + val reader = spark.read + basePath.foreach(reader.option("basePath", _)) + val testDf = reader + .format(dataSourceName) + .load(path) + assert(expectedResult.isLeft, s"Error was expected with $path but result found") + checkAnswer(testDf, expectedResult.left.get) + } catch { + case e: java.util.NoSuchElementException if e.getMessage.contains("dataSchema") => + // Ignore error, the source format requires schema to be provided by user + // This is needed for SimpleTextHadoopFsRelationSuite as SimpleTextSource needs schema + + case e: Throwable => + assert(expectedResult.isRight, s"Was not expecting error with $path: " + e) + assert( + e.getMessage.contains(expectedResult.right.get), + s"Did not find expected error message wiht $path") + } + } + + object Error { + def apply(msg: String): Either[DataFrame, String] = Right(msg) + } + + object Result { + def apply(df: DataFrame): Either[DataFrame, String] = Left(df) + } + + // ---- Without base path set ---- + // Should find all the data with partitioning columns + check(s"$dir", Result(partitionedTestDF)) + + // Should fail as globbing finds dirs without files, only subdirs in them. + check(s"$dir/*/", Error("please set \"basePath\"")) + check(s"$dir/p1=*/", Error("please set \"basePath\"")) + + // Should not find partition columns as the globs resolve to p2 dirs + // with files in them + check(s"$dir/*/*", Result(partitionedTestDF.drop("p1", "p2"))) + check(s"$dir/p1=*/p2=foo", Result(partitionedTestDF.filter("p2 = 'foo'").drop("p1", "p2"))) + check(s"$dir/p1=1/p2=???", Result(partitionedTestDF.filter("p1 = 1").drop("p1", "p2"))) + + // Should find all data without the partitioning columns as the globs resolve to the files + check(s"$dir/*/*/*", Result(partitionedTestDF.drop("p1", "p2"))) + + // ---- With base path set ---- + val resultDf = partitionedTestDF.select("a", "b", "p1", "p2") + check(path = s"$dir/*", Result(resultDf), basePath = Some(dir)) + check(path = s"$dir/*/*", Result(resultDf), basePath = Some(dir)) + check(path = s"$dir/*/*/*", Result(resultDf), basePath = Some(dir)) + } + } + + test("SPARK-9735 Partition column type casting") { + withTempPath { file => + val df = (for { + i <- 1 to 3 + p2 <- Seq("foo", "bar") + } yield (i, s"val_$i", 1.0d, p2, 123, 123.123f)).toDF("a", "b", "p1", "p2", "p3", "f") + + val input = df.select( + 'a, + 'b, + 'p1.cast(StringType).as('ps1), + 'p2, + 'p3.cast(FloatType).as('pf1), + 'f) + + withTempView("t") { + input + .write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .partitionBy("ps1", "p2", "pf1", "f") + .saveAsTable("t") + + input + .write + .format(dataSourceName) + .mode(SaveMode.Append) + .partitionBy("ps1", "p2", "pf1", "f") + .saveAsTable("t") + + val realData = input.collect() + + checkAnswer(spark.table("t"), realData ++ realData) + } + } + } + + test("SPARK-7616: adjust column name order accordingly when saving partitioned table") { + val df = (1 to 3).map(i => (i, s"val_$i", i * 2)).toDF("a", "b", "c") + + df.write + .format(dataSourceName) + .mode(SaveMode.Overwrite) + .partitionBy("c", "a") + .saveAsTable("t") + + withTable("t") { + checkAnswer(spark.table("t").select('b, 'c, 'a), df.select('b, 'c, 'a).collect()) + } + } + + // NOTE: This test suite is not super deterministic. On nodes with only relatively few cores + // (4 or even 1), it's hard to reproduce the data loss issue. But on nodes with for example 8 or + // more cores, the issue can be reproduced steadily. Fortunately our Jenkins builder meets this + // requirement. We probably want to move this test case to spark-integration-tests or spark-perf + // later. + test("SPARK-8406: Avoids name collision while writing files") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark + .range(10000) + .repartition(250) + .write + .mode(SaveMode.Overwrite) + .format(dataSourceName) + .save(path) + + assertResult(10000) { + spark + .read + .format(dataSourceName) + .option("dataSchema", StructType(StructField("id", LongType) :: Nil).json) + .load(path) + .count() + } + } + } + + test("SPARK-8887: Explicitly define which data types can be used as dynamic partition columns") { + val df = Seq( + (1, "v1", Array(1, 2, 3), Map("k1" -> "v1"), Tuple2(1, "4")), + (2, "v2", Array(4, 5, 6), Map("k2" -> "v2"), Tuple2(2, "5")), + (3, "v3", Array(7, 8, 9), Map("k3" -> "v3"), Tuple2(3, "6"))).toDF("a", "b", "c", "d", "e") + withTempDir { file => + intercept[AnalysisException] { + df.write.format(dataSourceName).partitionBy("c", "d", "e").save(file.getCanonicalPath) + } + } + intercept[AnalysisException] { + df.write.format(dataSourceName).partitionBy("c", "d", "e").saveAsTable("t") + } + } + + test("Locality support for FileScanRDD") { + val options = Map[String, String]( + "fs.file.impl" -> classOf[LocalityTestFileSystem].getName, + "fs.file.impl.disable.cache" -> "true" + ) + withTempPath { dir => + val path = dir.toURI.toString + val df1 = spark.range(4) + df1.coalesce(1).write.mode("overwrite").options(options).format(dataSourceName).save(path) + df1.coalesce(1).write.mode("append").options(options).format(dataSourceName).save(path) + + def checkLocality(): Unit = { + val df2 = spark.read + .format(dataSourceName) + .option("dataSchema", df1.schema.json) + .options(options) + .load(path) + + val Some(fileScanRDD) = df2.queryExecution.executedPlan.collectFirst { + case scan: DataSourceScanExec if scan.inputRDDs().head.isInstanceOf[FileScanRDD] => + scan.inputRDDs().head.asInstanceOf[FileScanRDD] + } + + val partitions = fileScanRDD.partitions + val preferredLocations = partitions.flatMap(fileScanRDD.preferredLocations) + + assert(preferredLocations.distinct.length == 2) + } + + checkLocality() + + withSQLConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD.key -> "0") { + checkLocality() + } + } + } + + test("SPARK-16975: Partitioned table with the column having '_' should be read correctly") { + withTempDir { dir => + val childDir = new File(dir, dataSourceName).getCanonicalPath + val dataDf = spark.range(10).toDF() + val df = dataDf.withColumn("_col", $"id") + df.write.format(dataSourceName).partitionBy("_col").save(childDir) + val reader = spark.read.format(dataSourceName) + + // This is needed for SimpleTextHadoopFsRelationSuite as SimpleTextSource needs schema. + if (dataSourceName == classOf[SimpleTextSource].getCanonicalName) { + reader.option("dataSchema", dataDf.schema.json) + } + val readBack = reader.load(childDir) + checkAnswer(df, readBack) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala index ef37787137d0..49be30435ad2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala @@ -21,8 +21,8 @@ import java.math.BigDecimal import org.apache.hadoop.fs.Path -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.catalog.CatalogUtils import org.apache.spark.sql.types._ class JsonHadoopFsRelationSuite extends HadoopFsRelationTest { @@ -38,12 +38,9 @@ class JsonHadoopFsRelationSuite extends HadoopFsRelationTest { test("save()/load() - partitioned table - simple queries - partition columns in data") { withTempDir { file => - val basePath = new Path(file.getCanonicalPath) - val fs = basePath.getFileSystem(SparkHadoopUtil.get.conf) - val qualifiedBasePath = fs.makeQualified(basePath) - for (p1 <- 1 to 2; p2 <- Seq("foo", "bar")) { - val partitionDir = new Path(qualifiedBasePath, s"p1=$p1/p2=$p2") + val partitionDir = new Path( + CatalogUtils.URIToString(makeQualifiedPath(file.getCanonicalPath)), s"p1=$p1/p2=$p2") sparkContext .parallelize(for (i <- 1 to 3) yield s"""{"a":$i,"b":"val_$i"}""") .saveAsTextFile(partitionDir.toString) @@ -53,7 +50,7 @@ class JsonHadoopFsRelationSuite extends HadoopFsRelationTest { StructType(dataSchema.fields :+ StructField("p1", IntegerType, nullable = true)) checkQueries( - hiveContext.read.format(dataSourceName) + spark.read.format(dataSourceName) .option("dataSchema", dataSchemaWithPartition.json) .load(file.getCanonicalPath)) } @@ -71,14 +68,14 @@ class JsonHadoopFsRelationSuite extends HadoopFsRelationTest { val data = Row(Seq(1L, 2L, 3L), Map("m1" -> Row(4L))) :: Row(Seq(5L, 6L, 7L), Map("m2" -> Row(10L))) :: Nil - val df = hiveContext.createDataFrame(sparkContext.parallelize(data), schema) + val df = spark.createDataFrame(sparkContext.parallelize(data), schema) // Write the data out. df.write.format(dataSourceName).save(file.getCanonicalPath) // Read it back and check the result. checkAnswer( - hiveContext.read.format(dataSourceName).schema(schema).load(file.getCanonicalPath), + spark.read.format(dataSourceName).schema(schema).load(file.getCanonicalPath), df ) } @@ -96,14 +93,14 @@ class JsonHadoopFsRelationSuite extends HadoopFsRelationTest { Row(new BigDecimal("10.02")) :: Row(new BigDecimal("20000.99")) :: Row(new BigDecimal("10000")) :: Nil - val df = hiveContext.createDataFrame(sparkContext.parallelize(data), schema) + val df = spark.createDataFrame(sparkContext.parallelize(data), schema) // Write the data out. df.write.format(dataSourceName).save(file.getCanonicalPath) // Read it back and check the result. checkAnswer( - hiveContext.read.format(dataSourceName).schema(schema).load(file.getCanonicalPath), + spark.read.format(dataSourceName).schema(schema).load(file.getCanonicalPath), df ) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala index e2d754e80640..dce5bb7ddba6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala @@ -21,9 +21,12 @@ import java.io.File import com.google.common.io.Files import org.apache.hadoop.fs.Path +import org.apache.parquet.hadoop.ParquetOutputFormat -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.sql.{execution, AnalysisException, SaveMode} +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.catalog.CatalogUtils +import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -41,12 +44,9 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { test("save()/load() - partitioned table - simple queries - partition columns in data") { withTempDir { file => - val basePath = new Path(file.getCanonicalPath) - val fs = basePath.getFileSystem(SparkHadoopUtil.get.conf) - val qualifiedBasePath = fs.makeQualified(basePath) - for (p1 <- 1 to 2; p2 <- Seq("foo", "bar")) { - val partitionDir = new Path(qualifiedBasePath, s"p1=$p1/p2=$p2") + val partitionDir = new Path( + CatalogUtils.URIToString(makeQualifiedPath(file.getCanonicalPath)), s"p1=$p1/p2=$p2") sparkContext .parallelize(for (i <- 1 to 3) yield (i, s"val_$i", p1)) .toDF("a", "b", "p1") @@ -57,7 +57,7 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { StructType(dataSchema.fields :+ StructField("p1", IntegerType, nullable = true)) checkQueries( - hiveContext.read.format(dataSourceName) + spark.read.format(dataSourceName) .option("dataSchema", dataSchemaWithPartition.json) .load(file.getCanonicalPath)) } @@ -75,7 +75,7 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { .format("parquet") .save(s"${dir.getCanonicalPath}/_temporary") - checkAnswer(hiveContext.read.format("parquet").load(dir.getCanonicalPath), df.collect()) + checkAnswer(spark.read.format("parquet").load(dir.getCanonicalPath), df.collect()) } } @@ -103,7 +103,7 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { // This shouldn't throw anything. df.write.format("parquet").mode(SaveMode.Overwrite).save(path) - checkAnswer(hiveContext.read.format("parquet").load(path), df) + checkAnswer(spark.read.format("parquet").load(path), df) } } @@ -113,7 +113,7 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { // Parquet doesn't allow field names with spaces. Here we are intentionally making an // exception thrown from the `ParquetRelation2.prepareForWriteJob()` method to trigger // the bug. Please refer to spark-8079 for more details. - hiveContext.range(1, 10) + spark.range(1, 10) .withColumnRenamed("id", "a b") .write .format("parquet") @@ -123,36 +123,113 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { } test("SPARK-8604: Parquet data source should write summary file while doing appending") { + withSQLConf( + ParquetOutputFormat.ENABLE_JOB_SUMMARY -> "true", + SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> + classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName) { + withTempPath { dir => + val path = dir.getCanonicalPath + val df = spark.range(0, 5).toDF() + df.write.mode(SaveMode.Overwrite).parquet(path) + + val summaryPath = new Path(path, "_metadata") + val commonSummaryPath = new Path(path, "_common_metadata") + + val fs = summaryPath.getFileSystem(spark.sessionState.newHadoopConf()) + fs.delete(summaryPath, true) + fs.delete(commonSummaryPath, true) + + df.write.mode(SaveMode.Append).parquet(path) + checkAnswer(spark.read.parquet(path), df.union(df)) + + assert(fs.exists(summaryPath)) + assert(fs.exists(commonSummaryPath)) + } + } + } + + test("SPARK-10334 Projections and filters should be kept in physical plan") { withTempPath { dir => val path = dir.getCanonicalPath - val df = sqlContext.range(0, 5) - df.write.mode(SaveMode.Overwrite).parquet(path) - - val summaryPath = new Path(path, "_metadata") - val commonSummaryPath = new Path(path, "_common_metadata") - val fs = summaryPath.getFileSystem(hadoopConfiguration) - fs.delete(summaryPath, true) - fs.delete(commonSummaryPath, true) + spark.range(2).select('id as 'a, 'id as 'b).write.partitionBy("b").parquet(path) + val df = spark.read.parquet(path).filter('a === 0).select('b) + val physicalPlan = df.queryExecution.sparkPlan - df.write.mode(SaveMode.Append).parquet(path) - checkAnswer(sqlContext.read.parquet(path), df.unionAll(df)) + assert(physicalPlan.collect { case p: execution.ProjectExec => p }.length === 1) + assert(physicalPlan.collect { case p: execution.FilterExec => p }.length === 1) + } + } - assert(fs.exists(summaryPath)) - assert(fs.exists(commonSummaryPath)) + test("SPARK-11500: Not deterministic order of columns when using merging schemas.") { + import testImplicits._ + withSQLConf(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true") { + withTempPath { dir => + val pathOne = s"${dir.getCanonicalPath}/part=1" + Seq(1, 1).zipWithIndex.toDF("a", "b").write.parquet(pathOne) + val pathTwo = s"${dir.getCanonicalPath}/part=2" + Seq(1, 1).zipWithIndex.toDF("c", "b").write.parquet(pathTwo) + val pathThree = s"${dir.getCanonicalPath}/part=3" + Seq(1, 1).zipWithIndex.toDF("d", "b").write.parquet(pathThree) + + // The schema consists of the leading columns of the first part-file + // in the lexicographic order. + assert(spark.read.parquet(dir.getCanonicalPath).schema.map(_.name) + === Seq("a", "b", "c", "d", "part")) + } } } - test("SPARK-10334 Projections and filters should be kept in physical plan") { - withTempPath { dir => - val path = dir.getCanonicalPath + test(s"SPARK-13537: Fix readBytes in VectorizedPlainValuesReader") { + withTempPath { file => + val path = file.getCanonicalPath + + val schema = new StructType() + .add("index", IntegerType, nullable = false) + .add("col", ByteType, nullable = true) + + val data = Seq(Row(1, -33.toByte), Row(2, 0.toByte), Row(3, -55.toByte), Row(4, 56.toByte), + Row(5, 127.toByte), Row(6, -44.toByte), Row(7, 23.toByte), Row(8, -95.toByte), + Row(9, 127.toByte), Row(10, 13.toByte)) - sqlContext.range(2).select('id as 'a, 'id as 'b).write.partitionBy("b").parquet(path) - val df = sqlContext.read.parquet(path).filter('a === 0).select('b) - val physicalPlan = df.queryExecution.executedPlan + val rdd = spark.sparkContext.parallelize(data) + val df = spark.createDataFrame(rdd, schema).orderBy("index").coalesce(1) - assert(physicalPlan.collect { case p: execution.Project => p }.length === 1) - assert(physicalPlan.collect { case p: execution.Filter => p }.length === 1) + df.write + .mode("overwrite") + .format(dataSourceName) + .option("dataSchema", df.schema.json) + .save(path) + + val loadedDF = spark + .read + .format(dataSourceName) + .option("dataSchema", df.schema.json) + .schema(df.schema) + .load(path) + .orderBy("index") + + checkAnswer(loadedDF, df) + } + } + + test("SPARK-13543: Support for specifying compression codec for Parquet via option()") { + withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> "UNCOMPRESSED") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/table1" + val df = (1 to 5).map(i => (i, (i % 2).toString)).toDF("a", "b") + df.write + .option("compression", "GzIP") + .parquet(path) + + val compressedFiles = new File(path).listFiles() + assert(compressedFiles.exists(_.getName.endsWith(".gz.parquet"))) + + val copyDf = spark + .read + .parquet(path) + checkAnswer(df, copyDf) + } } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala index d945408341fc..2ec593b95c9b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala @@ -18,16 +18,12 @@ package org.apache.spark.sql.sources import org.apache.hadoop.fs.Path -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.CatalystTypeConverters -import org.apache.spark.sql.execution.PhysicalRDD -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types._ -class SimpleTextHadoopFsRelationSuite extends HadoopFsRelationTest { - import testImplicits._ +import org.apache.spark.sql.catalyst.catalog.CatalogUtils +import org.apache.spark.sql.catalyst.expressions.PredicateHelper +import org.apache.spark.sql.types._ +class SimpleTextHadoopFsRelationSuite extends HadoopFsRelationTest with PredicateHelper { override val dataSourceName: String = classOf[SimpleTextSource].getCanonicalName // We have a very limited number of supported types at here since it is just for a @@ -49,12 +45,9 @@ class SimpleTextHadoopFsRelationSuite extends HadoopFsRelationTest { test("save()/load() - partitioned table - simple queries - partition columns in data") { withTempDir { file => - val basePath = new Path(file.getCanonicalPath) - val fs = basePath.getFileSystem(SparkHadoopUtil.get.conf) - val qualifiedBasePath = fs.makeQualified(basePath) - for (p1 <- 1 to 2; p2 <- Seq("foo", "bar")) { - val partitionDir = new Path(qualifiedBasePath, s"p1=$p1/p2=$p2") + val partitionDir = new Path( + CatalogUtils.URIToString(makeQualifiedPath(file.getCanonicalPath)), s"p1=$p1/p2=$p2") sparkContext .parallelize(for (i <- 1 to 3) yield s"$i,val_$i,$p1") .saveAsTextFile(partitionDir.toString) @@ -64,49 +57,32 @@ class SimpleTextHadoopFsRelationSuite extends HadoopFsRelationTest { StructType(dataSchema.fields :+ StructField("p1", IntegerType, nullable = true)) checkQueries( - hiveContext.read.format(dataSourceName) + spark.read.format(dataSourceName) .option("dataSchema", dataSchemaWithPartition.json) .load(file.getCanonicalPath)) } } - private val writer = testDF.write.option("dataSchema", dataSchema.json).format(dataSourceName) - private val reader = sqlContext.read.option("dataSchema", dataSchema.json).format(dataSourceName) - - test("unhandledFilters") { - withTempPath { dir => - - val path = dir.getCanonicalPath - writer.save(s"$path/p=0") - writer.save(s"$path/p=1") - - val isOdd = udf((_: Int) % 2 == 1) - val df = reader.load(path) - .filter( - // This filter is inconvertible - isOdd('a) && - // This filter is convertible but unhandled - 'a > 1 && - // This filter is convertible and handled - 'b > "val_1" && - // This filter references a partiiton column, won't be pushed down - 'p === 1 - ).select('a, 'p) - val rawScan = df.queryExecution.executedPlan collect { - case p: PhysicalRDD => p - } match { - case Seq(p) => p - } - - val outputSchema = new StructType().add("a", IntegerType).add("p", IntegerType) - - assertResult(Set((2, 1), (3, 1))) { - rawScan.execute().collect() - .map { CatalystTypeConverters.convertToScala(_, outputSchema) } - .map { case Row(a, p) => (a, p) }.toSet - } - - checkAnswer(df, Row(3, 1)) + test("test hadoop conf option propagation") { + withTempPath { file => + // Test write side + val df = spark.range(10).selectExpr("cast(id as string)") + df.write + .option("some-random-write-option", "hahah-WRITE") + .option("some-null-value-option", null) // test null robustness + .option("dataSchema", df.schema.json) + .format(dataSourceName).save(file.getAbsolutePath) + assert(SimpleTextRelation.lastHadoopConf.get.get("some-random-write-option") == "hahah-WRITE") + + // Test read side + val df1 = spark.read + .option("some-random-read-option", "hahah-READ") + .option("some-null-value-option", null) // test null robustness + .option("dataSchema", df.schema.json) + .format(dataSourceName) + .load(file.getAbsolutePath) + df1.count() + assert(SimpleTextRelation.lastHadoopConf.get.get("some-random-read-option") == "hahah-READ") } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index da09e1b00ae4..60a4638f610b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -17,125 +17,68 @@ package org.apache.spark.sql.sources -import java.text.NumberFormat - -import com.google.common.base.Objects +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.hadoop.io.{NullWritable, Text} -import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, TextOutputFormat} -import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} - -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, expressions} -import org.apache.spark.sql.types.{DataType, StructType} -import org.apache.spark.sql.{Row, SQLContext, sources} - -/** - * A simple example [[HadoopFsRelationProvider]]. - */ -class SimpleTextSource extends HadoopFsRelationProvider { - override def createRelation( - sqlContext: SQLContext, - paths: Array[String], - schema: Option[StructType], - partitionColumns: Option[StructType], - parameters: Map[String, String]): HadoopFsRelation = { - new SimpleTextRelation(paths, schema, partitionColumns, parameters)(sqlContext) - } -} - -class AppendingTextOutputFormat(outputFile: Path) extends TextOutputFormat[NullWritable, Text] { - val numberFormat = NumberFormat.getInstance() - - numberFormat.setMinimumIntegerDigits(5) - numberFormat.setGroupingUsed(false) - - override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val configuration = SparkHadoopUtil.get.getConfigurationFromJobContext(context) - val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") - val taskAttemptId = SparkHadoopUtil.get.getTaskAttemptIDFromTaskAttemptContext(context) - val split = taskAttemptId.getTaskID.getId - val name = FileOutputFormat.getOutputName(context) - new Path(outputFile, s"$name-${numberFormat.format(split)}-$uniqueWriteJobId") - } -} +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} -class SimpleTextOutputWriter(path: String, context: TaskAttemptContext) extends OutputWriter { - private val recordWriter: RecordWriter[NullWritable, Text] = - new AppendingTextOutputFormat(new Path(path)).getRecordWriter(context) +import org.apache.spark.sql.{sources, SparkSession} +import org.apache.spark.sql.catalyst.{expressions, InternalRow} +import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, GenericInternalRow, InterpretedPredicate, InterpretedProjection, JoinedRow, Literal} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.util.SerializableConfiguration - override def write(row: Row): Unit = { - val serialized = row.toSeq.map { v => - if (v == null) "" else v.toString - }.mkString(",") - recordWriter.write(null, new Text(serialized)) - } +class SimpleTextSource extends TextBasedFileFormat with DataSourceRegister { + override def shortName(): String = "test" - override def close(): Unit = { - recordWriter.close(context) + override def inferSchema( + sparkSession: SparkSession, + options: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = { + Some(DataType.fromJson(options("dataSchema")).asInstanceOf[StructType]) } -} -/** - * A simple example [[HadoopFsRelation]], used for testing purposes. Data are stored as comma - * separated string lines. When scanning data, schema must be explicitly provided via data source - * option `"dataSchema"`. - */ -class SimpleTextRelation( - override val paths: Array[String], - val maybeDataSchema: Option[StructType], - override val userDefinedPartitionColumns: Option[StructType], - parameters: Map[String, String])( - @transient val sqlContext: SQLContext) - extends HadoopFsRelation { - - import sqlContext.sparkContext - - override val dataSchema: StructType = - maybeDataSchema.getOrElse(DataType.fromJson(parameters("dataSchema")).asInstanceOf[StructType]) - - override def equals(other: Any): Boolean = other match { - case that: SimpleTextRelation => - this.paths.sameElements(that.paths) && - this.maybeDataSchema == that.maybeDataSchema && - this.dataSchema == that.dataSchema && - this.partitionColumns == that.partitionColumns - - case _ => false - } - - override def hashCode(): Int = - Objects.hashCode(paths, maybeDataSchema, dataSchema, partitionColumns) - - override def buildScan(inputStatuses: Array[FileStatus]): RDD[Row] = { - val fields = dataSchema.map(_.dataType) + override def prepareWrite( + sparkSession: SparkSession, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + SimpleTextRelation.lastHadoopConf = Option(job.getConfiguration) + new OutputWriterFactory { + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new SimpleTextOutputWriter(path, dataSchema, context) + } - sparkContext.textFile(inputStatuses.map(_.getPath).mkString(",")).map { record => - Row(record.split(",", -1).zip(fields).map { case (v, dataType) => - val value = if (v == "") null else v - // `Cast`ed values are always of Catalyst types (i.e. UTF8String instead of String, etc.) - val catalystValue = Cast(Literal(value), dataType).eval() - // Here we're converting Catalyst values to Scala values to test `needsConversion` - CatalystTypeConverters.convertToScala(catalystValue, dataType) - }: _*) + override def getFileExtension(context: TaskAttemptContext): String = "" } } - override def buildScan( - requiredColumns: Array[String], - filters: Array[Filter], - inputFiles: Array[FileStatus]): RDD[Row] = { + override def buildReader( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { + SimpleTextRelation.lastHadoopConf = Option(hadoopConf) + SimpleTextRelation.requiredColumns = requiredSchema.fieldNames + SimpleTextRelation.pushedFilters = filters.toSet + + val fieldTypes = dataSchema.map(_.dataType) + val inputAttributes = dataSchema.toAttributes + val outputAttributes = requiredSchema.flatMap { field => + inputAttributes.find(_.name == field.name) + } - val fields = this.dataSchema.map(_.dataType) - val inputAttributes = this.dataSchema.toAttributes - val outputAttributes = requiredColumns.flatMap(name => inputAttributes.find(_.name == name)) - val dataSchema = this.dataSchema + val broadcastedHadoopConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) - val inputPaths = inputFiles.map(_.getPath).mkString(",") - sparkContext.textFile(inputPaths).mapPartitions { iterator => - // Constructs a filter predicate to simulate filter push-down + (file: PartitionedFile) => { val predicate = { val filterCondition: Expression = filters.collect { // According to `unhandledFilters`, `SimpleTextRelation` only handles `GreaterThan` filter @@ -149,81 +92,66 @@ class SimpleTextRelation( } // Uses a simple projection to simulate column pruning - val projection = new InterpretedMutableProjection(outputAttributes, inputAttributes) - val toScala = { - val requiredSchema = StructType.fromAttributes(outputAttributes) - CatalystTypeConverters.createToScalaConverter(requiredSchema) - } - - iterator.map { record => - new GenericInternalRow(record.split(",", -1).zip(fields).map { - case (v, dataType) => - val value = if (v == "") null else v - // `Cast`ed values are always of internal types (e.g. UTF8String instead of String) - Cast(Literal(value), dataType).eval() - }) - }.filter { row => - predicate(row) - }.map { row => - toScala(projection(row)).asInstanceOf[Row] + val projection = new InterpretedProjection(outputAttributes, inputAttributes) + + val unsafeRowIterator = + new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value).map { line => + val record = line.toString + new GenericInternalRow(record.split(",", -1).zip(fieldTypes).map { + case (v, dataType) => + val value = if (v == "") null else v + // `Cast`ed values are always of internal types (e.g. UTF8String instead of String) + Cast(Literal(value), dataType).eval() + }) + }.filter(predicate.eval).map(projection) + + // Appends partition values + val fullOutput = requiredSchema.toAttributes ++ partitionSchema.toAttributes + val joinedRow = new JoinedRow() + val appendPartitionColumns = GenerateUnsafeProjection.generate(fullOutput, fullOutput) + + unsafeRowIterator.map { dataRow => + appendPartitionColumns(joinedRow(dataRow, file.partitionValues)) } } } +} - override def prepareJobForWrite(job: Job): OutputWriterFactory = new OutputWriterFactory { - job.setOutputFormatClass(classOf[TextOutputFormat[_, _]]) +class SimpleTextOutputWriter(path: String, dataSchema: StructType, context: TaskAttemptContext) + extends OutputWriter { - override def newInstance( - path: String, - dataSchema: StructType, - context: TaskAttemptContext): OutputWriter = { - new SimpleTextOutputWriter(path, context) - } - } + private val writer = CodecStreams.createOutputStreamWriter(context, new Path(path)) - // `SimpleTextRelation` only handles `GreaterThan` filter. This is used to test filter push-down - // and `BaseRelation.unhandledFilters()`. - override def unhandledFilters(filters: Array[Filter]): Array[Filter] = { - filters.filter { - case _: GreaterThan => false - case _ => true - } + override def write(row: InternalRow): Unit = { + val serialized = row.toSeq(dataSchema).map { v => + if (v == null) "" else v.toString + }.mkString(",") + + writer.write(serialized) + writer.write('\n') } -} -/** - * A simple example [[HadoopFsRelationProvider]]. - */ -class CommitFailureTestSource extends HadoopFsRelationProvider { - override def createRelation( - sqlContext: SQLContext, - paths: Array[String], - schema: Option[StructType], - partitionColumns: Option[StructType], - parameters: Map[String, String]): HadoopFsRelation = { - new CommitFailureTestRelation(paths, schema, partitionColumns, parameters)(sqlContext) + override def close(): Unit = { + writer.close() } } -class CommitFailureTestRelation( - override val paths: Array[String], - maybeDataSchema: Option[StructType], - override val userDefinedPartitionColumns: Option[StructType], - parameters: Map[String, String])( - @transient sqlContext: SQLContext) - extends SimpleTextRelation( - paths, maybeDataSchema, userDefinedPartitionColumns, parameters)(sqlContext) { - override def prepareJobForWrite(job: Job): OutputWriterFactory = new OutputWriterFactory { - override def newInstance( - path: String, - dataSchema: StructType, - context: TaskAttemptContext): OutputWriter = { - new SimpleTextOutputWriter(path, context) { - override def close(): Unit = { - super.close() - sys.error("Intentional task commitment failure for testing purpose.") - } - } - } - } +object SimpleTextRelation { + // Used to test column pruning + var requiredColumns: Seq[String] = Nil + + // Used to test filter push-down + var pushedFilters: Set[Filter] = Set.empty + + // Used to test failed committer + var failCommitter = false + + // Used to test failed writer + var failWriter = false + + // Used to test failure callback + var callbackCalled = false + + // Used by the test case to check the value propagated in the hadoop confs. + var lastHadoopConf: Option[Configuration] = None } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala deleted file mode 100644 index 100b97137cff..000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala +++ /dev/null @@ -1,745 +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.sources - -import scala.collection.JavaConverters._ - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} -import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter -import org.apache.parquet.hadoop.ParquetOutputCommitter - -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.sql._ -import org.apache.spark.sql.execution.ConvertToUnsafe -import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.sql.types._ - - -abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with TestHiveSingleton { - import sqlContext.implicits._ - - val dataSourceName: String - - protected def supportsDataType(dataType: DataType): Boolean = true - - val dataSchema = - StructType( - Seq( - StructField("a", IntegerType, nullable = false), - StructField("b", StringType, nullable = false))) - - lazy val testDF = (1 to 3).map(i => (i, s"val_$i")).toDF("a", "b") - - lazy val partitionedTestDF1 = (for { - i <- 1 to 3 - p2 <- Seq("foo", "bar") - } yield (i, s"val_$i", 1, p2)).toDF("a", "b", "p1", "p2") - - lazy val partitionedTestDF2 = (for { - i <- 1 to 3 - p2 <- Seq("foo", "bar") - } yield (i, s"val_$i", 2, p2)).toDF("a", "b", "p1", "p2") - - lazy val partitionedTestDF = partitionedTestDF1.unionAll(partitionedTestDF2) - - def checkQueries(df: DataFrame): Unit = { - // Selects everything - checkAnswer( - df, - for (i <- 1 to 3; p1 <- 1 to 2; p2 <- Seq("foo", "bar")) yield Row(i, s"val_$i", p1, p2)) - - // Simple filtering and partition pruning - checkAnswer( - df.filter('a > 1 && 'p1 === 2), - for (i <- 2 to 3; p2 <- Seq("foo", "bar")) yield Row(i, s"val_$i", 2, p2)) - - // Simple projection and filtering - checkAnswer( - df.filter('a > 1).select('b, 'a + 1), - for (i <- 2 to 3; _ <- 1 to 2; _ <- Seq("foo", "bar")) yield Row(s"val_$i", i + 1)) - - // Simple projection and partition pruning - checkAnswer( - df.filter('a > 1 && 'p1 < 2).select('b, 'p1), - for (i <- 2 to 3; _ <- Seq("foo", "bar")) yield Row(s"val_$i", 1)) - - // Project many copies of columns with different types (reproduction for SPARK-7858) - checkAnswer( - df.filter('a > 1 && 'p1 < 2).select('b, 'b, 'b, 'b, 'p1, 'p1, 'p1, 'p1), - for (i <- 2 to 3; _ <- Seq("foo", "bar")) - yield Row(s"val_$i", s"val_$i", s"val_$i", s"val_$i", 1, 1, 1, 1)) - - // Self-join - df.registerTempTable("t") - withTempTable("t") { - checkAnswer( - sql( - """SELECT l.a, r.b, l.p1, r.p2 - |FROM t l JOIN t r - |ON l.a = r.a AND l.p1 = r.p1 AND l.p2 = r.p2 - """.stripMargin), - for (i <- 1 to 3; p1 <- 1 to 2; p2 <- Seq("foo", "bar")) yield Row(i, s"val_$i", p1, p2)) - } - } - - private val supportedDataTypes = Seq( - StringType, BinaryType, - NullType, BooleanType, - ByteType, ShortType, IntegerType, LongType, - FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), - DateType, TimestampType, - ArrayType(IntegerType), - MapType(StringType, LongType), - new StructType() - .add("f1", FloatType, nullable = true) - .add("f2", ArrayType(BooleanType, containsNull = true), nullable = true), - new MyDenseVectorUDT() - ).filter(supportsDataType) - - for (dataType <- supportedDataTypes) { - test(s"test all data types - $dataType") { - withTempPath { file => - val path = file.getCanonicalPath - - val dataGenerator = RandomDataGenerator.forType( - dataType = dataType, - nullable = true, - seed = Some(System.nanoTime()) - ).getOrElse { - fail(s"Failed to create data generator for schema $dataType") - } - - // Create a DF for the schema with random data. The index field is used to sort the - // DataFrame. This is a workaround for SPARK-10591. - val schema = new StructType() - .add("index", IntegerType, nullable = false) - .add("col", dataType, nullable = true) - val rdd = sqlContext.sparkContext.parallelize((1 to 10).map(i => Row(i, dataGenerator()))) - val df = sqlContext.createDataFrame(rdd, schema).orderBy("index").coalesce(1) - - df.write - .mode("overwrite") - .format(dataSourceName) - .option("dataSchema", df.schema.json) - .save(path) - - val loadedDF = sqlContext - .read - .format(dataSourceName) - .option("dataSchema", df.schema.json) - .schema(df.schema) - .load(path) - .orderBy("index") - - checkAnswer(loadedDF, df) - } - } - } - - test("save()/load() - non-partitioned table - Overwrite") { - withTempPath { file => - testDF.write.mode(SaveMode.Overwrite).format(dataSourceName).save(file.getCanonicalPath) - testDF.write.mode(SaveMode.Overwrite).format(dataSourceName).save(file.getCanonicalPath) - - checkAnswer( - sqlContext.read.format(dataSourceName) - .option("path", file.getCanonicalPath) - .option("dataSchema", dataSchema.json) - .load(), - testDF.collect()) - } - } - - test("save()/load() - non-partitioned table - Append") { - withTempPath { file => - testDF.write.mode(SaveMode.Overwrite).format(dataSourceName).save(file.getCanonicalPath) - testDF.write.mode(SaveMode.Append).format(dataSourceName).save(file.getCanonicalPath) - - checkAnswer( - sqlContext.read.format(dataSourceName) - .option("dataSchema", dataSchema.json) - .load(file.getCanonicalPath).orderBy("a"), - testDF.unionAll(testDF).orderBy("a").collect()) - } - } - - test("save()/load() - non-partitioned table - ErrorIfExists") { - withTempDir { file => - intercept[AnalysisException] { - testDF.write.format(dataSourceName).mode(SaveMode.ErrorIfExists).save(file.getCanonicalPath) - } - } - } - - test("save()/load() - non-partitioned table - Ignore") { - withTempDir { file => - testDF.write.mode(SaveMode.Ignore).format(dataSourceName).save(file.getCanonicalPath) - - val path = new Path(file.getCanonicalPath) - val fs = path.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) - assert(fs.listStatus(path).isEmpty) - } - } - - test("save()/load() - partitioned table - simple queries") { - withTempPath { file => - partitionedTestDF.write - .format(dataSourceName) - .mode(SaveMode.ErrorIfExists) - .partitionBy("p1", "p2") - .save(file.getCanonicalPath) - - checkQueries( - sqlContext.read.format(dataSourceName) - .option("dataSchema", dataSchema.json) - .load(file.getCanonicalPath)) - } - } - - test("save()/load() - partitioned table - Overwrite") { - withTempPath { file => - partitionedTestDF.write - .format(dataSourceName) - .mode(SaveMode.Overwrite) - .partitionBy("p1", "p2") - .save(file.getCanonicalPath) - - partitionedTestDF.write - .format(dataSourceName) - .mode(SaveMode.Overwrite) - .partitionBy("p1", "p2") - .save(file.getCanonicalPath) - - checkAnswer( - sqlContext.read.format(dataSourceName) - .option("dataSchema", dataSchema.json) - .load(file.getCanonicalPath), - partitionedTestDF.collect()) - } - } - - test("save()/load() - partitioned table - Append") { - withTempPath { file => - partitionedTestDF.write - .format(dataSourceName) - .mode(SaveMode.Overwrite) - .partitionBy("p1", "p2") - .save(file.getCanonicalPath) - - partitionedTestDF.write - .format(dataSourceName) - .mode(SaveMode.Append) - .partitionBy("p1", "p2") - .save(file.getCanonicalPath) - - checkAnswer( - sqlContext.read.format(dataSourceName) - .option("dataSchema", dataSchema.json) - .load(file.getCanonicalPath), - partitionedTestDF.unionAll(partitionedTestDF).collect()) - } - } - - test("save()/load() - partitioned table - Append - new partition values") { - withTempPath { file => - partitionedTestDF1.write - .format(dataSourceName) - .mode(SaveMode.Overwrite) - .partitionBy("p1", "p2") - .save(file.getCanonicalPath) - - partitionedTestDF2.write - .format(dataSourceName) - .mode(SaveMode.Append) - .partitionBy("p1", "p2") - .save(file.getCanonicalPath) - - checkAnswer( - sqlContext.read.format(dataSourceName) - .option("dataSchema", dataSchema.json) - .load(file.getCanonicalPath), - partitionedTestDF.collect()) - } - } - - test("save()/load() - partitioned table - ErrorIfExists") { - withTempDir { file => - intercept[AnalysisException] { - partitionedTestDF.write - .format(dataSourceName) - .mode(SaveMode.ErrorIfExists) - .partitionBy("p1", "p2") - .save(file.getCanonicalPath) - } - } - } - - test("save()/load() - partitioned table - Ignore") { - withTempDir { file => - partitionedTestDF.write - .format(dataSourceName).mode(SaveMode.Ignore).save(file.getCanonicalPath) - - val path = new Path(file.getCanonicalPath) - val fs = path.getFileSystem(SparkHadoopUtil.get.conf) - assert(fs.listStatus(path).isEmpty) - } - } - - test("saveAsTable()/load() - non-partitioned table - Overwrite") { - testDF.write.format(dataSourceName).mode(SaveMode.Overwrite) - .option("dataSchema", dataSchema.json) - .saveAsTable("t") - - withTable("t") { - checkAnswer(sqlContext.table("t"), testDF.collect()) - } - } - - test("saveAsTable()/load() - non-partitioned table - Append") { - testDF.write.format(dataSourceName).mode(SaveMode.Overwrite).saveAsTable("t") - testDF.write.format(dataSourceName).mode(SaveMode.Append).saveAsTable("t") - - withTable("t") { - checkAnswer(sqlContext.table("t"), testDF.unionAll(testDF).orderBy("a").collect()) - } - } - - test("saveAsTable()/load() - non-partitioned table - ErrorIfExists") { - Seq.empty[(Int, String)].toDF().registerTempTable("t") - - withTempTable("t") { - intercept[AnalysisException] { - testDF.write.format(dataSourceName).mode(SaveMode.ErrorIfExists).saveAsTable("t") - } - } - } - - test("saveAsTable()/load() - non-partitioned table - Ignore") { - Seq.empty[(Int, String)].toDF().registerTempTable("t") - - withTempTable("t") { - testDF.write.format(dataSourceName).mode(SaveMode.Ignore).saveAsTable("t") - assert(sqlContext.table("t").collect().isEmpty) - } - } - - test("saveAsTable()/load() - partitioned table - simple queries") { - partitionedTestDF.write.format(dataSourceName) - .mode(SaveMode.Overwrite) - .option("dataSchema", dataSchema.json) - .saveAsTable("t") - - withTable("t") { - checkQueries(sqlContext.table("t")) - } - } - - test("saveAsTable()/load() - partitioned table - boolean type") { - sqlContext.range(2) - .select('id, ('id % 2 === 0).as("b")) - .write.partitionBy("b").saveAsTable("t") - - withTable("t") { - checkAnswer( - sqlContext.table("t").sort('id), - Row(0, true) :: Row(1, false) :: Nil - ) - } - } - - test("saveAsTable()/load() - partitioned table - Overwrite") { - partitionedTestDF.write - .format(dataSourceName) - .mode(SaveMode.Overwrite) - .option("dataSchema", dataSchema.json) - .partitionBy("p1", "p2") - .saveAsTable("t") - - partitionedTestDF.write - .format(dataSourceName) - .mode(SaveMode.Overwrite) - .option("dataSchema", dataSchema.json) - .partitionBy("p1", "p2") - .saveAsTable("t") - - withTable("t") { - checkAnswer(sqlContext.table("t"), partitionedTestDF.collect()) - } - } - - test("saveAsTable()/load() - partitioned table - Append") { - partitionedTestDF.write - .format(dataSourceName) - .mode(SaveMode.Overwrite) - .option("dataSchema", dataSchema.json) - .partitionBy("p1", "p2") - .saveAsTable("t") - - partitionedTestDF.write - .format(dataSourceName) - .mode(SaveMode.Append) - .option("dataSchema", dataSchema.json) - .partitionBy("p1", "p2") - .saveAsTable("t") - - withTable("t") { - checkAnswer(sqlContext.table("t"), partitionedTestDF.unionAll(partitionedTestDF).collect()) - } - } - - test("saveAsTable()/load() - partitioned table - Append - new partition values") { - partitionedTestDF1.write - .format(dataSourceName) - .mode(SaveMode.Overwrite) - .option("dataSchema", dataSchema.json) - .partitionBy("p1", "p2") - .saveAsTable("t") - - partitionedTestDF2.write - .format(dataSourceName) - .mode(SaveMode.Append) - .option("dataSchema", dataSchema.json) - .partitionBy("p1", "p2") - .saveAsTable("t") - - withTable("t") { - checkAnswer(sqlContext.table("t"), partitionedTestDF.collect()) - } - } - - test("saveAsTable()/load() - partitioned table - Append - mismatched partition columns") { - partitionedTestDF1.write - .format(dataSourceName) - .mode(SaveMode.Overwrite) - .option("dataSchema", dataSchema.json) - .partitionBy("p1", "p2") - .saveAsTable("t") - - // Using only a subset of all partition columns - intercept[Throwable] { - partitionedTestDF2.write - .format(dataSourceName) - .mode(SaveMode.Append) - .option("dataSchema", dataSchema.json) - .partitionBy("p1") - .saveAsTable("t") - } - } - - test("saveAsTable()/load() - partitioned table - ErrorIfExists") { - Seq.empty[(Int, String)].toDF().registerTempTable("t") - - withTempTable("t") { - intercept[AnalysisException] { - partitionedTestDF.write - .format(dataSourceName) - .mode(SaveMode.ErrorIfExists) - .option("dataSchema", dataSchema.json) - .partitionBy("p1", "p2") - .saveAsTable("t") - } - } - } - - test("saveAsTable()/load() - partitioned table - Ignore") { - Seq.empty[(Int, String)].toDF().registerTempTable("t") - - withTempTable("t") { - partitionedTestDF.write - .format(dataSourceName) - .mode(SaveMode.Ignore) - .option("dataSchema", dataSchema.json) - .partitionBy("p1", "p2") - .saveAsTable("t") - - assert(sqlContext.table("t").collect().isEmpty) - } - } - - test("Hadoop style globbing") { - withTempPath { file => - partitionedTestDF.write - .format(dataSourceName) - .mode(SaveMode.Overwrite) - .partitionBy("p1", "p2") - .save(file.getCanonicalPath) - - val df = sqlContext.read - .format(dataSourceName) - .option("dataSchema", dataSchema.json) - .load(s"${file.getCanonicalPath}/p1=*/p2=???") - - val expectedPaths = Set( - s"${file.getCanonicalFile}/p1=1/p2=foo", - s"${file.getCanonicalFile}/p1=2/p2=foo", - s"${file.getCanonicalFile}/p1=1/p2=bar", - s"${file.getCanonicalFile}/p1=2/p2=bar" - ).map { p => - val path = new Path(p) - val fs = path.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) - path.makeQualified(fs.getUri, fs.getWorkingDirectory).toString - } - - val actualPaths = df.queryExecution.analyzed.collectFirst { - case LogicalRelation(relation: HadoopFsRelation, _) => - relation.paths.toSet - }.getOrElse { - fail("Expect an FSBasedRelation, but none could be found") - } - - assert(actualPaths === expectedPaths) - checkAnswer(df, partitionedTestDF.collect()) - } - } - - test("SPARK-9735 Partition column type casting") { - withTempPath { file => - val df = (for { - i <- 1 to 3 - p2 <- Seq("foo", "bar") - } yield (i, s"val_$i", 1.0d, p2, 123, 123.123f)).toDF("a", "b", "p1", "p2", "p3", "f") - - val input = df.select( - 'a, - 'b, - 'p1.cast(StringType).as('ps1), - 'p2, - 'p3.cast(FloatType).as('pf1), - 'f) - - withTempTable("t") { - input - .write - .format(dataSourceName) - .mode(SaveMode.Overwrite) - .partitionBy("ps1", "p2", "pf1", "f") - .saveAsTable("t") - - input - .write - .format(dataSourceName) - .mode(SaveMode.Append) - .partitionBy("ps1", "p2", "pf1", "f") - .saveAsTable("t") - - val realData = input.collect() - - checkAnswer(sqlContext.table("t"), realData ++ realData) - } - } - } - - test("SPARK-7616: adjust column name order accordingly when saving partitioned table") { - val df = (1 to 3).map(i => (i, s"val_$i", i * 2)).toDF("a", "b", "c") - - df.write - .format(dataSourceName) - .mode(SaveMode.Overwrite) - .partitionBy("c", "a") - .saveAsTable("t") - - withTable("t") { - checkAnswer(sqlContext.table("t"), df.select('b, 'c, 'a).collect()) - } - } - - // NOTE: This test suite is not super deterministic. On nodes with only relatively few cores - // (4 or even 1), it's hard to reproduce the data loss issue. But on nodes with for example 8 or - // more cores, the issue can be reproduced steadily. Fortunately our Jenkins builder meets this - // requirement. We probably want to move this test case to spark-integration-tests or spark-perf - // later. - test("SPARK-8406: Avoids name collision while writing files") { - withTempPath { dir => - val path = dir.getCanonicalPath - sqlContext - .range(10000) - .repartition(250) - .write - .mode(SaveMode.Overwrite) - .format(dataSourceName) - .save(path) - - assertResult(10000) { - sqlContext - .read - .format(dataSourceName) - .option("dataSchema", StructType(StructField("id", LongType) :: Nil).json) - .load(path) - .count() - } - } - } - - test("SPARK-8578 specified custom output committer will not be used to append data") { - val clonedConf = new Configuration(hadoopConfiguration) - try { - val df = sqlContext.range(1, 10).toDF("i") - withTempPath { dir => - df.write.mode("append").format(dataSourceName).save(dir.getCanonicalPath) - hadoopConfiguration.set( - SQLConf.OUTPUT_COMMITTER_CLASS.key, - classOf[AlwaysFailOutputCommitter].getName) - // Since Parquet has its own output committer setting, also set it - // to AlwaysFailParquetOutputCommitter at here. - hadoopConfiguration.set("spark.sql.parquet.output.committer.class", - classOf[AlwaysFailParquetOutputCommitter].getName) - // Because there data already exists, - // this append should succeed because we will use the output committer associated - // with file format and AlwaysFailOutputCommitter will not be used. - df.write.mode("append").format(dataSourceName).save(dir.getCanonicalPath) - checkAnswer( - sqlContext.read - .format(dataSourceName) - .option("dataSchema", df.schema.json) - .load(dir.getCanonicalPath), - df.unionAll(df)) - - // This will fail because AlwaysFailOutputCommitter is used when we do append. - intercept[Exception] { - df.write.mode("overwrite").format(dataSourceName).save(dir.getCanonicalPath) - } - } - withTempPath { dir => - hadoopConfiguration.set( - SQLConf.OUTPUT_COMMITTER_CLASS.key, - classOf[AlwaysFailOutputCommitter].getName) - // Since Parquet has its own output committer setting, also set it - // to AlwaysFailParquetOutputCommitter at here. - hadoopConfiguration.set("spark.sql.parquet.output.committer.class", - classOf[AlwaysFailParquetOutputCommitter].getName) - // Because there is no existing data, - // this append will fail because AlwaysFailOutputCommitter is used when we do append - // and there is no existing data. - intercept[Exception] { - df.write.mode("append").format(dataSourceName).save(dir.getCanonicalPath) - } - } - } finally { - // Hadoop 1 doesn't have `Configuration.unset` - hadoopConfiguration.clear() - clonedConf.asScala.foreach(entry => hadoopConfiguration.set(entry.getKey, entry.getValue)) - } - } - - test("SPARK-8887: Explicitly define which data types can be used as dynamic partition columns") { - val df = Seq( - (1, "v1", Array(1, 2, 3), Map("k1" -> "v1"), Tuple2(1, "4")), - (2, "v2", Array(4, 5, 6), Map("k2" -> "v2"), Tuple2(2, "5")), - (3, "v3", Array(7, 8, 9), Map("k3" -> "v3"), Tuple2(3, "6"))).toDF("a", "b", "c", "d", "e") - withTempDir { file => - intercept[AnalysisException] { - df.write.format(dataSourceName).partitionBy("c", "d", "e").save(file.getCanonicalPath) - } - } - intercept[AnalysisException] { - df.write.format(dataSourceName).partitionBy("c", "d", "e").saveAsTable("t") - } - } - - test("SPARK-9899 Disable customized output committer when speculation is on") { - val clonedConf = new Configuration(hadoopConfiguration) - val speculationEnabled = - sqlContext.sparkContext.conf.getBoolean("spark.speculation", defaultValue = false) - - try { - withTempPath { dir => - // Enables task speculation - sqlContext.sparkContext.conf.set("spark.speculation", "true") - - // Uses a customized output committer which always fails - hadoopConfiguration.set( - SQLConf.OUTPUT_COMMITTER_CLASS.key, - classOf[AlwaysFailOutputCommitter].getName) - - // Code below shouldn't throw since customized output committer should be disabled. - val df = sqlContext.range(10).coalesce(1) - df.write.format(dataSourceName).save(dir.getCanonicalPath) - checkAnswer( - sqlContext - .read - .format(dataSourceName) - .option("dataSchema", df.schema.json) - .load(dir.getCanonicalPath), - df) - } - } finally { - // Hadoop 1 doesn't have `Configuration.unset` - hadoopConfiguration.clear() - clonedConf.asScala.foreach(entry => hadoopConfiguration.set(entry.getKey, entry.getValue)) - sqlContext.sparkContext.conf.set("spark.speculation", speculationEnabled.toString) - } - } - - test("HadoopFsRelation produces UnsafeRow") { - withTempTable("test_unsafe") { - withTempPath { dir => - val path = dir.getCanonicalPath - sqlContext.range(3).write.format(dataSourceName).save(path) - sqlContext.read - .format(dataSourceName) - .option("dataSchema", new StructType().add("id", LongType, nullable = false).json) - .load(path) - .registerTempTable("test_unsafe") - - val df = sqlContext.sql( - """SELECT COUNT(*) - |FROM test_unsafe a JOIN test_unsafe b - |WHERE a.id = b.id - """.stripMargin) - - val plan = df.queryExecution.executedPlan - - assert( - plan.collect { case plan: ConvertToUnsafe => plan }.isEmpty, - s"""Query plan shouldn't have ${classOf[ConvertToUnsafe].getSimpleName} node(s): - |$plan - """.stripMargin) - - checkAnswer(df, Row(3)) - } - } - } -} - -// This class is used to test SPARK-8578. We should not use any custom output committer when -// we actually append data to an existing dir. -class AlwaysFailOutputCommitter( - outputPath: Path, - context: TaskAttemptContext) - extends FileOutputCommitter(outputPath, context) { - - override def commitJob(context: JobContext): Unit = { - sys.error("Intentional job commitment failure for testing purpose.") - } -} - -// This class is used to test SPARK-8578. We should not use any custom output committer when -// we actually append data to an existing dir. -class AlwaysFailParquetOutputCommitter( - outputPath: Path, - context: TaskAttemptContext) - extends ParquetOutputCommitter(outputPath, context) { - - override def commitJob(context: JobContext): Unit = { - sys.error("Intentional job commitment failure for testing purpose.") - } -} diff --git a/sql/mkdocs.yml b/sql/mkdocs.yml new file mode 100644 index 000000000000..c34c891bb9e4 --- /dev/null +++ b/sql/mkdocs.yml @@ -0,0 +1,19 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +site_name: Spark SQL, Built-in Functions +theme: readthedocs +pages: + - 'Functions': 'index.md' diff --git a/streaming/pom.xml b/streaming/pom.xml index 145c8a7321c0..fea882ad1123 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -20,13 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT + spark-parent_2.11 + 2.3.0-SNAPSHOT ../pom.xml - org.apache.spark - spark-streaming_2.10 + spark-streaming_2.11 streaming @@ -49,7 +48,18 @@ org.apache.spark - spark-test-tags_${scala.binary.version} + spark-tags_${scala.binary.version} + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test @@ -77,6 +87,10 @@ org.eclipse.jetty jetty-servlet + + org.eclipse.jetty + jetty-servlets + @@ -93,6 +107,16 @@ selenium-java test + + org.seleniumhq.selenium + selenium-htmlunit-driver + test + + + org.mockito + mockito-core + test + target/scala-${scala.binary.version}/classes diff --git a/streaming/src/main/java/org/apache/spark/status/api/v1/streaming/BatchStatus.java b/streaming/src/main/java/org/apache/spark/status/api/v1/streaming/BatchStatus.java new file mode 100644 index 000000000000..1bbca5a2259d --- /dev/null +++ b/streaming/src/main/java/org/apache/spark/status/api/v1/streaming/BatchStatus.java @@ -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.status.api.v1.streaming; + +import org.apache.spark.util.EnumUtil; + +public enum BatchStatus { + COMPLETED, + QUEUED, + PROCESSING; + + public static BatchStatus fromString(String str) { + return EnumUtil.parseIgnoreCase(BatchStatus.class, str); + } +} diff --git a/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLog.java b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLog.java index 3738fc1a235c..00c59728748f 100644 --- a/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLog.java +++ b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLog.java @@ -37,26 +37,26 @@ public abstract class WriteAheadLog { * ensure that the written data is durable and readable (using the record handle) by the * time this function returns. */ - abstract public WriteAheadLogRecordHandle write(ByteBuffer record, long time); + public abstract WriteAheadLogRecordHandle write(ByteBuffer record, long time); /** * Read a written record based on the given record handle. */ - abstract public ByteBuffer read(WriteAheadLogRecordHandle handle); + public abstract ByteBuffer read(WriteAheadLogRecordHandle handle); /** * Read and return an iterator of all the records that have been written but not yet cleaned up. */ - abstract public Iterator readAll(); + public abstract Iterator readAll(); /** * Clean all the records that are older than the threshold time. It can wait for * the completion of the deletion. */ - abstract public void clean(long threshTime, boolean waitForCompletion); + public abstract void clean(long threshTime, boolean waitForCompletion); /** - * Close this log and release any resources. + * Close this log and release any resources. It must be idempotent. */ - abstract public void close(); + public abstract void close(); } diff --git a/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java index 662889e779fb..3c5cc7e2cae1 100644 --- a/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java +++ b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java @@ -23,7 +23,7 @@ * This abstract class represents a handle that refers to a record written in a * {@link org.apache.spark.streaming.util.WriteAheadLog WriteAheadLog}. * It must contain all the information necessary for the record to be read and returned by - * an implemenation of the WriteAheadLog class. + * an implementation of the WriteAheadLog class. * * @see org.apache.spark.streaming.util.WriteAheadLog */ diff --git a/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js b/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js index f82323a1cdd9..d004f34ab186 100644 --- a/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js +++ b/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js @@ -169,7 +169,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, unitY, batchInterval) { .style("cursor", "pointer") .attr("cx", function(d) { return x(d.x); }) .attr("cy", function(d) { return y(d.y); }) - .attr("r", function(d) { return isFailedBatch(d.x) ? "2" : "0";}) + .attr("r", function(d) { return isFailedBatch(d.x) ? "2" : "3";}) .on('mouseover', function(d) { var tip = formatYValue(d.y) + " " + unitY + " at " + timeFormat[d.x]; showBootstrapTooltip(d3.select(this).node(), tip); @@ -187,7 +187,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, unitY, batchInterval) { .attr("stroke", function(d) { return isFailedBatch(d.x) ? "red" : "white";}) .attr("fill", function(d) { return isFailedBatch(d.x) ? "red" : "white";}) .attr("opacity", function(d) { return isFailedBatch(d.x) ? "1" : "0";}) - .attr("r", function(d) { return isFailedBatch(d.x) ? "2" : "0";}); + .attr("r", function(d) { return isFailedBatch(d.x) ? "2" : "3";}); }) .on("click", function(d) { if (lastTimeout != null) { diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllBatchesResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllBatchesResource.scala new file mode 100644 index 000000000000..3a51ae609303 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllBatchesResource.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.status.api.v1.streaming + +import java.util.{ArrayList => JArrayList, Arrays => JArrays, Date, List => JList} +import javax.ws.rs.{GET, Produces, QueryParam} +import javax.ws.rs.core.MediaType + +import org.apache.spark.status.api.v1.streaming.AllBatchesResource._ +import org.apache.spark.streaming.ui.StreamingJobProgressListener + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class AllBatchesResource(listener: StreamingJobProgressListener) { + + @GET + def batchesList(@QueryParam("status") statusParams: JList[BatchStatus]): Seq[BatchInfo] = { + batchInfoList(listener, statusParams).sortBy(- _.batchId) + } +} + +private[v1] object AllBatchesResource { + + def batchInfoList( + listener: StreamingJobProgressListener, + statusParams: JList[BatchStatus] = new JArrayList[BatchStatus]()): Seq[BatchInfo] = { + + listener.synchronized { + val statuses = + if (statusParams.isEmpty) JArrays.asList(BatchStatus.values(): _*) else statusParams + val statusToBatches = Seq( + BatchStatus.COMPLETED -> listener.retainedCompletedBatches, + BatchStatus.QUEUED -> listener.waitingBatches, + BatchStatus.PROCESSING -> listener.runningBatches + ) + + val batchInfos = for { + (status, batches) <- statusToBatches + batch <- batches if statuses.contains(status) + } yield { + val batchId = batch.batchTime.milliseconds + val firstFailureReason = batch.outputOperations.flatMap(_._2.failureReason).headOption + + new BatchInfo( + batchId = batchId, + batchTime = new Date(batchId), + status = status.toString, + batchDuration = listener.batchDuration, + inputSize = batch.numRecords, + schedulingDelay = batch.schedulingDelay, + processingTime = batch.processingDelay, + totalDelay = batch.totalDelay, + numActiveOutputOps = batch.numActiveOutputOp, + numCompletedOutputOps = batch.numCompletedOutputOp, + numFailedOutputOps = batch.numFailedOutputOp, + numTotalOutputOps = batch.outputOperations.size, + firstFailureReason = firstFailureReason + ) + } + + batchInfos + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllOutputOperationsResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllOutputOperationsResource.scala new file mode 100644 index 000000000000..0eb649f0e1b7 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllOutputOperationsResource.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.status.api.v1.streaming + +import java.util.Date +import javax.ws.rs.{GET, PathParam, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.status.api.v1.NotFoundException +import org.apache.spark.status.api.v1.streaming.AllOutputOperationsResource._ +import org.apache.spark.streaming.Time +import org.apache.spark.streaming.ui.StreamingJobProgressListener + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class AllOutputOperationsResource(listener: StreamingJobProgressListener) { + + @GET + def operationsList(@PathParam("batchId") batchId: Long): Seq[OutputOperationInfo] = { + outputOperationInfoList(listener, batchId).sortBy(_.outputOpId) + } +} + +private[v1] object AllOutputOperationsResource { + + def outputOperationInfoList( + listener: StreamingJobProgressListener, + batchId: Long): Seq[OutputOperationInfo] = { + + listener.synchronized { + listener.getBatchUIData(Time(batchId)) match { + case Some(batch) => + for ((opId, op) <- batch.outputOperations) yield { + val jobIds = batch.outputOpIdSparkJobIdPairs + .filter(_.outputOpId == opId).map(_.sparkJobId).toSeq.sorted + + new OutputOperationInfo( + outputOpId = opId, + name = op.name, + description = op.description, + startTime = op.startTime.map(new Date(_)), + endTime = op.endTime.map(new Date(_)), + duration = op.duration, + failureReason = op.failureReason, + jobIds = jobIds + ) + } + case None => throw new NotFoundException("unknown batch: " + batchId) + } + }.toSeq + } +} diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllReceiversResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllReceiversResource.scala new file mode 100644 index 000000000000..5a276a9236a0 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/AllReceiversResource.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.status.api.v1.streaming + +import java.util.Date +import javax.ws.rs.{GET, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.status.api.v1.streaming.AllReceiversResource._ +import org.apache.spark.streaming.ui.StreamingJobProgressListener + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class AllReceiversResource(listener: StreamingJobProgressListener) { + + @GET + def receiversList(): Seq[ReceiverInfo] = { + receiverInfoList(listener).sortBy(_.streamId) + } +} + +private[v1] object AllReceiversResource { + + def receiverInfoList(listener: StreamingJobProgressListener): Seq[ReceiverInfo] = { + listener.synchronized { + listener.receivedRecordRateWithBatchTime.map { case (streamId, eventRates) => + + val receiverInfo = listener.receiverInfo(streamId) + val streamName = receiverInfo.map(_.name) + .orElse(listener.streamName(streamId)).getOrElse(s"Stream-$streamId") + val avgEventRate = + if (eventRates.isEmpty) None else Some(eventRates.map(_._2).sum / eventRates.size) + + val (errorTime, errorMessage, error) = receiverInfo match { + case None => (None, None, None) + case Some(info) => + val someTime = + if (info.lastErrorTime >= 0) Some(new Date(info.lastErrorTime)) else None + val someMessage = + if (info.lastErrorMessage.length > 0) Some(info.lastErrorMessage) else None + val someError = + if (info.lastError.length > 0) Some(info.lastError) else None + + (someTime, someMessage, someError) + } + + new ReceiverInfo( + streamId = streamId, + streamName = streamName, + isActive = receiverInfo.map(_.active), + executorId = receiverInfo.map(_.executorId), + executorHost = receiverInfo.map(_.location), + lastErrorTime = errorTime, + lastErrorMessage = errorMessage, + lastError = error, + avgEventRate = avgEventRate, + eventRates = eventRates + ) + }.toSeq + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/ApiStreamingApp.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/ApiStreamingApp.scala new file mode 100644 index 000000000000..aea75d5a9c8d --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/ApiStreamingApp.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.status.api.v1.streaming + +import javax.ws.rs.{Path, PathParam} + +import org.apache.spark.status.api.v1.ApiRequestContext + +@Path("/v1") +private[v1] class ApiStreamingApp extends ApiRequestContext { + + @Path("applications/{appId}/streaming") + def getStreamingRoot(@PathParam("appId") appId: String): ApiStreamingRootResource = { + withSparkUI(appId, None) { ui => + new ApiStreamingRootResource(ui) + } + } + + @Path("applications/{appId}/{attemptId}/streaming") + def getStreamingRoot( + @PathParam("appId") appId: String, + @PathParam("attemptId") attemptId: String): ApiStreamingRootResource = { + withSparkUI(appId, Some(attemptId)) { ui => + new ApiStreamingRootResource(ui) + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/ApiStreamingRootResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/ApiStreamingRootResource.scala new file mode 100644 index 000000000000..1ccd586c848b --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/ApiStreamingRootResource.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.status.api.v1.streaming + +import javax.ws.rs.Path + +import org.apache.spark.status.api.v1.NotFoundException +import org.apache.spark.streaming.ui.StreamingJobProgressListener +import org.apache.spark.ui.SparkUI + +private[v1] class ApiStreamingRootResource(ui: SparkUI) { + + import org.apache.spark.status.api.v1.streaming.ApiStreamingRootResource._ + + @Path("statistics") + def getStreamingStatistics(): StreamingStatisticsResource = { + new StreamingStatisticsResource(getListener(ui)) + } + + @Path("receivers") + def getReceivers(): AllReceiversResource = { + new AllReceiversResource(getListener(ui)) + } + + @Path("receivers/{streamId: \\d+}") + def getReceiver(): OneReceiverResource = { + new OneReceiverResource(getListener(ui)) + } + + @Path("batches") + def getBatches(): AllBatchesResource = { + new AllBatchesResource(getListener(ui)) + } + + @Path("batches/{batchId: \\d+}") + def getBatch(): OneBatchResource = { + new OneBatchResource(getListener(ui)) + } + + @Path("batches/{batchId: \\d+}/operations") + def getOutputOperations(): AllOutputOperationsResource = { + new AllOutputOperationsResource(getListener(ui)) + } + + @Path("batches/{batchId: \\d+}/operations/{outputOpId: \\d+}") + def getOutputOperation(): OneOutputOperationResource = { + new OneOutputOperationResource(getListener(ui)) + } + +} + +private[v1] object ApiStreamingRootResource { + def getListener(ui: SparkUI): StreamingJobProgressListener = { + ui.getStreamingJobProgressListener match { + case Some(listener) => listener.asInstanceOf[StreamingJobProgressListener] + case None => throw new NotFoundException("no streaming listener attached to " + ui.getAppName) + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneBatchResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneBatchResource.scala new file mode 100644 index 000000000000..d3c689c790cf --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneBatchResource.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.status.api.v1.streaming + +import javax.ws.rs.{GET, PathParam, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.status.api.v1.NotFoundException +import org.apache.spark.streaming.ui.StreamingJobProgressListener + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class OneBatchResource(listener: StreamingJobProgressListener) { + + @GET + def oneBatch(@PathParam("batchId") batchId: Long): BatchInfo = { + val someBatch = AllBatchesResource.batchInfoList(listener) + .find { _.batchId == batchId } + someBatch.getOrElse(throw new NotFoundException("unknown batch: " + batchId)) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneOutputOperationResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneOutputOperationResource.scala new file mode 100644 index 000000000000..aabcdb29b0d4 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneOutputOperationResource.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.status.api.v1.streaming + +import javax.ws.rs.{GET, PathParam, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.status.api.v1.NotFoundException +import org.apache.spark.streaming.ui.StreamingJobProgressListener +import org.apache.spark.streaming.ui.StreamingJobProgressListener._ + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class OneOutputOperationResource(listener: StreamingJobProgressListener) { + + @GET + def oneOperation( + @PathParam("batchId") batchId: Long, + @PathParam("outputOpId") opId: OutputOpId): OutputOperationInfo = { + + val someOutputOp = AllOutputOperationsResource.outputOperationInfoList(listener, batchId) + .find { _.outputOpId == opId } + someOutputOp.getOrElse(throw new NotFoundException("unknown output operation: " + opId)) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneReceiverResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneReceiverResource.scala new file mode 100644 index 000000000000..c0cc99da3a9c --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/OneReceiverResource.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.status.api.v1.streaming + +import javax.ws.rs.{GET, PathParam, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.status.api.v1.NotFoundException +import org.apache.spark.streaming.ui.StreamingJobProgressListener + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class OneReceiverResource(listener: StreamingJobProgressListener) { + + @GET + def oneReceiver(@PathParam("streamId") streamId: Int): ReceiverInfo = { + val someReceiver = AllReceiversResource.receiverInfoList(listener) + .find { _.streamId == streamId } + someReceiver.getOrElse(throw new NotFoundException("unknown receiver: " + streamId)) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/StreamingStatisticsResource.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/StreamingStatisticsResource.scala new file mode 100644 index 000000000000..6cff87be59ca --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/StreamingStatisticsResource.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.status.api.v1.streaming + +import java.util.Date +import javax.ws.rs.{GET, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.streaming.ui.StreamingJobProgressListener + +@Produces(Array(MediaType.APPLICATION_JSON)) +private[v1] class StreamingStatisticsResource(listener: StreamingJobProgressListener) { + + @GET + def streamingStatistics(): StreamingStatistics = { + listener.synchronized { + val batches = listener.retainedBatches + val avgInputRate = avgRate(batches.map(_.numRecords * 1000.0 / listener.batchDuration)) + val avgSchedulingDelay = avgTime(batches.flatMap(_.schedulingDelay)) + val avgProcessingTime = avgTime(batches.flatMap(_.processingDelay)) + val avgTotalDelay = avgTime(batches.flatMap(_.totalDelay)) + + new StreamingStatistics( + startTime = new Date(listener.startTime), + batchDuration = listener.batchDuration, + numReceivers = listener.numReceivers, + numActiveReceivers = listener.numActiveReceivers, + numInactiveReceivers = listener.numInactiveReceivers, + numTotalCompletedBatches = listener.numTotalCompletedBatches, + numRetainedCompletedBatches = listener.retainedCompletedBatches.size, + numActiveBatches = listener.numUnprocessedBatches, + numProcessedRecords = listener.numTotalProcessedRecords, + numReceivedRecords = listener.numTotalReceivedRecords, + avgInputRate = avgInputRate, + avgSchedulingDelay = avgSchedulingDelay, + avgProcessingTime = avgProcessingTime, + avgTotalDelay = avgTotalDelay + ) + } + } + + private def avgRate(data: Seq[Double]): Option[Double] = { + if (data.isEmpty) None else Some(data.sum / data.size) + } + + private def avgTime(data: Seq[Long]): Option[Long] = { + if (data.isEmpty) None else Some(data.sum / data.size) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/api.scala b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/api.scala new file mode 100644 index 000000000000..403b0eb0b5d6 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/status/api/v1/streaming/api.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.status.api.v1.streaming + +import java.util.Date + +import org.apache.spark.streaming.ui.StreamingJobProgressListener._ + +class StreamingStatistics private[spark]( + val startTime: Date, + val batchDuration: Long, + val numReceivers: Int, + val numActiveReceivers: Int, + val numInactiveReceivers: Int, + val numTotalCompletedBatches: Long, + val numRetainedCompletedBatches: Long, + val numActiveBatches: Long, + val numProcessedRecords: Long, + val numReceivedRecords: Long, + val avgInputRate: Option[Double], + val avgSchedulingDelay: Option[Long], + val avgProcessingTime: Option[Long], + val avgTotalDelay: Option[Long]) + +class ReceiverInfo private[spark]( + val streamId: Int, + val streamName: String, + val isActive: Option[Boolean], + val executorId: Option[String], + val executorHost: Option[String], + val lastErrorTime: Option[Date], + val lastErrorMessage: Option[String], + val lastError: Option[String], + val avgEventRate: Option[Double], + val eventRates: Seq[(Long, Double)]) + +class BatchInfo private[spark]( + val batchId: Long, + val batchTime: Date, + val status: String, + val batchDuration: Long, + val inputSize: Long, + val schedulingDelay: Option[Long], + val processingTime: Option[Long], + val totalDelay: Option[Long], + val numActiveOutputOps: Int, + val numCompletedOutputOps: Int, + val numFailedOutputOps: Int, + val numTotalOutputOps: Int, + val firstFailureReason: Option[String]) + +class OutputOperationInfo private[spark]( + val outputOpId: OutputOpId, + val name: String, + val description: String, + val startTime: Option[Date], + val endTime: Option[Date], + val duration: Option[Long], + val failureReason: Option[String], + val jobIds: Seq[SparkJobId]) 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 b7de6dde61c6..b8c780db07c9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -18,18 +18,18 @@ package org.apache.spark.streaming import java.io._ -import java.util.concurrent.Executors -import java.util.concurrent.RejectedExecutionException +import java.util.concurrent.{ArrayBlockingQueue, RejectedExecutionException, + ThreadPoolExecutor, TimeUnit} -import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.{SparkException, SparkConf, Logging} +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{MetadataCleaner, Utils} import org.apache.spark.streaming.scheduler.JobGenerator - +import org.apache.spark.util.Utils private[streaming] class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) @@ -41,7 +41,6 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) val checkpointDir = ssc.checkpointDir val checkpointDuration = ssc.checkpointDuration val pendingTimes = ssc.scheduler.getPendingTimes().toArray - val delaySeconds = MetadataCleaner.getDelaySeconds(ssc.conf) val sparkConfPairs = ssc.conf.getAll def createSparkConf(): SparkConf = { @@ -55,7 +54,11 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) "spark.driver.port", "spark.master", "spark.yarn.keytab", - "spark.yarn.principal") + "spark.yarn.principal", + "spark.yarn.credentials.file", + "spark.yarn.credentials.renewalTime", + "spark.yarn.credentials.updateTime", + "spark.ui.filters") val newSparkConf = new SparkConf(loadDefaults = false).setAll(sparkConfPairs) .remove("spark.driver.host") @@ -66,6 +69,16 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) newSparkConf.set(prop, value) } } + + // Add Yarn proxy filter specific configurations to the recovered SparkConf + val filter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" + val filterPrefix = s"spark.$filter.param." + newReloadConf.getAll.foreach { case (k, v) => + if (k.startsWith(filterPrefix) && k.length > filterPrefix.length) { + newSparkConf.set(k, v) + } + } + newSparkConf } @@ -74,7 +87,7 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) assert(framework != null, "Checkpoint.framework is null") assert(graph != null, "Checkpoint.graph is null") assert(checkpointTime != null, "Checkpoint.checkpointTime is null") - logInfo("Checkpoint for time " + checkpointTime + " validated") + logInfo(s"Checkpoint for time $checkpointTime validated") } } @@ -93,7 +106,10 @@ object Checkpoint extends Logging { new Path(checkpointDir, PREFIX + checkpointTime.milliseconds + ".bk") } - /** Get checkpoint files present in the give directory, ordered by oldest-first */ + /** + * @param checkpointDir checkpoint directory to read checkpoint files from + * @return checkpoint files from the `checkpointDir` checkpoint directory, ordered by oldest-first + */ def getCheckpointFiles(checkpointDir: String, fsOption: Option[FileSystem] = None): Seq[Path] = { def sortFunc(path1: Path, path2: Path): Boolean = { @@ -104,19 +120,20 @@ object Checkpoint extends Logging { val path = new Path(checkpointDir) val fs = fsOption.getOrElse(path.getFileSystem(SparkHadoopUtil.get.conf)) - if (fs.exists(path)) { + try { val statuses = fs.listStatus(path) if (statuses != null) { val paths = statuses.map(_.getPath) val filtered = paths.filter(p => REGEX.findFirstIn(p.toString).nonEmpty) filtered.sortWith(sortFunc) } else { - logWarning("Listing " + path + " returned null") + logWarning(s"Listing $path returned null") Seq.empty } - } else { - logInfo("Checkpoint directory " + path + " does not exist") - Seq.empty + } catch { + case _: FileNotFoundException => + logWarning(s"Checkpoint directory $path does not exist") + Seq.empty } } @@ -141,7 +158,7 @@ object Checkpoint extends Logging { Utils.tryWithSafeFinally { // ObjectInputStream uses the last defined user-defined class loader in the stack - // to find classes, which maybe the wrong class loader. Hence, a inherited version + // to find classes, which maybe the wrong class loader. Hence, an inherited version // of ObjectInputStream is used to explicitly use the current thread's default class // loader to find and load classes. This is a well know Java issue and has popped up // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627) @@ -171,32 +188,52 @@ class CheckpointWriter( hadoopConf: Configuration ) extends Logging { val MAX_ATTEMPTS = 3 - val executor = Executors.newFixedThreadPool(1) + + // Single-thread executor which rejects executions when a large amount have queued up. + // This fails fast since this typically means the checkpoint store will never keep up, and + // will otherwise lead to filling memory with waiting payloads of byte[] to write. + val executor = new ThreadPoolExecutor( + 1, 1, + 0L, TimeUnit.MILLISECONDS, + new ArrayBlockingQueue[Runnable](1000)) val compressionCodec = CompressionCodec.createCodec(conf) private var stopped = false - private var fs_ : FileSystem = _ + @volatile private[this] var fs: FileSystem = null + @volatile private var latestCheckpointTime: Time = null class CheckpointWriteHandler( checkpointTime: Time, bytes: Array[Byte], clearCheckpointDataLater: Boolean) extends Runnable { def run() { + if (latestCheckpointTime == null || latestCheckpointTime < checkpointTime) { + latestCheckpointTime = checkpointTime + } + if (fs == null) { + fs = new Path(checkpointDir).getFileSystem(hadoopConf) + } var attempts = 0 val startTime = System.currentTimeMillis() val tempFile = new Path(checkpointDir, "temp") - val checkpointFile = Checkpoint.checkpointFile(checkpointDir, checkpointTime) - val backupFile = Checkpoint.checkpointBackupFile(checkpointDir, checkpointTime) + // We will do checkpoint when generating a batch and completing a batch. When the processing + // time of a batch is greater than the batch interval, checkpointing for completing an old + // batch may run after checkpointing of a new batch. If this happens, checkpoint of an old + // batch actually has the latest information, so we want to recovery from it. Therefore, we + // also use the latest checkpoint time as the file name, so that we can recover from the + // latest checkpoint file. + // + // Note: there is only one thread writing the checkpoint files, so we don't need to worry + // about thread-safety. + val checkpointFile = Checkpoint.checkpointFile(checkpointDir, latestCheckpointTime) + val backupFile = Checkpoint.checkpointBackupFile(checkpointDir, latestCheckpointTime) while (attempts < MAX_ATTEMPTS && !stopped) { attempts += 1 try { - logInfo("Saving checkpoint for time " + checkpointTime + " to file '" + checkpointFile - + "'") + logInfo(s"Saving checkpoint for time $checkpointTime to file '$checkpointFile'") // Write checkpoint to temp file - if (fs.exists(tempFile)) { - fs.delete(tempFile, true) // just in case it exists - } + fs.delete(tempFile, true) // just in case it exists val fos = fs.create(tempFile) Utils.tryWithSafeFinally { fos.write(bytes) @@ -207,43 +244,40 @@ class CheckpointWriter( // If the checkpoint file exists, back it up // If the backup exists as well, just delete it, otherwise rename will fail if (fs.exists(checkpointFile)) { - if (fs.exists(backupFile)){ - fs.delete(backupFile, true) // just in case it exists - } + fs.delete(backupFile, true) // just in case it exists if (!fs.rename(checkpointFile, backupFile)) { - logWarning("Could not rename " + checkpointFile + " to " + backupFile) + logWarning(s"Could not rename $checkpointFile to $backupFile") } } // Rename temp file to the final checkpoint file if (!fs.rename(tempFile, checkpointFile)) { - logWarning("Could not rename " + tempFile + " to " + checkpointFile) + logWarning(s"Could not rename $tempFile to $checkpointFile") } // Delete old checkpoint files val allCheckpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, Some(fs)) if (allCheckpointFiles.size > 10) { - allCheckpointFiles.take(allCheckpointFiles.size - 10).foreach(file => { - logInfo("Deleting " + file) + allCheckpointFiles.take(allCheckpointFiles.size - 10).foreach { file => + logInfo(s"Deleting $file") fs.delete(file, true) - }) + } } // All done, print success val finishTime = System.currentTimeMillis() - logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + checkpointFile + - "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " ms") + logInfo(s"Checkpoint for time $checkpointTime saved to file '$checkpointFile'" + + s", took ${bytes.length} bytes and ${finishTime - startTime} ms") jobGenerator.onCheckpointCompletion(checkpointTime, clearCheckpointDataLater) return } catch { case ioe: IOException => - logWarning("Error in attempt " + attempts + " of writing checkpoint to " - + checkpointFile, ioe) - reset() + val msg = s"Error in attempt $attempts of writing checkpoint to '$checkpointFile'" + logWarning(msg, ioe) + fs = null } } - logWarning("Could not write checkpoint for time " + checkpointTime + " to file " - + checkpointFile + "'") + logWarning(s"Could not write checkpoint for time $checkpointTime to file '$checkpointFile'") } } @@ -252,7 +286,7 @@ class CheckpointWriter( val bytes = Checkpoint.serialize(checkpoint, conf) executor.execute(new CheckpointWriteHandler( checkpoint.checkpointTime, bytes, clearCheckpointDataLater)) - logDebug("Submitted checkpoint of time " + checkpoint.checkpointTime + " writer queue") + logInfo(s"Submitted checkpoint of time ${checkpoint.checkpointTime} to writer queue") } catch { case rej: RejectedExecutionException => logError("Could not submit checkpoint task to the thread pool executor", rej) @@ -269,19 +303,10 @@ class CheckpointWriter( executor.shutdownNow() } val endTime = System.currentTimeMillis() - logInfo("CheckpointWriter executor terminated ? " + terminated + - ", waited for " + (endTime - startTime) + " ms.") + logInfo(s"CheckpointWriter executor terminated? $terminated," + + s" waited for ${endTime - startTime} ms.") stopped = true } - - private def fs = synchronized { - if (fs_ == null) fs_ = new Path(checkpointDir).getFileSystem(hadoopConf) - fs_ - } - - private def reset() = synchronized { - fs_ = null - } } @@ -310,8 +335,7 @@ object CheckpointReader extends Logging { ignoreReadError: Boolean = false): Option[Checkpoint] = { val checkpointPath = new Path(checkpointDir) - // TODO(rxin): Why is this a def?! - def fs: FileSystem = checkpointPath.getFileSystem(hadoopConf) + val fs = checkpointPath.getFileSystem(hadoopConf) // Try to find the checkpoint files val checkpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, Some(fs)).reverse @@ -320,22 +344,22 @@ object CheckpointReader extends Logging { } // Try to read the checkpoint files in the order - logInfo("Checkpoint files found: " + checkpointFiles.mkString(",")) + logInfo(s"Checkpoint files found: ${checkpointFiles.mkString(",")}") var readError: Exception = null - checkpointFiles.foreach(file => { - logInfo("Attempting to load checkpoint from file " + file) + checkpointFiles.foreach { file => + logInfo(s"Attempting to load checkpoint from file $file") try { val fis = fs.open(file) val cp = Checkpoint.deserialize(fis, conf) - logInfo("Checkpoint successfully loaded from file " + file) - logInfo("Checkpoint was generated at time " + cp.checkpointTime) + logInfo(s"Checkpoint successfully loaded from file $file") + logInfo(s"Checkpoint was generated at time ${cp.checkpointTime}") return Some(cp) } catch { case e: Exception => readError = e - logWarning("Error reading checkpoint from file " + file, e) + logWarning(s"Error reading checkpoint from file $file", e) } - }) + } // If none of checkpoint files could be read, then throw exception if (!ignoreReadError) { @@ -347,8 +371,8 @@ object CheckpointReader extends Logging { } private[streaming] -class ObjectInputStreamWithLoader(inputStream_ : InputStream, loader: ClassLoader) - extends ObjectInputStream(inputStream_) { +class ObjectInputStreamWithLoader(_inputStream: InputStream, loader: ClassLoader) + extends ObjectInputStream(_inputStream) { override def resolveClass(desc: ObjectStreamClass): Class[_] = { try { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index 1b0b7890b3b0..dce2028b4887 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -17,11 +17,13 @@ package org.apache.spark.streaming +import java.io.{IOException, ObjectInputStream, ObjectOutputStream} + import scala.collection.mutable.ArrayBuffer -import java.io.{ObjectInputStream, IOException, ObjectOutputStream} -import org.apache.spark.Logging + +import org.apache.spark.internal.Logging +import org.apache.spark.streaming.dstream.{DStream, InputDStream, ReceiverInputDStream} import org.apache.spark.streaming.scheduler.Job -import org.apache.spark.streaming.dstream.{DStream, ReceiverInputDStream, InputDStream} import org.apache.spark.util.Utils final private[streaming] class DStreamGraph extends Serializable with Logging { @@ -29,12 +31,15 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { private val inputStreams = new ArrayBuffer[InputDStream[_]]() private val outputStreams = new ArrayBuffer[DStream[_]]() + @volatile private var inputStreamNameAndID: Seq[(String, Int)] = Nil + var rememberDuration: Duration = null var checkpointInProgress = false var zeroTime: Time = null var startTime: Time = null var batchDuration: Duration = null + @volatile private var numReceivers: Int = 0 def start(time: Time) { this.synchronized { @@ -43,7 +48,9 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { startTime = time outputStreams.foreach(_.initialize(zeroTime)) outputStreams.foreach(_.remember(rememberDuration)) - outputStreams.foreach(_.validateAtStart) + outputStreams.foreach(_.validateAtStart()) + numReceivers = inputStreams.count(_.isInstanceOf[ReceiverInputDStream[_]]) + inputStreamNameAndID = inputStreams.map(is => (is.name, is.id)) inputStreams.par.foreach(_.start()) } } @@ -104,9 +111,9 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { .toArray } - def getInputStreamName(streamId: Int): Option[String] = synchronized { - inputStreams.find(_.id == streamId).map(_.name) - } + def getNumReceivers: Int = numReceivers + + def getInputStreamNameAndID: Seq[(String, Int)] = inputStreamNameAndID def generateJobs(time: Time): Seq[Job] = { logDebug("Generating jobs for time " + time) @@ -158,7 +165,7 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { require(batchDuration != null, "Batch duration has not been set") // assert(batchDuration >= Milliseconds(100), "Batch duration of " + batchDuration + // " is very low") - require(getOutputStreams().size > 0, "No output operations registered, so nothing to execute") + require(getOutputStreams().nonEmpty, "No output operations registered, so nothing to execute") } } @@ -167,7 +174,8 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { * safe remember duration which can be used to perform cleanup operations. */ def getMaxInputStreamRememberDuration(): Duration = { - inputStreams.map { _.rememberDuration }.maxBy { _.milliseconds } + // If an InputDStream is not used, its `rememberDuration` will be null and we can ignore them + inputStreams.map(_.rememberDuration).filter(_ != null).maxBy(_.milliseconds) } @throws(classOf[IOException]) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/State.scala b/streaming/src/main/scala/org/apache/spark/streaming/State.scala new file mode 100644 index 000000000000..734c6ef42696 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/State.scala @@ -0,0 +1,216 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +import scala.language.implicitConversions + +import org.apache.spark.annotation.Experimental + +/** + * :: Experimental :: + * Abstract class for getting and updating the state in mapping function used in the `mapWithState` + * operation of a [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]] (Scala) + * or a [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream]] (Java). + * + * Scala example of using `State`: + * {{{ + * // A mapping function that maintains an integer state and returns a String + * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = { + * // Check if state exists + * if (state.exists) { + * val existingState = state.get // Get the existing state + * val shouldRemove = ... // Decide whether to remove the state + * if (shouldRemove) { + * state.remove() // Remove the state + * } else { + * val newState = ... + * state.update(newState) // Set the new state + * } + * } else { + * val initialState = ... + * state.update(initialState) // Set the initial state + * } + * ... // return something + * } + * + * }}} + * + * Java example of using `State`: + * {{{ + * // A mapping function that maintains an integer state and returns a String + * Function3, State, String> mappingFunction = + * new Function3, State, String>() { + * + * @Override + * public String call(String key, Optional value, State state) { + * if (state.exists()) { + * int existingState = state.get(); // Get the existing state + * boolean shouldRemove = ...; // Decide whether to remove the state + * if (shouldRemove) { + * state.remove(); // Remove the state + * } else { + * int newState = ...; + * state.update(newState); // Set the new state + * } + * } else { + * int initialState = ...; // Set the initial state + * state.update(initialState); + * } + * // return something + * } + * }; + * }}} + * + * @tparam S Class of the state + */ +@Experimental +sealed abstract class State[S] { + + /** Whether the state already exists */ + def exists(): Boolean + + /** + * Get the state if it exists, otherwise it will throw `java.util.NoSuchElementException`. + * Check with `exists()` whether the state exists or not before calling `get()`. + * + * @throws java.util.NoSuchElementException If the state does not exist. + */ + def get(): S + + /** + * Update the state with a new value. + * + * State cannot be updated if it has been already removed (that is, `remove()` has already been + * called) or it is going to be removed due to timeout (that is, `isTimingOut()` is `true`). + * + * @throws java.lang.IllegalArgumentException If the state has already been removed, or is + * going to be removed + */ + def update(newState: S): Unit + + /** + * Remove the state if it exists. + * + * State cannot be updated if it has been already removed (that is, `remove()` has already been + * called) or it is going to be removed due to timeout (that is, `isTimingOut()` is `true`). + */ + def remove(): Unit + + /** + * Whether the state is timing out and going to be removed by the system after the current batch. + * This timeout can occur if timeout duration has been specified in the + * [[org.apache.spark.streaming.StateSpec StatSpec]] and the key has not received any new data + * for that timeout duration. + */ + def isTimingOut(): Boolean + + /** + * Get the state as a `scala.Option`. It will be `Some(state)` if it exists, otherwise `None`. + */ + @inline final def getOption(): Option[S] = if (exists) Some(get()) else None + + @inline final override def toString(): String = { + getOption.map { _.toString }.getOrElse("") + } +} + +/** Internal implementation of the [[State]] interface */ +private[streaming] class StateImpl[S] extends State[S] { + + private var state: S = null.asInstanceOf[S] + private var defined: Boolean = false + private var timingOut: Boolean = false + private var updated: Boolean = false + private var removed: Boolean = false + + // ========= Public API ========= + override def exists(): Boolean = { + defined + } + + override def get(): S = { + if (defined) { + state + } else { + throw new NoSuchElementException("State is not set") + } + } + + override def update(newState: S): Unit = { + require(!removed, "Cannot update the state after it has been removed") + require(!timingOut, "Cannot update the state that is timing out") + state = newState + defined = true + updated = true + } + + override def isTimingOut(): Boolean = { + timingOut + } + + override def remove(): Unit = { + require(!timingOut, "Cannot remove the state that is timing out") + require(!removed, "Cannot remove the state that has already been removed") + defined = false + updated = false + removed = true + } + + // ========= Internal API ========= + + /** Whether the state has been marked for removing */ + def isRemoved(): Boolean = { + removed + } + + /** Whether the state has been updated */ + def isUpdated(): Boolean = { + updated + } + + /** + * Update the internal data and flags in `this` to the given state option. + * This method allows `this` object to be reused across many state records. + */ + def wrap(optionalState: Option[S]): Unit = { + optionalState match { + case Some(newState) => + this.state = newState + defined = true + + case None => + this.state = null.asInstanceOf[S] + defined = false + } + timingOut = false + removed = false + updated = false + } + + /** + * Update the internal data and flags in `this` to the given state that is going to be timed out. + * This method allows `this` object to be reused across many state records. + */ + def wrapTimingOutState(newState: S): Unit = { + this.state = newState + defined = true + timingOut = true + removed = false + updated = false + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StateSpec.scala b/streaming/src/main/scala/org/apache/spark/streaming/StateSpec.scala new file mode 100644 index 000000000000..dcd698c860d8 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/StateSpec.scala @@ -0,0 +1,279 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +import org.apache.spark.{HashPartitioner, Partitioner} +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.{JavaPairRDD, JavaUtils, Optional} +import org.apache.spark.api.java.function.{Function3 => JFunction3, Function4 => JFunction4} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.ClosureCleaner + +/** + * :: Experimental :: + * Abstract class representing all the specifications of the DStream transformation + * `mapWithState` operation of a + * [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]] (Scala) or a + * [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream]] (Java). + * Use `org.apache.spark.streaming.StateSpec.function()` factory methods + * to create instances of this class. + * + * Example in Scala: + * {{{ + * // A mapping function that maintains an integer state and return a String + * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = { + * // Use state.exists(), state.get(), state.update() and state.remove() + * // to manage state, and return the necessary string + * } + * + * val spec = StateSpec.function(mappingFunction).numPartitions(10) + * + * val mapWithStateDStream = keyValueDStream.mapWithState[StateType, MappedType](spec) + * }}} + * + * Example in Java: + * {{{ + * // A mapping function that maintains an integer state and return a string + * Function3, State, String> mappingFunction = + * new Function3, State, String>() { + * @Override + * public Optional call(Optional value, State state) { + * // Use state.exists(), state.get(), state.update() and state.remove() + * // to manage state, and return the necessary string + * } + * }; + * + * JavaMapWithStateDStream mapWithStateDStream = + * keyValueDStream.mapWithState(StateSpec.function(mappingFunc)); + * }}} + * + * @tparam KeyType Class of the state key + * @tparam ValueType Class of the state value + * @tparam StateType Class of the state data + * @tparam MappedType Class of the mapped elements + */ +@Experimental +sealed abstract class StateSpec[KeyType, ValueType, StateType, MappedType] extends Serializable { + + /** + * Set the RDD containing the initial states that will be used by `mapWithState` + */ + def initialState(rdd: RDD[(KeyType, StateType)]): this.type + + /** + * Set the RDD containing the initial states that will be used by `mapWithState` + */ + def initialState(javaPairRDD: JavaPairRDD[KeyType, StateType]): this.type + + /** + * Set the number of partitions by which the state RDDs generated by `mapWithState` + * will be partitioned. Hash partitioning will be used. + */ + def numPartitions(numPartitions: Int): this.type + + /** + * Set the partitioner by which the state RDDs generated by `mapWithState` will be partitioned. + */ + def partitioner(partitioner: Partitioner): this.type + + /** + * Set the duration after which the state of an idle key will be removed. A key and its state is + * considered idle if it has not received any data for at least the given duration. The + * mapping function will be called one final time on the idle states that are going to be + * removed; [[org.apache.spark.streaming.State State.isTimingOut()]] set + * to `true` in that call. + */ + def timeout(idleDuration: Duration): this.type +} + + +/** + * :: Experimental :: + * Builder object for creating instances of `org.apache.spark.streaming.StateSpec` + * that is used for specifying the parameters of the DStream transformation `mapWithState` + * that is used for specifying the parameters of the DStream transformation + * `mapWithState` operation of a + * [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]] (Scala) or a + * [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream]] (Java). + * + * Example in Scala: + * {{{ + * // A mapping function that maintains an integer state and return a String + * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = { + * // Use state.exists(), state.get(), state.update() and state.remove() + * // to manage state, and return the necessary string + * } + * + * val spec = StateSpec.function(mappingFunction).numPartitions(10) + * + * val mapWithStateDStream = keyValueDStream.mapWithState[StateType, MappedType](spec) + * }}} + * + * Example in Java: + * {{{ + * // A mapping function that maintains an integer state and return a string + * Function3, State, String> mappingFunction = + * new Function3, State, String>() { + * @Override + * public Optional call(Optional value, State state) { + * // Use state.exists(), state.get(), state.update() and state.remove() + * // to manage state, and return the necessary string + * } + * }; + * + * JavaMapWithStateDStream mapWithStateDStream = + * keyValueDStream.mapWithState(StateSpec.function(mappingFunc)); + *}}} + */ +@Experimental +object StateSpec { + /** + * Create a [[org.apache.spark.streaming.StateSpec StateSpec]] for setting all the specifications + * of the `mapWithState` operation on a + * [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]]. + * + * @param mappingFunction The function applied on every data item to manage the associated state + * and generate the mapped data + * @tparam KeyType Class of the keys + * @tparam ValueType Class of the values + * @tparam StateType Class of the states data + * @tparam MappedType Class of the mapped data + */ + def function[KeyType, ValueType, StateType, MappedType]( + mappingFunction: (Time, KeyType, Option[ValueType], State[StateType]) => Option[MappedType] + ): StateSpec[KeyType, ValueType, StateType, MappedType] = { + ClosureCleaner.clean(mappingFunction, checkSerializable = true) + new StateSpecImpl(mappingFunction) + } + + /** + * Create a [[org.apache.spark.streaming.StateSpec StateSpec]] for setting all the specifications + * of the `mapWithState` operation on a + * [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]]. + * + * @param mappingFunction The function applied on every data item to manage the associated state + * and generate the mapped data + * @tparam ValueType Class of the values + * @tparam StateType Class of the states data + * @tparam MappedType Class of the mapped data + */ + def function[KeyType, ValueType, StateType, MappedType]( + mappingFunction: (KeyType, Option[ValueType], State[StateType]) => MappedType + ): StateSpec[KeyType, ValueType, StateType, MappedType] = { + ClosureCleaner.clean(mappingFunction, checkSerializable = true) + val wrappedFunction = + (time: Time, key: KeyType, value: Option[ValueType], state: State[StateType]) => { + Some(mappingFunction(key, value, state)) + } + new StateSpecImpl(wrappedFunction) + } + + /** + * Create a [[org.apache.spark.streaming.StateSpec StateSpec]] for setting all + * the specifications of the `mapWithState` operation on a + * [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream]]. + * + * @param mappingFunction The function applied on every data item to manage the associated + * state and generate the mapped data + * @tparam KeyType Class of the keys + * @tparam ValueType Class of the values + * @tparam StateType Class of the states data + * @tparam MappedType Class of the mapped data + */ + def function[KeyType, ValueType, StateType, MappedType](mappingFunction: + JFunction4[Time, KeyType, Optional[ValueType], State[StateType], Optional[MappedType]]): + StateSpec[KeyType, ValueType, StateType, MappedType] = { + val wrappedFunc = (time: Time, k: KeyType, v: Option[ValueType], s: State[StateType]) => { + val t = mappingFunction.call(time, k, JavaUtils.optionToOptional(v), s) + if (t.isPresent) { + Some(t.get) + } else { + None + } + } + StateSpec.function(wrappedFunc) + } + + /** + * Create a [[org.apache.spark.streaming.StateSpec StateSpec]] for setting all the specifications + * of the `mapWithState` operation on a + * [[org.apache.spark.streaming.api.java.JavaPairDStream JavaPairDStream]]. + * + * @param mappingFunction The function applied on every data item to manage the associated + * state and generate the mapped data + * @tparam ValueType Class of the values + * @tparam StateType Class of the states data + * @tparam MappedType Class of the mapped data + */ + def function[KeyType, ValueType, StateType, MappedType]( + mappingFunction: JFunction3[KeyType, Optional[ValueType], State[StateType], MappedType]): + StateSpec[KeyType, ValueType, StateType, MappedType] = { + val wrappedFunc = (k: KeyType, v: Option[ValueType], s: State[StateType]) => { + mappingFunction.call(k, JavaUtils.optionToOptional(v), s) + } + StateSpec.function(wrappedFunc) + } +} + + +/** Internal implementation of [[org.apache.spark.streaming.StateSpec]] interface. */ +private[streaming] +case class StateSpecImpl[K, V, S, T]( + function: (Time, K, Option[V], State[S]) => Option[T]) extends StateSpec[K, V, S, T] { + + require(function != null) + + @volatile private var partitioner: Partitioner = null + @volatile private var initialStateRDD: RDD[(K, S)] = null + @volatile private var timeoutInterval: Duration = null + + override def initialState(rdd: RDD[(K, S)]): this.type = { + this.initialStateRDD = rdd + this + } + + override def initialState(javaPairRDD: JavaPairRDD[K, S]): this.type = { + this.initialStateRDD = javaPairRDD.rdd + this + } + + override def numPartitions(numPartitions: Int): this.type = { + this.partitioner(new HashPartitioner(numPartitions)) + this + } + + override def partitioner(partitioner: Partitioner): this.type = { + this.partitioner = partitioner + this + } + + override def timeout(interval: Duration): this.type = { + this.timeoutInterval = interval + this + } + + // ================= Private Methods ================= + + private[streaming] def getFunction(): (Time, K, Option[V], State[S]) => Option[T] = function + + private[streaming] def getInitialStateRDD(): Option[RDD[(K, S)]] = Option(initialStateRDD) + + private[streaming] def getPartitioner(): Option[Partitioner] = Option(partitioner) + + private[streaming] def getTimeoutInterval(): Option[Duration] = Option(timeoutInterval) +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 97113835f3bd..027403816f53 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming import java.io.{InputStream, NotSerializableException} +import java.util.Properties import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import scala.collection.Map @@ -25,24 +26,27 @@ import scala.collection.mutable.Queue import scala.reflect.ClassTag import scala.util.control.NonFatal -import akka.actor.{Props, SupervisorStrategy} +import org.apache.commons.lang3.SerializationUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{BytesWritable, LongWritable, Text} -import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.spark._ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.input.FixedLengthBinaryInputFormat +import org.apache.spark.internal.Logging import org.apache.spark.rdd.{RDD, RDDOperationScope} +import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.SerializationDebugger import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContextState._ import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receiver.{ActorReceiver, ActorSupervisorStrategy, Receiver} -import org.apache.spark.streaming.scheduler.{JobScheduler, StreamingListener} +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.scheduler. + {ExecutorAllocationManager, JobScheduler, StreamingListener, StreamingListenerStreamingStarted} import org.apache.spark.streaming.ui.{StreamingJobProgressListener, StreamingTab} import org.apache.spark.util.{CallSite, ShutdownHookManager, ThreadUtils, Utils} @@ -58,9 +62,9 @@ import org.apache.spark.util.{CallSite, ShutdownHookManager, ThreadUtils, Utils} * of the context by `stop()` or by an exception. */ class StreamingContext private[streaming] ( - sc_ : SparkContext, - cp_ : Checkpoint, - batchDur_ : Duration + _sc: SparkContext, + _cp: Checkpoint, + _batchDur: Duration ) extends Logging { /** @@ -105,7 +109,7 @@ class StreamingContext private[streaming] ( * HDFS compatible filesystems */ def this(path: String, hadoopConf: Configuration) = - this(null, CheckpointReader.read(path, new SparkConf(), hadoopConf).get, null) + this(null, CheckpointReader.read(path, new SparkConf(), hadoopConf).orNull, null) /** * Recreate a StreamingContext from a checkpoint file. @@ -121,23 +125,20 @@ class StreamingContext private[streaming] ( def this(path: String, sparkContext: SparkContext) = { this( sparkContext, - CheckpointReader.read(path, sparkContext.conf, sparkContext.hadoopConfiguration).get, + CheckpointReader.read(path, sparkContext.conf, sparkContext.hadoopConfiguration).orNull, null) } + require(_sc != null || _cp != null, + "Spark Streaming cannot be initialized with both SparkContext and checkpoint as null") - if (sc_ == null && cp_ == null) { - throw new Exception("Spark Streaming cannot be initialized with " + - "both SparkContext and checkpoint as null") - } - - private[streaming] val isCheckpointPresent = (cp_ != null) + private[streaming] val isCheckpointPresent: Boolean = _cp != null private[streaming] val sc: SparkContext = { - if (sc_ != null) { - sc_ + if (_sc != null) { + _sc } else if (isCheckpointPresent) { - SparkContext.getOrCreate(cp_.createSparkConf()) + SparkContext.getOrCreate(_cp.createSparkConf()) } else { throw new SparkException("Cannot create StreamingContext without a SparkContext") } @@ -154,13 +155,13 @@ class StreamingContext private[streaming] ( private[streaming] val graph: DStreamGraph = { if (isCheckpointPresent) { - cp_.graph.setContext(this) - cp_.graph.restoreCheckpointData() - cp_.graph + _cp.graph.setContext(this) + _cp.graph.restoreCheckpointData() + _cp.graph } else { - require(batchDur_ != null, "Batch duration for StreamingContext cannot be null") + require(_batchDur != null, "Batch duration for StreamingContext cannot be null") val newGraph = new DStreamGraph() - newGraph.setBatchDuration(batchDur_) + newGraph.setBatchDuration(_batchDur) newGraph } } @@ -169,15 +170,15 @@ class StreamingContext private[streaming] ( private[streaming] var checkpointDir: String = { if (isCheckpointPresent) { - sc.setCheckpointDir(cp_.checkpointDir) - cp_.checkpointDir + sc.setCheckpointDir(_cp.checkpointDir) + _cp.checkpointDir } else { null } } private[streaming] val checkpointDuration: Duration = { - if (isCheckpointPresent) cp_.checkpointDuration else graph.batchDuration + if (isCheckpointPresent) _cp.checkpointDuration else graph.batchDuration } private[streaming] val scheduler = new JobScheduler(this) @@ -200,6 +201,10 @@ class StreamingContext private[streaming] ( private val startSite = new AtomicReference[CallSite](null) + // Copy of thread-local properties from SparkContext. These properties will be set in all tasks + // submitted by this StreamingContext after start. + private[streaming] val savedProperties = new AtomicReference[Properties](new Properties) + private[streaming] def getStartSite(): CallSite = startSite.get() private var shutdownHookRef: AnyRef = _ @@ -212,8 +217,8 @@ class StreamingContext private[streaming] ( def sparkContext: SparkContext = sc /** - * Set each DStreams in this context to remember RDDs it generated in the last given duration. - * DStreams remember RDDs only for a limited duration of time and releases them for garbage + * Set each DStream in this context to remember RDDs it generated in the last given duration. + * DStreams remember RDDs only for a limited duration of time and release them for garbage * collection. This method allows the developer to specify how long to remember the RDDs ( * if the developer wishes to query old data outside the DStream computation). * @param duration Minimum duration that each DStream should remember its RDDs @@ -226,7 +231,7 @@ class StreamingContext private[streaming] ( * Set the context to periodically checkpoint the DStream operations for driver * fault-tolerance. * @param directory HDFS-compatible directory where the checkpoint data will be reliably stored. - * Note that this must be a fault-tolerant file system like HDFS for + * Note that this must be a fault-tolerant file system like HDFS. */ def checkpoint(directory: String) { if (directory != null) { @@ -246,7 +251,7 @@ class StreamingContext private[streaming] ( } private[streaming] def initialCheckpoint: Checkpoint = { - if (isCheckpointPresent) cp_ else null + if (isCheckpointPresent) _cp else null } private[streaming] def getNewInputStreamId() = nextInputStreamId.getAndIncrement() @@ -271,21 +276,7 @@ class StreamingContext private[streaming] ( /** * Create an input stream with any arbitrary user implemented receiver. - * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html - * @param receiver Custom implementation of Receiver - * - * @deprecated As of 1.0.0", replaced by `receiverStream`. - */ - @deprecated("Use receiverStream", "1.0.0") - def networkStream[T: ClassTag](receiver: Receiver[T]): ReceiverInputDStream[T] = { - withNamedScope("network stream") { - receiverStream(receiver) - } - } - - /** - * Create an input stream with any arbitrary user implemented receiver. - * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html + * Find more details at http://spark.apache.org/docs/latest/streaming-custom-receivers.html * @param receiver Custom implementation of Receiver */ def receiverStream[T: ClassTag](receiver: Receiver[T]): ReceiverInputDStream[T] = { @@ -295,34 +286,14 @@ class StreamingContext private[streaming] ( } /** - * Create an input stream with any arbitrary user implemented actor receiver. - * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html - * @param props Props object defining creation of the actor - * @param name Name of the actor - * @param storageLevel RDD storage level (default: StorageLevel.MEMORY_AND_DISK_SER_2) - * - * @note An important point to note: - * Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e parametrized type of data received and actorStream - * should be same. - */ - def actorStream[T: ClassTag]( - props: Props, - name: String, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, - supervisorStrategy: SupervisorStrategy = ActorSupervisorStrategy.defaultStrategy - ): ReceiverInputDStream[T] = withNamedScope("actor stream") { - receiverStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) - } - - /** - * Create a input stream from TCP source hostname:port. Data is received using + * Creates an input stream from TCP source hostname:port. Data is received using * a TCP socket and the receive bytes is interpreted as UTF8 encoded `\n` delimited * lines. * @param hostname Hostname to connect to for receiving data * @param port Port to connect to for receiving data * @param storageLevel Storage level to use for storing the received objects * (default: StorageLevel.MEMORY_AND_DISK_SER_2) + * @see [[socketStream]] */ def socketTextStream( hostname: String, @@ -333,8 +304,8 @@ class StreamingContext private[streaming] ( } /** - * Create a input stream from TCP source hostname:port. Data is received using - * a TCP socket and the receive bytes it interepreted as object using the given + * Creates an input stream from TCP source hostname:port. Data is received using + * a TCP socket and the receive bytes it interpreted as object using the given * converter. * @param hostname Hostname to connect to for receiving data * @param port Port to connect to for receiving data @@ -352,7 +323,7 @@ class StreamingContext private[streaming] ( } /** - * Create a input stream from network source hostname:port, where data is received + * Create an input stream from network source hostname:port, where data is received * as serialized blocks (serialized using the Spark's serializer) that can be directly * pushed into the block manager without deserializing them. This is the most efficient * way to receive data. @@ -371,7 +342,7 @@ class StreamingContext private[streaming] ( } /** - * Create a input stream that monitors a Hadoop-compatible filesystem + * Create an input stream that monitors a Hadoop-compatible filesystem * for new files and reads them using the given key-value types and input format. * Files must be written to the monitored directory by "moving" them from another * location within the same file system. File names starting with . are ignored. @@ -389,7 +360,7 @@ class StreamingContext private[streaming] ( } /** - * Create a input stream that monitors a Hadoop-compatible filesystem + * Create an input stream that monitors a Hadoop-compatible filesystem * for new files and reads them using the given key-value types and input format. * Files must be written to the monitored directory by "moving" them from another * location within the same file system. @@ -409,7 +380,7 @@ class StreamingContext private[streaming] ( } /** - * Create a input stream that monitors a Hadoop-compatible filesystem + * Create an input stream that monitors a Hadoop-compatible filesystem * for new files and reads them using the given key-value types and input format. * Files must be written to the monitored directory by "moving" them from another * location within the same file system. File names starting with . are ignored. @@ -433,7 +404,7 @@ class StreamingContext private[streaming] ( } /** - * Create a input stream that monitors a Hadoop-compatible filesystem + * Create an input stream that monitors a Hadoop-compatible filesystem * for new files and reads them as text files (using key as LongWritable, value * as Text and input format as TextInputFormat). Files must be written to the * monitored directory by "moving" them from another location within the same @@ -451,58 +422,57 @@ class StreamingContext private[streaming] ( * by "moving" them from another location within the same file system. File names * starting with . are ignored. * - * '''Note:''' We ensure that the byte array for each record in the - * resulting RDDs of the DStream has the provided record length. - * * @param directory HDFS directory to monitor for new file * @param recordLength length of each record in bytes + * + * @note We ensure that the byte array for each record in the + * resulting RDDs of the DStream has the provided record length. */ def binaryRecordsStream( directory: String, recordLength: Int): DStream[Array[Byte]] = withNamedScope("binary records stream") { - val conf = sc_.hadoopConfiguration + val conf = _sc.hadoopConfiguration conf.setInt(FixedLengthBinaryInputFormat.RECORD_LENGTH_PROPERTY, recordLength) val br = fileStream[LongWritable, BytesWritable, FixedLengthBinaryInputFormat]( directory, FileInputDStream.defaultFilter: Path => Boolean, newFilesOnly = true, conf) - val data = br.map { case (k, v) => - val bytes = v.getBytes + br.map { case (k, v) => + val bytes = v.copyBytes() require(bytes.length == recordLength, "Byte array does not have correct length. " + s"${bytes.length} did not equal recordLength: $recordLength") bytes } - data } /** * Create an input stream from a queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * - * NOTE: Arbitrary RDDs can be added to `queueStream`, there is no way to recover data of - * those RDDs, so `queueStream` doesn't support checkpointing. - * - * @param queue Queue of RDDs + * @param queue Queue of RDDs. Modifications to this data structure must be synchronized. * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval * @tparam T Type of objects in the RDD + * + * @note Arbitrary RDDs can be added to `queueStream`, there is no way to recover data of + * those RDDs, so `queueStream` doesn't support checkpointing. */ def queueStream[T: ClassTag]( queue: Queue[RDD[T]], oneAtATime: Boolean = true ): InputDStream[T] = { - queueStream(queue, oneAtATime, sc.makeRDD(Seq[T](), 1)) + queueStream(queue, oneAtATime, sc.makeRDD(Seq.empty[T], 1)) } /** * Create an input stream from a queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * - * NOTE: Arbitrary RDDs can be added to `queueStream`, there is no way to recover data of - * those RDDs, so `queueStream` doesn't support checkpointing. - * - * @param queue Queue of RDDs + * @param queue Queue of RDDs. Modifications to this data structure must be synchronized. * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval * @param defaultRDD Default RDD is returned by the DStream when the queue is empty. * Set as null if no RDD should be returned when empty * @tparam T Type of objects in the RDD + * + * @note Arbitrary RDDs can be added to `queueStream`, there is no way to recover data of + * those RDDs, so `queueStream` doesn't support checkpointing. */ def queueStream[T: ClassTag]( queue: Queue[RDD[T]], @@ -530,9 +500,10 @@ class StreamingContext private[streaming] ( new TransformedDStream[T](dstreams, sparkContext.clean(transformFunc)) } - /** Add a [[org.apache.spark.streaming.scheduler.StreamingListener]] object for - * receiving system events related to streaming. - */ + /** + * Add a [[org.apache.spark.streaming.scheduler.StreamingListener]] object for + * receiving system events related to streaming. + */ def addStreamingListener(streamingListener: StreamingListener) { scheduler.listenerBus.addListener(streamingListener) } @@ -549,7 +520,7 @@ class StreamingContext private[streaming] ( // Verify whether the DStream checkpoint is serializable if (isCheckpointingEnabled) { - val checkpoint = new Checkpoint(this, Time.apply(0)) + val checkpoint = new Checkpoint(this, Time(0)) try { Checkpoint.serialize(checkpoint, conf) } catch { @@ -562,11 +533,12 @@ class StreamingContext private[streaming] ( } } - if (Utils.isDynamicAllocationEnabled(sc.conf)) { + if (Utils.isDynamicAllocationEnabled(sc.conf) || + ExecutorAllocationManager.isDynamicAllocationEnabled(conf)) { logWarning("Dynamic Allocation is enabled for this application. " + "Enabling Dynamic allocation for Spark Streaming applications can cause data loss if " + "Write Ahead Log is not enabled for non-replayable sources like Flume. " + - "See the programming guide for details on how to enable the Write Ahead Log") + "See the programming guide for details on how to enable the Write Ahead Log.") } } @@ -574,11 +546,12 @@ class StreamingContext private[streaming] ( * :: DeveloperApi :: * * Return the current state of the context. The context can be in three possible states - - * - StreamingContextState.INTIALIZED - The context has been created, but not been started yet. - * Input DStreams, transformations and output operations can be created on the context. - * - StreamingContextState.ACTIVE - The context has been started, and been not stopped. - * Input DStreams, transformations and output operations cannot be created on the context. - * - StreamingContextState.STOPPED - The context has been stopped and cannot be used any more. + * + * - StreamingContextState.INITIALIZED - The context has been created, but not started yet. + * Input DStreams, transformations and output operations can be created on the context. + * - StreamingContextState.ACTIVE - The context has been started, and not stopped. + * Input DStreams, transformations and output operations cannot be created on the context. + * - StreamingContextState.STOPPED - The context has been stopped and cannot be used any more. */ @DeveloperApi def getState(): StreamingContextState = synchronized { @@ -606,9 +579,12 @@ class StreamingContext private[streaming] ( sparkContext.setCallSite(startSite.get) sparkContext.clearJobGroup() sparkContext.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "false") + savedProperties.set(SerializationUtils.clone(sparkContext.localProperties.get())) scheduler.start() } state = StreamingContextState.ACTIVE + scheduler.listenerBus.post( + StreamingListenerStreamingStarted(System.currentTimeMillis())) } catch { case NonFatal(e) => logError("Error starting the context, marking it as stopped", e) @@ -618,8 +594,9 @@ class StreamingContext private[streaming] ( } StreamingContext.setActiveContext(this) } + logDebug("Adding shutdown hook") // force eager creation of logger shutdownHookRef = ShutdownHookManager.addShutdownHook( - StreamingContext.SHUTDOWN_HOOK_PRIORITY)(stopOnShutdown) + StreamingContext.SHUTDOWN_HOOK_PRIORITY)(() => stopOnShutdown()) // Registering Streaming Metrics at the start of the StreamingContext assert(env.metricsSystem != null) env.metricsSystem.registerSource(streamingSource) @@ -641,18 +618,6 @@ class StreamingContext private[streaming] ( waiter.waitForStopOrError() } - /** - * Wait for the execution to stop. Any exceptions that occurs during the execution - * will be thrown in this thread. - * @param timeout time to wait in milliseconds - * - * @deprecated As of 1.3.0, replaced by `awaitTerminationOrTimeout(Long)`. - */ - @deprecated("Use awaitTerminationOrTimeout(Long) instead", "1.3.0") - def awaitTermination(timeout: Long) { - waiter.waitForStopOrError(timeout) - } - /** * Wait for the execution to stop. Any exceptions that occurs during the execution * will be thrown in this thread. @@ -693,29 +658,45 @@ class StreamingContext private[streaming] ( */ def stop(stopSparkContext: Boolean, stopGracefully: Boolean): Unit = { var shutdownHookRefToRemove: AnyRef = null + if (LiveListenerBus.withinListenerThread.value) { + throw new SparkException(s"Cannot stop StreamingContext within listener bus thread.") + } synchronized { - try { - state match { - case INITIALIZED => - logWarning("StreamingContext has not been started yet") - case STOPPED => - logWarning("StreamingContext has already been stopped") - case ACTIVE => + // The state should always be Stopped after calling `stop()`, even if we haven't started yet + state match { + case INITIALIZED => + logWarning("StreamingContext has not been started yet") + state = STOPPED + case STOPPED => + logWarning("StreamingContext has already been stopped") + state = STOPPED + case ACTIVE => + // It's important that we don't set state = STOPPED until the very end of this case, + // since we need to ensure that we're still able to call `stop()` to recover from + // a partially-stopped StreamingContext which resulted from this `stop()` call being + // interrupted. See SPARK-12001 for more details. Because the body of this case can be + // executed twice in the case of a partial stop, all methods called here need to be + // idempotent. + Utils.tryLogNonFatalError { scheduler.stop(stopGracefully) - // Removing the streamingSource to de-register the metrics on stop() + } + // Removing the streamingSource to de-register the metrics on stop() + Utils.tryLogNonFatalError { env.metricsSystem.removeSource(streamingSource) + } + Utils.tryLogNonFatalError { uiTab.foreach(_.detach()) - StreamingContext.setActiveContext(null) + } + StreamingContext.setActiveContext(null) + Utils.tryLogNonFatalError { waiter.notifyStop() - if (shutdownHookRef != null) { - shutdownHookRefToRemove = shutdownHookRef - shutdownHookRef = null - } - logInfo("StreamingContext stopped successfully") - } - } finally { - // The state should always be Stopped after calling `stop()`, even if we haven't started yet - state = STOPPED + } + if (shutdownHookRef != null) { + shutdownHookRefToRemove = shutdownHookRef + shutdownHookRef = null + } + logInfo("StreamingContext stopped successfully") + state = STOPPED } } if (shutdownHookRefToRemove != null) { @@ -780,18 +761,6 @@ object StreamingContext extends Logging { } } - /** - * @deprecated As of 1.3.0, replaced by implicit functions in the DStream companion object. - * This is kept here only for backward compatibility. - */ - @deprecated("Replaced by implicit functions in the DStream companion object. This is " + - "kept here only for backward compatibility.", "1.3.0") - def toPairDStreamFunctions[K, V](stream: DStream[(K, V)]) - (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) - : PairDStreamFunctions[K, V] = { - DStream.toPairDStreamFunctions(stream)(kt, vt, ord) - } - /** * :: Experimental :: * @@ -882,12 +851,25 @@ object StreamingContext extends Logging { } private[streaming] def rddToFileName[T](prefix: String, suffix: String, time: Time): String = { - if (prefix == null) { - time.milliseconds.toString - } else if (suffix == null || suffix.length ==0) { - prefix + "-" + time.milliseconds - } else { - prefix + "-" + time.milliseconds + "." + suffix + var result = time.milliseconds.toString + if (prefix != null && prefix.length > 0) { + result = s"$prefix-$result" } + if (suffix != null && suffix.length > 0) { + result = s"$result.$suffix" + } + result + } +} + +private class StreamingContextPythonHelper { + + /** + * This is a private method only for Python to implement `getOrCreate`. + */ + def tryRecoverFromCheckpoint(checkpointPath: String): Option[StreamingContext] = { + val checkpointOption = CheckpointReader.read( + checkpointPath, new SparkConf(), SparkHadoopUtil.get.conf, ignoreReadError = false) + checkpointOption.map(new StreamingContext(null, _, null)) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala index 9697437dd2fe..0b306a28d1a5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala @@ -87,11 +87,11 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { // Gauge for last received batch, useful for monitoring the streaming job's running status, // displayed data -1 for any abnormal condition. registerGaugeWithOption("lastReceivedBatch_submissionTime", - _.lastCompletedBatch.map(_.submissionTime), -1L) + _.lastReceivedBatch.map(_.submissionTime), -1L) registerGaugeWithOption("lastReceivedBatch_processingStartTime", - _.lastCompletedBatch.flatMap(_.processingStartTime), -1L) + _.lastReceivedBatch.flatMap(_.processingStartTime), -1L) registerGaugeWithOption("lastReceivedBatch_processingEndTime", - _.lastCompletedBatch.flatMap(_.processingEndTime), -1L) + _.lastReceivedBatch.flatMap(_.processingEndTime), -1L) // Gauge for last received batch records. registerGauge("lastReceivedBatch_records", _.lastReceivedBatchRecords.values.sum, 0L) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index 01cdcb057404..a59f4efccb57 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -17,14 +17,14 @@ package org.apache.spark.streaming.api.java -import org.apache.spark.streaming.{Duration, Time} -import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.api.java.JavaRDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.rdd.RDD - import scala.language.implicitConversions import scala.reflect.ClassTag + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Duration, Time} import org.apache.spark.streaming.dstream.DStream /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index edfa474677f1..4a0ec31b5f3c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.api.java -import java.lang.{Long => JLong} +import java.{lang => jl} import java.util.{List => JList} import scala.collection.JavaConverters._ @@ -27,7 +27,7 @@ import scala.reflect.ClassTag import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaRDDLike} import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag -import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, Function3 => JFunction3, _} +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, Function3 => JFunction3, VoidFunction => JVoidFunction, VoidFunction2 => JVoidFunction2, _} import org.apache.spark.rdd.RDD import org.apache.spark.streaming._ import org.apache.spark.streaming.api.java.JavaDStream._ @@ -50,8 +50,10 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T def wrapRDD(in: RDD[T]): R - implicit def scalaIntToJavaLong(in: DStream[Long]): JavaDStream[JLong] = { - in.map(new JLong(_)) + // This is just unfortunate we made a mistake in naming -- should be scalaLongToJavaLong. + // Don't fix this for now as it would break binary compatibility. + implicit def scalaIntToJavaLong(in: DStream[Long]): JavaDStream[jl.Long] = { + in.map(jl.Long.valueOf) } /** @@ -74,14 +76,14 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * Return a new DStream in which each RDD has a single element generated by counting each RDD * of this DStream. */ - def count(): JavaDStream[JLong] = dstream.count() + def count(): JavaDStream[jl.Long] = dstream.count() /** * Return a new DStream in which each RDD contains the counts of each distinct value in * each RDD of this DStream. Hash partitioning is used to generate the RDDs with * Spark's default number of partitions. */ - def countByValue(): JavaPairDStream[T, JLong] = { + def countByValue(): JavaPairDStream[T, jl.Long] = { JavaPairDStream.scalaToJavaLong(dstream.countByValue()) } @@ -91,7 +93,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * partitions. * @param numPartitions number of partitions of each RDD in the new DStream. */ - def countByValue(numPartitions: Int): JavaPairDStream[T, JLong] = { + def countByValue(numPartitions: Int): JavaPairDStream[T, jl.Long] = { JavaPairDStream.scalaToJavaLong(dstream.countByValue(numPartitions)) } @@ -101,7 +103,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * of elements in a window over this DStream. windowDuration and slideDuration are as defined in * the window() operation. This is equivalent to window(windowDuration, slideDuration).count() */ - def countByWindow(windowDuration: Duration, slideDuration: Duration) : JavaDStream[JLong] = { + def countByWindow(windowDuration: Duration, slideDuration: Duration): JavaDStream[jl.Long] = { dstream.countByWindow(windowDuration, slideDuration) } @@ -116,7 +118,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * DStream's batching interval */ def countByValueAndWindow(windowDuration: Duration, slideDuration: Duration) - : JavaPairDStream[T, JLong] = { + : JavaPairDStream[T, jl.Long] = { JavaPairDStream.scalaToJavaLong( dstream.countByValueAndWindow(windowDuration, slideDuration)) } @@ -133,7 +135,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * @param numPartitions number of partitions of each RDD in the new DStream. */ def countByValueAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int) - : JavaPairDStream[T, JLong] = { + : JavaPairDStream[T, jl.Long] = { JavaPairDStream.scalaToJavaLong( dstream.countByValueAndWindow(windowDuration, slideDuration, numPartitions)) } @@ -151,7 +153,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T def context(): StreamingContext = dstream.context /** Return a new DStream by applying a function to all elements of this DStream. */ - def map[R](f: JFunction[T, R]): JavaDStream[R] = { + def map[U](f: JFunction[T, U]): JavaDStream[U] = { new JavaDStream(dstream.map(f)(fakeClassTag))(fakeClassTag) } @@ -166,8 +168,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * and then flattening the results */ def flatMap[U](f: FlatMapFunction[T, U]): JavaDStream[U] = { - import scala.collection.JavaConverters._ - def fn: (T) => Iterable[U] = (x: T) => f.call(x).asScala + def fn: (T) => Iterator[U] = (x: T) => f.call(x).asScala new JavaDStream(dstream.flatMap(fn)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -176,8 +177,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * and then flattening the results */ def flatMapToPair[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairDStream[K2, V2] = { - import scala.collection.JavaConverters._ - def fn: (T) => Iterable[(K2, V2)] = (x: T) => f.call(x).asScala + def fn: (T) => Iterator[(K2, V2)] = (x: T) => f.call(x).asScala def cm: ClassTag[(K2, V2)] = fakeClassTag new JavaPairDStream(dstream.flatMap(fn)(cm))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -189,7 +189,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T */ def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U]): JavaDStream[U] = { def fn: (Iterator[T]) => Iterator[U] = { - (x: Iterator[T]) => f.call(x.asJava).iterator().asScala + (x: Iterator[T]) => f.call(x.asJava).asScala } new JavaDStream(dstream.mapPartitions(fn)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -202,7 +202,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]) : JavaPairDStream[K2, V2] = { def fn: (Iterator[T]) => Iterator[(K2, V2)] = { - (x: Iterator[T]) => f.call(x.asJava).iterator().asScala + (x: Iterator[T]) => f.call(x.asJava).asScala } new JavaPairDStream(dstream.mapPartitions(fn))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -216,27 +216,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T /** * Return a new DStream in which each RDD has a single element generated by reducing all * elements in a sliding window over this DStream. - * @param reduceFunc associative reduce function - * @param windowDuration width of the window; must be a multiple of this DStream's - * batching interval - * @param slideDuration sliding interval of the window (i.e., the interval after which - * the new DStream will generate RDDs); must be a multiple of this - * DStream's batching interval - * @deprecated As this API is not Java compatible. - */ - @deprecated("Use Java-compatible version of reduceByWindow", "1.3.0") - def reduceByWindow( - reduceFunc: (T, T) => T, - windowDuration: Duration, - slideDuration: Duration - ): DStream[T] = { - dstream.reduceByWindow(reduceFunc, windowDuration, slideDuration) - } - - /** - * Return a new DStream in which each RDD has a single element generated by reducing all - * elements in a sliding window over this DStream. - * @param reduceFunc associative reduce function + * @param reduceFunc associative and commutative reduce function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which @@ -259,8 +239,9 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) * This is more efficient than reduceByWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". - * @param reduceFunc associative reduce function - * @param invReduceFunc inverse reduce function + * @param reduceFunc associative and commutative reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which @@ -283,33 +264,11 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T dstream.slice(fromTime, toTime).map(wrapRDD).asJava } - /** - * Apply a function to each RDD in this DStream. This is an output operator, so - * 'this' DStream will be registered as an output stream and therefore materialized. - * - * @deprecated As of release 0.9.0, replaced by foreachRDD - */ - @deprecated("Use foreachRDD", "0.9.0") - def foreach(foreachFunc: JFunction[R, Void]) { - foreachRDD(foreachFunc) - } - - /** - * Apply a function to each RDD in this DStream. This is an output operator, so - * 'this' DStream will be registered as an output stream and therefore materialized. - * - * @deprecated As of release 0.9.0, replaced by foreachRDD - */ - @deprecated("Use foreachRDD", "0.9.0") - def foreach(foreachFunc: JFunction2[R, Time, Void]) { - foreachRDD(foreachFunc) - } - /** * Apply a function to each RDD in this DStream. This is an output operator, so * 'this' DStream will be registered as an output stream and therefore materialized. */ - def foreachRDD(foreachFunc: JFunction[R, Void]) { + def foreachRDD(foreachFunc: JVoidFunction[R]) { dstream.foreachRDD(rdd => foreachFunc.call(wrapRDD(rdd))) } @@ -317,7 +276,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * Apply a function to each RDD in this DStream. This is an output operator, so * 'this' DStream will be registered as an output stream and therefore materialized. */ - def foreachRDD(foreachFunc: JFunction2[R, Time, Void]) { + def foreachRDD(foreachFunc: JVoidFunction2[R, Time]) { dstream.foreachRDD((rdd, time) => foreachFunc.call(wrapRDD(rdd), time)) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaMapWithStateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaMapWithStateDStream.scala new file mode 100644 index 000000000000..16c0d6fff822 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaMapWithStateDStream.scala @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.java + +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.streaming.dstream.MapWithStateDStream + +/** + * :: Experimental :: + * DStream representing the stream of data generated by `mapWithState` operation on a + * [[JavaPairDStream]]. Additionally, it also gives access to the + * stream of state snapshots, that is, the state data of all keys after a batch has updated them. + * + * @tparam KeyType Class of the keys + * @tparam ValueType Class of the values + * @tparam StateType Class of the state data + * @tparam MappedType Class of the mapped data + */ +@Experimental +class JavaMapWithStateDStream[KeyType, ValueType, StateType, MappedType] private[streaming]( + dstream: MapWithStateDStream[KeyType, ValueType, StateType, MappedType]) + extends JavaDStream[MappedType](dstream)(JavaSparkContext.fakeClassTag) { + + def stateSnapshots(): JavaPairDStream[KeyType, StateType] = + new JavaPairDStream(dstream.stateSnapshots())( + JavaSparkContext.fakeClassTag, + JavaSparkContext.fakeClassTag) +} 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 e2aec6c2f63e..2ec907c8cfd5 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 @@ -17,19 +17,21 @@ package org.apache.spark.streaming.api.java -import java.lang.{Long => JLong, Iterable => JIterable} +import java.{lang => jl} +import java.lang.{Iterable => JIterable} import java.util.{List => JList} import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.ClassTag -import com.google.common.base.Optional import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.{JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} + import org.apache.spark.Partitioner -import org.apache.spark.api.java.{JavaPairRDD, JavaUtils} +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaUtils, Optional} import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} @@ -72,7 +74,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( */ def repartition(numPartitions: Int): JavaPairDStream[K, V] = dstream.repartition(numPartitions) - /** Method that generates a RDD for the given Duration */ + /** Method that generates an RDD for the given Duration */ def compute(validTime: Time): JavaPairRDD[K, V] = { dstream.compute(validTime) match { case Some(rdd) => new JavaPairRDD(rdd) @@ -136,8 +138,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are - * merged using the associative reduce function. Hash partitioning is used to generate the RDDs - * with Spark's default number of partitions. + * merged using the associative and commutative reduce function. Hash partitioning is used to + * generate the RDDs with Spark's default number of partitions. */ def reduceByKey(func: JFunction2[V, V, V]): JavaPairDStream[K, V] = dstream.reduceByKey(func) @@ -153,7 +155,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control - * thepartitioning of each RDD. + * the partitioning of each RDD. */ def reduceByKey(func: JFunction2[V, V, V], partitioner: Partitioner): JavaPairDStream[K, V] = { dstream.reduceByKey(func, partitioner) @@ -255,7 +257,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Similar to `DStream.reduceByKey()`, but applies it over a sliding window. The new DStream * generates RDDs with the same interval as this DStream. Hash partitioning is used to generate * the RDDs with Spark's default number of partitions. - * @param reduceFunc associative reduce function + * @param reduceFunc associative and commutative reduce function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval */ @@ -268,7 +270,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. - * @param reduceFunc associative reduce function + * @param reduceFunc associative and commutative reduce function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which @@ -287,7 +289,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. - * @param reduceFunc associative reduce function + * @param reduceFunc associative and commutative reduce function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which @@ -307,7 +309,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to * `DStream.reduceByKey()`, but applies it over a sliding window. - * @param reduceFunc associative reduce function + * @param reduceFunc associative rand commutative educe function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which @@ -333,8 +335,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * This is more efficient that reduceByKeyAndWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. - * @param reduceFunc associative reduce function - * @param invReduceFunc inverse function + * @param reduceFunc associative and commutative reduce function + * @param invReduceFunc inverse function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which @@ -358,7 +361,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * This is more efficient that reduceByKeyAndWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. - * @param reduceFunc associative reduce function + * @param reduceFunc associative and commutative reduce function * @param invReduceFunc inverse function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval @@ -395,7 +398,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) * This is more efficient that reduceByKeyAndWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". - * @param reduceFunc associative reduce function + * @param reduceFunc associative and commutative reduce function * @param invReduceFunc inverse function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval @@ -426,15 +429,52 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( ) } + /** + * :: Experimental :: + * Return a [[JavaMapWithStateDStream]] by applying a function to every key-value element of + * `this` stream, while maintaining some state data for each unique key. The mapping function + * and other specification (e.g. partitioners, timeouts, initial state data, etc.) of this + * transformation can be specified using `StateSpec` class. The state data is accessible in + * as a parameter of type `State` in the mapping function. + * + * Example of using `mapWithState`: + * {{{ + * // A mapping function that maintains an integer state and return a string + * Function3, State, String> mappingFunction = + * new Function3, State, String>() { + * @Override + * public Optional call(Optional value, State state) { + * // Use state.exists(), state.get(), state.update() and state.remove() + * // to manage state, and return the necessary string + * } + * }; + * + * JavaMapWithStateDStream mapWithStateDStream = + * keyValueDStream.mapWithState(StateSpec.function(mappingFunc)); + *}}} + * + * @param spec Specification of this transformation + * @tparam StateType Class type of the state data + * @tparam MappedType Class type of the mapped data + */ + @Experimental + def mapWithState[StateType, MappedType](spec: StateSpec[K, V, StateType, MappedType]): + JavaMapWithStateDStream[K, V, StateType, MappedType] = { + new JavaMapWithStateDStream(dstream.mapWithState(spec)( + JavaSparkContext.fakeClassTag, + JavaSparkContext.fakeClassTag)) + } + private def convertUpdateStateFunction[S](in: JFunction2[JList[V], Optional[S], Optional[S]]): (Seq[V], Option[S]) => Option[S] = { val scalaFunc: (Seq[V], Option[S]) => Option[S] = (values, state) => { val list: JList[V] = values.asJava val scalaState: Optional[S] = JavaUtils.optionToOptional(state) val result: Optional[S] = in.apply(list, scalaState) - result.isPresent match { - case true => Some(result.get()) - case _ => None + if (result.isPresent) { + Some(result.get()) + } else { + None } } scalaFunc @@ -809,7 +849,7 @@ object JavaPairDStream { } def scalaToJavaLong[K: ClassTag](dstream: JavaPairDStream[K, Long]) - : JavaPairDStream[K, JLong] = { - DStream.toPairDStreamFunctions(dstream.dstream).mapValues(new JLong(_)) + : JavaPairDStream[K, jl.Long] = { + DStream.toPairDStreamFunctions(dstream.dstream).mapValues(jl.Long.valueOf) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala index e6ff8a0cb545..da0db02236a1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala @@ -17,11 +17,11 @@ package org.apache.spark.streaming.api.java -import org.apache.spark.streaming.dstream.InputDStream - import scala.language.implicitConversions import scala.reflect.ClassTag +import org.apache.spark.streaming.dstream.InputDStream + /** * A Java-friendly interface to [[org.apache.spark.streaming.dstream.InputDStream]] of * key-value pairs. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 8f21c79a760c..982e72cffbf3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -17,19 +17,18 @@ package org.apache.spark.streaming.api.java -import java.lang.{Boolean => JBoolean} import java.io.{Closeable, InputStream} +import java.lang.{Boolean => JBoolean} import java.util.{List => JList, Map => JMap} import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import akka.actor.{Props, SupervisorStrategy} +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} import org.apache.spark.api.java.function.{Function0 => JFunction0} @@ -37,16 +36,15 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ -import org.apache.spark.streaming.scheduler.StreamingListener import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.receiver.Receiver -import org.apache.hadoop.conf.Configuration +import org.apache.spark.streaming.scheduler.StreamingListener /** * A Java-friendly version of [[org.apache.spark.streaming.StreamingContext]] which is the main * entry point for Spark Streaming functionality. It provides methods to create * [[org.apache.spark.streaming.api.java.JavaDStream]] and - * [[org.apache.spark.streaming.api.java.JavaPairDStream.]] from input sources. The internal + * [[org.apache.spark.streaming.api.java.JavaPairDStream]] from input sources. The internal * org.apache.spark.api.java.JavaSparkContext (see core Spark documentation) can be accessed * using `context.sparkContext`. After creating and transforming DStreams, the streaming * computation can be started and stopped using `context.start()` and `context.stop()`, @@ -155,12 +153,6 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { /** The underlying SparkContext */ val sparkContext = new JavaSparkContext(ssc.sc) - /** - * @deprecated As of 0.9.0, replaced by `sparkContext` - */ - @deprecated("use sparkContext", "0.9.0") - val sc: JavaSparkContext = sparkContext - /** * Create an input stream from network source hostname:port. Data is received using * a TCP socket and the receive bytes is interpreted as UTF8 encoded \n delimited @@ -226,11 +218,11 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * for new files and reads them as flat binary files with fixed record lengths, * yielding byte arrays * - * '''Note:''' We ensure that the byte array for each record in the - * resulting RDDs of the DStream has the provided record length. - * * @param directory HDFS directory to monitor for new files * @param recordLength The length at which to split the records + * + * @note We ensure that the byte array for each record in the + * resulting RDDs of the DStream has the provided record length. */ def binaryRecordsStream(directory: String, recordLength: Int): JavaDStream[Array[Byte]] = { ssc.binaryRecordsStream(directory, recordLength) @@ -357,79 +349,16 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { } /** - * Create an input stream with any arbitrary user implemented actor receiver. - * @param props Props object defining creation of the actor - * @param name Name of the actor - * @param storageLevel Storage level to use for storing the received objects - * - * @note An important point to note: - * Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e parametrized type of data received and actorStream - * should be same. - */ - def actorStream[T]( - props: Props, - name: String, - storageLevel: StorageLevel, - supervisorStrategy: SupervisorStrategy - ): JavaReceiverInputDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - ssc.actorStream[T](props, name, storageLevel, supervisorStrategy) - } - - /** - * Create an input stream with any arbitrary user implemented actor receiver. - * @param props Props object defining creation of the actor - * @param name Name of the actor - * @param storageLevel Storage level to use for storing the received objects - * - * @note An important point to note: - * Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e parametrized type of data received and actorStream - * should be same. - */ - def actorStream[T]( - props: Props, - name: String, - storageLevel: StorageLevel - ): JavaReceiverInputDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - ssc.actorStream[T](props, name, storageLevel) - } - - /** - * Create an input stream with any arbitrary user implemented actor receiver. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param props Props object defining creation of the actor - * @param name Name of the actor - * - * @note An important point to note: - * Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e parametrized type of data received and actorStream - * should be same. - */ - def actorStream[T]( - props: Props, - name: String - ): JavaReceiverInputDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - ssc.actorStream[T](props, name) - } - - /** - * Create an input stream from an queue of RDDs. In each batch, + * Create an input stream from a queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * - * NOTE: + * @param queue Queue of RDDs + * @tparam T Type of objects in the RDD + * + * @note * 1. Changes to the queue after the stream is created will not be recognized. * 2. Arbitrary RDDs can be added to `queueStream`, there is no way to recover data of * those RDDs, so `queueStream` doesn't support checkpointing. - * - * @param queue Queue of RDDs - * @tparam T Type of objects in the RDD */ def queueStream[T](queue: java.util.Queue[JavaRDD[T]]): JavaDStream[T] = { implicit val cm: ClassTag[T] = @@ -440,17 +369,17 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { } /** - * Create an input stream from an queue of RDDs. In each batch, + * Create an input stream from a queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * - * NOTE: - * 1. Changes to the queue after the stream is created will not be recognized. - * 2. Arbitrary RDDs can be added to `queueStream`, there is no way to recover data of - * those RDDs, so `queueStream` doesn't support checkpointing. - * * @param queue Queue of RDDs * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval * @tparam T Type of objects in the RDD + * + * @note + * 1. Changes to the queue after the stream is created will not be recognized. + * 2. Arbitrary RDDs can be added to `queueStream`, there is no way to recover data of + * those RDDs, so `queueStream` doesn't support checkpointing. */ def queueStream[T]( queue: java.util.Queue[JavaRDD[T]], @@ -464,10 +393,10 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { } /** - * Create an input stream from an queue of RDDs. In each batch, + * Create an input stream from a queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * - * NOTE: + * @note * 1. Changes to the queue after the stream is created will not be recognized. * 2. Arbitrary RDDs can be added to `queueStream`, there is no way to recover data of * those RDDs, so `queueStream` doesn't support checkpointing. @@ -525,9 +454,10 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { /** * Create a new DStream in which each RDD is generated by applying a function on RDDs of * the DStreams. The order of the JavaRDDs in the transform function parameter will be the - * same as the order of corresponding DStreams in the list. Note that for adding a - * JavaPairDStream in the list of JavaDStreams, convert it to a JavaDStream using - * [[org.apache.spark.streaming.api.java.JavaPairDStream]].toJavaDStream(). + * same as the order of corresponding DStreams in the list. + * + * @note For adding a JavaPairDStream in the list of JavaDStreams, convert it to a + * JavaDStream using [[org.apache.spark.streaming.api.java.JavaPairDStream]].toJavaDStream(). * In the transform function, convert the JavaRDD corresponding to that JavaDStream to * a JavaPairRDD using org.apache.spark.api.java.JavaPairRDD.fromJavaRDD(). */ @@ -547,9 +477,10 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { /** * Create a new DStream in which each RDD is generated by applying a function on RDDs of * the DStreams. The order of the JavaRDDs in the transform function parameter will be the - * same as the order of corresponding DStreams in the list. Note that for adding a - * JavaPairDStream in the list of JavaDStreams, convert it to a JavaDStream using - * [[org.apache.spark.streaming.api.java.JavaPairDStream]].toJavaDStream(). + * same as the order of corresponding DStreams in the list. + * + * @note For adding a JavaPairDStream in the list of JavaDStreams, convert it to + * a JavaDStream using [[org.apache.spark.streaming.api.java.JavaPairDStream]].toJavaDStream(). * In the transform function, convert the JavaRDD corresponding to that JavaDStream to * a JavaPairRDD using org.apache.spark.api.java.JavaPairRDD.fromJavaRDD(). */ @@ -588,9 +519,10 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { ssc.remember(duration) } - /** Add a [[org.apache.spark.streaming.scheduler.StreamingListener]] object for - * receiving system events related to streaming. - */ + /** + * Add a [[org.apache.spark.streaming.scheduler.StreamingListener]] object for + * receiving system events related to streaming. + */ def addStreamingListener(streamingListener: StreamingListener) { ssc.addStreamingListener(streamingListener) } @@ -601,7 +533,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * Return the current state of the context. The context can be in three possible states - *
      *
    • - * StreamingContextState.INTIALIZED - The context has been created, but not been started yet. + * StreamingContextState.INITIALIZED - The context has been created, but not been started yet. * Input DStreams, transformations and output operations can be created on the context. *
    • *
    • @@ -628,21 +560,11 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * Wait for the execution to stop. Any exceptions that occurs during the execution * will be thrown in this thread. */ + @throws[InterruptedException] def awaitTermination(): Unit = { ssc.awaitTermination() } - /** - * Wait for the execution to stop. Any exceptions that occurs during the execution - * will be thrown in this thread. - * @param timeout time to wait in milliseconds - * @deprecated As of 1.3.0, replaced by `awaitTerminationOrTimeout(Long)`. - */ - @deprecated("Use awaitTerminationOrTimeout(Long) instead", "1.3.0") - def awaitTermination(timeout: Long): Unit = { - ssc.awaitTermination(timeout) - } - /** * Wait for the execution to stop. Any exceptions that occurs during the execution * will be thrown in this thread. @@ -651,6 +573,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * @return `true` if it's stopped; or throw the reported error during the execution; or `false` * if the waiting time elapsed before returning from the method. */ + @throws[InterruptedException] def awaitTerminationOrTimeout(timeout: Long): Boolean = { ssc.awaitTerminationOrTimeout(timeout) } @@ -687,78 +610,6 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { */ object JavaStreamingContext { - /** - * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. - * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be - * recreated from the checkpoint data. If the data does not exist, then the provided factory - * will be used to create a JavaStreamingContext. - * - * @param checkpointPath Checkpoint directory used in an earlier JavaStreamingContext program - * @param factory JavaStreamingContextFactory object to create a new JavaStreamingContext - * @deprecated As of 1.4.0, replaced by `getOrCreate` without JavaStreamingContextFactor. - */ - @deprecated("use getOrCreate without JavaStreamingContextFactor", "1.4.0") - def getOrCreate( - checkpointPath: String, - factory: JavaStreamingContextFactory - ): JavaStreamingContext = { - val ssc = StreamingContext.getOrCreate(checkpointPath, () => { - factory.create.ssc - }) - new JavaStreamingContext(ssc) - } - - /** - * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. - * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be - * recreated from the checkpoint data. If the data does not exist, then the provided factory - * will be used to create a JavaStreamingContext. - * - * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program - * @param factory JavaStreamingContextFactory object to create a new JavaStreamingContext - * @param hadoopConf Hadoop configuration if necessary for reading from any HDFS compatible - * file system - * @deprecated As of 1.4.0, replaced by `getOrCreate` without JavaStreamingContextFactor. - */ - @deprecated("use getOrCreate without JavaStreamingContextFactory", "1.4.0") - def getOrCreate( - checkpointPath: String, - hadoopConf: Configuration, - factory: JavaStreamingContextFactory - ): JavaStreamingContext = { - val ssc = StreamingContext.getOrCreate(checkpointPath, () => { - factory.create.ssc - }, hadoopConf) - new JavaStreamingContext(ssc) - } - - /** - * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. - * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be - * recreated from the checkpoint data. If the data does not exist, then the provided factory - * will be used to create a JavaStreamingContext. - * - * @param checkpointPath Checkpoint directory used in an earlier StreamingContext program - * @param factory JavaStreamingContextFactory object to create a new JavaStreamingContext - * @param hadoopConf Hadoop configuration if necessary for reading from any HDFS compatible - * file system - * @param createOnError Whether to create a new JavaStreamingContext if there is an - * error in reading checkpoint data. - * @deprecated As of 1.4.0, replaced by `getOrCreate` without JavaStreamingContextFactor. - */ - @deprecated("use getOrCreate without JavaStreamingContextFactory", "1.4.0") - def getOrCreate( - checkpointPath: String, - hadoopConf: Configuration, - factory: JavaStreamingContextFactory, - createOnError: Boolean - ): JavaStreamingContext = { - val ssc = StreamingContext.getOrCreate(checkpointPath, () => { - factory.create.ssc - }, hadoopConf, createOnError) - new JavaStreamingContext(ssc) - } - /** * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. * If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be @@ -831,10 +682,3 @@ object JavaStreamingContext { */ def jarOfClass(cls: Class[_]): Array[String] = SparkContext.jarOfClass(cls).toArray } - -/** - * Factory interface for creating a new JavaStreamingContext - */ -trait JavaStreamingContextFactory { - def create(): JavaStreamingContext -} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingListener.scala new file mode 100644 index 000000000000..28cb86c9f31f --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingListener.scala @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.java + +import org.apache.spark.streaming.Time + +private[streaming] trait PythonStreamingListener{ + + /** Called when the streaming has been started */ + def onStreamingStarted(streamingStarted: JavaStreamingListenerStreamingStarted) { } + + /** Called when a receiver has been started */ + def onReceiverStarted(receiverStarted: JavaStreamingListenerReceiverStarted) { } + + /** Called when a receiver has reported an error */ + def onReceiverError(receiverError: JavaStreamingListenerReceiverError) { } + + /** Called when a receiver has been stopped */ + def onReceiverStopped(receiverStopped: JavaStreamingListenerReceiverStopped) { } + + /** Called when a batch of jobs has been submitted for processing. */ + def onBatchSubmitted(batchSubmitted: JavaStreamingListenerBatchSubmitted) { } + + /** Called when processing of a batch of jobs has started. */ + def onBatchStarted(batchStarted: JavaStreamingListenerBatchStarted) { } + + /** Called when processing of a batch of jobs has completed. */ + def onBatchCompleted(batchCompleted: JavaStreamingListenerBatchCompleted) { } + + /** Called when processing of a job of a batch has started. */ + def onOutputOperationStarted( + outputOperationStarted: JavaStreamingListenerOutputOperationStarted) { } + + /** Called when processing of a job of a batch has completed. */ + def onOutputOperationCompleted( + outputOperationCompleted: JavaStreamingListenerOutputOperationCompleted) { } +} + +private[streaming] class PythonStreamingListenerWrapper(listener: PythonStreamingListener) + extends JavaStreamingListener { + + /** Called when the streaming has been started */ + override def onStreamingStarted(streamingStarted: JavaStreamingListenerStreamingStarted): Unit = { + listener.onStreamingStarted(streamingStarted) + } + + /** Called when a receiver has been started */ + override def onReceiverStarted(receiverStarted: JavaStreamingListenerReceiverStarted): Unit = { + listener.onReceiverStarted(receiverStarted) + } + + /** Called when a receiver has reported an error */ + override def onReceiverError(receiverError: JavaStreamingListenerReceiverError): Unit = { + listener.onReceiverError(receiverError) + } + + /** Called when a receiver has been stopped */ + override def onReceiverStopped(receiverStopped: JavaStreamingListenerReceiverStopped): Unit = { + listener.onReceiverStopped(receiverStopped) + } + + /** Called when a batch of jobs has been submitted for processing. */ + override def onBatchSubmitted(batchSubmitted: JavaStreamingListenerBatchSubmitted): Unit = { + listener.onBatchSubmitted(batchSubmitted) + } + + /** Called when processing of a batch of jobs has started. */ + override def onBatchStarted(batchStarted: JavaStreamingListenerBatchStarted): Unit = { + listener.onBatchStarted(batchStarted) + } + + /** Called when processing of a batch of jobs has completed. */ + override def onBatchCompleted(batchCompleted: JavaStreamingListenerBatchCompleted): Unit = { + listener.onBatchCompleted(batchCompleted) + } + + /** Called when processing of a job of a batch has started. */ + override def onOutputOperationStarted( + outputOperationStarted: JavaStreamingListenerOutputOperationStarted): Unit = { + listener.onOutputOperationStarted(outputOperationStarted) + } + + /** Called when processing of a job of a batch has completed. */ + override def onOutputOperationCompleted( + outputOperationCompleted: JavaStreamingListenerOutputOperationCompleted): Unit = { + listener.onOutputOperationCompleted(outputOperationCompleted) + } +} + +/** + * A listener interface for receiving information about an ongoing streaming computation. + */ +private[streaming] class JavaStreamingListener { + + /** Called when the streaming has been started */ + def onStreamingStarted(streamingStarted: JavaStreamingListenerStreamingStarted): Unit = { } + + /** Called when a receiver has been started */ + def onReceiverStarted(receiverStarted: JavaStreamingListenerReceiverStarted): Unit = { } + + /** Called when a receiver has reported an error */ + def onReceiverError(receiverError: JavaStreamingListenerReceiverError): Unit = { } + + /** Called when a receiver has been stopped */ + def onReceiverStopped(receiverStopped: JavaStreamingListenerReceiverStopped): Unit = { } + + /** Called when a batch of jobs has been submitted for processing. */ + def onBatchSubmitted(batchSubmitted: JavaStreamingListenerBatchSubmitted): Unit = { } + + /** Called when processing of a batch of jobs has started. */ + def onBatchStarted(batchStarted: JavaStreamingListenerBatchStarted): Unit = { } + + /** Called when processing of a batch of jobs has completed. */ + def onBatchCompleted(batchCompleted: JavaStreamingListenerBatchCompleted): Unit = { } + + /** Called when processing of a job of a batch has started. */ + def onOutputOperationStarted( + outputOperationStarted: JavaStreamingListenerOutputOperationStarted): Unit = { } + + /** Called when processing of a job of a batch has completed. */ + def onOutputOperationCompleted( + outputOperationCompleted: JavaStreamingListenerOutputOperationCompleted): Unit = { } +} + +/** + * Base trait for events related to JavaStreamingListener + */ +private[streaming] sealed trait JavaStreamingListenerEvent + +private[streaming] class JavaStreamingListenerStreamingStarted(val time: Long) + extends JavaStreamingListenerEvent + +private[streaming] class JavaStreamingListenerBatchSubmitted(val batchInfo: JavaBatchInfo) + extends JavaStreamingListenerEvent + +private[streaming] class JavaStreamingListenerBatchCompleted(val batchInfo: JavaBatchInfo) + extends JavaStreamingListenerEvent + +private[streaming] class JavaStreamingListenerBatchStarted(val batchInfo: JavaBatchInfo) + extends JavaStreamingListenerEvent + +private[streaming] class JavaStreamingListenerOutputOperationStarted( + val outputOperationInfo: JavaOutputOperationInfo) extends JavaStreamingListenerEvent + +private[streaming] class JavaStreamingListenerOutputOperationCompleted( + val outputOperationInfo: JavaOutputOperationInfo) extends JavaStreamingListenerEvent + +private[streaming] class JavaStreamingListenerReceiverStarted(val receiverInfo: JavaReceiverInfo) + extends JavaStreamingListenerEvent + +private[streaming] class JavaStreamingListenerReceiverError(val receiverInfo: JavaReceiverInfo) + extends JavaStreamingListenerEvent + +private[streaming] class JavaStreamingListenerReceiverStopped(val receiverInfo: JavaReceiverInfo) + extends JavaStreamingListenerEvent + +/** + * Class having information on batches. + * + * @param batchTime Time of the batch + * @param streamIdToInputInfo A map of input stream id to its input info + * @param submissionTime Clock time of when jobs of this batch was submitted to the streaming + * scheduler queue + * @param processingStartTime Clock time of when the first job of this batch started processing. + * `-1` means the batch has not yet started + * @param processingEndTime Clock time of when the last job of this batch finished processing. `-1` + * means the batch has not yet completed. + * @param schedulingDelay Time taken for the first job of this batch to start processing from the + * time this batch was submitted to the streaming scheduler. Essentially, it + * is `processingStartTime` - `submissionTime`. `-1` means the batch has not + * yet started + * @param processingDelay Time taken for the all jobs of this batch to finish processing from the + * time they started processing. Essentially, it is + * `processingEndTime` - `processingStartTime`. `-1` means the batch has not + * yet completed. + * @param totalDelay Time taken for all the jobs of this batch to finish processing from the time + * they were submitted. Essentially, it is `processingDelay` + `schedulingDelay`. + * `-1` means the batch has not yet completed. + * @param numRecords The number of recorders received by the receivers in this batch + * @param outputOperationInfos The output operations in this batch + */ +private[streaming] case class JavaBatchInfo( + batchTime: Time, + streamIdToInputInfo: java.util.Map[Int, JavaStreamInputInfo], + submissionTime: Long, + processingStartTime: Long, + processingEndTime: Long, + schedulingDelay: Long, + processingDelay: Long, + totalDelay: Long, + numRecords: Long, + outputOperationInfos: java.util.Map[Int, JavaOutputOperationInfo]) + +/** + * Track the information of input stream at specified batch time. + * + * @param inputStreamId the input stream id + * @param numRecords the number of records in a batch + * @param metadata metadata for this batch. It should contain at least one standard field named + * "Description" which maps to the content that will be shown in the UI. + * @param metadataDescription description of this input stream + */ +private[streaming] case class JavaStreamInputInfo( + inputStreamId: Int, + numRecords: Long, + metadata: java.util.Map[String, Any], + metadataDescription: String) + +/** + * Class having information about a receiver + */ +private[streaming] case class JavaReceiverInfo( + streamId: Int, + name: String, + active: Boolean, + location: String, + executorId: String, + lastErrorMessage: String, + lastError: String, + lastErrorTime: Long) + +/** + * Class having information on output operations. + * + * @param batchTime Time of the batch + * @param id Id of this output operation. Different output operations have different ids in a batch. + * @param name The name of this output operation. + * @param description The description of this output operation. + * @param startTime Clock time of when the output operation started processing. `-1` means the + * output operation has not yet started + * @param endTime Clock time of when the output operation started processing. `-1` means the output + * operation has not yet completed + * @param failureReason Failure reason if this output operation fails. If the output operation is + * successful, this field is `null`. + */ +private[streaming] case class JavaOutputOperationInfo( + batchTime: Time, + id: Int, + name: String, + description: String, + startTime: Long, + endTime: Long, + failureReason: String) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapper.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapper.scala new file mode 100644 index 000000000000..ee8370d26260 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapper.scala @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.api.java + +import scala.collection.JavaConverters._ + +import org.apache.spark.streaming.scheduler._ + +/** + * A wrapper to convert a [[JavaStreamingListener]] to a [[StreamingListener]]. + */ +private[streaming] class JavaStreamingListenerWrapper(javaStreamingListener: JavaStreamingListener) + extends StreamingListener { + + private def toJavaReceiverInfo(receiverInfo: ReceiverInfo): JavaReceiverInfo = { + JavaReceiverInfo( + receiverInfo.streamId, + receiverInfo.name, + receiverInfo.active, + receiverInfo.location, + receiverInfo.executorId, + receiverInfo.lastErrorMessage, + receiverInfo.lastError, + receiverInfo.lastErrorTime + ) + } + + private def toJavaStreamInputInfo(streamInputInfo: StreamInputInfo): JavaStreamInputInfo = { + JavaStreamInputInfo( + streamInputInfo.inputStreamId, + streamInputInfo.numRecords: Long, + streamInputInfo.metadata.asJava, + streamInputInfo.metadataDescription.orNull + ) + } + + private def toJavaOutputOperationInfo( + outputOperationInfo: OutputOperationInfo): JavaOutputOperationInfo = { + JavaOutputOperationInfo( + outputOperationInfo.batchTime, + outputOperationInfo.id, + outputOperationInfo.name, + outputOperationInfo.description: String, + outputOperationInfo.startTime.getOrElse(-1), + outputOperationInfo.endTime.getOrElse(-1), + outputOperationInfo.failureReason.orNull + ) + } + + private def toJavaBatchInfo(batchInfo: BatchInfo): JavaBatchInfo = { + JavaBatchInfo( + batchInfo.batchTime, + batchInfo.streamIdToInputInfo.mapValues(toJavaStreamInputInfo(_)).asJava, + batchInfo.submissionTime, + batchInfo.processingStartTime.getOrElse(-1), + batchInfo.processingEndTime.getOrElse(-1), + batchInfo.schedulingDelay.getOrElse(-1), + batchInfo.processingDelay.getOrElse(-1), + batchInfo.totalDelay.getOrElse(-1), + batchInfo.numRecords, + batchInfo.outputOperationInfos.mapValues(toJavaOutputOperationInfo(_)).asJava + ) + } + + override def onStreamingStarted(streamingStarted: StreamingListenerStreamingStarted): Unit = { + javaStreamingListener.onStreamingStarted( + new JavaStreamingListenerStreamingStarted(streamingStarted.time)) + } + + override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted): Unit = { + javaStreamingListener.onReceiverStarted( + new JavaStreamingListenerReceiverStarted(toJavaReceiverInfo(receiverStarted.receiverInfo))) + } + + override def onReceiverError(receiverError: StreamingListenerReceiverError): Unit = { + javaStreamingListener.onReceiverError( + new JavaStreamingListenerReceiverError(toJavaReceiverInfo(receiverError.receiverInfo))) + } + + override def onReceiverStopped(receiverStopped: StreamingListenerReceiverStopped): Unit = { + javaStreamingListener.onReceiverStopped( + new JavaStreamingListenerReceiverStopped(toJavaReceiverInfo(receiverStopped.receiverInfo))) + } + + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit = { + javaStreamingListener.onBatchSubmitted( + new JavaStreamingListenerBatchSubmitted(toJavaBatchInfo(batchSubmitted.batchInfo))) + } + + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = { + javaStreamingListener.onBatchStarted( + new JavaStreamingListenerBatchStarted(toJavaBatchInfo(batchStarted.batchInfo))) + } + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { + javaStreamingListener.onBatchCompleted( + new JavaStreamingListenerBatchCompleted(toJavaBatchInfo(batchCompleted.batchInfo))) + } + + override def onOutputOperationStarted( + outputOperationStarted: StreamingListenerOutputOperationStarted): Unit = { + javaStreamingListener.onOutputOperationStarted(new JavaStreamingListenerOutputOperationStarted( + toJavaOutputOperationInfo(outputOperationStarted.outputOperationInfo))) + } + + override def onOutputOperationCompleted( + outputOperationCompleted: StreamingListenerOutputOperationCompleted): Unit = { + javaStreamingListener.onOutputOperationCompleted( + new JavaStreamingListenerOutputOperationCompleted( + toJavaOutputOperationInfo(outputOperationCompleted.outputOperationInfo))) + } + +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/package-info.java b/streaming/src/main/scala/org/apache/spark/streaming/api/java/package-info.java index d43d949d76bb..348d21d49ac4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/package-info.java +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/package-info.java @@ -18,4 +18,4 @@ /** * Java APIs for spark streaming. */ -package org.apache.spark.streaming.api.java; \ No newline at end of file +package org.apache.spark.streaming.api.java; diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala index dfc569451df8..46bfc6085645 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala @@ -24,22 +24,30 @@ import java.util.{ArrayList => JArrayList, List => JList} import scala.collection.JavaConverters._ import scala.language.existentials -import py4j.GatewayServer +import py4j.Py4JException +import org.apache.spark.SparkException import org.apache.spark.api.java._ +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Interval, Duration, Time} -import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.{Duration, Interval, StreamingContext, Time} import org.apache.spark.streaming.api.java._ +import org.apache.spark.streaming.dstream._ import org.apache.spark.util.Utils - /** * Interface for Python callback function which is used to transform RDDs */ private[python] trait PythonTransformFunction { def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]] + + /** + * Get the failure, if any, in the last call to `call`. + * + * @return the failure message if there was a failure, or `null` if there was no failure. + */ + def getLastFailure: String } /** @@ -48,6 +56,13 @@ private[python] trait PythonTransformFunction { private[python] trait PythonTransformFunctionSerializer { def dumps(id: String): Array[Byte] def loads(bytes: Array[Byte]): PythonTransformFunction + + /** + * Get the failure, if any, in the last call to `dumps` or `loads`. + * + * @return the failure message if there was a failure, or `null` if there was no failure. + */ + def getLastFailure: String } /** @@ -59,18 +74,27 @@ private[python] class TransformFunction(@transient var pfunc: PythonTransformFun extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] { def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = { - Option(pfunc.call(time.milliseconds, List(rdd.map(JavaRDD.fromRDD(_)).orNull).asJava)) - .map(_.rdd) + val rdds = List(rdd.map(JavaRDD.fromRDD(_)).orNull).asJava + Option(callPythonTransformFunction(time.milliseconds, rdds)).map(_.rdd) } def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = { val rdds = List(rdd.map(JavaRDD.fromRDD(_)).orNull, rdd2.map(JavaRDD.fromRDD(_)).orNull).asJava - Option(pfunc.call(time.milliseconds, rdds)).map(_.rdd) + Option(callPythonTransformFunction(time.milliseconds, rdds)).map(_.rdd) } // for function.Function2 def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = { - pfunc.call(time.milliseconds, rdds) + callPythonTransformFunction(time.milliseconds, rdds) + } + + private def callPythonTransformFunction(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]] = { + val resultRDD = pfunc.call(time, rdds) + val failure = pfunc.getLastFailure + if (failure != null) { + throw new SparkException("An exception was raised by Python:\n" + failure) + } + resultRDD } private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { @@ -103,30 +127,40 @@ private[python] object PythonTransformFunctionSerializer { /* * Register a serializer from Python, should be called during initialization */ - def register(ser: PythonTransformFunctionSerializer): Unit = { + def register(ser: PythonTransformFunctionSerializer): Unit = synchronized { serializer = ser } - def serialize(func: PythonTransformFunction): Array[Byte] = { + def serialize(func: PythonTransformFunction): Array[Byte] = synchronized { require(serializer != null, "Serializer has not been registered!") // get the id of PythonTransformFunction in py4j val h = Proxy.getInvocationHandler(func.asInstanceOf[Proxy]) val f = h.getClass().getDeclaredField("id") f.setAccessible(true) val id = f.get(h).asInstanceOf[String] - serializer.dumps(id) + val results = serializer.dumps(id) + val failure = serializer.getLastFailure + if (failure != null) { + throw new SparkException("An exception was raised by Python:\n" + failure) + } + results } - def deserialize(bytes: Array[Byte]): PythonTransformFunction = { + def deserialize(bytes: Array[Byte]): PythonTransformFunction = synchronized { require(serializer != null, "Serializer has not been registered!") - serializer.loads(bytes) + val pfunc = serializer.loads(bytes) + val failure = serializer.getLastFailure + if (failure != null) { + throw new SparkException("An exception was raised by Python:\n" + failure) + } + pfunc } } /** * Helper functions, which are called from Python via Py4J. */ -private[python] object PythonDStream { +private[streaming] object PythonDStream { /** * can not access PythonTransformFunctionSerializer.register() via Py4j @@ -136,16 +170,6 @@ private[python] object PythonDStream { PythonTransformFunctionSerializer.register(ser) } - /** - * Update the port of callback client to `port` - */ - def updatePythonGatewayPort(gws: GatewayServer, port: Int): Unit = { - val cl = gws.getCallbackClient - val f = cl.getClass.getDeclaredField("port") - f.setAccessible(true) - f.setInt(cl, port) - } - /** * helper function for DStream.foreachRDD(), * cannot be `foreachRDD`, it will confusing py4j @@ -163,6 +187,32 @@ private[python] object PythonDStream { rdds.asScala.foreach(queue.add) queue } + + /** + * Stop [[StreamingContext]] if the Python process crashes (E.g., OOM) in case the user cannot + * stop it in the Python side. + */ + def stopStreamingContextIfPythonProcessIsDead(e: Throwable): Unit = { + // These two special messages are from: + // scalastyle:off + // https://github.com/bartdag/py4j/blob/5cbb15a21f857e8cf334ce5f675f5543472f72eb/py4j-java/src/main/java/py4j/CallbackClient.java#L218 + // https://github.com/bartdag/py4j/blob/5cbb15a21f857e8cf334ce5f675f5543472f72eb/py4j-java/src/main/java/py4j/CallbackClient.java#L340 + // scalastyle:on + if (e.isInstanceOf[Py4JException] && + ("Cannot obtain a new communication channel" == e.getMessage || + "Error while obtaining a new communication channel" == e.getMessage)) { + // Start a new thread to stop StreamingContext to avoid deadlock. + new Thread("Stop-StreamingContext") with Logging { + setDaemon(true) + + override def run(): Unit = { + logError( + "Cannot connect to Python process. It's probably dead. Stopping StreamingContext.", e) + StreamingContext.getActive().foreach(_.stop(stopSparkContext = false)) + } + }.start() + } + } } /** @@ -230,9 +280,19 @@ private[python] class PythonTransformed2DStream( */ private[python] class PythonStateDStream( parent: DStream[Array[Byte]], - reduceFunc: PythonTransformFunction) + reduceFunc: PythonTransformFunction, + initialRDD: Option[RDD[Array[Byte]]]) extends PythonDStream(parent, reduceFunc) { + def this( + parent: DStream[Array[Byte]], + reduceFunc: PythonTransformFunction) = this(parent, reduceFunc, None) + + def this( + parent: DStream[Array[Byte]], + reduceFunc: PythonTransformFunction, + initialRDD: JavaRDD[Array[Byte]]) = this(parent, reduceFunc, Some(initialRDD.rdd)) + super.persist(StorageLevel.MEMORY_ONLY) override val mustCheckpoint = true @@ -240,7 +300,7 @@ private[python] class PythonStateDStream( val lastState = getOrCompute(validTime - slideDuration) val rdd = parent.getOrCompute(validTime) if (rdd.isDefined) { - func(lastState, rdd, validTime) + func(lastState.orElse(initialRDD), rdd, validTime) } else { lastState } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala index 4eb92dd8b105..995470ec8dea 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala @@ -20,13 +20,13 @@ package org.apache.spark.streaming.dstream import scala.reflect.ClassTag import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.{Time, StreamingContext} +import org.apache.spark.streaming.{StreamingContext, Time} /** - * An input stream that always returns the same RDD on each timestep. Useful for testing. + * An input stream that always returns the same RDD on each time step. Useful for testing. */ -class ConstantInputDStream[T: ClassTag](ssc_ : StreamingContext, rdd: RDD[T]) - extends InputDStream[T](ssc_) { +class ConstantInputDStream[T: ClassTag](_ssc: StreamingContext, rdd: RDD[T]) + extends InputDStream[T](_ssc) { require(rdd != null, "parameter rdd null is illegal, which will lead to NPE in the following transformation") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 1da0b0a54df0..e23edfa50651 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -25,14 +25,16 @@ import scala.language.implicitConversions import scala.reflect.ClassTag import scala.util.matching.Regex -import org.apache.spark.{Logging, SparkContext, SparkException} -import org.apache.spark.rdd.{BlockRDD, PairRDDFunctions, RDD, RDDOperationScope} +import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io.SparkHadoopWriterUtils +import org.apache.spark.rdd.{BlockRDD, RDD, RDDOperationScope} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext.rddToFileName import org.apache.spark.streaming.scheduler.Job import org.apache.spark.streaming.ui.UIUtils -import org.apache.spark.util.{CallSite, MetadataCleaner, Utils} +import org.apache.spark.util.{CallSite, Utils} /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous @@ -51,7 +53,7 @@ import org.apache.spark.util.{CallSite, MetadataCleaner, Utils} * `join`. These operations are automatically available on any DStream of pairs * (e.g., DStream[(Int, Int)] through implicit conversions. * - * DStreams internally is characterized by a few basic properties: + * A DStream internally is characterized by a few basic properties: * - A list of other DStreams that the DStream depends on * - A time interval at which the DStream generates an RDD * - A function that is used to generate an RDD after each time interval @@ -67,13 +69,13 @@ abstract class DStream[T: ClassTag] ( // Methods that should be implemented by subclasses of DStream // ======================================================================= - /** Time interval after which the DStream generates a RDD */ + /** Time interval after which the DStream generates an RDD */ def slideDuration: Duration /** List of parent DStreams on which this DStream depends on */ def dependencies: List[DStream[_]] - /** Method that generates a RDD for the given time */ + /** Method that generates an RDD for the given time */ def compute(validTime: Time): Option[RDD[T]] // ======================================================================= @@ -82,7 +84,7 @@ abstract class DStream[T: ClassTag] ( // RDDs generated, marked as private[streaming] so that testsuites can access it @transient - private[streaming] var generatedRDDs = new HashMap[Time, RDD[T]] () + private[streaming] var generatedRDDs = new HashMap[Time, RDD[T]]() // Time zero for the DStream private[streaming] var zeroTime: Time = null @@ -97,11 +99,13 @@ abstract class DStream[T: ClassTag] ( private[streaming] val mustCheckpoint = false private[streaming] var checkpointDuration: Duration = null private[streaming] val checkpointData = new DStreamCheckpointData(this) + @transient + private var restoredFromCheckpointData = false // Reference to whole DStream graph private[streaming] var graph: DStreamGraph = null - private[streaming] def isInitialized = (zeroTime != null) + private[streaming] def isInitialized = zeroTime != null // Duration for which the DStream requires its parent DStream to remember each RDD created private[streaming] def parentRememberDuration = rememberDuration @@ -154,7 +158,7 @@ abstract class DStream[T: ClassTag] ( def persist(level: StorageLevel): DStream[T] = { if (this.isInitialized) { throw new UnsupportedOperationException( - "Cannot change storage level of an DStream after streaming context has started") + "Cannot change storage level of a DStream after streaming context has started") } this.storageLevel = level this @@ -173,7 +177,7 @@ abstract class DStream[T: ClassTag] ( def checkpoint(interval: Duration): DStream[T] = { if (isInitialized) { throw new UnsupportedOperationException( - "Cannot change checkpoint interval of an DStream after streaming context has started") + "Cannot change checkpoint interval of a DStream after streaming context has started") } persist() checkpointDuration = interval @@ -187,15 +191,15 @@ abstract class DStream[T: ClassTag] ( */ private[streaming] def initialize(time: Time) { if (zeroTime != null && zeroTime != time) { - throw new SparkException("ZeroTime is already initialized to " + zeroTime - + ", cannot initialize it again to " + time) + throw new SparkException(s"ZeroTime is already initialized to $zeroTime" + + s", cannot initialize it again to $time") } zeroTime = time // Set the checkpoint interval to be slideDuration or 10 seconds, which ever is larger if (mustCheckpoint && checkpointDuration == null) { checkpointDuration = slideDuration * math.ceil(Seconds(10) / slideDuration).toInt - logInfo("Checkpoint interval automatically set to " + checkpointDuration) + logInfo(s"Checkpoint interval automatically set to $checkpointDuration") } // Set the minimum value of the rememberDuration if not already set @@ -232,7 +236,7 @@ abstract class DStream[T: ClassTag] ( require( !mustCheckpoint || checkpointDuration != null, - "The checkpoint interval for " + this.getClass.getSimpleName + " has not been set." + + s"The checkpoint interval for ${this.getClass.getSimpleName} has not been set." + " Please use DStream.checkpoint() to set the interval." ) @@ -243,65 +247,53 @@ abstract class DStream[T: ClassTag] ( require( checkpointDuration == null || checkpointDuration >= slideDuration, - "The checkpoint interval for " + this.getClass.getSimpleName + " has been set to " + - checkpointDuration + " which is lower than its slide time (" + slideDuration + "). " + - "Please set it to at least " + slideDuration + "." + s"The checkpoint interval for ${this.getClass.getSimpleName} has been set to " + + s"$checkpointDuration which is lower than its slide time ($slideDuration). " + + s"Please set it to at least $slideDuration." ) require( checkpointDuration == null || checkpointDuration.isMultipleOf(slideDuration), - "The checkpoint interval for " + this.getClass.getSimpleName + " has been set to " + - checkpointDuration + " which not a multiple of its slide time (" + slideDuration + "). " + - "Please set it to a multiple of " + slideDuration + "." + s"The checkpoint interval for ${this.getClass.getSimpleName} has been set to " + + s" $checkpointDuration which not a multiple of its slide time ($slideDuration). " + + s"Please set it to a multiple of $slideDuration." ) require( checkpointDuration == null || storageLevel != StorageLevel.NONE, - "" + this.getClass.getSimpleName + " has been marked for checkpointing but the storage " + + s"${this.getClass.getSimpleName} has been marked for checkpointing but the storage " + "level has not been set to enable persisting. Please use DStream.persist() to set the " + "storage level to use memory for better checkpointing performance." ) require( checkpointDuration == null || rememberDuration > checkpointDuration, - "The remember duration for " + this.getClass.getSimpleName + " has been set to " + - rememberDuration + " which is not more than the checkpoint interval (" + - checkpointDuration + "). Please set it to higher than " + checkpointDuration + "." - ) - - val metadataCleanerDelay = MetadataCleaner.getDelaySeconds(ssc.conf) - logInfo("metadataCleanupDelay = " + metadataCleanerDelay) - require( - metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000, - "It seems you are doing some DStream window operation or setting a checkpoint interval " + - "which requires " + this.getClass.getSimpleName + " to remember generated RDDs for more " + - "than " + rememberDuration.milliseconds / 1000 + " seconds. But Spark's metadata cleanup" + - "delay is set to " + metadataCleanerDelay + " seconds, which is not sufficient. Please " + - "set the Java cleaner delay to more than " + - math.ceil(rememberDuration.milliseconds / 1000.0).toInt + " seconds." + s"The remember duration for ${this.getClass.getSimpleName} has been set to " + + s" $rememberDuration which is not more than the checkpoint interval" + + s" ($checkpointDuration). Please set it to a value higher than $checkpointDuration." ) dependencies.foreach(_.validateAtStart()) - logInfo("Slide time = " + slideDuration) - logInfo("Storage level = " + storageLevel) - logInfo("Checkpoint interval = " + checkpointDuration) - logInfo("Remember duration = " + rememberDuration) - logInfo("Initialized and validated " + this) + logInfo(s"Slide time = $slideDuration") + logInfo(s"Storage level = ${storageLevel.description}") + logInfo(s"Checkpoint interval = $checkpointDuration") + logInfo(s"Remember interval = $rememberDuration") + logInfo(s"Initialized and validated $this") } private[streaming] def setContext(s: StreamingContext) { if (ssc != null && ssc != s) { - throw new SparkException("Context is already set in " + this + ", cannot set it again") + throw new SparkException(s"Context must not be set again for $this") } ssc = s - logInfo("Set context for " + this) + logInfo(s"Set context for $this") dependencies.foreach(_.setContext(ssc)) } private[streaming] def setGraph(g: DStreamGraph) { if (graph != null && graph != g) { - throw new SparkException("Graph is already set in " + this + ", cannot set it again") + throw new SparkException(s"Graph must not be set again for $this") } graph = g dependencies.foreach(_.setGraph(graph)) @@ -310,7 +302,7 @@ abstract class DStream[T: ClassTag] ( private[streaming] def remember(duration: Duration) { if (duration != null && (rememberDuration == null || duration > rememberDuration)) { rememberDuration = duration - logInfo("Duration for remembering RDDs set to " + rememberDuration + " for " + this) + logInfo(s"Duration for remembering RDDs set to $rememberDuration for $this") } dependencies.foreach(_.remember(parentRememberDuration)) } @@ -320,11 +312,11 @@ abstract class DStream[T: ClassTag] ( if (!isInitialized) { throw new SparkException (this + " has not been initialized") } else if (time <= zeroTime || ! (time - zeroTime).isMultipleOf(slideDuration)) { - logInfo("Time " + time + " is invalid as zeroTime is " + zeroTime + - " and slideDuration is " + slideDuration + " and difference is " + (time - zeroTime)) + logInfo(s"Time $time is invalid as zeroTime is $zeroTime" + + s" , slideDuration is $slideDuration and difference is ${time - zeroTime}") false } else { - logDebug("Time " + time + " is valid") + logDebug(s"Time $time is valid") true } } @@ -341,12 +333,12 @@ abstract class DStream[T: ClassTag] ( // of RDD generation, else generate nothing. if (isTimeValid(time)) { - val rddOption = createRDDWithLocalProperties(time) { + val rddOption = createRDDWithLocalProperties(time, displayInnerRDDOps = false) { // Disable checks for existing output directories in jobs launched by the streaming // scheduler, since we may need to write output to an existing directory during checkpoint // recovery; see SPARK-4835 for more details. We need to have this call here because // compute() might cause Spark jobs to be launched. - PairRDDFunctions.disableOutputSpecValidation.withValue(true) { + SparkHadoopWriterUtils.disableOutputSpecValidation.withValue(true) { compute(time) } } @@ -373,27 +365,52 @@ abstract class DStream[T: ClassTag] ( /** * Wrap a body of code such that the call site and operation scope * information are passed to the RDDs created in this body properly. - */ - protected def createRDDWithLocalProperties[U](time: Time)(body: => U): U = { + * @param body RDD creation code to execute with certain local properties. + * @param time Current batch time that should be embedded in the scope names + * @param displayInnerRDDOps Whether the detailed callsites and scopes of the inner RDDs generated + * by `body` will be displayed in the UI; only the scope and callsite + * of the DStream operation that generated `this` will be displayed. + */ + protected[streaming] def createRDDWithLocalProperties[U]( + time: Time, + displayInnerRDDOps: Boolean)(body: => U): U = { val scopeKey = SparkContext.RDD_SCOPE_KEY val scopeNoOverrideKey = SparkContext.RDD_SCOPE_NO_OVERRIDE_KEY // Pass this DStream's operation scope and creation site information to RDDs through // thread-local properties in our SparkContext. Since this method may be called from another // DStream, we need to temporarily store any old scope and creation site information to // restore them later after setting our own. - val prevCallSite = ssc.sparkContext.getCallSite() + val prevCallSite = CallSite( + ssc.sparkContext.getLocalProperty(CallSite.SHORT_FORM), + ssc.sparkContext.getLocalProperty(CallSite.LONG_FORM) + ) val prevScope = ssc.sparkContext.getLocalProperty(scopeKey) val prevScopeNoOverride = ssc.sparkContext.getLocalProperty(scopeNoOverrideKey) try { - ssc.sparkContext.setCallSite(creationSite) + if (displayInnerRDDOps) { + // Unset the short form call site, so that generated RDDs get their own + ssc.sparkContext.setLocalProperty(CallSite.SHORT_FORM, null) + ssc.sparkContext.setLocalProperty(CallSite.LONG_FORM, null) + } else { + // Set the callsite, so that the generated RDDs get the DStream's call site and + // the internal RDD call sites do not get displayed + ssc.sparkContext.setCallSite(creationSite) + } + // Use the DStream's base scope for this RDD so we can (1) preserve the higher level // DStream operation name, and (2) share this scope with other DStreams created in the // same operation. Disallow nesting so that low-level Spark primitives do not show up. // TODO: merge callsites with scopes so we can just reuse the code there makeScope(time).foreach { s => ssc.sparkContext.setLocalProperty(scopeKey, s.toJson) - ssc.sparkContext.setLocalProperty(scopeNoOverrideKey, "true") + if (displayInnerRDDOps) { + // Allow inner RDDs to add inner scopes + ssc.sparkContext.setLocalProperty(scopeNoOverrideKey, null) + } else { + // Do not allow inner RDDs to override the scope set by DStream + ssc.sparkContext.setLocalProperty(scopeNoOverrideKey, "true") + } } body @@ -413,13 +430,12 @@ abstract class DStream[T: ClassTag] ( */ private[streaming] def generateJob(time: Time): Option[Job] = { getOrCompute(time) match { - case Some(rdd) => { + case Some(rdd) => val jobFunc = () => { val emptyFunc = { (iterator: Iterator[T]) => {} } context.sparkContext.runJob(rdd, emptyFunc) } Some(new Job(time, jobFunc)) - } case None => None } } @@ -437,20 +453,20 @@ abstract class DStream[T: ClassTag] ( oldRDDs.map(x => s"${x._1} -> ${x._2.id}").mkString(", ") + "]") generatedRDDs --= oldRDDs.keys if (unpersistData) { - logDebug("Unpersisting old RDDs: " + oldRDDs.values.map(_.id).mkString(", ")) + logDebug(s"Unpersisting old RDDs: ${oldRDDs.values.map(_.id).mkString(", ")}") oldRDDs.values.foreach { rdd => rdd.unpersist(false) // Explicitly remove blocks of BlockRDD rdd match { case b: BlockRDD[_] => - logInfo("Removing blocks of RDD " + b + " of time " + time) + logInfo(s"Removing blocks of RDD $b of time $time") b.removeBlocks() case _ => } } } - logDebug("Cleared " + oldRDDs.size + " RDDs that were older than " + - (time - rememberDuration) + ": " + oldRDDs.keys.mkString(", ")) + logDebug(s"Cleared ${oldRDDs.size} RDDs that were older than " + + s"${time - rememberDuration}: ${oldRDDs.keys.mkString(", ")}") dependencies.foreach(_.clearMetadata(time)) } @@ -462,10 +478,10 @@ abstract class DStream[T: ClassTag] ( * this method to save custom checkpoint data. */ private[streaming] def updateCheckpointData(currentTime: Time) { - logDebug("Updating checkpoint data for time " + currentTime) + logDebug(s"Updating checkpoint data for time $currentTime") checkpointData.update(currentTime) dependencies.foreach(_.updateCheckpointData(currentTime)) - logDebug("Updated checkpoint data for time " + currentTime + ": " + checkpointData) + logDebug(s"Updated checkpoint data for time $currentTime: $checkpointData") } private[streaming] def clearCheckpointData(time: Time) { @@ -482,22 +498,25 @@ abstract class DStream[T: ClassTag] ( * override the updateCheckpointData() method would also need to override this method. */ private[streaming] def restoreCheckpointData() { - // Create RDDs from the checkpoint data - logInfo("Restoring checkpoint data") - checkpointData.restore() - dependencies.foreach(_.restoreCheckpointData()) - logInfo("Restored checkpoint data") + if (!restoredFromCheckpointData) { + // Create RDDs from the checkpoint data + logInfo("Restoring checkpoint data") + checkpointData.restore() + dependencies.foreach(_.restoreCheckpointData()) + restoredFromCheckpointData = true + logInfo("Restored checkpoint data") + } } @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { - logDebug(this.getClass().getSimpleName + ".writeObject used") + logDebug(s"${this.getClass().getSimpleName}.writeObject used") if (graph != null) { graph.synchronized { if (graph.checkpointInProgress) { oos.defaultWriteObject() } else { - val msg = "Object of " + this.getClass.getName + " is being serialized " + + val msg = s"Object of ${this.getClass.getName} is being serialized " + " possibly as a part of closure of an RDD operation. This is because " + " the DStream object is being referred to from within the closure. " + " Please rewrite the RDD operation inside this DStream to avoid this. " + @@ -514,9 +533,9 @@ abstract class DStream[T: ClassTag] ( @throws(classOf[IOException]) private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { - logDebug(this.getClass().getSimpleName + ".readObject used") + logDebug(s"${this.getClass().getSimpleName}.readObject used") ois.defaultReadObject() - generatedRDDs = new HashMap[Time, RDD[T]] () + generatedRDDs = new HashMap[Time, RDD[T]]() } // ======================================================================= @@ -532,7 +551,7 @@ abstract class DStream[T: ClassTag] ( * Return a new DStream by applying a function to all elements of this DStream, * and then flattening the results */ - def flatMap[U: ClassTag](flatMapFunc: T => Traversable[U]): DStream[U] = ssc.withScope { + def flatMap[U: ClassTag](flatMapFunc: T => TraversableOnce[U]): DStream[U] = ssc.withScope { new FlatMappedDStream(this, context.sparkContext.clean(flatMapFunc)) } @@ -575,7 +594,7 @@ abstract class DStream[T: ClassTag] ( * of this DStream. */ def reduce(reduceFunc: (T, T) => T): DStream[T] = ssc.withScope { - this.map(x => (null, x)).reduceByKey(reduceFunc, 1).map(_._2) + this.map((null, _)).reduceByKey(reduceFunc, 1).map(_._2) } /** @@ -597,29 +616,7 @@ abstract class DStream[T: ClassTag] ( */ def countByValue(numPartitions: Int = ssc.sc.defaultParallelism)(implicit ord: Ordering[T] = null) : DStream[(T, Long)] = ssc.withScope { - this.map(x => (x, 1L)).reduceByKey((x: Long, y: Long) => x + y, numPartitions) - } - - /** - * Apply a function to each RDD in this DStream. This is an output operator, so - * 'this' DStream will be registered as an output stream and therefore materialized. - * - * @deprecated As of 0.9.0, replaced by `foreachRDD`. - */ - @deprecated("use foreachRDD", "0.9.0") - def foreach(foreachFunc: RDD[T] => Unit): Unit = ssc.withScope { - this.foreachRDD(foreachFunc) - } - - /** - * Apply a function to each RDD in this DStream. This is an output operator, so - * 'this' DStream will be registered as an output stream and therefore materialized. - * - * @deprecated As of 0.9.0, replaced by `foreachRDD`. - */ - @deprecated("use foreachRDD", "0.9.0") - def foreach(foreachFunc: (RDD[T], Time) => Unit): Unit = ssc.withScope { - this.foreachRDD(foreachFunc) + this.map((_, 1L)).reduceByKey((x: Long, y: Long) => x + y, numPartitions) } /** @@ -628,7 +625,7 @@ abstract class DStream[T: ClassTag] ( */ def foreachRDD(foreachFunc: RDD[T] => Unit): Unit = ssc.withScope { val cleanedF = context.sparkContext.clean(foreachFunc, false) - this.foreachRDD((r: RDD[T], t: Time) => cleanedF(r)) + foreachRDD((r: RDD[T], _: Time) => cleanedF(r), displayInnerRDDOps = true) } /** @@ -639,7 +636,23 @@ abstract class DStream[T: ClassTag] ( // because the DStream is reachable from the outer object here, and because // DStreams can't be serialized with closures, we can't proactively check // it for serializability and so we pass the optional false to SparkContext.clean - new ForEachDStream(this, context.sparkContext.clean(foreachFunc, false)).register() + foreachRDD(foreachFunc, displayInnerRDDOps = true) + } + + /** + * Apply a function to each RDD in this DStream. This is an output operator, so + * 'this' DStream will be registered as an output stream and therefore materialized. + * @param foreachFunc foreachRDD function + * @param displayInnerRDDOps Whether the detailed callsites and scopes of the RDDs generated + * in the `foreachFunc` to be displayed in the UI. If `false`, then + * only the scopes and callsites of `foreachRDD` will override those + * of the RDDs on the display. + */ + private def foreachRDD( + foreachFunc: (RDD[T], Time) => Unit, + displayInnerRDDOps: Boolean): Unit = { + new ForEachDStream(this, + context.sparkContext.clean(foreachFunc, false), displayInnerRDDOps).register() } /** @@ -651,7 +664,7 @@ abstract class DStream[T: ClassTag] ( // DStreams can't be serialized with closures, we can't proactively check // it for serializability and so we pass the optional false to SparkContext.clean val cleanedF = context.sparkContext.clean(transformFunc, false) - transform((r: RDD[T], t: Time) => cleanedF(r)) + transform((r: RDD[T], _: Time) => cleanedF(r)) } /** @@ -722,7 +735,7 @@ abstract class DStream[T: ClassTag] ( val firstNum = rdd.take(num + 1) // scalastyle:off println println("-------------------------------------------") - println("Time: " + time) + println(s"Time: $time") println("-------------------------------------------") firstNum.take(num).foreach(println) if (firstNum.length > num) println("...") @@ -730,7 +743,7 @@ abstract class DStream[T: ClassTag] ( // scalastyle:on println } } - new ForEachDStream(this, context.sparkContext.clean(foreachFunc)).register() + foreachRDD(context.sparkContext.clean(foreachFunc), displayInnerRDDOps = false) } /** @@ -757,7 +770,7 @@ abstract class DStream[T: ClassTag] ( /** * Return a new DStream in which each RDD has a single element generated by reducing all * elements in a sliding window over this DStream. - * @param reduceFunc associative reduce function + * @param reduceFunc associative and commutative reduce function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which @@ -780,8 +793,9 @@ abstract class DStream[T: ClassTag] ( * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) * This is more efficient than reduceByWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". - * @param reduceFunc associative reduce function - * @param invReduceFunc inverse reduce function + * @param reduceFunc associative and commutative reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which @@ -794,7 +808,7 @@ abstract class DStream[T: ClassTag] ( windowDuration: Duration, slideDuration: Duration ): DStream[T] = ssc.withScope { - this.map(x => (1, x)) + this.map((1, _)) .reduceByKeyAndWindow(reduceFunc, invReduceFunc, windowDuration, slideDuration, 1) .map(_._2) } @@ -833,7 +847,7 @@ abstract class DStream[T: ClassTag] ( numPartitions: Int = ssc.sc.defaultParallelism) (implicit ord: Ordering[T] = null) : DStream[(T, Long)] = ssc.withScope { - this.map(x => (x, 1L)).reduceByKeyAndWindow( + this.map((_, 1L)).reduceByKeyAndWindow( (x: Long, y: Long) => x + y, (x: Long, y: Long) => x - y, windowDuration, @@ -869,25 +883,23 @@ abstract class DStream[T: ClassTag] ( val alignedToTime = if ((toTime - zeroTime).isMultipleOf(slideDuration)) { toTime } else { - logWarning("toTime (" + toTime + ") is not a multiple of slideDuration (" - + slideDuration + ")") - toTime.floor(slideDuration, zeroTime) + logWarning(s"toTime ($toTime) is not a multiple of slideDuration ($slideDuration)") + toTime.floor(slideDuration, zeroTime) } val alignedFromTime = if ((fromTime - zeroTime).isMultipleOf(slideDuration)) { fromTime } else { - logWarning("fromTime (" + fromTime + ") is not a multiple of slideDuration (" - + slideDuration + ")") + logWarning(s"fromTime ($fromTime) is not a multiple of slideDuration ($slideDuration)") fromTime.floor(slideDuration, zeroTime) } - logInfo("Slicing from " + fromTime + " to " + toTime + - " (aligned to " + alignedFromTime + " and " + alignedToTime + ")") + logInfo(s"Slicing from $fromTime to $toTime" + + s" (aligned to $alignedFromTime and $alignedToTime)") - alignedFromTime.to(alignedToTime, slideDuration).flatMap(time => { + alignedFromTime.to(alignedToTime, slideDuration).flatMap { time => if (time >= zeroTime) getOrCompute(time) else None - }) + } } /** @@ -900,7 +912,7 @@ abstract class DStream[T: ClassTag] ( val file = rddToFileName(prefix, suffix, time) rdd.saveAsObjectFile(file) } - this.foreachRDD(saveFunc) + this.foreachRDD(saveFunc, displayInnerRDDOps = false) } /** @@ -913,7 +925,7 @@ abstract class DStream[T: ClassTag] ( val file = rddToFileName(prefix, suffix, time) rdd.saveAsTextFile(file) } - this.foreachRDD(saveFunc) + this.foreachRDD(saveFunc, displayInnerRDDOps = false) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala index 39fd21342813..e73837eb9602 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala @@ -17,17 +17,19 @@ package org.apache.spark.streaming.dstream +import java.io.{IOException, ObjectInputStream, ObjectOutputStream} + import scala.collection.mutable.HashMap import scala.reflect.ClassTag -import java.io.{ObjectOutputStream, ObjectInputStream, IOException} -import org.apache.hadoop.fs.Path -import org.apache.hadoop.fs.FileSystem -import org.apache.spark.Logging + +import org.apache.hadoop.fs.{FileSystem, Path} + +import org.apache.spark.internal.Logging import org.apache.spark.streaming.Time import org.apache.spark.util.Utils private[streaming] -class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) +class DStreamCheckpointData[T: ClassTag](dstream: DStream[T]) extends Serializable with Logging { protected val data = new HashMap[Time, AnyRef]() @@ -37,13 +39,13 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) // in that batch's checkpoint data @transient private var timeToOldestCheckpointFileTime = new HashMap[Time, Time] - @transient private var fileSystem : FileSystem = null + @transient private var fileSystem: FileSystem = null protected[streaming] def currentCheckpointFiles = data.asInstanceOf[HashMap[Time, String]] /** * Updates the checkpoint data of the DStream. This gets called every time * the graph checkpoint is initiated. Default implementation records the - * checkpoint files to which the generate RDDs of the DStream has been saved. + * checkpoint files at which the generated RDDs of the DStream have been saved. */ def update(time: Time) { @@ -101,16 +103,15 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) /** * Restore the checkpoint data. This gets called once when the DStream graph - * (along with its DStreams) are being restored from a graph checkpoint file. + * (along with its output DStreams) is being restored from a graph checkpoint file. * Default implementation restores the RDDs from their checkpoint files. */ def restore() { // Create RDDs from the checkpoint data currentCheckpointFiles.foreach { - case(time, file) => { + case(time, file) => logInfo("Restoring checkpointed RDD for time " + time + " from file '" + file + "'") dstream.generatedRDDs += ((time, dstream.context.sparkContext.checkpointFile[T](file))) - } } } 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 40208a64861f..b8a5a96faf15 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 @@ -42,6 +42,7 @@ import org.apache.spark.util.{SerializableConfiguration, TimeStampedHashMap, Uti * class remembers the information about the files selected in past batches for * a certain duration (say, "remember window") as shown in the figure below. * + * {{{ * |<----- remember window ----->| * ignore threshold --->| |<--- current batch time * |____.____.____.____.____.____| @@ -49,6 +50,7 @@ import org.apache.spark.util.{SerializableConfiguration, TimeStampedHashMap, Uti * ---------------------|----|----|----|----|----|----|-----------------------> Time * |____|____|____|____|____|____| * remembered batches + * }}} * * The trailing end of the window is the "ignore threshold" and all files whose mod times * are less than this threshold are assumed to have already been selected and are therefore @@ -59,24 +61,25 @@ import org.apache.spark.util.{SerializableConfiguration, TimeStampedHashMap, Uti * `isNewFile` for more details. * * This makes some assumptions from the underlying file system that the system is monitoring. - * - The clock of the file system is assumed to synchronized with the clock of the machine running - * the streaming app. - * - If a file is to be visible in the directory listings, it must be visible within a certain - * duration of the mod time of the file. This duration is the "remember window", which is set to - * 1 minute (see `FileInputDStream.minRememberDuration`). Otherwise, the file will never be - * selected as the mod time will be less than the ignore threshold when it becomes visible. - * - Once a file is visible, the mod time cannot change. If it does due to appends, then the - * processing semantics are undefined. + * + * - The clock of the file system is assumed to synchronized with the clock of the machine running + * the streaming app. + * - If a file is to be visible in the directory listings, it must be visible within a certain + * duration of the mod time of the file. This duration is the "remember window", which is set to + * 1 minute (see `FileInputDStream.minRememberDuration`). Otherwise, the file will never be + * selected as the mod time will be less than the ignore threshold when it becomes visible. + * - Once a file is visible, the mod time cannot change. If it does due to appends, then the + * processing semantics are undefined. */ private[streaming] class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( - ssc_ : StreamingContext, + _ssc: StreamingContext, directory: String, filter: Path => Boolean = FileInputDStream.defaultFilter, newFilesOnly: Boolean = true, conf: Option[Configuration] = None) (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]) - extends InputDStream[(K, V)](ssc_) { + extends InputDStream[(K, V)](_ssc) { private val serializableConfOpt = conf.map(new SerializableConfiguration(_)) @@ -114,7 +117,7 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( // Map of batch-time to selected file info for the remembered batches // This is a concurrent map because it's also accessed in unit tests @transient private[streaming] var batchTimeToSelectedFiles = - new mutable.HashMap[Time, Array[String]] with mutable.SynchronizedMap[Time, Array[String]] + new mutable.HashMap[Time, Array[String]] // Set of files that were selected in the remembered batches @transient private var recentlySelectedFiles = new mutable.HashSet[String]() @@ -125,8 +128,8 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( // Timestamp of the last round of finding files @transient private var lastNewFileFindingTime = 0L - @transient private var path_ : Path = null - @transient private var fs_ : FileSystem = null + @transient private var _path: Path = null + @transient private var _fs: FileSystem = null override def start() { } @@ -145,7 +148,9 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( // Find new files val newFiles = findNewFiles(validTime.milliseconds) logInfo("New files at time " + validTime + ":\n" + newFiles.mkString("\n")) - batchTimeToSelectedFiles += ((validTime, newFiles)) + batchTimeToSelectedFiles.synchronized { + batchTimeToSelectedFiles += ((validTime, newFiles)) + } recentlySelectedFiles ++= newFiles val rdds = Some(filesToRDD(newFiles)) // Copy newFiles to immutable.List to prevent from being modified by the user @@ -160,13 +165,15 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( /** Clear the old time-to-files mappings along with old RDDs */ protected[streaming] override def clearMetadata(time: Time) { super.clearMetadata(time) - val oldFiles = batchTimeToSelectedFiles.filter(_._1 < (time - rememberDuration)) - batchTimeToSelectedFiles --= oldFiles.keys - recentlySelectedFiles --= oldFiles.values.flatten - logInfo("Cleared " + oldFiles.size + " old files that were older than " + - (time - rememberDuration) + ": " + oldFiles.keys.mkString(", ")) - logDebug("Cleared files are:\n" + - oldFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n")) + batchTimeToSelectedFiles.synchronized { + val oldFiles = batchTimeToSelectedFiles.filter(_._1 < (time - rememberDuration)) + batchTimeToSelectedFiles --= oldFiles.keys + recentlySelectedFiles --= oldFiles.values.flatten + logInfo("Cleared " + oldFiles.size + " old files that were older than " + + (time - rememberDuration) + ": " + oldFiles.keys.mkString(", ")) + logDebug("Cleared files are:\n" + + oldFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n")) + } // Delete file mod times that weren't accessed in the last round of getting new files fileToModTime.clearOldValues(lastNewFileFindingTime - 1) } @@ -189,10 +196,16 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( ) logDebug(s"Getting new files for time $currentTime, " + s"ignoring files older than $modTimeIgnoreThreshold") - val filter = new PathFilter { + + val newFileFilter = new PathFilter { def accept(path: Path): Boolean = isNewFile(path, currentTime, modTimeIgnoreThreshold) } - val newFiles = fs.listStatus(directoryPath, filter).map(_.getPath.toString) + val directoryFilter = new PathFilter { + override def accept(path: Path): Boolean = fs.getFileStatus(path).isDirectory + } + val directories = fs.globStatus(directoryPath, directoryFilter).map(_.getPath) + val newFiles = directories.flatMap(dir => + fs.listStatus(dir, newFileFilter).map(_.getPath.toString)) val timeTaken = clock.getTimeMillis() - lastNewFileFindingTime logInfo("Finding new files took " + timeTaken + " ms") logDebug("# cached file times = " + fileToModTime.size) @@ -218,7 +231,7 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( * - It must pass the user-provided file filter. * - It must be newer than the ignore threshold. It is assumed that files older than the ignore * threshold have already been considered or are existing files before start - * (when newFileOnly = true). + * (when newFilesOnly = true). * - It must not be present in the recently selected files that this class remembers. * - It must not be newer than the time of the batch (i.e. `currentTime` for which this * file is being tested. This can occur if the driver was recovered, and the missing batches @@ -270,7 +283,7 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( config) case None => context.sparkContext.newAPIHadoopFile[K, V, F](file) } - if (rdd.partitions.size == 0) { + if (rdd.partitions.isEmpty) { logError("File " + file + " has no data in it. Spark Streaming can only ingest " + "files that have been \"moved\" to the directory assigned to the file stream. " + "Refer to the streaming programming guide for more details.") @@ -286,17 +299,17 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( } private def directoryPath: Path = { - if (path_ == null) path_ = new Path(directory) - path_ + if (_path == null) _path = new Path(directory) + _path } private def fs: FileSystem = { - if (fs_ == null) fs_ = directoryPath.getFileSystem(ssc.sparkContext.hadoopConfiguration) - fs_ + if (_fs == null) _fs = directoryPath.getFileSystem(ssc.sparkContext.hadoopConfiguration) + _fs } private def reset() { - fs_ = null + _fs = null } @throws(classOf[IOException]) @@ -304,8 +317,7 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( logDebug(this.getClass().getSimpleName + ".readObject used") ois.defaultReadObject() generatedRDDs = new mutable.HashMap[Time, RDD[(K, V)]]() - batchTimeToSelectedFiles = - new mutable.HashMap[Time, Array[String]] with mutable.SynchronizedMap[Time, Array[String]] + batchTimeToSelectedFiles = new mutable.HashMap[Time, Array[String]] recentlySelectedFiles = new mutable.HashSet[String]() fileToModTime = new TimeStampedHashMap[String, Long](true) } @@ -321,21 +333,20 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( override def update(time: Time) { hadoopFiles.clear() - hadoopFiles ++= batchTimeToSelectedFiles + batchTimeToSelectedFiles.synchronized { hadoopFiles ++= batchTimeToSelectedFiles } } override def cleanup(time: Time) { } override def restore() { hadoopFiles.toSeq.sortBy(_._1)(Time.ordering).foreach { - case (t, f) => { + case (t, f) => // Restore the metadata in both files and generatedRDDs logInfo("Restoring files for time " + t + " - " + f.mkString("[", ", ", "]") ) - batchTimeToSelectedFiles += ((t, f)) + batchTimeToSelectedFiles.synchronized { batchTimeToSelectedFiles += ((t, f)) } recentlySelectedFiles ++= f generatedRDDs += ((t, filesToRDD(f))) - } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala index fcd5216f101a..43079880b235 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala @@ -17,10 +17,11 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.{Duration, Time} -import org.apache.spark.rdd.RDD import scala.reflect.ClassTag +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Duration, Time} + private[streaming] class FilteredDStream[T: ClassTag]( parent: DStream[T], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala index 9d09a3baf37c..f5b1e5f3a145 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala @@ -17,11 +17,11 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.{Duration, Time} -import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext._ import scala.reflect.ClassTag +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Duration, Time} + private[streaming] class FlatMapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag]( parent: DStream[(K, V)], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala index 475ea2d2d4f3..d60a6179782e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala @@ -17,14 +17,15 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.{Duration, Time} -import org.apache.spark.rdd.RDD import scala.reflect.ClassTag +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Duration, Time} + private[streaming] class FlatMappedDStream[T: ClassTag, U: ClassTag]( parent: DStream[T], - flatMapFunc: T => Traversable[U] + flatMapFunc: T => TraversableOnce[U] ) extends DStream[U](parent.ssc) { override def dependencies: List[DStream[_]] = List(parent) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala index c109ceccc698..a0fadee8a984 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala @@ -17,15 +17,25 @@ package org.apache.spark.streaming.dstream +import scala.reflect.ClassTag + import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Duration, Time} import org.apache.spark.streaming.scheduler.Job -import scala.reflect.ClassTag +/** + * An internal DStream used to represent output operations like DStream.foreachRDD. + * @param parent Parent DStream + * @param foreachFunc Function to apply on each RDD generated by the parent DStream + * @param displayInnerRDDOps Whether the detailed callsites and scopes of the RDDs generated + * by `foreachFunc` will be displayed in the UI; only the scope and + * callsite of `DStream.foreachRDD` will be displayed. + */ private[streaming] class ForEachDStream[T: ClassTag] ( parent: DStream[T], - foreachFunc: (RDD[T], Time) => Unit + foreachFunc: (RDD[T], Time) => Unit, + displayInnerRDDOps: Boolean ) extends DStream[Unit](parent.ssc) { override def dependencies: List[DStream[_]] = List(parent) @@ -37,8 +47,7 @@ class ForEachDStream[T: ClassTag] ( override def generateJob(time: Time): Option[Job] = { parent.getOrCompute(time) match { case Some(rdd) => - val jobFunc = () => createRDDWithLocalProperties(time) { - ssc.sparkContext.setCallSite(creationSite) + val jobFunc = () => createRDDWithLocalProperties(time, displayInnerRDDOps) { foreachFunc(rdd, time) } Some(new Job(time, jobFunc)) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala index dbb295fe54f7..9f1252f091a6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala @@ -17,10 +17,11 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.{Duration, Time} -import org.apache.spark.rdd.RDD import scala.reflect.ClassTag +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Duration, Time} + private[streaming] class GlommedDStream[T: ClassTag](parent: DStream[T]) extends DStream[Array[T]](parent.ssc) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala index 95994c983c0c..931f015f03b6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala @@ -17,18 +17,20 @@ package org.apache.spark.streaming.dstream +import java.util.Locale + import scala.reflect.ClassTag import org.apache.spark.SparkContext import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.streaming.{Duration, StreamingContext, Time} import org.apache.spark.streaming.scheduler.RateController -import org.apache.spark.streaming.scheduler.rate.RateEstimator import org.apache.spark.util.Utils /** * This is the abstract base class for all input streams. This class provides methods - * start() and stop() which is called by Spark Streaming system to start and stop receiving data. + * start() and stop() which are called by Spark Streaming system to start and stop + * receiving data, respectively. * Input streams that can generate RDDs from new data by running a service/thread only on * the driver node (that is, without running a receiver on worker nodes), can be * implemented by directly inheriting this InputDStream. For example, @@ -37,10 +39,10 @@ import org.apache.spark.util.Utils * that requires running a receiver on the worker nodes, use * [[org.apache.spark.streaming.dstream.ReceiverInputDStream]] as the parent class. * - * @param ssc_ Streaming context that will execute this input stream + * @param _ssc Streaming context that will execute this input stream */ -abstract class InputDStream[T: ClassTag] (ssc_ : StreamingContext) - extends DStream[T](ssc_) { +abstract class InputDStream[T: ClassTag](_ssc: StreamingContext) + extends DStream[T](_ssc) { private[streaming] var lastValidTime: Time = null @@ -60,7 +62,7 @@ abstract class InputDStream[T: ClassTag] (ssc_ : StreamingContext) .split("(?=[A-Z])") .filter(_.nonEmpty) .mkString(" ") - .toLowerCase + .toLowerCase(Locale.ROOT) .capitalize s"$newName [$id]" } @@ -74,7 +76,7 @@ abstract class InputDStream[T: ClassTag] (ssc_ : StreamingContext) protected[streaming] override val baseScope: Option[String] = { val scopeName = Option(ssc.sc.getLocalProperty(SparkContext.RDD_SCOPE_KEY)) .map { json => RDDOperationScope.fromJson(json).name + s" [$id]" } - .getOrElse(name.toLowerCase) + .getOrElse(name.toLowerCase(Locale.ROOT)) Some(new RDDOperationScope(scopeName).toJson) } @@ -88,10 +90,10 @@ abstract class InputDStream[T: ClassTag] (ssc_ : StreamingContext) if (!super.isTimeValid(time)) { false // Time not valid } else { - // Time is valid, but check it it is more than lastValidTime + // Time is valid, but check it is more than lastValidTime if (lastValidTime != null && time < lastValidTime) { - logWarning("isTimeValid called with " + time + " where as last valid time is " + - lastValidTime) + logWarning(s"isTimeValid called with $time whereas the last valid time " + + s"is $lastValidTime") } lastValidTime = time true @@ -107,8 +109,8 @@ abstract class InputDStream[T: ClassTag] (ssc_ : StreamingContext) } /** Method called to start receiving data. Subclasses must implement this method. */ - def start() + def start(): Unit /** Method called to stop receiving data. Subclasses must implement this method. */ - def stop() + def stop(): Unit } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala index 5994bc1e23f2..bcdf1752e61e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala @@ -17,10 +17,11 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.{Duration, Time} -import org.apache.spark.rdd.RDD import scala.reflect.ClassTag +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Duration, Time} + private[streaming] class MapPartitionedDStream[T: ClassTag, U: ClassTag]( parent: DStream[T], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala index 954d2eb4a7b0..c209f86c864a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala @@ -17,11 +17,11 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.{Duration, Time} -import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext._ import scala.reflect.ClassTag +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Duration, Time} + private[streaming] class MapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag]( parent: DStream[(K, V)], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapWithStateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapWithStateDStream.scala new file mode 100644 index 000000000000..9512db7d7d75 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapWithStateDStream.scala @@ -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. + */ + +package org.apache.spark.streaming.dstream + +import scala.reflect.ClassTag + +import org.apache.spark._ +import org.apache.spark.annotation.Experimental +import org.apache.spark.rdd.{EmptyRDD, RDD} +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming._ +import org.apache.spark.streaming.dstream.InternalMapWithStateDStream._ +import org.apache.spark.streaming.rdd.{MapWithStateRDD, MapWithStateRDDRecord} + +/** + * :: Experimental :: + * DStream representing the stream of data generated by `mapWithState` operation on a + * [[org.apache.spark.streaming.dstream.PairDStreamFunctions pair DStream]]. + * Additionally, it also gives access to the stream of state snapshots, that is, the state data of + * all keys after a batch has updated them. + * + * @tparam KeyType Class of the key + * @tparam ValueType Class of the value + * @tparam StateType Class of the state data + * @tparam MappedType Class of the mapped data + */ +@Experimental +sealed abstract class MapWithStateDStream[KeyType, ValueType, StateType, MappedType: ClassTag]( + ssc: StreamingContext) extends DStream[MappedType](ssc) { + + /** Return a pair DStream where each RDD is the snapshot of the state of all the keys. */ + def stateSnapshots(): DStream[(KeyType, StateType)] +} + +/** Internal implementation of the [[MapWithStateDStream]] */ +private[streaming] class MapWithStateDStreamImpl[ + KeyType: ClassTag, ValueType: ClassTag, StateType: ClassTag, MappedType: ClassTag]( + dataStream: DStream[(KeyType, ValueType)], + spec: StateSpecImpl[KeyType, ValueType, StateType, MappedType]) + extends MapWithStateDStream[KeyType, ValueType, StateType, MappedType](dataStream.context) { + + private val internalStream = + new InternalMapWithStateDStream[KeyType, ValueType, StateType, MappedType](dataStream, spec) + + override def slideDuration: Duration = internalStream.slideDuration + + override def dependencies: List[DStream[_]] = List(internalStream) + + override def compute(validTime: Time): Option[RDD[MappedType]] = { + internalStream.getOrCompute(validTime).map { _.flatMap[MappedType] { _.mappedData } } + } + + /** + * Forward the checkpoint interval to the internal DStream that computes the state maps. This + * to make sure that this DStream does not get checkpointed, only the internal stream. + */ + override def checkpoint(checkpointInterval: Duration): DStream[MappedType] = { + internalStream.checkpoint(checkpointInterval) + this + } + + /** Return a pair DStream where each RDD is the snapshot of the state of all the keys. */ + def stateSnapshots(): DStream[(KeyType, StateType)] = { + internalStream.flatMap { + _.stateMap.getAll().map { case (k, s, _) => (k, s) }.toTraversable } + } + + def keyClass: Class[_] = implicitly[ClassTag[KeyType]].runtimeClass + + def valueClass: Class[_] = implicitly[ClassTag[ValueType]].runtimeClass + + def stateClass: Class[_] = implicitly[ClassTag[StateType]].runtimeClass + + def mappedClass: Class[_] = implicitly[ClassTag[MappedType]].runtimeClass +} + +/** + * A DStream that allows per-key state to be maintained, and arbitrary records to be generated + * based on updates to the state. This is the main DStream that implements the `mapWithState` + * operation on DStreams. + * + * @param parent Parent (key, value) stream that is the source + * @param spec Specifications of the mapWithState operation + * @tparam K Key type + * @tparam V Value type + * @tparam S Type of the state maintained + * @tparam E Type of the mapped data + */ +private[streaming] +class InternalMapWithStateDStream[K: ClassTag, V: ClassTag, S: ClassTag, E: ClassTag]( + parent: DStream[(K, V)], spec: StateSpecImpl[K, V, S, E]) + extends DStream[MapWithStateRDDRecord[K, S, E]](parent.context) { + + persist(StorageLevel.MEMORY_ONLY) + + private val partitioner = spec.getPartitioner().getOrElse( + new HashPartitioner(ssc.sc.defaultParallelism)) + + private val mappingFunction = spec.getFunction() + + override def slideDuration: Duration = parent.slideDuration + + override def dependencies: List[DStream[_]] = List(parent) + + /** Enable automatic checkpointing */ + override val mustCheckpoint = true + + /** Override the default checkpoint duration */ + override def initialize(time: Time): Unit = { + if (checkpointDuration == null) { + checkpointDuration = slideDuration * DEFAULT_CHECKPOINT_DURATION_MULTIPLIER + } + super.initialize(time) + } + + /** Method that generates an RDD for the given time */ + override def compute(validTime: Time): Option[RDD[MapWithStateRDDRecord[K, S, E]]] = { + // Get the previous state or create a new empty state RDD + val prevStateRDD = getOrCompute(validTime - slideDuration) match { + case Some(rdd) => + if (rdd.partitioner != Some(partitioner)) { + // If the RDD is not partitioned the right way, let us repartition it using the + // partition index as the key. This is to ensure that state RDD is always partitioned + // before creating another state RDD using it + MapWithStateRDD.createFromRDD[K, V, S, E]( + rdd.flatMap { _.stateMap.getAll() }, partitioner, validTime) + } else { + rdd + } + case None => + MapWithStateRDD.createFromPairRDD[K, V, S, E]( + spec.getInitialStateRDD().getOrElse(new EmptyRDD[(K, S)](ssc.sparkContext)), + partitioner, + validTime + ) + } + + + // Compute the new state RDD with previous state RDD and partitioned data RDD + // Even if there is no data RDD, use an empty one to create a new state RDD + val dataRDD = parent.getOrCompute(validTime).getOrElse { + context.sparkContext.emptyRDD[(K, V)] + } + val partitionedDataRDD = dataRDD.partitionBy(partitioner) + val timeoutThresholdTime = spec.getTimeoutInterval().map { interval => + (validTime - interval).milliseconds + } + Some(new MapWithStateRDD( + prevStateRDD, partitionedDataRDD, mappingFunction, validTime, timeoutThresholdTime)) + } +} + +private[streaming] object InternalMapWithStateDStream { + private val DEFAULT_CHECKPOINT_DURATION_MULTIPLIER = 10 +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala index fa14b2e897c3..e11d82697af8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala @@ -17,10 +17,11 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.{Duration, Time} -import org.apache.spark.rdd.RDD import scala.reflect.ClassTag +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Duration, Time} + private[streaming] class MappedDStream[T: ClassTag, U: ClassTag] ( parent: DStream[T], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 71bec96d46c8..dcb51d72fa58 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -25,8 +25,9 @@ import org.apache.hadoop.mapred.{JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.spark.{HashPartitioner, Partitioner} +import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.{Duration, Time} +import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext.rddToFileName import org.apache.spark.util.{SerializableConfiguration, SerializableJobConf} @@ -35,8 +36,7 @@ import org.apache.spark.util.{SerializableConfiguration, SerializableJobConf} */ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K]) - extends Serializable -{ + extends Serializable { private[streaming] def ssc = self.ssc private[streaming] def sparkContext = self.context.sparkContext @@ -75,8 +75,8 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are - * merged using the associative reduce function. Hash partitioning is used to generate the RDDs - * with Spark's default number of partitions. + * merged using the associative and commutative reduce function. Hash partitioning is used to + * generate the RDDs with Spark's default number of partitions. */ def reduceByKey(reduceFunc: (V, V) => V): DStream[(K, V)] = ssc.withScope { reduceByKey(reduceFunc, defaultPartitioner()) @@ -204,7 +204,7 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) * Similar to `DStream.reduceByKey()`, but applies it over a sliding window. The new DStream * generates RDDs with the same interval as this DStream. Hash partitioning is used to generate * the RDDs with Spark's default number of partitions. - * @param reduceFunc associative reduce function + * @param reduceFunc associative and commutative reduce function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval */ @@ -219,7 +219,7 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. - * @param reduceFunc associative reduce function + * @param reduceFunc associative and commutative reduce function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which @@ -238,7 +238,7 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. - * @param reduceFunc associative reduce function + * @param reduceFunc associative and commutative reduce function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which @@ -259,7 +259,7 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) /** * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to * `DStream.reduceByKey()`, but applies it over a sliding window. - * @param reduceFunc associative reduce function + * @param reduceFunc associative and commutative reduce function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which @@ -289,8 +289,9 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. - * @param reduceFunc associative reduce function - * @param invReduceFunc inverse reduce function + * @param reduceFunc associative and commutative reduce function + * @param invReduceFunc inverse reduce function; such that for all y, invertible x: + * `invReduceFunc(reduceFunc(x, y), x) = y` * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval * @param slideDuration sliding interval of the window (i.e., the interval after which @@ -320,7 +321,7 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function. * However, it is applicable to only "invertible reduce functions". - * @param reduceFunc associative reduce function + * @param reduceFunc associative and commutative reduce function * @param invReduceFunc inverse reduce function * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval @@ -350,9 +351,45 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) ) } + /** + * :: Experimental :: + * Return a [[MapWithStateDStream]] by applying a function to every key-value element of + * `this` stream, while maintaining some state data for each unique key. The mapping function + * and other specification (e.g. partitioners, timeouts, initial state data, etc.) of this + * transformation can be specified using `StateSpec` class. The state data is accessible in + * as a parameter of type `State` in the mapping function. + * + * Example of using `mapWithState`: + * {{{ + * // A mapping function that maintains an integer state and return a String + * def mappingFunction(key: String, value: Option[Int], state: State[Int]): Option[String] = { + * // Use state.exists(), state.get(), state.update() and state.remove() + * // to manage state, and return the necessary string + * } + * + * val spec = StateSpec.function(mappingFunction).numPartitions(10) + * + * val mapWithStateDStream = keyValueDStream.mapWithState[StateType, MappedType](spec) + * }}} + * + * @param spec Specification of this transformation + * @tparam StateType Class type of the state data + * @tparam MappedType Class type of the mapped data + */ + @Experimental + def mapWithState[StateType: ClassTag, MappedType: ClassTag]( + spec: StateSpec[K, V, StateType, MappedType] + ): MapWithStateDStream[K, V, StateType, MappedType] = { + new MapWithStateDStreamImpl[K, V, StateType, MappedType]( + self, + spec.asInstanceOf[StateSpecImpl[K, V, StateType, MappedType]] + ) + } + /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. + * In every batch the updateFunc will be called for each state even if there are no new values. * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. @@ -367,6 +404,7 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. + * In every batch the updateFunc will be called for each state even if there are no new values. * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. @@ -383,7 +421,8 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of the key. - * org.apache.spark.Partitioner is used to control the partitioning of each RDD. + * In every batch the updateFunc will be called for each state even if there are no new values. + * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. * @param partitioner Partitioner for controlling the partitioning of each RDD in the new @@ -404,27 +443,32 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. - * org.apache.spark.Partitioner is used to control the partitioning of each RDD. + * In every batch the updateFunc will be called for each state even if there are no new values. + * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. * @param updateFunc State update function. Note, that this function may generate a different * tuple with a different key than the input key. Therefore keys may be removed * or added in this way. It is up to the developer to decide whether to * remember the partitioner despite the key being changed. * @param partitioner Partitioner for controlling the partitioning of each RDD in the new * DStream - * @param rememberPartitioner Whether to remember the paritioner object in the generated RDDs. + * @param rememberPartitioner Whether to remember the partitioner object in the generated RDDs. * @tparam S State type */ def updateStateByKey[S: ClassTag]( updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)], partitioner: Partitioner, - rememberPartitioner: Boolean - ): DStream[(K, S)] = ssc.withScope { - new StateDStream(self, ssc.sc.clean(updateFunc), partitioner, rememberPartitioner, None) + rememberPartitioner: Boolean): DStream[(K, S)] = ssc.withScope { + val cleanedFunc = ssc.sc.clean(updateFunc) + val newUpdateFunc = (_: Time, it: Iterator[(K, Seq[V], Option[S])]) => { + cleanedFunc(it) + } + new StateDStream(self, newUpdateFunc, partitioner, rememberPartitioner, None) } /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of the key. + * In every batch the updateFunc will be called for each state even if there are no new values. * org.apache.spark.Partitioner is used to control the partitioning of each RDD. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. @@ -448,6 +492,7 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. + * In every batch the updateFunc will be called for each state even if there are no new values. * org.apache.spark.Partitioner is used to control the partitioning of each RDD. * @param updateFunc State update function. Note, that this function may generate a different * tuple with a different key than the input key. Therefore keys may be removed @@ -455,7 +500,7 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) * remember the partitioner despite the key being changed. * @param partitioner Partitioner for controlling the partitioning of each RDD in the new * DStream - * @param rememberPartitioner Whether to remember the paritioner object in the generated RDDs. + * @param rememberPartitioner Whether to remember the partitioner object in the generated RDDs. * @param initialRDD initial state value of each key. * @tparam S State type */ @@ -463,10 +508,34 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)], partitioner: Partitioner, rememberPartitioner: Boolean, - initialRDD: RDD[(K, S)] - ): DStream[(K, S)] = ssc.withScope { - new StateDStream(self, ssc.sc.clean(updateFunc), partitioner, - rememberPartitioner, Some(initialRDD)) + initialRDD: RDD[(K, S)]): DStream[(K, S)] = ssc.withScope { + val cleanedFunc = ssc.sc.clean(updateFunc) + val newUpdateFunc = (_: Time, it: Iterator[(K, Seq[V], Option[S])]) => { + cleanedFunc(it) + } + new StateDStream(self, newUpdateFunc, partitioner, rememberPartitioner, Some(initialRDD)) + } + + /** + * Return a new "state" DStream where the state for each key is updated by applying + * the given function on the previous state of the key and the new values of the key. + * In every batch the updateFunc will be called for each state even if there are no new values. + * org.apache.spark.Partitioner is used to control the partitioning of each RDD. + * @param updateFunc State update function. If `this` function returns None, then + * corresponding state key-value pair will be eliminated. + * @param partitioner Partitioner for controlling the partitioning of each RDD in the new + * DStream. + * @tparam S State type + */ + def updateStateByKey[S: ClassTag](updateFunc: (Time, K, Seq[V], Option[S]) => Option[S], + partitioner: Partitioner, + rememberPartitioner: Boolean, + initialRDD: Option[RDD[(K, S)]] = None): DStream[(K, S)] = ssc.withScope { + val cleanedFunc = ssc.sc.clean(updateFunc) + val newUpdateFunc = (time: Time, iterator: Iterator[(K, Seq[V], Option[S])]) => { + iterator.flatMap(t => cleanedFunc(time, t._1, t._2, t._3).map(s => (t._1, s))) + } + new StateDStream(self, newUpdateFunc, partitioner, rememberPartitioner, initialRDD) } /** @@ -692,7 +761,8 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) val serializableConf = new SerializableJobConf(conf) val saveFunc = (rdd: RDD[(K, V)], time: Time) => { val file = rddToFileName(prefix, suffix, time) - rdd.saveAsHadoopFile(file, keyClass, valueClass, outputFormatClass, serializableConf.value) + rdd.saveAsHadoopFile(file, keyClass, valueClass, outputFormatClass, + new JobConf(serializableConf.value)) } self.foreachRDD(saveFunc) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala index 002aac9f4361..e003ddb96c86 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala @@ -17,14 +17,15 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.StreamingContext import scala.reflect.ClassTag + +import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.receiver.Receiver private[streaming] class PluggableInputDStream[T: ClassTag]( - ssc_ : StreamingContext, - receiver: Receiver[T]) extends ReceiverInputDStream[T](ssc_) { + _ssc: StreamingContext, + receiver: Receiver[T]) extends ReceiverInputDStream[T](_ssc) { def getReceiver(): Receiver[T] = { receiver diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala index cd073646370d..f9c78699164a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.{ArrayBuffer, Queue} import scala.reflect.ClassTag import org.apache.spark.rdd.{RDD, UnionRDD} -import org.apache.spark.streaming.{Time, StreamingContext} +import org.apache.spark.streaming.{StreamingContext, Time} private[streaming] class QueueInputDStream[T: ClassTag]( @@ -48,12 +48,15 @@ class QueueInputDStream[T: ClassTag]( override def compute(validTime: Time): Option[RDD[T]] = { val buffer = new ArrayBuffer[RDD[T]]() - if (oneAtATime && queue.size > 0) { - buffer += queue.dequeue() - } else { - buffer ++= queue.dequeueAll(_ => true) + queue.synchronized { + if (oneAtATime && queue.nonEmpty) { + buffer += queue.dequeue() + } else { + buffer ++= queue + queue.clear() + } } - if (buffer.size > 0) { + if (buffer.nonEmpty) { if (oneAtATime) { Some(buffer.head) } else { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala index 5a9eda7c1277..b2ec33e82dda 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala @@ -17,19 +17,18 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.storage.{StorageLevel, StreamBlockId} -import org.apache.spark.streaming.StreamingContext - -import scala.reflect.ClassTag - +import java.io.EOFException import java.net.InetSocketAddress import java.nio.ByteBuffer import java.nio.channels.{ReadableByteChannel, SocketChannel} -import java.io.EOFException import java.util.concurrent.ArrayBlockingQueue -import org.apache.spark.streaming.receiver.Receiver +import scala.reflect.ClassTag + +import org.apache.spark.internal.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.receiver.Receiver /** * An input stream that reads blocks of serialized objects from a given network address. @@ -39,11 +38,11 @@ import org.apache.spark.streaming.receiver.Receiver */ private[streaming] class RawInputDStream[T: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, host: String, port: Int, storageLevel: StorageLevel - ) extends ReceiverInputDStream[T](ssc_ ) with Logging { + ) extends ReceiverInputDStream[T](_ssc) with Logging { def getReceiver(): Receiver[T] = { new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[Receiver[T]] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala index 87c20afd5c13..fd3e72e41be2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala @@ -21,25 +21,25 @@ import scala.reflect.ClassTag import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.BlockId +import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.scheduler.{RateController, ReceivedBlockInfo, StreamInputInfo} import org.apache.spark.streaming.scheduler.rate.RateEstimator -import org.apache.spark.streaming.scheduler.{ReceivedBlockInfo, RateController, StreamInputInfo} import org.apache.spark.streaming.util.WriteAheadLogUtils -import org.apache.spark.streaming.{StreamingContext, Time} /** * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] * that has to start a receiver on worker nodes to receive external data. * Specific implementations of ReceiverInputDStream must - * define `the getReceiver()` function that gets the receiver object of type + * define [[getReceiver]] function that gets the receiver object of type * [[org.apache.spark.streaming.receiver.Receiver]] that will be sent * to the workers to receive data. - * @param ssc_ Streaming context that will execute this input stream + * @param _ssc Streaming context that will execute this input stream * @tparam T Class type of the object of this stream */ -abstract class ReceiverInputDStream[T: ClassTag](ssc_ : StreamingContext) - extends InputDStream[T](ssc_) { +abstract class ReceiverInputDStream[T: ClassTag](_ssc: StreamingContext) + extends InputDStream[T](_ssc) { /** * Asynchronously maintains & sends new rate limits to the receiver through the receiver tracker. @@ -108,7 +108,7 @@ abstract class ReceiverInputDStream[T: ClassTag](ssc_ : StreamingContext) } else { // Else, create a BlockRDD. However, if there are some blocks with WAL info but not // others then that is unexpected and log a warning accordingly. - if (blockInfos.find(_.walRecordHandleOption.nonEmpty).nonEmpty) { + if (blockInfos.exists(_.walRecordHandleOption.nonEmpty)) { if (WriteAheadLogUtils.enableReceiverLog(ssc.conf)) { logError("Some blocks do not have Write Ahead Log information; " + "this is unexpected and data may not be recoverable after driver failures") @@ -119,9 +119,9 @@ abstract class ReceiverInputDStream[T: ClassTag](ssc_ : StreamingContext) val validBlockIds = blockIds.filter { id => ssc.sparkContext.env.blockManager.master.contains(id) } - if (validBlockIds.size != blockIds.size) { + if (validBlockIds.length != blockIds.length) { logWarning("Some blocks could not be recovered as they were not found in memory. " + - "To prevent such data loss, enabled Write Ahead Log (see programming guide " + + "To prevent such data loss, enable Write Ahead Log (see programming guide " + "for more details.") } new BlockRDD[T](ssc.sc, validBlockIds) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala index 6a583bf2a362..a9e93838b8bf 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -17,18 +17,14 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.{CoGroupedRDD, MapPartitionsRDD} +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + import org.apache.spark.Partitioner -import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.{CoGroupedRDD, RDD} import org.apache.spark.storage.StorageLevel - -import scala.collection.mutable.ArrayBuffer import org.apache.spark.streaming.{Duration, Interval, Time} -import scala.collection.mutable.ArrayBuffer -import scala.reflect.ClassTag - private[streaming] class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( parent: DStream[(K, V)], @@ -91,7 +87,7 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( logDebug("Window time = " + windowDuration) logDebug("Slide time = " + slideDuration) - logDebug("ZeroTime = " + zeroTime) + logDebug("Zero time = " + zeroTime) logDebug("Current window = " + currentWindow) logDebug("Previous window = " + previousWindow) @@ -132,7 +128,7 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( val numNewValues = newRDDs.size val mergeValues = (arrayOfValues: Array[Iterable[V]]) => { - if (arrayOfValues.size != 1 + numOldValues + numNewValues) { + if (arrayOfValues.length != 1 + numOldValues + numNewValues) { throw new Exception("Unexpected number of sequences of reduced values") } // Getting reduced values "old time steps" that will be removed from current window diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala index e0ffd5d86b43..6971a66b380d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala @@ -17,11 +17,11 @@ package org.apache.spark.streaming.dstream +import scala.reflect.ClassTag + import org.apache.spark.Partitioner import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext._ import org.apache.spark.streaming.{Duration, Time} -import scala.reflect.ClassTag private[streaming] class ShuffledDStream[K: ClassTag, V: ClassTag, C: ClassTag]( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index de84e0c9a498..7853af562368 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -17,27 +17,27 @@ package org.apache.spark.streaming.dstream -import scala.util.control.NonFatal - -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.NextIterator +import java.io._ +import java.net.{ConnectException, Socket} +import java.nio.charset.StandardCharsets import scala.reflect.ClassTag +import scala.util.control.NonFatal -import java.io._ -import java.net.{UnknownHostException, Socket} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.util.NextIterator private[streaming] class SocketInputDStream[T: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, host: String, port: Int, bytesToObjects: InputStream => Iterator[T], storageLevel: StorageLevel - ) extends ReceiverInputDStream[T](ssc_) { + ) extends ReceiverInputDStream[T](_ssc) { def getReceiver(): Receiver[T] = { new SocketReceiver(host, port, bytesToObjects, storageLevel) @@ -52,7 +52,20 @@ class SocketReceiver[T: ClassTag]( storageLevel: StorageLevel ) extends Receiver[T](storageLevel) with Logging { + private var socket: Socket = _ + def onStart() { + + logInfo(s"Connecting to $host:$port") + try { + socket = new Socket(host, port) + } catch { + case e: ConnectException => + restart(s"Error connecting to $host:$port", e) + return + } + logInfo(s"Connected to $host:$port") + // Start the thread that receives data over a connection new Thread("Socket Receiver") { setDaemon(true) @@ -61,20 +74,22 @@ class SocketReceiver[T: ClassTag]( } def onStop() { - // There is nothing much to do as the thread calling receive() - // is designed to stop by itself isStopped() returns false + // in case restart thread close it twice + synchronized { + if (socket != null) { + socket.close() + socket = null + logInfo(s"Closed socket to $host:$port") + } + } } /** Create a socket connection and receive data until receiver is stopped */ def receive() { - var socket: Socket = null try { - logInfo("Connecting to " + host + ":" + port) - socket = new Socket(host, port) - logInfo("Connected to " + host + ":" + port) val iterator = bytesToObjects(socket.getInputStream()) while(!isStopped && iterator.hasNext) { - store(iterator.next) + store(iterator.next()) } if (!isStopped()) { restart("Socket data stream had no more data") @@ -82,16 +97,11 @@ class SocketReceiver[T: ClassTag]( logInfo("Stopped receiving") } } catch { - case e: java.net.ConnectException => - restart("Error connecting to " + host + ":" + port, e) case NonFatal(e) => logWarning("Error receiving data", e) restart("Error receiving data", e) } finally { - if (socket != null) { - socket.close() - logInfo("Closed socket to " + host + ":" + port) - } + onStop() } } } @@ -104,7 +114,8 @@ object SocketReceiver { * to '\n' delimited strings and returns an iterator to access the strings. */ def bytesToLines(inputStream: InputStream): Iterator[String] = { - val dataInputStream = new BufferedReader(new InputStreamReader(inputStream, "UTF-8")) + val dataInputStream = new BufferedReader( + new InputStreamReader(inputStream, StandardCharsets.UTF_8)) new NextIterator[String] { protected override def getNext() = { val nextValue = dataInputStream.readLine() diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index 621d6dff788f..d1a5e9179370 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -17,21 +17,20 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag + import org.apache.spark.Partitioner -import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Duration, Time} -import scala.reflect.ClassTag - private[streaming] class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( parent: DStream[(K, V)], - updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)], + updateFunc: (Time, Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)], partitioner: Partitioner, preservePartitioning: Boolean, - initialRDD : Option[RDD[(K, S)]] + initialRDD: Option[RDD[(K, S)]] ) extends DStream[(K, S)](parent.ssc) { super.persist(StorageLevel.MEMORY_ONLY_SER) @@ -42,19 +41,21 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( override val mustCheckpoint = true - private [this] def computeUsingPreviousRDD ( - parentRDD : RDD[(K, V)], prevStateRDD : RDD[(K, S)]) = { + private [this] def computeUsingPreviousRDD( + batchTime: Time, + parentRDD: RDD[(K, V)], + prevStateRDD: RDD[(K, S)]) = { // Define the function for the mapPartition operation on cogrouped RDD; // first map the cogrouped tuple to tuples of required type, // and then apply the update function val updateFuncLocal = updateFunc val finalFunc = (iterator: Iterator[(K, (Iterable[V], Iterable[S]))]) => { - val i = iterator.map(t => { + val i = iterator.map { t => val itr = t._2._2.iterator val headOption = if (itr.hasNext) Some(itr.next()) else None (t._1, t._2._1.toSeq, headOption) - }) - updateFuncLocal(i) + } + updateFuncLocal(batchTime, i) } val cogroupedRDD = parentRDD.cogroup(prevStateRDD, partitioner) val stateRDD = cogroupedRDD.mapPartitions(finalFunc, preservePartitioning) @@ -66,58 +67,48 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( // Try to get the previous state RDD getOrCompute(validTime - slideDuration) match { - case Some(prevStateRDD) => { // If previous state RDD exists - + case Some(prevStateRDD) => // If previous state RDD exists // Try to get the parent RDD parent.getOrCompute(validTime) match { - case Some(parentRDD) => { // If parent RDD exists, then compute as usual - computeUsingPreviousRDD (parentRDD, prevStateRDD) - } - case None => { // If parent RDD does not exist - + case Some(parentRDD) => // If parent RDD exists, then compute as usual + computeUsingPreviousRDD (validTime, parentRDD, prevStateRDD) + case None => // If parent RDD does not exist // Re-apply the update function to the old state RDD val updateFuncLocal = updateFunc val finalFunc = (iterator: Iterator[(K, S)]) => { - val i = iterator.map(t => (t._1, Seq[V](), Option(t._2))) - updateFuncLocal(i) + val i = iterator.map(t => (t._1, Seq.empty[V], Option(t._2))) + updateFuncLocal(validTime, i) } val stateRDD = prevStateRDD.mapPartitions(finalFunc, preservePartitioning) Some(stateRDD) - } } - } - - case None => { // If previous session RDD does not exist (first input data) + case None => // If previous session RDD does not exist (first input data) // Try to get the parent RDD parent.getOrCompute(validTime) match { - case Some(parentRDD) => { // If parent RDD exists, then compute as usual + case Some(parentRDD) => // If parent RDD exists, then compute as usual initialRDD match { - case None => { + case None => // Define the function for the mapPartition operation on grouped RDD; // first map the grouped tuple to tuples of required type, // and then apply the update function val updateFuncLocal = updateFunc - val finalFunc = (iterator : Iterator[(K, Iterable[V])]) => { - updateFuncLocal (iterator.map (tuple => (tuple._1, tuple._2.toSeq, None))) + val finalFunc = (iterator: Iterator[(K, Iterable[V])]) => { + updateFuncLocal (validTime, + iterator.map (tuple => (tuple._1, tuple._2.toSeq, None))) } - val groupedRDD = parentRDD.groupByKey (partitioner) - val sessionRDD = groupedRDD.mapPartitions (finalFunc, preservePartitioning) + val groupedRDD = parentRDD.groupByKey(partitioner) + val sessionRDD = groupedRDD.mapPartitions(finalFunc, preservePartitioning) // logDebug("Generating state RDD for time " + validTime + " (first)") Some (sessionRDD) - } - case Some (initialStateRDD) => { - computeUsingPreviousRDD(parentRDD, initialStateRDD) - } + case Some (initialStateRDD) => + computeUsingPreviousRDD(validTime, parentRDD, initialStateRDD) } - } - case None => { // If parent RDD does not exist, then nothing to do! + case None => // If parent RDD does not exist, then nothing to do! // logDebug("Not generating state RDD (no previous state, no parent)") None - } } - } } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala index 5eabdf63dc8d..0dde12092757 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala @@ -29,7 +29,7 @@ class TransformedDStream[U: ClassTag] ( transformFunc: (Seq[RDD[_]], Time) => RDD[U] ) extends DStream[U](parents.head.ssc) { - require(parents.length > 0, "List of DStreams to transform is empty") + require(parents.nonEmpty, "List of DStreams to transform is empty") require(parents.map(_.ssc).distinct.size == 1, "Some of the DStreams have different contexts") require(parents.map(_.slideDuration).distinct.size == 1, "Some of the DStreams have different slide durations") @@ -51,4 +51,17 @@ class TransformedDStream[U: ClassTag] ( } Some(transformedRDD) } + + /** + * Wrap a body of code such that the call site and operation scope + * information are passed to the RDDs created in this body properly. + * This has been overridden to make sure that `displayInnerRDDOps` is always `true`, that is, + * the inner scopes and callsites of RDDs generated in `DStream.transform` are always + * displayed in the UI. + */ + override protected[streaming] def createRDDWithLocalProperties[U]( + time: Time, + displayInnerRDDOps: Boolean)(body: => U): U = { + super.createRDDWithLocalProperties(time, displayInnerRDDOps = true)(body) + } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala index d73ffdfd84d2..d46c0a01e05d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala @@ -21,17 +21,16 @@ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import org.apache.spark.SparkException -import org.apache.spark.streaming.{Duration, Time} import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.UnionRDD +import org.apache.spark.streaming.{Duration, Time} private[streaming] class UnionDStream[T: ClassTag](parents: Array[DStream[T]]) extends DStream[T](parents.head.ssc) { require(parents.length > 0, "List of DStreams to union is empty") - require(parents.map(_.ssc).distinct.size == 1, "Some of the DStreams have different contexts") - require(parents.map(_.slideDuration).distinct.size == 1, + require(parents.map(_.ssc).distinct.length == 1, "Some of the DStreams have different contexts") + require(parents.map(_.slideDuration).distinct.length == 1, "Some of the DStreams have different slide durations") override def dependencies: List[DStream[_]] = parents.toList @@ -45,8 +44,8 @@ class UnionDStream[T: ClassTag](parents: Array[DStream[T]]) case None => throw new SparkException("Could not generate RDD from a parent for unifying at" + s" time $validTime") } - if (rdds.size > 0) { - Some(new UnionRDD(ssc.sc, rdds)) + if (rdds.nonEmpty) { + Some(ssc.sc.union(rdds)) } else { None } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala index 4efba039f895..fe0f87552566 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala @@ -17,13 +17,13 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.rdd.{PartitionerAwareUnionRDD, RDD, UnionRDD} +import scala.reflect.ClassTag + +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.Duration -import scala.reflect.ClassTag - private[streaming] class WindowedDStream[T: ClassTag]( parent: DStream[T], @@ -63,13 +63,6 @@ class WindowedDStream[T: ClassTag]( override def compute(validTime: Time): Option[RDD[T]] = { val currentWindow = new Interval(validTime - windowDuration + parent.slideDuration, validTime) val rddsInWindow = parent.slice(currentWindow) - val windowRDD = if (rddsInWindow.flatMap(_.partitioner).distinct.length == 1) { - logDebug("Using partition aware union for windowing at " + validTime) - new PartitionerAwareUnionRDD(ssc.sc, rddsInWindow) - } else { - logDebug("Using normal union for windowing at " + validTime) - new UnionRDD(ssc.sc, rddsInWindow) - } - Some(windowRDD) + Some(ssc.sc.union(rddsInWindow)) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/package-info.java b/streaming/src/main/scala/org/apache/spark/streaming/dstream/package-info.java index 05ca2ddffd3c..4d08afcbfea3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/package-info.java +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/package-info.java @@ -18,4 +18,4 @@ /** * Various implementations of DStreams. */ -package org.apache.spark.streaming.dstream; \ No newline at end of file +package org.apache.spark.streaming.dstream; diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala new file mode 100644 index 000000000000..15d3c7e54b8d --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/MapWithStateRDD.scala @@ -0,0 +1,228 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.rdd + +import java.io.{IOException, ObjectOutputStream} + +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + +import org.apache.spark._ +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{State, StateImpl, Time} +import org.apache.spark.streaming.util.{EmptyStateMap, StateMap} +import org.apache.spark.util.Utils + +/** + * Record storing the keyed-state [[MapWithStateRDD]]. Each record contains a `StateMap` and a + * sequence of records returned by the mapping function of `mapWithState`. + */ +private[streaming] case class MapWithStateRDDRecord[K, S, E]( + var stateMap: StateMap[K, S], var mappedData: Seq[E]) + +private[streaming] object MapWithStateRDDRecord { + def updateRecordWithData[K: ClassTag, V: ClassTag, S: ClassTag, E: ClassTag]( + prevRecord: Option[MapWithStateRDDRecord[K, S, E]], + dataIterator: Iterator[(K, V)], + mappingFunction: (Time, K, Option[V], State[S]) => Option[E], + batchTime: Time, + timeoutThresholdTime: Option[Long], + removeTimedoutData: Boolean + ): MapWithStateRDDRecord[K, S, E] = { + // Create a new state map by cloning the previous one (if it exists) or by creating an empty one + val newStateMap = prevRecord.map { _.stateMap.copy() }. getOrElse { new EmptyStateMap[K, S]() } + + val mappedData = new ArrayBuffer[E] + val wrappedState = new StateImpl[S]() + + // Call the mapping function on each record in the data iterator, and accordingly + // update the states touched, and collect the data returned by the mapping function + dataIterator.foreach { case (key, value) => + wrappedState.wrap(newStateMap.get(key)) + val returned = mappingFunction(batchTime, key, Some(value), wrappedState) + if (wrappedState.isRemoved) { + newStateMap.remove(key) + } else if (wrappedState.isUpdated + || (wrappedState.exists && timeoutThresholdTime.isDefined)) { + newStateMap.put(key, wrappedState.get(), batchTime.milliseconds) + } + mappedData ++= returned + } + + // Get the timed out state records, call the mapping function on each and collect the + // data returned + if (removeTimedoutData && timeoutThresholdTime.isDefined) { + newStateMap.getByTime(timeoutThresholdTime.get).foreach { case (key, state, _) => + wrappedState.wrapTimingOutState(state) + val returned = mappingFunction(batchTime, key, None, wrappedState) + mappedData ++= returned + newStateMap.remove(key) + } + } + + MapWithStateRDDRecord(newStateMap, mappedData) + } +} + +/** + * Partition of the [[MapWithStateRDD]], which depends on corresponding partitions of prev state + * RDD, and a partitioned keyed-data RDD + */ +private[streaming] class MapWithStateRDDPartition( + override val index: Int, + @transient private var prevStateRDD: RDD[_], + @transient private var partitionedDataRDD: RDD[_]) extends Partition { + + private[rdd] var previousSessionRDDPartition: Partition = null + private[rdd] var partitionedDataRDDPartition: Partition = null + + override def hashCode(): Int = index + + override def equals(other: Any): Boolean = other match { + case that: MapWithStateRDDPartition => index == that.index + case _ => false + } + + @throws(classOf[IOException]) + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { + // Update the reference to parent split at the time of task serialization + previousSessionRDDPartition = prevStateRDD.partitions(index) + partitionedDataRDDPartition = partitionedDataRDD.partitions(index) + oos.defaultWriteObject() + } +} + + +/** + * RDD storing the keyed states of `mapWithState` operation and corresponding mapped data. + * Each partition of this RDD has a single record of type [[MapWithStateRDDRecord]]. This contains a + * `StateMap` (containing the keyed-states) and the sequence of records returned by the mapping + * function of `mapWithState`. + * @param prevStateRDD The previous MapWithStateRDD on whose StateMap data `this` RDD + * will be created + * @param partitionedDataRDD The partitioned data RDD which is used update the previous StateMaps + * in the `prevStateRDD` to create `this` RDD + * @param mappingFunction The function that will be used to update state and return new data + * @param batchTime The time of the batch to which this RDD belongs to. Use to update + * @param timeoutThresholdTime The time to indicate which keys are timeout + */ +private[streaming] class MapWithStateRDD[K: ClassTag, V: ClassTag, S: ClassTag, E: ClassTag]( + private var prevStateRDD: RDD[MapWithStateRDDRecord[K, S, E]], + private var partitionedDataRDD: RDD[(K, V)], + mappingFunction: (Time, K, Option[V], State[S]) => Option[E], + batchTime: Time, + timeoutThresholdTime: Option[Long] + ) extends RDD[MapWithStateRDDRecord[K, S, E]]( + partitionedDataRDD.sparkContext, + List( + new OneToOneDependency[MapWithStateRDDRecord[K, S, E]](prevStateRDD), + new OneToOneDependency(partitionedDataRDD)) + ) { + + @volatile private var doFullScan = false + + require(prevStateRDD.partitioner.nonEmpty) + require(partitionedDataRDD.partitioner == prevStateRDD.partitioner) + + override val partitioner = prevStateRDD.partitioner + + override def checkpoint(): Unit = { + super.checkpoint() + doFullScan = true + } + + override def compute( + partition: Partition, context: TaskContext): Iterator[MapWithStateRDDRecord[K, S, E]] = { + + val stateRDDPartition = partition.asInstanceOf[MapWithStateRDDPartition] + val prevStateRDDIterator = prevStateRDD.iterator( + stateRDDPartition.previousSessionRDDPartition, context) + val dataIterator = partitionedDataRDD.iterator( + stateRDDPartition.partitionedDataRDDPartition, context) + + val prevRecord = if (prevStateRDDIterator.hasNext) Some(prevStateRDDIterator.next()) else None + val newRecord = MapWithStateRDDRecord.updateRecordWithData( + prevRecord, + dataIterator, + mappingFunction, + batchTime, + timeoutThresholdTime, + removeTimedoutData = doFullScan // remove timedout data only when full scan is enabled + ) + Iterator(newRecord) + } + + override protected def getPartitions: Array[Partition] = { + Array.tabulate(prevStateRDD.partitions.length) { i => + new MapWithStateRDDPartition(i, prevStateRDD, partitionedDataRDD)} + } + + override def clearDependencies(): Unit = { + super.clearDependencies() + prevStateRDD = null + partitionedDataRDD = null + } + + def setFullScan(): Unit = { + doFullScan = true + } +} + +private[streaming] object MapWithStateRDD { + + def createFromPairRDD[K: ClassTag, V: ClassTag, S: ClassTag, E: ClassTag]( + pairRDD: RDD[(K, S)], + partitioner: Partitioner, + updateTime: Time): MapWithStateRDD[K, V, S, E] = { + + val stateRDD = pairRDD.partitionBy(partitioner).mapPartitions ({ iterator => + val stateMap = StateMap.create[K, S](SparkEnv.get.conf) + iterator.foreach { case (key, state) => stateMap.put(key, state, updateTime.milliseconds) } + Iterator(MapWithStateRDDRecord(stateMap, Seq.empty[E])) + }, preservesPartitioning = true) + + val emptyDataRDD = pairRDD.sparkContext.emptyRDD[(K, V)].partitionBy(partitioner) + + val noOpFunc = (time: Time, key: K, value: Option[V], state: State[S]) => None + + new MapWithStateRDD[K, V, S, E]( + stateRDD, emptyDataRDD, noOpFunc, updateTime, None) + } + + def createFromRDD[K: ClassTag, V: ClassTag, S: ClassTag, E: ClassTag]( + rdd: RDD[(K, S, Long)], + partitioner: Partitioner, + updateTime: Time): MapWithStateRDD[K, V, S, E] = { + + val pairRDD = rdd.map { x => (x._1, (x._2, x._3)) } + val stateRDD = pairRDD.partitionBy(partitioner).mapPartitions({ iterator => + val stateMap = StateMap.create[K, S](SparkEnv.get.conf) + iterator.foreach { case (key, (state, updateTime)) => + stateMap.put(key, state, updateTime) + } + Iterator(MapWithStateRDDRecord(stateMap, Seq.empty[E])) + }, preservesPartitioning = true) + + val emptyDataRDD = pairRDD.sparkContext.emptyRDD[(K, V)].partitionBy(partitioner) + + val noOpFunc = (time: Time, key: K, value: Option[V], state: State[S]) => None + + new MapWithStateRDD[K, V, S, E]( + stateRDD, emptyDataRDD, noOpFunc, updateTime, None) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala index f811784b25c8..844760ab61d2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -27,12 +27,14 @@ import org.apache.spark._ import org.apache.spark.rdd.BlockRDD import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.streaming.util._ -import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.util._ +import org.apache.spark.util.io.ChunkedByteBuffer /** * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]]. * It contains information about the id of the blocks having this partition's data and * the corresponding record handle in the write ahead log that backs the partition. + * * @param index index of the partition * @param blockId id of the block having the partition data * @param isBlockIdValid Whether the block Ids are valid (i.e., the blocks are present in the Spark @@ -59,7 +61,6 @@ class WriteAheadLogBackedBlockRDDPartition( * correctness, and it can be used in situations where it is known that the block * does not exist in the Spark executors (e.g. after a failed driver is restarted). * - * * @param sc SparkContext * @param _blockIds Ids of the blocks that contains this RDD's data * @param walRecordHandles Record handles in write ahead logs that contain this RDD's data @@ -114,11 +115,12 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( assertValid() val hadoopConf = broadcastedHadoopConf.value val blockManager = SparkEnv.get.blockManager + val serializerManager = SparkEnv.get.serializerManager val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition] val blockId = partition.blockId def getBlockFromBlockManager(): Option[Iterator[T]] = { - blockManager.get(blockId).map(_.data.asInstanceOf[Iterator[T]]) + blockManager.get[T](blockId).map(_.data.asInstanceOf[Iterator[T]]) } def getBlockFromWriteAheadLog(): Iterator[T] = { @@ -156,11 +158,15 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( logInfo(s"Read partition data of $this from write ahead log, record handle " + partition.walRecordHandle) if (storeInBlockManager) { - blockManager.putBytes(blockId, dataRead, storageLevel) + blockManager.putBytes(blockId, new ChunkedByteBuffer(dataRead.duplicate()), storageLevel) logDebug(s"Stored partition data of $this into block manager with level $storageLevel") dataRead.rewind() } - blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]] + serializerManager + .dataDeserializeStream( + blockId, + new ChunkedByteBuffer(dataRead).toInputStream())(elementClassTag) + .asInstanceOf[Iterator[T]] } if (partition.isBlockIdValid) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala deleted file mode 100644 index 7ec74016a1c2..000000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala +++ /dev/null @@ -1,201 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.receiver - -import java.nio.ByteBuffer -import java.util.concurrent.atomic.AtomicInteger - -import scala.concurrent.duration._ -import scala.language.postfixOps -import scala.reflect.ClassTag - -import akka.actor._ -import akka.actor.SupervisorStrategy.{Escalate, Restart} - -import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.storage.StorageLevel - -/** - * :: DeveloperApi :: - * A helper with set of defaults for supervisor strategy - */ -@DeveloperApi -object ActorSupervisorStrategy { - - val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = - 15 millis) { - case _: RuntimeException => Restart - case _: Exception => Escalate - } -} - -/** - * :: DeveloperApi :: - * A receiver trait to be mixed in with your Actor to gain access to - * the API for pushing received data into Spark Streaming for being processed. - * - * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html - * - * @example {{{ - * class MyActor extends Actor with ActorHelper{ - * def receive { - * case anything: String => store(anything) - * } - * } - * - * // Can be used with an actorStream as follows - * ssc.actorStream[String](Props(new MyActor),"MyActorReceiver") - * - * }}} - * - * @note Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e parametrized type of push block and InputDStream - * should be same. - */ -@DeveloperApi -trait ActorHelper extends Logging{ - - self: Actor => // to ensure that this can be added to Actor classes only - - /** Store an iterator of received data as a data block into Spark's memory. */ - def store[T](iter: Iterator[T]) { - logDebug("Storing iterator") - context.parent ! IteratorData(iter) - } - - /** - * Store the bytes of received data as a data block into Spark's memory. Note - * that the data in the ByteBuffer must be serialized using the same serializer - * that Spark is configured to use. - */ - def store(bytes: ByteBuffer) { - logDebug("Storing Bytes") - context.parent ! ByteBufferData(bytes) - } - - /** - * Store a single item of received data to Spark's memory. - * These single items will be aggregated together into data blocks before - * being pushed into Spark's memory. - */ - def store[T](item: T) { - logDebug("Storing item") - context.parent ! SingleItemData(item) - } -} - -/** - * :: DeveloperApi :: - * Statistics for querying the supervisor about state of workers. Used in - * conjunction with `StreamingContext.actorStream` and - * [[org.apache.spark.streaming.receiver.ActorHelper]]. - */ -@DeveloperApi -case class Statistics(numberOfMsgs: Int, - numberOfWorkers: Int, - numberOfHiccups: Int, - otherInfo: String) - -/** Case class to receive data sent by child actors */ -private[streaming] sealed trait ActorReceiverData -private[streaming] case class SingleItemData[T](item: T) extends ActorReceiverData -private[streaming] case class IteratorData[T](iterator: Iterator[T]) extends ActorReceiverData -private[streaming] case class ByteBufferData(bytes: ByteBuffer) extends ActorReceiverData - -/** - * Provides Actors as receivers for receiving stream. - * - * As Actors can also be used to receive data from almost any stream source. - * A nice set of abstraction(s) for actors as receivers is already provided for - * a few general cases. It is thus exposed as an API where user may come with - * their own Actor to run as receiver for Spark Streaming input source. - * - * This starts a supervisor actor which starts workers and also provides - * [http://doc.akka.io/docs/akka/snapshot/scala/fault-tolerance.html fault-tolerance]. - * - * Here's a way to start more supervisor/workers as its children. - * - * @example {{{ - * context.parent ! Props(new Supervisor) - * }}} OR {{{ - * context.parent ! Props(new Worker, "Worker") - * }}} - */ -private[streaming] class ActorReceiver[T: ClassTag]( - props: Props, - name: String, - storageLevel: StorageLevel, - receiverSupervisorStrategy: SupervisorStrategy - ) extends Receiver[T](storageLevel) with Logging { - - protected lazy val actorSupervisor = SparkEnv.get.actorSystem.actorOf(Props(new Supervisor), - "Supervisor" + streamId) - - class Supervisor extends Actor { - - override val supervisorStrategy = receiverSupervisorStrategy - private val worker = context.actorOf(props, name) - logInfo("Started receiver worker at:" + worker.path) - - private val n: AtomicInteger = new AtomicInteger(0) - private val hiccups: AtomicInteger = new AtomicInteger(0) - - override def receive: PartialFunction[Any, Unit] = { - - case IteratorData(iterator) => - logDebug("received iterator") - store(iterator.asInstanceOf[Iterator[T]]) - - case SingleItemData(msg) => - logDebug("received single") - store(msg.asInstanceOf[T]) - n.incrementAndGet - - case ByteBufferData(bytes) => - logDebug("received bytes") - store(bytes) - - case props: Props => - val worker = context.actorOf(props) - logInfo("Started receiver worker at:" + worker.path) - sender ! worker - - case (props: Props, name: String) => - val worker = context.actorOf(props, name) - logInfo("Started receiver worker at:" + worker.path) - sender ! worker - - case _: PossiblyHarmful => hiccups.incrementAndGet() - - case _: Statistics => - val workers = context.children - sender ! Statistics(n.get, workers.size, hiccups.get, workers.mkString("\n")) - - } - } - - def onStart(): Unit = { - actorSupervisor - logInfo("Supervision tree for receivers initialized at:" + actorSupervisor.path) - } - - def onStop(): Unit = { - actorSupervisor ! PoisonPill - } -} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index 421d60ae359f..90309c0145ae 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -21,7 +21,8 @@ import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{SparkException, Logging, SparkConf} +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.util.RecurringTimer import org.apache.spark.util.{Clock, SystemClock} @@ -36,7 +37,7 @@ private[streaming] trait BlockGeneratorListener { * that will be useful when a block is generated. Any long blocking operation in this callback * will hurt the throughput. */ - def onAddData(data: Any, metadata: Any) + def onAddData(data: Any, metadata: Any): Unit /** * Called when a new block of data is generated by the block generator. The block generation @@ -46,7 +47,7 @@ private[streaming] trait BlockGeneratorListener { * be useful when the block has been successfully stored. Any long blocking operation in this * callback will hurt the throughput. */ - def onGenerateBlock(blockId: StreamBlockId) + def onGenerateBlock(blockId: StreamBlockId): Unit /** * Called when a new block is ready to be pushed. Callers are supposed to store the block into @@ -54,13 +55,13 @@ private[streaming] trait BlockGeneratorListener { * thread, that is not synchronized with any other callbacks. Hence it is okay to do long * blocking operation in this callback. */ - def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) + def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]): Unit /** * Called when an error has occurred in the BlockGenerator. Can be called form many places * so better to not do any long block operation in this callback. */ - def onError(message: String, throwable: Throwable) + def onError(message: String, throwable: Throwable): Unit } /** @@ -84,13 +85,14 @@ private[streaming] class BlockGenerator( /** * The BlockGenerator can be in 5 possible states, in the order as follows. - * - Initialized: Nothing has been started - * - Active: start() has been called, and it is generating blocks on added data. - * - StoppedAddingData: stop() has been called, the adding of data has been stopped, - * but blocks are still being generated and pushed. - * - StoppedGeneratingBlocks: Generating of blocks has been stopped, but - * they are still being pushed. - * - StoppedAll: Everything has stopped, and the BlockGenerator object can be GCed. + * + * - Initialized: Nothing has been started. + * - Active: start() has been called, and it is generating blocks on added data. + * - StoppedAddingData: stop() has been called, the adding of data has been stopped, + * but blocks are still being generated and pushed. + * - StoppedGeneratingBlocks: Generating of blocks has been stopped, but + * they are still being pushed. + * - StoppedAll: Everything has been stopped, and the BlockGenerator object can be GCed. */ private object GeneratorState extends Enumeration { type GeneratorState = Value @@ -125,9 +127,10 @@ private[streaming] class BlockGenerator( /** * Stop everything in the right order such that all the data added is pushed out correctly. - * - First, stop adding data to the current buffer. - * - Second, stop generating blocks. - * - Finally, wait for queue of to-be-pushed blocks to be drained. + * + * - First, stop adding data to the current buffer. + * - Second, stop generating blocks. + * - Finally, wait for queue of to-be-pushed blocks to be drained. */ def stop(): Unit = { // Set the state to stop adding data @@ -145,7 +148,7 @@ private[streaming] class BlockGenerator( blockIntervalTimer.stop(interruptTimer = false) synchronized { state = StoppedGeneratingBlocks } - // Wait for the queue to drain and mark generated as stopped + // Wait for the queue to drain and mark state as StoppedAll logInfo("Waiting for block pushing thread to terminate") blockPushingThread.join() synchronized { state = StoppedAll } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala index bca1fbc8fda2..fbac4880bdf6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala @@ -19,24 +19,26 @@ package org.apache.spark.streaming.receiver import com.google.common.util.concurrent.{RateLimiter => GuavaRateLimiter} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging -/** Provides waitToPush() method to limit the rate at which receivers consume data. - * - * waitToPush method will block the thread if too many messages have been pushed too quickly, - * and only return when a new message has been pushed. It assumes that only one message is - * pushed at a time. - * - * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages - * per second that each receiver will accept. - * - * @param conf spark configuration - */ +/** + * Provides waitToPush() method to limit the rate at which receivers consume data. + * + * waitToPush method will block the thread if too many messages have been pushed too quickly, + * and only return when a new message has been pushed. It assumes that only one message is + * pushed at a time. + * + * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages + * per second that each receiver will accept. + * + * @param conf spark configuration + */ private[receiver] abstract class RateLimiter(conf: SparkConf) extends Logging { // treated as an upper limit private val maxRateLimit = conf.getLong("spark.streaming.receiver.maxRate", Long.MaxValue) - private lazy val rateLimiter = GuavaRateLimiter.create(maxRateLimit.toDouble) + private lazy val rateLimiter = GuavaRateLimiter.create(getInitialRateLimit().toDouble) def waitToPush() { rateLimiter.acquire() @@ -51,7 +53,7 @@ private[receiver] abstract class RateLimiter(conf: SparkConf) extends Logging { * Set the rate limit to `newRate`. The new rate will not exceed the maximum rate configured by * {{{spark.streaming.receiver.maxRate}}}, even if `newRate` is higher than that. * - * @param newRate A new rate in events per second. It has no effect if it's 0 or negative. + * @param newRate A new rate in records per second. It has no effect if it's 0 or negative. */ private[receiver] def updateRate(newRate: Long): Unit = if (newRate > 0) { @@ -61,4 +63,11 @@ private[receiver] abstract class RateLimiter(conf: SparkConf) extends Logging { rateLimiter.setRate(newRate) } } + + /** + * Get the initial rateLimit to initial rateLimiter + */ + private def getInitialRateLimit(): Long = { + math.min(conf.getLong("spark.streaming.backpressure.initialRate", maxRateLimit), maxRateLimit) + } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala index 47968afef2db..8c3a7977beae 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala @@ -31,5 +31,5 @@ private[streaming] case class ArrayBufferBlock(arrayBuffer: ArrayBuffer[_]) exte /** class representing a block received as an Iterator */ private[streaming] case class IteratorBlock(iterator: Iterator[_]) extends ReceivedBlock -/** class representing a block received as an ByteBuffer */ +/** class representing a block received as a ByteBuffer */ private[streaming] case class ByteBufferBlock(byteBuffer: ByteBuffer) extends ReceivedBlock diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index 5f6c5b024085..80c07958b41f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -17,18 +17,21 @@ package org.apache.spark.streaming.receiver +import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ -import scala.concurrent.{Await, ExecutionContext, Future} import scala.language.{existentials, postfixOps} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging +import org.apache.spark.serializer.SerializerManager import org.apache.spark.storage._ import org.apache.spark.streaming.receiver.WriteAheadLogBasedBlockHandler._ import org.apache.spark.streaming.util.{WriteAheadLogRecordHandle, WriteAheadLogUtils} import org.apache.spark.util.{Clock, SystemClock, ThreadUtils} -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.util.io.ChunkedByteBuffer /** Trait that represents the metadata related to storage of blocks */ private[streaming] trait ReceivedBlockStoreResult { @@ -45,7 +48,7 @@ private[streaming] trait ReceivedBlockHandler { def storeBlock(blockId: StreamBlockId, receivedBlock: ReceivedBlock): ReceivedBlockStoreResult /** Cleanup old blocks older than the given threshold time */ - def cleanupOldBlocks(threshTime: Long) + def cleanupOldBlocks(threshTime: Long): Unit } @@ -69,9 +72,9 @@ private[streaming] class BlockManagerBasedBlockHandler( def storeBlock(blockId: StreamBlockId, block: ReceivedBlock): ReceivedBlockStoreResult = { - var numRecords = None: Option[Long] + var numRecords: Option[Long] = None - val putResult: Seq[(BlockId, BlockStatus)] = block match { + val putSucceeded: Boolean = block match { case ArrayBufferBlock(arrayBuffer) => numRecords = Some(arrayBuffer.size.toLong) blockManager.putIterator(blockId, arrayBuffer.iterator, storageLevel, @@ -83,12 +86,13 @@ private[streaming] class BlockManagerBasedBlockHandler( numRecords = countIterator.count putResult case ByteBufferBlock(byteBuffer) => - blockManager.putBytes(blockId, byteBuffer, storageLevel, tellMaster = true) + blockManager.putBytes( + blockId, new ChunkedByteBuffer(byteBuffer.duplicate()), storageLevel, tellMaster = true) case o => throw new SparkException( s"Could not store $blockId to block manager, unexpected block type ${o.getClass.getName}") } - if (!putResult.map { _._1 }.contains(blockId)) { + if (!putSucceeded) { throw new SparkException( s"Could not store $blockId to block manager with storage level $storageLevel") } @@ -120,6 +124,7 @@ private[streaming] case class WriteAheadLogBasedStoreResult( */ private[streaming] class WriteAheadLogBasedBlockHandler( blockManager: BlockManager, + serializerManager: SerializerManager, streamId: Int, storageLevel: StorageLevel, conf: SparkConf, @@ -165,28 +170,31 @@ private[streaming] class WriteAheadLogBasedBlockHandler( */ def storeBlock(blockId: StreamBlockId, block: ReceivedBlock): ReceivedBlockStoreResult = { - var numRecords = None: Option[Long] + var numRecords = Option.empty[Long] // Serialize the block so that it can be inserted into both val serializedBlock = block match { case ArrayBufferBlock(arrayBuffer) => numRecords = Some(arrayBuffer.size.toLong) - blockManager.dataSerialize(blockId, arrayBuffer.iterator) + serializerManager.dataSerialize(blockId, arrayBuffer.iterator) case IteratorBlock(iterator) => val countIterator = new CountingIterator(iterator) - val serializedBlock = blockManager.dataSerialize(blockId, countIterator) + val serializedBlock = serializerManager.dataSerialize(blockId, countIterator) numRecords = countIterator.count serializedBlock case ByteBufferBlock(byteBuffer) => - byteBuffer + new ChunkedByteBuffer(byteBuffer.duplicate()) case _ => throw new Exception(s"Could not push $blockId to block manager, unexpected block type") } // Store the block in block manager val storeInBlockManagerFuture = Future { - val putResult = - blockManager.putBytes(blockId, serializedBlock, effectiveStorageLevel, tellMaster = true) - if (!putResult.map { _._1 }.contains(blockId)) { + val putSucceeded = blockManager.putBytes( + blockId, + serializedBlock, + effectiveStorageLevel, + tellMaster = true) + if (!putSucceeded) { throw new SparkException( s"Could not store $blockId to block manager with storage level $storageLevel") } @@ -194,12 +202,12 @@ private[streaming] class WriteAheadLogBasedBlockHandler( // Store the block in write ahead log val storeInWriteAheadLogFuture = Future { - writeAheadLog.write(serializedBlock, clock.getTimeMillis()) + writeAheadLog.write(serializedBlock.toByteBuffer, clock.getTimeMillis()) } // Combine the futures, wait for both to complete, and return the write ahead log record handle val combinedFuture = storeInBlockManagerFuture.zip(storeInWriteAheadLogFuture).map(_._2) - val walRecordHandle = Await.result(combinedFuture, blockStoreTimeout) + val walRecordHandle = ThreadUtils.awaitResult(combinedFuture, blockStoreTimeout) WriteAheadLogBasedStoreResult(blockId, numRecords, walRecordHandle) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala index 2252e28f22af..31a88730d163 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -19,11 +19,11 @@ package org.apache.spark.streaming.receiver import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer -import org.apache.spark.storage.StorageLevel import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.storage.StorageLevel /** * :: DeveloperApi :: @@ -32,7 +32,7 @@ import org.apache.spark.annotation.DeveloperApi * should define the setup steps necessary to start receiving data, * and `onStop()` should define the cleanup steps necessary to stop receiving data. * Exceptions while receiving can be handled either by restarting the receiver with `restart(...)` - * or stopped completely by `stop(...)` or + * or stopped completely by `stop(...)`. * * A custom receiver in Scala would look like this. * @@ -45,7 +45,7 @@ import org.apache.spark.annotation.DeveloperApi * // Call store(...) in those threads to store received data into Spark's memory. * * // Call stop(...), restart(...) or reportError(...) on any thread based on how - * // different errors needs to be handled. + * // different errors need to be handled. * * // See corresponding method documentation for more details * } @@ -71,7 +71,7 @@ import org.apache.spark.annotation.DeveloperApi * // Call store(...) in those threads to store received data into Spark's memory. * * // Call stop(...), restart(...) or reportError(...) on any thread based on how - * // different errors needs to be handled. + * // different errors need to be handled. * * // See corresponding method documentation for more details * } @@ -99,16 +99,16 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable * (iii) `restart(...)` can be called to restart the receiver. This will call `onStop()` * immediately, and then `onStart()` after a delay. */ - def onStart() + def onStart(): Unit /** * This method is called by the system when the receiver is stopped. All resources - * (threads, buffers, etc.) setup in `onStart()` must be cleaned up in this method. + * (threads, buffers, etc.) set up in `onStart()` must be cleaned up in this method. */ - def onStop() + def onStop(): Unit /** Override this to specify a preferred location (hostname). */ - def preferredLocation : Option[String] = None + def preferredLocation: Option[String] = None /** * Store a single item of received data to Spark's memory. @@ -257,11 +257,11 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable private var id: Int = -1 /** Handler object that runs the receiver. This is instantiated lazily in the worker. */ - @transient private var _supervisor : ReceiverSupervisor = null + @transient private var _supervisor: ReceiverSupervisor = null /** Set the ID of the DStream that this receiver is associated with. */ - private[streaming] def setReceiverId(id_ : Int) { - id = id_ + private[streaming] def setReceiverId(_id: Int) { + id = _id } /** Attach Network Receiver executor to this receiver. */ @@ -273,7 +273,7 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable /** Get the attached supervisor. */ private[streaming] def supervisor: ReceiverSupervisor = { assert(_supervisor != null, - "A ReceiverSupervisor have not been attached to the receiver yet. Maybe you are starting " + + "A ReceiverSupervisor has not been attached to the receiver yet. Maybe you are starting " + "some computation in the receiver before the Receiver.onStart() has been called.") _supervisor } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 158d1ba2f183..faf6db82d5b1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -24,9 +24,10 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent._ import scala.util.control.NonFatal -import org.apache.spark.{SparkEnv, Logging, SparkConf} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.storage.StreamBlockId -import org.apache.spark.util.{Utils, ThreadUtils} +import org.apache.spark.util.{ThreadUtils, Utils} /** * Abstract class that is responsible for supervising a Receiver in the worker. @@ -69,28 +70,28 @@ private[streaming] abstract class ReceiverSupervisor( @volatile private[streaming] var receiverState = Initialized /** Push a single data item to backend data store. */ - def pushSingle(data: Any) + def pushSingle(data: Any): Unit /** Store the bytes of received data as a data block into Spark's memory. */ def pushBytes( bytes: ByteBuffer, optionalMetadata: Option[Any], optionalBlockId: Option[StreamBlockId] - ) + ): Unit - /** Store a iterator of received data as a data block into Spark's memory. */ + /** Store an iterator of received data as a data block into Spark's memory. */ def pushIterator( iterator: Iterator[_], optionalMetadata: Option[Any], optionalBlockId: Option[StreamBlockId] - ) + ): Unit /** Store an ArrayBuffer of received data as a data block into Spark's memory. */ def pushArrayBuffer( arrayBuffer: ArrayBuffer[_], optionalMetadata: Option[Any], optionalBlockId: Option[StreamBlockId] - ) + ): Unit /** * Create a custom [[BlockGenerator]] that the receiver implementation can directly control @@ -102,7 +103,7 @@ private[streaming] abstract class ReceiverSupervisor( def createBlockGenerator(blockGeneratorListener: BlockGeneratorListener): BlockGenerator /** Report errors. */ - def reportError(message: String, throwable: Throwable) + def reportError(message: String, throwable: Throwable): Unit /** * Called when supervisor is started. @@ -143,10 +144,10 @@ private[streaming] abstract class ReceiverSupervisor( def startReceiver(): Unit = synchronized { try { if (onReceiverStart()) { - logInfo("Starting receiver") + logInfo(s"Starting receiver $streamId") receiverState = Started receiver.onStart() - logInfo("Called receiver onStart") + logInfo(s"Called receiver $streamId onStart") } else { // The driver refused us stop("Registered unsuccessfully because Driver refused to start receiver " + streamId, None) @@ -174,7 +175,7 @@ private[streaming] abstract class ReceiverSupervisor( } } catch { case NonFatal(t) => - logError("Error stopping receiver " + streamId + t.getStackTraceString) + logError(s"Error stopping receiver $streamId ${Utils.exceptionString(t)}") } } @@ -218,11 +219,9 @@ private[streaming] abstract class ReceiverSupervisor( stopLatch.await() if (stoppingError != null) { logError("Stopped receiver with error: " + stoppingError) + throw stoppingError } else { logInfo("Stopped receiver without error") } - if (stoppingError != null) { - throw stoppingError - } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 167f56aa4228..27644a645727 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -18,21 +18,23 @@ package org.apache.spark.streaming.receiver import java.nio.ByteBuffer +import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.atomic.AtomicLong -import scala.collection.mutable +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import com.google.common.base.Throwables import org.apache.hadoop.conf.Configuration +import org.apache.spark.{SparkEnv, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.Time import org.apache.spark.streaming.scheduler._ import org.apache.spark.streaming.util.WriteAheadLogUtils import org.apache.spark.util.RpcUtils -import org.apache.spark.{Logging, SparkEnv, SparkException} /** * Concrete implementation of [[org.apache.spark.streaming.receiver.ReceiverSupervisor]] @@ -58,7 +60,7 @@ private[streaming] class ReceiverSupervisorImpl( "Please use streamingContext.checkpoint() to set the checkpoint directory. " + "See documentation for more details.") } - new WriteAheadLogBasedBlockHandler(env.blockManager, receiver.streamId, + new WriteAheadLogBasedBlockHandler(env.blockManager, env.serializerManager, receiver.streamId, receiver.storageLevel, env.conf, hadoopConf, checkpointDirOption.get) } else { new BlockManagerBasedBlockHandler(env.blockManager, receiver.storageLevel) @@ -83,7 +85,7 @@ private[streaming] class ReceiverSupervisorImpl( cleanupOldBlocks(threshTime) case UpdateRateLimit(eps) => logInfo(s"Received a new rate limit: $eps.") - registeredBlockGenerators.foreach { bg => + registeredBlockGenerators.asScala.foreach { bg => bg.updateRate(eps) } } @@ -92,8 +94,7 @@ private[streaming] class ReceiverSupervisorImpl( /** Unique block ids if one wants to add blocks directly */ private val newBlockId = new AtomicLong(System.currentTimeMillis()) - private val registeredBlockGenerators = new mutable.ArrayBuffer[BlockGenerator] - with mutable.SynchronizedBuffer[BlockGenerator] + private val registeredBlockGenerators = new ConcurrentLinkedQueue[BlockGenerator]() /** Divides received data records into data blocks for pushing in BlockManager. */ private val defaultBlockGeneratorListener = new BlockGeneratorListener { @@ -128,7 +129,7 @@ private[streaming] class ReceiverSupervisorImpl( pushAndReportBlock(ArrayBufferBlock(arrayBuffer), metadataOption, blockIdOption) } - /** Store a iterator of received data as a data block into Spark's memory. */ + /** Store an iterator of received data as a data block into Spark's memory. */ def pushIterator( iterator: Iterator[_], metadataOption: Option[Any], @@ -158,7 +159,7 @@ private[streaming] class ReceiverSupervisorImpl( logDebug(s"Pushed block $blockId in ${(System.currentTimeMillis - time)} ms") val numRecords = blockStoreResult.numRecords val blockInfo = ReceivedBlockInfo(streamId, numRecords, metadataOption, blockStoreResult) - trackerEndpoint.askWithRetry[Boolean](AddBlock(blockInfo)) + trackerEndpoint.askSync[Boolean](AddBlock(blockInfo)) logDebug(s"Reported block $blockId") } @@ -170,34 +171,41 @@ private[streaming] class ReceiverSupervisorImpl( } override protected def onStart() { - registeredBlockGenerators.foreach { _.start() } + registeredBlockGenerators.asScala.foreach { _.start() } } override protected def onStop(message: String, error: Option[Throwable]) { - registeredBlockGenerators.foreach { _.stop() } + receivedBlockHandler match { + case handler: WriteAheadLogBasedBlockHandler => + // Write ahead log should be closed. + handler.stop() + case _ => + } + registeredBlockGenerators.asScala.foreach { _.stop() } env.rpcEnv.stop(endpoint) } override protected def onReceiverStart(): Boolean = { val msg = RegisterReceiver( streamId, receiver.getClass.getSimpleName, host, executorId, endpoint) - trackerEndpoint.askWithRetry[Boolean](msg) + trackerEndpoint.askSync[Boolean](msg) } override protected def onReceiverStop(message: String, error: Option[Throwable]) { logInfo("Deregistering receiver " + streamId) val errorString = error.map(Throwables.getStackTraceAsString).getOrElse("") - trackerEndpoint.askWithRetry[Boolean](DeregisterReceiver(streamId, message, errorString)) + trackerEndpoint.askSync[Boolean](DeregisterReceiver(streamId, message, errorString)) logInfo("Stopped receiver " + streamId) } override def createBlockGenerator( blockGeneratorListener: BlockGeneratorListener): BlockGenerator = { // Cleanup BlockGenerators that have already been stopped - registeredBlockGenerators --= registeredBlockGenerators.filter{ _.isStopped() } + val stoppedGenerators = registeredBlockGenerators.asScala.filter{ _.isStopped() } + stoppedGenerators.foreach(registeredBlockGenerators.remove(_)) val newBlockGenerator = new BlockGenerator(blockGeneratorListener, streamId, env.conf) - registeredBlockGenerators += newBlockGenerator + registeredBlockGenerators.add(newBlockGenerator) newBlockGenerator } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala index 436eb0a56614..5b2b959f8138 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -41,9 +41,6 @@ case class BatchInfo( outputOperationInfos: Map[Int, OutputOperationInfo] ) { - @deprecated("Use streamIdToInputInfo instead", "1.5.0") - def streamIdToNumRecords: Map[Int, Long] = streamIdToInputInfo.mapValues(_.numRecords) - /** * Time taken for the first job of this batch to start processing from the time this batch * was submitted to the streaming scheduler. Essentially, it is diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala new file mode 100644 index 000000000000..7b29b40668de --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.spark.streaming.scheduler + +import scala.util.Random + +import org.apache.spark.{ExecutorAllocationClient, SparkConf} +import org.apache.spark.internal.Logging +import org.apache.spark.streaming.util.RecurringTimer +import org.apache.spark.util.{Clock, Utils} + +/** + * Class that manages executor allocated to a StreamingContext, and dynamically request or kill + * executors based on the statistics of the streaming computation. This is different from the core + * dynamic allocation policy; the core policy relies on executors being idle for a while, but the + * micro-batch model of streaming prevents any particular executors from being idle for a long + * time. Instead, the measure of "idle-ness" needs to be based on the time taken to process + * each batch. + * + * At a high level, the policy implemented by this class is as follows: + * - Use StreamingListener interface get batch processing times of completed batches + * - Periodically take the average batch completion times and compare with the batch interval + * - If (avg. proc. time / batch interval) >= scaling up ratio, then request more executors. + * The number of executors requested is based on the ratio = (avg. proc. time / batch interval). + * - If (avg. proc. time / batch interval) <= scaling down ratio, then try to kill an executor that + * is not running a receiver. + * + * This features should ideally be used in conjunction with backpressure, as backpressure ensures + * system stability, while executors are being readjusted. + */ +private[streaming] class ExecutorAllocationManager( + client: ExecutorAllocationClient, + receiverTracker: ReceiverTracker, + conf: SparkConf, + batchDurationMs: Long, + clock: Clock) extends StreamingListener with Logging { + + import ExecutorAllocationManager._ + + private val scalingIntervalSecs = conf.getTimeAsSeconds( + SCALING_INTERVAL_KEY, + s"${SCALING_INTERVAL_DEFAULT_SECS}s") + private val scalingUpRatio = conf.getDouble(SCALING_UP_RATIO_KEY, SCALING_UP_RATIO_DEFAULT) + private val scalingDownRatio = conf.getDouble(SCALING_DOWN_RATIO_KEY, SCALING_DOWN_RATIO_DEFAULT) + private val minNumExecutors = conf.getInt( + MIN_EXECUTORS_KEY, + math.max(1, receiverTracker.numReceivers)) + private val maxNumExecutors = conf.getInt(MAX_EXECUTORS_KEY, Integer.MAX_VALUE) + private val timer = new RecurringTimer(clock, scalingIntervalSecs * 1000, + _ => manageAllocation(), "streaming-executor-allocation-manager") + + @volatile private var batchProcTimeSum = 0L + @volatile private var batchProcTimeCount = 0 + + validateSettings() + + def start(): Unit = { + timer.start() + logInfo(s"ExecutorAllocationManager started with " + + s"ratios = [$scalingUpRatio, $scalingDownRatio] and interval = $scalingIntervalSecs sec") + } + + def stop(): Unit = { + timer.stop(interruptTimer = true) + logInfo("ExecutorAllocationManager stopped") + } + + /** + * Manage executor allocation by requesting or killing executors based on the collected + * batch statistics. + */ + private def manageAllocation(): Unit = synchronized { + logInfo(s"Managing executor allocation with ratios = [$scalingUpRatio, $scalingDownRatio]") + if (batchProcTimeCount > 0) { + val averageBatchProcTime = batchProcTimeSum / batchProcTimeCount + val ratio = averageBatchProcTime.toDouble / batchDurationMs + logInfo(s"Average: $averageBatchProcTime, ratio = $ratio" ) + if (ratio >= scalingUpRatio) { + logDebug("Requesting executors") + val numNewExecutors = math.max(math.round(ratio).toInt, 1) + requestExecutors(numNewExecutors) + } else if (ratio <= scalingDownRatio) { + logDebug("Killing executors") + killExecutor() + } + } + batchProcTimeSum = 0 + batchProcTimeCount = 0 + } + + /** Request the specified number of executors over the currently active one */ + private def requestExecutors(numNewExecutors: Int): Unit = { + require(numNewExecutors >= 1) + val allExecIds = client.getExecutorIds() + logDebug(s"Executors (${allExecIds.size}) = ${allExecIds}") + val targetTotalExecutors = + math.max(math.min(maxNumExecutors, allExecIds.size + numNewExecutors), minNumExecutors) + client.requestTotalExecutors(targetTotalExecutors, 0, Map.empty) + logInfo(s"Requested total $targetTotalExecutors executors") + } + + /** Kill an executor that is not running any receiver, if possible */ + private def killExecutor(): Unit = { + val allExecIds = client.getExecutorIds() + logDebug(s"Executors (${allExecIds.size}) = ${allExecIds}") + + if (allExecIds.nonEmpty && allExecIds.size > minNumExecutors) { + val execIdsWithReceivers = receiverTracker.allocatedExecutors.values.flatten.toSeq + logInfo(s"Executors with receivers (${execIdsWithReceivers.size}): ${execIdsWithReceivers}") + + val removableExecIds = allExecIds.diff(execIdsWithReceivers) + logDebug(s"Removable executors (${removableExecIds.size}): ${removableExecIds}") + if (removableExecIds.nonEmpty) { + val execIdToRemove = removableExecIds(Random.nextInt(removableExecIds.size)) + client.killExecutor(execIdToRemove) + logInfo(s"Requested to kill executor $execIdToRemove") + } else { + logInfo(s"No non-receiver executors to kill") + } + } else { + logInfo("No available executor to kill") + } + } + + private def addBatchProcTime(timeMs: Long): Unit = synchronized { + batchProcTimeSum += timeMs + batchProcTimeCount += 1 + logDebug( + s"Added batch processing time $timeMs, sum = $batchProcTimeSum, count = $batchProcTimeCount") + } + + private def validateSettings(): Unit = { + require( + scalingIntervalSecs > 0, + s"Config $SCALING_INTERVAL_KEY must be more than 0") + + require( + scalingUpRatio > 0, + s"Config $SCALING_UP_RATIO_KEY must be more than 0") + + require( + scalingDownRatio > 0, + s"Config $SCALING_DOWN_RATIO_KEY must be more than 0") + + require( + minNumExecutors > 0, + s"Config $MIN_EXECUTORS_KEY must be more than 0") + + require( + maxNumExecutors > 0, + s"$MAX_EXECUTORS_KEY must be more than 0") + + require( + scalingUpRatio > scalingDownRatio, + s"Config $SCALING_UP_RATIO_KEY must be more than config $SCALING_DOWN_RATIO_KEY") + + if (conf.contains(MIN_EXECUTORS_KEY) && conf.contains(MAX_EXECUTORS_KEY)) { + require( + maxNumExecutors >= minNumExecutors, + s"Config $MAX_EXECUTORS_KEY must be more than config $MIN_EXECUTORS_KEY") + } + } + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { + logDebug("onBatchCompleted called: " + batchCompleted) + if (!batchCompleted.batchInfo.outputOperationInfos.values.exists(_.failureReason.nonEmpty)) { + batchCompleted.batchInfo.processingDelay.foreach(addBatchProcTime) + } + } +} + +private[streaming] object ExecutorAllocationManager extends Logging { + val ENABLED_KEY = "spark.streaming.dynamicAllocation.enabled" + + val SCALING_INTERVAL_KEY = "spark.streaming.dynamicAllocation.scalingInterval" + val SCALING_INTERVAL_DEFAULT_SECS = 60 + + val SCALING_UP_RATIO_KEY = "spark.streaming.dynamicAllocation.scalingUpRatio" + val SCALING_UP_RATIO_DEFAULT = 0.9 + + val SCALING_DOWN_RATIO_KEY = "spark.streaming.dynamicAllocation.scalingDownRatio" + val SCALING_DOWN_RATIO_DEFAULT = 0.3 + + val MIN_EXECUTORS_KEY = "spark.streaming.dynamicAllocation.minExecutors" + + val MAX_EXECUTORS_KEY = "spark.streaming.dynamicAllocation.maxExecutors" + + def isDynamicAllocationEnabled(conf: SparkConf): Boolean = { + val numExecutor = conf.getInt("spark.executor.instances", 0) + val streamingDynamicAllocationEnabled = conf.getBoolean(ENABLED_KEY, false) + if (numExecutor != 0 && streamingDynamicAllocationEnabled) { + throw new IllegalArgumentException( + "Dynamic Allocation for streaming cannot be enabled while spark.executor.instances is set.") + } + if (Utils.isDynamicAllocationEnabled(conf) && streamingDynamicAllocationEnabled) { + throw new IllegalArgumentException( + """ + |Dynamic Allocation cannot be enabled for both streaming and core at the same time. + |Please disable core Dynamic Allocation by setting spark.dynamicAllocation.enabled to + |false to use Dynamic Allocation in streaming. + """.stripMargin) + } + val testing = conf.getBoolean("spark.streaming.dynamicAllocation.testing", false) + numExecutor == 0 && streamingDynamicAllocationEnabled && (!Utils.isLocalMaster(conf) || testing) + } + + def createIfEnabled( + client: ExecutorAllocationClient, + receiverTracker: ReceiverTracker, + conf: SparkConf, + batchDurationMs: Long, + clock: Clock): Option[ExecutorAllocationManager] = { + if (isDynamicAllocationEnabled(conf) && client != null) { + Some(new ExecutorAllocationManager(client, receiverTracker, conf, batchDurationMs, clock)) + } else None + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala index deb15d075975..639ac6de4f5d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala @@ -19,9 +19,9 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable -import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.streaming.{Time, StreamingContext} +import org.apache.spark.internal.Logging +import org.apache.spark.streaming.{StreamingContext, Time} /** * :: DeveloperApi :: @@ -66,8 +66,8 @@ private[streaming] class InputInfoTracker(ssc: StreamingContext) extends Logging new mutable.HashMap[Int, StreamInputInfo]()) if (inputInfos.contains(inputInfo.inputStreamId)) { - throw new IllegalStateException(s"Input stream ${inputInfo.inputStreamId} for batch" + - s"$batchTime is already added into InputInfoTracker, this is a illegal state") + throw new IllegalStateException(s"Input stream ${inputInfo.inputStreamId} for batch " + + s"$batchTime is already added into InputInfoTracker, this is an illegal state") } inputInfos += ((inputInfo.inputStreamId, inputInfo)) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala index ab1b3565fcc1..7050d7ef4524 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala @@ -20,7 +20,7 @@ package org.apache.spark.streaming.scheduler import scala.util.{Failure, Try} import org.apache.spark.streaming.Time -import org.apache.spark.util.{Utils, CallSite} +import org.apache.spark.util.{CallSite, Utils} /** * Class representing a Spark computation. It may contain multiple Spark jobs. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 2de035d166e7..8d83dc8a8fc0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -19,10 +19,12 @@ package org.apache.spark.streaming.scheduler import scala.util.{Failure, Success, Try} -import org.apache.spark.{SparkEnv, Logging} +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Checkpoint, CheckpointWriter, Time} +import org.apache.spark.streaming.api.python.PythonDStream import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.{Utils, Clock, EventLoop, ManualClock} +import org.apache.spark.util.{Clock, EventLoop, ManualClock, Utils} /** Event classes for JobGenerator */ private[scheduler] sealed trait JobGeneratorEvent @@ -152,9 +154,9 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { graph.stop() } - // Stop the event loop and checkpoint writer - if (shouldCheckpoint) checkpointWriter.stop() + // First stop the event loop, then stop the checkpoint writer; see SPARK-14701 eventLoop.stop() + if (shouldCheckpoint) checkpointWriter.stop() logInfo("Stopped JobGenerator") } @@ -217,11 +219,12 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { // Batches that were unprocessed before failure val pendingTimes = ssc.initialCheckpoint.pendingTimes.sorted(Time.ordering) - logInfo("Batches pending processing (" + pendingTimes.size + " batches): " + + logInfo("Batches pending processing (" + pendingTimes.length + " batches): " + pendingTimes.mkString(", ")) // Reschedule jobs for these times - val timesToReschedule = (pendingTimes ++ downTimes).distinct.sorted(Time.ordering) - logInfo("Batches to reschedule (" + timesToReschedule.size + " batches): " + + val timesToReschedule = (pendingTimes ++ downTimes).filter { _ < restartTime } + .distinct.sorted(Time.ordering) + logInfo("Batches to reschedule (" + timesToReschedule.length + " batches): " + timesToReschedule.mkString(", ")) timesToReschedule.foreach { time => // Allocate the related blocks when recovering from failure, because some blocks that were @@ -236,12 +239,11 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { logInfo("Restarted JobGenerator at " + restartTime) } - /** Generate jobs and perform checkpoint for the given `time`. */ + /** Generate jobs and perform checkpointing for the given `time`. */ private def generateJobs(time: Time) { - // Set the SparkEnv in this thread, so that job generation code can access the environment - // Example: BlockRDDs are created in this thread, and it needs to access BlockManager - // Update: This is probably redundant after threadlocal stuff in SparkEnv has been removed. - SparkEnv.set(ssc.env) + // Checkpoint all RDDs marked for checkpointing to ensure their lineages are + // truncated periodically. Otherwise, we may run into stack overflows (SPARK-6847). + ssc.sparkContext.setLocalProperty(RDD.CHECKPOINT_ALL_MARKED_ANCESTORS, "true") Try { jobScheduler.receiverTracker.allocateBlocksToBatch(time) // allocate received blocks to batch graph.generateJobs(time) // generate jobs using allocated block @@ -251,6 +253,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { jobScheduler.submitJobSet(JobSet(time, jobs, streamIdToInputInfos)) case Failure(e) => jobScheduler.reportError("Error generating jobs for time " + time, e) + PythonDStream.stopStreamingContextIfPythonProcessIsDead(e) } eventLoop.post(DoCheckpoint(time, clearCheckpointDataLater = false)) } @@ -286,12 +289,14 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { markBatchFullyProcessed(time) } - /** Perform checkpoint for the give `time`. */ + /** Perform checkpoint for the given `time`. */ private def doCheckpoint(time: Time, clearCheckpointDataLater: Boolean) { if (shouldCheckpoint && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) { logInfo("Checkpointing graph for time " + time) ssc.graph.updateCheckpointData(time) checkpointWriter.write(new Checkpoint(ssc, time), clearCheckpointDataLater) + } else if (clearCheckpointDataLater) { + markBatchFullyProcessed(time) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 2480b4ec093e..2fa3bf7d5230 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -22,11 +22,16 @@ import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import scala.collection.JavaConverters._ import scala.util.Failure -import org.apache.spark.Logging -import org.apache.spark.rdd.PairRDDFunctions +import org.apache.commons.lang3.SerializationUtils + +import org.apache.spark.ExecutorAllocationClient +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io.SparkHadoopWriterUtils +import org.apache.spark.rdd.RDD import org.apache.spark.streaming._ +import org.apache.spark.streaming.api.python.PythonDStream import org.apache.spark.streaming.ui.UIUtils -import org.apache.spark.util.{EventLoop, ThreadUtils, Utils} +import org.apache.spark.util.{EventLoop, ThreadUtils} private[scheduler] sealed trait JobSchedulerEvent @@ -49,7 +54,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { ThreadUtils.newDaemonFixedThreadPool(numConcurrentJobs, "streaming-job-executor") private val jobGenerator = new JobGenerator(this) val clock = jobGenerator.clock - val listenerBus = new StreamingListenerBus() + val listenerBus = new StreamingListenerBus(ssc.sparkContext.listenerBus) // These two are created only when scheduler starts. // eventLoop not being null means the scheduler has been started and not stopped @@ -57,6 +62,8 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { // A tracker to track all the input stream information as well as processed record number var inputInfoTracker: InputInfoTracker = null + private var executorAllocationManager: Option[ExecutorAllocationManager] = None + private var eventLoop: EventLoop[JobSchedulerEvent] = null def start(): Unit = synchronized { @@ -76,11 +83,25 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { rateController <- inputDStream.rateController } ssc.addStreamingListener(rateController) - listenerBus.start(ssc.sparkContext) + listenerBus.start() receiverTracker = new ReceiverTracker(ssc) inputInfoTracker = new InputInfoTracker(ssc) + + val executorAllocClient: ExecutorAllocationClient = ssc.sparkContext.schedulerBackend match { + case b: ExecutorAllocationClient => b.asInstanceOf[ExecutorAllocationClient] + case _ => null + } + + executorAllocationManager = ExecutorAllocationManager.createIfEnabled( + executorAllocClient, + receiverTracker, + ssc.conf, + ssc.graph.batchDuration.milliseconds, + clock) + executorAllocationManager.foreach(ssc.addStreamingListener) receiverTracker.start() jobGenerator.start() + executorAllocationManager.foreach(_.start()) logInfo("Started JobScheduler") } @@ -88,8 +109,14 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { if (eventLoop == null) return // scheduler has already been stopped logDebug("Stopping JobScheduler") - // First, stop receiving - receiverTracker.stop(processAllReceivedData) + if (receiverTracker != null) { + // First, stop receiving + receiverTracker.stop(processAllReceivedData) + } + + if (executorAllocationManager != null) { + executorAllocationManager.foreach(_.stop()) + } // Second, stop generating jobs. If it has to process all received data, // then this will wait for all the processing through JobScheduler to be over. @@ -174,31 +201,36 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { listenerBus.post(StreamingListenerOutputOperationCompleted(job.toOutputOperationInfo)) logInfo("Finished job " + job.id + " from job set of time " + jobSet.time) if (jobSet.hasCompleted) { - jobSets.remove(jobSet.time) - jobGenerator.onBatchCompletion(jobSet.time) - logInfo("Total delay: %.3f s for time %s (execution: %.3f s)".format( - jobSet.totalDelay / 1000.0, jobSet.time.toString, - jobSet.processingDelay / 1000.0 - )) listenerBus.post(StreamingListenerBatchCompleted(jobSet.toBatchInfo)) } job.result match { case Failure(e) => reportError("Error running job " + job, e) case _ => + if (jobSet.hasCompleted) { + jobSets.remove(jobSet.time) + jobGenerator.onBatchCompletion(jobSet.time) + logInfo("Total delay: %.3f s for time %s (execution: %.3f s)".format( + jobSet.totalDelay / 1000.0, jobSet.time.toString, + jobSet.processingDelay / 1000.0 + )) + } } } private def handleError(msg: String, e: Throwable) { logError(msg, e) ssc.waiter.notifyError(e) + PythonDStream.stopStreamingContextIfPythonProcessIsDead(e) } private class JobHandler(job: Job) extends Runnable with Logging { import JobScheduler._ def run() { + val oldProps = ssc.sparkContext.getLocalProperties try { + ssc.sparkContext.setLocalProperties(SerializationUtils.clone(ssc.savedProperties.get())) val formattedTime = UIUtils.formatBatchTime( job.time.milliseconds, ssc.graph.batchDuration.milliseconds, showYYYYMMSS = false) val batchUrl = s"/streaming/batch/?id=${job.time.milliseconds}" @@ -208,6 +240,9 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { s"""Streaming job from $batchLinkText""") ssc.sc.setLocalProperty(BATCH_TIME_PROPERTY_KEY, job.time.milliseconds.toString) ssc.sc.setLocalProperty(OUTPUT_OP_ID_PROPERTY_KEY, job.outputOpId.toString) + // Checkpoint all RDDs marked for checkpointing to ensure their lineages are + // truncated periodically. Otherwise, we may run into stack overflows (SPARK-6847). + ssc.sparkContext.setLocalProperty(RDD.CHECKPOINT_ALL_MARKED_ANCESTORS, "true") // We need to assign `eventLoop` to a temp variable. Otherwise, because // `JobScheduler.stop(false)` may set `eventLoop` to null when this method is running, then @@ -218,7 +253,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { // Disable checks for existing output directories in jobs launched by the streaming // scheduler, since we may need to write output to an existing directory during checkpoint // recovery; see SPARK-4835 for more details. - PairRDDFunctions.disableOutputSpecValidation.withValue(true) { + SparkHadoopWriterUtils.disableOutputSpecValidation.withValue(true) { job.run() } _eventLoop = eventLoop @@ -229,8 +264,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { // JobScheduler has been stopped. } } finally { - ssc.sc.setLocalProperty(JobScheduler.BATCH_TIME_PROPERTY_KEY, null) - ssc.sc.setLocalProperty(JobScheduler.OUTPUT_OP_ID_PROPERTY_KEY, null) + ssc.sparkContext.setLocalProperties(oldProps) } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala index f76300351e3c..0baedaf275d6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala @@ -18,14 +18,13 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable.HashSet -import scala.util.Failure import org.apache.spark.streaming.Time -import org.apache.spark.util.Utils -/** Class representing a set of Jobs - * belong to the same batch. - */ +/** + * Class representing a set of Jobs + * belong to the same batch. + */ private[streaming] case class JobSet( time: Time, @@ -59,17 +58,15 @@ case class JobSet( // Time taken to process all the jobs from the time they were submitted // (i.e. including the time they wait in the streaming scheduler queue) - def totalDelay: Long = { - processingEndTime - time.milliseconds - } + def totalDelay: Long = processingEndTime - time.milliseconds def toBatchInfo: BatchInfo = { BatchInfo( time, streamIdToInputInfo, submissionTime, - if (processingStartTime >= 0) Some(processingStartTime) else None, - if (processingEndTime >= 0) Some(processingEndTime) else None, + if (hasStarted) Some(processingStartTime) else None, + if (hasCompleted) Some(processingEndTime) else None, jobs.map { job => (job.outputOpId, job.toOutputOperationInfo) }.toMap ) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index f2711d1355e6..5d9a8ac0d929 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -22,14 +22,17 @@ import java.nio.ByteBuffer import scala.collection.JavaConverters._ import scala.collection.mutable import scala.language.implicitConversions +import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.network.util.JavaUtils import org.apache.spark.streaming.Time import org.apache.spark.streaming.util.{WriteAheadLog, WriteAheadLogUtils} import org.apache.spark.util.{Clock, Utils} -import org.apache.spark.{Logging, SparkConf} /** Trait representing any event in the ReceivedBlockTracker that updates its state. */ private[streaming] sealed trait ReceivedBlockTrackerLogEvent @@ -41,7 +44,6 @@ private[streaming] case class BatchAllocationEvent(time: Time, allocatedBlocks: private[streaming] case class BatchCleanupEvent(times: Seq[Time]) extends ReceivedBlockTrackerLogEvent - /** Class representing the blocks of all the streams allocated to a batch */ private[streaming] case class AllocatedBlocks(streamIdToAllocatedBlocks: Map[Int, Seq[ReceivedBlockInfo]]) { @@ -82,15 +84,22 @@ private[streaming] class ReceivedBlockTracker( } /** Add received block. This event will get written to the write ahead log (if enabled). */ - def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = synchronized { + def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = { try { - writeToLog(BlockAdditionEvent(receivedBlockInfo)) - getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo - logDebug(s"Stream ${receivedBlockInfo.streamId} received " + - s"block ${receivedBlockInfo.blockStoreResult.blockId}") - true + val writeResult = writeToLog(BlockAdditionEvent(receivedBlockInfo)) + if (writeResult) { + synchronized { + getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo + } + logDebug(s"Stream ${receivedBlockInfo.streamId} received " + + s"block ${receivedBlockInfo.blockStoreResult.blockId}") + } else { + logDebug(s"Failed to acknowledge stream ${receivedBlockInfo.streamId} receiving " + + s"block ${receivedBlockInfo.blockStoreResult.blockId} in the Write Ahead Log.") + } + writeResult } catch { - case e: Exception => + case NonFatal(e) => logError(s"Error adding block $receivedBlockInfo", e) false } @@ -106,10 +115,12 @@ private[streaming] class ReceivedBlockTracker( (streamId, getReceivedBlockQueue(streamId).dequeueAll(x => true)) }.toMap val allocatedBlocks = AllocatedBlocks(streamIdToBlocks) - writeToLog(BatchAllocationEvent(batchTime, allocatedBlocks)) - timeToAllocatedBlocks(batchTime) = allocatedBlocks - lastAllocatedBatchTime = batchTime - allocatedBlocks + if (writeToLog(BatchAllocationEvent(batchTime, allocatedBlocks))) { + timeToAllocatedBlocks.put(batchTime, allocatedBlocks) + lastAllocatedBatchTime = batchTime + } else { + logInfo(s"Possibly processed batch $batchTime needs to be processed again in WAL recovery") + } } else { // This situation occurs when: // 1. WAL is ended with BatchAllocationEvent, but without BatchCleanupEvent, @@ -118,7 +129,7 @@ private[streaming] class ReceivedBlockTracker( // 2. Slow checkpointing makes recovered batch time older than WAL recovered // lastAllocatedBatchTime. // This situation will only occurs in recovery time. - logInfo(s"Possibly processed batch $batchTime need to be processed again in WAL recovery") + logInfo(s"Possibly processed batch $batchTime needs to be processed again in WAL recovery") } } @@ -156,10 +167,13 @@ private[streaming] class ReceivedBlockTracker( def cleanupOldBatches(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = synchronized { require(cleanupThreshTime.milliseconds < clock.getTimeMillis()) val timesToCleanup = timeToAllocatedBlocks.keys.filter { _ < cleanupThreshTime }.toSeq - logInfo("Deleting batches " + timesToCleanup) - writeToLog(BatchCleanupEvent(timesToCleanup)) - timeToAllocatedBlocks --= timesToCleanup - writeAheadLogOption.foreach(_.clean(cleanupThreshTime.milliseconds, waitForCompletion)) + logInfo(s"Deleting batches: ${timesToCleanup.mkString(" ")}") + if (writeToLog(BatchCleanupEvent(timesToCleanup))) { + timeToAllocatedBlocks --= timesToCleanup + writeAheadLogOption.foreach(_.clean(cleanupThreshTime.milliseconds, waitForCompletion)) + } else { + logWarning("Failed to acknowledge batch clean up in the Write Ahead Log.") + } } /** Stop the block tracker. */ @@ -185,8 +199,8 @@ private[streaming] class ReceivedBlockTracker( logTrace(s"Recovery: Inserting allocated batch for time $batchTime to " + s"${allocatedBlocks.streamIdToAllocatedBlocks}") streamIdToUnallocatedBlockQueues.values.foreach { _.clear() } - lastAllocatedBatchTime = batchTime timeToAllocatedBlocks.put(batchTime, allocatedBlocks) + lastAllocatedBatchTime = batchTime } // Cleanup the batch allocations @@ -198,9 +212,9 @@ private[streaming] class ReceivedBlockTracker( writeAheadLogOption.foreach { writeAheadLog => logInfo(s"Recovering from write ahead logs in ${checkpointDirOption.get}") writeAheadLog.readAll().asScala.foreach { byteBuffer => - logTrace("Recovering record " + byteBuffer) + logInfo("Recovering record " + byteBuffer) Utils.deserialize[ReceivedBlockTrackerLogEvent]( - byteBuffer.array, Thread.currentThread().getContextClassLoader) match { + JavaUtils.bufferToArray(byteBuffer), Thread.currentThread().getContextClassLoader) match { case BlockAdditionEvent(receivedBlockInfo) => insertAddedBlock(receivedBlockInfo) case BatchAllocationEvent(time, allocatedBlocks) => @@ -213,12 +227,20 @@ private[streaming] class ReceivedBlockTracker( } /** Write an update to the tracker to the write ahead log */ - private def writeToLog(record: ReceivedBlockTrackerLogEvent) { + private def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = { if (isWriteAheadLogEnabled) { - logDebug(s"Writing to log $record") - writeAheadLogOption.foreach { logManager => - logManager.write(ByteBuffer.wrap(Utils.serialize(record)), clock.getTimeMillis()) + logTrace(s"Writing record: $record") + try { + writeAheadLogOption.get.write(ByteBuffer.wrap(Utils.serialize(record)), + clock.getTimeMillis()) + true + } catch { + case NonFatal(e) => + logWarning(s"Exception thrown while writing record: $record to the WriteAheadLog.", e) + false } + } else { + true } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala index 59df892397fe..d16e158da35c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala @@ -18,7 +18,6 @@ package org.apache.spark.streaming.scheduler import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.rpc.RpcEndpointRef /** * :: DeveloperApi :: @@ -30,6 +29,7 @@ case class ReceiverInfo( name: String, active: Boolean, location: String, + executorId: String, lastErrorMessage: String = "", lastError: String = "", lastErrorTime: Long = -1L diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala index 234bc8660da8..4105171a3db2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala @@ -27,28 +27,29 @@ import org.apache.spark.streaming.receiver.Receiver * A class that tries to schedule receivers with evenly distributed. There are two phases for * scheduling receivers. * - * - The first phase is global scheduling when ReceiverTracker is starting and we need to schedule - * all receivers at the same time. ReceiverTracker will call `scheduleReceivers` at this phase. - * It will try to schedule receivers such that they are evenly distributed. ReceiverTracker should - * update its `receiverTrackingInfoMap` according to the results of `scheduleReceivers`. - * `ReceiverTrackingInfo.scheduledLocations` for each receiver should be set to an location list - * that contains the scheduled locations. Then when a receiver is starting, it will send a - * register request and `ReceiverTracker.registerReceiver` will be called. In - * `ReceiverTracker.registerReceiver`, if a receiver's scheduled locations is set, it should check - * if the location of this receiver is one of the scheduled locations, if not, the register will - * be rejected. - * - The second phase is local scheduling when a receiver is restarting. There are two cases of - * receiver restarting: - * - If a receiver is restarting because it's rejected due to the real location and the scheduled - * locations mismatching, in other words, it fails to start in one of the locations that - * `scheduleReceivers` suggested, `ReceiverTracker` should firstly choose the executors that are - * still alive in the list of scheduled locations, then use them to launch the receiver job. - * - If a receiver is restarting without a scheduled locations list, or the executors in the list - * are dead, `ReceiverTracker` should call `rescheduleReceiver`. If so, `ReceiverTracker` should - * not set `ReceiverTrackingInfo.scheduledLocations` for this receiver, instead, it should clear - * it. Then when this receiver is registering, we can know this is a local scheduling, and - * `ReceiverTrackingInfo` should call `rescheduleReceiver` again to check if the launching - * location is matching. + * - The first phase is global scheduling when ReceiverTracker is starting and we need to schedule + * all receivers at the same time. ReceiverTracker will call `scheduleReceivers` at this phase. + * It will try to schedule receivers such that they are evenly distributed. ReceiverTracker + * should update its `receiverTrackingInfoMap` according to the results of `scheduleReceivers`. + * `ReceiverTrackingInfo.scheduledLocations` for each receiver should be set to a location list + * that contains the scheduled locations. Then when a receiver is starting, it will send a + * register request and `ReceiverTracker.registerReceiver` will be called. In + * `ReceiverTracker.registerReceiver`, if a receiver's scheduled locations is set, it should + * check if the location of this receiver is one of the scheduled locations, if not, the register + * will be rejected. + * - The second phase is local scheduling when a receiver is restarting. There are two cases of + * receiver restarting: + * - If a receiver is restarting because it's rejected due to the real location and the scheduled + * locations mismatching, in other words, it fails to start in one of the locations that + * `scheduleReceivers` suggested, `ReceiverTracker` should firstly choose the executors that + * are still alive in the list of scheduled locations, then use them to launch the receiver + * job. + * - If a receiver is restarting without a scheduled locations list, or the executors in the list + * are dead, `ReceiverTracker` should call `rescheduleReceiver`. If so, `ReceiverTracker` + * should not set `ReceiverTrackingInfo.scheduledLocations` for this receiver, instead, it + * should clear it. Then when this receiver is registering, we can know this is a local + * scheduling, and `ReceiverTrackingInfo` should call `rescheduleReceiver` again to check if + * the launching location is matching. * * In conclusion, we should make a global schedule, try to achieve that exactly as long as possible, * otherwise do local scheduling. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index b183d856f50c..6f130c803f31 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -25,9 +25,10 @@ import scala.language.existentials import scala.util.{Failure, Success} import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.rpc._ -import org.apache.spark.scheduler.{TaskLocation, ExecutorCacheTaskLocation} +import org.apache.spark.scheduler.{ExecutorCacheTaskLocation, TaskLocation} import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.receiver._ import org.apache.spark.streaming.util.WriteAheadLogUtils @@ -91,6 +92,8 @@ private[streaming] case object AllReceiverIds extends ReceiverTrackerLocalMessag private[streaming] case class UpdateReceiverRateLimit(streamUID: Int, newRate: Long) extends ReceiverTrackerLocalMessage +private[streaming] case object GetAllReceiverInfo extends ReceiverTrackerLocalMessage + /** * This class manages the execution of the receivers of ReceiverInputDStreams. Instance of * this class must be created after all input streams have been added and StreamingContext.start() @@ -131,7 +134,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // Track the active receiver job number. When a receiver job exits ultimately, countDown will // be called. - private val receiverJobExitLatch = new CountDownLatch(receiverInputStreams.size) + private val receiverJobExitLatch = new CountDownLatch(receiverInputStreams.length) /** * Track all receivers' information. The key is the receiver id, the value is the receiver info. @@ -162,12 +165,12 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** Stop the receiver execution thread. */ def stop(graceful: Boolean): Unit = synchronized { - if (isTrackerStarted) { - // First, stop the receivers - trackerState = Stopping + val isStarted: Boolean = isTrackerStarted + trackerState = Stopping + if (isStarted) { if (!skipReceiverLaunch) { - // Send the stop signal to all the receivers - endpoint.askWithRetry[Boolean](StopAllReceivers) + // First, stop the receivers. Send the stop signal to all the receivers + endpoint.askSync[Boolean](StopAllReceivers) // Wait for the Spark job that runs the receivers to be over // That is, for the receivers to quit gracefully. @@ -180,7 +183,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } // Check if all the receivers have been deregistered or not - val receivers = endpoint.askWithRetry[Seq[Int]](AllReceiverIds) + val receivers = endpoint.askSync[Seq[Int]](AllReceiverIds) if (receivers.nonEmpty) { logWarning("Not all of the receivers have deregistered, " + receivers) } else { @@ -191,10 +194,13 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // Finally, stop the endpoint ssc.env.rpcEnv.stop(endpoint) endpoint = null - receivedBlockTracker.stop() - logInfo("ReceiverTracker stopped") - trackerState = Stopped } + + // `ReceivedBlockTracker` is open when this instance is created. We should + // close this even if this `ReceiverTracker` is not started. + receivedBlockTracker.stop() + logInfo("ReceiverTracker stopped") + trackerState = Stopped } /** Allocate all unallocated blocks to the given batch. */ @@ -233,6 +239,26 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } } + /** + * Get the executors allocated to each receiver. + * @return a map containing receiver ids to optional executor ids. + */ + def allocatedExecutors(): Map[Int, Option[String]] = synchronized { + if (isTrackerStarted) { + endpoint.askSync[Map[Int, ReceiverTrackingInfo]](GetAllReceiverInfo).mapValues { + _.runningExecutor.map { + _.executorId + } + } + } else { + Map.empty + } + } + + def numReceivers(): Int = { + receiverInputStreams.size + } + /** Register a receiver */ private def registerReceiver( streamId: Int, @@ -411,11 +437,11 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false * worker nodes as a parallel collection, and runs them. */ private def launchReceivers(): Unit = { - val receivers = receiverInputStreams.map(nis => { + val receivers = receiverInputStreams.map { nis => val rcvr = nis.getReceiver() rcvr.setReceiverId(nis.id) rcvr - }) + } runDummySparkJob() @@ -435,9 +461,10 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** RpcEndpoint to receive messages from the receivers. */ private class ReceiverTrackerEndpoint(override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint { - // TODO Remove this thread pool after https://github.com/apache/spark/issues/7385 is merged - private val submitJobThreadPool = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("submit-job-thead-pool")) + private val walBatchingThreadPool = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("wal-batching-thread-pool")) + + @volatile private var active: Boolean = true override def receive: PartialFunction[Any, Unit] = { // Local messages @@ -488,13 +515,28 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false registerReceiver(streamId, typ, host, executorId, receiverEndpoint, context.senderAddress) context.reply(successful) case AddBlock(receivedBlockInfo) => - context.reply(addBlock(receivedBlockInfo)) + if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf, isDriver = true)) { + walBatchingThreadPool.execute(new Runnable { + override def run(): Unit = Utils.tryLogNonFatalError { + if (active) { + context.reply(addBlock(receivedBlockInfo)) + } else { + throw new IllegalStateException("ReceiverTracker RpcEndpoint shut down.") + } + } + }) + } else { + context.reply(addBlock(receivedBlockInfo)) + } case DeregisterReceiver(streamId, message, error) => deregisterReceiver(streamId, message, error) context.reply(true) + // Local messages case AllReceiverIds => context.reply(receiverTrackingInfos.filter(_._2.state != ReceiverState.INACTIVE).keys.toSeq) + case GetAllReceiverInfo => + context.reply(receiverTrackingInfos.toMap) case StopAllReceivers => assert(isTrackerStopping || isTrackerStopped) stopReceivers() @@ -593,12 +635,13 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false logInfo(s"Restarting Receiver $receiverId") self.send(RestartReceiver(receiver)) } - }(submitJobThreadPool) + }(ThreadUtils.sameThread) logInfo(s"Receiver ${receiver.streamId} started") } override def onStop(): Unit = { - submitJobThreadPool.shutdownNow() + active = false + walBatchingThreadPool.shutdown() } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTrackingInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTrackingInfo.scala index ab0a84f05214..4dc5bb9c3bfb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTrackingInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTrackingInfo.scala @@ -49,6 +49,7 @@ private[streaming] case class ReceiverTrackingInfo( name.getOrElse(""), state == ReceiverState.ACTIVE, location = runningExecutor.map(_.host).getOrElse(""), + executorId = runningExecutor.map(_.executorId).getOrElse(""), lastErrorMessage = errorInfo.map(_.lastErrorMessage).getOrElse(""), lastError = errorInfo.map(_.lastError).getOrElse(""), lastErrorTime = errorInfo.map(_.lastErrorTime).getOrElse(-1L) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala index d19bdbb443c5..b57f9b772f8c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala @@ -19,8 +19,8 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable.Queue -import org.apache.spark.util.Distribution import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Distribution /** * :: DeveloperApi :: @@ -29,6 +29,9 @@ import org.apache.spark.annotation.DeveloperApi @DeveloperApi sealed trait StreamingListenerEvent +@DeveloperApi +case class StreamingListenerStreamingStarted(time: Long) extends StreamingListenerEvent + @DeveloperApi case class StreamingListenerBatchSubmitted(batchInfo: BatchInfo) extends StreamingListenerEvent @@ -66,6 +69,9 @@ case class StreamingListenerReceiverStopped(receiverInfo: ReceiverInfo) @DeveloperApi trait StreamingListener { + /** Called when the streaming has been started */ + def onStreamingStarted(streamingStarted: StreamingListenerStreamingStarted) { } + /** Called when a receiver has been started */ def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala index ca111bb636ed..6a70bf7406b3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala @@ -17,19 +17,37 @@ package org.apache.spark.streaming.scheduler -import java.util.concurrent.atomic.AtomicBoolean +import org.apache.spark.scheduler.{LiveListenerBus, SparkListener, SparkListenerEvent} +import org.apache.spark.util.ListenerBus -import org.apache.spark.Logging -import org.apache.spark.util.AsynchronousListenerBus +/** + * A Streaming listener bus to forward events to StreamingListeners. This one will wrap received + * Streaming events as WrappedStreamingListenerEvent and send them to Spark listener bus. It also + * registers itself with Spark listener bus, so that it can receive WrappedStreamingListenerEvents, + * unwrap them as StreamingListenerEvent and dispatch them to StreamingListeners. + */ +private[streaming] class StreamingListenerBus(sparkListenerBus: LiveListenerBus) + extends SparkListener with ListenerBus[StreamingListener, StreamingListenerEvent] { -/** Asynchronously passes StreamingListenerEvents to registered StreamingListeners. */ -private[spark] class StreamingListenerBus - extends AsynchronousListenerBus[StreamingListener, StreamingListenerEvent]("StreamingListenerBus") - with Logging { + /** + * Post a StreamingListenerEvent to the Spark listener bus asynchronously. This event will be + * dispatched to all StreamingListeners in the thread of the Spark listener bus. + */ + def post(event: StreamingListenerEvent) { + sparkListenerBus.post(new WrappedStreamingListenerEvent(event)) + } - private val logDroppedEvent = new AtomicBoolean(false) + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case WrappedStreamingListenerEvent(e) => + postToAll(e) + case _ => + } + } - override def onPostEvent(listener: StreamingListener, event: StreamingListenerEvent): Unit = { + protected override def doPostEvent( + listener: StreamingListener, + event: StreamingListenerEvent): Unit = { event match { case receiverStarted: StreamingListenerReceiverStarted => listener.onReceiverStarted(receiverStarted) @@ -47,16 +65,37 @@ private[spark] class StreamingListenerBus listener.onOutputOperationStarted(outputOperationStarted) case outputOperationCompleted: StreamingListenerOutputOperationCompleted => listener.onOutputOperationCompleted(outputOperationCompleted) + case streamingStarted: StreamingListenerStreamingStarted => + listener.onStreamingStarted(streamingStarted) case _ => } } - override def onDropEvent(event: StreamingListenerEvent): Unit = { - if (logDroppedEvent.compareAndSet(false, true)) { - // Only log the following message once to avoid duplicated annoying logs. - logError("Dropping StreamingListenerEvent because no remaining room in event queue. " + - "This likely means one of the StreamingListeners is too slow and cannot keep up with the " + - "rate at which events are being started by the scheduler.") - } + /** + * Register this one with the Spark listener bus so that it can receive Streaming events and + * forward them to StreamingListeners. + */ + def start(): Unit = { + sparkListenerBus.addToStatusQueue(this) + } + + /** + * Unregister this one with the Spark listener bus and all StreamingListeners won't receive any + * events after that. + */ + def stop(): Unit = { + sparkListenerBus.removeListener(this) + } + + /** + * Wrapper for StreamingListenerEvent as SparkListenerEvent so that it can be posted to Spark + * listener bus. + */ + private case class WrappedStreamingListenerEvent(streamingListenerEvent: StreamingListenerEvent) + extends SparkListenerEvent { + + // Do not log streaming events in event log as history server does not support streaming + // events (SPARK-12140). TODO Once SPARK-12140 is resolved we should set it to true. + protected[spark] override def logEvent: Boolean = false } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala index 84a3ca9d74e5..dc02062b9eb4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.scheduler.rate -import org.apache.spark.Logging +import org.apache.spark.internal.Logging /** * Implements a proportional-integral-derivative (PID) controller which acts on @@ -26,7 +26,7 @@ import org.apache.spark.Logging * case of Spark Streaming the error is the difference between the measured processing * rate (number of elements/processing delay) and the previous rate. * - * @see https://en.wikipedia.org/wiki/PID_controller + * @see PID controller (Wikipedia) * * @param batchIntervalMillis the batch duration, in milliseconds * @param proportional how much the correction should depend on the current diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/RateEstimator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/RateEstimator.scala index d7210f64fcc3..e4b9dffee04f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/RateEstimator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/RateEstimator.scala @@ -21,18 +21,20 @@ import org.apache.spark.SparkConf import org.apache.spark.streaming.Duration /** - * A component that estimates the rate at wich an InputDStream should ingest - * elements, based on updates at every batch completion. + * A component that estimates the rate at which an `InputDStream` should ingest + * records, based on updates at every batch completion. + * + * Please see `org.apache.spark.streaming.scheduler.RateController` for more details. */ private[streaming] trait RateEstimator extends Serializable { /** - * Computes the number of elements the stream attached to this `RateEstimator` + * Computes the number of records the stream attached to this `RateEstimator` * should ingest per second, given an update on the size and completion * times of the latest batch. * - * @param time The timetamp of the current batch interval that just finished - * @param elements The number of elements that were processed in this batch + * @param time The timestamp of the current batch interval that just finished + * @param elements The number of records that were processed in this batch * @param processingDelay The time in ms that took for the job to complete * @param schedulingDelay The time in ms that the job spent in the scheduling queue */ @@ -46,13 +48,13 @@ private[streaming] trait RateEstimator extends Serializable { object RateEstimator { /** - * Return a new RateEstimator based on the value of `spark.streaming.RateEstimator`. + * Return a new `RateEstimator` based on the value of + * `spark.streaming.backpressure.rateEstimator`. * - * The only known estimator right now is `pid`. + * The only known and acceptable estimator right now is `pid`. * * @return An instance of RateEstimator - * @throws IllegalArgumentException if there is a configured RateEstimator that doesn't match any - * known estimators. + * @throws IllegalArgumentException if the configured RateEstimator is not `pid`. */ def create(conf: SparkConf, batchInterval: Duration): RateEstimator = conf.get("spark.streaming.backpressure.rateEstimator", "pid") match { @@ -64,6 +66,6 @@ object RateEstimator { new PIDRateEstimator(batchInterval.milliseconds, proportional, integral, derived, minRate) case estimator => - throw new IllegalArgumentException(s"Unkown rate estimator: $estimator") + throw new IllegalArgumentException(s"Unknown rate estimator: $estimator") } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala index 125cafd41b8a..f1070e9029cb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala @@ -25,7 +25,7 @@ private[ui] abstract class BatchTableBase(tableId: String, batchInterval: Long) protected def columns: Seq[Node] = {
    - + @@ -33,10 +33,26 @@ private[ui] abstract class BatchTableBase(tableId: String, batchInterval: Long) {SparkUIUtils.tooltip("Time taken to process all jobs of a batch", "top")} } + /** + * Return the first failure reason if finding in the batches. + */ + protected def getFirstFailureReason(batches: Seq[BatchUIData]): Option[String] = { + batches.flatMap(_.outputOperations.flatMap(_._2.failureReason)).headOption + } + + protected def getFirstFailureTableCell(batch: BatchUIData): Seq[Node] = { + val firstFailureReason = batch.outputOperations.flatMap(_._2.failureReason).headOption + firstFailureReason.map { failureReason => + val failureReasonForUI = UIUtils.createOutputOperationFailureForUI(failureReason) + UIUtils.failureReasonCell( + failureReasonForUI, rowspan = 1, includeFirstLineInExpandDetails = false) + }.getOrElse() + } + protected def baseRow(batch: BatchUIData): Seq[Node] = { val batchTime = batch.batchTime.milliseconds val formattedBatchTime = UIUtils.formatBatchTime(batchTime, batchInterval) - val eventCount = batch.numRecords + val numRecords = batch.numRecords val schedulingDelay = batch.schedulingDelay val formattedSchedulingDelay = schedulingDelay.map(SparkUIUtils.formatDuration).getOrElse("-") val processingTime = batch.processingDelay @@ -49,7 +65,7 @@ private[ui] abstract class BatchTableBase(tableId: String, batchInterval: Long) {formattedBatchTime} - + @@ -81,6 +97,7 @@ private[ui] abstract class BatchTableBase(tableId: String, batchInterval: Long) completed = batch.numCompletedOutputOp, failed = batch.numFailedOutputOp, skipped = 0, + reasonToNumKilled = Map.empty, total = batch.outputOperations.size) } @@ -97,9 +114,17 @@ private[ui] class ActiveBatchTable( waitingBatches: Seq[BatchUIData], batchInterval: Long) extends BatchTableBase("active-batches-table", batchInterval) { + private val firstFailureReason = getFirstFailureReason(runningBatches) + override protected def columns: Seq[Node] = super.columns ++ { - + ++ { + if (firstFailureReason.nonEmpty) { + + } else { + Nil + } + } } override protected def renderRows: Seq[Node] = { @@ -110,20 +135,41 @@ private[ui] class ActiveBatchTable( } private def runningBatchRow(batch: BatchUIData): Seq[Node] = { - baseRow(batch) ++ createOutputOperationProgressBar(batch) ++ + baseRow(batch) ++ createOutputOperationProgressBar(batch) ++ ++ { + if (firstFailureReason.nonEmpty) { + getFirstFailureTableCell(batch) + } else { + Nil + } + } } private def waitingBatchRow(batch: BatchUIData): Seq[Node] = { - baseRow(batch) ++ createOutputOperationProgressBar(batch) ++ + baseRow(batch) ++ createOutputOperationProgressBar(batch) ++ ++ { + if (firstFailureReason.nonEmpty) { + // Waiting batches have not run yet, so must have no failure reasons. + + } else { + Nil + } + } } } private[ui] class CompletedBatchTable(batches: Seq[BatchUIData], batchInterval: Long) extends BatchTableBase("completed-batches-table", batchInterval) { + private val firstFailureReason = getFirstFailureReason(batches) + override protected def columns: Seq[Node] = super.columns ++ { - + ++ { + if (firstFailureReason.nonEmpty) { + + } else { + Nil + } + } } override protected def renderRows: Seq[Node] = { @@ -138,6 +184,12 @@ private[ui] class CompletedBatchTable(batches: Seq[BatchUIData], batchInterval: - } ++ createOutputOperationProgressBar(batch) + } ++ createOutputOperationProgressBar(batch)++ { + if (firstFailureReason.nonEmpty) { + getFirstFailureTableCell(batch) + } else { + Nil + } + } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala index 2ed925572826..69e15655ad79 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala @@ -24,9 +24,9 @@ import scala.xml._ import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.streaming.Time -import org.apache.spark.streaming.ui.StreamingJobProgressListener.{OutputOpId, SparkJobId} -import org.apache.spark.ui.jobs.UIData.JobUIData +import org.apache.spark.streaming.ui.StreamingJobProgressListener.SparkJobId import org.apache.spark.ui.{UIUtils => SparkUIUtils, WebUIPage} +import org.apache.spark.ui.jobs.UIData.JobUIData private[ui] case class SparkJobIdWithUIData(sparkJobId: SparkJobId, jobUIData: Option[JobUIData]) @@ -37,10 +37,10 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { private def columns: Seq[Node] = { - + - + @@ -86,7 +86,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { /** * Generate a row for a Spark Job. Because duplicated output op infos needs to be collapsed into - * one cell, we use "rowspan" for the first row of a output op. + * one cell, we use "rowspan" for the first row of an output op. */ private def generateNormalJobRow( outputOpData: OutputOperationUIData, @@ -146,10 +146,11 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { completed = sparkJob.numCompletedTasks, failed = sparkJob.numFailedTasks, skipped = sparkJob.numSkippedTasks, + reasonToNumKilled = sparkJob.reasonToNumKilled, total = sparkJob.numTasks - sparkJob.numSkippedTasks) } - {failureReasonCell(lastFailureReason, rowspan = 1)} + {UIUtils.failureReasonCell(lastFailureReason)} } @@ -245,48 +246,6 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { } - private def failureReasonCell( - failureReason: String, - rowspan: Int, - includeFirstLineInExpandDetails: Boolean = true): Seq[Node] = { - val isMultiline = failureReason.indexOf('\n') >= 0 - // Display the first line by default - val failureReasonSummary = StringEscapeUtils.escapeHtml4( - if (isMultiline) { - failureReason.substring(0, failureReason.indexOf('\n')) - } else { - failureReason - }) - val failureDetails = - if (isMultiline && !includeFirstLineInExpandDetails) { - // Skip the first line - failureReason.substring(failureReason.indexOf('\n') + 1) - } else { - failureReason - } - val details = if (isMultiline) { - // scalastyle:off - - +details - ++ - - // scalastyle:on - } else { - "" - } - - if (rowspan == 1) { - - } else { - - } - } - private def getJobData(sparkJobId: SparkJobId): Option[JobUIData] = { sparkListener.activeJobs.get(sparkJobId).orElse { sparkListener.completedJobs.find(_.jobId == sparkJobId).orElse { @@ -301,7 +260,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { } else { var nextLineIndex = failure.indexOf("\n") if (nextLineIndex < 0) { - nextLineIndex = failure.size + nextLineIndex = failure.length } val firstLine = failure.substring(0, nextLineIndex) s"Failed due to error: $firstLine\n$failure" @@ -315,7 +274,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { val outputOpIdToSparkJobIds = batchUIData.outputOpIdSparkJobIdPairs.groupBy(_.outputOpId). map { case (outputOpId, outputOpIdAndSparkJobIds) => // sort SparkJobIds for each OutputOpId - (outputOpId, outputOpIdAndSparkJobIds.map(_.sparkJobId).sorted) + (outputOpId, outputOpIdAndSparkJobIds.map(_.sparkJobId).toSeq.sorted) } val outputOps: Seq[(OutputOperationUIData, Seq[SparkJobId])] = @@ -345,7 +304,10 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { } def render(request: HttpServletRequest): Seq[Node] = streamingListener.synchronized { - val batchTime = Option(request.getParameter("id")).map(id => Time(id.toLong)).getOrElse { + // stripXSS is called first to remove suspicious characters used in XSS attacks + val batchTime = + Option(SparkUIUtils.stripXSS(request.getParameter("id"))).map(id => Time(id.toLong)) + .getOrElse { throw new IllegalArgumentException(s"Missing id parameter") } val formattedBatchTime = @@ -434,8 +396,9 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { private def outputOpStatusCell(outputOp: OutputOperationUIData, rowspan: Int): Seq[Node] = { outputOp.failureReason match { case Some(failureReason) => - val failureReasonForUI = generateOutputOperationStatusForUI(failureReason) - failureReasonCell(failureReasonForUI, rowspan, includeFirstLineInExpandDetails = false) + val failureReasonForUI = UIUtils.createOutputOperationFailureForUI(failureReason) + UIUtils.failureReasonCell( + failureReasonForUI, rowspan, includeFirstLineInExpandDetails = false) case None => if (outputOp.endTime.isEmpty) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchUIData.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchUIData.scala index 3ef3689de1c4..1af60857bc77 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchUIData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchUIData.scala @@ -33,7 +33,7 @@ private[ui] case class BatchUIData( val processingStartTime: Option[Long], val processingEndTime: Option[Long], val outputOperations: mutable.HashMap[OutputOpId, OutputOperationUIData] = mutable.HashMap(), - var outputOpIdSparkJobIdPairs: Seq[OutputOpIdAndSparkJobId] = Seq.empty) { + var outputOpIdSparkJobIdPairs: Iterable[OutputOpIdAndSparkJobId] = Seq.empty) { /** * Time taken for the first job of this batch to start processing from the time this batch diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index f6cc6edf2569..ed4c1e484efd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -17,22 +17,18 @@ package org.apache.spark.streaming.ui -import java.util.LinkedHashMap -import java.util.{Map => JMap} -import java.util.Properties +import java.util.{LinkedHashMap, Map => JMap, Properties} +import java.util.concurrent.ConcurrentLinkedQueue -import scala.collection.mutable.{ArrayBuffer, Queue, HashMap, SynchronizedBuffer} +import scala.collection.JavaConverters._ +import scala.collection.mutable.{HashMap, Queue} import org.apache.spark.scheduler._ -import org.apache.spark.streaming.{Time, StreamingContext} +import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.scheduler._ -import org.apache.spark.streaming.scheduler.StreamingListenerReceiverStarted -import org.apache.spark.streaming.scheduler.StreamingListenerBatchStarted -import org.apache.spark.streaming.scheduler.StreamingListenerBatchSubmitted - -private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) - extends StreamingListener with SparkListener { +private[spark] class StreamingJobProgressListener(ssc: StreamingContext) + extends SparkListener with StreamingListener { private val waitingBatchUIData = new HashMap[Time, BatchUIData] private val runningBatchUIData = new HashMap[Time, BatchUIData] @@ -43,13 +39,15 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) private var totalProcessedRecords = 0L private val receiverInfos = new HashMap[Int, ReceiverInfo] + private var _startTime = -1L + // Because onJobStart and onBatchXXX messages are processed in different threads, // we may not be able to get the corresponding BatchUIData when receiving onJobStart. So here we // cannot use a map of (Time, BatchUIData). private[ui] val batchTimeToOutputOpIdSparkJobIdPair = - new LinkedHashMap[Time, SynchronizedBuffer[OutputOpIdAndSparkJobId]] { + new LinkedHashMap[Time, ConcurrentLinkedQueue[OutputOpIdAndSparkJobId]] { override def removeEldestEntry( - p1: JMap.Entry[Time, SynchronizedBuffer[OutputOpIdAndSparkJobId]]): Boolean = { + p1: JMap.Entry[Time, ConcurrentLinkedQueue[OutputOpIdAndSparkJobId]]): Boolean = { // If a lot of "onBatchCompleted"s happen before "onJobStart" (image if // SparkContext.listenerBus is very slow), "batchTimeToOutputOpIdToSparkJobIds" // may add some information for a removed batch when processing "onJobStart". It will be a @@ -70,6 +68,10 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) val batchDuration = ssc.graph.batchDuration.milliseconds + override def onStreamingStarted(streamingStarted: StreamingListenerStreamingStarted) { + _startTime = streamingStarted.time + } + override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { synchronized { receiverInfos(receiverStarted.receiverInfo.streamId) = receiverStarted.receiverInfo @@ -97,7 +99,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = synchronized { val batchUIData = BatchUIData(batchStarted.batchInfo) - runningBatchUIData(batchStarted.batchInfo.batchTime) = BatchUIData(batchStarted.batchInfo) + runningBatchUIData(batchStarted.batchInfo.batchTime) = batchUIData waitingBatchUIData.remove(batchStarted.batchInfo.batchTime) totalReceivedRecords += batchUIData.numRecords @@ -137,12 +139,10 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) getBatchTimeAndOutputOpId(jobStart.properties).foreach { case (batchTime, outputOpId) => var outputOpIdToSparkJobIds = batchTimeToOutputOpIdSparkJobIdPair.get(batchTime) if (outputOpIdToSparkJobIds == null) { - outputOpIdToSparkJobIds = - new ArrayBuffer[OutputOpIdAndSparkJobId]() - with SynchronizedBuffer[OutputOpIdAndSparkJobId] + outputOpIdToSparkJobIds = new ConcurrentLinkedQueue[OutputOpIdAndSparkJobId]() batchTimeToOutputOpIdSparkJobIdPair.put(batchTime, outputOpIdToSparkJobIds) } - outputOpIdToSparkJobIds += OutputOpIdAndSparkJobId(outputOpId, jobStart.jobId) + outputOpIdToSparkJobIds.add(OutputOpIdAndSparkJobId(outputOpId, jobStart.jobId)) } } @@ -158,6 +158,8 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } } + def startTime: Long = _startTime + def numReceivers: Int = synchronized { receiverInfos.size } @@ -167,7 +169,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } def numInactiveReceivers: Int = { - ssc.graph.getReceiverInputStreams().size - numActiveReceivers + ssc.graph.getNumReceivers - numActiveReceivers } def numTotalCompletedBatches: Long = synchronized { @@ -195,34 +197,34 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } def retainedCompletedBatches: Seq[BatchUIData] = synchronized { - completedBatchUIData.toSeq + completedBatchUIData.toIndexedSeq } def streamName(streamId: Int): Option[String] = { - ssc.graph.getInputStreamName(streamId) + ssc.graph.getInputStreamNameAndID.find(_._2 == streamId).map(_._1) } /** * Return all InputDStream Ids */ - def streamIds: Seq[Int] = ssc.graph.getInputStreams().map(_.id) + def streamIds: Seq[Int] = ssc.graph.getInputStreamNameAndID.map(_._2) /** - * Return all of the event rates for each InputDStream in each batch. The key of the return value - * is the stream id, and the value is a sequence of batch time with its event rate. + * Return all of the record rates for each InputDStream in each batch. The key of the return value + * is the stream id, and the value is a sequence of batch time with its record rate. */ - def receivedEventRateWithBatchTime: Map[Int, Seq[(Long, Double)]] = synchronized { + def receivedRecordRateWithBatchTime: Map[Int, Seq[(Long, Double)]] = synchronized { val _retainedBatches = retainedBatches val latestBatches = _retainedBatches.map { batchUIData => (batchUIData.batchTime.milliseconds, batchUIData.streamIdToInputInfo.mapValues(_.numRecords)) } streamIds.map { streamId => - val eventRates = latestBatches.map { + val recordRates = latestBatches.map { case (batchTime, streamIdToNumRecords) => val numRecords = streamIdToNumRecords.getOrElse(streamId, 0L) (batchTime, numRecords * 1000.0 / batchDuration) } - (streamId, eventRates) + (streamId, recordRates) }.toMap } @@ -262,15 +264,18 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } } batchUIData.foreach { _batchUIData => - val outputOpIdToSparkJobIds = - Option(batchTimeToOutputOpIdSparkJobIdPair.get(batchTime)).getOrElse(Seq.empty) + // We use an Iterable rather than explicitly converting to a seq so that updates + // will propagate + val outputOpIdToSparkJobIds: Iterable[OutputOpIdAndSparkJobId] = + Option(batchTimeToOutputOpIdSparkJobIdPair.get(batchTime)).map(_.asScala) + .getOrElse(Seq.empty) _batchUIData.outputOpIdSparkJobIdPairs = outputOpIdToSparkJobIds } batchUIData } } -private[streaming] object StreamingJobProgressListener { +private[spark] object StreamingJobProgressListener { type SparkJobId = Int type OutputOpId = Int } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 96d943e75d27..7abafd6ba790 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -17,15 +17,13 @@ package org.apache.spark.streaming.ui -import java.text.SimpleDateFormat -import java.util.Date import java.util.concurrent.TimeUnit import javax.servlet.http.HttpServletRequest import scala.collection.mutable.ArrayBuffer import scala.xml.{Node, Unparsed} -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.ui._ import org.apache.spark.ui.{UIUtils => SparkUIUtils} @@ -127,9 +125,9 @@ private[ui] class MillisecondsStatUIData(data: Seq[(Long, Long)]) { * A helper class for "input rate" to generate data that will be used in the timeline and histogram * graphs. * - * @param data (batchTime, event-rate). + * @param data (batch time, record rate). */ -private[ui] class EventRateUIData(val data: Seq[(Long, Double)]) { +private[ui] class RecordRateUIData(val data: Seq[(Long, Double)]) { val avg: Option[Double] = if (data.isEmpty) None else Some(data.map(_._2).sum / data.size) @@ -145,7 +143,8 @@ private[ui] class StreamingPage(parent: StreamingTab) import StreamingPage._ private val listener = parent.listener - private val startTime = System.currentTimeMillis() + + private def startTime: Long = listener.startTime /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { @@ -217,7 +216,7 @@ private[ui] class StreamingPage(parent: StreamingTab) val minBatchTime = if (batchTimes.isEmpty) startTime else batchTimes.min val maxBatchTime = if (batchTimes.isEmpty) startTime else batchTimes.max - val eventRateForAllStreams = new EventRateUIData(batches.map { batchInfo => + val recordRateForAllStreams = new RecordRateUIData(batches.map { batchInfo => (batchInfo.batchTime.milliseconds, batchInfo.numRecords * 1000.0 / listener.batchDuration) }) @@ -243,24 +242,24 @@ private[ui] class StreamingPage(parent: StreamingTab) // Use the max input rate for all InputDStreams' graphs to make the Y axis ranges same. // If it's not an integral number, just use its ceil integral number. - val maxEventRate = eventRateForAllStreams.max.map(_.ceil.toLong).getOrElse(0L) - val minEventRate = 0L + val maxRecordRate = recordRateForAllStreams.max.map(_.ceil.toLong).getOrElse(0L) + val minRecordRate = 0L val batchInterval = UIUtils.convertToTimeUnit(listener.batchDuration, normalizedUnit) val jsCollector = new JsCollector - val graphUIDataForEventRateOfAllStreams = + val graphUIDataForRecordRateOfAllStreams = new GraphUIData( - "all-stream-events-timeline", - "all-stream-events-histogram", - eventRateForAllStreams.data, + "all-stream-records-timeline", + "all-stream-records-histogram", + recordRateForAllStreams.data, minBatchTime, maxBatchTime, - minEventRate, - maxEventRate, - "events/sec") - graphUIDataForEventRateOfAllStreams.generateDataJs(jsCollector) + minRecordRate, + maxRecordRate, + "records/sec") + graphUIDataForRecordRateOfAllStreams.generateDataJs(jsCollector) val graphUIDataForSchedulingDelay = new GraphUIData( @@ -336,16 +335,16 @@ private[ui] class StreamingPage(parent: StreamingTab)
    Receivers: {listener.numActiveReceivers} / {numReceivers} active
    } } -
    Avg: {eventRateForAllStreams.formattedAvg} events/sec
    +
    Avg: {recordRateForAllStreams.formattedAvg} records/sec
    - - + + {if (hasStream) { }} @@ -392,9 +391,17 @@ private[ui] class StreamingPage(parent: StreamingTab) maxX: Long, minY: Double, maxY: Double): Seq[Node] = { - val content = listener.receivedEventRateWithBatchTime.map { case (streamId, eventRates) => - generateInputDStreamRow(jsCollector, streamId, eventRates, minX, maxX, minY, maxY) - }.foldLeft[Seq[Node]](Nil)(_ ++ _) + val maxYCalculated = listener.receivedRecordRateWithBatchTime.values + .flatMap { case streamAndRates => streamAndRates.map { case (_, recordRate) => recordRate } } + .reduceOption[Double](math.max) + .map(_.ceil.toLong) + .getOrElse(0L) + + val content: Seq[Node] = listener.receivedRecordRateWithBatchTime.toList.sortBy(_._1).flatMap { + case (streamId, recordRates) => + generateInputDStreamRow( + jsCollector, streamId, recordRates, minX, maxX, minY, maxYCalculated) + } // scalastyle:off
    @@ -118,14 +130,12 @@ private[ui] abstract class ExecutionTable( {failedJobs}
    {summary}{details} {errorSummary}{details}Batch TimeInput SizeRecords Scheduling Delay {SparkUIUtils.tooltip("Time taken by Streaming scheduler to submit jobs of a batch", "top")} - {eventCount.toString} events{numRecords.toString} records {formattedSchedulingDelay} Output Ops: Succeeded/TotalStatusStatusErrorprocessingprocessingqueuedqueued-Total Delay {SparkUIUtils.tooltip("Total time taken to handle a batch", "top")}Output Ops: Succeeded/TotalOutput Ops: Succeeded/TotalError {formattedTotalDelay} Output Op Id DescriptionDurationOutput Op Duration Status Job IdDurationJob Duration Stages: Succeeded/Total Tasks (for all stages): Succeeded/Total Error
    {failureReasonSummary}{details} - {failureReasonSummary}{details} - - {graphUIDataForEventRateOfAllStreams.generateTimelineHtml(jsCollector)}{graphUIDataForEventRateOfAllStreams.generateHistogramHtml(jsCollector)}{graphUIDataForRecordRateOfAllStreams.generateTimelineHtml(jsCollector)}{graphUIDataForRecordRateOfAllStreams.generateHistogramHtml(jsCollector)}
    @@ -402,7 +409,7 @@ private[ui] class StreamingPage(parent: StreamingTab) - + @@ -417,7 +424,7 @@ private[ui] class StreamingPage(parent: StreamingTab) private def generateInputDStreamRow( jsCollector: JsCollector, streamId: Int, - eventRates: Seq[(Long, Double)], + recordRates: Seq[(Long, Double)], minX: Long, maxX: Long, minY: Double, @@ -430,33 +437,37 @@ private[ui] class StreamingPage(parent: StreamingTab) val receiverActive = receiverInfo.map { info => if (info.active) "ACTIVE" else "INACTIVE" }.getOrElse(emptyCell) - val receiverLocation = receiverInfo.map(_.location).getOrElse(emptyCell) + val receiverLocation = receiverInfo.map { info => + val executorId = if (info.executorId.isEmpty) emptyCell else info.executorId + val location = if (info.location.isEmpty) emptyCell else info.location + s"$executorId / $location" + }.getOrElse(emptyCell) val receiverLastError = receiverInfo.map { info => val msg = s"${info.lastErrorMessage} - ${info.lastError}" - if (msg.size > 100) msg.take(97) + "..." else msg + if (msg.length > 100) msg.take(97) + "..." else msg }.getOrElse(emptyCell) val receiverLastErrorTime = receiverInfo.map { r => if (r.lastErrorTime < 0) "-" else SparkUIUtils.formatDate(r.lastErrorTime) }.getOrElse(emptyCell) - val receivedRecords = new EventRateUIData(eventRates) + val receivedRecords = new RecordRateUIData(recordRates) - val graphUIDataForEventRate = + val graphUIDataForRecordRate = new GraphUIData( - s"stream-$streamId-events-timeline", - s"stream-$streamId-events-histogram", + s"stream-$streamId-records-timeline", + s"stream-$streamId-records-histogram", receivedRecords.data, minX, maxX, minY, maxY, - "events/sec") - graphUIDataForEventRate.generateDataJs(jsCollector) + "records/sec") + graphUIDataForRecordRate.generateDataJs(jsCollector) @@ -466,9 +477,9 @@ private[ui] class StreamingPage(parent: StreamingTab) - + } @@ -549,7 +560,7 @@ private[ui] class JsCollector { def toHtml: Seq[Node] = { val js = s""" - |$$(document).ready(function(){ + |$$(document).ready(function() { | ${preparedStatements.mkString("\n")} | ${statements.mkString("\n")} |});""".stripMargin diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index bc53f2a31f6d..9d1b82a6341b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -17,18 +17,19 @@ package org.apache.spark.streaming.ui -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging import org.apache.spark.streaming.StreamingContext import org.apache.spark.ui.{SparkUI, SparkUITab} -import StreamingTab._ - /** * Spark Web UI tab that shows statistics of a streaming job. * This assumes the given SparkContext has enabled its SparkUI. */ private[spark] class StreamingTab(val ssc: StreamingContext) - extends SparkUITab(getSparkUI(ssc), "streaming") with Logging { + extends SparkUITab(StreamingTab.getSparkUI(ssc), "streaming") with Logging { + + import StreamingTab._ private val STATIC_RESOURCE_DIR = "org/apache/spark/streaming/ui/static" @@ -37,6 +38,7 @@ private[spark] class StreamingTab(val ssc: StreamingContext) ssc.addStreamingListener(listener) ssc.sc.addSparkListener(listener) + parent.setStreamingJobProgressListener(listener) attachPage(new StreamingPage(this)) attachPage(new BatchPage(this)) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala index 86cfb1fa4737..84ecf81abfbf 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala @@ -18,9 +18,13 @@ package org.apache.spark.streaming.ui import java.text.SimpleDateFormat -import java.util.TimeZone +import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit +import scala.xml.Node + +import org.apache.commons.lang3.StringEscapeUtils + private[streaming] object UIUtils { /** @@ -76,11 +80,13 @@ private[streaming] object UIUtils { // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. private val batchTimeFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + override def initialValue(): SimpleDateFormat = + new SimpleDateFormat("yyyy/MM/dd HH:mm:ss", Locale.US) } private val batchTimeFormatWithMilliseconds = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS") + override def initialValue(): SimpleDateFormat = + new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS", Locale.US) } /** @@ -124,4 +130,60 @@ private[streaming] object UIUtils { } } } + + def createOutputOperationFailureForUI(failure: String): String = { + if (failure.startsWith("org.apache.spark.Spark")) { + // SparkException or SparkDriverExecutionException + "Failed due to Spark job error\n" + failure + } else { + var nextLineIndex = failure.indexOf("\n") + if (nextLineIndex < 0) { + nextLineIndex = failure.length + } + val firstLine = failure.substring(0, nextLineIndex) + s"Failed due to error: $firstLine\n$failure" + } + } + + def failureReasonCell( + failureReason: String, + rowspan: Int = 1, + includeFirstLineInExpandDetails: Boolean = true): Seq[Node] = { + val isMultiline = failureReason.indexOf('\n') >= 0 + // Display the first line by default + val failureReasonSummary = StringEscapeUtils.escapeHtml4( + if (isMultiline) { + failureReason.substring(0, failureReason.indexOf('\n')) + } else { + failureReason + }) + val failureDetails = + if (isMultiline && !includeFirstLineInExpandDetails) { + // Skip the first line + failureReason.substring(failureReason.indexOf('\n') + 1) + } else { + failureReason + } + val details = if (isMultiline) { + // scalastyle:off + + +details + ++ + + // scalastyle:on + } else { + "" + } + + if (rowspan == 1) { + + } else { + + } + } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala new file mode 100644 index 000000000000..71b86d16866e --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.util + +import java.nio.ByteBuffer +import java.util.{Iterator => JIterator} +import java.util.concurrent.LinkedBlockingQueue +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.Promise +import scala.concurrent.duration._ +import scala.util.control.NonFatal + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.util.{ThreadUtils, Utils} + +/** + * A wrapper for a WriteAheadLog that batches records before writing data. Handles aggregation + * during writes, and de-aggregation in the `readAll` method. The end consumer has to handle + * de-aggregation after the `read` method. In addition, the `WriteAheadLogRecordHandle` returned + * after the write will contain the batch of records rather than individual records. + * + * When writing a batch of records, the `time` passed to the `wrappedLog` will be the timestamp + * of the latest record in the batch. This is very important in achieving correctness. Consider the + * following example: + * We receive records with timestamps 1, 3, 5, 7. We use "log-1" as the filename. Once we receive + * a clean up request for timestamp 3, we would clean up the file "log-1", and lose data regarding + * 5 and 7. + * + * This means the caller can assume the same write semantics as any other WriteAheadLog + * implementation despite the batching in the background - when the write() returns, the data is + * written to the WAL and is durable. To take advantage of the batching, the caller can write from + * multiple threads, each of which will stay blocked until the corresponding data has been written. + * + * All other methods of the WriteAheadLog interface will be passed on to the wrapped WriteAheadLog. + */ +private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: SparkConf) + extends WriteAheadLog with Logging { + + import BatchedWriteAheadLog._ + + private val walWriteQueue = new LinkedBlockingQueue[Record]() + + // Whether the writer thread is active + private val active: AtomicBoolean = new AtomicBoolean(true) + private val buffer = new ArrayBuffer[Record]() + + private val batchedWriterThread = startBatchedWriterThread() + + /** + * Write a byte buffer to the log file. This method adds the byteBuffer to a queue and blocks + * until the record is properly written by the parent. + */ + override def write(byteBuffer: ByteBuffer, time: Long): WriteAheadLogRecordHandle = { + val promise = Promise[WriteAheadLogRecordHandle]() + val putSuccessfully = synchronized { + if (active.get()) { + walWriteQueue.offer(Record(byteBuffer, time, promise)) + true + } else { + false + } + } + if (putSuccessfully) { + ThreadUtils.awaitResult( + promise.future, WriteAheadLogUtils.getBatchingTimeout(conf).milliseconds) + } else { + throw new IllegalStateException("close() was called on BatchedWriteAheadLog before " + + s"write request with time $time could be fulfilled.") + } + } + + /** + * This method is not supported as the resulting ByteBuffer would actually require de-aggregation. + * This method is primarily used in testing, and to ensure that it is not used in production, + * we throw an UnsupportedOperationException. + */ + override def read(segment: WriteAheadLogRecordHandle): ByteBuffer = { + throw new UnsupportedOperationException("read() is not supported for BatchedWriteAheadLog " + + "as the data may require de-aggregation.") + } + + /** + * Read all the existing logs from the log directory. The output of the wrapped WriteAheadLog + * will be de-aggregated. + */ + override def readAll(): JIterator[ByteBuffer] = { + wrappedLog.readAll().asScala.flatMap(deaggregate).asJava + } + + /** + * Delete the log files that are older than the threshold time. + * + * This method is handled by the parent WriteAheadLog. + */ + override def clean(threshTime: Long, waitForCompletion: Boolean): Unit = { + wrappedLog.clean(threshTime, waitForCompletion) + } + + + /** + * Stop the batched writer thread, fulfill promises with failures and close the wrapped WAL. + */ + override def close(): Unit = { + logInfo(s"BatchedWriteAheadLog shutting down at time: ${System.currentTimeMillis()}.") + if (!active.getAndSet(false)) return + batchedWriterThread.interrupt() + batchedWriterThread.join() + while (!walWriteQueue.isEmpty) { + val Record(_, time, promise) = walWriteQueue.poll() + promise.failure(new IllegalStateException("close() was called on BatchedWriteAheadLog " + + s"before write request with time $time could be fulfilled.")) + } + wrappedLog.close() + } + + /** Start the actual log writer on a separate thread. */ + private def startBatchedWriterThread(): Thread = { + val thread = new Thread(new Runnable { + override def run(): Unit = { + while (active.get()) { + try { + flushRecords() + } catch { + case NonFatal(e) => + logWarning("Encountered exception in Batched Writer Thread.", e) + } + } + logInfo("BatchedWriteAheadLog Writer thread exiting.") + } + }, "BatchedWriteAheadLog Writer") + thread.setDaemon(true) + thread.start() + thread + } + + /** Write all the records in the buffer to the write ahead log. */ + private def flushRecords(): Unit = { + try { + buffer += walWriteQueue.take() + val numBatched = walWriteQueue.drainTo(buffer.asJava) + 1 + logDebug(s"Received $numBatched records from queue") + } catch { + case _: InterruptedException => + logWarning("BatchedWriteAheadLog Writer queue interrupted.") + } + try { + var segment: WriteAheadLogRecordHandle = null + if (buffer.nonEmpty) { + logDebug(s"Batched ${buffer.length} records for Write Ahead Log write") + // threads may not be able to add items in order by time + val sortedByTime = buffer.sortBy(_.time) + // We take the latest record for the timestamp. Please refer to the class Javadoc for + // detailed explanation + val time = sortedByTime.last.time + segment = wrappedLog.write(aggregate(sortedByTime), time) + } + buffer.foreach(_.promise.success(segment)) + } catch { + case e: InterruptedException => + logWarning("BatchedWriteAheadLog Writer queue interrupted.", e) + buffer.foreach(_.promise.failure(e)) + case NonFatal(e) => + logWarning(s"BatchedWriteAheadLog Writer failed to write $buffer", e) + buffer.foreach(_.promise.failure(e)) + } finally { + buffer.clear() + } + } + + /** Method for querying the queue length. Should only be used in tests. */ + private def getQueueLength(): Int = walWriteQueue.size() +} + +/** Static methods for aggregating and de-aggregating records. */ +private[util] object BatchedWriteAheadLog { + + /** + * Wrapper class for representing the records that we will write to the WriteAheadLog. Coupled + * with the timestamp for the write request of the record, and the promise that will block the + * write request, while a separate thread is actually performing the write. + */ + case class Record(data: ByteBuffer, time: Long, promise: Promise[WriteAheadLogRecordHandle]) + + /** Aggregate multiple serialized ReceivedBlockTrackerLogEvents in a single ByteBuffer. */ + def aggregate(records: Seq[Record]): ByteBuffer = { + ByteBuffer.wrap(Utils.serialize[Array[Array[Byte]]]( + records.map(record => JavaUtils.bufferToArray(record.data)).toArray)) + } + + /** + * De-aggregate serialized ReceivedBlockTrackerLogEvents in a single ByteBuffer. + * A stream may not have used batching initially, but started using it after a restart. This + * method therefore needs to be backwards compatible. + */ + def deaggregate(buffer: ByteBuffer): Array[ByteBuffer] = { + val prevPosition = buffer.position() + try { + Utils.deserialize[Array[Array[Byte]]](JavaUtils.bufferToArray(buffer)).map(ByteBuffer.wrap) + } catch { + case _: ClassCastException => // users may restart a stream with batching enabled + // Restore `position` so that the user can read `buffer` later + buffer.position(prevPosition) + Array(buffer) + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala index bc3f2486c21f..d6e15cfdd272 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala @@ -16,25 +16,30 @@ */ package org.apache.spark.streaming.util +import java.io.FileNotFoundException import java.nio.ByteBuffer import java.util.{Iterator => JIterator} +import java.util.concurrent.RejectedExecutionException import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer +import scala.collection.parallel.ExecutionContextTaskSupport import scala.concurrent.{Await, ExecutionContext, Future} import scala.language.postfixOps import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.util.{CompletionIterator, ThreadUtils} -import org.apache.spark.{Logging, SparkConf} /** * This class manages write ahead log files. - * - Writes records (bytebuffers) to periodically rotating log files. - * - Recovers the log files and the reads the recovered records upon failures. - * - Cleans up old log files. + * + * - Writes records (bytebuffers) to periodically rotating log files. + * - Recovers the log files and the reads the recovered records upon failures. + * - Cleans up old log files. * * Uses [[org.apache.spark.streaming.util.FileBasedWriteAheadLogWriter]] to write * and [[org.apache.spark.streaming.util.FileBasedWriteAheadLogReader]] to read. @@ -54,12 +59,18 @@ private[streaming] class FileBasedWriteAheadLog( import FileBasedWriteAheadLog._ private val pastLogs = new ArrayBuffer[LogInfo] - private val callerNameTag = getCallerName.map(c => s" for $c").getOrElse("") + private val callerName = getCallerName + + private val threadpoolName = { + "WriteAheadLogManager" + callerName.map(c => s" for $c").getOrElse("") + } + private val forkJoinPool = ThreadUtils.newForkJoinPool(threadpoolName, 20) + private val executionContext = ExecutionContext.fromExecutorService(forkJoinPool) - private val threadpoolName = s"WriteAheadLogManager $callerNameTag" - implicit private val executionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonSingleThreadExecutor(threadpoolName)) - override protected val logName = s"WriteAheadLogManager $callerNameTag" + override protected def logName = { + getClass.getName.stripSuffix("$") + + callerName.map("_" + _).getOrElse("").replaceAll("[ ]", "_") + } private var currentLogPath: Option[String] = None private var currentLogWriter: FileBasedWriteAheadLogWriter = null @@ -124,13 +135,19 @@ private[streaming] class FileBasedWriteAheadLog( */ def readAll(): JIterator[ByteBuffer] = synchronized { val logFilesToRead = pastLogs.map{ _.path} ++ currentLogPath - logInfo("Reading from the logs: " + logFilesToRead.mkString("\n")) - - logFilesToRead.iterator.map { file => + logInfo("Reading from the logs:\n" + logFilesToRead.mkString("\n")) + def readFile(file: String): Iterator[ByteBuffer] = { logDebug(s"Creating log reader with $file") val reader = new FileBasedWriteAheadLogReader(file, hadoopConf) CompletionIterator[ByteBuffer, Iterator[ByteBuffer]](reader, reader.close _) - }.flatten.asJava + } + if (!closeFileAfterWrite) { + logFilesToRead.iterator.map(readFile).flatten.asJava + } else { + // For performance gains, it makes sense to parallelize the recovery if + // closeFileAfterWrite = true + seqToParIterator(executionContext, logFilesToRead, readFile).asJava + } } /** @@ -146,30 +163,41 @@ private[streaming] class FileBasedWriteAheadLog( * asynchronously. */ def clean(threshTime: Long, waitForCompletion: Boolean): Unit = { - val oldLogFiles = synchronized { pastLogs.filter { _.endTime < threshTime } } + val oldLogFiles = synchronized { + val expiredLogs = pastLogs.filter { _.endTime < threshTime } + pastLogs --= expiredLogs + expiredLogs + } logInfo(s"Attempting to clear ${oldLogFiles.size} old log files in $logDirectory " + s"older than $threshTime: ${oldLogFiles.map { _.path }.mkString("\n")}") - def deleteFiles() { - oldLogFiles.foreach { logInfo => - try { - val path = new Path(logInfo.path) - val fs = HdfsUtils.getFileSystemForPath(path, hadoopConf) - fs.delete(path, true) - synchronized { pastLogs -= logInfo } - logDebug(s"Cleared log file $logInfo") - } catch { - case ex: Exception => - logWarning(s"Error clearing write ahead log file $logInfo", ex) - } + def deleteFile(walInfo: LogInfo): Unit = { + try { + val path = new Path(walInfo.path) + val fs = HdfsUtils.getFileSystemForPath(path, hadoopConf) + fs.delete(path, true) + logDebug(s"Cleared log file $walInfo") + } catch { + case ex: Exception => + logWarning(s"Error clearing write ahead log file $walInfo", ex) } logInfo(s"Cleared log files in $logDirectory older than $threshTime") } - if (!executionContext.isShutdown) { - val f = Future { deleteFiles() } - if (waitForCompletion) { - import scala.concurrent.duration._ - Await.ready(f, 1 second) + oldLogFiles.foreach { logInfo => + if (!executionContext.isShutdown) { + try { + val f = Future { deleteFile(logInfo) }(executionContext) + if (waitForCompletion) { + import scala.concurrent.duration._ + // scalastyle:off awaitready + Await.ready(f, 1 second) + // scalastyle:on awaitready + } + } catch { + case e: RejectedExecutionException => + logWarning("Execution context shutdown before deleting old WriteAheadLogs. " + + "This would not affect recovery correctness.", e) + } } } } @@ -177,10 +205,12 @@ private[streaming] class FileBasedWriteAheadLog( /** Stop the manager, close any open log writer */ def close(): Unit = synchronized { - if (currentLogWriter != null) { - currentLogWriter.close() + if (!executionContext.isShutdown) { + if (currentLogWriter != null) { + currentLogWriter.close() + } + executionContext.shutdown() } - executionContext.shutdown() logInfo("Stopped write ahead log manager") } @@ -206,12 +236,25 @@ private[streaming] class FileBasedWriteAheadLog( val logDirectoryPath = new Path(logDirectory) val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) - if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) { - val logFileInfo = logFilesTologInfo(fileSystem.listStatus(logDirectoryPath).map { _.getPath }) - pastLogs.clear() - pastLogs ++= logFileInfo - logInfo(s"Recovered ${logFileInfo.size} write ahead log files from $logDirectory") - logDebug(s"Recovered files are:\n${logFileInfo.map(_.path).mkString("\n")}") + try { + // If you call listStatus(file) it returns a stat of the file in the array, + // rather than an array listing all the children. + // This makes it hard to differentiate listStatus(file) and + // listStatus(dir-with-one-child) except by examining the name of the returned status, + // and once you've got symlinks in the mix that differentiation isn't easy. + // Checking for the path being a directory is one more call to the filesystem, but + // leads to much clearer code. + if (fileSystem.getFileStatus(logDirectoryPath).isDirectory) { + val logFileInfo = logFilesTologInfo( + fileSystem.listStatus(logDirectoryPath).map { _.getPath }) + pastLogs.clear() + pastLogs ++= logFileInfo + logInfo(s"Recovered ${logFileInfo.size} write ahead log files from $logDirectory") + logDebug(s"Recovered files are:\n${logFileInfo.map(_.path).mkString("\n")}") + } + } catch { + case _: FileNotFoundException => + // there is no log directory, hence nothing to recover } } @@ -234,8 +277,12 @@ private[streaming] object FileBasedWriteAheadLog { } def getCallerName(): Option[String] = { - val stackTraceClasses = Thread.currentThread.getStackTrace().map(_.getClassName) - stackTraceClasses.find(!_.contains("WriteAheadLog")).flatMap(_.split(".").lastOption) + val blacklist = Seq("WriteAheadLog", "Logging", "java.lang", "scala.") + Thread.currentThread.getStackTrace() + .map(_.getClassName) + .find { c => !blacklist.exists(c.contains) } + .flatMap(_.split("\\.").lastOption) + .flatMap(_.split("\\$\\$").headOption) } /** Convert a sequence of files to a sequence of sorted LogInfo objects */ @@ -251,4 +298,24 @@ private[streaming] object FileBasedWriteAheadLog { } }.sortBy { _.startTime } } + + /** + * This creates an iterator from a parallel collection, by keeping at most `n` objects in memory + * at any given time, where `n` is at most the max of the size of the thread pool or 8. This is + * crucial for use cases where we create `FileBasedWriteAheadLogReader`s during parallel recovery. + * We don't want to open up `k` streams altogether where `k` is the size of the Seq that we want + * to parallelize. + */ + def seqToParIterator[I, O]( + executionContext: ExecutionContext, + source: Seq[I], + handler: I => Iterator[O]): Iterator[O] = { + val taskSupport = new ExecutionContextTaskSupport(executionContext) + val groupSize = taskSupport.parallelismLevel.max(8) + source.grouped(groupSize).flatMap { group => + val parallelCollection = group.par + parallelCollection.tasksupport = taskSupport + parallelCollection.map(handler) + }.flatten + } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogRandomReader.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogRandomReader.scala index f7168229ec15..56d4977da0b5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogRandomReader.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogRandomReader.scala @@ -30,7 +30,7 @@ private[streaming] class FileBasedWriteAheadLogRandomReader(path: String, conf: extends Closeable { private val instream = HdfsUtils.getInputStream(path, conf) - private var closed = false + private var closed = (instream == null) // the file may be deleted as we're opening the stream def read(segment: FileBasedWriteAheadLogSegment): ByteBuffer = synchronized { assertOpen() diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogReader.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogReader.scala index c3bb59f3fef9..14d9bc94a123 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogReader.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogReader.scala @@ -16,11 +16,12 @@ */ package org.apache.spark.streaming.util -import java.io.{Closeable, EOFException} +import java.io.{Closeable, EOFException, IOException} import java.nio.ByteBuffer import org.apache.hadoop.conf.Configuration -import org.apache.spark.Logging + +import org.apache.spark.internal.Logging /** * A reader for reading write ahead log files written using @@ -32,7 +33,7 @@ private[streaming] class FileBasedWriteAheadLogReader(path: String, conf: Config extends Iterator[ByteBuffer] with Closeable with Logging { private val instream = HdfsUtils.getInputStream(path, conf) - private var closed = false + private var closed = (instream == null) // the file may be deleted as we're opening the stream private var nextItem: Option[ByteBuffer] = None override def hasNext: Boolean = synchronized { @@ -55,6 +56,19 @@ private[streaming] class FileBasedWriteAheadLogReader(path: String, conf: Config logDebug("Error reading next item, EOF reached", e) close() false + case e: IOException => + logWarning("Error while trying to read data. If the file was deleted, " + + "this should be okay.", e) + close() + if (HdfsUtils.checkFileExists(path, conf)) { + // If file exists, this could be a legitimate error + throw e + } else { + // File was deleted. This can occur when the daemon cleanup thread takes time to + // delete the file during recovery. + false + } + case e: Exception => logWarning("Error while trying to read data from HDFS.", e) close() diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogWriter.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogWriter.scala index e146bec32a45..1f5c1d4369b5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogWriter.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLogWriter.scala @@ -19,10 +19,9 @@ package org.apache.spark.streaming.util import java.io._ import java.nio.ByteBuffer -import scala.util.Try - import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FSDataOutputStream + +import org.apache.spark.util.Utils /** * A writer for writing byte-buffers to a write ahead log file. @@ -32,11 +31,6 @@ private[streaming] class FileBasedWriteAheadLogWriter(path: String, hadoopConf: private lazy val stream = HdfsUtils.getOutputStream(path, hadoopConf) - private lazy val hadoopFlushMethod = { - // Use reflection to get the right flush operation - val cls = classOf[FSDataOutputStream] - Try(cls.getMethod("hflush")).orElse(Try(cls.getMethod("sync"))).toOption - } private var nextOffset = stream.getPos() private var closed = false @@ -48,17 +42,7 @@ private[streaming] class FileBasedWriteAheadLogWriter(path: String, hadoopConf: val lengthToWrite = data.remaining() val segment = new FileBasedWriteAheadLogSegment(path, nextOffset, lengthToWrite) stream.writeInt(lengthToWrite) - if (data.hasArray) { - stream.write(data.array()) - } else { - // If the buffer is not backed by an array, we transfer using temp array - // Note that despite the extra array copy, this should be faster than byte-by-byte copy - while (data.hasRemaining) { - val array = new Array[Byte](data.remaining) - data.get(array) - stream.write(array) - } - } + Utils.writeByteBuffer(data, stream: OutputStream) flush() nextOffset = stream.getPos() segment @@ -70,7 +54,7 @@ private[streaming] class FileBasedWriteAheadLogWriter(path: String, hadoopConf: } private def flush() { - hadoopFlushMethod.foreach { _.invoke(stream) } + stream.hflush() // Useful for local file system where hflush/sync does not work (HADOOP-7844) stream.getWrappedStream.flush() } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala index f60688f173c4..6a3b3200dccd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.streaming.util +import java.io.{FileNotFoundException, IOException} + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ @@ -42,8 +44,17 @@ private[streaming] object HdfsUtils { def getInputStream(path: String, conf: Configuration): FSDataInputStream = { val dfsPath = new Path(path) val dfs = getFileSystemForPath(dfsPath, conf) - val instream = dfs.open(dfsPath) - instream + try { + dfs.open(dfsPath) + } catch { + case _: FileNotFoundException => + null + case e: IOException => + // If we are really unlucky, the file may be deleted as we're opening the stream. + // This can happen as clean up is performed by daemon threads that may be left over from + // previous runs. + if (!dfs.isFile(dfsPath)) null else throw e + } } def checkState(state: Boolean, errorMsg: => String) { @@ -71,4 +82,11 @@ private[streaming] object HdfsUtils { case _ => fs } } + + /** Check if the file exists at the given path. */ + def checkFileExists(path: String, conf: Configuration): Boolean = { + val hdpPath = new Path(path) + val fs = getFileSystemForPath(hdpPath, conf) + fs.isFile(hdpPath) + } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala index a96e2924a0b4..29cc1fa00ac0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala @@ -17,13 +17,12 @@ package org.apache.spark.streaming.util -import scala.annotation.tailrec - import java.io.OutputStream import java.util.concurrent.TimeUnit._ -import org.apache.spark.Logging +import scala.annotation.tailrec +import org.apache.spark.internal.Logging private[streaming] class RateLimitedOutputStream(out: OutputStream, desiredBytesPerSec: Int) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala index 408936653c79..eb9996ece377 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala @@ -63,7 +63,6 @@ object RawTextHelper { var i = 0 var len = 0 - var done = false var value: (String, Long) = null var swap: (String, Long) = null var count = 0 diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala index 6addb9675203..9667af97f03b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala @@ -23,7 +23,8 @@ import java.nio.ByteBuffer import scala.io.Source -import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.IntParam diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala index 0148cb51c6f0..62e681e3e964 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.util -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.util.{Clock, SystemClock} private[streaming] @@ -72,10 +72,10 @@ class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: /** * Stop the timer, and return the last time the callback was made. - * - interruptTimer = true will interrupt the callback - * if it is in progress (not guaranteed to give correct time in this case). - * - interruptTimer = false guarantees that there will be at least one callback after `stop` has - * been called. + * + * @param interruptTimer True will interrupt the callback if it is in progress (not guaranteed to + * give correct time in this case). False guarantees that there will be at + * least one callback after `stop` has been called. */ def stop(interruptTimer: Boolean): Long = synchronized { if (!stopped) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala new file mode 100644 index 000000000000..3a21cfae5ac2 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala @@ -0,0 +1,375 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.util + +import java.io._ + +import scala.reflect.ClassTag + +import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.esotericsoftware.kryo.io.{Input, Output} + +import org.apache.spark.SparkConf +import org.apache.spark.serializer.{KryoInputObjectInputBridge, KryoOutputObjectOutputBridge} +import org.apache.spark.streaming.util.OpenHashMapBasedStateMap._ +import org.apache.spark.util.collection.OpenHashMap + +/** Internal interface for defining the map that keeps track of sessions. */ +private[streaming] abstract class StateMap[K, S] extends Serializable { + + /** Get the state for a key if it exists */ + def get(key: K): Option[S] + + /** Get all the keys and states whose updated time is older than the given threshold time */ + def getByTime(threshUpdatedTime: Long): Iterator[(K, S, Long)] + + /** Get all the keys and states in this map. */ + def getAll(): Iterator[(K, S, Long)] + + /** Add or update state */ + def put(key: K, state: S, updatedTime: Long): Unit + + /** Remove a key */ + def remove(key: K): Unit + + /** + * Shallow copy `this` map to create a new state map. + * Updates to the new map should not mutate `this` map. + */ + def copy(): StateMap[K, S] + + def toDebugString(): String = toString() +} + +/** Companion object for [[StateMap]], with utility methods */ +private[streaming] object StateMap { + def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S] + + def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = { + val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold", + DELTA_CHAIN_LENGTH_THRESHOLD) + new OpenHashMapBasedStateMap[K, S](deltaChainThreshold) + } +} + +/** Implementation of StateMap interface representing an empty map */ +private[streaming] class EmptyStateMap[K, S] extends StateMap[K, S] { + override def put(key: K, session: S, updateTime: Long): Unit = { + throw new NotImplementedError("put() should not be called on an EmptyStateMap") + } + override def get(key: K): Option[S] = None + override def getByTime(threshUpdatedTime: Long): Iterator[(K, S, Long)] = Iterator.empty + override def getAll(): Iterator[(K, S, Long)] = Iterator.empty + override def copy(): StateMap[K, S] = this + override def remove(key: K): Unit = { } + override def toDebugString(): String = "" +} + +/** Implementation of StateMap based on Spark's [[org.apache.spark.util.collection.OpenHashMap]] */ +private[streaming] class OpenHashMapBasedStateMap[K, S]( + @transient @volatile var parentStateMap: StateMap[K, S], + private var initialCapacity: Int = DEFAULT_INITIAL_CAPACITY, + private var deltaChainThreshold: Int = DELTA_CHAIN_LENGTH_THRESHOLD + )(implicit private var keyClassTag: ClassTag[K], private var stateClassTag: ClassTag[S]) + extends StateMap[K, S] with KryoSerializable { self => + + def this(initialCapacity: Int, deltaChainThreshold: Int) + (implicit keyClassTag: ClassTag[K], stateClassTag: ClassTag[S]) = this( + new EmptyStateMap[K, S], + initialCapacity = initialCapacity, + deltaChainThreshold = deltaChainThreshold) + + def this(deltaChainThreshold: Int) + (implicit keyClassTag: ClassTag[K], stateClassTag: ClassTag[S]) = this( + initialCapacity = DEFAULT_INITIAL_CAPACITY, deltaChainThreshold = deltaChainThreshold) + + def this()(implicit keyClassTag: ClassTag[K], stateClassTag: ClassTag[S]) = { + this(DELTA_CHAIN_LENGTH_THRESHOLD) + } + + require(initialCapacity >= 1, "Invalid initial capacity") + require(deltaChainThreshold >= 1, "Invalid delta chain threshold") + + @transient @volatile private var deltaMap = new OpenHashMap[K, StateInfo[S]](initialCapacity) + + /** Get the session data if it exists */ + override def get(key: K): Option[S] = { + val stateInfo = deltaMap(key) + if (stateInfo != null) { + if (!stateInfo.deleted) { + Some(stateInfo.data) + } else { + None + } + } else { + parentStateMap.get(key) + } + } + + /** Get all the keys and states whose updated time is older than the give threshold time */ + override def getByTime(threshUpdatedTime: Long): Iterator[(K, S, Long)] = { + val oldStates = parentStateMap.getByTime(threshUpdatedTime).filter { case (key, value, _) => + !deltaMap.contains(key) + } + + val updatedStates = deltaMap.iterator.filter { case (_, stateInfo) => + !stateInfo.deleted && stateInfo.updateTime < threshUpdatedTime + }.map { case (key, stateInfo) => + (key, stateInfo.data, stateInfo.updateTime) + } + oldStates ++ updatedStates + } + + /** Get all the keys and states in this map. */ + override def getAll(): Iterator[(K, S, Long)] = { + + val oldStates = parentStateMap.getAll().filter { case (key, _, _) => + !deltaMap.contains(key) + } + + val updatedStates = deltaMap.iterator.filter { ! _._2.deleted }.map { case (key, stateInfo) => + (key, stateInfo.data, stateInfo.updateTime) + } + oldStates ++ updatedStates + } + + /** Add or update state */ + override def put(key: K, state: S, updateTime: Long): Unit = { + val stateInfo = deltaMap(key) + if (stateInfo != null) { + stateInfo.update(state, updateTime) + } else { + deltaMap.update(key, new StateInfo(state, updateTime)) + } + } + + /** Remove a state */ + override def remove(key: K): Unit = { + val stateInfo = deltaMap(key) + if (stateInfo != null) { + stateInfo.markDeleted() + } else { + val newInfo = new StateInfo[S](deleted = true) + deltaMap.update(key, newInfo) + } + } + + /** + * Shallow copy the map to create a new session store. Updates to the new map + * should not mutate `this` map. + */ + override def copy(): StateMap[K, S] = { + new OpenHashMapBasedStateMap[K, S](this, deltaChainThreshold = deltaChainThreshold) + } + + /** Whether the delta chain length is long enough that it should be compacted */ + def shouldCompact: Boolean = { + deltaChainLength >= deltaChainThreshold + } + + /** Length of the delta chains of this map */ + def deltaChainLength: Int = parentStateMap match { + case map: OpenHashMapBasedStateMap[_, _] => map.deltaChainLength + 1 + case _ => 0 + } + + /** + * Approximate number of keys in the map. This is an overestimation that is mainly used to + * reserve capacity in a new map at delta compaction time. + */ + def approxSize: Int = deltaMap.size + { + parentStateMap match { + case s: OpenHashMapBasedStateMap[_, _] => s.approxSize + case _ => 0 + } + } + + /** Get all the data of this map as string formatted as a tree based on the delta depth */ + override def toDebugString(): String = { + val tabs = if (deltaChainLength > 0) { + (" " * (deltaChainLength - 1)) + "+--- " + } else "" + parentStateMap.toDebugString() + "\n" + deltaMap.iterator.mkString(tabs, "\n" + tabs, "") + } + + override def toString(): String = { + s"[${System.identityHashCode(this)}, ${System.identityHashCode(parentStateMap)}]" + } + + /** + * Serialize the map data. Besides serialization, this method actually compact the deltas + * (if needed) in a single pass over all the data in the map. + */ + private def writeObjectInternal(outputStream: ObjectOutput): Unit = { + // Write the data in the delta of this state map + outputStream.writeInt(deltaMap.size) + val deltaMapIterator = deltaMap.iterator + var deltaMapCount = 0 + while (deltaMapIterator.hasNext) { + deltaMapCount += 1 + val (key, stateInfo) = deltaMapIterator.next() + outputStream.writeObject(key) + outputStream.writeObject(stateInfo) + } + assert(deltaMapCount == deltaMap.size) + + // Write the data in the parent state map while copying the data into a new parent map for + // compaction (if needed) + val doCompaction = shouldCompact + val newParentSessionStore = if (doCompaction) { + val initCapacity = if (approxSize > 0) approxSize else 64 + new OpenHashMapBasedStateMap[K, S](initialCapacity = initCapacity, deltaChainThreshold) + } else { null } + + val iterOfActiveSessions = parentStateMap.getAll() + + var parentSessionCount = 0 + + // First write the approximate size of the data to be written, so that readObject can + // allocate appropriately sized OpenHashMap. + outputStream.writeInt(approxSize) + + while(iterOfActiveSessions.hasNext) { + parentSessionCount += 1 + + val (key, state, updateTime) = iterOfActiveSessions.next() + outputStream.writeObject(key) + outputStream.writeObject(state) + outputStream.writeLong(updateTime) + + if (doCompaction) { + newParentSessionStore.deltaMap.update( + key, StateInfo(state, updateTime, deleted = false)) + } + } + + // Write the final limit marking object with the correct count of records written. + val limiterObj = new LimitMarker(parentSessionCount) + outputStream.writeObject(limiterObj) + if (doCompaction) { + parentStateMap = newParentSessionStore + } + } + + /** Deserialize the map data. */ + private def readObjectInternal(inputStream: ObjectInput): Unit = { + // Read the data of the delta + val deltaMapSize = inputStream.readInt() + deltaMap = if (deltaMapSize != 0) { + new OpenHashMap[K, StateInfo[S]](deltaMapSize) + } else { + new OpenHashMap[K, StateInfo[S]](initialCapacity) + } + var deltaMapCount = 0 + while (deltaMapCount < deltaMapSize) { + val key = inputStream.readObject().asInstanceOf[K] + val sessionInfo = inputStream.readObject().asInstanceOf[StateInfo[S]] + deltaMap.update(key, sessionInfo) + deltaMapCount += 1 + } + + + // Read the data of the parent map. Keep reading records, until the limiter is reached + // First read the approximate number of records to expect and allocate properly size + // OpenHashMap + val parentStateMapSizeHint = inputStream.readInt() + val newStateMapInitialCapacity = math.max(parentStateMapSizeHint, DEFAULT_INITIAL_CAPACITY) + val newParentSessionStore = new OpenHashMapBasedStateMap[K, S]( + initialCapacity = newStateMapInitialCapacity, deltaChainThreshold) + + // Read the records until the limit marking object has been reached + var parentSessionLoopDone = false + while(!parentSessionLoopDone) { + val obj = inputStream.readObject() + if (obj.isInstanceOf[LimitMarker]) { + parentSessionLoopDone = true + val expectedCount = obj.asInstanceOf[LimitMarker].num + assert(expectedCount == newParentSessionStore.deltaMap.size) + } else { + val key = obj.asInstanceOf[K] + val state = inputStream.readObject().asInstanceOf[S] + val updateTime = inputStream.readLong() + newParentSessionStore.deltaMap.update( + key, StateInfo(state, updateTime, deleted = false)) + } + } + parentStateMap = newParentSessionStore + } + + private def writeObject(outputStream: ObjectOutputStream): Unit = { + // Write all the non-transient fields, especially class tags, etc. + outputStream.defaultWriteObject() + writeObjectInternal(outputStream) + } + + private def readObject(inputStream: ObjectInputStream): Unit = { + // Read the non-transient fields, especially class tags, etc. + inputStream.defaultReadObject() + readObjectInternal(inputStream) + } + + override def write(kryo: Kryo, output: Output): Unit = { + output.writeInt(initialCapacity) + output.writeInt(deltaChainThreshold) + kryo.writeClassAndObject(output, keyClassTag) + kryo.writeClassAndObject(output, stateClassTag) + writeObjectInternal(new KryoOutputObjectOutputBridge(kryo, output)) + } + + override def read(kryo: Kryo, input: Input): Unit = { + initialCapacity = input.readInt() + deltaChainThreshold = input.readInt() + keyClassTag = kryo.readClassAndObject(input).asInstanceOf[ClassTag[K]] + stateClassTag = kryo.readClassAndObject(input).asInstanceOf[ClassTag[S]] + readObjectInternal(new KryoInputObjectInputBridge(kryo, input)) + } +} + +/** + * Companion object of [[OpenHashMapBasedStateMap]] having associated helper + * classes and methods + */ +private[streaming] object OpenHashMapBasedStateMap { + + /** Internal class to represent the state information */ + case class StateInfo[S]( + var data: S = null.asInstanceOf[S], + var updateTime: Long = -1, + var deleted: Boolean = false) { + + def markDeleted(): Unit = { + deleted = true + } + + def update(newData: S, newUpdateTime: Long): Unit = { + data = newData + updateTime = newUpdateTime + deleted = false + } + } + + /** + * Internal class to represent a marker the demarkate the end of all state data in the + * serialized bytes. + */ + class LimitMarker(val num: Int) extends Serializable + + val DELTA_CHAIN_LENGTH_THRESHOLD = 20 + + val DEFAULT_INITIAL_CAPACITY = 64 +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala index 0ea970e61b69..7542e2f5ecf2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala @@ -21,8 +21,9 @@ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils -import org.apache.spark.{Logging, SparkConf, SparkException} /** A helper class with utility functions related to the WriteAheadLog interface */ private[streaming] object WriteAheadLogUtils extends Logging { @@ -38,6 +39,8 @@ private[streaming] object WriteAheadLogUtils extends Logging { val DRIVER_WAL_ROLLING_INTERVAL_CONF_KEY = "spark.streaming.driver.writeAheadLog.rollingIntervalSecs" val DRIVER_WAL_MAX_FAILURES_CONF_KEY = "spark.streaming.driver.writeAheadLog.maxFailures" + val DRIVER_WAL_BATCHING_CONF_KEY = "spark.streaming.driver.writeAheadLog.allowBatching" + val DRIVER_WAL_BATCHING_TIMEOUT_CONF_KEY = "spark.streaming.driver.writeAheadLog.batchingTimeout" val DRIVER_WAL_CLOSE_AFTER_WRITE_CONF_KEY = "spark.streaming.driver.writeAheadLog.closeFileAfterWrite" @@ -64,6 +67,18 @@ private[streaming] object WriteAheadLogUtils extends Logging { } } + def isBatchingEnabled(conf: SparkConf, isDriver: Boolean): Boolean = { + isDriver && conf.getBoolean(DRIVER_WAL_BATCHING_CONF_KEY, defaultValue = true) + } + + /** + * How long we will wait for the wrappedLog in the BatchedWriteAheadLog to write the records + * before we fail the write attempt to unblock receivers. + */ + def getBatchingTimeout(conf: SparkConf): Long = { + conf.getLong(DRIVER_WAL_BATCHING_TIMEOUT_CONF_KEY, defaultValue = 5000) + } + def shouldCloseFileAfterWrite(conf: SparkConf, isDriver: Boolean): Boolean = { if (isDriver) { conf.getBoolean(DRIVER_WAL_CLOSE_AFTER_WRITE_CONF_KEY, defaultValue = false) @@ -115,7 +130,7 @@ private[streaming] object WriteAheadLogUtils extends Logging { } else { sparkConf.getOption(RECEIVER_WAL_CLASS_CONF_KEY) } - classNameOption.map { className => + val wal = classNameOption.map { className => try { instantiateClass( Utils.classForName(className).asInstanceOf[Class[_ <: WriteAheadLog]], sparkConf) @@ -128,6 +143,11 @@ private[streaming] object WriteAheadLogUtils extends Logging { getRollingIntervalSecs(sparkConf, isDriver), getMaxFailures(sparkConf, isDriver), shouldCloseFileAfterWrite(sparkConf, isDriver)) } + if (isBatchingEnabled(sparkConf, isDriver)) { + new BatchedWriteAheadLog(wal, sparkConf) + } else { + wal + } } /** Instantiate the class, either using single arg constructor or zero arg constructor */ diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java deleted file mode 100644 index c5217149224e..000000000000 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ /dev/null @@ -1,1918 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming; - -import java.io.*; -import java.nio.charset.Charset; -import java.util.*; -import java.util.concurrent.atomic.AtomicBoolean; - -import scala.Tuple2; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; - -import org.junit.Assert; -import org.junit.Test; - -import com.google.common.base.Optional; -import com.google.common.io.Files; -import com.google.common.collect.Sets; - -import org.apache.spark.HashPartitioner; -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.*; -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.api.java.*; -import org.apache.spark.util.Utils; -import org.apache.spark.SparkConf; - -// The test suite itself is Serializable so that anonymous Function implementations can be -// serialized, as an alternative to converting these anonymous classes to static inner classes; -// see http://stackoverflow.com/questions/758570/. -public class JavaAPISuite extends LocalJavaStreamingContext implements Serializable { - - public static void equalIterator(Iterator a, Iterator b) { - while (a.hasNext() && b.hasNext()) { - Assert.assertEquals(a.next(), b.next()); - } - Assert.assertEquals(a.hasNext(), b.hasNext()); - } - - public static void equalIterable(Iterable a, Iterable b) { - equalIterator(a.iterator(), b.iterator()); - } - - @Test - public void testInitialization() { - Assert.assertNotNull(ssc.sparkContext()); - } - - @SuppressWarnings("unchecked") - @Test - public void testContextState() { - List> inputData = Arrays.asList(Arrays.asList(1, 2, 3, 4)); - Assert.assertEquals(StreamingContextState.INITIALIZED, ssc.getState()); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaTestUtils.attachTestOutputStream(stream); - Assert.assertEquals(StreamingContextState.INITIALIZED, ssc.getState()); - ssc.start(); - Assert.assertEquals(StreamingContextState.ACTIVE, ssc.getState()); - ssc.stop(); - Assert.assertEquals(StreamingContextState.STOPPED, ssc.getState()); - } - - @SuppressWarnings("unchecked") - @Test - public void testCount() { - List> inputData = Arrays.asList( - Arrays.asList(1,2,3,4), - Arrays.asList(3,4,5), - Arrays.asList(3)); - - List> expected = Arrays.asList( - Arrays.asList(4L), - Arrays.asList(3L), - Arrays.asList(1L)); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream count = stream.count(); - JavaTestUtils.attachTestOutputStream(count); - List> result = JavaTestUtils.runStreams(ssc, 3, 3); - assertOrderInvariantEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testMap() { - List> inputData = Arrays.asList( - Arrays.asList("hello", "world"), - Arrays.asList("goodnight", "moon")); - - List> expected = Arrays.asList( - Arrays.asList(5,5), - Arrays.asList(9,4)); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream letterCount = stream.map(new Function() { - @Override - public Integer call(String s) { - return s.length(); - } - }); - JavaTestUtils.attachTestOutputStream(letterCount); - List> result = JavaTestUtils.runStreams(ssc, 2, 2); - - assertOrderInvariantEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testWindow() { - List> inputData = Arrays.asList( - Arrays.asList(1,2,3), - Arrays.asList(4,5,6), - Arrays.asList(7,8,9)); - - List> expected = Arrays.asList( - Arrays.asList(1,2,3), - Arrays.asList(4,5,6,1,2,3), - Arrays.asList(7,8,9,4,5,6), - Arrays.asList(7,8,9)); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream windowed = stream.window(new Duration(2000)); - JavaTestUtils.attachTestOutputStream(windowed); - List> result = JavaTestUtils.runStreams(ssc, 4, 4); - - assertOrderInvariantEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testWindowWithSlideDuration() { - List> inputData = Arrays.asList( - Arrays.asList(1,2,3), - Arrays.asList(4,5,6), - Arrays.asList(7,8,9), - Arrays.asList(10,11,12), - Arrays.asList(13,14,15), - Arrays.asList(16,17,18)); - - List> expected = Arrays.asList( - Arrays.asList(1,2,3,4,5,6), - Arrays.asList(1,2,3,4,5,6,7,8,9,10,11,12), - Arrays.asList(7,8,9,10,11,12,13,14,15,16,17,18), - Arrays.asList(13,14,15,16,17,18)); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream windowed = stream.window(new Duration(4000), new Duration(2000)); - JavaTestUtils.attachTestOutputStream(windowed); - List> result = JavaTestUtils.runStreams(ssc, 8, 4); - - assertOrderInvariantEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testFilter() { - List> inputData = Arrays.asList( - Arrays.asList("giants", "dodgers"), - Arrays.asList("yankees", "red sox")); - - List> expected = Arrays.asList( - Arrays.asList("giants"), - Arrays.asList("yankees")); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream filtered = stream.filter(new Function() { - @Override - public Boolean call(String s) { - return s.contains("a"); - } - }); - JavaTestUtils.attachTestOutputStream(filtered); - List> result = JavaTestUtils.runStreams(ssc, 2, 2); - - assertOrderInvariantEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testRepartitionMorePartitions() { - List> inputData = Arrays.asList( - Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10), - Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); - JavaDStream stream = - JavaTestUtils.attachTestInputStream(ssc, inputData, 2); - JavaDStreamLike,JavaRDD> repartitioned = - stream.repartition(4); - JavaTestUtils.attachTestOutputStream(repartitioned); - List>> result = JavaTestUtils.runStreamsWithPartitions(ssc, 2, 2); - Assert.assertEquals(2, result.size()); - for (List> rdd : result) { - Assert.assertEquals(4, rdd.size()); - Assert.assertEquals( - 10, rdd.get(0).size() + rdd.get(1).size() + rdd.get(2).size() + rdd.get(3).size()); - } - } - - @SuppressWarnings("unchecked") - @Test - public void testRepartitionFewerPartitions() { - List> inputData = Arrays.asList( - Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10), - Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); - JavaDStream stream = - JavaTestUtils.attachTestInputStream(ssc, inputData, 4); - JavaDStreamLike,JavaRDD> repartitioned = - stream.repartition(2); - JavaTestUtils.attachTestOutputStream(repartitioned); - List>> result = JavaTestUtils.runStreamsWithPartitions(ssc, 2, 2); - Assert.assertEquals(2, result.size()); - for (List> rdd : result) { - Assert.assertEquals(2, rdd.size()); - Assert.assertEquals(10, rdd.get(0).size() + rdd.get(1).size()); - } - } - - @SuppressWarnings("unchecked") - @Test - public void testGlom() { - List> inputData = Arrays.asList( - Arrays.asList("giants", "dodgers"), - Arrays.asList("yankees", "red sox")); - - List>> expected = Arrays.asList( - Arrays.asList(Arrays.asList("giants", "dodgers")), - Arrays.asList(Arrays.asList("yankees", "red sox"))); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream> glommed = stream.glom(); - JavaTestUtils.attachTestOutputStream(glommed); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testMapPartitions() { - List> inputData = Arrays.asList( - Arrays.asList("giants", "dodgers"), - Arrays.asList("yankees", "red sox")); - - List> expected = Arrays.asList( - Arrays.asList("GIANTSDODGERS"), - Arrays.asList("YANKEESRED SOX")); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream mapped = stream.mapPartitions( - new FlatMapFunction, String>() { - @Override - public Iterable call(Iterator in) { - StringBuilder out = new StringBuilder(); - while (in.hasNext()) { - out.append(in.next().toUpperCase(Locale.ENGLISH)); - } - return Arrays.asList(out.toString()); - } - }); - JavaTestUtils.attachTestOutputStream(mapped); - List> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - private static class IntegerSum implements Function2 { - @Override - public Integer call(Integer i1, Integer i2) { - return i1 + i2; - } - } - - private static class IntegerDifference implements Function2 { - @Override - public Integer call(Integer i1, Integer i2) { - return i1 - i2; - } - } - - @SuppressWarnings("unchecked") - @Test - public void testReduce() { - List> inputData = Arrays.asList( - Arrays.asList(1,2,3), - Arrays.asList(4,5,6), - Arrays.asList(7,8,9)); - - List> expected = Arrays.asList( - Arrays.asList(6), - Arrays.asList(15), - Arrays.asList(24)); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream reduced = stream.reduce(new IntegerSum()); - JavaTestUtils.attachTestOutputStream(reduced); - List> result = JavaTestUtils.runStreams(ssc, 3, 3); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testReduceByWindowWithInverse() { - testReduceByWindow(true); - } - - @SuppressWarnings("unchecked") - @Test - public void testReduceByWindowWithoutInverse() { - testReduceByWindow(false); - } - - @SuppressWarnings("unchecked") - private void testReduceByWindow(boolean withInverse) { - List> inputData = Arrays.asList( - Arrays.asList(1,2,3), - Arrays.asList(4,5,6), - Arrays.asList(7,8,9)); - - List> expected = Arrays.asList( - Arrays.asList(6), - Arrays.asList(21), - Arrays.asList(39), - Arrays.asList(24)); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream reducedWindowed; - if (withInverse) { - reducedWindowed = stream.reduceByWindow(new IntegerSum(), - new IntegerDifference(), new Duration(2000), new Duration(1000)); - } else { - reducedWindowed = stream.reduceByWindow(new IntegerSum(), - new Duration(2000), new Duration(1000)); - } - JavaTestUtils.attachTestOutputStream(reducedWindowed); - List> result = JavaTestUtils.runStreams(ssc, 4, 4); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testQueueStream() { - ssc.stop(); - // Create a new JavaStreamingContext without checkpointing - SparkConf conf = new SparkConf() - .setMaster("local[2]") - .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); - ssc = new JavaStreamingContext(conf, new Duration(1000)); - - List> expected = Arrays.asList( - Arrays.asList(1,2,3), - Arrays.asList(4,5,6), - Arrays.asList(7,8,9)); - - JavaSparkContext jsc = new JavaSparkContext(ssc.ssc().sc()); - JavaRDD rdd1 = jsc.parallelize(Arrays.asList(1, 2, 3)); - JavaRDD rdd2 = jsc.parallelize(Arrays.asList(4, 5, 6)); - JavaRDD rdd3 = jsc.parallelize(Arrays.asList(7,8,9)); - - Queue> rdds = new LinkedList<>(); - rdds.add(rdd1); - rdds.add(rdd2); - rdds.add(rdd3); - - JavaDStream stream = ssc.queueStream(rdds); - JavaTestUtils.attachTestOutputStream(stream); - List> result = JavaTestUtils.runStreams(ssc, 3, 3); - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testTransform() { - List> inputData = Arrays.asList( - Arrays.asList(1,2,3), - Arrays.asList(4,5,6), - Arrays.asList(7,8,9)); - - List> expected = Arrays.asList( - Arrays.asList(3,4,5), - Arrays.asList(6,7,8), - Arrays.asList(9,10,11)); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream transformed = stream.transform( - new Function, JavaRDD>() { - @Override - public JavaRDD call(JavaRDD in) { - return in.map(new Function() { - @Override - public Integer call(Integer i) { - return i + 2; - } - }); - } - }); - - JavaTestUtils.attachTestOutputStream(transformed); - List> result = JavaTestUtils.runStreams(ssc, 3, 3); - - assertOrderInvariantEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testVariousTransform() { - // tests whether all variations of transform can be called from Java - - List> inputData = Arrays.asList(Arrays.asList(1)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - - List>> pairInputData = - Arrays.asList(Arrays.asList(new Tuple2<>("x", 1))); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream( - JavaTestUtils.attachTestInputStream(ssc, pairInputData, 1)); - - stream.transform( - new Function, JavaRDD>() { - @Override - public JavaRDD call(JavaRDD in) { - return null; - } - } - ); - - stream.transform( - new Function2, Time, JavaRDD>() { - @Override public JavaRDD call(JavaRDD in, Time time) { - return null; - } - } - ); - - stream.transformToPair( - new Function, JavaPairRDD>() { - @Override public JavaPairRDD call(JavaRDD in) { - return null; - } - } - ); - - stream.transformToPair( - new Function2, Time, JavaPairRDD>() { - @Override public JavaPairRDD call(JavaRDD in, Time time) { - return null; - } - } - ); - - pairStream.transform( - new Function, JavaRDD>() { - @Override public JavaRDD call(JavaPairRDD in) { - return null; - } - } - ); - - pairStream.transform( - new Function2, Time, JavaRDD>() { - @Override public JavaRDD call(JavaPairRDD in, Time time) { - return null; - } - } - ); - - pairStream.transformToPair( - new Function, JavaPairRDD>() { - @Override public JavaPairRDD call(JavaPairRDD in) { - return null; - } - } - ); - - pairStream.transformToPair( - new Function2, Time, JavaPairRDD>() { - @Override public JavaPairRDD call(JavaPairRDD in, Time time) { - return null; - } - } - ); - - } - - @SuppressWarnings("unchecked") - @Test - public void testTransformWith() { - List>> stringStringKVStream1 = Arrays.asList( - Arrays.asList( - new Tuple2<>("california", "dodgers"), - new Tuple2<>("new york", "yankees")), - Arrays.asList( - new Tuple2<>("california", "sharks"), - new Tuple2<>("new york", "rangers"))); - - List>> stringStringKVStream2 = Arrays.asList( - Arrays.asList( - new Tuple2<>("california", "giants"), - new Tuple2<>("new york", "mets")), - Arrays.asList( - new Tuple2<>("california", "ducks"), - new Tuple2<>("new york", "islanders"))); - - - List>>> expected = Arrays.asList( - Sets.newHashSet( - new Tuple2<>("california", - new Tuple2<>("dodgers", "giants")), - new Tuple2<>("new york", - new Tuple2<>("yankees", "mets"))), - Sets.newHashSet( - new Tuple2<>("california", - new Tuple2<>("sharks", "ducks")), - new Tuple2<>("new york", - new Tuple2<>("rangers", "islanders")))); - - JavaDStream> stream1 = JavaTestUtils.attachTestInputStream( - ssc, stringStringKVStream1, 1); - JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream(stream1); - - JavaDStream> stream2 = JavaTestUtils.attachTestInputStream( - ssc, stringStringKVStream2, 1); - JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); - - JavaPairDStream> joined = pairStream1.transformWithToPair( - pairStream2, - new Function3< - JavaPairRDD, - JavaPairRDD, - Time, - JavaPairRDD>>() { - @Override - public JavaPairRDD> call( - JavaPairRDD rdd1, - JavaPairRDD rdd2, - Time time) { - return rdd1.join(rdd2); - } - } - ); - - JavaTestUtils.attachTestOutputStream(joined); - List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - List>>> unorderedResult = new ArrayList<>(); - for (List>> res: result) { - unorderedResult.add(Sets.newHashSet(res)); - } - - Assert.assertEquals(expected, unorderedResult); - } - - - @SuppressWarnings("unchecked") - @Test - public void testVariousTransformWith() { - // tests whether all variations of transformWith can be called from Java - - List> inputData1 = Arrays.asList(Arrays.asList(1)); - List> inputData2 = Arrays.asList(Arrays.asList("x")); - JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 1); - JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 1); - - List>> pairInputData1 = - Arrays.asList(Arrays.asList(new Tuple2<>("x", 1))); - List>> pairInputData2 = - Arrays.asList(Arrays.asList(new Tuple2<>(1.0, 'x'))); - JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream( - JavaTestUtils.attachTestInputStream(ssc, pairInputData1, 1)); - JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream( - JavaTestUtils.attachTestInputStream(ssc, pairInputData2, 1)); - - stream1.transformWith( - stream2, - new Function3, JavaRDD, Time, JavaRDD>() { - @Override - public JavaRDD call(JavaRDD rdd1, JavaRDD rdd2, Time time) { - return null; - } - } - ); - - stream1.transformWith( - pairStream1, - new Function3, JavaPairRDD, Time, JavaRDD>() { - @Override - public JavaRDD call(JavaRDD rdd1, JavaPairRDD rdd2, Time time) { - return null; - } - } - ); - - stream1.transformWithToPair( - stream2, - new Function3, JavaRDD, Time, JavaPairRDD>() { - @Override - public JavaPairRDD call(JavaRDD rdd1, JavaRDD rdd2, Time time) { - return null; - } - } - ); - - stream1.transformWithToPair( - pairStream1, - new Function3, JavaPairRDD, Time, JavaPairRDD>() { - @Override - public JavaPairRDD call(JavaRDD rdd1, JavaPairRDD rdd2, Time time) { - return null; - } - } - ); - - pairStream1.transformWith( - stream2, - new Function3, JavaRDD, Time, JavaRDD>() { - @Override - public JavaRDD call(JavaPairRDD rdd1, JavaRDD rdd2, Time time) { - return null; - } - } - ); - - pairStream1.transformWith( - pairStream1, - new Function3, JavaPairRDD, Time, JavaRDD>() { - @Override - public JavaRDD call(JavaPairRDD rdd1, JavaPairRDD rdd2, Time time) { - return null; - } - } - ); - - pairStream1.transformWithToPair( - stream2, - new Function3, JavaRDD, Time, JavaPairRDD>() { - @Override - public JavaPairRDD call(JavaPairRDD rdd1, JavaRDD rdd2, Time time) { - return null; - } - } - ); - - pairStream1.transformWithToPair( - pairStream2, - new Function3, JavaPairRDD, Time, JavaPairRDD>() { - @Override - public JavaPairRDD call(JavaPairRDD rdd1, JavaPairRDD rdd2, Time time) { - return null; - } - } - ); - } - - @SuppressWarnings("unchecked") - @Test - public void testStreamingContextTransform(){ - List> stream1input = Arrays.asList( - Arrays.asList(1), - Arrays.asList(2) - ); - - List> stream2input = Arrays.asList( - Arrays.asList(3), - Arrays.asList(4) - ); - - List>> pairStream1input = Arrays.asList( - Arrays.asList(new Tuple2<>(1, "x")), - Arrays.asList(new Tuple2<>(2, "y")) - ); - - List>>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>(1, new Tuple2<>(1, "x"))), - Arrays.asList(new Tuple2<>(2, new Tuple2<>(2, "y"))) - ); - - JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, stream1input, 1); - JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, stream2input, 1); - JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream( - JavaTestUtils.attachTestInputStream(ssc, pairStream1input, 1)); - - List> listOfDStreams1 = Arrays.>asList(stream1, stream2); - - // This is just to test whether this transform to JavaStream compiles - ssc.transform( - listOfDStreams1, - new Function2>, Time, JavaRDD>() { - @Override - public JavaRDD call(List> listOfRDDs, Time time) { - Assert.assertEquals(2, listOfRDDs.size()); - return null; - } - } - ); - - List> listOfDStreams2 = - Arrays.>asList(stream1, stream2, pairStream1.toJavaDStream()); - - JavaPairDStream> transformed2 = ssc.transformToPair( - listOfDStreams2, - new Function2>, Time, JavaPairRDD>>() { - @Override - public JavaPairRDD> call(List> listOfRDDs, Time time) { - Assert.assertEquals(3, listOfRDDs.size()); - JavaRDD rdd1 = (JavaRDD)listOfRDDs.get(0); - JavaRDD rdd2 = (JavaRDD)listOfRDDs.get(1); - JavaRDD> rdd3 = (JavaRDD>)listOfRDDs.get(2); - JavaPairRDD prdd3 = JavaPairRDD.fromJavaRDD(rdd3); - PairFunction mapToTuple = new PairFunction() { - @Override - public Tuple2 call(Integer i) { - return new Tuple2<>(i, i); - } - }; - return rdd1.union(rdd2).mapToPair(mapToTuple).join(prdd3); - } - } - ); - JavaTestUtils.attachTestOutputStream(transformed2); - List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testFlatMap() { - List> inputData = Arrays.asList( - Arrays.asList("go", "giants"), - Arrays.asList("boo", "dodgers"), - Arrays.asList("athletics")); - - List> expected = Arrays.asList( - Arrays.asList("g","o","g","i","a","n","t","s"), - Arrays.asList("b", "o", "o", "d","o","d","g","e","r","s"), - Arrays.asList("a","t","h","l","e","t","i","c","s")); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream flatMapped = stream.flatMap(new FlatMapFunction() { - @Override - public Iterable call(String x) { - return Arrays.asList(x.split("(?!^)")); - } - }); - JavaTestUtils.attachTestOutputStream(flatMapped); - List> result = JavaTestUtils.runStreams(ssc, 3, 3); - - assertOrderInvariantEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testPairFlatMap() { - List> inputData = Arrays.asList( - Arrays.asList("giants"), - Arrays.asList("dodgers"), - Arrays.asList("athletics")); - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>(6, "g"), - new Tuple2<>(6, "i"), - new Tuple2<>(6, "a"), - new Tuple2<>(6, "n"), - new Tuple2<>(6, "t"), - new Tuple2<>(6, "s")), - Arrays.asList( - new Tuple2<>(7, "d"), - new Tuple2<>(7, "o"), - new Tuple2<>(7, "d"), - new Tuple2<>(7, "g"), - new Tuple2<>(7, "e"), - new Tuple2<>(7, "r"), - new Tuple2<>(7, "s")), - Arrays.asList( - new Tuple2<>(9, "a"), - new Tuple2<>(9, "t"), - new Tuple2<>(9, "h"), - new Tuple2<>(9, "l"), - new Tuple2<>(9, "e"), - new Tuple2<>(9, "t"), - new Tuple2<>(9, "i"), - new Tuple2<>(9, "c"), - new Tuple2<>(9, "s"))); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream flatMapped = stream.flatMapToPair( - new PairFlatMapFunction() { - @Override - public Iterable> call(String in) { - List> out = new ArrayList<>(); - for (String letter: in.split("(?!^)")) { - out.add(new Tuple2<>(in.length(), letter)); - } - return out; - } - }); - JavaTestUtils.attachTestOutputStream(flatMapped); - List>> result = JavaTestUtils.runStreams(ssc, 3, 3); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testUnion() { - List> inputData1 = Arrays.asList( - Arrays.asList(1,1), - Arrays.asList(2,2), - Arrays.asList(3,3)); - - List> inputData2 = Arrays.asList( - Arrays.asList(4,4), - Arrays.asList(5,5), - Arrays.asList(6,6)); - - List> expected = Arrays.asList( - Arrays.asList(1,1,4,4), - Arrays.asList(2,2,5,5), - Arrays.asList(3,3,6,6)); - - JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2); - JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2); - - JavaDStream unioned = stream1.union(stream2); - JavaTestUtils.attachTestOutputStream(unioned); - List> result = JavaTestUtils.runStreams(ssc, 3, 3); - - assertOrderInvariantEquals(expected, result); - } - - /* - * Performs an order-invariant comparison of lists representing two RDD streams. This allows - * us to account for ordering variation within individual RDD's which occurs during windowing. - */ - public static void assertOrderInvariantEquals( - List> expected, List> actual) { - List> expectedSets = new ArrayList<>(); - for (List list: expected) { - expectedSets.add(Collections.unmodifiableSet(new HashSet<>(list))); - } - List> actualSets = new ArrayList<>(); - for (List list: actual) { - actualSets.add(Collections.unmodifiableSet(new HashSet<>(list))); - } - Assert.assertEquals(expectedSets, actualSets); - } - - - // PairDStream Functions - @SuppressWarnings("unchecked") - @Test - public void testPairFilter() { - List> inputData = Arrays.asList( - Arrays.asList("giants", "dodgers"), - Arrays.asList("yankees", "red sox")); - - List>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>("giants", 6)), - Arrays.asList(new Tuple2<>("yankees", 7))); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = stream.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(String in) { - return new Tuple2<>(in, in.length()); - } - }); - - JavaPairDStream filtered = pairStream.filter( - new Function, Boolean>() { - @Override - public Boolean call(Tuple2 in) { - return in._1().contains("a"); - } - }); - JavaTestUtils.attachTestOutputStream(filtered); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - private final List>> stringStringKVStream = Arrays.asList( - Arrays.asList(new Tuple2<>("california", "dodgers"), - new Tuple2<>("california", "giants"), - new Tuple2<>("new york", "yankees"), - new Tuple2<>("new york", "mets")), - Arrays.asList(new Tuple2<>("california", "sharks"), - new Tuple2<>("california", "ducks"), - new Tuple2<>("new york", "rangers"), - new Tuple2<>("new york", "islanders"))); - - @SuppressWarnings("unchecked") - private final List>> stringIntKVStream = Arrays.asList( - Arrays.asList( - new Tuple2<>("california", 1), - new Tuple2<>("california", 3), - new Tuple2<>("new york", 4), - new Tuple2<>("new york", 1)), - Arrays.asList( - new Tuple2<>("california", 5), - new Tuple2<>("california", 5), - new Tuple2<>("new york", 3), - new Tuple2<>("new york", 1))); - - @SuppressWarnings("unchecked") - @Test - public void testPairMap() { // Maps pair -> pair of different type - List>> inputData = stringIntKVStream; - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>(1, "california"), - new Tuple2<>(3, "california"), - new Tuple2<>(4, "new york"), - new Tuple2<>(1, "new york")), - Arrays.asList( - new Tuple2<>(5, "california"), - new Tuple2<>(5, "california"), - new Tuple2<>(3, "new york"), - new Tuple2<>(1, "new york"))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream reversed = pairStream.mapToPair( - new PairFunction, Integer, String>() { - @Override - public Tuple2 call(Tuple2 in) { - return in.swap(); - } - }); - - JavaTestUtils.attachTestOutputStream(reversed); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testPairMapPartitions() { // Maps pair -> pair of different type - List>> inputData = stringIntKVStream; - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>(1, "california"), - new Tuple2<>(3, "california"), - new Tuple2<>(4, "new york"), - new Tuple2<>(1, "new york")), - Arrays.asList( - new Tuple2<>(5, "california"), - new Tuple2<>(5, "california"), - new Tuple2<>(3, "new york"), - new Tuple2<>(1, "new york"))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream reversed = pairStream.mapPartitionsToPair( - new PairFlatMapFunction>, Integer, String>() { - @Override - public Iterable> call(Iterator> in) { - List> out = new LinkedList<>(); - while (in.hasNext()) { - Tuple2 next = in.next(); - out.add(next.swap()); - } - return out; - } - }); - - JavaTestUtils.attachTestOutputStream(reversed); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testPairMap2() { // Maps pair -> single - List>> inputData = stringIntKVStream; - - List> expected = Arrays.asList( - Arrays.asList(1, 3, 4, 1), - Arrays.asList(5, 5, 3, 1)); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaDStream reversed = pairStream.map( - new Function, Integer>() { - @Override - public Integer call(Tuple2 in) { - return in._2(); - } - }); - - JavaTestUtils.attachTestOutputStream(reversed); - List> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair - List>> inputData = Arrays.asList( - Arrays.asList( - new Tuple2<>("hi", 1), - new Tuple2<>("ho", 2)), - Arrays.asList( - new Tuple2<>("hi", 1), - new Tuple2<>("ho", 2))); - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>(1, "h"), - new Tuple2<>(1, "i"), - new Tuple2<>(2, "h"), - new Tuple2<>(2, "o")), - Arrays.asList( - new Tuple2<>(1, "h"), - new Tuple2<>(1, "i"), - new Tuple2<>(2, "h"), - new Tuple2<>(2, "o"))); - - JavaDStream> stream = - JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream flatMapped = pairStream.flatMapToPair( - new PairFlatMapFunction, Integer, String>() { - @Override - public Iterable> call(Tuple2 in) { - List> out = new LinkedList<>(); - for (Character s : in._1().toCharArray()) { - out.add(new Tuple2<>(in._2(), s.toString())); - } - return out; - } - }); - JavaTestUtils.attachTestOutputStream(flatMapped); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testPairGroupByKey() { - List>> inputData = stringStringKVStream; - - List>>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>("california", Arrays.asList("dodgers", "giants")), - new Tuple2<>("new york", Arrays.asList("yankees", "mets"))), - Arrays.asList( - new Tuple2<>("california", Arrays.asList("sharks", "ducks")), - new Tuple2<>("new york", Arrays.asList("rangers", "islanders")))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream> grouped = pairStream.groupByKey(); - JavaTestUtils.attachTestOutputStream(grouped); - List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected.size(), result.size()); - Iterator>>> resultItr = result.iterator(); - Iterator>>> expectedItr = expected.iterator(); - while (resultItr.hasNext() && expectedItr.hasNext()) { - Iterator>> resultElements = resultItr.next().iterator(); - Iterator>> expectedElements = expectedItr.next().iterator(); - while (resultElements.hasNext() && expectedElements.hasNext()) { - Tuple2> resultElement = resultElements.next(); - Tuple2> expectedElement = expectedElements.next(); - Assert.assertEquals(expectedElement._1(), resultElement._1()); - equalIterable(expectedElement._2(), resultElement._2()); - } - Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext()); - } - } - - @SuppressWarnings("unchecked") - @Test - public void testPairReduceByKey() { - List>> inputData = stringIntKVStream; - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>("california", 4), - new Tuple2<>("new york", 5)), - Arrays.asList( - new Tuple2<>("california", 10), - new Tuple2<>("new york", 4))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream( - ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream reduced = pairStream.reduceByKey(new IntegerSum()); - - JavaTestUtils.attachTestOutputStream(reduced); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testCombineByKey() { - List>> inputData = stringIntKVStream; - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>("california", 4), - new Tuple2<>("new york", 5)), - Arrays.asList( - new Tuple2<>("california", 10), - new Tuple2<>("new york", 4))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream( - ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream combined = pairStream.combineByKey( - new Function() { - @Override - public Integer call(Integer i) { - return i; - } - }, new IntegerSum(), new IntegerSum(), new HashPartitioner(2)); - - JavaTestUtils.attachTestOutputStream(combined); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testCountByValue() { - List> inputData = Arrays.asList( - Arrays.asList("hello", "world"), - Arrays.asList("hello", "moon"), - Arrays.asList("hello")); - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>("hello", 1L), - new Tuple2<>("world", 1L)), - Arrays.asList( - new Tuple2<>("hello", 1L), - new Tuple2<>("moon", 1L)), - Arrays.asList( - new Tuple2<>("hello", 1L))); - - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream counted = stream.countByValue(); - JavaTestUtils.attachTestOutputStream(counted); - List>> result = JavaTestUtils.runStreams(ssc, 3, 3); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testGroupByKeyAndWindow() { - List>> inputData = stringIntKVStream; - - List>>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>("california", Arrays.asList(1, 3)), - new Tuple2<>("new york", Arrays.asList(1, 4)) - ), - Arrays.asList( - new Tuple2<>("california", Arrays.asList(1, 3, 5, 5)), - new Tuple2<>("new york", Arrays.asList(1, 1, 3, 4)) - ), - Arrays.asList( - new Tuple2<>("california", Arrays.asList(5, 5)), - new Tuple2<>("new york", Arrays.asList(1, 3)) - ) - ); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream> groupWindowed = - pairStream.groupByKeyAndWindow(new Duration(2000), new Duration(1000)); - JavaTestUtils.attachTestOutputStream(groupWindowed); - List>>> result = JavaTestUtils.runStreams(ssc, 3, 3); - - Assert.assertEquals(expected.size(), result.size()); - for (int i = 0; i < result.size(); i++) { - Assert.assertEquals(convert(expected.get(i)), convert(result.get(i))); - } - } - - private static Set>> convert(List>> listOfTuples) { - List>> newListOfTuples = new ArrayList<>(); - for (Tuple2> tuple: listOfTuples) { - newListOfTuples.add(convert(tuple)); - } - return new HashSet<>(newListOfTuples); - } - - private static Tuple2> convert(Tuple2> tuple) { - return new Tuple2<>(tuple._1(), new HashSet<>(tuple._2())); - } - - @SuppressWarnings("unchecked") - @Test - public void testReduceByKeyAndWindow() { - List>> inputData = stringIntKVStream; - - List>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>("california", 4), - new Tuple2<>("new york", 5)), - Arrays.asList(new Tuple2<>("california", 14), - new Tuple2<>("new york", 9)), - Arrays.asList(new Tuple2<>("california", 10), - new Tuple2<>("new york", 4))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream reduceWindowed = - pairStream.reduceByKeyAndWindow(new IntegerSum(), new Duration(2000), new Duration(1000)); - JavaTestUtils.attachTestOutputStream(reduceWindowed); - List>> result = JavaTestUtils.runStreams(ssc, 3, 3); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testUpdateStateByKey() { - List>> inputData = stringIntKVStream; - - List>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>("california", 4), - new Tuple2<>("new york", 5)), - Arrays.asList(new Tuple2<>("california", 14), - new Tuple2<>("new york", 9)), - Arrays.asList(new Tuple2<>("california", 14), - new Tuple2<>("new york", 9))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream updated = pairStream.updateStateByKey( - new Function2, Optional, Optional>() { - @Override - public Optional call(List values, Optional state) { - int out = 0; - if (state.isPresent()) { - out += state.get(); - } - for (Integer v : values) { - out += v; - } - return Optional.of(out); - } - }); - JavaTestUtils.attachTestOutputStream(updated); - List>> result = JavaTestUtils.runStreams(ssc, 3, 3); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testUpdateStateByKeyWithInitial() { - List>> inputData = stringIntKVStream; - - List> initial = Arrays.asList ( - new Tuple2<>("california", 1), - new Tuple2<>("new york", 2)); - - JavaRDD> tmpRDD = ssc.sparkContext().parallelize(initial); - JavaPairRDD initialRDD = JavaPairRDD.fromJavaRDD (tmpRDD); - - List>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>("california", 5), - new Tuple2<>("new york", 7)), - Arrays.asList(new Tuple2<>("california", 15), - new Tuple2<>("new york", 11)), - Arrays.asList(new Tuple2<>("california", 15), - new Tuple2<>("new york", 11))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream updated = pairStream.updateStateByKey( - new Function2, Optional, Optional>() { - @Override - public Optional call(List values, Optional state) { - int out = 0; - if (state.isPresent()) { - out += state.get(); - } - for (Integer v : values) { - out += v; - } - return Optional.of(out); - } - }, new HashPartitioner(1), initialRDD); - JavaTestUtils.attachTestOutputStream(updated); - List>> result = JavaTestUtils.runStreams(ssc, 3, 3); - - assertOrderInvariantEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testReduceByKeyAndWindowWithInverse() { - List>> inputData = stringIntKVStream; - - List>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>("california", 4), - new Tuple2<>("new york", 5)), - Arrays.asList(new Tuple2<>("california", 14), - new Tuple2<>("new york", 9)), - Arrays.asList(new Tuple2<>("california", 10), - new Tuple2<>("new york", 4))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream reduceWindowed = - pairStream.reduceByKeyAndWindow(new IntegerSum(), new IntegerDifference(), - new Duration(2000), new Duration(1000)); - JavaTestUtils.attachTestOutputStream(reduceWindowed); - List>> result = JavaTestUtils.runStreams(ssc, 3, 3); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testCountByValueAndWindow() { - List> inputData = Arrays.asList( - Arrays.asList("hello", "world"), - Arrays.asList("hello", "moon"), - Arrays.asList("hello")); - - List>> expected = Arrays.asList( - Sets.newHashSet( - new Tuple2<>("hello", 1L), - new Tuple2<>("world", 1L)), - Sets.newHashSet( - new Tuple2<>("hello", 2L), - new Tuple2<>("world", 1L), - new Tuple2<>("moon", 1L)), - Sets.newHashSet( - new Tuple2<>("hello", 2L), - new Tuple2<>("moon", 1L))); - - JavaDStream stream = JavaTestUtils.attachTestInputStream( - ssc, inputData, 1); - JavaPairDStream counted = - stream.countByValueAndWindow(new Duration(2000), new Duration(1000)); - JavaTestUtils.attachTestOutputStream(counted); - List>> result = JavaTestUtils.runStreams(ssc, 3, 3); - List>> unorderedResult = new ArrayList<>(); - for (List> res: result) { - unorderedResult.add(Sets.newHashSet(res)); - } - - Assert.assertEquals(expected, unorderedResult); - } - - @SuppressWarnings("unchecked") - @Test - public void testPairTransform() { - List>> inputData = Arrays.asList( - Arrays.asList( - new Tuple2<>(3, 5), - new Tuple2<>(1, 5), - new Tuple2<>(4, 5), - new Tuple2<>(2, 5)), - Arrays.asList( - new Tuple2<>(2, 5), - new Tuple2<>(3, 5), - new Tuple2<>(4, 5), - new Tuple2<>(1, 5))); - - List>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>(1, 5), - new Tuple2<>(2, 5), - new Tuple2<>(3, 5), - new Tuple2<>(4, 5)), - Arrays.asList( - new Tuple2<>(1, 5), - new Tuple2<>(2, 5), - new Tuple2<>(3, 5), - new Tuple2<>(4, 5))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream( - ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream sorted = pairStream.transformToPair( - new Function, JavaPairRDD>() { - @Override - public JavaPairRDD call(JavaPairRDD in) { - return in.sortByKey(); - } - }); - - JavaTestUtils.attachTestOutputStream(sorted); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testPairToNormalRDDTransform() { - List>> inputData = Arrays.asList( - Arrays.asList( - new Tuple2<>(3, 5), - new Tuple2<>(1, 5), - new Tuple2<>(4, 5), - new Tuple2<>(2, 5)), - Arrays.asList( - new Tuple2<>(2, 5), - new Tuple2<>(3, 5), - new Tuple2<>(4, 5), - new Tuple2<>(1, 5))); - - List> expected = Arrays.asList( - Arrays.asList(3,1,4,2), - Arrays.asList(2,3,4,1)); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream( - ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaDStream firstParts = pairStream.transform( - new Function, JavaRDD>() { - @Override - public JavaRDD call(JavaPairRDD in) { - return in.map(new Function, Integer>() { - @Override - public Integer call(Tuple2 in2) { - return in2._1(); - } - }); - } - }); - - JavaTestUtils.attachTestOutputStream(firstParts); - List> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testMapValues() { - List>> inputData = stringStringKVStream; - - List>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>("california", "DODGERS"), - new Tuple2<>("california", "GIANTS"), - new Tuple2<>("new york", "YANKEES"), - new Tuple2<>("new york", "METS")), - Arrays.asList(new Tuple2<>("california", "SHARKS"), - new Tuple2<>("california", "DUCKS"), - new Tuple2<>("new york", "RANGERS"), - new Tuple2<>("new york", "ISLANDERS"))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream( - ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - JavaPairDStream mapped = pairStream.mapValues(new Function() { - @Override - public String call(String s) { - return s.toUpperCase(Locale.ENGLISH); - } - }); - - JavaTestUtils.attachTestOutputStream(mapped); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testFlatMapValues() { - List>> inputData = stringStringKVStream; - - List>> expected = Arrays.asList( - Arrays.asList(new Tuple2<>("california", "dodgers1"), - new Tuple2<>("california", "dodgers2"), - new Tuple2<>("california", "giants1"), - new Tuple2<>("california", "giants2"), - new Tuple2<>("new york", "yankees1"), - new Tuple2<>("new york", "yankees2"), - new Tuple2<>("new york", "mets1"), - new Tuple2<>("new york", "mets2")), - Arrays.asList(new Tuple2<>("california", "sharks1"), - new Tuple2<>("california", "sharks2"), - new Tuple2<>("california", "ducks1"), - new Tuple2<>("california", "ducks2"), - new Tuple2<>("new york", "rangers1"), - new Tuple2<>("new york", "rangers2"), - new Tuple2<>("new york", "islanders1"), - new Tuple2<>("new york", "islanders2"))); - - JavaDStream> stream = JavaTestUtils.attachTestInputStream( - ssc, inputData, 1); - JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - - - JavaPairDStream flatMapped = pairStream.flatMapValues( - new Function>() { - @Override - public Iterable call(String in) { - List out = new ArrayList<>(); - out.add(in + "1"); - out.add(in + "2"); - return out; - } - }); - - JavaTestUtils.attachTestOutputStream(flatMapped); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testCoGroup() { - List>> stringStringKVStream1 = Arrays.asList( - Arrays.asList(new Tuple2<>("california", "dodgers"), - new Tuple2<>("new york", "yankees")), - Arrays.asList(new Tuple2<>("california", "sharks"), - new Tuple2<>("new york", "rangers"))); - - List>> stringStringKVStream2 = Arrays.asList( - Arrays.asList(new Tuple2<>("california", "giants"), - new Tuple2<>("new york", "mets")), - Arrays.asList(new Tuple2<>("california", "ducks"), - new Tuple2<>("new york", "islanders"))); - - - List, List>>>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>("california", - new Tuple2<>(Arrays.asList("dodgers"), Arrays.asList("giants"))), - new Tuple2<>("new york", - new Tuple2<>(Arrays.asList("yankees"), Arrays.asList("mets")))), - Arrays.asList( - new Tuple2<>("california", - new Tuple2<>(Arrays.asList("sharks"), Arrays.asList("ducks"))), - new Tuple2<>("new york", - new Tuple2<>(Arrays.asList("rangers"), Arrays.asList("islanders"))))); - - - JavaDStream> stream1 = JavaTestUtils.attachTestInputStream( - ssc, stringStringKVStream1, 1); - JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream(stream1); - - JavaDStream> stream2 = JavaTestUtils.attachTestInputStream( - ssc, stringStringKVStream2, 1); - JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); - - JavaPairDStream, Iterable>> grouped = pairStream1.cogroup(pairStream2); - JavaTestUtils.attachTestOutputStream(grouped); - List, Iterable>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected.size(), result.size()); - Iterator, Iterable>>>> resultItr = result.iterator(); - Iterator, List>>>> expectedItr = expected.iterator(); - while (resultItr.hasNext() && expectedItr.hasNext()) { - Iterator, Iterable>>> resultElements = resultItr.next().iterator(); - Iterator, List>>> expectedElements = expectedItr.next().iterator(); - while (resultElements.hasNext() && expectedElements.hasNext()) { - Tuple2, Iterable>> resultElement = resultElements.next(); - Tuple2, List>> expectedElement = expectedElements.next(); - Assert.assertEquals(expectedElement._1(), resultElement._1()); - equalIterable(expectedElement._2()._1(), resultElement._2()._1()); - equalIterable(expectedElement._2()._2(), resultElement._2()._2()); - } - Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext()); - } - } - - @SuppressWarnings("unchecked") - @Test - public void testJoin() { - List>> stringStringKVStream1 = Arrays.asList( - Arrays.asList(new Tuple2<>("california", "dodgers"), - new Tuple2<>("new york", "yankees")), - Arrays.asList(new Tuple2<>("california", "sharks"), - new Tuple2<>("new york", "rangers"))); - - List>> stringStringKVStream2 = Arrays.asList( - Arrays.asList(new Tuple2<>("california", "giants"), - new Tuple2<>("new york", "mets")), - Arrays.asList(new Tuple2<>("california", "ducks"), - new Tuple2<>("new york", "islanders"))); - - - List>>> expected = Arrays.asList( - Arrays.asList( - new Tuple2<>("california", - new Tuple2<>("dodgers", "giants")), - new Tuple2<>("new york", - new Tuple2<>("yankees", "mets"))), - Arrays.asList( - new Tuple2<>("california", - new Tuple2<>("sharks", "ducks")), - new Tuple2<>("new york", - new Tuple2<>("rangers", "islanders")))); - - - JavaDStream> stream1 = JavaTestUtils.attachTestInputStream( - ssc, stringStringKVStream1, 1); - JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream(stream1); - - JavaDStream> stream2 = JavaTestUtils.attachTestInputStream( - ssc, stringStringKVStream2, 1); - JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); - - JavaPairDStream> joined = pairStream1.join(pairStream2); - JavaTestUtils.attachTestOutputStream(joined); - List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testLeftOuterJoin() { - List>> stringStringKVStream1 = Arrays.asList( - Arrays.asList(new Tuple2<>("california", "dodgers"), - new Tuple2<>("new york", "yankees")), - Arrays.asList(new Tuple2<>("california", "sharks") )); - - List>> stringStringKVStream2 = Arrays.asList( - Arrays.asList(new Tuple2<>("california", "giants") ), - Arrays.asList(new Tuple2<>("new york", "islanders") ) - - ); - - List> expected = Arrays.asList(Arrays.asList(2L), Arrays.asList(1L)); - - JavaDStream> stream1 = JavaTestUtils.attachTestInputStream( - ssc, stringStringKVStream1, 1); - JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream(stream1); - - JavaDStream> stream2 = JavaTestUtils.attachTestInputStream( - ssc, stringStringKVStream2, 1); - JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); - - JavaPairDStream>> joined = pairStream1.leftOuterJoin(pairStream2); - JavaDStream counted = joined.count(); - JavaTestUtils.attachTestOutputStream(counted); - List> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testCheckpointMasterRecovery() throws InterruptedException { - List> inputData = Arrays.asList( - Arrays.asList("this", "is"), - Arrays.asList("a", "test"), - Arrays.asList("counting", "letters")); - - List> expectedInitial = Arrays.asList( - Arrays.asList(4,2)); - List> expectedFinal = Arrays.asList( - Arrays.asList(1,4), - Arrays.asList(8,7)); - - File tempDir = Files.createTempDir(); - tempDir.deleteOnExit(); - ssc.checkpoint(tempDir.getAbsolutePath()); - - JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream letterCount = stream.map(new Function() { - @Override - public Integer call(String s) { - return s.length(); - } - }); - JavaCheckpointTestUtils.attachTestOutputStream(letterCount); - List> initialResult = JavaTestUtils.runStreams(ssc, 1, 1); - - assertOrderInvariantEquals(expectedInitial, initialResult); - Thread.sleep(1000); - ssc.stop(); - - ssc = new JavaStreamingContext(tempDir.getAbsolutePath()); - // Tweak to take into consideration that the last batch before failure - // will be re-processed after recovery - List> finalResult = JavaCheckpointTestUtils.runStreams(ssc, 2, 3); - assertOrderInvariantEquals(expectedFinal, finalResult.subList(1, 3)); - Utils.deleteRecursively(tempDir); - } - - @SuppressWarnings("unchecked") - @Test - public void testContextGetOrCreate() throws InterruptedException { - ssc.stop(); - - final SparkConf conf = new SparkConf() - .setMaster("local[2]") - .setAppName("test") - .set("newContext", "true"); - - File emptyDir = Files.createTempDir(); - emptyDir.deleteOnExit(); - StreamingContextSuite contextSuite = new StreamingContextSuite(); - String corruptedCheckpointDir = contextSuite.createCorruptedCheckpoint(); - String checkpointDir = contextSuite.createValidCheckpoint(); - - // Function to create JavaStreamingContext without any output operations - // (used to detect the new context) - final AtomicBoolean newContextCreated = new AtomicBoolean(false); - Function0 creatingFunc = new Function0() { - @Override - public JavaStreamingContext call() { - newContextCreated.set(true); - return new JavaStreamingContext(conf, Seconds.apply(1)); - } - }; - - newContextCreated.set(false); - ssc = JavaStreamingContext.getOrCreate(emptyDir.getAbsolutePath(), creatingFunc); - Assert.assertTrue("new context not created", newContextCreated.get()); - ssc.stop(); - - newContextCreated.set(false); - ssc = JavaStreamingContext.getOrCreate(corruptedCheckpointDir, creatingFunc, - new Configuration(), true); - Assert.assertTrue("new context not created", newContextCreated.get()); - ssc.stop(); - - newContextCreated.set(false); - ssc = JavaStreamingContext.getOrCreate(checkpointDir, creatingFunc, - new Configuration()); - Assert.assertTrue("old context not recovered", !newContextCreated.get()); - ssc.stop(); - - newContextCreated.set(false); - JavaSparkContext sc = new JavaSparkContext(conf); - ssc = JavaStreamingContext.getOrCreate(checkpointDir, creatingFunc, - new Configuration()); - Assert.assertTrue("old context not recovered", !newContextCreated.get()); - ssc.stop(); - } - - /* TEST DISABLED: Pending a discussion about checkpoint() semantics with TD - @SuppressWarnings("unchecked") - @Test - public void testCheckpointofIndividualStream() throws InterruptedException { - List> inputData = Arrays.asList( - Arrays.asList("this", "is"), - Arrays.asList("a", "test"), - Arrays.asList("counting", "letters")); - - List> expected = Arrays.asList( - Arrays.asList(4,2), - Arrays.asList(1,4), - Arrays.asList(8,7)); - - JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream letterCount = stream.map(new Function() { - @Override - public Integer call(String s) { - return s.length(); - } - }); - JavaCheckpointTestUtils.attachTestOutputStream(letterCount); - - letterCount.checkpoint(new Duration(1000)); - - List> result1 = JavaCheckpointTestUtils.runStreams(ssc, 3, 3); - assertOrderInvariantEquals(expected, result1); - } - */ - - // Input stream tests. These mostly just test that we can instantiate a given InputStream with - // Java arguments and assign it to a JavaDStream without producing type errors. Testing of the - // InputStream functionality is deferred to the existing Scala tests. - @Test - public void testSocketTextStream() { - ssc.socketTextStream("localhost", 12345); - } - - @Test - public void testSocketString() { - ssc.socketStream( - "localhost", - 12345, - new Function>() { - @Override - public Iterable call(InputStream in) throws IOException { - List out = new ArrayList<>(); - try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) { - for (String line; (line = reader.readLine()) != null;) { - out.add(line); - } - } - return out; - } - }, - StorageLevel.MEMORY_ONLY()); - } - - @SuppressWarnings("unchecked") - @Test - public void testTextFileStream() throws IOException { - File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); - List> expected = fileTestPrepare(testDir); - - JavaDStream input = ssc.textFileStream(testDir.toString()); - JavaTestUtils.attachTestOutputStream(input); - List> result = JavaTestUtils.runStreams(ssc, 1, 1); - - assertOrderInvariantEquals(expected, result); - } - - @SuppressWarnings("unchecked") - @Test - public void testFileStream() throws IOException { - File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); - List> expected = fileTestPrepare(testDir); - - JavaPairInputDStream inputStream = ssc.fileStream( - testDir.toString(), - LongWritable.class, - Text.class, - TextInputFormat.class, - new Function() { - @Override - public Boolean call(Path v1) { - return Boolean.TRUE; - } - }, - true); - - JavaDStream test = inputStream.map( - new Function, String>() { - @Override - public String call(Tuple2 v1) { - return v1._2().toString(); - } - }); - - JavaTestUtils.attachTestOutputStream(test); - List> result = JavaTestUtils.runStreams(ssc, 1, 1); - - assertOrderInvariantEquals(expected, result); - } - - @Test - public void testRawSocketStream() { - ssc.rawSocketStream("localhost", 12345); - } - - private static List> fileTestPrepare(File testDir) throws IOException { - File existingFile = new File(testDir, "0"); - Files.write("0\n", existingFile, Charset.forName("UTF-8")); - Assert.assertTrue(existingFile.setLastModified(1000)); - Assert.assertEquals(1000, existingFile.lastModified()); - return Arrays.asList(Arrays.asList("0")); - } - - @SuppressWarnings("unchecked") - // 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); - } - -} diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java new file mode 100644 index 000000000000..b1367b8f2aed --- /dev/null +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +import scala.Tuple2; + +import com.google.common.collect.Sets; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.util.ManualClock; +import org.junit.Assert; +import org.junit.Test; + +import org.apache.spark.HashPartitioner; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.Optional; +import org.apache.spark.api.java.function.Function3; +import org.apache.spark.api.java.function.Function4; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaMapWithStateDStream; + +public class JavaMapWithStateSuite extends LocalJavaStreamingContext implements Serializable { + + /** + * This test is only for testing the APIs. It's not necessary to run it. + */ + public void testAPI() { + JavaPairRDD initialRDD = null; + JavaPairDStream wordsDstream = null; + + Function4, State, Optional> mappingFunc = + (time, word, one, state) -> { + // Use all State's methods here + state.exists(); + state.get(); + state.isTimingOut(); + state.remove(); + state.update(true); + return Optional.of(2.0); + }; + + JavaMapWithStateDStream stateDstream = + wordsDstream.mapWithState( + StateSpec.function(mappingFunc) + .initialState(initialRDD) + .numPartitions(10) + .partitioner(new HashPartitioner(10)) + .timeout(Durations.seconds(10))); + + stateDstream.stateSnapshots(); + + Function3, State, Double> mappingFunc2 = + (key, one, state) -> { + // Use all State's methods here + state.exists(); + state.get(); + state.isTimingOut(); + state.remove(); + state.update(true); + return 2.0; + }; + + JavaMapWithStateDStream stateDstream2 = + wordsDstream.mapWithState( + StateSpec.function(mappingFunc2) + .initialState(initialRDD) + .numPartitions(10) + .partitioner(new HashPartitioner(10)) + .timeout(Durations.seconds(10))); + + stateDstream2.stateSnapshots(); + } + + @Test + public void testBasicFunction() { + List> inputData = Arrays.asList( + Collections.emptyList(), + Arrays.asList("a"), + Arrays.asList("a", "b"), + Arrays.asList("a", "b", "c"), + Arrays.asList("a", "b"), + Arrays.asList("a"), + Collections.emptyList() + ); + + List> outputData = Arrays.asList( + Collections.emptySet(), + Sets.newHashSet(1), + Sets.newHashSet(2, 1), + Sets.newHashSet(3, 2, 1), + Sets.newHashSet(4, 3), + Sets.newHashSet(5), + Collections.emptySet() + ); + + @SuppressWarnings("unchecked") + List>> stateData = Arrays.asList( + Collections.>emptySet(), + Sets.newHashSet(new Tuple2<>("a", 1)), + Sets.newHashSet(new Tuple2<>("a", 2), new Tuple2<>("b", 1)), + Sets.newHashSet(new Tuple2<>("a", 3), new Tuple2<>("b", 2), new Tuple2<>("c", 1)), + Sets.newHashSet(new Tuple2<>("a", 4), new Tuple2<>("b", 3), new Tuple2<>("c", 1)), + Sets.newHashSet(new Tuple2<>("a", 5), new Tuple2<>("b", 3), new Tuple2<>("c", 1)), + Sets.newHashSet(new Tuple2<>("a", 5), new Tuple2<>("b", 3), new Tuple2<>("c", 1)) + ); + + Function3, State, Integer> mappingFunc = + (key, value, state) -> { + int sum = value.orElse(0) + (state.exists() ? state.get() : 0); + state.update(sum); + return sum; + }; + testOperation( + inputData, + StateSpec.function(mappingFunc), + outputData, + stateData); + } + + private void testOperation( + List> input, + StateSpec mapWithStateSpec, + List> expectedOutputs, + List>> expectedStateSnapshots) { + int numBatches = expectedOutputs.size(); + JavaDStream inputStream = JavaTestUtils.attachTestInputStream(ssc, input, 2); + JavaMapWithStateDStream mapWithStateDStream = JavaPairDStream.fromJavaDStream( + inputStream.map(x -> new Tuple2<>(x, 1))).mapWithState(mapWithStateSpec); + + List> collectedOutputs = + Collections.synchronizedList(new ArrayList>()); + mapWithStateDStream.foreachRDD(rdd -> collectedOutputs.add(Sets.newHashSet(rdd.collect()))); + List>> collectedStateSnapshots = + Collections.synchronizedList(new ArrayList>>()); + mapWithStateDStream.stateSnapshots().foreachRDD(rdd -> + collectedStateSnapshots.add(Sets.newHashSet(rdd.collect()))); + BatchCounter batchCounter = new BatchCounter(ssc.ssc()); + ssc.start(); + ((ManualClock) ssc.ssc().scheduler().clock()) + .advance(ssc.ssc().progressListener().batchDuration() * numBatches + 1); + batchCounter.waitUntilBatchesCompleted(numBatches, 10000); + + Assert.assertEquals(expectedOutputs, collectedOutputs); + Assert.assertEquals(expectedStateSnapshots, collectedStateSnapshots); + } +} diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java index ec2bffd6a5b9..91560472446a 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java @@ -18,12 +18,14 @@ package org.apache.spark.streaming; import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import static org.junit.Assert.*; +import com.google.common.io.Closeables; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -36,6 +38,7 @@ import java.io.Serializable; import java.net.ConnectException; import java.net.Socket; +import java.nio.charset.StandardCharsets; import java.util.concurrent.atomic.AtomicLong; public class JavaReceiverAPISuite implements Serializable { @@ -55,25 +58,16 @@ public void testReceiver() throws InterruptedException { TestServer server = new TestServer(0); server.start(); - final AtomicLong dataCounter = new AtomicLong(0); + AtomicLong dataCounter = new AtomicLong(0); try { JavaStreamingContext ssc = new JavaStreamingContext("local[2]", "test", new Duration(200)); JavaReceiverInputDStream input = ssc.receiverStream(new JavaSocketReceiver("localhost", server.port())); - JavaDStream mapped = input.map(new Function() { - @Override - public String call(String v1) { - return v1 + "."; - } - }); - mapped.foreachRDD(new Function, Void>() { - @Override - public Void call(JavaRDD rdd) { - long count = rdd.count(); - dataCounter.addAndGet(count); - return null; - } + JavaDStream mapped = input.map((Function) v1 -> v1 + "."); + mapped.foreachRDD((VoidFunction>) rdd -> { + long count = rdd.count(); + dataCounter.addAndGet(count); }); ssc.start(); @@ -89,7 +83,7 @@ public Void call(JavaRDD rdd) { Thread.sleep(100); } ssc.stop(); - assertTrue(dataCounter.get() > 0); + Assert.assertTrue(dataCounter.get() > 0); } finally { server.stop(); } @@ -97,8 +91,8 @@ public Void call(JavaRDD rdd) { private static class JavaSocketReceiver extends Receiver { - String host = null; - int port = -1; + private String host = null; + private int port = -1; JavaSocketReceiver(String host_ , int port_) { super(StorageLevel.MEMORY_AND_DISK()); @@ -108,11 +102,7 @@ private static class JavaSocketReceiver extends Receiver { @Override public void onStart() { - new Thread() { - @Override public void run() { - receive(); - } - }.start(); + new Thread(this::receive).start(); } @Override @@ -121,14 +111,20 @@ public void onStop() { private void receive() { try { - Socket socket = new Socket(host, port); - BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream())); - String userInput; - while ((userInput = in.readLine()) != null) { - store(userInput); + Socket socket = null; + BufferedReader in = null; + try { + socket = new Socket(host, port); + in = new BufferedReader( + new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8)); + String userInput; + while ((userInput = in.readLine()) != null) { + store(userInput); + } + } finally { + Closeables.close(in, /* swallowIOException = */ true); + Closeables.close(socket, /* swallowIOException = */ true); } - in.close(); - socket.close(); } catch(ConnectException ce) { ce.printStackTrace(); restart("Could not connect", ce); diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaStreamingListenerAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaStreamingListenerAPISuite.java new file mode 100644 index 000000000000..63fd6c442244 --- /dev/null +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaStreamingListenerAPISuite.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.spark.streaming; + +import org.apache.spark.streaming.api.java.*; + +public class JavaStreamingListenerAPISuite extends JavaStreamingListener { + + @Override + public void onStreamingStarted(JavaStreamingListenerStreamingStarted streamingStarted) { + super.onStreamingStarted(streamingStarted); + } + + @Override + public void onReceiverStarted(JavaStreamingListenerReceiverStarted receiverStarted) { + JavaReceiverInfo receiverInfo = receiverStarted.receiverInfo(); + receiverInfo.streamId(); + receiverInfo.name(); + receiverInfo.active(); + receiverInfo.location(); + receiverInfo.executorId(); + receiverInfo.lastErrorMessage(); + receiverInfo.lastError(); + receiverInfo.lastErrorTime(); + } + + @Override + public void onReceiverError(JavaStreamingListenerReceiverError receiverError) { + JavaReceiverInfo receiverInfo = receiverError.receiverInfo(); + receiverInfo.streamId(); + receiverInfo.name(); + receiverInfo.active(); + receiverInfo.location(); + receiverInfo.executorId(); + receiverInfo.lastErrorMessage(); + receiverInfo.lastError(); + receiverInfo.lastErrorTime(); + } + + @Override + public void onReceiverStopped(JavaStreamingListenerReceiverStopped receiverStopped) { + JavaReceiverInfo receiverInfo = receiverStopped.receiverInfo(); + receiverInfo.streamId(); + receiverInfo.name(); + receiverInfo.active(); + receiverInfo.location(); + receiverInfo.executorId(); + receiverInfo.lastErrorMessage(); + receiverInfo.lastError(); + receiverInfo.lastErrorTime(); + } + + @Override + public void onBatchSubmitted(JavaStreamingListenerBatchSubmitted batchSubmitted) { + super.onBatchSubmitted(batchSubmitted); + } + + @Override + public void onBatchStarted(JavaStreamingListenerBatchStarted batchStarted) { + super.onBatchStarted(batchStarted); + } + + @Override + public void onBatchCompleted(JavaStreamingListenerBatchCompleted batchCompleted) { + super.onBatchCompleted(batchCompleted); + } + + @Override + public void onOutputOperationStarted( + JavaStreamingListenerOutputOperationStarted outputOperationStarted) { + super.onOutputOperationStarted(outputOperationStarted); + } + + @Override + public void onOutputOperationCompleted( + JavaStreamingListenerOutputOperationCompleted outputOperationCompleted) { + super.onOutputOperationCompleted(outputOperationCompleted); + } +} diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala index 57b50bdfd652..0c4a64ccc513 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import scala.reflect.ClassTag import org.apache.spark.api.java.JavaRDDLike -import org.apache.spark.streaming.api.java.{JavaDStreamLike, JavaDStream, JavaStreamingContext} +import org.apache.spark.streaming.api.java.{JavaDStream, JavaDStreamLike, JavaStreamingContext} /** Exposes streaming test functionality in a Java-friendly way. */ trait JavaTestBase extends TestSuiteBase { @@ -35,7 +35,7 @@ trait JavaTestBase extends TestSuiteBase { def attachTestInputStream[T]( ssc: JavaStreamingContext, data: JList[JList[T]], - numPartitions: Int) = { + numPartitions: Int): JavaDStream[T] = { val seqData = data.asScala.map(_.asScala) implicit val cm: ClassTag[T] = @@ -47,9 +47,9 @@ trait JavaTestBase extends TestSuiteBase { /** * Attach a provided stream to it's associated StreamingContext as a * [[org.apache.spark.streaming.TestOutputStream]]. - **/ + */ def attachTestOutputStream[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]]( - dstream: JavaDStreamLike[T, This, R]) = { + dstream: JavaDStreamLike[T, This, R]): Unit = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val ostream = new TestOutputStreamWithPartitions(dstream.dstream) @@ -69,7 +69,7 @@ trait JavaTestBase extends TestSuiteBase { implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] ssc.getState() val res = runStreams[V](ssc.ssc, numBatches, numExpectedOutput) - res.map(_.asJava).asJava + res.map(_.asJava).toSeq.asJava } /** @@ -85,15 +85,15 @@ trait JavaTestBase extends TestSuiteBase { implicit val cm: ClassTag[V] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] val res = runStreamsWithPartitions[V](ssc.ssc, numBatches, numExpectedOutput) - res.map(entry => entry.map(_.asJava).asJava).asJava + res.map(entry => entry.map(_.asJava).asJava).toSeq.asJava } } object JavaTestUtils extends JavaTestBase { - override def maxWaitTimeMillis = 20000 + override def maxWaitTimeMillis: Int = 20000 } object JavaCheckpointTestUtils extends JavaTestBase { - override def actuallyWait = true + override def actuallyWait: Boolean = true } diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaWriteAheadLogSuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaWriteAheadLogSuite.java index 175b8a496b4e..3f4e6ddb216e 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaWriteAheadLogSuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaWriteAheadLogSuite.java @@ -17,16 +17,15 @@ package org.apache.spark.streaming; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Iterator; import java.util.List; -import com.google.common.base.Function; import com.google.common.collect.Iterators; import org.apache.spark.SparkConf; +import org.apache.spark.network.util.JavaUtils; import org.apache.spark.streaming.util.WriteAheadLog; import org.apache.spark.streaming.util.WriteAheadLogRecordHandle; import org.apache.spark.streaming.util.WriteAheadLogUtils; @@ -81,12 +80,7 @@ public ByteBuffer read(WriteAheadLogRecordHandle handle) { @Override public Iterator readAll() { - return Iterators.transform(records.iterator(), new Function() { - @Override - public ByteBuffer apply(Record input) { - return input.buffer; - } - }); + return Iterators.transform(records.iterator(), input -> input.buffer); } @Override @@ -108,23 +102,23 @@ public void close() { public void testCustomWAL() { SparkConf conf = new SparkConf(); conf.set("spark.streaming.driver.writeAheadLog.class", JavaWriteAheadLogSuite.class.getName()); + conf.set("spark.streaming.driver.writeAheadLog.allowBatching", "false"); WriteAheadLog wal = WriteAheadLogUtils.createLogForDriver(conf, null, null); String data1 = "data1"; - WriteAheadLogRecordHandle handle = - wal.write(ByteBuffer.wrap(data1.getBytes(StandardCharsets.UTF_8)), 1234); + WriteAheadLogRecordHandle handle = wal.write(JavaUtils.stringToBytes(data1), 1234); Assert.assertTrue(handle instanceof JavaWriteAheadLogSuiteHandle); - Assert.assertEquals(new String(wal.read(handle).array(), StandardCharsets.UTF_8), data1); + Assert.assertEquals(data1, JavaUtils.bytesToString(wal.read(handle))); - wal.write(ByteBuffer.wrap("data2".getBytes(StandardCharsets.UTF_8)), 1235); - wal.write(ByteBuffer.wrap("data3".getBytes(StandardCharsets.UTF_8)), 1236); - wal.write(ByteBuffer.wrap("data4".getBytes(StandardCharsets.UTF_8)), 1237); + wal.write(JavaUtils.stringToBytes("data2"), 1235); + wal.write(JavaUtils.stringToBytes("data3"), 1236); + wal.write(JavaUtils.stringToBytes("data4"), 1237); wal.clean(1236, false); Iterator dataIterator = wal.readAll(); List readData = new ArrayList<>(); while (dataIterator.hasNext()) { - readData.add(new String(dataIterator.next().array(), StandardCharsets.UTF_8)); + readData.add(JavaUtils.bytesToString(dataIterator.next())); } Assert.assertEquals(readData, Arrays.asList("data3", "data4")); } diff --git a/streaming/src/test/java/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapperSuite.scala b/streaming/src/test/java/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapperSuite.scala new file mode 100644 index 000000000000..cfd4323531bd --- /dev/null +++ b/streaming/src/test/java/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapperSuite.scala @@ -0,0 +1,303 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.java + +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.streaming.Time +import org.apache.spark.streaming.scheduler._ + +class JavaStreamingListenerWrapperSuite extends SparkFunSuite { + + test("basic") { + val listener = new TestJavaStreamingListener() + val listenerWrapper = new JavaStreamingListenerWrapper(listener) + + val streamingStarted = StreamingListenerStreamingStarted(1000L) + listenerWrapper.onStreamingStarted(streamingStarted) + assert(listener.streamingStarted.time === streamingStarted.time) + + val receiverStarted = StreamingListenerReceiverStarted(ReceiverInfo( + streamId = 2, + name = "test", + active = true, + location = "localhost", + executorId = "1" + )) + listenerWrapper.onReceiverStarted(receiverStarted) + assertReceiverInfo(listener.receiverStarted.receiverInfo, receiverStarted.receiverInfo) + + val receiverStopped = StreamingListenerReceiverStopped(ReceiverInfo( + streamId = 2, + name = "test", + active = false, + location = "localhost", + executorId = "1" + )) + listenerWrapper.onReceiverStopped(receiverStopped) + assertReceiverInfo(listener.receiverStopped.receiverInfo, receiverStopped.receiverInfo) + + val receiverError = StreamingListenerReceiverError(ReceiverInfo( + streamId = 2, + name = "test", + active = false, + location = "localhost", + executorId = "1", + lastErrorMessage = "failed", + lastError = "failed", + lastErrorTime = System.currentTimeMillis() + )) + listenerWrapper.onReceiverError(receiverError) + assertReceiverInfo(listener.receiverError.receiverInfo, receiverError.receiverInfo) + + val batchSubmitted = StreamingListenerBatchSubmitted(BatchInfo( + batchTime = Time(1000L), + streamIdToInputInfo = Map( + 0 -> StreamInputInfo( + inputStreamId = 0, + numRecords = 1000, + metadata = Map(StreamInputInfo.METADATA_KEY_DESCRIPTION -> "receiver1")), + 1 -> StreamInputInfo( + inputStreamId = 1, + numRecords = 2000, + metadata = Map(StreamInputInfo.METADATA_KEY_DESCRIPTION -> "receiver2"))), + submissionTime = 1001L, + None, + None, + outputOperationInfos = Map( + 0 -> OutputOperationInfo( + batchTime = Time(1000L), + id = 0, + name = "op1", + description = "operation1", + startTime = None, + endTime = None, + failureReason = None), + 1 -> OutputOperationInfo( + batchTime = Time(1000L), + id = 1, + name = "op2", + description = "operation2", + startTime = None, + endTime = None, + failureReason = None)) + )) + listenerWrapper.onBatchSubmitted(batchSubmitted) + assertBatchInfo(listener.batchSubmitted.batchInfo, batchSubmitted.batchInfo) + + val batchStarted = StreamingListenerBatchStarted(BatchInfo( + batchTime = Time(1000L), + streamIdToInputInfo = Map( + 0 -> StreamInputInfo( + inputStreamId = 0, + numRecords = 1000, + metadata = Map(StreamInputInfo.METADATA_KEY_DESCRIPTION -> "receiver1")), + 1 -> StreamInputInfo( + inputStreamId = 1, + numRecords = 2000, + metadata = Map(StreamInputInfo.METADATA_KEY_DESCRIPTION -> "receiver2"))), + submissionTime = 1001L, + Some(1002L), + None, + outputOperationInfos = Map( + 0 -> OutputOperationInfo( + batchTime = Time(1000L), + id = 0, + name = "op1", + description = "operation1", + startTime = Some(1003L), + endTime = None, + failureReason = None), + 1 -> OutputOperationInfo( + batchTime = Time(1000L), + id = 1, + name = "op2", + description = "operation2", + startTime = Some(1005L), + endTime = None, + failureReason = None)) + )) + listenerWrapper.onBatchStarted(batchStarted) + assertBatchInfo(listener.batchStarted.batchInfo, batchStarted.batchInfo) + + val batchCompleted = StreamingListenerBatchCompleted(BatchInfo( + batchTime = Time(1000L), + streamIdToInputInfo = Map( + 0 -> StreamInputInfo( + inputStreamId = 0, + numRecords = 1000, + metadata = Map(StreamInputInfo.METADATA_KEY_DESCRIPTION -> "receiver1")), + 1 -> StreamInputInfo( + inputStreamId = 1, + numRecords = 2000, + metadata = Map(StreamInputInfo.METADATA_KEY_DESCRIPTION -> "receiver2"))), + submissionTime = 1001L, + Some(1002L), + Some(1010L), + outputOperationInfos = Map( + 0 -> OutputOperationInfo( + batchTime = Time(1000L), + id = 0, + name = "op1", + description = "operation1", + startTime = Some(1003L), + endTime = Some(1004L), + failureReason = None), + 1 -> OutputOperationInfo( + batchTime = Time(1000L), + id = 1, + name = "op2", + description = "operation2", + startTime = Some(1005L), + endTime = Some(1010L), + failureReason = None)) + )) + listenerWrapper.onBatchCompleted(batchCompleted) + assertBatchInfo(listener.batchCompleted.batchInfo, batchCompleted.batchInfo) + + val outputOperationStarted = StreamingListenerOutputOperationStarted(OutputOperationInfo( + batchTime = Time(1000L), + id = 0, + name = "op1", + description = "operation1", + startTime = Some(1003L), + endTime = None, + failureReason = None + )) + listenerWrapper.onOutputOperationStarted(outputOperationStarted) + assertOutputOperationInfo(listener.outputOperationStarted.outputOperationInfo, + outputOperationStarted.outputOperationInfo) + + val outputOperationCompleted = StreamingListenerOutputOperationCompleted(OutputOperationInfo( + batchTime = Time(1000L), + id = 0, + name = "op1", + description = "operation1", + startTime = Some(1003L), + endTime = Some(1004L), + failureReason = None + )) + listenerWrapper.onOutputOperationCompleted(outputOperationCompleted) + assertOutputOperationInfo(listener.outputOperationCompleted.outputOperationInfo, + outputOperationCompleted.outputOperationInfo) + } + + private def assertReceiverInfo( + javaReceiverInfo: JavaReceiverInfo, receiverInfo: ReceiverInfo): Unit = { + assert(javaReceiverInfo.streamId === receiverInfo.streamId) + assert(javaReceiverInfo.name === receiverInfo.name) + assert(javaReceiverInfo.active === receiverInfo.active) + assert(javaReceiverInfo.location === receiverInfo.location) + assert(javaReceiverInfo.executorId === receiverInfo.executorId) + assert(javaReceiverInfo.lastErrorMessage === receiverInfo.lastErrorMessage) + assert(javaReceiverInfo.lastError === receiverInfo.lastError) + assert(javaReceiverInfo.lastErrorTime === receiverInfo.lastErrorTime) + } + + private def assertBatchInfo(javaBatchInfo: JavaBatchInfo, batchInfo: BatchInfo): Unit = { + assert(javaBatchInfo.batchTime === batchInfo.batchTime) + assert(javaBatchInfo.streamIdToInputInfo.size === batchInfo.streamIdToInputInfo.size) + batchInfo.streamIdToInputInfo.foreach { case (streamId, streamInputInfo) => + assertStreamingInfo(javaBatchInfo.streamIdToInputInfo.get(streamId), streamInputInfo) + } + assert(javaBatchInfo.submissionTime === batchInfo.submissionTime) + assert(javaBatchInfo.processingStartTime === batchInfo.processingStartTime.getOrElse(-1)) + assert(javaBatchInfo.processingEndTime === batchInfo.processingEndTime.getOrElse(-1)) + assert(javaBatchInfo.schedulingDelay === batchInfo.schedulingDelay.getOrElse(-1)) + assert(javaBatchInfo.processingDelay === batchInfo.processingDelay.getOrElse(-1)) + assert(javaBatchInfo.totalDelay === batchInfo.totalDelay.getOrElse(-1)) + assert(javaBatchInfo.numRecords === batchInfo.numRecords) + assert(javaBatchInfo.outputOperationInfos.size === batchInfo.outputOperationInfos.size) + batchInfo.outputOperationInfos.foreach { case (outputOperationId, outputOperationInfo) => + assertOutputOperationInfo( + javaBatchInfo.outputOperationInfos.get(outputOperationId), outputOperationInfo) + } + } + + private def assertStreamingInfo( + javaStreamInputInfo: JavaStreamInputInfo, streamInputInfo: StreamInputInfo): Unit = { + assert(javaStreamInputInfo.inputStreamId === streamInputInfo.inputStreamId) + assert(javaStreamInputInfo.numRecords === streamInputInfo.numRecords) + assert(javaStreamInputInfo.metadata === streamInputInfo.metadata.asJava) + assert(javaStreamInputInfo.metadataDescription === streamInputInfo.metadataDescription.orNull) + } + + private def assertOutputOperationInfo( + javaOutputOperationInfo: JavaOutputOperationInfo, + outputOperationInfo: OutputOperationInfo): Unit = { + assert(javaOutputOperationInfo.batchTime === outputOperationInfo.batchTime) + assert(javaOutputOperationInfo.id === outputOperationInfo.id) + assert(javaOutputOperationInfo.name === outputOperationInfo.name) + assert(javaOutputOperationInfo.description === outputOperationInfo.description) + assert(javaOutputOperationInfo.startTime === outputOperationInfo.startTime.getOrElse(-1)) + assert(javaOutputOperationInfo.endTime === outputOperationInfo.endTime.getOrElse(-1)) + assert(javaOutputOperationInfo.failureReason === outputOperationInfo.failureReason.orNull) + } +} + +class TestJavaStreamingListener extends JavaStreamingListener { + + var streamingStarted: JavaStreamingListenerStreamingStarted = null + var receiverStarted: JavaStreamingListenerReceiverStarted = null + var receiverError: JavaStreamingListenerReceiverError = null + var receiverStopped: JavaStreamingListenerReceiverStopped = null + var batchSubmitted: JavaStreamingListenerBatchSubmitted = null + var batchStarted: JavaStreamingListenerBatchStarted = null + var batchCompleted: JavaStreamingListenerBatchCompleted = null + var outputOperationStarted: JavaStreamingListenerOutputOperationStarted = null + var outputOperationCompleted: JavaStreamingListenerOutputOperationCompleted = null + + override def onStreamingStarted(streamingStarted: JavaStreamingListenerStreamingStarted): Unit = { + this.streamingStarted = streamingStarted + } + + override def onReceiverStarted(receiverStarted: JavaStreamingListenerReceiverStarted): Unit = { + this.receiverStarted = receiverStarted + } + + override def onReceiverError(receiverError: JavaStreamingListenerReceiverError): Unit = { + this.receiverError = receiverError + } + + override def onReceiverStopped(receiverStopped: JavaStreamingListenerReceiverStopped): Unit = { + this.receiverStopped = receiverStopped + } + + override def onBatchSubmitted(batchSubmitted: JavaStreamingListenerBatchSubmitted): Unit = { + this.batchSubmitted = batchSubmitted + } + + override def onBatchStarted(batchStarted: JavaStreamingListenerBatchStarted): Unit = { + this.batchStarted = batchStarted + } + + override def onBatchCompleted(batchCompleted: JavaStreamingListenerBatchCompleted): Unit = { + this.batchCompleted = batchCompleted + } + + override def onOutputOperationStarted( + outputOperationStarted: JavaStreamingListenerOutputOperationStarted): Unit = { + this.outputOperationStarted = outputOperationStarted + } + + override def onOutputOperationCompleted( + outputOperationCompleted: JavaStreamingListenerOutputOperationCompleted): Unit = { + this.outputOperationCompleted = outputOperationCompleted + } +} diff --git a/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java b/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java new file mode 100644 index 000000000000..90d1f8c5035b --- /dev/null +++ b/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java @@ -0,0 +1,898 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.streaming; + +import java.io.Serializable; +import java.util.*; + +import org.apache.spark.api.java.function.Function3; +import org.apache.spark.api.java.function.Function4; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.Durations; +import org.apache.spark.streaming.JavaTestUtils; +import org.apache.spark.streaming.LocalJavaStreamingContext; +import org.apache.spark.streaming.State; +import org.apache.spark.streaming.StateSpec; +import org.apache.spark.streaming.Time; +import scala.Tuple2; + +import com.google.common.collect.Sets; +import org.junit.Assert; +import org.junit.Test; + +import org.apache.spark.HashPartitioner; +import org.apache.spark.api.java.Optional; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaMapWithStateDStream; + +/** + * Most of these tests replicate org.apache.spark.streaming.JavaAPISuite using java 8 + * lambda syntax. + */ +@SuppressWarnings("unchecked") +public class Java8APISuite extends LocalJavaStreamingContext implements Serializable { + + @Test + public void testMap() { + List> inputData = Arrays.asList( + Arrays.asList("hello", "world"), + Arrays.asList("goodnight", "moon")); + + List> expected = Arrays.asList( + Arrays.asList(5, 5), + Arrays.asList(9, 4)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream letterCount = stream.map(String::length); + JavaTestUtils.attachTestOutputStream(letterCount); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + assertOrderInvariantEquals(expected, result); + } + + @Test + public void testFilter() { + List> inputData = Arrays.asList( + Arrays.asList("giants", "dodgers"), + Arrays.asList("yankees", "red sox")); + + List> expected = Arrays.asList( + Arrays.asList("giants"), + Arrays.asList("yankees")); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream filtered = stream.filter(s -> s.contains("a")); + JavaTestUtils.attachTestOutputStream(filtered); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + assertOrderInvariantEquals(expected, result); + } + + @Test + public void testMapPartitions() { + List> inputData = Arrays.asList( + Arrays.asList("giants", "dodgers"), + Arrays.asList("yankees", "red sox")); + + List> expected = Arrays.asList( + Arrays.asList("GIANTSDODGERS"), + Arrays.asList("YANKEESRED SOX")); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream mapped = stream.mapPartitions(in -> { + String out = ""; + while (in.hasNext()) { + out = out + in.next().toUpperCase(Locale.ROOT); + } + return Arrays.asList(out).iterator(); + }); + JavaTestUtils.attachTestOutputStream(mapped); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testReduce() { + List> inputData = Arrays.asList( + Arrays.asList(1, 2, 3), + Arrays.asList(4, 5, 6), + Arrays.asList(7, 8, 9)); + + List> expected = Arrays.asList( + Arrays.asList(6), + Arrays.asList(15), + Arrays.asList(24)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream reduced = stream.reduce((x, y) -> x + y); + JavaTestUtils.attachTestOutputStream(reduced); + List> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + @Test + public void testReduceByWindow() { + List> inputData = Arrays.asList( + Arrays.asList(1, 2, 3), + Arrays.asList(4, 5, 6), + Arrays.asList(7, 8, 9)); + + List> expected = Arrays.asList( + Arrays.asList(6), + Arrays.asList(21), + Arrays.asList(39), + Arrays.asList(24)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream reducedWindowed = stream.reduceByWindow( + (x, y) -> x + y, (x, y) -> x - y, new Duration(2000), new Duration(1000)); + JavaTestUtils.attachTestOutputStream(reducedWindowed); + List> result = JavaTestUtils.runStreams(ssc, 4, 4); + + Assert.assertEquals(expected, result); + } + + @Test + public void testTransform() { + List> inputData = Arrays.asList( + Arrays.asList(1, 2, 3), + Arrays.asList(4, 5, 6), + Arrays.asList(7, 8, 9)); + + List> expected = Arrays.asList( + Arrays.asList(3, 4, 5), + Arrays.asList(6, 7, 8), + Arrays.asList(9, 10, 11)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream transformed = stream.transform(in -> in.map(i -> i + 2)); + + JavaTestUtils.attachTestOutputStream(transformed); + List> result = JavaTestUtils.runStreams(ssc, 3, 3); + + assertOrderInvariantEquals(expected, result); + } + + @Test + public void testVariousTransform() { + // tests whether all variations of transform can be called from Java + + List> inputData = Arrays.asList(Arrays.asList(1)); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + + List>> pairInputData = + Arrays.asList(Arrays.asList(new Tuple2<>("x", 1))); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream( + JavaTestUtils.attachTestInputStream(ssc, pairInputData, 1)); + + JavaDStream transformed1 = stream.transform(in -> null); + JavaDStream transformed2 = stream.transform((x, time) -> null); + JavaPairDStream transformed3 = stream.transformToPair(x -> null); + JavaPairDStream transformed4 = stream.transformToPair((x, time) -> null); + JavaDStream pairTransformed1 = pairStream.transform(x -> null); + JavaDStream pairTransformed2 = pairStream.transform((x, time) -> null); + JavaPairDStream pairTransformed3 = pairStream.transformToPair(x -> null); + JavaPairDStream pairTransformed4 = + pairStream.transformToPair((x, time) -> null); + + } + + @Test + public void testTransformWith() { + List>> stringStringKVStream1 = Arrays.asList( + Arrays.asList( + new Tuple2<>("california", "dodgers"), + new Tuple2<>("new york", "yankees")), + Arrays.asList( + new Tuple2<>("california", "sharks"), + new Tuple2<>("new york", "rangers"))); + + List>> stringStringKVStream2 = Arrays.asList( + Arrays.asList( + new Tuple2<>("california", "giants"), + new Tuple2<>("new york", "mets")), + Arrays.asList( + new Tuple2<>("california", "ducks"), + new Tuple2<>("new york", "islanders"))); + + + List>>> expected = Arrays.asList( + Sets.newHashSet( + new Tuple2<>("california", + new Tuple2<>("dodgers", "giants")), + new Tuple2<>("new york", + new Tuple2<>("yankees", "mets"))), + Sets.newHashSet( + new Tuple2<>("california", + new Tuple2<>("sharks", "ducks")), + new Tuple2<>("new york", + new Tuple2<>("rangers", "islanders")))); + + JavaDStream> stream1 = JavaTestUtils.attachTestInputStream( + ssc, stringStringKVStream1, 1); + JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream(stream1); + + JavaDStream> stream2 = JavaTestUtils.attachTestInputStream( + ssc, stringStringKVStream2, 1); + JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); + + JavaPairDStream> joined = + pairStream1.transformWithToPair(pairStream2,(x, y, z) -> x.join(y)); + + JavaTestUtils.attachTestOutputStream(joined); + List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List>>> unorderedResult = new ArrayList<>(); + for (List>> res : result) { + unorderedResult.add(Sets.newHashSet(res)); + } + + Assert.assertEquals(expected, unorderedResult); + } + + + @Test + public void testVariousTransformWith() { + // tests whether all variations of transformWith can be called from Java + + List> inputData1 = Arrays.asList(Arrays.asList(1)); + List> inputData2 = Arrays.asList(Arrays.asList("x")); + JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 1); + JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 1); + + List>> pairInputData1 = + Arrays.asList(Arrays.asList(new Tuple2<>("x", 1))); + List>> pairInputData2 = + Arrays.asList(Arrays.asList(new Tuple2<>(1.0, 'x'))); + JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream( + JavaTestUtils.attachTestInputStream(ssc, pairInputData1, 1)); + JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream( + JavaTestUtils.attachTestInputStream(ssc, pairInputData2, 1)); + + JavaDStream transformed1 = stream1.transformWith(stream2, (x, y, z) -> null); + JavaDStream transformed2 = stream1.transformWith(pairStream1,(x, y, z) -> null); + + JavaPairDStream transformed3 = + stream1.transformWithToPair(stream2,(x, y, z) -> null); + + JavaPairDStream transformed4 = + stream1.transformWithToPair(pairStream1,(x, y, z) -> null); + + JavaDStream pairTransformed1 = pairStream1.transformWith(stream2,(x, y, z) -> null); + + JavaDStream pairTransformed2_ = + pairStream1.transformWith(pairStream1,(x, y, z) -> null); + + JavaPairDStream pairTransformed3 = + pairStream1.transformWithToPair(stream2,(x, y, z) -> null); + + JavaPairDStream pairTransformed4 = + pairStream1.transformWithToPair(pairStream2,(x, y, z) -> null); + } + + @Test + public void testStreamingContextTransform() { + List> stream1input = Arrays.asList( + Arrays.asList(1), + Arrays.asList(2) + ); + + List> stream2input = Arrays.asList( + Arrays.asList(3), + Arrays.asList(4) + ); + + List>> pairStream1input = Arrays.asList( + Arrays.asList(new Tuple2<>(1, "x")), + Arrays.asList(new Tuple2<>(2, "y")) + ); + + List>>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>(1, new Tuple2<>(1, "x"))), + Arrays.asList(new Tuple2<>(2, new Tuple2<>(2, "y"))) + ); + + JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, stream1input, 1); + JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, stream2input, 1); + JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream( + JavaTestUtils.attachTestInputStream(ssc, pairStream1input, 1)); + + List> listOfDStreams1 = Arrays.asList(stream1, stream2); + + // This is just to test whether this transform to JavaStream compiles + JavaDStream transformed1 = ssc.transform( + listOfDStreams1, (List> listOfRDDs, Time time) -> { + Assert.assertEquals(2, listOfRDDs.size()); + return null; + }); + + List> listOfDStreams2 = + Arrays.asList(stream1, stream2, pairStream1.toJavaDStream()); + + JavaPairDStream> transformed2 = ssc.transformToPair( + listOfDStreams2, (List> listOfRDDs, Time time) -> { + Assert.assertEquals(3, listOfRDDs.size()); + JavaRDD rdd1 = (JavaRDD) listOfRDDs.get(0); + JavaRDD rdd2 = (JavaRDD) listOfRDDs.get(1); + JavaRDD> rdd3 = (JavaRDD>) listOfRDDs.get(2); + JavaPairRDD prdd3 = JavaPairRDD.fromJavaRDD(rdd3); + PairFunction mapToTuple = + (Integer i) -> new Tuple2<>(i, i); + return rdd1.union(rdd2).mapToPair(mapToTuple).join(prdd3); + }); + JavaTestUtils.attachTestOutputStream(transformed2); + List>>> result = + JavaTestUtils.runStreams(ssc, 2, 2); + Assert.assertEquals(expected, result); + } + + @Test + public void testFlatMap() { + List> inputData = Arrays.asList( + Arrays.asList("go", "giants"), + Arrays.asList("boo", "dodgers"), + Arrays.asList("athletics")); + + List> expected = Arrays.asList( + Arrays.asList("g", "o", "g", "i", "a", "n", "t", "s"), + Arrays.asList("b", "o", "o", "d", "o", "d", "g", "e", "r", "s"), + Arrays.asList("a", "t", "h", "l", "e", "t", "i", "c", "s")); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream flatMapped = stream.flatMap( + s -> Arrays.asList(s.split("(?!^)")).iterator()); + JavaTestUtils.attachTestOutputStream(flatMapped); + List> result = JavaTestUtils.runStreams(ssc, 3, 3); + + assertOrderInvariantEquals(expected, result); + } + + @Test + public void testPairFlatMap() { + List> inputData = Arrays.asList( + Arrays.asList("giants"), + Arrays.asList("dodgers"), + Arrays.asList("athletics")); + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>(6, "g"), + new Tuple2<>(6, "i"), + new Tuple2<>(6, "a"), + new Tuple2<>(6, "n"), + new Tuple2<>(6, "t"), + new Tuple2<>(6, "s")), + Arrays.asList( + new Tuple2<>(7, "d"), + new Tuple2<>(7, "o"), + new Tuple2<>(7, "d"), + new Tuple2<>(7, "g"), + new Tuple2<>(7, "e"), + new Tuple2<>(7, "r"), + new Tuple2<>(7, "s")), + Arrays.asList( + new Tuple2<>(9, "a"), + new Tuple2<>(9, "t"), + new Tuple2<>(9, "h"), + new Tuple2<>(9, "l"), + new Tuple2<>(9, "e"), + new Tuple2<>(9, "t"), + new Tuple2<>(9, "i"), + new Tuple2<>(9, "c"), + new Tuple2<>(9, "s"))); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream flatMapped = stream.flatMapToPair(s -> { + List> out = new ArrayList<>(); + for (String letter : s.split("(?!^)")) { + out.add(new Tuple2<>(s.length(), letter)); + } + return out.iterator(); + }); + + JavaTestUtils.attachTestOutputStream(flatMapped); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + /* + * Performs an order-invariant comparison of lists representing two RDD streams. This allows + * us to account for ordering variation within individual RDD's which occurs during windowing. + */ + public static > void assertOrderInvariantEquals( + List> expected, List> actual) { + expected.forEach(Collections::sort); + List> sortedActual = new ArrayList<>(); + actual.forEach(list -> { + List sortedList = new ArrayList<>(list); + Collections.sort(sortedList); + sortedActual.add(sortedList); + }); + Assert.assertEquals(expected, sortedActual); + } + + @Test + public void testPairFilter() { + List> inputData = Arrays.asList( + Arrays.asList("giants", "dodgers"), + Arrays.asList("yankees", "red sox")); + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>("giants", 6)), + Arrays.asList(new Tuple2<>("yankees", 7))); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = + stream.mapToPair(x -> new Tuple2<>(x, x.length())); + JavaPairDStream filtered = pairStream.filter(x -> x._1().contains("a")); + JavaTestUtils.attachTestOutputStream(filtered); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + List>> stringStringKVStream = Arrays.asList( + Arrays.asList(new Tuple2<>("california", "dodgers"), + new Tuple2<>("california", "giants"), + new Tuple2<>("new york", "yankees"), + new Tuple2<>("new york", "mets")), + Arrays.asList(new Tuple2<>("california", "sharks"), + new Tuple2<>("california", "ducks"), + new Tuple2<>("new york", "rangers"), + new Tuple2<>("new york", "islanders"))); + + List>> stringIntKVStream = Arrays.asList( + Arrays.asList( + new Tuple2<>("california", 1), + new Tuple2<>("california", 3), + new Tuple2<>("new york", 4), + new Tuple2<>("new york", 1)), + Arrays.asList( + new Tuple2<>("california", 5), + new Tuple2<>("california", 5), + new Tuple2<>("new york", 3), + new Tuple2<>("new york", 1))); + + @Test + public void testPairMap() { // Maps pair -> pair of different type + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>(1, "california"), + new Tuple2<>(3, "california"), + new Tuple2<>(4, "new york"), + new Tuple2<>(1, "new york")), + Arrays.asList( + new Tuple2<>(5, "california"), + new Tuple2<>(5, "california"), + new Tuple2<>(3, "new york"), + new Tuple2<>(1, "new york"))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaPairDStream reversed = pairStream.mapToPair(Tuple2::swap); + JavaTestUtils.attachTestOutputStream(reversed); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testPairMapPartitions() { // Maps pair -> pair of different type + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>(1, "california"), + new Tuple2<>(3, "california"), + new Tuple2<>(4, "new york"), + new Tuple2<>(1, "new york")), + Arrays.asList( + new Tuple2<>(5, "california"), + new Tuple2<>(5, "california"), + new Tuple2<>(3, "new york"), + new Tuple2<>(1, "new york"))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaPairDStream reversed = pairStream.mapPartitionsToPair(in -> { + LinkedList> out = new LinkedList<>(); + while (in.hasNext()) { + Tuple2 next = in.next(); + out.add(next.swap()); + } + return out.iterator(); + }); + + JavaTestUtils.attachTestOutputStream(reversed); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testPairMap2() { // Maps pair -> single + List>> inputData = stringIntKVStream; + + List> expected = Arrays.asList( + Arrays.asList(1, 3, 4, 1), + Arrays.asList(5, 5, 3, 1)); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaDStream reversed = pairStream.map(Tuple2::_2); + JavaTestUtils.attachTestOutputStream(reversed); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair + List>> inputData = Arrays.asList( + Arrays.asList( + new Tuple2<>("hi", 1), + new Tuple2<>("ho", 2)), + Arrays.asList( + new Tuple2<>("hi", 1), + new Tuple2<>("ho", 2))); + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>(1, "h"), + new Tuple2<>(1, "i"), + new Tuple2<>(2, "h"), + new Tuple2<>(2, "o")), + Arrays.asList( + new Tuple2<>(1, "h"), + new Tuple2<>(1, "i"), + new Tuple2<>(2, "h"), + new Tuple2<>(2, "o"))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaPairDStream flatMapped = pairStream.flatMapToPair(in -> { + List> out = new LinkedList<>(); + for (Character s : in._1().toCharArray()) { + out.add(new Tuple2<>(in._2(), s.toString())); + } + return out.iterator(); + }); + + JavaTestUtils.attachTestOutputStream(flatMapped); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testPairReduceByKey() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), + Arrays.asList( + new Tuple2<>("california", 10), + new Tuple2<>("new york", 4))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream reduced = pairStream.reduceByKey((x, y) -> x + y); + + JavaTestUtils.attachTestOutputStream(reduced); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testCombineByKey() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), + Arrays.asList( + new Tuple2<>("california", 10), + new Tuple2<>("new york", 4))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream combined = pairStream.combineByKey(i -> i, + (x, y) -> x + y, (x, y) -> x + y, new HashPartitioner(2)); + + JavaTestUtils.attachTestOutputStream(combined); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testReduceByKeyAndWindow() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), + Arrays.asList(new Tuple2<>("california", 14), + new Tuple2<>("new york", 9)), + Arrays.asList(new Tuple2<>("california", 10), + new Tuple2<>("new york", 4))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream reduceWindowed = + pairStream.reduceByKeyAndWindow((x, y) -> x + y, new Duration(2000), new Duration(1000)); + JavaTestUtils.attachTestOutputStream(reduceWindowed); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + @Test + public void testUpdateStateByKey() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), + Arrays.asList(new Tuple2<>("california", 14), + new Tuple2<>("new york", 9)), + Arrays.asList(new Tuple2<>("california", 14), + new Tuple2<>("new york", 9))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream updated = pairStream.updateStateByKey((values, state) -> { + int out = 0; + if (state.isPresent()) { + out = out + state.get(); + } + for (Integer v : values) { + out = out + v; + } + return Optional.of(out); + }); + + JavaTestUtils.attachTestOutputStream(updated); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + @Test + public void testReduceByKeyAndWindowWithInverse() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), + Arrays.asList(new Tuple2<>("california", 14), + new Tuple2<>("new york", 9)), + Arrays.asList(new Tuple2<>("california", 10), + new Tuple2<>("new york", 4))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream reduceWindowed = + pairStream.reduceByKeyAndWindow((x, y) -> x + y, (x, y) -> x - y, new Duration(2000), + new Duration(1000)); + JavaTestUtils.attachTestOutputStream(reduceWindowed); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + @Test + public void testPairTransform() { + List>> inputData = Arrays.asList( + Arrays.asList( + new Tuple2<>(3, 5), + new Tuple2<>(1, 5), + new Tuple2<>(4, 5), + new Tuple2<>(2, 5)), + Arrays.asList( + new Tuple2<>(2, 5), + new Tuple2<>(3, 5), + new Tuple2<>(4, 5), + new Tuple2<>(1, 5))); + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>(1, 5), + new Tuple2<>(2, 5), + new Tuple2<>(3, 5), + new Tuple2<>(4, 5)), + Arrays.asList( + new Tuple2<>(1, 5), + new Tuple2<>(2, 5), + new Tuple2<>(3, 5), + new Tuple2<>(4, 5))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream sorted = pairStream.transformToPair(in -> in.sortByKey()); + + JavaTestUtils.attachTestOutputStream(sorted); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testPairToNormalRDDTransform() { + List>> inputData = Arrays.asList( + Arrays.asList( + new Tuple2<>(3, 5), + new Tuple2<>(1, 5), + new Tuple2<>(4, 5), + new Tuple2<>(2, 5)), + Arrays.asList( + new Tuple2<>(2, 5), + new Tuple2<>(3, 5), + new Tuple2<>(4, 5), + new Tuple2<>(1, 5))); + + List> expected = Arrays.asList( + Arrays.asList(3, 1, 4, 2), + Arrays.asList(2, 3, 4, 1)); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaDStream firstParts = pairStream.transform(in -> in.map(x -> x._1())); + JavaTestUtils.attachTestOutputStream(firstParts); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testMapValues() { + List>> inputData = stringStringKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>("california", "DODGERS"), + new Tuple2<>("california", "GIANTS"), + new Tuple2<>("new york", "YANKEES"), + new Tuple2<>("new york", "METS")), + Arrays.asList(new Tuple2<>("california", "SHARKS"), + new Tuple2<>("california", "DUCKS"), + new Tuple2<>("new york", "RANGERS"), + new Tuple2<>("new york", "ISLANDERS"))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream mapped = + pairStream.mapValues(s -> s.toUpperCase(Locale.ROOT)); + JavaTestUtils.attachTestOutputStream(mapped); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testFlatMapValues() { + List>> inputData = stringStringKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>("california", "dodgers1"), + new Tuple2<>("california", "dodgers2"), + new Tuple2<>("california", "giants1"), + new Tuple2<>("california", "giants2"), + new Tuple2<>("new york", "yankees1"), + new Tuple2<>("new york", "yankees2"), + new Tuple2<>("new york", "mets1"), + new Tuple2<>("new york", "mets2")), + Arrays.asList(new Tuple2<>("california", "sharks1"), + new Tuple2<>("california", "sharks2"), + new Tuple2<>("california", "ducks1"), + new Tuple2<>("california", "ducks2"), + new Tuple2<>("new york", "rangers1"), + new Tuple2<>("new york", "rangers2"), + new Tuple2<>("new york", "islanders1"), + new Tuple2<>("new york", "islanders2"))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream flatMapped = + pairStream.flatMapValues(in -> Arrays.asList(in + "1", in + "2")); + JavaTestUtils.attachTestOutputStream(flatMapped); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + Assert.assertEquals(expected, result); + } + + /** + * This test is only for testing the APIs. It's not necessary to run it. + */ + public void testMapWithStateAPI() { + JavaPairRDD initialRDD = null; + JavaPairDStream wordsDstream = null; + + Function4, State, Optional> mapFn = + (time, key, value, state) -> { + // Use all State's methods here + state.exists(); + state.get(); + state.isTimingOut(); + state.remove(); + state.update(true); + return Optional.of(2.0); + }; + + JavaMapWithStateDStream stateDstream = + wordsDstream.mapWithState( + StateSpec.function(mapFn) + .initialState(initialRDD) + .numPartitions(10) + .partitioner(new HashPartitioner(10)) + .timeout(Durations.seconds(10))); + + JavaPairDStream emittedRecords = stateDstream.stateSnapshots(); + + Function3, State, Double> mapFn2 = + (key, value, state) -> { + state.exists(); + state.get(); + state.isTimingOut(); + state.remove(); + state.update(true); + return 2.0; + }; + + JavaMapWithStateDStream stateDstream2 = + wordsDstream.mapWithState( + StateSpec.function(mapFn2) + .initialState(initialRDD) + .numPartitions(10) + .partitioner(new HashPartitioner(10)) + .timeout(Durations.seconds(10))); + + JavaPairDStream mappedDStream = stateDstream2.stateSnapshots(); + } +} diff --git a/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java new file mode 100644 index 000000000000..6c86cacec827 --- /dev/null +++ b/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java @@ -0,0 +1,1717 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.streaming; + +import java.io.*; +import java.nio.charset.StandardCharsets; +import java.util.*; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.JavaCheckpointTestUtils; +import org.apache.spark.streaming.JavaTestUtils; +import org.apache.spark.streaming.LocalJavaStreamingContext; +import org.apache.spark.streaming.Seconds; +import org.apache.spark.streaming.StreamingContextState; +import org.apache.spark.streaming.StreamingContextSuite; +import scala.Tuple2; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; + +import org.junit.Assert; +import org.junit.Test; + +import com.google.common.io.Files; +import com.google.common.collect.Sets; + +import org.apache.spark.HashPartitioner; +import org.apache.spark.SparkConf; +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.Optional; +import org.apache.spark.api.java.function.*; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.api.java.*; +import org.apache.spark.util.LongAccumulator; +import org.apache.spark.util.Utils; + +// The test suite itself is Serializable so that anonymous Function implementations can be +// serialized, as an alternative to converting these anonymous classes to static inner classes; +// see http://stackoverflow.com/questions/758570/. +public class JavaAPISuite extends LocalJavaStreamingContext implements Serializable { + + public static void equalIterator(Iterator a, Iterator b) { + while (a.hasNext() && b.hasNext()) { + Assert.assertEquals(a.next(), b.next()); + } + Assert.assertEquals(a.hasNext(), b.hasNext()); + } + + public static void equalIterable(Iterable a, Iterable b) { + equalIterator(a.iterator(), b.iterator()); + } + + @Test + public void testInitialization() { + Assert.assertNotNull(ssc.sparkContext()); + } + + @SuppressWarnings("unchecked") + @Test + public void testContextState() { + List> inputData = Arrays.asList(Arrays.asList(1, 2, 3, 4)); + Assert.assertEquals(StreamingContextState.INITIALIZED, ssc.getState()); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaTestUtils.attachTestOutputStream(stream); + Assert.assertEquals(StreamingContextState.INITIALIZED, ssc.getState()); + ssc.start(); + Assert.assertEquals(StreamingContextState.ACTIVE, ssc.getState()); + ssc.stop(); + Assert.assertEquals(StreamingContextState.STOPPED, ssc.getState()); + } + + @SuppressWarnings("unchecked") + @Test + public void testCount() { + List> inputData = Arrays.asList( + Arrays.asList(1,2,3,4), + Arrays.asList(3,4,5), + Arrays.asList(3)); + + List> expected = Arrays.asList( + Arrays.asList(4L), + Arrays.asList(3L), + Arrays.asList(1L)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream count = stream.count(); + JavaTestUtils.attachTestOutputStream(count); + List> result = JavaTestUtils.runStreams(ssc, 3, 3); + assertOrderInvariantEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testMap() { + List> inputData = Arrays.asList( + Arrays.asList("hello", "world"), + Arrays.asList("goodnight", "moon")); + + List> expected = Arrays.asList( + Arrays.asList(5,5), + Arrays.asList(9,4)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream letterCount = stream.map(String::length); + JavaTestUtils.attachTestOutputStream(letterCount); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + assertOrderInvariantEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testWindow() { + List> inputData = Arrays.asList( + Arrays.asList(1,2,3), + Arrays.asList(4,5,6), + Arrays.asList(7,8,9)); + + List> expected = Arrays.asList( + Arrays.asList(1,2,3), + Arrays.asList(4,5,6,1,2,3), + Arrays.asList(7,8,9,4,5,6), + Arrays.asList(7,8,9)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream windowed = stream.window(new Duration(2000)); + JavaTestUtils.attachTestOutputStream(windowed); + List> result = JavaTestUtils.runStreams(ssc, 4, 4); + + assertOrderInvariantEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testWindowWithSlideDuration() { + List> inputData = Arrays.asList( + Arrays.asList(1,2,3), + Arrays.asList(4,5,6), + Arrays.asList(7,8,9), + Arrays.asList(10,11,12), + Arrays.asList(13,14,15), + Arrays.asList(16,17,18)); + + List> expected = Arrays.asList( + Arrays.asList(1,2,3,4,5,6), + Arrays.asList(1,2,3,4,5,6,7,8,9,10,11,12), + Arrays.asList(7,8,9,10,11,12,13,14,15,16,17,18), + Arrays.asList(13,14,15,16,17,18)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream windowed = stream.window(new Duration(4000), new Duration(2000)); + JavaTestUtils.attachTestOutputStream(windowed); + List> result = JavaTestUtils.runStreams(ssc, 8, 4); + + assertOrderInvariantEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testFilter() { + List> inputData = Arrays.asList( + Arrays.asList("giants", "dodgers"), + Arrays.asList("yankees", "red sox")); + + List> expected = Arrays.asList( + Arrays.asList("giants"), + Arrays.asList("yankees")); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream filtered = stream.filter(s -> s.contains("a")); + JavaTestUtils.attachTestOutputStream(filtered); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + assertOrderInvariantEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testRepartitionMorePartitions() { + List> inputData = Arrays.asList( + Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10), + Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); + JavaDStream stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 2); + JavaDStreamLike,JavaRDD> repartitioned = + stream.repartition(4); + JavaTestUtils.attachTestOutputStream(repartitioned); + List>> result = JavaTestUtils.runStreamsWithPartitions(ssc, 2, 2); + Assert.assertEquals(2, result.size()); + for (List> rdd : result) { + Assert.assertEquals(4, rdd.size()); + Assert.assertEquals( + 10, rdd.get(0).size() + rdd.get(1).size() + rdd.get(2).size() + rdd.get(3).size()); + } + } + + @SuppressWarnings("unchecked") + @Test + public void testRepartitionFewerPartitions() { + List> inputData = Arrays.asList( + Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10), + Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); + JavaDStream stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 4); + JavaDStreamLike,JavaRDD> repartitioned = + stream.repartition(2); + JavaTestUtils.attachTestOutputStream(repartitioned); + List>> result = JavaTestUtils.runStreamsWithPartitions(ssc, 2, 2); + Assert.assertEquals(2, result.size()); + for (List> rdd : result) { + Assert.assertEquals(2, rdd.size()); + Assert.assertEquals(10, rdd.get(0).size() + rdd.get(1).size()); + } + } + + @SuppressWarnings("unchecked") + @Test + public void testGlom() { + List> inputData = Arrays.asList( + Arrays.asList("giants", "dodgers"), + Arrays.asList("yankees", "red sox")); + + List>> expected = Arrays.asList( + Arrays.asList(Arrays.asList("giants", "dodgers")), + Arrays.asList(Arrays.asList("yankees", "red sox"))); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream> glommed = stream.glom(); + JavaTestUtils.attachTestOutputStream(glommed); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testMapPartitions() { + List> inputData = Arrays.asList( + Arrays.asList("giants", "dodgers"), + Arrays.asList("yankees", "red sox")); + + List> expected = Arrays.asList( + Arrays.asList("GIANTSDODGERS"), + Arrays.asList("YANKEESRED SOX")); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream mapped = stream.mapPartitions(in -> { + StringBuilder out = new StringBuilder(); + while (in.hasNext()) { + out.append(in.next().toUpperCase(Locale.ROOT)); + } + return Arrays.asList(out.toString()).iterator(); + }); + JavaTestUtils.attachTestOutputStream(mapped); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + private static class IntegerSum implements Function2 { + @Override + public Integer call(Integer i1, Integer i2) { + return i1 + i2; + } + } + + private static class IntegerDifference implements Function2 { + @Override + public Integer call(Integer i1, Integer i2) { + return i1 - i2; + } + } + + @SuppressWarnings("unchecked") + @Test + public void testReduce() { + List> inputData = Arrays.asList( + Arrays.asList(1,2,3), + Arrays.asList(4,5,6), + Arrays.asList(7,8,9)); + + List> expected = Arrays.asList( + Arrays.asList(6), + Arrays.asList(15), + Arrays.asList(24)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream reduced = stream.reduce(new IntegerSum()); + JavaTestUtils.attachTestOutputStream(reduced); + List> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testReduceByWindowWithInverse() { + testReduceByWindow(true); + } + + @SuppressWarnings("unchecked") + @Test + public void testReduceByWindowWithoutInverse() { + testReduceByWindow(false); + } + + @SuppressWarnings("unchecked") + private void testReduceByWindow(boolean withInverse) { + List> inputData = Arrays.asList( + Arrays.asList(1,2,3), + Arrays.asList(4,5,6), + Arrays.asList(7,8,9)); + + List> expected = Arrays.asList( + Arrays.asList(6), + Arrays.asList(21), + Arrays.asList(39), + Arrays.asList(24)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream reducedWindowed; + if (withInverse) { + reducedWindowed = stream.reduceByWindow(new IntegerSum(), + new IntegerDifference(), + new Duration(2000), + new Duration(1000)); + } else { + reducedWindowed = stream.reduceByWindow(new IntegerSum(), + new Duration(2000), new Duration(1000)); + } + JavaTestUtils.attachTestOutputStream(reducedWindowed); + List> result = JavaTestUtils.runStreams(ssc, 4, 4); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testQueueStream() { + ssc.stop(); + // Create a new JavaStreamingContext without checkpointing + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); + + List> expected = Arrays.asList( + Arrays.asList(1,2,3), + Arrays.asList(4,5,6), + Arrays.asList(7,8,9)); + + JavaSparkContext jsc = new JavaSparkContext(ssc.ssc().sc()); + JavaRDD rdd1 = jsc.parallelize(Arrays.asList(1, 2, 3)); + JavaRDD rdd2 = jsc.parallelize(Arrays.asList(4, 5, 6)); + JavaRDD rdd3 = jsc.parallelize(Arrays.asList(7,8,9)); + + Queue> rdds = new LinkedList<>(); + rdds.add(rdd1); + rdds.add(rdd2); + rdds.add(rdd3); + + JavaDStream stream = ssc.queueStream(rdds); + JavaTestUtils.attachTestOutputStream(stream); + List> result = JavaTestUtils.runStreams(ssc, 3, 3); + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testTransform() { + List> inputData = Arrays.asList( + Arrays.asList(1,2,3), + Arrays.asList(4,5,6), + Arrays.asList(7,8,9)); + + List> expected = Arrays.asList( + Arrays.asList(3,4,5), + Arrays.asList(6,7,8), + Arrays.asList(9,10,11)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream transformed = stream.transform(in -> in.map(i -> i + 2)); + + JavaTestUtils.attachTestOutputStream(transformed); + List> result = JavaTestUtils.runStreams(ssc, 3, 3); + + assertOrderInvariantEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testVariousTransform() { + // tests whether all variations of transform can be called from Java + + List> inputData = Arrays.asList(Arrays.asList(1)); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + + List>> pairInputData = + Arrays.asList(Arrays.asList(new Tuple2<>("x", 1))); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream( + JavaTestUtils.attachTestInputStream(ssc, pairInputData, 1)); + + stream.transform(in -> null); + + stream.transform((in, time) -> null); + + stream.transformToPair(in -> null); + + stream.transformToPair((in, time) -> null); + + pairStream.transform(in -> null); + + pairStream.transform((in, time) -> null); + + pairStream.transformToPair(in -> null); + + pairStream.transformToPair((in, time) -> null); + + } + + @SuppressWarnings("unchecked") + @Test + public void testTransformWith() { + List>> stringStringKVStream1 = Arrays.asList( + Arrays.asList( + new Tuple2<>("california", "dodgers"), + new Tuple2<>("new york", "yankees")), + Arrays.asList( + new Tuple2<>("california", "sharks"), + new Tuple2<>("new york", "rangers"))); + + List>> stringStringKVStream2 = Arrays.asList( + Arrays.asList( + new Tuple2<>("california", "giants"), + new Tuple2<>("new york", "mets")), + Arrays.asList( + new Tuple2<>("california", "ducks"), + new Tuple2<>("new york", "islanders"))); + + + List>>> expected = Arrays.asList( + Sets.newHashSet( + new Tuple2<>("california", + new Tuple2<>("dodgers", "giants")), + new Tuple2<>("new york", + new Tuple2<>("yankees", "mets"))), + Sets.newHashSet( + new Tuple2<>("california", + new Tuple2<>("sharks", "ducks")), + new Tuple2<>("new york", + new Tuple2<>("rangers", "islanders")))); + + JavaDStream> stream1 = JavaTestUtils.attachTestInputStream( + ssc, stringStringKVStream1, 1); + JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream(stream1); + + JavaDStream> stream2 = JavaTestUtils.attachTestInputStream( + ssc, stringStringKVStream2, 1); + JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); + + JavaPairDStream> joined = pairStream1.transformWithToPair( + pairStream2, + (rdd1, rdd2, time) -> rdd1.join(rdd2) + ); + + JavaTestUtils.attachTestOutputStream(joined); + List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List>>> unorderedResult = new ArrayList<>(); + for (List>> res: result) { + unorderedResult.add(Sets.newHashSet(res)); + } + + Assert.assertEquals(expected, unorderedResult); + } + + + @SuppressWarnings("unchecked") + @Test + public void testVariousTransformWith() { + // tests whether all variations of transformWith can be called from Java + + List> inputData1 = Arrays.asList(Arrays.asList(1)); + List> inputData2 = Arrays.asList(Arrays.asList("x")); + JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 1); + JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 1); + + List>> pairInputData1 = + Arrays.asList(Arrays.asList(new Tuple2<>("x", 1))); + List>> pairInputData2 = + Arrays.asList(Arrays.asList(new Tuple2<>(1.0, 'x'))); + JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream( + JavaTestUtils.attachTestInputStream(ssc, pairInputData1, 1)); + JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream( + JavaTestUtils.attachTestInputStream(ssc, pairInputData2, 1)); + + stream1.transformWith(stream2, (rdd1, rdd2, time) -> null); + + stream1.transformWith(pairStream1, (rdd1, rdd2, time) -> null); + + stream1.transformWithToPair(stream2, (rdd1, rdd2, time) -> null); + + stream1.transformWithToPair(pairStream1, (rdd1, rdd2, time) -> null); + + pairStream1.transformWith(stream2, (rdd1, rdd2, time) -> null); + + pairStream1.transformWith(pairStream1, (rdd1, rdd2, time) -> null); + + pairStream1.transformWithToPair(stream2, (rdd1, rdd2, time) -> null); + + pairStream1.transformWithToPair(pairStream2, (rdd1, rdd2, time) -> null); + } + + @SuppressWarnings("unchecked") + @Test + public void testStreamingContextTransform(){ + List> stream1input = Arrays.asList( + Arrays.asList(1), + Arrays.asList(2) + ); + + List> stream2input = Arrays.asList( + Arrays.asList(3), + Arrays.asList(4) + ); + + List>> pairStream1input = Arrays.asList( + Arrays.asList(new Tuple2<>(1, "x")), + Arrays.asList(new Tuple2<>(2, "y")) + ); + + List>>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>(1, new Tuple2<>(1, "x"))), + Arrays.asList(new Tuple2<>(2, new Tuple2<>(2, "y"))) + ); + + JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, stream1input, 1); + JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, stream2input, 1); + JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream( + JavaTestUtils.attachTestInputStream(ssc, pairStream1input, 1)); + + List> listOfDStreams1 = Arrays.asList(stream1, stream2); + + // This is just to test whether this transform to JavaStream compiles + ssc.transform( + listOfDStreams1, + (listOfRDDs, time) -> { + Assert.assertEquals(2, listOfRDDs.size()); + return null; + } + ); + + List> listOfDStreams2 = + Arrays.asList(stream1, stream2, pairStream1.toJavaDStream()); + + JavaPairDStream> transformed2 = ssc.transformToPair( + listOfDStreams2, + (listOfRDDs, time) -> { + Assert.assertEquals(3, listOfRDDs.size()); + JavaRDD rdd1 = (JavaRDD)listOfRDDs.get(0); + JavaRDD rdd2 = (JavaRDD)listOfRDDs.get(1); + JavaRDD> rdd3 = + (JavaRDD>)listOfRDDs.get(2); + JavaPairRDD prdd3 = JavaPairRDD.fromJavaRDD(rdd3); + PairFunction mapToTuple = + (PairFunction) i -> new Tuple2<>(i, i); + return rdd1.union(rdd2).mapToPair(mapToTuple).join(prdd3); + } + ); + JavaTestUtils.attachTestOutputStream(transformed2); + List>>> result = + JavaTestUtils.runStreams(ssc, 2, 2); + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testFlatMap() { + List> inputData = Arrays.asList( + Arrays.asList("go", "giants"), + Arrays.asList("boo", "dodgers"), + Arrays.asList("athletics")); + + List> expected = Arrays.asList( + Arrays.asList("g","o","g","i","a","n","t","s"), + Arrays.asList("b", "o", "o", "d","o","d","g","e","r","s"), + Arrays.asList("a","t","h","l","e","t","i","c","s")); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream flatMapped = + stream.flatMap(x -> Arrays.asList(x.split("(?!^)")).iterator()); + JavaTestUtils.attachTestOutputStream(flatMapped); + List> result = JavaTestUtils.runStreams(ssc, 3, 3); + + assertOrderInvariantEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testForeachRDD() { + final LongAccumulator accumRdd = ssc.sparkContext().sc().longAccumulator(); + final LongAccumulator accumEle = ssc.sparkContext().sc().longAccumulator(); + List> inputData = Arrays.asList( + Arrays.asList(1,1,1), + Arrays.asList(1,1,1)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaTestUtils.attachTestOutputStream(stream.count()); // dummy output + + stream.foreachRDD(rdd -> { + accumRdd.add(1); + rdd.foreach(i -> accumEle.add(1)); + }); + + // This is a test to make sure foreachRDD(VoidFunction2) can be called from Java + stream.foreachRDD((rdd, time) -> {}); + + JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(2, accumRdd.value().intValue()); + Assert.assertEquals(6, accumEle.value().intValue()); + } + + @SuppressWarnings("unchecked") + @Test + public void testPairFlatMap() { + List> inputData = Arrays.asList( + Arrays.asList("giants"), + Arrays.asList("dodgers"), + Arrays.asList("athletics")); + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>(6, "g"), + new Tuple2<>(6, "i"), + new Tuple2<>(6, "a"), + new Tuple2<>(6, "n"), + new Tuple2<>(6, "t"), + new Tuple2<>(6, "s")), + Arrays.asList( + new Tuple2<>(7, "d"), + new Tuple2<>(7, "o"), + new Tuple2<>(7, "d"), + new Tuple2<>(7, "g"), + new Tuple2<>(7, "e"), + new Tuple2<>(7, "r"), + new Tuple2<>(7, "s")), + Arrays.asList( + new Tuple2<>(9, "a"), + new Tuple2<>(9, "t"), + new Tuple2<>(9, "h"), + new Tuple2<>(9, "l"), + new Tuple2<>(9, "e"), + new Tuple2<>(9, "t"), + new Tuple2<>(9, "i"), + new Tuple2<>(9, "c"), + new Tuple2<>(9, "s"))); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream flatMapped = stream.flatMapToPair(in -> { + List> out = new ArrayList<>(); + for (String letter : in.split("(?!^)")) { + out.add(new Tuple2<>(in.length(), letter)); + } + return out.iterator(); + }); + JavaTestUtils.attachTestOutputStream(flatMapped); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testUnion() { + List> inputData1 = Arrays.asList( + Arrays.asList(1,1), + Arrays.asList(2,2), + Arrays.asList(3,3)); + + List> inputData2 = Arrays.asList( + Arrays.asList(4,4), + Arrays.asList(5,5), + Arrays.asList(6,6)); + + List> expected = Arrays.asList( + Arrays.asList(1,1,4,4), + Arrays.asList(2,2,5,5), + Arrays.asList(3,3,6,6)); + + JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2); + JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2); + + JavaDStream unioned = stream1.union(stream2); + JavaTestUtils.attachTestOutputStream(unioned); + List> result = JavaTestUtils.runStreams(ssc, 3, 3); + + assertOrderInvariantEquals(expected, result); + } + + /* + * Performs an order-invariant comparison of lists representing two RDD streams. This allows + * us to account for ordering variation within individual RDD's which occurs during windowing. + */ + public static void assertOrderInvariantEquals( + List> expected, List> actual) { + List> expectedSets = new ArrayList<>(); + for (List list: expected) { + expectedSets.add(Collections.unmodifiableSet(new HashSet<>(list))); + } + List> actualSets = new ArrayList<>(); + for (List list: actual) { + actualSets.add(Collections.unmodifiableSet(new HashSet<>(list))); + } + Assert.assertEquals(expectedSets, actualSets); + } + + + // PairDStream Functions + @SuppressWarnings("unchecked") + @Test + public void testPairFilter() { + List> inputData = Arrays.asList( + Arrays.asList("giants", "dodgers"), + Arrays.asList("yankees", "red sox")); + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>("giants", 6)), + Arrays.asList(new Tuple2<>("yankees", 7))); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = + stream.mapToPair(in -> new Tuple2<>(in, in.length())); + + JavaPairDStream filtered = pairStream.filter(in -> in._1().contains("a")); + JavaTestUtils.attachTestOutputStream(filtered); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + private final List>> stringStringKVStream = Arrays.asList( + Arrays.asList(new Tuple2<>("california", "dodgers"), + new Tuple2<>("california", "giants"), + new Tuple2<>("new york", "yankees"), + new Tuple2<>("new york", "mets")), + Arrays.asList(new Tuple2<>("california", "sharks"), + new Tuple2<>("california", "ducks"), + new Tuple2<>("new york", "rangers"), + new Tuple2<>("new york", "islanders"))); + + @SuppressWarnings("unchecked") + private final List>> stringIntKVStream = Arrays.asList( + Arrays.asList( + new Tuple2<>("california", 1), + new Tuple2<>("california", 3), + new Tuple2<>("new york", 4), + new Tuple2<>("new york", 1)), + Arrays.asList( + new Tuple2<>("california", 5), + new Tuple2<>("california", 5), + new Tuple2<>("new york", 3), + new Tuple2<>("new york", 1))); + + @SuppressWarnings("unchecked") + @Test + public void testPairMap() { // Maps pair -> pair of different type + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>(1, "california"), + new Tuple2<>(3, "california"), + new Tuple2<>(4, "new york"), + new Tuple2<>(1, "new york")), + Arrays.asList( + new Tuple2<>(5, "california"), + new Tuple2<>(5, "california"), + new Tuple2<>(3, "new york"), + new Tuple2<>(1, "new york"))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaPairDStream reversed = pairStream.mapToPair(Tuple2::swap); + + JavaTestUtils.attachTestOutputStream(reversed); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testPairMapPartitions() { // Maps pair -> pair of different type + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>(1, "california"), + new Tuple2<>(3, "california"), + new Tuple2<>(4, "new york"), + new Tuple2<>(1, "new york")), + Arrays.asList( + new Tuple2<>(5, "california"), + new Tuple2<>(5, "california"), + new Tuple2<>(3, "new york"), + new Tuple2<>(1, "new york"))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaPairDStream reversed = pairStream.mapPartitionsToPair(in -> { + List> out = new LinkedList<>(); + while (in.hasNext()) { + Tuple2 next = in.next(); + out.add(next.swap()); + } + return out.iterator(); + }); + + JavaTestUtils.attachTestOutputStream(reversed); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testPairMap2() { // Maps pair -> single + List>> inputData = stringIntKVStream; + + List> expected = Arrays.asList( + Arrays.asList(1, 3, 4, 1), + Arrays.asList(5, 5, 3, 1)); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaDStream reversed = pairStream.map(in -> in._2()); + + JavaTestUtils.attachTestOutputStream(reversed); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair + List>> inputData = Arrays.asList( + Arrays.asList( + new Tuple2<>("hi", 1), + new Tuple2<>("ho", 2)), + Arrays.asList( + new Tuple2<>("hi", 1), + new Tuple2<>("ho", 2))); + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>(1, "h"), + new Tuple2<>(1, "i"), + new Tuple2<>(2, "h"), + new Tuple2<>(2, "o")), + Arrays.asList( + new Tuple2<>(1, "h"), + new Tuple2<>(1, "i"), + new Tuple2<>(2, "h"), + new Tuple2<>(2, "o"))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaPairDStream flatMapped = pairStream.flatMapToPair(in -> { + List> out = new LinkedList<>(); + for (Character s : in._1().toCharArray()) { + out.add(new Tuple2<>(in._2(), s.toString())); + } + return out.iterator(); + }); + JavaTestUtils.attachTestOutputStream(flatMapped); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testPairGroupByKey() { + List>> inputData = stringStringKVStream; + + List>>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>("california", Arrays.asList("dodgers", "giants")), + new Tuple2<>("new york", Arrays.asList("yankees", "mets"))), + Arrays.asList( + new Tuple2<>("california", Arrays.asList("sharks", "ducks")), + new Tuple2<>("new york", Arrays.asList("rangers", "islanders")))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream> grouped = pairStream.groupByKey(); + JavaTestUtils.attachTestOutputStream(grouped); + List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected.size(), result.size()); + Iterator>>> resultItr = result.iterator(); + Iterator>>> expectedItr = expected.iterator(); + while (resultItr.hasNext() && expectedItr.hasNext()) { + Iterator>> resultElements = resultItr.next().iterator(); + Iterator>> expectedElements = expectedItr.next().iterator(); + while (resultElements.hasNext() && expectedElements.hasNext()) { + Tuple2> resultElement = resultElements.next(); + Tuple2> expectedElement = expectedElements.next(); + Assert.assertEquals(expectedElement._1(), resultElement._1()); + equalIterable(expectedElement._2(), resultElement._2()); + } + Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext()); + } + } + + @SuppressWarnings("unchecked") + @Test + public void testPairReduceByKey() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), + Arrays.asList( + new Tuple2<>("california", 10), + new Tuple2<>("new york", 4))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream reduced = pairStream.reduceByKey(new IntegerSum()); + + JavaTestUtils.attachTestOutputStream(reduced); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testCombineByKey() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), + Arrays.asList( + new Tuple2<>("california", 10), + new Tuple2<>("new york", 4))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream combined = pairStream.combineByKey( + i -> i, new IntegerSum(), new IntegerSum(), new HashPartitioner(2)); + + JavaTestUtils.attachTestOutputStream(combined); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testCountByValue() { + List> inputData = Arrays.asList( + Arrays.asList("hello", "world"), + Arrays.asList("hello", "moon"), + Arrays.asList("hello")); + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>("hello", 1L), + new Tuple2<>("world", 1L)), + Arrays.asList( + new Tuple2<>("hello", 1L), + new Tuple2<>("moon", 1L)), + Arrays.asList( + new Tuple2<>("hello", 1L))); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream counted = stream.countByValue(); + JavaTestUtils.attachTestOutputStream(counted); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testGroupByKeyAndWindow() { + List>> inputData = stringIntKVStream; + + List>>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>("california", Arrays.asList(1, 3)), + new Tuple2<>("new york", Arrays.asList(1, 4)) + ), + Arrays.asList( + new Tuple2<>("california", Arrays.asList(1, 3, 5, 5)), + new Tuple2<>("new york", Arrays.asList(1, 1, 3, 4)) + ), + Arrays.asList( + new Tuple2<>("california", Arrays.asList(5, 5)), + new Tuple2<>("new york", Arrays.asList(1, 3)) + ) + ); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream> groupWindowed = + pairStream.groupByKeyAndWindow(new Duration(2000), new Duration(1000)); + JavaTestUtils.attachTestOutputStream(groupWindowed); + List>>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected.size(), result.size()); + for (int i = 0; i < result.size(); i++) { + Assert.assertEquals(convert(expected.get(i)), convert(result.get(i))); + } + } + + private static Set>> + convert(List>> listOfTuples) { + List>> newListOfTuples = new ArrayList<>(); + for (Tuple2> tuple: listOfTuples) { + newListOfTuples.add(convert(tuple)); + } + return new HashSet<>(newListOfTuples); + } + + private static Tuple2> convert(Tuple2> tuple) { + return new Tuple2<>(tuple._1(), new HashSet<>(tuple._2())); + } + + @SuppressWarnings("unchecked") + @Test + public void testReduceByKeyAndWindow() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), + Arrays.asList(new Tuple2<>("california", 14), + new Tuple2<>("new york", 9)), + Arrays.asList(new Tuple2<>("california", 10), + new Tuple2<>("new york", 4))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream reduceWindowed = + pairStream.reduceByKeyAndWindow(new IntegerSum(), new Duration(2000), new Duration(1000)); + JavaTestUtils.attachTestOutputStream(reduceWindowed); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testUpdateStateByKey() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), + Arrays.asList(new Tuple2<>("california", 14), + new Tuple2<>("new york", 9)), + Arrays.asList(new Tuple2<>("california", 14), + new Tuple2<>("new york", 9))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream updated = pairStream.updateStateByKey((values, state) -> { + int out = 0; + if (state.isPresent()) { + out += state.get(); + } + for (Integer v : values) { + out += v; + } + return Optional.of(out); + }); + JavaTestUtils.attachTestOutputStream(updated); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testUpdateStateByKeyWithInitial() { + List>> inputData = stringIntKVStream; + + List> initial = Arrays.asList( + new Tuple2<>("california", 1), + new Tuple2<>("new york", 2)); + + JavaRDD> tmpRDD = ssc.sparkContext().parallelize(initial); + JavaPairRDD initialRDD = JavaPairRDD.fromJavaRDD(tmpRDD); + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>("california", 5), + new Tuple2<>("new york", 7)), + Arrays.asList(new Tuple2<>("california", 15), + new Tuple2<>("new york", 11)), + Arrays.asList(new Tuple2<>("california", 15), + new Tuple2<>("new york", 11))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream updated = pairStream.updateStateByKey((values, state) -> { + int out = 0; + if (state.isPresent()) { + out += state.get(); + } + for (Integer v : values) { + out += v; + } + return Optional.of(out); + }, new HashPartitioner(1), initialRDD); + JavaTestUtils.attachTestOutputStream(updated); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + assertOrderInvariantEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testReduceByKeyAndWindowWithInverse() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>("california", 4), + new Tuple2<>("new york", 5)), + Arrays.asList(new Tuple2<>("california", 14), + new Tuple2<>("new york", 9)), + Arrays.asList(new Tuple2<>("california", 10), + new Tuple2<>("new york", 4))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream reduceWindowed = + pairStream.reduceByKeyAndWindow(new IntegerSum(), new IntegerDifference(), + new Duration(2000), new Duration(1000)); + JavaTestUtils.attachTestOutputStream(reduceWindowed); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testCountByValueAndWindow() { + List> inputData = Arrays.asList( + Arrays.asList("hello", "world"), + Arrays.asList("hello", "moon"), + Arrays.asList("hello")); + + List>> expected = Arrays.asList( + Sets.newHashSet( + new Tuple2<>("hello", 1L), + new Tuple2<>("world", 1L)), + Sets.newHashSet( + new Tuple2<>("hello", 2L), + new Tuple2<>("world", 1L), + new Tuple2<>("moon", 1L)), + Sets.newHashSet( + new Tuple2<>("hello", 2L), + new Tuple2<>("moon", 1L))); + + JavaDStream stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream counted = + stream.countByValueAndWindow(new Duration(2000), new Duration(1000)); + JavaTestUtils.attachTestOutputStream(counted); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + List>> unorderedResult = new ArrayList<>(); + for (List> res: result) { + unorderedResult.add(Sets.newHashSet(res)); + } + + Assert.assertEquals(expected, unorderedResult); + } + + @SuppressWarnings("unchecked") + @Test + public void testPairTransform() { + List>> inputData = Arrays.asList( + Arrays.asList( + new Tuple2<>(3, 5), + new Tuple2<>(1, 5), + new Tuple2<>(4, 5), + new Tuple2<>(2, 5)), + Arrays.asList( + new Tuple2<>(2, 5), + new Tuple2<>(3, 5), + new Tuple2<>(4, 5), + new Tuple2<>(1, 5))); + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>(1, 5), + new Tuple2<>(2, 5), + new Tuple2<>(3, 5), + new Tuple2<>(4, 5)), + Arrays.asList( + new Tuple2<>(1, 5), + new Tuple2<>(2, 5), + new Tuple2<>(3, 5), + new Tuple2<>(4, 5))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream sorted = pairStream.transformToPair(in -> in.sortByKey()); + + JavaTestUtils.attachTestOutputStream(sorted); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testPairToNormalRDDTransform() { + List>> inputData = Arrays.asList( + Arrays.asList( + new Tuple2<>(3, 5), + new Tuple2<>(1, 5), + new Tuple2<>(4, 5), + new Tuple2<>(2, 5)), + Arrays.asList( + new Tuple2<>(2, 5), + new Tuple2<>(3, 5), + new Tuple2<>(4, 5), + new Tuple2<>(1, 5))); + + List> expected = Arrays.asList( + Arrays.asList(3,1,4,2), + Arrays.asList(2,3,4,1)); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaDStream firstParts = pairStream.transform(in -> in.map(in2 -> in2._1())); + + JavaTestUtils.attachTestOutputStream(firstParts); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testMapValues() { + List>> inputData = stringStringKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>("california", "DODGERS"), + new Tuple2<>("california", "GIANTS"), + new Tuple2<>("new york", "YANKEES"), + new Tuple2<>("new york", "METS")), + Arrays.asList(new Tuple2<>("california", "SHARKS"), + new Tuple2<>("california", "DUCKS"), + new Tuple2<>("new york", "RANGERS"), + new Tuple2<>("new york", "ISLANDERS"))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream mapped = + pairStream.mapValues(s -> s.toUpperCase(Locale.ROOT)); + + JavaTestUtils.attachTestOutputStream(mapped); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testFlatMapValues() { + List>> inputData = stringStringKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2<>("california", "dodgers1"), + new Tuple2<>("california", "dodgers2"), + new Tuple2<>("california", "giants1"), + new Tuple2<>("california", "giants2"), + new Tuple2<>("new york", "yankees1"), + new Tuple2<>("new york", "yankees2"), + new Tuple2<>("new york", "mets1"), + new Tuple2<>("new york", "mets2")), + Arrays.asList(new Tuple2<>("california", "sharks1"), + new Tuple2<>("california", "sharks2"), + new Tuple2<>("california", "ducks1"), + new Tuple2<>("california", "ducks2"), + new Tuple2<>("new york", "rangers1"), + new Tuple2<>("new york", "rangers2"), + new Tuple2<>("new york", "islanders1"), + new Tuple2<>("new york", "islanders2"))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + + JavaPairDStream flatMapped = pairStream.flatMapValues(in -> { + List out = new ArrayList<>(); + out.add(in + "1"); + out.add(in + "2"); + return out; + }); + + JavaTestUtils.attachTestOutputStream(flatMapped); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testCoGroup() { + List>> stringStringKVStream1 = Arrays.asList( + Arrays.asList(new Tuple2<>("california", "dodgers"), + new Tuple2<>("new york", "yankees")), + Arrays.asList(new Tuple2<>("california", "sharks"), + new Tuple2<>("new york", "rangers"))); + + List>> stringStringKVStream2 = Arrays.asList( + Arrays.asList(new Tuple2<>("california", "giants"), + new Tuple2<>("new york", "mets")), + Arrays.asList(new Tuple2<>("california", "ducks"), + new Tuple2<>("new york", "islanders"))); + + + List, List>>>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>("california", + new Tuple2<>(Arrays.asList("dodgers"), Arrays.asList("giants"))), + new Tuple2<>("new york", + new Tuple2<>(Arrays.asList("yankees"), Arrays.asList("mets")))), + Arrays.asList( + new Tuple2<>("california", + new Tuple2<>(Arrays.asList("sharks"), Arrays.asList("ducks"))), + new Tuple2<>("new york", + new Tuple2<>(Arrays.asList("rangers"), Arrays.asList("islanders"))))); + + + JavaDStream> stream1 = JavaTestUtils.attachTestInputStream( + ssc, stringStringKVStream1, 1); + JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream(stream1); + + JavaDStream> stream2 = JavaTestUtils.attachTestInputStream( + ssc, stringStringKVStream2, 1); + JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); + + JavaPairDStream, Iterable>> grouped = + pairStream1.cogroup(pairStream2); + JavaTestUtils.attachTestOutputStream(grouped); + List, Iterable>>>> result = + JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected.size(), result.size()); + Iterator, Iterable>>>> resultItr = + result.iterator(); + Iterator, List>>>> expectedItr = + expected.iterator(); + while (resultItr.hasNext() && expectedItr.hasNext()) { + Iterator, Iterable>>> resultElements = + resultItr.next().iterator(); + Iterator, List>>> expectedElements = + expectedItr.next().iterator(); + while (resultElements.hasNext() && expectedElements.hasNext()) { + Tuple2, Iterable>> resultElement = + resultElements.next(); + Tuple2, List>> expectedElement = + expectedElements.next(); + Assert.assertEquals(expectedElement._1(), resultElement._1()); + equalIterable(expectedElement._2()._1(), resultElement._2()._1()); + equalIterable(expectedElement._2()._2(), resultElement._2()._2()); + } + Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext()); + } + } + + @SuppressWarnings("unchecked") + @Test + public void testJoin() { + List>> stringStringKVStream1 = Arrays.asList( + Arrays.asList(new Tuple2<>("california", "dodgers"), + new Tuple2<>("new york", "yankees")), + Arrays.asList(new Tuple2<>("california", "sharks"), + new Tuple2<>("new york", "rangers"))); + + List>> stringStringKVStream2 = Arrays.asList( + Arrays.asList(new Tuple2<>("california", "giants"), + new Tuple2<>("new york", "mets")), + Arrays.asList(new Tuple2<>("california", "ducks"), + new Tuple2<>("new york", "islanders"))); + + + List>>> expected = Arrays.asList( + Arrays.asList( + new Tuple2<>("california", + new Tuple2<>("dodgers", "giants")), + new Tuple2<>("new york", + new Tuple2<>("yankees", "mets"))), + Arrays.asList( + new Tuple2<>("california", + new Tuple2<>("sharks", "ducks")), + new Tuple2<>("new york", + new Tuple2<>("rangers", "islanders")))); + + + JavaDStream> stream1 = JavaTestUtils.attachTestInputStream( + ssc, stringStringKVStream1, 1); + JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream(stream1); + + JavaDStream> stream2 = JavaTestUtils.attachTestInputStream( + ssc, stringStringKVStream2, 1); + JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); + + JavaPairDStream> joined = pairStream1.join(pairStream2); + JavaTestUtils.attachTestOutputStream(joined); + List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testLeftOuterJoin() { + List>> stringStringKVStream1 = Arrays.asList( + Arrays.asList(new Tuple2<>("california", "dodgers"), + new Tuple2<>("new york", "yankees")), + Arrays.asList(new Tuple2<>("california", "sharks") )); + + List>> stringStringKVStream2 = Arrays.asList( + Arrays.asList(new Tuple2<>("california", "giants") ), + Arrays.asList(new Tuple2<>("new york", "islanders") ) + + ); + + List> expected = Arrays.asList(Arrays.asList(2L), Arrays.asList(1L)); + + JavaDStream> stream1 = JavaTestUtils.attachTestInputStream( + ssc, stringStringKVStream1, 1); + JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream(stream1); + + JavaDStream> stream2 = JavaTestUtils.attachTestInputStream( + ssc, stringStringKVStream2, 1); + JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); + + JavaPairDStream>> joined = + pairStream1.leftOuterJoin(pairStream2); + JavaDStream counted = joined.count(); + JavaTestUtils.attachTestOutputStream(counted); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testCheckpointMasterRecovery() throws InterruptedException { + List> inputData = Arrays.asList( + Arrays.asList("this", "is"), + Arrays.asList("a", "test"), + Arrays.asList("counting", "letters")); + + List> expectedInitial = Arrays.asList( + Arrays.asList(4,2)); + List> expectedFinal = Arrays.asList( + Arrays.asList(1,4), + Arrays.asList(8,7)); + + File tempDir = Files.createTempDir(); + tempDir.deleteOnExit(); + ssc.checkpoint(tempDir.getAbsolutePath()); + + JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream letterCount = stream.map(String::length); + JavaCheckpointTestUtils.attachTestOutputStream(letterCount); + List> initialResult = JavaTestUtils.runStreams(ssc, 1, 1); + + assertOrderInvariantEquals(expectedInitial, initialResult); + Thread.sleep(1000); + ssc.stop(); + + ssc = new JavaStreamingContext(tempDir.getAbsolutePath()); + // Tweak to take into consideration that the last batch before failure + // will be re-processed after recovery + List> finalResult = JavaCheckpointTestUtils.runStreams(ssc, 2, 3); + assertOrderInvariantEquals(expectedFinal, finalResult.subList(1, 3)); + ssc.stop(); + Utils.deleteRecursively(tempDir); + } + + @SuppressWarnings("unchecked") + @Test + public void testContextGetOrCreate() throws InterruptedException { + ssc.stop(); + + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("newContext", "true"); + + File emptyDir = Files.createTempDir(); + emptyDir.deleteOnExit(); + StreamingContextSuite contextSuite = new StreamingContextSuite(); + String corruptedCheckpointDir = contextSuite.createCorruptedCheckpoint(); + String checkpointDir = contextSuite.createValidCheckpoint(); + + // Function to create JavaStreamingContext without any output operations + // (used to detect the new context) + AtomicBoolean newContextCreated = new AtomicBoolean(false); + Function0 creatingFunc = () -> { + newContextCreated.set(true); + return new JavaStreamingContext(conf, Seconds.apply(1)); + }; + + newContextCreated.set(false); + ssc = JavaStreamingContext.getOrCreate(emptyDir.getAbsolutePath(), creatingFunc); + Assert.assertTrue("new context not created", newContextCreated.get()); + ssc.stop(); + + newContextCreated.set(false); + ssc = JavaStreamingContext.getOrCreate(corruptedCheckpointDir, creatingFunc, + new Configuration(), true); + Assert.assertTrue("new context not created", newContextCreated.get()); + ssc.stop(); + + newContextCreated.set(false); + ssc = JavaStreamingContext.getOrCreate(checkpointDir, creatingFunc, + new Configuration()); + Assert.assertTrue("old context not recovered", !newContextCreated.get()); + ssc.stop(); + + newContextCreated.set(false); + JavaSparkContext sc = new JavaSparkContext(conf); + ssc = JavaStreamingContext.getOrCreate(checkpointDir, creatingFunc, + new Configuration()); + Assert.assertTrue("old context not recovered", !newContextCreated.get()); + ssc.stop(); + } + + /* TEST DISABLED: Pending a discussion about checkpoint() semantics with TD + @SuppressWarnings("unchecked") + @Test + public void testCheckpointofIndividualStream() throws InterruptedException { + List> inputData = Arrays.asList( + Arrays.asList("this", "is"), + Arrays.asList("a", "test"), + Arrays.asList("counting", "letters")); + + List> expected = Arrays.asList( + Arrays.asList(4,2), + Arrays.asList(1,4), + Arrays.asList(8,7)); + + JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream letterCount = stream.map(new Function() { + @Override + public Integer call(String s) { + return s.length(); + } + }); + JavaCheckpointTestUtils.attachTestOutputStream(letterCount); + + letterCount.checkpoint(new Duration(1000)); + + List> result1 = JavaCheckpointTestUtils.runStreams(ssc, 3, 3); + assertOrderInvariantEquals(expected, result1); + } + */ + + // Input stream tests. These mostly just test that we can instantiate a given InputStream with + // Java arguments and assign it to a JavaDStream without producing type errors. Testing of the + // InputStream functionality is deferred to the existing Scala tests. + @Test + public void testSocketTextStream() { + ssc.socketTextStream("localhost", 12345); + } + + @Test + public void testSocketString() { + ssc.socketStream( + "localhost", + 12345, + in -> { + List out = new ArrayList<>(); + try (BufferedReader reader = new BufferedReader( + new InputStreamReader(in, StandardCharsets.UTF_8))) { + for (String line; (line = reader.readLine()) != null;) { + out.add(line); + } + } + return out; + }, + StorageLevel.MEMORY_ONLY()); + } + + @SuppressWarnings("unchecked") + @Test + public void testTextFileStream() throws IOException { + File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); + List> expected = fileTestPrepare(testDir); + + JavaDStream input = ssc.textFileStream(testDir.toString()); + JavaTestUtils.attachTestOutputStream(input); + List> result = JavaTestUtils.runStreams(ssc, 1, 1); + + assertOrderInvariantEquals(expected, result); + } + + @SuppressWarnings("unchecked") + @Test + public void testFileStream() throws IOException { + File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); + List> expected = fileTestPrepare(testDir); + + JavaPairInputDStream inputStream = ssc.fileStream( + testDir.toString(), + LongWritable.class, + Text.class, + TextInputFormat.class, + v1 -> Boolean.TRUE, + true); + + JavaDStream test = inputStream.map(v1 -> v1._2().toString()); + + JavaTestUtils.attachTestOutputStream(test); + List> result = JavaTestUtils.runStreams(ssc, 1, 1); + + assertOrderInvariantEquals(expected, result); + } + + @Test + public void testRawSocketStream() { + ssc.rawSocketStream("localhost", 12345); + } + + private static List> fileTestPrepare(File testDir) throws IOException { + File existingFile = new File(testDir, "0"); + Files.write("0\n", existingFile, StandardCharsets.UTF_8); + Assert.assertTrue(existingFile.setLastModified(1000)); + Assert.assertEquals(1000, existingFile.lastModified()); + return Arrays.asList(Arrays.asList("0")); + } + + @SuppressWarnings("unchecked") + // 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); + } + +} diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 75e3b53a093f..fd51f8faf56b 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN +log4j.logger.org.spark_project.jetty=WARN diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 9d296c6d3ef8..6f62c7a88dc3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -17,18 +17,19 @@ package org.apache.spark.streaming +import java.util.concurrent.ConcurrentLinkedQueue + import scala.collection.mutable -import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import scala.language.existentials import scala.reflect.ClassTag -import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.SparkContext._ +import org.scalatest.concurrent.Eventually.eventually + +import org.apache.spark.{HashPartitioner, SparkConf, SparkException} import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.{DStream, WindowedDStream} import org.apache.spark.util.{Clock, ManualClock} -import org.apache.spark.HashPartitioner class BasicOperationsSuite extends TestSuiteBase { test("map") { @@ -74,7 +75,7 @@ class BasicOperationsSuite extends TestSuiteBase { assert(numInputPartitions === 2, "Number of input partitions has been changed from 2") val input = Seq(1 to 4, 5 to 8, 9 to 12) val output = Seq(Seq(3, 7), Seq(11, 15), Seq(19, 23)) - val operation = (r: DStream[Int]) => r.mapPartitions(x => Iterator(x.reduce(_ + _))) + val operation = (r: DStream[Int]) => r.mapPartitions(x => Iterator(x.sum)) testOperation(input, operation, output, true) } @@ -84,9 +85,10 @@ class BasicOperationsSuite extends TestSuiteBase { withStreamingContext(setupStreams(input, operation, 2)) { ssc => val output = runStreamsWithPartitions(ssc, 3, 3) assert(output.size === 3) - val first = output(0) - val second = output(1) - val third = output(2) + val outputArray = output.toArray + val first = outputArray(0) + val second = outputArray(1) + val third = outputArray(2) assert(first.size === 5) assert(second.size === 5) @@ -104,9 +106,10 @@ class BasicOperationsSuite extends TestSuiteBase { withStreamingContext(setupStreams(input, operation, 5)) { ssc => val output = runStreamsWithPartitions(ssc, 3, 3) assert(output.size === 3) - val first = output(0) - val second = output(1) - val third = output(2) + val outputArray = output.toArray + val first = outputArray(0) + val second = outputArray(1) + val third = outputArray(2) assert(first.size === 2) assert(second.size === 2) @@ -186,7 +189,7 @@ class BasicOperationsSuite extends TestSuiteBase { val output = Seq(1 to 8, 101 to 108, 201 to 208) testOperation( input, - (s: DStream[Int]) => s.union(s.map(_ + 4)) , + (s: DStream[Int]) => s.union(s.map(_ + 4)), output ) } @@ -467,6 +470,72 @@ class BasicOperationsSuite extends TestSuiteBase { testOperation(inputData, updateStateOperation, outputData, true) } + test("updateStateByKey - testing time stamps as input") { + type StreamingState = Long + val initial: Seq[(String, StreamingState)] = Seq(("a", 0L), ("c", 0L)) + + val inputData = + Seq( + Seq("a"), + Seq("a", "b"), + Seq("a", "b", "c"), + Seq("a", "b"), + Seq("a"), + Seq() + ) + + // a -> 1000, 3000, 6000, 10000, 15000, 15000 + // b -> 0, 2000, 5000, 9000, 9000, 9000 + // c -> 1000, 1000, 3000, 3000, 3000, 3000 + + val outputData: Seq[Seq[(String, StreamingState)]] = Seq( + Seq( + ("a", 1000L), + ("c", 0L)), // t = 1000 + Seq( + ("a", 3000L), + ("b", 2000L), + ("c", 0L)), // t = 2000 + Seq( + ("a", 6000L), + ("b", 5000L), + ("c", 3000L)), // t = 3000 + Seq( + ("a", 10000L), + ("b", 9000L), + ("c", 3000L)), // t = 4000 + Seq( + ("a", 15000L), + ("b", 9000L), + ("c", 3000L)), // t = 5000 + Seq( + ("a", 15000L), + ("b", 9000L), + ("c", 3000L)) // t = 6000 + ) + + val updateStateOperation = (s: DStream[String]) => { + val initialRDD = s.context.sparkContext.makeRDD(initial) + val updateFunc = (time: Time, + key: String, + values: Seq[Int], + state: Option[StreamingState]) => { + // Update only if we receive values for this key during the batch. + if (values.nonEmpty) { + Option(time.milliseconds + state.getOrElse(0L)) + } else { + Option(state.getOrElse(0L)) + } + } + s.map(x => (x, 1)).updateStateByKey[StreamingState](updateFunc = updateFunc, + partitioner = new HashPartitioner (numInputPartitions), rememberPartitioner = false, + initialRDD = Option(initialRDD)) + } + + testOperation(input = inputData, operation = updateStateOperation, + expectedOutput = outputData, useSet = true) + } + test("updateStateByKey - with initial value RDD") { val initial = Seq(("a", 1), ("c", 2)) @@ -534,10 +603,9 @@ class BasicOperationsSuite extends TestSuiteBase { val stateObj = state.getOrElse(new StateObject) values.sum match { case 0 => stateObj.expireCounter += 1 // no new values - case n => { // has new values, increment and reset expireCounter + case n => // has new values, increment and reset expireCounter stateObj.counter += n stateObj.expireCounter = 0 - } } stateObj.expireCounter match { case 2 => None // seen twice with no new values, give it the boot @@ -588,48 +656,57 @@ class BasicOperationsSuite extends TestSuiteBase { .window(Seconds(4), Seconds(2)) } - val operatedStream = runCleanupTest(conf, operation _, - numExpectedOutput = cleanupTestInput.size / 2, rememberDuration = Seconds(3)) - val windowedStream2 = operatedStream.asInstanceOf[WindowedDStream[_]] - val windowedStream1 = windowedStream2.dependencies.head.asInstanceOf[WindowedDStream[_]] - val mappedStream = windowedStream1.dependencies.head - - // Checkpoint remember durations - assert(windowedStream2.rememberDuration === rememberDuration) - assert(windowedStream1.rememberDuration === rememberDuration + windowedStream2.windowDuration) - assert(mappedStream.rememberDuration === - rememberDuration + windowedStream2.windowDuration + windowedStream1.windowDuration) - - // WindowedStream2 should remember till 7 seconds: 10, 9, 8, 7 - // WindowedStream1 should remember till 4 seconds: 10, 9, 8, 7, 6, 5, 4 - // MappedStream should remember till 2 seconds: 10, 9, 8, 7, 6, 5, 4, 3, 2 - - // WindowedStream2 - assert(windowedStream2.generatedRDDs.contains(Time(10000))) - assert(windowedStream2.generatedRDDs.contains(Time(8000))) - assert(!windowedStream2.generatedRDDs.contains(Time(6000))) - - // WindowedStream1 - assert(windowedStream1.generatedRDDs.contains(Time(10000))) - assert(windowedStream1.generatedRDDs.contains(Time(4000))) - assert(!windowedStream1.generatedRDDs.contains(Time(3000))) - - // MappedStream - assert(mappedStream.generatedRDDs.contains(Time(10000))) - assert(mappedStream.generatedRDDs.contains(Time(2000))) - assert(!mappedStream.generatedRDDs.contains(Time(1000))) + runCleanupTest( + conf, + operation _, + numExpectedOutput = cleanupTestInput.size / 2, + rememberDuration = Seconds(3)) { operatedStream => + eventually(eventuallyTimeout) { + val windowedStream2 = operatedStream.asInstanceOf[WindowedDStream[_]] + val windowedStream1 = windowedStream2.dependencies.head.asInstanceOf[WindowedDStream[_]] + val mappedStream = windowedStream1.dependencies.head + + // Checkpoint remember durations + assert(windowedStream2.rememberDuration === rememberDuration) + assert( + windowedStream1.rememberDuration === rememberDuration + windowedStream2.windowDuration) + assert(mappedStream.rememberDuration === + rememberDuration + windowedStream2.windowDuration + windowedStream1.windowDuration) + + // WindowedStream2 should remember till 7 seconds: 10, 9, 8, 7 + // WindowedStream1 should remember till 4 seconds: 10, 9, 8, 7, 6, 5, 4 + // MappedStream should remember till 2 seconds: 10, 9, 8, 7, 6, 5, 4, 3, 2 + + // WindowedStream2 + assert(windowedStream2.generatedRDDs.contains(Time(10000))) + assert(windowedStream2.generatedRDDs.contains(Time(8000))) + assert(!windowedStream2.generatedRDDs.contains(Time(6000))) + + // WindowedStream1 + assert(windowedStream1.generatedRDDs.contains(Time(10000))) + assert(windowedStream1.generatedRDDs.contains(Time(4000))) + assert(!windowedStream1.generatedRDDs.contains(Time(3000))) + + // MappedStream + assert(mappedStream.generatedRDDs.contains(Time(10000))) + assert(mappedStream.generatedRDDs.contains(Time(2000))) + assert(!mappedStream.generatedRDDs.contains(Time(1000))) + } + } } test("rdd cleanup - updateStateByKey") { val updateFunc = (values: Seq[Int], state: Option[Int]) => { Some(values.sum + state.getOrElse(0)) } - val stateStream = runCleanupTest( - conf, _.map(_ -> 1).updateStateByKey(updateFunc).checkpoint(Seconds(3))) - - assert(stateStream.rememberDuration === stateStream.checkpointDuration * 2) - assert(stateStream.generatedRDDs.contains(Time(10000))) - assert(!stateStream.generatedRDDs.contains(Time(4000))) + runCleanupTest( + conf, _.map(_ -> 1).updateStateByKey(updateFunc).checkpoint(Seconds(3))) { stateStream => + eventually(eventuallyTimeout) { + assert(stateStream.rememberDuration === stateStream.checkpointDuration * 2) + assert(stateStream.generatedRDDs.contains(Time(10000))) + assert(!stateStream.generatedRDDs.contains(Time(4000))) + } + } } test("rdd cleanup - input blocks and persisted RDDs") { @@ -645,8 +722,8 @@ class BasicOperationsSuite extends TestSuiteBase { val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) val mappedStream = networkStream.map(_ + ".").persist() - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - val outputStream = new TestOutputStream(mappedStream, outputBuffer) + val outputQueue = new ConcurrentLinkedQueue[Seq[String]] + val outputStream = new TestOutputStream(mappedStream, outputQueue) outputStream.register() ssc.start() @@ -685,7 +762,7 @@ class BasicOperationsSuite extends TestSuiteBase { testServer.stop() // verify data has been received - assert(outputBuffer.size > 0) + assert(!outputQueue.isEmpty) assert(blockRdds.size > 0) assert(persistentRddIds.size > 0) @@ -710,13 +787,16 @@ class BasicOperationsSuite extends TestSuiteBase { } } - /** Test cleanup of RDDs in DStream metadata */ + /** + * Test cleanup of RDDs in DStream metadata. `assertCleanup` is the function that asserts the + * cleanup of RDDs is successful. + */ def runCleanupTest[T: ClassTag]( conf2: SparkConf, operation: DStream[Int] => DStream[T], numExpectedOutput: Int = cleanupTestInput.size, rememberDuration: Duration = null - ): DStream[T] = { + )(assertCleanup: (DStream[T]) => Unit): DStream[T] = { // Setup the stream computation assert(batchDuration === Seconds(1), @@ -725,7 +805,11 @@ class BasicOperationsSuite extends TestSuiteBase { val operatedStream = ssc.graph.getOutputStreams().head.dependencies.head.asInstanceOf[DStream[T]] if (rememberDuration != null) ssc.remember(rememberDuration) - val output = runStreams[(Int, Int)](ssc, cleanupTestInput.size, numExpectedOutput) + val output = runStreams[(Int, Int)]( + ssc, + cleanupTestInput.size, + numExpectedOutput, + () => assertCleanup(operatedStream)) val clock = ssc.scheduler.clock.asInstanceOf[Clock] assert(clock.getTimeMillis() === Seconds(10).milliseconds) assert(output.size === numExpectedOutput) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 84f5294aa39c..ee2fd45a7e85 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -17,32 +17,197 @@ package org.apache.spark.streaming -import java.io.{ObjectOutputStream, ByteArrayOutputStream, ByteArrayInputStream, File} -import org.apache.spark.TestUtils +import java.io._ +import java.nio.charset.StandardCharsets +import java.util.concurrent.ConcurrentLinkedQueue -import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} +import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import com.google.common.base.Charsets import com.google.common.io.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.{IntWritable, Text} import org.apache.hadoop.mapred.TextOutputFormat import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} +import org.mockito.Mockito.mock import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.streaming.dstream.{DStream, FileInputDStream} -import org.apache.spark.streaming.scheduler.{ConstantEstimator, RateTestInputDStream, RateTestReceiver} -import org.apache.spark.util.{MutableURLClassLoader, Clock, ManualClock, Utils} +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite, TestUtils} +import org.apache.spark.internal.config._ +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.util.{Clock, ManualClock, MutableURLClassLoader, ResetSystemProperties, + Utils} + +/** + * A input stream that records the times of restore() invoked + */ +private[streaming] +class CheckpointInputDStream(_ssc: StreamingContext) extends InputDStream[Int](_ssc) { + protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData + override def start(): Unit = { } + override def stop(): Unit = { } + override def compute(time: Time): Option[RDD[Int]] = Some(ssc.sc.makeRDD(Seq(1))) + private[streaming] + class FileInputDStreamCheckpointData extends DStreamCheckpointData(this) { + @transient + var restoredTimes = 0 + override def restore() { + restoredTimes += 1 + super.restore() + } + } +} + +/** + * A trait of that can be mixed in to get methods for testing DStream operations under + * DStream checkpointing. Note that the implementations of this trait has to implement + * the `setupCheckpointOperation` + */ +trait DStreamCheckpointTester { self: SparkFunSuite => + + /** + * Tests a streaming operation under checkpointing, by restarting the operation + * from checkpoint file and verifying whether the final output is correct. + * The output is assumed to have come from a reliable queue which a replay + * data as required. + * + * NOTE: This takes into consideration that the last batch processed before + * master failure will be re-processed after restart/recovery. + */ + protected def testCheckpointedOperation[U: ClassTag, V: ClassTag]( + input: Seq[Seq[U]], + operation: DStream[U] => DStream[V], + expectedOutput: Seq[Seq[V]], + numBatchesBeforeRestart: Int, + batchDuration: Duration = Milliseconds(500), + stopSparkContextAfterTest: Boolean = true + ) { + require(numBatchesBeforeRestart < expectedOutput.size, + "Number of batches before context restart less than number of expected output " + + "(i.e. number of total batches to run)") + require(StreamingContext.getActive().isEmpty, + "Cannot run test with already active streaming context") + + // Current code assumes that number of batches to be run = number of inputs + val totalNumBatches = input.size + val batchDurationMillis = batchDuration.milliseconds + + // Setup the stream computation + val checkpointDir = Utils.createTempDir(this.getClass.getSimpleName()).toString + logDebug(s"Using checkpoint directory $checkpointDir") + val ssc = createContextForCheckpointOperation(batchDuration) + require(ssc.conf.get("spark.streaming.clock") === classOf[ManualClock].getName, + "Cannot run test without manual clock in the conf") + + val inputStream = new TestInputStream(ssc, input, numPartitions = 2) + val operatedStream = operation(inputStream) + operatedStream.print() + val outputStream = new TestOutputStreamWithPartitions(operatedStream, + new ConcurrentLinkedQueue[Seq[Seq[V]]]) + outputStream.register() + ssc.checkpoint(checkpointDir) + + // Do the computation for initial number of batches, create checkpoint file and quit + val beforeRestartOutput = generateOutput[V](ssc, + Time(batchDurationMillis * numBatchesBeforeRestart), checkpointDir, stopSparkContextAfterTest) + assertOutput(beforeRestartOutput, expectedOutput, beforeRestart = true) + // Restart and complete the computation from checkpoint file + logInfo( + "\n-------------------------------------------\n" + + " Restarting stream computation " + + "\n-------------------------------------------\n" + ) + + val restartedSsc = new StreamingContext(checkpointDir) + val afterRestartOutput = generateOutput[V](restartedSsc, + Time(batchDurationMillis * totalNumBatches), checkpointDir, stopSparkContextAfterTest) + assertOutput(afterRestartOutput, expectedOutput, beforeRestart = false) + } + + protected def createContextForCheckpointOperation(batchDuration: Duration): StreamingContext = { + val conf = new SparkConf().setMaster("local").setAppName(this.getClass.getSimpleName) + conf.set("spark.streaming.clock", classOf[ManualClock].getName()) + new StreamingContext(SparkContext.getOrCreate(conf), batchDuration) + } + + /** + * Get the first TestOutputStreamWithPartitions, does not check the provided generic type. + */ + protected def getTestOutputStream[V: ClassTag](streams: Array[DStream[_]]): + TestOutputStreamWithPartitions[V] = { + streams.collect { + case ds: TestOutputStreamWithPartitions[V @unchecked] => ds + }.head + } + + + protected def generateOutput[V: ClassTag]( + ssc: StreamingContext, + targetBatchTime: Time, + checkpointDir: String, + stopSparkContext: Boolean + ): Seq[Seq[V]] = { + try { + val batchCounter = new BatchCounter(ssc) + ssc.start() + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + + logInfo("Manual clock before advancing = " + clock.getTimeMillis()) + clock.setTime(targetBatchTime.milliseconds) + logInfo("Manual clock after advancing = " + clock.getTimeMillis()) + + val outputStream = getTestOutputStream[V](ssc.graph.getOutputStreams()) + + eventually(timeout(10 seconds)) { + ssc.awaitTerminationOrTimeout(10) + assert(batchCounter.getLastCompletedBatchTime === targetBatchTime) + } + + eventually(timeout(10 seconds)) { + val checkpointFilesOfLatestTime = Checkpoint.getCheckpointFiles(checkpointDir).filter { + _.getName.contains(clock.getTimeMillis.toString) + } + // Checkpoint files are written twice for every batch interval. So assert that both + // are written to make sure that both of them have been written. + assert(checkpointFilesOfLatestTime.size === 2) + } + outputStream.output.asScala.map(_.flatten).toSeq + + } finally { + ssc.stop(stopSparkContext = stopSparkContext) + } + } + + private def assertOutput[V: ClassTag]( + output: Seq[Seq[V]], + expectedOutput: Seq[Seq[V]], + beforeRestart: Boolean): Unit = { + val expectedPartialOutput = if (beforeRestart) { + expectedOutput.take(output.size) + } else { + expectedOutput.takeRight(output.size) + } + val setComparison = output.zip(expectedPartialOutput).forall { + case (o, e) => o.toSet === e.toSet + } + assert(setComparison, s"set comparison failed\n" + + s"Expected output items:\n${expectedPartialOutput.mkString("\n")}\n" + + s"Generated output items: ${output.mkString("\n")}" + ) + } +} /** * This test suites tests the checkpointing functionality of DStreams - * the checkpointing of a DStream's RDDs as well as the checkpointing of * the whole DStream graph. */ -class CheckpointSuite extends TestSuiteBase { +class CheckpointSuite extends TestSuiteBase with DStreamCheckpointTester + with ResetSystemProperties { var ssc: StreamingContext = null @@ -54,9 +219,17 @@ class CheckpointSuite extends TestSuiteBase { } override def afterFunction() { - super.afterFunction() - if (ssc != null) ssc.stop() - Utils.deleteRecursively(new File(checkpointDir)) + try { + if (ssc != null) { ssc.stop() } + Utils.deleteRecursively(new File(checkpointDir)) + } finally { + super.afterFunction() + } + } + + test("non-existent checkpoint dir") { + // SPARK-13211 + intercept[IllegalArgumentException](new StreamingContext("nosuchdirectory")) } test("basic rdd checkpoints + dstream graph checkpoint recovery") { @@ -93,10 +266,9 @@ class CheckpointSuite extends TestSuiteBase { assert(!stateStream.checkpointData.currentCheckpointFiles.isEmpty, "No checkpointed RDDs in state stream before first failure") stateStream.checkpointData.currentCheckpointFiles.foreach { - case (time, file) => { + case (time, file) => assert(fs.exists(new Path(file)), "Checkpoint file '" + file +"' for time " + time + " for state stream before first failure does not exist") - } } // Run till a further time such that previous checkpoint files in the stream would be deleted @@ -123,10 +295,9 @@ class CheckpointSuite extends TestSuiteBase { assert(!stateStream.checkpointData.currentCheckpointFiles.isEmpty, "No checkpointed RDDs in state stream before second failure") stateStream.checkpointData.currentCheckpointFiles.foreach { - case (time, file) => { + case (time, file) => assert(fs.exists(new Path(file)), "Checkpoint file '" + file +"' for time " + time + " for state stream before seconds failure does not exist") - } } ssc.stop() @@ -234,7 +405,8 @@ class CheckpointSuite extends TestSuiteBase { // explicitly. ssc = new StreamingContext(null, newCp, null) val restoredConf1 = ssc.conf - assert(restoredConf1.get("spark.driver.host") === "localhost") + val defaultConf = new SparkConf() + assert(restoredConf1.get("spark.driver.host") === defaultConf.get(DRIVER_HOST_ADDRESS)) assert(restoredConf1.get("spark.driver.port") !== "9999") } @@ -250,7 +422,9 @@ class CheckpointSuite extends TestSuiteBase { Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), - Seq(("", 2)), Seq() ), + Seq(("", 2)), + Seq() + ), 3 ) } @@ -430,7 +604,7 @@ class CheckpointSuite extends TestSuiteBase { // Set up the streaming context and input streams val batchDuration = Seconds(2) // Due to 1-second resolution of setLastModified() on some OS's. val testDir = Utils.createTempDir() - val outputBuffer = new ArrayBuffer[Seq[Int]] with SynchronizedBuffer[Seq[Int]] + val outputBuffer = new ConcurrentLinkedQueue[Seq[Int]] /** * Writes a file named `i` (which contains the number `i`) to the test directory and sets its @@ -438,7 +612,7 @@ class CheckpointSuite extends TestSuiteBase { */ def writeFile(i: Int, clock: Clock): Unit = { val file = new File(testDir, i.toString) - Files.write(i + "\n", file, Charsets.UTF_8) + Files.write(i + "\n", file, StandardCharsets.UTF_8) assert(file.setLastModified(clock.getTimeMillis())) // Check that the file's modification date is actually the value we wrote, since rounding or // truncation will break the test: @@ -451,8 +625,9 @@ class CheckpointSuite extends TestSuiteBase { def recordedFiles(ssc: StreamingContext): Seq[Int] = { val fileInputDStream = ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]] - val filenames = fileInputDStream.batchTimeToSelectedFiles.values.flatten - filenames.map(_.split(File.separator).last.toInt).toSeq.sorted + val filenames = fileInputDStream.batchTimeToSelectedFiles.synchronized + { fileInputDStream.batchTimeToSelectedFiles.values.flatten } + filenames.map(_.split("/").last.toInt).toSeq.sorted } try { @@ -465,16 +640,18 @@ class CheckpointSuite extends TestSuiteBase { val fileStream = ssc.textFileStream(testDir.toString) // Make value 3 take a large time to process, to ensure that the driver // shuts down in the middle of processing the 3rd batch - CheckpointSuite.batchThreeShouldBlockIndefinitely = true - val mappedStream = fileStream.map(s => { + CheckpointSuite.batchThreeShouldBlockALongTime = true + val mappedStream = fileStream.map { s => val i = s.toInt if (i == 3) { - while (CheckpointSuite.batchThreeShouldBlockIndefinitely) { - Thread.sleep(Long.MaxValue) + if (CheckpointSuite.batchThreeShouldBlockALongTime) { + // It's not a good idea to let the thread run forever + // as resource won't be correctly released + Thread.sleep(6000) } } i - }) + } // Reducing over a large window to ensure that recovery from driver failure // requires reprocessing of all the files seen before the failure @@ -510,11 +687,11 @@ class CheckpointSuite extends TestSuiteBase { ssc.stop() // Check that we shut down while the third batch was being processed assert(batchCounter.getNumCompletedBatches === 2) - assert(outputStream.output.flatten === Seq(1, 3)) + assert(outputStream.output.asScala.toSeq.flatten === Seq(1, 3)) } // The original StreamingContext has now been stopped. - CheckpointSuite.batchThreeShouldBlockIndefinitely = false + CheckpointSuite.batchThreeShouldBlockALongTime = false // Create files while the streaming driver is down for (i <- Seq(4, 5, 6)) { @@ -560,7 +737,7 @@ class CheckpointSuite extends TestSuiteBase { assert(batchCounter.getNumCompletedBatches === index + numBatchesAfterRestart + 1) } } - logInfo("Output after restart = " + outputStream.output.mkString("[", ", ", "]")) + logInfo("Output after restart = " + outputStream.output.asScala.mkString("[", ", ", "]")) assert(outputStream.output.size > 0, "No files processed after restart") ssc.stop() @@ -569,14 +746,49 @@ class CheckpointSuite extends TestSuiteBase { assert(recordedFiles(ssc) === (1 to 9)) // Append the new output to the old buffer - outputBuffer ++= outputStream.output + outputBuffer.addAll(outputStream.output) // Verify whether all the elements received are as expected val expectedOutput = Seq(1, 3, 6, 10, 15, 21, 28, 36, 45) - assert(outputBuffer.flatten.toSet === expectedOutput.toSet) + assert(outputBuffer.asScala.flatten.toSet === expectedOutput.toSet) } } finally { - Utils.deleteRecursively(testDir) + try { + // As the driver shuts down in the middle of processing and the thread above sleeps + // for a while, `testDir` can be not closed correctly at this point which causes the + // test failure on Windows. + Utils.deleteRecursively(testDir) + } catch { + case e: IOException if Utils.isWindows => + logWarning(e.getMessage) + } + } + } + + test("DStreamCheckpointData.restore invoking times") { + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + ssc.checkpoint(checkpointDir) + val inputDStream = new CheckpointInputDStream(ssc) + val checkpointData = inputDStream.checkpointData + val mappedDStream = inputDStream.map(_ + 100) + val outputStream = new TestOutputStreamWithPartitions(mappedDStream) + outputStream.register() + // do two more times output + mappedDStream.foreachRDD(rdd => rdd.count()) + mappedDStream.foreachRDD(rdd => rdd.count()) + assert(checkpointData.restoredTimes === 0) + val batchDurationMillis = ssc.progressListener.batchDuration + generateOutput(ssc, Time(batchDurationMillis * 3), checkpointDir, stopSparkContext = true) + assert(checkpointData.restoredTimes === 0) + } + logInfo("*********** RESTARTING ************") + withStreamingContext(new StreamingContext(checkpointDir)) { ssc => + val checkpointData = + ssc.graph.getInputStreams().head.asInstanceOf[CheckpointInputDStream].checkpointData + assert(checkpointData.restoredTimes === 1) + ssc.start() + ssc.stop() + assert(checkpointData.restoredTimes === 1) } } @@ -609,60 +821,106 @@ class CheckpointSuite extends TestSuiteBase { val ois = new ObjectInputStreamWithLoader( new ByteArrayInputStream(bos.toByteArray), loader) assert(ois.readObject().asInstanceOf[Class[_]].getName == "[LtestClz;") + ois.close() } - /** - * Tests a streaming operation under checkpointing, by restarting the operation - * from checkpoint file and verifying whether the final output is correct. - * The output is assumed to have come from a reliable queue which an replay - * data as required. - * - * NOTE: This takes into consideration that the last batch processed before - * master failure will be re-processed after restart/recovery. - */ - def testCheckpointedOperation[U: ClassTag, V: ClassTag]( - input: Seq[Seq[U]], - operation: DStream[U] => DStream[V], - expectedOutput: Seq[Seq[V]], - initialNumBatches: Int - ) { - - // Current code assumes that: - // number of inputs = number of outputs = number of batches to be run - val totalNumBatches = input.size - val nextNumBatches = totalNumBatches - initialNumBatches - val initialNumExpectedOutputs = initialNumBatches - val nextNumExpectedOutputs = expectedOutput.size - initialNumExpectedOutputs + 1 - // because the last batch will be processed again + test("SPARK-11267: the race condition of two checkpoints in a batch") { + val jobGenerator = mock(classOf[JobGenerator]) + val checkpointDir = Utils.createTempDir().toString + val checkpointWriter = + new CheckpointWriter(jobGenerator, conf, checkpointDir, new Configuration()) + val bytes1 = Array.fill[Byte](10)(1) + new checkpointWriter.CheckpointWriteHandler( + Time(2000), bytes1, clearCheckpointDataLater = false).run() + val bytes2 = Array.fill[Byte](10)(2) + new checkpointWriter.CheckpointWriteHandler( + Time(1000), bytes2, clearCheckpointDataLater = true).run() + val checkpointFiles = Checkpoint.getCheckpointFiles(checkpointDir).reverse.map { path => + new File(path.toUri) + } + assert(checkpointFiles.size === 2) + // Although bytes2 was written with an old time, it contains the latest status, so we should + // try to read from it at first. + assert(Files.toByteArray(checkpointFiles(0)) === bytes2) + assert(Files.toByteArray(checkpointFiles(1)) === bytes1) + checkpointWriter.stop() + } - // Do the computation for initial number of batches, create checkpoint file and quit - ssc = setupStreams[U, V](input, operation) - ssc.start() - val output = advanceTimeWithRealDelay[V](ssc, initialNumBatches) - ssc.stop() - verifyOutput[V](output, expectedOutput.take(initialNumBatches), true) - Thread.sleep(1000) + test("SPARK-6847: stack overflow when updateStateByKey is followed by a checkpointed dstream") { + // In this test, there are two updateStateByKey operators. The RDD DAG is as follows: + // + // batch 1 batch 2 batch 3 ... + // + // 1) input rdd input rdd input rdd + // | | | + // v v v + // 2) cogroup rdd ---> cogroup rdd ---> cogroup rdd ... + // | / | / | + // v / v / v + // 3) map rdd --- map rdd --- map rdd ... + // | | | + // v v v + // 4) cogroup rdd ---> cogroup rdd ---> cogroup rdd ... + // | / | / | + // v / v / v + // 5) map rdd --- map rdd --- map rdd ... + // + // Every batch depends on its previous batch, so "updateStateByKey" needs to do checkpoint to + // break the RDD chain. However, before SPARK-6847, when the state RDD (layer 5) of the second + // "updateStateByKey" does checkpoint, it won't checkpoint the state RDD (layer 3) of the first + // "updateStateByKey" (Note: "updateStateByKey" has already marked that its state RDD (layer 3) + // should be checkpointed). Hence, the connections between layer 2 and layer 3 won't be broken + // and the RDD chain will grow infinitely and cause StackOverflow. + // + // Therefore SPARK-6847 introduces "spark.checkpoint.checkpointAllMarked" to force checkpointing + // all marked RDDs in the DAG to resolve this issue. (For the previous example, it will break + // connections between layer 2 and layer 3) + ssc = new StreamingContext(master, framework, batchDuration) + val batchCounter = new BatchCounter(ssc) + ssc.checkpoint(checkpointDir) + val inputDStream = new CheckpointInputDStream(ssc) + val updateFunc = (values: Seq[Int], state: Option[Int]) => { + Some(values.sum + state.getOrElse(0)) + } + @volatile var shouldCheckpointAllMarkedRDDs = false + @volatile var rddsCheckpointed = false + inputDStream.map(i => (i, i)) + .updateStateByKey(updateFunc).checkpoint(batchDuration) + .updateStateByKey(updateFunc).checkpoint(batchDuration) + .foreachRDD { rdd => + /** + * Find all RDDs that are marked for checkpointing in the specified RDD and its ancestors. + */ + def findAllMarkedRDDs(rdd: RDD[_]): List[RDD[_]] = { + val markedRDDs = rdd.dependencies.flatMap(dep => findAllMarkedRDDs(dep.rdd)).toList + if (rdd.checkpointData.isDefined) { + rdd :: markedRDDs + } else { + markedRDDs + } + } - // Restart and complete the computation from checkpoint file - logInfo( - "\n-------------------------------------------\n" + - " Restarting stream computation " + - "\n-------------------------------------------\n" - ) - ssc = new StreamingContext(checkpointDir) + shouldCheckpointAllMarkedRDDs = + Option(rdd.sparkContext.getLocalProperty(RDD.CHECKPOINT_ALL_MARKED_ANCESTORS)). + map(_.toBoolean).getOrElse(false) + + val stateRDDs = findAllMarkedRDDs(rdd) + rdd.count() + // Check the two state RDDs are both checkpointed + rddsCheckpointed = stateRDDs.size == 2 && stateRDDs.forall(_.isCheckpointed) + } ssc.start() - val outputNew = advanceTimeWithRealDelay[V](ssc, nextNumBatches) - // the first element will be re-processed data of the last batch before restart - verifyOutput[V](outputNew, expectedOutput.takeRight(nextNumExpectedOutputs), true) - ssc.stop() - ssc = null + batchCounter.waitUntilBatchesCompleted(1, 10000) + assert(shouldCheckpointAllMarkedRDDs === true) + assert(rddsCheckpointed === true) } /** * Advances the manual clock on the streaming scheduler by given number of batches. * It also waits for the expected amount of time for each batch. */ - def advanceTimeWithRealDelay[V: ClassTag](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = + def advanceTimeWithRealDelay[V: ClassTag](ssc: StreamingContext, numBatches: Long): + Iterable[Seq[V]] = { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] logInfo("Manual clock before advancing = " + clock.getTimeMillis()) @@ -673,13 +931,11 @@ class CheckpointSuite extends TestSuiteBase { logInfo("Manual clock after advancing = " + clock.getTimeMillis()) Thread.sleep(batchDuration.milliseconds) - val outputStream = ssc.graph.getOutputStreams().filter { dstream => - dstream.isInstanceOf[TestOutputStreamWithPartitions[V]] - }.head.asInstanceOf[TestOutputStreamWithPartitions[V]] - outputStream.output.map(_.flatten) + val outputStream = getTestOutputStream[V](ssc.graph.getOutputStreams()) + outputStream.output.asScala.map(_.flatten) } } private object CheckpointSuite extends Serializable { - var batchThreeShouldBlockIndefinitely: Boolean = true + var batchThreeShouldBlockALongTime: Boolean = true } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/DStreamClosureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/DStreamClosureSuite.scala index 9b5e4dc819a2..2ab600ab817e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/DStreamClosureSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/DStreamClosureSuite.scala @@ -33,13 +33,18 @@ class DStreamClosureSuite extends SparkFunSuite with BeforeAndAfterAll { private var ssc: StreamingContext = null override def beforeAll(): Unit = { + super.beforeAll() val sc = new SparkContext("local", "test") ssc = new StreamingContext(sc, Seconds(1)) } override def afterAll(): Unit = { - ssc.stop(stopSparkContext = true) - ssc = null + try { + ssc.stop(stopSparkContext = true) + ssc = null + } finally { + super.afterAll() + } } test("user provided closures are actually cleaned") { @@ -51,7 +56,6 @@ class DStreamClosureSuite extends SparkFunSuite with BeforeAndAfterAll { testFilter(dstream) testMapPartitions(dstream) testReduce(dstream) - testForeach(dstream) testForeachRDD(dstream) testTransform(dstream) testTransformWith(dstream) @@ -101,12 +105,6 @@ class DStreamClosureSuite extends SparkFunSuite with BeforeAndAfterAll { private def testReduce(ds: DStream[Int]): Unit = expectCorrectException { ds.reduce { case (_, _) => return; 1 } } - private def testForeach(ds: DStream[Int]): Unit = { - val foreachF1 = (rdd: RDD[Int], t: Time) => return - val foreachF2 = (rdd: RDD[Int]) => return - expectCorrectException { ds.foreach(foreachF1) } - expectCorrectException { ds.foreach(foreachF2) } - } private def testForeachRDD(ds: DStream[Int]): Unit = { val foreachRDDF1 = (rdd: RDD[Int], t: Time) => return val foreachRDDF2 = (rdd: RDD[Int]) => return @@ -166,6 +164,10 @@ class DStreamClosureSuite extends SparkFunSuite with BeforeAndAfterAll { private def testUpdateStateByKey(ds: DStream[(Int, Int)]): Unit = { val updateF1 = (_: Seq[Int], _: Option[Int]) => { return; Some(1) } val updateF2 = (_: Iterator[(Int, Seq[Int], Option[Int])]) => { return; Seq((1, 1)).toIterator } + val updateF3 = (_: Time, _: Int, _: Seq[Int], _: Option[Int]) => { + return + Option(1) + } val initialRDD = ds.ssc.sparkContext.emptyRDD[Int].map { i => (i, i) } expectCorrectException { ds.updateStateByKey(updateF1) } expectCorrectException { ds.updateStateByKey(updateF1, 5) } @@ -179,6 +181,14 @@ class DStreamClosureSuite extends SparkFunSuite with BeforeAndAfterAll { expectCorrectException { ds.updateStateByKey(updateF2, new HashPartitioner(5), true, initialRDD) } + expectCorrectException { + ds.updateStateByKey( + updateFunc = updateF3, + partitioner = new HashPartitioner(5), + rememberPartitioner = true, + initialRDD = Option(initialRDD) + ) + } } private def testMapValues(ds: DStream[(Int, Int)]): Unit = expectCorrectException { ds.mapValues { _ => return; 1 } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala index 8844c9d74b93..94f1bcebc3a3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/DStreamScopeSuite.scala @@ -17,12 +17,15 @@ package org.apache.spark.streaming +import scala.collection.mutable.ArrayBuffer + import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} -import org.apache.spark.{SparkContext, SparkFunSuite} -import org.apache.spark.rdd.RDDOperationScope +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.ui.UIUtils +import org.apache.spark.util.ManualClock /** * Tests whether scope information is passed from DStream operations to RDDs correctly. @@ -32,11 +35,18 @@ class DStreamScopeSuite extends SparkFunSuite with BeforeAndAfter with BeforeAnd private val batchDuration: Duration = Seconds(1) override def beforeAll(): Unit = { - ssc = new StreamingContext(new SparkContext("local", "test"), batchDuration) + super.beforeAll() + val conf = new SparkConf().setMaster("local").setAppName("test") + conf.set("spark.streaming.clock", classOf[ManualClock].getName()) + ssc = new StreamingContext(new SparkContext(conf), batchDuration) } override def afterAll(): Unit = { - ssc.stop(stopSparkContext = true) + try { + ssc.stop(stopSparkContext = true) + } finally { + super.afterAll() + } } before { assertPropertiesNotSet() } @@ -103,6 +113,8 @@ class DStreamScopeSuite extends SparkFunSuite with BeforeAndAfter with BeforeAnd test("scoping nested operations") { val inputStream = new DummyInputDStream(ssc) + // countByKeyAndWindow internally uses reduceByKeyAndWindow, but only countByKeyAndWindow + // should appear in scope val countStream = inputStream.countByWindow(Seconds(10), Seconds(1)) countStream.initialize(Time(0)) @@ -137,6 +149,57 @@ class DStreamScopeSuite extends SparkFunSuite with BeforeAndAfter with BeforeAnd testStream(countStream) } + test("transform should allow RDD operations to be captured in scopes") { + val inputStream = new DummyInputDStream(ssc) + val transformedStream = inputStream.transform { _.map { _ -> 1}.reduceByKey(_ + _) } + transformedStream.initialize(Time(0)) + + val transformScopeBase = transformedStream.baseScope.map(RDDOperationScope.fromJson) + val transformScope1 = transformedStream.getOrCompute(Time(1000)).get.scope + val transformScope2 = transformedStream.getOrCompute(Time(2000)).get.scope + val transformScope3 = transformedStream.getOrCompute(Time(3000)).get.scope + + // Assert that all children RDDs inherit the DStream operation name correctly + assertDefined(transformScopeBase, transformScope1, transformScope2, transformScope3) + assert(transformScopeBase.get.name === "transform") + assertNestedScopeCorrect(transformScope1.get, 1000) + assertNestedScopeCorrect(transformScope2.get, 2000) + assertNestedScopeCorrect(transformScope3.get, 3000) + + def assertNestedScopeCorrect(rddScope: RDDOperationScope, batchTime: Long): Unit = { + assert(rddScope.name === "reduceByKey") + assert(rddScope.parent.isDefined) + assertScopeCorrect(transformScopeBase.get, rddScope.parent.get, batchTime) + } + } + + test("foreachRDD should allow RDD operations to be captured in scope") { + val inputStream = new DummyInputDStream(ssc) + val generatedRDDs = new ArrayBuffer[RDD[(Int, Int)]] + inputStream.foreachRDD { rdd => + generatedRDDs += rdd.map { _ -> 1}.reduceByKey(_ + _) + } + val batchCounter = new BatchCounter(ssc) + ssc.start() + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + clock.advance(3000) + batchCounter.waitUntilBatchesCompleted(3, 10000) + assert(generatedRDDs.size === 3) + + val foreachBaseScope = + ssc.graph.getOutputStreams().head.baseScope.map(RDDOperationScope.fromJson) + assertDefined(foreachBaseScope) + assert(foreachBaseScope.get.name === "foreachRDD") + + val rddScopes = generatedRDDs.map { _.scope } + assertDefined(rddScopes: _*) + rddScopes.zipWithIndex.foreach { case (rddScope, idx) => + assert(rddScope.get.name === "reduceByKey") + assert(rddScope.get.parent.isDefined) + assertScopeCorrect(foreachBaseScope.get, rddScope.get.parent.get, (idx + 1) * 1000) + } + } + /** Assert that the RDD operation scope properties are not set in our SparkContext. */ private def assertPropertiesNotSet(): Unit = { assert(ssc != null) @@ -149,19 +212,12 @@ class DStreamScopeSuite extends SparkFunSuite with BeforeAndAfter with BeforeAnd baseScope: RDDOperationScope, rddScope: RDDOperationScope, batchTime: Long): Unit = { - assertScopeCorrect(baseScope.id, baseScope.name, rddScope, batchTime) - } - - /** Assert that the given RDD scope inherits the base name and ID correctly. */ - private def assertScopeCorrect( - baseScopeId: String, - baseScopeName: String, - rddScope: RDDOperationScope, - batchTime: Long): Unit = { + val (baseScopeId, baseScopeName) = (baseScope.id, baseScope.name) val formattedBatchTime = UIUtils.formatBatchTime( batchTime, ssc.graph.batchDuration.milliseconds, showYYYYMMSS = false) assert(rddScope.id === s"${baseScopeId}_$batchTime") assert(rddScope.name.replaceAll("\\n", " ") === s"$baseScopeName @ $formattedBatchTime") + assert(rddScope.parent.isEmpty) // There should not be any higher scope } /** Assert that all the specified options are defined. */ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala index e82c2fa4e72a..19ceb748e07f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala @@ -21,7 +21,8 @@ import java.io.File import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkFunSuite, Logging} +import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** @@ -43,6 +44,9 @@ class FailureSuite extends SparkFunSuite with BeforeAndAfter with Logging { Utils.deleteRecursively(directory) } StreamingContext.getActive().foreach { _.stop() } + + // Stop SparkContext if active + SparkContext.getOrCreate(new SparkConf().setMaster("local").setAppName("bla")).stop() } test("multiple failures with map") { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 047e38ef9099..b5d36a36513a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -17,30 +17,29 @@ package org.apache.spark.streaming -import java.io.{File, BufferedWriter, OutputStreamWriter} -import java.net.{Socket, SocketException, ServerSocket} -import java.nio.charset.Charset -import java.util.concurrent.{CountDownLatch, Executors, TimeUnit, ArrayBlockingQueue} +import java.io.{BufferedWriter, File, OutputStreamWriter} +import java.net.{ServerSocket, Socket, SocketException} +import java.nio.charset.StandardCharsets +import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger -import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer, SynchronizedQueue} -import scala.language.postfixOps +import scala.collection.JavaConverters._ +import scala.collection.mutable import com.google.common.io.Files -import org.apache.hadoop.io.{Text, LongWritable} -import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.{LongWritable, Text} +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ -import org.apache.spark.Logging +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.scheduler.{StreamingListenerBatchCompleted, StreamingListener} -import org.apache.spark.util.{ManualClock, Utils} import org.apache.spark.streaming.dstream.{InputDStream, ReceiverInputDStream} import org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.util.{ManualClock, Utils} class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { @@ -58,52 +57,43 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val batchCounter = new BatchCounter(ssc) val networkStream = ssc.socketTextStream( "localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - val outputStream = new TestOutputStream(networkStream, outputBuffer) + val outputQueue = new ConcurrentLinkedQueue[Seq[String]] + val outputStream = new TestOutputStream(networkStream, outputQueue) outputStream.register() ssc.start() // Feed data to the server to send to the network receiver val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] val expectedOutput = input.map(_.toString) - for (i <- 0 until input.size) { + for (i <- input.indices) { testServer.send(input(i).toString + "\n") - Thread.sleep(500) clock.advance(batchDuration.milliseconds) } - // Make sure we finish all batches before "stop" - if (!batchCounter.waitUntilBatchesCompleted(input.size, 30000)) { - fail("Timeout: cannot finish all batches in 30 seconds") + + eventually(eventuallyTimeout) { + clock.advance(batchDuration.milliseconds) + // Verify whether data received was as expected + logInfo("--------------------------------") + logInfo("output.size = " + outputQueue.size) + logInfo("output") + outputQueue.asScala.foreach(x => logInfo("[" + x.mkString(",") + "]")) + logInfo("expected output.size = " + expectedOutput.size) + logInfo("expected output") + expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]")) + logInfo("--------------------------------") + + // Verify whether all the elements received are as expected + // (whether the elements were received one in each interval is not verified) + val output = outputQueue.asScala.flatten.toArray + assert(output.length === expectedOutput.size) + for (i <- output.indices) { + assert(output(i) === expectedOutput(i)) + } } - // Ensure progress listener has been notified of all events - ssc.scheduler.listenerBus.waitUntilEmpty(500) - - // Verify all "InputInfo"s have been reported - assert(ssc.progressListener.numTotalReceivedRecords === input.size) - assert(ssc.progressListener.numTotalProcessedRecords === input.size) - - logInfo("Stopping server") - testServer.stop() - logInfo("Stopping context") - ssc.stop() - - // Verify whether data received was as expected - logInfo("--------------------------------") - logInfo("output.size = " + outputBuffer.size) - logInfo("output") - outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("expected output.size = " + expectedOutput.size) - logInfo("expected output") - expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("--------------------------------") - - // Verify whether all the elements received are as expected - // (whether the elements were received one in each interval is not verified) - val output: ArrayBuffer[String] = outputBuffer.flatMap(x => x) - assert(output.size === expectedOutput.size) - for (i <- 0 until output.size) { - assert(output(i) === expectedOutput(i)) + eventually(eventuallyTimeout) { + assert(ssc.progressListener.numTotalReceivedRecords === input.length) + assert(ssc.progressListener.numTotalProcessedRecords === input.length) } } } @@ -119,8 +109,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val batchCounter = new BatchCounter(ssc) val networkStream = ssc.socketTextStream( "localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - val outputStream = new TestOutputStream(networkStream, outputBuffer) + val outputQueue = new ConcurrentLinkedQueue[Seq[String]] + val outputStream = new TestOutputStream(networkStream, outputQueue) outputStream.register() ssc.start() @@ -140,13 +130,13 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } test("binary records stream") { - val testDir: File = null + var testDir: File = null try { val batchDuration = Seconds(2) - val testDir = Utils.createTempDir() + testDir = Utils.createTempDir() // Create a file that exists before the StreamingContext is created: val existingFile = new File(testDir, "0") - Files.write("0\n", existingFile, Charset.forName("UTF-8")) + Files.write("0\n", existingFile, StandardCharsets.UTF_8) assert(existingFile.setLastModified(10000) && existingFile.lastModified === 10000) // Set up the streaming context and input streams @@ -156,9 +146,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { clock.setTime(existingFile.lastModified + batchDuration.milliseconds) val batchCounter = new BatchCounter(ssc) val fileStream = ssc.binaryRecordsStream(testDir.toString, 1) - val outputBuffer = new ArrayBuffer[Seq[Array[Byte]]] - with SynchronizedBuffer[Seq[Array[Byte]]] - val outputStream = new TestOutputStream(fileStream, outputBuffer) + val outputQueue = new ConcurrentLinkedQueue[Seq[Array[Byte]]] + val outputStream = new TestOutputStream(fileStream, outputQueue) outputStream.register() ssc.start() @@ -166,14 +155,15 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // not enough to trigger a batch clock.advance(batchDuration.milliseconds / 2) - val input = Seq(1, 2, 3, 4, 5) - input.foreach { i => + val numCopies = 3 + val input = Array[Byte](1, 2, 3, 4, 5) + for (i <- 0 until numCopies) { Thread.sleep(batchDuration.milliseconds) val file = new File(testDir, i.toString) - Files.write(Array[Byte](i.toByte), file) + Files.write(input.map(b => (b + i).toByte), file) assert(file.setLastModified(clock.getTimeMillis())) assert(file.lastModified === clock.getTimeMillis()) - logInfo("Created file " + file) + logInfo(s"Created file $file") // Advance the clock after creating the file to avoid a race when // setting its modification time clock.advance(batchDuration.milliseconds) @@ -181,10 +171,10 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { assert(batchCounter.getNumCompletedBatches === i) } } - - val expectedOutput = input.map(i => i.toByte) - val obtainedOutput = outputBuffer.flatten.toList.map(i => i(0).toByte) - assert(obtainedOutput === expectedOutput) + val obtainedOutput = outputQueue.asScala.map(_.flatten).toSeq + for (i <- obtainedOutput.indices) { + assert(obtainedOutput(i) === input.map(b => (b + i).toByte)) + } } } finally { if (testDir != null) Utils.deleteRecursively(testDir) @@ -199,6 +189,68 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { testFileStream(newFilesOnly = false) } + test("file input stream - wildcard") { + var testDir: File = null + try { + val batchDuration = Seconds(2) + testDir = Utils.createTempDir() + val testSubDir1 = Utils.createDirectory(testDir.toString, "tmp1") + val testSubDir2 = Utils.createDirectory(testDir.toString, "tmp2") + + // Create a file that exists before the StreamingContext is created: + val existingFile = new File(testDir, "0") + Files.write("0\n", existingFile, StandardCharsets.UTF_8) + assert(existingFile.setLastModified(10000) && existingFile.lastModified === 10000) + + val pathWithWildCard = testDir.toString + "/*/" + + // Set up the streaming context and input streams + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + clock.setTime(existingFile.lastModified + batchDuration.milliseconds) + val batchCounter = new BatchCounter(ssc) + // monitor "testDir/*/" + val fileStream = ssc.fileStream[LongWritable, Text, TextInputFormat]( + pathWithWildCard).map(_._2.toString) + val outputQueue = new ConcurrentLinkedQueue[Seq[String]] + val outputStream = new TestOutputStream(fileStream, outputQueue) + outputStream.register() + ssc.start() + + // Advance the clock so that the files are created after StreamingContext starts, but + // not enough to trigger a batch + clock.advance(batchDuration.milliseconds / 2) + + def createFileAndAdvenceTime(data: Int, dir: File): Unit = { + val file = new File(testSubDir1, data.toString) + Files.write(data + "\n", file, StandardCharsets.UTF_8) + assert(file.setLastModified(clock.getTimeMillis())) + assert(file.lastModified === clock.getTimeMillis()) + logInfo("Created file " + file) + // Advance the clock after creating the file to avoid a race when + // setting its modification time + clock.advance(batchDuration.milliseconds) + eventually(eventuallyTimeout) { + assert(batchCounter.getNumCompletedBatches === data) + } + } + // Over time, create files in the temp directory 1 + val input1 = Seq(1, 2, 3, 4, 5) + input1.foreach(i => createFileAndAdvenceTime(i, testSubDir1)) + + // Over time, create files in the temp directory 1 + val input2 = Seq(6, 7, 8, 9, 10) + input2.foreach(i => createFileAndAdvenceTime(i, testSubDir2)) + + // Verify that all the files have been read + val expectedOutput = (input1 ++ input2).map(_.toString).toSet + assert(outputQueue.asScala.flatten.toSet === expectedOutput) + } + } finally { + if (testDir != null) Utils.deleteRecursively(testDir) + } + } + test("multi-thread receiver") { // set up the test receiver val numThreads = 10 @@ -206,69 +258,73 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val numTotalRecords = numThreads * numRecordsPerThread val testReceiver = new MultiThreadTestReceiver(numThreads, numRecordsPerThread) MultiThreadTestReceiver.haveAllThreadsFinished = false + val outputQueue = new ConcurrentLinkedQueue[Seq[Long]] + def output: Iterable[Long] = outputQueue.asScala.flatten // set up the network stream using the test receiver - val ssc = new StreamingContext(conf, batchDuration) - val networkStream = ssc.receiverStream[Int](testReceiver) - val countStream = networkStream.count - val outputBuffer = new ArrayBuffer[Seq[Long]] with SynchronizedBuffer[Seq[Long]] - val outputStream = new TestOutputStream(countStream, outputBuffer) - def output: ArrayBuffer[Long] = outputBuffer.flatMap(x => x) - outputStream.register() - ssc.start() - - // Let the data from the receiver be received - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - val startTime = System.currentTimeMillis() - while((!MultiThreadTestReceiver.haveAllThreadsFinished || output.sum < numTotalRecords) && - System.currentTimeMillis() - startTime < 5000) { - Thread.sleep(100) - clock.advance(batchDuration.milliseconds) + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + val networkStream = ssc.receiverStream[Int](testReceiver) + val countStream = networkStream.count + + val outputStream = new TestOutputStream(countStream, outputQueue) + outputStream.register() + ssc.start() + + // Let the data from the receiver be received + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val startTime = System.currentTimeMillis() + while ((!MultiThreadTestReceiver.haveAllThreadsFinished || output.sum < numTotalRecords) && + System.currentTimeMillis() - startTime < 5000) { + Thread.sleep(100) + clock.advance(batchDuration.milliseconds) + } + Thread.sleep(1000) } - Thread.sleep(1000) - logInfo("Stopping context") - ssc.stop() // Verify whether data received was as expected logInfo("--------------------------------") - logInfo("output.size = " + outputBuffer.size) + logInfo("output.size = " + outputQueue.size) logInfo("output") - outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) + outputQueue.asScala.foreach(x => logInfo("[" + x.mkString(",") + "]")) logInfo("--------------------------------") assert(output.sum === numTotalRecords) } test("queue input stream - oneAtATime = true") { - // Set up the streaming context and input streams - val ssc = new StreamingContext(conf, batchDuration) - val queue = new SynchronizedQueue[RDD[String]]() - val queueStream = ssc.queueStream(queue, oneAtATime = true) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - val outputStream = new TestOutputStream(queueStream, outputBuffer) - def output: ArrayBuffer[Seq[String]] = outputBuffer.filter(_.size > 0) - outputStream.register() - ssc.start() - - // Setup data queued into the stream - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] val input = Seq("1", "2", "3", "4", "5") val expectedOutput = input.map(Seq(_)) + val outputQueue = new ConcurrentLinkedQueue[Seq[String]] + def output: Iterable[Seq[String]] = outputQueue.asScala.filter(_.nonEmpty) - val inputIterator = input.toIterator - for (i <- 0 until input.size) { - // Enqueue more than 1 item per tick but they should dequeue one at a time - inputIterator.take(2).foreach(i => queue += ssc.sparkContext.makeRDD(Seq(i))) - clock.advance(batchDuration.milliseconds) + // Set up the streaming context and input streams + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + val queue = new mutable.Queue[RDD[String]]() + val queueStream = ssc.queueStream(queue, oneAtATime = true) + val outputStream = new TestOutputStream(queueStream, outputQueue) + outputStream.register() + ssc.start() + + // Setup data queued into the stream + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + + val inputIterator = input.toIterator + for (i <- input.indices) { + // Enqueue more than 1 item per tick but they should dequeue one at a time + inputIterator.take(2).foreach { i => + queue.synchronized { + queue += ssc.sparkContext.makeRDD(Seq(i)) + } + } + clock.advance(batchDuration.milliseconds) + } + Thread.sleep(1000) } - Thread.sleep(1000) - logInfo("Stopping context") - ssc.stop() // Verify whether data received was as expected logInfo("--------------------------------") - logInfo("output.size = " + outputBuffer.size) + logInfo("output.size = " + outputQueue.size) logInfo("output") - outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) + outputQueue.asScala.foreach(x => logInfo("[" + x.mkString(",") + "]")) logInfo("expected output.size = " + expectedOutput.size) logInfo("expected output") expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]")) @@ -276,45 +332,51 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Verify whether all the elements received are as expected assert(output.size === expectedOutput.size) - for (i <- 0 until output.size) { - assert(output(i) === expectedOutput(i)) - } + output.zipWithIndex.foreach{case (e, i) => assert(e == expectedOutput(i))} } test("queue input stream - oneAtATime = false") { - // Set up the streaming context and input streams - val ssc = new StreamingContext(conf, batchDuration) - val queue = new SynchronizedQueue[RDD[String]]() - val queueStream = ssc.queueStream(queue, oneAtATime = false) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - val outputStream = new TestOutputStream(queueStream, outputBuffer) - def output: ArrayBuffer[Seq[String]] = outputBuffer.filter(_.size > 0) - outputStream.register() - ssc.start() - - // Setup data queued into the stream - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val outputQueue = new ConcurrentLinkedQueue[Seq[String]] + def output: Iterable[Seq[String]] = outputQueue.asScala.filter(_.nonEmpty) val input = Seq("1", "2", "3", "4", "5") val expectedOutput = Seq(Seq("1", "2", "3"), Seq("4", "5")) - // Enqueue the first 3 items (one by one), they should be merged in the next batch - val inputIterator = input.toIterator - inputIterator.take(3).foreach(i => queue += ssc.sparkContext.makeRDD(Seq(i))) - clock.advance(batchDuration.milliseconds) - Thread.sleep(1000) + // Set up the streaming context and input streams + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + val queue = new mutable.Queue[RDD[String]]() + val queueStream = ssc.queueStream(queue, oneAtATime = false) + val outputStream = new TestOutputStream(queueStream, outputQueue) + outputStream.register() + ssc.start() + + // Setup data queued into the stream + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + + // Enqueue the first 3 items (one by one), they should be merged in the next batch + val inputIterator = input.toIterator + inputIterator.take(3).foreach { i => + queue.synchronized { + queue += ssc.sparkContext.makeRDD(Seq(i)) + } + } + clock.advance(batchDuration.milliseconds) + Thread.sleep(1000) - // Enqueue the remaining items (again one by one), merged in the final batch - inputIterator.foreach(i => queue += ssc.sparkContext.makeRDD(Seq(i))) - clock.advance(batchDuration.milliseconds) - Thread.sleep(1000) - logInfo("Stopping context") - ssc.stop() + // Enqueue the remaining items (again one by one), merged in the final batch + inputIterator.foreach { i => + queue.synchronized { + queue += ssc.sparkContext.makeRDD(Seq(i)) + } + } + clock.advance(batchDuration.milliseconds) + Thread.sleep(1000) + } // Verify whether data received was as expected logInfo("--------------------------------") - logInfo("output.size = " + outputBuffer.size) + logInfo("output.size = " + outputQueue.size) logInfo("output") - outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) + outputQueue.asScala.foreach(x => logInfo("[" + x.mkString(",") + "]")) logInfo("expected output.size = " + expectedOutput.size) logInfo("expected output") expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]")) @@ -322,9 +384,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Verify whether all the elements received are as expected assert(output.size === expectedOutput.size) - for (i <- 0 until output.size) { - assert(output(i) === expectedOutput(i)) - } + output.zipWithIndex.foreach{case (e, i) => assert(e == expectedOutput(i))} } test("test track the number of input stream") { @@ -356,13 +416,13 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } def testFileStream(newFilesOnly: Boolean) { - val testDir: File = null + var testDir: File = null try { val batchDuration = Seconds(2) - val testDir = Utils.createTempDir() + testDir = Utils.createTempDir() // Create a file that exists before the StreamingContext is created: val existingFile = new File(testDir, "0") - Files.write("0\n", existingFile, Charset.forName("UTF-8")) + Files.write("0\n", existingFile, StandardCharsets.UTF_8) assert(existingFile.setLastModified(10000) && existingFile.lastModified === 10000) // Set up the streaming context and input streams @@ -373,8 +433,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val batchCounter = new BatchCounter(ssc) val fileStream = ssc.fileStream[LongWritable, Text, TextInputFormat]( testDir.toString, (x: Path) => true, newFilesOnly = newFilesOnly).map(_._2.toString) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - val outputStream = new TestOutputStream(fileStream, outputBuffer) + val outputQueue = new ConcurrentLinkedQueue[Seq[String]] + val outputStream = new TestOutputStream(fileStream, outputQueue) outputStream.register() ssc.start() @@ -386,7 +446,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val input = Seq(1, 2, 3, 4, 5) input.foreach { i => val file = new File(testDir, i.toString) - Files.write(i + "\n", file, Charset.forName("UTF-8")) + Files.write(i + "\n", file, StandardCharsets.UTF_8) assert(file.setLastModified(clock.getTimeMillis())) assert(file.lastModified === clock.getTimeMillis()) logInfo("Created file " + file) @@ -404,7 +464,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } else { (Seq(0) ++ input).map(_.toString).toSet } - assert(outputBuffer.flatten.toSet === expectedOutput) + assert(outputQueue.asScala.flatten.toSet === expectedOutput) } } finally { if (testDir != null) Utils.deleteRecursively(testDir) @@ -441,7 +501,7 @@ class TestServer(portToBind: Int = 0) extends Logging { try { clientSocket.setTcpNoDelay(true) val outputStream = new BufferedWriter( - new OutputStreamWriter(clientSocket.getOutputStream)) + new OutputStreamWriter(clientSocket.getOutputStream, StandardCharsets.UTF_8)) while (clientSocket.isConnected) { val msg = queue.poll(100, TimeUnit.MILLISECONDS) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala new file mode 100644 index 000000000000..3b662ec1833a --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala @@ -0,0 +1,586 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +import java.io.File +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.collection.JavaConverters._ +import scala.reflect.ClassTag + +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.PrivateMethodTester._ + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.streaming.dstream.{DStream, InternalMapWithStateDStream, MapWithStateDStream, MapWithStateDStreamImpl} +import org.apache.spark.util.{ManualClock, Utils} + +class MapWithStateSuite extends SparkFunSuite + with DStreamCheckpointTester with BeforeAndAfterAll with BeforeAndAfter { + + private var sc: SparkContext = null + protected var checkpointDir: File = null + protected val batchDuration = Seconds(1) + + before { + StreamingContext.getActive().foreach { _.stop(stopSparkContext = false) } + checkpointDir = Utils.createTempDir("checkpoint") + } + + after { + StreamingContext.getActive().foreach { _.stop(stopSparkContext = false) } + if (checkpointDir != null) { + Utils.deleteRecursively(checkpointDir) + } + } + + override def beforeAll(): Unit = { + super.beforeAll() + val conf = new SparkConf().setMaster("local").setAppName("MapWithStateSuite") + conf.set("spark.streaming.clock", classOf[ManualClock].getName()) + sc = new SparkContext(conf) + } + + override def afterAll(): Unit = { + try { + if (sc != null) { + sc.stop() + } + } finally { + super.afterAll() + } + } + + test("state - get, exists, update, remove, ") { + var state: StateImpl[Int] = null + + def testState( + expectedData: Option[Int], + shouldBeUpdated: Boolean = false, + shouldBeRemoved: Boolean = false, + shouldBeTimingOut: Boolean = false + ): Unit = { + if (expectedData.isDefined) { + assert(state.exists) + assert(state.get() === expectedData.get) + assert(state.getOption() === expectedData) + assert(state.getOption.getOrElse(-1) === expectedData.get) + } else { + assert(!state.exists) + intercept[NoSuchElementException] { + state.get() + } + assert(state.getOption() === None) + assert(state.getOption.getOrElse(-1) === -1) + } + + assert(state.isTimingOut() === shouldBeTimingOut) + if (shouldBeTimingOut) { + intercept[IllegalArgumentException] { + state.remove() + } + intercept[IllegalArgumentException] { + state.update(-1) + } + } + + assert(state.isUpdated() === shouldBeUpdated) + + assert(state.isRemoved() === shouldBeRemoved) + if (shouldBeRemoved) { + intercept[IllegalArgumentException] { + state.remove() + } + intercept[IllegalArgumentException] { + state.update(-1) + } + } + } + + state = new StateImpl[Int]() + testState(None) + + state.wrap(None) + testState(None) + + state.wrap(Some(1)) + testState(Some(1)) + + state.update(2) + testState(Some(2), shouldBeUpdated = true) + + state = new StateImpl[Int]() + state.update(2) + testState(Some(2), shouldBeUpdated = true) + + state.remove() + testState(None, shouldBeRemoved = true) + + state.wrapTimingOutState(3) + testState(Some(3), shouldBeTimingOut = true) + } + + test("mapWithState - basic operations with simple API") { + val inputData = + Seq( + Seq(), + Seq("a"), + Seq("a", "b"), + Seq("a", "b", "c"), + Seq("a", "b"), + Seq("a"), + Seq() + ) + + val outputData = + Seq( + Seq(), + Seq(1), + Seq(2, 1), + Seq(3, 2, 1), + Seq(4, 3), + Seq(5), + Seq() + ) + + val stateData = + Seq( + Seq(), + Seq(("a", 1)), + Seq(("a", 2), ("b", 1)), + Seq(("a", 3), ("b", 2), ("c", 1)), + Seq(("a", 4), ("b", 3), ("c", 1)), + Seq(("a", 5), ("b", 3), ("c", 1)), + Seq(("a", 5), ("b", 3), ("c", 1)) + ) + + // state maintains running count, and updated count is returned + val mappingFunc = (key: String, value: Option[Int], state: State[Int]) => { + val sum = value.getOrElse(0) + state.getOption.getOrElse(0) + state.update(sum) + sum + } + + testOperation[String, Int, Int]( + inputData, StateSpec.function(mappingFunc), outputData, stateData) + } + + test("mapWithState - basic operations with advanced API") { + val inputData = + Seq( + Seq(), + Seq("a"), + Seq("a", "b"), + Seq("a", "b", "c"), + Seq("a", "b"), + Seq("a"), + Seq() + ) + + val outputData = + Seq( + Seq(), + Seq("aa"), + Seq("aa", "bb"), + Seq("aa", "bb", "cc"), + Seq("aa", "bb"), + Seq("aa"), + Seq() + ) + + val stateData = + Seq( + Seq(), + Seq(("a", 1)), + Seq(("a", 2), ("b", 1)), + Seq(("a", 3), ("b", 2), ("c", 1)), + Seq(("a", 4), ("b", 3), ("c", 1)), + Seq(("a", 5), ("b", 3), ("c", 1)), + Seq(("a", 5), ("b", 3), ("c", 1)) + ) + + // state maintains running count, key string doubled and returned + val mappingFunc = (batchTime: Time, key: String, value: Option[Int], state: State[Int]) => { + val sum = value.getOrElse(0) + state.getOption.getOrElse(0) + state.update(sum) + Some(key * 2) + } + + testOperation(inputData, StateSpec.function(mappingFunc), outputData, stateData) + } + + test("mapWithState - type inferencing and class tags") { + + // Simple track state function with value as Int, state as Double and mapped type as Double + val simpleFunc = (key: String, value: Option[Int], state: State[Double]) => { + 0L + } + + // Advanced track state function with key as String, value as Int, state as Double and + // mapped type as Double + val advancedFunc = (time: Time, key: String, value: Option[Int], state: State[Double]) => { + Some(0L) + } + + def testTypes(dstream: MapWithStateDStream[_, _, _, _]): Unit = { + val dstreamImpl = dstream.asInstanceOf[MapWithStateDStreamImpl[_, _, _, _]] + assert(dstreamImpl.keyClass === classOf[String]) + assert(dstreamImpl.valueClass === classOf[Int]) + assert(dstreamImpl.stateClass === classOf[Double]) + assert(dstreamImpl.mappedClass === classOf[Long]) + } + val ssc = new StreamingContext(sc, batchDuration) + val inputStream = new TestInputStream[(String, Int)](ssc, Seq.empty, numPartitions = 2) + + // Defining StateSpec inline with mapWithState and simple function implicitly gets the types + val simpleFunctionStateStream1 = inputStream.mapWithState( + StateSpec.function(simpleFunc).numPartitions(1)) + testTypes(simpleFunctionStateStream1) + + // Separately defining StateSpec with simple function requires explicitly specifying types + val simpleFuncSpec = StateSpec.function[String, Int, Double, Long](simpleFunc) + val simpleFunctionStateStream2 = inputStream.mapWithState(simpleFuncSpec) + testTypes(simpleFunctionStateStream2) + + // Separately defining StateSpec with advanced function implicitly gets the types + val advFuncSpec1 = StateSpec.function(advancedFunc) + val advFunctionStateStream1 = inputStream.mapWithState(advFuncSpec1) + testTypes(advFunctionStateStream1) + + // Defining StateSpec inline with mapWithState and advanced func implicitly gets the types + val advFunctionStateStream2 = inputStream.mapWithState( + StateSpec.function(simpleFunc).numPartitions(1)) + testTypes(advFunctionStateStream2) + + // Defining StateSpec inline with mapWithState and advanced func implicitly gets the types + val advFuncSpec2 = StateSpec.function[String, Int, Double, Long](advancedFunc) + val advFunctionStateStream3 = inputStream.mapWithState[Double, Long](advFuncSpec2) + testTypes(advFunctionStateStream3) + } + + test("mapWithState - states as mapped data") { + val inputData = + Seq( + Seq(), + Seq("a"), + Seq("a", "b"), + Seq("a", "b", "c"), + Seq("a", "b"), + Seq("a"), + Seq() + ) + + val outputData = + Seq( + Seq(), + Seq(("a", 1)), + Seq(("a", 2), ("b", 1)), + Seq(("a", 3), ("b", 2), ("c", 1)), + Seq(("a", 4), ("b", 3)), + Seq(("a", 5)), + Seq() + ) + + val stateData = + Seq( + Seq(), + Seq(("a", 1)), + Seq(("a", 2), ("b", 1)), + Seq(("a", 3), ("b", 2), ("c", 1)), + Seq(("a", 4), ("b", 3), ("c", 1)), + Seq(("a", 5), ("b", 3), ("c", 1)), + Seq(("a", 5), ("b", 3), ("c", 1)) + ) + + val mappingFunc = (time: Time, key: String, value: Option[Int], state: State[Int]) => { + val sum = value.getOrElse(0) + state.getOption.getOrElse(0) + val output = (key, sum) + state.update(sum) + Some(output) + } + + testOperation(inputData, StateSpec.function(mappingFunc), outputData, stateData) + } + + test("mapWithState - initial states, with nothing returned as from mapping function") { + + val initialState = Seq(("a", 5), ("b", 10), ("c", -20), ("d", 0)) + + val inputData = + Seq( + Seq(), + Seq("a"), + Seq("a", "b"), + Seq("a", "b", "c"), + Seq("a", "b"), + Seq("a"), + Seq() + ) + + val outputData = Seq.fill(inputData.size)(Seq.empty[Int]) + + val stateData = + Seq( + Seq(("a", 5), ("b", 10), ("c", -20), ("d", 0)), + Seq(("a", 6), ("b", 10), ("c", -20), ("d", 0)), + Seq(("a", 7), ("b", 11), ("c", -20), ("d", 0)), + Seq(("a", 8), ("b", 12), ("c", -19), ("d", 0)), + Seq(("a", 9), ("b", 13), ("c", -19), ("d", 0)), + Seq(("a", 10), ("b", 13), ("c", -19), ("d", 0)), + Seq(("a", 10), ("b", 13), ("c", -19), ("d", 0)) + ) + + val mappingFunc = (time: Time, key: String, value: Option[Int], state: State[Int]) => { + val sum = value.getOrElse(0) + state.getOption.getOrElse(0) + val output = (key, sum) + state.update(sum) + None.asInstanceOf[Option[Int]] + } + + val mapWithStateSpec = StateSpec.function(mappingFunc).initialState(sc.makeRDD(initialState)) + testOperation(inputData, mapWithStateSpec, outputData, stateData) + } + + test("mapWithState - state removing") { + val inputData = + Seq( + Seq(), + Seq("a"), + Seq("a", "b"), // a will be removed + Seq("a", "b", "c"), // b will be removed + Seq("a", "b", "c"), // a and c will be removed + Seq("a", "b"), // b will be removed + Seq("a"), // a will be removed + Seq() + ) + + // States that were removed + val outputData = + Seq( + Seq(), + Seq(), + Seq("a"), + Seq("b"), + Seq("a", "c"), + Seq("b"), + Seq("a"), + Seq() + ) + + val stateData = + Seq( + Seq(), + Seq(("a", 1)), + Seq(("b", 1)), + Seq(("a", 1), ("c", 1)), + Seq(("b", 1)), + Seq(("a", 1)), + Seq(), + Seq() + ) + + val mappingFunc = (time: Time, key: String, value: Option[Int], state: State[Int]) => { + if (state.exists) { + state.remove() + Some(key) + } else { + state.update(value.get) + None + } + } + + testOperation( + inputData, StateSpec.function(mappingFunc).numPartitions(1), outputData, stateData) + } + + test("mapWithState - state timing out") { + val inputData = + Seq( + Seq("a", "b", "c"), + Seq("a", "b"), + Seq("a"), + Seq(), // c will time out + Seq(), // b will time out + Seq("a") // a will not time out + ) ++ Seq.fill(20)(Seq("a")) // a will continue to stay active + + val mappingFunc = (time: Time, key: String, value: Option[Int], state: State[Int]) => { + if (value.isDefined) { + state.update(1) + } + if (state.isTimingOut) { + Some(key) + } else { + None + } + } + + val (collectedOutputs, collectedStateSnapshots) = getOperationOutput( + inputData, StateSpec.function(mappingFunc).timeout(Seconds(3)), 20) + + // b and c should be returned once each, when they were marked as expired + assert(collectedOutputs.flatten.sorted === Seq("b", "c")) + + // States for a, b, c should be defined at one point of time + assert(collectedStateSnapshots.exists { + _.toSet == Set(("a", 1), ("b", 1), ("c", 1)) + }) + + // Finally state should be defined only for a + assert(collectedStateSnapshots.last.toSet === Set(("a", 1))) + } + + test("mapWithState - checkpoint durations") { + val privateMethod = PrivateMethod[InternalMapWithStateDStream[_, _, _, _]]('internalStream) + + def testCheckpointDuration( + batchDuration: Duration, + expectedCheckpointDuration: Duration, + explicitCheckpointDuration: Option[Duration] = None + ): Unit = { + val ssc = new StreamingContext(sc, batchDuration) + + try { + val inputStream = new TestInputStream(ssc, Seq.empty[Seq[Int]], 2).map(_ -> 1) + val dummyFunc = (key: Int, value: Option[Int], state: State[Int]) => 0 + val mapWithStateStream = inputStream.mapWithState(StateSpec.function(dummyFunc)) + val internalmapWithStateStream = mapWithStateStream invokePrivate privateMethod() + + explicitCheckpointDuration.foreach { d => + mapWithStateStream.checkpoint(d) + } + mapWithStateStream.register() + ssc.checkpoint(checkpointDir.toString) + ssc.start() // should initialize all the checkpoint durations + assert(mapWithStateStream.checkpointDuration === null) + assert(internalmapWithStateStream.checkpointDuration === expectedCheckpointDuration) + } finally { + ssc.stop(stopSparkContext = false) + } + } + + testCheckpointDuration(Milliseconds(100), Seconds(1)) + testCheckpointDuration(Seconds(1), Seconds(10)) + testCheckpointDuration(Seconds(10), Seconds(100)) + + testCheckpointDuration(Milliseconds(100), Seconds(2), Some(Seconds(2))) + testCheckpointDuration(Seconds(1), Seconds(2), Some(Seconds(2))) + testCheckpointDuration(Seconds(10), Seconds(20), Some(Seconds(20))) + } + + + test("mapWithState - driver failure recovery") { + val inputData = + Seq( + Seq(), + Seq("a"), + Seq("a", "b"), + Seq("a", "b", "c"), + Seq("a", "b"), + Seq("a"), + Seq() + ) + + val stateData = + Seq( + Seq(), + Seq(("a", 1)), + Seq(("a", 2), ("b", 1)), + Seq(("a", 3), ("b", 2), ("c", 1)), + Seq(("a", 4), ("b", 3), ("c", 1)), + Seq(("a", 5), ("b", 3), ("c", 1)), + Seq(("a", 5), ("b", 3), ("c", 1)) + ) + + def operation(dstream: DStream[String]): DStream[(String, Int)] = { + + val checkpointDuration = batchDuration * (stateData.size / 2) + + val runningCount = (key: String, value: Option[Int], state: State[Int]) => { + state.update(state.getOption().getOrElse(0) + value.getOrElse(0)) + state.get() + } + + val mapWithStateStream = dstream.map { _ -> 1 }.mapWithState( + StateSpec.function(runningCount)) + // Set interval make sure there is one RDD checkpointing + mapWithStateStream.checkpoint(checkpointDuration) + mapWithStateStream.stateSnapshots() + } + + testCheckpointedOperation(inputData, operation, stateData, inputData.size / 2, + batchDuration = batchDuration, stopSparkContextAfterTest = false) + } + + private def testOperation[K: ClassTag, S: ClassTag, T: ClassTag]( + input: Seq[Seq[K]], + mapWithStateSpec: StateSpec[K, Int, S, T], + expectedOutputs: Seq[Seq[T]], + expectedStateSnapshots: Seq[Seq[(K, S)]] + ): Unit = { + require(expectedOutputs.size == expectedStateSnapshots.size) + + val (collectedOutputs, collectedStateSnapshots) = + getOperationOutput(input, mapWithStateSpec, expectedOutputs.size) + assert(expectedOutputs, collectedOutputs, "outputs") + assert(expectedStateSnapshots, collectedStateSnapshots, "state snapshots") + } + + private def getOperationOutput[K: ClassTag, S: ClassTag, T: ClassTag]( + input: Seq[Seq[K]], + mapWithStateSpec: StateSpec[K, Int, S, T], + numBatches: Int + ): (Seq[Seq[T]], Seq[Seq[(K, S)]]) = { + + // Setup the stream computation + val ssc = new StreamingContext(sc, Seconds(1)) + val inputStream = new TestInputStream(ssc, input, numPartitions = 2) + val trackeStateStream = inputStream.map(x => (x, 1)).mapWithState(mapWithStateSpec) + val collectedOutputs = new ConcurrentLinkedQueue[Seq[T]] + val outputStream = new TestOutputStream(trackeStateStream, collectedOutputs) + val collectedStateSnapshots = new ConcurrentLinkedQueue[Seq[(K, S)]] + val stateSnapshotStream = new TestOutputStream( + trackeStateStream.stateSnapshots(), collectedStateSnapshots) + outputStream.register() + stateSnapshotStream.register() + + val batchCounter = new BatchCounter(ssc) + ssc.checkpoint(checkpointDir.toString) + ssc.start() + + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + clock.advance(batchDuration.milliseconds * numBatches) + + batchCounter.waitUntilBatchesCompleted(numBatches, 10000) + ssc.stop(stopSparkContext = false) + (collectedOutputs.asScala.toSeq, collectedStateSnapshots.asScala.toSeq) + } + + private def assert[U](expected: Seq[Seq[U]], collected: Seq[Seq[U]], typ: String) { + val debugString = "\nExpected:\n" + expected.mkString("\n") + + "\nCollected:\n" + collected.mkString("\n") + assert(expected.size === collected.size, + s"number of collected $typ (${collected.size}) different from expected (${expected.size})" + + debugString) + expected.zip(collected).foreach { case (c, e) => + assert(c.toSet === e.toSet, + s"collected $typ is different from expected $debugString" + ) + } + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala index 0e64b57e0ffd..fff2d6fbace3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -17,23 +17,22 @@ package org.apache.spark.streaming -import org.apache.spark.Logging -import org.apache.spark.streaming.dstream.DStream -import org.apache.spark.util.Utils +import java.io.{File, IOException} +import java.nio.charset.StandardCharsets +import java.util.UUID -import scala.util.Random +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag - -import java.io.{File, IOException} -import java.nio.charset.Charset -import java.util.UUID +import scala.util.Random import com.google.common.io.Files - -import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.spark.internal.Logging +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.util.Utils private[streaming] object MasterFailureTest extends Logging { @@ -165,6 +164,7 @@ object MasterFailureTest extends Logging { val mergedOutput = runStreams(ssc, lastExpectedOutput, maxTimeToRun) fileGeneratingThread.join() + ssc.stop() fs.delete(checkpointDir, true) fs.delete(testDir, true) logInfo("Finished test after " + killCount + " failures") @@ -202,12 +202,12 @@ object MasterFailureTest extends Logging { * the last expected output is generated. Finally, return */ private def runStreams[T: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, lastExpectedOutput: T, maxTimeToRun: Long ): Seq[T] = { - var ssc = ssc_ + var ssc = _ssc var totalTimeRan = 0L var isLastOutputGenerated = false var isTimedOut = false @@ -217,8 +217,8 @@ object MasterFailureTest extends Logging { while(!isLastOutputGenerated && !isTimedOut) { // Get the output buffer - val outputBuffer = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[T]].output - def output = outputBuffer.flatMap(x => x) + val outputQueue = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[T]].output + def output = outputQueue.asScala.flatten // Start the thread to kill the streaming after some time killed = false @@ -243,6 +243,8 @@ object MasterFailureTest extends Logging { } } catch { case e: Exception => logError("Error running streaming context", e) + } finally { + ssc.stop() } if (killingThread.isAlive) { killingThread.interrupt() @@ -251,7 +253,6 @@ object MasterFailureTest extends Logging { // to null after the next test creates the new SparkContext and fail the test. killingThread.join() } - ssc.stop() logInfo("Has been killed = " + killed) logInfo("Is last output generated = " + isLastOutputGenerated) @@ -259,9 +260,9 @@ object MasterFailureTest extends Logging { // Verify whether the output of each batch has only one element or no element // and then merge the new output with all the earlier output - mergedOutput ++= output + mergedOutput ++= output.toSeq totalTimeRan += timeRan - logInfo("New output = " + output) + logInfo("New output = " + output.toSeq) logInfo("Merged output = " + mergedOutput) logInfo("Time ran = " + timeRan) logInfo("Total time ran = " + totalTimeRan) @@ -371,7 +372,7 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) val localFile = new File(localTestDir, (i + 1).toString) val hadoopFile = new Path(testDir, (i + 1).toString) val tempHadoopFile = new Path(testDir, ".tmp_" + (i + 1).toString) - Files.write(input(i) + "\n", localFile, Charset.forName("UTF-8")) + Files.write(input(i) + "\n", localFile, StandardCharsets.UTF_8) var tries = 0 var done = false while (!done && tries < maxTries) { @@ -382,11 +383,10 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) fs.rename(tempHadoopFile, hadoopFile) done = true } catch { - case ioe: IOException => { + case ioe: IOException => fs = testDir.getFileSystem(new Configuration()) logWarning("Attempt " + tries + " at generating file " + hadoopFile + " failed.", ioe) - } } } if (!done) { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index c17fb7238151..fe65353b9d50 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -23,40 +23,57 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.postfixOps +import scala.reflect.ClassTag import org.apache.hadoop.conf.Configuration import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually._ import org.apache.spark._ +import org.apache.spark.broadcast.BroadcastManager +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ import org.apache.spark.memory.StaticMemoryManager import org.apache.spark.network.netty.NettyBlockTransferService import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.LiveListenerBus -import org.apache.spark.serializer.KryoSerializer -import org.apache.spark.shuffle.hash.HashShuffleManager +import org.apache.spark.security.CryptoStreamUtils +import org.apache.spark.serializer.{KryoSerializer, SerializerManager} +import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.storage._ import org.apache.spark.streaming.receiver._ import org.apache.spark.streaming.util._ import org.apache.spark.util.{ManualClock, Utils} -import WriteAheadLogBasedBlockHandler._ -import WriteAheadLogSuite._ +import org.apache.spark.util.io.ChunkedByteBuffer -class ReceivedBlockHandlerSuite +abstract class BaseReceivedBlockHandlerSuite(enableEncryption: Boolean) extends SparkFunSuite with BeforeAndAfter with Matchers with Logging { + import WriteAheadLogBasedBlockHandler._ + import WriteAheadLogSuite._ + val conf = new SparkConf() .set("spark.streaming.receiver.writeAheadLog.rollingIntervalSecs", "1") .set("spark.app.id", "streaming-test") + .set(IO_ENCRYPTION_ENABLED, enableEncryption) + val encryptionKey = + if (enableEncryption) { + Some(CryptoStreamUtils.createKey(conf)) + } else { + None + } + val hadoopConf = new Configuration() val streamId = 1 - val securityMgr = new SecurityManager(conf) - val mapOutputTracker = new MapOutputTrackerMaster(conf) - val shuffleManager = new HashShuffleManager(conf) + val securityMgr = new SecurityManager(conf, encryptionKey) + val broadcastManager = new BroadcastManager(true, conf, securityMgr) + val mapOutputTracker = new MapOutputTrackerMaster(conf, broadcastManager, true) + val shuffleManager = new SortShuffleManager(conf) val serializer = new KryoSerializer(conf) + var serializerManager = new SerializerManager(serializer, conf, encryptionKey) val manualClock = new ManualClock val blockManagerSize = 10000000 val blockManagerBuffer = new ArrayBuffer[BlockManager]() @@ -72,7 +89,8 @@ class ReceivedBlockHandlerSuite conf.set("spark.driver.port", rpcEnv.address.port.toString) blockManagerMaster = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", - new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus)), conf, true) + new BlockManagerMasterEndpoint(rpcEnv, true, conf, + new LiveListenerBus(conf))), conf, true) storageLevel = StorageLevel.MEMORY_ONLY_SER blockManager = createBlockManager(blockManagerSize, conf) @@ -105,7 +123,10 @@ class ReceivedBlockHandlerSuite testBlockStoring(handler) { case (data, blockIds, storeResults) => // Verify the data in block manager is correct val storedData = blockIds.flatMap { blockId => - blockManager.getLocal(blockId).map(_.data.map(_.toString).toList).getOrElse(List.empty) + blockManager + .getLocalValues(blockId) + .map(_.data.map(_.toString).toList) + .getOrElse(List.empty) }.toList storedData shouldEqual data @@ -129,7 +150,10 @@ class ReceivedBlockHandlerSuite testBlockStoring(handler) { case (data, blockIds, storeResults) => // Verify the data in block manager is correct val storedData = blockIds.flatMap { blockId => - blockManager.getLocal(blockId).map(_.data.map(_.toString).toList).getOrElse(List.empty) + blockManager + .getLocalValues(blockId) + .map(_.data.map(_.toString).toList) + .getOrElse(List.empty) }.toList storedData shouldEqual data @@ -147,7 +171,9 @@ class ReceivedBlockHandlerSuite val reader = new FileBasedWriteAheadLogRandomReader(fileSegment.path, hadoopConf) val bytes = reader.read(fileSegment) reader.close() - blockManager.dataDeserialize(generateBlockId(), bytes).toList + serializerManager.dataDeserializeStream( + generateBlockId(), + new ChunkedByteBuffer(bytes).toInputStream())(ClassTag.Any).toList } loggedData shouldEqual data } @@ -191,17 +217,19 @@ class ReceivedBlockHandlerSuite sparkConf.set("spark.storage.unrollMemoryThreshold", "512") // spark.storage.unrollFraction set to 0.4 for BlockManager sparkConf.set("spark.storage.unrollFraction", "0.4") + + sparkConf.set(IO_ENCRYPTION_ENABLED, enableEncryption) // Block Manager with 12000 * 0.4 = 4800 bytes of free space for unroll blockManager = createBlockManager(12000, sparkConf) // there is not enough space to store this block in MEMORY, - // But BlockManager will be able to sereliaze this block to WAL + // But BlockManager will be able to serialize this block to WAL // and hence count returns correct value. testRecordcount(false, StorageLevel.MEMORY_ONLY, IteratorBlock((List.fill(70)(new Array[Byte](100))).iterator), blockManager, Some(70)) // there is not enough space to store this block in MEMORY, - // But BlockManager will be able to sereliaze this block to DISK + // But BlockManager will be able to serialize this block to DISK // and hence count returns correct value. testRecordcount(true, StorageLevel.MEMORY_AND_DISK, IteratorBlock((List.fill(70)(new Array[Byte](100))).iterator), blockManager, Some(70)) @@ -255,8 +283,8 @@ class ReceivedBlockHandlerSuite conf: SparkConf, name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { val memManager = new StaticMemoryManager(conf, Long.MaxValue, maxMem, numCores = 1) - val transfer = new NettyBlockTransferService(conf, securityMgr, numCores = 1) - val blockManager = new BlockManager(name, rpcEnv, blockManagerMaster, serializer, conf, + val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1) + val blockManager = new BlockManager(name, rpcEnv, blockManagerMaster, serializerManager, conf, memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) memManager.setMemoryStore(blockManager.memoryStore) blockManager.initialize("app-id") @@ -265,7 +293,7 @@ class ReceivedBlockHandlerSuite } /** - * Test storing of data using different types of Handler, StorageLevle and ReceivedBlocks + * Test storing of data using different types of Handler, StorageLevel and ReceivedBlocks * and verify the correct record count */ private def testRecordcount(isBlockManagedBasedBlockHandler: Boolean, @@ -325,13 +353,14 @@ class ReceivedBlockHandlerSuite } } - def dataToByteBuffer(b: Seq[String]) = blockManager.dataSerialize(generateBlockId, b.iterator) + def dataToByteBuffer(b: Seq[String]) = + serializerManager.dataSerialize(generateBlockId, b.iterator) val blocks = data.grouped(10).toSeq storeAndVerify(blocks.map { b => IteratorBlock(b.toIterator) }) storeAndVerify(blocks.map { b => ArrayBufferBlock(new ArrayBuffer ++= b) }) - storeAndVerify(blocks.map { b => ByteBufferBlock(dataToByteBuffer(b)) }) + storeAndVerify(blocks.map { b => ByteBufferBlock(dataToByteBuffer(b).toByteBuffer) }) } /** Test error handling when blocks that cannot be stored */ @@ -357,8 +386,8 @@ class ReceivedBlockHandlerSuite /** Instantiate a WriteAheadLogBasedBlockHandler and run a code with it */ private def withWriteAheadLogBasedBlockHandler(body: WriteAheadLogBasedBlockHandler => Unit) { require(WriteAheadLogUtils.getRollingIntervalSecs(conf, isDriver = false) === 1) - val receivedBlockHandler = new WriteAheadLogBasedBlockHandler(blockManager, 1, - storageLevel, conf, hadoopConf, tempDirectory.toString, manualClock) + val receivedBlockHandler = new WriteAheadLogBasedBlockHandler(blockManager, serializerManager, + 1, storageLevel, conf, hadoopConf, tempDirectory.toString, manualClock) try { body(receivedBlockHandler) } finally { @@ -400,3 +429,6 @@ class ReceivedBlockHandlerSuite private def generateBlockId(): StreamBlockId = StreamBlockId(streamId, scala.util.Random.nextLong) } +class ReceivedBlockHandlerSuite extends BaseReceivedBlockHandlerSuite(false) + +class ReceivedBlockHandlerWithEncryptionSuite extends BaseReceivedBlockHandlerSuite(true) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index f793a12843b2..107c3f5dcc08 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming import java.io.File +import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ @@ -28,11 +29,12 @@ import org.apache.hadoop.conf.Configuration import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually._ -import org.apache.spark.{Logging, SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.receiver.BlockManagerBasedStoreResult import org.apache.spark.streaming.scheduler._ -import org.apache.spark.streaming.util.{WriteAheadLogUtils, FileBasedWriteAheadLogReader} +import org.apache.spark.streaming.util._ import org.apache.spark.streaming.util.WriteAheadLogSuite._ import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} @@ -40,7 +42,6 @@ class ReceivedBlockTrackerSuite extends SparkFunSuite with BeforeAndAfter with Matchers with Logging { val hadoopConf = new Configuration() - val akkaTimeout = 10 seconds val streamId = 1 var allReceivedBlockTrackers = new ArrayBuffer[ReceivedBlockTracker]() @@ -133,6 +134,7 @@ class ReceivedBlockTrackerSuite val expectedWrittenData1 = blockInfos1.map(BlockAdditionEvent) getWrittenLogData() shouldEqual expectedWrittenData1 getWriteAheadLogFiles() should have size 1 + tracker1.stop() incrementTime() @@ -140,6 +142,7 @@ class ReceivedBlockTrackerSuite val tracker1_ = createTracker(clock = manualClock, recoverFromWriteAheadLog = false) tracker1_.getUnallocatedBlocks(streamId) shouldBe empty tracker1_.hasUnallocatedReceivedBlocks should be (false) + tracker1_.stop() // Restart tracker and verify recovered list of unallocated blocks val tracker2 = createTracker(clock = manualClock, recoverFromWriteAheadLog = true) @@ -162,6 +165,7 @@ class ReceivedBlockTrackerSuite val blockInfos2 = addBlockInfos(tracker2) tracker2.allocateBlocksToBatch(batchTime2) tracker2.getBlocksOfBatchAndStream(batchTime2, streamId) shouldEqual blockInfos2 + tracker2.stop() // Verify whether log has correct contents val expectedWrittenData2 = expectedWrittenData1 ++ @@ -191,6 +195,7 @@ class ReceivedBlockTrackerSuite getWriteAheadLogFiles() should not contain oldestLogFile } printLogFiles("After clean") + tracker3.stop() // Restart tracker and verify recovered state, specifically whether info about the first // batch has been removed, but not the second batch @@ -199,6 +204,7 @@ class ReceivedBlockTrackerSuite tracker4.getUnallocatedBlocks(streamId) shouldBe empty tracker4.getBlocksOfBatchAndStream(batchTime1, streamId) shouldBe empty // should be cleaned tracker4.getBlocksOfBatchAndStream(batchTime2, streamId) shouldEqual blockInfos2 + tracker4.stop() } test("disable write ahead log when checkpoint directory is not set") { @@ -207,6 +213,75 @@ class ReceivedBlockTrackerSuite tracker1.isWriteAheadLogEnabled should be (false) } + test("parallel file deletion in FileBasedWriteAheadLog is robust to deletion error") { + conf.set("spark.streaming.driver.writeAheadLog.rollingIntervalSecs", "1") + require(WriteAheadLogUtils.getRollingIntervalSecs(conf, isDriver = true) === 1) + + val addBlocks = generateBlockInfos() + val batch1 = addBlocks.slice(0, 1) + val batch2 = addBlocks.slice(1, 3) + val batch3 = addBlocks.slice(3, addBlocks.length) + + assert(getWriteAheadLogFiles().length === 0) + + // list of timestamps for files + val t = Seq.tabulate(5)(i => i * 1000) + + writeEventsManually(getLogFileName(t(0)), Seq(createBatchCleanup(t(0)))) + assert(getWriteAheadLogFiles().length === 1) + + // The goal is to create several log files which should have been cleaned up. + // If we face any issue during recovery, because these old files exist, then we need to make + // deletion more robust rather than a parallelized operation where we fire and forget + val batch1Allocation = createBatchAllocation(t(1), batch1) + writeEventsManually(getLogFileName(t(1)), batch1.map(BlockAdditionEvent) :+ batch1Allocation) + + writeEventsManually(getLogFileName(t(2)), Seq(createBatchCleanup(t(1)))) + + val batch2Allocation = createBatchAllocation(t(3), batch2) + writeEventsManually(getLogFileName(t(3)), batch2.map(BlockAdditionEvent) :+ batch2Allocation) + + writeEventsManually(getLogFileName(t(4)), batch3.map(BlockAdditionEvent)) + + // We should have 5 different log files as we called `writeEventsManually` with 5 different + // timestamps + assert(getWriteAheadLogFiles().length === 5) + + // Create the tracker to recover from the log files. We're going to ask the tracker to clean + // things up, and then we're going to rewrite that data, and recover using a different tracker. + // They should have identical data no matter what + val tracker = createTracker(recoverFromWriteAheadLog = true, clock = new ManualClock(t(4))) + + def compareTrackers(base: ReceivedBlockTracker, subject: ReceivedBlockTracker): Unit = { + subject.getBlocksOfBatchAndStream(t(3), streamId) should be( + base.getBlocksOfBatchAndStream(t(3), streamId)) + subject.getBlocksOfBatchAndStream(t(1), streamId) should be( + base.getBlocksOfBatchAndStream(t(1), streamId)) + subject.getBlocksOfBatchAndStream(t(0), streamId) should be(Nil) + } + + // ask the tracker to clean up some old files + tracker.cleanupOldBatches(t(3), waitForCompletion = true) + assert(getWriteAheadLogFiles().length === 3) + + val tracker2 = createTracker(recoverFromWriteAheadLog = true, clock = new ManualClock(t(4))) + compareTrackers(tracker, tracker2) + + // rewrite first file + writeEventsManually(getLogFileName(t(0)), Seq(createBatchCleanup(t(0)))) + assert(getWriteAheadLogFiles().length === 4) + // make sure trackers are consistent + val tracker3 = createTracker(recoverFromWriteAheadLog = true, clock = new ManualClock(t(4))) + compareTrackers(tracker, tracker3) + + // rewrite second file + writeEventsManually(getLogFileName(t(1)), batch1.map(BlockAdditionEvent) :+ batch1Allocation) + assert(getWriteAheadLogFiles().length === 5) + // make sure trackers are consistent + val tracker4 = createTracker(recoverFromWriteAheadLog = true, clock = new ManualClock(t(4))) + compareTrackers(tracker, tracker4) + } + /** * Create tracker object with the optional provided clock. Use fake clock if you * want to control time by manually incrementing it to test log clean. @@ -228,11 +303,30 @@ class ReceivedBlockTrackerSuite BlockManagerBasedStoreResult(StreamBlockId(streamId, math.abs(Random.nextInt)), Some(0L)))) } + /** + * Write received block tracker events to a file manually. + */ + def writeEventsManually(filePath: String, events: Seq[ReceivedBlockTrackerLogEvent]): Unit = { + val writer = HdfsUtils.getOutputStream(filePath, hadoopConf) + events.foreach { event => + val bytes = Utils.serialize(event) + writer.writeInt(bytes.size) + writer.write(bytes) + } + writer.close() + } + /** Get all the data written in the given write ahead log file. */ def getWrittenLogData(logFile: String): Seq[ReceivedBlockTrackerLogEvent] = { getWrittenLogData(Seq(logFile)) } + /** Get the log file name for the given log start time. */ + def getLogFileName(time: Long, rollingIntervalSecs: Int = 1): String = { + checkpointDirectory.toString + File.separator + "receivedBlockMetadata" + + File.separator + s"log-$time-${time + rollingIntervalSecs * 1000}" + } + /** * Get all the data written in the given write ahead log files. By default, it will read all * files in the test log directory. @@ -241,8 +335,13 @@ class ReceivedBlockTrackerSuite : Seq[ReceivedBlockTrackerLogEvent] = { logFiles.flatMap { file => new FileBasedWriteAheadLogReader(file, hadoopConf).toSeq - }.map { byteBuffer => - Utils.deserialize[ReceivedBlockTrackerLogEvent](byteBuffer.array) + }.flatMap { byteBuffer => + val validBuffer = if (WriteAheadLogUtils.isBatchingEnabled(conf, isDriver = true)) { + Utils.deserialize[Array[Array[Byte]]](byteBuffer.array()).map(ByteBuffer.wrap) + } else { + Array(byteBuffer) + } + validBuffer.map(b => Utils.deserialize[ReceivedBlockTrackerLogEvent](b.array())) }.toList } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala index 6d388d9624d9..0349e11224cf 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala @@ -21,6 +21,7 @@ import scala.util.Random import org.scalatest.BeforeAndAfterAll +import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.rdd.BlockRDD import org.apache.spark.storage.{StorageLevel, StreamBlockId} import org.apache.spark.streaming.dstream.ReceiverInputDStream @@ -28,12 +29,15 @@ import org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD import org.apache.spark.streaming.receiver.{BlockManagerBasedStoreResult, Receiver, WriteAheadLogBasedStoreResult} import org.apache.spark.streaming.scheduler.ReceivedBlockInfo import org.apache.spark.streaming.util.{WriteAheadLogRecordHandle, WriteAheadLogUtils} -import org.apache.spark.{SparkConf, SparkEnv} class ReceiverInputDStreamSuite extends TestSuiteBase with BeforeAndAfterAll { override def afterAll(): Unit = { - StreamingContext.getActive().map { _.stop() } + try { + StreamingContext.getActive().foreach(_.stop()) + } finally { + super.afterAll() + } } testWithoutWAL("createBlockRDD creates empty BlockRDD when no block info") { receiverStream => @@ -93,7 +97,7 @@ class ReceiverInputDStreamSuite extends TestSuiteBase with BeforeAndAfterAll { assert(blockRDD.walRecordHandles.toSeq === blockInfos.map { _.walRecordHandleOption.get }) } - testWithWAL("createBlockRDD creates BlockRDD when some block info dont have WAL info") { + testWithWAL("createBlockRDD creates BlockRDD when some block info don't have WAL info") { receiverStream => val blockInfos1 = Seq.fill(2) { createBlockInfo(withWALInfo = true) } val blockInfos2 = Seq.fill(3) { createBlockInfo(withWALInfo = false) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index 01279b34f73d..5fc626c1f78b 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -24,7 +24,7 @@ import java.util.concurrent.Semaphore import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.scalatest.concurrent.Timeouts +import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ @@ -36,7 +36,7 @@ import org.apache.spark.streaming.receiver.WriteAheadLogBasedBlockHandler._ import org.apache.spark.util.Utils /** Testsuite for testing the network receiver behavior */ -class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { +class ReceiverSuite extends TestSuiteBase with TimeLimits with Serializable { test("receiver life cycle") { @@ -60,6 +60,8 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { // Verify that the receiver intercept[Exception] { + // Necessary to make failAfter interrupt awaitTermination() in ScalaTest 3.x + implicit val signaler: Signaler = ThreadSignaler failAfter(200 millis) { executingThread.join() } @@ -215,7 +217,7 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { def getCurrentLogFiles(logDirectory: File): Seq[String] = { try { if (logDirectory.exists()) { - logDirectory1.listFiles().filter { _.getName.startsWith("log") }.map { _.toString } + logDirectory.listFiles().filter { _.getName.startsWith("log") }.map { _.toString } } else { Seq.empty } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala new file mode 100644 index 000000000000..484f3733e842 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala @@ -0,0 +1,393 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +import scala.collection.{immutable, mutable, Map} +import scala.reflect.ClassTag +import scala.util.Random + +import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.esotericsoftware.kryo.io.{Input, Output} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.serializer._ +import org.apache.spark.streaming.rdd.MapWithStateRDDRecord +import org.apache.spark.streaming.util.{EmptyStateMap, OpenHashMapBasedStateMap, StateMap} + +class StateMapSuite extends SparkFunSuite { + + private val conf = new SparkConf() + + test("EmptyStateMap") { + val map = new EmptyStateMap[Int, Int] + intercept[scala.NotImplementedError] { + map.put(1, 1, 1) + } + assert(map.get(1) === None) + assert(map.getByTime(10000).isEmpty) + assert(map.getAll().isEmpty) + map.remove(1) // no exception + assert(map.copy().eq(map)) + } + + test("OpenHashMapBasedStateMap - put, get, getByTime, getAll, remove") { + val map = new OpenHashMapBasedStateMap[Int, Int]() + + map.put(1, 100, 10) + assert(map.get(1) === Some(100)) + assert(map.get(2) === None) + assert(map.getByTime(11).toSet === Set((1, 100, 10))) + assert(map.getByTime(10).toSet === Set.empty) + assert(map.getByTime(9).toSet === Set.empty) + assert(map.getAll().toSet === Set((1, 100, 10))) + + map.put(2, 200, 20) + assert(map.getByTime(21).toSet === Set((1, 100, 10), (2, 200, 20))) + assert(map.getByTime(11).toSet === Set((1, 100, 10))) + assert(map.getByTime(10).toSet === Set.empty) + assert(map.getByTime(9).toSet === Set.empty) + assert(map.getAll().toSet === Set((1, 100, 10), (2, 200, 20))) + + map.remove(1) + assert(map.get(1) === None) + assert(map.getAll().toSet === Set((2, 200, 20))) + } + + test("OpenHashMapBasedStateMap - put, get, getByTime, getAll, remove with copy") { + val parentMap = new OpenHashMapBasedStateMap[Int, Int]() + parentMap.put(1, 100, 1) + parentMap.put(2, 200, 2) + parentMap.remove(1) + + // Create child map and make changes + val map = parentMap.copy() + assert(map.get(1) === None) + assert(map.get(2) === Some(200)) + assert(map.getByTime(10).toSet === Set((2, 200, 2))) + assert(map.getByTime(2).toSet === Set.empty) + assert(map.getAll().toSet === Set((2, 200, 2))) + + // Add new items + map.put(3, 300, 3) + assert(map.get(3) === Some(300)) + map.put(4, 400, 4) + assert(map.get(4) === Some(400)) + assert(map.getByTime(10).toSet === Set((2, 200, 2), (3, 300, 3), (4, 400, 4))) + assert(map.getByTime(4).toSet === Set((2, 200, 2), (3, 300, 3))) + assert(map.getAll().toSet === Set((2, 200, 2), (3, 300, 3), (4, 400, 4))) + assert(parentMap.getAll().toSet === Set((2, 200, 2))) + + // Remove items + map.remove(4) + assert(map.get(4) === None) // item added in this map, then removed in this map + map.remove(2) + assert(map.get(2) === None) // item removed in parent map, then added in this map + assert(map.getAll().toSet === Set((3, 300, 3))) + assert(parentMap.getAll().toSet === Set((2, 200, 2))) + + // Update items + map.put(1, 1000, 100) + assert(map.get(1) === Some(1000)) // item removed in parent map, then added in this map + map.put(2, 2000, 200) + assert(map.get(2) === Some(2000)) // item added in parent map, then removed + added in this map + map.put(3, 3000, 300) + assert(map.get(3) === Some(3000)) // item added + updated in this map + map.put(4, 4000, 400) + assert(map.get(4) === Some(4000)) // item removed + updated in this map + + assert(map.getAll().toSet === + Set((1, 1000, 100), (2, 2000, 200), (3, 3000, 300), (4, 4000, 400))) + assert(parentMap.getAll().toSet === Set((2, 200, 2))) + + map.remove(2) // remove item present in parent map, so that its not visible in child map + + // Create child map and see availability of items + val childMap = map.copy() + assert(childMap.getAll().toSet === map.getAll().toSet) + assert(childMap.get(1) === Some(1000)) // item removed in grandparent, but added in parent map + assert(childMap.get(2) === None) // item added in grandparent, but removed in parent map + assert(childMap.get(3) === Some(3000)) // item added and updated in parent map + + childMap.put(2, 20000, 200) + assert(childMap.get(2) === Some(20000)) // item map + } + + test("OpenHashMapBasedStateMap - serializing and deserializing") { + val map1 = new OpenHashMapBasedStateMap[Int, Int]() + testSerialization(map1, "error deserializing and serialized empty map") + + map1.put(1, 100, 1) + map1.put(2, 200, 2) + testSerialization(map1, "error deserializing and serialized map with data + no delta") + + val map2 = map1.copy().asInstanceOf[OpenHashMapBasedStateMap[Int, Int]] + // Do not test compaction + assert(map2.shouldCompact === false) + testSerialization(map2, "error deserializing and serialized map with 1 delta + no new data") + + map2.put(3, 300, 3) + map2.put(4, 400, 4) + testSerialization(map2, "error deserializing and serialized map with 1 delta + new data") + + val map3 = map2.copy().asInstanceOf[OpenHashMapBasedStateMap[Int, Int]] + assert(map3.shouldCompact === false) + testSerialization(map3, "error deserializing and serialized map with 2 delta + no new data") + map3.put(3, 600, 3) + map3.remove(2) + testSerialization(map3, "error deserializing and serialized map with 2 delta + new data") + } + + test("OpenHashMapBasedStateMap - serializing and deserializing with compaction") { + val targetDeltaLength = 10 + val deltaChainThreshold = 5 + + var map = new OpenHashMapBasedStateMap[Int, Int]( + deltaChainThreshold = deltaChainThreshold) + + // Make large delta chain with length more than deltaChainThreshold + for(i <- 1 to targetDeltaLength) { + map.put(Random.nextInt(), Random.nextInt(), 1) + map = map.copy().asInstanceOf[OpenHashMapBasedStateMap[Int, Int]] + } + assert(map.deltaChainLength > deltaChainThreshold) + assert(map.shouldCompact === true) + + val deser_map = testSerialization(map, "Deserialized + compacted map not same as original map") + assert(deser_map.deltaChainLength < deltaChainThreshold) + assert(deser_map.shouldCompact === false) + } + + test("OpenHashMapBasedStateMap - all possible sequences of operations with copies ") { + /* + * This tests the map using all permutations of sequences operations, across multiple map + * copies as well as between copies. It is to ensure complete coverage, though it is + * kind of hard to debug this. It is set up as follows. + * + * - For any key, there can be 2 types of update ops on a state map - put or remove + * + * - These operations are done on a test map in "sets". After each set, the map is "copied" + * to create a new map, and the next set of operations are done on the new one. This tests + * whether the map data persist correctly across copies. + * + * - Within each set, there are a number of operations to test whether the map correctly + * updates and removes data without affecting the parent state map. + * + * - Overall this creates (numSets * numOpsPerSet) operations, each of which that can 2 types + * of operations. This leads to a total of [2 ^ (numSets * numOpsPerSet)] different sequence + * of operations, which we will test with different keys. + * + * Example: With numSets = 2, and numOpsPerSet = 2 give numTotalOps = 4. This means that + * 2 ^ 4 = 16 possible permutations needs to be tested using 16 keys. + * _______________________________________________ + * | | Set1 | Set2 | + * | |-----------------|-----------------| + * | | Op1 Op2 |c| Op3 Op4 | + * |---------|----------------|o|----------------| + * | key 0 | put put |p| put put | + * | key 1 | put put |y| put rem | + * | key 2 | put put | | rem put | + * | key 3 | put put |t| rem rem | + * | key 4 | put rem |h| put put | + * | key 5 | put rem |e| put rem | + * | key 6 | put rem | | rem put | + * | key 7 | put rem |s| rem rem | + * | key 8 | rem put |t| put put | + * | key 9 | rem put |a| put rem | + * | key 10 | rem put |t| rem put | + * | key 11 | rem put |e| rem rem | + * | key 12 | rem rem | | put put | + * | key 13 | rem rem |m| put rem | + * | key 14 | rem rem |a| rem put | + * | key 15 | rem rem |p| rem rem | + * |_________|________________|_|________________| + */ + + val numTypeMapOps = 2 // 0 = put a new value, 1 = remove value + val numSets = 3 + val numOpsPerSet = 3 // to test seq of ops like update -> remove -> update in same set + val numTotalOps = numOpsPerSet * numSets + val numKeys = math.pow(numTypeMapOps, numTotalOps).toInt // to get all combinations of ops + + val refMap = new mutable.HashMap[Int, (Int, Long)]() + var prevSetRefMap: immutable.Map[Int, (Int, Long)] = null + + var stateMap: StateMap[Int, Int] = new OpenHashMapBasedStateMap[Int, Int]() + var prevSetStateMap: StateMap[Int, Int] = null + + var time = 1L + + for (setId <- 0 until numSets) { + for (opInSetId <- 0 until numOpsPerSet) { + val opId = setId * numOpsPerSet + opInSetId + for (keyId <- 0 until numKeys) { + time += 1 + // Find the operation type that needs to be done + // This is similar to finding the nth bit value of a binary number + // E.g. nth bit from the right of any binary number B is [ B / (2 ^ (n - 1)) ] % 2 + val opCode = + (keyId / math.pow(numTypeMapOps, numTotalOps - opId - 1).toInt) % numTypeMapOps + opCode match { + case 0 => + val value = Random.nextInt() + stateMap.put(keyId, value, time) + refMap.put(keyId, (value, time)) + case 1 => + stateMap.remove(keyId) + refMap.remove(keyId) + } + } + + // Test whether the current state map after all key updates is correct + assertMap(stateMap, refMap, time, "State map does not match reference map") + + // Test whether the previous map before copy has not changed + if (prevSetStateMap != null && prevSetRefMap != null) { + assertMap(prevSetStateMap, prevSetRefMap, time, + "Parent state map somehow got modified, does not match corresponding reference map") + } + } + + // Copy the map and remember the previous maps for future tests + prevSetStateMap = stateMap + prevSetRefMap = refMap.toMap + stateMap = stateMap.copy() + + // Assert that the copied map has the same data + assertMap(stateMap, prevSetRefMap, time, + "State map does not match reference map after copying") + } + assertMap(stateMap, refMap.toMap, time, "Final state map does not match reference map") + } + + private def testSerialization[T: ClassTag]( + map: OpenHashMapBasedStateMap[T, T], msg: String): OpenHashMapBasedStateMap[T, T] = { + testSerialization(new JavaSerializer(conf), map, msg) + testSerialization(new KryoSerializer(conf), map, msg) + } + + private def testSerialization[T: ClassTag]( + serializer: Serializer, + map: OpenHashMapBasedStateMap[T, T], + msg: String): OpenHashMapBasedStateMap[T, T] = { + val deserMap = serializeAndDeserialize(serializer, map) + assertMap(deserMap, map, 1, msg) + deserMap + } + + // Assert whether all the data and operations on a state map matches that of a reference state map + private def assertMap[T]( + mapToTest: StateMap[T, T], + refMapToTestWith: StateMap[T, T], + time: Long, + msg: String): Unit = { + withClue(msg) { + // Assert all the data is same as the reference map + assert(mapToTest.getAll().toSet === refMapToTestWith.getAll().toSet) + + // Assert that get on every key returns the right value + for (keyId <- refMapToTestWith.getAll().map { _._1 }) { + assert(mapToTest.get(keyId) === refMapToTestWith.get(keyId)) + } + + // Assert that every time threshold returns the correct data + for (t <- 0L to (time + 1)) { + assert(mapToTest.getByTime(t).toSet === refMapToTestWith.getByTime(t).toSet) + } + } + } + + // Assert whether all the data and operations on a state map matches that of a reference map + private def assertMap( + mapToTest: StateMap[Int, Int], + refMapToTestWith: Map[Int, (Int, Long)], + time: Long, + msg: String): Unit = { + withClue(msg) { + // Assert all the data is same as the reference map + assert(mapToTest.getAll().toSet === + refMapToTestWith.iterator.map { x => (x._1, x._2._1, x._2._2) }.toSet) + + // Assert that get on every key returns the right value + for (keyId <- refMapToTestWith.keys) { + assert(mapToTest.get(keyId) === refMapToTestWith.get(keyId).map { _._1 }) + } + + // Assert that every time threshold returns the correct data + for (t <- 0L to (time + 1)) { + val expectedRecords = + refMapToTestWith.iterator.filter { _._2._2 < t }.map { x => (x._1, x._2._1, x._2._2) } + assert(mapToTest.getByTime(t).toSet === expectedRecords.toSet) + } + } + } + + test("OpenHashMapBasedStateMap - serializing and deserializing with KryoSerializable states") { + val map = new OpenHashMapBasedStateMap[KryoState, KryoState]() + map.put(new KryoState("a"), new KryoState("b"), 1) + testSerialization( + new KryoSerializer(conf), map, "error deserializing and serialized KryoSerializable states") + } + + test("EmptyStateMap - serializing and deserializing") { + val map = StateMap.empty[KryoState, KryoState] + // Since EmptyStateMap doesn't contains any date, KryoState won't break JavaSerializer. + assert(serializeAndDeserialize(new JavaSerializer(conf), map). + isInstanceOf[EmptyStateMap[KryoState, KryoState]]) + assert(serializeAndDeserialize(new KryoSerializer(conf), map). + isInstanceOf[EmptyStateMap[KryoState, KryoState]]) + } + + test("MapWithStateRDDRecord - serializing and deserializing with KryoSerializable states") { + val map = new OpenHashMapBasedStateMap[KryoState, KryoState]() + map.put(new KryoState("a"), new KryoState("b"), 1) + + val record = + MapWithStateRDDRecord[KryoState, KryoState, KryoState](map, Seq(new KryoState("c"))) + val deserRecord = serializeAndDeserialize(new KryoSerializer(conf), record) + assert(!(record eq deserRecord)) + assert(record.stateMap.getAll().toSeq === deserRecord.stateMap.getAll().toSeq) + assert(record.mappedData === deserRecord.mappedData) + } + + private def serializeAndDeserialize[T: ClassTag](serializer: Serializer, t: T): T = { + val serializerInstance = serializer.newInstance() + serializerInstance.deserialize[T]( + serializerInstance.serialize(t), Thread.currentThread().getContextClassLoader) + } +} + +/** A class that only supports Kryo serialization. */ +private[streaming] final class KryoState(var state: String) extends KryoSerializable { + + override def write(kryo: Kryo, output: Output): Unit = { + kryo.writeClassAndObject(output, state) + } + + override def read(kryo: Kryo, input: Input): Unit = { + state = kryo.readClassAndObject(input).asInstanceOf[String] + } + + override def equals(other: Any): Boolean = other match { + case that: KryoState => state == that.state + case _ => false + } + + override def hashCode(): Int = { + if (state == null) 0 else state.hashCode() + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index c7a877142b37..5810e73f4098 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.streaming import java.io.{File, NotSerializableException} +import java.util.Locale +import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.ArrayBuffer @@ -25,12 +27,13 @@ import scala.collection.mutable.Queue import org.apache.commons.io.FileUtils import org.scalatest.{Assertions, BeforeAndAfter, PrivateMethodTester} +import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.concurrent.Eventually._ -import org.scalatest.concurrent.Timeouts import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.SpanSugar._ import org.apache.spark._ +import org.apache.spark.internal.Logging import org.apache.spark.metrics.MetricsSystem import org.apache.spark.metrics.source.Source import org.apache.spark.storage.StorageLevel @@ -39,7 +42,7 @@ import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.util.Utils -class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeouts with Logging { +class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeLimits with Logging { val master = "local[2]" val appName = this.getClass.getSimpleName @@ -81,9 +84,9 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo test("from conf with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10s") + myConf.set("spark.dummyTimeConfig", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) + assert(ssc.conf.getTimeAsSeconds("spark.dummyTimeConfig", "-1") === 10) } test("from existing SparkContext") { @@ -93,26 +96,27 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo test("from existing SparkContext with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10s") + myConf.set("spark.dummyTimeConfig", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) + assert(ssc.conf.getTimeAsSeconds("spark.dummyTimeConfig", "-1") === 10) } test("from checkpoint") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10s") + myConf.set("spark.dummyTimeConfig", "10s") val ssc1 = new StreamingContext(myConf, batchDuration) addInputStream(ssc1).register() ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) assert( Utils.timeStringAsSeconds(cp.sparkConfPairs - .toMap.getOrElse("spark.cleaner.ttl", "-1")) === 10) + .toMap.getOrElse("spark.dummyTimeConfig", "-1")) === 10) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(newCp.createSparkConf().getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) + assert( + newCp.createSparkConf().getTimeAsSeconds("spark.dummyTimeConfig", "-1") === 10) ssc = new StreamingContext(null, newCp, null) - assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) + assert(ssc.conf.getTimeAsSeconds("spark.dummyTimeConfig", "-1") === 10) } test("checkPoint from conf") { @@ -146,7 +150,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo } } - test("start with non-seriazable DStream checkpoints") { + test("start with non-serializable DStream checkpoints") { val checkpointDir = Utils.createTempDir() ssc = new StreamingContext(conf, batchDuration) ssc.checkpoint(checkpointDir.getAbsolutePath) @@ -180,7 +184,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo assert(ssc.scheduler.isStarted === false) } - test("start should set job group and description of streaming jobs correctly") { + test("start should set local properties of streaming jobs correctly") { ssc = new StreamingContext(conf, batchDuration) ssc.sc.setJobGroup("non-streaming", "non-streaming", true) val sc = ssc.sc @@ -188,16 +192,22 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo @volatile var jobGroupFound: String = "" @volatile var jobDescFound: String = "" @volatile var jobInterruptFound: String = "" + @volatile var customPropFound: String = "" @volatile var allFound: Boolean = false addInputStream(ssc).foreachRDD { rdd => jobGroupFound = sc.getLocalProperty(SparkContext.SPARK_JOB_GROUP_ID) jobDescFound = sc.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION) jobInterruptFound = sc.getLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL) + customPropFound = sc.getLocalProperty("customPropKey") allFound = true } + ssc.sc.setLocalProperty("customPropKey", "value1") ssc.start() + // Local props set after start should be ignored + ssc.sc.setLocalProperty("customPropKey", "value2") + eventually(timeout(10 seconds), interval(10 milliseconds)) { assert(allFound === true) } @@ -206,11 +216,13 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo assert(jobGroupFound === null) assert(jobDescFound.contains("Streaming job from")) assert(jobInterruptFound === "false") + assert(customPropFound === "value1") // Verify current thread's thread-local properties have not changed assert(sc.getLocalProperty(SparkContext.SPARK_JOB_GROUP_ID) === "non-streaming") assert(sc.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION) === "non-streaming") assert(sc.getLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL) === "true") + assert(sc.getLocalProperty("customPropKey") === "value2") } test("start multiple times") { @@ -288,7 +300,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo test("stop gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) - conf.set("spark.cleaner.ttl", "3600s") + conf.set("spark.dummyTimeConfig", "3600s") sc = new SparkContext(conf) for (i <- 1 to 4) { logInfo("==================================\n\n\n") @@ -394,6 +406,8 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo // test whether awaitTermination() does not exit if not time is given val exception = intercept[Exception] { + // Necessary to make failAfter interrupt awaitTermination() in ScalaTest 3.x + implicit val signaler: Signaler = ThreadSignaler failAfter(1000 millis) { ssc.awaitTermination() throw new Exception("Did not wait for stop") @@ -561,8 +575,6 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo test("getActive and getActiveOrCreate") { require(StreamingContext.getActive().isEmpty, "context exists from before") - sc = new SparkContext(conf) - var newContextCreated = false def creatingFunc(): StreamingContext = { @@ -589,6 +601,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo // getActiveOrCreate should create new context and getActive should return it only // after starting the context testGetActiveOrCreate { + sc = new SparkContext(conf) ssc = StreamingContext.getActiveOrCreate(creatingFunc _) assert(ssc != null, "no context created") assert(newContextCreated === true, "new context not created") @@ -608,6 +621,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo // getActiveOrCreate and getActive should return independently created context after activating testGetActiveOrCreate { + sc = new SparkContext(conf) ssc = creatingFunc() // Create assert(StreamingContext.getActive().isEmpty, "new initialized context returned before starting") @@ -734,7 +748,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo val ex = intercept[IllegalStateException] { body } - assert(ex.getMessage.toLowerCase().contains(expectedErrorMsg)) + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains(expectedErrorMsg)) } } @@ -780,6 +794,52 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo "Please don't use queueStream when checkpointing is enabled.")) } + test("Creating an InputDStream but not using it should not crash") { + ssc = new StreamingContext(master, appName, batchDuration) + val input1 = addInputStream(ssc) + val input2 = addInputStream(ssc) + val output = new TestOutputStream(input2) + output.register() + val batchCount = new BatchCounter(ssc) + ssc.start() + // Just wait for completing 2 batches to make sure it triggers + // `DStream.getMaxInputStreamRememberDuration` + batchCount.waitUntilBatchesCompleted(2, 10000) + // Throw the exception if crash + ssc.awaitTerminationOrTimeout(1) + ssc.stop() + } + + test("SPARK-18560 Receiver data should be deserialized properly.") { + // Start a two nodes cluster, so receiver will use one node, and Spark jobs will use the + // other one. Then Spark jobs need to fetch remote blocks and it will trigger SPARK-18560. + val conf = new SparkConf().setMaster("local-cluster[2,1,1024]").setAppName(appName) + ssc = new StreamingContext(conf, Milliseconds(100)) + val input = ssc.receiverStream(new TestReceiver) + val latch = new CountDownLatch(1) + @volatile var stopping = false + input.count().foreachRDD { rdd => + // Make sure we can read from BlockRDD + if (rdd.collect().headOption.getOrElse(0L) > 0 && !stopping) { + // Stop StreamingContext to unblock "awaitTerminationOrTimeout" + stopping = true + new Thread() { + setDaemon(true) + override def run(): Unit = { + ssc.stop(stopSparkContext = true, stopGracefully = false) + latch.countDown() + } + }.start() + } + } + ssc.start() + ssc.awaitTerminationOrTimeout(60000) + // Wait until `ssc.top` returns. Otherwise, we may finish this test too fast and leak an active + // SparkContext. Note: the stop codes in `after` will just do nothing if `ssc.stop` in this test + // is running. + assert(latch.await(60, TimeUnit.SECONDS)) + } + def addInputStream(s: StreamingContext): DStream[Int] = { val input = (1 to 100).map(i => 1 to i) val inputStream = new TestInputStream(s, input, 1) @@ -793,10 +853,13 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo ssc.checkpoint(checkpointDirectory) ssc.textFileStream(testDirectory).foreachRDD { rdd => rdd.count() } ssc.start() - eventually(timeout(10000 millis)) { - assert(Checkpoint.getCheckpointFiles(checkpointDirectory).size > 1) + try { + eventually(timeout(30000 millis)) { + assert(Checkpoint.getCheckpointFiles(checkpointDirectory).size > 1) + } + } finally { + ssc.stop() } - ssc.stop() checkpointDirectory } @@ -879,7 +942,7 @@ object SlowTestReceiver { package object testPackage extends Assertions { def test() { val conf = new SparkConf().setMaster("local").setAppName("CreationSite test") - val ssc = new StreamingContext(conf , Milliseconds(100)) + val ssc = new StreamingContext(conf, Milliseconds(100)) try { val inputStream = ssc.receiverStream(new TestReceiver) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 5dc0472c7770..0f957a1b5570 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -17,20 +17,25 @@ package org.apache.spark.streaming -import scala.collection.mutable.{ArrayBuffer, HashMap, SynchronizedBuffer, SynchronizedMap} -import scala.concurrent.Future +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.collection.JavaConverters._ +import scala.collection.mutable.HashMap import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.Future + +import org.mockito.Mockito.{mock, reset, verifyNoMoreInteractions} +import org.scalatest.Matchers +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.streaming.scheduler._ -import org.scalatest.Matchers -import org.scalatest.concurrent.Eventually._ -import org.scalatest.time.SpanSugar._ -import org.apache.spark.Logging - class StreamingListenerSuite extends TestSuiteBase with Matchers { val input = (1 to 4).map(Seq(_)).toSeq @@ -60,43 +65,43 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { val batchInfosSubmitted = collector.batchInfosSubmitted batchInfosSubmitted should have size 4 - batchInfosSubmitted.foreach(info => { + batchInfosSubmitted.asScala.foreach(info => { info.schedulingDelay should be (None) info.processingDelay should be (None) info.totalDelay should be (None) }) - batchInfosSubmitted.foreach { info => + batchInfosSubmitted.asScala.foreach { info => info.numRecords should be (1L) info.streamIdToInputInfo should be (Map(0 -> StreamInputInfo(0, 1L))) } - isInIncreasingOrder(batchInfosSubmitted.map(_.submissionTime)) should be (true) + isInIncreasingOrder(batchInfosSubmitted.asScala.map(_.submissionTime)) should be (true) // SPARK-6766: processingStartTime of batch info should not be None when starting val batchInfosStarted = collector.batchInfosStarted batchInfosStarted should have size 4 - batchInfosStarted.foreach(info => { + batchInfosStarted.asScala.foreach(info => { info.schedulingDelay should not be None info.schedulingDelay.get should be >= 0L info.processingDelay should be (None) info.totalDelay should be (None) }) - batchInfosStarted.foreach { info => + batchInfosStarted.asScala.foreach { info => info.numRecords should be (1L) info.streamIdToInputInfo should be (Map(0 -> StreamInputInfo(0, 1L))) } - isInIncreasingOrder(batchInfosStarted.map(_.submissionTime)) should be (true) - isInIncreasingOrder(batchInfosStarted.map(_.processingStartTime.get)) should be (true) + isInIncreasingOrder(batchInfosStarted.asScala.map(_.submissionTime)) should be (true) + isInIncreasingOrder(batchInfosStarted.asScala.map(_.processingStartTime.get)) should be (true) // test onBatchCompleted val batchInfosCompleted = collector.batchInfosCompleted batchInfosCompleted should have size 4 - batchInfosCompleted.foreach(info => { + batchInfosCompleted.asScala.foreach(info => { info.schedulingDelay should not be None info.processingDelay should not be None info.totalDelay should not be None @@ -105,14 +110,14 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { info.totalDelay.get should be >= 0L }) - batchInfosCompleted.foreach { info => + batchInfosCompleted.asScala.foreach { info => info.numRecords should be (1L) info.streamIdToInputInfo should be (Map(0 -> StreamInputInfo(0, 1L))) } - isInIncreasingOrder(batchInfosCompleted.map(_.submissionTime)) should be (true) - isInIncreasingOrder(batchInfosCompleted.map(_.processingStartTime.get)) should be (true) - isInIncreasingOrder(batchInfosCompleted.map(_.processingEndTime.get)) should be (true) + isInIncreasingOrder(batchInfosCompleted.asScala.map(_.submissionTime)) should be (true) + isInIncreasingOrder(batchInfosCompleted.asScala.map(_.processingStartTime.get)) should be (true) + isInIncreasingOrder(batchInfosCompleted.asScala.map(_.processingEndTime.get)) should be (true) } test("receiver info reporting") { @@ -127,13 +132,13 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { try { eventually(timeout(30 seconds), interval(20 millis)) { collector.startedReceiverStreamIds.size should equal (1) - collector.startedReceiverStreamIds(0) should equal (0) - collector.stoppedReceiverStreamIds should have size 1 - collector.stoppedReceiverStreamIds(0) should equal (0) + collector.startedReceiverStreamIds.peek() should equal (0) + collector.stoppedReceiverStreamIds.size should equal (1) + collector.stoppedReceiverStreamIds.peek() should equal (0) collector.receiverErrors should have size 1 - collector.receiverErrors(0)._1 should equal (0) - collector.receiverErrors(0)._2 should include ("report error") - collector.receiverErrors(0)._3 should include ("report exception") + collector.receiverErrors.peek()._1 should equal (0) + collector.receiverErrors.peek()._2 should include ("report error") + collector.receiverErrors.peek()._3 should include ("report exception") } } finally { ssc.stop() @@ -153,14 +158,22 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { ssc.start() try { eventually(timeout(30 seconds), interval(20 millis)) { - collector.startedOutputOperationIds.take(3) should be (Seq(0, 1, 2)) - collector.completedOutputOperationIds.take(3) should be (Seq(0, 1, 2)) + collector.startedOutputOperationIds.asScala.take(3) should be (Seq(0, 1, 2)) + collector.completedOutputOperationIds.asScala.take(3) should be (Seq(0, 1, 2)) } } finally { ssc.stop() } } + test("don't call ssc.stop in listener") { + ssc = new StreamingContext("local[2]", "ssc", Milliseconds(1000)) + val inputStream = ssc.receiverStream(new StreamingListenerSuiteReceiver) + inputStream.foreachRDD(_.count) + + startStreamingContextAndCallStop(ssc) + } + test("onBatchCompleted with successful batch") { ssc = new StreamingContext("local[2]", "test", Milliseconds(1000)) val inputStream = ssc.receiverStream(new StreamingListenerSuiteReceiver) @@ -207,6 +220,42 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { assert(failureReasons(1).contains("This is another failed job")) } + test("StreamingListener receives no events after stopping StreamingListenerBus") { + val streamingListener = mock(classOf[StreamingListener]) + + ssc = new StreamingContext("local[2]", "test", Milliseconds(1000)) + ssc.addStreamingListener(streamingListener) + val inputStream = ssc.receiverStream(new StreamingListenerSuiteReceiver) + inputStream.foreachRDD(_.count) + ssc.start() + ssc.stop() + + // Because "streamingListener" has already received some events, let's clear that. + reset(streamingListener) + + // Post a Streaming event after stopping StreamingContext + val receiverInfoStopped = ReceiverInfo(0, "test", false, "localhost", "0") + ssc.scheduler.listenerBus.post(StreamingListenerReceiverStopped(receiverInfoStopped)) + ssc.sparkContext.listenerBus.waitUntilEmpty(1000) + // The StreamingListener should not receive any event + verifyNoMoreInteractions(streamingListener) + } + + private def startStreamingContextAndCallStop(_ssc: StreamingContext): Unit = { + val contextStoppingCollector = new StreamingContextStoppingCollector(_ssc) + _ssc.addStreamingListener(contextStoppingCollector) + val batchCounter = new BatchCounter(_ssc) + _ssc.start() + // Make sure running at least one batch + if (!batchCounter.waitUntilBatchesCompleted(expectedNumCompletedBatches = 1, timeout = 10000)) { + fail("The first batch cannot complete in 10 seconds") + } + // When reaching here, we can make sure `StreamingContextStoppingCollector` won't call + // `ssc.stop()`, so it's safe to call `_ssc.stop()` now. + _ssc.stop() + assert(contextStoppingCollector.sparkExSeen) + } + private def startStreamingContextAndCollectFailureReasons( _ssc: StreamingContext, isFailed: Boolean = false): Map[Int, String] = { val failureReasonsCollector = new FailureReasonsCollector() @@ -221,73 +270,70 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { } } _ssc.stop() - failureReasonsCollector.failureReasons.toMap + failureReasonsCollector.failureReasons.synchronized + { + failureReasonsCollector.failureReasons.toMap + } } /** Check if a sequence of numbers is in increasing order */ - def isInIncreasingOrder(seq: Seq[Long]): Boolean = { - for (i <- 1 until seq.size) { - if (seq(i - 1) > seq(i)) { - return false - } - } - true + def isInIncreasingOrder(data: Iterable[Long]): Boolean = { + !data.sliding(2).exists { itr => itr.size == 2 && itr.head > itr.tail.head } } } /** Listener that collects information on processed batches */ class BatchInfoCollector extends StreamingListener { - val batchInfosCompleted = new ArrayBuffer[BatchInfo] with SynchronizedBuffer[BatchInfo] - val batchInfosStarted = new ArrayBuffer[BatchInfo] with SynchronizedBuffer[BatchInfo] - val batchInfosSubmitted = new ArrayBuffer[BatchInfo] with SynchronizedBuffer[BatchInfo] + val batchInfosCompleted = new ConcurrentLinkedQueue[BatchInfo] + val batchInfosStarted = new ConcurrentLinkedQueue[BatchInfo] + val batchInfosSubmitted = new ConcurrentLinkedQueue[BatchInfo] override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) { - batchInfosSubmitted += batchSubmitted.batchInfo + batchInfosSubmitted.add(batchSubmitted.batchInfo) } override def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { - batchInfosStarted += batchStarted.batchInfo + batchInfosStarted.add(batchStarted.batchInfo) } override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { - batchInfosCompleted += batchCompleted.batchInfo + batchInfosCompleted.add(batchCompleted.batchInfo) } } /** Listener that collects information on processed batches */ class ReceiverInfoCollector extends StreamingListener { - val startedReceiverStreamIds = new ArrayBuffer[Int] with SynchronizedBuffer[Int] - val stoppedReceiverStreamIds = new ArrayBuffer[Int] with SynchronizedBuffer[Int] - val receiverErrors = - new ArrayBuffer[(Int, String, String)] with SynchronizedBuffer[(Int, String, String)] + val startedReceiverStreamIds = new ConcurrentLinkedQueue[Int] + val stoppedReceiverStreamIds = new ConcurrentLinkedQueue[Int] + val receiverErrors = new ConcurrentLinkedQueue[(Int, String, String)] override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { - startedReceiverStreamIds += receiverStarted.receiverInfo.streamId + startedReceiverStreamIds.add(receiverStarted.receiverInfo.streamId) } override def onReceiverStopped(receiverStopped: StreamingListenerReceiverStopped) { - stoppedReceiverStreamIds += receiverStopped.receiverInfo.streamId + stoppedReceiverStreamIds.add(receiverStopped.receiverInfo.streamId) } override def onReceiverError(receiverError: StreamingListenerReceiverError) { - receiverErrors += ((receiverError.receiverInfo.streamId, - receiverError.receiverInfo.lastErrorMessage, receiverError.receiverInfo.lastError)) + receiverErrors.add(((receiverError.receiverInfo.streamId, + receiverError.receiverInfo.lastErrorMessage, receiverError.receiverInfo.lastError))) } } /** Listener that collects information on processed output operations */ class OutputOperationInfoCollector extends StreamingListener { - val startedOutputOperationIds = new ArrayBuffer[Int] with SynchronizedBuffer[Int] - val completedOutputOperationIds = new ArrayBuffer[Int] with SynchronizedBuffer[Int] + val startedOutputOperationIds = new ConcurrentLinkedQueue[Int]() + val completedOutputOperationIds = new ConcurrentLinkedQueue[Int]() override def onOutputOperationStarted( outputOperationStarted: StreamingListenerOutputOperationStarted): Unit = { - startedOutputOperationIds += outputOperationStarted.outputOperationInfo.id + startedOutputOperationIds.add(outputOperationStarted.outputOperationInfo.id) } override def onOutputOperationCompleted( outputOperationCompleted: StreamingListenerOutputOperationCompleted): Unit = { - completedOutputOperationIds += outputOperationCompleted.outputOperationInfo.id + completedOutputOperationIds.add(outputOperationCompleted.outputOperationInfo.id) } } @@ -311,12 +357,38 @@ class StreamingListenerSuiteReceiver extends Receiver[Any](StorageLevel.MEMORY_O */ class FailureReasonsCollector extends StreamingListener { - val failureReasons = new HashMap[Int, String] with SynchronizedMap[Int, String] + val failureReasons = new HashMap[Int, String] override def onOutputOperationCompleted( outputOperationCompleted: StreamingListenerOutputOperationCompleted): Unit = { outputOperationCompleted.outputOperationInfo.failureReason.foreach { f => - failureReasons(outputOperationCompleted.outputOperationInfo.id) = f + failureReasons.synchronized + { + failureReasons(outputOperationCompleted.outputOperationInfo.id) = f + } + } + } +} +/** + * A StreamingListener that calls StreamingContext.stop(). + */ +class StreamingContextStoppingCollector(val ssc: StreamingContext) extends StreamingListener { + @volatile var sparkExSeen = false + + private var isFirstBatch = true + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { + if (isFirstBatch) { + // We should only call `ssc.stop()` in the first batch. Otherwise, it's possible that the main + // thread is calling `ssc.stop()`, while StreamingContextStoppingCollector is also calling + // `ssc.stop()` in the listener thread, which becomes a dead-lock. + isFirstBatch = false + try { + ssc.stop() + } catch { + case se: SparkException => + sparkExSeen = true + } } } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 0d58a7b54412..dbab70886102 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -17,21 +17,22 @@ package org.apache.spark.streaming -import java.io.{ObjectInputStream, IOException} +import java.io.{IOException, ObjectInputStream} +import java.util.concurrent.ConcurrentLinkedQueue -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.SynchronizedBuffer +import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.ClassTag import org.scalatest.BeforeAndAfter -import org.scalatest.time.{Span, Seconds => ScalaTestSeconds} import org.scalatest.concurrent.Eventually.timeout import org.scalatest.concurrent.PatienceConfiguration +import org.scalatest.time.{Seconds => ScalaTestSeconds, Span} -import org.apache.spark.{Logging, SparkConf, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.dstream.{DStream, InputDStream, ForEachDStream} +import org.apache.spark.streaming.dstream.{DStream, ForEachDStream, InputDStream} import org.apache.spark.streaming.scheduler._ import org.apache.spark.util.{ManualClock, Utils} @@ -56,10 +57,10 @@ private[streaming] class DummyInputDStream(ssc: StreamingContext) extends InputD /** * This is a input stream just for the testsuites. This is equivalent to a checkpointable, * replayable, reliable message queue like Kafka. It requires a sequence as input, and - * returns the i_th element at the i_th batch unde manual clock. + * returns the i_th element at the i_th batch under manual clock. */ -class TestInputStream[T: ClassTag](ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int) - extends InputDStream[T](ssc_) { +class TestInputStream[T: ClassTag](_ssc: StreamingContext, input: Seq[Seq[T]], numPartitions: Int) + extends InputDStream[T](_ssc) { def start() {} @@ -87,18 +88,18 @@ class TestInputStream[T: ClassTag](ssc_ : StreamingContext, input: Seq[Seq[T]], /** * This is a output stream just for the testsuites. All the output is collected into a - * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint. + * ConcurrentLinkedQueue. This queue is wiped clean on being restored from checkpoint. * - * The buffer contains a sequence of RDD's, each containing a sequence of items + * The buffer contains a sequence of RDD's, each containing a sequence of items. */ class TestOutputStream[T: ClassTag]( parent: DStream[T], - val output: SynchronizedBuffer[Seq[T]] = - new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]] + val output: ConcurrentLinkedQueue[Seq[T]] = + new ConcurrentLinkedQueue[Seq[T]]() ) extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => { val collected = rdd.collect() - output += collected - }) { + output.add(collected) + }, false) { // This is to clear the output buffer every it is read from a checkpoint @throws(classOf[IOException]) @@ -110,19 +111,19 @@ class TestOutputStream[T: ClassTag]( /** * This is a output stream just for the testsuites. All the output is collected into a - * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint. + * ConcurrentLinkedQueue. This queue is wiped clean on being restored from checkpoint. * - * The buffer contains a sequence of RDD's, each containing a sequence of partitions, each + * The queue contains a sequence of RDD's, each containing a sequence of partitions, each * containing a sequence of items. */ class TestOutputStreamWithPartitions[T: ClassTag]( parent: DStream[T], - val output: SynchronizedBuffer[Seq[Seq[T]]] = - new ArrayBuffer[Seq[Seq[T]]] with SynchronizedBuffer[Seq[Seq[T]]]) + val output: ConcurrentLinkedQueue[Seq[Seq[T]]] = + new ConcurrentLinkedQueue[Seq[Seq[T]]]()) extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => { val collected = rdd.glom().collect().map(_.toSeq) - output += collected - }) { + output.add(collected) + }, false) { // This is to clear the output buffer every it is read from a checkpoint @throws(classOf[IOException]) @@ -142,6 +143,7 @@ class BatchCounter(ssc: StreamingContext) { // All access to this state should be guarded by `BatchCounter.this.synchronized` private var numCompletedBatches = 0 private var numStartedBatches = 0 + private var lastCompletedBatchTime: Time = null private val listener = new StreamingListener { override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = @@ -152,6 +154,7 @@ class BatchCounter(ssc: StreamingContext) { override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = BatchCounter.this.synchronized { numCompletedBatches += 1 + lastCompletedBatchTime = batchCompleted.batchInfo.batchTime BatchCounter.this.notifyAll() } } @@ -165,6 +168,10 @@ class BatchCounter(ssc: StreamingContext) { numStartedBatches } + def getLastCompletedBatchTime: Time = this.synchronized { + lastCompletedBatchTime + } + /** * Wait until `expectedNumCompletedBatches` batches are completed, or timeout. Return true if * `expectedNumCompletedBatches` batches are completed. Otherwise, return false to indicate it's @@ -316,7 +323,7 @@ trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { val inputStream = new TestInputStream(ssc, input, numPartitions) val operatedStream = operation(inputStream) val outputStream = new TestOutputStreamWithPartitions(operatedStream, - new ArrayBuffer[Seq[Seq[V]]] with SynchronizedBuffer[Seq[Seq[V]]]) + new ConcurrentLinkedQueue[Seq[Seq[V]]]) outputStream.register() ssc } @@ -341,7 +348,7 @@ trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { val inputStream2 = new TestInputStream(ssc, input2, numInputPartitions) val operatedStream = operation(inputStream1, inputStream2) val outputStream = new TestOutputStreamWithPartitions(operatedStream, - new ArrayBuffer[Seq[Seq[W]]] with SynchronizedBuffer[Seq[Seq[W]]]) + new ConcurrentLinkedQueue[Seq[Seq[W]]]) outputStream.register() ssc } @@ -352,14 +359,20 @@ trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { * output data has been collected or timeout (set by `maxWaitTimeMillis`) is reached. * * Returns a sequence of items for each RDD. + * + * @param ssc The StreamingContext + * @param numBatches The number of batches should be run + * @param numExpectedOutput The number of expected output + * @param preStop The function to run before stopping StreamingContext */ def runStreams[V: ClassTag]( ssc: StreamingContext, numBatches: Int, - numExpectedOutput: Int + numExpectedOutput: Int, + preStop: () => Unit = () => {} ): Seq[Seq[V]] = { // Flatten each RDD into a single Seq - runStreamsWithPartitions(ssc, numBatches, numExpectedOutput).map(_.flatten.toSeq) + runStreamsWithPartitions(ssc, numBatches, numExpectedOutput, preStop).map(_.flatten.toSeq) } /** @@ -369,11 +382,17 @@ trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { * * Returns a sequence of RDD's. Each RDD is represented as several sequences of items, each * representing one partition. + * + * @param ssc The StreamingContext + * @param numBatches The number of batches should be run + * @param numExpectedOutput The number of expected output + * @param preStop The function to run before stopping StreamingContext */ def runStreamsWithPartitions[V: ClassTag]( ssc: StreamingContext, numBatches: Int, - numExpectedOutput: Int + numExpectedOutput: Int, + preStop: () => Unit = () => {} ): Seq[Seq[Seq[V]]] = { assert(numBatches > 0, "Number of batches to run stream computation is zero") assert(numExpectedOutput > 0, "Number of expected outputs after " + numBatches + " is zero") @@ -412,15 +431,16 @@ trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { } val timeTaken = System.currentTimeMillis() - startTime logInfo("Output generated in " + timeTaken + " milliseconds") - output.foreach(x => logInfo("[" + x.mkString(",") + "]")) + output.asScala.foreach(x => logInfo("[" + x.mkString(",") + "]")) assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") assert(output.size === numExpectedOutput, "Unexpected number of outputs generated") Thread.sleep(100) // Give some time for the forgetting old RDDs to complete + preStop() } finally { ssc.stop(stopSparkContext = true) } - output + output.asScala.toSeq } /** @@ -495,7 +515,7 @@ trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { val numBatches_ = if (numBatches > 0) numBatches else expectedOutput.size withStreamingContext(setupStreams[U, V](input, operation)) { ssc => val output = runStreams[V](ssc, numBatches_, expectedOutput.size) - verifyOutput[V](output, expectedOutput, useSet) + verifyOutput[V](output.toSeq, expectedOutput, useSet) } } @@ -534,7 +554,7 @@ trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { val numBatches_ = if (numBatches > 0) numBatches else expectedOutput.size withStreamingContext(setupStreams[U, V, W](input1, input2, operation)) { ssc => val output = runStreams[W](ssc, numBatches_, expectedOutput.size) - verifyOutput[W](output, expectedOutput, useSet) + verifyOutput[W](output.toSeq, expectedOutput, useSet) } } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index a5744a9009c1..f2204a187093 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -38,14 +38,19 @@ class UISeleniumSuite implicit var webDriver: WebDriver = _ override def beforeAll(): Unit = { + super.beforeAll() webDriver = new HtmlUnitDriver { getWebClient.setCssErrorHandler(new SparkUICssErrorHandler) } } override def afterAll(): Unit = { - if (webDriver != null) { - webDriver.quit() + try { + if (webDriver != null) { + webDriver.quit() + } + } finally { + super.afterAll() } } @@ -87,13 +92,13 @@ class UISeleniumSuite val sparkUI = ssc.sparkContext.ui.get eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sparkUI.appUIAddress.stripSuffix("/")) + go to (sparkUI.webUrl.stripSuffix("/")) find(cssSelector( """ul li a[href*="streaming"]""")) should not be (None) } eventually(timeout(10 seconds), interval(50 milliseconds)) { // check whether streaming page exists - go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming") + go to (sparkUI.webUrl.stripSuffix("/") + "/streaming") val h3Text = findAll(cssSelector("h3")).map(_.text).toSeq h3Text should contain("Streaming Statistics") @@ -116,11 +121,11 @@ class UISeleniumSuite h4Text.exists(_.matches("Completed Batches \\(last \\d+ out of \\d+\\)")) should be (true) findAll(cssSelector("""#active-batches-table th""")).map(_.text).toSeq should be { - List("Batch Time", "Input Size", "Scheduling Delay (?)", "Processing Time (?)", + List("Batch Time", "Records", "Scheduling Delay (?)", "Processing Time (?)", "Output Ops: Succeeded/Total", "Status") } findAll(cssSelector("""#completed-batches-table th""")).map(_.text).toSeq should be { - List("Batch Time", "Input Size", "Scheduling Delay (?)", "Processing Time (?)", + List("Batch Time", "Records", "Scheduling Delay (?)", "Processing Time (?)", "Total Delay (?)", "Output Ops: Succeeded/Total") } @@ -138,8 +143,9 @@ class UISeleniumSuite summaryText should contain ("Total delay:") findAll(cssSelector("""#batch-job-table th""")).map(_.text).toSeq should be { - List("Output Op Id", "Description", "Duration", "Status", "Job Id", "Duration", - "Stages: Succeeded/Total", "Tasks (for all stages): Succeeded/Total", "Error") + List("Output Op Id", "Description", "Output Op Duration", "Status", "Job Id", + "Job Duration", "Stages: Succeeded/Total", "Tasks (for all stages): Succeeded/Total", + "Error") } // Check we have 2 output op ids @@ -155,17 +161,17 @@ class UISeleniumSuite jobLinks.size should be (4) // Check stage progress - findAll(cssSelector(""".stage-progress-cell""")).map(_.text).toSeq should be - (List("1/1", "1/1", "1/1", "0/1 (1 failed)")) + findAll(cssSelector(""".stage-progress-cell""")).map(_.text).toList should be ( + List("1/1", "1/1", "1/1", "0/1 (1 failed)")) // Check job progress - findAll(cssSelector(""".progress-cell""")).map(_.text).toSeq should be - (List("1/1", "1/1", "1/1", "0/1 (1 failed)")) + findAll(cssSelector(""".progress-cell""")).map(_.text).toList should be ( + List("4/4", "4/4", "4/4", "0/4 (1 failed)")) // Check stacktrace - val errorCells = findAll(cssSelector(""".stacktrace-details""")).map(_.text).toSeq + val errorCells = findAll(cssSelector(""".stacktrace-details""")).map(_.underlying).toSeq errorCells should have size 1 - errorCells(0) should include("java.lang.RuntimeException: Oops") + // Can't get the inner (invisible) text without running JS // Check the job link in the batch page is right go to (jobLinks(0)) @@ -174,23 +180,23 @@ class UISeleniumSuite jobDetails should contain("Completed Stages:") // Check a batch page without id - go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming/batch/") + go to (sparkUI.webUrl.stripSuffix("/") + "/streaming/batch/") webDriver.getPageSource should include ("Missing id parameter") // Check a non-exist batch - go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming/batch/?id=12345") + go to (sparkUI.webUrl.stripSuffix("/") + "/streaming/batch/?id=12345") webDriver.getPageSource should include ("does not exist") } ssc.stop(false) eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sparkUI.appUIAddress.stripSuffix("/")) + go to (sparkUI.webUrl.stripSuffix("/")) find(cssSelector( """ul li a[href*="streaming"]""")) should be(None) } eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming") + go to (sparkUI.webUrl.stripSuffix("/") + "/streaming") val h3Text = findAll(cssSelector("h3")).map(_.text).toSeq h3Text should not contain("Streaming Statistics") } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala index c39ad05f4152..c7d085ec0799 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.streaming -import org.apache.spark.streaming.dstream.DStream import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.DStream class WindowOperationsSuite extends TestSuiteBase { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/rdd/MapWithStateRDDSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/rdd/MapWithStateRDDSuite.scala new file mode 100644 index 000000000000..9b6bc71c7a5b --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/rdd/MapWithStateRDDSuite.scala @@ -0,0 +1,399 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.rdd + +import java.io.File + +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark._ +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{State, Time} +import org.apache.spark.streaming.util.OpenHashMapBasedStateMap +import org.apache.spark.util.Utils + +class MapWithStateRDDSuite extends SparkFunSuite with RDDCheckpointTester with BeforeAndAfterAll { + + private var sc: SparkContext = null + private var checkpointDir: File = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sc = new SparkContext( + new SparkConf().setMaster("local").setAppName("MapWithStateRDDSuite")) + checkpointDir = Utils.createTempDir() + sc.setCheckpointDir(checkpointDir.toString) + } + + override def afterAll(): Unit = { + try { + if (sc != null) { + sc.stop() + } + Utils.deleteRecursively(checkpointDir) + } finally { + super.afterAll() + } + } + + override def sparkContext: SparkContext = sc + + test("creation from pair RDD") { + val data = Seq((1, "1"), (2, "2"), (3, "3")) + val partitioner = new HashPartitioner(10) + val rdd = MapWithStateRDD.createFromPairRDD[Int, Int, String, Int]( + sc.parallelize(data), partitioner, Time(123)) + assertRDD[Int, Int, String, Int](rdd, data.map { x => (x._1, x._2, 123)}.toSet, Set.empty) + assert(rdd.partitions.size === partitioner.numPartitions) + + assert(rdd.partitioner === Some(partitioner)) + } + + test("updating state and generating mapped data in MapWithStateRDDRecord") { + + val initialTime = 1000L + val updatedTime = 2000L + val thresholdTime = 1500L + @volatile var functionCalled = false + + /** + * Assert that applying given data on a prior record generates correct updated record, with + * correct state map and mapped data + */ + def assertRecordUpdate( + initStates: Iterable[Int], + data: Iterable[String], + expectedStates: Iterable[(Int, Long)], + timeoutThreshold: Option[Long] = None, + removeTimedoutData: Boolean = false, + expectedOutput: Iterable[Int] = None, + expectedTimingOutStates: Iterable[Int] = None, + expectedRemovedStates: Iterable[Int] = None + ): Unit = { + val initialStateMap = new OpenHashMapBasedStateMap[String, Int]() + initStates.foreach { s => initialStateMap.put("key", s, initialTime) } + functionCalled = false + val record = MapWithStateRDDRecord[String, Int, Int](initialStateMap, Seq.empty) + val dataIterator = data.map { v => ("key", v) }.iterator + val removedStates = new ArrayBuffer[Int] + val timingOutStates = new ArrayBuffer[Int] + /** + * Mapping function that updates/removes state based on instructions in the data, and + * return state (when instructed or when state is timing out). + */ + def testFunc(t: Time, key: String, data: Option[String], state: State[Int]): Option[Int] = { + functionCalled = true + + assert(t.milliseconds === updatedTime, "mapping func called with wrong time") + + data match { + case Some("noop") => + None + case Some("get-state") => + Some(state.getOption().getOrElse(-1)) + case Some("update-state") => + if (state.exists) state.update(state.get + 1) else state.update(0) + None + case Some("remove-state") => + removedStates += state.get() + state.remove() + None + case None => + assert(state.isTimingOut() === true, "State is not timing out when data = None") + timingOutStates += state.get() + None + case _ => + fail("Unexpected test data") + } + } + + val updatedRecord = MapWithStateRDDRecord.updateRecordWithData[String, String, Int, Int]( + Some(record), dataIterator, testFunc, + Time(updatedTime), timeoutThreshold, removeTimedoutData) + + val updatedStateData = updatedRecord.stateMap.getAll().map { x => (x._2, x._3) } + assert(updatedStateData.toSet === expectedStates.toSet, + "states do not match after updating the MapWithStateRDDRecord") + + assert(updatedRecord.mappedData.toSet === expectedOutput.toSet, + "mapped data do not match after updating the MapWithStateRDDRecord") + + assert(timingOutStates.toSet === expectedTimingOutStates.toSet, "timing out states do not " + + "match those that were expected to do so while updating the MapWithStateRDDRecord") + + assert(removedStates.toSet === expectedRemovedStates.toSet, "removed states do not " + + "match those that were expected to do so while updating the MapWithStateRDDRecord") + + } + + // No data, no state should be changed, function should not be called, + assertRecordUpdate(initStates = Nil, data = None, expectedStates = Nil) + assert(functionCalled === false) + assertRecordUpdate(initStates = Seq(0), data = None, expectedStates = Seq((0, initialTime))) + assert(functionCalled === false) + + // Data present, function should be called irrespective of whether state exists + assertRecordUpdate(initStates = Seq(0), data = Seq("noop"), + expectedStates = Seq((0, initialTime))) + assert(functionCalled === true) + assertRecordUpdate(initStates = None, data = Some("noop"), expectedStates = None) + assert(functionCalled === true) + + // Function called with right state data + assertRecordUpdate(initStates = None, data = Seq("get-state"), + expectedStates = None, expectedOutput = Seq(-1)) + assertRecordUpdate(initStates = Seq(123), data = Seq("get-state"), + expectedStates = Seq((123, initialTime)), expectedOutput = Seq(123)) + + // Update state and timestamp, when timeout not present + assertRecordUpdate(initStates = Nil, data = Seq("update-state"), + expectedStates = Seq((0, updatedTime))) + assertRecordUpdate(initStates = Seq(0), data = Seq("update-state"), + expectedStates = Seq((1, updatedTime))) + + // Remove state + assertRecordUpdate(initStates = Seq(345), data = Seq("remove-state"), + expectedStates = Nil, expectedRemovedStates = Seq(345)) + + // State strictly older than timeout threshold should be timed out + assertRecordUpdate(initStates = Seq(123), data = Nil, + timeoutThreshold = Some(initialTime), removeTimedoutData = true, + expectedStates = Seq((123, initialTime)), expectedTimingOutStates = Nil) + + assertRecordUpdate(initStates = Seq(123), data = Nil, + timeoutThreshold = Some(initialTime + 1), removeTimedoutData = true, + expectedStates = Nil, expectedTimingOutStates = Seq(123)) + + // State should not be timed out after it has received data + assertRecordUpdate(initStates = Seq(123), data = Seq("noop"), + timeoutThreshold = Some(initialTime + 1), removeTimedoutData = true, + expectedStates = Seq((123, updatedTime)), expectedTimingOutStates = Nil) + assertRecordUpdate(initStates = Seq(123), data = Seq("remove-state"), + timeoutThreshold = Some(initialTime + 1), removeTimedoutData = true, + expectedStates = Nil, expectedTimingOutStates = Nil, expectedRemovedStates = Seq(123)) + + // If a state is not set but timeoutThreshold is defined, we should ignore this state. + // Previously it threw NoSuchElementException (SPARK-13195). + assertRecordUpdate(initStates = Seq(), data = Seq("noop"), + timeoutThreshold = Some(initialTime + 1), removeTimedoutData = true, + expectedStates = Nil, expectedTimingOutStates = Nil) + } + + test("states generated by MapWithStateRDD") { + val initStates = Seq(("k1", 0), ("k2", 0)) + val initTime = 123 + val initStateWthTime = initStates.map { x => (x._1, x._2, initTime) }.toSet + val partitioner = new HashPartitioner(2) + val initStateRDD = MapWithStateRDD.createFromPairRDD[String, Int, Int, Int]( + sc.parallelize(initStates), partitioner, Time(initTime)).persist() + assertRDD(initStateRDD, initStateWthTime, Set.empty) + + val updateTime = 345 + + /** + * Test that the test state RDD, when operated with new data, + * creates a new state RDD with expected states + */ + def testStateUpdates( + testStateRDD: MapWithStateRDD[String, Int, Int, Int], + testData: Seq[(String, Int)], + expectedStates: Set[(String, Int, Int)]): MapWithStateRDD[String, Int, Int, Int] = { + + // Persist the test MapWithStateRDD so that its not recomputed while doing the next operation. + // This is to make sure that we only touch which state keys are being touched in the next op. + testStateRDD.persist().count() + + // To track which keys are being touched + MapWithStateRDDSuite.touchedStateKeys.clear() + + val mappingFunction = (time: Time, key: String, data: Option[Int], state: State[Int]) => { + + // Track the key that has been touched + MapWithStateRDDSuite.touchedStateKeys += key + + // If the data is 0, do not do anything with the state + // else if the data is 1, increment the state if it exists, or set new state to 0 + // else if the data is 2, remove the state if it exists + data match { + case Some(1) => + if (state.exists()) { state.update(state.get + 1) } + else state.update(0) + case Some(2) => + state.remove() + case _ => + } + None.asInstanceOf[Option[Int]] // Do not return anything, not being tested + } + val newDataRDD = sc.makeRDD(testData).partitionBy(testStateRDD.partitioner.get) + + // Assert that the new state RDD has expected state data + val newStateRDD = assertOperation( + testStateRDD, newDataRDD, mappingFunction, updateTime, expectedStates, Set.empty) + + // Assert that the function was called only for the keys present in the data + assert(MapWithStateRDDSuite.touchedStateKeys.size === testData.size, + "More number of keys are being touched than that is expected") + assert(MapWithStateRDDSuite.touchedStateKeys.toSet === testData.toMap.keys, + "Keys not in the data are being touched unexpectedly") + + // Assert that the test RDD's data has not changed + assertRDD(initStateRDD, initStateWthTime, Set.empty) + newStateRDD + } + + // Test no-op, no state should change + testStateUpdates(initStateRDD, Seq(), initStateWthTime) // should not scan any state + testStateUpdates( + initStateRDD, Seq(("k1", 0)), initStateWthTime) // should not update existing state + testStateUpdates( + initStateRDD, Seq(("k3", 0)), initStateWthTime) // should not create new state + + // Test creation of new state + val rdd1 = testStateUpdates(initStateRDD, Seq(("k3", 1)), // should create k3's state as 0 + Set(("k1", 0, initTime), ("k2", 0, initTime), ("k3", 0, updateTime))) + + val rdd2 = testStateUpdates(rdd1, Seq(("k4", 1)), // should create k4's state as 0 + Set(("k1", 0, initTime), ("k2", 0, initTime), ("k3", 0, updateTime), ("k4", 0, updateTime))) + + // Test updating of state + val rdd3 = testStateUpdates( + initStateRDD, Seq(("k1", 1)), // should increment k1's state 0 -> 1 + Set(("k1", 1, updateTime), ("k2", 0, initTime))) + + val rdd4 = testStateUpdates(rdd3, + Seq(("x", 0), ("k2", 1), ("k2", 1), ("k3", 1)), // should update k2, 0 -> 2 and create k3, 0 + Set(("k1", 1, updateTime), ("k2", 2, updateTime), ("k3", 0, updateTime))) + + val rdd5 = testStateUpdates( + rdd4, Seq(("k3", 1)), // should update k3's state 0 -> 2 + Set(("k1", 1, updateTime), ("k2", 2, updateTime), ("k3", 1, updateTime))) + + // Test removing of state + val rdd6 = testStateUpdates( // should remove k1's state + initStateRDD, Seq(("k1", 2)), Set(("k2", 0, initTime))) + + val rdd7 = testStateUpdates( // should remove k2's state + rdd6, Seq(("k2", 2), ("k0", 2), ("k3", 1)), Set(("k3", 0, updateTime))) + + val rdd8 = testStateUpdates( // should remove k3's state + rdd7, Seq(("k3", 2)), Set()) + } + + test("checkpointing") { + /** + * This tests whether the MapWithStateRDD correctly truncates any references to its parent RDDs + * - the data RDD and the parent MapWithStateRDD. + */ + def rddCollectFunc(rdd: RDD[MapWithStateRDDRecord[Int, Int, Int]]) + : Set[(List[(Int, Int, Long)], List[Int])] = { + rdd.map { record => (record.stateMap.getAll().toList, record.mappedData.toList) } + .collect.toSet + } + + /** Generate MapWithStateRDD with data RDD having a long lineage */ + def makeStateRDDWithLongLineageDataRDD(longLineageRDD: RDD[Int]) + : MapWithStateRDD[Int, Int, Int, Int] = { + MapWithStateRDD.createFromPairRDD(longLineageRDD.map { _ -> 1}, partitioner, Time(0)) + } + + testRDD( + makeStateRDDWithLongLineageDataRDD, reliableCheckpoint = true, rddCollectFunc _) + testRDDPartitions( + makeStateRDDWithLongLineageDataRDD, reliableCheckpoint = true, rddCollectFunc _) + + /** Generate MapWithStateRDD with parent state RDD having a long lineage */ + def makeStateRDDWithLongLineageParenttateRDD( + longLineageRDD: RDD[Int]): MapWithStateRDD[Int, Int, Int, Int] = { + + // Create a MapWithStateRDD that has a long lineage using the data RDD with a long lineage + val stateRDDWithLongLineage = makeStateRDDWithLongLineageDataRDD(longLineageRDD) + + // Create a new MapWithStateRDD, with the lineage MapWithStateRDD as the parent + new MapWithStateRDD[Int, Int, Int, Int]( + stateRDDWithLongLineage, + stateRDDWithLongLineage.sparkContext.emptyRDD[(Int, Int)].partitionBy(partitioner), + (time: Time, key: Int, value: Option[Int], state: State[Int]) => None, + Time(10), + None + ) + } + + testRDD( + makeStateRDDWithLongLineageParenttateRDD, reliableCheckpoint = true, rddCollectFunc _) + testRDDPartitions( + makeStateRDDWithLongLineageParenttateRDD, reliableCheckpoint = true, rddCollectFunc _) + } + + test("checkpointing empty state RDD") { + val emptyStateRDD = MapWithStateRDD.createFromPairRDD[Int, Int, Int, Int]( + sc.emptyRDD[(Int, Int)], new HashPartitioner(10), Time(0)) + emptyStateRDD.checkpoint() + assert(emptyStateRDD.flatMap { _.stateMap.getAll() }.collect().isEmpty) + val cpRDD = sc.checkpointFile[MapWithStateRDDRecord[Int, Int, Int]]( + emptyStateRDD.getCheckpointFile.get) + assert(cpRDD.flatMap { _.stateMap.getAll() }.collect().isEmpty) + } + + /** Assert whether the `mapWithState` operation generates expected results */ + private def assertOperation[K: ClassTag, V: ClassTag, S: ClassTag, T: ClassTag]( + testStateRDD: MapWithStateRDD[K, V, S, T], + newDataRDD: RDD[(K, V)], + mappingFunction: (Time, K, Option[V], State[S]) => Option[T], + currentTime: Long, + expectedStates: Set[(K, S, Int)], + expectedMappedData: Set[T], + doFullScan: Boolean = false + ): MapWithStateRDD[K, V, S, T] = { + + val partitionedNewDataRDD = if (newDataRDD.partitioner != testStateRDD.partitioner) { + newDataRDD.partitionBy(testStateRDD.partitioner.get) + } else { + newDataRDD + } + + val newStateRDD = new MapWithStateRDD[K, V, S, T]( + testStateRDD, newDataRDD, mappingFunction, Time(currentTime), None) + if (doFullScan) newStateRDD.setFullScan() + + // Persist to make sure that it gets computed only once and we can track precisely how many + // state keys the computing touched + newStateRDD.persist().count() + assertRDD(newStateRDD, expectedStates, expectedMappedData) + newStateRDD + } + + /** Assert whether the [[MapWithStateRDD]] has the expected state and mapped data */ + private def assertRDD[K: ClassTag, V: ClassTag, S: ClassTag, T: ClassTag]( + stateRDD: MapWithStateRDD[K, V, S, T], + expectedStates: Set[(K, S, Int)], + expectedMappedData: Set[T]): Unit = { + val states = stateRDD.flatMap { _.stateMap.getAll() }.collect().toSet + val mappedData = stateRDD.flatMap { _.mappedData }.collect().toSet + assert(states === expectedStates, + "states after mapWithState operation were not as expected") + assert(mappedData === expectedMappedData, + "mapped data after mapWithState operation were not as expected") + } +} + +object MapWithStateRDDSuite { + private val touchedStateKeys = new ArrayBuffer[String]() +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala index cb017b798b2a..aa69be7ca993 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala @@ -23,10 +23,12 @@ import scala.util.Random import org.apache.hadoop.conf.Configuration import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} +import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} +import org.apache.spark.internal.config._ +import org.apache.spark.serializer.SerializerManager import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId} import org.apache.spark.streaming.util.{FileBasedWriteAheadLogSegment, FileBasedWriteAheadLogWriter} import org.apache.spark.util.Utils -import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} class WriteAheadLogBackedBlockRDDSuite extends SparkFunSuite with BeforeAndAfterAll with BeforeAndAfterEach { @@ -39,25 +41,51 @@ class WriteAheadLogBackedBlockRDDSuite var sparkContext: SparkContext = null var blockManager: BlockManager = null + var serializerManager: SerializerManager = null var dir: File = null override def beforeEach(): Unit = { + super.beforeEach() + initSparkContext() dir = Utils.createTempDir() } override def afterEach(): Unit = { - Utils.deleteRecursively(dir) + try { + Utils.deleteRecursively(dir) + } finally { + super.afterEach() + } } - override def beforeAll(): Unit = { - sparkContext = new SparkContext(conf) - blockManager = sparkContext.env.blockManager + override def afterAll(): Unit = { + try { + stopSparkContext() + } finally { + super.afterAll() + } } - override def afterAll(): Unit = { + private def initSparkContext(_conf: Option[SparkConf] = None): Unit = { + if (sparkContext == null) { + sparkContext = new SparkContext(_conf.getOrElse(conf)) + blockManager = sparkContext.env.blockManager + serializerManager = sparkContext.env.serializerManager + } + } + + private def stopSparkContext(): Unit = { // Copied from LocalSparkContext, simpler than to introduced test dependencies to core tests. - sparkContext.stop() - System.clearProperty("spark.driver.port") + try { + if (sparkContext != null) { + sparkContext.stop() + } + System.clearProperty("spark.driver.port") + blockManager = null + serializerManager = null + } finally { + sparkContext = null + } } test("Read data available in both block manager and write ahead log") { @@ -91,14 +119,23 @@ class WriteAheadLogBackedBlockRDDSuite numPartitions = 5, numPartitionsInBM = 0, numPartitionsInWAL = 5, testStoreInBM = true) } + test("read data in block manager and WAL with encryption on") { + stopSparkContext() + try { + val testConf = conf.clone().set(IO_ENCRYPTION_ENABLED, true) + initSparkContext(Some(testConf)) + testRDD(numPartitions = 5, numPartitionsInBM = 3, numPartitionsInWAL = 2) + } finally { + stopSparkContext() + } + } + /** * Test the WriteAheadLogBackedRDD, by writing some partitions of the data to block manager - * and the rest to a write ahead log, and then reading reading it all back using the RDD. + * and the rest to a write ahead log, and then reading it all back using the RDD. * It can also test if the partitions that were read from the log were again stored in * block manager. * - * - * * @param numPartitions Number of partitions in RDD * @param numPartitionsInBM Number of partitions to write to the BlockManager. * Partitions 0 to (numPartitionsInBM-1) will be written to BlockManager @@ -173,7 +210,7 @@ class WriteAheadLogBackedBlockRDDSuite assert(rdd.collect() === data.flatten) // Verify that the block fetching is skipped when isBlockValid is set to false. - // This is done by using a RDD whose data is only in memory but is set to skip block fetching + // This is done by using an RDD whose data is only in memory but is set to skip block fetching // Using that RDD will throw exception, as it skips block fetching even if the blocks are in // in BlockManager. if (testIsBlockValid) { @@ -213,7 +250,7 @@ class WriteAheadLogBackedBlockRDDSuite require(blockData.size === blockIds.size) val writer = new FileBasedWriteAheadLogWriter(new File(dir, "logFile").toString, hadoopConf) val segments = blockData.zip(blockIds).map { case (data, id) => - writer.write(blockManager.dataSerialize(id, data.iterator)) + writer.write(serializerManager.dataSerialize(id, data.iterator).toByteBuffer) } writer.close() segments diff --git a/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala index 2f11b255f110..898da4445e46 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala @@ -17,20 +17,25 @@ package org.apache.spark.streaming.receiver +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.collection.JavaConverters._ import scala.collection.mutable import org.scalatest.BeforeAndAfter import org.scalatest.Matchers._ -import org.scalatest.concurrent.Timeouts._ +import org.scalatest.concurrent.{Signaler, ThreadSignaler} import org.scalatest.concurrent.Eventually._ +import org.scalatest.concurrent.TimeLimits._ import org.scalatest.time.SpanSugar._ +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} import org.apache.spark.storage.StreamBlockId import org.apache.spark.util.ManualClock -import org.apache.spark.{SparkException, SparkConf, SparkFunSuite} class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter { + implicit val defaultSignaler: Signaler = ThreadSignaler private val blockIntervalMs = 10 private val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs}ms") @volatile private var blockGenerator: BlockGenerator = null @@ -83,29 +88,32 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter { assert(listener.onPushBlockCalled === true) } } - listener.pushedData should contain theSameElementsInOrderAs (data1) + listener.pushedData.asScala.toSeq should contain theSameElementsInOrderAs (data1) assert(listener.onAddDataCalled === false) // should be called only with addDataWithCallback() - // Verify addDataWithCallback() add data+metadata and and callbacks are called correctly + // Verify addDataWithCallback() add data+metadata and callbacks are called correctly val data2 = 11 to 20 val metadata2 = data2.map { _.toString } data2.zip(metadata2).foreach { case (d, m) => blockGenerator.addDataWithCallback(d, m) } assert(listener.onAddDataCalled === true) - listener.addedData should contain theSameElementsInOrderAs (data2) - listener.addedMetadata should contain theSameElementsInOrderAs (metadata2) + listener.addedData.asScala.toSeq should contain theSameElementsInOrderAs (data2) + listener.addedMetadata.asScala.toSeq should contain theSameElementsInOrderAs (metadata2) clock.advance(blockIntervalMs) // advance clock to generate blocks eventually(timeout(1 second)) { - listener.pushedData should contain theSameElementsInOrderAs (data1 ++ data2) + val combined = data1 ++ data2 + listener.pushedData.asScala.toSeq should contain theSameElementsInOrderAs combined } - // Verify addMultipleDataWithCallback() add data+metadata and and callbacks are called correctly + // Verify addMultipleDataWithCallback() add data+metadata and callbacks are called correctly val data3 = 21 to 30 val metadata3 = "metadata" blockGenerator.addMultipleDataWithCallback(data3.iterator, metadata3) - listener.addedMetadata should contain theSameElementsInOrderAs (metadata2 :+ metadata3) + val combinedMetadata = metadata2 :+ metadata3 + listener.addedMetadata.asScala.toSeq should contain theSameElementsInOrderAs (combinedMetadata) clock.advance(blockIntervalMs) // advance clock to generate blocks eventually(timeout(1 second)) { - listener.pushedData should contain theSameElementsInOrderAs (data1 ++ data2 ++ data3) + val combinedData = data1 ++ data2 ++ data3 + listener.pushedData.asScala.toSeq should contain theSameElementsInOrderAs (combinedData) } // Stop the block generator by starting the stop on a different thread and @@ -190,26 +198,22 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter { assert(thread.isAlive === false) } assert(blockGenerator.isStopped() === true) // generator has finally been completely stopped - assert(listener.pushedData === data, "All data not pushed by stop()") + assert(listener.pushedData.asScala.toSeq === data, "All data not pushed by stop()") } test("block push errors are reported") { val listener = new TestBlockGeneratorListener { - @volatile var errorReported = false override def onPushBlock( blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { throw new SparkException("test") } - override def onError(message: String, throwable: Throwable): Unit = { - errorReported = true - } } blockGenerator = new BlockGenerator(listener, 0, conf) blockGenerator.start() - assert(listener.errorReported === false) + assert(listener.onErrorCalled === false) blockGenerator.addData(1) eventually(timeout(1 second), interval(10 milliseconds)) { - assert(listener.errorReported === true) + assert(listener.onErrorCalled === true) } blockGenerator.stop() } @@ -230,24 +234,27 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter { /** A listener for BlockGenerator that records the data in the callbacks */ private class TestBlockGeneratorListener extends BlockGeneratorListener { - val pushedData = new mutable.ArrayBuffer[Any] with mutable.SynchronizedBuffer[Any] - val addedData = new mutable.ArrayBuffer[Any] with mutable.SynchronizedBuffer[Any] - val addedMetadata = new mutable.ArrayBuffer[Any] with mutable.SynchronizedBuffer[Any] + val pushedData = new ConcurrentLinkedQueue[Any] + val addedData = new ConcurrentLinkedQueue[Any] + val addedMetadata = new ConcurrentLinkedQueue[Any] @volatile var onGenerateBlockCalled = false @volatile var onAddDataCalled = false @volatile var onPushBlockCalled = false + @volatile var onErrorCalled = false override def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { - pushedData ++= arrayBuffer + pushedData.addAll(arrayBuffer.asJava) onPushBlockCalled = true } - override def onError(message: String, throwable: Throwable): Unit = {} + override def onError(message: String, throwable: Throwable): Unit = { + onErrorCalled = true + } override def onGenerateBlock(blockId: StreamBlockId): Unit = { onGenerateBlockCalled = true } override def onAddData(data: Any, metadata: Any): Unit = { - addedData += data - addedMetadata += metadata + addedData.add(data) + addedMetadata.add(metadata) onAddDataCalled = true } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/receiver/RateLimiterSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/receiver/RateLimiterSuite.scala index c6330eb3673f..ee3817c4b605 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/receiver/RateLimiterSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/receiver/RateLimiterSuite.scala @@ -25,21 +25,21 @@ class RateLimiterSuite extends SparkFunSuite { test("rate limiter initializes even without a maxRate set") { val conf = new SparkConf() - val rateLimiter = new RateLimiter(conf){} + val rateLimiter = new RateLimiter(conf) {} rateLimiter.updateRate(105) assert(rateLimiter.getCurrentLimit == 105) } test("rate limiter updates when below maxRate") { val conf = new SparkConf().set("spark.streaming.receiver.maxRate", "110") - val rateLimiter = new RateLimiter(conf){} + val rateLimiter = new RateLimiter(conf) {} rateLimiter.updateRate(105) assert(rateLimiter.getCurrentLimit == 105) } test("rate limiter stays below maxRate despite large updates") { val conf = new SparkConf().set("spark.streaming.receiver.maxRate", "100") - val rateLimiter = new RateLimiter(conf){} + val rateLimiter = new RateLimiter(conf) {} rateLimiter.updateRate(105) assert(rateLimiter.getCurrentLimit === 100) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala new file mode 100644 index 000000000000..8d81b582e4d3 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala @@ -0,0 +1,424 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.scheduler + +import org.mockito.Matchers.{eq => meq} +import org.mockito.Mockito._ +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, PrivateMethodTester} +import org.scalatest.concurrent.Eventually.{eventually, timeout} +import org.scalatest.mockito.MockitoSugar +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.{ExecutorAllocationClient, SparkConf, SparkFunSuite} +import org.apache.spark.streaming.{DummyInputDStream, Seconds, StreamingContext} +import org.apache.spark.util.{ManualClock, Utils} + + +class ExecutorAllocationManagerSuite extends SparkFunSuite + with BeforeAndAfter with BeforeAndAfterAll with MockitoSugar with PrivateMethodTester { + + import ExecutorAllocationManager._ + + private val batchDurationMillis = 1000L + private var allocationClient: ExecutorAllocationClient = null + private var clock: StreamManualClock = null + + before { + allocationClient = mock[ExecutorAllocationClient] + clock = new StreamManualClock() + } + + test("basic functionality") { + // Test that adding batch processing time info to allocation manager + // causes executors to be requested and killed accordingly + + // There is 1 receiver, and exec 1 has been allocated to it + withAllocationManager(numReceivers = 1) { case (receiverTracker, allocationManager) => + when(receiverTracker.allocatedExecutors).thenReturn(Map(1 -> Some("1"))) + + /** Add data point for batch processing time and verify executor allocation */ + def addBatchProcTimeAndVerifyAllocation(batchProcTimeMs: Double)(body: => Unit): Unit = { + // 2 active executors + reset(allocationClient) + when(allocationClient.getExecutorIds()).thenReturn(Seq("1", "2")) + addBatchProcTime(allocationManager, batchProcTimeMs.toLong) + val advancedTime = SCALING_INTERVAL_DEFAULT_SECS * 1000 + 1 + val expectedWaitTime = clock.getTimeMillis() + advancedTime + clock.advance(advancedTime) + // Make sure ExecutorAllocationManager.manageAllocation is called + eventually(timeout(10 seconds)) { + assert(clock.isStreamWaitingAt(expectedWaitTime)) + } + body + } + + /** Verify that the expected number of total executor were requested */ + def verifyTotalRequestedExecs(expectedRequestedTotalExecs: Option[Int]): Unit = { + if (expectedRequestedTotalExecs.nonEmpty) { + require(expectedRequestedTotalExecs.get > 0) + verify(allocationClient, times(1)).requestTotalExecutors( + meq(expectedRequestedTotalExecs.get), meq(0), meq(Map.empty)) + } else { + verify(allocationClient, never).requestTotalExecutors(0, 0, Map.empty) + } + } + + /** Verify that a particular executor was killed */ + def verifyKilledExec(expectedKilledExec: Option[String]): Unit = { + if (expectedKilledExec.nonEmpty) { + verify(allocationClient, times(1)).killExecutor(meq(expectedKilledExec.get)) + } else { + verify(allocationClient, never).killExecutor(null) + } + } + + // Batch proc time = batch interval, should increase allocation by 1 + addBatchProcTimeAndVerifyAllocation(batchDurationMillis) { + verifyTotalRequestedExecs(Some(3)) // one already allocated, increase allocation by 1 + verifyKilledExec(None) + } + + // Batch proc time = batch interval * 2, should increase allocation by 2 + addBatchProcTimeAndVerifyAllocation(batchDurationMillis * 2) { + verifyTotalRequestedExecs(Some(4)) + verifyKilledExec(None) + } + + // Batch proc time slightly more than the scale up ratio, should increase allocation by 1 + addBatchProcTimeAndVerifyAllocation(batchDurationMillis * SCALING_UP_RATIO_DEFAULT + 1) { + verifyTotalRequestedExecs(Some(3)) + verifyKilledExec(None) + } + + // Batch proc time slightly less than the scale up ratio, should not change allocation + addBatchProcTimeAndVerifyAllocation(batchDurationMillis * SCALING_UP_RATIO_DEFAULT - 1) { + verifyTotalRequestedExecs(None) + verifyKilledExec(None) + } + + // Batch proc time slightly more than the scale down ratio, should not change allocation + addBatchProcTimeAndVerifyAllocation(batchDurationMillis * SCALING_DOWN_RATIO_DEFAULT + 1) { + verifyTotalRequestedExecs(None) + verifyKilledExec(None) + } + + // Batch proc time slightly more than the scale down ratio, should not change allocation + addBatchProcTimeAndVerifyAllocation(batchDurationMillis * SCALING_DOWN_RATIO_DEFAULT - 1) { + verifyTotalRequestedExecs(None) + verifyKilledExec(Some("2")) + } + } + } + + test("requestExecutors policy") { + + /** Verify that the expected number of total executor were requested */ + def verifyRequestedExecs( + numExecs: Int, + numNewExecs: Int, + expectedRequestedTotalExecs: Int)( + implicit allocationManager: ExecutorAllocationManager): Unit = { + reset(allocationClient) + when(allocationClient.getExecutorIds()).thenReturn((1 to numExecs).map(_.toString)) + requestExecutors(allocationManager, numNewExecs) + verify(allocationClient, times(1)).requestTotalExecutors( + meq(expectedRequestedTotalExecs), meq(0), meq(Map.empty)) + } + + withAllocationManager(numReceivers = 1) { case (_, allocationManager) => + implicit val am = allocationManager + intercept[IllegalArgumentException] { + verifyRequestedExecs(numExecs = 0, numNewExecs = 0, 0) + } + verifyRequestedExecs(numExecs = 0, numNewExecs = 1, expectedRequestedTotalExecs = 1) + verifyRequestedExecs(numExecs = 1, numNewExecs = 1, expectedRequestedTotalExecs = 2) + verifyRequestedExecs(numExecs = 2, numNewExecs = 2, expectedRequestedTotalExecs = 4) + } + + withAllocationManager(numReceivers = 2) { case(_, allocationManager) => + implicit val am = allocationManager + + verifyRequestedExecs(numExecs = 0, numNewExecs = 1, expectedRequestedTotalExecs = 2) + verifyRequestedExecs(numExecs = 1, numNewExecs = 1, expectedRequestedTotalExecs = 2) + verifyRequestedExecs(numExecs = 2, numNewExecs = 2, expectedRequestedTotalExecs = 4) + } + + withAllocationManager( + // Test min 2 executors + new SparkConf().set("spark.streaming.dynamicAllocation.minExecutors", "2")) { + case (_, allocationManager) => + implicit val am = allocationManager + + verifyRequestedExecs(numExecs = 0, numNewExecs = 1, expectedRequestedTotalExecs = 2) + verifyRequestedExecs(numExecs = 0, numNewExecs = 3, expectedRequestedTotalExecs = 3) + verifyRequestedExecs(numExecs = 1, numNewExecs = 1, expectedRequestedTotalExecs = 2) + verifyRequestedExecs(numExecs = 1, numNewExecs = 2, expectedRequestedTotalExecs = 3) + verifyRequestedExecs(numExecs = 2, numNewExecs = 1, expectedRequestedTotalExecs = 3) + verifyRequestedExecs(numExecs = 2, numNewExecs = 2, expectedRequestedTotalExecs = 4) + } + + withAllocationManager( + // Test with max 2 executors + new SparkConf().set("spark.streaming.dynamicAllocation.maxExecutors", "2")) { + case (_, allocationManager) => + implicit val am = allocationManager + + verifyRequestedExecs(numExecs = 0, numNewExecs = 1, expectedRequestedTotalExecs = 1) + verifyRequestedExecs(numExecs = 0, numNewExecs = 3, expectedRequestedTotalExecs = 2) + verifyRequestedExecs(numExecs = 1, numNewExecs = 2, expectedRequestedTotalExecs = 2) + verifyRequestedExecs(numExecs = 2, numNewExecs = 1, expectedRequestedTotalExecs = 2) + verifyRequestedExecs(numExecs = 2, numNewExecs = 2, expectedRequestedTotalExecs = 2) + } + } + + test("killExecutor policy") { + + /** + * Verify that a particular executor was killed, given active executors and executors + * allocated to receivers. + */ + def verifyKilledExec( + execIds: Seq[String], + receiverExecIds: Map[Int, Option[String]], + expectedKilledExec: Option[String])( + implicit x: (ReceiverTracker, ExecutorAllocationManager)): Unit = { + val (receiverTracker, allocationManager) = x + + reset(allocationClient) + when(allocationClient.getExecutorIds()).thenReturn(execIds) + when(receiverTracker.allocatedExecutors).thenReturn(receiverExecIds) + killExecutor(allocationManager) + if (expectedKilledExec.nonEmpty) { + verify(allocationClient, times(1)).killExecutor(meq(expectedKilledExec.get)) + } else { + verify(allocationClient, never).killExecutor(null) + } + } + + withAllocationManager() { case (receiverTracker, allocationManager) => + implicit val rcvrTrackerAndExecAllocMgr = (receiverTracker, allocationManager) + + verifyKilledExec(Nil, Map.empty, None) + verifyKilledExec(Seq("1", "2"), Map.empty, None) + verifyKilledExec(Seq("1"), Map(1 -> Some("1")), None) + verifyKilledExec(Seq("1", "2"), Map(1 -> Some("1")), Some("2")) + verifyKilledExec(Seq("1", "2"), Map(1 -> Some("1"), 2 -> Some("2")), None) + } + + withAllocationManager( + new SparkConf().set("spark.streaming.dynamicAllocation.minExecutors", "2")) { + case (receiverTracker, allocationManager) => + implicit val rcvrTrackerAndExecAllocMgr = (receiverTracker, allocationManager) + + verifyKilledExec(Seq("1", "2"), Map.empty, None) + verifyKilledExec(Seq("1", "2", "3"), Map(1 -> Some("1"), 2 -> Some("2")), Some("3")) + } + } + + test("parameter validation") { + + def validateParams( + numReceivers: Int = 1, + scalingIntervalSecs: Option[Int] = None, + scalingUpRatio: Option[Double] = None, + scalingDownRatio: Option[Double] = None, + minExecs: Option[Int] = None, + maxExecs: Option[Int] = None): Unit = { + require(numReceivers > 0) + val receiverTracker = mock[ReceiverTracker] + when(receiverTracker.numReceivers()).thenReturn(numReceivers) + val conf = new SparkConf() + if (scalingIntervalSecs.nonEmpty) { + conf.set( + "spark.streaming.dynamicAllocation.scalingInterval", + s"${scalingIntervalSecs.get}s") + } + if (scalingUpRatio.nonEmpty) { + conf.set("spark.streaming.dynamicAllocation.scalingUpRatio", scalingUpRatio.get.toString) + } + if (scalingDownRatio.nonEmpty) { + conf.set( + "spark.streaming.dynamicAllocation.scalingDownRatio", + scalingDownRatio.get.toString) + } + if (minExecs.nonEmpty) { + conf.set("spark.streaming.dynamicAllocation.minExecutors", minExecs.get.toString) + } + if (maxExecs.nonEmpty) { + conf.set("spark.streaming.dynamicAllocation.maxExecutors", maxExecs.get.toString) + } + new ExecutorAllocationManager( + allocationClient, receiverTracker, conf, batchDurationMillis, clock) + } + + validateParams(numReceivers = 1) + validateParams(numReceivers = 2, minExecs = Some(1)) + validateParams(numReceivers = 2, minExecs = Some(3)) + validateParams(numReceivers = 2, maxExecs = Some(3)) + validateParams(numReceivers = 2, maxExecs = Some(1)) + validateParams(minExecs = Some(3), maxExecs = Some(3)) + validateParams(scalingIntervalSecs = Some(1)) + validateParams(scalingUpRatio = Some(1.1)) + validateParams(scalingDownRatio = Some(0.1)) + validateParams(scalingUpRatio = Some(1.1), scalingDownRatio = Some(0.1)) + + intercept[IllegalArgumentException] { + validateParams(minExecs = Some(0)) + } + intercept[IllegalArgumentException] { + validateParams(minExecs = Some(-1)) + } + intercept[IllegalArgumentException] { + validateParams(maxExecs = Some(0)) + } + intercept[IllegalArgumentException] { + validateParams(maxExecs = Some(-1)) + } + intercept[IllegalArgumentException] { + validateParams(minExecs = Some(4), maxExecs = Some(3)) + } + intercept[IllegalArgumentException] { + validateParams(scalingIntervalSecs = Some(-1)) + } + intercept[IllegalArgumentException] { + validateParams(scalingIntervalSecs = Some(0)) + } + intercept[IllegalArgumentException] { + validateParams(scalingUpRatio = Some(-0.1)) + } + intercept[IllegalArgumentException] { + validateParams(scalingUpRatio = Some(0)) + } + intercept[IllegalArgumentException] { + validateParams(scalingDownRatio = Some(-0.1)) + } + intercept[IllegalArgumentException] { + validateParams(scalingDownRatio = Some(0)) + } + intercept[IllegalArgumentException] { + validateParams(scalingUpRatio = Some(0.5), scalingDownRatio = Some(0.5)) + } + intercept[IllegalArgumentException] { + validateParams(scalingUpRatio = Some(0.3), scalingDownRatio = Some(0.5)) + } + } + + test("enabling and disabling") { + withStreamingContext(new SparkConf()) { ssc => + ssc.start() + assert(getExecutorAllocationManager(ssc).isEmpty) + } + + withStreamingContext( + new SparkConf().set("spark.streaming.dynamicAllocation.enabled", "true")) { ssc => + ssc.start() + assert(getExecutorAllocationManager(ssc).nonEmpty) + } + + val confWithBothDynamicAllocationEnabled = new SparkConf() + .set("spark.streaming.dynamicAllocation.enabled", "true") + .set("spark.dynamicAllocation.enabled", "true") + .set("spark.dynamicAllocation.testing", "true") + require(Utils.isDynamicAllocationEnabled(confWithBothDynamicAllocationEnabled) === true) + withStreamingContext(confWithBothDynamicAllocationEnabled) { ssc => + intercept[IllegalArgumentException] { + ssc.start() + } + } + } + + private def withAllocationManager( + conf: SparkConf = new SparkConf, + numReceivers: Int = 1 + )(body: (ReceiverTracker, ExecutorAllocationManager) => Unit): Unit = { + + val receiverTracker = mock[ReceiverTracker] + when(receiverTracker.numReceivers()).thenReturn(numReceivers) + + val manager = new ExecutorAllocationManager( + allocationClient, receiverTracker, conf, batchDurationMillis, clock) + try { + manager.start() + body(receiverTracker, manager) + } finally { + manager.stop() + } + } + + private val _addBatchProcTime = PrivateMethod[Unit]('addBatchProcTime) + private val _requestExecutors = PrivateMethod[Unit]('requestExecutors) + private val _killExecutor = PrivateMethod[Unit]('killExecutor) + private val _executorAllocationManager = + PrivateMethod[Option[ExecutorAllocationManager]]('executorAllocationManager) + + private def addBatchProcTime(manager: ExecutorAllocationManager, timeMs: Long): Unit = { + manager invokePrivate _addBatchProcTime(timeMs) + } + + private def requestExecutors(manager: ExecutorAllocationManager, newExecs: Int): Unit = { + manager invokePrivate _requestExecutors(newExecs) + } + + private def killExecutor(manager: ExecutorAllocationManager): Unit = { + manager invokePrivate _killExecutor() + } + + private def getExecutorAllocationManager( + ssc: StreamingContext): Option[ExecutorAllocationManager] = { + ssc.scheduler invokePrivate _executorAllocationManager() + } + + private def withStreamingContext(conf: SparkConf)(body: StreamingContext => Unit): Unit = { + conf.setMaster("myDummyLocalExternalClusterManager") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.dynamicAllocation.testing", "true") // to test dynamic allocation + + var ssc: StreamingContext = null + try { + ssc = new StreamingContext(conf, Seconds(1)) + new DummyInputDStream(ssc).foreachRDD(_ => { }) + body(ssc) + } finally { + if (ssc != null) ssc.stop() + } + } +} + +/** + * A special manual clock that provide `isStreamWaitingAt` to allow the user to check if the clock + * is blocking. + */ +class StreamManualClock(time: Long = 0L) extends ManualClock(time) with Serializable { + private var waitStartTime: Option[Long] = None + + override def waitTillTime(targetTime: Long): Long = synchronized { + try { + waitStartTime = Some(getTimeMillis()) + super.waitTillTime(targetTime) + } finally { + waitStartTime = None + } + } + + /** + * Returns if the clock is blocking and the time it started to block is the parameter `time`. + */ + def isStreamWaitingAt(time: Long): Boolean = synchronized { + waitStartTime == Some(time) + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/InputInfoTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/InputInfoTrackerSuite.scala index f5248acf712b..a7e365649d3e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/InputInfoTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/InputInfoTrackerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.streaming.scheduler import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.streaming.{Time, Duration, StreamingContext} +import org.apache.spark.streaming.{Duration, StreamingContext, Time} class InputInfoTrackerSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala index 9b6cd4bc4e31..5f7f7fa5e67f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala @@ -56,8 +56,7 @@ class JobGeneratorSuite extends TestSuiteBase { // 4. allow subsequent batches to be generated (to allow premature deletion of 3rd batch metadata) // 5. verify whether 3rd batch's block metadata still exists // - // TODO: SPARK-7420 enable this test - ignore("SPARK-6222: Do not clear received block data too soon") { + test("SPARK-6222: Do not clear received block data too soon") { import JobGeneratorSuite._ val checkpointDir = Utils.createTempDir() val testConf = conf @@ -124,6 +123,7 @@ class JobGeneratorSuite extends TestSuiteBase { assert(getBlocksOfBatch(longBatchTime).nonEmpty, "blocks of incomplete batch already deleted") assert(batchCounter.getNumCompletedBatches < longBatchNumber) waitLatch.countDown() + ssc.stop() } } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/RateControllerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/RateControllerSuite.scala index 1eb52b7029a2..37ca0ce2f6a3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/RateControllerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/RateControllerSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.streaming.scheduler -import scala.collection.mutable - import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala index 3bd8d086abf7..c206d3169d77 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart, TaskLo import org.apache.spark.scheduler.TaskLocality.TaskLocality import org.apache.spark.storage.{StorageLevel, StreamBlockId} import org.apache.spark.streaming._ -import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.streaming.dstream.{ConstantInputDStream, ReceiverInputDStream} import org.apache.spark.streaming.receiver._ /** Testsuite for receiver scheduling */ @@ -34,8 +34,6 @@ class ReceiverTrackerSuite extends TestSuiteBase { test("send rate update to receivers") { withStreamingContext(new StreamingContext(conf, Milliseconds(100))) { ssc => - ssc.scheduler.listenerBus.start(ssc.sc) - val newRateLimit = 100L val inputDStream = new RateTestInputDStream(ssc) val tracker = new ReceiverTracker(ssc) @@ -59,6 +57,8 @@ class ReceiverTrackerSuite extends TestSuiteBase { } } finally { tracker.stop(false) + // Make sure it is idempotent. + tracker.stop(false) } } } @@ -104,11 +104,32 @@ class ReceiverTrackerSuite extends TestSuiteBase { } } } + + test("get allocated executors") { + // Test get allocated executors when 1 receiver is registered + withStreamingContext(new StreamingContext(conf, Milliseconds(100))) { ssc => + val input = ssc.receiverStream(new TestReceiver) + val output = new TestOutputStream(input) + output.register() + ssc.start() + assert(ssc.scheduler.receiverTracker.allocatedExecutors().size === 1) + } + + // Test get allocated executors when there's no receiver registered + withStreamingContext(new StreamingContext(conf, Milliseconds(100))) { ssc => + val rdd = ssc.sc.parallelize(1 to 10) + val input = new ConstantInputDStream(ssc, rdd) + val output = new TestOutputStream(input) + output.register() + ssc.start() + assert(ssc.scheduler.receiverTracker.allocatedExecutors() === Map.empty) + } + } } /** An input DStream with for testing rate controlling */ -private[streaming] class RateTestInputDStream(@transient ssc_ : StreamingContext) - extends ReceiverInputDStream[Int](ssc_) { +private[streaming] class RateTestInputDStream(_ssc: StreamingContext) + extends ReceiverInputDStream[Int](_ssc) { override def getReceiver(): Receiver[Int] = new RateTestReceiver(id) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala index a1af95be81c8..1a0460cd669a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimatorSuite.scala @@ -119,7 +119,7 @@ class PIDRateEstimatorSuite extends SparkFunSuite with Matchers { test("with no accumulated but some positive error, |I| > 0, follow the processing speed") { val p = new PIDRateEstimator(20, 1D, 1D, 0D, 10) - // prepare a series of batch updates, one every 20ms with an decreasing number of processed + // prepare a series of batch updates, one every 20ms with a decreasing number of processed // elements in each batch, but constant processing time, and no accumulated error. Even though // the integral part is non-zero, the estimated rate should follow only the proportional term, // asking for less and less elements diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala index af4718b4eb70..56b400850fdd 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala @@ -62,12 +62,17 @@ class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { 0 -> StreamInputInfo(0, 300L), 1 -> StreamInputInfo(1, 300L, Map(StreamInputInfo.METADATA_KEY_DESCRIPTION -> "test"))) + // onStreamingStarted + listener.onStreamingStarted(StreamingListenerStreamingStarted(100L)) + listener.startTime should be (100) + // onBatchSubmitted val batchInfoSubmitted = BatchInfo(Time(1000), streamIdToInputInfo, 1000, None, None, Map.empty) listener.onBatchSubmitted(StreamingListenerBatchSubmitted(batchInfoSubmitted)) listener.waitingBatches should be (List(BatchUIData(batchInfoSubmitted))) listener.runningBatches should be (Nil) listener.retainedCompletedBatches should be (Nil) + listener.lastReceivedBatch should be (Some(BatchUIData(batchInfoSubmitted))) listener.lastCompletedBatch should be (None) listener.numUnprocessedBatches should be (1) listener.numTotalCompletedBatches should be (0) @@ -81,6 +86,7 @@ class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { listener.waitingBatches should be (Nil) listener.runningBatches should be (List(BatchUIData(batchInfoStarted))) listener.retainedCompletedBatches should be (Nil) + listener.lastReceivedBatch should be (Some(BatchUIData(batchInfoStarted))) listener.lastCompletedBatch should be (None) listener.numUnprocessedBatches should be (1) listener.numTotalCompletedBatches should be (0) @@ -123,6 +129,7 @@ class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { listener.waitingBatches should be (Nil) listener.runningBatches should be (Nil) listener.retainedCompletedBatches should be (List(BatchUIData(batchInfoCompleted))) + listener.lastReceivedBatch should be (Some(BatchUIData(batchInfoCompleted))) listener.lastCompletedBatch should be (Some(BatchUIData(batchInfoCompleted))) listener.numUnprocessedBatches should be (0) listener.numTotalCompletedBatches should be (1) @@ -130,20 +137,20 @@ class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { listener.numTotalReceivedRecords should be (600) // onReceiverStarted - val receiverInfoStarted = ReceiverInfo(0, "test", true, "localhost") + val receiverInfoStarted = ReceiverInfo(0, "test", true, "localhost", "0") listener.onReceiverStarted(StreamingListenerReceiverStarted(receiverInfoStarted)) listener.receiverInfo(0) should be (Some(receiverInfoStarted)) listener.receiverInfo(1) should be (None) // onReceiverError - val receiverInfoError = ReceiverInfo(1, "test", true, "localhost") + val receiverInfoError = ReceiverInfo(1, "test", true, "localhost", "1") listener.onReceiverError(StreamingListenerReceiverError(receiverInfoError)) listener.receiverInfo(0) should be (Some(receiverInfoStarted)) listener.receiverInfo(1) should be (Some(receiverInfoError)) listener.receiverInfo(2) should be (None) // onReceiverStopped - val receiverInfoStopped = ReceiverInfo(2, "test", true, "localhost") + val receiverInfoStopped = ReceiverInfo(2, "test", true, "localhost", "2") listener.onReceiverStopped(StreamingListenerReceiverStopped(receiverInfoStopped)) listener.receiverInfo(0) should be (Some(receiverInfoStarted)) listener.receiverInfo(1) should be (Some(receiverInfoError)) @@ -200,7 +207,7 @@ class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { batchUIData.get.totalDelay should be (batchInfoSubmitted.totalDelay) batchUIData.get.streamIdToInputInfo should be (Map.empty) batchUIData.get.numRecords should be (0) - batchUIData.get.outputOpIdSparkJobIdPairs should be (Seq(OutputOpIdAndSparkJobId(0, 0))) + batchUIData.get.outputOpIdSparkJobIdPairs.toSeq should be (Seq(OutputOpIdAndSparkJobId(0, 0))) // A lot of "onBatchCompleted"s happen before "onJobStart" for(i <- limit + 1 to limit * 2) { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala index 78fc344b0017..6d9c80d99206 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.util import java.io.ByteArrayOutputStream +import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit._ import org.apache.spark.SparkFunSuite @@ -34,7 +35,7 @@ class RateLimitedOutputStreamSuite extends SparkFunSuite { val underlying = new ByteArrayOutputStream val data = "X" * 41000 val stream = new RateLimitedOutputStream(underlying, desiredBytesPerSec = 10000) - val elapsedNs = benchmark { stream.write(data.getBytes("UTF-8")) } + val elapsedNs = benchmark { stream.write(data.getBytes(StandardCharsets.UTF_8)) } val seconds = SECONDS.convert(elapsedNs, NANOSECONDS) assert(seconds >= 4, s"Seconds value ($seconds) is less than 4.") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/RecurringTimerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/RecurringTimerSuite.scala index 0544972d95c0..25b70a3d089e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/RecurringTimerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/RecurringTimerSuite.scala @@ -17,7 +17,9 @@ package org.apache.spark.streaming.util -import scala.collection.mutable +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.collection.JavaConverters._ import scala.concurrent.duration._ import org.scalatest.PrivateMethodTester @@ -30,34 +32,34 @@ class RecurringTimerSuite extends SparkFunSuite with PrivateMethodTester { test("basic") { val clock = new ManualClock() - val results = new mutable.ArrayBuffer[Long]() with mutable.SynchronizedBuffer[Long] + val results = new ConcurrentLinkedQueue[Long]() val timer = new RecurringTimer(clock, 100, time => { - results += time + results.add(time) }, "RecurringTimerSuite-basic") timer.start(0) eventually(timeout(10.seconds), interval(10.millis)) { - assert(results === Seq(0L)) + assert(results.asScala.toSeq === Seq(0L)) } clock.advance(100) eventually(timeout(10.seconds), interval(10.millis)) { - assert(results === Seq(0L, 100L)) + assert(results.asScala.toSeq === Seq(0L, 100L)) } clock.advance(200) eventually(timeout(10.seconds), interval(10.millis)) { - assert(results === Seq(0L, 100L, 200L, 300L)) + assert(results.asScala.toSeq === Seq(0L, 100L, 200L, 300L)) } assert(timer.stop(interruptTimer = true) === 300L) } test("SPARK-10224: call 'callback' after stopping") { val clock = new ManualClock() - val results = new mutable.ArrayBuffer[Long]() with mutable.SynchronizedBuffer[Long] + val results = new ConcurrentLinkedQueue[Long] val timer = new RecurringTimer(clock, 100, time => { - results += time + results.add(time) }, "RecurringTimerSuite-SPARK-10224") timer.start(0) eventually(timeout(10.seconds), interval(10.millis)) { - assert(results === Seq(0L)) + assert(results.asScala.toSeq === Seq(0L)) } @volatile var lastTime = -1L // Now RecurringTimer is waiting for the next interval @@ -77,7 +79,7 @@ class RecurringTimerSuite extends SparkFunSuite with PrivateMethodTester { // Then it will find `stopped` is true and exit the loop, but it should call `callback` again // before exiting its internal thread. thread.join() - assert(results === Seq(0L, 100L, 200L)) + assert(results.asScala.toSeq === Seq(0L, 100L, 200L)) assert(lastTime === 200L) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 93ae41a3d2ec..4a2549fc0a96 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -18,31 +18,45 @@ package org.apache.spark.streaming.util import java.io._ import java.nio.ByteBuffer -import java.util +import java.util.{Iterator => JIterator} +import java.util.concurrent.{CountDownLatch, RejectedExecutionException, ThreadPoolExecutor, TimeUnit} +import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer +import scala.concurrent._ import scala.concurrent.duration._ import scala.language.{implicitConversions, postfixOps} -import scala.reflect.ClassTag import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.mockito.ArgumentCaptor +import org.mockito.Matchers.{eq => meq, _} +import org.mockito.Mockito._ +import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach, PrivateMethodTester} +import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually._ -import org.scalatest.BeforeAndAfter +import org.scalatest.mockito.MockitoSugar -import org.apache.spark.util.{ManualClock, Utils} import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.util.{CompletionIterator, ManualClock, ThreadUtils, Utils} -class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter { +/** Common tests for WriteAheadLogs that we would like to test with different configurations. */ +abstract class CommonWriteAheadLogTests( + allowBatching: Boolean, + closeFileAfterWrite: Boolean, + testTag: String = "") + extends SparkFunSuite with BeforeAndAfter { import WriteAheadLogSuite._ - val hadoopConf = new Configuration() - var tempDir: File = null - var testDir: String = null - var testFile: String = null - var writeAheadLog: FileBasedWriteAheadLog = null + protected val hadoopConf = new Configuration() + protected var tempDir: File = null + protected var testDir: String = null + protected var testFile: String = null + protected var writeAheadLog: WriteAheadLog = null + protected def testPrefix = if (testTag != "") testTag + " - " else testTag before { tempDir = Utils.createTempDir() @@ -58,47 +72,214 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter { Utils.deleteRecursively(tempDir) } - test("WriteAheadLogUtils - log selection and creation") { - val logDir = Utils.createTempDir().getAbsolutePath() + test(testPrefix + "read all logs") { + // Write data manually for testing reading through WriteAheadLog + val writtenData = (1 to 10).flatMap { i => + val data = generateRandomData() + val file = testDir + s"/log-$i-$i" + writeDataManually(data, file, allowBatching) + data + } + + val logDirectoryPath = new Path(testDir) + val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) + assert(fileSystem.exists(logDirectoryPath) === true) + + // Read data using manager and verify + val readData = readDataUsingWriteAheadLog(testDir, closeFileAfterWrite, allowBatching) + assert(readData === writtenData) + } + + test(testPrefix + "write logs") { + // Write data with rotation using WriteAheadLog class + val dataToWrite = generateRandomData() + writeDataUsingWriteAheadLog(testDir, dataToWrite, closeFileAfterWrite = closeFileAfterWrite, + allowBatching = allowBatching) + + // Read data manually to verify the written data + val logFiles = getLogFilesInDirectory(testDir) + assert(logFiles.size > 1) + val writtenData = readAndDeserializeDataManually(logFiles, allowBatching) + assert(writtenData === dataToWrite) + } + + test(testPrefix + "read all logs after write") { + // Write data with manager, recover with new manager and verify + val dataToWrite = generateRandomData() + writeDataUsingWriteAheadLog(testDir, dataToWrite, closeFileAfterWrite, allowBatching) + val logFiles = getLogFilesInDirectory(testDir) + assert(logFiles.size > 1) + val readData = readDataUsingWriteAheadLog(testDir, closeFileAfterWrite, allowBatching) + assert(dataToWrite === readData) + } + + test(testPrefix + "clean old logs") { + logCleanUpTest(waitForCompletion = false) + } + + test(testPrefix + "clean old logs synchronously") { + logCleanUpTest(waitForCompletion = true) + } + + private def logCleanUpTest(waitForCompletion: Boolean): Unit = { + // Write data with manager, recover with new manager and verify + val manualClock = new ManualClock + val dataToWrite = generateRandomData() + writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, closeFileAfterWrite, + allowBatching, manualClock, closeLog = false) + val logFiles = getLogFilesInDirectory(testDir) + assert(logFiles.size > 1) + + writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion) + + if (waitForCompletion) { + assert(getLogFilesInDirectory(testDir).size < logFiles.size) + } else { + eventually(Eventually.timeout(1 second), interval(10 milliseconds)) { + assert(getLogFilesInDirectory(testDir).size < logFiles.size) + } + } + writeAheadLog.close() + // Make sure it is idempotent. + writeAheadLog.close() + } + + test(testPrefix + "handling file errors while reading rotating logs") { + // Generate a set of log files + val manualClock = new ManualClock + val dataToWrite1 = generateRandomData() + writeDataUsingWriteAheadLog(testDir, dataToWrite1, closeFileAfterWrite, allowBatching, + manualClock) + val logFiles1 = getLogFilesInDirectory(testDir) + assert(logFiles1.size > 1) - def assertDriverLogClass[T <: WriteAheadLog: ClassTag](conf: SparkConf): WriteAheadLog = { - val log = WriteAheadLogUtils.createLogForDriver(conf, logDir, hadoopConf) - assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) - log + + // Recover old files and generate a second set of log files + val dataToWrite2 = generateRandomData() + manualClock.advance(100000) + writeDataUsingWriteAheadLog(testDir, dataToWrite2, closeFileAfterWrite, allowBatching, + manualClock) + val logFiles2 = getLogFilesInDirectory(testDir) + assert(logFiles2.size > logFiles1.size) + + // Read the files and verify that all the written data can be read + val readData1 = readDataUsingWriteAheadLog(testDir, closeFileAfterWrite, allowBatching) + assert(readData1 === (dataToWrite1 ++ dataToWrite2)) + + // Corrupt the first set of files so that they are basically unreadable + logFiles1.foreach { f => + val raf = new FileOutputStream(f, true).getChannel() + raf.truncate(1) + raf.close() } - def assertReceiverLogClass[T: ClassTag](conf: SparkConf): WriteAheadLog = { - val log = WriteAheadLogUtils.createLogForReceiver(conf, logDir, hadoopConf) - assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) - log + // Verify that the corrupted files do not prevent reading of the second set of data + val readData = readDataUsingWriteAheadLog(testDir, closeFileAfterWrite, allowBatching) + assert(readData === dataToWrite2) + } + + test(testPrefix + "do not create directories or files unless write") { + val nonexistentTempPath = File.createTempFile("test", "") + nonexistentTempPath.delete() + assert(!nonexistentTempPath.exists()) + + val writtenSegment = writeDataManually(generateRandomData(), testFile, allowBatching) + val wal = createWriteAheadLog(testDir, closeFileAfterWrite, allowBatching) + assert(!nonexistentTempPath.exists(), "Directory created just by creating log object") + if (allowBatching) { + intercept[UnsupportedOperationException](wal.read(writtenSegment.head)) + } else { + wal.read(writtenSegment.head) } + assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment") + } - val emptyConf = new SparkConf() // no log configuration - assertDriverLogClass[FileBasedWriteAheadLog](emptyConf) - assertReceiverLogClass[FileBasedWriteAheadLog](emptyConf) + test(testPrefix + "parallel recovery not enabled if closeFileAfterWrite = false") { + // write some data + val writtenData = (1 to 10).flatMap { i => + val data = generateRandomData() + val file = testDir + s"/log-$i-$i" + writeDataManually(data, file, allowBatching) + data + } - // Verify setting driver WAL class - val conf1 = new SparkConf().set("spark.streaming.driver.writeAheadLog.class", - classOf[MockWriteAheadLog0].getName()) - assertDriverLogClass[MockWriteAheadLog0](conf1) - assertReceiverLogClass[FileBasedWriteAheadLog](conf1) + val wal = createWriteAheadLog(testDir, closeFileAfterWrite, allowBatching) + // create iterator but don't materialize it + val readData = wal.readAll().asScala.map(byteBufferToString) + wal.close() + if (closeFileAfterWrite) { + // the threadpool is shutdown by the wal.close call above, therefore we shouldn't be able + // to materialize the iterator with parallel recovery + intercept[RejectedExecutionException](readData.toArray) + } else { + assert(readData.toSeq === writtenData) + } + } +} - // Verify setting receiver WAL class - val receiverWALConf = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", - classOf[MockWriteAheadLog0].getName()) - assertDriverLogClass[FileBasedWriteAheadLog](receiverWALConf) - assertReceiverLogClass[MockWriteAheadLog0](receiverWALConf) +class FileBasedWriteAheadLogSuite + extends CommonWriteAheadLogTests(false, false, "FileBasedWriteAheadLog") { - // Verify setting receiver WAL class with 1-arg constructor - val receiverWALConf2 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", - classOf[MockWriteAheadLog1].getName()) - assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf2) + import WriteAheadLogSuite._ - // Verify failure setting receiver WAL class with 2-arg constructor - intercept[SparkException] { - val receiverWALConf3 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", - classOf[MockWriteAheadLog2].getName()) - assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf3) + test("FileBasedWriteAheadLog - seqToParIterator") { + /* + If the setting `closeFileAfterWrite` is enabled, we start generating a very large number of + files. This causes recovery to take a very long time. In order to make it quicker, we + parallelized the reading of these files. This test makes sure that we limit the number of + open files to the size of the number of threads in our thread pool rather than the size of + the list of files. + */ + val numThreads = 8 + val fpool = ThreadUtils.newForkJoinPool("wal-test-thread-pool", numThreads) + val executionContext = ExecutionContext.fromExecutorService(fpool) + + class GetMaxCounter { + private val value = new AtomicInteger() + @volatile private var max: Int = 0 + def increment(): Unit = synchronized { + val atInstant = value.incrementAndGet() + if (atInstant > max) max = atInstant + } + def decrement(): Unit = synchronized { value.decrementAndGet() } + def get(): Int = synchronized { value.get() } + def getMax(): Int = synchronized { max } + } + try { + // If Jenkins is slow, we may not have a chance to run many threads simultaneously. Having + // a latch will make sure that all the threads can be launched altogether. + val latch = new CountDownLatch(1) + val testSeq = 1 to 1000 + val counter = new GetMaxCounter() + def handle(value: Int): Iterator[Int] = { + new CompletionIterator[Int, Iterator[Int]](Iterator(value)) { + counter.increment() + // block so that other threads also launch + latch.await(10, TimeUnit.SECONDS) + override def completion() { counter.decrement() } + } + } + @volatile var collected: Seq[Int] = Nil + val t = new Thread() { + override def run() { + // run the calculation on a separate thread so that we can release the latch + val iterator = FileBasedWriteAheadLog.seqToParIterator[Int, Int](executionContext, + testSeq, handle) + collected = iterator.toSeq + } + } + t.start() + eventually(Eventually.timeout(10.seconds)) { + // make sure we are doing a parallel computation! + assert(counter.getMax() > 1) + } + latch.countDown() + t.join(10000) + assert(collected === testSeq) + // make sure we didn't open too many Iterators + assert(counter.getMax() <= numThreads) + } finally { + fpool.shutdownNow() } } @@ -122,7 +303,7 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter { test("FileBasedWriteAheadLogReader - sequentially reading data") { val writtenData = generateRandomData() - writeDataManually(writtenData, testFile) + writeDataManually(writtenData, testFile, allowBatching = false) val reader = new FileBasedWriteAheadLogReader(testFile, hadoopConf) val readData = reader.toSeq.map(byteBufferToString) assert(readData === writtenData) @@ -163,10 +344,30 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter { assert(readDataUsingReader(testFile) === (dataToWrite.dropRight(1))) } + test("FileBasedWriteAheadLogReader - handles errors when file doesn't exist") { + // Write data manually for testing the sequential reader + val dataToWrite = generateRandomData() + writeDataUsingWriter(testFile, dataToWrite) + val tFile = new File(testFile) + assert(tFile.exists()) + // Verify the data can be read and is same as the one correctly written + assert(readDataUsingReader(testFile) === dataToWrite) + + tFile.delete() + assert(!tFile.exists()) + + val reader = new FileBasedWriteAheadLogReader(testFile, hadoopConf) + assert(!reader.hasNext) + reader.close() + + // Verify that no exception is thrown if file doesn't exist + assert(readDataUsingReader(testFile) === Nil) + } + test("FileBasedWriteAheadLogRandomReader - reading data using random reader") { // Write data manually for testing the random reader val writtenData = generateRandomData() - val segments = writeDataManually(writtenData, testFile) + val segments = writeDataManually(writtenData, testFile, allowBatching = false) // Get a random order of these segments and read them back val writtenDataAndSegments = writtenData.zip(segments).toSeq.permutations.take(10).flatten @@ -190,163 +391,225 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter { } reader.close() } +} - test("FileBasedWriteAheadLog - write rotating logs") { - // Write data with rotation using WriteAheadLog class - val dataToWrite = generateRandomData() - writeDataUsingWriteAheadLog(testDir, dataToWrite) - - // Read data manually to verify the written data - val logFiles = getLogFilesInDirectory(testDir) - assert(logFiles.size > 1) - val writtenData = logFiles.flatMap { file => readDataManually(file)} - assert(writtenData === dataToWrite) - } +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String) + extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) { - test("FileBasedWriteAheadLog - close after write flag") { + import WriteAheadLogSuite._ + test(testPrefix + "close after write flag") { // Write data with rotation using WriteAheadLog class val numFiles = 3 val dataToWrite = Seq.tabulate(numFiles)(_.toString) // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100, - closeFileAfterWrite = true) + closeFileAfterWrite = true, allowBatching = allowBatching) // Read data manually to verify the written data val logFiles = getLogFilesInDirectory(testDir) assert(logFiles.size === numFiles) - val writtenData = logFiles.flatMap { file => readDataManually(file)} + val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching) assert(writtenData === dataToWrite) } +} - test("FileBasedWriteAheadLog - read rotating logs") { - // Write data manually for testing reading through WriteAheadLog - val writtenData = (1 to 10).map { i => - val data = generateRandomData() - val file = testDir + s"/log-$i-$i" - writeDataManually(data, file) - data - }.flatten +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite + extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog") - val logDirectoryPath = new Path(testDir) - val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) - assert(fileSystem.exists(logDirectoryPath) === true) +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( + allowBatching = true, + closeFileAfterWrite = false, + "BatchedWriteAheadLog") + with MockitoSugar + with BeforeAndAfterEach + with Eventually + with PrivateMethodTester { - // Read data using manager and verify - val readData = readDataUsingWriteAheadLog(testDir) - assert(readData === writtenData) - } + import BatchedWriteAheadLog._ + import WriteAheadLogSuite._ - test("FileBasedWriteAheadLog - recover past logs when creating new manager") { - // Write data with manager, recover with new manager and verify - val dataToWrite = generateRandomData() - writeDataUsingWriteAheadLog(testDir, dataToWrite) - val logFiles = getLogFilesInDirectory(testDir) - assert(logFiles.size > 1) - val readData = readDataUsingWriteAheadLog(testDir) - assert(dataToWrite === readData) - } + private var wal: WriteAheadLog = _ + private var walHandle: WriteAheadLogRecordHandle = _ + private var walBatchingThreadPool: ThreadPoolExecutor = _ + private var walBatchingExecutionContext: ExecutionContextExecutorService = _ + private val sparkConf = new SparkConf() - test("FileBasedWriteAheadLog - clean old logs") { - logCleanUpTest(waitForCompletion = false) - } + private val queueLength = PrivateMethod[Int]('getQueueLength) - test("FileBasedWriteAheadLog - clean old logs synchronously") { - logCleanUpTest(waitForCompletion = true) + override def beforeEach(): Unit = { + super.beforeEach() + wal = mock[WriteAheadLog] + walHandle = mock[WriteAheadLogRecordHandle] + walBatchingThreadPool = ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool") + walBatchingExecutionContext = ExecutionContext.fromExecutorService(walBatchingThreadPool) } - private def logCleanUpTest(waitForCompletion: Boolean): Unit = { - // Write data with manager, recover with new manager and verify - val manualClock = new ManualClock - val dataToWrite = generateRandomData() - writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false) - val logFiles = getLogFilesInDirectory(testDir) - assert(logFiles.size > 1) - - writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion) - - if (waitForCompletion) { - assert(getLogFilesInDirectory(testDir).size < logFiles.size) - } else { - eventually(timeout(1 second), interval(10 milliseconds)) { - assert(getLogFilesInDirectory(testDir).size < logFiles.size) + override def afterEach(): Unit = { + try { + if (walBatchingExecutionContext != null) { + walBatchingExecutionContext.shutdownNow() } + } finally { + super.afterEach() } } - test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") { - // Generate a set of log files - val manualClock = new ManualClock - val dataToWrite1 = generateRandomData() - writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock) - val logFiles1 = getLogFilesInDirectory(testDir) - assert(logFiles1.size > 1) + test("BatchedWriteAheadLog - serializing and deserializing batched records") { + val events = Seq( + BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)), + BatchAllocationEvent(null, null), + BatchCleanupEvent(Nil) + ) + val buffers = events.map(e => Record(ByteBuffer.wrap(Utils.serialize(e)), 0L, null)) + val batched = BatchedWriteAheadLog.aggregate(buffers) + val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer => + Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array())) - // Recover old files and generate a second set of log files - val dataToWrite2 = generateRandomData() - manualClock.advance(100000) - writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock) - val logFiles2 = getLogFilesInDirectory(testDir) - assert(logFiles2.size > logFiles1.size) + assert(deaggregate.toSeq === events) + } - // Read the files and verify that all the written data can be read - val readData1 = readDataUsingWriteAheadLog(testDir) - assert(readData1 === (dataToWrite1 ++ dataToWrite2)) + test("BatchedWriteAheadLog - failures in wrappedLog get bubbled up") { + when(wal.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!")) + // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes + val batchedWal = new BatchedWriteAheadLog(wal, sparkConf) - // Corrupt the first set of files so that they are basically unreadable - logFiles1.foreach { f => - val raf = new FileOutputStream(f, true).getChannel() - raf.truncate(1) - raf.close() + val e = intercept[SparkException] { + val buffer = mock[ByteBuffer] + batchedWal.write(buffer, 2L) } - - // Verify that the corrupted files do not prevent reading of the second set of data - val readData = readDataUsingWriteAheadLog(testDir) - assert(readData === dataToWrite2) + assert(e.getCause.getMessage === "Hello!") } - test("FileBasedWriteAheadLog - do not create directories or files unless write") { - val nonexistentTempPath = File.createTempFile("test", "") - nonexistentTempPath.delete() - assert(!nonexistentTempPath.exists()) + // we make the write requests in separate threads so that we don't block the test thread + private def writeAsync(wal: WriteAheadLog, event: String, time: Long): Promise[Unit] = { + val p = Promise[Unit]() + p.completeWith(Future[Unit] { + val v = wal.write(event, time) + assert(v === walHandle) + }(walBatchingExecutionContext)) + p + } - val writtenSegment = writeDataManually(generateRandomData(), testFile) - val wal = new FileBasedWriteAheadLog(new SparkConf(), tempDir.getAbsolutePath, - new Configuration(), 1, 1, closeFileAfterWrite = false) - assert(!nonexistentTempPath.exists(), "Directory created just by creating log object") - wal.read(writtenSegment.head) - assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment") + test("BatchedWriteAheadLog - name log with the highest timestamp of aggregated entries") { + val blockingWal = new BlockingWriteAheadLog(wal, walHandle) + val batchedWal = new BatchedWriteAheadLog(blockingWal, sparkConf) + + val event1 = "hello" + val event2 = "world" + val event3 = "this" + val event4 = "is" + val event5 = "doge" + + // The queue.take() immediately takes the 3, and there is nothing left in the queue at that + // moment. Then the promise blocks the writing of 3. The rest get queued. + writeAsync(batchedWal, event1, 3L) + eventually(timeout(1 second)) { + assert(blockingWal.isBlocked) + assert(batchedWal.invokePrivate(queueLength()) === 0) + } + // rest of the records will be batched while it takes time for 3 to get written + writeAsync(batchedWal, event2, 5L) + writeAsync(batchedWal, event3, 8L) + // we would like event 5 to be written before event 4 in order to test that they get + // sorted before being aggregated + writeAsync(batchedWal, event5, 12L) + eventually(timeout(1 second)) { + assert(blockingWal.isBlocked) + assert(batchedWal.invokePrivate(queueLength()) === 3) + } + writeAsync(batchedWal, event4, 10L) + eventually(timeout(1 second)) { + assert(walBatchingThreadPool.getActiveCount === 5) + assert(batchedWal.invokePrivate(queueLength()) === 4) + } + blockingWal.allowWrite() + + val buffer = wrapArrayArrayByte(Array(event1)) + val queuedEvents = Set(event2, event3, event4, event5) + + eventually(timeout(1 second)) { + assert(batchedWal.invokePrivate(queueLength()) === 0) + verify(wal, times(1)).write(meq(buffer), meq(3L)) + // the file name should be the timestamp of the last record, as events should be naturally + // in order of timestamp, and we need the last element. + val bufferCaptor = ArgumentCaptor.forClass(classOf[ByteBuffer]) + verify(wal, times(1)).write(bufferCaptor.capture(), meq(12L)) + val records = BatchedWriteAheadLog.deaggregate(bufferCaptor.getValue).map(byteBufferToString) + assert(records.toSet === queuedEvents) + } } -} -object WriteAheadLogSuite { + test("BatchedWriteAheadLog - shutdown properly") { + val batchedWal = new BatchedWriteAheadLog(wal, sparkConf) + batchedWal.close() + verify(wal, times(1)).close() - class MockWriteAheadLog0() extends WriteAheadLog { - override def write(record: ByteBuffer, time: Long): WriteAheadLogRecordHandle = { null } - override def read(handle: WriteAheadLogRecordHandle): ByteBuffer = { null } - override def readAll(): util.Iterator[ByteBuffer] = { null } - override def clean(threshTime: Long, waitForCompletion: Boolean): Unit = { } - override def close(): Unit = { } + intercept[IllegalStateException](batchedWal.write(mock[ByteBuffer], 12L)) } - class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0() + test("BatchedWriteAheadLog - fail everything in queue during shutdown") { + val blockingWal = new BlockingWriteAheadLog(wal, walHandle) + val batchedWal = new BatchedWriteAheadLog(blockingWal, sparkConf) + + val event1 = "hello" + val event2 = "world" + val event3 = "this" + + // The queue.take() immediately takes the 3, and there is nothing left in the queue at that + // moment. Then the promise blocks the writing of 3. The rest get queued. + val promise1 = writeAsync(batchedWal, event1, 3L) + eventually(timeout(1 second)) { + assert(blockingWal.isBlocked) + assert(batchedWal.invokePrivate(queueLength()) === 0) + } + // rest of the records will be batched while it takes time for 3 to get written + val promise2 = writeAsync(batchedWal, event2, 5L) + val promise3 = writeAsync(batchedWal, event3, 8L) + + eventually(timeout(1 second)) { + assert(walBatchingThreadPool.getActiveCount === 3) + assert(blockingWal.isBlocked) + assert(batchedWal.invokePrivate(queueLength()) === 2) // event1 is being written + } + + val writePromises = Seq(promise1, promise2, promise3) - class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0() + batchedWal.close() + eventually(timeout(1 second)) { + assert(writePromises.forall(_.isCompleted)) + assert(writePromises.forall(_.future.value.get.isFailure)) // all should have failed + } + } +} + +class BatchedWriteAheadLogWithCloseFileAfterWriteSuite + extends CloseFileAfterWriteTests(allowBatching = true, "BatchedWriteAheadLog") +object WriteAheadLogSuite { private val hadoopConf = new Configuration() /** Write data to a file directly and return an array of the file segments written. */ - def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = { + def writeDataManually( + data: Seq[String], + file: String, + allowBatching: Boolean): Seq[FileBasedWriteAheadLogSegment] = { val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]() val writer = HdfsUtils.getOutputStream(file, hadoopConf) - data.foreach { item => + def writeToStream(bytes: Array[Byte]): Unit = { val offset = writer.getPos - val bytes = Utils.serialize(item) writer.writeInt(bytes.size) writer.write(bytes) segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size) } + if (allowBatching) { + writeToStream(wrapArrayArrayByte(data.toArray[String]).array()) + } else { + data.foreach { item => + writeToStream(Utils.serialize(item)) + } + } writer.close() segments } @@ -356,8 +619,7 @@ object WriteAheadLogSuite { */ def writeDataUsingWriter( filePath: String, - data: Seq[String] - ): Seq[FileBasedWriteAheadLogSegment] = { + data: Seq[String]): Seq[FileBasedWriteAheadLogSegment] = { val writer = new FileBasedWriteAheadLogWriter(filePath, hadoopConf) val segments = data.map { item => writer.write(item) @@ -370,13 +632,13 @@ object WriteAheadLogSuite { def writeDataUsingWriteAheadLog( logDirectory: String, data: Seq[String], + closeFileAfterWrite: Boolean, + allowBatching: Boolean, manualClock: ManualClock = new ManualClock, closeLog: Boolean = true, - clockAdvanceTime: Int = 500, - closeFileAfterWrite: Boolean = false): FileBasedWriteAheadLog = { + clockAdvanceTime: Int = 500): WriteAheadLog = { if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000) - val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1, - closeFileAfterWrite) + val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching) // Ensure that 500 does not get sorted after 2000, so put a high base value. data.foreach { item => @@ -406,16 +668,16 @@ object WriteAheadLogSuite { } /** Read all the data from a log file directly and return the list of byte buffers. */ - def readDataManually(file: String): Seq[String] = { + def readDataManually[T](file: String): Seq[T] = { val reader = HdfsUtils.getInputStream(file, hadoopConf) - val buffer = new ArrayBuffer[String] + val buffer = new ArrayBuffer[T] try { while (true) { // Read till EOF is thrown val length = reader.readInt() val bytes = new Array[Byte](length) reader.read(bytes) - buffer += Utils.deserialize[String](bytes) + buffer += Utils.deserialize[T](bytes) } } catch { case ex: EOFException => @@ -434,20 +696,23 @@ object WriteAheadLogSuite { } /** Read all the data in the log file in a directory using the WriteAheadLog class. */ - def readDataUsingWriteAheadLog(logDirectory: String): Seq[String] = { - val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1, - closeFileAfterWrite = false) - val data = wal.readAll().asScala.map(byteBufferToString).toSeq + def readDataUsingWriteAheadLog( + logDirectory: String, + closeFileAfterWrite: Boolean, + allowBatching: Boolean): Seq[String] = { + val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching) + val data = wal.readAll().asScala.map(byteBufferToString).toArray wal.close() data } - /** Get the log files in a direction */ + /** Get the log files in a directory. */ def getLogFilesInDirectory(directory: String): Seq[String] = { val logDirectoryPath = new Path(directory) val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) - if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) { + if (fileSystem.exists(logDirectoryPath) && + fileSystem.getFileStatus(logDirectoryPath).isDirectory) { fileSystem.listStatus(logDirectoryPath).map { _.getPath() }.sortBy { _.getName().split("-")(1).toLong }.map { @@ -458,10 +723,31 @@ object WriteAheadLogSuite { } } + def createWriteAheadLog( + logDirectory: String, + closeFileAfterWrite: Boolean, + allowBatching: Boolean): WriteAheadLog = { + val sparkConf = new SparkConf + val wal = new FileBasedWriteAheadLog(sparkConf, logDirectory, hadoopConf, 1, 1, + closeFileAfterWrite) + if (allowBatching) new BatchedWriteAheadLog(wal, sparkConf) else wal + } + def generateRandomData(): Seq[String] = { (1 to 100).map { _.toString } } + def readAndDeserializeDataManually(logFiles: Seq[String], allowBatching: Boolean): Seq[String] = { + if (allowBatching) { + logFiles.flatMap { file => + val data = readDataManually[Array[Array[Byte]]](file) + data.flatMap(byteArray => byteArray.map(Utils.deserialize[String])) + } + } else { + logFiles.flatMap { file => readDataManually[String](file)} + } + } + implicit def stringToByteBuffer(str: String): ByteBuffer = { ByteBuffer.wrap(Utils.serialize(str)) } @@ -469,4 +755,41 @@ object WriteAheadLogSuite { implicit def byteBufferToString(byteBuffer: ByteBuffer): String = { Utils.deserialize[String](byteBuffer.array) } + + def wrapArrayArrayByte[T](records: Array[T]): ByteBuffer = { + ByteBuffer.wrap(Utils.serialize[Array[Array[Byte]]](records.map(Utils.serialize[T]))) + } + + /** + * A wrapper WriteAheadLog that blocks the write function to allow batching with the + * BatchedWriteAheadLog. + */ + class BlockingWriteAheadLog( + wal: WriteAheadLog, + handle: WriteAheadLogRecordHandle) extends WriteAheadLog { + @volatile private var isWriteCalled: Boolean = false + @volatile private var blockWrite: Boolean = true + + override def write(record: ByteBuffer, time: Long): WriteAheadLogRecordHandle = { + isWriteCalled = true + eventually(Eventually.timeout(2 second)) { + assert(!blockWrite) + } + wal.write(record, time) + isWriteCalled = false + handle + } + override def read(segment: WriteAheadLogRecordHandle): ByteBuffer = wal.read(segment) + override def readAll(): JIterator[ByteBuffer] = wal.readAll() + override def clean(threshTime: Long, waitForCompletion: Boolean): Unit = { + wal.clean(threshTime, waitForCompletion) + } + override def close(): Unit = wal.close() + + def allowWrite(): Unit = { + blockWrite = false + } + + def isBlocked: Boolean = isWriteCalled + } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala new file mode 100644 index 000000000000..2a41177a5e63 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.util + +import java.nio.ByteBuffer +import java.util + +import scala.reflect.ClassTag + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.util.Utils + +class WriteAheadLogUtilsSuite extends SparkFunSuite { + import WriteAheadLogUtilsSuite._ + + private val logDir = Utils.createTempDir().getAbsolutePath() + private val hadoopConf = new Configuration() + + def assertDriverLogClass[T <: WriteAheadLog: ClassTag]( + conf: SparkConf, + isBatched: Boolean = false): WriteAheadLog = { + val log = WriteAheadLogUtils.createLogForDriver(conf, logDir, hadoopConf) + if (isBatched) { + assert(log.isInstanceOf[BatchedWriteAheadLog]) + val parentLog = log.asInstanceOf[BatchedWriteAheadLog].wrappedLog + assert(parentLog.getClass === implicitly[ClassTag[T]].runtimeClass) + } else { + assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) + } + log + } + + def assertReceiverLogClass[T <: WriteAheadLog: ClassTag](conf: SparkConf): WriteAheadLog = { + val log = WriteAheadLogUtils.createLogForReceiver(conf, logDir, hadoopConf) + assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) + log + } + + test("log selection and creation") { + + val emptyConf = new SparkConf() // no log configuration + assertDriverLogClass[FileBasedWriteAheadLog](emptyConf, isBatched = true) + assertReceiverLogClass[FileBasedWriteAheadLog](emptyConf) + + // Verify setting driver WAL class + val driverWALConf = new SparkConf().set("spark.streaming.driver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[MockWriteAheadLog0](driverWALConf, isBatched = true) + assertReceiverLogClass[FileBasedWriteAheadLog](driverWALConf) + + // Verify setting receiver WAL class + val receiverWALConf = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[FileBasedWriteAheadLog](receiverWALConf, isBatched = true) + assertReceiverLogClass[MockWriteAheadLog0](receiverWALConf) + + // Verify setting receiver WAL class with 1-arg constructor + val receiverWALConf2 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog1].getName()) + assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf2) + + // Verify failure setting receiver WAL class with 2-arg constructor + intercept[SparkException] { + val receiverWALConf3 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog2].getName()) + assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf3) + } + } + + test("wrap WriteAheadLog in BatchedWriteAheadLog when batching is enabled") { + def getBatchedSparkConf: SparkConf = + new SparkConf().set("spark.streaming.driver.writeAheadLog.allowBatching", "true") + + val justBatchingConf = getBatchedSparkConf + assertDriverLogClass[FileBasedWriteAheadLog](justBatchingConf, isBatched = true) + assertReceiverLogClass[FileBasedWriteAheadLog](justBatchingConf) + + // Verify setting driver WAL class + val driverWALConf = getBatchedSparkConf.set("spark.streaming.driver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[MockWriteAheadLog0](driverWALConf, isBatched = true) + assertReceiverLogClass[FileBasedWriteAheadLog](driverWALConf) + + // Verify receivers are not wrapped + val receiverWALConf = getBatchedSparkConf.set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[FileBasedWriteAheadLog](receiverWALConf, isBatched = true) + assertReceiverLogClass[MockWriteAheadLog0](receiverWALConf) + } + + test("batching is enabled by default in WriteAheadLog") { + val conf = new SparkConf() + assert(WriteAheadLogUtils.isBatchingEnabled(conf, isDriver = true)) + // batching is not valid for receiver WALs + assert(!WriteAheadLogUtils.isBatchingEnabled(conf, isDriver = false)) + } + + test("closeFileAfterWrite is disabled by default in WriteAheadLog") { + val conf = new SparkConf() + assert(!WriteAheadLogUtils.shouldCloseFileAfterWrite(conf, isDriver = true)) + assert(!WriteAheadLogUtils.shouldCloseFileAfterWrite(conf, isDriver = false)) + } +} + +object WriteAheadLogUtilsSuite { + + class MockWriteAheadLog0() extends WriteAheadLog { + override def write(record: ByteBuffer, time: Long): WriteAheadLogRecordHandle = { null } + override def read(handle: WriteAheadLogRecordHandle): ByteBuffer = { null } + override def readAll(): util.Iterator[ByteBuffer] = { null } + override def clean(threshTime: Long, waitForCompletion: Boolean): Unit = { } + override def close(): Unit = { } + } + + class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0() + + class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0() +} diff --git a/tags/pom.xml b/tags/pom.xml deleted file mode 100644 index ca93722e7334..000000000000 --- a/tags/pom.xml +++ /dev/null @@ -1,50 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../pom.xml - - - org.apache.spark - spark-test-tags_2.10 - jar - Spark Project Test Tags - http://spark.apache.org/ - - test-tags - - - - - org.scalatest - scalatest_${scala.binary.version} - compile - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/tools/pom.xml b/tools/pom.xml index 1e64f280e5be..37427e8da62d 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -19,13 +19,12 @@ 4.0.0 org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT + spark-parent_2.11 + 2.3.0-SNAPSHOT ../pom.xml - org.apache.spark - spark-tools_2.10 + spark-tools_2.11 tools @@ -34,16 +33,6 @@ http://spark.apache.org/ - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - org.scala-lang scala-reflect @@ -52,6 +41,11 @@ org.scala-lang scala-compiler + + org.clapper + classutil_${scala.binary.version} + 1.1.2 + diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala index a0524cabff2d..c9058ff40989 100644 --- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -18,15 +18,13 @@ // scalastyle:off classforname package org.apache.spark.tools -import java.io.File -import java.util.jar.JarFile - import scala.collection.mutable -import scala.collection.JavaConverters._ -import scala.reflect.runtime.universe.runtimeMirror import scala.reflect.runtime.{universe => unv} +import scala.reflect.runtime.universe.runtimeMirror import scala.util.Try +import org.clapper.classutil.ClassFinder + /** * A tool for generating classes to be excluded during binary checking with MIMA. It is expected * that this tool is run with ./spark-class. @@ -42,14 +40,6 @@ object GenerateMIMAIgnore { private val classLoader = Thread.currentThread().getContextClassLoader private val mirror = runtimeMirror(classLoader) - - private def isDeveloperApi(sym: unv.Symbol) = - sym.annotations.exists(_.tpe =:= unv.typeOf[org.apache.spark.annotation.DeveloperApi]) - - private def isExperimental(sym: unv.Symbol) = - sym.annotations.exists(_.tpe =:= unv.typeOf[org.apache.spark.annotation.Experimental]) - - private def isPackagePrivate(sym: unv.Symbol) = !sym.privateWithin.fullName.startsWith("") @@ -58,7 +48,7 @@ object GenerateMIMAIgnore { /** * For every class checks via scala reflection if the class itself or contained members - * have DeveloperApi or Experimental annotations or they are package private. + * are package private. * Returns the tuple of such classes and members. */ private def privateWithin(packageName: String): (Set[String], Set[String]) = { @@ -72,9 +62,9 @@ object GenerateMIMAIgnore { val classSymbol = mirror.classSymbol(Class.forName(className, false, classLoader)) val moduleSymbol = mirror.staticModule(className) val directlyPrivateSpark = - isPackagePrivate(classSymbol) || isPackagePrivateModule(moduleSymbol) - val developerApi = isDeveloperApi(classSymbol) || isDeveloperApi(moduleSymbol) - val experimental = isExperimental(classSymbol) || isExperimental(moduleSymbol) + isPackagePrivate(classSymbol) || + isPackagePrivateModule(moduleSymbol) || + classSymbol.isPrivate /* Inner classes defined within a private[spark] class or object are effectively invisible, so we account for them as package private. */ lazy val indirectlyPrivateSpark = { @@ -86,7 +76,7 @@ object GenerateMIMAIgnore { false } } - if (directlyPrivateSpark || indirectlyPrivateSpark || developerApi || experimental) { + if (directlyPrivateSpark || indirectlyPrivateSpark) { ignoredClasses += className } // check if this class has package-private/annotated members. @@ -101,10 +91,11 @@ object GenerateMIMAIgnore { (ignoredClasses.flatMap(c => Seq(c, c.replace("$", "#"))).toSet, ignoredMembers.toSet) } - /** Scala reflection does not let us see inner function even if they are upgraded - * to public for some reason. So had to resort to java reflection to get all inner - * functions with $$ in there name. - */ + /** + * Scala reflection does not let us see inner function even if they are upgraded + * to public for some reason. So had to resort to java reflection to get all inner + * functions with $$ in there name. + */ def getInnerFunctions(classSymbol: unv.ClassSymbol): Seq[String] = { try { Class.forName(classSymbol.fullName, false, classLoader).getMethods.map(_.getName) @@ -121,9 +112,7 @@ object GenerateMIMAIgnore { private def getAnnotatedOrPackagePrivateMembers(classSymbol: unv.ClassSymbol) = { classSymbol.typeSignature.members.filterNot(x => x.fullName.startsWith("java") || x.fullName.startsWith("scala") - ).filter(x => - isPackagePrivate(x) || isDeveloperApi(x) || isExperimental(x) - ).map(_.fullName) ++ getInnerFunctions(classSymbol) + ).filter(x => isPackagePrivate(x)).map(_.fullName) ++ getInnerFunctions(classSymbol) } def main(args: Array[String]) { @@ -143,7 +132,6 @@ object GenerateMIMAIgnore { // scalastyle:on println } - private def shouldExclude(name: String) = { // Heuristic to remove JVM classes that do not correspond to user-facing classes in Scala name.contains("anon") || @@ -158,35 +146,13 @@ object GenerateMIMAIgnore { * and subpackages both from directories and jars present on the classpath. */ private def getClasses(packageName: String): Set[String] = { - val path = packageName.replace('.', '/') - val resources = classLoader.getResources(path) - - val jars = resources.asScala.filter(_.getProtocol == "jar") - .map(_.getFile.split(":")(1).split("!")(0)).toSeq - - jars.flatMap(getClassesFromJar(_, path)) - .map(_.getName) - .filterNot(shouldExclude).toSet - } - - /** - * Get all classes in a package from a jar file. - */ - private def getClassesFromJar(jarPath: String, packageName: String) = { - import scala.collection.mutable - val jar = new JarFile(new File(jarPath)) - val enums = jar.entries().asScala.map(_.getName).filter(_.startsWith(packageName)) - val classes = mutable.HashSet[Class[_]]() - for (entry <- enums if entry.endsWith(".class")) { - try { - classes += Class.forName(entry.replace('/', '.').stripSuffix(".class"), false, classLoader) - } catch { - // scalastyle:off println - case _: Throwable => println("Unable to load:" + entry) - // scalastyle:on println - } - } - classes + val finder = ClassFinder() + finder + .getClasses + .map(_.name) + .filter(_.startsWith(packageName)) + .filterNot(shouldExclude) + .toSet } } // scalastyle:on classforname diff --git a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala deleted file mode 100644 index 856ea177a9a1..000000000000 --- a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala +++ /dev/null @@ -1,367 +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.tools - -import java.lang.reflect.{Type, Method} - -import scala.collection.mutable.ArrayBuffer -import scala.language.existentials - -import org.apache.spark._ -import org.apache.spark.api.java._ -import org.apache.spark.rdd.{RDD, DoubleRDDFunctions, PairRDDFunctions, OrderedRDDFunctions} -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.{DStream, PairDStreamFunctions} - - -private[spark] abstract class SparkType(val name: String) - -private[spark] case class BaseType(override val name: String) extends SparkType(name) { - override def toString: String = { - name - } -} - -private[spark] -case class ParameterizedType(override val name: String, - parameters: Seq[SparkType], - typebounds: String = "") extends SparkType(name) { - override def toString: String = { - if (typebounds != "") { - typebounds + " " + name + "<" + parameters.mkString(", ") + ">" - } else { - name + "<" + parameters.mkString(", ") + ">" - } - } -} - -private[spark] -case class SparkMethod(name: String, returnType: SparkType, parameters: Seq[SparkType]) { - override def toString: String = { - returnType + " " + name + "(" + parameters.mkString(", ") + ")" - } -} - -/** - * A tool for identifying methods that need to be ported from Scala to the Java API. - * - * It uses reflection to find methods in the Scala API and rewrites those methods' signatures - * into appropriate Java equivalents. If those equivalent methods have not been implemented in - * the Java API, they are printed. - */ -object JavaAPICompletenessChecker { - - private def parseType(typeStr: String): SparkType = { - if (!typeStr.contains("<")) { - // Base types might begin with "class" or "interface", so we have to strip that off: - BaseType(typeStr.trim.split(" ").last) - } else if (typeStr.endsWith("[]")) { - ParameterizedType("Array", Seq(parseType(typeStr.stripSuffix("[]")))) - } else { - val parts = typeStr.split("<", 2) - val name = parts(0).trim - assert (parts(1).last == '>') - val parameters = parts(1).dropRight(1) - ParameterizedType(name, parseTypeList(parameters)) - } - } - - private def parseTypeList(typeStr: String): Seq[SparkType] = { - val types: ArrayBuffer[SparkType] = new ArrayBuffer[SparkType] - var stack = 0 - var token: StringBuffer = new StringBuffer() - for (c <- typeStr.trim) { - if (c == ',' && stack == 0) { - types += parseType(token.toString) - token = new StringBuffer() - } else if (c == ' ' && stack != 0) { - // continue - } else { - if (c == '<') { - stack += 1 - } else if (c == '>') { - stack -= 1 - } - token.append(c) - } - } - assert (stack == 0) - if (token.toString != "") { - types += parseType(token.toString) - } - types.toSeq - } - - private def parseReturnType(typeStr: String): SparkType = { - if (typeStr(0) == '<') { - val parts = typeStr.drop(0).split(">", 2) - val parsed = parseType(parts(1)).asInstanceOf[ParameterizedType] - ParameterizedType(parsed.name, parsed.parameters, parts(0)) - } else { - parseType(typeStr) - } - } - - private def toSparkMethod(method: Method): SparkMethod = { - val returnType = parseReturnType(method.getGenericReturnType.toString) - val name = method.getName - val parameters = method.getGenericParameterTypes.map(t => parseType(t.toString)) - SparkMethod(name, returnType, parameters) - } - - private def toJavaType(scalaType: SparkType, isReturnType: Boolean): SparkType = { - val renameSubstitutions = Map( - "scala.collection.Map" -> "java.util.Map", - // TODO: the JavaStreamingContext API accepts Array arguments - // instead of Lists, so this isn't a trivial translation / sub: - "scala.collection.Seq" -> "java.util.List", - "scala.Function2" -> "org.apache.spark.api.java.function.Function2", - "scala.collection.Iterator" -> "java.util.Iterator", - "scala.collection.mutable.Queue" -> "java.util.Queue", - "double" -> "java.lang.Double" - ) - // Keep applying the substitutions until we've reached a fixedpoint. - def applySubs(scalaType: SparkType): SparkType = { - scalaType match { - case ParameterizedType(name, parameters, typebounds) => - name match { - case "org.apache.spark.rdd.RDD" => - if (parameters(0).name == classOf[Tuple2[_, _]].getName) { - val tupleParams = - parameters(0).asInstanceOf[ParameterizedType].parameters.map(applySubs) - ParameterizedType(classOf[JavaPairRDD[_, _]].getName, tupleParams) - } else { - ParameterizedType(classOf[JavaRDD[_]].getName, parameters.map(applySubs)) - } - case "org.apache.spark.streaming.dstream.DStream" => - if (parameters(0).name == classOf[Tuple2[_, _]].getName) { - val tupleParams = - parameters(0).asInstanceOf[ParameterizedType].parameters.map(applySubs) - ParameterizedType("org.apache.spark.streaming.api.java.JavaPairDStream", - tupleParams) - } else { - ParameterizedType("org.apache.spark.streaming.api.java.JavaDStream", - parameters.map(applySubs)) - } - case "scala.Option" => { - if (isReturnType) { - ParameterizedType("com.google.common.base.Optional", parameters.map(applySubs)) - } else { - applySubs(parameters(0)) - } - } - case "scala.Function1" => - val firstParamName = parameters.last.name - if (firstParamName.startsWith("scala.collection.Traversable") || - firstParamName.startsWith("scala.collection.Iterator")) { - ParameterizedType("org.apache.spark.api.java.function.FlatMapFunction", - Seq(parameters(0), - parameters.last.asInstanceOf[ParameterizedType].parameters(0)).map(applySubs)) - } else if (firstParamName == "scala.runtime.BoxedUnit") { - ParameterizedType("org.apache.spark.api.java.function.VoidFunction", - parameters.dropRight(1).map(applySubs)) - } else { - ParameterizedType("org.apache.spark.api.java.function.Function", - parameters.map(applySubs)) - } - case _ => - ParameterizedType(renameSubstitutions.getOrElse(name, name), - parameters.map(applySubs)) - } - case BaseType(name) => - if (renameSubstitutions.contains(name)) { - BaseType(renameSubstitutions(name)) - } else { - scalaType - } - } - } - var oldType = scalaType - var newType = applySubs(scalaType) - while (oldType != newType) { - oldType = newType - newType = applySubs(scalaType) - } - newType - } - - private def toJavaMethod(method: SparkMethod): SparkMethod = { - val params = method.parameters - .filterNot(_.name == "scala.reflect.ClassTag") - .map(toJavaType(_, isReturnType = false)) - SparkMethod(method.name, toJavaType(method.returnType, isReturnType = true), params) - } - - private def isExcludedByName(method: Method): Boolean = { - val name = method.getDeclaringClass.getName + "." + method.getName - // Scala methods that are declared as private[mypackage] become public in the resulting - // Java bytecode. As a result, we need to manually exclude those methods here. - // This list also includes a few methods that are only used by the web UI or other - // internal Spark components. - val excludedNames = Seq( - "org.apache.spark.rdd.RDD.origin", - "org.apache.spark.rdd.RDD.elementClassTag", - "org.apache.spark.rdd.RDD.checkpointData", - "org.apache.spark.rdd.RDD.partitioner", - "org.apache.spark.rdd.RDD.partitions", - "org.apache.spark.rdd.RDD.firstParent", - "org.apache.spark.rdd.RDD.doCheckpoint", - "org.apache.spark.rdd.RDD.markCheckpointed", - "org.apache.spark.rdd.RDD.clearDependencies", - "org.apache.spark.rdd.RDD.getDependencies", - "org.apache.spark.rdd.RDD.getPartitions", - "org.apache.spark.rdd.RDD.dependencies", - "org.apache.spark.rdd.RDD.getPreferredLocations", - "org.apache.spark.rdd.RDD.collectPartitions", - "org.apache.spark.rdd.RDD.computeOrReadCheckpoint", - "org.apache.spark.rdd.PairRDDFunctions.getKeyClass", - "org.apache.spark.rdd.PairRDDFunctions.getValueClass", - "org.apache.spark.SparkContext.stringToText", - "org.apache.spark.SparkContext.makeRDD", - "org.apache.spark.SparkContext.runJob", - "org.apache.spark.SparkContext.runApproximateJob", - "org.apache.spark.SparkContext.clean", - "org.apache.spark.SparkContext.metadataCleaner", - "org.apache.spark.SparkContext.ui", - "org.apache.spark.SparkContext.newShuffleId", - "org.apache.spark.SparkContext.newRddId", - "org.apache.spark.SparkContext.cleanup", - "org.apache.spark.SparkContext.receiverJobThread", - "org.apache.spark.SparkContext.getRDDStorageInfo", - "org.apache.spark.SparkContext.addedFiles", - "org.apache.spark.SparkContext.addedJars", - "org.apache.spark.SparkContext.persistentRdds", - "org.apache.spark.SparkContext.executorEnvs", - "org.apache.spark.SparkContext.checkpointDir", - "org.apache.spark.SparkContext.getSparkHome", - "org.apache.spark.SparkContext.executorMemoryRequested", - "org.apache.spark.SparkContext.getExecutorStorageStatus", - "org.apache.spark.streaming.dstream.DStream.generatedRDDs", - "org.apache.spark.streaming.dstream.DStream.zeroTime", - "org.apache.spark.streaming.dstream.DStream.rememberDuration", - "org.apache.spark.streaming.dstream.DStream.storageLevel", - "org.apache.spark.streaming.dstream.DStream.mustCheckpoint", - "org.apache.spark.streaming.dstream.DStream.checkpointDuration", - "org.apache.spark.streaming.dstream.DStream.checkpointData", - "org.apache.spark.streaming.dstream.DStream.graph", - "org.apache.spark.streaming.dstream.DStream.isInitialized", - "org.apache.spark.streaming.dstream.DStream.parentRememberDuration", - "org.apache.spark.streaming.dstream.DStream.initialize", - "org.apache.spark.streaming.dstream.DStream.validate", - "org.apache.spark.streaming.dstream.DStream.setContext", - "org.apache.spark.streaming.dstream.DStream.setGraph", - "org.apache.spark.streaming.dstream.DStream.remember", - "org.apache.spark.streaming.dstream.DStream.getOrCompute", - "org.apache.spark.streaming.dstream.DStream.generateJob", - "org.apache.spark.streaming.dstream.DStream.clearOldMetadata", - "org.apache.spark.streaming.dstream.DStream.addMetadata", - "org.apache.spark.streaming.dstream.DStream.updateCheckpointData", - "org.apache.spark.streaming.dstream.DStream.restoreCheckpointData", - "org.apache.spark.streaming.dstream.DStream.isTimeValid", - "org.apache.spark.streaming.StreamingContext.nextNetworkInputStreamId", - "org.apache.spark.streaming.StreamingContext.checkpointDir", - "org.apache.spark.streaming.StreamingContext.checkpointDuration", - "org.apache.spark.streaming.StreamingContext.receiverJobThread", - "org.apache.spark.streaming.StreamingContext.scheduler", - "org.apache.spark.streaming.StreamingContext.initialCheckpoint", - "org.apache.spark.streaming.StreamingContext.getNewNetworkStreamId", - "org.apache.spark.streaming.StreamingContext.validate", - "org.apache.spark.streaming.StreamingContext.createNewSparkContext", - "org.apache.spark.streaming.StreamingContext.rddToFileName", - "org.apache.spark.streaming.StreamingContext.getSparkCheckpointDir", - "org.apache.spark.streaming.StreamingContext.env", - "org.apache.spark.streaming.StreamingContext.graph", - "org.apache.spark.streaming.StreamingContext.isCheckpointPresent" - ) - val excludedPatterns = Seq( - """^org\.apache\.spark\.SparkContext\..*To.*Functions""", - """^org\.apache\.spark\.SparkContext\..*WritableConverter""", - """^org\.apache\.spark\.SparkContext\..*To.*Writable""" - ).map(_.r) - lazy val excludedByPattern = - !excludedPatterns.map(_.findFirstIn(name)).filter(_.isDefined).isEmpty - name.contains("$") || excludedNames.contains(name) || excludedByPattern - } - - private def isExcludedByInterface(method: Method): Boolean = { - val excludedInterfaces = - Set("org.apache.spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil") - def toComparisionKey(method: Method): (Class[_], String, Type) = - (method.getReturnType, method.getName, method.getGenericReturnType) - val interfaces = method.getDeclaringClass.getInterfaces.filter { i => - excludedInterfaces.contains(i.getName) - } - val excludedMethods = interfaces.flatMap(_.getMethods.map(toComparisionKey)) - excludedMethods.contains(toComparisionKey(method)) - } - - private def printMissingMethods(scalaClass: Class[_], javaClass: Class[_]) { - val methods = scalaClass.getMethods - .filterNot(_.isAccessible) - .filterNot(isExcludedByName) - .filterNot(isExcludedByInterface) - val javaEquivalents = methods.map(m => toJavaMethod(toSparkMethod(m))).toSet - - val javaMethods = javaClass.getMethods.map(toSparkMethod).toSet - - val missingMethods = javaEquivalents -- javaMethods - - for (method <- missingMethods) { - // scalastyle:off println - println(method) - // scalastyle:on println - } - } - - def main(args: Array[String]) { - // scalastyle:off println - println("Missing RDD methods") - printMissingMethods(classOf[RDD[_]], classOf[JavaRDD[_]]) - println() - - println("Missing PairRDD methods") - printMissingMethods(classOf[PairRDDFunctions[_, _]], classOf[JavaPairRDD[_, _]]) - println() - - println("Missing DoubleRDD methods") - printMissingMethods(classOf[DoubleRDDFunctions], classOf[JavaDoubleRDD]) - println() - - println("Missing OrderedRDD methods") - printMissingMethods(classOf[OrderedRDDFunctions[_, _, _]], classOf[JavaPairRDD[_, _]]) - println() - - println("Missing SparkContext methods") - printMissingMethods(classOf[SparkContext], classOf[JavaSparkContext]) - println() - - println("Missing StreamingContext methods") - printMissingMethods(classOf[StreamingContext], classOf[JavaStreamingContext]) - println() - - println("Missing DStream methods") - printMissingMethods(classOf[DStream[_]], classOf[JavaDStream[_]]) - println() - - println("Missing PairDStream methods") - printMissingMethods(classOf[PairDStreamFunctions[_, _]], classOf[JavaPairDStream[_, _]]) - println() - // scalastyle:on println - } -} diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala deleted file mode 100644 index 0dc2861253f1..000000000000 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ /dev/null @@ -1,111 +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.tools - -import java.util.concurrent.{CountDownLatch, Executors} -import java.util.concurrent.atomic.AtomicLong - -import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.serializer.KryoSerializer -import org.apache.spark.shuffle.hash.HashShuffleManager -import org.apache.spark.util.Utils - -/** - * Internal utility for micro-benchmarking shuffle write performance. - * - * Writes simulated shuffle output from several threads and records the observed throughput. - */ -object StoragePerfTester { - def main(args: Array[String]): Unit = { - /** Total amount of data to generate. Distributed evenly amongst maps and reduce splits. */ - val dataSizeMb = Utils.memoryStringToMb(sys.env.getOrElse("OUTPUT_DATA", "1g")) - - /** Number of map tasks. All tasks execute concurrently. */ - val numMaps = sys.env.get("NUM_MAPS").map(_.toInt).getOrElse(8) - - /** Number of reduce splits for each map task. */ - val numOutputSplits = sys.env.get("NUM_REDUCERS").map(_.toInt).getOrElse(500) - - val recordLength = 1000 // ~1KB records - val totalRecords = dataSizeMb * 1000 - val recordsPerMap = totalRecords / numMaps - - val writeKey = "1" * (recordLength / 2) - val writeValue = "1" * (recordLength / 2) - val executor = Executors.newFixedThreadPool(numMaps) - - val conf = new SparkConf() - .set("spark.shuffle.compress", "false") - .set("spark.shuffle.sync", "true") - .set("spark.shuffle.manager", "org.apache.spark.shuffle.hash.HashShuffleManager") - - // This is only used to instantiate a BlockManager. All thread scheduling is done manually. - val sc = new SparkContext("local[4]", "Write Tester", conf) - val hashShuffleManager = sc.env.shuffleManager.asInstanceOf[HashShuffleManager] - - def writeOutputBytes(mapId: Int, total: AtomicLong): Unit = { - val shuffle = hashShuffleManager.shuffleBlockResolver.forMapTask(1, mapId, numOutputSplits, - new KryoSerializer(sc.conf), new ShuffleWriteMetrics()) - val writers = shuffle.writers - for (i <- 1 to recordsPerMap) { - writers(i % numOutputSplits).write(writeKey, writeValue) - } - writers.map { w => - w.commitAndClose() - total.addAndGet(w.fileSegment().length) - } - - shuffle.releaseWriters(true) - } - - val start = System.currentTimeMillis() - val latch = new CountDownLatch(numMaps) - val totalBytes = new AtomicLong() - for (task <- 1 to numMaps) { - executor.submit(new Runnable() { - override def run(): Unit = { - try { - writeOutputBytes(task, totalBytes) - latch.countDown() - } catch { - case e: Exception => - // scalastyle:off println - println("Exception in child thread: " + e + " " + e.getMessage) - // scalastyle:on println - System.exit(1) - } - } - }) - } - latch.await() - val end = System.currentTimeMillis() - val time = (end - start) / 1000.0 - val bytesPerSecond = totalBytes.get() / time - val bytesPerFile = (totalBytes.get() / (numOutputSplits * numMaps.toDouble)).toLong - - // scalastyle:off println - System.err.println("files_total\t\t%s".format(numMaps * numOutputSplits)) - System.err.println("bytes_per_file\t\t%s".format(Utils.bytesToString(bytesPerFile))) - System.err.println("agg_throughput\t\t%s/s".format(Utils.bytesToString(bytesPerSecond.toLong))) - // scalastyle:on println - - executor.shutdown() - sc.stop() - } -} diff --git a/unsafe/pom.xml b/unsafe/pom.xml deleted file mode 100644 index caf1f77890b5..000000000000 --- a/unsafe/pom.xml +++ /dev/null @@ -1,106 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../pom.xml - - - org.apache.spark - spark-unsafe_2.10 - jar - Spark Project Unsafe - http://spark.apache.org/ - - unsafe - - - - - - - com.google.code.findbugs - jsr305 - - - com.google.guava - guava - - - - - org.slf4j - slf4j-api - provided - - - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - org.mockito - mockito-core - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - org.apache.commons - commons-lang3 - test - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - - net.alchim31.maven - scala-maven-plugin - - - - -XDignore.symbol.file - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - - -XDignore.symbol.file - - - - - - - diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java deleted file mode 100644 index 1c16da982923..000000000000 --- a/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ /dev/null @@ -1,155 +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.unsafe; - -import java.lang.reflect.Field; - -import sun.misc.Unsafe; - -public final class Platform { - - private static final Unsafe _UNSAFE; - - public static final int BYTE_ARRAY_OFFSET; - - public static final int INT_ARRAY_OFFSET; - - public static final int LONG_ARRAY_OFFSET; - - public static final int DOUBLE_ARRAY_OFFSET; - - public static int getInt(Object object, long offset) { - return _UNSAFE.getInt(object, offset); - } - - public static void putInt(Object object, long offset, int value) { - _UNSAFE.putInt(object, offset, value); - } - - public static boolean getBoolean(Object object, long offset) { - return _UNSAFE.getBoolean(object, offset); - } - - public static void putBoolean(Object object, long offset, boolean value) { - _UNSAFE.putBoolean(object, offset, value); - } - - public static byte getByte(Object object, long offset) { - return _UNSAFE.getByte(object, offset); - } - - public static void putByte(Object object, long offset, byte value) { - _UNSAFE.putByte(object, offset, value); - } - - public static short getShort(Object object, long offset) { - return _UNSAFE.getShort(object, offset); - } - - public static void putShort(Object object, long offset, short value) { - _UNSAFE.putShort(object, offset, value); - } - - public static long getLong(Object object, long offset) { - return _UNSAFE.getLong(object, offset); - } - - public static void putLong(Object object, long offset, long value) { - _UNSAFE.putLong(object, offset, value); - } - - public static float getFloat(Object object, long offset) { - return _UNSAFE.getFloat(object, offset); - } - - public static void putFloat(Object object, long offset, float value) { - _UNSAFE.putFloat(object, offset, value); - } - - public static double getDouble(Object object, long offset) { - return _UNSAFE.getDouble(object, offset); - } - - public static void putDouble(Object object, long offset, double value) { - _UNSAFE.putDouble(object, offset, value); - } - - public static Object getObjectVolatile(Object object, long offset) { - return _UNSAFE.getObjectVolatile(object, offset); - } - - public static void putObjectVolatile(Object object, long offset, Object value) { - _UNSAFE.putObjectVolatile(object, offset, value); - } - - public static long allocateMemory(long size) { - return _UNSAFE.allocateMemory(size); - } - - public static void freeMemory(long address) { - _UNSAFE.freeMemory(address); - } - - public static void copyMemory( - Object src, long srcOffset, Object dst, long dstOffset, long length) { - while (length > 0) { - long size = Math.min(length, UNSAFE_COPY_THRESHOLD); - _UNSAFE.copyMemory(src, srcOffset, dst, dstOffset, size); - length -= size; - srcOffset += size; - dstOffset += size; - } - } - - /** - * Raises an exception bypassing compiler checks for checked exceptions. - */ - public static void throwException(Throwable t) { - _UNSAFE.throwException(t); - } - - /** - * Limits the number of bytes to copy per {@link Unsafe#copyMemory(long, long, long)} to - * allow safepoint polling during a large copy. - */ - private static final long UNSAFE_COPY_THRESHOLD = 1024L * 1024L; - - static { - sun.misc.Unsafe unsafe; - try { - Field unsafeField = Unsafe.class.getDeclaredField("theUnsafe"); - unsafeField.setAccessible(true); - unsafe = (sun.misc.Unsafe) unsafeField.get(null); - } catch (Throwable cause) { - unsafe = null; - } - _UNSAFE = unsafe; - - if (_UNSAFE != null) { - BYTE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(byte[].class); - INT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(int[].class); - LONG_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(long[].class); - DOUBLE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(double[].class); - } else { - BYTE_ARRAY_OFFSET = 0; - INT_ARRAY_OFFSET = 0; - LONG_ARRAY_OFFSET = 0; - DOUBLE_ARRAY_OFFSET = 0; - } - } -} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java b/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java deleted file mode 100644 index cf42877bf9fd..000000000000 --- a/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.unsafe.array; - -import org.apache.spark.unsafe.Platform; - -public class ByteArrayMethods { - - private ByteArrayMethods() { - // Private constructor, since this class only contains static methods. - } - - /** Returns the next number greater or equal num that is power of 2. */ - public static long nextPowerOf2(long num) { - final long highBit = Long.highestOneBit(num); - return (highBit == num) ? num : highBit << 1; - } - - public static int roundNumberOfBytesToNearestWord(int numBytes) { - int remainder = numBytes & 0x07; // This is equivalent to `numBytes % 8` - if (remainder == 0) { - return numBytes; - } else { - return numBytes + (8 - remainder); - } - } - - /** - * Optimized byte array equality check for byte arrays. - * @return true if the arrays are equal, false otherwise - */ - public static boolean arrayEquals( - Object leftBase, long leftOffset, Object rightBase, long rightOffset, final long length) { - int i = 0; - while (i <= length - 8) { - if (Platform.getLong(leftBase, leftOffset + i) != - Platform.getLong(rightBase, rightOffset + i)) { - return false; - } - i += 8; - } - while (i < length) { - if (Platform.getByte(leftBase, leftOffset + i) != - Platform.getByte(rightBase, rightOffset + i)) { - return false; - } - i += 1; - } - return true; - } -} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java deleted file mode 100644 index 4276f25c2165..000000000000 --- a/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ /dev/null @@ -1,100 +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.unsafe.hash; - -import org.apache.spark.unsafe.Platform; - -/** - * 32-bit Murmur3 hasher. This is based on Guava's Murmur3_32HashFunction. - */ -public final class Murmur3_x86_32 { - private static final int C1 = 0xcc9e2d51; - private static final int C2 = 0x1b873593; - - private final int seed; - - public Murmur3_x86_32(int seed) { - this.seed = seed; - } - - @Override - public String toString() { - return "Murmur3_32(seed=" + seed + ")"; - } - - public int hashInt(int input) { - int k1 = mixK1(input); - int h1 = mixH1(seed, k1); - - return fmix(h1, 4); - } - - public int hashUnsafeWords(Object base, long offset, int lengthInBytes) { - return hashUnsafeWords(base, offset, lengthInBytes, seed); - } - - public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, int seed) { - // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. - assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)"; - int h1 = seed; - for (int i = 0; i < lengthInBytes; i += 4) { - int halfWord = Platform.getInt(base, offset + i); - int k1 = mixK1(halfWord); - h1 = mixH1(h1, k1); - } - return fmix(h1, lengthInBytes); - } - - public int hashLong(long input) { - int low = (int) input; - int high = (int) (input >>> 32); - - int k1 = mixK1(low); - int h1 = mixH1(seed, k1); - - k1 = mixK1(high); - h1 = mixH1(h1, k1); - - return fmix(h1, 8); - } - - private static int mixK1(int k1) { - k1 *= C1; - k1 = Integer.rotateLeft(k1, 15); - k1 *= C2; - return k1; - } - - private static int mixH1(int h1, int k1) { - h1 ^= k1; - h1 = Integer.rotateLeft(h1, 13); - h1 = h1 * 5 + 0xe6546b64; - return h1; - } - - // Finalization mix - force all bits of a hash block to avalanche - private static int fmix(int h1, int length) { - h1 ^= length; - h1 ^= h1 >>> 16; - h1 *= 0x85ebca6b; - h1 ^= h1 >>> 13; - h1 *= 0xc2b2ae35; - h1 ^= h1 >>> 16; - return h1; - } -} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java b/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java deleted file mode 100644 index 5192f68c862c..000000000000 --- a/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.unsafe.memory; - -public interface MemoryAllocator { - - /** - * Allocates a contiguous block of memory. Note that the allocated memory is not guaranteed - * to be zeroed out (call `zero()` on the result if this is necessary). - */ - MemoryBlock allocate(long size) throws OutOfMemoryError; - - void free(MemoryBlock memory); - - MemoryAllocator UNSAFE = new UnsafeMemoryAllocator(); - - MemoryAllocator HEAP = new HeapMemoryAllocator(); -} diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java deleted file mode 100644 index b7aecb5102ba..000000000000 --- a/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ /dev/null @@ -1,1006 +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.unsafe.types; - -import javax.annotation.Nonnull; -import java.io.*; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.Arrays; -import java.util.Map; - -import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.array.ByteArrayMethods; - -import static org.apache.spark.unsafe.Platform.*; - - -/** - * A UTF-8 String for internal Spark use. - *

    - * A String encoded in UTF-8 as an Array[Byte], which can be used for comparison, - * search, see http://en.wikipedia.org/wiki/UTF-8 for details. - *

    - * Note: This is not designed for general use cases, should not be used outside SQL. - */ -public final class UTF8String implements Comparable, Externalizable { - - // These are only updated by readExternal() - @Nonnull - private Object base; - private long offset; - private int numBytes; - - public Object getBaseObject() { return base; } - public long getBaseOffset() { return offset; } - - private static int[] bytesOfCodePointInUTF8 = {2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, - 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, - 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 4, 4, 4, 4, 4, 4, 4, 4, - 5, 5, 5, 5, - 6, 6}; - - private static boolean isLittleEndian = ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN; - - private static final UTF8String COMMA_UTF8 = UTF8String.fromString(","); - public static final UTF8String EMPTY_UTF8 = UTF8String.fromString(""); - - /** - * Creates an UTF8String from byte array, which should be encoded in UTF-8. - * - * Note: `bytes` will be hold by returned UTF8String. - */ - public static UTF8String fromBytes(byte[] bytes) { - if (bytes != null) { - return new UTF8String(bytes, BYTE_ARRAY_OFFSET, bytes.length); - } else { - return null; - } - } - - /** - * Creates an UTF8String from byte array, which should be encoded in UTF-8. - * - * Note: `bytes` will be hold by returned UTF8String. - */ - public static UTF8String fromBytes(byte[] bytes, int offset, int numBytes) { - if (bytes != null) { - return new UTF8String(bytes, BYTE_ARRAY_OFFSET + offset, numBytes); - } else { - return null; - } - } - - /** - * Creates an UTF8String from given address (base and offset) and length. - */ - public static UTF8String fromAddress(Object base, long offset, int numBytes) { - return new UTF8String(base, offset, numBytes); - } - - /** - * Creates an UTF8String from String. - */ - public static UTF8String fromString(String str) { - if (str == null) return null; - try { - return fromBytes(str.getBytes("utf-8")); - } catch (UnsupportedEncodingException e) { - // Turn the exception into unchecked so we can find out about it at runtime, but - // don't need to add lots of boilerplate code everywhere. - throwException(e); - return null; - } - } - - /** - * Creates an UTF8String that contains `length` spaces. - */ - public static UTF8String blankString(int length) { - byte[] spaces = new byte[length]; - Arrays.fill(spaces, (byte) ' '); - return fromBytes(spaces); - } - - protected UTF8String(Object base, long offset, int numBytes) { - this.base = base; - this.offset = offset; - this.numBytes = numBytes; - } - - // for serialization - public UTF8String() { - this(null, 0, 0); - } - - /** - * Writes the content of this string into a memory address, identified by an object and an offset. - * The target memory address must already been allocated, and have enough space to hold all the - * bytes in this string. - */ - public void writeToMemory(Object target, long targetOffset) { - Platform.copyMemory(base, offset, target, targetOffset, numBytes); - } - - public void writeTo(ByteBuffer buffer) { - assert(buffer.hasArray()); - byte[] target = buffer.array(); - int offset = buffer.arrayOffset(); - int pos = buffer.position(); - writeToMemory(target, Platform.BYTE_ARRAY_OFFSET + offset + pos); - buffer.position(pos + numBytes); - } - - /** - * Returns the number of bytes for a code point with the first byte as `b` - * @param b The first byte of a code point - */ - private static int numBytesForFirstByte(final byte b) { - final int offset = (b & 0xFF) - 192; - return (offset >= 0) ? bytesOfCodePointInUTF8[offset] : 1; - } - - /** - * Returns the number of bytes - */ - public int numBytes() { - return numBytes; - } - - /** - * Returns the number of code points in it. - */ - public int numChars() { - int len = 0; - for (int i = 0; i < numBytes; i += numBytesForFirstByte(getByte(i))) { - len += 1; - } - return len; - } - - /** - * Returns a 64-bit integer that can be used as the prefix used in sorting. - */ - public long getPrefix() { - // Since JVMs are either 4-byte aligned or 8-byte aligned, we check the size of the string. - // If size is 0, just return 0. - // If size is between 0 and 4 (inclusive), assume data is 4-byte aligned under the hood and - // use a getInt to fetch the prefix. - // If size is greater than 4, assume we have at least 8 bytes of data to fetch. - // After getting the data, we use a mask to mask out data that is not part of the string. - long p; - long mask = 0; - if (isLittleEndian) { - if (numBytes >= 8) { - p = Platform.getLong(base, offset); - } else if (numBytes > 4) { - p = Platform.getLong(base, offset); - mask = (1L << (8 - numBytes) * 8) - 1; - } else if (numBytes > 0) { - p = (long) Platform.getInt(base, offset); - mask = (1L << (8 - numBytes) * 8) - 1; - } else { - p = 0; - } - p = java.lang.Long.reverseBytes(p); - } else { - // byteOrder == ByteOrder.BIG_ENDIAN - if (numBytes >= 8) { - p = Platform.getLong(base, offset); - } else if (numBytes > 4) { - p = Platform.getLong(base, offset); - mask = (1L << (8 - numBytes) * 8) - 1; - } else if (numBytes > 0) { - p = ((long) Platform.getInt(base, offset)) << 32; - mask = (1L << (8 - numBytes) * 8) - 1; - } else { - p = 0; - } - } - p &= ~mask; - return p; - } - - /** - * Returns the underline bytes, will be a copy of it if it's part of another array. - */ - public byte[] getBytes() { - // avoid copy if `base` is `byte[]` - if (offset == BYTE_ARRAY_OFFSET && base instanceof byte[] - && ((byte[]) base).length == numBytes) { - return (byte[]) base; - } else { - byte[] bytes = new byte[numBytes]; - copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); - return bytes; - } - } - - /** - * Returns a substring of this. - * @param start the position of first code point - * @param until the position after last code point, exclusive. - */ - public UTF8String substring(final int start, final int until) { - if (until <= start || start >= numBytes) { - return EMPTY_UTF8; - } - - int i = 0; - int c = 0; - while (i < numBytes && c < start) { - i += numBytesForFirstByte(getByte(i)); - c += 1; - } - - int j = i; - while (i < numBytes && c < until) { - i += numBytesForFirstByte(getByte(i)); - c += 1; - } - - if (i > j) { - byte[] bytes = new byte[i - j]; - copyMemory(base, offset + j, bytes, BYTE_ARRAY_OFFSET, i - j); - return fromBytes(bytes); - } else { - return EMPTY_UTF8; - } - } - - public UTF8String substringSQL(int pos, int length) { - // Information regarding the pos calculation: - // Hive and SQL use one-based indexing for SUBSTR arguments but also accept zero and - // negative indices for start positions. If a start index i is greater than 0, it - // refers to element i-1 in the sequence. If a start index i is less than 0, it refers - // to the -ith element before the end of the sequence. If a start index i is 0, it - // refers to the first element. - int len = numChars(); - int start = (pos > 0) ? pos -1 : ((pos < 0) ? len + pos : 0); - int end = (length == Integer.MAX_VALUE) ? len : start + length; - return substring(start, end); - } - - /** - * Returns whether this contains `substring` or not. - */ - public boolean contains(final UTF8String substring) { - if (substring.numBytes == 0) { - return true; - } - - byte first = substring.getByte(0); - for (int i = 0; i <= numBytes - substring.numBytes; i++) { - if (getByte(i) == first && matchAt(substring, i)) { - return true; - } - } - return false; - } - - /** - * Returns the byte at position `i`. - */ - private byte getByte(int i) { - return Platform.getByte(base, offset + i); - } - - private boolean matchAt(final UTF8String s, int pos) { - if (s.numBytes + pos > numBytes || pos < 0) { - return false; - } - return ByteArrayMethods.arrayEquals(base, offset + pos, s.base, s.offset, s.numBytes); - } - - public boolean startsWith(final UTF8String prefix) { - return matchAt(prefix, 0); - } - - public boolean endsWith(final UTF8String suffix) { - return matchAt(suffix, numBytes - suffix.numBytes); - } - - /** - * Returns the upper case of this string - */ - public UTF8String toUpperCase() { - if (numBytes == 0) { - return EMPTY_UTF8; - } - - byte[] bytes = new byte[numBytes]; - bytes[0] = (byte) Character.toTitleCase(getByte(0)); - for (int i = 0; i < numBytes; i++) { - byte b = getByte(i); - if (numBytesForFirstByte(b) != 1) { - // fallback - return toUpperCaseSlow(); - } - int upper = Character.toUpperCase((int) b); - if (upper > 127) { - // fallback - return toUpperCaseSlow(); - } - bytes[i] = (byte) upper; - } - return fromBytes(bytes); - } - - private UTF8String toUpperCaseSlow() { - return fromString(toString().toUpperCase()); - } - - /** - * Returns the lower case of this string - */ - public UTF8String toLowerCase() { - if (numBytes == 0) { - return EMPTY_UTF8; - } - - byte[] bytes = new byte[numBytes]; - bytes[0] = (byte) Character.toTitleCase(getByte(0)); - for (int i = 0; i < numBytes; i++) { - byte b = getByte(i); - if (numBytesForFirstByte(b) != 1) { - // fallback - return toLowerCaseSlow(); - } - int lower = Character.toLowerCase((int) b); - if (lower > 127) { - // fallback - return toLowerCaseSlow(); - } - bytes[i] = (byte) lower; - } - return fromBytes(bytes); - } - - private UTF8String toLowerCaseSlow() { - return fromString(toString().toLowerCase()); - } - - /** - * Returns the title case of this string, that could be used as title. - */ - public UTF8String toTitleCase() { - if (numBytes == 0) { - return EMPTY_UTF8; - } - - byte[] bytes = new byte[numBytes]; - for (int i = 0; i < numBytes; i++) { - byte b = getByte(i); - if (i == 0 || getByte(i - 1) == ' ') { - if (numBytesForFirstByte(b) != 1) { - // fallback - return toTitleCaseSlow(); - } - int upper = Character.toTitleCase(b); - if (upper > 127) { - // fallback - return toTitleCaseSlow(); - } - bytes[i] = (byte) upper; - } else { - bytes[i] = b; - } - } - return fromBytes(bytes); - } - - private UTF8String toTitleCaseSlow() { - StringBuffer sb = new StringBuffer(); - String s = toString(); - sb.append(s); - sb.setCharAt(0, Character.toTitleCase(sb.charAt(0))); - for (int i = 1; i < s.length(); i++) { - if (sb.charAt(i - 1) == ' ') { - sb.setCharAt(i, Character.toTitleCase(sb.charAt(i))); - } - } - return fromString(sb.toString()); - } - - /* - * Returns the index of the string `match` in this String. This string has to be a comma separated - * list. If `match` contains a comma 0 will be returned. If the `match` isn't part of this String, - * 0 will be returned, else the index of match (1-based index) - */ - public int findInSet(UTF8String match) { - if (match.contains(COMMA_UTF8)) { - return 0; - } - - int n = 1, lastComma = -1; - for (int i = 0; i < numBytes; i++) { - if (getByte(i) == (byte) ',') { - if (i - (lastComma + 1) == match.numBytes && - ByteArrayMethods.arrayEquals(base, offset + (lastComma + 1), match.base, match.offset, - match.numBytes)) { - return n; - } - lastComma = i; - n++; - } - } - if (numBytes - (lastComma + 1) == match.numBytes && - ByteArrayMethods.arrayEquals(base, offset + (lastComma + 1), match.base, match.offset, - match.numBytes)) { - return n; - } - return 0; - } - - /** - * Copy the bytes from the current UTF8String, and make a new UTF8String. - * @param start the start position of the current UTF8String in bytes. - * @param end the end position of the current UTF8String in bytes. - * @return a new UTF8String in the position of [start, end] of current UTF8String bytes. - */ - private UTF8String copyUTF8String(int start, int end) { - int len = end - start + 1; - byte[] newBytes = new byte[len]; - copyMemory(base, offset + start, newBytes, BYTE_ARRAY_OFFSET, len); - return UTF8String.fromBytes(newBytes); - } - - public UTF8String trim() { - int s = 0; - int e = this.numBytes - 1; - // skip all of the space (0x20) in the left side - while (s < this.numBytes && getByte(s) <= 0x20 && getByte(s) >= 0x00) s++; - // skip all of the space (0x20) in the right side - while (e >= 0 && getByte(e) <= 0x20 && getByte(e) >= 0x00) e--; - if (s > e) { - // empty string - return UTF8String.fromBytes(new byte[0]); - } else { - return copyUTF8String(s, e); - } - } - - public UTF8String trimLeft() { - int s = 0; - // skip all of the space (0x20) in the left side - while (s < this.numBytes && getByte(s) <= 0x20 && getByte(s) >= 0x00) s++; - if (s == this.numBytes) { - // empty string - return UTF8String.fromBytes(new byte[0]); - } else { - return copyUTF8String(s, this.numBytes - 1); - } - } - - public UTF8String trimRight() { - int e = numBytes - 1; - // skip all of the space (0x20) in the right side - while (e >= 0 && getByte(e) <= 0x20 && getByte(e) >= 0x00) e--; - - if (e < 0) { - // empty string - return UTF8String.fromBytes(new byte[0]); - } else { - return copyUTF8String(0, e); - } - } - - public UTF8String reverse() { - byte[] result = new byte[this.numBytes]; - - int i = 0; // position in byte - while (i < numBytes) { - int len = numBytesForFirstByte(getByte(i)); - copyMemory(this.base, this.offset + i, result, - BYTE_ARRAY_OFFSET + result.length - i - len, len); - - i += len; - } - - return UTF8String.fromBytes(result); - } - - public UTF8String repeat(int times) { - if (times <= 0) { - return EMPTY_UTF8; - } - - byte[] newBytes = new byte[numBytes * times]; - copyMemory(this.base, this.offset, newBytes, BYTE_ARRAY_OFFSET, numBytes); - - int copied = 1; - while (copied < times) { - int toCopy = Math.min(copied, times - copied); - System.arraycopy(newBytes, 0, newBytes, copied * numBytes, numBytes * toCopy); - copied += toCopy; - } - - return UTF8String.fromBytes(newBytes); - } - - /** - * Returns the position of the first occurrence of substr in - * current string from the specified position (0-based index). - * - * @param v the string to be searched - * @param start the start position of the current string for searching - * @return the position of the first occurrence of substr, if not found, -1 returned. - */ - public int indexOf(UTF8String v, int start) { - if (v.numBytes() == 0) { - return 0; - } - - // locate to the start position. - int i = 0; // position in byte - int c = 0; // position in character - while (i < numBytes && c < start) { - i += numBytesForFirstByte(getByte(i)); - c += 1; - } - - do { - if (i + v.numBytes > numBytes) { - return -1; - } - if (ByteArrayMethods.arrayEquals(base, offset + i, v.base, v.offset, v.numBytes)) { - return c; - } - i += numBytesForFirstByte(getByte(i)); - c += 1; - } while (i < numBytes); - - return -1; - } - - /** - * Find the `str` from left to right. - */ - private int find(UTF8String str, int start) { - assert (str.numBytes > 0); - while (start <= numBytes - str.numBytes) { - if (ByteArrayMethods.arrayEquals(base, offset + start, str.base, str.offset, str.numBytes)) { - return start; - } - start += 1; - } - return -1; - } - - /** - * Find the `str` from right to left. - */ - private int rfind(UTF8String str, int start) { - assert (str.numBytes > 0); - while (start >= 0) { - if (ByteArrayMethods.arrayEquals(base, offset + start, str.base, str.offset, str.numBytes)) { - return start; - } - start -= 1; - } - return -1; - } - - /** - * Returns the substring from string str before count occurrences of the delimiter delim. - * If count is positive, everything the left of the final delimiter (counting from left) is - * returned. If count is negative, every to the right of the final delimiter (counting from the - * right) is returned. subStringIndex performs a case-sensitive match when searching for delim. - */ - public UTF8String subStringIndex(UTF8String delim, int count) { - if (delim.numBytes == 0 || count == 0) { - return EMPTY_UTF8; - } - if (count > 0) { - int idx = -1; - while (count > 0) { - idx = find(delim, idx + 1); - if (idx >= 0) { - count --; - } else { - // can not find enough delim - return this; - } - } - if (idx == 0) { - return EMPTY_UTF8; - } - byte[] bytes = new byte[idx]; - copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, idx); - return fromBytes(bytes); - - } else { - int idx = numBytes - delim.numBytes + 1; - count = -count; - while (count > 0) { - idx = rfind(delim, idx - 1); - if (idx >= 0) { - count --; - } else { - // can not find enough delim - return this; - } - } - if (idx + delim.numBytes == numBytes) { - return EMPTY_UTF8; - } - int size = numBytes - delim.numBytes - idx; - byte[] bytes = new byte[size]; - copyMemory(base, offset + idx + delim.numBytes, bytes, BYTE_ARRAY_OFFSET, size); - return fromBytes(bytes); - } - } - - /** - * Returns str, right-padded with pad to a length of len - * For example: - * ('hi', 5, '??') => 'hi???' - * ('hi', 1, '??') => 'h' - */ - public UTF8String rpad(int len, UTF8String pad) { - int spaces = len - this.numChars(); // number of char need to pad - if (spaces <= 0 || pad.numBytes() == 0) { - // no padding at all, return the substring of the current string - return substring(0, len); - } else { - int padChars = pad.numChars(); - int count = spaces / padChars; // how many padding string needed - // the partial string of the padding - UTF8String remain = pad.substring(0, spaces - padChars * count); - - byte[] data = new byte[this.numBytes + pad.numBytes * count + remain.numBytes]; - copyMemory(this.base, this.offset, data, BYTE_ARRAY_OFFSET, this.numBytes); - int offset = this.numBytes; - int idx = 0; - while (idx < count) { - copyMemory(pad.base, pad.offset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); - ++ idx; - offset += pad.numBytes; - } - copyMemory(remain.base, remain.offset, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); - - return UTF8String.fromBytes(data); - } - } - - /** - * Returns str, left-padded with pad to a length of len. - * For example: - * ('hi', 5, '??') => '???hi' - * ('hi', 1, '??') => 'h' - */ - public UTF8String lpad(int len, UTF8String pad) { - int spaces = len - this.numChars(); // number of char need to pad - if (spaces <= 0 || pad.numBytes() == 0) { - // no padding at all, return the substring of the current string - return substring(0, len); - } else { - int padChars = pad.numChars(); - int count = spaces / padChars; // how many padding string needed - // the partial string of the padding - UTF8String remain = pad.substring(0, spaces - padChars * count); - - byte[] data = new byte[this.numBytes + pad.numBytes * count + remain.numBytes]; - - int offset = 0; - int idx = 0; - while (idx < count) { - copyMemory(pad.base, pad.offset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); - ++ idx; - offset += pad.numBytes; - } - copyMemory(remain.base, remain.offset, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); - offset += remain.numBytes; - copyMemory(this.base, this.offset, data, BYTE_ARRAY_OFFSET + offset, numBytes()); - - return UTF8String.fromBytes(data); - } - } - - /** - * Concatenates input strings together into a single string. Returns null if any input is null. - */ - public static UTF8String concat(UTF8String... inputs) { - // Compute the total length of the result. - int totalLength = 0; - for (int i = 0; i < inputs.length; i++) { - if (inputs[i] != null) { - totalLength += inputs[i].numBytes; - } else { - return null; - } - } - - // Allocate a new byte array, and copy the inputs one by one into it. - final byte[] result = new byte[totalLength]; - int offset = 0; - for (int i = 0; i < inputs.length; i++) { - int len = inputs[i].numBytes; - copyMemory( - inputs[i].base, inputs[i].offset, - result, BYTE_ARRAY_OFFSET + offset, - len); - offset += len; - } - return fromBytes(result); - } - - /** - * Concatenates input strings together into a single string using the separator. - * A null input is skipped. For example, concat(",", "a", null, "c") would yield "a,c". - */ - public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { - if (separator == null) { - return null; - } - - int numInputBytes = 0; // total number of bytes from the inputs - int numInputs = 0; // number of non-null inputs - for (int i = 0; i < inputs.length; i++) { - if (inputs[i] != null) { - numInputBytes += inputs[i].numBytes; - numInputs++; - } - } - - if (numInputs == 0) { - // Return an empty string if there is no input, or all the inputs are null. - return fromBytes(new byte[0]); - } - - // Allocate a new byte array, and copy the inputs one by one into it. - // The size of the new array is the size of all inputs, plus the separators. - final byte[] result = new byte[numInputBytes + (numInputs - 1) * separator.numBytes]; - int offset = 0; - - for (int i = 0, j = 0; i < inputs.length; i++) { - if (inputs[i] != null) { - int len = inputs[i].numBytes; - copyMemory( - inputs[i].base, inputs[i].offset, - result, BYTE_ARRAY_OFFSET + offset, - len); - offset += len; - - j++; - // Add separator if this is not the last input. - if (j < numInputs) { - copyMemory( - separator.base, separator.offset, - result, BYTE_ARRAY_OFFSET + offset, - separator.numBytes); - offset += separator.numBytes; - } - } - } - return fromBytes(result); - } - - public UTF8String[] split(UTF8String pattern, int limit) { - String[] splits = toString().split(pattern.toString(), limit); - UTF8String[] res = new UTF8String[splits.length]; - for (int i = 0; i < res.length; i++) { - res[i] = fromString(splits[i]); - } - return res; - } - - // TODO: Need to use `Code Point` here instead of Char in case the character longer than 2 bytes - public UTF8String translate(Map dict) { - String srcStr = this.toString(); - - StringBuilder sb = new StringBuilder(); - for(int k = 0; k< srcStr.length(); k++) { - if (null == dict.get(srcStr.charAt(k))) { - sb.append(srcStr.charAt(k)); - } else if ('\0' != dict.get(srcStr.charAt(k))){ - sb.append(dict.get(srcStr.charAt(k))); - } - } - return fromString(sb.toString()); - } - - @Override - public String toString() { - try { - return new String(getBytes(), "utf-8"); - } catch (UnsupportedEncodingException e) { - // Turn the exception into unchecked so we can find out about it at runtime, but - // don't need to add lots of boilerplate code everywhere. - throwException(e); - return "unknown"; // we will never reach here. - } - } - - @Override - public UTF8String clone() { - return fromBytes(getBytes()); - } - - @Override - public int compareTo(@Nonnull final UTF8String other) { - int len = Math.min(numBytes, other.numBytes); - // TODO: compare 8 bytes as unsigned long - for (int i = 0; i < len; i ++) { - // In UTF-8, the byte should be unsigned, so we should compare them as unsigned int. - int res = (getByte(i) & 0xFF) - (other.getByte(i) & 0xFF); - if (res != 0) { - return res; - } - } - return numBytes - other.numBytes; - } - - public int compare(final UTF8String other) { - return compareTo(other); - } - - @Override - public boolean equals(final Object other) { - if (other instanceof UTF8String) { - UTF8String o = (UTF8String) other; - if (numBytes != o.numBytes) { - return false; - } - return ByteArrayMethods.arrayEquals(base, offset, o.base, o.offset, numBytes); - } else { - return false; - } - } - - /** - * Levenshtein distance is a metric for measuring the distance of two strings. The distance is - * defined by the minimum number of single-character edits (i.e. insertions, deletions or - * substitutions) that are required to change one of the strings into the other. - */ - public int levenshteinDistance(UTF8String other) { - // Implementation adopted from org.apache.common.lang3.StringUtils.getLevenshteinDistance - - int n = numChars(); - int m = other.numChars(); - - if (n == 0) { - return m; - } else if (m == 0) { - return n; - } - - UTF8String s, t; - - if (n <= m) { - s = this; - t = other; - } else { - s = other; - t = this; - int swap; - swap = n; - n = m; - m = swap; - } - - int p[] = new int[n + 1]; - int d[] = new int[n + 1]; - int swap[]; - - int i, i_bytes, j, j_bytes, num_bytes_j, cost; - - for (i = 0; i <= n; i++) { - p[i] = i; - } - - for (j = 0, j_bytes = 0; j < m; j_bytes += num_bytes_j, j++) { - num_bytes_j = numBytesForFirstByte(t.getByte(j_bytes)); - d[0] = j + 1; - - for (i = 0, i_bytes = 0; i < n; i_bytes += numBytesForFirstByte(s.getByte(i_bytes)), i++) { - if (s.getByte(i_bytes) != t.getByte(j_bytes) || - num_bytes_j != numBytesForFirstByte(s.getByte(i_bytes))) { - cost = 1; - } else { - cost = (ByteArrayMethods.arrayEquals(t.base, t.offset + j_bytes, s.base, - s.offset + i_bytes, num_bytes_j)) ? 0 : 1; - } - d[i + 1] = Math.min(Math.min(d[i] + 1, p[i + 1] + 1), p[i] + cost); - } - - swap = p; - p = d; - d = swap; - } - - return p[n]; - } - - @Override - public int hashCode() { - int result = 1; - for (int i = 0; i < numBytes; i ++) { - result = 31 * result + getByte(i); - } - return result; - } - - /** - * Soundex mapping table - */ - private static final byte[] US_ENGLISH_MAPPING = {'0', '1', '2', '3', '0', '1', '2', '7', - '0', '2', '2', '4', '5', '5', '0', '1', '2', '6', '2', '3', '0', '1', '7', '2', '0', '2'}; - - /** - * Encodes a string into a Soundex value. Soundex is an encoding used to relate similar names, - * but can also be used as a general purpose scheme to find word with similar phonemes. - * https://en.wikipedia.org/wiki/Soundex - */ - public UTF8String soundex() { - if (numBytes == 0) { - return EMPTY_UTF8; - } - - byte b = getByte(0); - if ('a' <= b && b <= 'z') { - b -= 32; - } else if (b < 'A' || 'Z' < b) { - // first character must be a letter - return this; - } - byte sx[] = {'0', '0', '0', '0'}; - sx[0] = b; - int sxi = 1; - int idx = b - 'A'; - byte lastCode = US_ENGLISH_MAPPING[idx]; - - for (int i = 1; i < numBytes; i++) { - b = getByte(i); - if ('a' <= b && b <= 'z') { - b -= 32; - } else if (b < 'A' || 'Z' < b) { - // not a letter, skip it - lastCode = '0'; - continue; - } - idx = b - 'A'; - byte code = US_ENGLISH_MAPPING[idx]; - if (code == '7') { - // ignore it - } else { - if (code != '0' && code != lastCode) { - sx[sxi++] = code; - if (sxi > 3) break; - } - lastCode = code; - } - } - return UTF8String.fromBytes(sx); - } - - public void writeExternal(ObjectOutput out) throws IOException { - byte[] bytes = getBytes(); - out.writeInt(bytes.length); - out.write(bytes); - } - - public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - offset = BYTE_ARRAY_OFFSET; - numBytes = in.readInt(); - base = new byte[numBytes]; - in.readFully((byte[]) base); - } - -} diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java deleted file mode 100644 index e21ffdcff9ab..000000000000 --- a/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ /dev/null @@ -1,492 +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.unsafe.types; - -import java.io.UnsupportedEncodingException; -import java.util.Arrays; -import java.util.HashMap; - -import com.google.common.collect.ImmutableMap; -import org.junit.Test; - -import static org.junit.Assert.*; - -import static org.apache.spark.unsafe.types.UTF8String.*; - -public class UTF8StringSuite { - - private static void checkBasic(String str, int len) throws UnsupportedEncodingException { - UTF8String s1 = fromString(str); - UTF8String s2 = fromBytes(str.getBytes("utf8")); - assertEquals(s1.numChars(), len); - assertEquals(s2.numChars(), len); - - assertEquals(s1.toString(), str); - assertEquals(s2.toString(), str); - assertEquals(s1, s2); - - assertEquals(s1.hashCode(), s2.hashCode()); - - assertEquals(0, s1.compareTo(s2)); - - assertTrue(s1.contains(s2)); - assertTrue(s2.contains(s1)); - assertTrue(s1.startsWith(s1)); - assertTrue(s1.endsWith(s1)); - } - - @Test - public void basicTest() throws UnsupportedEncodingException { - checkBasic("", 0); - checkBasic("hello", 5); - checkBasic("大 千 世 界", 7); - } - - @Test - public void emptyStringTest() { - assertEquals(EMPTY_UTF8, fromString("")); - assertEquals(EMPTY_UTF8, fromBytes(new byte[0])); - assertEquals(0, EMPTY_UTF8.numChars()); - assertEquals(0, EMPTY_UTF8.numBytes()); - } - - @Test - public void prefix() { - assertTrue(fromString("a").getPrefix() - fromString("b").getPrefix() < 0); - assertTrue(fromString("ab").getPrefix() - fromString("b").getPrefix() < 0); - assertTrue( - fromString("abbbbbbbbbbbasdf").getPrefix() - fromString("bbbbbbbbbbbbasdf").getPrefix() < 0); - assertTrue(fromString("").getPrefix() - fromString("a").getPrefix() < 0); - assertTrue(fromString("你好").getPrefix() - fromString("世界").getPrefix() > 0); - - byte[] buf1 = {1, 2, 3, 4, 5, 6, 7, 8, 9}; - byte[] buf2 = {1, 2, 3}; - UTF8String str1 = fromBytes(buf1, 0, 3); - UTF8String str2 = fromBytes(buf1, 0, 8); - UTF8String str3 = fromBytes(buf2); - assertTrue(str1.getPrefix() - str2.getPrefix() < 0); - assertEquals(str1.getPrefix(), str3.getPrefix()); - } - - @Test - public void compareTo() { - assertTrue(fromString("").compareTo(fromString("a")) < 0); - assertTrue(fromString("abc").compareTo(fromString("ABC")) > 0); - assertTrue(fromString("abc0").compareTo(fromString("abc")) > 0); - assertTrue(fromString("abcabcabc").compareTo(fromString("abcabcabc")) == 0); - assertTrue(fromString("aBcabcabc").compareTo(fromString("Abcabcabc")) > 0); - assertTrue(fromString("Abcabcabc").compareTo(fromString("abcabcabC")) < 0); - assertTrue(fromString("abcabcabc").compareTo(fromString("abcabcabC")) > 0); - - assertTrue(fromString("abc").compareTo(fromString("世界")) < 0); - assertTrue(fromString("你好").compareTo(fromString("世界")) > 0); - assertTrue(fromString("你好123").compareTo(fromString("你好122")) > 0); - } - - protected static void testUpperandLower(String upper, String lower) { - UTF8String us = fromString(upper); - UTF8String ls = fromString(lower); - assertEquals(ls, us.toLowerCase()); - assertEquals(us, ls.toUpperCase()); - assertEquals(us, us.toUpperCase()); - assertEquals(ls, ls.toLowerCase()); - } - - @Test - public void upperAndLower() { - testUpperandLower("", ""); - testUpperandLower("0123456", "0123456"); - testUpperandLower("ABCXYZ", "abcxyz"); - testUpperandLower("ЀЁЂѺΏỀ", "ѐёђѻώề"); - testUpperandLower("大千世界 数据砖头", "大千世界 数据砖头"); - } - - @Test - public void titleCase() { - assertEquals(fromString(""), fromString("").toTitleCase()); - assertEquals(fromString("Ab Bc Cd"), fromString("ab bc cd").toTitleCase()); - assertEquals(fromString("Ѐ Ё Ђ Ѻ Ώ Ề"), fromString("ѐ ё ђ ѻ ώ ề").toTitleCase()); - assertEquals(fromString("大千世界 数据砖头"), fromString("大千世界 数据砖头").toTitleCase()); - } - - @Test - public void concatTest() { - assertEquals(EMPTY_UTF8, concat()); - assertNull(concat((UTF8String) null)); - assertEquals(EMPTY_UTF8, concat(EMPTY_UTF8)); - assertEquals(fromString("ab"), concat(fromString("ab"))); - assertEquals(fromString("ab"), concat(fromString("a"), fromString("b"))); - assertEquals(fromString("abc"), concat(fromString("a"), fromString("b"), fromString("c"))); - assertNull(concat(fromString("a"), null, fromString("c"))); - assertNull(concat(fromString("a"), null, null)); - assertNull(concat(null, null, null)); - assertEquals(fromString("数据砖头"), concat(fromString("数据"), fromString("砖头"))); - } - - @Test - public void concatWsTest() { - // Returns null if the separator is null - assertNull(concatWs(null, (UTF8String) null)); - assertNull(concatWs(null, fromString("a"))); - - // If separator is null, concatWs should skip all null inputs and never return null. - UTF8String sep = fromString("哈哈"); - assertEquals( - EMPTY_UTF8, - concatWs(sep, EMPTY_UTF8)); - assertEquals( - fromString("ab"), - concatWs(sep, fromString("ab"))); - assertEquals( - fromString("a哈哈b"), - concatWs(sep, fromString("a"), fromString("b"))); - assertEquals( - fromString("a哈哈b哈哈c"), - concatWs(sep, fromString("a"), fromString("b"), fromString("c"))); - assertEquals( - fromString("a哈哈c"), - concatWs(sep, fromString("a"), null, fromString("c"))); - assertEquals( - fromString("a"), - concatWs(sep, fromString("a"), null, null)); - assertEquals( - EMPTY_UTF8, - concatWs(sep, null, null, null)); - assertEquals( - fromString("数据哈哈砖头"), - concatWs(sep, fromString("数据"), fromString("砖头"))); - } - - @Test - public void contains() { - assertTrue(EMPTY_UTF8.contains(EMPTY_UTF8)); - assertTrue(fromString("hello").contains(fromString("ello"))); - assertFalse(fromString("hello").contains(fromString("vello"))); - assertFalse(fromString("hello").contains(fromString("hellooo"))); - assertTrue(fromString("大千世界").contains(fromString("千世界"))); - assertFalse(fromString("大千世界").contains(fromString("世千"))); - assertFalse(fromString("大千世界").contains(fromString("大千世界好"))); - } - - @Test - public void startsWith() { - assertTrue(EMPTY_UTF8.startsWith(EMPTY_UTF8)); - assertTrue(fromString("hello").startsWith(fromString("hell"))); - assertFalse(fromString("hello").startsWith(fromString("ell"))); - assertFalse(fromString("hello").startsWith(fromString("hellooo"))); - assertTrue(fromString("数据砖头").startsWith(fromString("数据"))); - assertFalse(fromString("大千世界").startsWith(fromString("千"))); - assertFalse(fromString("大千世界").startsWith(fromString("大千世界好"))); - } - - @Test - public void endsWith() { - assertTrue(EMPTY_UTF8.endsWith(EMPTY_UTF8)); - assertTrue(fromString("hello").endsWith(fromString("ello"))); - assertFalse(fromString("hello").endsWith(fromString("ellov"))); - assertFalse(fromString("hello").endsWith(fromString("hhhello"))); - assertTrue(fromString("大千世界").endsWith(fromString("世界"))); - assertFalse(fromString("大千世界").endsWith(fromString("世"))); - assertFalse(fromString("数据砖头").endsWith(fromString("我的数据砖头"))); - } - - @Test - public void substring() { - assertEquals(EMPTY_UTF8, fromString("hello").substring(0, 0)); - assertEquals(fromString("el"), fromString("hello").substring(1, 3)); - assertEquals(fromString("数"), fromString("数据砖头").substring(0, 1)); - assertEquals(fromString("据砖"), fromString("数据砖头").substring(1, 3)); - assertEquals(fromString("头"), fromString("数据砖头").substring(3, 5)); - assertEquals(fromString("ߵ梷"), fromString("ߵ梷").substring(0, 2)); - } - - @Test - public void trims() { - assertEquals(fromString("hello"), fromString(" hello ").trim()); - assertEquals(fromString("hello "), fromString(" hello ").trimLeft()); - assertEquals(fromString(" hello"), fromString(" hello ").trimRight()); - - assertEquals(EMPTY_UTF8, fromString(" ").trim()); - assertEquals(EMPTY_UTF8, fromString(" ").trimLeft()); - assertEquals(EMPTY_UTF8, fromString(" ").trimRight()); - - assertEquals(fromString("数据砖头"), fromString(" 数据砖头 ").trim()); - assertEquals(fromString("数据砖头 "), fromString(" 数据砖头 ").trimLeft()); - assertEquals(fromString(" 数据砖头"), fromString(" 数据砖头 ").trimRight()); - - assertEquals(fromString("数据砖头"), fromString("数据砖头").trim()); - assertEquals(fromString("数据砖头"), fromString("数据砖头").trimLeft()); - assertEquals(fromString("数据砖头"), fromString("数据砖头").trimRight()); - } - - @Test - public void indexOf() { - assertEquals(0, EMPTY_UTF8.indexOf(EMPTY_UTF8, 0)); - assertEquals(-1, EMPTY_UTF8.indexOf(fromString("l"), 0)); - assertEquals(0, fromString("hello").indexOf(EMPTY_UTF8, 0)); - assertEquals(2, fromString("hello").indexOf(fromString("l"), 0)); - assertEquals(3, fromString("hello").indexOf(fromString("l"), 3)); - assertEquals(-1, fromString("hello").indexOf(fromString("a"), 0)); - assertEquals(2, fromString("hello").indexOf(fromString("ll"), 0)); - assertEquals(-1, fromString("hello").indexOf(fromString("ll"), 4)); - assertEquals(1, fromString("数据砖头").indexOf(fromString("据砖"), 0)); - assertEquals(-1, fromString("数据砖头").indexOf(fromString("数"), 3)); - assertEquals(0, fromString("数据砖头").indexOf(fromString("数"), 0)); - assertEquals(3, fromString("数据砖头").indexOf(fromString("头"), 0)); - } - - @Test - public void substring_index() { - assertEquals(fromString("www.apache.org"), - fromString("www.apache.org").subStringIndex(fromString("."), 3)); - assertEquals(fromString("www.apache"), - fromString("www.apache.org").subStringIndex(fromString("."), 2)); - assertEquals(fromString("www"), - fromString("www.apache.org").subStringIndex(fromString("."), 1)); - assertEquals(fromString(""), - fromString("www.apache.org").subStringIndex(fromString("."), 0)); - assertEquals(fromString("org"), - fromString("www.apache.org").subStringIndex(fromString("."), -1)); - assertEquals(fromString("apache.org"), - fromString("www.apache.org").subStringIndex(fromString("."), -2)); - assertEquals(fromString("www.apache.org"), - fromString("www.apache.org").subStringIndex(fromString("."), -3)); - // str is empty string - assertEquals(fromString(""), - fromString("").subStringIndex(fromString("."), 1)); - // empty string delim - assertEquals(fromString(""), - fromString("www.apache.org").subStringIndex(fromString(""), 1)); - // delim does not exist in str - assertEquals(fromString("www.apache.org"), - fromString("www.apache.org").subStringIndex(fromString("#"), 2)); - // delim is 2 chars - assertEquals(fromString("www||apache"), - fromString("www||apache||org").subStringIndex(fromString("||"), 2)); - assertEquals(fromString("apache||org"), - fromString("www||apache||org").subStringIndex(fromString("||"), -2)); - // non ascii chars - assertEquals(fromString("大千世界大"), - fromString("大千世界大千世界").subStringIndex(fromString("千"), 2)); - // overlapped delim - assertEquals(fromString("||"), fromString("||||||").subStringIndex(fromString("|||"), 3)); - assertEquals(fromString("|||"), fromString("||||||").subStringIndex(fromString("|||"), -4)); - } - - @Test - public void reverse() { - assertEquals(fromString("olleh"), fromString("hello").reverse()); - assertEquals(EMPTY_UTF8, EMPTY_UTF8.reverse()); - assertEquals(fromString("者行孙"), fromString("孙行者").reverse()); - assertEquals(fromString("者行孙 olleh"), fromString("hello 孙行者").reverse()); - } - - @Test - public void repeat() { - assertEquals(fromString("数d数d数d数d数d"), fromString("数d").repeat(5)); - assertEquals(fromString("数d"), fromString("数d").repeat(1)); - assertEquals(EMPTY_UTF8, fromString("数d").repeat(-1)); - } - - @Test - public void pad() { - assertEquals(fromString("hel"), fromString("hello").lpad(3, fromString("????"))); - assertEquals(fromString("hello"), fromString("hello").lpad(5, fromString("????"))); - assertEquals(fromString("?hello"), fromString("hello").lpad(6, fromString("????"))); - assertEquals(fromString("???????hello"), fromString("hello").lpad(12, fromString("????"))); - assertEquals(fromString("?????hello"), fromString("hello").lpad(10, fromString("?????"))); - assertEquals(fromString("???????"), EMPTY_UTF8.lpad(7, fromString("?????"))); - - assertEquals(fromString("hel"), fromString("hello").rpad(3, fromString("????"))); - assertEquals(fromString("hello"), fromString("hello").rpad(5, fromString("????"))); - assertEquals(fromString("hello?"), fromString("hello").rpad(6, fromString("????"))); - assertEquals(fromString("hello???????"), fromString("hello").rpad(12, fromString("????"))); - assertEquals(fromString("hello?????"), fromString("hello").rpad(10, fromString("?????"))); - assertEquals(fromString("???????"), EMPTY_UTF8.rpad(7, fromString("?????"))); - - assertEquals(fromString("数据砖"), fromString("数据砖头").lpad(3, fromString("????"))); - assertEquals(fromString("?数据砖头"), fromString("数据砖头").lpad(5, fromString("????"))); - assertEquals(fromString("??数据砖头"), fromString("数据砖头").lpad(6, fromString("????"))); - assertEquals(fromString("孙行数据砖头"), fromString("数据砖头").lpad(6, fromString("孙行者"))); - assertEquals(fromString("孙行者数据砖头"), fromString("数据砖头").lpad(7, fromString("孙行者"))); - assertEquals( - fromString("孙行者孙行者孙行数据砖头"), - fromString("数据砖头").lpad(12, fromString("孙行者"))); - - assertEquals(fromString("数据砖"), fromString("数据砖头").rpad(3, fromString("????"))); - assertEquals(fromString("数据砖头?"), fromString("数据砖头").rpad(5, fromString("????"))); - assertEquals(fromString("数据砖头??"), fromString("数据砖头").rpad(6, fromString("????"))); - assertEquals(fromString("数据砖头孙行"), fromString("数据砖头").rpad(6, fromString("孙行者"))); - assertEquals(fromString("数据砖头孙行者"), fromString("数据砖头").rpad(7, fromString("孙行者"))); - assertEquals( - fromString("数据砖头孙行者孙行者孙行"), - fromString("数据砖头").rpad(12, fromString("孙行者"))); - - assertEquals(EMPTY_UTF8, fromString("数据砖头").lpad(-10, fromString("孙行者"))); - assertEquals(EMPTY_UTF8, fromString("数据砖头").lpad(-10, EMPTY_UTF8)); - assertEquals(fromString("数据砖头"), fromString("数据砖头").lpad(5, EMPTY_UTF8)); - assertEquals(fromString("数据砖"), fromString("数据砖头").lpad(3, EMPTY_UTF8)); - assertEquals(EMPTY_UTF8, EMPTY_UTF8.lpad(3, EMPTY_UTF8)); - - assertEquals(EMPTY_UTF8, fromString("数据砖头").rpad(-10, fromString("孙行者"))); - assertEquals(EMPTY_UTF8, fromString("数据砖头").rpad(-10, EMPTY_UTF8)); - assertEquals(fromString("数据砖头"), fromString("数据砖头").rpad(5, EMPTY_UTF8)); - assertEquals(fromString("数据砖"), fromString("数据砖头").rpad(3, EMPTY_UTF8)); - assertEquals(EMPTY_UTF8, EMPTY_UTF8.rpad(3, EMPTY_UTF8)); - } - - @Test - public void substringSQL() { - UTF8String e = fromString("example"); - assertEquals(e.substringSQL(0, 2), fromString("ex")); - assertEquals(e.substringSQL(1, 2), fromString("ex")); - assertEquals(e.substringSQL(0, 7), fromString("example")); - assertEquals(e.substringSQL(1, 2), fromString("ex")); - assertEquals(e.substringSQL(0, 100), fromString("example")); - assertEquals(e.substringSQL(1, 100), fromString("example")); - assertEquals(e.substringSQL(2, 2), fromString("xa")); - assertEquals(e.substringSQL(1, 6), fromString("exampl")); - assertEquals(e.substringSQL(2, 100), fromString("xample")); - assertEquals(e.substringSQL(0, 0), fromString("")); - assertEquals(e.substringSQL(100, 4), EMPTY_UTF8); - assertEquals(e.substringSQL(0, Integer.MAX_VALUE), fromString("example")); - assertEquals(e.substringSQL(1, Integer.MAX_VALUE), fromString("example")); - assertEquals(e.substringSQL(2, Integer.MAX_VALUE), fromString("xample")); - } - - @Test - public void split() { - assertTrue(Arrays.equals(fromString("ab,def,ghi").split(fromString(","), -1), - new UTF8String[]{fromString("ab"), fromString("def"), fromString("ghi")})); - assertTrue(Arrays.equals(fromString("ab,def,ghi").split(fromString(","), 2), - new UTF8String[]{fromString("ab"), fromString("def,ghi")})); - assertTrue(Arrays.equals(fromString("ab,def,ghi").split(fromString(","), 2), - new UTF8String[]{fromString("ab"), fromString("def,ghi")})); - } - - @Test - public void levenshteinDistance() { - assertEquals(0, EMPTY_UTF8.levenshteinDistance(EMPTY_UTF8)); - assertEquals(1, EMPTY_UTF8.levenshteinDistance(fromString("a"))); - assertEquals(7, fromString("aaapppp").levenshteinDistance(EMPTY_UTF8)); - assertEquals(1, fromString("frog").levenshteinDistance(fromString("fog"))); - assertEquals(3, fromString("fly").levenshteinDistance(fromString("ant"))); - assertEquals(7, fromString("elephant").levenshteinDistance(fromString("hippo"))); - assertEquals(7, fromString("hippo").levenshteinDistance(fromString("elephant"))); - assertEquals(8, fromString("hippo").levenshteinDistance(fromString("zzzzzzzz"))); - assertEquals(1, fromString("hello").levenshteinDistance(fromString("hallo"))); - assertEquals(4, fromString("世界千世").levenshteinDistance(fromString("千a世b"))); - } - - @Test - public void translate() { - assertEquals( - fromString("1a2s3ae"), - fromString("translate").translate(ImmutableMap.of( - 'r', '1', - 'n', '2', - 'l', '3', - 't', '\0' - ))); - assertEquals( - fromString("translate"), - fromString("translate").translate(new HashMap())); - assertEquals( - fromString("asae"), - fromString("translate").translate(ImmutableMap.of( - 'r', '\0', - 'n', '\0', - 'l', '\0', - 't', '\0' - ))); - assertEquals( - fromString("aa世b"), - fromString("花花世界").translate(ImmutableMap.of( - '花', 'a', - '界', 'b' - ))); - } - - @Test - public void createBlankString() { - assertEquals(fromString(" "), blankString(1)); - assertEquals(fromString(" "), blankString(2)); - assertEquals(fromString(" "), blankString(3)); - assertEquals(fromString(""), blankString(0)); - } - - @Test - public void findInSet() { - assertEquals(1, fromString("ab").findInSet(fromString("ab"))); - assertEquals(2, fromString("a,b").findInSet(fromString("b"))); - assertEquals(3, fromString("abc,b,ab,c,def").findInSet(fromString("ab"))); - assertEquals(1, fromString("ab,abc,b,ab,c,def").findInSet(fromString("ab"))); - assertEquals(4, fromString(",,,ab,abc,b,ab,c,def").findInSet(fromString("ab"))); - assertEquals(1, fromString(",ab,abc,b,ab,c,def").findInSet(fromString(""))); - assertEquals(4, fromString("数据砖头,abc,b,ab,c,def").findInSet(fromString("ab"))); - assertEquals(6, fromString("数据砖头,abc,b,ab,c,def").findInSet(fromString("def"))); - } - - @Test - public void soundex() { - assertEquals(fromString("Robert").soundex(), fromString("R163")); - assertEquals(fromString("Rupert").soundex(), fromString("R163")); - assertEquals(fromString("Rubin").soundex(), fromString("R150")); - assertEquals(fromString("Ashcraft").soundex(), fromString("A261")); - assertEquals(fromString("Ashcroft").soundex(), fromString("A261")); - assertEquals(fromString("Burroughs").soundex(), fromString("B620")); - assertEquals(fromString("Burrows").soundex(), fromString("B620")); - assertEquals(fromString("Ekzampul").soundex(), fromString("E251")); - assertEquals(fromString("Example").soundex(), fromString("E251")); - assertEquals(fromString("Ellery").soundex(), fromString("E460")); - assertEquals(fromString("Euler").soundex(), fromString("E460")); - assertEquals(fromString("Ghosh").soundex(), fromString("G200")); - assertEquals(fromString("Gauss").soundex(), fromString("G200")); - assertEquals(fromString("Gutierrez").soundex(), fromString("G362")); - assertEquals(fromString("Heilbronn").soundex(), fromString("H416")); - assertEquals(fromString("Hilbert").soundex(), fromString("H416")); - assertEquals(fromString("Jackson").soundex(), fromString("J250")); - assertEquals(fromString("Kant").soundex(), fromString("K530")); - assertEquals(fromString("Knuth").soundex(), fromString("K530")); - assertEquals(fromString("Lee").soundex(), fromString("L000")); - assertEquals(fromString("Lukasiewicz").soundex(), fromString("L222")); - assertEquals(fromString("Lissajous").soundex(), fromString("L222")); - assertEquals(fromString("Ladd").soundex(), fromString("L300")); - assertEquals(fromString("Lloyd").soundex(), fromString("L300")); - assertEquals(fromString("Moses").soundex(), fromString("M220")); - assertEquals(fromString("O'Hara").soundex(), fromString("O600")); - assertEquals(fromString("Pfister").soundex(), fromString("P236")); - assertEquals(fromString("Rubin").soundex(), fromString("R150")); - assertEquals(fromString("Robert").soundex(), fromString("R163")); - assertEquals(fromString("Rupert").soundex(), fromString("R163")); - assertEquals(fromString("Soundex").soundex(), fromString("S532")); - assertEquals(fromString("Sownteks").soundex(), fromString("S532")); - assertEquals(fromString("Tymczak").soundex(), fromString("T522")); - assertEquals(fromString("VanDeusen").soundex(), fromString("V532")); - assertEquals(fromString("Washington").soundex(), fromString("W252")); - assertEquals(fromString("Wheaton").soundex(), fromString("W350")); - - assertEquals(fromString("a").soundex(), fromString("A000")); - assertEquals(fromString("ab").soundex(), fromString("A100")); - assertEquals(fromString("abc").soundex(), fromString("A120")); - assertEquals(fromString("abcd").soundex(), fromString("A123")); - assertEquals(fromString("").soundex(), fromString("")); - assertEquals(fromString("123").soundex(), fromString("123")); - assertEquals(fromString("世界千世").soundex(), fromString("世界千世")); - } -} diff --git a/yarn/pom.xml b/yarn/pom.xml deleted file mode 100644 index 989b820bec9e..000000000000 --- a/yarn/pom.xml +++ /dev/null @@ -1,197 +0,0 @@ - - - - 4.0.0 - - org.apache.spark - spark-parent_2.10 - 1.6.0-SNAPSHOT - ../pom.xml - - - org.apache.spark - spark-yarn_2.10 - jar - Spark Project YARN - - yarn - - - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-network-yarn_${scala.binary.version} - ${project.version} - test - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - org.apache.hadoop - hadoop-yarn-api - - - org.apache.hadoop - hadoop-yarn-common - - - org.apache.hadoop - hadoop-yarn-server-web-proxy - - - org.apache.hadoop - hadoop-yarn-client - - - org.apache.hadoop - hadoop-client - - - - - com.google.guava - guava - - - org.eclipse.jetty - jetty-server - - - org.eclipse.jetty - jetty-plus - - - org.eclipse.jetty - jetty-util - - - org.eclipse.jetty - jetty-http - - - org.eclipse.jetty - jetty-servlet - - - - - - org.eclipse.jetty.orbit - javax.servlet.jsp - 2.2.0.v201112011158 - test - - - org.eclipse.jetty.orbit - javax.servlet.jsp.jstl - 1.2.0.v201105211821 - test - - - - - - org.apache.hadoop - hadoop-yarn-server-tests - tests - test - - - org.mockito - mockito-core - test - - - org.mortbay.jetty - jetty - 6.1.26 - - - org.mortbay.jetty - servlet-api - - - test - - - com.sun.jersey - jersey-core - test - - - com.sun.jersey - jersey-json - test - - - com.sun.jersey - jersey-server - test - - - - - ${hive.group} - hive-exec - test - - - ${hive.group} - hive-metastore - test - - - org.apache.thrift - libthrift - test - - - org.apache.thrift - libfb303 - test - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala deleted file mode 100644 index 56e4741b9387..000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala +++ /dev/null @@ -1,209 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.yarn - -import java.security.PrivilegedExceptionAction -import java.util.concurrent.{Executors, TimeUnit} - -import scala.language.postfixOps - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.security.UserGroupInformation -import org.apache.spark.deploy.SparkHadoopUtil - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.util.ThreadUtils - -/* - * The following methods are primarily meant to make sure long-running apps like Spark - * Streaming apps can run without interruption while writing to secure HDFS. The - * scheduleLoginFromKeytab method is called on the driver when the - * CoarseGrainedScheduledBackend starts up. This method wakes up a thread that logs into the KDC - * once 75% of the renewal interval of the original delegation tokens used for the container - * has elapsed. It then creates new delegation tokens and writes them to HDFS in a - * pre-specified location - the prefix of which is specified in the sparkConf by - * spark.yarn.credentials.file (so the file(s) would be named c-1, c-2 etc. - each update goes - * to a new file, with a monotonically increasing suffix). After this, the credentials are - * updated once 75% of the new tokens renewal interval has elapsed. - * - * On the executor side, the updateCredentialsIfRequired method is called once 80% of the - * validity of the original tokens has elapsed. At that time the executor finds the - * credentials file with the latest timestamp and checks if it has read those credentials - * before (by keeping track of the suffix of the last file it read). If a new file has - * appeared, it will read the credentials and update the currently running UGI with it. This - * process happens again once 80% of the validity of this has expired. - */ -private[yarn] class AMDelegationTokenRenewer( - sparkConf: SparkConf, - hadoopConf: Configuration) extends Logging { - - private var lastCredentialsFileSuffix = 0 - - private val delegationTokenRenewer = - Executors.newSingleThreadScheduledExecutor( - ThreadUtils.namedThreadFactory("Delegation Token Refresh Thread")) - - private val hadoopUtil = YarnSparkHadoopUtil.get - - private val credentialsFile = sparkConf.get("spark.yarn.credentials.file") - private val daysToKeepFiles = - sparkConf.getInt("spark.yarn.credentials.file.retention.days", 5) - private val numFilesToKeep = - sparkConf.getInt("spark.yarn.credentials.file.retention.count", 5) - private val freshHadoopConf = - hadoopUtil.getConfBypassingFSCache(hadoopConf, new Path(credentialsFile).toUri.getScheme) - - /** - * Schedule a login from the keytab and principal set using the --principal and --keytab - * arguments to spark-submit. This login happens only when the credentials of the current user - * are about to expire. This method reads spark.yarn.principal and spark.yarn.keytab from - * SparkConf to do the login. This method is a no-op in non-YARN mode. - * - */ - private[spark] def scheduleLoginFromKeytab(): Unit = { - val principal = sparkConf.get("spark.yarn.principal") - val keytab = sparkConf.get("spark.yarn.keytab") - - /** - * Schedule re-login and creation of new tokens. If tokens have already expired, this method - * will synchronously create new ones. - */ - def scheduleRenewal(runnable: Runnable): Unit = { - val credentials = UserGroupInformation.getCurrentUser.getCredentials - val renewalInterval = hadoopUtil.getTimeFromNowToRenewal(sparkConf, 0.75, credentials) - // Run now! - if (renewalInterval <= 0) { - logInfo("HDFS tokens have expired, creating new tokens now.") - runnable.run() - } else { - logInfo(s"Scheduling login from keytab in $renewalInterval millis.") - delegationTokenRenewer.schedule(runnable, renewalInterval, TimeUnit.MILLISECONDS) - } - } - - // This thread periodically runs on the driver to update the delegation tokens on HDFS. - val driverTokenRenewerRunnable = - new Runnable { - override def run(): Unit = { - try { - writeNewTokensToHDFS(principal, keytab) - cleanupOldFiles() - } catch { - case e: Exception => - // Log the error and try to write new tokens back in an hour - logWarning("Failed to write out new credentials to HDFS, will try again in an " + - "hour! If this happens too often tasks will fail.", e) - delegationTokenRenewer.schedule(this, 1, TimeUnit.HOURS) - return - } - scheduleRenewal(this) - } - } - // Schedule update of credentials. This handles the case of updating the tokens right now - // as well, since the renenwal interval will be 0, and the thread will get scheduled - // immediately. - scheduleRenewal(driverTokenRenewerRunnable) - } - - // Keeps only files that are newer than daysToKeepFiles days, and deletes everything else. At - // least numFilesToKeep files are kept for safety - private def cleanupOldFiles(): Unit = { - import scala.concurrent.duration._ - try { - val remoteFs = FileSystem.get(freshHadoopConf) - val credentialsPath = new Path(credentialsFile) - val thresholdTime = System.currentTimeMillis() - (daysToKeepFiles days).toMillis - hadoopUtil.listFilesSorted( - remoteFs, credentialsPath.getParent, - credentialsPath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION) - .dropRight(numFilesToKeep) - .takeWhile(_.getModificationTime < thresholdTime) - .foreach(x => remoteFs.delete(x.getPath, true)) - } catch { - // Such errors are not fatal, so don't throw. Make sure they are logged though - case e: Exception => - logWarning("Error while attempting to cleanup old tokens. If you are seeing many such " + - "warnings there may be an issue with your HDFS cluster.", e) - } - } - - private def writeNewTokensToHDFS(principal: String, keytab: String): Unit = { - // Keytab is copied by YARN to the working directory of the AM, so full path is - // not needed. - - // HACK: - // HDFS will not issue new delegation tokens, if the Credentials object - // passed in already has tokens for that FS even if the tokens are expired (it really only - // checks if there are tokens for the service, and not if they are valid). So the only real - // way to get new tokens is to make sure a different Credentials object is used each time to - // get new tokens and then the new tokens are copied over the the current user's Credentials. - // So: - // - we login as a different user and get the UGI - // - use that UGI to get the tokens (see doAs block below) - // - copy the tokens over to the current user's credentials (this will overwrite the tokens - // in the current user's Credentials object for this FS). - // The login to KDC happens each time new tokens are required, but this is rare enough to not - // have to worry about (like once every day or so). This makes this code clearer than having - // to login and then relogin every time (the HDFS API may not relogin since we don't use this - // UGI directly for HDFS communication. - logInfo(s"Attempting to login to KDC using principal: $principal") - val keytabLoggedInUGI = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) - logInfo("Successfully logged into KDC.") - val tempCreds = keytabLoggedInUGI.getCredentials - val credentialsPath = new Path(credentialsFile) - val dst = credentialsPath.getParent - keytabLoggedInUGI.doAs(new PrivilegedExceptionAction[Void] { - // Get a copy of the credentials - override def run(): Void = { - val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + dst - hadoopUtil.obtainTokensForNamenodes(nns, freshHadoopConf, tempCreds) - null - } - }) - // Add the temp credentials back to the original ones. - UserGroupInformation.getCurrentUser.addCredentials(tempCreds) - val remoteFs = FileSystem.get(freshHadoopConf) - // If lastCredentialsFileSuffix is 0, then the AM is either started or restarted. If the AM - // was restarted, then the lastCredentialsFileSuffix might be > 0, so find the newest file - // and update the lastCredentialsFileSuffix. - if (lastCredentialsFileSuffix == 0) { - hadoopUtil.listFilesSorted( - remoteFs, credentialsPath.getParent, - credentialsPath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION) - .lastOption.foreach { status => - lastCredentialsFileSuffix = hadoopUtil.getSuffixForCredentialsPath(status.getPath) - } - } - val nextSuffix = lastCredentialsFileSuffix + 1 - val tokenPathStr = - credentialsFile + SparkHadoopUtil.SPARK_YARN_CREDS_COUNTER_DELIM + nextSuffix - val tokenPath = new Path(tokenPathStr) - val tempTokenPath = new Path(tokenPathStr + SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION) - logInfo("Writing out delegation tokens to " + tempTokenPath.toString) - val credentials = UserGroupInformation.getCurrentUser.getCredentials - credentials.writeTokenStorageFile(tempTokenPath, freshHadoopConf) - logInfo(s"Delegation Tokens written out successfully. Renaming file to $tokenPathStr") - remoteFs.rename(tempTokenPath, tokenPath) - logInfo("Delegation token file rename complete.") - lastCredentialsFileSuffix = nextSuffix - } - - def stop(): Unit = { - delegationTokenRenewer.shutdown() - } -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala deleted file mode 100644 index 50ae7ffeec4c..000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ /dev/null @@ -1,677 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import scala.util.control.NonFatal - -import java.io.{File, IOException} -import java.lang.reflect.InvocationTargetException -import java.net.{Socket, URL} -import java.util.concurrent.atomic.AtomicReference - -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.conf.YarnConfiguration - -import org.apache.spark.rpc._ -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv, - SparkException, SparkUserAppException} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.history.HistoryServer -import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, YarnSchedulerBackend} -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util._ - -/** - * Common application master functionality for Spark on Yarn. - */ -private[spark] class ApplicationMaster( - args: ApplicationMasterArguments, - client: YarnRMClient) - extends Logging { - - // Load the properties file with the Spark configuration and set entries as system properties, - // so that user code run inside the AM also has access to them. - if (args.propertiesFile != null) { - Utils.getPropertiesFromFile(args.propertiesFile).foreach { case (k, v) => - sys.props(k) = v - } - } - - // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be - // optimal as more containers are available. Might need to handle this better. - - private val sparkConf = new SparkConf() - private val yarnConf: YarnConfiguration = SparkHadoopUtil.get.newConfiguration(sparkConf) - .asInstanceOf[YarnConfiguration] - private val isClusterMode = args.userClass != null - - // Default to twice the number of executors (twice the maximum number of executors if dynamic - // allocation is enabled), with a minimum of 3. - - private val maxNumExecutorFailures = { - val defaultKey = - if (Utils.isDynamicAllocationEnabled(sparkConf)) { - "spark.dynamicAllocation.maxExecutors" - } else { - "spark.executor.instances" - } - val effectiveNumExecutors = sparkConf.getInt(defaultKey, 0) - val defaultMaxNumExecutorFailures = math.max(3, 2 * effectiveNumExecutors) - - sparkConf.getInt("spark.yarn.max.executor.failures", defaultMaxNumExecutorFailures) - } - - @volatile private var exitCode = 0 - @volatile private var unregistered = false - @volatile private var finished = false - @volatile private var finalStatus = getDefaultFinalStatus - @volatile private var finalMsg: String = "" - @volatile private var userClassThread: Thread = _ - - @volatile private var reporterThread: Thread = _ - @volatile private var allocator: YarnAllocator = _ - - // Lock for controlling the allocator (heartbeat) thread. - private val allocatorLock = new Object() - - // Steady state heartbeat interval. We want to be reasonably responsive without causing too many - // requests to RM. - private val heartbeatInterval = { - // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses. - val expiryInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) - math.max(0, math.min(expiryInterval / 2, - sparkConf.getTimeAsMs("spark.yarn.scheduler.heartbeat.interval-ms", "3s"))) - } - - // Initial wait interval before allocator poll, to allow for quicker ramp up when executors are - // being requested. - private val initialAllocationInterval = math.min(heartbeatInterval, - sparkConf.getTimeAsMs("spark.yarn.scheduler.initial-allocation.interval", "200ms")) - - // Next wait interval before allocator poll. - private var nextAllocationInterval = initialAllocationInterval - - // Fields used in client mode. - private var rpcEnv: RpcEnv = null - private var amEndpoint: RpcEndpointRef = _ - - // Fields used in cluster mode. - private val sparkContextRef = new AtomicReference[SparkContext](null) - - private var delegationTokenRenewerOption: Option[AMDelegationTokenRenewer] = None - - final def run(): Int = { - try { - val appAttemptId = client.getAttemptId() - - if (isClusterMode) { - // Set the web ui port to be ephemeral for yarn so we don't conflict with - // other spark processes running on the same box - System.setProperty("spark.ui.port", "0") - - // Set the master property to match the requested mode. - System.setProperty("spark.master", "yarn-cluster") - - // Propagate the application ID so that YarnClusterSchedulerBackend can pick it up. - System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString()) - - // Propagate the attempt if, so that in case of event logging, - // different attempt's logs gets created in different directory - System.setProperty("spark.yarn.app.attemptId", appAttemptId.getAttemptId().toString()) - } - - logInfo("ApplicationAttemptId: " + appAttemptId) - - val fs = FileSystem.get(yarnConf) - - // This shutdown hook should run *after* the SparkContext is shut down. - val priority = ShutdownHookManager.SPARK_CONTEXT_SHUTDOWN_PRIORITY - 1 - ShutdownHookManager.addShutdownHook(priority) { () => - val maxAppAttempts = client.getMaxRegAttempts(sparkConf, yarnConf) - val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts - - if (!finished) { - // This happens when the user application calls System.exit(). We have the choice - // of either failing or succeeding at this point. We report success to avoid - // retrying applications that have succeeded (System.exit(0)), which means that - // applications that explicitly exit with a non-zero status will also show up as - // succeeded in the RM UI. - finish(finalStatus, - ApplicationMaster.EXIT_SUCCESS, - "Shutdown hook called before final status was reported.") - } - - if (!unregistered) { - // we only want to unregister if we don't want the RM to retry - if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) { - unregister(finalStatus, finalMsg) - cleanupStagingDir(fs) - } - } - } - - // Call this to force generation of secret so it gets populated into the - // Hadoop UGI. This has to happen before the startUserApplication which does a - // doAs in order for the credentials to be passed on to the executor containers. - val securityMgr = new SecurityManager(sparkConf) - - // If the credentials file config is present, we must periodically renew tokens. So create - // a new AMDelegationTokenRenewer - if (sparkConf.contains("spark.yarn.credentials.file")) { - delegationTokenRenewerOption = Some(new AMDelegationTokenRenewer(sparkConf, yarnConf)) - // If a principal and keytab have been set, use that to create new credentials for executors - // periodically - delegationTokenRenewerOption.foreach(_.scheduleLoginFromKeytab()) - } - - if (isClusterMode) { - runDriver(securityMgr) - } else { - runExecutorLauncher(securityMgr) - } - } catch { - case e: Exception => - // catch everything else if not specifically handled - logError("Uncaught exception: ", e) - finish(FinalApplicationStatus.FAILED, - ApplicationMaster.EXIT_UNCAUGHT_EXCEPTION, - "Uncaught exception: " + e) - } - exitCode - } - - /** - * Set the default final application status for client mode to UNDEFINED to handle - * if YARN HA restarts the application so that it properly retries. Set the final - * status to SUCCEEDED in cluster mode to handle if the user calls System.exit - * from the application code. - */ - final def getDefaultFinalStatus(): FinalApplicationStatus = { - if (isClusterMode) { - FinalApplicationStatus.SUCCEEDED - } else { - FinalApplicationStatus.UNDEFINED - } - } - - /** - * unregister is used to completely unregister the application from the ResourceManager. - * This means the ResourceManager will not retry the application attempt on your behalf if - * a failure occurred. - */ - final def unregister(status: FinalApplicationStatus, diagnostics: String = null): Unit = { - synchronized { - if (!unregistered) { - logInfo(s"Unregistering ApplicationMaster with $status" + - Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse("")) - unregistered = true - client.unregister(status, Option(diagnostics).getOrElse("")) - } - } - } - - final def finish(status: FinalApplicationStatus, code: Int, msg: String = null): Unit = { - synchronized { - if (!finished) { - val inShutdown = ShutdownHookManager.inShutdown() - logInfo(s"Final app status: $status, exitCode: $code" + - Option(msg).map(msg => s", (reason: $msg)").getOrElse("")) - exitCode = code - finalStatus = status - finalMsg = msg - finished = true - if (!inShutdown && Thread.currentThread() != reporterThread && reporterThread != null) { - logDebug("shutting down reporter thread") - reporterThread.interrupt() - } - if (!inShutdown && Thread.currentThread() != userClassThread && userClassThread != null) { - logDebug("shutting down user thread") - userClassThread.interrupt() - } - if (!inShutdown) delegationTokenRenewerOption.foreach(_.stop()) - } - } - } - - private def sparkContextInitialized(sc: SparkContext) = { - sparkContextRef.synchronized { - sparkContextRef.compareAndSet(null, sc) - sparkContextRef.notifyAll() - } - } - - private def sparkContextStopped(sc: SparkContext) = { - sparkContextRef.compareAndSet(sc, null) - } - - private def registerAM( - _rpcEnv: RpcEnv, - driverRef: RpcEndpointRef, - uiAddress: String, - securityMgr: SecurityManager) = { - val sc = sparkContextRef.get() - - val appId = client.getAttemptId().getApplicationId().toString() - val attemptId = client.getAttemptId().getAttemptId().toString() - val historyAddress = - sparkConf.getOption("spark.yarn.historyServer.address") - .map { text => SparkHadoopUtil.get.substituteHadoopVariables(text, yarnConf) } - .map { address => s"${address}${HistoryServer.UI_PATH_PREFIX}/${appId}/${attemptId}" } - .getOrElse("") - - val _sparkConf = if (sc != null) sc.getConf else sparkConf - val driverUrl = _rpcEnv.uriOf( - SparkEnv.driverActorSystemName, - RpcAddress(_sparkConf.get("spark.driver.host"), _sparkConf.get("spark.driver.port").toInt), - CoarseGrainedSchedulerBackend.ENDPOINT_NAME) - allocator = client.register(driverUrl, - driverRef, - yarnConf, - _sparkConf, - uiAddress, - historyAddress, - securityMgr) - - allocator.allocateResources() - reporterThread = launchReporterThread() - } - - /** - * Create an [[RpcEndpoint]] that communicates with the driver. - * - * In cluster mode, the AM and the driver belong to same process - * so the AMEndpoint need not monitor lifecycle of the driver. - * - * @return A reference to the driver's RPC endpoint. - */ - private def runAMEndpoint( - host: String, - port: String, - isClusterMode: Boolean): RpcEndpointRef = { - val driverEndpoint = rpcEnv.setupEndpointRef( - SparkEnv.driverActorSystemName, - RpcAddress(host, port.toInt), - YarnSchedulerBackend.ENDPOINT_NAME) - amEndpoint = - rpcEnv.setupEndpoint("YarnAM", new AMEndpoint(rpcEnv, driverEndpoint, isClusterMode)) - driverEndpoint - } - - private def runDriver(securityMgr: SecurityManager): Unit = { - addAmIpFilter() - userClassThread = startUserApplication() - - // This a bit hacky, but we need to wait until the spark.driver.port property has - // been set by the Thread executing the user class. - val sc = waitForSparkContextInitialized() - - // If there is no SparkContext at this point, just fail the app. - if (sc == null) { - finish(FinalApplicationStatus.FAILED, - ApplicationMaster.EXIT_SC_NOT_INITED, - "Timed out waiting for SparkContext.") - } else { - rpcEnv = sc.env.rpcEnv - val driverRef = runAMEndpoint( - sc.getConf.get("spark.driver.host"), - sc.getConf.get("spark.driver.port"), - isClusterMode = true) - registerAM(rpcEnv, driverRef, sc.ui.map(_.appUIAddress).getOrElse(""), securityMgr) - userClassThread.join() - } - } - - private def runExecutorLauncher(securityMgr: SecurityManager): Unit = { - val port = sparkConf.getInt("spark.yarn.am.port", 0) - rpcEnv = RpcEnv.create("sparkYarnAM", Utils.localHostName, port, sparkConf, securityMgr, - clientMode = true) - val driverRef = waitForSparkDriver() - addAmIpFilter() - registerAM(rpcEnv, driverRef, sparkConf.get("spark.driver.appUIAddress", ""), securityMgr) - - // In client mode the actor will stop the reporter thread. - reporterThread.join() - } - - private def launchReporterThread(): Thread = { - // The number of failures in a row until Reporter thread give up - val reporterMaxFailures = sparkConf.getInt("spark.yarn.scheduler.reporterThread.maxFailures", 5) - - val t = new Thread { - override def run() { - var failureCount = 0 - while (!finished) { - try { - if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) { - finish(FinalApplicationStatus.FAILED, - ApplicationMaster.EXIT_MAX_EXECUTOR_FAILURES, - s"Max number of executor failures ($maxNumExecutorFailures) reached") - } else { - logDebug("Sending progress") - allocator.allocateResources() - } - failureCount = 0 - } catch { - case i: InterruptedException => - case e: Throwable => { - failureCount += 1 - if (!NonFatal(e) || failureCount >= reporterMaxFailures) { - finish(FinalApplicationStatus.FAILED, - ApplicationMaster.EXIT_REPORTER_FAILURE, "Exception was thrown " + - s"$failureCount time(s) from Reporter thread.") - } else { - logWarning(s"Reporter thread fails $failureCount time(s) in a row.", e) - } - } - } - try { - val numPendingAllocate = allocator.getPendingAllocate.size - allocatorLock.synchronized { - val sleepInterval = - if (numPendingAllocate > 0 || allocator.getNumPendingLossReasonRequests > 0) { - val currentAllocationInterval = - math.min(heartbeatInterval, nextAllocationInterval) - nextAllocationInterval = currentAllocationInterval * 2 // avoid overflow - currentAllocationInterval - } else { - nextAllocationInterval = initialAllocationInterval - heartbeatInterval - } - logDebug(s"Number of pending allocations is $numPendingAllocate. " + - s"Sleeping for $sleepInterval.") - allocatorLock.wait(sleepInterval) - } - } catch { - case e: InterruptedException => - } - } - } - } - // setting to daemon status, though this is usually not a good idea. - t.setDaemon(true) - t.setName("Reporter") - t.start() - logInfo(s"Started progress reporter thread with (heartbeat : $heartbeatInterval, " + - s"initial allocation : $initialAllocationInterval) intervals") - t - } - - /** - * Clean up the staging directory. - */ - private def cleanupStagingDir(fs: FileSystem) { - var stagingDirPath: Path = null - try { - val preserveFiles = sparkConf.getBoolean("spark.yarn.preserve.staging.files", false) - if (!preserveFiles) { - stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) - if (stagingDirPath == null) { - logError("Staging directory is null") - return - } - logInfo("Deleting staging directory " + stagingDirPath) - fs.delete(stagingDirPath, true) - } - } catch { - case ioe: IOException => - logError("Failed to cleanup staging dir " + stagingDirPath, ioe) - } - } - - private def waitForSparkContextInitialized(): SparkContext = { - logInfo("Waiting for spark context initialization") - sparkContextRef.synchronized { - val totalWaitTime = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", "100s") - val deadline = System.currentTimeMillis() + totalWaitTime - - while (sparkContextRef.get() == null && System.currentTimeMillis < deadline && !finished) { - logInfo("Waiting for spark context initialization ... ") - sparkContextRef.wait(10000L) - } - - val sparkContext = sparkContextRef.get() - if (sparkContext == null) { - logError(("SparkContext did not initialize after waiting for %d ms. Please check earlier" - + " log output for errors. Failing the application.").format(totalWaitTime)) - } - sparkContext - } - } - - private def waitForSparkDriver(): RpcEndpointRef = { - logInfo("Waiting for Spark driver to be reachable.") - var driverUp = false - val hostport = args.userArgs(0) - val (driverHost, driverPort) = Utils.parseHostPort(hostport) - - // Spark driver should already be up since it launched us, but we don't want to - // wait forever, so wait 100 seconds max to match the cluster mode setting. - val totalWaitTimeMs = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", "100s") - val deadline = System.currentTimeMillis + totalWaitTimeMs - - while (!driverUp && !finished && System.currentTimeMillis < deadline) { - try { - val socket = new Socket(driverHost, driverPort) - socket.close() - logInfo("Driver now available: %s:%s".format(driverHost, driverPort)) - driverUp = true - } catch { - case e: Exception => - logError("Failed to connect to driver at %s:%s, retrying ...". - format(driverHost, driverPort)) - Thread.sleep(100L) - } - } - - if (!driverUp) { - throw new SparkException("Failed to connect to driver!") - } - - sparkConf.set("spark.driver.host", driverHost) - sparkConf.set("spark.driver.port", driverPort.toString) - - runAMEndpoint(driverHost, driverPort.toString, isClusterMode = false) - } - - /** Add the Yarn IP filter that is required for properly securing the UI. */ - private def addAmIpFilter() = { - val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) - val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" - val params = client.getAmIpFilterParams(yarnConf, proxyBase) - if (isClusterMode) { - System.setProperty("spark.ui.filters", amFilter) - params.foreach { case (k, v) => System.setProperty(s"spark.$amFilter.param.$k", v) } - } else { - amEndpoint.send(AddWebUIFilter(amFilter, params.toMap, proxyBase)) - } - } - - /** - * Start the user class, which contains the spark driver, in a separate Thread. - * If the main routine exits cleanly or exits with System.exit(N) for any N - * we assume it was successful, for all other cases we assume failure. - * - * Returns the user thread that was started. - */ - private def startUserApplication(): Thread = { - logInfo("Starting the user application in a separate Thread") - - val classpath = Client.getUserClasspath(sparkConf) - val urls = classpath.map { entry => - new URL("file:" + new File(entry.getPath()).getAbsolutePath()) - } - val userClassLoader = - if (Client.isUserClassPathFirst(sparkConf, isDriver = true)) { - new ChildFirstURLClassLoader(urls, Utils.getContextOrSparkClassLoader) - } else { - new MutableURLClassLoader(urls, Utils.getContextOrSparkClassLoader) - } - - var userArgs = args.userArgs - if (args.primaryPyFile != null && args.primaryPyFile.endsWith(".py")) { - // When running pyspark, the app is run using PythonRunner. The second argument is the list - // of files to add to PYTHONPATH, which Client.scala already handles, so it's empty. - userArgs = Seq(args.primaryPyFile, "") ++ userArgs - } - if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) { - // TODO(davies): add R dependencies here - } - val mainMethod = userClassLoader.loadClass(args.userClass) - .getMethod("main", classOf[Array[String]]) - - val userThread = new Thread { - override def run() { - try { - mainMethod.invoke(null, userArgs.toArray) - finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) - logDebug("Done running users class") - } catch { - case e: InvocationTargetException => - e.getCause match { - case _: InterruptedException => - // Reporter thread can interrupt to stop user class - case SparkUserAppException(exitCode) => - val msg = s"User application exited with status $exitCode" - logError(msg) - finish(FinalApplicationStatus.FAILED, exitCode, msg) - case cause: Throwable => - logError("User class threw exception: " + cause, cause) - finish(FinalApplicationStatus.FAILED, - ApplicationMaster.EXIT_EXCEPTION_USER_CLASS, - "User class threw exception: " + cause) - } - } - } - } - userThread.setContextClassLoader(userClassLoader) - userThread.setName("Driver") - userThread.start() - userThread - } - - private def resetAllocatorInterval(): Unit = allocatorLock.synchronized { - nextAllocationInterval = initialAllocationInterval - allocatorLock.notifyAll() - } - - /** - * An [[RpcEndpoint]] that communicates with the driver's scheduler backend. - */ - private class AMEndpoint( - override val rpcEnv: RpcEnv, driver: RpcEndpointRef, isClusterMode: Boolean) - extends RpcEndpoint with Logging { - - override def onStart(): Unit = { - driver.send(RegisterClusterManager(self)) - } - - override def receive: PartialFunction[Any, Unit] = { - case x: AddWebUIFilter => - logInfo(s"Add WebUI Filter. $x") - driver.send(x) - } - - override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount) => - Option(allocator) match { - case Some(a) => - if (a.requestTotalExecutorsWithPreferredLocalities(requestedTotal, - localityAwareTasks, hostToLocalTaskCount)) { - resetAllocatorInterval() - } - - case None => - logWarning("Container allocator is not ready to request executors yet.") - } - context.reply(true) - - case KillExecutors(executorIds) => - logInfo(s"Driver requested to kill executor(s) ${executorIds.mkString(", ")}.") - Option(allocator) match { - case Some(a) => executorIds.foreach(a.killExecutor) - case None => logWarning("Container allocator is not ready to kill executors yet.") - } - context.reply(true) - - case GetExecutorLossReason(eid) => - Option(allocator) match { - case Some(a) => - a.enqueueGetLossReasonRequest(eid, context) - resetAllocatorInterval() - case None => - logWarning("Container allocator is not ready to find executor loss reasons yet.") - } - } - - override def onDisconnected(remoteAddress: RpcAddress): Unit = { - // In cluster mode, do not rely on the disassociated event to exit - // This avoids potentially reporting incorrect exit codes if the driver fails - if (!isClusterMode) { - logInfo(s"Driver terminated or disconnected! Shutting down. $remoteAddress") - finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) - } - } - } - -} - -object ApplicationMaster extends Logging { - - // exit codes for different causes, no reason behind the values - private val EXIT_SUCCESS = 0 - private val EXIT_UNCAUGHT_EXCEPTION = 10 - private val EXIT_MAX_EXECUTOR_FAILURES = 11 - private val EXIT_REPORTER_FAILURE = 12 - private val EXIT_SC_NOT_INITED = 13 - private val EXIT_SECURITY = 14 - private val EXIT_EXCEPTION_USER_CLASS = 15 - - private var master: ApplicationMaster = _ - - def main(args: Array[String]): Unit = { - SignalLogger.register(log) - val amArgs = new ApplicationMasterArguments(args) - SparkHadoopUtil.get.runAsSparkUser { () => - master = new ApplicationMaster(amArgs, new YarnRMClient(amArgs)) - System.exit(master.run()) - } - } - - private[spark] def sparkContextInitialized(sc: SparkContext): Unit = { - master.sparkContextInitialized(sc) - } - - private[spark] def sparkContextStopped(sc: SparkContext): Boolean = { - master.sparkContextStopped(sc) - } - -} - -/** - * This object does not provide any special functionality. It exists so that it's easy to tell - * apart the client-mode AM from the cluster-mode AM when using tools such as ps or jps. - */ -object ExecutorLauncher { - - def main(args: Array[String]): Unit = { - ApplicationMaster.main(args) - } - -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala deleted file mode 100644 index a3f33d80184a..000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ /dev/null @@ -1,1458 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.io.{ByteArrayInputStream, DataInputStream, File, FileOutputStream, IOException, - OutputStreamWriter} -import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} -import java.nio.ByteBuffer -import java.security.PrivilegedExceptionAction -import java.util.{Properties, UUID} -import java.util.zip.{ZipEntry, ZipOutputStream} - -import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map} -import scala.reflect.runtime.universe -import scala.util.{Try, Success, Failure} -import scala.util.control.NonFatal - -import com.google.common.base.Charsets.UTF_8 -import com.google.common.base.Objects -import com.google.common.io.Files - -import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier -import org.apache.hadoop.fs._ -import org.apache.hadoop.fs.permission.FsPermission -import org.apache.hadoop.io.Text -import org.apache.hadoop.mapreduce.MRJobConfig -import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.hadoop.security.token.{TokenIdentifier, Token} -import org.apache.hadoop.util.StringUtils -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication} -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException -import org.apache.hadoop.yarn.util.Records - -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} -import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.util.Utils - -private[spark] class Client( - val args: ClientArguments, - val hadoopConf: Configuration, - val sparkConf: SparkConf) - extends Logging { - - import Client._ - - def this(clientArgs: ClientArguments, spConf: SparkConf) = - this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf) - - private val yarnClient = YarnClient.createYarnClient - private val yarnConf = new YarnConfiguration(hadoopConf) - private var credentials: Credentials = null - private val amMemoryOverhead = args.amMemoryOverhead // MB - private val executorMemoryOverhead = args.executorMemoryOverhead // MB - private val distCacheMgr = new ClientDistributedCacheManager() - private val isClusterMode = args.isClusterMode - - private var loginFromKeytab = false - private var principal: String = null - private var keytab: String = null - - private val launcherBackend = new LauncherBackend() { - override def onStopRequest(): Unit = { - if (isClusterMode && appId != null) { - yarnClient.killApplication(appId) - } else { - setState(SparkAppHandle.State.KILLED) - stop() - } - } - } - private val fireAndForget = isClusterMode && - !sparkConf.getBoolean("spark.yarn.submit.waitAppCompletion", true) - - private var appId: ApplicationId = null - - def reportLauncherState(state: SparkAppHandle.State): Unit = { - launcherBackend.setState(state) - } - - def stop(): Unit = { - launcherBackend.close() - yarnClient.stop() - // Unset YARN mode system env variable, to allow switching between cluster types. - System.clearProperty("SPARK_YARN_MODE") - } - - /** - * Submit an application running our ApplicationMaster to the ResourceManager. - * - * The stable Yarn API provides a convenience method (YarnClient#createApplication) for - * creating applications and setting up the application submission context. This was not - * available in the alpha API. - */ - def submitApplication(): ApplicationId = { - var appId: ApplicationId = null - try { - launcherBackend.connect() - // Setup the credentials before doing anything else, - // so we have don't have issues at any point. - setupCredentials() - yarnClient.init(yarnConf) - yarnClient.start() - - logInfo("Requesting a new application from cluster with %d NodeManagers" - .format(yarnClient.getYarnClusterMetrics.getNumNodeManagers)) - - // Get a new application from our RM - val newApp = yarnClient.createApplication() - val newAppResponse = newApp.getNewApplicationResponse() - appId = newAppResponse.getApplicationId() - reportLauncherState(SparkAppHandle.State.SUBMITTED) - launcherBackend.setAppId(appId.toString()) - - // Verify whether the cluster has enough resources for our AM - verifyClusterResources(newAppResponse) - - // Set up the appropriate contexts to launch our AM - val containerContext = createContainerLaunchContext(newAppResponse) - val appContext = createApplicationSubmissionContext(newApp, containerContext) - - // Finally, submit and monitor the application - logInfo(s"Submitting application ${appId.getId} to ResourceManager") - yarnClient.submitApplication(appContext) - appId - } catch { - case e: Throwable => - if (appId != null) { - cleanupStagingDir(appId) - } - throw e - } - } - - /** - * Cleanup application staging directory. - */ - private def cleanupStagingDir(appId: ApplicationId): Unit = { - val appStagingDir = getAppStagingDir(appId) - try { - val preserveFiles = sparkConf.getBoolean("spark.yarn.preserve.staging.files", false) - val stagingDirPath = new Path(appStagingDir) - val fs = FileSystem.get(hadoopConf) - if (!preserveFiles && fs.exists(stagingDirPath)) { - logInfo("Deleting staging directory " + stagingDirPath) - fs.delete(stagingDirPath, true) - } - } catch { - case ioe: IOException => - logWarning("Failed to cleanup staging dir " + appStagingDir, ioe) - } - } - - /** - * Set up the context for submitting our ApplicationMaster. - * This uses the YarnClientApplication not available in the Yarn alpha API. - */ - def createApplicationSubmissionContext( - newApp: YarnClientApplication, - containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { - val appContext = newApp.getApplicationSubmissionContext - appContext.setApplicationName(args.appName) - appContext.setQueue(args.amQueue) - appContext.setAMContainerSpec(containerContext) - appContext.setApplicationType("SPARK") - sparkConf.getOption(CONF_SPARK_YARN_APPLICATION_TAGS) - .map(StringUtils.getTrimmedStringCollection(_)) - .filter(!_.isEmpty()) - .foreach { tagCollection => - try { - // The setApplicationTags method was only introduced in Hadoop 2.4+, so we need to use - // reflection to set it, printing a warning if a tag was specified but the YARN version - // doesn't support it. - val method = appContext.getClass().getMethod( - "setApplicationTags", classOf[java.util.Set[String]]) - method.invoke(appContext, new java.util.HashSet[String](tagCollection)) - } catch { - case e: NoSuchMethodException => - logWarning(s"Ignoring $CONF_SPARK_YARN_APPLICATION_TAGS because this version of " + - "YARN does not support it") - } - } - sparkConf.getOption("spark.yarn.maxAppAttempts").map(_.toInt) match { - case Some(v) => appContext.setMaxAppAttempts(v) - case None => logDebug("spark.yarn.maxAppAttempts is not set. " + - "Cluster's default value will be used.") - } - - if (sparkConf.contains("spark.yarn.am.attemptFailuresValidityInterval")) { - try { - val interval = sparkConf.getTimeAsMs("spark.yarn.am.attemptFailuresValidityInterval") - val method = appContext.getClass().getMethod( - "setAttemptFailuresValidityInterval", classOf[Long]) - method.invoke(appContext, interval: java.lang.Long) - } catch { - case e: NoSuchMethodException => - logWarning("Ignoring spark.yarn.am.attemptFailuresValidityInterval because the version " + - "of YARN does not support it") - } - } - - val capability = Records.newRecord(classOf[Resource]) - capability.setMemory(args.amMemory + amMemoryOverhead) - capability.setVirtualCores(args.amCores) - appContext.setResource(capability) - appContext - } - - /** Set up security tokens for launching our ApplicationMaster container. */ - private def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { - val dob = new DataOutputBuffer - credentials.writeTokenStorageToStream(dob) - amContainer.setTokens(ByteBuffer.wrap(dob.getData)) - } - - /** Get the application report from the ResourceManager for an application we have submitted. */ - def getApplicationReport(appId: ApplicationId): ApplicationReport = - yarnClient.getApplicationReport(appId) - - /** - * Return the security token used by this client to communicate with the ApplicationMaster. - * If no security is enabled, the token returned by the report is null. - */ - private def getClientToken(report: ApplicationReport): String = - Option(report.getClientToAMToken).map(_.toString).getOrElse("") - - /** - * Fail fast if we have requested more resources per container than is available in the cluster. - */ - private def verifyClusterResources(newAppResponse: GetNewApplicationResponse): Unit = { - val maxMem = newAppResponse.getMaximumResourceCapability().getMemory() - logInfo("Verifying our application has not requested more than the maximum " + - s"memory capability of the cluster ($maxMem MB per container)") - val executorMem = args.executorMemory + executorMemoryOverhead - if (executorMem > maxMem) { - throw new IllegalArgumentException(s"Required executor memory (${args.executorMemory}" + - s"+$executorMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster! " + - "Please increase the value of 'yarn.scheduler.maximum-allocation-mb'.") - } - val amMem = args.amMemory + amMemoryOverhead - if (amMem > maxMem) { - throw new IllegalArgumentException(s"Required AM memory (${args.amMemory}" + - s"+$amMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster! " + - "Please increase the value of 'yarn.scheduler.maximum-allocation-mb'.") - } - logInfo("Will allocate AM container, with %d MB memory including %d MB overhead".format( - amMem, - amMemoryOverhead)) - - // We could add checks to make sure the entire cluster has enough resources but that involves - // getting all the node reports and computing ourselves. - } - - /** - * Copy the given file to a remote file system (e.g. HDFS) if needed. - * The file is only copied if the source and destination file systems are different. This is used - * for preparing resources for launching the ApplicationMaster container. Exposed for testing. - */ - private[yarn] def copyFileToRemote( - destDir: Path, - srcPath: Path, - replication: Short): Path = { - val destFs = destDir.getFileSystem(hadoopConf) - val srcFs = srcPath.getFileSystem(hadoopConf) - var destPath = srcPath - if (!compareFs(srcFs, destFs)) { - destPath = new Path(destDir, srcPath.getName()) - logInfo(s"Uploading resource $srcPath -> $destPath") - FileUtil.copy(srcFs, srcPath, destFs, destPath, false, hadoopConf) - destFs.setReplication(destPath, replication) - destFs.setPermission(destPath, new FsPermission(APP_FILE_PERMISSION)) - } else { - logInfo(s"Source and destination file systems are the same. Not copying $srcPath") - } - // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific - // version shows the specific version in the distributed cache configuration - val qualifiedDestPath = destFs.makeQualified(destPath) - val fc = FileContext.getFileContext(qualifiedDestPath.toUri(), hadoopConf) - fc.resolvePath(qualifiedDestPath) - } - - /** - * Upload any resources to the distributed cache if needed. If a resource is intended to be - * consumed locally, set up the appropriate config for downstream code to handle it properly. - * This is used for setting up a container launch context for our ApplicationMaster. - * Exposed for testing. - */ - def prepareLocalResources( - appStagingDir: String, - pySparkArchives: Seq[String]): HashMap[String, LocalResource] = { - logInfo("Preparing resources for our AM container") - // Upload Spark and the application JAR to the remote file system if necessary, - // and add them as local resources to the application master. - val fs = FileSystem.get(hadoopConf) - val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + dst - YarnSparkHadoopUtil.get.obtainTokensForNamenodes(nns, hadoopConf, credentials) - // Used to keep track of URIs added to the distributed cache. If the same URI is added - // multiple times, YARN will fail to launch containers for the app with an internal - // error. - val distributedUris = new HashSet[String] - obtainTokenForHiveMetastore(sparkConf, hadoopConf, credentials) - obtainTokenForHBase(sparkConf, hadoopConf, credentials) - - val replication = sparkConf.getInt("spark.yarn.submit.file.replication", - fs.getDefaultReplication(dst)).toShort - val localResources = HashMap[String, LocalResource]() - FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION)) - - val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - - val oldLog4jConf = Option(System.getenv("SPARK_LOG4J_CONF")) - if (oldLog4jConf.isDefined) { - logWarning( - "SPARK_LOG4J_CONF detected in the system environment. This variable has been " + - "deprecated. Please refer to the \"Launching Spark on YARN\" documentation " + - "for alternatives.") - } - - def addDistributedUri(uri: URI): Boolean = { - val uriStr = uri.toString() - if (distributedUris.contains(uriStr)) { - logWarning(s"Resource $uri added multiple times to distributed cache.") - false - } else { - distributedUris += uriStr - true - } - } - - /** - * Distribute a file to the cluster. - * - * If the file's path is a "local:" URI, it's actually not distributed. Other files are copied - * to HDFS (if not already there) and added to the application's distributed cache. - * - * @param path URI of the file to distribute. - * @param resType Type of resource being distributed. - * @param destName Name of the file in the distributed cache. - * @param targetDir Subdirectory where to place the file. - * @param appMasterOnly Whether to distribute only to the AM. - * @return A 2-tuple. First item is whether the file is a "local:" URI. Second item is the - * localized path for non-local paths, or the input `path` for local paths. - * The localized path will be null if the URI has already been added to the cache. - */ - def distribute( - path: String, - resType: LocalResourceType = LocalResourceType.FILE, - destName: Option[String] = None, - targetDir: Option[String] = None, - appMasterOnly: Boolean = false): (Boolean, String) = { - val trimmedPath = path.trim() - val localURI = Utils.resolveURI(trimmedPath) - if (localURI.getScheme != LOCAL_SCHEME) { - if (addDistributedUri(localURI)) { - val localPath = getQualifiedLocalPath(localURI, hadoopConf) - val linkname = targetDir.map(_ + "/").getOrElse("") + - destName.orElse(Option(localURI.getFragment())).getOrElse(localPath.getName()) - val destPath = copyFileToRemote(dst, localPath, replication) - val destFs = FileSystem.get(destPath.toUri(), hadoopConf) - distCacheMgr.addResource( - destFs, hadoopConf, destPath, localResources, resType, linkname, statCache, - appMasterOnly = appMasterOnly) - (false, linkname) - } else { - (false, null) - } - } else { - (true, trimmedPath) - } - } - - // If we passed in a keytab, make sure we copy the keytab to the staging directory on - // HDFS, and setup the relevant environment vars, so the AM can login again. - if (loginFromKeytab) { - logInfo("To enable the AM to login from keytab, credentials are being copied over to the AM" + - " via the YARN Secure Distributed Cache.") - val (_, localizedPath) = distribute(keytab, - destName = Some(sparkConf.get("spark.yarn.keytab")), - appMasterOnly = true) - require(localizedPath != null, "Keytab file already distributed.") - } - - /** - * Copy the given main resource to the distributed cache if the scheme is not "local". - * Otherwise, set the corresponding key in our SparkConf to handle it downstream. - * Each resource is represented by a 3-tuple of: - * (1) destination resource name, - * (2) local path to the resource, - * (3) Spark property key to set if the scheme is not local - */ - List( - (SPARK_JAR, sparkJar(sparkConf), CONF_SPARK_JAR), - (APP_JAR, args.userJar, CONF_SPARK_USER_JAR), - ("log4j.properties", oldLog4jConf.orNull, null) - ).foreach { case (destName, path, confKey) => - if (path != null && !path.trim().isEmpty()) { - val (isLocal, localizedPath) = distribute(path, destName = Some(destName)) - if (isLocal && confKey != null) { - require(localizedPath != null, s"Path $path already distributed.") - // If the resource is intended for local use only, handle this downstream - // by setting the appropriate property - sparkConf.set(confKey, localizedPath) - } - } - } - - /** - * Do the same for any additional resources passed in through ClientArguments. - * Each resource category is represented by a 3-tuple of: - * (1) comma separated list of resources in this category, - * (2) resource type, and - * (3) whether to add these resources to the classpath - */ - val cachedSecondaryJarLinks = ListBuffer.empty[String] - List( - (args.addJars, LocalResourceType.FILE, true), - (args.files, LocalResourceType.FILE, false), - (args.archives, LocalResourceType.ARCHIVE, false) - ).foreach { case (flist, resType, addToClasspath) => - if (flist != null && !flist.isEmpty()) { - flist.split(',').foreach { file => - val (_, localizedPath) = distribute(file, resType = resType) - require(localizedPath != null) - if (addToClasspath) { - cachedSecondaryJarLinks += localizedPath - } - } - } - } - if (cachedSecondaryJarLinks.nonEmpty) { - sparkConf.set(CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) - } - - if (isClusterMode && args.primaryPyFile != null) { - distribute(args.primaryPyFile, appMasterOnly = true) - } - - pySparkArchives.foreach { f => distribute(f) } - - // The python files list needs to be treated especially. All files that are not an - // archive need to be placed in a subdirectory that will be added to PYTHONPATH. - args.pyFiles.foreach { f => - val targetDir = if (f.endsWith(".py")) Some(LOCALIZED_PYTHON_DIR) else None - distribute(f, targetDir = targetDir) - } - - // Distribute an archive with Hadoop and Spark configuration for the AM. - val (_, confLocalizedPath) = distribute(createConfArchive().toURI().getPath(), - resType = LocalResourceType.ARCHIVE, - destName = Some(LOCALIZED_CONF_DIR), - appMasterOnly = true) - require(confLocalizedPath != null) - - localResources - } - - /** - * Create an archive with the config files for distribution. - * - * These are only used by the AM, since executors will use the configuration object broadcast by - * the driver. The files are zipped and added to the job as an archive, so that YARN will explode - * it when distributing to the AM. This directory is then added to the classpath of the AM - * process, just to make sure that everybody is using the same default config. - * - * This follows the order of precedence set by the startup scripts, in which HADOOP_CONF_DIR - * shows up in the classpath before YARN_CONF_DIR. - * - * Currently this makes a shallow copy of the conf directory. If there are cases where a - * Hadoop config directory contains subdirectories, this code will have to be fixed. - * - * The archive also contains some Spark configuration. Namely, it saves the contents of - * SparkConf in a file to be loaded by the AM process. - */ - private def createConfArchive(): File = { - val hadoopConfFiles = new HashMap[String, File]() - - // Uploading $SPARK_CONF_DIR/log4j.properties file to the distributed cache to make sure that - // the executors will use the latest configurations instead of the default values. This is - // required when user changes log4j.properties directly to set the log configurations. If - // configuration file is provided through --files then executors will be taking configurations - // from --files instead of $SPARK_CONF_DIR/log4j.properties. - val log4jFileName = "log4j.properties" - Option(Utils.getContextOrSparkClassLoader.getResource(log4jFileName)).foreach { url => - if (url.getProtocol == "file") { - hadoopConfFiles(log4jFileName) = new File(url.getPath) - } - } - - Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR").foreach { envKey => - sys.env.get(envKey).foreach { path => - val dir = new File(path) - if (dir.isDirectory()) { - dir.listFiles().foreach { file => - if (file.isFile && !hadoopConfFiles.contains(file.getName())) { - hadoopConfFiles(file.getName()) = file - } - } - } - } - } - - val confArchive = File.createTempFile(LOCALIZED_CONF_DIR, ".zip", - new File(Utils.getLocalDir(sparkConf))) - val confStream = new ZipOutputStream(new FileOutputStream(confArchive)) - - try { - confStream.setLevel(0) - hadoopConfFiles.foreach { case (name, file) => - if (file.canRead()) { - confStream.putNextEntry(new ZipEntry(name)) - Files.copy(file, confStream) - confStream.closeEntry() - } - } - - // Save Spark configuration to a file in the archive. - val props = new Properties() - sparkConf.getAll.foreach { case (k, v) => props.setProperty(k, v) } - confStream.putNextEntry(new ZipEntry(SPARK_CONF_FILE)) - val writer = new OutputStreamWriter(confStream, UTF_8) - props.store(writer, "Spark configuration.") - writer.flush() - confStream.closeEntry() - } finally { - confStream.close() - } - confArchive - } - - /** - * Get the renewal interval for tokens. - */ - private def getTokenRenewalInterval(stagingDirPath: Path): Long = { - // We cannot use the tokens generated above since those have renewer yarn. Trying to renew - // those will fail with an access control issue. So create new tokens with the logged in - // user as renewer. - val creds = new Credentials() - val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + stagingDirPath - YarnSparkHadoopUtil.get.obtainTokensForNamenodes( - nns, hadoopConf, creds, Some(sparkConf.get("spark.yarn.principal"))) - val t = creds.getAllTokens.asScala - .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) - .head - val newExpiration = t.renew(hadoopConf) - val identifier = new DelegationTokenIdentifier() - identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier))) - val interval = newExpiration - identifier.getIssueDate - logInfo(s"Renewal Interval set to $interval") - interval - } - - /** - * Set up the environment for launching our ApplicationMaster container. - */ - private def setupLaunchEnv( - stagingDir: String, - pySparkArchives: Seq[String]): HashMap[String, String] = { - logInfo("Setting up the launch environment for our AM container") - val env = new HashMap[String, String]() - val extraCp = sparkConf.getOption("spark.driver.extraClassPath") - populateClasspath(args, yarnConf, sparkConf, env, true, extraCp) - env("SPARK_YARN_MODE") = "true" - env("SPARK_YARN_STAGING_DIR") = stagingDir - env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() - if (loginFromKeytab) { - val remoteFs = FileSystem.get(hadoopConf) - val stagingDirPath = new Path(remoteFs.getHomeDirectory, stagingDir) - val credentialsFile = "credentials-" + UUID.randomUUID().toString - sparkConf.set( - "spark.yarn.credentials.file", new Path(stagingDirPath, credentialsFile).toString) - logInfo(s"Credentials file set to: $credentialsFile") - val renewalInterval = getTokenRenewalInterval(stagingDirPath) - sparkConf.set("spark.yarn.token.renewal.interval", renewalInterval.toString) - } - - // Pick up any environment variables for the AM provided through spark.yarn.appMasterEnv.* - val amEnvPrefix = "spark.yarn.appMasterEnv." - sparkConf.getAll - .filter { case (k, v) => k.startsWith(amEnvPrefix) } - .map { case (k, v) => (k.substring(amEnvPrefix.length), v) } - .foreach { case (k, v) => YarnSparkHadoopUtil.addPathToEnvironment(env, k, v) } - - // Keep this for backwards compatibility but users should move to the config - sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => - // Allow users to specify some environment variables. - YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) - // Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments. - env("SPARK_YARN_USER_ENV") = userEnvs - } - - // If pyFiles contains any .py files, we need to add LOCALIZED_PYTHON_DIR to the PYTHONPATH - // of the container processes too. Add all non-.py files directly to PYTHONPATH. - // - // NOTE: the code currently does not handle .py files defined with a "local:" scheme. - val pythonPath = new ListBuffer[String]() - val (pyFiles, pyArchives) = args.pyFiles.partition(_.endsWith(".py")) - if (pyFiles.nonEmpty) { - pythonPath += buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), - LOCALIZED_PYTHON_DIR) - } - (pySparkArchives ++ pyArchives).foreach { path => - val uri = Utils.resolveURI(path) - if (uri.getScheme != LOCAL_SCHEME) { - pythonPath += buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), - new Path(uri).getName()) - } else { - pythonPath += uri.getPath() - } - } - - // Finally, update the Spark config to propagate PYTHONPATH to the AM and executors. - if (pythonPath.nonEmpty) { - val pythonPathStr = (sys.env.get("PYTHONPATH") ++ pythonPath) - .mkString(YarnSparkHadoopUtil.getClassPathSeparator) - env("PYTHONPATH") = pythonPathStr - sparkConf.setExecutorEnv("PYTHONPATH", pythonPathStr) - } - - // In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to - // executors. But we can't just set spark.executor.extraJavaOptions, because the driver's - // SparkContext will not let that set spark* system properties, which is expected behavior for - // Yarn clients. So propagate it through the environment. - // - // Note that to warn the user about the deprecation in cluster mode, some code from - // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition - // described above). - if (isClusterMode) { - sys.env.get("SPARK_JAVA_OPTS").foreach { value => - val warning = - s""" - |SPARK_JAVA_OPTS was detected (set to '$value'). - |This is deprecated in Spark 1.0+. - | - |Please instead use: - | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application - | - ./spark-submit with --driver-java-options to set -X options for a driver - | - spark.executor.extraJavaOptions to set -X options for executors - """.stripMargin - logWarning(warning) - for (proc <- Seq("driver", "executor")) { - val key = s"spark.$proc.extraJavaOptions" - if (sparkConf.contains(key)) { - throw new SparkException(s"Found both $key and SPARK_JAVA_OPTS. Use only the former.") - } - } - env("SPARK_JAVA_OPTS") = value - } - } - - sys.env.get(ENV_DIST_CLASSPATH).foreach { dcp => - env(ENV_DIST_CLASSPATH) = dcp - } - - env - } - - /** - * Set up a ContainerLaunchContext to launch our ApplicationMaster container. - * This sets up the launch environment, java options, and the command for launching the AM. - */ - private def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) - : ContainerLaunchContext = { - logInfo("Setting up container launch context for our AM") - val appId = newAppResponse.getApplicationId - val appStagingDir = getAppStagingDir(appId) - val pySparkArchives = - if (sparkConf.getBoolean("spark.yarn.isPython", false)) { - findPySparkArchives() - } else { - Nil - } - val launchEnv = setupLaunchEnv(appStagingDir, pySparkArchives) - val localResources = prepareLocalResources(appStagingDir, pySparkArchives) - - // Set the environment variables to be passed on to the executors. - distCacheMgr.setDistFilesEnv(launchEnv) - distCacheMgr.setDistArchivesEnv(launchEnv) - - val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) - amContainer.setLocalResources(localResources.asJava) - amContainer.setEnvironment(launchEnv.asJava) - - val javaOpts = ListBuffer[String]() - - // Set the environment variable through a command prefix - // to append to the existing value of the variable - var prefixEnv: Option[String] = None - - // Add Xmx for AM memory - javaOpts += "-Xmx" + args.amMemory + "m" - - val tmpDir = new Path( - YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), - YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR - ) - javaOpts += "-Djava.io.tmpdir=" + tmpDir - - // TODO: Remove once cpuset version is pushed out. - // The context is, default gc for server class machines ends up using all cores to do gc - - // hence if there are multiple containers in same node, Spark GC affects all other containers' - // performance (which can be that of other Spark containers) - // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in - // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset - // of cores on a node. - val useConcurrentAndIncrementalGC = launchEnv.get("SPARK_USE_CONC_INCR_GC").exists(_.toBoolean) - if (useConcurrentAndIncrementalGC) { - // In our expts, using (default) throughput collector has severe perf ramifications in - // multi-tenant machines - javaOpts += "-XX:+UseConcMarkSweepGC" - javaOpts += "-XX:MaxTenuringThreshold=31" - javaOpts += "-XX:SurvivorRatio=8" - javaOpts += "-XX:+CMSIncrementalMode" - javaOpts += "-XX:+CMSIncrementalPacing" - javaOpts += "-XX:CMSIncrementalDutyCycleMin=0" - javaOpts += "-XX:CMSIncrementalDutyCycle=10" - } - - // Include driver-specific java options if we are launching a driver - if (isClusterMode) { - val driverOpts = sparkConf.getOption("spark.driver.extraJavaOptions") - .orElse(sys.env.get("SPARK_JAVA_OPTS")) - driverOpts.foreach { opts => - javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell) - } - val libraryPaths = Seq(sys.props.get("spark.driver.extraLibraryPath"), - sys.props.get("spark.driver.libraryPath")).flatten - if (libraryPaths.nonEmpty) { - prefixEnv = Some(getClusterPath(sparkConf, Utils.libraryPathEnvPrefix(libraryPaths))) - } - if (sparkConf.getOption("spark.yarn.am.extraJavaOptions").isDefined) { - logWarning("spark.yarn.am.extraJavaOptions will not take effect in cluster mode") - } - } else { - // Validate and include yarn am specific java options in yarn-client mode. - val amOptsKey = "spark.yarn.am.extraJavaOptions" - val amOpts = sparkConf.getOption(amOptsKey) - amOpts.foreach { opts => - if (opts.contains("-Dspark")) { - val msg = s"$amOptsKey is not allowed to set Spark options (was '$opts'). " - throw new SparkException(msg) - } - if (opts.contains("-Xmx") || opts.contains("-Xms")) { - val msg = s"$amOptsKey is not allowed to alter memory settings (was '$opts')." - throw new SparkException(msg) - } - javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell) - } - - sparkConf.getOption("spark.yarn.am.extraLibraryPath").foreach { paths => - prefixEnv = Some(getClusterPath(sparkConf, Utils.libraryPathEnvPrefix(Seq(paths)))) - } - } - - // For log4j configuration to reference - javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) - YarnCommandBuilderUtils.addPermGenSizeOpt(javaOpts) - - val userClass = - if (isClusterMode) { - Seq("--class", YarnSparkHadoopUtil.escapeForShell(args.userClass)) - } else { - Nil - } - val userJar = - if (args.userJar != null) { - Seq("--jar", args.userJar) - } else { - Nil - } - val primaryPyFile = - if (isClusterMode && args.primaryPyFile != null) { - Seq("--primary-py-file", new Path(args.primaryPyFile).getName()) - } else { - Nil - } - val primaryRFile = - if (args.primaryRFile != null) { - Seq("--primary-r-file", args.primaryRFile) - } else { - Nil - } - val amClass = - if (isClusterMode) { - Utils.classForName("org.apache.spark.deploy.yarn.ApplicationMaster").getName - } else { - Utils.classForName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName - } - if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) { - args.userArgs = ArrayBuffer(args.primaryRFile) ++ args.userArgs - } - val userArgs = args.userArgs.flatMap { arg => - Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg)) - } - val amArgs = - Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ primaryRFile ++ - userArgs ++ Seq( - "--executor-memory", args.executorMemory.toString + "m", - "--executor-cores", args.executorCores.toString, - "--properties-file", buildPath(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), - LOCALIZED_CONF_DIR, SPARK_CONF_FILE)) - - // Command for the ApplicationMaster - val commands = prefixEnv ++ Seq( - YarnSparkHadoopUtil.expandEnvironment(Environment.JAVA_HOME) + "/bin/java", "-server" - ) ++ - javaOpts ++ amArgs ++ - Seq( - "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", - "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") - - // TODO: it would be nicer to just make sure there are no null commands here - val printableCommands = commands.map(s => if (s == null) "null" else s).toList - amContainer.setCommands(printableCommands.asJava) - - logDebug("===============================================================================") - logDebug("YARN AM launch context:") - logDebug(s" user class: ${Option(args.userClass).getOrElse("N/A")}") - logDebug(" env:") - launchEnv.foreach { case (k, v) => logDebug(s" $k -> $v") } - logDebug(" resources:") - localResources.foreach { case (k, v) => logDebug(s" $k -> $v")} - logDebug(" command:") - logDebug(s" ${printableCommands.mkString(" ")}") - logDebug("===============================================================================") - - // send the acl settings into YARN to control who has access via YARN interfaces - val securityManager = new SecurityManager(sparkConf) - amContainer.setApplicationACLs( - YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager).asJava) - setupSecurityToken(amContainer) - UserGroupInformation.getCurrentUser().addCredentials(credentials) - - amContainer - } - - def setupCredentials(): Unit = { - loginFromKeytab = args.principal != null || sparkConf.contains("spark.yarn.principal") - if (loginFromKeytab) { - principal = - if (args.principal != null) args.principal else sparkConf.get("spark.yarn.principal") - keytab = { - if (args.keytab != null) { - args.keytab - } else { - sparkConf.getOption("spark.yarn.keytab").orNull - } - } - - require(keytab != null, "Keytab must be specified when principal is specified.") - logInfo("Attempting to login to the Kerberos" + - s" using principal: $principal and keytab: $keytab") - val f = new File(keytab) - // Generate a file name that can be used for the keytab file, that does not conflict - // with any user file. - val keytabFileName = f.getName + "-" + UUID.randomUUID().toString - sparkConf.set("spark.yarn.keytab", keytabFileName) - sparkConf.set("spark.yarn.principal", principal) - } - credentials = UserGroupInformation.getCurrentUser.getCredentials - } - - /** - * Report the state of an application until it has exited, either successfully or - * due to some failure, then return a pair of the yarn application state (FINISHED, FAILED, - * KILLED, or RUNNING) and the final application state (UNDEFINED, SUCCEEDED, FAILED, - * or KILLED). - * - * @param appId ID of the application to monitor. - * @param returnOnRunning Whether to also return the application state when it is RUNNING. - * @param logApplicationReport Whether to log details of the application report every iteration. - * @return A pair of the yarn application state and the final application state. - */ - def monitorApplication( - appId: ApplicationId, - returnOnRunning: Boolean = false, - logApplicationReport: Boolean = true): (YarnApplicationState, FinalApplicationStatus) = { - val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) - var lastState: YarnApplicationState = null - while (true) { - Thread.sleep(interval) - val report: ApplicationReport = - try { - getApplicationReport(appId) - } catch { - case e: ApplicationNotFoundException => - logError(s"Application $appId not found.") - return (YarnApplicationState.KILLED, FinalApplicationStatus.KILLED) - case NonFatal(e) => - logError(s"Failed to contact YARN for application $appId.", e) - return (YarnApplicationState.FAILED, FinalApplicationStatus.FAILED) - } - val state = report.getYarnApplicationState - - if (logApplicationReport) { - logInfo(s"Application report for $appId (state: $state)") - - // If DEBUG is enabled, log report details every iteration - // Otherwise, log them every time the application changes state - if (log.isDebugEnabled) { - logDebug(formatReportDetails(report)) - } else if (lastState != state) { - logInfo(formatReportDetails(report)) - } - } - - if (lastState != state) { - state match { - case YarnApplicationState.RUNNING => - reportLauncherState(SparkAppHandle.State.RUNNING) - case YarnApplicationState.FINISHED => - reportLauncherState(SparkAppHandle.State.FINISHED) - case YarnApplicationState.FAILED => - reportLauncherState(SparkAppHandle.State.FAILED) - case YarnApplicationState.KILLED => - reportLauncherState(SparkAppHandle.State.KILLED) - case _ => - } - } - - if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.FAILED || - state == YarnApplicationState.KILLED) { - cleanupStagingDir(appId) - return (state, report.getFinalApplicationStatus) - } - - if (returnOnRunning && state == YarnApplicationState.RUNNING) { - return (state, report.getFinalApplicationStatus) - } - - lastState = state - } - - // Never reached, but keeps compiler happy - throw new SparkException("While loop is depleted! This should never happen...") - } - - private def formatReportDetails(report: ApplicationReport): String = { - val details = Seq[(String, String)]( - ("client token", getClientToken(report)), - ("diagnostics", report.getDiagnostics), - ("ApplicationMaster host", report.getHost), - ("ApplicationMaster RPC port", report.getRpcPort.toString), - ("queue", report.getQueue), - ("start time", report.getStartTime.toString), - ("final status", report.getFinalApplicationStatus.toString), - ("tracking URL", report.getTrackingUrl), - ("user", report.getUser) - ) - - // Use more loggable format if value is null or empty - details.map { case (k, v) => - val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A") - s"\n\t $k: $newValue" - }.mkString("") - } - - /** - * Submit an application to the ResourceManager. - * If set spark.yarn.submit.waitAppCompletion to true, it will stay alive - * reporting the application's status until the application has exited for any reason. - * Otherwise, the client process will exit after submission. - * If the application finishes with a failed, killed, or undefined status, - * throw an appropriate SparkException. - */ - def run(): Unit = { - this.appId = submitApplication() - if (!launcherBackend.isConnected() && fireAndForget) { - val report = getApplicationReport(appId) - val state = report.getYarnApplicationState - logInfo(s"Application report for $appId (state: $state)") - logInfo(formatReportDetails(report)) - if (state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) { - throw new SparkException(s"Application $appId finished with status: $state") - } - } else { - val (yarnApplicationState, finalApplicationStatus) = monitorApplication(appId) - if (yarnApplicationState == YarnApplicationState.FAILED || - finalApplicationStatus == FinalApplicationStatus.FAILED) { - throw new SparkException(s"Application $appId finished with failed status") - } - if (yarnApplicationState == YarnApplicationState.KILLED || - finalApplicationStatus == FinalApplicationStatus.KILLED) { - throw new SparkException(s"Application $appId is killed") - } - if (finalApplicationStatus == FinalApplicationStatus.UNDEFINED) { - throw new SparkException(s"The final status of application $appId is undefined") - } - } - } - - private def findPySparkArchives(): Seq[String] = { - sys.env.get("PYSPARK_ARCHIVES_PATH") - .map(_.split(",").toSeq) - .getOrElse { - val pyLibPath = Seq(sys.env("SPARK_HOME"), "python", "lib").mkString(File.separator) - val pyArchivesFile = new File(pyLibPath, "pyspark.zip") - require(pyArchivesFile.exists(), - "pyspark.zip not found; cannot run pyspark application in YARN mode.") - val py4jFile = new File(pyLibPath, "py4j-0.9-src.zip") - require(py4jFile.exists(), - "py4j-0.9-src.zip not found; cannot run pyspark application in YARN mode.") - Seq(pyArchivesFile.getAbsolutePath(), py4jFile.getAbsolutePath()) - } - } - -} - -object Client extends Logging { - - def main(argStrings: Array[String]) { - if (!sys.props.contains("SPARK_SUBMIT")) { - logWarning("WARNING: This client is deprecated and will be removed in a " + - "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") - } - - // Set an env variable indicating we are running in YARN mode. - // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes - System.setProperty("SPARK_YARN_MODE", "true") - val sparkConf = new SparkConf - - val args = new ClientArguments(argStrings, sparkConf) - // to maintain backwards-compatibility - if (!Utils.isDynamicAllocationEnabled(sparkConf)) { - sparkConf.setIfMissing("spark.executor.instances", args.numExecutors.toString) - } - new Client(args, sparkConf).run() - } - - // Alias for the Spark assembly jar and the user jar - val SPARK_JAR: String = "__spark__.jar" - val APP_JAR: String = "__app__.jar" - - // URI scheme that identifies local resources - val LOCAL_SCHEME = "local" - - // Staging directory for any temporary jars or files - val SPARK_STAGING: String = ".sparkStaging" - - // Location of any user-defined Spark jars - val CONF_SPARK_JAR = "spark.yarn.jar" - val ENV_SPARK_JAR = "SPARK_JAR" - - // Internal config to propagate the location of the user's jar to the driver/executors - val CONF_SPARK_USER_JAR = "spark.yarn.user.jar" - - // Internal config to propagate the locations of any extra jars to add to the classpath - // of the executors - val CONF_SPARK_YARN_SECONDARY_JARS = "spark.yarn.secondary.jars" - - // Comma-separated list of strings to pass through as YARN application tags appearing - // in YARN ApplicationReports, which can be used for filtering when querying YARN. - val CONF_SPARK_YARN_APPLICATION_TAGS = "spark.yarn.tags" - - // Staging directory is private! -> rwx-------- - val STAGING_DIR_PERMISSION: FsPermission = - FsPermission.createImmutable(Integer.parseInt("700", 8).toShort) - - // App files are world-wide readable and owner writable -> rw-r--r-- - val APP_FILE_PERMISSION: FsPermission = - FsPermission.createImmutable(Integer.parseInt("644", 8).toShort) - - // Distribution-defined classpath to add to processes - val ENV_DIST_CLASSPATH = "SPARK_DIST_CLASSPATH" - - // Subdirectory where the user's Spark and Hadoop config files will be placed. - val LOCALIZED_CONF_DIR = "__spark_conf__" - - // Name of the file in the conf archive containing Spark configuration. - val SPARK_CONF_FILE = "__spark_conf__.properties" - - // Subdirectory where the user's python files (not archives) will be placed. - val LOCALIZED_PYTHON_DIR = "__pyfiles__" - - /** - * Find the user-defined Spark jar if configured, or return the jar containing this - * class if not. - * - * This method first looks in the SparkConf object for the CONF_SPARK_JAR key, and in the - * user environment if that is not found (for backwards compatibility). - */ - private def sparkJar(conf: SparkConf): String = { - if (conf.contains(CONF_SPARK_JAR)) { - conf.get(CONF_SPARK_JAR) - } else if (System.getenv(ENV_SPARK_JAR) != null) { - logWarning( - s"$ENV_SPARK_JAR detected in the system environment. This variable has been deprecated " + - s"in favor of the $CONF_SPARK_JAR configuration variable.") - System.getenv(ENV_SPARK_JAR) - } else { - SparkContext.jarOfClass(this.getClass).getOrElse(throw new SparkException("Could not " - + "find jar containing Spark classes. The jar can be defined using the " - + "spark.yarn.jar configuration option. If testing Spark, either set that option or " - + "make sure SPARK_PREPEND_CLASSES is not set.")) - } - } - - /** - * Return the path to the given application's staging directory. - */ - private def getAppStagingDir(appId: ApplicationId): String = { - buildPath(SPARK_STAGING, appId.toString()) - } - - /** - * Populate the classpath entry in the given environment map with any application - * classpath specified through the Hadoop and Yarn configurations. - */ - private[yarn] def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) - : Unit = { - val classPathElementsToAdd = getYarnAppClasspath(conf) ++ getMRAppClasspath(conf) - for (c <- classPathElementsToAdd.flatten) { - YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, c.trim) - } - } - - private def getYarnAppClasspath(conf: Configuration): Option[Seq[String]] = - Option(conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) match { - case Some(s) => Some(s.toSeq) - case None => getDefaultYarnApplicationClasspath - } - - private def getMRAppClasspath(conf: Configuration): Option[Seq[String]] = - Option(conf.getStrings("mapreduce.application.classpath")) match { - case Some(s) => Some(s.toSeq) - case None => getDefaultMRApplicationClasspath - } - - private[yarn] def getDefaultYarnApplicationClasspath: Option[Seq[String]] = { - val triedDefault = Try[Seq[String]] { - val field = classOf[YarnConfiguration].getField("DEFAULT_YARN_APPLICATION_CLASSPATH") - val value = field.get(null).asInstanceOf[Array[String]] - value.toSeq - } recoverWith { - case e: NoSuchFieldException => Success(Seq.empty[String]) - } - - triedDefault match { - case f: Failure[_] => - logError("Unable to obtain the default YARN Application classpath.", f.exception) - case s: Success[Seq[String]] => - logDebug(s"Using the default YARN application classpath: ${s.get.mkString(",")}") - } - - triedDefault.toOption - } - - private[yarn] def getDefaultMRApplicationClasspath: Option[Seq[String]] = { - val triedDefault = Try[Seq[String]] { - val field = classOf[MRJobConfig].getField("DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH") - StringUtils.getStrings(field.get(null).asInstanceOf[String]).toSeq - } recoverWith { - case e: NoSuchFieldException => Success(Seq.empty[String]) - } - - triedDefault match { - case f: Failure[_] => - logError("Unable to obtain the default MR Application classpath.", f.exception) - case s: Success[Seq[String]] => - logDebug(s"Using the default MR application classpath: ${s.get.mkString(",")}") - } - - triedDefault.toOption - } - - /** - * Populate the classpath entry in the given environment map. - * - * User jars are generally not added to the JVM's system classpath; those are handled by the AM - * and executor backend. When the deprecated `spark.yarn.user.classpath.first` is used, user jars - * are included in the system classpath, though. The extra class path and other uploaded files are - * always made available through the system class path. - * - * @param args Client arguments (when starting the AM) or null (when starting executors). - */ - private[yarn] def populateClasspath( - args: ClientArguments, - conf: Configuration, - sparkConf: SparkConf, - env: HashMap[String, String], - isAM: Boolean, - extraClassPath: Option[String] = None): Unit = { - extraClassPath.foreach { cp => - addClasspathEntry(getClusterPath(sparkConf, cp), env) - } - addClasspathEntry(YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), env) - - if (isAM) { - addClasspathEntry( - YarnSparkHadoopUtil.expandEnvironment(Environment.PWD) + Path.SEPARATOR + - LOCALIZED_CONF_DIR, env) - } - - if (sparkConf.getBoolean("spark.yarn.user.classpath.first", false)) { - // in order to properly add the app jar when user classpath is first - // we have to do the mainJar separate in order to send the right thing - // into addFileToClasspath - val mainJar = - if (args != null) { - getMainJarUri(Option(args.userJar)) - } else { - getMainJarUri(sparkConf.getOption(CONF_SPARK_USER_JAR)) - } - mainJar.foreach(addFileToClasspath(sparkConf, conf, _, APP_JAR, env)) - - val secondaryJars = - if (args != null) { - getSecondaryJarUris(Option(args.addJars)) - } else { - getSecondaryJarUris(sparkConf.getOption(CONF_SPARK_YARN_SECONDARY_JARS)) - } - secondaryJars.foreach { x => - addFileToClasspath(sparkConf, conf, x, null, env) - } - } - addFileToClasspath(sparkConf, conf, new URI(sparkJar(sparkConf)), SPARK_JAR, env) - populateHadoopClasspath(conf, env) - sys.env.get(ENV_DIST_CLASSPATH).foreach { cp => - addClasspathEntry(getClusterPath(sparkConf, cp), env) - } - } - - /** - * Returns a list of URIs representing the user classpath. - * - * @param conf Spark configuration. - */ - def getUserClasspath(conf: SparkConf): Array[URI] = { - val mainUri = getMainJarUri(conf.getOption(CONF_SPARK_USER_JAR)) - val secondaryUris = getSecondaryJarUris(conf.getOption(CONF_SPARK_YARN_SECONDARY_JARS)) - (mainUri ++ secondaryUris).toArray - } - - private def getMainJarUri(mainJar: Option[String]): Option[URI] = { - mainJar.flatMap { path => - val uri = Utils.resolveURI(path) - if (uri.getScheme == LOCAL_SCHEME) Some(uri) else None - }.orElse(Some(new URI(APP_JAR))) - } - - private def getSecondaryJarUris(secondaryJars: Option[String]): Seq[URI] = { - secondaryJars.map(_.split(",")).toSeq.flatten.map(new URI(_)) - } - - /** - * Adds the given path to the classpath, handling "local:" URIs correctly. - * - * If an alternate name for the file is given, and it's not a "local:" file, the alternate - * name will be added to the classpath (relative to the job's work directory). - * - * If not a "local:" file and no alternate name, the linkName will be added to the classpath. - * - * @param conf Spark configuration. - * @param hadoopConf Hadoop configuration. - * @param uri URI to add to classpath (optional). - * @param fileName Alternate name for the file (optional). - * @param env Map holding the environment variables. - */ - private def addFileToClasspath( - conf: SparkConf, - hadoopConf: Configuration, - uri: URI, - fileName: String, - env: HashMap[String, String]): Unit = { - if (uri != null && uri.getScheme == LOCAL_SCHEME) { - addClasspathEntry(getClusterPath(conf, uri.getPath), env) - } else if (fileName != null) { - addClasspathEntry(buildPath( - YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), fileName), env) - } else if (uri != null) { - val localPath = getQualifiedLocalPath(uri, hadoopConf) - val linkName = Option(uri.getFragment()).getOrElse(localPath.getName()) - addClasspathEntry(buildPath( - YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), linkName), env) - } - } - - /** - * Add the given path to the classpath entry of the given environment map. - * If the classpath is already set, this appends the new path to the existing classpath. - */ - private def addClasspathEntry(path: String, env: HashMap[String, String]): Unit = - YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, path) - - /** - * Returns the path to be sent to the NM for a path that is valid on the gateway. - * - * This method uses two configuration values: - * - * - spark.yarn.config.gatewayPath: a string that identifies a portion of the input path that may - * only be valid in the gateway node. - * - spark.yarn.config.replacementPath: a string with which to replace the gateway path. This may - * contain, for example, env variable references, which will be expanded by the NMs when - * starting containers. - * - * If either config is not available, the input path is returned. - */ - def getClusterPath(conf: SparkConf, path: String): String = { - val localPath = conf.get("spark.yarn.config.gatewayPath", null) - val clusterPath = conf.get("spark.yarn.config.replacementPath", null) - if (localPath != null && clusterPath != null) { - path.replace(localPath, clusterPath) - } else { - path - } - } - - /** - * Obtains token for the Hive metastore and adds them to the credentials. - */ - private def obtainTokenForHiveMetastore( - sparkConf: SparkConf, - conf: Configuration, - credentials: Credentials) { - if (shouldGetTokens(sparkConf, "hive") && UserGroupInformation.isSecurityEnabled) { - YarnSparkHadoopUtil.get.obtainTokenForHiveMetastore(conf).foreach { - credentials.addToken(new Text("hive.server2.delegation.token"), _) - } - } - } - - /** - * Obtain security token for HBase. - */ - def obtainTokenForHBase( - sparkConf: SparkConf, - conf: Configuration, - credentials: Credentials): Unit = { - if (shouldGetTokens(sparkConf, "hbase") && UserGroupInformation.isSecurityEnabled) { - val mirror = universe.runtimeMirror(getClass.getClassLoader) - - try { - val confCreate = mirror.classLoader. - loadClass("org.apache.hadoop.hbase.HBaseConfiguration"). - getMethod("create", classOf[Configuration]) - val obtainToken = mirror.classLoader. - loadClass("org.apache.hadoop.hbase.security.token.TokenUtil"). - getMethod("obtainToken", classOf[Configuration]) - - logDebug("Attempting to fetch HBase security token.") - - val hbaseConf = confCreate.invoke(null, conf).asInstanceOf[Configuration] - if ("kerberos" == hbaseConf.get("hbase.security.authentication")) { - val token = obtainToken.invoke(null, hbaseConf).asInstanceOf[Token[TokenIdentifier]] - credentials.addToken(token.getService, token) - logInfo("Added HBase security token to credentials.") - } - } catch { - case e: java.lang.NoSuchMethodException => - logInfo("HBase Method not found: " + e) - case e: java.lang.ClassNotFoundException => - logDebug("HBase Class not found: " + e) - case e: java.lang.NoClassDefFoundError => - logDebug("HBase Class not found: " + e) - case e: Exception => - logError("Exception when obtaining HBase security token: " + e) - } - } - } - - /** - * Return whether the two file systems are the same. - */ - private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = { - val srcUri = srcFs.getUri() - val dstUri = destFs.getUri() - if (srcUri.getScheme() == null || srcUri.getScheme() != dstUri.getScheme()) { - return false - } - - var srcHost = srcUri.getHost() - var dstHost = dstUri.getHost() - - // In HA or when using viewfs, the host part of the URI may not actually be a host, but the - // name of the HDFS namespace. Those names won't resolve, so avoid even trying if they - // match. - if (srcHost != null && dstHost != null && srcHost != dstHost) { - try { - srcHost = InetAddress.getByName(srcHost).getCanonicalHostName() - dstHost = InetAddress.getByName(dstHost).getCanonicalHostName() - } catch { - case e: UnknownHostException => - return false - } - } - - Objects.equal(srcHost, dstHost) && srcUri.getPort() == dstUri.getPort() - } - - /** - * Given a local URI, resolve it and return a qualified local path that corresponds to the URI. - * This is used for preparing local resources to be included in the container launch context. - */ - private def getQualifiedLocalPath(localURI: URI, hadoopConf: Configuration): Path = { - val qualifiedURI = - if (localURI.getScheme == null) { - // If not specified, assume this is in the local filesystem to keep the behavior - // consistent with that of Hadoop - new URI(FileSystem.getLocal(hadoopConf).makeQualified(new Path(localURI)).toString) - } else { - localURI - } - new Path(qualifiedURI) - } - - /** - * Whether to consider jars provided by the user to have precedence over the Spark jars when - * loading user classes. - */ - def isUserClassPathFirst(conf: SparkConf, isDriver: Boolean): Boolean = { - if (isDriver) { - conf.getBoolean("spark.driver.userClassPathFirst", false) - } else { - conf.getBoolean("spark.executor.userClassPathFirst", false) - } - } - - /** - * Joins all the path components using Path.SEPARATOR. - */ - def buildPath(components: String*): String = { - components.mkString(Path.SEPARATOR) - } - - /** - * Return whether delegation tokens should be retrieved for the given service when security is - * enabled. By default, tokens are retrieved, but that behavior can be changed by setting - * a service-specific configuration. - */ - def shouldGetTokens(conf: SparkConf, service: String): Boolean = { - conf.getBoolean(s"spark.yarn.security.tokens.${service}.enabled", true) - } - -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala deleted file mode 100644 index 1165061db21e..000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ /dev/null @@ -1,276 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ -import org.apache.spark.util.{IntParam, MemoryParam, Utils} - -// TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware ! -private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) { - var addJars: String = null - var files: String = null - var archives: String = null - var userJar: String = null - var userClass: String = null - var pyFiles: Seq[String] = Nil - var primaryPyFile: String = null - var primaryRFile: String = null - var userArgs: ArrayBuffer[String] = new ArrayBuffer[String]() - var executorMemory = 1024 // MB - var executorCores = 1 - var numExecutors = DEFAULT_NUMBER_EXECUTORS - var amQueue = sparkConf.get("spark.yarn.queue", "default") - var amMemory: Int = 512 // MB - var amCores: Int = 1 - var appName: String = "Spark" - var priority = 0 - var principal: String = null - var keytab: String = null - def isClusterMode: Boolean = userClass != null - - private var driverMemory: Int = Utils.DEFAULT_DRIVER_MEM_MB // MB - private var driverCores: Int = 1 - private val driverMemOverheadKey = "spark.yarn.driver.memoryOverhead" - private val amMemKey = "spark.yarn.am.memory" - private val amMemOverheadKey = "spark.yarn.am.memoryOverhead" - private val driverCoresKey = "spark.driver.cores" - private val amCoresKey = "spark.yarn.am.cores" - private val isDynamicAllocationEnabled = Utils.isDynamicAllocationEnabled(sparkConf) - - parseArgs(args.toList) - loadEnvironmentArgs() - validateArgs() - - // Additional memory to allocate to containers - val amMemoryOverheadConf = if (isClusterMode) driverMemOverheadKey else amMemOverheadKey - val amMemoryOverhead = sparkConf.getInt(amMemoryOverheadConf, - math.max((MEMORY_OVERHEAD_FACTOR * amMemory).toInt, MEMORY_OVERHEAD_MIN)) - - val executorMemoryOverhead = sparkConf.getInt("spark.yarn.executor.memoryOverhead", - math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toInt, MEMORY_OVERHEAD_MIN)) - - /** Load any default arguments provided through environment variables and Spark properties. */ - private def loadEnvironmentArgs(): Unit = { - // For backward compatibility, SPARK_YARN_DIST_{ARCHIVES/FILES} should be resolved to hdfs://, - // while spark.yarn.dist.{archives/files} should be resolved to file:// (SPARK-2051). - files = Option(files) - .orElse(sparkConf.getOption("spark.yarn.dist.files").map(p => Utils.resolveURIs(p))) - .orElse(sys.env.get("SPARK_YARN_DIST_FILES")) - .orNull - archives = Option(archives) - .orElse(sparkConf.getOption("spark.yarn.dist.archives").map(p => Utils.resolveURIs(p))) - .orElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES")) - .orNull - // If dynamic allocation is enabled, start at the configured initial number of executors. - // Default to minExecutors if no initialExecutors is set. - numExecutors = YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sparkConf) - principal = Option(principal) - .orElse(sparkConf.getOption("spark.yarn.principal")) - .orNull - keytab = Option(keytab) - .orElse(sparkConf.getOption("spark.yarn.keytab")) - .orNull - } - - /** - * Fail fast if any arguments provided are invalid. - * This is intended to be called only after the provided arguments have been parsed. - */ - private def validateArgs(): Unit = { - if (numExecutors < 0 || (!isDynamicAllocationEnabled && numExecutors == 0)) { - throw new IllegalArgumentException( - s""" - |Number of executors was $numExecutors, but must be at least 1 - |(or 0 if dynamic executor allocation is enabled). - |${getUsageMessage()} - """.stripMargin) - } - if (executorCores < sparkConf.getInt("spark.task.cpus", 1)) { - throw new SparkException("Executor cores must not be less than " + - "spark.task.cpus.") - } - // scalastyle:off println - if (isClusterMode) { - for (key <- Seq(amMemKey, amMemOverheadKey, amCoresKey)) { - if (sparkConf.contains(key)) { - println(s"$key is set but does not apply in cluster mode.") - } - } - amMemory = driverMemory - amCores = driverCores - } else { - for (key <- Seq(driverMemOverheadKey, driverCoresKey)) { - if (sparkConf.contains(key)) { - println(s"$key is set but does not apply in client mode.") - } - } - sparkConf.getOption(amMemKey) - .map(Utils.memoryStringToMb) - .foreach { mem => amMemory = mem } - sparkConf.getOption(amCoresKey) - .map(_.toInt) - .foreach { cores => amCores = cores } - } - // scalastyle:on println - } - - private def parseArgs(inputArgs: List[String]): Unit = { - var args = inputArgs - - // scalastyle:off println - while (!args.isEmpty) { - args match { - case ("--jar") :: value :: tail => - userJar = value - args = tail - - case ("--class") :: value :: tail => - userClass = value - args = tail - - case ("--primary-py-file") :: value :: tail => - primaryPyFile = value - args = tail - - case ("--primary-r-file") :: value :: tail => - primaryRFile = value - args = tail - - case ("--args" | "--arg") :: value :: tail => - if (args(0) == "--args") { - println("--args is deprecated. Use --arg instead.") - } - userArgs += value - args = tail - - case ("--master-class" | "--am-class") :: value :: tail => - println(s"${args(0)} is deprecated and is not used anymore.") - args = tail - - case ("--master-memory" | "--driver-memory") :: MemoryParam(value) :: tail => - if (args(0) == "--master-memory") { - println("--master-memory is deprecated. Use --driver-memory instead.") - } - driverMemory = value - args = tail - - case ("--driver-cores") :: IntParam(value) :: tail => - driverCores = value - args = tail - - case ("--num-workers" | "--num-executors") :: IntParam(value) :: tail => - if (args(0) == "--num-workers") { - println("--num-workers is deprecated. Use --num-executors instead.") - } - numExecutors = value - args = tail - - case ("--worker-memory" | "--executor-memory") :: MemoryParam(value) :: tail => - if (args(0) == "--worker-memory") { - println("--worker-memory is deprecated. Use --executor-memory instead.") - } - executorMemory = value - args = tail - - case ("--worker-cores" | "--executor-cores") :: IntParam(value) :: tail => - if (args(0) == "--worker-cores") { - println("--worker-cores is deprecated. Use --executor-cores instead.") - } - executorCores = value - args = tail - - case ("--queue") :: value :: tail => - amQueue = value - args = tail - - case ("--name") :: value :: tail => - appName = value - args = tail - - case ("--addJars") :: value :: tail => - addJars = value - args = tail - - case ("--py-files") :: value :: tail => - pyFiles = value.split(",") - args = tail - - case ("--files") :: value :: tail => - files = value - args = tail - - case ("--archives") :: value :: tail => - archives = value - args = tail - - case ("--principal") :: value :: tail => - principal = value - args = tail - - case ("--keytab") :: value :: tail => - keytab = value - args = tail - - case Nil => - - case _ => - throw new IllegalArgumentException(getUsageMessage(args)) - } - } - // scalastyle:on println - - if (primaryPyFile != null && primaryRFile != null) { - throw new IllegalArgumentException("Cannot have primary-py-file and primary-r-file" + - " at the same time") - } - } - - private def getUsageMessage(unknownParam: List[String] = null): String = { - val message = if (unknownParam != null) s"Unknown/unsupported param $unknownParam\n" else "" - val mem_mb = Utils.DEFAULT_DRIVER_MEM_MB - message + - s""" - |Usage: org.apache.spark.deploy.yarn.Client [options] - |Options: - | --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster - | mode) - | --class CLASS_NAME Name of your application's main class (required) - | --primary-py-file A main Python file - | --primary-r-file A main R file - | --arg ARG Argument to be passed to your application's main class. - | Multiple invocations are possible, each will be passed in order. - | --num-executors NUM Number of executors to start (Default: 2) - | --executor-cores NUM Number of cores per executor (Default: 1). - | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: $mem_mb Mb) - | --driver-cores NUM Number of cores used by the driver (Default: 1). - | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G) - | --name NAME The name of your application (Default: Spark) - | --queue QUEUE The hadoop queue to use for allocation requests (Default: - | 'default') - | --addJars jars Comma separated list of local jars that want SparkContext.addJar - | to work with. - | --py-files PY_FILES Comma-separated list of .zip, .egg, or .py files to - | place on the PYTHONPATH for Python apps. - | --files files Comma separated list of files to be distributed with the job. - | --archives archives Comma separated list of archives to be distributed with the job. - """.stripMargin - } -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala deleted file mode 100644 index 3d3a966960e9..000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ /dev/null @@ -1,207 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.net.URI - -import scala.collection.mutable.{HashMap, LinkedHashMap, Map} - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} -import org.apache.hadoop.fs.permission.FsAction -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.util.{Records, ConverterUtils} - -import org.apache.spark.Logging - -/** Client side methods to setup the Hadoop distributed cache */ -private[spark] class ClientDistributedCacheManager() extends Logging { - - // Mappings from remote URI to (file status, modification time, visibility) - private val distCacheFiles: Map[String, (String, String, String)] = - LinkedHashMap[String, (String, String, String)]() - private val distCacheArchives: Map[String, (String, String, String)] = - LinkedHashMap[String, (String, String, String)]() - - - /** - * Add a resource to the list of distributed cache resources. This list can - * be sent to the ApplicationMaster and possibly the executors so that it can - * be downloaded into the Hadoop distributed cache for use by this application. - * Adds the LocalResource to the localResources HashMap passed in and saves - * the stats of the resources to they can be sent to the executors and verified. - * - * @param fs FileSystem - * @param conf Configuration - * @param destPath path to the resource - * @param localResources localResource hashMap to insert the resource into - * @param resourceType LocalResourceType - * @param link link presented in the distributed cache to the destination - * @param statCache cache to store the file/directory stats - * @param appMasterOnly Whether to only add the resource to the app master - */ - def addResource( - fs: FileSystem, - conf: Configuration, - destPath: Path, - localResources: HashMap[String, LocalResource], - resourceType: LocalResourceType, - link: String, - statCache: Map[URI, FileStatus], - appMasterOnly: Boolean = false): Unit = { - val destStatus = fs.getFileStatus(destPath) - val amJarRsrc = Records.newRecord(classOf[LocalResource]) - amJarRsrc.setType(resourceType) - val visibility = getVisibility(conf, destPath.toUri(), statCache) - amJarRsrc.setVisibility(visibility) - amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(destPath)) - amJarRsrc.setTimestamp(destStatus.getModificationTime()) - amJarRsrc.setSize(destStatus.getLen()) - if (link == null || link.isEmpty()) throw new Exception("You must specify a valid link name") - localResources(link) = amJarRsrc - - if (!appMasterOnly) { - val uri = destPath.toUri() - val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link) - if (resourceType == LocalResourceType.FILE) { - distCacheFiles(pathURI.toString()) = (destStatus.getLen().toString(), - destStatus.getModificationTime().toString(), visibility.name()) - } else { - distCacheArchives(pathURI.toString()) = (destStatus.getLen().toString(), - destStatus.getModificationTime().toString(), visibility.name()) - } - } - } - - /** - * Adds the necessary cache file env variables to the env passed in - */ - def setDistFilesEnv(env: Map[String, String]): Unit = { - val (keys, tupleValues) = distCacheFiles.unzip - val (sizes, timeStamps, visibilities) = tupleValues.unzip3 - if (keys.size > 0) { - env("SPARK_YARN_CACHE_FILES") = keys.reduceLeft[String] { (acc, n) => acc + "," + n } - env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") = - timeStamps.reduceLeft[String] { (acc, n) => acc + "," + n } - env("SPARK_YARN_CACHE_FILES_FILE_SIZES") = - sizes.reduceLeft[String] { (acc, n) => acc + "," + n } - env("SPARK_YARN_CACHE_FILES_VISIBILITIES") = - visibilities.reduceLeft[String] { (acc, n) => acc + "," + n } - } - } - - /** - * Adds the necessary cache archive env variables to the env passed in - */ - def setDistArchivesEnv(env: Map[String, String]): Unit = { - val (keys, tupleValues) = distCacheArchives.unzip - val (sizes, timeStamps, visibilities) = tupleValues.unzip3 - if (keys.size > 0) { - env("SPARK_YARN_CACHE_ARCHIVES") = keys.reduceLeft[String] { (acc, n) => acc + "," + n } - env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") = - timeStamps.reduceLeft[String] { (acc, n) => acc + "," + n } - env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") = - sizes.reduceLeft[String] { (acc, n) => acc + "," + n } - env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") = - visibilities.reduceLeft[String] { (acc, n) => acc + "," + n } - } - } - - /** - * Returns the local resource visibility depending on the cache file permissions - * @return LocalResourceVisibility - */ - def getVisibility( - conf: Configuration, - uri: URI, - statCache: Map[URI, FileStatus]): LocalResourceVisibility = { - if (isPublic(conf, uri, statCache)) { - LocalResourceVisibility.PUBLIC - } else { - LocalResourceVisibility.PRIVATE - } - } - - /** - * Returns a boolean to denote whether a cache file is visible to all (public) - * @return true if the path in the uri is visible to all, false otherwise - */ - def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = { - val fs = FileSystem.get(uri, conf) - val current = new Path(uri.getPath()) - // the leaf level file should be readable by others - if (!checkPermissionOfOther(fs, current, FsAction.READ, statCache)) { - return false - } - ancestorsHaveExecutePermissions(fs, current.getParent(), statCache) - } - - /** - * Returns true if all ancestors of the specified path have the 'execute' - * permission set for all users (i.e. that other users can traverse - * the directory hierarchy to the given path) - * @return true if all ancestors have the 'execute' permission set for all users - */ - def ancestorsHaveExecutePermissions( - fs: FileSystem, - path: Path, - statCache: Map[URI, FileStatus]): Boolean = { - var current = path - while (current != null) { - // the subdirs in the path should have execute permissions for others - if (!checkPermissionOfOther(fs, current, FsAction.EXECUTE, statCache)) { - return false - } - current = current.getParent() - } - true - } - - /** - * Checks for a given path whether the Other permissions on it - * imply the permission in the passed FsAction - * @return true if the path in the uri is visible to all, false otherwise - */ - def checkPermissionOfOther( - fs: FileSystem, - path: Path, - action: FsAction, - statCache: Map[URI, FileStatus]): Boolean = { - val status = getFileStatus(fs, path.toUri(), statCache) - val perms = status.getPermission() - val otherAction = perms.getOtherAction() - otherAction.implies(action) - } - - /** - * Checks to see if the given uri exists in the cache, if it does it - * returns the existing FileStatus, otherwise it stats the uri, stores - * it in the cache, and returns the FileStatus. - * @return FileStatus - */ - def getFileStatus(fs: FileSystem, uri: URI, statCache: Map[URI, FileStatus]): FileStatus = { - val stat = statCache.get(uri) match { - case Some(existstat) => existstat - case None => - val newStat = fs.getFileStatus(new Path(uri)) - statCache.put(uri, newStat) - newStat - } - stat - } -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala deleted file mode 100644 index 94feb6393fd6..000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorDelegationTokenUpdater.scala +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.yarn - -import java.util.concurrent.{Executors, TimeUnit} - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.security.{Credentials, UserGroupInformation} - -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.util.{ThreadUtils, Utils} - -import scala.util.control.NonFatal - -private[spark] class ExecutorDelegationTokenUpdater( - sparkConf: SparkConf, - hadoopConf: Configuration) extends Logging { - - @volatile private var lastCredentialsFileSuffix = 0 - - private val credentialsFile = sparkConf.get("spark.yarn.credentials.file") - private val freshHadoopConf = - SparkHadoopUtil.get.getConfBypassingFSCache( - hadoopConf, new Path(credentialsFile).toUri.getScheme) - - private val delegationTokenRenewer = - Executors.newSingleThreadScheduledExecutor( - ThreadUtils.namedThreadFactory("Delegation Token Refresh Thread")) - - // On the executor, this thread wakes up and picks up new tokens from HDFS, if any. - private val executorUpdaterRunnable = - new Runnable { - override def run(): Unit = Utils.logUncaughtExceptions(updateCredentialsIfRequired()) - } - - def updateCredentialsIfRequired(): Unit = { - try { - val credentialsFilePath = new Path(credentialsFile) - val remoteFs = FileSystem.get(freshHadoopConf) - SparkHadoopUtil.get.listFilesSorted( - remoteFs, credentialsFilePath.getParent, - credentialsFilePath.getName, SparkHadoopUtil.SPARK_YARN_CREDS_TEMP_EXTENSION) - .lastOption.foreach { credentialsStatus => - val suffix = SparkHadoopUtil.get.getSuffixForCredentialsPath(credentialsStatus.getPath) - if (suffix > lastCredentialsFileSuffix) { - logInfo("Reading new delegation tokens from " + credentialsStatus.getPath) - val newCredentials = getCredentialsFromHDFSFile(remoteFs, credentialsStatus.getPath) - lastCredentialsFileSuffix = suffix - UserGroupInformation.getCurrentUser.addCredentials(newCredentials) - logInfo("Tokens updated from credentials file.") - } else { - // Check every hour to see if new credentials arrived. - logInfo("Updated delegation tokens were expected, but the driver has not updated the " + - "tokens yet, will check again in an hour.") - delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) - return - } - } - val timeFromNowToRenewal = - SparkHadoopUtil.get.getTimeFromNowToRenewal( - sparkConf, 0.8, UserGroupInformation.getCurrentUser.getCredentials) - if (timeFromNowToRenewal <= 0) { - executorUpdaterRunnable.run() - } else { - logInfo(s"Scheduling token refresh from HDFS in $timeFromNowToRenewal millis.") - delegationTokenRenewer.schedule( - executorUpdaterRunnable, timeFromNowToRenewal, TimeUnit.MILLISECONDS) - } - } catch { - // Since the file may get deleted while we are reading it, catch the Exception and come - // back in an hour to try again - case NonFatal(e) => - logWarning("Error while trying to update credentials, will try again in 1 hour", e) - delegationTokenRenewer.schedule(executorUpdaterRunnable, 1, TimeUnit.HOURS) - } - } - - private def getCredentialsFromHDFSFile(remoteFs: FileSystem, tokenPath: Path): Credentials = { - val stream = remoteFs.open(tokenPath) - try { - val newCredentials = new Credentials() - newCredentials.readTokenStorageStream(stream) - newCredentials - } finally { - stream.close() - } - } - - def stop(): Unit = { - delegationTokenRenewer.shutdown() - } - -} 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 deleted file mode 100644 index 2232ffba473b..000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ /dev/null @@ -1,324 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.io.File -import java.net.URI -import java.nio.ByteBuffer -import java.util.Collections - -import scala.collection.JavaConverters._ -import scala.collection.mutable.{HashMap, ListBuffer} - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.security.UserGroupInformation -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.api.NMClient -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.{Logging, SecurityManager, SparkConf, SparkException} -import org.apache.spark.launcher.YarnCommandBuilderUtils -import org.apache.spark.network.util.JavaUtils -import org.apache.spark.util.Utils - -class ExecutorRunnable( - container: Container, - conf: Configuration, - sparkConf: SparkConf, - masterAddress: String, - slaveId: String, - hostname: String, - executorMemory: Int, - executorCores: Int, - appId: String, - securityMgr: SecurityManager) - extends Runnable with Logging { - - var rpc: YarnRPC = YarnRPC.create(conf) - var nmClient: NMClient = _ - val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - lazy val env = prepareEnvironment(container) - - override def run(): Unit = { - logInfo("Starting Executor Container") - nmClient = NMClient.createNMClient() - nmClient.init(yarnConf) - nmClient.start() - startContainer() - } - - def startContainer(): java.util.Map[String, ByteBuffer] = { - logInfo("Setting up ContainerLaunchContext") - - val ctx = Records.newRecord(classOf[ContainerLaunchContext]) - .asInstanceOf[ContainerLaunchContext] - - val localResources = prepareLocalResources - ctx.setLocalResources(localResources.asJava) - - ctx.setEnvironment(env.asJava) - - val credentials = UserGroupInformation.getCurrentUser().getCredentials() - val dob = new DataOutputBuffer() - credentials.writeTokenStorageToStream(dob) - ctx.setTokens(ByteBuffer.wrap(dob.getData())) - - val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, - appId, localResources) - - logInfo(s""" - |=============================================================================== - |YARN executor launch context: - | env: - |${env.map { case (k, v) => s" $k -> $v\n" }.mkString} - | command: - | ${commands.mkString(" ")} - |=============================================================================== - """.stripMargin) - - ctx.setCommands(commands.asJava) - ctx.setApplicationACLs( - YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr).asJava) - - // If external shuffle service is enabled, register with the Yarn shuffle service already - // started on the NodeManager and, if authentication is enabled, provide it with our secret - // key for fetching shuffle files later - if (sparkConf.getBoolean("spark.shuffle.service.enabled", false)) { - val secretString = securityMgr.getSecretKey() - val secretBytes = - if (secretString != null) { - // This conversion must match how the YarnShuffleService decodes our secret - JavaUtils.stringToBytes(secretString) - } else { - // Authentication is not enabled, so just provide dummy metadata - ByteBuffer.allocate(0) - } - ctx.setServiceData(Collections.singletonMap("spark_shuffle", secretBytes)) - } - - // Send the start request to the ContainerManager - 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( - masterAddress: String, - slaveId: String, - hostname: String, - executorMemory: Int, - executorCores: Int, - appId: String, - localResources: HashMap[String, LocalResource]): List[String] = { - // Extra options for the JVM - val javaOpts = ListBuffer[String]() - - // Set the environment variable through a command prefix - // to append to the existing value of the variable - var prefixEnv: Option[String] = None - - // Set the JVM memory - val executorMemoryString = executorMemory + "m" - javaOpts += "-Xms" + executorMemoryString - javaOpts += "-Xmx" + executorMemoryString - - // Set extra Java options for the executor, if defined - sys.props.get("spark.executor.extraJavaOptions").foreach { opts => - javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell) - } - sys.env.get("SPARK_JAVA_OPTS").foreach { opts => - javaOpts ++= Utils.splitCommandString(opts).map(YarnSparkHadoopUtil.escapeForShell) - } - sys.props.get("spark.executor.extraLibraryPath").foreach { p => - prefixEnv = Some(Client.getClusterPath(sparkConf, Utils.libraryPathEnvPrefix(Seq(p)))) - } - - javaOpts += "-Djava.io.tmpdir=" + - new Path( - YarnSparkHadoopUtil.expandEnvironment(Environment.PWD), - YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR - ) - - // Certain configs need to be passed here because they are needed before the Executor - // registers with the Scheduler and transfers the spark configs. Since the Executor backend - // uses Akka to connect to the scheduler, the akka settings are needed as well as the - // authentication settings. - sparkConf.getAll - .filter { case (k, v) => SparkConf.isExecutorStartupConf(k) } - .foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } - - // Commenting it out for now - so that people can refer to the properties if required. Remove - // it once cpuset version is pushed out. - // The context is, default gc for server class machines end up using all cores to do gc - hence - // if there are multiple containers in same node, spark gc effects all other containers - // performance (which can also be other spark containers) - // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in - // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset - // of cores on a node. - /* - else { - // If no java_opts specified, default to using -XX:+CMSIncrementalMode - // It might be possible that other modes/config is being done in - // spark.executor.extraJavaOptions, so we dont want to mess with it. - // In our expts, using (default) throughput collector has severe perf ramnifications in - // multi-tennent machines - // The options are based on - // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use - // %20the%20Concurrent%20Low%20Pause%20Collector|outline - javaOpts += "-XX:+UseConcMarkSweepGC" - javaOpts += "-XX:+CMSIncrementalMode" - javaOpts += "-XX:+CMSIncrementalPacing" - javaOpts += "-XX:CMSIncrementalDutyCycleMin=0" - javaOpts += "-XX:CMSIncrementalDutyCycle=10" - } - */ - - // For log4j configuration to reference - javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) - YarnCommandBuilderUtils.addPermGenSizeOpt(javaOpts) - - val userClassPath = Client.getUserClasspath(sparkConf).flatMap { uri => - val absPath = - if (new File(uri.getPath()).isAbsolute()) { - Client.getClusterPath(sparkConf, uri.getPath()) - } else { - Client.buildPath(Environment.PWD.$(), uri.getPath()) - } - Seq("--user-class-path", "file:" + absPath) - }.toSeq - - val commands = prefixEnv ++ Seq( - YarnSparkHadoopUtil.expandEnvironment(Environment.JAVA_HOME) + "/bin/java", - "-server", - // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. - // Not killing the task leaves various aspects of the executor and (to some extent) the jvm in - // an inconsistent state. - // TODO: If the OOM is not recoverable by rescheduling it on different node, then do - // 'something' to fail job ... akin to blacklisting trackers in mapred ? - YarnSparkHadoopUtil.getOutOfMemoryErrorArgument) ++ - javaOpts ++ - Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend", - "--driver-url", masterAddress.toString, - "--executor-id", slaveId.toString, - "--hostname", hostname.toString, - "--cores", executorCores.toString, - "--app-id", appId) ++ - userClassPath ++ - Seq( - "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", - "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") - - // TODO: it would be nicer to just make sure there are no null commands here - commands.map(s => if (s == null) "null" else s).toList - } - - private def setupDistributedCache( - file: String, - rtype: LocalResourceType, - localResources: HashMap[String, LocalResource], - timestamp: String, - size: String, - vis: String): Unit = { - val uri = new URI(file) - val amJarRsrc = Records.newRecord(classOf[LocalResource]) - amJarRsrc.setType(rtype) - amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis)) - amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri)) - amJarRsrc.setTimestamp(timestamp.toLong) - amJarRsrc.setSize(size.toLong) - localResources(uri.getFragment()) = amJarRsrc - } - - private def prepareLocalResources: HashMap[String, LocalResource] = { - logInfo("Preparing Local resources") - val localResources = HashMap[String, LocalResource]() - - if (System.getenv("SPARK_YARN_CACHE_FILES") != null) { - val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') - val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') - val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',') - val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',') - for( i <- 0 to distFiles.length - 1) { - setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i), - fileSizes(i), visibilities(i)) - } - } - - if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) { - val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',') - val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',') - val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',') - val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',') - for( i <- 0 to distArchives.length - 1) { - setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources, - timeStamps(i), fileSizes(i), visibilities(i)) - } - } - - logInfo("Prepared Local resources " + localResources) - localResources - } - - private def prepareEnvironment(container: Container): HashMap[String, String] = { - val env = new HashMap[String, String]() - val extraCp = sparkConf.getOption("spark.executor.extraClassPath") - Client.populateClasspath(null, yarnConf, sparkConf, env, false, extraCp) - - sparkConf.getExecutorEnv.foreach { case (key, value) => - // This assumes each executor environment variable set here is a path - // This is kept for backward compatibility and consistency with hadoop - YarnSparkHadoopUtil.addPathToEnvironment(env, key, value) - } - - // Keep this for backwards compatibility but users should move to the config - sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => - YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) - } - - // lookup appropriate http scheme for container log urls - val yarnHttpPolicy = yarnConf.get( - YarnConfiguration.YARN_HTTP_POLICY_KEY, - YarnConfiguration.YARN_HTTP_POLICY_DEFAULT - ) - val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" - - // Add log urls - sys.env.get("SPARK_USER").foreach { user => - val containerId = ConverterUtils.toString(container.getId) - val address = container.getNodeHttpAddress - val baseUrl = s"$httpScheme$address/node/containerlogs/$containerId/$user" - - env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=-4096" - env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=-4096" - } - - System.getenv().asScala.filterKeys(_.startsWith("SPARK")) - .foreach { case (k, v) => env(k) = v } - env - } -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala deleted file mode 100644 index 4d9e777cb413..000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ /dev/null @@ -1,607 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.util.Collections -import java.util.concurrent._ -import java.util.regex.Pattern - -import scala.collection.mutable -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.collection.JavaConverters._ - -import com.google.common.util.concurrent.ThreadFactoryBuilder - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.api.AMRMClient -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -import org.apache.hadoop.yarn.util.RackResolver - -import org.apache.log4j.{Level, Logger} - -import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ -import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef} -import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason} -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor -import org.apache.spark.util.Utils - -/** - * YarnAllocator is charged with requesting containers from the YARN ResourceManager and deciding - * what to do with containers when YARN fulfills these requests. - * - * This class makes use of YARN's AMRMClient APIs. We interact with the AMRMClient in three ways: - * * Making our resource needs known, which updates local bookkeeping about containers requested. - * * Calling "allocate", which syncs our local container requests with the RM, and returns any - * containers that YARN has granted to us. This also functions as a heartbeat. - * * Processing the containers granted to us to possibly launch executors inside of them. - * - * The public methods of this class are thread-safe. All methods that mutate state are - * synchronized. - */ -private[yarn] class YarnAllocator( - driverUrl: String, - driverRef: RpcEndpointRef, - conf: Configuration, - sparkConf: SparkConf, - amClient: AMRMClient[ContainerRequest], - appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments, - securityMgr: SecurityManager) - extends Logging { - - import YarnAllocator._ - - // RackResolver logs an INFO message whenever it resolves a rack, which is way too often. - if (Logger.getLogger(classOf[RackResolver]).getLevel == null) { - Logger.getLogger(classOf[RackResolver]).setLevel(Level.WARN) - } - - // Visible for testing. - val allocatedHostToContainersMap = new HashMap[String, collection.mutable.Set[ContainerId]] - val allocatedContainerToHostMap = new HashMap[ContainerId, String] - - // Containers that we no longer care about. We've either already told the RM to release them or - // will on the next heartbeat. Containers get removed from this map after the RM tells us they've - // completed. - private val releasedContainers = Collections.newSetFromMap[ContainerId]( - new ConcurrentHashMap[ContainerId, java.lang.Boolean]) - - @volatile private var numExecutorsRunning = 0 - // Used to generate a unique ID per executor - private var executorIdCounter = 0 - @volatile private var numExecutorsFailed = 0 - - @volatile private var targetNumExecutors = - YarnSparkHadoopUtil.getInitialTargetExecutorNumber(sparkConf) - - // Executor loss reason requests that are pending - maps from executor ID for inquiry to a - // list of requesters that should be responded to once we find out why the given executor - // was lost. - private val pendingLossReasonRequests = new HashMap[String, mutable.Buffer[RpcCallContext]] - - // Keep track of which container is running which executor to remove the executors later - // Visible for testing. - private[yarn] val executorIdToContainer = new HashMap[String, Container] - - private var numUnexpectedContainerRelease = 0L - private val containerIdToExecutorId = new HashMap[ContainerId, String] - - // Executor memory in MB. - protected val executorMemory = args.executorMemory - // Additional memory overhead. - protected val memoryOverhead: Int = sparkConf.getInt("spark.yarn.executor.memoryOverhead", - math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toInt, MEMORY_OVERHEAD_MIN)) - // Number of cores per executor. - protected val executorCores = args.executorCores - // Resource capability requested for each executors - private[yarn] val resource = Resource.newInstance(executorMemory + memoryOverhead, executorCores) - - private val launcherPool = new ThreadPoolExecutor( - // max pool size of Integer.MAX_VALUE is ignored because we use an unbounded queue - sparkConf.getInt("spark.yarn.containerLauncherMaxThreads", 25), Integer.MAX_VALUE, - 1, TimeUnit.MINUTES, - new LinkedBlockingQueue[Runnable](), - new ThreadFactoryBuilder().setNameFormat("ContainerLauncher #%d").setDaemon(true).build()) - launcherPool.allowCoreThreadTimeOut(true) - - // For testing - private val launchContainers = sparkConf.getBoolean("spark.yarn.launchContainers", true) - - private val labelExpression = sparkConf.getOption("spark.yarn.executor.nodeLabelExpression") - - // ContainerRequest constructor that can take a node label expression. We grab it through - // reflection because it's only available in later versions of YARN. - private val nodeLabelConstructor = labelExpression.flatMap { expr => - try { - Some(classOf[ContainerRequest].getConstructor(classOf[Resource], - classOf[Array[String]], classOf[Array[String]], classOf[Priority], classOf[Boolean], - classOf[String])) - } catch { - case e: NoSuchMethodException => { - logWarning(s"Node label expression $expr will be ignored because YARN version on" + - " classpath does not support it.") - None - } - } - } - - // A map to store preferred hostname and possible task numbers running on it. - private var hostToLocalTaskCounts: Map[String, Int] = Map.empty - - // Number of tasks that have locality preferences in active stages - private var numLocalityAwareTasks: Int = 0 - - // A container placement strategy based on pending tasks' locality preference - private[yarn] val containerPlacementStrategy = - new LocalityPreferredContainerPlacementStrategy(sparkConf, conf, resource) - - def getNumExecutorsRunning: Int = numExecutorsRunning - - def getNumExecutorsFailed: Int = numExecutorsFailed - - /** - * A sequence of pending container requests that have not yet been fulfilled. - */ - def getPendingAllocate: Seq[ContainerRequest] = getPendingAtLocation(ANY_HOST) - - /** - * A sequence of pending container requests at the given location that have not yet been - * fulfilled. - */ - private def getPendingAtLocation(location: String): Seq[ContainerRequest] = { - amClient.getMatchingRequests(RM_REQUEST_PRIORITY, location, resource).asScala - .flatMap(_.asScala) - .toSeq - } - - /** - * Request as many executors from the ResourceManager as needed to reach the desired total. If - * the requested total is smaller than the current number of running executors, no executors will - * be killed. - * @param requestedTotal total number of containers requested - * @param localityAwareTasks number of locality aware tasks to be used as container placement hint - * @param hostToLocalTaskCount a map of preferred hostname to possible task counts to be used as - * container placement hint. - * @return Whether the new requested total is different than the old value. - */ - def requestTotalExecutorsWithPreferredLocalities( - requestedTotal: Int, - localityAwareTasks: Int, - hostToLocalTaskCount: Map[String, Int]): Boolean = synchronized { - this.numLocalityAwareTasks = localityAwareTasks - this.hostToLocalTaskCounts = hostToLocalTaskCount - - if (requestedTotal != targetNumExecutors) { - logInfo(s"Driver requested a total number of $requestedTotal executor(s).") - targetNumExecutors = requestedTotal - true - } else { - false - } - } - - /** - * Request that the ResourceManager release the container running the specified executor. - */ - def killExecutor(executorId: String): Unit = synchronized { - if (executorIdToContainer.contains(executorId)) { - val container = executorIdToContainer.remove(executorId).get - containerIdToExecutorId.remove(container.getId) - internalReleaseContainer(container) - numExecutorsRunning -= 1 - } else { - logWarning(s"Attempted to kill unknown executor $executorId!") - } - } - - /** - * Request resources such that, if YARN gives us all we ask for, we'll have a number of containers - * equal to maxExecutors. - * - * Deal with any containers YARN has granted to us by possibly launching executors in them. - * - * This must be synchronized because variables read in this method are mutated by other methods. - */ - def allocateResources(): Unit = synchronized { - updateResourceRequests() - - val progressIndicator = 0.1f - // Poll the ResourceManager. This doubles as a heartbeat if there are no pending container - // requests. - val allocateResponse = amClient.allocate(progressIndicator) - - val allocatedContainers = allocateResponse.getAllocatedContainers() - - if (allocatedContainers.size > 0) { - logDebug("Allocated containers: %d. Current executor count: %d. Cluster resources: %s." - .format( - allocatedContainers.size, - numExecutorsRunning, - allocateResponse.getAvailableResources)) - - handleAllocatedContainers(allocatedContainers.asScala) - } - - val completedContainers = allocateResponse.getCompletedContainersStatuses() - if (completedContainers.size > 0) { - logDebug("Completed %d containers".format(completedContainers.size)) - processCompletedContainers(completedContainers.asScala) - logDebug("Finished processing %d completed containers. Current running executor count: %d." - .format(completedContainers.size, numExecutorsRunning)) - } - } - - /** - * Update the set of container requests that we will sync with the RM based on the number of - * executors we have currently running and our target number of executors. - * - * Visible for testing. - */ - def updateResourceRequests(): Unit = { - val pendingAllocate = getPendingAllocate - val numPendingAllocate = pendingAllocate.size - val missing = targetNumExecutors - numPendingAllocate - numExecutorsRunning - - if (missing > 0) { - logInfo(s"Will request $missing executor containers, each with ${resource.getVirtualCores} " + - s"cores and ${resource.getMemory} MB memory including $memoryOverhead MB overhead") - - // Split the pending container request into three groups: locality matched list, locality - // unmatched list and non-locality list. Take the locality matched container request into - // consideration of container placement, treat as allocated containers. - // For locality unmatched and locality free container requests, cancel these container - // requests, since required locality preference has been changed, recalculating using - // container placement strategy. - val (localityMatched, localityUnMatched, localityFree) = splitPendingAllocationsByLocality( - hostToLocalTaskCounts, pendingAllocate) - - // Remove the outdated container request and recalculate the requested container number - localityUnMatched.foreach(amClient.removeContainerRequest) - localityFree.foreach(amClient.removeContainerRequest) - val updatedNumContainer = missing + localityUnMatched.size + localityFree.size - - val containerLocalityPreferences = containerPlacementStrategy.localityOfRequestedContainers( - updatedNumContainer, numLocalityAwareTasks, hostToLocalTaskCounts, - allocatedHostToContainersMap, localityMatched) - - for (locality <- containerLocalityPreferences) { - val request = createContainerRequest(resource, locality.nodes, locality.racks) - amClient.addContainerRequest(request) - val nodes = request.getNodes - val hostStr = if (nodes == null || nodes.isEmpty) "Any" else nodes.asScala.last - logInfo(s"Container request (host: $hostStr, capability: $resource)") - } - } else if (missing < 0) { - val numToCancel = math.min(numPendingAllocate, -missing) - logInfo(s"Canceling requests for $numToCancel executor containers") - - val matchingRequests = amClient.getMatchingRequests(RM_REQUEST_PRIORITY, ANY_HOST, resource) - if (!matchingRequests.isEmpty) { - matchingRequests.iterator().next().asScala - .take(numToCancel).foreach(amClient.removeContainerRequest) - } else { - logWarning("Expected to find pending requests, but found none.") - } - } - } - - /** - * Creates a container request, handling the reflection required to use YARN features that were - * added in recent versions. - */ - private def createContainerRequest( - resource: Resource, - nodes: Array[String], - racks: Array[String]): ContainerRequest = { - nodeLabelConstructor.map { constructor => - constructor.newInstance(resource, nodes, racks, RM_REQUEST_PRIORITY, true: java.lang.Boolean, - labelExpression.orNull) - }.getOrElse(new ContainerRequest(resource, nodes, racks, RM_REQUEST_PRIORITY)) - } - - /** - * Handle containers granted by the RM by launching executors on them. - * - * Due to the way the YARN allocation protocol works, certain healthy race conditions can result - * in YARN granting containers that we no longer need. In this case, we release them. - * - * Visible for testing. - */ - def handleAllocatedContainers(allocatedContainers: Seq[Container]): Unit = { - val containersToUse = new ArrayBuffer[Container](allocatedContainers.size) - - // Match incoming requests by host - val remainingAfterHostMatches = new ArrayBuffer[Container] - for (allocatedContainer <- allocatedContainers) { - matchContainerToRequest(allocatedContainer, allocatedContainer.getNodeId.getHost, - containersToUse, remainingAfterHostMatches) - } - - // Match remaining by rack - val remainingAfterRackMatches = new ArrayBuffer[Container] - for (allocatedContainer <- remainingAfterHostMatches) { - val rack = RackResolver.resolve(conf, allocatedContainer.getNodeId.getHost).getNetworkLocation - matchContainerToRequest(allocatedContainer, rack, containersToUse, - remainingAfterRackMatches) - } - - // Assign remaining that are neither node-local nor rack-local - val remainingAfterOffRackMatches = new ArrayBuffer[Container] - for (allocatedContainer <- remainingAfterRackMatches) { - matchContainerToRequest(allocatedContainer, ANY_HOST, containersToUse, - remainingAfterOffRackMatches) - } - - if (!remainingAfterOffRackMatches.isEmpty) { - logDebug(s"Releasing ${remainingAfterOffRackMatches.size} unneeded containers that were " + - s"allocated to us") - for (container <- remainingAfterOffRackMatches) { - internalReleaseContainer(container) - } - } - - runAllocatedContainers(containersToUse) - - logInfo("Received %d containers from YARN, launching executors on %d of them." - .format(allocatedContainers.size, containersToUse.size)) - } - - /** - * Looks for requests for the given location that match the given container allocation. If it - * finds one, removes the request so that it won't be submitted again. Places the container into - * containersToUse or remaining. - * - * @param allocatedContainer container that was given to us by YARN - * @param location resource name, either a node, rack, or * - * @param containersToUse list of containers that will be used - * @param remaining list of containers that will not be used - */ - private def matchContainerToRequest( - allocatedContainer: Container, - location: String, - containersToUse: ArrayBuffer[Container], - remaining: ArrayBuffer[Container]): Unit = { - // SPARK-6050: certain Yarn configurations return a virtual core count that doesn't match the - // request; for example, capacity scheduler + DefaultResourceCalculator. So match on requested - // memory, but use the asked vcore count for matching, effectively disabling matching on vcore - // count. - val matchingResource = Resource.newInstance(allocatedContainer.getResource.getMemory, - resource.getVirtualCores) - val matchingRequests = amClient.getMatchingRequests(allocatedContainer.getPriority, location, - matchingResource) - - // Match the allocation to a request - if (!matchingRequests.isEmpty) { - val containerRequest = matchingRequests.get(0).iterator.next - amClient.removeContainerRequest(containerRequest) - containersToUse += allocatedContainer - } else { - remaining += allocatedContainer - } - } - - /** - * Launches executors in the allocated containers. - */ - private def runAllocatedContainers(containersToUse: ArrayBuffer[Container]): Unit = { - for (container <- containersToUse) { - numExecutorsRunning += 1 - assert(numExecutorsRunning <= targetNumExecutors) - val executorHostname = container.getNodeId.getHost - val containerId = container.getId - executorIdCounter += 1 - val executorId = executorIdCounter.toString - - assert(container.getResource.getMemory >= resource.getMemory) - - logInfo("Launching container %s for on host %s".format(containerId, executorHostname)) - executorIdToContainer(executorId) = container - containerIdToExecutorId(container.getId) = executorId - - val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, - new HashSet[ContainerId]) - - containerSet += containerId - allocatedContainerToHostMap.put(containerId, executorHostname) - - val executorRunnable = new ExecutorRunnable( - container, - conf, - sparkConf, - driverUrl, - executorId, - executorHostname, - executorMemory, - executorCores, - appAttemptId.getApplicationId.toString, - securityMgr) - if (launchContainers) { - logInfo("Launching ExecutorRunnable. driverUrl: %s, executorHostname: %s".format( - driverUrl, executorHostname)) - launcherPool.execute(executorRunnable) - } - } - } - - // Visible for testing. - private[yarn] def processCompletedContainers(completedContainers: Seq[ContainerStatus]): Unit = { - for (completedContainer <- completedContainers) { - val containerId = completedContainer.getContainerId - val alreadyReleased = releasedContainers.remove(containerId) - val hostOpt = allocatedContainerToHostMap.get(containerId) - val onHostStr = hostOpt.map(host => s" on host: $host").getOrElse("") - val exitReason = if (!alreadyReleased) { - // Decrement the number of executors running. The next iteration of - // the ApplicationMaster's reporting thread will take care of allocating. - numExecutorsRunning -= 1 - logInfo("Completed container %s%s (state: %s, exit status: %s)".format( - containerId, - onHostStr, - completedContainer.getState, - completedContainer.getExitStatus)) - // Hadoop 2.2.X added a ContainerExitStatus we should switch to use - // there are some exit status' we shouldn't necessarily count against us, but for - // now I think its ok as none of the containers are expected to exit. - val exitStatus = completedContainer.getExitStatus - val (exitCausedByApp, containerExitReason) = exitStatus match { - case ContainerExitStatus.SUCCESS => - (false, s"Executor for container $containerId exited because of a YARN event (e.g., " + - "pre-emption) and not because of an error in the running job.") - case ContainerExitStatus.PREEMPTED => - // Preemption is not the fault of the running tasks, since YARN preempts containers - // merely to do resource sharing, and tasks that fail due to preempted executors could - // just as easily finish on any other executor. See SPARK-8167. - (false, s"Container ${containerId}${onHostStr} was preempted.") - // Should probably still count memory exceeded exit codes towards task failures - case VMEM_EXCEEDED_EXIT_CODE => - (true, memLimitExceededLogMessage( - completedContainer.getDiagnostics, - VMEM_EXCEEDED_PATTERN)) - case PMEM_EXCEEDED_EXIT_CODE => - (true, memLimitExceededLogMessage( - completedContainer.getDiagnostics, - PMEM_EXCEEDED_PATTERN)) - case unknown => - numExecutorsFailed += 1 - (true, "Container marked as failed: " + containerId + onHostStr + - ". Exit status: " + completedContainer.getExitStatus + - ". Diagnostics: " + completedContainer.getDiagnostics) - - } - if (exitCausedByApp) { - logWarning(containerExitReason) - } else { - logInfo(containerExitReason) - } - ExecutorExited(0, exitCausedByApp, containerExitReason) - } else { - // If we have already released this container, then it must mean - // that the driver has explicitly requested it to be killed - ExecutorExited(completedContainer.getExitStatus, exitCausedByApp = false, - s"Container $containerId exited from explicit termination request.") - } - - for { - host <- hostOpt - containerSet <- allocatedHostToContainersMap.get(host) - } { - containerSet.remove(containerId) - if (containerSet.isEmpty) { - allocatedHostToContainersMap.remove(host) - } else { - allocatedHostToContainersMap.update(host, containerSet) - } - - allocatedContainerToHostMap.remove(containerId) - } - - containerIdToExecutorId.remove(containerId).foreach { eid => - executorIdToContainer.remove(eid) - pendingLossReasonRequests.remove(eid).foreach { pendingRequests => - // Notify application of executor loss reasons so it can decide whether it should abort - pendingRequests.foreach(_.reply(exitReason)) - } - if (!alreadyReleased) { - // The executor could have gone away (like no route to host, node failure, etc) - // Notify backend about the failure of the executor - numUnexpectedContainerRelease += 1 - driverRef.send(RemoveExecutor(eid, exitReason)) - } - } - } - } - - /** - * Register that some RpcCallContext has asked the AM why the executor was lost. Note that - * we can only find the loss reason to send back in the next call to allocateResources(). - */ - private[yarn] def enqueueGetLossReasonRequest( - eid: String, - context: RpcCallContext): Unit = synchronized { - if (executorIdToContainer.contains(eid)) { - pendingLossReasonRequests - .getOrElseUpdate(eid, new ArrayBuffer[RpcCallContext]) += context - } else { - logWarning(s"Tried to get the loss reason for non-existent executor $eid") - } - } - - private def internalReleaseContainer(container: Container): Unit = { - releasedContainers.add(container.getId()) - amClient.releaseAssignedContainer(container.getId()) - } - - private[yarn] def getNumUnexpectedContainerRelease = numUnexpectedContainerRelease - - private[yarn] def getNumPendingLossReasonRequests: Int = synchronized { - pendingLossReasonRequests.size - } - - /** - * Split the pending container requests into 3 groups based on current localities of pending - * tasks. - * @param hostToLocalTaskCount a map of preferred hostname to possible task counts to be used as - * container placement hint. - * @param pendingAllocations A sequence of pending allocation container request. - * @return A tuple of 3 sequences, first is a sequence of locality matched container - * requests, second is a sequence of locality unmatched container requests, and third is a - * sequence of locality free container requests. - */ - private def splitPendingAllocationsByLocality( - hostToLocalTaskCount: Map[String, Int], - pendingAllocations: Seq[ContainerRequest] - ): (Seq[ContainerRequest], Seq[ContainerRequest], Seq[ContainerRequest]) = { - val localityMatched = ArrayBuffer[ContainerRequest]() - val localityUnMatched = ArrayBuffer[ContainerRequest]() - val localityFree = ArrayBuffer[ContainerRequest]() - - val preferredHosts = hostToLocalTaskCount.keySet - pendingAllocations.foreach { cr => - val nodes = cr.getNodes - if (nodes == null) { - localityFree += cr - } else if (nodes.asScala.toSet.intersect(preferredHosts).nonEmpty) { - localityMatched += cr - } else { - localityUnMatched += cr - } - } - - (localityMatched.toSeq, localityUnMatched.toSeq, localityFree.toSeq) - } - -} - -private object YarnAllocator { - val MEM_REGEX = "[0-9.]+ [KMG]B" - val PMEM_EXCEEDED_PATTERN = - Pattern.compile(s"$MEM_REGEX of $MEM_REGEX physical memory used") - val VMEM_EXCEEDED_PATTERN = - Pattern.compile(s"$MEM_REGEX of $MEM_REGEX virtual memory used") - val VMEM_EXCEEDED_EXIT_CODE = -103 - val PMEM_EXCEEDED_EXIT_CODE = -104 - - def memLimitExceededLogMessage(diagnostics: String, pattern: Pattern): String = { - val matcher = pattern.matcher(diagnostics) - val diag = if (matcher.find()) " " + matcher.group() + "." else "" - ("Container killed by YARN for exceeding memory limits." + diag - + " Consider boosting spark.yarn.executor.memoryOverhead.") - } -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala deleted file mode 100644 index d2a211f6711f..000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.util.{List => JList} - -import scala.collection.JavaConverters._ -import scala.collection.{Map, Set} -import scala.util.Try - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.api.AMRMClient -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.webapp.util.WebAppUtils - -import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.scheduler.SplitInfo -import org.apache.spark.util.Utils - -/** - * Handles registering and unregistering the application with the YARN ResourceManager. - */ -private[spark] class YarnRMClient(args: ApplicationMasterArguments) extends Logging { - - private var amClient: AMRMClient[ContainerRequest] = _ - private var uiHistoryAddress: String = _ - private var registered: Boolean = false - - /** - * Registers the application master with the RM. - * - * @param conf The Yarn configuration. - * @param sparkConf The Spark configuration. - * @param uiAddress Address of the SparkUI. - * @param uiHistoryAddress Address of the application on the History Server. - */ - def register( - driverUrl: String, - driverRef: RpcEndpointRef, - conf: YarnConfiguration, - sparkConf: SparkConf, - uiAddress: String, - uiHistoryAddress: String, - securityMgr: SecurityManager - ): YarnAllocator = { - amClient = AMRMClient.createAMRMClient() - amClient.init(conf) - amClient.start() - this.uiHistoryAddress = uiHistoryAddress - - logInfo("Registering the ApplicationMaster") - synchronized { - amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) - registered = true - } - new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, getAttemptId(), args, - securityMgr) - } - - /** - * Unregister the AM. Guaranteed to only be called once. - * - * @param status The final status of the AM. - * @param diagnostics Diagnostics message to include in the final status. - */ - def unregister(status: FinalApplicationStatus, diagnostics: String = ""): Unit = synchronized { - if (registered) { - amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) - } - } - - /** Returns the attempt ID. */ - def getAttemptId(): ApplicationAttemptId = { - YarnSparkHadoopUtil.get.getContainerId.getApplicationAttemptId() - } - - /** Returns the configuration for the AmIpFilter to add to the Spark UI. */ - def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String): Map[String, String] = { - // Figure out which scheme Yarn is using. Note the method seems to have been added after 2.2, - // so not all stable releases have it. - val prefix = Try(classOf[WebAppUtils].getMethod("getHttpSchemePrefix", classOf[Configuration]) - .invoke(null, conf).asInstanceOf[String]).getOrElse("http://") - - // If running a new enough Yarn, use the HA-aware API for retrieving the RM addresses. - try { - val method = classOf[WebAppUtils].getMethod("getProxyHostsAndPortsForAmFilter", - classOf[Configuration]) - val proxies = method.invoke(null, conf).asInstanceOf[JList[String]] - val hosts = proxies.asScala.map { proxy => proxy.split(":")(0) } - val uriBases = proxies.asScala.map { proxy => prefix + proxy + proxyBase } - Map("PROXY_HOSTS" -> hosts.mkString(","), "PROXY_URI_BASES" -> uriBases.mkString(",")) - } catch { - case e: NoSuchMethodException => - val proxy = WebAppUtils.getProxyHostAndPort(conf) - val parts = proxy.split(":") - val uriBase = prefix + proxy + proxyBase - Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase) - } - } - - /** Returns the maximum number of attempts to register the AM. */ - def getMaxRegAttempts(sparkConf: SparkConf, yarnConf: YarnConfiguration): Int = { - val sparkMaxAttempts = sparkConf.getOption("spark.yarn.maxAppAttempts").map(_.toInt) - val yarnMaxAttempts = yarnConf.getInt( - YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) - val retval: Int = sparkMaxAttempts match { - case Some(x) => if (x <= yarnMaxAttempts) x else yarnMaxAttempts - case None => yarnMaxAttempts - } - - retval - } - -} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala deleted file mode 100644 index 561ad79ee022..000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ /dev/null @@ -1,415 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.io.File -import java.nio.charset.StandardCharsets.UTF_8 -import java.util.regex.Matcher -import java.util.regex.Pattern - -import scala.collection.mutable.HashMap -import scala.reflect.runtime._ -import scala.util.Try - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier -import org.apache.hadoop.io.Text -import org.apache.hadoop.mapred.{Master, JobConf} -import org.apache.hadoop.security.Credentials -import org.apache.hadoop.security.UserGroupInformation -import org.apache.hadoop.security.token.Token -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.api.ApplicationConstants -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import org.apache.hadoop.yarn.api.records.{ApplicationAccessType, ContainerId, Priority} -import org.apache.hadoop.yarn.util.ConverterUtils - -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.launcher.YarnCommandBuilderUtils -import org.apache.spark.{SecurityManager, SparkConf, SparkException} -import org.apache.spark.util.Utils - -/** - * Contains util methods to interact with Hadoop from spark. - */ -class YarnSparkHadoopUtil extends SparkHadoopUtil { - - private var tokenRenewer: Option[ExecutorDelegationTokenUpdater] = None - - override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { - dest.addCredentials(source.getCredentials()) - } - - // Note that all params which start with SPARK are propagated all the way through, so if in yarn - // mode, this MUST be set to true. - override def isYarnMode(): Boolean = { true } - - // Return an appropriate (subclass) of Configuration. Creating a config initializes some Hadoop - // subsystems. Always create a new config, dont reuse yarnConf. - override def newConfiguration(conf: SparkConf): Configuration = - new YarnConfiguration(super.newConfiguration(conf)) - - // Add any user credentials to the job conf which are necessary for running on a secure Hadoop - // cluster - override def addCredentials(conf: JobConf) { - val jobCreds = conf.getCredentials() - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) - } - - override def getCurrentUserCredentials(): Credentials = { - UserGroupInformation.getCurrentUser().getCredentials() - } - - override def addCurrentUserCredentials(creds: Credentials) { - UserGroupInformation.getCurrentUser().addCredentials(creds) - } - - override def addSecretKeyToUserCredentials(key: String, secret: String) { - val creds = new Credentials() - creds.addSecretKey(new Text(key), secret.getBytes(UTF_8)) - addCurrentUserCredentials(creds) - } - - override def getSecretKeyFromUserCredentials(key: String): Array[Byte] = { - val credentials = getCurrentUserCredentials() - if (credentials != null) credentials.getSecretKey(new Text(key)) else null - } - - /** - * Get the list of namenodes the user may access. - */ - def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { - sparkConf.get("spark.yarn.access.namenodes", "") - .split(",") - .map(_.trim()) - .filter(!_.isEmpty) - .map(new Path(_)) - .toSet - } - - def getTokenRenewer(conf: Configuration): String = { - val delegTokenRenewer = Master.getMasterPrincipal(conf) - logDebug("delegation token renewer is: " + delegTokenRenewer) - if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { - val errorMessage = "Can't get Master Kerberos principal for use as renewer" - logError(errorMessage) - throw new SparkException(errorMessage) - } - delegTokenRenewer - } - - /** - * Obtains tokens for the namenodes passed in and adds them to the credentials. - */ - def obtainTokensForNamenodes( - paths: Set[Path], - conf: Configuration, - creds: Credentials, - renewer: Option[String] = None - ): Unit = { - if (UserGroupInformation.isSecurityEnabled()) { - val delegTokenRenewer = renewer.getOrElse(getTokenRenewer(conf)) - paths.foreach { dst => - val dstFs = dst.getFileSystem(conf) - logInfo("getting token for namenode: " + dst) - dstFs.addDelegationTokens(delegTokenRenewer, creds) - } - } - } - - private[spark] override def startExecutorDelegationTokenRenewer(sparkConf: SparkConf): Unit = { - tokenRenewer = Some(new ExecutorDelegationTokenUpdater(sparkConf, conf)) - tokenRenewer.get.updateCredentialsIfRequired() - } - - private[spark] override def stopExecutorDelegationTokenRenewer(): Unit = { - tokenRenewer.foreach(_.stop()) - } - - private[spark] def getContainerId: ContainerId = { - val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) - ConverterUtils.toContainerId(containerIdString) - } - - /** - * Obtains token for the Hive metastore, using the current user as the principal. - * Some exceptions are caught and downgraded to a log message. - * @param conf hadoop configuration; the Hive configuration will be based on this - * @return a token, or `None` if there's no need for a token (no metastore URI or principal - * in the config), or if a binding exception was caught and downgraded. - */ - def obtainTokenForHiveMetastore(conf: Configuration): Option[Token[DelegationTokenIdentifier]] = { - try { - obtainTokenForHiveMetastoreInner(conf, UserGroupInformation.getCurrentUser().getUserName) - } catch { - case e: ClassNotFoundException => - logInfo(s"Hive class not found $e") - logDebug("Hive class not found", e) - None - } - } - - /** - * Inner routine to obtains token for the Hive metastore; exceptions are raised on any problem. - * @param conf hadoop configuration; the Hive configuration will be based on this. - * @param username the username of the principal requesting the delegating token. - * @return a delegation token - */ - private[yarn] def obtainTokenForHiveMetastoreInner(conf: Configuration, - username: String): Option[Token[DelegationTokenIdentifier]] = { - val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader) - - // the hive configuration class is a subclass of Hadoop Configuration, so can be cast down - // to a Configuration and used without reflection - val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf") - // using the (Configuration, Class) constructor allows the current configuratin to be included - // in the hive config. - val ctor = hiveConfClass.getDeclaredConstructor(classOf[Configuration], - classOf[Object].getClass) - val hiveConf = ctor.newInstance(conf, hiveConfClass).asInstanceOf[Configuration] - val metastoreUri = hiveConf.getTrimmed("hive.metastore.uris", "") - - // Check for local metastore - if (metastoreUri.nonEmpty) { - require(username.nonEmpty, "Username undefined") - val principalKey = "hive.metastore.kerberos.principal" - val principal = hiveConf.getTrimmed(principalKey, "") - require(principal.nonEmpty, "Hive principal $principalKey undefined") - logDebug(s"Getting Hive delegation token for $username against $principal at $metastoreUri") - val hiveClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.ql.metadata.Hive") - val closeCurrent = hiveClass.getMethod("closeCurrent") - try { - // get all the instance methods before invoking any - val getDelegationToken = hiveClass.getMethod("getDelegationToken", - classOf[String], classOf[String]) - val getHive = hiveClass.getMethod("get", hiveConfClass) - - // invoke - val hive = getHive.invoke(null, hiveConf) - val tokenStr = getDelegationToken.invoke(hive, username, principal).asInstanceOf[String] - val hive2Token = new Token[DelegationTokenIdentifier]() - hive2Token.decodeFromUrlString(tokenStr) - Some(hive2Token) - } finally { - Utils.tryLogNonFatalError { - closeCurrent.invoke(null) - } - } - } else { - logDebug("HiveMetaStore configured in localmode") - None - } - } -} - -object YarnSparkHadoopUtil { - // Additional memory overhead - // 10% was arrived at experimentally. In the interest of minimizing memory waste while covering - // the common cases. Memory overhead tends to grow with container size. - - val MEMORY_OVERHEAD_FACTOR = 0.10 - val MEMORY_OVERHEAD_MIN = 384 - - val ANY_HOST = "*" - - val DEFAULT_NUMBER_EXECUTORS = 2 - - // All RM requests are issued with same priority : we do not (yet) have any distinction between - // request types (like map/reduce in hadoop for example) - val RM_REQUEST_PRIORITY = Priority.newInstance(1) - - def get: YarnSparkHadoopUtil = { - val yarnMode = java.lang.Boolean.valueOf( - System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) - if (!yarnMode) { - throw new SparkException("YarnSparkHadoopUtil is not available in non-YARN mode!") - } - SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil] - } - /** - * Add a path variable to the given environment map. - * If the map already contains this key, append the value to the existing value instead. - */ - def addPathToEnvironment(env: HashMap[String, String], key: String, value: String): Unit = { - val newValue = if (env.contains(key)) { env(key) + getClassPathSeparator + value } else value - env.put(key, newValue) - } - - /** - * Set zero or more environment variables specified by the given input string. - * The input string is expected to take the form "KEY1=VAL1,KEY2=VAL2,KEY3=VAL3". - */ - def setEnvFromInputString(env: HashMap[String, String], inputString: String): Unit = { - if (inputString != null && inputString.length() > 0) { - val childEnvs = inputString.split(",") - val p = Pattern.compile(environmentVariableRegex) - for (cEnv <- childEnvs) { - val parts = cEnv.split("=") // split on '=' - val m = p.matcher(parts(1)) - val sb = new StringBuffer - while (m.find()) { - val variable = m.group(1) - var replace = "" - if (env.get(variable) != None) { - replace = env.get(variable).get - } else { - // if this key is not configured for the child .. get it from the env - replace = System.getenv(variable) - if (replace == null) { - // the env key is note present anywhere .. simply set it - replace = "" - } - } - m.appendReplacement(sb, Matcher.quoteReplacement(replace)) - } - m.appendTail(sb) - // This treats the environment variable as path variable delimited by `File.pathSeparator` - // This is kept for backward compatibility and consistency with Hadoop's behavior - addPathToEnvironment(env, parts(0), sb.toString) - } - } - } - - private val environmentVariableRegex: String = { - if (Utils.isWindows) { - "%([A-Za-z_][A-Za-z0-9_]*?)%" - } else { - "\\$([A-Za-z_][A-Za-z0-9_]*)" - } - } - - /** - * The handler if an OOM Exception is thrown by the JVM must be configured on Windows - * differently: the 'taskkill' command should be used, whereas Unix-based systems use 'kill'. - * - * As the JVM interprets both %p and %%p as the same, we can use either of them. However, - * some tests on Windows computers suggest, that the JVM only accepts '%%p'. - * - * Furthermore, the behavior of the character '%' on the Windows command line differs from - * the behavior of '%' in a .cmd file: it gets interpreted as an incomplete environment - * variable. Windows .cmd files escape a '%' by '%%'. Thus, the correct way of writing - * '%%p' in an escaped way is '%%%%p'. - * - * @return The correct OOM Error handler JVM option, platform dependent. - */ - def getOutOfMemoryErrorArgument : String = { - if (Utils.isWindows) { - escapeForShell("-XX:OnOutOfMemoryError=taskkill /F /PID %%%%p") - } else { - "-XX:OnOutOfMemoryError='kill %p'" - } - } - - /** - * Escapes a string for inclusion in a command line executed by Yarn. Yarn executes commands - * using either - * - * (Unix-based) `bash -c "command arg1 arg2"` and that means plain quoting doesn't really work. - * The argument is enclosed in single quotes and some key characters are escaped. - * - * (Windows-based) part of a .cmd file in which case windows escaping for each argument must be - * applied. Windows is quite lenient, however it is usually Java that causes trouble, needing to - * distinguish between arguments starting with '-' and class names. If arguments are surrounded - * by ' java takes the following string as is, hence an argument is mistakenly taken as a class - * name which happens to start with a '-'. The way to avoid this, is to surround nothing with - * a ', but instead with a ". - * - * @param arg A single argument. - * @return Argument quoted for execution via Yarn's generated shell script. - */ - def escapeForShell(arg: String): String = { - if (arg != null) { - if (Utils.isWindows) { - YarnCommandBuilderUtils.quoteForBatchScript(arg) - } else { - val escaped = new StringBuilder("'") - for (i <- 0 to arg.length() - 1) { - arg.charAt(i) match { - case '$' => escaped.append("\\$") - case '"' => escaped.append("\\\"") - case '\'' => escaped.append("'\\''") - case c => escaped.append(c) - } - } - escaped.append("'").toString() - } - } else { - arg - } - } - - def getApplicationAclsForYarn(securityMgr: SecurityManager) - : Map[ApplicationAccessType, String] = { - Map[ApplicationAccessType, String] ( - ApplicationAccessType.VIEW_APP -> securityMgr.getViewAcls, - ApplicationAccessType.MODIFY_APP -> securityMgr.getModifyAcls - ) - } - - /** - * Expand environment variable using Yarn API. - * If environment.$$() is implemented, return the result of it. - * Otherwise, return the result of environment.$() - * Note: $$() is added in Hadoop 2.4. - */ - private lazy val expandMethod = - Try(classOf[Environment].getMethod("$$")) - .getOrElse(classOf[Environment].getMethod("$")) - - def expandEnvironment(environment: Environment): String = - expandMethod.invoke(environment).asInstanceOf[String] - - /** - * Get class path separator using Yarn API. - * If ApplicationConstants.CLASS_PATH_SEPARATOR is implemented, return it. - * Otherwise, return File.pathSeparator - * Note: CLASS_PATH_SEPARATOR is added in Hadoop 2.4. - */ - private lazy val classPathSeparatorField = - Try(classOf[ApplicationConstants].getField("CLASS_PATH_SEPARATOR")) - .getOrElse(classOf[File].getField("pathSeparator")) - - def getClassPathSeparator(): String = { - classPathSeparatorField.get(null).asInstanceOf[String] - } - - /** - * Getting the initial target number of executors depends on whether dynamic allocation is - * enabled. - */ - def getInitialTargetExecutorNumber(conf: SparkConf): Int = { - if (Utils.isDynamicAllocationEnabled(conf)) { - val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", 0) - val initialNumExecutors = - conf.getInt("spark.dynamicAllocation.initialExecutors", minNumExecutors) - val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", Int.MaxValue) - require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors, - s"initial executor number $initialNumExecutors must between min executor number" + - s"$minNumExecutors and max executor number $maxNumExecutors") - - initialNumExecutors - } else { - val targetNumExecutors = - sys.env.get("SPARK_EXECUTOR_INSTANCES").map(_.toInt).getOrElse(DEFAULT_NUMBER_EXECUTORS) - // System property can override environment variable. - conf.getInt("spark.executor.instances", targetNumExecutors) - } - } -} - diff --git a/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala b/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala deleted file mode 100644 index 7d246bf40712..000000000000 --- a/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala +++ /dev/null @@ -1,44 +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.launcher - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ListBuffer - -/** - * Exposes methods from the launcher library that are used by the YARN backend. - */ -private[spark] object YarnCommandBuilderUtils { - - def quoteForBatchScript(arg: String): String = { - CommandBuilderUtils.quoteForBatchScript(arg) - } - - /** - * Adds the perm gen configuration to the list of java options if needed and not yet added. - * - * Note that this method adds the option based on the local JVM version; if the node where - * the container is running has a different Java version, there's a risk that the option will - * not be added (e.g. if the AM is running Java 8 but the container's node is set up to use - * Java 7). - */ - def addPermGenSizeOpt(args: ListBuffer[String]): Unit = { - CommandBuilderUtils.addPermGenSizeOpt(args.asJava) - } - -} diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala deleted file mode 100644 index 20771f655473..000000000000 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ /dev/null @@ -1,202 +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.scheduler.cluster - -import scala.collection.mutable.ArrayBuffer - -import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} - -import org.apache.spark.{SparkException, Logging, SparkContext} -import org.apache.spark.deploy.yarn.{Client, ClientArguments, YarnSparkHadoopUtil} -import org.apache.spark.launcher.SparkAppHandle -import org.apache.spark.scheduler.TaskSchedulerImpl - -private[spark] class YarnClientSchedulerBackend( - scheduler: TaskSchedulerImpl, - sc: SparkContext) - extends YarnSchedulerBackend(scheduler, sc) - with Logging { - - private var client: Client = null - private var appId: ApplicationId = null - private var monitorThread: MonitorThread = null - - /** - * Create a Yarn client to submit an application to the ResourceManager. - * This waits until the application is running. - */ - override def start() { - val driverHost = conf.get("spark.driver.host") - val driverPort = conf.get("spark.driver.port") - val hostport = driverHost + ":" + driverPort - sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.appUIAddress) } - - val argsArrayBuf = new ArrayBuffer[String]() - argsArrayBuf += ("--arg", hostport) - argsArrayBuf ++= getExtraClientArguments - - logDebug("ClientArguments called with: " + argsArrayBuf.mkString(" ")) - val args = new ClientArguments(argsArrayBuf.toArray, conf) - totalExpectedExecutors = args.numExecutors - client = new Client(args, conf) - appId = client.submitApplication() - - // SPARK-8687: Ensure all necessary properties have already been set before - // we initialize our driver scheduler backend, which serves these properties - // to the executors - super.start() - - waitForApplication() - - // SPARK-8851: In yarn-client mode, the AM still does the credentials refresh. The driver - // reads the credentials from HDFS, just like the executors and updates its own credentials - // cache. - if (conf.contains("spark.yarn.credentials.file")) { - YarnSparkHadoopUtil.get.startExecutorDelegationTokenRenewer(conf) - } - monitorThread = asyncMonitorApplication() - monitorThread.start() - } - - /** - * Return any extra command line arguments to be passed to Client provided in the form of - * environment variables or Spark properties. - */ - private def getExtraClientArguments: Seq[String] = { - val extraArgs = new ArrayBuffer[String] - // List of (target Client argument, environment variable, Spark property) - val optionTuples = - List( - ("--executor-memory", "SPARK_WORKER_MEMORY", "spark.executor.memory"), - ("--executor-memory", "SPARK_EXECUTOR_MEMORY", "spark.executor.memory"), - ("--executor-cores", "SPARK_WORKER_CORES", "spark.executor.cores"), - ("--executor-cores", "SPARK_EXECUTOR_CORES", "spark.executor.cores"), - ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue"), - ("--py-files", null, "spark.submit.pyFiles") - ) - // Warn against the following deprecated environment variables: env var -> suggestion - val deprecatedEnvVars = Map( - "SPARK_WORKER_MEMORY" -> "SPARK_EXECUTOR_MEMORY or --executor-memory through spark-submit", - "SPARK_WORKER_CORES" -> "SPARK_EXECUTOR_CORES or --executor-cores through spark-submit") - optionTuples.foreach { case (optionName, envVar, sparkProp) => - if (sc.getConf.contains(sparkProp)) { - extraArgs += (optionName, sc.getConf.get(sparkProp)) - } else if (envVar != null && System.getenv(envVar) != null) { - extraArgs += (optionName, System.getenv(envVar)) - if (deprecatedEnvVars.contains(envVar)) { - logWarning(s"NOTE: $envVar is deprecated. Use ${deprecatedEnvVars(envVar)} instead.") - } - } - } - // The app name is a special case because "spark.app.name" is required of all applications. - // As a result, the corresponding "SPARK_YARN_APP_NAME" is already handled preemptively in - // SparkSubmitArguments if "spark.app.name" is not explicitly set by the user. (SPARK-5222) - sc.getConf.getOption("spark.app.name").foreach(v => extraArgs += ("--name", v)) - extraArgs - } - - /** - * Report the state of the application until it is running. - * If the application has finished, failed or been killed in the process, throw an exception. - * This assumes both `client` and `appId` have already been set. - */ - private def waitForApplication(): Unit = { - assert(client != null && appId != null, "Application has not been submitted yet!") - val (state, _) = client.monitorApplication(appId, returnOnRunning = true) // blocking - if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.FAILED || - state == YarnApplicationState.KILLED) { - throw new SparkException("Yarn application has already ended! " + - "It might have been killed or unable to launch application master.") - } - if (state == YarnApplicationState.RUNNING) { - logInfo(s"Application $appId has started running.") - } - } - - /** - * We create this class for SPARK-9519. Basically when we interrupt the monitor thread it's - * because the SparkContext is being shut down(sc.stop() called by user code), but if - * monitorApplication return, it means the Yarn application finished before sc.stop() was called, - * which means we should call sc.stop() here, and we don't allow the monitor to be interrupted - * before SparkContext stops successfully. - */ - private class MonitorThread extends Thread { - private var allowInterrupt = true - - override def run() { - try { - val (state, _) = client.monitorApplication(appId, logApplicationReport = false) - logError(s"Yarn application has already exited with state $state!") - allowInterrupt = false - sc.stop() - } catch { - case e: InterruptedException => logInfo("Interrupting monitor thread") - } - } - - def stopMonitor(): Unit = { - if (allowInterrupt) { - this.interrupt() - } - } - } - - /** - * Monitor the application state in a separate thread. - * If the application has exited for any reason, stop the SparkContext. - * This assumes both `client` and `appId` have already been set. - */ - private def asyncMonitorApplication(): MonitorThread = { - assert(client != null && appId != null, "Application has not been submitted yet!") - val t = new MonitorThread - t.setName("Yarn application state monitor") - t.setDaemon(true) - t - } - - /** - * Stop the scheduler. This assumes `start()` has already been called. - */ - override def stop() { - assert(client != null, "Attempted to stop this scheduler before starting it!") - if (monitorThread != null) { - monitorThread.stopMonitor() - } - - // Report a final state to the launcher if one is connected. This is needed since in client - // mode this backend doesn't let the app monitor loop run to completion, so it does not report - // the final state itself. - // - // Note: there's not enough information at this point to provide a better final state, - // so assume the application was successful. - client.reportLauncherState(SparkAppHandle.State.FINISHED) - - super.stop() - YarnSparkHadoopUtil.get.stopExecutorDelegationTokenRenewer() - client.stop() - logInfo("Stopped") - } - - override def applicationId(): String = { - Option(appId).map(_.toString).getOrElse { - logWarning("Application ID is not initialized yet.") - super.applicationId - } - } -} diff --git a/yarn/src/test/resources/log4j.properties b/yarn/src/test/resources/log4j.properties deleted file mode 100644 index 6b9a799954bf..000000000000 --- a/yarn/src/test/resources/log4j.properties +++ /dev/null @@ -1,31 +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. -# - -# Set everything to be logged to the file target/unit-tests.log -log4j.rootCategory=DEBUG, file -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=true -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Ignore messages below warning level from a few verbose libraries. -log4j.logger.com.sun.jersey=WARN -log4j.logger.org.apache.hadoop=WARN -log4j.logger.org.eclipse.jetty=WARN -log4j.logger.org.mortbay=WARN -log4j.logger.org.spark-project.jetty=WARN diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala deleted file mode 100644 index 804dfecde786..000000000000 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala +++ /dev/null @@ -1,221 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.net.URI - -import org.scalatest.mock.MockitoSugar -import org.mockito.Mockito.when - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileStatus -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.fs.Path -import org.apache.hadoop.fs.permission.FsAction -import org.apache.hadoop.yarn.api.records.LocalResource -import org.apache.hadoop.yarn.api.records.LocalResourceVisibility -import org.apache.hadoop.yarn.api.records.LocalResourceType -import org.apache.hadoop.yarn.util.{Records, ConverterUtils} - -import scala.collection.mutable.HashMap -import scala.collection.mutable.Map - -import org.apache.spark.SparkFunSuite - - -class ClientDistributedCacheManagerSuite extends SparkFunSuite with MockitoSugar { - - class MockClientDistributedCacheManager extends ClientDistributedCacheManager { - override def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): - LocalResourceVisibility = { - LocalResourceVisibility.PRIVATE - } - } - - test("test getFileStatus empty") { - val distMgr = new ClientDistributedCacheManager() - val fs = mock[FileSystem] - val uri = new URI("/tmp/testing") - when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus()) - val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - val stat = distMgr.getFileStatus(fs, uri, statCache) - assert(stat.getPath() === null) - } - - test("test getFileStatus cached") { - val distMgr = new ClientDistributedCacheManager() - val fs = mock[FileSystem] - val uri = new URI("/tmp/testing") - val realFileStatus = new FileStatus(10, false, 1, 1024, 10, 10, null, "testOwner", - null, new Path("/tmp/testing")) - when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus()) - val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus](uri -> realFileStatus) - val stat = distMgr.getFileStatus(fs, uri, statCache) - assert(stat.getPath().toString() === "/tmp/testing") - } - - test("test addResource") { - val distMgr = new MockClientDistributedCacheManager() - val fs = mock[FileSystem] - val conf = new Configuration() - val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") - val localResources = HashMap[String, LocalResource]() - val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - when(fs.getFileStatus(destPath)).thenReturn(new FileStatus()) - - distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, "link", - statCache, false) - val resource = localResources("link") - assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE) - assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath) - assert(resource.getTimestamp() === 0) - assert(resource.getSize() === 0) - assert(resource.getType() === LocalResourceType.FILE) - - val env = new HashMap[String, String]() - distMgr.setDistFilesEnv(env) - assert(env("SPARK_YARN_CACHE_FILES") === "file:/foo.invalid.com:8080/tmp/testing#link") - assert(env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === "0") - assert(env("SPARK_YARN_CACHE_FILES_FILE_SIZES") === "0") - assert(env("SPARK_YARN_CACHE_FILES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name()) - - distMgr.setDistArchivesEnv(env) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None) - - // add another one and verify both there and order correct - val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", - null, new Path("/tmp/testing2")) - val destPath2 = new Path("file:///foo.invalid.com:8080/tmp/testing2") - when(fs.getFileStatus(destPath2)).thenReturn(realFileStatus) - distMgr.addResource(fs, conf, destPath2, localResources, LocalResourceType.FILE, "link2", - statCache, false) - val resource2 = localResources("link2") - assert(resource2.getVisibility() === LocalResourceVisibility.PRIVATE) - assert(ConverterUtils.getPathFromYarnURL(resource2.getResource()) === destPath2) - assert(resource2.getTimestamp() === 10) - assert(resource2.getSize() === 20) - assert(resource2.getType() === LocalResourceType.FILE) - - val env2 = new HashMap[String, String]() - distMgr.setDistFilesEnv(env2) - val timestamps = env2("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') - val files = env2("SPARK_YARN_CACHE_FILES").split(',') - val sizes = env2("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') - val visibilities = env2("SPARK_YARN_CACHE_FILES_VISIBILITIES") .split(',') - assert(files(0) === "file:/foo.invalid.com:8080/tmp/testing#link") - assert(timestamps(0) === "0") - assert(sizes(0) === "0") - assert(visibilities(0) === LocalResourceVisibility.PRIVATE.name()) - - assert(files(1) === "file:/foo.invalid.com:8080/tmp/testing2#link2") - assert(timestamps(1) === "10") - assert(sizes(1) === "20") - assert(visibilities(1) === LocalResourceVisibility.PRIVATE.name()) - } - - test("test addResource link null") { - val distMgr = new MockClientDistributedCacheManager() - val fs = mock[FileSystem] - val conf = new Configuration() - val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") - val localResources = HashMap[String, LocalResource]() - val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - when(fs.getFileStatus(destPath)).thenReturn(new FileStatus()) - - intercept[Exception] { - distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, null, - statCache, false) - } - assert(localResources.get("link") === None) - assert(localResources.size === 0) - } - - test("test addResource appmaster only") { - val distMgr = new MockClientDistributedCacheManager() - val fs = mock[FileSystem] - val conf = new Configuration() - val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") - val localResources = HashMap[String, LocalResource]() - val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", - null, new Path("/tmp/testing")) - when(fs.getFileStatus(destPath)).thenReturn(realFileStatus) - - distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", - statCache, true) - val resource = localResources("link") - assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE) - assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath) - assert(resource.getTimestamp() === 10) - assert(resource.getSize() === 20) - assert(resource.getType() === LocalResourceType.ARCHIVE) - - val env = new HashMap[String, String]() - distMgr.setDistFilesEnv(env) - assert(env.get("SPARK_YARN_CACHE_FILES") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None) - - distMgr.setDistArchivesEnv(env) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None) - } - - test("test addResource archive") { - val distMgr = new MockClientDistributedCacheManager() - val fs = mock[FileSystem] - val conf = new Configuration() - val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") - val localResources = HashMap[String, LocalResource]() - val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", - null, new Path("/tmp/testing")) - when(fs.getFileStatus(destPath)).thenReturn(realFileStatus) - - distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", - statCache, false) - val resource = localResources("link") - assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE) - assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath) - assert(resource.getTimestamp() === 10) - assert(resource.getSize() === 20) - assert(resource.getType() === LocalResourceType.ARCHIVE) - - val env = new HashMap[String, String]() - - distMgr.setDistArchivesEnv(env) - assert(env("SPARK_YARN_CACHE_ARCHIVES") === "file:/foo.invalid.com:8080/tmp/testing#link") - assert(env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === "10") - assert(env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === "20") - assert(env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name()) - - distMgr.setDistFilesEnv(env) - assert(env.get("SPARK_YARN_CACHE_FILES") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None) - } - - -} diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala deleted file mode 100644 index e7f2501e7899..000000000000 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ /dev/null @@ -1,269 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.io.File -import java.net.URI - -import scala.collection.JavaConverters._ -import scala.collection.mutable.{HashMap => MutableHashMap} -import scala.reflect.ClassTag -import scala.util.Try - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce.MRJobConfig -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.api.YarnClientApplication -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.util.Records -import org.mockito.Matchers._ -import org.mockito.Mockito._ -import org.scalatest.{BeforeAndAfterAll, Matchers} - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.util.Utils - -class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll { - - override def beforeAll(): Unit = { - System.setProperty("SPARK_YARN_MODE", "true") - } - - override def afterAll(): Unit = { - System.clearProperty("SPARK_YARN_MODE") - } - - test("default Yarn application classpath") { - Client.getDefaultYarnApplicationClasspath should be(Some(Fixtures.knownDefYarnAppCP)) - } - - test("default MR application classpath") { - Client.getDefaultMRApplicationClasspath should be(Some(Fixtures.knownDefMRAppCP)) - } - - test("resultant classpath for an application that defines a classpath for YARN") { - withAppConf(Fixtures.mapYARNAppConf) { conf => - val env = newEnv - Client.populateHadoopClasspath(conf, env) - classpath(env) should be( - flatten(Fixtures.knownYARNAppCP, Client.getDefaultMRApplicationClasspath)) - } - } - - test("resultant classpath for an application that defines a classpath for MR") { - withAppConf(Fixtures.mapMRAppConf) { conf => - val env = newEnv - Client.populateHadoopClasspath(conf, env) - classpath(env) should be( - flatten(Client.getDefaultYarnApplicationClasspath, Fixtures.knownMRAppCP)) - } - } - - test("resultant classpath for an application that defines both classpaths, YARN and MR") { - withAppConf(Fixtures.mapAppConf) { conf => - val env = newEnv - Client.populateHadoopClasspath(conf, env) - classpath(env) should be(flatten(Fixtures.knownYARNAppCP, Fixtures.knownMRAppCP)) - } - } - - private val SPARK = "local:/sparkJar" - private val USER = "local:/userJar" - private val ADDED = "local:/addJar1,local:/addJar2,/addJar3" - - test("Local jar URIs") { - val conf = new Configuration() - val sparkConf = new SparkConf().set(Client.CONF_SPARK_JAR, SPARK) - .set("spark.yarn.user.classpath.first", "true") - val env = new MutableHashMap[String, String]() - val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) - - Client.populateClasspath(args, conf, sparkConf, env, true) - - val cp = env("CLASSPATH").split(":|;|") - s"$SPARK,$USER,$ADDED".split(",").foreach({ entry => - val uri = new URI(entry) - if (Client.LOCAL_SCHEME.equals(uri.getScheme())) { - cp should contain (uri.getPath()) - } else { - cp should not contain (uri.getPath()) - } - }) - val pwdVar = - if (classOf[Environment].getMethods().exists(_.getName == "$$")) { - "{{PWD}}" - } else if (Utils.isWindows) { - "%PWD%" - } else { - Environment.PWD.$() - } - cp should contain(pwdVar) - cp should contain (s"$pwdVar${Path.SEPARATOR}${Client.LOCALIZED_CONF_DIR}") - cp should not contain (Client.SPARK_JAR) - cp should not contain (Client.APP_JAR) - } - - test("Jar path propagation through SparkConf") { - val conf = new Configuration() - val sparkConf = new SparkConf().set(Client.CONF_SPARK_JAR, SPARK) - val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) - - val client = spy(new Client(args, conf, sparkConf)) - doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), - any(classOf[Path]), anyShort()) - - val tempDir = Utils.createTempDir() - try { - client.prepareLocalResources(tempDir.getAbsolutePath(), Nil) - sparkConf.getOption(Client.CONF_SPARK_USER_JAR) should be (Some(USER)) - - // The non-local path should be propagated by name only, since it will end up in the app's - // staging dir. - val expected = ADDED.split(",") - .map(p => { - val uri = new URI(p) - if (Client.LOCAL_SCHEME == uri.getScheme()) { - p - } else { - Option(uri.getFragment()).getOrElse(new File(p).getName()) - } - }) - .mkString(",") - - sparkConf.getOption(Client.CONF_SPARK_YARN_SECONDARY_JARS) should be (Some(expected)) - } finally { - Utils.deleteRecursively(tempDir) - } - } - - test("Cluster path translation") { - val conf = new Configuration() - val sparkConf = new SparkConf() - .set(Client.CONF_SPARK_JAR, "local:/localPath/spark.jar") - .set("spark.yarn.config.gatewayPath", "/localPath") - .set("spark.yarn.config.replacementPath", "/remotePath") - - Client.getClusterPath(sparkConf, "/localPath") should be ("/remotePath") - Client.getClusterPath(sparkConf, "/localPath/1:/localPath/2") should be ( - "/remotePath/1:/remotePath/2") - - val env = new MutableHashMap[String, String]() - Client.populateClasspath(null, conf, sparkConf, env, false, - extraClassPath = Some("/localPath/my1.jar")) - val cp = classpath(env) - cp should contain ("/remotePath/spark.jar") - cp should contain ("/remotePath/my1.jar") - } - - test("configuration and args propagate through createApplicationSubmissionContext") { - val conf = new Configuration() - // When parsing tags, duplicates and leading/trailing whitespace should be removed. - // Spaces between non-comma strings should be preserved as single tags. Empty strings may or - // may not be removed depending on the version of Hadoop being used. - val sparkConf = new SparkConf() - .set(Client.CONF_SPARK_YARN_APPLICATION_TAGS, ",tag1, dup,tag2 , ,multi word , dup") - .set("spark.yarn.maxAppAttempts", "42") - val args = new ClientArguments(Array( - "--name", "foo-test-app", - "--queue", "staging-queue"), sparkConf) - - val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) - val getNewApplicationResponse = Records.newRecord(classOf[GetNewApplicationResponse]) - val containerLaunchContext = Records.newRecord(classOf[ContainerLaunchContext]) - - val client = new Client(args, conf, sparkConf) - client.createApplicationSubmissionContext( - new YarnClientApplication(getNewApplicationResponse, appContext), - containerLaunchContext) - - appContext.getApplicationName should be ("foo-test-app") - appContext.getQueue should be ("staging-queue") - appContext.getAMContainerSpec should be (containerLaunchContext) - appContext.getApplicationType should be ("SPARK") - appContext.getClass.getMethods.filter(_.getName.equals("getApplicationTags")).foreach{ method => - val tags = method.invoke(appContext).asInstanceOf[java.util.Set[String]] - tags should contain allOf ("tag1", "dup", "tag2", "multi word") - tags.asScala.filter(_.nonEmpty).size should be (4) - } - appContext.getMaxAppAttempts should be (42) - } - - object Fixtures { - - val knownDefYarnAppCP: Seq[String] = - getFieldValue[Array[String], Seq[String]](classOf[YarnConfiguration], - "DEFAULT_YARN_APPLICATION_CLASSPATH", - Seq[String]())(a => a.toSeq) - - - val knownDefMRAppCP: Seq[String] = - getFieldValue2[String, Array[String], Seq[String]]( - classOf[MRJobConfig], - "DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH", - Seq[String]())(a => a.split(","))(a => a.toSeq) - - val knownYARNAppCP = Some(Seq("/known/yarn/path")) - - val knownMRAppCP = Some(Seq("/known/mr/path")) - - val mapMRAppConf = - Map("mapreduce.application.classpath" -> knownMRAppCP.map(_.mkString(":")).get) - - val mapYARNAppConf = - Map(YarnConfiguration.YARN_APPLICATION_CLASSPATH -> knownYARNAppCP.map(_.mkString(":")).get) - - val mapAppConf = mapYARNAppConf ++ mapMRAppConf - } - - def withAppConf(m: Map[String, String] = Map())(testCode: (Configuration) => Any) { - val conf = new Configuration - m.foreach { case (k, v) => conf.set(k, v, "ClientSpec") } - testCode(conf) - } - - def newEnv: MutableHashMap[String, String] = MutableHashMap[String, String]() - - def classpath(env: MutableHashMap[String, String]): Array[String] = - env(Environment.CLASSPATH.name).split(":|;|") - - def flatten(a: Option[Seq[String]], b: Option[Seq[String]]): Array[String] = - (a ++ b).flatten.toArray - - def getFieldValue[A, B](clazz: Class[_], field: String, defaults: => B)(mapTo: A => B): B = { - Try(clazz.getField(field)) - .map(_.get(null).asInstanceOf[A]) - .toOption - .map(mapTo) - .getOrElse(defaults) - } - - def getFieldValue2[A: ClassTag, A1: ClassTag, B]( - clazz: Class[_], - field: String, - defaults: => B)(mapTo: A => B)(mapTo1: A1 => B): B = { - Try(clazz.getField(field)).map(_.get(null)).map { - case v: A => mapTo(v) - case v1: A1 => mapTo1(v1) - case _ => defaults - }.toOption.getOrElse(defaults) - } - -} diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala deleted file mode 100644 index bd80036c5cfa..000000000000 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ /dev/null @@ -1,273 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.util.{Arrays, List => JList} - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.CommonConfigurationKeysPublic -import org.apache.hadoop.net.DNSToSwitchMapping -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.api.AMRMClient -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -import org.scalatest.{BeforeAndAfterEach, Matchers} - -import org.scalatest.{BeforeAndAfterEach, Matchers} -import org.mockito.Mockito._ - -import org.apache.spark.{SecurityManager, SparkFunSuite} -import org.apache.spark.SparkConf -import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ -import org.apache.spark.deploy.yarn.YarnAllocator._ -import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.scheduler.SplitInfo - -class MockResolver extends DNSToSwitchMapping { - - override def resolve(names: JList[String]): JList[String] = { - if (names.size > 0 && names.get(0) == "host3") Arrays.asList("/rack2") - else Arrays.asList("/rack1") - } - - override def reloadCachedMappings() {} - - def reloadCachedMappings(names: JList[String]) {} -} - -class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach { - val conf = new Configuration() - conf.setClass( - CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, - classOf[MockResolver], classOf[DNSToSwitchMapping]) - - val sparkConf = new SparkConf() - sparkConf.set("spark.driver.host", "localhost") - sparkConf.set("spark.driver.port", "4040") - sparkConf.set("spark.yarn.jar", "notarealjar.jar") - sparkConf.set("spark.yarn.launchContainers", "false") - - val appAttemptId = ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 0), 0) - - // Resource returned by YARN. YARN can give larger containers than requested, so give 6 cores - // instead of the 5 requested and 3 GB instead of the 2 requested. - val containerResource = Resource.newInstance(3072, 6) - - var rmClient: AMRMClient[ContainerRequest] = _ - - var containerNum = 0 - - override def beforeEach() { - rmClient = AMRMClient.createAMRMClient() - rmClient.init(conf) - rmClient.start() - } - - override def afterEach() { - rmClient.stop() - } - - class MockSplitInfo(host: String) extends SplitInfo(null, host, null, 1, null) { - override def equals(other: Any): Boolean = false - } - - def createAllocator(maxExecutors: Int = 5): YarnAllocator = { - val args = Array( - "--executor-cores", "5", - "--executor-memory", "2048", - "--jar", "somejar.jar", - "--class", "SomeClass") - val sparkConfClone = sparkConf.clone() - sparkConfClone.set("spark.executor.instances", maxExecutors.toString) - new YarnAllocator( - "not used", - mock(classOf[RpcEndpointRef]), - conf, - sparkConfClone, - rmClient, - appAttemptId, - new ApplicationMasterArguments(args), - new SecurityManager(sparkConf)) - } - - def createContainer(host: String): Container = { - val containerId = ContainerId.newInstance(appAttemptId, containerNum) - containerNum += 1 - val nodeId = NodeId.newInstance(host, 1000) - Container.newInstance(containerId, nodeId, "", containerResource, RM_REQUEST_PRIORITY, null) - } - - test("single container allocated") { - // request a single container and receive it - val handler = createAllocator(1) - handler.updateResourceRequests() - handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (1) - - val container = createContainer("host1") - handler.handleAllocatedContainers(Array(container)) - - handler.getNumExecutorsRunning should be (1) - handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") - handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId) - - val size = rmClient.getMatchingRequests(container.getPriority, "host1", containerResource).size - size should be (0) - } - - test("some containers allocated") { - // request a few containers and receive some of them - val handler = createAllocator(4) - handler.updateResourceRequests() - handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (4) - - val container1 = createContainer("host1") - val container2 = createContainer("host1") - val container3 = createContainer("host2") - handler.handleAllocatedContainers(Array(container1, container2, container3)) - - handler.getNumExecutorsRunning should be (3) - handler.allocatedContainerToHostMap.get(container1.getId).get should be ("host1") - handler.allocatedContainerToHostMap.get(container2.getId).get should be ("host1") - handler.allocatedContainerToHostMap.get(container3.getId).get should be ("host2") - handler.allocatedHostToContainersMap.get("host1").get should contain (container1.getId) - handler.allocatedHostToContainersMap.get("host1").get should contain (container2.getId) - handler.allocatedHostToContainersMap.get("host2").get should contain (container3.getId) - } - - test("receive more containers than requested") { - val handler = createAllocator(2) - handler.updateResourceRequests() - handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (2) - - val container1 = createContainer("host1") - val container2 = createContainer("host2") - val container3 = createContainer("host4") - handler.handleAllocatedContainers(Array(container1, container2, container3)) - - handler.getNumExecutorsRunning should be (2) - handler.allocatedContainerToHostMap.get(container1.getId).get should be ("host1") - handler.allocatedContainerToHostMap.get(container2.getId).get should be ("host2") - handler.allocatedContainerToHostMap.contains(container3.getId) should be (false) - handler.allocatedHostToContainersMap.get("host1").get should contain (container1.getId) - handler.allocatedHostToContainersMap.get("host2").get should contain (container2.getId) - handler.allocatedHostToContainersMap.contains("host4") should be (false) - } - - test("decrease total requested executors") { - val handler = createAllocator(4) - handler.updateResourceRequests() - handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (4) - - handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty) - handler.updateResourceRequests() - handler.getPendingAllocate.size should be (3) - - val container = createContainer("host1") - handler.handleAllocatedContainers(Array(container)) - - handler.getNumExecutorsRunning should be (1) - handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") - handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId) - - handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map.empty) - handler.updateResourceRequests() - handler.getPendingAllocate.size should be (1) - } - - test("decrease total requested executors to less than currently running") { - val handler = createAllocator(4) - handler.updateResourceRequests() - handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (4) - - handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty) - handler.updateResourceRequests() - handler.getPendingAllocate.size should be (3) - - val container1 = createContainer("host1") - val container2 = createContainer("host2") - handler.handleAllocatedContainers(Array(container1, container2)) - - handler.getNumExecutorsRunning should be (2) - - handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty) - handler.updateResourceRequests() - handler.getPendingAllocate.size should be (0) - handler.getNumExecutorsRunning should be (2) - } - - test("kill executors") { - val handler = createAllocator(4) - handler.updateResourceRequests() - handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (4) - - val container1 = createContainer("host1") - val container2 = createContainer("host2") - handler.handleAllocatedContainers(Array(container1, container2)) - - handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty) - handler.executorIdToContainer.keys.foreach { id => handler.killExecutor(id ) } - - val statuses = Seq(container1, container2).map { c => - ContainerStatus.newInstance(c.getId(), ContainerState.COMPLETE, "Finished", 0) - } - handler.updateResourceRequests() - handler.processCompletedContainers(statuses.toSeq) - handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (1) - } - - test("lost executor removed from backend") { - val handler = createAllocator(4) - handler.updateResourceRequests() - handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (4) - - val container1 = createContainer("host1") - val container2 = createContainer("host2") - handler.handleAllocatedContainers(Array(container1, container2)) - - handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map()) - - val statuses = Seq(container1, container2).map { c => - ContainerStatus.newInstance(c.getId(), ContainerState.COMPLETE, "Failed", -1) - } - handler.updateResourceRequests() - handler.processCompletedContainers(statuses.toSeq) - handler.updateResourceRequests() - handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (2) - handler.getNumExecutorsFailed should be (2) - handler.getNumUnexpectedContainerRelease should be (2) - } - - test("memory exceeded diagnostic regexes") { - val diagnostics = - "Container [pid=12465,containerID=container_1412887393566_0003_01_000002] is running " + - "beyond physical memory limits. Current usage: 2.1 MB of 2 GB physical memory used; " + - "5.8 GB of 4.2 GB virtual memory used. Killing container." - val vmemMsg = memLimitExceededLogMessage(diagnostics, VMEM_EXCEEDED_PATTERN) - val pmemMsg = memLimitExceededLogMessage(diagnostics, PMEM_EXCEEDED_PATTERN) - assert(vmemMsg.contains("5.8 GB of 4.2 GB virtual memory used.")) - assert(pmemMsg.contains("2.1 MB of 2 GB physical memory used.")) - } -} diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala deleted file mode 100644 index 6db012a77a93..000000000000 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ /dev/null @@ -1,347 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.io.File -import java.net.URL -import java.util.{HashMap => JHashMap, Properties} - -import scala.collection.mutable -import scala.concurrent.duration._ -import scala.language.postfixOps - -import com.google.common.base.Charsets.UTF_8 -import com.google.common.io.{ByteStreams, Files} -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.scalatest.Matchers -import org.scalatest.concurrent.Eventually._ - -import org.apache.spark._ -import org.apache.spark.launcher._ -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, - SparkListenerExecutorAdded} -import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.tags.ExtendedYarnTest -import org.apache.spark.util.Utils - -/** - * Integration tests for YARN; these tests use a mini Yarn cluster to run Spark-on-YARN - * applications, and require the Spark assembly to be built before they can be successfully - * run. - */ -@ExtendedYarnTest -class YarnClusterSuite extends BaseYarnClusterSuite { - - override def newYarnConfig(): YarnConfiguration = new YarnConfiguration() - - private val TEST_PYFILE = """ - |import mod1, mod2 - |import sys - |from operator import add - | - |from pyspark import SparkConf , SparkContext - |if __name__ == "__main__": - | if len(sys.argv) != 2: - | print >> sys.stderr, "Usage: test.py [result file]" - | exit(-1) - | sc = SparkContext(conf=SparkConf()) - | status = open(sys.argv[1],'w') - | result = "failure" - | rdd = sc.parallelize(range(10)).map(lambda x: x * mod1.func() * mod2.func()) - | cnt = rdd.count() - | if cnt == 10: - | result = "success" - | status.write(result) - | status.close() - | sc.stop() - """.stripMargin - - private val TEST_PYMODULE = """ - |def func(): - | return 42 - """.stripMargin - - test("run Spark in yarn-client mode") { - testBasicYarnApp(true) - } - - test("run Spark in yarn-cluster mode") { - testBasicYarnApp(false) - } - - test("run Spark in yarn-cluster mode unsuccessfully") { - // Don't provide arguments so the driver will fail. - val finalState = runSpark(false, mainClassName(YarnClusterDriver.getClass)) - finalState should be (SparkAppHandle.State.FAILED) - } - - test("run Python application in yarn-client mode") { - testPySpark(true) - } - - test("run Python application in yarn-cluster mode") { - testPySpark(false) - } - - test("user class path first in client mode") { - testUseClassPathFirst(true) - } - - test("user class path first in cluster mode") { - testUseClassPathFirst(false) - } - - test("monitor app using launcher library") { - val env = new JHashMap[String, String]() - env.put("YARN_CONF_DIR", hadoopConfDir.getAbsolutePath()) - - val propsFile = createConfFile() - val handle = new SparkLauncher(env) - .setSparkHome(sys.props("spark.test.home")) - .setConf("spark.ui.enabled", "false") - .setPropertiesFile(propsFile) - .setMaster("yarn-client") - .setAppResource("spark-internal") - .setMainClass(mainClassName(YarnLauncherTestApp.getClass)) - .startApplication() - - try { - eventually(timeout(30 seconds), interval(100 millis)) { - handle.getState() should be (SparkAppHandle.State.RUNNING) - } - - handle.getAppId() should not be (null) - handle.getAppId() should startWith ("application_") - handle.stop() - - eventually(timeout(30 seconds), interval(100 millis)) { - handle.getState() should be (SparkAppHandle.State.KILLED) - } - } finally { - handle.kill() - } - } - - private def testBasicYarnApp(clientMode: Boolean): Unit = { - val result = File.createTempFile("result", null, tempDir) - val finalState = runSpark(clientMode, mainClassName(YarnClusterDriver.getClass), - appArgs = Seq(result.getAbsolutePath())) - checkResult(finalState, result) - } - - private def testPySpark(clientMode: Boolean): Unit = { - val primaryPyFile = new File(tempDir, "test.py") - Files.write(TEST_PYFILE, primaryPyFile, UTF_8) - - // When running tests, let's not assume the user has built the assembly module, which also - // creates the pyspark archive. Instead, let's use PYSPARK_ARCHIVES_PATH to point at the - // needed locations. - val sparkHome = sys.props("spark.test.home"); - val pythonPath = Seq( - s"$sparkHome/python/lib/py4j-0.9-src.zip", - s"$sparkHome/python") - val extraEnv = Map( - "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator), - "PYTHONPATH" -> pythonPath.mkString(File.pathSeparator)) - - val moduleDir = - if (clientMode) { - // In client-mode, .py files added with --py-files are not visible in the driver. - // This is something that the launcher library would have to handle. - tempDir - } else { - val subdir = new File(tempDir, "pyModules") - subdir.mkdir() - subdir - } - val pyModule = new File(moduleDir, "mod1.py") - Files.write(TEST_PYMODULE, pyModule, UTF_8) - - val mod2Archive = TestUtils.createJarWithFiles(Map("mod2.py" -> TEST_PYMODULE), moduleDir) - val pyFiles = Seq(pyModule.getAbsolutePath(), mod2Archive.getPath()).mkString(",") - val result = File.createTempFile("result", null, tempDir) - - val finalState = runSpark(clientMode, primaryPyFile.getAbsolutePath(), - sparkArgs = Seq("--py-files" -> pyFiles), - appArgs = Seq(result.getAbsolutePath()), - extraEnv = extraEnv) - checkResult(finalState, result) - } - - private def testUseClassPathFirst(clientMode: Boolean): Unit = { - // Create a jar file that contains a different version of "test.resource". - val originalJar = TestUtils.createJarWithFiles(Map("test.resource" -> "ORIGINAL"), tempDir) - val userJar = TestUtils.createJarWithFiles(Map("test.resource" -> "OVERRIDDEN"), tempDir) - val driverResult = File.createTempFile("driver", null, tempDir) - val executorResult = File.createTempFile("executor", null, tempDir) - val finalState = runSpark(clientMode, mainClassName(YarnClasspathTest.getClass), - appArgs = Seq(driverResult.getAbsolutePath(), executorResult.getAbsolutePath()), - extraClassPath = Seq(originalJar.getPath()), - extraJars = Seq("local:" + userJar.getPath()), - extraConf = Map( - "spark.driver.userClassPathFirst" -> "true", - "spark.executor.userClassPathFirst" -> "true")) - checkResult(finalState, driverResult, "OVERRIDDEN") - checkResult(finalState, executorResult, "OVERRIDDEN") - } - -} - -private[spark] class SaveExecutorInfo extends SparkListener { - val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() - var driverLogs: Option[collection.Map[String, String]] = None - - override def onExecutorAdded(executor: SparkListenerExecutorAdded) { - addedExecutorInfos(executor.executorId) = executor.executorInfo - } - - override def onApplicationStart(appStart: SparkListenerApplicationStart): Unit = { - driverLogs = appStart.driverLogs - } -} - -private object YarnClusterDriver extends Logging with Matchers { - - val WAIT_TIMEOUT_MILLIS = 10000 - - def main(args: Array[String]): Unit = { - if (args.length != 1) { - // scalastyle:off println - System.err.println( - s""" - |Invalid command line: ${args.mkString(" ")} - | - |Usage: YarnClusterDriver [result file] - """.stripMargin) - // scalastyle:on println - System.exit(1) - } - - val sc = new SparkContext(new SparkConf() - .set("spark.extraListeners", classOf[SaveExecutorInfo].getName) - .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns")) - val conf = sc.getConf - val status = new File(args(0)) - var result = "failure" - try { - val data = sc.parallelize(1 to 4, 4).collect().toSet - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - data should be (Set(1, 2, 3, 4)) - result = "success" - } finally { - Files.write(result, status, UTF_8) - sc.stop() - } - - // verify log urls are present - val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] - assert(listeners.size === 1) - val listener = listeners(0) - val executorInfos = listener.addedExecutorInfos.values - assert(executorInfos.nonEmpty) - executorInfos.foreach { info => - assert(info.logUrlMap.nonEmpty) - } - - // If we are running in yarn-cluster mode, verify that driver logs links and present and are - // in the expected format. - if (conf.get("spark.master") == "yarn-cluster") { - assert(listener.driverLogs.nonEmpty) - val driverLogs = listener.driverLogs.get - assert(driverLogs.size === 2) - assert(driverLogs.contains("stderr")) - assert(driverLogs.contains("stdout")) - val urlStr = driverLogs("stderr") - // Ensure that this is a valid URL, else this will throw an exception - new URL(urlStr) - val containerId = YarnSparkHadoopUtil.get.getContainerId - val user = Utils.getCurrentUserName() - assert(urlStr.endsWith(s"/node/containerlogs/$containerId/$user/stderr?start=-4096")) - } - } - -} - -private object YarnClasspathTest extends Logging { - - var exitCode = 0 - - def error(m: String, ex: Throwable = null): Unit = { - logError(m, ex) - // scalastyle:off println - System.out.println(m) - if (ex != null) { - ex.printStackTrace(System.out) - } - // scalastyle:on println - } - - def main(args: Array[String]): Unit = { - if (args.length != 2) { - error( - s""" - |Invalid command line: ${args.mkString(" ")} - | - |Usage: YarnClasspathTest [driver result file] [executor result file] - """.stripMargin) - // scalastyle:on println - } - - readResource(args(0)) - val sc = new SparkContext(new SparkConf()) - try { - sc.parallelize(Seq(1)).foreach { x => readResource(args(1)) } - } finally { - sc.stop() - } - System.exit(exitCode) - } - - private def readResource(resultPath: String): Unit = { - var result = "failure" - try { - val ccl = Thread.currentThread().getContextClassLoader() - val resource = ccl.getResourceAsStream("test.resource") - val bytes = ByteStreams.toByteArray(resource) - result = new String(bytes, 0, bytes.length, UTF_8) - } catch { - case t: Throwable => - error(s"loading test.resource to $resultPath", t) - // set the exit code if not yet set - exitCode = 2 - } finally { - Files.write(result, new File(resultPath), UTF_8) - } - } - -} - -private object YarnLauncherTestApp { - - def main(args: Array[String]): Unit = { - // Do not stop the application; the test will stop it using the launcher lib. Just run a task - // that will prevent the process from exiting. - val sc = new SparkContext(new SparkConf()) - sc.parallelize(Seq(1)).foreach { i => - this.synchronized { - wait() - } - } - } - -} diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala deleted file mode 100644 index c17e8695c24f..000000000000 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ /dev/null @@ -1,111 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ - -package org.apache.spark.deploy.yarn - -import java.io.File - -import com.google.common.base.Charsets.UTF_8 -import com.google.common.io.Files -import org.apache.commons.io.FileUtils -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.scalatest.Matchers - -import org.apache.spark._ -import org.apache.spark.network.shuffle.ShuffleTestAccessor -import org.apache.spark.network.yarn.{YarnShuffleService, YarnTestAccessor} -import org.apache.spark.tags.ExtendedYarnTest - -/** - * Integration test for the external shuffle service with a yarn mini-cluster - */ -@ExtendedYarnTest -class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite { - - override def newYarnConfig(): YarnConfiguration = { - val yarnConfig = new YarnConfiguration() - yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle") - yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), - classOf[YarnShuffleService].getCanonicalName) - yarnConfig.set("spark.shuffle.service.port", "0") - yarnConfig - } - - test("external shuffle service") { - val shuffleServicePort = YarnTestAccessor.getShuffleServicePort - val shuffleService = YarnTestAccessor.getShuffleServiceInstance - - val registeredExecFile = YarnTestAccessor.getRegisteredExecutorFile(shuffleService) - - logInfo("Shuffle service port = " + shuffleServicePort) - val result = File.createTempFile("result", null, tempDir) - val finalState = runSpark( - false, - mainClassName(YarnExternalShuffleDriver.getClass), - appArgs = Seq(result.getAbsolutePath(), registeredExecFile.getAbsolutePath), - extraConf = Map( - "spark.shuffle.service.enabled" -> "true", - "spark.shuffle.service.port" -> shuffleServicePort.toString - ) - ) - checkResult(finalState, result) - assert(YarnTestAccessor.getRegisteredExecutorFile(shuffleService).exists()) - } -} - -private object YarnExternalShuffleDriver extends Logging with Matchers { - - val WAIT_TIMEOUT_MILLIS = 10000 - - def main(args: Array[String]): Unit = { - if (args.length != 2) { - // scalastyle:off println - System.err.println( - s""" - |Invalid command line: ${args.mkString(" ")} - | - |Usage: ExternalShuffleDriver [result file] [registed exec file] - """.stripMargin) - // scalastyle:on println - System.exit(1) - } - - val sc = new SparkContext(new SparkConf() - .setAppName("External Shuffle Test")) - val conf = sc.getConf - val status = new File(args(0)) - val registeredExecFile = new File(args(1)) - logInfo("shuffle service executor file = " + registeredExecFile) - var result = "failure" - val execStateCopy = new File(registeredExecFile.getAbsolutePath + "_dup") - try { - val data = sc.parallelize(0 until 100, 10).map { x => (x % 10) -> x }.reduceByKey{ _ + _ }. - collect().toSet - sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - data should be ((0 until 10).map{x => x -> (x * 10 + 450)}.toSet) - result = "success" - // only one process can open a leveldb file at a time, so we copy the files - FileUtils.copyDirectory(registeredExecFile, execStateCopy) - assert(!ShuffleTestAccessor.reloadRegisteredExecutors(execStateCopy).isEmpty) - } finally { - sc.stop() - FileUtils.deleteDirectory(execStateCopy) - Files.write(result, status, UTF_8) - } - } - -} diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala deleted file mode 100644 index a70e66d39a64..000000000000 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ /dev/null @@ -1,307 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.io.{File, IOException} -import java.lang.reflect.InvocationTargetException - -import com.google.common.io.{ByteStreams, Files} -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.ql.metadata.HiveException -import org.apache.hadoop.io.Text -import org.apache.hadoop.yarn.api.ApplicationConstants -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.scalatest.Matchers - -import org.apache.hadoop.yarn.api.records.ApplicationAccessType - -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException, SparkFunSuite} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.util.Utils - - -class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging { - - val hasBash = - try { - val exitCode = Runtime.getRuntime().exec(Array("bash", "--version")).waitFor() - exitCode == 0 - } catch { - case e: IOException => - false - } - - if (!hasBash) { - logWarning("Cannot execute bash, skipping bash tests.") - } - - def bashTest(name: String)(fn: => Unit): Unit = - if (hasBash) test(name)(fn) else ignore(name)(fn) - - bashTest("shell script escaping") { - val scriptFile = File.createTempFile("script.", ".sh", Utils.createTempDir()) - val args = Array("arg1", "${arg.2}", "\"arg3\"", "'arg4'", "$arg5", "\\arg6") - try { - val argLine = args.map(a => YarnSparkHadoopUtil.escapeForShell(a)).mkString(" ") - Files.write(("bash -c \"echo " + argLine + "\"").getBytes(), scriptFile) - scriptFile.setExecutable(true) - - val proc = Runtime.getRuntime().exec(Array(scriptFile.getAbsolutePath())) - val out = new String(ByteStreams.toByteArray(proc.getInputStream())).trim() - val err = new String(ByteStreams.toByteArray(proc.getErrorStream())) - val exitCode = proc.waitFor() - exitCode should be (0) - out should be (args.mkString(" ")) - } finally { - scriptFile.delete() - } - } - - test("Yarn configuration override") { - val key = "yarn.nodemanager.hostname" - val default = new YarnConfiguration() - - val sparkConf = new SparkConf() - .set("spark.hadoop." + key, "someHostName") - val yarnConf = new YarnSparkHadoopUtil().newConfiguration(sparkConf) - - yarnConf.getClass() should be (classOf[YarnConfiguration]) - yarnConf.get(key) should not be default.get(key) - } - - - test("test getApplicationAclsForYarn acls on") { - - // spark acls on, just pick up default user - val sparkConf = new SparkConf() - sparkConf.set("spark.acls.enable", "true") - - val securityMgr = new SecurityManager(sparkConf) - val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr) - - val viewAcls = acls.get(ApplicationAccessType.VIEW_APP) - val modifyAcls = acls.get(ApplicationAccessType.MODIFY_APP) - - viewAcls match { - case Some(vacls) => { - val aclSet = vacls.split(',').map(_.trim).toSet - assert(aclSet.contains(System.getProperty("user.name", "invalid"))) - } - case None => { - fail() - } - } - modifyAcls match { - case Some(macls) => { - val aclSet = macls.split(',').map(_.trim).toSet - assert(aclSet.contains(System.getProperty("user.name", "invalid"))) - } - case None => { - fail() - } - } - } - - test("test getApplicationAclsForYarn acls on and specify users") { - - // default spark acls are on and specify acls - val sparkConf = new SparkConf() - sparkConf.set("spark.acls.enable", "true") - sparkConf.set("spark.ui.view.acls", "user1,user2") - sparkConf.set("spark.modify.acls", "user3,user4") - - val securityMgr = new SecurityManager(sparkConf) - val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr) - - val viewAcls = acls.get(ApplicationAccessType.VIEW_APP) - val modifyAcls = acls.get(ApplicationAccessType.MODIFY_APP) - - viewAcls match { - case Some(vacls) => { - val aclSet = vacls.split(',').map(_.trim).toSet - assert(aclSet.contains("user1")) - assert(aclSet.contains("user2")) - assert(aclSet.contains(System.getProperty("user.name", "invalid"))) - } - case None => { - fail() - } - } - modifyAcls match { - case Some(macls) => { - val aclSet = macls.split(',').map(_.trim).toSet - assert(aclSet.contains("user3")) - assert(aclSet.contains("user4")) - assert(aclSet.contains(System.getProperty("user.name", "invalid"))) - } - case None => { - fail() - } - } - - } - - test("test expandEnvironment result") { - val target = Environment.PWD - if (classOf[Environment].getMethods().exists(_.getName == "$$")) { - YarnSparkHadoopUtil.expandEnvironment(target) should be ("{{" + target + "}}") - } else if (Utils.isWindows) { - YarnSparkHadoopUtil.expandEnvironment(target) should be ("%" + target + "%") - } else { - YarnSparkHadoopUtil.expandEnvironment(target) should be ("$" + target) - } - - } - - test("test getClassPathSeparator result") { - if (classOf[ApplicationConstants].getFields().exists(_.getName == "CLASS_PATH_SEPARATOR")) { - YarnSparkHadoopUtil.getClassPathSeparator() should be ("") - } else if (Utils.isWindows) { - YarnSparkHadoopUtil.getClassPathSeparator() should be (";") - } else { - YarnSparkHadoopUtil.getClassPathSeparator() should be (":") - } - } - - test("check access nns empty") { - val sparkConf = new SparkConf() - val util = new YarnSparkHadoopUtil - sparkConf.set("spark.yarn.access.namenodes", "") - val nns = util.getNameNodesToAccess(sparkConf) - nns should be(Set()) - } - - test("check access nns unset") { - val sparkConf = new SparkConf() - val util = new YarnSparkHadoopUtil - val nns = util.getNameNodesToAccess(sparkConf) - nns should be(Set()) - } - - test("check access nns") { - val sparkConf = new SparkConf() - sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032") - val util = new YarnSparkHadoopUtil - val nns = util.getNameNodesToAccess(sparkConf) - nns should be(Set(new Path("hdfs://nn1:8032"))) - } - - test("check access nns space") { - val sparkConf = new SparkConf() - sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032, ") - val util = new YarnSparkHadoopUtil - val nns = util.getNameNodesToAccess(sparkConf) - nns should be(Set(new Path("hdfs://nn1:8032"))) - } - - test("check access two nns") { - val sparkConf = new SparkConf() - sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032,hdfs://nn2:8032") - val util = new YarnSparkHadoopUtil - val nns = util.getNameNodesToAccess(sparkConf) - nns should be(Set(new Path("hdfs://nn1:8032"), new Path("hdfs://nn2:8032"))) - } - - test("check token renewer") { - val hadoopConf = new Configuration() - hadoopConf.set("yarn.resourcemanager.address", "myrm:8033") - hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:8032@SPARKTEST.COM") - val util = new YarnSparkHadoopUtil - val renewer = util.getTokenRenewer(hadoopConf) - renewer should be ("yarn/myrm:8032@SPARKTEST.COM") - } - - test("check token renewer default") { - val hadoopConf = new Configuration() - val util = new YarnSparkHadoopUtil - val caught = - intercept[SparkException] { - util.getTokenRenewer(hadoopConf) - } - assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer") - } - - test("check different hadoop utils based on env variable") { - try { - System.setProperty("SPARK_YARN_MODE", "true") - assert(SparkHadoopUtil.get.getClass === classOf[YarnSparkHadoopUtil]) - System.setProperty("SPARK_YARN_MODE", "false") - assert(SparkHadoopUtil.get.getClass === classOf[SparkHadoopUtil]) - } finally { - System.clearProperty("SPARK_YARN_MODE") - } - } - - test("Obtain tokens For HiveMetastore") { - val hadoopConf = new Configuration() - hadoopConf.set("hive.metastore.kerberos.principal", "bob") - // thrift picks up on port 0 and bails out, without trying to talk to endpoint - hadoopConf.set("hive.metastore.uris", "http://localhost:0") - val util = new YarnSparkHadoopUtil - assertNestedHiveException(intercept[InvocationTargetException] { - util.obtainTokenForHiveMetastoreInner(hadoopConf, "alice") - }) - // expect exception trapping code to unwind this hive-side exception - assertNestedHiveException(intercept[InvocationTargetException] { - util.obtainTokenForHiveMetastore(hadoopConf) - }) - } - - private def assertNestedHiveException(e: InvocationTargetException): Throwable = { - val inner = e.getCause - if (inner == null) { - fail("No inner cause", e) - } - if (!inner.isInstanceOf[HiveException]) { - fail("Not a hive exception", inner) - } - inner - } - - // This test needs to live here because it depends on isYarnMode returning true, which can only - // happen in the YARN module. - test("security manager token generation") { - try { - System.setProperty("SPARK_YARN_MODE", "true") - val initial = SparkHadoopUtil.get - .getSecretKeyFromUserCredentials(SecurityManager.SECRET_LOOKUP_KEY) - assert(initial === null || initial.length === 0) - - val conf = new SparkConf() - .set(SecurityManager.SPARK_AUTH_CONF, "true") - .set(SecurityManager.SPARK_AUTH_SECRET_CONF, "unused") - val sm = new SecurityManager(conf) - - val generated = SparkHadoopUtil.get - .getSecretKeyFromUserCredentials(SecurityManager.SECRET_LOOKUP_KEY) - assert(generated != null) - val genString = new Text(generated).toString() - assert(genString != "unused") - assert(sm.getSecretKey() === genString) - } finally { - // removeSecretKey() was only added in Hadoop 2.6, so instead we just set the secret - // to an empty string. - SparkHadoopUtil.get.addSecretKeyToUserCredentials(SecurityManager.SECRET_LOOKUP_KEY, "") - System.clearProperty("SPARK_YARN_MODE") - } - } - -} diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala deleted file mode 100644 index 6aa8c814cd4f..000000000000 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ /dev/null @@ -1,234 +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.network.yarn - -import java.io.{DataOutputStream, File, FileOutputStream} - -import scala.annotation.tailrec - -import org.apache.commons.io.FileUtils -import org.apache.hadoop.yarn.api.records.ApplicationId -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.server.api.{ApplicationInitializationContext, ApplicationTerminationContext} -import org.scalatest.{BeforeAndAfterEach, Matchers} - -import org.apache.spark.SparkFunSuite -import org.apache.spark.network.shuffle.ShuffleTestAccessor -import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo - -class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach { - private[yarn] var yarnConfig: YarnConfiguration = new YarnConfiguration - - override def beforeEach(): Unit = { - yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle") - yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), - classOf[YarnShuffleService].getCanonicalName) - yarnConfig.setInt("spark.shuffle.service.port", 0) - - yarnConfig.get("yarn.nodemanager.local-dirs").split(",").foreach { dir => - val d = new File(dir) - if (d.exists()) { - FileUtils.deleteDirectory(d) - } - FileUtils.forceMkdir(d) - logInfo(s"creating yarn.nodemanager.local-dirs: $d") - } - } - - var s1: YarnShuffleService = null - var s2: YarnShuffleService = null - var s3: YarnShuffleService = null - - override def afterEach(): Unit = { - if (s1 != null) { - s1.stop() - s1 = null - } - if (s2 != null) { - s2.stop() - s2 = null - } - if (s3 != null) { - s3.stop() - s3 = null - } - } - - test("executor state kept across NM restart") { - s1 = new YarnShuffleService - s1.init(yarnConfig) - val app1Id = ApplicationId.newInstance(0, 1) - val app1Data: ApplicationInitializationContext = - new ApplicationInitializationContext("user", app1Id, null) - s1.initializeApplication(app1Data) - val app2Id = ApplicationId.newInstance(0, 2) - val app2Data: ApplicationInitializationContext = - new ApplicationInitializationContext("user", app2Id, null) - s1.initializeApplication(app2Data) - - val execStateFile = s1.registeredExecutorFile - execStateFile should not be (null) - val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") - val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, "hash") - - val blockHandler = s1.blockHandler - val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler) - ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile) - - blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1) - blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2) - ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", blockResolver) should - be (Some(shuffleInfo1)) - ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", blockResolver) should - be (Some(shuffleInfo2)) - - if (!execStateFile.exists()) { - @tailrec def findExistingParent(file: File): File = { - if (file == null) file - else if (file.exists()) file - else findExistingParent(file.getParentFile()) - } - val existingParent = findExistingParent(execStateFile) - assert(false, s"$execStateFile does not exist -- closest existing parent is $existingParent") - } - assert(execStateFile.exists(), s"$execStateFile did not exist") - - // now we pretend the shuffle service goes down, and comes back up - s1.stop() - s2 = new YarnShuffleService - s2.init(yarnConfig) - s2.registeredExecutorFile should be (execStateFile) - - val handler2 = s2.blockHandler - val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2) - - // now we reinitialize only one of the apps, and expect yarn to tell us that app2 was stopped - // during the restart - s2.initializeApplication(app1Data) - s2.stopApplication(new ApplicationTerminationContext(app2Id)) - ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver2) should be (Some(shuffleInfo1)) - ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (None) - - // Act like the NM restarts one more time - s2.stop() - s3 = new YarnShuffleService - s3.init(yarnConfig) - s3.registeredExecutorFile should be (execStateFile) - - val handler3 = s3.blockHandler - val resolver3 = ShuffleTestAccessor.getBlockResolver(handler3) - - // app1 is still running - s3.initializeApplication(app1Data) - ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver3) should be (Some(shuffleInfo1)) - ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (None) - s3.stop() - } - - test("removed applications should not be in registered executor file") { - s1 = new YarnShuffleService - s1.init(yarnConfig) - val app1Id = ApplicationId.newInstance(0, 1) - val app1Data: ApplicationInitializationContext = - new ApplicationInitializationContext("user", app1Id, null) - s1.initializeApplication(app1Data) - val app2Id = ApplicationId.newInstance(0, 2) - val app2Data: ApplicationInitializationContext = - new ApplicationInitializationContext("user", app2Id, null) - s1.initializeApplication(app2Data) - - val execStateFile = s1.registeredExecutorFile - execStateFile should not be (null) - val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") - val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, "hash") - - val blockHandler = s1.blockHandler - val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler) - ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile) - - blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1) - blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2) - - val db = ShuffleTestAccessor.shuffleServiceLevelDB(blockResolver) - ShuffleTestAccessor.reloadRegisteredExecutors(db) should not be empty - - s1.stopApplication(new ApplicationTerminationContext(app1Id)) - ShuffleTestAccessor.reloadRegisteredExecutors(db) should not be empty - s1.stopApplication(new ApplicationTerminationContext(app2Id)) - ShuffleTestAccessor.reloadRegisteredExecutors(db) shouldBe empty - } - - test("shuffle service should be robust to corrupt registered executor file") { - s1 = new YarnShuffleService - s1.init(yarnConfig) - val app1Id = ApplicationId.newInstance(0, 1) - val app1Data: ApplicationInitializationContext = - new ApplicationInitializationContext("user", app1Id, null) - s1.initializeApplication(app1Data) - - val execStateFile = s1.registeredExecutorFile - val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") - - val blockHandler = s1.blockHandler - val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler) - ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile) - - blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1) - - // now we pretend the shuffle service goes down, and comes back up. But we'll also - // make a corrupt registeredExecutor File - s1.stop() - - execStateFile.listFiles().foreach{_.delete()} - - val out = new DataOutputStream(new FileOutputStream(execStateFile + "/CURRENT")) - out.writeInt(42) - out.close() - - s2 = new YarnShuffleService - s2.init(yarnConfig) - s2.registeredExecutorFile should be (execStateFile) - - val handler2 = s2.blockHandler - val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2) - - // we re-initialize app1, but since the file was corrupt there is nothing we can do about it ... - s2.initializeApplication(app1Data) - // however, when we initialize a totally new app2, everything is still happy - val app2Id = ApplicationId.newInstance(0, 2) - val app2Data: ApplicationInitializationContext = - new ApplicationInitializationContext("user", app2Id, null) - s2.initializeApplication(app2Data) - val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, "hash") - resolver2.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2) - ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (Some(shuffleInfo2)) - s2.stop() - - // another stop & restart should be fine though (eg., we recover from previous corruption) - s3 = new YarnShuffleService - s3.init(yarnConfig) - s3.registeredExecutorFile should be (execStateFile) - val handler3 = s3.blockHandler - val resolver3 = ShuffleTestAccessor.getBlockResolver(handler3) - - s3.initializeApplication(app2Data) - ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (Some(shuffleInfo2)) - s3.stop() - - } - -}

    Status
    Location
    Executor ID / Host
    Last Error Time
    Last Error Message
    -
    {receiverName}
    -
    Avg: {receivedRecords.formattedAvg} events/sec
    +
    {receiverName}
    +
    Avg: {receivedRecords.formattedAvg} records/sec
    {receiverActive}
    - {graphUIDataForEventRate.generateTimelineHtml(jsCollector)} + {graphUIDataForRecordRate.generateTimelineHtml(jsCollector)} {graphUIDataForEventRate.generateHistogramHtml(jsCollector)}{graphUIDataForRecordRate.generateHistogramHtml(jsCollector)}
    {failureReasonSummary}{details} + {failureReasonSummary}{details} +